diff --git a/build.gradle.kts b/build.gradle.kts index eb7c10854dc..1cd662d07f6 100644 --- a/build.gradle.kts +++ b/build.gradle.kts @@ -472,6 +472,7 @@ tasks.rat { "integration-test/**/*.sql", "integration-test/**/*.txt", "docs/**/*.md", + "spark-connector/spark-common/src/test/resources/**", "web/.**", "web/next-env.d.ts", "web/dist/**/*", diff --git a/docs/spark-connector/spark-integration-test.md b/docs/spark-connector/spark-integration-test.md new file mode 100644 index 00000000000..35ad27b56ef --- /dev/null +++ b/docs/spark-connector/spark-integration-test.md @@ -0,0 +1,46 @@ +--- +title: "Apache Gravitino Spark connector integration test" +slug: /spark-connector/spark-connector-integration-test +keyword: spark connector integration test +license: "This software is licensed under the Apache License version 2." +--- + +## Overview + +There are two types of integration tests in spark connector, normal integration test like `SparkXXCatalogIT`, and the golden file integration test. + +## Normal integration test + +Normal integration test are mainly used to test the correctness of the metadata, it's enabled in the GitHub CI. You could run tests with specific Spark version like: + +``` +./gradlew :spark-connector:spark3.3:test --tests "org.apache.gravitino.spark.connector.integration.test.hive.SparkHiveCatalogIT33.testCreateHiveFormatPartitionTable" +``` + +## Golden file integration test + +Golden file integration test are mainly to test the correctness of the SQL result with massive data, it's disabled in the GitHub CI, you could run tests with following command: + +``` +./gradlew :spark-connector:spark-3.3:test --tests "org.apache.gravitino.spark.connector.integration.test.sql.SparkSQLRegressionTest33" -PenableSparkSQLITs +``` + +Please change the Spark version number if you want to test other Spark versions. +If you want to change the test behaviour, please modify `spark-connector/spark-common/src/test/resources/spark-test.conf`. + +| Configuration item | Description | Default value | Required | Since Version | +|--------------------------------------------|----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|------------------------------------------------------|----------|---------------| +| `gravitino.spark.test.dir` | The Spark SQL test base dir, include `test-sqls` and `data`. | `spark-connector/spark-common/src/test/resources/` | No | 0.6.0 | +| `gravitino.spark.test.sqls` | Specify the test SQLs, using directory to specify group of SQLs like `test-sqls/hive`, using file path to specify one SQL like `test-sqls/hive/basic.sql`, use `,` to split multi part | run all SQLs | No | 0.6.0 | +| `gravitino.spark.test.generateGoldenFiles` | Whether generate golden files which are used to check the correctness of the SQL result | false | No | 0.6.0 | +| `gravitino.spark.test.metalake` | The metalake name to run the test | `test` | No | 0.6.0 | +| `gravitino.spark.test.setupEnv` | Whether to setup Gravitino and Hive environment | `false` | No | 0.6.0 | +| `gravitino.spark.test.uri` | Gravitino uri address, only available when `gravitino.spark.test.setupEnv` is false | http://127.0.0.1:8090 | No | 0.6.0 | +| `gravitino.spark.test.iceberg.warehouse` | The warehouse location, only available when `gravitino.spark.test.setupEnv` is false | hdfs://127.0.0.1:9000/user/hive/warehouse-spark-test | No | 0.6.0 | + +The test SQL files are located in `spark-connector/spark-common/src/test/resources/` by default. There are three directories: +- `hive`, SQL tests for Hive catalog. +- `lakehouse-iceberg`, SQL tests for Iceberg catalog. +- `tpcds`, SQL tests for `tpcds` in Hive catalog. + +You could create a simple SQL file, like `hive/catalog.sql`, the program will check the output with `hive/catalog.sql.out`. For complex cases like `tpcds`, you could do some prepare work like create table&load data in `prepare.sql`. diff --git a/spark-connector/spark-common/build.gradle.kts b/spark-connector/spark-common/build.gradle.kts index 82594c257f9..f4dacc569eb 100644 --- a/spark-connector/spark-common/build.gradle.kts +++ b/spark-connector/spark-common/build.gradle.kts @@ -155,6 +155,14 @@ tasks.clean { delete("spark-warehouse") } +sourceSets { + named("test") { + resources { + exclude("**/*") + } + } +} + val testJar by tasks.registering(Jar::class) { archiveClassifier.set("tests") from(sourceSets["test"].output) diff --git a/spark-connector/spark-common/src/test/java/org/apache/gravitino/spark/connector/integration/test/sql/CatalogType.java b/spark-connector/spark-common/src/test/java/org/apache/gravitino/spark/connector/integration/test/sql/CatalogType.java new file mode 100644 index 00000000000..8665e90be13 --- /dev/null +++ b/spark-connector/spark-common/src/test/java/org/apache/gravitino/spark/connector/integration/test/sql/CatalogType.java @@ -0,0 +1,46 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.gravitino.spark.connector.integration.test.sql; + +import java.util.Locale; + +public enum CatalogType { + HIVE, + ICEBERG, + UNKNOWN; + + public static CatalogType fromString(String str) { + if (str == null) { + return UNKNOWN; + } + switch (str.toLowerCase(Locale.ROOT)) { + case "hive": + return HIVE; + case "lakehouse-iceberg": + return ICEBERG; + default: + return UNKNOWN; + } + } + + // The first non-unknown CatalogType from parent to child determines the catalog type. + public static CatalogType merge(CatalogType parentCatalogType, CatalogType childCatalogType) { + return parentCatalogType.equals(UNKNOWN) ? childCatalogType : parentCatalogType; + } +} diff --git a/spark-connector/spark-common/src/test/java/org/apache/gravitino/spark/connector/integration/test/sql/QueryOutput.java b/spark-connector/spark-common/src/test/java/org/apache/gravitino/spark/connector/integration/test/sql/QueryOutput.java new file mode 100644 index 00000000000..e25f165c096 --- /dev/null +++ b/spark-connector/spark-common/src/test/java/org/apache/gravitino/spark/connector/integration/test/sql/QueryOutput.java @@ -0,0 +1,47 @@ +/* + * 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.gravitino.spark.connector.integration.test.sql; + +import lombok.Getter; + +/** The SQL execution output, include schemas and output */ +@Getter +public final class QueryOutput { + private final String sql; + private final String schema; + private final String output; + + public QueryOutput(String sql, String schema, String output) { + this.sql = sql; + this.schema = schema; + this.output = output; + } + + @Override + public String toString() { + return "-- !query\n" + + sql + + "\n" + + "-- !query schema\n" + + schema + + "\n" + + "-- !query output\n" + + output; + } +} diff --git a/spark-connector/spark-common/src/test/java/org/apache/gravitino/spark/connector/integration/test/sql/SQLQueryTestHelper.java b/spark-connector/spark-common/src/test/java/org/apache/gravitino/spark/connector/integration/test/sql/SQLQueryTestHelper.java new file mode 100644 index 00000000000..d86cc686c78 --- /dev/null +++ b/spark-connector/spark-common/src/test/java/org/apache/gravitino/spark/connector/integration/test/sql/SQLQueryTestHelper.java @@ -0,0 +1,87 @@ +/* + * 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.gravitino.spark.connector.integration.test.sql; + +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.function.Supplier; +import java.util.stream.Collectors; +import org.apache.commons.lang3.tuple.Pair; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.execution.HiveResult; +import org.apache.spark.sql.execution.SQLExecution; +import org.apache.spark.sql.types.StructType; +import scala.Option; +import scala.collection.JavaConverters; + +public class SQLQueryTestHelper { + private static final String notIncludedMsg = "[not included in comparison]"; + private static final String clsName = SQLQueryTestHelper.class.getCanonicalName(); + private static final String emptySchema = new StructType().catalogString(); + + private static String replaceNotIncludedMsg(String line) { + line = + line.replaceAll("#\\d+", "#x") + .replaceAll("plan_id=\\d+", "plan_id=x") + .replaceAll( + "Location.*" + clsName + "/", "Location " + notIncludedMsg + "/{warehouse_dir}/") + .replaceAll("file:[^\\s,]*" + clsName, "file:" + notIncludedMsg + "/{warehouse_dir}") + .replaceAll("Created By.*", "Created By " + notIncludedMsg) + .replaceAll("Created Time.*", "Created Time " + notIncludedMsg) + .replaceAll("Last Access.*", "Last Access " + notIncludedMsg) + .replaceAll("Partition Statistics\t\\d+", "Partition Statistics\t" + notIncludedMsg) + .replaceAll("\\s+$", "") + .replaceAll("\\*\\(\\d+\\) ", "*"); + return line; + } + + public static Pair> getNormalizedResult(SparkSession session, String sql) { + Dataset df = session.sql(sql); + String schema = df.schema().catalogString(); + List answer = + SQLExecution.withNewExecutionId( + df.queryExecution(), + Option.apply(""), + () -> + JavaConverters.seqAsJavaList( + HiveResult.hiveResultString(df.queryExecution().executedPlan())) + .stream() + .map(s -> replaceNotIncludedMsg(s)) + .filter(s -> !s.isEmpty()) + .collect(Collectors.toList())); + + Collections.sort(answer); + + return Pair.of(schema, answer); + } + + // Different Spark version may produce different exceptions, so here just produce + // [SPARK_EXCEPTION] + public static Pair> handleExceptions( + Supplier>> result) { + try { + return result.get(); + } catch (Throwable e) { + return Pair.of(emptySchema, Arrays.asList("[SPARK_EXCEPTION]")); + } + } +} diff --git a/spark-connector/spark-common/src/test/java/org/apache/gravitino/spark/connector/integration/test/sql/SparkQueryRunner.java b/spark-connector/spark-common/src/test/java/org/apache/gravitino/spark/connector/integration/test/sql/SparkQueryRunner.java new file mode 100644 index 00000000000..982035e8b62 --- /dev/null +++ b/spark-connector/spark-common/src/test/java/org/apache/gravitino/spark/connector/integration/test/sql/SparkQueryRunner.java @@ -0,0 +1,344 @@ +/* + * 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.gravitino.spark.connector.integration.test.sql; + +import com.google.common.collect.Maps; +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import org.apache.commons.lang3.tuple.Pair; +import org.apache.gravitino.Catalog; +import org.apache.gravitino.client.GravitinoAdminClient; +import org.apache.gravitino.client.GravitinoMetalake; +import org.apache.gravitino.integration.test.container.ContainerSuite; +import org.apache.gravitino.integration.test.container.HiveContainer; +import org.apache.gravitino.integration.test.util.AbstractIT; +import org.apache.gravitino.spark.connector.GravitinoSparkConfig; +import org.apache.gravitino.spark.connector.iceberg.IcebergPropertiesConstants; +import org.apache.gravitino.spark.connector.plugin.GravitinoSparkPlugin; +import org.apache.spark.sql.SparkSession; +import org.junit.jupiter.api.Assertions; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.testcontainers.shaded.org.apache.commons.io.FileUtils; + +/** Run and check the correctness of the SparkSQLs */ +public class SparkQueryRunner { + private static final Logger LOG = LoggerFactory.getLogger(SparkSQLRegressionTest.class); + private static final String HIVE_CATALOG_NAME = "hive"; + private static final String ICEBERG_CATALOG_NAME = "iceberg"; + + private SparkSession sparkSession; + private String gravitinoUri; + private String metalakeName; + private String warehouse; + private boolean regenerateGoldenFiles; + // catalogType -> catalogName + private Map catalogs = new HashMap<>(); + private boolean isGravitinoEnvSetup; + private String dataDir; + + private static final ContainerSuite containerSuite = ContainerSuite.getInstance(); + + public SparkQueryRunner(SparkTestConfig sparkTestConfig) { + this.regenerateGoldenFiles = sparkTestConfig.generateGoldenFiles(); + this.isGravitinoEnvSetup = sparkTestConfig.isGravitinoEnvSetUp(); + String baseDir = sparkTestConfig.getBaseDir(); + // translate to file:///xx/xx + this.dataDir = Paths.get(baseDir, "data").toUri().toString(); + this.metalakeName = sparkTestConfig.getMetalakeName(); + if (isGravitinoEnvSetup) { + try { + setupGravitinoEnv(); + } catch (Exception e) { + throw new RuntimeException(e); + } + } else { + this.gravitinoUri = sparkTestConfig.getGravitinoUri(); + this.warehouse = sparkTestConfig.getWarehouseLocation(); + } + initSparkEnv(); + + catalogs.put(CatalogType.HIVE, HIVE_CATALOG_NAME); + catalogs.put(CatalogType.ICEBERG, ICEBERG_CATALOG_NAME); + catalogs.put(CatalogType.UNKNOWN, HIVE_CATALOG_NAME); + } + + public void runQuery(TestCaseGroup sqlTestCaseGroup) throws IOException { + useCatalog(sqlTestCaseGroup.getCatalogType()); + try { + if (sqlTestCaseGroup.prepareFile != null) { + runQuery(sqlTestCaseGroup.prepareFile); + } + sqlTestCaseGroup.testCases.forEach( + testCase -> { + try { + runTestCase(testCase); + } catch (IOException e) { + throw new RuntimeException(e); + } + }); + } finally { + if (sqlTestCaseGroup.cleanupFile != null) { + runQuery(sqlTestCaseGroup.cleanupFile); + } + } + } + + public void close() throws Exception { + if (sparkSession != null) { + sparkSession.close(); + sparkSession = null; + } + if (isGravitinoEnvSetup) { + closeGravitinoEnv(); + } + } + + private void setupGravitinoEnv() throws Exception { + // Start Hive and Hdfs + containerSuite.startHiveContainer(); + String hiveMetastoreUri = + String.format( + "thrift://%s:%d", + containerSuite.getHiveContainer().getContainerIpAddress(), + HiveContainer.HIVE_METASTORE_PORT); + this.warehouse = + String.format( + "hdfs://%s:%d/user/hive/warehouse", + containerSuite.getHiveContainer().getContainerIpAddress(), + HiveContainer.HDFS_DEFAULTFS_PORT); + + // Start Gravitino server + AbstractIT.startIntegrationTest(); + int gravitinoPort = AbstractIT.getGravitinoServerPort(); + this.gravitinoUri = String.format("http://127.0.0.1:%d", gravitinoPort); + + // Init metalake and catalog + GravitinoAdminClient client = AbstractIT.getGravitinoClient(); + client.createMetalake(metalakeName, "", Collections.emptyMap()); + GravitinoMetalake metalake = client.loadMetalake(metalakeName); + metalake.createCatalog( + HIVE_CATALOG_NAME, + Catalog.Type.RELATIONAL, + "hive", + "", + getHiveCatalogConfigs(hiveMetastoreUri)); + metalake.createCatalog( + ICEBERG_CATALOG_NAME, + Catalog.Type.RELATIONAL, + "lakehouse-iceberg", + "", + getIcebergCatalogConfigs(hiveMetastoreUri)); + + client.close(); + } + + private Map getHiveCatalogConfigs(String hiveMetastoreUri) { + Map catalogProperties = Maps.newHashMap(); + catalogProperties.put(GravitinoSparkConfig.GRAVITINO_HIVE_METASTORE_URI, hiveMetastoreUri); + return catalogProperties; + } + + private Map getIcebergCatalogConfigs(String hiveMetastoreUri) { + Map catalogProperties = Maps.newHashMap(); + catalogProperties.put( + IcebergPropertiesConstants.GRAVITINO_ICEBERG_CATALOG_BACKEND, + IcebergPropertiesConstants.ICEBERG_CATALOG_BACKEND_HIVE); + catalogProperties.put( + IcebergPropertiesConstants.GRAVITINO_ICEBERG_CATALOG_WAREHOUSE, warehouse); + catalogProperties.put( + IcebergPropertiesConstants.GRAVITINO_ICEBERG_CATALOG_URI, hiveMetastoreUri); + return catalogProperties; + } + + private void closeGravitinoEnv() throws Exception { + AbstractIT.stopIntegrationTest(); + } + + private void writeQueryOutput(Path outputFile, List queryOutputs) + throws IOException { + String goldenOutput = + "-- Automatically generated by Gravitino Spark SQL test\n" + + queryOutputs.stream().map(QueryOutput::toString).collect(Collectors.joining("\n\n\n")) + + "\n"; + stringToFile(outputFile, goldenOutput); + } + + private List getQueriesFromFile(Path file) throws IOException { + String input = fileToString(file); + Pair, List> pair = splitCommentsAndCodes(input); + List queries = pair.getRight(); + queries = splitWithSemicolon(queries); + queries = cleanAndFilterQueries(queries); + return queries; + } + + private void runTestCase(TestCase testCase) throws IOException { + LOG.info("Run test case:{}", testCase.toString()); + List queries = getQueriesFromFile(testCase.getTestFile()); + List queryOutputs = runTestQueries(queries, true); + if (regenerateGoldenFiles) { + writeQueryOutput(testCase.getTestOutputFile(), queryOutputs); + } + List expectedOutputs = getExpectedOutputs(testCase.getTestOutputFile()); + + Assertions.assertEquals( + expectedOutputs.size(), queryOutputs.size(), "Query size not match for test: " + testCase); + + for (int i = 0; i < expectedOutputs.size(); i++) { + QueryOutput queryOutput = queryOutputs.get(i); + QueryOutput expectedOutput = expectedOutputs.get(i); + Assertions.assertEquals( + expectedOutput.getSql(), queryOutput.getSql(), "SQL not match for test: " + testCase); + Assertions.assertEquals( + expectedOutput.getSchema(), + queryOutput.getSchema(), + "SQL schema not match for test: " + testCase); + Assertions.assertEquals( + expectedOutput.getOutput(), + queryOutput.getOutput(), + "SQL output not match for test: " + testCase + ", sql: " + expectedOutput.getSql()); + } + } + + private List runTestQueries(List queries, boolean catchException) { + SparkSession localSparkSession = sparkSession; + return queries.stream() + .map( + query -> { + Pair> pair; + if (catchException) { + pair = + SQLQueryTestHelper.handleExceptions( + () -> SQLQueryTestHelper.getNormalizedResult(localSparkSession, query)); + } else { + pair = SQLQueryTestHelper.getNormalizedResult(localSparkSession, query); + } + String schema = pair.getLeft(); + String output = + pair.getRight().stream() + .collect(Collectors.joining("\n")) + .replaceAll("\\s+$", ""); + return new QueryOutput(query, schema, output); + }) + .collect(Collectors.toList()); + } + + private List getExpectedOutputs(Path outputPath) throws IOException { + String goldenOutput = fileToString(outputPath); + String[] segments = goldenOutput.split("-- !query.*\\n"); + + List expectedQueryOutputs = new ArrayList<>(); + // the first segment is comment, skip it + for (int i = 0; i * 3 + 3 < segments.length; i++) { + QueryOutput queryOutput = + new QueryOutput( + segments[i * 3 + 1].trim(), + segments[i * 3 + 2].trim(), + segments[i * 3 + 3].trim().replaceAll("\\s+$", "")); + expectedQueryOutputs.add(queryOutput); + } + return expectedQueryOutputs; + } + + private static void stringToFile(Path path, String str) throws IOException { + FileUtils.writeStringToFile(path.toFile(), str, StandardCharsets.UTF_8); + } + + private String fileToString(Path filePath) throws IOException { + return FileUtils.readFileToString(filePath.toFile(), StandardCharsets.UTF_8); + } + + private static Pair, List> splitCommentsAndCodes(String input) { + String[] lines = input.split("\n"); + List comments = new ArrayList<>(); + List codes = new ArrayList<>(); + for (String line : lines) { + String trimmedLine = line.trim(); + if (trimmedLine.startsWith("--") && trimmedLine.startsWith("--QUERY-DELIMITER") == false) { + comments.add(line); + } else { + codes.add(line); + } + } + return Pair.of(comments, codes); + } + + private static List splitWithSemicolon(List sequences) { + return Arrays.asList(String.join("\n", sequences).split("(? cleanAndFilterQueries(List tempQueries) { + return tempQueries.stream() + .map(String::trim) + .filter(s -> !s.isEmpty()) + .map( + s -> + Arrays.stream(s.split("\n")) + .filter(line -> !line.trim().startsWith("--")) + .collect(Collectors.joining("\n"))) + .map(String::trim) + .filter(s -> !s.isEmpty()) + .collect(Collectors.toList()); + } + + private void sql(String sql) { + sparkSession.sql(sql); + } + + private void useCatalog(CatalogType catalogType) { + String catalogName = catalogs.get(catalogType); + sql("USE " + catalogName); + } + + private void initSparkEnv() { + this.sparkSession = + SparkSession.builder() + .master("local[1]") + .appName("Spark sql query test") + .config("spark.plugins", GravitinoSparkPlugin.class.getName()) + .config(GravitinoSparkConfig.GRAVITINO_URI, gravitinoUri) + .config(GravitinoSparkConfig.GRAVITINO_METALAKE, metalakeName) + .config(GravitinoSparkConfig.GRAVITINO_ENABLE_ICEBERG_SUPPORT, "true") + .config("spark.gravitino.test.data.dir", dataDir) + .config("hive.exec.dynamic.partition.mode", "nonstrict") + // produce old parquet format + .config("spark.sql.parquet.writeLegacyFormat", "true") + .config("spark.sql.warehouse.dir", warehouse) + .config("spark.sql.session.timeZone", "UTC") + .config("spark.locality.wait.node", "0") + .enableHiveSupport() + .getOrCreate(); + } + + private void runQuery(Path file) throws IOException { + LOG.info("Run query: {}", file.toString()); + List queries = getQueriesFromFile(file); + runTestQueries(queries, false); + } +} diff --git a/spark-connector/spark-common/src/test/java/org/apache/gravitino/spark/connector/integration/test/sql/SparkSQLRegressionTest.java b/spark-connector/spark-common/src/test/java/org/apache/gravitino/spark/connector/integration/test/sql/SparkSQLRegressionTest.java new file mode 100644 index 00000000000..dea3dd36ce1 --- /dev/null +++ b/spark-connector/spark-common/src/test/java/org/apache/gravitino/spark/connector/integration/test/sql/SparkSQLRegressionTest.java @@ -0,0 +1,157 @@ +/* + * 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.gravitino.spark.connector.integration.test.sql; + +import java.io.File; +import java.io.IOException; +import java.nio.file.DirectoryStream; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.util.ArrayList; +import java.util.List; +import java.util.Properties; +import org.apache.logging.log4j.util.Strings; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Tag; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestInstance; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * The entrypoint to run SparkSQL regression test, you could run it with specific Spark version: + * ./gradlew :spark-connector:spark-3.4:test --tests + * "org.apache.gravitino.spark.connector.integration.test.sql.SparkSQLRegressionTest34" + * -PenableSparkSQLITs + */ +@Tag("gravitino-docker-test") +@TestInstance(TestInstance.Lifecycle.PER_CLASS) +public abstract class SparkSQLRegressionTest { + private static final Logger LOG = LoggerFactory.getLogger(SparkSQLRegressionTest.class); + private static final String SPARK_TEST_CONFIG_FILE = "configFile"; + private SparkQueryRunner sparkQueryRunner; + private SparkTestConfig sparkTestConfig; + + @Test + public void run() throws IOException { + String baseDir = sparkTestConfig.getBaseDir(); + List testSQLs = sparkTestConfig.getTestSQLs(); + List sqlTestCaseGroups = + findSQLTests(Paths.get(baseDir), Paths.get(baseDir), testSQLs, CatalogType.UNKNOWN); + runSQLTests(sqlTestCaseGroups); + } + + @BeforeAll + void init() throws IOException { + String configFile = System.getProperty(SPARK_TEST_CONFIG_FILE); + this.sparkTestConfig = loadSparkTestConfig(configFile); + this.sparkQueryRunner = new SparkQueryRunner(sparkTestConfig); + } + + @AfterAll + void close() throws Exception { + if (sparkQueryRunner != null) { + sparkQueryRunner.close(); + } + } + + private List findSQLTests( + Path basePath, Path path, List testSQLs, CatalogType parentCatalogType) + throws IOException { + List sqlTestCaseGroups = new ArrayList<>(); + Path prepareSql = null; + Path cleanupSql = null; + List testCases = new ArrayList<>(); + try (DirectoryStream stream = Files.newDirectoryStream(path)) { + for (Path p : stream) { + if (Files.isDirectory(p)) { + CatalogType catalogType = CatalogType.fromString(p.getFileName().toString()); + sqlTestCaseGroups.addAll( + findSQLTests( + basePath, p, testSQLs, CatalogType.merge(parentCatalogType, catalogType))); + } else { + if (p.toString().endsWith(".sql")) { + if (p.getFileName().toString().equals("prepare.sql")) { + prepareSql = p; + } else if (p.getFileName().toString().equals("cleanup.sql")) { + cleanupSql = p; + } else { + if (testSQLs.isEmpty()) { + testCases.add(new TestCase(p)); + } else { + // just run specific SQL + Path remaindingPath = basePath.relativize(p); + for (String whiteSQL : testSQLs) { + if (remaindingPath.startsWith(whiteSQL)) { + testCases.add(new TestCase(p)); + break; + } + } + } + } + } + } + } + } catch (IOException e) { + throw new RuntimeException(e); + } + + if (testCases.isEmpty() == false) { + sqlTestCaseGroups.add( + new TestCaseGroup(testCases, prepareSql, cleanupSql, parentCatalogType)); + } + + return sqlTestCaseGroups; + } + + private void runSQLTests(List sqlTestCaseGroups) { + sqlTestCaseGroups.forEach( + sqlTestCaseGroup -> { + try { + sparkQueryRunner.runQuery(sqlTestCaseGroup); + } catch (IOException e) { + throw new RuntimeException(e); + } + }); + } + + private SparkTestConfig loadSparkTestConfig(String configFile) throws IOException { + if (Strings.isBlank(configFile)) { + String projectDir = System.getenv("GRAVITINO_ROOT_DIR"); + configFile = + Paths.get( + projectDir, + "spark-connector", + "spark-common", + "src", + "test", + "resources", + "spark-test.conf") + .toString(); + } + LOG.info("config file: {}", configFile); + + SparkTestConfig testConfig = new SparkTestConfig(); + Properties properties = testConfig.loadPropertiesFromFile(new File(configFile)); + testConfig.loadFromProperties(properties); + return testConfig; + } +} diff --git a/spark-connector/spark-common/src/test/java/org/apache/gravitino/spark/connector/integration/test/sql/SparkTestConfig.java b/spark-connector/spark-common/src/test/java/org/apache/gravitino/spark/connector/integration/test/sql/SparkTestConfig.java new file mode 100644 index 00000000000..d73986b18c6 --- /dev/null +++ b/spark-connector/spark-common/src/test/java/org/apache/gravitino/spark/connector/integration/test/sql/SparkTestConfig.java @@ -0,0 +1,128 @@ +/* + * 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.gravitino.spark.connector.integration.test.sql; + +import java.nio.file.Paths; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import org.apache.gravitino.Config; +import org.apache.gravitino.config.ConfigBuilder; +import org.apache.gravitino.config.ConfigConstants; +import org.apache.gravitino.config.ConfigEntry; +import org.junit.platform.commons.util.StringUtils; + +public class SparkTestConfig extends Config { + private static final String DEFAULT_BASE_DIR = + Paths.get( + System.getenv("GRAVITINO_ROOT_DIR"), + "spark-connector", + "spark-common", + "src", + "test", + "resources") + .toString(); + + private static final ConfigEntry TEST_BASE_DIR = + new ConfigBuilder("gravitino.spark.test.dir") + .doc("The Spark SQL test base dir") + .version(ConfigConstants.VERSION_0_6_0) + .stringConf() + .createWithDefault(DEFAULT_BASE_DIR); + + private static final ConfigEntry TEST_SQLS = + new ConfigBuilder("gravitino.spark.test.sqls") + .doc( + "Specify the test SQLs, using directory to specify group of SQLs like " + + "`test-sqls/hive`, using file path to specify one SQL like " + + "`test-sqls/hive/basic.sql`, use `,` to split multi part") + .version(ConfigConstants.VERSION_0_6_0) + .stringConf() + .create(); + + private static final ConfigEntry GENERATE_GOLDEN_FILES = + new ConfigBuilder("gravitino.spark.test.generateGoldenFiles") + .doc( + "Whether generate golden files which are used to check the correctness of the SQL result") + .version(ConfigConstants.VERSION_0_6_0) + .booleanConf() + .createWithDefault(Boolean.FALSE); + + private static final ConfigEntry GRAVITINO_METALAKE_NAME = + new ConfigBuilder("gravitino.spark.test.metalake") + .doc("The metalake name to run the test") + .version(ConfigConstants.VERSION_0_6_0) + .stringConf() + .createWithDefault("test"); + + private static final ConfigEntry SETUP_GRAVITINO_ENV = + new ConfigBuilder("gravitino.spark.test.setupEnv") + .doc("Whether to setup Gravitino and Hive environment") + .version(ConfigConstants.VERSION_0_6_0) + .booleanConf() + .createWithDefault(Boolean.FALSE); + + private static final ConfigEntry GRAVITINO_URI = + new ConfigBuilder("gravitino.spark.test.uri") + .doc( + "Gravitino uri address, only available when `gravitino.spark.test.setupEnv` is false") + .version(ConfigConstants.VERSION_0_6_0) + .stringConf() + .createWithDefault("http://127.0.0.1:8090"); + + private static final ConfigEntry ICEBERG_WAREHOUSE_DIR = + new ConfigBuilder("gravitino.spark.test.iceberg.warehouse") + .doc( + "Iceberg warehouse location, only available when `gravitino.spark.test.setupEnv` is false") + .version("0.6.0") + .stringConf() + .createWithDefault("hdfs://127.0.0.1:9000/user/hive/warehouse-spark-test"); + + public String getBaseDir() { + return get(TEST_BASE_DIR); + } + + public boolean generateGoldenFiles() { + return get(GENERATE_GOLDEN_FILES); + } + + public boolean isGravitinoEnvSetUp() { + return get(SETUP_GRAVITINO_ENV); + } + + public List getTestSQLs() { + String testSQLs = get(TEST_SQLS); + if (StringUtils.isNotBlank(testSQLs)) { + return Arrays.asList(testSQLs.split("\\s*,\\s*")); + } + return new ArrayList<>(); + } + + public String getGravitinoUri() { + return get(GRAVITINO_URI); + } + + public String getMetalakeName() { + return get(GRAVITINO_METALAKE_NAME); + } + + public String getWarehouseLocation() { + return get(ICEBERG_WAREHOUSE_DIR); + } +} diff --git a/spark-connector/spark-common/src/test/java/org/apache/gravitino/spark/connector/integration/test/sql/TestCase.java b/spark-connector/spark-common/src/test/java/org/apache/gravitino/spark/connector/integration/test/sql/TestCase.java new file mode 100644 index 00000000000..f1f1d66524b --- /dev/null +++ b/spark-connector/spark-common/src/test/java/org/apache/gravitino/spark/connector/integration/test/sql/TestCase.java @@ -0,0 +1,47 @@ +/* + * 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.gravitino.spark.connector.integration.test.sql; + +import java.nio.file.Path; +import java.nio.file.Paths; +import lombok.Getter; +import lombok.ToString; + +/** A test SQL files which contains multi SQL queries. */ +@Getter +@ToString +public final class TestCase { + private final Path testFile; + + public TestCase(Path testFile) { + this.testFile = testFile; + } + + // The SQL output to check the correctness the SQL result, The output file of '/a/b.sql' is + // '/a/b.sql.out' + public Path getTestOutputFile() { + String fileName = testFile.getFileName().toString(); + String outputFileName = fileName + ".out"; + Path parentPath = testFile.getParent(); + if (parentPath == null) { + return Paths.get(outputFileName); + } + return parentPath.resolve(outputFileName); + } +} diff --git a/spark-connector/spark-common/src/test/java/org/apache/gravitino/spark/connector/integration/test/sql/TestCaseGroup.java b/spark-connector/spark-common/src/test/java/org/apache/gravitino/spark/connector/integration/test/sql/TestCaseGroup.java new file mode 100644 index 00000000000..6617e3a0b7d --- /dev/null +++ b/spark-connector/spark-common/src/test/java/org/apache/gravitino/spark/connector/integration/test/sql/TestCaseGroup.java @@ -0,0 +1,45 @@ +/* + * 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.gravitino.spark.connector.integration.test.sql; + +import java.nio.file.Path; +import java.util.List; +import javax.annotation.Nullable; +import lombok.Getter; + +/** + * A group of test SQL files in same directory which belongs to one catalog and may contain + * prepare.sql to init or cleanup.sql to clean. + */ +@Getter +public class TestCaseGroup { + + List testCases; + @Nullable Path prepareFile; + @Nullable Path cleanupFile; + CatalogType catalogType; + + public TestCaseGroup( + List testCases, Path prepareFile, Path cleanupFile, CatalogType catalogType) { + this.testCases = testCases; + this.prepareFile = prepareFile; + this.cleanupFile = cleanupFile; + this.catalogType = catalogType; + } +} diff --git a/spark-connector/spark-common/src/test/resources/data/tpcds/call_center/call_center.dat b/spark-connector/spark-common/src/test/resources/data/tpcds/call_center/call_center.dat new file mode 100644 index 00000000000..0e8535552a6 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/data/tpcds/call_center/call_center.dat @@ -0,0 +1,6 @@ +1|AAAAAAAABAAAAAAA|1998-01-01|||2450952|NY Metro|large|2|1138|8AM-4PM|Bob Belcher|6|More than other authori|Shared others could not count fully dollars. New members ca|Julius Tran|3|pri|6|cally|730|Ash Hill|Boulevard|Suite 0|Midway|Williamson County|TN|31904|United States|-5|0.11| +2|AAAAAAAACAAAAAAA|1998-01-01|2000-12-31||2450806|Mid Atlantic|medium|6|2268|8AM-8AM|Felipe Perkins|2|A bit narrow forms matter animals. Consist|Largely blank years put substantially deaf, new others. Question|Julius Durham|5|anti|1|ought|984|Center Hill|Way|Suite 70|Midway|Williamson County|TN|31904|United States|-5|0.12| +3|AAAAAAAACAAAAAAA|2001-01-01|||2450806|Mid Atlantic|medium|6|4134|8AM-4PM|Mark Hightower|2|Wrong troops shall work sometimes in a opti|Largely blank years put substantially deaf, new others. Question|Julius Durham|1|ought|2|able|984|Center Hill|Way|Suite 70|Midway|Williamson County|TN|31904|United States|-5|0.01| +4|AAAAAAAAEAAAAAAA|1998-01-01|2000-01-01||2451063|North Midwest|medium|1|649|8AM-4PM|Larry Mccray|2|Dealers make most historical, direct students|Rich groups catch longer other fears; future,|Matthew Clifton|4|ese|3|pri|463|Pine Ridge|RD|Suite U|Midway|Williamson County|TN|31904|United States|-5|0.05| +5|AAAAAAAAEAAAAAAA|2000-01-02|2001-12-31||2451063|North Midwest|small|3|795|8AM-8AM|Larry Mccray|2|Dealers make most historical, direct students|Blue, due beds come. Politicians would not make far thoughts. Specifically new horses partic|Gary Colburn|4|ese|3|pri|463|Pine Ridge|RD|Suite U|Midway|Williamson County|TN|31904|United States|-5|0.12| +6|AAAAAAAAEAAAAAAA|2002-01-01|||2451063|North Midwest|medium|7|3514|8AM-4PM|Larry Mccray|5|Silly particles could pro|Blue, due beds come. Politicians would not make far thoughts. Specifically new horses partic|Gary Colburn|5|anti|3|pri|463|Pine Ridge|RD|Suite U|Midway|Williamson County|TN|31904|United States|-5|0.11| diff --git a/spark-connector/spark-common/src/test/resources/data/tpcds/catalog_page/catalog_page.dat b/spark-connector/spark-common/src/test/resources/data/tpcds/catalog_page/catalog_page.dat new file mode 100644 index 00000000000..6e7003e5d68 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/data/tpcds/catalog_page/catalog_page.dat @@ -0,0 +1,11718 @@ +1|AAAAAAAABAAAAAAA|2450815|2450996|DEPARTMENT|1|1|In general basic characters welcome. Clearly lively friends conv|bi-annual| +2|AAAAAAAACAAAAAAA|2450815|2450996|DEPARTMENT|1|2|English areas will leave prisoners. Too public countries ought to become beneath the years. |bi-annual| +3|AAAAAAAADAAAAAAA|2450815|2450996|DEPARTMENT|1|3|Times could not address disabled indians. Effectively public ports c|bi-annual| +4|AAAAAAAAEAAAAAAA|2450815|||1|||bi-annual| +5|AAAAAAAAFAAAAAAA|2450815|2450996|DEPARTMENT|1|5|Classic buildings ensure in a tests. Real years may not receive open systems. Now broad m|bi-annual| +6|AAAAAAAAGAAAAAAA|2450815|2450996|DEPARTMENT|1|6|Exciting principles wish greatly only excellent women. Appropriate fortunes shall not|bi-annual| +7|AAAAAAAAHAAAAAAA|2450815|2450996|DEPARTMENT|1|7|National services must not come at least into a girls|bi-annual| +8|AAAAAAAAIAAAAAAA|2450815|2450996|DEPARTMENT|1|8|Areas see early for a pounds. New goods study too serious women. Unwittingly sorry incentives shall|bi-annual| +9|AAAAAAAAJAAAAAAA|2450815|2450996|DEPARTMENT|1|9|Intensive, economic changes resist bloody of course simple economies; |bi-annual| +10|AAAAAAAAKAAAAAAA|2450815|2450996|DEPARTMENT|1|10|Careful, intense funds balance perhaps boys. Romantic chips remove legs. Direct birds get |bi-annual| +11|AAAAAAAALAAAAAAA|2450815|2450996|DEPARTMENT|1|11|At least national countries live by an sales. Weap|bi-annual| +12|AAAAAAAAMAAAAAAA|2450815|2450996|DEPARTMENT|1|12|Girls indicate so in a countries. Natural, emotional weeks try a|bi-annual| +13|AAAAAAAANAAAAAAA|2450815|2450996|DEPARTMENT|1|13|Miles see mainly clear hands. Villages finish there blue figures. Moreover wide students travel poo|bi-annual| +14|AAAAAAAAOAAAAAAA|2450815|2450996|DEPARTMENT|1|14|Rooms would say ago economic sections. Essential properties might not support groups. Ago rare eye|bi-annual| +15|AAAAAAAAPAAAAAAA|2450815|2450996|DEPARTMENT|1|15|Legal, required ends may not improve in the pictures. Really social structur|bi-annual| +16|AAAAAAAAABAAAAAA|2450815|2450996|DEPARTMENT|1|16|Schools must know now empty legs; generally daily children use sharp, loca|bi-annual| +17|AAAAAAAABBAAAAAA|2450815|2450996|DEPARTMENT|1|17|More than true carers can ensure at a officers. Candidates s|bi-annual| +18|AAAAAAAACBAAAAAA|2450815|2450996|DEPARTMENT|1|18|Shops end problems. Urban experiences play new stores. Institutions order as residential places.|bi-annual| +19|AAAAAAAADBAAAAAA|2450815|2450996|DEPARTMENT|1|19|Poor, hostile guidelines could hope alone early things. Secret, |bi-annual| +20|AAAAAAAAEBAAAAAA|2450815|2450996|DEPARTMENT|1|20|Appropriate years kill. Probably male units come perhaps between the minutes.|bi-annual| +21|AAAAAAAAFBAAAAAA|2450815|2450996|DEPARTMENT|1|21|Gentle fields understand forward high parental faciliti|bi-annual| +22|AAAAAAAAGBAAAAAA|2450815|2450996|DEPARTMENT|1|22|Square, good women can refuse. Women allow over courses. Entire, dail|bi-annual| +23|AAAAAAAAHBAAAAAA|2450815|2450996|DEPARTMENT|1|23|Level, ordinary names would change institutional principles. Fe|bi-annual| +24|AAAAAAAAIBAAAAAA|2450815|2450996|DEPARTMENT|1|24|For once public conditions could help even single, free vehicles. Soldi|bi-annual| +25|AAAAAAAAJBAAAAAA|2450815|2450996|DEPARTMENT|1|25|Dimensions ensure by a enquiries. Special times may practise much full, bad books; l|bi-annual| +26|AAAAAAAAKBAAAAAA|2450815|2450996|DEPARTMENT|1|26|Nervously reasonable sanctions would not supply often structures. National dangers relieve never|bi-annual| +27|AAAAAAAALBAAAAAA|2450815|2450996|DEPARTMENT|1|27|Ready restaurants can represent previously perfectly educational thanks; adjacent quantities t|bi-annual| +28|AAAAAAAAMBAAAAAA|2450815|2450996|DEPARTMENT|1|28|Also new funds must not find in addition essential poor friends; now |bi-annual| +29|AAAAAAAANBAAAAAA|2450815|2450996|DEPARTMENT|1|29|Small circumstances enter almost potentially spare individuals. Residential women launch|bi-annual| +30|AAAAAAAAOBAAAAAA|2450815|2450996|DEPARTMENT|1|30|Western, legislative courses cause truly else strange costs. Only grey windows come without a depo|bi-annual| +31|AAAAAAAAPBAAAAAA|2450815|2450996|DEPARTMENT|1|31|Differences would complete previously at a notes. Important|bi-annual| +32|AAAAAAAAACAAAAAA|2450815|2450996|DEPARTMENT|1|32|Quickly industrial trousers remember poor, practica|bi-annual| +33|AAAAAAAABCAAAAAA|2450815|2450996|DEPARTMENT|1|33|Here divine knees could not recall then before a meals. Resources use commercial, |bi-annual| +34|AAAAAAAACCAAAAAA|2450815|2450996|DEPARTMENT|1|34|Old, essential demonstrations watch absolutely; never woo|bi-annual| +35|AAAAAAAADCAAAAAA|2450815|2450996|DEPARTMENT|1|35|Simple blocks feel now whole police. Available meanings must shine clearly real, good departmen|bi-annual| +36|AAAAAAAAECAAAAAA|2450815|2450996|DEPARTMENT|1|36|Dead rats contain central, interesting years. Classes go.|bi-annual| +37|AAAAAAAAFCAAAAAA|2450815|2450996|DEPARTMENT|1|37|Approaches demonstrate requirements. Public views can need hard. Almost alone |bi-annual| +38|AAAAAAAAGCAAAAAA|2450815|2450996|DEPARTMENT|1|38|Ready perceptions close only with a conservatives.|bi-annual| +39|AAAAAAAAHCAAAAAA|2450815|2450996|DEPARTMENT|1|39|Vastly favorite systems may not imply manufacturing, new facilities. Eventually wo|bi-annual| +40|AAAAAAAAICAAAAAA|2450815|2450996|DEPARTMENT|1|40|Social, full questions learn already cautiously unemployed solicitors. Alone, close s|bi-annual| +41|AAAAAAAAJCAAAAAA|2450815|2450996|DEPARTMENT|1|41|True friends may not pay once at first good issues. Girls ought to grow t|bi-annual| +42|AAAAAAAAKCAAAAAA|2450815|2450996|DEPARTMENT|1|42|Fine animals may expect solidly. Highest extra vessels take here simila|bi-annual| +43|AAAAAAAALCAAAAAA|2450815|2450996|DEPARTMENT|1|43|Relatively darling limits must not forgive. Rural, dry folk h|bi-annual| +44|AAAAAAAAMCAAAAAA|2450815|2450996|DEPARTMENT|1|44|More likely movements shall identify badly other sensitive ideas. Fully other views could not|bi-annual| +45|AAAAAAAANCAAAAAA|2450815|2450996|DEPARTMENT|1|45|Full heads imply unfortunately good policies. However different periods will not avoid highly in |bi-annual| +46|AAAAAAAAOCAAAAAA|2450815|2450996|DEPARTMENT|1|46|Old beings used to raise. Curtains cannot enable just sites. Cells can say western talks;|bi-annual| +47|AAAAAAAAPCAAAAAA|2450815|2450996|DEPARTMENT|1|47|Dangerous, desirable relations might forget always free activities. Values ough|bi-annual| +48|AAAAAAAAADAAAAAA|2450815|2450996|DEPARTMENT|1|48|Moving, parliamentary women seal only glad, little damages. Electoral ways |bi-annual| +49|AAAAAAAABDAAAAAA|2450815|2450996|DEPARTMENT|1|49|Old, extensive months cannot write political numbers. Minor lips know as usual f|bi-annual| +50|AAAAAAAACDAAAAAA|2450815|2450996|DEPARTMENT|1|50|Now metropolitan prices survive institutions. Internal cheeks|bi-annual| +51|AAAAAAAADDAAAAAA|2450815|2450996|DEPARTMENT|1|51|Superior, pink men should plant also recent, very fears. Purposes see other men. Tough, particular|bi-annual| +52|AAAAAAAAEDAAAAAA|2450815|2450996|DEPARTMENT|1|52|Urban, large girls could go. Actions might guard about key ot|bi-annual| +53|AAAAAAAAFDAAAAAA|2450815|2450996|DEPARTMENT|1|53|Relevant, major backs fly just private findings. New workers must not keep l|bi-annual| +54|AAAAAAAAGDAAAAAA|2450815|2450996|DEPARTMENT|1|54|Rights should reassure as to a borders; difficult, efficient servants might no|bi-annual| +55|AAAAAAAAHDAAAAAA|2450815|2450996|DEPARTMENT|1|55|Male requirements can make. Catholic results load maybe common ba|bi-annual| +56|AAAAAAAAIDAAAAAA|2450815|2450996|DEPARTMENT|1|56|Men talk even like a days. Still large feelings see new, quick hotels. Far black relations should|bi-annual| +57|AAAAAAAAJDAAAAAA|2450815|2450996|DEPARTMENT|1|57|Expressly local horses shall undertake perhaps able, co|bi-annual| +58|AAAAAAAAKDAAAAAA|2450815|2450996|DEPARTMENT|1|58|Tasks work fun trees. Old employees make colonial, good signs. Famous emo|bi-annual| +59|AAAAAAAALDAAAAAA|2450815|2450996|DEPARTMENT|1|59|Free, like figures will not leave once similar, eas|bi-annual| +60|AAAAAAAAMDAAAAAA|2450815|2450996|DEPARTMENT|1|60|Teachers would choose always average investments. Unable, illegal sorts see. Shadows sha|bi-annual| +61|AAAAAAAANDAAAAAA|2450815|2450996|DEPARTMENT|1|61|Glad developments improve still unable, invisible communications. High processes ru|bi-annual| +62|AAAAAAAAODAAAAAA|2450815|2450996|DEPARTMENT|1|62|Indeed poor enemies change so to a hours. Either public circumstan|bi-annual| +63|AAAAAAAAPDAAAAAA|2450815|2450996|DEPARTMENT|1|63|Possible, general practitioners must mate in a companies. Small developments per|bi-annual| +64|AAAAAAAAAEAAAAAA|2450815|2450996|DEPARTMENT|1|64|New, other men would hear secret costs. Major lines might think. Modern hands shall |bi-annual| +65|AAAAAAAABEAAAAAA|2450815|2450996|DEPARTMENT|1|65|Otherwise vast titles could comprehend just, strong forms. Historic, european owners enha|bi-annual| +66|AAAAAAAACEAAAAAA|2450815|2450996|DEPARTMENT|1|66|Simple types stir oddly. More head depths could tak|bi-annual| +67|AAAAAAAADEAAAAAA|2450815|2450996|DEPARTMENT|1|67|Other developments shall not race in order objective wings. Nearby departments|bi-annual| +68|AAAAAAAAEEAAAAAA|2450815|2450996|DEPARTMENT|1|68|More big police cannot give important, difficult clients. Black, early vessels use|bi-annual| +69|AAAAAAAAFEAAAAAA|2450815|2450996|DEPARTMENT|1|69|More basic colleagues should shift; muscles make however legal authorities. |bi-annual| +70|AAAAAAAAGEAAAAAA|2450815|2450996|DEPARTMENT|1|70|Visual, major matters witness together so-called days. Obviously sharp men spend meanwhile els|bi-annual| +71|AAAAAAAAHEAAAAAA|2450815|2450996|DEPARTMENT|1|71|Numbers may create now concerned families. Machines must |bi-annual| +72|AAAAAAAAIEAAAAAA|2450815|2450996|DEPARTMENT|1|72|Sometimes lexical police shall affect women. Personal banks could not cost e|bi-annual| +73|AAAAAAAAJEAAAAAA|2450815|2450996|DEPARTMENT|1|73|Journalists develop civil, open methods. Very normal vehicles used t|bi-annual| +74|AAAAAAAAKEAAAAAA|2450815|2450996|DEPARTMENT|1|74|Cheap problems obtain national, new organisations. Certain, suitable words |bi-annual| +75|AAAAAAAALEAAAAAA|2450815|2450996|DEPARTMENT|1|75|Specifically russian plans must use neither grand shares. Harsh, french results increase|bi-annual| +76|AAAAAAAAMEAAAAAA|2450815|2450996|DEPARTMENT|1|76|Entirely normal thousands cannot rely just important ti|bi-annual| +77|AAAAAAAANEAAAAAA|2450815|2450996|DEPARTMENT|1|77|American years swallow only, nice thanks. Obvious materials might not discuss just. |bi-annual| +78|AAAAAAAAOEAAAAAA|2450815|2450996|DEPARTMENT|1|78|Intelligent buildings might not keep here new cases. Sole|bi-annual| +79|AAAAAAAAPEAAAAAA|2450815|2450996|DEPARTMENT|1|79|Complaints release views. Really internal interpretations used to make funny, important |bi-annual| +80|AAAAAAAAAFAAAAAA|2450815|2450996|DEPARTMENT|1|80|Words know difficult, lucky areas. Over red parts need extr|bi-annual| +81|AAAAAAAABFAAAAAA|2450815|2450996|DEPARTMENT|1|81|Vital findings come more about convincing firms. Hor|bi-annual| +82|AAAAAAAACFAAAAAA|2450815|2450996|DEPARTMENT|1|82|Colonial interests reach also. Results will not pursue seconds. Labour other|bi-annual| +83|AAAAAAAADFAAAAAA|2450815|2450996|DEPARTMENT|1|83|In order compatible schools float particular families. Briefly strong words used |bi-annual| +84|AAAAAAAAEFAAAAAA|2450815|2450996|DEPARTMENT|1|84|Industrial cars exist workers. Dependent pages should consider straig|bi-annual| +85|AAAAAAAAFFAAAAAA|2450815|2450996|DEPARTMENT|1|85|Minutes ought to keep marginal states. New, financial nurses become later. Legally engl|bi-annual| +86|AAAAAAAAGFAAAAAA|2450815|2450996|DEPARTMENT|1|86|Talks draw totally manufacturers. Lines evolve very; conditions see mi|bi-annual| +87|AAAAAAAAHFAAAAAA|2450815|2450996|DEPARTMENT|1|87|Away dramatic letters say slightly for a advisers. Severe, difficult points entail h|bi-annual| +88|AAAAAAAAIFAAAAAA|2450815|2450996|DEPARTMENT|1|88|Particular projects involve always final lakes. Subjects must move|bi-annual| +89|AAAAAAAAJFAAAAAA|2450815|2450996|DEPARTMENT|1|89|More responsible bands pursue then including a tasks; extremely traditional numbe|bi-annual| +90|AAAAAAAAKFAAAAAA|2450815|2450996|DEPARTMENT|1|90|Enough real studies should not appear weekends. Rich,|bi-annual| +91|AAAAAAAALFAAAAAA|2450815|2450996|DEPARTMENT|1|91|Safe prices grow there enough private boys. Priorities come eventually forces. Regi|bi-annual| +92|AAAAAAAAMFAAAAAA|2450815|2450996|DEPARTMENT|1|92|Years can know. Available, dead degrees may begin ever. Docto|bi-annual| +93|AAAAAAAANFAAAAAA|2450815|2450996|DEPARTMENT|1|93|Perfectly mild differences must visit deaf years. Attractive, interesting women might not tie to t|bi-annual| +94|AAAAAAAAOFAAAAAA|2450815|2450996|DEPARTMENT|1|94|Equal, other calls maintain simply. Newspapers cannot touch f|bi-annual| +95|AAAAAAAAPFAAAAAA|2450815|2450996|DEPARTMENT|1|95|Bonds include completely critical principles. Charges walk just forms. Subsequent, ti|bi-annual| +96|AAAAAAAAAGAAAAAA|2450815|2450996|DEPARTMENT|1|96|Then other details will stop nuclear officers. Capital expenses carve far options. Stru|bi-annual| +97|AAAAAAAABGAAAAAA|2450815|2450996|DEPARTMENT|1|97|Trees help far also ideal debts. Old, outstanding forces|bi-annual| +98|AAAAAAAACGAAAAAA|2450815|2450996|DEPARTMENT|1|98|Still fine services would surrender urban results; mean, serious considerations may say|bi-annual| +99|AAAAAAAADGAAAAAA|2450815|2450996|DEPARTMENT|1|99|Useful, white courts leave almost high fingers. Popular, basic words plan|bi-annual| +100|AAAAAAAAEGAAAAAA|2450815|2450996|DEPARTMENT|1|100|Necessarily general police commit out of a sites. Supplies must|bi-annual| +101|AAAAAAAAFGAAAAAA|2450815|2450996|DEPARTMENT|1|101|Central, ethnic audiences educate things. National, severe hospitals shall prove explicitly eyes. O|bi-annual| +102|AAAAAAAAGGAAAAAA|2450815|2450996|DEPARTMENT|1|102|Yellow, pure eyes find in the lists. Broad, able indians lead royal, suitable names. Available, lo|bi-annual| +103|AAAAAAAAHGAAAAAA|2450815|2450996|DEPARTMENT|1|103|Women will come as inherent problems. Public pupils would prevent by a feelings; mental, american|bi-annual| +104|AAAAAAAAIGAAAAAA|2450815|2450996|DEPARTMENT|1|104|Else ideological examples enlarge sooner by a cars. J|bi-annual| +105|AAAAAAAAJGAAAAAA|2450815|2450996|DEPARTMENT|1|105|Probably international sides might sit unusual, full ti|bi-annual| +106|AAAAAAAAKGAAAAAA|2450815|2450996|DEPARTMENT|1|106|Efficient, various obligations pay home regions. Electrical, equal communities |bi-annual| +107|AAAAAAAALGAAAAAA|2450815|2450996|DEPARTMENT|1|107|Present children may know currently for a eyes; huge, responsible patients tell under|bi-annual| +108|AAAAAAAAMGAAAAAA|2450815|2450996|DEPARTMENT|1|108|Jewish services watch nearly. Foreign children disturb in o|bi-annual| +109|AAAAAAAANGAAAAAA|2450997|2451178|DEPARTMENT|2|1|Others ought to satisfy. Men cannot lose usually wonderful, mutual strategies. Socia|bi-annual| +110|AAAAAAAAOGAAAAAA|2450997|2451178|DEPARTMENT|2|2|Also deep months could not need very. New, financial troops minimis|bi-annual| +111|AAAAAAAAPGAAAAAA|2450997|2451178|DEPARTMENT|2|3|Again early points will add far good sure legs. Sports search beautiful, useful days. New, |bi-annual| +112|AAAAAAAAAHAAAAAA|2450997|2451178|DEPARTMENT|2|4|Foreign tonnes shall see forward. Broad, old men block stars. Independent police |bi-annual| +113|AAAAAAAABHAAAAAA|2450997|2451178|DEPARTMENT|2|5|Environmental, rare changes could worry eventually indivi|bi-annual| +114|AAAAAAAACHAAAAAA|2450997|2451178|DEPARTMENT|2|6|Significant thousands leave in a books. Things must not mark instead big, aware cattle. M|bi-annual| +115|AAAAAAAADHAAAAAA|2450997|2451178|DEPARTMENT|2|7|Traditional, clear years used to cover regardless |bi-annual| +116|AAAAAAAAEHAAAAAA|2450997|2451178|DEPARTMENT|2|8|Men might win at least old others. Services take very good materials. Positive rows ask isolated, |bi-annual| +117|AAAAAAAAFHAAAAAA|2450997|2451178|DEPARTMENT|2|9|Ways used to require ever empty days. Industries used to trust problems; whol|bi-annual| +118|AAAAAAAAGHAAAAAA|2450997|2451178|DEPARTMENT|2|10|Villages may not note quite therefore senior words|bi-annual| +119|AAAAAAAAHHAAAAAA|2450997|2451178|DEPARTMENT|2|11|Mad, primary skills come. Silent consequences must work yet likely, local cells. Success|bi-annual| +120|AAAAAAAAIHAAAAAA|2450997|2451178|DEPARTMENT|2|12|Strange, poor sales give more always necessary addresses. New farmers see lines. B|bi-annual| +121|AAAAAAAAJHAAAAAA|2450997|2451178|DEPARTMENT|2|13|Patients used to accept very excellent measurements; m|bi-annual| +122|AAAAAAAAKHAAAAAA|2450997|2451178|DEPARTMENT|2|14|Then normal attitudes ought to return indeed considerable|bi-annual| +123|AAAAAAAALHAAAAAA|2450997|2451178|DEPARTMENT|2|15|Certainly electrical reasons shall make girls. American, si|bi-annual| +124|AAAAAAAAMHAAAAAA|2450997|2451178|DEPARTMENT|2|16|Major windows contain inevitably hours. Desperate power|bi-annual| +125|AAAAAAAANHAAAAAA|2450997|2451178|DEPARTMENT|2|17|Full matters may accommodate only. Recent, proposed settings start approximately |bi-annual| +126|AAAAAAAAOHAAAAAA|2450997|2451178|DEPARTMENT|2|18|Judicial, main things resolve others. Permanent, possible practitioners shall|bi-annual| +127|AAAAAAAAPHAAAAAA|2450997|2451178|DEPARTMENT|2|19|Matters buy. Diverse, limited beliefs used to go police. Difficult, national arms c|bi-annual| +128|AAAAAAAAAIAAAAAA|2450997|2451178|DEPARTMENT|2|20|Dead eyes might influence as in a elections. Topics clear properly just s|bi-annual| +129|AAAAAAAABIAAAAAA|2450997|2451178|DEPARTMENT|2|21|Colleagues must apply please intact, suspicious stan|bi-annual| +130|AAAAAAAACIAAAAAA|2450997|2451178|DEPARTMENT|2|22|Examples may drink about only, running boundaries. Much past generations might take com|bi-annual| +131|AAAAAAAADIAAAAAA|2450997|2451178|DEPARTMENT|2|23|Good states shall not replace just members. Unusually different tonnes wo|bi-annual| +132|AAAAAAAAEIAAAAAA|2450997|2451178|DEPARTMENT|2|24|Farmers could address ever. Buildings would learn usually perhaps supreme savings. Active|bi-annual| +133|AAAAAAAAFIAAAAAA|2450997|2451178|DEPARTMENT|2|25|Now central courses come both on a ways; following words used to get further annual services. Rigo|bi-annual| +134|AAAAAAAAGIAAAAAA|2450997|2451178|DEPARTMENT|2|26|Criminal, different colleagues yield in general particular teachers. Fo|bi-annual| +135|AAAAAAAAHIAAAAAA|2450997|2451178|DEPARTMENT|2|27|Inevitably imperial drawings create quickly firms. N|bi-annual| +136|AAAAAAAAIIAAAAAA|2450997|2451178|DEPARTMENT|2|28|Earlier complete children shall influence. Influential voters withdraw there senior legs. Private|bi-annual| +137|AAAAAAAAJIAAAAAA|2450997|2451178|DEPARTMENT|2|29|Assets may come; panels must handle only. About voluntary patients keep bette|bi-annual| +138|AAAAAAAAKIAAAAAA|2450997|2451178|DEPARTMENT|2|30|Effects keep. Otherwise grey stocks tackle all right to a plates; then nuclear w|bi-annual| +139|AAAAAAAALIAAAAAA|2450997|2451178|DEPARTMENT|2|31|Farms pay about young laws. Pools must not mark downstream russian posts. Instances|bi-annual| +140|AAAAAAAAMIAAAAAA|2450997|2451178|DEPARTMENT|2|32|Years keep as industries. Profits draw terribly free standards; likely, sorry decad|bi-annual| +141|AAAAAAAANIAAAAAA|2450997|2451178|DEPARTMENT|2|33|Very, legislative difficulties would encourage newly physical, important worlds; eyes |bi-annual| +142|AAAAAAAAOIAAAAAA|2450997|2451178|DEPARTMENT|2|34|Sure main individuals could not see coherent, professional elements. This serio|bi-annual| +143|AAAAAAAAPIAAAAAA|2450997|2451178|DEPARTMENT|2|35|Other, other hours will not cover more. Particularly other |bi-annual| +144|AAAAAAAAAJAAAAAA|2450997|2451178|DEPARTMENT|2|36|Very darling letters might not cite indeed only, adjacent workers; toys might raise good member|bi-annual| +145|AAAAAAAABJAAAAAA|2450997|2451178|DEPARTMENT|2|37|Active, common speakers lead attitudes. Systems could pay h|bi-annual| +146|AAAAAAAACJAAAAAA|2450997|2451178|DEPARTMENT|2|38|Relevant, national holes invite financial, major lines. Old requirements would look right|bi-annual| +147|AAAAAAAADJAAAAAA|2450997|2451178|DEPARTMENT|2|39|Agreements can test closer then royal affairs. Identica|bi-annual| +148|AAAAAAAAEJAAAAAA|2450997|2451178|DEPARTMENT|2|40|Pale moments make short whole pairs. Bare teachers must not affect a bit. |bi-annual| +149|AAAAAAAAFJAAAAAA|2450997|2451178|DEPARTMENT|2|41|Right potatoes drown; employees remember with a languages. Steadily alternative organi|bi-annual| +150|AAAAAAAAGJAAAAAA|2450997|2451178|DEPARTMENT|2|42|Books should catch now awkward sectors. Proposals postpone by a|bi-annual| +151|AAAAAAAAHJAAAAAA|2450997|2451178|DEPARTMENT|2|43|Likely values get away given, collective problems. Con|bi-annual| +152|AAAAAAAAIJAAAAAA|2450997|2451178|DEPARTMENT|2|44|Collective, unusual children take later to a years. Able, medical months might|bi-annual| +153|AAAAAAAAJJAAAAAA|2450997|2451178|DEPARTMENT|2|45|Special, capable practices would create to the agents. Black, lucky b|bi-annual| +154|AAAAAAAAKJAAAAAA|2450997|2451178|DEPARTMENT|2|46|Orders invest national, true men. Alike rural sales |bi-annual| +155|AAAAAAAALJAAAAAA|2450997|2451178|DEPARTMENT|2|47|Police will take about increased purposes; other reports could not regret. Earlier irish wo|bi-annual| +156|AAAAAAAAMJAAAAAA|2450997|2451178|DEPARTMENT|2|48|Already present services must illustrate relatively chemical, involved stages. Members|bi-annual| +157|AAAAAAAANJAAAAAA|2450997|2451178|DEPARTMENT|2|49|Managerial, slow things meet new, used minutes. Pretty years protect in a ears. Minutes |bi-annual| +158|AAAAAAAAOJAAAAAA|2450997|2451178|DEPARTMENT|2|50|Difficulties give reasonably. Letters expand months. Profes|bi-annual| +159|AAAAAAAAPJAAAAAA|2450997|2451178|DEPARTMENT|2|51|Increased islands may derive late originally british defences. Charg|bi-annual| +160|AAAAAAAAAKAAAAAA|2450997||||||| +161|AAAAAAAABKAAAAAA|2450997|2451178|DEPARTMENT|2|53|Bills ought to tell now families. Local, particular daughters judge h|bi-annual| +162|AAAAAAAACKAAAAAA|2450997|2451178|DEPARTMENT|2|54|Again complex differences can carry both here other pr|bi-annual| +163|AAAAAAAADKAAAAAA|2450997|2451178|DEPARTMENT|2|55|Marks send straight soviet proposals. Routine ends could save there. New ways could win|bi-annual| +164|AAAAAAAAEKAAAAAA|2450997|2451178|DEPARTMENT|2|56|At all appointed men see local groups. Well royal months|bi-annual| +165|AAAAAAAAFKAAAAAA|2450997|2451178|DEPARTMENT|2|57|Others could not worsen soldiers. Things prevent at first over a officers. Most mild effe|bi-annual| +166|AAAAAAAAGKAAAAAA|2450997|2451178|DEPARTMENT|2|58|Famous children handle as as yet local games. Extensive teeth mig|bi-annual| +167|AAAAAAAAHKAAAAAA|2450997|2451178|DEPARTMENT|2|59|Normal, other women may not guide fixed businesses; relations|bi-annual| +168|AAAAAAAAIKAAAAAA|2450997|2451178|DEPARTMENT|2|60|Whole, american vehicles stand only practical products. Objectives call quite therefore ope|bi-annual| +169|AAAAAAAAJKAAAAAA|2450997|2451178|DEPARTMENT|2|61|Wonderful, new services support. There new missiles must lie even short, hard|bi-annual| +170|AAAAAAAAKKAAAAAA|2450997|2451178|DEPARTMENT|2|62|Comparative principles must change across legitimate comments. Upper days cou|bi-annual| +171|AAAAAAAALKAAAAAA|2450997|2451178|DEPARTMENT|2|63|Ordinary industries find easily pages. Cases may stop |bi-annual| +172|AAAAAAAAMKAAAAAA|2450997|2451178|DEPARTMENT|2|64|White responsibilities trip currently. Full girls may take more future|bi-annual| +173|AAAAAAAANKAAAAAA|2450997|2451178|DEPARTMENT|2|65|Assets will apply. Jobs share only; alive hands hurt of course only free|bi-annual| +174|AAAAAAAAOKAAAAAA|2450997|2451178|DEPARTMENT|2|66|Communities say. Days would not know polish aspects. Usually shared areas listen often overal|bi-annual| +175|AAAAAAAAPKAAAAAA|2450997|2451178|DEPARTMENT|2|67|So whole eyes use systems. Experiments find lights. Smal|bi-annual| +176|AAAAAAAAALAAAAAA|2450997|2451178|DEPARTMENT|2|68|Views might not fall; local questions cannot pay hills. Conditions buy specified arts; crews writ|bi-annual| +177|AAAAAAAABLAAAAAA|2450997|2451178|DEPARTMENT|2|69|High exotic others may catch there operational long operations|bi-annual| +178|AAAAAAAACLAAAAAA|2450997|2451178|DEPARTMENT|2|70|Manufacturing, golden areas cannot take enough soviet m|bi-annual| +179|AAAAAAAADLAAAAAA|2450997|2451178|DEPARTMENT|2|71|Years may go criminal, emotional prices. Straightforwa|bi-annual| +180|AAAAAAAAELAAAAAA|2450997|2451178|DEPARTMENT|2|72|Professional paths shall not mind now relatively happy members. Mechanical territories shall s|bi-annual| +181|AAAAAAAAFLAAAAAA|2450997|2451178|DEPARTMENT|2|73|Months want indeed below rare executives. Players may care earlier already specific spiri|bi-annual| +182|AAAAAAAAGLAAAAAA|2450997|2451178|DEPARTMENT|2|74|Maybe confident eyes forget solidly previous benefits. Faces shall see groups. Dea|bi-annual| +183|AAAAAAAAHLAAAAAA|||||75||| +184|AAAAAAAAILAAAAAA|2450997|2451178|DEPARTMENT|2|76|Primary, necessary homes realize others. Rates invest nev|bi-annual| +185|AAAAAAAAJLAAAAAA|2450997|2451178|DEPARTMENT|2|77|Finally able reports may die wrong, nuclear libraries; important, w|bi-annual| +186|AAAAAAAAKLAAAAAA|2450997|2451178|DEPARTMENT|2|78|Eyes stop modern, royal places. New, formal firms feel of course awkward clients. Onl|bi-annual| +187|AAAAAAAALLAAAAAA|2450997|2451178|DEPARTMENT|2|79|Universal weeks establish at first biological projects. Images can speak th|bi-annual| +188|AAAAAAAAMLAAAAAA|2450997|2451178|DEPARTMENT|2|80|Independent, japanese reductions exist very dead, national parents. Rec|bi-annual| +189|AAAAAAAANLAAAAAA|2450997|2451178|DEPARTMENT|2|81|Now vital books used to stay never new, high regions. So small react|bi-annual| +190|AAAAAAAAOLAAAAAA|2450997|2451178|DEPARTMENT|2|82|Radical, royal states understand officially busy women; as a whole given |bi-annual| +191|AAAAAAAAPLAAAAAA|2450997|2451178|DEPARTMENT|2|83|Serious, intelligent tories might disappoint indeed; national purposes c|bi-annual| +192|AAAAAAAAAMAAAAAA|2450997|2451178|||84||| +193|AAAAAAAABMAAAAAA|2450997|2451178|DEPARTMENT|2|85|Impressive, quiet aspects must afford colourful, legal we|bi-annual| +194|AAAAAAAACMAAAAAA|2450997|2451178|DEPARTMENT|2|86|Written numbers will catch later long, metropolitan conditions. Clin|bi-annual| +195|AAAAAAAADMAAAAAA|2450997|2451178|DEPARTMENT|2|87|Very, main houses should play magnetic prices. However aware thanks might not choose so with|bi-annual| +196|AAAAAAAAEMAAAAAA|2450997|2451178|DEPARTMENT|2|88|Effects change easy, real types; young, corporate |bi-annual| +197|AAAAAAAAFMAAAAAA|2450997|2451178|DEPARTMENT|2|89|Industrial, usual others could make here in order particular firms. Natural patients li|bi-annual| +198|AAAAAAAAGMAAAAAA|2450997|2451178|DEPARTMENT|2|90|Great, extra affairs travel therefore foreign, alone gover|bi-annual| +199|AAAAAAAAHMAAAAAA|2450997|2451178|DEPARTMENT|2|91|Pure, outside shops give; public schools get most. Twice abl|bi-annual| +200|AAAAAAAAIMAAAAAA|2450997|2451178|DEPARTMENT|2|92|Very parliamentary reports should not stay now other general defences.|bi-annual| +201|AAAAAAAAJMAAAAAA|2450997|2451178|DEPARTMENT|2|93|Different sites begin surely. Numbers interview yet|bi-annual| +202|AAAAAAAAKMAAAAAA|2450997|2451178|DEPARTMENT|2|94|Leaders reach different trusts. Enormous, long detai|bi-annual| +203|AAAAAAAALMAAAAAA|2450997|2451178|DEPARTMENT|2|95|Appeals would not happen perfect, able individuals. Then other|bi-annual| +204|AAAAAAAAMMAAAAAA|2450997|2451178|DEPARTMENT|2|96|Small legs might accept political men. Accounts used to worry notably intimate, able sentences. Thi|bi-annual| +205|AAAAAAAANMAAAAAA|2450997|2451178|DEPARTMENT|2|97|Responsible, american families shall train on the matters. Changes ought to like book|bi-annual| +206|AAAAAAAAOMAAAAAA|2450997|2451178|DEPARTMENT|2|98|Critical, clear types use badly only sources. Independent interviews com|bi-annual| +207|AAAAAAAAPMAAAAAA|2450997|2451178|DEPARTMENT|2|99|Now political tools could cope together in a hours. Old,|bi-annual| +208|AAAAAAAAANAAAAAA|2450997|2451178|DEPARTMENT|2|100|New applications win much. Important, actual humans will n|bi-annual| +209|AAAAAAAABNAAAAAA|2450997|2451178|DEPARTMENT|2|101|Crowds could apply then women. Remote, national experiments|bi-annual| +210|AAAAAAAACNAAAAAA|2450997|2451178|DEPARTMENT|2|102|Required, small months will become officially level, domestic agreements. Animals can continue|bi-annual| +211|AAAAAAAADNAAAAAA|2450997|2451178|DEPARTMENT|2|103|Numerous changes gain only, similar factors. Mountains lan|bi-annual| +212|AAAAAAAAENAAAAAA|2450997|2451178|DEPARTMENT|2|104|Usually available levels investigate more white cats. Previous differences wan|bi-annual| +213|AAAAAAAAFNAAAAAA|2450997|2451178|DEPARTMENT|2|105|Highest good others might not go electronic differe|bi-annual| +214|AAAAAAAAGNAAAAAA|2450997|2451178|DEPARTMENT|2|106|Years would take relatively crucial, black hands. Large |bi-annual| +215|AAAAAAAAHNAAAAAA|2450997|2451178|DEPARTMENT|2|107|Actual matters can happen willing markets. Glad, soft lines refine a|bi-annual| +216|AAAAAAAAINAAAAAA|2450997|2451178|DEPARTMENT|2|108|Enough dark members read pupils. Things must afford just in a areas. English statements b|bi-annual| +217|AAAAAAAAJNAAAAAA|2450815|2450905|DEPARTMENT|3|1|Junior, select books savour alone aware, other colleagues. Men help ordinary, efficient bro|quarterly| +218|AAAAAAAAKNAAAAAA|2450815|2450905|DEPARTMENT|3|2|Hot, nice others will prove now huge, criminal subject|quarterly| +219|AAAAAAAALNAAAAAA|2450815|2450905|DEPARTMENT|3|3|Aims should not pay bitter speakers. Quite independent measures learn ago themes. High, t|quarterly| +220|AAAAAAAAMNAAAAAA|2450815|2450905|DEPARTMENT|3|4|Upstairs informal shares can repeat very thick, commercial reasons; handsome values will not|quarterly| +221|AAAAAAAANNAAAAAA|2450815|2450905|DEPARTMENT|3|5|Easy bars contract completely more reliable produc|quarterly| +222|AAAAAAAAONAAAAAA|2450815|2450905|DEPARTMENT|3|6|Present, canadian sites regulate yet corporate organisations; ol|quarterly| +223|AAAAAAAAPNAAAAAA|2450815|2450905|DEPARTMENT|3|7|Practical incomes might help over no goods. Actual|quarterly| +224|AAAAAAAAAOAAAAAA|2450815|2450905|DEPARTMENT|3|8|Heavy conditions can consult in a members. Different, early opponents shall sell nonethel|quarterly| +225|AAAAAAAABOAAAAAA|2450815|2450905|DEPARTMENT|3|9|Small, high proposals widen constant, successful sales|quarterly| +226|AAAAAAAACOAAAAAA|2450815|2450905|DEPARTMENT|3|10|Never minimum terms illustrate again extra, advers|quarterly| +227|AAAAAAAADOAAAAAA|2450815|2450905|DEPARTMENT|3|11|Beautiful, high years set only. Again other clients live fort|quarterly| +228|AAAAAAAAEOAAAAAA|2450815|2450905|DEPARTMENT|3|12|Inc, international purposes might ease hard, good products. Urban objects may not|quarterly| +229|AAAAAAAAFOAAAAAA|2450815|2450905|DEPARTMENT|3|13|Services cannot charge just earlier little songs. Miles sh|quarterly| +230|AAAAAAAAGOAAAAAA|2450815|2450905|DEPARTMENT|3|14|Human, existing words might sense forward birds. Large, original fruits will ref|quarterly| +231|AAAAAAAAHOAAAAAA|2450815|2450905|DEPARTMENT|3|15|Subject, major cells ensure. Normal signs may verify tomorrow pink situations. Reports prep|quarterly| +232|AAAAAAAAIOAAAAAA|2450815|2450905|DEPARTMENT|3|16|Years would not give extra criteria. Useful, part-time systems stress only normal rebels. Possib|quarterly| +233|AAAAAAAAJOAAAAAA|2450815|2450905|DEPARTMENT|3|17|At once economic minutes ought to break different members. Enough level roots think behind |quarterly| +234|AAAAAAAAKOAAAAAA|2450815|2450905|DEPARTMENT|3|18|So total measures take. Enough main symptoms would expect social changes. Only huge reader|quarterly| +235|AAAAAAAALOAAAAAA|2450815|2450905|DEPARTMENT|3|19|Contemporary types would not take therefore applicants. Full pho|quarterly| +236|AAAAAAAAMOAAAAAA|2450815|2450905|DEPARTMENT|3|20|About possible orders earn more doubtful companies. Afraid machines get gardens. Then external res|quarterly| +237|AAAAAAAANOAAAAAA|2450815|2450905|DEPARTMENT|3|21|Wide developments see against the children. Characters happen there defensiv|quarterly| +238|AAAAAAAAOOAAAAAA|2450815|2450905|DEPARTMENT|3|22|Approaches used to approve in a partners. Final allowances ou|quarterly| +239|AAAAAAAAPOAAAAAA|2450815|2450905|DEPARTMENT|3|23|Foreign depths offer hardly bad plants. More subsequent pupils give otherwise hands. Etc econ|quarterly| +240|AAAAAAAAAPAAAAAA|2450815|2450905|DEPARTMENT|3|24|Yesterday formal reforms commend immediate errors. Hard others fathom only. Much other|quarterly| +241|AAAAAAAABPAAAAAA|2450815|2450905|DEPARTMENT|3|25|Normal, particular sources may see for a heads. Deep, sophisticated lines |quarterly| +242|AAAAAAAACPAAAAAA|2450815|2450905|DEPARTMENT|3|26|Over large colleges will find in a days. Young aspects hang minutes. External, nort|quarterly| +243|AAAAAAAADPAAAAAA|2450815|2450905|DEPARTMENT|3|27|Only open police would support anything but there excessive machi|quarterly| +244|AAAAAAAAEPAAAAAA|2450815|2450905|DEPARTMENT|3|28|Genuine, long officials compete; cases should submit before years. Different, a|quarterly| +245|AAAAAAAAFPAAAAAA|2450815|2450905|DEPARTMENT|3|29|Successful visitors will create perhaps fresh colleagues. Important opinions could not |quarterly| +246|AAAAAAAAGPAAAAAA|2450815|2450905|DEPARTMENT|3|30|Green years will know unemployed, double adults. Year|quarterly| +247|AAAAAAAAHPAAAAAA|2450815|2450905|DEPARTMENT|3|31|Times force too times; suddenly big waves write more today |quarterly| +248|AAAAAAAAIPAAAAAA|2450815|2450905|DEPARTMENT|3|32|Studies shall not make about a players; mean, social rules |quarterly| +249|AAAAAAAAJPAAAAAA|2450815|2450905|DEPARTMENT|3|33|Often quiet beings might not remain yet. Public activi|quarterly| +250|AAAAAAAAKPAAAAAA|2450815|2450905|DEPARTMENT|3|34|Great, absent sources may listen historically; empty mines shall stand now ridiculous thousan|quarterly| +251|AAAAAAAALPAAAAAA|2450815|2450905|DEPARTMENT|3|35|Electronic, other heads pay now small daughters. Liable companies give s|quarterly| +252|AAAAAAAAMPAAAAAA|2450815|2450905|DEPARTMENT|3|36|As senior winners allocate poor, left sessions. Small ages ought to ho|quarterly| +253|AAAAAAAANPAAAAAA|2450815|2450905|DEPARTMENT|3|37|Seriously theoretical variables occur more interested readers. Plans trace possibly. |quarterly| +254|AAAAAAAAOPAAAAAA|2450815|2450905|DEPARTMENT|3|38|Particular, key rights take else practical, local telecommunications. Co|quarterly| +255|AAAAAAAAPPAAAAAA|2450815|2450905|DEPARTMENT|3|39|Criminal, military difficulties add possibly domestic sounds. Rather major f|quarterly| +256|AAAAAAAAAABAAAAA|2450815|2450905|DEPARTMENT|3|40|Lines will govern late immediate, sole practices. Urban force|quarterly| +257|AAAAAAAABABAAAAA|2450815|2450905|DEPARTMENT|3|41|More welsh reservations pay. Now total eyes affect. Years ought to plan possibly inland|quarterly| +258|AAAAAAAACABAAAAA|2450815|2450905|DEPARTMENT|3|42|Just other subjects want years. Major players might wipe for an sources. Things go |quarterly| +259|AAAAAAAADABAAAAA|2450815|2450905|DEPARTMENT|3|43|Workers might introduce existing, public matters. Else supr|quarterly| +260|AAAAAAAAEABAAAAA|2450815|2450905|DEPARTMENT|3|44|Net courts cover just gastric jobs; difficult, other occasions claim only th|quarterly| +261|AAAAAAAAFABAAAAA|2450815|2450905|DEPARTMENT|3|45|Flights ought to remember serious walls. Arab years know totally final expens|quarterly| +262|AAAAAAAAGABAAAAA|2450815|2450905|DEPARTMENT|3|46|Sometimes shy relations include very to a mistakes. Just new commen|quarterly| +263|AAAAAAAAHABAAAAA|2450815|2450905|DEPARTMENT|3|47|Orders shall thank cars; electoral ties say others. Co|quarterly| +264|AAAAAAAAIABAAAAA|2450815|2450905|DEPARTMENT|3|48|International rebels would make very from a movemen|quarterly| +265|AAAAAAAAJABAAAAA|2450815|2450905|DEPARTMENT|3|49|Difficult features might walk. Further important fat|quarterly| +266|AAAAAAAAKABAAAAA|2450815|2450905|DEPARTMENT|3|50|Regular, cheap weekends could stay even new figures;|quarterly| +267|AAAAAAAALABAAAAA|2450815|2450905|DEPARTMENT|3|51|Soon old policies will find facilities. Fiscal, other words could put particularly a|quarterly| +268|AAAAAAAAMABAAAAA|2450815|2450905|DEPARTMENT|3|52|Industrial, head times force then so other councils. Days find qualities. Impossible|quarterly| +269|AAAAAAAANABAAAAA|2450815|2450905|DEPARTMENT|3|53|Useful, accurate details suggest at last black suggestions.|quarterly| +270|AAAAAAAAOABAAAAA|2450815|2450905|DEPARTMENT|3|54|Then central police depend also married rates. Actual times cannot come never|quarterly| +271|AAAAAAAAPABAAAAA|2450815|2450905|DEPARTMENT|3|55|Black, other times shall sanction therefore pale odds. Actual, gradual costs vote to a sets. Unus|quarterly| +272|AAAAAAAAABBAAAAA|2450815|2450905|DEPARTMENT|3|56|Constantly particular workers expose away particular days. Much dry|quarterly| +273|AAAAAAAABBBAAAAA|2450815|2450905|DEPARTMENT|3|57|Standard schools might list there minor visitors. Privileges could see so little parent|quarterly| +274|AAAAAAAACBBAAAAA|2450815|2450905|DEPARTMENT|3|58|Stories take young, far degrees. Small, new instruments could consult|quarterly| +275|AAAAAAAADBBAAAAA|2450815|2450905|DEPARTMENT|3|59|Words see now apparent, interested things. Occupational, spare children would keep furth|quarterly| +276|AAAAAAAAEBBAAAAA|2450815|2450905|DEPARTMENT|3|60|Molecular practices work in a hours. Genes should obtain as|quarterly| +277|AAAAAAAAFBBAAAAA|2450815|2450905|DEPARTMENT|3|61|Tests shall not feel heavily lessons; fun, sure women throw through a comments. Legitimate, new|quarterly| +278|AAAAAAAAGBBAAAAA|2450815|2450905|DEPARTMENT|3|62|Eastern, other customers appear ideally young rights. Public funds re|quarterly| +279|AAAAAAAAHBBAAAAA|2450815|2450905|DEPARTMENT|3|63|Single, final rules can clear victims. International, current speakers ought to show |quarterly| +280|AAAAAAAAIBBAAAAA|2450815|2450905|DEPARTMENT|3|64|Highly social eyes roll at a funds. Elections might remember by a parents; political damage|quarterly| +281|AAAAAAAAJBBAAAAA|2450815|2450905|DEPARTMENT|3|65|More unknown sisters snap then during a attitudes. Instead pat|quarterly| +282|AAAAAAAAKBBAAAAA|2450815|2450905|DEPARTMENT|3|66|Demands raise much key friends; top gardens will call yesterday likely measures. H|quarterly| +283|AAAAAAAALBBAAAAA|2450815|2450905|DEPARTMENT|3|67|Close new characters imagine voluntary lines; sorry, coastal unions preserve very difficu|quarterly| +284|AAAAAAAAMBBAAAAA|2450815|2450905|DEPARTMENT|3|68|Certainly good shares ought to make eyes. Easy, important sh|quarterly| +285|AAAAAAAANBBAAAAA|2450815|2450905|DEPARTMENT|3|69|Romantic, royal officers explain then low police. Injuries need immediately before an details. |quarterly| +286|AAAAAAAAOBBAAAAA|2450815|2450905|DEPARTMENT|3|70|Developments should ensure strong, special ideas. Possi|quarterly| +287|AAAAAAAAPBBAAAAA|2450815|2450905|DEPARTMENT|3|71|Both main papers improve too wise cars. Other rooms believe even. Years should go |quarterly| +288|AAAAAAAAACBAAAAA|2450815|2450905|DEPARTMENT|3|72|Central, other decisions improve really systems. Glad, environmental miles will help as concerned,|quarterly| +289|AAAAAAAABCBAAAAA|2450815|2450905|DEPARTMENT|3|73|Educational communities provide again responsible, fresh things. Clear, technical remark|quarterly| +290|AAAAAAAACCBAAAAA|2450815|2450905|DEPARTMENT|3|74|Previously surprising lives think still for a cuts; only substances plan every|quarterly| +291|AAAAAAAADCBAAAAA|2450815|2450905|DEPARTMENT|3|75|Japanese, possible months could cover men. Brave, lengthy farms speak hitherto in the sales. Spe|quarterly| +292|AAAAAAAAECBAAAAA|2450815|2450905|DEPARTMENT|3|76|Allies will encourage else issues; old clothes will rest races. |quarterly| +293|AAAAAAAAFCBAAAAA|2450815|2450905|DEPARTMENT|3|77|Mysterious cases must not send specifically links. Rights would not come briefly men. Free police |quarterly| +294|AAAAAAAAGCBAAAAA|2450815|2450905|DEPARTMENT|3|78|Other methods would afford walls. Certainly social illustrations say all|quarterly| +295|AAAAAAAAHCBAAAAA|2450815|2450905|DEPARTMENT|3|79|Stories inspire more natural, major individuals; white, other buildings will compete at|quarterly| +296|AAAAAAAAICBAAAAA|2450815|2450905|DEPARTMENT|3|80|Conventional, easy windows know to a rights; conveniently new |quarterly| +297|AAAAAAAAJCBAAAAA|2450815|2450905|DEPARTMENT|3|81|Able ideas will not endorse. Also wrong cases practise. Again sl|quarterly| +298|AAAAAAAAKCBAAAAA|2450815|2450905|DEPARTMENT|3|82|Various flowers take. Just genetic machines should not suppose slightly full, significant |quarterly| +299|AAAAAAAALCBAAAAA|2450815|2450905|DEPARTMENT|3|83|Dramatic, old forms work there to a windows. Local, professional rights may not follow very; comm|quarterly| +300|AAAAAAAAMCBAAAAA|2450815|2450905|DEPARTMENT|3|84|Academic, complex boys will not look with a others; yesterday southern powers can repla|quarterly| +301|AAAAAAAANCBAAAAA|2450815|2450905|DEPARTMENT|3|85|Yet urban officers pop now; local proposals want really since a service|quarterly| +302|AAAAAAAAOCBAAAAA|2450815|2450905|DEPARTMENT|3|86|Minutes think cultural changes. Homes know so in the tha|quarterly| +303|AAAAAAAAPCBAAAAA|2450815|2450905|DEPARTMENT|3|87|Resources cannot entertain guilty, interesting others. Silly, ke|quarterly| +304|AAAAAAAAADBAAAAA|2450815|2450905|DEPARTMENT|3|88|True, local pressures rectify merely possible, severe pressur|quarterly| +305|AAAAAAAABDBAAAAA|2450815|2450905|DEPARTMENT|3|89|Stairs begin spirits. Functions pick. Holders shall remove. Other, ge|quarterly| +306|AAAAAAAACDBAAAAA|2450815|2450905|DEPARTMENT|3|90|Young, minute patterns order on a interviews. Main, expensive others would |quarterly| +307|AAAAAAAADDBAAAAA|2450815|2450905|DEPARTMENT|3|91|Conflicts return doubts. Video-taped, powerful effects will make yesterday there chinese walls; al|quarterly| +308|AAAAAAAAEDBAAAAA|2450815|2450905|DEPARTMENT|3|92|Italian, uncertain services deal valid powers. Moving earnings might se|quarterly| +309|AAAAAAAAFDBAAAAA|2450815|2450905|DEPARTMENT|3|93|So interesting agencies come slightly certain cells. Temporary, particular years used to say |quarterly| +310|AAAAAAAAGDBAAAAA|2450815|2450905|DEPARTMENT|3|94|Unacceptable, inadequate plants seek so much as angry, |quarterly| +311|AAAAAAAAHDBAAAAA|2450815|2450905|DEPARTMENT|3|95|Very likely days notice parts; lands detect simply into the exchanges. Conditions|quarterly| +312|AAAAAAAAIDBAAAAA|2450815|2450905|DEPARTMENT|3|96|Large industries know apparent devices; architects disturb here during the|quarterly| +313|AAAAAAAAJDBAAAAA|2450815|2450905|DEPARTMENT|3|97|Basic reasons ought to take perhaps. Indian, important paramet|quarterly| +314|AAAAAAAAKDBAAAAA|2450815|2450905|DEPARTMENT|3|98|Cases pay then high, long-term experiments; just russian reasons might no|quarterly| +315|AAAAAAAALDBAAAAA|2450815|2450905|DEPARTMENT|3|99|Things might expect near individual companies. Already private facilities give again|quarterly| +316|AAAAAAAAMDBAAAAA|2450815|2450905|DEPARTMENT|3|100|Names would see on a obligations. Much major degrees develop other, other servants. Western, old s|quarterly| +317|AAAAAAAANDBAAAAA|2450815|2450905|DEPARTMENT|3|101|Working-class, right years shall spend also new wings. Key, financial publisher|quarterly| +318|AAAAAAAAODBAAAAA|2450815|2450905|DEPARTMENT|3|102|Traditional, ridiculous women shall signal well both overall resources. National, old st|quarterly| +319|AAAAAAAAPDBAAAAA|2450815|2450905|DEPARTMENT|3|103|Less than tired kinds go on the differences. Social |quarterly| +320|AAAAAAAAAEBAAAAA|2450815|2450905|DEPARTMENT|3|104|Overall, supposed others cannot ensure considerably in the con|quarterly| +321|AAAAAAAABEBAAAAA|2450815|2450905|DEPARTMENT|3|105|Other rules might clinch indeed on the opportunities. Left|quarterly| +322|AAAAAAAACEBAAAAA|2450815|2450905|DEPARTMENT|3|106|Also other rises take so major, main cities. Resources sh|quarterly| +323|AAAAAAAADEBAAAAA|2450815|2450905|DEPARTMENT|3|107|Exactly british regions import never british, capable boys. Ot|quarterly| +324|AAAAAAAAEEBAAAAA|2450815|2450905|DEPARTMENT|3|108|Then future payments need already. Remarkable, coming researchers may double afte|quarterly| +325|AAAAAAAAFEBAAAAA|2450906|2450996|DEPARTMENT|4|1|Needs condemn absolute participants. International protests shall not go very in a d|quarterly| +326|AAAAAAAAGEBAAAAA|2450906|2450996|DEPARTMENT|4|2|Quickly horrible police will not quit topics. Texts may wait black surfaces. Again remarkabl|quarterly| +327|AAAAAAAAHEBAAAAA|2450906|2450996|DEPARTMENT|4|3|Now large clouds will care. Functional hands confront in a times. Detailed, french atti|quarterly| +328|AAAAAAAAIEBAAAAA|2450906|2450996|DEPARTMENT|4|4|Very smooth birds would not put; sweet, lively visits would ascertain specialist members. Well |quarterly| +329|AAAAAAAAJEBAAAAA|2450906|2450996|DEPARTMENT|4|5|Tough thanks ought to settle for example in a women. Famous, working motives lie we|quarterly| +330|AAAAAAAAKEBAAAAA|2450906|2450996|DEPARTMENT|4|6|New, firm groups retire firmly alone, front artists; either |quarterly| +331|AAAAAAAALEBAAAAA|2450906|2450996|DEPARTMENT|4|7|Children disregard other years. Mainly other poems must indulge ultima|quarterly| +332|AAAAAAAAMEBAAAAA|2450906|2450996|DEPARTMENT|4|8|Natural secrets go purely there public jobs; hours run accordingly. Forcefully open others sho|quarterly| +333|AAAAAAAANEBAAAAA|2450906|2450996|DEPARTMENT|4|9|Questions may stress soon never married parents; apar|quarterly| +334|AAAAAAAAOEBAAAAA|2450906|2450996|DEPARTMENT|4|10|Widely widespread sides sit letters. More little representatives could|quarterly| +335|AAAAAAAAPEBAAAAA|2450906|2450996|DEPARTMENT|4|11|Prices used to reach usually existing, lovely children. Rights talk too contempora|quarterly| +336|AAAAAAAAAFBAAAAA|2450906|2450996|DEPARTMENT|4|12|British, base rocks know recently late, gradual shares. Either jo|quarterly| +337|AAAAAAAABFBAAAAA|2450906|2450996|DEPARTMENT|4|13|Areas say finally extra factors. Similar designs forget players. Courses used to|quarterly| +338|AAAAAAAACFBAAAAA|2450906|2450996|DEPARTMENT|4|14|Interests facilitate true members; further small police make perfectly s|quarterly| +339|AAAAAAAADFBAAAAA|2450906|2450996|DEPARTMENT|4|15|Rather main years must not justify apparently new, good centres. Secretly com|quarterly| +340|AAAAAAAAEFBAAAAA|2450906|2450996|DEPARTMENT|4|16|Big, unique ambitions could fall today small bones; visitors would not die nice, standard sol|quarterly| +341|AAAAAAAAFFBAAAAA|2450906|2450996|DEPARTMENT|4|17|More massive quantities should advise. Certain schools may not evade; desperate, precise hund|quarterly| +342|AAAAAAAAGFBAAAAA|2450906|2450996|DEPARTMENT|4|18|Past, dark officers shall pay obligations. Elderly affairs remember other, irish letter|quarterly| +343|AAAAAAAAHFBAAAAA|2450906|2450996|DEPARTMENT|4|19|Months must provide here continuous requirements. Conscious,|quarterly| +344|AAAAAAAAIFBAAAAA|2450906|2450996|DEPARTMENT|4|20|Pleased instructions will know rich candidates. Uncomfort|quarterly| +345|AAAAAAAAJFBAAAAA|2450906|2450996|DEPARTMENT|4|21|Colours tackle meals; large, possible campaigns must head|quarterly| +346|AAAAAAAAKFBAAAAA|2450906|2450996|DEPARTMENT|4|22|Equal, only changes must know. Large things increase |quarterly| +347|AAAAAAAALFBAAAAA|2450906|2450996|DEPARTMENT|4|23|Simple, human teeth help. Natural friends explain features; always british difference|quarterly| +348|AAAAAAAAMFBAAAAA|2450906|2450996|DEPARTMENT|4|24|As well terrible conditions select big, basic goods. Questions prov|quarterly| +349|AAAAAAAANFBAAAAA|2450906|2450996|DEPARTMENT|4|25|Again social efforts divide also different, low requirements. Good, aware limits could pay |quarterly| +350|AAAAAAAAOFBAAAAA|2450906|2450996|DEPARTMENT|4|26|There public results know patients. So ancient veg|quarterly| +351|AAAAAAAAPFBAAAAA|2450906|2450996|DEPARTMENT|4|27|Dead servants must control general versions. More alone principles use sympathetic, g|quarterly| +352|AAAAAAAAAGBAAAAA|2450906|2450996|DEPARTMENT|4|28|Minute millions may know much new, particular farmers. Current |quarterly| +353|AAAAAAAABGBAAAAA|2450906|2450996|DEPARTMENT|4|29|Angry, primary demands know neither either little police; away african hills know once. Average|quarterly| +354|AAAAAAAACGBAAAAA|2450906|2450996|DEPARTMENT|4|30|Games notice german, internal officers. Financial students will not steal private places. Re|quarterly| +355|AAAAAAAADGBAAAAA|2450906|2450996|DEPARTMENT|4|31|Texts shall hide in the others. Small changes take usually total, detailed films. O|quarterly| +356|AAAAAAAAEGBAAAAA|2450906|2450996|DEPARTMENT|4|32|Nuclear workers must conjure. Ideas can acquire now new, assistant models. Files marry. S|quarterly| +357|AAAAAAAAFGBAAAAA|2450906|2450996|DEPARTMENT|4|33|Attitudes might disregard later various, big questions. Relevant, short walls eat l|quarterly| +358|AAAAAAAAGGBAAAAA|2450906|2450996|DEPARTMENT|4|34|Once other terms can encourage new ways. However other tables play |quarterly| +359|AAAAAAAAHGBAAAAA|2450906|2450996|DEPARTMENT|4|35|Still high factories like only. Perfect, public newspapers must not know about with a visito|quarterly| +360|AAAAAAAAIGBAAAAA|2450906|2450996|DEPARTMENT|4|36|Opportunities must not prevent much so continuing qualifications. Nuclear towns might find also c|quarterly| +361|AAAAAAAAJGBAAAAA|2450906|2450996|DEPARTMENT|4|37|Other fingers talk all forms; just french studies must not put current|quarterly| +362|AAAAAAAAKGBAAAAA|2450906|2450996|DEPARTMENT|4|38|Old, whole men could not prosper somehow gay times. As it were new years turn so total, likely phen|quarterly| +363|AAAAAAAALGBAAAAA|2450906|2450996|DEPARTMENT|4|39|Less relevant producers should get together earlier political babies. Diff|quarterly| +364|AAAAAAAAMGBAAAAA|2450906|2450996|DEPARTMENT|4|40|Important pictures ought to wear managers. Details consider all considerable part|quarterly| +365|AAAAAAAANGBAAAAA||||4|41||quarterly| +366|AAAAAAAAOGBAAAAA|2450906|2450996|DEPARTMENT|4|42|Leading, rare models could know clubs. Elegant, proposed organisers can break little the|quarterly| +367|AAAAAAAAPGBAAAAA|2450906|2450996|DEPARTMENT|4|43|Good, valid hours shall not get. Other costs may point by now tight days. British, social int|quarterly| +368|AAAAAAAAAHBAAAAA|2450906|2450996|DEPARTMENT|4|44|Indeed basic shares get then by a years. Never dramatic hands s|quarterly| +369|AAAAAAAABHBAAAAA|2450906|2450996|DEPARTMENT|4|45|Slow sides arrange most so internal models. Just necessary men could say|quarterly| +370|AAAAAAAACHBAAAAA|2450906|2450996|DEPARTMENT|4|46|Altogether great conditions may open else available, foreign fans. Pr|quarterly| +371|AAAAAAAADHBAAAAA|2450906|2450996|DEPARTMENT|4|47|Already wide costs support different, silver books. Powerful seats must feel now patient chi|quarterly| +372|AAAAAAAAEHBAAAAA|2450906|2450996|DEPARTMENT|4|48|Good cells could open at a areas. Organizations square perfectly parental users; soviet years|quarterly| +373|AAAAAAAAFHBAAAAA|2450906|2450996|DEPARTMENT|4|49|Men concentrate finally major owners. Effects would relax new years|quarterly| +374|AAAAAAAAGHBAAAAA|2450906|2450996|DEPARTMENT|4|50|Quietly revolutionary commentators establish never nec|quarterly| +375|AAAAAAAAHHBAAAAA|2450906|2450996|DEPARTMENT|4|51|Widely human strengths must not call notably obvious, clinic|quarterly| +376|AAAAAAAAIHBAAAAA|2450906|2450996|DEPARTMENT|4|52|Then necessary sanctions should respond. Old houses|quarterly| +377|AAAAAAAAJHBAAAAA|2450906|2450996|DEPARTMENT|4|53|Routine users may miss on a problems. Just general seconds involve high, german jeans. Visit|quarterly| +378|AAAAAAAAKHBAAAAA|2450906|2450996|DEPARTMENT|4|54|Years shield however most formal hundreds. Men ought t|quarterly| +379|AAAAAAAALHBAAAAA|2450906|2450996|DEPARTMENT|4|55|Special demands stimulate today for a companies. Very nice factors would work far from the te|quarterly| +380|AAAAAAAAMHBAAAAA|2450906|2450996|DEPARTMENT|4|56|Willing managers run. Clear gifts suit weeks. Similar services remove then best everyday s|quarterly| +381|AAAAAAAANHBAAAAA|2450906|2450996|DEPARTMENT|4|57|British issues talk new, lengthy teachers. British, economic categ|quarterly| +382|AAAAAAAAOHBAAAAA|2450906|2450996|DEPARTMENT|4|58|Additional connections ought to take distinct, ready bodies. Sad, likely boo|quarterly| +383|AAAAAAAAPHBAAAAA|2450906|2450996|DEPARTMENT|4|||quarterly| +384|AAAAAAAAAIBAAAAA|2450906|2450996|DEPARTMENT|4|60|Additional parts vary deaf features. Other beans equip poorly. Others should take about dif|quarterly| +385|AAAAAAAABIBAAAAA|2450906|2450996|DEPARTMENT|4|61|Misleading, damp places should not obtain ruling, particular assets. Bills ought to find widely fo|quarterly| +386|AAAAAAAACIBAAAAA|2450906|2450996|DEPARTMENT|4|62|Direct women carry even fascinating women. Little companies rely afterwards. Clear, single fut|quarterly| +387|AAAAAAAADIBAAAAA|2450906|2450996|DEPARTMENT|4|63|Alone contemporary eyes might lose nowhere unexpected recent principles. Very ident|quarterly| +388|AAAAAAAAEIBAAAAA|2450906|2450996|DEPARTMENT|4|64|Movements used to cure potatoes. Active shapes may not come practi|quarterly| +389|AAAAAAAAFIBAAAAA|2450906|2450996|DEPARTMENT|4|65|Historical, unaware days skip trends. Dramatic, fine circumstances see everywhere times. Les|quarterly| +390|AAAAAAAAGIBAAAAA|2450906|2450996|DEPARTMENT|4|66|Skills used to sweep reasonably smooth, other members. Open disputes mi|quarterly| +391|AAAAAAAAHIBAAAAA|2450906|2450996|DEPARTMENT|4|67|Largely liable principles ensure largely. Just polite|quarterly| +392|AAAAAAAAIIBAAAAA|2450906|2450996|DEPARTMENT|4|68|Leading, scientific police should not apply. Disputes throw as. Royal, reasonable weapons get|quarterly| +393|AAAAAAAAJIBAAAAA|2450906|2450996|DEPARTMENT|4|69|Subsequently total leaders exclude better. Strong, brief scientis|quarterly| +394|AAAAAAAAKIBAAAAA|2450906|2450996|DEPARTMENT|4|70|Slightly other students ought to tell sometimes relative partners. Restrictions might def|quarterly| +395|AAAAAAAALIBAAAAA|2450906|2450996|DEPARTMENT|4|71|Original, severe genes might find better; years would develop there exclusive, large|quarterly| +396|AAAAAAAAMIBAAAAA|2450906|2450996|DEPARTMENT|4|72|Different, expensive officers find more young, recent causes. Important schoo|quarterly| +397|AAAAAAAANIBAAAAA|2450906|2450996|DEPARTMENT|4|73|Forward african cases used to protect at a patients. Nice, french levels |quarterly| +398|AAAAAAAAOIBAAAAA|2450906|2450996|DEPARTMENT|4|74|Forests reconsider now with a views. Soft stairs know very words. Yet angry difficulties t|quarterly| +399|AAAAAAAAPIBAAAAA|2450906|2450996|DEPARTMENT|4|75|Various, other papers should emulate certainly psychiatric w|quarterly| +400|AAAAAAAAAJBAAAAA|2450906|2450996|DEPARTMENT|4|76|Old shops may meet upward grim others. Useful, important town|quarterly| +401|AAAAAAAABJBAAAAA|2450906|2450996|DEPARTMENT|4|77|Parents end courts. Feet might dream much still short difficulties. Famous schemes shal|quarterly| +402|AAAAAAAACJBAAAAA|2450906|2450996|DEPARTMENT|4|78|Detailed reforms might not see standards. Large contexts could un|quarterly| +403|AAAAAAAADJBAAAAA|2450906|2450996|DEPARTMENT|4|79|Scientists may try precisely political, difficult sta|quarterly| +404|AAAAAAAAEJBAAAAA|2450906|2450996|DEPARTMENT|4|80|Outdoor, complete agreements sit downstairs. Writers profit. Birds will eat as scientific f|quarterly| +405|AAAAAAAAFJBAAAAA|2450906|2450996|DEPARTMENT|4|81|Boards refrain later meetings. Capital events will not sue by |quarterly| +406|AAAAAAAAGJBAAAAA|2450906|2450996|DEPARTMENT|4|82|Large, social eyes provide criteria. Habits fulfil together sure insects. Simple|quarterly| +407|AAAAAAAAHJBAAAAA|2450906|2450996|DEPARTMENT|4|83|Parents would break often illegal seconds. Possible, standard services should not tell s|quarterly| +408|AAAAAAAAIJBAAAAA|2450906|2450996|DEPARTMENT|4|84|Most little genes can establish never quietly professional plants.|quarterly| +409|AAAAAAAAJJBAAAAA|2450906|2450996|DEPARTMENT|4|85|So certain lines can buy between a papers. Stages touch extra fortunately responsible cons|quarterly| +410|AAAAAAAAKJBAAAAA|2450906|2450996|DEPARTMENT|4|86|Really familiar funds park beautifully. Payments use professional guests; official minute|quarterly| +411|AAAAAAAALJBAAAAA|2450906|2450996|DEPARTMENT|4|87|Absolute, young officials may want movements; alone years would not deny in a|quarterly| +412|AAAAAAAAMJBAAAAA|2450906|2450996|DEPARTMENT|4|88|Almost big accidents could take less from a months; clearly individual hills should not put mo|quarterly| +413|AAAAAAAANJBAAAAA|2450906|2450996|DEPARTMENT|4|89|Alive hours might give attitudes. National kings result here regardless trying y|quarterly| +414|AAAAAAAAOJBAAAAA|2450906|2450996|DEPARTMENT|4|90|Things take. Jewish, ordinary symbols go only, past countries. Good, total|quarterly| +415|AAAAAAAAPJBAAAAA|2450906|2450996|DEPARTMENT|4|91|Rich, full members diminish in a amounts. Fresh months consider then inc hours. |quarterly| +416|AAAAAAAAAKBAAAAA|2450906|2450996|DEPARTMENT|4|92|Ultimate, different shares separate here independent thousands. Increasing tories say. Fair b|quarterly| +417|AAAAAAAABKBAAAAA|2450906|2450996|DEPARTMENT|4|93|Schools elicit narrowly important, international developments. Limited, |quarterly| +418|AAAAAAAACKBAAAAA|2450906|2450996|DEPARTMENT|4|94|Young, emotional men accelerate. Bad, whole changes might not remember groups. Away |quarterly| +419|AAAAAAAADKBAAAAA|2450906|2450996|DEPARTMENT|4|95|So possible newspapers win at the areas. Great comments|quarterly| +420|AAAAAAAAEKBAAAAA|2450906|2450996|DEPARTMENT|4|96|Since well-known laws shall follow now involved good teams. Various actions used to testify|quarterly| +421|AAAAAAAAFKBAAAAA|2450906|2450996|DEPARTMENT|4|97|Of course joint operations leave. Also so-called eggs cannot get subsequently. Already federa|quarterly| +422|AAAAAAAAGKBAAAAA|2450906|2450996|DEPARTMENT|4|98|Large years learn; already southern boundaries cannot tend really in a writers. Individual labora|quarterly| +423|AAAAAAAAHKBAAAAA|2450906|2450996|DEPARTMENT|4|99|Methods wave gold actions. There horizontal units acknowledge happy, supr|quarterly| +424|AAAAAAAAIKBAAAAA|2450906|2450996|DEPARTMENT|4|100|Games will imagine then. Disabled, young issues know weeks. Then correc|quarterly| +425|AAAAAAAAJKBAAAAA|2450906|2450996|DEPARTMENT|4|101|Strategic institutions should not wait from time to time. Again environme|quarterly| +426|AAAAAAAAKKBAAAAA|2450906|2450996|DEPARTMENT|4|102|Integrated, sorry techniques must kill then supreme responsibilities. Also criminal f|quarterly| +427|AAAAAAAALKBAAAAA|2450906|2450996|DEPARTMENT|4|103|Duties complete absolutely. Years create only cells; players |quarterly| +428|AAAAAAAAMKBAAAAA|2450906|2450996|DEPARTMENT|4|104|Ages must explain human arts. Hands could not help new, white marks. Speakers will |quarterly| +429|AAAAAAAANKBAAAAA|2450906|2450996|DEPARTMENT|4|105|False, irrelevant departments might touch over here short customers. Actions must make detai|quarterly| +430|AAAAAAAAOKBAAAAA|2450906|2450996|DEPARTMENT|4|106|Great schools swim women. Full lines eat thereby formal, annual feet. Co|quarterly| +431|AAAAAAAAPKBAAAAA|2450906|2450996|DEPARTMENT|4|107|Grey insects travel over there nearby lights. Areas limit public clubs|quarterly| +432|AAAAAAAAALBAAAAA|2450906|2450996|DEPARTMENT|4|108|Months must sort authorities. No longer english books |quarterly| +433|AAAAAAAABLBAAAAA|2450997|2451087|DEPARTMENT|5|1|Partially unfair visitors ought to say. Later british users say patterns. Nice benefi|quarterly| +434|AAAAAAAACLBAAAAA|2450997|2451087|DEPARTMENT|5|2|Unnecessary, appropriate eyes accept less special manufacture|quarterly| +435|AAAAAAAADLBAAAAA|2450997|2451087|DEPARTMENT|5|3|Very images leave less personal prices; states come in the numbe|quarterly| +436|AAAAAAAAELBAAAAA|2450997|2451087|DEPARTMENT|5|4|Old arguments can persuade in the guests; beautiful items live anyway to a s|quarterly| +437|AAAAAAAAFLBAAAAA|2450997|2451087|DEPARTMENT|5|5|Unchanged records go ibid. to the animals. Less natural americans should support n|quarterly| +438|AAAAAAAAGLBAAAAA|2450997|2451087|DEPARTMENT|5|6|Meanwhile scottish clergy shall use chief witnesses. Acceptable, new attitudes may consist well n|quarterly| +439|AAAAAAAAHLBAAAAA|2450997|2451087|DEPARTMENT|5|7|Here open positions press entirely social sides; surfaces must not change big times; significant pr|quarterly| +440|AAAAAAAAILBAAAAA|2450997|2451087|DEPARTMENT|5|8|Years might rest european, bad feelings. Ill elderly animals lead; branches breed still rare|quarterly| +441|AAAAAAAAJLBAAAAA|2450997|2451087|DEPARTMENT|5|9|Ago formal others ought to use enough sales. Victorian, acceptable services inform rather than a i|quarterly| +442|AAAAAAAAKLBAAAAA|2450997|2451087|DEPARTMENT|5|10|Good, interested resources must go effectively manufacturing e|quarterly| +443|AAAAAAAALLBAAAAA|2450997|2451087|DEPARTMENT|5|11|Very separate markets used to handle. Huge users contain economically golden circumst|quarterly| +444|AAAAAAAAMLBAAAAA|2450997|2451087|DEPARTMENT|5|12|German levels move already independent, british issues. Long estimates may succeed n|quarterly| +445|AAAAAAAANLBAAAAA|2450997|2451087|DEPARTMENT|5|13|Short, different experiments must not continue too sometimes academic directors. Civil services mi|quarterly| +446|AAAAAAAAOLBAAAAA|2450997|2451087|DEPARTMENT|5|14|Steep, spanish animals include. However formal elements find conditions. Li|quarterly| +447|AAAAAAAAPLBAAAAA|2450997|2451087|DEPARTMENT|5|15|National, white numbers look equally silly rules. Men cross bravely change|quarterly| +448|AAAAAAAAAMBAAAAA|2450997|2451087|DEPARTMENT|5|16|Liable standards realise far thousands. Local tests think officers.|quarterly| +449|AAAAAAAABMBAAAAA|2450997|2451087|DEPARTMENT|5|17|Positive, thick studies used to make now admirably good tensions. Thorough feet mu|quarterly| +450|AAAAAAAACMBAAAAA|2450997|2451087|DEPARTMENT|5|18|Common, capable elections stop birds. Streets negotiate especially flexible, educational ey|quarterly| +451|AAAAAAAADMBAAAAA|2450997|2451087|DEPARTMENT|5|19|Just racial hands should include in a ministers. Final, natural|quarterly| +452|AAAAAAAAEMBAAAAA|2450997|2451087|DEPARTMENT|5|20|Unacceptable, closed companies may estimate mainly schools. Languages|quarterly| +453|AAAAAAAAFMBAAAAA|2450997|2451087|DEPARTMENT|5|21|Birds might see just. Trials can carry for a months. Thu|quarterly| +454|AAAAAAAAGMBAAAAA|2450997|2451087|DEPARTMENT|5|22|As rapid degrees will not say applicable forces. Interested, sensitive y|quarterly| +455|AAAAAAAAHMBAAAAA|2450997|2451087|DEPARTMENT|5|23|Italian, retail weeks read probably things; suitable|quarterly| +456|AAAAAAAAIMBAAAAA|2450997|2451087|DEPARTMENT|5|24|Sexually nice parts must not pursue as for a services. Pupils know earlier f|quarterly| +457|AAAAAAAAJMBAAAAA||2451087|||25||| +458|AAAAAAAAKMBAAAAA|2450997|2451087|DEPARTMENT|5|26|Perfect, lacking months cannot remark; chief, various words shall not catch. Parts indi|quarterly| +459|AAAAAAAALMBAAAAA|2450997|2451087|DEPARTMENT|5|27|Soon very wages think more than; individual, single directions must write. Central, seriou|quarterly| +460|AAAAAAAAMMBAAAAA|2450997|2451087|DEPARTMENT|5|28|Apologetically old notes rise strategies. Difficult, sorr|quarterly| +461|AAAAAAAANMBAAAAA|2450997|2451087|DEPARTMENT|5|29|Major problems believe against a goals. Frequently just conditi|quarterly| +462|AAAAAAAAOMBAAAAA|2450997|2451087|DEPARTMENT|5|30|Major, other things can go all alternative duties; most ultimate duties should |quarterly| +463|AAAAAAAAPMBAAAAA|2450997|2451087|DEPARTMENT|5|31|Enough pure years may see. Traditional, primary periods prepare right blocks. Video-taped, new i|quarterly| +464|AAAAAAAAANBAAAAA|2450997|2451087|DEPARTMENT|5|32|Long rocks used to give only by a women; certain systems should a|quarterly| +465|AAAAAAAABNBAAAAA|2450997|2451087|DEPARTMENT|5|33|Levels should make steady other keys. Rules include for a ne|quarterly| +466|AAAAAAAACNBAAAAA|2450997|2451087|DEPARTMENT|5|34|Formal, minor telecommunications like usually previous libraries. Thoroughly growing tea|quarterly| +467|AAAAAAAADNBAAAAA|2450997|2451087|DEPARTMENT|5|35|In order other factors see rather. Open, local rates can make firmly anxious children|quarterly| +468|AAAAAAAAENBAAAAA|2450997|2451087|DEPARTMENT|5|36|Cells will donate phenomena. Too political changes eliminate now local priorities. Due minutes t|quarterly| +469|AAAAAAAAFNBAAAAA|2450997|2451087|DEPARTMENT|5|37|Just early sales can finish as normal years. Organisations should not begin yesterday like |quarterly| +470|AAAAAAAAGNBAAAAA|2450997|2451087|DEPARTMENT|5|38|Small, total improvements suit therefore. General products draw better regulations. General,|quarterly| +471|AAAAAAAAHNBAAAAA|2450997|2451087|DEPARTMENT|5|39|Children hold coherent boots; good cells come wonderful, |quarterly| +472|AAAAAAAAINBAAAAA|2450997|2451087|DEPARTMENT|5|40|Photographs will not strike especially. Then secret affairs apply then general, essential partners|quarterly| +473|AAAAAAAAJNBAAAAA|2450997|2451087|DEPARTMENT|5|41|More long-term members must deliver also to a strings. Tight federal am|quarterly| +474|AAAAAAAAKNBAAAAA|2450997|2451087|DEPARTMENT|5|42|Particular, junior sales appear merely firm, dead aspects. Increasin|quarterly| +475|AAAAAAAALNBAAAAA|2450997|2451087|DEPARTMENT|5|43|Private, fundamental heads might want please models. Little, dead cities would let legal ends|quarterly| +476|AAAAAAAAMNBAAAAA|2450997|2451087|DEPARTMENT|5|44|Effective sports see also future, primary schools. More certain beds shall not l|quarterly| +477|AAAAAAAANNBAAAAA|2450997|2451087|DEPARTMENT|5|45|Much entire animals get thus really colonial tourists. Hence good fee|quarterly| +478|AAAAAAAAONBAAAAA|2450997|2451087|DEPARTMENT|5|46|New, major problems should come too eyes. Theoretical, prep|quarterly| +479|AAAAAAAAPNBAAAAA|2450997|2451087|DEPARTMENT|5|47|Impossible, new cases ensure. Central priorities look |quarterly| +480|AAAAAAAAAOBAAAAA|2450997|2451087|DEPARTMENT|5|48|Experiences tend genuine, social friends. Useful skills meet readily d|quarterly| +481|AAAAAAAABOBAAAAA|2450997|2451087|DEPARTMENT|5|49|Fresh, large documents appear. Different, open wom|quarterly| +482|AAAAAAAACOBAAAAA|2450997|2451087|DEPARTMENT|5|50|Main motives used to correspond guilty numbers. Years cannot perceive then researchers. |quarterly| +483|AAAAAAAADOBAAAAA|2450997|2451087|DEPARTMENT|5|51|Dogs sell now incomes; normally aware men must prove ironically international eyes|quarterly| +484|AAAAAAAAEOBAAAAA|2450997|2451087|DEPARTMENT|5|52|And so on reliable rocks could accept details; established schemes put; officers would not w|quarterly| +485|AAAAAAAAFOBAAAAA|2450997|2451087|DEPARTMENT|5|53|About clear years take models. Local, fresh authorities caus|quarterly| +486|AAAAAAAAGOBAAAAA|2450997|2451087|DEPARTMENT|5|54|Practices attack so. Top ambitions would put now. Fiel|quarterly| +487|AAAAAAAAHOBAAAAA|2450997|2451087|DEPARTMENT|5|55|Days could study immediately wrong, perfect phenome|quarterly| +488|AAAAAAAAIOBAAAAA|2450997|2451087|DEPARTMENT|5|56|Other, practical men find perhaps. Heavily amazing police b|quarterly| +489|AAAAAAAAJOBAAAAA|2450997|2451087|DEPARTMENT|5|57|Also compatible cheeks come by the peasants. Affairs should not stand simple p|quarterly| +490|AAAAAAAAKOBAAAAA|2450997|2451087|DEPARTMENT|5|58|Other parties reap more public, detailed players. Good, difficult legs cannot sense|quarterly| +491|AAAAAAAALOBAAAAA|2450997|2451087|DEPARTMENT|5|59|Other, similar teachers should not get most during a questions. White |quarterly| +492|AAAAAAAAMOBAAAAA|2450997|2451087|DEPARTMENT|5|60|Rare, rich hopes shall not bring particularly yellow, vast democrats. Old, typical bene|quarterly| +493|AAAAAAAANOBAAAAA|2450997|2451087|DEPARTMENT|5|61|More able consequences shall not restore; meetings go toward|quarterly| +494|AAAAAAAAOOBAAAAA|2450997|2451087|DEPARTMENT|5|62|Separate, structural thousands let. Perhaps male instit|quarterly| +495|AAAAAAAAPOBAAAAA|2450997|2451087|DEPARTMENT|5|63|Fully free powers like suddenly slight bishops. Deeper naval enemies could |quarterly| +496|AAAAAAAAAPBAAAAA|2450997|2451087|DEPARTMENT|5|64|Copies must use social techniques; clear, careful hours could not make unreaso|quarterly| +497|AAAAAAAABPBAAAAA|2450997|2451087|DEPARTMENT|5|65|Novel, historical things would resist likely, fresh sections. |quarterly| +498|AAAAAAAACPBAAAAA|2450997|2451087|DEPARTMENT|5|66|Manufacturers see then funny, intimate leaders. Shows carry commercial|quarterly| +499|AAAAAAAADPBAAAAA|2450997|2451087|DEPARTMENT|5|67|Anxious, alone reactions must hear new, following eyes. Young t|quarterly| +500|AAAAAAAAEPBAAAAA|2450997|2451087|DEPARTMENT|5|68|Able, whole families cannot mean simply still legal workers. Extra, serious databases should lower|quarterly| +501|AAAAAAAAFPBAAAAA|2450997|2451087|DEPARTMENT|5|69|Dead years used to strive reluctantly cells. Perfect systems |quarterly| +502|AAAAAAAAGPBAAAAA|2450997|2451087|DEPARTMENT|5|70|Active, central chains remain quite available shelves.|quarterly| +503|AAAAAAAAHPBAAAAA|2450997|2451087|DEPARTMENT|5|71|Supporters could not comply elderly duties; increased, full |quarterly| +504|AAAAAAAAIPBAAAAA|2450997|2451087|DEPARTMENT|5|72|Most different sections resist in a cases. External, old c|quarterly| +505|AAAAAAAAJPBAAAAA|2450997|2451087|DEPARTMENT|5|73|Little, old patients confuse new, short men. Limitations admit thus months. Eve|quarterly| +506|AAAAAAAAKPBAAAAA|2450997|2451087|DEPARTMENT|5|74|Children feel carefully critical, initial measures. Crazy costs add prim|quarterly| +507|AAAAAAAALPBAAAAA|2450997|2451087|DEPARTMENT|5|75|Specific pp. indicate carefully; words must not sink also hard instructions|quarterly| +508|AAAAAAAAMPBAAAAA|2450997|2451087|DEPARTMENT|5|76|Main, different arrangements cannot argue only. Patients set problems. Sti|quarterly| +509|AAAAAAAANPBAAAAA|2450997|2451087|DEPARTMENT|5|77|Quite unable heroes could head never types. Easy, massive years migh|quarterly| +510|AAAAAAAAOPBAAAAA|2450997|2451087|DEPARTMENT|5|78|Typical, lovely groups conclude there in the books; m|quarterly| +511|AAAAAAAAPPBAAAAA|2450997|2451087|DEPARTMENT|5|79|Away social lips speed well poor games. Lives take in a homes. Names pers|quarterly| +512|AAAAAAAAAACAAAAA|2450997|2451087|DEPARTMENT|5|80|Seldom creative trials could not talk important, spiritual chips; odds join narrow, large mat|quarterly| +513|AAAAAAAABACAAAAA|2450997|2451087|DEPARTMENT|5|81|Points might matter here despite a ends. Cars think much labou|quarterly| +514|AAAAAAAACACAAAAA|2450997|2451087|DEPARTMENT|5|82|Exceptions can locate societies. Also certain years should happen act|quarterly| +515|AAAAAAAADACAAAAA|2450997|2451087|DEPARTMENT|5|83|Worthy feet engage clearly creative years. Hands will meet |quarterly| +516|AAAAAAAAEACAAAAA|2450997|2451087|DEPARTMENT|5|84|Individuals shall not help likely, free features. Gods may not assume much annual decades. F|quarterly| +517|AAAAAAAAFACAAAAA|2450997|2451087|DEPARTMENT|5|85|Victorian functions could keep however executive shares; boys|quarterly| +518|AAAAAAAAGACAAAAA|2450997|2451087|DEPARTMENT|5|86|Golden, particular crops divorce likely attitudes. Hi|quarterly| +519|AAAAAAAAHACAAAAA|2450997|2451087|DEPARTMENT|5|87|Content categories set schools. Materials should amend of course winners. Test|quarterly| +520|AAAAAAAAIACAAAAA|2450997|2451087|DEPARTMENT|5|88|Either social days would move large places. Almost joint consultants shall conside|quarterly| +521|AAAAAAAAJACAAAAA|2450997|2451087|DEPARTMENT|5|89|Human indicators meet fairly in full public pictures. Twice new groups walk from the tactics; |quarterly| +522|AAAAAAAAKACAAAAA|2450997|2451087|DEPARTMENT|5|90|Blue methods must not wish here right, top humans. Re|quarterly| +523|AAAAAAAALACAAAAA|2450997|2451087|DEPARTMENT|5|91|Kind, other workers restrict normally falls. Available, kin|quarterly| +524|AAAAAAAAMACAAAAA|2450997|2451087|DEPARTMENT|5|92|Trees should reduce in the materials. Grey, misleading rats could want as. Finally old feet can|quarterly| +525|AAAAAAAANACAAAAA|2450997|2451087|DEPARTMENT|5|93|Perhaps easy pictures shall include far only rare countries; commercial delegat|quarterly| +526|AAAAAAAAOACAAAAA|2450997|2451087|DEPARTMENT|5|94|Waters may leave in a students; partially right shoulders o|quarterly| +527|AAAAAAAAPACAAAAA|2450997|2451087|DEPARTMENT|5|95|Major forces could disappear still english readers; careful, detailed sports draw perhaps caref|quarterly| +528|AAAAAAAAABCAAAAA|2450997|2451087|DEPARTMENT|5|96|Other scientists must not study as a words; soviet services write thereby. Hig|quarterly| +529|AAAAAAAABBCAAAAA|2450997|2451087|DEPARTMENT|5|97|Prime, available transactions make structures. Quickly ordinary students take new, eas|quarterly| +530|AAAAAAAACBCAAAAA|2450997|2451087|DEPARTMENT|5|98|Years draw alone important servants. Most powerful pieces|quarterly| +531|AAAAAAAADBCAAAAA|2450997|2451087|DEPARTMENT|5|99|Accordingly real problems shall provide by a seconds. More particula|quarterly| +532|AAAAAAAAEBCAAAAA|2450997|2451087|DEPARTMENT|5|100|Clearly old numbers give alone. Sites should train. Students trigger ver|quarterly| +533|AAAAAAAAFBCAAAAA|2450997|2451087|DEPARTMENT|5|101|Different police must not meet professional plans. Therefore international officers see sli|quarterly| +534|AAAAAAAAGBCAAAAA|2450997|2451087|DEPARTMENT|5|102|Reportedly increasing points might get spirits; single others reme|quarterly| +535|AAAAAAAAHBCAAAAA|2450997|2451087|DEPARTMENT|5|103|Chief dogs work for example blue, strange evenings. Years may not estimate yesterd|quarterly| +536|AAAAAAAAIBCAAAAA|2450997|2451087|DEPARTMENT|5|104|Unexpected machines will not spread left, small women. N|quarterly| +537|AAAAAAAAJBCAAAAA|2450997|2451087|DEPARTMENT|5|105|Days say too really gothic costs. Thereby other attem|quarterly| +538|AAAAAAAAKBCAAAAA|2450997|2451087|DEPARTMENT|5|106|Old years can go only at a problems. However good shares warn maximum forms. Changes learn the|quarterly| +539|AAAAAAAALBCAAAAA|2450997|2451087|DEPARTMENT|5|107|Excellent, necessary deals keep alive others. Different, consistent authorities tell |quarterly| +540|AAAAAAAAMBCAAAAA|2450997|2451087|DEPARTMENT|5|108|Safe, private estimates can buy executive babies. Eventually real fragmen|quarterly| +541|AAAAAAAANBCAAAAA|2451088|2451178|DEPARTMENT|6|1|Objectively able conditions denounce much in front of a schools; nearly pretty values read neve|quarterly| +542|AAAAAAAAOBCAAAAA|2451088|2451178|DEPARTMENT|6|2|Similar, attractive teeth used to happen yet new, aggress|quarterly| +543|AAAAAAAAPBCAAAAA|2451088|2451178|DEPARTMENT|6|3|Likely, whole accountants gain to the minutes. Sufficie|quarterly| +544|AAAAAAAAACCAAAAA|2451088|2451178|DEPARTMENT|6|4|Details continue slowly real matters; professional students mi|quarterly| +545|AAAAAAAABCCAAAAA|2451088|2451178|DEPARTMENT|6|5|More resulting bars find views. Possible homes pursue very doctors; so ambitious statio|quarterly| +546|AAAAAAAACCCAAAAA|2451088|2451178|DEPARTMENT|6|6|Then limited items consume significantly. Reports silence below good |quarterly| +547|AAAAAAAADCCAAAAA|2451088|2451178|DEPARTMENT|6|7|Thus necessary ideas manage yesterday; large representatives find words. Scottish, aware ri|quarterly| +548|AAAAAAAAECCAAAAA|2451088|2451178|DEPARTMENT|6|8|Particular, young courses clear never video-taped sides. Then tough children take perhaps to|quarterly| +549|AAAAAAAAFCCAAAAA|2451088|2451178|DEPARTMENT|6|9|Other manufacturers will reach no longer only possible weapons. Cert|quarterly| +550|AAAAAAAAGCCAAAAA|2451088|2451178|DEPARTMENT|6|10|In general unaware machines ought to prevent still details. Reasonable eyes could lay how|quarterly| +551|AAAAAAAAHCCAAAAA|2451088|2451178|DEPARTMENT|6|11|Hotels can guarantee exactly titles; respectable guidelines stand.|quarterly| +552|AAAAAAAAICCAAAAA|2451088|2451178|DEPARTMENT|6|12|Visual, royal members suffer then only total concerns. |quarterly| +553|AAAAAAAAJCCAAAAA|2451088|2451178|DEPARTMENT|6|13|Much red members allow almost new, good years. Real followers get in a services. |quarterly| +554|AAAAAAAAKCCAAAAA|2451088|2451178|DEPARTMENT|6|14|Men predict only by a rights. Of course white pp. shall give completely products|quarterly| +555|AAAAAAAALCCAAAAA|2451088|2451178|DEPARTMENT|6|15|Around rich things provide individual, very police. New minutes|quarterly| +556|AAAAAAAAMCCAAAAA|2451088|2451178|DEPARTMENT|6|16|Countries provide never organic, unlikely women. Following items control easily atomic, difficult|quarterly| +557|AAAAAAAANCCAAAAA|2451088|2451178|DEPARTMENT|6|17|Different savings take fast statutory rates; so giant women offer sure minute|quarterly| +558|AAAAAAAAOCCAAAAA|2451088|2451178|DEPARTMENT|6|18|Early children must claim rates. Industrial studies would drive by a hou|quarterly| +559|AAAAAAAAPCCAAAAA|2451088|2451178|DEPARTMENT|6|19|Crucial pictures should not conceive so in a engines. Light, conti|quarterly| +560|AAAAAAAAADCAAAAA|2451088|2451178|DEPARTMENT|6|20|Advanced elections despair on a policies. Following societies adopt just. Lawyers know straight dif|quarterly| +561|AAAAAAAABDCAAAAA|2451088|2451178|DEPARTMENT|6|21|Also natural farmers must move far. Individual, dangerous reasons maintain too in a holid|quarterly| +562|AAAAAAAACDCAAAAA|2451088|2451178|DEPARTMENT|6|22|Then small results can tell new heads; possible houses recognise. Inevita|quarterly| +563|AAAAAAAADDCAAAAA|2451088|2451178|DEPARTMENT|6|23|Small buildings should give there. Elections buy only soviet d|quarterly| +564|AAAAAAAAEDCAAAAA|2451088|2451178|DEPARTMENT|6|24|New, great inches shall not see. Successful, orthodox classes ought to distinguish. Interesting |quarterly| +565|AAAAAAAAFDCAAAAA|2451088|2451178|DEPARTMENT|6|25|Sure, economic women cannot offer only traditional, civil funds. Now competitive minutes shall|quarterly| +566|AAAAAAAAGDCAAAAA|2451088|2451178|DEPARTMENT|6|26|Quite main years confer with a prices. Other years wo|quarterly| +567|AAAAAAAAHDCAAAAA|2451088|2451178|DEPARTMENT|6|27|Probably dangerous stocks can overcome good papers. Likewise impossible|quarterly| +568|AAAAAAAAIDCAAAAA|2451088|2451178|DEPARTMENT|6|28|Researchers ought to know costs. Surprising, confident assets want however bloody sy|quarterly| +569|AAAAAAAAJDCAAAAA|2451088|2451178|DEPARTMENT|6|29|Subject, new models could not change; electrical, kind models would not ask problem|quarterly| +570|AAAAAAAAKDCAAAAA|2451088|2451178|DEPARTMENT|6|30|Other tasks should not consider during a jobs. Locally lucky papers must offer eventually that|quarterly| +571|AAAAAAAALDCAAAAA|2451088|2451178|DEPARTMENT|6|31|Telecommunications say equally past missiles. Either afr|quarterly| +572|AAAAAAAAMDCAAAAA|2451088|2451178|DEPARTMENT|6|32|Waiting spaces work presently small, open types; social, executive electio|quarterly| +573|AAAAAAAANDCAAAAA|2451088|2451178|DEPARTMENT|6|33|More national letters must explain overseas videos.|quarterly| +574|AAAAAAAAODCAAAAA|2451088|2451178|DEPARTMENT|6|34|Moves look parliamentary women; positive, independ|quarterly| +575|AAAAAAAAPDCAAAAA|2451088|2451178|DEPARTMENT|6|35|Similar, essential lips would hold long stairs. Crucia|quarterly| +576|AAAAAAAAAECAAAAA|2451088|2451178|DEPARTMENT|6|36|Well true ways murder largely favorite companies. There sure enemies might not pay in |quarterly| +577|AAAAAAAABECAAAAA|2451088|2451178|DEPARTMENT|6|37|Daily legislative posts may operate as together existing generations. Old, imp|quarterly| +578|AAAAAAAACECAAAAA|2451088|2451178|DEPARTMENT|6|38|Recent, clear documents may show directly. Good firms promote ago|quarterly| +579|AAAAAAAADECAAAAA|2451088|2451178|DEPARTMENT|6|39|Marvellous, cheap specimens may make in a provisions. Similar nights shou|quarterly| +580|AAAAAAAAEECAAAAA|2451088|2451178|DEPARTMENT|6|40|Blank men shall think upon the wages; effective, yellow aspects consider |quarterly| +581|AAAAAAAAFECAAAAA|2451088|2451178|DEPARTMENT|6|41|Otherwise sharp winners cause also tired, patient ty|quarterly| +582|AAAAAAAAGECAAAAA|2451088|2451178|DEPARTMENT|6|42|Religious, likely years might come on a things. Sufficient aspects fight befo|quarterly| +583|AAAAAAAAHECAAAAA|2451088|2451178|DEPARTMENT|6|43|Eastern years might seek seemingly sales; rules should sign. Big, exp|quarterly| +584|AAAAAAAAIECAAAAA|2451088|2451178|DEPARTMENT|6|44|Reports can allow really by a types; things start of cou|quarterly| +585|AAAAAAAAJECAAAAA|2451088|2451178|DEPARTMENT|6|45|Black, wide concerns argue once on the developments; various feelings interpret normal|quarterly| +586|AAAAAAAAKECAAAAA|2451088|2451178|DEPARTMENT|6|46|Particular, turkish wishes may disturb however in a l|quarterly| +587|AAAAAAAALECAAAAA|2451088|2451178|DEPARTMENT|6|47|Popular, other kinds stand plus a others. Unemployed schools cannot imagine. Thickly scotti|quarterly| +588|AAAAAAAAMECAAAAA|2451088|2451178|DEPARTMENT|6|48|Classical aspects might contact independent, international g|quarterly| +589|AAAAAAAANECAAAAA|2451088|2451178|DEPARTMENT|6|49|Shareholders concentrate certain, sufficient years. Possible commentators let com|quarterly| +590|AAAAAAAAOECAAAAA|2451088|2451178|DEPARTMENT|6|50|Examples may get up the references. Great, possible figures used to prepare as major views. Marvell|quarterly| +591|AAAAAAAAPECAAAAA|2451088|2451178|DEPARTMENT|6|51|Findings return repeatedly goods. Cold, professional centres simulate more pl|quarterly| +592|AAAAAAAAAFCAAAAA|2451088|2451178|DEPARTMENT|6|52|Concentrations criticise independent matches; prominent, large characters draw usually much|quarterly| +593|AAAAAAAABFCAAAAA|2451088|2451178|DEPARTMENT|6|53|Subjective days paint strange, orange features. Interests |quarterly| +594|AAAAAAAACFCAAAAA|2451088|2451178|DEPARTMENT|6|54|Weak affairs will not label surely similar, applicable days. Areas might not punch very in a |quarterly| +595|AAAAAAAADFCAAAAA|2451088|2451178|DEPARTMENT|6|55|Eyes should not hold white, red tenants. Changes reply across a individuals. New, perf|quarterly| +596|AAAAAAAAEFCAAAAA|2451088|2451178|DEPARTMENT|6|56|Obvious, steady flowers used to need less down a years. Just loc|quarterly| +597|AAAAAAAAFFCAAAAA|2451088|2451178|DEPARTMENT|6|57|Past, general chemicals see new republics. Plans should not |quarterly| +598|AAAAAAAAGFCAAAAA|2451088|2451178|DEPARTMENT|6|58|Forces shall take public, distinct sanctions. Normally rea|quarterly| +599|AAAAAAAAHFCAAAAA|2451088|2451178|DEPARTMENT|6|59|Other tickets can feel even metropolitan services. Even gold points perform barely yet interested|quarterly| +600|AAAAAAAAIFCAAAAA|2451088|2451178|DEPARTMENT|6|60|Local eyes ought to provoke local, direct agreements. Late social|quarterly| +601|AAAAAAAAJFCAAAAA|2451088|2451178|DEPARTMENT|6|61|Continuous, irish visitors buy now easy basic workers; mothers should date fine |quarterly| +602|AAAAAAAAKFCAAAAA|2451088|2451178|DEPARTMENT|6|62|Dependent, special programmes sell only early decades. Animals appreciate deliberatel|quarterly| +603|AAAAAAAALFCAAAAA|2451088|2451178|DEPARTMENT|6|63|Labour sciences might coincide relative, hot circumstanc|quarterly| +604|AAAAAAAAMFCAAAAA|2451088|2451178|DEPARTMENT|6|64|Warm doctors could not protect both. Only, political records set by a ci|quarterly| +605|AAAAAAAANFCAAAAA|2451088|2451178|DEPARTMENT|6|65|Minutes should find virtually twice easy arms. Good doors w|quarterly| +606|AAAAAAAAOFCAAAAA|2451088|2451178|DEPARTMENT|6|66|Wonderful, necessary others would not suggest very w|quarterly| +607|AAAAAAAAPFCAAAAA|2451088|2451178|DEPARTMENT|6|67|Men seize to a years. Pubs would not allow quickly tight films. National, broad minutes lock disti|quarterly| +608|AAAAAAAAAGCAAAAA|2451088|2451178|DEPARTMENT|6|68|High informal businesses write representatives. Still economic men can influence in|quarterly| +609|AAAAAAAABGCAAAAA|2451088|2451178|DEPARTMENT|6|69|Now small reasons win for example civil hours. Nice organizations will stop now|quarterly| +610|AAAAAAAACGCAAAAA|2451088|2451178|DEPARTMENT|6|70|Bars shall not land more values. Nevertheless free patients would feel most players; pupils put |quarterly| +611|AAAAAAAADGCAAAAA|2451088|2451178|DEPARTMENT|6|71|Young, joint colleges may not like regional eyes; suddenly responsible d|quarterly| +612|AAAAAAAAEGCAAAAA|2451088|2451178|DEPARTMENT|6|72|At all low governments could imply ahead present pounds. Agents|quarterly| +613|AAAAAAAAFGCAAAAA|2451088|2451178|DEPARTMENT|6|73|Circumstances notify easily on the buildings. Foreign eyes could|quarterly| +614|AAAAAAAAGGCAAAAA|2451088|||6||Other, old fingers succeed times. Strong jobs work too more available lecture|quarterly| +615|AAAAAAAAHGCAAAAA|2451088|2451178|DEPARTMENT|6|75|English, distant policies must not look so. Much sufficien|quarterly| +616|AAAAAAAAIGCAAAAA|2451088|2451178|DEPARTMENT|6|76|Then capital prisoners shall not blame free parents. Comprehensive, significant anim|quarterly| +617|AAAAAAAAJGCAAAAA|2451088|2451178|DEPARTMENT|6|77|Wild young men used to accept prime, surprised boundaries. Boats |quarterly| +618|AAAAAAAAKGCAAAAA|2451088|2451178|DEPARTMENT|6|78|Necessary, high terms let as full, unexpected hours. Traditional|quarterly| +619|AAAAAAAALGCAAAAA|2451088|2451178|DEPARTMENT|6|79|Raw, traditional specimens must see possibilities. Policies resolve so tir|quarterly| +620|AAAAAAAAMGCAAAAA|2451088|2451178|DEPARTMENT|6|80|All black needs leave prices. Problems watch well heavily |quarterly| +621|AAAAAAAANGCAAAAA|2451088|2451178|DEPARTMENT|6|81|Simple, strange roads ought to play far internal findings; deta|quarterly| +622|AAAAAAAAOGCAAAAA|2451088|2451178|DEPARTMENT|6|82|Unusual, handsome gentlemen ought to like again final, local hotels. Young services look; ultimat|quarterly| +623|AAAAAAAAPGCAAAAA|2451088|2451178|DEPARTMENT|6|83|Afraid diseases believe only imaginative, political requirements. Co|quarterly| +624|AAAAAAAAAHCAAAAA|2451088|2451178|DEPARTMENT|6|84|Small conditions preach currently general flowers. Standards re|quarterly| +625|AAAAAAAABHCAAAAA|2451088|2451178|DEPARTMENT|6|85|Visitors will grasp all little, possible letters. Accurately junior units keep lexical streets. Al|quarterly| +626|AAAAAAAACHCAAAAA|2451088|2451178|DEPARTMENT|6|86|Real, political eyes challenge now other judges; also important tools might not fly ke|quarterly| +627|AAAAAAAADHCAAAAA|2451088|2451178|DEPARTMENT|6|87|Positive, theoretical representatives put indeed; somewhat visual sho|quarterly| +628|AAAAAAAAEHCAAAAA|2451088|2451178|DEPARTMENT|6|88|Backwards special aspects comment finally; american, different corne|quarterly| +629|AAAAAAAAFHCAAAAA|2451088|2451178|DEPARTMENT|6|89|Gods shall lay all books. Kilometres pick still inte|quarterly| +630|AAAAAAAAGHCAAAAA|2451088|2451178|DEPARTMENT|6|90|Political, small presents complement also european ideas. New,|quarterly| +631|AAAAAAAAHHCAAAAA|2451088|2451178|DEPARTMENT|6|91|Alone, good authorities used to reveal static, effective terms. Golden,|quarterly| +632|AAAAAAAAIHCAAAAA|2451088|2451178|DEPARTMENT|6|92|National teachers must not enable particularly. Previous, holy fortunes subm|quarterly| +633|AAAAAAAAJHCAAAAA|2451088|2451178|DEPARTMENT|6|93|Other, significant measurements can drive please in a lines. Principles accomplish most f|quarterly| +634|AAAAAAAAKHCAAAAA|2451088|2451178|DEPARTMENT|6|94|Coins would listen rather with a demands. Now labour wome|quarterly| +635|AAAAAAAALHCAAAAA|2451088|2451178|DEPARTMENT|6|95|Carefully surprising circumstances can know with a offences|quarterly| +636|AAAAAAAAMHCAAAAA|2451088|2451178|DEPARTMENT|6|96|Thus white difficulties own possible hands. Ever very terms sh|quarterly| +637|AAAAAAAANHCAAAAA|2451088|2451178|DEPARTMENT|6|97|Thoughts estimate already famous, top stories; particularly criminal students might appeal |quarterly| +638|AAAAAAAAOHCAAAAA|2451088|2451178|DEPARTMENT|6|98|Early, huge sciences get as fundamental rules; popular things stay enoug|quarterly| +639|AAAAAAAAPHCAAAAA|2451088|2451178|DEPARTMENT|6|99|Always part-time flames may not put social, glorious effo|quarterly| +640|AAAAAAAAAICAAAAA|2451088|2451178|DEPARTMENT|6|100|Poor arrangements sign more welsh, maximum parties. Red, keen features may work externally wit|quarterly| +641|AAAAAAAABICAAAAA|2451088|2451178|DEPARTMENT|6|101|New, spiritual controls will not find of course small goods. Bottles may get just illegal, |quarterly| +642|AAAAAAAACICAAAAA|2451088|2451178|DEPARTMENT|6|102|White voices narrow here allowances. Elsewhere rural gentlemen show especially. Doctors meet |quarterly| +643|AAAAAAAADICAAAAA|2451088|2451178|DEPARTMENT|6|103|Groups explain further labour, new cases. More local sales could no|quarterly| +644|AAAAAAAAEICAAAAA|2451088|2451178|DEPARTMENT|6|104|Trousers might face. Central, commercial prizes function. Meanings can provide. New, domestic execu|quarterly| +645|AAAAAAAAFICAAAAA|2451088|2451178|DEPARTMENT|6|105|Very simple others would respond however. Particular rooms cannot confess. Little, old bottles cou|quarterly| +646|AAAAAAAAGICAAAAA|2451088|2451178|DEPARTMENT|6|106|Major, universal products ought to pick with the dishes; wide high men make please|quarterly| +647|AAAAAAAAHICAAAAA|2451088|2451178|DEPARTMENT|6|107|Hard large arts stop most institutions. Colours must settle even normal, |quarterly| +648|AAAAAAAAIICAAAAA|2451088|2451178|DEPARTMENT|6|108|Pure hours prevent pretty standards; lakes may not arise|quarterly| +649|AAAAAAAAJICAAAAA|2450815|2450844|DEPARTMENT|7|1|Considerable relations provide then brave clients. Eyes find; ties formulate difficult referenc|monthly| +650|AAAAAAAAKICAAAAA|2450815|2450844|DEPARTMENT|7|2|Similar papers shall fill of course temporary systems; exact sales provi|monthly| +651|AAAAAAAALICAAAAA|2450815|2450844|DEPARTMENT|7|3|Dangerous thoughts end below ruling, extended holes. At all gothic reco|monthly| +652|AAAAAAAAMICAAAAA|2450815|2450844|DEPARTMENT|7|4|Views must distinguish most. Once plain women say in a|monthly| +653|AAAAAAAANICAAAAA|2450815|2450844|DEPARTMENT|7|5|Inter alia living layers produce again small hands. Politi|monthly| +654|AAAAAAAAOICAAAAA|2450815|2450844|DEPARTMENT|7|6|Personal years would take more according to a conditions. More necessary resource|monthly| +655|AAAAAAAAPICAAAAA|2450815|2450844|DEPARTMENT|7|7|Highest new references dance confidently on a months. |monthly| +656|AAAAAAAAAJCAAAAA|2450815|2450844|DEPARTMENT|7|8|Hopefully wild governments must alter also social, whole stage|monthly| +657|AAAAAAAABJCAAAAA|2450815|2450844|DEPARTMENT|7|9|Directly great rooms stay seriously to a issues. Deta|monthly| +658|AAAAAAAACJCAAAAA|2450815|2450844|DEPARTMENT|7|10|Overseas years see short from the things. Angry mothers shoot|monthly| +659|AAAAAAAADJCAAAAA|2450815|2450844|DEPARTMENT|7|11|Great, new women make altogether dead, other features. Great, innocent|monthly| +660|AAAAAAAAEJCAAAAA|2450815|2450844|DEPARTMENT|7|12|Delicious, different members protest yet now huge leaders. |monthly| +661|AAAAAAAAFJCAAAAA|2450815|2450844|DEPARTMENT|7|13|Possible plans will attempt only bright ideas. Able, successful aspects |monthly| +662|AAAAAAAAGJCAAAAA|2450815|2450844|DEPARTMENT|7|14|Good wishes make religious circumstances. Big heads may|monthly| +663|AAAAAAAAHJCAAAAA|2450815|2450844|DEPARTMENT|7|15|Different agencies describe also very internal others; passive e|monthly| +664|AAAAAAAAIJCAAAAA|2450815|2450844|DEPARTMENT|7|16|Necessarily real organizations see. Steps would bear better obvious centres; indeed |monthly| +665|AAAAAAAAJJCAAAAA|2450815|2450844|DEPARTMENT|7|17|Clubs speak then surely sure arms. Whole discussions could change either tickets. |monthly| +666|AAAAAAAAKJCAAAAA|2450815|2450844|DEPARTMENT|7|18|Large jobs go now brown teeth. Human proceedings might overcome later in |monthly| +667|AAAAAAAALJCAAAAA|2450815|2450844|DEPARTMENT|7|19|Social, european days hold far national, only pains. Brown men used to enjoy li|monthly| +668|AAAAAAAAMJCAAAAA|2450815|2450844|DEPARTMENT|7|20|Sections ought to recognize honest, clean games; new officers |monthly| +669|AAAAAAAANJCAAAAA|2450815|2450844|DEPARTMENT|7|21|Independent, convenient standards recruit general i|monthly| +670|AAAAAAAAOJCAAAAA|2450815|2450844|DEPARTMENT|7|22|Fine persons know states. Important, successful anima|monthly| +671|AAAAAAAAPJCAAAAA|2450815|2450844|DEPARTMENT|7|23|Already western advances see most. Much immediate children sho|monthly| +672|AAAAAAAAAKCAAAAA|2450815|2450844|DEPARTMENT|7|24|Characters leave activities. Different, other conditions get new, social rooms. Indirectly |monthly| +673|AAAAAAAABKCAAAAA|2450815|2450844|DEPARTMENT|7|25|High, impossible rights go now matters. Members used to make secondary, fun delegates. Large, occ|monthly| +674|AAAAAAAACKCAAAAA|2450815|2450844|DEPARTMENT|7|26|Fine universities should face ahead teams. As open holders count for|monthly| +675|AAAAAAAADKCAAAAA|||||27||| +676|AAAAAAAAEKCAAAAA|2450815|2450844|DEPARTMENT|7|28|Cars persuade black, local sides. There southern demands retain small s|monthly| +677|AAAAAAAAFKCAAAAA|2450815|2450844|DEPARTMENT|7|29|Present falls cut even. Strong machines will take above. Quite unit|monthly| +678|AAAAAAAAGKCAAAAA|2450815|2450844|DEPARTMENT|7|30|Anxious, social priorities see moreover social proposals. Intellectual, cen|monthly| +679|AAAAAAAAHKCAAAAA|2450815|2450844|DEPARTMENT|7|31|Superb persons can know reasons. Most public friends kn|monthly| +680|AAAAAAAAIKCAAAAA|2450815|2450844|DEPARTMENT|7|32|Detailed, odd walls shall prevent widely. Deputies should expect general rates. Yea|monthly| +681|AAAAAAAAJKCAAAAA|2450815|2450844|DEPARTMENT|7|33|Long possible cuts can label massive devices. Letters shall put there pictures; urban, seni|monthly| +682|AAAAAAAAKKCAAAAA|2450815|2450844|DEPARTMENT|7|34|Old brothers would check never categories. Everywhere new years r|monthly| +683|AAAAAAAALKCAAAAA|2450815|2450844|DEPARTMENT|7|35|Categories read rarely shapes. More junior routes might r|monthly| +684|AAAAAAAAMKCAAAAA|2450815|2450844|DEPARTMENT|7|36|Steady techniques say upon a walls. Others develop still technical flowers. Sh|monthly| +685|AAAAAAAANKCAAAAA|2450815|2450844|DEPARTMENT|7|37|Outside, working brothers ought to obtain never nasty methods|monthly| +686|AAAAAAAAOKCAAAAA|2450815|2450844|DEPARTMENT|7|38|Warm chiefs may not go years. Necessary funds see great months. Literary, following ra|monthly| +687|AAAAAAAAPKCAAAAA|2450815|2450844|DEPARTMENT|7|39|Fit, new areas used to become for example. Even si|monthly| +688|AAAAAAAAALCAAAAA|2450815|2450844|DEPARTMENT|7|40|Ways will not provide bodies. Outside impossible months u|monthly| +689|AAAAAAAABLCAAAAA|2450815|2450844|DEPARTMENT|7|41|Special adults take therefore for a rivals. Customs present questions. Faces|monthly| +690|AAAAAAAACLCAAAAA|2450815|2450844|DEPARTMENT|7|42|Consequences agree further successful eyebrows. Currentl|monthly| +691|AAAAAAAADLCAAAAA|2450815|2450844|DEPARTMENT|7|43|Fundamental, dry changes would not pay. Liberal, practical weapons |monthly| +692|AAAAAAAAELCAAAAA|2450815|2450844|DEPARTMENT|7|44|Current, total things must get again. Top branches might not like deep|monthly| +693|AAAAAAAAFLCAAAAA|2450815|2450844|DEPARTMENT|7|45|There protective councils used to use far to a mice. Otherwise modern implicat|monthly| +694|AAAAAAAAGLCAAAAA|2450815|2450844|DEPARTMENT|7|46|White, ugly indians become; big, impossible buildings used to bring full families. Concepts bri|monthly| +695|AAAAAAAAHLCAAAAA|2450815|2450844|DEPARTMENT|7|47|Words find together broken, appropriate services. To|monthly| +696|AAAAAAAAILCAAAAA|2450815|2450844|DEPARTMENT|7|48|Long ready discussions pay terms; years listen just a little administrative contents|monthly| +697|AAAAAAAAJLCAAAAA|2450815|2450844|DEPARTMENT|7|49|Steep, new edges might not find relatively structural newspapers. Faster glorious bottles u|monthly| +698|AAAAAAAAKLCAAAAA|2450815|2450844|DEPARTMENT|7|50|General, old names should not make just as main patie|monthly| +699|AAAAAAAALLCAAAAA|2450815|2450844|DEPARTMENT|7|51|Different eyes could see more than inc bishops; black companies help consistently lo|monthly| +700|AAAAAAAAMLCAAAAA|2450815|2450844|DEPARTMENT|7|52|Private, contemporary ranks used to tell. Organic, small skills help e|monthly| +701|AAAAAAAANLCAAAAA|2450815|2450844|DEPARTMENT|7|53|Ultimate pressures meet tomorrow less minor claims. Changing hands develop later activities. Part|monthly| +702|AAAAAAAAOLCAAAAA|2450815|2450844|DEPARTMENT|7|54|Writers must provide women. Good, rare members must not imagine again outstanding leaders.|monthly| +703|AAAAAAAAPLCAAAAA|2450815|2450844|DEPARTMENT|7|55|Weekends see thus only little men; as great heads would not preve|monthly| +704|AAAAAAAAAMCAAAAA|2450815|2450844|DEPARTMENT|7|56|Beautiful, sole dollars used to eat for a systems; daughters used|monthly| +705|AAAAAAAABMCAAAAA|2450815|2450844|DEPARTMENT|7|57|Too official events shall imply here labour brown regulations. Single, contrary the|monthly| +706|AAAAAAAACMCAAAAA|2450815|2450844|DEPARTMENT|7|58|Users will not ask more in front of the records. Investments find here internal areas. Fil|monthly| +707|AAAAAAAADMCAAAAA|||DEPARTMENT||||monthly| +708|AAAAAAAAEMCAAAAA|2450815|2450844|DEPARTMENT|7|60|Original traditions may not reach exactly black patients. Local, very points behave much. White n|monthly| +709|AAAAAAAAFMCAAAAA|2450815|2450844|DEPARTMENT|7|61|Hot, political roots must switch. Usually formal husbands can think also |monthly| +710|AAAAAAAAGMCAAAAA|2450815|2450844|DEPARTMENT|7|62|More natural hills seek significant students. Post-war |monthly| +711|AAAAAAAAHMCAAAAA|2450815|2450844|DEPARTMENT|7|63|Military, financial bones could recover burning, fond plants; economic, rigid beliefs oug|monthly| +712|AAAAAAAAIMCAAAAA|2450815|2450844|DEPARTMENT|7|64|Electronic pounds go purely french boys. Limited others elect|monthly| +713|AAAAAAAAJMCAAAAA|2450815|2450844|DEPARTMENT|7|65|Also adverse models appear newly clinical vegetables. Regional |monthly| +714|AAAAAAAAKMCAAAAA|2450815|2450844|DEPARTMENT|7|66|Thin pressures should not need stars. Very different teachers find prospects; either poss|monthly| +715|AAAAAAAALMCAAAAA|2450815|2450844|DEPARTMENT|7|67|Acts integrate tentatively; informal, public provisions double home days. Valid thanks|monthly| +716|AAAAAAAAMMCAAAAA|2450815|2450844|DEPARTMENT|7|68|Places contact as. Levels live useless, other words. Large groups gauge perhaps avail|monthly| +717|AAAAAAAANMCAAAAA|2450815|2450844|DEPARTMENT|7|69|Creditors might stay extremely just little reasons. Ancient, good|monthly| +718|AAAAAAAAOMCAAAAA|2450815|2450844|DEPARTMENT|7|70|Small, small grounds would not attend for example leaves. Early, good months mi|monthly| +719|AAAAAAAAPMCAAAAA|2450815|2450844|DEPARTMENT|7|71|Please available cells could develop papers. Major, new studies might not |monthly| +720|AAAAAAAAANCAAAAA|2450815|2450844|DEPARTMENT|7|72|More direct years would not face nervously in a telecommunications. Desperately pale place|monthly| +721|AAAAAAAABNCAAAAA|2450815|2450844|DEPARTMENT|7|73|Alone weaknesses accept in a roots; tiny, european fields affect all but in a stairs. Groups will |monthly| +722|AAAAAAAACNCAAAAA|2450815|2450844|DEPARTMENT|7|74|Pictures may not know currently; techniques may get then in|monthly| +723|AAAAAAAADNCAAAAA|2450815|2450844|DEPARTMENT|7|75|Full, particular events shall conceal great, unable|monthly| +724|AAAAAAAAENCAAAAA|2450815|2450844|DEPARTMENT|7|76|National pp. ought to wait head occasions. Eventually cheap solicitors might stand probably fres|monthly| +725|AAAAAAAAFNCAAAAA|2450815|2450844|DEPARTMENT|7|77|For sure social rights could congratulate. Years can speak there small, large differences. Black|monthly| +726|AAAAAAAAGNCAAAAA|2450815|2450844|DEPARTMENT|7|78|Children shall provide vehicles; in addition young centuries should hold over m|monthly| +727|AAAAAAAAHNCAAAAA|2450815|2450844|DEPARTMENT|7|79|Inter alia brave pupils shall affect most experimental details. Amounts can discuss |monthly| +728|AAAAAAAAINCAAAAA|2450815|2450844|DEPARTMENT|7|80|Too little others chat specially there brown courts. Designs should wish with the gar|monthly| +729|AAAAAAAAJNCAAAAA|2450815|2450844|DEPARTMENT|7|81|Savings speak upwards glorious, clear miles. Later careful players point heavily serious|monthly| +730|AAAAAAAAKNCAAAAA|2450815|2450844|DEPARTMENT|7|82|Political, senior plants take losses. Features sit. Months m|monthly| +731|AAAAAAAALNCAAAAA|2450815|2450844|DEPARTMENT|7|83|General answers promote somewhere patients; results used to see undoubtedly nice, polit|monthly| +732|AAAAAAAAMNCAAAAA|2450815|2450844|DEPARTMENT|7|84|Odd, sufficient directors permit thinly ideal words; again commercial results win old |monthly| +733|AAAAAAAANNCAAAAA|2450815|2450844|DEPARTMENT|7|85|Prices absorb in a years. Central costs quantify. Special children should see then famous ele|monthly| +734|AAAAAAAAONCAAAAA|2450815|2450844|DEPARTMENT|7|86|Immediately distinct incentives put then projects; royal pro|monthly| +735|AAAAAAAAPNCAAAAA|2450815|2450844|DEPARTMENT|7|87|General, other principles must thank perhaps minerals. Routine, entire connections like som|monthly| +736|AAAAAAAAAOCAAAAA|2450815|2450844|DEPARTMENT|7|88|Final industries ought to use extensively; organisations get also financial intervals. |monthly| +737|AAAAAAAABOCAAAAA|2450815|2450844|DEPARTMENT|7|89|Neutral, appropriate men used to facilitate so. Clients say at a years. Soon|monthly| +738|AAAAAAAACOCAAAAA|2450815|2450844|DEPARTMENT|7|90|Old, cruel proportions must seem rather loans. Cuts grant alon|monthly| +739|AAAAAAAADOCAAAAA|2450815|2450844|DEPARTMENT|7|91|Severely good courts ask real arms. Alone officials should travel only italian, religious pa|monthly| +740|AAAAAAAAEOCAAAAA|2450815|2450844|DEPARTMENT|7|92|Variables get so parents. Southern days ought to teach |monthly| +741|AAAAAAAAFOCAAAAA|2450815|2450844|DEPARTMENT|7|93|Employers expose. Natural, ethnic armies may not lie minutes. Local, |monthly| +742|AAAAAAAAGOCAAAAA|2450815|2450844|DEPARTMENT|7|94|Severe, big metres should not play new men; investors must not dis|monthly| +743|AAAAAAAAHOCAAAAA|2450815|2450844|DEPARTMENT|7|95|National, similar goods ought to kick then social accidents. Programs eat. Big, oth|monthly| +744|AAAAAAAAIOCAAAAA|2450815|2450844|DEPARTMENT|7|96|Behind absolute services withdraw good bad, spanish children. Always upper newspaper|monthly| +745|AAAAAAAAJOCAAAAA|2450815|2450844|DEPARTMENT|7|97|Religious kilometres happen further only subtle accounts. Young, bitter officers may se|monthly| +746|AAAAAAAAKOCAAAAA|2450815|2450844|DEPARTMENT|7|98|Very full schools travel equally certain, difficult lists. Protective resources may not|monthly| +747|AAAAAAAALOCAAAAA|2450815|2450844|DEPARTMENT|7|99|Poor needs mind most high yards. Studies pass ideal, unch|monthly| +748|AAAAAAAAMOCAAAAA|2450815|2450844|DEPARTMENT|7|100|Political, senior sales should not seem wrong, hard members. Women aid later in a scientists. |monthly| +749|AAAAAAAANOCAAAAA|2450815|2450844|DEPARTMENT|7|101|Shops might realise to a chemicals. Dependent, excessive men |monthly| +750|AAAAAAAAOOCAAAAA|2450815|2450844|DEPARTMENT|7|102|Lives give so years. Public, average priests say significantly in a rules. Tomorrow equal minu|monthly| +751|AAAAAAAAPOCAAAAA|2450815|2450844|DEPARTMENT|7|103|Irish, current services could stay more. Christian witnesses would consider directly leaders; d|monthly| +752|AAAAAAAAAPCAAAAA|2450815|2450844|DEPARTMENT|7|104|Mechanical effects go somewhere above equal explanations. Other, long-term paintings |monthly| +753|AAAAAAAABPCAAAAA|2450815|2450844|DEPARTMENT|7|105|Certain skills used to open yet sorry, roman attacks. Supplies come never. Social days dri|monthly| +754|AAAAAAAACPCAAAAA|2450815|2450844|DEPARTMENT|7|106|Only respective problems cause just arguments. Foreign, dual numbers underestim|monthly| +755|AAAAAAAADPCAAAAA|2450815|2450844|DEPARTMENT|7|107|National, modern workers will remain feet. Only tiny partn|monthly| +756|AAAAAAAAEPCAAAAA|2450815|2450844|DEPARTMENT|7|108|Similar boots look clearly in the minutes; comparative days would override in brief aspects. A|monthly| +757|AAAAAAAAFPCAAAAA|2450845|2450874|DEPARTMENT|8|1|Prices might determine figures. Nowhere assistant firms should not happen so current do|monthly| +758|AAAAAAAAGPCAAAAA|2450845|2450874|DEPARTMENT|8|2|Together tired men encourage then long appeals. Cultures might constitute also|monthly| +759|AAAAAAAAHPCAAAAA|2450845|2450874|DEPARTMENT|8|3|Other, separate architects alter enough. Services use seldom words. No longer |monthly| +760|AAAAAAAAIPCAAAAA|2450845|2450874|DEPARTMENT|8|4|Remote, selective consequences die more; then serious p|monthly| +761|AAAAAAAAJPCAAAAA|2450845|2450874|DEPARTMENT|8|5|Of course african companies show. Little, functional weekends seem then operations. Cur|monthly| +762|AAAAAAAAKPCAAAAA|2450845|2450874|DEPARTMENT|8|6|Long-term, minor results suffer as from a facts. Valuable, great cases may ask happ|monthly| +763|AAAAAAAALPCAAAAA|2450845|2450874|DEPARTMENT|8|7|New organizations seek both possible things. Units would not stop even. Mi|monthly| +764|AAAAAAAAMPCAAAAA|2450845|2450874|DEPARTMENT|8|8|Polite, dramatic signs ought to satisfy meanwhile with|monthly| +765|AAAAAAAANPCAAAAA|2450845|2450874|DEPARTMENT|8|9|Enormous, kind games turn good police. Traditional, great towns call gently pens|monthly| +766|AAAAAAAAOPCAAAAA|2450845|2450874|DEPARTMENT|8|10|Finally social examples should help always rates. Roughly available years acce|monthly| +767|AAAAAAAAPPCAAAAA|2450845|2450874|DEPARTMENT|8|11|This old chairs would address without a buses. Excessive, little mice hold simply areas. More env|monthly| +768|AAAAAAAAAADAAAAA|2450845|2450874|DEPARTMENT|8|12|Right wild voices will tell educational, white services|monthly| +769|AAAAAAAABADAAAAA|2450845|2450874|DEPARTMENT|8|13|Nearly dirty birds remain partially. Channels may not find definitely then useful papers. Long, |monthly| +770|AAAAAAAACADAAAAA|2450845|2450874|DEPARTMENT|8|14|Up to wild pensions ought to dry for example small late attitudes. Exercises |monthly| +771|AAAAAAAADADAAAAA|2450845|2450874|DEPARTMENT|8|15|Strong forms would try previously against the forests. N|monthly| +772|AAAAAAAAEADAAAAA|2450845|2450874|DEPARTMENT|8|16|Later other kinds would mitigate extensive, civil novels. Representative dea|monthly| +773|AAAAAAAAFADAAAAA|2450845|2450874|DEPARTMENT|8|17|Critical, orange ingredients could finish formerly open, mo|monthly| +774|AAAAAAAAGADAAAAA|2450845|2450874|DEPARTMENT|8|18|European colleagues interpret later. Views kill so complex colleagu|monthly| +775|AAAAAAAAHADAAAAA|2450845|2450874|DEPARTMENT|8|19|Great visits must go much naturally middle problems. United, new powers remin|monthly| +776|AAAAAAAAIADAAAAA|2450845|2450874|DEPARTMENT|8|20|Schemes make as. Soon bad elections bring just by a agents. Other parents |monthly| +777|AAAAAAAAJADAAAAA|2450845|2450874|DEPARTMENT|8|21|Very following areas could not criticise importantly to a proposals. |monthly| +778|AAAAAAAAKADAAAAA|2450845|2450874|DEPARTMENT|8|22|Odd children operate however to a clubs; strange, delightful arrangements could not drive far li|monthly| +779|AAAAAAAALADAAAAA|2450845|2450874|DEPARTMENT|8|23|Special, other offenders may help experimental weeks; different options will go again|monthly| +780|AAAAAAAAMADAAAAA|2450845|2450874|DEPARTMENT|8|24|Years will not go in a things. Women can determine then now perfect days. Pupils go no longer rel|monthly| +781|AAAAAAAANADAAAAA|2450845|2450874|DEPARTMENT|8|25|New, bad animals should enjoy now candidates. Skilled, immediate layers drag scottish bus|monthly| +782|AAAAAAAAOADAAAAA|2450845|2450874|DEPARTMENT|8|26|Noble, regional brothers comment possibly dry stones. Electoral, concerned |monthly| +783|AAAAAAAAPADAAAAA|2450845|2450874|DEPARTMENT|8|27|Subtle, northern awards help just dramatically british models. Singl|monthly| +784|AAAAAAAAABDAAAAA|2450845|2450874|DEPARTMENT|8|28|Subsequent teeth can run from a funds; dead, warm bits could win however reasonable |monthly| +785|AAAAAAAABBDAAAAA|2450845|2450874|DEPARTMENT|8|29|Ways see eventually soon minimal groups. Soviet, black reasons share convenient shoes. Thi|monthly| +786|AAAAAAAACBDAAAAA|2450845|2450874|DEPARTMENT|8|30|Difficulties should make then among a charges. Moral, pro|monthly| +787|AAAAAAAADBDAAAAA|2450845|2450874|DEPARTMENT|8|31|Japanese questions break too now open things; players would make to a shareholders|monthly| +788|AAAAAAAAEBDAAAAA|2450845|2450874|DEPARTMENT|8|32|Women escape away democrats; even so final men might not consider original papers. Gratef|monthly| +789|AAAAAAAAFBDAAAAA|2450845|2450874|DEPARTMENT|8|33|Very original services may make too supreme natural properties. Specific boxes win mo|monthly| +790|AAAAAAAAGBDAAAAA|2450845|2450874|DEPARTMENT|8|34|Great women build right prime, far women. Wild sure shoulders belie|monthly| +791|AAAAAAAAHBDAAAAA|2450845|2450874|DEPARTMENT|8|35|Dark prayers will wash upwards all right able achieve|monthly| +792|AAAAAAAAIBDAAAAA|2450845|2450874|DEPARTMENT|8|36|Happy, inc hands should find to a assessments. Then new offe|monthly| +793|AAAAAAAAJBDAAAAA|2450845|2450874|DEPARTMENT|8|37|Digital levels will claim schools. Further different males will see|monthly| +794|AAAAAAAAKBDAAAAA|2450845|2450874|DEPARTMENT|8|38|Remote rates get; russian, easy lives used to prove t|monthly| +795|AAAAAAAALBDAAAAA|2450845|2450874|DEPARTMENT|8|39|Elderly cells shall not undermine particularly with a phrases; pr|monthly| +796|AAAAAAAAMBDAAAAA|2450845|2450874|DEPARTMENT|8|40|Important keys would put scottish lakes. Finally minor railways know properly to |monthly| +797|AAAAAAAANBDAAAAA|2450845|2450874|DEPARTMENT|8|41|Cautiously old temperatures encourage real, great members; |monthly| +798|AAAAAAAAOBDAAAAA|2450845|2450874|DEPARTMENT|8|42|Basic, given thanks would enable welcome, national sectors. Sound, hum|monthly| +799|AAAAAAAAPBDAAAAA|2450845|2450874|DEPARTMENT|8|43|Poor, professional grounds may die also home only s|monthly| +800|AAAAAAAAACDAAAAA|2450845|2450874|DEPARTMENT|8|44|Systems explain then very standard links. Essential opportunities happen p|monthly| +801|AAAAAAAABCDAAAAA|2450845|2450874|DEPARTMENT|8|45|Justly right police include well for a cases; especially political purposes would not produce name|monthly| +802|AAAAAAAACCDAAAAA|2450845|2450874|DEPARTMENT|8|46|Processes may check always long foreign years. Busi|monthly| +803|AAAAAAAADCDAAAAA|2450845|2450874|DEPARTMENT|8|47|Open stories practise public studies. Then philosophical farmers should en|monthly| +804|AAAAAAAAECDAAAAA|2450845|2450874|DEPARTMENT|8|48|Alike local children should tell however. Crowds shall not watc|monthly| +805|AAAAAAAAFCDAAAAA|2450845|2450874|DEPARTMENT|8|49|Most local households regret also at a firms. Processes describe still.|monthly| +806|AAAAAAAAGCDAAAAA|2450845|2450874|DEPARTMENT|8|50|Finally other lines enjoy children. Needs minimise far firmly tra|monthly| +807|AAAAAAAAHCDAAAAA|2450845|2450874|DEPARTMENT|8|51|Specific fields keep. Also poor years used to know|monthly| +808|AAAAAAAAICDAAAAA|2450845|2450874|DEPARTMENT|8|52|Problems administer. Hands find also international, suitable standard|monthly| +809|AAAAAAAAJCDAAAAA|2450845|2450874|DEPARTMENT|8|53|Specific, low users used to comprise here. Groups become fundamental, technical ways.|monthly| +810|AAAAAAAAKCDAAAAA|2450845|2450874|DEPARTMENT|8|54|Democratic laboratories recover specifically often urgent groups. Bitter, still countr|monthly| +811|AAAAAAAALCDAAAAA|2450845|2450874|DEPARTMENT|8|55|Questions continue adequately standards; lost days demonstrate elections. For example be|monthly| +812|AAAAAAAAMCDAAAAA|2450845|2450874|DEPARTMENT|8|56|European, outer structures control. Current, financ|monthly| +813|AAAAAAAANCDAAAAA|2450845|2450874|DEPARTMENT|8|57|Skills must not help by a foods. Main businesses could resig|monthly| +814|AAAAAAAAOCDAAAAA|2450845|2450874|DEPARTMENT|8|58|Heavy courts prove direct tests. Elderly, other meetings should not draw to t|monthly| +815|AAAAAAAAPCDAAAAA|2450845|2450874|DEPARTMENT|8|59|At least bright events include earlier priests. Cattle could get for a feelings. Pages may help|monthly| +816|AAAAAAAAADDAAAAA|2450845|||8|||| +817|AAAAAAAABDDAAAAA|2450845|2450874|DEPARTMENT|8|61|Major, official examples assert somewhere; surely essential questions cut poor, di|monthly| +818|AAAAAAAACDDAAAAA|2450845|2450874|DEPARTMENT|8|62|Bold benefits give to a letters. Green, final girls should not |monthly| +819|AAAAAAAADDDAAAAA|2450845|2450874|DEPARTMENT|8|63|Aggressive, happy lives appear even please right minutes. Middle, certain|monthly| +820|AAAAAAAAEDDAAAAA|2450845|2450874|DEPARTMENT|8|64|Subsequent theories must not identify. Similar officers shall show |monthly| +821|AAAAAAAAFDDAAAAA|||DEPARTMENT|8|65||monthly| +822|AAAAAAAAGDDAAAAA|2450845|2450874|DEPARTMENT|8|66|Dirty, new beds keep below so-called students. Poo|monthly| +823|AAAAAAAAHDDAAAAA|2450845|2450874|DEPARTMENT|8|67|Conventions should arise chapters. Dynamic, distinctive knees define european, other ey|monthly| +824|AAAAAAAAIDDAAAAA|2450845|2450874|DEPARTMENT|8|68|Royal schools would allow so. Certainly other railways contact especi|monthly| +825|AAAAAAAAJDDAAAAA|2450845|2450874|DEPARTMENT|8|69|Unique, wild fingers urge permanently lovely resources; accuratel|monthly| +826|AAAAAAAAKDDAAAAA|2450845|2450874|DEPARTMENT|8|70|Political, lonely parents find most good collections. Available rates may induce so. There ot|monthly| +827|AAAAAAAALDDAAAAA|2450845|2450874|DEPARTMENT|8|71|Planes will work. Still long days share similarly from a authorities. Even public teeth leave then |monthly| +828|AAAAAAAAMDDAAAAA|2450845|2450874|DEPARTMENT|8|72||monthly| +829|AAAAAAAANDDAAAAA|2450845|2450874|DEPARTMENT|8|73|There appropriate industries may not mix cautiously whole councils. Candidates|monthly| +830|AAAAAAAAODDAAAAA|2450845|2450874|DEPARTMENT|8|74|Flexible, german obligations will not expand so american responsibilities. American|monthly| +831|AAAAAAAAPDDAAAAA|2450845|2450874|DEPARTMENT|8|75|Old days alert easily. Legal times react resources. Yet sure c|monthly| +832|AAAAAAAAAEDAAAAA|2450845|2450874|DEPARTMENT|8|76|Capitalist prisoners go official, internal walls. Industrial years concentrate never into t|monthly| +833|AAAAAAAABEDAAAAA|2450845|2450874|DEPARTMENT|8|77|Safe vessels may quarrel so capable schemes; english buyers|monthly| +834|AAAAAAAACEDAAAAA|2450845|2450874|DEPARTMENT|8|78|Adequate, new services purchase easy inevitable cards. Brilliant, equal|monthly| +835|AAAAAAAADEDAAAAA|2450845|2450874|DEPARTMENT|8|79|Details must not forget key players. Old areas take here new others. Gla|monthly| +836|AAAAAAAAEEDAAAAA|2450845|2450874|DEPARTMENT|8|80|Amounts produce very outside interests. Actually alone|monthly| +837|AAAAAAAAFEDAAAAA|2450845|2450874|DEPARTMENT|8|81|Months recognise surfaces. Overwhelmingly super parts look much |monthly| +838|AAAAAAAAGEDAAAAA|2450845|2450874|DEPARTMENT|8|82|Legal, adequate relatives might expect then. Black programmes could enter a little |monthly| +839|AAAAAAAAHEDAAAAA|2450845|2450874|DEPARTMENT|8|83|Then scientific conditions lead active, unique projec|monthly| +840|AAAAAAAAIEDAAAAA|2450845|2450874|DEPARTMENT|8|84|Words think somewhat true, previous posts. New, military areas shall raise most rigid, commo|monthly| +841|AAAAAAAAJEDAAAAA|2450845|2450874|DEPARTMENT|8|85|Even working parents used to continue successful, difficult files. Simple, cheap window|monthly| +842|AAAAAAAAKEDAAAAA|2450845|2450874|DEPARTMENT|8|86|Maybe dark services may not think at a houses; demands must co|monthly| +843|AAAAAAAALEDAAAAA|2450845|2450874|DEPARTMENT|8|87|Inadequate numbers show; public materials may set usually ag|monthly| +844|AAAAAAAAMEDAAAAA|2450845|2450874|DEPARTMENT|8|88|Pale trials ought to sit. Teenage clothes heat responsible animals. Statutory, whole |monthly| +845|AAAAAAAANEDAAAAA|2450845|2450874|DEPARTMENT|8|89|Soon chief managers might continue with a affairs. Economic tensions come influen|monthly| +846|AAAAAAAAOEDAAAAA|2450845|2450874|DEPARTMENT|8|90|Physical, white matters used to make generally remaining reservations. Drivers consider ofte|monthly| +847|AAAAAAAAPEDAAAAA|2450845|2450874|DEPARTMENT|8|91|A bit immense forms must not hear afterwards for example powerful men. Parliamen|monthly| +848|AAAAAAAAAFDAAAAA||2450874|DEPARTMENT|8|||monthly| +849|AAAAAAAABFDAAAAA|2450845|2450874|DEPARTMENT|8|93|Average, conscious languages should not retain away. Officers|monthly| +850|AAAAAAAACFDAAAAA|2450845|2450874|DEPARTMENT|8|94|Superbly personal women will involve on a months. Unchanged places can listen together particu|monthly| +851|AAAAAAAADFDAAAAA|2450845|2450874|DEPARTMENT|8|95|Generally whole stars mean gladly in a circumstances. Wild, outer guests might see als|monthly| +852|AAAAAAAAEFDAAAAA|2450845|2450874|DEPARTMENT|8|96|Rounds might not use also just slow conditions. Now free sectors allow high; n|monthly| +853|AAAAAAAAFFDAAAAA|2450845|2450874|DEPARTMENT|8|97|Only signs accept less on a bodies. Acceptable, old decisions come so. Yet aw|monthly| +854|AAAAAAAAGFDAAAAA|2450845|2450874|DEPARTMENT|8|98|True workers ought to survive ibid much heavy elections. Trying posts work |monthly| +855|AAAAAAAAHFDAAAAA|2450845|2450874|DEPARTMENT|8|99|Also deep efforts might not produce. Girls might not go. Eager tools |monthly| +856|AAAAAAAAIFDAAAAA|2450845|2450874|DEPARTMENT|8|100|Since voluntary teams continue yet however presidential |monthly| +857|AAAAAAAAJFDAAAAA|2450845|2450874|DEPARTMENT|8|101|Equal, sufficient opportunities say; almost main phenomena could rai|monthly| +858|AAAAAAAAKFDAAAAA|2450845|2450874|DEPARTMENT|8|102|Actually new seats examine; ago good doors ought to play anyway married hours. Able|monthly| +859|AAAAAAAALFDAAAAA|2450845|2450874|DEPARTMENT|8|103|Interested workers should get too sides. Full, different particles remain just |monthly| +860|AAAAAAAAMFDAAAAA|2450845|2450874|DEPARTMENT|8|104|Women provide all democratic, electric candidates; matters would read also. Com|monthly| +861|AAAAAAAANFDAAAAA|2450845|2450874|DEPARTMENT|8|105|Yesterday little troubles get matters. Black components keep perha|monthly| +862|AAAAAAAAOFDAAAAA|2450845|2450874|DEPARTMENT|8|106|American days would not achieve flat joint rules. Wrong, nuclear cla|monthly| +863|AAAAAAAAPFDAAAAA|2450845|2450874|DEPARTMENT|8|107|New, royal institutions search monetary, financial stocks. Details record just possible, joint ri|monthly| +864|AAAAAAAAAGDAAAAA|2450845|2450874|DEPARTMENT|8|108|Instead grey costs listen. New days charge yet. Often neighbouring memories shall |monthly| +865|AAAAAAAABGDAAAAA|2450875|2450904|DEPARTMENT|9|1|Circumstances hear competitive weeks. Drugs might res|monthly| +866|AAAAAAAACGDAAAAA|2450875|2450904|DEPARTMENT|9|2|Yet keen objectives sit just; minutes take simply other patients. Social, other|monthly| +867|AAAAAAAADGDAAAAA|2450875|2450904|DEPARTMENT|9|3|Forward, agricultural words elucidate immediately. Hands miss like a relations. Still asl|monthly| +868|AAAAAAAAEGDAAAAA|2450875|2450904|DEPARTMENT|9|4|Ill current publications could not dry political, alternativ|monthly| +869|AAAAAAAAFGDAAAAA|2450875|2450904|DEPARTMENT|9|5|Usually tall eyes used to go on a homes. Democratic, new months like then large tests; s|monthly| +870|AAAAAAAAGGDAAAAA|2450875|2450904|DEPARTMENT|9|6|Hours vary. Chiefs will produce now. Key, available connections go; provisions|monthly| +871|AAAAAAAAHGDAAAAA|2450875|2450904|DEPARTMENT|9|7|Alternative appointments ought to make really. Firstly high arrangeme|monthly| +872|AAAAAAAAIGDAAAAA|2450875|2450904|DEPARTMENT|9|8|Married nations apply more voluntary, only years; agreements take v|monthly| +873|AAAAAAAAJGDAAAAA|2450875|2450904|DEPARTMENT|9|9|Previously personal designs pull. Particular words shall see. Publications could maintain bitt|monthly| +874|AAAAAAAAKGDAAAAA|2450875|2450904|DEPARTMENT|9|10|New, proper households kill only once more other degrees. Importantly|monthly| +875|AAAAAAAALGDAAAAA|2450875|2450904|DEPARTMENT|9|11|Stable operators could play statistical, big troops; colours make pro|monthly| +876|AAAAAAAAMGDAAAAA|2450875|2450904|DEPARTMENT|9|12|Extensively political hands can send overall across a details. Winners ought t|monthly| +877|AAAAAAAANGDAAAAA|2450875|2450904|DEPARTMENT|9|13|Further fundamental rates must not find enough fiercely dependent services. Relevant se|monthly| +878|AAAAAAAAOGDAAAAA|2450875|2450904|DEPARTMENT|9|14|Steps take stars. Aware children could not love only at all different hours; small po|monthly| +879|AAAAAAAAPGDAAAAA|2450875|2450904|DEPARTMENT|9|15|Popular plants might not know also; happily tiny words ca|monthly| +880|AAAAAAAAAHDAAAAA|2450875|2450904|DEPARTMENT|9|16|Different colours may understand more handsome days; more a|monthly| +881|AAAAAAAABHDAAAAA|2450875|2450904||9|17|Already occasional terms follow by a meetings. Annual times may establish. Usual vote|monthly| +882|AAAAAAAACHDAAAAA|2450875|2450904|DEPARTMENT|9|18|Successful, delicate occasions should need nations. R|monthly| +883|AAAAAAAADHDAAAAA|2450875|2450904|DEPARTMENT|9|19|Red foundations should not achieve then rarely afraid ste|monthly| +884|AAAAAAAAEHDAAAAA|2450875|2450904|DEPARTMENT|9|20|Obviously whole penalties should maintain also in a cuts. Though old rights mig|monthly| +885|AAAAAAAAFHDAAAAA|2450875|2450904|DEPARTMENT|9|21|Doors shall not solve gradually. Now economic women change in particular tiny centres. New, terri|monthly| +886|AAAAAAAAGHDAAAAA|2450875|2450904|DEPARTMENT|9|22|Popular thanks make now sources. Silly, central cos|monthly| +887|AAAAAAAAHHDAAAAA|2450875|2450904|DEPARTMENT|9|23|Military legs help so significant programmes. Independent, wise parties may l|monthly| +888|AAAAAAAAIHDAAAAA|2450875|2450904|DEPARTMENT|9|24|Now rare committees will tell with the sentences. Events show seldom els|monthly| +889|AAAAAAAAJHDAAAAA|2450875|2450904|DEPARTMENT|9|25|Poles put annual, single thousands. Big nations will not rejoin however for instance do|monthly| +890|AAAAAAAAKHDAAAAA|2450875|2450904|DEPARTMENT|9|26|Critical beliefs should provide yet to the crowds. Asleep, familiar limits create meanwhile. Too|monthly| +891|AAAAAAAALHDAAAAA|2450875|2450904|DEPARTMENT|9|27|Rapidly local others go then substantial classes. Impor|monthly| +892|AAAAAAAAMHDAAAAA|2450875|2450904|DEPARTMENT|9|28|Things will not hear good eyes. Schools should contin|monthly| +893|AAAAAAAANHDAAAAA|2450875|2450904|DEPARTMENT|9|29|Following, just streets should offer less far white drinks. Medical leade|monthly| +894|AAAAAAAAOHDAAAAA|2450875|2450904|DEPARTMENT|9|30|Again interesting groups should enable certainly unable i|monthly| +895|AAAAAAAAPHDAAAAA|2450875|2450904|DEPARTMENT|9|31|Problems act only from an pupils; away labour windows try here. Previous devic|monthly| +896|AAAAAAAAAIDAAAAA|2450875|2450904|DEPARTMENT|9|32|More other countries would not show still political remaining produc|monthly| +897|AAAAAAAABIDAAAAA|2450875|2450904|DEPARTMENT|9|33|Feet will bite cool in a days. Responsible awards ought to consult |monthly| +898|AAAAAAAACIDAAAAA|2450875|2450904|DEPARTMENT|9|34|More naval armies point terrible teachers. Grey systems cannot take ah|monthly| +899|AAAAAAAADIDAAAAA|2450875|2450904|DEPARTMENT|9|35|Poor, specific characteristics shall help simply; emotio|monthly| +900|AAAAAAAAEIDAAAAA|2450875|2450904|DEPARTMENT|9|36|Areas might make national, chief stories. Changes go highly to a records. Still la|monthly| +901|AAAAAAAAFIDAAAAA|2450875|2450904|DEPARTMENT|9|37|Permanent sales earn rather families. Systems love territorial branches. Particularly red featu|monthly| +902|AAAAAAAAGIDAAAAA|2450875|2450904|DEPARTMENT|9|38|Possible, other months follow christian measures. Ov|monthly| +903|AAAAAAAAHIDAAAAA|2450875|2450904|DEPARTMENT|9|39|Others used to get please exchanges. Markets reclaim probably; enough |monthly| +904|AAAAAAAAIIDAAAAA|2450875|2450904|DEPARTMENT|9|40|Sessions think successive, statistical careers. Then other dogs could not enco|monthly| +905|AAAAAAAAJIDAAAAA|2450875|2450904|DEPARTMENT|9|41|Big standards improve urgently practically beautiful years. Hi|monthly| +906|AAAAAAAAKIDAAAAA|2450875|2450904|DEPARTMENT|9|42|Later decisive operations can differ men. Material skills cry correct walls. Alon|monthly| +907|AAAAAAAALIDAAAAA|2450875|2450904|DEPARTMENT|9|43|Sexual legs shall see away identical, coastal vehicles. Industrial, german investors used to b|monthly| +908|AAAAAAAAMIDAAAAA|2450875|2450904|DEPARTMENT|9|44|Sales enter consequences. Others would not read true,|monthly| +909|AAAAAAAANIDAAAAA|2450875|2450904|DEPARTMENT|9|45|Words appear to a areas. New, happy scales should look about free, aware men. |monthly| +910|AAAAAAAAOIDAAAAA|2450875|2450904|DEPARTMENT|9|46|Seconds say industrial centres. There gradual phenom|monthly| +911|AAAAAAAAPIDAAAAA|2450875|2450904|DEPARTMENT|9|47|Great, german investors must not come for a views. Ears ought to persuade therefore che|monthly| +912|AAAAAAAAAJDAAAAA|2450875|2450904|DEPARTMENT|9|48|Themes used to live then minor ways. Sure, left designs should not describe just b|monthly| +913|AAAAAAAABJDAAAAA|2450875|2450904|DEPARTMENT|9|49|Targets talk. Pleasant, mathematical variables will not boost. Small, functional win|monthly| +914|AAAAAAAACJDAAAAA|2450875|2450904|DEPARTMENT|9|50|Sites might expect too. Other, young members want probably |monthly| +915|AAAAAAAADJDAAAAA|2450875|2450904|DEPARTMENT|9|51|Now severe limits used to hear difficult, great reports; proteins leave quite sys|monthly| +916|AAAAAAAAEJDAAAAA|2450875|2450904|DEPARTMENT|9|52|New statements support busy, acute vessels. Occasional girls find just terri|monthly| +917|AAAAAAAAFJDAAAAA|2450875|2450904|DEPARTMENT|9|53|Later difficult points become once between a parents. Monetary, current reservations sha|monthly| +918|AAAAAAAAGJDAAAAA|2450875|2450904|DEPARTMENT|9|54|General, french notions might go very poles. Local, rational sales used to d|monthly| +919|AAAAAAAAHJDAAAAA|2450875|2450904|DEPARTMENT|9|55|Developers retain appropriate, extreme problems. Just public de|monthly| +920|AAAAAAAAIJDAAAAA|2450875|2450904|DEPARTMENT|9|56|Of course urban buildings may not become. Dead, other personnel live reasonably|monthly| +921|AAAAAAAAJJDAAAAA|2450875|2450904|DEPARTMENT|9|57|Trying services shall seem now. Clear, academic individuals help. Scholars lead probably alr|monthly| +922|AAAAAAAAKJDAAAAA|2450875|2450904|DEPARTMENT|9|58|Folk might play; main, normal effects help by a positions. Then effective|monthly| +923|AAAAAAAALJDAAAAA|2450875|2450904|DEPARTMENT|9|59|Others ring on a claims. Technological, willing firms give for a e|monthly| +924|AAAAAAAAMJDAAAAA|2450875|2450904|DEPARTMENT|9|60|Western miles can start also for a proposals. Real, wrong clients imagine at least ey|monthly| +925|AAAAAAAANJDAAAAA|2450875|2450904|DEPARTMENT|9|61|Bad, global relations present nuclear, fit objects. Front, whole arrangements |monthly| +926|AAAAAAAAOJDAAAAA|2450875|2450904|DEPARTMENT|9|62|Average, past ears must avoid always by a employees. Net factors must ma|monthly| +927|AAAAAAAAPJDAAAAA|2450875|2450904|DEPARTMENT|9|63|Warm, full symptoms could not feel. However other walls can want spec|monthly| +928|AAAAAAAAAKDAAAAA|2450875|2450904|DEPARTMENT|9|64|Political germans attend for the mechanisms. Honest police |monthly| +929|AAAAAAAABKDAAAAA|2450875|2450904|DEPARTMENT|9|65|Perhaps particular men fill. Only accused prices determine now al|monthly| +930|AAAAAAAACKDAAAAA|2450875|2450904|DEPARTMENT|9|66|Dear, financial eggs clear closely huge characteristics. Senior we|monthly| +931|AAAAAAAADKDAAAAA|2450875|2450904|DEPARTMENT|9|67|Honest steps tell probably only years. Exact, local variables subscribe well isolated |monthly| +932|AAAAAAAAEKDAAAAA|2450875|2450904|DEPARTMENT|9|68|Publications should not last huskily in a houses. Long tanks like large-scale familie|monthly| +933|AAAAAAAAFKDAAAAA|2450875|2450904|DEPARTMENT|9|69|As personal friends would prevent more just succes|monthly| +934|AAAAAAAAGKDAAAAA|2450875|2450904|DEPARTMENT|9|70|Little, other women turn there national characters. Level reserva|monthly| +935|AAAAAAAAHKDAAAAA|2450875|2450904|DEPARTMENT|9|71|Straightforward, excessive executives ought to exceed |monthly| +936|AAAAAAAAIKDAAAAA|2450875|2450904|DEPARTMENT|9|72|From time to time visible boards stand offices; func|monthly| +937|AAAAAAAAJKDAAAAA|2450875|2450904|DEPARTMENT|9|73|Blue, unlike methods must make buses; more secret eyes furnish perhaps christian wounds. Even les|monthly| +938|AAAAAAAAKKDAAAAA|2450875|2450904|DEPARTMENT|9|74|At all present addresses find major, busy members. Now only dates make properly|monthly| +939|AAAAAAAALKDAAAAA|2450875|2450904|DEPARTMENT|9|75|Residents should not go artists. However late processes would release white windows. Here|monthly| +940|AAAAAAAAMKDAAAAA|2450875|2450904|DEPARTMENT|9|76|Right, full seconds should shop now indeed social faces. Interests fill now in a |monthly| +941|AAAAAAAANKDAAAAA|2450875|2450904|DEPARTMENT|9|77|New relations might run eagerly limited systems. Gently free tra|monthly| +942|AAAAAAAAOKDAAAAA|2450875|2450904|DEPARTMENT|9|78|Expectations shall not agree eyes. Rates might tolerate prime applicants. Just easy claims co|monthly| +943|AAAAAAAAPKDAAAAA|2450875|2450904|DEPARTMENT|9|79|Guests will administer special, only skills. Swiss years could get|monthly| +944|AAAAAAAAALDAAAAA|2450875|2450904|DEPARTMENT|9|80|Men want more disabled friends. Major, common programs worry successful colleges; economic, stro|monthly| +945|AAAAAAAABLDAAAAA|2450875|2450904|DEPARTMENT|9|81|Games could say tomorrow equivalent products. Parties check around rura|monthly| +946|AAAAAAAACLDAAAAA|2450875|2450904|DEPARTMENT|9|82|Sure directors must come instead large, final benefits; important poets may say as. Regional, ambi|monthly| +947|AAAAAAAADLDAAAAA|2450875|2450904|DEPARTMENT|9|83|Talks will not take in a states. Feet put orange, possible years. |monthly| +948|AAAAAAAAELDAAAAA|2450875|2450904|DEPARTMENT|9|84|Recently necessary guests appear exactly soft offices. Similar, full worke|monthly| +949|AAAAAAAAFLDAAAAA|2450875|2450904|DEPARTMENT|9|85|Evolutionary volunteers can cope about low usual groups. Adequa|monthly| +950|AAAAAAAAGLDAAAAA|2450875|2450904|DEPARTMENT|9|86|Early troops might not think local, medical teachers. Young|monthly| +951|AAAAAAAAHLDAAAAA|2450875|2450904|DEPARTMENT|9|87|Then inadequate years must not cost other, recent villages. Well political experiences would |monthly| +952|AAAAAAAAILDAAAAA|2450875|2450904|DEPARTMENT|9|88|Publicly mathematical operations use women. New groups open partly at a students. Financial |monthly| +953|AAAAAAAAJLDAAAAA|2450875|2450904|DEPARTMENT|9|89|Prematurely modern blocks must not convince close to a places. References shall not|monthly| +954|AAAAAAAAKLDAAAAA|2450875|2450904|DEPARTMENT|9|90|Main, formal prisoners fill tomorrow deep years. Now other countries m|monthly| +955|AAAAAAAALLDAAAAA|2450875|2450904|DEPARTMENT|9|91|Critical lights could not like now likely individuals. Producers must |monthly| +956|AAAAAAAAMLDAAAAA|2450875|2450904|DEPARTMENT|9|92|Often independent courses tell out of a children; only foll|monthly| +957|AAAAAAAANLDAAAAA|2450875|2450904|DEPARTMENT|9|93|Western, low parents matter formerly new, racial germans. Old, brit|monthly| +958|AAAAAAAAOLDAAAAA|2450875|2450904|DEPARTMENT|9|94|National, sure areas review. Much financial bombs walk les|monthly| +959|AAAAAAAAPLDAAAAA|2450875|2450904|DEPARTMENT|9|95|Continuous men get large pounds. So rich parents should end at least other oth|monthly| +960|AAAAAAAAAMDAAAAA|2450875|2450904|DEPARTMENT|9|96|Decades need severely highly great groups. Individual, flexib|monthly| +961|AAAAAAAABMDAAAAA|2450875|2450904|DEPARTMENT|9|97|Horizontal, early objectives justify generally. Attempts avoid once main circumstances. Civ|monthly| +962|AAAAAAAACMDAAAAA|2450875|2450904|DEPARTMENT|9|98|Full mice combat automatically ill children. Also different t|monthly| +963|AAAAAAAADMDAAAAA|2450875|2450904|DEPARTMENT|9|99|Significant views might not ease; presidential, brief|monthly| +964|AAAAAAAAEMDAAAAA|2450875|2450904|DEPARTMENT|9|100|Available plants resign cool by an women. Markets induce separate, f|monthly| +965|AAAAAAAAFMDAAAAA|2450875|2450904|DEPARTMENT|9|101|Much main methods used to think certain, only others. Blue |monthly| +966|AAAAAAAAGMDAAAAA|2450875|2450904|DEPARTMENT|9|102|Industries could not mix at all temporary, current questions; o|monthly| +967|AAAAAAAAHMDAAAAA|2450875|2450904|DEPARTMENT|9|103|Now successful sales understand along with a arms. Sh|monthly| +968|AAAAAAAAIMDAAAAA|2450875|2450904|DEPARTMENT|9|104|True forces make nearly to the proceedings. Only labour things act clear members. Ethnic|monthly| +969|AAAAAAAAJMDAAAAA|2450875|2450904|DEPARTMENT|9|105|Major marks would feel more. Very asleep legs ensure authorities. Parents will bur|monthly| +970|AAAAAAAAKMDAAAAA|2450875|2450904|DEPARTMENT|9|106|Inc ranks will not imply then away heavy faces. Particular authorit|monthly| +971|AAAAAAAALMDAAAAA|2450875|2450904|DEPARTMENT|9|107|Anyway ill women last responsible minutes. Global childre|monthly| +972|AAAAAAAAMMDAAAAA|2450875|2450904|DEPARTMENT|9|108|Papers go only military, separate rules. Cold actors should walk successfully considerable f|monthly| +973|AAAAAAAANMDAAAAA|2450905|2450934|DEPARTMENT|10|1|Leaders protect chinese characteristics; already practical|monthly| +974|AAAAAAAAOMDAAAAA|2450905|2450934|DEPARTMENT|10|2|Societies monitor members. Negotiations say american, short areas; local parts avoid so f|monthly| +975|AAAAAAAAPMDAAAAA|2450905|2450934|DEPARTMENT|10|3|Alone courts must mean very features. National factors used|monthly| +976|AAAAAAAAANDAAAAA|2450905|2450934|DEPARTMENT|10|4|Free windows must not allow sentences; ready, small cases escape formerly public children; tr|monthly| +977|AAAAAAAABNDAAAAA|2450905|2450934|DEPARTMENT|10|5|Secret yards tap so immediate hands. Possible initia|monthly| +978|AAAAAAAACNDAAAAA|2450905|2450934|DEPARTMENT|10|6|Hours redeem elements; respectively statutory words go of course dangers. Still different pr|monthly| +979|AAAAAAAADNDAAAAA|2450905|2450934|DEPARTMENT|10|7|Poor, usual questions would last quite top advantages. Developments get surely from a sounds. Oc|monthly| +980|AAAAAAAAENDAAAAA|2450905|2450934|DEPARTMENT|10|8|Unfortunately sure industries shall not feel together inland, appropri|monthly| +981|AAAAAAAAFNDAAAAA|2450905|2450934|DEPARTMENT|10|9|Customers might produce more hence tall parties. Slowly goo|monthly| +982|AAAAAAAAGNDAAAAA|2450905|2450934|DEPARTMENT|10|10|At last important locations possess intellectual, only stories. Sorry standards cou|monthly| +983|AAAAAAAAHNDAAAAA|2450905|2450934|DEPARTMENT|10|11|Remaining farmers must want once more tight lively colleges. Ready, other years appeal ac|monthly| +984|AAAAAAAAINDAAAAA|2450905|2450934|DEPARTMENT|10|12|Long, central americans cannot take thoroughly. Other|monthly| +985|AAAAAAAAJNDAAAAA|2450905|2450934|DEPARTMENT|10|13|Men take usually out of a relationships. Likely officers become so other clothe|monthly| +986|AAAAAAAAKNDAAAAA|2450905|2450934|DEPARTMENT|10|14|Beautifully lengthy minutes require newly. Forces ought t|monthly| +987|AAAAAAAALNDAAAAA|2450905|2450934|DEPARTMENT|10|15|Mean, bad transactions should bring. Standards will get much military, real shares. Passages |monthly| +988|AAAAAAAAMNDAAAAA|2450905|2450934|DEPARTMENT|10|16|Old seconds ride nearly against the sites. Different words undertake further views. Responsibili|monthly| +989|AAAAAAAANNDAAAAA|2450905|2450934|DEPARTMENT|10|17|Normal victims save only new years. Lips provide quietly; difficult, weekly buses en|monthly| +990|AAAAAAAAONDAAAAA|2450905|2450934|DEPARTMENT|10|18|Just real prisons could not rewrite already poor jobs. Specific children can appea|monthly| +991|AAAAAAAAPNDAAAAA|2450905|2450934|DEPARTMENT|10|19|Forces make even in a figures. Novel years will die in|monthly| +992|AAAAAAAAAODAAAAA|2450905|2450934|DEPARTMENT|10|20|Numbers help just measures. Very similar qualifications could not play all unusual con|monthly| +993|AAAAAAAABODAAAAA|2450905|2450934|DEPARTMENT|10|21|Slowly suitable messages go certainly with the activities; things show sadly social, polish p|monthly| +994|AAAAAAAACODAAAAA|2450905|2450934|DEPARTMENT|10|22|Only vast meals rule from a tales. American, necessary cars clean determined, ancient offices. Po|monthly| +995|AAAAAAAADODAAAAA|2450905|2450934|DEPARTMENT|10|23|Changes might not frustrate just new cattle. Main, new successes think afraid, primary policies. |monthly| +996|AAAAAAAAEODAAAAA|2450905|2450934|DEPARTMENT|10|24|Regular effects introduce real, past eyes. Absolutely reliable reasons pro|monthly| +997|AAAAAAAAFODAAAAA|2450905|2450934|DEPARTMENT|10|25|Vast eyes approach about present, useful women. Fiscal,|monthly| +998|AAAAAAAAGODAAAAA|2450905|2450934|DEPARTMENT|10|26|Inherent members should not save difficult, other prob|monthly| +999|AAAAAAAAHODAAAAA|2450905|2450934|DEPARTMENT|10|27|Things increase already attacks. As other areas should be|monthly| +1000|AAAAAAAAIODAAAAA|2450905|2450934|DEPARTMENT|10|28|Assets bring academic writers. Downstairs clear women will |monthly| +1001|AAAAAAAAJODAAAAA|2450905|2450934|DEPARTMENT|10|29|Common, faint cattle say for a questions. Personal policies show. Vague, economic plans imp|monthly| +1002|AAAAAAAAKODAAAAA|2450905|2450934|DEPARTMENT|10|30|Normally full neighbours ought to take a little litt|monthly| +1003|AAAAAAAALODAAAAA|2450905|2450934|DEPARTMENT|10|31|Available, turkish interests pursue huge months. Also defensive women should dig suitably. Neig|monthly| +1004|AAAAAAAAMODAAAAA|2450905|2450934|DEPARTMENT|10|32|Legal flames would help by the operations; new, early ti|monthly| +1005|AAAAAAAANODAAAAA|2450905|2450934|DEPARTMENT|10|33|Long, small files look well public, local friends. Then political minutes can consent|monthly| +1006|AAAAAAAAOODAAAAA|2450905|2450934|DEPARTMENT|10|34|Natural, male languages promote sometimes only kids. Important, flexible views must|monthly| +1007|AAAAAAAAPODAAAAA|2450905|2450934|DEPARTMENT|10|35|Probably sick times sound. Labour pictures provide states. Different sides |monthly| +1008|AAAAAAAAAPDAAAAA|2450905|2450934|DEPARTMENT|10|36|Terrible miles go ever for a feet. Fast situations desire enough badly blue |monthly| +1009|AAAAAAAABPDAAAAA|2450905|2450934|DEPARTMENT|10|37|Then permanent orders would join carefully once local drugs. Revolut|monthly| +1010|AAAAAAAACPDAAAAA|2450905|2450934|DEPARTMENT|10|38|Japanese difficulties would ask home video-taped changes. Schools carry little characteristics. Fo|monthly| +1011|AAAAAAAADPDAAAAA|2450905|2450934|DEPARTMENT|10|39|British ways emerge prior to a units; students question most questions. Odd arguments go conti|monthly| +1012|AAAAAAAAEPDAAAAA|2450905|2450934|DEPARTMENT|10|40|Full, golden bodies could become above only major guests. Restric|monthly| +1013|AAAAAAAAFPDAAAAA|2450905|2450934|DEPARTMENT|10|41|Complex economies go independently also wet issues; unique, systematic trusts |monthly| +1014|AAAAAAAAGPDAAAAA|2450905|2450934|DEPARTMENT|10|42|Long, european eyes might see on a cattle. At least local |monthly| +1015|AAAAAAAAHPDAAAAA|2450905|2450934|DEPARTMENT|10|43|Just western conservatives work probably from a days. Individuals work often in a jobs. Even |monthly| +1016|AAAAAAAAIPDAAAAA|2450905|2450934|DEPARTMENT|10|44|Illegal others complete. Things ought to become monthly metres. Short-t|monthly| +1017|AAAAAAAAJPDAAAAA|2450905|2450934|DEPARTMENT|10|45|Just similar owners see. National specialists maint|monthly| +1018|AAAAAAAAKPDAAAAA|2450905|2450934|DEPARTMENT|10|46|Drivers drop in order. Effects shall think under a shops. Very unive|monthly| +1019|AAAAAAAALPDAAAAA|2450905|2450934|DEPARTMENT|10|47|Often simple fears might not sustain now organisations; important, yellow parents take |monthly| +1020|AAAAAAAAMPDAAAAA|2450905|2450934|DEPARTMENT|10|48|Various members may not appeal again fat, royal clothes. Objects |monthly| +1021|AAAAAAAANPDAAAAA|2450905|2450934|DEPARTMENT|10|49|Times may not decrease chief, strong tonnes; russian, immediate authorities bear at l|monthly| +1022|AAAAAAAAOPDAAAAA|2450905|2450934|DEPARTMENT|10|50|Typical doubts ought to make comparable signs; nati|monthly| +1023|AAAAAAAAPPDAAAAA|2450905|2450934|DEPARTMENT|10|51|Careful, new forms should help during a stages. Gre|monthly| +1024|AAAAAAAAAAEAAAAA|2450905|2450934|DEPARTMENT|10|52|Only, other men rally there wild problems. Hardly only possib|monthly| +1025|AAAAAAAABAEAAAAA|2450905|2450934|DEPARTMENT|10|53|Positive, generous conventions may match active, polish names. Usually moral reasons might not|monthly| +1026|AAAAAAAACAEAAAAA|2450905|2450934|DEPARTMENT|10|54|Great months could enter there for a schemes. Spectacular, other elections take above obvious, c|monthly| +1027|AAAAAAAADAEAAAAA|2450905|2450934|DEPARTMENT|10|55|Developing forces shall believe; however great shares co|monthly| +1028|AAAAAAAAEAEAAAAA|2450905|2450934|DEPARTMENT|10|56|Distinctive managers might keep. Polite, likely sites vote away on a cha|monthly| +1029|AAAAAAAAFAEAAAAA|2450905|2450934|DEPARTMENT|10|57|Professional, expert tenants say final, new ways. Social friends pay|monthly| +1030|AAAAAAAAGAEAAAAA|2450905|2450934|DEPARTMENT|10|58|Famous, important animals will work that now social |monthly| +1031|AAAAAAAAHAEAAAAA|2450905|2450934|DEPARTMENT|10|59|Other changes might request individually; other pupils may fight above movements. Gene|monthly| +1032|AAAAAAAAIAEAAAAA|2450905|2450934|DEPARTMENT|10|60|Groups may not assist gifts. Proudly large lives b|monthly| +1033|AAAAAAAAJAEAAAAA|2450905|2450934|DEPARTMENT|10|61|Most unique others help sometimes between the subjects. Effective values will n|monthly| +1034|AAAAAAAAKAEAAAAA|2450905|2450934|DEPARTMENT|10|62|Final articles resolve yet relations; big, distant dealers s|monthly| +1035|AAAAAAAALAEAAAAA|2450905|2450934|DEPARTMENT|10|63|Young cultures know also hands. Services used to examine; now external terms should not go much s|monthly| +1036|AAAAAAAAMAEAAAAA|2450905|2450934|DEPARTMENT|10|64|Active, good officers must not look rather never rapid book|monthly| +1037|AAAAAAAANAEAAAAA|2450905|2450934|DEPARTMENT|10|65|Hands used to regard. New, important papers sound high, strange e|monthly| +1038|AAAAAAAAOAEAAAAA|2450905|2450934|DEPARTMENT|10|66|Beyond formal proposals should give central, effective trouse|monthly| +1039|AAAAAAAAPAEAAAAA|2450905|2450934|DEPARTMENT|10|67|Important, practical photographs should scare numbers. Australian waves would play impatiently eff|monthly| +1040|AAAAAAAAABEAAAAA|2450905|2450934|DEPARTMENT|10|68|Splendid, labour divisions accompany british, aware officials.|monthly| +1041|AAAAAAAABBEAAAAA|2450905|2450934|DEPARTMENT|10|69|Great companies may jeopardise usually small programmes; pag|monthly| +1042|AAAAAAAACBEAAAAA|2450905|2450934|DEPARTMENT|10|70|Bodies avoid countries. Most effective members ought to bother german, significant g|monthly| +1043|AAAAAAAADBEAAAAA|2450905|2450934|DEPARTMENT|10|71|For instance careful women used to see both correct attempts. Possible specialists us|monthly| +1044|AAAAAAAAEBEAAAAA|2450905|2450934|DEPARTMENT|10|72|Before available things will check about the schemes. Only old men o|monthly| +1045|AAAAAAAAFBEAAAAA|2450905|2450934|DEPARTMENT|10|73|Indians serve perhaps obvious classes. Remaining, following parties must not provide. Small|monthly| +1046|AAAAAAAAGBEAAAAA|2450905|2450934|DEPARTMENT|10|74|Marine areas grow plans. Wonderful areas must recall almost. Students get as alternative ter|monthly| +1047|AAAAAAAAHBEAAAAA|2450905|2450934|DEPARTMENT|10|75|More frequent words haul good children. Mistakes must prom|monthly| +1048|AAAAAAAAIBEAAAAA|2450905|2450934|DEPARTMENT|10|76|Both inevitable officers must not get well directly white gifts; little services should no|monthly| +1049|AAAAAAAAJBEAAAAA|2450905|2450934|DEPARTMENT|10|77|Contracts remember directly traditional girls. Frequently co|monthly| +1050|AAAAAAAAKBEAAAAA|2450905|2450934|DEPARTMENT|10|78|Important, vital tonnes must stay. Very severe articles may not consider. Years c|monthly| +1051|AAAAAAAALBEAAAAA|2450905|2450934|DEPARTMENT|10|79|Therefore clear parties need funny authorities. Rich plans ought to miss tough|monthly| +1052|AAAAAAAAMBEAAAAA|2450905|2450934|DEPARTMENT|10|80|Views include initially relationships; disciplinary campaigns e|monthly| +1053|AAAAAAAANBEAAAAA|2450905|2450934|DEPARTMENT|10|81|More like facts see mistakes. Other, rare men ought to sue different, go|monthly| +1054|AAAAAAAAOBEAAAAA|2450905|2450934|DEPARTMENT|10|82|Rich rates may not give further in particular liberal hours. Forward simple images shall bury c|monthly| +1055|AAAAAAAAPBEAAAAA|2450905|2450934|DEPARTMENT|10|83|Average, different forces deserve for the operations. Low, useful attempts would hear o|monthly| +1056|AAAAAAAAACEAAAAA|2450905|2450934|DEPARTMENT|10|84|Cases should mitigate. Consequences ought to choose hardly with a year|monthly| +1057|AAAAAAAABCEAAAAA|2450905|2450934|DEPARTMENT|10|85|Tremendous cases mind amazingly different societies; whole, quick bones use yet characters|monthly| +1058|AAAAAAAACCEAAAAA|2450905|2450934|DEPARTMENT|10|86|European, parental views merit such as a activities. Labour, financial ap|monthly| +1059|AAAAAAAADCEAAAAA|2450905|2450934|DEPARTMENT|10|87|International, relevant germans intend barely later national courts. New, similar reco|monthly| +1060|AAAAAAAAECEAAAAA|2450905|2450934|DEPARTMENT|10|88|Great, willing areas test to a wages. Children could not observe mean, possible thi|monthly| +1061|AAAAAAAAFCEAAAAA|2450905|2450934|DEPARTMENT|10|89|Ambitious services shall count. Silent colours increase. Samples get hardly to a police; |monthly| +1062|AAAAAAAAGCEAAAAA|2450905|2450934|DEPARTMENT|10|90|Now relative facilities travel close sure official patients. Sociali|monthly| +1063|AAAAAAAAHCEAAAAA|2450905|2450934|DEPARTMENT|10|91|Fair huge pressures might switch more miles. Ministers lead just old count|monthly| +1064|AAAAAAAAICEAAAAA|2450905|2450934|DEPARTMENT|10|92|Societies shall reassure members. Ever accessible farmers may not give se|monthly| +1065|AAAAAAAAJCEAAAAA|2450905|2450934|DEPARTMENT|10|93|More great powers must enable just new, bad groups. Large, appropriate hours will buy alternative p|monthly| +1066|AAAAAAAAKCEAAAAA|2450905|2450934|DEPARTMENT|10|94|Together commercial concerns help automatically other years. Open investors will affect in a lang|monthly| +1067|AAAAAAAALCEAAAAA|2450905|2450934|DEPARTMENT|10|95|Other months leave also documents. Head, selective errors must go at last arts. New courses may |monthly| +1068|AAAAAAAAMCEAAAAA|2450905|2450934|DEPARTMENT|10|96|Lost, real numbers expand songs. Perfect, occupational actions show t|monthly| +1069|AAAAAAAANCEAAAAA|2450905|2450934|DEPARTMENT|10|97|Thousands afford for the improvements. Of course usual persons would travel strongly a|monthly| +1070|AAAAAAAAOCEAAAAA|2450905|2450934|DEPARTMENT|10|98|Low, oral jobs look badly alone pairs. Resources driv|monthly| +1071|AAAAAAAAPCEAAAAA|2450905|2450934|DEPARTMENT|10|99|Simple, only fees cannot save. Relationships shall not see; patients worry under the results; f|monthly| +1072|AAAAAAAAADEAAAAA|2450905|2450934|DEPARTMENT|10|100|Other ways could not want industrial, stupid standards. Ma|monthly| +1073|AAAAAAAABDEAAAAA|2450905|2450934|DEPARTMENT|10|101|Now great reasons would visit well rules. Round un|monthly| +1074|AAAAAAAACDEAAAAA|2450905|2450934|DEPARTMENT|10|102|Large, serious facilities might lose new, interesting st|monthly| +1075|AAAAAAAADDEAAAAA|2450905|2450934|DEPARTMENT|10|103|Earlier true others will think at all medical following letters. Ever national |monthly| +1076|AAAAAAAAEDEAAAAA|2450905|2450934|DEPARTMENT|10|104|In addition simple communities come at a patients. Free, wild directions cannot l|monthly| +1077|AAAAAAAAFDEAAAAA|2450905|2450934|DEPARTMENT|10|105|Much neutral services will make also conventional little demands. Professional words can encoura|monthly| +1078|AAAAAAAAGDEAAAAA|2450905|2450934|DEPARTMENT|10|106|Special times shall deliver never tiny, common states. Champions may not k|monthly| +1079|AAAAAAAAHDEAAAAA|2450905|2450934|DEPARTMENT|10|107|Social risks worry large, main members. Densely alone authoritie|monthly| +1080|AAAAAAAAIDEAAAAA|2450905|2450934|DEPARTMENT|10|108|Low, wild skills think now decent provinces; cultural, key weeks |monthly| +1081|AAAAAAAAJDEAAAAA|2450935|2450964|DEPARTMENT|11|1|Finally rich members will move knees. Mean, right things stay also numerous females; resour|monthly| +1082|AAAAAAAAKDEAAAAA|2450935|2450964|DEPARTMENT|11|2|Kindly perfect tasks will get duties. Figures must not risk really. Senior, expensive thanks|monthly| +1083|AAAAAAAALDEAAAAA|2450935|2450964|DEPARTMENT|11|3|Hands deliver potentially great, electronic relation|monthly| +1084|AAAAAAAAMDEAAAAA|2450935|2450964|DEPARTMENT|11|4|Generally strong calculations must not understand classes. Tiny, good words see pro|monthly| +1085|AAAAAAAANDEAAAAA|2450935|2450964|DEPARTMENT|11|5|Vastly existing times carry as seeds. Roads believe q|monthly| +1086|AAAAAAAAODEAAAAA|2450935|2450964|DEPARTMENT|11|6|Trustees go rapidly anxious considerations. Agricultural groups convince almost. Groups might bec|monthly| +1087|AAAAAAAAPDEAAAAA|2450935|2450964|DEPARTMENT|11|7|Personally possible residents may continue. More full-time hand|monthly| +1088|AAAAAAAAAEEAAAAA|2450935|2450964|DEPARTMENT|11|8|Following employees should arrange surely in a days. Social quantiti|monthly| +1089|AAAAAAAABEEAAAAA|2450935|2450964|DEPARTMENT|11|9|Preferably mechanical potatoes apply minutes; african, constitutional weeks |monthly| +1090|AAAAAAAACEEAAAAA|2450935|2450964|DEPARTMENT|11|10|Simple circumstances would take alone policies. Main years might not cost w|monthly| +1091|AAAAAAAADEEAAAAA|2450935|2450964|DEPARTMENT|11|11|Important settlements would back more british historia|monthly| +1092|AAAAAAAAEEEAAAAA|2450935|2450964|DEPARTMENT|11|12|Local, subsequent fields could not remain ages. Massive, suf|monthly| +1093|AAAAAAAAFEEAAAAA|2450935|2450964|DEPARTMENT|11|13|Different, new terms must not sound. Crazy prices shall compare really; secret figures ought to |monthly| +1094|AAAAAAAAGEEAAAAA|2450935|2450964|DEPARTMENT|11|14|Articles could levy out of a practices. Interesting limits used to wait especially basic proteins. |monthly| +1095|AAAAAAAAHEEAAAAA|2450935|2450964|DEPARTMENT|11|15|Policies cannot receive. Questions say at first. Commercial shops might get amounts. Top prices |monthly| +1096|AAAAAAAAIEEAAAAA|2450935|2450964|DEPARTMENT|11|16|Days remain eventually far tough systems. Suitable, foreign solicitors must not point rather |monthly| +1097|AAAAAAAAJEEAAAAA|2450935|2450964|DEPARTMENT|11|17|Tough, unique women promote just impossible accounts. Appropriate others set more true roses|monthly| +1098|AAAAAAAAKEEAAAAA|2450935|2450964|DEPARTMENT|11|18|Children ensure standards; communities will break im|monthly| +1099|AAAAAAAALEEAAAAA|2450935|2450964|DEPARTMENT|11|19|Liberal approaches avoid regularly details. Fun thoughts believe then. Eyes affect then police. Mos|monthly| +1100|AAAAAAAAMEEAAAAA|2450935|2450964|DEPARTMENT|11|20|Serious, great prices might deliver anyway private ideas. Proposals|monthly| +1101|AAAAAAAANEEAAAAA|2450935|2450964|DEPARTMENT|11|21|Periods should enjoy swiftly today full supplies. Techniques m|monthly| +1102|AAAAAAAAOEEAAAAA|2450935|2450964|DEPARTMENT|11|22|Aspects get now subsequent, distant poems. Similar hotels dry very. Sales stay all dark, bloody wa|monthly| +1103|AAAAAAAAPEEAAAAA|2450935|2450964|DEPARTMENT|11|23|Conventional, permanent resources take always big, tough men. Fundamental, basic kinds |monthly| +1104|AAAAAAAAAFEAAAAA|2450935|2450964|DEPARTMENT|11|24|Sectors will not turn as weeks. Actively local years might not embark often social |monthly| +1105|AAAAAAAABFEAAAAA|2450935|2450964|DEPARTMENT|11|25|Small, proper views will explain main, universal firms. Bills could act usually|monthly| +1106|AAAAAAAACFEAAAAA|2450935|2450964|DEPARTMENT|11|26|Probably keen eyes boost only young groups. Intermediat|monthly| +1107|AAAAAAAADFEAAAAA|2450935|2450964|DEPARTMENT|11|27|Publications control actually. National designs mig|monthly| +1108|AAAAAAAAEFEAAAAA|2450935|2450964|DEPARTMENT|11|28|Military, red steps say so literally changing hands. Physical, eastern communities deal neces|monthly| +1109|AAAAAAAAFFEAAAAA|2450935|2450964|DEPARTMENT|11|29|Strong kids support. Quietly reduced restrictions shall enjoy; also warm comb|monthly| +1110|AAAAAAAAGFEAAAAA|2450935|2450964|DEPARTMENT|11|30|Ports must support astonishingly equal, foreign companies. Public girls will not feel appar|monthly| +1111|AAAAAAAAHFEAAAAA|2450935|2450964|DEPARTMENT|11|31|Elderly sides meet under in a yards. Available days ought to tra|monthly| +1112|AAAAAAAAIFEAAAAA|2450935|2450964|DEPARTMENT|11|32|Knees provide views. Relative, running objectives used to resolve by a exchanges; liberal, im|monthly| +1113|AAAAAAAAJFEAAAAA|2450935|2450964|DEPARTMENT|11|33|National, young bits will produce chips. Cognitive, huge foundations should get also political chai|monthly| +1114|AAAAAAAAKFEAAAAA|2450935|2450964|DEPARTMENT|11|34|Just personal times switch perfectly. True hours may not play natural,|monthly| +1115|AAAAAAAALFEAAAAA|2450935|2450964|DEPARTMENT|11|35|Opposite meetings attract by a images. Natural, great surfaces meet. Etc |monthly| +1116|AAAAAAAAMFEAAAAA|2450935|2450964|DEPARTMENT|11|36|Police might move largely; powerful pounds should provide in a miles. Rational eyes used to |monthly| +1117|AAAAAAAANFEAAAAA|2450935|2450964|DEPARTMENT|11|37|Observations know accordingly rather than a judges. Great nations enter. As high questions i|monthly| +1118|AAAAAAAAOFEAAAAA|2450935|2450964|DEPARTMENT|11|38|Good, old lessons present never other proceedings. |monthly| +1119|AAAAAAAAPFEAAAAA|2450935|2450964|DEPARTMENT|11|39|Local banks secure otherwise. Sexual, environmental |monthly| +1120|AAAAAAAAAGEAAAAA|2450935|2450964|DEPARTMENT|11|40|Annual, present laboratories remind very. Varieties behave more past pockets. Settings g|monthly| +1121|AAAAAAAABGEAAAAA|2450935|2450964|DEPARTMENT|11|41|Often good goals might not matter new levels. Dead|monthly| +1122|AAAAAAAACGEAAAAA|2450935|2450964|DEPARTMENT|11|42|Even ideological forms take prime ministers. Perfectly living s|monthly| +1123|AAAAAAAADGEAAAAA|2450935||DEPARTMENT|11|||monthly| +1124|AAAAAAAAEGEAAAAA|2450935|2450964|DEPARTMENT|11|44|Rational conditions would not allow otherwise. Instead intellectual lips must listen. |monthly| +1125|AAAAAAAAFGEAAAAA|2450935|2450964|DEPARTMENT|11|45|Possible, domestic skills should work much final, n|monthly| +1126|AAAAAAAAGGEAAAAA|2450935|2450964|DEPARTMENT|11|46|Real, scientific questions imagine to a kids. Activities sh|monthly| +1127|AAAAAAAAHGEAAAAA|2450935|2450964|DEPARTMENT|11|47|Both personal damages would not promise back waters. Social|monthly| +1128|AAAAAAAAIGEAAAAA|2450935|2450964|DEPARTMENT|11|48|Useful, responsible thoughts cause companies. Star fields support yet on a numbers; g|monthly| +1129|AAAAAAAAJGEAAAAA|2450935|2450964|DEPARTMENT|11|49|Light, large companies pull twice regional, inner rights.|monthly| +1130|AAAAAAAAKGEAAAAA|2450935|2450964|DEPARTMENT|11|50|Provinces make then even forthcoming weapons. Local areas might w|monthly| +1131|AAAAAAAALGEAAAAA|2450935|2450964|DEPARTMENT|11|51|Others see supreme, other thousands. In particular fat facilities will|monthly| +1132|AAAAAAAAMGEAAAAA|2450935|2450964|DEPARTMENT|11|52|Practitioners own machines. Regional, obvious girls will establish all the same designs. N|monthly| +1133|AAAAAAAANGEAAAAA|2450935|2450964|DEPARTMENT|11|53|Yet unlikely homes help currently. A little other glasses see popular, professional res|monthly| +1134|AAAAAAAAOGEAAAAA|2450935|2450964|DEPARTMENT|11|54|Laws mention short campaigns. Maybe various reasons may fall again extra findings. Similar, vulnera|monthly| +1135|AAAAAAAAPGEAAAAA|2450935|2450964|DEPARTMENT|11|55|Birds must forget still with the things. Forward jewish fields might hold reasonabl|monthly| +1136|AAAAAAAAAHEAAAAA|2450935|2450964|DEPARTMENT|11|56|Large points handle only early strings. Other, important sources may depend social, h|monthly| +1137|AAAAAAAABHEAAAAA|2450935|2450964|DEPARTMENT|11|57|Artificial, soviet organs may excite objectives. Earlier free|monthly| +1138|AAAAAAAACHEAAAAA|2450935|2450964|DEPARTMENT|11|58|Marginally human developers may not settle as comfor|monthly| +1139|AAAAAAAADHEAAAAA|2450935|2450964|DEPARTMENT|11|59|National workers start. Officials ought to look growing, obvious |monthly| +1140|AAAAAAAAEHEAAAAA|2450935|2450964|DEPARTMENT|11|60|Effective difficulties know essentially on a rates. Wealthy, eco|monthly| +1141|AAAAAAAAFHEAAAAA|2450935|2450964|DEPARTMENT|11|61|Over miserable patients might alleviate perhaps most good point|monthly| +1142|AAAAAAAAGHEAAAAA|2450935|2450964|DEPARTMENT|11|62|Aspects will protest at length dead branches. Studies |monthly| +1143|AAAAAAAAHHEAAAAA|2450935|2450964|DEPARTMENT|11|63|Possibly key organizations must bring near good demands. Pieces must not |monthly| +1144|AAAAAAAAIHEAAAAA|2450935|2450964|DEPARTMENT|11|64|Happy, powerful forces train. Defiantly exclusive forms must ge|monthly| +1145|AAAAAAAAJHEAAAAA|2450935|2450964|DEPARTMENT|11|65|Only technical parents differ partly in a procedures. Computers c|monthly| +1146|AAAAAAAAKHEAAAAA|2450935|2450964|DEPARTMENT|11|66|Right, bad cells can pass at least simple frames; mass, important fans could provid|monthly| +1147|AAAAAAAALHEAAAAA|2450935|2450964|DEPARTMENT|11|67|Countries walk sincerely to the men; owners shield capable prices. Large, other metals should |monthly| +1148|AAAAAAAAMHEAAAAA|2450935|2450964|DEPARTMENT|11|68|Rather military types must not lead even real other ways. Mos|monthly| +1149|AAAAAAAANHEAAAAA|2450935|2450964|DEPARTMENT|11|69|Hardly usual spaces tackle human, wide risks. English databases move still. Other,|monthly| +1150|AAAAAAAAOHEAAAAA|2450935|2450964|DEPARTMENT|11|70|Atomic minutes know vitally. Asleep figures strengthen by a aspects. Absolutely ri|monthly| +1151|AAAAAAAAPHEAAAAA|2450935|2450964|DEPARTMENT|11|71|Branches may cost. Only big ends reduce never. International, bizarre entries exist her|monthly| +1152|AAAAAAAAAIEAAAAA|2450935|2450964|DEPARTMENT|11|72|Local, video-taped arms can seek voters. Partners complain most; rich examp|monthly| +1153|AAAAAAAABIEAAAAA|2450935|2450964|DEPARTMENT|11|73|So available towns go however bloody countries. Questions|monthly| +1154|AAAAAAAACIEAAAAA|2450935|2450964|DEPARTMENT|11|74|Electrical girls used to compete in a members; complete, other years shall not mean behind fort|monthly| +1155|AAAAAAAADIEAAAAA|2450935|2450964|DEPARTMENT|11|75|Vegetables understand still results. Identical, major characteristics |monthly| +1156|AAAAAAAAEIEAAAAA|2450935|2450964|DEPARTMENT|11|76|Clearly final consequences go. Mean hotels may detect messages. Only valid communists sh|monthly| +1157|AAAAAAAAFIEAAAAA|2450935|2450964|DEPARTMENT|11|77|Worthy, enormous cases may assess in a feelings; small subj|monthly| +1158|AAAAAAAAGIEAAAAA|2450935|2450964|DEPARTMENT|11|78|Clean children provide by a police. More odd walls|monthly| +1159|AAAAAAAAHIEAAAAA|2450935|2450964|DEPARTMENT|11|79|Features cannot refer speedily months. Just items sleep t|monthly| +1160|AAAAAAAAIIEAAAAA|2450935|2450964|DEPARTMENT|11|80|Dead jeans crash spectacular, poor dreams. Local, related years keep with a consequences.|monthly| +1161|AAAAAAAAJIEAAAAA|2450935|2450964|DEPARTMENT|11|81|Responsible techniques would love so important days. Effects would hide likely, economic auth|monthly| +1162|AAAAAAAAKIEAAAAA|2450935|2450964|DEPARTMENT|11|82|Certainly private windows might not fulfil simply correct pr|monthly| +1163|AAAAAAAALIEAAAAA|2450935|2450964|DEPARTMENT|11|83|Children ought to refer best clear cases. Sufficiently sp|monthly| +1164|AAAAAAAAMIEAAAAA|2450935|2450964|DEPARTMENT|11|84|Lectures cry properly most total mistakes; senior keys can eliminate to a hands|monthly| +1165|AAAAAAAANIEAAAAA|2450935|2450964|DEPARTMENT|11|85|Major windows will expect about away mean dates; places use q|monthly| +1166|AAAAAAAAOIEAAAAA|2450935|2450964|DEPARTMENT|11|86|Studies offer socially disabled applications. At last satisfactory s|monthly| +1167|AAAAAAAAPIEAAAAA|2450935|2450964|DEPARTMENT|11|87|Western rocks ought to examine at least free, various relations. Neither traditional gra|monthly| +1168|AAAAAAAAAJEAAAAA|2450935|2450964|DEPARTMENT|11|88|International girls gain unique, democratic sciences. Hard |monthly| +1169|AAAAAAAABJEAAAAA|2450935|2450964|DEPARTMENT|11|89|Bodies can add other, little supporters; different months ought to pro|monthly| +1170|AAAAAAAACJEAAAAA|2450935|2450964|DEPARTMENT|11|90|Pale, high ways share under to a patients. White, middl|monthly| +1171|AAAAAAAADJEAAAAA|2450935|2450964|DEPARTMENT|11|91|Clubs will note. Both financial numbers beat just perhaps good w|monthly| +1172|AAAAAAAAEJEAAAAA|||DEPARTMENT|11||Heavy points cannot leave political products; changes contain so in the rocks. New|monthly| +1173|AAAAAAAAFJEAAAAA|2450935|2450964|DEPARTMENT|11|93|Equal buildings must run in order centres. Players keep high, young taxes; prime, occupa|monthly| +1174|AAAAAAAAGJEAAAAA|2450935|2450964|DEPARTMENT|11|94|Examples may allow most empty employees. Areas attend of course|monthly| +1175|AAAAAAAAHJEAAAAA|2450935|2450964|DEPARTMENT|11|95|Best private trees ought to release normal charges. Divisions can fi|monthly| +1176|AAAAAAAAIJEAAAAA|2450935|2450964|DEPARTMENT|11|96|In order important days know so deaf, minute notes. Features|monthly| +1177|AAAAAAAAJJEAAAAA|2450935|2450964|DEPARTMENT|11|97|Beautiful, sexual days will not pay good shares. Studies may not retire narrow, internation|monthly| +1178|AAAAAAAAKJEAAAAA|2450935|2450964|DEPARTMENT|11|98|Deliberately strong differences shall not seem then large, other lips|monthly| +1179|AAAAAAAALJEAAAAA|2450935|2450964|DEPARTMENT|11|99|Friendly, awful arrangements can lead soon right, critical pressures. Rich questions keep posit|monthly| +1180|AAAAAAAAMJEAAAAA|2450935|2450964|DEPARTMENT|11|100|Angles might not describe in the futures; local words should offset however commercia|monthly| +1181|AAAAAAAANJEAAAAA|2450935|2450964|DEPARTMENT|11|101|For example bitter years ought to force so grim, cu|monthly| +1182|AAAAAAAAOJEAAAAA|2450935|2450964|DEPARTMENT|11|102|Very industrial men should not market contributions; |monthly| +1183|AAAAAAAAPJEAAAAA|2450935|2450964|DEPARTMENT|11|103|Weak others shall not get however finally other circumstances. Different users shoul|monthly| +1184|AAAAAAAAAKEAAAAA|2450935|2450964|DEPARTMENT|11|104|As recent companies will regard namely agricultural, local boxes. Standard, roman groups br|monthly| +1185|AAAAAAAABKEAAAAA|2450935|2450964|DEPARTMENT|11|105|Ties may work as. Females mean in an terms. Important, only developments attempt more specific de|monthly| +1186|AAAAAAAACKEAAAAA|2450935|2450964|DEPARTMENT|11|106|Special, big programmes may open. National, only products will insist away from th|monthly| +1187|AAAAAAAADKEAAAAA|2450935|2450964|DEPARTMENT|11|107|Tiles could get. There imperial employees must go rather |monthly| +1188|AAAAAAAAEKEAAAAA|2450935|2450964|DEPARTMENT|11|108|Truly big others differentiate; in between past students get there ideal, scottish |monthly| +1189|AAAAAAAAFKEAAAAA|2450965|2450994|DEPARTMENT|12|1|Main pages cannot read very; other circumstances conclude often days. Nu|monthly| +1190|AAAAAAAAGKEAAAAA|2450965|2450994|DEPARTMENT|12|2|Creditors cannot render most companies. Relatively concerned types|monthly| +1191|AAAAAAAAHKEAAAAA|2450965|2450994|DEPARTMENT|12|3|Drastically familiar brothers could seem days; stones might not|monthly| +1192|AAAAAAAAIKEAAAAA|2450965|2450994|DEPARTMENT|12|4|Vocational lists carry however civil products. Very able tears should not distinguish still there|monthly| +1193|AAAAAAAAJKEAAAAA|2450965|2450994|DEPARTMENT|12|5|Voluntary, supposed rules know thousands. English regions might discov|monthly| +1194|AAAAAAAAKKEAAAAA|2450965|2450994|DEPARTMENT|12|6|Politely innocent flats ban also other women. Notes may forgi|monthly| +1195|AAAAAAAALKEAAAAA|2450965|2450994|DEPARTMENT|12|7|At all complete men miss administrative, fair customers. Finally ca|monthly| +1196|AAAAAAAAMKEAAAAA|2450965|2450994|DEPARTMENT|12|8|Important, high bits shall study only political organisers. Cr|monthly| +1197|AAAAAAAANKEAAAAA|2450965|2450994|DEPARTMENT|12|9|Large-scale differences think completely cultural, distinctive countries. More mass|monthly| +1198|AAAAAAAAOKEAAAAA|2450965|2450994|DEPARTMENT|12|10|Users would overcome too for a years. Rather jewish boxes get great skills.|monthly| +1199|AAAAAAAAPKEAAAAA|2450965|2450994|DEPARTMENT|12|11|Wrongly concerned parents will reduce softly. Concepts matter more views; distant eggs refuse |monthly| +1200|AAAAAAAAALEAAAAA|2450965|2450994|DEPARTMENT|12|12|Prime, new agents say naturally social men. Particularly male|monthly| +1201|AAAAAAAABLEAAAAA|2450965|2450994|DEPARTMENT|12|13|Little, able bones must settle; royal modules might find only citizens|monthly| +1202|AAAAAAAACLEAAAAA|2450965|2450994|DEPARTMENT|12|14|Here long processes improve however policies. Probably national holes would apply howeve|monthly| +1203|AAAAAAAADLEAAAAA|2450965|2450994|DEPARTMENT|12|15|Random, responsible questions forgive as techniques. Male, lo|monthly| +1204|AAAAAAAAELEAAAAA|2450965|2450994|DEPARTMENT|12|16|Financial contents drive dead principles. No doubt necessary circumstances should become now worldw|monthly| +1205|AAAAAAAAFLEAAAAA|2450965|2450994|DEPARTMENT|12|17|Double teachers used to cut. Able times shall not conjure at least enough national propert|monthly| +1206|AAAAAAAAGLEAAAAA|2450965|2450994|DEPARTMENT|12|18|Separately powerful years could flee now talks. Bc internationa|monthly| +1207|AAAAAAAAHLEAAAAA|2450965|2450994|DEPARTMENT|12|19|Alone, surprised reasons shall decide satisfactorily. True subjects could mak|monthly| +1208|AAAAAAAAILEAAAAA|2450965|2450994|DEPARTMENT|12|20|Theoretical, international households cost strongly strangers. Servants create|monthly| +1209|AAAAAAAAJLEAAAAA|2450965|2450994|DEPARTMENT|12|21|Firmly evolutionary homes marry hardly. Normal, extraordinary paintings might not get european, |monthly| +1210|AAAAAAAAKLEAAAAA|2450965|2450994|DEPARTMENT|12|22|Successful cars call slowly powerful, proper things. Home|monthly| +1211|AAAAAAAALLEAAAAA|2450965|2450994|DEPARTMENT|12|23|Manufacturers may tell always dead implications. Elements may make again cards. Much|monthly| +1212|AAAAAAAAMLEAAAAA|2450965|2450994|DEPARTMENT|12|24|General, lost lands might not find positively; dead aspects go in a interes|monthly| +1213|AAAAAAAANLEAAAAA|2450965|2450994|DEPARTMENT|12|25|Formal, continued deals cannot leave sure so false women; |monthly| +1214|AAAAAAAAOLEAAAAA|2450965|2450994|DEPARTMENT|12|26|Sexually full rates should come here enthusiastically low women. Rural events vote a lit|monthly| +1215|AAAAAAAAPLEAAAAA|2450965|2450994|DEPARTMENT|12|27|Subsequently spatial walls would not carry still sc|monthly| +1216|AAAAAAAAAMEAAAAA|2450965|2450994|DEPARTMENT|12|28|Local forces heal; real systems used to choose perhaps difficult|monthly| +1217|AAAAAAAABMEAAAAA|2450965|2450994|DEPARTMENT|12|29|Allowances depend on a details. Utterly new reasons remember just as central wo|monthly| +1218|AAAAAAAACMEAAAAA|2450965|2450994|DEPARTMENT|12|30|Wet, important stories know sort of companies. Councils could not carry enough short veh|monthly| +1219|AAAAAAAADMEAAAAA|2450965|2450994|DEPARTMENT|12|31|Only big users shall hit children. Operational limitations will bring especially|monthly| +1220|AAAAAAAAEMEAAAAA|2450965|2450994|DEPARTMENT|12|32|Urban products carry also cultural, huge friends. Poor, special plants allow. Recent masters|monthly| +1221|AAAAAAAAFMEAAAAA|2450965|2450994|DEPARTMENT|12|33|Usual, wooden words lose a little cheerfully other products. New sessions en|monthly| +1222|AAAAAAAAGMEAAAAA|2450965|2450994|DEPARTMENT|12|34|Often little branches shall win as social lakes. Schools may not make away angry characteristics. |monthly| +1223|AAAAAAAAHMEAAAAA|2450965|2450994|DEPARTMENT|12|35|Perhaps high cells see out the buyers; often related points beat. Surprisingly local frien|monthly| +1224|AAAAAAAAIMEAAAAA|2450965|2450994|DEPARTMENT|12|36|Direct, new areas see firstly small circumstances. Different dogs may not|monthly| +1225|AAAAAAAAJMEAAAAA|2450965|2450994|DEPARTMENT|12|37|Almost great copies decline mental, recent dangers. Of |monthly| +1226|AAAAAAAAKMEAAAAA|2450965|2450994|DEPARTMENT|12|38|Plants may break. Other countries shall not like just theoretical, legal issue|monthly| +1227|AAAAAAAALMEAAAAA|2450965|2450994|DEPARTMENT|12|39|Really old rounds used to start also with a moments. So clear individuals |monthly| +1228|AAAAAAAAMMEAAAAA|2450965|2450994|DEPARTMENT|12|40|Jobs give by a thoughts. Young, serious costs know forward to a facilities. Other chips can deliv|monthly| +1229|AAAAAAAANMEAAAAA|2450965|2450994|DEPARTMENT|12|41|Generally quiet criteria tackle out of a forests. Large, important leaders attend then horizontal t|monthly| +1230|AAAAAAAAOMEAAAAA|2450965|2450994|DEPARTMENT|12|42|Individuals surrender straight in a stations; customers help|monthly| +1231|AAAAAAAAPMEAAAAA|2450965|2450994|DEPARTMENT|12|43|Thousands dispose institutions. Inc areas get just spe|monthly| +1232|AAAAAAAAANEAAAAA|2450965|2450994|DEPARTMENT|12|44|Main, dirty arrangements may take parts. Thus high lips may mind precious faces. Responses happen |monthly| +1233|AAAAAAAABNEAAAAA|2450965|2450994|DEPARTMENT|12|45|International, original others should answer far s|monthly| +1234|AAAAAAAACNEAAAAA|2450965|2450994|DEPARTMENT|12|46|Other, major phenomena might not see high quick rounds. Personal, regular years observe just in|monthly| +1235|AAAAAAAADNEAAAAA|2450965|2450994|DEPARTMENT|12|47|Examples used to alert unfortunately double governments; true, useful adults will appla|monthly| +1236|AAAAAAAAENEAAAAA|2450965|2450994|DEPARTMENT|12|48|Cognitive, unacceptable forces cannot ask clinically|monthly| +1237|AAAAAAAAFNEAAAAA|2450965|2450994|DEPARTMENT|12|49|Services make strongly national years. Routine, reasonable opponents must not account more |monthly| +1238|AAAAAAAAGNEAAAAA|2450965|2450994|DEPARTMENT|12|50|Developing benefits shall feel to a companies. Days say marks. Brilliant police might t|monthly| +1239|AAAAAAAAHNEAAAAA|2450965|2450994|DEPARTMENT|12|51|Troops shall not experience great goals. Available, normal stu|monthly| +1240|AAAAAAAAINEAAAAA|2450965|2450994|DEPARTMENT|12|52|Little real awards work with a women. As marginal architects talk really rates. Si|monthly| +1241|AAAAAAAAJNEAAAAA|2450965|2450994|DEPARTMENT|12|53|Aback hostile numbers ought to feel particularly great, slow members. Fine supporters|monthly| +1242|AAAAAAAAKNEAAAAA|2450965|2450994|DEPARTMENT|12|54|English, simple feet recognise often. Moral, logical rats come then bones. Still popular sides c|monthly| +1243|AAAAAAAALNEAAAAA|2450965|2450994|DEPARTMENT|12|55|Excellent, political miles come more into the vegetables. Historical, nuclea|monthly| +1244|AAAAAAAAMNEAAAAA|2450965|2450994|DEPARTMENT|12|56|Circumstances may not put for a children. Windows used to f|monthly| +1245|AAAAAAAANNEAAAAA|2450965|2450994|DEPARTMENT|12|57|Legal, united companies buy very good, friendly organisers. Partial, economic men cut customer|monthly| +1246|AAAAAAAAONEAAAAA|2450965|2450994|DEPARTMENT|12|58|Substantially free tories get millions. Subsequent, female legs think passengers. Teacher|monthly| +1247|AAAAAAAAPNEAAAAA|2450965|2450994|DEPARTMENT|12|59|Too scottish members will open square, old questions; other quarters should afford|monthly| +1248|AAAAAAAAAOEAAAAA||2450994|DEPARTMENT||||monthly| +1249|AAAAAAAABOEAAAAA|2450965|2450994|DEPARTMENT|12|61|Round, correct items grow english, great arms. Arms put elegant, sensible strategies. M|monthly| +1250|AAAAAAAACOEAAAAA|2450965|2450994|DEPARTMENT|12|62|Children ought to establish detailed, following facts. European lights would cry |monthly| +1251|AAAAAAAADOEAAAAA|2450965|2450994|DEPARTMENT|12|63|Losses hate aware demonstrations. Types would not fly bad active responses. Permanently d|monthly| +1252|AAAAAAAAEOEAAAAA|2450965|2450994|DEPARTMENT|12|64|Video-taped, known children like actively daily nurses. High modern sche|monthly| +1253|AAAAAAAAFOEAAAAA|2450965|2450994|DEPARTMENT|12|65|Bright, japanese patterns recognise together essential, normal victims. However favorite machine|monthly| +1254|AAAAAAAAGOEAAAAA|2450965|2450994|DEPARTMENT|12|66|Sure, clear years must minimise also; unusual groups would abolish. Whole, co|monthly| +1255|AAAAAAAAHOEAAAAA|2450965|2450994|DEPARTMENT|12|67|Approximately likely representations may not find hands. Big experts cover rapidly othe|monthly| +1256|AAAAAAAAIOEAAAAA|2450965|2450994|DEPARTMENT|12|68|Certain, real hands find yet intensive, ancient sanctions. Practices allow |monthly| +1257|AAAAAAAAJOEAAAAA|2450965|2450994|DEPARTMENT|12|69|Ever dead aspects end flowers. Most economic boys remain|monthly| +1258|AAAAAAAAKOEAAAAA|2450965|2450994|DEPARTMENT|12|70|Perfect, exceptional actions used to address always on a year|monthly| +1259|AAAAAAAALOEAAAAA|2450965|2450994|DEPARTMENT|12|71|Limited, available songs must achieve legitimate ideas. R|monthly| +1260|AAAAAAAAMOEAAAAA|2450965|2450994|DEPARTMENT|12|72|Never fundamental ideas go instead. Full, national charges make regularly good calls. Exact|monthly| +1261|AAAAAAAANOEAAAAA|2450965|2450994|DEPARTMENT|12|73|Social parties could not know. Colleges say metropolitan stores. Sorry changes weaken|monthly| +1262|AAAAAAAAOOEAAAAA|2450965|2450994|DEPARTMENT|12|74|Welsh, rare persons share deeply for the responses. Able colleag|monthly| +1263|AAAAAAAAPOEAAAAA|2450965|2450994|DEPARTMENT|12|75|Following, other patterns will not supervise remaining p|monthly| +1264|AAAAAAAAAPEAAAAA|2450965|2450994|DEPARTMENT|12|76|Open payments pass trains. Numbers could dispose. Correct things can search so maj|monthly| +1265|AAAAAAAABPEAAAAA|2450965|2450994|DEPARTMENT|12|77|Distant areas stop yesterday on the teachers. However private rights cannot read always|monthly| +1266|AAAAAAAACPEAAAAA|2450965|2450994|DEPARTMENT|12|78|Studies ensure in a followers. Small, overseas cameras date best old courts|monthly| +1267|AAAAAAAADPEAAAAA|2450965|2450994|DEPARTMENT|12|79|Documents should not tell with the areas. About ready chil|monthly| +1268|AAAAAAAAEPEAAAAA|2450965|2450994|DEPARTMENT|12|80|Aware, inappropriate contracts remember in a partners. Customers ought to seem efforts;|monthly| +1269|AAAAAAAAFPEAAAAA|2450965|2450994|DEPARTMENT|12|81|Solid days ought to protect even wide days. White flats borro|monthly| +1270|AAAAAAAAGPEAAAAA|2450965|2450994|DEPARTMENT|12|82|Cattle can leave as in a examples. Others may control years. New days lau|monthly| +1271|AAAAAAAAHPEAAAAA|2450965|2450994|DEPARTMENT|12|83|Roots require very in a systems. Changes find meanwhile strange|monthly| +1272|AAAAAAAAIPEAAAAA|2450965|2450994|DEPARTMENT|12|84|Particular, foreign items could obtain economic, p|monthly| +1273|AAAAAAAAJPEAAAAA|2450965|2450994|DEPARTMENT|12|85|Arms know often sure forms. Pleased, conservative waters must care public occasions|monthly| +1274|AAAAAAAAKPEAAAAA|2450965|2450994|DEPARTMENT|12|86|Major women cannot infer parts. Universities take today. Policemen used to replace|monthly| +1275|AAAAAAAALPEAAAAA|2450965|2450994|DEPARTMENT|12|87|Contemporary, available years may not reveal mainly in a authoriti|monthly| +1276|AAAAAAAAMPEAAAAA|2450965|2450994|DEPARTMENT|12|88|Regular, giant goals can act in a eyes. Whole plans fetch things. Priv|monthly| +1277|AAAAAAAANPEAAAAA|2450965|2450994|DEPARTMENT|12|89|Only nuclear rights pick soon around a components. Bright representations shall not bot|monthly| +1278|AAAAAAAAOPEAAAAA|2450965|2450994|DEPARTMENT|12|90|Cards will open. Eyes shall lose. Subjects may not lose|monthly| +1279|AAAAAAAAPPEAAAAA|2450965|2450994|DEPARTMENT|12|91|Private women ought to work real, far taxes. Firms must g|monthly| +1280|AAAAAAAAAAFAAAAA|2450965|2450994|DEPARTMENT|12|92|Ways used to wish perhaps negative moves. Wet, respective shops shall provide much ancient interest|monthly| +1281|AAAAAAAABAFAAAAA|2450965|2450994|DEPARTMENT|12|93|Only tragic implications ought to change occasionally. Young, european police will div|monthly| +1282|AAAAAAAACAFAAAAA|2450965|2450994|DEPARTMENT|12|94|Couples handle. Partly old children record there of course australian discussions|monthly| +1283|AAAAAAAADAFAAAAA|2450965|2450994|DEPARTMENT|12|95|Facts could mark so numbers. False, red times shall answer either perhaps fat years. Eventually |monthly| +1284|AAAAAAAAEAFAAAAA|2450965|2450994|DEPARTMENT|12|96|Already popular companies must reform. Now high arts may indicate formally poor prac|monthly| +1285|AAAAAAAAFAFAAAAA|2450965|2450994|DEPARTMENT|12|97|Concerned firms come now scientific areas. Upper boys keep mainly princ|monthly| +1286|AAAAAAAAGAFAAAAA|2450965|2450994|DEPARTMENT|12|98|There traditional sides indicate as native conditions. Qualiti|monthly| +1287|AAAAAAAAHAFAAAAA|2450965|2450994|DEPARTMENT|12|99|Coming crews must describe often final conditions; good, social strategies shall not risk e|monthly| +1288|AAAAAAAAIAFAAAAA|2450965|2450994|DEPARTMENT|12|100|Conclusions will contribute briefly economic women. Early effects can disagree rather. Far|monthly| +1289|AAAAAAAAJAFAAAAA|2450965|2450994|DEPARTMENT|12|101|Members shall eat almost scottish, regional men. Men cannot go as gre|monthly| +1290|AAAAAAAAKAFAAAAA|2450965|2450994|DEPARTMENT|12|102|Outer, wide differences go terribly quite social moments. Recently other conditions cannot |monthly| +1291|AAAAAAAALAFAAAAA|2450965||DEPARTMENT|12|||| +1292|AAAAAAAAMAFAAAAA|2450965|2450994|DEPARTMENT|12|104|Professional operations might make in order. Everywhere physical boundaries will open|monthly| +1293|AAAAAAAANAFAAAAA|2450965|2450994|DEPARTMENT|12|105|Isolated churches take then to a interests. Just financial jobs f|monthly| +1294|AAAAAAAAOAFAAAAA|2450965|2450994|DEPARTMENT|12|106|Ready, quiet changes could write never theories. Clean, unple|monthly| +1295|AAAAAAAAPAFAAAAA|2450965|2450994|DEPARTMENT|12|107|Levels may allow markets; distinguished, technical firms shall not come final doubts. |monthly| +1296|AAAAAAAAABFAAAAA|2450965|2450994|DEPARTMENT|12|108|Below other forces become forward. Ideas ask later at a |monthly| +1297|AAAAAAAABBFAAAAA|2450995|2451024|DEPARTMENT|13|1|Most brief records analyse all still french ends. Southe|monthly| +1298|AAAAAAAACBFAAAAA|2450995|2451024|DEPARTMENT|13|2|European, permanent targets should know real executives. Li|monthly| +1299|AAAAAAAADBFAAAAA|2450995|2451024|DEPARTMENT|13|3|Powers will not tune. Major, aware strangers could work rather absolutely asian accidents. Then|monthly| +1300|AAAAAAAAEBFAAAAA|2450995|2451024|DEPARTMENT|13|4|Sheets continue away pp.. Modules could talk available, human han|monthly| +1301|AAAAAAAAFBFAAAAA|2450995|2451024|DEPARTMENT|13|5|Narrow properties may not survive there services. Other areas s|monthly| +1302|AAAAAAAAGBFAAAAA|2450995|2451024|DEPARTMENT|13|6|Really black things can indicate to the theories. Unions must pro|monthly| +1303|AAAAAAAAHBFAAAAA|2450995|2451024|DEPARTMENT|13|7|Large, little bars cannot contribute shortly. Rules oppose perhaps therefore chi|monthly| +1304|AAAAAAAAIBFAAAAA|2450995|2451024|DEPARTMENT|13|8|Small, good advertisements should not climb. Foreign ladies ought to prevent anywhere b|monthly| +1305|AAAAAAAAJBFAAAAA|2450995|2451024|DEPARTMENT|13|9|Future, short deputies cope remarkable, african wa|monthly| +1306|AAAAAAAAKBFAAAAA|2450995|2451024|DEPARTMENT|13|10|Very local rats work now thoroughly eligible methods. Things will |monthly| +1307|AAAAAAAALBFAAAAA|2450995|2451024|DEPARTMENT|13|11|Right correct ambitions may go numbers. Western, total results cause never risks. Permanent|monthly| +1308|AAAAAAAAMBFAAAAA|2450995|2451024|DEPARTMENT|13|12|Possible women should dominate hard, eager books. Small, english talk|monthly| +1309|AAAAAAAANBFAAAAA|2450995|2451024|DEPARTMENT|13|13|General, british islands apply new patients. Now appropriate others satisfy various|monthly| +1310|AAAAAAAAOBFAAAAA|2450995|2451024|DEPARTMENT|13|14|Beliefs think costs. Differences assume then winners; british genes will keep. Other, nationa|monthly| +1311|AAAAAAAAPBFAAAAA|2450995|2451024|DEPARTMENT|13|15|Years may manage legally special, old subjects. Inc, nuclear pressures|monthly| +1312|AAAAAAAAACFAAAAA|2450995|2451024|DEPARTMENT|13|16|Narrow parties ought to create. More tiny speeches go quickly rights. Perhaps fresh |monthly| +1313|AAAAAAAABCFAAAAA|2450995|2451024|DEPARTMENT|13|17|Teams earn again through the topics. Therefore alone concerns obtain in a eyes. Really poor manu|monthly| +1314|AAAAAAAACCFAAAAA|2450995|2451024|DEPARTMENT|13|18|Angry teachers shall meet remarks. Requirements make e|monthly| +1315|AAAAAAAADCFAAAAA|2450995|2451024|DEPARTMENT|13|19|So international funds state then true actors. Too other communities should become therefore|monthly| +1316|AAAAAAAAECFAAAAA|2450995|2451024|DEPARTMENT|13|20|Regional, abstract families could improve further decades. Individual statements |monthly| +1317|AAAAAAAAFCFAAAAA|2450995|2451024|DEPARTMENT|13|21|Remaining, new teachers order. Glorious lips could move litt|monthly| +1318|AAAAAAAAGCFAAAAA|2450995|2451024|DEPARTMENT|13|22|New days find there public goods. Drugs say so full men. Rose|monthly| +1319|AAAAAAAAHCFAAAAA|2450995|2451024|DEPARTMENT|13|23|Carefully cold years would dispel now. Honest, great titles |monthly| +1320|AAAAAAAAICFAAAAA|2450995|2451024|DEPARTMENT|13|24|Causes should fall games. Young feet might give together. Rarely unknown men may make italian, ot|monthly| +1321|AAAAAAAAJCFAAAAA|2450995|2451024|DEPARTMENT|13|25|European, direct members could sense well issues; expensive plans used to|monthly| +1322|AAAAAAAAKCFAAAAA|2450995|2451024|DEPARTMENT|13|26|Devices discover permanently by a facts. Only european arms leave only out of the claims.|monthly| +1323|AAAAAAAALCFAAAAA|2450995|2451024|DEPARTMENT|13|27|Policies might take only soviet posts. Cases shall speak th|monthly| +1324|AAAAAAAAMCFAAAAA|2450995|2451024|DEPARTMENT|13|28|Hours seem however for a years; rich areas might not stay sure strong authori|monthly| +1325|AAAAAAAANCFAAAAA|2450995|2451024|DEPARTMENT|13|29|Now asleep owners shall wake. Presidential, odd sorts sh|monthly| +1326|AAAAAAAAOCFAAAAA|2450995|2451024|DEPARTMENT|13|30|Processes must mind most unfair, illegal applications. Loans behave materials. Colleges |monthly| +1327|AAAAAAAAPCFAAAAA|2450995|2451024|DEPARTMENT|13|31|Appropriate, vital banks should need then extreme facilities. Ani|monthly| +1328|AAAAAAAAADFAAAAA|2450995|2451024|DEPARTMENT|13|32|Traditionally present sites understand often levels. Both legal territorie|monthly| +1329|AAAAAAAABDFAAAAA|2450995|2451024|DEPARTMENT|13|33|Private, long ends form much new powers. Now naked approaches provide |monthly| +1330|AAAAAAAACDFAAAAA|2450995|2451024|DEPARTMENT|13|34|Long, high girls must not account as official, clinical meals. Able, difficult h|monthly| +1331|AAAAAAAADDFAAAAA|2450995|2451024|DEPARTMENT|13|35|Easily cheap lines tell always women. Men could hear to a males. Anonymous, prospective o|monthly| +1332|AAAAAAAAEDFAAAAA|2450995|2451024|DEPARTMENT|13|36|Stages take so raw followers. Fingers order as specially nuclear sentences. Carefully sa|monthly| +1333|AAAAAAAAFDFAAAAA|2450995|2451024|DEPARTMENT|13|37|At all federal topics control enough just nuclear references. Arra|monthly| +1334|AAAAAAAAGDFAAAAA|2450995|2451024|DEPARTMENT|13|38|Mainly systematic members speak now after a schools. Civil states |monthly| +1335|AAAAAAAAHDFAAAAA|2450995|2451024|DEPARTMENT|13|39|Courts can answer easy eyes; too civil levels sound able women. Agai|monthly| +1336|AAAAAAAAIDFAAAAA|2450995|2451024|DEPARTMENT|13|40|Special, semantic bishops think as in a statements; u|monthly| +1337|AAAAAAAAJDFAAAAA|2450995|2451024|DEPARTMENT|13|41|Ordinary, corporate buildings tell. Forms see tables. Dry thousands m|monthly| +1338|AAAAAAAAKDFAAAAA|2450995|2451024|DEPARTMENT|13|42|Extra responses continue alternative products. Either extended comparisons attack as|monthly| +1339|AAAAAAAALDFAAAAA|2450995|2451024|DEPARTMENT|13|43|Shy, modern tonnes must investigate tiny, early years. More than fundam|monthly| +1340|AAAAAAAAMDFAAAAA|2450995|2451024|DEPARTMENT|13|44|Pale arts lead functional authorities; well educational findings start too companies. Democratic,|monthly| +1341|AAAAAAAANDFAAAAA|2450995|2451024|DEPARTMENT|13|45|Environmental, entire arrangements welcome today joint effects. Drugs may match various yards. |monthly| +1342|AAAAAAAAODFAAAAA|2450995|2451024|DEPARTMENT|13|46|Patients pay. Red, white rooms would not prevent internal governments. For ex|monthly| +1343|AAAAAAAAPDFAAAAA|2450995|2451024|DEPARTMENT|13|47|Records encourage economically. Simple, human industries speed subsequently always you|monthly| +1344|AAAAAAAAAEFAAAAA|2450995|2451024|DEPARTMENT|13|48|Important expectations would distinguish yet moreover sharp |monthly| +1345|AAAAAAAABEFAAAAA|2450995|2451024|DEPARTMENT|13|49|Far, rigid services enter really unusual musicians. Children bring therefore |monthly| +1346|AAAAAAAACEFAAAAA|2450995|2451024|DEPARTMENT|13|50|Sure controls gain indeed maximum theories. Users used to take originally mater|monthly| +1347|AAAAAAAADEFAAAAA|2450995|2451024|DEPARTMENT|13|51|Neither rapid years know efficient, personal teachers. Words |monthly| +1348|AAAAAAAAEEFAAAAA|2450995|2451024|DEPARTMENT|13|52|Clear, criminal employees end deliberately warm police. St|monthly| +1349|AAAAAAAAFEFAAAAA|2450995|2451024|DEPARTMENT|13|53|Relations could lead growing conditions. Owners might defeat|monthly| +1350|AAAAAAAAGEFAAAAA|2450995|2451024|DEPARTMENT|13|54|Above correct shelves used to end neither by an humans. Also old spirits should tu|monthly| +1351|AAAAAAAAHEFAAAAA|2450995|2451024|DEPARTMENT|13|55|Highly other parents could make thus main, heavy procedures. So-call|monthly| +1352|AAAAAAAAIEFAAAAA|2450995|2451024|DEPARTMENT|13|56|Convenient activities see able relations. All firm musicians forget so on a days. Public,|monthly| +1353|AAAAAAAAJEFAAAAA|2450995|2451024|DEPARTMENT|13|57|Most bad times try geographical children. Famous, criminal example|monthly| +1354|AAAAAAAAKEFAAAAA|2450995|2451024|DEPARTMENT|13|58|Again urban films describe only. Usually pleasant artis|monthly| +1355|AAAAAAAALEFAAAAA|2450995|2451024|DEPARTMENT|13|59|Legal processes will support new, careful activities. Cars punch so systems; economie|monthly| +1356|AAAAAAAAMEFAAAAA|2450995|2451024|DEPARTMENT|13|60|Different, competitive components see employees. Loss|monthly| +1357|AAAAAAAANEFAAAAA|2450995|2451024|DEPARTMENT|13|61|Problems must suffice internationally in a parts. Women identify even. Troops will inv|monthly| +1358|AAAAAAAAOEFAAAAA|2450995|2451024|DEPARTMENT|13|62|External, personal benefits will make; demonstrations mean little wo|monthly| +1359|AAAAAAAAPEFAAAAA|2450995|2451024|DEPARTMENT|13|63|Good references pay difficult services. Only examples stand on|monthly| +1360|AAAAAAAAAFFAAAAA|2450995|2451024|DEPARTMENT|13|64|Fingers run too glorious contributions. Just combined questions may not|monthly| +1361|AAAAAAAABFFAAAAA|2450995|2451024|DEPARTMENT|13|65|Regularly respective relationships ought to swim today|monthly| +1362|AAAAAAAACFFAAAAA|2450995|2451024|DEPARTMENT|13|66|Now wide sorts shall make very in a techniques. Features go. British institutio|monthly| +1363|AAAAAAAADFFAAAAA|2450995|2451024|DEPARTMENT|13|67|Scales challenge issues; capital games cannot break now by the processes. Rather afr|monthly| +1364|AAAAAAAAEFFAAAAA|2450995|2451024|DEPARTMENT|13|68|Fair songs take however elderly, european roads. All fine chapters m|monthly| +1365|AAAAAAAAFFFAAAAA|2450995|2451024|DEPARTMENT|13|69|Long-term customers offer further prime years; senten|monthly| +1366|AAAAAAAAGFFAAAAA|2450995|2451024|DEPARTMENT|13|70|Very large events should get good pupils; exciting, massive leaders used to know nice waters.|monthly| +1367|AAAAAAAAHFFAAAAA|2450995|2451024|DEPARTMENT|13|71|Anyway national crops must not miss in a casualties. Bottom, basic areas cause there in |monthly| +1368|AAAAAAAAIFFAAAAA|2450995|2451024|DEPARTMENT|13|72|Tears win military children. Clear orders affect pu|monthly| +1369|AAAAAAAAJFFAAAAA|2450995|2451024|DEPARTMENT|13|73|Classical relationships will bear tonight environmental, l|monthly| +1370|AAAAAAAAKFFAAAAA|||||74|More deep points might not go only young little governors. To|| +1371|AAAAAAAALFFAAAAA|2450995|2451024|DEPARTMENT|13|75|Ruling drinks entitle. Social, immense structures remind major patterns. Yet video-taped tex|monthly| +1372|AAAAAAAAMFFAAAAA|2450995|2451024|DEPARTMENT|13|76|Constraints make home able, dry applications; pp. get very most fr|monthly| +1373|AAAAAAAANFFAAAAA|2450995|2451024|DEPARTMENT|13|77|Major organisations choose central technologies. As international cases discredit in|monthly| +1374|AAAAAAAAOFFAAAAA|2450995|2451024|DEPARTMENT|13|78|Full, good others shall not accept especially faint, american|monthly| +1375|AAAAAAAAPFFAAAAA|2450995|2451024|DEPARTMENT|13|79|Generous guards indicate manually likely, natural cells; here autom|monthly| +1376|AAAAAAAAAGFAAAAA|2450995|2451024|DEPARTMENT|13|80|Reports may continue only problems. Blacks accept from the objects. New, constant all|monthly| +1377|AAAAAAAABGFAAAAA|2450995|2451024|DEPARTMENT|13|81|Full, international results could issue finally later |monthly| +1378|AAAAAAAACGFAAAAA|2450995|2451024|DEPARTMENT|13|82|Good institutions could sign elsewhere different crops. Just importa|monthly| +1379|AAAAAAAADGFAAAAA|2450995|2451024|DEPARTMENT|13|83|Radical priorities will complete less true errors. Trees may not ventur|monthly| +1380|AAAAAAAAEGFAAAAA|2450995|2451024|DEPARTMENT|13|84|Simply upper components should not encourage only in a bacteria. Funds work right simple ma|monthly| +1381|AAAAAAAAFGFAAAAA|2450995|2451024|DEPARTMENT|13|85|Years would reconcile. Social, british groups enter recently in|monthly| +1382|AAAAAAAAGGFAAAAA|2450995|2451024|DEPARTMENT|13|86|Generously organisational reasons tell from a users.|monthly| +1383|AAAAAAAAHGFAAAAA|2450995|2451024|DEPARTMENT|13|87|British opportunities cannot hear over private, usual |monthly| +1384|AAAAAAAAIGFAAAAA|2450995|2451024|DEPARTMENT|13|88|Softly natural years quit men. Substances make animals; just general e|monthly| +1385|AAAAAAAAJGFAAAAA|2450995|2451024|DEPARTMENT|13|89|Dry, white years tell remotely easy troops. Also political subjects must|monthly| +1386|AAAAAAAAKGFAAAAA|2450995|2451024|DEPARTMENT|13|90|Errors come again american states. Practitioners change so neverthele|monthly| +1387|AAAAAAAALGFAAAAA|2450995|2451024|DEPARTMENT|13|91|Visitors must burst. Defences plan important types. Atlantic years might leave equally for |monthly| +1388|AAAAAAAAMGFAAAAA|2450995|2451024|DEPARTMENT|13|92|Real, economic illustrations understand by the implicatio|monthly| +1389|AAAAAAAANGFAAAAA|2450995|2451024|DEPARTMENT|13|93|Fortunately human ways leave different candidates. Real, j|monthly| +1390|AAAAAAAAOGFAAAAA|2450995|2451024|DEPARTMENT|13|94|Houses can wish also red, fast months. In vain concerned arrangements would mention rather small, n|monthly| +1391|AAAAAAAAPGFAAAAA|2450995|2451024|DEPARTMENT|13|95|Late busy teeth enforce yet social years. Public mammal|monthly| +1392|AAAAAAAAAHFAAAAA|2450995|2451024|DEPARTMENT|13|96|Christian mothers build masses; almost possible men could dispense sooner fresh different words. |monthly| +1393|AAAAAAAABHFAAAAA|2450995|2451024|DEPARTMENT|13|97|Units imagine. Partly foreign hospitals find more by|monthly| +1394|AAAAAAAACHFAAAAA|2450995|2451024|DEPARTMENT|13|98|Moving, other priorities see in the years. Useful seconds ask s|monthly| +1395|AAAAAAAADHFAAAAA|2450995|2451024|DEPARTMENT|13|99|Payable things must appear now. Other comments make. In|monthly| +1396|AAAAAAAAEHFAAAAA|2450995|2451024|DEPARTMENT|13|100|Similar, double activities realize years. Yet clear attitudes must not smash in the teachers. Stand|monthly| +1397|AAAAAAAAFHFAAAAA|2450995|2451024|DEPARTMENT|13|101|Faithfully late difficulties learn very walls. Also secondary sources upgrade areas. Sov|monthly| +1398|AAAAAAAAGHFAAAAA|2450995|2451024|DEPARTMENT|13|102|Eyes leave matters. Kilometres may not make much old homes. Low international eyes demo|monthly| +1399|AAAAAAAAHHFAAAAA|2450995|2451024|DEPARTMENT|13|103|Old, other feelings spot grand, certain bases. Ora|monthly| +1400|AAAAAAAAIHFAAAAA|2450995|2451024|DEPARTMENT|13|104|Massive theories shall get whole, following pieces. |monthly| +1401|AAAAAAAAJHFAAAAA|2450995|2451024|DEPARTMENT|13|105|Individuals ought to identify requirements. Proud, comprehensive ye|monthly| +1402|AAAAAAAAKHFAAAAA|2450995|2451024|DEPARTMENT|13|106|National, real others sound probably enthusiastic w|monthly| +1403|AAAAAAAALHFAAAAA|2450995|2451024|DEPARTMENT|13|107|More fixed periods shall ignore just there strong intentions. Entire, twin conditions set just wi|monthly| +1404|AAAAAAAAMHFAAAAA|2450995|2451024|DEPARTMENT|13|108|Usually competitive resources might not notify. Over american qualities must speak still o|monthly| +1405|AAAAAAAANHFAAAAA|2451025|2451054|DEPARTMENT|14|1|However immediate photographs would shape later; there indepen|monthly| +1406|AAAAAAAAOHFAAAAA|2451025|2451054|DEPARTMENT|14|2|Important, necessary tensions call specific effects. Extensive r|monthly| +1407|AAAAAAAAPHFAAAAA|2451025|2451054|DEPARTMENT|14|3|For good diplomatic movements used to enjoy already different, good shoes. However residential c|monthly| +1408|AAAAAAAAAIFAAAAA|2451025|2451054|DEPARTMENT|14|4|Foreign, professional sites should not start also. Difficult, ne|monthly| +1409|AAAAAAAABIFAAAAA|2451025|2451054|DEPARTMENT|14|5|Wrong workers survive as for a years. Here final feelings rea|monthly| +1410|AAAAAAAACIFAAAAA|2451025|2451054|DEPARTMENT|14|6|Dependent mines come shoes. Efforts use from a books. Centres must start. Military obligatio|monthly| +1411|AAAAAAAADIFAAAAA|2451025|2451054|DEPARTMENT|14|7|Factors want during a functions; interested charts sit. Blue operations qualify |monthly| +1412|AAAAAAAAEIFAAAAA|2451025|2451054|DEPARTMENT|14|8|Well active links see. Public facilities find clothes. Dutch, minor colours could ident|monthly| +1413|AAAAAAAAFIFAAAAA|2451025|2451054|DEPARTMENT|14|9|Relationships leave never actually entire unions. Pregnant, busy shares would pay already at|monthly| +1414|AAAAAAAAGIFAAAAA|2451025|2451054|DEPARTMENT|14|10|Critically religious cases could eat to a eyes. Very payable years go now requirements. Beings make|monthly| +1415|AAAAAAAAHIFAAAAA|2451025|2451054|DEPARTMENT|14|11|Open heads take neatly relative times. Red, social girls contac|monthly| +1416|AAAAAAAAIIFAAAAA|2451025|2451054|DEPARTMENT|14|12|Real glasses help. Plain, other arrangements should control as alone matte|monthly| +1417|AAAAAAAAJIFAAAAA|2451025|2451054|DEPARTMENT|14|13|Pupils might take. Cautiously usual times explore in a miles. Small, african employers recognise fa|monthly| +1418|AAAAAAAAKIFAAAAA|2451025|2451054|DEPARTMENT|14|14|Squarely underlying methods might not matter possibly children. Fundamental hours should no|monthly| +1419|AAAAAAAALIFAAAAA|2451025|2451054|DEPARTMENT|14|15|Regional areas get about settings. Direct, sweet par|monthly| +1420|AAAAAAAAMIFAAAAA|2451025|2451054|DEPARTMENT|14|16|Rapid mice should change most current wines. Years see wrong experiments. Advanced |monthly| +1421|AAAAAAAANIFAAAAA|2451025|2451054|DEPARTMENT|14|17|Subject, public workers can distort now. Certain, ps|monthly| +1422|AAAAAAAAOIFAAAAA|2451025|2451054|DEPARTMENT|14|18|Even wild deals bring interests. Additional lads find obvi|monthly| +1423|AAAAAAAAPIFAAAAA|2451025|2451054|DEPARTMENT|14|19|Following executives dismiss hard reasons. Cases facilitate at all at a vis|monthly| +1424|AAAAAAAAAJFAAAAA|2451025|2451054|DEPARTMENT|14|20|Difficult, obvious patients remain about the measurements. Interesting sanction|monthly| +1425|AAAAAAAABJFAAAAA||2451054|DEPARTMENT||21||| +1426|AAAAAAAACJFAAAAA|2451025|2451054|DEPARTMENT|14|22|Practical ages find less feelings. Territorial members complete as well ma|monthly| +1427|AAAAAAAADJFAAAAA|2451025|2451054|DEPARTMENT|14|23|Free, other police ensure as the words. Still automatic assessments should not solve eng|monthly| +1428|AAAAAAAAEJFAAAAA|2451025|2451054|DEPARTMENT|14|24|Words could wander now records. Only, social debts can know both. Hi|monthly| +1429|AAAAAAAAFJFAAAAA|2451025|2451054|DEPARTMENT|14|25|Smartly grey schools could like so at a terms. Faces become today so national men. Patients remem|monthly| +1430|AAAAAAAAGJFAAAAA|2451025|2451054|DEPARTMENT|14|26|So semantic months must not take servants. Sound children serve th|monthly| +1431|AAAAAAAAHJFAAAAA|2451025|2451054|DEPARTMENT|14|27|Certainly standard details will tend similar, likely designs; other, small months can expect. Onl|monthly| +1432|AAAAAAAAIJFAAAAA|2451025|2451054|DEPARTMENT|14|28|Afterwards unusual officials provide able studies. No doubt small candidates consolidate |monthly| +1433|AAAAAAAAJJFAAAAA|2451025|2451054|DEPARTMENT|14|29|Important students ought to get. Wrong, environmental men reach sometimes. Pr|monthly| +1434|AAAAAAAAKJFAAAAA|2451025|2451054|DEPARTMENT|14|30|Poor, right relationships question programmes. Further good ar|monthly| +1435|AAAAAAAALJFAAAAA|2451025|2451054|DEPARTMENT|14|31|Lands would not listen yet; away local men will not need r|monthly| +1436|AAAAAAAAMJFAAAAA|2451025|2451054|DEPARTMENT|14|32|Public, tiny objects prescribe of course high differences; origins hit long plants. Defensive f|monthly| +1437|AAAAAAAANJFAAAAA|2451025|2451054|DEPARTMENT|14|33|Examples ought to understand more than social affairs; |monthly| +1438|AAAAAAAAOJFAAAAA|2451025|2451054|DEPARTMENT|14|34|Negative years can find less. Too possible years contact to a negotiations. St|monthly| +1439|AAAAAAAAPJFAAAAA|2451025|2451054|DEPARTMENT|14|35|More correct points would admit then remote things; powers pay fortunate, responsible beans. |monthly| +1440|AAAAAAAAAKFAAAAA|2451025|2451054|DEPARTMENT|14|36|Left, recent standards keep already; certain, mental relationships tell in order to the|monthly| +1441|AAAAAAAABKFAAAAA|2451025|2451054|DEPARTMENT|14|37|Certain decisions learn old, previous police; years can prove valuable, honest|monthly| +1442|AAAAAAAACKFAAAAA|2451025|2451054|||38|More rural rules would not produce so issues. Cultural,|monthly| +1443|AAAAAAAADKFAAAAA|2451025|2451054|DEPARTMENT|14|39|Alternative, british jobs ought to know. Wrong levels raise beyond a consequences. Still burnin|monthly| +1444|AAAAAAAAEKFAAAAA|2451025|2451054|DEPARTMENT|14|40|Hot things make. Books must not mean evil, mutual bases. New, prop|monthly| +1445|AAAAAAAAFKFAAAAA|2451025|2451054|DEPARTMENT|14|41|Children could not represent ministers. Empty, apparent conclusions should|monthly| +1446|AAAAAAAAGKFAAAAA|2451025|2451054|DEPARTMENT|14|42|Later black workers used to survive thoughts. Thick, bright ra|monthly| +1447|AAAAAAAAHKFAAAAA|2451025|2451054|DEPARTMENT|14|43|Effective, warm interpretations could not practise imports. Common,|monthly| +1448|AAAAAAAAIKFAAAAA|2451025|2451054|DEPARTMENT|14|44|Fond, other groups should produce immediately both french companies. Necessary |monthly| +1449|AAAAAAAAJKFAAAAA|2451025|2451054|DEPARTMENT|14|45|Typical numbers shall serve far from thus commercia|monthly| +1450|AAAAAAAAKKFAAAAA|2451025|2451054|DEPARTMENT|14|46|Customers choose as well sales; very legal generations minimise perfect adults. Boundaries sho|monthly| +1451|AAAAAAAALKFAAAAA|2451025|2451054|DEPARTMENT|14|47|Top representatives must provide traditional years; over logical males will not fail readily fo|monthly| +1452|AAAAAAAAMKFAAAAA|2451025|2451054|DEPARTMENT|14|48|Substances know preferences. Short police suppose probably rights. Minutes should rememb|monthly| +1453|AAAAAAAANKFAAAAA|2451025|2451054|DEPARTMENT|14|49|Items note twice public, sure questions. Precise, conventional years will not receive e|monthly| +1454|AAAAAAAAOKFAAAAA|2451025|2451054|DEPARTMENT|14|50|Women would keep extremely black things. Only significant countries cannot |monthly| +1455|AAAAAAAAPKFAAAAA|2451025|2451054|DEPARTMENT|14|51|Sorry falls hold carefully troops. Currently little directors will d|monthly| +1456|AAAAAAAAALFAAAAA|2451025|2451054|DEPARTMENT|14|52|Poor ideas must quit never silent conflicts. Phases save well senses. Inevitable ambiti|monthly| +1457|AAAAAAAABLFAAAAA|2451025|2451054|DEPARTMENT|14|53|Meanwhile left needs enjoy; special, sufficient operations say for example as the com|monthly| +1458|AAAAAAAACLFAAAAA|2451025|2451054|DEPARTMENT|14|54|Neighbours get already educational jobs. Commercial|monthly| +1459|AAAAAAAADLFAAAAA|2451025|2451054|DEPARTMENT|14|55|Additional services shall reassert in a difficulties; degrees used to release for instance just w|monthly| +1460|AAAAAAAAELFAAAAA|2451025|2451054|DEPARTMENT|14|56|Skills provide healthy times; industrial, religious problems |monthly| +1461|AAAAAAAAFLFAAAAA|2451025|2451054|DEPARTMENT|14|57|Bizarre, professional objectives concentrate both at least simple traditions. Significant dif|monthly| +1462|AAAAAAAAGLFAAAAA|2451025|2451054|DEPARTMENT|14|58|Other, new tickets can say at a areas. Married houses offer again o|monthly| +1463|AAAAAAAAHLFAAAAA|2451025|2451054|DEPARTMENT|14|59|Rats complain then alone marginal bodies. Candidates mu|monthly| +1464|AAAAAAAAILFAAAAA|2451025|2451054|DEPARTMENT|14|60|Reasonable, judicial democrats produce unfortunately little|monthly| +1465|AAAAAAAAJLFAAAAA|2451025|2451054|DEPARTMENT|14|61|Possible, experimental patients might not prove that then marked hundreds. Em|monthly| +1466|AAAAAAAAKLFAAAAA|2451025|2451054|DEPARTMENT|14|62|English advantages require just necessary fragments. Legal sports might not know. Ea|monthly| +1467|AAAAAAAALLFAAAAA|2451025|2451054||||Groups may wish in addition outside, different cheeks. Shares disclose so as familiar, personal s|monthly| +1468|AAAAAAAAMLFAAAAA|2451025|2451054|DEPARTMENT|14|64|Ironically great ministers will not help lately possible conditions. Coherent, succes|monthly| +1469|AAAAAAAANLFAAAAA|2451025|2451054|DEPARTMENT|14|65|Separate newspapers like human, automatic goals. Areas used to help ab|monthly| +1470|AAAAAAAAOLFAAAAA|2451025|2451054|DEPARTMENT|14|66|Wheels used to see very highly new calls. Big relations can guarantee unlikely, hidden auth|monthly| +1471|AAAAAAAAPLFAAAAA|2451025|2451054|DEPARTMENT|14|67|Doubtful officers say measures; workers give too foreig|monthly| +1472|AAAAAAAAAMFAAAAA|2451025|2451054|DEPARTMENT|14|68|Days would get parents. New models will not convey even conse|monthly| +1473|AAAAAAAABMFAAAAA|2451025|2451054|DEPARTMENT|14|69|Now russian miles must remain perhaps. Asleep, previous notes classify political, direct|monthly| +1474|AAAAAAAACMFAAAAA|2451025|2451054|DEPARTMENT|14|70|Studies could answer now in a subjects. Overwhelmingly recent estat|monthly| +1475|AAAAAAAADMFAAAAA|2451025|2451054|DEPARTMENT|14|71|Voluntarily possible backs step there by a leads. For example only cr|monthly| +1476|AAAAAAAAEMFAAAAA|2451025|2451054|DEPARTMENT|14|72|Institutions perform there ever local books; settings compose by a changes. As conservative|monthly| +1477|AAAAAAAAFMFAAAAA|2451025|2451054|DEPARTMENT|14|73|Women apply in vain in the minutes. Officials can |monthly| +1478|AAAAAAAAGMFAAAAA|2451025|2451054|DEPARTMENT|14|74|White jobs used to seem then bad views. Just major times would ask children. All ba|monthly| +1479|AAAAAAAAHMFAAAAA|2451025|2451054|DEPARTMENT|14|75|Stupid, reasonable goods engage then. Adequate, misleading branches might not |monthly| +1480|AAAAAAAAIMFAAAAA|2451025|2451054|DEPARTMENT|14|76|Official figures stop here like a attitudes; residential m|monthly| +1481|AAAAAAAAJMFAAAAA|2451025|2451054|DEPARTMENT|14|77|Just radical decisions should try yet chronic educational services. Religious, military |monthly| +1482|AAAAAAAAKMFAAAAA|2451025|2451054|DEPARTMENT|14|78|Elderly customs go also even fine children. National contracts|monthly| +1483|AAAAAAAALMFAAAAA|2451025|2451054|DEPARTMENT|14|79|Hands shall make already about distinct claims; countries call just odd, vulnerable pl|monthly| +1484|AAAAAAAAMMFAAAAA|2451025|2451054|DEPARTMENT|14|80|Prominent, welsh changes mention still clinical, g|monthly| +1485|AAAAAAAANMFAAAAA|2451025|2451054|DEPARTMENT|14|81|Political users keep well proceedings; major policies can control little from |monthly| +1486|AAAAAAAAOMFAAAAA|2451025|2451054|DEPARTMENT|14|82|Organs think words. Brilliant provisions might place perhaps dates. Rights re|monthly| +1487|AAAAAAAAPMFAAAAA|2451025|2451054|DEPARTMENT|14|83|Superior publications look. Again other rivers det|monthly| +1488|AAAAAAAAANFAAAAA|2451025|2451054|DEPARTMENT|14|84|Brightly black needs used to like intensive problems. Labour restriction|monthly| +1489|AAAAAAAABNFAAAAA|2451025|2451054|DEPARTMENT|14|85|Children might allow almost western, foreign backs. White, considerable categor|monthly| +1490|AAAAAAAACNFAAAAA|2451025|2451054|DEPARTMENT|14|86|Children may clean secondly medical, underlying affairs. Original, left |monthly| +1491|AAAAAAAADNFAAAAA|2451025|2451054|DEPARTMENT|14|87|Very far sciences supply closer. Most aware things describe mo|monthly| +1492|AAAAAAAAENFAAAAA|2451025|2451054|DEPARTMENT|14|88|Differences live qualities. Deep, empty leaves qualif|monthly| +1493|AAAAAAAAFNFAAAAA|2451025|2451054|DEPARTMENT|14|89|Special, eager members ought to telephone british, naval books. |monthly| +1494|AAAAAAAAGNFAAAAA|2451025|2451054|DEPARTMENT|14|90|Complaints take properly national schools. Cars shall not want freque|monthly| +1495|AAAAAAAAHNFAAAAA|2451025|2451054|DEPARTMENT|14|91|Later economic publishers suspend visitors; applications ought to al|monthly| +1496|AAAAAAAAINFAAAAA|2451025|2451054|DEPARTMENT|14|92|Years restore. Added guidelines may wear past; very other memories will not|monthly| +1497|AAAAAAAAJNFAAAAA|2451025|2451054|DEPARTMENT|14|93|Criminal, other features prove police; issues serve prices. G|monthly| +1498|AAAAAAAAKNFAAAAA|2451025|2451054|DEPARTMENT|14|94|Upstream soviet soldiers would arise never facts. Deep eyes risk independently|monthly| +1499|AAAAAAAALNFAAAAA|2451025|2451054|DEPARTMENT|14|95|Enough proud men apply well italian measurements. Question|monthly| +1500|AAAAAAAAMNFAAAAA|2451025|2451054|DEPARTMENT|14|96|Practitioners foresee further. Black, able forces used to influe|monthly| +1501|AAAAAAAANNFAAAAA|2451025|2451054|DEPARTMENT|14|97|Only pretty schemes should not act exactly french items. Tomorrow priva|monthly| +1502|AAAAAAAAONFAAAAA|2451025|2451054|DEPARTMENT|14|98|About educational stories cannot play superior, sweet keys. Mammals may b|monthly| +1503|AAAAAAAAPNFAAAAA|2451025|2451054|DEPARTMENT|14|99|Patients might not point. At least obvious years m|monthly| +1504|AAAAAAAAAOFAAAAA|2451025|2451054|DEPARTMENT|14|100|British, open skills make mentally easily remarkable counties. Most|monthly| +1505|AAAAAAAABOFAAAAA|2451025|2451054|DEPARTMENT|14|101|Educational, good objectives ask. Unnecessarily useful prisoners practise really fat costs. Oth|monthly| +1506|AAAAAAAACOFAAAAA|2451025|2451054|DEPARTMENT|14|102|Breasts might mean either before a tons. Annual friends may n|monthly| +1507|AAAAAAAADOFAAAAA|2451025|2451054|DEPARTMENT|14|103|Officially oral shadows shall adapt now good records; users let as patients. Small |monthly| +1508|AAAAAAAAEOFAAAAA|2451025|2451054|DEPARTMENT|14|104|Lives could give. Clear races want respectively important players. R|monthly| +1509|AAAAAAAAFOFAAAAA|2451025|2451054|DEPARTMENT|14|105|Prices shall promote. Projects might not wave. Lives protect still. Front, religi|monthly| +1510|AAAAAAAAGOFAAAAA|2451025|2451054|DEPARTMENT|14|106|Different, english stories shall check subjective duties; immediately only men get otherw|monthly| +1511|AAAAAAAAHOFAAAAA|2451025|2451054|DEPARTMENT|14|107|Other effects know equally ambitious, black minutes. Objects give further single, old grounds. W|monthly| +1512|AAAAAAAAIOFAAAAA|2451025|2451054|DEPARTMENT|14|108|Apparent times move now high suspicious heels; original, established a|monthly| +1513|AAAAAAAAJOFAAAAA|2451055|2451084|DEPARTMENT|15|1|Particularly available schools shall not abuse then for|monthly| +1514|AAAAAAAAKOFAAAAA|2451055|2451084|DEPARTMENT|15|2|Vital rooms can prompt double, fine forces. Fixed, industrial |monthly| +1515|AAAAAAAALOFAAAAA|2451055|2451084|DEPARTMENT|15|3|Available relations look straight immediate, special colleges. New fragments treat. Mus|monthly| +1516|AAAAAAAAMOFAAAAA|2451055|2451084|DEPARTMENT|15|4|Content students shall provide rather. Basic, public soldiers accompany grea|monthly| +1517|AAAAAAAANOFAAAAA|2451055|2451084|DEPARTMENT|15|5|Authorities should not exercise special, bright numbers; feet sup|monthly| +1518|AAAAAAAAOOFAAAAA|2451055|2451084|DEPARTMENT|15|6|Huge, new eyes can give in a numbers. Then new parts co|monthly| +1519|AAAAAAAAPOFAAAAA|2451055|2451084|DEPARTMENT|15|7|Women can exist forces; interesting, capital years ob|monthly| +1520|AAAAAAAAAPFAAAAA|2451055|2451084|DEPARTMENT|15|8|Again additional studies face no longer successful democrat|monthly| +1521|AAAAAAAABPFAAAAA|2451055|2451084|DEPARTMENT|15|9|Terms may not convince. Locations can believe perfect|monthly| +1522|AAAAAAAACPFAAAAA|2451055|2451084|DEPARTMENT|15|10|Effective, weak members used to cause highly ruling seconds. Almost political facto|monthly| +1523|AAAAAAAADPFAAAAA|2451055|2451084|DEPARTMENT|15|11|Wines use both traditions. Common, top legs could not attract faster. E|monthly| +1524|AAAAAAAAEPFAAAAA|2451055|2451084|DEPARTMENT|15|12|Efforts go great, unknown operations; extra, efficient rates accept eastern, manufacturing mode|monthly| +1525|AAAAAAAAFPFAAAAA|2451055|2451084|DEPARTMENT|15|13|Likely payments see difficult eyes. Boards retain servants. Independent, rough conditions shal|monthly| +1526|AAAAAAAAGPFAAAAA|2451055|2451084|DEPARTMENT|15|14|Universities could bring green, high results. Classes ought to take |monthly| +1527|AAAAAAAAHPFAAAAA|2451055|2451084|DEPARTMENT|15|15|So following homes pursue political police. Days put f|monthly| +1528|AAAAAAAAIPFAAAAA|2451055|2451084|DEPARTMENT|15|16|Main exceptions develop hours. Prisoners require finally capital fo|monthly| +1529|AAAAAAAAJPFAAAAA|2451055|2451084|DEPARTMENT|15|17|Regional shares cannot call dead dogs; activities cannot w|monthly| +1530|AAAAAAAAKPFAAAAA|2451055|2451084|DEPARTMENT|15|18|Small officials may include most too dry lads. Par|monthly| +1531|AAAAAAAALPFAAAAA|2451055|2451084|DEPARTMENT|15|19|Complete projects will travel technical weeks. Yet unique plans determin|monthly| +1532|AAAAAAAAMPFAAAAA|2451055|2451084|DEPARTMENT|15|20|Dual, poor events might not end at last proteins; democ|monthly| +1533|AAAAAAAANPFAAAAA|2451055|2451084|DEPARTMENT|15|21|Prominent, numerous shoulders recall so in the ranks. Doubts get further by a lad|monthly| +1534|AAAAAAAAOPFAAAAA|2451055|2451084|DEPARTMENT|15|22|Still serious patterns shall break major, dark ideas. Just fri|monthly| +1535|AAAAAAAAPPFAAAAA|2451055|2451084|DEPARTMENT|15|23|Optimistic titles must not exceed. Commercial pp. feel further. Other, creative offices |monthly| +1536|AAAAAAAAAAGAAAAA|2451055|2451084|DEPARTMENT|15|24|Tender kids repair strongly evil symptoms. Tools attempt prices. Imaginative games deve|monthly| +1537|AAAAAAAABAGAAAAA|2451055|2451084|DEPARTMENT|15|25|Women would cope now new settings. Babies map to a years. Popular, theoretical months|monthly| +1538|AAAAAAAACAGAAAAA|2451055|2451084|DEPARTMENT|15|26|Schemes look however carefully conventional residents. Addresses get as applicable regions. For|monthly| +1539|AAAAAAAADAGAAAAA|2451055|2451084|DEPARTMENT|15|27|Internal families congratulate even available years. Low|monthly| +1540|AAAAAAAAEAGAAAAA|2451055|2451084|DEPARTMENT|15|28|Actually little students must not get there exact, labour town|monthly| +1541|AAAAAAAAFAGAAAAA|2451055|2451084|DEPARTMENT|15|29|Also long heads cannot deal already new extensive men. Good, integrated designers would clear just|monthly| +1542|AAAAAAAAGAGAAAAA|2451055|2451084|DEPARTMENT|15|30|Particularly bad folk cut so in a associations. New, logical p|monthly| +1543|AAAAAAAAHAGAAAAA|2451055|2451084|DEPARTMENT|15|31|Illegal speeches hide from a classes. Major, good bits may pu|monthly| +1544|AAAAAAAAIAGAAAAA|2451055|2451084|DEPARTMENT|15|32|Unknown types run to a thousands. Possible, short quanti|monthly| +1545|AAAAAAAAJAGAAAAA|2451055|2451084|DEPARTMENT|15|33|Arts take namely lines. Whole nerves would achieve th|monthly| +1546|AAAAAAAAKAGAAAAA|2451055|2451084|DEPARTMENT|15|34|Medical, average days will not take too in the risks. Statements might rewrite only neg|monthly| +1547|AAAAAAAALAGAAAAA|2451055|2451084|DEPARTMENT|15|35|Scottish, conscious policies bother very perhaps public floors; constant, white teeth realis|monthly| +1548|AAAAAAAAMAGAAAAA|2451055|2451084|DEPARTMENT|15|36|Members would not get only, old children. Teams could not live still things. Internatio|monthly| +1549|AAAAAAAANAGAAAAA|2451055|2451084|DEPARTMENT|15|37|Hardly other minutes deal essentially english tactics. Tiny men would increase more to a i|monthly| +1550|AAAAAAAAOAGAAAAA|2451055|2451084|DEPARTMENT|15|38|Typical women should not persuade soon sensitive metres. Single, new heads can strike before aver|monthly| +1551|AAAAAAAAPAGAAAAA|2451055|2451084|DEPARTMENT|15|39|Members discourage mainly equal dates; now available revenues may point hu|monthly| +1552|AAAAAAAAABGAAAAA|2451055|2451084|DEPARTMENT|15|40|Constantly chief rules should consider over technically proper animals; winners should for|monthly| +1553|AAAAAAAABBGAAAAA|2451055|2451084|DEPARTMENT|15|41|Titles could understand away open, civil dollars. Meetings ha|monthly| +1554|AAAAAAAACBGAAAAA|2451055|2451084|DEPARTMENT|15|42|Fresh years send still rigorously traditional aspects. Rich parents |monthly| +1555|AAAAAAAADBGAAAAA|2451055|2451084|DEPARTMENT|15|43|Serious, previous earnings would combat countries; other, good programmes |monthly| +1556|AAAAAAAAEBGAAAAA|2451055|2451084|DEPARTMENT|15|44|Already historical workers discourage neatly for a offic|monthly| +1557|AAAAAAAAFBGAAAAA|2451055|2451084|DEPARTMENT|15|45|Also single ages shall understand mechanisms. Key, valid prices will set again course|monthly| +1558|AAAAAAAAGBGAAAAA|2451055|2451084|DEPARTMENT|15|46|More parliamentary points change past, social years; years feel nevertheless beautiful, ea|monthly| +1559|AAAAAAAAHBGAAAAA|2451055|2451084|DEPARTMENT|15|47|Tall topics take green, central nations. Right, wet kings stand in the pat|monthly| +1560|AAAAAAAAIBGAAAAA|2451055|2451084|DEPARTMENT|15|48|Sexually good words drive also. Now firm ears will r|monthly| +1561|AAAAAAAAJBGAAAAA|2451055|2451084|DEPARTMENT|15|49|Once soft regions would wait whole bones. Numerous, relevant tenants should take all unne|monthly| +1562|AAAAAAAAKBGAAAAA|2451055|2451084|DEPARTMENT|15|50|Lists could run also as disabled emotions. Potentially reliable hands sta|monthly| +1563|AAAAAAAALBGAAAAA|2451055|2451084|DEPARTMENT|15|51|Ever local auditors will go fresh homes; steady, detailed times must beg in|monthly| +1564|AAAAAAAAMBGAAAAA|2451055|2451084|DEPARTMENT|15|52|Frequent, familiar books seem elsewhere. Different,|monthly| +1565|AAAAAAAANBGAAAAA|2451055|2451084|DEPARTMENT|15|53|National, large services draw sometimes also cold cells. Rela|monthly| +1566|AAAAAAAAOBGAAAAA|2451055|2451084|DEPARTMENT|15|54|Directly black skills used to appear very more than american difficulties; ill, asleep days |monthly| +1567|AAAAAAAAPBGAAAAA|2451055|2451084|DEPARTMENT|15|55|Measures could react fees; stones write black jobs. Interested, wid|monthly| +1568|AAAAAAAAACGAAAAA|2451055|2451084|DEPARTMENT|15|56|Unhappy texts can take more long-term limitations. Days s|monthly| +1569|AAAAAAAABCGAAAAA|2451055|2451084|DEPARTMENT|15|57|Prices come more heavy, clear signs. Natural agents might wear only ways. Great, massive movement|monthly| +1570|AAAAAAAACCGAAAAA|2451055|2451084|DEPARTMENT|15|58|Excellent, able buyers may resist sometimes prominent hundreds. Cities look gently. Years alter;|monthly| +1571|AAAAAAAADCGAAAAA|2451055|2451084|DEPARTMENT|15|59|Now other courses comply a bit out of a owners. Enough responsible russians tell around new ca|monthly| +1572|AAAAAAAAECGAAAAA|2451055|2451084|DEPARTMENT||60||monthly| +1573|AAAAAAAAFCGAAAAA|2451055|2451084|DEPARTMENT|15|61|Children used to get in the potatoes. Significant, different officers take c|monthly| +1574|AAAAAAAAGCGAAAAA|2451055|2451084|DEPARTMENT|15|62|Emotional, traditional attitudes pick as close, proper restrictions;|monthly| +1575|AAAAAAAAHCGAAAAA|2451055|2451084|DEPARTMENT|15|63|Strange, educational plants discuss too private actions. Inte|monthly| +1576|AAAAAAAAICGAAAAA|2451055|2451084|DEPARTMENT|15|64|Final, domestic decisions should not regard criminal, different men. Clear wor|monthly| +1577|AAAAAAAAJCGAAAAA|2451055|2451084|DEPARTMENT|15|65|Practices will not maintain more experiences. Wages indicate long |monthly| +1578|AAAAAAAAKCGAAAAA|2451055|2451084|DEPARTMENT|15|66|Too advanced horses shift profoundly. Metropolitan, rural women b|monthly| +1579|AAAAAAAALCGAAAAA|2451055|2451084|DEPARTMENT|15|67|Then accurate children may make with a boys. Silly, early interes|monthly| +1580|AAAAAAAAMCGAAAAA||2451084|DEPARTMENT||68|Situations produce like a hands; managers will know. Years|monthly| +1581|AAAAAAAANCGAAAAA|2451055|2451084|DEPARTMENT|15|69|Local yards might take enough; other, private provisions guard soon from a imag|monthly| +1582|AAAAAAAAOCGAAAAA|2451055|2451084|DEPARTMENT|15|70|Black changes must come at a questions. Precious, annual points cov|monthly| +1583|AAAAAAAAPCGAAAAA|2451055|2451084|DEPARTMENT|15|71|Reservations want uncertain studies; japanese regulations mig|monthly| +1584|AAAAAAAAADGAAAAA|2451055|2451084|DEPARTMENT|15|72|Competent hours will apply long, social measures. Relations begin as a businesses. Months rema|monthly| +1585|AAAAAAAABDGAAAAA|2451055|2451084|DEPARTMENT|15|73|Then unacceptable requirements ought to gaze still hostile, general hands. Assets use main conditi|monthly| +1586|AAAAAAAACDGAAAAA|2451055|2451084|DEPARTMENT|15|74|Excellent terms rob there deaf, blue things. Other countries sing spe|monthly| +1587|AAAAAAAADDGAAAAA|2451055|2451084|DEPARTMENT|15|75|Also political members say historic men. Easy equal men|monthly| +1588|AAAAAAAAEDGAAAAA|2451055|2451084|DEPARTMENT|15|76|Upper days could allow upright public buildings. Representatives stop complete kings. Profit|monthly| +1589|AAAAAAAAFDGAAAAA|2451055|2451084|DEPARTMENT|15|77|Also magic businesses may cost low things. Impossible, other points fill nationally |monthly| +1590|AAAAAAAAGDGAAAAA|2451055|2451084|DEPARTMENT|15|78|Obvious plants achieve largely problems. Willing, nuclear arts weigh somewhere|monthly| +1591|AAAAAAAAHDGAAAAA|2451055|2451084|DEPARTMENT|15|79|Physical, industrial doubts attract at all. Activities build often services. High, industrial w|monthly| +1592|AAAAAAAAIDGAAAAA|2451055|2451084|DEPARTMENT|15|80|Certain services go; more blank countries come else w|monthly| +1593|AAAAAAAAJDGAAAAA|2451055|2451084|DEPARTMENT|15|81|However ideological boxes look good hearts. Also convincing members like somewhat inter|monthly| +1594|AAAAAAAAKDGAAAAA||2451084|DEPARTMENT||82|Other, exciting quantities should work yesterday normal, ultimate designs. Social fingers must up|| +1595|AAAAAAAALDGAAAAA|2451055|2451084|DEPARTMENT|15|83|British parents let ahead willing, long services. Single conditions mea|monthly| +1596|AAAAAAAAMDGAAAAA|2451055|2451084|DEPARTMENT|15|84|British, different elections enter as military conditions. Horrib|monthly| +1597|AAAAAAAANDGAAAAA|2451055|2451084|DEPARTMENT|15|85|Appropriate, individual procedures would meet cognitive, good areas. In addition dange|monthly| +1598|AAAAAAAAODGAAAAA|2451055|2451084|DEPARTMENT|15|86|Wonderful firms may not get most. Free months dispel so as with a groups. Public claim|monthly| +1599|AAAAAAAAPDGAAAAA|2451055|2451084|DEPARTMENT|15|87|Occasionally appropriate letters want especially central aspects; elements seem yesterday predom|monthly| +1600|AAAAAAAAAEGAAAAA|2451055|2451084|DEPARTMENT|15|88|Open experts might not use never social records. Other functions mention other solicit|monthly| +1601|AAAAAAAABEGAAAAA|2451055|2451084|DEPARTMENT|15|89|Representatives used to develop police. All new prices should speak so shar|monthly| +1602|AAAAAAAACEGAAAAA|2451055|2451084|DEPARTMENT|15|90|Long, nuclear enemies can settle much mere windows. Roads preach in a feet. Blocks mark. J|monthly| +1603|AAAAAAAADEGAAAAA|2451055|2451084|DEPARTMENT|15|91|Centrally post-war months persuade expensive uses. Considerable, poor players think for|monthly| +1604|AAAAAAAAEEGAAAAA|2451055|2451084|DEPARTMENT|15|92|Here other others help past a resources. Sales could see specialist difficul|monthly| +1605|AAAAAAAAFEGAAAAA|2451055|2451084|DEPARTMENT|15|93|Rivers would not see of course just different servic|monthly| +1606|AAAAAAAAGEGAAAAA|2451055|2451084|DEPARTMENT|15|94|Complex, disabled risks must see with a remarks. Able items know rather |monthly| +1607|AAAAAAAAHEGAAAAA|2451055|2451084|DEPARTMENT|15|95|Years may help. Extraordinary bits show increasingly in a roots. Rapid, separate condit|monthly| +1608|AAAAAAAAIEGAAAAA|2451055|2451084|DEPARTMENT|15|96|Local, double offences shall know on a names. Polite sent|monthly| +1609|AAAAAAAAJEGAAAAA|2451055|2451084|DEPARTMENT|15|97|Companies should not buy now activities. Other, physical things answer however. Tin|monthly| +1610|AAAAAAAAKEGAAAAA|2451055|2451084|DEPARTMENT|15|98|Daily partners used to change with a hands. Major, correc|monthly| +1611|AAAAAAAALEGAAAAA|2451055|2451084|DEPARTMENT|15|99|As serious problems hold well proper, different hands. Collective awards could wander as hard, new |monthly| +1612|AAAAAAAAMEGAAAAA|2451055|2451084|DEPARTMENT|15|100|Previous developers used to make always heads. Modern, open levels |monthly| +1613|AAAAAAAANEGAAAAA|2451055|2451084|DEPARTMENT|15|101|Towns leave more real, twin pieces; common, external forests w|monthly| +1614|AAAAAAAAOEGAAAAA|2451055|2451084|DEPARTMENT|15|102|Ancient faces come too in a months; possible ends step without a policemen|monthly| +1615|AAAAAAAAPEGAAAAA|2451055|2451084|DEPARTMENT|15|103|Vast, unlikely banks open at a citizens. Potential, interesting relations can walk easily by a me|monthly| +1616|AAAAAAAAAFGAAAAA|2451055|2451084|DEPARTMENT|15|104|Advanced, special members use average, various systems. Very significant pl|monthly| +1617|AAAAAAAABFGAAAAA|2451055|2451084|DEPARTMENT|15|105|Emotional reactions take as working parents. Actions find nearly essential cases. U|monthly| +1618|AAAAAAAACFGAAAAA|2451055|2451084|DEPARTMENT|15|106|Ordinary schools used to analyse only only, evil factors. Children say|monthly| +1619|AAAAAAAADFGAAAAA|2451055|2451084|DEPARTMENT|15|107|Foreigners provide now on a plans. Communities ought to reflect days. Given files tak|monthly| +1620|AAAAAAAAEFGAAAAA|2451055|2451084|DEPARTMENT|15|108|Mental countries used to formulate currently especially big sites. Too toxic windows calculate|monthly| +1621|AAAAAAAAFFGAAAAA|2451085|2451114|DEPARTMENT|16|1|Important, private children carry. Secondary, suitable tests rise acceptable sys|monthly| +1622|AAAAAAAAGFGAAAAA|2451085|2451114|DEPARTMENT|16|2|Australian years divert national difficulties. Social, lively exhibitions believe too for a syste|monthly| +1623|AAAAAAAAHFGAAAAA|2451085|2451114|DEPARTMENT|16|3|Remarkable, large prices will form there on the farmers. Courses realise very simply small particip|monthly| +1624|AAAAAAAAIFGAAAAA|2451085|2451114|DEPARTMENT|16|4|Heads hide further. In general natural police get organisms. Limited systems see old, |monthly| +1625|AAAAAAAAJFGAAAAA|2451085|2451114|DEPARTMENT|16|5|Large pp. could come new organs. Events used to like condition|monthly| +1626|AAAAAAAAKFGAAAAA|2451085|2451114|DEPARTMENT|16|6|British, great horses employ young drugs. Surprised, functional findings |monthly| +1627|AAAAAAAALFGAAAAA|2451085|2451114|DEPARTMENT|16|7|Definitions would not mean back functions; courses may not remain deep, round months. At al|monthly| +1628|AAAAAAAAMFGAAAAA|2451085|2451114|DEPARTMENT|16|8|Wooden police used to grasp employees. More common minds intercept ages. Mad|monthly| +1629|AAAAAAAANFGAAAAA|2451085|2451114|DEPARTMENT|16|9|Yet negative changes would act industrial economies. Quite little policies co|monthly| +1630|AAAAAAAAOFGAAAAA|2451085|2451114|DEPARTMENT|16|10|Lives must recognise now again working areas. Bitter operations shall not include forward. Rele|monthly| +1631|AAAAAAAAPFGAAAAA|2451085|2451114|DEPARTMENT|16|11|Crucial benefits drop british stores. New researchers appreciate lo|monthly| +1632|AAAAAAAAAGGAAAAA|2451085|2451114|DEPARTMENT|16|12|High men must not blame so late minerals. Careers like quickly. Deep|monthly| +1633|AAAAAAAABGGAAAAA|2451085|2451114|DEPARTMENT|16|13|Much social arms might not consider only chemicals. Other, l|monthly| +1634|AAAAAAAACGGAAAAA|2451085|2451114|DEPARTMENT|16|14|Typical, unnecessary refugees facilitate now. Firms accommod|monthly| +1635|AAAAAAAADGGAAAAA|2451085|2451114|DEPARTMENT|16|15|Different discussions used to care far; talks put never thin, cheap times. Young solicitors shal|monthly| +1636|AAAAAAAAEGGAAAAA|2451085|2451114|DEPARTMENT|16|16|Pressures experience small, positive years; young, external fees see also accused cities; |monthly| +1637|AAAAAAAAFGGAAAAA|2451085|2451114|DEPARTMENT|16|17|So difficult plants might not make badly undoubted|monthly| +1638|AAAAAAAAGGGAAAAA|2451085|2451114|DEPARTMENT|16|18|Responses determine already backwards other arrangements. Precise, |monthly| +1639|AAAAAAAAHGGAAAAA|2451085|2451114|DEPARTMENT|16|19|In common numerous men may convey. Children will allow years. Already skilled |monthly| +1640|AAAAAAAAIGGAAAAA|2451085|2451114|DEPARTMENT|16|20|Modern places lead for the advantages. Just heavy sales must come still|monthly| +1641|AAAAAAAAJGGAAAAA|2451085|2451114|DEPARTMENT|16|21|Responsible, new companies maintain. Women may travel also complex exp|monthly| +1642|AAAAAAAAKGGAAAAA|2451085|2451114|DEPARTMENT|16|22|There obvious passengers might keep suddenly despite a c|monthly| +1643|AAAAAAAALGGAAAAA|2451085|2451114|DEPARTMENT|16|23|Again small increases withdraw increasingly; new feet |monthly| +1644|AAAAAAAAMGGAAAAA|2451085|2451114|DEPARTMENT|16|24|Conclusions must keep. Shallow arms should avoid only colourful year|monthly| +1645|AAAAAAAANGGAAAAA|2451085|2451114|DEPARTMENT|16|25|So free views feel also; typical variations used to know in a principles. Local creatures could sh|monthly| +1646|AAAAAAAAOGGAAAAA|2451085|2451114|DEPARTMENT|16|26|Major, broken times shall see importantly patient, full rights. Suddenly concerned li|monthly| +1647|AAAAAAAAPGGAAAAA|2451085|2451114|DEPARTMENT|16|27|Years withdraw most various, corresponding areas. |monthly| +1648|AAAAAAAAAHGAAAAA|2451085|2451114|DEPARTMENT|16|28|Major times sleep only trees. Wide, forward forces assist enormous p|monthly| +1649|AAAAAAAABHGAAAAA|2451085|2451114|DEPARTMENT|16|29|Existing events would bear better mass kids. Fathers walk also on an colours. Problem|monthly| +1650|AAAAAAAACHGAAAAA|2451085|2451114|DEPARTMENT|16|30|Medical, important producers go much to a stones. Just just accidents o|monthly| +1651|AAAAAAAADHGAAAAA|2451085|2451114|DEPARTMENT|16|31|Civil, new increases should stop at a matters. Democrat|monthly| +1652|AAAAAAAAEHGAAAAA|2451085|2451114|DEPARTMENT|16|32|Officers must extend more liberal securities. Effects|monthly| +1653|AAAAAAAAFHGAAAAA|2451085|2451114|DEPARTMENT|16|33|Heads make reasons. Efficient, bare leaders could understand mo|monthly| +1654|AAAAAAAAGHGAAAAA|2451085|2451114|DEPARTMENT|16|34|Seconds assist. Newspapers cannot restrict extremely normal, open ways. Soon p|monthly| +1655|AAAAAAAAHHGAAAAA|2451085|2451114|DEPARTMENT|16|35|National leaves know still political profits. Years should accept. Following, accurate operatio|monthly| +1656|AAAAAAAAIHGAAAAA|2451085|2451114|DEPARTMENT|16|36|Today dark months will not meet poor, official journalists. Le|monthly| +1657|AAAAAAAAJHGAAAAA|2451085|2451114|DEPARTMENT|16|37|Inc, individual emotions know at least. Magic, old tasks might strip for a taxes. Various c|monthly| +1658|AAAAAAAAKHGAAAAA|2451085|2451114|DEPARTMENT|16|38|Content, political shows need even. Hundreds come. Opinions give only. As major carers reach d|monthly| +1659|AAAAAAAALHGAAAAA|2451085|2451114|DEPARTMENT|16|39|Governments might creep at least international terms. Certainly back records find closely al|monthly| +1660|AAAAAAAAMHGAAAAA|2451085|2451114|DEPARTMENT|16|40|Later steep duties will get as. Activities might like again; programme|monthly| +1661|AAAAAAAANHGAAAAA|2451085|2451114|DEPARTMENT|16|41|Voices must run proper improvements. Clear, conservative positions shall manage ever most ma|monthly| +1662|AAAAAAAAOHGAAAAA|2451085|2451114|DEPARTMENT|16|42|Rarely occasional employees might match quick, young volunteers. Bc other winners may help almost |monthly| +1663|AAAAAAAAPHGAAAAA|2451085|2451114|DEPARTMENT|16|43|Candidates used to pay almost religious categories; evidently right dimensions could |monthly| +1664|AAAAAAAAAIGAAAAA|2451085|2451114|DEPARTMENT|16|44|Alternative, black objections challenge simply to a achievements. Also fede|monthly| +1665|AAAAAAAABIGAAAAA|2451085|2451114|DEPARTMENT|16|45|Real, great findings choose in a others. Cultures agree wa|monthly| +1666|AAAAAAAACIGAAAAA|2451085|2451114|DEPARTMENT|16|46|Particularly eastern jobs see jointly in a females. Techn|monthly| +1667|AAAAAAAADIGAAAAA|2451085|2451114|DEPARTMENT|16|47|Records handle as. Long inches tell such as a claims. Sound, good areas could p|monthly| +1668|AAAAAAAAEIGAAAAA|2451085|2451114|DEPARTMENT|16|48|Various elections might start only too nuclear minutes. As likely rou|monthly| +1669|AAAAAAAAFIGAAAAA|2451085|2451114|DEPARTMENT|16|49|Birds cannot run please different methods. Probably direct conditions cannot take in order royal,|monthly| +1670|AAAAAAAAGIGAAAAA|2451085|2451114|DEPARTMENT|16|50|Criminal, corporate cells want employees; professional, dead drivers must clear. So sati|monthly| +1671|AAAAAAAAHIGAAAAA|2451085|2451114|DEPARTMENT|16|51|Other, yellow forces would use most in response to the models. Colours wou|monthly| +1672|AAAAAAAAIIGAAAAA|2451085|2451114|DEPARTMENT|16|52|Just lost women highlight specific, certain troops. Living syste|monthly| +1673|AAAAAAAAJIGAAAAA|2451085|2451114|DEPARTMENT|16|53|Able miles offer across possibly small examples. New, english chapters may make on board the bodi|monthly| +1674|AAAAAAAAKIGAAAAA|2451085|2451114|DEPARTMENT|16|54|Simply sorry movements go late, italian systems; days date then o|monthly| +1675|AAAAAAAALIGAAAAA|2451085|2451114|DEPARTMENT|16|55|Early poles believe also alive, hard trees. Warm, visible police shall give too |monthly| +1676|AAAAAAAAMIGAAAAA|2451085|2451114|DEPARTMENT|16|56|Coherent, free words forgo into a budgets. Quickly regular children will store however. Le|monthly| +1677|AAAAAAAANIGAAAAA|2451085|2451114|DEPARTMENT|16|57|Fast underlying versions could say in a definitions. Meal|monthly| +1678|AAAAAAAAOIGAAAAA|2451085|2451114|DEPARTMENT|16|58|Wide, original offices would provide silly, special i|monthly| +1679|AAAAAAAAPIGAAAAA|2451085|2451114|DEPARTMENT|16|59|Companies might not oblige; social workers present straight |monthly| +1680|AAAAAAAAAJGAAAAA|2451085|2451114|DEPARTMENT|16|60|Doors predict domestic poles. Violent, left heads can afford eyes. Remarkable ministers score|monthly| +1681|AAAAAAAABJGAAAAA|2451085|2451114|DEPARTMENT|16|61|Local speeches might put in order with the windows. Courts arise relatively decisions. |monthly| +1682|AAAAAAAACJGAAAAA|2451085|2451114|DEPARTMENT|16|62|Military women shall take private, interesting teachers; fair phrases need again shows. So main v|monthly| +1683|AAAAAAAADJGAAAAA|2451085|2451114|DEPARTMENT|16|63|Prime, comprehensive matters forget suddenly as a wars. Arts should add together yesterday sex|monthly| +1684|AAAAAAAAEJGAAAAA|2451085|2451114|DEPARTMENT|16|64|White girls drill on a categories. Small months should not tell direct|monthly| +1685|AAAAAAAAFJGAAAAA|2451085|2451114|DEPARTMENT|16|65|Simple spots might play completely national numbers; male, european partners run. Worthy |monthly| +1686|AAAAAAAAGJGAAAAA|2451085|2451114|DEPARTMENT|16|66|Tired, light services shall not abolish here fairly open sales. Useful features say yet eye|monthly| +1687|AAAAAAAAHJGAAAAA|2451085|2451114|DEPARTMENT|16|67|In order other gardens must not fit most no doubt good places. Too responsible members |monthly| +1688|AAAAAAAAIJGAAAAA|2451085|2451114|DEPARTMENT|16|68|Successful, general ways release musical, dark ears. Doubts shall not say s|monthly| +1689|AAAAAAAAJJGAAAAA|2451085|2451114|DEPARTMENT|16|69|New years shall not help here other techniques. Effectively g|monthly| +1690|AAAAAAAAKJGAAAAA|2451085|2451114|DEPARTMENT|16|70|Different, comfortable environments may clear yet years. Eyes think; police must|monthly| +1691|AAAAAAAALJGAAAAA|2451085|2451114|DEPARTMENT|16|71|Sheer differences should not change popular, great organisations. Equally physical men call in the |monthly| +1692|AAAAAAAAMJGAAAAA|2451085|2451114|DEPARTMENT|16|72|Best available shapes can take on a deaths. Hard good police might enlarge contemporary windows.|monthly| +1693|AAAAAAAANJGAAAAA|2451085|2451114|||73|Of course subsequent stocks combat communists. Cheap, labour terms speak eventual, double frien|monthly| +1694|AAAAAAAAOJGAAAAA|2451085|2451114|DEPARTMENT|16|74|Individual shareholders get only later different services. Places would not cut accordin|monthly| +1695|AAAAAAAAPJGAAAAA|2451085|2451114|DEPARTMENT|16|75|Rare, civil countries pose usually; probable residents ou|monthly| +1696|AAAAAAAAAKGAAAAA|2451085|2451114||16|76|Private, particular systems should concentrate that is useful models. Below charming d|monthly| +1697|AAAAAAAABKGAAAAA|2451085|2451114|DEPARTMENT|16|77|Well-known, massive offers may forget at least scottish mechanisms. |monthly| +1698|AAAAAAAACKGAAAAA|2451085|2451114|DEPARTMENT|16|78|Reductions minimize especially lately new barriers; lead|monthly| +1699|AAAAAAAADKGAAAAA|2451085|2451114|DEPARTMENT|16|79|Powers collect popular, proposed years. Distant, seco|monthly| +1700|AAAAAAAAEKGAAAAA|2451085|2451114|DEPARTMENT|16|80|Institutional, foreign cars used to judge so. Elsewhere good sentences s|monthly| +1701|AAAAAAAAFKGAAAAA|2451085|2451114|DEPARTMENT|16|81|Following, specific circumstances sell particularly oth|monthly| +1702|AAAAAAAAGKGAAAAA||2451114||16||Complete, pretty books escape. Theoretical fields taste rapidly || +1703|AAAAAAAAHKGAAAAA|2451085|2451114|DEPARTMENT|16|83|Annual, large patterns like early members. Old surfaces mu|monthly| +1704|AAAAAAAAIKGAAAAA|2451085|2451114|DEPARTMENT|16|84|Immediate children cover equally front, little doctors; re|monthly| +1705|AAAAAAAAJKGAAAAA|2451085|2451114|DEPARTMENT|16|85|Quite central auditors shall come more in a notions. Suitably unique titles look therefore patie|monthly| +1706|AAAAAAAAKKGAAAAA|2451085|2451114|DEPARTMENT|16|86|Efficiently voluntary proteins ought to go later for a authorities. Eventually vast skil|monthly| +1707|AAAAAAAALKGAAAAA|2451085|2451114|DEPARTMENT|16|87|Medical words rely for example traditional experiments. Rea|monthly| +1708|AAAAAAAAMKGAAAAA|2451085|2451114|DEPARTMENT|16|88|Alone, artificial women realise rather eventually incredible products. |monthly| +1709|AAAAAAAANKGAAAAA|2451085|2451114|DEPARTMENT|16|89|Rather european things can keep fair hands. Sure practices shall profit roughly intern|monthly| +1710|AAAAAAAAOKGAAAAA|2451085|2451114|DEPARTMENT|16|90|Other, top services buy clean democrats. Then small banks ought to love professional cases. |monthly| +1711|AAAAAAAAPKGAAAAA|2451085|2451114|DEPARTMENT|16|91|Fields live men. Whole, early times will not give just social pages. Cultural pupils |monthly| +1712|AAAAAAAAALGAAAAA|2451085|2451114|DEPARTMENT|16|92|Sadly wise leaders trust situations. Important firms see wild, soci|monthly| +1713|AAAAAAAABLGAAAAA|2451085|2451114|DEPARTMENT|16|93|Goals go decisive weeks; associated, fiscal teams listen as still so-called stores. Now certain cen|monthly| +1714|AAAAAAAACLGAAAAA|2451085|2451114|DEPARTMENT|16|94|Articles make then from a peasants; narrow, free times rewrite regional winners. Independ|monthly| +1715|AAAAAAAADLGAAAAA|2451085|2451114|DEPARTMENT|16|95|Actually keen principles accommodate much in a industries; fortunate, thin resp|monthly| +1716|AAAAAAAAELGAAAAA|2451085|2451114|DEPARTMENT|16|96|Secondary units manage only above, good boys. Industrial differences coor|monthly| +1717|AAAAAAAAFLGAAAAA|2451085|2451114|DEPARTMENT|16|97|Elsewhere gentle decisions award also rather than a ingredients. Points play als|monthly| +1718|AAAAAAAAGLGAAAAA|2451085|2451114|DEPARTMENT|16|98|British, revolutionary ministers believe. Months will |monthly| +1719|AAAAAAAAHLGAAAAA|2451085|2451114|DEPARTMENT|16|99|Full, japanese problems go future forces; films determine too at leas|monthly| +1720|AAAAAAAAILGAAAAA|2451085|2451114|DEPARTMENT|16|100|Personnel study aside like a services. Subsequently level businesses shall not alter currently. |monthly| +1721|AAAAAAAAJLGAAAAA|2451085|2451114|DEPARTMENT|16|101|Poor, english manufacturers show from a breasts. Particular occasions used|monthly| +1722|AAAAAAAAKLGAAAAA|2451085|2451114|DEPARTMENT|16|102|Nearly wide goods know. Ago voluntary rates look harder select lads; selective, fr|monthly| +1723|AAAAAAAALLGAAAAA|2451085|2451114|DEPARTMENT|16|103|High other sources shall expect weeks. Continually successful years wi|monthly| +1724|AAAAAAAAMLGAAAAA|2451085|2451114|DEPARTMENT|16|104|However southern materials used to identify commitments. Long annual |monthly| +1725|AAAAAAAANLGAAAAA|2451085|2451114|DEPARTMENT|16|105|Long thousands leave with a developments. Political, unable methods form very clear months. Oth|monthly| +1726|AAAAAAAAOLGAAAAA|2451085|2451114|DEPARTMENT|16|106|Sorry, positive seeds would not allow local, determined men. Necessary, social|monthly| +1727|AAAAAAAAPLGAAAAA|2451085|2451114|DEPARTMENT|16|107|Merely high respects see at all about a results. Germans might light passengers. Special, impossib|monthly| +1728|AAAAAAAAAMGAAAAA|2451085|2451114|DEPARTMENT|16|108|Metres used to imagine complete, special questions. As inte|monthly| +1729|AAAAAAAABMGAAAAA|2451115|2451144|DEPARTMENT|17|1|Achievements would not know. Powers operate never |monthly| +1730|AAAAAAAACMGAAAAA|2451115|2451144|DEPARTMENT|17|2|Social requirements identify to a purposes. Important, black feet attend only scientifi|monthly| +1731|AAAAAAAADMGAAAAA|2451115|2451144|DEPARTMENT|17|3|Local, right workers should not go only. Files see sharp problems. Wrong|monthly| +1732|AAAAAAAAEMGAAAAA|2451115|2451144|DEPARTMENT|17|4|Even so individual feet hear still by a charges. New nurses can tell here recent courses. Sure fre|monthly| +1733|AAAAAAAAFMGAAAAA|2451115|2451144|DEPARTMENT|17|5|Polls will like ever possible protests. Good services will find so small parts. Courses|monthly| +1734|AAAAAAAAGMGAAAAA|2451115|2451144|DEPARTMENT|17|6|Consequences protect brilliantly by a signs. Proper words may draw triumphantly |monthly| +1735|AAAAAAAAHMGAAAAA|2451115|2451144|DEPARTMENT|17|7|Open police would pretend purely other duties. Near new circumstances like. Mass, frequent|monthly| +1736|AAAAAAAAIMGAAAAA|2451115|2451144|DEPARTMENT|17|8|Engineers should not become weak, ready thousands;|monthly| +1737|AAAAAAAAJMGAAAAA|2451115|2451144|DEPARTMENT|17|9|Major interests live soon; extended words ought to take only weeks. Almost european memb|monthly| +1738|AAAAAAAAKMGAAAAA|2451115|2451144|DEPARTMENT|17|10|Full, other plans should accept highly free, uncomfortable states. Votes rest concerned to|monthly| +1739|AAAAAAAALMGAAAAA|2451115|2451144|DEPARTMENT|17|11|Others receive statutory relationships. Resources bear other negotiatio|monthly| +1740|AAAAAAAAMMGAAAAA|2451115|2451144|DEPARTMENT|17|12|Successful barriers should not affect however agricultural, left servic|monthly| +1741|AAAAAAAANMGAAAAA|2451115|2451144|DEPARTMENT|17|13|Businesses ought to manage at a pieces; social years c|monthly| +1742|AAAAAAAAOMGAAAAA|2451115|2451144|DEPARTMENT|17|14|Ago modern connections shall feel also economic real newspa|monthly| +1743|AAAAAAAAPMGAAAAA|2451115|2451144|DEPARTMENT|17|15|Dark, white leaders ought to allow on a payments; cases see therefore still goo|monthly| +1744|AAAAAAAAANGAAAAA|2451115|2451144|DEPARTMENT|17|16|Short boys could not preserve nearly generally effective foreigners|monthly| +1745|AAAAAAAABNGAAAAA|2451115|2451144|DEPARTMENT|17|17|Specific conclusions must see really never experimental parents; extremel|monthly| +1746|AAAAAAAACNGAAAAA|2451115|2451144|DEPARTMENT|17|18|So just occasions might not move new rooms. High facilit|monthly| +1747|AAAAAAAADNGAAAAA|2451115|2451144|DEPARTMENT|17|19|Difficult, glad males deal above. Civil, early times used to miss quietly anywher|monthly| +1748|AAAAAAAAENGAAAAA|2451115|2451144|DEPARTMENT|17|20|Short factors concede in a purposes; always other co|monthly| +1749|AAAAAAAAFNGAAAAA|2451115|2451144|DEPARTMENT|17|21|Satisfactory, new muscles may take here good colleagues. Passengers woul|monthly| +1750|AAAAAAAAGNGAAAAA|2451115|2451144|DEPARTMENT|17|22|Only appointed employees follow necessarily new, active agencies. Even possible unions gath|monthly| +1751|AAAAAAAAHNGAAAAA|2451115|2451144|DEPARTMENT|17|23|Experimental, effective standards carry ever dutch pupils. Messages may pay there|monthly| +1752|AAAAAAAAINGAAAAA|2451115|2451144|DEPARTMENT|17|24|Political members discover approximately social sessions. Necessary,|monthly| +1753|AAAAAAAAJNGAAAAA|2451115|2451144|DEPARTMENT|17|25|Records could not point really complex fields. Proposals study together.|monthly| +1754|AAAAAAAAKNGAAAAA|2451115|2451144|DEPARTMENT|17|26|Courts force public amounts. Now responsible children ensure for a bonds. Frequent, light men mini|monthly| +1755|AAAAAAAALNGAAAAA|2451115|2451144|DEPARTMENT|17|27|Layers could come; now previous mothers remember obviously public, fut|monthly| +1756|AAAAAAAAMNGAAAAA|2451115|2451144|DEPARTMENT|17|28|Prices used to play like a arms. Creative comments canno|monthly| +1757|AAAAAAAANNGAAAAA|2451115|2451144|DEPARTMENT|17|29|New, obvious ministers may reach properties. Then powerful supplies move |monthly| +1758|AAAAAAAAONGAAAAA|2451115|2451144|DEPARTMENT|17|30|Potential plans shall wish then beneficial weeks. Always special companies work other, large court|monthly| +1759|AAAAAAAAPNGAAAAA|2451115|2451144|DEPARTMENT|17|31|Then disastrous skills require in a others. Different, bloody children clear closer impo|monthly| +1760|AAAAAAAAAOGAAAAA|2451115|2451144|DEPARTMENT|17|32|Ears used to give only other pounds. Popular marks may not deliv|monthly| +1761|AAAAAAAABOGAAAAA|2451115|2451144|DEPARTMENT|17|33|Free, vague changes know never in a days. Light, visible emotions turn too ind|monthly| +1762|AAAAAAAACOGAAAAA|2451115|2451144|DEPARTMENT|17|34|Together social appeals win at least to a issues. Consumers shoul|monthly| +1763|AAAAAAAADOGAAAAA|2451115|2451144|DEPARTMENT|17|35|Head, apparent services inquire now weak, average |monthly| +1764|AAAAAAAAEOGAAAAA|2451115|2451144|DEPARTMENT|17|36|Bloody, constitutional seconds get immediate, temporary children. Leaders could |monthly| +1765|AAAAAAAAFOGAAAAA|2451115|2451144|DEPARTMENT|17|37|Big teams could not agree then only alive weeks; communists could not look western pounds. Compr|monthly| +1766|AAAAAAAAGOGAAAAA|2451115|2451144|DEPARTMENT|17|38|National departments enhance here neighbouring, unlike diffe|monthly| +1767|AAAAAAAAHOGAAAAA|2451115|2451144|DEPARTMENT|17|39|Individual, unnecessary effects explain royal, successful yea|monthly| +1768|AAAAAAAAIOGAAAAA|2451115|2451144|DEPARTMENT|17|40|Particular, normal men used to accept however for the children. Japanese|monthly| +1769|AAAAAAAAJOGAAAAA|2451115|2451144|DEPARTMENT|17|41|Solid roots could become as. Great, poor calls could proceed always bad hands. Days sit movi|monthly| +1770|AAAAAAAAKOGAAAAA|2451115|2451144|DEPARTMENT|17|42|Only industrial barriers understand skills; particular, conventi|monthly| +1771|AAAAAAAALOGAAAAA|2451115|2451144|DEPARTMENT|17|43|Good, secret officials must convince professional responsibilities. Successful terms could turn|monthly| +1772|AAAAAAAAMOGAAAAA|2451115|2451144|DEPARTMENT|17|44|Empty, serious children shall not control meanwhile here w|monthly| +1773|AAAAAAAANOGAAAAA|2451115|2451144|DEPARTMENT|17|45|Expensive, industrial studies pick more various premises. Students give|monthly| +1774|AAAAAAAAOOGAAAAA|2451115|2451144|DEPARTMENT|17|46|Costs ought to like most entirely limited children. Ade|monthly| +1775|AAAAAAAAPOGAAAAA|2451115|2451144|DEPARTMENT|17|47|Lives get. Enough national hundreds say away attractive,|monthly| +1776|AAAAAAAAAPGAAAAA|2451115|2451144|DEPARTMENT|17|48|Actual, available years shall not make perhaps public, minor unit|monthly| +1777|AAAAAAAABPGAAAAA|2451115|2451144|DEPARTMENT|17|49|Books would visit appropriate owners. White prices retain also apparently deaf trees. |monthly| +1778|AAAAAAAACPGAAAAA|2451115|2451144|DEPARTMENT|17|50|Experienced, primary prices flow approximately. Old players compete me|monthly| +1779|AAAAAAAADPGAAAAA|2451115|2451144|DEPARTMENT|17|51|Important economies play activities. Great relationships could visit as; still british lines|monthly| +1780|AAAAAAAAEPGAAAAA|2451115|2451144|DEPARTMENT|17|52|Just creative homes learn somewhere in the patients. Ex|monthly| +1781|AAAAAAAAFPGAAAAA|2451115|2451144|DEPARTMENT|17|53|Different, great courts can sue. Individual, regional participants like automatically ce|monthly| +1782|AAAAAAAAGPGAAAAA|2451115|2451144|DEPARTMENT|17|54|National steps may not like; unlikely residents gain as concerned lines. Mentally other teams |monthly| +1783|AAAAAAAAHPGAAAAA|2451115|2451144|DEPARTMENT|17|55|Large children stay somewhere then irish stars. Thickly elderly we|monthly| +1784|AAAAAAAAIPGAAAAA|2451115|2451144|DEPARTMENT|17|56|Places ought to try still families; successful, high reasons shoul|monthly| +1785|AAAAAAAAJPGAAAAA|2451115|2451144|DEPARTMENT|17|57|Great, coming assumptions ought to shout other sources. Only, bad ports lift here hidden, pr|monthly| +1786|AAAAAAAAKPGAAAAA|2451115|2451144|DEPARTMENT|17|58|Bright, similar courses deserve elsewhere samples; dist|monthly| +1787|AAAAAAAALPGAAAAA|2451115|2451144|DEPARTMENT|17|59|Usual, marvellous officers would not prevent new, political units. Local |monthly| +1788|AAAAAAAAMPGAAAAA|2451115|2451144|DEPARTMENT|17|60|However national floors used to take. Local, overseas l|monthly| +1789|AAAAAAAANPGAAAAA|2451115|2451144|DEPARTMENT|17|61|Political, legal babies might give. Concentrations must not begin ever love|monthly| +1790|AAAAAAAAOPGAAAAA|2451115|2451144|DEPARTMENT|17|62|Men cannot order forever afraid regulations. Troops|monthly| +1791|AAAAAAAAPPGAAAAA|2451115|2451144|DEPARTMENT|17|63|Less national amounts cannot occur again plain avail|monthly| +1792|AAAAAAAAAAHAAAAA|2451115|2451144|DEPARTMENT|17|64|More different meanings shall back thus appointments. Books might hope more than early, rich piec|monthly| +1793|AAAAAAAABAHAAAAA|2451115|2451144|DEPARTMENT|17|65|Secondary cells serve. Old, single boards could live enough defensive obligations. Current th|monthly| +1794|AAAAAAAACAHAAAAA|2451115|2451144|DEPARTMENT|17|66|Then international resources should not implement finally sentences. British, bright words show|monthly| +1795|AAAAAAAADAHAAAAA|2451115|2451144|DEPARTMENT|17|67|Lines lose depths. Active numbers ought to give already different rep|monthly| +1796|AAAAAAAAEAHAAAAA|2451115|2451144|DEPARTMENT|17|68|Apparently coloured papers enable primarily always full subjects. Over new houses exercise |monthly| +1797|AAAAAAAAFAHAAAAA|2451115|2451144|DEPARTMENT|17|69|Ago small users ought to make very in the devices. New articles would clarify papers. Internation|monthly| +1798|AAAAAAAAGAHAAAAA|2451115|2451144|DEPARTMENT|17|70|Parents would play significant, separate games. Institutions change somewhat grateful,|monthly| +1799|AAAAAAAAHAHAAAAA|2451115|2451144|DEPARTMENT|17|71|Private, respectable chips conclude. Main, interesting members should obey secret,|monthly| +1800|AAAAAAAAIAHAAAAA|2451115|2451144|DEPARTMENT|17|72|So underlying years compile so so quiet years. Numbers accompany el|monthly| +1801|AAAAAAAAJAHAAAAA|2451115|2451144|DEPARTMENT|17|73|Ideal others need useful, old groups. Lips shall turn foreign w|monthly| +1802|AAAAAAAAKAHAAAAA|2451115|2451144|DEPARTMENT|17|74|Waste actions find earlier necessary projects; almost different ch|monthly| +1803|AAAAAAAALAHAAAAA|2451115|2451144|DEPARTMENT|17|75|Words would not support into a eyes. Customers used to proceed more mutual women. W|monthly| +1804|AAAAAAAAMAHAAAAA|2451115|2451144|DEPARTMENT|17|76|At least essential projects ought to accumulate northwards capi|monthly| +1805|AAAAAAAANAHAAAAA|2451115|2451144|DEPARTMENT|17|77|Important, additional lights drop. At all human conditio|monthly| +1806|AAAAAAAAOAHAAAAA|2451115|2451144|DEPARTMENT|17|78|Possible, short ways see british countries; probably normal orders can dominate then yester|monthly| +1807|AAAAAAAAPAHAAAAA|2451115|2451144|DEPARTMENT|17|79|Offices contribute both sources. Colleagues shall ask here essenti|monthly| +1808|AAAAAAAAABHAAAAA|2451115|2451144|DEPARTMENT|17|80|Still perfect police note sharply royal, overseas colours. H|monthly| +1809|AAAAAAAABBHAAAAA|2451115|2451144|DEPARTMENT|17|81|Books should get from an territories; true, possible|monthly| +1810|AAAAAAAACBHAAAAA|2451115|2451144||17||Examples can pretend today blue minds. Thick special features could understan|| +1811|AAAAAAAADBHAAAAA|2451115|2451144|DEPARTMENT|17|83|Combinations should stand good, able schools. Possibly conscious risks remain; later lo|monthly| +1812|AAAAAAAAEBHAAAAA|2451115|2451144|DEPARTMENT|17|84|Important, elderly entries would hide local girls. Cor|monthly| +1813|AAAAAAAAFBHAAAAA|2451115|2451144|DEPARTMENT|17|85|Physical governments would know much never front countries. Ind|monthly| +1814|AAAAAAAAGBHAAAAA|2451115|2451144|DEPARTMENT|17|86|Great, practical schools demonstrate still. Spanish studies m|monthly| +1815|AAAAAAAAHBHAAAAA|2451115|2451144|DEPARTMENT|17|87|Excellent, interesting boys bring. Appropriate boys know more potentially |monthly| +1816|AAAAAAAAIBHAAAAA|2451115|2451144|DEPARTMENT|17|88|Again famous issues spend to the products. Christian suggestions understan|monthly| +1817|AAAAAAAAJBHAAAAA|2451115|2451144|DEPARTMENT|17|89|Likely, blue years defeat annual years. Modern, regu|monthly| +1818|AAAAAAAAKBHAAAAA|2451115|2451144|DEPARTMENT|17|90|Mental countries pick really by a estimates. Happy options will not foster even. Anywa|monthly| +1819|AAAAAAAALBHAAAAA|2451115|||17|91||| +1820|AAAAAAAAMBHAAAAA|2451115|2451144|DEPARTMENT|17|92|Often specific features say in a policies. Years used to be|monthly| +1821|AAAAAAAANBHAAAAA|2451115|2451144|DEPARTMENT|17|93|Corresponding, white parties must trust so without a candidates. Common, appropriate arts wou|monthly| +1822|AAAAAAAAOBHAAAAA|2451115|2451144|DEPARTMENT|17|94|Neither great supporters bet now british contents. Eventually severe occasions |monthly| +1823|AAAAAAAAPBHAAAAA|2451115|2451144|DEPARTMENT|17|95|Full winds ought to see only well-known personnel. Low,|monthly| +1824|AAAAAAAAACHAAAAA|2451115|2451144|DEPARTMENT|17|96|Working women keep then to a settings. Dry accounts acquire. High, fo|monthly| +1825|AAAAAAAABCHAAAAA|2451115|2451144|DEPARTMENT|17|97|Chronic friends may know so long, small persons. More subsequent agencies produce tom|monthly| +1826|AAAAAAAACCHAAAAA|2451115|2451144|DEPARTMENT|17|98|Things show shares. Rates may become as a circumstan|monthly| +1827|AAAAAAAADCHAAAAA|2451115|2451144|DEPARTMENT|17|99|Explicit, military hands shall lose even large prisoners. About es|monthly| +1828|AAAAAAAAECHAAAAA|2451115|2451144|DEPARTMENT|17|100|Married, autonomous ministers might remember more extra, underlying teachers; healthy subject|monthly| +1829|AAAAAAAAFCHAAAAA|2451115|2451144|DEPARTMENT|17|101|Brilliant employers know suddenly great, difficult stations. Particular inte|monthly| +1830|AAAAAAAAGCHAAAAA|2451115|2451144|DEPARTMENT|17|102|Together initial papers used to move single influences; later good c|monthly| +1831|AAAAAAAAHCHAAAAA|2451115|2451144|DEPARTMENT|17|103|Sudden, new trees would set serious, civil creatures. Kids could put. Fine points used to r|monthly| +1832|AAAAAAAAICHAAAAA|2451115|2451144|DEPARTMENT|17|104|Old machines lead quite different instruments. Naturally major clubs might not |monthly| +1833|AAAAAAAAJCHAAAAA|2451115|2451144|DEPARTMENT|17|105|Terms spread officers. Senior, mechanical forces realise totally marginal accounts. Difficult|monthly| +1834|AAAAAAAAKCHAAAAA|2451115|2451144|DEPARTMENT|17|106|Often existing estates shall not provide. Cold, ent|monthly| +1835|AAAAAAAALCHAAAAA|2451115|2451144|DEPARTMENT|17|107|Instead black sections may reach important, physical programmes. Li|monthly| +1836|AAAAAAAAMCHAAAAA|2451115|2451144|DEPARTMENT|17|108|Factors could offer wide, integrated issues. Light, main f|monthly| +1837|AAAAAAAANCHAAAAA|2451145|2451174|DEPARTMENT|18|1|Champions should not increase economic, different men. Numbers get normally questions. P|monthly| +1838|AAAAAAAAOCHAAAAA|2451145|2451174|DEPARTMENT|18|2|Rich notes shall not discuss through the offices. New, bizarre buildi|monthly| +1839|AAAAAAAAPCHAAAAA|2451145|2451174|DEPARTMENT|18|3|Available parents may supply otherwise old, physical|monthly| +1840|AAAAAAAAADHAAAAA|2451145|2451174|DEPARTMENT|18|4|Mental, wonderful standards could not follow reasonably; essential, other loans must id|monthly| +1841|AAAAAAAABDHAAAAA|2451145|2451174|DEPARTMENT|18|5|Workers could tell. There slow attitudes may supply dependent citizens; systemati|monthly| +1842|AAAAAAAACDHAAAAA|2451145|2451174|DEPARTMENT|18|6|Regular, optimistic scholars start among the affairs. Today distinguished g|monthly| +1843|AAAAAAAADDHAAAAA|2451145|2451174|DEPARTMENT|18|7|Serious years lead different authorities; resources bru|monthly| +1844|AAAAAAAAEDHAAAAA|2451145|2451174|DEPARTMENT|18|8|High good members tell earlier at a children. Musicians look at the fee|monthly| +1845|AAAAAAAAFDHAAAAA|2451145|2451174|DEPARTMENT|18|9|Operational, other experts speak closely. Open rules enjoy now now alone fiel|monthly| +1846|AAAAAAAAGDHAAAAA|2451145|2451174|DEPARTMENT|18|10|Around bottom years might make. Occasions will give parents. Meetings mean even governments. |monthly| +1847|AAAAAAAAHDHAAAAA|2451145|2451174|DEPARTMENT|18|11|Royal, northern goods make annual explanations. Over glad|monthly| +1848|AAAAAAAAIDHAAAAA|2451145|2451174|DEPARTMENT|18|12|Savings look. Regional, other years shall not make ahead of the counties. Different matters tell ha|monthly| +1849|AAAAAAAAJDHAAAAA|2451145|2451174|DEPARTMENT|18|13|Magistrates may take basic relations. Words expose both on |monthly| +1850|AAAAAAAAKDHAAAAA|2451145|2451174|DEPARTMENT|18|14|Just little times might restrict yet sad rates. Widely likely participa|monthly| +1851|AAAAAAAALDHAAAAA|2451145|2451174|DEPARTMENT|18|15|Well useful messages might put quickly. Fat, prime hours used to s|monthly| +1852|AAAAAAAAMDHAAAAA|2451145|2451174|DEPARTMENT|18|16|Customers will answer upon a months. Little resources should know carefully per|monthly| +1853|AAAAAAAANDHAAAAA|2451145|2451174|DEPARTMENT|18|17|Scientific businesses might not offer. Windows spend already aside active tests. Ot|monthly| +1854|AAAAAAAAODHAAAAA|2451145|2451174|DEPARTMENT|18|18|Hands cope here early photographs. General, small cups spe|monthly| +1855|AAAAAAAAPDHAAAAA|2451145|2451174|DEPARTMENT|18|19|Hot teams play at all american arrangements. Much strong |monthly| +1856|AAAAAAAAAEHAAAAA|2451145|2451174|DEPARTMENT|18|20|Orange hands shall exist more reasonable meanings. Always flat benefits would mee|monthly| +1857|AAAAAAAABEHAAAAA|2451145|2451174|DEPARTMENT|18|21|Possible conditions descend soon needs. Questions might understand very spe|monthly| +1858|AAAAAAAACEHAAAAA|2451145|2451174|DEPARTMENT|18|22|Dangerously minor girls may rain just wrong clear banks; small figures should read funny sch|monthly| +1859|AAAAAAAADEHAAAAA|2451145|2451174|DEPARTMENT|18|23|Experienced complaints might make above heels; able ears sign initial, hot children. Sh|monthly| +1860|AAAAAAAAEEHAAAAA|2451145|2451174|DEPARTMENT|18|24|Greatly direct years would not live in order awful centuries. Earlier serious syste|monthly| +1861|AAAAAAAAFEHAAAAA|2451145|2451174|DEPARTMENT|18|25|Entirely capital families ought to guide. So desirable pictures persuade unexpec|monthly| +1862|AAAAAAAAGEHAAAAA||2451174|||||monthly| +1863|AAAAAAAAHEHAAAAA|2451145|2451174|DEPARTMENT|18|27|Problems call very. Consequences must win eventually wel|monthly| +1864|AAAAAAAAIEHAAAAA|2451145|2451174|DEPARTMENT|18|28|Times expect still imports. Distinctly net firms go once more available rates. Here double |monthly| +1865|AAAAAAAAJEHAAAAA|2451145|2451174|DEPARTMENT|18|29|Strong, high children ought to deny absolutely dogs. Technical, glad|monthly| +1866|AAAAAAAAKEHAAAAA|2451145|2451174|DEPARTMENT|18|30|Considerable procedures can like again old adults. Odd reasons afford earlier mana|monthly| +1867|AAAAAAAALEHAAAAA|2451145|2451174|DEPARTMENT|18|31|Exotic issues accept sisters. Long useless societies should not provide also official noble par|monthly| +1868|AAAAAAAAMEHAAAAA|2451145|2451174|DEPARTMENT|18|32|Clear months might improve. Please likely activities want mo|monthly| +1869|AAAAAAAANEHAAAAA|2451145|2451174|DEPARTMENT|18|33|Today grateful matters meet logically lengths. Rea|monthly| +1870|AAAAAAAAOEHAAAAA|2451145|2451174|DEPARTMENT|18|34|Natural bodies must persuade weeks. English, certain figures |monthly| +1871|AAAAAAAAPEHAAAAA|2451145|2451174|DEPARTMENT|18|35|Significant speakers tell detailed, complete women. Normal powers should a|monthly| +1872|AAAAAAAAAFHAAAAA|2451145|2451174|DEPARTMENT|18|36|European houses see also; ugly others contact large, personal women. N|monthly| +1873|AAAAAAAABFHAAAAA|2451145|2451174|DEPARTMENT|18|37|Things let on the parties. Resources eliminate too al|monthly| +1874|AAAAAAAACFHAAAAA|2451145|2451174|DEPARTMENT|18|38|Tests should not come various years; whole firms should address developments|monthly| +1875|AAAAAAAADFHAAAAA|2451145|2451174|DEPARTMENT|18|39|Enormous steps know fierce, foreign patients. Cells show even. Resources can|monthly| +1876|AAAAAAAAEFHAAAAA|2451145|2451174|DEPARTMENT|18|40|Successfully national cities continue therefore payments. Often |monthly| +1877|AAAAAAAAFFHAAAAA|2451145|2451174|DEPARTMENT|18|41|Particularly minor goods listen there good numbers. Up to sharp criticisms return nev|monthly| +1878|AAAAAAAAGFHAAAAA|2451145|2451174|DEPARTMENT|18|42|Human courts could include now opportunities. Numerous cells gain |monthly| +1879|AAAAAAAAHFHAAAAA|2451145|2451174|DEPARTMENT|18|43|Types save gently councils. National points should not seem as proper patients. However avai|monthly| +1880|AAAAAAAAIFHAAAAA|2451145|2451174|DEPARTMENT|18|44|Good, commercial windows speak. National negotiations shout ju|monthly| +1881|AAAAAAAAJFHAAAAA|2451145|2451174|DEPARTMENT|18|45|Prime, vast voters may occur already. Common games break now blind |monthly| +1882|AAAAAAAAKFHAAAAA|2451145|2451174|DEPARTMENT|18|46|Of course beautiful days could pay alone. Frequent, basic books may not reach surely. Over |monthly| +1883|AAAAAAAALFHAAAAA|2451145|2451174|DEPARTMENT|18|47|Perhaps new thoughts help again. Round orders maint|monthly| +1884|AAAAAAAAMFHAAAAA|2451145|2451174|DEPARTMENT|18|48|Lawyers would worry medical, immediate prices. Months shou|monthly| +1885|AAAAAAAANFHAAAAA|2451145|2451174|DEPARTMENT|18|49|Back residents tell also over a weekends. Real yards try plain, other paintings. Prime, red probl|monthly| +1886|AAAAAAAAOFHAAAAA|2451145|2451174|DEPARTMENT|18|50|Average hours hurry still now senior years. Dead, sw|monthly| +1887|AAAAAAAAPFHAAAAA|2451145|2451174|DEPARTMENT|18|51|Specific buildings put therefore sexual, ordinary companies. Recent, english ins|monthly| +1888|AAAAAAAAAGHAAAAA|2451145|2451174|DEPARTMENT|18|52|Women consider other, difficult clubs. Tomorrow black activities hunt accounts; |monthly| +1889|AAAAAAAABGHAAAAA|2451145|2451174|DEPARTMENT|18|53|Huge women play there national, practical areas. Somewhat professional job|monthly| +1890|AAAAAAAACGHAAAAA|2451145|2451174|DEPARTMENT|18|54|However perfect types could propose by the fortunes. Deep rights change long cer|monthly| +1891|AAAAAAAADGHAAAAA|2451145|2451174|DEPARTMENT|18|55|Capable fingers take also; existing men will not note. Old, amazing books weep for good more ltd. g|monthly| +1892|AAAAAAAAEGHAAAAA|2451145|2451174|DEPARTMENT|18|56|Functions will not exceed friendly gates. Difficult ca|monthly| +1893|AAAAAAAAFGHAAAAA|2451145|2451174|DEPARTMENT|18|57|Just local products used to note well; areas note |monthly| +1894|AAAAAAAAGGHAAAAA|2451145|2451174|DEPARTMENT|18|58|Equations escape indian others. Real, small parts must contact also|monthly| +1895|AAAAAAAAHGHAAAAA|2451145|2451174|DEPARTMENT|18|59|Initiatives draw closely civil rights. Soldiers might instil married, sad months. Late, availab|monthly| +1896|AAAAAAAAIGHAAAAA|2451145|2451174|DEPARTMENT|18|60|Orange needs know months. Always extraordinary workers find howev|monthly| +1897|AAAAAAAAJGHAAAAA|2451145|2451174|DEPARTMENT|18|61|Originally negative women ensure likely, political things. Canadian, deep families used to h|monthly| +1898|AAAAAAAAKGHAAAAA|2451145|2451174|DEPARTMENT|18|62|Absent, high kilometres take as usual friends. Clear signs shall give only twice |monthly| +1899|AAAAAAAALGHAAAAA|2451145|2451174|DEPARTMENT|18|63|Ends give. Citizens go now afraid, professional tears. Light metals must occur. Traditio|monthly| +1900|AAAAAAAAMGHAAAAA|2451145|2451174|DEPARTMENT|18|64|High critics cannot alter. Explicitly serious instruments |monthly| +1901|AAAAAAAANGHAAAAA|2451145|2451174|DEPARTMENT|18|65|Too urgent hands could not begin men; powers could ease es|monthly| +1902|AAAAAAAAOGHAAAAA|2451145|2451174|DEPARTMENT|18|66|Small conditions exclude again reasons. Foreign men occur premier hours|monthly| +1903|AAAAAAAAPGHAAAAA|2451145|2451174|DEPARTMENT|18|67|Quite distinct articles would not say without a forms. Experimental stocks used to play however aus|monthly| +1904|AAAAAAAAAHHAAAAA|2451145|2451174|DEPARTMENT|18|68|High bars give years. Values shall describe efficiently figures. Diff|monthly| +1905|AAAAAAAABHHAAAAA|2451145|2451174|DEPARTMENT|18|69|Political, particular matters take largely religious others. Excellent prices date other indu|monthly| +1906|AAAAAAAACHHAAAAA|2451145|2451174|DEPARTMENT|18|70|Schools become with a councils. Effective figures bind |monthly| +1907|AAAAAAAADHHAAAAA|2451145|2451174|DEPARTMENT|18|71|Good, natural resources speed to a metres. Global, large women ought to live exa|monthly| +1908|AAAAAAAAEHHAAAAA|2451145|2451174|DEPARTMENT|18|72|Commercial, great controls would not operate therefore for a drawings. Less personal years |monthly| +1909|AAAAAAAAFHHAAAAA|2451145|2451174|DEPARTMENT|18|73|Young, adequate hands can follow psychological, early weeks. Steps will allow as appro|monthly| +1910|AAAAAAAAGHHAAAAA|2451145|2451174|DEPARTMENT|18|74|Authorities might rise still large hopes. Borders shall write enough revolutionary units. Appoin|monthly| +1911|AAAAAAAAHHHAAAAA|2451145|2451174|DEPARTMENT|18|75|Different gods will sing. Wide bodies extend clearly. So|monthly| +1912|AAAAAAAAIHHAAAAA|2451145|2451174|DEPARTMENT|18|76|Projects pick sometimes; alive demands will not imagin|monthly| +1913|AAAAAAAAJHHAAAAA|2451145|2451174|DEPARTMENT|18|77|Features must make through the cells. Even foreign departments spend already ideas. O|monthly| +1914|AAAAAAAAKHHAAAAA|2451145|2451174|DEPARTMENT|18|78|So public descriptions decrease little triumphantly little workers. Conservative c|monthly| +1915|AAAAAAAALHHAAAAA|2451145|2451174|DEPARTMENT|18|79|Pp. follow very large, crucial fires. Historic, tremendous police|monthly| +1916|AAAAAAAAMHHAAAAA|2451145|2451174|DEPARTMENT|18|80|Young values make very original, national managers. |monthly| +1917|AAAAAAAANHHAAAAA|2451145|2451174|DEPARTMENT|18|81|Sure, competitive studies buy on a ears; surprising units wi|monthly| +1918|AAAAAAAAOHHAAAAA|2451145|2451174|DEPARTMENT|18|82|Leaders comment strongly local members; real sides |monthly| +1919|AAAAAAAAPHHAAAAA|2451145|2451174|DEPARTMENT|18|83|Eyes promote. Occasionally white words might not give marine acids. Also new feet maintain from a|monthly| +1920|AAAAAAAAAIHAAAAA|2451145|2451174|DEPARTMENT|18|84|New, dark doctors hide home global, very words. Significant, vital restaur|monthly| +1921|AAAAAAAABIHAAAAA|2451145|2451174|DEPARTMENT|18|85|Partly vocational signs show instead. Conservative personnel collapse apparently social fam|monthly| +1922|AAAAAAAACIHAAAAA|2451145|2451174|DEPARTMENT|18|86|Early other opportunities could provide personal res|monthly| +1923|AAAAAAAADIHAAAAA|2451145|2451174|DEPARTMENT|18|87|Essential parties charge chemicals. Main estates run character|monthly| +1924|AAAAAAAAEIHAAAAA|2451145|2451174|DEPARTMENT|18|88|Also emotional ideas tell large, friendly panels. Duties see of course in a preferences. Separate o|monthly| +1925|AAAAAAAAFIHAAAAA|2451145|2451174|DEPARTMENT|18|89|Items look etc on a forces. Parties could begin alm|monthly| +1926|AAAAAAAAGIHAAAAA|2451145|2451174|DEPARTMENT|18|90|Only, welsh years would not go all right. About righ|monthly| +1927|AAAAAAAAHIHAAAAA|2451145|2451174|DEPARTMENT|18|91|Male thoughts should possess personal, successful costs. |monthly| +1928|AAAAAAAAIIHAAAAA|2451145|2451174|DEPARTMENT|18|92|Massive readers will bear with a experts. Talks breach clearly for a police; jud|monthly| +1929|AAAAAAAAJIHAAAAA|2451145|2451174|DEPARTMENT|18|93|Numbers work under. Schools justify. Then british ways will end straight over a|monthly| +1930|AAAAAAAAKIHAAAAA|2451145|2451174|DEPARTMENT|18|94|More pale sequences cannot seem solar groups. Ready ar|monthly| +1931|AAAAAAAALIHAAAAA|2451145|2451174|DEPARTMENT|18|95|Tonnes get somehow sick examples. Both amazing supplies shall not|monthly| +1932|AAAAAAAAMIHAAAAA|2451145|2451174|DEPARTMENT|18|96|Crazy parts may go in a years. Even successful police use enough english schools; ri|monthly| +1933|AAAAAAAANIHAAAAA|2451145|2451174|DEPARTMENT|18|97|Languages must investigate costly costs. Private, other tears|monthly| +1934|AAAAAAAAOIHAAAAA|2451145|2451174|DEPARTMENT|18|98|Others will not accept more adequately flat households. Good sources |monthly| +1935|AAAAAAAAPIHAAAAA|2451145|2451174|DEPARTMENT|18|99|Apparently recent results can respond quietly natural, primary co|monthly| +1936|AAAAAAAAAJHAAAAA|2451145|2451174|DEPARTMENT|18|100|Reservations must not support direct, correct camps; police look enormous conflicts. |monthly| +1937|AAAAAAAABJHAAAAA|2451145|2451174|DEPARTMENT|18|101|Items read to a secrets. Prime, indian aspects believe exceptions; political, ideal margins shou|monthly| +1938|AAAAAAAACJHAAAAA|2451145|2451174|DEPARTMENT|18|102|Liberal numbers tell communist, social numbers. There blue clothes might think. Friends kn|monthly| +1939|AAAAAAAADJHAAAAA|2451145|2451174|DEPARTMENT|18|103|Levels will not see good issues. Employers result houses. Enough future mach|monthly| +1940|AAAAAAAAEJHAAAAA|2451145|2451174|DEPARTMENT|18|104|Violently spare details must not learn further solicitors. Main types may not suffice cl|monthly| +1941|AAAAAAAAFJHAAAAA|2451145|2451174|DEPARTMENT|18|105|Extremely appointed experts read often all right residential programmes. Young|monthly| +1942|AAAAAAAAGJHAAAAA|2451145|2451174|DEPARTMENT|18|106|Neat awards used to change more at a settlements. Able, basic rates display historically.|monthly| +1943|AAAAAAAAHJHAAAAA|2451145|2451174|DEPARTMENT|18|107|Experienced, long metres like as to a designs. As public offices specif|monthly| +1944|AAAAAAAAIJHAAAAA|2451145|2451174|DEPARTMENT|18|108|Representative eyes used to save as exceptions. Minutes could conserve |monthly| +1945|AAAAAAAAJJHAAAAA|2451180|2451361|DEPARTMENT|19|1|Departments order more happy, literary materials. Ite|bi-annual| +1946|AAAAAAAAKJHAAAAA|2451180|2451361|DEPARTMENT|19|2|Indeed private provinces will give strange, excellent rates; annual|bi-annual| +1947|AAAAAAAALJHAAAAA|2451180|2451361|DEPARTMENT|19|3|Long, able files would match well. Totally ready applications make closely; opposite places |bi-annual| +1948|AAAAAAAAMJHAAAAA|2451180|2451361|DEPARTMENT|19|4|Forms kill different styles. Concerns should swear most. Previous names com|bi-annual| +1949|AAAAAAAANJHAAAAA|2451180|2451361|DEPARTMENT|19|5|Feet might forget both from a years. Families might watch just|bi-annual| +1950|AAAAAAAAOJHAAAAA|2451180|2451361|DEPARTMENT|19|6|Lovely issues should say nearly old books. Helpful issues matter once u|bi-annual| +1951|AAAAAAAAPJHAAAAA|2451180|2451361|DEPARTMENT|19|7|Names matter days. Urban awards cannot construct still misleading, responsible ideas. Experiences c|bi-annual| +1952|AAAAAAAAAKHAAAAA|2451180|2451361|DEPARTMENT|19|8|Fans favour more than never old doctors. Sexual, whole agreements use hours. Fully front ch|bi-annual| +1953|AAAAAAAABKHAAAAA|2451180|2451361|DEPARTMENT|19|9|Highest entire cells get clear, unlike years. So regional motives describe. Fair different int|bi-annual| +1954|AAAAAAAACKHAAAAA|2451180|2451361|DEPARTMENT|19|10|Groups chat in particular as of a days. Polite grounds cut birds; weak, huge falls t|bi-annual| +1955|AAAAAAAADKHAAAAA|2451180|2451361|DEPARTMENT|19|11|Possible, other molecules eat collections. More double agencies shall take recently units. Late|bi-annual| +1956|AAAAAAAAEKHAAAAA|2451180|2451361|DEPARTMENT|19|12|Cities must not discover wrong members. Willing, fi|bi-annual| +1957|AAAAAAAAFKHAAAAA|2451180|2451361|DEPARTMENT|19|13|Economic angles can view right other, very police. Heavy, low issues last gm|bi-annual| +1958|AAAAAAAAGKHAAAAA|2451180|2451361|DEPARTMENT|19|14|Illegal leaves might return casually. Severe, possible mi|bi-annual| +1959|AAAAAAAAHKHAAAAA|2451180|2451361|DEPARTMENT|19|15|Straight inland notes go ahead in a books. Findings make. Men could te|bi-annual| +1960|AAAAAAAAIKHAAAAA|2451180|2451361|DEPARTMENT|19|16|Wrong, formal numbers look increasingly workers. Different, different women talk ago in the arms|bi-annual| +1961|AAAAAAAAJKHAAAAA|2451180|2451361|DEPARTMENT|19|17|Flat, other cases could ask different, crucial drivers. Very specific efforts should not sit. |bi-annual| +1962|AAAAAAAAKKHAAAAA|2451180|2451361|DEPARTMENT|19|18|Leading defences could not bring actually small reasons. Centres could not discover wide. Plain pra|bi-annual| +1963|AAAAAAAALKHAAAAA|2451180|2451361|DEPARTMENT|19|19|Independent, potential men can safeguard probably previous years; economic, fre|bi-annual| +1964|AAAAAAAAMKHAAAAA|2451180|2451361|DEPARTMENT|19|20|Dogs used to restore great firms. As local reservations rule considerably s|bi-annual| +1965|AAAAAAAANKHAAAAA|2451180|2451361|DEPARTMENT|19|21|Steady, chief stands spare in a guns. Prime teachers ought to mind away regular con|bi-annual| +1966|AAAAAAAAOKHAAAAA|2451180|2451361|DEPARTMENT|19|22|Now good numbers would provide in particular recently total women. Expli|bi-annual| +1967|AAAAAAAAPKHAAAAA|2451180|2451361|DEPARTMENT|19|23|Positive, good numbers cannot beware then runs. Only mild students tell there simple, intimate |bi-annual| +1968|AAAAAAAAALHAAAAA|2451180|2451361|DEPARTMENT|19|24|Important subjects can play so characteristics. Important, key bene|bi-annual| +1969|AAAAAAAABLHAAAAA|2451180|2451361|DEPARTMENT|19|25|Intellectual, british details know including the times|bi-annual| +1970|AAAAAAAACLHAAAAA|2451180|2451361|DEPARTMENT|19|26|Simple, necessary emotions might arise. Sheer days shall identify much facto|bi-annual| +1971|AAAAAAAADLHAAAAA|2451180|2451361|DEPARTMENT|19|27|Kinds give really general, clear procedures; now real refuge|bi-annual| +1972|AAAAAAAAELHAAAAA|2451180|2451361|DEPARTMENT|19|28|Increasing authorities can remove important, growing images. Often afraid standards enter bad, |bi-annual| +1973|AAAAAAAAFLHAAAAA|2451180|2451361|DEPARTMENT|19|29|Privately old individuals would not recognise horrible thoughts. Rather addition|bi-annual| +1974|AAAAAAAAGLHAAAAA|2451180|2451361|DEPARTMENT|19|30|Delegates shall give honours. Afraid, new names should hold furiously in the quantities. Responsibl|bi-annual| +1975|AAAAAAAAHLHAAAAA|2451180|2451361|DEPARTMENT|19|31|Final, recent pieces break deeper young, active in|bi-annual| +1976|AAAAAAAAILHAAAAA|2451180||DEPARTMENT|19|32|Useful hands go internal, light sides. Grounds pay gently liberal paintings. Foreign fe|bi-annual| +1977|AAAAAAAAJLHAAAAA|2451180|2451361|DEPARTMENT|19|33|Sciences might capture by a resources; investments would not reduce at le|bi-annual| +1978|AAAAAAAAKLHAAAAA|2451180|2451361|DEPARTMENT|19|34|Possible periods would address less far things. Far, conservative officials will not talk rather |bi-annual| +1979|AAAAAAAALLHAAAAA|2451180|2451361|DEPARTMENT|19|35|Right circumstances take a little religious legs. N|bi-annual| +1980|AAAAAAAAMLHAAAAA|2451180|2451361|DEPARTMENT|19|36|Schools would get general, unhappy questions. Entirely little acts say so structural|bi-annual| +1981|AAAAAAAANLHAAAAA||||19|||bi-annual| +1982|AAAAAAAAOLHAAAAA|2451180|2451361|DEPARTMENT|19|38|Satisfactory needs shall supply for example. Other |bi-annual| +1983|AAAAAAAAPLHAAAAA|2451180|2451361|DEPARTMENT|19|39|Christian paintings build; at least final women would drive ever excellent, |bi-annual| +1984|AAAAAAAAAMHAAAAA|2451180|2451361|DEPARTMENT|19|40|Internal visitors ought to form proposed acts. Fresh employees find well key, medium conclusio|bi-annual| +1985|AAAAAAAABMHAAAAA|2451180|2451361|DEPARTMENT|19|41|Even educational branches could consider at the democrats. Simultaneously happy gr|bi-annual| +1986|AAAAAAAACMHAAAAA|2451180|2451361|DEPARTMENT|19|42|Advantages shall not release inside increased honours. Important,|bi-annual| +1987|AAAAAAAADMHAAAAA|2451180|2451361|DEPARTMENT|19|43|Also civil reductions will require much long, collective departments. Homes must n|bi-annual| +1988|AAAAAAAAEMHAAAAA|2451180|2451361|DEPARTMENT|19|44|Pregnant, clear employees may mind again pairs. Prices may not monitor h|bi-annual| +1989|AAAAAAAAFMHAAAAA|2451180|2451361|DEPARTMENT|19|45|Organs set no longer towards a modules; cars may not exclude that wi|bi-annual| +1990|AAAAAAAAGMHAAAAA|2451180|2451361|DEPARTMENT|19|46|More important things go anyway existing, silent services. Concerned, direct lips|bi-annual| +1991|AAAAAAAAHMHAAAAA|2451180|2451361|DEPARTMENT|19|47|Annually existing ways secure with a thoughts. Electric legs see for|bi-annual| +1992|AAAAAAAAIMHAAAAA|2451180|2451361|DEPARTMENT|19|48|Penalties shift from a systems; now only relations increase al|bi-annual| +1993|AAAAAAAAJMHAAAAA|2451180|2451361|DEPARTMENT|19|49|Husbands will not find. Newly full championships sh|bi-annual| +1994|AAAAAAAAKMHAAAAA|2451180|2451361|DEPARTMENT|19|50|Glorious, real pieces should not prove more religious, rural weeks; conservative months |bi-annual| +1995|AAAAAAAALMHAAAAA|2451180|2451361|DEPARTMENT|19|51|Black men start profits. Real, steady remains must deci|bi-annual| +1996|AAAAAAAAMMHAAAAA|2451180|2451361|DEPARTMENT|19|52|Big, national fingers should not enable both single, protective forces. Courses look hard|bi-annual| +1997|AAAAAAAANMHAAAAA|2451180|2451361|DEPARTMENT|19|53|Priorities ought to give then still evil agreements. Actually broad women might know from time t|bi-annual| +1998|AAAAAAAAOMHAAAAA|2451180|2451361|DEPARTMENT|19|54|Linear, capable others state environmental songs. Changes deny large, possibl|bi-annual| +1999|AAAAAAAAPMHAAAAA|2451180|2451361|DEPARTMENT|19|55|Gay circumstances produce. Courts show however internal protests. Technically big rivers say |bi-annual| +2000|AAAAAAAAANHAAAAA|2451180|2451361|DEPARTMENT|19|56|Agricultural hands move possible, delicate friends. Mostly sure times could make of|bi-annual| +2001|AAAAAAAABNHAAAAA|2451180|2451361|DEPARTMENT|19|57|Famous, quick critics return. Cheerful, medical laws under|bi-annual| +2002|AAAAAAAACNHAAAAA|2451180|2451361|DEPARTMENT|19|58|Favorite visitors keep only wrong hours; as progressive areas must not look just m|bi-annual| +2003|AAAAAAAADNHAAAAA|2451180|2451361|DEPARTMENT|19|59|Therefore military processes accept by now ideal advances|bi-annual| +2004|AAAAAAAAENHAAAAA|2451180|2451361|DEPARTMENT|19|60|Limited, english parties discuss that; good, confident technique|bi-annual| +2005|AAAAAAAAFNHAAAAA|2451180|2451361|DEPARTMENT|19|61|Following, generous methods put offices; dominant miles finish|bi-annual| +2006|AAAAAAAAGNHAAAAA|2451180|2451361|DEPARTMENT|19|62|Bright men boil over worth the authorities. Thereby plain stations might demonstrate sheets. |bi-annual| +2007|AAAAAAAAHNHAAAAA|2451180|2451361|DEPARTMENT|19|63|Effects demonstrate sometimes of course terrible applications. Here red obligations go le|bi-annual| +2008|AAAAAAAAINHAAAAA|2451180|2451361|DEPARTMENT|19|64|Available volunteers seem more effects. Economically lar|bi-annual| +2009|AAAAAAAAJNHAAAAA|2451180|2451361|DEPARTMENT|19|65|Different tears may save then by a alternatives. Western, known legs will speak best el|bi-annual| +2010|AAAAAAAAKNHAAAAA|2451180|2451361|DEPARTMENT|19|66|Issues will not pray real attempts. Bars must not work there various areas. Clean, responsible we|bi-annual| +2011|AAAAAAAALNHAAAAA|2451180|2451361|DEPARTMENT|19|67|Industrial managers settle internally preparations; beautiful areas gain local chapters. Extremel|bi-annual| +2012|AAAAAAAAMNHAAAAA|2451180|2451361|DEPARTMENT|19|68|Intentionally different teams run there; even final hours should not reach in a sharehold|bi-annual| +2013|AAAAAAAANNHAAAAA|2451180|2451361|DEPARTMENT|19|69|Too able theories ignore certainly black, turkish teeth. Young, positive faces fi|bi-annual| +2014|AAAAAAAAONHAAAAA|2451180|2451361|DEPARTMENT|19|70|Details should pass welsh sections. Even dull proceedings ought to dec|bi-annual| +2015|AAAAAAAAPNHAAAAA|2451180|2451361|DEPARTMENT|19|71|Somehow limited readers rear then cold, local stairs. Systems giv|bi-annual| +2016|AAAAAAAAAOHAAAAA|2451180|2451361|DEPARTMENT|19|72|Examples may stop. Equal, serious waters show. Powerful, international settlements win days; things|bi-annual| +2017|AAAAAAAABOHAAAAA|2451180|2451361|DEPARTMENT|19|73|Major terms can know only, delicious women. Hidden, detailed men enhance long minor, dark seconds|bi-annual| +2018|AAAAAAAACOHAAAAA|2451180|2451361|DEPARTMENT|19|74|Relevant colours return however times. Reliable, basic groups could concern not for the hopes.|bi-annual| +2019|AAAAAAAADOHAAAAA|2451180|2451361|DEPARTMENT|19|75|Presidential eyes would make matters. Recent universities minimise. Plain able f|bi-annual| +2020|AAAAAAAAEOHAAAAA|2451180|2451361|DEPARTMENT|19|76|Flowers must invite sharply positive governments. Children meet even just private rats. Indeed|bi-annual| +2021|AAAAAAAAFOHAAAAA|2451180|2451361|DEPARTMENT|19|77|New resources might go from a proposals. Urgently |bi-annual| +2022|AAAAAAAAGOHAAAAA|2451180|2451361|DEPARTMENT|19|78|Villages get as; just good directions should carry i|bi-annual| +2023|AAAAAAAAHOHAAAAA|2451180|2451361|DEPARTMENT|19|79|Experiments will report plain with a areas. Neighbours extend only through the consequences|bi-annual| +2024|AAAAAAAAIOHAAAAA|2451180|2451361|DEPARTMENT|19|80|Functions promote black, industrial women. Good, f|bi-annual| +2025|AAAAAAAAJOHAAAAA|2451180|2451361|DEPARTMENT|19|81|Heavy, whole tests should not avoid personal, personal|bi-annual| +2026|AAAAAAAAKOHAAAAA|2451180|2451361|DEPARTMENT|19|82|Professional depths control technical products. Subsequently vocatio|bi-annual| +2027|AAAAAAAALOHAAAAA|2451180|2451361|DEPARTMENT|19|83|Young systems may not increase always honestly full windows. Young items may exist ago wrong tasks.|bi-annual| +2028|AAAAAAAAMOHAAAAA|2451180|2451361|DEPARTMENT|19|84|Recent, formal windows produce respectively to a developments. Future, imme|bi-annual| +2029|AAAAAAAANOHAAAAA|2451180|2451361|DEPARTMENT|19|85|Expensive, past measures can lock new, big options. Well extreme contracts put |bi-annual| +2030|AAAAAAAAOOHAAAAA|2451180|2451361|DEPARTMENT|19|86|Public opinions get especially. Resources catch about a ways. British penalties may not make|bi-annual| +2031|AAAAAAAAPOHAAAAA|2451180|2451361|DEPARTMENT|19|87|Nevertheless historical things boost much clearly new poems. Dirty subjects should|bi-annual| +2032|AAAAAAAAAPHAAAAA|2451180|2451361|DEPARTMENT|19|88|Small christians could sell sometimes aware items. Really new flights shall no|bi-annual| +2033|AAAAAAAABPHAAAAA|2451180|2451361|DEPARTMENT|19|89|Authorities complicate often soft, red men. Alone workers|bi-annual| +2034|AAAAAAAACPHAAAAA|2451180|2451361|DEPARTMENT|19|90|More little experts focus so. Again criminal acts shall seem so |bi-annual| +2035|AAAAAAAADPHAAAAA|2451180|2451361|DEPARTMENT|19|91|Guns used to meet there even internal numbers; good sciences d|bi-annual| +2036|AAAAAAAAEPHAAAAA|2451180|2451361|DEPARTMENT|19|92|Independent boys design good effects. Months know as dead colonies. Easy direct emissions ought to|bi-annual| +2037|AAAAAAAAFPHAAAAA|2451180|2451361|DEPARTMENT|19|93|Mad, growing schools hope little positive, bad provisi|bi-annual| +2038|AAAAAAAAGPHAAAAA|2451180|2451361|DEPARTMENT|19|94|Now black injuries create only identical patients. Bitter, physical places offer against the ga|bi-annual| +2039|AAAAAAAAHPHAAAAA|2451180|2451361|DEPARTMENT|19|95|Effective parts shall interpret particularly over a qualifications. Responses|bi-annual| +2040|AAAAAAAAIPHAAAAA|2451180|2451361|DEPARTMENT|19|96|Far forces must continue also concerned, other answers. Seconds play professional, sexual |bi-annual| +2041|AAAAAAAAJPHAAAAA|2451180|2451361|DEPARTMENT|19|97|Straight lists may displace. Great days might not stop usually in|bi-annual| +2042|AAAAAAAAKPHAAAAA|2451180|2451361|DEPARTMENT|19|98|Arms get etc national elections. Strings know again indeed provincial ac|bi-annual| +2043|AAAAAAAALPHAAAAA|2451180|2451361|DEPARTMENT|19|99|Issues may say just reasons. Clearly new lips specify yesterday exceptional, ner|bi-annual| +2044|AAAAAAAAMPHAAAAA|2451180|2451361|DEPARTMENT|19|100|Clubs know. Other areas shall market thus to a relations. Brilliant cases must test si|bi-annual| +2045|AAAAAAAANPHAAAAA|2451180|2451361|DEPARTMENT|19|101|Programmes ought to engage so national lists. Cars would not |bi-annual| +2046|AAAAAAAAOPHAAAAA|2451180|2451361|DEPARTMENT|19|102|Chosen, large contents live respectively legitimate journals. Economic, true goods try issues. Ju|bi-annual| +2047|AAAAAAAAPPHAAAAA|2451180|2451361|DEPARTMENT|19|103|Round parts develop still political, special officers. Ne|bi-annual| +2048|AAAAAAAAAAIAAAAA|2451180|2451361|DEPARTMENT|19|104|Just industrial documents would not say; cuts keep sounds. Short schools|bi-annual| +2049|AAAAAAAABAIAAAAA|2451180|2451361|DEPARTMENT|19|105|Nuclear, common years meet thus at a soldiers. Pub|bi-annual| +2050|AAAAAAAACAIAAAAA|2451180|2451361|DEPARTMENT|19|106|Only hours mean at a designers. Distinct gardens ought to appear so reports. Re|bi-annual| +2051|AAAAAAAADAIAAAAA|2451180|2451361|DEPARTMENT|19|107|So as local offices affect together legal mechanisms; shoulders must reflect angri|bi-annual| +2052|AAAAAAAAEAIAAAAA|2451180|2451361|DEPARTMENT|19|108|Principally urban individuals meet soon great players. Electoral, |bi-annual| +2053|AAAAAAAAFAIAAAAA|2451362|2451543|DEPARTMENT|20|1|Large values support just urban clubs. Short assumptions reflect quickly fresh, other f|bi-annual| +2054|AAAAAAAAGAIAAAAA|2451362|2451543|DEPARTMENT|20|2|Wages leave now for the practitioners. New families bend armies; professionals may not supply |bi-annual| +2055|AAAAAAAAHAIAAAAA|2451362|2451543|DEPARTMENT|20|3|Factors might work then sudden tests; again political dreams join all ways. T|bi-annual| +2056|AAAAAAAAIAIAAAAA|2451362|2451543|DEPARTMENT|20|4|Powers will owe french, military sales. Respective branches help i|bi-annual| +2057|AAAAAAAAJAIAAAAA|2451362|2451543|DEPARTMENT|20|5|Labour sons shall focus kids. Classes should take in a areas. Points follow closely large, rea|bi-annual| +2058|AAAAAAAAKAIAAAAA|2451362|2451543|DEPARTMENT|20|6|National users will explore local occasions; operations used to join less|bi-annual| +2059|AAAAAAAALAIAAAAA|2451362|2451543|DEPARTMENT|20|7|Workers cannot ask. Annual documents attend past on a services. |bi-annual| +2060|AAAAAAAAMAIAAAAA|2451362|2451543|DEPARTMENT|20|8|Problems credit enough fine relatives. Afraid, different practices think h|bi-annual| +2061|AAAAAAAANAIAAAAA|2451362|2451543|DEPARTMENT|20|9|Things must pick genuinely previously intimate miles. Pieces will not conside|bi-annual| +2062|AAAAAAAAOAIAAAAA|2451362|2451543|DEPARTMENT|20|10|Sure goals used to get valuable directions. White me|bi-annual| +2063|AAAAAAAAPAIAAAAA|2451362|2451543|DEPARTMENT|20|11|Well british readers look good, old colours. Accidents give early. Of course live houses go very |bi-annual| +2064|AAAAAAAAABIAAAAA|2451362|2451543|DEPARTMENT|20|12|Social, substantial officers should not make etc in a criteria. Politi|bi-annual| +2065|AAAAAAAABBIAAAAA|2451362|2451543|DEPARTMENT|20|13|Other, relevant twins need promptly. Very new police should not place fairly predominantly |bi-annual| +2066|AAAAAAAACBIAAAAA|2451362|2451543|DEPARTMENT|20|14|Full systems will obtain by far essential ways. Different adults must not get now total, wrong for|bi-annual| +2067|AAAAAAAADBIAAAAA|2451362|2451543|DEPARTMENT|20|15|Old, pleased authorities move more with a visitors. Good, ol|bi-annual| +2068|AAAAAAAAEBIAAAAA|2451362|2451543|DEPARTMENT|20|16|Like special attacks react possibly problems. Existing|bi-annual| +2069|AAAAAAAAFBIAAAAA|2451362|2451543|DEPARTMENT|20|17|Realistic, russian practitioners describe furthermore so aggressive decisions. Names can see degree|bi-annual| +2070|AAAAAAAAGBIAAAAA|2451362|2451543|DEPARTMENT|20|18|Risks must not come. Streets should not watch also |bi-annual| +2071|AAAAAAAAHBIAAAAA|2451362|2451543|DEPARTMENT|20|19|Often narrow states laugh perhaps individuals; black, guilty questions say to a|bi-annual| +2072|AAAAAAAAIBIAAAAA|2451362|2451543|DEPARTMENT|20|20|Strong minutes take still. Particularly usual earnings would develop now duties. Only poor experime|bi-annual| +2073|AAAAAAAAJBIAAAAA|2451362|2451543|DEPARTMENT|20|21|Painful, great countries prefer past, early bedrooms. Wives overcome examples. O|bi-annual| +2074|AAAAAAAAKBIAAAAA|2451362|2451543|DEPARTMENT|20|22|Lists find by the levels. Present orders shall inflict mo|bi-annual| +2075|AAAAAAAALBIAAAAA|2451362|2451543|DEPARTMENT|20|23|Entirely original damages refrain angry, real proceedings; sorry p|bi-annual| +2076|AAAAAAAAMBIAAAAA|2451362|2451543|DEPARTMENT|20|24|Neither great taxes keep really. Fast grand lessons shall make about in a days. Genuine, actual for|bi-annual| +2077|AAAAAAAANBIAAAAA|2451362|2451543|DEPARTMENT|20|25|Developing issues shall not work a little with the schemes. Papers used to need tho|bi-annual| +2078|AAAAAAAAOBIAAAAA|2451362||DEPARTMENT|||Most holy workers believe partly new needs. Easy local books see more shareholders. Commercial,|| +2079|AAAAAAAAPBIAAAAA|2451362|2451543|DEPARTMENT|20|27|Children can combine however serious operations; related assumptions complete struct|bi-annual| +2080|AAAAAAAAACIAAAAA|2451362|2451543|DEPARTMENT|20|28|As fine jobs may make now old, british blocks. Dept|bi-annual| +2081|AAAAAAAABCIAAAAA|2451362|2451543|DEPARTMENT|20|29|Early harsh changes look strict files. Preparations get disc|bi-annual| +2082|AAAAAAAACCIAAAAA|2451362|2451543|DEPARTMENT|20|30|Good, educational police materialise on a views. Finally simple doctors exist |bi-annual| +2083|AAAAAAAADCIAAAAA|2451362|2451543|DEPARTMENT|20|31|Active programs might not discover regular, national |bi-annual| +2084|AAAAAAAAECIAAAAA|2451362|2451543|DEPARTMENT|20|32|Old, national kids should not alter surely fair needs. Full materials might |bi-annual| +2085|AAAAAAAAFCIAAAAA|2451362|2451543|DEPARTMENT|20|33|Classic, running cities used to alter characters. Blue questions will make together w|bi-annual| +2086|AAAAAAAAGCIAAAAA|2451362|2451543|DEPARTMENT|20|34|Quiet, urban posts shall not bring together largely scotti|bi-annual| +2087|AAAAAAAAHCIAAAAA|2451362|2451543|DEPARTMENT|20|35|Boots must not feel finally with a uses. Main, judicial bonds st|bi-annual| +2088|AAAAAAAAICIAAAAA|2451362|2451543|DEPARTMENT|20|36|Readers select over. Protective, different resources ought to knock more local policies. Rates g|bi-annual| +2089|AAAAAAAAJCIAAAAA|2451362|2451543|DEPARTMENT|20|37|Yesterday personal cars determine firstly badly religious employers. Statistical nations sa|bi-annual| +2090|AAAAAAAAKCIAAAAA|2451362|2451543|DEPARTMENT|20|38|Full firms shall spend unfortunately live types. Royal, compatible de|bi-annual| +2091|AAAAAAAALCIAAAAA|2451362|2451543|DEPARTMENT|20|39|Rights can see patients. Industrial months tell so wide paren|bi-annual| +2092|AAAAAAAAMCIAAAAA|2451362|2451543|DEPARTMENT|20|40|Servants rest then in a decisions. Departments should happ|bi-annual| +2093|AAAAAAAANCIAAAAA|2451362|2451543|DEPARTMENT|20|41|Differences explain in an hours. Efforts would help all in general s|bi-annual| +2094|AAAAAAAAOCIAAAAA|2451362|||20|||bi-annual| +2095|AAAAAAAAPCIAAAAA|2451362|2451543|DEPARTMENT|20|43|Overall, aware nations continue yet historians. Possible, |bi-annual| +2096|AAAAAAAAADIAAAAA|2451362|2451543|DEPARTMENT|20|44|Factors stay much farmers. Products say in a losses. Dead periods should reduce too for th|bi-annual| +2097|AAAAAAAABDIAAAAA|2451362|2451543|DEPARTMENT|20|45|Always numerous funds act roughly shares. Once fundamental eyes can mean fu|bi-annual| +2098|AAAAAAAACDIAAAAA|2451362|2451543|DEPARTMENT|20|46|Excellent objects would not give maybe about leading streets|bi-annual| +2099|AAAAAAAADDIAAAAA|2451362|2451543|DEPARTMENT|20|47|Quiet buildings shall carry reasons; regional children used to own however obvious, indi|bi-annual| +2100|AAAAAAAAEDIAAAAA|2451362|2451543|DEPARTMENT|20|48|Guns cannot renew just nowhere national leaves. Long, orange readers rega|bi-annual| +2101|AAAAAAAAFDIAAAAA|2451362|2451543|DEPARTMENT|20|49|Difficult servants could take barely. Certainly scientific pieces build positive, local f|bi-annual| +2102|AAAAAAAAGDIAAAAA|2451362|2451543|DEPARTMENT|20|50|More powerful governments shall adapt. Shops believe. Studies claim great true values; old, ser|bi-annual| +2103|AAAAAAAAHDIAAAAA|2451362|2451543|DEPARTMENT|20|51|Different, romantic systems look very skills. Articles beg peculiarly new actions. Refo|bi-annual| +2104|AAAAAAAAIDIAAAAA|2451362|2451543|DEPARTMENT|20|52|Moving, important methods give certainly naval, full hills. Violent, different pupils |bi-annual| +2105|AAAAAAAAJDIAAAAA|2451362|2451543|DEPARTMENT|20|53|So literary terms would ensure possibly new cuts. Relevant requirements may see already res|bi-annual| +2106|AAAAAAAAKDIAAAAA|2451362|2451543|DEPARTMENT|20|54|Shallow, bad years compete healthy modules. Sides stay over |bi-annual| +2107|AAAAAAAALDIAAAAA|2451362|2451543|DEPARTMENT|20|55|Potentially modern practices answer teachers. Payments should think new, social depa|bi-annual| +2108|AAAAAAAAMDIAAAAA|2451362|2451543|DEPARTMENT|20|56|Like, constitutional lectures laugh enough quite likely ways. Local meanings drive sensitive, b|bi-annual| +2109|AAAAAAAANDIAAAAA|2451362|2451543|DEPARTMENT|20|57|Bags mean variously questions. Police continue much. Rates would not |bi-annual| +2110|AAAAAAAAODIAAAAA|2451362|2451543|DEPARTMENT|20|58|Mixed, other plans manage usually after the ways; obvious groups thank therefore dangerous|bi-annual| +2111|AAAAAAAAPDIAAAAA|2451362|2451543|DEPARTMENT|20|59|Fatal, raw years would know. Good operations see more appropriate, enormous operations. Hu|bi-annual| +2112|AAAAAAAAAEIAAAAA|2451362|2451543|DEPARTMENT|20|60|Very national hands help even. High, large materials play surely in a feelings. Quick|bi-annual| +2113|AAAAAAAABEIAAAAA|2451362|2451543|DEPARTMENT|20|61|British agencies grieve all right independent, old c|bi-annual| +2114|AAAAAAAACEIAAAAA|2451362|2451543|DEPARTMENT|20|62|Children will talk professionally human points. Good, inappropriate legs s|bi-annual| +2115|AAAAAAAADEIAAAAA|2451362|2451543|DEPARTMENT|20|63|Grey, contemporary types ought to lie communist, fit adva|bi-annual| +2116|AAAAAAAAEEIAAAAA|2451362|2451543|DEPARTMENT|20|64|Rapid, german children put rapidly. Alternative countries apply values. Remarkable communi|bi-annual| +2117|AAAAAAAAFEIAAAAA|2451362|2451543|DEPARTMENT|20|65|Even whole products cope perhaps absolute, free patients. Ordinarily lin|bi-annual| +2118|AAAAAAAAGEIAAAAA|2451362|2451543|DEPARTMENT|20|66|Debts help. Thus old videos should hear modules. Odd, sharp hills contact tra|bi-annual| +2119|AAAAAAAAHEIAAAAA|2451362|2451543|DEPARTMENT|20|67|Governments establish. Of course global statements enforce now. Leaders shall |bi-annual| +2120|AAAAAAAAIEIAAAAA|2451362|2451543|DEPARTMENT|20|68|Especially free families stay thus political employers. Events may give so o|bi-annual| +2121|AAAAAAAAJEIAAAAA|2451362|2451543|DEPARTMENT|20|69|More big women might not wish patients. Skills will not help; completely local |bi-annual| +2122|AAAAAAAAKEIAAAAA|2451362|2451543|DEPARTMENT|20|70|Words must not hurt early average examples; particularly eventual re|bi-annual| +2123|AAAAAAAALEIAAAAA|2451362|2451543|DEPARTMENT|20|71|Again main patients think sides; feet will pay more matters. Opposite, old di|bi-annual| +2124|AAAAAAAAMEIAAAAA|2451362|2451543|DEPARTMENT|20|72|Now economic problems become just drugs. Women may keep nearly numerous, clean items. Strong,|bi-annual| +2125|AAAAAAAANEIAAAAA|2451362|2451543|DEPARTMENT|20|73|Kids must go sounds; windows look firmly english cases. Firms trade enough casual reasons; moder|bi-annual| +2126|AAAAAAAAOEIAAAAA|2451362|2451543|DEPARTMENT|20|74|Social businessmen bring previously areas. Right copies go then by a crit|bi-annual| +2127|AAAAAAAAPEIAAAAA|2451362|2451543|DEPARTMENT|20|75|Public cities would not attract pretty. Lips come developing points. Special, sh|bi-annual| +2128|AAAAAAAAAFIAAAAA|2451362|2451543|DEPARTMENT|20|76|Representatives used to co-operate already streets. Bands last for|bi-annual| +2129|AAAAAAAABFIAAAAA|2451362|2451543|DEPARTMENT|20|77|Now aware factors let just full buildings. Sections cannot suit still init|bi-annual| +2130|AAAAAAAACFIAAAAA|2451362|2451543|DEPARTMENT|20|78|Positive, similar affairs grasp gracefully remarkable spo|bi-annual| +2131|AAAAAAAADFIAAAAA|2451362|2451543|DEPARTMENT|20|79|Correct days flourish also. Problems might make. Bold rates kill o|bi-annual| +2132|AAAAAAAAEFIAAAAA|2451362|2451543|DEPARTMENT|20|80|Main, left systems close magnetic, rational banks. Ideas may say thus only busy years. Rig|bi-annual| +2133|AAAAAAAAFFIAAAAA|2451362|2451543|DEPARTMENT|20|81|Schools used to put. Initially soft women may give implications. Police explain for the events.|bi-annual| +2134|AAAAAAAAGFIAAAAA|2451362|2451543|DEPARTMENT|20|82|Issues may cover very serious patients. Now european techniques include even.|bi-annual| +2135|AAAAAAAAHFIAAAAA|2451362|2451543|DEPARTMENT|20|83|Figures used to walk much nurses; british hours show initially|bi-annual| +2136|AAAAAAAAIFIAAAAA|2451362|2451543|DEPARTMENT|20|84|Payments could not work as musical activities. Other tenants esta|bi-annual| +2137|AAAAAAAAJFIAAAAA|2451362|2451543|DEPARTMENT|20|85|Over creative children would know too assessments. Mental, other demands expect however. Eno|bi-annual| +2138|AAAAAAAAKFIAAAAA|2451362|2451543|DEPARTMENT|20|86|Severe others eat largely about high ways; unusual boys shall not sound still completely averag|bi-annual| +2139|AAAAAAAALFIAAAAA|2451362|2451543|DEPARTMENT|20|87|Useful, conventional laws shall not write commonly at the years; miner|bi-annual| +2140|AAAAAAAAMFIAAAAA|2451362|2451543|DEPARTMENT|20|88|More complete officials shall join necessary, ready posts. Awards could report hu|bi-annual| +2141|AAAAAAAANFIAAAAA|2451362|2451543|DEPARTMENT|20|89|Russian troops establish nearly to a terms. Therefore complex mach|bi-annual| +2142|AAAAAAAAOFIAAAAA|2451362|2451543|DEPARTMENT|20|90|Important arms may not like in order various, democratic workers; already personal us|bi-annual| +2143|AAAAAAAAPFIAAAAA|2451362|2451543|DEPARTMENT|20|91|Successful, practical reasons kill future islands. Sole shows cannot buy always yel|bi-annual| +2144|AAAAAAAAAGIAAAAA|2451362|2451543|DEPARTMENT|20|92|Favorite months might say more than long, real skills. Tired|bi-annual| +2145|AAAAAAAABGIAAAAA|2451362|2451543|DEPARTMENT|20|93|Indicators run necessary officers. Full flowers may aim tool|bi-annual| +2146|AAAAAAAACGIAAAAA|2451362|2451543|DEPARTMENT|20|94|Men may read thus unfortunately possible cards; increasi|bi-annual| +2147|AAAAAAAADGIAAAAA|2451362|2451543|DEPARTMENT|20|95|Heavy definitions might launch pleasant issues. Rare p|bi-annual| +2148|AAAAAAAAEGIAAAAA|2451362|2451543|DEPARTMENT|20|96|Of course narrow schools go central, social others. Stiffly new columns shall not rest tru|bi-annual| +2149|AAAAAAAAFGIAAAAA|2451362|2451543|DEPARTMENT|20|97|Daily trials need in a roots. Frequent, private passengers act; scho|bi-annual| +2150|AAAAAAAAGGIAAAAA|2451362|2451543|DEPARTMENT|20|98|Large, hot councils show already electronic, creative elections. Patterns mu|bi-annual| +2151|AAAAAAAAHGIAAAAA|2451362|2451543|DEPARTMENT|20|99|Distant, able years shall stand little. Practices make to a years. Best good areas could give |bi-annual| +2152|AAAAAAAAIGIAAAAA|2451362|2451543|DEPARTMENT|20|100|Chief, general delegates know instantly now poor res|bi-annual| +2153|AAAAAAAAJGIAAAAA|2451362|2451543|DEPARTMENT|20|101|Leaders shall press early books. Electronic, mental lands |bi-annual| +2154|AAAAAAAAKGIAAAAA|2451362|2451543|DEPARTMENT|20|102|Straight, known phenomena will not get ago big times. Other |bi-annual| +2155|AAAAAAAALGIAAAAA|2451362|2451543|DEPARTMENT|20|103|Times want later chief terms. Sources will employ therefor|bi-annual| +2156|AAAAAAAAMGIAAAAA|2451362|2451543|DEPARTMENT|20|104|Too full managers could not see more profits. Features help then so useful candidates. Form|bi-annual| +2157|AAAAAAAANGIAAAAA|2451362|2451543|DEPARTMENT|20|105|Respective kids would not like very important friends. Possible benefits remember organisations. |bi-annual| +2158|AAAAAAAAOGIAAAAA|2451362|2451543|DEPARTMENT|20|106|Of course other windows could remain here by a personnel. New needs must not |bi-annual| +2159|AAAAAAAAPGIAAAAA|2451362|2451543|DEPARTMENT|20|107|Projects scrape no doubt american relations. Simply obvious toys let with a bloc|bi-annual| +2160|AAAAAAAAAHIAAAAA|2451362|2451543|DEPARTMENT|20|108|Current members understand warily; beings ask maybe functi|bi-annual| +2161|AAAAAAAABHIAAAAA|2451180|2451270|DEPARTMENT|21|1|Medieval, modern units should not reduce now. Effectively private trous|quarterly| +2162|AAAAAAAACHIAAAAA|2451180|2451270|DEPARTMENT|21|2|Nice things crumble occasionally coastal considerations. Public, early facilities |quarterly| +2163|AAAAAAAADHIAAAAA|2451180|2451270|DEPARTMENT|21|3|Authorities fail with the trousers. Other, free eyes should find over soldiers. Close prob|quarterly| +2164|AAAAAAAAEHIAAAAA|2451180|2451270|DEPARTMENT|21|4|Villages select global, welsh feet. Earlier wooden shops take asian costs; authorities could lose |quarterly| +2165|AAAAAAAAFHIAAAAA|2451180|2451270|DEPARTMENT|21|5|White standards stay forward. Wrong years attack from the memories. Now liberal elements |quarterly| +2166|AAAAAAAAGHIAAAAA|2451180|2451270|DEPARTMENT|21|6|Political, hard classes may highlight. Successful things pull often as |quarterly| +2167|AAAAAAAAHHIAAAAA|||DEPARTMENT||7|True mothers shall not give. Only, hard guests feel |quarterly| +2168|AAAAAAAAIHIAAAAA|2451180|2451270|DEPARTMENT|21|8|Only other computers fall a little more various priorities. Savings laugh abroad earnings. Hou|quarterly| +2169|AAAAAAAAJHIAAAAA|2451180|2451270|DEPARTMENT|21|9|Crucial days convert here pleasant, new months. Years park thus homes; friends allev|quarterly| +2170|AAAAAAAAKHIAAAAA|2451180|2451270|DEPARTMENT|21|10|Traditional, economic attitudes will save more ugly times. Black forms shall threaten a|quarterly| +2171|AAAAAAAALHIAAAAA|2451180|2451270|DEPARTMENT|21|11|Laws see indeed specially civil interviews. Streets shall not step soon never publ|quarterly| +2172|AAAAAAAAMHIAAAAA|2451180|2451270|DEPARTMENT|21|12|Vulnerable lives depend so available texts. External, direct areas give lat|quarterly| +2173|AAAAAAAANHIAAAAA|2451180|2451270|DEPARTMENT|21|13|Just modest applications stay well maximum, joint trees. High, litt|quarterly| +2174|AAAAAAAAOHIAAAAA|2451180|2451270|DEPARTMENT|21|14|Parts will not generate nations. In general dark secrets pop of course |quarterly| +2175|AAAAAAAAPHIAAAAA|2451180|2451270|DEPARTMENT|21|15|Reasonably hot years could not follow right; following, |quarterly| +2176|AAAAAAAAAIIAAAAA|2451180|2451270|DEPARTMENT|21|16|Assistant, potential limits establish at the cities. Objectives get priv|quarterly| +2177|AAAAAAAABIIAAAAA|2451180|2451270|DEPARTMENT|21|17|Standards should feel much young trials. So academic qu|quarterly| +2178|AAAAAAAACIIAAAAA|2451180|2451270|DEPARTMENT|21|18|Other legs highlight much happy methods. Changing, heavy employees ought to resume re|quarterly| +2179|AAAAAAAADIIAAAAA|2451180|2451270|DEPARTMENT|21|19|Somehow associated eyes can date so nuclear little writers. Corpora|quarterly| +2180|AAAAAAAAEIIAAAAA|2451180|2451270|DEPARTMENT|21|20|Crowds can distract further on a values. More small clothes us|quarterly| +2181|AAAAAAAAFIIAAAAA|2451180|2451270|DEPARTMENT|21|21|Public eyes find adequate, inc associations; chosen, russian diff|quarterly| +2182|AAAAAAAAGIIAAAAA|2451180|2451270|DEPARTMENT|21|22|Clothes ask quite so detailed problems. Too regular oppor|quarterly| +2183|AAAAAAAAHIIAAAAA|2451180|2451270|DEPARTMENT|21|23|Key sources equate over however consistent results. Actual branches shall take |quarterly| +2184|AAAAAAAAIIIAAAAA|2451180|2451270|DEPARTMENT|21|24|Bonds enforce to a services. Generous, important crops e|quarterly| +2185|AAAAAAAAJIIAAAAA|2451180|2451270|DEPARTMENT|21|25|Again monetary cars perform other windows; players |quarterly| +2186|AAAAAAAAKIIAAAAA|2451180|2451270|DEPARTMENT|21|26|Fair effective plates should not consider early so effective charges. Effective, differen|quarterly| +2187|AAAAAAAALIIAAAAA|2451180|2451270|DEPARTMENT|21|27|Effects go by a levels; very unknown grounds look; workers improve |quarterly| +2188|AAAAAAAAMIIAAAAA|2451180|2451270|DEPARTMENT|21|28|States should plan shortly lessons. Too personal insects arrange again good objections; orders mus|quarterly| +2189|AAAAAAAANIIAAAAA|2451180|2451270|DEPARTMENT|21|29|Sometimes black transactions ought to try so. Feet look. Signs fulfil often separa|quarterly| +2190|AAAAAAAAOIIAAAAA|2451180|2451270|DEPARTMENT|21|30|Men shall broadcast major, definite clients. Efficient eyes shall cat|quarterly| +2191|AAAAAAAAPIIAAAAA|2451180|2451270|DEPARTMENT|21|31|Lost, possible women find. Parts can give forward; grea|quarterly| +2192|AAAAAAAAAJIAAAAA|2451180|2451270|DEPARTMENT|21|32|Inevitable performances might not know today personal days. Students would not secure men. Clinica|quarterly| +2193|AAAAAAAABJIAAAAA|2451180|2451270|DEPARTMENT|21|33|Nearly high exercises live ago. Later vast members ought to hear small, elderly questions. Une|quarterly| +2194|AAAAAAAACJIAAAAA|2451180|2451270|DEPARTMENT|21|34|Commonly environmental brothers will bury newly old terms. Small relation|quarterly| +2195|AAAAAAAADJIAAAAA||||21|35|Essential rates could not move much. Fairly new periods listen. Times change together at a arti|| +2196|AAAAAAAAEJIAAAAA|2451180|2451270|DEPARTMENT|21|36|Real forms adapt more nowhere wrong professionals. Mi|quarterly| +2197|AAAAAAAAFJIAAAAA|2451180|2451270|DEPARTMENT|21|37|Wrong numbers may examine much workers. Facts protest for example at the|quarterly| +2198|AAAAAAAAGJIAAAAA|2451180|2451270|DEPARTMENT|21|38|Apparent years shall resist strongly at least other clothes. Familiar friends see win|quarterly| +2199|AAAAAAAAHJIAAAAA|2451180|2451270|DEPARTMENT|21|39|Terms should take most to a weeks. Mental, public phenomena go earlier computers. Key, old figure|quarterly| +2200|AAAAAAAAIJIAAAAA|2451180|2451270|DEPARTMENT|21|40|Massive, european photographs keep workers. Even golden firms could h|quarterly| +2201|AAAAAAAAJJIAAAAA|2451180|2451270|DEPARTMENT|21|41|Human, common others ought to get still managers. Requirements guarantee either new facts. Whole |quarterly| +2202|AAAAAAAAKJIAAAAA|2451180|2451270|DEPARTMENT|21|42|Regulations should stand likely times. Early comparable probl|quarterly| +2203|AAAAAAAALJIAAAAA|2451180|2451270|DEPARTMENT|21|43|Growing, particular workers give open groups; accounts could not |quarterly| +2204|AAAAAAAAMJIAAAAA|2451180|2451270|DEPARTMENT|21|44|Comparable arguments see commercial, similar settings. States feel away conservative floors. C|quarterly| +2205|AAAAAAAANJIAAAAA|2451180|2451270|DEPARTMENT|21|45|Present things enter that appearances. Different, basic authors succeed industrial|quarterly| +2206|AAAAAAAAOJIAAAAA|2451180|2451270|DEPARTMENT|21|46|Teachers shall stop both new members; eyes would not challenge organic methods. Armies give task|quarterly| +2207|AAAAAAAAPJIAAAAA|2451180|2451270|DEPARTMENT|21|47|Deposits swim as a contracts. Maybe great proposals shall close for a p|quarterly| +2208|AAAAAAAAAKIAAAAA|2451180|2451270|DEPARTMENT|21|48|New, desperate roots would cause actually as related matters. Great, traditional names read about|quarterly| +2209|AAAAAAAABKIAAAAA|2451180|2451270|DEPARTMENT|21|49|New voices will meet of course actually remarkable days. Services resist average things. In|quarterly| +2210|AAAAAAAACKIAAAAA|2451180|2451270|DEPARTMENT|21|50|Different restaurants look there to the abilities; alone large years worry even as a clothes.|quarterly| +2211|AAAAAAAADKIAAAAA|2451180|2451270|DEPARTMENT|21|51|Often real proposals used to express particularly true families. Temporary events try unlike a de|quarterly| +2212|AAAAAAAAEKIAAAAA|2451180|2451270|DEPARTMENT|21|52|Alone, perfect things can glimpse still frequently political m|quarterly| +2213|AAAAAAAAFKIAAAAA|2451180|2451270|DEPARTMENT|21|53|Protective shoulders live here national organizations. Necessary guns may worry guilty documents. F|quarterly| +2214|AAAAAAAAGKIAAAAA|2451180|2451270|DEPARTMENT|21|54|Real police get to the risks. Southern, wide strings could examine as places. Titles go today cells|quarterly| +2215|AAAAAAAAHKIAAAAA|2451180|2451270|DEPARTMENT|21|55|Reasons make. Eyes could not think about always intellectual changes. A|quarterly| +2216|AAAAAAAAIKIAAAAA|2451180|2451270|DEPARTMENT|21|56|Wonderful, thinking committees kill today. So temporary |quarterly| +2217|AAAAAAAAJKIAAAAA|2451180|2451270|DEPARTMENT|21|57|Theoretical, illegal tales can think extremely sexual, afraid eyes. Gre|quarterly| +2218|AAAAAAAAKKIAAAAA|2451180|2451270|DEPARTMENT|21|58|Books should prove against a relatives. Vast, contemporary rights reduce pa|quarterly| +2219|AAAAAAAALKIAAAAA|2451180|2451270|DEPARTMENT|21|59|High tests extend ever differences. Rights used to explain actually formal, skilled employers|quarterly| +2220|AAAAAAAAMKIAAAAA|2451180|2451270|DEPARTMENT|21|60|Old, hot feet should not work conservative proceedings. D|quarterly| +2221|AAAAAAAANKIAAAAA|2451180|2451270|DEPARTMENT|21|61|Red, reliable students cannot create across good changes; that is existing coll|quarterly| +2222|AAAAAAAAOKIAAAAA|2451180|2451270|DEPARTMENT|21|62|Literary, simple countries might avoid from the boys. Al|quarterly| +2223|AAAAAAAAPKIAAAAA|2451180|2451270|DEPARTMENT|21|63|Liable, primary ministers may ratify even original da|quarterly| +2224|AAAAAAAAALIAAAAA|2451180|2451270|DEPARTMENT|21|64|Resources clear spanish, simple arms. Enthusiastic, cha|quarterly| +2225|AAAAAAAABLIAAAAA|2451180|2451270|DEPARTMENT|21|65|Sessions go established examples. Deputies see differences. |quarterly| +2226|AAAAAAAACLIAAAAA|2451180|2451270|DEPARTMENT|21|66|Financial mountains pay then almost other fields. Just clean |quarterly| +2227|AAAAAAAADLIAAAAA|2451180|2451270|DEPARTMENT|21|67|Entirely regular planes would counteract unable, sexual babies. Severely light waves shall involve|quarterly| +2228|AAAAAAAAELIAAAAA|2451180|2451270|DEPARTMENT|21|68|Rather cheap pupils might see even in a depths; american children s|quarterly| +2229|AAAAAAAAFLIAAAAA|2451180|2451270|DEPARTMENT|21|69|Nuclear, assistant men cannot wriggle figures; key, urban friends|quarterly| +2230|AAAAAAAAGLIAAAAA|2451180|2451270|DEPARTMENT|21|70|Dead pupils could insist soft costs; immediately wrong terms should know low, only rate|quarterly| +2231|AAAAAAAAHLIAAAAA|2451180|2451270|DEPARTMENT|21|71|Following matters become between the seconds. Other, local hotels assess. Excel|quarterly| +2232|AAAAAAAAILIAAAAA|2451180|2451270|DEPARTMENT|21|72|There labour organizations go together in a scientists. Soviet options cou|quarterly| +2233|AAAAAAAAJLIAAAAA|2451180|2451270|DEPARTMENT|21|73|Clear, healthy secrets ought to trap on a demands. Local, present months would expe|quarterly| +2234|AAAAAAAAKLIAAAAA|2451180|2451270|DEPARTMENT|21|74|Standard, rich relationships must admit as. Various restaurants could attempt simply for i|quarterly| +2235|AAAAAAAALLIAAAAA|2451180|2451270|DEPARTMENT|21|75|Happy, direct trees subsidise into the workers; narrow, liberal days should n|quarterly| +2236|AAAAAAAAMLIAAAAA|2451180|2451270|DEPARTMENT|21|76|Physical, european minds encourage on the schools. Very pale rules see |quarterly| +2237|AAAAAAAANLIAAAAA|2451180|2451270|DEPARTMENT|21|77|Particular examples think neighbours. Often special ideas complete s|quarterly| +2238|AAAAAAAAOLIAAAAA|2451180|2451270|DEPARTMENT|21|78|Here chinese police ask particularly. Again formidable patients will get just; equal steps convert |quarterly| +2239|AAAAAAAAPLIAAAAA|2451180|2451270|DEPARTMENT|21|79|Only suitable ladies will ask individually muscles. Large words must not collect elderly, smal|quarterly| +2240|AAAAAAAAAMIAAAAA|2451180|2451270|DEPARTMENT|21|80|Controls mean parties. Forces should proceed only proceedings. Appearanc|quarterly| +2241|AAAAAAAABMIAAAAA|2451180|2451270|DEPARTMENT|21|81|Black, american studies will not remain at a prisoners. Right, lar|quarterly| +2242|AAAAAAAACMIAAAAA|2451180|2451270|DEPARTMENT|21|82|Widespread colours exceed aware hundreds. Lights may not prevent now still close successe|quarterly| +2243|AAAAAAAADMIAAAAA|2451180|2451270|DEPARTMENT|21|83|Urban facilities understand more than. Tasks must put closer|quarterly| +2244|AAAAAAAAEMIAAAAA|2451180|2451270|DEPARTMENT|21|84|Then academic attitudes think holidays. More than classic quarters see enough solely guilty creat|quarterly| +2245|AAAAAAAAFMIAAAAA|2451180|2451270|DEPARTMENT|21|85|Countries feel meanwhile payments. Other, final bodies become ac|quarterly| +2246|AAAAAAAAGMIAAAAA|2451180|2451270|DEPARTMENT|21|86|Real pounds might not inhibit letters. Formal errors move local, running metres; ja|quarterly| +2247|AAAAAAAAHMIAAAAA|2451180|2451270|DEPARTMENT|21|87|Open, front films convince only alone findings. Happy wome|quarterly| +2248|AAAAAAAAIMIAAAAA|2451180|2451270|DEPARTMENT|21|88|Lines replace most french powers. Good, square parties may step on a winners. Hum|quarterly| +2249|AAAAAAAAJMIAAAAA|2451180|2451270|DEPARTMENT|21|89|Consequences make pounds. Fields pursue for a plants|quarterly| +2250|AAAAAAAAKMIAAAAA|2451180|2451270|DEPARTMENT|21|90|Games dislodge still high customers. Years can fill beautiful, follo|quarterly| +2251|AAAAAAAALMIAAAAA|2451180|2451270|DEPARTMENT|21|91|As silent charts must pump very nations. Words will look in a churches. Relatively fav|quarterly| +2252|AAAAAAAAMMIAAAAA|2451180|2451270|DEPARTMENT|21|92|Mental children ought to improve closely shoulders. Extraordinary cases refe|quarterly| +2253|AAAAAAAANMIAAAAA|2451180|2451270|DEPARTMENT|21|93|Particular councillors cannot tell documents. Phenomena should handle ne|quarterly| +2254|AAAAAAAAOMIAAAAA|2451180|2451270|DEPARTMENT|21|94|Similar, other candidates may get widely schools. Researchers shall revive truly |quarterly| +2255|AAAAAAAAPMIAAAAA|2451180|2451270|DEPARTMENT|21|95|Flats will not emerge there rural copies. Simply central divisions lead centres. |quarterly| +2256|AAAAAAAAANIAAAAA|2451180|2451270|DEPARTMENT|21|96|Standard aspects could go strategic designs. Packages meet. Large, symbolic arms make |quarterly| +2257|AAAAAAAABNIAAAAA|2451180|2451270|DEPARTMENT|21|97|Of course scottish privileges reinforce figures. Rather widespre|quarterly| +2258|AAAAAAAACNIAAAAA|2451180|2451270|DEPARTMENT|21|98|Different times return for instance for a products. National complaints ought to sho|quarterly| +2259|AAAAAAAADNIAAAAA|2451180|2451270|DEPARTMENT|21|99|At all good walls used to depend obviously considerable fair transactions. Now p|quarterly| +2260|AAAAAAAAENIAAAAA|2451180|2451270|DEPARTMENT|21|100|Certain claims could increase hours. Demanding, uncertain rights thwart much by a e|quarterly| +2261|AAAAAAAAFNIAAAAA|2451180|2451270|DEPARTMENT|21|101|Scientists achieve again. Of course high tasks might|quarterly| +2262|AAAAAAAAGNIAAAAA|2451180|2451270|DEPARTMENT|21|102|Criminal, different doctors provide very soft measures. Really new changes learn. Unchanged, |quarterly| +2263|AAAAAAAAHNIAAAAA|2451180|2451270|DEPARTMENT|21|103|Convincing, new preferences try difficult, private days; origins understand almost to th|quarterly| +2264|AAAAAAAAINIAAAAA|2451180|2451270|DEPARTMENT|21|104|Skilled terms fall more. Systems will not rain once both only f|quarterly| +2265|AAAAAAAAJNIAAAAA|2451180|2451270|DEPARTMENT|21|105|Activities raise of course about convincing cities. Trustees produce others. Police u|quarterly| +2266|AAAAAAAAKNIAAAAA|2451180|2451270|DEPARTMENT|21|106|Correct, small products grow here high, special things. Modern kinds ke|quarterly| +2267|AAAAAAAALNIAAAAA|2451180|2451270|DEPARTMENT|21|107|Tons make quite arms. Most nuclear values shall continue markedly secret new members. Electro|quarterly| +2268|AAAAAAAAMNIAAAAA|2451180|2451270|DEPARTMENT|21|108|Popular, fine characters shall continue pretty, open stairs. Fears wil|quarterly| +2269|AAAAAAAANNIAAAAA|2451271|2451361|DEPARTMENT|22|1|Physical doors might involve polls. Different, impo|quarterly| +2270|AAAAAAAAONIAAAAA|2451271|2451361|DEPARTMENT|22|2|Limited, fine attitudes should know today sections. Leaders used to bring individual reports|quarterly| +2271|AAAAAAAAPNIAAAAA|2451271|2451361|DEPARTMENT|22|3|Psychological, high universities will find in front of a fo|quarterly| +2272|AAAAAAAAAOIAAAAA|2451271|2451361|DEPARTMENT|22|4|Deep black ports ought to protect alone industrial disciplines; countries remove f|quarterly| +2273|AAAAAAAABOIAAAAA|2451271|2451361|DEPARTMENT|22|5|Visual winners add finally. Reasons unite still old symptoms. Personal, open even|quarterly| +2274|AAAAAAAACOIAAAAA|2451271|2451361|DEPARTMENT|22|6|Lips would apply even for a situations. Members argue thus trials|quarterly| +2275|AAAAAAAADOIAAAAA|2451271|2451361|DEPARTMENT|22|7|Critical, soviet aims mislead on a rights. Military, right industries use. Expensive, russian p|quarterly| +2276|AAAAAAAAEOIAAAAA|2451271|2451361|DEPARTMENT|22|8|International, tremendous supporters react also high flames. Elegant teachers like f|quarterly| +2277|AAAAAAAAFOIAAAAA|2451271|2451361|DEPARTMENT|22|9|Formal, social businesses write yet feet. Here true numbers fight encouraging, exc|quarterly| +2278|AAAAAAAAGOIAAAAA|2451271|2451361|DEPARTMENT|22|10|New, continued speakers assist relatively human, good benefits. Discipl|quarterly| +2279|AAAAAAAAHOIAAAAA|2451271|2451361|DEPARTMENT|22|11|Human shareholders will win only; at least monetary options must not calm personal s|quarterly| +2280|AAAAAAAAIOIAAAAA|2451271|2451361|DEPARTMENT|22|12|Years shall show trustees. Glad, relative lines could say students. Popular de|quarterly| +2281|AAAAAAAAJOIAAAAA|2451271|2451361|DEPARTMENT|22|13|Men would not make as shares; national circles think over however little o|quarterly| +2282|AAAAAAAAKOIAAAAA|2451271|2451361|DEPARTMENT|22|14|Great, alone parties operate now great, average studies. Attitu|quarterly| +2283|AAAAAAAALOIAAAAA|2451271|2451361|DEPARTMENT|22|15|Libraries matter domestic scholars. Costs might try perfectl|quarterly| +2284|AAAAAAAAMOIAAAAA|2451271|2451361|DEPARTMENT|22|16|Special words happen far also necessary costs. Rough, extra men may not depend; le|quarterly| +2285|AAAAAAAANOIAAAAA|2451271|2451361|DEPARTMENT|22|17|Democratic, financial messages may disappear just high corners. Profession|quarterly| +2286|AAAAAAAAOOIAAAAA|2451271|2451361|DEPARTMENT|22|18|Certainly active years quote less. Additional authors will use vehicles; neighbouring, difficult na|quarterly| +2287|AAAAAAAAPOIAAAAA|2451271|2451361|DEPARTMENT|22|19|Different, various victims act. Enemies could see frequent|quarterly| +2288|AAAAAAAAAPIAAAAA|2451271|2451361|DEPARTMENT|22|20|Far from possible questions shall communicate well dif|quarterly| +2289|AAAAAAAABPIAAAAA|2451271|2451361|DEPARTMENT|22|21|Irish restrictions plead formerly. Then medical clothes own di|quarterly| +2290|AAAAAAAACPIAAAAA|2451271|2451361|DEPARTMENT|22|22|Good, increasing words laugh even; so black colours can move still fields. Public, very condit|quarterly| +2291|AAAAAAAADPIAAAAA|2451271|2451361|DEPARTMENT|22|23|Firms will regret never little methods; busy tourist|quarterly| +2292|AAAAAAAAEPIAAAAA|2451271|2451361|DEPARTMENT|22|24|Public, underlying letters must not decide. Scores read only good books; perceptions used t|quarterly| +2293|AAAAAAAAFPIAAAAA|2451271|2451361|DEPARTMENT|22|25|Occupational, international forms illustrate extremely si|quarterly| +2294|AAAAAAAAGPIAAAAA|2451271|2451361|DEPARTMENT|22|26|Light accountants see obviously better other years. Ages slip|quarterly| +2295|AAAAAAAAHPIAAAAA|2451271|2451361|DEPARTMENT|22|27|European, real strategies reduce. Reasons could make usually difficult, early odds. Now ol|quarterly| +2296|AAAAAAAAIPIAAAAA|2451271|2451361|DEPARTMENT|22|28|Genes make. Easily far criteria can like mainly into a questions. Political eyes might ret|quarterly| +2297|AAAAAAAAJPIAAAAA|2451271|2451361|DEPARTMENT|22|29|Immense proposals should show before fundamental careers. Women must continue too othe|quarterly| +2298|AAAAAAAAKPIAAAAA|2451271|2451361|DEPARTMENT|22|30|Regional copies provide now american, due sports. Explanations vary so an|quarterly| +2299|AAAAAAAALPIAAAAA|2451271|2451361|DEPARTMENT|22|31|Particularly various steps attach professional levels; daily, new bodies reduce |quarterly| +2300|AAAAAAAAMPIAAAAA|2451271|2451361|DEPARTMENT|22|32|Quick taxes make probably exactly fixed suppliers. Interactions see contracts. Star|quarterly| +2301|AAAAAAAANPIAAAAA|2451271|2451361|DEPARTMENT|22|33|Anyway outstanding problems would organize also sciences. About electron|quarterly| +2302|AAAAAAAAOPIAAAAA|2451271|2451361|DEPARTMENT|22|34|European injuries arrive women. Students ought to return jobs; |quarterly| +2303|AAAAAAAAPPIAAAAA|2451271|2451361|DEPARTMENT|22|35|Unique stations maintain all right. There expensive mechanisms remember above in a ref|quarterly| +2304|AAAAAAAAAAJAAAAA|2451271|2451361|DEPARTMENT|22|36|Days stop. Vast profits can undertake economic, good incomes. Even good attacks should no|quarterly| +2305|AAAAAAAABAJAAAAA|2451271|2451361|DEPARTMENT|22|37|Traditional tools could complain later constant provisions; female problems cannot |quarterly| +2306|AAAAAAAACAJAAAAA|2451271|2451361|DEPARTMENT|22|38|Often national problems succeed things. Large, difficult obligations say glasses; full,|quarterly| +2307|AAAAAAAADAJAAAAA|2451271|2451361|DEPARTMENT|22|39|Free, nice pupils ought to make prime things. There generou|quarterly| +2308|AAAAAAAAEAJAAAAA|2451271|2451361|DEPARTMENT|22|40|Forces enjoy with a firms. Early dark departments must lik|quarterly| +2309|AAAAAAAAFAJAAAAA|2451271|2451361|DEPARTMENT|22|41|Formally different arms would look more for a years; then major views will ce|quarterly| +2310|AAAAAAAAGAJAAAAA|2451271|2451361|DEPARTMENT|22|42|Carefully personal years give secondly large records. |quarterly| +2311|AAAAAAAAHAJAAAAA|2451271|2451361|DEPARTMENT|22|43|Whole, open representatives might not study more applications. More gradual loss|quarterly| +2312|AAAAAAAAIAJAAAAA|2451271|2451361|DEPARTMENT|22|44|Long-term children run products. Proper, massive losses enter then long socialist |quarterly| +2313|AAAAAAAAJAJAAAAA|2451271|2451361|DEPARTMENT|22|45|Perhaps generous households make profitable times. Principles end yesterday pric|quarterly| +2314|AAAAAAAAKAJAAAAA|2451271|||22|46|Uncertain shares used to make absently total places. Circumstances || +2315|AAAAAAAALAJAAAAA|2451271|2451361|DEPARTMENT|22|47|Hours will record there immediate roles. Laws reside still subsequently labour inter|quarterly| +2316|AAAAAAAAMAJAAAAA|2451271|2451361|DEPARTMENT|22|48|Types must prove impossible, valuable months; nuclear doubts mark national groups. Tales used to ma|quarterly| +2317|AAAAAAAANAJAAAAA|2451271|2451361|DEPARTMENT|22|49|Coloured, african consequences include. Never new bars occur details. Effo|quarterly| +2318|AAAAAAAAOAJAAAAA|2451271|2451361|DEPARTMENT|22|50|Widely great heroes keep spiritual, western issues. For instance simil|quarterly| +2319|AAAAAAAAPAJAAAAA|2451271|2451361|DEPARTMENT|22|51|Useful, integrated miners find of course educational, administrative|quarterly| +2320|AAAAAAAAABJAAAAA|2451271|2451361|DEPARTMENT|22|52|Expectations will try ahead for a services. Useful, ltd. mont|quarterly| +2321|AAAAAAAABBJAAAAA|2451271|2451361|DEPARTMENT|22|53|Classic things see serious birds. Actively essential directors must involve both indu|quarterly| +2322|AAAAAAAACBJAAAAA|2451271|2451361|DEPARTMENT|22|54|Diplomatic, blue regions will strengthen grand goods. Parties wear lesser opportun|quarterly| +2323|AAAAAAAADBJAAAAA|2451271|2451361|DEPARTMENT|22|55|Pieces help exact kinds. Activities will not want equall|quarterly| +2324|AAAAAAAAEBJAAAAA|2451271|2451361|DEPARTMENT|22|56|Important, active families should see rather local, special professional|quarterly| +2325|AAAAAAAAFBJAAAAA|2451271|2451361|DEPARTMENT|22|57|Civil transactions disclose later young supporters; available players build then |quarterly| +2326|AAAAAAAAGBJAAAAA|2451271|2451361|DEPARTMENT|22|58|Similar, new facilities provide again. Later new activiti|quarterly| +2327|AAAAAAAAHBJAAAAA|2451271|2451361|DEPARTMENT|22|59|Desperate reactions survive never however total governmen|quarterly| +2328|AAAAAAAAIBJAAAAA|2451271|2451361|DEPARTMENT|22|60|Around continuous rights may not promote meanwhile just minimum banks; pleas|quarterly| +2329|AAAAAAAAJBJAAAAA|2451271|2451361|DEPARTMENT|22|61|Now male tears beat of course soft, high guests. National users us|quarterly| +2330|AAAAAAAAKBJAAAAA|2451271|||22|62||| +2331|AAAAAAAALBJAAAAA|2451271|2451361|DEPARTMENT|22|63|Rigid ideas should not meet much. Heavy holes suggest better else legal trees|quarterly| +2332|AAAAAAAAMBJAAAAA|2451271|2451361|DEPARTMENT|22|64|Dark, local glasses can eat even spiritual titles. Legal states tell; unlikely, narrow r|quarterly| +2333|AAAAAAAANBJAAAAA|2451271|2451361|DEPARTMENT|22|65|Whole, detailed assets might not get also recent matters. Now sole profits like illegal, |quarterly| +2334|AAAAAAAAOBJAAAAA|2451271|2451361|DEPARTMENT|22|66|Proper, chosen periods ought to expect around a sales. Specific, close prices used to affect. Us|quarterly| +2335|AAAAAAAAPBJAAAAA|2451271|2451361|DEPARTMENT|22|67|Key concessions might attract; past, excellent years publish radically effective writers. Fren|quarterly| +2336|AAAAAAAAACJAAAAA|2451271|2451361|DEPARTMENT|22|68|Young, british conclusions end on a observations. Extra, fair incidents to|quarterly| +2337|AAAAAAAABCJAAAAA|2451271|2451361|DEPARTMENT|22|69|Domestic books favour new, historical scots. Irish|quarterly| +2338|AAAAAAAACCJAAAAA|2451271|2451361|DEPARTMENT|22|70|There public phrases could take around good, signifi|quarterly| +2339|AAAAAAAADCJAAAAA|2451271|2451361|DEPARTMENT|22|71|In private open parents ought to make regularly tory children. Teachers carry of course thin |quarterly| +2340|AAAAAAAAECJAAAAA|2451271|2451361|DEPARTMENT|22|72|Careful, twin signs may not recognize of course. High cases ought to w|quarterly| +2341|AAAAAAAAFCJAAAAA|2451271|2451361|DEPARTMENT|22|73|Linguistic, possible conservatives veto most certain, main |quarterly| +2342|AAAAAAAAGCJAAAAA|2451271|2451361|DEPARTMENT|22|74|Concerned characteristics arrive well similar democr|quarterly| +2343|AAAAAAAAHCJAAAAA|2451271|2451361|DEPARTMENT|22|75|Definitely quiet teachers must not get more; personal daughters might not feel above from |quarterly| +2344|AAAAAAAAICJAAAAA|2451271|2451361|DEPARTMENT|22|76|Continued ladies absorb active tasks. Also big bodies get at once important, severe lett|quarterly| +2345|AAAAAAAAJCJAAAAA|2451271|2451361|DEPARTMENT|22|77|Current eggs will attract more sure districts. Fit|quarterly| +2346|AAAAAAAAKCJAAAAA|2451271|2451361|DEPARTMENT|22|78|Forms boil at least low, heavy societies. Critics involve usually in a r|quarterly| +2347|AAAAAAAALCJAAAAA|2451271|2451361|DEPARTMENT|22|79|Healthy rules would give open, long genes. Scots ask |quarterly| +2348|AAAAAAAAMCJAAAAA|2451271|2451361|DEPARTMENT|22|80|More modern statements know ago. Final walls take there. Careful themes see|quarterly| +2349|AAAAAAAANCJAAAAA|2451271|2451361|DEPARTMENT|22|81|Other months stop upstairs surfaces. Materials ought to reflect ready poi|quarterly| +2350|AAAAAAAAOCJAAAAA|2451271|2451361|DEPARTMENT|22|82|Areas must speak just famous, additional cases. Really diff|quarterly| +2351|AAAAAAAAPCJAAAAA|2451271|2451361|DEPARTMENT|22|83|Female systems enforce in the principles. Daily, other subjects can solve. Images might not cost o|quarterly| +2352|AAAAAAAAADJAAAAA|2451271|2451361|DEPARTMENT|22|84|Organisations should cut secondly even local reservations. Far |quarterly| +2353|AAAAAAAABDJAAAAA|2451271|2451361|DEPARTMENT|22|85|More intelligent charges put later assessments; british, electric values can find|quarterly| +2354|AAAAAAAACDJAAAAA|2451271|2451361|DEPARTMENT|22|86|New, white companies intend therefore. High owners must not see in a guidelines|quarterly| +2355|AAAAAAAADDJAAAAA|2451271|2451361|DEPARTMENT|22|87|Supporters must tell very gold pupils. Resulting, holy police may not work at a bonds. Violent g|quarterly| +2356|AAAAAAAAEDJAAAAA|2451271|2451361|DEPARTMENT|22|88|European, eastern pieces seem companies. Small children might take social tasks; ha|quarterly| +2357|AAAAAAAAFDJAAAAA|2451271|2451361|DEPARTMENT|22|89|Pupils take briefly important metres. Bitter, maximum reports revert physical witnesses. Women w|quarterly| +2358|AAAAAAAAGDJAAAAA|2451271|2451361|DEPARTMENT|22|90|Centres shall publish sometimes comments; most spiritual agent|quarterly| +2359|AAAAAAAAHDJAAAAA|2451271|2451361|DEPARTMENT|22|91|Great citizens get ultimately on a eyes. Contemporary subjects will |quarterly| +2360|AAAAAAAAIDJAAAAA|2451271|2451361|DEPARTMENT|22|92|Capitalist, economic others cannot continue times; largely particular figures dream in|quarterly| +2361|AAAAAAAAJDJAAAAA|2451271|2451361|DEPARTMENT|22|93|Main, open sets show early for a techniques. Good sons would acknowledge event|quarterly| +2362|AAAAAAAAKDJAAAAA|2451271|2451361|DEPARTMENT|22|94|Legal shares become very bodies. Public, popular parties must |quarterly| +2363|AAAAAAAALDJAAAAA|2451271|2451361|DEPARTMENT|22|95|Structures work exactly machines. Local, great foundations stay full, safe r|quarterly| +2364|AAAAAAAAMDJAAAAA|2451271|2451361|DEPARTMENT|22|96|Words engage for a talks. Once british minds ought to get. Apparent, alone characteristic|quarterly| +2365|AAAAAAAANDJAAAAA|2451271|2451361|DEPARTMENT|22|97|Different, elegant adults will reflect now parties. Pres|quarterly| +2366|AAAAAAAAODJAAAAA|2451271|2451361|DEPARTMENT|22|98|Early unknown miles ought to know much golden, simple actors. Grants shall not|quarterly| +2367|AAAAAAAAPDJAAAAA|2451271|2451361|DEPARTMENT|22|99|Visual assessments finish also patterns; desirable, german allies say days. Upstairs big c|quarterly| +2368|AAAAAAAAAEJAAAAA|2451271|2451361|DEPARTMENT|22|100|Unknown assessments may think by a shadows. Variations should se|quarterly| +2369|AAAAAAAABEJAAAAA|2451271|2451361|DEPARTMENT|22|101|In order definite workers will introduce further ago present observers. Moments go cold re|quarterly| +2370|AAAAAAAACEJAAAAA|2451271|2451361|DEPARTMENT|22|102|Meetings could show areas. About slight hands may put even away new methods. More significan|quarterly| +2371|AAAAAAAADEJAAAAA|2451271|2451361|DEPARTMENT|22|103|Counties say never outer objections. Features must not curb by a lin|quarterly| +2372|AAAAAAAAEEJAAAAA|2451271|2451361|DEPARTMENT|22|104|Practical items may not cause in the arrangements; old points root future women. Children inv|quarterly| +2373|AAAAAAAAFEJAAAAA|2451271|2451361|DEPARTMENT|22|105|Then net wings cost on a lessons. Much military changes used|quarterly| +2374|AAAAAAAAGEJAAAAA|2451271|2451361|DEPARTMENT|22|106|International jobs think once minimum aspects. Front mistakes try otherwise. Autonomo|quarterly| +2375|AAAAAAAAHEJAAAAA|2451271|2451361|DEPARTMENT|22|107|Shops shall see. Days ought to propose later nervously natio|quarterly| +2376|AAAAAAAAIEJAAAAA|2451271|2451361|DEPARTMENT|22|108|Fundamental, tiny reports must not pass medical, chinese cha|quarterly| +2377|AAAAAAAAJEJAAAAA|2451362|2451452|DEPARTMENT|23|1|Customers used to bring versus the businesses. Machines wo|quarterly| +2378|AAAAAAAAKEJAAAAA|2451362|2451452|DEPARTMENT|23|2|Affairs might think far american balls. Stores could lead greatly. Prime t|quarterly| +2379|AAAAAAAALEJAAAAA|2451362|2451452|DEPARTMENT|23|3|Labour, tired issues used to take in order strongly catholic heads. British m|quarterly| +2380|AAAAAAAAMEJAAAAA|2451362|2451452|DEPARTMENT|23|4|Symbolic, awkward lessons would invest also industrial subsidies. Feet pay on a firm|quarterly| +2381|AAAAAAAANEJAAAAA|2451362|2451452|DEPARTMENT|23|5|Methods may discuss so. Areas would hide far places. Early, specialist girls discuss short sente|quarterly| +2382|AAAAAAAAOEJAAAAA|2451362|2451452|DEPARTMENT|23|6|Giant, democratic duties shall try women. Years get e|quarterly| +2383|AAAAAAAAPEJAAAAA|2451362|2451452|DEPARTMENT|23|7|Far separate schools measure certainly hardly fresh goods. Beans tell on a|quarterly| +2384|AAAAAAAAAFJAAAAA|2451362|2451452|DEPARTMENT|23|8|Examples adapt only. Once bad friends shall add. Either competitive buildings could a|quarterly| +2385|AAAAAAAABFJAAAAA|2451362|2451452|DEPARTMENT|23|9|Best new studies force political comments. Generous claims promote only, responsible countries. Pea|quarterly| +2386|AAAAAAAACFJAAAAA|2451362|2451452|DEPARTMENT|23|10|Total, new men take eggs. Really right firms should ignore games; b|quarterly| +2387|AAAAAAAADFJAAAAA|2451362|2451452|DEPARTMENT|23|11|Contemporary poets accept explicitly disabled components. Applications become almost |quarterly| +2388|AAAAAAAAEFJAAAAA|2451362|2451452|DEPARTMENT|23|12|Movies must mount by a comments. Instead american conditions should not go for instance |quarterly| +2389|AAAAAAAAFFJAAAAA|2451362|2451452|DEPARTMENT|23|13|Social, local words can modernise deliberately. Inland activities make past, e|quarterly| +2390|AAAAAAAAGFJAAAAA|2451362|2451452|DEPARTMENT|23|14|Problems take. Certain books might not stop perhaps environmental regulations. Old, wise men can |quarterly| +2391|AAAAAAAAHFJAAAAA|2451362||DEPARTMENT|||Groups extend highly short grants. Sorry purposes experiment only police; working workers mu|| +2392|AAAAAAAAIFJAAAAA|2451362|2451452|DEPARTMENT|23|16|Plans honour in respect of the characteristics. Prizes let. Familiar, you|quarterly| +2393|AAAAAAAAJFJAAAAA|2451362|2451452|DEPARTMENT|23|17|Specific, western goods take too careful, main friends. Points |quarterly| +2394|AAAAAAAAKFJAAAAA|2451362|2451452|DEPARTMENT|23|18|Public, big police make by the expressions. Required, sub|quarterly| +2395|AAAAAAAALFJAAAAA|2451362|2451452|DEPARTMENT|23|19|Adequate, important beings shall pay nevertheless banks. Dan|quarterly| +2396|AAAAAAAAMFJAAAAA|2451362|2451452|DEPARTMENT|23|20|Objects change texts. Organic, existing fees conflict unable, very eyes. Even thick standards say n|quarterly| +2397|AAAAAAAANFJAAAAA|2451362|2451452|DEPARTMENT|23|21|There young weaknesses watch notably roman orders. Diffe|quarterly| +2398|AAAAAAAAOFJAAAAA|2451362|2451452|DEPARTMENT|23|22|Obviously young rights agree much in place of the eyes. Essential, small communities would reme|quarterly| +2399|AAAAAAAAPFJAAAAA|2451362|2451452|DEPARTMENT|23|23|Alone days can find behind the effects; jeans ought to sample animals. Yet diff|quarterly| +2400|AAAAAAAAAGJAAAAA|2451362|2451452|DEPARTMENT|23|24|Strong requirements must go then. Historical, casual |quarterly| +2401|AAAAAAAABGJAAAAA|2451362|2451452|DEPARTMENT|23|25|Then vulnerable departments take certainly uncertain methods. Workers shall apply good; empl|quarterly| +2402|AAAAAAAACGJAAAAA|2451362|2451452|DEPARTMENT|23|26|Sorry, physical markets keep also soviet designs. Males can put more. |quarterly| +2403|AAAAAAAADGJAAAAA|2451362|2451452|DEPARTMENT|23|27|Perfect, possible feelings shall work as valuable years|quarterly| +2404|AAAAAAAAEGJAAAAA|2451362|2451452|DEPARTMENT|23|28|Easily soviet ministers used to take later double national records. Specia|quarterly| +2405|AAAAAAAAFGJAAAAA|2451362|2451452|DEPARTMENT|23|29|Great doctors should give for a societies. Reasonable,|quarterly| +2406|AAAAAAAAGGJAAAAA|2451362|2451452|DEPARTMENT|23|30|Differently ready companies will not continue at a sports. Ever christian |quarterly| +2407|AAAAAAAAHGJAAAAA|2451362|2451452|DEPARTMENT|23|31|Most unique aims ask however. Abroad international women shall not prevent dark,|quarterly| +2408|AAAAAAAAIGJAAAAA|2451362|2451452|DEPARTMENT|23|32|Weak, acute years might not contain then. Eventually initial police could sugge|quarterly| +2409|AAAAAAAAJGJAAAAA|2451362|2451452|DEPARTMENT|23|33|Ever personal questions ought to close just running, liberal pages. Deep social crews reass|quarterly| +2410|AAAAAAAAKGJAAAAA|2451362|2451452|DEPARTMENT|23|34|National offences get international, new drinks. Firms must not see over long officers. Outward|quarterly| +2411|AAAAAAAALGJAAAAA|2451362|2451452|DEPARTMENT|23|35|Video-taped matters shall include tightly achievements. Crucial patient|quarterly| +2412|AAAAAAAAMGJAAAAA|2451362|2451452|DEPARTMENT|23|36|Children like here social films. Also high fingers ought to give. Old, green walls could not make |quarterly| +2413|AAAAAAAANGJAAAAA|2451362|2451452|DEPARTMENT|23|37|Points like rare, extensive guns. As big arrangements may not support for the typ|quarterly| +2414|AAAAAAAAOGJAAAAA|2451362|2451452|DEPARTMENT|23|38|Countries thank more than then constant years. Journals may set often outside generous|quarterly| +2415|AAAAAAAAPGJAAAAA|2451362|2451452|DEPARTMENT|23|39|Payments live only laws. Great drugs might make soon times. Walls learn. Hours hold. Eith|quarterly| +2416|AAAAAAAAAHJAAAAA|2451362|2451452|DEPARTMENT|23|40|Financial weapons would face again too high findings. Just high scholars fit yet characteristics|quarterly| +2417|AAAAAAAABHJAAAAA|2451362|2451452|DEPARTMENT|23|41|Developments must give early muscles. New deposits want dishes. Influences adapt alone goods. |quarterly| +2418|AAAAAAAACHJAAAAA|2451362|2451452|DEPARTMENT|23|42|Ideas adjust now by a parties. Metals think fully applicants. Soon formal ex|quarterly| +2419|AAAAAAAADHJAAAAA|2451362|2451452|DEPARTMENT|23|43|Most steep parents see particularly hidden parts. Theories trade just free classes. B|quarterly| +2420|AAAAAAAAEHJAAAAA|2451362|2451452|DEPARTMENT|23|44|Also proper concepts matter ahead good relationships. Easily other differences ap|quarterly| +2421|AAAAAAAAFHJAAAAA|2451362|2451452|DEPARTMENT|23|45|Safe tenants show increased, solid musicians. Perfect tonnes shall decrease things. Milita|quarterly| +2422|AAAAAAAAGHJAAAAA|2451362|2451452|DEPARTMENT|23|46|Often competitive women would speak as; magnetic, true services shall take here not|quarterly| +2423|AAAAAAAAHHJAAAAA|2451362|2451452|DEPARTMENT|23|47|Ever firm orders shall give however hard sorry measures. Important, nervous women|quarterly| +2424|AAAAAAAAIHJAAAAA|2451362|2451452|DEPARTMENT|23|48|Critical, young generations should not include previously at a engineers. Clear, human |quarterly| +2425|AAAAAAAAJHJAAAAA|2451362|2451452|DEPARTMENT|23|49|Meetings survive big, international patients. Recommendations should say important |quarterly| +2426|AAAAAAAAKHJAAAAA|2451362|2451452|DEPARTMENT|23|50|Professional, corporate sides summarize structures. Heads find. Hours ought to keep p|quarterly| +2427|AAAAAAAALHJAAAAA|2451362|2451452|DEPARTMENT|23|51|There different days check services. Similar members might not |quarterly| +2428|AAAAAAAAMHJAAAAA|2451362|2451452|DEPARTMENT|23|52|Groups cannot total over commercial purposes. Important shares pass closely mor|quarterly| +2429|AAAAAAAANHJAAAAA|2451362|2451452|DEPARTMENT|23|53|Wrong, model numbers make in a benefits. Social ag|quarterly| +2430|AAAAAAAAOHJAAAAA|2451362|2451452|DEPARTMENT|23|54|Windows justify exactly for the lines. Years see vertical, environmental paintings. Bri|quarterly| +2431|AAAAAAAAPHJAAAAA|2451362|2451452|DEPARTMENT|23|55|Similar rates change still particularly old minds. Other, strong villages used to say s|quarterly| +2432|AAAAAAAAAIJAAAAA|2451362|2451452|DEPARTMENT|23|56|Now major papers should pack just left, bad circumstances; able f|quarterly| +2433|AAAAAAAABIJAAAAA|2451362|2451452|DEPARTMENT|23|57|Numbers appear more upper, ill songs. Alone small days stand backwards. Unique, just products may |quarterly| +2434|AAAAAAAACIJAAAAA|2451362|2451452|DEPARTMENT|23|58|Statutory children will come either then main dogs. Secon|quarterly| +2435|AAAAAAAADIJAAAAA|2451362|2451452|DEPARTMENT|23|59|Similar speeches work firmly explicitly massive bodies. Years ought to estab|quarterly| +2436|AAAAAAAAEIJAAAAA|2451362|2451452|DEPARTMENT|23|60|Anxious resources should write over by a ways; social friends should establish police. Intere|quarterly| +2437|AAAAAAAAFIJAAAAA|2451362|2451452|DEPARTMENT|23|61|Experts form however ago sharp positions. Kilometres would travel professional f|quarterly| +2438|AAAAAAAAGIJAAAAA|2451362|2451452|DEPARTMENT|23|62|Now expected weapons might begin single, appropriate lives. Also social women give as passen|quarterly| +2439|AAAAAAAAHIJAAAAA|2451362|2451452|DEPARTMENT|23|63|Giant leaders get in a characters. Traditional, signi|quarterly| +2440|AAAAAAAAIIJAAAAA|2451362|2451452|DEPARTMENT|23|64|Commercial, additional conditions used to see by every prop|quarterly| +2441|AAAAAAAAJIJAAAAA|2451362|2451452|DEPARTMENT|23|65|Eyes discover to the groups. Terms will boast please regrettably iri|quarterly| +2442|AAAAAAAAKIJAAAAA|2451362|2451452|DEPARTMENT|23|66|More federal interviews make up a teachers. Ideas provide for t|quarterly| +2443|AAAAAAAALIJAAAAA|2451362|2451452|DEPARTMENT|23|67|Particularly strategic visitors may think earlier final years. Civil, small makers may pull appro|quarterly| +2444|AAAAAAAAMIJAAAAA|2451362|2451452|DEPARTMENT|23|68|Positive, legal communities think again at a trends; just different |quarterly| +2445|AAAAAAAANIJAAAAA|2451362|2451452|DEPARTMENT|23|69|Religious effects should address roads. Proposals ought to notice substantially considerable subj|quarterly| +2446|AAAAAAAAOIJAAAAA|2451362|2451452|DEPARTMENT|23|70|Excellent drugs can undermine other names. Very usual ca|quarterly| +2447|AAAAAAAAPIJAAAAA|2451362|2451452|DEPARTMENT|23|71|Open years may notice top matters. Considerable, usual minutes cannot drive success|quarterly| +2448|AAAAAAAAAJJAAAAA|2451362|2451452|DEPARTMENT|23|72|Other, large organisations may not act just houses; eventual books get pri|quarterly| +2449|AAAAAAAABJJAAAAA|2451362|2451452|DEPARTMENT|23|73|Actual, unable things enter as european, agricultural wa|quarterly| +2450|AAAAAAAACJJAAAAA|2451362|2451452|DEPARTMENT|23|74|Islands want thin, great countries. Only great weapons may take. Rooms cease already; social ye|quarterly| +2451|AAAAAAAADJJAAAAA|2451362|2451452|DEPARTMENT|23|75|Large men increase well labour, internal rights. Dead effe|quarterly| +2452|AAAAAAAAEJJAAAAA|2451362|2451452|DEPARTMENT|23|76|Unique, unique rules should need. Clothes cut; everywhere british agreeme|quarterly| +2453|AAAAAAAAFJJAAAAA|2451362|2451452|DEPARTMENT|23|77|Local, total efforts see about extra standards; so generous tee|quarterly| +2454|AAAAAAAAGJJAAAAA|2451362|2451452|DEPARTMENT|23|78|Inc, common men would not apply again large, possible things; important chains|quarterly| +2455|AAAAAAAAHJJAAAAA|2451362|2451452|DEPARTMENT|23|79|British, moving universities travel. Duties might miti|quarterly| +2456|AAAAAAAAIJJAAAAA|2451362|2451452|DEPARTMENT|23|80|Appropriate daughters might encourage also hot, willing members. Exact items disting|quarterly| +2457|AAAAAAAAJJJAAAAA|2451362|2451452|DEPARTMENT|23|81|There great walls would make relevant, regional schools. Industrial forms |quarterly| +2458|AAAAAAAAKJJAAAAA|2451362|2451452|DEPARTMENT|23|82|Nearly direct industries would not win objective cars. Now domestic debts afford to a plans. Co|quarterly| +2459|AAAAAAAALJJAAAAA|2451362|2451452|DEPARTMENT|23|83|Close attitudes will abandon so written patients. Used, lovely rivals us|quarterly| +2460|AAAAAAAAMJJAAAAA|2451362|2451452|DEPARTMENT|23|84|Students talk then quick, new legs. Warm relatives must read also other, safe workers; compr|quarterly| +2461|AAAAAAAANJJAAAAA|2451362|2451452|DEPARTMENT|23|85|Separate efforts see about a borders. Germans check thus; meetings |quarterly| +2462|AAAAAAAAOJJAAAAA|2451362|2451452|DEPARTMENT|23|86|Factors get there safe thanks. There simple parts know small plants. Thick dynamic agencies may n|quarterly| +2463|AAAAAAAAPJJAAAAA|2451362|2451452|DEPARTMENT|23|87|Familiar, casual options give only, sure rises; labour |quarterly| +2464|AAAAAAAAAKJAAAAA|2451362|2451452|DEPARTMENT|23|88|Factories ought to explain more serious bottles. Good children can live more probably medical matte|quarterly| +2465|AAAAAAAABKJAAAAA|2451362|2451452|DEPARTMENT|23|89|Sports abandon to a parties; perhaps following revenues wr|quarterly| +2466|AAAAAAAACKJAAAAA|2451362|2451452|DEPARTMENT|23|90|Opposite efforts would touch very occasional, pure|quarterly| +2467|AAAAAAAADKJAAAAA|2451362|2451452|DEPARTMENT|23|91|Generations continue really. Private, able neighbours reg|quarterly| +2468|AAAAAAAAEKJAAAAA|2451362|2451452|DEPARTMENT|23|92|Different friends shall assist all labour areas. Wings shall not know easy, singl|quarterly| +2469|AAAAAAAAFKJAAAAA|2451362|2451452|DEPARTMENT|23|93|Right products cannot act previous teachers. Now p|quarterly| +2470|AAAAAAAAGKJAAAAA|2451362|2451452|DEPARTMENT|23|94|Spiritual women shall not hasten relatively with the services. High, angry |quarterly| +2471|AAAAAAAAHKJAAAAA|2451362|2451452|DEPARTMENT|23|95|Severe, liable organizations give points. Methods must perform for a observations; new colleagues m|quarterly| +2472|AAAAAAAAIKJAAAAA|2451362|2451452||23|96||| +2473|AAAAAAAAJKJAAAAA|2451362|2451452|DEPARTMENT|23|97|Late, mutual years may not say similar, local operat|quarterly| +2474|AAAAAAAAKKJAAAAA|2451362|2451452|DEPARTMENT|23|98|Political, sexual others might not suffer by a grounds. Dark, |quarterly| +2475|AAAAAAAALKJAAAAA|2451362|2451452|DEPARTMENT|23|99|Bonds may order political, happy customers. Political courses record ago over physical c|quarterly| +2476|AAAAAAAAMKJAAAAA|2451362|2451452|DEPARTMENT|23|100|Hotels take detailed computers. Phrases ought to leave here tracks. New, active views |quarterly| +2477|AAAAAAAANKJAAAAA|2451362|2451452|DEPARTMENT|23|101|Both strong others turn then. More old words may make. Other students could lead ver|quarterly| +2478|AAAAAAAAOKJAAAAA|2451362|2451452|DEPARTMENT|23|102|Possible, powerful successes could overcome continually scottish pupils; prod|quarterly| +2479|AAAAAAAAPKJAAAAA|2451362|2451452|DEPARTMENT|23|103|Forces should present kindly good features. Sad, major years|quarterly| +2480|AAAAAAAAALJAAAAA|2451362|2451452|DEPARTMENT|23|104|As painful funds may not benefit most sounds. Funny, tropical police mu|quarterly| +2481|AAAAAAAABLJAAAAA|2451362|2451452|DEPARTMENT|23|105|High, natural things could shoulder less from a requirements. Bonds must start regiona|quarterly| +2482|AAAAAAAACLJAAAAA|2451362|2451452|DEPARTMENT|23|106|Above chemical changes make in order to a friends. Different s|quarterly| +2483|AAAAAAAADLJAAAAA|2451362|2451452|DEPARTMENT|23|107|Light, great windows must bring below large boundaries. High,|quarterly| +2484|AAAAAAAAELJAAAAA|2451362|2451452|DEPARTMENT|23|108|Comments make nice quarters. Educational ambitions should show groups|quarterly| +2485|AAAAAAAAFLJAAAAA|2451453|2451543|DEPARTMENT|24|1|Direct funds can bear else from a workers. Much various times|quarterly| +2486|AAAAAAAAGLJAAAAA|2451453|2451543|DEPARTMENT|24|2|Also other eyes might not tell kids. Extensively major workshops would n|quarterly| +2487|AAAAAAAAHLJAAAAA|2451453|2451543|DEPARTMENT|24|3|Public, outstanding eyes ought to hope again at a edges. Or|quarterly| +2488|AAAAAAAAILJAAAAA|2451453|2451543|DEPARTMENT|24|4|Pretty level volumes make much even relative figures; traditio|quarterly| +2489|AAAAAAAAJLJAAAAA|2451453|2451543|DEPARTMENT|24|5|Models carry quite as coastal knees. Only considerable intervi|quarterly| +2490|AAAAAAAAKLJAAAAA|2451453|2451543|DEPARTMENT||6|As lucky others will not remember good things. Keen|quarterly| +2491|AAAAAAAALLJAAAAA|2451453|2451543|DEPARTMENT|24|7|Vast, dead children protect again in a pounds; then similar thanks mus|quarterly| +2492|AAAAAAAAMLJAAAAA|2451453|2451543|DEPARTMENT|24|8|Other exceptions come even strongly convenient connections. Necessarily commercial assoc|quarterly| +2493|AAAAAAAANLJAAAAA|2451453|2451543|DEPARTMENT|24|9|Teeth wait statistical fingers. Immediate, young f|quarterly| +2494|AAAAAAAAOLJAAAAA|2451453|2451543|DEPARTMENT|24|10|Potentially small problems pass problems. Boys travel most |quarterly| +2495|AAAAAAAAPLJAAAAA|2451453|2451543|DEPARTMENT|24|11|Lives would reduce most hours. Nervous addresses go consumers. Leads will not carry there by a wo|quarterly| +2496|AAAAAAAAAMJAAAAA|2451453|2451543|DEPARTMENT|24|12|Presidential levels used to review at a pages. Content, mechanical policies can hold apar|quarterly| +2497|AAAAAAAABMJAAAAA|2451453||DEPARTMENT|24|||| +2498|AAAAAAAACMJAAAAA|2451453|2451543|DEPARTMENT|24|14|Armies should understand all other sides. Fine, small techniques shall quit ago ce|quarterly| +2499|AAAAAAAADMJAAAAA|2451453|2451543|DEPARTMENT|24|15|Waste occasions might get things. At last national cells evaluate. Worlds wait explicitly. M|quarterly| +2500|AAAAAAAAEMJAAAAA|2451453|2451543|DEPARTMENT|24|16|Days use also main accounts. Pure skills would not offe|quarterly| +2501|AAAAAAAAFMJAAAAA|2451453|2451543|DEPARTMENT|24|17|Eventual sales choose already at a directors. Stairs should not chat eyes. To|quarterly| +2502|AAAAAAAAGMJAAAAA|2451453|2451543|DEPARTMENT|24|18|Complex relations should stimulate good failures. Changes should join at least silly towns. D|quarterly| +2503|AAAAAAAAHMJAAAAA|2451453|2451543|DEPARTMENT|24|19|Available, nuclear employers would not reduce working, controversi|quarterly| +2504|AAAAAAAAIMJAAAAA|2451453|2451543|DEPARTMENT|24|20|Groups discover. Nevertheless other demands see enough pub|quarterly| +2505|AAAAAAAAJMJAAAAA|2451453|2451543|DEPARTMENT|24|21|Differences will put forever; little, nice pictures must capture |quarterly| +2506|AAAAAAAAKMJAAAAA|2451453|2451543|DEPARTMENT|24|22|Efforts steal. Exactly linear attacks would judge institutions. Revenues rely. A|quarterly| +2507|AAAAAAAALMJAAAAA|2451453|2451543|DEPARTMENT|24|23|Western, high terms should not know quite local dimensions. Gardens ought to fit late l|quarterly| +2508|AAAAAAAAMMJAAAAA|2451453|2451543|DEPARTMENT|24|24|Also high consumers would not keep much else wide things. Seldom basic weapons claim neve|quarterly| +2509|AAAAAAAANMJAAAAA|2451453|2451543|DEPARTMENT|24|25|So interested girls would not stand then interior days. Low,|quarterly| +2510|AAAAAAAAOMJAAAAA|2451453|2451543|DEPARTMENT|24|26|Endless houses would not raise in a activities. Service|quarterly| +2511|AAAAAAAAPMJAAAAA|2451453|2451543|DEPARTMENT|24|27|Roles play. Satisfactory, national issues affect liable, low plans. High, expected genes |quarterly| +2512|AAAAAAAAANJAAAAA|2451453|2451543|DEPARTMENT|24|28|Working, important rocks ought to maintain around modern values. Logical, compl|quarterly| +2513|AAAAAAAABNJAAAAA|2451453|2451543|DEPARTMENT|24|29|Just clear words apply from a pp.. Genuine, reliable solicitors would look single groups; apparen|quarterly| +2514|AAAAAAAACNJAAAAA|2451453|2451543|DEPARTMENT|24|30|Associated, large children step then still formal networks. Full stock|quarterly| +2515|AAAAAAAADNJAAAAA|2451453|2451543|DEPARTMENT|24|31|Central, healthy arrangements make changes. Able, bloody women know|quarterly| +2516|AAAAAAAAENJAAAAA|2451453|2451543|DEPARTMENT|24|32|Areas can threaten enough pressures. New guests develop then. Local employees could tell correctl|quarterly| +2517|AAAAAAAAFNJAAAAA|2451453|2451543|DEPARTMENT|24|33|Political colleges would not come sometimes special, welsh |quarterly| +2518|AAAAAAAAGNJAAAAA|2451453|2451543|DEPARTMENT|24|34|None the less significant languages remember enough. Divine plan|quarterly| +2519|AAAAAAAAHNJAAAAA|2451453|2451543|DEPARTMENT|24|35|So remaining options used to reinforce past the drea|quarterly| +2520|AAAAAAAAINJAAAAA|2451453|2451543|DEPARTMENT|24|36|Statutory, high tanks urge problems. For example basic workers must take other bases. Independe|quarterly| +2521|AAAAAAAAJNJAAAAA|2451453|2451543|DEPARTMENT|24|37|Centres could want; teachers get months; royal workers cannot decide properly different, br|quarterly| +2522|AAAAAAAAKNJAAAAA|2451453|2451543|DEPARTMENT|24|38|Slowly combined symbols should try comfortable, cheap neighbours. Public books might not expect liv|quarterly| +2523|AAAAAAAALNJAAAAA|2451453|2451543|DEPARTMENT|24|39|Square, widespread skills get very inner, good types; lucky, primary result|quarterly| +2524|AAAAAAAAMNJAAAAA|2451453|2451543|DEPARTMENT|24|40|Delegates will meet apparently from a magistrates. For example available wome|quarterly| +2525|AAAAAAAANNJAAAAA|2451453|2451543|DEPARTMENT|24|41|Subject matters could act ever from a results. Worthwhile, appropriate costs gi|quarterly| +2526|AAAAAAAAONJAAAAA|2451453|2451543|DEPARTMENT|24|42|Agricultural pupils show comparable birds. Competitive, white ide|quarterly| +2527|AAAAAAAAPNJAAAAA|2451453|2451543|DEPARTMENT|24|43|Different, large paintings take elsewhere right, mixed languages;|quarterly| +2528|AAAAAAAAAOJAAAAA|2451453|2451543|DEPARTMENT|24|44|Valid hands should encourage indirectly other leaders. Women will not take best at the pro|quarterly| +2529|AAAAAAAABOJAAAAA|2451453|2451543|DEPARTMENT|24|45|Essential, upper meetings will see also intelligent features. Crimes could e|quarterly| +2530|AAAAAAAACOJAAAAA|2451453|2451543|DEPARTMENT|24|46|Particular relations used to improve recently cultural, s|quarterly| +2531|AAAAAAAADOJAAAAA|2451453|2451543|DEPARTMENT|24|47|Short holidays move however perhaps real elements. Also other arts migh|quarterly| +2532|AAAAAAAAEOJAAAAA|2451453|2451543|DEPARTMENT|24|48|Black, current minutes would undertake abruptly very grea|quarterly| +2533|AAAAAAAAFOJAAAAA|2451453|2451543|DEPARTMENT|24|49|Ahead obvious cattle ride just tough interests. Sile|quarterly| +2534|AAAAAAAAGOJAAAAA|2451453|2451543|DEPARTMENT|24|50|Remarks could pretend approximately too usual cars. Diverse effects oug|quarterly| +2535|AAAAAAAAHOJAAAAA|2451453|2451543|DEPARTMENT|24|51|Also dead communities should talk probably known, clear reactions. Largely unique process|quarterly| +2536|AAAAAAAAIOJAAAAA|2451453|2451543|DEPARTMENT|24|52|Large, full-time careers should adapt holders. Full thousands will not help other, single |quarterly| +2537|AAAAAAAAJOJAAAAA|2451453|2451543|DEPARTMENT|24|53|Formal, central windows might give complex, welsh packages. Certainly new men u|quarterly| +2538|AAAAAAAAKOJAAAAA|2451453|2451543|DEPARTMENT|24|54|Early others let to the sites. Open tiny skills strengthen easily|quarterly| +2539|AAAAAAAALOJAAAAA|2451453|2451543|DEPARTMENT|24|55|So true minutes design other, great lands. Local, beneficial minutes warn as a whole thus|quarterly| +2540|AAAAAAAAMOJAAAAA|2451453|2451543|DEPARTMENT|24|56|Now other hours understand even. British priorities cannot leave birds. D|quarterly| +2541|AAAAAAAANOJAAAAA|2451453|2451543|DEPARTMENT|24|57|Mercifully free revenues hit in a regulations. Grimly valuable sides|quarterly| +2542|AAAAAAAAOOJAAAAA|2451453|2451543|DEPARTMENT|24|58|Readers ought to see arguments. Stars might answer in the roads. Outer, s|quarterly| +2543|AAAAAAAAPOJAAAAA|2451453|2451543|DEPARTMENT|24|59|Basic thousands would go out of a governments. Quietly avai|quarterly| +2544|AAAAAAAAAPJAAAAA|2451453|2451543|DEPARTMENT|24|60|Different groups may surrender as so irish men. Vulnerable, relative folk could not forg|quarterly| +2545|AAAAAAAABPJAAAAA|2451453|2451543|DEPARTMENT|24|61|Bones may consider close alternative, acute years. Policies enjoy always|quarterly| +2546|AAAAAAAACPJAAAAA|2451453|2451543|DEPARTMENT|24|62|Interviews would cope gradually foreign windows. Often recent lives aut|quarterly| +2547|AAAAAAAADPJAAAAA|2451453|2451543|DEPARTMENT|24|63|Various, long strings would discipline almost to a payments. Different|quarterly| +2548|AAAAAAAAEPJAAAAA|2451453|2451543|DEPARTMENT|24|64|Very royal men may happen years. Girls shall not find immediately historic|quarterly| +2549|AAAAAAAAFPJAAAAA|2451453|2451543|DEPARTMENT|24|65|As well different offenders afford in private into a hours. Weeks publish successfully on a|quarterly| +2550|AAAAAAAAGPJAAAAA|2451453|2451543|DEPARTMENT|24|66|Demonstrations transfer actually improvements. Questions maximize christian|quarterly| +2551|AAAAAAAAHPJAAAAA|2451453|2451543|DEPARTMENT|24|67|Royal resources lighten children. Surprising pools should rise enough on the models; s|quarterly| +2552|AAAAAAAAIPJAAAAA|2451453|2451543|DEPARTMENT|24|68|Companies gain minor, industrial debts. Too important teachers shall maintain|quarterly| +2553|AAAAAAAAJPJAAAAA|2451453|2451543|DEPARTMENT|24|69|Honest types commit strangely native police. Social, prime others see there great reside|quarterly| +2554|AAAAAAAAKPJAAAAA|2451453|2451543|DEPARTMENT|24|70|Other colleges ought to think suddenly for a priorities. Long sm|quarterly| +2555|AAAAAAAALPJAAAAA|2451453|2451543|DEPARTMENT|24|71|Almost common bodies consider about owners. Here financial boxes pla|quarterly| +2556|AAAAAAAAMPJAAAAA|2451453|2451543|DEPARTMENT|24|72|Equally concerned goods will use high, constitutional voters. Brown, medical touris|quarterly| +2557|AAAAAAAANPJAAAAA|2451453|2451543|DEPARTMENT|24|73|Long common words find immensely. Available changes could not leave at a attitudes. Past buildi|quarterly| +2558|AAAAAAAAOPJAAAAA|2451453|2451543|DEPARTMENT|24|74|Special, new writers will apply long. Much victorian musicians admit regardless. About |quarterly| +2559|AAAAAAAAPPJAAAAA|2451453|2451543|DEPARTMENT|24|75|Incidents must work best for a years. Even ancient documents sh|quarterly| +2560|AAAAAAAAAAKAAAAA|2451453|2451543|DEPARTMENT|24|76|German eyes used to wait financial, different crimes. Partly corresponding boards say so in t|quarterly| +2561|AAAAAAAABAKAAAAA|2451453|2451543|DEPARTMENT|24|77|Addresses find in particular for a areas. Very acciden|quarterly| +2562|AAAAAAAACAKAAAAA|2451453|2451543|DEPARTMENT|24|78|Positive, formal considerations shall go apparently political, national limits.|quarterly| +2563|AAAAAAAADAKAAAAA|2451453|2451543|DEPARTMENT|24|79|Major systems write. Established, liable expenses could say ever creative|quarterly| +2564|AAAAAAAAEAKAAAAA|2451453|2451543|DEPARTMENT|24|80|Possible conditions should wake always circumstances. More outer women afford|quarterly| +2565|AAAAAAAAFAKAAAAA|2451453|2451543|DEPARTMENT|24|81|Diplomatic, linear words face children. Greatly common feat|quarterly| +2566|AAAAAAAAGAKAAAAA|2451453|2451543|DEPARTMENT|24|82|Usually complex words must not follow as well good jewish shots. Perhaps great|quarterly| +2567|AAAAAAAAHAKAAAAA|2451453|2451543|DEPARTMENT|24|83|Groups must like ashore; notions ought to sail useful, present goods; neutral services cannot stay|quarterly| +2568|AAAAAAAAIAKAAAAA|2451453|2451543|DEPARTMENT|24|84|Military, deep cases get left, major assumptions. Only papers take en|quarterly| +2569|AAAAAAAAJAKAAAAA|2451453|2451543|DEPARTMENT|24|85|Possible groups can extend accounts. Black carers feel there by the holes. Years cannot wa|quarterly| +2570|AAAAAAAAKAKAAAAA|2451453|2451543|DEPARTMENT|24|86|Wide, little weeks find with the visitors. Expensive, l|quarterly| +2571|AAAAAAAALAKAAAAA|2451453|2451543|DEPARTMENT|24|87|Ancient, new classes can protect then for example probable buses; notions ought to go relatively n|quarterly| +2572|AAAAAAAAMAKAAAAA|2451453|2451543|DEPARTMENT|24|88|Again violent tools please sufficiently. Men tell things; costs come separately copie|quarterly| +2573|AAAAAAAANAKAAAAA|2451453|2451543|DEPARTMENT|24|89|Emotional applicants run other, final transactions; vital figures complete soft, left jobs. Dail|quarterly| +2574|AAAAAAAAOAKAAAAA|2451453|2451543|DEPARTMENT|24|90|Models take impressive groups. Easy agents make alone. Indeed main areas emphasise. Hot, str|quarterly| +2575|AAAAAAAAPAKAAAAA|2451453|2451543|DEPARTMENT|24|91|Directly added walls may take only old helpful loans. |quarterly| +2576|AAAAAAAAABKAAAAA|2451453|2451543|DEPARTMENT|24|92|Central, potential limits must perform. High, future clothes must not note|quarterly| +2577|AAAAAAAABBKAAAAA|2451453|2451543|DEPARTMENT|24|93|New, conscious keys force initially events; due voters ought to propose more during a machines; br|quarterly| +2578|AAAAAAAACBKAAAAA|2451453|2451543|DEPARTMENT|24|94|Operations talk originally full years. Flexible men op|quarterly| +2579|AAAAAAAADBKAAAAA|2451453|2451543|DEPARTMENT|24|95|Elections may fall great financial skills; branches turn social inte|quarterly| +2580|AAAAAAAAEBKAAAAA|2451453|2451543|DEPARTMENT|24|96|Great, detailed signals live more conditions. Daily, wild complaints used to find |quarterly| +2581|AAAAAAAAFBKAAAAA|2451453||DEPARTMENT|24|97|Still diverse notions exist almost immediately international pro|quarterly| +2582|AAAAAAAAGBKAAAAA|2451453|2451543|DEPARTMENT|24|98|Ill plans shall discover services. Never strong individuals like here claims. As sure vic|quarterly| +2583|AAAAAAAAHBKAAAAA|2451453|2451543|DEPARTMENT|24|99|Large, relevant years could transform local policies. Also central problems might |quarterly| +2584|AAAAAAAAIBKAAAAA|2451453|2451543|DEPARTMENT|24|100|Important criteria like european, usual children. Prisons want by a w|quarterly| +2585|AAAAAAAAJBKAAAAA|2451453|2451543|DEPARTMENT|24|101|Grey, long schools would create workers. Soviet, main wings see t|quarterly| +2586|AAAAAAAAKBKAAAAA|2451453|2451543|DEPARTMENT|24|102|Levels used to help aged a policemen. Violent differences could|quarterly| +2587|AAAAAAAALBKAAAAA|2451453|2451543|DEPARTMENT|24|103|High, dirty interests return as councils. Then light costs ought to see |quarterly| +2588|AAAAAAAAMBKAAAAA|2451453|2451543|DEPARTMENT|24|104|Years could pay hundreds. Directly inc features should use espe|quarterly| +2589|AAAAAAAANBKAAAAA|2451453|2451543|DEPARTMENT|24|105|About other studies will not look here high new objects. Cl|quarterly| +2590|AAAAAAAAOBKAAAAA|2451453|2451543|DEPARTMENT|24|106|Sizes set grand, growing factors; immediate beings afford wise, new trades. Different serv|quarterly| +2591|AAAAAAAAPBKAAAAA|2451453|2451543|DEPARTMENT|24|107|Crops match well national designs. Premises allow later from a premises. Open, present c|quarterly| +2592|AAAAAAAAACKAAAAA|2451453|2451543|DEPARTMENT|24|108|Obvious stars browse various, american books. So local ideas might increase very requ|quarterly| +2593|AAAAAAAABCKAAAAA|2451180|2451209|DEPARTMENT|25|1|Recent children finish anyway. Stories eat here gross, black names. Str|monthly| +2594|AAAAAAAACCKAAAAA|2451180|2451209|DEPARTMENT|25|2|Large, expert groups execute less parts; successful, new countries|monthly| +2595|AAAAAAAADCKAAAAA|2451180|2451209|DEPARTMENT|25|3|Items will bear yet rural grants; whole, adult levels sha|monthly| +2596|AAAAAAAAECKAAAAA|2451180|2451209|DEPARTMENT|25|4|Current children should test always also sure efforts. Onwards free ways should not go|monthly| +2597|AAAAAAAAFCKAAAAA|2451180|2451209|DEPARTMENT|25|5|Small, related names should go. Groups highlight previously as young materials. Already|monthly| +2598|AAAAAAAAGCKAAAAA|2451180|2451209|DEPARTMENT|25|6|Legs cannot concentrate suddenly all right scientific visitors. Individual |monthly| +2599|AAAAAAAAHCKAAAAA|2451180|2451209|DEPARTMENT|25|7|Curtains could identify other processes. Still vital negotiations mean as well subjects. Other, bri|monthly| +2600|AAAAAAAAICKAAAAA|2451180|2451209|DEPARTMENT|25|8|Communist officials proceed at the visits. American mothers paint again weaknesses. Usually|monthly| +2601|AAAAAAAAJCKAAAAA|2451180|2451209|DEPARTMENT|25|9|Too other abilities ought to take highly in relation t|monthly| +2602|AAAAAAAAKCKAAAAA|2451180|2451209|DEPARTMENT|25|10|Local sectors should see of course subjective words. Lives employ often small in|monthly| +2603|AAAAAAAALCKAAAAA|2451180|2451209|DEPARTMENT|25|11|Initial, following transactions offend rather similar conclusions. As loose experts |monthly| +2604|AAAAAAAAMCKAAAAA|2451180|2451209|DEPARTMENT|25|12|Rarely given skills should not influence. Surprised, independent books set; royal skill|monthly| +2605|AAAAAAAANCKAAAAA|2451180|2451209|DEPARTMENT|25|13|Straight, military flowers shall walk as straightforward, relevant w|monthly| +2606|AAAAAAAAOCKAAAAA|2451180|2451209|DEPARTMENT|25|14|Eastern, cultural days help at least. Centuries should stay as special corners. Monthly potential|monthly| +2607|AAAAAAAAPCKAAAAA|2451180|2451209|DEPARTMENT|25|15|New stones must knit also local actions. As english women might find to a words. Most inc p|monthly| +2608|AAAAAAAAADKAAAAA|2451180|2451209|DEPARTMENT|25|16|Long democrats shall remain. Lightly individual forces could n|monthly| +2609|AAAAAAAABDKAAAAA|2451180|2451209|DEPARTMENT|25|17|All financial gaps come regional, good services. Times must play teachers. Gentle |monthly| +2610|AAAAAAAACDKAAAAA|2451180|2451209|DEPARTMENT|25|18|Particularly high students drive. Lists run also to a foreigners. Earnings would car|monthly| +2611|AAAAAAAADDKAAAAA|2451180|2451209|DEPARTMENT|25|19|Never standard groups might not go existing languages. Still |monthly| +2612|AAAAAAAAEDKAAAAA|2451180|2451209|DEPARTMENT|25|20|Able, various girls will see. Western, french reports ought to assess really wealt|monthly| +2613|AAAAAAAAFDKAAAAA|2451180|2451209|DEPARTMENT|25|21|Overall, old years see alone dirty conflicts. Normal relations will be|monthly| +2614|AAAAAAAAGDKAAAAA|2451180|2451209|DEPARTMENT|25|22|Meanwhile desirable pp. may not fall overall in place of the soldiers. Closely sco|monthly| +2615|AAAAAAAAHDKAAAAA|2451180|2451209|DEPARTMENT|25|23|Unchanged levels ensure; social, other books let russian par|monthly| +2616|AAAAAAAAIDKAAAAA|2451180|2451209|DEPARTMENT|25|24|Years stand racial forces. Schools receive schools. Social piece|monthly| +2617|AAAAAAAAJDKAAAAA|2451180|2451209|DEPARTMENT|25|25|Just young things should tell just similar civil reports. Low, |monthly| +2618|AAAAAAAAKDKAAAAA|2451180|2451209|DEPARTMENT|25|26|Important campaigns may increase about other others; guilty articles form|monthly| +2619|AAAAAAAALDKAAAAA|2451180|2451209|DEPARTMENT|25|27|No longer useful days must bring so. Rapid, absolute drugs ought to get frequently |monthly| +2620|AAAAAAAAMDKAAAAA|2451180|2451209|DEPARTMENT|25|28|Processes might not buy big individual pictures. Months secure fundamental word|monthly| +2621|AAAAAAAANDKAAAAA|2451180|2451209|DEPARTMENT|25|29|Everywhere alternative times would not give. Names will not make|monthly| +2622|AAAAAAAAODKAAAAA|2451180|2451209|DEPARTMENT|25|30|Cool bombs maintain top things. Successful days make social problems. Symbols face again hotels|monthly| +2623|AAAAAAAAPDKAAAAA|2451180|2451209|DEPARTMENT|25|31|Important, big decisions check also glad, long examples. Already natio|monthly| +2624|AAAAAAAAAEKAAAAA|2451180|2451209|DEPARTMENT|25|32|Scientific, big leaders should ease main centres. Usual fee|monthly| +2625|AAAAAAAABEKAAAAA|2451180|2451209|DEPARTMENT|25|33|Original wounds could not understand sure parts. Standard, ch|monthly| +2626|AAAAAAAACEKAAAAA|2451180|2451209|DEPARTMENT|25|34|Able, large houses last useful investors; working, possible paintings might not say after the |monthly| +2627|AAAAAAAADEKAAAAA||2451209|DEPARTMENT|25|||monthly| +2628|AAAAAAAAEEKAAAAA|2451180|2451209|DEPARTMENT|25|36|Exceptional, main offices must establish really grand, civil co|monthly| +2629|AAAAAAAAFEKAAAAA|2451180|2451209|DEPARTMENT|25|37|Vulnerable types would not go. French, white polls used to balance as |monthly| +2630|AAAAAAAAGEKAAAAA|2451180|2451209|DEPARTMENT|25|38|Weak, average costs may continue much because of a t|monthly| +2631|AAAAAAAAHEKAAAAA|2451180|2451209|DEPARTMENT|25|39|Residents mention at last human tests. Major, strong kinds |monthly| +2632|AAAAAAAAIEKAAAAA|2451180|2451209|DEPARTMENT|25|40|Democratic, big police receive already in a persons. Comp|monthly| +2633|AAAAAAAAJEKAAAAA|2451180|2451209|DEPARTMENT|25|41|Faint mines explain just financial parts. Plans implement obviously left chairs. Gentlem|monthly| +2634|AAAAAAAAKEKAAAAA|2451180|2451209|DEPARTMENT|25|42|Roman parents may laugh properly pounds. Estates must not reduce|monthly| +2635|AAAAAAAALEKAAAAA|2451180|2451209|DEPARTMENT|25|43|Shareholders claim years. Also cheap objectives could not allow similar areas. Long childr|monthly| +2636|AAAAAAAAMEKAAAAA|2451180|2451209|DEPARTMENT|25|44|Desperately fresh feet suggest there shareholders. Previous children bring both act|monthly| +2637|AAAAAAAANEKAAAAA|2451180|2451209|DEPARTMENT|25|45|Communications say digital books. Special eyes may refer temporary children; descripti|monthly| +2638|AAAAAAAAOEKAAAAA|2451180|2451209|DEPARTMENT|25|46|Legitimate users would appear often contemporary, marine classes. Some|monthly| +2639|AAAAAAAAPEKAAAAA|2451180|2451209|DEPARTMENT|25|47|Budgets must take now full characteristics. Good standards must prevent wholly primary, britis|monthly| +2640|AAAAAAAAAFKAAAAA|2451180|2451209|DEPARTMENT|25|48|Words used to change miles. Wide provisions ought to|monthly| +2641|AAAAAAAABFKAAAAA|2451180|2451209|DEPARTMENT|25|49|Civil, old libraries might eat figures. True teams achieve smartly just various ho|monthly| +2642|AAAAAAAACFKAAAAA|2451180|2451209|DEPARTMENT|25|50|Cautiously common managers prevent extra foreign adults. |monthly| +2643|AAAAAAAADFKAAAAA|2451180|2451209|DEPARTMENT|25|51|Players used to establish strategic, impossible wives. Major, full days|monthly| +2644|AAAAAAAAEFKAAAAA|2451180|2451209|DEPARTMENT|25|52|Police tell again. Men cool else at a forms. Rarely original concepts say days. Sure|monthly| +2645|AAAAAAAAFFKAAAAA|2451180|2451209|DEPARTMENT|25|53|Other, great books must look political methods. Solid strings used to talk instead social dema|monthly| +2646|AAAAAAAAGFKAAAAA|2451180|2451209|DEPARTMENT|25|54|British hands support formally democratic applications. Again black approaches |monthly| +2647|AAAAAAAAHFKAAAAA|2451180|2451209|DEPARTMENT|25|55|Burning, natural points may form in a years; public restrictions will feel distinctly r|monthly| +2648|AAAAAAAAIFKAAAAA|2451180|2451209|DEPARTMENT|25|56|National, live elections could not stress later great |monthly| +2649|AAAAAAAAJFKAAAAA|2451180|2451209|DEPARTMENT|25|57|Odd guidelines might not see very only full parties. Features welcome current, separa|monthly| +2650|AAAAAAAAKFKAAAAA|2451180|2451209|DEPARTMENT|25|58|Nearby, divine areas demonstrate just. Also top words speed very especially appropriate policies. F|monthly| +2651|AAAAAAAALFKAAAAA|2451180|2451209|DEPARTMENT|25|59|Hardly great eyes challenge today political, imperial streets. Things would not fly new, keen ty|monthly| +2652|AAAAAAAAMFKAAAAA|2451180|2451209|DEPARTMENT|25|60|That is dead gifts support as well relevant walls. Never nuclear women heal false, go|monthly| +2653|AAAAAAAANFKAAAAA|2451180|2451209|DEPARTMENT|25|61|Strategic, chief interviews know regional rooms. Always small numbers|monthly| +2654|AAAAAAAAOFKAAAAA|2451180|2451209|DEPARTMENT|25|62|Professional images must minimise no doubt mad teachers. Then rich years |monthly| +2655|AAAAAAAAPFKAAAAA|2451180|2451209|DEPARTMENT|25|63|Most illegal arguments return. Expensive workers used to deve|monthly| +2656|AAAAAAAAAGKAAAAA|2451180|2451209|DEPARTMENT|25|64|Other hands follow panels. Prices test human board|monthly| +2657|AAAAAAAABGKAAAAA|2451180|2451209|DEPARTMENT|25|65|Perfect, scientific comparisons go yesterday from a women. Remote, clear characters make ever |monthly| +2658|AAAAAAAACGKAAAAA|2451180|2451209|DEPARTMENT|25|66|Systematically english applications promise enough interior respo|monthly| +2659|AAAAAAAADGKAAAAA|2451180|2451209|DEPARTMENT|25|67|Victorian skills will not raise also in a fields. Old issues become wi|monthly| +2660|AAAAAAAAEGKAAAAA|2451180|2451209|DEPARTMENT|25|68|Small teams would not disappear instead so histori|monthly| +2661|AAAAAAAAFGKAAAAA|2451180|2451209|DEPARTMENT|25|69|Overall local units long either settings. Nevertheless huge women used to pu|monthly| +2662|AAAAAAAAGGKAAAAA|2451180|2451209|DEPARTMENT|25|70|Just single changes ought to defeat somehow tall, narrow mist|monthly| +2663|AAAAAAAAHGKAAAAA|2451180|2451209|DEPARTMENT|25|71|Unpleasant, available teachers marry often certain |monthly| +2664|AAAAAAAAIGKAAAAA|2451180|2451209|DEPARTMENT|25|72|Great societies should not finish no longer. More numerous members would not call belo|monthly| +2665|AAAAAAAAJGKAAAAA|2451180|2451209|DEPARTMENT|25|73|New, cognitive customers catch. Old, social reactions think else previous funds; limits will not w|monthly| +2666|AAAAAAAAKGKAAAAA|2451180|2451209|DEPARTMENT|25|74|Ready, dangerous effects achieve then; nearly necessary co|monthly| +2667|AAAAAAAALGKAAAAA|2451180|2451209|DEPARTMENT|25|75|Entire, charming calculations shout. Trees ought to move to a fi|monthly| +2668|AAAAAAAAMGKAAAAA|2451180|2451209|DEPARTMENT|25|76|Brief fingers obey even here good needs; always heavy studies could not demonstrate in|monthly| +2669|AAAAAAAANGKAAAAA|2451180|2451209|DEPARTMENT|25|77|Dreams call immense orders; languages seem widely favorable payments. Front, |monthly| +2670|AAAAAAAAOGKAAAAA|2451180|2451209|DEPARTMENT|25|78|Further personal decades could not find far public premises. Eyes use miles. Runnin|monthly| +2671|AAAAAAAAPGKAAAAA|2451180|2451209|DEPARTMENT|25|79|Vehicles might watch in order wide, late things. Obvious ends must shoot forever levels.|monthly| +2672|AAAAAAAAAHKAAAAA|2451180|2451209|DEPARTMENT|25|80|Directly eastern boys would congratulate general, necessary|monthly| +2673|AAAAAAAABHKAAAAA|2451180|2451209|DEPARTMENT|25|81|Merely modest studies ought to assume nearly sometimes warm police. Well concrete goals could|monthly| +2674|AAAAAAAACHKAAAAA|2451180|2451209|DEPARTMENT|25|82|Also upper demands must not contain able links. Great customers would give pre|monthly| +2675|AAAAAAAADHKAAAAA|2451180|2451209|DEPARTMENT|25|83|Difficult, contemporary notes eat as. Intelligent users tackle eyes. Right, crazy word|monthly| +2676|AAAAAAAAEHKAAAAA|2451180|2451209|DEPARTMENT|25|84|Extra, accessible managers cause only just significant wishes. Theories take mor|monthly| +2677|AAAAAAAAFHKAAAAA|2451180|2451209|DEPARTMENT|25|85|Brown, regular models promote as only other men; ancient rocks shal|monthly| +2678|AAAAAAAAGHKAAAAA|2451180|2451209|DEPARTMENT|25|86|Traditional provisions suffer neat, important difficulties. Never traditional gam|monthly| +2679|AAAAAAAAHHKAAAAA|2451180|2451209|DEPARTMENT|25|87|Now chemical practitioners can keep in a ways. Social, old numbers might not call later|monthly| +2680|AAAAAAAAIHKAAAAA|2451180|2451209|DEPARTMENT|25|88|Representatives might select all apparently urgent minutes. Children collec|monthly| +2681|AAAAAAAAJHKAAAAA|2451180|2451209|DEPARTMENT|25|89|Underlying, professional methods extend at all from|monthly| +2682|AAAAAAAAKHKAAAAA|2451180|2451209|DEPARTMENT|25|90|Years compensate also again interesting miles. Part|monthly| +2683|AAAAAAAALHKAAAAA|2451180|2451209|DEPARTMENT|25|91|Purely rich comments may not save too drinks. Ill se|monthly| +2684|AAAAAAAAMHKAAAAA|2451180|2451209|DEPARTMENT|25|92|Small, sophisticated stars should generalize else corporate, complete controls; early change|monthly| +2685|AAAAAAAANHKAAAAA|2451180|2451209|DEPARTMENT|25|93|Other, shared eyes explain here less than complete areas. Ideal, old manag|monthly| +2686|AAAAAAAAOHKAAAAA|2451180|2451209|DEPARTMENT|25|94|Programmes realise old employees. Candidates must need among a things; courses |monthly| +2687|AAAAAAAAPHKAAAAA|2451180|2451209|DEPARTMENT|25|95|Meanwhile dear others get difficult parents. Miles walk in order books. Nationa|monthly| +2688|AAAAAAAAAIKAAAAA|2451180|2451209|DEPARTMENT|25|96|Late necessary lawyers try always hot, professional manufacturers. Sli|monthly| +2689|AAAAAAAABIKAAAAA|2451180|2451209|DEPARTMENT|25|97|Public, cool comments shall use quite small, perfect circles. Strict wi|monthly| +2690|AAAAAAAACIKAAAAA|2451180|2451209|DEPARTMENT|25|98|Players keep labour friends. Unknown points would take infinitely now medical modules. Materia|monthly| +2691|AAAAAAAADIKAAAAA|2451180|2451209|DEPARTMENT|25|99|Labour officers consider originally violent, elderly items; huge teachers work earlier differe|monthly| +2692|AAAAAAAAEIKAAAAA|2451180|2451209|DEPARTMENT|25|100|Consistent girls move always from the areas; green, living hous|monthly| +2693|AAAAAAAAFIKAAAAA|2451180|2451209|DEPARTMENT|25|101|Briefly distinguished vehicles should not know too presen|monthly| +2694|AAAAAAAAGIKAAAAA|2451180|2451209|DEPARTMENT|25|102|Equivalent, old interactions stick so ready, sure pensions. Too local authorities must fi|monthly| +2695|AAAAAAAAHIKAAAAA|2451180|2451209|DEPARTMENT|25|103|Friends shall find. Players ignore yet. Costs change then letters. Waste powers ought to include |monthly| +2696|AAAAAAAAIIKAAAAA|2451180|2451209|DEPARTMENT|25|104|Other, endless authorities would not await in no police; at last potential problems work fund|monthly| +2697|AAAAAAAAJIKAAAAA|2451180|2451209|DEPARTMENT|25|105|Real, fatal groups support perhaps finally important types. Poin|monthly| +2698|AAAAAAAAKIKAAAAA|2451180|2451209|DEPARTMENT|25|106|Other, high flights lose much academic variations; pleased factors shall|monthly| +2699|AAAAAAAALIKAAAAA|2451180|2451209|DEPARTMENT|25|107|Gaps acquire lawyers. Types throw now; economic minutes used to find however. Patterns|monthly| +2700|AAAAAAAAMIKAAAAA|2451180|2451209|DEPARTMENT|25|108|Again clear addresses will not lead there estimated precious companies. Little rules shall not |monthly| +2701|AAAAAAAANIKAAAAA|2451210|2451239|DEPARTMENT|26|1|Nurses remove more early, bitter children. Relatives will make usually. Reall|monthly| +2702|AAAAAAAAOIKAAAAA|2451210|2451239|DEPARTMENT|26|2|Inner, strong steps should end however almost private lips. Across social voic|monthly| +2703|AAAAAAAAPIKAAAAA|2451210|2451239|DEPARTMENT|26|3|Students get to the effects. Powerful members hear below in the lines. |monthly| +2704|AAAAAAAAAJKAAAAA|2451210|2451239|DEPARTMENT|26|4|Parliamentary eyes may not keep old questions. Profound, beautiful job|monthly| +2705|AAAAAAAABJKAAAAA|2451210|2451239|DEPARTMENT|26|5|Free objectives happen natural clothes. However little sente|monthly| +2706|AAAAAAAACJKAAAAA|2451210|2451239|DEPARTMENT|26|6|Very holy managers survive nevertheless. Changes must persuade. Effectiv|monthly| +2707|AAAAAAAADJKAAAAA|2451210|2451239|DEPARTMENT|26|7|Efficient, related days used to think only. Anywhere technical centres may sink always massive, |monthly| +2708|AAAAAAAAEJKAAAAA|2451210|2451239|DEPARTMENT|26|8|Mean, young rights go so in a areas. Successful clients ought to ban di|monthly| +2709|AAAAAAAAFJKAAAAA|2451210|2451239|DEPARTMENT|26|9|Visual months should achieve for a workshops. Most new losses may not keep impatiently items.|monthly| +2710|AAAAAAAAGJKAAAAA|2451210|2451239|DEPARTMENT|26|10|Keen things like very other, fiscal assumptions. Leading, national courts oug|monthly| +2711|AAAAAAAAHJKAAAAA|2451210|2451239|DEPARTMENT|26|11|Numbers explain never shy, inc shows. Green aspects could|monthly| +2712|AAAAAAAAIJKAAAAA|2451210|||||Dry, necessary holes used to believe direct, national|| +2713|AAAAAAAAJJKAAAAA|2451210|2451239|DEPARTMENT|26|13|Also serious cars work points. Countries insure no|monthly| +2714|AAAAAAAAKJKAAAAA|2451210|2451239|DEPARTMENT|26|14|More comfortable lines could worsen twice social, prominent payments. S|monthly| +2715|AAAAAAAALJKAAAAA|2451210|2451239|DEPARTMENT|26|15|Cases cannot go ago prime, various wars. Domestic, old police happen. Public, light sc|monthly| +2716|AAAAAAAAMJKAAAAA|2451210|2451239|DEPARTMENT|26|16|As usual small lights would not go things. Weekly, massive p|monthly| +2717|AAAAAAAANJKAAAAA|2451210|2451239|DEPARTMENT|26|17|Publicly direct facilities save tight pictures; also single characters come very new p|monthly| +2718|AAAAAAAAOJKAAAAA|2451210|2451239|DEPARTMENT|26|18|Memories wield quite both strong brothers; much pers|monthly| +2719|AAAAAAAAPJKAAAAA|2451210|2451239|DEPARTMENT|26|19|Women should go blue, lesser positions. Rapid, white circumstances glimpse too great, p|monthly| +2720|AAAAAAAAAKKAAAAA|2451210|2451239|DEPARTMENT|26|20|Both obvious cells used to offer naturally; items might act as particular materials. Agents be|monthly| +2721|AAAAAAAABKKAAAAA|2451210|2451239|DEPARTMENT|26|21|International women leave also small devices. Awful proceedings should discredit often unfortun|monthly| +2722|AAAAAAAACKKAAAAA|2451210|2451239|DEPARTMENT|26|22|A bit russian eyes go days. Left, real rivers shall pop against a eggs. Accep|monthly| +2723|AAAAAAAADKKAAAAA|2451210|2451239|DEPARTMENT|26|23|Smart, soft conclusions like particular, similar inter|monthly| +2724|AAAAAAAAEKKAAAAA|2451210|2451239|DEPARTMENT|26|24|Religious, primary months may perform; british, interior schools may stay. Serious, d|monthly| +2725|AAAAAAAAFKKAAAAA|||||25|Events get again advanced services. Separate periods speed virtuall|monthly| +2726|AAAAAAAAGKKAAAAA|2451210|2451239|DEPARTMENT|26|26|Perfect, difficult resources should dream further with the times; other studies |monthly| +2727|AAAAAAAAHKKAAAAA|2451210|2451239|DEPARTMENT|26|27|Ready, false answers matter more by a servants. Trying months explain alwa|monthly| +2728|AAAAAAAAIKKAAAAA|2451210|2451239|DEPARTMENT|26|28|Different, powerful firms examine however sufficient, fellow b|monthly| +2729|AAAAAAAAJKKAAAAA|2451210|2451239|DEPARTMENT|26|29|More quiet proposals used to show. So other services |monthly| +2730|AAAAAAAAKKKAAAAA|2451210|2451239|DEPARTMENT|26|30|Maybe right women receive local, considerable months. Usually national parties buy within a risks.|monthly| +2731|AAAAAAAALKKAAAAA|2451210|2451239|DEPARTMENT|26|31|Available houses lie pp.. Enemies can commemorate all together obvious awards. Busy expectat|monthly| +2732|AAAAAAAAMKKAAAAA|2451210|2451239|DEPARTMENT|26|32|Holy regions arise limitations. Previously beautiful fr|monthly| +2733|AAAAAAAANKKAAAAA|2451210|2451239|DEPARTMENT|26|33|Comfortable partners order amazing, unique areas. Scottish shops would |monthly| +2734|AAAAAAAAOKKAAAAA|2451210|2451239|DEPARTMENT|26|34|Even formal workers want more. Social, other eyes should |monthly| +2735|AAAAAAAAPKKAAAAA|2451210|2451239|DEPARTMENT|26|35|Now very ways may like hard organisational pattern|monthly| +2736|AAAAAAAAALKAAAAA|2451210|2451239|DEPARTMENT|26|36|Essential, pregnant forces should afford carefully; particularly variable stages d|monthly| +2737|AAAAAAAABLKAAAAA|2451210|2451239|DEPARTMENT|26|37|Difficult, liberal feet claim. So practical children describe in addit|monthly| +2738|AAAAAAAACLKAAAAA|2451210|2451239|DEPARTMENT|26|38|Measures will bring others. Subject police should not know yet important systems. Well ultim|monthly| +2739|AAAAAAAADLKAAAAA|2451210|2451239|DEPARTMENT|26|39|Perceptions welcome to the men. Short annual messages ought |monthly| +2740|AAAAAAAAELKAAAAA|2451210|2451239|DEPARTMENT|26|40|New, immediate teachers join also personal audiences. Little, ser|monthly| +2741|AAAAAAAAFLKAAAAA|2451210|2451239|DEPARTMENT|26|41|Profits cannot think different errors. Techniques change even. Almost|monthly| +2742|AAAAAAAAGLKAAAAA|2451210|2451239|DEPARTMENT|26|42|Visual, written minutes feel fellow activities. Effects prove exactly domestic, pure st|monthly| +2743|AAAAAAAAHLKAAAAA|2451210|2451239|DEPARTMENT|26|43|Walls sum actual companies. Economic reasons get; dramatic fees love behind the men. Bet|monthly| +2744|AAAAAAAAILKAAAAA|2451210|2451239|DEPARTMENT|26|44|Never necessary wages hear high, large studies. Simple events reform particularly;|monthly| +2745|AAAAAAAAJLKAAAAA|2451210|2451239|DEPARTMENT|26|45|Sites freeze potentially different farmers. Famous, paren|monthly| +2746|AAAAAAAAKLKAAAAA|2451210|2451239|DEPARTMENT|26|46|Wide, adequate men could see still previous, necessary firms; elaborate things write forever di|monthly| +2747|AAAAAAAALLKAAAAA|2451210|2451239|DEPARTMENT|26|47|Homes see very alone, only designers. Isolated, black police cost exch|monthly| +2748|AAAAAAAAMLKAAAAA|2451210|2451239|DEPARTMENT|26|48|Autonomous seeds used to go always empty, little minutes; effective others|monthly| +2749|AAAAAAAANLKAAAAA|2451210|2451239|DEPARTMENT|26|49|As small honours encourage electoral, african deaths. Eviden|monthly| +2750|AAAAAAAAOLKAAAAA|2451210|2451239|DEPARTMENT|26|50|New males could adopt; priests equip over in a subjects. United, big systems host easi|monthly| +2751|AAAAAAAAPLKAAAAA|2451210|2451239|DEPARTMENT|26|51|Typically european taxes imply finally by a ideas. Western, bad changes mu|monthly| +2752|AAAAAAAAAMKAAAAA|2451210|2451239|DEPARTMENT|26|52|Also waiting studies take partly too aware objectives. E|monthly| +2753|AAAAAAAABMKAAAAA|2451210|2451239|DEPARTMENT|26|53|High, white duties direct honestly limited, short nurses. Of course clear trees stud|monthly| +2754|AAAAAAAACMKAAAAA|2451210|2451239|DEPARTMENT|26|54|Carers must save more available groups. Organisational rights sell dynamic foods. Full p|monthly| +2755|AAAAAAAADMKAAAAA|2451210|2451239|DEPARTMENT|26|55|Changes see theoretically happy things. British, simple pp. should not get up to|monthly| +2756|AAAAAAAAEMKAAAAA|2451210|2451239|DEPARTMENT|26|56|Else different steps choose very into a talks. Even|monthly| +2757|AAAAAAAAFMKAAAAA|2451210|2451239|DEPARTMENT|26|57|Daily, entire charges support original, similar mat|monthly| +2758|AAAAAAAAGMKAAAAA|2451210|2451239|DEPARTMENT|26|58|Rates come knowingly priests. Now young markets experience at last. Of course clear|monthly| +2759|AAAAAAAAHMKAAAAA|2451210|2451239|DEPARTMENT|26|59|Much key standards give increasing nations. Only proposed windows go local|monthly| +2760|AAAAAAAAIMKAAAAA|2451210|2451239|DEPARTMENT|26|60|Upper, comprehensive properties may realise so simple |monthly| +2761|AAAAAAAAJMKAAAAA|2451210|2451239|DEPARTMENT|26|61|Fierce years shall say about a children. Groups raise. Also unable hours want never rational wa|monthly| +2762|AAAAAAAAKMKAAAAA|2451210|2451239|DEPARTMENT|26|62|Well severe years should mean so applications. There emotional circumstances protect. Often o|monthly| +2763|AAAAAAAALMKAAAAA|2451210|2451239|DEPARTMENT|26|63|New rooms begin. Away royal years used to end to a |monthly| +2764|AAAAAAAAMMKAAAAA|2451210|2451239|DEPARTMENT|26|64|Especially new audiences will improve tonight practica|monthly| +2765|AAAAAAAANMKAAAAA|2451210|2451239|DEPARTMENT|26|65|Attempts arise currently recent, special managers; soviet words ought to undermine |monthly| +2766|AAAAAAAAOMKAAAAA|2451210|2451239|DEPARTMENT|26|66|Early, dear woods will not keep details. Now working communities give appropriate, certain bars|monthly| +2767|AAAAAAAAPMKAAAAA|2451210|2451239|DEPARTMENT|26|67|Years need more suddenly back rates. There moral babies must not cooperate so good european recor|monthly| +2768|AAAAAAAAANKAAAAA|2451210|2451239|DEPARTMENT|26|68|Below naval letters join red feet. Apparently public excha|monthly| +2769|AAAAAAAABNKAAAAA|2451210|2451239|DEPARTMENT|26|69|Drivers should get once again. Recent lists may carry comfortably. Civil, labour recommendations |monthly| +2770|AAAAAAAACNKAAAAA|2451210|2451239|DEPARTMENT|26|70|Vital, pleasant notions must develop most unlikely boys. White shops must supply in the m|monthly| +2771|AAAAAAAADNKAAAAA|2451210|2451239|DEPARTMENT|26|71|Very literary patients used to summon at a materials. Labour, good eyes|monthly| +2772|AAAAAAAAENKAAAAA|2451210|2451239|DEPARTMENT|26|72|Peaceful, powerful forests put very famous women; often economic sections shall cost over|monthly| +2773|AAAAAAAAFNKAAAAA|2451210|2451239|DEPARTMENT|26|73|Australian, public units will arrange especially poor things. E|monthly| +2774|AAAAAAAAGNKAAAAA|2451210|2451239|DEPARTMENT|26|74|Objectives used to provide here during a minutes. Unique windo|monthly| +2775|AAAAAAAAHNKAAAAA|2451210|2451239|DEPARTMENT|26|75|Months argue very classic meetings. More than working agents can delive|monthly| +2776|AAAAAAAAINKAAAAA|2451210|2451239|DEPARTMENT|26|76|Categories roll. Important, national changes will ensure more possible institutions. Old levels i|monthly| +2777|AAAAAAAAJNKAAAAA|2451210|2451239|DEPARTMENT|26|77|Subsidies form more fixed, serious types. Holidays shall matter only externa|monthly| +2778|AAAAAAAAKNKAAAAA|2451210|2451239|DEPARTMENT|26|78|Able plans fathom more than vehicles. Sufficient, concerned men contact in the times. Much|monthly| +2779|AAAAAAAALNKAAAAA|2451210|2451239|DEPARTMENT|26|79|Pieces would provide there; about complex types thank. Here required lads must not elect l|monthly| +2780|AAAAAAAAMNKAAAAA|2451210|2451239|DEPARTMENT|26|80|Commitments make for a losses. Particular, special |monthly| +2781|AAAAAAAANNKAAAAA|2451210|2451239|DEPARTMENT|26|81|Mental, substantial novels will join possible ideas. Future cheeks can fight very russian birds|monthly| +2782|AAAAAAAAONKAAAAA||||26|||monthly| +2783|AAAAAAAAPNKAAAAA|2451210|2451239|DEPARTMENT|26|83|Available stages must not throw against the users; likely friends should not feel ill |monthly| +2784|AAAAAAAAAOKAAAAA|2451210|2451239|DEPARTMENT|26|84|Brown lives tell today. Other, high proportions give never ec|monthly| +2785|AAAAAAAABOKAAAAA|2451210|2451239|DEPARTMENT|26|85|Specific, only affairs affect prices. However new windows consider by a requirements. Dire|monthly| +2786|AAAAAAAACOKAAAAA|2451210|2451239|DEPARTMENT|26|86|Complete, available countries will pursue groups. Fo|monthly| +2787|AAAAAAAADOKAAAAA|2451210|2451239|DEPARTMENT|26|87|Full, dynamic feet arrive managers. Mutual, different exper|monthly| +2788|AAAAAAAAEOKAAAAA|2451210|2451239|DEPARTMENT|26|88|As confident things must bend before impossible successive routes. Private, appropriate childr|monthly| +2789|AAAAAAAAFOKAAAAA|2451210|2451239|DEPARTMENT|26|89|Gradually similar institutions know yet good years. Again rare tactics enabl|monthly| +2790|AAAAAAAAGOKAAAAA|2451210|2451239|DEPARTMENT|26|90|Then certain miles maintain levels. Grounds develop. Judges l|monthly| +2791|AAAAAAAAHOKAAAAA|2451210|2451239|DEPARTMENT|26|91|Minds reflect naturally special firms. Golden, surprised laws might not run meetings.|monthly| +2792|AAAAAAAAIOKAAAAA|2451210|2451239|DEPARTMENT|26|92|Nevertheless international issues could not transmit standard years. Curren|monthly| +2793|AAAAAAAAJOKAAAAA|2451210|2451239|DEPARTMENT|26|93|Pounds will not complete ashore then direct demands. Others make also|monthly| +2794|AAAAAAAAKOKAAAAA|2451210|2451239|DEPARTMENT|26|94|Reduced materials can work fine, new boys. Rather econom|monthly| +2795|AAAAAAAALOKAAAAA|2451210|2451239|DEPARTMENT|26|95|Benefits show as constant agents; pictures ought t|monthly| +2796|AAAAAAAAMOKAAAAA|2451210|2451239|DEPARTMENT|26|96|Slightly international arms could rate supreme lines; modern, other |monthly| +2797|AAAAAAAANOKAAAAA|2451210|2451239|DEPARTMENT|26|97|Symbolic customers teach. Old achievements ought to come anyway only prospective men. Ever aware |monthly| +2798|AAAAAAAAOOKAAAAA|2451210|2451239|DEPARTMENT|26|98|Sessions can remember again strong, various books. Che|monthly| +2799|AAAAAAAAPOKAAAAA|2451210|2451239|DEPARTMENT|26|99|Open facilities guess yellow ears. Extra, double c|monthly| +2800|AAAAAAAAAPKAAAAA|2451210|2451239|DEPARTMENT|26|100|Police prevent continental, obvious eggs. Pilots must play|monthly| +2801|AAAAAAAABPKAAAAA|2451210|2451239|DEPARTMENT|26|101|Commercial, good organisers test around wild, straight decades. Li|monthly| +2802|AAAAAAAACPKAAAAA|2451210|2451239|DEPARTMENT|26|102|For instance high details ought to see so personal, germa|monthly| +2803|AAAAAAAADPKAAAAA|2451210|2451239|DEPARTMENT|26|103|Even suitable groups should not mend coloured hands; general, equal countries consider |monthly| +2804|AAAAAAAAEPKAAAAA|2451210|2451239|DEPARTMENT|26|104|Difficult, appropriate islands might tell. Differently economic procedures improv|monthly| +2805|AAAAAAAAFPKAAAAA|2451210|2451239|DEPARTMENT|26|105|Attacks may not go strange, known services. Young areas offer in|monthly| +2806|AAAAAAAAGPKAAAAA|2451210|2451239|DEPARTMENT|26|106|Per annum other occupations ought to provide consi|monthly| +2807|AAAAAAAAHPKAAAAA|2451210|2451239|DEPARTMENT|26|107|Easy children might steal open, female groups. Tired databases go now sharp relat|monthly| +2808|AAAAAAAAIPKAAAAA|2451210|2451239|DEPARTMENT|26|108|Organisations shall occur indeed just black hands.|monthly| +2809|AAAAAAAAJPKAAAAA|2451240|2451269|DEPARTMENT|27|1|Correct communities could provide blue, difficult things. Common effects|monthly| +2810|AAAAAAAAKPKAAAAA|2451240|2451269|DEPARTMENT|27|2|Obvious, external children will not account small, foreign seats|monthly| +2811|AAAAAAAALPKAAAAA|2451240|2451269|DEPARTMENT|27|3|Free, basic reactions should enjoy too aware servants; still local patients move ever compon|monthly| +2812|AAAAAAAAMPKAAAAA|2451240|2451269|DEPARTMENT|27|4|More active years might retrieve so through a results. Bright things plunge generally surel|monthly| +2813|AAAAAAAANPKAAAAA|2451240|2451269|DEPARTMENT|27|5|British books prevent usually important subjects. Games may cha|monthly| +2814|AAAAAAAAOPKAAAAA|2451240|2451269|DEPARTMENT|27|6|Even fine signals would pick even foreign other limitations. Here other aff|monthly| +2815|AAAAAAAAPPKAAAAA|2451240|2451269|DEPARTMENT|27|7|Payable animals ought to ensure about. Opposite projects shou|monthly| +2816|AAAAAAAAAALAAAAA|2451240|2451269|DEPARTMENT|27|8|Christian, universal boys die as. Familiar, different ships i|monthly| +2817|AAAAAAAABALAAAAA|2451240|2451269|DEPARTMENT|27|9|Free companies must not grow. Doctors install private options. Ministers should obscure sales. Qu|monthly| +2818|AAAAAAAACALAAAAA|2451240|2451269|DEPARTMENT|27|10|Attractive colours bow sharply pupils; almost indep|monthly| +2819|AAAAAAAADALAAAAA|2451240|2451269|DEPARTMENT|27|11|Consciously small wines shall satisfy nice, confident advisers. Groups reduce alleged, |monthly| +2820|AAAAAAAAEALAAAAA|2451240|2451269|DEPARTMENT|27|12|Perfect, rare theories make twice boards; continuous dollars woul|monthly| +2821|AAAAAAAAFALAAAAA|2451240|2451269|DEPARTMENT|27|13|Alone, great records would not try. Now total courts might not ask fully a|monthly| +2822|AAAAAAAAGALAAAAA|2451240|2451269|DEPARTMENT|27|14|Modes take there again red tools. As intimate legs cannot feel around sort of experienced l|monthly| +2823|AAAAAAAAHALAAAAA|2451240|2451269|DEPARTMENT|27|15|Also able tests make parts. Primary, able companies find |monthly| +2824|AAAAAAAAIALAAAAA|2451240|2451269|DEPARTMENT|27|16|Patiently nuclear laws shall fight heavily hard reasons. Up to persistent meetin|monthly| +2825|AAAAAAAAJALAAAAA|2451240|2451269|DEPARTMENT|27|17|Still heavy lines would think in a areas. Provincial laws must not enabl|monthly| +2826|AAAAAAAAKALAAAAA|2451240|2451269|DEPARTMENT|27|18|Double, obvious directions could take precisely children. Rather right items may report m|monthly| +2827|AAAAAAAALALAAAAA|2451240|2451269|DEPARTMENT|27|19|Annual values feed henceforth invariably general daughters. Scholars shall |monthly| +2828|AAAAAAAAMALAAAAA|2451240|||27|20|Years go sideways thoroughly beautiful members. Far days might|monthly| +2829|AAAAAAAANALAAAAA|2451240|2451269|DEPARTMENT|27|21|Industries distinguish late immediate, particular cards. Extra, re|monthly| +2830|AAAAAAAAOALAAAAA|2451240|2451269|DEPARTMENT|27|22|Faces should dwell now true, different authorities; true, economic fans overturn so to an|monthly| +2831|AAAAAAAAPALAAAAA|2451240|2451269|DEPARTMENT|27|23|Certain, poor days smile all successful services. Rooms cannot favour just; skilled|monthly| +2832|AAAAAAAAABLAAAAA|2451240|2451269|DEPARTMENT|27|24|Auditors lead hard studies. Free schemes should trace entirely others. Ruling times|monthly| +2833|AAAAAAAABBLAAAAA|2451240|2451269|DEPARTMENT|27|25|Australian, other companies contain problems. French things sh|monthly| +2834|AAAAAAAACBLAAAAA|2451240|2451269|DEPARTMENT|27|26|Social offences may admit ago classical unions. Financial years may not wake howe|monthly| +2835|AAAAAAAADBLAAAAA|2451240|2451269|DEPARTMENT|27|27|Hard personal parents find young, british conditions. Ri|monthly| +2836|AAAAAAAAEBLAAAAA|2451240|2451269|DEPARTMENT|27|28|Readers might look suddenly important, italian features; parts make. Investors migh|monthly| +2837|AAAAAAAAFBLAAAAA|2451240|2451269|DEPARTMENT|27|29|New sides know hard in the individuals; wide, possible thousands might take so; workers wi|monthly| +2838|AAAAAAAAGBLAAAAA|2451240|2451269|DEPARTMENT|27|30|Other others see here women. Given times used to last equally certain seeds. So normal po|monthly| +2839|AAAAAAAAHBLAAAAA|2451240|2451269|DEPARTMENT|27|31|More alone profits should not believe in respect of |monthly| +2840|AAAAAAAAIBLAAAAA|2451240|2451269|DEPARTMENT|27|32|Psychological, close effects promote again to a senses. Categories carry objects. Curre|monthly| +2841|AAAAAAAAJBLAAAAA|2451240|2451269|DEPARTMENT|27|33|Traditional leaves must remain very pretty vertical conditions. Wide times may enable; eastern,|monthly| +2842|AAAAAAAAKBLAAAAA|2451240|2451269|DEPARTMENT|27|34|As secondary details think further. Often able problems shall not exploit arguments; then natio|monthly| +2843|AAAAAAAALBLAAAAA|2451240|2451269|DEPARTMENT|27|35|Old, political friends cover industrial, broad men. Wrong failures can grant socia|monthly| +2844|AAAAAAAAMBLAAAAA|2451240|2451269|DEPARTMENT|27|36|Cats incorporate new, long requirements. Adverse relatives |monthly| +2845|AAAAAAAANBLAAAAA|2451240|2451269|DEPARTMENT|27|37|Social, front reasons ought to roll particular, sensitive terms. Effective characters |monthly| +2846|AAAAAAAAOBLAAAAA|2451240|2451269|DEPARTMENT|27|38|Yet overall sounds would reduce others. Central, full attitudes will think of|monthly| +2847|AAAAAAAAPBLAAAAA|2451240|2451269|DEPARTMENT|27|39|Effective, weak stages go forward soon national competitors. However ready|monthly| +2848|AAAAAAAAACLAAAAA|2451240|2451269|DEPARTMENT|27|40|Red, united daughters know ever low years. Hours channel simply short, major bodi|monthly| +2849|AAAAAAAABCLAAAAA|2451240|2451269|DEPARTMENT|27|41|Good years embark remote, political guns. Factors would grow also existing connectio|monthly| +2850|AAAAAAAACCLAAAAA|2451240|2451269|DEPARTMENT|27|42|Very distinct prisons attract actors. There fundamental elements write observers. Then real pic|monthly| +2851|AAAAAAAADCLAAAAA|2451240|2451269|DEPARTMENT|27|43|Areas visit legal, absolute advantages. Grey, light bodies used to think children. Particular co|monthly| +2852|AAAAAAAAECLAAAAA|2451240|2451269|DEPARTMENT|27|44|Very high heads must not feel years; now spanish circles will occur quite; jewish fami|monthly| +2853|AAAAAAAAFCLAAAAA|2451240|2451269|DEPARTMENT|27|45|Windows will stop yesterday potential grounds. Capital, international languages may|monthly| +2854|AAAAAAAAGCLAAAAA|2451240|2451269|DEPARTMENT|27|46|Monetary women go instead among the profits. Judicial, visual t|monthly| +2855|AAAAAAAAHCLAAAAA|2451240|2451269|DEPARTMENT|27|47|Sounds rest women. Eyes could receive however purposes; rounds may install to a years; cen|monthly| +2856|AAAAAAAAICLAAAAA|2451240|2451269|DEPARTMENT|27|48|Plans shall not smoke frequent, ordinary banks. Crucial, |monthly| +2857|AAAAAAAAJCLAAAAA|2451240|2451269|DEPARTMENT|27|49|Marginal, senior missiles back normal, greek futures; final ways tru|monthly| +2858|AAAAAAAAKCLAAAAA|2451240|2451269|DEPARTMENT|27|50|Hours strike either in a gaps. Simply important centres help. New ends must ensure so old hu|monthly| +2859|AAAAAAAALCLAAAAA|2451240|2451269|DEPARTMENT|27|51|Equations should manage never; better industrial eyebrows coul|monthly| +2860|AAAAAAAAMCLAAAAA|2451240|2451269|DEPARTMENT|27|52|Details might make guests. Today certain securities shall get nevertheless bea|monthly| +2861|AAAAAAAANCLAAAAA|2451240|2451269|DEPARTMENT|27|53|Equally vertical drinks emphasize public, automatic arms. Categories go well alone remai|monthly| +2862|AAAAAAAAOCLAAAAA|2451240|2451269|DEPARTMENT|27|54|Able, aware newspapers shall transfer anyway seconds. Quie|monthly| +2863|AAAAAAAAPCLAAAAA|2451240|2451269|DEPARTMENT|27|55|So possible relations may slip also other changing levels. Corporate, dynamic compani|monthly| +2864|AAAAAAAAADLAAAAA|2451240|2451269|DEPARTMENT|27|56|However particular circumstances ought to make hard|monthly| +2865|AAAAAAAABDLAAAAA|2451240|2451269|DEPARTMENT|27|57|Easily local things go still urban, genuine areas. Thoughtfully modern |monthly| +2866|AAAAAAAACDLAAAAA|2451240|2451269|DEPARTMENT|27|58|Substantial, marginal questions feel now. More complete methods must implement as yet in a months.|monthly| +2867|AAAAAAAADDLAAAAA|2451240|2451269|DEPARTMENT|27|59|Names shall not supply in a feet. Atomic others would meet aside dangerous conditions. Case|monthly| +2868|AAAAAAAAEDLAAAAA|2451240|2451269|DEPARTMENT|27|60|More large changes can see homes. English, small proposals help. Hundreds try for a mice. Main, mad|monthly| +2869|AAAAAAAAFDLAAAAA|2451240|2451269|DEPARTMENT|27|61|Western values would not decrease very capital members. As new fingers ma|monthly| +2870|AAAAAAAAGDLAAAAA|2451240|2451269|DEPARTMENT|27|62|Certainly general employees could not stop for example minutes|monthly| +2871|AAAAAAAAHDLAAAAA|2451240|2451269|DEPARTMENT|27|63|Asleep, symbolic steps might not provoke currently in a sons; new users used to leav|monthly| +2872|AAAAAAAAIDLAAAAA|2451240|2451269|DEPARTMENT|27|64|Odd sheets ought to write to the earnings. Pupils h|monthly| +2873|AAAAAAAAJDLAAAAA|2451240|2451269|DEPARTMENT|27|65|Crazy, civil pieces cause just royal months. Times should agree so happy sales. Metres ensure|monthly| +2874|AAAAAAAAKDLAAAAA|2451240|2451269|DEPARTMENT|27|66|Authorities achieve rightly private measures; early th|monthly| +2875|AAAAAAAALDLAAAAA|2451240|2451269|DEPARTMENT|27|67|Terrible influences sign sympathetic, extensive cl|monthly| +2876|AAAAAAAAMDLAAAAA|2451240|2451269|DEPARTMENT|27|68|New wives make probably as new missiles. Supporters win with a are|monthly| +2877|AAAAAAAANDLAAAAA|2451240|2451269|DEPARTMENT|27|69|New worlds shall move as troops. Connections live yet. Greatl|monthly| +2878|AAAAAAAAODLAAAAA|2451240|2451269|DEPARTMENT|27|70|Right, urban areas stay only in a magazines. Human, difficult signs follo|monthly| +2879|AAAAAAAAPDLAAAAA|2451240|2451269|DEPARTMENT|27|71|Little, national proceedings can produce most other reasons; elegantly |monthly| +2880|AAAAAAAAAELAAAAA||||27|||| +2881|AAAAAAAABELAAAAA|2451240|2451269|DEPARTMENT|27|73|Questions shall expect secret, good payments. Massive, mixed hou|monthly| +2882|AAAAAAAACELAAAAA|2451240|2451269|DEPARTMENT|27|74|As well direct patients get then in the deals; natural tim|monthly| +2883|AAAAAAAADELAAAAA|2451240|2451269|DEPARTMENT|27|75|Thousands will not exist politicians. Young teachers could not come youngsters.|monthly| +2884|AAAAAAAAEELAAAAA|2451240|2451269|DEPARTMENT|27|76|Really ready trusts must ignore now private bold sectors. Then |monthly| +2885|AAAAAAAAFELAAAAA|2451240|2451269|DEPARTMENT|27|77|Respectively other colleagues must come various, un|monthly| +2886|AAAAAAAAGELAAAAA|2451240|2451269|DEPARTMENT|27|78|Also common details could not keep also final, white beliefs. Blue letter|monthly| +2887|AAAAAAAAHELAAAAA|2451240|2451269|DEPARTMENT|27|79|Short, top problems might not welcome. Firms shall make recently alone countrie|monthly| +2888|AAAAAAAAIELAAAAA|2451240|2451269|DEPARTMENT|27|80|Young, proud numbers avoid remarkably centres. Complex criteri|monthly| +2889|AAAAAAAAJELAAAAA|2451240|2451269|DEPARTMENT|27|81|Home special schools trace so. Instantly payable reasons might not lo|monthly| +2890|AAAAAAAAKELAAAAA|2451240|2451269|DEPARTMENT|27|82|Already main colonies keep generally sweet, technical bodies. Signific|monthly| +2891|AAAAAAAALELAAAAA|2451240|2451269|DEPARTMENT|27|83|Educational towns should not entail never adult ends. Spanish blocks blame s|monthly| +2892|AAAAAAAAMELAAAAA|2451240|2451269|DEPARTMENT|27|84|Profits must save briefly modern, main places; clear investments pay |monthly| +2893|AAAAAAAANELAAAAA|2451240|2451269|DEPARTMENT|27|85|Seats used to avoid strong techniques. Harsh clouds will seem. Jobs s|monthly| +2894|AAAAAAAAOELAAAAA|2451240|2451269|DEPARTMENT|27|86|National prices ought to comply. There spiritual eye|monthly| +2895|AAAAAAAAPELAAAAA|2451240|2451269|DEPARTMENT|27|87|Functional locations may not produce throughout a farms. Clearly |monthly| +2896|AAAAAAAAAFLAAAAA|2451240|2451269|DEPARTMENT|27|88|National fragments might not include neighbouring, afraid|monthly| +2897|AAAAAAAABFLAAAAA|2451240|2451269|DEPARTMENT|27|89|Sure legs use from a facilities; able days should come more successful projects. Grounds continue |monthly| +2898|AAAAAAAACFLAAAAA|2451240|2451269|DEPARTMENT|27|90|Shared, economic trials keep even available drugs. Strong women cannot produce during a obligat|monthly| +2899|AAAAAAAADFLAAAAA|2451240|2451269|DEPARTMENT|27|91|Safe problems appear most whole objectives. English, new shareholders could t|monthly| +2900|AAAAAAAAEFLAAAAA|2451240|2451269|DEPARTMENT|27|92|Friends go. Possible, effective times feel and so on in the publications. Foreign, fashionable r|monthly| +2901|AAAAAAAAFFLAAAAA|2451240|2451269|DEPARTMENT|27|93|Relationships shall guarantee great. Low sanctions may not use for th|monthly| +2902|AAAAAAAAGFLAAAAA|2451240|2451269|DEPARTMENT|27|94|Jewish, other officers believe most above tiny vehicles. Details shall fish so. Really|monthly| +2903|AAAAAAAAHFLAAAAA|2451240|2451269|DEPARTMENT|27|95|Different needs get often clear, free offices; consequen|monthly| +2904|AAAAAAAAIFLAAAAA|2451240|2451269|DEPARTMENT|27|96|Adequate, heavy countries shall not communicate perhaps problems. Furious, english leaders ough|monthly| +2905|AAAAAAAAJFLAAAAA|2451240|2451269|DEPARTMENT|27|97|Materials ought to say grudgingly questions. Able, fresh student|monthly| +2906|AAAAAAAAKFLAAAAA|2451240|2451269|DEPARTMENT|27|98|Creative months throw little up to a clubs. About good|monthly| +2907|AAAAAAAALFLAAAAA|2451240|2451269|DEPARTMENT|27|99|British, sexual statements like pensioners. Also horrible trousers could land more ea|monthly| +2908|AAAAAAAAMFLAAAAA|2451240|2451269|DEPARTMENT|27|100|Necessary assessments should bear proteins; terms ma|monthly| +2909|AAAAAAAANFLAAAAA|2451240|2451269|DEPARTMENT|27|101|Available concentrations work changes. Real, rigid ways could make assessments. Aback vul|monthly| +2910|AAAAAAAAOFLAAAAA|2451240|2451269|DEPARTMENT|27|102|Grants lose over trends. Unfortunately clinical effects need more than remaining games; unabl|monthly| +2911|AAAAAAAAPFLAAAAA|2451240|2451269|DEPARTMENT|27|103|Types acknowledge for a events. International, red |monthly| +2912|AAAAAAAAAGLAAAAA||2451269|DEPARTMENT|27|104|Soviet deaths cross countries; nuclear farmers accept. True savings cannot confirm as well year|monthly| +2913|AAAAAAAABGLAAAAA|2451240|2451269|DEPARTMENT|27|105|Old manufacturers get never personal failures. Surely tiny|monthly| +2914|AAAAAAAACGLAAAAA|2451240|||27|106||| +2915|AAAAAAAADGLAAAAA|2451240|2451269|DEPARTMENT|27|107|New, small conditions know in a symptoms. Directly important places would need so women. Childre|monthly| +2916|AAAAAAAAEGLAAAAA|2451240|2451269|DEPARTMENT|27|108|Hundreds shall want sheets. Especially representative sec|monthly| +2917|AAAAAAAAFGLAAAAA|2451270|2451299|DEPARTMENT|28|1|So large parts will give vital, other years. Carefully living mont|monthly| +2918|AAAAAAAAGGLAAAAA|2451270|2451299|DEPARTMENT|28|2|Young, level decisions dance safely scientific ideas. Certain, particular ti|monthly| +2919|AAAAAAAAHGLAAAAA|2451270|2451299|DEPARTMENT|28|3|Still general weapons might not impart now pictures. Maybe managerial function|monthly| +2920|AAAAAAAAIGLAAAAA|2451270|2451299|DEPARTMENT|28|4|Horses use there even increasing investors. Double, clear weeks go. Names can aff|monthly| +2921|AAAAAAAAJGLAAAAA|2451270|2451299|DEPARTMENT|28|5|Hundreds must pay directors. Really heavy services would tell with a countries. General impl|monthly| +2922|AAAAAAAAKGLAAAAA|2451270|2451299|DEPARTMENT|28|6|Critical players speak finely sessions. Occasions kill perhaps big, small m|monthly| +2923|AAAAAAAALGLAAAAA|2451270|2451299|DEPARTMENT|28|7|Troops shake appropriate, leading circumstances. Simple, temporary standards let even in a |monthly| +2924|AAAAAAAAMGLAAAAA|2451270|2451299|DEPARTMENT|28|8|Complex markets eat bright, short governments; pink measure|monthly| +2925|AAAAAAAANGLAAAAA|2451270|2451299|DEPARTMENT|28|9|Good, major standards know future commitments. Partly brief regions supervise for instan|monthly| +2926|AAAAAAAAOGLAAAAA|2451270|2451299|DEPARTMENT|28|10|Expensive doors shall persuade. Very british companies|monthly| +2927|AAAAAAAAPGLAAAAA|2451270|2451299|DEPARTMENT|28|11|Local, other drivers can supply now great nations; numbers|monthly| +2928|AAAAAAAAAHLAAAAA|2451270|2451299|DEPARTMENT|28|12|Really other rates shall not say nowhere able, likely teeth. Wooden, larg|monthly| +2929|AAAAAAAABHLAAAAA|2451270|2451299|DEPARTMENT|28|13|Fires must not use more shareholders. Surely other american|monthly| +2930|AAAAAAAACHLAAAAA|2451270|2451299|DEPARTMENT|28|14|More dangerous things stop early local, fun things; social funds would not elim|monthly| +2931|AAAAAAAADHLAAAAA|2451270|2451299|DEPARTMENT|28|15|Spare imports may not cover so short devices. Negotiations know |monthly| +2932|AAAAAAAAEHLAAAAA|2451270|2451299|DEPARTMENT|28|16|In addition long stars meet really for example sick|monthly| +2933|AAAAAAAAFHLAAAAA|2451270|2451299|DEPARTMENT|28|17|Scientific, new reasons perform bad personnel. Autonomous, diverse|monthly| +2934|AAAAAAAAGHLAAAAA|2451270|2451299|DEPARTMENT|28|18|Safe observations calculate either clear reductions; single years should not counteract o|monthly| +2935|AAAAAAAAHHLAAAAA|2451270|2451299|DEPARTMENT|28|19|Great intervals understand available, interested bags. Short societies get. Behind possibl|monthly| +2936|AAAAAAAAIHLAAAAA|2451270|2451299|DEPARTMENT|28|20|Most long eyes shall not reject feelings. Features find so in a|monthly| +2937|AAAAAAAAJHLAAAAA|2451270|2451299|DEPARTMENT|28|21|Passive services ought to see else necessary, collective bills. Difficulties cannot p|monthly| +2938|AAAAAAAAKHLAAAAA|2451270|2451299|DEPARTMENT|28|22|Systems would not work still monthly, open problems. Statements used t|monthly| +2939|AAAAAAAALHLAAAAA|2451270|2451299|DEPARTMENT|28|23|Even executive eyes cheer rarely public owners. Heads must run now heavy, importa|monthly| +2940|AAAAAAAAMHLAAAAA|2451270|2451299|DEPARTMENT|28|24|Today contemporary resources take for example stages|monthly| +2941|AAAAAAAANHLAAAAA|2451270|2451299|DEPARTMENT|28|25|Inappropriate, recent characters learn for a shops. Soviet things walk copies. Never de|monthly| +2942|AAAAAAAAOHLAAAAA|2451270|2451299|DEPARTMENT|28|26|Questions used to permit. Likely, thorough improvements shall |monthly| +2943|AAAAAAAAPHLAAAAA|2451270|2451299|DEPARTMENT|28|27|Yet clean positions collect local, true reports; privileges cou|monthly| +2944|AAAAAAAAAILAAAAA|2451270|2451299|DEPARTMENT|28|28|Fine, national hours fall also ashamed visitors; light years assess too. Local jour|monthly| +2945|AAAAAAAABILAAAAA|2451270|2451299|DEPARTMENT|28|29|Orders would use professional authorities. Different, present devices concede very competent, |monthly| +2946|AAAAAAAACILAAAAA|2451270|2451299|DEPARTMENT|28|30|Dangerous forms must get. Recently small cups teach for the branches. Sales used to force |monthly| +2947|AAAAAAAADILAAAAA|2451270|2451299|DEPARTMENT|28|31|Centuries shall think inside. Local years protect chief, l|monthly| +2948|AAAAAAAAEILAAAAA|2451270|2451299|DEPARTMENT|28|32|Public memories note often correct parties. Generally small cell|monthly| +2949|AAAAAAAAFILAAAAA|2451270|2451299|DEPARTMENT|28|33|More soviet employees ought to accept traditionall|monthly| +2950|AAAAAAAAGILAAAAA|2451270|2451299|DEPARTMENT|28|34|Railways would mention chiefly only students. All available weeks maintain definitely british mont|monthly| +2951|AAAAAAAAHILAAAAA|2451270|2451299|DEPARTMENT|28|35|Interior stores wish. Young terms happen then wrong shareholders. Educational, new others wil|monthly| +2952|AAAAAAAAIILAAAAA|2451270|||||Committees may get quite services. Well-known scientists will not dig now small stairs|| +2953|AAAAAAAAJILAAAAA|2451270|2451299|DEPARTMENT|28|37|Various elections think serious, normal forms. Sug|monthly| +2954|AAAAAAAAKILAAAAA|2451270|2451299|DEPARTMENT|28|38|English, ultimate types see slim, other men. So famous protest|monthly| +2955|AAAAAAAALILAAAAA|2451270|2451299|DEPARTMENT|28|39|Long nice attacks enter particularly here full weeks|monthly| +2956|AAAAAAAAMILAAAAA|2451270|2451299|DEPARTMENT|28|40|Excellent men meet for instance light stairs. Advanced sales less|monthly| +2957|AAAAAAAANILAAAAA|2451270|2451299|DEPARTMENT|28|41|Waiting, left samples might help as particularly new cats. Together individual |monthly| +2958|AAAAAAAAOILAAAAA|2451270|2451299|DEPARTMENT|28|42|Bonds choose however far other horses. Different surfaces|monthly| +2959|AAAAAAAAPILAAAAA|2451270|2451299|DEPARTMENT|28|43|Pleasant conferences ensure more agencies. Young, individual teachers g|monthly| +2960|AAAAAAAAAJLAAAAA|2451270|2451299|DEPARTMENT|28|44|Teachers write restrictions; private, political neighbours|monthly| +2961|AAAAAAAABJLAAAAA|2451270|2451299|DEPARTMENT|28|45|Passive, responsible values increase so to a heads. Easy concerned leaders might arrange. |monthly| +2962|AAAAAAAACJLAAAAA||2451299|DEPARTMENT|28|||| +2963|AAAAAAAADJLAAAAA|2451270|2451299|DEPARTMENT|28|47|Computers think very please mere symptoms. Economic|monthly| +2964|AAAAAAAAEJLAAAAA|2451270|2451299|DEPARTMENT|28|48|Public, local stages become well dull, irish proposals. Natural, poor fathe|monthly| +2965|AAAAAAAAFJLAAAAA|2451270|2451299|DEPARTMENT|28|49|Sufficiently pregnant firms live above harsh clouds. Local, parliamentary records preven|monthly| +2966|AAAAAAAAGJLAAAAA|2451270|2451299|DEPARTMENT|28|50|Late phenomena should believe pockets. As hard figures woul|monthly| +2967|AAAAAAAAHJLAAAAA|2451270|2451299|DEPARTMENT|28|51|Classes used to pick for good payable, scientific observations; also geographical systems might n|monthly| +2968|AAAAAAAAIJLAAAAA|2451270|2451299|DEPARTMENT|28|52|Local parents may grasp great, competitive approaches. There tired stages wi|monthly| +2969|AAAAAAAAJJLAAAAA|2451270|2451299|DEPARTMENT|28|53|Doctors would help immediately pictures. Activities instruct just. Modules turn short pr|monthly| +2970|AAAAAAAAKJLAAAAA|2451270|2451299|DEPARTMENT|28|54|Other politicians used to counter communications. Major years call below obviously complex years. |monthly| +2971|AAAAAAAALJLAAAAA|2451270|2451299|DEPARTMENT|28|55|Rare women can calculate also final streets. Errors clean very still dutch drawi|monthly| +2972|AAAAAAAAMJLAAAAA|2451270|2451299|DEPARTMENT|28|56|Undoubtedly teenage principles must not undertake large, likely practices. Only little practic|monthly| +2973|AAAAAAAANJLAAAAA|2451270|2451299|DEPARTMENT|28|57|Possible others ought to result cars. Nevertheless rich boys ple|monthly| +2974|AAAAAAAAOJLAAAAA|2451270|2451299|DEPARTMENT|28|58|So large troubles shall render general, just sanctions. Fr|| +2975|AAAAAAAAPJLAAAAA|2451270|2451299|DEPARTMENT|28|59|Very unlikely clubs accomplish most foreign service|monthly| +2976|AAAAAAAAAKLAAAAA|2451270|2451299|DEPARTMENT|28|60|Then main diseases mean ever to a structures. Regional, ind|monthly| +2977|AAAAAAAABKLAAAAA|2451270|2451299|DEPARTMENT|28|61|Local, usual supporters specify disabled experiments. Once equal families could not m|monthly| +2978|AAAAAAAACKLAAAAA|2451270|2451299|DEPARTMENT|28|62|Foreign members replace grimly about fixed years. Weekly, substantial charac|monthly| +2979|AAAAAAAADKLAAAAA|2451270|2451299|DEPARTMENT|28|63|Rights take. Glad, overseas parents choose well too established courses; deep m|monthly| +2980|AAAAAAAAEKLAAAAA|2451270|2451299|DEPARTMENT|28|64|Years co-operate typical agencies. Securities evaluate times. Able |monthly| +2981|AAAAAAAAFKLAAAAA|2451270|2451299|DEPARTMENT|28|65|Chief, intense months would not lie then upstairs familiar days. Properties used to see more|monthly| +2982|AAAAAAAAGKLAAAAA|2451270|2451299|DEPARTMENT|28|66|Again extra boys relieve new, poor priorities. Suitable, empty component|monthly| +2983|AAAAAAAAHKLAAAAA|2451270|2451299|DEPARTMENT|28|67|Then other years would not analyse yesterday in between a eyes. Surfaces return|monthly| +2984|AAAAAAAAIKLAAAAA|2451270|2451299|DEPARTMENT|28|68|Fatal, christian purposes live simply more than identi|monthly| +2985|AAAAAAAAJKLAAAAA|2451270|2451299|DEPARTMENT|28|69|Problems might not need as on a followers. Total, internat|monthly| +2986|AAAAAAAAKKLAAAAA|2451270|2451299|DEPARTMENT|28|70|Main threats cannot provide certain banks. Suspicious, capable years flush only only socia|monthly| +2987|AAAAAAAALKLAAAAA|2451270|2451299|DEPARTMENT|28|71|Rich, horrible penalties used to survive now expected|monthly| +2988|AAAAAAAAMKLAAAAA|2451270|2451299|DEPARTMENT|28|72|Suddenly likely branches negotiate here; vertical, old rounds might not hear. N|monthly| +2989|AAAAAAAANKLAAAAA|2451270|2451299|DEPARTMENT|28|73|Valuable, traditional years must make only, local regions; substantial w|monthly| +2990|AAAAAAAAOKLAAAAA||2451299|DEPARTMENT|28||Valuable children use there general supporters. Hostile officers reac|monthly| +2991|AAAAAAAAPKLAAAAA|2451270|2451299|DEPARTMENT|28|75|Dead, current shops ought to tackle both financial, historic measures. Controls|monthly| +2992|AAAAAAAAALLAAAAA|2451270|2451299|DEPARTMENT|28|76|Equal twins play quite to the words. Customers tell cells; |monthly| +2993|AAAAAAAABLLAAAAA|2451270|2451299|DEPARTMENT|28|77|More new cases would not take roughly museums. More bad contracts choose however then spe|monthly| +2994|AAAAAAAACLLAAAAA|2451270|2451299|DEPARTMENT|28|78|Available, new parameters hide however. Clearly good parents may raise far|monthly| +2995|AAAAAAAADLLAAAAA|2451270|2451299|DEPARTMENT|28|79|Thus young universities find distinctive, other girls. Reliable holidays would help wel|monthly| +2996|AAAAAAAAELLAAAAA|2451270|2451299|DEPARTMENT|28|80|Differences love primarily familiar concentrations. Both open clo|monthly| +2997|AAAAAAAAFLLAAAAA|2451270|2451299|DEPARTMENT|28|81|Large, available perceptions might not rely much similar circumstances|monthly| +2998|AAAAAAAAGLLAAAAA|2451270|2451299|DEPARTMENT|28|82|Military, real users might not promote. Fatal months pick within a models; so large columns make |monthly| +2999|AAAAAAAAHLLAAAAA|2451270|2451299|DEPARTMENT|28|83|Gross, nice scientists live partly. Local, special schools could|monthly| +3000|AAAAAAAAILLAAAAA|2451270|2451299|DEPARTMENT|28|84|Numerous, old recordings might look. Serious, major cards like smartly artificial pu|monthly| +3001|AAAAAAAAJLLAAAAA|2451270|2451299|DEPARTMENT|28|85|Colleagues reinforce. Still important classes might know public, other |monthly| +3002|AAAAAAAAKLLAAAAA|2451270|2451299|DEPARTMENT|28|86|Gothic, specific pp. make never inevitable arrangements.|monthly| +3003|AAAAAAAALLLAAAAA|2451270|2451299|DEPARTMENT|28|87|Real reductions shall range high, cultural relations. Best ge|monthly| +3004|AAAAAAAAMLLAAAAA|2451270|2451299|DEPARTMENT|28|88|Powerful, large ladies should add. Alternative users will show still left years. Longer g|monthly| +3005|AAAAAAAANLLAAAAA|2451270|2451299|DEPARTMENT|28|89|Values act only nuclear earnings. Good trends would feel just st|monthly| +3006|AAAAAAAAOLLAAAAA|2451270|2451299|DEPARTMENT|28|90|Else large residents might stay then in a stages; separate, important co|monthly| +3007|AAAAAAAAPLLAAAAA|2451270|2451299|DEPARTMENT|28|91|Local, human friends live so net, central conditions. Environmental, long departm|monthly| +3008|AAAAAAAAAMLAAAAA|2451270|2451299|DEPARTMENT|28|92|Together large friends may attack all over detailed pp.. Natio|monthly| +3009|AAAAAAAABMLAAAAA|2451270|2451299|DEPARTMENT|28|93|Now total standards may visit recently too other effects. Possible, great tests come all special|monthly| +3010|AAAAAAAACMLAAAAA|2451270|2451299|DEPARTMENT|28|94|Also main purposes advance really for a senses. Women ought to argue blue, subjective authori|monthly| +3011|AAAAAAAADMLAAAAA|2451270|2451299|DEPARTMENT|28|95|Satisfied, unlikely buildings go towards a activities. Involuntarily top sisters contribu|monthly| +3012|AAAAAAAAEMLAAAAA|2451270|2451299|DEPARTMENT|28|96|Beautifully good networks get. Then stable contents reduce as |monthly| +3013|AAAAAAAAFMLAAAAA|2451270|2451299|DEPARTMENT|28|97|Secondly high times could live too simple visible times. Minutes may not say. Too poss|monthly| +3014|AAAAAAAAGMLAAAAA|2451270|2451299|DEPARTMENT|28|98|Much modern companies may not use steadily about the days. Temporary, dead weapons tell now als|monthly| +3015|AAAAAAAAHMLAAAAA|2451270|2451299|DEPARTMENT|28|99|Simply presidential months should not accept qualities. So good arrangements l|monthly| +3016|AAAAAAAAIMLAAAAA|2451270|2451299|DEPARTMENT|28|100|Grounds used to inform international funds. Single|monthly| +3017|AAAAAAAAJMLAAAAA|2451270|2451299|DEPARTMENT|28|101|Different, bad notes should develop also rather than a views. New reports |monthly| +3018|AAAAAAAAKMLAAAAA|2451270|2451299|DEPARTMENT|28|102|Guards remain slightly simply new bags. Strict doctors could|monthly| +3019|AAAAAAAALMLAAAAA|2451270|2451299|DEPARTMENT|28|103|Quick, reliable protests quote with a years. Very back y|monthly| +3020|AAAAAAAAMMLAAAAA|2451270|2451299|DEPARTMENT|28|104|Images get more also new eyes. Clear, dead families prevent patien|monthly| +3021|AAAAAAAANMLAAAAA|2451270|2451299|DEPARTMENT|28|105|Human, decisive years seek properly open, european bottles. Inevitably easte|monthly| +3022|AAAAAAAAOMLAAAAA|2451270|2451299|DEPARTMENT|28|106|Inherent, full attitudes used to control ready miles. Alternative judge|monthly| +3023|AAAAAAAAPMLAAAAA|2451270|2451299|DEPARTMENT|28|107|Only old members can figure signs. Domestic, previous lecture|monthly| +3024|AAAAAAAAANLAAAAA|2451270|2451299|DEPARTMENT|28|108|Sure, clear differences meet handsome holes. Societies h|monthly| +3025|AAAAAAAABNLAAAAA|2451300|2451329|DEPARTMENT|29|1|Interested institutions would not follow again great, private germans. Things involve with|monthly| +3026|AAAAAAAACNLAAAAA|2451300|2451329|DEPARTMENT|29|2|Fair qualifications get from the courses. Cases make large fields. V|monthly| +3027|AAAAAAAADNLAAAAA|2451300|2451329|DEPARTMENT|29|3|Then full problems hear about social women. Gently additional|monthly| +3028|AAAAAAAAENLAAAAA|2451300|2451329|DEPARTMENT|29|4|Even previous parts make functions. Industrial disciplines ask only days. Little trains cannot w|monthly| +3029|AAAAAAAAFNLAAAAA|2451300|2451329|DEPARTMENT|29|5|Cases might eat probably banks. Red, multiple elements realise explicitly sa|monthly| +3030|AAAAAAAAGNLAAAAA|2451300|2451329|DEPARTMENT|29|6|Answers should know then better financial parts; that annual wo|monthly| +3031|AAAAAAAAHNLAAAAA|2451300|2451329|DEPARTMENT|29|7|Then colonial eyes occur only powers. Languages seem very usually parliamentary lines. Full, re|monthly| +3032|AAAAAAAAINLAAAAA|2451300|2451329|DEPARTMENT|29|8|More final houses would assist. Companies should assume|monthly| +3033|AAAAAAAAJNLAAAAA|2451300|2451329|DEPARTMENT|29|9|Yesterday british farmers might accept more than long-term worlds. Too local designers would not|monthly| +3034|AAAAAAAAKNLAAAAA|2451300|2451329|DEPARTMENT|29|10|Working, short teachers take in order from a eyes. Recent sorts reach at all. Different, democr|monthly| +3035|AAAAAAAALNLAAAAA|2451300|2451329|DEPARTMENT|29|11|Essential, arab schools get secondly even dead measures; conflicts make |monthly| +3036|AAAAAAAAMNLAAAAA|2451300|2451329|DEPARTMENT|29|12|Databases might not want there big, intelligent stocks. Little |monthly| +3037|AAAAAAAANNLAAAAA|2451300|2451329|DEPARTMENT|29|13|Still ideal preferences might cast. Very distinct months ough|monthly| +3038|AAAAAAAAONLAAAAA|2451300|2451329|DEPARTMENT|29|14|Local instruments rear here easily sufficient imports. Special, good eggs must pr|monthly| +3039|AAAAAAAAPNLAAAAA|2451300|2451329|DEPARTMENT|29|15|Obvious, aware rights require tonight rather than a |monthly| +3040|AAAAAAAAAOLAAAAA|2451300|2451329|DEPARTMENT|29|16|Residents play also social increases. Single mammals|monthly| +3041|AAAAAAAABOLAAAAA|2451300|2451329|DEPARTMENT|29|17|In particular whole results arrange only other women. Then human soldiers t|monthly| +3042|AAAAAAAACOLAAAAA|2451300|2451329|DEPARTMENT|29|18|Words must not look at the files. Soviet, certain defences might invite ch|monthly| +3043|AAAAAAAADOLAAAAA|2451300|2451329|DEPARTMENT|29|19|Other islands used to stay therefore voluntary, pro|monthly| +3044|AAAAAAAAEOLAAAAA|2451300|2451329|DEPARTMENT|29|20|Classical, red defendants go also high, guilty roads. Days would complain still meals. National ey|monthly| +3045|AAAAAAAAFOLAAAAA|2451300|2451329|DEPARTMENT|29|21|Too european stages would stop in full offices; mostly lar|monthly| +3046|AAAAAAAAGOLAAAAA|2451300|2451329|DEPARTMENT|29|22|Children might make in a details. Waves may miss today necessary, large|monthly| +3047|AAAAAAAAHOLAAAAA|2451300|2451329|DEPARTMENT|29|23|Jobs accept straight wild, industrial stages. Only communist |monthly| +3048|AAAAAAAAIOLAAAAA|2451300|2451329|DEPARTMENT|29|24|Then recent police improve dramatically often black ele|monthly| +3049|AAAAAAAAJOLAAAAA|2451300|2451329|DEPARTMENT|29|25|Roles write also sad players; national, strong years |monthly| +3050|AAAAAAAAKOLAAAAA|2451300|2451329|DEPARTMENT|29|26|Social fields go natural, confident products. Only|monthly| +3051|AAAAAAAALOLAAAAA|2451300|2451329|DEPARTMENT|29|27|Top, dead contributions ought to open political criticisms; catholic p|monthly| +3052|AAAAAAAAMOLAAAAA|2451300|2451329|DEPARTMENT|29|28|Also only shares see new, impressive schemes. Researchers understand ever political|monthly| +3053|AAAAAAAANOLAAAAA|2451300|2451329|DEPARTMENT|29|29|Different studies reassure outside forces. Poor miles cannot choose n|monthly| +3054|AAAAAAAAOOLAAAAA|2451300|2451329|DEPARTMENT|29|30|High, special schemes should attempt. At all australian terms should|monthly| +3055|AAAAAAAAPOLAAAAA|2451300|2451329|DEPARTMENT|29|31|Russian, close holders could stop therefore guests; relationships meet long alternative class|monthly| +3056|AAAAAAAAAPLAAAAA|2451300|2451329|DEPARTMENT|29|32|Much able subjects must break long. British lips support on the errors; old lives ou|monthly| +3057|AAAAAAAABPLAAAAA|2451300|2451329|DEPARTMENT|29|33|Central observations can develop just about rough influences. Eur|monthly| +3058|AAAAAAAACPLAAAAA|2451300|2451329|DEPARTMENT|29|34|Main, cultural players depend wide types. Courts will |monthly| +3059|AAAAAAAADPLAAAAA|2451300|2451329|DEPARTMENT|29|35|Weeks would exploit then aware, new positions. Available problems reduce there females. Parts p|monthly| +3060|AAAAAAAAEPLAAAAA|2451300|2451329|DEPARTMENT|29|36|Strong, early rights form. Channels could result ever. Facts|monthly| +3061|AAAAAAAAFPLAAAAA|2451300|2451329|DEPARTMENT|29|37|National tories create too no longer young worlds. Blue feet shall make able thin|monthly| +3062|AAAAAAAAGPLAAAAA|2451300|2451329|DEPARTMENT|29|38|Issues may win still poor authorities. Often subject situations might improve u|monthly| +3063|AAAAAAAAHPLAAAAA|2451300|2451329|DEPARTMENT|29|39|High facilities guess. Local, equal men protect then practica|monthly| +3064|AAAAAAAAIPLAAAAA|2451300|2451329|DEPARTMENT|29|40|Again concerned points travel. Young suppliers get old problems; plans go upwards sons.|monthly| +3065|AAAAAAAAJPLAAAAA|2451300|2451329|DEPARTMENT|29|41|Opening systems cannot consolidate thus new drugs. Further interested effects grant companies. |monthly| +3066|AAAAAAAAKPLAAAAA|2451300|2451329|DEPARTMENT|29|42|Effortlessly extra teams may not stare less. Swiss |monthly| +3067|AAAAAAAALPLAAAAA|2451300|2451329|DEPARTMENT|29|43|Less good houses must expect taxes. Openly only engines will not develop tests|monthly| +3068|AAAAAAAAMPLAAAAA|2451300|2451329|DEPARTMENT|29|44|Normal colleagues would matter of course to a bags; |monthly| +3069|AAAAAAAANPLAAAAA|2451300|2451329|DEPARTMENT|29|45|Ideas cure possible, large police. Readily financial children may not note discussi|monthly| +3070|AAAAAAAAOPLAAAAA|2451300|2451329|DEPARTMENT|29|46|Forces climb today pleasant images; foreign bones lose e|monthly| +3071|AAAAAAAAPPLAAAAA|2451300|2451329|DEPARTMENT|29|47|Hardly growing comments would become proposals; before quiet othe|monthly| +3072|AAAAAAAAAAMAAAAA|2451300|2451329|DEPARTMENT|29|48|Political, monetary rates find tiny, available others; social windows shall read however mo|monthly| +3073|AAAAAAAABAMAAAAA|2451300|2451329|DEPARTMENT|29|49|Political cells swim enough particular concerns. National, national magistrates sel|monthly| +3074|AAAAAAAACAMAAAAA|2451300|2451329|DEPARTMENT|29|50|Comprehensive, various millions shall quote. Real girls would|monthly| +3075|AAAAAAAADAMAAAAA|2451300|2451329|DEPARTMENT|29|51|Full changes will paint arts. Police ought to return |monthly| +3076|AAAAAAAAEAMAAAAA|2451300|2451329|DEPARTMENT|29|52|Groups find further blue generations. Pilots would not see here old specialists. Other, traditional|monthly| +3077|AAAAAAAAFAMAAAAA|2451300|2451329|DEPARTMENT|29|53|Years shall expect military meals. Rough, personal references would read specific, minor trees. S|monthly| +3078|AAAAAAAAGAMAAAAA|2451300|2451329|DEPARTMENT|29|54|Probably particular transactions fear. Servants would not win |monthly| +3079|AAAAAAAAHAMAAAAA|2451300|2451329|DEPARTMENT|29|55|Better recent centres like. Terrible, other goals pay|monthly| +3080|AAAAAAAAIAMAAAAA|2451300|2451329|DEPARTMENT|29|56|Royal streets see maybe as legal practices; women mu|monthly| +3081|AAAAAAAAJAMAAAAA|2451300|2451329|DEPARTMENT|29|57|Personal, particular positions spend connections. Members want nu|monthly| +3082|AAAAAAAAKAMAAAAA|2451300|2451329|DEPARTMENT|29|58|However new models follow again. Windows must not rain d|monthly| +3083|AAAAAAAALAMAAAAA|2451300|2451329|DEPARTMENT|29|59|Responsible, high directors respond particularly days. Conscious, ava|monthly| +3084|AAAAAAAAMAMAAAAA|2451300|2451329|DEPARTMENT|29|60|Colleges drink on a others; also global cases abandon interesting|monthly| +3085|AAAAAAAANAMAAAAA|2451300|2451329|DEPARTMENT|29|61|Anyway true kids leave imaginative, modern characteristic|monthly| +3086|AAAAAAAAOAMAAAAA|2451300|2451329||29||Nearly deaf talks seem barely written days; missiles cannot require too diplomatic pressures. |monthly| +3087|AAAAAAAAPAMAAAAA|2451300|2451329|DEPARTMENT|29|63|Better fine individuals predict small, optimistic |monthly| +3088|AAAAAAAAABMAAAAA|2451300|2451329|DEPARTMENT|29|64|Substances shall not appear goods; presently fixed payments take so ex|monthly| +3089|AAAAAAAABBMAAAAA|2451300|2451329|DEPARTMENT|29|65|Extensive orders see honestly about available advantages. Similarly obvious sec|monthly| +3090|AAAAAAAACBMAAAAA|2451300|2451329|DEPARTMENT|29|66|Mutually fond practitioners hear more intensive estimates. Plants award only|monthly| +3091|AAAAAAAADBMAAAAA|2451300|2451329|DEPARTMENT|29|67|Visible trustees stimulate quite environmental tears. Cases worry just e|monthly| +3092|AAAAAAAAEBMAAAAA|2451300|2451329|DEPARTMENT|29|68|Candidates inflict as. Soviet, european children must spend all pure letters. Light, grey ta|monthly| +3093|AAAAAAAAFBMAAAAA|2451300|2451329|DEPARTMENT|29|69|Implications drive always vague pictures; old, adequate risks in|monthly| +3094|AAAAAAAAGBMAAAAA|2451300|2451329|DEPARTMENT|29|70|Too great leaders use ago increased evenings. In general |monthly| +3095|AAAAAAAAHBMAAAAA|2451300|2451329|DEPARTMENT|29|71|Big, complete children allow vividly also additional|monthly| +3096|AAAAAAAAIBMAAAAA|2451300|2451329|DEPARTMENT|29|72|Artists will return in a books. Teams come. Rather mode|monthly| +3097|AAAAAAAAJBMAAAAA|2451300|2451329|DEPARTMENT|29|73|Typical products carry alone, due lines. American, only ter|monthly| +3098|AAAAAAAAKBMAAAAA|2451300|2451329|DEPARTMENT|29|74|Trousers reach multiple, high hands. In vain suitable companies play slow, full charg|monthly| +3099|AAAAAAAALBMAAAAA|2451300|2451329|DEPARTMENT|29|75|Behind rare owners move comments. Schools may tell very firms. Arguments want still in the lines. O|monthly| +3100|AAAAAAAAMBMAAAAA|2451300|2451329|DEPARTMENT|29|76|Years build happy, good fields. Cases may respond to a actions. Natural services shall |monthly| +3101|AAAAAAAANBMAAAAA|2451300|2451329|DEPARTMENT|29|77|Available, mean others should perform so yesterday black wo|monthly| +3102|AAAAAAAAOBMAAAAA|2451300|2451329|DEPARTMENT|29|78|Real friends will not offer about from a directions; completely early refugees |monthly| +3103|AAAAAAAAPBMAAAAA|2451300|2451329|DEPARTMENT|29|79|Fine, good bills would not match yesterday with the statements. Countries take just particular |monthly| +3104|AAAAAAAAACMAAAAA|2451300|2451329|DEPARTMENT|29|80|Priorities conceive generously at a hours. Thereafter consid|monthly| +3105|AAAAAAAABCMAAAAA|2451300|2451329|DEPARTMENT|29|81|National, current millions must make artificial injuries. Instead elderly yards must survive|monthly| +3106|AAAAAAAACCMAAAAA|2451300|2451329|DEPARTMENT|29|82|Sooner central members connect. Seriously expert aims would mean|monthly| +3107|AAAAAAAADCMAAAAA|2451300|2451329|DEPARTMENT|29|83|Services please often open, domestic features. Simply existing units concentrate as. Fresh ar|monthly| +3108|AAAAAAAAECMAAAAA|2451300|2451329|DEPARTMENT|29|84|Calmly young factors could read rather. Only subsequent lengths ought to get wives. Fully large|monthly| +3109|AAAAAAAAFCMAAAAA|2451300|2451329|DEPARTMENT|29|85|National lines should tell. Rather national arts may mind further royal only repo|monthly| +3110|AAAAAAAAGCMAAAAA|2451300|2451329|DEPARTMENT|29|86|Never canadian poems eat old casualties. Too rough fir|monthly| +3111|AAAAAAAAHCMAAAAA|2451300|2451329|DEPARTMENT|29|87|New demands could become significant signals. Public re|monthly| +3112|AAAAAAAAICMAAAAA|2451300|2451329|DEPARTMENT|29|88|Also variable estates introduce so different, large tim|monthly| +3113|AAAAAAAAJCMAAAAA|2451300|2451329|DEPARTMENT|29|89|Successful houses must not see much distinctive troops. Long w|monthly| +3114|AAAAAAAAKCMAAAAA|2451300|2451329|DEPARTMENT|29|90|Resources drop at least. Intermediate, significant bishops consider heavy, simila|monthly| +3115|AAAAAAAALCMAAAAA|2451300|2451329|DEPARTMENT|29|91|Mean years ought to remember of course workers; at once large periods change particularly g|monthly| +3116|AAAAAAAAMCMAAAAA|2451300|2451329|DEPARTMENT|29|92|Still likely banks see working, available males. Women appear about. Expe|monthly| +3117|AAAAAAAANCMAAAAA|2451300|2451329|DEPARTMENT|29|93|Pale, remarkable merchants might find more firms; roman women used to evaluat|monthly| +3118|AAAAAAAAOCMAAAAA|2451300|2451329|DEPARTMENT|29|94|Schools should read wonderfully clubs. Strange, social |monthly| +3119|AAAAAAAAPCMAAAAA||2451329|||95||| +3120|AAAAAAAAADMAAAAA|2451300|2451329|DEPARTMENT|29|96|Certain, other lessons might create too ethnic subjects. |monthly| +3121|AAAAAAAABDMAAAAA|2451300|2451329|DEPARTMENT|29|97|Agreements become often as a years. Similar, proper courts regar|monthly| +3122|AAAAAAAACDMAAAAA|2451300|2451329|DEPARTMENT|29|98|For example sudden arrangements would forgive also main hours. Correspondin|monthly| +3123|AAAAAAAADDMAAAAA|2451300|2451329|DEPARTMENT|29|99|Mental artists gather more. American, national technologies would not di|monthly| +3124|AAAAAAAAEDMAAAAA|2451300|2451329|DEPARTMENT|29|100|Private women may not fail so unknown, healthy pare|monthly| +3125|AAAAAAAAFDMAAAAA|2451300|2451329|DEPARTMENT|29|101|Open men know now doctors. Often hot regulations could not |monthly| +3126|AAAAAAAAGDMAAAAA|2451300|2451329|DEPARTMENT|29|102|Things improve like, remote words; more special churches include somewhere also fa|monthly| +3127|AAAAAAAAHDMAAAAA|2451300|2451329|DEPARTMENT|29|103|Political girls share once again per a women. Faint, new parents use very with a communities. Fri|monthly| +3128|AAAAAAAAIDMAAAAA|2451300|2451329|DEPARTMENT|29|104|Others must know facilities. Other, historic books could not increase magic arrangemen|monthly| +3129|AAAAAAAAJDMAAAAA|2451300|2451329|DEPARTMENT|29|105|Thoughts shed precisely. Attractive factories must not swallow just. Present policemen b|monthly| +3130|AAAAAAAAKDMAAAAA|2451300|2451329|DEPARTMENT|29|106|Forward laws cannot help general, direct rights; simply real sales bear well at|monthly| +3131|AAAAAAAALDMAAAAA|2451300|2451329|DEPARTMENT|29|107|Agreements make only. Similar, final results turn perhaps and so on busy changes;|monthly| +3132|AAAAAAAAMDMAAAAA|2451300|2451329|DEPARTMENT|29|108|Ready institutions get in a members. Large, final forms could reveal actually. Overall years m|monthly| +3133|AAAAAAAANDMAAAAA|2451330|2451359|DEPARTMENT|30|1|English problems used to learn. However familiar boxes build essential events. Grateful,|monthly| +3134|AAAAAAAAODMAAAAA|2451330|2451359|DEPARTMENT|30|2|Tracks must not strike more especially identical rig|monthly| +3135|AAAAAAAAPDMAAAAA|2451330|2451359|DEPARTMENT|30|3|Points represent ltd., twin companies. No longer single levels cannot say comfortable, brit|monthly| +3136|AAAAAAAAAEMAAAAA|2451330|2451359|DEPARTMENT|30|4|Drinks work very british thoughts. Orders would think essential, real develop|monthly| +3137|AAAAAAAABEMAAAAA|2451330|2451359|DEPARTMENT|30|5|Other specialists shall feel here modern principles. Eyes understand far possible, upper|monthly| +3138|AAAAAAAACEMAAAAA|2451330|2451359|DEPARTMENT|30|6|Considerable members could not believe also years. Surprising, sp|monthly| +3139|AAAAAAAADEMAAAAA|2451330|2451359|DEPARTMENT|30|7|Level, general phenomena say here areas. Vehicles use|monthly| +3140|AAAAAAAAEEMAAAAA|2451330|2451359|DEPARTMENT|30|8|Much left issues walk musical, regional rights. Meals ought to wear |monthly| +3141|AAAAAAAAFEMAAAAA|2451330|2451359|DEPARTMENT|30|9|All selective calls shall communicate afterwards payments. More wide sets can help |monthly| +3142|AAAAAAAAGEMAAAAA|2451330|2451359|DEPARTMENT|30|10|Large, convincing pressures put then general eyes;|monthly| +3143|AAAAAAAAHEMAAAAA|2451330|2451359|DEPARTMENT|30|11|Early rules shall put over usually different families. Sick, labour references used to excuse now|monthly| +3144|AAAAAAAAIEMAAAAA|2451330|2451359|DEPARTMENT|30|12|Free folk might communicate ever. Likely police shall not see alone already|monthly| +3145|AAAAAAAAJEMAAAAA|2451330|2451359|DEPARTMENT|30|13|Techniques ought to follow seriously american affairs. Tiny, similar courses say ve|monthly| +3146|AAAAAAAAKEMAAAAA|2451330|2451359|DEPARTMENT|30|14|Other police enter more to a representatives; new parties take endless, appropriate schools. Only i|monthly| +3147|AAAAAAAALEMAAAAA|2451330|2451359|DEPARTMENT|30|15|Terms nod rather other months. Private, whole men can flourish also reservations. Great, ult|monthly| +3148|AAAAAAAAMEMAAAAA|2451330|2451359|DEPARTMENT|30|16|About agricultural years possess new officials. Below social games meet yet n|monthly| +3149|AAAAAAAANEMAAAAA|2451330|2451359|DEPARTMENT|30|17|Pupils clean further. New, sure christians may complain legal, spiritua|monthly| +3150|AAAAAAAAOEMAAAAA|2451330|2451359|DEPARTMENT|30|18|Federal centuries should make below for a feelings. Other problems make even empir|monthly| +3151|AAAAAAAAPEMAAAAA|2451330|2451359|DEPARTMENT|30|19|Other, additional answers understand leading, urgent thousands. Holes mention ori|monthly| +3152|AAAAAAAAAFMAAAAA|2451330|2451359|DEPARTMENT|30|20|Performances get fully italian, old ladies. Much personal claims|monthly| +3153|AAAAAAAABFMAAAAA|2451330|2451359|DEPARTMENT|30|21|Blue, easy terms perform more absolute hundreds. Friends see sha|monthly| +3154|AAAAAAAACFMAAAAA|2451330|2451359|DEPARTMENT|30|22|Economic, equivalent states provide categories; guilty, effective |monthly| +3155|AAAAAAAADFMAAAAA|2451330|2451359|DEPARTMENT|30|23|Years bring more; tonnes might bring chief, remarkable|monthly| +3156|AAAAAAAAEFMAAAAA|2451330|2451359|DEPARTMENT|30|24|Personal documents stop just junior, effective eyes. Channels make immediate cars. Above |monthly| +3157|AAAAAAAAFFMAAAAA|2451330|2451359|DEPARTMENT|30|25|Either bad ways can tell clearly. Volunteers complete here flowers. Functions make on a polls.|monthly| +3158|AAAAAAAAGFMAAAAA|2451330|2451359|DEPARTMENT|30|26|Specific limits protect usually units. Children think then significant, relative customers. Men g|monthly| +3159|AAAAAAAAHFMAAAAA|2451330|2451359|DEPARTMENT|30|27|Small, economic ships will detect rarely countries; familiar, o|monthly| +3160|AAAAAAAAIFMAAAAA|2451330|2451359|DEPARTMENT|30|28|Fully leading groups suppose chiefly successful, polite years. More special t|monthly| +3161|AAAAAAAAJFMAAAAA|2451330|2451359|DEPARTMENT|30|29|Plants require. Dull workers deliver about more clean b|monthly| +3162|AAAAAAAAKFMAAAAA|2451330|2451359|DEPARTMENT|30|30|Odd, other changes must convince perfectly common, friendly groups. Things discuss rec|monthly| +3163|AAAAAAAALFMAAAAA|2451330|2451359|DEPARTMENT|30|31|Companies must continue completely. Pure drugs can w|monthly| +3164|AAAAAAAAMFMAAAAA|2451330|2451359|DEPARTMENT|30|32|Resources might concentrate teachers. Really waiting att|monthly| +3165|AAAAAAAANFMAAAAA|2451330|2451359|DEPARTMENT|30|33|Burning negotiations could not understand long. Respective, pr|monthly| +3166|AAAAAAAAOFMAAAAA|2451330|2451359|DEPARTMENT|30|34|Classes must manufacture later here financial ministers; high c|monthly| +3167|AAAAAAAAPFMAAAAA|2451330|2451359|DEPARTMENT|30|35|Little new purposes might include correspondingly to a |monthly| +3168|AAAAAAAAAGMAAAAA|2451330|2451359|DEPARTMENT|30|36|Just double expenses may build then elderly valid materials. Sco|monthly| +3169|AAAAAAAABGMAAAAA|2451330|2451359|DEPARTMENT|30|37|Pounds might not expect now; here underlying gifts trust la|monthly| +3170|AAAAAAAACGMAAAAA|2451330|2451359|DEPARTMENT|30|38|Therein sophisticated shoes save indeed bodies. Whole miner|monthly| +3171|AAAAAAAADGMAAAAA|2451330|2451359|DEPARTMENT|30|39|Considerably precise organizations make recently full, standard conditions|monthly| +3172|AAAAAAAAEGMAAAAA|2451330|2451359|DEPARTMENT|30|40|Large principles shall become. International activities ship. Single, great project|monthly| +3173|AAAAAAAAFGMAAAAA|2451330|2451359|DEPARTMENT|30|41|Other, blue computers hear often. Vague, english sales should not arise a littl|monthly| +3174|AAAAAAAAGGMAAAAA|2451330|2451359|DEPARTMENT|30|42|Political, full words succeed then. Fast, local dec|monthly| +3175|AAAAAAAAHGMAAAAA|2451330|2451359|DEPARTMENT|30|43|Alternative books raise international thousands. Chinese tons follow then in a things. |monthly| +3176|AAAAAAAAIGMAAAAA|2451330|2451359|DEPARTMENT|30|44|Goods ought to help actually then british systems. Intimately important materials shall not reward|monthly| +3177|AAAAAAAAJGMAAAAA|2451330|2451359|DEPARTMENT|30|45|Economic, russian features should get for example new seeds. American, immediate levels would |monthly| +3178|AAAAAAAAKGMAAAAA|2451330|2451359|DEPARTMENT|30|46|Thinking minutes report regular, legal ends. Metres include yet twin representatives. Etc pr|monthly| +3179|AAAAAAAALGMAAAAA|2451330|2451359|DEPARTMENT|30|47|Cultural, key months should ensure farms; american sales authorise estates. |monthly| +3180|AAAAAAAAMGMAAAAA|2451330|2451359|DEPARTMENT|30|48|Then liberal volumes arrange on a years; single trials go over very equal decisi|monthly| +3181|AAAAAAAANGMAAAAA|2451330|2451359|DEPARTMENT|30|49|Enquiries may not turn days. Aware, weak ends shall|monthly| +3182|AAAAAAAAOGMAAAAA|2451330|2451359|DEPARTMENT|30|50|Old, different articles may recognize indeed great positions; bloody, common industries th|monthly| +3183|AAAAAAAAPGMAAAAA|2451330|2451359|DEPARTMENT|30|51|Possible, religious taxes take. Fascinating, underlying documents cannot use half able eyes. S|monthly| +3184|AAAAAAAAAHMAAAAA|2451330|2451359|DEPARTMENT|30|52|Inadequate, open years should redress there able arms. Soon ordinary|monthly| +3185|AAAAAAAABHMAAAAA|2451330|2451359|DEPARTMENT|30|53|Immediate, social matters must co-ordinate either types. Reliable, poor lives play most e|monthly| +3186|AAAAAAAACHMAAAAA|2451330|2451359|DEPARTMENT|30|54|Busy, contemporary ideas would not get on a values. Soviet supporters see quickly|monthly| +3187|AAAAAAAADHMAAAAA|2451330|2451359|DEPARTMENT|30|55|Recommendations cut loudly. Ideal, public items might|monthly| +3188|AAAAAAAAEHMAAAAA|2451330|2451359|DEPARTMENT|30|56|Audiences should not see well different miles. Canadian, national members would think just all i|monthly| +3189|AAAAAAAAFHMAAAAA|2451330|2451359|DEPARTMENT|30|57|Meals should create somehow psychological, major children; old lights waste good, bright towns.|monthly| +3190|AAAAAAAAGHMAAAAA|2451330|2451359|DEPARTMENT|30|58|Examples shall not seem. Current miles offer also deposits. Strange, only|monthly| +3191|AAAAAAAAHHMAAAAA|2451330|2451359|DEPARTMENT|30|59|Later other tears ought to offer times. Real colour|monthly| +3192|AAAAAAAAIHMAAAAA|2451330|2451359|DEPARTMENT|30|60|Expert, thick gardens recommend quickly english words. Available, contemporary patients take forw|monthly| +3193|AAAAAAAAJHMAAAAA|2451330|2451359|DEPARTMENT|30|61|Happy needs could allow nearly; examinations should not lose above s|monthly| +3194|AAAAAAAAKHMAAAAA|2451330|2451359|DEPARTMENT|30|62|Rural, new flames would talk. Good, existing priorities know processes. New symbols wri|monthly| +3195|AAAAAAAALHMAAAAA|2451330|2451359|DEPARTMENT|30|63|Sentences would pick well better strong things. Pounds place as good periods; coming, |monthly| +3196|AAAAAAAAMHMAAAAA|2451330|2451359|DEPARTMENT|30|64|Members give initially through a moments. Prices restrain almost shows. Different,|monthly| +3197|AAAAAAAANHMAAAAA|2451330|2451359|DEPARTMENT|30|65|Final groups would not struggle ears. Significant friends choose v|monthly| +3198|AAAAAAAAOHMAAAAA|2451330|2451359|DEPARTMENT|30|66|Useful, annual hands might not measure. Early, org|monthly| +3199|AAAAAAAAPHMAAAAA|2451330|2451359|DEPARTMENT|30|67|Historic windows believe others. Healthy, large rights will cause aggressive arts. Step|monthly| +3200|AAAAAAAAAIMAAAAA|2451330|2451359|DEPARTMENT|30|68|Concerned laws see immediately years. Kids may hear really by way of a clothes. Early, rational|monthly| +3201|AAAAAAAABIMAAAAA|2451330|2451359|DEPARTMENT|30|69|Deep, full actions complete direct. Long forces could increase m|monthly| +3202|AAAAAAAACIMAAAAA|2451330|2451359|DEPARTMENT|30|70|African, british bottles shall validate principally good years. Normal democr|monthly| +3203|AAAAAAAADIMAAAAA|2451330|2451359|DEPARTMENT|30|71|Woods go to the applications. Pensioners can kill there entire libraries. Alone, great i|monthly| +3204|AAAAAAAAEIMAAAAA|2451330|2451359|DEPARTMENT|30|72|Difficult girls express solely. Much cheap years c|monthly| +3205|AAAAAAAAFIMAAAAA|2451330|2451359|DEPARTMENT|30|73|Hours make quickly fellow savings. Effective, strange directors may feel now women. N|monthly| +3206|AAAAAAAAGIMAAAAA|2451330|2451359|DEPARTMENT|30|74|Daily areas work by a proteins. Also strange waters go german voters. Just past demands may expect |monthly| +3207|AAAAAAAAHIMAAAAA|2451330|2451359|DEPARTMENT|30|75|Possible researchers interfere essential ages. Weeks ought to hit as central cases. Badly soc|monthly| +3208|AAAAAAAAIIMAAAAA|2451330|2451359|DEPARTMENT|30|76|New, right miles determine however. Rarely clear patients shall fight direct weak, new towns. Appli|monthly| +3209|AAAAAAAAJIMAAAAA|2451330|2451359|DEPARTMENT|30|77|Physically small others may recruit generally. Actually italian step|monthly| +3210|AAAAAAAAKIMAAAAA|2451330|2451359|DEPARTMENT|30|78|Capable pictures used to close behind permanent, old styles; physical buildings used to understa|monthly| +3211|AAAAAAAALIMAAAAA|2451330|2451359|DEPARTMENT|30|79|Major subsidies play following, new cars; accounts keep actually radical, new|monthly| +3212|AAAAAAAAMIMAAAAA|2451330|2451359|DEPARTMENT|30|80|Environmental, additional units come so; seriously wrong pupils make particularly bad room|monthly| +3213|AAAAAAAANIMAAAAA|2451330|2451359|DEPARTMENT|30|81|So old-fashioned years happen steeply republics. Gene|monthly| +3214|AAAAAAAAOIMAAAAA|2451330|2451359|DEPARTMENT|30|82|Unfair communications read. Other, positive facilities may not know yesterday even i|monthly| +3215|AAAAAAAAPIMAAAAA|2451330|2451359|DEPARTMENT|30|83|Consequently good awards must pass pupils. Arab modes might conform much now organisati|monthly| +3216|AAAAAAAAAJMAAAAA|2451330|2451359|DEPARTMENT|30|84|Problems can develop short, public steps. Radical, payable factors could not trust. Results need u|monthly| +3217|AAAAAAAABJMAAAAA|2451330|2451359|DEPARTMENT|30|85|Chief, key yards must tell gold hands. Brilliant times must believe about th|monthly| +3218|AAAAAAAACJMAAAAA|2451330|2451359|DEPARTMENT|30|86|White parties ought to walk well. Key details can figure below prime numbers. Inc services f|monthly| +3219|AAAAAAAADJMAAAAA|2451330|2451359|DEPARTMENT|30|87|Deliberate, different fans would not move for a women. Effective, politic|monthly| +3220|AAAAAAAAEJMAAAAA|2451330|2451359|DEPARTMENT|30|88|Difficult contexts should not obtain most furthermore typical expectations. Other values work |monthly| +3221|AAAAAAAAFJMAAAAA|2451330|2451359|DEPARTMENT|30|89|Existing comments bring however for a companies. Other, supreme |monthly| +3222|AAAAAAAAGJMAAAAA|2451330|2451359|DEPARTMENT|30|90|English, good techniques will quit more; conveniently certain folk na|monthly| +3223|AAAAAAAAHJMAAAAA|2451330|2451359|DEPARTMENT|30|91|Long, physical ladies will not see now in a papers. Honest daught|monthly| +3224|AAAAAAAAIJMAAAAA|2451330|2451359|DEPARTMENT|30|92|Personal elements may close to a systems. Religious problems go perhaps extens|monthly| +3225|AAAAAAAAJJMAAAAA|2451330|2451359|DEPARTMENT|30|93|White, young problems take still. Assumptions lead workshops. Part|monthly| +3226|AAAAAAAAKJMAAAAA|2451330|2451359|DEPARTMENT|30|94|Much main ends move in a chains. Other arms would give also via a pup|monthly| +3227|AAAAAAAALJMAAAAA|2451330|2451359|DEPARTMENT|30|95|Bright, efficient kinds ask more back, aware references. Differently existing methods could|monthly| +3228|AAAAAAAAMJMAAAAA|2451330|2451359|DEPARTMENT|30|96|Hours should take previous resources. Circumstances used to print both. At least possible fun|monthly| +3229|AAAAAAAANJMAAAAA|2451330|2451359|DEPARTMENT|30|97|Patients shall get for example early, reduced letters. Meetings |monthly| +3230|AAAAAAAAOJMAAAAA|2451330|2451359|DEPARTMENT|30|98|Only machines could not go also growing individuals. Eventually low hours remember theref|monthly| +3231|AAAAAAAAPJMAAAAA|2451330|2451359|DEPARTMENT|30|99|Still suitable hands wind local, white settings. However big types can make thus. New words migh|monthly| +3232|AAAAAAAAAKMAAAAA|2451330|2451359|DEPARTMENT|30|100|Now additional rates like before the weapons. Detail|monthly| +3233|AAAAAAAABKMAAAAA|2451330|2451359|DEPARTMENT|30|101|Concerns guarantee very clients. Still new actions focus ruling, religious christians; there|monthly| +3234|AAAAAAAACKMAAAAA|2451330|2451359|DEPARTMENT|30|102|Partly pale games should provide suddenly; great, new children should eat so even possible pr|monthly| +3235|AAAAAAAADKMAAAAA|2451330|2451359|DEPARTMENT|30|103|Men may not point somewhere fine police. More deaf flowers s|monthly| +3236|AAAAAAAAEKMAAAAA|2451330|2451359|DEPARTMENT|30|104|Electrical limits might not send for a women. Socialist, simple posts believe days.|monthly| +3237|AAAAAAAAFKMAAAAA|2451330|2451359|DEPARTMENT|30|105|Easy, blank mothers begin equal, other applications. At all accused bills would |monthly| +3238|AAAAAAAAGKMAAAAA|2451330|2451359|DEPARTMENT|30|106|New, concrete eyes must not maintain. Thick cells shall differ all left studies. |monthly| +3239|AAAAAAAAHKMAAAAA|2451330|2451359|DEPARTMENT|30|107|Also future books may not complement also video-taped, common models; appropri|monthly| +3240|AAAAAAAAIKMAAAAA|2451330|2451359|DEPARTMENT|30|108|New, chemical producers know regional teeth. Old, moving feet sho|monthly| +3241|AAAAAAAAJKMAAAAA|2451360|2451389|DEPARTMENT|31|1|By now potential managers lead now legal, residential|monthly| +3242|AAAAAAAAKKMAAAAA|2451360|2451389|DEPARTMENT|31|2|Single officials might continue thus kinds; deep executives manipulate enough |monthly| +3243|AAAAAAAALKMAAAAA|2451360|2451389|DEPARTMENT|31|3|Grand, superb institutions help about logical details. Annual|monthly| +3244|AAAAAAAAMKMAAAAA|2451360|2451389|DEPARTMENT|31|4|Americans call most. Yet royal grants cease evil changes. Critical|monthly| +3245|AAAAAAAANKMAAAAA|2451360|2451389|DEPARTMENT|31|5|Refugees may catch probably however tall waves. Needs become once more |monthly| +3246|AAAAAAAAOKMAAAAA|2451360|2451389|DEPARTMENT|31|6|Neighbours come local drivers. Never explicit times say incentives. Big, internal teachers|monthly| +3247|AAAAAAAAPKMAAAAA|2451360|2451389|DEPARTMENT|31|7|Different systems show locally terms. Enough bare tonnes react honestly excellent winds. U|monthly| +3248|AAAAAAAAALMAAAAA|2451360|2451389|DEPARTMENT|31|8|Examples catch. Internally old problems undertake already simple, democ|monthly| +3249|AAAAAAAABLMAAAAA|2451360|2451389|DEPARTMENT|31|9|Rooms ought to get total directions. Implications will work too g|monthly| +3250|AAAAAAAACLMAAAAA|2451360|2451389|DEPARTMENT|31|10|Horses should not seem certainly actually other cards. Good, familiar came|monthly| +3251|AAAAAAAADLMAAAAA|2451360|2451389|DEPARTMENT|31|11|Now upper thousands relate then particularly great writings; deeper|monthly| +3252|AAAAAAAAELMAAAAA|2451360|2451389|DEPARTMENT|31|12|Together russian weekends go away average methods. Curious, reduced r|monthly| +3253|AAAAAAAAFLMAAAAA|2451360|2451389|DEPARTMENT|31|13|Awkward words intervene adversely. Social, prime businessm|monthly| +3254|AAAAAAAAGLMAAAAA|2451360|2451389|DEPARTMENT|31|14|Just other items must form more ways. Light, old records could find else for a things. Gene|monthly| +3255|AAAAAAAAHLMAAAAA|2451360|2451389|DEPARTMENT|31|15|More economic films identify much different arms. Steps should not fly ro|monthly| +3256|AAAAAAAAILMAAAAA|2451360|2451389|DEPARTMENT|31|16|Effects must open workers. Areas take; however white fathers shall believe very ruling, sim|monthly| +3257|AAAAAAAAJLMAAAAA|2451360|2451389|DEPARTMENT|31|17|Sheets develop by a funds. Short newspapers may look letters. Vocational books get|monthly| +3258|AAAAAAAAKLMAAAAA|2451360|2451389|DEPARTMENT|31|18|Fine babies may not belong months. Even white judges catch schools. Persona|monthly| +3259|AAAAAAAALLMAAAAA|2451360|2451389|DEPARTMENT|31|19|Properly important neighbours see maybe superior, following problems; usual s|monthly| +3260|AAAAAAAAMLMAAAAA|2451360|2451389|DEPARTMENT|31|20|As long-term years may not bring again. Healthy, physi|monthly| +3261|AAAAAAAANLMAAAAA|2451360|2451389|DEPARTMENT|31|21|Casually working-class letters replace again. Great, blue levels leave perhaps budgets|monthly| +3262|AAAAAAAAOLMAAAAA|2451360|2451389|DEPARTMENT|31|22|So social workers would not live courses. Other, pure products find much corporate |monthly| +3263|AAAAAAAAPLMAAAAA|2451360|2451389|DEPARTMENT|31|23|Expectations come actually other police. In general convincing words result a|monthly| +3264|AAAAAAAAAMMAAAAA|2451360|2451389|DEPARTMENT|31|24|Doubtfully able categories cannot respond less to a tenants. Much good cust|monthly| +3265|AAAAAAAABMMAAAAA|2451360|2451389|DEPARTMENT|31|25|Small, different sons ensure rather brown creditors. Facilities could present at last so industria|monthly| +3266|AAAAAAAACMMAAAAA|2451360|2451389|DEPARTMENT|31|26|Friendly, average animals may explain hardly altogether whol|monthly| +3267|AAAAAAAADMMAAAAA|2451360|2451389|DEPARTMENT|31|27|Tracks could make highly good, urgent words. Very other points identi|monthly| +3268|AAAAAAAAEMMAAAAA|2451360|2451389|DEPARTMENT|31|28|Things find sites. American things provide local, considerable objects; th|monthly| +3269|AAAAAAAAFMMAAAAA|2451360|2451389|DEPARTMENT|31|29|Miners could become already. Abruptly associated goods shall re|monthly| +3270|AAAAAAAAGMMAAAAA|2451360|2451389|DEPARTMENT|31|30|Actually good techniques drop at all. Aware years shall ask historians. Models check past new |monthly| +3271|AAAAAAAAHMMAAAAA|2451360|2451389|DEPARTMENT|31|31|Black, sad ways see nearly all different bands. Serious, typical readers must like however a|monthly| +3272|AAAAAAAAIMMAAAAA|2451360|2451389|DEPARTMENT|31|32|Dominant, northern workers cannot help successful, |monthly| +3273|AAAAAAAAJMMAAAAA|2451360|2451389|DEPARTMENT|31|33|Threats seem fair partial, other decades. As royal months make remote, main |monthly| +3274|AAAAAAAAKMMAAAAA|2451360|2451389|DEPARTMENT|31|34|Free, tired foods might help still at a families. Really detailed ef|monthly| +3275|AAAAAAAALMMAAAAA|2451360|2451389|DEPARTMENT|31|35|Details may not handle never. Mad, english officials can alt|monthly| +3276|AAAAAAAAMMMAAAAA|2451360|2451389|DEPARTMENT|31|36|Then misleading weaknesses reach genuine prices; however aware shareholders encourage als|monthly| +3277|AAAAAAAANMMAAAAA|2451360|2451389|DEPARTMENT|31|37|Horses pop there for a relations. National systems add great, small contracts. |monthly| +3278|AAAAAAAAOMMAAAAA|2451360|2451389|DEPARTMENT|31|38|Good ministers say roles. Burning, neutral positions will not see. Critical reasons woul|monthly| +3279|AAAAAAAAPMMAAAAA|2451360|2451389|DEPARTMENT|31|39|Concerned eyes shall get by a ears. Techniques come less good, new assets. Small, careful i|monthly| +3280|AAAAAAAAANMAAAAA|2451360|2451389|DEPARTMENT|31|40|Immediately major workshops highlight. Social, good mechanisms |monthly| +3281|AAAAAAAABNMAAAAA|2451360|2451389|DEPARTMENT|31|41|Only extra lines begin equal, good secrets. Powerful, so|monthly| +3282|AAAAAAAACNMAAAAA|2451360|2451389|DEPARTMENT|31|42|Conditions happen already because of a users. So solid topics will not forg|monthly| +3283|AAAAAAAADNMAAAAA|2451360|2451389|DEPARTMENT|31|43|Occupational features make heavy things. Disabled, broad problems will not make su|monthly| +3284|AAAAAAAAENMAAAAA|2451360|2451389|DEPARTMENT|31|44|Only nuclear policies understand so basic courts. |monthly| +3285|AAAAAAAAFNMAAAAA|2451360|2451389|DEPARTMENT|31|45|Public, possible youngsters could not take today young minu|monthly| +3286|AAAAAAAAGNMAAAAA|2451360|2451389|DEPARTMENT|31|46|Reports ascertain new shows. Involved opportunities shall get|monthly| +3287|AAAAAAAAHNMAAAAA|2451360|2451389|DEPARTMENT|31|47|Meetings blame; later high profits will join so local serious studies. D|monthly| +3288|AAAAAAAAINMAAAAA|2451360|2451389|DEPARTMENT|31|48|Services avoid middle, united collections. Interested players could not know actual practi|monthly| +3289|AAAAAAAAJNMAAAAA|2451360|2451389|DEPARTMENT|31|49|Less political problems realise nervous implications. Times ought to attempt long|monthly| +3290|AAAAAAAAKNMAAAAA|2451360|2451389|DEPARTMENT|31|50|Stupid, total sites might not get in the needs. Stori|monthly| +3291|AAAAAAAALNMAAAAA|2451360|2451389|DEPARTMENT|31|51|Independent curtains used to get in a sales; administrative books give charges. Environmentall|monthly| +3292|AAAAAAAAMNMAAAAA|2451360|2451389|DEPARTMENT|31|52|Terms behave therefore ever widespread properties. Weeks could not risk in the steps. Di|monthly| +3293|AAAAAAAANNMAAAAA|2451360|2451389|DEPARTMENT|31|53|Titles need much mad years. Full patients use at least previously modern days|monthly| +3294|AAAAAAAAONMAAAAA|2451360|2451389|DEPARTMENT|31|54|Mysterious, surprised children like local laws. Russian ad|monthly| +3295|AAAAAAAAPNMAAAAA|2451360|2451389|DEPARTMENT|31|55|Sure relations follow onto a personnel. Chief situations dream; sudden applications must ma|monthly| +3296|AAAAAAAAAOMAAAAA|2451360|2451389|DEPARTMENT|31|56|New proposals might mean; aspects used to need calls. So educa|monthly| +3297|AAAAAAAABOMAAAAA|2451360|2451389||31|||monthly| +3298|AAAAAAAACOMAAAAA|2451360|2451389|DEPARTMENT|31|58|Various funds could influence with a troops. Great, shared recor|monthly| +3299|AAAAAAAADOMAAAAA|2451360|2451389|DEPARTMENT|31|59|Parts would go to a policies. New effects believe s|monthly| +3300|AAAAAAAAEOMAAAAA|2451360|2451389|DEPARTMENT|31|60|Of course different months find recently colours. Very initial earnings recommend all busy d|monthly| +3301|AAAAAAAAFOMAAAAA|2451360|2451389|DEPARTMENT|31|61|Also severe parties could not make things. Subjects know. Dear managers cause once capitali|monthly| +3302|AAAAAAAAGOMAAAAA|2451360|2451389|DEPARTMENT|31|62|Full days might respond well available, bad ports.|monthly| +3303|AAAAAAAAHOMAAAAA|2451360|2451389|DEPARTMENT|31|63|Especially faint children can write. Clear suggestions deliv|monthly| +3304|AAAAAAAAIOMAAAAA|2451360|2451389|DEPARTMENT|31|64|As well pure hearts review usually still mental doors; offshore human sce|monthly| +3305|AAAAAAAAJOMAAAAA|2451360|2451389|DEPARTMENT|31|65|Open, select posts must not determine japanese resources. Increased|monthly| +3306|AAAAAAAAKOMAAAAA|2451360|2451389|DEPARTMENT|31|66|American, difficult ships visit then. Perfectly only times should not see too. Ar|monthly| +3307|AAAAAAAALOMAAAAA|2451360|2451389|DEPARTMENT|31|67|Pressures might get instantly documents; glad, little years would serve different, generous sound|monthly| +3308|AAAAAAAAMOMAAAAA|2451360|2451389|DEPARTMENT|31|68|Wrong, blue emissions shall remove furthermore good, major earnings; areas raise home substantia|monthly| +3309|AAAAAAAANOMAAAAA|2451360|2451389|DEPARTMENT|31|69|Critical lips like initial components; artificial champions l|monthly| +3310|AAAAAAAAOOMAAAAA|2451360|2451389|DEPARTMENT|31|70|Great arrangements could tell always; detailed directions mind more than aga|monthly| +3311|AAAAAAAAPOMAAAAA|2451360|2451389|DEPARTMENT|31|71|Levels will not go forwards remaining things. Then emotional classes understand fairly huge, old |monthly| +3312|AAAAAAAAAPMAAAAA|2451360|2451389|DEPARTMENT|31|72|Determined, important costs say between the creatu|monthly| +3313|AAAAAAAABPMAAAAA|2451360|2451389|DEPARTMENT|31|73|However english doors offer by a sons. Legislative, s|monthly| +3314|AAAAAAAACPMAAAAA|2451360|2451389|DEPARTMENT|31|74|Also willing neighbours pay and so on technical, concerned needs. New, key changes should worry |monthly| +3315|AAAAAAAADPMAAAAA|2451360|2451389|DEPARTMENT|31|75|Tales look too finally severe bodies; historical, main children wa|monthly| +3316|AAAAAAAAEPMAAAAA|2451360|2451389|DEPARTMENT|31|76|Popular forms could fit really. Financial, future actions guarantee|monthly| +3317|AAAAAAAAFPMAAAAA|2451360|2451389|DEPARTMENT|31|77|Methods mention still with the dishes. Dry troops look western types. De|monthly| +3318|AAAAAAAAGPMAAAAA|2451360|2451389|DEPARTMENT|31|78|Accountants ought to concentrate just thousands. Responsible, big rights should use in a players|monthly| +3319|AAAAAAAAHPMAAAAA|2451360|2451389|DEPARTMENT|31|79|Difficult, real advantages would relate to a qualifications; wrong, british hands like prime o|monthly| +3320|AAAAAAAAIPMAAAAA|2451360|2451389|DEPARTMENT|31|80|Practically new newspapers spend children. Lonely, real |monthly| +3321|AAAAAAAAJPMAAAAA|2451360|2451389|DEPARTMENT|31|81|Even armed deputies sell that alongside a qualities. Scottish units used t|monthly| +3322|AAAAAAAAKPMAAAAA|2451360|2451389|DEPARTMENT|31|82|Away head connections take effectively american voices. Economic resources |monthly| +3323|AAAAAAAALPMAAAAA|2451360|2451389|DEPARTMENT|31|83|Major terms refuse friends. Unaware humans get proper, american levels. Redunda|monthly| +3324|AAAAAAAAMPMAAAAA|2451360|2451389|DEPARTMENT|31|84|Managers used to derive obvious, nuclear parties; seats know. Metals happen too seriously small ho|monthly| +3325|AAAAAAAANPMAAAAA|2451360|2451389|DEPARTMENT|31|85|Equal, continuing appearances construct appointed pages. Words say in a |monthly| +3326|AAAAAAAAOPMAAAAA|2451360|2451389|DEPARTMENT|31|86|Certain eyes beat centres. Young players stare; cheap, royal|monthly| +3327|AAAAAAAAPPMAAAAA|2451360|2451389|DEPARTMENT|31|87|Equally beneficial nations put. Too local representatives ought to take usually. Joint|monthly| +3328|AAAAAAAAAANAAAAA|2451360|2451389|DEPARTMENT|31|88|Critical months detect ever separate experiments. Happy women used to phone insi|monthly| +3329|AAAAAAAABANAAAAA|2451360|2451389|DEPARTMENT|31|89|Deep fresh mechanisms refuse even on a proposals. Moral surface|monthly| +3330|AAAAAAAACANAAAAA|2451360|2451389|DEPARTMENT|31|90|Institutions sense new, greek wishes. Hands come medium, strong churches|monthly| +3331|AAAAAAAADANAAAAA|2451360|2451389|DEPARTMENT|31|91|Large years create single, similar authorities; apparent damages stim|monthly| +3332|AAAAAAAAEANAAAAA|2451360|2451389|DEPARTMENT|31|92|Associations might plunge visible, scientific duties. Under dead doctors breathe elsewhere|monthly| +3333|AAAAAAAAFANAAAAA|2451360|2451389|DEPARTMENT|31|93|Now good games prove strongly independent problems. Then financial re|monthly| +3334|AAAAAAAAGANAAAAA|2451360|2451389|DEPARTMENT|31|94|Early single organisms listen at least benefits. Plants apply; far pregnant points |monthly| +3335|AAAAAAAAHANAAAAA|2451360|2451389|DEPARTMENT|31|95|Rules shall prepare even closer tiny bedrooms. Friends face always. |monthly| +3336|AAAAAAAAIANAAAAA|2451360|2451389|DEPARTMENT|31|96|Visual, manufacturing videos change here. Special leve|monthly| +3337|AAAAAAAAJANAAAAA|2451360|2451389|DEPARTMENT|31|97|Always economic places protect to a keys. Tonight top fans could go other, long purposes.|monthly| +3338|AAAAAAAAKANAAAAA|||DEPARTMENT||||| +3339|AAAAAAAALANAAAAA|2451360|2451389|DEPARTMENT|31|99|Actual women expand in a prices. Quite growing solicitors shall go members. Almost|monthly| +3340|AAAAAAAAMANAAAAA|2451360|2451389|DEPARTMENT|31|100|Final fears pay neither liable, central rates. Dangers |monthly| +3341|AAAAAAAANANAAAAA|2451360|2451389|DEPARTMENT|31|101|Numbers must contradict in the main up a workers. Unfortunately different tanks ru|monthly| +3342|AAAAAAAAOANAAAAA|2451360|2451389|DEPARTMENT|31|102|Decent parties say usually twins; late, concerned gifts ca|monthly| +3343|AAAAAAAAPANAAAAA|2451360|2451389|DEPARTMENT|31|103|Orders might cause successful, irish forms. Organisational, just|monthly| +3344|AAAAAAAAABNAAAAA|2451360|2451389|DEPARTMENT|31|104|Standards might consider really good men. Now official ladies work |monthly| +3345|AAAAAAAABBNAAAAA|2451360|2451389|DEPARTMENT|31|105|Ultimately psychological things cannot see with a skills. Good students commit electoral, able |monthly| +3346|AAAAAAAACBNAAAAA|2451360|2451389|DEPARTMENT|31|106|Young elections must indicate too roman, light other|monthly| +3347|AAAAAAAADBNAAAAA|2451360|2451389|DEPARTMENT|31|107|Narrow forms would collect by a plans. Nasty, additional e|monthly| +3348|AAAAAAAAEBNAAAAA|2451360|2451389|DEPARTMENT|31|108|New, effective events will portray industries. Central,|monthly| +3349|AAAAAAAAFBNAAAAA|2451390|2451419|DEPARTMENT|32|1|Contrary, casual letters would not wait popular, great dev|monthly| +3350|AAAAAAAAGBNAAAAA|2451390|2451419|DEPARTMENT|32|2|Top, actual years would pop very, new problems. Long-term, vague others might not appear |monthly| +3351|AAAAAAAAHBNAAAAA|2451390|2451419|DEPARTMENT|32|3|Social ways go in the others. Young sizes might not amount other, quiet ter|monthly| +3352|AAAAAAAAIBNAAAAA|2451390|2451419|DEPARTMENT|32|4|Methods lead commercially under a courses. New jews limit daily|monthly| +3353|AAAAAAAAJBNAAAAA|2451390|2451419|DEPARTMENT|32|5|Numerous tales may fish at last then tiny churches. Tremendous, impor|monthly| +3354|AAAAAAAAKBNAAAAA|2451390|2451419|DEPARTMENT|32|6|Major cases research occasionally. Holders will not get well indiv|monthly| +3355|AAAAAAAALBNAAAAA|2451390|2451419|DEPARTMENT|32|7|Much quiet laws would not add usually in a levels; forthcoming, new friends justify then natural|monthly| +3356|AAAAAAAAMBNAAAAA|2451390|2451419|DEPARTMENT|32|8|Clear shoes say above. Later low years wish later structures. Too hot citizens meet. Waitin|monthly| +3357|AAAAAAAANBNAAAAA|2451390|2451419|DEPARTMENT|32|9|Old, sweet talks draw others. Families shall pay tomorrow more asleep te|monthly| +3358|AAAAAAAAOBNAAAAA|2451390|2451419|DEPARTMENT|32|10|Combined, black cars used to hold then only conditions. Deposits reflect in a flowers. No doubt ma|monthly| +3359|AAAAAAAAPBNAAAAA|2451390|2451419|DEPARTMENT|32|11|Already alone terms mind parents. Primary others add again main dogs. Di|monthly| +3360|AAAAAAAAACNAAAAA|2451390|2451419|DEPARTMENT|32|12|So external ports will leave. Little others refuse human friends; soft, pregnant st|monthly| +3361|AAAAAAAABCNAAAAA|2451390|||32||Effects could back now other estimates; others might make too well nuclear leaders; goods sleep i|monthly| +3362|AAAAAAAACCNAAAAA|2451390|2451419|DEPARTMENT|32|14|No longer similar systems get very private, modern schools. Awfully r|monthly| +3363|AAAAAAAADCNAAAAA|2451390|2451419|DEPARTMENT|32|15|For example other complaints prevent; associations take. Children support apparentl|monthly| +3364|AAAAAAAAECNAAAAA|2451390|2451419|DEPARTMENT|32|16|Both secondary newspapers see all. Places take almost. Rights exceed especially j|monthly| +3365|AAAAAAAAFCNAAAAA|2451390|2451419|DEPARTMENT|32|17|Likely, economic teachers feel around teachers; predominantly sound terms follow here speci|monthly| +3366|AAAAAAAAGCNAAAAA|2451390|2451419|DEPARTMENT|32|18|Weeks should not come there fine times. Symptoms would cover mainly logical sta|monthly| +3367|AAAAAAAAHCNAAAAA|2451390|2451419|DEPARTMENT|32|19|Easy, scientific enemies give about a homes. Factories |monthly| +3368|AAAAAAAAICNAAAAA|2451390|2451419|DEPARTMENT|32|20|Years shall not qualify tomorrow loose players. Now common tim|monthly| +3369|AAAAAAAAJCNAAAAA|2451390|2451419|DEPARTMENT|32|21|Aware, distinct versions used to let immediately sm|monthly| +3370|AAAAAAAAKCNAAAAA|2451390|2451419|DEPARTMENT|32|22|Distinctive feelings may stand all. Other rates tak|monthly| +3371|AAAAAAAALCNAAAAA|2451390|2451419|DEPARTMENT|32|23|Early arts may not see mutually remains. Pieces believe originally |monthly| +3372|AAAAAAAAMCNAAAAA|2451390|2451419|DEPARTMENT|32|24|And so on violent agents would not sink less gifts. Sympathetic, white patients wi|monthly| +3373|AAAAAAAANCNAAAAA|2451390|2451419|DEPARTMENT|32|25|Books should send even interesting, social officials; years smoke on|monthly| +3374|AAAAAAAAOCNAAAAA|2451390|2451419|DEPARTMENT|32|26|Open numbers could review minutes. Different, direct |monthly| +3375|AAAAAAAAPCNAAAAA|2451390|2451419|DEPARTMENT|32|27|Low years will launch enough minimal, running relations.|monthly| +3376|AAAAAAAAADNAAAAA|2451390|2451419|DEPARTMENT|32|28|Historic organisations can prevent right particular du|monthly| +3377|AAAAAAAABDNAAAAA|2451390|2451419|DEPARTMENT|32|29|Practically comfortable boys will appeal effective, canadia|monthly| +3378|AAAAAAAACDNAAAAA|2451390|2451419|DEPARTMENT|32|30|Services must interfere in a years. Men say at the station|monthly| +3379|AAAAAAAADDNAAAAA||2451419|||31|Months touch too. Away available mechanisms want recently. St|monthly| +3380|AAAAAAAAEDNAAAAA|2451390|2451419|DEPARTMENT|32|32|Unacceptable, good women achieve early evident customers. Patients should lose above. Nor|monthly| +3381|AAAAAAAAFDNAAAAA|2451390|2451419|DEPARTMENT|32|33|Largely national places use then in the women. Here particular classes must help loose women|monthly| +3382|AAAAAAAAGDNAAAAA|2451390|2451419|DEPARTMENT|32|34|Now main differences must complement there. Very short years let now to |monthly| +3383|AAAAAAAAHDNAAAAA|2451390|2451419|DEPARTMENT|32|35|Away controversial examples relieve before a troops. Else persona|monthly| +3384|AAAAAAAAIDNAAAAA|2451390|2451419|DEPARTMENT|32|36|Manufacturers avoid much real hours. Unable developments |monthly| +3385|AAAAAAAAJDNAAAAA|2451390|2451419|DEPARTMENT|32|37|Probably certain circumstances must describe partly inside good words.|monthly| +3386|AAAAAAAAKDNAAAAA|2451390|2451419|DEPARTMENT|32|38|High democratic times should not understand major, wide te|monthly| +3387|AAAAAAAALDNAAAAA|2451390|2451419|DEPARTMENT|32|39|Differences view in the types. Far articles ought to make so. Carefu|monthly| +3388|AAAAAAAAMDNAAAAA|2451390|2451419|DEPARTMENT|32|40|Single, fine rules might not assume now. Governments protect difficu|monthly| +3389|AAAAAAAANDNAAAAA|2451390|2451419|DEPARTMENT|32|41|Big, individual bands tell from a actions. Passive, pe|monthly| +3390|AAAAAAAAODNAAAAA|2451390|2451419|DEPARTMENT|32|42|Interested, bad things clear at a needs. Similar acts shall extract across mature |monthly| +3391|AAAAAAAAPDNAAAAA|2451390|2451419|DEPARTMENT|32|43|Long, attractive police test as so keen men. More local relations will come also metals. Mu|monthly| +3392|AAAAAAAAAENAAAAA|2451390|2451419|DEPARTMENT|32|44|Already small memories sit seriously too good opportunities. Outside different go|monthly| +3393|AAAAAAAABENAAAAA|2451390|2451419|DEPARTMENT|32|45|Old, little ministers deny for example by a steps. Little, yellow fans alter national, possible ma|monthly| +3394|AAAAAAAACENAAAAA|2451390|2451419|DEPARTMENT|32|46|Years track. Complete tools should not own there after a cars|monthly| +3395|AAAAAAAADENAAAAA|2451390|2451419|DEPARTMENT|32|47|Normal, small months get often. Members cover around at a lights. Fixed wall|monthly| +3396|AAAAAAAAEENAAAAA|2451390|2451419|DEPARTMENT|32|48|Past, very limitations win precisely according to a papers. Really other levels start in fu|monthly| +3397|AAAAAAAAFENAAAAA|2451390|2451419|DEPARTMENT|32|49|Vast procedures send again royal findings. Economic, only designers should |monthly| +3398|AAAAAAAAGENAAAAA|2451390|2451419|DEPARTMENT|32|50|Now fundamental words may not give later social writings. Politicians avoid |monthly| +3399|AAAAAAAAHENAAAAA|2451390|2451419|DEPARTMENT|32|51|Mothers help only brown, national keys. Popular programmes specify so thin |monthly| +3400|AAAAAAAAIENAAAAA|2451390|2451419|DEPARTMENT|32|52|Sharply international police proceed certainly historians. Later effective girls may allow in a li|monthly| +3401|AAAAAAAAJENAAAAA|2451390|2451419|DEPARTMENT|32|53|Upper, internal cars will not hand too in terms of a cells. |monthly| +3402|AAAAAAAAKENAAAAA|2451390|2451419|DEPARTMENT|32|54|Tasks leave all right critical hours. Italian legs limit now international pp.. Co|monthly| +3403|AAAAAAAALENAAAAA|2451390|2451419|DEPARTMENT|32|55|As common teachers could not cry all; streets warn changes. Other losses obey about conservat|monthly| +3404|AAAAAAAAMENAAAAA|2451390|2451419|DEPARTMENT|32|56|Above old departments shall afford. Inches may see already. Educational, other t|monthly| +3405|AAAAAAAANENAAAAA|2451390|2451419|DEPARTMENT|32|57|Often holy issues should improve full patients; silly, expected initiatives avo|monthly| +3406|AAAAAAAAOENAAAAA|2451390|2451419|DEPARTMENT|32|58|Orders should not cope in a voters. Disciplines descr|monthly| +3407|AAAAAAAAPENAAAAA||2451419|DEPARTMENT||59|Secret, minute lessons used to despise now big streets. Open planes observe again on a || +3408|AAAAAAAAAFNAAAAA|2451390|2451419|DEPARTMENT|32|60|Awards must turn also against a millions. Sole, relevant securities might take recen|monthly| +3409|AAAAAAAABFNAAAAA|2451390|2451419|DEPARTMENT|32|61|Either possible senses ought to motivate soon certainly global sheets. Efficient expenses visit e|monthly| +3410|AAAAAAAACFNAAAAA|2451390|2451419|DEPARTMENT|32|62|Skills would give even. Complete women find later slow centres. Social, useful |monthly| +3411|AAAAAAAADFNAAAAA|2451390|2451419|DEPARTMENT|32|63|There long details will freeze artistic, large agencies. E|monthly| +3412|AAAAAAAAEFNAAAAA|2451390|2451419|DEPARTMENT|32|64|High, private boys would sample foreign earnings. Artificial, sudden elections double|monthly| +3413|AAAAAAAAFFNAAAAA|2451390|2451419|DEPARTMENT|32|65|Important, important hospitals would not buy on a vessels. Interesting, guilty weeks indicat|monthly| +3414|AAAAAAAAGFNAAAAA|2451390|2451419|DEPARTMENT|32|66|Relationships cannot underline women; asleep, key speakers must not tell even so early, propos|monthly| +3415|AAAAAAAAHFNAAAAA|2451390|2451419|DEPARTMENT|32|67|New, following reasons find now with a groups. Common, young generations wear how|monthly| +3416|AAAAAAAAIFNAAAAA|2451390|2451419|DEPARTMENT|32|68|Political, european employees weave nearly. Often lively contexts change together valuable, blac|monthly| +3417|AAAAAAAAJFNAAAAA|2451390|2451419|DEPARTMENT|32|69|Final, steady players should take whole, previous unions. P|monthly| +3418|AAAAAAAAKFNAAAAA|2451390|2451419|DEPARTMENT|32|70|True animals go certainly bodies. Also awkward experiences keep no longer sm|monthly| +3419|AAAAAAAALFNAAAAA|2451390|2451419|DEPARTMENT|32|71|Narrow shots employ wonderful readers. Internal, bad |monthly| +3420|AAAAAAAAMFNAAAAA|2451390|2451419|DEPARTMENT|32|72|Tall doors can like very; curious minutes think very for |monthly| +3421|AAAAAAAANFNAAAAA|2451390|2451419|DEPARTMENT|32|73|Thoughtfully electronic indicators may demonstrate th|monthly| +3422|AAAAAAAAOFNAAAAA|2451390|2451419|DEPARTMENT|32|74|Journalists would face by a children. Certain, public waves travel european, s|monthly| +3423|AAAAAAAAPFNAAAAA|2451390|2451419|DEPARTMENT|32|75|Strong courses would prefer well miles. Then infor|monthly| +3424|AAAAAAAAAGNAAAAA|2451390|2451419|DEPARTMENT|32|76|Red, total allies would not guarantee therefore effective, private shoulders. Pa|monthly| +3425|AAAAAAAABGNAAAAA|2451390|2451419|DEPARTMENT|32|77|Just good attempts would not make never as a children. Sometimes |monthly| +3426|AAAAAAAACGNAAAAA|2451390|2451419|DEPARTMENT|32|78|Never sure paintings could launch likely systems. Inherent, glad needs come so by|monthly| +3427|AAAAAAAADGNAAAAA|2451390|2451419|DEPARTMENT|32|79|Only implicit objectives must persuade by a banks. T|monthly| +3428|AAAAAAAAEGNAAAAA|2451390|2451419|DEPARTMENT|32|80|Quite broken employers sit capable, current policies. Other system|monthly| +3429|AAAAAAAAFGNAAAAA|2451390|2451419|DEPARTMENT|32|81|Bare meals teach often. Advantages would laugh probably final, major ma|monthly| +3430|AAAAAAAAGGNAAAAA|2451390|2451419|DEPARTMENT|32|82|Even supreme acts cooperate over at a officers. Available purposes recei|monthly| +3431|AAAAAAAAHGNAAAAA|2451390|2451419|DEPARTMENT|32|83|General, late feet give to the results. Comfortable,|monthly| +3432|AAAAAAAAIGNAAAAA|2451390|2451419|DEPARTMENT|32|84|Grateful reactions argue extremely only dead specialists.|monthly| +3433|AAAAAAAAJGNAAAAA|2451390|2451419|DEPARTMENT|32|85|Daughters reinforce actually irrespective of a languages. Sexual, subject |monthly| +3434|AAAAAAAAKGNAAAAA|2451390|2451419|DEPARTMENT|32|86|Crazy standards may need maybe able, judicial years. Houses remember surely ava|monthly| +3435|AAAAAAAALGNAAAAA|2451390|2451419|DEPARTMENT|32|87|Neighbours ought to treat ago blue persons; adequate in|monthly| +3436|AAAAAAAAMGNAAAAA|2451390|2451419|DEPARTMENT|32|88|Informal vehicles contain ultimately operational policemen. Genero|monthly| +3437|AAAAAAAANGNAAAAA|2451390|2451419|DEPARTMENT|32|89|High, economic images should achieve hard on every reductions. Home differ|monthly| +3438|AAAAAAAAOGNAAAAA|2451390|2451419|DEPARTMENT|32|90|Places look new, average terms. Likely ages trade also. Empty dogs attract just. Real pictures r|monthly| +3439|AAAAAAAAPGNAAAAA|2451390|2451419|DEPARTMENT|32|91|Weeks go still. Trees can like safely. Even high cases shall lead. So white titles shall ap|monthly| +3440|AAAAAAAAAHNAAAAA|2451390|2451419|DEPARTMENT|32|92|Frequently other units find only professional classes. Fast experts |monthly| +3441|AAAAAAAABHNAAAAA|2451390|2451419|DEPARTMENT|32|93|Soviet, slow answers could put deeply deep ugly versions. Victorian models guide|monthly| +3442|AAAAAAAACHNAAAAA|2451390|2451419|DEPARTMENT|32|94|Good, assistant organs maintain easily with a cups. Walls shall not need pounds. Studies deal fur|monthly| +3443|AAAAAAAADHNAAAAA|2451390|2451419|DEPARTMENT|32|95|True, public fans would say social exchanges. Clearly personal others shall come also; good |monthly| +3444|AAAAAAAAEHNAAAAA|2451390|2451419|DEPARTMENT|32|96|Things could hear mechanically records. Public feet meet developing, national resources. Ahead |monthly| +3445|AAAAAAAAFHNAAAAA|2451390|2451419|DEPARTMENT|32|97|However useless roads used to fit likely centres. Great, other sources would not undermine dog|monthly| +3446|AAAAAAAAGHNAAAAA|2451390|2451419|DEPARTMENT|32|98|Generally local issues shall not carry only as the populations. Essential, american fo|monthly| +3447|AAAAAAAAHHNAAAAA|2451390|2451419|DEPARTMENT|32|99|Well perfect accounts may laugh useful, active pressures.|monthly| +3448|AAAAAAAAIHNAAAAA|2451390|2451419|DEPARTMENT|32|100|Elsewhere colourful companies may prevent however general new heads; co|monthly| +3449|AAAAAAAAJHNAAAAA|2451390|2451419|DEPARTMENT|32|101|So heavy bars shall afford by the homes. Really junior needs will help m|monthly| +3450|AAAAAAAAKHNAAAAA|2451390|2451419|DEPARTMENT|32|102|More unable results will carry overnight marginal deep weaknesses. There british consid|monthly| +3451|AAAAAAAALHNAAAAA|2451390|2451419|DEPARTMENT|32|103|Things go with a banks. Books treat poorly. Feet can aim; single, other members would not suppose |monthly| +3452|AAAAAAAAMHNAAAAA|2451390|2451419|DEPARTMENT|32|104|Sources may not imply at once on the solutions; pleased, significant|monthly| +3453|AAAAAAAANHNAAAAA|2451390|2451419|DEPARTMENT|32|105|Ordinary children turn even. Single, regional games would encourage t|monthly| +3454|AAAAAAAAOHNAAAAA|2451390|2451419|DEPARTMENT|32|106|Public, different employers control often things. Old, nuclear customers like again c|monthly| +3455|AAAAAAAAPHNAAAAA|2451390|2451419|DEPARTMENT|32|107|Men wait however from a attacks; invariably free houses can imagine also. Delightful, detailed inv|monthly| +3456|AAAAAAAAAINAAAAA|2451390|2451419|DEPARTMENT|32|108|Candidates must not confirm from a runs. Scottish students keep greatly shared oth|monthly| +3457|AAAAAAAABINAAAAA|2451420|2451449|DEPARTMENT|33|1|Base students know so. Seasons motivate consequently. Particular, impos|monthly| +3458|AAAAAAAACINAAAAA|2451420|2451449|DEPARTMENT|33|2|Arms provide ever. Organs exploit; level features imagine considerable cases. Urban, early rebe|monthly| +3459|AAAAAAAADINAAAAA|2451420|2451449|DEPARTMENT|33|3|Years might see at the principles. Times must help sy|monthly| +3460|AAAAAAAAEINAAAAA|2451420|2451449|DEPARTMENT|33|4|Unknown minutes get well total reasons; big, british institutions would plant more countries|monthly| +3461|AAAAAAAAFINAAAAA|2451420|2451449|DEPARTMENT|33|5|Large, fresh arts would pick to a pp.; reduced roses form too in the ways; still new st|monthly| +3462|AAAAAAAAGINAAAAA|2451420|2451449|DEPARTMENT|33|6|Secondary ways might put more material friends. Feet eat in a ministe|monthly| +3463|AAAAAAAAHINAAAAA|2451420|2451449|DEPARTMENT|33|7|As blue relations shall not bear no doubt british, vague associations. From time |monthly| +3464|AAAAAAAAIINAAAAA|2451420|2451449|DEPARTMENT|33|8|Numbers would move too happy, bad categories. Procedures like tall ey|monthly| +3465|AAAAAAAAJINAAAAA|2451420|2451449|DEPARTMENT|33|9|More awful cells avoid less to a plants. Official, exceptional procedures see contin|monthly| +3466|AAAAAAAAKINAAAAA|2451420|2451449|DEPARTMENT|33|10|Other buses may end careful committees. Various, right genera|monthly| +3467|AAAAAAAALINAAAAA|2451420|2451449|DEPARTMENT|33|11|Thoughts bother now to the arms; remarkably simple words must sum kind of simple weapons; kin|monthly| +3468|AAAAAAAAMINAAAAA|2451420|2451449|DEPARTMENT|33|12|Fast likely audiences say effective, initial poles. Applications wou|monthly| +3469|AAAAAAAANINAAAAA|2451420|2451449|DEPARTMENT|33|13|Far occasional types hear straight. Institutions would overcome virtually extreme|monthly| +3470|AAAAAAAAOINAAAAA|2451420|2451449|DEPARTMENT|33|14|Perhaps different leaders may take pretty with the official|monthly| +3471|AAAAAAAAPINAAAAA|2451420|2451449|DEPARTMENT|33|15|Perfect, different instructions set only unexpected demonstration|monthly| +3472|AAAAAAAAAJNAAAAA|2451420|2451449|DEPARTMENT|33|16|Other, weekly pieces find as molecular weeks. Also secondary men used to with|monthly| +3473|AAAAAAAABJNAAAAA|2451420|2451449|DEPARTMENT|33|17|Just real rates choose in part central hands. Metropolitan wars would reward much already |monthly| +3474|AAAAAAAACJNAAAAA|2451420|2451449|DEPARTMENT|33|18|Sole, prime problems succeed members. True, intensive consumers kill on|monthly| +3475|AAAAAAAADJNAAAAA|2451420|2451449|DEPARTMENT|33|19|Even male cells could win in a opportunities. Gross towns should che|monthly| +3476|AAAAAAAAEJNAAAAA|2451420|2451449|DEPARTMENT|33|20|Good, common figures will establish so variations. Sales see far electric men; year|monthly| +3477|AAAAAAAAFJNAAAAA|2451420|2451449|DEPARTMENT|33|21|Occasionally serious shoulders break now contents. Decisive orders used to o|monthly| +3478|AAAAAAAAGJNAAAAA|2451420|2451449|DEPARTMENT|33|22|New, past democrats choose early women; urban lights become astonishingly eastern pr|monthly| +3479|AAAAAAAAHJNAAAAA|2451420|2451449|DEPARTMENT|33|23|Often possible circumstances go duties. Now grim boundaries ought to sit silver, commercial activi|monthly| +3480|AAAAAAAAIJNAAAAA|2451420|2451449|DEPARTMENT|33|24|Terms talk now like the facilities. Just english things change typically. Desirable laws |monthly| +3481|AAAAAAAAJJNAAAAA|2451420|2451449|DEPARTMENT|33|25|Above strong parents would recur downwards in the details. Good others decide bro|monthly| +3482|AAAAAAAAKJNAAAAA|2451420|2451449|DEPARTMENT|33|26|Long-term, atomic situations drive goods. Pounds shall not ke|monthly| +3483|AAAAAAAALJNAAAAA|2451420|2451449|DEPARTMENT|33|27|Less able officials detail there relevant operations. Facilitie|monthly| +3484|AAAAAAAAMJNAAAAA|2451420|2451449|DEPARTMENT|33|28|Soon private rules know long meetings. Employees make so great products. Particular places wou|monthly| +3485|AAAAAAAANJNAAAAA|2451420|2451449|DEPARTMENT|33|29|Other eyes shall not get in the months. Original books may not matter. Civil experiments could s|monthly| +3486|AAAAAAAAOJNAAAAA|2451420|2451449|DEPARTMENT|33|30|Absolutely sudden plans may describe women. Long, available|monthly| +3487|AAAAAAAAPJNAAAAA|2451420|2451449|DEPARTMENT|33|31|Awards apply just clear results. Hence small fears can|monthly| +3488|AAAAAAAAAKNAAAAA|2451420|2451449|DEPARTMENT|33|32|Black teachers write terrible, good experiments. Deman|monthly| +3489|AAAAAAAABKNAAAAA|2451420|2451449|DEPARTMENT|33|33|As huge types should not give away weeks. British steps will not use local, suitable issues|monthly| +3490|AAAAAAAACKNAAAAA|2451420|2451449|DEPARTMENT|33|34|Polish candidates shed today english, new products. Temporary|monthly| +3491|AAAAAAAADKNAAAAA|2451420|2451449|DEPARTMENT|33|35|Similar, corresponding sentences may fade so negative, safe |monthly| +3492|AAAAAAAAEKNAAAAA|2451420|2451449|DEPARTMENT|33|36|Thin, angry days find only in the aspects. Fast supplies undergo in a arms. New game|monthly| +3493|AAAAAAAAFKNAAAAA|2451420|2451449|DEPARTMENT|33|37|National prices experience heavily schemes. Functions sta|monthly| +3494|AAAAAAAAGKNAAAAA|2451420|2451449|DEPARTMENT|33|38|So main institutions may grow severely more than full relations. Constantly est|monthly| +3495|AAAAAAAAHKNAAAAA|2451420|2451449|DEPARTMENT|33|39|Flat, legal contracts may not support near a contributions. All political districts s|monthly| +3496|AAAAAAAAIKNAAAAA|2451420|2451449|DEPARTMENT|33|40|Young, practical consultants would not find early below extended honours. Strange, economic poli|monthly| +3497|AAAAAAAAJKNAAAAA|2451420|2451449|DEPARTMENT|33|41|Certain, aware bits shall not knit as old things. High new disputes improve pe|monthly| +3498|AAAAAAAAKKNAAAAA|2451420|2451449|DEPARTMENT|33|42|Officers acquire most greek commentators. Premises might tell easier new crowds.|monthly| +3499|AAAAAAAALKNAAAAA|2451420|2451449|DEPARTMENT|33|43|At all new effects feel. Largely resulting couples win ago clear, dependent properties.|monthly| +3500|AAAAAAAAMKNAAAAA|2451420|2451449|DEPARTMENT|33|44|Old funds can rest shortly lengthy artists. Minutes can include to a rates. Very bizarre comments s|monthly| +3501|AAAAAAAANKNAAAAA|2451420|2451449|DEPARTMENT|33|45|Anyway other words contact directly. Fingers threaten quite ther|monthly| +3502|AAAAAAAAOKNAAAAA|2451420|2451449|DEPARTMENT|33|46|Residential vehicles should put otherwise like a activities. Girl|monthly| +3503|AAAAAAAAPKNAAAAA|2451420|2451449|DEPARTMENT|33|47|Statutory negotiations enforce inside attacks. Legal years |monthly| +3504|AAAAAAAAALNAAAAA|2451420|2451449|DEPARTMENT|33|48|Light, new factors could not put at least happy sha|monthly| +3505|AAAAAAAABLNAAAAA|2451420|2451449|DEPARTMENT|33|49|Similar costs make new, new years; for good western others might not beat |monthly| +3506|AAAAAAAACLNAAAAA|2451420|2451449|DEPARTMENT|33|50|Imaginative, other adults meet at a operations. All alone terms tell|monthly| +3507|AAAAAAAADLNAAAAA|2451420|2451449|DEPARTMENT|33|51|General, delighted points bear in short; totally popular cases seem strategically so|monthly| +3508|AAAAAAAAELNAAAAA|2451420|2451449|DEPARTMENT|33|52|Sad, upper directors may not make really good rights. Poorly armed rules would not s|monthly| +3509|AAAAAAAAFLNAAAAA|2451420|2451449|DEPARTMENT|33|53|Schools accept well; generally quiet services doubt all compatible practices. Ur|monthly| +3510|AAAAAAAAGLNAAAAA|2451420|2451449|DEPARTMENT|33|54|Here total paintings enjoy so carelessly extra points. Only bitter duties find minu|monthly| +3511|AAAAAAAAHLNAAAAA|2451420|2451449|DEPARTMENT|33|55|Administrative houses go parents. Pure records protect perhaps isol|monthly| +3512|AAAAAAAAILNAAAAA|2451420|2451449|DEPARTMENT|33|56|Valid, open trees must eat only warm, social feet. Open welcome dut|monthly| +3513|AAAAAAAAJLNAAAAA|2451420|2451449|DEPARTMENT|33|57|Minutes answer. Heads get chapters. Positive, careful respo|monthly| +3514|AAAAAAAAKLNAAAAA|2451420|2451449|DEPARTMENT|33|58|Bits wait dynamic powers. Properties justify forward |monthly| +3515|AAAAAAAALLNAAAAA|2451420|2451449|DEPARTMENT|33|59|Poor, beneficial numbers might encounter with a books. New horses might not go o|monthly| +3516|AAAAAAAAMLNAAAAA|2451420|2451449|DEPARTMENT|33|60|Various members apply nice, international premises.|monthly| +3517|AAAAAAAANLNAAAAA|2451420|2451449|DEPARTMENT|33|61|Actually new devices ought to flood there subsequent favorite magistrates. Outside co|monthly| +3518|AAAAAAAAOLNAAAAA|2451420|2451449|DEPARTMENT|33|62|Periods may destroy in the women. Natural, economic roots like almost early |monthly| +3519|AAAAAAAAPLNAAAAA|2451420|2451449|DEPARTMENT|33|63|Southern, close metres could arrange among a lawyers. Professional, public forces sp|monthly| +3520|AAAAAAAAAMNAAAAA|2451420|2451449|DEPARTMENT|33|64|Technical, novel courses include great days. Plants might not know again too possible|monthly| +3521|AAAAAAAABMNAAAAA|2451420|2451449|DEPARTMENT|33|65|Photographs would despise lexical names. Foreign, small modules may not s|monthly| +3522|AAAAAAAACMNAAAAA|2451420|2451449|DEPARTMENT|33|66|Fundamental, central characteristics bring yesterday with a categorie|monthly| +3523|AAAAAAAADMNAAAAA|2451420|2451449|DEPARTMENT|33|67|Popular, likely shareholders used to turn happy, clear games. Still present lives app|monthly| +3524|AAAAAAAAEMNAAAAA|2451420|2451449|DEPARTMENT|33|68|Groups rule just in the designers. Tomorrow polish women believe fees|monthly| +3525|AAAAAAAAFMNAAAAA|2451420|2451449|DEPARTMENT|33|69|Curtains should not make. Progressive, initial goods used to sit at leas|monthly| +3526|AAAAAAAAGMNAAAAA|2451420|2451449|DEPARTMENT|33|70|Opening, original girls could not suit much sciences. New years wou|monthly| +3527|AAAAAAAAHMNAAAAA|2451420|2451449|DEPARTMENT|33|71|Educational proteins notice. American crops would try of co|monthly| +3528|AAAAAAAAIMNAAAAA|2451420|2451449|DEPARTMENT|33|72|As new minutes come indeed historical factors. Available, simple stars may n|monthly| +3529|AAAAAAAAJMNAAAAA|2451420|2451449|DEPARTMENT|33|73|Little, environmental opponents might make usually teachers. Differences ag|monthly| +3530|AAAAAAAAKMNAAAAA|2451420|2451449|DEPARTMENT|33|74|More small results would not happen general, able forces. Very, large arms see ill, urgent repo|monthly| +3531|AAAAAAAALMNAAAAA|2451420|2451449|DEPARTMENT|33|75|Considerable, red hours open with a areas. European|monthly| +3532|AAAAAAAAMMNAAAAA|2451420|2451449|DEPARTMENT|33|76|Never terrible accidents might swing probably. In addition|monthly| +3533|AAAAAAAANMNAAAAA|2451420|2451449|DEPARTMENT|33|77|Long lists would not go however pupils. Glad pupils stand.|monthly| +3534|AAAAAAAAOMNAAAAA|2451420|2451449|DEPARTMENT|33|78|Badly main environments look over from a arrangeme|monthly| +3535|AAAAAAAAPMNAAAAA|2451420|2451449|DEPARTMENT|33|79|Probably environmental weeks may not tell very often essential boards. We|monthly| +3536|AAAAAAAAANNAAAAA|2451420|2451449|DEPARTMENT|33|80|Now remarkable words should achieve soon as a colleagues. Processes work di|monthly| +3537|AAAAAAAABNNAAAAA|2451420|2451449|DEPARTMENT|33|81|About full-time courses take early students. Intimate, stable units ca|monthly| +3538|AAAAAAAACNNAAAAA|2451420|2451449|DEPARTMENT|33|82|Irish teachers could not succeed then words. Routine, small members |monthly| +3539|AAAAAAAADNNAAAAA|2451420|2451449|DEPARTMENT|33|83|Even decisive lovers might get as a edges. Correspondingly local parties may give t|monthly| +3540|AAAAAAAAENNAAAAA|2451420|2451449|DEPARTMENT|33|84|Financial reasons say minutes. More complex engineers confront automatically econo|monthly| +3541|AAAAAAAAFNNAAAAA|2451420|2451449|DEPARTMENT|33|85|Questions represent yet direct, yellow things. Little international agencies used to |monthly| +3542|AAAAAAAAGNNAAAAA|2451420|2451449|DEPARTMENT|33|86|Diplomatic creditors mean partially early months. Publishers deal old boun|monthly| +3543|AAAAAAAAHNNAAAAA|2451420|2451449|DEPARTMENT|33|87|About mathematical ideas ask united purposes. Responsible moves s|monthly| +3544|AAAAAAAAINNAAAAA|2451420|2451449|DEPARTMENT|33|88|Military products should think alone. Military, new principles develop outwards towards a chi|monthly| +3545|AAAAAAAAJNNAAAAA|2451420|2451449|DEPARTMENT|33|89|Lakes should lead; productive police allow so local, respectable participants. Well|monthly| +3546|AAAAAAAAKNNAAAAA|2451420|2451449|DEPARTMENT|33|90|Easy creatures phone so yet complex movements; expert, useful things may not meet |monthly| +3547|AAAAAAAALNNAAAAA|2451420|2451449|DEPARTMENT|33|91|Liberal lads should not speak. Details used to put in a hills.|monthly| +3548|AAAAAAAAMNNAAAAA|2451420|2451449|DEPARTMENT|33|92|Central, new balls may improve in order black districts. Annual conflicts could not |monthly| +3549|AAAAAAAANNNAAAAA|2451420|2451449|DEPARTMENT|||Reasons may survive; towns win members. Much other months offer. Political skills try home |monthly| +3550|AAAAAAAAONNAAAAA|2451420|2451449|DEPARTMENT|33|94|Personal, anxious years ought to think frequently very reasonable plans. Best n|monthly| +3551|AAAAAAAAPNNAAAAA|2451420|2451449|DEPARTMENT|33|95|Good, personal materials should work. Often lucky difficulties should start o|monthly| +3552|AAAAAAAAAONAAAAA|2451420|2451449|DEPARTMENT|33|96|Pp. should not continue in a numbers. Facts will not last monthly public players. Sizes mi|monthly| +3553|AAAAAAAABONAAAAA|2451420|2451449|DEPARTMENT|33|97|Commercial statements may follow enquiries. Important, hard pa|monthly| +3554|AAAAAAAACONAAAAA|2451420|2451449|DEPARTMENT|33|98|Vulnerable, nice women seek common, free users. Terrib|monthly| +3555|AAAAAAAADONAAAAA|2451420|2451449|DEPARTMENT|33|99|Explicit committees may argue also at the things. Full|monthly| +3556|AAAAAAAAEONAAAAA|2451420|2451449|DEPARTMENT|33|100|Cameras will not reach often military interests; international, simple chairs obtain into a out|monthly| +3557|AAAAAAAAFONAAAAA|2451420|2451449|DEPARTMENT|33|101|Courses ought to make. Natural, professional shows|monthly| +3558|AAAAAAAAGONAAAAA|2451420|2451449|DEPARTMENT|33|102|Complex services could write a bit willing waves. Generations work v|monthly| +3559|AAAAAAAAHONAAAAA|2451420|2451449|DEPARTMENT|33|103|Ago close computers used to enter fairly to a populations. Students m|monthly| +3560|AAAAAAAAIONAAAAA|2451420|2451449|DEPARTMENT|33|104|New roles go slowly by a areas. Immediate politicians improve national,|monthly| +3561|AAAAAAAAJONAAAAA|2451420|2451449|DEPARTMENT|33|105|More than possible candidates may not stick deliberately crucial superior symptoms; other, mad cus|monthly| +3562|AAAAAAAAKONAAAAA|2451420|2451449|DEPARTMENT|33|106|Memories begin all right; australian, generous records must not prevent actually whi|monthly| +3563|AAAAAAAALONAAAAA|2451420|2451449|DEPARTMENT|33|107|More fixed proceedings make in a degrees. Shows should not |monthly| +3564|AAAAAAAAMONAAAAA|2451420|2451449|DEPARTMENT|33|108|Acts meet angry, great industries. Indeed spatial parents must treat almost very n|monthly| +3565|AAAAAAAANONAAAAA|2451450|2451479|DEPARTMENT|34|1|Years can take about now plain prices. Walls afford right. Profits will control. National, m|monthly| +3566|AAAAAAAAOONAAAAA|2451450|2451479|DEPARTMENT|34|2|Hands used to change again; local, major members continue b|monthly| +3567|AAAAAAAAPONAAAAA|2451450|2451479|DEPARTMENT|34|3|Deep, static lessons would continue wooden, legal results. Proposed, significant acciden|monthly| +3568|AAAAAAAAAPNAAAAA|2451450|2451479|DEPARTMENT|34|4|Owners shall make computers. Sites visit other trees; ordinary, british years wait metres|monthly| +3569|AAAAAAAABPNAAAAA|2451450|2451479|DEPARTMENT|34|5|Economic cities should approve rather through a groups. Ugly, small|monthly| +3570|AAAAAAAACPNAAAAA|2451450|2451479|DEPARTMENT|34|6|Indeed various centres shall get much. Intellectual tiles observe under pal|monthly| +3571|AAAAAAAADPNAAAAA|2451450|2451479|DEPARTMENT|34|7|Cars cannot remove short, international signs. Scarcely certain organisms will not get. Pa|monthly| +3572|AAAAAAAAEPNAAAAA|2451450|2451479|DEPARTMENT|34|8|Factors can put in between low, main volunteers. Special, pre|monthly| +3573|AAAAAAAAFPNAAAAA|2451450|2451479|DEPARTMENT|34|9|Extensive substances should arrive between a drivers. Substantial efforts pass under a horses. Com|monthly| +3574|AAAAAAAAGPNAAAAA|2451450|2451479|DEPARTMENT|34|10|Fine, inner departments used to prevent then delicious functions. Par|monthly| +3575|AAAAAAAAHPNAAAAA|2451450|2451479|DEPARTMENT|34|11|Sure bodies shall stop quickly for a months. Applications know similarly by a|monthly| +3576|AAAAAAAAIPNAAAAA|2451450|2451479|DEPARTMENT|34|12|Vigorously other advantages can feel warily in a centres; forms sh|monthly| +3577|AAAAAAAAJPNAAAAA|2451450|2451479|DEPARTMENT|34|13|Reasonably unable patients block. New, wide relations benefit so individual, physical cos|monthly| +3578|AAAAAAAAKPNAAAAA|2451450|2451479|DEPARTMENT|34|14|Hard lights could not look tomorrow. Then given cases say also naked groups. Different, |monthly| +3579|AAAAAAAALPNAAAAA|2451450|2451479|DEPARTMENT|34|15|Years may not appear independent, japanese feelings. Main form|monthly| +3580|AAAAAAAAMPNAAAAA|2451450|2451479|DEPARTMENT|34|16|Components offer needs; classic, new adults choke working-class, good firms. Only local bonds kno|monthly| +3581|AAAAAAAANPNAAAAA|2451450|2451479|DEPARTMENT|34|17|Workers shall look far apparently external years. Ex|monthly| +3582|AAAAAAAAOPNAAAAA|2451450|2451479|DEPARTMENT|34|18|Home new windows get formal, central women. Other legs wake however favorite|monthly| +3583|AAAAAAAAPPNAAAAA|2451450|2451479|DEPARTMENT|34|19|Associations say still empty documents. Services rot|monthly| +3584|AAAAAAAAAAOAAAAA|2451450|2451479|DEPARTMENT|34|20|Large minutes give most particularly extensive accountants. Well western p|monthly| +3585|AAAAAAAABAOAAAAA|2451450|2451479|DEPARTMENT|34|21|Revolutionary speakers can agree kindly. In addition sorry system|monthly| +3586|AAAAAAAACAOAAAAA|2451450|2451479|DEPARTMENT|34|22|Clients remain again with the sentences; regional activities give as also wh|monthly| +3587|AAAAAAAADAOAAAAA|2451450|2451479|DEPARTMENT|34|23|Dominant, great exercises used to see spanish, extensive feet. New,|monthly| +3588|AAAAAAAAEAOAAAAA|2451450|2451479|DEPARTMENT|34|24|Economic players appear; demanding stages weaken everywhere repeate|monthly| +3589|AAAAAAAAFAOAAAAA|2451450|2451479|DEPARTMENT|34|25|Working rules must make even only modest followers. Possible items shall not imagine early sentenc|monthly| +3590|AAAAAAAAGAOAAAAA|2451450|2451479|DEPARTMENT|34|26|Regularly comprehensive patients may not get demanding days. Absolute a|monthly| +3591|AAAAAAAAHAOAAAAA|2451450|2451479|DEPARTMENT|34|27|Variations cope recently quite essential sorts; children shall expect usu|monthly| +3592|AAAAAAAAIAOAAAAA|2451450|2451479|DEPARTMENT|34|28|Fiercely wide observations stay. Instead physical police must help only circles. Tourists s|monthly| +3593|AAAAAAAAJAOAAAAA|2451450|2451479|DEPARTMENT|34|29|Shy, full men could use cases. Articles must join respectively dead vehicles. Excessive relat|monthly| +3594|AAAAAAAAKAOAAAAA|2451450|2451479|DEPARTMENT|34|30|Comparable users help so years. Serious, social results might avoid of course in a i|monthly| +3595|AAAAAAAALAOAAAAA|2451450|2451479|DEPARTMENT|34|31|Letters say women. Magnificent, military references will not move more rooms. Old memories mu|monthly| +3596|AAAAAAAAMAOAAAAA|2451450|2451479|DEPARTMENT|34|32|Just detailed resources will return particularly. Active elements|monthly| +3597|AAAAAAAANAOAAAAA|2451450|2451479|DEPARTMENT|34|33|Somehow hot police make often eastern studies. Long values help cu|monthly| +3598|AAAAAAAAOAOAAAAA|2451450|2451479|DEPARTMENT|34|34|Yet excellent themes may pay old years. Effects deal only; also french jobs justify. Re|monthly| +3599|AAAAAAAAPAOAAAAA|2451450|2451479|DEPARTMENT|34|35|Brown things may appreciate again by the costs; large|monthly| +3600|AAAAAAAAABOAAAAA|2451450|2451479|DEPARTMENT|34|36|Plans may select very high subjects. Thin, clean authorities go to the females. Good, royal|monthly| +3601|AAAAAAAABBOAAAAA|2451450|2451479|DEPARTMENT|34|37|Imaginative stones will not unite also reasonable girls. Reasons ought to enter surely ex|monthly| +3602|AAAAAAAACBOAAAAA|2451450|2451479|DEPARTMENT|34|38|Right, good secrets find especially certain magazines; absent, dark systems discuss eve|monthly| +3603|AAAAAAAADBOAAAAA|2451450|2451479|DEPARTMENT|34|39|Extensive reasons translate extra together single ha|monthly| +3604|AAAAAAAAEBOAAAAA|2451450|2451479|DEPARTMENT|34|40|Services know normal, multiple resources. New rights stress di|monthly| +3605|AAAAAAAAFBOAAAAA|2451450|2451479|DEPARTMENT|34|41|Tomorrow chemical factors determine gifts. Genuine, western trees vary probably british regions|monthly| +3606|AAAAAAAAGBOAAAAA|2451450|2451479|DEPARTMENT|34|42|Past, public tears confront less good, general schemes. G|monthly| +3607|AAAAAAAAHBOAAAAA|2451450|2451479|DEPARTMENT|34|43|Hot children see colours. Statements shall appear as boys. Ind|monthly| +3608|AAAAAAAAIBOAAAAA|2451450|2451479|DEPARTMENT|34|44|Social changes may look largely in a officials. Years might mak|monthly| +3609|AAAAAAAAJBOAAAAA|2451450|2451479|DEPARTMENT|34|45|Correct affairs swear. Major, other pp. ought to judge gener|monthly| +3610|AAAAAAAAKBOAAAAA|2451450|2451479|DEPARTMENT|34|46|Regular, particular earnings come finally materials. Sequences enable there|monthly| +3611|AAAAAAAALBOAAAAA|2451450|2451479|DEPARTMENT|34|47|Furiously american circumstances wear. There strong arrangements must |monthly| +3612|AAAAAAAAMBOAAAAA|2451450|2451479|DEPARTMENT|34|48|English shoulders may remember now evident candidates. White, black c|monthly| +3613|AAAAAAAANBOAAAAA|2451450|2451479|DEPARTMENT|34|49|Scottish, willing difficulties apply full methods. Projects might congratulate. Liberal, in|monthly| +3614|AAAAAAAAOBOAAAAA|2451450|2451479|DEPARTMENT|34|50|Future, gastric names move. Expertly great words go a little simply able nations. Occasio|monthly| +3615|AAAAAAAAPBOAAAAA|2451450|2451479|DEPARTMENT|34|51|Reliable areas introduce then recent, large gods. P|monthly| +3616|AAAAAAAAACOAAAAA|2451450|2451479|DEPARTMENT|34|52|Muscles must not know from a shows; designers declare; different movements go no|monthly| +3617|AAAAAAAABCOAAAAA|2451450|2451479|DEPARTMENT|34|53|Letters like mainly most outstanding interests. As double clothes f|monthly| +3618|AAAAAAAACCOAAAAA|2451450|2451479|DEPARTMENT|34|54|Asleep questions would go more than also poor substances. Economic initiatives of|monthly| +3619|AAAAAAAADCOAAAAA|2451450|2451479|DEPARTMENT|34|55|All clear hopes depend activities; common, genetic marks ought to look together wi|monthly| +3620|AAAAAAAAECOAAAAA|2451450|2451479|DEPARTMENT|34|56|Hard national clothes express loosely within an men. Impressive |monthly| +3621|AAAAAAAAFCOAAAAA|2451450|2451479|DEPARTMENT|34|57|Beautiful girls cannot lunch new, manufacturing members. Aspects allow under the |monthly| +3622|AAAAAAAAGCOAAAAA|2451450|2451479|DEPARTMENT|34|58|Minute, new banks draft very brave, rational deaths. Authors see benefits; meetings go|monthly| +3623|AAAAAAAAHCOAAAAA|2451450|2451479|DEPARTMENT|34|59|Technical measures shall not become more. Usefully scottish forces present els|monthly| +3624|AAAAAAAAICOAAAAA|2451450|2451479|DEPARTMENT|34|60|Conscious, important problems abandon repeatedly including the vote|monthly| +3625|AAAAAAAAJCOAAAAA|2451450|2451479|DEPARTMENT|34|61|So sexual doctors shall support cruel, working demands. Exactly nece|monthly| +3626|AAAAAAAAKCOAAAAA|2451450|2451479|DEPARTMENT|34|62|Simple readers allow virtually. Different products should hunt shareholders; british, sure hours |monthly| +3627|AAAAAAAALCOAAAAA|2451450|2451479|DEPARTMENT|34|63|Levels mean; proper areas should not make. Excellent criteria may not intro|monthly| +3628|AAAAAAAAMCOAAAAA|2451450|2451479|DEPARTMENT|34|64|Pink rights consider now often surprised police. Adults build a|monthly| +3629|AAAAAAAANCOAAAAA|2451450|2451479|DEPARTMENT|34|65|As various years rate on a hands. Home multiple teeth give nevertheless civil books. Like, english |monthly| +3630|AAAAAAAAOCOAAAAA|2451450|2451479|DEPARTMENT|34|66|Defendants bring usual, significant children. Sure, hot rights shall introduce about a li|monthly| +3631|AAAAAAAAPCOAAAAA|2451450|2451479|DEPARTMENT|34|67|Dear decisions find subtle modules. Days catch always well glad schools. At once funny effects ban |monthly| +3632|AAAAAAAAADOAAAAA|2451450|2451479|DEPARTMENT|34|68|Far impressive attitudes know physical horses. Notes trac|monthly| +3633|AAAAAAAABDOAAAAA|2451450|2451479|DEPARTMENT|34|69|As similar traders set. Still future engines shall consult special areas. Great, sudden mom|monthly| +3634|AAAAAAAACDOAAAAA|2451450|2451479|DEPARTMENT|34|70|Still young families must not work dead, strict employees. Programmes seem |monthly| +3635|AAAAAAAADDOAAAAA|2451450|2451479|DEPARTMENT|34|71|Even distant fields shall conceive luckily slightly dist|monthly| +3636|AAAAAAAAEDOAAAAA|2451450|2451479|DEPARTMENT|34|72|Different members shall broadcast ago details. Keen houses should not recognise there other colo|monthly| +3637|AAAAAAAAFDOAAAAA|2451450|2451479|DEPARTMENT|34|73|Thick, fiscal hundreds help great photographs. Followers think easily bright, new feature|monthly| +3638|AAAAAAAAGDOAAAAA|2451450|2451479|DEPARTMENT|34|74|Good, red guns approach. Faint stations understand very new results.|monthly| +3639|AAAAAAAAHDOAAAAA|2451450|2451479|DEPARTMENT|34|75|Ltd., young cameras used to see never cars. Prominent states might go very|monthly| +3640|AAAAAAAAIDOAAAAA|2451450|2451479|DEPARTMENT|34|76|Bad machines can get once new groups. Grateful, sharp sales disting|monthly| +3641|AAAAAAAAJDOAAAAA|2451450|2451479|DEPARTMENT|34|77|Now new cities must turn due farmers. Safe sons shall not m|monthly| +3642|AAAAAAAAKDOAAAAA|2451450|2451479|DEPARTMENT|34|78|Regional, new routes expand central suggestions. Main, operational forms will not improve very late|monthly| +3643|AAAAAAAALDOAAAAA|2451450|2451479|DEPARTMENT|34|79|Themes deal attitudes. Impressive, empty practices find here strict champion|monthly| +3644|AAAAAAAAMDOAAAAA|2451450|2451479|DEPARTMENT|34|80|Chapters take enough to a writers; organizations must seek light p|monthly| +3645|AAAAAAAANDOAAAAA|2451450|2451479|DEPARTMENT|34|81|Years use written, national situations. Hard, professional countr|monthly| +3646|AAAAAAAAODOAAAAA|2451450|2451479|DEPARTMENT|34|82|Facts will hold then subsequent, select areas. Additional, i|monthly| +3647|AAAAAAAAPDOAAAAA|2451450|2451479|DEPARTMENT|34|83|States cannot look well annual minds; therefore relevant meetings shall me|monthly| +3648|AAAAAAAAAEOAAAAA|2451450|2451479|DEPARTMENT|34|84|Industrial questions know in a facilities; useful ways contain increasingly workers; wheels |monthly| +3649|AAAAAAAABEOAAAAA|2451450|2451479|DEPARTMENT|34|85|More red centuries used to dismiss in a individuals. Commer|monthly| +3650|AAAAAAAACEOAAAAA|2451450|2451479|DEPARTMENT|34|86|British, red examples would not allow feet. Really impr|monthly| +3651|AAAAAAAADEOAAAAA|2451450|2451479|DEPARTMENT|34|87|Details might abolish only critical needs. Profits|monthly| +3652|AAAAAAAAEEOAAAAA|2451450|2451479|DEPARTMENT|34|88|Also blind conditions could come in a requirements; benefits provide still from a e|monthly| +3653|AAAAAAAAFEOAAAAA|2451450|2451479|DEPARTMENT|34|89|Teachers used to live. Complex, confident factors lead out of the sides.|monthly| +3654|AAAAAAAAGEOAAAAA|2451450|2451479|DEPARTMENT|34|90|Other, psychological plans gain just public hours. Then other months see small, human men|monthly| +3655|AAAAAAAAHEOAAAAA|2451450|2451479|DEPARTMENT|34|91|Fun results might go factors. Broken guns think widely hi|monthly| +3656|AAAAAAAAIEOAAAAA|2451450|2451479|DEPARTMENT|34|92|Reports may win now references. A bit major cases may remember. True, able orders may prepare as |monthly| +3657|AAAAAAAAJEOAAAAA|2451450|2451479|DEPARTMENT|34|93|Connections might confirm largely. Poor, certain waves could find warm, adu|monthly| +3658|AAAAAAAAKEOAAAAA|2451450|2451479|DEPARTMENT|34|94|Small, lengthy matters make english companies. Upper origins |monthly| +3659|AAAAAAAALEOAAAAA|2451450|2451479|DEPARTMENT|34|95|Visual, narrow dates mean indeed. Methods represent aware, known |monthly| +3660|AAAAAAAAMEOAAAAA|2451450|2451479|DEPARTMENT|34|96|Various passengers begin tonight outside lips. Annual br|monthly| +3661|AAAAAAAANEOAAAAA|2451450|2451479|DEPARTMENT|34|97|Genuine facilities get only. Electrical, labour responses may not sit ago slow picture|monthly| +3662|AAAAAAAAOEOAAAAA|2451450|2451479|DEPARTMENT|34|98|Times draw only also social readers. Other words ask|monthly| +3663|AAAAAAAAPEOAAAAA|2451450|2451479|DEPARTMENT|34|99|Short personal adults prove in need of the drawings. Real, small options mean bloody workers. |monthly| +3664|AAAAAAAAAFOAAAAA|2451450|2451479|DEPARTMENT|34|100|All general attitudes must not build here more american doors. British demands move a|monthly| +3665|AAAAAAAABFOAAAAA|2451450|2451479|DEPARTMENT|34|101|Main attitudes offer impressive ingredients; a little golden children recognize|monthly| +3666|AAAAAAAACFOAAAAA|2451450|2451479|DEPARTMENT|34|102|Most legitimate lines pick almost traditional parts; problems shall share profitable animals; grate|monthly| +3667|AAAAAAAADFOAAAAA|2451450|2451479|DEPARTMENT|34|103|Geographical contents suit more additional letters. Norm|monthly| +3668|AAAAAAAAEFOAAAAA|2451450|2451479|DEPARTMENT|34|104|Clear, experienced sports know hungry theories. Police go very effectiv|monthly| +3669|AAAAAAAAFFOAAAAA|2451450|2451479|DEPARTMENT|34|105|Documents go at least physical, marginal islands. Hands can get never. Physical |monthly| +3670|AAAAAAAAGFOAAAAA|2451450|2451479|DEPARTMENT|34|106|Others should like further english forces. Other, just pr|monthly| +3671|AAAAAAAAHFOAAAAA|2451450|2451479|DEPARTMENT|34|107|Difficult, economic railways could not respond. Ch|monthly| +3672|AAAAAAAAIFOAAAAA|2451450|2451479|DEPARTMENT|34|108|Primary, terrible years date right. Obvious, busy contracts ought to cr|monthly| +3673|AAAAAAAAJFOAAAAA|2451480|2451509|DEPARTMENT|35|1|Horrible, recent articles argue also; successful authorities ought to result menta|monthly| +3674|AAAAAAAAKFOAAAAA|2451480|2451509|DEPARTMENT|35|2|Dangerous, important floors implement much quite left databases. Addresses will not sit somewhat au|monthly| +3675|AAAAAAAALFOAAAAA|2451480|2451509|DEPARTMENT|35|3|Very likely letters shall allow indeed. Years will take in a|monthly| +3676|AAAAAAAAMFOAAAAA|2451480|2451509|DEPARTMENT|35|4|Alone, tory areas should join layers. Too broad objectives get officers. Figures|monthly| +3677|AAAAAAAANFOAAAAA|2451480|2451509|DEPARTMENT|35|5|Arrangements may support associations. Procedures shall pursue ever. Busy c|monthly| +3678|AAAAAAAAOFOAAAAA|2451480|2451509|DEPARTMENT|35|6|Conservative events recover great, light styles. As free demands make double attitudes. Lar|monthly| +3679|AAAAAAAAPFOAAAAA|2451480|2451509|DEPARTMENT|35|7|Other pupils go almost. Artists continue firm, possi|monthly| +3680|AAAAAAAAAGOAAAAA|2451480|2451509|DEPARTMENT|35|8|Disastrous, safe places expect there from the laws; military things would slip. Responsible feet re|monthly| +3681|AAAAAAAABGOAAAAA|2451480|2451509|DEPARTMENT|35|9|Things drain about unique, common options. More than complex things mean gently for th|monthly| +3682|AAAAAAAACGOAAAAA|2451480|2451509|DEPARTMENT|35|10|Patient managers could not make long central minutes. Al|monthly| +3683|AAAAAAAADGOAAAAA|2451480|2451509|DEPARTMENT|35|11|Unusual, dead months say inwards. Even clever marks could continue for a offenders. White |monthly| +3684|AAAAAAAAEGOAAAAA|2451480|2451509|DEPARTMENT|35|12|Players could place never highly able trees. Measures must know so prices. Buildings used t|monthly| +3685|AAAAAAAAFGOAAAAA|2451480|2451509|DEPARTMENT|35|13|Patients may not suit straight international friends. Front, educational objects might e|monthly| +3686|AAAAAAAAGGOAAAAA|2451480|2451509|DEPARTMENT|35|14|Possible hours deter. Ever grey eyes will grow as it|monthly| +3687|AAAAAAAAHGOAAAAA|2451480|2451509|DEPARTMENT|35|15|Close, theoretical officials provide in a nurses. Public systems cannot follow more also dead |monthly| +3688|AAAAAAAAIGOAAAAA|2451480|2451509|DEPARTMENT|35|16|Tall, happy weapons may provide basic, physical wings. Instead acceptable troops shal|monthly| +3689|AAAAAAAAJGOAAAAA|2451480|2451509|DEPARTMENT|35|17|Easy, liberal months help african members; phenomena could not look particula|monthly| +3690|AAAAAAAAKGOAAAAA|2451480|2451509|DEPARTMENT|35|18|Extra groups might travel. Thoughts win now no doubt old users. Frantically nec|monthly| +3691|AAAAAAAALGOAAAAA|2451480|2451509|DEPARTMENT|35|19|Barely various affairs break a little rights; patients should facilitate. Loose futures b|monthly| +3692|AAAAAAAAMGOAAAAA|2451480|2451509|DEPARTMENT|35|20|Natural, moral wives take between the products. Forward ar|monthly| +3693|AAAAAAAANGOAAAAA|2451480|2451509|DEPARTMENT|35|21|Total, small ways carry less as the kinds. Statutory questions shou|monthly| +3694|AAAAAAAAOGOAAAAA|2451480|2451509|DEPARTMENT|35|22|Months should regulate in a walls; loose persons cannot imagine well after|monthly| +3695|AAAAAAAAPGOAAAAA|2451480|2451509|DEPARTMENT|35|23|New, long competitors used to let methods. Pupils ought to introduce in full at a comme|monthly| +3696|AAAAAAAAAHOAAAAA|2451480|2451509|DEPARTMENT|35|24|Procedures attend separate ears. Male, helpful policies can believe. Strongly |monthly| +3697|AAAAAAAABHOAAAAA|2451480|2451509|DEPARTMENT|35|25|Tools would not conceal meanwhile useful words. Possible politicians might not tell re|monthly| +3698|AAAAAAAACHOAAAAA|2451480|2451509|DEPARTMENT|35|26|Difficult things resolve far late modern advantages. Free ope|monthly| +3699|AAAAAAAADHOAAAAA|2451480|2451509|DEPARTMENT|35|27|Real pictures will make more free cases. Structures could take. Ultimately rig|monthly| +3700|AAAAAAAAEHOAAAAA|2451480|2451509|DEPARTMENT|35|28|Generally old holes work ready, main borders; fresh, bad texts manage to a |monthly| +3701|AAAAAAAAFHOAAAAA|2451480|2451509|DEPARTMENT|35|29|Once again lucky interactions shall go indefinitely most t|monthly| +3702|AAAAAAAAGHOAAAAA|2451480|2451509|DEPARTMENT|35|30|Firmly young votes come partly available, full things. Else main practices wo|monthly| +3703|AAAAAAAAHHOAAAAA|2451480|2451509|DEPARTMENT|35|31|National, american reasons will see also final parties. Very above allowances sel|monthly| +3704|AAAAAAAAIHOAAAAA|2451480|2451509|DEPARTMENT|35|32|More central beliefs put as equal forces; figures try police. Lines|monthly| +3705|AAAAAAAAJHOAAAAA|2451480|2451509|DEPARTMENT|35|33|Individuals will not say girls; tropical, brilliant arms could disrupt ages. Nee|monthly| +3706|AAAAAAAAKHOAAAAA|2451480|2451509|DEPARTMENT|35|34|Nevertheless financial miles draw concerned, clear hours. Geographical, work|monthly| +3707|AAAAAAAALHOAAAAA|2451480|2451509|DEPARTMENT|35|35|Decisions could take forces. Perhaps cold situations |monthly| +3708|AAAAAAAAMHOAAAAA|2451480|2451509|DEPARTMENT|35|36|Other, bare women should not bother forms. Typical, ful|monthly| +3709|AAAAAAAANHOAAAAA|2451480|2451509|DEPARTMENT|35|37|Practical designs get. As rough restaurants fill so-called, estimated wee|monthly| +3710|AAAAAAAAOHOAAAAA|2451480|2451509|DEPARTMENT|35|38|Administrative, big differences prove italian, massive kinds. Ahead ancient argu|monthly| +3711|AAAAAAAAPHOAAAAA|2451480|2451509|DEPARTMENT|35|39|Significantly responsible sentences meet all chinese figures. Also large forces|monthly| +3712|AAAAAAAAAIOAAAAA|2451480|2451509|DEPARTMENT|35|40|Designers must assume canadian, empty arrangements. Little mem|monthly| +3713|AAAAAAAABIOAAAAA|2451480|2451509|DEPARTMENT|35|41|Available tories might not support both shared studies. Languages can want still silver|monthly| +3714|AAAAAAAACIOAAAAA|2451480|2451509|DEPARTMENT|35|42|Full funds talk also revenues. Dull, hidden students may support also only, cen|monthly| +3715|AAAAAAAADIOAAAAA|2451480|2451509|DEPARTMENT|35|43|Too leading proposals will close nearly skilled political problems. Usual areas succeed |monthly| +3716|AAAAAAAAEIOAAAAA|2451480|2451509|DEPARTMENT|35|44|Democratic items must apply toxic, little lawyers. Especially specific agents may |monthly| +3717|AAAAAAAAFIOAAAAA|2451480|2451509|DEPARTMENT|35|45|Only religious influences will carry. Measures may insert very ways. Whole, |monthly| +3718|AAAAAAAAGIOAAAAA|2451480|2451509|DEPARTMENT|35|46|Open clothes extend deals. Cuts buy serious missiles; evidently middle members|monthly| +3719|AAAAAAAAHIOAAAAA|2451480|2451509|DEPARTMENT|35|47|Good, wrong deaths take. Special, chemical boundaries worry forward so-called hands. Sound items ca|monthly| +3720|AAAAAAAAIIOAAAAA|2451480|2451509|DEPARTMENT|35|48|Courses acknowledge as the scientists. Liberal, formal sales ent|monthly| +3721|AAAAAAAAJIOAAAAA|2451480|2451509|DEPARTMENT|35|49|Sounds cannot figure often here wrong settings. Crucial, religious premises start clear, operat|monthly| +3722|AAAAAAAAKIOAAAAA|2451480|2451509|DEPARTMENT|35|50|United heroes shall not obtain often; professional, total procedures could not go researchers|monthly| +3723|AAAAAAAALIOAAAAA|2451480|2451509|DEPARTMENT|35|51|Industrial bodies come in a others. Private facilit|monthly| +3724|AAAAAAAAMIOAAAAA|2451480|2451509|DEPARTMENT|35|52|Fierce, free grounds can seem foreign, superior perso|monthly| +3725|AAAAAAAANIOAAAAA|2451480|2451509|DEPARTMENT|35|53|Young imports could miss here to a women. Essential, valuable peasants|monthly| +3726|AAAAAAAAOIOAAAAA|2451480|2451509|DEPARTMENT|35|54|None the less modest documents shall affect both hence british lines. Actual, total figures|monthly| +3727|AAAAAAAAPIOAAAAA|2451480|2451509|DEPARTMENT|35|55|Important, political clothes will use politically matters. Empty, rus|monthly| +3728|AAAAAAAAAJOAAAAA|2451480|2451509|DEPARTMENT|35|56|Old events assume full numbers. Common consequences ought to mobilize. B|monthly| +3729|AAAAAAAABJOAAAAA|2451480|2451509|DEPARTMENT|35|57|White, local objectives used to help of course good twins. Other, surprising words know still|monthly| +3730|AAAAAAAACJOAAAAA|2451480|2451509|DEPARTMENT|35|58|Clear friends rely from the laws. Absolutely central proposals claim certain matters. |monthly| +3731|AAAAAAAADJOAAAAA|2451480|2451509|DEPARTMENT|35|59|Complex amounts will explain more genuine patients. Mental ages should not wait only i|monthly| +3732|AAAAAAAAEJOAAAAA|2451480|2451509|DEPARTMENT|35|60|Sympathetic, british authorities shall roll of course specific, specific mome|monthly| +3733|AAAAAAAAFJOAAAAA|2451480|2451509|DEPARTMENT|35|61|Expensive activities will accomplish together measures. Sometimes green bodies |monthly| +3734|AAAAAAAAGJOAAAAA|2451480|2451509|DEPARTMENT|35|62|Inc others take political days; services retain soon bad, poor years. Notes see quickly so wo|monthly| +3735|AAAAAAAAHJOAAAAA|2451480|2451509|DEPARTMENT|35|63|Typical individuals trip. Willing, experimental women would budge in the |monthly| +3736|AAAAAAAAIJOAAAAA|2451480|2451509|DEPARTMENT|35|64|Firmly social eyes will engage on a possibilities. Alone arms negotiate only frien|monthly| +3737|AAAAAAAAJJOAAAAA|2451480|2451509|DEPARTMENT|35|65|Preliminary animals shall not find immediately customers. Domestic|monthly| +3738|AAAAAAAAKJOAAAAA|2451480|2451509|DEPARTMENT|35|66|Times cast. Widely final decisions may encourage. Arts will keep. Legs would know perhaps late|monthly| +3739|AAAAAAAALJOAAAAA|2451480|2451509|DEPARTMENT|35|67|Trials used to halt. Main, serious affairs give quietly things. |monthly| +3740|AAAAAAAAMJOAAAAA|2451480|2451509|DEPARTMENT|35|68|Unnecessarily correct costs know throughout a rights; for instance prime ingredients |monthly| +3741|AAAAAAAANJOAAAAA|2451480|2451509|DEPARTMENT|35|69|Asleep, impossible borders will not take problems. Tough fingers co|monthly| +3742|AAAAAAAAOJOAAAAA|2451480|2451509|DEPARTMENT|35|70|Cultural, electronic experiments regulate however with a|monthly| +3743|AAAAAAAAPJOAAAAA|2451480|2451509|DEPARTMENT|35|71|Minutes try quite different, greek dangers. New men publish elsewhere similar |monthly| +3744|AAAAAAAAAKOAAAAA|2451480|2451509|DEPARTMENT|35|72|Parliamentary, public terms sustain electronic, new weekends. Extra hopes thrive in a problems. Eff|monthly| +3745|AAAAAAAABKOAAAAA|2451480|2451509|DEPARTMENT|35|73|About military sides ought to know so defendants. Almost previous subjects offset |monthly| +3746|AAAAAAAACKOAAAAA|2451480|2451509|DEPARTMENT|35|74|Very large parties go twice. Great, horrible eyes may not accept again. However key parts might l|monthly| +3747|AAAAAAAADKOAAAAA|2451480|2451509|DEPARTMENT|35|75|Cattle blame today in order complicated characteris|monthly| +3748|AAAAAAAAEKOAAAAA|2451480|2451509|DEPARTMENT|35|76|Books should benefit only sorry patients. Ways will protect barely for the |monthly| +3749|AAAAAAAAFKOAAAAA|2451480|2451509|DEPARTMENT|35|77|Racial, european relations ought to say quietly basi|monthly| +3750|AAAAAAAAGKOAAAAA|2451480|2451509|DEPARTMENT|35|78|Certainly limited friends check figures. Foreign, careful pe|monthly| +3751|AAAAAAAAHKOAAAAA|2451480|2451509|DEPARTMENT|35|79|Arms enter of course sure members. Relations could not prefer so groups. Str|monthly| +3752|AAAAAAAAIKOAAAAA|2451480|2451509|DEPARTMENT|35|80|Most other taxes look occasionally doubts. Necessary, able views might take. Intere|monthly| +3753|AAAAAAAAJKOAAAAA|2451480|2451509|DEPARTMENT|35|81|Upstairs great affairs send from a words. Personal, pink areas will not use lives; more publi|monthly| +3754|AAAAAAAAKKOAAAAA|2451480|2451509|DEPARTMENT|35|82|Different, sensitive authorities must appeal very questions. Impossible, national co|monthly| +3755|AAAAAAAALKOAAAAA|2451480|2451509|DEPARTMENT|35|83|Premier procedures try especially for the years; old, s|monthly| +3756|AAAAAAAAMKOAAAAA|2451480|2451509|DEPARTMENT|35|84|Different, reliable years ought to pursue anyway u|monthly| +3757|AAAAAAAANKOAAAAA|2451480|2451509|DEPARTMENT|35|85|Seriously social years may explain ago from a values. Endle|monthly| +3758|AAAAAAAAOKOAAAAA|2451480|2451509|DEPARTMENT|35|86|Estates cut virtually likely differences. Deeply sudden facilities congratulate also into a |monthly| +3759|AAAAAAAAPKOAAAAA|2451480|2451509|DEPARTMENT|35|87|Likely, only curtains might like please; clever men talk|monthly| +3760|AAAAAAAAALOAAAAA|2451480|2451509|DEPARTMENT|35|88|New communists like thin, whole solicitors. Best young th|monthly| +3761|AAAAAAAABLOAAAAA|2451480|2451509|DEPARTMENT|35|89|Direct professionals involve both red directions. Relatio|monthly| +3762|AAAAAAAACLOAAAAA|2451480|2451509|DEPARTMENT|35|90|Likely, brief factors may leave normally. Private, small months keep as well also daily teeth. F|monthly| +3763|AAAAAAAADLOAAAAA|2451480|2451509|DEPARTMENT|35|91|Female, confident variables must not inherit given, military changes. Political, dry c|monthly| +3764|AAAAAAAAELOAAAAA|2451480|2451509|DEPARTMENT|35|92|Long, other days would say forces. Long settings put completely outcomes. Mental, wr|monthly| +3765|AAAAAAAAFLOAAAAA|2451480|2451509|DEPARTMENT|35|93|Dead, cautious views bind so employers. Settings cover including a stones. Legs will |monthly| +3766|AAAAAAAAGLOAAAAA|2451480|2451509|DEPARTMENT|35|94|Cells used to labour worse results. Other views used to know by a sisters. Cheerful clubs note|monthly| +3767|AAAAAAAAHLOAAAAA|2451480|2451509|DEPARTMENT|35|95|Goods should not persuade sometimes. Boys guarantee very of course red area|monthly| +3768|AAAAAAAAILOAAAAA|2451480|2451509|DEPARTMENT|35|96|International clergy might battle really journals. Political groups|monthly| +3769|AAAAAAAAJLOAAAAA|2451480|2451509|DEPARTMENT|35|97|Local cards could get in a terms; savings express too without a ears. Ev|monthly| +3770|AAAAAAAAKLOAAAAA|2451480|2451509|DEPARTMENT|35|98|Fingers think over however necessary years. There outdo|monthly| +3771|AAAAAAAALLOAAAAA|2451480|2451509|DEPARTMENT|35|99|Authors join. Indeed possible parents get even in the banks. Shares find widely. Ke|monthly| +3772|AAAAAAAAMLOAAAAA|2451480|2451509|DEPARTMENT|35|100|Certainly unnecessary volumes will know. Other, diffic|monthly| +3773|AAAAAAAANLOAAAAA|2451480|2451509|DEPARTMENT|35|101|Political chemicals produce external voices. Aware, ear|monthly| +3774|AAAAAAAAOLOAAAAA|2451480|2451509|DEPARTMENT|35|102|At once female successes used to treat sources; local complaints could warn thus. Final, alone sh|monthly| +3775|AAAAAAAAPLOAAAAA|2451480|2451509|DEPARTMENT|35|103|More national teeth can lend well major, likely lines. Redu|monthly| +3776|AAAAAAAAAMOAAAAA|2451480|2451509|DEPARTMENT|35|104|Holy, easy stones shall sit now historical, ultimate reaso|monthly| +3777|AAAAAAAABMOAAAAA|2451480|2451509|DEPARTMENT|35|105|Short, western bags express for a buildings. Arms must experience usually at all primary manufa|monthly| +3778|AAAAAAAACMOAAAAA|2451480|2451509|DEPARTMENT|35|106|As obvious magistrates convince. Almost far services serve very concerned doors. Rather |monthly| +3779|AAAAAAAADMOAAAAA|2451480|2451509|DEPARTMENT|35|107|Strengths see so. Better forthcoming themes might not help. Social stars may |monthly| +3780|AAAAAAAAEMOAAAAA|2451480|2451509|DEPARTMENT|35|108|Special, young conditions might benefit just for a parts. Proposals can turn f|monthly| +3781|AAAAAAAAFMOAAAAA|2451510|2451539|DEPARTMENT|36|1|Large directions obtain easily also different prices. Real books should give as t|monthly| +3782|AAAAAAAAGMOAAAAA|2451510|2451539|DEPARTMENT|36|2|Mental numbers may hold in a results. Logical men change in a d|monthly| +3783|AAAAAAAAHMOAAAAA|2451510|2451539|DEPARTMENT|36|3|Enemies ride. Special, other boys must not restrain co|monthly| +3784|AAAAAAAAIMOAAAAA|2451510|2451539|DEPARTMENT|36|4|Tiny, apparent plans explore always off a examples. |monthly| +3785|AAAAAAAAJMOAAAAA|2451510|2451539|DEPARTMENT|36|5|White, old stories shall wipe probably bad clients; relevant r|monthly| +3786|AAAAAAAAKMOAAAAA|2451510|2451539|DEPARTMENT|36|6|More maximum peasants give nearby systems. Literary, small thousands could become m|monthly| +3787|AAAAAAAALMOAAAAA|2451510|2451539|DEPARTMENT|36|7|Available, fine sales will ring arms. Public, alive t|monthly| +3788|AAAAAAAAMMOAAAAA|2451510|2451539|DEPARTMENT|36|8|Political, strong proceedings ring before in a methods. Owners |monthly| +3789|AAAAAAAANMOAAAAA|2451510|2451539|DEPARTMENT|36|9|Other, certain rooms meet longer more additional theories. Original, fun abilities shall|monthly| +3790|AAAAAAAAOMOAAAAA|2451510|2451539|DEPARTMENT|36|10|Ways cover solar months. Single, huge units will affect similar, retail sectors. Movements repa|monthly| +3791|AAAAAAAAPMOAAAAA|2451510|2451539|DEPARTMENT|36|11|European, critical efforts might help players. Then international components record|monthly| +3792|AAAAAAAAANOAAAAA|2451510|2451539|DEPARTMENT|36|12|Central charges hold largely inherent subjects. Children will not join loudly no doubt|monthly| +3793|AAAAAAAABNOAAAAA|2451510|2451539|DEPARTMENT|36|13|Other, extraordinary instruments might not derive casually to a matters.|monthly| +3794|AAAAAAAACNOAAAAA|2451510|2451539|DEPARTMENT|36|14|Other polls mark. More residential countries would not want also surprisin|monthly| +3795|AAAAAAAADNOAAAAA|2451510|2451539|DEPARTMENT|36|15|Powerful events enjoy systems. Wild dogs remain completely on a months; |monthly| +3796|AAAAAAAAENOAAAAA|2451510|2451539|DEPARTMENT|36|16|Independent, other constraints could act at all in a buildings; long cells see ver|monthly| +3797|AAAAAAAAFNOAAAAA|2451510|2451539|DEPARTMENT|36|17|Thick technical calls increase national cards. Complex, spatial events sha|monthly| +3798|AAAAAAAAGNOAAAAA|2451510|2451539|DEPARTMENT|36|18|Just other elections will not underestimate later chemical, electoral enemies. Much|monthly| +3799|AAAAAAAAHNOAAAAA|2451510|2451539|DEPARTMENT|36|19|Else gold knees will not return to the numbers. Differences can go then. Main h|monthly| +3800|AAAAAAAAINOAAAAA|2451510|2451539|DEPARTMENT|36|20|Deep, main items used to say perhaps for the fears. Shops should leave academic days; detect|monthly| +3801|AAAAAAAAJNOAAAAA|2451510|2451539|DEPARTMENT|36|21|Rules know. Common nights would find immediately controls. Human, good problems used to feel; |monthly| +3802|AAAAAAAAKNOAAAAA|2451510|2451539|DEPARTMENT|36|22|Only main claims describe average, religious voters. Hospitals understand local cameras; jobs shal|monthly| +3803|AAAAAAAALNOAAAAA|2451510|2451539|DEPARTMENT|36|23|International centres see new, cruel implications. Gaps ought to form up a minutes. Rational, righ|monthly| +3804|AAAAAAAAMNOAAAAA|2451510|2451539|DEPARTMENT|36|24|General officers could tie simply banks. Domestic, deliberate groups shall g|monthly| +3805|AAAAAAAANNOAAAAA|2451510|2451539|DEPARTMENT|36|25|Rights construct quite little spatial members. Direct, apparent solicitors must |monthly| +3806|AAAAAAAAONOAAAAA|2451510|2451539|DEPARTMENT|36|26|Diplomatic, huge observers might work often necessary americans. Western, potential fe|monthly| +3807|AAAAAAAAPNOAAAAA|2451510|2451539|DEPARTMENT|36|27|Poor teachers tell there surprised, dirty benefits. In order const|monthly| +3808|AAAAAAAAAOOAAAAA|2451510|2451539|DEPARTMENT|36|28|Industrial, difficult skills will not make direct attacks; please domestic sales g|monthly| +3809|AAAAAAAABOOAAAAA|2451510|2451539|DEPARTMENT|36|29|Ordinary, regulatory questions used to run just; efforts used to |monthly| +3810|AAAAAAAACOOAAAAA|2451510|2451539|DEPARTMENT|36|30|Mental, thin courses pretend eventually positive spiri|monthly| +3811|AAAAAAAADOOAAAAA|2451510|2451539|DEPARTMENT|36|31|Foreign girls reach global, head companies. Elsewhere total faces could take far com|monthly| +3812|AAAAAAAAEOOAAAAA|2451510|2451539|DEPARTMENT|36|32|Royal, regional hours would take eventually real teachers. Very|monthly| +3813|AAAAAAAAFOOAAAAA|2451510|2451539|DEPARTMENT|36|33|Political purposes make. Sheets help then determined, good wo|monthly| +3814|AAAAAAAAGOOAAAAA|2451510|2451539|DEPARTMENT|36|34|Economic pieces leave children; almost irish shoes say due branches. Clean, serious |monthly| +3815|AAAAAAAAHOOAAAAA|2451510|2451539|DEPARTMENT|36|35|Factories shall describe boring, regular others. Projects will spend just in a men. Just cool g|monthly| +3816|AAAAAAAAIOOAAAAA|2451510|2451539|DEPARTMENT|36|36|Parts might remember more; increasing provinces sit; full, l|monthly| +3817|AAAAAAAAJOOAAAAA|||DEPARTMENT|||Candidates shall not want. Thanks take over extra, statistical volunte|| +3818|AAAAAAAAKOOAAAAA|2451510|2451539|DEPARTMENT|36|38|Living systems repay even easier likely applications. Versions may want elected, military plants. W|monthly| +3819|AAAAAAAALOOAAAAA|2451510|2451539|DEPARTMENT|36|39|Changing payments come various options. Neither efficient months must talk new messages. Less gr|monthly| +3820|AAAAAAAAMOOAAAAA|2451510|2451539|DEPARTMENT|36|40|Countries must not tell students. Ever english machines lay p|monthly| +3821|AAAAAAAANOOAAAAA|2451510|2451539|DEPARTMENT|36|41|Additional, major ways can fly alone significantly important|monthly| +3822|AAAAAAAAOOOAAAAA|2451510|2451539|DEPARTMENT|36|42|New, practical matters talk soon top copies. Fine representati|monthly| +3823|AAAAAAAAPOOAAAAA|2451510|2451539|DEPARTMENT|36|43|Conditions shall not encourage more used arms. Simp|monthly| +3824|AAAAAAAAAPOAAAAA|2451510|2451539|DEPARTMENT|36|44|Yet average claims survive visits; powerful forms go just. D|monthly| +3825|AAAAAAAABPOAAAAA|2451510|2451539|DEPARTMENT|36|45|Here underlying studies shall divert upon a groups. Politic|monthly| +3826|AAAAAAAACPOAAAAA|2451510|2451539|DEPARTMENT|36|46|Further gentle principles cover very high occupations; then different youngsters |monthly| +3827|AAAAAAAADPOAAAAA|2451510|2451539|DEPARTMENT|36|47|Now little boys add high. Sons offend namely naturally low prob|monthly| +3828|AAAAAAAAEPOAAAAA|2451510|2451539|DEPARTMENT|36|48|Teachers would not depend central, available tables; pressures could lick. Particularly i|monthly| +3829|AAAAAAAAFPOAAAAA|2451510|2451539|DEPARTMENT|36|49|Either detailed makers arise here local holders. Social|monthly| +3830|AAAAAAAAGPOAAAAA|2451510|2451539|DEPARTMENT|36|50|Ideas mean much prepared, managerial materials; th|monthly| +3831|AAAAAAAAHPOAAAAA|2451510|2451539|DEPARTMENT|36|51|Stupid eyebrows must control in a hills. Seats distribu|monthly| +3832|AAAAAAAAIPOAAAAA|2451510|2451539|DEPARTMENT|36|52|Too protective enquiries will pull twice aware barriers. Sexual, long charges ough|monthly| +3833|AAAAAAAAJPOAAAAA|2451510|2451539|DEPARTMENT|36|53|Speedily modern materials worry extremely particular sites. Special runs fall also small walls. |monthly| +3834|AAAAAAAAKPOAAAAA|2451510|2451539|DEPARTMENT|36|54|Applications can operate amidst a profits; fast mean functions ought to go today |monthly| +3835|AAAAAAAALPOAAAAA|2451510|2451539|DEPARTMENT|36|55|Known paintings get in the men. Present candidates should drive. Just british figures will ma|monthly| +3836|AAAAAAAAMPOAAAAA|2451510|2451539|DEPARTMENT|36|56|United, new animals shall not see also possibly subjective pressures. T|monthly| +3837|AAAAAAAANPOAAAAA|2451510|2451539|DEPARTMENT|36|57|Wild, fiscal difficulties must not sound tonight male t|monthly| +3838|AAAAAAAAOPOAAAAA|2451510|2451539|DEPARTMENT|36|58|Simply senior babies must pay especially strongly personal days; co|monthly| +3839|AAAAAAAAPPOAAAAA|2451510|2451539|DEPARTMENT|36|59|Troubles rely chief, fresh results. Open, great concerns |monthly| +3840|AAAAAAAAAAPAAAAA|2451510|2451539|DEPARTMENT|36|60|Rare police break however strong, different elections. |monthly| +3841|AAAAAAAABAPAAAAA|2451510|2451539|DEPARTMENT|36|61|Most inherent arts agree on to the thousands. Relevant, able developmen|monthly| +3842|AAAAAAAACAPAAAAA|2451510|2451539|DEPARTMENT|36|62|Just necessary hopes speed too. Narrow, new teachers ought to think. |monthly| +3843|AAAAAAAADAPAAAAA|2451510|2451539|DEPARTMENT|36|63|Special, new eyes shall wonder just hidden eyes. Key othe|monthly| +3844|AAAAAAAAEAPAAAAA|2451510|2451539|DEPARTMENT|36|64|British messages carry certainly. Other, typical surfaces should ease mid|monthly| +3845|AAAAAAAAFAPAAAAA|2451510|2451539|DEPARTMENT|36|65|Either british functions must die rather new fat areas; yet experienced |monthly| +3846|AAAAAAAAGAPAAAAA|2451510|2451539|DEPARTMENT|36|66|Houses tackle minutes. Sites tell even similar players. Typi|monthly| +3847|AAAAAAAAHAPAAAAA|2451510|2451539|DEPARTMENT|36|67|Dangerous, comprehensive boots say about only, available lengths. Early|monthly| +3848|AAAAAAAAIAPAAAAA|2451510|2451539|DEPARTMENT|36|68|Modern, natural sales develop familiar, poor artists. Dead, digital prisoners afford|monthly| +3849|AAAAAAAAJAPAAAAA|2451510|2451539|DEPARTMENT|36|69|Specially digital differences see above a women. Irish types see closely soviet instruments|monthly| +3850|AAAAAAAAKAPAAAAA|2451510|2451539|DEPARTMENT|36|70|Laws use there social, great men. Good suggestions used to c|monthly| +3851|AAAAAAAALAPAAAAA|2451510|2451539|DEPARTMENT|36|71|Results develop worthy, widespread vessels. European arms m|monthly| +3852|AAAAAAAAMAPAAAAA|2451510|2451539|DEPARTMENT|36|72|Primarily friendly churches will live detailed, distinct quantities. Processes may relax quickly go|monthly| +3853|AAAAAAAANAPAAAAA|2451510|2451539|DEPARTMENT|36|73|Original farmers fight certainly well only pieces. Away great countries curb still well l|monthly| +3854|AAAAAAAAOAPAAAAA|2451510|2451539|DEPARTMENT|36|74|Much controversial officials confirm friends. Claims co-ordinate nuclear materials. So small le|monthly| +3855|AAAAAAAAPAPAAAAA|2451510|2451539|DEPARTMENT|36|75|Little cultural women used to win relatively often available countries. Once previous futures r|monthly| +3856|AAAAAAAAABPAAAAA|2451510|2451539|DEPARTMENT|36|76|There domestic costs prove ago big buildings; past, worthy villages ought to r|monthly| +3857|AAAAAAAABBPAAAAA|2451510|2451539|DEPARTMENT|36|77|Studies decide under a miles. Difficult pieces know harsh ta|monthly| +3858|AAAAAAAACBPAAAAA|2451510|2451539|DEPARTMENT|36|78|Then thin numbers come years; likely humans need so youn|monthly| +3859|AAAAAAAADBPAAAAA|2451510|2451539|DEPARTMENT|36|79|Professional sides may take from a cars. Open necessary bodies pay firstly private, social prices;|monthly| +3860|AAAAAAAAEBPAAAAA|2451510|2451539|DEPARTMENT|36|80|Individuals lose. Stations give with a horses; old purposes must drive by |monthly| +3861|AAAAAAAAFBPAAAAA|2451510|2451539|DEPARTMENT|36|81|Again close hours shall not provide as central windows. Necessary, rational |monthly| +3862|AAAAAAAAGBPAAAAA|2451510|2451539|DEPARTMENT|36|82|Passages say as imports. About regulatory samples need l|monthly| +3863|AAAAAAAAHBPAAAAA|2451510|2451539|DEPARTMENT|36|83|Virtually physical bars subscribe classical patterns. Commentators may ask bl|monthly| +3864|AAAAAAAAIBPAAAAA|2451510|2451539|DEPARTMENT|36|84|Pregnant, other territories clear already chief groups. Minor countries might know. Negat|monthly| +3865|AAAAAAAAJBPAAAAA|2451510|2451539|DEPARTMENT|36|85|Poles give. Silly women tie certainly common members; pupils would leave dif|monthly| +3866|AAAAAAAAKBPAAAAA|2451510|2451539|DEPARTMENT|36|86|Sports may put of course; female, suitable places shall jo|monthly| +3867|AAAAAAAALBPAAAAA|2451510|2451539|DEPARTMENT|36|87|True men carry about a addresses. Elections might worry deep unions; main players en|monthly| +3868|AAAAAAAAMBPAAAAA|2451510|2451539|DEPARTMENT|36|88|Basic accounts could explain particularly ancient, relative ships. Other, |monthly| +3869|AAAAAAAANBPAAAAA|2451510|2451539|DEPARTMENT|36|89|Substantial creatures should cancel simply. Other, professional operat|monthly| +3870|AAAAAAAAOBPAAAAA|2451510|2451539|DEPARTMENT|36|90|Years make fair muscles. Premier, white years might select parents. Around bad findings may re|monthly| +3871|AAAAAAAAPBPAAAAA|2451510|2451539|DEPARTMENT|36|91|Elections may go acceptable products; golden, main others would not te|monthly| +3872|AAAAAAAAACPAAAAA|2451510|2451539|DEPARTMENT|36|92|Common, great persons ought to work noticeably for the tra|monthly| +3873|AAAAAAAABCPAAAAA|2451510|2451539|DEPARTMENT|36|93|Partly used areas might not encourage serious teachers. Enough medical members must meet at the cl|monthly| +3874|AAAAAAAACCPAAAAA|2451510|2451539|DEPARTMENT|36|94|New, electric funds ought to advise altogether. Dry, unaware eff|monthly| +3875|AAAAAAAADCPAAAAA|2451510|2451539|DEPARTMENT|36|95|Modes find also. Parts make on a activities. Fingers ought to know foreign, fresh problem|monthly| +3876|AAAAAAAAECPAAAAA|2451510|2451539|DEPARTMENT|36|96|Cleverly sad minutes mention wives; actual levels update tomorrow; existing, monthly se|monthly| +3877|AAAAAAAAFCPAAAAA|2451510|2451539|DEPARTMENT|36|97|Entirely adult incidents turn. Bizarre, tremendous plants return between the unions; straig|monthly| +3878|AAAAAAAAGCPAAAAA|2451510|2451539|DEPARTMENT|36|98|Somewhere able allowances advise better more royal others. Experimental, yellow years call sole |monthly| +3879|AAAAAAAAHCPAAAAA|2451510|2451539|DEPARTMENT|36|99|Chief eyes must feel concerned, liberal poems. Urban rates may go well remarkab|monthly| +3880|AAAAAAAAICPAAAAA|||||||| +3881|AAAAAAAAJCPAAAAA||2451539||||More special negotiations make yesterday evident activities.|monthly| +3882|AAAAAAAAKCPAAAAA|2451510|2451539|DEPARTMENT|36|102|National services must avoid. Large police might not win |monthly| +3883|AAAAAAAALCPAAAAA|2451510|2451539|DEPARTMENT|36|103|Public, limited casualties guess. Ever agricultural miles might clarify nations. Internation|monthly| +3884|AAAAAAAAMCPAAAAA|2451510|2451539|DEPARTMENT|36|104|Days would lie more soon foreign changes. Skilled, avai|monthly| +3885|AAAAAAAANCPAAAAA|2451510|2451539|DEPARTMENT|36|105|Alone, great cases might see previously complete suggestions. Previous pa|monthly| +3886|AAAAAAAAOCPAAAAA|2451510|2451539|DEPARTMENT|36|106|National systems must require wide photographs. Appropriate, liberal |monthly| +3887|AAAAAAAAPCPAAAAA|2451510|2451539|DEPARTMENT|36|107|Popular losses go main shares. Male arts solve practical words. Interested, new sources ensure huma|monthly| +3888|AAAAAAAAADPAAAAA||2451539||36||Views could not repeat as the germans. Young systems lead leaves. Public|monthly| +3889|AAAAAAAABDPAAAAA|2451545|2451726|DEPARTMENT|37|1|Different things ought to meet increasingly proposed, traditional others. Normal, satisfie|bi-annual| +3890|AAAAAAAACDPAAAAA|2451545|2451726|DEPARTMENT|37|2|Economic committees attract delighted, presidential officials. Children mi|bi-annual| +3891|AAAAAAAADDPAAAAA|2451545|2451726|DEPARTMENT|37|3|Free, international interpretations may order concerned practices. Soon environmental proceedi|bi-annual| +3892|AAAAAAAAEDPAAAAA|2451545|2451726|DEPARTMENT|37|4|As sensible profits would turn. Real, private others might kill very poor women. H|bi-annual| +3893|AAAAAAAAFDPAAAAA|2451545|2451726|DEPARTMENT|37|5|Public, other occasions might not keep then. Roads hide only to no others. Efforts make. Wi|bi-annual| +3894|AAAAAAAAGDPAAAAA|2451545|2451726|DEPARTMENT|37|6|Slow, alternative worlds serve dogs. Angry, right clothes bla|bi-annual| +3895|AAAAAAAAHDPAAAAA|2451545|2451726|DEPARTMENT|37|7|Only bitter problems can close actually great blues; thus possible hours |bi-annual| +3896|AAAAAAAAIDPAAAAA|2451545|2451726|DEPARTMENT|37|8|Whole, maximum habits may select true, domestic reductions. Effects will |bi-annual| +3897|AAAAAAAAJDPAAAAA|2451545|2451726|DEPARTMENT|37|9|Actions import much; powers want because of a events. Finally catholic weeks try. True press|bi-annual| +3898|AAAAAAAAKDPAAAAA|2451545|2451726|DEPARTMENT|37|10|Much empty colleagues might not come. Unique groups go particular, young contexts. |bi-annual| +3899|AAAAAAAALDPAAAAA|2451545|2451726|DEPARTMENT|37|11|Hotels boost rather. New, average pictures will he|bi-annual| +3900|AAAAAAAAMDPAAAAA|2451545|2451726|DEPARTMENT|37|12|Presumably usual marks fire much similar times; systems draw relati|bi-annual| +3901|AAAAAAAANDPAAAAA|2451545|2451726|DEPARTMENT|37|13|Fair secondary moments shall not touch. Players will see brief |bi-annual| +3902|AAAAAAAAODPAAAAA|2451545|2451726|DEPARTMENT|37|14|Somewhere conservative attitudes send other variations; only aware eyes suggest. Alr|bi-annual| +3903|AAAAAAAAPDPAAAAA|2451545|2451726|DEPARTMENT|37|15|About ordinary feet must figure medical classes. Cases cope administrative, useful doctors. Th|bi-annual| +3904|AAAAAAAAAEPAAAAA|||DEPARTMENT|37||Policies give royal arts. Terribly complex things would not imagine there fast h|bi-annual| +3905|AAAAAAAABEPAAAAA|2451545|2451726|DEPARTMENT|37|17|Again welsh years get later eventually economic forces.|bi-annual| +3906|AAAAAAAACEPAAAAA|2451545|2451726|DEPARTMENT|37|18|Feelings wish already limited, delicious claims. Jobs think also years. Suitable cate|bi-annual| +3907|AAAAAAAADEPAAAAA|2451545|2451726|DEPARTMENT|37|19|Available events take around to a citizens. Open standards wo|bi-annual| +3908|AAAAAAAAEEPAAAAA|2451545|2451726|DEPARTMENT|37|20|Other, old readers should not know notably also previous tr|bi-annual| +3909|AAAAAAAAFEPAAAAA|2451545|2451726|DEPARTMENT|37|21|Genuine years remember thus major planes. Parts us|bi-annual| +3910|AAAAAAAAGEPAAAAA|2451545|2451726|DEPARTMENT|37|22|Recent champions must not drive more measures. Difficult, common modules tie ther|bi-annual| +3911|AAAAAAAAHEPAAAAA|2451545|2451726|DEPARTMENT|37|23|Prisoners enter forward men. Easy, human men get allegedly public crops. |bi-annual| +3912|AAAAAAAAIEPAAAAA|2451545|2451726|DEPARTMENT|37|24|Left, dominant prices used to consider finally appointed sides. Possible, english runs go too for a|bi-annual| +3913|AAAAAAAAJEPAAAAA|2451545|2451726|DEPARTMENT|37|25|Well little lines used to see in a children; supporters might go subjects. Far, n|bi-annual| +3914|AAAAAAAAKEPAAAAA|2451545|2451726|DEPARTMENT|37|26|Full institutions check fine eyes. More elected feet might see |bi-annual| +3915|AAAAAAAALEPAAAAA|2451545|2451726|DEPARTMENT|37|27|British, past leads know clear children. Thin, future bags will not include kids. Thirdl|bi-annual| +3916|AAAAAAAAMEPAAAAA|2451545|2451726|DEPARTMENT|37|28|Past, massive nerves start special, random diseases; teeth attempt soon economic properties; |bi-annual| +3917|AAAAAAAANEPAAAAA|2451545|2451726|DEPARTMENT|37|29|Alone considerable studies introduce gradually personal shapes. Wonderful drivers open t|bi-annual| +3918|AAAAAAAAOEPAAAAA|2451545|2451726|DEPARTMENT|37|30|Comprehensive plates cannot get outer holidays. Women may not|bi-annual| +3919|AAAAAAAAPEPAAAAA|2451545|2451726|DEPARTMENT|37|31|Other knees ought to visit sure, young standards. Efforts talk. Military, r|bi-annual| +3920|AAAAAAAAAFPAAAAA|2451545|2451726|DEPARTMENT|37|32|Raw interests should replace over police. Major years form in a sales. Especially great yea|bi-annual| +3921|AAAAAAAABFPAAAAA|2451545|2451726|DEPARTMENT|37|33|Actions will watch most different, old tonnes. Military, good windows |bi-annual| +3922|AAAAAAAACFPAAAAA|2451545|2451726|DEPARTMENT|37|34|Old, available readers exist similarly by the panels. British students|bi-annual| +3923|AAAAAAAADFPAAAAA|2451545|2451726|DEPARTMENT|37|35|Regions must talk quickly. Late political councils might not uncover also manufacturers. Ge|bi-annual| +3924|AAAAAAAAEFPAAAAA|2451545|2451726|DEPARTMENT|37|36|Then public lips ought to sleep frequent events. Only mild weeks tend more eyes. |bi-annual| +3925|AAAAAAAAFFPAAAAA|2451545|2451726|DEPARTMENT|37|37|Electoral sets achieve commonly ways. Poor residents say|bi-annual| +3926|AAAAAAAAGFPAAAAA|2451545|2451726|DEPARTMENT|37|38|Heavily beautiful chips take really chains. Generously financial wor|bi-annual| +3927|AAAAAAAAHFPAAAAA|2451545|2451726|DEPARTMENT|37|39|Even white voters look. Good, previous cats lose. More direct arms keep. Services write regula|bi-annual| +3928|AAAAAAAAIFPAAAAA|2451545|2451726|DEPARTMENT|37|40|Resources support. Often diplomatic questions would not advance good o|bi-annual| +3929|AAAAAAAAJFPAAAAA|2451545|2451726|DEPARTMENT|37|41|British facilities require all competitive executives. Even essential modes achieve alway|bi-annual| +3930|AAAAAAAAKFPAAAAA|2451545|2451726|DEPARTMENT|37|42|As top artists might fill straight in a travellers; values could protect to a activ|bi-annual| +3931|AAAAAAAALFPAAAAA|2451545|2451726|DEPARTMENT|37|43|Words might not find effective rules. Years want developing, fond books. Only available|bi-annual| +3932|AAAAAAAAMFPAAAAA|2451545|2451726|DEPARTMENT|37|44|There international women can change. Central, sufficient years help amounts; rights may get how|bi-annual| +3933|AAAAAAAANFPAAAAA|2451545|2451726|DEPARTMENT|37|45|Attempts take really with a members. Spiritual forms highlight offices. Important lead|bi-annual| +3934|AAAAAAAAOFPAAAAA|2451545|2451726|DEPARTMENT|37|46|Rather national issues associate findings. Soon previous mi|bi-annual| +3935|AAAAAAAAPFPAAAAA|2451545|2451726|DEPARTMENT|37|47|Offences could produce good, true photographs. Operational friends ow|bi-annual| +3936|AAAAAAAAAGPAAAAA|2451545|2451726|DEPARTMENT|37|48|Patients should not fly from a candidates. Open, interested horses mig|bi-annual| +3937|AAAAAAAABGPAAAAA|2451545|2451726|DEPARTMENT|37|49|Conclusions learn absolutely necessary, critical ends. Above fine lips should not subdue ot|bi-annual| +3938|AAAAAAAACGPAAAAA|2451545|2451726|DEPARTMENT|37|50|Jeans prevent very. Small marks will get and so on scottish metres;|bi-annual| +3939|AAAAAAAADGPAAAAA|2451545|2451726|DEPARTMENT|37|51|Specialist obligations pay directly in a groups. Surprising, selective conditions make much i|bi-annual| +3940|AAAAAAAAEGPAAAAA|2451545|2451726|DEPARTMENT|37|52|Wooden courses expect. Unable workers see visitors. Risks shall not|bi-annual| +3941|AAAAAAAAFGPAAAAA|2451545|2451726|DEPARTMENT|37|53|Preferences would not fill. Rights become only; particular studies ought to say no longer for a pa|bi-annual| +3942|AAAAAAAAGGPAAAAA|2451545|2451726|DEPARTMENT|37|54|New, special organisations restore. Attractive, fi|bi-annual| +3943|AAAAAAAAHGPAAAAA|2451545|2451726|DEPARTMENT|37|55|Late western signs go perhaps with a patterns. As yet fair holes take anxiously once aga|bi-annual| +3944|AAAAAAAAIGPAAAAA|2451545|2451726|DEPARTMENT|37|56|Emotions will not identify annual, orthodox ships. Dull, final towns land much formal arms; thus li|bi-annual| +3945|AAAAAAAAJGPAAAAA||2451726|||||bi-annual| +3946|AAAAAAAAKGPAAAAA|2451545|2451726|DEPARTMENT|37|58|Busy, angry tools stop ordinary, current questions; great, english prison|bi-annual| +3947|AAAAAAAALGPAAAAA|2451545|2451726|DEPARTMENT|37|59|Rules feel real items. Things look aware, separate men. Co|bi-annual| +3948|AAAAAAAAMGPAAAAA|2451545|2451726|DEPARTMENT|37|60|Problems find both british stands. More cheerful owner|bi-annual| +3949|AAAAAAAANGPAAAAA|2451545|2451726|DEPARTMENT|37|61|At all necessary emotions might prevent more men. New, far activities must answer gifts. Potatoes g|bi-annual| +3950|AAAAAAAAOGPAAAAA|2451545|2451726|DEPARTMENT|37|62|Geographical, numerous passengers act strong candidates. Securities shall agree henc|bi-annual| +3951|AAAAAAAAPGPAAAAA|2451545|2451726|DEPARTMENT|37|63|Well arab proposals should close issues. Necessary, american guests justify more natural forces; |bi-annual| +3952|AAAAAAAAAHPAAAAA|2451545|2451726|DEPARTMENT|37|64|Anxious members come thus extraordinarily equal areas. Short, clear |bi-annual| +3953|AAAAAAAABHPAAAAA|2451545|2451726|DEPARTMENT|37|65|Guilty, specific years must identify here similar st|bi-annual| +3954|AAAAAAAACHPAAAAA|2451545|2451726|DEPARTMENT|37|66|Academic, appropriate applications can build terribly capable s|bi-annual| +3955|AAAAAAAADHPAAAAA|2451545|2451726|DEPARTMENT|37|67|Canadian explanations would like right to a standards. Foreign, economic cars must not |bi-annual| +3956|AAAAAAAAEHPAAAAA|2451545|2451726|DEPARTMENT|37|68|Patients let men. Then nuclear tables shall help perhaps strong author|bi-annual| +3957|AAAAAAAAFHPAAAAA|2451545|2451726|DEPARTMENT|37|69|Capable, interesting companies should measure agents. Often e|bi-annual| +3958|AAAAAAAAGHPAAAAA|2451545|2451726|DEPARTMENT|37|70|External, single defences may understand actually. Only dry coun|bi-annual| +3959|AAAAAAAAHHPAAAAA|2451545|2451726|DEPARTMENT|37|71|Environmental, public figures used to create subjects; politicians|bi-annual| +3960|AAAAAAAAIHPAAAAA|2451545|2451726|DEPARTMENT|37|72|Churches work worth the animals. Social principles used to talk heavy, secondary parents. Inter|bi-annual| +3961|AAAAAAAAJHPAAAAA|2451545|2451726|DEPARTMENT|37|73|Away ideal conventions could ask more by a women. Ugl|bi-annual| +3962|AAAAAAAAKHPAAAAA|2451545|2451726|DEPARTMENT|37|74|Clinical, old rivers conserve even. Either shared populations ought t|bi-annual| +3963|AAAAAAAALHPAAAAA|2451545|2451726|DEPARTMENT|37|75|Typical, agricultural attacks ought to achieve forei|bi-annual| +3964|AAAAAAAAMHPAAAAA|2451545|2451726|DEPARTMENT|37|76|Even difficult newspapers could find smoothly jobs. Happy, difficult employers hit too to the ses|bi-annual| +3965|AAAAAAAANHPAAAAA|2451545|2451726|DEPARTMENT|37|77|Big values must take immediately popular farmers. Rivers go other, happy offers. Financial, n|bi-annual| +3966|AAAAAAAAOHPAAAAA|2451545|2451726|DEPARTMENT|37|78|Good leaders might not find. Capital policies must not survive so clear eggs. Steps sho|bi-annual| +3967|AAAAAAAAPHPAAAAA|2451545|2451726|DEPARTMENT|37|79|Old charges outline tender, true seconds. New, publ|bi-annual| +3968|AAAAAAAAAIPAAAAA|2451545|2451726|DEPARTMENT|37|80|National sisters might not take. Sharp, living lengths must meet some|bi-annual| +3969|AAAAAAAABIPAAAAA|2451545|2451726|DEPARTMENT|37|81|Public plates should make elsewhere political tests. The|bi-annual| +3970|AAAAAAAACIPAAAAA|2451545|2451726|DEPARTMENT|37|82|Pregnant, important levels presume etc poor, early days. Reasonable, reduced posts|bi-annual| +3971|AAAAAAAADIPAAAAA|2451545|2451726|DEPARTMENT|37|83|Close terms matter far obvious, integrated books. Police regulate functions. Etc careful papers f|bi-annual| +3972|AAAAAAAAEIPAAAAA|2451545|2451726|DEPARTMENT|37|84|Ever english trials shall not eat games; other, old phase|bi-annual| +3973|AAAAAAAAFIPAAAAA|2451545|2451726|DEPARTMENT|37|85|Cards ought to hit at a missiles. Names ought to think special, married lin|bi-annual| +3974|AAAAAAAAGIPAAAAA|2451545|2451726|DEPARTMENT|37|86|Safe, sensitive elections shall select men; hardly tra|bi-annual| +3975|AAAAAAAAHIPAAAAA|2451545|2451726|DEPARTMENT|37|87|Then real hospitals might clear. Simply private months dispose incidents. Dual characters |bi-annual| +3976|AAAAAAAAIIPAAAAA|2451545|2451726|DEPARTMENT|37|88|Pieces pause only big, parliamentary eyes. Obvious, special pictures may laugh often soft |bi-annual| +3977|AAAAAAAAJIPAAAAA|2451545|2451726|DEPARTMENT|37|89|Current, whole months might say so as at least trying interests; never spe|bi-annual| +3978|AAAAAAAAKIPAAAAA|2451545|2451726|DEPARTMENT|37|90|Other, long numbers win worthwhile couples. Systems will reach forth. Major, big anima|bi-annual| +3979|AAAAAAAALIPAAAAA|2451545|2451726|DEPARTMENT|37|91|Questions take police; measures want otherwise young books; meanwhile early|bi-annual| +3980|AAAAAAAAMIPAAAAA|2451545|2451726|DEPARTMENT|37|92|A little ordinary cases shall put recent, conventional opportunities. Eyes mu|bi-annual| +3981|AAAAAAAANIPAAAAA|2451545|2451726|DEPARTMENT|37|93|Forms change alternatively to a colleagues. Definite, inc others tell; m|bi-annual| +3982|AAAAAAAAOIPAAAAA|2451545|2451726|DEPARTMENT|37|94|Employees know too concentrations. Part-time times will hurt for a pra|bi-annual| +3983|AAAAAAAAPIPAAAAA|2451545|2451726|DEPARTMENT|37|95|Over low sources shall open somewhat really excellent thought|bi-annual| +3984|AAAAAAAAAJPAAAAA|2451545|2451726|DEPARTMENT|37|96|Normally coloured nerves study over similar critics|bi-annual| +3985|AAAAAAAABJPAAAAA|2451545|2451726|DEPARTMENT|37|97|Chemical, young years cannot operate namely. Else new services ought to drop possible fee|bi-annual| +3986|AAAAAAAACJPAAAAA|2451545|2451726|DEPARTMENT|37|98|New weapons use facilities. Now dangerous abilities could not reduce; annual cities might make both|bi-annual| +3987|AAAAAAAADJPAAAAA|2451545|2451726|DEPARTMENT|37|99|Colours might not study too tracks. Helpful, human stairs take royal, little|bi-annual| +3988|AAAAAAAAEJPAAAAA|2451545|2451726|DEPARTMENT|37|100|Years must turn here by the persons. Old mice shall pull on a years; wooden miles would spare ad|bi-annual| +3989|AAAAAAAAFJPAAAAA|2451545|2451726|DEPARTMENT|37|101|Primary, strange students would get as a letters. Friends lead soon human steps; single examples |bi-annual| +3990|AAAAAAAAGJPAAAAA|2451545|2451726|DEPARTMENT|37|102|Subjects raise together international, good attitudes. Too elaborate t|bi-annual| +3991|AAAAAAAAHJPAAAAA|2451545|2451726|DEPARTMENT|37|103|Sad terms ought to turn english, possible miles. Hard, different specialists may meet outside old d|bi-annual| +3992|AAAAAAAAIJPAAAAA|2451545|2451726|DEPARTMENT|37|104|For once old months pass however too colourful numbers. Coming, brown factors give once aga|bi-annual| +3993|AAAAAAAAJJPAAAAA|2451545|2451726|DEPARTMENT|37|105|New projects may not find in a speakers. Small, established vehicles |bi-annual| +3994|AAAAAAAAKJPAAAAA|2451545|2451726|DEPARTMENT|37|106|Legal cars make else faint, other interests. Completely monthly countries may no|bi-annual| +3995|AAAAAAAALJPAAAAA|2451545|2451726|DEPARTMENT|37|107|Certainly main panels may not go so at a comments. Opening police launch ashore right internation|bi-annual| +3996|AAAAAAAAMJPAAAAA|2451545|2451726|DEPARTMENT|37|108|Educational activities react more parts. Identical, only powers str|bi-annual| +3997|AAAAAAAANJPAAAAA|2451727|2451908|DEPARTMENT|38|1|Important, european services can raise never notably short costs. |bi-annual| +3998|AAAAAAAAOJPAAAAA|2451727|2451908|DEPARTMENT|38|2|Pounds plough real, old police; miserably political|bi-annual| +3999|AAAAAAAAPJPAAAAA|2451727|2451908|DEPARTMENT|38|3|Now good plants wait strangely essential contents. More central cases take involved|bi-annual| +4000|AAAAAAAAAKPAAAAA|2451727|2451908|DEPARTMENT|38|4|English, free police cannot make to the defences; banks must ensure increased risks. Q|bi-annual| +4001|AAAAAAAABKPAAAAA|2451727|2451908|DEPARTMENT|38|5|Both sufficient activities check there principles. |bi-annual| +4002|AAAAAAAACKPAAAAA|2451727|2451908|DEPARTMENT|38|6|Cells should make very maybe efficient women; just young results underpin most possible exam|bi-annual| +4003|AAAAAAAADKPAAAAA|2451727|2451908|DEPARTMENT|38|7|New, wet perceptions might not grieve only. Basic measures will not |bi-annual| +4004|AAAAAAAAEKPAAAAA|2451727|2451908|DEPARTMENT|38|8|Supreme models relax kind of islands; concerned, single men work m|bi-annual| +4005|AAAAAAAAFKPAAAAA|2451727|2451908|DEPARTMENT|38|9|Indeed modern mammals might call entirely somehow i|bi-annual| +4006|AAAAAAAAGKPAAAAA|2451727|2451908|DEPARTMENT|38|10|Now good steps write completely on a factors. Here popular falls advise legal sales. Alone sorr|bi-annual| +4007|AAAAAAAAHKPAAAAA|2451727|2451908|DEPARTMENT|38|11|Professional, initial cases resolve rational, current persons. Standard, large authorities may n|bi-annual| +4008|AAAAAAAAIKPAAAAA|2451727|2451908||38|12||bi-annual| +4009|AAAAAAAAJKPAAAAA|2451727|2451908|DEPARTMENT|38|13|Ago sound procedures deepen only in the men. Besides primary schools think|bi-annual| +4010|AAAAAAAAKKPAAAAA|2451727|2451908|DEPARTMENT|38|14|Magnetic wives cannot collapse now terms. Mad friends want at onc|bi-annual| +4011|AAAAAAAALKPAAAAA|2451727|2451908|DEPARTMENT|38|15|Straight proud ministers would retain in particular with a facts. Magnetic memb|bi-annual| +4012|AAAAAAAAMKPAAAAA|2451727|2451908|DEPARTMENT|38|16|Merely ugly elements used to tell presumably legal wrong |bi-annual| +4013|AAAAAAAANKPAAAAA|2451727|2451908|DEPARTMENT|38|17|Here german rates discuss in general old, true services. No doubt concerned arrangeme|bi-annual| +4014|AAAAAAAAOKPAAAAA|2451727|2451908|DEPARTMENT|38|18|Primary, key libraries sail eventually mainly real savings. Big areas sho|bi-annual| +4015|AAAAAAAAPKPAAAAA|2451727|2451908|DEPARTMENT|38|19|Lights might hold then detailed arms; thoughts cannot remain most to a goods. Ori|bi-annual| +4016|AAAAAAAAALPAAAAA|2451727|2451908|DEPARTMENT|38|20|Books ensure ago proper years. Then wrong issues might not indicate v|bi-annual| +4017|AAAAAAAABLPAAAAA|2451727|2451908|DEPARTMENT|38|21|Exceptional, executive attitudes throw parts. Ancient responsibilities need; clini|bi-annual| +4018|AAAAAAAACLPAAAAA|2451727|2451908|DEPARTMENT|38|22|Priorities would not surprise possible officers. Historically similar friends fe|bi-annual| +4019|AAAAAAAADLPAAAAA|2451727|2451908|DEPARTMENT|38|23|Cuts may not come open heavy calls. Months must rise on the units. Budgets used to remember uncer|bi-annual| +4020|AAAAAAAAELPAAAAA|2451727|2451908|DEPARTMENT|38|24|Instruments ought to grow again with a movements. Net, full years answer able |bi-annual| +4021|AAAAAAAAFLPAAAAA|2451727|2451908|DEPARTMENT|38|25|Public bacteria will present then british changes; easily long friends learn softly residen|bi-annual| +4022|AAAAAAAAGLPAAAAA|2451727|2451908|DEPARTMENT|38|26|Previous years consult all figures. So good miles call too. Versions know really comfortabl|bi-annual| +4023|AAAAAAAAHLPAAAAA|2451727|2451908|DEPARTMENT|38|27|Officers will consider anyway very democratic senses. A|bi-annual| +4024|AAAAAAAAILPAAAAA|2451727|2451908|DEPARTMENT|38|28|British, german problems reappear else sometimes social services. Separ|bi-annual| +4025|AAAAAAAAJLPAAAAA|2451727|2451908|DEPARTMENT|38|29|Tears take between the sections; simple mechanisms record then. Re|bi-annual| +4026|AAAAAAAAKLPAAAAA|2451727|2451908|DEPARTMENT|38|30|Fans might want. Downstairs short structures could |bi-annual| +4027|AAAAAAAALLPAAAAA|2451727|2451908|DEPARTMENT|38|31|Ships would expect supreme, professional candidates. Conditions pull qui|bi-annual| +4028|AAAAAAAAMLPAAAAA|2451727|2451908|DEPARTMENT|38|32|Arms use at a doctors; firm, long years include now complex limits. Social contri|bi-annual| +4029|AAAAAAAANLPAAAAA|2451727|2451908|DEPARTMENT|38|33|Local circumstances examine difficult methods. Open, active rooms pre|bi-annual| +4030|AAAAAAAAOLPAAAAA|2451727|2451908|DEPARTMENT|38|34|Particularly blue sets sell also. So different tools cover now only nat|bi-annual| +4031|AAAAAAAAPLPAAAAA|2451727|2451908|DEPARTMENT|38|35|Perhaps unusual reserves should generate. Below patient techniques like kind of |bi-annual| +4032|AAAAAAAAAMPAAAAA|2451727|2451908|DEPARTMENT|38|36|Likely hands squeeze temporary, different days. Tests coul|bi-annual| +4033|AAAAAAAABMPAAAAA|2451727|2451908|DEPARTMENT|38|37|Usual, difficult cases will not say too. So functional losses repe|bi-annual| +4034|AAAAAAAACMPAAAAA|2451727|2451908|DEPARTMENT|38|38|Existing brothers determine cold children; proteins obtain na|bi-annual| +4035|AAAAAAAADMPAAAAA|2451727|2451908|DEPARTMENT|38|39|Model, simple times can help from now on numbers. Forces ap|bi-annual| +4036|AAAAAAAAEMPAAAAA|2451727|2451908|DEPARTMENT|38|40|Appropriate businesses should exist. Easy authors should exist just current, independent |bi-annual| +4037|AAAAAAAAFMPAAAAA|2451727|2451908|DEPARTMENT|38|41|Else legal drinks get. Undoubtedly practical years wi|bi-annual| +4038|AAAAAAAAGMPAAAAA|2451727|2451908|DEPARTMENT|38|42|Often foreign jobs borrow now main authorities. Potential, racial months should |bi-annual| +4039|AAAAAAAAHMPAAAAA|2451727|2451908|DEPARTMENT|38|43|Victorian applications would not achieve over here. Organisational|bi-annual| +4040|AAAAAAAAIMPAAAAA|2451727|2451908|DEPARTMENT|38|44|Commitments replace all. Machines would not support implications|bi-annual| +4041|AAAAAAAAJMPAAAAA|2451727|2451908|DEPARTMENT|38|45|Aware areas improve however in the causes. Outside rights amount just top cells. Related, wes|bi-annual| +4042|AAAAAAAAKMPAAAAA|2451727|2451908|DEPARTMENT|38|46|Ideas shall obtain. Dry, legal readers throw more local, secondary man|bi-annual| +4043|AAAAAAAALMPAAAAA|2451727|2451908|DEPARTMENT|38|47|Relatively material years could include. Developers shall|bi-annual| +4044|AAAAAAAAMMPAAAAA|2451727|2451908|DEPARTMENT|38|48|British, lacking ideas can return never other months. Southern, |bi-annual| +4045|AAAAAAAANMPAAAAA|2451727|2451908|DEPARTMENT|38|49|Doctors might know. Native problems appear new, good experts. Crude pol|bi-annual| +4046|AAAAAAAAOMPAAAAA|2451727|2451908|DEPARTMENT|38|50|Women keep subjects. Efforts might supply nearly architects. Borders may provide arm|bi-annual| +4047|AAAAAAAAPMPAAAAA|2451727|2451908|DEPARTMENT|38|51|Constantly theoretical abilities grant at least in a standards. Over prime stu|bi-annual| +4048|AAAAAAAAANPAAAAA|2451727|2451908|DEPARTMENT|38|52|Great, local pools stop initially requests. Hot elements|bi-annual| +4049|AAAAAAAABNPAAAAA|2451727|2451908|DEPARTMENT|38|53|Strong versions shall play especially records. New |bi-annual| +4050|AAAAAAAACNPAAAAA|2451727|2451908|DEPARTMENT|38|54|Other doubts join then in an buildings. Quiet times render simply like the sol|bi-annual| +4051|AAAAAAAADNPAAAAA|2451727|2451908|DEPARTMENT|38|55|Earnings benefit usually easy questions. Questions see far |bi-annual| +4052|AAAAAAAAENPAAAAA|2451727|2451908|DEPARTMENT|38|56|Ancient, total goals shall know just. Reliable years c|bi-annual| +4053|AAAAAAAAFNPAAAAA|2451727|2451908|DEPARTMENT|38|57|Academic beds used to remain local, public parents; dimensions c|bi-annual| +4054|AAAAAAAAGNPAAAAA|2451727|2451908|DEPARTMENT|38|58|Exotic affairs provide otherwise apparent flames. Other chi|bi-annual| +4055|AAAAAAAAHNPAAAAA|2451727|2451908|DEPARTMENT|38|59|Private, noble populations help regardless private birds. New, technical thing|bi-annual| +4056|AAAAAAAAINPAAAAA|2451727|2451908|DEPARTMENT|38|60|Other, black owners come moments. Complex, long calls will say months. Bizarre a|bi-annual| +4057|AAAAAAAAJNPAAAAA|2451727|2451908|DEPARTMENT|38|61|Nations track once. Foreign eyes will not express. Grateful methods can see s|bi-annual| +4058|AAAAAAAAKNPAAAAA|2451727|2451908|DEPARTMENT|38|62|Parties decay for the time being also old partners. Eff|bi-annual| +4059|AAAAAAAALNPAAAAA|2451727|2451908|DEPARTMENT|38|63|Existing, coloured opportunities used to concentrate yet academic things. Centrally other nati|bi-annual| +4060|AAAAAAAAMNPAAAAA|2451727|2451908|DEPARTMENT|38|64|Well european women used to prepare just on the students; |bi-annual| +4061|AAAAAAAANNPAAAAA|2451727|2451908|DEPARTMENT|38|65|True things can advocate under to a strategies; yet suitable things push top laws; there s|bi-annual| +4062|AAAAAAAAONPAAAAA|2451727|2451908|DEPARTMENT|38|66|Below criminal shows include highly palestinian rules. Int|bi-annual| +4063|AAAAAAAAPNPAAAAA|2451727|2451908|DEPARTMENT|38|67|Rather parliamentary relations shall not know english, due advantages. About independent librar|bi-annual| +4064|AAAAAAAAAOPAAAAA|2451727|2451908|DEPARTMENT|38|68|Eventually technological papers may not model bad final awful things; electron|bi-annual| +4065|AAAAAAAABOPAAAAA|2451727|2451908|DEPARTMENT|38|69|Sad years take again of course real specimens. Conditions acqu|bi-annual| +4066|AAAAAAAACOPAAAAA||||38||Very shallow parents could feel things. Still availabl|| +4067|AAAAAAAADOPAAAAA|2451727|2451908|DEPARTMENT|38|71|Once angry studies feel there real, general programs. Human measures avoid pages. Human ac|bi-annual| +4068|AAAAAAAAEOPAAAAA|2451727|2451908|DEPARTMENT|38|72|Customers can prove nice, effective pieces; only areas s|bi-annual| +4069|AAAAAAAAFOPAAAAA|2451727|2451908|DEPARTMENT|38|73|Forces ought to find; others think more. International ends will get as|bi-annual| +4070|AAAAAAAAGOPAAAAA|2451727|2451908|DEPARTMENT|38|74|Home bitter possibilities need rough, familiar ministers. P|bi-annual| +4071|AAAAAAAAHOPAAAAA|2451727|2451908|DEPARTMENT|38|75|Old windows show now low problems. For instance pregn|bi-annual| +4072|AAAAAAAAIOPAAAAA|2451727|2451908|DEPARTMENT|38|76|Correct, senior groups may not try. Factors attract. Material, arb|bi-annual| +4073|AAAAAAAAJOPAAAAA|2451727|2451908|DEPARTMENT|38|77|Australian, wrong lovers might go even eventually cultural bars. Social issues cannot p|bi-annual| +4074|AAAAAAAAKOPAAAAA|2451727|2451908|DEPARTMENT|38|78|Long, important forests check red holidays. Maybe capable pictures may plan over important, poss|bi-annual| +4075|AAAAAAAALOPAAAAA|2451727|2451908|DEPARTMENT|38|79|Services clean sufficiently. American, english miles cool appropriate |bi-annual| +4076|AAAAAAAAMOPAAAAA|2451727|2451908|DEPARTMENT|38|80|Permanent, foreign nurses think else women. British divisions should go claims|bi-annual| +4077|AAAAAAAANOPAAAAA|2451727|2451908|DEPARTMENT|38|81|Early feet lie great sales. Early long years update; dark, national bot|bi-annual| +4078|AAAAAAAAOOPAAAAA|2451727|2451908|DEPARTMENT|38|82|Often good eyes comply eventually. Main minutes would boast reall|bi-annual| +4079|AAAAAAAAPOPAAAAA|2451727|2451908|DEPARTMENT|38|83|Issues repeat right tonnes. Tiny, other forms must not undertake from a practices. Suprem|bi-annual| +4080|AAAAAAAAAPPAAAAA|2451727|2451908|DEPARTMENT|38|84|Well-known, different fields shall search thus effectively |bi-annual| +4081|AAAAAAAABPPAAAAA|2451727|2451908|DEPARTMENT|38|85|Future holidays may speak by a arguments. Right, video-taped years must observ|bi-annual| +4082|AAAAAAAACPPAAAAA|2451727|2451908|DEPARTMENT|38|86|Authorities cannot smile also internal sons. Quick, new eggs might play international managers. No|bi-annual| +4083|AAAAAAAADPPAAAAA|2451727|2451908|DEPARTMENT|38|87|Levels used to get then healthy weeks. Writers shall|bi-annual| +4084|AAAAAAAAEPPAAAAA|2451727|2451908|DEPARTMENT|38|88|Excessive limitations unite again perfect industries. Authorities li|bi-annual| +4085|AAAAAAAAFPPAAAAA|2451727|2451908|DEPARTMENT|38|89|Outdoor groups cross effectively officials. Low young years cannot make |bi-annual| +4086|AAAAAAAAGPPAAAAA||2451908|DEPARTMENT|38|90||| +4087|AAAAAAAAHPPAAAAA|2451727|2451908|DEPARTMENT|38|91|Disciplinary, foreign situations will embarrass never advanced ne|bi-annual| +4088|AAAAAAAAIPPAAAAA|2451727|2451908|DEPARTMENT|38|92|Once liable sides sign never; just, new matters mu|bi-annual| +4089|AAAAAAAAJPPAAAAA|2451727|2451908|DEPARTMENT|38|93|However social conditions recover quickly in a males|bi-annual| +4090|AAAAAAAAKPPAAAAA|2451727|2451908|DEPARTMENT|38|94|Busy feet apologise in addition. Nations ought to work g|bi-annual| +4091|AAAAAAAALPPAAAAA|2451727|2451908|DEPARTMENT|38|95|Black police like so primary influences; previous leaders may|bi-annual| +4092|AAAAAAAAMPPAAAAA|2451727|2451908|DEPARTMENT|38|96|Formal coins should allow paths. Original taxes cannot compare. Responsi|bi-annual| +4093|AAAAAAAANPPAAAAA|2451727|2451908|DEPARTMENT|38|97|Estimates continue different services; so-called shelves might not go then|bi-annual| +4094|AAAAAAAAOPPAAAAA|2451727|2451908|DEPARTMENT|38|98|National policemen ask so polite interests. Main, rare savings continue now|bi-annual| +4095|AAAAAAAAPPPAAAAA|2451727|2451908|DEPARTMENT|38|99|Other materials could take nationally. Sources cannot seem presumably just|bi-annual| +4096|AAAAAAAAAAABAAAA|2451727|2451908|DEPARTMENT|38|100|Other languages will not sell eventually to a buildings. Impossible miles ought to object.|bi-annual| +4097|AAAAAAAABAABAAAA|2451727|2451908|DEPARTMENT|38|101|Small, english women let boldly internal victims. Possible, official bo|bi-annual| +4098|AAAAAAAACAABAAAA|2451727|2451908|DEPARTMENT|38|102|Different sales will stand english, interested miners. Local girls go t|bi-annual| +4099|AAAAAAAADAABAAAA|2451727|2451908|DEPARTMENT|38|103|Rough families can happen legs. Freely warm hundreds meet i|bi-annual| +4100|AAAAAAAAEAABAAAA|2451727|2451908|DEPARTMENT|38|104|Minutes announce writings. Sexual items recall japanese, only |bi-annual| +4101|AAAAAAAAFAABAAAA|2451727|2451908|DEPARTMENT|38|105|Deeply social activities should work chemicals. Intensely arab men come in particular fo|bi-annual| +4102|AAAAAAAAGAABAAAA|2451727|2451908|DEPARTMENT|38|106|Residents combat very suspicious incentives. Less clever abilities exercise separately wages. C|bi-annual| +4103|AAAAAAAAHAABAAAA|2451727|2451908|DEPARTMENT|38|107|Clergy pass effective, japanese areas. Mutual plans worry later even great politicians. Exce|bi-annual| +4104|AAAAAAAAIAABAAAA|2451727|2451908|DEPARTMENT|38|108|Findings must go later existing, hard ways. Dry, international attac|bi-annual| +4105|AAAAAAAAJAABAAAA|2451545|2451635|DEPARTMENT|39|1|Laws go only quite modern workers. Very public nerv|quarterly| +4106|AAAAAAAAKAABAAAA|2451545|2451635|DEPARTMENT|39|2|Hands admit directly interests. Local, steady trusts involve tools. Experi|quarterly| +4107|AAAAAAAALAABAAAA|2451545|2451635|DEPARTMENT|39|3|Visitors end costs; boys go lovely, clever towns; beautiful, great |quarterly| +4108|AAAAAAAAMAABAAAA|2451545|2451635|DEPARTMENT|39|4|Small years might give circumstances; losses can arrange rational pictures. Again academic w|quarterly| +4109|AAAAAAAANAABAAAA|2451545|2451635|DEPARTMENT|39|5|National, fit hours like slowly. Firmly similar neighbours like long patients. |quarterly| +4110|AAAAAAAAOAABAAAA||2451635|DEPARTMENT||6|Men say really. Late young examples contact within the signs. Corresponding models w|quarterly| +4111|AAAAAAAAPAABAAAA|2451545|2451635|DEPARTMENT|39|7|Powerful, patient weeks call ever existing decisions. Applications would see there fin|quarterly| +4112|AAAAAAAAABABAAAA|2451545|2451635|DEPARTMENT|39|8|Areas break old items; areas should not carry certainly authorities. Too negative plan|quarterly| +4113|AAAAAAAABBABAAAA|2451545|2451635|DEPARTMENT|39|9|Foreign payments can record flights. Students may provide more in a women. Future, little |quarterly| +4114|AAAAAAAACBABAAAA|2451545|2451635|DEPARTMENT|39|10|Other groups can get on a structures. Adequate, surprising stages may not see good teachers. |quarterly| +4115|AAAAAAAADBABAAAA|2451545|2451635|DEPARTMENT|39|11|Big, able hands shall come between the plants. Days follow national, super documents. Facili|quarterly| +4116|AAAAAAAAEBABAAAA|2451545|2451635|DEPARTMENT|39|12|Even likely worlds must attract increasingly difficult gentle boys. Formal, econ|quarterly| +4117|AAAAAAAAFBABAAAA|2451545|2451635|DEPARTMENT|39|13|Bold, net men might gain. Serious, retail others tell past after a women. Foreign, british wing|quarterly| +4118|AAAAAAAAGBABAAAA|2451545|2451635|DEPARTMENT|39|14|Here unable ideas swing sure resources; so likely mistakes exercise. Observations must not appear |quarterly| +4119|AAAAAAAAHBABAAAA|2451545|2451635|DEPARTMENT|39|15|Happy, deliberate employees use personal areas. Small, surprised germ|quarterly| +4120|AAAAAAAAIBABAAAA|2451545|2451635|DEPARTMENT|39|16|Genuinely only eyes grow to a defences. Appropriate roles se|quarterly| +4121|AAAAAAAAJBABAAAA|2451545|2451635|DEPARTMENT|39|17|Effective, british tables will think. Minimum, post-war partie|quarterly| +4122|AAAAAAAAKBABAAAA|2451545|2451635|DEPARTMENT|39|18|And so on opposite charges should shake above colourfu|quarterly| +4123|AAAAAAAALBABAAAA|2451545|2451635|DEPARTMENT|39|19|Most planned standards become months. Tomorrow respective women prompt|quarterly| +4124|AAAAAAAAMBABAAAA|2451545|2451635|DEPARTMENT|39|20|Bad, other friends talk. Hungry, high miles might put then for an sig|quarterly| +4125|AAAAAAAANBABAAAA|2451545|2451635|DEPARTMENT|39|21|Minutes would know locations; sexual, small trousers bind eventually e|quarterly| +4126|AAAAAAAAOBABAAAA|2451545|2451635|DEPARTMENT|39|22|Now eastern men should install quite secondary, whit|quarterly| +4127|AAAAAAAAPBABAAAA|2451545|2451635|DEPARTMENT|39|23|New entries suspend overhead. Single carers should not depict railways. Now a|quarterly| +4128|AAAAAAAAACABAAAA|2451545|2451635|DEPARTMENT|39|24|Total parts comment elections; safe, superb projects may attend more|quarterly| +4129|AAAAAAAABCABAAAA|2451545|2451635|DEPARTMENT|39|25|Alone ways will defend social accountants. Indian churches see insufficient pictures. Chemical |quarterly| +4130|AAAAAAAACCABAAAA|2451545|2451635|DEPARTMENT|39|26|Quite uncomfortable members look in general certain points. Studies |quarterly| +4131|AAAAAAAADCABAAAA|2451545|2451635|DEPARTMENT|39|27|Once social days regard of course tragic, thin professionals. Regions answer; trousers |quarterly| +4132|AAAAAAAAECABAAAA|2451545|2451635|DEPARTMENT|39|28|Slowly modern workers check. Problems shall not bury. Low, tragic cases must not restore|quarterly| +4133|AAAAAAAAFCABAAAA|2451545|2451635|DEPARTMENT|39|29|New, dirty hospitals must walk members. Forward comfortable emotions ought to come la|quarterly| +4134|AAAAAAAAGCABAAAA|2451545|2451635|DEPARTMENT|39|30|Members shall not secure available, good deaths. Cognitive roses may |quarterly| +4135|AAAAAAAAHCABAAAA|2451545|2451635|DEPARTMENT|39|31|Pp. disturb at first neighbours. Dangerous, right views separate fairly assumpti|quarterly| +4136|AAAAAAAAICABAAAA|2451545|2451635|DEPARTMENT|39|32|Too regional provisions would accept authorities. Co|quarterly| +4137|AAAAAAAAJCABAAAA|2451545|2451635|DEPARTMENT|39|33|Terms get very inc years; cultural, available points exa|quarterly| +4138|AAAAAAAAKCABAAAA|2451545|2451635|DEPARTMENT|39|34|Instantly black pounds get during the strategies; books will try almost early things. Cool |quarterly| +4139|AAAAAAAALCABAAAA|2451545|2451635|DEPARTMENT|39|35|Close names should set earlier proper structures; ingredients should live individually|quarterly| +4140|AAAAAAAAMCABAAAA|2451545|2451635|DEPARTMENT|39|36|Sensibly high floors share throughout for a pictures. Like emplo|quarterly| +4141|AAAAAAAANCABAAAA|2451545|2451635|DEPARTMENT|39|37|No doubt great thanks arrive today. Inner schemes cou|quarterly| +4142|AAAAAAAAOCABAAAA|2451545|2451635|DEPARTMENT|39|38|Whole, redundant hills like new, powerful recordings. Controls ought to arrive perh|quarterly| +4143|AAAAAAAAPCABAAAA|2451545|2451635|DEPARTMENT|39|39|Lines can want single activities; finally single org|quarterly| +4144|AAAAAAAAADABAAAA|2451545|2451635|DEPARTMENT|39|40|Appropriately high vehicles travel then; social, alleged environments can take almost statistical |quarterly| +4145|AAAAAAAABDABAAAA|2451545|2451635|DEPARTMENT|39|41|Merchants might not say relatively strong, brilliant girls. Total, fashionable |quarterly| +4146|AAAAAAAACDABAAAA|2451545|2451635|DEPARTMENT|39|42|Then conventional voices emigrate. Serious years ought to rent from an estates; special, ro|quarterly| +4147|AAAAAAAADDABAAAA|2451545|2451635|DEPARTMENT|39|43|Laws come at a premises. Central materials should eat furthermore leaves. Great, |quarterly| +4148|AAAAAAAAEDABAAAA|2451545|2451635|DEPARTMENT|39|44|Cultural boards might face schools. There national conclusions take informal boys. Just |quarterly| +4149|AAAAAAAAFDABAAAA|2451545|2451635|DEPARTMENT|39|45|Instruments drink bright flames. Just serious traditio|quarterly| +4150|AAAAAAAAGDABAAAA|2451545|2451635|DEPARTMENT|39|46|Minimal, important effects shall not get over a views. Certain|quarterly| +4151|AAAAAAAAHDABAAAA|2451545|2451635|DEPARTMENT|39|47|Major, visual feet used to suffer more. Guilty, medi|quarterly| +4152|AAAAAAAAIDABAAAA|2451545|2451635|DEPARTMENT|39|48|Thus enormous woods must take. Then local breasts appreciat|quarterly| +4153|AAAAAAAAJDABAAAA|2451545|2451635|DEPARTMENT|39|49|Ordinary mountains reduce preliminary, certain des|quarterly| +4154|AAAAAAAAKDABAAAA|2451545|2451635|DEPARTMENT|39|50|Apparently small words remember more certain stars; convenient partners consider for a police|quarterly| +4155|AAAAAAAALDABAAAA|2451545|2451635|DEPARTMENT|39|51|As educational women could get; certain delegates afford sure, right genes. Parties would us|quarterly| +4156|AAAAAAAAMDABAAAA|2451545|2451635|DEPARTMENT|39|52|Sometimes soviet devices lead more emotions. Communities satisfy blues. Hours look magic,|quarterly| +4157|AAAAAAAANDABAAAA|2451545|2451635|DEPARTMENT|39|53|Farms overcome for a publications. Years could gain also again wrong powers. W|quarterly| +4158|AAAAAAAAODABAAAA|2451545|2451635|DEPARTMENT|39|54|Wearily future deputies can help just parties. National, complete w|quarterly| +4159|AAAAAAAAPDABAAAA|2451545|2451635|DEPARTMENT|39|55|Assets know then characteristics. Possibilities lower. Intelligent, darling democrats find her|quarterly| +4160|AAAAAAAAAEABAAAA|2451545|2451635|DEPARTMENT|39|56|Excellent, full-time curtains break by a tactics. Rates can want however correc|quarterly| +4161|AAAAAAAABEABAAAA|2451545|2451635|DEPARTMENT|39|57|Painful germans ought to match. Numerous possibilities raise almost right patients; commercia|quarterly| +4162|AAAAAAAACEABAAAA|2451545|2451635|DEPARTMENT|39|58|More complex miles understand also then happy regulations. Both environmental conclusions might t|quarterly| +4163|AAAAAAAADEABAAAA|2451545|2451635|DEPARTMENT|39|59|Star, internal weapons maintain. Things wield too social|quarterly| +4164|AAAAAAAAEEABAAAA|2451545|2451635|DEPARTMENT|39|60|Men might examine however after a words. Eyebrows counter else. Categories would not distract. Ri|quarterly| +4165|AAAAAAAAFEABAAAA|2451545|2451635|DEPARTMENT|39|61|Months may insert then national rights. Waves will experi|quarterly| +4166|AAAAAAAAGEABAAAA|2451545|2451635|DEPARTMENT|39|62|Rare materials might find probably for the difficulties. Men cannot|quarterly| +4167|AAAAAAAAHEABAAAA|2451545|2451635|DEPARTMENT|39|63|Able, additional children gather in particular common friends. Common parents used to worry similar|quarterly| +4168|AAAAAAAAIEABAAAA|2451545|2451635|DEPARTMENT|39|64|Royal, international banks can continue very long-term democrats. Symbo|quarterly| +4169|AAAAAAAAJEABAAAA|2451545|2451635|DEPARTMENT|39|65|Plants find in a weapons. Always human lawyers break modern, minute bodies. Events examine in a ch|quarterly| +4170|AAAAAAAAKEABAAAA|2451545|2451635|DEPARTMENT|39|66|Entirely sexual letters suggest probably. Wooden years may take as. Aut|quarterly| +4171|AAAAAAAALEABAAAA|2451545|2451635|DEPARTMENT|39|67|Left techniques shall not produce notably members. Agai|quarterly| +4172|AAAAAAAAMEABAAAA|2451545|2451635|DEPARTMENT|39|68|Commercial, sure terms feel solid, sure rules. Normal aims break rightly publications. Regular gui|quarterly| +4173|AAAAAAAANEABAAAA|2451545|2451635|DEPARTMENT|39|69|White, long accidents let with a forms. Left problems might take lea|quarterly| +4174|AAAAAAAAOEABAAAA|2451545|2451635|DEPARTMENT|39|70|High groups finish more tests. Opening, logical thre|quarterly| +4175|AAAAAAAAPEABAAAA|2451545|2451635|DEPARTMENT|39|71|Spare, old women make indirect letters. National, true pray|quarterly| +4176|AAAAAAAAAFABAAAA|2451545|2451635|DEPARTMENT|39|72|Too close babies blend generally increased, close ministers. Just social hands shall |quarterly| +4177|AAAAAAAABFABAAAA|2451545|2451635|DEPARTMENT|39|73|Other, firm problems dig fully methods. At all similar years will not stay again to a attempts. Gro|quarterly| +4178|AAAAAAAACFABAAAA||2451635||39|||| +4179|AAAAAAAADFABAAAA|2451545|2451635|DEPARTMENT|39|75|Original, british relatives would tell really in a references. Permanent |quarterly| +4180|AAAAAAAAEFABAAAA|2451545|2451635|DEPARTMENT|39|76|New, adequate criticisms may require hence to the problems. Long tim|quarterly| +4181|AAAAAAAAFFABAAAA|2451545|2451635|DEPARTMENT|39|77|Confidently operational officers care there cold hotels. Se|quarterly| +4182|AAAAAAAAGFABAAAA|2451545|2451635|DEPARTMENT|39|78|For instance holy conclusions might put no longer associations|quarterly| +4183|AAAAAAAAHFABAAAA|2451545|2451635|DEPARTMENT|39|79|Courses should disclose on a roses. Sales shall help. Ordinary days can solve reportedly |quarterly| +4184|AAAAAAAAIFABAAAA|2451545|2451635|DEPARTMENT|39|80|No doubt sure nations study no short, key troops. Excellent, other groups cannot hap|quarterly| +4185|AAAAAAAAJFABAAAA|2451545|2451635|DEPARTMENT|39|81|Workers cause sure documents; cases would not enhan|quarterly| +4186|AAAAAAAAKFABAAAA|2451545|2451635|DEPARTMENT|39|82|Worse only parties incorporate certainly enough advanced rules. Tools fall |quarterly| +4187|AAAAAAAALFABAAAA|2451545|2451635|DEPARTMENT|39|83|Things ought to change at the votes. Regular, impossible|quarterly| +4188|AAAAAAAAMFABAAAA|2451545|2451635|DEPARTMENT|39|84|Fully blue documents become things. Greatly other men would add long in a steps. C|quarterly| +4189|AAAAAAAANFABAAAA|2451545|2451635|DEPARTMENT|39|85|Numbers will marry temporarily in order constitutional spaces. A|quarterly| +4190|AAAAAAAAOFABAAAA|2451545|2451635|DEPARTMENT|39|86|Women used to use once. Busy pockets ought to represent far strong, existing images. S|quarterly| +4191|AAAAAAAAPFABAAAA|2451545|2451635|DEPARTMENT|39|87|Prime days get now literary artists. New names can move ho|quarterly| +4192|AAAAAAAAAGABAAAA|2451545|2451635|DEPARTMENT|39|88|Softly minor topics see. Available, physical services get ot|quarterly| +4193|AAAAAAAABGABAAAA|2451545|2451635|DEPARTMENT|39|89|Crimes must not enter green, wrong theories. A little different needs used to make in part e|quarterly| +4194|AAAAAAAACGABAAAA|2451545|2451635|DEPARTMENT|39|90|Environmental, intermediate men might combine later in a sites. Slim, safe m|quarterly| +4195|AAAAAAAADGABAAAA|2451545|2451635|DEPARTMENT|39|91|New, flexible specimens might shoot once very organisms. Also serious years suppos|quarterly| +4196|AAAAAAAAEGABAAAA|2451545|2451635|DEPARTMENT|39|92|Much financial members assemble possible children. E|quarterly| +4197|AAAAAAAAFGABAAAA|2451545|2451635|DEPARTMENT|39|93|Single, following jobs can find in the conditions; available, hard communities shall grasp. Vict|quarterly| +4198|AAAAAAAAGGABAAAA|2451545|2451635|DEPARTMENT|39|94|Easy, different services should avoid in a areas; guns pick most sensitive leaders. Even |quarterly| +4199|AAAAAAAAHGABAAAA|2451545|2451635|DEPARTMENT|39|95|Just specialist points give great, historic options. Near mental blocks can believe only female, |quarterly| +4200|AAAAAAAAIGABAAAA|2451545|2451635|DEPARTMENT|39|96|Soon legal programmes hang worse times. Correct, lik|quarterly| +4201|AAAAAAAAJGABAAAA|2451545|2451635|DEPARTMENT|39|97|Aloud different stars remember base heads; pairs may pass here. Areas may see together |quarterly| +4202|AAAAAAAAKGABAAAA|2451545|2451635|DEPARTMENT|39|98|Difficult, wide tables expand remarkable boards. Hostile, cent|quarterly| +4203|AAAAAAAALGABAAAA|2451545|2451635|DEPARTMENT|39|99|A little visible colours follow here different, political scholars. Common, priva|quarterly| +4204|AAAAAAAAMGABAAAA|2451545|2451635|DEPARTMENT|39|100|More civil pounds could not take just; successfully mysterious t|quarterly| +4205|AAAAAAAANGABAAAA|2451545|2451635|DEPARTMENT|39|101|Natural divisions must get commercial, difficult grounds. Unlike, british policies may not t|quarterly| +4206|AAAAAAAAOGABAAAA|2451545|2451635|DEPARTMENT|39|102|Much toxic children know so royal, great shareholders. A|quarterly| +4207|AAAAAAAAPGABAAAA|2451545|2451635|DEPARTMENT|39|103|Leaders shall become. Later back teachers may not shake e|quarterly| +4208|AAAAAAAAAHABAAAA|2451545|2451635|DEPARTMENT|39|104|Perhaps domestic hundreds ask well ltd., expected patients. Exact vehicles recogn|quarterly| +4209|AAAAAAAABHABAAAA|2451545|2451635|DEPARTMENT|39|105|For example leading times shall navigate in a rights. Careful players|quarterly| +4210|AAAAAAAACHABAAAA|2451545|2451635|DEPARTMENT|39|106|Years confirm simply. German futures change certainly. Double fields demand early, single subj|quarterly| +4211|AAAAAAAADHABAAAA|2451545|2451635|DEPARTMENT|39|107|Laws would not possess well possible implications; signs increase. Seconds shall |quarterly| +4212|AAAAAAAAEHABAAAA|2451545|2451635|DEPARTMENT|39|108|Able goods would not cool almost possible years. Fo|quarterly| +4213|AAAAAAAAFHABAAAA|2451636|2451726|DEPARTMENT|40|1|Open, related offences hammer; widely other contributions min|quarterly| +4214|AAAAAAAAGHABAAAA|2451636|2451726|DEPARTMENT|40|2|Local workshops could not understand especially; substantial funds ought|quarterly| +4215|AAAAAAAAHHABAAAA|2451636|2451726|DEPARTMENT|40|3|Protective, public blues used to occur sales. Large, jewish vessels go enou|quarterly| +4216|AAAAAAAAIHABAAAA|2451636|2451726|DEPARTMENT|40|4|Rapid, european reports used to return more at a products; underly|quarterly| +4217|AAAAAAAAJHABAAAA|2451636|2451726|DEPARTMENT|40|5|Other, great rules may live high facilities; prepared women will maintain |quarterly| +4218|AAAAAAAAKHABAAAA|2451636|2451726|DEPARTMENT|40|6|Actually magnetic lights might diminish around generally contrary agents. Schools can |quarterly| +4219|AAAAAAAALHABAAAA|2451636|2451726|DEPARTMENT|40|7|National claims could convert. Thus clear points might change at an periods. Appropriate books|quarterly| +4220|AAAAAAAAMHABAAAA|2451636|2451726|DEPARTMENT|40|8|Standard, international pupils give circumstances. Well new occasions i|quarterly| +4221|AAAAAAAANHABAAAA|2451636|2451726|DEPARTMENT|40|9|Arms send. Rational angles cultivate only bases; teache|quarterly| +4222|AAAAAAAAOHABAAAA|2451636|2451726|DEPARTMENT|40|10|And so on historic computers can lie often through the partners; le|quarterly| +4223|AAAAAAAAPHABAAAA|2451636|2451726|DEPARTMENT|40|11|Over detailed artists recognize further swiss wines; winners used to imitat|quarterly| +4224|AAAAAAAAAIABAAAA|2451636|2451726|DEPARTMENT|40|12|Working, capable months might find moreover foreign reasons. Vi|quarterly| +4225|AAAAAAAABIABAAAA|2451636|2451726|DEPARTMENT|40|13|For example large figures ought to postpone. Following,|quarterly| +4226|AAAAAAAACIABAAAA|2451636|2451726|DEPARTMENT|40|14|Feelings smile either limited speeches. Other, psychiatric plants co|quarterly| +4227|AAAAAAAADIABAAAA|2451636|2451726|DEPARTMENT|40|15|Northwards different sites feel then concerned, costly cars. Other minutes m|quarterly| +4228|AAAAAAAAEIABAAAA|2451636|2451726|DEPARTMENT|40|16|Single, major plants wait pounds; words can like well formal workers. Price|quarterly| +4229|AAAAAAAAFIABAAAA|2451636|2451726|DEPARTMENT|40|17|Freely mental designs convince however annual, helpful beings. Elegant arguments|quarterly| +4230|AAAAAAAAGIABAAAA|2451636|2451726|DEPARTMENT|40|18|Foreign, valuable conferences take almost fine, rapid earnings. Conditions fee|quarterly| +4231|AAAAAAAAHIABAAAA|2451636|2451726|DEPARTMENT|40|19|Anyway catholic knees used to modify above natural oth|quarterly| +4232|AAAAAAAAIIABAAAA|2451636|2451726|DEPARTMENT|40|20|Places prevail main, small members. Proportions ought to break essentiall|quarterly| +4233|AAAAAAAAJIABAAAA|2451636|2451726|DEPARTMENT|40|21|Gay countries could increase as on top of a projects. Unable, holy trees mig|quarterly| +4234|AAAAAAAAKIABAAAA|2451636|2451726|DEPARTMENT|40|22|Wet experts restore surely final benefits; independent, daily deposits equate |quarterly| +4235|AAAAAAAALIABAAAA|2451636|2451726|DEPARTMENT|40|23|Brothers become even police; necessary teachers must not stifle roles. Very|quarterly| +4236|AAAAAAAAMIABAAAA|2451636|2451726|DEPARTMENT|40|24|Suitable, similar managers solve for instance others; servants ought to lie then natural amount|quarterly| +4237|AAAAAAAANIABAAAA|2451636|2451726|DEPARTMENT|40|25|Capital offenders heat often empty parents. Sales enjoy for a days. Royal eyes share; und|quarterly| +4238|AAAAAAAAOIABAAAA|2451636|2451726|DEPARTMENT|40|26|Areas used to try simply available, unchanged girls. Unusual offers ought to help shares; i|quarterly| +4239|AAAAAAAAPIABAAAA|2451636|2451726|DEPARTMENT|40|27|Japanese, illegal factors will tell. Weeks warm stron|quarterly| +4240|AAAAAAAAAJABAAAA|2451636|2451726|DEPARTMENT|40|28|Rebels trigger only to a words; no longer underlying numbers stop figures. M|quarterly| +4241|AAAAAAAABJABAAAA|2451636|2451726|DEPARTMENT|40|29|Proposals would produce. Major casualties make british cells. Copies could continue outsta|quarterly| +4242|AAAAAAAACJABAAAA|2451636|2451726|DEPARTMENT|40|30|Detailed relations shall sell of course sole, difficul|quarterly| +4243|AAAAAAAADJABAAAA|2451636|2451726|DEPARTMENT|40|31|Commercial changes ask. Great companies call to a operations. Times use quietly plea|quarterly| +4244|AAAAAAAAEJABAAAA|2451636|2451726|DEPARTMENT|40|32|Recommendations go new, international animals; and s|quarterly| +4245|AAAAAAAAFJABAAAA|2451636|2451726|DEPARTMENT|40|33|However dependent years blame strongly. International, natio|quarterly| +4246|AAAAAAAAGJABAAAA|2451636|2451726|DEPARTMENT|40|34|Others must get. Also slight enemies endure great, extreme wars. Expert men can us|quarterly| +4247|AAAAAAAAHJABAAAA|2451636|2451726|DEPARTMENT|40|35|Physical feelings facilitate at least from a weeks. Wonderful, additional patients consider ill |quarterly| +4248|AAAAAAAAIJABAAAA|2451636|2451726|DEPARTMENT|40|36|As good needs can take studies. Forthcoming, vulnerable terms ought to dry into a feet. Ofte|quarterly| +4249|AAAAAAAAJJABAAAA|2451636|2451726|DEPARTMENT|40|37|Likely, strange activities should see so coming items. Only, separate plans hear. Natur|quarterly| +4250|AAAAAAAAKJABAAAA|2451636|2451726|DEPARTMENT|40|38|Multiple, old factors observe large services; political, |quarterly| +4251|AAAAAAAALJABAAAA|2451636|2451726|DEPARTMENT|40|39|Units explain. Available, only women deliver far in a contexts. Prev|quarterly| +4252|AAAAAAAAMJABAAAA|2451636|2451726|DEPARTMENT|40|40|Academic, minimal methods result long full, hot members|quarterly| +4253|AAAAAAAANJABAAAA|2451636|2451726|DEPARTMENT|40|41|Just old efforts pack still right funds. Plants could share perhaps objectives. So as social |quarterly| +4254|AAAAAAAAOJABAAAA|2451636|2451726|DEPARTMENT|40|42|Perhaps collective ministers must not separate true, central off|quarterly| +4255|AAAAAAAAPJABAAAA|2451636|2451726|DEPARTMENT|40|43|Standards take points; practical, new intentions work positive players. Tickets upset pr|quarterly| +4256|AAAAAAAAAKABAAAA|2451636|2451726|DEPARTMENT|40|44|Annually previous goals could express risks. Programmes ensure just new months. L|quarterly| +4257|AAAAAAAABKABAAAA|2451636|2451726|DEPARTMENT|40|45|High top services give. Here natural stars shall keep faces. Usually|quarterly| +4258|AAAAAAAACKABAAAA|2451636|2451726|DEPARTMENT|40|46|Novels would not demonstrate. Sales produce much shareholders. Then old talks ignore with a genes|quarterly| +4259|AAAAAAAADKABAAAA|2451636|2451726|DEPARTMENT|40|47|Just professional terms will activate indeed national, old facts; near present children |quarterly| +4260|AAAAAAAAEKABAAAA|2451636|2451726|DEPARTMENT|40|48|Workers could not supply in a children. Profitable, psychological worker|quarterly| +4261|AAAAAAAAFKABAAAA|2451636|2451726|DEPARTMENT|40|49|Local ranks prevent to a women. Things will leave hard. Intimate cases own better great remark|quarterly| +4262|AAAAAAAAGKABAAAA|2451636|2451726|DEPARTMENT|40|50|Major, profound lips can go sooner good, special months; tests may not match for example c|quarterly| +4263|AAAAAAAAHKABAAAA|2451636|2451726|DEPARTMENT|40|51|Ancient goods may get used messages. Studies may investigate only between a troop|quarterly| +4264|AAAAAAAAIKABAAAA|2451636|2451726|DEPARTMENT|40|52|Still other risks must find future areas. Entirely political members coul|quarterly| +4265|AAAAAAAAJKABAAAA|2451636|2451726|DEPARTMENT|40|53|Terms might not respond major, hot causes. Local measures used to find natural ships; |quarterly| +4266|AAAAAAAAKKABAAAA|2451636|2451726|DEPARTMENT|40|54|Clothes know there increases. White men wear wrongly within a lines; tonnes used to think|quarterly| +4267|AAAAAAAALKABAAAA|2451636|2451726|DEPARTMENT|40|55|Hours would pass ago easy programmes. Standard factors cannot happen only, good cultures. New, n|quarterly| +4268|AAAAAAAAMKABAAAA|2451636|2451726|DEPARTMENT|40|56|Examples shrink better looks; so good lives admit large others. L|quarterly| +4269|AAAAAAAANKABAAAA|2451636|2451726|DEPARTMENT|40|57|Distinct, total lips can suspect alarmingly potential properties; current, big equatio|quarterly| +4270|AAAAAAAAOKABAAAA|2451636|2451726|DEPARTMENT|40|58|Important years need instead following, poor clients. Full, eventual issues must move ever; cas|quarterly| +4271|AAAAAAAAPKABAAAA|2451636|2451726|DEPARTMENT|40|59|Young, whole policies must not come quite there high sta|quarterly| +4272|AAAAAAAAALABAAAA|2451636|2451726|DEPARTMENT|40|60|Distinctly religious women release then about important individuals|quarterly| +4273|AAAAAAAABLABAAAA|2451636|2451726|DEPARTMENT|40|61|Certainly head islands must show also. Changes provid|quarterly| +4274|AAAAAAAACLABAAAA|2451636|2451726|DEPARTMENT|40|62|Private savings must destroy too. Just, green breasts stay little, particular coi|quarterly| +4275|AAAAAAAADLABAAAA|2451636|2451726|DEPARTMENT|40|63|Common tests might not find earlier open, small pol|quarterly| +4276|AAAAAAAAELABAAAA|2451636|2451726|DEPARTMENT|40|64|Hands take expressly thousands. Properly professional voices carry ever duties. Littl|quarterly| +4277|AAAAAAAAFLABAAAA|2451636|2451726|DEPARTMENT|40|65|Powers would turn before today toxic benefits. Thin, sensitive years shall cease clearl|quarterly| +4278|AAAAAAAAGLABAAAA|2451636|2451726|DEPARTMENT|40|66|Then early delegates cancel military jobs. Much new styles could |quarterly| +4279|AAAAAAAAHLABAAAA|2451636|2451726|DEPARTMENT|40|67|Even rigid areas take else local hands; classes must not argue during a others|quarterly| +4280|AAAAAAAAILABAAAA|2451636|2451726|DEPARTMENT|40|68|From time to time complex sources shall freeze across t|quarterly| +4281|AAAAAAAAJLABAAAA|2451636|2451726|DEPARTMENT|40|69|Really expensive intervals might take. Expenses must not |quarterly| +4282|AAAAAAAAKLABAAAA|2451636|2451726|DEPARTMENT|40|70|Subsequent doors would find perhaps as medical years. Then scottish costs may not s|quarterly| +4283|AAAAAAAALLABAAAA|2451636|2451726|DEPARTMENT|40|71|Never short accounts rely here senior runs. Solid hours work reasons. Blue, publi|quarterly| +4284|AAAAAAAAMLABAAAA|2451636|2451726|DEPARTMENT|40|72|Clean, alone potatoes construct. So past arrangements give here |quarterly| +4285|AAAAAAAANLABAAAA|2451636|2451726|DEPARTMENT|40|73|Straight established writers help carefully days. Available, mental thanks pa|quarterly| +4286|AAAAAAAAOLABAAAA|2451636|2451726|DEPARTMENT|40|74|Hidden police act asian days. Then difficult years must not cut further in |quarterly| +4287|AAAAAAAAPLABAAAA|2451636|2451726|DEPARTMENT|40|75|All immediate years must not go at all parties. Home mere paren|quarterly| +4288|AAAAAAAAAMABAAAA|2451636|2451726|DEPARTMENT|40|76|Other countries satisfy good, white duties; services must play mos|quarterly| +4289|AAAAAAAABMABAAAA|2451636|2451726|DEPARTMENT|40|77|Unknown conditions tell more in a books; present, large pains could generat|quarterly| +4290|AAAAAAAACMABAAAA|2451636|2451726|DEPARTMENT|40|78|Empty, inevitable names retain factors; strong, full dea|quarterly| +4291|AAAAAAAADMABAAAA|2451636|2451726|DEPARTMENT|40|79|Nuclear, famous events sail political advisers. Sometimes ph|quarterly| +4292|AAAAAAAAEMABAAAA|2451636|2451726|DEPARTMENT|40|80|Quite strange tourists could start well letters. Permanently desperate elements exp|quarterly| +4293|AAAAAAAAFMABAAAA|2451636|2451726|DEPARTMENT|40|81|New, civil respects shall dispute careful scientists. Gradually necessary weeks should cr|quarterly| +4294|AAAAAAAAGMABAAAA|2451636|2451726|DEPARTMENT|40|82|Fair, able terms will not impose however economic lives. Other, able homes seek. Now p|quarterly| +4295|AAAAAAAAHMABAAAA|2451636|2451726|DEPARTMENT|40|83|Familiar members cover. Running, social letters keep home in front of the cases. F|quarterly| +4296|AAAAAAAAIMABAAAA|2451636|2451726|DEPARTMENT|40|84|Environmental letters climb later patients. British |quarterly| +4297|AAAAAAAAJMABAAAA|2451636|2451726|DEPARTMENT|40|85|Large, different moments remember all right personnel; given risks used |quarterly| +4298|AAAAAAAAKMABAAAA|2451636|2451726|DEPARTMENT|40|86|Over great pounds put never payments; wars appear even big, critical|quarterly| +4299|AAAAAAAALMABAAAA|2451636|2451726|DEPARTMENT|40|87|Fairly dominant views meet new, final problems. Detailed, si|quarterly| +4300|AAAAAAAAMMABAAAA|2451636|2451726|DEPARTMENT|40|88|Digital, free men take high. Lives remind yet requirements. Children act rewards; sole natio|quarterly| +4301|AAAAAAAANMABAAAA|2451636|2451726|DEPARTMENT|40|89|Eventually central requirements may boot home as ready mi|quarterly| +4302|AAAAAAAAOMABAAAA|2451636|2451726|DEPARTMENT|40|90|Wide, literary decisions introduce times. Materials|quarterly| +4303|AAAAAAAAPMABAAAA|2451636|2451726|DEPARTMENT|40|91|Complete, warm minutes may not think now associated rights. Again great victims shall believe other|quarterly| +4304|AAAAAAAAANABAAAA|2451636|2451726|DEPARTMENT|40|92|Front terms must emerge at a projects. Small, economic rates may say minimal, right s|quarterly| +4305|AAAAAAAABNABAAAA|2451636|2451726|DEPARTMENT|40|93|New, human classes begin different, beautiful thoughts. Early, clear figu|quarterly| +4306|AAAAAAAACNABAAAA|2451636|2451726|DEPARTMENT|40|94|Young, pleased effects can lose applications. Always new opportunities let |quarterly| +4307|AAAAAAAADNABAAAA|2451636|2451726|DEPARTMENT|40|95|Safe, single friends see probably unpleasant, increased gla|quarterly| +4308|AAAAAAAAENABAAAA|2451636|2451726|DEPARTMENT|40|96|Little doors should afford. Big clubs could not wear prime, political premises. New colours ough|quarterly| +4309|AAAAAAAAFNABAAAA|2451636|2451726|DEPARTMENT|40|97|Things consider alone. Minor, good sanctions cannot shape often. |quarterly| +4310|AAAAAAAAGNABAAAA|2451636|2451726|DEPARTMENT|40|98|Attractive, national minutes like. Days reconsider at least small doctors. Brother|quarterly| +4311|AAAAAAAAHNABAAAA|2451636|2451726|DEPARTMENT|40|99|Both main users would not join even over a teachers. Medical, large orga|quarterly| +4312|AAAAAAAAINABAAAA|2451636|2451726|DEPARTMENT|40|100|Explicit, good officers remember sincerely fields. Elements could damage some|quarterly| +4313|AAAAAAAAJNABAAAA|2451636|2451726|DEPARTMENT|40|101|New, other waters offer with a members. So white achievements would imagine presen|quarterly| +4314|AAAAAAAAKNABAAAA|2451636|2451726|DEPARTMENT|40|102|Disastrous, spiritual numbers keep levels; other, famous functions will withdraw|quarterly| +4315|AAAAAAAALNABAAAA|2451636|2451726|DEPARTMENT|40|103|Also fundamental players choose well poor years. Long difficult cases would not mean far cat|quarterly| +4316|AAAAAAAAMNABAAAA|2451636|2451726|DEPARTMENT|40|104|Late interests may not dance low concerns; heavy, willing problems teac|quarterly| +4317|AAAAAAAANNABAAAA|2451636|2451726|DEPARTMENT|40|105|Users want small, geographical cells. Nations will not reflect from a que|quarterly| +4318|AAAAAAAAONABAAAA|2451636|2451726|DEPARTMENT|40|106|Fairly independent letters establish at present classical seco|quarterly| +4319|AAAAAAAAPNABAAAA|2451636|2451726|DEPARTMENT|40|107|There difficult factors insist administrative, military tickets; fresh, appropriate women s|quarterly| +4320|AAAAAAAAAOABAAAA|2451636|2451726|DEPARTMENT|40|108|Usually other responses used to get just more extended companies. Alive|quarterly| +4321|AAAAAAAABOABAAAA|2451727|2451817|DEPARTMENT|41|1|Necessary, new views should not make new rates. Parties shall feel further stones. Past ye|quarterly| +4322|AAAAAAAACOABAAAA|2451727|2451817|DEPARTMENT|41|2|Unexpected, natural workers like italian levels. Rural, social requ|quarterly| +4323|AAAAAAAADOABAAAA|2451727|2451817|DEPARTMENT|41|3|Once ultimate days could not talk there. Trees realise |quarterly| +4324|AAAAAAAAEOABAAAA|2451727|2451817|DEPARTMENT|41|4|Open, marginal accounts understand major terms. Serious workers could not|quarterly| +4325|AAAAAAAAFOABAAAA|2451727|2451817|DEPARTMENT|41|5|Extremely total ends understand less specified, difficult patients|quarterly| +4326|AAAAAAAAGOABAAAA|2451727|2451817|DEPARTMENT|41|6|Open eggs might follow however private courses. Gre|quarterly| +4327|AAAAAAAAHOABAAAA|2451727|2451817|DEPARTMENT|41|7|Various, active papers cannot provide as. Dead services surprise potential, eng|quarterly| +4328|AAAAAAAAIOABAAAA|2451727|2451817|DEPARTMENT|41|8|Then medical terms increase rules. More base forms ought to appear successfully|quarterly| +4329|AAAAAAAAJOABAAAA|2451727|2451817|DEPARTMENT|41|9|Relatively inappropriate views kill for a rooms; special w|quarterly| +4330|AAAAAAAAKOABAAAA|2451727|2451817|DEPARTMENT|41|10|Lives look ago components. Comparative, soft women allow recently words. Different, |quarterly| +4331|AAAAAAAALOABAAAA|2451727|2451817|DEPARTMENT|41|11|Hands would adopt flat ways. Here usual changes shall disrupt by the |quarterly| +4332|AAAAAAAAMOABAAAA|2451727|2451817|DEPARTMENT|41|12|Nice hundreds discuss almost ideological, just effects|quarterly| +4333|AAAAAAAANOABAAAA|2451727|2451817|DEPARTMENT|41|13|Qualifications expel organisms; free soldiers make now independent issues|quarterly| +4334|AAAAAAAAOOABAAAA|2451727|2451817|DEPARTMENT|41|14|Names could fancy never peculiar gains. Corresponding defendants must wake el|quarterly| +4335|AAAAAAAAPOABAAAA|2451727|2451817|DEPARTMENT|41|15|Only main historians reply however. Germans visit well times. Either minimal applica|quarterly| +4336|AAAAAAAAAPABAAAA|2451727|2451817|DEPARTMENT|41|16|Users swallow then strongly existing forms. There military subjects expand more |quarterly| +4337|AAAAAAAABPABAAAA|2451727|2451817|DEPARTMENT|41|17|Colleagues need of course active buildings. Simple children read a|quarterly| +4338|AAAAAAAACPABAAAA|2451727|2451817|DEPARTMENT|41|18|Known rights make for the candidates. Very other d|quarterly| +4339|AAAAAAAADPABAAAA|2451727|2451817|DEPARTMENT|41|19|Matters decorate enormously remarkable, prospective organizations. |quarterly| +4340|AAAAAAAAEPABAAAA|2451727|2451817|DEPARTMENT|41|20|Expected, cool issues should manage friends. Important workers t|quarterly| +4341|AAAAAAAAFPABAAAA|2451727|2451817|DEPARTMENT|41|21|Acute responses would join purposes. Particular, other patients avoid actually wet,|quarterly| +4342|AAAAAAAAGPABAAAA|2451727|2451817|DEPARTMENT|41|22|Available, new sites could not become; tears come long by a women. Other |quarterly| +4343|AAAAAAAAHPABAAAA|2451727|2451817|DEPARTMENT|41|23|Empty galleries should want from a items; plans must try so red ser|quarterly| +4344|AAAAAAAAIPABAAAA|2451727|2451817|DEPARTMENT|41|24|Direct employees continue industrial, still developments. Qu|quarterly| +4345|AAAAAAAAJPABAAAA|2451727|2451817|DEPARTMENT|41|25|Tests can sue away true, small years. Foreign, desirable newspap|quarterly| +4346|AAAAAAAAKPABAAAA|2451727|2451817|DEPARTMENT|41|26|Reactions would aim in a molecules. Soviet questions may not come slow hot vehicles.|quarterly| +4347|AAAAAAAALPABAAAA|2451727|2451817|DEPARTMENT|41|27|Public, basic matters bow only from a readers. Other schools fight local, differen|quarterly| +4348|AAAAAAAAMPABAAAA|2451727|2451817|DEPARTMENT|41|28|Labour, similar jeans turn very. British, strong premises should like so against a ki|quarterly| +4349|AAAAAAAANPABAAAA|2451727|2451817|DEPARTMENT|41|29|Live, pale systems provide to a parents. Etc internatio|quarterly| +4350|AAAAAAAAOPABAAAA|2451727|2451817|DEPARTMENT|41|30|International, happy wives make even so new women. Certain news|quarterly| +4351|AAAAAAAAPPABAAAA|2451727|2451817|DEPARTMENT|41|31|Different, local stands promote relationships. Raw reasons end rather p|quarterly| +4352|AAAAAAAAAABBAAAA|2451727|2451817|DEPARTMENT|41|32|Cases handle besides hands. Records will not avoid odd, important passengers. Gradual, proposed fie|quarterly| +4353|AAAAAAAABABBAAAA|2451727|2451817|DEPARTMENT|41|33|Available, illegal occasions ought to depend again recent, s|quarterly| +4354|AAAAAAAACABBAAAA|2451727|2451817|DEPARTMENT|41|34|So individual companies could dispose new friends; shortly irish pr|quarterly| +4355|AAAAAAAADABBAAAA|2451727|2451817|DEPARTMENT|41|35|Most central pairs treat quite small, general results. Physical, detailed |quarterly| +4356|AAAAAAAAEABBAAAA|2451727|2451817|DEPARTMENT|41|36|Technically western events get quite foreign meanings. Mess|quarterly| +4357|AAAAAAAAFABBAAAA|2451727|2451817|DEPARTMENT|41|37|Customs go. Rarely miserable hours must not return circumstances; islamic details would consul|quarterly| +4358|AAAAAAAAGABBAAAA|2451727|2451817|DEPARTMENT|41|38|Always british pictures may not witness fatal executives. S|quarterly| +4359|AAAAAAAAHABBAAAA|2451727|2451817|DEPARTMENT|41|39|National miners will not support even publications|quarterly| +4360|AAAAAAAAIABBAAAA|2451727|2451817||||Horses support future, great days. Good days shoul|quarterly| +4361|AAAAAAAAJABBAAAA|2451727|2451817|DEPARTMENT|41|41|Appropriate interests could climb even thereafter particular concerns. |quarterly| +4362|AAAAAAAAKABBAAAA|2451727|2451817|DEPARTMENT|41|42|Difficult, recent friends maintain present aspirations. Home automatic skills ought to expa|quarterly| +4363|AAAAAAAALABBAAAA|2451727|2451817|DEPARTMENT|41|43|Rough, bright years would visit simply writings. Minutes shoot othe|quarterly| +4364|AAAAAAAAMABBAAAA|2451727|2451817|DEPARTMENT|41|44|Others let thus local eyes. Directly new courses might help again; bed|quarterly| +4365|AAAAAAAANABBAAAA|2451727|2451817|DEPARTMENT|41|45|Western, available errors may not go true, special ways; well good places shall know|quarterly| +4366|AAAAAAAAOABBAAAA|2451727|2451817|DEPARTMENT|41|46|Trials would attain then possible centres. Main days |quarterly| +4367|AAAAAAAAPABBAAAA|2451727|2451817|DEPARTMENT|41|47|Words would take so as new children; wrong researchers name here years.|quarterly| +4368|AAAAAAAAABBBAAAA|2451727|2451817|DEPARTMENT|41|48|So recent arms switch short. Parts used to eliminate living, long others. More important|quarterly| +4369|AAAAAAAABBBBAAAA|2451727|2451817|DEPARTMENT|41|49|Possible forces sit then german, typical operations. Nationally excessive benefits must not employ |quarterly| +4370|AAAAAAAACBBBAAAA|2451727|2451817|DEPARTMENT|41|50|Great versions may not benefit by a thoughts; blue, active trades must |quarterly| +4371|AAAAAAAADBBBAAAA|2451727|2451817|DEPARTMENT|41|51|Expected claims may not recognise. Inside personal facts could replace |quarterly| +4372|AAAAAAAAEBBBAAAA|2451727|2451817|DEPARTMENT|41|52|Labour friends avoid overall strange, bad men. Little, long terms learn economies. Black, regular c|quarterly| +4373|AAAAAAAAFBBBAAAA|2451727|2451817|DEPARTMENT|41|53|No doubt special difficulties will not talk relatively. Systems will not finish conse|quarterly| +4374|AAAAAAAAGBBBAAAA|2451727|2451817|DEPARTMENT|41|54|Councillors represent real, white streets. Men keep still mor|quarterly| +4375|AAAAAAAAHBBBAAAA|2451727|2451817|DEPARTMENT|41|55|Spanish, unable plants should not bother workers. Rou|quarterly| +4376|AAAAAAAAIBBBAAAA|2451727|2451817|DEPARTMENT|41|56|Letters might come warm relationships. New, rigid bodies hold never enough random p|quarterly| +4377|AAAAAAAAJBBBAAAA|2451727|2451817|DEPARTMENT|41|57|Common subjects remain otherwise. Neighbours organise then perhaps new m|quarterly| +4378|AAAAAAAAKBBBAAAA|2451727|2451817|DEPARTMENT|41|58|New, other others ought to need. Suddenly great pictures could help now about young cos|quarterly| +4379|AAAAAAAALBBBAAAA|2451727|2451817|DEPARTMENT|41|59|Absolutely molecular services would not indicate nearly dem|quarterly| +4380|AAAAAAAAMBBBAAAA|2451727|2451817|DEPARTMENT|41|60|Competitive, social stocks used to suffer historical opportunities. Written, unl|quarterly| +4381|AAAAAAAANBBBAAAA|2451727|2451817|DEPARTMENT|41|61|Rooms might reach single years. Homes revert then extreme parts; shadows must wear spec|quarterly| +4382|AAAAAAAAOBBBAAAA|2451727|2451817|DEPARTMENT|41|62|Years strike so new, huge improvements; both appropria|quarterly| +4383|AAAAAAAAPBBBAAAA|2451727|2451817|DEPARTMENT|41|63|Only human areas ought to care usually surprised successful breasts. Highest rough |quarterly| +4384|AAAAAAAAACBBAAAA|2451727|2451817|DEPARTMENT|41|64|Lengthy, european wives will not buy also words. Unlikely members think finger|quarterly| +4385|AAAAAAAABCBBAAAA|2451727|2451817|DEPARTMENT|41|65|Private movements give again. Right, short villages prevent then against the visitors. Estates da|quarterly| +4386|AAAAAAAACCBBAAAA|2451727|2451817|DEPARTMENT|41|66|Local, single occupations can speak most meetings. More new effects update at least miners. Other, |quarterly| +4387|AAAAAAAADCBBAAAA|2451727|2451817|DEPARTMENT|41|67|Ancient universities produce thereafter effective offic|quarterly| +4388|AAAAAAAAECBBAAAA|2451727|2451817|DEPARTMENT|41|68|Chief, only hours go full, free chemicals. Pounds draw situations. Numbers shall bring onl|quarterly| +4389|AAAAAAAAFCBBAAAA|2451727|2451817|DEPARTMENT|41|69|Acceptable rights result long-term patients. Senior ag|quarterly| +4390|AAAAAAAAGCBBAAAA|2451727|2451817|DEPARTMENT|41|70|Now right years must not understand very only social workers|quarterly| +4391|AAAAAAAAHCBBAAAA|2451727|2451817|DEPARTMENT|41|71|Sure decisions will head perfectly. Running eyes ought to increase. Basic thoughts close alone chi|quarterly| +4392|AAAAAAAAICBBAAAA|2451727|2451817|DEPARTMENT|41|72|New services learn small changes; guns generate still popular parents. High, main sentences|quarterly| +4393|AAAAAAAAJCBBAAAA|2451727|2451817|DEPARTMENT|41|73|Certain sources will speed old holders; later classic|quarterly| +4394|AAAAAAAAKCBBAAAA|2451727|2451817|DEPARTMENT|41|74|Contrary, catholic hands could not equate. Anxious numbers admit clearly together separate |quarterly| +4395|AAAAAAAALCBBAAAA|2451727|2451817|DEPARTMENT|41|75|Patients cannot lead highly labour, automatic cells. New, common things allow price|quarterly| +4396|AAAAAAAAMCBBAAAA|2451727|2451817|DEPARTMENT|41|76|Techniques could not need often publicly permanent discussions. Important, effective view|quarterly| +4397|AAAAAAAANCBBAAAA|2451727|2451817|DEPARTMENT|41|77|Consumers used to increase for a decades. Schools explore consequently in a employers. Hidde|quarterly| +4398|AAAAAAAAOCBBAAAA|2451727|2451817|DEPARTMENT|41|78|Unknown poems yield hard, present votes. Couples could move|quarterly| +4399|AAAAAAAAPCBBAAAA|2451727|2451817|DEPARTMENT|41|79|However british glasses can avail supporters. Inside dangerous firms clean youn|quarterly| +4400|AAAAAAAAADBBAAAA|2451727|2451817|DEPARTMENT|41|80|Still local products can study later powerful origi|quarterly| +4401|AAAAAAAABDBBAAAA|2451727|2451817|DEPARTMENT|41|81|Figures must not gain never. High, good parents create today by|quarterly| +4402|AAAAAAAACDBBAAAA|2451727|2451817|DEPARTMENT|41|82|Related times buy particularly so old-fashioned household|quarterly| +4403|AAAAAAAADDBBAAAA|2451727|2451817|DEPARTMENT|41|83|Inches open confidently later atlantic players. Public h|quarterly| +4404|AAAAAAAAEDBBAAAA|2451727|2451817|DEPARTMENT|41|84|Upper, able costs prevent social, intellectual volumes. Operational, proper cheeks might n|quarterly| +4405|AAAAAAAAFDBBAAAA|2451727|2451817|DEPARTMENT|41|85|Workers hit on board different, public boys. Subject tests tempt already however legal ser|quarterly| +4406|AAAAAAAAGDBBAAAA|2451727|2451817|DEPARTMENT|41|86|Confident interests can lead yesterday social feet. Specified, new times like occasio|quarterly| +4407|AAAAAAAAHDBBAAAA|2451727|2451817|DEPARTMENT|41|87|Clubs could know prime, natural holders. Unusual minis|quarterly| +4408|AAAAAAAAIDBBAAAA|2451727|2451817|DEPARTMENT|41|88|Railways make somehow shoulders. Stiffly married minutes shall act|quarterly| +4409|AAAAAAAAJDBBAAAA|2451727|2451817|DEPARTMENT|41|89|Sexual, sensible years must rub carefully still alive categories; dramatic, popular text|quarterly| +4410|AAAAAAAAKDBBAAAA|2451727|2451817|DEPARTMENT|41|90|Thus eager visitors used to fit under a faces. Level, different applications will not build s|quarterly| +4411|AAAAAAAALDBBAAAA|2451727|2451817|DEPARTMENT|41|91|Perfect areas ought to give available, efficient women. Rich lines start changes. Instead|quarterly| +4412|AAAAAAAAMDBBAAAA|2451727|2451817|DEPARTMENT|41|92|National, physical soldiers date regular, vague appointments. Senior prices supplem|quarterly| +4413|AAAAAAAANDBBAAAA|2451727|2451817|DEPARTMENT|41|93|Easy days may not play useful, white purposes. Healthy, whole words shall take however civil, sim|quarterly| +4414|AAAAAAAAODBBAAAA|2451727|2451817|DEPARTMENT|41|94|Very italian sessions can know thereafter nice leading ways. Tiny, charming bones must pre|quarterly| +4415|AAAAAAAAPDBBAAAA|2451727|2451817|DEPARTMENT|41|95|Statutory, naked children must conflict strictly on a det|quarterly| +4416|AAAAAAAAAEBBAAAA|2451727|2451817|DEPARTMENT|41|96|Possible, private officers declare inherently; single, political years score i|quarterly| +4417|AAAAAAAABEBBAAAA|2451727|2451817|DEPARTMENT|41|97|Confidential possibilities hear apparently large, occupational models; quite appropriate wor|quarterly| +4418|AAAAAAAACEBBAAAA|2451727|2451817|DEPARTMENT|41|98|Informally new floors can shake newly times. Suitable, exceptional qu|quarterly| +4419|AAAAAAAADEBBAAAA|2451727|2451817|DEPARTMENT|41|99|Right international abilities select white, alternative telecommuni|quarterly| +4420|AAAAAAAAEEBBAAAA|2451727|2451817|DEPARTMENT|41|100|Parliamentary awards could like senior, necessary streets. Usually peaceful aim|quarterly| +4421|AAAAAAAAFEBBAAAA|2451727|2451817|DEPARTMENT|41|101|Real items would know through a mountains; mountains leave evident|quarterly| +4422|AAAAAAAAGEBBAAAA|2451727|2451817|DEPARTMENT|41|102|Males will seduce. Future, delicious trees understand te|quarterly| +4423|AAAAAAAAHEBBAAAA|2451727|2451817|DEPARTMENT|41|103|Of course previous ministers should let relatively relationships. Cheerful thanks may n|quarterly| +4424|AAAAAAAAIEBBAAAA|2451727|2451817|DEPARTMENT|41|104|Rows agree partially simple millions. Chinese particles find. Diplomatic, s|quarterly| +4425|AAAAAAAAJEBBAAAA||2451817|DEPARTMENT||105||| +4426|AAAAAAAAKEBBAAAA|2451727|2451817|DEPARTMENT|41|106|Friends can stand at all on a men. Difficult days will survive long complete users.|quarterly| +4427|AAAAAAAALEBBAAAA|2451727|2451817|DEPARTMENT|41|107|Sorry measures will remember again beautiful pounds. Judicial factors take plates. Stup|quarterly| +4428|AAAAAAAAMEBBAAAA|2451727|2451817|DEPARTMENT|41|108|Of course star eyes keep on a practices. Inner problems must concentrate social, young|quarterly| +4429|AAAAAAAANEBBAAAA|2451818|2451908|DEPARTMENT|42|1|Significant children used to get. For example local lives |quarterly| +4430|AAAAAAAAOEBBAAAA|2451818|2451908|DEPARTMENT|42|2|Good coins shall go appropriately. Mostly top time|quarterly| +4431|AAAAAAAAPEBBAAAA|2451818|2451908|DEPARTMENT|42|3|Results would stand then open colours. Then personal chips will forget open low video-tape|quarterly| +4432|AAAAAAAAAFBBAAAA|2451818|2451908|DEPARTMENT|42|4|Views appreciate sure. Strong, scientific sales ought to go european activitie|quarterly| +4433|AAAAAAAABFBBAAAA|2451818|2451908|DEPARTMENT|42|5|Traditionally early facts can tell for a readers; well wild grounds should not help |quarterly| +4434|AAAAAAAACFBBAAAA|2451818|2451908|DEPARTMENT|42|6|Plans would create as inadequate bills. Electoral,|quarterly| +4435|AAAAAAAADFBBAAAA|2451818|2451908|DEPARTMENT|42|7|Splendid, national materials see. Ideas let apparently popular cases. Ministers would dig less d|quarterly| +4436|AAAAAAAAEFBBAAAA|2451818|2451908|DEPARTMENT|42|8|Judicial, future leaders help hands. Probably eastern dreams envisage very|quarterly| +4437|AAAAAAAAFFBBAAAA|2451818|2451908|DEPARTMENT|42|9|Other elements might win in a men. Times analyse yet large standards. Together sur|quarterly| +4438|AAAAAAAAGFBBAAAA|2451818|2451908|DEPARTMENT|42|10|At least practical lives remember internal minutes. Financial eyes should n|quarterly| +4439|AAAAAAAAHFBBAAAA|2451818|2451908|DEPARTMENT|42|11|Rooms keep in the elections. Special studies must not |quarterly| +4440|AAAAAAAAIFBBAAAA|2451818|2451908|DEPARTMENT|42|12|Thorough children shall continue further particular, sacred m|quarterly| +4441|AAAAAAAAJFBBAAAA|2451818|2451908|DEPARTMENT|42|13|Different months ought to take. Rather possible owners must operate strategic opportunitie|quarterly| +4442|AAAAAAAAKFBBAAAA|2451818|2451908|DEPARTMENT|42|14|Available, old ambitions accept poor, young moments. Forces solve united, who|quarterly| +4443|AAAAAAAALFBBAAAA|2451818|2451908|DEPARTMENT|42|15|Twice expected eggs shall provide then glasses. Unlikely, certain troops know. E|quarterly| +4444|AAAAAAAAMFBBAAAA|2451818|2451908|DEPARTMENT|42|16|Still national figures can need during a leaders. Affairs see for the ye|quarterly| +4445|AAAAAAAANFBBAAAA|2451818|2451908|DEPARTMENT|42|17|Little items increase in a prices. Purposes could like meeti|quarterly| +4446|AAAAAAAAOFBBAAAA|2451818|2451908|DEPARTMENT|42|18|English rules make otherwise. Even left eyes own severe inch|quarterly| +4447|AAAAAAAAPFBBAAAA|2451818|2451908|DEPARTMENT|42|19|Inc, subsequent children let so scottish, social e|quarterly| +4448|AAAAAAAAAGBBAAAA|2451818|2451908|DEPARTMENT|42|20|Nuclear, relevant letters know much old cases. Feet cannot offer si|quarterly| +4449|AAAAAAAABGBBAAAA|2451818|2451908|DEPARTMENT|42|21|Shops must not present goods. Also competitive values make respectively spec|quarterly| +4450|AAAAAAAACGBBAAAA|2451818|2451908|DEPARTMENT|42|22|Sweet operations might help; fine, major stories used to establish young |quarterly| +4451|AAAAAAAADGBBAAAA|2451818|2451908|DEPARTMENT|42|23|Direct, high things mean. Severe drugs will see newly rewards; toda|quarterly| +4452|AAAAAAAAEGBBAAAA|2451818|2451908|DEPARTMENT|42|24|Barriers must concern through a centres; very serious problems should get for a windows. Indust|quarterly| +4453|AAAAAAAAFGBBAAAA|2451818|2451908|DEPARTMENT|42|25|Separate, independent yards will differ much; agencies sing across the animals. |quarterly| +4454|AAAAAAAAGGBBAAAA|2451818|2451908|DEPARTMENT|42|26|Long, dark goods need thus senior kinds. Direct, contemporary forms turn. Prospects go r|quarterly| +4455|AAAAAAAAHGBBAAAA|2451818|2451908|DEPARTMENT|42|27|Fresh, round affairs play certainly bright schools; relatively wrong stores take|quarterly| +4456|AAAAAAAAIGBBAAAA|2451818|2451908|DEPARTMENT|42|28|Difficult countries would change dramatic, soviet pati|quarterly| +4457|AAAAAAAAJGBBAAAA|2451818|2451908|DEPARTMENT|42|29|Still novel families alter also needs; sexual, sole grounds must not come re|quarterly| +4458|AAAAAAAAKGBBAAAA|2451818|2451908|DEPARTMENT|42|30|Years could not combat buildings; projects must go now statistical |quarterly| +4459|AAAAAAAALGBBAAAA|2451818|2451908|DEPARTMENT|42|31|Small humans can respond fires. Suitable, scottish groups would cultivate once tea|quarterly| +4460|AAAAAAAAMGBBAAAA|2451818|2451908|DEPARTMENT|42|32|Applications return also silver, subsequent others; steep students come partic|quarterly| +4461|AAAAAAAANGBBAAAA|2451818|2451908|DEPARTMENT|42|33|Rare, innocent courts shall give much resulting, america|quarterly| +4462|AAAAAAAAOGBBAAAA|2451818|2451908|DEPARTMENT|42|34|Roman heads answer later distinct years. Still great parents ta|quarterly| +4463|AAAAAAAAPGBBAAAA|2451818|2451908|DEPARTMENT|42|35|Schemes run large, contemporary circumstances. Wrong real cities address produ|quarterly| +4464|AAAAAAAAAHBBAAAA|2451818|2451908|DEPARTMENT|42|36|Similar, social candidates forget private officers. Definite|quarterly| +4465|AAAAAAAABHBBAAAA|2451818|2451908|DEPARTMENT|42|37|Mines could not use women; useful, main persons lo|quarterly| +4466|AAAAAAAACHBBAAAA|2451818|2451908|DEPARTMENT|42|38|Leading situations may avoid still. Tests pay late, n|quarterly| +4467|AAAAAAAADHBBAAAA|2451818|2451908|DEPARTMENT|42|39|Social workers will live so certain clubs. Pp. might establish gratefully light |quarterly| +4468|AAAAAAAAEHBBAAAA|2451818|2451908|DEPARTMENT|42|40|Hundreds concentrate therefore living pieces. Years sho|quarterly| +4469|AAAAAAAAFHBBAAAA|2451818|2451908|DEPARTMENT|42|41|Vital terms continue possible members. Only normal|quarterly| +4470|AAAAAAAAGHBBAAAA|2451818|2451908|DEPARTMENT|42|42|Effects swing just similar values; now round ships|quarterly| +4471|AAAAAAAAHHBBAAAA|2451818|2451908|DEPARTMENT|42|43|Emissions require between a examples. Inadequate, long accounts know|quarterly| +4472|AAAAAAAAIHBBAAAA|2451818|2451908|DEPARTMENT|42|44|Just fine friends ought to cross quite groups. Right, sign|quarterly| +4473|AAAAAAAAJHBBAAAA|2451818|2451908|DEPARTMENT|42|45|Responsible, identical employers safeguard actually girls; appropriate, initial deg|quarterly| +4474|AAAAAAAAKHBBAAAA|2451818|2451908|DEPARTMENT|42|46|Men must go still new records. Sections might help exactly. Calls might sell still|quarterly| +4475|AAAAAAAALHBBAAAA|2451818|2451908|DEPARTMENT|42|47|Things would refine units; special, private men take concer|quarterly| +4476|AAAAAAAAMHBBAAAA|2451818|2451908|DEPARTMENT|42|48|Long, important pages should pick bare, local points; direct sales may go thu|quarterly| +4477|AAAAAAAANHBBAAAA|2451818|2451908|DEPARTMENT|42|49|Wooden, temporary trustees mean all the same from a cups. Other organisations might wear soon. Legs|quarterly| +4478|AAAAAAAAOHBBAAAA|2451818|2451908|DEPARTMENT|42|50|Officials might suggest mere twins. Private, independent populations work new, |quarterly| +4479|AAAAAAAAPHBBAAAA|2451818|2451908|DEPARTMENT|42|51|Forests could happen to a tests. Departments must breed thick about the women. V|quarterly| +4480|AAAAAAAAAIBBAAAA|2451818|2451908|DEPARTMENT|42|52|Individuals stop widely hands. Funny writings pick conseque|quarterly| +4481|AAAAAAAABIBBAAAA|2451818|2451908|DEPARTMENT|42|53|Magnificent facilities win all. Later front bands shall w|quarterly| +4482|AAAAAAAACIBBAAAA|2451818|2451908|DEPARTMENT|42|54|Rules correspond besides black hours; so central payments combine never. Reductions acqui|quarterly| +4483|AAAAAAAADIBBAAAA|2451818|2451908|DEPARTMENT|42|55|Different, annual men make under the policies. In order single|quarterly| +4484|AAAAAAAAEIBBAAAA|2451818|2451908|DEPARTMENT|42|56|Facilities know changes. Now difficult changes go |quarterly| +4485|AAAAAAAAFIBBAAAA|2451818|2451908|DEPARTMENT|42|57|Culturally british gifts see only ancient, long-term classes. Particular talks twist. Locations go|quarterly| +4486|AAAAAAAAGIBBAAAA|2451818|2451908|DEPARTMENT|42|58|Academic, previous effects can retaliate later significant approaches|quarterly| +4487|AAAAAAAAHIBBAAAA|2451818|2451908|DEPARTMENT|42|59|Patients could record for a bills. Children will not know already lives.|quarterly| +4488|AAAAAAAAIIBBAAAA||2451908|DEPARTMENT|42|60|Other concentrations wait never. Enough final concentrations raise low minutes. Contents shall not |quarterly| +4489|AAAAAAAAJIBBAAAA|2451818|2451908|DEPARTMENT|42|61|Facts reach public, eastern products; good, able trains see only like the audiences. Pract|quarterly| +4490|AAAAAAAAKIBBAAAA|2451818|2451908|DEPARTMENT|42|62|Cells make most healthy, other contracts. Quickly other |quarterly| +4491|AAAAAAAALIBBAAAA|2451818|2451908|DEPARTMENT|42|63|Companies would not tempt in a houses; increasing horses halt apparently at an tensions. Particular|quarterly| +4492|AAAAAAAAMIBBAAAA|2451818|2451908|DEPARTMENT|42|64|Difficult babies expect simply under an sides. Otherwise chief messages love new r|quarterly| +4493|AAAAAAAANIBBAAAA|2451818|2451908|DEPARTMENT|42|65|Doubtless long flowers check even colleagues. Groups used to make|quarterly| +4494|AAAAAAAAOIBBAAAA|2451818|2451908|DEPARTMENT|42|66|Educational criteria move very international groups. Crimina|quarterly| +4495|AAAAAAAAPIBBAAAA|2451818|2451908|DEPARTMENT|42|67|Good, existing studies enter again japanese, private criteria. Social propos|quarterly| +4496|AAAAAAAAAJBBAAAA|2451818|2451908|DEPARTMENT|42|68|Abroad early things would go useful, following days. Waiting, tot|quarterly| +4497|AAAAAAAABJBBAAAA|2451818|2451908|DEPARTMENT|42|69|Negative, limited shares would feel at least. Attractive years ought to |quarterly| +4498|AAAAAAAACJBBAAAA|2451818|2451908|DEPARTMENT|42|70|Alternative, local objectives must come on an practices. Public, powerful men used to|quarterly| +4499|AAAAAAAADJBBAAAA|2451818|2451908|DEPARTMENT|42|71|Today internal opportunities extend else local months.|quarterly| +4500|AAAAAAAAEJBBAAAA|2451818|2451908|DEPARTMENT|42|72|Political, old trends should not accept less sweet rooms. Ha|quarterly| +4501|AAAAAAAAFJBBAAAA|2451818|2451908|DEPARTMENT|42|73|Leaders protect only single women. French arms impose|quarterly| +4502|AAAAAAAAGJBBAAAA|2451818|2451908|DEPARTMENT|42|74|Police regain able passengers. International jobs cry even always other righ|quarterly| +4503|AAAAAAAAHJBBAAAA|2451818|2451908|DEPARTMENT|42|75|Regional, vast terms respond reports. Individual, acute plants used to offer far ov|quarterly| +4504|AAAAAAAAIJBBAAAA|2451818|2451908|DEPARTMENT|42|76|Quite different wings explain tomorrow never new forms. Hotels|quarterly| +4505|AAAAAAAAJJBBAAAA|2451818|2451908|DEPARTMENT|42|77|So significant students make now. Serious, final d|quarterly| +4506|AAAAAAAAKJBBAAAA|2451818|2451908|DEPARTMENT|42|78|Thereby additional groups become short periods. Geographical membe|quarterly| +4507|AAAAAAAALJBBAAAA|2451818|2451908|DEPARTMENT|42|79|Obvious, average hearts distinguish traditionally national, english po|quarterly| +4508|AAAAAAAAMJBBAAAA|2451818|2451908|DEPARTMENT|42|80|Combined, religious sentences should refer. Teachers get here things. Physical, sign|quarterly| +4509|AAAAAAAANJBBAAAA|2451818|2451908|DEPARTMENT|42|81|Systems think in a years. Students possess perhaps. Extended offices get children. Police squ|quarterly| +4510|AAAAAAAAOJBBAAAA|2451818|2451908|DEPARTMENT|42|82|Parties carry effectively new, other leaders. Only matches s|quarterly| +4511|AAAAAAAAPJBBAAAA|2451818|2451908|DEPARTMENT|42|83|Colleges take. Key, protective years get anyway then clear refugees. Likely, useful purposes used t|quarterly| +4512|AAAAAAAAAKBBAAAA|2451818|2451908|DEPARTMENT|42|84|Activities would focus so for the senses. Police see quite; limits take things. Users may |quarterly| +4513|AAAAAAAABKBBAAAA|2451818|2451908|DEPARTMENT|42|85|Problems might not adopt companies. Prices might not discuss just totally rare terms. T|quarterly| +4514|AAAAAAAACKBBAAAA|2451818|2451908|DEPARTMENT|42|86|Words require high inner workers. Certain agreements meet all only, everyday systems. Here or|quarterly| +4515|AAAAAAAADKBBAAAA|2451818|2451908|DEPARTMENT|42|87|Songs should ban minor, complex computers. Fully large counties w|quarterly| +4516|AAAAAAAAEKBBAAAA|2451818|2451908|DEPARTMENT|42|88|Slightly major professionals might prove different homes; bad, pale fu|quarterly| +4517|AAAAAAAAFKBBAAAA|2451818|2451908|DEPARTMENT|42|89|Brilliant periods can start too existing months. Impo|quarterly| +4518|AAAAAAAAGKBBAAAA|||DEPARTMENT|42|90||| +4519|AAAAAAAAHKBBAAAA|2451818|2451908|DEPARTMENT|42|91|Farmers attempt high, worried resources. Simply social|quarterly| +4520|AAAAAAAAIKBBAAAA|2451818|2451908|DEPARTMENT|42|92|Presumably upper levels need other, wild voices. Public suggestions shall win frequently limit|quarterly| +4521|AAAAAAAAJKBBAAAA|2451818|2451908|DEPARTMENT|42|93|Always bright miles accord immediately considerable, various value|quarterly| +4522|AAAAAAAAKKBBAAAA|2451818|2451908|DEPARTMENT|42|94|Romantic, poor products would render even hardly great branche|quarterly| +4523|AAAAAAAALKBBAAAA|2451818|2451908|DEPARTMENT|42|95|Novel years increase never good teachers. Equal stones complete as natural ho|quarterly| +4524|AAAAAAAAMKBBAAAA|2451818|2451908|DEPARTMENT|42|96|Moreover british pieces shall not relinquish together eyes. Plain h|quarterly| +4525|AAAAAAAANKBBAAAA|2451818|2451908|DEPARTMENT|42|97|Other practitioners can give thankfully men. Already|quarterly| +4526|AAAAAAAAOKBBAAAA|2451818|2451908|DEPARTMENT|42|98|Spiritual, necessary efforts pay correctly sure, other books. Barely unable times should answer p|quarterly| +4527|AAAAAAAAPKBBAAAA|2451818|2451908|DEPARTMENT|42|99|Scottish groups will not say nearly marked classes. Clearly potential shares s|quarterly| +4528|AAAAAAAAALBBAAAA|2451818|2451908|DEPARTMENT|42|100|Different, commercial issues can make then children. Strange activiti|quarterly| +4529|AAAAAAAABLBBAAAA|2451818|2451908|DEPARTMENT|42|101|Glasses will discuss clearly new jobs. Places take very national services. Much new members |quarterly| +4530|AAAAAAAACLBBAAAA|2451818|2451908|DEPARTMENT|42|102|Other organisms must analyse within the bonds; other, local conside|quarterly| +4531|AAAAAAAADLBBAAAA|2451818|2451908|DEPARTMENT|42|103|Fundamental, sophisticated students work totally readers. Necessarily furious musicians ca|quarterly| +4532|AAAAAAAAELBBAAAA|2451818|2451908|DEPARTMENT|42|104|Irrelevant, commercial bars start very new mountains. Conventiona|quarterly| +4533|AAAAAAAAFLBBAAAA|2451818|2451908|DEPARTMENT|42|105|Both continued periods could help spanish, similar generations; popular, scientific diffi|quarterly| +4534|AAAAAAAAGLBBAAAA|2451818|2451908|DEPARTMENT|42|106|Black, horizontal partners shall name at the exper|quarterly| +4535|AAAAAAAAHLBBAAAA|2451818|2451908|DEPARTMENT|42|107|Real arrangements provide however and so on specific volunteers. Children think too d|quarterly| +4536|AAAAAAAAILBBAAAA|2451818|2451908|DEPARTMENT|42|108|Here straightforward branches adjust brilliantly generally elect|quarterly| +4537|AAAAAAAAJLBBAAAA|2451545|2451574|DEPARTMENT|43|1|Commercial, alive drawings ought to like definite ears. More multiple mo|monthly| +4538|AAAAAAAAKLBBAAAA|2451545|2451574|DEPARTMENT|43|2|Considerable minutes shall start much strong techniques. Films could come living auditors. Then oth|monthly| +4539|AAAAAAAALLBBAAAA|2451545|2451574|DEPARTMENT|43|3|No longer new comments walk always aside large ideas. High, remarka|monthly| +4540|AAAAAAAAMLBBAAAA|2451545|2451574|DEPARTMENT|43|4|Combined, identical colleagues might make present tempe|monthly| +4541|AAAAAAAANLBBAAAA|2451545|2451574|DEPARTMENT|43|5|Unlikely abilities may make properly short tickets; repeatedly wooden countries may realise e|monthly| +4542|AAAAAAAAOLBBAAAA|2451545|2451574|DEPARTMENT|43|6|Personal, appropriate officers should not panic here capitalist fortunes. Long socia|monthly| +4543|AAAAAAAAPLBBAAAA|2451545|2451574|DEPARTMENT|43|7|Home sophisticated days say locally; minutes may not save often roman lengths; tonn|monthly| +4544|AAAAAAAAAMBBAAAA|2451545|2451574|DEPARTMENT|43|8|Good, white rights could accommodate parents. Social, geog|monthly| +4545|AAAAAAAABMBBAAAA|2451545|2451574|DEPARTMENT|43|9|Awkward, short-term deaths used to correct still miles; worlds might pray sometime|monthly| +4546|AAAAAAAACMBBAAAA|2451545|2451574|DEPARTMENT|43|10|Continental proposals want with a years. Numbers could not kill over a pa|monthly| +4547|AAAAAAAADMBBAAAA|2451545|2451574|DEPARTMENT|43|11|Christians would run more; simple, able periods speak constant, o|monthly| +4548|AAAAAAAAEMBBAAAA|2451545|2451574|DEPARTMENT|43|12|Already hot children would not continue highly in a sales. Even available hours|monthly| +4549|AAAAAAAAFMBBAAAA|2451545|2451574|DEPARTMENT|43|13|Wrong days cannot weigh solutions. Agricultural names constitute equally on|monthly| +4550|AAAAAAAAGMBBAAAA|2451545|2451574|DEPARTMENT|43|14|Months get words. Later able sites follow clearly totally mere areas. Smart, nati|monthly| +4551|AAAAAAAAHMBBAAAA|2451545|2451574|DEPARTMENT|43|15|Black books give. Women might not experience bases. Necessarily comfortable bills s|monthly| +4552|AAAAAAAAIMBBAAAA|2451545|2451574|DEPARTMENT|43|16|No longer soft friends may modify via a projects; open,|monthly| +4553|AAAAAAAAJMBBAAAA|2451545|2451574|DEPARTMENT|43|17|Open possible months think very only, long jews. Long, comprehensive|monthly| +4554|AAAAAAAAKMBBAAAA|2451545|2451574|DEPARTMENT|43|18|High industrial terms suppose still major traders. Differences try also a|monthly| +4555|AAAAAAAALMBBAAAA|2451545|2451574|DEPARTMENT|43|19|Ever social operations shall grow better windows. Political, small lectures may come invariably d|monthly| +4556|AAAAAAAAMMBBAAAA|2451545|2451574|DEPARTMENT|43|20|Crucial conditions take; difficult, perfect stages should not take much light servi|monthly| +4557|AAAAAAAANMBBAAAA|2451545|2451574|DEPARTMENT|43|21|Social, light bodies lower among a others. White ci|monthly| +4558|AAAAAAAAOMBBAAAA|2451545|2451574|DEPARTMENT|43|22|Too wonderful types will not suppose most quickly low points. Emotional, miserable fe|monthly| +4559|AAAAAAAAPMBBAAAA|2451545|2451574|DEPARTMENT|43|23|Ways deny good ideas; days talk white students. Other, new moments wil|monthly| +4560|AAAAAAAAANBBAAAA|2451545|2451574|DEPARTMENT|43|24|Yet rich issues shall discuss years; schools carry. Coming, dead communications use|monthly| +4561|AAAAAAAABNBBAAAA|2451545|2451574|DEPARTMENT|43|25|Youngsters should get very. Bad, necessary years must pick telecommunications. Co|monthly| +4562|AAAAAAAACNBBAAAA|2451545|2451574|DEPARTMENT|43|26|Even automatic daughters make equally by the ways. Rights may not read flat|monthly| +4563|AAAAAAAADNBBAAAA|2451545|2451574|DEPARTMENT|43|27|Chemicals accompany beyond a organisations. Nations might not in|monthly| +4564|AAAAAAAAENBBAAAA|2451545|2451574|DEPARTMENT|43|28|Soviet, true services should wait well high, difficult problems; national, bitter structu|monthly| +4565|AAAAAAAAFNBBAAAA|2451545|2451574|DEPARTMENT|43|29|Moves could not see at first regulations. Social object|monthly| +4566|AAAAAAAAGNBBAAAA|2451545|2451574|DEPARTMENT|43|30|Great articles believe even. Hours will cut finally cultural, certain molecules. Months cou|monthly| +4567|AAAAAAAAHNBBAAAA|2451545|2451574|DEPARTMENT|43|31|Trying others may not work suddenly so partial women. High local attacks used to look |monthly| +4568|AAAAAAAAINBBAAAA|2451545|2451574|DEPARTMENT|43|32|Sorry authorities afford aspects. Enough dead supporters harness young obligations. Nec|monthly| +4569|AAAAAAAAJNBBAAAA|2451545|2451574|DEPARTMENT|43|33|Descriptions mark as farmers. Above southern gifts |monthly| +4570|AAAAAAAAKNBBAAAA|2451545|2451574|DEPARTMENT|43|34|Supporters release roughly. Atomic officers speak very virt|monthly| +4571|AAAAAAAALNBBAAAA|2451545|2451574|DEPARTMENT|43|35|Then permanent statements should not make legal, new cells. Counties would run sizes; as pri|monthly| +4572|AAAAAAAAMNBBAAAA|2451545|2451574|DEPARTMENT|43|36|Therefore true germans may pull new, successful flights. Doors encourage far fo|monthly| +4573|AAAAAAAANNBBAAAA|2451545|2451574|DEPARTMENT|43|37|Prime days fall proceedings; rates would not move closely t|monthly| +4574|AAAAAAAAONBBAAAA|2451545|2451574|DEPARTMENT|43|38|Relevant, necessary hours leave usually. Industries get mainly only events. M|monthly| +4575|AAAAAAAAPNBBAAAA|2451545|2451574|DEPARTMENT|43|39|Quite dramatic words sign. Common owners bar; then whit|monthly| +4576|AAAAAAAAAOBBAAAA|2451545|2451574|DEPARTMENT|43|40|Specific seats get increasingly good, original tiles. Vulnerable, political students a|monthly| +4577|AAAAAAAABOBBAAAA|2451545|2451574|DEPARTMENT|43|41|Exports will express ever in a organizations. Less similar cars wear economic, hard birds. So s|monthly| +4578|AAAAAAAACOBBAAAA|2451545|2451574|DEPARTMENT|43|42|All regional details count just. Quickly financial persons would|monthly| +4579|AAAAAAAADOBBAAAA|2451545|2451574|DEPARTMENT|43|43|Due, human police used to think there civil, evolutionary groups. Historical, old communica|monthly| +4580|AAAAAAAAEOBBAAAA|2451545|2451574|DEPARTMENT|43|44|Again familiar times reflect golden, public words. Only busy students can say instead single |monthly| +4581|AAAAAAAAFOBBAAAA|2451545|2451574|DEPARTMENT|43|45|Boards mean highly humans. Profits occur however on a notes. Sessions should no|monthly| +4582|AAAAAAAAGOBBAAAA|2451545|2451574|DEPARTMENT|43|46|Almost bad wishes compete hardly natural offences. Successful, possible rates look|monthly| +4583|AAAAAAAAHOBBAAAA|2451545|2451574|DEPARTMENT|43|47|Gradually principal drugs ignore only influences. Tests ought to attract ex|monthly| +4584|AAAAAAAAIOBBAAAA|2451545|2451574|DEPARTMENT|43|48|Large, clear proceedings used to comfort about with the numbers. Efficient levels perform anywhere |monthly| +4585|AAAAAAAAJOBBAAAA|2451545|2451574|DEPARTMENT|43|49|Enough human subjects must not ensure taxes; certainly due agen|monthly| +4586|AAAAAAAAKOBBAAAA|2451545|2451574|DEPARTMENT|43|50|Ships seem historical buses. Yesterday geographical items could deserve perhaps. |monthly| +4587|AAAAAAAALOBBAAAA|2451545|2451574|DEPARTMENT|43|51|Only numerous products cut resolutely live, happy weeks. However magnetic |monthly| +4588|AAAAAAAAMOBBAAAA|2451545|||43|52||monthly| +4589|AAAAAAAANOBBAAAA|2451545|2451574|DEPARTMENT|43|53|Yet little sequences set about. Statistical posts will hide with a poli|monthly| +4590|AAAAAAAAOOBBAAAA|2451545|2451574|DEPARTMENT|43|54|More legal needs will get once habits. Positions should lea|monthly| +4591|AAAAAAAAPOBBAAAA|2451545|2451574|DEPARTMENT|43|55|Actual boxes form at a services; usual documents target however nur|monthly| +4592|AAAAAAAAAPBBAAAA|2451545|2451574|DEPARTMENT|43|56|Once again definite potatoes should lead flats; as small rounds |monthly| +4593|AAAAAAAABPBBAAAA|2451545|2451574|DEPARTMENT|43|57|Right, thin children believe there. Necessary, ethical fingers close slowly il|monthly| +4594|AAAAAAAACPBBAAAA|2451545|2451574|DEPARTMENT|43|58|Abroad large tests may overthrow both origins; typical, small men must mind wrong, t|monthly| +4595|AAAAAAAADPBBAAAA|2451545|2451574|DEPARTMENT|43|59|Big words might not seem both perhaps early actors. Now other re|monthly| +4596|AAAAAAAAEPBBAAAA|2451545|2451574|DEPARTMENT|43|60|Fashionable workers must not support so years. Businesses should not|monthly| +4597|AAAAAAAAFPBBAAAA|2451545|2451574|DEPARTMENT|43|61|Details would survive top experiments. Suddenly specialist birds shall calculat|monthly| +4598|AAAAAAAAGPBBAAAA|2451545|2451574|DEPARTMENT|43|62|Comfortable movements leave subsequently in a moments. |monthly| +4599|AAAAAAAAHPBBAAAA|2451545|2451574|DEPARTMENT|43|63|Now dual names must fill just; lexical principles ought to recognise inc looks. Free p|monthly| +4600|AAAAAAAAIPBBAAAA|2451545|2451574|DEPARTMENT|43|64|Developments mature. Never elderly terms will move gracefully|monthly| +4601|AAAAAAAAJPBBAAAA|2451545|2451574|DEPARTMENT|43|65|Contemporary, left changes see away on a powers. Activities used to benefit ever. Absen|monthly| +4602|AAAAAAAAKPBBAAAA|2451545|2451574|DEPARTMENT|43|66|Sorry, spiritual benefits used to read special, famous goods. Roy|monthly| +4603|AAAAAAAALPBBAAAA|2451545|2451574|DEPARTMENT|43|67|Then front hours define never tracks; vital males introduce ne|monthly| +4604|AAAAAAAAMPBBAAAA|2451545|2451574|DEPARTMENT|43|68|Necessary children keep fingers. Perfect officers might not remove here odd losses. Also hard right|monthly| +4605|AAAAAAAANPBBAAAA|2451545|2451574|DEPARTMENT|43|69|Facts ought to open more traditionally central degrees. About coastal others go. Thus current sp|monthly| +4606|AAAAAAAAOPBBAAAA|2451545|2451574|DEPARTMENT|43|70|Wonderful losses see others; important, gastric payments |monthly| +4607|AAAAAAAAPPBBAAAA|2451545|2451574|DEPARTMENT|43|71|Proceedings shall take so ultimate bacteria. Involved, regulatory matters used |monthly| +4608|AAAAAAAAAACBAAAA|2451545|2451574|DEPARTMENT|43|72|Determined factories define double jews. Sure permanent clubs could not go systems. General|monthly| +4609|AAAAAAAABACBAAAA|2451545|2451574|DEPARTMENT|43|73|Dealers find once dependent positions. Futures win just initial benefits|monthly| +4610|AAAAAAAACACBAAAA|2451545|2451574|DEPARTMENT|43|74|Somewhere free seeds would need better dark environmental objects. Free walls |monthly| +4611|AAAAAAAADACBAAAA|2451545|2451574|DEPARTMENT|43|75|Particular, partial prisoners connect properly as available resources.|monthly| +4612|AAAAAAAAEACBAAAA|2451545|2451574|DEPARTMENT|43|76|Often central children may invite internationally as sensitive friends. Changes hire things.|monthly| +4613|AAAAAAAAFACBAAAA|2451545|2451574|DEPARTMENT|43|77|Only, political mistakes would continue for a things. Major plates shall not increase hardly beca|monthly| +4614|AAAAAAAAGACBAAAA|2451545|2451574|DEPARTMENT|43|78|Ever proper systems get however democrats. Expectations shoul|monthly| +4615|AAAAAAAAHACBAAAA|2451545|2451574|DEPARTMENT|43|79|Away national leaves will hurt prime cameras. Figures m|monthly| +4616|AAAAAAAAIACBAAAA|2451545|2451574|DEPARTMENT|43|80|Academic, right controls come as economic limitations; good territories cou|monthly| +4617|AAAAAAAAJACBAAAA|2451545|2451574|DEPARTMENT|43|81|Limitations forget years. Other lines ensure notes. Actions buy s|monthly| +4618|AAAAAAAAKACBAAAA|2451545|2451574|DEPARTMENT|43|82|Securities ought to shift more often broken incentives. Po|monthly| +4619|AAAAAAAALACBAAAA|2451545|2451574|DEPARTMENT|43|83|Eyes establish following men. Numbers give later black pockets. Events suit at least po|monthly| +4620|AAAAAAAAMACBAAAA|2451545|2451574|DEPARTMENT|43|84|Apparently long points would not come frequently expensive, o|monthly| +4621|AAAAAAAANACBAAAA|2451545|2451574|DEPARTMENT|43|85|Also responsible relations should agree finally however flat tax|monthly| +4622|AAAAAAAAOACBAAAA|2451545|2451574|DEPARTMENT|43|86|Even social affairs function at all sometimes inevi|monthly| +4623|AAAAAAAAPACBAAAA|2451545|2451574|DEPARTMENT|43|87|Deeply comprehensive marks cannot treat overseas wo|monthly| +4624|AAAAAAAAABCBAAAA|2451545|2451574|DEPARTMENT|43|88|Authorities could continue prisoners. Unfortunately civil lengths|monthly| +4625|AAAAAAAABBCBAAAA|2451545|2451574|DEPARTMENT|43|89|Silent instruments make complete, alleged settings. Catholic years could serve|monthly| +4626|AAAAAAAACBCBAAAA|2451545|2451574|DEPARTMENT|43|90|Following authorities go at least specific relations. Candidat|monthly| +4627|AAAAAAAADBCBAAAA|2451545|2451574|DEPARTMENT|43|91|Properly interested factors shall ban well papers. Workers throw so econom|monthly| +4628|AAAAAAAAEBCBAAAA|2451545|2451574|DEPARTMENT|43|92|French, unchanged products believe there skilled lights. Personal,|monthly| +4629|AAAAAAAAFBCBAAAA|2451545|2451574|DEPARTMENT|43|93|Forward public obligations used to assess now again dange|monthly| +4630|AAAAAAAAGBCBAAAA|2451545|2451574|DEPARTMENT|43|94|Common, continuing rights monitor easily. Once great parties could start other |monthly| +4631|AAAAAAAAHBCBAAAA|2451545|2451574|DEPARTMENT|43|95|Still overseas studies can assist very by a depths. Low problems s|monthly| +4632|AAAAAAAAIBCBAAAA|2451545|2451574|DEPARTMENT|43|96|Sides rebuild also individual partners. In particular ad|monthly| +4633|AAAAAAAAJBCBAAAA|2451545|2451574|DEPARTMENT|43|97|Olympic, new parties may love hands. Girls make usually irish points. British, bold years|monthly| +4634|AAAAAAAAKBCBAAAA|2451545|2451574|DEPARTMENT|43|98|Scottish, new bars drink then warm lines. Most double groups should go ago there tot|monthly| +4635|AAAAAAAALBCBAAAA|2451545|2451574|DEPARTMENT|43|99|Weak, expensive senses help. For example military hours take certainly as a bits. Togeth|monthly| +4636|AAAAAAAAMBCBAAAA|2451545|2451574|DEPARTMENT|43|100|Markets back dear, initial hundreds. Indeed sensible conditions stop parents. Su|monthly| +4637|AAAAAAAANBCBAAAA|2451545|2451574|DEPARTMENT|43|101|Immediate, economic elements say efficiently police; presidential, new councils should|monthly| +4638|AAAAAAAAOBCBAAAA|2451545|2451574|DEPARTMENT|43|102|Streets may find then very complete children. Great, social books may increase other than |monthly| +4639|AAAAAAAAPBCBAAAA|2451545|2451574|DEPARTMENT|43|103|Normal, certain controls talk. Academic, soviet resour|monthly| +4640|AAAAAAAAACCBAAAA|2451545|2451574|DEPARTMENT|43|104|Possible books attempt however nice families. Underlyi|monthly| +4641|AAAAAAAABCCBAAAA|2451545|2451574|DEPARTMENT|43|105|Processes will know later major, national patients; everyday walls appo|monthly| +4642|AAAAAAAACCCBAAAA|2451545|2451574|DEPARTMENT|43|106|Right lines might play somehow here rigid forests. Individuals might not hide happily ethnic,|monthly| +4643|AAAAAAAADCCBAAAA|2451545|2451574|DEPARTMENT|43|107|Cheap products might collapse far later fortunate ministers; heavy scenes should achieve t|monthly| +4644|AAAAAAAAECCBAAAA|2451545|2451574|DEPARTMENT|43|108|Again integrated controls shall explain social interests. Different clients may |monthly| +4645|AAAAAAAAFCCBAAAA|2451575|2451604|DEPARTMENT|44|1|Bodies must suppose again normally active standards. Aw|monthly| +4646|AAAAAAAAGCCBAAAA|2451575|2451604|DEPARTMENT|44|2|Given, possible funds sit then easy goals. Fine, shallow faces could not feel yea|monthly| +4647|AAAAAAAAHCCBAAAA|2451575|2451604|DEPARTMENT|44|3|Domestic, sharp courts shall give at present in a affairs. Political, |monthly| +4648|AAAAAAAAICCBAAAA|2451575|2451604|DEPARTMENT|44|4|Neatly proper stories harm narrowly nuclear words; english customers would avoid then f|monthly| +4649|AAAAAAAAJCCBAAAA|2451575|2451604|DEPARTMENT|44|5|Experts shall drive hard, white producers. Full, go|monthly| +4650|AAAAAAAAKCCBAAAA|2451575|2451604|DEPARTMENT|44|6|Exactly entire collections indicate within a prices. |monthly| +4651|AAAAAAAALCCBAAAA|2451575|2451604|DEPARTMENT|44|7|Economic cards may not form finally crude sources; very el|monthly| +4652|AAAAAAAAMCCBAAAA|2451575|2451604|DEPARTMENT|44|8|Operations should not budge as open, certain affairs. Changes |monthly| +4653|AAAAAAAANCCBAAAA|2451575|2451604|DEPARTMENT|44|9|Companies may not ensure national patients. Females might s|monthly| +4654|AAAAAAAAOCCBAAAA|2451575|2451604|DEPARTMENT|44|10|Too other parts like. Orders would see to a visitors. Inland b|monthly| +4655|AAAAAAAAPCCBAAAA|2451575|2451604|DEPARTMENT|44|11|Bad plants know students. Men locate in a individuals. Chief opportunities must |monthly| +4656|AAAAAAAAADCBAAAA|2451575|2451604|DEPARTMENT|44|12|Possible views handle however at a rounds. Prayers require social wom|monthly| +4657|AAAAAAAABDCBAAAA|2451575|2451604|DEPARTMENT|44|13|Too effective scientists must not discuss too police. Places discuss maybe large policies. Import|monthly| +4658|AAAAAAAACDCBAAAA|2451575|2451604|DEPARTMENT|44|14|Important paintings must go particularly years. Written claims could c|monthly| +4659|AAAAAAAADDCBAAAA|2451575|2451604|DEPARTMENT|44|15|Months cannot work; mainly permanent conventions obtain rather social, good windows. Quickly full|monthly| +4660|AAAAAAAAEDCBAAAA|2451575|2451604|DEPARTMENT|44|16|Young, silver children must not think for certain dominant arrangements. Almost|monthly| +4661|AAAAAAAAFDCBAAAA|2451575|2451604|DEPARTMENT|44|17|Else military consequences used to collect previously languages. Contemporary, responsible boot|monthly| +4662|AAAAAAAAGDCBAAAA|2451575|2451604|DEPARTMENT|44|18|Ever public varieties cannot get other, good words. Authorities swim now count|monthly| +4663|AAAAAAAAHDCBAAAA|2451575|2451604|DEPARTMENT|44|19|Gifts speculate special, final ministers. Parliamentary services mus|monthly| +4664|AAAAAAAAIDCBAAAA|2451575|2451604|DEPARTMENT|44|20|Sure assumptions make friends. Little free details imagine coins. P|monthly| +4665|AAAAAAAAJDCBAAAA|2451575|2451604|DEPARTMENT|44|21|Sorry nurses watch just. Requirements imitate almos|monthly| +4666|AAAAAAAAKDCBAAAA|2451575|2451604|DEPARTMENT|44|22|Right small concerns shall not move intact conventions. Ethnic blocks might encourage t|monthly| +4667|AAAAAAAALDCBAAAA|2451575|2451604|DEPARTMENT|44|23|Old-fashioned, other questions must print. Applicatio|monthly| +4668|AAAAAAAAMDCBAAAA|2451575|2451604|DEPARTMENT|44|24|Nearly senior miners protect generally outside, dominant heads. Correct resu|monthly| +4669|AAAAAAAANDCBAAAA|2451575|2451604|DEPARTMENT|44|25|Heavily necessary sequences pass. Much available bodies may take with a sig|monthly| +4670|AAAAAAAAODCBAAAA|2451575|2451604|DEPARTMENT|44|26|Rather long hands will share knees. Windows must not dispose local, |monthly| +4671|AAAAAAAAPDCBAAAA|2451575|2451604|DEPARTMENT|44|27|All assistant students sound good, polite years; legal, organisational places think therefore |monthly| +4672|AAAAAAAAAECBAAAA|2451575||||28|Other friends go no longer forward seconds. Normal factors like. Co|| +4673|AAAAAAAABECBAAAA|2451575|2451604|DEPARTMENT|44|29|Sudden, individual days discuss now for a agencies|monthly| +4674|AAAAAAAACECBAAAA|2451575|2451604|DEPARTMENT|44|30|Possible children stay probably. Birds make new, dear pla|monthly| +4675|AAAAAAAADECBAAAA|2451575|2451604|DEPARTMENT|44|31|Political others may not assess quietly. Small, simple arts leave fair tough auth|monthly| +4676|AAAAAAAAEECBAAAA|2451575|2451604|DEPARTMENT|44|32|Also available creditors go never political, complete use|monthly| +4677|AAAAAAAAFECBAAAA|2451575|2451604|DEPARTMENT|44|33|Wrong, national decisions would not win only powerful products; strong electio|monthly| +4678|AAAAAAAAGECBAAAA|2451575|2451604|DEPARTMENT|44|34|Hands may avoid also. Important, possible gaps should not s|monthly| +4679|AAAAAAAAHECBAAAA|2451575|2451604|DEPARTMENT|44|35|Then political companies should not fix nevertheless; rooms cannot let nearly fier|monthly| +4680|AAAAAAAAIECBAAAA|2451575|2451604|DEPARTMENT|44|36|Ready, lovely readers stand american, free resources. New, public relations shall ent|monthly| +4681|AAAAAAAAJECBAAAA|2451575|2451604|DEPARTMENT|44|37|Appropriate features make anyway able employees. Available speakers need colonial,|monthly| +4682|AAAAAAAAKECBAAAA|2451575|2451604|DEPARTMENT|44|38|Subsequent plans will not conceal always acts; parents should start |monthly| +4683|AAAAAAAALECBAAAA|2451575|2451604|DEPARTMENT|44|39|More neutral studies might not realise. Losses would not show obviously bright actions. Fe|monthly| +4684|AAAAAAAAMECBAAAA|2451575|2451604|DEPARTMENT|44|40|Hard songs talk very simple, european performances. Very holy relationships exe|monthly| +4685|AAAAAAAANECBAAAA|2451575|2451604|DEPARTMENT|44|41|Common, keen discussions answer so new children. Deep expensive police dete|monthly| +4686|AAAAAAAAOECBAAAA|2451575|2451604|DEPARTMENT|44|42|Levels can believe early; men state again years. Able, new facts mean correctl|monthly| +4687|AAAAAAAAPECBAAAA|2451575|2451604|DEPARTMENT|44|43|Fair members will mean cruelly; far, major firms might help then of course dependent|monthly| +4688|AAAAAAAAAFCBAAAA|2451575|2451604|DEPARTMENT|44|44|Mild applications should evaluate. Chosen individuals manage obvious, english sides. Feelings appea|monthly| +4689|AAAAAAAABFCBAAAA|2451575|2451604|DEPARTMENT|44|45|Other scales might kill all powerful activities; servants should not increase then years;|monthly| +4690|AAAAAAAACFCBAAAA|2451575|2451604|DEPARTMENT|44|46|Practical, international rates eat ahead during a children. Here practical questions w|monthly| +4691|AAAAAAAADFCBAAAA|2451575|2451604|DEPARTMENT|44|47|As adequate minutes visit bright girls. Other, electoral towns might use very |monthly| +4692|AAAAAAAAEFCBAAAA|2451575|2451604|DEPARTMENT|44|48|Gradual, only prospects last under way surely entire minds; once great system|monthly| +4693|AAAAAAAAFFCBAAAA|2451575|2451604|DEPARTMENT|44|49|Easily high miles recur only personal suppliers. Surveys walk j|monthly| +4694|AAAAAAAAGFCBAAAA|2451575|2451604|DEPARTMENT|44|50|Things become inwards close, main structures; nearly suitable months assess. Dul|monthly| +4695|AAAAAAAAHFCBAAAA|2451575|2451604|DEPARTMENT|44|51|Open honours might not take together. Prices will settle words.|monthly| +4696|AAAAAAAAIFCBAAAA|2451575|2451604|DEPARTMENT|44|52|Short materials used to make indian, wrong stones. Royal children might not reduc|monthly| +4697|AAAAAAAAJFCBAAAA|2451575|2451604|DEPARTMENT|44|53|Specific, mean pp. learn more. Methods run as local questions. As special s|monthly| +4698|AAAAAAAAKFCBAAAA|2451575|2451604|DEPARTMENT|44|54|As well automatic standards think well. Policies achieve|monthly| +4699|AAAAAAAALFCBAAAA|2451575|2451604|DEPARTMENT|44|55|So much as legal days introduce much. Above interesting targe|monthly| +4700|AAAAAAAAMFCBAAAA|2451575|2451604|DEPARTMENT|44|56|Now proposed ministers will identify suddenly crucial |monthly| +4701|AAAAAAAANFCBAAAA|2451575|2451604|DEPARTMENT|44|57|Services trust more of course national properties; even different employees end ago sufficiently|monthly| +4702|AAAAAAAAOFCBAAAA|2451575|2451604|DEPARTMENT|44|58|Only hot groups talk turkish personnel; major situations might not publish vast,|monthly| +4703|AAAAAAAAPFCBAAAA|2451575|2451604|DEPARTMENT|44|59|Dangerous goods could make to a reforms. Special wages tak|monthly| +4704|AAAAAAAAAGCBAAAA|2451575|2451604|DEPARTMENT|44|60|Over military classes must use able, similar members. Hands used to bear even closed, interna|monthly| +4705|AAAAAAAABGCBAAAA|2451575|2451604|DEPARTMENT|44|61|Slightly little mothers might not pull perhaps documen|monthly| +4706|AAAAAAAACGCBAAAA|2451575|2451604|DEPARTMENT|44|62|Initial, other legs could explore more decent, marine estimates|monthly| +4707|AAAAAAAADGCBAAAA|2451575|2451604|DEPARTMENT|44|63|Greatly unlike topics light in a considerations. Friends|monthly| +4708|AAAAAAAAEGCBAAAA|2451575|2451604|DEPARTMENT|44|64|Jobs remain hopefully small offices. Big, other provinces consider as to a sites. Mor|monthly| +4709|AAAAAAAAFGCBAAAA|2451575|2451604|DEPARTMENT|44|65|Fascinating months should not gauge. Accused, foreign rights could marry |monthly| +4710|AAAAAAAAGGCBAAAA|2451575|2451604|DEPARTMENT|44|66|Interested letters say scenes. Very direct details appeal never premier, magic measures.|monthly| +4711|AAAAAAAAHGCBAAAA|2451575|2451604|DEPARTMENT|44|67|Arms admit even especially recent stages; new, nice houses may introduce once ag|monthly| +4712|AAAAAAAAIGCBAAAA|2451575|2451604|DEPARTMENT|44|68|Aware authorities ensure capable, long-term wings. Natural years cons|monthly| +4713|AAAAAAAAJGCBAAAA|2451575|2451604|DEPARTMENT|44|69|Yet great studies should not create at least there sharp messages. About firm faces contin|monthly| +4714|AAAAAAAAKGCBAAAA|2451575|2451604|DEPARTMENT|44|70|Important shares lead great, very approaches. Young prayers will see only. Gen|monthly| +4715|AAAAAAAALGCBAAAA|2451575|2451604|DEPARTMENT|44|71|Inner, busy payments ought to expect rapidly yellow poli|monthly| +4716|AAAAAAAAMGCBAAAA|2451575|2451604|DEPARTMENT|44|72|Blue, old errors will not make perhaps other partners. Beautiful, other meanings shall not |monthly| +4717|AAAAAAAANGCBAAAA|2451575|2451604|DEPARTMENT|44|73|Formal rights get twice other effects. Likely, available jobs would not care. Flexible, important m|monthly| +4718|AAAAAAAAOGCBAAAA|2451575|2451604|DEPARTMENT|44|74|Official streets must attack open in a players; primary, political schools should m|monthly| +4719|AAAAAAAAPGCBAAAA|2451575|2451604|DEPARTMENT|44|75|Unable costs should put still recent rights. Asian centres see e|monthly| +4720|AAAAAAAAAHCBAAAA|2451575|2451604|DEPARTMENT|44|76|Please additional organisations want clearly new secrets. Right p|monthly| +4721|AAAAAAAABHCBAAAA|2451575|2451604|DEPARTMENT|44|77|High, able roots may know dark, crucial conditions. Early|monthly| +4722|AAAAAAAACHCBAAAA|2451575|2451604|DEPARTMENT|44|78|Councillors give for a drivers. Likely forests can d|monthly| +4723|AAAAAAAADHCBAAAA|2451575|2451604|DEPARTMENT|44|79|Governments appear only. Olympic, novel questions murder more students. However following figures |monthly| +4724|AAAAAAAAEHCBAAAA|2451575|2451604|DEPARTMENT|44|80|In order theoretical crops would answer so. Public, poor ships would occur c|monthly| +4725|AAAAAAAAFHCBAAAA|2451575|2451604|DEPARTMENT|44|81|Accurate others would drive forward years. American, american ears depend yet all |monthly| +4726|AAAAAAAAGHCBAAAA|2451575|2451604|DEPARTMENT|44|82|British patterns employ afterwards at a consequences.|monthly| +4727|AAAAAAAAHHCBAAAA|2451575|2451604|DEPARTMENT|44|83|Therefore mathematical senses play in a adults. Related off|monthly| +4728|AAAAAAAAIHCBAAAA|2451575|2451604|DEPARTMENT|44|84|Shows increase actually unable, russian boys. Trying, jewish activities re|monthly| +4729|AAAAAAAAJHCBAAAA|2451575|2451604|DEPARTMENT|44|85|Past polls distort. Great, old crowds permit publicly home m|monthly| +4730|AAAAAAAAKHCBAAAA|2451575|2451604|DEPARTMENT|44|86|Only white years should not like dramatic, strong meals. Significant, soci|monthly| +4731|AAAAAAAALHCBAAAA|2451575|2451604|DEPARTMENT|44|87|Pictures must buy. Overall companies would not know correct, statutory principles. |monthly| +4732|AAAAAAAAMHCBAAAA|2451575|2451604|DEPARTMENT|44|88|Firm days could work from a unions. Women survive as well hands; tired, external societies shall s|monthly| +4733|AAAAAAAANHCBAAAA|2451575|||||Communities may concentrate below by the approaches; almost available surveys keep essent|| +4734|AAAAAAAAOHCBAAAA|2451575|2451604|DEPARTMENT|44|90|Birds work together words. Nearly marked patients might drink rarely; papers understand officially|monthly| +4735|AAAAAAAAPHCBAAAA|2451575|2451604|DEPARTMENT|44|91|Close, japanese details used to receive well rapid homes. Units become both. A bit full effects ex|monthly| +4736|AAAAAAAAAICBAAAA|2451575|2451604|DEPARTMENT|44|92|Professional, independent results go republics. Proceedings appreciate above holders. Young,|monthly| +4737|AAAAAAAABICBAAAA|2451575|2451604|DEPARTMENT|44|93|Children might arise then literally small difficulties. Monetary, total children rank well. |monthly| +4738|AAAAAAAACICBAAAA|2451575|2451604|DEPARTMENT|44|94|Now round months write quite. Fashionable views meet just main trades. Secondary, |monthly| +4739|AAAAAAAADICBAAAA|2451575|2451604|DEPARTMENT|44|95|Far from suitable causes help there. Sharp liable doubts should not stare yet |monthly| +4740|AAAAAAAAEICBAAAA|2451575|2451604|DEPARTMENT|44|96|Democrats beat really simply professional details. Years find from a pupi|monthly| +4741|AAAAAAAAFICBAAAA|2451575|2451604|DEPARTMENT|44|97|Long religious examples might match buyers; democratic tracks keep|monthly| +4742|AAAAAAAAGICBAAAA|2451575|2451604|DEPARTMENT|44|98|Special practices think so good windows; economic, historic results will run bet|monthly| +4743|AAAAAAAAHICBAAAA|2451575|2451604|DEPARTMENT|44|99|Expressions fetch both with the words. Broad lives used|monthly| +4744|AAAAAAAAIICBAAAA|2451575|2451604|DEPARTMENT|44|100|Then prime areas will not hesitate skills. Alternative, main measures say. |monthly| +4745|AAAAAAAAJICBAAAA|2451575|2451604|DEPARTMENT|44|101|Economies would want chief, environmental kinds. Visual, american concentrations offer orig|monthly| +4746|AAAAAAAAKICBAAAA|2451575|2451604|DEPARTMENT|44|102|Then strategic years must represent. Specific, little problems keep from |monthly| +4747|AAAAAAAALICBAAAA|2451575|2451604|DEPARTMENT|44|103|New, effective islands record new clients. Simply contemporary f|monthly| +4748|AAAAAAAAMICBAAAA|2451575|2451604|DEPARTMENT|44|104|Safe, organic careers would face towns. Other countries might not move p|monthly| +4749|AAAAAAAANICBAAAA|2451575|2451604|DEPARTMENT|44|105|Immediately quiet issues provide quite new sections; continental houses could brin|monthly| +4750|AAAAAAAAOICBAAAA|2451575|2451604|DEPARTMENT|44|106|Committees used to envisage then christian areas. Years earn fina|monthly| +4751|AAAAAAAAPICBAAAA|2451575|2451604|DEPARTMENT|44|107|Small, expensive models require successfully. Always increa|monthly| +4752|AAAAAAAAAJCBAAAA|2451575|2451604|DEPARTMENT|44|108|Real schools give neatly shy systems; so limited positions improve still. So necessary sympt|monthly| +4753|AAAAAAAABJCBAAAA|2451605|2451634|DEPARTMENT|45|1|New children could make then royal males. Tough, other m|monthly| +4754|AAAAAAAACJCBAAAA|2451605|2451634|DEPARTMENT|45|2|As social details maximise african, british patients. New citizens break later othe|monthly| +4755|AAAAAAAADJCBAAAA|2451605|2451634|DEPARTMENT|45|3|Comparable, international times could succeed now really very paintin|monthly| +4756|AAAAAAAAEJCBAAAA|2451605|2451634|DEPARTMENT|45|4|Different experts will not emphasise else colourful seconds. Particula|monthly| +4757|AAAAAAAAFJCBAAAA|2451605|2451634|DEPARTMENT|45|5|Articles could tempt in a members. Nervous, middle years used to continue he|monthly| +4758|AAAAAAAAGJCBAAAA|2451605|2451634|DEPARTMENT|45|6|Useless accidents come. Positive, creative prices may investigate therefore without a ma|monthly| +4759|AAAAAAAAHJCBAAAA|2451605|2451634|DEPARTMENT|45|7|Rich, foreign stars require into a policies. Permanently front facilities ta|monthly| +4760|AAAAAAAAIJCBAAAA|2451605|2451634|DEPARTMENT|45|8|Joint thousands will not mention thus enterprises. As real careers must see somewhe|monthly| +4761|AAAAAAAAJJCBAAAA|2451605|2451634|DEPARTMENT|45|9|Causes may cope more successful members. Small, mean rates tell most new stars. Still|monthly| +4762|AAAAAAAAKJCBAAAA|2451605|2451634|DEPARTMENT|45|10|Children ought to win for a police. Jobs tick possible, awkward kids. Legal, single sons |monthly| +4763|AAAAAAAALJCBAAAA|2451605|2451634|DEPARTMENT|45|11|Sometimes royal operations can get books. Pale shoes break later|monthly| +4764|AAAAAAAAMJCBAAAA|2451605|2451634|DEPARTMENT|45|12|Materials manage entirely financial, working things; boys like schools. Far, elderly assets |monthly| +4765|AAAAAAAANJCBAAAA|2451605|2451634|DEPARTMENT|45|13|Keys allocate. Changes will lead members. Fair, fundamental members drive some|monthly| +4766|AAAAAAAAOJCBAAAA|2451605|2451634|DEPARTMENT|45|14|Hard profits like recently social, golden talks. Important arms see more elderly voices. Both demo|monthly| +4767|AAAAAAAAPJCBAAAA|2451605|2451634|DEPARTMENT|45|15|Right drivers clinch mothers. Working, high cars would not reconcile there attractive columns. T|monthly| +4768|AAAAAAAAAKCBAAAA|2451605|2451634|DEPARTMENT|45|16|Needs enjoy classes. National, particular programmes exc|monthly| +4769|AAAAAAAABKCBAAAA|2451605|2451634|DEPARTMENT|45|17|Roads might call for a theories. Once more innocent cells ought to seem silly, b|monthly| +4770|AAAAAAAACKCBAAAA|2451605|2451634|DEPARTMENT|45|18|Virtually ancient degrees understand on the months. Questions ought to imagine issues. Level, b|monthly| +4771|AAAAAAAADKCBAAAA|2451605|2451634|DEPARTMENT|45|19|Largely alleged issues take further young, human bodies.|monthly| +4772|AAAAAAAAEKCBAAAA|2451605|2451634|DEPARTMENT|45|20|Necessary, british types would not play because of a notions. Joint|monthly| +4773|AAAAAAAAFKCBAAAA|2451605|2451634|DEPARTMENT|45|21|Normally slow signs will die nevertheless as well steep savings; formal sounds used to|monthly| +4774|AAAAAAAAGKCBAAAA|2451605|2451634|DEPARTMENT|45|22|Then top objectives see tightly. Important, big journalists dip otherwise |monthly| +4775|AAAAAAAAHKCBAAAA|2451605||DEPARTMENT||||monthly| +4776|AAAAAAAAIKCBAAAA|2451605|2451634|DEPARTMENT|45|24|Other, economic chairs get now applications. Correct, du|monthly| +4777|AAAAAAAAJKCBAAAA|2451605|2451634|DEPARTMENT|45|25|Details will perform today subsequent ways; effects cannot assemble also small |monthly| +4778|AAAAAAAAKKCBAAAA|2451605|2451634|DEPARTMENT|45|26|Sad operations reassure sharp, negative strategies. Partners must not imagine firmly respondents.|monthly| +4779|AAAAAAAALKCBAAAA|2451605|2451634|DEPARTMENT|45|27|Grounds start to a times. Different years shall review sometimes grim, great managers. Total, cle|monthly| +4780|AAAAAAAAMKCBAAAA|2451605|2451634|DEPARTMENT|45|28|Difficult, afraid affairs shall overturn. Once more |monthly| +4781|AAAAAAAANKCBAAAA|2451605|2451634|DEPARTMENT|45|29|Prisoners shall make european, poor pounds. Right post-war tons undergo usual industries; men lear|monthly| +4782|AAAAAAAAOKCBAAAA|2451605|2451634|DEPARTMENT|45|30|Young procedures know more social changes; british, strong days report also. Inner, early ele|monthly| +4783|AAAAAAAAPKCBAAAA|2451605|2451634|DEPARTMENT|45|31|Systematically nuclear houses will suspect of course addresses. Logical, overall lives investig|monthly| +4784|AAAAAAAAALCBAAAA|2451605|2451634|DEPARTMENT|45|32|Happy audiences cannot believe indeed growing, conscious feet. There professiona|monthly| +4785|AAAAAAAABLCBAAAA|2451605|2451634|DEPARTMENT|45|33|Live, dead weeks may not ask only commercially ordinary years. Fu|monthly| +4786|AAAAAAAACLCBAAAA|2451605|2451634|DEPARTMENT|45|34|Serious years cannot make either black months; there other forests take about a t|monthly| +4787|AAAAAAAADLCBAAAA|2451605|2451634|DEPARTMENT|45|35|Only particular authorities believe never. Physical sides buy |monthly| +4788|AAAAAAAAELCBAAAA|2451605|2451634|DEPARTMENT|45|36|Interests ought to come severe areas. Parts might matter years. Poor, positive investors return m|monthly| +4789|AAAAAAAAFLCBAAAA|2451605|2451634|DEPARTMENT|45|37|Large computers inhibit easy, new parts. Other, unknown moments store about with the feet. Differe|monthly| +4790|AAAAAAAAGLCBAAAA|2451605|2451634|DEPARTMENT|45|38|Toxic problems shed also. Back years match as. Already asian actions drive ever with a feat|monthly| +4791|AAAAAAAAHLCBAAAA|2451605|2451634|DEPARTMENT|45|39|Much political institutions present at least years. Photogr|monthly| +4792|AAAAAAAAILCBAAAA|2451605|2451634|DEPARTMENT|45|40|Husbands protect years. Harsh, bad occasions could put. Months could con|monthly| +4793|AAAAAAAAJLCBAAAA|2451605|2451634|DEPARTMENT|45|41|Very, easy kinds could not seem on the lips. Short sys|monthly| +4794|AAAAAAAAKLCBAAAA|2451605|2451634|DEPARTMENT|45|42|Earlier particular visitors communicate old, local troops. British, ce|monthly| +4795|AAAAAAAALLCBAAAA|2451605|2451634|DEPARTMENT|45|43|Western, coming errors get entirely on a rates. Years should protect as thus miserable off|monthly| +4796|AAAAAAAAMLCBAAAA|2451605|2451634|DEPARTMENT|45|44|Young, public police will speak of course individual operations. Sisters replace easi|monthly| +4797|AAAAAAAANLCBAAAA|2451605|2451634|DEPARTMENT|45|45|Usual, legal failures could not comply so arts. Well desp|monthly| +4798|AAAAAAAAOLCBAAAA|2451605|2451634|DEPARTMENT|45|46|However long consumers pay technically raw, other words. Log|monthly| +4799|AAAAAAAAPLCBAAAA|2451605|2451634|DEPARTMENT|45|47|Full eyes should call immediately times. Underlying, free centres cannot ca|monthly| +4800|AAAAAAAAAMCBAAAA|2451605|2451634|DEPARTMENT|45|48|Other schemes might inform usually. So single teachers will |monthly| +4801|AAAAAAAABMCBAAAA|2451605|2451634|DEPARTMENT|45|49|Camps will not depress real, good attitudes. Japanese rights contain here. Thus ready res|monthly| +4802|AAAAAAAACMCBAAAA|2451605|2451634|DEPARTMENT|45|50|Similar, mad companies say. Blue, smooth combinations identify suddenly at a sec|monthly| +4803|AAAAAAAADMCBAAAA|2451605|2451634|DEPARTMENT|45|51|Ordinary awards ask again. Years could not like only finally other proceedings. Difficult colleague|monthly| +4804|AAAAAAAAEMCBAAAA|2451605|2451634|DEPARTMENT|45|52|Walls call vaguely happy towns. Gardens shall stop. Ideas mean problems.|monthly| +4805|AAAAAAAAFMCBAAAA|2451605|2451634|DEPARTMENT|45|53|Soft, low developments might make very rules. Dimensions used to set internation|monthly| +4806|AAAAAAAAGMCBAAAA|2451605|2451634|DEPARTMENT|45|54|Accessible laboratories penetrate rightly attitudes. Then var|monthly| +4807|AAAAAAAAHMCBAAAA|2451605|2451634|DEPARTMENT|45|55|Diverse, appropriate chips can incorporate nice operations; once as|monthly| +4808|AAAAAAAAIMCBAAAA|2451605|2451634|DEPARTMENT|45|56|Similar, pale representatives may forget in general|monthly| +4809|AAAAAAAAJMCBAAAA|2451605|2451634|DEPARTMENT|45|57|Gradual items cause most major structures. Bands see. So phy|monthly| +4810|AAAAAAAAKMCBAAAA|2451605|2451634|DEPARTMENT|45|58|Together special months manage eventually even current minutes. Important bits achieve elections|monthly| +4811|AAAAAAAALMCBAAAA|2451605|2451634|DEPARTMENT|45|59|Current interests must not find for a resources. Comparatively alternative lines find |monthly| +4812|AAAAAAAAMMCBAAAA|2451605|2451634|DEPARTMENT|45|60|Low women might prove comparatively due, dramatic wings. Dreadful, ideal years shall consider|monthly| +4813|AAAAAAAANMCBAAAA|2451605|2451634|DEPARTMENT|45|61|More other cells kill much papers. Better red directors used to adopt now less|monthly| +4814|AAAAAAAAOMCBAAAA|2451605|2451634|DEPARTMENT|45|62|True perceptions ought to enjoy here professional goods. |monthly| +4815|AAAAAAAAPMCBAAAA|2451605|2451634|DEPARTMENT|45|63|Effects shall not prove public weeks. Also social patterns |monthly| +4816|AAAAAAAAANCBAAAA|2451605|2451634|DEPARTMENT|45|64|Unexpected, european minds admire much long pictures. Matches murde|monthly| +4817|AAAAAAAABNCBAAAA|2451605|2451634|DEPARTMENT|45|65|Bodies used to ask also. Indeed sufficient funds provide as long huma|monthly| +4818|AAAAAAAACNCBAAAA|2451605|2451634|DEPARTMENT|45|66|Firm, public machines must not play really common, only member|monthly| +4819|AAAAAAAADNCBAAAA|2451605|2451634|DEPARTMENT|45|67|Meetings ought to participate finally. Long, electric women enlarge even notable situati|monthly| +4820|AAAAAAAAENCBAAAA|2451605|2451634|DEPARTMENT|45|68|Hours should see increasingly decisions. Short, special directions ought t|monthly| +4821|AAAAAAAAFNCBAAAA|2451605|2451634|DEPARTMENT|45|69|Animals must not ensure well straightforward partic|monthly| +4822|AAAAAAAAGNCBAAAA|2451605|2451634|DEPARTMENT|45|70|Narrow, rare elections come. Public, chinese characters will cross notably men. Emotiona|monthly| +4823|AAAAAAAAHNCBAAAA|2451605|2451634|DEPARTMENT|45|71|Other students shall not go swiftly very toxic offers. Outer years m|monthly| +4824|AAAAAAAAINCBAAAA|2451605|2451634|DEPARTMENT|45|72|Hard certain workers lend only; different points used t|monthly| +4825|AAAAAAAAJNCBAAAA|2451605|2451634|DEPARTMENT|45|73|Scientific, local things buy now red, clear members. Schemes see in addition special|monthly| +4826|AAAAAAAAKNCBAAAA|2451605|2451634|DEPARTMENT|45|74|Unchanged, only services might risk here attitudes. Moral, beautiful prices consist hardly thus |monthly| +4827|AAAAAAAALNCBAAAA|2451605|2451634|DEPARTMENT|45|75|Extremely individual implications should not control systems. Visible losses will reduce. |monthly| +4828|AAAAAAAAMNCBAAAA|2451605|2451634|DEPARTMENT|45|76|Present women could not deal then; years find too either mass owners|monthly| +4829|AAAAAAAANNCBAAAA|2451605|2451634|DEPARTMENT|45|77|Social, causal numbers could divide more impossible|monthly| +4830|AAAAAAAAONCBAAAA|2451605|2451634|DEPARTMENT|45|78|Reasons buy no longer possible attitudes; digital, scottish years shall beg|monthly| +4831|AAAAAAAAPNCBAAAA||||45|||| +4832|AAAAAAAAAOCBAAAA|2451605|2451634|DEPARTMENT|45|80|For instance true things give in a aspirations. Months die usual eyes. So single machines sanction |monthly| +4833|AAAAAAAABOCBAAAA|2451605|2451634|DEPARTMENT|45|81|Particular shares cannot win relevant names. As effective levels ought to win too s|monthly| +4834|AAAAAAAACOCBAAAA|2451605|2451634|DEPARTMENT|45|82|Almost small days shall not argue ever; awfully specific houses would not learn yet. Police rel|monthly| +4835|AAAAAAAADOCBAAAA|2451605|2451634|DEPARTMENT|45|83|Diverse, historic authorities may not complete straight. Forces mark|monthly| +4836|AAAAAAAAEOCBAAAA|2451605|2451634|DEPARTMENT|45|84|Yet high groups recognise never by a children. Less other humans should post in public realistic p|monthly| +4837|AAAAAAAAFOCBAAAA|2451605|2451634|DEPARTMENT|45|85|Sure machines phone white minds. Even open shoulders mus|monthly| +4838|AAAAAAAAGOCBAAAA|2451605|2451634|DEPARTMENT|45|86|Strictly industrial decisions give brutally citizens. Versions may rest very fre|monthly| +4839|AAAAAAAAHOCBAAAA|2451605|2451634|DEPARTMENT|45|87|Victorian, automatic conditions influence direct cultures; basic, good applications deliver over|monthly| +4840|AAAAAAAAIOCBAAAA|2451605|2451634|DEPARTMENT|45|88|Issues may visit towards a sides. Available countries can display so |monthly| +4841|AAAAAAAAJOCBAAAA|2451605|2451634|DEPARTMENT|45|89|Perfect events must operate with a steps. Knees satisfy different, |monthly| +4842|AAAAAAAAKOCBAAAA|2451605|2451634|DEPARTMENT|45|90|Regional teeth shall avoid. Basic measures may mimic to a cells. Home ancient months put a|monthly| +4843|AAAAAAAALOCBAAAA|2451605|2451634|DEPARTMENT|45|91|Linear areas shall extract precisely less associated interests.|monthly| +4844|AAAAAAAAMOCBAAAA|2451605|2451634|DEPARTMENT|45|92|Then traditional criteria could wear then groups. Girls turn just towards the visitors. Homes red|monthly| +4845|AAAAAAAANOCBAAAA|2451605|2451634|DEPARTMENT|45|93|Too poor rates should cope in a police. Shares will discuss new interpretations.|monthly| +4846|AAAAAAAAOOCBAAAA|2451605|2451634|DEPARTMENT|45|94|Public, short players would work so for a aspects. Glad, concerned leaders tell further services.|monthly| +4847|AAAAAAAAPOCBAAAA|2451605|2451634|DEPARTMENT|45|95|English, secondary rooms confront efficient, soviet children; ab|monthly| +4848|AAAAAAAAAPCBAAAA|2451605|2451634|DEPARTMENT|45|96|Rates can look years. New, good women used to come again f|monthly| +4849|AAAAAAAABPCBAAAA|2451605|2451634|DEPARTMENT|45|97|Blue facilities resign serious, separate persons. Long, open cont|monthly| +4850|AAAAAAAACPCBAAAA|2451605|2451634|DEPARTMENT|45|98|Junior, possible values shall help well off a taxes. Old, particular le|monthly| +4851|AAAAAAAADPCBAAAA|2451605|2451634|DEPARTMENT|45|99|Abilities determine for a men. Principal, remaining circumstances visit po|monthly| +4852|AAAAAAAAEPCBAAAA|2451605|2451634|DEPARTMENT|45|100|There wrong teams devote elsewhere. Almost public days would |monthly| +4853|AAAAAAAAFPCBAAAA|2451605|2451634|DEPARTMENT|45|101|Possible, afraid others spend probably red members; words|monthly| +4854|AAAAAAAAGPCBAAAA|2451605|2451634|DEPARTMENT|45|102|Standards claim at a ears. Readily single objects should provide somet|monthly| +4855|AAAAAAAAHPCBAAAA|2451605|2451634|DEPARTMENT|45|103|Photographs would not disperse loud on a seats. Possible documents hold even m|monthly| +4856|AAAAAAAAIPCBAAAA|2451605|2451634|DEPARTMENT|45|104|Educational, secure details may check as in a classes. Very public forms practise enou|monthly| +4857|AAAAAAAAJPCBAAAA|2451605|2451634|DEPARTMENT|45|105|Naturally western hands would make contracts. Presidential parts find strongly fir|monthly| +4858|AAAAAAAAKPCBAAAA|2451605|2451634|DEPARTMENT|45|106|Internal pp. bring yet good operations. Numbers charge instead members. Pp. wea|monthly| +4859|AAAAAAAALPCBAAAA|2451605|2451634|DEPARTMENT|45|107|Silver trends used to come either later political holders. Good rural organizati|monthly| +4860|AAAAAAAAMPCBAAAA|2451605|2451634|DEPARTMENT|45|108|Posts see by a months. New leaves opt large activities. Managers achieve however great|monthly| +4861|AAAAAAAANPCBAAAA|2451635|2451664|DEPARTMENT|46|1|Pages might make just. Elections give as levels. Heavy, important employees |monthly| +4862|AAAAAAAAOPCBAAAA|2451635|2451664|DEPARTMENT|46|2|Ready, initial bands believe fairly particular sheets. Ideas approve now branches. Early|monthly| +4863|AAAAAAAAPPCBAAAA|2451635|2451664|DEPARTMENT|46|3|Decisions ought to say. Creative circumstances describe old, necessary rates; s|monthly| +4864|AAAAAAAAAADBAAAA|2451635|2451664|DEPARTMENT|46|4|Local guests challenge therefore about normal courses. Fellow, obvious|monthly| +4865|AAAAAAAABADBAAAA|2451635|2451664|DEPARTMENT|46|5|Just early points go appropriate, local insects. Simple,|monthly| +4866|AAAAAAAACADBAAAA|2451635|2451664|DEPARTMENT|46|6|Main homes may know so. Common departments invest at least. Laws must su|monthly| +4867|AAAAAAAADADBAAAA|2451635|2451664|DEPARTMENT|46|7|General, only years tell only issues. True horses will not know most conditions. Ex|monthly| +4868|AAAAAAAAEADBAAAA|2451635|2451664|DEPARTMENT|46|8|Pupils will damage else enough clear arms. Banks accept important, difficult countries. Child|monthly| +4869|AAAAAAAAFADBAAAA|2451635|2451664|DEPARTMENT|46|9|Safely secondary rules occur much from a parties. Upr|monthly| +4870|AAAAAAAAGADBAAAA|2451635|2451664|DEPARTMENT|46|10|Even new teams shall know more into a others. Workers stretc|monthly| +4871|AAAAAAAAHADBAAAA|2451635|2451664|DEPARTMENT|46|11|Careful, human changes refer expensive, necessary women; aspects ought to achieve in the em|monthly| +4872|AAAAAAAAIADBAAAA|2451635|2451664|DEPARTMENT|46|12|Questions could not mean against a signs. Thus very teachers must bring. New sales ought|monthly| +4873|AAAAAAAAJADBAAAA|2451635|2451664|DEPARTMENT|46|13|Probably atomic limitations will help to a papers. Civil|monthly| +4874|AAAAAAAAKADBAAAA|2451635|2451664|DEPARTMENT|46|14|Speeches must communicate hot ways. Heads leave findings. So as|monthly| +4875|AAAAAAAALADBAAAA|2451635|2451664|DEPARTMENT|46|15|Assistant, english birds could split exactly positive duties. Elsewhere availa|monthly| +4876|AAAAAAAAMADBAAAA|2451635|2451664|DEPARTMENT|46|16|International notions would not face in a laws; political, universal evenings need. Days c|monthly| +4877|AAAAAAAANADBAAAA|2451635|2451664|DEPARTMENT|46|17|Additional, small points serve ago hardly new discu|monthly| +4878|AAAAAAAAOADBAAAA|2451635|2451664|DEPARTMENT|46|18|Flowers clear physically keen rights. Now only friends shall stretch |monthly| +4879|AAAAAAAAPADBAAAA|2451635|2451664|DEPARTMENT|46|19|Walls apply also classes. Conscious, weekly arms could recognize steeply. Ears may s|monthly| +4880|AAAAAAAAABDBAAAA|2451635|2451664|DEPARTMENT|46|20|Dark, new firms know abroad; odd opportunities stare also. Radical job|monthly| +4881|AAAAAAAABBDBAAAA|2451635|2451664|DEPARTMENT|46|21|Clearly current talks must test good systems. Full things may take menta|monthly| +4882|AAAAAAAACBDBAAAA|2451635|2451664|DEPARTMENT|46|22|Golden, free males address indeed political, extra|monthly| +4883|AAAAAAAADBDBAAAA|2451635|2451664|DEPARTMENT|46|23|Different police will not defeat tory, wrong agreements. So-called, ordinary y|monthly| +4884|AAAAAAAAEBDBAAAA|2451635|2451664|DEPARTMENT|46|24|Previous, ideal police hear exclusively significant toys; spare, british features|monthly| +4885|AAAAAAAAFBDBAAAA|2451635|2451664|DEPARTMENT|46|25|Royal, regional organisations might not suspect as southern matters;|monthly| +4886|AAAAAAAAGBDBAAAA|2451635|2451664|DEPARTMENT|46|26|Groups used to mean eventually. Months race much there classical systems. Sug|monthly| +4887|AAAAAAAAHBDBAAAA|2451635|2451664|DEPARTMENT|46|27|Below private heads shall settle widely years. National rocks ou|monthly| +4888|AAAAAAAAIBDBAAAA|2451635|2451664|DEPARTMENT|46|28|Consumers consider. Animals must not hold also national services. About annual years prevail cu|monthly| +4889|AAAAAAAAJBDBAAAA|2451635|2451664|DEPARTMENT|46|29|Also able men should make respectively roman, little girls. Simpl|monthly| +4890|AAAAAAAAKBDBAAAA|2451635|2451664|DEPARTMENT|46|30|Private, important units tell across tensions. Good, soft things sell in the branches. Foll|monthly| +4891|AAAAAAAALBDBAAAA|2451635|2451664|DEPARTMENT|46|31|Now useless allowances help ideas. Boards stop much equally pe|monthly| +4892|AAAAAAAAMBDBAAAA|2451635|2451664|DEPARTMENT|46|32|Nuclear suggestions used to provide generally sections; economic, othe|monthly| +4893|AAAAAAAANBDBAAAA|2451635|2451664|DEPARTMENT|46|33|Enough bloody services should select eyes. Serious societies used to come at least truly due da|monthly| +4894|AAAAAAAAOBDBAAAA|2451635|2451664|DEPARTMENT|46|34|Public, other problems must die surely in the fans. Short, possible farmers would |monthly| +4895|AAAAAAAAPBDBAAAA|2451635|2451664|DEPARTMENT|46|35|Prime examples will not cause often in a sounds. Auditors see only, new cou|monthly| +4896|AAAAAAAAACDBAAAA|2451635|2451664|DEPARTMENT|46|36|So heavy versions would not recognize so relations. Material, close|monthly| +4897|AAAAAAAABCDBAAAA|2451635|||||Great, financial forms shall assess either emotional mini|monthly| +4898|AAAAAAAACCDBAAAA|2451635|2451664|DEPARTMENT|46|38|Gifts speak especially absolutely other increases. I|monthly| +4899|AAAAAAAADCDBAAAA|2451635|2451664|DEPARTMENT|46|39|Thoughtfully chief studies used to examine now other, c|monthly| +4900|AAAAAAAAECDBAAAA|2451635|2451664|DEPARTMENT|46|40|Complete signs should take behind emotional efforts. Ahead relevant parts must say central, p|monthly| +4901|AAAAAAAAFCDBAAAA|2451635|2451664|DEPARTMENT|46|41|New, long names feel also continuously fair rights|monthly| +4902|AAAAAAAAGCDBAAAA|2451635|2451664|DEPARTMENT|46|42|Circumstances will buy now tons. Local, annual events activate particular guns; techniques may|monthly| +4903|AAAAAAAAHCDBAAAA|2451635|2451664|DEPARTMENT|46|43|Legal bodies will push quite bad, external stones. Very italian c|monthly| +4904|AAAAAAAAICDBAAAA|2451635|2451664|DEPARTMENT|46|44|Children see commercial, structural sides; straight excellent communications wish dang|monthly| +4905|AAAAAAAAJCDBAAAA|2451635|2451664|DEPARTMENT|46|45|Aware standards must not give variables. Tonight living direc|monthly| +4906|AAAAAAAAKCDBAAAA|2451635|2451664|DEPARTMENT|46|46|Yet racial newspapers shall work cases. Great agents connect even. Particular, important resour|monthly| +4907|AAAAAAAALCDBAAAA|2451635|2451664|DEPARTMENT|46|47|Now bitter matters may supply privately english results. Various foreigners scream also at a st|monthly| +4908|AAAAAAAAMCDBAAAA|2451635|2451664|DEPARTMENT|46|48|Gradual ladies could meet entire players. Smooth times avoid yet studie|monthly| +4909|AAAAAAAANCDBAAAA|2451635|2451664|DEPARTMENT|46|49|Hard gaps may help never famous days. Common, male policies could |monthly| +4910|AAAAAAAAOCDBAAAA|2451635|2451664|DEPARTMENT|46|50|Similar, great exhibitions touch of course human ideas. Only specialist findings serve also w|monthly| +4911|AAAAAAAAPCDBAAAA|2451635|2451664|DEPARTMENT|46|51|Only prime services tell prime, liable rounds. Particular letters should sell implicit, pu|monthly| +4912|AAAAAAAAADDBAAAA|2451635|2451664|DEPARTMENT|46|52|Existing, urgent directors return home finally labour benefits. |monthly| +4913|AAAAAAAABDDBAAAA|2451635|2451664|DEPARTMENT|46|53|Feelings will come. Lively, nice roots think types; thanks used to create wholly gay cards; mo|monthly| +4914|AAAAAAAACDDBAAAA|2451635|2451664|DEPARTMENT|46|54|Different, other streets could specify therefore around simple courts. Again other hands ought t|monthly| +4915|AAAAAAAADDDBAAAA|2451635|2451664|DEPARTMENT|46|55|Rates continue for a blues. So electrical problems might drop already|monthly| +4916|AAAAAAAAEDDBAAAA|2451635|2451664|DEPARTMENT|46|56|All sure prisoners must not benefit elements. Privileges |monthly| +4917|AAAAAAAAFDDBAAAA|2451635|2451664|DEPARTMENT|46|57|Things shall complete attractive, corresponding generations. Successful c|monthly| +4918|AAAAAAAAGDDBAAAA|2451635|2451664|DEPARTMENT|46|58|Even main levels cannot want at a parties. Arbitrary e|monthly| +4919|AAAAAAAAHDDBAAAA|2451635|2451664|DEPARTMENT|46|59|Animals must visit very gold, main councils. Sure, domestic relationships regard|monthly| +4920|AAAAAAAAIDDBAAAA|2451635|2451664|DEPARTMENT|46|60|Services may see aspects. Broad, busy instructions start behind quite able voters. Little|monthly| +4921|AAAAAAAAJDDBAAAA|2451635|2451664|DEPARTMENT|46|61|Conservatives want also cells. Just acceptable others stick|monthly| +4922|AAAAAAAAKDDBAAAA|2451635|2451664|DEPARTMENT|46|62|Small, distinctive visitors could go living situations. Actually good eyes might bre|monthly| +4923|AAAAAAAALDDBAAAA|2451635|2451664|DEPARTMENT|46|63|Little demands see about by a charges. Also forward relations may not see pairs. Truly terr|monthly| +4924|AAAAAAAAMDDBAAAA|2451635|2451664|DEPARTMENT|46|64|Everyday, favorite wishes grow even to a conditions. Students know v|monthly| +4925|AAAAAAAANDDBAAAA|2451635|2451664|DEPARTMENT|46|65|Closely new films would see again at the years. Proposals go yet yellow com|monthly| +4926|AAAAAAAAODDBAAAA|2451635|2451664|DEPARTMENT|46|66|Aware towns could not answer very agricultural teeth. Silver, ori|monthly| +4927|AAAAAAAAPDDBAAAA|2451635|2451664|DEPARTMENT|46|67|Again additional words campaign shortly simply early centres. In|monthly| +4928|AAAAAAAAAEDBAAAA|2451635|2451664|DEPARTMENT|46|68|Concessions will thank yet as a affairs. Largely political numbers close more men.|monthly| +4929|AAAAAAAABEDBAAAA|2451635|2451664|DEPARTMENT|46|69|Sufficient animals set simply parental options. Urban, ltd. women explain real par|monthly| +4930|AAAAAAAACEDBAAAA|2451635|2451664|DEPARTMENT|46|70|Guidelines shall reinforce basic, good circumstances. Errors wait only major men. Social expen|monthly| +4931|AAAAAAAADEDBAAAA|2451635|2451664|DEPARTMENT|46|71|Students attend however here specialist properties. Policies control over a rewards. So as go|monthly| +4932|AAAAAAAAEEDBAAAA|2451635|2451664|DEPARTMENT|46|72|Instructions use just new details. Only damp wives mention more businessm|monthly| +4933|AAAAAAAAFEDBAAAA|2451635|2451664|DEPARTMENT|46|73|Now essential decisions draw too tons. Great, strong months |monthly| +4934|AAAAAAAAGEDBAAAA|2451635|2451664|DEPARTMENT|46|74|There aware problems can wear safe industries. Accordingly aware things must win|monthly| +4935|AAAAAAAAHEDBAAAA|2451635|2451664|DEPARTMENT|46|75|Just national residents make firstly excellent discussions. So regional term|monthly| +4936|AAAAAAAAIEDBAAAA|2451635|2451664|DEPARTMENT|46|76|Simple judges may dig high, private women; little, good ambitions cl|monthly| +4937|AAAAAAAAJEDBAAAA|2451635|2451664|DEPARTMENT|46|77|Little other days admit enough everyday, upper changes. Men used |monthly| +4938|AAAAAAAAKEDBAAAA|2451635|2451664|DEPARTMENT|46|78|Special principles might seek altogether plants; ago good g|monthly| +4939|AAAAAAAALEDBAAAA|2451635|2451664|DEPARTMENT|46|79|Well related activities may get finally games. Strong, fixed charges would|monthly| +4940|AAAAAAAAMEDBAAAA|2451635|2451664|DEPARTMENT|46|80|Red factories want truly with a cells. Sometimes cold paren|monthly| +4941|AAAAAAAANEDBAAAA|2451635|2451664|DEPARTMENT|46|81|Natural pictures get; at all high police can help to the tasks. |monthly| +4942|AAAAAAAAOEDBAAAA|2451635|2451664|DEPARTMENT|46|82|Final arguments will not point again always other words. Questions talk|monthly| +4943|AAAAAAAAPEDBAAAA|2451635|2451664|DEPARTMENT|46|83|Managers used to communicate square premises. Male members make indeed in a representatives; c|monthly| +4944|AAAAAAAAAFDBAAAA|2451635|2451664|DEPARTMENT|46|84|Brothers might avoid obviously. Corporate, clear r|monthly| +4945|AAAAAAAABFDBAAAA|2451635|2451664|DEPARTMENT|46|85|Rather simple players might address easier other international children. La|monthly| +4946|AAAAAAAACFDBAAAA|2451635|2451664|DEPARTMENT|46|86|Remaining, single counties would expect too fine reactions. Short ki|monthly| +4947|AAAAAAAADFDBAAAA|2451635|2451664|DEPARTMENT|46|87|Local, passive hours meet very effects. Persistent teachers may spe|monthly| +4948|AAAAAAAAEFDBAAAA|2451635|2451664|DEPARTMENT|46|88|Old, large-scale kinds will not reduce; wives shall look here new workers; i|monthly| +4949|AAAAAAAAFFDBAAAA|2451635|2451664|DEPARTMENT|46|89|Activities will protect far crude, slight weeks. Markets should|monthly| +4950|AAAAAAAAGFDBAAAA|2451635|2451664|DEPARTMENT|46|90|Bodies shall control away parties. Easy, ready tables must please coastal men. Bla|monthly| +4951|AAAAAAAAHFDBAAAA|2451635|2451664|DEPARTMENT|46|91|Levels clarify organizations. Small, main ways might run wrongly wor|monthly| +4952|AAAAAAAAIFDBAAAA|2451635|2451664|DEPARTMENT|46|92|Military, clear cities could find proposed years. Ne|monthly| +4953|AAAAAAAAJFDBAAAA|2451635|2451664|DEPARTMENT|46|93|Physical, reliable pounds win almost large posts. Well social hours shall not accompany however alw|monthly| +4954|AAAAAAAAKFDBAAAA|2451635|2451664|DEPARTMENT|46|94|Awards should come. Below short years think similar officers; probably new friends enable |monthly| +4955|AAAAAAAALFDBAAAA|2451635|2451664|DEPARTMENT|46|95|Students choose. Soviet negotiations would know away out of a examples. Powerful, head things w|monthly| +4956|AAAAAAAAMFDBAAAA|2451635|2451664|DEPARTMENT|46|96|No longer brief police run numerous duties. Paintings return; ordinary volumes f|monthly| +4957|AAAAAAAANFDBAAAA|2451635|2451664|DEPARTMENT|46|97|Islamic decisions wait practically nearly passive rates. L|monthly| +4958|AAAAAAAAOFDBAAAA|2451635|2451664|DEPARTMENT|46|98|Various men may not want roughly powers; roots can identify wrong clubs. Laws sa|monthly| +4959|AAAAAAAAPFDBAAAA|2451635|2451664|DEPARTMENT|46|99|Fingers shall get afterwards at the activities. Countries arrive both regional fans; mor|monthly| +4960|AAAAAAAAAGDBAAAA|2451635|2451664|DEPARTMENT|46|100|Winds assume no longer stupid ministers; available, large matters succee|monthly| +4961|AAAAAAAABGDBAAAA|2451635|2451664|DEPARTMENT|46|101|Specified courts would go good troops. Sexual tests question after a problems. Too great probl|monthly| +4962|AAAAAAAACGDBAAAA|2451635|2451664|DEPARTMENT|46|102|Together institutional managers ought to get hesitantly good projects. Now present sales mig|monthly| +4963|AAAAAAAADGDBAAAA|2451635|2451664|DEPARTMENT|46|103|Clear attempts should not reveal more. Also poor grounds tick other years. Too small hours could|monthly| +4964|AAAAAAAAEGDBAAAA|2451635|2451664|DEPARTMENT|46|104|Always potential figures swallow domestic plants. Matters would rest now appropriate markets; m|monthly| +4965|AAAAAAAAFGDBAAAA|2451635||DEPARTMENT||105||| +4966|AAAAAAAAGGDBAAAA|2451635|2451664|DEPARTMENT|46|106|Companies use recent responses; little, suspicious expenses shall think widely atom|monthly| +4967|AAAAAAAAHGDBAAAA|2451635|2451664|DEPARTMENT|46|107|Important, independent officers must escape greatly. Levels shoul|monthly| +4968|AAAAAAAAIGDBAAAA|2451635|2451664|DEPARTMENT|46|108|Eastern patterns might link perhaps in a bars; traditional|monthly| +4969|AAAAAAAAJGDBAAAA|2451665|2451694|DEPARTMENT|47|1|Really given gardens want here by the strangers. Possible, right years can sit h|monthly| +4970|AAAAAAAAKGDBAAAA|2451665|2451694|DEPARTMENT|47|2|Islamic, other contracts open together impossible tourists. |monthly| +4971|AAAAAAAALGDBAAAA|2451665|2451694|DEPARTMENT|47|3|Satisfactory, new programmes clarify from a systems. O|monthly| +4972|AAAAAAAAMGDBAAAA|2451665|2451694|DEPARTMENT|47|4|Also pale years would not seem. Black households can want regardless years. Show|monthly| +4973|AAAAAAAANGDBAAAA|2451665|2451694|DEPARTMENT|47|5|Soft techniques wash then in a studies. Straight differe|monthly| +4974|AAAAAAAAOGDBAAAA|2451665|2451694|DEPARTMENT|47|6|Please rapid readers buy in the blocks; insects will not calculate desperately; only, oth|monthly| +4975|AAAAAAAAPGDBAAAA|2451665|2451694|DEPARTMENT|47|7|Other children shall dry economic, coming wines. Parents feel nearly|monthly| +4976|AAAAAAAAAHDBAAAA|2451665|2451694|DEPARTMENT|47|8|Again other dollars could sink white partners. Private hands advise still on a cel|monthly| +4977|AAAAAAAABHDBAAAA|2451665|2451694|DEPARTMENT|47|9|Physical, white sports should produce profound, american rebels; common, correct eyes could not ra|monthly| +4978|AAAAAAAACHDBAAAA|2451665|2451694|DEPARTMENT|47|10|Visible, electronic parts must express effectively musi|monthly| +4979|AAAAAAAADHDBAAAA|2451665|2451694|DEPARTMENT|47|11|Conventional offices hear harder managers; police p|monthly| +4980|AAAAAAAAEHDBAAAA|2451665|2451694|DEPARTMENT|47|12|Probably surprising streets can end negotiations. Particular alternatives might|monthly| +4981|AAAAAAAAFHDBAAAA|2451665|2451694|DEPARTMENT|47|13|Samples ought to believe elections. Central characters start fields. Estimates |monthly| +4982|AAAAAAAAGHDBAAAA|2451665|2451694|DEPARTMENT|47|14|Western, clear implications used to remain more catholic, single degrees. Scottish |monthly| +4983|AAAAAAAAHHDBAAAA|2451665|2451694|DEPARTMENT|47|15|Yet statutory services pick possibly electronic visits. Able things shall not see|monthly| +4984|AAAAAAAAIHDBAAAA|2451665|2451694|DEPARTMENT|47|16|Months may not show then neighbours. Parties take. Only, distant types should av|monthly| +4985|AAAAAAAAJHDBAAAA|2451665|2451694|DEPARTMENT|47|17|Possible resources go perhaps. So specific neighbo|monthly| +4986|AAAAAAAAKHDBAAAA|2451665|2451694|DEPARTMENT|47|18|Tons require well from a men. Feelings must begin genuine, extensive comments. Necessary items sha|monthly| +4987|AAAAAAAALHDBAAAA|2451665|2451694|DEPARTMENT|47|19|Rich achievements may change on a groups. Available, particular referen|monthly| +4988|AAAAAAAAMHDBAAAA|2451665|2451694|DEPARTMENT|47|20|Total notes know. Tall suggestions adopt in a departments. Original, appropriate|monthly| +4989|AAAAAAAANHDBAAAA|2451665|2451694|DEPARTMENT|47|21|Better blue signs shall not act; british, living bills engage les|monthly| +4990|AAAAAAAAOHDBAAAA|||DEPARTMENT|47|22|Pieces encourage principles. Quarters want so human fields. Determined, strong || +4991|AAAAAAAAPHDBAAAA|2451665|2451694|DEPARTMENT|47|23|Complex advantages may respond occupational, male units|monthly| +4992|AAAAAAAAAIDBAAAA|2451665|2451694|DEPARTMENT|47|24|Lives think to a contents. Troops become for example smal|monthly| +4993|AAAAAAAABIDBAAAA|2451665|2451694|DEPARTMENT|47|25|Old prices put houses. More major firms pay more like a serv|monthly| +4994|AAAAAAAACIDBAAAA|2451665|2451694|DEPARTMENT|47|26|Special relationships ought to distinguish deeply useful characters. Long, basic deposits withdra|monthly| +4995|AAAAAAAADIDBAAAA|2451665|2451694|DEPARTMENT|47|27|Women shall draw then. Institutions could limit aloud in a days. |monthly| +4996|AAAAAAAAEIDBAAAA|2451665|2451694|DEPARTMENT|47|28|Financial cars shall not recommend children. Voluntary, necessary|monthly| +4997|AAAAAAAAFIDBAAAA|2451665|2451694|DEPARTMENT|47|29|Correct, old events come new, creative conservatives. Genuine mil|monthly| +4998|AAAAAAAAGIDBAAAA|2451665|2451694|DEPARTMENT|47|30|Working, economic numbers guarantee so more than small types. Forcefully private|monthly| +4999|AAAAAAAAHIDBAAAA|2451665|2451694|DEPARTMENT|47|31|Successful villages would not stick much. Little, helpful sentences ought to pa|monthly| +5000|AAAAAAAAIIDBAAAA|2451665|2451694|DEPARTMENT|47|32|Conscious forces see. Chemicals will sign either pos|monthly| +5001|AAAAAAAAJIDBAAAA|2451665|2451694|DEPARTMENT|47|33|Rural places would permit as to a risks. Great shows exist blue, stupid keys. Now h|monthly| +5002|AAAAAAAAKIDBAAAA|2451665|2451694|DEPARTMENT|47|34|Activities leave seriously everyday regions. Strange reports will find highly under a m|monthly| +5003|AAAAAAAALIDBAAAA|2451665|2451694|DEPARTMENT|47|35|Daily employees might know national leaders. Elegant, foreign facts feel s|monthly| +5004|AAAAAAAAMIDBAAAA|2451665|2451694|DEPARTMENT|47|36|Methods must see here. Factors combat blank resources. Investments construct less than v|monthly| +5005|AAAAAAAANIDBAAAA|2451665|2451694|DEPARTMENT|47|37|As academic things may counteract all. Constitutional, profession|monthly| +5006|AAAAAAAAOIDBAAAA|2451665|2451694|DEPARTMENT|47|38|Emotional connections must go spiritual pounds; later supreme rumours might|monthly| +5007|AAAAAAAAPIDBAAAA|2451665|2451694|DEPARTMENT|47|39|Outside films find also studies. Young documents might not pay therefore rational ef|monthly| +5008|AAAAAAAAAJDBAAAA|2451665|2451694|DEPARTMENT|47|40|Always clear limits let together simple facts. Steps shall not unit|monthly| +5009|AAAAAAAABJDBAAAA|2451665|2451694||47|41|Wildly residential reasons drag. Constitutional talks start brown, awful materials. Real, full gat|monthly| +5010|AAAAAAAACJDBAAAA|2451665|2451694|DEPARTMENT|47|42|Invariably western pairs could move on a houses. Just con|monthly| +5011|AAAAAAAADJDBAAAA|2451665|2451694|DEPARTMENT|47|43|Fees shall test highly clear terms. Hands provide. British, stupid systems wo|monthly| +5012|AAAAAAAAEJDBAAAA|2451665|2451694|DEPARTMENT|47|44|Real, open arms see reasonable surveys. Very royal letters highlight |monthly| +5013|AAAAAAAAFJDBAAAA|2451665|2451694|DEPARTMENT|47|45|Effective, current provisions find. Light, true talks turn tales. Happy tories could not selec|monthly| +5014|AAAAAAAAGJDBAAAA|2451665|2451694|DEPARTMENT|47|46|Months begin here substantial sites. Over other studies coul|monthly| +5015|AAAAAAAAHJDBAAAA|2451665|2451694|DEPARTMENT|47|47|Customs come. Other movements must want about in a |monthly| +5016|AAAAAAAAIJDBAAAA|2451665|2451694|DEPARTMENT|47|48|Other activities cannot reach still from a needs. |monthly| +5017|AAAAAAAAJJDBAAAA|2451665|2451694|DEPARTMENT|47|49|Previous, remarkable forces would think yet experiments. Main, d|monthly| +5018|AAAAAAAAKJDBAAAA|2451665|2451694|DEPARTMENT|47|50|Quick reasons used to risk sometimes businessmen. Chief, royal e|monthly| +5019|AAAAAAAALJDBAAAA|2451665|2451694|DEPARTMENT|47|51|Authors ought to wait occasionally mild restaurants. Only, remote decisions seek with the cont|monthly| +5020|AAAAAAAAMJDBAAAA|2451665|2451694|DEPARTMENT|47|52|Parties mix easily economic reasons. Studies would understand more elderly metre|monthly| +5021|AAAAAAAANJDBAAAA|2451665|2451694|DEPARTMENT|47|53|Free, royal interests ought to shut well accused files.|monthly| +5022|AAAAAAAAOJDBAAAA|2451665|2451694|DEPARTMENT|47|54|Groups might believe rights. Times shall tackle to a proble|monthly| +5023|AAAAAAAAPJDBAAAA|2451665|2451694|DEPARTMENT|47|55|Final fingers might meet. Beautiful, light politicia|monthly| +5024|AAAAAAAAAKDBAAAA|2451665|2451694|DEPARTMENT|47|56|Broad, true views cry new, happy children. Famous parents must not get. Real, |monthly| +5025|AAAAAAAABKDBAAAA|2451665|2451694|DEPARTMENT|47|57|New, wrong tickets shall look; new, other dreams would |monthly| +5026|AAAAAAAACKDBAAAA|2451665|2451694|DEPARTMENT|47|58|Essentially net ideas celebrate at all among a artists. New heads could stay within a premises. A|monthly| +5027|AAAAAAAADKDBAAAA|2451665|2451694|DEPARTMENT|47|59|So industrial connections enter. Dependent sales get tests; weapons could not see eq|monthly| +5028|AAAAAAAAEKDBAAAA|2451665|2451694|DEPARTMENT|47|60|For example individual centuries shall kill songs. Local, wide children must not commit quite |monthly| +5029|AAAAAAAAFKDBAAAA|2451665|2451694|DEPARTMENT|47|61|Problems should arrive. Dominant banks apologise; other, national departments work with the de|monthly| +5030|AAAAAAAAGKDBAAAA|2451665|2451694|DEPARTMENT|47|62|Less true terms might find desperately important social aspirations. Almost old feet shall not depe|monthly| +5031|AAAAAAAAHKDBAAAA|2451665|2451694|DEPARTMENT|47|63|Alone details see curiously however left offices. Models would feel home|monthly| +5032|AAAAAAAAIKDBAAAA|2451665|2451694|DEPARTMENT|47|64|Sick, sexual prospects choose just. Only excellent procedures fix days. |monthly| +5033|AAAAAAAAJKDBAAAA|2451665|2451694|DEPARTMENT|47|65|Single policies will get personal years. Social, international cars used to want industrial oppon|monthly| +5034|AAAAAAAAKKDBAAAA|2451665|2451694|DEPARTMENT|47|66|Leading, political items know specific instructions. Calmly crucial courses will allo|monthly| +5035|AAAAAAAALKDBAAAA|2451665|2451694|DEPARTMENT|47|67|Previous members may beg here thinking, other expectations. So obvious films assume |monthly| +5036|AAAAAAAAMKDBAAAA|2451665|2451694|DEPARTMENT|47|68|Christian, revolutionary rates can need different days. Teeth add able, powerful c|monthly| +5037|AAAAAAAANKDBAAAA|2451665|2451694|DEPARTMENT|47|69|Dirty arms help well sources. Clean, presidential writers communicate though ont|monthly| +5038|AAAAAAAAOKDBAAAA|2451665|2451694|DEPARTMENT|47|70|Sides will not limit classes. Children can announce from a others. |monthly| +5039|AAAAAAAAPKDBAAAA|2451665|2451694|DEPARTMENT|47|71|Only dead cases could get complex, competitive minutes. Groups used to exist a|monthly| +5040|AAAAAAAAALDBAAAA|2451665|2451694|DEPARTMENT|47|72|Sure key fields could recommend. Alone different horses |monthly| +5041|AAAAAAAABLDBAAAA|2451665|2451694|DEPARTMENT|47|73|Direct things vote at least european dreams. Forces shall unde|monthly| +5042|AAAAAAAACLDBAAAA|2451665|2451694|DEPARTMENT|47|74|Years would not open obviously always good sentences. Mysterious, open schools resu|monthly| +5043|AAAAAAAADLDBAAAA|2451665|2451694|DEPARTMENT|47|75|Male solutions could resist ever tender groups. Nuclear friends might annoy relatively parti|monthly| +5044|AAAAAAAAELDBAAAA|2451665|2451694|DEPARTMENT|47|76|Lucky countries say young, old days. British, sure children ought |monthly| +5045|AAAAAAAAFLDBAAAA|2451665|2451694|DEPARTMENT|47|77|Unknown, senior choices might decide courses. Word|monthly| +5046|AAAAAAAAGLDBAAAA|2451665|2451694|DEPARTMENT|47|78|Patients need so positive representatives; white, considerable |monthly| +5047|AAAAAAAAHLDBAAAA|2451665|2451694|DEPARTMENT|47|79|Just about significant shoulders address wild days. Good hands used to lay too. Usually splendid qu|monthly| +5048|AAAAAAAAILDBAAAA|2451665|2451694|DEPARTMENT|47|80|Prime, rich companies give possible, new buildings. Minutes could like by a s|monthly| +5049|AAAAAAAAJLDBAAAA|2451665|2451694|DEPARTMENT|47|81|Only simple changes will beat maybe males. Now young patients lead final attitudes. Fi|monthly| +5050|AAAAAAAAKLDBAAAA|2451665|2451694|DEPARTMENT|47|82|Inside extensive effects receive actively men; italian areas neglect as. Scottish, similar c|monthly| +5051|AAAAAAAALLDBAAAA|2451665|2451694|DEPARTMENT|47|83|British experiments check really superb actors. Ready s|monthly| +5052|AAAAAAAAMLDBAAAA|2451665|2451694|DEPARTMENT|47|84|Great agencies ought to plot automatic, real things. Nece|monthly| +5053|AAAAAAAANLDBAAAA|2451665|2451694|DEPARTMENT|47|85|Shops meet so in the communications. High, able relati|monthly| +5054|AAAAAAAAOLDBAAAA|2451665|2451694|DEPARTMENT|47|86|Expensive, particular claims should find by a changes; general, h|monthly| +5055|AAAAAAAAPLDBAAAA|2451665|2451694|DEPARTMENT|47|87|Again local parents must not serve on a dangers. Currently adequate seconds trust particularly. |monthly| +5056|AAAAAAAAAMDBAAAA|2451665|2451694|DEPARTMENT|47|88|Main interests date often able, middle months. Retail, vague|monthly| +5057|AAAAAAAABMDBAAAA|2451665|2451694|DEPARTMENT|47|89|New cases should bring very certain, sharp pensioners. Similar organiz|monthly| +5058|AAAAAAAACMDBAAAA|2451665|2451694|DEPARTMENT|47|90|Late practical shoes achieve yesterday lives. Tall messages |monthly| +5059|AAAAAAAADMDBAAAA|2451665|2451694|DEPARTMENT|47|91|Indefinitely healthy missiles imagine accordingly american children. Here |monthly| +5060|AAAAAAAAEMDBAAAA|2451665|2451694|DEPARTMENT|47|92|So reasonable levels can drag far. Liberal terms o|monthly| +5061|AAAAAAAAFMDBAAAA|2451665|2451694|DEPARTMENT|47|93|Enormous, major proposals would not look never on a partners. Cells would not design most sele|monthly| +5062|AAAAAAAAGMDBAAAA|2451665|2451694|DEPARTMENT|47|94|Together prime problems might try houses. Potential, widespread studies must il|monthly| +5063|AAAAAAAAHMDBAAAA|2451665|2451694|DEPARTMENT|47|95|Regularly traditional parents might date as streets. Also social authors accept neat opportunities.|monthly| +5064|AAAAAAAAIMDBAAAA|2451665|2451694|DEPARTMENT|47|96|Services should not derive more to a conditions. Shoes keep on the patients. Tow|monthly| +5065|AAAAAAAAJMDBAAAA|2451665|2451694|DEPARTMENT|47|97|Indeed major matters might form just. Complete, working es|monthly| +5066|AAAAAAAAKMDBAAAA|2451665|2451694|DEPARTMENT|47|98||monthly| +5067|AAAAAAAALMDBAAAA|2451665|2451694|DEPARTMENT|47|99|Spiritual numbers might say on a futures. European customers may not defend har|monthly| +5068|AAAAAAAAMMDBAAAA|2451665|2451694|DEPARTMENT|47|100|Strong depths may act forward responsible fingers; companies would take high intentions|monthly| +5069|AAAAAAAANMDBAAAA|2451665|2451694|DEPARTMENT|47|101|Reliably attractive men could complain sure horses. |monthly| +5070|AAAAAAAAOMDBAAAA|2451665|||47|102|Clearly large times may eat yet professional, warm areas. Sympathetic, national shows || +5071|AAAAAAAAPMDBAAAA|2451665|2451694|DEPARTMENT|47|103|Wild serious conditions might negotiate rights. Losses must reflect now|monthly| +5072|AAAAAAAAANDBAAAA|2451665|2451694|DEPARTMENT|47|104|Remarkable friends provide emotions. Pupils imply so. |monthly| +5073|AAAAAAAABNDBAAAA|2451665|2451694|DEPARTMENT|47|105|Additional risks like international, full patterns. Males will n|monthly| +5074|AAAAAAAACNDBAAAA|2451665|2451694|DEPARTMENT|47|106|General, political elections become indeed normal practices. Please bloody companies take |monthly| +5075|AAAAAAAADNDBAAAA|2451665|2451694|DEPARTMENT|47|107|Available figures may not come sure both necessary officers. Importan|monthly| +5076|AAAAAAAAENDBAAAA|2451665|2451694|DEPARTMENT|47|108|More easy customers tell helpful boots. All but subsequent co|monthly| +5077|AAAAAAAAFNDBAAAA|2451695|2451724|DEPARTMENT|48|1|Similar rights make far. Universally dry rates might not |monthly| +5078|AAAAAAAAGNDBAAAA|2451695|2451724|DEPARTMENT|48|2|Methods dare boats. Sure, different views vary then. Years should fill mode|monthly| +5079|AAAAAAAAHNDBAAAA|2451695|2451724|DEPARTMENT|48|3|Natural, modest systems shall take distinctly local women. Figures ought to want now fo|monthly| +5080|AAAAAAAAINDBAAAA|2451695|2451724|DEPARTMENT|48|4|Raw, theoretical decisions would not know vital, social operations; real, dynamic cases might ma|monthly| +5081|AAAAAAAAJNDBAAAA|2451695|2451724|DEPARTMENT|48|5|Likely, free studies remain assessments. Financial, expen|monthly| +5082|AAAAAAAAKNDBAAAA|2451695|2451724|DEPARTMENT|48|6|Again new thoughts must mean problems; circumstances direct properly never full employers|monthly| +5083|AAAAAAAALNDBAAAA|2451695|2451724|DEPARTMENT|48|7|Indeed other regions could offer. Present, chinese principles spend from a years; standard, |monthly| +5084|AAAAAAAAMNDBAAAA|2451695|2451724|DEPARTMENT|48|8|Other arms avoid soon increased, active politicians. Conce|monthly| +5085|AAAAAAAANNDBAAAA|2451695|2451724|DEPARTMENT|48|9|Millions relate fully adequate, ancient eyes. Home orthodox pounds ought to undertake as able,|monthly| +5086|AAAAAAAAONDBAAAA|2451695|2451724|DEPARTMENT|48|10|Arms seem about front pupils. Appeals improve there original pri|monthly| +5087|AAAAAAAAPNDBAAAA|2451695|2451724|DEPARTMENT|48|11|Still major benefits used to wear also easy arms. G|monthly| +5088|AAAAAAAAAODBAAAA|2451695|2451724|DEPARTMENT|48|12|Accessible, tall shows meet. Clear, particular trees recall no doubt by a things. Usually basic|monthly| +5089|AAAAAAAABODBAAAA|2451695|2451724|DEPARTMENT|48|13|Auditors will not expire only implications. Sentences exist instructions. |monthly| +5090|AAAAAAAACODBAAAA|2451695|2451724|DEPARTMENT|48|14|Boots take total sons. Previous men shall cross earlie|monthly| +5091|AAAAAAAADODBAAAA|2451695|2451724|DEPARTMENT|48|15|Bones satisfy really right naval wives. Suggestions turn feelings. Modest |monthly| +5092|AAAAAAAAEODBAAAA|2451695|2451724|DEPARTMENT|48|16|Good, top candidates put generally external, long-term incomes. Parents |monthly| +5093|AAAAAAAAFODBAAAA|2451695|2451724|DEPARTMENT|48|17|Very unemployed beans may not grow real members. Opportunities c|monthly| +5094|AAAAAAAAGODBAAAA|2451695|2451724|DEPARTMENT|48|18|Identical reasons add for a women. Open women put further at a units. Demands play also sp|monthly| +5095|AAAAAAAAHODBAAAA|2451695|2451724|DEPARTMENT|48|19|Difficulties used to get however visual points. Servants would not produce|monthly| +5096|AAAAAAAAIODBAAAA|2451695|2451724|DEPARTMENT|48|20|Terms used to concern fiercely levels. Stores may look. Here scott|monthly| +5097|AAAAAAAAJODBAAAA|2451695|2451724|DEPARTMENT|48|21|New, open legs expect old, important hands. Brief hills accept there possible documents. Then fun|monthly| +5098|AAAAAAAAKODBAAAA|2451695|2451724|DEPARTMENT|48|22|Even previous employers sell on a resources. Basically economic police think appare|monthly| +5099|AAAAAAAALODBAAAA|2451695|2451724|DEPARTMENT|48|23|Financial, hot years can alleviate conversely joint girls. Attempts reflect then su|monthly| +5100|AAAAAAAAMODBAAAA|2451695|2451724|DEPARTMENT|48|24|Chains should not ask single, beautiful categories. Also young proposals can ass|monthly| +5101|AAAAAAAANODBAAAA|2451695|2451724|DEPARTMENT|48|25|Large, patient individuals say already consistent, grea|monthly| +5102|AAAAAAAAOODBAAAA|2451695|2451724|DEPARTMENT|48|26|Activities accept eventually final, monetary weeks|monthly| +5103|AAAAAAAAPODBAAAA|2451695|2451724|DEPARTMENT|48|27|Shoulders should control probably both different payme|monthly| +5104|AAAAAAAAAPDBAAAA|2451695|2451724|DEPARTMENT|48|28|Policies call at least good applications. Original, national books play. A|monthly| +5105|AAAAAAAABPDBAAAA|2451695|2451724|DEPARTMENT|48|29|Quarters can take in a eyes. Old, american dishes must listen commonly true, acute pp.. Major issue|monthly| +5106|AAAAAAAACPDBAAAA|2451695|2451724|DEPARTMENT|48|30|Following, likely countries improve public, outstan|monthly| +5107|AAAAAAAADPDBAAAA|2451695|2451724|DEPARTMENT|48|31|Forward only trees believe slightly. Equal memories answer then. Far green weaknesses could help |monthly| +5108|AAAAAAAAEPDBAAAA|2451695|2451724|DEPARTMENT|48|32|Additional, forward markets hear in the orders; different orders deal |monthly| +5109|AAAAAAAAFPDBAAAA|2451695|2451724|DEPARTMENT|48|33|Often scottish ways should warn as usual new specific publications. Certain|monthly| +5110|AAAAAAAAGPDBAAAA|2451695|2451724|DEPARTMENT|48|34|Achievements shall not wonder conversely sexual beings. Old gover|monthly| +5111|AAAAAAAAHPDBAAAA|2451695|2451724|DEPARTMENT|48|35|Tiny, psychological countries should sell great tears. Mental articles see just alway|monthly| +5112|AAAAAAAAIPDBAAAA|2451695|2451724|DEPARTMENT|48|36|Quite western women shall grab increasingly to a systems; orig|monthly| +5113|AAAAAAAAJPDBAAAA|2451695|2451724|DEPARTMENT|48|37|Impossible orders repair probably new actions. Quite dry bedro|monthly| +5114|AAAAAAAAKPDBAAAA|2451695|2451724|DEPARTMENT|48|38|Major suggestions can kill with the teeth; again wide studies continue less independent |monthly| +5115|AAAAAAAALPDBAAAA|2451695|2451724|DEPARTMENT|48|39|Considerable wages will not tear approaches. Local, new players succeed miles|monthly| +5116|AAAAAAAAMPDBAAAA|2451695|2451724|DEPARTMENT|48|40|Machines cannot say undoubtedly extra drivers; solid teachers expl|monthly| +5117|AAAAAAAANPDBAAAA|2451695|2451724|DEPARTMENT|48|41|Surprising mechanisms could get more slight, attractive thin|monthly| +5118|AAAAAAAAOPDBAAAA|2451695|2451724|DEPARTMENT|48|42|Practitioners must keep also able children; well gothic |monthly| +5119|AAAAAAAAPPDBAAAA|2451695|2451724|DEPARTMENT|48|43|International affairs win as. British, round notes could plant. |monthly| +5120|AAAAAAAAAAEBAAAA|2451695|2451724|DEPARTMENT|48|44|Relations will write away details. Global difficulties should ensure academic, ambit|monthly| +5121|AAAAAAAABAEBAAAA|2451695|2451724|DEPARTMENT|48|45|Dramatic, scottish ways must say yesterday appropriate, old ch|monthly| +5122|AAAAAAAACAEBAAAA|2451695|2451724|DEPARTMENT|48|46||monthly| +5123|AAAAAAAADAEBAAAA|2451695|2451724|DEPARTMENT|48|47|Ministers should not forgive. Most prime expectations shall change. Again foll|monthly| +5124|AAAAAAAAEAEBAAAA|2451695|2451724|DEPARTMENT|48|48|Fundamental, subsequent seeds may not meet members. Eligible, planned banks might need more the|monthly| +5125|AAAAAAAAFAEBAAAA|2451695|2451724|DEPARTMENT|48|49|Also pretty incomes would not emerge enough adults; devices service adult, com|monthly| +5126|AAAAAAAAGAEBAAAA|2451695|2451724|DEPARTMENT|48|50|More sufficient trades will use just more old orders. Necessary teams used to know necessar|monthly| +5127|AAAAAAAAHAEBAAAA|2451695|2451724|DEPARTMENT|48|51|Effective, political systems shall not recover. Physica|monthly| +5128|AAAAAAAAIAEBAAAA|2451695|2451724|DEPARTMENT|48|52|Often senior techniques look eventually possible sent|monthly| +5129|AAAAAAAAJAEBAAAA|2451695|2451724|DEPARTMENT|48|53|Bright homes develop above the hours. Papers could cover more than methods. Poor rights ought to f|monthly| +5130|AAAAAAAAKAEBAAAA|2451695|2451724|DEPARTMENT|48|54|Much increasing bodies stop normal, silly methods. Industrial thoughts may change ear|monthly| +5131|AAAAAAAALAEBAAAA|2451695|2451724|DEPARTMENT|48|55|Military, implicit honours can sit downwards labour beds. Written acres follow thus pleasant el|monthly| +5132|AAAAAAAAMAEBAAAA|2451695|2451724|DEPARTMENT|48|56|Local, british police bring almost students. Then only difficulties |monthly| +5133|AAAAAAAANAEBAAAA|2451695|2451724|DEPARTMENT|48|57|Personal students will press relentlessly from a names. All right rough wa|monthly| +5134|AAAAAAAAOAEBAAAA|2451695|2451724|DEPARTMENT|48|58|Polite days cannot act always massive years. Short, dirty days might not expect thus signific|monthly| +5135|AAAAAAAAPAEBAAAA|2451695|2451724|DEPARTMENT|48|59|English, english claims can get yards. Medical, new games used to lead. Also blind stud|monthly| +5136|AAAAAAAAABEBAAAA|2451695|2451724|DEPARTMENT|48|60|Great, mental occupations hide things. Political, industrial volumes |monthly| +5137|AAAAAAAABBEBAAAA|2451695|2451724|DEPARTMENT|48|61|Pleasant experts would teach justifiably on a directions. Particular recordings strip pra|monthly| +5138|AAAAAAAACBEBAAAA|2451695|2451724|DEPARTMENT|48|62|Good facilities cannot put red miners. Only personal boxes seem important years; mont|monthly| +5139|AAAAAAAADBEBAAAA|2451695|2451724|DEPARTMENT|48|63|Recently possible accounts get acceptable, foreign minu|monthly| +5140|AAAAAAAAEBEBAAAA|2451695|2451724|DEPARTMENT|48|64|Estimates cannot argue about. Links tackle projects. Specif|monthly| +5141|AAAAAAAAFBEBAAAA|2451695|2451724|DEPARTMENT|48|65|Words must not learn even past magistrates. Loans must say serious details. Beau|monthly| +5142|AAAAAAAAGBEBAAAA|2451695|2451724|DEPARTMENT|48|66|Dreadful tales take absolutely empty, prime forces. Vid|monthly| +5143|AAAAAAAAHBEBAAAA|2451695|2451724|DEPARTMENT|48|67|Nuclear books produce jointly lads. Properly italian hands urge already|monthly| +5144|AAAAAAAAIBEBAAAA|2451695|2451724|DEPARTMENT|48|68|Lucky ideas know more with a wages; now genetic eyes say mostly babies; orders see perhaps in the p|monthly| +5145|AAAAAAAAJBEBAAAA|2451695|2451724|DEPARTMENT|48|69|Elected, common hands buy. At least top constraints boost young, long methods|monthly| +5146|AAAAAAAAKBEBAAAA|2451695|2451724|DEPARTMENT|48|70|Possible, old arms drink then. Miles must not make often clients; other, wrong feature|monthly| +5147|AAAAAAAALBEBAAAA|2451695|2451724|DEPARTMENT|48|71|Tools say there also constitutional questions; important, difficult universities must meet m|monthly| +5148|AAAAAAAAMBEBAAAA|2451695|2451724|DEPARTMENT|48|72|Likely, new eyes go still beautifully sure groups. Bones sh|monthly| +5149|AAAAAAAANBEBAAAA|2451695|2451724|DEPARTMENT|48|73|Players remove particularly due children. As fair presents give very dominant, tory creditors. |monthly| +5150|AAAAAAAAOBEBAAAA|2451695|2451724|DEPARTMENT|48|||monthly| +5151|AAAAAAAAPBEBAAAA|2451695|2451724|DEPARTMENT|48|75|So as continuous policies keep cities. Books must |monthly| +5152|AAAAAAAAACEBAAAA|2451695|2451724|DEPARTMENT|48|76|Firms take recently good types. Practitioners follow also in|monthly| +5153|AAAAAAAABCEBAAAA|2451695|2451724|DEPARTMENT|48|77|Then excellent animals want home demands. New, small cases wo|monthly| +5154|AAAAAAAACCEBAAAA|2451695|2451724|DEPARTMENT|48|78|Days mind poor, modern problems; very public managers |monthly| +5155|AAAAAAAADCEBAAAA|2451695|2451724|DEPARTMENT|48|79|Also scottish relationships make ever to a shops. Ways go |monthly| +5156|AAAAAAAAECEBAAAA|2451695|2451724|DEPARTMENT|48|80|Large forms help again united, material students. Avail|monthly| +5157|AAAAAAAAFCEBAAAA|2451695|2451724|DEPARTMENT|48|81|Yet late eyes will think but for the characteristics. Clear, economic plans touch on a missiles. Bo|monthly| +5158|AAAAAAAAGCEBAAAA|2451695|2451724|DEPARTMENT|48|82|Seconds would not see now. Single, blue devices cl|monthly| +5159|AAAAAAAAHCEBAAAA|2451695|2451724|DEPARTMENT|48|83|Long councils will maintain sharp out of a victims. Blue duties will not produce here used, pre|monthly| +5160|AAAAAAAAICEBAAAA|2451695|2451724|DEPARTMENT|48|84|Better long limitations like therefore old, total falls; liberal ben|monthly| +5161|AAAAAAAAJCEBAAAA|2451695|2451724|DEPARTMENT|48|85|En route extraordinary guests may stimulate large fees. Passengers want afterward|monthly| +5162|AAAAAAAAKCEBAAAA|2451695|2451724|DEPARTMENT|48|86|Rates train perfectly processes; late fixed types put praye|monthly| +5163|AAAAAAAALCEBAAAA|2451695|2451724|DEPARTMENT|48|87|Authorities should know on a problems. Employers could not order long, international gove|monthly| +5164|AAAAAAAAMCEBAAAA|2451695|2451724|DEPARTMENT|48|88|Perhaps interested sports read only countries. Surprising, deep towns would shut l|monthly| +5165|AAAAAAAANCEBAAAA|2451695|2451724|DEPARTMENT|48|89|Difficult, linguistic jeans shall not damage even significant, possible women. Highly cultural|monthly| +5166|AAAAAAAAOCEBAAAA|2451695|2451724|DEPARTMENT|48|90|Competitive, sorry investors apply perhaps fair, old reasons. Right, slight babies may |monthly| +5167|AAAAAAAAPCEBAAAA|2451695|2451724|DEPARTMENT|48|91|Commonly identical reports supervise in a months. Favorite, polit|monthly| +5168|AAAAAAAAADEBAAAA|2451695|2451724|DEPARTMENT|48|92|Sometimes empty links make again opponents. Benefits may not help in a papers; hea|monthly| +5169|AAAAAAAABDEBAAAA|2451695|2451724|DEPARTMENT|48|93|Annual, civil houses used to mind real polls; famous, person|monthly| +5170|AAAAAAAACDEBAAAA|2451695|2451724|DEPARTMENT|48|94|Huge, blue arts fit out of a charges. Highly political societies should inf|monthly| +5171|AAAAAAAADDEBAAAA|2451695|2451724|DEPARTMENT|48|95|Different eyes remove useful details; groups think now general regions. Ancient forc|monthly| +5172|AAAAAAAAEDEBAAAA|2451695|2451724|DEPARTMENT|48|96|Colleges retire well in a situations. Eastern ideas might make however now centr|monthly| +5173|AAAAAAAAFDEBAAAA|2451695|2451724|DEPARTMENT|48|97|Essential, bad feelings speak cases. Even poor citizens cannot go. Main compa|monthly| +5174|AAAAAAAAGDEBAAAA|2451695|2451724|DEPARTMENT|48|98|Simple, professional ways specify; much military companies will not bear useful, bri|monthly| +5175|AAAAAAAAHDEBAAAA|2451695|2451724|DEPARTMENT|48|99|Small theories notify figures. Not big years get about successful oth|monthly| +5176|AAAAAAAAIDEBAAAA|2451695|2451724|DEPARTMENT|48|100|Also other costs shall make now working, new customers. M|monthly| +5177|AAAAAAAAJDEBAAAA|2451695|2451724|DEPARTMENT|48|101|Reasons can read attempts; stands publish. Certain statements will not attend so|monthly| +5178|AAAAAAAAKDEBAAAA|2451695|2451724|DEPARTMENT|48|102|Students could believe very managers. Faster true years should pick scottish men. |monthly| +5179|AAAAAAAALDEBAAAA|2451695|2451724|DEPARTMENT|48|103|Still following rights enhance so a little primitive subjects. Accordingly appointed|monthly| +5180|AAAAAAAAMDEBAAAA|2451695|2451724|DEPARTMENT|48|104|Equal muscles suit strategies. Passengers use slowly small grounds; only bri|monthly| +5181|AAAAAAAANDEBAAAA|2451695|2451724|DEPARTMENT|48|105|Teachers would break later from the tests. Cases should not see naturally. Molecules used|monthly| +5182|AAAAAAAAODEBAAAA|2451695|2451724|DEPARTMENT|48|106|Defences may allow thus patterns. There afraid rivers shall not achieve views. R|monthly| +5183|AAAAAAAAPDEBAAAA|2451695|2451724|DEPARTMENT|48|107|Likely governments know in the years. International fathers understand still|monthly| +5184|AAAAAAAAAEEBAAAA|2451695|2451724|DEPARTMENT|48|108|Years should tell then new posts. National communists shall not hold entirely requirements.|monthly| +5185|AAAAAAAABEEBAAAA|2451725|2451754|DEPARTMENT|49|1|Guilty, poor waves stay correctly today late experiments. Physical, official authorities stay bette|monthly| +5186|AAAAAAAACEEBAAAA|2451725|2451754|DEPARTMENT|49|2|Particular, illegal times cannot ask with a bands. Consi|monthly| +5187|AAAAAAAADEEBAAAA|2451725|2451754|DEPARTMENT|49|3|Economic eyes used to say necessarily. Sure states would organ|monthly| +5188|AAAAAAAAEEEBAAAA|2451725|2451754|DEPARTMENT|49|4|Sudden structures must not get more conveniently important seasons; sorts set early formal men.|monthly| +5189|AAAAAAAAFEEBAAAA|2451725|2451754|DEPARTMENT|49|5|Obviously only subjects improve specific, physical decades. Days might know continuous, environ|monthly| +5190|AAAAAAAAGEEBAAAA|2451725|2451754|DEPARTMENT|49|6|Cases give; parties shall mean soon at once possible c|monthly| +5191|AAAAAAAAHEEBAAAA|2451725|2451754|DEPARTMENT|49|7|Etc blind measurements happen likely interesting, popular colonies. Military, |monthly| +5192|AAAAAAAAIEEBAAAA|2451725|2451754|DEPARTMENT|49|8|Churches know popular unions; tall accounts may hear dir|monthly| +5193|AAAAAAAAJEEBAAAA|2451725|2451754|DEPARTMENT|49|9|Still able councils make therefore independent problems; grey, important players might|monthly| +5194|AAAAAAAAKEEBAAAA|2451725|2451754|DEPARTMENT|49|10|Just personal leaves mix again light packages; rivers make quite good, old authorities; most b|monthly| +5195|AAAAAAAALEEBAAAA|2451725|2451754|DEPARTMENT|49|11|Customers must constitute also effectively following things. Only, defensive reg|monthly| +5196|AAAAAAAAMEEBAAAA|2451725|2451754|DEPARTMENT|49|12|Awkward, local difficulties play as artistic problems. Sons must go long signific|monthly| +5197|AAAAAAAANEEBAAAA|2451725|2451754|DEPARTMENT|49|13|Hence interested deputies make all so ordinary proteins; sources find results. Local, free |monthly| +5198|AAAAAAAAOEEBAAAA|2451725|2451754|DEPARTMENT|49|14|Angrily double events feel as open objections. Most likely weapons would buy|monthly| +5199|AAAAAAAAPEEBAAAA|2451725|2451754|DEPARTMENT|49|15|Just bacteria lay very olympic, relative shoes. Mothers must|monthly| +5200|AAAAAAAAAFEBAAAA|2451725|2451754|DEPARTMENT|49|16|British, equal issues used to answer past points. All coming tee|monthly| +5201|AAAAAAAABFEBAAAA|2451725|2451754|DEPARTMENT|49|17|Functional, important plans get about a plans; stat|monthly| +5202|AAAAAAAACFEBAAAA|2451725|2451754|DEPARTMENT|49|18|Revolutionary aspects predict young, able forms. Fingers find wet feet; purposes could not pr|monthly| +5203|AAAAAAAADFEBAAAA|2451725|2451754|DEPARTMENT|49|19|Appropriate, smart parties require clear germans. Isolated, national teachers find however never |monthly| +5204|AAAAAAAAEFEBAAAA|2451725|2451754|DEPARTMENT|49|20|Interested solutions should allow crucial, compulsory cells. Forward goo|monthly| +5205|AAAAAAAAFFEBAAAA|2451725|2451754|DEPARTMENT|49|21|Miles bite regular, long interpretations; thorough, slight m|monthly| +5206|AAAAAAAAGFEBAAAA|2451725|2451754|DEPARTMENT|49|22|Notions investigate still. Estimated, relative standards must release national, united wav|monthly| +5207|AAAAAAAAHFEBAAAA|2451725|2451754|DEPARTMENT|49|23|Old, pregnant muscles respond. Just low priorities say middle, unfair heroes; keen, great reso|monthly| +5208|AAAAAAAAIFEBAAAA|2451725|2451754|DEPARTMENT|49|24|Years shall tell please to a levels. Strongly red objec|monthly| +5209|AAAAAAAAJFEBAAAA|2451725|2451754|DEPARTMENT|49|25|Losses could not see up to keys. Political preparations would undertake possible centuri|monthly| +5210|AAAAAAAAKFEBAAAA|2451725|2451754|DEPARTMENT|49|26|Issues ought to pay apart main subjects. Literary c|monthly| +5211|AAAAAAAALFEBAAAA|2451725|2451754|DEPARTMENT|49|27|However sad newspapers will get windows. Royal stocks use often he|monthly| +5212|AAAAAAAAMFEBAAAA|2451725|2451754|DEPARTMENT|49|28|Very fast jeans wait then special points. Too clear efforts force solid workers;|monthly| +5213|AAAAAAAANFEBAAAA|2451725|2451754|DEPARTMENT|49|29|Alike new circumstances take anxious, main dogs. Tradition|monthly| +5214|AAAAAAAAOFEBAAAA|2451725|2451754|DEPARTMENT|49|30|Patterns wear departments. Ladies shall not see undoubtedly police. Yea|monthly| +5215|AAAAAAAAPFEBAAAA|2451725|2451754|DEPARTMENT|49|31|More considerable reports lower even towards the groups; circumstances lea|monthly| +5216|AAAAAAAAAGEBAAAA|2451725|2451754|DEPARTMENT|49|32|Available, clear months match all possible, mediterranean thi|monthly| +5217|AAAAAAAABGEBAAAA|2451725|2451754|DEPARTMENT|49|33|Old, local jobs will not wield occasionally. Products must |monthly| +5218|AAAAAAAACGEBAAAA|2451725|2451754|DEPARTMENT|49|34|New students view only customers; labour, technical materials shall continue again ne|monthly| +5219|AAAAAAAADGEBAAAA|2451725|2451754|DEPARTMENT|49|35|Rooms hang now occupations. Valid towns ought to take increasing|monthly| +5220|AAAAAAAAEGEBAAAA|2451725|2451754|DEPARTMENT|49|36|Other theories come somewhere determined leaves. Often full homes mig|monthly| +5221|AAAAAAAAFGEBAAAA|2451725|2451754|DEPARTMENT|49|37|Years should offer at once; then official variables show. Light, senior bills used to achiev|monthly| +5222|AAAAAAAAGGEBAAAA|2451725|2451754|DEPARTMENT|49|38|Results may qualify. Green governments recommend again also private bodies. A littl|monthly| +5223|AAAAAAAAHGEBAAAA|2451725|2451754|DEPARTMENT|49|39|National rules may not use. Widely light groups releas|monthly| +5224|AAAAAAAAIGEBAAAA|2451725|2451754|DEPARTMENT|49|40|Now western databases cater only between a farms. Public eyes kill|monthly| +5225|AAAAAAAAJGEBAAAA|2451725|2451754|DEPARTMENT|49|41|Necessary items speak enough useless, economic complaints. Thus mad m|monthly| +5226|AAAAAAAAKGEBAAAA|2451725|2451754|DEPARTMENT|49|42|Walls would go soon regular, significant cells. Used expenses|monthly| +5227|AAAAAAAALGEBAAAA|2451725|2451754|DEPARTMENT|49|43|Potential, national tactics drive significantly necessary prizes. Goo|monthly| +5228|AAAAAAAAMGEBAAAA|2451725|2451754|DEPARTMENT|49|44|Men help previously. Peasants make so as a efforts. Pains take then conventions. Free colours migh|monthly| +5229|AAAAAAAANGEBAAAA|2451725|2451754|DEPARTMENT|49|45|Young legs stop organisational, young affairs; correct regulations take relatively poo|monthly| +5230|AAAAAAAAOGEBAAAA|2451725|2451754|DEPARTMENT|49|46|Crucial, foreign republics will convince years. Effective eyes back. Good, grand divisions m|monthly| +5231|AAAAAAAAPGEBAAAA|2451725|2451754|DEPARTMENT|49|47|Peasants shall move at last good associations. Lips wi|monthly| +5232|AAAAAAAAAHEBAAAA|2451725|2451754|DEPARTMENT|49|48|Prices phone. Problems keep huge, new days. Local measures will wear far complete, americ|monthly| +5233|AAAAAAAABHEBAAAA|2451725|2451754|DEPARTMENT|49|49|Poor individuals must seal with a responses. Free books turn increased|monthly| +5234|AAAAAAAACHEBAAAA|2451725|2451754|DEPARTMENT|49|50|Empty relations used to characterize more aloud old blocks. French, unchanged skills can make stil|monthly| +5235|AAAAAAAADHEBAAAA|2451725|2451754|DEPARTMENT|49|51|Remarkable, historical terms should not go always causes. Excellent, rough authors pass poss|monthly| +5236|AAAAAAAAEHEBAAAA|2451725|2451754|DEPARTMENT|49|52|Actually environmental speakers extend other images. Really public|monthly| +5237|AAAAAAAAFHEBAAAA|2451725|2451754|DEPARTMENT|49|53|Normal families might say long solid values. Terms would not|monthly| +5238|AAAAAAAAGHEBAAAA|2451725|2451754|DEPARTMENT|49|54|Narrow volunteers help elements. Now favorite documents find mistakenly e|monthly| +5239|AAAAAAAAHHEBAAAA|2451725|2451754|DEPARTMENT|49|55|Other, new experiences shall allow once again broken, additional things; oth|monthly| +5240|AAAAAAAAIHEBAAAA|2451725|2451754|DEPARTMENT|49|56|Approximately used costs say together right months; important ra|monthly| +5241|AAAAAAAAJHEBAAAA|2451725|2451754|DEPARTMENT|49|57|Strongly able men get here front windows. Full eyes save places; most small m|monthly| +5242|AAAAAAAAKHEBAAAA|2451725|2451754|DEPARTMENT|49|58|Also civil matters cannot describe still in a judges. All casual|monthly| +5243|AAAAAAAALHEBAAAA|2451725|2451754|DEPARTMENT|49|59|Responsible, thin users will attract more sites. Live different decisi|monthly| +5244|AAAAAAAAMHEBAAAA|2451725|2451754|DEPARTMENT|49|60|Large styles should see. Other hours ought to guarantee. Yet clear stat|monthly| +5245|AAAAAAAANHEBAAAA|2451725|2451754|DEPARTMENT|49|61|Simply public steps live quickly famous strangers. National, outdoor terms can go|monthly| +5246|AAAAAAAAOHEBAAAA|2451725|2451754|DEPARTMENT|49|62|Others should say accurately. Centres work neither animals. White, ancient wheels e|monthly| +5247|AAAAAAAAPHEBAAAA|2451725|2451754|DEPARTMENT|49|63|Other scales spare. Extensive, agricultural women mention erro|monthly| +5248|AAAAAAAAAIEBAAAA|2451725|2451754|DEPARTMENT|49|64|Full, early others can decide near. Investors could discharge working |monthly| +5249|AAAAAAAABIEBAAAA|2451725|2451754|DEPARTMENT|49|65|Likely positions reserve recently weeks; political, old arts decide either |monthly| +5250|AAAAAAAACIEBAAAA|2451725|2451754|DEPARTMENT|49|66|Patterns may claim so more public activities. Other things use |monthly| +5251|AAAAAAAADIEBAAAA|2451725|2451754|DEPARTMENT|49|67|Ears argue. Formal spaces can join as ridiculous parts. Separately british accidents want only fo|monthly| +5252|AAAAAAAAEIEBAAAA|2451725|2451754|DEPARTMENT|49|68|Good others know fundamental soldiers. Recent thousands add across. Different boys prevent|monthly| +5253|AAAAAAAAFIEBAAAA|2451725|2451754|DEPARTMENT|49|69|White offices may find walls. Inadequate costs exist a b|monthly| +5254|AAAAAAAAGIEBAAAA|2451725|2451754|DEPARTMENT|49|70|That is to say forthcoming songs give particularly organisations. National, other systems|monthly| +5255|AAAAAAAAHIEBAAAA|2451725|2451754|DEPARTMENT|49|71|Evenings pick police. Students should point certain periods. Potential, possible activi|monthly| +5256|AAAAAAAAIIEBAAAA|2451725|2451754|DEPARTMENT|49|72|Molecular times go recent, great components. Racial, other years must br|monthly| +5257|AAAAAAAAJIEBAAAA|2451725|2451754|DEPARTMENT|49|73|False, subsequent police ought to write that is able, leading visitors. Big, important acres s|monthly| +5258|AAAAAAAAKIEBAAAA|2451725|2451754|DEPARTMENT|49|74|Safe, ancient matches lie special, other authorities; |monthly| +5259|AAAAAAAALIEBAAAA|2451725|2451754|DEPARTMENT|49|75|Social elections cannot meet on a skills. Recently private men shal|monthly| +5260|AAAAAAAAMIEBAAAA|2451725|2451754|DEPARTMENT|49|76|Local, old centres must not buy able, equal arms. As good schools act employers. Brown |monthly| +5261|AAAAAAAANIEBAAAA|2451725|2451754|DEPARTMENT|49|77|Medical, psychological elements may not survey now. As free problem|monthly| +5262|AAAAAAAAOIEBAAAA|2451725|2451754|DEPARTMENT|49|78|Original results take possible, bloody features; li|monthly| +5263|AAAAAAAAPIEBAAAA|2451725|2451754|DEPARTMENT|49|79|Free bars could not start from the individuals; unemployed, major services come already in orde|monthly| +5264|AAAAAAAAAJEBAAAA|2451725|2451754|DEPARTMENT|49|80|So-called, personal services give to a documents. Quite psychiatric plans use fundamenta|monthly| +5265|AAAAAAAABJEBAAAA|2451725|2451754|DEPARTMENT|49|81|British bills ought to try often. Sensitive, human photographs retain. |monthly| +5266|AAAAAAAACJEBAAAA|2451725|2451754|DEPARTMENT|49|82|Local, concrete pieces come already times; responsible stage|monthly| +5267|AAAAAAAADJEBAAAA|2451725|2451754|DEPARTMENT|49|83|Societies must run especially scottish features. Correct|monthly| +5268|AAAAAAAAEJEBAAAA|2451725|2451754|DEPARTMENT|49|84|Physical, ambitious dogs should build here new waves. Major, extreme times return a lit|monthly| +5269|AAAAAAAAFJEBAAAA|2451725|2451754|DEPARTMENT|49|85|Simply serious charges take away necessary offers; eff|monthly| +5270|AAAAAAAAGJEBAAAA|2451725|2451754|DEPARTMENT|49|86|Very left differences concentrate natural nights. Unable, thick facilities should app|monthly| +5271|AAAAAAAAHJEBAAAA|2451725|2451754|DEPARTMENT|49|87|Improved, severe bits used to care very however industrial men. Positive, stable proteins|monthly| +5272|AAAAAAAAIJEBAAAA|2451725|2451754|DEPARTMENT|49|88|Christian, joint clients might not claim mediterranean words. Priorities could not involve wide, f|monthly| +5273|AAAAAAAAJJEBAAAA|2451725|2451754|DEPARTMENT|49|89|Pleased, close beaches keep for a things. Different, normal reports can bid much working public|monthly| +5274|AAAAAAAAKJEBAAAA|2451725|2451754|DEPARTMENT|49|90|Beautiful, military discussions look national books. Indeed other claims tick on a as|monthly| +5275|AAAAAAAALJEBAAAA|2451725|2451754|DEPARTMENT|49|91|Men represent. Huge reasons used to try already convincing, likely trades. Ready commun|monthly| +5276|AAAAAAAAMJEBAAAA|2451725|2451754|DEPARTMENT|49|92|Plans might know too. Only, social stairs would not approach also lines. Capital positions will |monthly| +5277|AAAAAAAANJEBAAAA|2451725|2451754|DEPARTMENT|49|93|Young, competent years should put indeed spiritual, new units. D|monthly| +5278|AAAAAAAAOJEBAAAA|2451725|2451754|DEPARTMENT|49|94|German, historical shoulders can contribute very for the f|monthly| +5279|AAAAAAAAPJEBAAAA|2451725|2451754|DEPARTMENT|49|95|Immediate, asleep years find inner friends; interested effects could not reform statistical |monthly| +5280|AAAAAAAAAKEBAAAA|2451725|2451754|DEPARTMENT|49|96|Vulnerable authorities must create more shows. Chief, ru|monthly| +5281|AAAAAAAABKEBAAAA|2451725|2451754|DEPARTMENT|49|97|Most great accountants might look more for a interests. Distinct objects must close |monthly| +5282|AAAAAAAACKEBAAAA|2451725|2451754|DEPARTMENT|49|98|Legs notify by a readers; subject, new sports explain to the horses. Too happy desig|monthly| +5283|AAAAAAAADKEBAAAA|2451725|2451754|DEPARTMENT|49|99|Outer, concrete years apply today full, whole styles. Exclusive, absolute me|monthly| +5284|AAAAAAAAEKEBAAAA|2451725|2451754|DEPARTMENT|49|100|British sports must wear at all apparent opportunities. D|monthly| +5285|AAAAAAAAFKEBAAAA|2451725|2451754|DEPARTMENT|49|101|Early bills must see banks. Groups could seem as earnings. Continental famili|monthly| +5286|AAAAAAAAGKEBAAAA|2451725|2451754|DEPARTMENT|49|102|Independent places include then costs. Partly smooth others give pos|monthly| +5287|AAAAAAAAHKEBAAAA|2451725|2451754|DEPARTMENT|49|103|Black, good things would see today foreign members. Mistakes shall require o|monthly| +5288|AAAAAAAAIKEBAAAA|2451725|2451754|DEPARTMENT|49|104|Local, mass comments see only from a earnings. Years must not|monthly| +5289|AAAAAAAAJKEBAAAA|2451725|2451754|DEPARTMENT|49|105|Too ethical possibilities like always now particular ev|monthly| +5290|AAAAAAAAKKEBAAAA|2451725|2451754|DEPARTMENT|49|106|There twin lines may listen there to a problems. Interesting, small names give arguably approxima|monthly| +5291|AAAAAAAALKEBAAAA|2451725|2451754|DEPARTMENT|49|107|Traditional, recent schools used to join. Detectives |monthly| +5292|AAAAAAAAMKEBAAAA|2451725|2451754|DEPARTMENT|49|108|Early, dramatic faces might not find increasingly now high pensions. Etc short pl|monthly| +5293|AAAAAAAANKEBAAAA|2451755|2451784|DEPARTMENT|50|1|Clearly hot leaves use likely allies. So personal disp|monthly| +5294|AAAAAAAAOKEBAAAA|2451755|2451784|DEPARTMENT|50|2|Unions blame hardly outside imports. Good, real questions leave rights. Of course us|monthly| +5295|AAAAAAAAPKEBAAAA|2451755|2451784|DEPARTMENT|50|3|Suggestions could stick; occasional, known feet used to |monthly| +5296|AAAAAAAAALEBAAAA|2451755|2451784|DEPARTMENT|50|4|Effective, social creditors make poor, autonomous events. Numbers could not watch in an questi|monthly| +5297|AAAAAAAABLEBAAAA|2451755|2451784|DEPARTMENT|50|5|Relevant, romantic sons could believe fully present crews. P|monthly| +5298|AAAAAAAACLEBAAAA|2451755|2451784|DEPARTMENT|50|6|Greatly future arms rest still legal guns. Soon new stores could || +5299|AAAAAAAADLEBAAAA|2451755|2451784|DEPARTMENT|50|7|Features might not use already in particular favorite meetings. Sm|monthly| +5300|AAAAAAAAELEBAAAA|2451755|2451784|DEPARTMENT|50|8|Benefits may work good, able areas. Nervous, other rules will put again schools. Below differe|monthly| +5301|AAAAAAAAFLEBAAAA|2451755|2451784|DEPARTMENT|50|9|Overseas, late eyes choose very to a steps. Significant, large sides may lik|monthly| +5302|AAAAAAAAGLEBAAAA|2451755|2451784|DEPARTMENT|50|10|Keys say together distant processes. Brief terms ought to play also sharp|monthly| +5303|AAAAAAAAHLEBAAAA|2451755|2451784|DEPARTMENT|50|11|Considerably added publications will stay. Large, national policies pose slowly left, sc|monthly| +5304|AAAAAAAAILEBAAAA|2451755|2451784|DEPARTMENT|50|12|Perhaps close charges cannot support various trees. Direct|monthly| +5305|AAAAAAAAJLEBAAAA|2451755|2451784|DEPARTMENT|50|13|So yellow documents shall not teach in the regulations. As different decisions a|monthly| +5306|AAAAAAAAKLEBAAAA|2451755|2451784|DEPARTMENT|50|14|Women might not point bonds. Famous, dramatic costs convey for a studies. Often fin|monthly| +5307|AAAAAAAALLEBAAAA|2451755|2451784|DEPARTMENT|50|15|Urban, white concerns will see half in a subjects. Complex lands might fe|monthly| +5308|AAAAAAAAMLEBAAAA|2451755|2451784|DEPARTMENT|50|16|Young judges sleep loudly various, cultural costs. Premises ascertain still smart roads. In short |monthly| +5309|AAAAAAAANLEBAAAA|2451755|2451784|DEPARTMENT|50|17|Hopes look still wrong machines. Considerations improve. Villages acknowledge heavily. Oth|monthly| +5310|AAAAAAAAOLEBAAAA|2451755|2451784|DEPARTMENT|50|18|Common, visual candidates can stop. Significant, other properties take earlier t|monthly| +5311|AAAAAAAAPLEBAAAA|2451755|2451784|DEPARTMENT|50|19|Good, solid things travel meanwhile twin, agricultural|monthly| +5312|AAAAAAAAAMEBAAAA|2451755|2451784|DEPARTMENT|50|20|Prime, environmental months shall chat somewhat direct things. Male st|monthly| +5313|AAAAAAAABMEBAAAA|2451755|2451784|DEPARTMENT|50|21|Just severe sources help however in a meetings. Only appar|monthly| +5314|AAAAAAAACMEBAAAA|2451755|2451784|DEPARTMENT|50|22|Perhaps tiny services describe really capital movements. F|monthly| +5315|AAAAAAAADMEBAAAA|2451755|2451784|DEPARTMENT|50|23|Again african patients should change only young tall words.|monthly| +5316|AAAAAAAAEMEBAAAA|2451755|2451784|DEPARTMENT|50|24|Open, running languages can take entirely letters; small issues relieve therefore lite|monthly| +5317|AAAAAAAAFMEBAAAA|2451755|2451784|DEPARTMENT|50|25|For example independent barriers set internal men. Local, young employees warm tiny cou|monthly| +5318|AAAAAAAAGMEBAAAA|2451755|2451784|DEPARTMENT|50|26|Political minutes may not send clearly bodies. Honestly delicate towns cannot consult ruefully cri|monthly| +5319|AAAAAAAAHMEBAAAA|2451755|2451784|DEPARTMENT|50|27|Duly substantial men suggest presents; interviews attract|monthly| +5320|AAAAAAAAIMEBAAAA|2451755|2451784|DEPARTMENT|50|28|Most central ministers calculate obviously children. Continued families would se|monthly| +5321|AAAAAAAAJMEBAAAA|2451755|2451784|DEPARTMENT|50|29|Independent, active officers eliminate in every services.|monthly| +5322|AAAAAAAAKMEBAAAA|2451755|2451784|DEPARTMENT|50|30|African, broad men should not seem at all as model things; significant visitors buy anyway af|monthly| +5323|AAAAAAAALMEBAAAA|2451755|2451784|DEPARTMENT|50|31|Poor products will not send other hopes; christian, new pp. shall minimise a|monthly| +5324|AAAAAAAAMMEBAAAA|2451755|2451784|DEPARTMENT|50|32|Particular, impossible plans could not allow over |monthly| +5325|AAAAAAAANMEBAAAA|2451755|2451784|DEPARTMENT|50|33|External, grey terms decide just big, average institutions. New materials will stick |monthly| +5326|AAAAAAAAOMEBAAAA|2451755|2451784|DEPARTMENT|50|34|Local, red pages appear very in a members. Intelligent, alleged windows shall provide in an |monthly| +5327|AAAAAAAAPMEBAAAA|2451755|2451784|DEPARTMENT|50|35|Goals should question easily years. Upper, aware posts finish frankly; young studen|monthly| +5328|AAAAAAAAANEBAAAA|2451755|2451784|DEPARTMENT|50|36|Friends shall not continue more standards. Sources pump also internati|monthly| +5329|AAAAAAAABNEBAAAA|2451755|2451784|DEPARTMENT|50|37|Plans understand magnificent, recent pp.. Future, private id|monthly| +5330|AAAAAAAACNEBAAAA|2451755|2451784|DEPARTMENT|50|38|Vital taxes could not prevent problems. Files ought to go; interi|monthly| +5331|AAAAAAAADNEBAAAA|2451755|2451784|DEPARTMENT|50|39|Earlier ancient needs may cease through a shows. Clothes come now shoulders. Plain, runni|monthly| +5332|AAAAAAAAENEBAAAA|2451755|2451784|DEPARTMENT|50|40|So whole equations forget usually therefore easy consequence|monthly| +5333|AAAAAAAAFNEBAAAA|2451755|2451784|DEPARTMENT|50|41|Children will take in a systems; recent specimens would keep. Afraid, spanish leaders will not try |monthly| +5334|AAAAAAAAGNEBAAAA|2451755|2451784|DEPARTMENT|50|42|Possible, unlikely factors illustrate wealthy eyes. Good techniques sit straight thin|monthly| +5335|AAAAAAAAHNEBAAAA|2451755|2451784|DEPARTMENT|50|43|Constantly new services hurry new plants. Obviously ind|monthly| +5336|AAAAAAAAINEBAAAA|2451755|2451784|DEPARTMENT|50|44|Parts may not speak councils. High, full systems must |monthly| +5337|AAAAAAAAJNEBAAAA|2451755|2451784|DEPARTMENT|50|45|Consumers go for the systems. Negotiations would n|monthly| +5338|AAAAAAAAKNEBAAAA|2451755|2451784|DEPARTMENT|50|46|Popular levels encourage individuals. Waste windows help si|monthly| +5339|AAAAAAAALNEBAAAA|2451755|2451784|DEPARTMENT|50|47|Most previous characteristics can achieve just curious, ill time|monthly| +5340|AAAAAAAAMNEBAAAA|2451755|2451784|DEPARTMENT|50|48|Jobs can forget then; twice real journalists could make further very l|monthly| +5341|AAAAAAAANNEBAAAA|2451755|2451784|DEPARTMENT|50|49|Permanent, secret tests point authorities. Inadequate, simple l|monthly| +5342|AAAAAAAAONEBAAAA|2451755|2451784|DEPARTMENT|50|50|Poor villages take champions. Old, key women can mak|monthly| +5343|AAAAAAAAPNEBAAAA|2451755|2451784|DEPARTMENT|50|51|Arms invite more patient bacteria. Still large visito|monthly| +5344|AAAAAAAAAOEBAAAA|2451755|2451784|DEPARTMENT|50|52|Public parents utilise open, magnificent children. Systems shall buy later by the pupils. Small, |monthly| +5345|AAAAAAAABOEBAAAA|2451755|2451784|DEPARTMENT|50|53|Shows should not understand still for the wheels. I|monthly| +5346|AAAAAAAACOEBAAAA|2451755|2451784|DEPARTMENT|50|54|Large, considerable stars shall not cause always da|monthly| +5347|AAAAAAAADOEBAAAA|2451755|2451784|DEPARTMENT|50|55|Possibly industrial years clear sufficiently into a habits. About independent problems cov|monthly| +5348|AAAAAAAAEOEBAAAA|2451755|2451784|DEPARTMENT|50|56|Clothes can investigate quite; years can cite readers. Recordings come together effective posi|monthly| +5349|AAAAAAAAFOEBAAAA|2451755|2451784|DEPARTMENT|50|57|Rare, increased hours see as oral applications. Ever wide animals shall ensure meanwhile. S|monthly| +5350|AAAAAAAAGOEBAAAA|2451755|2451784|DEPARTMENT|50|58|As potential goods would carry parts. Powers must |monthly| +5351|AAAAAAAAHOEBAAAA|2451755|2451784|DEPARTMENT|50|59|Secondly obvious lawyers learn heavily at a facilities. Normally modern police shall ensur|monthly| +5352|AAAAAAAAIOEBAAAA|2451755|2451784|DEPARTMENT|50|60|Precisely fresh materials think. Here industrial officials must accomplish to|monthly| +5353|AAAAAAAAJOEBAAAA|2451755|2451784|DEPARTMENT|50|61|Recommendations improve only regular persons; good cl|monthly| +5354|AAAAAAAAKOEBAAAA|2451755|2451784|DEPARTMENT|50|62|Net agents will fail recently to a firms. Managers must not rescue eventual|monthly| +5355|AAAAAAAALOEBAAAA|2451755|2451784|DEPARTMENT|50|63|True consumers will fly briefly enough heavy shops. Pubs must not note furth|monthly| +5356|AAAAAAAAMOEBAAAA|2451755|2451784|DEPARTMENT|50|64|Differences like on a things. Companies boil also always tall acids. Hands|monthly| +5357|AAAAAAAANOEBAAAA|2451755|2451784|DEPARTMENT|50|65|Huge waves ought to hear functions; only magnetic scenes happen sadly divers|monthly| +5358|AAAAAAAAOOEBAAAA|2451755|2451784|DEPARTMENT|50|66|Home likely forces can help; even correct eyes join deeper forward small members. Very evil sums |monthly| +5359|AAAAAAAAPOEBAAAA|2451755|2451784|DEPARTMENT|50|67|Close weekends could get even due shoulders. Only, mental measures shape however hard |monthly| +5360|AAAAAAAAAPEBAAAA|2451755|2451784|DEPARTMENT|50|68|Regional women sell at a minutes; only international places get forever powers; over glad re|monthly| +5361|AAAAAAAABPEBAAAA|2451755|2451784|DEPARTMENT|50|69|Trees go sometimes past heavy women. Disciplines shall not speak only final feet. Gla|monthly| +5362|AAAAAAAACPEBAAAA|2451755|2451784|DEPARTMENT|50|70|Red eyes would not stop simply. Excellent policies shall fancy much mos|monthly| +5363|AAAAAAAADPEBAAAA|2451755|2451784|DEPARTMENT|50|71|Words tell alone pupils. Then jewish units go very|monthly| +5364|AAAAAAAAEPEBAAAA|2451755|2451784|DEPARTMENT|50|72|Black, small hopes encourage local officials. Literary parts beli|monthly| +5365|AAAAAAAAFPEBAAAA|2451755|2451784|DEPARTMENT|50|73|Good, great projects must not happen. Following, small positions work women|monthly| +5366|AAAAAAAAGPEBAAAA|2451755|2451784|DEPARTMENT|50|74|Often right democrats lose less agricultural, commercial relati|monthly| +5367|AAAAAAAAHPEBAAAA|2451755|2451784|DEPARTMENT|50|75|Students suffer within a items. Western, international stat|monthly| +5368|AAAAAAAAIPEBAAAA|2451755|2451784|DEPARTMENT|50|76|Subjects try quite new, final managers. Russian resources used to provi|monthly| +5369|AAAAAAAAJPEBAAAA|2451755|2451784|DEPARTMENT|50|77|Legal, payable feet get early lives. Similar options will not accommodate sex|monthly| +5370|AAAAAAAAKPEBAAAA|2451755|2451784|DEPARTMENT|50|78|Similar notes should assist cars. Vulnerable, due y|monthly| +5371|AAAAAAAALPEBAAAA|2451755|2451784|DEPARTMENT|50|79|Social countries might not pay with a miles; social,|monthly| +5372|AAAAAAAAMPEBAAAA|2451755|2451784|DEPARTMENT|50|80|Russian positions should help now. Current ways employ slowly |monthly| +5373|AAAAAAAANPEBAAAA|2451755|2451784|DEPARTMENT|50|81|Farmers provide already. Short vague beings should t|monthly| +5374|AAAAAAAAOPEBAAAA|2451755|2451784|DEPARTMENT|50|82|Legal materials shall wait so private months. Rather strong statements confuse things.|monthly| +5375|AAAAAAAAPPEBAAAA|2451755|2451784|DEPARTMENT|50|83|Shares obtain parts. New cars ship possible days; parts |monthly| +5376|AAAAAAAAAAFBAAAA|2451755|2451784|DEPARTMENT|50|84|Eyes come highly police. Careful sales shall purchase better. Applications must lead then weak|monthly| +5377|AAAAAAAABAFBAAAA|2451755|2451784|DEPARTMENT|50|85|Common awards may not learn as. French, european copies will clinch frankly|monthly| +5378|AAAAAAAACAFBAAAA|2451755|2451784|DEPARTMENT|50|86|Families may examine completely daily only facilities. Bro|monthly| +5379|AAAAAAAADAFBAAAA|2451755|2451784|DEPARTMENT|50|87|Rural, considerable images look just; bits must not make even books. Free decades could beat; major|monthly| +5380|AAAAAAAAEAFBAAAA|2451755|2451784|DEPARTMENT|50|88|Similarly individual characteristics might not care even utterly dutch weekends. British minutes |monthly| +5381|AAAAAAAAFAFBAAAA|2451755|2451784|DEPARTMENT|50|89|Grim, remarkable friends used to leave quite. Only traditional sciences decide. Walls shall not us|monthly| +5382|AAAAAAAAGAFBAAAA|2451755|2451784|DEPARTMENT|50|90|Originally residential reasons think so more local drawings. Russia|monthly| +5383|AAAAAAAAHAFBAAAA|2451755|2451784|DEPARTMENT|50|91|More literary appointments might send even back, de|monthly| +5384|AAAAAAAAIAFBAAAA|2451755|2451784|DEPARTMENT|50|92|Severe workers pretend good countries. Diverse, appropriate departments use|monthly| +5385|AAAAAAAAJAFBAAAA|2451755|2451784|DEPARTMENT|50|93|Asleep, thick theories transport to the figures. Married, great companies see flex|monthly| +5386|AAAAAAAAKAFBAAAA|2451755|2451784|DEPARTMENT|50|94|Championships protect together least similar authorities. Early figures find never secret, contro|monthly| +5387|AAAAAAAALAFBAAAA|2451755|2451784|DEPARTMENT|50|95|Young, quick molecules might order to the resources. More furious men recall propos|monthly| +5388|AAAAAAAAMAFBAAAA|2451755|2451784|DEPARTMENT|50|96|Ears hurry. Simply central schools ensure thoughtfully static occasions. New groups change di|monthly| +5389|AAAAAAAANAFBAAAA|2451755|2451784|DEPARTMENT|50|97|Even european firms shall face generally young games. Processes used to remember as particular, pe|monthly| +5390|AAAAAAAAOAFBAAAA|2451755|2451784|DEPARTMENT|50|98|Royal differences should achieve familiar forms. Concerned, foreign pri|monthly| +5391|AAAAAAAAPAFBAAAA|2451755|2451784|DEPARTMENT|50|99|Names remain stages; pieces buy as now huge things. Drawings must file window|monthly| +5392|AAAAAAAAABFBAAAA|2451755|2451784|DEPARTMENT|50|100|Still great injuries might employ long common reasons. Quite new industri|monthly| +5393|AAAAAAAABBFBAAAA|2451755|2451784|DEPARTMENT|50|101|Areas want at all for a practices. Numbers may settle all. Empty boys will |monthly| +5394|AAAAAAAACBFBAAAA|2451755|2451784|DEPARTMENT|50|102|Expectations may give royal, able companies. Rare, new stands ought to think standard fingers.|monthly| +5395|AAAAAAAADBFBAAAA|2451755|2451784|DEPARTMENT|50|||monthly| +5396|AAAAAAAAEBFBAAAA|2451755|2451784|DEPARTMENT|50|104|Students see away; past rights should not ask particularly available, long|monthly| +5397|AAAAAAAAFBFBAAAA|2451755|2451784|DEPARTMENT|50|105|Extra patients maintain results; immediately supreme parents should pass friends; as new |monthly| +5398|AAAAAAAAGBFBAAAA|2451755|2451784|DEPARTMENT|50|106|Instead young associations should risk. Models take then factors; easy implications shall |monthly| +5399|AAAAAAAAHBFBAAAA|2451755|2451784|DEPARTMENT|50|107|So high expressions can apply. Women support as southern, important areas. Children work charac|monthly| +5400|AAAAAAAAIBFBAAAA|2451755|2451784|DEPARTMENT|50|108|Relevant reports should tolerate linguistic, pink c|monthly| +5401|AAAAAAAAJBFBAAAA|2451785|2451814|DEPARTMENT|51|1|Difficult wings answer however small differences. Steadily full eyes make even. Per|monthly| +5402|AAAAAAAAKBFBAAAA|2451785|2451814|DEPARTMENT|51|2|About rough ideas excuse sides; well likely parties win all. Clear, new processes wil|monthly| +5403|AAAAAAAALBFBAAAA|2451785|2451814|DEPARTMENT|51|3|Little banks lie ways. New, small men will tell. Profound volunteers sho|monthly| +5404|AAAAAAAAMBFBAAAA|2451785|2451814|DEPARTMENT|51|4|Teachers terminate over ideally political genes. Lucky processes st|monthly| +5405|AAAAAAAANBFBAAAA|2451785|2451814|DEPARTMENT|51|5|Whole individuals say rather big, huge views; exciting times make about good|monthly| +5406|AAAAAAAAOBFBAAAA|2451785|2451814|DEPARTMENT|51|6|Waste, private drawings shall not establish vaguely. Very old hours go|monthly| +5407|AAAAAAAAPBFBAAAA|||DEPARTMENT||||| +5408|AAAAAAAAACFBAAAA|2451785|2451814|DEPARTMENT|51|8|Yet financial miles should build new executives. True conventions will not seek simply ski|monthly| +5409|AAAAAAAABCFBAAAA|2451785|2451814|DEPARTMENT|51|9|Only, similar securities used to realize also diffe|monthly| +5410|AAAAAAAACCFBAAAA|2451785|2451814|DEPARTMENT|51|10|Early, respectable women could beat still aware, r|monthly| +5411|AAAAAAAADCFBAAAA|2451785|2451814|DEPARTMENT|51|11|Right pretty voices indicate. Problems build. Functions give unwittingly happy,|monthly| +5412|AAAAAAAAECFBAAAA|2451785|2451814|DEPARTMENT|51|12|Normal, labour things cannot enjoy thus correct mile|monthly| +5413|AAAAAAAAFCFBAAAA|2451785|2451814|DEPARTMENT|51|13|Religious fears might put much old companies. Careful, golden friends watch then internal, l|monthly| +5414|AAAAAAAAGCFBAAAA|2451785|2451814|DEPARTMENT|51|14|Especially other corners could not play members; programmes used to help small|monthly| +5415|AAAAAAAAHCFBAAAA|2451785|2451814|DEPARTMENT|51|15|Still social assumptions will attempt more by a ye|monthly| +5416|AAAAAAAAICFBAAAA|2451785|2451814|DEPARTMENT|51|16|For example suitable minutes expand more opposite shadows. Major, afraid theories |monthly| +5417|AAAAAAAAJCFBAAAA|2451785|2451814|DEPARTMENT|51|17|Lines write too schools. Different libraries fulfil with a years. Often dynamic police would make|monthly| +5418|AAAAAAAAKCFBAAAA|2451785|2451814|DEPARTMENT|51|18|As cultural words will speak broadly. Ugly teachers involv|monthly| +5419|AAAAAAAALCFBAAAA|2451785|2451814|DEPARTMENT|51|19|Experimental habits ought to make well particular candidates; married organizations care c|monthly| +5420|AAAAAAAAMCFBAAAA|2451785|2451814|DEPARTMENT|51|20|Dependent eyes change international customers. Easy ministers cannot skip only by a limits. Points|monthly| +5421|AAAAAAAANCFBAAAA|2451785|2451814|DEPARTMENT|51|21|Hard, foreign attacks enable advisory, old colours. Ready|monthly| +5422|AAAAAAAAOCFBAAAA|2451785|2451814|DEPARTMENT|51|22|Beneficial services could contract up a environments. Foreign, other terms shall see ann|monthly| +5423|AAAAAAAAPCFBAAAA|2451785|2451814|DEPARTMENT|51|23|So complex groups may not help. Internal systems could|monthly| +5424|AAAAAAAAADFBAAAA|2451785|2451814|DEPARTMENT|51|24|Styles mention available, mental attitudes. Other, important workers contribute now.|monthly| +5425|AAAAAAAABDFBAAAA|2451785|2451814|DEPARTMENT|51|25|Recordings observe. Neither sudden studies recognise both indeed coastal years. Long, left s|monthly| +5426|AAAAAAAACDFBAAAA|2451785|2451814|DEPARTMENT|51|26|Students give rarely. Specific posts should allow still dete|monthly| +5427|AAAAAAAADDFBAAAA|2451785|2451814|DEPARTMENT|51|27|Times play eventually gross, interested unions. Victorian systems set legs. Miserable, grea|monthly| +5428|AAAAAAAAEDFBAAAA|2451785|2451814|DEPARTMENT|51|28|Various chapters worry again. Sentences drive tall, aware heads; together young signs m|monthly| +5429|AAAAAAAAFDFBAAAA|2451785|2451814|DEPARTMENT|51|29|Great, labour entries make; really initial kinds define periods. Only usua|monthly| +5430|AAAAAAAAGDFBAAAA|2451785|2451814|DEPARTMENT|51|30|Scientists could insist local industries. Already dangerous ideas accrue s|monthly| +5431|AAAAAAAAHDFBAAAA|2451785|2451814|DEPARTMENT|51|31|Now forward tensions use so. Clearly islamic activities attend a.d. powers. Pre|monthly| +5432|AAAAAAAAIDFBAAAA|2451785|2451814|DEPARTMENT|51|32|Armed, true equations could arrive quickly likely, corresponding d|monthly| +5433|AAAAAAAAJDFBAAAA|2451785|2451814|DEPARTMENT|51|33|By far genetic boats should worry alone. Special women will not publish instead personally|monthly| +5434|AAAAAAAAKDFBAAAA|2451785|2451814|DEPARTMENT|51|34|Beautiful writers would not tell in a forces. Only late studies |monthly| +5435|AAAAAAAALDFBAAAA|2451785|2451814|DEPARTMENT|51|35|Views put faster. Indeed deep companies may investigate well sure enquiries. |monthly| +5436|AAAAAAAAMDFBAAAA|2451785|2451814|DEPARTMENT|51|36|Easy, british thousands increase so suggestions; never different sessions may contrib|monthly| +5437|AAAAAAAANDFBAAAA|2451785|2451814|DEPARTMENT|51|37|Experiments might demonstrate times. Actions will manage practically; western|monthly| +5438|AAAAAAAAODFBAAAA|2451785|2451814|DEPARTMENT|51|38|Foreign, political forms must pick here french gir|monthly| +5439|AAAAAAAAPDFBAAAA|2451785|2451814|DEPARTMENT|51|39|Social measures used to bring features. Growing, available band|monthly| +5440|AAAAAAAAAEFBAAAA|2451785|2451814|DEPARTMENT|51|40|Never inc gardens seek all right upon a leaders. Public lines shall think due leaders. Simpl|monthly| +5441|AAAAAAAABEFBAAAA|2451785|2451814|DEPARTMENT|51|41|Brown representatives cannot prove. Major, british|monthly| +5442|AAAAAAAACEFBAAAA|2451785|2451814|DEPARTMENT|51|42|Democratic, international parents try primary matters.|monthly| +5443|AAAAAAAADEFBAAAA|2451785|2451814|DEPARTMENT|51|43|Social, usual carers must manage also slowly nice |monthly| +5444|AAAAAAAAEEFBAAAA|2451785|2451814|DEPARTMENT|51|44|Random, entire futures continue never; types shall take usually national, ligh|monthly| +5445|AAAAAAAAFEFBAAAA|2451785|2451814|DEPARTMENT|51|45|Angry, great months help persistently. Even contemporary functions may |monthly| +5446|AAAAAAAAGEFBAAAA|2451785|2451814|DEPARTMENT|51|46|Little potential signs ought to rest closely natural high prin|monthly| +5447|AAAAAAAAHEFBAAAA|2451785|2451814|DEPARTMENT|51|47|Other, alone years shrug most. Even green forms employ schools. Nervously apparent co|monthly| +5448|AAAAAAAAIEFBAAAA|2451785|2451814|DEPARTMENT|51|48|Cases ought to pick in the services. Public, average depths should suppose then other |monthly| +5449|AAAAAAAAJEFBAAAA|2451785|2451814|DEPARTMENT|51|49|Proposed loans catch necessarily techniques. Investments support|monthly| +5450|AAAAAAAAKEFBAAAA|2451785|2451814|DEPARTMENT|51|50|Political buildings could track high markets. Cameras cannot let poor governments. Di|monthly| +5451|AAAAAAAALEFBAAAA|2451785|2451814|DEPARTMENT|51|51|Often essential conditions provide at a generations. Often separate interactions buy |monthly| +5452|AAAAAAAAMEFBAAAA|2451785|2451814|DEPARTMENT|51|52|Available problems shall walk individual times; boots sh|monthly| +5453|AAAAAAAANEFBAAAA|2451785|2451814|DEPARTMENT|51|53|Normal, unexpected loans eat usually to the roots. Rather|monthly| +5454|AAAAAAAAOEFBAAAA|2451785|2451814|DEPARTMENT|51|54|Arguments clean much heavily original rates. Other, british|monthly| +5455|AAAAAAAAPEFBAAAA|2451785|2451814|DEPARTMENT|51|55|Then important expectations may keep more social open persons. Various|monthly| +5456|AAAAAAAAAFFBAAAA|2451785|2451814|DEPARTMENT|51|56|Labour elements make to a discussions. Residents used|monthly| +5457|AAAAAAAABFFBAAAA|2451785|2451814|DEPARTMENT|51|57|Most local edges must expect later views. Studies should need whi|monthly| +5458|AAAAAAAACFFBAAAA|2451785|2451814|DEPARTMENT|51|58|Also only numbers must not like regulations; criteria would not attack like the sons. Sets |monthly| +5459|AAAAAAAADFFBAAAA|2451785|2451814|DEPARTMENT|51|59|Simultaneously pregnant experiences shoot really accurate jobs. Go|monthly| +5460|AAAAAAAAEFFBAAAA|2451785|2451814|DEPARTMENT|51|60|Doctors look copies. Happy items must stop certainly with a babies. Good tradition|monthly| +5461|AAAAAAAAFFFBAAAA|2451785|2451814|DEPARTMENT|51|61|Available, top women used to believe things. Appropriate, neat others should cope|monthly| +5462|AAAAAAAAGFFBAAAA|2451785||DEPARTMENT||62||| +5463|AAAAAAAAHFFBAAAA|2451785|2451814|DEPARTMENT|51|63|Old, available points say often there sure events; large, nice men ought to call to the folk.|monthly| +5464|AAAAAAAAIFFBAAAA|2451785|2451814|DEPARTMENT|51|64|Actually high figures find as the examples. Average, private women must get really therefore t|monthly| +5465|AAAAAAAAJFFBAAAA|2451785|2451814|DEPARTMENT|51|65|Shelves might not make obvious members; glad associations|monthly| +5466|AAAAAAAAKFFBAAAA|2451785|2451814|DEPARTMENT|51|66|Poor pressures would know public sums. Long particular prices like; concerned |monthly| +5467|AAAAAAAALFFBAAAA|2451785|2451814|DEPARTMENT|51|67|More than major facilities refuse perhaps; coloured seco|monthly| +5468|AAAAAAAAMFFBAAAA|2451785|2451814|DEPARTMENT|51|68|Hence administrative teachers keep. More fresh patterns could cope |monthly| +5469|AAAAAAAANFFBAAAA|2451785|2451814|DEPARTMENT|51|69|Little low results may not say also black jobs. Huge guns must not wa|monthly| +5470|AAAAAAAAOFFBAAAA|2451785|2451814|DEPARTMENT|51|70|Digital ears could not use outstanding, parliamentary courses. Windows can reco|monthly| +5471|AAAAAAAAPFFBAAAA|2451785|2451814|DEPARTMENT|51|71|Now main exhibitions check beautifully equally labour plans. Maj|monthly| +5472|AAAAAAAAAGFBAAAA|2451785|2451814|DEPARTMENT|51|72|Close domestic issues may not support in a books. Local committees must |monthly| +5473|AAAAAAAABGFBAAAA|2451785|2451814|DEPARTMENT|51|73|Estimated, ancient charges could not mend anywhere great worke|monthly| +5474|AAAAAAAACGFBAAAA|2451785|2451814|DEPARTMENT|51|74|Less growing meetings occur guests. Sooner consistent areas cannot deci|monthly| +5475|AAAAAAAADGFBAAAA|2451785|2451814|DEPARTMENT|51|75|Yesterday subtle notes think vastly facts. Voters carry probably national,|monthly| +5476|AAAAAAAAEGFBAAAA|2451785|2451814|DEPARTMENT|51|76|High, important lines might form western professio|monthly| +5477|AAAAAAAAFGFBAAAA|2451785|2451814|DEPARTMENT|51|77|Usually distinctive conditions look. Internal authorities purchase just examples. Financial fe|monthly| +5478|AAAAAAAAGGFBAAAA|2451785|2451814|DEPARTMENT|51|78|White jews shall occur better political words. Later national ton|monthly| +5479|AAAAAAAAHGFBAAAA|2451785|2451814|DEPARTMENT|51|79|Inner circumstances shall specify cultures. Stairs can supply then frequent th|monthly| +5480|AAAAAAAAIGFBAAAA|2451785|2451814|DEPARTMENT|51|80|In short local partners shall carve brief, able servants. New, remarkable lines will reliev|monthly| +5481|AAAAAAAAJGFBAAAA|2451785|2451814|DEPARTMENT|51|81|Issues will keep over difficult, good arrangements. Aver|monthly| +5482|AAAAAAAAKGFBAAAA|2451785|2451814|DEPARTMENT|51|82|National eyes imagine bitterly. Only acts may take psychologi|monthly| +5483|AAAAAAAALGFBAAAA|2451785|2451814|DEPARTMENT|51|83|Hands might perform now primary plans. Really mutual officers would not recei|monthly| +5484|AAAAAAAAMGFBAAAA|2451785|2451814|DEPARTMENT|51|84|Dynamic ideas prevent too fields. Specially regular students ought to talk; never rel|monthly| +5485|AAAAAAAANGFBAAAA|2451785|2451814|DEPARTMENT|51|85|Written dreams get scarcely then great legs; broad, reliable|monthly| +5486|AAAAAAAAOGFBAAAA|2451785|2451814|DEPARTMENT|51|86|Legal years mean. Blue newspapers limit brief flowers. Possible, obvio|monthly| +5487|AAAAAAAAPGFBAAAA|2451785|2451814|DEPARTMENT|51|87|Alike big courts shall think far acceptable, main sums. Local purposes might work educational,|monthly| +5488|AAAAAAAAAHFBAAAA|2451785|2451814|DEPARTMENT|51|88|About useful prices could not blame resources; particularly harsh spaces woul|monthly| +5489|AAAAAAAABHFBAAAA|2451785|2451814|DEPARTMENT|51|89|Extra, wooden scots may make briefly chapters; yet new materials ought to tell still ready hand|monthly| +5490|AAAAAAAACHFBAAAA|2451785|2451814|DEPARTMENT|51|90|Arms make long. Beautiful machines must learn at the stairs. Cars sleep rather level s|monthly| +5491|AAAAAAAADHFBAAAA|2451785|2451814|DEPARTMENT|51|91|Current, new terms enter quickly terrible partners. Annual, adjacent players must n|monthly| +5492|AAAAAAAAEHFBAAAA|2451785|2451814|DEPARTMENT|51|92|Discussions return just general sales; democratic characteristics arise active statements. Rela|monthly| +5493|AAAAAAAAFHFBAAAA|2451785|2451814|DEPARTMENT|51|93|Nasty, impossible polls write possible, large accidents. Clearly new minister|monthly| +5494|AAAAAAAAGHFBAAAA|2451785|2451814|DEPARTMENT|51|94|Practical clothes will get never low students. Available provisions may not give vividly very au|monthly| +5495|AAAAAAAAHHFBAAAA|2451785|2451814|DEPARTMENT|51|95|Head things may answer actively always logical pockets; clearly|monthly| +5496|AAAAAAAAIHFBAAAA|2451785|2451814|DEPARTMENT|51|96|Narrow years must not meet usually. Only mixed unions turn soon magnificent times. So loose |monthly| +5497|AAAAAAAAJHFBAAAA|2451785|2451814|DEPARTMENT|51|97|Businesses may get of course children. Responsible tools may bring. Hands raise main proceedings. N|monthly| +5498|AAAAAAAAKHFBAAAA|2451785|2451814|DEPARTMENT|51|98|New houses turn alone single months. Tiny, complete arts pro|monthly| +5499|AAAAAAAALHFBAAAA|2451785|2451814|DEPARTMENT|51|99|Old, permanent types cannot make; little contributions used to genera|monthly| +5500|AAAAAAAAMHFBAAAA|2451785|2451814|DEPARTMENT|51|100|Other, heavy forces see legal affairs. Royal, similar mountains could revise then. Creative, |monthly| +5501|AAAAAAAANHFBAAAA|2451785|2451814|DEPARTMENT|51|101|Centres should find much; american, beautiful boys prove also real|monthly| +5502|AAAAAAAAOHFBAAAA|2451785|2451814|DEPARTMENT|51|102|Public, public applications could undermine unable structures. Most soft cha|monthly| +5503|AAAAAAAAPHFBAAAA|2451785|2451814|DEPARTMENT|51|103|Years examine now with a groups. Initial companies cannot tell. A|monthly| +5504|AAAAAAAAAIFBAAAA|2451785|2451814|DEPARTMENT|51|104|Sure, nuclear cattle can interfere perhaps. Eyes shall rest just beautiful children; firm|monthly| +5505|AAAAAAAABIFBAAAA|2451785|2451814|DEPARTMENT|51|105|High servants show items. Right labour matches stay following delegates.|monthly| +5506|AAAAAAAACIFBAAAA|2451785|2451814|DEPARTMENT|51|106|Satisfied numbers say always at a years. Neighbours might achieve foreign years. Areas contin|monthly| +5507|AAAAAAAADIFBAAAA|2451785|2451814|DEPARTMENT|51|107|Parents will not help tables. Influential, extreme hotels can recognise public, effective servi|monthly| +5508|AAAAAAAAEIFBAAAA|2451785|2451814|DEPARTMENT|51|108|Women hear here from a humans; special members say available decisions. Onl|monthly| +5509|AAAAAAAAFIFBAAAA|2451815|2451844|DEPARTMENT|52|1|Figures state different, important securities. Reports can sing later bad, irish pages. Pract|monthly| +5510|AAAAAAAAGIFBAAAA|2451815|2451844|DEPARTMENT|52|2|Lines ask most small, narrow times. Only other dogs finish old, different sources. On|monthly| +5511|AAAAAAAAHIFBAAAA|2451815|2451844|DEPARTMENT|52|3|Clean, different fingers decide also roman, overseas charts. Narrowly g|monthly| +5512|AAAAAAAAIIFBAAAA|2451815|2451844|DEPARTMENT|52|4|Firms must start briefly traditional, ordinary duties. |monthly| +5513|AAAAAAAAJIFBAAAA|2451815|2451844|DEPARTMENT|52|5|Even implicit stairs can add just profound areas; elderly|monthly| +5514|AAAAAAAAKIFBAAAA|2451815|2451844|DEPARTMENT|52|6|Possible ears need big files. Apart warm activities look above. Eur|monthly| +5515|AAAAAAAALIFBAAAA|2451815|2451844|DEPARTMENT|52|7|Religious purposes modify quite confident states. Medical,|monthly| +5516|AAAAAAAAMIFBAAAA|2451815|2451844|DEPARTMENT|52|8|Animals wait more than. Young, western days will not work for a years.|monthly| +5517|AAAAAAAANIFBAAAA|2451815|2451844|DEPARTMENT|52|9|Ears should modify significantly married, simple arms. Here excellent fires shall sh|monthly| +5518|AAAAAAAAOIFBAAAA|2451815|2451844|DEPARTMENT|52|10|Good, existing looks might not allow there racial, adviso|monthly| +5519|AAAAAAAAPIFBAAAA|2451815|2451844|DEPARTMENT|52|11|Human, wrong things fall totally cases. Different candidates may not make expec|monthly| +5520|AAAAAAAAAJFBAAAA|2451815|2451844|DEPARTMENT|52|12|Broken, direct states offer religious police. Ideological men come|monthly| +5521|AAAAAAAABJFBAAAA|2451815|2451844|DEPARTMENT|52|13|Rules ought to breed hence for the corners; new millions seek in a goods. Revenues drop cou|monthly| +5522|AAAAAAAACJFBAAAA|2451815|2451844|DEPARTMENT|52|14|Personal, main months shall use. Sophisticated patients shall not come useful clouds|monthly| +5523|AAAAAAAADJFBAAAA|2451815|2451844|DEPARTMENT|52|15|Royal, only gates must show. More dead liabilities shall work yesterday int|monthly| +5524|AAAAAAAAEJFBAAAA|2451815|2451844|DEPARTMENT|52|16|Original parents say; roots may argue. Special, small families used to realize. Particul|monthly| +5525|AAAAAAAAFJFBAAAA|2451815|2451844|DEPARTMENT|52|17|Scientific, general decisions clear upstream in the others; even black newspapers off|monthly| +5526|AAAAAAAAGJFBAAAA|2451815|2451844|DEPARTMENT|52|18|Bits like psychiatric years. Both good children rely now compatible book|monthly| +5527|AAAAAAAAHJFBAAAA|2451815|2451844|DEPARTMENT|52|19|Years go much. Open products seize cheeks. High lip|monthly| +5528|AAAAAAAAIJFBAAAA|2451815|2451844|DEPARTMENT|52|20|Willing obligations will take alone so anxious fingers. Persons can implement |monthly| +5529|AAAAAAAAJJFBAAAA|2451815|2451844|DEPARTMENT|52|21|Local qualities may advise far careful conclusions. Economic process|monthly| +5530|AAAAAAAAKJFBAAAA|2451815|2451844|DEPARTMENT|52|22|Only connections teach always; supreme, tall languages may remember nati|monthly| +5531|AAAAAAAALJFBAAAA|2451815|2451844|DEPARTMENT|52|23|Barely naked prices see previously following, extra police. S|monthly| +5532|AAAAAAAAMJFBAAAA|2451815|2451844|DEPARTMENT|52|24|Tasks know forcibly. English estates must get offices; schools try almost enormous processes|monthly| +5533|AAAAAAAANJFBAAAA|2451815|2451844|DEPARTMENT|52|25|General components try extra, little purposes. Faint groups for|monthly| +5534|AAAAAAAAOJFBAAAA|2451815|2451844|DEPARTMENT|52|26|Measures make much cultural, only reforms. Objects could bring short; hundre|monthly| +5535|AAAAAAAAPJFBAAAA|2451815|2451844|DEPARTMENT|52|27|New weeks could not resist. Economic resources sha|monthly| +5536|AAAAAAAAAKFBAAAA|2451815|2451844|DEPARTMENT|52|28|United, external contacts must not enjoy ideas; convenient, significant mach|monthly| +5537|AAAAAAAABKFBAAAA|2451815|2451844|DEPARTMENT|52|29|Catholic, competitive actions know jointly to the numbers. Common members know nearby condit|monthly| +5538|AAAAAAAACKFBAAAA|2451815|2451844|DEPARTMENT|52|30|Fellow times spend eyes; women can represent ever items. Able, horizontal beds could not go reso|monthly| +5539|AAAAAAAADKFBAAAA|2451815|2451844|DEPARTMENT|52|31|Far subjects spend tory times. Royal, nice women maintain|monthly| +5540|AAAAAAAAEKFBAAAA|2451815|2451844|DEPARTMENT|52|32|Likewise light reasons influence diverse clients. Australian inhabitants can come a|monthly| +5541|AAAAAAAAFKFBAAAA|2451815|2451844|DEPARTMENT|52|33|Old lands would hold highly blue individuals; houses might not find also organ|monthly| +5542|AAAAAAAAGKFBAAAA|2451815|2451844|DEPARTMENT|52|34|Likely, agricultural parts shall give from a resources; excellent changes develop a|monthly| +5543|AAAAAAAAHKFBAAAA|2451815|2451844|DEPARTMENT|52|35|Generations might not find in a changes. Perhaps imp|monthly| +5544|AAAAAAAAIKFBAAAA|2451815|2451844|DEPARTMENT|52|36|Young machines can expect never patients. Healthy, different babies ought to know perhaps then ea|monthly| +5545|AAAAAAAAJKFBAAAA|2451815|2451844|DEPARTMENT|52|37|Standards should not head still happy exchanges. Young, revolutionary stars forget. Also oth|monthly| +5546|AAAAAAAAKKFBAAAA|2451815|2451844|DEPARTMENT|52|38|Recent fruits used to start often raw processes. Formerly obvious groups must join significantly|monthly| +5547|AAAAAAAALKFBAAAA|2451815|2451844|DEPARTMENT|52|39|Often good investors give heavily men. Horses may not come unexpectedly physical corne|monthly| +5548|AAAAAAAAMKFBAAAA|2451815|2451844|DEPARTMENT|52|40|Anyway true locations support ready, good states. Special, personal limits deal here st|monthly| +5549|AAAAAAAANKFBAAAA|2451815|2451844|DEPARTMENT|52|41|Reports feel well fingers. Sections try newly final hills. Copies believe common objects; places e|monthly| +5550|AAAAAAAAOKFBAAAA|2451815|2451844|DEPARTMENT|52|42|So back women cannot keep more clear red terms. So golden worker|monthly| +5551|AAAAAAAAPKFBAAAA|2451815|2451844|DEPARTMENT|52|43|Open odds would expand even on a courts. Now direct teachers step exactly constant effects. Again f|monthly| +5552|AAAAAAAAALFBAAAA|2451815|2451844|DEPARTMENT|52|44|Young needs can appear altogether also limited recommendations. New foods decide across about |monthly| +5553|AAAAAAAABLFBAAAA|2451815|2451844|DEPARTMENT|52|45|Bits used to increase now by a instructions. Primary teachers could talk pap|monthly| +5554|AAAAAAAACLFBAAAA|2451815|2451844|DEPARTMENT|52|46|Rather scientific courses used to adjust now private, proposed days. Active|monthly| +5555|AAAAAAAADLFBAAAA|2451815|2451844|DEPARTMENT|52|47|Talks stay. Countries bring now from a schools; recently strong germans produce in a fam|monthly| +5556|AAAAAAAAELFBAAAA||2451844|DEPARTMENT||48|Hard, large guests shall need golden, normal saving|monthly| +5557|AAAAAAAAFLFBAAAA|2451815|2451844|DEPARTMENT|52|49|Apparent representatives may speak high other points. Fond arts shall not reac|monthly| +5558|AAAAAAAAGLFBAAAA|2451815|2451844|DEPARTMENT|52|50|Great, constant resources could hold truly more complete remarks. Important, rural reco|monthly| +5559|AAAAAAAAHLFBAAAA|2451815|2451844|DEPARTMENT|52|51|Again poor words would not target places. Arms widen across full lines. Year|monthly| +5560|AAAAAAAAILFBAAAA|2451815|2451844|DEPARTMENT|52|52|Payments get very dead customers. Directly prospective changes can flour|monthly| +5561|AAAAAAAAJLFBAAAA|2451815|2451844|DEPARTMENT|52|53|Prime problems worry brown, young magistrates. Times would c|monthly| +5562|AAAAAAAAKLFBAAAA|2451815|2451844|DEPARTMENT|52|54|Fairly available choices serve always also other miles; servants risk |monthly| +5563|AAAAAAAALLFBAAAA|2451815|2451844|DEPARTMENT|52|55|Only final speakers should believe members. At least extraordinary |monthly| +5564|AAAAAAAAMLFBAAAA|2451815|2451844|DEPARTMENT|52|56|Important subsidies can appear remotely actual profits. |monthly| +5565|AAAAAAAANLFBAAAA|2451815|2451844|DEPARTMENT|52|57|Claims could remain again politicians. Short, dangerous women put groups. Howev|monthly| +5566|AAAAAAAAOLFBAAAA|2451815|2451844|DEPARTMENT|52|58|Great attitudes bend now below able arts. Again su|monthly| +5567|AAAAAAAAPLFBAAAA|2451815|2451844|DEPARTMENT|52|59|Particular rates might lick either large police. Problems see powerful years. Certainly ruling|monthly| +5568|AAAAAAAAAMFBAAAA|2451815|2451844|DEPARTMENT|52|60|Contracts will lead. Children will not pay here specific observati|monthly| +5569|AAAAAAAABMFBAAAA|2451815|2451844|DEPARTMENT|52|61|Romantic views take so linguistic ways; enemies can survive; gener|monthly| +5570|AAAAAAAACMFBAAAA|2451815|2451844|DEPARTMENT|52|62|Times may detail french groups. Huge strategies go now. Well helpf|monthly| +5571|AAAAAAAADMFBAAAA|2451815|2451844|DEPARTMENT|52|63|Personal, popular weeks should not vary; men ought to take |monthly| +5572|AAAAAAAAEMFBAAAA|2451815|2451844|DEPARTMENT|52|64|Well ashamed officers pull free, popular points. Religious altern|monthly| +5573|AAAAAAAAFMFBAAAA|2451815|2451844|DEPARTMENT|52|65|Brothers describe best interested forces. Numbers find pounds. |monthly| +5574|AAAAAAAAGMFBAAAA|2451815|2451844|DEPARTMENT|52|66|Pupils will not form duties; scientific proposals get already. Federal, british example|monthly| +5575|AAAAAAAAHMFBAAAA|2451815|2451844|DEPARTMENT|52|67|Sometime royal questions buy unexpected, primary months. Even famous errors de|monthly| +5576|AAAAAAAAIMFBAAAA|2451815|2451844|DEPARTMENT|52|68|Already new years take days. More private actions must not find mo|monthly| +5577|AAAAAAAAJMFBAAAA|2451815|2451844|DEPARTMENT|52|69|Gross terms ought to understand different heels. Husbands come carefully. Red, l|monthly| +5578|AAAAAAAAKMFBAAAA|2451815|2451844|DEPARTMENT|52|70|Annual, aware groups mind applications. Controls used to come most|monthly| +5579|AAAAAAAALMFBAAAA|2451815|2451844|DEPARTMENT|52|71|Weeks build currently important illustrations. Already capitalist days hear black friends.|monthly| +5580|AAAAAAAAMMFBAAAA|2451815|2451844|DEPARTMENT|52|72|About direct experiments eat brilliant, other details. Sudden, bold lessons |monthly| +5581|AAAAAAAANMFBAAAA|2451815|2451844|DEPARTMENT|52|73|Particularly large years continue. Skills save lips. Officers should come always per|monthly| +5582|AAAAAAAAOMFBAAAA|2451815|2451844|DEPARTMENT|52|74|New, crucial days will forget by a authorities. New, intense things|monthly| +5583|AAAAAAAAPMFBAAAA|2451815|2451844|DEPARTMENT|52|75|Easy, able jobs suggest now soft sounds. Related photographs call slowly f|monthly| +5584|AAAAAAAAANFBAAAA|2451815|2451844|DEPARTMENT|52|76|Futures can appear all to the activities. Large studies must think in the terms. S|monthly| +5585|AAAAAAAABNFBAAAA|2451815|2451844|DEPARTMENT|52|77|Estimated children used to fall just willing sites. Costs wou|monthly| +5586|AAAAAAAACNFBAAAA|2451815|2451844|DEPARTMENT|52|78|Political eyes would coincide. Provincial, useful newspapers finish nevertheless redundant,|monthly| +5587|AAAAAAAADNFBAAAA|2451815|2451844|DEPARTMENT|52|79|Abroad original clothes cannot generate during a thousands. African, cultural factors watch for th|monthly| +5588|AAAAAAAAENFBAAAA|2451815|2451844|DEPARTMENT|52|80|Thus local maps can fit more following following facilit|monthly| +5589|AAAAAAAAFNFBAAAA|2451815|2451844|DEPARTMENT|52|81|Stories might determine resources. Literally hard wee|monthly| +5590|AAAAAAAAGNFBAAAA|2451815|2451844|DEPARTMENT|52|82|Then high houses know together. Nearly vital studies ma|monthly| +5591|AAAAAAAAHNFBAAAA|2451815|2451844|DEPARTMENT|52|83|Decisions would not present extremely economic cus|monthly| +5592|AAAAAAAAINFBAAAA|2451815|2451844|DEPARTMENT|52|84|Real weapons fill. Formal ministers obtain. Very new ideas look high experiences; obvi|monthly| +5593|AAAAAAAAJNFBAAAA|2451815|2451844|DEPARTMENT|52|85|Large, bad months will not see below surprising, able pupils; happy others get considerably si|monthly| +5594|AAAAAAAAKNFBAAAA|2451815|2451844|DEPARTMENT|52|86|British funds would not sleep as a differences. About necessary users see new, legal cases; fe|monthly| +5595|AAAAAAAALNFBAAAA|2451815||DEPARTMENT|||Local fingers may become english vegetables. Colleagues |monthly| +5596|AAAAAAAAMNFBAAAA|2451815|2451844|DEPARTMENT|52|88|Close versions know still large, particular products. Dangerous, mas|monthly| +5597|AAAAAAAANNFBAAAA|2451815|2451844|DEPARTMENT|52|89|Just, quick facilities will match in the states. Romantic seeds get frantically other mach|monthly| +5598|AAAAAAAAONFBAAAA|2451815|2451844|DEPARTMENT|52|90|Old, new reasons could halt yet coastal, male periods. Masses mi|monthly| +5599|AAAAAAAAPNFBAAAA|2451815|2451844|DEPARTMENT|52|91|Round programs guide sufficiently actions; days may speak |monthly| +5600|AAAAAAAAAOFBAAAA|2451815|2451844|DEPARTMENT|52|92|Across able types want countries. Old, real children would go vir|monthly| +5601|AAAAAAAABOFBAAAA|2451815|2451844|DEPARTMENT|52|93|Senior levels explain also low pounds. Types would put now other activities. At last other eyes |monthly| +5602|AAAAAAAACOFBAAAA|2451815|2451844|DEPARTMENT|52|94|Nations go never jeans. Even soviet miles go however international options|monthly| +5603|AAAAAAAADOFBAAAA|2451815|2451844|DEPARTMENT|52|95|Still sexual actions prevent; relative prices see only religious studies. Ince|monthly| +5604|AAAAAAAAEOFBAAAA|2451815|2451844|DEPARTMENT|52|96|Groups will not reach; dramatic, particular owners go correctly things. Video-taped, little|monthly| +5605|AAAAAAAAFOFBAAAA|2451815|2451844|DEPARTMENT|52|97|Just proper taxes answer more certain expectations. A|monthly| +5606|AAAAAAAAGOFBAAAA|2451815|2451844|DEPARTMENT|52|98|So different stairs used to call early, other stair|monthly| +5607|AAAAAAAAHOFBAAAA|2451815|2451844|DEPARTMENT|52|99|Expensive others buy short, consistent forces. Yet new customers reply at leas|monthly| +5608|AAAAAAAAIOFBAAAA|2451815|2451844|DEPARTMENT|52|100|Actually important techniques may not dominate often units. New, small notes will not go always; |monthly| +5609|AAAAAAAAJOFBAAAA||2451844|DEPARTMENT|52|101|Almost complete children shall draw strong artists. Wide arms rise good authorities. Days say wide|| +5610|AAAAAAAAKOFBAAAA|2451815|2451844|DEPARTMENT|52|102|Recent, little weeks rely favorable, useful shops. Actors shall retain relations. Attempts ought t|monthly| +5611|AAAAAAAALOFBAAAA|2451815|2451844|DEPARTMENT|52|103|Capable movements may not entertain in the areas. Elected, pale patient|monthly| +5612|AAAAAAAAMOFBAAAA|2451815|2451844|DEPARTMENT|52|104|Currently coming accounts follow immediately clear r|monthly| +5613|AAAAAAAANOFBAAAA|2451815|2451844|DEPARTMENT|52|105|Tears produce prisons. Likely questions shall join particularly ways; external measures could app|monthly| +5614|AAAAAAAAOOFBAAAA|2451815|2451844|DEPARTMENT|52|106|Traditions ought to sleep just equal witnesses. Yet full |monthly| +5615|AAAAAAAAPOFBAAAA|2451815|2451844|DEPARTMENT|52|107|Old executives could help never even environmental feet. Equal incidents make hours; researchers |monthly| +5616|AAAAAAAAAPFBAAAA|2451815|2451844|DEPARTMENT|52|108|Characters include please members; good, other men used to accept chemicals.|monthly| +5617|AAAAAAAABPFBAAAA|2451845|2451874|DEPARTMENT|53|1|Local, recent offices correspond. Trying pupils must reinforce so obvious centres. Un|monthly| +5618|AAAAAAAACPFBAAAA|2451845|2451874|DEPARTMENT|53|2|Important, basic feet induce just most charming streets. Active, |monthly| +5619|AAAAAAAADPFBAAAA|2451845|2451874|DEPARTMENT|53|3|Far historic reasons know strongly. Old things keep firmly indeed dead factories. Home|monthly| +5620|AAAAAAAAEPFBAAAA|2451845|2451874|DEPARTMENT|53|4|Members live very competitive ears. Able months hold just other benefits. Simple tickets cann|monthly| +5621|AAAAAAAAFPFBAAAA|2451845|2451874|DEPARTMENT|53|5|Hundreds may see unlikely figures. However appropriate fans confirm visual, important arts; comple|monthly| +5622|AAAAAAAAGPFBAAAA|2451845|2451874|DEPARTMENT|53|6|Available, big germans ought to get. Arguments must not sit thereafter|monthly| +5623|AAAAAAAAHPFBAAAA|2451845|2451874|DEPARTMENT|53|7|Personal thousands should select. Nuclear, original figures can sell for example guilty tabl|monthly| +5624|AAAAAAAAIPFBAAAA|2451845|2451874|DEPARTMENT|53|8|High popular affairs must not see points. Long, possible crops would make now prices|monthly| +5625|AAAAAAAAJPFBAAAA|2451845|2451874|DEPARTMENT|53|9|Remote forms can contact already difficult companies. Interests sit little faint men. Yea|monthly| +5626|AAAAAAAAKPFBAAAA|2451845|2451874|DEPARTMENT|53|10|More great cases can determine. Certain relationships help|monthly| +5627|AAAAAAAALPFBAAAA|2451845|2451874|DEPARTMENT|53|11|Prime, new years might avoid so on a clients. Effects|monthly| +5628|AAAAAAAAMPFBAAAA|2451845|2451874|DEPARTMENT|53|12|Paintings defend outside from a days. Peculiar grounds could raise jus|monthly| +5629|AAAAAAAANPFBAAAA|2451845|2451874|DEPARTMENT|53|13|Even suitable lines know apart like a children. Electoral, free costs occur technical, lesser f|monthly| +5630|AAAAAAAAOPFBAAAA|2451845|2451874|DEPARTMENT|53|14|Public, net fears might stare available parts; already long stars must go o|monthly| +5631|AAAAAAAAPPFBAAAA|2451845|2451874|DEPARTMENT|53|15|Savings see really men. Standards develop big, limited tables.|monthly| +5632|AAAAAAAAAAGBAAAA|2451845|2451874|DEPARTMENT|53|16|At least certain numbers might attack addresses. Upright spanish |monthly| +5633|AAAAAAAABAGBAAAA|2451845|2451874|DEPARTMENT|53|17|Ever social actions might not bring fair. Officers |monthly| +5634|AAAAAAAACAGBAAAA|2451845|2451874|DEPARTMENT|53|18|Dear, public men ought to represent admittedly mai|monthly| +5635|AAAAAAAADAGBAAAA|2451845|2451874|DEPARTMENT|53|19|Heavy, japanese stocks may see ever dark years; rocks know of course liberal proces|monthly| +5636|AAAAAAAAEAGBAAAA|2451845|2451874|DEPARTMENT|53|20|Guns form japanese sheets. Delicious, previous times seek now certain, armed plants. Soc|monthly| +5637|AAAAAAAAFAGBAAAA|2451845|2451874|DEPARTMENT|53|21|Hard teeth handle also normal relations. Good, concerned b|monthly| +5638|AAAAAAAAGAGBAAAA|2451845|2451874|DEPARTMENT|53|22|Institutions require by the laws. Games give. Much normal director|monthly| +5639|AAAAAAAAHAGBAAAA|2451845|2451874|DEPARTMENT|53|23|Ears begin too to a duties. Poor commentators used to blame. Yet globa|monthly| +5640|AAAAAAAAIAGBAAAA|2451845|2451874|DEPARTMENT|53|24|Possible parties ought to make immediately schools|monthly| +5641|AAAAAAAAJAGBAAAA|2451845|2451874|DEPARTMENT|53|25|Most little connections will run at a children; simple emissions decide there events. Sa|monthly| +5642|AAAAAAAAKAGBAAAA|2451845|2451874|DEPARTMENT|53|26|Details will deal over. Too successful points used to use wrong often representa|monthly| +5643|AAAAAAAALAGBAAAA|2451845|2451874|DEPARTMENT|53|27|Techniques must not issue below in a minutes. Large, useless cards ma|monthly| +5644|AAAAAAAAMAGBAAAA|2451845|2451874|DEPARTMENT|53|28|Original, obvious answers may not use over various, artificial partners. All hot developers shoul|monthly| +5645|AAAAAAAANAGBAAAA|2451845|2451874|DEPARTMENT|53|29|Police defend nevertheless personnel. Specialist societies would free at a examples. Onl|monthly| +5646|AAAAAAAAOAGBAAAA|2451845|2451874|DEPARTMENT|53|30|Successful pressures should summarise sometimes. Long years influen|monthly| +5647|AAAAAAAAPAGBAAAA|2451845|2451874|DEPARTMENT|53|31|Individual odds assess. Strong police check dramatically. Heavy exceptions make already essential |monthly| +5648|AAAAAAAAABGBAAAA|2451845|2451874|DEPARTMENT|53|32|Months think hotels. Unlikely, interesting sales seem probably medical, capable re|monthly| +5649|AAAAAAAABBGBAAAA|2451845|2451874|DEPARTMENT|53|33|Famous, french adults ought to direct. Advanced, normal agents will bother also |monthly| +5650|AAAAAAAACBGBAAAA|2451845|2451874|DEPARTMENT|53|34|Now political objects can leave substantially certain, secondary members. Legal, co|monthly| +5651|AAAAAAAADBGBAAAA|2451845|2451874|DEPARTMENT|53|35|Perhaps important societies know never deaf compla|monthly| +5652|AAAAAAAAEBGBAAAA|2451845|2451874|DEPARTMENT|53|36|Also sudden keys ought to look both birds. There new minutes us|monthly| +5653|AAAAAAAAFBGBAAAA|2451845|2451874|DEPARTMENT|53|37|Speeches ought to know now. Male arms used to pay slightly; far specific t|monthly| +5654|AAAAAAAAGBGBAAAA|2451845|2451874|DEPARTMENT|53|38|International, close men must follow very new ends. Clearly unexpected residents can prove near a |monthly| +5655|AAAAAAAAHBGBAAAA|2451845|2451874|DEPARTMENT|53|39|Very, limited proposals might not worry sharply as a recordings; potenti|monthly| +5656|AAAAAAAAIBGBAAAA|2451845|2451874|DEPARTMENT|53|40|Very careful police might help very into a engines. Cou|monthly| +5657|AAAAAAAAJBGBAAAA|2451845|2451874|DEPARTMENT|53|41|Views should not exist particularly proud ambitions. Certain, existing temperatures|monthly| +5658|AAAAAAAAKBGBAAAA|2451845|2451874|DEPARTMENT|53|42|Western, good shoes mean agricultural scientists. Seeds check enough. Financial men heal hope|monthly| +5659|AAAAAAAALBGBAAAA|2451845|2451874|DEPARTMENT|53|43|More certain paintings could not accept united days. More new rules ou|monthly| +5660|AAAAAAAAMBGBAAAA|2451845|2451874|DEPARTMENT|53|44|More direct legs make recommendations. Together tall shares should n|monthly| +5661|AAAAAAAANBGBAAAA|2451845|2451874|DEPARTMENT|53|45|Fierce lakes cheer now with the principles; now full instruments support then. So-called, |monthly| +5662|AAAAAAAAOBGBAAAA|2451845|2451874|DEPARTMENT|53|46|Concerned, wild activities see; close, small opportunitie|monthly| +5663|AAAAAAAAPBGBAAAA|2451845|2451874|DEPARTMENT|53|47|Resources shall take employees. Then able blues explain enou|monthly| +5664|AAAAAAAAACGBAAAA|2451845|2451874|DEPARTMENT|53|48|As comprehensive inhabitants think. Personal, peaceful rounds envi|monthly| +5665|AAAAAAAABCGBAAAA|2451845|2451874|DEPARTMENT|53|49|Organizations must not tackle though in a men; other names can stay as appropriately other no|monthly| +5666|AAAAAAAACCGBAAAA|2451845|2451874|DEPARTMENT|53|50|Today other powers may keep with a vessels. Arrangements might occur eas|monthly| +5667|AAAAAAAADCGBAAAA|2451845|2451874|DEPARTMENT|53|51|So secure ways should not bother important, good qu|monthly| +5668|AAAAAAAAECGBAAAA|2451845|2451874|DEPARTMENT|53|52|Still new requirements might not pay standard hands; relatively spec|monthly| +5669|AAAAAAAAFCGBAAAA|2451845|2451874|DEPARTMENT|53|53|Inevitable arrangements shall analyse only british detectives.|monthly| +5670|AAAAAAAAGCGBAAAA|2451845|2451874|DEPARTMENT|53|54|Excessive days can substitute in addition twin definitions. Beyo|monthly| +5671|AAAAAAAAHCGBAAAA|2451845|2451874|DEPARTMENT|53|55|Stars can seem great, very forms. More important holes make well eyes. Libraries should turn|monthly| +5672|AAAAAAAAICGBAAAA|2451845|2451874|DEPARTMENT|53|56|Structural, small profits focus women. Visitors look at the years. |monthly| +5673|AAAAAAAAJCGBAAAA|2451845|2451874|DEPARTMENT|53|57|Successful, able groups must not operate really immediate materials. D|monthly| +5674|AAAAAAAAKCGBAAAA|2451845|2451874|DEPARTMENT|53|58|Possible, brief practitioners may nod serious rumours; ho|monthly| +5675|AAAAAAAALCGBAAAA|2451845|2451874|DEPARTMENT|53|59|Symptoms may offer here legal, large products; chiefs would call the|monthly| +5676|AAAAAAAAMCGBAAAA|2451845|2451874|DEPARTMENT|53|60|Short months ought to need also early, certain lists. Closest|monthly| +5677|AAAAAAAANCGBAAAA|2451845|2451874|DEPARTMENT|53|61|Special, economic championships will get. Stages produce too corp|monthly| +5678|AAAAAAAAOCGBAAAA|2451845|2451874|DEPARTMENT|53|62|Professional differences need more major jeans; musicians may determine aboard p|monthly| +5679|AAAAAAAAPCGBAAAA|2451845|2451874|DEPARTMENT|53|63|Children draw often prime, explicit men; old eggs will spend already involved games.|monthly| +5680|AAAAAAAAADGBAAAA|2451845|2451874|DEPARTMENT|53|64|Circumstances must challenge there most particular types. Main, exist|monthly| +5681|AAAAAAAABDGBAAAA|2451845|2451874|DEPARTMENT|53|65|Rich, short levels should not establish intently full years. Second|monthly| +5682|AAAAAAAACDGBAAAA|2451845|2451874|DEPARTMENT|53|66|Parts want rather seemingly communist men. Forward offences used to know big companies; |monthly| +5683|AAAAAAAADDGBAAAA|2451845|2451874|DEPARTMENT|53|67|Holes prepare short sometimes early parties. New policies may offer military, d|monthly| +5684|AAAAAAAAEDGBAAAA|2451845|2451874|DEPARTMENT|53|68|Special prisoners cut so. Too huge accountants shall not walk even. Also impor|monthly| +5685|AAAAAAAAFDGBAAAA|2451845|2451874|DEPARTMENT|53|69|Single feet cannot see degrees. Social centres can get per the men. Vide|monthly| +5686|AAAAAAAAGDGBAAAA|2451845|2451874|DEPARTMENT|53|70|Differently specific years should not meet only. Natural services encompass well relative, pers|monthly| +5687|AAAAAAAAHDGBAAAA|2451845|2451874|DEPARTMENT|53|71|Only, available newspapers support only for ever simple respects; sure, possible proposals |monthly| +5688|AAAAAAAAIDGBAAAA|2451845|2451874|DEPARTMENT|53|72|Extensive obligations want often elegant measures. Families shall not linger grey, l|monthly| +5689|AAAAAAAAJDGBAAAA|2451845|2451874|DEPARTMENT|53|73|Certainly scottish partners may make efficiently appropriate endle|monthly| +5690|AAAAAAAAKDGBAAAA|2451845|2451874|DEPARTMENT|53|74|Loose proud bodies take by a months. Polish, considerable jobs used to modernise at |monthly| +5691|AAAAAAAALDGBAAAA|2451845|2451874|DEPARTMENT|53|75|Ever new problems talk at the doctors. Genuine days work also strong, political passa|monthly| +5692|AAAAAAAAMDGBAAAA|2451845|2451874|DEPARTMENT|53|76|Utterly initial ministers accept presumably degrees. Sen|monthly| +5693|AAAAAAAANDGBAAAA|2451845|2451874|DEPARTMENT|53|77|Following, european records might not help over democ|monthly| +5694|AAAAAAAAODGBAAAA|2451845|2451874|DEPARTMENT|53|78|Keys feel ever sides; for the most part legal stories see as groups; as important fu|monthly| +5695|AAAAAAAAPDGBAAAA|2451845|2451874|DEPARTMENT|53|79|Impossible men watch of course; late communications can hire o|monthly| +5696|AAAAAAAAAEGBAAAA|2451845|2451874|DEPARTMENT|53|80|Straightforward effects shape soon in a artists. National, concerned cust|monthly| +5697|AAAAAAAABEGBAAAA|2451845|2451874|DEPARTMENT|53|81|Familiar, other individuals used to devise sometimes in|monthly| +5698|AAAAAAAACEGBAAAA|2451845|2451874|DEPARTMENT|53|82|Years see. Certain, alone things fly today patients; very honest uses re|monthly| +5699|AAAAAAAADEGBAAAA|2451845|2451874|DEPARTMENT|53|83|Alleged relationships will like underneath powerful authorities. |monthly| +5700|AAAAAAAAEEGBAAAA|2451845|2451874|DEPARTMENT|53|84|Feet could win political vegetables. Warm drugs notify irish, short stories. Just academic optio|monthly| +5701|AAAAAAAAFEGBAAAA|2451845|2451874|DEPARTMENT|53|85|Unemployed patterns run complex, level details. Right, alternative aspects must exclude on a bo|monthly| +5702|AAAAAAAAGEGBAAAA|2451845|2451874|DEPARTMENT|53|86|Difficult grounds occur onto a doctors. Empty influences detect smoothly local sc|monthly| +5703|AAAAAAAAHEGBAAAA|2451845|2451874|DEPARTMENT|53|87|Empirical years ought to develop sharp. Institutions decide|monthly| +5704|AAAAAAAAIEGBAAAA|2451845|2451874|DEPARTMENT|53|88|There sacred images reveal here about heavy publishers. Able functions wou|monthly| +5705|AAAAAAAAJEGBAAAA|2451845|2451874|DEPARTMENT|53|89|Decades go. Specific, experienced products develop in contact with the weapons.|monthly| +5706|AAAAAAAAKEGBAAAA||||53||Individual findings satisfy. Traditional, left companies inv|| +5707|AAAAAAAALEGBAAAA|2451845|2451874|DEPARTMENT|53|91|New committees must organise instead different, right boots. Even electrical families dri|monthly| +5708|AAAAAAAAMEGBAAAA|2451845|2451874|DEPARTMENT|53|92|Whole, distinguished figures help american events. Encouraging events go practically as new hills. |monthly| +5709|AAAAAAAANEGBAAAA|2451845|2451874|DEPARTMENT|53|93|Female, english observations shall not know firmly new children. Open o|monthly| +5710|AAAAAAAAOEGBAAAA|2451845|2451874|DEPARTMENT|53|94|Over tremendous options let lists. Small conditions could confirm just m|monthly| +5711|AAAAAAAAPEGBAAAA|2451845|2451874|DEPARTMENT|53|95|Paradoxically moral communications used to need other companies; possible schools shall give ready|monthly| +5712|AAAAAAAAAFGBAAAA|2451845|2451874|DEPARTMENT|53|96|Others should make however curious demands. Following results dare particularly old|monthly| +5713|AAAAAAAABFGBAAAA|2451845|2451874|DEPARTMENT|53|97|Special children must explore for a ways. Impossible functions may rebuild powerful |monthly| +5714|AAAAAAAACFGBAAAA|2451845|2451874|DEPARTMENT|53|98|Days could not make to an notes. Current, permanent affairs should learn to a birds. Main, bitter |monthly| +5715|AAAAAAAADFGBAAAA|2451845|2451874|DEPARTMENT|53|99|Direct letters should not destroy relatively in a gaps. Possible, reluctant others make educatio|monthly| +5716|AAAAAAAAEFGBAAAA|2451845|2451874|DEPARTMENT|53|100|Different, light universities might call true words. Rum|monthly| +5717|AAAAAAAAFFGBAAAA|2451845|2451874|DEPARTMENT|53|101|Days tell too painful situations. Files deal finally eggs. Now upper politicians see nearly righ|monthly| +5718|AAAAAAAAGFGBAAAA|2451845|2451874|DEPARTMENT|53|102|Old tools rule industrial words. English, exact ways get |monthly| +5719|AAAAAAAAHFGBAAAA|2451845|2451874|DEPARTMENT|53|103|Very royal teachers used to lack soon genuine french contacts. Subsequently go|monthly| +5720|AAAAAAAAIFGBAAAA|2451845|2451874|DEPARTMENT|53|104|Changes master all right at a subjects; children guard on a relationships. Just so|monthly| +5721|AAAAAAAAJFGBAAAA|2451845|2451874|DEPARTMENT|53|105|Too future ladies should not protect relatively complete p|monthly| +5722|AAAAAAAAKFGBAAAA|2451845|2451874|DEPARTMENT|53|106|So other students would seek never. Sure difficult computers alter also|monthly| +5723|AAAAAAAALFGBAAAA|2451845|2451874|DEPARTMENT|53|107|More english taxes suggest less. Variations represent. Simple, decent st|monthly| +5724|AAAAAAAAMFGBAAAA|2451845|2451874|DEPARTMENT|53|108|Existing earnings must come more eyes. At least give|monthly| +5725|AAAAAAAANFGBAAAA||2451904|||1||monthly| +5726|AAAAAAAAOFGBAAAA|2451875|2451904|DEPARTMENT|54|2|Problems will seek forever conclusions. Years would not |monthly| +5727|AAAAAAAAPFGBAAAA|2451875|2451904|DEPARTMENT|54|3|Quick views raise other, strong orders. Things fin|monthly| +5728|AAAAAAAAAGGBAAAA|2451875|2451904|DEPARTMENT|54|4|Photographs would tell here to a practitioners. Victims shall believe mostly unusua|monthly| +5729|AAAAAAAABGGBAAAA|2451875|2451904|DEPARTMENT|54|5|Too sheer profits get together small directions. Old points shall revive also pr|monthly| +5730|AAAAAAAACGGBAAAA|2451875|2451904|DEPARTMENT|54|6|Both whole conservatives will not call. New relations smile less necessary |monthly| +5731|AAAAAAAADGGBAAAA|2451875|2451904|DEPARTMENT|54|7|Workers serve naturally due, subject services. Able crimes used to provide. Leading ideas think n|monthly| +5732|AAAAAAAAEGGBAAAA|2451875|2451904|DEPARTMENT|54|8|Now formal companies will recall absolutely reduced officers; thus national times |monthly| +5733|AAAAAAAAFGGBAAAA|2451875|2451904|DEPARTMENT|54|9|Attitudes used to take precisely new, uncomfortable words. Small, daily tales could not|monthly| +5734|AAAAAAAAGGGBAAAA|2451875|2451904|DEPARTMENT|54|10|Different contracts reduce weeks. Attractive, proposed things can advise memories. Unable, fina|monthly| +5735|AAAAAAAAHGGBAAAA|2451875|2451904|DEPARTMENT|54|11|Likely officials stem already. Good, legal plans might c|monthly| +5736|AAAAAAAAIGGBAAAA|2451875|2451904|DEPARTMENT|54|12|Temporary, unaware details might not exist much close possible y|monthly| +5737|AAAAAAAAJGGBAAAA|2451875|2451904|DEPARTMENT|54|13|Relations can dream less useful, arab indicators. Grounds can tell fiercely. Judicial views shall |monthly| +5738|AAAAAAAAKGGBAAAA|2451875|2451904|DEPARTMENT|54|14|Lovely arms involve even at the knees. Hard british charts might not arrange british men. Certain, |monthly| +5739|AAAAAAAALGGBAAAA|2451875|2451904|DEPARTMENT|54|15|Inherent, single hours get etc habits. Large, imaginative details might care increased features. Re|monthly| +5740|AAAAAAAAMGGBAAAA|2451875|2451904|DEPARTMENT|54|16|Causes build so-called farmers; really great movements cou|monthly| +5741|AAAAAAAANGGBAAAA|2451875|2451904|DEPARTMENT|54|17|Blue, happy scientists report solid, complex factors; unfortunately suitable shares |monthly| +5742|AAAAAAAAOGGBAAAA|2451875|2451904|DEPARTMENT|54|18|Accordingly different pp. could not obtain as ago european colleges|monthly| +5743|AAAAAAAAPGGBAAAA|2451875|2451904|DEPARTMENT|54|19|Shortly new costs shall leave potential, naval police. Preferen|monthly| +5744|AAAAAAAAAHGBAAAA|2451875|2451904|DEPARTMENT|54|20|Controls could not hear firmly no doubt other programmes. But great negotiations shou|monthly| +5745|AAAAAAAABHGBAAAA|2451875|2451904|DEPARTMENT|54|21|Particularly preliminary payments will see quickly in a childre|monthly| +5746|AAAAAAAACHGBAAAA|2451875|2451904|DEPARTMENT|54|22|Political, beautiful years wish patients; unequivoc|monthly| +5747|AAAAAAAADHGBAAAA|2451875|2451904|DEPARTMENT|54|23|Typical associations must see consequences. English, medical successes balance col|monthly| +5748|AAAAAAAAEHGBAAAA|2451875|2451904|DEPARTMENT|54|24|Front needs understand nearly good pages. Hardly whole areas shall not want little reforms; |monthly| +5749|AAAAAAAAFHGBAAAA|2451875|2451904|DEPARTMENT|54|25|Deep, wealthy goods carry most full, unknown opportuni|monthly| +5750|AAAAAAAAGHGBAAAA|2451875|2451904|DEPARTMENT||||| +5751|AAAAAAAAHHGBAAAA|2451875|2451904|DEPARTMENT|54|27|Still fiscal products must make legs. Other clients would gather too|monthly| +5752|AAAAAAAAIHGBAAAA|2451875|2451904|DEPARTMENT|54|28|Free politicians might not become. Early letters shall climb origina|monthly| +5753|AAAAAAAAJHGBAAAA|2451875|2451904|DEPARTMENT|54|29|Arts should not disregard readily companies. Suddenly legal |monthly| +5754|AAAAAAAAKHGBAAAA|2451875|2451904|DEPARTMENT|54|30|Strong states will remind types. Medical refugees could dis|monthly| +5755|AAAAAAAALHGBAAAA|2451875|2451904|DEPARTMENT|54|31|Sets must talk perhaps wonderful subjects. Intimate titles |monthly| +5756|AAAAAAAAMHGBAAAA|2451875|2451904|DEPARTMENT|54|32|Sharp, small parents play merely responsible, poor rates. |monthly| +5757|AAAAAAAANHGBAAAA|2451875|2451904|DEPARTMENT|54|33|Other hours might not come upstairs by a cars. Expectations get mu|monthly| +5758|AAAAAAAAOHGBAAAA|2451875|2451904|DEPARTMENT|54|34|About unable dollars may send thoroughly such as a fields; more other profits give |monthly| +5759|AAAAAAAAPHGBAAAA|2451875|2451904|DEPARTMENT|54|35|Royal, parliamentary animals exist upstairs. Well western animals think subsequent, w|monthly| +5760|AAAAAAAAAIGBAAAA|2451875|2451904|DEPARTMENT|54|36|Then only parents would stick frequent services. Suddenly dull me|monthly| +5761|AAAAAAAABIGBAAAA|2451875|2451904|DEPARTMENT|54|37|Normal machines shall not discover also procedures|monthly| +5762|AAAAAAAACIGBAAAA|2451875|2451904|DEPARTMENT|54|38|New, recent principles give previously teachers. Hopelessly white benefits can use |monthly| +5763|AAAAAAAADIGBAAAA|2451875|2451904|DEPARTMENT|54|39|Right far democrats shall result quite in the shares. Requirements would go |monthly| +5764|AAAAAAAAEIGBAAAA|2451875|2451904|DEPARTMENT|54|40|Either crucial years used to break cultural creditors; possible children coul|monthly| +5765|AAAAAAAAFIGBAAAA|2451875|2451904|DEPARTMENT|54|41|Sites object well further vague girls. Ideas would not sound e|monthly| +5766|AAAAAAAAGIGBAAAA|2451875|2451904|DEPARTMENT|54|42|Most other circumstances used to handle with a rules. Upper developments must remove til|monthly| +5767|AAAAAAAAHIGBAAAA|2451875|2451904|DEPARTMENT|54|43|Still western mammals pronounce normal hours. Slowly separate children re|monthly| +5768|AAAAAAAAIIGBAAAA|2451875|2451904|DEPARTMENT|54|44|Interesting neighbours control there large schools. Examples mark ever acc|monthly| +5769|AAAAAAAAJIGBAAAA|2451875|2451904|DEPARTMENT|54|45|Children clarify english, subject structures. Suitable, other organs fit already areas; d|monthly| +5770|AAAAAAAAKIGBAAAA|2451875|2451904|DEPARTMENT|54|46|British writers will not combine more. Only, reasonable words know rarely. |monthly| +5771|AAAAAAAALIGBAAAA|2451875|2451904|DEPARTMENT|54|47|However old questions might consent. Yesterday familiar policies will forget almost cor|monthly| +5772|AAAAAAAAMIGBAAAA|2451875|2451904|DEPARTMENT|54|48|Gross qualifications will happen rather together prospective boys. Fea|monthly| +5773|AAAAAAAANIGBAAAA|2451875|2451904|DEPARTMENT|54|49|Changes would not read hard sexual competitors. Good, other studies might put |monthly| +5774|AAAAAAAAOIGBAAAA|2451875|2451904|DEPARTMENT|54|50|Environmental, other clothes must not see. Thickly basic markets|monthly| +5775|AAAAAAAAPIGBAAAA|2451875|2451904|DEPARTMENT|54|51|Important, national others fight enough sexual things. Obvious, white patients say more |monthly| +5776|AAAAAAAAAJGBAAAA|2451875|2451904|DEPARTMENT|54|52|More than real words ought to execute. International sums attempt more than; scientific forces ta|monthly| +5777|AAAAAAAABJGBAAAA|2451875|2451904|DEPARTMENT|54|53|Enthusiastically absolute gardens prepare even experts. Possible drivers could keep very t|monthly| +5778|AAAAAAAACJGBAAAA|2451875|2451904|DEPARTMENT|54|54|Now palestinian streets survive please total intere|monthly| +5779|AAAAAAAADJGBAAAA|2451875|2451904|DEPARTMENT|54|55|Scenes will occur over spiritual, white ears. Good, wrong children m|monthly| +5780|AAAAAAAAEJGBAAAA|2451875|2451904|DEPARTMENT|54|56|Round, famous days might publish later even central centuries. Main, local tradit|monthly| +5781|AAAAAAAAFJGBAAAA|2451875|2451904|DEPARTMENT|54|57|Sure girls look once again discussions. Objects should let main, effective workers. Natu|monthly| +5782|AAAAAAAAGJGBAAAA|2451875|2451904|DEPARTMENT|54|58|Practical jobs can stop almost; sciences might give agains|monthly| +5783|AAAAAAAAHJGBAAAA|2451875|2451904|DEPARTMENT|54|59|Large ingredients share courses. Yet rich parties may lose directly; |monthly| +5784|AAAAAAAAIJGBAAAA|2451875|2451904|DEPARTMENT|54|60|Days find too. Objectives say today peaceful friends. Carefully corporate others may not result for|monthly| +5785|AAAAAAAAJJGBAAAA|2451875|2451904|DEPARTMENT|54|61|Gold changes want more english, ready causes; now small lawyers ought to descr|monthly| +5786|AAAAAAAAKJGBAAAA|2451875|2451904|DEPARTMENT|54|62|Ever modern police would not establish all right regulations. Clea|monthly| +5787|AAAAAAAALJGBAAAA|2451875|2451904|DEPARTMENT|54|63|Firms ought to fly young thoughts; early serious fl|monthly| +5788|AAAAAAAAMJGBAAAA|2451875|2451904|DEPARTMENT|54|64|Social, urban crowds inflict over inadequate, total|monthly| +5789|AAAAAAAANJGBAAAA|2451875|2451904|DEPARTMENT|54|65|Too faint enquiries should not follow as to a objections. Ab|monthly| +5790|AAAAAAAAOJGBAAAA|2451875|2451904|DEPARTMENT|54|66|A bit popular leaders travel just alternative cheeks. Considerable firms say to|monthly| +5791|AAAAAAAAPJGBAAAA|2451875|2451904|DEPARTMENT|54|67|Activities rush. Other, inc rocks mean thus flat months. Particularly unchanged types will promote|monthly| +5792|AAAAAAAAAKGBAAAA|2451875|2451904|DEPARTMENT|54|68|Both other activities may say possibly enough criminal mothers; linguistic police offer |monthly| +5793|AAAAAAAABKGBAAAA|2451875|2451904|DEPARTMENT|54|69|Never high affairs shall serve about a requirements. Gentle, defensi|monthly| +5794|AAAAAAAACKGBAAAA|2451875|2451904|DEPARTMENT|54|70|Much sweet humans can fill quite more following servants. Surprised songs like redu|monthly| +5795|AAAAAAAADKGBAAAA|2451875|2451904|DEPARTMENT|54|71|Alone alive appeals rise; standard aspects happen especially goods. Genuine st|monthly| +5796|AAAAAAAAEKGBAAAA|2451875|2451904|DEPARTMENT|54|72|Alone comparisons should make enough parliamentary, cheap acts. Practical witnesses shall not|monthly| +5797|AAAAAAAAFKGBAAAA|2451875|2451904|DEPARTMENT|54|73|Men may need of course. Students might not stay approximately daily seats. Traditionally briti|monthly| +5798|AAAAAAAAGKGBAAAA|2451875|2451904|DEPARTMENT|54|74|Heads seem unnecessarily main guards. Political forces must not increase liab|monthly| +5799|AAAAAAAAHKGBAAAA|2451875|2451904|DEPARTMENT|54|75|Only old problems perceive; english, clear words used to win good minutes. |monthly| +5800|AAAAAAAAIKGBAAAA|2451875|2451904|DEPARTMENT|54|76|Demands continue in a scientists. Old sciences fit too able, similar humans. Valuable, long|monthly| +5801|AAAAAAAAJKGBAAAA|2451875|2451904|DEPARTMENT|54|77|Front arrangements visit. Certainly willing columns can make actually. Likely eggs cannot p|monthly| +5802|AAAAAAAAKKGBAAAA|2451875|2451904|DEPARTMENT|54|78|Games could not keep now parliamentary, small minutes; now professional detectives|monthly| +5803|AAAAAAAALKGBAAAA|2451875|2451904|DEPARTMENT|54|79|Issues join into a acts. Previous hotels could write freshly over public ye|monthly| +5804|AAAAAAAAMKGBAAAA|2451875|2451904|DEPARTMENT|54|80|Foreign patterns ought to convert men. Effective statements kill. Foreign|monthly| +5805|AAAAAAAANKGBAAAA|2451875|2451904|DEPARTMENT|54|81|Developments cancel police. Foreign links get fiercely silly, main miners. Cold hospitals rouse i|monthly| +5806|AAAAAAAAOKGBAAAA|2451875|2451904|DEPARTMENT|54|82|Closer past children may go also parents. Fast possible eyes may say so |monthly| +5807|AAAAAAAAPKGBAAAA|2451875|2451904|DEPARTMENT|54|83|For certain orthodox experts would gather. Anxious, national child|monthly| +5808|AAAAAAAAALGBAAAA|2451875|2451904|DEPARTMENT|54|84|Objects run into the individuals. Others put again precisely american times. Plans provide now. |monthly| +5809|AAAAAAAABLGBAAAA|2451875|2451904|DEPARTMENT|54|85|Big, cheap things continue; originally initial women cut hardly q|monthly| +5810|AAAAAAAACLGBAAAA|2451875|2451904|DEPARTMENT|54|86|Late, excellent years supply always american wages. A|monthly| +5811|AAAAAAAADLGBAAAA|2451875|2451904|DEPARTMENT|54|87|Harsh drivers may note then bottom men. Direct rooms should win right rules. Dry minutes sha|monthly| +5812|AAAAAAAAELGBAAAA|2451875|2451904|DEPARTMENT|54|88|Both good qualities would let quite. Stars reject heavily|monthly| +5813|AAAAAAAAFLGBAAAA|2451875|2451904|DEPARTMENT|54|89|Able tears ought to know however central, available keys; international, illegal lea|monthly| +5814|AAAAAAAAGLGBAAAA|2451875|2451904|DEPARTMENT|54|90|Old police shall happen perhaps because of a paintings. Angry, british orga|monthly| +5815|AAAAAAAAHLGBAAAA|2451875|2451904|DEPARTMENT|54|91|Internal children close used, general authors. Suddenly different houses help interior years. Adv|monthly| +5816|AAAAAAAAILGBAAAA|2451875|2451904|DEPARTMENT|54|92|Proud steps can shoot less magnetic, common days. English days open |monthly| +5817|AAAAAAAAJLGBAAAA|2451875|2451904|DEPARTMENT|54|93|Alone locations look. Then british members ought to s|monthly| +5818|AAAAAAAAKLGBAAAA|2451875|2451904|DEPARTMENT|54|94|At present economic assumptions operate as. Circles would place well social windows. Unable, old re|monthly| +5819|AAAAAAAALLGBAAAA|2451875|2451904|DEPARTMENT|54|95|Polite arguments must ask so as right friends. Now difficult wars used to|monthly| +5820|AAAAAAAAMLGBAAAA|2451875|2451904|DEPARTMENT|54|96|Military thousands should not rely particularly leading, ready boys. So |monthly| +5821|AAAAAAAANLGBAAAA|2451875|2451904|DEPARTMENT|54|97|Linguistic, other things leave just from a elections. Key thousands may not succeed suddenly|monthly| +5822|AAAAAAAAOLGBAAAA|2451875|2451904|DEPARTMENT|54|98|Open, important heads shall cover only on the women. Convi|monthly| +5823|AAAAAAAAPLGBAAAA|2451875|2451904|DEPARTMENT|54|99|Direct passengers play western, full jews. Years invest essentially neither full effec|monthly| +5824|AAAAAAAAAMGBAAAA|2451875|2451904|DEPARTMENT|54|100|At least evolutionary countries might not see as original, social work|monthly| +5825|AAAAAAAABMGBAAAA|2451875|2451904|DEPARTMENT|54|101|Homes shall know; leaders see also obvious decisions. Mostly historic years foster. Much following|monthly| +5826|AAAAAAAACMGBAAAA|2451875|2451904|DEPARTMENT|54|102|Seemingly local ways ought to work also electronic, diffe|monthly| +5827|AAAAAAAADMGBAAAA|2451875|2451904|DEPARTMENT|54|103|Feet avoid harder. Previous, proper circumstances take free, minor books. Systems calc|monthly| +5828|AAAAAAAAEMGBAAAA|2451875|2451904|DEPARTMENT|54|104|Sensible components should give groups. Plants consider completely other things. Much|monthly| +5829|AAAAAAAAFMGBAAAA|2451875|2451904|DEPARTMENT|54|105|Long, wrong authorities see as usual full fees. Busy, top developmen|monthly| +5830|AAAAAAAAGMGBAAAA|2451875|2451904|DEPARTMENT|54|106|Over new eyes consider dark politicians. Concerned mil|monthly| +5831|AAAAAAAAHMGBAAAA|2451875|2451904|DEPARTMENT|54|107|Able, other teachers apply constantly different, nic|monthly| +5832|AAAAAAAAIMGBAAAA|2451875|2451904|DEPARTMENT|54|108|Late, main pp. might occur ill long maps. Now monetary words should not park othe|monthly| +5833|AAAAAAAAJMGBAAAA|2451910|2452091|DEPARTMENT|55|1|Wild figures should delay drily future systems; plans give therefore to a countries;|bi-annual| +5834|AAAAAAAAKMGBAAAA|2451910|2452091|DEPARTMENT|55|2|Just adequate activities levy very. Past big cases could fall professional, extensive c|bi-annual| +5835|AAAAAAAALMGBAAAA|2451910|2452091|DEPARTMENT|55|3|Red, likely pounds may not grow actual, concerned ways. Animal|bi-annual| +5836|AAAAAAAAMMGBAAAA|2451910|2452091|DEPARTMENT|55|4|Plans might not meet new, autonomous beliefs. Other, ne|bi-annual| +5837|AAAAAAAANMGBAAAA|2451910|2452091|DEPARTMENT|55|5|Nasty, potential cells must review eventually meetings. Determined hands would create. |bi-annual| +5838|AAAAAAAAOMGBAAAA|2451910|2452091|DEPARTMENT|55|6|Areas cover just never serious courses; heroes keep particularly back questions. Selec|bi-annual| +5839|AAAAAAAAPMGBAAAA|2451910|2452091|DEPARTMENT|55|7|Luckily powerful pairs bring long also dead arguments; real options join too. Years shall spend |bi-annual| +5840|AAAAAAAAANGBAAAA|2451910|2452091|DEPARTMENT|55|8|Children see books. Perhaps natural pictures say reportedly surpris|bi-annual| +5841|AAAAAAAABNGBAAAA|2451910|2452091|DEPARTMENT|55|9|Sorry scientists pull for the priests. Rules shoul|bi-annual| +5842|AAAAAAAACNGBAAAA|2451910|2452091|DEPARTMENT|55|10|Also other depths shock also. Eyes land commonly legal, comfortable |bi-annual| +5843|AAAAAAAADNGBAAAA|2451910|2452091|DEPARTMENT|55|11|Too democratic constraints may compete yet deaths. Short-term, other practices|bi-annual| +5844|AAAAAAAAENGBAAAA|2451910|2452091|DEPARTMENT|55|12|Days may not think as; naval, firm birds could escort clearly pal|bi-annual| +5845|AAAAAAAAFNGBAAAA|2451910|2452091|DEPARTMENT|55|13|Easy, legal materials must worry at first only independent pur|bi-annual| +5846|AAAAAAAAGNGBAAAA|2451910|2452091|DEPARTMENT|55|14|General problems know. Both old years separate also to a factors. Facts get as; reliable, ne|bi-annual| +5847|AAAAAAAAHNGBAAAA|2451910|2452091|DEPARTMENT|55|15|Forces must not estimate essential, possible facts. |bi-annual| +5848|AAAAAAAAINGBAAAA|2451910|2452091|DEPARTMENT|55|16|Beautiful, hard years can facilitate main, close days. Yet head b|bi-annual| +5849|AAAAAAAAJNGBAAAA|2451910|2452091|DEPARTMENT|55|17|Users see services; scottish years like fully radical, horrible purposes. |bi-annual| +5850|AAAAAAAAKNGBAAAA|2451910|2452091|DEPARTMENT|55|18|Questions might not meet still at the thousands. Beautiful ways sleep. Lengths|bi-annual| +5851|AAAAAAAALNGBAAAA|2451910|2452091|DEPARTMENT|55|19|Social objects exist really problems. Passages show underneath british, differ|bi-annual| +5852|AAAAAAAAMNGBAAAA|2451910|2452091|DEPARTMENT|55|20|Videos interfere conferences. Parliamentary, used thousands will feel especially results. More than|bi-annual| +5853|AAAAAAAANNGBAAAA|2451910|2452091|DEPARTMENT|55|21|Perfect, working changes would not come figures. Commercia|bi-annual| +5854|AAAAAAAAONGBAAAA|2451910|2452091|DEPARTMENT|55|22|A bit asleep rooms cannot feel short dry secondary leads. Ab|bi-annual| +5855|AAAAAAAAPNGBAAAA|2451910|2452091|DEPARTMENT|55|23|Thus local members must reach together possible, successful periods. International provisi|bi-annual| +5856|AAAAAAAAAOGBAAAA|2451910|2452091|DEPARTMENT|55|24|Already late activities shall not neglect sure depend|bi-annual| +5857|AAAAAAAABOGBAAAA|2451910|2452091|DEPARTMENT|55|25|Blank, good attitudes remember then various countries. Different birds could sink properly ext|bi-annual| +5858|AAAAAAAACOGBAAAA|2451910|2452091|DEPARTMENT|55|26|Over poor bonds may go originally from a generations. Scenes mean markedly ther|bi-annual| +5859|AAAAAAAADOGBAAAA|2451910|2452091|DEPARTMENT|55|27|Genes see feet. Only, good birds can take certainly only emotional years. E|bi-annual| +5860|AAAAAAAAEOGBAAAA|2451910|2452091|DEPARTMENT|55|28|Expensive, mental nations pass of course from a investmen|bi-annual| +5861|AAAAAAAAFOGBAAAA|2451910|2452091|DEPARTMENT|55|29|Elements find very at a instructions. Final, new directo|bi-annual| +5862|AAAAAAAAGOGBAAAA|2451910|2452091|DEPARTMENT|55|30|Black areas surrender half. Matters used to listen also measures. Public, western shoes shall n|bi-annual| +5863|AAAAAAAAHOGBAAAA|2451910|2452091|DEPARTMENT|55|31|All right good arrangements prevent traditional incidents. At least socia|bi-annual| +5864|AAAAAAAAIOGBAAAA|2451910|2452091|DEPARTMENT|55|32|Findings imply. Financial details would act nearly. S|bi-annual| +5865|AAAAAAAAJOGBAAAA|2451910|2452091|DEPARTMENT|55|33|Today unexpected techniques turn others. Also western lim|bi-annual| +5866|AAAAAAAAKOGBAAAA|2451910|2452091|DEPARTMENT|55|34|Free situations get black trees. Buildings used to show private prices. Maximum, following hours|bi-annual| +5867|AAAAAAAALOGBAAAA|2451910|2452091|DEPARTMENT|55|35|Countries can find right, real balls; rather chinese pieces put commonly amongst the boys. Prioriti|bi-annual| +5868|AAAAAAAAMOGBAAAA|2451910|2452091|DEPARTMENT|55|36|Quite only grants ought to give thus partners. Lat|bi-annual| +5869|AAAAAAAANOGBAAAA|2451910|2452091|DEPARTMENT|55|37|Great machines ought to appear slim, perfect police. Possible, sharp insects used|bi-annual| +5870|AAAAAAAAOOGBAAAA|2451910|2452091|DEPARTMENT|55|38|Questions tell seriously social, simple effects. Mutual, |bi-annual| +5871|AAAAAAAAPOGBAAAA|2451910|2452091|DEPARTMENT|55|39|Loose forces could manage other consumers. Conscious prices cover. Foreign arrangements |bi-annual| +5872|AAAAAAAAAPGBAAAA|2451910|2452091|DEPARTMENT|55|40|Technical residents dispel offences. Controls occur just good, gre|bi-annual| +5873|AAAAAAAABPGBAAAA|2451910|2452091|DEPARTMENT|55|41|Therefore substantial options select however single|bi-annual| +5874|AAAAAAAACPGBAAAA|2451910|2452091|DEPARTMENT|55|42|Specific, good services may go alternative elements. Also specific|bi-annual| +5875|AAAAAAAADPGBAAAA|2451910|2452091|DEPARTMENT|55|43|Constitutional, economic tears used to leave more lines; changes could decide subsequently o|bi-annual| +5876|AAAAAAAAEPGBAAAA|2451910|2452091|DEPARTMENT|55|44|Jewish, dark principles break. Always monetary consumers sell. Alone, desperate month|bi-annual| +5877|AAAAAAAAFPGBAAAA|2451910|2452091|DEPARTMENT|55|45|Substances will not ensure so without the diseases. Members shall not discuss on|bi-annual| +5878|AAAAAAAAGPGBAAAA|2451910|2452091|DEPARTMENT|55|46|Courses will create too traditional, american newspapers; communities could|bi-annual| +5879|AAAAAAAAHPGBAAAA|2451910|2452091|DEPARTMENT|55|47|Silent representatives may see still german buildings. Other papers remedy in the titles;|bi-annual| +5880|AAAAAAAAIPGBAAAA|2451910|2452091|DEPARTMENT|55|48|Workers used to complain windows. Assumptions visit today likely images. Possi|bi-annual| +5881|AAAAAAAAJPGBAAAA|2451910|2452091|DEPARTMENT|55|49|However central injuries would not succeed mainly years. Huge, person|bi-annual| +5882|AAAAAAAAKPGBAAAA|2451910|2452091|DEPARTMENT|55|50|Much secondary days can participate cool onto a technologies. Serious thanks should r|bi-annual| +5883|AAAAAAAALPGBAAAA|2451910|2452091|DEPARTMENT|55|51|Principal, right efforts ought to get at all. Significant|bi-annual| +5884|AAAAAAAAMPGBAAAA|2451910|2452091|DEPARTMENT|55|52|More roman tasks could spend difficult, interested problems. Figures support planes.|bi-annual| +5885|AAAAAAAANPGBAAAA|2451910|2452091|DEPARTMENT|55|53|Groups deploy only legislative courts. American, inst|bi-annual| +5886|AAAAAAAAOPGBAAAA|2451910|2452091|DEPARTMENT|55|54|Green losses see educational, cheerful systems; new, |bi-annual| +5887|AAAAAAAAPPGBAAAA|2451910|2452091|DEPARTMENT|55|55|Legal, clear differences take too more simple needs|bi-annual| +5888|AAAAAAAAAAHBAAAA|2451910|2452091|DEPARTMENT|55|56|Also black officers test a bit only costs. Human, slow books re|bi-annual| +5889|AAAAAAAABAHBAAAA|2451910|2452091|DEPARTMENT|55|57|Later emotional hands keep. Traditional steps deny attractive activities; full a|bi-annual| +5890|AAAAAAAACAHBAAAA|2451910|2452091|DEPARTMENT|55|58|Chief, little holidays see merely total lines. Teams would call virtually.|bi-annual| +5891|AAAAAAAADAHBAAAA|2451910|2452091|DEPARTMENT|55|59|Commercial, simple wives should exclude areas; members develop corporate, simple organizatio|bi-annual| +5892|AAAAAAAAEAHBAAAA|2451910|2452091|DEPARTMENT|55|60|Rules would direct still to a users. Only academic c|bi-annual| +5893|AAAAAAAAFAHBAAAA|2451910|2452091|DEPARTMENT|55|61|Appropriate, great months recognise perhaps poor waters. At least internatio|bi-annual| +5894|AAAAAAAAGAHBAAAA|2451910|2452091|DEPARTMENT|55|62|Accidents put too quite royal differences. Around difficult churches kick local lines. Ever good|bi-annual| +5895|AAAAAAAAHAHBAAAA|2451910|2452091|DEPARTMENT|55|63|Common, able books may promote also lives; alone, wrong legs should think |bi-annual| +5896|AAAAAAAAIAHBAAAA|2451910|2452091|DEPARTMENT|55|64|Labour, separate bases shall use; bright, special steps make rates; waves take |bi-annual| +5897|AAAAAAAAJAHBAAAA|2451910|2452091|DEPARTMENT|55|65|Free children find workers. Proper coins pin at least similar, goo|bi-annual| +5898|AAAAAAAAKAHBAAAA|2451910|2452091|DEPARTMENT|55|66|Areas may argue tory settings. Annual, direct facts see more bizarre c|bi-annual| +5899|AAAAAAAALAHBAAAA|2451910|2452091|DEPARTMENT|55|67|Similarly clear elements bring to a tables. Left minutes |bi-annual| +5900|AAAAAAAAMAHBAAAA|2451910|2452091|DEPARTMENT|55|68|Details shall want closely; wholly increased thoughts find near |bi-annual| +5901|AAAAAAAANAHBAAAA|2451910|2452091|DEPARTMENT|55|69|Odd, important liabilities get processes. Future, current standards should start v|bi-annual| +5902|AAAAAAAAOAHBAAAA|2451910|2452091|DEPARTMENT|55|70|Always vital aspects could not appear never effective, wrong studies. Professional institu|bi-annual| +5903|AAAAAAAAPAHBAAAA|2451910|2452091|DEPARTMENT|55|71|Words leave more about a years. Reluctantly social purposes can make then boring, ne|bi-annual| +5904|AAAAAAAAABHBAAAA|2451910|2452091|DEPARTMENT|55|72|Deliberately old payments occur at a services. Immediately my|bi-annual| +5905|AAAAAAAABBHBAAAA||2452091|||73||| +5906|AAAAAAAACBHBAAAA|2451910|2452091|DEPARTMENT|55|74|Characteristics must decide steady. Best reasonable statements|bi-annual| +5907|AAAAAAAADBHBAAAA|2451910|2452091|DEPARTMENT|55|75|Democrats will govern either small reports. Also routine products must not know ago; gaps|bi-annual| +5908|AAAAAAAAEBHBAAAA|2451910|2452091|DEPARTMENT|55|76|Later new animals ought to cover also in a pages. States might not look na|bi-annual| +5909|AAAAAAAAFBHBAAAA|2451910|2452091|DEPARTMENT|55|77|About environmental weeks ought to grow male trends.|bi-annual| +5910|AAAAAAAAGBHBAAAA|2451910|2452091|DEPARTMENT|55|78|Ideological stages injure sure bones. Extra associated funds shall know f|bi-annual| +5911|AAAAAAAAHBHBAAAA|2451910|2452091|DEPARTMENT|55|79|Other, concerned relations might challenge already panel|bi-annual| +5912|AAAAAAAAIBHBAAAA|2451910|2452091|DEPARTMENT|55|80|Visual, polite companies might ring wrong homes. Incredible, ruling hands remove y|bi-annual| +5913|AAAAAAAAJBHBAAAA|2451910|2452091|DEPARTMENT|55|81|Left comparisons should not end blue times. Over living ships take again equal things. |bi-annual| +5914|AAAAAAAAKBHBAAAA|2451910|2452091|DEPARTMENT|55|82|Essential, labour services can read always in general positive services; actual, low neighb|bi-annual| +5915|AAAAAAAALBHBAAAA|2451910|2452091|DEPARTMENT|55|83|Really blue children may not combine only with a prisons. Pupils hope major, favorable companies. |bi-annual| +5916|AAAAAAAAMBHBAAAA|2451910|2452091|DEPARTMENT|55|84|Hands present fine things. Royal, public members can become wi|bi-annual| +5917|AAAAAAAANBHBAAAA|2451910|2452091|DEPARTMENT|55|85|Nice things must not cause with a factors. Men take hence. Police sense all rig|bi-annual| +5918|AAAAAAAAOBHBAAAA|2451910|2452091|DEPARTMENT|55|86|Left departments get less. Contracts replace still about a st|bi-annual| +5919|AAAAAAAAPBHBAAAA|2451910|2452091|DEPARTMENT|55|87|Both strange levels would recognise annually prime, kind vehicles; vulnerable, s|bi-annual| +5920|AAAAAAAAACHBAAAA|2451910|2452091|DEPARTMENT|55|88|Days might not make full mountains. Dead old solicitors wil|bi-annual| +5921|AAAAAAAABCHBAAAA|2451910|2452091|DEPARTMENT|55|89|Things might function only similar friends. Windows shall not sing now owners. Effecti|bi-annual| +5922|AAAAAAAACCHBAAAA|2451910|2452091|DEPARTMENT|55|90|Circles want directly little results. Then rare students should win at a funds. Now opening chai|bi-annual| +5923|AAAAAAAADCHBAAAA|2451910|2452091|DEPARTMENT|55|91|Ideological aspects add new, cultural issues; complaints own unive|bi-annual| +5924|AAAAAAAAECHBAAAA|2451910|2452091|DEPARTMENT|55|92|Children buy notably wet centres. Shadows used to put surprisingly. Al|bi-annual| +5925|AAAAAAAAFCHBAAAA|2451910|2452091|DEPARTMENT|55|93|Political, famous students would meet only. Great cameras help both poor miles. |bi-annual| +5926|AAAAAAAAGCHBAAAA|2451910|2452091|DEPARTMENT|55|94|Seriously massive members would hear then to a men. Immediate,|bi-annual| +5927|AAAAAAAAHCHBAAAA|2451910|2452091|DEPARTMENT|55|95|Girls may offer maybe white lectures. Uncertain sources may know b|bi-annual| +5928|AAAAAAAAICHBAAAA|2451910|2452091|DEPARTMENT|55|96|Prices used to like sometime drinks. Ever daily years must want always for the|bi-annual| +5929|AAAAAAAAJCHBAAAA|2451910|2452091|DEPARTMENT|55|97|More happy millions might divert almost respective forms. Card|bi-annual| +5930|AAAAAAAAKCHBAAAA|2451910|2452091|DEPARTMENT|55|98|Other, young intentions used to complete never; good spirits|bi-annual| +5931|AAAAAAAALCHBAAAA|2451910|2452091|DEPARTMENT|55|99|Indeed sufficient moments should not support with the children. Others used |bi-annual| +5932|AAAAAAAAMCHBAAAA|2451910|2452091|DEPARTMENT|55|100|Regular folk help here. Soon unusual police vote probably o|bi-annual| +5933|AAAAAAAANCHBAAAA|2451910|2452091|DEPARTMENT|55|101|General lives used to react before a ages. Good kids will determine perhaps to a years. P|bi-annual| +5934|AAAAAAAAOCHBAAAA|2451910|2452091|DEPARTMENT|55|102|Slightly tired days would not make sometimes regional, future players. Weeks must take good, new|bi-annual| +5935|AAAAAAAAPCHBAAAA|2451910|2452091|DEPARTMENT|55|103|Royal, net bonds achieve formally nearly medical members. Officers can wind to a boards. Of|bi-annual| +5936|AAAAAAAAADHBAAAA|2451910|2452091|DEPARTMENT|55|104|Copies celebrate on a variations. Handsome, traditional features might come prima|bi-annual| +5937|AAAAAAAABDHBAAAA|2451910|2452091|DEPARTMENT|55|105|So scientific things divide scottish, tiny papers. Special, good mice improve then too working |bi-annual| +5938|AAAAAAAACDHBAAAA|2451910|2452091|DEPARTMENT|55|106|Holidays could succeed perhaps accused claims. Years seduce as hour|bi-annual| +5939|AAAAAAAADDHBAAAA|2451910|2452091|DEPARTMENT|55|107|Open, charming rights make more environmental organizations. Highest p|bi-annual| +5940|AAAAAAAAEDHBAAAA|2451910|2452091|DEPARTMENT|55|108|Certainly foreign authorities would not believe sup|bi-annual| +5941|AAAAAAAAFDHBAAAA|2452092|2452273|DEPARTMENT|56|1|Houses comply even symbolic, old sports. Functions might not examine only; q|bi-annual| +5942|AAAAAAAAGDHBAAAA|2452092|2452273|DEPARTMENT|56|2|Rights look half typical, economic police. Natural accounts can|bi-annual| +5943|AAAAAAAAHDHBAAAA|2452092|2452273|DEPARTMENT|56|3|Rates would receive to a groups. Years know close |bi-annual| +5944|AAAAAAAAIDHBAAAA|2452092|2452273|DEPARTMENT|56|4|Before careful differences find respectable, blue inco|bi-annual| +5945|AAAAAAAAJDHBAAAA|2452092|2452273|DEPARTMENT|56|5|Uncomfortable teachers cannot suggest only. Small eff|bi-annual| +5946|AAAAAAAAKDHBAAAA|2452092|2452273|DEPARTMENT|56|6|Only sole hands go perhaps less necessary prices; forms exist very as a grounds. Tonnes may restric|bi-annual| +5947|AAAAAAAALDHBAAAA|2452092|2452273|DEPARTMENT|56|7|Forward comparative forces will not fill to a industries. Attitudes ma|bi-annual| +5948|AAAAAAAAMDHBAAAA|2452092|2452273|DEPARTMENT|56|8|Also shallow relations get there students. International|bi-annual| +5949|AAAAAAAANDHBAAAA|2452092|2452273|DEPARTMENT|56|9|Members drop to a students. Universities might condemn just fat children. Fin|bi-annual| +5950|AAAAAAAAODHBAAAA|2452092|2452273|DEPARTMENT|56|10|Entirely conservative users operate there on a years. Poin|bi-annual| +5951|AAAAAAAAPDHBAAAA|2452092|2452273|DEPARTMENT|56|11|Local, unlike services must not think between a negotiations. Proud men fail. Unions d|bi-annual| +5952|AAAAAAAAAEHBAAAA|2452092|2452273|DEPARTMENT|56|12|Blank, old profits may suggest actually. Curiously awful areas c|bi-annual| +5953|AAAAAAAABEHBAAAA|2452092|2452273|DEPARTMENT|56|13|Once like cars hunt socially american, available colours. Better sexual things can require however |bi-annual| +5954|AAAAAAAACEHBAAAA|2452092|2452273|DEPARTMENT|56|14|Major, wet wines may not turn small, unique times. Good methods would not tel|bi-annual| +5955|AAAAAAAADEHBAAAA|2452092|2452273|DEPARTMENT|56|15|Vulnerable benefits ought to minimize previous, other variations. European solicitors believe.|bi-annual| +5956|AAAAAAAAEEHBAAAA|2452092|2452273|DEPARTMENT|56|16|Simply occasional movements become hostile answers. Pal|bi-annual| +5957|AAAAAAAAFEHBAAAA|2452092|2452273|DEPARTMENT|56|17|Only, sexual schools draw then. Changes take most with a arts. New eyes explain now s|bi-annual| +5958|AAAAAAAAGEHBAAAA|2452092|2452273|DEPARTMENT|56|18|Western, black examples think so small fortunes. There sic|bi-annual| +5959|AAAAAAAAHEHBAAAA|2452092|2452273|DEPARTMENT|56|19|More american advantages emerge to a quantities. Industrial, coherent friends formulate pr|bi-annual| +5960|AAAAAAAAIEHBAAAA|2452092|2452273|DEPARTMENT|56|20|Areas adopt finally from the windows. Total, acceptable eyes mi|bi-annual| +5961|AAAAAAAAJEHBAAAA|2452092|2452273|DEPARTMENT|56|21|Emotional things would not suffer about the illustrations. Days must think only. |bi-annual| +5962|AAAAAAAAKEHBAAAA|2452092|2452273|DEPARTMENT|56|22|New men ought to face home entire grounds. Years would|bi-annual| +5963|AAAAAAAALEHBAAAA|2452092|2452273|DEPARTMENT|56|23|Championships may find real companies. Pages perceive; months should seem vari|bi-annual| +5964|AAAAAAAAMEHBAAAA|2452092|2452273|DEPARTMENT|56|24|Later particular organisations hope there costs. Late appointments mi|bi-annual| +5965|AAAAAAAANEHBAAAA|2452092|2452273|DEPARTMENT|56|25|White practitioners drive however small values. Streets used t|bi-annual| +5966|AAAAAAAAOEHBAAAA|2452092|2452273|DEPARTMENT|56|26|Inner lips ought to develop. Particular, clear eyes may like always |bi-annual| +5967|AAAAAAAAPEHBAAAA|2452092|2452273|DEPARTMENT|56|27|Only only numbers iron early goods. More specific windows ca|bi-annual| +5968|AAAAAAAAAFHBAAAA|2452092|2452273|DEPARTMENT|56|28|Fit ships could not try well open parents. Again full l|bi-annual| +5969|AAAAAAAABFHBAAAA|2452092|2452273|DEPARTMENT|56|29|Banks cannot get in a customers. Anxious, due suppliers need about wars. Heads appear complete|bi-annual| +5970|AAAAAAAACFHBAAAA|2452092|2452273|DEPARTMENT|56|30|Foreign, southern governments should compete about final, responsible animals. Years should de|bi-annual| +5971|AAAAAAAADFHBAAAA|2452092|2452273|DEPARTMENT|56|31|Neither physical films must visit currently. Over s|bi-annual| +5972|AAAAAAAAEFHBAAAA|2452092|2452273|DEPARTMENT|56|32|Late labour pupils separate. Actually weak customers should not follow to the ye|bi-annual| +5973|AAAAAAAAFFHBAAAA|2452092|2452273|DEPARTMENT|56|33|Old men place all for example interesting nurses; particu|bi-annual| +5974|AAAAAAAAGFHBAAAA|2452092|2452273|DEPARTMENT|56|34|Formal, big changes come else female, important students|bi-annual| +5975|AAAAAAAAHFHBAAAA|2452092|2452273|DEPARTMENT|56|35|Blind users can understand foreign actors. Likely, firm powers raise probabl|bi-annual| +5976|AAAAAAAAIFHBAAAA|2452092|2452273|DEPARTMENT|56|36|American, green police would not think mistakes. Hands say again so othe|bi-annual| +5977|AAAAAAAAJFHBAAAA|2452092|2452273|DEPARTMENT|56|37|Mothers might replace scottish, industrial teachers; possible colours|bi-annual| +5978|AAAAAAAAKFHBAAAA|2452092|2452273|DEPARTMENT|56|38|Positions could open indeed recent officers. Earnings might no|bi-annual| +5979|AAAAAAAALFHBAAAA|2452092|2452273|DEPARTMENT|56|39|Educational, eastern solutions reduce so. Even necessary leaders provide circumstances.|bi-annual| +5980|AAAAAAAAMFHBAAAA|2452092|2452273|DEPARTMENT|56|40|Ethnic relatives attempt just. Always poor elections ought to sustain later certain applica|bi-annual| +5981|AAAAAAAANFHBAAAA|2452092|2452273|DEPARTMENT|56|41|Irish arrangements might not work mild modes; light services come well to an food|bi-annual| +5982|AAAAAAAAOFHBAAAA|2452092|2452273|DEPARTMENT|56|42|Final, hot characteristics draw ago with the survey|bi-annual| +5983|AAAAAAAAPFHBAAAA|2452092|2452273|DEPARTMENT|56|43|Strong, old bedrooms ought to transport true object|bi-annual| +5984|AAAAAAAAAGHBAAAA|2452092|2452273|DEPARTMENT|56|44|Bizarre, usual powers will drive. Shares leave companies. Professional ad|bi-annual| +5985|AAAAAAAABGHBAAAA|2452092|2452273|DEPARTMENT|56|45|Margins will enable so. Contacts shall annoy especially general, fo|bi-annual| +5986|AAAAAAAACGHBAAAA|2452092|2452273|DEPARTMENT|56|46|Compulsory employees resist measures. Adequately wrong bodi|bi-annual| +5987|AAAAAAAADGHBAAAA|2452092|2452273|DEPARTMENT|56|47|Already involved bodies find other creatures; police would want differently new, r|bi-annual| +5988|AAAAAAAAEGHBAAAA|2452092|2452273|DEPARTMENT|56|48|Increased feet might not restore old parts. Large-scale, possible voters woul|bi-annual| +5989|AAAAAAAAFGHBAAAA|2452092|2452273|DEPARTMENT|56|49|Numbers find. Now suitable patterns may improve more leaders. Able, si|bi-annual| +5990|AAAAAAAAGGHBAAAA|2452092|2452273|DEPARTMENT|56|50|Interested, informal pockets close enough acute measures. Indeed real |bi-annual| +5991|AAAAAAAAHGHBAAAA|2452092|2452273|DEPARTMENT|56|51|Rural, new positions may accept much months. So able bones ought to emerge ever for the|bi-annual| +5992|AAAAAAAAIGHBAAAA|2452092|2452273|DEPARTMENT|56|52|Actually temporary parts suggest modules. Originally bold gifts g|bi-annual| +5993|AAAAAAAAJGHBAAAA|2452092|2452273|DEPARTMENT|56|53|However substantial issues used to obtain. Parts can appeal for example evidently great preferenc|bi-annual| +5994|AAAAAAAAKGHBAAAA|2452092|2452273|DEPARTMENT|56|54|Really other answers must not want practical, small officials. Different mome|bi-annual| +5995|AAAAAAAALGHBAAAA|2452092|2452273|DEPARTMENT|56|55|Otherwise local grounds say strongly ahead sound forms. Hard local reductions will ask a|bi-annual| +5996|AAAAAAAAMGHBAAAA|2452092|2452273|DEPARTMENT|56|56|Large, minor workers would loose still meals; police used to bring more |bi-annual| +5997|AAAAAAAANGHBAAAA|2452092|2452273|DEPARTMENT|56|57|Words may give against a weeks. Rural rights take as ma|bi-annual| +5998|AAAAAAAAOGHBAAAA|2452092|2452273|DEPARTMENT|56|58|Districts might not notice in a bishops. Pounds choose meanwhile protective, gentle bran|bi-annual| +5999|AAAAAAAAPGHBAAAA|2452092|2452273|DEPARTMENT|56|59|Mature clients fire certainly at least miserable rules. Places will begin yeste|bi-annual| +6000|AAAAAAAAAHHBAAAA|2452092|2452273|DEPARTMENT|56|60|Legal techniques need. Patients copy somewhat interesting plans. Blue calls shall construct cl|bi-annual| +6001|AAAAAAAABHHBAAAA|2452092|2452273|DEPARTMENT|56|61|Far, significant days display. Complex, political issues make new units. Activit|bi-annual| +6002|AAAAAAAACHHBAAAA|2452092|||56|||| +6003|AAAAAAAADHHBAAAA|2452092|2452273|DEPARTMENT|56|63|Unknown, absolute opinions might meet women. Computers read recen|bi-annual| +6004|AAAAAAAAEHHBAAAA|2452092|2452273|DEPARTMENT|56|64|Tall schools specify political questions. Scottish, political parties respond here social, consi|bi-annual| +6005|AAAAAAAAFHHBAAAA|2452092|2452273|DEPARTMENT|56|65|Pockets can attend cases. Authors will ship little on a pup|bi-annual| +6006|AAAAAAAAGHHBAAAA|2452092|2452273|DEPARTMENT|56|66|Suitable players come with a costs. Considerable, experimental governments would not like.|bi-annual| +6007|AAAAAAAAHHHBAAAA|2452092|2452273|DEPARTMENT|56|67|Faces will not get too too standard schools. Alone supporters find as due stages. La|bi-annual| +6008|AAAAAAAAIHHBAAAA|2452092|2452273|DEPARTMENT|56|68|Continuous children must not inspire little literary ways. Strategic things see early then|bi-annual| +6009|AAAAAAAAJHHBAAAA|2452092|2452273|DEPARTMENT|56|69|However separate representatives can benefit as good operations. Special, final friends |bi-annual| +6010|AAAAAAAAKHHBAAAA|2452092|2452273|DEPARTMENT|56|70|Branches should rob to a beds. Forces shall not take inherently only determined stores. Most tr|bi-annual| +6011|AAAAAAAALHHBAAAA|2452092|2452273|DEPARTMENT|56|71|Happy cheeks combat fees. Social nurses shall work as other |bi-annual| +6012|AAAAAAAAMHHBAAAA|2452092|2452273|DEPARTMENT|56|72|Earnings proceed qualifications. Electric spaces take high, unlike techniques. Dead, bri|bi-annual| +6013|AAAAAAAANHHBAAAA|2452092|2452273|DEPARTMENT|56|73|Home urban patients mean nearer studies. Just like places must not direct|bi-annual| +6014|AAAAAAAAOHHBAAAA|2452092|2452273|DEPARTMENT|56|74|Months prevent much regular, active individuals. Front, special|bi-annual| +6015|AAAAAAAAPHHBAAAA|2452092|2452273|DEPARTMENT|56|75|Often nice signals shall not take earlier theoretical extensive initiative|bi-annual| +6016|AAAAAAAAAIHBAAAA|2452092|2452273|DEPARTMENT|56|76|Royal, central applications throw simply cases; good doubts mention to a parents. New,|bi-annual| +6017|AAAAAAAABIHBAAAA|2452092|2452273|DEPARTMENT|56|77|Economic, domestic others change. Old decades need continually from a m|bi-annual| +6018|AAAAAAAACIHBAAAA|2452092|2452273|DEPARTMENT|56|78|Obviously strong servants say surprisingly parliamentary minds; cle|bi-annual| +6019|AAAAAAAADIHBAAAA|2452092|2452273|DEPARTMENT|56|79|Important subjects can show easy children. New grou|bi-annual| +6020|AAAAAAAAEIHBAAAA|2452092|2452273|DEPARTMENT|56|80|Completely american terms would stop so most local farmers. Foreign, ne|bi-annual| +6021|AAAAAAAAFIHBAAAA|2452092||DEPARTMENT|56|81|Always possible boundaries obtain even so upper, legal women. Lucky, fierce neighbours happe|| +6022|AAAAAAAAGIHBAAAA|2452092|2452273|DEPARTMENT|56|82|Children might plant directly comfortable subjects. Commonly final requirem|bi-annual| +6023|AAAAAAAAHIHBAAAA|2452092|2452273|DEPARTMENT|56|83|Short controls may not rebuild home as bare strategies. Clothes ensure generally dreams|bi-annual| +6024|AAAAAAAAIIHBAAAA|2452092|2452273|DEPARTMENT|56|84|Able refugees shall not come. Moreover mere plants control aside during|bi-annual| +6025|AAAAAAAAJIHBAAAA|2452092|2452273|DEPARTMENT|56|85|Considerable, deaf ideas cannot depend operational sources. Lucky years explain |bi-annual| +6026|AAAAAAAAKIHBAAAA|2452092|2452273|DEPARTMENT|56|86|Major, tall nights remove just rather square experts. In particular close terms discuss |bi-annual| +6027|AAAAAAAALIHBAAAA|2452092|2452273|DEPARTMENT|56|87|Either wonderful islands used to predict dutch jobs. Nearby, |bi-annual| +6028|AAAAAAAAMIHBAAAA|2452092|2452273|DEPARTMENT|56|88|Positions leave so true inherent brothers. Important, other dis|bi-annual| +6029|AAAAAAAANIHBAAAA|2452092|2452273|DEPARTMENT|56|89|Additional, thick judges continue then possible, particular objections. Alternative pol|bi-annual| +6030|AAAAAAAAOIHBAAAA|2452092|2452273|DEPARTMENT|56|90|High provisions concentrate on a minutes. Later chief users may not meet only golden|bi-annual| +6031|AAAAAAAAPIHBAAAA|2452092|2452273|DEPARTMENT|56|91|Things cover gently personal, annual rights. Causes sh|bi-annual| +6032|AAAAAAAAAJHBAAAA|2452092|2452273|DEPARTMENT|56|92|European countries carry too to a problems. Others admit immediately public days; child|bi-annual| +6033|AAAAAAAABJHBAAAA|2452092|2452273|DEPARTMENT|56|93|Shows might tell of course about multiple profits. Labour things |bi-annual| +6034|AAAAAAAACJHBAAAA|2452092|2452273|DEPARTMENT|56|94|Lucky waters emerge primarily final, modern charts|bi-annual| +6035|AAAAAAAADJHBAAAA|2452092|2452273|DEPARTMENT|56|95|Duties blow please women. Relations remain however new|bi-annual| +6036|AAAAAAAAEJHBAAAA|2452092|2452273|DEPARTMENT|56|96|At least legal votes help cats. Monetary, important forces analyse against a files. Sud|bi-annual| +6037|AAAAAAAAFJHBAAAA|2452092|2452273|DEPARTMENT|56|97|Perhaps gothic cars will get for the years. Clean processes deal less very|bi-annual| +6038|AAAAAAAAGJHBAAAA|2452092|2452273|DEPARTMENT|56|98|However safe genes must not supplement armed, special concepts. Obligations shall make alone, nice |bi-annual| +6039|AAAAAAAAHJHBAAAA|2452092|2452273|DEPARTMENT|56|99|Native, sensible events believe widely since soviet countries. Now social accounts mean off a stai|bi-annual| +6040|AAAAAAAAIJHBAAAA|2452092|2452273|DEPARTMENT|56|100|Afterwards suspicious students might bear necessarily minimum loss|bi-annual| +6041|AAAAAAAAJJHBAAAA|2452092|2452273|DEPARTMENT|56|101|As substantial lads see difficult ages; schools know also. Pairs convince in order with a|bi-annual| +6042|AAAAAAAAKJHBAAAA|2452092|2452273|DEPARTMENT|56|102|Authorities make less scottish, intact increases. Sim|bi-annual| +6043|AAAAAAAALJHBAAAA|2452092|2452273|DEPARTMENT|56|103|Enormous, young attacks leave from a servants; excelle|bi-annual| +6044|AAAAAAAAMJHBAAAA|2452092|2452273|DEPARTMENT|56|104|Views find. Good parents overcome tests. Double, light functions can travel massiv|bi-annual| +6045|AAAAAAAANJHBAAAA|2452092|2452273|DEPARTMENT|56|105|Friends cannot throw conferences. Visible, unpleasant miles cut reportedly contin|bi-annual| +6046|AAAAAAAAOJHBAAAA|2452092|2452273|DEPARTMENT|56|106|Domestic bands ought to extend in the results. Companies attend sharply early oth|bi-annual| +6047|AAAAAAAAPJHBAAAA|2452092|2452273|DEPARTMENT|56|107|Markets analyse similar, negative reasons. Corporate, o|bi-annual| +6048|AAAAAAAAAKHBAAAA|2452092|2452273|DEPARTMENT|56|108|Pupils could not please. Heavy architects express accordin|bi-annual| +6049|AAAAAAAABKHBAAAA|2451910|2452000|DEPARTMENT|57|1|Red ideas may leave later in a rats; federal, modern months work originally times. Under way n|quarterly| +6050|AAAAAAAACKHBAAAA|2451910|2452000|DEPARTMENT|57|2|Different, national christians ask more remarkable men; times m|quarterly| +6051|AAAAAAAADKHBAAAA|2451910|2452000|DEPARTMENT|57|3|National, english eyes could think for example by a faces. Affectionately good kinds h|quarterly| +6052|AAAAAAAAEKHBAAAA|2451910|2452000|DEPARTMENT|57|4|Social, old territories used to weaken in an communities. Never green letters should mak|quarterly| +6053|AAAAAAAAFKHBAAAA|2451910|2452000|DEPARTMENT|57|5|Busy units should last well very real occasions. International, other cases would c|quarterly| +6054|AAAAAAAAGKHBAAAA|2451910|2452000|DEPARTMENT|57|6|Acute, general thanks might not show enough public, tr|quarterly| +6055|AAAAAAAAHKHBAAAA|2451910|2452000|DEPARTMENT|57|7|Major, various trades pay once. Complete contents recognise merely simple, profitable walls; alte|quarterly| +6056|AAAAAAAAIKHBAAAA|2451910|2452000|DEPARTMENT|57|8|Tough traditions must eat like plain men. Minutes must see thus by the months; ge|quarterly| +6057|AAAAAAAAJKHBAAAA|2451910|2452000|DEPARTMENT|57|9|Tonight considerable results argue false, main develop|quarterly| +6058|AAAAAAAAKKHBAAAA|2451910|2452000|DEPARTMENT|57|10|Patients could react apparently children; quite new teachers used to muster especially inter|quarterly| +6059|AAAAAAAALKHBAAAA|2451910|2452000|DEPARTMENT|57|11|Other, exact responses used to kill unknown schemes. Great, fond areas tell; young thoughts ou|quarterly| +6060|AAAAAAAAMKHBAAAA|2451910|2452000|DEPARTMENT|57|12|Children will not provoke more in a jobs. Also black activ|quarterly| +6061|AAAAAAAANKHBAAAA|2451910|2452000|DEPARTMENT|57|13|Columns must not say to a shapes. Mere, light actions must not recognise mildly certain, |quarterly| +6062|AAAAAAAAOKHBAAAA|2451910|2452000|DEPARTMENT|57|14|Employees shall need however women; close pieces demand in a word|quarterly| +6063|AAAAAAAAPKHBAAAA|2451910|2452000|DEPARTMENT|57|15|Negative problems should not worry also documents. Social arrangements watch clearly spe|quarterly| +6064|AAAAAAAAALHBAAAA|2451910|2452000|DEPARTMENT|57|16|Then adult contexts may defend all; similar, good negotiations might explain under |quarterly| +6065|AAAAAAAABLHBAAAA|2451910|2452000|DEPARTMENT|57|17|Activities can contain. Prices should eliminate more great, serious milli|quarterly| +6066|AAAAAAAACLHBAAAA|2451910|2452000|DEPARTMENT|57|18|Just true trees say days. Christian legs must not see as in t|quarterly| +6067|AAAAAAAADLHBAAAA|2451910|2452000|DEPARTMENT|57|19|For example local needs used to interfere only rich formal windows. Also bitter group|quarterly| +6068|AAAAAAAAELHBAAAA|2451910|2452000|DEPARTMENT|57|20|Earlier net firms happen only workers. Single losses lead more clear questio|quarterly| +6069|AAAAAAAAFLHBAAAA|2451910|2452000|DEPARTMENT|57|21|Publishers ought to assure more for example different females. Regular times could not lock. Open|quarterly| +6070|AAAAAAAAGLHBAAAA|2451910|2452000|DEPARTMENT|57|22|Gently dead women lose available, toxic businesses. Well initial figures use almost difficult f|quarterly| +6071|AAAAAAAAHLHBAAAA|2451910|2452000|DEPARTMENT|57|23|Also deep pieces maintain grand videos. Medical standards dat|quarterly| +6072|AAAAAAAAILHBAAAA|2451910|2452000|DEPARTMENT|57|24|Clear, powerful cups build cases. Ill, tragic days shall not |quarterly| +6073|AAAAAAAAJLHBAAAA|2451910|2452000|DEPARTMENT|57|25|Ethnic police shall acquire hardly due to a experiences. Public, random fruits bec|quarterly| +6074|AAAAAAAAKLHBAAAA|2451910|2452000|DEPARTMENT|57|26|Romantic weapons get early, other months. Reasonably level months red|quarterly| +6075|AAAAAAAALLHBAAAA|2451910|2452000|DEPARTMENT|57|27|Certain conditions go simply similar, official classes. Cases cannot remain there sexual pro|quarterly| +6076|AAAAAAAAMLHBAAAA|2451910|2452000|DEPARTMENT|57|28|For example huge wages ask active, worthy men. New, industrial claims could |quarterly| +6077|AAAAAAAANLHBAAAA|2451910|2452000|DEPARTMENT|57|29|Relations pinpoint hard from a patients. Holes shall get p|quarterly| +6078|AAAAAAAAOLHBAAAA|2451910|2452000|DEPARTMENT|57|30|Old men believe joint, small standards; excessive proposa|quarterly| +6079|AAAAAAAAPLHBAAAA|2451910|2452000|DEPARTMENT|57|31|Aware weeks will run increased states; other participants w|quarterly| +6080|AAAAAAAAAMHBAAAA|2451910|2452000|DEPARTMENT|57|32|Inappropriate, new centuries would set slow, inevitable years. Good, su|quarterly| +6081|AAAAAAAABMHBAAAA|2451910|2452000|DEPARTMENT|57|33|Types review later great, specific methods. Chemical, american feet go. Things find earlier|quarterly| +6082|AAAAAAAACMHBAAAA|2451910|2452000|DEPARTMENT|57|34|Particularly obvious years maintain obviously economic politicians. Monthly, other affairs w|quarterly| +6083|AAAAAAAADMHBAAAA|2451910|2452000|DEPARTMENT|57|35|Other lights would limit then british conditions. Functions see considerable, prime arts. E|quarterly| +6084|AAAAAAAAEMHBAAAA|2451910|2452000|DEPARTMENT|57|36|Deeply dead definitions can go. Excellent, thin miles depart qu|quarterly| +6085|AAAAAAAAFMHBAAAA|2451910|2452000|DEPARTMENT|57|37|Equal duties may get today finally labour copies. Endless, |quarterly| +6086|AAAAAAAAGMHBAAAA|2451910|2452000|DEPARTMENT|57|38|There tight books should not fall mainly other, clear product|quarterly| +6087|AAAAAAAAHMHBAAAA|2451910|2452000|DEPARTMENT|57|39|However little members may see as odd funds; american, wonderful pp. must go also men. Econo|quarterly| +6088|AAAAAAAAIMHBAAAA|2451910|2452000|DEPARTMENT|57|40|Permanent, delighted cities could remind otherwise powerful, large-scale movements; menta|quarterly| +6089|AAAAAAAAJMHBAAAA|2451910|2452000|DEPARTMENT|57|41|Even bad traders used to face well; hastily little syste|quarterly| +6090|AAAAAAAAKMHBAAAA|2451910|2452000|DEPARTMENT|57|42|Temperatures like then fresh, additional funds. Suitable |quarterly| +6091|AAAAAAAALMHBAAAA|2451910|2452000|DEPARTMENT|57|43|Private, regular tools keep. Silently possible months occur classic, internation|quarterly| +6092|AAAAAAAAMMHBAAAA|2451910|2452000|DEPARTMENT|57|44|Specific arms must play all right right, royal resources. Coins go. Big days get serious proport|quarterly| +6093|AAAAAAAANMHBAAAA|2451910|2452000|DEPARTMENT|57|45|Tremendously standard men live rather terms. By no means military weeks must adapt available st|quarterly| +6094|AAAAAAAAOMHBAAAA|2451910|2452000|DEPARTMENT|57|46|Consultants used to keep in a functions. Women must no|quarterly| +6095|AAAAAAAAPMHBAAAA|2451910|2452000|DEPARTMENT|57|47|Most final investigations take as more true plants. Yet possible other|quarterly| +6096|AAAAAAAAANHBAAAA|2451910|2452000|DEPARTMENT|57|48|Employees wish as needs. Thoughts locate; always so-call|quarterly| +6097|AAAAAAAABNHBAAAA|2451910|2452000|DEPARTMENT|57|49|Personal miles risk always openly responsible particles. Labour, pr|quarterly| +6098|AAAAAAAACNHBAAAA|2451910|2452000|DEPARTMENT|57|50|Sure, wild organizations used to participate white, female assets. Extensive, out|quarterly| +6099|AAAAAAAADNHBAAAA|2451910|2452000|DEPARTMENT|57|51|Types know low, new assessments. Practices interrupt|quarterly| +6100|AAAAAAAAENHBAAAA||2452000|DEPARTMENT||||| +6101|AAAAAAAAFNHBAAAA|2451910|2452000|DEPARTMENT|57|53|Alternatively southern targets review so managers. Legally|quarterly| +6102|AAAAAAAAGNHBAAAA|2451910|2452000|DEPARTMENT|57|54|Men feel most traditionally wild records. Effective, sensitive patterns would n|quarterly| +6103|AAAAAAAAHNHBAAAA|2451910|2452000|DEPARTMENT|57|55|Men return on the fingers. British, live minutes could think just companies; high,|quarterly| +6104|AAAAAAAAINHBAAAA|2451910|2452000|DEPARTMENT|57|56|Available hands shall not create over a arts. White, great|quarterly| +6105|AAAAAAAAJNHBAAAA|2451910|2452000|DEPARTMENT|57|57|Dogs improve; glasses say yet. Chief, current losses may not see. Exc|quarterly| +6106|AAAAAAAAKNHBAAAA|2451910|2452000|DEPARTMENT|57|58|Casual, southern changes shall reclaim with a matters. Dishes support international, safe |quarterly| +6107|AAAAAAAALNHBAAAA|2451910|2452000|DEPARTMENT|57|59|Members set always industrial, local arms. Prayers could a|quarterly| +6108|AAAAAAAAMNHBAAAA|2451910|2452000|DEPARTMENT|57|60|Responsible, social procedures like. Recently poor divisions may ap|quarterly| +6109|AAAAAAAANNHBAAAA|2451910|2452000|DEPARTMENT|57|61|Then different members could lose for a feelings. Different stories w|quarterly| +6110|AAAAAAAAONHBAAAA|2451910|2452000|DEPARTMENT|57|62|Readers cannot provide too internal, endless ideas. Critical bacteria |quarterly| +6111|AAAAAAAAPNHBAAAA|2451910|2452000|DEPARTMENT|57|63|Subject, other clients shall appear novel cities. Vertical, statutory prizes shall sign west|quarterly| +6112|AAAAAAAAAOHBAAAA|2451910|2452000|DEPARTMENT|57|64|So false books should meet merely. More clear sets should visit legal tactics. |quarterly| +6113|AAAAAAAABOHBAAAA|2451910|2452000|DEPARTMENT|57|65|Good banks see speeches. Corporate authorities follow british, running muscl|quarterly| +6114|AAAAAAAACOHBAAAA|2451910|2452000|DEPARTMENT|57|66|Little, new members continue totally public, russian times. |quarterly| +6115|AAAAAAAADOHBAAAA|2451910|2452000|DEPARTMENT|57|67|Categories affect mutual topics. Goals will not sort films; elections must figu|quarterly| +6116|AAAAAAAAEOHBAAAA|2451910|2452000|DEPARTMENT|57|68|Fundamental ends shall encounter great, new flowers. Long, chines|quarterly| +6117|AAAAAAAAFOHBAAAA|2451910|2452000|DEPARTMENT|57|69|Top contracts should not wait ever free, various years. Great|quarterly| +6118|AAAAAAAAGOHBAAAA|2451910|2452000|DEPARTMENT|57|70|Other parts reach already with a factors. However available professionals mobilize for a met|quarterly| +6119|AAAAAAAAHOHBAAAA|2451910|2452000|DEPARTMENT|57|71|Soon bad things cannot call now always social students. Different, royal tr|quarterly| +6120|AAAAAAAAIOHBAAAA|2451910|2452000|DEPARTMENT|57|72|More necessary children ought to see here national physical years; yet major matters stan|quarterly| +6121|AAAAAAAAJOHBAAAA|2451910|2452000|DEPARTMENT|57|73|Enough british votes undergo. Reasonable, environmental years may provide adequat|quarterly| +6122|AAAAAAAAKOHBAAAA|2451910|2452000|DEPARTMENT|57|74|Expenses lay only workers. Common days could save without a provisions. Common, strong m|quarterly| +6123|AAAAAAAALOHBAAAA|2451910|2452000|DEPARTMENT|57|75|Drugs shall think most pubs. Natural legs could hide only either good standards. More hid|quarterly| +6124|AAAAAAAAMOHBAAAA|2451910|2452000|DEPARTMENT|57|76|Able objects shall prove now terrible factors. Direct, wrong procedures take changes. Cheerfull|quarterly| +6125|AAAAAAAANOHBAAAA|2451910|2452000|DEPARTMENT|57|77|Even rich results must not run till a parents. Days identify widely gardens; minutes conf|quarterly| +6126|AAAAAAAAOOHBAAAA|2451910|2452000|DEPARTMENT|57|78|Other, likely comments might not cover quietly political courses. Cold ways should enable eventual|quarterly| +6127|AAAAAAAAPOHBAAAA|2451910|2452000|DEPARTMENT|57|79|Also liberal pp. would make now countries. New, other|quarterly| +6128|AAAAAAAAAPHBAAAA|2451910|2452000|DEPARTMENT|57|80|In addition similar musicians should see individuals. Irrelevant films deny so after the |quarterly| +6129|AAAAAAAABPHBAAAA|2451910|2452000|DEPARTMENT|57|81|Big troops express. Especially other eyes can curb conservative shops. Only, single dep|quarterly| +6130|AAAAAAAACPHBAAAA|2451910|2452000|DEPARTMENT|57|82|Political, old women inform more major, short plants. Units agree unexpecte|quarterly| +6131|AAAAAAAADPHBAAAA|2451910|2452000|DEPARTMENT|57|83|Schools cross now men. English, long children become in a|quarterly| +6132|AAAAAAAAEPHBAAAA|2451910|2452000|DEPARTMENT|57|84|Under new conventions may seek finally branches. Clear films believe. Possible trusts cannot kee|quarterly| +6133|AAAAAAAAFPHBAAAA|2451910|2452000|DEPARTMENT|57|85|Please critical ingredients contact clearly federal oc|quarterly| +6134|AAAAAAAAGPHBAAAA|2451910|2452000|DEPARTMENT|57|86|Errors accompany absolutely alone great contracts. Great temperatures impro|quarterly| +6135|AAAAAAAAHPHBAAAA|2451910|2452000|DEPARTMENT|57|87|Users will not salvage. Essential, industrial visitors will feel ever. |quarterly| +6136|AAAAAAAAIPHBAAAA|2451910|2452000|DEPARTMENT|57|88|Houses use in a groups; new eyes dismiss feet; popular situa|quarterly| +6137|AAAAAAAAJPHBAAAA|2451910|2452000|DEPARTMENT|57|89|Cards used to destroy far working, free studies. Signs put quantities. |quarterly| +6138|AAAAAAAAKPHBAAAA|2451910|2452000|DEPARTMENT|57|90|Content, psychiatric traditions break for the boys. Neat, annual patients aim before even natura|quarterly| +6139|AAAAAAAALPHBAAAA|2451910|2452000|DEPARTMENT|57|91|So able reductions will help proper, poor efforts. |quarterly| +6140|AAAAAAAAMPHBAAAA|2451910|2452000|DEPARTMENT|57|92|As yet colonial rights move daily police. Labour measures sense uncertainly public police|quarterly| +6141|AAAAAAAANPHBAAAA|2451910|2452000|DEPARTMENT|57|93|Now independent securities will determine sufficiently in a products. Other years|quarterly| +6142|AAAAAAAAOPHBAAAA|2451910|2452000|DEPARTMENT|57|94|Far public shadows last public, armed decisions. So important towns would mean expressio|quarterly| +6143|AAAAAAAAPPHBAAAA|2451910|2452000|DEPARTMENT|57|95|Then low fields make less than tiny, biological are|quarterly| +6144|AAAAAAAAAAIBAAAA|2451910|2452000|DEPARTMENT|57|96|So rich banks might not become main, recent places. National section|quarterly| +6145|AAAAAAAABAIBAAAA|2451910|2452000|DEPARTMENT|57|97|Deeply fast musicians would result somewhere about a times. Yet damp homes achieve agains|quarterly| +6146|AAAAAAAACAIBAAAA|2451910|2452000|DEPARTMENT|57|98|Merely able types act about recent officers. Services can get into a advantages. Victims work e|quarterly| +6147|AAAAAAAADAIBAAAA|2451910|2452000|DEPARTMENT|57|99|Changes might ensure buildings. Levels undergo right, e|quarterly| +6148|AAAAAAAAEAIBAAAA|2451910|2452000|DEPARTMENT|57|100|Primarily red services transfer leading voices. Originally rus|quarterly| +6149|AAAAAAAAFAIBAAAA|2451910|2452000|DEPARTMENT|57|101|Significant years ought to occur round, obvious patients. Eu|quarterly| +6150|AAAAAAAAGAIBAAAA|2451910|2452000|DEPARTMENT|57|102|Inside linear children give talks. Hundreds will not free equally prime leads. More than o|quarterly| +6151|AAAAAAAAHAIBAAAA|2451910|2452000|DEPARTMENT|57|103|Equal, professional claims could ask for a sales. Little industrial requir|quarterly| +6152|AAAAAAAAIAIBAAAA|2451910|2452000|DEPARTMENT|57|104|Rules must not miss also. Growing losses might stay right sports. Dark schemes t|quarterly| +6153|AAAAAAAAJAIBAAAA|2451910|2452000|DEPARTMENT|57|105|Resources will exercise even american ideas. Too small models compete years. Percept|quarterly| +6154|AAAAAAAAKAIBAAAA|2451910|2452000|DEPARTMENT|57|106|Certain, current titles could observe keenly states.|quarterly| +6155|AAAAAAAALAIBAAAA|2451910|2452000|DEPARTMENT|57|107|Black things may not double sometimes rapid, entire roles. Simple, old documents cope wide ris|quarterly| +6156|AAAAAAAAMAIBAAAA|2451910|2452000|DEPARTMENT|57|108|Police shall not refer thereafter consequences. Sh|quarterly| +6157|AAAAAAAANAIBAAAA|2452001|2452091|DEPARTMENT|58|1|Existing years could think all mere communications. True products record; managers sh|quarterly| +6158|AAAAAAAAOAIBAAAA|2452001|2452091|DEPARTMENT|58|2|Whole, separate records might not comfort correct pressures. Entire, careful resource|quarterly| +6159|AAAAAAAAPAIBAAAA|2452001|2452091|DEPARTMENT|58|3|Drawings sink simply open lawyers. Recent, good discussions turn common forces. Ex|quarterly| +6160|AAAAAAAAABIBAAAA|2452001|2452091|DEPARTMENT|58|4|Often soft police see then particular, national brothers. Suddenly |quarterly| +6161|AAAAAAAABBIBAAAA|2452001|2452091|DEPARTMENT|58|5|Components develop indeed in a problems. Whole conservatives appear nice, sub|quarterly| +6162|AAAAAAAACBIBAAAA|2452001|2452091|DEPARTMENT|58|6|Young, moral arms will want mental priorities. German, inner reasons will love. Immedia|quarterly| +6163|AAAAAAAADBIBAAAA|2452001|2452091|DEPARTMENT|58|7|Content things imagine members. Gradually prime columns|quarterly| +6164|AAAAAAAAEBIBAAAA|2452001|2452091|DEPARTMENT|58|8|Right industrial employers invite ever responses; |quarterly| +6165|AAAAAAAAFBIBAAAA|2452001|2452091|DEPARTMENT|58|9|Perfect workers would check still in a men. Aware cases defend utterly serious fi|quarterly| +6166|AAAAAAAAGBIBAAAA|2452001|2452091|DEPARTMENT|58|10|Total millions get. All blank areas help further in a prod|quarterly| +6167|AAAAAAAAHBIBAAAA|2452001|2452091|DEPARTMENT|58|11|Authors used to include previous areas. Modern, legal groups |quarterly| +6168|AAAAAAAAIBIBAAAA|2452001|2452091|DEPARTMENT|58|12|Modern, creative prices try often attractive others. Ot|quarterly| +6169|AAAAAAAAJBIBAAAA|2452001|2452091|DEPARTMENT|58|13|Jobs shall find politely objectives. Loudly young events employ caref|quarterly| +6170|AAAAAAAAKBIBAAAA|2452001|2452091|DEPARTMENT|58|14|Objective meanings must go only. Technical, international communists may risk. Of course soc|quarterly| +6171|AAAAAAAALBIBAAAA|2452001|2452091|DEPARTMENT|58|15|Capital programmes sell then around a clothes. Various, various resources shall not follow n|quarterly| +6172|AAAAAAAAMBIBAAAA|2452001|2452091|DEPARTMENT|58|16|Teenage parties used to declare important thousands. Years help exactly. Public, a|quarterly| +6173|AAAAAAAANBIBAAAA|2452001|2452091|DEPARTMENT|58|17|New, good trials silence adversely women. Often wide details us|quarterly| +6174|AAAAAAAAOBIBAAAA|2452001|2452091|DEPARTMENT|58|18|Identical, accessible members look once parts. More main bones could respond on|quarterly| +6175|AAAAAAAAPBIBAAAA|2452001|2452091|DEPARTMENT|58|19|Primary, violent ministers might come chief prices; centuries sell heavily. Private, deep pare|quarterly| +6176|AAAAAAAAACIBAAAA|2452001|2452091|DEPARTMENT|58|20|Parameters must pass well like a feet. Difficult publications use never sexual pol|quarterly| +6177|AAAAAAAABCIBAAAA|2452001|2452091|DEPARTMENT|58|21|Total, whole forces report often soviet police. Perhaps lon|quarterly| +6178|AAAAAAAACCIBAAAA|2452001|2452091|DEPARTMENT|58|22|Generous, local chemicals secure quiet periods. Dangerous types must apply then similar|quarterly| +6179|AAAAAAAADCIBAAAA|2452001|2452091|DEPARTMENT|58|23|Holes shall assist outside reports. Complete resources shall not say shares. Terms ask|quarterly| +6180|AAAAAAAAECIBAAAA||2452091|DEPARTMENT|58|||| +6181|AAAAAAAAFCIBAAAA|2452001|2452091|DEPARTMENT|58|25|Mild posts can support really familiar years. Indeed silver increases may say overall here co|quarterly| +6182|AAAAAAAAGCIBAAAA|2452001|2452091|DEPARTMENT|58|26|Gently contrary plans should not follow sorts. Attractive, attractive marks look now|quarterly| +6183|AAAAAAAAHCIBAAAA|2452001|2452091|DEPARTMENT|58|27|Famous chips depend usually international machines. Even large|quarterly| +6184|AAAAAAAAICIBAAAA|2452001|2452091|DEPARTMENT|58|28|Pieces should tend. So large libraries cost only right|quarterly| +6185|AAAAAAAAJCIBAAAA|2452001|2452091|DEPARTMENT|58|29|Trees will seem apparently lines. Beds take less play|quarterly| +6186|AAAAAAAAKCIBAAAA|2452001|2452091|DEPARTMENT|58|30|Australian, desperate friends come actually local rivals. Simply close positions hide to|quarterly| +6187|AAAAAAAALCIBAAAA|2452001|2452091|DEPARTMENT|58|31|Developing targets would not form recently at an values|quarterly| +6188|AAAAAAAAMCIBAAAA|2452001|2452091|DEPARTMENT|58|32|Written authorities take tall, double sections. Likely, marked second|quarterly| +6189|AAAAAAAANCIBAAAA|2452001|2452091|DEPARTMENT|58|33|Likely arms must raise ahead different regions; left|quarterly| +6190|AAAAAAAAOCIBAAAA|2452001|2452091|DEPARTMENT|58|34|Other, surprising companies import just complicated others. Of course|quarterly| +6191|AAAAAAAAPCIBAAAA|2452001|2452091|DEPARTMENT|58|35|Universal leaders cool by no means goods. Working, other characters mi|quarterly| +6192|AAAAAAAAADIBAAAA|2452001|2452091|DEPARTMENT|58|36|Women get just. Articles impose often basic colleges. Glad, chief items shall resign fresh, l|quarterly| +6193|AAAAAAAABDIBAAAA|2452001|2452091|DEPARTMENT|58|37|Old illustrations understand families; fun, successful products offer rathe|quarterly| +6194|AAAAAAAACDIBAAAA|2452001|2452091|DEPARTMENT|58|38|Global measurements must not empty circumstances. White, religious ind|quarterly| +6195|AAAAAAAADDIBAAAA|2452001|2452091|DEPARTMENT|58|39|Tall, good services equate. Other days may make young companies. Of course acceptable p|quarterly| +6196|AAAAAAAAEDIBAAAA|2452001|2452091|DEPARTMENT|58|40|Historical, international kilometres take; old, good others could not resul|quarterly| +6197|AAAAAAAAFDIBAAAA|2452001|2452091|DEPARTMENT|58|41|Brief offices evaluate individuals. Distinctive, legal consequences ought t|quarterly| +6198|AAAAAAAAGDIBAAAA|2452001|2452091|DEPARTMENT|58|42|Pupils constitute anywhere days; jobs must know mainly potential skills. Professional days g|quarterly| +6199|AAAAAAAAHDIBAAAA|2452001|2452091|DEPARTMENT|58|43|Modern, criminal occasions represent too often living |quarterly| +6200|AAAAAAAAIDIBAAAA|2452001|2452091|DEPARTMENT|58|44|Chapters would see. Then young patients make conservative, full eyes. H|quarterly| +6201|AAAAAAAAJDIBAAAA|2452001|2452091|DEPARTMENT|58|45|Good, other words must go conservatives. Full, important officials can give more political s|quarterly| +6202|AAAAAAAAKDIBAAAA|2452001|2452091|DEPARTMENT|58|46|National incentives may drink as authorities. Teach|quarterly| +6203|AAAAAAAALDIBAAAA|2452001|2452091|DEPARTMENT|58|47|Clear, inner imports play increasingly. Men remember impossible sides. Statistical, encouragin|quarterly| +6204|AAAAAAAAMDIBAAAA|2452001|2452091|DEPARTMENT|58|48|Aware, available ways might see; magazines play also; there|quarterly| +6205|AAAAAAAANDIBAAAA|2452001|2452091|DEPARTMENT|58|49|Years shall save so complete writers. Famous decisions might |quarterly| +6206|AAAAAAAAODIBAAAA|2452001|2452091|DEPARTMENT|58|50|Independently defensive points will set indeed. Sharp, ancient |quarterly| +6207|AAAAAAAAPDIBAAAA|2452001|2452091|DEPARTMENT|58|51|Problems will start classical, optimistic problems. Bodies |quarterly| +6208|AAAAAAAAAEIBAAAA|2452001|2452091|DEPARTMENT|58|52|Primary cases give successfully easy friends. Old, crucial men might |quarterly| +6209|AAAAAAAABEIBAAAA|2452001|2452091|DEPARTMENT|58|53|Sure components open at least. Photographs will choose too pretty unions; members must not so|quarterly| +6210|AAAAAAAACEIBAAAA|2452001|2452091|DEPARTMENT|58|54|At least social daughters assess much words. Black methods retain specifically to a i|quarterly| +6211|AAAAAAAADEIBAAAA|2452001|2452091|DEPARTMENT|58|55|Most scottish studies used to assess tall newspapers. Good fingers study |quarterly| +6212|AAAAAAAAEEIBAAAA|2452001|2452091|DEPARTMENT|58|56|Great, burning persons find. Children hold currently professional officials. Inappropriate r|quarterly| +6213|AAAAAAAAFEIBAAAA|2452001|2452091|DEPARTMENT|58|57|Military cases will see major posts. Lovers forget surprisingly by the notions. Mai|quarterly| +6214|AAAAAAAAGEIBAAAA|2452001|2452091|DEPARTMENT|58|58|Days put. Little, normal proceedings must not reflect safely dead estates; parents tak|quarterly| +6215|AAAAAAAAHEIBAAAA|2452001|2452091|DEPARTMENT|58|59|Less cold years used to complain twice; physically upper systems might not establish almost|quarterly| +6216|AAAAAAAAIEIBAAAA|2452001|2452091|DEPARTMENT|58|60|Distant, mean plants should not listen home remaining sessions. Professional way|quarterly| +6217|AAAAAAAAJEIBAAAA|2452001|2452091|DEPARTMENT|58|61|Still typical arms shall vote home african, different men. European, willing weakne|quarterly| +6218|AAAAAAAAKEIBAAAA|2452001|2452091|DEPARTMENT|58|62|More than poor tanks make national questions. Problems |quarterly| +6219|AAAAAAAALEIBAAAA|2452001|2452091|DEPARTMENT|58|63|Old, only thousands must fetch as lonely, elderly schools; proposal|quarterly| +6220|AAAAAAAAMEIBAAAA|2452001|2452091|DEPARTMENT|58|64|Inevitably blue forces could take probably new, economic eyes. Systems shou|quarterly| +6221|AAAAAAAANEIBAAAA|2452001|2452091|DEPARTMENT|58|65|Occasionally full properties hunt to the actions. Groups might see away often val|quarterly| +6222|AAAAAAAAOEIBAAAA|2452001|2452091|DEPARTMENT|58|66|Powerful, annual resources may blame otherwise much exciting thousands. Late other terms cannot i|quarterly| +6223|AAAAAAAAPEIBAAAA|2452001|2452091|DEPARTMENT|58|67|Supporters work fine, bad bands; twice just weeks make surely public sections; great, insufficient|quarterly| +6224|AAAAAAAAAFIBAAAA|2452001|2452091|DEPARTMENT|58|68|Results should believe today particularly local countries. Organisations enhance firm, short |quarterly| +6225|AAAAAAAABFIBAAAA|2452001|2452091|DEPARTMENT|58|69|Top, other hands start seldom for a signs. Concerne|quarterly| +6226|AAAAAAAACFIBAAAA|2452001|2452091|DEPARTMENT|58|70|Substantially nuclear sources remember existing rules.|quarterly| +6227|AAAAAAAADFIBAAAA|2452001|2452091|DEPARTMENT|58|71|Examples will know particularly as isolated doctors. Fortu|quarterly| +6228|AAAAAAAAEFIBAAAA|2452001|2452091|DEPARTMENT|58|72|Permanent terms preclude a little at first local facilities. Forw|quarterly| +6229|AAAAAAAAFFIBAAAA|2452001|2452091|DEPARTMENT|58|73|Barriers go local feet. Hardly social conditions arise like the courses.|quarterly| +6230|AAAAAAAAGFIBAAAA|2452001|2452091|DEPARTMENT|58|74|Also other circumstances remain still grants. Fun, commercial affairs oug|quarterly| +6231|AAAAAAAAHFIBAAAA|2452001|2452091|DEPARTMENT|58|75|Light shoes take historical, industrial places. Children must go now british patients. Clearly due |quarterly| +6232|AAAAAAAAIFIBAAAA|2452001|2452091|DEPARTMENT|58|76|Roles provide in the conditions. Evidently interesting pages should not move various accoun|quarterly| +6233|AAAAAAAAJFIBAAAA|2452001|2452091|DEPARTMENT|58|77|Very weeks mean often here new models. Chief masters used to show qui|quarterly| +6234|AAAAAAAAKFIBAAAA|2452001|2452091|DEPARTMENT|58|78|Small, interesting premises come by the poems; simply high results might eat |quarterly| +6235|AAAAAAAALFIBAAAA|2452001|2452091|DEPARTMENT|58|79|Again civil criticisms agree relationships. Plain, typical costs continue. Often public rivers wil|quarterly| +6236|AAAAAAAAMFIBAAAA|2452001|2452091|DEPARTMENT|58|80|Heavy, considerable authorities can hold equally arms. Other, low steps used to like soon. Ev|quarterly| +6237|AAAAAAAANFIBAAAA|2452001|2452091|DEPARTMENT|58|81|Grey, likely windows may preserve probably; colleges used to slip |quarterly| +6238|AAAAAAAAOFIBAAAA|2452001|2452091|DEPARTMENT|58|82|Top, enormous assumptions used to bring different cases. Sharp studies express angrily western c|quarterly| +6239|AAAAAAAAPFIBAAAA|2452001|2452091|DEPARTMENT|58|83|Now wooden locations should not cultivate sharply even ge|quarterly| +6240|AAAAAAAAAGIBAAAA|2452001|2452091|DEPARTMENT|58|84|Inherent, sole purposes understand also together attractive things. Thus acceptable |quarterly| +6241|AAAAAAAABGIBAAAA|2452001|2452091|DEPARTMENT|58|85|Weekly demands forget substantially scottish, good children. Pa|quarterly| +6242|AAAAAAAACGIBAAAA|2452001|2452091|DEPARTMENT|58|86|Rates take there economic details. Ties may not regi|quarterly| +6243|AAAAAAAADGIBAAAA|2452001|2452091|DEPARTMENT|58|87|Awards think. Votes should observe poor elections. Complete states realis|quarterly| +6244|AAAAAAAAEGIBAAAA|2452001|2452091|DEPARTMENT|58|88|Low controversial governments ought to know now applicants; enough foreign crews get worth a|quarterly| +6245|AAAAAAAAFGIBAAAA|2452001|2452091|DEPARTMENT|58|89|Anxious, common eyes restrain then specially previous projects. Employers ought to take |quarterly| +6246|AAAAAAAAGGIBAAAA|2452001|2452091|DEPARTMENT|58|90|Military ends can provide normal, international times. Funny, independent gifts would write ag|quarterly| +6247|AAAAAAAAHGIBAAAA|2452001|2452091|DEPARTMENT|58|91|Soft, soviet goods attend actions; either warm women look at last likely, american calculations|quarterly| +6248|AAAAAAAAIGIBAAAA|2452001|2452091|DEPARTMENT|58|92|Well surprised studies could get immediately representatives. Gifts|quarterly| +6249|AAAAAAAAJGIBAAAA|2452001|2452091|DEPARTMENT|58|93|New, new provisions abandon better in a subsidies. English, local bonds deal then o|quarterly| +6250|AAAAAAAAKGIBAAAA|2452001|2452091|DEPARTMENT|58|94|Different claims sink new branches. Popular, sure children see |quarterly| +6251|AAAAAAAALGIBAAAA|2452001|2452091|DEPARTMENT|58|95|Theoretical values must not visit forward, asleep machines. For example critic|quarterly| +6252|AAAAAAAAMGIBAAAA|2452001|2452091|DEPARTMENT|58|96|Little constraints let quick japanese, large wives. Only contrary boys may not instruct jobs. |quarterly| +6253|AAAAAAAANGIBAAAA|2452001|2452091|DEPARTMENT|58|97|Running, necessary standards should pitch then decisive co|quarterly| +6254|AAAAAAAAOGIBAAAA|2452001|2452091|DEPARTMENT|58|98|Remarkable theories like members; currently early factors may live added n|quarterly| +6255|AAAAAAAAPGIBAAAA|2452001|2452091|DEPARTMENT|58|99|Regions label at a breasts; other, visual thoughts see. Then white activities pursue.|quarterly| +6256|AAAAAAAAAHIBAAAA|2452001|2452091|DEPARTMENT|58|100|Great sites will like as on a accounts. Families control chronic curtains. Important chara|quarterly| +6257|AAAAAAAABHIBAAAA|2452001|2452091|DEPARTMENT|58|101|Aside harsh needs persist also new, other stars. T|quarterly| +6258|AAAAAAAACHIBAAAA|2452001|2452091|DEPARTMENT|58|102|Years say somewhat difficulties. In order different responses ensure tonight still |quarterly| +6259|AAAAAAAADHIBAAAA|2452001|2452091|DEPARTMENT|58|103|Entire things make particular, late instances; for example excellent profits say further golden |quarterly| +6260|AAAAAAAAEHIBAAAA|2452001|2452091|DEPARTMENT|58|104|Other activities should take. Brown, single devices arise merely directly religious n|quarterly| +6261|AAAAAAAAFHIBAAAA|2452001|2452091|DEPARTMENT|58|105|Already surprising dimensions might develop almost years. Russian ages change mothers. Only|quarterly| +6262|AAAAAAAAGHIBAAAA|2452001|2452091|DEPARTMENT|58|106|Sad, marvellous things carry completely to a opinions. Sick, nuclear customers w|quarterly| +6263|AAAAAAAAHHIBAAAA|2452001|2452091|DEPARTMENT|58|107|Further clear times learn long ideas. Friends use soviet, innocent types. Average, other c|quarterly| +6264|AAAAAAAAIHIBAAAA|2452001|2452091|DEPARTMENT|58|108|Now other areas would find home suddenly local successes. Model, human things|quarterly| +6265|AAAAAAAAJHIBAAAA|2452092|2452182|DEPARTMENT|59|1|Red, financial months can try. Social colours shall decide as mer|quarterly| +6266|AAAAAAAAKHIBAAAA|2452092|2452182|DEPARTMENT|59|2|Words regret also surprising, appropriate students. There huge wome|quarterly| +6267|AAAAAAAALHIBAAAA|2452092|2452182|DEPARTMENT|59|3|Wrong, mental breasts must bring so substantial films; private th|quarterly| +6268|AAAAAAAAMHIBAAAA|2452092|2452182|DEPARTMENT|59|4|Further local wheels would establish directors. Now retail bits might take; golden year|quarterly| +6269|AAAAAAAANHIBAAAA|2452092|2452182|DEPARTMENT|59|5|Colours might turn for example. International, royal members jump. Noble citizens in|quarterly| +6270|AAAAAAAAOHIBAAAA|2452092|2452182|DEPARTMENT|59|6|Proposed, physical needs think together likely causes. So gay spots want |quarterly| +6271|AAAAAAAAPHIBAAAA|2452092|2452182|DEPARTMENT|59|7|Terms can think military results; dear, remaining operations may not ent|quarterly| +6272|AAAAAAAAAIIBAAAA|2452092|2452182|DEPARTMENT|59|8|Similarly interesting clothes take always again lig|quarterly| +6273|AAAAAAAABIIBAAAA|2452092|2452182|DEPARTMENT|59|9|Fairly new years look normally as necessary funds. Sp|quarterly| +6274|AAAAAAAACIIBAAAA|2452092|2452182|DEPARTMENT|59|10|Offences could acquire sure european flowers. Recently responsible events sho|quarterly| +6275|AAAAAAAADIIBAAAA|2452092|2452182|DEPARTMENT|59|11|Adverse books admit bright, related members. Sometimes black miles escape again easily bitter|quarterly| +6276|AAAAAAAAEIIBAAAA|2452092|2452182|DEPARTMENT|59|12|Administrative, criminal mountains see less different meetings; cool, inc rights |quarterly| +6277|AAAAAAAAFIIBAAAA|2452092|2452182|DEPARTMENT|59|13|Months must lose to an studies. Flexible, central shoulders should carry slight|quarterly| +6278|AAAAAAAAGIIBAAAA|2452092|2452182|DEPARTMENT|59|14|So big others ought to match papers; examples sound so great employees. Confe|quarterly| +6279|AAAAAAAAHIIBAAAA|2452092|2452182|DEPARTMENT|59|15|Here different trains enjoy. Hands can want wide months. Samples used to say even har|quarterly| +6280|AAAAAAAAIIIBAAAA|2452092|2452182|DEPARTMENT|59|16|Different governments celebrate. Average, national films will make; now|quarterly| +6281|AAAAAAAAJIIBAAAA|2452092|2452182|DEPARTMENT|59|17|Important, warm articles cannot suit as aware arts; subjects will publicise only|quarterly| +6282|AAAAAAAAKIIBAAAA|2452092|2452182|DEPARTMENT|59|18|Golden, powerful words may create carefully mental, ideal inhabitants.|quarterly| +6283|AAAAAAAALIIBAAAA|2452092|2452182|DEPARTMENT|59|19|Members see high, vulnerable contexts. Truly full chips believe little|quarterly| +6284|AAAAAAAAMIIBAAAA|2452092|2452182|DEPARTMENT|59|20|Other, available problems might get to the exports; visua|quarterly| +6285|AAAAAAAANIIBAAAA|2452092|2452182|DEPARTMENT|59|21|Plans may define simply coming terms. Blue duties require best rich ideas. Sorry, good stu|quarterly| +6286|AAAAAAAAOIIBAAAA|2452092|2452182|DEPARTMENT|59|22|Small, careful pubs used to clarify concrete, opposite reservations. Ago used hours see|quarterly| +6287|AAAAAAAAPIIBAAAA|2452092|2452182|DEPARTMENT|59|23|National individuals develop in a rewards. Political, intern|quarterly| +6288|AAAAAAAAAJIBAAAA|2452092|2452182|DEPARTMENT|59|24|Right effects fire systematically early, public practices. Tradition|quarterly| +6289|AAAAAAAABJIBAAAA|2452092|2452182|DEPARTMENT|59|25|Local, fine resources make abstract, mass children. Sorry, complex muscles de|quarterly| +6290|AAAAAAAACJIBAAAA|2452092|2452182|DEPARTMENT|59|26|Carefully young hours begin much. Possible, new figures run|quarterly| +6291|AAAAAAAADJIBAAAA|2452092|2452182|DEPARTMENT|59|27|Low, economic patterns will consider certainly about able books; young, recent wor|quarterly| +6292|AAAAAAAAEJIBAAAA|2452092|2452182|DEPARTMENT|59|28|Developments shall see well usually proper purposes. Techniques organis|quarterly| +6293|AAAAAAAAFJIBAAAA|2452092|2452182|DEPARTMENT|59|29|However complex games co-operate too now terrible photographs. Never exotic y|quarterly| +6294|AAAAAAAAGJIBAAAA|2452092|2452182|DEPARTMENT|59|30|Mice shall thank so. Wide women become immediately foreign for|quarterly| +6295|AAAAAAAAHJIBAAAA|2452092|2452182|DEPARTMENT|59|31|Other arms stay well right feelings. Mothers grant various, det|quarterly| +6296|AAAAAAAAIJIBAAAA|2452092|2452182|DEPARTMENT|59|32|Bad, separate services may not matter again famous, inte|quarterly| +6297|AAAAAAAAJJIBAAAA|2452092|2452182|DEPARTMENT|59|33|Financial users inquire downwards feet. Men must not like much|quarterly| +6298|AAAAAAAAKJIBAAAA|2452092|2452182|DEPARTMENT|59|34|Used, functional figures ought to complain faces. Parental, good shoulders pay nevertheles|quarterly| +6299|AAAAAAAALJIBAAAA|2452092|2452182|DEPARTMENT|59|35|As good exceptions should not give just other, smooth cats. New, certain changes would ex|quarterly| +6300|AAAAAAAAMJIBAAAA|2452092|2452182|DEPARTMENT|59|36|Young students will get french, convincing consequ|quarterly| +6301|AAAAAAAANJIBAAAA|2452092|2452182|DEPARTMENT|59|37|Yellow, dead names give economically expensive police. Overall, legal managers emulate princ|quarterly| +6302|AAAAAAAAOJIBAAAA|2452092|2452182|DEPARTMENT|59|38|Elsewhere big teeth might not see then similar, major powers. Too|quarterly| +6303|AAAAAAAAPJIBAAAA|2452092|2452182|DEPARTMENT|59|39|Users might recognise over social marks. Forms fight no longer free women. Open lines consi|quarterly| +6304|AAAAAAAAAKIBAAAA|2452092|2452182|DEPARTMENT|59|40|Simple, single debts pass gingerly under satisfactory features. More important troops expla|quarterly| +6305|AAAAAAAABKIBAAAA|2452092|2452182|DEPARTMENT|59|41|Backwards considerable results end red, industrial folk. Eyes open members. Already|quarterly| +6306|AAAAAAAACKIBAAAA|2452092|2452182|DEPARTMENT|59|42|Eyes play. Fine crews find misleading, new intentions. Trees p|quarterly| +6307|AAAAAAAADKIBAAAA|2452092|2452182|DEPARTMENT|59|43|Decisive benefits could shift perfectly periods. Houses ban on|quarterly| +6308|AAAAAAAAEKIBAAAA|2452092|2452182|DEPARTMENT|59|44|Techniques can say very procedures; scottish years could run with a n|quarterly| +6309|AAAAAAAAFKIBAAAA|2452092|2452182|DEPARTMENT|59|45|Affairs used to keep about ways. Green, equivalent persons provide very for a shares. Famous, ed|quarterly| +6310|AAAAAAAAGKIBAAAA|2452092|2452182|DEPARTMENT|59|46|Soon able years might want so real users. Events give top, other arts; main, holy officers shal|quarterly| +6311|AAAAAAAAHKIBAAAA|2452092|2452182|DEPARTMENT|59|47|Strange concentrations visit publishers. Libraries enlarge as red, double texts. International |quarterly| +6312|AAAAAAAAIKIBAAAA|2452092|2452182|DEPARTMENT|59|48|Very appropriate things ought to tuck so. Wives need dangerous, social months. Foreign co|quarterly| +6313|AAAAAAAAJKIBAAAA|2452092|2452182|DEPARTMENT|59|49|Original, major ships happen at all authorities; examp|quarterly| +6314|AAAAAAAAKKIBAAAA|2452092|2452182|DEPARTMENT|59|50|Healthy, liable groups could put often french, ill terms. Environmental, heavy p|quarterly| +6315|AAAAAAAALKIBAAAA|2452092|2452182|DEPARTMENT|59|51|Private activities speak else. Northern, separate em|quarterly| +6316|AAAAAAAAMKIBAAAA|2452092|2452182|DEPARTMENT|59|52|Too local schools may give examples. Forward, democratic systems know in a roads.|quarterly| +6317|AAAAAAAANKIBAAAA|2452092|2452182|DEPARTMENT|59|53|Seconds used to remain. Movements might agree bad workers. Then so|quarterly| +6318|AAAAAAAAOKIBAAAA|2452092|2452182|DEPARTMENT|59|54|Maps could want. Courts create now unexpected, good shows; |quarterly| +6319|AAAAAAAAPKIBAAAA|2452092|2452182|DEPARTMENT|59|55|Companies must move namely to a reactions. Able, other forces tell. Previous, doubl|quarterly| +6320|AAAAAAAAALIBAAAA|2452092|2452182|DEPARTMENT|59|56|As soft years ask recent, left trees. Aware competitors puni|quarterly| +6321|AAAAAAAABLIBAAAA|2452092|2452182|DEPARTMENT|59|57|Brothers fly structures. Necessary, white results co|quarterly| +6322|AAAAAAAACLIBAAAA|2452092|2452182|DEPARTMENT|59|58|Upper minutes offer for a forms. Still full hills get. Useful, responsible ro|quarterly| +6323|AAAAAAAADLIBAAAA|2452092|2452182|DEPARTMENT|59|59|Economic rights must gather initially as nice leaders. Collections may raise eventually alre|quarterly| +6324|AAAAAAAAELIBAAAA|2452092|2452182|DEPARTMENT|59|60|Just competitive experiences will not opt only. Sufficient|quarterly| +6325|AAAAAAAAFLIBAAAA|2452092|2452182|DEPARTMENT|59|61|Other winners wish social, correct thanks; painful police shall open always bc j|quarterly| +6326|AAAAAAAAGLIBAAAA|2452092|2452182|DEPARTMENT|59|62|Maybe friendly tears get classes. Labour, immediate|quarterly| +6327|AAAAAAAAHLIBAAAA|2452092|2452182|DEPARTMENT|59|63|Short, overseas workshops create normally actually natural exceptions. Things |quarterly| +6328|AAAAAAAAILIBAAAA|2452092|2452182|DEPARTMENT|59|64|Also american men shall eat finally worthwhile overseas systems. Ba|quarterly| +6329|AAAAAAAAJLIBAAAA|2452092|2452182|DEPARTMENT|59|65|Again natural times must keep most domestic effects; there significant wa|quarterly| +6330|AAAAAAAAKLIBAAAA|2452092|2452182|DEPARTMENT|59|66|Small friends could accept. Wings feed perhaps links. Years agree however tools. Pe|quarterly| +6331|AAAAAAAALLIBAAAA|2452092|2452182|DEPARTMENT|59|67|International banks will put here refugees. Only interested cards say deli|quarterly| +6332|AAAAAAAAMLIBAAAA|2452092|2452182|DEPARTMENT|59|68|Alone national potatoes should serve techniques. Hap|quarterly| +6333|AAAAAAAANLIBAAAA|2452092|2452182|DEPARTMENT|59|69|Good, old matters could mean very about a experiments. Busy, various characteristics f|quarterly| +6334|AAAAAAAAOLIBAAAA|2452092|2452182|DEPARTMENT|59|70|Circa soft funds check including the reports. No doubt modern girls drown a|quarterly| +6335|AAAAAAAAPLIBAAAA|2452092|2452182|DEPARTMENT|59|71|Long-term letters could provide actually into a programmes. Fields can see partic|quarterly| +6336|AAAAAAAAAMIBAAAA|2452092|2452182|DEPARTMENT|59|72|Profitable witnesses should not accept. Examples must pass at last courses. Cup|quarterly| +6337|AAAAAAAABMIBAAAA|2452092|2452182|DEPARTMENT|59|73|Performances must associate then. Low, ideal areas should not sell regulations. Genera|quarterly| +6338|AAAAAAAACMIBAAAA|2452092|2452182|DEPARTMENT|59|74|Rooms suffer then cases; back homes must not find w|quarterly| +6339|AAAAAAAADMIBAAAA|2452092|2452182|DEPARTMENT|59|75|Contemporary improvements used to know now secrets. Holidays w|quarterly| +6340|AAAAAAAAEMIBAAAA|2452092|2452182|DEPARTMENT|59|76|Large requirements ought to get cases; diplomatic ways change. Evident funds |quarterly| +6341|AAAAAAAAFMIBAAAA|2452092|2452182|DEPARTMENT|59|77|Earlier economic studies could adapt off the stars. Properties will give currently. Beautiful|quarterly| +6342|AAAAAAAAGMIBAAAA|2452092|2452182|DEPARTMENT|59|78|Able, blue children constitute tomorrow. Only psychological y|quarterly| +6343|AAAAAAAAHMIBAAAA|2452092|2452182|DEPARTMENT|59|79|Distinguished stations must not put later young matches; somewhere european years cann|quarterly| +6344|AAAAAAAAIMIBAAAA|2452092|2452182|DEPARTMENT|59|80|Already liberal years must not make. There special doctors convey from the cells. Enou|quarterly| +6345|AAAAAAAAJMIBAAAA|2452092|2452182|DEPARTMENT|59|81|Convincing ways could not bring here full, political miles. Front, different features used|quarterly| +6346|AAAAAAAAKMIBAAAA|2452092|2452182|DEPARTMENT|59|82|Here extra plans would choose far rocks. Categories can distinguish under wives. Tire|quarterly| +6347|AAAAAAAALMIBAAAA|2452092|2452182|DEPARTMENT|59|83|Families would undertake now for a cases. Economic applications install only single fam|quarterly| +6348|AAAAAAAAMMIBAAAA|2452092|2452182|DEPARTMENT|59|84|Unreasonably related problems allow open, whole hands;|quarterly| +6349|AAAAAAAANMIBAAAA|2452092|2452182|DEPARTMENT|59|85|Long, legitimate operators go problems. Times must see gold, weak w|quarterly| +6350|AAAAAAAAOMIBAAAA|2452092|2452182|DEPARTMENT|59|86|Senior, primary situations could not defend however. |quarterly| +6351|AAAAAAAAPMIBAAAA|2452092|2452182|DEPARTMENT|59|87|Industrial laws report over away open items. Burni|quarterly| +6352|AAAAAAAAANIBAAAA|2452092|2452182|DEPARTMENT|59|88|Too considerable rates see only large woods. All right dul|quarterly| +6353|AAAAAAAABNIBAAAA|2452092|2452182|DEPARTMENT|59|89|Almost official members take views; financial ministers can improve in particular. Then i|quarterly| +6354|AAAAAAAACNIBAAAA|2452092|2452182|DEPARTMENT|59|90|Groups see also. Questions may not form observations; joint opportunities writ|quarterly| +6355|AAAAAAAADNIBAAAA|2452092|2452182|DEPARTMENT|59|91|Forthcoming men augment successful, other arrangements. Basic patterns c|quarterly| +6356|AAAAAAAAENIBAAAA|2452092|2452182|DEPARTMENT|59|92|Mysterious years wear especially. Activities will create fortunate, catholic pa|quarterly| +6357|AAAAAAAAFNIBAAAA|2452092|2452182|DEPARTMENT|59|93|European choices used to support never teenage sales. Clear, old |quarterly| +6358|AAAAAAAAGNIBAAAA|2452092|2452182|DEPARTMENT|59|94|Never large metals see far actually normal systems. Known subjects shou|quarterly| +6359|AAAAAAAAHNIBAAAA|2452092|2452182|DEPARTMENT|59|95|Women remain well different, basic years. Kilometres concentrate investigations. Serious, grey poin|quarterly| +6360|AAAAAAAAINIBAAAA|2452092|2452182|DEPARTMENT|59|96|Windows must realise references. There exciting assessments inform just different men; inh|quarterly| +6361|AAAAAAAAJNIBAAAA|2452092|2452182|DEPARTMENT|59|97|Local, just initiatives could not move. Authorities used to th|quarterly| +6362|AAAAAAAAKNIBAAAA|2452092|||||Still whole others ought to help either late large kinds. Largely modern features comb|| +6363|AAAAAAAALNIBAAAA|2452092|2452182|DEPARTMENT|59|99|Very desperate books vary now only years. New sites s|quarterly| +6364|AAAAAAAAMNIBAAAA|2452092|2452182|DEPARTMENT|59|100|Strategies used to allow future eyes. Still subsequent prod|quarterly| +6365|AAAAAAAANNIBAAAA|2452092|2452182|DEPARTMENT|59|101|Inner, critical discussions might not keep soon drugs. Unique nurses may establish actual, clear op|quarterly| +6366|AAAAAAAAONIBAAAA|2452092|2452182|DEPARTMENT|59|102|Eventually full stages convey limited children. Beliefs go bright things. Lucky elements travel p|quarterly| +6367|AAAAAAAAPNIBAAAA|2452092|2452182|DEPARTMENT|59|103|Typical, acute flowers ought to seek on board deaf women. By|quarterly| +6368|AAAAAAAAAOIBAAAA|2452092|2452182|DEPARTMENT|59|104|Best immediate departments used to establish now late, tall direct|quarterly| +6369|AAAAAAAABOIBAAAA|2452092|2452182|DEPARTMENT|59|105|Tiny, dear windows want in addition human clothes. Forms enable recently seats. Satisfied w|quarterly| +6370|AAAAAAAACOIBAAAA|2452092|2452182|DEPARTMENT|59|106|Clear, specific researchers shall read. Accessible, gener|quarterly| +6371|AAAAAAAADOIBAAAA|2452092|2452182|DEPARTMENT|59|107|Modest, common files used to argue over isolated meeting|quarterly| +6372|AAAAAAAAEOIBAAAA|2452092|2452182|DEPARTMENT|59|108|Specific, large clients should think despite no clubs. Poor com|quarterly| +6373|AAAAAAAAFOIBAAAA|2452183|2452273|DEPARTMENT|60|1|Perhaps inherent affairs thank prices; most important supp|quarterly| +6374|AAAAAAAAGOIBAAAA|2452183|2452273|DEPARTMENT|60|2|Young phrases would expect however women; programmes will want. State|quarterly| +6375|AAAAAAAAHOIBAAAA|2452183|2452273|DEPARTMENT|60|3|Physical, important leaders may not accept free posts. Stupid, other collec|quarterly| +6376|AAAAAAAAIOIBAAAA|2452183|2452273|DEPARTMENT|60|4|Original claims should manage before utterly complex services; quite alternative feet get then co|quarterly| +6377|AAAAAAAAJOIBAAAA|2452183|2452273|DEPARTMENT|60|5|Particularly other processes may not suggest about good demands. At first great |quarterly| +6378|AAAAAAAAKOIBAAAA|2452183|2452273|DEPARTMENT|60|6|Ways will express thus spanish practices. Rapid sections will not want very. |quarterly| +6379|AAAAAAAALOIBAAAA|2452183|2452273|DEPARTMENT|60|7|Occasions become below fine strangers. New, recent shares could assure then scientifi|quarterly| +6380|AAAAAAAAMOIBAAAA|2452183|2452273|DEPARTMENT|60|8|Far from small eyes bring thus during the horses. Just nuclear gods will destroy private, br|quarterly| +6381|AAAAAAAANOIBAAAA|2452183|2452273|DEPARTMENT|60|9|Necessary, universal men shall not speak correct, particular circumstances. Meanwhile long conse|quarterly| +6382|AAAAAAAAOOIBAAAA|2452183|2452273|DEPARTMENT|60|10|Never small doors tie. Please likely members should admit ho|quarterly| +6383|AAAAAAAAPOIBAAAA|2452183|2452273|DEPARTMENT|60|11|Rivers cannot consider anyway then poor funds. Stories remain sometimes most sad times; ideologic|quarterly| +6384|AAAAAAAAAPIBAAAA|2452183|2452273|DEPARTMENT|60|12|Years should protect results. Effective chairs prevent specially animals|quarterly| +6385|AAAAAAAABPIBAAAA|2452183|2452273|DEPARTMENT|60|13|Great women learn exciting principles; long goals live; current|quarterly| +6386|AAAAAAAACPIBAAAA|2452183|2452273|DEPARTMENT|60|14|Years would continue today in a years. Mental materials trip. Hopes appreciate p|quarterly| +6387|AAAAAAAADPIBAAAA|2452183|2452273|DEPARTMENT|60|15|Professional eyes sell now certain cars. White areas convert about hard connecti|quarterly| +6388|AAAAAAAAEPIBAAAA|2452183|2452273|DEPARTMENT|60|16|Just foreign areas may behave above islamic rates. Then b|quarterly| +6389|AAAAAAAAFPIBAAAA|2452183|2452273|DEPARTMENT|60|17|At all reasonable authors see for the eyes. Also english states may not guaran|quarterly| +6390|AAAAAAAAGPIBAAAA|2452183|2452273|DEPARTMENT|60|18|Full, considerable events take by a messages; favorable, fiscal examples pay thick progressive |quarterly| +6391|AAAAAAAAHPIBAAAA|2452183|2452273|DEPARTMENT|60|19|Amazing, political stories stand therefore long futures. Also inter|quarterly| +6392|AAAAAAAAIPIBAAAA|2452183|2452273|DEPARTMENT|60|20|Molecules work huge animals. Universal others disturb possible windows; tropical, li|quarterly| +6393|AAAAAAAAJPIBAAAA|2452183|2452273|DEPARTMENT|60|21|Local studies say casually very letters. German programs used to mean like a mammals. Ind|quarterly| +6394|AAAAAAAAKPIBAAAA|2452183|2452273|DEPARTMENT|60|22|Ideas force so times. Positions spell then well alternative trees. Further crimin|quarterly| +6395|AAAAAAAALPIBAAAA|2452183|2452273|DEPARTMENT|60|23|Sales should not take. Further whole bits can control so. Considerably statutory bu|quarterly| +6396|AAAAAAAAMPIBAAAA|2452183|2452273|DEPARTMENT|60|24|Parts can like very black techniques. Other, new figures used to produce compani|quarterly| +6397|AAAAAAAANPIBAAAA|2452183|2452273|DEPARTMENT|60|25|Already disabled changes shall not indicate employees. Here clear terms record ever sure depend|quarterly| +6398|AAAAAAAAOPIBAAAA|2452183|2452273|DEPARTMENT|60|26|For example special disputes must take mediterranean levels. Areas shall|quarterly| +6399|AAAAAAAAPPIBAAAA|2452183|2452273|DEPARTMENT|60|27|Most poor trees advise participants. Collections mean electronic pages. Variable, new hands giv|quarterly| +6400|AAAAAAAAAAJBAAAA|2452183|2452273|DEPARTMENT|60|28|Tall, pale months should expand important publications. Jewish, common others build however h|quarterly| +6401|AAAAAAAABAJBAAAA|2452183|2452273|DEPARTMENT|60|29|Papers would need yet now impossible methods. Exciting studies may provide less. Extremely faint p|quarterly| +6402|AAAAAAAACAJBAAAA|2452183|2452273|DEPARTMENT|60|30|Expressions relinquish in a years. Types give perhaps mor|quarterly| +6403|AAAAAAAADAJBAAAA|2452183|2452273|DEPARTMENT|60|31|Nuclear, intimate hours shall order otherwise forward quantities. |quarterly| +6404|AAAAAAAAEAJBAAAA|2452183|2452273|DEPARTMENT|60|32|Elderly drawings ought to call new fragments. Terms afford as clubs. Oral|quarterly| +6405|AAAAAAAAFAJBAAAA|2452183|2452273|DEPARTMENT|60|33|So tiny men panic at least much other reforms. Women cannot bring here new characters. Maximum, p|quarterly| +6406|AAAAAAAAGAJBAAAA|2452183|2452273|DEPARTMENT|60|34|Secrets should issue days; spiritual, real houses go however total, oth|quarterly| +6407|AAAAAAAAHAJBAAAA|2452183|2452273|DEPARTMENT|60|35|Monthly beds make past. Enough true advantages recommend assis|quarterly| +6408|AAAAAAAAIAJBAAAA|2452183|2452273|DEPARTMENT|60|36|Parameters cannot abandon particularly public days; british scholars understand straigh|quarterly| +6409|AAAAAAAAJAJBAAAA|2452183|2452273|DEPARTMENT|60|37|Educational results keep finally therefore lexical schools. Recent effects may not drop welsh, livi|quarterly| +6410|AAAAAAAAKAJBAAAA|2452183|2452273|DEPARTMENT|60|38|Possible jobs give free, exciting demands; main, public courses get probably top problems. Free|quarterly| +6411|AAAAAAAALAJBAAAA|2452183|2452273|DEPARTMENT|60|39|Actually armed bodies care drinks. New occupations shall n|quarterly| +6412|AAAAAAAAMAJBAAAA|2452183|2452273|DEPARTMENT|60|40|Mental, simple shots make yet international versions. Males drive just simple, da|quarterly| +6413|AAAAAAAANAJBAAAA|2452183|2452273|DEPARTMENT|60|41|Boys shall choose carefully direct feet. Improved fans used to |quarterly| +6414|AAAAAAAAOAJBAAAA|2452183|2452273|DEPARTMENT|60|42|Here alone effects look. Fair, german teams afford |quarterly| +6415|AAAAAAAAPAJBAAAA|2452183|2452273|DEPARTMENT|60|43|Crucial activities may work even. Small strings take ever good, real feet. Women come ag|quarterly| +6416|AAAAAAAAABJBAAAA|2452183|2452273|DEPARTMENT|60|44|Parents shall say in a standards. Children keep. Massive, |quarterly| +6417|AAAAAAAABBJBAAAA|2452183|2452273|DEPARTMENT|60|45|Measures show emphatically to the students. For good mechanical shops will not te|quarterly| +6418|AAAAAAAACBJBAAAA|2452183|2452273|DEPARTMENT|60|46|Measures enable complex years. Tightly simple positions appeal welsh, white opinio|quarterly| +6419|AAAAAAAADBJBAAAA|2452183|2452273|DEPARTMENT|60|47|Weak incentives should not say soviet, present ties. Pp. get intermittently by the leade|quarterly| +6420|AAAAAAAAEBJBAAAA|2452183|2452273|DEPARTMENT|60|48|Only kids could not help big regions; prime, local ages may n|quarterly| +6421|AAAAAAAAFBJBAAAA|2452183|2452273|DEPARTMENT|60|49|Available tickets can wear once things. Directors see specifically stocks. Duties hold hi|quarterly| +6422|AAAAAAAAGBJBAAAA|2452183|2452273|DEPARTMENT|60|50|Seats dislodge basic contributions. Technical players eat normally with a jobs. Giant, h|quarterly| +6423|AAAAAAAAHBJBAAAA|2452183|2452273|DEPARTMENT|60|51|Economic, large areas stand however relationships. More than close hands should look develop|quarterly| +6424|AAAAAAAAIBJBAAAA|2452183|2452273|DEPARTMENT|60|52|Numbers could buy. Happy stairs become open, permanent men. Authorities choose obvious, public car|quarterly| +6425|AAAAAAAAJBJBAAAA|2452183|2452273|DEPARTMENT|60|53|A bit black rocks ought to happen level, wise tools. Shows shall know modern pupils. Actually new|quarterly| +6426|AAAAAAAAKBJBAAAA|2452183|2452273|DEPARTMENT|60|54|Moral, christian workers could not restore also old |quarterly| +6427|AAAAAAAALBJBAAAA|2452183|2452273|DEPARTMENT|60|55|Tales cannot seem only laws. Attractive, partial muscles must rate rarely separate interests;|quarterly| +6428|AAAAAAAAMBJBAAAA|2452183|2452273|DEPARTMENT|60|56|Errors recover to a things. European, whole fields co-ordinate sadly prime bo|quarterly| +6429|AAAAAAAANBJBAAAA|2452183|2452273|DEPARTMENT|60|57|Home public communists may ensure severe, major points; as close students o|quarterly| +6430|AAAAAAAAOBJBAAAA|2452183|2452273|DEPARTMENT|60|58|Social qualifications give better with the thanks. Federal pr|quarterly| +6431|AAAAAAAAPBJBAAAA|2452183|2452273|DEPARTMENT|60|59|Other articles shall not waste then in a readers; other, sweet groups might hold precise|quarterly| +6432|AAAAAAAAACJBAAAA|2452183|2452273|DEPARTMENT|60|60|Useless, great women would occur close societies. Services may not place national |quarterly| +6433|AAAAAAAABCJBAAAA|2452183|2452273|DEPARTMENT|60|61|Immense bacteria shall not hesitate so soldiers. Remaining, rich firms will see serious, mise|quarterly| +6434|AAAAAAAACCJBAAAA|2452183|2452273|DEPARTMENT|60|62|Guards could know far. Still contrary ages collect political, comfortable visi|quarterly| +6435|AAAAAAAADCJBAAAA|2452183|2452273|DEPARTMENT|60|63|Jewish, other miles discover as ultimate, easy jobs. Times telephone howe|quarterly| +6436|AAAAAAAAECJBAAAA|2452183|2452273|DEPARTMENT|60|64|Executives take there central, royal children. Tomorrow thin plans write nevertheless t|quarterly| +6437|AAAAAAAAFCJBAAAA|2452183|2452273|DEPARTMENT|60|65|Big, real abilities should not draw local, exotic fo|quarterly| +6438|AAAAAAAAGCJBAAAA|2452183|2452273|DEPARTMENT|60|66|Happy, compatible sanctions may lie for example public notes. Years may disrupt japanese fe|quarterly| +6439|AAAAAAAAHCJBAAAA|2452183|2452273|DEPARTMENT|60|67|Black, old dogs ought to work at all usual implications; fel|quarterly| +6440|AAAAAAAAICJBAAAA|2452183|2452273|DEPARTMENT|60|68|Short ordinary changes should see in order connections. Demonstr|quarterly| +6441|AAAAAAAAJCJBAAAA|2452183|2452273|DEPARTMENT|60|69|Genuine lights may not aim now average skills. Chief rows rate |quarterly| +6442|AAAAAAAAKCJBAAAA|2452183|2452273|DEPARTMENT|60|70|Certainly impossible children might understand open|quarterly| +6443|AAAAAAAALCJBAAAA|2452183|2452273|DEPARTMENT|60|71|Complete, little talks sleep therefore. No longer |quarterly| +6444|AAAAAAAAMCJBAAAA|2452183|2452273|DEPARTMENT|60|72|Wages might leave into a charges. Normal computers compare therefore. Securitie|quarterly| +6445|AAAAAAAANCJBAAAA|2452183|2452273|DEPARTMENT|60|73|Years enable comments. Still pink results would act besides for a characteristics; as permanent sci|quarterly| +6446|AAAAAAAAOCJBAAAA|2452183|2452273|DEPARTMENT|60|74|Far victorian roads start communist times; trees bear |quarterly| +6447|AAAAAAAAPCJBAAAA|2452183|2452273|DEPARTMENT|60|75|Controversial reductions ought to throw lips. Moving reasons use|quarterly| +6448|AAAAAAAAADJBAAAA|2452183|2452273|DEPARTMENT|60|76|As ready industries give then black, social firms. Major, certai|quarterly| +6449|AAAAAAAABDJBAAAA|2452183|2452273|DEPARTMENT|60|77|Creative colours ought to call social, fun discussions. Old, impressive years used to |quarterly| +6450|AAAAAAAACDJBAAAA|2452183|2452273|DEPARTMENT|60|78|By now crucial cities could not see days. Carefully free layers can deli|quarterly| +6451|AAAAAAAADDJBAAAA|2452183|2452273|DEPARTMENT|60|79|Just hot questions work also still apparent roads. Years may find all racial yards|quarterly| +6452|AAAAAAAAEDJBAAAA|2452183|2452273|DEPARTMENT|60|80|Shoulders cannot recapture. Electronic, major times back aware stones. Il|quarterly| +6453|AAAAAAAAFDJBAAAA|2452183|2452273|DEPARTMENT|60|81|Good hours can cut. So wrong years notice by a trusts; measures go as in a leade|quarterly| +6454|AAAAAAAAGDJBAAAA|2452183|2452273|DEPARTMENT|60|82|Long, local newspapers used to describe; good, real conflicts appreciate neith|quarterly| +6455|AAAAAAAAHDJBAAAA|2452183|2452273|DEPARTMENT|60|83|Theories could give powers. Old, obvious women must n|quarterly| +6456|AAAAAAAAIDJBAAAA|2452183|2452273|DEPARTMENT|60|84|Original cases may continue even skilled, other clients; journali|quarterly| +6457|AAAAAAAAJDJBAAAA|2452183|2452273|DEPARTMENT|60|85|Small cats must not buy then specific horses. Individual, e|quarterly| +6458|AAAAAAAAKDJBAAAA|2452183|2452273|DEPARTMENT|60|86|Strong, subtle skills reduce only in a members. New, di|quarterly| +6459|AAAAAAAALDJBAAAA|2452183|2452273|DEPARTMENT|60|87|Aside major phenomena ought to drop a bit wet significant floors. Individual, gastric|quarterly| +6460|AAAAAAAAMDJBAAAA|2452183|2452273|DEPARTMENT|60|88|Unexpected officers should not find into the casualties. Exc|quarterly| +6461|AAAAAAAANDJBAAAA|2452183|2452273|DEPARTMENT|60|89|Annual years might reveal also never high lips. Areas keep now seats. Groups cannot tru|quarterly| +6462|AAAAAAAAODJBAAAA|2452183|2452273|DEPARTMENT|60|90|Important weeks could not make very impressive weeks. Royal, royal ages shall not pr|quarterly| +6463|AAAAAAAAPDJBAAAA|2452183|2452273|DEPARTMENT|60|91|Margins might feel broken, responsible trains; trees will argue creative, elderly chemicals|quarterly| +6464|AAAAAAAAAEJBAAAA|2452183|2452273|DEPARTMENT|60|92|Much tremendous skills would carry tired things. Dark players stand obviously maj|quarterly| +6465|AAAAAAAABEJBAAAA|2452183|2452273|DEPARTMENT|60|93|Statistically natural months die total humans. Extraordinari|quarterly| +6466|AAAAAAAACEJBAAAA|2452183|2452273|DEPARTMENT|60|94|Please public figures go early now good cars. Exact methods think little |quarterly| +6467|AAAAAAAADEJBAAAA|2452183|2452273|DEPARTMENT|60|95|Rather effective candidates might hear general depths; distinct, wron|quarterly| +6468|AAAAAAAAEEJBAAAA|||DEPARTMENT||96|Good cells may consider visually earlier tall roots. Days must sit railways. Histo|quarterly| +6469|AAAAAAAAFEJBAAAA|2452183|2452273|DEPARTMENT|60|97|Goals offer both entirely functional hundreds. Again modern hands ought to get. Already complex cou|quarterly| +6470|AAAAAAAAGEJBAAAA|2452183|2452273|DEPARTMENT|60|98|Electric, military others might refuse to a blacks. Hours appoint at a women.|quarterly| +6471|AAAAAAAAHEJBAAAA|2452183|2452273|DEPARTMENT|60|99|Friendly forms restrain against a farmers. Black, |quarterly| +6472|AAAAAAAAIEJBAAAA|2452183|2452273|DEPARTMENT|60|100|Obvious males may remember now slightly national meanings. Highly confident |quarterly| +6473|AAAAAAAAJEJBAAAA|2452183|2452273|DEPARTMENT|60|101|Low, good pages create in the words. More serious issu|quarterly| +6474|AAAAAAAAKEJBAAAA|2452183|2452273|DEPARTMENT|60|102|Social, british lights may reduce so new origins; in short unique me|quarterly| +6475|AAAAAAAALEJBAAAA|2452183|2452273|DEPARTMENT|60|103|Of course other countries meet as pleasant things. Flats figure huge, different patients. Open term|quarterly| +6476|AAAAAAAAMEJBAAAA|2452183|2452273|DEPARTMENT|60|104|Bright, previous supporters cannot hope also in a men. Forward sophisti|quarterly| +6477|AAAAAAAANEJBAAAA|2452183|2452273|DEPARTMENT|60|105|So working-class thanks ought to give; now single sto|quarterly| +6478|AAAAAAAAOEJBAAAA|2452183|2452273|DEPARTMENT|60|106|Formidable patterns used to fire. Times provide now various pubs. Posts would open also local|quarterly| +6479|AAAAAAAAPEJBAAAA|2452183|2452273|DEPARTMENT|60|107|Black, interested patients examine distinct, democratic s|quarterly| +6480|AAAAAAAAAFJBAAAA|2452183|2452273|DEPARTMENT|60|108|Partly back photographs gather attractive services|quarterly| +6481|AAAAAAAABFJBAAAA|2451910|2451939|DEPARTMENT|61|1|Others boil differently significant, recent features. Patterns lack. Holid|monthly| +6482|AAAAAAAACFJBAAAA|2451910|2451939|DEPARTMENT|61|2|Lightly implicit doors will scare. Drawings know eyes. Even rural te|monthly| +6483|AAAAAAAADFJBAAAA|2451910|2451939|DEPARTMENT|61|3|Aware, close voters ought to say sexual, communist classes. Even high jobs may need inte|monthly| +6484|AAAAAAAAEFJBAAAA|||DEPARTMENT||||| +6485|AAAAAAAAFFJBAAAA|2451910|2451939|DEPARTMENT|61|5|Recent, current magistrates make as tables. Alone, criminal students|monthly| +6486|AAAAAAAAGFJBAAAA|2451910|2451939|DEPARTMENT|61|6|Relevant, ethical councils explore just above unable words. Then sorry changes remain for |monthly| +6487|AAAAAAAAHFJBAAAA|2451910|||61||Conscious tools perform complex, necessary phrases. Available, valuable services will help now a|| +6488|AAAAAAAAIFJBAAAA|2451910|2451939|DEPARTMENT|61|8|Over molecular terms handle about a sons. Large dimensions shall oppose most; different children|monthly| +6489|AAAAAAAAJFJBAAAA|2451910|2451939|DEPARTMENT|61|9|Members touch from a boundaries. Enough australian animals take much elections. Players |monthly| +6490|AAAAAAAAKFJBAAAA|2451910|2451939|DEPARTMENT|61|10|Pupils must introduce most just long parents. Leaders could|monthly| +6491|AAAAAAAALFJBAAAA|2451910|2451939|DEPARTMENT|61|11|Entirely clean stars favour never to a funds. Yet great sisters should reduce then waiting ni|monthly| +6492|AAAAAAAAMFJBAAAA|2451910|2451939|DEPARTMENT|61|12|Therefore involved costs should remain well elsewhere origin|monthly| +6493|AAAAAAAANFJBAAAA|2451910|2451939|DEPARTMENT|61|13|Certain charts will not ensure good, marvellous hands. Meanings used to revise|monthly| +6494|AAAAAAAAOFJBAAAA|2451910|2451939|DEPARTMENT|61|14|Less than able practitioners should look so rich rights. Ne|monthly| +6495|AAAAAAAAPFJBAAAA|2451910|2451939|DEPARTMENT|61|15|Rights call interesting, internal communications. Cle|monthly| +6496|AAAAAAAAAGJBAAAA|2451910|2451939|DEPARTMENT|61|16|Social, strong bombs ought to like kinds. Sharply strong parts might depend later always honest |monthly| +6497|AAAAAAAABGJBAAAA|2451910|2451939|DEPARTMENT|61|17|Below local users can provide soldiers. Great men answer best nevertheless good ins|monthly| +6498|AAAAAAAACGJBAAAA|2451910|2451939|DEPARTMENT|61|18|Financial countries help only objectives. General, annual requirements render finge|monthly| +6499|AAAAAAAADGJBAAAA|2451910|2451939|DEPARTMENT|61|19|Public details find items. Determined deposits den|monthly| +6500|AAAAAAAAEGJBAAAA|2451910|2451939|DEPARTMENT|61|20|High good features must not play suitable functions. Steep, individual variab|monthly| +6501|AAAAAAAAFGJBAAAA|2451910|2451939|DEPARTMENT|61|21|Indeed tory measures may tell; changing, new indians might buy again huge,|monthly| +6502|AAAAAAAAGGJBAAAA|2451910|2451939|DEPARTMENT|61|22|New charges edit hurriedly rarely new principles. Matters can investigate as symbols. Sepa|monthly| +6503|AAAAAAAAHGJBAAAA|2451910|2451939|DEPARTMENT|61|23|Dangerous, soviet miles provide in a projects. Completely|monthly| +6504|AAAAAAAAIGJBAAAA|2451910|2451939|DEPARTMENT|61|24|New, personal things shall go about. Academic, acceptable moments put more in an hundreds; days|monthly| +6505|AAAAAAAAJGJBAAAA|2451910|2451939|DEPARTMENT|61|25|Small difficulties like feet. Cases introduce ancient, real requirements. Runnin|monthly| +6506|AAAAAAAAKGJBAAAA|2451910|2451939|DEPARTMENT|61|26|Foreign, good markets make women. Merely true cards include senten|monthly| +6507|AAAAAAAALGJBAAAA|2451910|2451939|DEPARTMENT|61|27|Perfect designers should continue at least for a members. Too lucky unions want writt|monthly| +6508|AAAAAAAAMGJBAAAA|2451910|2451939|DEPARTMENT|61|28|Types get types. Hardly comprehensive symbols swap new conserva|monthly| +6509|AAAAAAAANGJBAAAA|2451910|2451939|DEPARTMENT|61|29|Small places could slip probably good, new effects. Educational children worry ind|monthly| +6510|AAAAAAAAOGJBAAAA|2451910|2451939|DEPARTMENT|61|30|Almost electronic lights will not behave; exotic contributions can want so professional colleague|monthly| +6511|AAAAAAAAPGJBAAAA|2451910|2451939|DEPARTMENT|61|31|Values used to know besides anxious, blue representations; police generate o|monthly| +6512|AAAAAAAAAHJBAAAA|2451910|2451939|DEPARTMENT|61|32|British, extra songs might not mean very western friends. Main, new rates cou|monthly| +6513|AAAAAAAABHJBAAAA|2451910|2451939|DEPARTMENT|61|33|Financial lists would return also on a men. Groups know more ide|monthly| +6514|AAAAAAAACHJBAAAA|2451910|2451939|DEPARTMENT|61|34|Supporters exercise boats. Good, different nations |monthly| +6515|AAAAAAAADHJBAAAA|2451910|2451939|DEPARTMENT|61|35|Spanish initiatives will procure apart for the days. Light, great earnings should not st|monthly| +6516|AAAAAAAAEHJBAAAA|2451910|2451939|DEPARTMENT|61|36|Precious stones miss female ministers. Right sites |monthly| +6517|AAAAAAAAFHJBAAAA|2451910|2451939|DEPARTMENT|61|37|Yet indian companies must try still to a aspects. Political spots may mak|monthly| +6518|AAAAAAAAGHJBAAAA|2451910|2451939|DEPARTMENT|61|38|Important, major eggs must not worsen cool present righ|monthly| +6519|AAAAAAAAHHJBAAAA|2451910|2451939|DEPARTMENT|61|39|Clearly mature relations would pass conservatives. New prices give pleased, considerable c|monthly| +6520|AAAAAAAAIHJBAAAA|2451910|2451939|DEPARTMENT|61|40|External issues stop probably for a eggs. Conflicts bear now outwards high cases. Considerab|monthly| +6521|AAAAAAAAJHJBAAAA|2451910|2451939|DEPARTMENT|61|41|Strong, aware leads used to consider to a reasons; soci|monthly| +6522|AAAAAAAAKHJBAAAA|2451910|2451939|DEPARTMENT|61|42|Associated, old parties want full programmes. Difficult, rat|monthly| +6523|AAAAAAAALHJBAAAA|2451910|2451939|DEPARTMENT|61|43|Firm, difficult children see less traditional supporters.|monthly| +6524|AAAAAAAAMHJBAAAA|2451910|2451939|DEPARTMENT|61|44|Most cool things would comment then typical days. Cu|monthly| +6525|AAAAAAAANHJBAAAA|2451910|2451939|DEPARTMENT|61|45|Leaves should become elected men. Retail, gold positions could rebuild together|monthly| +6526|AAAAAAAAOHJBAAAA|2451910|2451939|DEPARTMENT|61|46|Perhaps sweet branches will not buy women. Bad, important officials ought to advance in particula|monthly| +6527|AAAAAAAAPHJBAAAA|2451910|2451939|DEPARTMENT|61|47|Continuously obvious events refer effectively. Like, economic eggs cou|monthly| +6528|AAAAAAAAAIJBAAAA|2451910|2451939|DEPARTMENT|61|48|Regular telecommunications ask particularly with the americans. Bonds|monthly| +6529|AAAAAAAABIJBAAAA|2451910|2451939|DEPARTMENT|61|49|Days must not support mainly high types. Really positive others shall b|monthly| +6530|AAAAAAAACIJBAAAA|2451910|2451939|DEPARTMENT|61|50|Journalists regard even. Various equations live so|monthly| +6531|AAAAAAAADIJBAAAA|2451910|2451939|DEPARTMENT|61|51|Losses should shoulder in a animals. Seeds see at all most open towns. Reserves may|monthly| +6532|AAAAAAAAEIJBAAAA|2451910|2451939|DEPARTMENT|61|52|Single, significant officials would give. New pieces stand sure remainin|monthly| +6533|AAAAAAAAFIJBAAAA|2451910|2451939|DEPARTMENT|61|53|Also technical objections break as well sensitive shoulders. Formal, |monthly| +6534|AAAAAAAAGIJBAAAA|2451910|2451939|DEPARTMENT|61|54|Right streets become well labour factories. Films boil in a courses. Little rough others |monthly| +6535|AAAAAAAAHIJBAAAA|2451910|2451939|DEPARTMENT|61|55|Actually burning issues manage better yet correct decisions; comprehensive areas overthrow merely|monthly| +6536|AAAAAAAAIIJBAAAA|2451910|2451939|DEPARTMENT|61|||monthly| +6537|AAAAAAAAJIJBAAAA|2451910|2451939|DEPARTMENT|61|57|Afraid games fancy here to a recommendations. Options dispose as a teams. Only certain feet s|monthly| +6538|AAAAAAAAKIJBAAAA|2451910|2451939|DEPARTMENT|61|58|Yesterday similar substances allow quite. Electrical, royal boots sta|monthly| +6539|AAAAAAAALIJBAAAA|2451910|2451939|DEPARTMENT|61|59|Soviet, male individuals provide with a police. Subsequent, legal rules |monthly| +6540|AAAAAAAAMIJBAAAA|2451910|2451939|DEPARTMENT|61|60|Also geographical allowances enhance moreover for a cases. Me|monthly| +6541|AAAAAAAANIJBAAAA|2451910|2451939|DEPARTMENT|61|61|Active, russian blues see in general to a policies|monthly| +6542|AAAAAAAAOIJBAAAA|2451910|2451939|DEPARTMENT|61|62|Economic opponents report hard children. Hours used to mov|monthly| +6543|AAAAAAAAPIJBAAAA|2451910|2451939|DEPARTMENT|61|63|More essential children spare never then new markets; again top reports ought to know to a|monthly| +6544|AAAAAAAAAJJBAAAA|2451910|2451939|DEPARTMENT|61|64|Other poems stay very; very potential recordings could not l|monthly| +6545|AAAAAAAABJJBAAAA|2451910|2451939|DEPARTMENT|61|65|Activities facilitate faithfully skills. Ladies must be|monthly| +6546|AAAAAAAACJJBAAAA|2451910|2451939|DEPARTMENT|61|66|Just, local types will rise partially with a applications. Commercial, small rights shall ask ric|monthly| +6547|AAAAAAAADJJBAAAA|2451910|2451939|DEPARTMENT|61|67|Restaurants could think values. Able genes used to take between |monthly| +6548|AAAAAAAAEJJBAAAA|2451910|2451939|DEPARTMENT|61|68|Local visitors can compare so in a cells; peculiar stories relie|monthly| +6549|AAAAAAAAFJJBAAAA|2451910|2451939|DEPARTMENT|61|69|Happy, emotional institutions connect highly continuous personnel. |monthly| +6550|AAAAAAAAGJJBAAAA|2451910|2451939|DEPARTMENT|61|70|Customers make social, general premises. None the less close priorities would supervise f|monthly| +6551|AAAAAAAAHJJBAAAA|2451910|2451939|DEPARTMENT|61|71|Other, various occasions ought to make then basic rates. Different,|monthly| +6552|AAAAAAAAIJJBAAAA|2451910|2451939||61|||| +6553|AAAAAAAAJJJBAAAA|2451910|2451939|DEPARTMENT|61|73|Never provincial forces find once more complex, mental characters. On|monthly| +6554|AAAAAAAAKJJBAAAA|2451910|2451939|DEPARTMENT|61|74|Long, new weeks can bring absolutely tall cups. Students shall not generate cap|monthly| +6555|AAAAAAAALJJBAAAA|2451910|2451939|DEPARTMENT|61|75|Objectives co-ordinate hardly cats; years might find; extra, european shows should know |monthly| +6556|AAAAAAAAMJJBAAAA|2451910|2451939|DEPARTMENT|61|76|Institutions bear then active glasses. Natural chains cannot integrate genuinely d|monthly| +6557|AAAAAAAANJJBAAAA|2451910|2451939|DEPARTMENT|61|77|Generally local deposits can know ever patients. Anyway right days o|monthly| +6558|AAAAAAAAOJJBAAAA|2451910|2451939|DEPARTMENT|61|78|Primary loans shall concede due fingers; policemen see cr|monthly| +6559|AAAAAAAAPJJBAAAA|2451910|2451939|DEPARTMENT|61|79|Marginal, noble examples tell hard central policies. Membe|monthly| +6560|AAAAAAAAAKJBAAAA|2451910|2451939|DEPARTMENT|61|80|Factors must intend as prime, other circumstances; radical boats shall not |monthly| +6561|AAAAAAAABKJBAAAA|2451910|2451939|DEPARTMENT|61|81|Clear, young years would become only eyes. Likely, soviet goods assist sooner. Only high designers |monthly| +6562|AAAAAAAACKJBAAAA|2451910|2451939|DEPARTMENT|61|82|Thoughts might try long more suitable lights. Exports should pick quite normal jobs. More dry|monthly| +6563|AAAAAAAADKJBAAAA|2451910|2451939|DEPARTMENT|61|83|Marginal, particular agreements confirm later growing owners. Technical structures |monthly| +6564|AAAAAAAAEKJBAAAA|2451910|2451939|DEPARTMENT|61|84|Grim, external experiments ought to kick usually from a structur|monthly| +6565|AAAAAAAAFKJBAAAA|2451910|2451939|DEPARTMENT|61|85|Hence proper troops prove hands. New features ought to correspond natural lips. Forces d|monthly| +6566|AAAAAAAAGKJBAAAA|2451910|2451939|DEPARTMENT|61|86|Repeatedly able customers shall attend with the communists. Average days see still con|monthly| +6567|AAAAAAAAHKJBAAAA|2451910|2451939|DEPARTMENT|61|87|Countries shall provide however; old, useless groups see hours. Full, perfect teams may n|monthly| +6568|AAAAAAAAIKJBAAAA|2451910|2451939|DEPARTMENT|61|88|Soft, foreign managers stay personal points; only, whole rooms migh|monthly| +6569|AAAAAAAAJKJBAAAA|2451910|2451939|DEPARTMENT|61|89|Units need actually in an regions. Major signals may mean all times. Scottish, functional mem|monthly| +6570|AAAAAAAAKKJBAAAA|2451910|2451939|DEPARTMENT|61|90|New, apparent parents would not enable in accordance with the women. Nervous, local th|monthly| +6571|AAAAAAAALKJBAAAA|2451910|2451939|DEPARTMENT|61|91|However other solutions may meet big, likely observations. Students ne|monthly| +6572|AAAAAAAAMKJBAAAA|2451910|2451939|DEPARTMENT|61|92|Aspects see twice rare, previous meetings. Almost old standards send|monthly| +6573|AAAAAAAANKJBAAAA|2451910|2451939|DEPARTMENT|61|93|Actual plans incorporate long investments. A little unknown files handle rich years. External|monthly| +6574|AAAAAAAAOKJBAAAA|2451910|2451939|DEPARTMENT|61|94|Then domestic years treat decisions. Later other reasons make only official days. Mere, c|monthly| +6575|AAAAAAAAPKJBAAAA|2451910|2451939|DEPARTMENT|61|95|Seriously noble reasons put almost also social causes. Very |monthly| +6576|AAAAAAAAALJBAAAA|2451910|2451939|DEPARTMENT|61|96|Able, long-term kilometres ought to observe extremely. Genuine parents take compa|monthly| +6577|AAAAAAAABLJBAAAA|2451910|2451939|DEPARTMENT|61|97|Profits might not make even as the needs. Forth able trees must confirm|monthly| +6578|AAAAAAAACLJBAAAA|2451910|2451939|DEPARTMENT|61|98|Working-class countries cannot join especially remar|monthly| +6579|AAAAAAAADLJBAAAA|2451910|2451939|DEPARTMENT|61|99|Just great men matter from the masters. Marginal, certain applications may deal at all cen|monthly| +6580|AAAAAAAAELJBAAAA|2451910|2451939|DEPARTMENT|61|100|More local expenses shall not prefer also glad, conservative improvements. Heavy areas may detect |monthly| +6581|AAAAAAAAFLJBAAAA|2451910|2451939|DEPARTMENT|61|101|Blue colonies might see severely american, warm examples. Urban objects|monthly| +6582|AAAAAAAAGLJBAAAA|2451910|2451939|DEPARTMENT|61|102|Likely, static subjects knock in the interests; political b|monthly| +6583|AAAAAAAAHLJBAAAA|2451910|2451939|DEPARTMENT|61|103|Just free boxes might not make of course actions. Aware troops cannot beg ships. Minutes provide to|monthly| +6584|AAAAAAAAILJBAAAA|2451910|2451939|DEPARTMENT|61|104|Experts would obtain from now on local, agricultural teachers. Later a|monthly| +6585|AAAAAAAAJLJBAAAA|2451910|2451939|DEPARTMENT|61|105|Dramatic risks think otherwise. Other representatives mind however types. Correct developments |monthly| +6586|AAAAAAAAKLJBAAAA|2451910|2451939|DEPARTMENT|61|106|Comprehensive, other women come very important children. Complex, subject waters must take by |monthly| +6587|AAAAAAAALLJBAAAA|2451910|2451939|DEPARTMENT|61|107|Acids get remarkable, spectacular rats. Ever general needs would not work unfortunately cultural, |monthly| +6588|AAAAAAAAMLJBAAAA|2451910|2451939|DEPARTMENT|61|108|Annually long feet may find then old, active men; parties ought to decide as certain mountain|monthly| +6589|AAAAAAAANLJBAAAA|2451940|2451969|DEPARTMENT|62|1|Accurately new dates must remember here added good explanations|monthly| +6590|AAAAAAAAOLJBAAAA|2451940|2451969|DEPARTMENT|62|2|Atomic, other places might assume well happy pictures. Serv|monthly| +6591|AAAAAAAAPLJBAAAA|2451940|2451969|DEPARTMENT|62|3|International sequences would not hear thanks. Forces should not get. Comprehensive, developing f|monthly| +6592|AAAAAAAAAMJBAAAA|2451940|2451969|DEPARTMENT|62|4|Witnesses might generate now also social places. So competent goods tak|monthly| +6593|AAAAAAAABMJBAAAA|2451940|2451969|DEPARTMENT|62|5|Semantic, sexual sections shall not carry. Small travellers make alleged, annual dogs. Ot|monthly| +6594|AAAAAAAACMJBAAAA|2451940|2451969|DEPARTMENT|62|6|Just extensive police allow also. Good, human interviews play pol|monthly| +6595|AAAAAAAADMJBAAAA||2451969|||||monthly| +6596|AAAAAAAAEMJBAAAA|2451940|2451969|DEPARTMENT|62|8|Then great hours must not impress often as the stones. Little, social f|monthly| +6597|AAAAAAAAFMJBAAAA|2451940|2451969|DEPARTMENT|62|9|Rural, aware homes notice today red, sharp employers; folk improve in a patients|monthly| +6598|AAAAAAAAGMJBAAAA|2451940|2451969|DEPARTMENT|62|10|Clubs may respond as frequent, scottish talks. Years determine particular, economic flame|monthly| +6599|AAAAAAAAHMJBAAAA|2451940|2451969|DEPARTMENT|62|11|Inevitably level police must discover so till a wome|monthly| +6600|AAAAAAAAIMJBAAAA|2451940|2451969|DEPARTMENT|62|12|Methods used to pursue nevertheless classical, inc men. Relations look once in a|monthly| +6601|AAAAAAAAJMJBAAAA|2451940|2451969|DEPARTMENT|62|13|Different, good pictures will not afford none the less true feeli|monthly| +6602|AAAAAAAAKMJBAAAA|2451940|2451969|DEPARTMENT|62|14|Cold comments come half annual accounts. Ministers would investigate there similar le|monthly| +6603|AAAAAAAALMJBAAAA|2451940|2451969|DEPARTMENT|62|15|Never other households expect as with a americans. Initial, other |monthly| +6604|AAAAAAAAMMJBAAAA|2451940|2451969|DEPARTMENT|62|16|Only other feet can provide contemporary, modern phrases. Royal waters go. Surprised|monthly| +6605|AAAAAAAANMJBAAAA|2451940|2451969|DEPARTMENT|62|17|Other managers must not include behind a duties. Logical, sure standards happen big pic|monthly| +6606|AAAAAAAAOMJBAAAA|2451940|2451969|DEPARTMENT|62|18|Centres buy possible players; signs will not enjoy determined, detailed women.|monthly| +6607|AAAAAAAAPMJBAAAA|2451940|2451969|DEPARTMENT|62|19|Important, local matters used to replace properly more firm genes. Widely other r|monthly| +6608|AAAAAAAAANJBAAAA|2451940|2451969|DEPARTMENT|62|20|Kinds keep enough historical, ready days. Fun economies|monthly| +6609|AAAAAAAABNJBAAAA|2451940|2451969|DEPARTMENT|62|21|National messages may use really managers. Various men ha|monthly| +6610|AAAAAAAACNJBAAAA|2451940|2451969|DEPARTMENT|62|22|Police shall not make instead substantial costs. Creative items|monthly| +6611|AAAAAAAADNJBAAAA|2451940|2451969|DEPARTMENT|62|23|European sources reproduce actually banks; also foreign chi|monthly| +6612|AAAAAAAAENJBAAAA|2451940|2451969|DEPARTMENT|62|24|Fundamental unions should take very parts. Suddenly mutu|monthly| +6613|AAAAAAAAFNJBAAAA|2451940|2451969|DEPARTMENT|62|25|New, sound consequences might go alternatively strong,|monthly| +6614|AAAAAAAAGNJBAAAA|2451940|2451969|DEPARTMENT|62|26|Minutes sit artistic, initial materials. Only large tables borrow equally to|monthly| +6615|AAAAAAAAHNJBAAAA|2451940|2451969|DEPARTMENT|62|27|Schemes address as a questions. Officers cannot consider old, acceptable kilometres; pr|monthly| +6616|AAAAAAAAINJBAAAA|2451940|2451969|DEPARTMENT|62|28|All right living doors play then. So only authorities should feed ever firmly special than|monthly| +6617|AAAAAAAAJNJBAAAA|2451940|2451969|DEPARTMENT|62|29|Other, other signs wear only dangerous, different expressions; units solve never bizarre,|monthly| +6618|AAAAAAAAKNJBAAAA|2451940|2451969|DEPARTMENT|62|30|Also right jobs would discriminate eventually. Reports co|monthly| +6619|AAAAAAAALNJBAAAA|2451940|2451969|DEPARTMENT|62|31|Cold partners shoot no longer; agencies ignore most as the subjects. B|monthly| +6620|AAAAAAAAMNJBAAAA|2451940|2451969|DEPARTMENT|62|32|Qualifications could carry to a cuts; net times will stop. Narrow subjects ensure|monthly| +6621|AAAAAAAANNJBAAAA|2451940|2451969||||Decisions take as as other movements. Towns remember other,|| +6622|AAAAAAAAONJBAAAA|2451940|2451969|DEPARTMENT|62|34|Also common problems experience yesterday offices; prisoners devote already international, bea|monthly| +6623|AAAAAAAAPNJBAAAA|2451940|2451969|DEPARTMENT|62|35|Lucky, soft agreements care private states. Free experts stand widely |monthly| +6624|AAAAAAAAAOJBAAAA|2451940|2451969|DEPARTMENT|62|36|Major, important plans claim apart from a costs. Then environment|monthly| +6625|AAAAAAAABOJBAAAA|2451940|2451969|DEPARTMENT|62|37|Kinds used to find only political eyes. Good cars can |monthly| +6626|AAAAAAAACOJBAAAA|2451940|2451969|DEPARTMENT|62|38|Sometimes only sentences consider only real minutes. International, young taxes|monthly| +6627|AAAAAAAADOJBAAAA|2451940|2451969|DEPARTMENT|62|39|Dramatic teachers help still. Provisions make dates. Waters could affect from the practic|monthly| +6628|AAAAAAAAEOJBAAAA|2451940|2451969|DEPARTMENT|62|40|Expected, dead cups help kindly environmental situations; ro|monthly| +6629|AAAAAAAAFOJBAAAA|2451940|2451969|DEPARTMENT|62|41|Then new powers shall hope then under a parameters. Plans must remember once|monthly| +6630|AAAAAAAAGOJBAAAA|2451940|2451969|DEPARTMENT|62|42|Now available courts ought to replicate fairly. Possible|monthly| +6631|AAAAAAAAHOJBAAAA|2451940|2451969|DEPARTMENT|62|43|Protests ask red, conventional ages; great, terrible animals cut rather |monthly| +6632|AAAAAAAAIOJBAAAA|2451940|2451969|DEPARTMENT|62|44|Fundamental gaps would qualify rather medical, possible wings. Happy eyes mu|monthly| +6633|AAAAAAAAJOJBAAAA|2451940|2451969|DEPARTMENT|62|45|There dominant areas might not make ever married, important facts. Single countries get mostly co|monthly| +6634|AAAAAAAAKOJBAAAA|2451940|2451969|DEPARTMENT|62|46|New schools will not establish once royal, foreign policies. Citizens could not sell good, tr|monthly| +6635|AAAAAAAALOJBAAAA|2451940|2451969|DEPARTMENT|62|47|Northern, long hands clean over so different minutes. Afraid,|monthly| +6636|AAAAAAAAMOJBAAAA|2451940|2451969|DEPARTMENT|62|48|Annual choices remember about for a camps. Formal priests migh|monthly| +6637|AAAAAAAANOJBAAAA|2451940|2451969|DEPARTMENT|62|49|Thirdly professional groups become so illegal, green arts. Mere, swiss shop|monthly| +6638|AAAAAAAAOOJBAAAA|2451940|2451969|DEPARTMENT|62|50|Military, preliminary entries must see exactly; good reasons remind for|monthly| +6639|AAAAAAAAPOJBAAAA|2451940|2451969|DEPARTMENT|62|51|Seldom actual fees pick in the causes. Institutions arrive almost good, mutual stages. Ever able m|monthly| +6640|AAAAAAAAAPJBAAAA|2451940|2451969|DEPARTMENT|62|52|Then various opportunities can prove most true literary studies. Enormous, inappropriate organisat|monthly| +6641|AAAAAAAABPJBAAAA|2451940|2451969|DEPARTMENT|62|53|And so on olympic problems look; nasty, considerable applications sell. Clean products |monthly| +6642|AAAAAAAACPJBAAAA|2451940|2451969|DEPARTMENT|62|54|For example hidden figures afford evidently familiar prop|monthly| +6643|AAAAAAAADPJBAAAA|2451940|2451969|DEPARTMENT|62|55|Evolutionary moments may make only points. Exceptional, current fac|monthly| +6644|AAAAAAAAEPJBAAAA|2451940|2451969|DEPARTMENT|62|56|Numerous, domestic jobs shall take from a assumptions. Workers shall depend dogs. Black prices |monthly| +6645|AAAAAAAAFPJBAAAA|2451940|2451969|DEPARTMENT|62|57|However familiar allegations will not pin also probably true po|monthly| +6646|AAAAAAAAGPJBAAAA|2451940|2451969|DEPARTMENT|62|58|Figures would not love. Economic, scottish countries remain; legiti|monthly| +6647|AAAAAAAAHPJBAAAA|2451940|2451969|DEPARTMENT|62|59|Surprised, national policies should promote inwardly long estimates. Maps ca|monthly| +6648|AAAAAAAAIPJBAAAA|2451940|2451969|DEPARTMENT|62|60|Almost light children face easily about the developments; cards hang partially groups. White|monthly| +6649|AAAAAAAAJPJBAAAA|2451940|2451969|DEPARTMENT|62|61|Existing, remarkable years go however female shelves; also sensible p|monthly| +6650|AAAAAAAAKPJBAAAA|2451940|2451969|DEPARTMENT|62|62|Peculiar industries might prevent whole women. Large flowers host. Sides re-|monthly| +6651|AAAAAAAALPJBAAAA|2451940|2451969|DEPARTMENT|62|63|Around surprising policies defend as equal men. Existing, japanese|monthly| +6652|AAAAAAAAMPJBAAAA|2451940|2451969|DEPARTMENT|62|64|Only knees shall not play about. Satisfactory members will vote softly nearly blue forms. Ev|monthly| +6653|AAAAAAAANPJBAAAA|2451940|2451969|DEPARTMENT|62|65|Suitable, tough revenues can know home general perce|monthly| +6654|AAAAAAAAOPJBAAAA|2451940|2451969|DEPARTMENT|62|66|Public churches shall not recruit by the merchants; then difficult studen|monthly| +6655|AAAAAAAAPPJBAAAA|2451940|2451969|DEPARTMENT|62|67|Therefore christian demands sell probably in a bishops. International, qu|monthly| +6656|AAAAAAAAAAKBAAAA|2451940|2451969|DEPARTMENT|62|68|Main, hostile humans may contain united weeks. Cert|monthly| +6657|AAAAAAAABAKBAAAA|2451940|2451969|DEPARTMENT|62|69|Glad pensioners strengthen with a details. Long, growing aspects swim firmly german skills. Evi|monthly| +6658|AAAAAAAACAKBAAAA|2451940|2451969|DEPARTMENT|62|70|Workers should feel now in the men. Very common things might pretend main, ready cost|monthly| +6659|AAAAAAAADAKBAAAA|2451940|2451969|DEPARTMENT|62|71|Now full reports stand industrial rooms. Imports ought to ea|monthly| +6660|AAAAAAAAEAKBAAAA|2451940|2451969|DEPARTMENT|62|72|Largely short companies put merely naturally small points. Young, left rules may not su|monthly| +6661|AAAAAAAAFAKBAAAA|2451940|2451969|DEPARTMENT|62|73|Only permanent charges could want of course so sure women; forwar|monthly| +6662|AAAAAAAAGAKBAAAA|2451940|2451969|DEPARTMENT|62|74|Women come finally animals. Short-term, general boxes should not count task|monthly| +6663|AAAAAAAAHAKBAAAA|2451940|2451969||||New, large rates should not make there available, single days. Satisfied d|| +6664|AAAAAAAAIAKBAAAA|2451940|2451969|DEPARTMENT|62|76|Police cannot resume thus social, subsequent problems. Just contrary officials exist wom|monthly| +6665|AAAAAAAAJAKBAAAA|2451940|2451969|DEPARTMENT|62|77|Correct funds would not see still again beautiful trousers. Often ci|monthly| +6666|AAAAAAAAKAKBAAAA|2451940|2451969|DEPARTMENT|62|78|False figures get centres. Fine, impossible guns get pote|monthly| +6667|AAAAAAAALAKBAAAA|2451940|2451969|DEPARTMENT|62|79|Weekends come however well increased forms. Just dangerous offic|monthly| +6668|AAAAAAAAMAKBAAAA|2451940|2451969|DEPARTMENT|62|80|High databases stamp by the chemicals. Great, able neighbo|monthly| +6669|AAAAAAAANAKBAAAA|2451940|2451969|DEPARTMENT|62|81|Suspicious texts aim better prisoners. Runs lose greatly better temporary weeks.|monthly| +6670|AAAAAAAAOAKBAAAA|2451940|2451969|DEPARTMENT|62|82|Only words can listen in a cases. Holidays used to tra|monthly| +6671|AAAAAAAAPAKBAAAA|2451940|2451969|DEPARTMENT|62|83|Days must stop well in a groups. Upper, enormous speeches drink however. Settlements |monthly| +6672|AAAAAAAAABKBAAAA|2451940|2451969|DEPARTMENT|62|84|Standards enjoy. Dark, young patterns suit then above, other years; real, na|monthly| +6673|AAAAAAAABBKBAAAA|2451940|2451969|DEPARTMENT|62|85|Rich republics impose nevertheless golden classes.|monthly| +6674|AAAAAAAACBKBAAAA|2451940|2451969|DEPARTMENT|62|86|Certainly warm libraries appear discussions. Sorry levels sha|monthly| +6675|AAAAAAAADBKBAAAA|2451940|2451969|DEPARTMENT|62|87|Grey, private publications shall not know. Libraries lighten later sma|monthly| +6676|AAAAAAAAEBKBAAAA|2451940|2451969|DEPARTMENT|62|88|Good, prime schools tell. Also special losses learn only |monthly| +6677|AAAAAAAAFBKBAAAA|2451940|2451969|DEPARTMENT|62|89|Overall, german reasons need never in a stairs. So|monthly| +6678|AAAAAAAAGBKBAAAA|2451940|2451969|DEPARTMENT|62|90|Forests cause barely to the cuts. Others would guess most as other c|monthly| +6679|AAAAAAAAHBKBAAAA|2451940|2451969|DEPARTMENT|62|91|Largely british constraints ought to connect objective|monthly| +6680|AAAAAAAAIBKBAAAA|2451940|2451969|DEPARTMENT|62|92|Clearly honest problems would resist full, different|monthly| +6681|AAAAAAAAJBKBAAAA|2451940|2451969|DEPARTMENT|62|93|All prospective rights shall try sufficient, safe depths. Families charge stro|monthly| +6682|AAAAAAAAKBKBAAAA|2451940|2451969|DEPARTMENT|62|94|Scottish, good methods gain about rights. Concrete yards shall shoot by no means sligh|monthly| +6683|AAAAAAAALBKBAAAA|2451940|2451969|DEPARTMENT|62|95|Labour, great households feel more little, possible words. Easier real buildings must not find mu|monthly| +6684|AAAAAAAAMBKBAAAA|2451940|2451969|DEPARTMENT|62|96|Civil, public organizations will not offer then indire|monthly| +6685|AAAAAAAANBKBAAAA|2451940|2451969|DEPARTMENT|62|97|More dead risks will disguise professional instances. Consistently large parents build for t|monthly| +6686|AAAAAAAAOBKBAAAA|2451940|2451969|DEPARTMENT|62|98|Vast, broad products will work concerned, poor reaso|monthly| +6687|AAAAAAAAPBKBAAAA|2451940|2451969|DEPARTMENT|62|99|Huge forms vote too. Gently similar hands will know soon in a employees. Advant|monthly| +6688|AAAAAAAAACKBAAAA|2451940|2451969|DEPARTMENT|62|100|Then adverse questions recognize moments. Christian, reluctant beliefs say slowly with a blo|monthly| +6689|AAAAAAAABCKBAAAA|2451940|2451969|DEPARTMENT|62|101|Now different teeth carry also sad needs. Different notions respond only. Types might no|monthly| +6690|AAAAAAAACCKBAAAA|2451940|2451969|DEPARTMENT|62|102|Quiet instances speak roles; natural, neat cases shall assist much in a options; rather past|monthly| +6691|AAAAAAAADCKBAAAA|2451940|2451969|DEPARTMENT|62|103|Special, leading mice should listen even european unions.|monthly| +6692|AAAAAAAAECKBAAAA|2451940|2451969|DEPARTMENT|62|104|Cruel, nuclear things put just; all reasonable prices ought to suit alternately. Soci|monthly| +6693|AAAAAAAAFCKBAAAA|2451940|2451969|DEPARTMENT|62|105|Great, traditional bodies wear also only expectations. Respondents might appear initi|monthly| +6694|AAAAAAAAGCKBAAAA|2451940|2451969|DEPARTMENT|62|106|Children replace of course. Little parties should work quite whole years. The|monthly| +6695|AAAAAAAAHCKBAAAA|2451940|2451969|DEPARTMENT|62|107|Often negative streets provide under very aware office|monthly| +6696|AAAAAAAAICKBAAAA|2451940|2451969|DEPARTMENT|62|108|Small, small elections pay as large police. Mental images stabilise finally bri|monthly| +6697|AAAAAAAAJCKBAAAA|2451970|2451999|DEPARTMENT|63|1|American sales find obviously suspicious studies; concerned, agricultural owners would not ful|monthly| +6698|AAAAAAAAKCKBAAAA|2451970|2451999|DEPARTMENT|63|2|Useful, massive talks would not confess genuinely dead win|monthly| +6699|AAAAAAAALCKBAAAA|2451970|2451999|DEPARTMENT|63|3|Previous events sing with the committees. Great, high sales reduce so troops; also british elect|monthly| +6700|AAAAAAAAMCKBAAAA|2451970|2451999|DEPARTMENT|63|4|Long fashionable depths damage even. Most real winners marry upwards bad words. Great |monthly| +6701|AAAAAAAANCKBAAAA|2451970|2451999|DEPARTMENT|63|5|Married spirits squeeze later late previous books. Only poles should not interrupt about between|monthly| +6702|AAAAAAAAOCKBAAAA|2451970|2451999|DEPARTMENT|63|6|Long single relations shall sell. Full backs call to a years; judicial, supreme artists canno|monthly| +6703|AAAAAAAAPCKBAAAA|2451970|2451999|DEPARTMENT|63|7|Absently sure months put doctors. Places ought to |monthly| +6704|AAAAAAAAADKBAAAA|2451970|2451999|DEPARTMENT|63|8|Men make overtly upwards poor cases; big members cannot discriminate simply |monthly| +6705|AAAAAAAABDKBAAAA|2451970|2451999|DEPARTMENT|63|9|Low artists used to focus evident police. Conservatives m|monthly| +6706|AAAAAAAACDKBAAAA|2451970|2451999|DEPARTMENT|63|10|More urban teachers shall not increase again enquiries. Plea|monthly| +6707|AAAAAAAADDKBAAAA|2451970|2451999|DEPARTMENT|63|11|Important, important uses could introduce high obligations. Blocks call sacred, pri|monthly| +6708|AAAAAAAAEDKBAAAA|2451970|2451999|DEPARTMENT|63|12|All american newspapers confirm publicly full, genuine pieces. Light, welcome conditions used to s|monthly| +6709|AAAAAAAAFDKBAAAA||2451999||63|||monthly| +6710|AAAAAAAAGDKBAAAA|2451970|2451999|DEPARTMENT|63|14|Never angry streets could give indeed whole multiple sons; aspects prefer as necessa|monthly| +6711|AAAAAAAAHDKBAAAA|2451970|2451999|DEPARTMENT|63|15|Roman, heavy arms will not arrive as down a laboratories.|monthly| +6712|AAAAAAAAIDKBAAAA|2451970|2451999|DEPARTMENT|63|16|Thus dead differences shall become with a bodies. Disastr|monthly| +6713|AAAAAAAAJDKBAAAA|2451970|2451999|DEPARTMENT|63|17|Primarily busy weeks might not become always needs. Recently present sa|monthly| +6714|AAAAAAAAKDKBAAAA|2451970|2451999|DEPARTMENT|63|18|Else helpful accounts could persuade altogether annual, difficult childre|monthly| +6715|AAAAAAAALDKBAAAA|2451970|2451999|DEPARTMENT|63|19|Other estimates might fill. Raw results can acquire figures. Highly superb sessions |monthly| +6716|AAAAAAAAMDKBAAAA|2451970|2451999|DEPARTMENT|63|20|Troops shall not postpone today; thence close variations know clothes. Fin|monthly| +6717|AAAAAAAANDKBAAAA|2451970|2451999|DEPARTMENT|63|21|White others need. Responsibilities ought to believe to the chang|monthly| +6718|AAAAAAAAODKBAAAA|2451970|2451999|DEPARTMENT|63|22|Also particular cells turn far pounds. Female eyes hear about russian bonds; a lit|monthly| +6719|AAAAAAAAPDKBAAAA|2451970|2451999|DEPARTMENT|63|23|Average, close wines shall sing seriously common ways. Ever big things decide finally ju|monthly| +6720|AAAAAAAAAEKBAAAA|2451970|2451999|DEPARTMENT|63|24|Comments stop so united forces. Annual, big subjects would include critic|monthly| +6721|AAAAAAAABEKBAAAA|2451970|2451999|DEPARTMENT|63|25|Games take following, large things; original wages look there phases. Effects wo|monthly| +6722|AAAAAAAACEKBAAAA|2451970|2451999|DEPARTMENT|63|26|Folk shall not find very between a countries. Obviously equivalent prices wil|monthly| +6723|AAAAAAAADEKBAAAA|2451970|2451999|DEPARTMENT|63|27|Affairs will pull then into a months. Rules ought to drop; expenses would go scho|monthly| +6724|AAAAAAAAEEKBAAAA|2451970|2451999|DEPARTMENT|63|28|Fully other levels bring quite financial arms. Commonly international schools used t|monthly| +6725|AAAAAAAAFEKBAAAA|2451970|2451999|DEPARTMENT|63|29|Judicial, single houses obtain therefore less able thousands; e|monthly| +6726|AAAAAAAAGEKBAAAA|2451970|2451999|DEPARTMENT|63|30|Major champions shall hold low such as the properties. |monthly| +6727|AAAAAAAAHEKBAAAA|2451970|2451999|DEPARTMENT|63|31|Profits give loud in a police. Aware women assess in a pictures. Ago public developers will make|monthly| +6728|AAAAAAAAIEKBAAAA|2451970|2451999|DEPARTMENT|63|32|German, white reports could start very. Sometimes surprised reactions cannot find by |monthly| +6729|AAAAAAAAJEKBAAAA|2451970|2451999|DEPARTMENT|63|33|Remarkable, considerable rooms leave on a groups; references see warm, alive student|monthly| +6730|AAAAAAAAKEKBAAAA|2451970|2451999|DEPARTMENT|63|34|Years mind especially current methods. Important, ot|monthly| +6731|AAAAAAAALEKBAAAA|2451970|2451999|DEPARTMENT|63|35|Plants ought to get consciously for the things. Publications might offe|monthly| +6732|AAAAAAAAMEKBAAAA|2451970|2451999|DEPARTMENT|63|36|Little left children could vote then other, competitive area|monthly| +6733|AAAAAAAANEKBAAAA|2451970|2451999|DEPARTMENT|63|37|Finally unknown theories would look average trusts. Conve|monthly| +6734|AAAAAAAAOEKBAAAA|2451970|2451999|DEPARTMENT|63|38|Large, sure costs see secondly. Quite respectable grounds follow. Other, open f|monthly| +6735|AAAAAAAAPEKBAAAA|2451970|2451999|DEPARTMENT|63|39|Daughters should travel abroad. Cheap, historical sets insi|monthly| +6736|AAAAAAAAAFKBAAAA|2451970|2451999|DEPARTMENT|63|40|Blind, new places could act this from a associations. |monthly| +6737|AAAAAAAABFKBAAAA|2451970|2451999|DEPARTMENT|63|41|Rural cars will not go certain doors. Uncertain, complete |monthly| +6738|AAAAAAAACFKBAAAA|2451970|2451999|DEPARTMENT|63|42|Modern horses withdraw. Writers will think again experimental, wrong marks.|monthly| +6739|AAAAAAAADFKBAAAA|2451970|2451999|DEPARTMENT|63|43|Degrees shall not limit much. Possible, long doubts attack partly small, similar p|monthly| +6740|AAAAAAAAEFKBAAAA|2451970|2451999|DEPARTMENT|63|44|Dangerous problems used to follow above obvious hills. Later total books may dismiss on th|monthly| +6741|AAAAAAAAFFKBAAAA|2451970|2451999|DEPARTMENT|63|45|Anyway alternative others should not reduce hopefully fascinating, following calls. Major par|monthly| +6742|AAAAAAAAGFKBAAAA|2451970|2451999|DEPARTMENT|63|46|Long humans add accidentally long, grey times. Products might not spawn for a schools|monthly| +6743|AAAAAAAAHFKBAAAA|2451970|2451999|DEPARTMENT|63|47|Sales ought to mention more than with the errors. Real, electoral missiles|monthly| +6744|AAAAAAAAIFKBAAAA|2451970|2451999|DEPARTMENT|63|48|Dangerous elections ought to borrow opposite muscles. Economic measures ought to answer|monthly| +6745|AAAAAAAAJFKBAAAA|2451970|2451999|DEPARTMENT|63|49|Stages start away sad pilots. Successful, statistical shares could not supply ultimately ol|monthly| +6746|AAAAAAAAKFKBAAAA|2451970|2451999|DEPARTMENT|63|50|About physical payments may concentrate. Times might cut often |monthly| +6747|AAAAAAAALFKBAAAA|2451970|2451999|DEPARTMENT|63|51|Mad, rich needs control to the groups; teeth use currently. Representations could get green|monthly| +6748|AAAAAAAAMFKBAAAA|2451970|2451999|DEPARTMENT|63|52|Always deep frames flourish casual, social telecommunications. Carefully manufacturing pub|monthly| +6749|AAAAAAAANFKBAAAA|2451970|2451999|DEPARTMENT|63|53|For example legal patterns come only multiple, red negotiations. Yesterday nice othe|monthly| +6750|AAAAAAAAOFKBAAAA|2451970|2451999|DEPARTMENT|63|54|Other years see yet national governments. Strengths releas|monthly| +6751|AAAAAAAAPFKBAAAA|2451970|2451999|DEPARTMENT|63|55|Measures broadcast very valid, hard drawings. Brilliant, formal sets could need now |monthly| +6752|AAAAAAAAAGKBAAAA|2451970|2451999|DEPARTMENT|63|56|National figures could not draw working, white schemes. Also individual affairs work light rocks. |monthly| +6753|AAAAAAAABGKBAAAA|2451970|2451999|DEPARTMENT|63|57|Yet close opponents rape around white citizens. Social, interesting bodies want predi|monthly| +6754|AAAAAAAACGKBAAAA|2451970|2451999|DEPARTMENT|63|58|Economic lands point also by a conditions. Members may screen gaps. Agricultural, ab|monthly| +6755|AAAAAAAADGKBAAAA|2451970|2451999|DEPARTMENT|63|59|Available sports experience very questions. Following others make good pictures. P|monthly| +6756|AAAAAAAAEGKBAAAA|2451970|2451999|DEPARTMENT|63|60|Cool central jobs eat there to an names. Active, future girls may reinstate high men. Studies can l|monthly| +6757|AAAAAAAAFGKBAAAA|2451970|2451999|DEPARTMENT|63|61|Here leading groups should enact far from both isolated pol|monthly| +6758|AAAAAAAAGGKBAAAA|2451970|2451999|DEPARTMENT|63|62|Surroundings shall grab political, young drivers. |monthly| +6759|AAAAAAAAHGKBAAAA|2451970|2451999|DEPARTMENT|63|63|Just strange views cannot find perhaps. Exactly small problems could won|monthly| +6760|AAAAAAAAIGKBAAAA|2451970|2451999|DEPARTMENT|63|64|As liberal resources could not create only early close miles. Able, gentle pr|monthly| +6761|AAAAAAAAJGKBAAAA|2451970|2451999|DEPARTMENT|63|65|Major, different arms deter in the birds. Problems may satisfy even by a days. Terms repair a|monthly| +6762|AAAAAAAAKGKBAAAA|2451970|2451999|DEPARTMENT|63|66|However light spirits will acknowledge again with a terms. Therefore real islands can mon|monthly| +6763|AAAAAAAALGKBAAAA|2451970|2451999|DEPARTMENT|63|67|Good, early limits must see moments. Themes ought to feel th|monthly| +6764|AAAAAAAAMGKBAAAA|2451970|2451999|DEPARTMENT|63|68|Months face words. Studies depend only historical taxes; official parties should wish secure direct|monthly| +6765|AAAAAAAANGKBAAAA|2451970|2451999|DEPARTMENT|63|69|Other, single banks near personal states; yesterday difficult houses maintain today at all person|monthly| +6766|AAAAAAAAOGKBAAAA|2451970|2451999|DEPARTMENT|63|70|Political drawings argue indeed large locations. Foreign, good jobs ought to give from a|monthly| +6767|AAAAAAAAPGKBAAAA|2451970|2451999|DEPARTMENT|63|71|Large views throw never ago similar emissions. Delighted, industrial women would care effec|monthly| +6768|AAAAAAAAAHKBAAAA|2451970|2451999|DEPARTMENT|63|72|Goals should offer early, detailed years. Cattle used to w|monthly| +6769|AAAAAAAABHKBAAAA|2451970|2451999|DEPARTMENT|63|73|Main, national matters should enter small reports. As spanish parts|monthly| +6770|AAAAAAAACHKBAAAA|2451970|2451999|DEPARTMENT|63|74|Chemical plans would not make far then southern laws. Never|monthly| +6771|AAAAAAAADHKBAAAA|2451970|2451999|DEPARTMENT|63|75|Labour principles ought to leave crude matters. More blind |monthly| +6772|AAAAAAAAEHKBAAAA|2451970|2451999|DEPARTMENT|63|76|Occasional, possible classes adapt well by a wishes. Shortly responsible arguments |monthly| +6773|AAAAAAAAFHKBAAAA|2451970|2451999|DEPARTMENT|63|77|Hidden, special families want. Good degrees exchange just. So royal children|monthly| +6774|AAAAAAAAGHKBAAAA|2451970|2451999|DEPARTMENT|63|78|Attempts will reveal exclusively. Monthly, regional errors follow again other words. Long, new c|monthly| +6775|AAAAAAAAHHKBAAAA|2451970|2451999|DEPARTMENT|63|79|Political, possible metres must oppose to a conditions. Certainly gentle members will try any |monthly| +6776|AAAAAAAAIHKBAAAA|2451970|2451999|DEPARTMENT|63|80|Occupational, similar years boost well of course rich representatives. Vulnerable foundations|monthly| +6777|AAAAAAAAJHKBAAAA|2451970|2451999|DEPARTMENT|63|81|Keen, relative companies might allow anyhow. Operational, royal organis|monthly| +6778|AAAAAAAAKHKBAAAA|2451970|2451999|DEPARTMENT|63|82|Other, general mice must touch able, french requirements. Concessions g|monthly| +6779|AAAAAAAALHKBAAAA|2451970|2451999|DEPARTMENT|63|83|Probably particular areas could rest. Children will sustain; groups would meet sligh|monthly| +6780|AAAAAAAAMHKBAAAA|2451970|2451999|DEPARTMENT|63|84|More serious elections could tell suddenly. Open, following years share heavily students. Alway|monthly| +6781|AAAAAAAANHKBAAAA|2451970|2451999|DEPARTMENT|63|85|New, national instances will not warn still years. Secondly primary rises will not eat at all fro|monthly| +6782|AAAAAAAAOHKBAAAA|2451970|2451999|DEPARTMENT|63|86|At once racial refugees ought to safeguard days. Small creditors us|monthly| +6783|AAAAAAAAPHKBAAAA|2451970|2451999|DEPARTMENT|63|87|Around strange stocks ought to bring perfect, administrative windows.|monthly| +6784|AAAAAAAAAIKBAAAA|2451970|2451999|DEPARTMENT|63|88|Cold, soft facilities give now new, respective minutes. British, expected artists see |monthly| +6785|AAAAAAAABIKBAAAA|2451970|2451999|DEPARTMENT|63|89|Then high hours live away of course military lines. Able instruct|monthly| +6786|AAAAAAAACIKBAAAA|2451970|2451999|DEPARTMENT|63|90|Poor, current areas may lose nations. Similar eyes reduce |monthly| +6787|AAAAAAAADIKBAAAA|2451970|2451999|DEPARTMENT|63|91|Both early patients want british, entire chairs. External, full shoes take upon a pro|monthly| +6788|AAAAAAAAEIKBAAAA|2451970|2451999|DEPARTMENT|63|92|Now level values might precipitate broadly relevant industrial arti|monthly| +6789|AAAAAAAAFIKBAAAA|2451970|2451999|DEPARTMENT|63|93|Particular, easy pp. hear still new, inner curtains. Similar, capital divisions sh|monthly| +6790|AAAAAAAAGIKBAAAA|2451970|2451999|DEPARTMENT|63|94|Confident benefits acknowledge freely large studies. Spare, national years could matt|monthly| +6791|AAAAAAAAHIKBAAAA|2451970|2451999|DEPARTMENT|63|95|Industrial, detailed thoughts go light vehicles. Great, future symptoms prevent politically sensibl|monthly| +6792|AAAAAAAAIIKBAAAA|2451970|2451999|DEPARTMENT|63|96|Libraries live. Black farmers cannot hear on the lips. Years change major guns. Italian powe|monthly| +6793|AAAAAAAAJIKBAAAA|2451970|2451999|DEPARTMENT|63|97|For example different methods could set really blacks. Terms ought t|monthly| +6794|AAAAAAAAKIKBAAAA|2451970|2451999||63|||monthly| +6795|AAAAAAAALIKBAAAA|2451970|2451999|DEPARTMENT|63|99|Different departments want indeed by a votes. Financial moments know st|monthly| +6796|AAAAAAAAMIKBAAAA|2451970|2451999|DEPARTMENT|63|100|Front, video-taped patients believe; proposed services see cheap miles. Ethnic, necessary talks m|monthly| +6797|AAAAAAAANIKBAAAA|2451970|2451999|DEPARTMENT|63|101|Literally surprising rights reach as very availabl|monthly| +6798|AAAAAAAAOIKBAAAA|2451970|2451999|DEPARTMENT|63|102|Different consequences carry single bills. So political users capture only revenues. J|monthly| +6799|AAAAAAAAPIKBAAAA|2451970|2451999|DEPARTMENT|63|103|Difficult devices shall display apparently by a operators; ideal f|monthly| +6800|AAAAAAAAAJKBAAAA|2451970|2451999|DEPARTMENT|63|104|Owners may go to a areas. Boys might bear mentally. Arguments nea|monthly| +6801|AAAAAAAABJKBAAAA|2451970|2451999|DEPARTMENT|63|105|Immediate, complex rewards meet very new beliefs. Areas must show upper miles. Modern difficu|monthly| +6802|AAAAAAAACJKBAAAA|2451970|2451999|DEPARTMENT|63|106|Administrative sales would not arrive enough in a banks. Courts will suggest by a sa|monthly| +6803|AAAAAAAADJKBAAAA|2451970|2451999|DEPARTMENT|63|107|Neighbours should not use better only shared members. Slow subjects sho|monthly| +6804|AAAAAAAAEJKBAAAA|2451970|2451999|DEPARTMENT|63|108|Now religious years will find merely legal, interes|monthly| +6805|AAAAAAAAFJKBAAAA|2452000|2452029|DEPARTMENT|64|1|Procedures can tell now modern trees. Eyes protect however chief differences; even small re|monthly| +6806|AAAAAAAAGJKBAAAA|2452000|2452029|DEPARTMENT|64|2|There local sheets may set officially in a languages. Direct, complete s|monthly| +6807|AAAAAAAAHJKBAAAA|2452000|2452029|DEPARTMENT|64|3|Direct others compete. Even industrial shoulders make even annual aspects. Recen|monthly| +6808|AAAAAAAAIJKBAAAA|2452000|2452029|DEPARTMENT|64|4|Goals sound really local, immediate wives. Official probl|monthly| +6809|AAAAAAAAJJKBAAAA|2452000|2452029|DEPARTMENT|64|5|Simple, rural programmes might unveil from no controls. Sales cou|monthly| +6810|AAAAAAAAKJKBAAAA|2452000|2452029|DEPARTMENT|64|6|Over social demands continue at last. Religious, large beliefs see; generously new bands merge |monthly| +6811|AAAAAAAALJKBAAAA|2452000|2452029|DEPARTMENT|64|7|Then related systems arise respective, economic concessions. Additional price|monthly| +6812|AAAAAAAAMJKBAAAA|2452000|2452029|DEPARTMENT|64|8|Well british recordings translate redundant firms. Political finge|monthly| +6813|AAAAAAAANJKBAAAA|2452000|2452029|DEPARTMENT|64|9|Free, marginal workers consider beautiful supporters. Components might shed very alo|monthly| +6814|AAAAAAAAOJKBAAAA|2452000|2452029|DEPARTMENT|64|10|Routes will get progressively external places. Facts f|monthly| +6815|AAAAAAAAPJKBAAAA|2452000|2452029|DEPARTMENT|64|11|Recent terms shall pass more determined rates; specimen|monthly| +6816|AAAAAAAAAKKBAAAA|2452000|2452029|DEPARTMENT|64|12|Golden intentions will not achieve others. Public, positive thanks can know just good dimensi|monthly| +6817|AAAAAAAABKKBAAAA|2452000|2452029|DEPARTMENT|64|13|Exactly agricultural questions mention anyway. Women|monthly| +6818|AAAAAAAACKKBAAAA|2452000|2452029|DEPARTMENT|64|14|Areas may stretch less current schools. Years could not ensure really members. English, political |monthly| +6819|AAAAAAAADKKBAAAA|2452000|2452029|DEPARTMENT|64|15|Customers used to recall widely. Years might ensure highly excit|monthly| +6820|AAAAAAAAEKKBAAAA|2452000|2452029|DEPARTMENT|64|16|Long, large relations might interfere somewhat best different pai|monthly| +6821|AAAAAAAAFKKBAAAA|2452000|2452029|DEPARTMENT|64|17|Also large changes help young, necessary topics; forms prove later. Prod|monthly| +6822|AAAAAAAAGKKBAAAA|2452000|2452029|DEPARTMENT|64|18|Christians learn proceedings. Perhaps responsible reports find in par|monthly| +6823|AAAAAAAAHKKBAAAA|2452000|2452029|DEPARTMENT|64|19|Representative photographs may not give huge years. Only bare societies fit|monthly| +6824|AAAAAAAAIKKBAAAA|2452000|2452029|DEPARTMENT|64|20|Efficiently bizarre actions get almost administrative, difficu|monthly| +6825|AAAAAAAAJKKBAAAA|2452000|2452029|DEPARTMENT|64|21|Issues protest so final rats. Reasons must hear; competi|monthly| +6826|AAAAAAAAKKKBAAAA|2452000|2452029|DEPARTMENT|64|22|Normal, legitimate years accept below the countries. Early disciplines conve|monthly| +6827|AAAAAAAALKKBAAAA|2452000|2452029|DEPARTMENT|64|23|Competitive times used to support then much traditional products. Statistical forests help li|monthly| +6828|AAAAAAAAMKKBAAAA|2452000|2452029|DEPARTMENT|64|24|National, old families know real, living kilometres. D|monthly| +6829|AAAAAAAANKKBAAAA|2452000|2452029|DEPARTMENT|64|25|Later high resources shall not change about other books; unexpected tears used to te|monthly| +6830|AAAAAAAAOKKBAAAA|2452000|2452029|DEPARTMENT|64|26|Subsequent, local features may set there simple pieces. Active, available |monthly| +6831|AAAAAAAAPKKBAAAA|2452000|2452029|DEPARTMENT|64|27|Particular, right galleries might not hang relatively interior, separate copies. Basic |monthly| +6832|AAAAAAAAALKBAAAA|2452000|2452029|DEPARTMENT|64|28|Modules will help well by the forces; young, unique plant|monthly| +6833|AAAAAAAABLKBAAAA|2452000|2452029|DEPARTMENT|64|29|Still historic places may lose always from a thoughts; emotional, economic elections like |monthly| +6834|AAAAAAAACLKBAAAA|2452000|2452029|DEPARTMENT|64|30|Flat negative principles join possible tickets. Both good terms|monthly| +6835|AAAAAAAADLKBAAAA|2452000|2452029|DEPARTMENT|64|31|Weapons should not say right centuries. Early campaigns |monthly| +6836|AAAAAAAAELKBAAAA|2452000|2452029|DEPARTMENT|64|32|Requirements will not go then alone, crude plans. Wrongly square characteristics can continue prob|monthly| +6837|AAAAAAAAFLKBAAAA|2452000|2452029|DEPARTMENT|64|33|Special, appointed schools ought to acquire heavily new paths. De|monthly| +6838|AAAAAAAAGLKBAAAA|2452000|2452029|DEPARTMENT|64|34|Civil blocks will expect original cells. Rights may lend so police. Very sheets convince of|monthly| +6839|AAAAAAAAHLKBAAAA|2452000|2452029|DEPARTMENT|64|35|Labour tensions should present still inevitable concessions|monthly| +6840|AAAAAAAAILKBAAAA|2452000|2452029|DEPARTMENT|64|36|Chief, irish states involve also. Questions would take as yet too back minutes. Politi|monthly| +6841|AAAAAAAAJLKBAAAA|2452000|2452029|DEPARTMENT|64|37|Secondly wild hours would not say actually intrinsically comparative users|monthly| +6842|AAAAAAAAKLKBAAAA|2452000|2452029|DEPARTMENT|64|38|Key, unique bands must land simply in the relatives. Convincing, elect|monthly| +6843|AAAAAAAALLKBAAAA|2452000|2452029|DEPARTMENT|64|39|Days understand both multiple ideas. Modern families turn men. Explanations |monthly| +6844|AAAAAAAAMLKBAAAA|2452000|2452029|DEPARTMENT|64|40|Needs ought to trust for the events. At first social others hang |monthly| +6845|AAAAAAAANLKBAAAA|2452000|2452029|DEPARTMENT|64|41|Average, foreign banks will deter specially eventually equal tonnes. Sor|monthly| +6846|AAAAAAAAOLKBAAAA|2452000|2452029|DEPARTMENT|64|42|Little, simple countries use hence; entirely large m|monthly| +6847|AAAAAAAAPLKBAAAA|2452000|2452029|DEPARTMENT|64|43|Basic feet assess previously late perfect conditions. Certain parties make unfortunately; arrangem|monthly| +6848|AAAAAAAAAMKBAAAA|2452000|2452029|DEPARTMENT|64|44|Systems come even expected, complex consequences. New, victorian|monthly| +6849|AAAAAAAABMKBAAAA|2452000|2452029|DEPARTMENT|64|45|Different, true premises know professional payments. Coming, alone members m|monthly| +6850|AAAAAAAACMKBAAAA|2452000|2452029|DEPARTMENT|64|46|Together white boys should not express. Men demand serv|monthly| +6851|AAAAAAAADMKBAAAA|2452000|2452029|DEPARTMENT|64|47|Records would pay; features take as basic countries. |monthly| +6852|AAAAAAAAEMKBAAAA|2452000|2452029|DEPARTMENT|64|48|British, interior months should happen therefore tremendo|monthly| +6853|AAAAAAAAFMKBAAAA|2452000|2452029|DEPARTMENT|64|49|Sufficient, professional results help more shoulders|monthly| +6854|AAAAAAAAGMKBAAAA|2452000|2452029|DEPARTMENT|64|50|Again sexual claims can act international, global attitudes. New conditions can marry perhaps |monthly| +6855|AAAAAAAAHMKBAAAA|2452000|2452029|DEPARTMENT|64|51|Low stations promote more multiple sciences. Now creative obligations |monthly| +6856|AAAAAAAAIMKBAAAA|2452000|2452029|DEPARTMENT|64|52|Novel, important findings must feel by the movements.|monthly| +6857|AAAAAAAAJMKBAAAA|2452000|2452029|DEPARTMENT|64|53|Away unable directors used to hold remotely in a tools. Supporters might commit methods. O|monthly| +6858|AAAAAAAAKMKBAAAA|2452000|2452029|DEPARTMENT|64|54|Authorities ensure current, palestinian rivers; alone, sexual attempts d|monthly| +6859|AAAAAAAALMKBAAAA|2452000|2452029|DEPARTMENT|64|55|Old difficulties go in the practices. Relations shall remain very japanese points; |monthly| +6860|AAAAAAAAMMKBAAAA|2452000|2452029|DEPARTMENT|64|56|Simple, political poems make in a weapons. Serious, other eyes used to seem. Kind units ought to|monthly| +6861|AAAAAAAANMKBAAAA|2452000|2452029|DEPARTMENT|64|57|Schools may not replace large, large months; active, ex|monthly| +6862|AAAAAAAAOMKBAAAA|2452000|2452029|DEPARTMENT|64|58|Services must want forward here conservative parts. Reluctantly main boots len|monthly| +6863|AAAAAAAAPMKBAAAA|2452000|2452029|DEPARTMENT|64|59|Monetary assets take then financial thoughts. Different, average societies may start away cl|monthly| +6864|AAAAAAAAANKBAAAA|2452000|2452029|DEPARTMENT|64|60|Personal, long studies set. So young kids recognise then from a windows. Poor differences |monthly| +6865|AAAAAAAABNKBAAAA|2452000|2452029|DEPARTMENT|64|61|Scarcely cultural bonds will not say events. At all other parents feel simply individual, |monthly| +6866|AAAAAAAACNKBAAAA|2452000|2452029|DEPARTMENT|64|62|Prime, existing rates may not add new changes. Coherent children bu|monthly| +6867|AAAAAAAADNKBAAAA|2452000|2452029|DEPARTMENT|64|63|Responsible, young events announce simply lists. Normal, individ|monthly| +6868|AAAAAAAAENKBAAAA|2452000|2452029|DEPARTMENT|64|64|Indeed individual windows should work theories. Devices climb. Also prime|monthly| +6869|AAAAAAAAFNKBAAAA|2452000|2452029|DEPARTMENT|64|65|Big acids express old costs; demanding types seem really out of a members. Tropical detai|monthly| +6870|AAAAAAAAGNKBAAAA|2452000|2452029|DEPARTMENT|64|66|Subsequent, industrial situations write on the days. Difficult, hot friends must need today |monthly| +6871|AAAAAAAAHNKBAAAA|2452000|2452029|DEPARTMENT|64|67|Full, important bars define carefully sharp fires. Primary, international signs |monthly| +6872|AAAAAAAAINKBAAAA|2452000|2452029|DEPARTMENT|64|68|National members substitute forward, lucky minutes. Industrial employees|monthly| +6873|AAAAAAAAJNKBAAAA|2452000|2452029|DEPARTMENT|64|69|Industries may get today unknown assumptions; areas operate again below ni|monthly| +6874|AAAAAAAAKNKBAAAA|2452000|2452029|DEPARTMENT|64|70|Adversely complex decades might glance; also positive situations d|monthly| +6875|AAAAAAAALNKBAAAA|2452000|2452029|DEPARTMENT|64|71|That is to say young cats could not make sufficient differences|monthly| +6876|AAAAAAAAMNKBAAAA|2452000|2452029|DEPARTMENT|64|72|Quite important methods hurry in the experts. Early, recent|monthly| +6877|AAAAAAAANNKBAAAA|2452000|2452029|DEPARTMENT|64|73|Mice may express eventually technical elements. Capable, private addresses present. Offic|monthly| +6878|AAAAAAAAONKBAAAA|2452000|2452029|DEPARTMENT|64|74|Eyes command only rapid parents. Careful, major places hasten|monthly| +6879|AAAAAAAAPNKBAAAA|2452000|2452029|DEPARTMENT|64|75|Free, numerous instructions demand of course in the police. Men shall not seek inches; |monthly| +6880|AAAAAAAAAOKBAAAA|2452000|2452029|DEPARTMENT|64|76|Now european numbers could not include objections. Objects get; left standards stay high|monthly| +6881|AAAAAAAABOKBAAAA|2452000|2452029|DEPARTMENT|64|77|Relevant, available historians used to mention even full|monthly| +6882|AAAAAAAACOKBAAAA|2452000|2452029|DEPARTMENT|64|78|Special, small months stay under fundamental customers. Pp. will not be|monthly| +6883|AAAAAAAADOKBAAAA|2452000|2452029|DEPARTMENT|64|79|Acts might not cause as genuinely worried experts. Narrow, peculiar goals get geogr|monthly| +6884|AAAAAAAAEOKBAAAA|2452000|2452029|DEPARTMENT|64|80|Recent workers practise as well. Different fingers ought to name friends. Genetic, a|monthly| +6885|AAAAAAAAFOKBAAAA|2452000|2452029|DEPARTMENT|64|81|Now familiar bags turn large, difficult decades; large, central c|monthly| +6886|AAAAAAAAGOKBAAAA|2452000|2452029|DEPARTMENT|64|82|Single, available figures will revive then exports. Scottish, mean men sh|monthly| +6887|AAAAAAAAHOKBAAAA|2452000|2452029|DEPARTMENT|64|83|So military prices could use only similar, other schools. Military, sweet |monthly| +6888|AAAAAAAAIOKBAAAA|2452000|2452029|DEPARTMENT|64|84|Particles come times; easy, short-term notions sha|monthly| +6889|AAAAAAAAJOKBAAAA|2452000|2452029|DEPARTMENT|64|85|Probably major features insert entirely rather ali|monthly| +6890|AAAAAAAAKOKBAAAA|2452000|2452029|DEPARTMENT|64|86|Only neighbouring purposes may progress engines. Mo|monthly| +6891|AAAAAAAALOKBAAAA|2452000|2452029|DEPARTMENT|64|87|However elderly walls distract with a prisoners. Old hospitals would not discover |monthly| +6892|AAAAAAAAMOKBAAAA|2452000|2452029|DEPARTMENT|64|88|Diplomatic years miss approximately reforms. Here natural designers deny jus|monthly| +6893|AAAAAAAANOKBAAAA|2452000|2452029|DEPARTMENT|64|89|Single, additional bombs will break about in a children. Combined, natural sto|monthly| +6894|AAAAAAAAOOKBAAAA|2452000|2452029|DEPARTMENT|64|90|Large cases enter in a shops. New allegations could come site|monthly| +6895|AAAAAAAAPOKBAAAA|2452000|2452029|DEPARTMENT|64|91|Raw stones opt to a arms. Home simple blues want either. Now e|monthly| +6896|AAAAAAAAAPKBAAAA|2452000|2452029|DEPARTMENT|64|92|Here common opportunities think late, sudden budgets. Moments hold much single insect|monthly| +6897|AAAAAAAABPKBAAAA|2452000|2452029|DEPARTMENT|64|93|Genuinely primary wives used to tell affairs. Services ought to relax blue prefe|monthly| +6898|AAAAAAAACPKBAAAA|2452000|2452029|DEPARTMENT|64|94|American, other years would not move just early, red stocks. Other, expensive |monthly| +6899|AAAAAAAADPKBAAAA|2452000|2452029|DEPARTMENT|64|95|Curious, financial parents ought to see more total banks; months support furthe|monthly| +6900|AAAAAAAAEPKBAAAA|2452000|2452029|DEPARTMENT|64|96|A little level records pick even common trends. Thus irish deaths|monthly| +6901|AAAAAAAAFPKBAAAA|2452000|2452029|DEPARTMENT|64|97|Simple, following attempts share years. Successful, provincial relations used to impose moral|monthly| +6902|AAAAAAAAGPKBAAAA|2452000|2452029|DEPARTMENT|64|98|Imperial, unlikely parents influence aggressively resources; long d|monthly| +6903|AAAAAAAAHPKBAAAA|2452000|||64||Extreme, academic workers may make funny stands. Boots will ride together mysteriously top |monthly| +6904|AAAAAAAAIPKBAAAA|2452000|2452029|DEPARTMENT|64|100|Scottish, international birds will not use now then clinical years. Just local s|monthly| +6905|AAAAAAAAJPKBAAAA|2452000|2452029|DEPARTMENT|64|101|Curtains get again pupils. Situations used to reveal over important, intimate men; holes sha|monthly| +6906|AAAAAAAAKPKBAAAA|2452000|2452029|DEPARTMENT|64|102|National, clean vehicles pay with a ships. Longer unhap|monthly| +6907|AAAAAAAALPKBAAAA|2452000|2452029|DEPARTMENT|64|103|Home other efforts print sometimes rigid, full crops. Logically easy states ent|monthly| +6908|AAAAAAAAMPKBAAAA|2452000|2452029|DEPARTMENT|64|104|Men remember there; dishes speak old trustees. Subsequent, medical churches could go |monthly| +6909|AAAAAAAANPKBAAAA|2452000|2452029|DEPARTMENT|64|105|Full, naked balls continue anyway; reluctant poles know again however fiscal |monthly| +6910|AAAAAAAAOPKBAAAA|2452000|2452029|DEPARTMENT|64|106|Absolute talks find strong kings. Modern, full details could not ge|monthly| +6911|AAAAAAAAPPKBAAAA|2452000|2452029|DEPARTMENT|64|107|Parliamentary, special days might deny intimate, competitive years; fun, northern de|monthly| +6912|AAAAAAAAAALBAAAA|2452000|2452029|DEPARTMENT|64|108|However social purposes regard yesterday considerable years. Environmental messages remember here|monthly| +6913|AAAAAAAABALBAAAA|2452030|2452059|DEPARTMENT|65|1|Plates may not take soon for the time being separate demands; now releva|monthly| +6914|AAAAAAAACALBAAAA|2452030|2452059|DEPARTMENT|65|2|Regularly national miles take only good languages. Really present opt|monthly| +6915|AAAAAAAADALBAAAA|2452030|2452059|DEPARTMENT|65|3|Actually aware sports may help hardly international days. Suddenly ea|monthly| +6916|AAAAAAAAEALBAAAA|2452030|2452059|DEPARTMENT|65|4|Marvellous, wooden minutes should not take more. Words m|monthly| +6917|AAAAAAAAFALBAAAA|2452030|2452059|DEPARTMENT|65|5|Good, private poles arise always right, domestic thoughts. Horrible games establ|monthly| +6918|AAAAAAAAGALBAAAA|2452030||DEPARTMENT|65|||monthly| +6919|AAAAAAAAHALBAAAA|2452030|2452059|DEPARTMENT|65|7|A bit internal benefits could operate anyway of co|monthly| +6920|AAAAAAAAIALBAAAA|2452030|2452059|DEPARTMENT|65|8|Views could exist about possible divisions. Clothes would fade|monthly| +6921|AAAAAAAAJALBAAAA|2452030|2452059|DEPARTMENT|65|9|Grounds should say just dramatic comments. Necessary regions |monthly| +6922|AAAAAAAAKALBAAAA|2452030|2452059|DEPARTMENT|65|10|Original photographs may contrast also guards. Always individual eyes make so vividly atomi|monthly| +6923|AAAAAAAALALBAAAA|2452030|2452059|DEPARTMENT|65|11|Sensitive members listen further; young interests progress probably simp|monthly| +6924|AAAAAAAAMALBAAAA|2452030|2452059|DEPARTMENT|65|12|American districts keep only strong, other changes. Also helpful countries sha|monthly| +6925|AAAAAAAANALBAAAA|2452030|2452059|DEPARTMENT|65|13|For instance general germans allow hardly spanish, accurate numbers. Proposals should writ|monthly| +6926|AAAAAAAAOALBAAAA|2452030|2452059|DEPARTMENT|65|14|Random perceptions direct generally about the years. Fina|monthly| +6927|AAAAAAAAPALBAAAA|2452030|2452059|DEPARTMENT|65|15|Sadly normal rights believe yet; travellers consider only characteristic|monthly| +6928|AAAAAAAAABLBAAAA|2452030|2452059|DEPARTMENT|65|16|Measures please central, very services; departments cannot rear almost private experiences; kind|monthly| +6929|AAAAAAAABBLBAAAA|2452030|2452059|DEPARTMENT|65|17|Tourists could not remember even eyes. Rather likely years declin|monthly| +6930|AAAAAAAACBLBAAAA|2452030|2452059|DEPARTMENT|65|18|Roman breasts preclude also essential databases. Dear problems improve|monthly| +6931|AAAAAAAADBLBAAAA|2452030|2452059|DEPARTMENT|65|19|Surprising changes meet almost tiny, natural lists. Clear thousands digest pe|monthly| +6932|AAAAAAAAEBLBAAAA|2452030|2452059|DEPARTMENT|65|20|Miles will come good managers. Temporary years could play|monthly| +6933|AAAAAAAAFBLBAAAA|2452030|2452059|DEPARTMENT|65|21|Over obvious hands will give carefully more living years. Closer significant re|monthly| +6934|AAAAAAAAGBLBAAAA|2452030|2452059|DEPARTMENT|65|22|Major guns repeat already against the mammals. Soon impressive flowers introduce all exci|monthly| +6935|AAAAAAAAHBLBAAAA|2452030|2452059|DEPARTMENT|65|23|Proper banks enjoy carefully by the men; men wish always white wome|monthly| +6936|AAAAAAAAIBLBAAAA|2452030|2452059|DEPARTMENT|65|24|Ordinary, common sports want forth essential citizens. Different parts gather societies; extra,|monthly| +6937|AAAAAAAAJBLBAAAA|2452030|2452059|DEPARTMENT|65|25|Narrow, young parts shall share notably quietly familiar police; children please lately for|monthly| +6938|AAAAAAAAKBLBAAAA|2452030|2452059|DEPARTMENT|65|26|Poor clients defend light, high rates. Different, future powers acknowledge then.|monthly| +6939|AAAAAAAALBLBAAAA|2452030|2452059|DEPARTMENT|65|27|Perhaps steep children might offer at no police. Front, other roads could n|monthly| +6940|AAAAAAAAMBLBAAAA|2452030|2452059|DEPARTMENT|65|28|Limited authorities raise from a services. Brilliant sources fly still blind proble|monthly| +6941|AAAAAAAANBLBAAAA|2452030|2452059|DEPARTMENT|65|29|Angry, front resources must come increasingly guilty, elected colours. Very availab|monthly| +6942|AAAAAAAAOBLBAAAA|2452030|2452059|DEPARTMENT|65|30|Troops bring below concerned chemicals; supporters ought to serve at first programmes; dangerou|monthly| +6943|AAAAAAAAPBLBAAAA|2452030|2452059|DEPARTMENT|65|31|Capital feelings matter yet; aloud statutory problems can like elegant, new bo|monthly| +6944|AAAAAAAAACLBAAAA|2452030|2452059|DEPARTMENT|65|32|English, narrow chips show however tall colleagues. Sharp scholars must catch others. About|monthly| +6945|AAAAAAAABCLBAAAA|2452030|2452059|DEPARTMENT|65|33|Quickly thinking figures shall hold most arrangements. Highly local reports may grow alw|monthly| +6946|AAAAAAAACCLBAAAA|2452030|2452059|DEPARTMENT|65|34|Yellow nations must let french surveys; leaves take. So front childre|monthly| +6947|AAAAAAAADCLBAAAA|2452030|2452059|DEPARTMENT|65|35|Critical, new securities ought to carry thereafter on a generations. |monthly| +6948|AAAAAAAAECLBAAAA|2452030|2452059|DEPARTMENT|65|36|Foreign thanks get both most evil duties. Once difficult structures would extend tired cit|monthly| +6949|AAAAAAAAFCLBAAAA|2452030|2452059|DEPARTMENT|65|37|Proposals could not open about a instructions. Open others get vag|monthly| +6950|AAAAAAAAGCLBAAAA|2452030|2452059|DEPARTMENT|65|38|Big, medical expenses might support. Hundreds would retain together inadequate signs. Bus|monthly| +6951|AAAAAAAAHCLBAAAA|2452030|2452059|DEPARTMENT|65|39|Around upper women accept quickly junior payments. Spiritual matters knock able parts. More good ye|monthly| +6952|AAAAAAAAICLBAAAA|2452030|2452059|DEPARTMENT|65|40|Aware eyes ought to save very indeed high individuals.|monthly| +6953|AAAAAAAAJCLBAAAA|2452030|2452059|DEPARTMENT|65|41|Home dangerous courses would say today as everyday students. Years used to put.|monthly| +6954|AAAAAAAAKCLBAAAA|2452030|2452059|DEPARTMENT|65|42|Really advanced orders should buy political governors. Skilled, tru|monthly| +6955|AAAAAAAALCLBAAAA|2452030|2452059|DEPARTMENT|65|43|Originally big sectors see well. British reports teach as; again co|monthly| +6956|AAAAAAAAMCLBAAAA|2452030|2452059|DEPARTMENT|65|44|Certainly fine seconds measure recently as british indiv|monthly| +6957|AAAAAAAANCLBAAAA|2452030|2452059|DEPARTMENT|65|45|Bottom, sufficient problems fall to the actions. Pictures should not imagine just nev|monthly| +6958|AAAAAAAAOCLBAAAA|2452030|2452059|DEPARTMENT|65|46|Immediately diplomatic numbers should help therefor|monthly| +6959|AAAAAAAAPCLBAAAA|2452030|2452059|DEPARTMENT|65|47|Flexible police regard very accounts. Then soft sides would abandon frankly similar, late d|monthly| +6960|AAAAAAAAADLBAAAA|2452030|2452059|DEPARTMENT|65|48|Changes can take young prospects. New, functional women bring much crucial, content|monthly| +6961|AAAAAAAABDLBAAAA|2452030|2452059|DEPARTMENT|65|49|Plants meet for instance parties; elsewhere critical differences think approximately|monthly| +6962|AAAAAAAACDLBAAAA|2452030|2452059|DEPARTMENT|65|50|More than sensible words favour previously. Busy regions|monthly| +6963|AAAAAAAADDLBAAAA|2452030|2452059|DEPARTMENT|65|51|Religious, important seasons should stop super books. Special, happy costs he|monthly| +6964|AAAAAAAAEDLBAAAA|2452030|2452059|DEPARTMENT|65|52|Evolutionary, religious goods may sustain even volumes. Recordings cover badly i|monthly| +6965|AAAAAAAAFDLBAAAA|2452030|2452059|DEPARTMENT|65|53|Probably quick words give pretty so strong activities. Rec|monthly| +6966|AAAAAAAAGDLBAAAA|2452030|2452059|DEPARTMENT|65|54|Dark pupils used to produce still young effects; processes appoint away still other roads. Remote|monthly| +6967|AAAAAAAAHDLBAAAA|2452030|2452059|DEPARTMENT|65|55|Systems used to give then future, other pupils. Countries appea|monthly| +6968|AAAAAAAAIDLBAAAA|2452030|2452059|DEPARTMENT|65|56|New sports spread beautifully major origins; american missiles wander high mem|monthly| +6969|AAAAAAAAJDLBAAAA|2452030|2452059|DEPARTMENT|65|57|Italian, ready years must worry. Probably severe methods may experience recent, new years; person|monthly| +6970|AAAAAAAAKDLBAAAA|2452030|2452059|DEPARTMENT|65|58|Scientific children might plant once hot years. Particularly briti|monthly| +6971|AAAAAAAALDLBAAAA|2452030|2452059|DEPARTMENT|65|59|Legally young instructions might not meet so as failures. Records tell |monthly| +6972|AAAAAAAAMDLBAAAA|2452030|2452059|DEPARTMENT|65|60|Partially likely dollars read all available, real levels. Please scientific agencies may liste|monthly| +6973|AAAAAAAANDLBAAAA|2452030|2452059|DEPARTMENT|65|61|Possible contacts disagree especially visual, remaining windows. Acts take i|monthly| +6974|AAAAAAAAODLBAAAA|2452030|2452059|DEPARTMENT|65|62|Distinct, small details can want evenly electoral, red democrat|monthly| +6975|AAAAAAAAPDLBAAAA|2452030|2452059|DEPARTMENT|65|63|Elements would sell now allies; welsh sources seem on a changes. Lo|monthly| +6976|AAAAAAAAAELBAAAA|2452030|2452059|DEPARTMENT|65|64|Also dependent sales must change soon modes; positive,|monthly| +6977|AAAAAAAABELBAAAA|2452030|2452059|DEPARTMENT|65|65|Administrative statements break ordinary points. Individual failures ought to rebuild cer|monthly| +6978|AAAAAAAACELBAAAA|2452030|2452059|DEPARTMENT|65|66|Always fun holders die notably demands. Right charts must remedy; obligations report casual polit|monthly| +6979|AAAAAAAADELBAAAA|2452030|2452059|DEPARTMENT|65|67|Inadequate, specific elections shall respond now neighbouring times. Feelings might pay large, o|monthly| +6980|AAAAAAAAEELBAAAA|2452030|2452059|DEPARTMENT|65|68|Specific metres revive presumably. So high notes would not see again exceptional friendly r|monthly| +6981|AAAAAAAAFELBAAAA||||65|69|Conditions should recall often enough short places; very open communities|monthly| +6982|AAAAAAAAGELBAAAA|2452030|2452059|DEPARTMENT|65|70|Only regular studies propose later; points used to mak|monthly| +6983|AAAAAAAAHELBAAAA|2452030|2452059|DEPARTMENT|65|71|Dangerous problems might not make well to a men. Royal families dust about hands. Boards prove ofte|monthly| +6984|AAAAAAAAIELBAAAA|2452030|2452059|DEPARTMENT|65|72|Early days should not slow complete, human notes. Students can prevent exclusively for |monthly| +6985|AAAAAAAAJELBAAAA|2452030|2452059|DEPARTMENT|65|73|Well cold men disappear hills. General, recent approaches pick there.|monthly| +6986|AAAAAAAAKELBAAAA|2452030|2452059|DEPARTMENT|65|74|Right, urban shoes become also essential packages. Genes shall pay much thanks. Spatial plat|monthly| +6987|AAAAAAAALELBAAAA|2452030|2452059|DEPARTMENT|65|75|Again spare customers provide on a odds. Gay, good questions control from a teachers; just valua|monthly| +6988|AAAAAAAAMELBAAAA|2452030|2452059|DEPARTMENT|65|76|Patiently successful purposes omit. Possible posts|monthly| +6989|AAAAAAAANELBAAAA|2452030|2452059|DEPARTMENT|65|77|Orders must see always in the fingers. Great, legal trend|monthly| +6990|AAAAAAAAOELBAAAA|2452030|2452059|DEPARTMENT|65|78|Candidates ask extremely active, physical ideas. Too major farmers eliminate|monthly| +6991|AAAAAAAAPELBAAAA|2452030|2452059|DEPARTMENT|65|79|Red, joint roses enforce as unable years. Again successful problems post fo|monthly| +6992|AAAAAAAAAFLBAAAA|2452030|2452059|DEPARTMENT|65|80|Other, empty positions say glad, normal measures. Arms ch|monthly| +6993|AAAAAAAABFLBAAAA|2452030|2452059|DEPARTMENT|65|81|Fixed, running elections must increase already women. Whole reasons could not ask so. Relative|monthly| +6994|AAAAAAAACFLBAAAA|2452030|2452059|DEPARTMENT|65|82|National, good times persuade german workers. Environmentally recent hands may perform ge|monthly| +6995|AAAAAAAADFLBAAAA|2452030|2452059|DEPARTMENT|65|83|Girls define real children; important, english tests coul|monthly| +6996|AAAAAAAAEFLBAAAA|2452030|2452059|DEPARTMENT|65|84|Attitudes would look much molecular doubts. Well narrow men may h|monthly| +6997|AAAAAAAAFFLBAAAA|2452030|2452059|DEPARTMENT|65|85|Fiscal doubts must not admit never christian, local workers. Forever full words m|monthly| +6998|AAAAAAAAGFLBAAAA|2452030|2452059|DEPARTMENT|65|86|Still adequate years can sign even bonds; previously base years re|monthly| +6999|AAAAAAAAHFLBAAAA|2452030|2452059|DEPARTMENT|65|87|Again brown emotions should prefer tired children. Moral, unable places will cl|monthly| +7000|AAAAAAAAIFLBAAAA|2452030|2452059|DEPARTMENT|65|88|Adults should believe favorite genes. Parliamentary, national sites admire overall weapons. |monthly| +7001|AAAAAAAAJFLBAAAA|2452030|2452059|DEPARTMENT|65|89|Girls locate later still current communications. National, futu|monthly| +7002|AAAAAAAAKFLBAAAA|2452030|2452059|DEPARTMENT|65|90|This intimate matters know however earnings. Strong spots must wish almost close pie|monthly| +7003|AAAAAAAALFLBAAAA|2452030|2452059|DEPARTMENT|65|91|Contemporary, necessary rooms cover out of a claims. Organic pr|monthly| +7004|AAAAAAAAMFLBAAAA|2452030|2452059|DEPARTMENT|65|92|Movements leave after a inches; good, unaware hands see once outdoor situations. Yet short min|monthly| +7005|AAAAAAAANFLBAAAA|2452030|2452059|DEPARTMENT|65|93|Most international authorities look merely in a enterprises. Events cannot improve always slow, |monthly| +7006|AAAAAAAAOFLBAAAA|2452030|2452059|DEPARTMENT|65|94|Able, true waves accompany sometimes present, different methods; social reports ask howev|monthly| +7007|AAAAAAAAPFLBAAAA|2452030|2452059|DEPARTMENT|65|95|Frequently sensitive circumstances wonder too wast|monthly| +7008|AAAAAAAAAGLBAAAA|2452030|2452059|DEPARTMENT|65|96|Members run major, special stages. Improvements ensure again also possibl|monthly| +7009|AAAAAAAABGLBAAAA|2452030|2452059|DEPARTMENT|65|97|Statements tell enormously successfully white conditions. Available, magne|monthly| +7010|AAAAAAAACGLBAAAA|2452030|2452059|DEPARTMENT|65|98|Sensitive documents ought to watch wrong very foreign girls. |monthly| +7011|AAAAAAAADGLBAAAA|2452030|2452059|DEPARTMENT|65|99|Miners require fiercely also monthly teams. Ministers will benefit either wo|monthly| +7012|AAAAAAAAEGLBAAAA|2452030|2452059|DEPARTMENT|65|100|Classes know low, unusual legs; soon surprising waters realise desperately |monthly| +7013|AAAAAAAAFGLBAAAA|2452030|2452059|DEPARTMENT|65|101|Only, western hours might work perhaps. Separate, good limits would create repeatedly.|monthly| +7014|AAAAAAAAGGLBAAAA|2452030|2452059|DEPARTMENT|65|102|Years could look administrative indians; female tons would go au|monthly| +7015|AAAAAAAAHGLBAAAA|2452030|2452059|DEPARTMENT|65|103|Too electronic sources used to capture hotels. Vast rumours help together sites. Jobs should no|monthly| +7016|AAAAAAAAIGLBAAAA|2452030|2452059|DEPARTMENT|65|104|Trying, only pensions must not utter just in a wal|monthly| +7017|AAAAAAAAJGLBAAAA|2452030|2452059|DEPARTMENT|65|105|As specific doors might take far. True, financial pictures would accept therefore single rounds. U|monthly| +7018|AAAAAAAAKGLBAAAA|2452030|2452059|DEPARTMENT|65|106|Other opportunities used to produce pleasant, whole principles. Cheap, professi|monthly| +7019|AAAAAAAALGLBAAAA|2452030|2452059|DEPARTMENT|65|107|Dutch, effective fingers must not make dry boys; much foreign things could belie|monthly| +7020|AAAAAAAAMGLBAAAA|2452030|2452059|DEPARTMENT|65|108|Important thousands might weigh casual, modern cases; dull, other lines used to w|monthly| +7021|AAAAAAAANGLBAAAA|2452060|2452089|DEPARTMENT|66|1|Eventually current fragments use. Statistical positions could not know. Scottis|monthly| +7022|AAAAAAAAOGLBAAAA|2452060|2452089|DEPARTMENT|66|2|Able forms might not make previously feet. Long questions shall adapt |monthly| +7023|AAAAAAAAPGLBAAAA|2452060|2452089|DEPARTMENT|66|3|National methods buy rooms. Diplomatic, long legs check. Girls revise even certa|monthly| +7024|AAAAAAAAAHLBAAAA|2452060|2452089|DEPARTMENT|66|4|Forces succeed far there bold bodies; true, new problems may put secondary|monthly| +7025|AAAAAAAABHLBAAAA|2452060|2452089|DEPARTMENT|66|5|Parallel new shares get thus considerations. Plants identify entirely newl|monthly| +7026|AAAAAAAACHLBAAAA|2452060|2452089|DEPARTMENT|66|6|Southern, right years must play only on a pupils; severe boys appr|monthly| +7027|AAAAAAAADHLBAAAA|2452060|2452089|DEPARTMENT|66|7|Remaining artists increase after a years. Great imports calculate far then financial |monthly| +7028|AAAAAAAAEHLBAAAA|2452060|2452089|DEPARTMENT|66|8|Traditional, able members knit more too considerable answers. All sound con|monthly| +7029|AAAAAAAAFHLBAAAA||2452089|||9|Theoretical locations want grand programmes. Fun employers write games; enormous letters will rais|| +7030|AAAAAAAAGHLBAAAA|2452060|2452089|DEPARTMENT|66|10|New indians allow gmt national stars. Again standard years wi|monthly| +7031|AAAAAAAAHHLBAAAA|2452060|2452089|DEPARTMENT|66|11|Senior women may not form. Extremely novel problems go; events find so o|monthly| +7032|AAAAAAAAIHLBAAAA|2452060|2452089|DEPARTMENT|66|12|Relations may appreciate; however conscious orders drive|monthly| +7033|AAAAAAAAJHLBAAAA|2452060|2452089|DEPARTMENT|66|13|Requests attract in a bits. Ports could offer bett|monthly| +7034|AAAAAAAAKHLBAAAA|2452060|2452089|DEPARTMENT|66|14|Methods search molecular resources. Forms should issue else well christian women. Feet might take |monthly| +7035|AAAAAAAALHLBAAAA|2452060|2452089|DEPARTMENT|66|15|Well interested servants must go whole areas; just suitable accidents put rational obligation|monthly| +7036|AAAAAAAAMHLBAAAA|2452060|||||Standard solicitors should cut in terms of a parties. Public, practical teachers move abo|| +7037|AAAAAAAANHLBAAAA|2452060|2452089|DEPARTMENT|66|17|Services shall not identify then. Representatives might learn more f|monthly| +7038|AAAAAAAAOHLBAAAA|2452060|2452089|DEPARTMENT|66|18|Since basic police get always very raw changes; local, french ways identify ever together intell|monthly| +7039|AAAAAAAAPHLBAAAA|2452060|2452089|DEPARTMENT|66|19|Resulting hours lead faintly legal angles. Levels should take gene|monthly| +7040|AAAAAAAAAILBAAAA|2452060|2452089|DEPARTMENT|66|20|Models start hence ago internal dealers. Frequently public police may no|monthly| +7041|AAAAAAAABILBAAAA|2452060|2452089|DEPARTMENT|66|21|Good techniques launch excellent prices. Other, different masters can see upward concrete, light l|monthly| +7042|AAAAAAAACILBAAAA|2452060|2452089|DEPARTMENT|66|22|Guilty farmers should leave northern, bad others. Good, central men will put more political compa|monthly| +7043|AAAAAAAADILBAAAA|2452060|2452089|DEPARTMENT|66|23|Local men evaluate late other concentrations. High others com|monthly| +7044|AAAAAAAAEILBAAAA|2452060|2452089|DEPARTMENT|66|24|Daily jeans might know exclusively in a groups. Decisions|monthly| +7045|AAAAAAAAFILBAAAA|2452060|2452089|DEPARTMENT|66|25|In particular poor days must state controversial sums. Games wo|monthly| +7046|AAAAAAAAGILBAAAA|2452060|2452089|DEPARTMENT|66|26|Frequently irish steps avoid however powers. Direct, economic w|monthly| +7047|AAAAAAAAHILBAAAA|2452060|2452089|DEPARTMENT|66|27|Open studies restore only attitudes. Other pieces switch |monthly| +7048|AAAAAAAAIILBAAAA|2452060|2452089|DEPARTMENT|66|28|Approximately social words consult almost public, basic eyes; yellow researchers collect also supe|monthly| +7049|AAAAAAAAJILBAAAA|2452060|2452089|DEPARTMENT|66|29|Years keep mainly as usual long dogs. Connections come high developers; clever|monthly| +7050|AAAAAAAAKILBAAAA|2452060|2452089|DEPARTMENT|66|30|Citizens change. Military materials use pupils. Regularly cons|monthly| +7051|AAAAAAAALILBAAAA|2452060|2452089|DEPARTMENT|66|31|Courses could bother never there real women. Foreign, various years co|monthly| +7052|AAAAAAAAMILBAAAA|2452060|2452089|DEPARTMENT|66|32|Due, positive issues receive proposed directions. Past ashamed wars tell |monthly| +7053|AAAAAAAANILBAAAA|2452060|2452089|DEPARTMENT|66|33|British organizations make about fresh orders. Spectacular, usefu|monthly| +7054|AAAAAAAAOILBAAAA|2452060|2452089|DEPARTMENT|66|34|Scenes could not support as for a circumstances. Also extended americans mu|monthly| +7055|AAAAAAAAPILBAAAA|2452060|2452089|DEPARTMENT|66|35|Positive, high machines retire most natural values; ultimate players exploit old, large|monthly| +7056|AAAAAAAAAJLBAAAA|2452060|2452089|DEPARTMENT|66|36|Less stable rocks can contribute. Otherwise critical months stay y|monthly| +7057|AAAAAAAABJLBAAAA|2452060|2452089|DEPARTMENT|66|37|Conservative, medical shoulders get all at a questions. Hu|monthly| +7058|AAAAAAAACJLBAAAA|2452060|2452089|DEPARTMENT|66|38|Only, organisational miles will need comprehensive times. Spots shall not protest deliberately ever|monthly| +7059|AAAAAAAADJLBAAAA|2452060|2452089|DEPARTMENT|66|39|Words should make close. Women manage able maps; often regular limitations sta|monthly| +7060|AAAAAAAAEJLBAAAA|2452060|2452089|DEPARTMENT|66|40|Large groups can exercise above small, initial levels. Minutes can look so-called fingers; consist|monthly| +7061|AAAAAAAAFJLBAAAA|2452060|2452089|DEPARTMENT|66|41|Fit pressures meet main cards. Human, moral profess|monthly| +7062|AAAAAAAAGJLBAAAA|2452060|2452089|DEPARTMENT|66|42|Brief, productive conservatives want large, early exceptions. Her|monthly| +7063|AAAAAAAAHJLBAAAA|2452060|2452089|DEPARTMENT|66|43|Russian, occasional cases confront further bright, recent problems; tall readers would learn quic|monthly| +7064|AAAAAAAAIJLBAAAA|2452060|2452089|DEPARTMENT|66|44|Other, very regions win over a tracks. Rich levels shall live for an heads. Lively school|monthly| +7065|AAAAAAAAJJLBAAAA|2452060|2452089|DEPARTMENT|66|45|Happy, other payments could solve now sheer, social relations. New, general animals may |monthly| +7066|AAAAAAAAKJLBAAAA|2452060|2452089|DEPARTMENT|66|46|Likely, national owners learn more monthly, identical forces. Organic boats remain also never lik|monthly| +7067|AAAAAAAALJLBAAAA|2452060|2452089|DEPARTMENT|66|47|Other, religious rates ought to say often eyes. Interesting ha|monthly| +7068|AAAAAAAAMJLBAAAA|2452060|2452089|DEPARTMENT|66|48|Collections require always significant minutes. Activities|monthly| +7069|AAAAAAAANJLBAAAA|2452060|2452089|DEPARTMENT|66|49|Just traditional chips transfer new, western miles. Characteristics back similarly stiffly ugly |monthly| +7070|AAAAAAAAOJLBAAAA|2452060|2452089|DEPARTMENT|66|50|Formal feet mark funds; thereby important women will not wish just n|monthly| +7071|AAAAAAAAPJLBAAAA|2452060|2452089|DEPARTMENT|66|51|Liberal arrangements may come near. Poor, labour weeks cannot talk sho|monthly| +7072|AAAAAAAAAKLBAAAA|2452060|2452089|DEPARTMENT|66|52|Possible, empty signals ought to buy everyday, political clothes. Possi|monthly| +7073|AAAAAAAABKLBAAAA|2452060|2452089|DEPARTMENT|66|53|Integrated, dead rocks used to consider often clear, new candidates; childre|monthly| +7074|AAAAAAAACKLBAAAA|2452060|2452089|DEPARTMENT|66|54|Cases might lead benefits. Further other days may become too long wide centuries. Comfortab|monthly| +7075|AAAAAAAADKLBAAAA|2452060|2452089|DEPARTMENT|66|55|Private women play. As following expectations know a|monthly| +7076|AAAAAAAAEKLBAAAA|2452060|2452089|DEPARTMENT|66|56|Important, alone cards keep until a objects. Names shall use to an pat|monthly| +7077|AAAAAAAAFKLBAAAA|2452060|2452089|DEPARTMENT|66|57|Politicians raise in a officers. Of course scottish circumstances fight unwitti|monthly| +7078|AAAAAAAAGKLBAAAA|2452060|2452089|DEPARTMENT|66|58|Well inc powers create only. Names avail other, na|monthly| +7079|AAAAAAAAHKLBAAAA|2452060|2452089|DEPARTMENT|66|59|Sides buy content others. Just high minutes end as current reasons. Opposite, close incentives die|monthly| +7080|AAAAAAAAIKLBAAAA|2452060|2452089|DEPARTMENT|66|60|Mistakes date major, complicated holidays. Maybe existing partners purchase n|monthly| +7081|AAAAAAAAJKLBAAAA|2452060|2452089|DEPARTMENT|66|61|Fields could not control additional divisions; conditions mus|monthly| +7082|AAAAAAAAKKLBAAAA|2452060|2452089|DEPARTMENT|66|62|Political, urban reservations should not bring very towards a principles. Shots will consider also |monthly| +7083|AAAAAAAALKLBAAAA|2452060|2452089|DEPARTMENT|66|63|Here early restrictions push badly; regions go further.|monthly| +7084|AAAAAAAAMKLBAAAA|2452060|2452089|DEPARTMENT|66|64|Sports feel now assumptions; more traditional women know less quickly complete members. Then cen|monthly| +7085|AAAAAAAANKLBAAAA|2452060|2452089|DEPARTMENT|66|65|As available novels use that is permanent, rapid authors; hard, vulnerable figures overcom|monthly| +7086|AAAAAAAAOKLBAAAA|2452060|2452089|DEPARTMENT|66|66|Products will establish always by a views. Either revolutionary policies f|monthly| +7087|AAAAAAAAPKLBAAAA|2452060|2452089|DEPARTMENT|66|67|Also new projects analyse as long, usual times. Other, local representatives examine. Also chemi|monthly| +7088|AAAAAAAAALLBAAAA|2452060|2452089|DEPARTMENT|66|68|Economic samples used to lie always fat employers. Soft, essential arms f|monthly| +7089|AAAAAAAABLLBAAAA|2452060|2452089|DEPARTMENT|66|69|Systems can sell up the years; powerful, pretty concepts could leave clothes. Alw|monthly| +7090|AAAAAAAACLLBAAAA|2452060|2452089|DEPARTMENT|66|70|Magnificent women might return in a goods. Arrangements might not need international windows. Offic|monthly| +7091|AAAAAAAADLLBAAAA|2452060|2452089|DEPARTMENT|66|71|Examples could not hold national, economic readers; calculations build able, individual children. M|monthly| +7092|AAAAAAAAELLBAAAA|2452060|2452089|DEPARTMENT|66|72|Always emotional banks may not prepare now interesting, linguistic fathers. Present levels|monthly| +7093|AAAAAAAAFLLBAAAA|2452060|2452089|DEPARTMENT|66|73|Vast charges follow pretty long courts. Regularly practical holes must not tak|monthly| +7094|AAAAAAAAGLLBAAAA|2452060|2452089|DEPARTMENT|66|74|Changes might not get bright, poor patients. Users used to like th|monthly| +7095|AAAAAAAAHLLBAAAA|2452060|2452089|DEPARTMENT|66|75|Domestic accidents cannot move labour genes. Northern|monthly| +7096|AAAAAAAAILLBAAAA|2452060|2452089|DEPARTMENT|66|76|Blocks take completely historical fingers. Clear countries must suggest wi|monthly| +7097|AAAAAAAAJLLBAAAA|2452060|2452089|DEPARTMENT|66|77|Likely, clean cases would make just waiting beaches. Rel|monthly| +7098|AAAAAAAAKLLBAAAA|2452060|2452089|DEPARTMENT|66|78|Also technical points might not hear usually rigid, other quantiti|monthly| +7099|AAAAAAAALLLBAAAA|2452060|2452089|DEPARTMENT|66|79|As formal theories ought to permit national events. Early, permanent answers wou|monthly| +7100|AAAAAAAAMLLBAAAA|2452060|2452089|DEPARTMENT|66|80|Also serious areas create both details. Legal limits take. Simply big fingers may dr|monthly| +7101|AAAAAAAANLLBAAAA|2452060|2452089|DEPARTMENT|66|81|True, social offences undertake practitioners; mad accounts|monthly| +7102|AAAAAAAAOLLBAAAA|2452060|2452089|DEPARTMENT|66|82|Years ought to return in the cases. Items would not imitate e|monthly| +7103|AAAAAAAAPLLBAAAA|2452060|2452089|DEPARTMENT|66|83|Lovingly broad yards go even financial weeks; things accommodate |monthly| +7104|AAAAAAAAAMLBAAAA|2452060|2452089|DEPARTMENT|66|84|Courses determine highly conditions; just human respondents f|monthly| +7105|AAAAAAAABMLBAAAA|2452060|2452089|DEPARTMENT|66|85|Just safe pupils should not get now; national, premier dealers lose strongly goals.|monthly| +7106|AAAAAAAACMLBAAAA|2452060|2452089|DEPARTMENT|66|86|Active farms can know men. Businesses ought to fit late with a rela|monthly| +7107|AAAAAAAADMLBAAAA|2452060|2452089|DEPARTMENT|66|87|About average workers could spend largely. Military properties require teachers; welsh par|monthly| +7108|AAAAAAAAEMLBAAAA|2452060|2452089|DEPARTMENT|66|88|Members must not find with a signals. Fierce, new relations might not worr|monthly| +7109|AAAAAAAAFMLBAAAA|2452060|2452089|DEPARTMENT|66|89|Overall only pairs ought to relate silent facilities. Goals will contain on|monthly| +7110|AAAAAAAAGMLBAAAA|2452060|2452089|DEPARTMENT|66|90|Windows provide here formal experiences. Similar, possible questions fail for good. Fellow diffic|monthly| +7111|AAAAAAAAHMLBAAAA|2452060|2452089|DEPARTMENT|66|91|Different, beautiful weeks could press problems. Indeed active acts blame in a generations. Eye|monthly| +7112|AAAAAAAAIMLBAAAA|2452060|2452089|DEPARTMENT|66|92|As great cars should take very low significant days; great, internatio|monthly| +7113|AAAAAAAAJMLBAAAA|2452060|2452089|DEPARTMENT|66|93|Men must represent eyes. As ordinary requests bring on a ey|monthly| +7114|AAAAAAAAKMLBAAAA|2452060|2452089|DEPARTMENT|66|94|Seldom single organisations know more references. High careers used to find only bo|monthly| +7115|AAAAAAAALMLBAAAA|2452060|2452089|DEPARTMENT|66|95|Today only proposals occur abroad words. Long-term busines|monthly| +7116|AAAAAAAAMMLBAAAA|2452060|2452089|DEPARTMENT|66|96|Political years used to dream citizens. Too upper results shall not |monthly| +7117|AAAAAAAANMLBAAAA|2452060|2452089|DEPARTMENT|66|97|Personal miles wipe so-called changes. Indeed urban neighbours cure. Papers used to see. M|monthly| +7118|AAAAAAAAOMLBAAAA|2452060|2452089|DEPARTMENT|66|98|Open, running services understand police. Of course big month|monthly| +7119|AAAAAAAAPMLBAAAA|2452060|2452089|DEPARTMENT|66|99|Terrible, important sentences represent alone girls. Over chief procedures spend |monthly| +7120|AAAAAAAAANLBAAAA|2452060|2452089|DEPARTMENT|66|100|Days must go smartly capable, clear eyes. Generally german types could arouse therefore true, s|monthly| +7121|AAAAAAAABNLBAAAA|2452060|2452089|DEPARTMENT|66|101|Days opt difficult, worried casualties. Useful provinces brin|monthly| +7122|AAAAAAAACNLBAAAA||2452089||66|102||monthly| +7123|AAAAAAAADNLBAAAA|2452060|2452089|DEPARTMENT|66|103|Days should act respectively pretty lines. Afraid words must make men. So britis|monthly| +7124|AAAAAAAAENLBAAAA|2452060|2452089|DEPARTMENT|66|104|Uncomfortable, responsible teeth can draw alone with|monthly| +7125|AAAAAAAAFNLBAAAA|2452060|2452089|DEPARTMENT|66|105|Increased, local scots decide double, established troops. Unusual,|monthly| +7126|AAAAAAAAGNLBAAAA|2452060|2452089|DEPARTMENT|66|106|Major, european users ought to release only recent faces; sole, expensive |monthly| +7127|AAAAAAAAHNLBAAAA|2452060|2452089|DEPARTMENT|66|107|In part other materials see publicly. Very economic pr|monthly| +7128|AAAAAAAAINLBAAAA|2452060|2452089|DEPARTMENT|66|108|Desperate, wide issues tolerate neither as a metres. Nati|monthly| +7129|AAAAAAAAJNLBAAAA|2452090|2452119|DEPARTMENT|67|1|Unknown types must not leave suddenly dependent colours. Experts control still to a |monthly| +7130|AAAAAAAAKNLBAAAA|2452090|2452119|DEPARTMENT|67|2|Points generalise into the paths. Scottish, senior steps love howe|monthly| +7131|AAAAAAAALNLBAAAA|2452090|2452119|DEPARTMENT|67|3|Fairly recent stairs should not use cleverly for a miles. Often re|monthly| +7132|AAAAAAAAMNLBAAAA|2452090|2452119|DEPARTMENT|67|4|Rich, potential trustees accept for instance like a researchers; units set already outside|monthly| +7133|AAAAAAAANNLBAAAA|2452090|2452119|DEPARTMENT|67|5|Other, alone locations perform early from a members. Careful, fi|monthly| +7134|AAAAAAAAONLBAAAA|2452090|2452119|DEPARTMENT|67|6|Relations ought to impose colours. Available, important women |monthly| +7135|AAAAAAAAPNLBAAAA|2452090|2452119|DEPARTMENT|67|7|Manufacturing drivers know so. Permanently necessary things wo|monthly| +7136|AAAAAAAAAOLBAAAA|2452090|2452119|DEPARTMENT|67|8|Impressive applications catch easier just royal workers. Great, wide institutions tread |monthly| +7137|AAAAAAAABOLBAAAA||||67|9||monthly| +7138|AAAAAAAACOLBAAAA|2452090|2452119|DEPARTMENT|67|10|General types provide all birds. Reasons maximise among a acids; points reach therefore; rather cl|monthly| +7139|AAAAAAAADOLBAAAA|2452090|2452119|DEPARTMENT|67|11|Fundamentally social beaches know often. Unlikely letters arise fingers; societies shall like eve|monthly| +7140|AAAAAAAAEOLBAAAA|2452090|2452119|DEPARTMENT|67|12|Now financial holes preserve. Soon significant months must g|monthly| +7141|AAAAAAAAFOLBAAAA|2452090|2452119|DEPARTMENT|67|13|Dutch, principal others might not specify very ago n|monthly| +7142|AAAAAAAAGOLBAAAA|2452090|2452119|DEPARTMENT|67|14|Now wide drugs shall see full, incredible schemes. Odd orders |monthly| +7143|AAAAAAAAHOLBAAAA|2452090|2452119|DEPARTMENT|67|15|Lines might bring also houses. Problems would get variou|monthly| +7144|AAAAAAAAIOLBAAAA|2452090|2452119|DEPARTMENT|67|16|Reports license much very white problems. Patients can create|monthly| +7145|AAAAAAAAJOLBAAAA|2452090|2452119|DEPARTMENT|67|17|Political, new politicians may make free daughters. Good,|monthly| +7146|AAAAAAAAKOLBAAAA|2452090|2452119|DEPARTMENT|67|18|Wonderful, swiss characters can look. Probably extra|monthly| +7147|AAAAAAAALOLBAAAA|2452090|2452119|DEPARTMENT|67|19|Relations shall contain secrets. Also full muscles cater rapid, m|monthly| +7148|AAAAAAAAMOLBAAAA|2452090|2452119|DEPARTMENT|67|20|Lucky, national policies may pass even above harsh lines|monthly| +7149|AAAAAAAANOLBAAAA|2452090|2452119|DEPARTMENT|67|21|General, able others will feature by the breasts. Individual, middle opportunities put arou|monthly| +7150|AAAAAAAAOOLBAAAA|2452090|2452119|DEPARTMENT|67|22|Things achieve even big, royal findings. As local consequences should cross satisfactory reference|monthly| +7151|AAAAAAAAPOLBAAAA|2452090|2452119|DEPARTMENT|67|23|Times must lead now only effects. Italian cases must not take increasi|monthly| +7152|AAAAAAAAAPLBAAAA|2452090|2452119|DEPARTMENT|67|24|Yet old arguments produce there; positive actions follow. Scientific, industrial principles may m|monthly| +7153|AAAAAAAABPLBAAAA|2452090|2452119|DEPARTMENT|67|25|Uniquely medical planes used to locate. Problems would apply qualities. Light poin|monthly| +7154|AAAAAAAACPLBAAAA|2452090|2452119|DEPARTMENT|67|26|Bad, unchanged months ought to settle assistant, golden service|monthly| +7155|AAAAAAAADPLBAAAA|2452090|2452119|DEPARTMENT|67|27|Changes listen so. Here happy tests would not write|monthly| +7156|AAAAAAAAEPLBAAAA|2452090|2452119|DEPARTMENT|67|28|Too bad pressures used to sit a little generally particular methods. Rapid development|monthly| +7157|AAAAAAAAFPLBAAAA|2452090|2452119|DEPARTMENT|67|29|Essential, moral months may not pick once other, open names. Programmes must b|monthly| +7158|AAAAAAAAGPLBAAAA|2452090|2452119|DEPARTMENT|67|30|Statements elucidate well on the shops. Coming, li|monthly| +7159|AAAAAAAAHPLBAAAA|2452090|2452119|DEPARTMENT|67|31|Mainly late customers may want a little. Scenes might |monthly| +7160|AAAAAAAAIPLBAAAA|2452090|2452119|DEPARTMENT|67|32|Areas should not predict to a discussions. Minor, other women |monthly| +7161|AAAAAAAAJPLBAAAA|2452090|2452119|DEPARTMENT|67|33|Places might contract there rich, pure actions. Natural, elegant managers should know no l|monthly| +7162|AAAAAAAAKPLBAAAA|2452090|2452119|DEPARTMENT|67|34|New needs come so real children; political, interesti|monthly| +7163|AAAAAAAALPLBAAAA|2452090|2452119|DEPARTMENT|67|35|As european stages swim heads; old, theoretical ti|monthly| +7164|AAAAAAAAMPLBAAAA||2452119||67|36|High variations will pose. Too other rates might not bear probably poor indian games. Wa|| +7165|AAAAAAAANPLBAAAA|2452090|2452119|DEPARTMENT|67|37|Extremely hard studies must talk. Years would not want medical cases; a|monthly| +7166|AAAAAAAAOPLBAAAA|2452090|2452119|DEPARTMENT|67|38|Estates could not feel burning units; simple rules like so carefully british w|monthly| +7167|AAAAAAAAPPLBAAAA|2452090|2452119|DEPARTMENT|67|39|Different, spanish hospitals might not get rather ideas. |monthly| +7168|AAAAAAAAAAMBAAAA|2452090|2452119|DEPARTMENT|67|40|More other holders scramble in a relations. Impossible years find ages. Grateful, classi|monthly| +7169|AAAAAAAABAMBAAAA|2452090|2452119|DEPARTMENT|67|41|Whole limitations must manage to a forms. Progressively residential groups stay again expensive s|monthly| +7170|AAAAAAAACAMBAAAA|2452090|2452119|DEPARTMENT|67|42|Inc, mad speakers ought to show much; fully fine exp|monthly| +7171|AAAAAAAADAMBAAAA|2452090|2452119|DEPARTMENT|67|43|Real groups will not encourage now now civil readers. Able samples assess possibly true|monthly| +7172|AAAAAAAAEAMBAAAA|2452090|2452119|DEPARTMENT|67|44|Urgent forces solve scientific, huge studies. Growing, evil decisions ask directly intern|monthly| +7173|AAAAAAAAFAMBAAAA|2452090|2452119|DEPARTMENT|67|45|Top effects will seem often particularly clear areas. Prime, eastern months ring by a |monthly| +7174|AAAAAAAAGAMBAAAA|2452090|2452119|DEPARTMENT|67|46|Damp, happy parties will not become best concrete features; articles could make questions.|monthly| +7175|AAAAAAAAHAMBAAAA|2452090|2452119|DEPARTMENT|67|47|Also central daughters used to knock also other, other|monthly| +7176|AAAAAAAAIAMBAAAA|2452090|2452119|DEPARTMENT|67|48|A little right times will not surrender currently. D|monthly| +7177|AAAAAAAAJAMBAAAA|2452090|2452119|DEPARTMENT|67|49|By now back boys adjust as perhaps new parts; present, high steps would face suddenly techn|monthly| +7178|AAAAAAAAKAMBAAAA|2452090|2452119|DEPARTMENT|67|50|Friendly, long children ask. Only criminal walls en|monthly| +7179|AAAAAAAALAMBAAAA|2452090|2452119|DEPARTMENT|67|51|Crude, french clouds may not exchange. Arbitrary, average eyes work. Very abstract members used to |monthly| +7180|AAAAAAAAMAMBAAAA|2452090|2452119|DEPARTMENT|67|52|Properties should become too completely glad governments. Preferably arbitrary countries increa|monthly| +7181|AAAAAAAANAMBAAAA|2452090|2452119|DEPARTMENT|67|53|Gardens cannot support; small vehicles persist both right, different females; even shar|monthly| +7182|AAAAAAAAOAMBAAAA|2452090|2452119|DEPARTMENT|67|54|Popular members shall meet. Light, unacceptable incidents develop. Especially loc|monthly| +7183|AAAAAAAAPAMBAAAA|2452090|2452119|DEPARTMENT|67|55|Historical, responsible others want there new children. Basic things f|monthly| +7184|AAAAAAAAABMBAAAA|2452090|2452119|DEPARTMENT|67|56|Also contemporary areas compensate constantly short, old te|monthly| +7185|AAAAAAAABBMBAAAA|2452090|2452119|DEPARTMENT|67|57|Able prices shall feel already industrial, political women. New schools might begin even bott|monthly| +7186|AAAAAAAACBMBAAAA|2452090|2452119|DEPARTMENT|67|58|Companies contribute more local women. Years take together methods. External departments shou|monthly| +7187|AAAAAAAADBMBAAAA|2452090|2452119|DEPARTMENT|67|59|More parliamentary aspects operate now economic, foreign schools. Thi|monthly| +7188|AAAAAAAAEBMBAAAA|2452090|2452119|DEPARTMENT|67|60|Certain times point; good, economic hours upset brief years. Features matt|monthly| +7189|AAAAAAAAFBMBAAAA|2452090|2452119|DEPARTMENT|67|61|At last clinical blocks might not give significantly available, grey honours. Dif|monthly| +7190|AAAAAAAAGBMBAAAA|2452090|2452119|DEPARTMENT|67|62|Students may seem deep, previous forms. Major, fresh allowances control. Traditional loans can sa|monthly| +7191|AAAAAAAAHBMBAAAA|2452090|2452119|DEPARTMENT|67|63|Flowers break immediately small chairs. Peacefully different others use often only|monthly| +7192|AAAAAAAAIBMBAAAA|2452090|2452119|DEPARTMENT|67|64|Principal figures present about alone sides. Comprehensive types su|monthly| +7193|AAAAAAAAJBMBAAAA|2452090|2452119|DEPARTMENT|67|65|Huge patients will not provide. Available, social presents come |monthly| +7194|AAAAAAAAKBMBAAAA|2452090|2452119|DEPARTMENT|67|66|Others shall hear light, english rules. Considerably political injuries sha|monthly| +7195|AAAAAAAALBMBAAAA|2452090|2452119|DEPARTMENT|67|67|Situations must speak in a areas. National copies rot with a courts; already labour parents keep b|monthly| +7196|AAAAAAAAMBMBAAAA|2452090|2452119|DEPARTMENT|67|68|Male cars used to get too in comparison with a tears. Major w|monthly| +7197|AAAAAAAANBMBAAAA|2452090|2452119|DEPARTMENT|67|69|Fully other members save probably islamic developments. Great, minute groups tend either ver|monthly| +7198|AAAAAAAAOBMBAAAA|2452090|2452119|DEPARTMENT|67|70|Required forces affect better red, irish forms. Black courts mig|monthly| +7199|AAAAAAAAPBMBAAAA|2452090|2452119|DEPARTMENT|67|71|Asleep banks come just therefore sure prices. Managers might not understand nearly accused s|monthly| +7200|AAAAAAAAACMBAAAA|2452090|2452119|DEPARTMENT|67|72|Thanks find just. Articles may try. Historic purposes see; public, natural mode|monthly| +7201|AAAAAAAABCMBAAAA|2452090|2452119|DEPARTMENT|67|73|Alone, new crowds determine elsewhere both loose duties; surveys affect also |monthly| +7202|AAAAAAAACCMBAAAA|2452090|2452119|DEPARTMENT|67|74|Rivals declare social eggs. Again present groups may accept other, recent de|monthly| +7203|AAAAAAAADCMBAAAA|2452090|2452119|DEPARTMENT|67|75|Pilots could not invade as surprising, similar years. Expensive ot|monthly| +7204|AAAAAAAAECMBAAAA|2452090|2452119|DEPARTMENT|67|76|Ministers finish perfectly annual horses. Generally social components might identify altogethe|monthly| +7205|AAAAAAAAFCMBAAAA|2452090|2452119|DEPARTMENT|67|77|Necessary points should follow related pp.. External homes want just fina|monthly| +7206|AAAAAAAAGCMBAAAA|2452090|2452119|DEPARTMENT|67|78|Opportunities must not restore operations. Raw, mediterranean thousands know quite on|monthly| +7207|AAAAAAAAHCMBAAAA|2452090|2452119|DEPARTMENT|67|79|Here other ideas hold boys. Reliable artists would mobilize also. I|monthly| +7208|AAAAAAAAICMBAAAA|2452090|2452119|DEPARTMENT|67|80|Hungry prices shall reduce too small children; also proper towns shall|monthly| +7209|AAAAAAAAJCMBAAAA|2452090|2452119|DEPARTMENT|67|81|Letters would achieve so agricultural studies. Friends will seek essentially includin|monthly| +7210|AAAAAAAAKCMBAAAA|2452090|2452119|DEPARTMENT|67|82|Certain employees cannot hope even as a children; public members introduce great years. Gu|monthly| +7211|AAAAAAAALCMBAAAA|2452090|2452119|DEPARTMENT|67|83|Slightly great cases would open procedures; personal terms would not mention |monthly| +7212|AAAAAAAAMCMBAAAA|2452090|2452119|DEPARTMENT|67|84|Local children would not doubt. Alike provincial nights shall |monthly| +7213|AAAAAAAANCMBAAAA|2452090|2452119|DEPARTMENT|67|85|Various, dramatic statements can seem more than horses. So high accounts |monthly| +7214|AAAAAAAAOCMBAAAA|2452090|2452119|DEPARTMENT|67|86|Various, financial experts wait large faces. Exciting, political candidates send so generous,|monthly| +7215|AAAAAAAAPCMBAAAA|2452090|2452119|DEPARTMENT|67|87|Awful amounts imagine. Sales could not plan around a issues; well adverse posts acknowledge res|monthly| +7216|AAAAAAAAADMBAAAA|2452090|2452119|DEPARTMENT|67|88|Actually pleasant measures work. Forms used to know across the forces. Able creditor|monthly| +7217|AAAAAAAABDMBAAAA|2452090|2452119|DEPARTMENT|67|89|Determined, soviet men could change well sorry, different feet. Interests see thus social|monthly| +7218|AAAAAAAACDMBAAAA|2452090|2452119|DEPARTMENT|67|90|Of course alone problems comment usually little damages. Barely|monthly| +7219|AAAAAAAADDMBAAAA|2452090|2452119|DEPARTMENT|67|91|Light, golden hands can cause. Observations would not deal states. Social, soft relations se|monthly| +7220|AAAAAAAAEDMBAAAA|2452090|2452119|DEPARTMENT|67|92|Social, advanced students stop foreign, social factors. Months fight well floors. Diff|monthly| +7221|AAAAAAAAFDMBAAAA|2452090|2452119|DEPARTMENT|67|93|Members ignore. Likely, extreme men may share also metropolitan terms. Now great paren|monthly| +7222|AAAAAAAAGDMBAAAA|2452090|2452119|DEPARTMENT|67|94|Low real lips will enjoy across then exciting clubs. Unable sessions ought to purchase on|monthly| +7223|AAAAAAAAHDMBAAAA|2452090|2452119|DEPARTMENT|67|95|Unexpectedly historical employers test only important profi|monthly| +7224|AAAAAAAAIDMBAAAA|2452090|2452119|DEPARTMENT|67|96|Passengers should not remain only ends. Social objects happ|monthly| +7225|AAAAAAAAJDMBAAAA|2452090|2452119|DEPARTMENT|67|97|Women take for a seconds. Years come types. Possible, scottish terms come. Local,|monthly| +7226|AAAAAAAAKDMBAAAA|2452090|2452119|DEPARTMENT|67|98|Even proposed elections decide opportunities. Now real owners must fall angrily at a group|monthly| +7227|AAAAAAAALDMBAAAA|2452090|2452119|DEPARTMENT|67|99|Words can wait probably before no minutes. Legal, detailed am|monthly| +7228|AAAAAAAAMDMBAAAA|2452090|2452119|DEPARTMENT|67|100|Other, female things should warm polls. Primary, good equat|monthly| +7229|AAAAAAAANDMBAAAA|2452090|2452119|DEPARTMENT|67|101|Dark, aware hours must make economic circumstances. Psychiatric princ|monthly| +7230|AAAAAAAAODMBAAAA|2452090|2452119|DEPARTMENT|67|102|Chemical, important estimates detect researchers. Men must ask ago deliberate|monthly| +7231|AAAAAAAAPDMBAAAA|2452090|2452119|DEPARTMENT|67|103|Miles bother almost children. Agencies must get. Very automatic bills|monthly| +7232|AAAAAAAAAEMBAAAA|2452090|2452119|DEPARTMENT|67|104|Tensions might not love really. Attractive terms like workshops|monthly| +7233|AAAAAAAABEMBAAAA|2452090|2452119|DEPARTMENT|67|105|Of course internal pieces cannot like today systematic single p|monthly| +7234|AAAAAAAACEMBAAAA|2452090|2452119|DEPARTMENT|67|106|Various contents cover therefore possible, top rules. Types criticise |monthly| +7235|AAAAAAAADEMBAAAA|2452090|2452119|DEPARTMENT|67|107|Inland, outstanding years used to form at least shared arrangements. Pl|monthly| +7236|AAAAAAAAEEMBAAAA|2452090|2452119|DEPARTMENT|67|108|Proportions need; large teeth spend sideways simpl|monthly| +7237|AAAAAAAAFEMBAAAA|2452120|2452149|DEPARTMENT|68|1|Categories contain. Resources used to prove always strong risks. Sensible doors ought |monthly| +7238|AAAAAAAAGEMBAAAA|2452120|2452149|DEPARTMENT|68|2|Long native steps see for an children. Enough difficult men might function unreasonably from a frie|monthly| +7239|AAAAAAAAHEMBAAAA|2452120|2452149|DEPARTMENT|68|3|White questions could attend quite long wide glasses. Fair, careful barriers could not deliver m|monthly| +7240|AAAAAAAAIEMBAAAA|2452120|2452149|DEPARTMENT|68|4|Separate, single words must recommend songs. Just external prospects finance complete, local|monthly| +7241|AAAAAAAAJEMBAAAA|2452120|2452149|DEPARTMENT|68|5|Less rapid boys utilise via a legs. Intact, existing expectations|monthly| +7242|AAAAAAAAKEMBAAAA|2452120|2452149|DEPARTMENT|68|6|Good paths used to think like a schools. Initial, serious strangers woul|monthly| +7243|AAAAAAAALEMBAAAA|2452120|2452149|DEPARTMENT|68|7|Great, good examinations shall feed almost basic consumers. Econ|monthly| +7244|AAAAAAAAMEMBAAAA|2452120|2452149|DEPARTMENT|68|8|New blacks may not settle even. German, russian objec|monthly| +7245|AAAAAAAANEMBAAAA|2452120|2452149|DEPARTMENT|68|9|Able, only women could not give at all active floors. Particular writers lead home new |monthly| +7246|AAAAAAAAOEMBAAAA|2452120|2452149|DEPARTMENT|68|10|Royal changes will take hardly important, inevitable terms. Real societies accuse|monthly| +7247|AAAAAAAAPEMBAAAA|2452120|2452149|DEPARTMENT|68|11|Other, political members could not carry in a standards; able christians could maintain for the mo|monthly| +7248|AAAAAAAAAFMBAAAA|2452120|2452149|DEPARTMENT|68|12|Chronic, democratic months take however environmental childr|monthly| +7249|AAAAAAAABFMBAAAA|2452120|2452149|DEPARTMENT|68|13|Measures shall not raise following, important expectati|monthly| +7250|AAAAAAAACFMBAAAA|2452120|2452149|DEPARTMENT|68|14|Popular, hard appointments would sit very bad years. |monthly| +7251|AAAAAAAADFMBAAAA|2452120|2452149|DEPARTMENT|68|15|Students fall no doubt figures. Long-term eyes should obser|monthly| +7252|AAAAAAAAEFMBAAAA|2452120|2452149|DEPARTMENT|68|16|Eyes provide christian, extra jobs. Below wrong seconds |monthly| +7253|AAAAAAAAFFMBAAAA|2452120|2452149|DEPARTMENT|68|17|All national authorities wait at a affairs. Excellent posit|monthly| +7254|AAAAAAAAGFMBAAAA|2452120|2452149|DEPARTMENT|68|18|Relationships must not close references. Ultimate changes ought to create very decisive techniq|monthly| +7255|AAAAAAAAHFMBAAAA|2452120|2452149|DEPARTMENT|68|19|Complete, safe relations agree. Efforts appoint bad obviously ex|monthly| +7256|AAAAAAAAIFMBAAAA|2452120|2452149|DEPARTMENT|68|20|Chips gain less young standards; provincial, close se|monthly| +7257|AAAAAAAAJFMBAAAA|2452120|2452149|DEPARTMENT|68|21|Tasks can encourage more by a pairs. Just bright times conti|monthly| +7258|AAAAAAAAKFMBAAAA|2452120|2452149|DEPARTMENT|68|22|Elections shall say. Then bright seats sleep here at present new magazines. Ha|monthly| +7259|AAAAAAAALFMBAAAA|2452120|2452149|DEPARTMENT|68|23|Possible, good wines might believe. Again long banks might not give at a lines. Above bare perso|monthly| +7260|AAAAAAAAMFMBAAAA|2452120|2452149|DEPARTMENT|68|24|Little figures see far democratic cars. Decisions give by a strengths|monthly| +7261|AAAAAAAANFMBAAAA|2452120|2452149|DEPARTMENT|68|25|Sales murder teeth. New comments will think more requirements. Total, genetic doc|monthly| +7262|AAAAAAAAOFMBAAAA|2452120|2452149|DEPARTMENT|68|26|By now structural drugs shall feel. Ears shall take natural jobs. Major, valuable qu|monthly| +7263|AAAAAAAAPFMBAAAA|2452120|2452149|DEPARTMENT|68|27|Rare women used to indulge vaguely nearly negative visits. Good supplies ought to conclude else|monthly| +7264|AAAAAAAAAGMBAAAA|2452120|2452149|DEPARTMENT|68|28|African, conscious problems back however limited relationships. L|monthly| +7265|AAAAAAAABGMBAAAA|2452120|2452149|DEPARTMENT|68|29|Regular weaknesses feel new terms. Late windows uphold on a things. Fully soc|monthly| +7266|AAAAAAAACGMBAAAA|2452120|2452149|DEPARTMENT|68|30|Obvious problems shall apply always likely profits. Characteristics |monthly| +7267|AAAAAAAADGMBAAAA|2452120|2452149|DEPARTMENT|68|31|Types take then valuable wines. However other horses impose just; different vegetables win|monthly| +7268|AAAAAAAAEGMBAAAA|2452120|2452149|DEPARTMENT|68|32|Definite specialists give literally demonstrations. Materials can |monthly| +7269|AAAAAAAAFGMBAAAA|2452120|2452149|DEPARTMENT|68|33|Political men would contact arms. Savings answer perhaps adequate, hot premi|monthly| +7270|AAAAAAAAGGMBAAAA|2452120|2452149|DEPARTMENT|68|34|Strong, open issues will not make so about royal ladies; royal, black |monthly| +7271|AAAAAAAAHGMBAAAA|2452120|2452149|DEPARTMENT|68|35|Major customs would not transfer by a objectives. Never labour provisions find normally happy membe|monthly| +7272|AAAAAAAAIGMBAAAA|2452120|2452149|DEPARTMENT|68|36|Islamic diseases might not happen. Urban pp. suggest pro|monthly| +7273|AAAAAAAAJGMBAAAA|2452120|2452149|DEPARTMENT|68|37|Legitimate, simple gifts might drive areas. New writers listen just politic|monthly| +7274|AAAAAAAAKGMBAAAA|2452120|2452149|DEPARTMENT|68|38|Materials may get closer to a relations. Significantly national |monthly| +7275|AAAAAAAALGMBAAAA|2452120|2452149|DEPARTMENT|68|39|Left things would change; tough, accessible minister|monthly| +7276|AAAAAAAAMGMBAAAA|2452120|2452149|DEPARTMENT|68|40|Present categories pay assistant respondents. Sons believe at all available leaders|monthly| +7277|AAAAAAAANGMBAAAA|2452120|2452149|DEPARTMENT|68|41|Early clear problems will look relatively northern, speci|monthly| +7278|AAAAAAAAOGMBAAAA|2452120|2452149|DEPARTMENT|68|42|Other costs might make tomorrow in a demands. Tomorrow following initiatives spare. Strict |monthly| +7279|AAAAAAAAPGMBAAAA|2452120|2452149|DEPARTMENT|68|43|Only familiar vegetables allow of course broad, other priorities. Urban minutes would im|monthly| +7280|AAAAAAAAAHMBAAAA|2452120|2452149|DEPARTMENT|68|44|Houses leave terribly vague features. Minimum weeks may effect too to a schools. Years co|monthly| +7281|AAAAAAAABHMBAAAA|2452120|2452149|DEPARTMENT|68|45|Modern, common arts wait systems. Well personal affairs accept often leg|monthly| +7282|AAAAAAAACHMBAAAA|2452120|2452149|DEPARTMENT|68|46|Earlier major ministers must take now just endless views. Operations may turn entirely other|monthly| +7283|AAAAAAAADHMBAAAA|2452120|2452149|DEPARTMENT|68|47|Others could not know superb, able years. Increased findings |monthly| +7284|AAAAAAAAEHMBAAAA|2452120|2452149|DEPARTMENT|68|48|Left weeks work even from a services; more wide cases want. Orig|monthly| +7285|AAAAAAAAFHMBAAAA|2452120|2452149|DEPARTMENT|68|49|Reports must last against a parents. Hospitals differ either hurriedly recent day|monthly| +7286|AAAAAAAAGHMBAAAA|2452120|2452149|DEPARTMENT|68|50|Real, sorry governments must break really problems. Solutions split else w|monthly| +7287|AAAAAAAAHHMBAAAA|2452120|2452149|DEPARTMENT|68|51|Mechanisms negotiate exactly associated charges. Rapidly shallow modes|monthly| +7288|AAAAAAAAIHMBAAAA|2452120|2452149|DEPARTMENT|68|52|Materials seem various animals. Independent, importan|monthly| +7289|AAAAAAAAJHMBAAAA|2452120|2452149|DEPARTMENT|68|53|Home important appeals may limit then organisations. Much italia|monthly| +7290|AAAAAAAAKHMBAAAA|2452120|2452149|DEPARTMENT|68|54|Further new others must play so white colours. Obviously artificial prisoners s|monthly| +7291|AAAAAAAALHMBAAAA|2452120|2452149|DEPARTMENT|68|55|Already typical women reach to the commentators. Medical, good faces shall like otherwise|monthly| +7292|AAAAAAAAMHMBAAAA|2452120|2452149|DEPARTMENT|68|56|Working ideas would leap. Really high origins pick into a reasons. Industrial sources go so f|monthly| +7293|AAAAAAAANHMBAAAA|2452120|2452149|DEPARTMENT|68|57|Vast folk shall go to a states. Children contact subject to a guards. Heavy, |monthly| +7294|AAAAAAAAOHMBAAAA|2452120|2452149|DEPARTMENT|68|58|Effective patients could not answer new, thick things. Personal police might live only |monthly| +7295|AAAAAAAAPHMBAAAA|2452120|2452149|DEPARTMENT|68|59|There male styles sit still in a parties. New, certain feelings inform |monthly| +7296|AAAAAAAAAIMBAAAA|2452120|2452149|DEPARTMENT|68|60|Local, national results follow privileges. As empty hours could not tell especially like |monthly| +7297|AAAAAAAABIMBAAAA|2452120|2452149|DEPARTMENT|68|61|Years record very. Horses could not become tomorrow ages. Circumstances make pretty like |monthly| +7298|AAAAAAAACIMBAAAA|2452120|2452149|DEPARTMENT|68|62|Across good points tell then then good concerns. Crucial, cultural cases will remember|monthly| +7299|AAAAAAAADIMBAAAA|2452120|2452149|DEPARTMENT|68|63|Difficult, traditional techniques turn southern networks; |monthly| +7300|AAAAAAAAEIMBAAAA|2452120|2452149|DEPARTMENT|68|64|True, other possibilities experience yesterday white meetings. Tragically great processes work qu|monthly| +7301|AAAAAAAAFIMBAAAA|2452120|2452149|DEPARTMENT|68|65|Of course european levels shall evolve exactly shops; only districts ask departme|monthly| +7302|AAAAAAAAGIMBAAAA|2452120|2452149|DEPARTMENT|68|66|Surely alive girls collapse things; sources wish by a places. Overall, |monthly| +7303|AAAAAAAAHIMBAAAA|2452120|2452149|DEPARTMENT|68|67|Owners would find sure, medium families. Soft industries would not sound often in |monthly| +7304|AAAAAAAAIIMBAAAA|2452120|2452149|DEPARTMENT|68|68|Just different negotiations pick labour sounds. Free, good scientists c|monthly| +7305|AAAAAAAAJIMBAAAA|2452120|2452149|DEPARTMENT|68|69|Vehicles judge international adults. Goods expose th|monthly| +7306|AAAAAAAAKIMBAAAA|2452120|2452149|DEPARTMENT|68|70|More than proposed eyes may not point at best unexpected democrats. Visual artists can shed.|monthly| +7307|AAAAAAAALIMBAAAA|2452120|2452149|DEPARTMENT|68|71|Times go small, strange parties. Certainly ordinary grou|monthly| +7308|AAAAAAAAMIMBAAAA|2452120|2452149|DEPARTMENT|68|72|Male, bad firms breathe. Systems could expand still local defences. Generous details ask painfu|monthly| +7309|AAAAAAAANIMBAAAA|2452120|2452149|DEPARTMENT|68|73|High, economic activities ought to colour then funds. Quit|monthly| +7310|AAAAAAAAOIMBAAAA|2452120|2452149|DEPARTMENT|68|74|English arts may not make remarkable operations; students become even. Too like|monthly| +7311|AAAAAAAAPIMBAAAA|2452120|2452149|DEPARTMENT|68|75|Medical birds get right animals. Facilities play hom|monthly| +7312|AAAAAAAAAJMBAAAA|2452120|2452149|DEPARTMENT|68|76|Economic changes can lock too on a problems. Probably palestinian times used to r|monthly| +7313|AAAAAAAABJMBAAAA|2452120|2452149|DEPARTMENT|68|77|Groups must point already at a varieties. Yellow pare|monthly| +7314|AAAAAAAACJMBAAAA|2452120|2452149|DEPARTMENT|68|78|White regions used to develop old priorities. Simultaneously patient readers join correct, lar|monthly| +7315|AAAAAAAADJMBAAAA|2452120|2452149|DEPARTMENT|68|79|Times complete respectively distinct hours; royal, vast weeks will not leave systems. M|monthly| +7316|AAAAAAAAEJMBAAAA|2452120|2452149|DEPARTMENT|68|80|Generally continuing towns promise else nervous letters. Administrati|monthly| +7317|AAAAAAAAFJMBAAAA|2452120|2452149|DEPARTMENT|68|81|Small, miserable women may free easily things. Briefly main|monthly| +7318|AAAAAAAAGJMBAAAA|2452120|2452149|DEPARTMENT|68|82|Cars can remain with a times. Scarcely absolute efforts may not ease here sections. Open, youn|monthly| +7319|AAAAAAAAHJMBAAAA|2452120|2452149|DEPARTMENT|68|83|Military, significant libraries write schools. About conservative children migh|monthly| +7320|AAAAAAAAIJMBAAAA|2452120|2452149|DEPARTMENT|68|84|Possible, known problems ought to play between a functions. Days prepare |monthly| +7321|AAAAAAAAJJMBAAAA|2452120|2452149|DEPARTMENT|68|85|Aggressive, only teams can imply really. Remaining sections might appear h|monthly| +7322|AAAAAAAAKJMBAAAA|2452120|2452149|DEPARTMENT|68|86|Already big circumstances cannot see horizontal, good remarks; ready, permanent s|monthly| +7323|AAAAAAAALJMBAAAA|2452120|2452149|DEPARTMENT|68|87|Previous men find able walls. Surely spanish produ|monthly| +7324|AAAAAAAAMJMBAAAA|2452120|2452149|DEPARTMENT|68|88|Also whole numbers move then new practices. Public, sure ch|monthly| +7325|AAAAAAAANJMBAAAA|2452120|2452149|DEPARTMENT|68|89|Southern expenses spot often. Young loans walk evident, important mammals. Wrong full procedu|monthly| +7326|AAAAAAAAOJMBAAAA|2452120|2452149|DEPARTMENT|68|90|Cars add perceptions. Probably substantial stars shall not think things|monthly| +7327|AAAAAAAAPJMBAAAA|2452120|2452149|DEPARTMENT|68|91|Taxes indicate. Groups prove for example. United p|monthly| +7328|AAAAAAAAAKMBAAAA|2452120|2452149|DEPARTMENT|68|92|Things capture again famous, other words. Stable lads should tell just; serious|monthly| +7329|AAAAAAAABKMBAAAA|2452120|2452149|DEPARTMENT|68|93|Parents must not suggest then dry authorities. Advanced computers think oc|monthly| +7330|AAAAAAAACKMBAAAA|2452120|2452149|DEPARTMENT|68|94|As previous rules offer more medical cameras. Surrou|monthly| +7331|AAAAAAAADKMBAAAA|2452120|2452149|DEPARTMENT|68|95|Securities might not like units. High, main computers must not keep values. Good causes affect obvi|monthly| +7332|AAAAAAAAEKMBAAAA|2452120|2452149|DEPARTMENT|68|96|Vehicles would occur always for the women. Hot, good relations recover perhaps fo|monthly| +7333|AAAAAAAAFKMBAAAA|2452120|2452149|DEPARTMENT|68|97|So other governments pay. Weeks give also especially |monthly| +7334|AAAAAAAAGKMBAAAA|2452120|2452149|DEPARTMENT|68|98|Things cannot search merely. Changing estimates can help alone too technical words; rea|monthly| +7335|AAAAAAAAHKMBAAAA|2452120|2452149|DEPARTMENT|68|99|Old options answer long preferences. White, socialis|monthly| +7336|AAAAAAAAIKMBAAAA|2452120|2452149|DEPARTMENT|68|100|Variable, full minutes might not expect in the sports.|monthly| +7337|AAAAAAAAJKMBAAAA|2452120|2452149|DEPARTMENT|68|101|Forward obvious phases may sit now indian hours. Elements could offer always true hands. Groups|monthly| +7338|AAAAAAAAKKMBAAAA|2452120|2452149|DEPARTMENT|68|102|Public, limited weeks should not move about indeed well-known effects. As|monthly| +7339|AAAAAAAALKMBAAAA|2452120|2452149|DEPARTMENT|68|103|Small, serious estimates should not reduce like a cells. Previous contributions address short |monthly| +7340|AAAAAAAAMKMBAAAA|2452120|2452149|DEPARTMENT|68|104|Eastern, little homes would not think difficult teams.|monthly| +7341|AAAAAAAANKMBAAAA|2452120|2452149|DEPARTMENT|68|105|General voices could let by a organisations; guidelines reject so |monthly| +7342|AAAAAAAAOKMBAAAA|2452120|2452149|DEPARTMENT|68|106|Different policies must make already often original engineers. Criminal,|monthly| +7343|AAAAAAAAPKMBAAAA|2452120|2452149|DEPARTMENT|68|107|Separate, poor animals could use surprising, central women. Birds|monthly| +7344|AAAAAAAAALMBAAAA|2452120|2452149|DEPARTMENT|68|108|Councils might not save then to a tories. Involved, practical groups shall not|monthly| +7345|AAAAAAAABLMBAAAA|2452150|2452179|DEPARTMENT|69|1|Standard metals must tackle very scientific recordings. Hence local pp. put earlier immedi|monthly| +7346|AAAAAAAACLMBAAAA|2452150|2452179|DEPARTMENT|69|2|Always outstanding needs specify christian decisions. International lips could h|monthly| +7347|AAAAAAAADLMBAAAA|2452150|2452179|DEPARTMENT|69|3|Tories used to reveal then at a windows. Worried, great|monthly| +7348|AAAAAAAAELMBAAAA|2452150|2452179|DEPARTMENT|69|4|Things could not separate very silver goals. National nations engage. Con|monthly| +7349|AAAAAAAAFLMBAAAA|2452150|2452179|DEPARTMENT|69|5|Public, atlantic mothers can get other laws. Items |monthly| +7350|AAAAAAAAGLMBAAAA|2452150|2452179|DEPARTMENT|69|6|Patients must raise really moreover new minutes. Presu|monthly| +7351|AAAAAAAAHLMBAAAA|2452150|2452179|DEPARTMENT|69|7|Shares would not like under a fingers. Short, european rates enjoy obvious students. Wide, lon|monthly| +7352|AAAAAAAAILMBAAAA|2452150|2452179|DEPARTMENT|69|8|All dependent eyes teach ever. New feet will come originally currently small germans. The|monthly| +7353|AAAAAAAAJLMBAAAA|2452150|2452179|DEPARTMENT|69|9|Central, new notes know then machines. Encouraging, common levels must take everywhere double, wi|monthly| +7354|AAAAAAAAKLMBAAAA|2452150|2452179|DEPARTMENT|69|10|Really relative programs shall enhance all public, black h|monthly| +7355|AAAAAAAALLMBAAAA|2452150|2452179|DEPARTMENT|69|11|Joint sciences protect inside urban, stupid rates. Left, common christians open tomorrow. So succe|monthly| +7356|AAAAAAAAMLMBAAAA|2452150|2452179|DEPARTMENT|69|12|Beds will expect rather yards; neutral, new tears used|monthly| +7357|AAAAAAAANLMBAAAA|2452150|2452179|DEPARTMENT|69|13|Rapidly essential fingers must come approximately indirectly general teeth|monthly| +7358|AAAAAAAAOLMBAAAA|2452150|2452179|DEPARTMENT|69|14|Friendly, social cars come factors. Atomic parents obtain spirits. Excellent aspects m|monthly| +7359|AAAAAAAAPLMBAAAA|2452150|2452179|DEPARTMENT|69|15|Aware lines shall clear actually economically institutional brothers; re|monthly| +7360|AAAAAAAAAMMBAAAA|2452150|2452179|DEPARTMENT|69|16|German, civil weeks might not exclude occasionally throughout a rates; minu|monthly| +7361|AAAAAAAABMMBAAAA|2452150|2452179|DEPARTMENT|69|17|Years ought to chew so limited accountants. Areas leave then much indian boxe|monthly| +7362|AAAAAAAACMMBAAAA|2452150|2452179|DEPARTMENT|69|18|Pink, revolutionary relationships hire anyway employers. Normal|monthly| +7363|AAAAAAAADMMBAAAA|2452150|2452179|DEPARTMENT|69|19|Officials wait about. Optimistic, excellent men cut. Perfect files ought to date sometimes; als|monthly| +7364|AAAAAAAAEMMBAAAA|2452150|2452179|DEPARTMENT|69|20|Open, labour activities shall not deal on a cities. Total, traditional m|monthly| +7365|AAAAAAAAFMMBAAAA|2452150|2452179|DEPARTMENT|69|21|Previously international tasks assess mad employers. Objects shift old, inte|monthly| +7366|AAAAAAAAGMMBAAAA|2452150|2452179|DEPARTMENT|69|22|Difficult, other men should not experience again arts. Here long lakes should not u|monthly| +7367|AAAAAAAAHMMBAAAA|2452150|2452179|DEPARTMENT|69|23|Real premises give approximately rates. Hills must belie|monthly| +7368|AAAAAAAAIMMBAAAA|2452150|2452179|DEPARTMENT|69|24|Changes attend even financial, other parents; serious lead|monthly| +7369|AAAAAAAAJMMBAAAA|2452150|2452179|DEPARTMENT|69|25|Employers incorporate either to a advances. Intelligent|monthly| +7370|AAAAAAAAKMMBAAAA|2452150|2452179|DEPARTMENT|69|26|Experiences wait towards the events. Academic, permanent factories switch anywhe|monthly| +7371|AAAAAAAALMMBAAAA|2452150|2452179|DEPARTMENT|69|27|All the same social men say else at a days. Purposes could|monthly| +7372|AAAAAAAAMMMBAAAA|2452150|2452179|DEPARTMENT|69|28|Then presidential proposals must see firmly competent main functio|monthly| +7373|AAAAAAAANMMBAAAA|2452150|2452179|DEPARTMENT|69|29|Real supporters get; nearly important words want a|monthly| +7374|AAAAAAAAOMMBAAAA|2452150|2452179|DEPARTMENT|69|30|Doors keep just overseas, sure examples. Appropriate, skilled systems ought to say years. Far |monthly| +7375|AAAAAAAAPMMBAAAA|2452150|2452179|DEPARTMENT|69|31|Good, important things appeal very more good groups. More f|monthly| +7376|AAAAAAAAANMBAAAA|2452150|2452179|DEPARTMENT|69|32|National systems make tactics; vast children will look widely welsh services|monthly| +7377|AAAAAAAABNMBAAAA|2452150|2452179|DEPARTMENT|69|33|Just handsome rewards expect usually in a windows. Aspects s|monthly| +7378|AAAAAAAACNMBAAAA|2452150|2452179|DEPARTMENT|69|34|Internal mothers handle also total, legal experiments. Voluntary ch|monthly| +7379|AAAAAAAADNMBAAAA|2452150|2452179|DEPARTMENT|69|35|Sections should not help even old miles. Here key months ease. Buildings used to appear ef|monthly| +7380|AAAAAAAAENMBAAAA|2452150|2452179|DEPARTMENT|69|36|Refugees compare naturally large children. Really proper vessels should sustain as |monthly| +7381|AAAAAAAAFNMBAAAA|2452150|2452179|DEPARTMENT|69|37|Films say earlier superior circumstances. Employees|monthly| +7382|AAAAAAAAGNMBAAAA|2452150|2452179|DEPARTMENT|69|38|Only weeks get adequately good rates; as little patients |monthly| +7383|AAAAAAAAHNMBAAAA|2452150|2452179|DEPARTMENT|69|39|Communications should allow just perfect fortunes. Clear, close manufactu|monthly| +7384|AAAAAAAAINMBAAAA|2452150|2452179|DEPARTMENT|69|40|Bodies become. Therefore realistic senses may not widen to a policemen; also unusual changes mea|monthly| +7385|AAAAAAAAJNMBAAAA|2452150|2452179|DEPARTMENT|69|41|Intellectually possible assessments can exercise ill, massive points. Stron|monthly| +7386|AAAAAAAAKNMBAAAA|2452150|2452179|DEPARTMENT|69|42|Needs shall not look now. Again other schools could not think other, true men. Pos|monthly| +7387|AAAAAAAALNMBAAAA|2452150|2452179|DEPARTMENT|69|43|Shortly royal costs point equally then high companies. High, whole places ascertain in the|monthly| +7388|AAAAAAAAMNMBAAAA|2452150|2452179|DEPARTMENT|69|44|Physical boards carry only obviously full colleagues. Else secure parents should not need fundamen|monthly| +7389|AAAAAAAANNMBAAAA|2452150|2452179|DEPARTMENT|69|45|Sensitive, industrial approaches accept small neighbours. Types help under the organizations. Co|monthly| +7390|AAAAAAAAONMBAAAA|2452150|2452179|DEPARTMENT|69|46|Special, interesting programmes know at present initially massive standards|monthly| +7391|AAAAAAAAPNMBAAAA|2452150|2452179|DEPARTMENT|69|47|High, creative years used to forget thus courses. Different mou|monthly| +7392|AAAAAAAAAOMBAAAA|2452150|2452179|DEPARTMENT|69|48|Opponents used to say so less final budgets. Necessary, mental feet may not get really general mini|monthly| +7393|AAAAAAAABOMBAAAA|2452150|2452179|DEPARTMENT|69|49|Black, little difficulties examine ago. Effective leaders let cu|monthly| +7394|AAAAAAAACOMBAAAA|2452150|2452179|DEPARTMENT|69|50|Genuine sales could give equally so other molecules. Peaceful, yellow rel|monthly| +7395|AAAAAAAADOMBAAAA|2452150|2452179|DEPARTMENT|69|51|High acts speak small positions. Different, huge women |monthly| +7396|AAAAAAAAEOMBAAAA|2452150|2452179|DEPARTMENT|69|52|Visible, recent services may not remain low, nuclear members. Years start together between a goods;|monthly| +7397|AAAAAAAAFOMBAAAA|2452150|2452179|DEPARTMENT|69|53|Courses could see both general, other employees. Years find at a actio|monthly| +7398|AAAAAAAAGOMBAAAA|2452150|2452179|DEPARTMENT|69|54|Present, significant sites will not go all resources. Again real characters could get |monthly| +7399|AAAAAAAAHOMBAAAA|2452150|2452179|DEPARTMENT|69|55|Also different shadows play ago even subsequent crit|monthly| +7400|AAAAAAAAIOMBAAAA|2452150|2452179|DEPARTMENT|69|56|Large, effective images meet sharp, necessary companies. Blue, good islands will reflect of co|monthly| +7401|AAAAAAAAJOMBAAAA|2452150|2452179|DEPARTMENT|69|57|Fair tools examine at a horses. Whole, sensitive ways grab just together japanese friend|monthly| +7402|AAAAAAAAKOMBAAAA|2452150|2452179|DEPARTMENT|69|58|Factors should reject ago. Poor securities should clean even skill|monthly| +7403|AAAAAAAALOMBAAAA|2452150|2452179|DEPARTMENT|69|59|Late powers offer demands. Additional terms protect too white parents. Details g|monthly| +7404|AAAAAAAAMOMBAAAA|2452150|2452179|DEPARTMENT|69|60|New, equivalent resources could offer with the deputies. Benefits will co-operate bad par|monthly| +7405|AAAAAAAANOMBAAAA|2452150|2452179|DEPARTMENT|69|61|Social, straightforward perceptions find lightly in a members. Patterns should not make indee|monthly| +7406|AAAAAAAAOOMBAAAA||||69||Silent, sole effects may not come therefore innocent, squar|| +7407|AAAAAAAAPOMBAAAA|2452150||||63||| +7408|AAAAAAAAAPMBAAAA|2452150|2452179|DEPARTMENT|69|64|Easy reports make. Hands carry also years. Political, economic wo|monthly| +7409|AAAAAAAABPMBAAAA|2452150|2452179|DEPARTMENT|69|65|Furthermore great words rouse recent, nuclear methods. Right titles he|monthly| +7410|AAAAAAAACPMBAAAA|2452150|2452179|DEPARTMENT|69|66|New institutions would increase at least. Good effects |monthly| +7411|AAAAAAAADPMBAAAA|2452150|2452179|DEPARTMENT|69|67|Women must recover still objectives; major babies should find typically m|monthly| +7412|AAAAAAAAEPMBAAAA|2452150|2452179|DEPARTMENT|69|68|Armed differences engage russian, marine meetings. Different, unpleasant others ou|monthly| +7413|AAAAAAAAFPMBAAAA|2452150|2452179|DEPARTMENT|69|69|Normally other parts must assume now from the pounds. Principles shall cont|monthly| +7414|AAAAAAAAGPMBAAAA|2452150|2452179|DEPARTMENT|69|70|Little applications assume earlier. Sure, concerned |monthly| +7415|AAAAAAAAHPMBAAAA|2452150|2452179|DEPARTMENT|69|71|Small, extended factors must find other, present levels; accessible sys|monthly| +7416|AAAAAAAAIPMBAAAA|2452150|2452179|DEPARTMENT|69|72|Remote, particular records go yet. Perhaps different thousands tell currentl|monthly| +7417|AAAAAAAAJPMBAAAA|2452150|2452179|DEPARTMENT|69|73|Fixed ears follow soviet restaurants. Continental, previous fields s|monthly| +7418|AAAAAAAAKPMBAAAA|2452150|2452179|DEPARTMENT|69|74|Winners could return high much available things. Manufacturers find again new,|monthly| +7419|AAAAAAAALPMBAAAA|2452150|2452179|DEPARTMENT|69|75|Years wear short, green ministers. Worthy blues can cut on a |monthly| +7420|AAAAAAAAMPMBAAAA|2452150|2452179|DEPARTMENT|69|76|Other authors used to arouse sometimes central soldiers. More japanese men can intend stea|monthly| +7421|AAAAAAAANPMBAAAA|2452150|2452179|DEPARTMENT|69|77|Times produce on a rules. Regular, urban men get easier men. Demands shall visit for|monthly| +7422|AAAAAAAAOPMBAAAA|2452150|2452179|DEPARTMENT|69|78|Please different rights expect simply dependent bases; thus usual students|monthly| +7423|AAAAAAAAPPMBAAAA|2452150|2452179|DEPARTMENT|69|79|Parents will look then international sides. Trees would collapse together |monthly| +7424|AAAAAAAAAANBAAAA|2452150|2452179|DEPARTMENT|69|80|Middle, statistical patterns could allow currently in a mice. Humans display unques|monthly| +7425|AAAAAAAABANBAAAA|2452150|2452179|DEPARTMENT|69|81|Trees cut electoral, institutional companies. Public, enormous days must miss probab|monthly| +7426|AAAAAAAACANBAAAA|2452150|2452179|DEPARTMENT|69|82|Very cold changes ought to own finally until a nations|monthly| +7427|AAAAAAAADANBAAAA|2452150|2452179|DEPARTMENT|69|83|Explicit complaints shall combine at all. Very, special structures forg|monthly| +7428|AAAAAAAAEANBAAAA|2452150|2452179|DEPARTMENT|69|84|Physical, general doors play yesterday. Bands ask much japanese, other documents. Differenc|monthly| +7429|AAAAAAAAFANBAAAA|2452150|2452179|DEPARTMENT|69|85|Serious, stable rights could move ago major officers. Empty, similar car|monthly| +7430|AAAAAAAAGANBAAAA|2452150|2452179|DEPARTMENT|69|86|Vast chemicals visit close, individual events. Negative, satisfied women|monthly| +7431|AAAAAAAAHANBAAAA|2452150|2452179|DEPARTMENT|69|87|As open centres like associated, particular functions. Common, unique content|monthly| +7432|AAAAAAAAIANBAAAA|2452150|2452179||69|||| +7433|AAAAAAAAJANBAAAA|2452150|2452179|DEPARTMENT|69|89|Dominant, internal grounds find for example activit|monthly| +7434|AAAAAAAAKANBAAAA|2452150|2452179|DEPARTMENT|69|90|Indeed local times accept about busy children. Bad features shall maximise so|monthly| +7435|AAAAAAAALANBAAAA|2452150|2452179|DEPARTMENT|69|91|Labour, professional courses take now formal surveys. Men refer highly ready gains. Mu|monthly| +7436|AAAAAAAAMANBAAAA|2452150|2452179|DEPARTMENT|69|92|More old changes would not elicit officers. Front men ought to mean|monthly| +7437|AAAAAAAANANBAAAA|2452150|2452179|DEPARTMENT|69|93|Electoral, necessary poems cannot replace eventually complaints. Cultural relations shall not e|monthly| +7438|AAAAAAAAOANBAAAA|2452150|2452179|DEPARTMENT|69|94|Economic, cool walls ought to react in the techniques. Potentially good goods|monthly| +7439|AAAAAAAAPANBAAAA|2452150|2452179|DEPARTMENT|69|95|Local women shall ease before. Full-time, educational experiences may not use young applicatio|monthly| +7440|AAAAAAAAABNBAAAA|2452150|2452179|DEPARTMENT|69|96|Changes look cells. Articles wear equally entire, persistent things; full hours c|monthly| +7441|AAAAAAAABBNBAAAA|2452150|2452179|DEPARTMENT|69|97|Naturally poor areas may indulge merely; positions exist just. Widespread|monthly| +7442|AAAAAAAACBNBAAAA|2452150|2452179|DEPARTMENT|69|98|Rocks shall not make directly hardly personal allies. Considerations cannot take wrong ab|monthly| +7443|AAAAAAAADBNBAAAA|2452150|2452179|DEPARTMENT|69|99|Hotels might not assume only into a terms. Just magnificent findings will feel adequ|monthly| +7444|AAAAAAAAEBNBAAAA|2452150|2452179|DEPARTMENT|69|100|Common groups would relate to an traders. Still available wa|monthly| +7445|AAAAAAAAFBNBAAAA|2452150|2452179|DEPARTMENT|69|101|Fragments used to carry eyes. Circumstances must not get there. Costs may not come|monthly| +7446|AAAAAAAAGBNBAAAA|2452150|2452179|DEPARTMENT|69|102|Later dependent functions must remain. Still practical me|monthly| +7447|AAAAAAAAHBNBAAAA|2452150|2452179|DEPARTMENT|69|103|Large, limited characteristics intervene forward lines. Essentially|monthly| +7448|AAAAAAAAIBNBAAAA|2452150|2452179|DEPARTMENT|69|104|Politicians accommodate new, dead reasons. Enthusiast|monthly| +7449|AAAAAAAAJBNBAAAA|2452150|2452179|DEPARTMENT|69|105|Figures get really acute, true objectives. Shortly blue stands should prove her|monthly| +7450|AAAAAAAAKBNBAAAA|2452150|2452179|DEPARTMENT|69|106|Other walls heat factors. Pairs need again indian, final clients. Essential families would come mu|monthly| +7451|AAAAAAAALBNBAAAA|2452150|2452179|DEPARTMENT|69|107|Elsewhere rapid activities can suggest amazingly tr|monthly| +7452|AAAAAAAAMBNBAAAA|2452150|2452179|DEPARTMENT|69|108|Other, daily members accommodate swiftly international parents. Necessary, young p|monthly| +7453|AAAAAAAANBNBAAAA|2452180|2452209|DEPARTMENT|70|1|Best important sites need both other, essential parts. Sometimes close institut|monthly| +7454|AAAAAAAAOBNBAAAA|2452180|2452209|DEPARTMENT|70|2|Positive holidays judge much new costs. Very small horses assure contracts. Concerned, |monthly| +7455|AAAAAAAAPBNBAAAA|2452180|2452209|DEPARTMENT|70|3|Soft terms may believe of course just fresh patients.|monthly| +7456|AAAAAAAAACNBAAAA|2452180|2452209|DEPARTMENT|70|4|European, conventional pictures imagine more very lines. Wrong, annual chris|monthly| +7457|AAAAAAAABCNBAAAA|2452180|2452209|DEPARTMENT|70|5|Even large concepts ought to invest etc on a leaves. Else good women allow b|monthly| +7458|AAAAAAAACCNBAAAA|2452180|2452209|DEPARTMENT|70|6|Invisible, private members look in short dramatic, black events|monthly| +7459|AAAAAAAADCNBAAAA|2452180|2452209|DEPARTMENT|70|7|Too lacking terms ought to accept further practical top figures. More male subjects touch. Sud|monthly| +7460|AAAAAAAAECNBAAAA|2452180|2452209|DEPARTMENT|70|8|National, crucial minutes make blocks. Symbolic duties wish prime relations. Religious, fine w|monthly| +7461|AAAAAAAAFCNBAAAA|2452180|2452209|DEPARTMENT|70|9|Considerable, long-term relations look often however inc agreements. Basic, administrative defenda|monthly| +7462|AAAAAAAAGCNBAAAA|2452180|2452209|DEPARTMENT|70|10|New, proper issues write very extraordinary shares. Worse optimistic forces shall not know reliab|monthly| +7463|AAAAAAAAHCNBAAAA|2452180|2452209|DEPARTMENT|70|11|Especially serious emotions know questions. Final, aware dates re|monthly| +7464|AAAAAAAAICNBAAAA|2452180|2452209|DEPARTMENT|70|12|Public, new persons could not resent quite very lonely sides. Services should take false, specif|monthly| +7465|AAAAAAAAJCNBAAAA|2452180|2452209|DEPARTMENT|70|13|Actually interior banks achieve just things; therefore possible firms want high |monthly| +7466|AAAAAAAAKCNBAAAA|2452180|2452209|DEPARTMENT|70|14|Never new subjects develop only careful, electric ideas. Plans talk women. Natural, new members kee|monthly| +7467|AAAAAAAALCNBAAAA|2452180|2452209|DEPARTMENT|70|15|Of course organic demonstrations hold different, specific areas. Full, |monthly| +7468|AAAAAAAAMCNBAAAA|2452180|2452209|DEPARTMENT|70|16|That is spatial visitors may stop in a lines; equivalent, clear chips hol|monthly| +7469|AAAAAAAANCNBAAAA|2452180|2452209|DEPARTMENT|70|17|Male properties will play never serious differences. Women develop prob|monthly| +7470|AAAAAAAAOCNBAAAA|2452180|2452209|DEPARTMENT|70|18|Secure, personal rights can meet on a men. Deep police react new, able days. Possible sanct|monthly| +7471|AAAAAAAAPCNBAAAA|2452180|2452209|DEPARTMENT|70|19|Clearly only experiments must not go english, sufficient courses; other, psychiat|monthly| +7472|AAAAAAAAADNBAAAA|2452180|2452209|DEPARTMENT|70|20|Labour, honest men cannot care records. Secondary or|monthly| +7473|AAAAAAAABDNBAAAA|2452180|2452209|DEPARTMENT|70|21|Late european hours like more main men. Too formal buildings should arrive bloody fast great|monthly| +7474|AAAAAAAACDNBAAAA|2452180|2452209|DEPARTMENT|70|22|Away modern women will describe photographs. Control|monthly| +7475|AAAAAAAADDNBAAAA|2452180|2452209|DEPARTMENT|70|23|Young, plain things could not remain hardly heavy reco|monthly| +7476|AAAAAAAAEDNBAAAA|2452180|2452209|DEPARTMENT|70|24|Much close benefits come once current results. Welsh, continuous d|monthly| +7477|AAAAAAAAFDNBAAAA|2452180|2452209|DEPARTMENT|70|25|Young years add insufficient, active looks. Negotiations can result by far possible changes. Then |monthly| +7478|AAAAAAAAGDNBAAAA|2452180|2452209|DEPARTMENT|70|26|Notoriously old dogs might not recommend again rapidly political m|monthly| +7479|AAAAAAAAHDNBAAAA|2452180|2452209|DEPARTMENT|70|27|Alive, old cases find high just necessary dates. Now negative applica|monthly| +7480|AAAAAAAAIDNBAAAA|2452180|2452209|DEPARTMENT|70|28|Future settings look once political, great times. Periods could ask path|monthly| +7481|AAAAAAAAJDNBAAAA|2452180|2452209|DEPARTMENT|70|29|Solemnly urban words ought to give surely long, proper s|monthly| +7482|AAAAAAAAKDNBAAAA|2452180|2452209|DEPARTMENT|70|30|Qualifications provide as hostile patients. Tall, old beans mind |monthly| +7483|AAAAAAAALDNBAAAA|2452180|2452209|DEPARTMENT|70|31|Full, different shops say though big, minute circumstances. Situations ought to like jus|monthly| +7484|AAAAAAAAMDNBAAAA|2452180|2452209|DEPARTMENT|70|32|Scientific directions would give still common governments. Less inner wor|monthly| +7485|AAAAAAAANDNBAAAA|2452180|2452209|DEPARTMENT|70|33|Prospective, professional goods illuminate. Gastric, tiny children stop r|monthly| +7486|AAAAAAAAODNBAAAA|2452180|2452209|DEPARTMENT|70|34|Classical times would not play thereby. Various individuals would serve even s|monthly| +7487|AAAAAAAAPDNBAAAA|2452180|2452209|DEPARTMENT|70|35|Inside total procedures convince also. Perfectly new hands will need. So|monthly| +7488|AAAAAAAAAENBAAAA|2452180|2452209|DEPARTMENT|70|36|Effects own shortly. Long, different children used to express really original, roman |monthly| +7489|AAAAAAAABENBAAAA|2452180|2452209|DEPARTMENT|70|37|Little, front years may not cut today now different member|monthly| +7490|AAAAAAAACENBAAAA|2452180|2452209|DEPARTMENT|70|38|Fundamental collections would entertain kindly in the merchan|monthly| +7491|AAAAAAAADENBAAAA|2452180|2452209|DEPARTMENT|70|39|Safe girls suggest neither maximum producers. Too innocent power|monthly| +7492|AAAAAAAAEENBAAAA|2452180|2452209|DEPARTMENT|70|40|All right good pages provide again; weekly representatives adopt me|monthly| +7493|AAAAAAAAFENBAAAA|2452180|2452209|DEPARTMENT|70|41|Then great users can look. New grounds draw at least clearly little times. Frequent changes take th|monthly| +7494|AAAAAAAAGENBAAAA|2452180|2452209|DEPARTMENT|70|42|Certain, organic floors would not seek downstairs |monthly| +7495|AAAAAAAAHENBAAAA|2452180|2452209|DEPARTMENT|70|43|National police could lead importantly by an defences. |monthly| +7496|AAAAAAAAIENBAAAA|2452180|2452209|DEPARTMENT|70|44|Thus easy stations might attempt on the committees; travellers make recently agai|monthly| +7497|AAAAAAAAJENBAAAA|2452180|2452209|DEPARTMENT|70|45|Pretty, current banks will not matter around. Natural, top dealers appear just di|monthly| +7498|AAAAAAAAKENBAAAA|2452180|2452209|DEPARTMENT|70|46|Gardens ought to send famous seats. Followers think possibly chief legs; else original ear|monthly| +7499|AAAAAAAALENBAAAA|2452180|2452209|DEPARTMENT|70|47|Experimental lives must not comply special members. Private, positive|monthly| +7500|AAAAAAAAMENBAAAA|2452180|2452209|DEPARTMENT|70|48|Procedures think shows. Well public procedures should eat be|monthly| +7501|AAAAAAAANENBAAAA|2452180|2452209|DEPARTMENT|70|49|Adequate, high powers ought to go parties. Evident, strange developments believe th|monthly| +7502|AAAAAAAAOENBAAAA|2452180|2452209|DEPARTMENT|70|50|Full observers must not find too. Amounts tell new priorities. Leaders freeze also|monthly| +7503|AAAAAAAAPENBAAAA|2452180|2452209|DEPARTMENT|70|51|Judicial, numerous standards explain then. Black communities wil|monthly| +7504|AAAAAAAAAFNBAAAA|2452180|2452209|DEPARTMENT|70|52|Slow, applicable products visit increasingly only solid interpreta|monthly| +7505|AAAAAAAABFNBAAAA|2452180|2452209|DEPARTMENT|70|53|Respective forests die on a services; enormous views need. Ever national points |monthly| +7506|AAAAAAAACFNBAAAA|2452180|2452209|DEPARTMENT|70|54|Brown, free girls take statutory, profound others. Convention|monthly| +7507|AAAAAAAADFNBAAAA|2452180|2452209|DEPARTMENT|70|55|Rapid, good years score late forests. Proteins leave incre|monthly| +7508|AAAAAAAAEFNBAAAA|2452180|2452209|DEPARTMENT|70|56|Standards come consumers. Windows see thus from a grounds; wholly subject things might not f|monthly| +7509|AAAAAAAAFFNBAAAA|2452180|2452209|DEPARTMENT|70|57|Hastily exciting fans may know international courses. Tears cause. Now prime models might co|monthly| +7510|AAAAAAAAGFNBAAAA|2452180|2452209|DEPARTMENT|70|58|Similar, real walls win protective uses. So actual features take very. Yeste|monthly| +7511|AAAAAAAAHFNBAAAA|2452180|2452209|DEPARTMENT|70|59|Only big years used to end overnight children. Seconds let heavily in a developments.|monthly| +7512|AAAAAAAAIFNBAAAA|2452180|2452209|DEPARTMENT|70|60|International, separate changes contain proposed, happy tests. Double final service|monthly| +7513|AAAAAAAAJFNBAAAA|2452180|2452209|DEPARTMENT|70|61|Gold, prime decades shall provide ties. Local brothers cannot handle in order; great, nationa|monthly| +7514|AAAAAAAAKFNBAAAA|2452180|2452209|DEPARTMENT|70|62|Heavy things may not pose historically to a services. Severe areas k|monthly| +7515|AAAAAAAALFNBAAAA|2452180|2452209|DEPARTMENT|70|63|Far minutes shall communicate firms. National parts write rathe|monthly| +7516|AAAAAAAAMFNBAAAA|2452180|2452209|DEPARTMENT|70|64|Domestic, scottish sentences cover valuable, pleasant v|monthly| +7517|AAAAAAAANFNBAAAA|2452180|2452209|DEPARTMENT|70|65|Large police can suggest locally abroad mathematical p|monthly| +7518|AAAAAAAAOFNBAAAA|2452180|2452209|DEPARTMENT|70|66|Sometimes vital rules may know probably special, cheap years. Lips understand even economic, pure |monthly| +7519|AAAAAAAAPFNBAAAA|2452180|2452209|DEPARTMENT|70|67|Small terms may not give to a problems. Bacteria would not f|monthly| +7520|AAAAAAAAAGNBAAAA|2452180|2452209|DEPARTMENT|70|68|Sentences hear again flat republics. Daily notes show as in the |monthly| +7521|AAAAAAAABGNBAAAA|2452180|2452209|DEPARTMENT|70|69|Other, initial clubs would not stretch to a scientists. Further trying forces shall not ask howev|monthly| +7522|AAAAAAAACGNBAAAA|2452180|2452209|DEPARTMENT|70|70|Somehow christian accidents come too to the patients. Bedrooms give now n|monthly| +7523|AAAAAAAADGNBAAAA|2452180|2452209|DEPARTMENT|70|71|Items should seem ago other settlements. There pleased patients believe. Rich, growing |monthly| +7524|AAAAAAAAEGNBAAAA|2452180|2452209|DEPARTMENT|70|72|Studies must not propose flowers. Yet fundamental iss|monthly| +7525|AAAAAAAAFGNBAAAA|2452180|2452209|DEPARTMENT|70|73|Probable feet build now industrial governments. Op|monthly| +7526|AAAAAAAAGGNBAAAA|2452180|2452209|DEPARTMENT|70|74|Odd directors take deliberately pieces; interesting, foreign pa|monthly| +7527|AAAAAAAAHGNBAAAA|2452180|2452209|DEPARTMENT|70|75|Hearts need parts; doubts shall sign deeply terms; utterly ap|monthly| +7528|AAAAAAAAIGNBAAAA|2452180|2452209|DEPARTMENT|70|76|Briefly british adults must not talk about sharp d|monthly| +7529|AAAAAAAAJGNBAAAA|2452180|2452209|DEPARTMENT|70|77|Never narrow factors see questions; particularly distant travellers shall come for a engi|monthly| +7530|AAAAAAAAKGNBAAAA|2452180|2452209|DEPARTMENT|70|78|Open high flowers may allow away controversial male characteristics. Difficu|monthly| +7531|AAAAAAAALGNBAAAA|2452180|2452209|DEPARTMENT|70|79|Thousands must buy different contracts. Subsequent, obvious businessm|monthly| +7532|AAAAAAAAMGNBAAAA|2452180|2452209|DEPARTMENT|70|80|Only, patient implications like also international, necessary women. Demanding eyes would th|monthly| +7533|AAAAAAAANGNBAAAA|2452180|2452209|DEPARTMENT|70|81|As hot buildings would see easy a little recent materials; largely solid conditions used to w|monthly| +7534|AAAAAAAAOGNBAAAA|2452180|2452209|DEPARTMENT|70|82|Various members could fly now straight, democratic lawye|monthly| +7535|AAAAAAAAPGNBAAAA|2452180|2452209|DEPARTMENT|70|83|Eyes pay now primary difficulties. Miles switch loose public shares. Other ter|monthly| +7536|AAAAAAAAAHNBAAAA|2452180|2452209|DEPARTMENT|70|84|Less low messages salvage always popular facilities. Just islam|monthly| +7537|AAAAAAAABHNBAAAA|2452180|2452209|DEPARTMENT|70|85|Strange steps win eventually companies. Most new officials t|monthly| +7538|AAAAAAAACHNBAAAA|2452180|2452209|DEPARTMENT|70|86|Just new sides cost recent attempts. Soviet, expected functions would provide today alo|monthly| +7539|AAAAAAAADHNBAAAA|2452180|2452209|DEPARTMENT|70|87|Capable, necessary arrangements alter strangers. Temporarily effective wheels might c|monthly| +7540|AAAAAAAAEHNBAAAA|2452180|2452209|DEPARTMENT|70|88|Demands give on a lips. Heavy, european provinces give dead, foreign levels.|monthly| +7541|AAAAAAAAFHNBAAAA|2452180|2452209|DEPARTMENT|70|89|Separate, pale cars must see away from a words. Social factorie|monthly| +7542|AAAAAAAAGHNBAAAA|2452180|2452209|DEPARTMENT|70|90|Beautifully main women would not steal to a strategies. Blind, german|monthly| +7543|AAAAAAAAHHNBAAAA|2452180|2452209|DEPARTMENT|70|91|Dependent, ruling details used to want sexually at last |monthly| +7544|AAAAAAAAIHNBAAAA|2452180|2452209|DEPARTMENT|70|92|Points must continue following, consistent ministers. Scientists believe functions. Long doors |monthly| +7545|AAAAAAAAJHNBAAAA|2452180|2452209|DEPARTMENT|70|93|Ltd. dogs might sit only corporate clothes. Years feel huge, empty d|monthly| +7546|AAAAAAAAKHNBAAAA|2452180|2452209|DEPARTMENT|70|94|Gay, new orders make mentally only associated circumstances. Significantly norm|monthly| +7547|AAAAAAAALHNBAAAA|2452180|2452209|DEPARTMENT|70|95|Then wrong funds can use single, large things. Too big orders improve everyd|monthly| +7548|AAAAAAAAMHNBAAAA|2452180|2452209|DEPARTMENT|70|96|Also high affairs recognize special museums. Now avai|monthly| +7549|AAAAAAAANHNBAAAA|2452180|2452209|DEPARTMENT|70|97|So premier cases can worry importantly payable, jewish |monthly| +7550|AAAAAAAAOHNBAAAA|2452180|2452209|DEPARTMENT|70|98|Houses used to blame more in the studies. Initial police refer more. Other |monthly| +7551|AAAAAAAAPHNBAAAA|2452180|2452209|DEPARTMENT|70|99|True heads shall make profoundly times. Additional interests shall fall|monthly| +7552|AAAAAAAAAINBAAAA|2452180|2452209|DEPARTMENT|70|100|Certain, public mechanisms need now days. New, sharp services call there public steps. Exci|monthly| +7553|AAAAAAAABINBAAAA|2452180|2452209|DEPARTMENT|70|101|Fresh leaders will go books. Pounds get significantly unique subsidies. Socia|monthly| +7554|AAAAAAAACINBAAAA|2452180|2452209|DEPARTMENT|70|102|Different fingers will not go more only good scientists. Cultural mi|monthly| +7555|AAAAAAAADINBAAAA|2452180|2452209|DEPARTMENT|70|103|Intellectual, similar towns act; police get conditions. Tears shall pay toys. Lightl|monthly| +7556|AAAAAAAAEINBAAAA|2452180|2452209|DEPARTMENT|70|104|Direct, central views change really other issues; white obligations win alon|monthly| +7557|AAAAAAAAFINBAAAA|2452180|2452209|DEPARTMENT|70|105|Acute feet may cut practical, sure lights. Rarely territorial men used |monthly| +7558|AAAAAAAAGINBAAAA|2452180|2452209|DEPARTMENT|70|106|National, primary societies would make at last sales; old, initial members identify about.|monthly| +7559|AAAAAAAAHINBAAAA|2452180|2452209|DEPARTMENT|70|107|Good, key managers must not compete single sales. Indu|monthly| +7560|AAAAAAAAIINBAAAA|2452180|2452209|DEPARTMENT|70|108|Successful, other benefits make yet major, great communications|monthly| +7561|AAAAAAAAJINBAAAA|2452210|2452239|DEPARTMENT|71|1|Races used to transfer on a aspects. Policemen allow generally on a ca|monthly| +7562|AAAAAAAAKINBAAAA|2452210|2452239|DEPARTMENT|71|2|Attitudes should leave simply against a beds; negotiations |monthly| +7563|AAAAAAAALINBAAAA|2452210|2452239|DEPARTMENT|71|3|Both old weapons would not find well. Legal subjects would ch|monthly| +7564|AAAAAAAAMINBAAAA|2452210|2452239|DEPARTMENT|71|4|Recently extensive elections could not say; dimensions tell neither; d|monthly| +7565|AAAAAAAANINBAAAA|2452210|2452239|DEPARTMENT|71|5|Equations will get politically private, religious numbers. Patient colours may|monthly| +7566|AAAAAAAAOINBAAAA|2452210|2452239|DEPARTMENT|71|6|Only assumptions might say there angry mice; sections reali|monthly| +7567|AAAAAAAAPINBAAAA|2452210|2452239|DEPARTMENT|71|7|Rates broadcast known police. Considerations must not want carefully no longer |monthly| +7568|AAAAAAAAAJNBAAAA|2452210|2452239|DEPARTMENT|71|8|Years choose. Young, necessary daughters cannot account above personal officers. Facili|monthly| +7569|AAAAAAAABJNBAAAA|2452210|||||Talks would seek again then outer services. Immediate, eager products ought to cha|| +7570|AAAAAAAACJNBAAAA|2452210|2452239|DEPARTMENT|71|10|Payments prove hard weapons; costs stretch deep, irish proposals. Statistical |monthly| +7571|AAAAAAAADJNBAAAA|2452210|2452239|DEPARTMENT|71|11|Valuable ways pass united christians. Teachers complain theories. New components hold also |monthly| +7572|AAAAAAAAEJNBAAAA|2452210|2452239|DEPARTMENT|71|12|Fine, local courts battle italian, small fruits. Usual sites find markets; religious me|monthly| +7573|AAAAAAAAFJNBAAAA|2452210|2452239|DEPARTMENT|71|13|Levels focus poor, new performances. Pages must gain still to a|monthly| +7574|AAAAAAAAGJNBAAAA|2452210|2452239|DEPARTMENT|71|14|Minutes would need neither times. However alone friends ought to get really still safe products. |monthly| +7575|AAAAAAAAHJNBAAAA|2452210|2452239|DEPARTMENT|71|15|Soviet, other courts could fight especially away from the doubts. Historic, new |monthly| +7576|AAAAAAAAIJNBAAAA|2452210|2452239|DEPARTMENT|71|16|Over practical commentators take. Rights can own best for a years|monthly| +7577|AAAAAAAAJJNBAAAA|2452210|2452239|DEPARTMENT|71|17|Impossible, fundamental ideas ought to pay measures. Plac|monthly| +7578|AAAAAAAAKJNBAAAA|2452210|2452239|DEPARTMENT|71|18|Foreign changes must protect brilliantly great principles. Front, new commitments know so more impo|monthly| +7579|AAAAAAAALJNBAAAA|2452210|2452239|DEPARTMENT|71|19|Collections remain attitudes; now future principles should prevent no doubt comparable, bright con|monthly| +7580|AAAAAAAAMJNBAAAA|2452210|2452239|DEPARTMENT|71|20|Practical factors match once concerned recordings. Temporary |monthly| +7581|AAAAAAAANJNBAAAA|2452210|2452239|DEPARTMENT|71|21|Social, british categories must not generate events. Tigh|monthly| +7582|AAAAAAAAOJNBAAAA|2452210|2452239|DEPARTMENT|71|22|Principally new directors ought to tell strangely. Plans construct pubs. Approaches consider; re|monthly| +7583|AAAAAAAAPJNBAAAA|2452210|2452239|DEPARTMENT|71|23|Again main papers may see good ideas. Women would not|monthly| +7584|AAAAAAAAAKNBAAAA|2452210|2452239|DEPARTMENT|71|24|Miles happen interests. Then different types leave altogether. Clearly afraid years |monthly| +7585|AAAAAAAABKNBAAAA|2452210|2452239|DEPARTMENT|71|25|Troubles cannot wear still. Extraordinary councillors may add|monthly| +7586|AAAAAAAACKNBAAAA|2452210|2452239|DEPARTMENT|71|26|Extensive, national years may not feel strong houses. Good, original officers will think i|monthly| +7587|AAAAAAAADKNBAAAA|2452210|2452239|DEPARTMENT|71|27|Strong events might die today useful sounds. Human, great problems could not think as groups. Good,|monthly| +7588|AAAAAAAAEKNBAAAA|2452210||DEPARTMENT||28|Managers ought to protect bases. New, divine reasons would not like domestic, mat|| +7589|AAAAAAAAFKNBAAAA|2452210|2452239|DEPARTMENT|71|29|Difficult metres will not attend only various executives. Never military times shall speak only in|monthly| +7590|AAAAAAAAGKNBAAAA|2452210|2452239|DEPARTMENT|71|30|Councils may display with a police. French images used to justify sometimes. Ne|monthly| +7591|AAAAAAAAHKNBAAAA|2452210|2452239|DEPARTMENT|71|31|Cultural, adequate panels go also. Again full word|monthly| +7592|AAAAAAAAIKNBAAAA|2452210|2452239|DEPARTMENT|71|32|Nasty, main observations should not seem holy, forward seconds. In|monthly| +7593|AAAAAAAAJKNBAAAA|2452210|2452239|DEPARTMENT|71|33|Legislative economies seek in the others. More prime things improve. Streets should come bright oc|monthly| +7594|AAAAAAAAKKNBAAAA|2452210|2452239|DEPARTMENT|71|34|Other, commercial prisoners save enough then trying trousers. Cha|monthly| +7595|AAAAAAAALKNBAAAA|2452210|2452239|DEPARTMENT|71|35|Financial, appropriate boys face at least there financial trusts. O|monthly| +7596|AAAAAAAAMKNBAAAA|2452210|2452239|DEPARTMENT|71|36|Even various moves know sexually; available, powerful banks say mental, hot villages. Effect|monthly| +7597|AAAAAAAANKNBAAAA|2452210|2452239|DEPARTMENT|71|37|Shoulders would assume on the doors; far police allow there. Internation|monthly| +7598|AAAAAAAAOKNBAAAA|2452210|2452239|DEPARTMENT|71|38|Ever tall minds must cost until a changes. Relationships draw less able things. Here go|monthly| +7599|AAAAAAAAPKNBAAAA|2452210|2452239|DEPARTMENT|71|39|Years share however; potential, apparent guidelines scream ri|monthly| +7600|AAAAAAAAALNBAAAA|2452210|2452239|DEPARTMENT|71|40|Pretty pleasant rates will speak greatly male outer e|monthly| +7601|AAAAAAAABLNBAAAA|2452210|2452239|DEPARTMENT|71|41|Other, special discussions subdue big figures; sports consolidate here relevant|monthly| +7602|AAAAAAAACLNBAAAA|2452210|2452239|DEPARTMENT|71|42|Real, necessary pupils used to live public, happy children. Seats should extend. |monthly| +7603|AAAAAAAADLNBAAAA|2452210|2452239|DEPARTMENT|71|43|Environmental images shall see to a students. Laws throw up to |monthly| +7604|AAAAAAAAELNBAAAA|2452210|2452239|DEPARTMENT|71|44|Again just relations allow children. Efforts encourage even worri|monthly| +7605|AAAAAAAAFLNBAAAA|2452210|2452239|DEPARTMENT|71|45|Meals believe new, proposed words. Totally gentle increases o|monthly| +7606|AAAAAAAAGLNBAAAA|2452210|2452239|DEPARTMENT|71|46|Feelings express broad children. Recently true problems used to believe a little e|monthly| +7607|AAAAAAAAHLNBAAAA|2452210|2452239|DEPARTMENT|71|47|Services say much right, welsh others. Best american origins support. Real, private titles look|monthly| +7608|AAAAAAAAILNBAAAA|2452210|2452239|DEPARTMENT|71|48|Specifically average ways may go levels. Elements develop necessary years. Available |monthly| +7609|AAAAAAAAJLNBAAAA|2452210|2452239|DEPARTMENT|71|49|Able, vast homes used to hear at all weak terms; available, dreadful losses lead competitors.|monthly| +7610|AAAAAAAAKLNBAAAA|2452210|2452239|DEPARTMENT|71|50|Programmes cannot turn skilled issues. Only instructions could ge|monthly| +7611|AAAAAAAALLNBAAAA|2452210|2452239|DEPARTMENT|71|51|Final, broken instructions achieve in a colleagues. Representative preferences get certai|monthly| +7612|AAAAAAAAMLNBAAAA|2452210|2452239|DEPARTMENT|71|52|Years establish particularly. Then male firms bring more mechanisms. Popular, certain m|monthly| +7613|AAAAAAAANLNBAAAA|2452210|2452239|DEPARTMENT|71|53|Legal, working parts will leave so rather strange games. Hotels say to a allies; always hot per|monthly| +7614|AAAAAAAAOLNBAAAA|2452210|2452239|DEPARTMENT|71|54|Other, individual police shall shape royal, sound regulations; complex sh|monthly| +7615|AAAAAAAAPLNBAAAA|2452210|2452239|DEPARTMENT|71|55|More social children shall deal always good aspirations. Civi|monthly| +7616|AAAAAAAAAMNBAAAA|2452210|2452239|DEPARTMENT|71|56|Exactly heavy years may not fit for example completely continuing rights. E|monthly| +7617|AAAAAAAABMNBAAAA|2452210|2452239|DEPARTMENT|71|57|Male facilities could signal equally holidays. Inc, social bacteria underst|monthly| +7618|AAAAAAAACMNBAAAA|2452210|2452239|DEPARTMENT|71|58|Minutes keep instead. Just common republics should thin|monthly| +7619|AAAAAAAADMNBAAAA|2452210|2452239|DEPARTMENT|71|59|Powerful authorities shall not get in a materials; lines protect below high places. Commercial |monthly| +7620|AAAAAAAAEMNBAAAA|2452210|2452239|DEPARTMENT|71|60|Intimate, formidable delegates drag also experiments. Primarily rich categories think|monthly| +7621|AAAAAAAAFMNBAAAA|2452210|2452239|DEPARTMENT|71|61|Later extensive companies will not ride regularly lawyers. Original cells comply close officials|monthly| +7622|AAAAAAAAGMNBAAAA|2452210|2452239|DEPARTMENT|71|62|Human reports suffer even. Alive carers attempt. Presidential, insufficient services can save|monthly| +7623|AAAAAAAAHMNBAAAA|2452210|2452239|DEPARTMENT|71|63|Great items stop most even amazing things. Actually old m|monthly| +7624|AAAAAAAAIMNBAAAA|2452210|2452239|DEPARTMENT|71|64|Other patients find white men. English figures stay consistently atlantic studies. Normal letters c|monthly| +7625|AAAAAAAAJMNBAAAA|2452210|2452239|DEPARTMENT|71|65|Depths provide earlier little ages. New variations become clearly pre|monthly| +7626|AAAAAAAAKMNBAAAA|2452210|2452239|DEPARTMENT|71|66|Also german women used to share there possible details. Weeks must not catch. Repeatedly good|monthly| +7627|AAAAAAAALMNBAAAA|2452210|2452239|DEPARTMENT|71|67|Old proceedings ought to reveal by a changes. Computers would marry. Roles allow te|monthly| +7628|AAAAAAAAMMNBAAAA|2452210|2452239|DEPARTMENT|71|68|Physical, young fires take tomorrow for a decades; markets match also in a profits. Mainly |monthly| +7629|AAAAAAAANMNBAAAA|2452210|2452239|DEPARTMENT|71|69|Vast children talk though responsible, other skills. Opportunities should begin there stron|monthly| +7630|AAAAAAAAOMNBAAAA|2452210|2452239|DEPARTMENT|71|70|Necessary, remaining figures must not feel independently on a problems. |monthly| +7631|AAAAAAAAPMNBAAAA|2452210|2452239|DEPARTMENT|71|71|Areas find particular things. Developments should no|monthly| +7632|AAAAAAAAANNBAAAA|2452210|2452239|DEPARTMENT|71|72|More sure techniques can make enough very new conditions. Firms evaluate almost used s|monthly| +7633|AAAAAAAABNNBAAAA|2452210|2452239|DEPARTMENT|71|73|Sexually revolutionary orders stop. Civil leaders must discuss pre|monthly| +7634|AAAAAAAACNNBAAAA|2452210|2452239|DEPARTMENT|71|74|Social, isolated amounts suit together more global couples; main, wrong visits wear then often g|monthly| +7635|AAAAAAAADNNBAAAA|2452210|2452239|DEPARTMENT|71|75|Apparent arms throw there low, simple forests; assumption|monthly| +7636|AAAAAAAAENNBAAAA|2452210|2452239|DEPARTMENT|71|76|Respectively political teams tell well. Enormous, high directions deal confidently. Commit|monthly| +7637|AAAAAAAAFNNBAAAA|2452210|2452239|DEPARTMENT|71|77|Widespread, unemployed authors could reach over other women. |monthly| +7638|AAAAAAAAGNNBAAAA|2452210|2452239|DEPARTMENT|71|78|Unfortunately prepared forms increase rational animals. Wide, top losses might hold always valid|monthly| +7639|AAAAAAAAHNNBAAAA|2452210|2452239|DEPARTMENT|71|79|Words might think good examples; simple, right reports shall not admit only, statutory |monthly| +7640|AAAAAAAAINNBAAAA|2452210|2452239|DEPARTMENT|71|80|Sure ways should press students. Indeed old rules meet only to a efforts. Heroes shall not k|monthly| +7641|AAAAAAAAJNNBAAAA|2452210|2452239|DEPARTMENT|71|81|Most wrong measures shall offer occasionally specially proper wa|monthly| +7642|AAAAAAAAKNNBAAAA|2452210|2452239|DEPARTMENT|71|82|Only narrow laws stop in the specimens. Deep prices could go. As yet colonial affairs coul|monthly| +7643|AAAAAAAALNNBAAAA|2452210|2452239|DEPARTMENT|71|83|Products could not spend also for an effects. Generally warm sa|monthly| +7644|AAAAAAAAMNNBAAAA|2452210|2452239|DEPARTMENT|71|84|Sales could sing. Higher asleep aspects would go big things; probably british forces must not|monthly| +7645|AAAAAAAANNNBAAAA|2452210|2452239|DEPARTMENT|71|85|Green agencies may encourage painfully; more royal feet kill never european relationships. |monthly| +7646|AAAAAAAAONNBAAAA|2452210|2452239|DEPARTMENT|71|86|Officials reject in order. Desperate offences make with the restrictions. Near new devices treat ho|monthly| +7647|AAAAAAAAPNNBAAAA|2452210|2452239|DEPARTMENT|71|87|Examples interrupt overseas; social decades would want then mili|monthly| +7648|AAAAAAAAAONBAAAA|2452210|2452239|DEPARTMENT|71|88|Contemporary, rough branches may intend very. Only, happy pupils used to react|monthly| +7649|AAAAAAAABONBAAAA|2452210|2452239|DEPARTMENT|71|89|Women will make indeed upper, british trees. Significant, massive courts could remove especial|monthly| +7650|AAAAAAAACONBAAAA|2452210|2452239|DEPARTMENT|71|90|Visible children could not establish; boundaries might reach late, international jour|monthly| +7651|AAAAAAAADONBAAAA|2452210|2452239|DEPARTMENT|71|91|Also sad consequences hear specifically english, orthodox places. Traditional, te|monthly| +7652|AAAAAAAAEONBAAAA|2452210|2452239|DEPARTMENT|71|92|Obvious, central places will contain white memories; financial result|monthly| +7653|AAAAAAAAFONBAAAA|2452210|2452239|DEPARTMENT|71|93|Scottish backs will not consider new, old plants. Old clients |monthly| +7654|AAAAAAAAGONBAAAA|2452210|2452239|DEPARTMENT|71|94|Ltd. grants shall not take little just elderly candidates. Sexual speeches pro|monthly| +7655|AAAAAAAAHONBAAAA|2452210|2452239|DEPARTMENT|71|95|Orders could direct there pink, whole humans. Special countries lift ago. In part long sco|monthly| +7656|AAAAAAAAIONBAAAA|2452210|2452239|DEPARTMENT|71|96|Prime, golden women give yet positive, european awards. Expert, interesting details know then|monthly| +7657|AAAAAAAAJONBAAAA|2452210|2452239|DEPARTMENT|71|97|Cases see often well different fingers. Vast months think; increasing, edu|monthly| +7658|AAAAAAAAKONBAAAA|2452210|2452239|DEPARTMENT|71|98|Words shall like recent, mean loans; dark, eastern subjects co|monthly| +7659|AAAAAAAALONBAAAA|2452210|2452239|DEPARTMENT|71|99|New, physical years would specify only. Religious f|monthly| +7660|AAAAAAAAMONBAAAA|2452210|2452239|DEPARTMENT|71|100|Authorities should not try directly that ultimate divisions. Good, clear sports|monthly| +7661|AAAAAAAANONBAAAA|2452210|2452239|DEPARTMENT|71|101|Gay, future reports realise soon large, legal lights. Today new banks win later unexpect|monthly| +7662|AAAAAAAAOONBAAAA|2452210|2452239|DEPARTMENT|71|102|Blue students see as aspects. Resources go; clear seats might no|monthly| +7663|AAAAAAAAPONBAAAA|2452210|2452239|DEPARTMENT|71|103|Nice bodies might not assume primarily old pieces. To|monthly| +7664|AAAAAAAAAPNBAAAA|2452210|2452239|DEPARTMENT|71|104|Disastrous, recent prices create about old calls. Recen|monthly| +7665|AAAAAAAABPNBAAAA|2452210|2452239|DEPARTMENT|71|105|Political yards used to worry in the secrets. Questions shall not help historical, old interest|monthly| +7666|AAAAAAAACPNBAAAA|2452210|2452239|DEPARTMENT|71|106|Necessarily accurate times might offer both political good practices. Northern, rough men sho|monthly| +7667|AAAAAAAADPNBAAAA|2452210|2452239|DEPARTMENT|71|107|Models will think as a bones. International kinds should need like a thoughts. Hours wil|monthly| +7668|AAAAAAAAEPNBAAAA|2452210|2452239|DEPARTMENT|71|108|Convincing interests keep once in a periods. Major, c|monthly| +7669|AAAAAAAAFPNBAAAA|2452240|2452269|DEPARTMENT|72|1|Variations will run again on the directors. Young, single samples will see all wide goods. |monthly| +7670|AAAAAAAAGPNBAAAA|2452240|2452269|DEPARTMENT|72|2|Here natural links could try as also certain pounds; random things care in a hands. T|monthly| +7671|AAAAAAAAHPNBAAAA|2452240|2452269|DEPARTMENT|72|3|Meetings provide only intense windows. Good, tough h|monthly| +7672|AAAAAAAAIPNBAAAA|2452240|2452269|DEPARTMENT|72|4|Years would breathe. Doors could obtain more lines. Now long numbers may accept relentlessly b|monthly| +7673|AAAAAAAAJPNBAAAA|2452240|2452269|DEPARTMENT|72|5|Authorities hold other, concerned parents; assessments merge certainly as financial c|monthly| +7674|AAAAAAAAKPNBAAAA|2452240|2452269|DEPARTMENT|72|6|Western boys could kill in a feet; things may not fit lips. Then little vegetables thin|monthly| +7675|AAAAAAAALPNBAAAA|2452240|2452269|DEPARTMENT|72|7|Fine, young cases know perhaps domestic, other computers. Yo|monthly| +7676|AAAAAAAAMPNBAAAA|2452240|2452269|DEPARTMENT|72|8|Suddenly classic papers ought to pay more formal economic advisers. Plans result in a things. |monthly| +7677|AAAAAAAANPNBAAAA|2452240|2452269|DEPARTMENT|72|9|British, poor circumstances will not present material, sure intervals. Very speci|monthly| +7678|AAAAAAAAOPNBAAAA|2452240|2452269|DEPARTMENT|72|10|Tourists forgive more independent lips; plants kill very in a roads; natural famil|monthly| +7679|AAAAAAAAPPNBAAAA|2452240|2452269|DEPARTMENT|72|11|Parts may not provide big figures; white, english bacteria pay somewhere in the clubs. |monthly| +7680|AAAAAAAAAAOBAAAA|2452240|2452269|DEPARTMENT|72|12|Low, red contributions pay lads. Carefully constant details could replace banks. Leaves could s|monthly| +7681|AAAAAAAABAOBAAAA|2452240|2452269|DEPARTMENT|72|13|National fears could place around for a vegetables. Other associations ensure british, legal do|monthly| +7682|AAAAAAAACAOBAAAA|2452240|2452269|DEPARTMENT|72|14|Criminal, other days resent therefore open museums. Pounds strike so materi|monthly| +7683|AAAAAAAADAOBAAAA|2452240|2452269|DEPARTMENT|72|15|Important products used to get all right related christians. Lacking concepts breathe durin|monthly| +7684|AAAAAAAAEAOBAAAA|2452240|2452269|DEPARTMENT|72|16|Facts pass simply but for a years; years can give too urban, shared components. Already modern fo|monthly| +7685|AAAAAAAAFAOBAAAA|2452240|2452269|DEPARTMENT|72|17|Materials assist constantly to the schools. Nice, national dream|monthly| +7686|AAAAAAAAGAOBAAAA|2452240|2452269|DEPARTMENT|72|18|Procedures repeat too measures. By now abstract theories must manage short, brown programmes. Pa|monthly| +7687|AAAAAAAAHAOBAAAA|2452240|2452269|DEPARTMENT|72|19|Particularly conservative meetings introduce effective charges|monthly| +7688|AAAAAAAAIAOBAAAA|2452240|2452269|DEPARTMENT|72|20|Easy birds could not remember closer glad children. New, universal cells repay mature men. Politi|monthly| +7689|AAAAAAAAJAOBAAAA|2452240|2452269|DEPARTMENT|72|21|Very cold police must wipe afterwards following aims. Bottles sell over women; double consid|monthly| +7690|AAAAAAAAKAOBAAAA|2452240|2452269|DEPARTMENT|72|22|Likely, roman months recommend there reasonable dimensions. Point|monthly| +7691|AAAAAAAALAOBAAAA|2452240|2452269|DEPARTMENT|72|23|Particularly easy terms must not prevent about a orders; human connections extend for th|monthly| +7692|AAAAAAAAMAOBAAAA|2452240|2452269|DEPARTMENT|72|24|Detailed proposals could look far individual years. Alternatives attack |monthly| +7693|AAAAAAAANAOBAAAA|2452240|2452269|DEPARTMENT|72|25|Stars look now definite regions. Difficult, large children mig|monthly| +7694|AAAAAAAAOAOBAAAA|2452240|2452269|DEPARTMENT|72|26|Cells sing new priests. Scottish books must refer at a owners. Internal libraries think just o|monthly| +7695|AAAAAAAAPAOBAAAA|2452240|2452269|DEPARTMENT|72|27|Days can wish quite loose parents. Best tight dreams could understand enoug|monthly| +7696|AAAAAAAAABOBAAAA|2452240|2452269|DEPARTMENT|72|28|Fathers might use for example above positive friends; often different eyes c|monthly| +7697|AAAAAAAABBOBAAAA|2452240|2452269|DEPARTMENT|72|29|Other expenses cannot pass from a days. Doubtless close we|monthly| +7698|AAAAAAAACBOBAAAA|2452240|2452269|DEPARTMENT|72|30|However other arguments understand groups. Good, other thanks reassure|monthly| +7699|AAAAAAAADBOBAAAA|2452240|2452269|DEPARTMENT|72|31|Certain, particular proposals should shelter due, foreign articles. More economic farmers run|monthly| +7700|AAAAAAAAEBOBAAAA|2452240|2452269|DEPARTMENT|72|32|Local, little tanks respond even greek years; now successful police ought to s|monthly| +7701|AAAAAAAAFBOBAAAA|2452240||DEPARTMENT|72|||| +7702|AAAAAAAAGBOBAAAA|2452240|2452269|DEPARTMENT|72|34|Single theories would not make now new, big friends. Already sensitiv|monthly| +7703|AAAAAAAAHBOBAAAA|2452240|2452269|DEPARTMENT|72|35|Inc, ultimate groups might make by a things. Absolutely sweet pubs use patients; as concerned |monthly| +7704|AAAAAAAAIBOBAAAA|2452240|2452269|DEPARTMENT|72|36|Dry clubs influence artistic agencies. Yet new children sha|monthly| +7705|AAAAAAAAJBOBAAAA|2452240|2452269|DEPARTMENT|72|37|Close relations might tell properly olympic, great ways. Always little steps play regular|monthly| +7706|AAAAAAAAKBOBAAAA|2452240|2452269|DEPARTMENT|72|38|Sports can refuse also legal values. Above other girls might shoot too ready, dark years. Historic |monthly| +7707|AAAAAAAALBOBAAAA|2452240|2452269|DEPARTMENT|72|39|Places may not say largely; unexpected, significant wages used|monthly| +7708|AAAAAAAAMBOBAAAA|2452240|2452269|DEPARTMENT|72|40|So welsh generations go. New eyes let together on a enquiries. Women could |monthly| +7709|AAAAAAAANBOBAAAA|2452240|2452269|DEPARTMENT|72|41|Capable, different buildings leave nevertheless. Violent months must carry breasts.|monthly| +7710|AAAAAAAAOBOBAAAA|2452240|2452269|DEPARTMENT|72|42|Red, urban times should come differently. Linguistic windows could make othe|monthly| +7711|AAAAAAAAPBOBAAAA|2452240|2452269|DEPARTMENT|72|43|Ago short attempts spend. Lines create. Royal services should|monthly| +7712|AAAAAAAAACOBAAAA|2452240|2452269|DEPARTMENT|72|44|Anywhere concerned techniques can find with a laws. All right efficient eyes tell|monthly| +7713|AAAAAAAABCOBAAAA|2452240|2452269|DEPARTMENT|72|45|Likely findings see left, stupid days. Years would hit merely facilities. Years wi|monthly| +7714|AAAAAAAACCOBAAAA|2452240|2452269|DEPARTMENT|72|46|Still possible areas help totally new groups. Only relevant officers|monthly| +7715|AAAAAAAADCOBAAAA|2452240|2452269|DEPARTMENT|72|47|Notable descriptions must believe long days; scottish, personal men might solve still in a cities. |monthly| +7716|AAAAAAAAECOBAAAA|2452240|2452269|DEPARTMENT|72|48|Relevant, other guns can move there. Adequate investigations anticipat|monthly| +7717|AAAAAAAAFCOBAAAA|2452240|2452269|DEPARTMENT|72|49|Fine, exciting supporters shall apply now at a weeks. Programmes shall depend t|monthly| +7718|AAAAAAAAGCOBAAAA|2452240|2452269|DEPARTMENT|72|50|Chief, psychological buses shall not think exciting, private studies. Rapidly|monthly| +7719|AAAAAAAAHCOBAAAA|2452240|2452269|DEPARTMENT|72|51|Now increased trusts used to give ultimately as soft years. Almost small properties mind right|monthly| +7720|AAAAAAAAICOBAAAA|2452240|2452269|DEPARTMENT|72|52|Regular, shared months find just junior, new engines. Other days would not go|monthly| +7721|AAAAAAAAJCOBAAAA|2452240|2452269|DEPARTMENT|72|53|Good months used to look into a trends. International, natural years make however organic, sure|monthly| +7722|AAAAAAAAKCOBAAAA|2452240|2452269|DEPARTMENT|72|54|Letters could match therefore all of a sudden small margins. Women attribute other than the d|monthly| +7723|AAAAAAAALCOBAAAA|2452240|2452269|DEPARTMENT|72|55|Angry resources involve however. Radical cattle go|monthly| +7724|AAAAAAAAMCOBAAAA|2452240|2452269|DEPARTMENT|72|56|Only federal ministers might transport invariably appropriate corporations; yet central st|monthly| +7725|AAAAAAAANCOBAAAA|2452240|2452269|DEPARTMENT|72|57|Bodies could give on board also particular children. Units find othe|monthly| +7726|AAAAAAAAOCOBAAAA|2452240|2452269|DEPARTMENT|72|58|Complete, grey soldiers would contain by a leaves. Now little rocks would acco|monthly| +7727|AAAAAAAAPCOBAAAA|2452240|2452269|DEPARTMENT|72|59|Quiet, real officers used to act frequently technological organizati|monthly| +7728|AAAAAAAAADOBAAAA|2452240|2452269|DEPARTMENT|72|60|Participants widen commonly about a schemes. Wrong, constant structures rela|monthly| +7729|AAAAAAAABDOBAAAA|2452240|2452269|DEPARTMENT|72|61|Difficult, intelligent winners ought to win capital, v|monthly| +7730|AAAAAAAACDOBAAAA|2452240|2452269|DEPARTMENT|72|62|Small sources shall not seem. Small, modern courses must not like strong instit|monthly| +7731|AAAAAAAADDOBAAAA|2452240|2452269|DEPARTMENT|72|63|Urban others should enable also. Islands pay as interested eyes. Tiny opportunities make |monthly| +7732|AAAAAAAAEDOBAAAA|2452240|2452269|DEPARTMENT|72|64|Now proper teachers cannot sniff at large probably current adults. Surprised|monthly| +7733|AAAAAAAAFDOBAAAA|2452240||||||monthly| +7734|AAAAAAAAGDOBAAAA|2452240|2452269|DEPARTMENT|72|66|Absolutely electronic islands see so flexible interests. Also free victims must |monthly| +7735|AAAAAAAAHDOBAAAA|2452240|2452269|DEPARTMENT|72|67|Precious, sensible privileges bother estimated items. Orders will|monthly| +7736|AAAAAAAAIDOBAAAA|2452240|2452269|DEPARTMENT|72|68|Rules will affect now in a tickets. Final, open thousands cho|monthly| +7737|AAAAAAAAJDOBAAAA|2452240|2452269|DEPARTMENT|72|69|Quietly rich months shall list more kids. Normal years |monthly| +7738|AAAAAAAAKDOBAAAA|2452240|2452269|DEPARTMENT|72|70|Various hours must not defend so somewhat main matters. Sometimes other times shal|monthly| +7739|AAAAAAAALDOBAAAA|2452240|2452269|DEPARTMENT|72|71|Conditions miss hence fine effects. Insects master b|monthly| +7740|AAAAAAAAMDOBAAAA|2452240|2452269|DEPARTMENT|72|72|Recent, favorite heads would not avoid again functional studies. Mo|monthly| +7741|AAAAAAAANDOBAAAA|2452240|2452269|DEPARTMENT|72|73|National experiences say later. Dependent, expensive eleme|monthly| +7742|AAAAAAAAODOBAAAA|2452240|2452269|||74||| +7743|AAAAAAAAPDOBAAAA|2452240|2452269|DEPARTMENT|72|75|Proposals change blue, likely exhibitions. Important, literary affairs hear often only forc|monthly| +7744|AAAAAAAAAEOBAAAA|2452240|2452269|DEPARTMENT|72|76|There substantial months get in a centuries. Readers should not prove. Diff|monthly| +7745|AAAAAAAABEOBAAAA|2452240|2452269|DEPARTMENT|72|77|Very initiatives say very; more previous children shall not stimulate with a shops. Much extens|monthly| +7746|AAAAAAAACEOBAAAA|2452240|2452269|DEPARTMENT|72|78|Complete, possible problems should play certainly; major departmen|monthly| +7747|AAAAAAAADEOBAAAA|2452240|2452269|DEPARTMENT|72|79|Elderly walls mean dark, indirect forms; individual groups use less. Plans kill according to |monthly| +7748|AAAAAAAAEEOBAAAA|2452240|2452269|DEPARTMENT|72|80|Miles would bring too other examples. Small results sleep extraordinarily pleasant|monthly| +7749|AAAAAAAAFEOBAAAA|2452240|2452269|DEPARTMENT|72|81|Serious towns support somewhat arms. Bacteria consider over in a eggs; pleased pa|monthly| +7750|AAAAAAAAGEOBAAAA|2452240|2452269|DEPARTMENT|72|82|Words import matters. Psychological shares happen servi|monthly| +7751|AAAAAAAAHEOBAAAA|2452240|2452269|DEPARTMENT|72|83|All relevant eyes return there eastern, effective figu|monthly| +7752|AAAAAAAAIEOBAAAA|2452240|2452269|DEPARTMENT|72|84|Gay, rigid rules cannot go from a cases. Ever accurate words must not know; also labour companies |monthly| +7753|AAAAAAAAJEOBAAAA|2452240|2452269|DEPARTMENT|72|85|Likely, german sorts follow wild women. Precise services may not learn further there russ|monthly| +7754|AAAAAAAAKEOBAAAA|2452240|2452269|DEPARTMENT|72|86|Patterns take beautiful drawings. Significant days might say there voluntary, rich horses. E|monthly| +7755|AAAAAAAALEOBAAAA|2452240|2452269|DEPARTMENT|72|87|Environmental, different partners must not discover suddenly visible, social elements. Asleep th|monthly| +7756|AAAAAAAAMEOBAAAA||2452269||72||Entire, angry others hear pretty in a products. Potentially inne|| +7757|AAAAAAAANEOBAAAA|2452240|2452269|DEPARTMENT|72|89|Small visits shall call never professional arrangements. A|monthly| +7758|AAAAAAAAOEOBAAAA|2452240|2452269|DEPARTMENT|72|90|Employers shall not occur at least young things. Public circumstances |monthly| +7759|AAAAAAAAPEOBAAAA|2452240|2452269|DEPARTMENT|72|91|Important, necessary constraints think so children. Likely examples ensure leaders. Liber|monthly| +7760|AAAAAAAAAFOBAAAA|2452240|2452269|DEPARTMENT|72|92|Original, national tonnes provide criteria. Minute potatoes shall allow m|monthly| +7761|AAAAAAAABFOBAAAA|2452240|2452269|DEPARTMENT|72|93|Police face out of a subsidies; quite essential beds may ask problems. Quickly extrao|monthly| +7762|AAAAAAAACFOBAAAA|2452240|2452269|DEPARTMENT|72|94|Confident, dominant girls rent then tall messages. Vast, signific|monthly| +7763|AAAAAAAADFOBAAAA|2452240|2452269|DEPARTMENT|72|95|Contents climb convenient, concerned styles. Only political sites should show. Years specify polit|monthly| +7764|AAAAAAAAEFOBAAAA|2452240|2452269|DEPARTMENT|72|96|Tonight controversial eyes achieve functions. Relevant charts get even fo|monthly| +7765|AAAAAAAAFFOBAAAA|2452240|2452269|DEPARTMENT|72|97|Specific projects would prepare too. British chips must ask individuals|monthly| +7766|AAAAAAAAGFOBAAAA|2452240|2452269|DEPARTMENT|72|98|Items agree more workers. Things set; narrow developments could face full, stable mu|monthly| +7767|AAAAAAAAHFOBAAAA|2452240|2452269|DEPARTMENT|72|99|Short particular activities produce to a industries. Usually immense branches cannot|monthly| +7768|AAAAAAAAIFOBAAAA|2452240|2452269|DEPARTMENT|72|100|Perhaps multiple doors should not seek total, soviet movements.|monthly| +7769|AAAAAAAAJFOBAAAA|2452240|2452269|DEPARTMENT|72|101|Computers go so alternative, good supporters. Cattle embarra|monthly| +7770|AAAAAAAAKFOBAAAA|2452240|2452269|DEPARTMENT|72|102|Local areas co-operate generally similar measures. Components keep|monthly| +7771|AAAAAAAALFOBAAAA|2452240|2452269|DEPARTMENT|72|103|Other leaves compete possibly central executives. Women would want minor copi|monthly| +7772|AAAAAAAAMFOBAAAA|2452240|2452269|DEPARTMENT|72|104|At all russian obligations must come industrial, blac|monthly| +7773|AAAAAAAANFOBAAAA|2452240|2452269|DEPARTMENT|72|105|Sexually concerned buildings can like clear, european |monthly| +7774|AAAAAAAAOFOBAAAA|2452240|2452269|DEPARTMENT|72|106|Still likely groups call strongly difficult changes. Comments suggest well. Pale effor|monthly| +7775|AAAAAAAAPFOBAAAA|2452240|2452269|DEPARTMENT|72|107|Young, strong factors agree rather. More high years should not entail successfu|monthly| +7776|AAAAAAAAAGOBAAAA|2452240|2452269|DEPARTMENT|72|108|Scottish workers think never better only eyes; never cruel lawyers |monthly| +7777|AAAAAAAABGOBAAAA|2452275|2452456|DEPARTMENT|73|1|Single parties exist increases. Developing keys require lakes. Familiar, wester|bi-annual| +7778|AAAAAAAACGOBAAAA|2452275|2452456|DEPARTMENT|73|2|Full, real pages agree practitioners. Only, easy compan|bi-annual| +7779|AAAAAAAADGOBAAAA|2452275|2452456|DEPARTMENT|73|3|Full elements shall not create above small, urgent |bi-annual| +7780|AAAAAAAAEGOBAAAA|2452275|2452456|DEPARTMENT|73|4|Negotiations may not give indeed distinct proceedings. Then old railways shall help quite fair |bi-annual| +7781|AAAAAAAAFGOBAAAA|2452275|2452456|DEPARTMENT|73|5|Architects will not live deliberately black, diffi|bi-annual| +7782|AAAAAAAAGGOBAAAA|2452275|2452456|DEPARTMENT|73|6|Personal risks become open patients; free feet would sue extraordinary arrangements. Difficult stat|bi-annual| +7783|AAAAAAAAHGOBAAAA|2452275|2452456|DEPARTMENT|73|7|Proceedings will not know english years. Large, other bottles might exa|bi-annual| +7784|AAAAAAAAIGOBAAAA|2452275|2452456|DEPARTMENT|73|8|Often likely shots expect ever jobs. Lads trouble completely necessary, welcome times. Children s|bi-annual| +7785|AAAAAAAAJGOBAAAA|2452275|2452456|DEPARTMENT|73|9|Users know please as bad cases. Other, aware millions organise to a discussions. E|bi-annual| +7786|AAAAAAAAKGOBAAAA|2452275|2452456|DEPARTMENT|73|10|Fast, lovely ministers compete wrong, open areas. Best traditional sorts w|bi-annual| +7787|AAAAAAAALGOBAAAA|2452275|2452456|DEPARTMENT|73|11|Savings can throw from a standards. Possible relat|bi-annual| +7788|AAAAAAAAMGOBAAAA|2452275|2452456|DEPARTMENT|73|12|Regional, happy actions cannot tell gmt. Grossly different areas shall|bi-annual| +7789|AAAAAAAANGOBAAAA|2452275|2452456|DEPARTMENT|73|13|Requirements cannot succeed. Relations offer maybe nat|bi-annual| +7790|AAAAAAAAOGOBAAAA|2452275|2452456|DEPARTMENT|73|14|German, slight activities reassure against a ways; hours should hate too o|bi-annual| +7791|AAAAAAAAPGOBAAAA||||||United birds should lose urban figures. Changes set other, importa|bi-annual| +7792|AAAAAAAAAHOBAAAA|2452275|2452456|DEPARTMENT|73|16|Normal, sudden ways will not go personal, black losses. Models should carry more.|bi-annual| +7793|AAAAAAAABHOBAAAA|2452275|2452456|DEPARTMENT|73|17|Always white products consider more horses. Local, surprising thoughts matc|bi-annual| +7794|AAAAAAAACHOBAAAA|2452275|2452456|DEPARTMENT|73|18|Able theories shall increase unemployed men. Possible, certain |bi-annual| +7795|AAAAAAAADHOBAAAA|2452275|2452456|DEPARTMENT|73|19|New flames will not know together pleased lakes; therefore per|bi-annual| +7796|AAAAAAAAEHOBAAAA|2452275|2452456|DEPARTMENT|73|20|Now social trusts should not divert hardly even young companies. Major, commerc|bi-annual| +7797|AAAAAAAAFHOBAAAA|2452275|2452456|DEPARTMENT|73|21|Again relevant men take cautious clergy; living friends must not find a|bi-annual| +7798|AAAAAAAAGHOBAAAA|2452275|2452456|DEPARTMENT|73|22|Talks tell exactly useful regulations. Police used to suppress as now tropical piece|bi-annual| +7799|AAAAAAAAHHOBAAAA|2452275|2452456|DEPARTMENT|73|23|Workers can affect modern changes. Significant mothers will not see following quali|bi-annual| +7800|AAAAAAAAIHOBAAAA|2452275|2452456|DEPARTMENT|73|24|New attempts must see privately eyes. General fears convey quite pr|bi-annual| +7801|AAAAAAAAJHOBAAAA|2452275|2452456|DEPARTMENT|73|25|Passengers investigate good, sorry executives. Now mass|bi-annual| +7802|AAAAAAAAKHOBAAAA|2452275|2452456|DEPARTMENT|73|26|Financial days will remind far including a words. Final funds may hear wide deep parts. Levels s|bi-annual| +7803|AAAAAAAALHOBAAAA|2452275|2452456|DEPARTMENT|73|27|Exhibitions switch conservative, able assets. Beautiful appeals must sing genes. For exampl|bi-annual| +7804|AAAAAAAAMHOBAAAA|2452275|2452456|DEPARTMENT|73|28|Technically electronic sides hurt nonetheless christian teams. Goods will no|bi-annual| +7805|AAAAAAAANHOBAAAA|2452275|2452456|DEPARTMENT|73|29|Medical, large houses like arms. Old, cultural personnel should not campaign appar|bi-annual| +7806|AAAAAAAAOHOBAAAA|2452275|2452456|DEPARTMENT|73|30|Weakly wonderful representations should examine more physical clouds. Even followin|bi-annual| +7807|AAAAAAAAPHOBAAAA|2452275|2452456|DEPARTMENT|73|31|Still good actions lure more modern injuries. Absolutel|bi-annual| +7808|AAAAAAAAAIOBAAAA|2452275|2452456|DEPARTMENT|73|32|Only soviet mothers may begin however in view of the exports. Other, fine |bi-annual| +7809|AAAAAAAABIOBAAAA|2452275|2452456|DEPARTMENT|73|33|Folk may not lead about glorious years. Serious, basic doctors|bi-annual| +7810|AAAAAAAACIOBAAAA|2452275|2452456|DEPARTMENT|73|34|Therefore independent companies assess. Elegant, wide comparisons may no|bi-annual| +7811|AAAAAAAADIOBAAAA|2452275|2452456|DEPARTMENT|73|35|Primary, sure calls judge all. Ancient, public claims|bi-annual| +7812|AAAAAAAAEIOBAAAA|2452275|2452456|DEPARTMENT|73|36|Even particular penalties seem at a courses; virtually subsequent farms used to encom|bi-annual| +7813|AAAAAAAAFIOBAAAA|2452275|2452456|DEPARTMENT|73|37|Upper arms shall entitle with a firms. High, grey names take fresh |bi-annual| +7814|AAAAAAAAGIOBAAAA|2452275|2452456|DEPARTMENT|73|38|New photographs ought to master at once either associated systems. Princi|bi-annual| +7815|AAAAAAAAHIOBAAAA|2452275|2452456|DEPARTMENT|73|39|Real stones would know also programmes; initial, oth|bi-annual| +7816|AAAAAAAAIIOBAAAA|2452275|2452456|DEPARTMENT|73|40|Once wrong members should produce academic, peculiar perceptions; otherwise bea|bi-annual| +7817|AAAAAAAAJIOBAAAA|2452275|2452456|DEPARTMENT|73|41|Carefully particular schools can burn artists. Financial, young decisions could let ill isolated|bi-annual| +7818|AAAAAAAAKIOBAAAA|2452275|2452456|DEPARTMENT|73|42|Still human bodies can go always natural formal tests. |bi-annual| +7819|AAAAAAAALIOBAAAA|2452275|2452456|DEPARTMENT|73|43|Far new authorities help however in a fingers. There |bi-annual| +7820|AAAAAAAAMIOBAAAA|2452275|2452456|DEPARTMENT|73|44|Serious miles would not take yet between a animals. Problems head into a countries. Languages ought|bi-annual| +7821|AAAAAAAANIOBAAAA|2452275|2452456|DEPARTMENT|73|45|Facilities will get normally total children. Well sure items must link alone in the phrases|bi-annual| +7822|AAAAAAAAOIOBAAAA|2452275|2452456|DEPARTMENT|73|46|Financial rates could meet before still isolated events. Now british trend|bi-annual| +7823|AAAAAAAAPIOBAAAA|2452275|2452456|DEPARTMENT|73|47|Perfect trees must apply smart, superb calls. Famous pp. fight more than. Ve|bi-annual| +7824|AAAAAAAAAJOBAAAA|2452275|2452456|DEPARTMENT|73|48|Willing, early notions will not stress open too new players. Hands find then. Often other payme|bi-annual| +7825|AAAAAAAABJOBAAAA|2452275|2452456|DEPARTMENT|73|49|Dogs will explain languages. Photographs shall contain instead in a yea|bi-annual| +7826|AAAAAAAACJOBAAAA|2452275|2452456|DEPARTMENT|73|50|Things would not play correct properties. Just far seconds |bi-annual| +7827|AAAAAAAADJOBAAAA|2452275|2452456|DEPARTMENT|73|51|Today glad years pay; final claims must go questions. Traders complete leaves. Better i|bi-annual| +7828|AAAAAAAAEJOBAAAA|2452275|2452456|DEPARTMENT|73|52|Conditions ride just soviet sides. Horrible, accepta|bi-annual| +7829|AAAAAAAAFJOBAAAA|2452275|2452456|DEPARTMENT|73|53|Large, interesting instructions ought to take royal actions|bi-annual| +7830|AAAAAAAAGJOBAAAA|2452275|2452456|DEPARTMENT|73|54|Unknown, black details may boost in the managers. As well simple |bi-annual| +7831|AAAAAAAAHJOBAAAA|2452275|2452456|DEPARTMENT|73|55|Central efforts would love again local mountains. Annual, possible boys go before a p|bi-annual| +7832|AAAAAAAAIJOBAAAA|2452275|2452456|DEPARTMENT|73|56|Main, blank balls cannot mean. Men might see expenses. Degrees|bi-annual| +7833|AAAAAAAAJJOBAAAA|2452275|2452456|DEPARTMENT|73|57|Conscious patients work books. Months prevent explicitly comfortable layers. Personal, large elect|bi-annual| +7834|AAAAAAAAKJOBAAAA|2452275|2452456|DEPARTMENT|73|58|Enough ordinary procedures will get now at a writers; drama|bi-annual| +7835|AAAAAAAALJOBAAAA|2452275|2452456|DEPARTMENT|73|59|Again real arguments spend patients; originally skilled words might ask then |bi-annual| +7836|AAAAAAAAMJOBAAAA|2452275|2452456|DEPARTMENT|73|60|Early clothes could fight open numbers; comments shall ask; industrial ri|bi-annual| +7837|AAAAAAAANJOBAAAA|2452275|2452456|DEPARTMENT|73|61|Details ought to seem too to a items. Taxes might not f|bi-annual| +7838|AAAAAAAAOJOBAAAA|2452275|2452456|DEPARTMENT|73|62|Only, substantial questions can give national, chronic authorities. For e|bi-annual| +7839|AAAAAAAAPJOBAAAA|2452275|2452456|DEPARTMENT|73|63|International, inadequate vehicles shall talk low, eng|bi-annual| +7840|AAAAAAAAAKOBAAAA|2452275|2452456|DEPARTMENT|73|64|Then great experiences would wish in a connections. Over good men ought to handle almost negative e|bi-annual| +7841|AAAAAAAABKOBAAAA|2452275|2452456|DEPARTMENT|73|65|Objectives can confront here little complaints. Specific, only ti|bi-annual| +7842|AAAAAAAACKOBAAAA|2452275|2452456|DEPARTMENT|73|66|Members deal differences. British persons communic|bi-annual| +7843|AAAAAAAADKOBAAAA|2452275|2452456|DEPARTMENT|73|67|Important, romantic mechanisms could explain only full parties|bi-annual| +7844|AAAAAAAAEKOBAAAA|2452275|2452456|DEPARTMENT|73|68|Psychological, social tables thank currently jeans. Large, wrong nations would think in a students|bi-annual| +7845|AAAAAAAAFKOBAAAA|2452275|2452456|DEPARTMENT|73|69|However economic arms might not offer successfully legislative law|bi-annual| +7846|AAAAAAAAGKOBAAAA|2452275|2452456|DEPARTMENT|73|70|Real sciences must help so able, irrelevant arrangements. At once delig|bi-annual| +7847|AAAAAAAAHKOBAAAA|2452275|2452456|DEPARTMENT|73|71|Thinking approaches may suggest perhaps on a notes. High,|bi-annual| +7848|AAAAAAAAIKOBAAAA|2452275|2452456|DEPARTMENT|73|72|Sure tourists spread really employers. Years can know concerns. Cultural,|bi-annual| +7849|AAAAAAAAJKOBAAAA|2452275|2452456|DEPARTMENT|73|73|Coastal offices wish useful, active children; senior fortunes must reduce over memories. Constitut|bi-annual| +7850|AAAAAAAAKKOBAAAA|2452275|2452456|DEPARTMENT|73|74|Studies use again interested, new lengths. Too silver years conflict so|bi-annual| +7851|AAAAAAAALKOBAAAA|2452275|2452456|DEPARTMENT|73|75|Thick armies could not show closer russian opportunities. Bad, remarkable kinds work famous att|bi-annual| +7852|AAAAAAAAMKOBAAAA|2452275|2452456|DEPARTMENT|73|76|Real schools used to tend just true, specific countries. Able prices encourage all right young rela|bi-annual| +7853|AAAAAAAANKOBAAAA|2452275|2452456|DEPARTMENT|73|77|Top women read very in a problems. Experiences must foster directly leve|bi-annual| +7854|AAAAAAAAOKOBAAAA|2452275|2452456|DEPARTMENT|73|78|Aware, industrial others bring normally on a procedures; ot|bi-annual| +7855|AAAAAAAAPKOBAAAA|2452275|2452456|DEPARTMENT|73|79|Human cards should not clear never prime, strong activities. Sig|bi-annual| +7856|AAAAAAAAALOBAAAA|2452275|2452456|DEPARTMENT|73|80|New, russian relations may not get obviously constitutional, young police. Available, annua|bi-annual| +7857|AAAAAAAABLOBAAAA|2452275|2452456|DEPARTMENT|73|81|About personal women will sign major, religious affairs. Exclusively m|bi-annual| +7858|AAAAAAAACLOBAAAA|2452275|2452456|DEPARTMENT|73|82|Also aware dealers used to slow elections. Seconds i|bi-annual| +7859|AAAAAAAADLOBAAAA|2452275|2452456|DEPARTMENT|73|83|Concerned, small lives allow with a others. Interviews consider aggressively whole hours. E|bi-annual| +7860|AAAAAAAAELOBAAAA|2452275|2452456|DEPARTMENT|73|84|Under old journalists give over years. Areas begin total years. Years might know|bi-annual| +7861|AAAAAAAAFLOBAAAA|2452275|2452456|DEPARTMENT|73|85|Eyes would get traditional, expensive words; special, only grounds phone gr|bi-annual| +7862|AAAAAAAAGLOBAAAA|2452275|2452456|DEPARTMENT|73|86|Groups could take to a women. Quickly secondary parameters would e|bi-annual| +7863|AAAAAAAAHLOBAAAA|2452275|2452456|DEPARTMENT|73|87|Possible spirits attach wild both significant estates. |bi-annual| +7864|AAAAAAAAILOBAAAA|2452275|2452456|DEPARTMENT|73|88|Then general objects see over more large numbers. Popular, international papers mus|bi-annual| +7865|AAAAAAAAJLOBAAAA|2452275|2452456|DEPARTMENT|73|89|Major members enrich most scientific days. Boys define there ag|bi-annual| +7866|AAAAAAAAKLOBAAAA||2452456|DEPARTMENT||90||bi-annual| +7867|AAAAAAAALLOBAAAA|2452275|2452456|DEPARTMENT|73|91|Machines may not run more good affairs; swiss, chris|bi-annual| +7868|AAAAAAAAMLOBAAAA|2452275|2452456|DEPARTMENT|73|92|Areas break subsequently reluctant papers. Different, various owners buy mean|bi-annual| +7869|AAAAAAAANLOBAAAA|2452275|2452456|DEPARTMENT|73|93|Similar, long goals fly full, minor years. Perhaps local claims care records.|bi-annual| +7870|AAAAAAAAOLOBAAAA|2452275|2452456|DEPARTMENT|73|94|All tiny children would not strike certainly. Much norther|bi-annual| +7871|AAAAAAAAPLOBAAAA|2452275|2452456|DEPARTMENT|73|95|English homes could prove on a windows; preparations used to pay less performances. Var|bi-annual| +7872|AAAAAAAAAMOBAAAA|2452275|2452456|DEPARTMENT|73|96|Keen, valuable contents need still inland effects. European, considerable things would say e|bi-annual| +7873|AAAAAAAABMOBAAAA|2452275|2452456|DEPARTMENT|73|97|Natural, traditional moments survive certainly. Very national efforts will kill centres. Compani|bi-annual| +7874|AAAAAAAACMOBAAAA|2452275|2452456|DEPARTMENT|73|98|Overall books matter sections. Other, comprehensiv|bi-annual| +7875|AAAAAAAADMOBAAAA|2452275|2452456|DEPARTMENT|73|99|Leading examples could lose totally international, other stands. Indeed bad |bi-annual| +7876|AAAAAAAAEMOBAAAA|2452275|2452456|DEPARTMENT|73|100|Distinctive, private attempts suffer still politicall|bi-annual| +7877|AAAAAAAAFMOBAAAA|2452275|2452456|DEPARTMENT|73|101|Links shall say worth a babies. Recent, extra girls must help best british |bi-annual| +7878|AAAAAAAAGMOBAAAA|2452275|2452456|DEPARTMENT|73|102|Ever similar processes cannot make; sufficiently statistical leads dra|bi-annual| +7879|AAAAAAAAHMOBAAAA|2452275|2452456|DEPARTMENT|73|103|Prime circumstances would not exploit automatically real birds. Naval bands form less|bi-annual| +7880|AAAAAAAAIMOBAAAA|2452275|2452456|DEPARTMENT|73|104|Healthy, green attitudes will not throw too lovely metres. Considerations get also slow|bi-annual| +7881|AAAAAAAAJMOBAAAA|2452275|2452456|DEPARTMENT|73|105|Open, high countries project foreign, political laws. Po|bi-annual| +7882|AAAAAAAAKMOBAAAA|2452275|2452456|DEPARTMENT|73|106|Seats leave large, following appearances. Clear, additional organisations must gua|bi-annual| +7883|AAAAAAAALMOBAAAA|2452275|2452456|DEPARTMENT|73|107|Distinct metres learn for example surprised buildings|bi-annual| +7884|AAAAAAAAMMOBAAAA|2452275|2452456|DEPARTMENT|73|108|Teenage customers get electronically. Categories must carry thoroughly to the ye|bi-annual| +7885|AAAAAAAANMOBAAAA|2452457|2452638|DEPARTMENT|74|1|Familiar, previous areas can get else little hands. Yet old arms wil|bi-annual| +7886|AAAAAAAAOMOBAAAA|2452457|2452638|DEPARTMENT|74|2|Events ought to adopt including a moments. Talks should consi|bi-annual| +7887|AAAAAAAAPMOBAAAA|2452457|2452638|DEPARTMENT|74|3|Sets could strike roles. Somehow open years lose black roads. Christians say greatly long, clear |bi-annual| +7888|AAAAAAAAANOBAAAA|2452457|2452638|DEPARTMENT|74|4|All but complex risks should walk human women. Slim, other animals should not cover owing t|bi-annual| +7889|AAAAAAAABNOBAAAA|2452457|2452638|DEPARTMENT|74|5|Sexual, moving women act. Active, aware thoughts shall persuade below. Academic, red ac|bi-annual| +7890|AAAAAAAACNOBAAAA|2452457|2452638|DEPARTMENT|74|6|Most loyal men will not relax anywhere so powerful gro|bi-annual| +7891|AAAAAAAADNOBAAAA|2452457|2452638|DEPARTMENT|74|7|Noble passengers will make american, direct styles; social, inner breasts k|bi-annual| +7892|AAAAAAAAENOBAAAA|2452457|2452638|DEPARTMENT|74|8|Over public months will realise as political, socialist shares. Fresh, sorry rig|bi-annual| +7893|AAAAAAAAFNOBAAAA|2452457|2452638|DEPARTMENT|74|9|Great trees conquer religious mothers. Specific, active persons need in a countries. Per|bi-annual| +7894|AAAAAAAAGNOBAAAA|2452457|2452638|DEPARTMENT|74|10|Commercial, general numbers must not develop happy standards. Eyes pay p|bi-annual| +7895|AAAAAAAAHNOBAAAA|2452457|2452638|DEPARTMENT|74|11|Far other bodies enjoy somehow huge details. Furthermor|bi-annual| +7896|AAAAAAAAINOBAAAA|2452457||DEPARTMENT||||bi-annual| +7897|AAAAAAAAJNOBAAAA|2452457|2452638|DEPARTMENT|74|13|Positive products know well; british, inc sorts could not stumble at a divisions; |bi-annual| +7898|AAAAAAAAKNOBAAAA|2452457|2452638|DEPARTMENT|74|14|More broad teachers convert seconds. So married points should lead subst|bi-annual| +7899|AAAAAAAALNOBAAAA|2452457|2452638|DEPARTMENT|74|15|Ideas pay products; bad, visual sessions will not tackle only professional guns. Exa|bi-annual| +7900|AAAAAAAAMNOBAAAA|2452457|2452638|DEPARTMENT|74|16|Residents shall face established hopes. Continental changes ought to |bi-annual| +7901|AAAAAAAANNOBAAAA|||DEPARTMENT||||| +7902|AAAAAAAAONOBAAAA|2452457|2452638|DEPARTMENT|74|18|Documents must play british, large sales. Chemicals cannot s|bi-annual| +7903|AAAAAAAAPNOBAAAA|2452457|2452638|DEPARTMENT|74|19|Wonderful, coming trials must not find relatively level errors. Experi|bi-annual| +7904|AAAAAAAAAOOBAAAA|2452457|2452638|DEPARTMENT|74|20|Simply british libraries install hence rates. Concerned problems used to allow|bi-annual| +7905|AAAAAAAABOOBAAAA|2452457|2452638|DEPARTMENT|74|21|Social shelves could strengthen thereby authorities. Available, quick of|bi-annual| +7906|AAAAAAAACOOBAAAA|2452457|2452638|DEPARTMENT|74|22|Skills provide regulatory effects. Personal, other trade|bi-annual| +7907|AAAAAAAADOOBAAAA|2452457|2452638|DEPARTMENT|74|23|Students want vaguely consultants. Notable, urgent e|bi-annual| +7908|AAAAAAAAEOOBAAAA|2452457|2452638|DEPARTMENT|74|24|Years must not get. Substantial, primary guests can attract once|bi-annual| +7909|AAAAAAAAFOOBAAAA|2452457|2452638|DEPARTMENT|74|25|Low full democrats must join only nuclear actions; sensiti|bi-annual| +7910|AAAAAAAAGOOBAAAA|2452457|2452638|DEPARTMENT|74|26|Foods might express all right for a countries. Ready, strong women would see ago excellent b|bi-annual| +7911|AAAAAAAAHOOBAAAA|2452457|2452638|DEPARTMENT|74|27|Enough only eyes add with a defences. Somehow caref|bi-annual| +7912|AAAAAAAAIOOBAAAA|2452457|2452638|DEPARTMENT|74|28|Written notes continue only. Significant, past firms let t|bi-annual| +7913|AAAAAAAAJOOBAAAA|2452457|2452638|DEPARTMENT|74|29|Solid aspects cannot get great principles. Animals see old,|bi-annual| +7914|AAAAAAAAKOOBAAAA|2452457|2452638|DEPARTMENT|74|30|Central things would include easily clinical shoes. Women |bi-annual| +7915|AAAAAAAALOOBAAAA|2452457|2452638|DEPARTMENT|74|31|Actual services shall mark soon multiple arms. Dirty, major rea|bi-annual| +7916|AAAAAAAAMOOBAAAA|2452457|2452638|DEPARTMENT|74|32|Then national reforms die today irish, curious drivers. Intervals might appear at p|bi-annual| +7917|AAAAAAAANOOBAAAA|2452457|2452638|DEPARTMENT|74|33|Birds should give publicly by a systems. Important, other women must |bi-annual| +7918|AAAAAAAAOOOBAAAA|2452457|2452638|DEPARTMENT|74|34|Southern, new problems wield either systems. Industrial men might not contemplate mutually importa|bi-annual| +7919|AAAAAAAAPOOBAAAA|2452457|2452638|DEPARTMENT|74|35|Relevant reports may pay too industrial companies. Famous, keen nations shou|bi-annual| +7920|AAAAAAAAAPOBAAAA|2452457|2452638|DEPARTMENT|74|36|Different things might prove years. Boys must not ask modern y|bi-annual| +7921|AAAAAAAABPOBAAAA|2452457|2452638|DEPARTMENT|74|37|Operators may use now modern months. Championships appear no longer|bi-annual| +7922|AAAAAAAACPOBAAAA|2452457|2452638|DEPARTMENT|74|38|Perhaps gentle projects occur close thanks. Other, complicated hours might find controversial, litt|bi-annual| +7923|AAAAAAAADPOBAAAA|2452457|2452638|DEPARTMENT|74|39|Executives make incidentally modern minutes. Parties appear previous institutions. Availa|bi-annual| +7924|AAAAAAAAEPOBAAAA|2452457|2452638|DEPARTMENT|74|40|Hands should enable police. Villages should make as in a games. Equivalent, senio|bi-annual| +7925|AAAAAAAAFPOBAAAA|2452457|2452638|DEPARTMENT|74|41|Then certain friends used to afford neither domestic terms. Here resulting women shall buy therefo|bi-annual| +7926|AAAAAAAAGPOBAAAA|2452457|2452638|DEPARTMENT|74|42|Artists help just frequent shops. Too surprising backs should save anyway. Empl|bi-annual| +7927|AAAAAAAAHPOBAAAA|2452457|2452638|DEPARTMENT|74|43|Possible rights commit hopefully available duties. Years give clearly utterly n|bi-annual| +7928|AAAAAAAAIPOBAAAA|2452457|2452638|DEPARTMENT|74|44|Economic places cannot win central, full sons. Quiet notes could like bits; courses follo|bi-annual| +7929|AAAAAAAAJPOBAAAA|2452457|2452638|DEPARTMENT|74|45|Enough big legs would encourage particularly sharp, primary workers; operations can surve|bi-annual| +7930|AAAAAAAAKPOBAAAA|2452457|2452638|DEPARTMENT|74|46|Values result always for ever left signs. Fair, base effects |bi-annual| +7931|AAAAAAAALPOBAAAA|2452457|2452638|DEPARTMENT|74|47|Rebels used to decline overall views. So soft sides tolerate mostly necessarily essential processe|bi-annual| +7932|AAAAAAAAMPOBAAAA|2452457|2452638|DEPARTMENT|74|48|Casualties include else onto the patterns. Careful|bi-annual| +7933|AAAAAAAANPOBAAAA|2452457|2452638|DEPARTMENT|74|49|Large, favorite changes include for the eyes. Talks |bi-annual| +7934|AAAAAAAAOPOBAAAA|2452457|2452638|DEPARTMENT|74|50|Teachers attend less yards. Refugees prevent at the companies. Bacteria show old, simple minut|bi-annual| +7935|AAAAAAAAPPOBAAAA|2452457|2452638|DEPARTMENT|74|51|Thus difficult areas happen simply with a roads. E|bi-annual| +7936|AAAAAAAAAAPBAAAA|2452457|2452638|DEPARTMENT|74|52|Figures include well natural values. Able words grant then. Available, |bi-annual| +7937|AAAAAAAABAPBAAAA|2452457|2452638|DEPARTMENT|74|53|Long, foreign ties hear. Financial others may remain. Trouser|bi-annual| +7938|AAAAAAAACAPBAAAA|2452457|2452638|DEPARTMENT|74|54|Millions would enhance. Military chairs shall play therefore later honest fo|bi-annual| +7939|AAAAAAAADAPBAAAA|2452457|2452638|DEPARTMENT|74|55|Small, regulatory plans can relate slightly central rooms. Personal goods cover. Only working |bi-annual| +7940|AAAAAAAAEAPBAAAA|2452457|2452638|DEPARTMENT|74|56|Bright partners might contribute current rights; busy, beautiful |bi-annual| +7941|AAAAAAAAFAPBAAAA|2452457|2452638|DEPARTMENT|74|57|Topics reveal thus in a agreements; men support more from a feet. Ne|bi-annual| +7942|AAAAAAAAGAPBAAAA|2452457|2452638|DEPARTMENT|74|58|Major weapons see always easy women; coastal pounds project i|bi-annual| +7943|AAAAAAAAHAPBAAAA|||DEPARTMENT||||| +7944|AAAAAAAAIAPBAAAA|2452457|2452638|DEPARTMENT|74|60|Accurate makers can see educational regions; personal gentlemen help most independent b|bi-annual| +7945|AAAAAAAAJAPBAAAA|2452457|2452638|DEPARTMENT|74|61|Specialist doors mislead very flat, only effects. Normal, high sales brighten. Top br|bi-annual| +7946|AAAAAAAAKAPBAAAA|2452457|2452638|DEPARTMENT|74|62|Gentle, great years must not give direct much irrelevant eyes; da|bi-annual| +7947|AAAAAAAALAPBAAAA|2452457|2452638|DEPARTMENT|74|63|Together new issues identify managers. Much single homes ensure different events|bi-annual| +7948|AAAAAAAAMAPBAAAA|2452457|2452638|DEPARTMENT|74|64|Net, key farmers say young, eligible countries. Aga|bi-annual| +7949|AAAAAAAANAPBAAAA|2452457|2452638|DEPARTMENT|74|65|Parents remove; particular objections can go home evident, great attitudes; sure syst|bi-annual| +7950|AAAAAAAAOAPBAAAA|2452457|2452638|DEPARTMENT|74|66|Acids used to extract children; apparently vulnerable members migh|bi-annual| +7951|AAAAAAAAPAPBAAAA|2452457|2452638|DEPARTMENT|74|67|Today attractive decades must suit even so excellent gr|bi-annual| +7952|AAAAAAAAABPBAAAA|2452457|2452638|DEPARTMENT|74|68|Perhaps joint studies exist further political services. Too additional weeks come related mountai|bi-annual| +7953|AAAAAAAABBPBAAAA|2452457|2452638|DEPARTMENT|74|69|Real, competitive eyes benefit; superior, final components sho|bi-annual| +7954|AAAAAAAACBPBAAAA|2452457|2452638|DEPARTMENT|74|70|Economic schools used to exist places. Busy directions ought to contribute australian police.|bi-annual| +7955|AAAAAAAADBPBAAAA|2452457|2452638|DEPARTMENT|74|71|Red, public costs remain similarly tables. Numbers must review. Eit|bi-annual| +7956|AAAAAAAAEBPBAAAA|2452457|2452638|DEPARTMENT|74|72|Early changes decide. Companies will not go most financial dishes. Order|bi-annual| +7957|AAAAAAAAFBPBAAAA|2452457|2452638|DEPARTMENT|74|73|Views succeed even other products; positions expect birds. Dominant datab|bi-annual| +7958|AAAAAAAAGBPBAAAA|2452457|2452638|DEPARTMENT|74|74|Pink results shall produce different, civil companies. Results take plea|bi-annual| +7959|AAAAAAAAHBPBAAAA|2452457|2452638|DEPARTMENT|74|75|High expected exercises dislodge individual habits. Obvious problems|bi-annual| +7960|AAAAAAAAIBPBAAAA|2452457|2452638|DEPARTMENT|74|76|Ideas consider by the individuals. Also old years can|bi-annual| +7961|AAAAAAAAJBPBAAAA|2452457|2452638|DEPARTMENT|74|77|Usual relationships rest once decades. Patients shall continue as a pla|bi-annual| +7962|AAAAAAAAKBPBAAAA|2452457|2452638|DEPARTMENT|74|78|Obvious, useful benefits support closely following days. Private |bi-annual| +7963|AAAAAAAALBPBAAAA|2452457|2452638|DEPARTMENT|74|79|Helpful relatives could take there ago free children. Politically crazy possibilities c|bi-annual| +7964|AAAAAAAAMBPBAAAA|2452457|2452638|DEPARTMENT|74|80|Much primary walls serve mildly. Things must remember most past a officials. Much domest|bi-annual| +7965|AAAAAAAANBPBAAAA|||DEPARTMENT|74|81|Active members negotiate perhaps. Small, great leaves used to meet always entire, r|bi-annual| +7966|AAAAAAAAOBPBAAAA|2452457|2452638|DEPARTMENT|74|82|Other, new persons shall articulate precisely other w|bi-annual| +7967|AAAAAAAAPBPBAAAA|2452457|2452638|DEPARTMENT|74|83|Constitutional stations put worlds. Following, welsh facilit|bi-annual| +7968|AAAAAAAAACPBAAAA|2452457|2452638|DEPARTMENT|74|84|For example economic members will visit statistically|bi-annual| +7969|AAAAAAAABCPBAAAA|2452457|2452638|DEPARTMENT|74|85|Earlier silent layers could take bits; almost bare |bi-annual| +7970|AAAAAAAACCPBAAAA|2452457|2452638|DEPARTMENT|74|86|Great, external waters may not eat other words. Participants end later little ships. Public |bi-annual| +7971|AAAAAAAADCPBAAAA|2452457|2452638|DEPARTMENT|74|87|Always general reductions work a little troops. Eyes start the|bi-annual| +7972|AAAAAAAAECPBAAAA|2452457|2452638|DEPARTMENT|74|88|Late late years pass however. Original, soft losses should receive more about only po|bi-annual| +7973|AAAAAAAAFCPBAAAA|2452457|2452638|DEPARTMENT|74|89|Small efforts might not say surprisingly domestic weeks; over empty dishes may not expect no do|bi-annual| +7974|AAAAAAAAGCPBAAAA|2452457|2452638|DEPARTMENT|74|90|Increasingly important parts make usually back years. Generou|bi-annual| +7975|AAAAAAAAHCPBAAAA|2452457|2452638|DEPARTMENT|74|91|Contemporary millions take also warmly old problems. Occasional variations reject plants. Y|bi-annual| +7976|AAAAAAAAICPBAAAA|2452457|2452638|DEPARTMENT|74|92|Slowly free questions should run. Good pieces express rather at a communities|bi-annual| +7977|AAAAAAAAJCPBAAAA|2452457|2452638|DEPARTMENT|74|93|Sudden years may serve both. Liable, real weeks could stay eng|bi-annual| +7978|AAAAAAAAKCPBAAAA|2452457|2452638|DEPARTMENT|74|94|Direct, possible eyes succeed well similar nights; still cr|bi-annual| +7979|AAAAAAAALCPBAAAA|2452457|2452638|DEPARTMENT|74|95|Completely worthwhile awards learn much tight rooms; steady perfect pp. shall prote|bi-annual| +7980|AAAAAAAAMCPBAAAA|2452457|2452638|DEPARTMENT|74|96|So similar numbers shall look selective years; final, open features supp|bi-annual| +7981|AAAAAAAANCPBAAAA|2452457|2452638|DEPARTMENT|74|97|British, other friends want together. Councils reinforce idly ago ame|bi-annual| +7982|AAAAAAAAOCPBAAAA|2452457|2452638|DEPARTMENT|74|98|Friendly, advanced shapes hear so right, unable positions. Rounds could look ju|bi-annual| +7983|AAAAAAAAPCPBAAAA|2452457|2452638|DEPARTMENT|74|99|Simply ordinary others must take to a deposits. Very black users end to a views. Lo|bi-annual| +7984|AAAAAAAAADPBAAAA|2452457|2452638|DEPARTMENT|74|100|New dogs live over by the cars. Institutions ought to get eventually. Partic|bi-annual| +7985|AAAAAAAABDPBAAAA|2452457|2452638|DEPARTMENT|74|101|Large times get assets. Too common terms could engage close times. As yet italian seconds |bi-annual| +7986|AAAAAAAACDPBAAAA|2452457|2452638|DEPARTMENT|74|102|Ready houses should notice enough moral items. Then responsible years represent known fact|bi-annual| +7987|AAAAAAAADDPBAAAA|2452457|2452638|DEPARTMENT|74|103|Very capital symptoms emerge easily. Frequently novel p|bi-annual| +7988|AAAAAAAAEDPBAAAA|2452457|2452638|DEPARTMENT|74|104|Immediate gifts take special, major features. Too new jo|bi-annual| +7989|AAAAAAAAFDPBAAAA|2452457|2452638|DEPARTMENT|74|105|Light, public knees obtain there similar requirements. New, high schools pr|bi-annual| +7990|AAAAAAAAGDPBAAAA|2452457|2452638|DEPARTMENT|74|106|Openly coherent eyes persuade at the programmes. Tired, senior dreams die latterly|bi-annual| +7991|AAAAAAAAHDPBAAAA|2452457|2452638|DEPARTMENT|74|107|Only, left mice repair for a topics; schools prevent with an costs. Popu|bi-annual| +7992|AAAAAAAAIDPBAAAA|2452457|2452638|DEPARTMENT|74|108|Recent tonnes shall not follow little, financial measures. Notable subjects could |bi-annual| +7993|AAAAAAAAJDPBAAAA|2452275|2452365|DEPARTMENT|75|1|Physically considerable concepts measure certainly mana|quarterly| +7994|AAAAAAAAKDPBAAAA|2452275|2452365|DEPARTMENT|75|2|Visitors may welcome high, material pupils. National details allow as by a hou|quarterly| +7995|AAAAAAAALDPBAAAA|2452275|2452365|DEPARTMENT|75|3|Combined powers might return with a girls. Internat|quarterly| +7996|AAAAAAAAMDPBAAAA|2452275|2452365|DEPARTMENT|75|4|Decisions must mean similar matches. Certainly personal barriers say please only cri|quarterly| +7997|AAAAAAAANDPBAAAA|2452275|2452365|DEPARTMENT|75|5|Similar candidates cannot become just black subjec|quarterly| +7998|AAAAAAAAODPBAAAA|2452275|2452365|DEPARTMENT|75|6|Enough powerful differences wait long relevant cells. Bad pre|quarterly| +7999|AAAAAAAAPDPBAAAA|2452275|2452365|DEPARTMENT|75|7|Others could go much nuclear events. Nonetheless limited investigation|quarterly| +8000|AAAAAAAAAEPBAAAA|2452275|2452365|DEPARTMENT|75|8|Agents shall not let lazily; of course good arms l|quarterly| +8001|AAAAAAAABEPBAAAA|2452275|2452365|DEPARTMENT|75|9|Sorry cells claim still present, detailed animals; rapi|quarterly| +8002|AAAAAAAACEPBAAAA|2452275|2452365|DEPARTMENT|75|10|Tiny minutes should compile then sure, philosophical authorities. Now wonderful w|quarterly| +8003|AAAAAAAADEPBAAAA|2452275|2452365|DEPARTMENT|75|11|Just free pressures go tired, english attitudes; still members will get later. Happy windows |quarterly| +8004|AAAAAAAAEEPBAAAA|2452275|2452365|DEPARTMENT|75|12|More parental years add shares. Ago different objectives sho|quarterly| +8005|AAAAAAAAFEPBAAAA|2452275|2452365|DEPARTMENT|75|13|Financial weekends see rarely large years; united amounts s|quarterly| +8006|AAAAAAAAGEPBAAAA|2452275|2452365|DEPARTMENT|75|14|Local, brilliant features report. Significant, public firms register. Western influences mig|quarterly| +8007|AAAAAAAAHEPBAAAA|2452275|2452365|DEPARTMENT|75|15|Perhaps general yards suffer then natural documents. So false interests raise concerned, conservati|quarterly| +8008|AAAAAAAAIEPBAAAA|2452275|2452365|DEPARTMENT|75|16|Irrelevant contacts can take. Less wide rises will result normal origin|quarterly| +8009|AAAAAAAAJEPBAAAA|2452275|2452365|DEPARTMENT|75|17|Activities show up to a men. Hence other years form just records. For long |quarterly| +8010|AAAAAAAAKEPBAAAA|2452275|2452365|DEPARTMENT|75|18|Then evolutionary parties would not compare nevertheless like, impossible w|quarterly| +8011|AAAAAAAALEPBAAAA|2452275|2452365|DEPARTMENT|75|19|Together likely plans should get to no members. Periods can e|quarterly| +8012|AAAAAAAAMEPBAAAA|2452275|2452365|DEPARTMENT|75|20|Exhibitions own. Arrangements could arise. Fast adult specimens should not live today low regi|quarterly| +8013|AAAAAAAANEPBAAAA|2452275|2452365|DEPARTMENT|75|21|Able laws walk home international words. Good, independent rights will get as a professionals. |quarterly| +8014|AAAAAAAAOEPBAAAA|2452275|2452365|DEPARTMENT|75|22|Thus reasonable nations shall result again. Large-scale, patient jobs used to call. Individua|quarterly| +8015|AAAAAAAAPEPBAAAA|2452275|2452365|DEPARTMENT|75|23|Previous, novel forms interview really patients. Families may determine |quarterly| +8016|AAAAAAAAAFPBAAAA|2452275|2452365|DEPARTMENT|75|24|As good as huge wages could not arrive even general meetings. New tes|quarterly| +8017|AAAAAAAABFPBAAAA|2452275|2452365|DEPARTMENT|75|25|Right royal votes take super cards. Away cold historians limit at all so|quarterly| +8018|AAAAAAAACFPBAAAA|2452275|2452365|DEPARTMENT|75|26|Available, commercial arrangements reserve again awful, str|quarterly| +8019|AAAAAAAADFPBAAAA|2452275|2452365|DEPARTMENT|75|27|More french posts see sorry feet. Unpleasant boots shall rea|quarterly| +8020|AAAAAAAAEFPBAAAA|2452275|2452365|DEPARTMENT|75|28|Extra, elaborate gods feel again much reduced artists. Total cases defer distinctive, |quarterly| +8021|AAAAAAAAFFPBAAAA|2452275|2452365|DEPARTMENT|75|29|American, possible ways should provide then to the houses; organisations wa|quarterly| +8022|AAAAAAAAGFPBAAAA|2452275|2452365|DEPARTMENT|75|30|Corporate demonstrations will get now. Really considerable years used to say now|quarterly| +8023|AAAAAAAAHFPBAAAA|2452275|2452365|DEPARTMENT|75|31|More private minutes shall appear always always practical speeches. Measures might fight with the y|quarterly| +8024|AAAAAAAAIFPBAAAA|2452275|2452365|DEPARTMENT|75|32|Only players might focus in a measures; strange others attend. Roads shall not cease |quarterly| +8025|AAAAAAAAJFPBAAAA|2452275|2452365|DEPARTMENT|75|33|Small, causal attacks would capture quickly spiritual contributions. Yet other pubs |quarterly| +8026|AAAAAAAAKFPBAAAA|2452275|2452365|DEPARTMENT|75|34|So standard limits know together women. Decisions may not speak apart particular, moder|quarterly| +8027|AAAAAAAALFPBAAAA|2452275|2452365|DEPARTMENT|75|35|Experiments may not drink straight because of a fees. Domestic foods organi|quarterly| +8028|AAAAAAAAMFPBAAAA|2452275|2452365|DEPARTMENT|75|36|Years clear comments; other, initial germans see democrats. Results will pay on a guards|quarterly| +8029|AAAAAAAANFPBAAAA|2452275|2452365|DEPARTMENT|75|37|Corporate, accused patients shall visit complete, british projects; sat|quarterly| +8030|AAAAAAAAOFPBAAAA|2452275|2452365|DEPARTMENT|75|38|Mammals used to spread at least families; independent, historic problems may not fall indian han|quarterly| +8031|AAAAAAAAPFPBAAAA|2452275|2452365|DEPARTMENT|75|39|Sensitive years may cast able companies. About sheer hands should not he|quarterly| +8032|AAAAAAAAAGPBAAAA|2452275|2452365|DEPARTMENT|75|40|Available friends neglect; essential, living researchers sue soviet, american danger|quarterly| +8033|AAAAAAAABGPBAAAA||2452365|DEPARTMENT|75|||| +8034|AAAAAAAACGPBAAAA|2452275|2452365|DEPARTMENT|75|42|Prime, great beans face once in a things. Huge statements mig|quarterly| +8035|AAAAAAAADGPBAAAA|2452275|2452365|DEPARTMENT|75|43|Local mechanisms should visit in a parts. Officers re|quarterly| +8036|AAAAAAAAEGPBAAAA|2452275|2452365|DEPARTMENT|75|44|So patient connections could not offer so indeed strong occ|quarterly| +8037|AAAAAAAAFGPBAAAA|2452275|2452365|DEPARTMENT|75|45|Dead measures say massive, precise policies; advisory, bl|quarterly| +8038|AAAAAAAAGGPBAAAA|2452275|2452365|DEPARTMENT|75|46|Sales should look interests. Aware patients make in a branches. Followers might hurt else a|quarterly| +8039|AAAAAAAAHGPBAAAA|2452275|2452365|DEPARTMENT|75|47|Early, middle days shall pull over over a police; re|quarterly| +8040|AAAAAAAAIGPBAAAA|2452275|2452365|DEPARTMENT|75|48|Soldiers get within a children. Really free objectives love in a days. Easy|quarterly| +8041|AAAAAAAAJGPBAAAA|2452275|2452365|DEPARTMENT|75|49|Most early responses rid here cases. Visual, independent scientists might not conserve more bad|quarterly| +8042|AAAAAAAAKGPBAAAA|2452275|2452365|DEPARTMENT|75|50|Lost pages must hear both on the fans; now political customers publish to a parent|quarterly| +8043|AAAAAAAALGPBAAAA|2452275|2452365|DEPARTMENT|75|51|Else small kids should leave complete tears. Warm, young members|quarterly| +8044|AAAAAAAAMGPBAAAA|2452275|2452365|DEPARTMENT|75|52|Just new studies shall live. Large decisions adapt in a r|quarterly| +8045|AAAAAAAANGPBAAAA|2452275|2452365|DEPARTMENT|75|53|Today good ideas acknowledge very else close fingers. C|quarterly| +8046|AAAAAAAAOGPBAAAA|2452275|2452365|DEPARTMENT|75|54|Young, good hundreds get virtually good, unusual centuries. |quarterly| +8047|AAAAAAAAPGPBAAAA|2452275|2452365|DEPARTMENT|75|55|Federal, dear services agree yesterday cases. Annual, different homes would emigrate actually b|quarterly| +8048|AAAAAAAAAHPBAAAA|2452275|2452365|DEPARTMENT|75|56|Positive, empty results sit so across a hands. Young aspects used to induce eve|quarterly| +8049|AAAAAAAABHPBAAAA|2452275|2452365|DEPARTMENT|75|57|Here important calls breathe in particular expressions. Clothes take then |quarterly| +8050|AAAAAAAACHPBAAAA|2452275|2452365|DEPARTMENT|75|58|Only suitable companies reply including a cases. Necessary, real attempts will mention |quarterly| +8051|AAAAAAAADHPBAAAA|2452275|2452365|DEPARTMENT|75|59|Small assumptions breed voluntary, moral firms. Joint, young disputes would not as|quarterly| +8052|AAAAAAAAEHPBAAAA|2452275|2452365|DEPARTMENT|75|60|Issues discuss extensive, outer buildings; small employees of|quarterly| +8053|AAAAAAAAFHPBAAAA|2452275|2452365|DEPARTMENT|75|61|Children take rather still usual features. Only weeks could secure other, low types.|quarterly| +8054|AAAAAAAAGHPBAAAA|2452275|2452365|DEPARTMENT|75|62|Never likely measures will not get difficult forms. Social |quarterly| +8055|AAAAAAAAHHPBAAAA|2452275|2452365|DEPARTMENT|75|63|Rural goals will not call never workers. Civil, international|quarterly| +8056|AAAAAAAAIHPBAAAA|2452275|2452365|DEPARTMENT|75|64|Children determine never corporate americans. Areas should look only onl|quarterly| +8057|AAAAAAAAJHPBAAAA|2452275|2452365|DEPARTMENT|75|65|Automatic, useful affairs ought to place notes. Chief, necessary bones think reasonable, |quarterly| +8058|AAAAAAAAKHPBAAAA|2452275|2452365|DEPARTMENT|75|66|Wide men used to read central rules. French, public parents constitute in a farmers. Changi|quarterly| +8059|AAAAAAAALHPBAAAA|2452275|2452365|DEPARTMENT|75|67|Final connections sue excellent, good responsibilities. High, major centr|quarterly| +8060|AAAAAAAAMHPBAAAA|2452275|2452365|DEPARTMENT|75|68|Impossible, lengthy tickets correct more. Disciplinary, open|quarterly| +8061|AAAAAAAANHPBAAAA|2452275|2452365|DEPARTMENT|75|69|Commercial sources could come changes. Separate, happy |quarterly| +8062|AAAAAAAAOHPBAAAA|2452275|2452365|DEPARTMENT|75|70|Temporary, previous techniques cannot come important, big dimensions. Around milita|quarterly| +8063|AAAAAAAAPHPBAAAA|2452275|2452365|DEPARTMENT|75|71|Home great environments qualify through the birds; national, other ways clear th|quarterly| +8064|AAAAAAAAAIPBAAAA|2452275|2452365|DEPARTMENT|75|72|Economic, expected statements like quickly. Important companies could go very doubtful |quarterly| +8065|AAAAAAAABIPBAAAA|2452275|2452365|DEPARTMENT|75|73|Rare, other occasions help economic, wonderful rivals. O|quarterly| +8066|AAAAAAAACIPBAAAA|2452275|2452365|DEPARTMENT|75|74|German, black differences drive together other operations. Positive troo|quarterly| +8067|AAAAAAAADIPBAAAA|2452275|2452365|DEPARTMENT|75|75|Officers should not act for a walls. Patients shall ride frequently important |quarterly| +8068|AAAAAAAAEIPBAAAA|2452275|2452365|DEPARTMENT|75|76|Radical symbols might return amounts. Immediate, so|quarterly| +8069|AAAAAAAAFIPBAAAA|2452275|2452365|DEPARTMENT|75|77|Other, high hospitals will believe about communist servants. American, low things would st|quarterly| +8070|AAAAAAAAGIPBAAAA|2452275|2452365|DEPARTMENT|75|78|Olympic, material countries shall take. Organisations will not pin gen|quarterly| +8071|AAAAAAAAHIPBAAAA|2452275|2452365|DEPARTMENT|75|79|Useful calculations shall not think there margins. Feet improve earlier police.|quarterly| +8072|AAAAAAAAIIPBAAAA|2452275|2452365|DEPARTMENT|75|80|Best small efforts shall wait sorry systems. More spiritual children can come|quarterly| +8073|AAAAAAAAJIPBAAAA|2452275|2452365|DEPARTMENT|75|81|Used, solid losses decide there european levels. Often confident persons could n|quarterly| +8074|AAAAAAAAKIPBAAAA|2452275|2452365|DEPARTMENT|75|82|Different tories blow. Expensive acres might leave at least in a |quarterly| +8075|AAAAAAAALIPBAAAA|2452275|2452365|DEPARTMENT|75|83|Current teachers must try no longer. Equally extensive men act british, fixed pools. Now useful bil|quarterly| +8076|AAAAAAAAMIPBAAAA|2452275|2452365|DEPARTMENT|75|84|Here poor methods remain eyes. Yet full things prevent |quarterly| +8077|AAAAAAAANIPBAAAA|2452275|2452365|DEPARTMENT|75|85|More legal ears come in a eyes. Even important standards convey home wounds. Reserves sle|quarterly| +8078|AAAAAAAAOIPBAAAA|2452275|2452365|DEPARTMENT|75|86|Yellow elements might not show quickly high, vulnerable experiences. Individual neigh|quarterly| +8079|AAAAAAAAPIPBAAAA|2452275|2452365|DEPARTMENT|75|87|Dramatic homes take never. Machines might settle always i|quarterly| +8080|AAAAAAAAAJPBAAAA|2452275|2452365|DEPARTMENT|75|88|Thus happy workers should not go also in a boys. As particu|quarterly| +8081|AAAAAAAABJPBAAAA|2452275|2452365|DEPARTMENT|75|89|Small, different windows say full, criminal troops; problems uncover there. Films must los|quarterly| +8082|AAAAAAAACJPBAAAA|2452275|2452365|DEPARTMENT|75|90|More inadequate hotels produce in general grey, strategic points. New, bli|quarterly| +8083|AAAAAAAADJPBAAAA|2452275|2452365|DEPARTMENT|75|91|Pounds will pass in the changes. Years spend bad, irrelevant sources. Very or|quarterly| +8084|AAAAAAAAEJPBAAAA|2452275|2452365|DEPARTMENT|75|92|Long pounds must produce cases; available activities might make relatively remarkable republics; c|quarterly| +8085|AAAAAAAAFJPBAAAA|2452275|2452365|DEPARTMENT|75|93|Contrary women pass other, good services. Cells ought to say interesting activities. Above ser|quarterly| +8086|AAAAAAAAGJPBAAAA|2452275|2452365|DEPARTMENT|75|94|Please high times might feel before the cuts. Still real flowers|quarterly| +8087|AAAAAAAAHJPBAAAA|2452275|2452365|DEPARTMENT|75|95|Neatly free jobs would not prevent suddenly runs. Common, easy cells kill associate|quarterly| +8088|AAAAAAAAIJPBAAAA|2452275|2452365|DEPARTMENT|75|96|Crowds should not remain on the elements; administrative images will s|quarterly| +8089|AAAAAAAAJJPBAAAA|2452275|2452365|DEPARTMENT|75|97|Quiet, full sides expand before a bands. Full men go again profess|quarterly| +8090|AAAAAAAAKJPBAAAA|2452275|2452365|DEPARTMENT|75|98|Modern, small forces begin possibly students. Full concentrations may study ill. In order civil son|quarterly| +8091|AAAAAAAALJPBAAAA|2452275|2452365|DEPARTMENT|75|99|Important devices guarantee. Fundamental, national plans wi|quarterly| +8092|AAAAAAAAMJPBAAAA|2452275|2452365|DEPARTMENT|75|100|Big, private hands fly as; good, important months ought to see at all sometimes continuous groups.|quarterly| +8093|AAAAAAAANJPBAAAA|2452275|2452365|DEPARTMENT|75|101|Institutions decrease besides light futures. High, easy communications retire so b|quarterly| +8094|AAAAAAAAOJPBAAAA|2452275|2452365|DEPARTMENT|75|102|Free, other things think further british winners. C|quarterly| +8095|AAAAAAAAPJPBAAAA|2452275|2452365|DEPARTMENT|75|103|Sufficient, special things include even simple, me|quarterly| +8096|AAAAAAAAAKPBAAAA|2452275|2452365|DEPARTMENT|75|104|Vehicles might not press there. Afraid, good flowers will not seem both habits. Act|quarterly| +8097|AAAAAAAABKPBAAAA|2452275|2452365|DEPARTMENT|75|105|More new eyes may not provide primarily chinese words.|quarterly| +8098|AAAAAAAACKPBAAAA|2452275|2452365|DEPARTMENT|75|106|Male, single teachers accept still only working-class teeth.|quarterly| +8099|AAAAAAAADKPBAAAA|2452275|2452365|DEPARTMENT|75|107|Equivalent numbers will maintain. Earlier logical efforts know painfully practical cons|quarterly| +8100|AAAAAAAAEKPBAAAA|2452275|2452365|DEPARTMENT|75|108|Open, moral families happen currently top, strong interests. V|quarterly| +8101|AAAAAAAAFKPBAAAA|2452366|2452456|DEPARTMENT|76|1|Direct lips must see tonight charming dimensions. Now good negotiations call wi|quarterly| +8102|AAAAAAAAGKPBAAAA|2452366|2452456|DEPARTMENT|76||Perhaps sad families retain really good teeth. Objectives will || +8103|AAAAAAAAHKPBAAAA|2452366|2452456|DEPARTMENT|76|3|More mean courses end changes. Eyes will not look all; also true churches w|quarterly| +8104|AAAAAAAAIKPBAAAA|2452366|2452456|DEPARTMENT|76|4|Funny, clear forces grow as a whole so new changes. Frequently clever aut|quarterly| +8105|AAAAAAAAJKPBAAAA|2452366|2452456|DEPARTMENT|76|5|Walls would not tell interested, european occasions. Important,|quarterly| +8106|AAAAAAAAKKPBAAAA|2452366|2452456|DEPARTMENT|76|6|Guilty, typical police begin also in a courses. Black, cultur|quarterly| +8107|AAAAAAAALKPBAAAA|2452366|2452456|DEPARTMENT|76|7|New, popular problems understand more then english guidelines. Ever voluntary backs can feed ofte|quarterly| +8108|AAAAAAAAMKPBAAAA|2452366|2452456|DEPARTMENT|76|8|Values wish bottles. Fierce, light peasants cover well now clear cups; prices shall not think befo|quarterly| +8109|AAAAAAAANKPBAAAA|2452366|2452456|DEPARTMENT|76|9|Urban, old drivers ask more difficult books. Steady effect|quarterly| +8110|AAAAAAAAOKPBAAAA|2452366|2452456|DEPARTMENT|76|10|Future magistrates sell just on a months; tomorrow closed words can breathe be|quarterly| +8111|AAAAAAAAPKPBAAAA|2452366|2452456|DEPARTMENT|76|11|Effective groups will not play even economic flights; safe limitations grant big countie|quarterly| +8112|AAAAAAAAALPBAAAA|2452366|2452456|DEPARTMENT|76|12|New, unexpected years would talk gmt cities. Advertisements le|quarterly| +8113|AAAAAAAABLPBAAAA|2452366|2452456|DEPARTMENT|76|13|Small others rescue yet parliamentary eggs; occasions die well. Miles should |quarterly| +8114|AAAAAAAACLPBAAAA|2452366|2452456|DEPARTMENT|76|14|Overall patients might lose; old, sure markets eat useful |quarterly| +8115|AAAAAAAADLPBAAAA|2452366|2452456|DEPARTMENT|76|15|Of course only comments assert even small, hot divisions. Left, inc packages |quarterly| +8116|AAAAAAAAELPBAAAA|2452366|2452456|DEPARTMENT|76|16|So regular actors shall obtain already also required deaths. Free, warm chara|quarterly| +8117|AAAAAAAAFLPBAAAA|2452366|2452456|DEPARTMENT|76|17|Straight husbands attend as flat times. Others find prope|quarterly| +8118|AAAAAAAAGLPBAAAA|2452366|2452456|DEPARTMENT|76|18|Personal, other exhibitions marry students; individual rates improve never annual, capitalis|quarterly| +8119|AAAAAAAAHLPBAAAA|2452366|2452456|DEPARTMENT|76|19|English boxes go as too principal numbers. Full, defensive plans wait happy, strong reaction|quarterly| +8120|AAAAAAAAILPBAAAA|2452366|2452456|DEPARTMENT|76|20|Even full books welcome payable, english councillors. Possible benefits stay |quarterly| +8121|AAAAAAAAJLPBAAAA|2452366|2452456|DEPARTMENT|76|21|Feet get bodies. Important, parental organisations might not tell too public |quarterly| +8122|AAAAAAAAKLPBAAAA|2452366|2452456|DEPARTMENT|76|22|Surveys must not recognise always late financial jobs. Str|quarterly| +8123|AAAAAAAALLPBAAAA|2452366|2452456|DEPARTMENT|76|23|Sites repair. Special worlds prevent further defences. Then brit|quarterly| +8124|AAAAAAAAMLPBAAAA|2452366|2452456|DEPARTMENT|76|24|Recent, temporary animals play difficulties. High, complex |quarterly| +8125|AAAAAAAANLPBAAAA|2452366|2452456|DEPARTMENT|76|25|Other, chinese personnel extend more now key abilities. Re|quarterly| +8126|AAAAAAAAOLPBAAAA|2452366|2452456|DEPARTMENT|76|26|Dangerous, multiple firms will get in order. Willing, outstanding shoes cannot render actually|quarterly| +8127|AAAAAAAAPLPBAAAA|2452366|2452456|DEPARTMENT|76|27|Funds apply. Benefits might not manage immediate wages. Privately social banks win far final |quarterly| +8128|AAAAAAAAAMPBAAAA|2452366|2452456|DEPARTMENT|76|28|Open, good things talk representative, national ideas. Adult, only directors should pu|quarterly| +8129|AAAAAAAABMPBAAAA|2452366|2452456|DEPARTMENT|76|29|Simple, military patients may go extremely new views; pr|quarterly| +8130|AAAAAAAACMPBAAAA|2452366|2452456|DEPARTMENT|76|30|Local, sure requirements put feet. Hours look always in a sons|quarterly| +8131|AAAAAAAADMPBAAAA|2452366|2452456|DEPARTMENT|76|31|Marginal, old days mind quite in a methods; able, |quarterly| +8132|AAAAAAAAEMPBAAAA|2452366|2452456|DEPARTMENT|76|32|Other, advanced legs would not see major, usual loans. Firmly american issues m|quarterly| +8133|AAAAAAAAFMPBAAAA|2452366|2452456|DEPARTMENT|76|33|Just specific rates must respond now. Public fans climb already women; afra|quarterly| +8134|AAAAAAAAGMPBAAAA||||76||Great, possible classes shall signify fellow, occupational || +8135|AAAAAAAAHMPBAAAA|2452366|2452456|DEPARTMENT|76|35|Important others tell suddenly. Pieces build fascinating houses. Too widespread feet clo|quarterly| +8136|AAAAAAAAIMPBAAAA|2452366|2452456|DEPARTMENT|76|36|Young, immediate lines could take often even married worlds. Full detectives would o|quarterly| +8137|AAAAAAAAJMPBAAAA|2452366|2452456|DEPARTMENT|76|37|Trees ought to threaten; huge, similar papers cool. Complete states|quarterly| +8138|AAAAAAAAKMPBAAAA|2452366|2452456|DEPARTMENT|76|38|More local stones reach possible chapters. Full, present contents assist guilty, lucky jobs. |quarterly| +8139|AAAAAAAALMPBAAAA|2452366|2452456|DEPARTMENT|76|39|New symptoms get also enough royal changes. Other, great companies shall consider; cells may not|quarterly| +8140|AAAAAAAAMMPBAAAA|2452366|2452456|DEPARTMENT|76|40|Democratic, other letters used to like about a others. Products peer still spec|quarterly| +8141|AAAAAAAANMPBAAAA|2452366|2452456|DEPARTMENT|76|41|Sensible, important parents sit about tourists. Unpleasant r|quarterly| +8142|AAAAAAAAOMPBAAAA|2452366|2452456|DEPARTMENT|76|42|Western methods might not know merely extra, early frien|quarterly| +8143|AAAAAAAAPMPBAAAA|2452366|2452456|DEPARTMENT|76|43|Little, empty experiments provide so courses. Civil managers must not grow |quarterly| +8144|AAAAAAAAANPBAAAA|2452366|2452456|DEPARTMENT|76|44|Causal eyes cannot mean just readers. Essential, ex|quarterly| +8145|AAAAAAAABNPBAAAA|2452366|2452456|DEPARTMENT|76|45|Base, common questions deal international, english days. Glasses m|quarterly| +8146|AAAAAAAACNPBAAAA|2452366|2452456|DEPARTMENT|76|46|Political preparations like. For example complete blacks may disappoint so perhaps hu|quarterly| +8147|AAAAAAAADNPBAAAA|2452366|2452456|DEPARTMENT|76|47|Dreadful, german years shall say. Effects could use unexpectedly internationa|quarterly| +8148|AAAAAAAAENPBAAAA|2452366|2452456|DEPARTMENT|76|48|Things used to produce as final stones. Main organisms may not get the|quarterly| +8149|AAAAAAAAFNPBAAAA|2452366|2452456|DEPARTMENT|76|49|Full services would not sell of course teeth; politic|quarterly| +8150|AAAAAAAAGNPBAAAA|2452366|2452456|DEPARTMENT|76|50|So sheer terms escape now even economic citizens. Together miserable|quarterly| +8151|AAAAAAAAHNPBAAAA|2452366|2452456|DEPARTMENT|76|51|Strongly very consequences help tender, legal details. Eyebrows comfort often se|quarterly| +8152|AAAAAAAAINPBAAAA|2452366|2452456|DEPARTMENT|76|52|Sometimes british years bring royal, other provisions. Relatives fin|quarterly| +8153|AAAAAAAAJNPBAAAA|2452366|2452456|DEPARTMENT|76|53|Studies make regulations. Players rest namely active components. Active|quarterly| +8154|AAAAAAAAKNPBAAAA|2452366|2452456|DEPARTMENT|76|54|Men bear hitherto wages. Now regular restrictions assist |quarterly| +8155|AAAAAAAALNPBAAAA|2452366|2452456|DEPARTMENT|76|55|Only hostile titles talk exactly long police. Clearly vocational parents succeed |quarterly| +8156|AAAAAAAAMNPBAAAA|2452366|2452456|DEPARTMENT|76|56|Important mountains should not exercise elsewhere. Really high|quarterly| +8157|AAAAAAAANNPBAAAA|2452366|2452456|DEPARTMENT|76|57|Old, eventual sources get earlier pieces. Forces could not tell never efficient, la|quarterly| +8158|AAAAAAAAONPBAAAA|2452366|2452456|DEPARTMENT|76|58|Successful, recent holidays should not expect recent days. Drivers kick more from|quarterly| +8159|AAAAAAAAPNPBAAAA|2452366|2452456|DEPARTMENT|76|59|Grounds purchase nearby central kinds; frequent plants must think vol|quarterly| +8160|AAAAAAAAAOPBAAAA|2452366|2452456|DEPARTMENT|76|60|Only particular relationships cannot permit safe, able provisions. Services must mai|quarterly| +8161|AAAAAAAABOPBAAAA|2452366|2452456|DEPARTMENT|76|61|Sales will not avoid great, huge results. Small functions date at all things.|quarterly| +8162|AAAAAAAACOPBAAAA|2452366|2452456|DEPARTMENT|76|62|Blue, other shapes might investigate more. Possible companie|quarterly| +8163|AAAAAAAADOPBAAAA|2452366|2452456|DEPARTMENT|76|63|Big, military criteria can take however almost reas|quarterly| +8164|AAAAAAAAEOPBAAAA|2452366|2452456|DEPARTMENT|76|64|Now high hands could not think. Really worthwhile farm|quarterly| +8165|AAAAAAAAFOPBAAAA|2452366|2452456|DEPARTMENT|76|65|Hot, tall sales meet away on a lakes. Patterns shall see already; also moral flowers step s|quarterly| +8166|AAAAAAAAGOPBAAAA|2452366|2452456|DEPARTMENT|76|66|Then new benefits cross old, careful stars. More full dates keep all right matt|quarterly| +8167|AAAAAAAAHOPBAAAA|2452366|2452456|DEPARTMENT|76|67|High, sexual fingers must help however honest transactions; following teachers shall call in t|quarterly| +8168|AAAAAAAAIOPBAAAA|2452366|2452456|DEPARTMENT|76|68|Cells tell to a savings. Royal players fall to a words. Acceptable, social parties must not co|quarterly| +8169|AAAAAAAAJOPBAAAA|2452366|2452456|DEPARTMENT|76|69|Services must not benefit again in a rates. Measurements would verify other attitudes|quarterly| +8170|AAAAAAAAKOPBAAAA|2452366|2452456|DEPARTMENT|76|70|Full, special roads could not illustrate quietly medical, small re|quarterly| +8171|AAAAAAAALOPBAAAA|2452366|2452456|DEPARTMENT|76|71|Great developers ought to occur only small boys. Particularly true rates should enjoy now t|quarterly| +8172|AAAAAAAAMOPBAAAA|2452366|2452456|DEPARTMENT|76|72|Old parents could save in a subjects. Inadequate, administrative nam|quarterly| +8173|AAAAAAAANOPBAAAA|2452366|2452456|DEPARTMENT|76|73|Farmers could not go hot colonies. Terms used to recover. |quarterly| +8174|AAAAAAAAOOPBAAAA|2452366|2452456|DEPARTMENT|76|74|Resources know for a periods. Harder american organisms fulfil remarkably to a backs. So n|quarterly| +8175|AAAAAAAAPOPBAAAA|2452366|2452456|DEPARTMENT|76|75|Wide, original boys must apply low legal folk. More formal heels need in ord|quarterly| +8176|AAAAAAAAAPPBAAAA|2452366|2452456|DEPARTMENT|76|76|Complex, environmental demands want especially powerful ta|quarterly| +8177|AAAAAAAABPPBAAAA|2452366|2452456|DEPARTMENT|76|77|Afraid parties think only. Committees may vary so near young co|quarterly| +8178|AAAAAAAACPPBAAAA|2452366|2452456|DEPARTMENT|76|78|Women give quicker. Good pairs thank extensive, old vehicles. Con|quarterly| +8179|AAAAAAAADPPBAAAA|2452366|2452456|DEPARTMENT|76|79|Initial, hungry practices will not see good forms. Co|quarterly| +8180|AAAAAAAAEPPBAAAA|2452366|2452456|DEPARTMENT|76|80|Most soviet instances see separate, rare things; excellent, causal years associate generally in |quarterly| +8181|AAAAAAAAFPPBAAAA|2452366|2452456|DEPARTMENT|76|81|Numbers must meet also. Sorry, previous residents might believe. Makers may not get in a words; roy|quarterly| +8182|AAAAAAAAGPPBAAAA|2452366|2452456|DEPARTMENT|76|82|Necessarily presidential systems will not stop later brief specialist trees. Appropriate, addition|quarterly| +8183|AAAAAAAAHPPBAAAA|2452366|2452456|DEPARTMENT|76|83|Major, young occasions take perhaps potential reasons. Clear groups used to form a|quarterly| +8184|AAAAAAAAIPPBAAAA|2452366|2452456|DEPARTMENT|76|84|Days think persistent countries. Payments come rural, interesting ingredients.|quarterly| +8185|AAAAAAAAJPPBAAAA|2452366|2452456|DEPARTMENT|76|85|Only, moral loans used to feel so significant personnel. Hig|quarterly| +8186|AAAAAAAAKPPBAAAA|2452366|2452456|DEPARTMENT|76|86|Long, shy boundaries acquire cold stones. Unique reforms suppose a little local years. Obviously |quarterly| +8187|AAAAAAAALPPBAAAA|2452366|2452456|DEPARTMENT|76|87|Relations shall find specific tensions. Commonly correct cars ought to climb well. Symptoms s|quarterly| +8188|AAAAAAAAMPPBAAAA|||DEPARTMENT|76|88||| +8189|AAAAAAAANPPBAAAA|2452366|2452456|DEPARTMENT|76|89|Ways set all right. Nervous, good results would come bare, chronic computers. Lectures would not |quarterly| +8190|AAAAAAAAOPPBAAAA|2452366|2452456|DEPARTMENT|76|90|Adults will not come hard under the women. Practical, favorable saving|quarterly| +8191|AAAAAAAAPPPBAAAA|2452366|2452456|DEPARTMENT|76|91|Basic members know phases. Pockets ought to see couples. Leaders may sleep mor|quarterly| +8192|AAAAAAAAAAACAAAA|2452366|2452456|DEPARTMENT|76|92|Efficient, environmental employers find now local months. Individ|quarterly| +8193|AAAAAAAABAACAAAA|2452366|2452456|DEPARTMENT|76|93|Small sides give often in a provisions. Black managers read a|quarterly| +8194|AAAAAAAACAACAAAA|2452366|2452456|DEPARTMENT|76|94|Straight amazing times used to get others. Much conscious representatives work too amer|quarterly| +8195|AAAAAAAADAACAAAA|2452366|2452456|DEPARTMENT|76|95|Keen mechanisms fly in a hours. Never new points tell fairly for a firms; even difficu|quarterly| +8196|AAAAAAAAEAACAAAA|2452366|2452456|DEPARTMENT|76|96|Variable, corporate children feel suddenly windows. Books could not force |quarterly| +8197|AAAAAAAAFAACAAAA|2452366|2452456|DEPARTMENT|76|97|Light, semantic sources differentiate english, bizarre manufacturers. |quarterly| +8198|AAAAAAAAGAACAAAA|2452366|2452456|DEPARTMENT|76|98|Classes should not get recently in the areas. Actually regi|quarterly| +8199|AAAAAAAAHAACAAAA|2452366|2452456|DEPARTMENT|76|99|Ideas might not happen under on a times. All limited individuals know|quarterly| +8200|AAAAAAAAIAACAAAA|2452366|2452456|DEPARTMENT|76|100|Concentrations would leave again damages. Often aware relat|quarterly| +8201|AAAAAAAAJAACAAAA|2452366|2452456|DEPARTMENT|76|101|Actually developing factors make also legal functions. Directors could e|quarterly| +8202|AAAAAAAAKAACAAAA|2452366|2452456|DEPARTMENT|76|102|Traditional, ready parts will not plough multiple, unlikely sales. Concerned, compl|quarterly| +8203|AAAAAAAALAACAAAA|2452366|2452456|DEPARTMENT|76|103|More happy police remember further splendid poets. |quarterly| +8204|AAAAAAAAMAACAAAA|2452366|2452456|DEPARTMENT|76|104|Various films quit in a parents; miles give. Young customs should serve constraints. Cr|quarterly| +8205|AAAAAAAANAACAAAA|2452366|2452456|DEPARTMENT|76|105|Often political services used to indicate normal pages. Expensive, whole languages will go more |quarterly| +8206|AAAAAAAAOAACAAAA|2452366|2452456|DEPARTMENT|76|106|Exciting changes should know more responsible forms. Germans should see bri|quarterly| +8207|AAAAAAAAPAACAAAA|2452366|2452456|DEPARTMENT|76|107|Issues can compete from a years. Industrial, late choices would pay home other metres; closely di|quarterly| +8208|AAAAAAAAABACAAAA|2452366|2452456|DEPARTMENT|76|108|Southern places might look both with a minutes. Principal, present conditions used to|quarterly| +8209|AAAAAAAABBACAAAA|2452457|2452547|DEPARTMENT|77|1|Small, personal faces will find perhaps communist patients. New, appropriate consumer|quarterly| +8210|AAAAAAAACBACAAAA|2452457|2452547|DEPARTMENT|77|2|Different conservatives may not treat increasingly on the ambition|quarterly| +8211|AAAAAAAADBACAAAA|2452457|2452547|DEPARTMENT|77|3|Clearly extra musicians might expect commonly other, simple years. Fun|quarterly| +8212|AAAAAAAAEBACAAAA|2452457|2452547|DEPARTMENT|77|4|Effectively mass holes depend only proper, new responses. Physical statements see so |quarterly| +8213|AAAAAAAAFBACAAAA|2452457|2452547|DEPARTMENT|77|5|Minerals play much to a variations. Intimate, personal elements take spor|quarterly| +8214|AAAAAAAAGBACAAAA|2452457|2452547|DEPARTMENT|77|6|Kind of political hands ought to like very national prime times. Private, diverse |quarterly| +8215|AAAAAAAAHBACAAAA|2452457|2452547|DEPARTMENT|77|7|Provisions affect direct countries. Nevertheless insufficient words want real, inter|quarterly| +8216|AAAAAAAAIBACAAAA|2452457|2452547|DEPARTMENT|77|8|Certain children might fulfil kids. Also new instructions might attack especially friendly, releva|quarterly| +8217|AAAAAAAAJBACAAAA|2452457|2452547|DEPARTMENT|77|9|Better loose visitors will see. Social, new times suit entir|quarterly| +8218|AAAAAAAAKBACAAAA|2452457|2452547|DEPARTMENT|77|10|Often compatible studies will think as ideas; cells would label abou|quarterly| +8219|AAAAAAAALBACAAAA|2452457|2452547|DEPARTMENT|77|11|Agencies stay fast duties. Central conditions dismantle nowhere. Flames follow over a members. |quarterly| +8220|AAAAAAAAMBACAAAA|2452457|2452547|DEPARTMENT|77|12|Even similar ministers confirm often international yards. Personal minu|quarterly| +8221|AAAAAAAANBACAAAA|2452457|2452547|DEPARTMENT|77|13|Necessary, nervous teachers would not involve even lines. Objectives should find very c|quarterly| +8222|AAAAAAAAOBACAAAA|2452457|2452547|DEPARTMENT|77|14|Simple, free developments should go acts. Other, free p|quarterly| +8223|AAAAAAAAPBACAAAA|2452457|2452547|DEPARTMENT|77|15|Likely, semantic situations see yet also unknown account|quarterly| +8224|AAAAAAAAACACAAAA|2452457|2452547|DEPARTMENT|77|16|Seats should try that is on the dishes. Workers cut also even bri|quarterly| +8225|AAAAAAAABCACAAAA|2452457|2452547|DEPARTMENT|77|17|Final considerations target outside actually dull words. Now left candidates represent n|quarterly| +8226|AAAAAAAACCACAAAA|2452457|2452547|DEPARTMENT|77|18|As great sources must try important, different alternatives. Running laboratories s|quarterly| +8227|AAAAAAAADCACAAAA|2452457|2452547|DEPARTMENT|77|19|Ltd., white arguments can mind either levels. Historical, golden markets safeguard there by an c|quarterly| +8228|AAAAAAAAECACAAAA|2452457|2452547|DEPARTMENT|77|20|Long, empty lessons must come rather by a tons. Large-scale services might fill even |quarterly| +8229|AAAAAAAAFCACAAAA|2452457|2452547|DEPARTMENT|77|21|Officials must accept in terms of a systems. Secrets may not stand still fine colonies. Royal, |quarterly| +8230|AAAAAAAAGCACAAAA|2452457|2452547|DEPARTMENT|77|22|Then royal women ensure popular patients. Various members will continue ideal|quarterly| +8231|AAAAAAAAHCACAAAA|2452457|2452547|DEPARTMENT|77|23|Polite countries used to grant; figures might predict about revolutionary ter|quarterly| +8232|AAAAAAAAICACAAAA|2452457|2452547|DEPARTMENT|77|24|Early fathers should not import. Central funds sack full, high numbers. Times receive|quarterly| +8233|AAAAAAAAJCACAAAA|2452457|2452547|DEPARTMENT|77|25|Feet would not cope. New, careful unions reach as indeed liberal oppor|quarterly| +8234|AAAAAAAAKCACAAAA|2452457|2452547|DEPARTMENT|77|26|Supplies come in general slight, military customers. Exp|quarterly| +8235|AAAAAAAALCACAAAA|2452457|2452547|DEPARTMENT|77|27|Relatively scottish pupils might not sign soon. Urban women cou|quarterly| +8236|AAAAAAAAMCACAAAA|2452457|2452547|DEPARTMENT|77|28|Obviously surprising men record together hence tough friends. Proudly new memori|quarterly| +8237|AAAAAAAANCACAAAA|2452457|2452547|DEPARTMENT|77|29|Full reasons accuse; days should wait surely. Activities could help almost proble|quarterly| +8238|AAAAAAAAOCACAAAA|2452457|2452547|DEPARTMENT|77|30|Growing hours can feel so. Dynamic days ought to gain. For long modern figu|quarterly| +8239|AAAAAAAAPCACAAAA|2452457|2452547|DEPARTMENT|77|31|Funds represent together at the eyes; conditions may walk between a member|quarterly| +8240|AAAAAAAAADACAAAA|2452457|2452547|DEPARTMENT|77|32|Similar weeks hold very in the children. Men would redress beds. Large, nationa|quarterly| +8241|AAAAAAAABDACAAAA|2452457|2452547|DEPARTMENT|77|33|Local products undermine yet fundamental friends; financial re|quarterly| +8242|AAAAAAAACDACAAAA|2452457|2452547|DEPARTMENT|77|34|Always absolute demands dare on the letters. National expectations become well isolated pa|quarterly| +8243|AAAAAAAADDACAAAA|2452457|2452547|DEPARTMENT|77|35|Lines might achieve probably social, different advisers. Widely possible others see due to a o|quarterly| +8244|AAAAAAAAEDACAAAA|2452457|2452547|DEPARTMENT|77|36|Towns sue apart at a poems. Colours would find. Whole, british nurses stop here |quarterly| +8245|AAAAAAAAFDACAAAA|2452457|2452547|DEPARTMENT|77|37|Indeed major skills will not understand more ways. Other develop|quarterly| +8246|AAAAAAAAGDACAAAA|2452457|2452547|DEPARTMENT|77|38|Loans involve fast healthy answers. Political cases apply libraries. Appropriate|quarterly| +8247|AAAAAAAAHDACAAAA|2452457|2452547|DEPARTMENT|77|39|Boys say in the walls. International ages could not ban |quarterly| +8248|AAAAAAAAIDACAAAA|2452457|2452547|DEPARTMENT|77|40|Existing, basic sources tell again special girls. Goods will leave typically |quarterly| +8249|AAAAAAAAJDACAAAA|2452457|2452547|DEPARTMENT|77|41|New factors will need into a origins. Drugs will not reme|quarterly| +8250|AAAAAAAAKDACAAAA|2452457|2452547|DEPARTMENT|77|42|Texts see. Easily specific writers might not lead mos|quarterly| +8251|AAAAAAAALDACAAAA|2452457|2452547|DEPARTMENT|77|43|Years follow at first goals. Walls would not get then abou|quarterly| +8252|AAAAAAAAMDACAAAA|2452457|2452547|DEPARTMENT|77|44|Just super weekends used to go possibly attitudes. Foreign, old games swallow p|quarterly| +8253|AAAAAAAANDACAAAA|2452457|2452547|DEPARTMENT|77|45|Managerial ages must not call formally by a eyes; relevant, small rights breed. |quarterly| +8254|AAAAAAAAODACAAAA|2452457|2452547|DEPARTMENT|77|46|Direct, limited duties bear likely years; ideological, difficult part|quarterly| +8255|AAAAAAAAPDACAAAA|2452457|2452547|DEPARTMENT|77|47|Of course clear plans smell only underlying, other words; low|quarterly| +8256|AAAAAAAAAEACAAAA|2452457|2452547|DEPARTMENT|77|48|Only areas will not speak directly without a names. Too fatal times sha|quarterly| +8257|AAAAAAAABEACAAAA|2452457|2452547|DEPARTMENT|77|49|Differences can put poor, central homes. Perhaps concerned stations shall no|quarterly| +8258|AAAAAAAACEACAAAA|2452457|2452547|DEPARTMENT|77|50|Classes will expect. Now capitalist services must live on a skill|quarterly| +8259|AAAAAAAADEACAAAA|2452457|2452547|DEPARTMENT|77|51|Again new numbers cannot direct so subsequent temporary clubs. Yellow women woul|quarterly| +8260|AAAAAAAAEEACAAAA|2452457|2452547|DEPARTMENT|77|52|Members should not meet activities. Efforts afford quite according to a |quarterly| +8261|AAAAAAAAFEACAAAA|2452457|2452547|DEPARTMENT|77|53|Parameters will not hit normally likely, short managers; also single patients absorb t|quarterly| +8262|AAAAAAAAGEACAAAA|2452457|2452547|DEPARTMENT|77|54|Friendly universities manipulate far other thanks; right officials ought to|quarterly| +8263|AAAAAAAAHEACAAAA|2452457|2452547|DEPARTMENT|77|55|Current churches shall not describe very exactly unable members. Gr|quarterly| +8264|AAAAAAAAIEACAAAA|2452457|2452547|DEPARTMENT|77|56|Local, low arrangements cannot lead european shares. Short colours carry anywhere ba|quarterly| +8265|AAAAAAAAJEACAAAA|2452457|2452547|DEPARTMENT|77|57|Children would convey simply. Scientists can provide to the references. Gradual|quarterly| +8266|AAAAAAAAKEACAAAA|2452457|2452547|DEPARTMENT|77|58|Activities must use anywhere; men lead more very long services. Litt|quarterly| +8267|AAAAAAAALEACAAAA|2452457|2452547|DEPARTMENT|77|59|Traditional others concern to a arrangements; acute|quarterly| +8268|AAAAAAAAMEACAAAA|2452457|2452547|DEPARTMENT|77|60|Also black premises tell properly public woods. Yet original adults could find |quarterly| +8269|AAAAAAAANEACAAAA|2452457|2452547|DEPARTMENT|77|61|Cheeks catch only dates. Only, formidable sales may give from the workers. Moments could look|quarterly| +8270|AAAAAAAAOEACAAAA|2452457|2452547|DEPARTMENT|77|62|Phenomena hit royal, effective circles. Major leads say only for a arrangements. Short, frenc|quarterly| +8271|AAAAAAAAPEACAAAA|2452457|2452547|DEPARTMENT|77|63|Open children note greatly once more only voices. Eyes go recen|quarterly| +8272|AAAAAAAAAFACAAAA|2452457|2452547|DEPARTMENT|77|64|Drugs would address hard efficient, growing models. Spatial males sha|quarterly| +8273|AAAAAAAABFACAAAA|2452457|2452547|DEPARTMENT|77|65|Also glad windows could not lose. Valid, new governors maintain|quarterly| +8274|AAAAAAAACFACAAAA|2452457|2452547|DEPARTMENT|77|66|Various, strong talks shall like; short-term cases occur better. Democratic women used to turn chie|quarterly| +8275|AAAAAAAADFACAAAA|2452457|2452547|DEPARTMENT|77|67|Characters range ideas; good variables leave otherwise very d|quarterly| +8276|AAAAAAAAEFACAAAA|2452457|2452547|DEPARTMENT|77|68|Sufficient facts take so indian eyes. Spanish, great parti|quarterly| +8277|AAAAAAAAFFACAAAA|2452457|2452547|DEPARTMENT|77|69|Memories will not accelerate only troops. Indeed other hours should protest very |quarterly| +8278|AAAAAAAAGFACAAAA|2452457|2452547|DEPARTMENT|77|70|Instructions establish. Entire, basic ministers look running payments. Fo|quarterly| +8279|AAAAAAAAHFACAAAA|2452457|2452547|DEPARTMENT|77|71|Big seeds can face police. Leading things should not tend. Authorities must |quarterly| +8280|AAAAAAAAIFACAAAA|2452457|2452547|DEPARTMENT|77|72|British, aware clothes cannot grow enough particular, di|quarterly| +8281|AAAAAAAAJFACAAAA|2452457|2452547|DEPARTMENT|77|73|Top nations go quickly items. Moreover amazing resources suppl|quarterly| +8282|AAAAAAAAKFACAAAA|2452457|2452547|DEPARTMENT|77|74|Odds can hide domestic, considerable stories. Citizens |quarterly| +8283|AAAAAAAALFACAAAA|2452457|2452547|DEPARTMENT|77|75|Rapid countries exist. Expected, essential resources will turn senior employers. Ther|quarterly| +8284|AAAAAAAAMFACAAAA|2452457|2452547|DEPARTMENT|77|76|Naval sales remember indeed; direct pupils tell also possibly following da|quarterly| +8285|AAAAAAAANFACAAAA|2452457|2452547|DEPARTMENT|77|77|Large, annual bodies say then false children. Miles could take repeatedly other activities; le|quarterly| +8286|AAAAAAAAOFACAAAA|2452457|2452547|DEPARTMENT|77|78|Joint, urgent chains must buy acute, anxious parties. Above|quarterly| +8287|AAAAAAAAPFACAAAA|2452457|2452547|DEPARTMENT|77|79|Military eyes shall need inevitable provisions; alone days tell |quarterly| +8288|AAAAAAAAAGACAAAA|2452457|2452547|DEPARTMENT|77|80|Limitations dictate therefore concerned, strong matters. For example crit|quarterly| +8289|AAAAAAAABGACAAAA|2452457|2452547|DEPARTMENT|77|81|Taxes answer for example good versions. Long, official tiles change |quarterly| +8290|AAAAAAAACGACAAAA|2452457|2452547|DEPARTMENT|77|82|Shops will flow broken pockets. Active doubts develop still else big arts. Other police might help|quarterly| +8291|AAAAAAAADGACAAAA|2452457|2452547|DEPARTMENT|77|83|For example past casualties may sell both colleges. Black males go c|quarterly| +8292|AAAAAAAAEGACAAAA|2452457|2452547|DEPARTMENT|77|84|Buildings eat also human ties. Everyday eggs try now joint unions. Live, subsequent diff|quarterly| +8293|AAAAAAAAFGACAAAA|2452457|2452547|DEPARTMENT|77|85|Dangerous patients use ultimately national designers. Single plans must help in general.|quarterly| +8294|AAAAAAAAGGACAAAA|2452457|2452547|DEPARTMENT|77|86|Things knock mass, special years. Ago strong needs used |quarterly| +8295|AAAAAAAAHGACAAAA|2452457|2452547|DEPARTMENT|77|87|Tiles must work furthermore because of a films. Personal, certai|quarterly| +8296|AAAAAAAAIGACAAAA|2452457|2452547|DEPARTMENT|77|88|Literary, right weeks might eat too others. Homes shall not go too final years. Altogether histor|quarterly| +8297|AAAAAAAAJGACAAAA|2452457|2452547|DEPARTMENT|77|89|Personal names will make young, small years; dry parents may see necessarily lonely hands. N|quarterly| +8298|AAAAAAAAKGACAAAA|2452457|2452547|DEPARTMENT|77|90|Busy elements make. Different conditions learn then basic areas. Great weeks|quarterly| +8299|AAAAAAAALGACAAAA|2452457|2452547|DEPARTMENT|77|91|Businessmen shall not look jewish, able citizens; |quarterly| +8300|AAAAAAAAMGACAAAA|2452457|2452547|DEPARTMENT|77|92|Also international versions happen. Proper, various views st|quarterly| +8301|AAAAAAAANGACAAAA|2452457|2452547|DEPARTMENT|77|93|Surprising communists used to move; big persons take huskily acts; |quarterly| +8302|AAAAAAAAOGACAAAA|2452457|2452547|DEPARTMENT|77|94|Kindly alternative instructions cannot appear sheer, similar charges. Total, new services |quarterly| +8303|AAAAAAAAPGACAAAA|2452457|2452547|DEPARTMENT|77|95|Big males should not eliminate then matters. Reasonably clever demands balance already nume|quarterly| +8304|AAAAAAAAAHACAAAA|2452457|2452547|DEPARTMENT|77|96|Quite particular events tackle red managers; active, small teeth tel|quarterly| +8305|AAAAAAAABHACAAAA|2452457|2452547|DEPARTMENT|77|97|Middle-class details fill as. Men shall respond fresh forests. Great, dead pounds co-ordinate o|quarterly| +8306|AAAAAAAACHACAAAA|2452457|2452547|DEPARTMENT|77|98|Social benefits leave however courses. Able decisions organise more to a tensions. Alm|quarterly| +8307|AAAAAAAADHACAAAA|2452457|2452547|DEPARTMENT|77|99|Classical, important systems can understand for a aspects. Long months bi|quarterly| +8308|AAAAAAAAEHACAAAA|2452457|2452547|DEPARTMENT|77|100|Authorities could succeed suddenly indian, asleep cars. Ex|quarterly| +8309|AAAAAAAAFHACAAAA|2452457|2452547|DEPARTMENT|77|101|Ministers will believe formal pilots. Neighbours may not pre|quarterly| +8310|AAAAAAAAGHACAAAA|2452457|2452547|DEPARTMENT|77|102|British observations will say. Recommendations may change even times. Fin|quarterly| +8311|AAAAAAAAHHACAAAA|2452457|2452547|DEPARTMENT|77|103|Present members shall own frequently tight, real teams. New, original rates cou|quarterly| +8312|AAAAAAAAIHACAAAA|2452457|2452547|DEPARTMENT|77|104|Serious points want; original countries amount fairly a little lit|quarterly| +8313|AAAAAAAAJHACAAAA|2452457|2452547|DEPARTMENT|77|105|Actually civil minutes shall not like perfectly programmes. Cultural, correct use|quarterly| +8314|AAAAAAAAKHACAAAA|2452457|2452547|DEPARTMENT|77|106|Plans might initiate farmers. Ago middle places used to rise eyes. Constant, othe|quarterly| +8315|AAAAAAAALHACAAAA|2452457|2452547|DEPARTMENT|77|107|Logical, american members put whole, considerable fears. Offices can know still either f|quarterly| +8316|AAAAAAAAMHACAAAA|2452457|2452547|DEPARTMENT|77|108|Parts manage reasons; successful, subsequent vehicles could not think now as racial results. Word|quarterly| +8317|AAAAAAAANHACAAAA|2452548|2452638|DEPARTMENT|78|1|Interests build difficulties. Original parties cannot say roads. Upside down equal tonnes may |quarterly| +8318|AAAAAAAAOHACAAAA|2452548|2452638|DEPARTMENT|78|2|As major difficulties must tackle qualifications. Exactly principal changes might|quarterly| +8319|AAAAAAAAPHACAAAA|2452548|2452638|DEPARTMENT|78|3|Even full letters used to fashion economically professional processes. Only, ind|quarterly| +8320|AAAAAAAAAIACAAAA|2452548|2452638|DEPARTMENT|78|4|Working, sure factories would sing only with the years. Small, common flowers catch|quarterly| +8321|AAAAAAAABIACAAAA|2452548|2452638|DEPARTMENT|78|5|Mainly far weeks would not describe together for the sectors. Obvious, red pa|quarterly| +8322|AAAAAAAACIACAAAA|2452548|2452638|DEPARTMENT|78|6|Yesterday legal services impose western, blue models. Very new families ought to carry e|quarterly| +8323|AAAAAAAADIACAAAA|2452548|2452638|DEPARTMENT|78|7|Still human doors produce houses; normal, video-taped animals oug|quarterly| +8324|AAAAAAAAEIACAAAA|2452548|2452638|DEPARTMENT|78|8|Variable cars might not say altogether. Very small causes deal over natural sk|quarterly| +8325|AAAAAAAAFIACAAAA|2452548|2452638|DEPARTMENT|78|9|National, warm answers used to travel basic, legal refugees. Only capable parties may not win large|quarterly| +8326|AAAAAAAAGIACAAAA|2452548|2452638|DEPARTMENT|78|10|Ambitious germans see. Duties must hold inside immediate, important arrangements. I|quarterly| +8327|AAAAAAAAHIACAAAA|2452548|2452638|DEPARTMENT|78|11|Military, only areas suit particularly only problems; enough |quarterly| +8328|AAAAAAAAIIACAAAA|2452548|2452638|DEPARTMENT|78|12|Premier, sure losses should stare also representatives. Obvious, horrible pensions would buy|quarterly| +8329|AAAAAAAAJIACAAAA|2452548|2452638|DEPARTMENT|78|13|There partial kinds used to like to a risks; diffi|quarterly| +8330|AAAAAAAAKIACAAAA|2452548|2452638|DEPARTMENT|78|14|Future points take inside exactly special shows. Physical, ine|quarterly| +8331|AAAAAAAALIACAAAA|2452548|2452638|DEPARTMENT|78|15|Hardly daily terms should satisfy light elections; ancient hill|quarterly| +8332|AAAAAAAAMIACAAAA|2452548|2452638|DEPARTMENT|78|16|Empty weapons want newly inevitable penalties. Urban, main children shall go |quarterly| +8333|AAAAAAAANIACAAAA|2452548|2452638|DEPARTMENT|78|17|There voluntary requirements find post-war minutes. Neither dry eyes cover too users. Othe|quarterly| +8334|AAAAAAAAOIACAAAA|2452548|2452638|DEPARTMENT|78|18|Public problems would provide well likely, ultimate areas. Palestinia|quarterly| +8335|AAAAAAAAPIACAAAA|2452548|2452638|DEPARTMENT|78|19|Also other terms relate as for a ways. Too wild relatives may emphasise polit|quarterly| +8336|AAAAAAAAAJACAAAA|2452548|2452638|DEPARTMENT|78|20|Much classical children shall send exclusive, rough |quarterly| +8337|AAAAAAAABJACAAAA|2452548|2452638|DEPARTMENT|78|21|Also classical men get. Parts may not decide areas. Economic police can make very familiar rises;|quarterly| +8338|AAAAAAAACJACAAAA|2452548|2452638|DEPARTMENT|78|22|Hitherto modern boards allow more in a individuals. Successful careers make long creative app|quarterly| +8339|AAAAAAAADJACAAAA|2452548|2452638|DEPARTMENT|78|23|Actual charges wreck jointly military methods. Early, abstract states score waves. Components c|quarterly| +8340|AAAAAAAAEJACAAAA|2452548|2452638|DEPARTMENT|78|24|Pieces tell. Inc, elegant islands ought to appeal |quarterly| +8341|AAAAAAAAFJACAAAA|2452548|2452638|DEPARTMENT|78|25|Issues wash there full, intact books. Grounds love tightly wages. Similar, christian roads will |quarterly| +8342|AAAAAAAAGJACAAAA|2452548|2452638|DEPARTMENT|78|26|Casual, original lines continue as foreign, previous shares. As usual whit|quarterly| +8343|AAAAAAAAHJACAAAA|2452548|2452638|DEPARTMENT|78|27|True companies might see particularly scottish states. Lesser, cautious forms give however long|quarterly| +8344|AAAAAAAAIJACAAAA|2452548|2452638|DEPARTMENT|78|28|Roads would not need close bizarre, revolutionary packages. Events cont|quarterly| +8345|AAAAAAAAJJACAAAA|2452548|2452638|DEPARTMENT|78|29|New studies record different, german departments. Exciting commentators will not agr|quarterly| +8346|AAAAAAAAKJACAAAA|2452548|2452638|DEPARTMENT|78|30|Working, upper restaurants visit. Old kilometres shall cease|quarterly| +8347|AAAAAAAALJACAAAA|2452548|2452638|DEPARTMENT|78|31|Now original earnings afford periods. Never enormous workers ought to base |quarterly| +8348|AAAAAAAAMJACAAAA|2452548|2452638|DEPARTMENT|78|32|Entirely famous heads criticise so offenders; islands might measure perhaps. Perfect proble|quarterly| +8349|AAAAAAAANJACAAAA|2452548|2452638|DEPARTMENT|78|33|New, statutory priorities must not feel in a councils. Dark years used to read and so|quarterly| +8350|AAAAAAAAOJACAAAA|2452548|2452638|DEPARTMENT|78|34|Things like damp, black premises; generally huge games make. Straight difficult s|quarterly| +8351|AAAAAAAAPJACAAAA|2452548|2452638|DEPARTMENT|78|35|Genetic services will not stop right. Measures provide for a types. Men must escape too |quarterly| +8352|AAAAAAAAAKACAAAA|2452548|2452638|DEPARTMENT|78|36|Doubts go neighbouring, subject applications. Women leave visual powers. Able, average teachers man|quarterly| +8353|AAAAAAAABKACAAAA|2452548|2452638|DEPARTMENT|78|37|Particular, organic sections used to signify just black, fun beliefs; figures will ma|quarterly| +8354|AAAAAAAACKACAAAA|2452548|2452638|DEPARTMENT|78|38|Mothers expect most. Other, high plans would turn in a visitors. Hands ought to pay |quarterly| +8355|AAAAAAAADKACAAAA|2452548|2452638|DEPARTMENT|78|39|Easily only results stay already vertical complaints; fine, mature americans might go |quarterly| +8356|AAAAAAAAEKACAAAA|2452548|2452638|DEPARTMENT|78|40|Industries might accept total values. Obviously left ea|quarterly| +8357|AAAAAAAAFKACAAAA|2452548|2452638|DEPARTMENT|78|41|Places must bring plainly old examples. Violent, legitimate years get never|quarterly| +8358|AAAAAAAAGKACAAAA|2452548|2452638|DEPARTMENT|78|42|Young cuts express again to the letters; estimates may allow independently particular l|quarterly| +8359|AAAAAAAAHKACAAAA|2452548|2452638|DEPARTMENT|78|43|Mass, original cars improve bad, olympic rights. Bright supporters shall lead|quarterly| +8360|AAAAAAAAIKACAAAA|2452548|2452638|DEPARTMENT|78|44|Carefully unknown months treat glad, national offices. Foreigners|quarterly| +8361|AAAAAAAAJKACAAAA|2452548|2452638|DEPARTMENT|78|45|Developments establish on a costs. New, able figures see physical categories; emplo|quarterly| +8362|AAAAAAAAKKACAAAA|2452548|2452638|DEPARTMENT|78|46|Vocational, extensive women start stages. Cats might know practices;|quarterly| +8363|AAAAAAAALKACAAAA|2452548|2452638|DEPARTMENT|78|47|Shares meet similar hours. Local, familiar stars signal eve|quarterly| +8364|AAAAAAAAMKACAAAA|2452548|2452638|DEPARTMENT|78|48|Easy products could meet as worthwhile, only eyes; around classical aff|quarterly| +8365|AAAAAAAANKACAAAA|2452548|2452638|DEPARTMENT|78|49|Cognitive shoes cope close, environmental actions.|quarterly| +8366|AAAAAAAAOKACAAAA|2452548|2452638|DEPARTMENT|78|50|Bad families may make. Current children afford strongly adm|quarterly| +8367|AAAAAAAAPKACAAAA|2452548|2452638|DEPARTMENT|78|51|Able effects commemorate. Occasionally small experiences must not want for a rivals|quarterly| +8368|AAAAAAAAALACAAAA|2452548|2452638|DEPARTMENT|78|52|Certain sorts might not come to a years. Applications could not stay thus. Powerful, g|quarterly| +8369|AAAAAAAABLACAAAA|2452548|2452638|DEPARTMENT|78|53|Tender, british fortunes ought to get regardless unusual emotions. Fast single pro|quarterly| +8370|AAAAAAAACLACAAAA|2452548|2452638|DEPARTMENT|78|54|Large, following sons could crumble even; important, new runs aid to a women. |quarterly| +8371|AAAAAAAADLACAAAA|2452548|2452638|DEPARTMENT|78|55|Beautiful patterns ought to witness fresh woods. Already good fa|quarterly| +8372|AAAAAAAAELACAAAA|2452548||||||| +8373|AAAAAAAAFLACAAAA|2452548|2452638|DEPARTMENT|78|57|Much domestic men refer so. As a matter of fact immediate industries look however r|quarterly| +8374|AAAAAAAAGLACAAAA|2452548|2452638|DEPARTMENT|78|58|Hard, new cells enlarge; christian lives shall return here. |quarterly| +8375|AAAAAAAAHLACAAAA|2452548|2452638|DEPARTMENT|78|59|Under way general teachers must generate more advantages. Urgent, presidential colours might not |quarterly| +8376|AAAAAAAAILACAAAA|2452548|2452638|DEPARTMENT|78|60|Skilled views affect just necessary signs; meetings publish only|quarterly| +8377|AAAAAAAAJLACAAAA|2452548|2452638|DEPARTMENT|78|61|Decisions gain thus on a students; chapters must not come. Most other |quarterly| +8378|AAAAAAAAKLACAAAA|2452548|2452638|DEPARTMENT|78|62|Appropriate police close perhaps cultural decisions. Centuries shall know again indeed onl|quarterly| +8379|AAAAAAAALLACAAAA|2452548|2452638|DEPARTMENT|78|63|Services could work days. Records could not make that is tod|quarterly| +8380|AAAAAAAAMLACAAAA|2452548|2452638|DEPARTMENT|78|64|Only limitations will not discern only. Workers build local name|quarterly| +8381|AAAAAAAANLACAAAA|2452548|2452638|DEPARTMENT|78|65|More foreign hands end finally ago poor christians. Known|quarterly| +8382|AAAAAAAAOLACAAAA|2452548|2452638|DEPARTMENT|78|66|Individuals pay more feet. Also available children shall not ensure even cheap, clear kids. |quarterly| +8383|AAAAAAAAPLACAAAA|2452548|2452638|DEPARTMENT|78|67|Successful services share at a items. Fairly right hours used to win both u|quarterly| +8384|AAAAAAAAAMACAAAA|2452548|2452638|DEPARTMENT|78|68|Intense aspects might interpret little, great letters; theoretical deputies might|quarterly| +8385|AAAAAAAABMACAAAA|2452548|2452638|DEPARTMENT|78|69|Dangerous periods leave softly potential, small amb|quarterly| +8386|AAAAAAAACMACAAAA|2452548|2452638|DEPARTMENT|78|70|Possible parents find western notions. Towns would not challenge countries. T|quarterly| +8387|AAAAAAAADMACAAAA|2452548|2452638|DEPARTMENT|78|71|Light, basic visitors pay dutch, german items. Sentences may stay less. Other|quarterly| +8388|AAAAAAAAEMACAAAA|2452548|2452638|DEPARTMENT|78|72|Thoughts must find. Various, proper elements can see f|quarterly| +8389|AAAAAAAAFMACAAAA||2452638|DEPARTMENT|78|73||quarterly| +8390|AAAAAAAAGMACAAAA|2452548|2452638|DEPARTMENT|78|74|Outside practical numbers strike either. Extreme men say also in a benefits. Maximum, vi|quarterly| +8391|AAAAAAAAHMACAAAA|2452548|2452638|DEPARTMENT|78|75|Easier advisory applications might not bear passengers. Just slight poi|quarterly| +8392|AAAAAAAAIMACAAAA|2452548|2452638|DEPARTMENT|78|76|Irish, elegant circumstances would appreciate yesterday there combined questions. We|quarterly| +8393|AAAAAAAAJMACAAAA|2452548|2452638|DEPARTMENT|78|77|Cruel affairs could seem sure uncomfortable, legislative guns; total, na|quarterly| +8394|AAAAAAAAKMACAAAA|2452548|2452638|DEPARTMENT|78|78|Political, ancient hours report as well ministers. N|quarterly| +8395|AAAAAAAALMACAAAA|2452548|2452638|DEPARTMENT|78|79|Departments get significant, aware times. Forward ne|quarterly| +8396|AAAAAAAAMMACAAAA|2452548|2452638|DEPARTMENT|78|80|So light topics trade subsequent eyes; very able weeks take confidently merely gro|quarterly| +8397|AAAAAAAANMACAAAA|2452548|2452638|DEPARTMENT|78|81|Social, suitable tears hope known grants. Slightly possible visitors produce now. |quarterly| +8398|AAAAAAAAOMACAAAA|2452548|2452638|DEPARTMENT|78|82|Certain, sound children stand now new sounds. Subst|quarterly| +8399|AAAAAAAAPMACAAAA|2452548|2452638|DEPARTMENT|78|83|Other, political projects will not pay just for the inches; payments register prioriti|quarterly| +8400|AAAAAAAAANACAAAA|2452548|2452638|DEPARTMENT|78|84|Schools see further dark, great suppliers. Immediately net grounds buy|quarterly| +8401|AAAAAAAABNACAAAA|2452548|2452638|DEPARTMENT|78|85|Now young children might impose there; mutual seconds will incr|quarterly| +8402|AAAAAAAACNACAAAA|2452548|2452638|DEPARTMENT|78|86|Very markets should not save both separate, small plants; legal officers mean potentia|quarterly| +8403|AAAAAAAADNACAAAA|2452548|2452638|DEPARTMENT|78|87|New laws might not use to a problems. Labour galleries move expensive, visual elections; then |quarterly| +8404|AAAAAAAAENACAAAA|2452548|2452638|DEPARTMENT|78|88|Temporary guests could not discuss new, basic specimens; so maximum sentence|quarterly| +8405|AAAAAAAAFNACAAAA|2452548|2452638|DEPARTMENT|78|89|Contemporary women show between an children. Horses could no|quarterly| +8406|AAAAAAAAGNACAAAA|2452548|2452638|DEPARTMENT|78|90|Words might prevent really. Regional lines supply still rural, vast boxes. Yet regional men will re|quarterly| +8407|AAAAAAAAHNACAAAA|2452548|2452638|DEPARTMENT|78|91|Wonderful, black markets used to know less eventually supposed po|quarterly| +8408|AAAAAAAAINACAAAA|2452548|2452638|DEPARTMENT|78|92|Obvious arms ought to see ever to the roads. Weeks lose by the eggs. Military groups fea|quarterly| +8409|AAAAAAAAJNACAAAA|2452548|2452638|DEPARTMENT|78|93|Services would not increase perhaps impossible banks; weeks establish still|quarterly| +8410|AAAAAAAAKNACAAAA|2452548|2452638|DEPARTMENT|78|94|Doctors will not bring. British, bright organizations could not talk again there other systems. We|quarterly| +8411|AAAAAAAALNACAAAA|2452548|2452638|DEPARTMENT|78|95|Indeed sorry sports bring also constant quarters. Spanish women throw necessarily |quarterly| +8412|AAAAAAAAMNACAAAA|2452548|2452638|DEPARTMENT|78|96|Boundaries ought to work so to a patients. Awkward differences require clearly english, tech|quarterly| +8413|AAAAAAAANNACAAAA|2452548|2452638|DEPARTMENT|78|97|Difficult, new councils order. Enquiries shall go. Facilities handle by a features.|quarterly| +8414|AAAAAAAAONACAAAA|2452548|2452638|DEPARTMENT|78|98|Likely states allow sexually on a children. Russian, christian twins ca|quarterly| +8415|AAAAAAAAPNACAAAA|2452548|2452638|DEPARTMENT|78|99|Mildly full relationships would not rely perhaps reluctant c|quarterly| +8416|AAAAAAAAAOACAAAA|2452548|2452638|DEPARTMENT|78|100|Either other planes appear helplessly now other police. Affairs g|quarterly| +8417|AAAAAAAABOACAAAA|2452548|2452638|DEPARTMENT|78|101|Silent pairs could install enough bright, early studies. Questi|quarterly| +8418|AAAAAAAACOACAAAA|2452548|2452638|DEPARTMENT|78|102|Politicians change only; planned wages could not repay. Events will not speak|quarterly| +8419|AAAAAAAADOACAAAA|2452548|2452638|DEPARTMENT|78|103|Steady services must want besides into a cases. Improvements advise outsid|quarterly| +8420|AAAAAAAAEOACAAAA|2452548|2452638|DEPARTMENT|78|104|Personal, old mechanisms fill sometimes forms. Future, royal parents must confine|quarterly| +8421|AAAAAAAAFOACAAAA|2452548|2452638|DEPARTMENT|78|105|Here adequate facts shall seem ago on a enterprises. More positive interest|quarterly| +8422|AAAAAAAAGOACAAAA|2452548|2452638|DEPARTMENT|78|106|Executives may not serve either during the industries. J|quarterly| +8423|AAAAAAAAHOACAAAA|2452548|2452638|DEPARTMENT|78|107|Different, fine schemes recognise hours; as considerable sources should not locate|quarterly| +8424|AAAAAAAAIOACAAAA|2452548|2452638|DEPARTMENT|78|108|American differences shall not represent over basic, significant needs; strong, new boundaries pay|quarterly| +8425|AAAAAAAAJOACAAAA|2452275|2452304|DEPARTMENT|79|1|Edges see then single regulations. All royal stands maximise common, respon|monthly| +8426|AAAAAAAAKOACAAAA|2452275|2452304|DEPARTMENT|79|2|Intense figures could not experience purely other,|monthly| +8427|AAAAAAAALOACAAAA|2452275|2452304|DEPARTMENT|79|3|Home extensive services could take. Other, favorite instru|monthly| +8428|AAAAAAAAMOACAAAA|2452275|2452304|DEPARTMENT|79|4|Quick, present firms develop incidentally then exact years. Pregnant residents choo|monthly| +8429|AAAAAAAANOACAAAA|2452275|2452304|DEPARTMENT|79|5|National properties shall prepare then new pilots. Post-war positions make rather forms. Writers mu|monthly| +8430|AAAAAAAAOOACAAAA|2452275|2452304|DEPARTMENT|79|6|Matters may justify so wealthy, necessary hands. Early, additional eyes |monthly| +8431|AAAAAAAAPOACAAAA|2452275|2452304|DEPARTMENT|79|7|Leaders receive to the friends. Responsible, modern ministers say institutio|monthly| +8432|AAAAAAAAAPACAAAA|2452275|2452304|DEPARTMENT|79|8|Even genuine men forget still no doubt tragic years. Satisfactory us|monthly| +8433|AAAAAAAABPACAAAA|2452275|2452304|DEPARTMENT|79|9|Contemporary cattle like details. Present executives ensure other years. Conditions could not su|monthly| +8434|AAAAAAAACPACAAAA|2452275|2452304|DEPARTMENT|79|10|Sorry, precious prices bear. Public interests must sta|monthly| +8435|AAAAAAAADPACAAAA|2452275|2452304|DEPARTMENT|79|11|White, rural reports use. Feet used to concentrate carefully by |monthly| +8436|AAAAAAAAEPACAAAA|2452275|2452304|DEPARTMENT|79|12|Final, huge protests tackle often other things. Steps recognise specified, apparent courts. Later |monthly| +8437|AAAAAAAAFPACAAAA|2452275|2452304|DEPARTMENT|79|13|Large arms will see. Likely, national women must not mate larg|monthly| +8438|AAAAAAAAGPACAAAA|2452275|2452304|DEPARTMENT|79|14|Really convenient opinions get hours. Inner, specific roses will |monthly| +8439|AAAAAAAAHPACAAAA|2452275|2452304|DEPARTMENT|79|15|Groups discuss happy officials. Outstanding, royal lives may not provide truly err|monthly| +8440|AAAAAAAAIPACAAAA|2452275|2452304|DEPARTMENT|79|16|Now natural houses go well through an changes; poor, satisfactory guards|monthly| +8441|AAAAAAAAJPACAAAA|2452275|2452304|DEPARTMENT|79|17|Suddenly successful paintings must explore ultimately men.|monthly| +8442|AAAAAAAAKPACAAAA|2452275|2452304|DEPARTMENT|79|18|Remote schools will see more by a men. Right practices listen issues. Successful parts shall c|monthly| +8443|AAAAAAAALPACAAAA|2452275|2452304|DEPARTMENT|79|19|Faintly tired relations differentiate smoothly overall incomes; black contents must b|monthly| +8444|AAAAAAAAMPACAAAA|2452275|2452304|DEPARTMENT|79|20|As well other fingers talk so silent values. Elderly, little police might improve e|monthly| +8445|AAAAAAAANPACAAAA|2452275|2452304|DEPARTMENT|79|21|Certain systems include then good futures. Dirty, sure tons pay courts; personal techn|monthly| +8446|AAAAAAAAOPACAAAA|2452275|2452304|DEPARTMENT|79|22|Important causes represent all aware centuries. Outside, ma|monthly| +8447|AAAAAAAAPPACAAAA|2452275|2452304|DEPARTMENT|79|23|Existing lines say to a times. Refugees would not see there domestic substances.|monthly| +8448|AAAAAAAAAABCAAAA|2452275|2452304|DEPARTMENT|79|24|Different examples would confirm at least poor, human women. So-called|monthly| +8449|AAAAAAAABABCAAAA|2452275|2452304|DEPARTMENT|79|25|Advertisements will go parallel from a results. Very |monthly| +8450|AAAAAAAACABCAAAA|2452275|2452304|DEPARTMENT|79|26|Tired, rough police pick now large, important requirements. |monthly| +8451|AAAAAAAADABCAAAA|2452275|2452304|DEPARTMENT|79|27|Vast looks would get confidently. Grey children drop current, clear claims. Requ|monthly| +8452|AAAAAAAAEABCAAAA|2452275|2452304|DEPARTMENT|79|28|Seldom small effects match then by reason of a schools. Things used to manage just level records.|monthly| +8453|AAAAAAAAFABCAAAA|2452275|2452304||||Women would not take clearly much other photographs. Doors mu|monthly| +8454|AAAAAAAAGABCAAAA|2452275|2452304|DEPARTMENT|79|30|Industrial, suitable teeth should see rather japanese details; balls used t|monthly| +8455|AAAAAAAAHABCAAAA|2452275|2452304|DEPARTMENT|79|31|Small, popular authorities see even on a strings. Illegal, re|monthly| +8456|AAAAAAAAIABCAAAA|2452275|2452304|DEPARTMENT|79|32|Freshly popular hills cannot maintain quickly comments. Clear,|monthly| +8457|AAAAAAAAJABCAAAA|2452275|2452304|DEPARTMENT|79|33|Likely, small concepts shall not live on a comments. Big|monthly| +8458|AAAAAAAAKABCAAAA||||79|34|Free scales might need weapons. Tables carry with a debts. Then corporate || +8459|AAAAAAAALABCAAAA|2452275|2452304|DEPARTMENT|79|35|Arguments ask then french values. Old animals note. Lonel|monthly| +8460|AAAAAAAAMABCAAAA|2452275|2452304|DEPARTMENT|79|36|Only blank notes convert there still industrial branches. Else differen|monthly| +8461|AAAAAAAANABCAAAA|2452275|2452304|DEPARTMENT|79|37|Reasons might think that is welsh services. Social enquiries lose|monthly| +8462|AAAAAAAAOABCAAAA|2452275|2452304|DEPARTMENT|79|38|Hard reasons know chief initiatives. Big losses will not find much r|monthly| +8463|AAAAAAAAPABCAAAA|2452275|2452304|DEPARTMENT|79|39|Incidentally other effects add in the years. Practical sys|monthly| +8464|AAAAAAAAABBCAAAA|2452275|2452304|DEPARTMENT|79|40|Small, different miles would afford sufficient, public incidents. Days ought to get already ser|monthly| +8465|AAAAAAAABBBCAAAA|2452275|2452304|DEPARTMENT|79|41|Competitive, major statements must suggest jolly throug|monthly| +8466|AAAAAAAACBBCAAAA|2452275|2452304|DEPARTMENT|79|42|Forms undertake companies; once silly legs ought to introduc|monthly| +8467|AAAAAAAADBBCAAAA|2452275|2452304|DEPARTMENT|79|43|Activities could come financially silver engines. Requirements |monthly| +8468|AAAAAAAAEBBCAAAA|2452275|2452304|DEPARTMENT|79|44|Expected eggs start surely social aims; instead great years shall think at least still |monthly| +8469|AAAAAAAAFBBCAAAA|2452275|2452304|DEPARTMENT|79|45|As local walls would not make from the facilities. For instance industrial classes would n|monthly| +8470|AAAAAAAAGBBCAAAA|2452275|2452304|DEPARTMENT|79|46|Conditions must trade most local, mathematical others. Dutch children take success|monthly| +8471|AAAAAAAAHBBCAAAA|2452275|2452304|DEPARTMENT|79|47|Major interests ought to keep well chosen goals. F|monthly| +8472|AAAAAAAAIBBCAAAA|2452275|2452304|DEPARTMENT|79|48|Too good laws shall ensure other towns. Vocational readers make relations. |monthly| +8473|AAAAAAAAJBBCAAAA|2452275|2452304|DEPARTMENT|79|49|Popular, small men must doubt differences; indeed open drawings should remember m|monthly| +8474|AAAAAAAAKBBCAAAA|2452275|2452304|DEPARTMENT|79|50|Real, right hours work about only, corporate reservations.|monthly| +8475|AAAAAAAALBBCAAAA|2452275|2452304|DEPARTMENT|79|51|Then considerable orders get then at least possible|monthly| +8476|AAAAAAAAMBBCAAAA|2452275|2452304|DEPARTMENT|79|52|Usually wrong machines shall bring foreign referenc|monthly| +8477|AAAAAAAANBBCAAAA|2452275|2452304|DEPARTMENT|79|53|Later useful legs used to fulfil now confidential musical relations. Goals show;|monthly| +8478|AAAAAAAAOBBCAAAA|2452275|2452304|DEPARTMENT|79|54|Serious, old elements would hit complicated, traditional families; all different t|monthly| +8479|AAAAAAAAPBBCAAAA|2452275|2452304|DEPARTMENT|79|55|Here dry members might change lawyers. Just basic prisoners will save always inter|monthly| +8480|AAAAAAAAACBCAAAA|2452275|2452304|DEPARTMENT|79|56|Regular flights build home warm, old goods. Popular, conscious debts may tell; s|monthly| +8481|AAAAAAAABCBCAAAA|2452275|2452304|DEPARTMENT|79|57|Rates can know from the men. Widely only agreements ought to determi|monthly| +8482|AAAAAAAACCBCAAAA|2452275|2452304|DEPARTMENT|79|58|Poor, white winds would marry risks. Overseas high teeth give well labour app|monthly| +8483|AAAAAAAADCBCAAAA|2452275|2452304|DEPARTMENT|79|59|Severe, fundamental computers would see soon. Presidential needs substitute natura|monthly| +8484|AAAAAAAAECBCAAAA|2452275|2452304|DEPARTMENT|79|60|Either ordinary surveys inhibit probably emotional, opposite children. Foreign drinks clo|monthly| +8485|AAAAAAAAFCBCAAAA|2452275|2452304|DEPARTMENT|79|61|Quick friends tell healthy, likely councils. Minutes may not pay over main, you|monthly| +8486|AAAAAAAAGCBCAAAA|2452275|2452304|DEPARTMENT|79|62|Sections could come possible, new classes; animals ought to tell now l|monthly| +8487|AAAAAAAAHCBCAAAA|2452275|2452304|DEPARTMENT|79|63|Cars think social, top persons. Too spectacular facilities cannot g|monthly| +8488|AAAAAAAAICBCAAAA|2452275|2452304|DEPARTMENT|79|64|Implicit, christian beings brush always weeks. Everyday remains will like|monthly| +8489|AAAAAAAAJCBCAAAA|2452275|2452304|DEPARTMENT|79|65|Changes ought to conduct all otherwise leading consequences; also ordinary cheeks may plan champion|monthly| +8490|AAAAAAAAKCBCAAAA|2452275|2452304|DEPARTMENT|79|66|Unknown, sure courses could cool already sad, secondary functions. Further terr|monthly| +8491|AAAAAAAALCBCAAAA|2452275|2452304|DEPARTMENT|79|67|Important experiences oppose there in a rules. Travellers might not work dramatically fu|monthly| +8492|AAAAAAAAMCBCAAAA|2452275|2452304|DEPARTMENT|79|68|Most private differences can say hardly. Finally perso|monthly| +8493|AAAAAAAANCBCAAAA|2452275|2452304|DEPARTMENT|79|69|Underlying, different levels think else; victorian, real leaders join successful, necessary years. |monthly| +8494|AAAAAAAAOCBCAAAA|2452275|2452304|DEPARTMENT|79|70|Products should not need. Today overall years would mean proudly results. Main games want more|monthly| +8495|AAAAAAAAPCBCAAAA|2452275|2452304|DEPARTMENT|79|71|Children would not stop reasons; perfect meetings appreciate there|monthly| +8496|AAAAAAAAADBCAAAA|2452275|2452304|DEPARTMENT|79|72|Interested wives might see more powerful, particular matters. New, large values attract simp|monthly| +8497|AAAAAAAABDBCAAAA|2452275|2452304|DEPARTMENT|79|73|Normal, normal points achieve across the qualities. Dir|monthly| +8498|AAAAAAAACDBCAAAA|2452275|2452304|DEPARTMENT|79|74|Emotions may address in order; words should insist now in a stairs. At last fine students c|monthly| +8499|AAAAAAAADDBCAAAA|2452275|2452304|DEPARTMENT|79|75|Less new reasons used to halt high as well mental observation|monthly| +8500|AAAAAAAAEDBCAAAA|2452275|2452304|DEPARTMENT|79|76|Flat readers must not try at the songs. National, right marks recur then more char|monthly| +8501|AAAAAAAAFDBCAAAA|2452275|2452304|DEPARTMENT|79|77|English, social rates may increase on a aspects. S|monthly| +8502|AAAAAAAAGDBCAAAA|2452275|2452304|DEPARTMENT|79|78|Sure blues prove loud firm committees. Well african regulations revise internal, various words. Po|monthly| +8503|AAAAAAAAHDBCAAAA|2452275|2452304|DEPARTMENT|79|79|Enthusiastically special interests help indeed from a combi|monthly| +8504|AAAAAAAAIDBCAAAA|2452275|2452304|DEPARTMENT|79|80|Coming, quick cases shall not permit yesterday. Foreign, british mountains sh|monthly| +8505|AAAAAAAAJDBCAAAA|2452275|2452304|DEPARTMENT|79|81|Easy, elegant months could predict hours. All the same arbitrary activities us|monthly| +8506|AAAAAAAAKDBCAAAA|2452275|2452304|DEPARTMENT|79|82|Then critical consequences discern positive answers; earlier silent drivers disrupt so. Large in|monthly| +8507|AAAAAAAALDBCAAAA|2452275|2452304|||83||| +8508|AAAAAAAAMDBCAAAA|2452275|2452304|DEPARTMENT|79|84|Groups will visit only from a recommendations. Soviet, n|monthly| +8509|AAAAAAAANDBCAAAA|2452275|2452304|DEPARTMENT|79|85|Feet must choose solely just european parties. Correct humans must n|monthly| +8510|AAAAAAAAODBCAAAA|2452275|2452304|DEPARTMENT|79|86|Therefore following cases shall eat activities. Reports can iden|monthly| +8511|AAAAAAAAPDBCAAAA|2452275|2452304|DEPARTMENT|79|87|New designers think particularly yesterday actual thanks. Responsible problems take indeed compete|monthly| +8512|AAAAAAAAAEBCAAAA|2452275|2452304|DEPARTMENT|79|88|Fine, helpful jobs might not escape still beaches. So spanish effects g|monthly| +8513|AAAAAAAABEBCAAAA|2452275|2452304|DEPARTMENT|79|89|Simply golden laboratories quantify socially; small, main plans should succumb. Obvious, common sen|monthly| +8514|AAAAAAAACEBCAAAA|2452275|2452304|DEPARTMENT|79|90|Up to only minutes learn neat details. Artists will approve nearly from a re|monthly| +8515|AAAAAAAADEBCAAAA|2452275|2452304|DEPARTMENT|79|91|Teachers will feature instead of the effects; local, sure others can mind en|monthly| +8516|AAAAAAAAEEBCAAAA|2452275|2452304|DEPARTMENT|79|92|Together vital children receive today so political parties. Spanish librari|monthly| +8517|AAAAAAAAFEBCAAAA|2452275|2452304|DEPARTMENT|79|93|Labour, traditional thanks might occur ago with a amounts. Boundaries get as. Experimen|monthly| +8518|AAAAAAAAGEBCAAAA|2452275|2452304|DEPARTMENT|79|94|Allowances might manage foreign, total materials. Enough trying subjects leave then|monthly| +8519|AAAAAAAAHEBCAAAA|2452275|2452304|DEPARTMENT|79|95|More small years compete actually relations. Criteria trace males. Just |monthly| +8520|AAAAAAAAIEBCAAAA|2452275|2452304|DEPARTMENT|79|96|Material consequences carry consumers; labour holes beat expertly up a victims. Additional,|monthly| +8521|AAAAAAAAJEBCAAAA|2452275|2452304|DEPARTMENT|79|97|Already liberal ports tell extremely elderly rules. Sources would conduct almost services. Odd, o|monthly| +8522|AAAAAAAAKEBCAAAA|2452275|2452304|DEPARTMENT|79|98|Eagerly marginal causes stay other, legitimate universities. Faint cases would drop in a|monthly| +8523|AAAAAAAALEBCAAAA|2452275|2452304|DEPARTMENT|79|99|More clear goods support after a areas. Only other universities cannot like. |monthly| +8524|AAAAAAAAMEBCAAAA|2452275|2452304|DEPARTMENT|79|100|Lovely drinks wear much good exchanges. Annual, other thoughts can explain wait|monthly| +8525|AAAAAAAANEBCAAAA|2452275|2452304|DEPARTMENT|79|101|Occasionally external policemen attend at last fro|monthly| +8526|AAAAAAAAOEBCAAAA|2452275|2452304|DEPARTMENT|79|102|Rather little heads used to prepare particularly american deaf competitors. Royal, alone v|monthly| +8527|AAAAAAAAPEBCAAAA|2452275|2452304|DEPARTMENT|79|103|New, obvious costs find wet, social causes. Yesterday|monthly| +8528|AAAAAAAAAFBCAAAA|2452275|2452304|DEPARTMENT|79|104|Little, electrical women shall dare only top, good facilities|monthly| +8529|AAAAAAAABFBCAAAA|2452275|2452304|DEPARTMENT|79|105|Parliamentary plants shall behave completely years. Really clear funds will help toget|monthly| +8530|AAAAAAAACFBCAAAA|2452275|2452304|DEPARTMENT|79|106|Political parts ought to see terms. Similar, working responses kill then similar doctor|monthly| +8531|AAAAAAAADFBCAAAA|2452275|2452304|DEPARTMENT|79|107|Royal matters may not seem very direct users. Asset|monthly| +8532|AAAAAAAAEFBCAAAA|2452275|2452304|DEPARTMENT|79|108|There following foods run only yet other crews. Long year|monthly| +8533|AAAAAAAAFFBCAAAA|2452305|2452334|DEPARTMENT|80|1|Courses could reduce efficiently less unique ears. Examinations may carry with a services. |monthly| +8534|AAAAAAAAGFBCAAAA|2452305|2452334|DEPARTMENT|80|2|Police make comfortable, narrow sites. Policies involve; long-term pref|monthly| +8535|AAAAAAAAHFBCAAAA|2452305|2452334|DEPARTMENT|80|3|Well various studies bring oddly in the funds; provincial, old projects know. Future|monthly| +8536|AAAAAAAAIFBCAAAA|2452305|2452334|DEPARTMENT|80|4|Complicated facilities must relax originally rich employers. L|monthly| +8537|AAAAAAAAJFBCAAAA|2452305|2452334|DEPARTMENT|80|5|Managers start so early applications. Technical, other services find as well absolut|monthly| +8538|AAAAAAAAKFBCAAAA|2452305|2452334|DEPARTMENT|80|6|Problems cannot laugh recently. Wild unions insist by a feet. Western things|monthly| +8539|AAAAAAAALFBCAAAA|2452305|2452334|DEPARTMENT|80|7|Topics make later classical, other hands. Considerable variations read from a months. Activ|monthly| +8540|AAAAAAAAMFBCAAAA|2452305|2452334|DEPARTMENT|80|8|Specifically strange options follow there about the |monthly| +8541|AAAAAAAANFBCAAAA|2452305|2452334|DEPARTMENT|80|9|Other, certain months know there at all final systems. Again relative terms record carefully regul|monthly| +8542|AAAAAAAAOFBCAAAA|2452305|2452334|DEPARTMENT|80|10|Most ancient questions cannot forget usually unnecessary police. The|monthly| +8543|AAAAAAAAPFBCAAAA|2452305|2452334|DEPARTMENT|80|11|Ways may love sweet principles. Furious, exotic groups get. Services ensure wild|monthly| +8544|AAAAAAAAAGBCAAAA|2452305|2452334|DEPARTMENT|80|12|Novels incorporate then rights. Special, russian pressures can|monthly| +8545|AAAAAAAABGBCAAAA|2452305|2452334|DEPARTMENT|80|13|Later happy visitors can pass so parental, inc systems; |monthly| +8546|AAAAAAAACGBCAAAA|2452305|2452334|DEPARTMENT|80|14|Poor ways know acutely weekly proposals. Beliefs wo|monthly| +8547|AAAAAAAADGBCAAAA|2452305|2452334|DEPARTMENT|80|15|Future sentences might not need eagerly. Companies would afford persona|monthly| +8548|AAAAAAAAEGBCAAAA|2452305|2452334|DEPARTMENT|80|16|British heads ought to devote. Reasonable, british months should drop quick|monthly| +8549|AAAAAAAAFGBCAAAA|2452305|2452334|DEPARTMENT|80|17|Short players would say else important stars; quite local companies might g|monthly| +8550|AAAAAAAAGGBCAAAA|2452305|2452334|DEPARTMENT|80|18|Birds should come single practices; too powerful det|monthly| +8551|AAAAAAAAHGBCAAAA|2452305|2452334|DEPARTMENT|80|19|Minor babies shall sleep vehemently internal men. Concrete, environmental fingers filter|monthly| +8552|AAAAAAAAIGBCAAAA|2452305|2452334|DEPARTMENT|80|20|Professional, early players pay sometimes. Scottish, natural plans ought to vary now too great men.|monthly| +8553|AAAAAAAAJGBCAAAA|2452305|2452334|DEPARTMENT|80|21|Free, new relations shall not wear little. Head, other fields convince mutual, good obligations; l|monthly| +8554|AAAAAAAAKGBCAAAA|2452305|2452334|DEPARTMENT|80|22|Other ways can examine from the things. Chiefly furious authorities m|monthly| +8555|AAAAAAAALGBCAAAA|2452305|2452334|DEPARTMENT|80|23|Especially commercial words pay markedly. Able, permanent readers think high. S|monthly| +8556|AAAAAAAAMGBCAAAA|2452305|2452334|DEPARTMENT|80|24|Simply new women shall become already silly happy chapters. |monthly| +8557|AAAAAAAANGBCAAAA|2452305|2452334|DEPARTMENT|80|25|Now free children stir alone, political feet. Theoretical securities cann|monthly| +8558|AAAAAAAAOGBCAAAA|2452305|2452334|DEPARTMENT|80|26|Years might contain there real samples; efforts produce for exam|monthly| +8559|AAAAAAAAPGBCAAAA|2452305|2452334|DEPARTMENT|80|27|Small, great pages eliminate for example liabilities. Minutes used to demonstra|monthly| +8560|AAAAAAAAAHBCAAAA|2452305|2452334|DEPARTMENT|80|28|Only prime securities would tell as more official illustrations. Large, high stu|monthly| +8561|AAAAAAAABHBCAAAA|2452305|2452334|DEPARTMENT|80|29|Small, successful days see alternatively more other commentators. Artists should not get|monthly| +8562|AAAAAAAACHBCAAAA|2452305|2452334|DEPARTMENT|80|30|Methods must accept increasingly certain, regional schools. Small, civil |monthly| +8563|AAAAAAAADHBCAAAA|2452305|2452334|DEPARTMENT|80|31|Special stations emphasise in a christians. Basic friends dive much useless, kee|monthly| +8564|AAAAAAAAEHBCAAAA|2452305|2452334|DEPARTMENT|80|32|Funny, dangerous relations will make most to a questions. Pressures stop thereby |monthly| +8565|AAAAAAAAFHBCAAAA|2452305|2452334|DEPARTMENT|80|33|Shoes know secondary, fortunate troops. Troops provide ma|monthly| +8566|AAAAAAAAGHBCAAAA|2452305|2452334|DEPARTMENT|80|34|Holidays offer front parameters. Immediately social foundations furnish almost s|monthly| +8567|AAAAAAAAHHBCAAAA|2452305|2452334|DEPARTMENT|80|35|Admittedly satisfactory students serve softly women; american, present investm|monthly| +8568|AAAAAAAAIHBCAAAA|2452305|2452334|DEPARTMENT|80|36|Public, young words mend at the customs. Best combined pounds allow in short lo|monthly| +8569|AAAAAAAAJHBCAAAA|2452305|2452334|DEPARTMENT|80|37|Hours may not ensure again long investigations. In |monthly| +8570|AAAAAAAAKHBCAAAA|2452305|2452334|DEPARTMENT|80|38|Only english companies look almost temporary, soci|monthly| +8571|AAAAAAAALHBCAAAA|2452305|2452334|DEPARTMENT|80|39|Good, fine terms keep main, other criticisms. Darling, new years will not raise typically ol|monthly| +8572|AAAAAAAAMHBCAAAA|2452305|2452334|DEPARTMENT|80|40|Wooden securities ought to cover full-time police. New guns descend to a fl|monthly| +8573|AAAAAAAANHBCAAAA|2452305|2452334|DEPARTMENT|80|41|Still political files will divide. Previous, marvellous villages shall make almost sh|monthly| +8574|AAAAAAAAOHBCAAAA|2452305|2452334|DEPARTMENT|80|42|Centres should not suggest personnel. Usually red skill|monthly| +8575|AAAAAAAAPHBCAAAA|2452305|2452334|DEPARTMENT|80|43|Other, severe forces separate previous titles. Social, small months go even hour|monthly| +8576|AAAAAAAAAIBCAAAA|2452305|2452334|DEPARTMENT|80|44|Thus working skills might damage in a techniques. Easy, burning employees hear as; major, im|monthly| +8577|AAAAAAAABIBCAAAA|2452305|2452334|DEPARTMENT|80|45|Birds rain. Seconds shall give alternatively difficult things; esp|monthly| +8578|AAAAAAAACIBCAAAA|2452305|2452334|DEPARTMENT|80|46|Seconds might cause surely rural measures. Guests shall not press; primary, green u|monthly| +8579|AAAAAAAADIBCAAAA|2452305|2452334|DEPARTMENT|80|47|Mass, real missiles might use enough less important years; sensible territories see |monthly| +8580|AAAAAAAAEIBCAAAA|2452305|2452334|DEPARTMENT|80|48|Writings might not display local, human profits. Corners ask either final criteria. M|monthly| +8581|AAAAAAAAFIBCAAAA|2452305|2452334|DEPARTMENT|80|49|British children would challenge sooner broad, negative developments. |monthly| +8582|AAAAAAAAGIBCAAAA|2452305|2452334|DEPARTMENT|80|50|Often conventional topics would find above, other services. Excessive projects arrive publ|monthly| +8583|AAAAAAAAHIBCAAAA|2452305|2452334|DEPARTMENT|80|51|Responsible statements treat; local, eastern rewards prove. Other, national agen|monthly| +8584|AAAAAAAAIIBCAAAA|2452305|2452334|DEPARTMENT|80|52|In general written solutions shall generate hands. Countries may stick new measures. Problems |monthly| +8585|AAAAAAAAJIBCAAAA|2452305|2452334|DEPARTMENT|80|53|Etc true methods must know major merchants; however foreign figures would interrupt. Vari|monthly| +8586|AAAAAAAAKIBCAAAA|2452305|2452334|DEPARTMENT|80|54|Written, digital applicants can listen better mere men. Blocks l|monthly| +8587|AAAAAAAALIBCAAAA|2452305|2452334|DEPARTMENT|80|55|Other officials may wonder. Social, hostile roads ought to strengthen eventually true provi|monthly| +8588|AAAAAAAAMIBCAAAA|2452305|2452334|DEPARTMENT|80|56|Gay sides meet harsh gods. Able, sole offences measure most necessary order|monthly| +8589|AAAAAAAANIBCAAAA|2452305|2452334|DEPARTMENT|80|57|Other plates may see even standards. Only tickets will entertain more norma|monthly| +8590|AAAAAAAAOIBCAAAA|2452305|2452334|DEPARTMENT|80|58|Slightly sensitive workers could tell less highly legal p|monthly| +8591|AAAAAAAAPIBCAAAA|2452305|2452334|DEPARTMENT|80|59|Old, famous lips could not consider about conscious months. Old levels will lift; christian dogs su|monthly| +8592|AAAAAAAAAJBCAAAA|2452305|2452334|DEPARTMENT|80|60|Low, chief girls may not go almost bright heads. Lost, able feelings would not|monthly| +8593|AAAAAAAABJBCAAAA|2452305|2452334|DEPARTMENT|80|61|Points may not prevent aware, mad foreigners. Different women may co|monthly| +8594|AAAAAAAACJBCAAAA|2452305|2452334|DEPARTMENT|80|62|Then new trousers ought to keep again again local memories. Central bird|monthly| +8595|AAAAAAAADJBCAAAA|2452305|2452334|DEPARTMENT|80|63|Forms shall lend sensible minutes. Questions could not want also known, basic ser|monthly| +8596|AAAAAAAAEJBCAAAA|2452305|2452334|DEPARTMENT|80|64|Formal, white results would try beforehand different, obvious copies; outsi|monthly| +8597|AAAAAAAAFJBCAAAA|2452305|||80|65||monthly| +8598|AAAAAAAAGJBCAAAA|2452305|2452334|DEPARTMENT|80|66|Enough social courses like enough grey markets. Parliamentary ears |monthly| +8599|AAAAAAAAHJBCAAAA|2452305|2452334|DEPARTMENT|80|67|Afraid signs vote much following methods; small duties walk to a females. Truly s|monthly| +8600|AAAAAAAAIJBCAAAA|2452305|2452334|DEPARTMENT|80|68|Military, working sessions might transfer new, unique lives. Colonial years|monthly| +8601|AAAAAAAAJJBCAAAA|2452305|2452334|DEPARTMENT|80|69|Certain, other teeth reduce primarily annual, satisfactory acres; cases travel just p|monthly| +8602|AAAAAAAAKJBCAAAA|2452305|2452334|DEPARTMENT|80|70|True, whole sheets appear businessmen. Conditions would get|monthly| +8603|AAAAAAAALJBCAAAA|2452305|2452334|DEPARTMENT|80|71|New, physical jobs nod most foreign, nuclear cells. Metho|monthly| +8604|AAAAAAAAMJBCAAAA|2452305|2452334|DEPARTMENT|80|72|Slight, standard molecules bat prior details. Around important lessons break silly n|monthly| +8605|AAAAAAAANJBCAAAA|2452305|2452334|DEPARTMENT|80|73|Political, old differences begin sure current days. There linguistic|monthly| +8606|AAAAAAAAOJBCAAAA|2452305|2452334|DEPARTMENT|80|74|Obvious rights shall not suffer. Big, fine minutes will not think high t|monthly| +8607|AAAAAAAAPJBCAAAA|2452305|2452334|DEPARTMENT|80|75|Positive lives find remarkably chinese numbers. Gr|monthly| +8608|AAAAAAAAAKBCAAAA|2452305|2452334|DEPARTMENT|80|76|Easy, only provisions must implement best legal, lovely |monthly| +8609|AAAAAAAABKBCAAAA|2452305|2452334|DEPARTMENT|80|77|Easy, old homes ought to want students. Hours provide there|monthly| +8610|AAAAAAAACKBCAAAA|2452305|2452334|DEPARTMENT|80|78|Drily dark sums would give allegedly technical cases; stable, occasional bodies|monthly| +8611|AAAAAAAADKBCAAAA|2452305|2452334|DEPARTMENT|80|79|Particularly irish gardens would tell also already european members; already elected lessons thin|monthly| +8612|AAAAAAAAEKBCAAAA|2452305|2452334|DEPARTMENT|80|80|Financial shops obtain all polish pupils. Agents may debate particularly|monthly| +8613|AAAAAAAAFKBCAAAA||||80||Female days consent yellow rules. Primary, russian fortunes shall not fet|monthly| +8614|AAAAAAAAGKBCAAAA|2452305|2452334|DEPARTMENT|80|82|Phrases must not keep. Direct speakers look only to a voices. Su|monthly| +8615|AAAAAAAAHKBCAAAA|2452305|2452334|DEPARTMENT|80|83|Just other differences cannot mind pretty little equations. Countries could tell as. Fairly oth|monthly| +8616|AAAAAAAAIKBCAAAA|2452305|2452334|DEPARTMENT|80|84|Very cultural points must bother monthly, managerial genes. Classical, british problems |monthly| +8617|AAAAAAAAJKBCAAAA|2452305|2452334|DEPARTMENT|80|85|Royal, wealthy employees draw boxes. Figures should repay effectivel|monthly| +8618|AAAAAAAAKKBCAAAA|2452305|2452334|DEPARTMENT|80|86|No vocational crews may find properly now british mothers. Here bottom years mig|monthly| +8619|AAAAAAAALKBCAAAA|2452305|2452334|DEPARTMENT|80|87|Acceptable, formal windows might not discuss far false, early methods. Centra|monthly| +8620|AAAAAAAAMKBCAAAA|2452305|2452334|DEPARTMENT|80|88|Great shareholders must not provide years; places want as no longer tired sh|monthly| +8621|AAAAAAAANKBCAAAA|2452305|2452334|DEPARTMENT|80|89|Mean houses may avoid critical issues. Unlike adults try arts. Gay, underl|monthly| +8622|AAAAAAAAOKBCAAAA|2452305|2452334|DEPARTMENT|80|90|Local losses work students. Decisive opportunities go undoubtedly years. Even |monthly| +8623|AAAAAAAAPKBCAAAA|2452305|2452334|DEPARTMENT|80|91|Later extraordinary communities will get then open, great sides. Later certain members shall not|monthly| +8624|AAAAAAAAALBCAAAA|2452305|2452334|DEPARTMENT|80|92|Guidelines enable on the stages. Warm, new forms build very afterwards aver|monthly| +8625|AAAAAAAABLBCAAAA|2452305|2452334|DEPARTMENT|80|93|Long combinations can encourage separate attacks; days reme|monthly| +8626|AAAAAAAACLBCAAAA|2452305|2452334|DEPARTMENT|80|94|Old possibilities imagine enough. At all clear demands should not reflect then to a att|monthly| +8627|AAAAAAAADLBCAAAA|2452305|2452334|DEPARTMENT|80|95|Drugs would accommodate simply contracts. Forests could re|monthly| +8628|AAAAAAAAELBCAAAA|2452305|2452334|DEPARTMENT|80|96|Before social nations must work slightly comprehen|monthly| +8629|AAAAAAAAFLBCAAAA|2452305|2452334|DEPARTMENT|80|97|Empty miles investigate bills. Various, inc fingers may await out of the authorities. Bonds r|monthly| +8630|AAAAAAAAGLBCAAAA|2452305|2452334|DEPARTMENT|80|98|Products send unlike, american claims; local, sharp keys say. Both sure women shall not exami|monthly| +8631|AAAAAAAAHLBCAAAA|2452305|2452334|DEPARTMENT|80|99|Nearly full stairs paint human, wonderful months. Other months look boats. Fields sh|monthly| +8632|AAAAAAAAILBCAAAA|2452305|2452334|DEPARTMENT|80|100|Actually small reforms view units; infrequently sheer statements must fail|monthly| +8633|AAAAAAAAJLBCAAAA|2452305|2452334|DEPARTMENT|80|101|Other, strategic years could drag passive, medium months. |monthly| +8634|AAAAAAAAKLBCAAAA|2452305|2452334|DEPARTMENT|80|102|Unable products must not throw active, expert layers. Days may contain urgently f|monthly| +8635|AAAAAAAALLBCAAAA|2452305|2452334|DEPARTMENT|80|103|Large, past resources pause directly. Commercial friends fill on|monthly| +8636|AAAAAAAAMLBCAAAA|2452305|2452334|DEPARTMENT|80|104|Principal islands break usually for the charts. Equally persistent imports sell from the th|monthly| +8637|AAAAAAAANLBCAAAA|2452305|2452334|DEPARTMENT|80|105|National police say a little authorities. Immediate lips can feed somewhat|monthly| +8638|AAAAAAAAOLBCAAAA|2452305|2452334|DEPARTMENT|80|106|Blue players could not keep most on the miles. Mil|monthly| +8639|AAAAAAAAPLBCAAAA|2452305|2452334|DEPARTMENT|80|107|Groups cut more french ranks. New, fast problems create splendid, demanding constraints; temporary|monthly| +8640|AAAAAAAAAMBCAAAA|2452305|2452334|DEPARTMENT|80|108|Statutory, upper committees assess police. Significant experiences would build above|monthly| +8641|AAAAAAAABMBCAAAA|2452335|2452364|DEPARTMENT|81|1|All military months seem still families. Minutes cannot complain significant tex|monthly| +8642|AAAAAAAACMBCAAAA|2452335|2452364|DEPARTMENT|81|2|There possible stairs offer here just long jeans. Times must not keep from a recordings. Sign|monthly| +8643|AAAAAAAADMBCAAAA|2452335|2452364|DEPARTMENT|81|3|Difficult others accord from a days. Wholly formal days eat. Soun|monthly| +8644|AAAAAAAAEMBCAAAA|2452335|2452364|DEPARTMENT|81|4|Absolutely significant others would break left damages. Retail, honest animals would cure partiall|monthly| +8645|AAAAAAAAFMBCAAAA|2452335|2452364|DEPARTMENT|81|5|Dead, political boundaries pull too secondary mana|monthly| +8646|AAAAAAAAGMBCAAAA|2452335|2452364|DEPARTMENT|81|6|Distinct, local churches play genetic, national animals; more happy things |monthly| +8647|AAAAAAAAHMBCAAAA|2452335|2452364|DEPARTMENT|81|7|Years ought to take often democrats. Representative, extra differenc|monthly| +8648|AAAAAAAAIMBCAAAA|2452335|2452364|DEPARTMENT|81|8|Relevant, public benefits could show however dangerous aspects; parents shall not look nev|monthly| +8649|AAAAAAAAJMBCAAAA|2452335|2452364|DEPARTMENT|81|9|Mysterious, straight sciences form. Always only individuals used to exhibit join|monthly| +8650|AAAAAAAAKMBCAAAA|2452335|2452364|DEPARTMENT|81|10|Obvious, successful criteria should read most deep, precise boys. Appeals us|monthly| +8651|AAAAAAAALMBCAAAA|2452335|2452364|DEPARTMENT|81|11|Ago passive adults occupy old officials. High goals invol|monthly| +8652|AAAAAAAAMMBCAAAA|2452335|2452364|DEPARTMENT|81|12|Important children ought to get. Alone colleges can walk further expenses. Brit|monthly| +8653|AAAAAAAANMBCAAAA|2452335|2452364|DEPARTMENT|81|13|Unnecessarily unusual jeans cannot race revenues. Bitter, labour |monthly| +8654|AAAAAAAAOMBCAAAA|2452335|2452364|DEPARTMENT|81|14|Legs could build much recent, large trees. Regular, abstract servi|monthly| +8655|AAAAAAAAPMBCAAAA|2452335|2452364|DEPARTMENT|81|15|Full waves shall contact figures. Even neat members should alter never|monthly| +8656|AAAAAAAAANBCAAAA|2452335|2452364|DEPARTMENT|81|16|Military police could not argue operations. Traditiona|monthly| +8657|AAAAAAAABNBCAAAA|2452335|2452364|DEPARTMENT|81|17|Personal offences earn others. Men achieve and so on occasio|monthly| +8658|AAAAAAAACNBCAAAA|2452335|2452364|DEPARTMENT|81|18|Soft, high others keep particularly ahead public times. Probably willing candid|monthly| +8659|AAAAAAAADNBCAAAA|2452335|2452364|DEPARTMENT|81|19|Ideas used to mention. Details will depend finally with the relatives; children|monthly| +8660|AAAAAAAAENBCAAAA|2452335|2452364|DEPARTMENT|81|20|Ever informal fans come now systems. Nearer remaining countries should prevent margins. Clear, |monthly| +8661|AAAAAAAAFNBCAAAA|2452335|2452364|DEPARTMENT|81|21|Powers shall follow british papers. In addition high deposits love more complaints. Long-term |monthly| +8662|AAAAAAAAGNBCAAAA|2452335|2452364|DEPARTMENT|81|22|Other, serious stories should give to a fingers. Me|monthly| +8663|AAAAAAAAHNBCAAAA|2452335|2452364|DEPARTMENT|81|23|Thick students ignore already to the sports. Effecti|monthly| +8664|AAAAAAAAINBCAAAA|2452335|2452364|DEPARTMENT|81|24|Quite unusual researchers could involve. Large, cultural|monthly| +8665|AAAAAAAAJNBCAAAA|2452335|2452364|DEPARTMENT|81|25|Days might come romantic, large ways. About foreign charges |monthly| +8666|AAAAAAAAKNBCAAAA|2452335|2452364|DEPARTMENT|81|26|There small individuals take others. Angry others ought to lunch. Estimated |monthly| +8667|AAAAAAAALNBCAAAA|2452335|2452364|DEPARTMENT|81|27|Compulsory adults ought to know on a remains. Continually certain r|monthly| +8668|AAAAAAAAMNBCAAAA|2452335|2452364|DEPARTMENT|81|28|Influences would sit internal laws. Recent offices used to help nava|monthly| +8669|AAAAAAAANNBCAAAA|2452335|2452364|DEPARTMENT|81|29|Small, complex shares change wooden, major results. Other computers will l|monthly| +8670|AAAAAAAAONBCAAAA|2452335|2452364|DEPARTMENT|81|30|Domestic, clinical women analyse sometimes slim, major agencies. Low, mental devices |monthly| +8671|AAAAAAAAPNBCAAAA|2452335|2452364|DEPARTMENT|81|31|Political children shall compromise away significantly safe equations. Corpo|monthly| +8672|AAAAAAAAAOBCAAAA|2452335|2452364|DEPARTMENT|81|32|Straight looks encourage apparently together daily structures. Main, si|monthly| +8673|AAAAAAAABOBCAAAA|2452335|2452364|DEPARTMENT|81|33|Also british cells may think here new other places. Elsewhere exclusi|monthly| +8674|AAAAAAAACOBCAAAA|2452335|2452364|DEPARTMENT|81|34|About tired types shall not highlight over right, different rights. Young, following others create|monthly| +8675|AAAAAAAADOBCAAAA|2452335|2452364|DEPARTMENT|81|35|National friends provide across; sympathetic relations should understand since governors. Whole, u|monthly| +8676|AAAAAAAAEOBCAAAA|2452335|2452364|DEPARTMENT|81|36|Responsible profits shall get rather critical others; yet different leaders finance most.|monthly| +8677|AAAAAAAAFOBCAAAA|2452335|2452364|DEPARTMENT|81|37|Difficult frames grasp too contemporary grounds. Much o|monthly| +8678|AAAAAAAAGOBCAAAA|2452335|2452364|DEPARTMENT|81|38|Dogs invest classic molecules. Literary, faint units will give nice talks. Conditions treat |monthly| +8679|AAAAAAAAHOBCAAAA|2452335|2452364|DEPARTMENT|81|39|New attacks maintain assets; gently cognitive figures could take |monthly| +8680|AAAAAAAAIOBCAAAA|2452335|2452364|DEPARTMENT|81|40|Keen, personal points may not clear to a problems. Cognitive stairs shall e|monthly| +8681|AAAAAAAAJOBCAAAA|2452335|2452364|DEPARTMENT|81|41|All right grey clubs may not study moving, rare ideas. Elderly, agricultural marks ought to win|monthly| +8682|AAAAAAAAKOBCAAAA|2452335|2452364|DEPARTMENT|81|42|Too prospective employees tackle by a hours. Opportuniti|monthly| +8683|AAAAAAAALOBCAAAA|2452335|2452364|DEPARTMENT|81|43|More than short services will turn problems. Even special years would not|monthly| +8684|AAAAAAAAMOBCAAAA|2452335|2452364|DEPARTMENT|81|44|Dead doubts shall earn as both bizarre men. Quite dramatic others meet; current ac|monthly| +8685|AAAAAAAANOBCAAAA|2452335|2452364|DEPARTMENT|81|45|Figures exercise enough on a hearts. Then new speakers sha|monthly| +8686|AAAAAAAAOOBCAAAA|2452335|2452364|DEPARTMENT|81|46|Again previous boys ought to visit severely general, other decis|monthly| +8687|AAAAAAAAPOBCAAAA|2452335|2452364|DEPARTMENT|81|47|Scottish objects could achieve users. Scottish words could bring anyway recent children. Ver|monthly| +8688|AAAAAAAAAPBCAAAA|2452335|2452364|DEPARTMENT|81|48|Systems ought to profit despite a sets; patient, electron|monthly| +8689|AAAAAAAABPBCAAAA|2452335|2452364|DEPARTMENT|81|49|Young, equal russians want on a ideas. Horrible fields could|monthly| +8690|AAAAAAAACPBCAAAA|2452335|2452364|DEPARTMENT|81|50|Glad arrangements shall disturb for instance. Vehi|monthly| +8691|AAAAAAAADPBCAAAA|2452335|2452364|DEPARTMENT|81|51|Just obvious adults get gay, important judges; annual, genuine jobs learn for a layers. Surf|monthly| +8692|AAAAAAAAEPBCAAAA|2452335|2452364|DEPARTMENT|81|52|Now structural departments guard. Bad, other students offer. Particular firms could|monthly| +8693|AAAAAAAAFPBCAAAA|2452335|2452364|DEPARTMENT|81|53|Senior, historical grounds should collect about. General, other opportunities face brightly priva|monthly| +8694|AAAAAAAAGPBCAAAA|2452335|2452364|DEPARTMENT|81|54|Scottish conditions may not give hard, other homes. |monthly| +8695|AAAAAAAAHPBCAAAA|2452335|2452364|DEPARTMENT|81|55|Formal demands know soon much following differences; duly dependent applicants used|monthly| +8696|AAAAAAAAIPBCAAAA|2452335|2452364|DEPARTMENT|81|56|Ago little miles claim quicker; hands ought to locate that i|monthly| +8697|AAAAAAAAJPBCAAAA|2452335|2452364|DEPARTMENT|81|57|High plants could read recently genes. Common homes cut then good calls. Private results get alwa|monthly| +8698|AAAAAAAAKPBCAAAA|2452335|2452364|DEPARTMENT|81|58|Places stick seldom regular proposals. Far days tak|monthly| +8699|AAAAAAAALPBCAAAA|2452335|2452364|DEPARTMENT|81|59|Passages pass nevertheless white, acceptable elements. Di|monthly| +8700|AAAAAAAAMPBCAAAA|2452335|2452364|DEPARTMENT|81|60|Southern, white units seem to a flowers. For example positive owners would not preserve hea|monthly| +8701|AAAAAAAANPBCAAAA|2452335|2452364|DEPARTMENT|81|61|American members might become early operations. Sentences enjoy mo|monthly| +8702|AAAAAAAAOPBCAAAA|2452335|2452364|DEPARTMENT|81|62|Children hear even. Remarkable parts might not take now only, white a|monthly| +8703|AAAAAAAAPPBCAAAA|2452335|2452364|DEPARTMENT|81|63|Unions treat then to a cases; specimens will remember at least theoretical, interesti|monthly| +8704|AAAAAAAAAACCAAAA|2452335|2452364|DEPARTMENT|81|64|High, large types fall here happy goods. Funds ought to pick medical, total arrange|monthly| +8705|AAAAAAAABACCAAAA|2452335|2452364|DEPARTMENT|81|65|Systems return above at a benefits. Conditions say generally in|monthly| +8706|AAAAAAAACACCAAAA|2452335|2452364|DEPARTMENT|81|66|Spiritual, divine results come measures; fit, little rules should leave besides working funds. He|monthly| +8707|AAAAAAAADACCAAAA|2452335|2452364|DEPARTMENT|81|67|French services used to believe just. Local brothers|monthly| +8708|AAAAAAAAEACCAAAA|2452335|2452364|DEPARTMENT|81|68|Superior levels used to ensure just for a factors. Foreign, psychological |monthly| +8709|AAAAAAAAFACCAAAA|2452335|2452364|DEPARTMENT|81|69|Coming, future stations shall not go new, false words; small boys shall matter more. Subsidie|monthly| +8710|AAAAAAAAGACCAAAA|2452335|2452364|DEPARTMENT|81|70|Independent, female places used to supply always external, foreign t|monthly| +8711|AAAAAAAAHACCAAAA|2452335|2452364|DEPARTMENT|81|71|Differences use. Generally minor forms support also previous regulations. Planne|monthly| +8712|AAAAAAAAIACCAAAA|2452335|2452364|DEPARTMENT|81|72|Sources should claim outdoor, other thoughts. Ethnic needs live thus by the reports. T|monthly| +8713|AAAAAAAAJACCAAAA|2452335|2452364|DEPARTMENT|81|73|Cold, very kinds used to work parents. Countries might not survive. Single, old terms wipe paradoxi|monthly| +8714|AAAAAAAAKACCAAAA|2452335|2452364|DEPARTMENT|81|74|United, top efforts might get; democratic goods go. Simply simple representations used to get to |monthly| +8715|AAAAAAAALACCAAAA|2452335|2452364|DEPARTMENT|81|75|Old, senior pilots can come just legal, available problems. More scottish org|monthly| +8716|AAAAAAAAMACCAAAA|2452335|2452364|DEPARTMENT|81|76|Meanwhile democratic stories shall stay of course from the steps. |monthly| +8717|AAAAAAAANACCAAAA|2452335|2452364|DEPARTMENT|81|77|Popular, pregnant measures might not see then civil, larg|monthly| +8718|AAAAAAAAOACCAAAA|2452335|2452364|DEPARTMENT|81|78|White, supposed journals agree possible feelings. Small, young departments bid military, effe|monthly| +8719|AAAAAAAAPACCAAAA|2452335|2452364|DEPARTMENT|81|79|Minor, wide cases used to preserve russian services; new, premier leaves fulfil high against a poli|monthly| +8720|AAAAAAAAABCCAAAA|2452335|2452364|DEPARTMENT|81|80|United designers fall about other women. Easy, right cases predict ever basic ways|monthly| +8721|AAAAAAAABBCCAAAA|2452335|2452364|DEPARTMENT|81|81|Please familiar divisions would lead. Better interested instruments shall react so a|monthly| +8722|AAAAAAAACBCCAAAA|2452335|2452364|DEPARTMENT|81|82|Current speakers used to care also. Well small countries should serve as a pupils. Chinese, l|monthly| +8723|AAAAAAAADBCCAAAA|2452335|2452364|DEPARTMENT|81|83|Then able types should aid already angry frequent things. Of course true obligatio|monthly| +8724|AAAAAAAAEBCCAAAA|2452335|2452364|DEPARTMENT|81|84|Neither given visitors used to discover so arab, pre|monthly| +8725|AAAAAAAAFBCCAAAA|2452335|2452364|DEPARTMENT|81|85|Main friends ought to begin dangerous, great trees. Much american books could happe|monthly| +8726|AAAAAAAAGBCCAAAA|2452335|2452364|DEPARTMENT|81|86|Never young hours ought to kill. Generally impressive woods might work bc either prop|monthly| +8727|AAAAAAAAHBCCAAAA|2452335|2452364|DEPARTMENT|81|87|Usual, american arms may cry also. Other troubles come now ne|monthly| +8728|AAAAAAAAIBCCAAAA|2452335|2452364|DEPARTMENT|81|88|Also old writers decide. Firmly religious transactions announce almost upper beliefs. Direc|monthly| +8729|AAAAAAAAJBCCAAAA|2452335|2452364|DEPARTMENT|81|89|Examples must contact equally short economic candidates. Small weapons might not feature. Later mod|monthly| +8730|AAAAAAAAKBCCAAAA|2452335|2452364|DEPARTMENT|81|90|Old, reduced courses raise feet. Chief grounds could taste nevertheless in a|monthly| +8731|AAAAAAAALBCCAAAA|2452335|2452364|DEPARTMENT|81|91|Forms must not join cheeks. Basic, concerned eyes might figure otherwise often jo|monthly| +8732|AAAAAAAAMBCCAAAA|2452335|2452364|DEPARTMENT|81|92|Different rules achieve. Usually certain difficulties imagine reserves. Perhaps|monthly| +8733|AAAAAAAANBCCAAAA|2452335|2452364|DEPARTMENT|81|93|Broad, academic terms may appreciate prisoners. Never |monthly| +8734|AAAAAAAAOBCCAAAA|2452335|2452364|DEPARTMENT|81|94|Good homes shall not meet almost with the leaders. Sadly weak patterns would make immediately |monthly| +8735|AAAAAAAAPBCCAAAA|2452335|2452364|DEPARTMENT|81|95|Little, only rates start more too high forces. Complete, other supplies guide minutes. Courses sit |monthly| +8736|AAAAAAAAACCCAAAA|2452335|2452364|DEPARTMENT|81|96|Big candidates listen very with a police. Dangerous notions use little parti|monthly| +8737|AAAAAAAABCCCAAAA|2452335|2452364|DEPARTMENT|81|97|Vegetables bring elsewhere statistical, glad conditions. More individual|monthly| +8738|AAAAAAAACCCCAAAA|2452335|2452364|DEPARTMENT|81|98|Monetary, small children will show instead vital, legal members. So good walls achieve|monthly| +8739|AAAAAAAADCCCAAAA|2452335|2452364|DEPARTMENT|81|99|Asleep, middle weapons suggest about british, high patients|monthly| +8740|AAAAAAAAECCCAAAA|2452335|2452364|DEPARTMENT|81|100|Relatively strong sports should not reach for example in|monthly| +8741|AAAAAAAAFCCCAAAA|2452335|2452364|DEPARTMENT|81|101|Too political years could become. Grim, total successes can tell then gr|monthly| +8742|AAAAAAAAGCCCAAAA|2452335|2452364|DEPARTMENT|81|102|Still chief followers ask tonnes. Inner eyes shall carry good, enthusiastic measurements. Eggs |monthly| +8743|AAAAAAAAHCCCAAAA|2452335|2452364|DEPARTMENT|81|103|Full, considerable systems replace mildly important letters. Existing, different tho|monthly| +8744|AAAAAAAAICCCAAAA|2452335|2452364|DEPARTMENT|81|104|Encouraging risks think open upon a areas. Small damages should notice under |monthly| +8745|AAAAAAAAJCCCAAAA|2452335|2452364|DEPARTMENT|81|105|Authorities could suggest always; at least professional principles should cope yesterday dull l|monthly| +8746|AAAAAAAAKCCCAAAA|2452335|2452364|DEPARTMENT|81|106|Relevant rounds must defend such as a divisions. However electoral solutio|monthly| +8747|AAAAAAAALCCCAAAA|2452335|2452364|DEPARTMENT|81|107|National teeth make. Beaches shall avoid too long magistrates. Wet, radical a|monthly| +8748|AAAAAAAAMCCCAAAA|2452335|2452364|DEPARTMENT|81|108|Rates get very possible benefits. Local schools must recommend however. Possible artists think the|monthly| +8749|AAAAAAAANCCCAAAA|2452365|2452394|DEPARTMENT|82|1|Peaceful conflicts cannot think as well at a hands. Professional, radical deve|monthly| +8750|AAAAAAAAOCCCAAAA|2452365|2452394|DEPARTMENT|82|2|Then large problems interpret there experts. Sufficient, right holders might catch th|monthly| +8751|AAAAAAAAPCCCAAAA|2452365||DEPARTMENT|82|3|Special functions replace implicit, distinctive stocks. Key oth|monthly| +8752|AAAAAAAAADCCAAAA|2452365|2452394|DEPARTMENT|82|4|Unlikely thoughts tell still at a years. Extraordinary, big drivers must not avoid th|monthly| +8753|AAAAAAAABDCCAAAA|2452365|2452394|DEPARTMENT|82|5|Right, vast children hang cases. Sad, agricultural developers |monthly| +8754|AAAAAAAACDCCAAAA|2452365|2452394|DEPARTMENT|82|6|New chips start ancient shareholders. Steadily interesting families emphasize. Even preciou|monthly| +8755|AAAAAAAADDCCAAAA|2452365|2452394|DEPARTMENT|82|7|Costs cannot mean then in a things. Different, great |monthly| +8756|AAAAAAAAEDCCAAAA|2452365|2452394|DEPARTMENT|82|8|Dead small terms must not introduce fully books. Members must not hit alone as a l|monthly| +8757|AAAAAAAAFDCCAAAA|2452365|2452394|DEPARTMENT|82|9|Modern, other days must tell difficult components. Red conditions would take long true h|monthly| +8758|AAAAAAAAGDCCAAAA|2452365|2452394|DEPARTMENT|82|10|Very adverse securities read also. Regional activities run only present,|monthly| +8759|AAAAAAAAHDCCAAAA|2452365|2452394|DEPARTMENT|82|11|Shoulders could shut traditionally still significant events. Double new|monthly| +8760|AAAAAAAAIDCCAAAA|2452365|2452394|DEPARTMENT|82|12|Violent others stop obviously. There similar reside|monthly| +8761|AAAAAAAAJDCCAAAA|2452365|2452394|DEPARTMENT|82|13|Accurate officials might not join confidently. Old, national things may get even figures. Late|monthly| +8762|AAAAAAAAKDCCAAAA|2452365|2452394|DEPARTMENT|82|14|Readers shall remember well desirable, specific workers. |monthly| +8763|AAAAAAAALDCCAAAA|2452365|2452394|DEPARTMENT|82|15|Years understand nervous, mediterranean notes. Level, acute classes ensure li|monthly| +8764|AAAAAAAAMDCCAAAA|2452365|2452394|DEPARTMENT|82|16|Nervous centuries may see to a men; unable seconds used to swap t|monthly| +8765|AAAAAAAANDCCAAAA|2452365|2452394|DEPARTMENT|82|17|Brilliant cases walk only in a seconds. Necessarily following p|monthly| +8766|AAAAAAAAODCCAAAA|2452365|2452394|DEPARTMENT|82|18|Nervous, strong women offend. Now small recordings shall not imagine now mont|monthly| +8767|AAAAAAAAPDCCAAAA|2452365|2452394|DEPARTMENT|82|19|Thorough, specific differences must go certainly in a children; ol|monthly| +8768|AAAAAAAAAECCAAAA|2452365|2452394|DEPARTMENT|82|20|Students execute on a clients; walls may drive years. Soft |monthly| +8769|AAAAAAAABECCAAAA|2452365|2452394|DEPARTMENT|82|21|Main reasons shall get promptly early scores. Probably black areas m|monthly| +8770|AAAAAAAACECCAAAA|2452365|2452394|DEPARTMENT|82|22|Local troops would fall again huge levels. Sure seconds could not retain together b|monthly| +8771|AAAAAAAADECCAAAA|2452365|2452394|DEPARTMENT|82|23|Other, early schools might blame. Currently skilled|monthly| +8772|AAAAAAAAEECCAAAA|2452365|2452394|DEPARTMENT|82|24|Dual, elaborate guns should flow politely fields. Only children comment times.|monthly| +8773|AAAAAAAAFECCAAAA|2452365|2452394|DEPARTMENT|82|25|Then full colleagues could bring so european, wide affairs. |monthly| +8774|AAAAAAAAGECCAAAA|2452365|2452394|DEPARTMENT|82|26|Quite supposed interviews would achieve. Perhaps dead m|monthly| +8775|AAAAAAAAHECCAAAA|2452365|2452394|DEPARTMENT|82|27|Victorian, important days threaten; perhaps rural c|monthly| +8776|AAAAAAAAIECCAAAA|2452365|2452394|DEPARTMENT|82|28|Alarmingly only positions must come eyes. Other cells will get far on|monthly| +8777|AAAAAAAAJECCAAAA|2452365|2452394|DEPARTMENT|82|29|Important, main ministers create regular, other futures; political, regular|monthly| +8778|AAAAAAAAKECCAAAA|2452365|2452394|DEPARTMENT|82|30|Natural, desperate farmers might not undergo even results.|monthly| +8779|AAAAAAAALECCAAAA|2452365|2452394|DEPARTMENT|82|31|Later social measures will not see late eyes. Hot,|monthly| +8780|AAAAAAAAMECCAAAA|2452365|2452394|DEPARTMENT|82|32|Times take totally articles. Terms could acquire nervous, inner areas. Alternative teachers prefe|monthly| +8781|AAAAAAAANECCAAAA|2452365|2452394|DEPARTMENT|82|33|Today quiet letters may get superb eyebrows. Popular, other proposals try. Big, special offend|monthly| +8782|AAAAAAAAOECCAAAA|2452365|2452394|DEPARTMENT|82|34|Other, rapid years include on a months. Labour areas describe even dependent, royal comp|monthly| +8783|AAAAAAAAPECCAAAA|2452365|2452394|DEPARTMENT|82|35|Recent possibilities identify so early, ill services. Immediate int|monthly| +8784|AAAAAAAAAFCCAAAA|2452365|2452394|DEPARTMENT|82|36|Physical, final sales might maintain circumstances. R|monthly| +8785|AAAAAAAABFCCAAAA|2452365|2452394|DEPARTMENT|82|37|Clients use mainly helplessly particular toys. Various, quiet clients record a|monthly| +8786|AAAAAAAACFCCAAAA|2452365|2452394|DEPARTMENT|82|38|Really normal feet develop primitive, active struct|monthly| +8787|AAAAAAAADFCCAAAA|2452365|2452394|DEPARTMENT|82|39|Available, usual numbers last indeed later widespread decisions. High insects feel lost varieties|monthly| +8788|AAAAAAAAEFCCAAAA|2452365|2452394|||40|Secondary, little uses stimulate however. Square, international commentators find|monthly| +8789|AAAAAAAAFFCCAAAA|2452365|2452394|DEPARTMENT|82|41|Rather clear issues assume nowhere. Eyes should worry worldwi|monthly| +8790|AAAAAAAAGFCCAAAA|2452365|2452394|DEPARTMENT|82|42|Consumers believe often on to a acts. Financial signals cause possible, national ears. Cur|monthly| +8791|AAAAAAAAHFCCAAAA|2452365|2452394|DEPARTMENT|82|43|Cases will advance often pale engines. Irish rules hear pleasant, old forces. High eff|monthly| +8792|AAAAAAAAIFCCAAAA|2452365|2452394|DEPARTMENT|82|44|Above occasions develop even great hours; national, new kinds migh|monthly| +8793|AAAAAAAAJFCCAAAA|2452365|2452394|DEPARTMENT|82|45|Foreign, other songs develop entries. Even new instructions operate nationa|monthly| +8794|AAAAAAAAKFCCAAAA|2452365|2452394|DEPARTMENT|82|46|Systems may not put etc plain, other eyes. Quiet, regional rates prevent great, front|monthly| +8795|AAAAAAAALFCCAAAA|2452365|2452394|DEPARTMENT|82|47|Now favorite banks feel in general too lacking fin|monthly| +8796|AAAAAAAAMFCCAAAA|2452365|2452394|DEPARTMENT|82|48|So upper views will shut branches. Allegations come now tall numbers. E|monthly| +8797|AAAAAAAANFCCAAAA|2452365|2452394|DEPARTMENT|82|49|So silly subjects shall not pose. Monetary, significant relations finance maybe new mee|monthly| +8798|AAAAAAAAOFCCAAAA|2452365|2452394|DEPARTMENT|82|50|So unlikely problems should come everywhere to the hands. Uppe|monthly| +8799|AAAAAAAAPFCCAAAA|2452365|2452394|DEPARTMENT|82|51|More modern years require relatively firms. Empty years will defend popular children. Conditions|monthly| +8800|AAAAAAAAAGCCAAAA|2452365|2452394|DEPARTMENT|82|52|Pp. rely soviet, other democrats; even national types may bring here temporary girls. Guideline|monthly| +8801|AAAAAAAABGCCAAAA|2452365|2452394|DEPARTMENT|82|53|Familiar, whole tourists must leave. Powers shall mind under long teeth; systems should lower o|monthly| +8802|AAAAAAAACGCCAAAA|2452365|2452394|DEPARTMENT|82|54|Modern, atlantic members may improve in a factors. Difficult qualifications read personal, so|monthly| +8803|AAAAAAAADGCCAAAA|2452365|2452394|DEPARTMENT|82|55|Issues ought to know here upon a voices; then rural classes used to com|monthly| +8804|AAAAAAAAEGCCAAAA|2452365|2452394|DEPARTMENT|82|56|Also general systems expect very illegal resources. Dead famili|monthly| +8805|AAAAAAAAFGCCAAAA|2452365|2452394|DEPARTMENT|82|57|Alone, philosophical items inform at least clear, diffe|monthly| +8806|AAAAAAAAGGCCAAAA|2452365|2452394|DEPARTMENT|82|58|Afraid calls might describe modes. Users should say kids. Social pla|monthly| +8807|AAAAAAAAHGCCAAAA|2452365|2452394|DEPARTMENT|82|59|Thin proceedings wait policies. Issues may make only well similar banks. Labo|monthly| +8808|AAAAAAAAIGCCAAAA|2452365|2452394|DEPARTMENT|82|60|Mad, bright women deliver silently significant, basic notions. Patien|monthly| +8809|AAAAAAAAJGCCAAAA|2452365|2452394|DEPARTMENT|82|61|Therefore young levels can prepare across new various years; stat|monthly| +8810|AAAAAAAAKGCCAAAA|2452365|2452394|DEPARTMENT|82|62|Studies spend both. Warm earnings must not suspect more years. |monthly| +8811|AAAAAAAALGCCAAAA|2452365|2452394|DEPARTMENT|82|63|Mad, new matters used to come surely. Surroundings legislate|monthly| +8812|AAAAAAAAMGCCAAAA|2452365|2452394|DEPARTMENT|82|64|Spaces overcome as drinks. Personally outer individuals hes|monthly| +8813|AAAAAAAANGCCAAAA|2452365|2452394|DEPARTMENT|82|65|Typical, public babies accept longer national sports. Already cheap bars may end then italian d|monthly| +8814|AAAAAAAAOGCCAAAA|2452365|2452394|DEPARTMENT|82|66|French, correct schools should not appreciate available, dark waves; sufficient, |monthly| +8815|AAAAAAAAPGCCAAAA|2452365|2452394|DEPARTMENT|82|67|Later full payments go allegedly goods. For long political finger|monthly| +8816|AAAAAAAAAHCCAAAA|2452365|2452394|DEPARTMENT|82|68|Other others shall give men. Old hearts hope. Nice, orthod|monthly| +8817|AAAAAAAABHCCAAAA|2452365|2452394|DEPARTMENT|82|69|Following children maintain most involved, eastern goods. Liberal, conv|monthly| +8818|AAAAAAAACHCCAAAA|2452365|2452394|DEPARTMENT|82|70|Nasty regulations carry silent rules. So-called, effective stocks|monthly| +8819|AAAAAAAADHCCAAAA|2452365|2452394|DEPARTMENT|82|71|National, new nurses promote regularly. Tourists go still c|monthly| +8820|AAAAAAAAEHCCAAAA|2452365|2452394|DEPARTMENT|82|72|Only major beings cater forwards figures. Areas should not ma|monthly| +8821|AAAAAAAAFHCCAAAA|2452365|2452394|DEPARTMENT|82|73|High girls supply together orders; leading, financial areas might not |monthly| +8822|AAAAAAAAGHCCAAAA|2452365|2452394|DEPARTMENT|82|74|Years must operate long very little gaps. Old patterns could not happen indeed; other|monthly| +8823|AAAAAAAAHHCCAAAA|2452365|2452394|DEPARTMENT|82|75|Other, british beds can choose as sometimes tory mines. Important offers need. Often bizarre drugs |monthly| +8824|AAAAAAAAIHCCAAAA|2452365|2452394|DEPARTMENT|82|76|Parts should not snatch now. Identical, angry degrees shall enable today differen|monthly| +8825|AAAAAAAAJHCCAAAA|2452365|2452394|DEPARTMENT|82|77|Southern, local managers come. General, great facts may not distract crimes. Human law|monthly| +8826|AAAAAAAAKHCCAAAA|2452365|2452394|DEPARTMENT|82|78|Good shy talks used to apply like a parents; interests shine perhaps by a classes. |monthly| +8827|AAAAAAAALHCCAAAA|2452365|2452394|DEPARTMENT|82|79|Poor conditions allow long, sorry weeks. Only industrial deta|monthly| +8828|AAAAAAAAMHCCAAAA|2452365|2452394|DEPARTMENT|82|80|Of course open agents would strengthen everyday services.|monthly| +8829|AAAAAAAANHCCAAAA|2452365|2452394|DEPARTMENT|82|81|Thousands pay bloody results; years may not turn increasingly to a |monthly| +8830|AAAAAAAAOHCCAAAA|2452365|2452394|DEPARTMENT|82|82|Good circumstances used to afford simply serious, british benefits. Charming, military ye|monthly| +8831|AAAAAAAAPHCCAAAA|2452365|2452394|DEPARTMENT|82|83|Exceptionally difficult answers know highest companies. Desperate, pot|monthly| +8832|AAAAAAAAAICCAAAA|2452365|2452394|DEPARTMENT|82|84|Afraid, normal categories remain very costs. Videos appear later once p|monthly| +8833|AAAAAAAABICCAAAA|2452365|2452394|DEPARTMENT|82|85|Various objects ought to beat students; exciting, chemical fig|monthly| +8834|AAAAAAAACICCAAAA|2452365|2452394|DEPARTMENT|82|86|Social times meet enough with a times. Once more necessary architects care thereafter over su|monthly| +8835|AAAAAAAADICCAAAA|2452365|2452394|DEPARTMENT|82|87|National fields could not see by now both monetary shares. Simple, po|monthly| +8836|AAAAAAAAEICCAAAA|2452365|2452394|DEPARTMENT|82|88|Germans ratify so. Material knees must not cause just photograph|monthly| +8837|AAAAAAAAFICCAAAA|||DEPARTMENT|||Real, british years support most available clients. Children will not serve only real clothes. Al|| +8838|AAAAAAAAGICCAAAA|2452365|2452394|DEPARTMENT|82|90|Old hours decide memories. Normal, presidential millions will not distinguish quite states; neverth|monthly| +8839|AAAAAAAAHICCAAAA|2452365|2452394|DEPARTMENT|82|91|Problems should lose right sorry cities; centres contain. Early, logical ministers |monthly| +8840|AAAAAAAAIICCAAAA|2452365|2452394|DEPARTMENT|82|92|Expert trees must not drink quickly. Letters expand with t|monthly| +8841|AAAAAAAAJICCAAAA|2452365|2452394|DEPARTMENT|82|93|Functions reach only. Anxious messages should not kn|monthly| +8842|AAAAAAAAKICCAAAA|2452365|||82||Real areas enjoy objects. Members recover there out of the troubles. Special years wi|monthly| +8843|AAAAAAAALICCAAAA|2452365|2452394|DEPARTMENT|82|95|Very sufficient ports last then arrangements. Foreign newspapers help issues; processes see sig|monthly| +8844|AAAAAAAAMICCAAAA|2452365|2452394|DEPARTMENT|82|96|New, low minutes could play communications; once valid windows may not dismiss rules.|monthly| +8845|AAAAAAAANICCAAAA|2452365|2452394|DEPARTMENT|82|97|Just simple times grow actually eager, correct achievements; black, impossible mice could fi|monthly| +8846|AAAAAAAAOICCAAAA|2452365|2452394|DEPARTMENT|82|98|English, distinct teachers mark at least usual brothers|monthly| +8847|AAAAAAAAPICCAAAA|2452365|2452394|DEPARTMENT|82|99|Words must reject commercial, new improvements. Necessary examples |monthly| +8848|AAAAAAAAAJCCAAAA|2452365|2452394|DEPARTMENT|82|100|Profound patients should convict foundations. Both current times pick bot|monthly| +8849|AAAAAAAABJCCAAAA|2452365|2452394|DEPARTMENT|82|101|Victims finish well districts; hardly serious theories may not rule namely. Strong, curio|monthly| +8850|AAAAAAAACJCCAAAA|2452365|2452394|DEPARTMENT|82|102|Separate, new eyes turn then separate years. Always effective seats begin following a |monthly| +8851|AAAAAAAADJCCAAAA|2452365|2452394|DEPARTMENT|82|103|Just high children say with a countries. Boards should not take simply off a systems. Much ave|monthly| +8852|AAAAAAAAEJCCAAAA|2452365|2452394|DEPARTMENT|82|104|Publications may back. Women cannot know from a movies. Appropriate, dar|monthly| +8853|AAAAAAAAFJCCAAAA|2452365|2452394|DEPARTMENT|82|105|Current, direct trends must throw before often likely cheek|monthly| +8854|AAAAAAAAGJCCAAAA||2452394|DEPARTMENT|82|||| +8855|AAAAAAAAHJCCAAAA|2452365|2452394|DEPARTMENT|82|107|Important, other sciences see however long, careful wo|monthly| +8856|AAAAAAAAIJCCAAAA|2452365|2452394|DEPARTMENT|82|108|Words argue base, limited talks. Perhaps specific sports might not leave. Steadi|monthly| +8857|AAAAAAAAJJCCAAAA|2452395|2452424|DEPARTMENT|83|1|Again urgent schools climb; useful, alternative resources shall clear on|monthly| +8858|AAAAAAAAKJCCAAAA|2452395|2452424|DEPARTMENT|83|2|Glad children join police. Then military days capture supr|monthly| +8859|AAAAAAAALJCCAAAA|2452395|2452424|DEPARTMENT|83|3|Scenes talk well disabled rats. Nowhere crude texts ought to work signs. Busy studies switc|monthly| +8860|AAAAAAAAMJCCAAAA|2452395|2452424|DEPARTMENT|83|4|Soft representations would not escape. Women shall take too constitut|monthly| +8861|AAAAAAAANJCCAAAA|2452395|2452424|DEPARTMENT|83|5|Obvious, brilliant expressions send now redundant, olympic firms. Important problems attac|monthly| +8862|AAAAAAAAOJCCAAAA|2452395|2452424|DEPARTMENT|83|6|Alone, great groups integrate by the options. Industrial, used weeks ca|monthly| +8863|AAAAAAAAPJCCAAAA|2452395|2452424|DEPARTMENT|83|7|Levels protest numerous activities. Responsible arms will occur please n|monthly| +8864|AAAAAAAAAKCCAAAA|2452395|2452424|DEPARTMENT|83|8|Full arguments make still; royal, different words love best powerfully political others. Weeks w|monthly| +8865|AAAAAAAABKCCAAAA|2452395|2452424|DEPARTMENT|83|9|Good, new boats would set in a windows. Private, severe examples shou|monthly| +8866|AAAAAAAACKCCAAAA|2452395|2452424|DEPARTMENT|83|10|Temporary, simple times understand really subsequent offices. Measure|monthly| +8867|AAAAAAAADKCCAAAA|2452395|2452424|DEPARTMENT|83|11|Leaders would curb at a hands. Studies ought to appear very then statutory words; germans retreat|monthly| +8868|AAAAAAAAEKCCAAAA|2452395|2452424|DEPARTMENT|83|12|Then key minutes would come overseas, dutch events. Very public years go in a players. Rou|monthly| +8869|AAAAAAAAFKCCAAAA|2452395|2452424|DEPARTMENT|83|13|Better immediate elections shall take. Sessions trade doors. C|monthly| +8870|AAAAAAAAGKCCAAAA|2452395|2452424|DEPARTMENT|83|14|Legs would not snap in a figures. Regular governments will not complete no|monthly| +8871|AAAAAAAAHKCCAAAA|2452395|2452424|DEPARTMENT|83|15|More angry profits should meet together customers. Big, easy agencie|monthly| +8872|AAAAAAAAIKCCAAAA|2452395|2452424|DEPARTMENT|83|16|Also able guests prevent as criminal problems. Only obvious minutes ought to|monthly| +8873|AAAAAAAAJKCCAAAA|2452395|2452424|DEPARTMENT|83|17|Foreign, necessary results back now together british servants. Adeq|monthly| +8874|AAAAAAAAKKCCAAAA|2452395|2452424|DEPARTMENT|83|18|Popular negotiations assert clinical points. New citizens ought to lik|monthly| +8875|AAAAAAAALKCCAAAA|2452395|2452424|DEPARTMENT|83|19|Fully real plans ought to improve. Low contacts should approve |monthly| +8876|AAAAAAAAMKCCAAAA|2452395|2452424|DEPARTMENT|83|20|Considerable patients shall not see only much british shadows; home positive costs could enjo|monthly| +8877|AAAAAAAANKCCAAAA|2452395|2452424|DEPARTMENT|83|21|Russian, social priorities ask also after a doubts. Refugees will not s|monthly| +8878|AAAAAAAAOKCCAAAA|2452395|2452424|DEPARTMENT|83|22|Red seats may look. Positions would encourage so. Requirements may withhold bette|monthly| +8879|AAAAAAAAPKCCAAAA|2452395|2452424|DEPARTMENT|83|23|Artists must carry well at a deals. So pale studies know slowly losses; again ambitious accounts o|monthly| +8880|AAAAAAAAALCCAAAA|2452395|2452424|DEPARTMENT|83|24|Single objects may not leave clearly on a voters. Corporate houses could |monthly| +8881|AAAAAAAABLCCAAAA|2452395|2452424|DEPARTMENT|83|25|True, charming forms ought to deter recently true, incredible pages. Different, electrica|monthly| +8882|AAAAAAAACLCCAAAA|2452395|2452424|DEPARTMENT|83|26|Massive, past months shall extract chemicals. Serious pictures support now arguments.|monthly| +8883|AAAAAAAADLCCAAAA|2452395|2452424|DEPARTMENT|83|27|Clear, available profits ought to transform as a whole disastrous mi|monthly| +8884|AAAAAAAAELCCAAAA|2452395|2452424|DEPARTMENT|83|28|Parts may take high, appropriate programmes. Nearly an|monthly| +8885|AAAAAAAAFLCCAAAA|2452395|2452424|DEPARTMENT|83|29|Early, local pairs may not warn often young aims; far white strategies go just elected, initial |monthly| +8886|AAAAAAAAGLCCAAAA|2452395|2452424|DEPARTMENT|83|30|Ever growing windows reverse to a galleries. Concerned, historic documents|monthly| +8887|AAAAAAAAHLCCAAAA|2452395|2452424|DEPARTMENT|83|31|Desirable horses may not try especially. Weaknesses should play often. Externa|monthly| +8888|AAAAAAAAILCCAAAA|2452395|2452424|DEPARTMENT|83|32|Exciting courts receive ever instructions. Figures date thus poor systems. Memb|monthly| +8889|AAAAAAAAJLCCAAAA|2452395|2452424|DEPARTMENT|83|33|Indoors creative tensions can take well ready days. Ri|monthly| +8890|AAAAAAAAKLCCAAAA|2452395|2452424|DEPARTMENT|83|34|Increasingly anxious proposals ensure; practical communities must not break under way institut|monthly| +8891|AAAAAAAALLCCAAAA|2452395|2452424|DEPARTMENT|83|35|Hospitals might sell as probably cultural calls. Important shoes cannot give|monthly| +8892|AAAAAAAAMLCCAAAA|2452395|2452424|DEPARTMENT|83|36|Fields meet as there unable criteria. Labour computers acquire as well fu|monthly| +8893|AAAAAAAANLCCAAAA|2452395|2452424|DEPARTMENT|83|37|Organisational, ordinary roses pursue moreover in the markets. Also cheap jobs might believe m|monthly| +8894|AAAAAAAAOLCCAAAA|2452395|2452424|DEPARTMENT|83|38|Men would fight coastal, likely police; financial, huge policies can tell principal terms; pe|monthly| +8895|AAAAAAAAPLCCAAAA|2452395|2452424|DEPARTMENT|83|39|Indignantly great drinks used to explore letters. Existing, various reservatio|monthly| +8896|AAAAAAAAAMCCAAAA|2452395|2452424|DEPARTMENT|83|40|Needs could not make therefore. Less large risks fit nowadays public, wide sym|monthly| +8897|AAAAAAAABMCCAAAA|2452395|2452424|DEPARTMENT|83|41|Large, remarkable years expect very. Mostly large thoughts b|monthly| +8898|AAAAAAAACMCCAAAA|2452395|2452424|DEPARTMENT|83|42|Authorities must strike further presumably impossible properties. Bot|monthly| +8899|AAAAAAAADMCCAAAA|2452395|2452424|DEPARTMENT|83|43|However useful pupils might not sell. Immensely european servants prod|monthly| +8900|AAAAAAAAEMCCAAAA|2452395|2452424|DEPARTMENT|83|44|Late holidays think even other groups. Human, civil degrees get from|monthly| +8901|AAAAAAAAFMCCAAAA|2452395|2452424|DEPARTMENT|83|45|Other, democratic years cannot know joint signals. Representations would re|monthly| +8902|AAAAAAAAGMCCAAAA|2452395|2452424|DEPARTMENT|83|46|Internally other days should not decide because of a coun|monthly| +8903|AAAAAAAAHMCCAAAA|2452395|2452424|DEPARTMENT|83|47|Now ethical concerns involve specifically results; already personal thousands would hear well |monthly| +8904|AAAAAAAAIMCCAAAA|2452395|2452424|DEPARTMENT|83|48|Never daily times ought to suppose. Now gentle citizens |monthly| +8905|AAAAAAAAJMCCAAAA|2452395|2452424|DEPARTMENT|83|49|Too young chiefs use. Other, efficient villages acknowledge recently. Able, available t|monthly| +8906|AAAAAAAAKMCCAAAA|2452395|2452424|DEPARTMENT|83|50|Black shares would get of course potentially final heads. Nation|monthly| +8907|AAAAAAAALMCCAAAA|2452395|2452424|DEPARTMENT|83|51|Sources ought to turn inc cases. Studies shall know exactly authorities. General, english |monthly| +8908|AAAAAAAAMMCCAAAA|2452395|2452424|DEPARTMENT|83|52|Managers get new interests. Light villages can get able areas. Early boundaries try in|monthly| +8909|AAAAAAAANMCCAAAA|2452395|2452424|DEPARTMENT|83|53|Available heroes make unknown bands. Movements ought to take poor, free vil|monthly| +8910|AAAAAAAAOMCCAAAA|2452395|2452424|DEPARTMENT|83|54|Popular, precious studies will not see yet in a in|monthly| +8911|AAAAAAAAPMCCAAAA|2452395|2452424|DEPARTMENT|83|55|As raw sides provide essential, middle devices. Right models would all|monthly| +8912|AAAAAAAAANCCAAAA||||||Free, political women can pay provisions. Momentarily disci|monthly| +8913|AAAAAAAABNCCAAAA|2452395|2452424|DEPARTMENT|83|57|Tremendously enormous items draw even different achi|monthly| +8914|AAAAAAAACNCCAAAA|2452395|2452424|DEPARTMENT|83|58|Profits leave close. Good, local novels get later on a |monthly| +8915|AAAAAAAADNCCAAAA|2452395|2452424|DEPARTMENT|83|59|Certain, various songs look probably at a programmes; deeply so-called men say there at a |monthly| +8916|AAAAAAAAENCCAAAA|2452395|2452424|DEPARTMENT|83|60|Liberal fields will know in a runs. Already well-known perso|monthly| +8917|AAAAAAAAFNCCAAAA|2452395|2452424|DEPARTMENT|83|61|Liberal, important games must not deal extremely nice, capital democrats. Sufficient, married o|monthly| +8918|AAAAAAAAGNCCAAAA|2452395|2452424|DEPARTMENT|83|62|Nuclear, average suppliers combat around roman jobs. Black, additional men would not secure by a p|monthly| +8919|AAAAAAAAHNCCAAAA|2452395|2452424|DEPARTMENT|83|63|Married experts would not fail largely from a difficulties. Words shall not think. Disc|monthly| +8920|AAAAAAAAINCCAAAA|2452395|2452424|DEPARTMENT|83|64|Deep offices sail poor, whole rises. Most toxic rights contribute normally. Poor, huma|monthly| +8921|AAAAAAAAJNCCAAAA|2452395|2452424|DEPARTMENT|83|65|Special deposits will swim original youngsters. Movies remember also personal thing|monthly| +8922|AAAAAAAAKNCCAAAA|2452395|2452424|DEPARTMENT|83|66|There local years define through a relationships. Other degrees ought to suffer late men. Qui|monthly| +8923|AAAAAAAALNCCAAAA|2452395|2452424|DEPARTMENT|83|67|Late secondary circumstances fail new horses. Large customers could not |monthly| +8924|AAAAAAAAMNCCAAAA|2452395|2452424|DEPARTMENT|83|68|Too wide orders ought to stop no doubt days; difficult stands cou|monthly| +8925|AAAAAAAANNCCAAAA|2452395|2452424|DEPARTMENT|83|69|Cultural members cannot paint. Empty tests must di|monthly| +8926|AAAAAAAAONCCAAAA|2452395|2452424|DEPARTMENT|83|70|Running schemes should not pass loyal, small asset|monthly| +8927|AAAAAAAAPNCCAAAA|2452395|2452424|DEPARTMENT|83|71|Families know; patients must result other rules. Previously y|monthly| +8928|AAAAAAAAAOCCAAAA|2452395|2452424|DEPARTMENT|83|72|British reforms would enable stubbornly in a fragments; single firms encoura|monthly| +8929|AAAAAAAABOCCAAAA|2452395|2452424|DEPARTMENT|83|73|Stages may produce secret, mad illustrations. Obvious organisms make for the increas|monthly| +8930|AAAAAAAACOCCAAAA|2452395|2452424|DEPARTMENT|83|74|Forces save almost so new charges. Now old experiments receive by a crimes.|monthly| +8931|AAAAAAAADOCCAAAA|2452395|2452424|DEPARTMENT|83|75|Only, only implications might protect partially months. Powerful systems could occur both ra|monthly| +8932|AAAAAAAAEOCCAAAA|2452395|2452424|DEPARTMENT|83|76|External, common children see times; global, international faces becom|monthly| +8933|AAAAAAAAFOCCAAAA|2452395|2452424|DEPARTMENT|83|77|True votes would not make decisive, political houses. Obvious, close mi|monthly| +8934|AAAAAAAAGOCCAAAA|2452395|2452424|DEPARTMENT|83|78|Correct, wide years ask perhaps workers. Today pregnant directions can go less horribl|monthly| +8935|AAAAAAAAHOCCAAAA|2452395|2452424|DEPARTMENT|83|79|Mature teeth ought to cut very exceptional months. Careful activities|monthly| +8936|AAAAAAAAIOCCAAAA|2452395|2452424|DEPARTMENT|83|80|Complex lakes like very continuing students. Days work also absent pp.. S|monthly| +8937|AAAAAAAAJOCCAAAA|2452395|2452424|DEPARTMENT|83|81|Numbers use in a miles. Restrictions will declare parties. More special qualities |monthly| +8938|AAAAAAAAKOCCAAAA|2452395|2452424|DEPARTMENT|83|82|At least future children will know very enquiries. Teachers shall guess quickly free ben|monthly| +8939|AAAAAAAALOCCAAAA|2452395|2452424|DEPARTMENT|83|83|Then special concentrations go over best consistent days; dogs ma|monthly| +8940|AAAAAAAAMOCCAAAA|2452395|2452424|DEPARTMENT|83|84|Free wounds could go naturally. Skills provide here sorry, european thing|monthly| +8941|AAAAAAAANOCCAAAA|2452395|2452424|DEPARTMENT|83|85|Keen materials will not leave as good as keen, short colleg|monthly| +8942|AAAAAAAAOOCCAAAA|2452395|2452424|DEPARTMENT|83|86|Colours come for good years; questions must not contact up to a voters. Slowly useful police go onl|monthly| +8943|AAAAAAAAPOCCAAAA|2452395|2452424|DEPARTMENT|83|87|Little, annual subjects will appropriate competitive thousands.|monthly| +8944|AAAAAAAAAPCCAAAA|2452395|2452424|DEPARTMENT|83|88|Unable sciences will grant still local, great needs. Careful, conscious pr|monthly| +8945|AAAAAAAABPCCAAAA|2452395|2452424|DEPARTMENT|83|89|Simple prisons sign currently competitive changes. Too commercial|monthly| +8946|AAAAAAAACPCCAAAA|2452395|2452424|DEPARTMENT|83|90|Services seem general, right cards; wooden, firm w|monthly| +8947|AAAAAAAADPCCAAAA|2452395|2452424|DEPARTMENT|83|91|Russians could need coming, impossible hands. Densel|monthly| +8948|AAAAAAAAEPCCAAAA|2452395|2452424|DEPARTMENT|83|92|Extraordinary, fascinating eyes should replicate l|monthly| +8949|AAAAAAAAFPCCAAAA|2452395|2452424|DEPARTMENT|83|93|Other, open affairs follow spanish records. Free, significant observations would a|monthly| +8950|AAAAAAAAGPCCAAAA|2452395|||83|||| +8951|AAAAAAAAHPCCAAAA|2452395|2452424|DEPARTMENT|83|95|Quick details reflect. Personal, absolute kinds photograph; in private sex|monthly| +8952|AAAAAAAAIPCCAAAA|2452395|2452424|DEPARTMENT|83|96|Widespread places go fully surprised restrictions. Tight, direct days must think|monthly| +8953|AAAAAAAAJPCCAAAA|2452395|2452424|DEPARTMENT|83|97|Specifically tropical months understand other findings. Constant |monthly| +8954|AAAAAAAAKPCCAAAA|2452395|2452424|DEPARTMENT|83|98|Games withdraw however then other investigations. Players design now|monthly| +8955|AAAAAAAALPCCAAAA|2452395|2452424|DEPARTMENT|83|99|Bright, other angles make less dry, important children. Maybe pure babies go here|monthly| +8956|AAAAAAAAMPCCAAAA|2452395|2452424|DEPARTMENT|83|100|Tools accept as terms. Annual, mental settlements will not avoid. Sometim|monthly| +8957|AAAAAAAANPCCAAAA|2452395|2452424|DEPARTMENT|83|101|Months collect reportedly; months lead earlier dramatically british dimensions. Defe|monthly| +8958|AAAAAAAAOPCCAAAA|2452395|2452424|DEPARTMENT|83|102|Daily pubs discuss earlier. Prices hope specifically from a potatoes. Questions could fig|monthly| +8959|AAAAAAAAPPCCAAAA|2452395|2452424|DEPARTMENT|83|103|True years may go especially contents; quite well-known drawing|monthly| +8960|AAAAAAAAAADCAAAA|2452395|2452424|DEPARTMENT|83|104|Even necessary words tell free houses. Shows produce ashamed r|monthly| +8961|AAAAAAAABADCAAAA|2452395|2452424|DEPARTMENT|83|105|Arrangements could offer. Due standards take simply then substantial ligh|monthly| +8962|AAAAAAAACADCAAAA|2452395|2452424|DEPARTMENT|83|106|Personal, substantial scales would not discuss actu|monthly| +8963|AAAAAAAADADCAAAA|2452395|2452424|DEPARTMENT|83|107|Soon old interests may not see already always exceptional carers. Yesterday only rules could|monthly| +8964|AAAAAAAAEADCAAAA|2452395|2452424|DEPARTMENT|83|108|Desirable, famous wishes tempt no effective, foreign benefits. Democratic, whole s|monthly| +8965|AAAAAAAAFADCAAAA|2452425|2452454|DEPARTMENT|84|1|New instructions used to help in a awards. Occasional, elderly institutions|monthly| +8966|AAAAAAAAGADCAAAA|2452425|2452454|DEPARTMENT|84|2|Anyway favorite companies used to wipe recently there public differences. Boats fall. Techni|monthly| +8967|AAAAAAAAHADCAAAA|2452425|2452454|DEPARTMENT|84|3|Communities might not remain now close concerned governments. Similarly new years think fortu|monthly| +8968|AAAAAAAAIADCAAAA|2452425|2452454|DEPARTMENT|84|4|Given, convincing pains establish in a politicians. Always medical proposals introduce a|monthly| +8969|AAAAAAAAJADCAAAA|2452425|2452454|DEPARTMENT|84|5|Little women must hurt. Also small owners put there. Lawyers could remain e|monthly| +8970|AAAAAAAAKADCAAAA|2452425|2452454|DEPARTMENT|84|6|Oral, able users stop normally too rural days. Tired, original governments|monthly| +8971|AAAAAAAALADCAAAA|2452425|2452454|DEPARTMENT|84|7|Unlikely, young stands revoke early guns. Other, special |monthly| +8972|AAAAAAAAMADCAAAA|2452425|2452454|DEPARTMENT|84|8|Fast similar situations find half over new affairs. Total author|monthly| +8973|AAAAAAAANADCAAAA|2452425|2452454|DEPARTMENT|84|9|Legal patterns begin then here industrial authorities. Inches should not transfer either |monthly| +8974|AAAAAAAAOADCAAAA|2452425|2452454|DEPARTMENT|84|10|Deliberately pretty options remain. Almost full records must |monthly| +8975|AAAAAAAAPADCAAAA|2452425|2452454|DEPARTMENT|84|11|Detailed, busy conditions must see thus back pockets. Transactions resolve w|monthly| +8976|AAAAAAAAABDCAAAA|2452425|2452454|DEPARTMENT|84|12|Modern, various subjects feel also. British, foreign men adopt regularly particularly round lan|monthly| +8977|AAAAAAAABBDCAAAA|2452425|2452454|DEPARTMENT|84|13|Major years would know into the books. Troops should not instruct. Carefully only|monthly| +8978|AAAAAAAACBDCAAAA|2452425|2452454|DEPARTMENT|84|14|Safely consistent relations solve more comprehensive things; current, evolutionary po|monthly| +8979|AAAAAAAADBDCAAAA|2452425|2452454|DEPARTMENT|84|15|Parts sleep; english exports tell in particular new years. Scientifically maximum allowances put o|monthly| +8980|AAAAAAAAEBDCAAAA|2452425|2452454|DEPARTMENT|84|16|Fragments might not change continuously royal members. Real, foreign children used to expan|monthly| +8981|AAAAAAAAFBDCAAAA|2452425|2452454|DEPARTMENT|84|17|Also potential respondents make new schools. Further hard computers should find so ships. Te|monthly| +8982|AAAAAAAAGBDCAAAA|2452425|2452454|DEPARTMENT|84|18|Only, natural years might not mean special, major subjects; used, low imports|monthly| +8983|AAAAAAAAHBDCAAAA|2452425|2452454|DEPARTMENT|84|19|New, usual stars make now on a pictures. Facilities must burst houses. Old soldiers will not |monthly| +8984|AAAAAAAAIBDCAAAA|2452425|2452454|DEPARTMENT|84|20|Patients will not meet desperately players. Primary count|monthly| +8985|AAAAAAAAJBDCAAAA|2452425|2452454|DEPARTMENT|84|21|So physical reactions must check authorities. Italian, critical assessm|monthly| +8986|AAAAAAAAKBDCAAAA|2452425|2452454|DEPARTMENT|84|22|At least clean engineers consider also. Clients shall not h|monthly| +8987|AAAAAAAALBDCAAAA|2452425|2452454|DEPARTMENT|84|23|Households detain long beautiful customers. Capital, similar per|monthly| +8988|AAAAAAAAMBDCAAAA|2452425|2452454|DEPARTMENT|84|24|So harsh councillors could give little examples. Internal, top respons|monthly| +8989|AAAAAAAANBDCAAAA|2452425|2452454|DEPARTMENT|84|25|Small, personal weeks say far jobs. Special, other concepts should not come more|monthly| +8990|AAAAAAAAOBDCAAAA|2452425|2452454|DEPARTMENT|84|26|Parents may eat severe, fair users. Agents can avoid mo|monthly| +8991|AAAAAAAAPBDCAAAA|2452425|2452454|DEPARTMENT|84|27|Other, bad activities should not go. Great substances cannot want just clear ar|monthly| +8992|AAAAAAAAACDCAAAA|2452425|2452454|DEPARTMENT|84|28|New policemen forget troops. Units make there scarcely gastric ra|monthly| +8993|AAAAAAAABCDCAAAA|2452425|2452454|DEPARTMENT|84|29|Original parts cannot start now major, distinctive days. Paths mean as a a|monthly| +8994|AAAAAAAACCDCAAAA|2452425|2452454|DEPARTMENT|84|30|Even old years go equally always advanced children. At last hot systems|monthly| +8995|AAAAAAAADCDCAAAA|2452425|2452454|DEPARTMENT|84|31|Obligations reflect regional laws. Women used to see enough british, following groups; |monthly| +8996|AAAAAAAAECDCAAAA|2452425|2452454|DEPARTMENT|84|32|Ago possible rates could spend just to the services. In general c|monthly| +8997|AAAAAAAAFCDCAAAA|2452425|2452454|DEPARTMENT|84|33|Quantities bring fatal eyes. So-called thousands will gi|monthly| +8998|AAAAAAAAGCDCAAAA|2452425|2452454|DEPARTMENT|84|34|Comprehensive considerations see required, modern ideas. Simple, eastern boots used to earn youn|monthly| +8999|AAAAAAAAHCDCAAAA|2452425|2452454|DEPARTMENT|84|35|Values say quite high cells; good, short police may ruin in a rats; rel|monthly| +9000|AAAAAAAAICDCAAAA|2452425|2452454|DEPARTMENT|84|36|More than prospective cities say surely depending on a places; feet must not help nevertheless natu|monthly| +9001|AAAAAAAAJCDCAAAA|2452425|2452454|DEPARTMENT|84|37|Also unacceptable needs travel again. Existing levels shal|monthly| +9002|AAAAAAAAKCDCAAAA|2452425|2452454|DEPARTMENT|84|38|Clergy interfere results. Average others would learn organisations; america|monthly| +9003|AAAAAAAALCDCAAAA|2452425|2452454|DEPARTMENT|84|39|Compatible shares attempt even. Useful, critical judg|monthly| +9004|AAAAAAAAMCDCAAAA|2452425|2452454|DEPARTMENT|84|40|Movements used to see; arms look especially. Capable, complex goods w|monthly| +9005|AAAAAAAANCDCAAAA|2452425|2452454|DEPARTMENT|84|41|Problems avoid properly central, primary masses. Just fundamental interests will not give har|monthly| +9006|AAAAAAAAOCDCAAAA|2452425|2452454|DEPARTMENT|84|42|Political occasions think lucky, financial men. Different, supe|monthly| +9007|AAAAAAAAPCDCAAAA|2452425|2452454|DEPARTMENT|84|43|Sad, continuous cars open effectively special costs. High, economic countries move rather. Appli|monthly| +9008|AAAAAAAAADDCAAAA|2452425|2452454|DEPARTMENT|84|44|Genes alert known operations. Then different lights would not appoint for the spirits. Sales speak |monthly| +9009|AAAAAAAABDDCAAAA|2452425|2452454|DEPARTMENT|84|45|Unaware, married police might not ban. Fully right meetings need at present high units. Gues|monthly| +9010|AAAAAAAACDDCAAAA|2452425|2452454|DEPARTMENT|84|46|Plants may not advance for a moves. Environmental, large magist|monthly| +9011|AAAAAAAADDDCAAAA|2452425|2452454|DEPARTMENT|84|47|Cases name now good ministers. Later distinct shares should say elderly report|monthly| +9012|AAAAAAAAEDDCAAAA|2452425|2452454|DEPARTMENT|84|48|Children post however activities. Criminal, clear incomes see so please |monthly| +9013|AAAAAAAAFDDCAAAA|2452425|2452454|DEPARTMENT|84|49|Only participants accept hereby directions. Neither special visitors used to teach mor|monthly| +9014|AAAAAAAAGDDCAAAA|2452425|2452454|DEPARTMENT|84|50|Only redundant children should care; often inc years follow|monthly| +9015|AAAAAAAAHDDCAAAA|2452425|2452454|DEPARTMENT|84|51|Economic, primary prices exclude now glorious wages. New depths should control only practical|monthly| +9016|AAAAAAAAIDDCAAAA|2452425|2452454|DEPARTMENT|84|52|Democrats contest highly only certain minutes. Scottish, variou|monthly| +9017|AAAAAAAAJDDCAAAA|2452425|2452454|DEPARTMENT|84|53|Changes used to reduce general faces. Recent jobs would pay ro|monthly| +9018|AAAAAAAAKDDCAAAA|2452425|2452454|DEPARTMENT|84|54|Groups will cook ready events. Legal, illegal paren|monthly| +9019|AAAAAAAALDDCAAAA|2452425|2452454|DEPARTMENT|84|55|Sound proposals could retire profits; now personal types buy at a days. Operations co|monthly| +9020|AAAAAAAAMDDCAAAA|2452425|2452454|DEPARTMENT|84|56|Complex, reasonable days would think now generally new hands; individual, independent quest|monthly| +9021|AAAAAAAANDDCAAAA|2452425|2452454|DEPARTMENT|84|57|Institutions take so great features; interests cannot take under bet|monthly| +9022|AAAAAAAAODDCAAAA|2452425|2452454|DEPARTMENT|84|58|Southern, ambitious patients must please on a purposes. Else inc sounds would not lose british|monthly| +9023|AAAAAAAAPDDCAAAA|2452425|2452454||84||Things accommodate able terms. Following, deep creatures should occur rapidly|monthly| +9024|AAAAAAAAAEDCAAAA|2452425|2452454|DEPARTMENT|84|60|Effects continue special pilots. Old, central recommendations could see to |monthly| +9025|AAAAAAAABEDCAAAA|2452425|2452454|DEPARTMENT|84|61|Interesting disputes sink late on a females; relative leaders register just medical el|monthly| +9026|AAAAAAAACEDCAAAA|2452425|2452454|DEPARTMENT|84|62|More certain answers might not recognize almost. So as whole pieces wi|monthly| +9027|AAAAAAAADEDCAAAA|2452425|2452454|DEPARTMENT|84|63|Dead poles celebrate at least. Other terms ought to undertake in a policies; general, res|monthly| +9028|AAAAAAAAEEDCAAAA|2452425|2452454|DEPARTMENT|84|64|Arms might not pay briefly high worlds. Unique, scottish dogs can penetrate again with |monthly| +9029|AAAAAAAAFEDCAAAA|2452425|2452454|DEPARTMENT|84|65|Male, private results used to act to a objectives. Significant, young results must |monthly| +9030|AAAAAAAAGEDCAAAA|2452425|2452454|DEPARTMENT|84|66|Federal, dark observations edit long, fine hours. Soon aware gardens expec|monthly| +9031|AAAAAAAAHEDCAAAA|2452425|2452454|DEPARTMENT|84|67|Details see too final tracks. Again inc officials arrive more than new soluti|monthly| +9032|AAAAAAAAIEDCAAAA|2452425|2452454|DEPARTMENT|84|68|Levels might not leave also extended manufacturers. Tiny artists understa|monthly| +9033|AAAAAAAAJEDCAAAA|2452425|2452454|DEPARTMENT|84|69|Women learn high single members. Awkward, lovely wings might restore only. Ear|monthly| +9034|AAAAAAAAKEDCAAAA|2452425|2452454|DEPARTMENT|84|70|Correct, international drivers postpone. Terms make more c|monthly| +9035|AAAAAAAALEDCAAAA|2452425|2452454|DEPARTMENT|84|71|Social circumstances leave in a forms; international planes |monthly| +9036|AAAAAAAAMEDCAAAA|2452425|2452454|DEPARTMENT|84|72|Political, additional terms feel. Involved ideas count;|monthly| +9037|AAAAAAAANEDCAAAA|2452425|2452454|DEPARTMENT|84|73|Services think here. Forms enjoy samples. Local goods may not meet g|monthly| +9038|AAAAAAAAOEDCAAAA|2452425|2452454|DEPARTMENT|84|74|Sorry courses ought to facilitate even local, lucky groups. Fa|monthly| +9039|AAAAAAAAPEDCAAAA|2452425|2452454|DEPARTMENT|84|75|Technical reasons could talk very. Indeed fatal years mention a|monthly| +9040|AAAAAAAAAFDCAAAA|2452425|2452454|DEPARTMENT|84|76|Important, black walls include inevitably. Also low subjects ought to rep|monthly| +9041|AAAAAAAABFDCAAAA|2452425|2452454|DEPARTMENT|84|77|Necessary runs tell. Individuals carry guidelines. Anywhere organisational shoes warn gardens. Suit|monthly| +9042|AAAAAAAACFDCAAAA|2452425|2452454|DEPARTMENT|84|78|Rich, video-taped relationships recover quickly doub|monthly| +9043|AAAAAAAADFDCAAAA|2452425|2452454|DEPARTMENT|84|79|Shops must not benefit purely areas. Low heavy fires en|monthly| +9044|AAAAAAAAEFDCAAAA|2452425|2452454|DEPARTMENT|84|80|Then terrible communities should press here as a resources. Organic, signific|monthly| +9045|AAAAAAAAFFDCAAAA|2452425|2452454|DEPARTMENT|84|81|Unlikely directions see communities. Traditional clergy |monthly| +9046|AAAAAAAAGFDCAAAA|2452425|2452454|DEPARTMENT|84|82|Absolute ways spend then individual, required parties. Days develop only abroad video|monthly| +9047|AAAAAAAAHFDCAAAA|2452425|2452454|DEPARTMENT|84|83|Strange, liberal plans should hurt eventually like the arm|monthly| +9048|AAAAAAAAIFDCAAAA|2452425|2452454|DEPARTMENT|84|84|Crucial, old authorities keep quick schools. Only visual events will not hope from now on positi|monthly| +9049|AAAAAAAAJFDCAAAA|2452425|2452454|DEPARTMENT|84|85|Sure, national players would not go emotionally. So difficult leve|monthly| +9050|AAAAAAAAKFDCAAAA|2452425|2452454|DEPARTMENT|84|86|Strong islands may choose about local calls. Then military offices might lik|monthly| +9051|AAAAAAAALFDCAAAA|2452425|2452454|DEPARTMENT|84|87|Limited women might make otherwise in the victims. Tomorrow regular practices pay much personal d|monthly| +9052|AAAAAAAAMFDCAAAA|2452425|2452454|DEPARTMENT|84|88|Poor goods must submit previously prime, available visits. Short customers co|monthly| +9053|AAAAAAAANFDCAAAA|2452425|2452454|DEPARTMENT|84|89|Too royal chapters would pay personal artists; methods eliminate specially as |monthly| +9054|AAAAAAAAOFDCAAAA|2452425|2452454|DEPARTMENT|84|90|Houses house heavily potential wages; european pairs achieve |monthly| +9055|AAAAAAAAPFDCAAAA|2452425|2452454|DEPARTMENT|84|91|Junior, apparent arms will refer on a courses. Over true legs could lose vital difficulties. Natio|monthly| +9056|AAAAAAAAAGDCAAAA|2452425|2452454|DEPARTMENT|84|92|Conventional, exotic requirements wait on a stones. Immediately local sta|monthly| +9057|AAAAAAAABGDCAAAA|2452425|2452454|DEPARTMENT|84|93|More foreign conclusions use above groups. Children threaten now e|monthly| +9058|AAAAAAAACGDCAAAA|2452425|2452454|DEPARTMENT|84|94|Subjects will know for a sciences. Financial meals shall result inside; about alive years could not|monthly| +9059|AAAAAAAADGDCAAAA|2452425|2452454|DEPARTMENT|84|95|Possible years believe. British, personal advisers coul|monthly| +9060|AAAAAAAAEGDCAAAA|2452425|2452454||||Far, female magistrates used to become on a others; recen|monthly| +9061|AAAAAAAAFGDCAAAA|2452425|2452454|DEPARTMENT|84|97|Eventually nuclear comparisons shall not live most acid|monthly| +9062|AAAAAAAAGGDCAAAA|2452425|2452454|DEPARTMENT|84|98|Full, significant attitudes ought to buy then for a s|monthly| +9063|AAAAAAAAHGDCAAAA|2452425|2452454|DEPARTMENT|84|99|Whole trains investigate later. New, valid levels study big, dependent students. Great winners coul|monthly| +9064|AAAAAAAAIGDCAAAA|2452425|2452454|DEPARTMENT|84|100|Boys would live parents. Quickly difficult youngsters would not take probably new gifts. Now happy |monthly| +9065|AAAAAAAAJGDCAAAA|2452425|2452454|DEPARTMENT|84|101|Significant banks go actually little, appointed roots; bri|monthly| +9066|AAAAAAAAKGDCAAAA|2452425|2452454|DEPARTMENT|84|102|Employers mean perfectly. Clothes should need major, central patients. Certain dimensions migh|monthly| +9067|AAAAAAAALGDCAAAA|2452425|2452454|DEPARTMENT|84|103|Continuing, alternative developers used to accuse open unlikely hospitals. Councils refut|monthly| +9068|AAAAAAAAMGDCAAAA|2452425|2452454|DEPARTMENT|84|104|In particular national voters must listen free groups. Future, a|monthly| +9069|AAAAAAAANGDCAAAA|2452425|2452454|DEPARTMENT|84|105|Later silly studies explain just vital, political forms|monthly| +9070|AAAAAAAAOGDCAAAA|2452425|2452454|DEPARTMENT|84|106|High, green pieces could not take different techniq|monthly| +9071|AAAAAAAAPGDCAAAA|2452425|2452454|DEPARTMENT|84|107|Brief cells ought to hurry here in a supplies. Good s|monthly| +9072|AAAAAAAAAHDCAAAA|2452425|2452454|DEPARTMENT|84|108|Formal, social customs ought to report now at a time|monthly| +9073|AAAAAAAABHDCAAAA|2452455|2452484|DEPARTMENT|85|1|Crucial thousands cannot build forwards demands. Worthy, i|monthly| +9074|AAAAAAAACHDCAAAA|2452455|2452484|DEPARTMENT|85|2|Specific days leave. Particular reactions speak from a child|monthly| +9075|AAAAAAAADHDCAAAA|2452455|2452484|DEPARTMENT|85|3|About generous cases arrest fiercely dynamic teeth. Extra, |monthly| +9076|AAAAAAAAEHDCAAAA|2452455|2452484|DEPARTMENT|85|4|Horses ought to increase quite wrong classes. Rate|monthly| +9077|AAAAAAAAFHDCAAAA|2452455|2452484|DEPARTMENT|85|5|Social, nuclear years try legs. Permanent, spatial carers should not develop both livi|monthly| +9078|AAAAAAAAGHDCAAAA|2452455|2452484|DEPARTMENT|85|6|Medical opportunities ought to appreciate particularly a little |monthly| +9079|AAAAAAAAHHDCAAAA|2452455|2452484|DEPARTMENT|85|7|Large, very practitioners should conclude circumstanc|monthly| +9080|AAAAAAAAIHDCAAAA|2452455|2452484|DEPARTMENT|85|8|Similar women should give. Local words help far from other bits; attractive symb|monthly| +9081|AAAAAAAAJHDCAAAA|2452455|2452484|DEPARTMENT|85|9|Conventional girls may not go on a cases. Established, inter|monthly| +9082|AAAAAAAAKHDCAAAA|2452455|2452484|DEPARTMENT|85|10|Local, helpful periods can come. Common, appropriate proceedings may not condemn clear p|monthly| +9083|AAAAAAAALHDCAAAA|2452455|2452484|DEPARTMENT|85|11|Rather full cards suffer new, armed findings; distinct businessmen may not give now under a ma|monthly| +9084|AAAAAAAAMHDCAAAA|2452455|2452484|DEPARTMENT|85|12|Above crucial passengers should make minimum mothers; slowly obvious companies ought to give even a|monthly| +9085|AAAAAAAANHDCAAAA|2452455|2452484|DEPARTMENT|85|13|Benefits become on a assumptions. Fine ideas imagine such as a designs. Mean, stupid expe|monthly| +9086|AAAAAAAAOHDCAAAA|2452455|2452484|DEPARTMENT|85|14|Ethnic, southern users isolate even opportunities. Famous women mention c|monthly| +9087|AAAAAAAAPHDCAAAA|2452455|2452484|DEPARTMENT|85|15|Eyes wear commonly merely white shareholders; frequently very chemicals should come lucky, old ta|monthly| +9088|AAAAAAAAAIDCAAAA|2452455|2452484|DEPARTMENT|85|16|Things might settle of course level lips. Only full conservatives give |monthly| +9089|AAAAAAAABIDCAAAA|2452455|2452484|DEPARTMENT|85|17|Pale, additional hours shall not look thoughts. Sick parents would cross ever|monthly| +9090|AAAAAAAACIDCAAAA|2452455|2452484|DEPARTMENT|85|18|Clearly certain pupils shall start at least below good decisions. Global eyes may want over diff|monthly| +9091|AAAAAAAADIDCAAAA|2452455|2452484|DEPARTMENT|85|19|Social, true days understand then tensions. Far chemicals|monthly| +9092|AAAAAAAAEIDCAAAA|2452455|2452484|DEPARTMENT|85|20|Parts return as in the boards; just funny boats intend musical, interest|monthly| +9093|AAAAAAAAFIDCAAAA|2452455|2452484|DEPARTMENT|85|21|Especially following agents would go really previous peoples. Young memorie|monthly| +9094|AAAAAAAAGIDCAAAA|2452455|2452484|DEPARTMENT|85|22|Relationships think also female looks. Low, jewish attitudes explain|monthly| +9095|AAAAAAAAHIDCAAAA|2452455|2452484|DEPARTMENT|85|23|Recently minute arguments should tell often cells. Low foreign refugees shal|monthly| +9096|AAAAAAAAIIDCAAAA|2452455|2452484|DEPARTMENT|85|24|Men move african, beautiful elements. Necessary, f|monthly| +9097|AAAAAAAAJIDCAAAA|2452455|2452484|DEPARTMENT|85|25|Other, little assumptions send also good cuts. Aware, fair others woul|monthly| +9098|AAAAAAAAKIDCAAAA|2452455|2452484|DEPARTMENT|85|26|Too whole types worry there small nurses. Moreover important generations find scarcel|monthly| +9099|AAAAAAAALIDCAAAA|2452455|2452484|DEPARTMENT|85|27|Remote, other courts would pay often social drawings. Normal considerations could make. Almos|monthly| +9100|AAAAAAAAMIDCAAAA|2452455|2452484|DEPARTMENT|85|28|Violent, safe vehicles could make well from a goods. Inadequate |monthly| +9101|AAAAAAAANIDCAAAA|2452455|2452484|DEPARTMENT|85|29|Real recordings shall not foster noticeably soon surprised communities. Coastal, full ide|monthly| +9102|AAAAAAAAOIDCAAAA|2452455|2452484|DEPARTMENT|85|30|Powerful, ethnic files see also misleading, past pp.. |monthly| +9103|AAAAAAAAPIDCAAAA|2452455|2452484|DEPARTMENT|85|31|Members want more for a divisions. Recently open visits may no|monthly| +9104|AAAAAAAAAJDCAAAA|2452455|2452484|DEPARTMENT|85|32|Islands used to bear new defendants. Conventional, dead streets ought to mean likely, different|monthly| +9105|AAAAAAAABJDCAAAA|2452455|2452484|DEPARTMENT|85|33|Low cheap followers could not behave above then local males; men sh|monthly| +9106|AAAAAAAACJDCAAAA|2452455|2452484|DEPARTMENT|85|34|Objectives must increase for example open so-called wood|monthly| +9107|AAAAAAAADJDCAAAA|2452455|2452484|DEPARTMENT|85|35|Tears like already bare considerations. Social, good |monthly| +9108|AAAAAAAAEJDCAAAA|2452455|2452484|DEPARTMENT|85|36|Old, distinguished railways can go there funds. Royal years may sound|monthly| +9109|AAAAAAAAFJDCAAAA|2452455|2452484|DEPARTMENT|85|37|Colleagues include relative rules. Branches should become highly large goods. Likely forces co|monthly| +9110|AAAAAAAAGJDCAAAA|2452455|2452484|DEPARTMENT|85|38|Forms should stay joint, silver meals. Industrial hopes die new years. O|monthly| +9111|AAAAAAAAHJDCAAAA|2452455|2452484|DEPARTMENT|85|39|Early units would prove classes. Following needs die too regional, inc operations. Small |monthly| +9112|AAAAAAAAIJDCAAAA|2452455|2452484|DEPARTMENT|85|40|Needs plan in short; rare others might not guide; risks follow already. Men take members. Good mil|monthly| +9113|AAAAAAAAJJDCAAAA|2452455|2452484|DEPARTMENT|85|41|Good skills need mentally. Possible resources can work w|monthly| +9114|AAAAAAAAKJDCAAAA|2452455|2452484|DEPARTMENT|85|42|Even necessary buses say badly already great courts|monthly| +9115|AAAAAAAALJDCAAAA|2452455|2452484|DEPARTMENT|85|43|Sorry, universal waters come flat open changes. Views shall discuss from a police. Steps will cause|monthly| +9116|AAAAAAAAMJDCAAAA|2452455|2452484|DEPARTMENT|85|44|Jointly good citizens select even within a differences. Soviet events used to stimulate d|monthly| +9117|AAAAAAAANJDCAAAA|2452455|2452484|DEPARTMENT|85|45|Correct, neutral stones know in short small, new reasons. There racial t|monthly| +9118|AAAAAAAAOJDCAAAA|2452455|2452484|DEPARTMENT|85|46|Wrong, ruling concepts may not rise as low, necessary buildings; sof|monthly| +9119|AAAAAAAAPJDCAAAA|2452455|2452484|DEPARTMENT|85|47|Questions cannot take still dead eyes. Very related attitudes confess joi|monthly| +9120|AAAAAAAAAKDCAAAA|2452455|2452484|DEPARTMENT|85|48|Problems breed. Skills remember for long ill years. General paintings cope cards. Re|monthly| +9121|AAAAAAAABKDCAAAA|2452455|2452484|DEPARTMENT|85|49|Completely sudden chips like again to a doors. Good, constitutional buildings would not exist her|monthly| +9122|AAAAAAAACKDCAAAA|2452455|2452484|DEPARTMENT|85|50|Models should say later popular, foreign rights. En|monthly| +9123|AAAAAAAADKDCAAAA|2452455|2452484|DEPARTMENT|85|51|Better present talks buy only well original networks. Female,|monthly| +9124|AAAAAAAAEKDCAAAA|2452455|2452484|DEPARTMENT|85|52|Original origins should leave to the degrees. Eyes th|monthly| +9125|AAAAAAAAFKDCAAAA|2452455|2452484|DEPARTMENT|85|53|Surroundings ask then contributions; most long-term efforts |monthly| +9126|AAAAAAAAGKDCAAAA|2452455|2452484|DEPARTMENT|85|54|Urban years believe too dangerous, small councils. Physical, minor figures may extend|monthly| +9127|AAAAAAAAHKDCAAAA|2452455|2452484|DEPARTMENT|85|55|Scientific items shall show conferences. Great strings find nev|monthly| +9128|AAAAAAAAIKDCAAAA|2452455|2452484|DEPARTMENT|85|56|Dangerous, different islands kiss loans. Below various years exist lovingly high exercises|monthly| +9129|AAAAAAAAJKDCAAAA|2452455|2452484|DEPARTMENT|85|57|Already fair customers send future papers. Subsequent, long profits co|monthly| +9130|AAAAAAAAKKDCAAAA|2452455|2452484|DEPARTMENT|85|58|Other, thin charges inform eyes. Users must not hold strongly. Relevant, clear words fi|monthly| +9131|AAAAAAAALKDCAAAA|2452455|2452484|DEPARTMENT|85|59|Specific years must get much; rigid things know for a homes. General areas used to a|monthly| +9132|AAAAAAAAMKDCAAAA|2452455|2452484|DEPARTMENT|85|60|More historical factors test off a scientists; too attractive results orga|monthly| +9133|AAAAAAAANKDCAAAA|2452455|2452484|DEPARTMENT|85|61|Remaining quantities ought to accompany only delighted parents; qui|monthly| +9134|AAAAAAAAOKDCAAAA|2452455|2452484|DEPARTMENT|85|62|Wrong, level enemies could help also. Only, only friends could not wo|monthly| +9135|AAAAAAAAPKDCAAAA|2452455|2452484|DEPARTMENT|85|63|Open premises analyse; weeks shall laugh. Figures protect successfully ci|monthly| +9136|AAAAAAAAALDCAAAA|2452455|2452484|DEPARTMENT|85|64|Leaders upset more. Recent authors could clean psychological, final temperatures. Particular c|monthly| +9137|AAAAAAAABLDCAAAA|2452455|2452484|DEPARTMENT|85|65|Sufficient, bad dates ought to relinquish here conventions. For instan|monthly| +9138|AAAAAAAACLDCAAAA|2452455|2452484|DEPARTMENT|85|66|Southern, big churches slow short vehicles. Usual, criti|monthly| +9139|AAAAAAAADLDCAAAA|2452455|2452484|DEPARTMENT|85|67|Bizarre, english problems halt. Also dangerous sciences can transm|monthly| +9140|AAAAAAAAELDCAAAA|2452455|2452484|DEPARTMENT|85|68|Yet variable problems convince apparently things; only international divisions|monthly| +9141|AAAAAAAAFLDCAAAA|2452455|2452484|DEPARTMENT|85|69|Concerned offices take for a arms. Structures cannot use increasingly; cheap, empty w|monthly| +9142|AAAAAAAAGLDCAAAA|2452455|2452484|DEPARTMENT|85|70|Doubts show constitutional, particular investors. Great groups|monthly| +9143|AAAAAAAAHLDCAAAA|2452455|2452484|DEPARTMENT|85|71|Real exhibitions may not win all interests; mainly various authorities might draw interes|monthly| +9144|AAAAAAAAILDCAAAA|2452455|2452484|DEPARTMENT|85|72|Original findings ought to remember then too grim others. Main deaths cannot of|monthly| +9145|AAAAAAAAJLDCAAAA|2452455|2452484|DEPARTMENT|85|73|Arrangements know all right details. Large trees would contend. Independen|monthly| +9146|AAAAAAAAKLDCAAAA||2452484|DEPARTMENT||74|Usual, national unions fancy hard mildly kind teams. Lives may for|| +9147|AAAAAAAALLDCAAAA|2452455|2452484|DEPARTMENT|85|75|Up to modern hotels should realize ever humans. Total, sensible houses may n|monthly| +9148|AAAAAAAAMLDCAAAA|2452455|2452484|DEPARTMENT|85|76|However usual concentrations would remember only, indu|monthly| +9149|AAAAAAAANLDCAAAA|2452455|2452484|DEPARTMENT|85|77|So other terms drink now only agencies. Games shall not return for example far comp|monthly| +9150|AAAAAAAAOLDCAAAA|2452455|2452484|DEPARTMENT|85|78|Quickly other years could use subtle, prominent deaths. Speci|monthly| +9151|AAAAAAAAPLDCAAAA|2452455|2452484|DEPARTMENT|85|79|Successful, available polls take immediately great chairs; local, slow libraries replace alone. Go|monthly| +9152|AAAAAAAAAMDCAAAA|2452455|2452484|DEPARTMENT|85|80|Spatial, important subjects notice sure, strange trousers. Brief patients might separ|monthly| +9153|AAAAAAAABMDCAAAA|2452455|2452484|DEPARTMENT|85|81|Poor conditions used to organise too happy, likely members. Optio|monthly| +9154|AAAAAAAACMDCAAAA|2452455|2452484|DEPARTMENT|85|82|Other, large stories counter at the items. Old, old patients provide young patients; opportunit|monthly| +9155|AAAAAAAADMDCAAAA|2452455|2452484|DEPARTMENT|85|83|Producers could run slightly readily subtle feet. Again national groups recall hel|monthly| +9156|AAAAAAAAEMDCAAAA|2452455|2452484|DEPARTMENT|85|84|Again constant provinces might say dark, limited weeks. Late similar things |monthly| +9157|AAAAAAAAFMDCAAAA|2452455|2452484|DEPARTMENT|85|85|Severe, other hundreds used to conserve then addresses. Old images will not celebrate days; e|monthly| +9158|AAAAAAAAGMDCAAAA|2452455|2452484|DEPARTMENT|85|86|Leaders should cast very children; basic, important cities wi|monthly| +9159|AAAAAAAAHMDCAAAA|2452455|2452484|DEPARTMENT|85|87|Ways indicate years. Social men can put completely expert classes. Still ma|monthly| +9160|AAAAAAAAIMDCAAAA|2452455|2452484|DEPARTMENT|85|88|Also aware weeks could not get however healthy powers. False, |monthly| +9161|AAAAAAAAJMDCAAAA|2452455|2452484|DEPARTMENT|85|89|Flexible, successful children help respective appearances. Only increased |monthly| +9162|AAAAAAAAKMDCAAAA|2452455|2452484|DEPARTMENT|85|90|Significant, young years would not come recently so r|monthly| +9163|AAAAAAAALMDCAAAA|2452455|2452484|DEPARTMENT|85|91|Severe others must not obtain provinces. Steps get black, low weeks. Labour, traditional clas|monthly| +9164|AAAAAAAAMMDCAAAA|2452455|2452484|DEPARTMENT|85|92|Honest women look public signs; civil details used to waste ill around a meanings. Clear|monthly| +9165|AAAAAAAANMDCAAAA|2452455|2452484|DEPARTMENT|85|93|Available, administrative children begin frequently bit|monthly| +9166|AAAAAAAAOMDCAAAA|2452455|2452484|DEPARTMENT|85|94|Dull plans reinforce little methods; shares go probably earlier odd |monthly| +9167|AAAAAAAAPMDCAAAA|2452455|2452484|DEPARTMENT|85|95|Hours may not fight peoples. Male exercises impleme|monthly| +9168|AAAAAAAAANDCAAAA|2452455|2452484|DEPARTMENT|85|96|Trends support. Mice whisper conventional waters; judicial years fall strong|monthly| +9169|AAAAAAAABNDCAAAA|2452455|2452484|DEPARTMENT|85|97|Tables escape to a eyes. Special points escape leaders. Demands start powerful relations. Just l|monthly| +9170|AAAAAAAACNDCAAAA|2452455|2452484|DEPARTMENT|85|98|Most hostile areas like sales. Members laugh by a artists.|monthly| +9171|AAAAAAAADNDCAAAA|2452455|2452484|DEPARTMENT|85|99|Fast experimental tears must begin extremely new, long co|monthly| +9172|AAAAAAAAENDCAAAA|2452455|2452484|DEPARTMENT|85|100|Royal, great processes speed far important territori|monthly| +9173|AAAAAAAAFNDCAAAA|2452455|2452484|DEPARTMENT|85|101|Plants cut more now current problems. Different ways win very empl|monthly| +9174|AAAAAAAAGNDCAAAA|2452455|2452484|DEPARTMENT|85|102|Content organisations provide by a trains. Bases might make never exclusive, metropolitan eyebrows|monthly| +9175|AAAAAAAAHNDCAAAA|2452455|2452484|DEPARTMENT|85|103|Ruling parties modify beautiful beliefs. Exceptional outcomes ought t|monthly| +9176|AAAAAAAAINDCAAAA|2452455|2452484|DEPARTMENT|85|104|Never prime components lead even. Broadly standard materials will not note much already voluntar|monthly| +9177|AAAAAAAAJNDCAAAA|2452455|2452484|DEPARTMENT|85|105|Responsible, certain orders might place also with a animals; eyes shift all right large |monthly| +9178|AAAAAAAAKNDCAAAA|2452455|2452484|DEPARTMENT|85|106|Shortly inevitable supporters might build european, international photographs. Already differen|monthly| +9179|AAAAAAAALNDCAAAA|2452455|2452484|DEPARTMENT|85|107|Certain seats lead more good chairs. Negotiations say outcomes. Good employees know then |monthly| +9180|AAAAAAAAMNDCAAAA|2452455|2452484|DEPARTMENT|85|108|Rates contend later again early eyes. Clearly inner vessels will analyse both to a systems. Com|monthly| +9181|AAAAAAAANNDCAAAA|2452485|2452514|DEPARTMENT|86|1|Most physical obligations stand often circumstances. Organisers should find ways. Single, imp|monthly| +9182|AAAAAAAAONDCAAAA|2452485|2452514|DEPARTMENT|86|2|Authorities must like very important, specific articles. Institutions prove f|monthly| +9183|AAAAAAAAPNDCAAAA|2452485|2452514|DEPARTMENT|86|3|Members call specific years; fierce eyes avoid indeed|monthly| +9184|AAAAAAAAAODCAAAA|2452485|2452514|DEPARTMENT|86|4|Rights will not remember students. True contacts could not vary children. Strong, careful op|monthly| +9185|AAAAAAAABODCAAAA|2452485|2452514|DEPARTMENT|86|5|Local hundreds will not say powers; influential managers shall claim there for|monthly| +9186|AAAAAAAACODCAAAA|2452485|2452514|DEPARTMENT|86|6|Moments think well medical, main agreements. Corners must put also in a |monthly| +9187|AAAAAAAADODCAAAA|2452485|2452514|DEPARTMENT|86|7|Companies suggest only left children. Vehicles used to allow slowly cent|monthly| +9188|AAAAAAAAEODCAAAA|2452485|2452514|DEPARTMENT|86|8|Only new sounds design thus wild, impossible efforts. Here black even|monthly| +9189|AAAAAAAAFODCAAAA|2452485|2452514|DEPARTMENT|86|9|Very apparent shops must like also keen animals; prospective, christia|monthly| +9190|AAAAAAAAGODCAAAA|2452485|2452514|DEPARTMENT|86|10|Theories die also services. Liable, national seconds wou|monthly| +9191|AAAAAAAAHODCAAAA|2452485|2452514|DEPARTMENT|86|11|Political, apparent things will not play as well such as a types; service|monthly| +9192|AAAAAAAAIODCAAAA|2452485|2452514|DEPARTMENT|86|12|For instance holy possibilities discuss carefully ap|monthly| +9193|AAAAAAAAJODCAAAA|2452485|2452514|DEPARTMENT|86|13|Either ethnic weeks may not last no already new clouds|monthly| +9194|AAAAAAAAKODCAAAA|2452485|2452514|DEPARTMENT|86|14|Satisfied matters hear high existing feelings. Sternly|monthly| +9195|AAAAAAAALODCAAAA|2452485|2452514|DEPARTMENT|86|15|Busy days return. Questions ought to play labour investors. Final findings should not think fo|monthly| +9196|AAAAAAAAMODCAAAA|2452485|2452514|DEPARTMENT|86|16|Here violent telecommunications contact right new situations. Discipl|monthly| +9197|AAAAAAAANODCAAAA|2452485|2452514|DEPARTMENT|86|17|Social developments must bring adverse, deep records. Gr|monthly| +9198|AAAAAAAAOODCAAAA|2452485|2452514|DEPARTMENT|86|18|Possible, old careers grow statutory countries. Permanent, complete tables prevent at a supporters.|monthly| +9199|AAAAAAAAPODCAAAA|2452485|2452514|DEPARTMENT|86|19|Encouraging, afraid ideas might not understand minu|monthly| +9200|AAAAAAAAAPDCAAAA|2452485|2452514|DEPARTMENT|86|20|Established, proper hours move again parties; conferences will not exp|monthly| +9201|AAAAAAAABPDCAAAA|2452485|2452514|DEPARTMENT|86|21|Drugs ought to transfer early generally thin qualities. Used, dry explana|monthly| +9202|AAAAAAAACPDCAAAA|2452485|2452514|DEPARTMENT|86|22|Children stay there provisions. Terms shall continue of course perhaps other area|monthly| +9203|AAAAAAAADPDCAAAA|2452485|2452514|DEPARTMENT|86|23|High banks should not feel at all growing machines; small levels could emulate tiny trees. Lo|monthly| +9204|AAAAAAAAEPDCAAAA|2452485|2452514|DEPARTMENT|86|24|Teams turn certainly sufficient schools. Sufficient, |monthly| +9205|AAAAAAAAFPDCAAAA|2452485|2452514|DEPARTMENT|86|25|Men used to share markedly examples. Inc problems tell old walls. Welsh films enter fa|monthly| +9206|AAAAAAAAGPDCAAAA|2452485|2452514|DEPARTMENT|86|26|Fresh, opposite residents will send so certain conflicts. Roots arrange almost |monthly| +9207|AAAAAAAAHPDCAAAA|2452485|2452514|DEPARTMENT|86|27|Just possible women remember results. Wrong, national leve|monthly| +9208|AAAAAAAAIPDCAAAA|2452485|2452514|DEPARTMENT|86|28|Players write owners; occasional, typical tracks ought to get|monthly| +9209|AAAAAAAAJPDCAAAA|2452485|2452514|DEPARTMENT|86|29|Too local documents clear british nations. Generally successful patie|monthly| +9210|AAAAAAAAKPDCAAAA|2452485|2452514|DEPARTMENT|86|30|Red, new decisions violate as other, organic personnel. Suitable, gen|monthly| +9211|AAAAAAAALPDCAAAA|2452485|2452514|DEPARTMENT|86|31|Tests send very distinct proposals; interested prisoners shut however labour terms|monthly| +9212|AAAAAAAAMPDCAAAA|2452485|2452514|DEPARTMENT|86|32|Opportunities offer other commentators. Good, full eyes supply authorities. |monthly| +9213|AAAAAAAANPDCAAAA|2452485|2452514|DEPARTMENT|86|33|General, parliamentary children ought to make as however other reserva|monthly| +9214|AAAAAAAAOPDCAAAA|2452485|2452514|DEPARTMENT|86|34|Likely, illegal eyes should take men. Too flat daughters should not set during the|monthly| +9215|AAAAAAAAPPDCAAAA|2452485|2452514|DEPARTMENT|86|35|Too lovely tables cannot create twice hills. Territorial, constitutional weapons shoul|monthly| +9216|AAAAAAAAAAECAAAA|2452485|2452514|DEPARTMENT|86|36|Whole, important days must define losses. Different s|monthly| +9217|AAAAAAAABAECAAAA|2452485|2452514|DEPARTMENT|86|37|Full, clear contents should hesitate. National, classical schem|monthly| +9218|AAAAAAAACAECAAAA|2452485|2452514|DEPARTMENT|86|38|Waiting, principal results should free dangerous, waste times. Different, brown a|monthly| +9219|AAAAAAAADAECAAAA|2452485|2452514|DEPARTMENT|86|39|Eligible, hidden strategies register really military conditions|monthly| +9220|AAAAAAAAEAECAAAA|2452485|2452514|DEPARTMENT|86|40|Important cells matter even blue sentences. Changing, british polic|monthly| +9221|AAAAAAAAFAECAAAA|2452485|2452514|DEPARTMENT|86|41|Years might know possible, white needs. Measures used to |monthly| +9222|AAAAAAAAGAECAAAA|2452485|2452514|DEPARTMENT|86|42|Professional years would close poor, chosen windows; virtually corporate farmers|monthly| +9223|AAAAAAAAHAECAAAA|2452485|2452514|DEPARTMENT|86|43|Charges receive in a origins. Very, regular records |monthly| +9224|AAAAAAAAIAECAAAA|2452485|2452514|DEPARTMENT|86|44|Right due representatives move required, similar kinds. English, eager lips might |monthly| +9225|AAAAAAAAJAECAAAA|2452485|2452514|DEPARTMENT|86|45|Correctly medium items will pay goods. Angry, only leaves bea|monthly| +9226|AAAAAAAAKAECAAAA|2452485|2452514|DEPARTMENT|86|46|Friends show. Other, apparent problems may know then rough |monthly| +9227|AAAAAAAALAECAAAA|2452485|2452514|DEPARTMENT|86|47|Extraordinary cheeks used to make tight medical, normal eggs|monthly| +9228|AAAAAAAAMAECAAAA|2452485|2452514|DEPARTMENT|86|48|Police should reduce. Financial, good times would not lead|monthly| +9229|AAAAAAAANAECAAAA|2452485|2452514|DEPARTMENT|86|49|Eastern hours reconstruct all high users. Other weeks beware yesterday so much as long artis|monthly| +9230|AAAAAAAAOAECAAAA|2452485|2452514|DEPARTMENT|86|50|Variations realize here; nice purposes set recently weak, joi|monthly| +9231|AAAAAAAAPAECAAAA|2452485|2452514|DEPARTMENT|86|51|Subsidies destroy with a documents; lucky beaches may knit more years. Just specific|monthly| +9232|AAAAAAAAABECAAAA|2452485|2452514|DEPARTMENT|86|52|Offices shall cry more fully negative advertisements. Balls |monthly| +9233|AAAAAAAABBECAAAA|2452485|2452514|DEPARTMENT|86|53|Details call sides. Completely complete stands charge however from the|monthly| +9234|AAAAAAAACBECAAAA|2452485|2452514|DEPARTMENT|86|54|Institutions govern then video-taped places. Probably reasonable police must not hear. Isl|monthly| +9235|AAAAAAAADBECAAAA|2452485|2452514|DEPARTMENT|86|55|Situations provide in a books. Cultural, subsequent notes want sav|monthly| +9236|AAAAAAAAEBECAAAA|2452485|2452514|DEPARTMENT|86|56|Cars would indicate elements. Small, great machines make|monthly| +9237|AAAAAAAAFBECAAAA|2452485|2452514|DEPARTMENT|86|57|Negative stories get years. Samples know grateful daughters. Sides like political flowers. Girls|monthly| +9238|AAAAAAAAGBECAAAA|2452485|2452514|DEPARTMENT|86|58|Continuous, parliamentary goods used to encourage weeks. Hours minimise so|monthly| +9239|AAAAAAAAHBECAAAA|2452485|2452514|DEPARTMENT|86|59|Public, prominent companies use above. Good, good successes appreciate exactly; national, open un|monthly| +9240|AAAAAAAAIBECAAAA|2452485|2452514|DEPARTMENT|86|60|Changes draw backwards newspapers. Local solutions will not know too |monthly| +9241|AAAAAAAAJBECAAAA|2452485|2452514|DEPARTMENT|86|61|Sexual facilities operate therefore fair functions. Modern, |monthly| +9242|AAAAAAAAKBECAAAA|2452485|2452514|DEPARTMENT|86|62|In order scottish rates raise more about a sports. Aware, central lists might become wooden eyes; |monthly| +9243|AAAAAAAALBECAAAA|2452485|2452514|DEPARTMENT|86|63|Even old hours allow so. Forward, heavy trees bring secondly powerful, new costs. Galleries rule n|monthly| +9244|AAAAAAAAMBECAAAA|2452485|2452514|DEPARTMENT|86|64|Financial reports see probably monetary units. Hot, narrow functions will think please b|monthly| +9245|AAAAAAAANBECAAAA|2452485|2452514|DEPARTMENT|86|65|Holidays increase at the players; logical, military maps should no|monthly| +9246|AAAAAAAAOBECAAAA|2452485|2452514|DEPARTMENT|86|66|Discreetly continuing users can thrive then super, forward years. Necessary, open conditions|monthly| +9247|AAAAAAAAPBECAAAA|2452485|2452514|DEPARTMENT|86|67|Dangers would not qualify little to a years. Rural, similar |monthly| +9248|AAAAAAAAACECAAAA|2452485|2452514|DEPARTMENT|86|68|Sudden dogs find then large regions. Easy ports could weaken then b|monthly| +9249|AAAAAAAABCECAAAA|2452485|2452514|DEPARTMENT|86|69|Mechanically high things would go better for the enquiries. True characters sh|monthly| +9250|AAAAAAAACCECAAAA|2452485|2452514|DEPARTMENT|86|70|Quite foreign accidents can serve possible, common politicians; ancient members would loo|monthly| +9251|AAAAAAAADCECAAAA|2452485|2452514|DEPARTMENT|86|71|Available days should involve; libraries fit in addition male, financial units. W|monthly| +9252|AAAAAAAAECECAAAA|2452485|2452514|DEPARTMENT|86|72|Words ensure entire, final troops; now certain spirits could not understand curiousl|monthly| +9253|AAAAAAAAFCECAAAA|2452485|2452514|DEPARTMENT|86|73|Increasingly other agents could utilise much low, tiny bene|monthly| +9254|AAAAAAAAGCECAAAA|2452485|2452514|DEPARTMENT|86|74|Strong, christian boats must remember inextricably beautiful, different papers. Windows ought to|monthly| +9255|AAAAAAAAHCECAAAA|2452485|2452514|DEPARTMENT|86|75|White environments must wait traditional, old articles. Images shall|monthly| +9256|AAAAAAAAICECAAAA|2452485|2452514|DEPARTMENT|86|76|Australian rocks stay early massive, strong things. Automatic mana|monthly| +9257|AAAAAAAAJCECAAAA|2452485|2452514|DEPARTMENT|86|77|Almost white projects intervene most problems. Standard problems start best technological rights; |monthly| +9258|AAAAAAAAKCECAAAA|2452485|2452514|DEPARTMENT|86|78|Common forces cover differently bold children. Other feet c|monthly| +9259|AAAAAAAALCECAAAA|2452485|2452514|DEPARTMENT|86|79|Funny terms might end a bit through a languages; also necessary representa|monthly| +9260|AAAAAAAAMCECAAAA|2452485|2452514|DEPARTMENT|86|80|Particularly personal years like significantly pri|monthly| +9261|AAAAAAAANCECAAAA|2452485|2452514|DEPARTMENT|86|81|Then theoretical children know above able children. As positive sessions sho|monthly| +9262|AAAAAAAAOCECAAAA|2452485|2452514|DEPARTMENT|86|82|Organisational, leading pounds slide in a services. Meanwhile likely qualitie|monthly| +9263|AAAAAAAAPCECAAAA|2452485|2452514|DEPARTMENT|86|83|Scottish qualities remind; strong ways resist nevertheless days. Rar|monthly| +9264|AAAAAAAAADECAAAA|2452485|2452514|DEPARTMENT|86|84|Written, existing types waste against a artists. Nevertheless high patients shall us|monthly| +9265|AAAAAAAABDECAAAA|2452485|2452514|DEPARTMENT|86|85|Differences talk maybe. Certain, unlikely systems must go. Formal s|monthly| +9266|AAAAAAAACDECAAAA|2452485|2452514|DEPARTMENT|86|86|Anonymous plants look seldom in a times. Even early armies s|monthly| +9267|AAAAAAAADDECAAAA|2452485|2452514|DEPARTMENT|86|87|Particularly white companies can allow prices. German trustees might not feel never differ|monthly| +9268|AAAAAAAAEDECAAAA|2452485|2452514|DEPARTMENT|86|88|Also major clothes stop at least appropriate, military eyes. As old trends must not li|monthly| +9269|AAAAAAAAFDECAAAA|2452485|2452514|DEPARTMENT|86|89|Political, united aims look opposite recommendations. Slowly compl|monthly| +9270|AAAAAAAAGDECAAAA|2452485|2452514|DEPARTMENT|86|90|Deeply british relatives play before a expenses. Good songs would order more safely great side|monthly| +9271|AAAAAAAAHDECAAAA|2452485|2452514|DEPARTMENT|86|91|British, agricultural feet may not stick for a corporations. Books keep neverthel|monthly| +9272|AAAAAAAAIDECAAAA|2452485|2452514|DEPARTMENT|86|92|Parliamentary questions can spend social names. Equally prime members could not die financial|monthly| +9273|AAAAAAAAJDECAAAA|2452485|2452514|DEPARTMENT|86|93|Likewise single eyes find then correct children; public taxes deliver so very|monthly| +9274|AAAAAAAAKDECAAAA|2452485|2452514|DEPARTMENT|86|94|Particular leaves support more. Available, revolutionary references will recognise gener|monthly| +9275|AAAAAAAALDECAAAA|2452485|2452514|DEPARTMENT|86|95|Then essential bits leave still. Also specific streets see often painful cells; head conve|monthly| +9276|AAAAAAAAMDECAAAA|2452485|2452514|DEPARTMENT|86|96|Sides produce essential, right standards; committee|monthly| +9277|AAAAAAAANDECAAAA|2452485|2452514|DEPARTMENT|86|97|Places shall employ for a weeks. Professional, effective poets co|monthly| +9278|AAAAAAAAODECAAAA|2452485|2452514|DEPARTMENT|86|98|Social poems would finish rather earnings. Clearly level difficulties help particularly tired |monthly| +9279|AAAAAAAAPDECAAAA|2452485|2452514|DEPARTMENT|86|99|Amazing sheets believe in a images. Thoughtfully unusual sample|monthly| +9280|AAAAAAAAAEECAAAA|2452485|2452514|DEPARTMENT|86|100|Sometimes pleased chapters ought to follow. Still old assets take old curt|monthly| +9281|AAAAAAAABEECAAAA|2452485|2452514|DEPARTMENT|86|101|More environmental gains could not stay employees. Employees name. Only e|monthly| +9282|AAAAAAAACEECAAAA|2452485|2452514|DEPARTMENT|86|102|For instance dual companies may talk solely independent single points. Re|monthly| +9283|AAAAAAAADEECAAAA|2452485|2452514|DEPARTMENT|86|103|Risks look recent museums. Attempts believe possibly unh|monthly| +9284|AAAAAAAAEEECAAAA|2452485|2452514|DEPARTMENT|86|104|Other wishes get quickly extremely wild problems; original guests take as plans. More posit|monthly| +9285|AAAAAAAAFEECAAAA|2452485|2452514|DEPARTMENT|86|105|Triumphantly whole remains must remember later at leas|monthly| +9286|AAAAAAAAGEECAAAA|2452485|2452514|DEPARTMENT|86|106|Local relationships could not meet still deliberate|monthly| +9287|AAAAAAAAHEECAAAA|2452485|2452514|DEPARTMENT|86|107|Right, new animals call new days; newly common adva|monthly| +9288|AAAAAAAAIEECAAAA|2452485|2452514|DEPARTMENT|86|108|Leaders should challenge male speeches. Only standard players might not open so. Tragic areas sta|monthly| +9289|AAAAAAAAJEECAAAA|2452515|2452544|DEPARTMENT|87|1|Important officers require enough final prayers. Actions answer soon in th|monthly| +9290|AAAAAAAAKEECAAAA|2452515|2452544|DEPARTMENT|87|2|Ways make respondents. Historic activities used to obtain international environments; times should |monthly| +9291|AAAAAAAALEECAAAA|2452515|2452544|DEPARTMENT|87|3|True records must want ill over a citizens. Darkly middle |monthly| +9292|AAAAAAAAMEECAAAA|2452515|2452544|DEPARTMENT|87|4|Major men go. Wholly glad areas come usually in a processes. High, american times m|monthly| +9293|AAAAAAAANEECAAAA|2452515|2452544|DEPARTMENT|87|5|Difficult, important results used to register tomorrow. Complete matters negotiate really work|monthly| +9294|AAAAAAAAOEECAAAA|2452515|2452544|DEPARTMENT|87|6|Competitive fields use healthy men. Public changes should wr|monthly| +9295|AAAAAAAAPEECAAAA|2452515|2452544|DEPARTMENT|87|7|Children open later european things. Social, contemporary|monthly| +9296|AAAAAAAAAFECAAAA|2452515|2452544|DEPARTMENT|87|8|Planned, concerned colleagues make with a years. Customers make. Foreign, brief |monthly| +9297|AAAAAAAABFECAAAA|2452515|2452544|DEPARTMENT|87|9|Following teams turn; schools attain high. Subject children contain easily hi|monthly| +9298|AAAAAAAACFECAAAA|2452515|2452544|DEPARTMENT|87|10|Level, good requirements could imagine also children. Spaces c|monthly| +9299|AAAAAAAADFECAAAA|2452515|2452544|DEPARTMENT|87|11|Important sites must gain in a boundaries. Deliberate stones should mention too to a|monthly| +9300|AAAAAAAAEFECAAAA|2452515|2452544|DEPARTMENT|87|12|Male, mean officials appreciate only. Pounds shall not ruin german girls. Various, she|monthly| +9301|AAAAAAAAFFECAAAA|2452515|2452544|DEPARTMENT|87|13|Well general events must find nowhere to a hours. Changes see then di|monthly| +9302|AAAAAAAAGFECAAAA|2452515|2452544|DEPARTMENT|87|14|Old, general services ought to admit full, full banks. Employers explain from a creatures. E|monthly| +9303|AAAAAAAAHFECAAAA|2452515|2452544|DEPARTMENT|87|15|Green, male theories mind very home major tasks. Consistent sequences chan|monthly| +9304|AAAAAAAAIFECAAAA|2452515|2452544|DEPARTMENT|87|16|About similar devices use very. Citizens should say divine|monthly| +9305|AAAAAAAAJFECAAAA|2452515|2452544|DEPARTMENT|87|17|Possible, low mothers must not please smoothly on a hands. Hence cr|monthly| +9306|AAAAAAAAKFECAAAA|2452515|2452544|DEPARTMENT|87|18|Never historical kings could distribute now urban, future years. Requirements make now prior to th|monthly| +9307|AAAAAAAALFECAAAA|2452515|2452544|DEPARTMENT|87|19|High thin questions look even crucial numbers. Critical, high responses visit i|monthly| +9308|AAAAAAAAMFECAAAA|2452515|2452544|DEPARTMENT|87|20|Usually light colleagues should not produce inevitabl|monthly| +9309|AAAAAAAANFECAAAA|2452515|2452544|DEPARTMENT|87|21|Males give new questions. Cameras may not carry moments; useful countries shall e|monthly| +9310|AAAAAAAAOFECAAAA|2452515|2452544|DEPARTMENT|87|22|Operations disperse by a years. High words discuss very wonderful police; now appropriate acts ge|monthly| +9311|AAAAAAAAPFECAAAA|2452515|2452544|DEPARTMENT|87|23|Women shall set too in a children. Top shows say by a leaders. Developments would not under|monthly| +9312|AAAAAAAAAGECAAAA|2452515|2452544|DEPARTMENT|87|24|Strong, initial characters must limit to a performances. Lines co|monthly| +9313|AAAAAAAABGECAAAA|2452515|2452544|DEPARTMENT|87|25|Soon level others guide by a rooms. Lines may not appear also powers; only foreign pa|monthly| +9314|AAAAAAAACGECAAAA|2452515|2452544|DEPARTMENT|87|26|Children make over in the elections. Dry, eastern limits might calculat|monthly| +9315|AAAAAAAADGECAAAA|2452515|2452544|DEPARTMENT|87|27|Home gentle columns shall expect high; secret, artificial emotions identify maybe on |monthly| +9316|AAAAAAAAEGECAAAA|2452515|2452544|DEPARTMENT|87|28|Almost big individuals used to set obvious others; years |monthly| +9317|AAAAAAAAFGECAAAA|2452515|2452544|DEPARTMENT|87|29|Symptoms get individuals. Likely, fellow words participate. Willingly significant |monthly| +9318|AAAAAAAAGGECAAAA|2452515|2452544|DEPARTMENT|87|30|Previously necessary jeans might go local, suitable |monthly| +9319|AAAAAAAAHGECAAAA|2452515|2452544|DEPARTMENT|87|31|Areas must become goods. Soft, other fans include certain|monthly| +9320|AAAAAAAAIGECAAAA|2452515|2452544|DEPARTMENT|87|32|Still aware levels used to provide to a pictures. Options may improve fascinatin|monthly| +9321|AAAAAAAAJGECAAAA|2452515|2452544|DEPARTMENT|87|33|Open services boast about mixed, crucial skills. Internal, european projects may cr|monthly| +9322|AAAAAAAAKGECAAAA|2452515|2452544|DEPARTMENT|87|34|Afterwards foreign resources think. Drivers used to illuminate from a elements; fierce ye|monthly| +9323|AAAAAAAALGECAAAA|2452515|2452544|DEPARTMENT|87|35|Otherwise required passengers shall bring at last typical |monthly| +9324|AAAAAAAAMGECAAAA|2452515|2452544|DEPARTMENT|87|36|Complex areas ought to summon angry, acceptable winds. Financial men investigate with the files. As|monthly| +9325|AAAAAAAANGECAAAA|2452515|2452544|DEPARTMENT|87|37|Only bitter things give loyal issues. Western, fine proceedings should belie|monthly| +9326|AAAAAAAAOGECAAAA|2452515|2452544|DEPARTMENT|87|38|All special rounds want of course like a topics. Whole students must use signifi|monthly| +9327|AAAAAAAAPGECAAAA|2452515|2452544|DEPARTMENT|87|39|Electoral methods like automatically on a lips. Large, small years confront about; smal|monthly| +9328|AAAAAAAAAHECAAAA|2452515|2452544|DEPARTMENT|87|40|Right instructions relate lately. New changes shall not check. Blue, alone children save actual|monthly| +9329|AAAAAAAABHECAAAA|2452515|2452544|DEPARTMENT|87|41|Delicate, major feelings ought to arrive really socialist tenants; never w|monthly| +9330|AAAAAAAACHECAAAA|2452515|2452544|DEPARTMENT|87|42|Parental, top columns could organise before; recent, new thousands make national, early|monthly| +9331|AAAAAAAADHECAAAA|2452515|2452544|DEPARTMENT|87|43|Obviously planned paintings would exercise. Bad, annual |monthly| +9332|AAAAAAAAEHECAAAA|2452515|2452544|DEPARTMENT|87|44|Lives could sign certainly only atlantic children. Special, remarkable police respect good flo|monthly| +9333|AAAAAAAAFHECAAAA|2452515|2452544|DEPARTMENT|87|45|For the time being successful falls let unable, short co|monthly| +9334|AAAAAAAAGHECAAAA|2452515|2452544|DEPARTMENT|87|46|Strikingly minor expressions would project now autonomous crimes. Results shall tast|monthly| +9335|AAAAAAAAHHECAAAA|2452515|2452544|DEPARTMENT|87|47|Precise, advanced plans might not learn beds. Greatly economic conclusions sho|monthly| +9336|AAAAAAAAIHECAAAA|2452515|2452544|DEPARTMENT|87|48|Churches acknowledge currently benefits. Equivalent, other st|monthly| +9337|AAAAAAAAJHECAAAA|2452515|2452544|DEPARTMENT|87|49|British, outstanding eyes cannot sound matters. Firms see|monthly| +9338|AAAAAAAAKHECAAAA|2452515|2452544|DEPARTMENT|87|50|Entirely concerned occasions run also at the plans; inter|monthly| +9339|AAAAAAAALHECAAAA|2452515|2452544|DEPARTMENT|87|51|Other, old users prove out of a months; extreme men might not change strange, evil trends. Quite m|monthly| +9340|AAAAAAAAMHECAAAA|2452515|2452544|DEPARTMENT|87|52|Crowds see with a results. Only, official ways may stuff again only proper eyes.|monthly| +9341|AAAAAAAANHECAAAA|2452515|2452544|DEPARTMENT|87|53|Groups could rush a bit in a hopes; well public members would |monthly| +9342|AAAAAAAAOHECAAAA|2452515|2452544|DEPARTMENT|87|54|Also international occupations should rely together in a others. Almost strong ministers go m|monthly| +9343|AAAAAAAAPHECAAAA|2452515|2452544|DEPARTMENT|87|55|Ladies sing yet certain children. Pensions could not talk probably operations. Miles applau|monthly| +9344|AAAAAAAAAIECAAAA|2452515|2452544|DEPARTMENT|87|56|National, elderly details ought to decide however. Simple prices will not celebrate over in the |monthly| +9345|AAAAAAAABIECAAAA|2452515|2452544|DEPARTMENT|87|57|Ancient men outweigh almost correct characters. Actually imp|monthly| +9346|AAAAAAAACIECAAAA|2452515|2452544|DEPARTMENT|87|58|Suddenly young children benefit mean days. Still prime animals must not put ofte|monthly| +9347|AAAAAAAADIECAAAA|2452515|||||At least unknown persons shall offer somewhere. Remarkable wages suggest twice in order single fund|| +9348|AAAAAAAAEIECAAAA|2452515|2452544|DEPARTMENT|87|60|Generously continuing teachers would offer exactly efficient bla|monthly| +9349|AAAAAAAAFIECAAAA|2452515|2452544|DEPARTMENT|87|61|Institutional, new actions fly under tall, safe lives. Discussions act in the kids. Ne|monthly| +9350|AAAAAAAAGIECAAAA|2452515|2452544|DEPARTMENT|87|62|Letters may look yet neither nervous members. National boys shal|monthly| +9351|AAAAAAAAHIECAAAA|2452515|2452544|DEPARTMENT|87|63|Public conditions will remove despite the divisions; large years would not hide wide, federal resul|monthly| +9352|AAAAAAAAIIECAAAA|2452515|2452544|DEPARTMENT|87|64|Arab, net terms cause sometimes male children; interested emotions should know. Police|monthly| +9353|AAAAAAAAJIECAAAA|2452515|2452544|DEPARTMENT|87|65|But popular pools prove already. Professional, general proceedings recognise very |monthly| +9354|AAAAAAAAKIECAAAA|2452515|2452544|DEPARTMENT|87|66|Only dangerous sections play both now good cases. Cases live more days; ideolo|monthly| +9355|AAAAAAAALIECAAAA|2452515|2452544|DEPARTMENT|87|67|Particularly raw complaints take very also clear guards. Expected, dark experiments s|monthly| +9356|AAAAAAAAMIECAAAA|2452515|2452544|DEPARTMENT|87|68|Criteria could think clear, british goods. Usually economic men add on th|monthly| +9357|AAAAAAAANIECAAAA|2452515|2452544|DEPARTMENT|87|69|Once bad images can find so literary details. Accounts could not get in a powers. Str|monthly| +9358|AAAAAAAAOIECAAAA|2452515|2452544|DEPARTMENT|87|70|Brothers grasp as men. Current rocks deal little closely fair drawings; e|monthly| +9359|AAAAAAAAPIECAAAA|2452515|2452544|DEPARTMENT|87|71|Whole flowers forget sometimes; expected principles can arise so; usually am|monthly| +9360|AAAAAAAAAJECAAAA|2452515|2452544|DEPARTMENT|87|72|Ibid german alternatives may not hold delicate, inevitable teachers. Never new losses might co|monthly| +9361|AAAAAAAABJECAAAA|2452515|2452544|DEPARTMENT|87|73|Different, national hotels may not die full goods. Cups see good directors. Drugs|monthly| +9362|AAAAAAAACJECAAAA|2452515|2452544|DEPARTMENT|87|74|By no means sophisticated cases lose over. Difficult villages shall not carry |monthly| +9363|AAAAAAAADJECAAAA|2452515|2452544|DEPARTMENT|87|75|Special, outer types escape for an years; american countries say much by the brothers. Context|monthly| +9364|AAAAAAAAEJECAAAA|2452515|2452544|DEPARTMENT|87|76|Extensively human specialists may not hear at all runs. Black, crazy records sha|monthly| +9365|AAAAAAAAFJECAAAA|2452515|2452544|DEPARTMENT|87|77|Useful services seem in a others. Unemployed roots|monthly| +9366|AAAAAAAAGJECAAAA|2452515|2452544|DEPARTMENT|87|78|Civil, environmental feet could speak also other, other farmers. Pure questions |monthly| +9367|AAAAAAAAHJECAAAA|2452515|2452544|DEPARTMENT|87|79|Sure, local appearances answer at a personnel. Designs help here now different times. Social, b|monthly| +9368|AAAAAAAAIJECAAAA|2452515|2452544|DEPARTMENT|87|80|Markets must fend eventually to a parties. Sure, white issues|monthly| +9369|AAAAAAAAJJECAAAA|2452515|2452544|DEPARTMENT|87|81|Traditional, theoretical layers would build all important, succe|monthly| +9370|AAAAAAAAKJECAAAA|2452515|2452544|DEPARTMENT|87|82|Mechanical, wealthy ministers can like still like the years. Roses must not escape rough|monthly| +9371|AAAAAAAALJECAAAA|2452515|2452544|DEPARTMENT|87|83|Entire facilities could not mount now elections. More standard enemies ought to respond f|monthly| +9372|AAAAAAAAMJECAAAA|2452515|2452544|DEPARTMENT|87|84|Again personal views like increasingly; clinical, steady frames could not|monthly| +9373|AAAAAAAANJECAAAA|2452515|2452544|DEPARTMENT|87|85|External sides must not help from a females. Democrats used to see apart dramati|monthly| +9374|AAAAAAAAOJECAAAA|2452515|2452544|DEPARTMENT|87|86|Enough evident dates might not make serious parts. Over political stages cannot get|monthly| +9375|AAAAAAAAPJECAAAA|2452515|2452544|DEPARTMENT|87|87|Western, just eggs judge huge transactions; today public relationships defend about|monthly| +9376|AAAAAAAAAKECAAAA|2452515|2452544|DEPARTMENT|87|88|Fixed, increased members shall not jeopardise only to the players. Deaths |monthly| +9377|AAAAAAAABKECAAAA|2452515|2452544|DEPARTMENT|87|89|Various, statutory things tell logically days. Experts come early others.|monthly| +9378|AAAAAAAACKECAAAA|2452515|2452544|DEPARTMENT|87|90|About foreign years recall only authorities; manufacturers might not fail relations. Rare, frequen|monthly| +9379|AAAAAAAADKECAAAA|2452515|2452544|DEPARTMENT|87|91|Continental, wooden curtains shall not trace too. Appropriate, other views exhibit|monthly| +9380|AAAAAAAAEKECAAAA|2452515|2452544|DEPARTMENT|87|92|Little rates give by a years; diplomatic, basic eyes hang in a times. Fresh, historical|monthly| +9381|AAAAAAAAFKECAAAA|2452515|2452544|DEPARTMENT|87|93|Virtually true others share still. Natural years would find either new, strong pa|monthly| +9382|AAAAAAAAGKECAAAA|2452515|2452544|DEPARTMENT|87|94|Similar, silent arrangements can share for the most part. Vulnerable, painful sites would|monthly| +9383|AAAAAAAAHKECAAAA|2452515|2452544|DEPARTMENT|87|95|Primary, dead systems shall not want centres. Giant, mal|monthly| +9384|AAAAAAAAIKECAAAA|2452515|2452544|DEPARTMENT|87|96|Political groups used to take at all. Views raise for a politicians; democratic, private styles ta|monthly| +9385|AAAAAAAAJKECAAAA|2452515|2452544|DEPARTMENT|87|97|Also pale matches must not counter always out of a feet; employers need. Unable,|monthly| +9386|AAAAAAAAKKECAAAA|2452515|2452544|DEPARTMENT|87|98|Perfectly human operations would come; useful, internal thanks must dismiss. Actions fathom ope|monthly| +9387|AAAAAAAALKECAAAA|2452515|2452544|DEPARTMENT|87|99|Times tell then by a schools. Never clear hundreds assume obviously in favour of a women. Ris|monthly| +9388|AAAAAAAAMKECAAAA|2452515|2452544|DEPARTMENT|87|100|Huge families would produce so cars. Efficient, close women try|monthly| +9389|AAAAAAAANKECAAAA|2452515|2452544|DEPARTMENT|87|101|New, tough relationships cover necessary, inevitable costs. Curious scho|monthly| +9390|AAAAAAAAOKECAAAA|2452515|2452544|DEPARTMENT|87|102|Statutory, high services turn sometimes wide persons. The|monthly| +9391|AAAAAAAAPKECAAAA|2452515|2452544|DEPARTMENT|87|103|Cells may go over again english children. Ever european teeth insert again more than fine|monthly| +9392|AAAAAAAAALECAAAA|2452515|2452544|DEPARTMENT|87|104|Now live assumptions should leave aside. Factors ought to look; very urgent skills may assist jobs.|monthly| +9393|AAAAAAAABLECAAAA|2452515|2452544|DEPARTMENT|87|105|Strange, european jobs ought to contain as considerable countries. Fears enter|monthly| +9394|AAAAAAAACLECAAAA|2452515|2452544|DEPARTMENT|87|106|Different proposals may not close able, other systems. Proceedings fit on |monthly| +9395|AAAAAAAADLECAAAA|2452515|2452544|DEPARTMENT|87|107|Most fun sections would not bear great, good schools. Autonomous |monthly| +9396|AAAAAAAAELECAAAA|2452515|2452544|DEPARTMENT|87|108|However bare countries could get european, easy rates. Public, america|monthly| +9397|AAAAAAAAFLECAAAA|2452545|2452574|DEPARTMENT|88|1|Possible others receive small women. National, strategic chairs sha|monthly| +9398|AAAAAAAAGLECAAAA|2452545|2452574|DEPARTMENT|88|2|Problems check about a signs. Quick, german differences silence afterwards known, gastric discussi|monthly| +9399|AAAAAAAAHLECAAAA|2452545|2452574|DEPARTMENT|88|3|Hence other miles accept sometimes remains; so various men may dissolve even neverthe|monthly| +9400|AAAAAAAAILECAAAA|2452545|2452574|DEPARTMENT|88|4|Very clear owners realise so military, democratic youngsters; girls understand forward however w|monthly| +9401|AAAAAAAAJLECAAAA|2452545|2452574|DEPARTMENT|88|5|Steps hesitate clear, excessive servants. Posts might replace glad, ancient g|monthly| +9402|AAAAAAAAKLECAAAA|2452545|2452574|DEPARTMENT|88|6|Vital, early lessons will not sound only at least as|monthly| +9403|AAAAAAAALLECAAAA|2452545|2452574|DEPARTMENT|88|7|Other, british strangers may not place cold, current gaps. Boundaries overcome slowly. M|monthly| +9404|AAAAAAAAMLECAAAA|2452545|2452574|DEPARTMENT|88|8|Clear, other circumstances encourage modern, darling enquiries; too permanent issues can fin|monthly| +9405|AAAAAAAANLECAAAA|2452545|2452574|DEPARTMENT|88|9|Problems cannot persuade finally fair accidents. Enough ro|monthly| +9406|AAAAAAAAOLECAAAA|2452545|2452574|DEPARTMENT|88|10|Bad lads must dream therefore modern, whole services. Very red requirements distribute parties. |monthly| +9407|AAAAAAAAPLECAAAA|2452545|2452574|DEPARTMENT|88|11|Good, silent types sit both more public hands. Normal, odd seconds must not purch|monthly| +9408|AAAAAAAAAMECAAAA|2452545|2452574|DEPARTMENT|88|12|International services will not transform recently stupid libraries. So yellow ar|monthly| +9409|AAAAAAAABMECAAAA|2452545|2452574|DEPARTMENT|88|13|Overall, constant cases admire too; teachers record countries. Important prices adopt into a hous|monthly| +9410|AAAAAAAACMECAAAA|2452545|2452574|DEPARTMENT|88|14|Environmental, economic members feed simply likely minutes. Small tories sh|monthly| +9411|AAAAAAAADMECAAAA|2452545|2452574|DEPARTMENT|88|15|Strong servants offset as practical managers. White results shal|monthly| +9412|AAAAAAAAEMECAAAA|2452545|2452574|DEPARTMENT|88|16|Months may explain very low regulations. All poor pict|monthly| +9413|AAAAAAAAFMECAAAA|2452545|2452574|DEPARTMENT|88|17|Problems kill and so on leading friends. Only smal|monthly| +9414|AAAAAAAAGMECAAAA|2452545|2452574|DEPARTMENT|88|18|Coherent patients should travel tomorrow stands; g|monthly| +9415|AAAAAAAAHMECAAAA|2452545|2452574|DEPARTMENT|88|19|Now african studies sit late worth a masters. Selective consumers ought to pi|monthly| +9416|AAAAAAAAIMECAAAA|2452545|2452574|DEPARTMENT|88|20|Results say. Officers speak good others; standards can labour ideal, full thin|monthly| +9417|AAAAAAAAJMECAAAA|2452545|2452574|DEPARTMENT|88|21|Plants exploit. Nearly extra contents tell possibilities. Particularly substantial|monthly| +9418|AAAAAAAAKMECAAAA|2452545|2452574|DEPARTMENT|88|22|International, german democrats should acquire however great, social councils. Local numbers should|monthly| +9419|AAAAAAAALMECAAAA|2452545|2452574|DEPARTMENT|88|23|Homes carry strongly coastal, good weeks. Popular police tell like the w|monthly| +9420|AAAAAAAAMMECAAAA|2452545|2452574|DEPARTMENT|88|24|Societies keep more almost wonderful questions. Sometimes warm boards swim heavily |monthly| +9421|AAAAAAAANMECAAAA|2452545|2452574|DEPARTMENT|88|25|Recent students should not look strictly. Operations would say; as |monthly| +9422|AAAAAAAAOMECAAAA|2452545|2452574|DEPARTMENT|88|26|Fortunate, british techniques involve fun, armed relations|monthly| +9423|AAAAAAAAPMECAAAA|2452545|2452574|DEPARTMENT|88|27|Immediately interested issues denote possibly. Christian, interesting tasks would improv|monthly| +9424|AAAAAAAAANECAAAA|2452545|2452574|DEPARTMENT|88|28|Lovely, particular decisions must show today. Necessarily delightful partie|monthly| +9425|AAAAAAAABNECAAAA|2452545|2452574|DEPARTMENT|88|29|Values may give long, early units. High questions used to make. Red, sure s|monthly| +9426|AAAAAAAACNECAAAA|2452545|2452574|DEPARTMENT|88|30|Anyway free buildings run. Following, average negotiations should|monthly| +9427|AAAAAAAADNECAAAA|2452545|2452574|DEPARTMENT|88|31|Women protest especially steady routes. Later new courts will turn quickly extra |monthly| +9428|AAAAAAAAENECAAAA|2452545|2452574|DEPARTMENT|88|32|Easily top students believe however; probably general regions come poss|monthly| +9429|AAAAAAAAFNECAAAA|2452545|2452574|DEPARTMENT|88|33|So interested figures gather finally; then new character|monthly| +9430|AAAAAAAAGNECAAAA|2452545|2452574|DEPARTMENT|88|34|Parliamentary sources call at least manufacturing, electronic characters. Nearer various|monthly| +9431|AAAAAAAAHNECAAAA|2452545|2452574|DEPARTMENT|88|35|Effective, great blues ought to take sadly by a workers. Sales would kee|monthly| +9432|AAAAAAAAINECAAAA|2452545|2452574|DEPARTMENT|88|36|Remote eggs possess new minds. Economic systems fly shares. Somewhere physical powers acquire|monthly| +9433|AAAAAAAAJNECAAAA|2452545|2452574|DEPARTMENT|88|37|Boots could consider. Fun, visual recommendations oug|monthly| +9434|AAAAAAAAKNECAAAA|2452545|2452574|DEPARTMENT|88|38|Railways might give only annual activities. Centres|monthly| +9435|AAAAAAAALNECAAAA|2452545|2452574|DEPARTMENT|88|39|Further superior investors believe unfortunately m|monthly| +9436|AAAAAAAAMNECAAAA|2452545|||88||Increased, practical fingers should see as please concerned words. So afraid years use. || +9437|AAAAAAAANNECAAAA|2452545|2452574|DEPARTMENT|88|41|British elections prejudice especially. Stores accept mo|monthly| +9438|AAAAAAAAONECAAAA|2452545|2452574|DEPARTMENT|88|42|Totally basic goals turn also as a whole important terms. Customers would |monthly| +9439|AAAAAAAAPNECAAAA|2452545|2452574|DEPARTMENT|88|43|Vast, little questions will not know investments. Weapons must appear. Companies might not show gen|monthly| +9440|AAAAAAAAAOECAAAA|2452545|2452574|DEPARTMENT|88|44|Friends enable now deep kinds. Especially peculiar agents may like quick wrong, deliberat|monthly| +9441|AAAAAAAABOECAAAA|2452545|2452574|DEPARTMENT|88|45|Wishes may acquire afraid, little servants. Scientific others protest ago sites|monthly| +9442|AAAAAAAACOECAAAA|2452545|2452574|DEPARTMENT|88|46|Always spectacular skills propose in a children. Ultimately good patients allow. Form|monthly| +9443|AAAAAAAADOECAAAA|2452545|2452574|DEPARTMENT|88|47|Only associated weeks delay maybe in a strengths; whole, foreign auditors must not remember but |monthly| +9444|AAAAAAAAEOECAAAA|2452545|2452574|DEPARTMENT|88|48|Acceptable women say also popular, consistent hours; considerable, free bottles should say therefor|monthly| +9445|AAAAAAAAFOECAAAA|2452545|2452574|DEPARTMENT|88|49|Giant, bare developments will develop necessary, existing appearances. Wives remembe|monthly| +9446|AAAAAAAAGOECAAAA|2452545|2452574|DEPARTMENT|88|50|Criticisms should own most in a women. Following sources lead political trials. Circles stay i|monthly| +9447|AAAAAAAAHOECAAAA|2452545|2452574|DEPARTMENT|88|51|Probably human matters may contribute. Detailed, british services enter here. Military minutes|monthly| +9448|AAAAAAAAIOECAAAA|2452545|2452574|DEPARTMENT|88|52|Other banks enforce in a reforms. Still general requests may |monthly| +9449|AAAAAAAAJOECAAAA|2452545|2452574|DEPARTMENT|88|53|Great, wild difficulties could not get indeed competitive, military ministers. Major, japanese p|monthly| +9450|AAAAAAAAKOECAAAA|2452545|2452574|DEPARTMENT|88|54|Documents resolve seriously. Customs shall fall eventual, important prices|monthly| +9451|AAAAAAAALOECAAAA|2452545|2452574|DEPARTMENT|88|55|Boats will not clear regular, surprising respondents; just economic co|monthly| +9452|AAAAAAAAMOECAAAA|2452545|2452574|DEPARTMENT|88|56|Yesterday mechanical interviews run more mountains. Possible, ma|monthly| +9453|AAAAAAAANOECAAAA|2452545|2452574|DEPARTMENT|88|57|Plates tell better ever internal meetings. Else vari|monthly| +9454|AAAAAAAAOOECAAAA|2452545|2452574|DEPARTMENT|88|58|Physical, new symptoms call; already different officer|monthly| +9455|AAAAAAAAPOECAAAA|2452545|2452574|DEPARTMENT|88|59|Ever small attitudes see far marks. Very old towns can remain very stan|monthly| +9456|AAAAAAAAAPECAAAA|2452545|2452574|DEPARTMENT|88|60|Important, new weeks close about to a legs; little point|monthly| +9457|AAAAAAAABPECAAAA|2452545|2452574|DEPARTMENT|88|61|Years complain. Special, patient degrees satisfy here regular scientists. Pretty houses |monthly| +9458|AAAAAAAACPECAAAA|2452545|2452574|DEPARTMENT|88|62|Underlying, little soldiers blame even. Local photographs woul|monthly| +9459|AAAAAAAADPECAAAA|2452545|2452574|DEPARTMENT|88|63|Studies should not live simply in a companies. Now fi|monthly| +9460|AAAAAAAAEPECAAAA|2452545|2452574|DEPARTMENT|88|64|Extended areas cannot escape parental, ultimate discussions. So irrelevant dollars |monthly| +9461|AAAAAAAAFPECAAAA|2452545|2452574|DEPARTMENT|88|65|Then american voters think out of a police. Southern, bad schools sh|monthly| +9462|AAAAAAAAGPECAAAA|2452545|2452574|DEPARTMENT|88|66|Still strict countries argue only good women. Sweet exercis|monthly| +9463|AAAAAAAAHPECAAAA|2452545|2452574|DEPARTMENT|88|67|Old, patient concepts could not arrange recently. Possible, other wages argue wrong, interest|monthly| +9464|AAAAAAAAIPECAAAA|2452545|2452574|DEPARTMENT|88|68|Profits try now stones. Different projects buy classes; elderly papers |monthly| +9465|AAAAAAAAJPECAAAA|2452545|2452574|DEPARTMENT|88|69|Potential parties allow daily clearly brief rights; obvious, commercial years ask cheaply. Poli|monthly| +9466|AAAAAAAAKPECAAAA|2452545|2452574|DEPARTMENT|88|70|Leaders educate as little machines. Undoubtedly short pic|monthly| +9467|AAAAAAAALPECAAAA|2452545|2452574|DEPARTMENT|88|71|Important, legal departments turn alone experiments. Different, fine mat|monthly| +9468|AAAAAAAAMPECAAAA|2452545|2452574|DEPARTMENT|88|72|Fingers ought to weave new troops. Then rare efforts play just urban leaves. Ol|monthly| +9469|AAAAAAAANPECAAAA|2452545|2452574|DEPARTMENT|88|73|Complex papers may assume never from a opinions. Hours note still allega|monthly| +9470|AAAAAAAAOPECAAAA|2452545|2452574|DEPARTMENT|88|74|There strong attacks see private, compatible relationships. Strong t|monthly| +9471|AAAAAAAAPPECAAAA|2452545|2452574|DEPARTMENT|88|75|Men might enrich enough really other rows. More golden hours keep directly realistic, on|monthly| +9472|AAAAAAAAAAFCAAAA|2452545|2452574|DEPARTMENT|88|76|Others consider yet. Items will not beat good friends. Other, other years review ve|monthly| +9473|AAAAAAAABAFCAAAA|2452545|2452574|DEPARTMENT|88|77|Stages operate however corporate, mental individuals. Possible firms cannot achieve|monthly| +9474|AAAAAAAACAFCAAAA|2452545|2452574|DEPARTMENT|88|78|Central, big forces should not know ever skilled assessments. There |monthly| +9475|AAAAAAAADAFCAAAA|2452545|2452574|DEPARTMENT|88|79|Children must not remember surely families. Legal, likely words will not re|monthly| +9476|AAAAAAAAEAFCAAAA|2452545|2452574|DEPARTMENT|88|80|Plants may train to the lives. Main, royal members mean here other, black chiefs. Comprehen|monthly| +9477|AAAAAAAAFAFCAAAA|2452545|2452574|DEPARTMENT|88|81|Skills want so labour photographs. Little numbers consider more successes. Better mag|monthly| +9478|AAAAAAAAGAFCAAAA|2452545|2452574|DEPARTMENT|88|82|Talks try. Successful, leading members forgive meetings.|monthly| +9479|AAAAAAAAHAFCAAAA|2452545|2452574|DEPARTMENT|88|83|Then ordinary trees expect. Proposals ignore just at the individuals. Dead things c|monthly| +9480|AAAAAAAAIAFCAAAA|2452545|2452574|DEPARTMENT|88|84|Other, hot assets make deliberately. Foreign, total kinds see previous w|monthly| +9481|AAAAAAAAJAFCAAAA|2452545|2452574|DEPARTMENT|88|85|Much trying farmers can attend most memories. Classes must evaluate therefore women; profits dem|monthly| +9482|AAAAAAAAKAFCAAAA|2452545|2452574|DEPARTMENT|88|86|Smooth ties will forgive here more internal kilometres. Immediate patients|monthly| +9483|AAAAAAAALAFCAAAA|2452545|2452574|DEPARTMENT|88|87|Colourful managers shall not tidy consciously vital, physical transactions. Hours should recall si|monthly| +9484|AAAAAAAAMAFCAAAA|2452545|2452574|DEPARTMENT|88|88|Clients could get most beneath the minutes; then fresh cells shall motivate no|monthly| +9485|AAAAAAAANAFCAAAA|2452545|2452574|DEPARTMENT|88|89|Subsequent, british children shall not become only on |monthly| +9486|AAAAAAAAOAFCAAAA|2452545|2452574|DEPARTMENT|88|90|Prices shall start less bad artists. Huge, eventual d|monthly| +9487|AAAAAAAAPAFCAAAA|2452545|2452574|DEPARTMENT|88|91|Ready, big girls ought to profit. Symbolic, possible companies provide. Underneath political n|monthly| +9488|AAAAAAAAABFCAAAA|2452545|2452574|DEPARTMENT|88|92|Human skills hide too local months. Waste, new results would ring even police; extr|monthly| +9489|AAAAAAAABBFCAAAA|2452545|2452574|DEPARTMENT|88|93|Real, white offices sit away from a conditions. Serious expectations catch. Low grou|monthly| +9490|AAAAAAAACBFCAAAA|2452545|2452574|DEPARTMENT|88|94|Problems end national, complete americans. Dead, only conservatives switch quickly thr|monthly| +9491|AAAAAAAADBFCAAAA|2452545|2452574|DEPARTMENT|88|95|Important, european problems know feet; long, obvious hills could prefe|monthly| +9492|AAAAAAAAEBFCAAAA|2452545|2452574|DEPARTMENT|88|96|Available plants might not live generally sometimes white departments. Rare,|monthly| +9493|AAAAAAAAFBFCAAAA|2452545|2452574|DEPARTMENT|88|97|White, easy churches deny regularly personal secrets. Available |monthly| +9494|AAAAAAAAGBFCAAAA|2452545|2452574|DEPARTMENT|88|98|Literary, modern employers go african, worthy plants. Countries shall not promote never bla|monthly| +9495|AAAAAAAAHBFCAAAA|2452545|2452574|DEPARTMENT|88|99|Actually bloody parents cannot get so black legal horses. Initiative|monthly| +9496|AAAAAAAAIBFCAAAA|2452545|2452574|DEPARTMENT|88|100|Suggestions ought to isolate other children. Evil men know for a years. Modern, |monthly| +9497|AAAAAAAAJBFCAAAA|2452545|2452574|DEPARTMENT|88|101|Thus important measures should move aware, local needs. Better good ame|monthly| +9498|AAAAAAAAKBFCAAAA|2452545|2452574|DEPARTMENT|88|102|Simply typical rules can bring thereafter patients. Bo|monthly| +9499|AAAAAAAALBFCAAAA|2452545|2452574|DEPARTMENT|88|103|Thereby bad guidelines ought to proceed local centuries. Often wonderful |monthly| +9500|AAAAAAAAMBFCAAAA|2452545|2452574|DEPARTMENT|88|104|Usually average runs find downstairs students. Intimate,|monthly| +9501|AAAAAAAANBFCAAAA|2452545|2452574|DEPARTMENT|88|105|Quite running bodies go afterwards medieval, rich s|monthly| +9502|AAAAAAAAOBFCAAAA|2452545|2452574|DEPARTMENT|88|106|Here average files boost by a industries. Willing, popular c|monthly| +9503|AAAAAAAAPBFCAAAA|2452545|2452574|DEPARTMENT|88|107|Fresh publications listen then backs. Alive bars used to inv|monthly| +9504|AAAAAAAAACFCAAAA|2452545|2452574|DEPARTMENT|88|108|Heavy aspects see letters. Human sheets may not express cons|monthly| +9505|AAAAAAAABCFCAAAA|2452575|2452604|DEPARTMENT|89|1|Effectively general proposals experience about on a leads. Too wonderful men |monthly| +9506|AAAAAAAACCFCAAAA|2452575|2452604|DEPARTMENT|89|2|Directors must convene. Difficult, brave views damage fun, political resu|monthly| +9507|AAAAAAAADCFCAAAA|2452575|2452604|DEPARTMENT|89|3|Precisely regular years will not arrive years. Students should prod|monthly| +9508|AAAAAAAAECFCAAAA|2452575|2452604|DEPARTMENT|89|4|Instead little centuries appear below from a nations. Early international ways used to mo|monthly| +9509|AAAAAAAAFCFCAAAA|2452575|2452604|DEPARTMENT|89|5|Popular councils might consider certainly poor, super customers. Campaigns will pay completel|monthly| +9510|AAAAAAAAGCFCAAAA|2452575|2452604|DEPARTMENT|89|6|Great children tell very under a lessons. Western elections may not take. Also ot|monthly| +9511|AAAAAAAAHCFCAAAA|2452575|2452604|DEPARTMENT|89|7|Old, armed materials go ill at a tests. Young, little beds|monthly| +9512|AAAAAAAAICFCAAAA|2452575|2452604|DEPARTMENT|89|8|There temporary relations ought to avoid essentially bad brothers. Pret|monthly| +9513|AAAAAAAAJCFCAAAA|2452575|2452604|DEPARTMENT|89|9|National, human names will provide then traditional sons. Just happy show|monthly| +9514|AAAAAAAAKCFCAAAA|2452575|2452604|DEPARTMENT|89|10|New cases shall organise today close walls; japanese plans would see considerable, |monthly| +9515|AAAAAAAALCFCAAAA|2452575|2452604|DEPARTMENT|89|11|Identical, elderly beings hold. Short potatoes become environmental friends. Difficult ente|monthly| +9516|AAAAAAAAMCFCAAAA|2452575|2452604|DEPARTMENT|89|12|Minutes should live very busy occasions; just popular workshops should borrow. Neither diffe|monthly| +9517|AAAAAAAANCFCAAAA|2452575|2452604|DEPARTMENT|89|13|Old developers may not curb from a police. Raw laws discuss especia|monthly| +9518|AAAAAAAAOCFCAAAA|2452575|2452604|DEPARTMENT|89|14|Secret, main proceedings purchase other children. Musicians will know s|monthly| +9519|AAAAAAAAPCFCAAAA|2452575|2452604|DEPARTMENT|89|15|Policies should not cover then. Easier additional models will rep|monthly| +9520|AAAAAAAAADFCAAAA|2452575|2452604|DEPARTMENT|89|16|There western days used to make else available foreign classes. Therefore proper areas get|monthly| +9521|AAAAAAAABDFCAAAA|2452575|2452604|DEPARTMENT|89|17|Helpful conditions may prejudice often; pounds will not reply j|monthly| +9522|AAAAAAAACDFCAAAA|2452575|2452604|DEPARTMENT|89|18|Councils go jobs; namely awkward barriers owe now. Recent|monthly| +9523|AAAAAAAADDFCAAAA|2452575|2452604|DEPARTMENT|89|19|Social programmes might produce any longer. Standards will not like about environmental wings|monthly| +9524|AAAAAAAAEDFCAAAA|2452575|2452604|DEPARTMENT|89|20|Major chapters can encourage old, growing years; major eyes could co|monthly| +9525|AAAAAAAAFDFCAAAA|2452575|2452604|DEPARTMENT|89|21|Bodies ought to send busy students. Problems climb c|monthly| +9526|AAAAAAAAGDFCAAAA|2452575|2452604|DEPARTMENT|89|22|Very middle-class standards might act goals. Items should convey sometimes natural banks. Es|monthly| +9527|AAAAAAAAHDFCAAAA|2452575|2452604|DEPARTMENT|89|23|Houses should put tories; huge years can look much great, commo|monthly| +9528|AAAAAAAAIDFCAAAA|2452575|2452604|DEPARTMENT|89|24|Of course individual heroes drain together; workers used to |monthly| +9529|AAAAAAAAJDFCAAAA|2452575|2452604|DEPARTMENT|89|25|More remaining prices ought to get perhaps in a areas. Apart likely delegates must al|monthly| +9530|AAAAAAAAKDFCAAAA|2452575|2452604|DEPARTMENT|89|26|German movements must not define only, keen horses. Eligible, basic years |monthly| +9531|AAAAAAAALDFCAAAA|2452575|2452604|DEPARTMENT|89|27|Even obvious championships shall not leap always so political|monthly| +9532|AAAAAAAAMDFCAAAA|2452575|2452604|DEPARTMENT|89|28|Public birds receive soon european agencies. Strong, fine notions might not build par|monthly| +9533|AAAAAAAANDFCAAAA|2452575|2452604|DEPARTMENT|89|29|Neutral, easy years would like images. Other, regional areas know however; other, l|monthly| +9534|AAAAAAAAODFCAAAA|2452575|2452604|DEPARTMENT|89|30|Complete, able supporters may not call as on a forces. Black auth|monthly| +9535|AAAAAAAAPDFCAAAA|2452575|2452604|DEPARTMENT|89|31|Only premises hope even characteristics; regional things will not get wel|monthly| +9536|AAAAAAAAAEFCAAAA|||||||monthly| +9537|AAAAAAAABEFCAAAA|2452575|2452604|DEPARTMENT|89|33|Quiet, labour trends try issues. Seldom great kings shou|monthly| +9538|AAAAAAAACEFCAAAA|2452575|2452604|DEPARTMENT|89|34|Following discussions shall determine also. Pregnant|monthly| +9539|AAAAAAAADEFCAAAA|2452575|2452604|DEPARTMENT|89|35|Aloud valuable glasses hear better almost busy notions. Happy, welsh seats should stand of cour|monthly| +9540|AAAAAAAAEEFCAAAA|2452575|2452604|DEPARTMENT|89|36|Badly other plans observe therefore miles; shops ruin today secondary, effective|monthly| +9541|AAAAAAAAFEFCAAAA|2452575|2452604|DEPARTMENT|89|37|Significant, old schemes prevent immediately individuals. Rather sorry years ought to|monthly| +9542|AAAAAAAAGEFCAAAA|2452575|2452604|DEPARTMENT|89|38|Additional, spanish days refer small, financial shoulders. So organisational plates breed small, i|monthly| +9543|AAAAAAAAHEFCAAAA|2452575|2452604|DEPARTMENT|89|39|Christian, fresh trades identify thereafter overseas leads. Long links can slow. Also aware arrang|monthly| +9544|AAAAAAAAIEFCAAAA|2452575|2452604|DEPARTMENT|89|40|Never increased miles disappear demanding regulations. Major, fair effects migh|monthly| +9545|AAAAAAAAJEFCAAAA|2452575|2452604|DEPARTMENT|89|41|Certain, similar teachers would stay again operatio|monthly| +9546|AAAAAAAAKEFCAAAA|2452575|2452604|DEPARTMENT|89|42|Social, new processes should accept by a guests. Tonight pink pupil|monthly| +9547|AAAAAAAALEFCAAAA|2452575|2452604|DEPARTMENT|89|43|Books resist more recent clothes. Today medium banks serve on a levels; ambitious gove|monthly| +9548|AAAAAAAAMEFCAAAA|2452575|2452604|DEPARTMENT|89|44|Active, artificial observers talk quite. Rights used to describe increasingly therefore cen|monthly| +9549|AAAAAAAANEFCAAAA|2452575|2452604|DEPARTMENT|89|45|Yesterday only stairs should fit deep countries. Empty, italian eyes see to a winners.|monthly| +9550|AAAAAAAAOEFCAAAA|2452575|2452604|DEPARTMENT|89|46|Perhaps legitimate territories begin neither about a shadows. M|monthly| +9551|AAAAAAAAPEFCAAAA|2452575|2452604|||||| +9552|AAAAAAAAAFFCAAAA|2452575|2452604|DEPARTMENT|89|48|Terms ought to promote regularly trials. Strategic, true in|monthly| +9553|AAAAAAAABFFCAAAA|2452575|2452604|DEPARTMENT|89|49|Actually concerned months can call beautiful, major count|monthly| +9554|AAAAAAAACFFCAAAA|2452575|2452604|DEPARTMENT|89|50|Even so front countries take immediately. Wild, other teachers address as well on the ey|monthly| +9555|AAAAAAAADFFCAAAA|2452575|2452604|DEPARTMENT|89|51|Existing, misleading industries cut farmers. Whole officers help about afraid, full details. L|monthly| +9556|AAAAAAAAEFFCAAAA|2452575|2452604|DEPARTMENT|89|52|Lucky, legal feet say. Weakly accused soldiers can fetch to a t|monthly| +9557|AAAAAAAAFFFCAAAA|2452575|2452604|DEPARTMENT|89|53|High days ought to involve informally wild losses. Af|monthly| +9558|AAAAAAAAGFFCAAAA|2452575|2452604|DEPARTMENT|89|54|Tory, whole consumers used to meet in a figures. Conservative assets|monthly| +9559|AAAAAAAAHFFCAAAA|2452575|2452604|DEPARTMENT|89|55|Missiles could not live accurately general, intelligent fir|monthly| +9560|AAAAAAAAIFFCAAAA|2452575|2452604|DEPARTMENT|89|56|Encouraging affairs help rapidly articles. Black tra|monthly| +9561|AAAAAAAAJFFCAAAA|2452575|2452604|DEPARTMENT|89|57|Necessary, key authorities may set often russian effects. Standard, separate governors kno|monthly| +9562|AAAAAAAAKFFCAAAA|2452575|2452604|DEPARTMENT|89|58|Various crews set appropriate, subject firms. Faint|monthly| +9563|AAAAAAAALFFCAAAA|2452575|2452604|DEPARTMENT|89|59|About dead losses used to give almost indian likely hours. Usual heels persuade pa|monthly| +9564|AAAAAAAAMFFCAAAA|2452575|2452604|DEPARTMENT|89|60|Operational, cold models shall include. All interesting principles authori|monthly| +9565|AAAAAAAANFFCAAAA|2452575|2452604|DEPARTMENT|89|61|Political tonnes make apparently white times. Straight pale thousands sh|monthly| +9566|AAAAAAAAOFFCAAAA|2452575|2452604|DEPARTMENT|89|62|Tests shall not think more about severe sales; diplomatic lawyers establish judicial, sensi|monthly| +9567|AAAAAAAAPFFCAAAA|2452575|2452604|DEPARTMENT|89|63|Other, beautiful things could question quite thanks. Tradi|monthly| +9568|AAAAAAAAAGFCAAAA|2452575|2452604|DEPARTMENT|89|64|Other decisions consider gently new names. Forces will not look as to a meetings; dependent, great|monthly| +9569|AAAAAAAABGFCAAAA|2452575|2452604|DEPARTMENT|89|65|Farms would look simply papers. Rights may bring fi|monthly| +9570|AAAAAAAACGFCAAAA|2452575|2452604|DEPARTMENT|89|66|Areas resign perhaps. Other farmers could enhance. Items look exp|monthly| +9571|AAAAAAAADGFCAAAA|2452575|2452604|DEPARTMENT|89|67|Possibly possible miles end great, financial services|monthly| +9572|AAAAAAAAEGFCAAAA|2452575|2452604|DEPARTMENT|89|68|Parents make royal, applicable years. Small members fall al|monthly| +9573|AAAAAAAAFGFCAAAA|2452575|2452604|DEPARTMENT|89|69|Personally equal weeks shall not dream fingers; important, social duties avoid at once |monthly| +9574|AAAAAAAAGGFCAAAA|2452575|2452604|DEPARTMENT|89|70|Days pay sometimes main females. Women make well good pensions. Deta|monthly| +9575|AAAAAAAAHGFCAAAA|2452575|2452604|DEPARTMENT|89|71|Probably particular implications must seek even sexual equ|monthly| +9576|AAAAAAAAIGFCAAAA|2452575|2452604|DEPARTMENT|89|72|Today public papers must play days. Real, handsome others will not t|monthly| +9577|AAAAAAAAJGFCAAAA|2452575|2452604|DEPARTMENT|89|73|Tests write enormously. Incentives cost widely top channels. Logical rats say somehow majo|monthly| +9578|AAAAAAAAKGFCAAAA|2452575|2452604|DEPARTMENT|89|74|Correct, new boots come neatly for the differences. Arm|monthly| +9579|AAAAAAAALGFCAAAA|2452575|2452604|DEPARTMENT|89|75|Social processes must not burst then large, fresh customers. Frequently natural videos used |monthly| +9580|AAAAAAAAMGFCAAAA|2452575|2452604|DEPARTMENT|89|76|Managers meet just independent men. Practically easy suppliers should concentrate never.|monthly| +9581|AAAAAAAANGFCAAAA|2452575|2452604|DEPARTMENT|89|77|Only local techniques know single papers. Little, other police examine hardly. |monthly| +9582|AAAAAAAAOGFCAAAA|2452575|2452604|DEPARTMENT|89|78|Years may fall over. Red workers steal independent, good years. Other police sh|monthly| +9583|AAAAAAAAPGFCAAAA|2452575|2452604|DEPARTMENT|89|79|Clear, white months should not know friends. Records se|monthly| +9584|AAAAAAAAAHFCAAAA|2452575|2452604|DEPARTMENT|89|80|Fixed patients shall look areas. Then other relations talk far|monthly| +9585|AAAAAAAABHFCAAAA|2452575|2452604|DEPARTMENT|89|81|Finally dry fields say perhaps anyway continued farme|monthly| +9586|AAAAAAAACHFCAAAA|2452575|2452604|DEPARTMENT|89|82|All right initial stories save either in a defendant|monthly| +9587|AAAAAAAADHFCAAAA|2452575|2452604|DEPARTMENT|89|83|Associated cheeks win. Political volunteers could not take enough. Economic, soci|monthly| +9588|AAAAAAAAEHFCAAAA|2452575|2452604|DEPARTMENT|89|84|Natural, social rivers must near. Social pupils use tomorrow pages. Kind, hard tensions can want|monthly| +9589|AAAAAAAAFHFCAAAA|2452575|2452604|DEPARTMENT|89|85|Old standards become limited walls. Natural rates will not understa|monthly| +9590|AAAAAAAAGHFCAAAA|2452575|2452604|DEPARTMENT|89|86|Rare conditions punch modern, written police. Closed, sufficient months let af|monthly| +9591|AAAAAAAAHHFCAAAA|2452575|2452604|DEPARTMENT|89|87|Both blind police catch late able tests. Tired, strong cells ho|monthly| +9592|AAAAAAAAIHFCAAAA|2452575|2452604|DEPARTMENT|89|88|French, foreign routes hear slightly small lawyers. Private, professional speeches live neither|monthly| +9593|AAAAAAAAJHFCAAAA|2452575|2452604|DEPARTMENT|89|89|Very interested balls used to fulfil mental, active proposals. Conditions ought to c|monthly| +9594|AAAAAAAAKHFCAAAA|2452575|2452604|DEPARTMENT|89|90|Prime, good areas take. Difficult reasons visit successful si|monthly| +9595|AAAAAAAALHFCAAAA|2452575|2452604|DEPARTMENT|89|91|Keen, private teeth cause here different directors; great groups should tea|monthly| +9596|AAAAAAAAMHFCAAAA|2452575|2452604|DEPARTMENT|89|92|Qualities cause students. Payable premises handle western, joint sales; local, various pa|monthly| +9597|AAAAAAAANHFCAAAA|2452575|2452604|DEPARTMENT|89|93|Smart periods may ensure obviously. Impressive, continuous points see |monthly| +9598|AAAAAAAAOHFCAAAA|2452575|2452604|DEPARTMENT|89|94|Significant conditions cause visitors. Easy, different children look leading dreams. |monthly| +9599|AAAAAAAAPHFCAAAA|2452575|2452604|DEPARTMENT|89|95|A bit right numbers think. Satisfactory matters fi|monthly| +9600|AAAAAAAAAIFCAAAA|2452575|2452604|DEPARTMENT|89|96|Political supporters should come. Never right heels shall not contemplate days. Pur|monthly| +9601|AAAAAAAABIFCAAAA|2452575|2452604|DEPARTMENT|89|97|National conclusions learn here. Afraid causes must not leave into a troops. M|monthly| +9602|AAAAAAAACIFCAAAA|2452575|2452604|DEPARTMENT|89|98|Circumstances pay beforehand only other individuals. Bu|monthly| +9603|AAAAAAAADIFCAAAA|2452575|2452604|DEPARTMENT|89|99|Acceptable, inc notes would know also new, soviet |monthly| +9604|AAAAAAAAEIFCAAAA|2452575|2452604|DEPARTMENT|89|100|Wrong formidable interactions might save so suspicious useful cases; financial types occur so o|monthly| +9605|AAAAAAAAFIFCAAAA|2452575|2452604|DEPARTMENT|89|101|Vigorously international sentences deal only kids. In particular usual othe|monthly| +9606|AAAAAAAAGIFCAAAA|2452575|2452604|DEPARTMENT|89|102|Thousands may bother for instance dreams. Much written pr|monthly| +9607|AAAAAAAAHIFCAAAA|2452575|2452604|DEPARTMENT|89|103|High main runs explain economic, sorry trees. Warm, relevant sales consolidate then inte|monthly| +9608|AAAAAAAAIIFCAAAA|2452575|2452604|DEPARTMENT|89|104|Words approve for a resources. Kids appear for a y|monthly| +9609|AAAAAAAAJIFCAAAA|2452575|2452604|DEPARTMENT|89|105|Rough, stupid children point representatives. Friends may take at the activities.|monthly| +9610|AAAAAAAAKIFCAAAA|2452575|2452604|DEPARTMENT|89|106|Protective, unlike woods will move then documents. Super, h|monthly| +9611|AAAAAAAALIFCAAAA|2452575|2452604|DEPARTMENT|89|107|Classes could pass; depths protect members. Conflicts turn elsewhere. Strong facilities |monthly| +9612|AAAAAAAAMIFCAAAA|2452575|2452604|DEPARTMENT|89|108|Certain times take restrictions; difficulties must not increase so findings|monthly| +9613|AAAAAAAANIFCAAAA|2452605|2452634|DEPARTMENT|90|1|Apparently fair minerals must take then simple shops. Closely useful|monthly| +9614|AAAAAAAAOIFCAAAA|2452605|2452634|DEPARTMENT|90|2|Regular, efficient videos change sometimes current workers. Just formal mountains build greatly |monthly| +9615|AAAAAAAAPIFCAAAA||2452634|DEPARTMENT||3|Questions assume inevitably bad interpretations; comprehens|monthly| +9616|AAAAAAAAAJFCAAAA|2452605|2452634|DEPARTMENT|90|4|Major, little pictures ought to see consistently. Sa|monthly| +9617|AAAAAAAABJFCAAAA|2452605|2452634|DEPARTMENT|90|5|Savings assess opposite, available creatures. Types hear apart open children. Final factors dr|monthly| +9618|AAAAAAAACJFCAAAA|2452605|2452634|DEPARTMENT|90|6|Complex, medical meetings cannot go massive, light newspapers. So occasional manage|monthly| +9619|AAAAAAAADJFCAAAA|2452605|2452634|DEPARTMENT|90|7|Local, technological shares could wait tonight. Normally violent items ought|monthly| +9620|AAAAAAAAEJFCAAAA|2452605|2452634|DEPARTMENT|90|8|Cold potatoes continue weeks. Top, dear friends must seek more autho|monthly| +9621|AAAAAAAAFJFCAAAA|2452605|2452634|DEPARTMENT|90|9|Loans know then men. Automatic months maintain so ugly bodies. Still other |monthly| +9622|AAAAAAAAGJFCAAAA|2452605|2452634|DEPARTMENT|90|10|Friends join still. Bare substances might smell animals. B|monthly| +9623|AAAAAAAAHJFCAAAA|2452605|2452634|DEPARTMENT|90|11|Easier immediate relatives might run purely outside financial efforts; previous gains|monthly| +9624|AAAAAAAAIJFCAAAA|2452605|2452634|DEPARTMENT|90|12|New buildings may represent still always good attacks. Regional eyes ought to burn old, nast|monthly| +9625|AAAAAAAAJJFCAAAA|2452605|2452634|DEPARTMENT|90|13|Deep things drop almost angry plants. Late, bad examinations ought to reflect sta|monthly| +9626|AAAAAAAAKJFCAAAA|2452605|2452634|DEPARTMENT|90|14|Old, immense doors might explore like the sentences. Fu|monthly| +9627|AAAAAAAALJFCAAAA|2452605|2452634|DEPARTMENT|90|15|Jobs ensure. Single, soviet needs shall remember corporate reade|monthly| +9628|AAAAAAAAMJFCAAAA|2452605|2452634|DEPARTMENT|90|16|Effectively socialist children carry never large thanks. Hard suppliers might not wr|monthly| +9629|AAAAAAAANJFCAAAA|2452605|2452634|DEPARTMENT|90|17|Darling, total contents assure. Swiss, different hundred|monthly| +9630|AAAAAAAAOJFCAAAA|2452605|2452634|DEPARTMENT|90|18|Political products approach however unfairly young members. Oft|monthly| +9631|AAAAAAAAPJFCAAAA|2452605|2452634|DEPARTMENT|90|19|Parts must not pick direct, social hours; exact union|monthly| +9632|AAAAAAAAAKFCAAAA|2452605|2452634|DEPARTMENT|90|20|Best different men exist differences. Both traditional accounts protect only. Institutions ge|monthly| +9633|AAAAAAAABKFCAAAA|2452605|2452634|DEPARTMENT|90|21|Sorry, technical professionals telephone from a schools; subsequent le|monthly| +9634|AAAAAAAACKFCAAAA|2452605|2452634|DEPARTMENT|90|22|At all internal names get most with a types. At once international |monthly| +9635|AAAAAAAADKFCAAAA|2452605|2452634|DEPARTMENT|90|23|Innocent, unable shareholders expect cameras. Quite special tests would not v|monthly| +9636|AAAAAAAAEKFCAAAA|2452605|2452634|DEPARTMENT|90|24|Correspondingly special shares ought to see actually; grey|monthly| +9637|AAAAAAAAFKFCAAAA|2452605|2452634|DEPARTMENT|90|25|Unable, revolutionary leaders play since home other p|monthly| +9638|AAAAAAAAGKFCAAAA|2452605|2452634|DEPARTMENT|90|26|Important, new hotels must not tell later forward possible talks; enough terrible humans sit best|monthly| +9639|AAAAAAAAHKFCAAAA|2452605|2452634|DEPARTMENT|90|27|Pure, happy legs can work increasingly british, powerfu|monthly| +9640|AAAAAAAAIKFCAAAA|2452605|2452634|DEPARTMENT|90|28|Right, slow countries discharge very evident cities. Successfu|monthly| +9641|AAAAAAAAJKFCAAAA|2452605|2452634|DEPARTMENT|90|29|Buildings control more. Hard wages look considerable actions. Keys used to fit better powers.|monthly| +9642|AAAAAAAAKKFCAAAA|2452605|2452634|DEPARTMENT|90|30|Flowers afford. Designs used to pick both doors. Ne|monthly| +9643|AAAAAAAALKFCAAAA|2452605|2452634|DEPARTMENT|90|31|Enterprises can marry. Often broad hearts give such as a conditions|monthly| +9644|AAAAAAAAMKFCAAAA|2452605|2452634|DEPARTMENT|90|32|Light, reliable looks ought to see. Formidable, islamic sy|monthly| +9645|AAAAAAAANKFCAAAA|2452605|2452634|DEPARTMENT|90|33|Firms shall abandon most; full subjects go sweet, similar things. Intense |monthly| +9646|AAAAAAAAOKFCAAAA|2452605|2452634|DEPARTMENT|90|34|Only common products used to use programmes; important, only researchers may earn then|monthly| +9647|AAAAAAAAPKFCAAAA|2452605|2452634|DEPARTMENT|90|35|Groups mind aloud out of a guidelines. Woods meet often; new kids shop more. Workers increase |monthly| +9648|AAAAAAAAALFCAAAA|2452605|2452634|DEPARTMENT|90|36|Programmes make also damp, blue relationships. Therefore suitable universities wil|monthly| +9649|AAAAAAAABLFCAAAA|2452605|2452634|DEPARTMENT|90|37|Final, written rates may talk twice with a stories. In order interesting cuts might r|monthly| +9650|AAAAAAAACLFCAAAA|2452605|2452634|DEPARTMENT|90|38|Annual lines may not pay. Obviously immediate assumptions abando|monthly| +9651|AAAAAAAADLFCAAAA|2452605|2452634|DEPARTMENT|90|39|Experts ignore somewhat to a women. Immense, new terms fight apparently top, particular |monthly| +9652|AAAAAAAAELFCAAAA|2452605|2452634|DEPARTMENT|90|40|Big, nervous goods offer upon a goods. Over there proper s|monthly| +9653|AAAAAAAAFLFCAAAA|2452605|2452634|DEPARTMENT|90|41|Large deaths answer equally national buildings. Acts would not suffer|monthly| +9654|AAAAAAAAGLFCAAAA|2452605|2452634|DEPARTMENT|90|42|Small homes would see positive problems. Respective, scientific groups see la|monthly| +9655|AAAAAAAAHLFCAAAA|2452605|2452634|DEPARTMENT|90|43|Decades could not increase heads. Groups envy plain now standard models. Small, united points could|monthly| +9656|AAAAAAAAILFCAAAA|2452605|2452634|DEPARTMENT|90|44|Just lengthy orders allow therefore largely technological decisions. |monthly| +9657|AAAAAAAAJLFCAAAA|2452605|2452634|DEPARTMENT|90|45|Light, common lawyers prove. Then broken hands fall moreover obvious seats. Somewhat |monthly| +9658|AAAAAAAAKLFCAAAA|2452605|2452634|DEPARTMENT|90|46|Due, old consequences react somewhere sure services. Original bodies stop about|monthly| +9659|AAAAAAAALLFCAAAA|2452605|2452634|DEPARTMENT|90|47|Only crucial ways must suffer valuable, huge months; circumstances fight only on a names. |monthly| +9660|AAAAAAAAMLFCAAAA|2452605|2452634|DEPARTMENT|90|48|Also large mines attract close, vital members; permanent influences see only mental eyes. |monthly| +9661|AAAAAAAANLFCAAAA|2452605|2452634|DEPARTMENT|90|49|Eventually domestic children will tell quite years. Poli|monthly| +9662|AAAAAAAAOLFCAAAA|2452605|2452634|DEPARTMENT|90|50|Advanced cases communicate. Large, free colleagues take both military courses. Level positions |monthly| +9663|AAAAAAAAPLFCAAAA|2452605|2452634|DEPARTMENT|90|51|Positive, international circumstances should see more waste miles; empty effects see with the hou|monthly| +9664|AAAAAAAAAMFCAAAA|2452605|2452634|DEPARTMENT|90|52|Findings could not find. Intentions take actions; sin|monthly| +9665|AAAAAAAABMFCAAAA|2452605|2452634|DEPARTMENT|90|53|Additional, labour shareholders will return anyway forms; numerous, s|monthly| +9666|AAAAAAAACMFCAAAA|2452605|2452634|DEPARTMENT|90|54|Subsequent, keen offices ought to organise especially moments. Earlier full men get by no means fur|monthly| +9667|AAAAAAAADMFCAAAA|2452605|2452634|DEPARTMENT|90|55|Figures will find stages. Ways shall make still. Claims think. Classes stop the|monthly| +9668|AAAAAAAAEMFCAAAA|2452605|2452634|DEPARTMENT|90|56|Left systems adopt by no means new changes; clothes shall not say streets. Subsequent, large he|monthly| +9669|AAAAAAAAFMFCAAAA|2452605|2452634|DEPARTMENT|90|57|Full values run new, black states. Successful, fatal products |monthly| +9670|AAAAAAAAGMFCAAAA|2452605|2452634|DEPARTMENT|90|58|Fingers reassess then possible parents. Neutral, famous powers take about a profits. |monthly| +9671|AAAAAAAAHMFCAAAA|2452605|2452634|DEPARTMENT|90|59|Rows would attract under opposite a husbands. Approximately foreign l|monthly| +9672|AAAAAAAAIMFCAAAA|2452605|2452634|DEPARTMENT|90|60|Statements want. Total accountants find years; existing, elec|monthly| +9673|AAAAAAAAJMFCAAAA|2452605|2452634|DEPARTMENT|90|61|English men can touch regular, skilled games. Publ|monthly| +9674|AAAAAAAAKMFCAAAA|2452605|2452634|DEPARTMENT|90|62|Differences rest also together statutory objects. Soviet services overthrow |monthly| +9675|AAAAAAAALMFCAAAA|2452605|2452634|DEPARTMENT|90|63|Free methods wait future services. Together sensible birds shall learn;|monthly| +9676|AAAAAAAAMMFCAAAA|2452605|2452634|DEPARTMENT|90|64|Blank arms used to discuss tight certain, european rig|monthly| +9677|AAAAAAAANMFCAAAA|2452605|2452634|DEPARTMENT|90|65|Religious, specific edges develop better heavy projects. Circumstances might not hope ar|monthly| +9678|AAAAAAAAOMFCAAAA|2452605|2452634|DEPARTMENT|90|66|Jobs used to cope here. Scarcely old occasions understand eyebrows; present, economic|monthly| +9679|AAAAAAAAPMFCAAAA|2452605|2452634|DEPARTMENT|90|67|Steps take especially by a relatives. Rights will tell. Actors used t|monthly| +9680|AAAAAAAAANFCAAAA|2452605|2452634|DEPARTMENT|90|68|Faces form sites. Implications want also often different institu|monthly| +9681|AAAAAAAABNFCAAAA|2452605|2452634|DEPARTMENT|90|69|Countries get very consequently organic texts. Furious results incur conservative tanks. W|monthly| +9682|AAAAAAAACNFCAAAA|2452605|2452634|DEPARTMENT|90|70|Hours may finish primarily difficult features. High thoro|monthly| +9683|AAAAAAAADNFCAAAA|2452605|2452634|DEPARTMENT|90|71|Now due programmes shall analyse also other, active functions. Programm|monthly| +9684|AAAAAAAAENFCAAAA|2452605|2452634|DEPARTMENT|90|72|Adjacent, other events ought to live welsh, public directions. M|monthly| +9685|AAAAAAAAFNFCAAAA|2452605|2452634|DEPARTMENT|90|73|Small, mixed resources may not select little years. Blue, new claims pass for ins|monthly| +9686|AAAAAAAAGNFCAAAA|2452605|2452634|DEPARTMENT|90|74|Provisions may not say. Great charges sell. Future, short engineers marry now clinical, only stairs|monthly| +9687|AAAAAAAAHNFCAAAA|2452605|2452634|DEPARTMENT|90|75|Accused, little principles hit. Structures give clearly military hotels. Neith|monthly| +9688|AAAAAAAAINFCAAAA|2452605|2452634|DEPARTMENT|90|76|Frequent, good months ought to reduce external, greek councils. Problems detect purposes|monthly| +9689|AAAAAAAAJNFCAAAA|2452605|2452634|DEPARTMENT|90|77|Types may make. Almost political guns make upwards|monthly| +9690|AAAAAAAAKNFCAAAA|2452605|2452634|DEPARTMENT|90|78|Financial, comfortable ears keep tomorrow in a bodies. Countries will|monthly| +9691|AAAAAAAALNFCAAAA|2452605|2452634|DEPARTMENT|90|79|Options come fully. Capable, conventional sides keep in a ideas. At present profe|monthly| +9692|AAAAAAAAMNFCAAAA|2452605|2452634|DEPARTMENT|90|80|Eyes ought to find women. Around additional children see as a points. Firms ought to say in a d|monthly| +9693|AAAAAAAANNFCAAAA|2452605|2452634|DEPARTMENT|90|81|Experimental, good ways should find well as economic plans. Mo|monthly| +9694|AAAAAAAAONFCAAAA|2452605|2452634|DEPARTMENT|90|82|Sometimes jewish processes find actually christian issues. Considerable cells embr|monthly| +9695|AAAAAAAAPNFCAAAA|2452605|2452634|DEPARTMENT|90|83|Still local leads distribute more low authors; patients feel across |monthly| +9696|AAAAAAAAAOFCAAAA|2452605|2452634|DEPARTMENT|90|84|Modest, shallow refugees know over minimum policemen. Major, new results mov|monthly| +9697|AAAAAAAABOFCAAAA|2452605|2452634|DEPARTMENT|90|85|Particular events call perhaps then cheap years. Currently eastern negotiations can beli|monthly| +9698|AAAAAAAACOFCAAAA|2452605|2452634|DEPARTMENT|90|86|Powers might not find small, new sciences. French, solar years|monthly| +9699|AAAAAAAADOFCAAAA|2452605|2452634|DEPARTMENT|90|87|Parliamentary authorities go working teeth. Then a|monthly| +9700|AAAAAAAAEOFCAAAA|2452605|2452634|DEPARTMENT|90|88|Degrees get in a clients. Partly considerable bands may restrict le|monthly| +9701|AAAAAAAAFOFCAAAA|2452605|2452634|DEPARTMENT|90|89|Single questions should want then large, great students. Military, hard infl|monthly| +9702|AAAAAAAAGOFCAAAA|2452605|2452634|DEPARTMENT|90|90|Old parents shall not keep yesterday; obvious cities obey very g|monthly| +9703|AAAAAAAAHOFCAAAA|2452605|2452634|DEPARTMENT|90|91|Often senior sales should reduce ever public recommendations. Complete character|monthly| +9704|AAAAAAAAIOFCAAAA|2452605|2452634|DEPARTMENT|90|92|Troops will encourage all jobs. International, various pictures al|monthly| +9705|AAAAAAAAJOFCAAAA|2452605|2452634|DEPARTMENT|90|93|Already likely councillors might not bind very especially |monthly| +9706|AAAAAAAAKOFCAAAA|2452605|2452634|DEPARTMENT|90|94|At once capable groups move pages. Both dangerous things express. Significant co|monthly| +9707|AAAAAAAALOFCAAAA|2452605|2452634|DEPARTMENT|90|95|Correctly new aspects fit safe, bright moments; positively social women|monthly| +9708|AAAAAAAAMOFCAAAA|2452605|2452634|DEPARTMENT|90|96|Wide opportunities understand; controversial, political arms would ma|monthly| +9709|AAAAAAAANOFCAAAA|2452605|2452634|DEPARTMENT|90|97|Aspects must illustrate for no committees. Years will|monthly| +9710|AAAAAAAAOOFCAAAA|2452605|2452634|DEPARTMENT|90|98|Original, vast bits cannot help frantically high parts. Special, subject s|monthly| +9711|AAAAAAAAPOFCAAAA|2452605|2452634|DEPARTMENT|90|99|Inc kinds assess most powers; tracks see needs. Au|monthly| +9712|AAAAAAAAAPFCAAAA|2452605|2452634|DEPARTMENT|90|100|British kids cope areas. Parties clear through the settings. United levels will not inc|monthly| +9713|AAAAAAAABPFCAAAA|2452605|2452634|DEPARTMENT|90|101|Excellent, new mines would reflect various, british w|monthly| +9714|AAAAAAAACPFCAAAA|2452605|2452634|DEPARTMENT|90|102|Stairs used to take environmental, royal laws. Silver, compe|monthly| +9715|AAAAAAAADPFCAAAA|2452605|2452634|DEPARTMENT|90|103|Patients assert for example hastily outer shoulders. Marginal appo|monthly| +9716|AAAAAAAAEPFCAAAA|2452605|2452634|DEPARTMENT|90|104|Vague sources move in a novels. Pure increases snatch other, |monthly| +9717|AAAAAAAAFPFCAAAA|2452605|2452634|DEPARTMENT|90|105|New, royal animals put in a programmes. Difficult proceedings could not |monthly| +9718|AAAAAAAAGPFCAAAA|2452605|2452634|DEPARTMENT|90|106|Both short followers shall not stop from a prisoners. Cultural, |monthly| +9719|AAAAAAAAHPFCAAAA|2452605|2452634|DEPARTMENT|90|107|Then social pressures threaten. Present details should not say secondly central parents; sho|monthly| +9720|AAAAAAAAIPFCAAAA|2452605|2452634|DEPARTMENT|90|108|Already specialist women could not avoid weekly, regional firms. Particular, everyda|monthly| +9721|AAAAAAAAJPFCAAAA|2452640|2452821|DEPARTMENT|91|1|Constant miles will hit continuously. Afraid times will pull linguistic, widespread millio|bi-annual| +9722|AAAAAAAAKPFCAAAA|2452640|2452821|DEPARTMENT|91|2|Simply national copies convince already recent bits. Fur|bi-annual| +9723|AAAAAAAALPFCAAAA|2452640|2452821|DEPARTMENT|91|3|Ill possible years prevent too awards. Old hours shall not benefit then modern police. Hours sh|bi-annual| +9724|AAAAAAAAMPFCAAAA|2452640|2452821|DEPARTMENT|91|4|Circumstances emphasize only. Governments cannot pay ter|bi-annual| +9725|AAAAAAAANPFCAAAA|2452640|2452821|DEPARTMENT|91|5|Complete, long-term customers would persuade across other years. Clean posts please apart on a f|bi-annual| +9726|AAAAAAAAOPFCAAAA|2452640|2452821|DEPARTMENT|91|6|Necessary others must catch sincerely probably national answers. Services sound ri|bi-annual| +9727|AAAAAAAAPPFCAAAA|2452640|2452821|DEPARTMENT|91|7|At least familiar managers embody hours. Together beautiful patients could not carry me|bi-annual| +9728|AAAAAAAAAAGCAAAA|2452640|2452821|DEPARTMENT|91|8|Commercial centres used to return. Elsewhere economic others m|bi-annual| +9729|AAAAAAAABAGCAAAA|2452640|2452821|DEPARTMENT|91|9|That possible items climb pretty on a developers. In |bi-annual| +9730|AAAAAAAACAGCAAAA|2452640|2452821|DEPARTMENT|91|10|Fashionable walls must not tell forward ugly homes; services will make over. Just us|bi-annual| +9731|AAAAAAAADAGCAAAA|2452640|2452821|DEPARTMENT|91|11|Particular examples apply far casual numbers. Homes should |bi-annual| +9732|AAAAAAAAEAGCAAAA|2452640|2452821|DEPARTMENT|91|12|Products get wonderful days. Very reduced arms prevent easily rus|bi-annual| +9733|AAAAAAAAFAGCAAAA|2452640|2452821|DEPARTMENT|91|13|Open, bright flowers know. Heavy parts consider fiscal companies. Rough, adeq|bi-annual| +9734|AAAAAAAAGAGCAAAA|2452640|2452821|DEPARTMENT|91|14|Dreams find possibly wet families. Groups will provi|bi-annual| +9735|AAAAAAAAHAGCAAAA|2452640|2452821|DEPARTMENT|91|15|Gently late needs raise simply fully small words. Never following s|bi-annual| +9736|AAAAAAAAIAGCAAAA|2452640|2452821|DEPARTMENT|91|16|Other, certain areas must not believe always tests. Apparent pp. m|bi-annual| +9737|AAAAAAAAJAGCAAAA|2452640|2452821|DEPARTMENT|91|17|Bad weeks purchase. Terms uncover hardly. General systems used to shrug quite old institutions; |bi-annual| +9738|AAAAAAAAKAGCAAAA|2452640|2452821|DEPARTMENT|91|18|Appointments go by the sectors. Much african gentlemen deter|bi-annual| +9739|AAAAAAAALAGCAAAA|2452640|2452821|DEPARTMENT|91|19|New, dependent words may not know overall, nearby costs; ideas get only without a|bi-annual| +9740|AAAAAAAAMAGCAAAA|2452640|2452821|DEPARTMENT|91|20|Papers deserve quick also literary children. Yesterday o|bi-annual| +9741|AAAAAAAANAGCAAAA|2452640|2452821|DEPARTMENT|91|21|Hot systems may regard however correct forms. Consistently comprehensive times construct librar|bi-annual| +9742|AAAAAAAAOAGCAAAA|2452640|2452821|DEPARTMENT||22||| +9743|AAAAAAAAPAGCAAAA|2452640|2452821|DEPARTMENT|91|23|Quick, environmental laboratories used to put really whole programmes. Variables select |bi-annual| +9744|AAAAAAAAABGCAAAA|2452640|2452821|DEPARTMENT|91|24|Modern members would set wide governments. Loyal windows could no|bi-annual| +9745|AAAAAAAABBGCAAAA|2452640|2452821|DEPARTMENT|91|25|Practitioners produce more for a roles. Final, continued claims must t|bi-annual| +9746|AAAAAAAACBGCAAAA|2452640|2452821|DEPARTMENT|91|26|Aware rooms will not like carers. Video-taped, agricultural instruments shall overcome ever work|bi-annual| +9747|AAAAAAAADBGCAAAA|2452640|2452821|DEPARTMENT|91|27|Terms will not integrate various, young attitudes. Peculiar, labour righ|bi-annual| +9748|AAAAAAAAEBGCAAAA|2452640|2452821|DEPARTMENT|91|28|Pensioners come similarly fast pleasant goals. Only specified centres re|bi-annual| +9749|AAAAAAAAFBGCAAAA|2452640|2452821|DEPARTMENT|91|29|Old, diplomatic costs may not choose too. Programmes might |bi-annual| +9750|AAAAAAAAGBGCAAAA|2452640|2452821|DEPARTMENT|91|30|Streets load this also entire questions. Major, universal notes should pa|bi-annual| +9751|AAAAAAAAHBGCAAAA|2452640|2452821|DEPARTMENT|91|31|Increasing, quiet lips would not get rapidly except for the panels. Democr|bi-annual| +9752|AAAAAAAAIBGCAAAA|2452640|2452821|DEPARTMENT|91|32|Only cultural feet cut ago at the teams; institutions may fin|bi-annual| +9753|AAAAAAAAJBGCAAAA|2452640|2452821|DEPARTMENT|91|33|Only basic events must move; problems will invoke possible numbers. Strange, just chil|bi-annual| +9754|AAAAAAAAKBGCAAAA|2452640|2452821|DEPARTMENT|91|34|More significant eyes support early frequent, common victims. Sharp attitudes look more for a |bi-annual| +9755|AAAAAAAALBGCAAAA|2452640|2452821|DEPARTMENT|91|35|As current groups follow only classes. Great, free patterns must mix enormously organic fo|bi-annual| +9756|AAAAAAAAMBGCAAAA|2452640|2452821|DEPARTMENT|91|36|Simple nations get then. Fans catch only generous, private days. Never old flights expre|bi-annual| +9757|AAAAAAAANBGCAAAA|2452640|2452821|DEPARTMENT|91|37|Times must assume. Small arrangements would not go almost agencie|bi-annual| +9758|AAAAAAAAOBGCAAAA|2452640|2452821|DEPARTMENT|91|38|Subsequently shy teeth should obtain only adult, holy standards; other, foreign |bi-annual| +9759|AAAAAAAAPBGCAAAA|2452640|2452821|DEPARTMENT|91|39|Houses would buy. Particular, nuclear weeks look as in a developme|bi-annual| +9760|AAAAAAAAACGCAAAA|2452640|2452821|DEPARTMENT|91|40|Evident, statutory details expect special tickets. Now established leaves settle more present schoo|bi-annual| +9761|AAAAAAAABCGCAAAA|2452640|2452821|DEPARTMENT|91|41|Very possible categories may not take exact, useful tourists; just brief mistakes could ru|bi-annual| +9762|AAAAAAAACCGCAAAA|2452640|2452821|DEPARTMENT|91|42|Still wide departments turn quite sophisticated bonds. Public, anxious developments might come oth|bi-annual| +9763|AAAAAAAADCGCAAAA|2452640|2452821|DEPARTMENT|91|43|Funds tax in a questions; actors might find usually. Organic lives make certainly exact opportuniti|bi-annual| +9764|AAAAAAAAECGCAAAA|2452640|2452821|DEPARTMENT|91|44|Still unpleasant sentences must study comments. Representatives s|bi-annual| +9765|AAAAAAAAFCGCAAAA|2452640|2452821|DEPARTMENT|91|45|French, rapid facilities shall wear again in a motives. Especially academic inches ensure even on|bi-annual| +9766|AAAAAAAAGCGCAAAA|2452640|2452821|DEPARTMENT|91|46|Chief, silly hands can exert public relations. Units rebuild likely, reasona|bi-annual| +9767|AAAAAAAAHCGCAAAA|2452640|2452821|DEPARTMENT|91|47|Quick, following sales amuse social, effective detail|bi-annual| +9768|AAAAAAAAICGCAAAA|2452640|2452821|DEPARTMENT|91|48|Mere schools provide now late ordinary effects. Initial officers lead even psychological r|bi-annual| +9769|AAAAAAAAJCGCAAAA|2452640|2452821|DEPARTMENT|91|49|Alone decisions encourage with a sales. Critical, common standard|bi-annual| +9770|AAAAAAAAKCGCAAAA|2452640|2452821|DEPARTMENT|91|50|Police may not oust ideas. About psychological yea|bi-annual| +9771|AAAAAAAALCGCAAAA|2452640|2452821|DEPARTMENT|91|51|Early, sorry games prefer with a purposes. Sound, central guests ought to pay eyes. Still p|bi-annual| +9772|AAAAAAAAMCGCAAAA||2452821|DEPARTMENT|||Recent offices may provide namely windows. Problems block notes. Also light results see|bi-annual| +9773|AAAAAAAANCGCAAAA|2452640|2452821|DEPARTMENT|91|53|Changes join. Others come new, natural others. Thick, large police believe also. Social, english |bi-annual| +9774|AAAAAAAAOCGCAAAA|2452640|2452821|DEPARTMENT|91|54|Outer chairs fulfil. Usually useless students used to g|bi-annual| +9775|AAAAAAAAPCGCAAAA|2452640|2452821|DEPARTMENT|91|55|Specific systems may leap surprisingly obviously social days. Sout|bi-annual| +9776|AAAAAAAAADGCAAAA|2452640|2452821|DEPARTMENT|91|56|Temporarily rural stages start ill tables; essential fri|bi-annual| +9777|AAAAAAAABDGCAAAA|2452640|2452821|DEPARTMENT|91|57|Happy names may separate currently really short trees. All blind answers would|bi-annual| +9778|AAAAAAAACDGCAAAA|2452640|2452821|DEPARTMENT|91|58|Children would not register periods. Differences comply. Sound things shall forget again easily lik|bi-annual| +9779|AAAAAAAADDGCAAAA|2452640|2452821|DEPARTMENT|91|59|Deliberately minimal computers used to get living resul|bi-annual| +9780|AAAAAAAAEDGCAAAA|2452640|2452821|DEPARTMENT|91|60|Clear findings cease less sure, right interests; reasonable, only courts may see constantly consta|bi-annual| +9781|AAAAAAAAFDGCAAAA|2452640|2452821|DEPARTMENT|91|61|Seats kill quickly deeply supreme books. Others could|bi-annual| +9782|AAAAAAAAGDGCAAAA|2452640|2452821|DEPARTMENT|91|62|New, financial resources could want at the others. So mad children shoul|bi-annual| +9783|AAAAAAAAHDGCAAAA|2452640|2452821|DEPARTMENT|91|63|Alone, old actions must need flames. There international officials must make explici|bi-annual| +9784|AAAAAAAAIDGCAAAA|2452640|2452821|DEPARTMENT|91|64|Original difficulties must not enjoy. Divine, afraid needs search successful, educational depos|bi-annual| +9785|AAAAAAAAJDGCAAAA|2452640|2452821|DEPARTMENT|91|65|Days use only in a matches. Quick, legal researchers fai|bi-annual| +9786|AAAAAAAAKDGCAAAA|2452640|2452821|DEPARTMENT|91|66|Now different departments would realize white, good nerves. Clos|bi-annual| +9787|AAAAAAAALDGCAAAA|2452640|2452821|DEPARTMENT|91|67|Children could not forget from the ways. Wide letters tell. Pleasant expectations ma|bi-annual| +9788|AAAAAAAAMDGCAAAA|2452640|2452821|DEPARTMENT|91|68|Clouds cast available matters. Directors might regulate in order. Sincerely personal friends lik|bi-annual| +9789|AAAAAAAANDGCAAAA|2452640|2452821|DEPARTMENT|91|69|Dark results used to end far. Chronic articles would make nowhere. Left stars might not |bi-annual| +9790|AAAAAAAAODGCAAAA|2452640|2452821|DEPARTMENT|91|70|Better universal losses abolish countries. Beds unite in a interpretations.|bi-annual| +9791|AAAAAAAAPDGCAAAA|2452640|2452821|DEPARTMENT|91|71|Available, light bits may exercise better high progr|bi-annual| +9792|AAAAAAAAAEGCAAAA||2452821|||72|New, radical men write effects. Slightly good women stay f|bi-annual| +9793|AAAAAAAABEGCAAAA|2452640|2452821|DEPARTMENT|91|73|European materials will halt so popular, acceptable procedures. Elements reduce|bi-annual| +9794|AAAAAAAACEGCAAAA|2452640|2452821|DEPARTMENT|91|74|Fortunately available libraries practise entirely around a ideas. Courts|bi-annual| +9795|AAAAAAAADEGCAAAA|2452640|2452821|DEPARTMENT|91|75|Systematically likely experiences prevent initially typical workers. Other, deaf waves will hap|bi-annual| +9796|AAAAAAAAEEGCAAAA|2452640|2452821|DEPARTMENT|91|76|Perfectly traditional feet wonder to a services. Also unnecessary risks get alone certain qualiti|bi-annual| +9797|AAAAAAAAFEGCAAAA|2452640|2452821|DEPARTMENT|91|77|Above expensive practitioners must approve hours. More dead degrees may persuade t|bi-annual| +9798|AAAAAAAAGEGCAAAA|2452640|2452821|DEPARTMENT|91|78|Working, new differences may not tell perhaps element|bi-annual| +9799|AAAAAAAAHEGCAAAA|2452640|2452821|DEPARTMENT|91|79|Permanently economic sets can go then. Conventional individuals can show yet; things receive as so|bi-annual| +9800|AAAAAAAAIEGCAAAA|2452640|2452821|DEPARTMENT|91|80|More other jobs inherit other workers. Almost clear times c|bi-annual| +9801|AAAAAAAAJEGCAAAA|2452640|2452821|DEPARTMENT|91|81|Seasons used to monitor however feelings. Much safe sess|bi-annual| +9802|AAAAAAAAKEGCAAAA|2452640|2452821|DEPARTMENT|91|82|Canadian days shall cost police. Proper incentives shall prevent criteria. Here proper christians a|bi-annual| +9803|AAAAAAAALEGCAAAA|2452640|2452821|DEPARTMENT|91|83|High, material times leave. Never main trains must not respond. Too international tickets shall an|bi-annual| +9804|AAAAAAAAMEGCAAAA|2452640|2452821|DEPARTMENT|91|84|Satisfied, tough things enjoy so on a months; later religious months e|bi-annual| +9805|AAAAAAAANEGCAAAA|2452640|2452821|DEPARTMENT|91|85|Men achieve only years. Natural systems describe so local |bi-annual| +9806|AAAAAAAAOEGCAAAA|2452640|2452821|DEPARTMENT|91|86|International seasons must carry outside able, southern holidays. Splendid, full ma|bi-annual| +9807|AAAAAAAAPEGCAAAA|2452640|2452821|DEPARTMENT|91|87|Blank, public feelings shall change considerably simply new ligh|bi-annual| +9808|AAAAAAAAAFGCAAAA|2452640|2452821|DEPARTMENT|91|88|Political conflicts persuade modern, subject systems. |bi-annual| +9809|AAAAAAAABFGCAAAA|2452640|2452821|DEPARTMENT|91|89|Services can consider on a conditions. Other friends might not hire. Small, public friend|bi-annual| +9810|AAAAAAAACFGCAAAA|2452640|2452821|DEPARTMENT|91|90|Sole leaders conform competitive, other colleagues. Practical, aggressive animals wait public,|bi-annual| +9811|AAAAAAAADFGCAAAA|2452640|2452821|DEPARTMENT|91|91|Strong ways shall not encourage very local times. Tales say well clothes; ho|bi-annual| +9812|AAAAAAAAEFGCAAAA|2452640|2452821|DEPARTMENT|91|92|Generally additional minds embrace of course. So private parts conduct. Reduced, automatic |bi-annual| +9813|AAAAAAAAFFGCAAAA|2452640|2452821|DEPARTMENT|91|93|Guidelines know military elections. Major names should cut critically wrong protests. Desperate ne|bi-annual| +9814|AAAAAAAAGFGCAAAA|2452640|2452821|DEPARTMENT|91|94|Pp. get indeed. Forces hit. Square, good theories may mark regional, immediate cars. P|bi-annual| +9815|AAAAAAAAHFGCAAAA|2452640|2452821|DEPARTMENT|91|95|Still working conditions must seem so in a hands. New, wrong blues move|bi-annual| +9816|AAAAAAAAIFGCAAAA|2452640|2452821|DEPARTMENT|91|96|Likely, dry papers might overcome. Correct thoughts rid. Partial, present questions ought to|bi-annual| +9817|AAAAAAAAJFGCAAAA|2452640|2452821|DEPARTMENT|91|97|Windows might not look then permanent parties. Decisions should|bi-annual| +9818|AAAAAAAAKFGCAAAA|2452640|2452821|DEPARTMENT|91|98|Cultural months must not call sometimes now distant|bi-annual| +9819|AAAAAAAALFGCAAAA|2452640|2452821|DEPARTMENT|91|99|Historians should not put into a lives. Individual affairs p|bi-annual| +9820|AAAAAAAAMFGCAAAA|2452640|2452821|DEPARTMENT|91|100|Poor, human solicitors could not see subjects. Past, busy pr|bi-annual| +9821|AAAAAAAANFGCAAAA|2452640|2452821|DEPARTMENT|91|101|Clear, successful centres think then legal, female groups. Clinical pri|bi-annual| +9822|AAAAAAAAOFGCAAAA|2452640|2452821|DEPARTMENT|91|102|Either unable grounds know officially real, important losses; overtly industr|bi-annual| +9823|AAAAAAAAPFGCAAAA|2452640|2452821|DEPARTMENT|91|103|Clothes provide barely steps. Given, ancient measur|bi-annual| +9824|AAAAAAAAAGGCAAAA|2452640|2452821|DEPARTMENT|91|104|Labour forces work financial, revolutionary reports. Inevitably great police could not see |bi-annual| +9825|AAAAAAAABGGCAAAA|2452640|||91|105|English objections could not protect at all components. Then good years might not get |bi-annual| +9826|AAAAAAAACGGCAAAA|2452640|2452821|DEPARTMENT|91|106|Even special years strike secondly. Deliberate journalists like circumsta|bi-annual| +9827|AAAAAAAADGGCAAAA|2452640|2452821|DEPARTMENT|91|107|Too political losses make other concepts. Opinions shall not affe|bi-annual| +9828|AAAAAAAAEGGCAAAA|2452640|2452821|DEPARTMENT|91|108|Walls allocate correct parts; large decisions will decide either miles; |bi-annual| +9829|AAAAAAAAFGGCAAAA|2452822|2453003|DEPARTMENT|92|1|Usual bones ask possible years. Comparatively concerned sha|bi-annual| +9830|AAAAAAAAGGGCAAAA|2452822|2453003|DEPARTMENT|92|2|Specialist resources find cities. Students continue. Contents help most for a seeds. St|bi-annual| +9831|AAAAAAAAHGGCAAAA|2452822|2453003|DEPARTMENT|92|3|As particular attacks shall mind. Constant, other shareholders |bi-annual| +9832|AAAAAAAAIGGCAAAA|2452822|2453003|DEPARTMENT|92|4|Officers will put sadly only full eggs. There above attitudes deter|bi-annual| +9833|AAAAAAAAJGGCAAAA|2452822|2453003|DEPARTMENT|92|5|Close, realistic societies could not drag proposals. N|bi-annual| +9834|AAAAAAAAKGGCAAAA|2452822|2453003|DEPARTMENT|92|6|Here monthly voters differ as very free parents; available, small trustees accept much sure e|bi-annual| +9835|AAAAAAAALGGCAAAA|2452822|2453003|DEPARTMENT|92|7|At least only users stay; brief contents bear in a chapters; wrong |bi-annual| +9836|AAAAAAAAMGGCAAAA|2452822|2453003|DEPARTMENT|92|8|Really other arrangements would exist by a corners; int|bi-annual| +9837|AAAAAAAANGGCAAAA|2452822|2453003|DEPARTMENT|92|9|Red, single activities die tasks. Various prices ca|bi-annual| +9838|AAAAAAAAOGGCAAAA|2452822|2453003|DEPARTMENT|92|10|Then cheerful faces buy simply good races. Different others can levy as r|bi-annual| +9839|AAAAAAAAPGGCAAAA|2452822|2453003|DEPARTMENT|92|11|More whole workers make important, necessary examples; often important days must not sell tough, |bi-annual| +9840|AAAAAAAAAHGCAAAA|2452822|2453003|DEPARTMENT|92|12|Expensive knees occur beforehand political, young funds. Imp|bi-annual| +9841|AAAAAAAABHGCAAAA|2452822|2453003|DEPARTMENT|92|13|Well persistent points would not buy new conditions. Whole efforts pay perso|bi-annual| +9842|AAAAAAAACHGCAAAA|2452822|2453003|DEPARTMENT|92|14|Open offences bargain further imports; players would provid|bi-annual| +9843|AAAAAAAADHGCAAAA|2452822|2453003|DEPARTMENT|92|15|Much safe designs distract. Fees control therefore scottish j|bi-annual| +9844|AAAAAAAAEHGCAAAA|2452822|2453003|DEPARTMENT|92|16|Healthy, federal women think often different feet. Windows receive never ava|bi-annual| +9845|AAAAAAAAFHGCAAAA|2452822|2453003|DEPARTMENT|92|17|Wages must tell both theories. Limits may become only prepared, far feet. Unive|bi-annual| +9846|AAAAAAAAGHGCAAAA|2452822|2453003|DEPARTMENT|92|18|Thanks ought to secure unlikely, everyday weeks; other, useful groups ask shortly |bi-annual| +9847|AAAAAAAAHHGCAAAA||||92|19||bi-annual| +9848|AAAAAAAAIHGCAAAA|2452822|2453003|DEPARTMENT|92|20|Paths break feet. Now friendly shows must not want particularly offic|bi-annual| +9849|AAAAAAAAJHGCAAAA|2452822|2453003|DEPARTMENT|92|21|Students speak hence. Additional, economic companies might not call alway|bi-annual| +9850|AAAAAAAAKHGCAAAA|2452822|2453003|DEPARTMENT|92|22|Details provide of course direct, other magistrates; then polit|bi-annual| +9851|AAAAAAAALHGCAAAA|2452822|2453003|DEPARTMENT|92|23|Years used to start already complex, able parts. Months must refer religious, fund|bi-annual| +9852|AAAAAAAAMHGCAAAA|2452822|2453003|DEPARTMENT|92|24|Visual changes might take also great surveys. Still interviews come. Political|bi-annual| +9853|AAAAAAAANHGCAAAA|2452822|2453003|DEPARTMENT|92|25|Russian, adequate areas tell in a weeks. Incentives honour little great, immediate shee|bi-annual| +9854|AAAAAAAAOHGCAAAA|2452822|2453003|DEPARTMENT|92|26|Grim others go elsewhere bad, whole stories; new, appropriate plants limit little, rel|bi-annual| +9855|AAAAAAAAPHGCAAAA|2452822|2453003|DEPARTMENT|92|27|Available, arab recordings could not allow locally alternative, small regions. Serious title|bi-annual| +9856|AAAAAAAAAIGCAAAA|2452822|2453003|DEPARTMENT|92|28|Original, new meetings search major companies. Great schools would not make rich nu|bi-annual| +9857|AAAAAAAABIGCAAAA|2452822|2453003|DEPARTMENT|92|29|Sweet, local roads can protect. Various, bad readers move just flowers; wonderful relations |bi-annual| +9858|AAAAAAAACIGCAAAA|2452822|2453003|DEPARTMENT|92|30|Leaders examine; young, video-taped rights could look kin|bi-annual| +9859|AAAAAAAADIGCAAAA|2452822|2453003|DEPARTMENT|92|31|Fine, modern arts will explain then; so important individuals move often other products. |bi-annual| +9860|AAAAAAAAEIGCAAAA|2452822|2453003|DEPARTMENT|92|32|Social, key moments could record poor remarks; there |bi-annual| +9861|AAAAAAAAFIGCAAAA|2452822|2453003|DEPARTMENT|92|33|Laws can go bottom eyes. Bad, various backs shall initiate pupils. Wives shoot. Periods stop nega|bi-annual| +9862|AAAAAAAAGIGCAAAA|2452822|2453003|DEPARTMENT|92|34|Sorts must begin sure, significant perceptions. Equally principal standards stay usually short|bi-annual| +9863|AAAAAAAAHIGCAAAA|2452822|2453003|DEPARTMENT|92|35|Gentle consequences would not come initially old, lo|bi-annual| +9864|AAAAAAAAIIGCAAAA|2452822|2453003|DEPARTMENT|92|36|Today unusual courses help enormously often domestic premises. Gaps br|bi-annual| +9865|AAAAAAAAJIGCAAAA|2452822|2453003|DEPARTMENT|92|37|Likely, fascinating words used to need. Primary nations would not|bi-annual| +9866|AAAAAAAAKIGCAAAA|2452822|2453003|DEPARTMENT|92|38|Social sciences appeal obviously prisons; only multiple arms might sell but for a costs|bi-annual| +9867|AAAAAAAALIGCAAAA|2452822|2453003|DEPARTMENT|92|39|Local, main effects must include like the prices. New, local pp. could witness late|bi-annual| +9868|AAAAAAAAMIGCAAAA|2452822|2453003|DEPARTMENT|92|40|Good, russian expectations could need then. Customers |bi-annual| +9869|AAAAAAAANIGCAAAA|2452822|2453003|DEPARTMENT|92|41|Men get badly hands. Ever little meetings may not talk. New categories fall impossible br|bi-annual| +9870|AAAAAAAAOIGCAAAA|2452822|2453003|DEPARTMENT|92|42|Most sexual vessels must come then social, easy cases. More impressive a|bi-annual| +9871|AAAAAAAAPIGCAAAA|2452822|2453003|DEPARTMENT||43||bi-annual| +9872|AAAAAAAAAJGCAAAA|2452822|2453003|DEPARTMENT|92|44|All direct refugees might kill just necessary officia|bi-annual| +9873|AAAAAAAABJGCAAAA|2452822|2453003|DEPARTMENT|92|45|Movements keep really young, live posts. Black tories shall learn ne|bi-annual| +9874|AAAAAAAACJGCAAAA|2452822|2453003|DEPARTMENT|92|46|Then national experiences say yet strong, interesting hours. Economically old industries w|bi-annual| +9875|AAAAAAAADJGCAAAA|2452822|2453003|DEPARTMENT|92|47|Primary, empty users enjoy flames. Residents obtain certainly rights. Round women|bi-annual| +9876|AAAAAAAAEJGCAAAA|2452822|2453003|DEPARTMENT|92|48|Exact vehicles like substantially to a women; possible things can know longer c|bi-annual| +9877|AAAAAAAAFJGCAAAA|2452822|2453003|DEPARTMENT|92|49|Angry, common years diminish. Thus male techniques could help to the conclusions. |bi-annual| +9878|AAAAAAAAGJGCAAAA|2452822|2453003|DEPARTMENT|92|50|Slow, islamic troops can use connections. All white rights get afterwards styles.|bi-annual| +9879|AAAAAAAAHJGCAAAA|2452822|2453003|DEPARTMENT|92|51|More european figures used to understand. White, good players turn rather policies.|bi-annual| +9880|AAAAAAAAIJGCAAAA|2452822|2453003|DEPARTMENT|92|52|Still other reports use still for an changes. Best free friends take more; for example|bi-annual| +9881|AAAAAAAAJJGCAAAA|2452822|2453003|DEPARTMENT|92|53|Never numerous factors eat scarcely mature shoulders; about capital forms used to explain claim|bi-annual| +9882|AAAAAAAAKJGCAAAA|2452822|2453003|DEPARTMENT|92|54|Real expectations must not return effectively. Disciplina|bi-annual| +9883|AAAAAAAALJGCAAAA|2452822|2453003|DEPARTMENT|92|55|Attitudes mask critical, obvious legs. Commercial weeks emulate respect|bi-annual| +9884|AAAAAAAAMJGCAAAA|2452822|2453003|DEPARTMENT|92|56|Institutions cure german, local parties; leaders mu|bi-annual| +9885|AAAAAAAANJGCAAAA|2452822|2453003|DEPARTMENT|92|57|Labour changes must not think around local, golden students. Austr|bi-annual| +9886|AAAAAAAAOJGCAAAA|2452822|2453003|DEPARTMENT|92|58|Children ought to derive severely wings. Then simi|bi-annual| +9887|AAAAAAAAPJGCAAAA|2452822|2453003|DEPARTMENT|92|59|Full, certain standards might treat gradually working stones|bi-annual| +9888|AAAAAAAAAKGCAAAA|2452822|2453003|DEPARTMENT|92|60|Orthodox members would spread on the men. Groups will end mammals; times|bi-annual| +9889|AAAAAAAABKGCAAAA|2452822|2453003|DEPARTMENT|92|61|Blue, other examples shall wish girls. Young studies become other bodies. Able men go perhaps|bi-annual| +9890|AAAAAAAACKGCAAAA|2452822|2453003|DEPARTMENT|92|62|Yet french trends may want here small institutions; ways call i|bi-annual| +9891|AAAAAAAADKGCAAAA|2452822|2453003|DEPARTMENT|92|63|Either regular others finish most within a institution|bi-annual| +9892|AAAAAAAAEKGCAAAA|2452822|2453003|DEPARTMENT|92|64|Proposals try by the projects; perhaps large women must make later current matches. New, contin|bi-annual| +9893|AAAAAAAAFKGCAAAA|2452822|2453003|DEPARTMENT|92|65|Then blue countries might give again responses. New|bi-annual| +9894|AAAAAAAAGKGCAAAA|2452822|2453003|DEPARTMENT|92|66|Always other rates think badly thick, selective sources; concerned, bla|bi-annual| +9895|AAAAAAAAHKGCAAAA|2452822|2453003|DEPARTMENT|92|67|Recent women snap as full, original computers. Employers wi|bi-annual| +9896|AAAAAAAAIKGCAAAA|2452822|2453003|DEPARTMENT|92|68|Very old eyes must introduce recent friends; professionally bold children must welcome; fixed fl|bi-annual| +9897|AAAAAAAAJKGCAAAA|2452822|2453003|DEPARTMENT|92|69|Different values shall escape presumably in the miners; superior, r|bi-annual| +9898|AAAAAAAAKKGCAAAA|2452822|2453003|DEPARTMENT|92|70|Great, tiny interests adapt into the nerves. Good, southern tenants must |bi-annual| +9899|AAAAAAAALKGCAAAA|2452822|2453003|DEPARTMENT|92|71|Alone new pounds used to constitute once informal weekly |bi-annual| +9900|AAAAAAAAMKGCAAAA|2452822|2453003|DEPARTMENT|92|72|Complete shareholders used to put private jobs. Away eager pieces fight usually l|bi-annual| +9901|AAAAAAAANKGCAAAA|2452822|2453003|DEPARTMENT|92|73|Social, final chains could not steal necessary, whol|bi-annual| +9902|AAAAAAAAOKGCAAAA|2452822|2453003|DEPARTMENT|92|74|Never married numbers must match already capital happy practitioners. Of course very st|bi-annual| +9903|AAAAAAAAPKGCAAAA|2452822|2453003|DEPARTMENT|92|75|Ready, european payments want habits. Both european regulations can work gradually. Full ideas sh|bi-annual| +9904|AAAAAAAAALGCAAAA|2452822|2453003|DEPARTMENT|92|76|Still reports will level else. Contents will not suspect available, natural rocks; feelings|bi-annual| +9905|AAAAAAAABLGCAAAA|2452822|2453003|DEPARTMENT|92|77|Agricultural contents give so heavily weak techniques. Toys settle yesterd|bi-annual| +9906|AAAAAAAACLGCAAAA|2452822|2453003|DEPARTMENT|92|78|New countries should bring as. Pupils say foreigners. Boys build clos|bi-annual| +9907|AAAAAAAADLGCAAAA|2452822|2453003|DEPARTMENT|92|79|Public, full qualities stop even close, elegant elements; civil, national letters wi|bi-annual| +9908|AAAAAAAAELGCAAAA|2452822|2453003|DEPARTMENT|92|80|Foreign, available ladies may no eyes. Strategic days play at a comments. Significant owners use of|bi-annual| +9909|AAAAAAAAFLGCAAAA|2452822|2453003|DEPARTMENT|92|81|Injuries ignore even levels. Outside, proper games should intervene almost major votes. Sec|bi-annual| +9910|AAAAAAAAGLGCAAAA|2452822|2453003|DEPARTMENT|92|82|Tonnes look a bit today other methods. Good, long differences meet today interna|bi-annual| +9911|AAAAAAAAHLGCAAAA|2452822|2453003|DEPARTMENT|92|83|Shoulders load from the members. Parties must construct more modern, other companies. Large, |bi-annual| +9912|AAAAAAAAILGCAAAA|2452822|2453003|DEPARTMENT|92|84|Appropriate years shall circulate earlier police. Hands may take yesterday closer po|bi-annual| +9913|AAAAAAAAJLGCAAAA|2452822|2453003|DEPARTMENT|92|85|Academic keys will take in a weeks. Brilliant, good heroes collect else; artists sit fo|bi-annual| +9914|AAAAAAAAKLGCAAAA|2452822|2453003|DEPARTMENT|92|86|Probably living users deny on a owners. Potential, great problems could help sen|bi-annual| +9915|AAAAAAAALLGCAAAA|2452822|2453003|DEPARTMENT|92|87|Financial clients cannot stand clubs. High, steep businesses fall somewhat wor|bi-annual| +9916|AAAAAAAAMLGCAAAA|2452822|2453003|DEPARTMENT|92|88|Old minutes may not say to a actors. Small trees go in a others. Forms shall not give certainl|bi-annual| +9917|AAAAAAAANLGCAAAA|2452822|2453003|DEPARTMENT|92|89|Rather other others would emphasize amounts; then fair resources mus|bi-annual| +9918|AAAAAAAAOLGCAAAA|2452822|2453003|DEPARTMENT|92|90|Ideas try by the opponents; citizens see schools. Simply growing countries finish sure d|bi-annual| +9919|AAAAAAAAPLGCAAAA|2452822|2453003|DEPARTMENT|92|91|Grand loans deliver much. More european origins may remain dry names. Gentle, m|bi-annual| +9920|AAAAAAAAAMGCAAAA|2452822|2453003|DEPARTMENT|92|92|Responsible, old hours form never also decent bacteria. Even other clas|bi-annual| +9921|AAAAAAAABMGCAAAA|2452822|2453003|DEPARTMENT|92|93|Potential, logical women beat parallel. Nearly other sons shall not choose immen|bi-annual| +9922|AAAAAAAACMGCAAAA|2452822|2453003|DEPARTMENT|92|94|Then labour rights should accept together similarly short-term sites. Again complex|bi-annual| +9923|AAAAAAAADMGCAAAA|2452822|2453003|DEPARTMENT|92|95|Governments must not maximise recently proposed, urban stages; true, different pattern|bi-annual| +9924|AAAAAAAAEMGCAAAA|2452822|2453003|DEPARTMENT|92|96|Appropriate, likely signs must not know other than a car|bi-annual| +9925|AAAAAAAAFMGCAAAA|2452822|2453003|DEPARTMENT|92|97|Machines follow skills. Together major bones should warn more at a miles. O|bi-annual| +9926|AAAAAAAAGMGCAAAA|2452822|2453003|DEPARTMENT|92|98|Volumes work purposes. Effective, previous children might telephone later. Now narrow items use |bi-annual| +9927|AAAAAAAAHMGCAAAA|2452822|2453003|DEPARTMENT|92|99|Workers ring in a restaurants. Special arts answer quite. Factors refer in a purposes. Far huge com|bi-annual| +9928|AAAAAAAAIMGCAAAA|2452822|2453003|DEPARTMENT|92|100|Fit conditions insist areas. Families ought to go early |bi-annual| +9929|AAAAAAAAJMGCAAAA|2452822|2453003|DEPARTMENT|92|101|Lines may decide inexorably in a circumstances. Inter|bi-annual| +9930|AAAAAAAAKMGCAAAA|2452822|2453003|DEPARTMENT|92|102|Entirely short hands obstruct most. Available, only materials contro|bi-annual| +9931|AAAAAAAALMGCAAAA|2452822|2453003|DEPARTMENT|92|103|Historical controls last medical, eventual firms. Particular, safe resources could see in ter|bi-annual| +9932|AAAAAAAAMMGCAAAA|2452822|2453003|DEPARTMENT|92|104|New facilities must extend. Just other others could not show other, key lines. Separate, f|bi-annual| +9933|AAAAAAAANMGCAAAA|2452822|2453003|DEPARTMENT|92|105|Recent days average national students. Naturally high words would obtain. Political managers wi|bi-annual| +9934|AAAAAAAAOMGCAAAA|2452822|2453003|DEPARTMENT|92|106|Medical meetings implement reportedly rules. Other words m|bi-annual| +9935|AAAAAAAAPMGCAAAA|2452822|2453003|DEPARTMENT|92|107|Irish sources suggest national, popular days. Thin, public plants exclude great|bi-annual| +9936|AAAAAAAAANGCAAAA|2452822|2453003|DEPARTMENT|92|108|As english courses must not conduct years. Fixed, local contra|bi-annual| +9937|AAAAAAAABNGCAAAA|2452640|2452730|DEPARTMENT|93|1|Years will realise wishes; top witnesses might get fatal, long-term arguments; there major accoun|quarterly| +9938|AAAAAAAACNGCAAAA||2452730||93||As new strengths could overthrow more unacceptable || +9939|AAAAAAAADNGCAAAA|2452640|2452730|DEPARTMENT|93|3|Broad children offer much black, broad arms. Relatively accessibl|quarterly| +9940|AAAAAAAAENGCAAAA|2452640|2452730|DEPARTMENT|93|4|Patient, inner hands would not cook models. Eager, public times pay never again ot|quarterly| +9941|AAAAAAAAFNGCAAAA|2452640|2452730|DEPARTMENT|93|5|Mainly natural walls ought to use new, other orders. Streets describe for instance into a circum|quarterly| +9942|AAAAAAAAGNGCAAAA|2452640|2452730|DEPARTMENT|93|6|Only serious funds would run for a trains. Never good names run social details; relevant forms|quarterly| +9943|AAAAAAAAHNGCAAAA|2452640|2452730|DEPARTMENT|93|7|Specific, poor years go better. That elected pupils would |quarterly| +9944|AAAAAAAAINGCAAAA|2452640|2452730|DEPARTMENT|93|8|Otherwise foreign estates cannot know forces. Previ|quarterly| +9945|AAAAAAAAJNGCAAAA|2452640|2452730|DEPARTMENT|93|9|More general members should go very other only forces. Papers |quarterly| +9946|AAAAAAAAKNGCAAAA|2452640|2452730|DEPARTMENT|93|10|Lights help stupid ideas. Forever slow families approach ever for a stars. |quarterly| +9947|AAAAAAAALNGCAAAA|2452640|2452730|DEPARTMENT|93|11|Large occupations ought to say as complete bags. Enough deep officers will make middle-cl|quarterly| +9948|AAAAAAAAMNGCAAAA|2452640|2452730|DEPARTMENT|93|12|Degrees shall entertain really subsequent, dependent duties. Small, narrow wages might get widely s|quarterly| +9949|AAAAAAAANNGCAAAA|2452640|2452730|DEPARTMENT|93|13|Endless conservatives must not round properly. Good |quarterly| +9950|AAAAAAAAONGCAAAA|2452640|2452730|DEPARTMENT|93|14|Following products reach young candidates. Fit standards refer. So spare terms may trade around |quarterly| +9951|AAAAAAAAPNGCAAAA|2452640|2452730|DEPARTMENT|93|15|British hours belong. Triumphantly young prices pay unknown, soci|quarterly| +9952|AAAAAAAAAOGCAAAA|2452640|2452730|DEPARTMENT|93|16|Wrong, national connections give briefly environmental quantities. Lawyers |quarterly| +9953|AAAAAAAABOGCAAAA|2452640|2452730|DEPARTMENT|93|17|Either small eyes exaggerate fortunate, unlike trees. Remaining prisons t|quarterly| +9954|AAAAAAAACOGCAAAA|2452640|2452730|DEPARTMENT|93|18|Classic edges help perhaps living rights. Miles produce suffi|quarterly| +9955|AAAAAAAADOGCAAAA|2452640|2452730|DEPARTMENT|93|19|Organisations may not count rather by the elements. Likely, oth|quarterly| +9956|AAAAAAAAEOGCAAAA|2452640|2452730|DEPARTMENT|93|20|Wild, fundamental homes think now on a offences. Theoretical w|quarterly| +9957|AAAAAAAAFOGCAAAA|2452640|2452730|DEPARTMENT|93|21|Days determine for the changes. Obvious, satisfactory nights le|quarterly| +9958|AAAAAAAAGOGCAAAA|2452640|2452730|DEPARTMENT|93|22|Results keep; very western funds lead as well communications. Beings|quarterly| +9959|AAAAAAAAHOGCAAAA|2452640|2452730|DEPARTMENT|93|23|Statutory, present times may not become partly single services. Larg|quarterly| +9960|AAAAAAAAIOGCAAAA|2452640|2452730|DEPARTMENT|93|24|Rather other classes include low, primary opportunities. S|quarterly| +9961|AAAAAAAAJOGCAAAA|2452640|2452730|DEPARTMENT|93|25|English communications create minus a complaints; fashionable, extra cases must wo|quarterly| +9962|AAAAAAAAKOGCAAAA|2452640|2452730|DEPARTMENT|93|26|Continually famous leaves ought to shoot countries. All small trees deter har|quarterly| +9963|AAAAAAAALOGCAAAA|2452640|2452730|DEPARTMENT|93|27|Urban careers must take initial, forthcoming molecules. Clearly ordinary h|quarterly| +9964|AAAAAAAAMOGCAAAA|2452640|2452730|DEPARTMENT|93|28|Tentatively ready proposals find too just german records. Anxiously molecular countri|quarterly| +9965|AAAAAAAANOGCAAAA|2452640|2452730|DEPARTMENT|93|29|Letters encourage so cool, top pages. Emotional, low eyes identify |quarterly| +9966|AAAAAAAAOOGCAAAA|2452640|2452730|DEPARTMENT|93|30|So separate hours used to review deeper wide, extended wings. Here afraid women would compet|quarterly| +9967|AAAAAAAAPOGCAAAA|2452640|2452730|DEPARTMENT|93|31|Close, legal months burn as just significant arts. Possible jobs shall rise heavy, cr|quarterly| +9968|AAAAAAAAAPGCAAAA|2452640|2452730|DEPARTMENT|93|32|Further necessary prayers stand just red, foreign regions. Directions must kee|quarterly| +9969|AAAAAAAABPGCAAAA|2452640|2452730|DEPARTMENT|93|33|Arguments accept about satisfactory ministers; variables ma|quarterly| +9970|AAAAAAAACPGCAAAA|2452640|2452730|DEPARTMENT|93|34|Increasing, necessary theories may not fancy then meanwhile bas|quarterly| +9971|AAAAAAAADPGCAAAA|2452640|2452730|DEPARTMENT|93|35|Parties improve close. Shared concepts used to find |quarterly| +9972|AAAAAAAAEPGCAAAA|2452640|2452730|DEPARTMENT|93|36|Facilities confirm whole, busy lectures. Nuclear components mean standard, irrelevant ch|quarterly| +9973|AAAAAAAAFPGCAAAA|2452640|2452730|DEPARTMENT|93|37|States cannot pay beautiful months; now high policies than|quarterly| +9974|AAAAAAAAGPGCAAAA|2452640|2452730|DEPARTMENT|93|38|Confident technologies go also. Premier methods call a|quarterly| +9975|AAAAAAAAHPGCAAAA|2452640|2452730|DEPARTMENT|93|39|Advanced, compulsory issues replace more in the windows. Inc councils would not read illegal studie|quarterly| +9976|AAAAAAAAIPGCAAAA|2452640|2452730|DEPARTMENT|93|40|Procedures may not touch useful branches. Deep, political categories may love most tho|quarterly| +9977|AAAAAAAAJPGCAAAA|2452640|2452730|DEPARTMENT|93|41|Alone, necessary messages open yet. Probably large words might emerge majo|quarterly| +9978|AAAAAAAAKPGCAAAA|2452640|2452730|DEPARTMENT|93|42|High words meet so. Financial soldiers will tell. Assumptions could wear over new ch|quarterly| +9979|AAAAAAAALPGCAAAA|2452640|2452730|DEPARTMENT|93|43|Features need worse in a years. Services would not increase minds. Impo|quarterly| +9980|AAAAAAAAMPGCAAAA|2452640|2452730|DEPARTMENT|93|44|Social, full children ought to attend a bit just origin|quarterly| +9981|AAAAAAAANPGCAAAA|2452640|2452730|DEPARTMENT|93|45|Regional shows ought to allow shares. Partly bitter workers might not think ahead efficient primar|quarterly| +9982|AAAAAAAAOPGCAAAA|2452640|2452730|DEPARTMENT|93|46|Hands must tell later. Busy methods must not save silently at a ideas. Internal, |quarterly| +9983|AAAAAAAAPPGCAAAA|2452640|2452730|DEPARTMENT|93|47|International shareholders might remove more on a committees. A|quarterly| +9984|AAAAAAAAAAHCAAAA|2452640|||93|48|Hours turn just everywhere late skills. Rules will not m|| +9985|AAAAAAAABAHCAAAA|2452640|2452730|DEPARTMENT|93|49|Always standard bands transfer here. Economic, desperate customers change|quarterly| +9986|AAAAAAAACAHCAAAA|2452640|2452730|DEPARTMENT|93|50|Lengths justify royal things; still central interests should me|quarterly| +9987|AAAAAAAADAHCAAAA|2452640|2452730|DEPARTMENT|93|51|Organisations must not pursue almost white players|quarterly| +9988|AAAAAAAAEAHCAAAA|2452640|2452730|DEPARTMENT|93|52|Possibly english features might not go softly finally central cars. Now th|quarterly| +9989|AAAAAAAAFAHCAAAA|2452640|2452730|DEPARTMENT|93|53|Systems should tell. Currently british arrangements |quarterly| +9990|AAAAAAAAGAHCAAAA|2452640|2452730|DEPARTMENT|93|54|Sons want without a profits. Yesterday bitter studies provide also universitie|quarterly| +9991|AAAAAAAAHAHCAAAA|2452640|2452730|DEPARTMENT|93|55|Serious arts should send else here involved lights. Slowly real children could wri|quarterly| +9992|AAAAAAAAIAHCAAAA|2452640|2452730|DEPARTMENT|93|56|Confident, clear versions will see less only changes. Del|quarterly| +9993|AAAAAAAAJAHCAAAA|2452640|2452730|DEPARTMENT|93|57|Activities ought to view harshly capable addresses; texts make thereby ne|quarterly| +9994|AAAAAAAAKAHCAAAA|2452640|2452730|DEPARTMENT|93|58|Similar boundaries would serve words; sexually average ac|quarterly| +9995|AAAAAAAALAHCAAAA|2452640|2452730|DEPARTMENT|93|59|Good, specialist buildings might portray occasionally royal, unaware cases. Alone ruli|quarterly| +9996|AAAAAAAAMAHCAAAA|2452640|2452730|DEPARTMENT|93|60|Institutions should serve even constant, specific products. Basic, n|quarterly| +9997|AAAAAAAANAHCAAAA|2452640|2452730|DEPARTMENT|93|61|Flat pounds could not entail in a streets. Rather sorr|quarterly| +9998|AAAAAAAAOAHCAAAA|2452640|2452730|DEPARTMENT|93|62|Leaves will not remain more for a lines. There supreme goals cannot take man|quarterly| +9999|AAAAAAAAPAHCAAAA|2452640|2452730|DEPARTMENT|93|63|Interested, linguistic fans ought to find slightly fu|quarterly| +10000|AAAAAAAAABHCAAAA|2452640|2452730|DEPARTMENT|93|64|Meetings prepare best. Firm consultants ought to recognise straight modern, certain pr|quarterly| +10001|AAAAAAAABBHCAAAA|2452640|2452730|DEPARTMENT|93|65|British, strategic restrictions develop regular, slow organisations. Rul|quarterly| +10002|AAAAAAAACBHCAAAA|2452640|2452730|DEPARTMENT|93|66|Eligible profits confide only male, essential points. Always young children shall not retire s|quarterly| +10003|AAAAAAAADBHCAAAA|2452640|2452730|DEPARTMENT|93|67|Considerations must not find still for a rows; empty payments assist remarkable, serious months. |quarterly| +10004|AAAAAAAAEBHCAAAA|2452640|2452730|DEPARTMENT|93|68|Scales go today bodies. Annually resulting factors drive however supposed patients. Full, r|quarterly| +10005|AAAAAAAAFBHCAAAA|2452640|2452730|DEPARTMENT|93|69|Dramatic parts cannot think black elections. Words used to acquire with a years. Japanese costs |quarterly| +10006|AAAAAAAAGBHCAAAA|2452640|2452730|DEPARTMENT|93|70|Organisations could not make inside excellent ways; too|quarterly| +10007|AAAAAAAAHBHCAAAA|2452640|2452730|DEPARTMENT|93|71|Old, due attempts make however english recordings. Hu|quarterly| +10008|AAAAAAAAIBHCAAAA|2452640|2452730|DEPARTMENT|93|72|Upper, main women want. Police would not find together sufficient names; then soviet million|quarterly| +10009|AAAAAAAAJBHCAAAA|2452640|2452730|DEPARTMENT|93|73|Eyes may need however yet following plants. Already good reports may not meet today cau|quarterly| +10010|AAAAAAAAKBHCAAAA|2452640|2452730|DEPARTMENT|93|74|Particular musicians understand organisations. Large, kind parents must use very enti|quarterly| +10011|AAAAAAAALBHCAAAA|2452640|2452730|DEPARTMENT|93|75|Independent, other homes jump initial, black shares. Concentrations face much safe, woode|quarterly| +10012|AAAAAAAAMBHCAAAA|2452640|2452730|DEPARTMENT|93|76|Grounds face thereafter historians. Soon central men used to|quarterly| +10013|AAAAAAAANBHCAAAA|2452640|2452730|DEPARTMENT|93|77|Patterns take men. Over additional weeks destroy sometimes pleasant, |quarterly| +10014|AAAAAAAAOBHCAAAA|2452640|2452730|DEPARTMENT|93|78|Able links set. Useful, german artists tell often heavy y|quarterly| +10015|AAAAAAAAPBHCAAAA|2452640|2452730|DEPARTMENT|93|79|Daily emissions help eventually authorities. Always effective seats indicate also; al|quarterly| +10016|AAAAAAAAACHCAAAA|2452640|2452730|DEPARTMENT|93|80|Important, imaginative ideas address arrangements; hard titles love usually. Dates perform furthe|quarterly| +10017|AAAAAAAABCHCAAAA|2452640|2452730|DEPARTMENT|93|81|Benefits attract. At all old churches shall not continue completely |quarterly| +10018|AAAAAAAACCHCAAAA|2452640|2452730|DEPARTMENT|93|82|Satisfied, emotional seats shall stay. Essential, pale systems|quarterly| +10019|AAAAAAAADCHCAAAA|2452640|2452730|DEPARTMENT|93|83|Important, little victims reflect dry, persistent shows. Indeed acute years must|quarterly| +10020|AAAAAAAAECHCAAAA|2452640|2452730|DEPARTMENT|93|84|Men put namely fields. Otherwise good voices may visit for once then natural operations. Yet |quarterly| +10021|AAAAAAAAFCHCAAAA|2452640|2452730|DEPARTMENT|93|85|Meetings keep traditional, long eyes. Local rooms |quarterly| +10022|AAAAAAAAGCHCAAAA|2452640|2452730|DEPARTMENT|93|86|Completely serious cars avoid sure; times get negative ages. New, detailed bottles see days|quarterly| +10023|AAAAAAAAHCHCAAAA|2452640|2452730|DEPARTMENT|93|87|Other titles used to relax key, able rights. New sales can speak |quarterly| +10024|AAAAAAAAICHCAAAA|2452640|2452730|DEPARTMENT|93|88|Double, capital years point to the achievements. Late new politicians will |quarterly| +10025|AAAAAAAAJCHCAAAA|2452640|2452730|DEPARTMENT|93|89|Photographs result too theoretical children. Schools must not reject orig|quarterly| +10026|AAAAAAAAKCHCAAAA|2452640|2452730|DEPARTMENT|93|90|Wild, chief banks could get. Other years know so by the events; sufficient qu|quarterly| +10027|AAAAAAAALCHCAAAA|2452640|2452730|DEPARTMENT|93|91|Now classical feet shall give commercial holidays. National years may say. Funds |quarterly| +10028|AAAAAAAAMCHCAAAA|2452640|2452730|DEPARTMENT|93|92|Clear, careful circumstances light dramatic, dependent eyes. Low days pr|quarterly| +10029|AAAAAAAANCHCAAAA|2452640|2452730|DEPARTMENT|93|93|Able, british applications could come alone clear elections. Gene|quarterly| +10030|AAAAAAAAOCHCAAAA|2452640|2452730|DEPARTMENT|93|94|New papers live at the years. Places would check detailed, heavy products. Delicious democrats r|quarterly| +10031|AAAAAAAAPCHCAAAA|2452640|2452730|DEPARTMENT|93|95|Options will discover almost english lines. Good, productive wea|quarterly| +10032|AAAAAAAAADHCAAAA|2452640|2452730|DEPARTMENT|93|96|Characters measure right. Little foreign years will not read as increasingly important |quarterly| +10033|AAAAAAAABDHCAAAA|2452640|2452730|DEPARTMENT|93|97|Usual gardens own for a bodies. Very possible pupils will lead goo|quarterly| +10034|AAAAAAAACDHCAAAA|2452640|2452730|DEPARTMENT|93|98|Accessible, modern scales allow well vessels. More military pictures will not say quite education|quarterly| +10035|AAAAAAAADDHCAAAA|2452640|2452730|DEPARTMENT|93|99|More military names would not come so only public sources; hospitals must attract sole, o|quarterly| +10036|AAAAAAAAEDHCAAAA|2452640|2452730|DEPARTMENT|93|100|Free, physical days used to remain often good others; british, possible scots appreciate thus. Tr|quarterly| +10037|AAAAAAAAFDHCAAAA|2452640|2452730|DEPARTMENT|93|101|In particular local features regain deliberate proposals. Acceptab|quarterly| +10038|AAAAAAAAGDHCAAAA|2452640|2452730|DEPARTMENT|93|102|Limited polls shall set from a councillors. Various |quarterly| +10039|AAAAAAAAHDHCAAAA|2452640|2452730|DEPARTMENT|93|103|New, influential pictures know small, basic member|quarterly| +10040|AAAAAAAAIDHCAAAA|2452640|2452730|DEPARTMENT|93|104|However complex restrictions might not close various, leading systems. Important, true effect|quarterly| +10041|AAAAAAAAJDHCAAAA|2452640|2452730|DEPARTMENT|93|105|Children know. Close hills give such as a lines. Most spanish operations impose to a buyers. A l|quarterly| +10042|AAAAAAAAKDHCAAAA|2452640|2452730|DEPARTMENT|93|106|Reforms may not take on a operations; unchanged ways should not get broadly |quarterly| +10043|AAAAAAAALDHCAAAA|2452640|2452730|DEPARTMENT|93|107|Just required researchers hope somewhat in the arrangements. Almost critical streets should g|quarterly| +10044|AAAAAAAAMDHCAAAA|2452640|2452730|DEPARTMENT|93|108|Indeed financial tables can turn closely australian heels; particular, sub|quarterly| +10045|AAAAAAAANDHCAAAA|2452731|2452821|DEPARTMENT|94|1|Arguments might allow years. Players cannot exploit also out of a advisers. Lesser|quarterly| +10046|AAAAAAAAODHCAAAA|2452731|2452821|DEPARTMENT|94|2|Payments should observe like a women. Normal citizens mean obvious, middle days. Mainly jud|quarterly| +10047|AAAAAAAAPDHCAAAA|2452731|2452821|DEPARTMENT|94|3|New ages know players. Intense, economic remarks accept viole|quarterly| +10048|AAAAAAAAAEHCAAAA|2452731|2452821|DEPARTMENT|94|4|Magic, favorite contributions separate more systematic, initial guests. Happy, old studies coul|quarterly| +10049|AAAAAAAABEHCAAAA|2452731|2452821|DEPARTMENT|94|5|Young funds ought to get. So high systems install else origin|quarterly| +10050|AAAAAAAACEHCAAAA|2452731|2452821|DEPARTMENT|94|6|Northern, large sons see quite. Recently private males travel more. Feet make advanc|quarterly| +10051|AAAAAAAADEHCAAAA|2452731|2452821|DEPARTMENT|94|7|Chinese horses know then. New, expensive miles get never sophisticated, good players. Ho|quarterly| +10052|AAAAAAAAEEHCAAAA|2452731|2452821||94|||| +10053|AAAAAAAAFEHCAAAA|2452731|2452821|DEPARTMENT|94|9|Quickly common children know cold, wooden tons. Married |quarterly| +10054|AAAAAAAAGEHCAAAA|2452731|2452821|DEPARTMENT|94|10|Federal parts can ensure similar economies. Interests used to eat whole, scottish principles. Cla|quarterly| +10055|AAAAAAAAHEHCAAAA|2452731|2452821|DEPARTMENT|94|11|Unlikely, special centres take as yet english feet. For example free countries guarantee all good |quarterly| +10056|AAAAAAAAIEHCAAAA|2452731|2452821|DEPARTMENT|94|12|Certain elections pass. Significant, other proteins must n|quarterly| +10057|AAAAAAAAJEHCAAAA|2452731|2452821|DEPARTMENT|94|13|Always limited hours feel dutch drivers. Years talk. Honours co|quarterly| +10058|AAAAAAAAKEHCAAAA|2452731|2452821|DEPARTMENT|94|14|More biological drinks can cover only however poor |quarterly| +10059|AAAAAAAALEHCAAAA|2452731|2452821|DEPARTMENT|94|15|Visual, particular players shall look very with a amounts. Problems p|quarterly| +10060|AAAAAAAAMEHCAAAA|2452731|2452821|DEPARTMENT|94|16|Goods can get. Competitive, detailed rewards break silly, able |quarterly| +10061|AAAAAAAANEHCAAAA|2452731|2452821|DEPARTMENT|94|17|Alternatively political years might not take here. Parties can prefer written ministers. Years |quarterly| +10062|AAAAAAAAOEHCAAAA|2452731|2452821|DEPARTMENT|94|18|Successful, local effects could not act as close humans. For ever familiar present|quarterly| +10063|AAAAAAAAPEHCAAAA|2452731|2452821|DEPARTMENT|94|19|Natural, optimistic systems shall not wish abroad from a miles. Applic|quarterly| +10064|AAAAAAAAAFHCAAAA|2452731|2452821|DEPARTMENT|94|20|Financial angles would not visit single, full sentences. Special, chronic clergy ough|quarterly| +10065|AAAAAAAABFHCAAAA|2452731|2452821|DEPARTMENT|94|21|Hungry, sad males will look bills. Dreadful, big officials start r|quarterly| +10066|AAAAAAAACFHCAAAA|2452731|2452821|DEPARTMENT|94|22|Here irish words offer worldwide assistant supplies. Individuals open much afri|quarterly| +10067|AAAAAAAADFHCAAAA|2452731|2452821|DEPARTMENT|94|23|Literary, vast powers can re-open years. Pink, minute model|quarterly| +10068|AAAAAAAAEFHCAAAA|2452731|2452821|DEPARTMENT|94|24|Tropical, sufficient types fall restaurants; writers validate awful year|quarterly| +10069|AAAAAAAAFFHCAAAA|2452731|2452821|DEPARTMENT|94|25|Certain, small patterns think. More conservative friends raise |quarterly| +10070|AAAAAAAAGFHCAAAA|2452731|2452821|DEPARTMENT|94|26|Fundamental hands may say enough galleries. Even native hundreds cling however. Good grou|quarterly| +10071|AAAAAAAAHFHCAAAA|2452731|2452821|DEPARTMENT|94|27|Open, public sources cannot make further working, competitive patterns. Present, british painti|quarterly| +10072|AAAAAAAAIFHCAAAA|2452731|2452821|DEPARTMENT|94|28|Natural, english books work operations. Tools must wait too consumers. Too joint failures should ho|quarterly| +10073|AAAAAAAAJFHCAAAA||||||Bad, wet orders get financial, close police; new s|quarterly| +10074|AAAAAAAAKFHCAAAA|2452731|2452821|DEPARTMENT|94|30|Likely arms help. Current, fundamental results raise long-term archit|quarterly| +10075|AAAAAAAALFHCAAAA|2452731|2452821|DEPARTMENT|94|31|Pp. ensure far from dirty, molecular teams; afraid stars criticise everywhere blues. Vast faces use|quarterly| +10076|AAAAAAAAMFHCAAAA|2452731|2452821|DEPARTMENT|94|32|Years must occupy then. Home universal contracts const|quarterly| +10077|AAAAAAAANFHCAAAA|2452731|2452821|DEPARTMENT|94|33|Consequences could not discriminate at a honours. Yesterday assistant stocks shall not|quarterly| +10078|AAAAAAAAOFHCAAAA|2452731|2452821|DEPARTMENT|94|34|Successful others could not bear at first tall players. |quarterly| +10079|AAAAAAAAPFHCAAAA|2452731|2452821|DEPARTMENT||||| +10080|AAAAAAAAAGHCAAAA|2452731|2452821|DEPARTMENT|94|36|Public, judicial numbers might not see however regular possibilities. Conditions can |quarterly| +10081|AAAAAAAABGHCAAAA|2452731|2452821|DEPARTMENT|94|37|Running, commercial wheels see silent animals; conventionally angry authorities generate in a p|quarterly| +10082|AAAAAAAACGHCAAAA|2452731|2452821|DEPARTMENT|94|38|Particular combinations take so long big economies. Local, new merchants should |quarterly| +10083|AAAAAAAADGHCAAAA|2452731|2452821|DEPARTMENT|94|39|Yet old terms worry only political creditors; therefore original titles i|quarterly| +10084|AAAAAAAAEGHCAAAA|2452731|2452821|DEPARTMENT|94|40|As other machines seem just on the boys. Institutions come elaborate sales;|quarterly| +10085|AAAAAAAAFGHCAAAA|2452731|2452821|DEPARTMENT|94|41|Types find clearly other judges. Local services pi|quarterly| +10086|AAAAAAAAGGHCAAAA|2452731|2452821|DEPARTMENT|94|42|Fundamental rules might face even hotels. Minerals must find. Later re|quarterly| +10087|AAAAAAAAHGHCAAAA|2452731|2452821|DEPARTMENT|94|43|Others will find often for a children. Smart parts may tell also local corners. Rounds sta|quarterly| +10088|AAAAAAAAIGHCAAAA|2452731|2452821|DEPARTMENT|94|44|Unacceptable hands control then again different sons; an|quarterly| +10089|AAAAAAAAJGHCAAAA|2452731|2452821|DEPARTMENT|94|45|Alone, commercial experiences must not conform extremely for a practitioners. Satisfactory, indepen|quarterly| +10090|AAAAAAAAKGHCAAAA|2452731|2452821|DEPARTMENT|94|46|About british figures contain rocks. Requirements cope then. Services must lie so as for a minut|quarterly| +10091|AAAAAAAALGHCAAAA|2452731|2452821|DEPARTMENT|94|47|Flat years make so real measures. More nuclear animals might allow as cr|quarterly| +10092|AAAAAAAAMGHCAAAA|2452731|2452821|DEPARTMENT|94|48|Flowers suppose further of course labour jobs; seasons introduce long|quarterly| +10093|AAAAAAAANGHCAAAA|2452731|2452821|DEPARTMENT|94|49|Economic procedures will control in particular deposits. Here true clubs coul|quarterly| +10094|AAAAAAAAOGHCAAAA|2452731|2452821|DEPARTMENT|94|50|Minor stairs race already at least personal tons. Available charges should question fully trees. |quarterly| +10095|AAAAAAAAPGHCAAAA|2452731|2452821|DEPARTMENT|94|51|Due details can lead. Responsible women perform for a clubs; long, theoreti|quarterly| +10096|AAAAAAAAAHHCAAAA|2452731|2452821|DEPARTMENT|94|52|Clinically informal services extract questions. Different, negative circles call just words. Clinic|quarterly| +10097|AAAAAAAABHHCAAAA|2452731|2452821|DEPARTMENT|94|53|New, separate terms find cheap documents; hard arrangements |quarterly| +10098|AAAAAAAACHHCAAAA|2452731|2452821|DEPARTMENT|94|54|Cool other men used to ride; vast, chief weeks leave socia|quarterly| +10099|AAAAAAAADHHCAAAA|2452731|2452821|DEPARTMENT|94|55|Authorities can work in the problems. German men ought to rely|quarterly| +10100|AAAAAAAAEHHCAAAA|2452731|2452821|DEPARTMENT|94|56|Writers hammer respectively substantial, young criteria; other, oppo|quarterly| +10101|AAAAAAAAFHHCAAAA|2452731|2452821|DEPARTMENT|94|57|Christian prices may know behind a partners. Fundamental occupations might see very charac|quarterly| +10102|AAAAAAAAGHHCAAAA|2452731|2452821|DEPARTMENT|94|58|Lights might continue strategic champions. British attitudes live |quarterly| +10103|AAAAAAAAHHHCAAAA|2452731|2452821|DEPARTMENT|94|59|Laws draw unfortunately to a societies; complicated, new hea|quarterly| +10104|AAAAAAAAIHHCAAAA|2452731|2452821|DEPARTMENT|94|60|Versions build more near national years. Enough various f|quarterly| +10105|AAAAAAAAJHHCAAAA|2452731|2452821|DEPARTMENT|94|61|Weaknesses believe earlier high relationships; metres think relatively |quarterly| +10106|AAAAAAAAKHHCAAAA|2452731|2452821|DEPARTMENT|94|62|Large-scale allowances should give. Inadequate feet could dr|quarterly| +10107|AAAAAAAALHHCAAAA|2452731|2452821|DEPARTMENT|94|63|Convincing companies may not compare then wide patients. Red, unlikely businesses should not say j|quarterly| +10108|AAAAAAAAMHHCAAAA|2452731|2452821|DEPARTMENT|94|64|Only options would not find with a championships. Us|quarterly| +10109|AAAAAAAANHHCAAAA|2452731|2452821|DEPARTMENT|94|65|Rural, large surroundings could not consider english flowers. Basic lines sa|quarterly| +10110|AAAAAAAAOHHCAAAA|2452731|2452821|DEPARTMENT|94|66|Previous customers could not bring more; under original vessels may report here acc|quarterly| +10111|AAAAAAAAPHHCAAAA||2452821|||67|Nice, sudden times cost of course foreign democrats. Expensive, various approaches see|| +10112|AAAAAAAAAIHCAAAA|2452731|2452821|DEPARTMENT|94|68|Changes could answer about social things. Patterns|quarterly| +10113|AAAAAAAABIHCAAAA|2452731|2452821|DEPARTMENT|94|69|Japanese years pay apart other groups. Tests change just fair mature lives. So other secrets pay|quarterly| +10114|AAAAAAAACIHCAAAA|2452731|2452821|DEPARTMENT|94|70|Walls preserve parents. Continuous days should reveal knowingly practical students; jus|quarterly| +10115|AAAAAAAADIHCAAAA|2452731|2452821|DEPARTMENT|94|71|Nuclear, legal deposits used to go still later base years. Girls mean specifically |quarterly| +10116|AAAAAAAAEIHCAAAA|2452731|2452821|DEPARTMENT|94|72|Free trials forget international qualifications. Loyal types implement both hence present decis|quarterly| +10117|AAAAAAAAFIHCAAAA|2452731|2452821|DEPARTMENT|94|73|Total targets want then local bonds. Continuous schemes get quick by the priests. Socially engli|quarterly| +10118|AAAAAAAAGIHCAAAA|2452731|2452821|DEPARTMENT|94|74|Sideways ethnic classes release. Conditions introduce lights.|quarterly| +10119|AAAAAAAAHIHCAAAA|2452731|2452821|DEPARTMENT|94|75|Unpleasant, old women veto on a eyes. Policies complete more. Also |quarterly| +10120|AAAAAAAAIIHCAAAA|2452731|2452821|DEPARTMENT|94|76|Years face still. Practical families should not witness by a institu|quarterly| +10121|AAAAAAAAJIHCAAAA|2452731|2452821|DEPARTMENT|94|77|Serious details might not settle surely criminal changes.|quarterly| +10122|AAAAAAAAKIHCAAAA|2452731|2452821|DEPARTMENT|94|78|Jews reclaim young projects; key levels shall save only. Games would not agree enough|quarterly| +10123|AAAAAAAALIHCAAAA|2452731|2452821|DEPARTMENT|94|79|Plans detect then separate conditions; difficult others use most acceptable reven|quarterly| +10124|AAAAAAAAMIHCAAAA|2452731|2452821|DEPARTMENT|94|80|Broad, financial participants should not blow. Also real members produce statements. Perfect |quarterly| +10125|AAAAAAAANIHCAAAA||2452821|DEPARTMENT|94|81|Perfect, likely hours assure once old words. Red, mechanical|| +10126|AAAAAAAAOIHCAAAA|2452731|2452821|DEPARTMENT|94|82|Quite total goods could put at the families. High funny connections can hurt artists|quarterly| +10127|AAAAAAAAPIHCAAAA|2452731|2452821|DEPARTMENT|94|83|Around useful strangers may not know complete, official experiences. Odd, national regulatio|quarterly| +10128|AAAAAAAAAJHCAAAA|2452731|2452821|DEPARTMENT|94|84|Useful, fortunate genes use. Sufficient, territorial cla|quarterly| +10129|AAAAAAAABJHCAAAA|2452731|2452821|DEPARTMENT|94|85|Old, substantial players should not take softly however future area|quarterly| +10130|AAAAAAAACJHCAAAA|2452731|2452821|DEPARTMENT|94|86|Rates might jump tomorrow political, royal topics. Old men build ho|quarterly| +10131|AAAAAAAADJHCAAAA|2452731|2452821|DEPARTMENT|94|87|Ordinary seconds can look well in a ways. Rights might say. Below b|quarterly| +10132|AAAAAAAAEJHCAAAA|2452731|2452821|DEPARTMENT|94|88|Occasions pay alike major changes. Already occupational poets leave otherwise o|quarterly| +10133|AAAAAAAAFJHCAAAA|2452731|2452821|DEPARTMENT|94|89|As clean relations match straight old perceptions. Bad offices say suddenl|quarterly| +10134|AAAAAAAAGJHCAAAA|2452731|2452821|DEPARTMENT|94|90|Names could not contact so only, quick changes. Children must not go in a years; young|quarterly| +10135|AAAAAAAAHJHCAAAA|2452731|2452821|DEPARTMENT|94|91|Then used years would go further animals. Figures shall not change enough |quarterly| +10136|AAAAAAAAIJHCAAAA|2452731|2452821|DEPARTMENT|94|92|Liberal factors move. Tons will not pay. Big payments per|quarterly| +10137|AAAAAAAAJJHCAAAA|2452731|2452821|DEPARTMENT|94|93|Running things might not bolster seemingly environmental, relevant years. Great |quarterly| +10138|AAAAAAAAKJHCAAAA|2452731|2452821|DEPARTMENT|94|94|Enthusiastic, white degrees might see important, imperial reasons. |quarterly| +10139|AAAAAAAALJHCAAAA|2452731|2452821|DEPARTMENT|94|95|Rules exceed major others. Patterns run. New, useless levels spend then true facts. Prime, famo|quarterly| +10140|AAAAAAAAMJHCAAAA|2452731|2452821|DEPARTMENT|94|96|Long, key copies balance drugs. Complex eyes can look months. Services reach to a mus|quarterly| +10141|AAAAAAAANJHCAAAA||2452821|DEPARTMENT|94||Options ought to waste around nice years; trees can remedy now local managers. Cha|| +10142|AAAAAAAAOJHCAAAA|2452731|2452821|DEPARTMENT|94|98|Other employers say. Large-scale languages move trees. Great, toxic stat|quarterly| +10143|AAAAAAAAPJHCAAAA|2452731|2452821|DEPARTMENT|94|99|Parents explain of course. Very black parts shall progress deeper internatio|quarterly| +10144|AAAAAAAAAKHCAAAA|2452731|2452821|DEPARTMENT|94|100|Adults may cover less clear views. Very simple articles take above |quarterly| +10145|AAAAAAAABKHCAAAA|2452731|2452821|DEPARTMENT|94|101|Other, fair books will know local, short-term muscles. High techniques |quarterly| +10146|AAAAAAAACKHCAAAA|2452731|2452821|DEPARTMENT|94|102|Payments shall not want needs. Here strategic ties will |quarterly| +10147|AAAAAAAADKHCAAAA|2452731|2452821|DEPARTMENT|94|103|Little flowers merge either. Always political plants make regular things. Con|quarterly| +10148|AAAAAAAAEKHCAAAA|2452731|2452821|DEPARTMENT|94|104|Very particular patients avoid tired reasons. Men see e|quarterly| +10149|AAAAAAAAFKHCAAAA|2452731|2452821|DEPARTMENT|94|105|Free, alternative architects would get doors. Schemes should|quarterly| +10150|AAAAAAAAGKHCAAAA|2452731|2452821|DEPARTMENT|94|106|United powers climb about possible rats. Supplies meet nowhere. Still national residents ta|quarterly| +10151|AAAAAAAAHKHCAAAA|2452731|2452821|DEPARTMENT|94|107|At all big operations fit emotional cities. Private others would not know materi|quarterly| +10152|AAAAAAAAIKHCAAAA|2452731|2452821|DEPARTMENT|94|108|Differences dare other, worried roads. Again good points will not c|quarterly| +10153|AAAAAAAAJKHCAAAA|2452822|2452912|DEPARTMENT|95|1|Big grounds forgive arms. Equal, brown figures could |quarterly| +10154|AAAAAAAAKKHCAAAA|2452822|2452912|DEPARTMENT|95|2|Social, supreme banks could believe. Annual ground|quarterly| +10155|AAAAAAAALKHCAAAA|||DEPARTMENT||3||| +10156|AAAAAAAAMKHCAAAA|2452822|2452912|DEPARTMENT|95|4|Applicants would not buy; then important powers shall find away clear, economic eye|quarterly| +10157|AAAAAAAANKHCAAAA|2452822|2452912|DEPARTMENT|95|5|Primary, lucky organizations could turn national, great maps. Virtual|quarterly| +10158|AAAAAAAAOKHCAAAA|2452822|2452912|DEPARTMENT|95|6|Traditional, polite intervals put still to the acres. Democratic types may work ideal names. |quarterly| +10159|AAAAAAAAPKHCAAAA|2452822|2452912|DEPARTMENT|95|7|Old, roman decades provide masses. French citizens o|quarterly| +10160|AAAAAAAAALHCAAAA|2452822|2452912|DEPARTMENT|95|8|So new instances sleep yet to a professionals. Complex, legal customers wou|quarterly| +10161|AAAAAAAABLHCAAAA|2452822|2452912|DEPARTMENT|95|9|Visual, dependent results would not think also advanced supp|quarterly| +10162|AAAAAAAACLHCAAAA|2452822|2452912|DEPARTMENT|95|10|Surprised, gay children get industrial, sure ways. True days measu|quarterly| +10163|AAAAAAAADLHCAAAA|2452822|2452912|DEPARTMENT|95|11|Australian, old women must not prove. Corporate, new sheets appreciate. Awful blocks see likely, su|quarterly| +10164|AAAAAAAAELHCAAAA|2452822|2452912|DEPARTMENT|95|12|Now maximum cells start there bad, positive problems; able, original trials would not t|quarterly| +10165|AAAAAAAAFLHCAAAA|2452822|2452912|DEPARTMENT|95|13|Wrong games should communicate then. Camps explain always finan|quarterly| +10166|AAAAAAAAGLHCAAAA|2452822|2452912|DEPARTMENT|95|14|Bad sources used to come proudly. However other boys place even friends. Now hostile issues a|quarterly| +10167|AAAAAAAAHLHCAAAA|2452822|2452912|DEPARTMENT|95|15|No longer front injuries break on the weeks. Just, material mate|quarterly| +10168|AAAAAAAAILHCAAAA|2452822|2452912|DEPARTMENT|95|16|Wines depend only interests. Related, economic countries can claim short, new offices|quarterly| +10169|AAAAAAAAJLHCAAAA|2452822|2452912|DEPARTMENT|95|17|Due patients discuss ready hands. Arguments see never. Frien|quarterly| +10170|AAAAAAAAKLHCAAAA|2452822|2452912|DEPARTMENT|95|18|Different, new members would sleep originally head firms; well little volunteers |quarterly| +10171|AAAAAAAALLHCAAAA|2452822|2452912|DEPARTMENT|95|19|National, neutral pieces use bloody to a players. Now high years go men. Also impossible men ma|quarterly| +10172|AAAAAAAAMLHCAAAA|2452822|2452912|DEPARTMENT|95|20|Specific, other designs understand annual, free times. Tough, chinese workers find in the are|quarterly| +10173|AAAAAAAANLHCAAAA|2452822|2452912|DEPARTMENT|95|21|However economic homes go main, different rates; approaches acco|quarterly| +10174|AAAAAAAAOLHCAAAA|2452822|2452912|DEPARTMENT|95|22|Events repeat exactly dear, clear children. Northe|quarterly| +10175|AAAAAAAAPLHCAAAA|2452822|2452912|DEPARTMENT|95|23|Real drugs may go. Little, correct costs might not grow out of a things. Players w|quarterly| +10176|AAAAAAAAAMHCAAAA|2452822|2452912|DEPARTMENT|95|24|Appropriate settings mind poor individuals. Winners allow to a secrets. Free, english affairs|quarterly| +10177|AAAAAAAABMHCAAAA|2452822|2452912|DEPARTMENT|95|25|Forever natural funds share hours. Yesterday normal cl|quarterly| +10178|AAAAAAAACMHCAAAA|2452822|2452912|DEPARTMENT|95|26|Loans could not act inevitably religious, public rooms; acco|quarterly| +10179|AAAAAAAADMHCAAAA|2452822|2452912|DEPARTMENT|95|27|Confident, good pages get easy. Customers will not consid|quarterly| +10180|AAAAAAAAEMHCAAAA|2452822|2452912|DEPARTMENT|95|28|Whole, constant questions read in a times; lines ban thus successful recommendations. A little to|quarterly| +10181|AAAAAAAAFMHCAAAA|2452822|2452912|DEPARTMENT|95|29|Patterns meet indeed full books. Gradually fresh agents cost so critically public detail|quarterly| +10182|AAAAAAAAGMHCAAAA|2452822|2452912|DEPARTMENT|95|30|Unhappy instructions will not prefer only valuable interests. Hopes res|quarterly| +10183|AAAAAAAAHMHCAAAA|2452822|2452912|DEPARTMENT|95|31|Other techniques go again friends. More other activities pay in a values. Parties shall cover twi|quarterly| +10184|AAAAAAAAIMHCAAAA|2452822|2452912|DEPARTMENT|95|32|Briefly different communications work. More sad jou|quarterly| +10185|AAAAAAAAJMHCAAAA|2452822|2452912|DEPARTMENT|95|33|Local languages need issues. Still, likely parents cannot r|quarterly| +10186|AAAAAAAAKMHCAAAA|2452822|2452912|DEPARTMENT|95|34|Annual, gross adults rely from a guidelines. Resident|quarterly| +10187|AAAAAAAALMHCAAAA|2452822|2452912|DEPARTMENT|95|35|Good, wrong bonds must not go around complicated, r|quarterly| +10188|AAAAAAAAMMHCAAAA|2452822|2452912|DEPARTMENT|95|36|Irish towns take wrong, nuclear eyes; exactly new supplies sit.|quarterly| +10189|AAAAAAAANMHCAAAA|2452822|2452912|DEPARTMENT|95|37|Solid companies live major areas. Big, proper circles might save really long mana|quarterly| +10190|AAAAAAAAOMHCAAAA|2452822|2452912|DEPARTMENT|95|38|Theories used to attribute for example. Reasonable, traditional courses can take after a polit|quarterly| +10191|AAAAAAAAPMHCAAAA|2452822|2452912|DEPARTMENT|95|39|Happy days help; old, local hours call. Environmental, military individ|quarterly| +10192|AAAAAAAAANHCAAAA|2452822|2452912|DEPARTMENT|95|40|Major books discuss southern, good problems. Serious, rational countries could av|quarterly| +10193|AAAAAAAABNHCAAAA|2452822|2452912|DEPARTMENT|95|41|Complex days learn also material discussions. Considerable sec|quarterly| +10194|AAAAAAAACNHCAAAA|2452822|2452912|DEPARTMENT|95|42|Additional reasons should not pick formerly in a men. Angry ch|quarterly| +10195|AAAAAAAADNHCAAAA|2452822|2452912|DEPARTMENT|95|43|Black foods must run gradually for a cases. Young, ne|quarterly| +10196|AAAAAAAAENHCAAAA|2452822|2452912|DEPARTMENT|95|44|Modern, hot incidents might wait tightly. Advantages must pay da|quarterly| +10197|AAAAAAAAFNHCAAAA|2452822|2452912|DEPARTMENT|95|45|Much select words may not mean no longer. Less far ey|quarterly| +10198|AAAAAAAAGNHCAAAA|2452822|2452912|DEPARTMENT|95|46|Hard, functional games encourage now. Relations come greatly repeatedly eastern reserv|quarterly| +10199|AAAAAAAAHNHCAAAA|2452822|2452912|DEPARTMENT|95|47|Legs run workers. Holidays cannot stand separate, old wee|quarterly| +10200|AAAAAAAAINHCAAAA|2452822|2452912|DEPARTMENT|95|48|Empty, young companies should form closely only whi|quarterly| +10201|AAAAAAAAJNHCAAAA|2452822|2452912|DEPARTMENT|95|49|Nuclear, hard organizations contact easy. Only careful procedures q|quarterly| +10202|AAAAAAAAKNHCAAAA|2452822|2452912|DEPARTMENT|95|50|Necessary, well-known methods could not discount old,|quarterly| +10203|AAAAAAAALNHCAAAA|2452822|2452912|DEPARTMENT|95|51|Subsequent, black miles shall not work more members; remains continue popular, great wo|quarterly| +10204|AAAAAAAAMNHCAAAA|2452822|2452912|DEPARTMENT|95|52|Scientific items come gifts. Other deals should receive mainly current res|quarterly| +10205|AAAAAAAANNHCAAAA|2452822|2452912|DEPARTMENT|95|53|Trends could see fortunately commercial rates. Traditionally disastrous|quarterly| +10206|AAAAAAAAONHCAAAA|2452822|2452912|DEPARTMENT|95|54|Payments use once single services. Regional, electoral women shou|quarterly| +10207|AAAAAAAAPNHCAAAA|2452822|2452912|DEPARTMENT|95|55|Common, great notes might not go all on a customers; all painful th|quarterly| +10208|AAAAAAAAAOHCAAAA|2452822|2452912|DEPARTMENT|95|56|Good experiences must make by now significantly great systems. Consequences co|quarterly| +10209|AAAAAAAABOHCAAAA|2452822|2452912|DEPARTMENT|95|57|Tender, possible marks could not come right direct controls. In particular sick |quarterly| +10210|AAAAAAAACOHCAAAA|2452822|2452912|DEPARTMENT|95|58|Roughly logical expectations could develop from a rights. Bizarre levels could explo|quarterly| +10211|AAAAAAAADOHCAAAA|2452822|2452912|DEPARTMENT|95|59|Years walk very to a courses. For example front prices give |quarterly| +10212|AAAAAAAAEOHCAAAA|2452822|2452912|DEPARTMENT|95|60|Long dramatic others used to print. Major, young friends ought to suppo|quarterly| +10213|AAAAAAAAFOHCAAAA|2452822|2452912|DEPARTMENT|95|61|Just statistical agencies shall not send european services. Irish, r|quarterly| +10214|AAAAAAAAGOHCAAAA|2452822|2452912|DEPARTMENT|95|62|Old systems hope other songs. Local, soviet increase|quarterly| +10215|AAAAAAAAHOHCAAAA|2452822|2452912|DEPARTMENT|95|63|Empty, useful claims modify completely. Professional, light p|quarterly| +10216|AAAAAAAAIOHCAAAA|2452822|2452912|DEPARTMENT|95|64|Current lists used to reveal. Ingredients withdraw in an n|quarterly| +10217|AAAAAAAAJOHCAAAA|2452822|2452912|DEPARTMENT|95|65|Video-taped pupils must read continuous bodies; practical,|quarterly| +10218|AAAAAAAAKOHCAAAA|2452822|2452912|DEPARTMENT|95|66|Too heavy changes used to talk very as pure results.|quarterly| +10219|AAAAAAAALOHCAAAA|2452822|2452912|DEPARTMENT|95|67|Old, good inches prevent incidents; other, single fortunes will feel for a troubles|quarterly| +10220|AAAAAAAAMOHCAAAA|2452822|2452912|DEPARTMENT|95|68|Electric managers ought to track intensely necessarily careful matters. Very essential scots swing |quarterly| +10221|AAAAAAAANOHCAAAA|2452822|2452912|DEPARTMENT|95|69|Regional, brown variations support; important classes|quarterly| +10222|AAAAAAAAOOHCAAAA|2452822|2452912|DEPARTMENT|95|70|Dark women make personally unable, prime organisations. U|quarterly| +10223|AAAAAAAAPOHCAAAA|2452822|2452912|DEPARTMENT|95|71|Companies see too criminal, high changes; times live new, european pupils; good g|quarterly| +10224|AAAAAAAAAPHCAAAA|2452822|2452912|DEPARTMENT|95|72|Fans will complain with the records. Governments carry much chi|quarterly| +10225|AAAAAAAABPHCAAAA|2452822|2452912|DEPARTMENT|95|73|Typically substantial values shall close enough other small waves. Co|quarterly| +10226|AAAAAAAACPHCAAAA|2452822|2452912|DEPARTMENT|95|74|Normally huge weeks will not favour others. All considerabl|quarterly| +10227|AAAAAAAADPHCAAAA|2452822|2452912|DEPARTMENT|95|75|Hands could not match quite negative methods. Awards move local, old rocks. Both|quarterly| +10228|AAAAAAAAEPHCAAAA|2452822|2452912|DEPARTMENT|95|76|Polite questions should set already small women. Current men make in a |quarterly| +10229|AAAAAAAAFPHCAAAA|2452822|2452912|DEPARTMENT|95|77|Financial things resolve also. There formal members tap years. Local, happy women score |quarterly| +10230|AAAAAAAAGPHCAAAA|2452822|2452912|DEPARTMENT|95|78|Blue, good songs shall become lightly. Specific workers prove originally c|quarterly| +10231|AAAAAAAAHPHCAAAA|2452822|2452912|DEPARTMENT|95|79|National authors achieve importantly whole, far nations. Less different others could|quarterly| +10232|AAAAAAAAIPHCAAAA|2452822|2452912|DEPARTMENT|95|80|Politically poor folk fly worse. Full, monetary find|quarterly| +10233|AAAAAAAAJPHCAAAA|2452822|2452912|DEPARTMENT|95|81|Weeks satisfy tomorrow wild things. Hands care other, great others; liabi|quarterly| +10234|AAAAAAAAKPHCAAAA|2452822|2452912|DEPARTMENT|95|82|Components could not co-ordinate never areas. Rela|quarterly| +10235|AAAAAAAALPHCAAAA|2452822|2452912|DEPARTMENT|95|83|Alternative, small weeks would not occur spectacular firms. Under wa|quarterly| +10236|AAAAAAAAMPHCAAAA|2452822|2452912|DEPARTMENT|95|84|Long, available sisters find. Good measures should n|quarterly| +10237|AAAAAAAANPHCAAAA|2452822|2452912|DEPARTMENT|95|85|Words determine at once correct, musical systems. Views must publish|quarterly| +10238|AAAAAAAAOPHCAAAA|2452822|2452912|DEPARTMENT|95|86|Intensive women promote later from a drugs. Other engines used to fill gm|quarterly| +10239|AAAAAAAAPPHCAAAA|2452822|2452912|DEPARTMENT|95|87|Particularly foreign standards begin holy boundaries; suddenly good roads tell buildings. Facilitie|quarterly| +10240|AAAAAAAAAAICAAAA|2452822|2452912|DEPARTMENT|95|88|Young years should deal later plus a emotions. Things fetch only, plain events. At leas|quarterly| +10241|AAAAAAAABAICAAAA|2452822|2452912|DEPARTMENT|95|89|Unpleasant, grim transactions ought to act long high cities. Furth|quarterly| +10242|AAAAAAAACAICAAAA|2452822|2452912|DEPARTMENT|95|90|Sure muscles will initiate almost; islands burst t|quarterly| +10243|AAAAAAAADAICAAAA|2452822|2452912|DEPARTMENT|95|91|Inside commercial arts must help for a needs. Companies ought to m|quarterly| +10244|AAAAAAAAEAICAAAA|2452822|2452912|DEPARTMENT|95|92|Common methods may not say other patients; simple arms could not inh|quarterly| +10245|AAAAAAAAFAICAAAA|2452822|2452912|DEPARTMENT|95|93|Pretty papers endure really protective, active employees. Steadily assistant measures will|quarterly| +10246|AAAAAAAAGAICAAAA|2452822|2452912|DEPARTMENT|95|94|Potential, great conditions get by the tests; barely domestic hands ma|quarterly| +10247|AAAAAAAAHAICAAAA|2452822|2452912|DEPARTMENT|95|95|More british mothers depart just different orders. Areas save scient|quarterly| +10248|AAAAAAAAIAICAAAA|2452822|2452912|DEPARTMENT|95|96|Hands will not teach much human players. Minor, continuous facts ough|quarterly| +10249|AAAAAAAAJAICAAAA|2452822|2452912|DEPARTMENT|95|97|Most public years describe to a tables. Metres will make somew|quarterly| +10250|AAAAAAAAKAICAAAA|2452822|2452912|DEPARTMENT|95|98|Usually long details base divine, environmental dimensions. Reports can co-ordinate now on t|quarterly| +10251|AAAAAAAALAICAAAA|2452822|2452912|DEPARTMENT|95|99|Primary successes identify now just successful facts. Realistically considerable off|quarterly| +10252|AAAAAAAAMAICAAAA|2452822|2452912|DEPARTMENT|95|100|Clearly suspicious leaders could travel as long-term|quarterly| +10253|AAAAAAAANAICAAAA|||||||| +10254|AAAAAAAAOAICAAAA|2452822|2452912|DEPARTMENT|95|102|Video-taped, good feet would help warm illustrations. Far goods hold cool, social |quarterly| +10255|AAAAAAAAPAICAAAA|2452822|2452912|DEPARTMENT|95|103|Certainly temporary sections get never new ideas. Elaborate inches could notice sp|quarterly| +10256|AAAAAAAAABICAAAA|2452822|2452912|DEPARTMENT|95|104|Finally correct relations should not afford also. Special scientists record r|quarterly| +10257|AAAAAAAABBICAAAA|2452822|2452912|DEPARTMENT|95|105|Just available hands leave for a problems. Perhaps social minds stop however from the companies. Sh|quarterly| +10258|AAAAAAAACBICAAAA|2452822|2452912|DEPARTMENT|95|106|Cities operate only. Witnesses learn better necessary, very experts. International |quarterly| +10259|AAAAAAAADBICAAAA|2452822|2452912|DEPARTMENT|95|107|Further geographical pupils shall not postpone unlikely, criminal building|quarterly| +10260|AAAAAAAAEBICAAAA|2452822|2452912|DEPARTMENT|95|108|Good branches must not make to a stages. Games test. Small,|quarterly| +10261|AAAAAAAAFBICAAAA|2452913|2453003|DEPARTMENT|96|1|Recent foods might go frequently universities. Arbitrary papers discover about inste|quarterly| +10262|AAAAAAAAGBICAAAA|2452913|2453003|DEPARTMENT|96|2|Regular, occupational relations can reckon feet. Over alone mines put other, intelligent|quarterly| +10263|AAAAAAAAHBICAAAA|2452913|2453003|DEPARTMENT|96|3|Names may express. Big, able structures tell reforms; yet fascinating forms allow for a pla|quarterly| +10264|AAAAAAAAIBICAAAA|2452913|2453003|DEPARTMENT|96|4|Governments help then important, firm letters. New details shall sell primary, new years|quarterly| +10265|AAAAAAAAJBICAAAA|2452913|2453003|DEPARTMENT|96|5|Unique factors swim presently years; doubtless other views would not cease perhaps new lev|quarterly| +10266|AAAAAAAAKBICAAAA|2452913|2453003|DEPARTMENT|96|6|Impressive beds join away. High blocks help indeed tomorrow future miles. Complete, sexual steps co|quarterly| +10267|AAAAAAAALBICAAAA|2452913|2453003|DEPARTMENT|96|7|Right, remote children could not build about in a organisms. Angry, |quarterly| +10268|AAAAAAAAMBICAAAA|2452913|2453003|DEPARTMENT|96|8|Early similar games must not tell invariably in a techniques. Ill running leaders should go|quarterly| +10269|AAAAAAAANBICAAAA|2452913|2453003|DEPARTMENT|96|9|Final experiments stand appropriate names. Considerable, easy children lea|quarterly| +10270|AAAAAAAAOBICAAAA|2452913|2453003|DEPARTMENT|96|10|Criteria keep sympathetically provisions. Harsh, environmental others cry fo|quarterly| +10271|AAAAAAAAPBICAAAA|2452913|2453003|DEPARTMENT|96|11|Regions go main, busy budgets. Absolute, deaf beings might hold|quarterly| +10272|AAAAAAAAACICAAAA|2452913|2453003|DEPARTMENT|96|12|Nuclear schools will supply white, general mountains. |quarterly| +10273|AAAAAAAABCICAAAA|2452913|2453003|DEPARTMENT|96|13|National signals can match pleasant revenues. Children must not c|quarterly| +10274|AAAAAAAACCICAAAA|2452913|2453003|DEPARTMENT|96|14|Conclusions ought to prevent now much initial years. Glorious, fixed managers clean o|quarterly| +10275|AAAAAAAADCICAAAA|2452913|2453003|DEPARTMENT|96|15|Old observers might wait soon candidates; working miles cannot turn for the objecti|quarterly| +10276|AAAAAAAAECICAAAA|2452913|2453003|DEPARTMENT|96|16|Mediterranean groups used to know now profitable, contemporary quest|quarterly| +10277|AAAAAAAAFCICAAAA|2452913|2453003|DEPARTMENT|96|17|Tests leave expensive details. Levels understand then bad thanks. Small solutions will guess paral|quarterly| +10278|AAAAAAAAGCICAAAA|2452913|2453003|DEPARTMENT|96|18|Bitter, likely glasses should not believe ever again oth|quarterly| +10279|AAAAAAAAHCICAAAA|2452913|2453003|DEPARTMENT|96|19|Enterprises might not sit later in a powers. High, necessary accounts woul|quarterly| +10280|AAAAAAAAICICAAAA|2452913|2453003|DEPARTMENT|96|20|Russians let to a shows. Loans apply anyway at a instances. Pr|quarterly| +10281|AAAAAAAAJCICAAAA|2452913|2453003|DEPARTMENT|96|21|Isolated amounts pick thereafter possible, insufficient communications|quarterly| +10282|AAAAAAAAKCICAAAA|2452913|2453003|DEPARTMENT|96|22|Yet high requirements get early from a professionals. Commonly poli|quarterly| +10283|AAAAAAAALCICAAAA|2452913|2453003|DEPARTMENT|96|23|Subsequent movements get very difficult leaves. Pop|quarterly| +10284|AAAAAAAAMCICAAAA|2452913|2453003|DEPARTMENT|96|24|Aspects used to target then happy, possible pieces; bad materials work participants|quarterly| +10285|AAAAAAAANCICAAAA|2452913|2453003|DEPARTMENT|96|25|Effects come too highest important years. Steps could not arrange in|quarterly| +10286|AAAAAAAAOCICAAAA|2452913|2453003|DEPARTMENT|96|26|Divisions will get about skills. Common, positive corners shall discover just english bodies. R|quarterly| +10287|AAAAAAAAPCICAAAA|2452913|2453003|DEPARTMENT|96|27|Dirty, royal men ought to pretend both with a councillors. Girls get techniques. Needs could not fa|quarterly| +10288|AAAAAAAAADICAAAA|2452913|2453003|DEPARTMENT|96|28|So identical girls must dismiss nationally expensive important students. Things could repo|quarterly| +10289|AAAAAAAABDICAAAA|2452913|2453003|DEPARTMENT|96|29|Important, real miles build also actually existing areas. Narrow, excellent investors interpre|quarterly| +10290|AAAAAAAACDICAAAA|2452913|2453003|DEPARTMENT|96|30|Sufficiently central materials take top, only ideas. Working, different sal|quarterly| +10291|AAAAAAAADDICAAAA|2452913|2453003|DEPARTMENT|96|31|Especially early sections must involve maximum, american shops. Sexual workers use favou|quarterly| +10292|AAAAAAAAEDICAAAA|2452913|2453003|DEPARTMENT|96|32|Originally natural years must forgive now cheeks. Good, long modules will ensure later traditional|quarterly| +10293|AAAAAAAAFDICAAAA|2452913|2453003|DEPARTMENT|96|33|Active, boring teams would not get most weak, political busines|quarterly| +10294|AAAAAAAAGDICAAAA|2452913|2453003|DEPARTMENT|96|34|Financial categories shall not lose. Other structures will grow ever even unemployed p|quarterly| +10295|AAAAAAAAHDICAAAA||2453003|||35|Tory managers say there to a politicians. Courts hear very with a approaches. Defensive ways sho|quarterly| +10296|AAAAAAAAIDICAAAA|2452913|2453003|DEPARTMENT|96|36|Past rules will delay regions; supplies give just |quarterly| +10297|AAAAAAAAJDICAAAA|2452913|2453003|DEPARTMENT|96|37|More other stones will kill by a forms. Real properties may contac|quarterly| +10298|AAAAAAAAKDICAAAA|2452913|2453003|DEPARTMENT|96|38|Women defend only public seconds. Quickly sharp rates must happ|quarterly| +10299|AAAAAAAALDICAAAA|2452913|2453003|DEPARTMENT|96|39|Expensive weeks could recover possibly firms. Monthly, regular authorities make oc|quarterly| +10300|AAAAAAAAMDICAAAA|2452913|2453003|DEPARTMENT|96|40|Other tasks earn then massive sources. Main, new books might not take for exa|quarterly| +10301|AAAAAAAANDICAAAA|2452913|2453003|DEPARTMENT|96|41|Products override also potatoes. Reasons walk too good trees. Absolutely human child|quarterly| +10302|AAAAAAAAODICAAAA|2452913|2453003|DEPARTMENT|96|42|Just difficult users hold specifically labour days. Universities might reso|quarterly| +10303|AAAAAAAAPDICAAAA|2452913||DEPARTMENT|96|43|Elderly proposals will test absolute women; slow rights need for example prima|quarterly| +10304|AAAAAAAAAEICAAAA|2452913|2453003|DEPARTMENT|96|44|Properties will head hard good, amazing years. New, little activities ought to |quarterly| +10305|AAAAAAAABEICAAAA|2452913|2453003|DEPARTMENT|96|45|Complete applications recognize great lines. General, str|quarterly| +10306|AAAAAAAACEICAAAA|2452913|2453003|DEPARTMENT|96|46|Then large metals shall drive otherwise unable, slow market|quarterly| +10307|AAAAAAAADEICAAAA|2452913|2453003|DEPARTMENT|96|47|Significant studies get on a movements. Essential exercises may prove. For example sec|quarterly| +10308|AAAAAAAAEEICAAAA|2452913|2453003|DEPARTMENT|96|48|Issues threaten at a concepts. Widely eastern guests will start often sorry occupatio|quarterly| +10309|AAAAAAAAFEICAAAA|2452913|2453003|DEPARTMENT|96|49|Seats could not recover parts. Large others used to happen as above the names. Almost willing win|quarterly| +10310|AAAAAAAAGEICAAAA|2452913|2453003|DEPARTMENT|96|50|Less appropriate groups pay tories. Places make at first eyes; quietly nuclear relation|quarterly| +10311|AAAAAAAAHEICAAAA|2452913|2453003|DEPARTMENT|96|51|Never visual examples used to develop about now organic names; above contracts ought|quarterly| +10312|AAAAAAAAIEICAAAA|2452913|2453003|DEPARTMENT|96|52|Walls remain almost single, full readers; social approaches will say enough eld|quarterly| +10313|AAAAAAAAJEICAAAA|2452913|2453003|DEPARTMENT|96|53|Huge standards might cost hard courts. Regional, great barrie|quarterly| +10314|AAAAAAAAKEICAAAA|2452913|2453003|DEPARTMENT|96|54|Incredible months avoid further comparative details. New qualifications close normally years; p|quarterly| +10315|AAAAAAAALEICAAAA|2452913|2453003|DEPARTMENT|96|55|Especially steady experiences used to form sports. Acceptable, formal customer|quarterly| +10316|AAAAAAAAMEICAAAA|2452913|2453003|DEPARTMENT|96|56|Appropriate, final children betray often. Ready, pleased grounds think new, great colle|quarterly| +10317|AAAAAAAANEICAAAA|2452913|2453003|DEPARTMENT|96|57|New generations should move forward enthusiastic, thin shares; fully fine difficulties mo|quarterly| +10318|AAAAAAAAOEICAAAA|2452913|2453003|DEPARTMENT|96|58|Classes shall get much. Likely, parliamentary sectors attend far in|quarterly| +10319|AAAAAAAAPEICAAAA|2452913|2453003|DEPARTMENT|96|59|Then new women could think however too available times. Eyes used to study broadly ancient, workin|quarterly| +10320|AAAAAAAAAFICAAAA|2452913|2453003|DEPARTMENT|96|60|Reluctantly keen centres might sing poorly primary factors. Video-taped|quarterly| +10321|AAAAAAAABFICAAAA|2452913|2453003|DEPARTMENT|96|61|Matters can seem so late proteins. Automatic friends |quarterly| +10322|AAAAAAAACFICAAAA|2452913|2453003|DEPARTMENT|96|62|Services see. Widespread minutes might let relatively char|quarterly| +10323|AAAAAAAADFICAAAA|2452913|2453003|DEPARTMENT|96|63|Original prices will double in a pictures; physical, vari|quarterly| +10324|AAAAAAAAEFICAAAA|2452913|2453003|DEPARTMENT|96|64|Financial copies ought to make rightly. Good styles cannot boos|quarterly| +10325|AAAAAAAAFFICAAAA|2452913|2453003|DEPARTMENT|96|65|Signs make also with the frames. Able, scientific meet|quarterly| +10326|AAAAAAAAGFICAAAA|2452913|2453003|DEPARTMENT|96|66|Mediterranean, eventual states used to win well literary girls. Proble|quarterly| +10327|AAAAAAAAHFICAAAA|2452913|2453003|DEPARTMENT|96|67|Able, other trees see real, important options. At least british spirits|quarterly| +10328|AAAAAAAAIFICAAAA|2452913|2453003|DEPARTMENT|96|68|Members could not make quickly popular objects. Pr|quarterly| +10329|AAAAAAAAJFICAAAA|2452913|2453003|DEPARTMENT|96|69|Both labour hills could join leading values. Clean sheets disguise for example in the word|quarterly| +10330|AAAAAAAAKFICAAAA|2452913|2453003|DEPARTMENT|96|70|Expected, pleasant seconds break to a criteria. Old projects reduc|quarterly| +10331|AAAAAAAALFICAAAA|2452913|2453003|DEPARTMENT|96|71|At all western matches meet yet straight representative|quarterly| +10332|AAAAAAAAMFICAAAA|2452913|2453003|DEPARTMENT|96|72|Issues cannot meet to the surveys. Human rights mi|quarterly| +10333|AAAAAAAANFICAAAA|2452913|2453003|DEPARTMENT|96|73|Fragments shall buy all main films; implications shall conduct |quarterly| +10334|AAAAAAAAOFICAAAA|2452913|2453003|DEPARTMENT|96|74|Real, external women would tell once there ancient attitudes. Of|quarterly| +10335|AAAAAAAAPFICAAAA|2452913|2453003|DEPARTMENT|96|75|Sexual, black theories return. Sports identify. Now good procedures m|quarterly| +10336|AAAAAAAAAGICAAAA|2452913|2453003|DEPARTMENT|96|76|Meals used to help backwards red lakes. Future, con|quarterly| +10337|AAAAAAAABGICAAAA|2452913|2453003|DEPARTMENT|96|77|Issues used to pay broad courses. Then symbolic jobs must not|quarterly| +10338|AAAAAAAACGICAAAA|2452913|2453003|DEPARTMENT|96|78|General, simple options talk. Huge, united hours kill entries. Similar, valid children play fine, |quarterly| +10339|AAAAAAAADGICAAAA|2452913|2453003|DEPARTMENT|96|79|Grudgingly possible offences must expect also systems|quarterly| +10340|AAAAAAAAEGICAAAA|2452913|2453003|DEPARTMENT|96|80|Colleges would suppose in addition hot meetings. Expecta|quarterly| +10341|AAAAAAAAFGICAAAA|2452913|2453003|DEPARTMENT|96|81|Numbers slip; only military police used to get surely parties. Possible, large directions |quarterly| +10342|AAAAAAAAGGICAAAA|2452913|2453003|DEPARTMENT|96|82|Representatives substitute on a investments. Political, |quarterly| +10343|AAAAAAAAHGICAAAA|2452913|2453003|DEPARTMENT|96|83|Organisations get however new, accessible taxes. Simple arms must get helpful, i|quarterly| +10344|AAAAAAAAIGICAAAA|2452913|2453003|DEPARTMENT|96|84|Sides give in a decisions. New, ancient boats let. Animals should not mak|quarterly| +10345|AAAAAAAAJGICAAAA|2452913|2453003|DEPARTMENT|96|85|Courses become into a studies; here british companie|quarterly| +10346|AAAAAAAAKGICAAAA|2452913|2453003|DEPARTMENT|96|86|Companies must get from a practices. Psychiatric, old organs wi|quarterly| +10347|AAAAAAAALGICAAAA|2452913|2453003|DEPARTMENT|96|87|Free, overall schools suggest also ordinary mountains. Discreetly arab benefits believe again |quarterly| +10348|AAAAAAAAMGICAAAA|2452913|2453003|DEPARTMENT|96|88|Thoroughly royal police must analyse rich kinds. Muscles defend entirely under a terms|quarterly| +10349|AAAAAAAANGICAAAA|2452913|2453003|DEPARTMENT|96|89|Ill recent teams used to visit eventually black reforms. Conscious, public visit|quarterly| +10350|AAAAAAAAOGICAAAA|2452913|2453003|DEPARTMENT|96|90|Previous, chosen children answer usually more individual allies. Agreements focus patien|quarterly| +10351|AAAAAAAAPGICAAAA|2452913|2453003|DEPARTMENT|96|91|Essential, disciplinary products should not like eventually sufficient writers. Stron|quarterly| +10352|AAAAAAAAAHICAAAA|2452913|2453003|DEPARTMENT|96|92|Dimensions achieve old, likely days; easy, other twins |quarterly| +10353|AAAAAAAABHICAAAA|2452913|2453003|DEPARTMENT|96|93|Other, royal members remove finally items. No longer necessary terms wil|quarterly| +10354|AAAAAAAACHICAAAA|2452913|2453003|DEPARTMENT|96|94|Dear teams conduct seriously. Actually crucial records cannot bring affai|quarterly| +10355|AAAAAAAADHICAAAA|2452913|2453003|DEPARTMENT|96|95|Weeks must look neither blocks. Colours would motiv|quarterly| +10356|AAAAAAAAEHICAAAA|2452913|2453003|DEPARTMENT|96|96|Dogs could light sternly. By now sick lines could use arbitrary trials. Firstly happy s|quarterly| +10357|AAAAAAAAFHICAAAA|2452913|2453003|DEPARTMENT|96|97|Most little sons join just seemingly young years. Fron|quarterly| +10358|AAAAAAAAGHICAAAA|2452913|2453003|DEPARTMENT|96|98|Low wet others should forgive hardly hot, religious heels; windows would|quarterly| +10359|AAAAAAAAHHICAAAA|2452913|2453003|DEPARTMENT|96|99|Problems create. Contents could occur over on the thi|quarterly| +10360|AAAAAAAAIHICAAAA|2452913|2453003|DEPARTMENT|96|100|Too important teachers shall not retire ready young new |quarterly| +10361|AAAAAAAAJHICAAAA|2452913|2453003|DEPARTMENT|96|101|At last interesting teachers give about in a situations. Thus certain scholars |quarterly| +10362|AAAAAAAAKHICAAAA|2452913|2453003|DEPARTMENT|96|102|Probably british years ought to like all ever regular conditions. New, strong parents defend to a l|quarterly| +10363|AAAAAAAALHICAAAA|2452913|2453003|DEPARTMENT|96|103|So black years renounce even crucial beliefs; at present full forests get far main ri|quarterly| +10364|AAAAAAAAMHICAAAA|2452913|2453003|DEPARTMENT|96|104|Sure thoughts purchase things. Future fields happen just free, good days. Children c|quarterly| +10365|AAAAAAAANHICAAAA|2452913|2453003|DEPARTMENT|96|105|Children see styles. Guidelines offer more ways. Markets weigh now english, past women. Ma|quarterly| +10366|AAAAAAAAOHICAAAA|2452913|2453003|DEPARTMENT|96|106|Cold procedures might not ask much necessary, regional rates. Issues might not contemplate aloud; i|quarterly| +10367|AAAAAAAAPHICAAAA|2452913|2453003|DEPARTMENT|96|107|Afraid, alone shows shall get in a girls. Perhaps responsible |quarterly| +10368|AAAAAAAAAIICAAAA|2452913|2453003|DEPARTMENT|96|108|Full, fellow accountants used to expect really bloody questions. Talks shall add even public, a|quarterly| +10369|AAAAAAAABIICAAAA|2452640|2452669|DEPARTMENT|97|1|Therefore young weeks shall boil earlier more formidable occasions. Operations cou|monthly| +10370|AAAAAAAACIICAAAA|2452640|2452669|DEPARTMENT|97|2|However large enquiries sum proposed, gastric colleagues.|monthly| +10371|AAAAAAAADIICAAAA|2452640|2452669|DEPARTMENT|97|3|Others move. Techniques should not beat by a pupils; necessary points will look f|monthly| +10372|AAAAAAAAEIICAAAA|2452640|2452669|DEPARTMENT|97|4|Conscious, detailed earnings shall like also political requir|monthly| +10373|AAAAAAAAFIICAAAA|2452640|2452669|DEPARTMENT|97|5|Important planes shall deliver sources; products achi|monthly| +10374|AAAAAAAAGIICAAAA|2452640|2452669|DEPARTMENT|97|6|Doctors know downstairs. National, quiet interests could take to a years. As|monthly| +10375|AAAAAAAAHIICAAAA|2452640|2452669|DEPARTMENT|97|7|Families must not clear funny, thin areas. Weak, ill|monthly| +10376|AAAAAAAAIIICAAAA|2452640|2452669|DEPARTMENT|97|8|Audiences may get. Final assets allow therefore high, moving heroes; efforts focus at last|monthly| +10377|AAAAAAAAJIICAAAA|2452640|2452669|DEPARTMENT|97|9|Also routine problems used to think here wrong interests. Years prescribe financial men. Oth|monthly| +10378|AAAAAAAAKIICAAAA|2452640|2452669|DEPARTMENT|97|10|Fingers should help main shareholders. Criteria stop just perhaps existing hundreds. International,|monthly| +10379|AAAAAAAALIICAAAA|2452640|2452669|DEPARTMENT|97|11|Fine police give low difficult, strong questions. Women shall not hope old, normal |monthly| +10380|AAAAAAAAMIICAAAA|2452640|2452669|DEPARTMENT|97|12|Areas could not note unique hopes. Most religious rates would speak for insta|monthly| +10381|AAAAAAAANIICAAAA|2452640|2452669|DEPARTMENT|97|13|Patients lose most interesting things. Social clothes will regain key families. Strict, national ca|monthly| +10382|AAAAAAAAOIICAAAA|2452640|2452669|DEPARTMENT|97|14|Popular, social levels would come against the architects. Rat|monthly| +10383|AAAAAAAAPIICAAAA|2452640|2452669|DEPARTMENT|97|15|Lives challenge outside for the schools. Weeks speak i|monthly| +10384|AAAAAAAAAJICAAAA|2452640|2452669|DEPARTMENT|97|16|Highly new centuries justify nevertheless industrial democrats. Wo|monthly| +10385|AAAAAAAABJICAAAA|2452640|2452669|DEPARTMENT|97|17|Final hands ought to become else. Measures could take adult, only terms. O|monthly| +10386|AAAAAAAACJICAAAA|2452640|2452669|DEPARTMENT|97|18|Weeks ought to join possible, reasonable effects. Organizations cannot miss more. Wrong, able |monthly| +10387|AAAAAAAADJICAAAA|2452640|2452669|DEPARTMENT|97|19|American heroes die numerous, appropriate expenses. Right day|monthly| +10388|AAAAAAAAEJICAAAA|2452640|2452669|DEPARTMENT|97|20|Other, unaware clothes allow particularly. Leading, new patter|monthly| +10389|AAAAAAAAFJICAAAA|2452640|2452669|DEPARTMENT|97|21|Able trees can announce. Players stand this obviou|monthly| +10390|AAAAAAAAGJICAAAA|2452640|2452669|DEPARTMENT|97|22|Rural lines undergo now much political pensioners. Effects used to like abroad other resources. |monthly| +10391|AAAAAAAAHJICAAAA|2452640|2452669|DEPARTMENT|97|23|Popular, normal games used to enforce in a eyes. Later recent|monthly| +10392|AAAAAAAAIJICAAAA|2452640|2452669|DEPARTMENT|97|24|Investigations used to date very neither small books. Prospects become ears. Practical, free girls|monthly| +10393|AAAAAAAAJJICAAAA|2452640|2452669|DEPARTMENT|97|25|Gifts may not determine alternatively large officials. Documents eat origi|monthly| +10394|AAAAAAAAKJICAAAA|2452640|2452669|DEPARTMENT|97|26|Social, tall homes shall receive more middle-class banks. Quiet rats give very happy, a|monthly| +10395|AAAAAAAALJICAAAA|2452640|2452669|DEPARTMENT|97|27|Right, necessary falls support just soldiers. Events|monthly| +10396|AAAAAAAAMJICAAAA|2452640|2452669|DEPARTMENT|97|28|Extensive governors describe all yesterday labour problems. Also frequent coll|monthly| +10397|AAAAAAAANJICAAAA|2452640|2452669|DEPARTMENT|97|29|Sadly waiting doctors sell ready students. Crucial, active hands think else. Terribly |monthly| +10398|AAAAAAAAOJICAAAA|2452640|2452669|DEPARTMENT|97|30|Decisions make now good companies. Old, available eyes arise friend|monthly| +10399|AAAAAAAAPJICAAAA|2452640|2452669|DEPARTMENT|97|31|Items collect regularly ancient assessments. Longer educational differenc|monthly| +10400|AAAAAAAAAKICAAAA|2452640||||32||monthly| +10401|AAAAAAAABKICAAAA|2452640||DEPARTMENT||||| +10402|AAAAAAAACKICAAAA|2452640|2452669|DEPARTMENT|97|34|Human interests could not retaliate contracts; best low weeks |monthly| +10403|AAAAAAAADKICAAAA|2452640|2452669|DEPARTMENT|97|35|Parties detect really full industries. Visual, heavy cases could not admit slowly ec|monthly| +10404|AAAAAAAAEKICAAAA|2452640|2452669|DEPARTMENT|97|36|Agents contain heads. Pregnant things would stop with a parents. High, subsequent|monthly| +10405|AAAAAAAAFKICAAAA|2452640|2452669|DEPARTMENT|97|37|Different, hot issues will suggest soon types. Guidel|monthly| +10406|AAAAAAAAGKICAAAA|2452640|2452669|DEPARTMENT|97|38|Once interested gardens get royal, general numbers. Now reas|monthly| +10407|AAAAAAAAHKICAAAA|2452640|2452669|DEPARTMENT|97|39|Other, chief sites shall not vary ever happy markets. |monthly| +10408|AAAAAAAAIKICAAAA|2452640|2452669|DEPARTMENT|97|40|Methods might not ban electoral, bad markets; direct videos explore presents. L|monthly| +10409|AAAAAAAAJKICAAAA|2452640|2452669|DEPARTMENT|97|41|Useful, public objects take important, political students. Sharply ab|monthly| +10410|AAAAAAAAKKICAAAA|2452640|2452669|DEPARTMENT|97|42|Then particular services may regard now questions; uncert|monthly| +10411|AAAAAAAALKICAAAA|2452640|2452669|DEPARTMENT|97|43|Significant, soft months agree. Legs should deal in the pur|monthly| +10412|AAAAAAAAMKICAAAA|2452640|2452669|DEPARTMENT|97|44|Economic, radical rooms sink more early different homes. Pp. can fix per|monthly| +10413|AAAAAAAANKICAAAA|2452640|2452669|DEPARTMENT|97|45|Cattle shall meet unacceptable days. Senior origins allow ahead yet united scho|monthly| +10414|AAAAAAAAOKICAAAA|2452640|2452669|DEPARTMENT|97|46|Tiny, dead places could not leave often social, trying|monthly| +10415|AAAAAAAAPKICAAAA|2452640|2452669|DEPARTMENT|97|47|Expensive, technical ideas should not see here trying types. Nights should defend little. A|monthly| +10416|AAAAAAAAALICAAAA|2452640|2452669|DEPARTMENT|97|48|Narrow, contemporary examples return rapidly. Really proud books o|monthly| +10417|AAAAAAAABLICAAAA|2452640|2452669|DEPARTMENT|97|49|Low police could share only still warm difficulties. Late, upper meetings|monthly| +10418|AAAAAAAACLICAAAA|2452640|2452669|DEPARTMENT|97|50|Complete, opposite patients absorb total lives; bad feet may fe|monthly| +10419|AAAAAAAADLICAAAA|2452640|2452669|DEPARTMENT|97|51|Parts used to assume urban yards. Issues cannot remember simi|monthly| +10420|AAAAAAAAELICAAAA|2452640|2452669|DEPARTMENT|97|52|Urban, scottish markets should not pass emotional, powerful memories. Likewise other compon|monthly| +10421|AAAAAAAAFLICAAAA|2452640|2452669|DEPARTMENT|97|53|Complete tickets kill social, dull foods; for instance video|monthly| +10422|AAAAAAAAGLICAAAA|2452640|2452669|DEPARTMENT|97|54|Only asleep eyes would market immediately certain eventual surfaces. Able nurses pay basic g|monthly| +10423|AAAAAAAAHLICAAAA|2452640|2452669|DEPARTMENT|97|55|Here perfect units evaluate very to a groups. Large persons shou|monthly| +10424|AAAAAAAAILICAAAA|2452640|2452669|DEPARTMENT|97|56|Smooth, good women would fancy just with a things. Contracts talk circumstances. Loc|monthly| +10425|AAAAAAAAJLICAAAA|2452640|2452669|DEPARTMENT|97|57|For instance good tears should not find only. Poor relations snap yet operat|monthly| +10426|AAAAAAAAKLICAAAA|2452640|2452669|DEPARTMENT|97|58|Unusual, old reasons used to bite always between a systems; other, new methods could n|monthly| +10427|AAAAAAAALLICAAAA|2452640|2452669|DEPARTMENT|97|59|Already extraordinary units ought to see better in a things. Areas would not suggest more|monthly| +10428|AAAAAAAAMLICAAAA|2452640|2452669|DEPARTMENT|97|60|Associated eyebrows bring spectacular films. Helpful thin|monthly| +10429|AAAAAAAANLICAAAA|2452640|2452669|DEPARTMENT|97|61|Nuclear visitors might not fill depending on a things. Now recent organisations wo|monthly| +10430|AAAAAAAAOLICAAAA|2452640|2452669|DEPARTMENT|97|62|Clearly powerful students should pick various, good ways. Almost true parties launc|monthly| +10431|AAAAAAAAPLICAAAA|2452640|2452669|DEPARTMENT|97|63|German armies start dry, fresh men. Years let well orange,|monthly| +10432|AAAAAAAAAMICAAAA|2452640|2452669|DEPARTMENT|97|64|In particular unlikely others gauge enormously. Countries will tell no longer combi|monthly| +10433|AAAAAAAABMICAAAA|2452640|2452669|DEPARTMENT|97|65|Origins indicate including a criteria. Primary teac|monthly| +10434|AAAAAAAACMICAAAA|2452640|2452669|DEPARTMENT|97|66|Artistic findings respond of course partial, rigid courts. Here immediate parts speak board|monthly| +10435|AAAAAAAADMICAAAA|2452640|2452669|DEPARTMENT|97|67|Implications put other times. Immediately able reso|monthly| +10436|AAAAAAAAEMICAAAA|2452640|2452669|DEPARTMENT|97|68|Electronic cases become live talks. Children get. Offic|monthly| +10437|AAAAAAAAFMICAAAA|2452640|2452669|DEPARTMENT|97|69|Great, adult woods might get later civil, good jobs. Taxes accept modern standards; da|monthly| +10438|AAAAAAAAGMICAAAA|2452640|2452669|DEPARTMENT|97|70|Following, interesting theories shall look rapidly modes. Essential, mature solutions shall lik|monthly| +10439|AAAAAAAAHMICAAAA|2452640|2452669|DEPARTMENT|97|71|Entire officials could bring. Please possible weapons creat|monthly| +10440|AAAAAAAAIMICAAAA|2452640|2452669|DEPARTMENT|97|72|Foreign, certain gentlemen visit only against the boats. G|monthly| +10441|AAAAAAAAJMICAAAA|2452640|2452669|DEPARTMENT|97|73|Full agreements would fight coolly; estimates will not consume noble, national units. Lo|monthly| +10442|AAAAAAAAKMICAAAA|2452640|2452669|DEPARTMENT|97|74|Hands must not stop for example all other skills. So |monthly| +10443|AAAAAAAALMICAAAA|2452640|2452669|DEPARTMENT|97|75|Clean, suitable arguments ensure in order; as formal manu|monthly| +10444|AAAAAAAAMMICAAAA|2452640|2452669|DEPARTMENT|97|76|Little, southern ranks comfort again. Measures will occur. Young ways identify women. Mo|monthly| +10445|AAAAAAAANMICAAAA|2452640|2452669|DEPARTMENT|97|77|Brutally parental governments used to come also in a winners. Warm deals will |monthly| +10446|AAAAAAAAOMICAAAA|2452640|2452669|DEPARTMENT|97|78|Different eyes might get only practical, rough societies.|monthly| +10447|AAAAAAAAPMICAAAA|2452640|2452669|DEPARTMENT|97|79|Doors will comprehend much to the resources. Immediatel|monthly| +10448|AAAAAAAAANICAAAA|2452640|2452669|DEPARTMENT|97|80|Bare directors used to think eventually without a things. Importantly major nations expire both|monthly| +10449|AAAAAAAABNICAAAA|2452640|2452669|DEPARTMENT|97|81|Other, real minerals think over the travellers. New, right police cannot |monthly| +10450|AAAAAAAACNICAAAA|2452640|2452669|DEPARTMENT|97|82|Scores work for the members. Reserves turn objectives; huge companie|monthly| +10451|AAAAAAAADNICAAAA|2452640|2452669|DEPARTMENT|97|83|Confident intentions could avoid rather local schools. Only|monthly| +10452|AAAAAAAAENICAAAA|2452640|2452669|DEPARTMENT|97|84|Easy, mechanical waves go earlier well continued movements. Children shall open legal sale|monthly| +10453|AAAAAAAAFNICAAAA|2452640|2452669|DEPARTMENT|97|85|Low systems wait ever japanese girls. Chronic fathers stress u|monthly| +10454|AAAAAAAAGNICAAAA|2452640|2452669|DEPARTMENT|97|86|Functions should see plain. Nice results make meanwhile frenc|monthly| +10455|AAAAAAAAHNICAAAA|2452640|2452669|DEPARTMENT|97|87|Now able managers could know so negative, small degrees. Leading, |monthly| +10456|AAAAAAAAINICAAAA|2452640|2452669|DEPARTMENT|97|88|Churches would not play. Links relate with a modules. Forces say for a economies. Co|monthly| +10457|AAAAAAAAJNICAAAA|2452640|2452669|DEPARTMENT|97|89|Long houses used to pause further. Years appear much naval, perma|monthly| +10458|AAAAAAAAKNICAAAA|2452640|2452669|DEPARTMENT|97|90|Then democratic sales must believe important, future assets. Social changes could n|monthly| +10459|AAAAAAAALNICAAAA|2452640|2452669|DEPARTMENT|97|91|Possible democrats must mean hardly along with the trials. Current, good windows cope others;|monthly| +10460|AAAAAAAAMNICAAAA|2452640|2452669|DEPARTMENT|97|92|Wrong, new obligations should reach now united, necessary c|monthly| +10461|AAAAAAAANNICAAAA|2452640|2452669|DEPARTMENT|97|93|Large, modern movements shall declare long neither sensitive flo|monthly| +10462|AAAAAAAAONICAAAA|2452640|2452669|DEPARTMENT|97|94|Public students get years. Parts must give almost large yea|monthly| +10463|AAAAAAAAPNICAAAA|2452640|2452669|DEPARTMENT|97|95|Likely, chief colleagues could not fight however public, urban difference|monthly| +10464|AAAAAAAAAOICAAAA|2452640|2452669|DEPARTMENT|97|96|Accounts stem simply social, strong hours. Also early nations must raise o|monthly| +10465|AAAAAAAABOICAAAA|2452640|2452669|DEPARTMENT|97|97|Children may reassert yet. Gold, single dates use almost like a limits. Somewhere thin days may not|monthly| +10466|AAAAAAAACOICAAAA|2452640|2452669|DEPARTMENT|97|98|Real points select henceforth sales. Again annual eyes provide other needs. Comparable, |monthly| +10467|AAAAAAAADOICAAAA|2452640|2452669|DEPARTMENT|97|99|Other, new speakers recover in the animals. Forthcoming books woul|monthly| +10468|AAAAAAAAEOICAAAA|2452640|2452669|DEPARTMENT|97|100|French, adult regions assure black, english attitudes. Wrong feet ought to destroy; stories |monthly| +10469|AAAAAAAAFOICAAAA|2452640|2452669|DEPARTMENT|97|101|General theories would shout openly new thanks. Pensioners emerge almost; charges shall not absorb|monthly| +10470|AAAAAAAAGOICAAAA|2452640|2452669|DEPARTMENT|97|102|Twin beds will pay to a voices. Effective, new problems will not see. Valuable, cheerful eyes wi|monthly| +10471|AAAAAAAAHOICAAAA|2452640|2452669|DEPARTMENT|97|103|Identical complaints find by no means from the approaches. So decent figures must |monthly| +10472|AAAAAAAAIOICAAAA|2452640|2452669|DEPARTMENT|97|104|Strong citizens become only names. Glad application|monthly| +10473|AAAAAAAAJOICAAAA|2452640|2452669|DEPARTMENT|97|105|Professional things used to remember. So deep others work howeve|monthly| +10474|AAAAAAAAKOICAAAA|2452640|2452669|DEPARTMENT|97|106|About new developers think just here regular clergy. Materials would talk tent|monthly| +10475|AAAAAAAALOICAAAA|2452640|2452669|DEPARTMENT|97|107|Sufficient ministers budge across over the fingers. Seasons ought to say likely readers. Alone li|monthly| +10476|AAAAAAAAMOICAAAA|2452640|2452669|DEPARTMENT|97|108|Evident, direct hands contribute also. Schools save independent, single vehicles; resources ta|monthly| +10477|AAAAAAAANOICAAAA|2452670|2452699|DEPARTMENT|98|1|Particular, estimated churches lose too ago dead ideas|monthly| +10478|AAAAAAAAOOICAAAA|2452670|2452699|DEPARTMENT|98|2|Gentle, important buyers shall marry soon. Irish, complete schools enhance initially citizen|monthly| +10479|AAAAAAAAPOICAAAA|2452670|2452699|DEPARTMENT|98|3|Public groups could absorb better white injuries. Titles mus|monthly| +10480|AAAAAAAAAPICAAAA|2452670|2452699|DEPARTMENT|98|4|Often old contacts go even clear languages. Desperately |monthly| +10481|AAAAAAAABPICAAAA|2452670|2452699|DEPARTMENT|98|5|Finally certain women should submit months; probabl|monthly| +10482|AAAAAAAACPICAAAA|2452670|2452699|DEPARTMENT|98|6|Natural, hot results should warn nearly fair police. Women wi|monthly| +10483|AAAAAAAADPICAAAA|2452670|2452699|DEPARTMENT|98|7|Busy events give at a libraries; standard years act with a regulatio|monthly| +10484|AAAAAAAAEPICAAAA|2452670|2452699|DEPARTMENT|98|8|Essential, cultural vehicles doubt continuously now good obj|monthly| +10485|AAAAAAAAFPICAAAA|2452670|2452699|DEPARTMENT|98|9|Representative, successful issues follow relatively african, warm months. Acute years may |monthly| +10486|AAAAAAAAGPICAAAA|2452670|2452699|DEPARTMENT|98|10|Upper plans may work too inappropriate questions. Just small uses take o|monthly| +10487|AAAAAAAAHPICAAAA|2452670|2452699|DEPARTMENT|98|11|Stores recognise fast following manufacturers. Again voluntary incomes |monthly| +10488|AAAAAAAAIPICAAAA|2452670|2452699|DEPARTMENT|98|12|Soviet, modest laws ensure. Services believe worst radic|monthly| +10489|AAAAAAAAJPICAAAA|2452670|2452699|DEPARTMENT|98|13|Systems might offer splendid, old years. Ago official |monthly| +10490|AAAAAAAAKPICAAAA|2452670|2452699|DEPARTMENT|98|14|Drinks should come probably tropical publications; overall questions commence broken|monthly| +10491|AAAAAAAALPICAAAA|2452670|2452699|DEPARTMENT|98|15|Creative, certain contents could go in the organisers. Humans calm high st|monthly| +10492|AAAAAAAAMPICAAAA|2452670|2452699|DEPARTMENT|98|16|Yesterday multiple children let. Domestic circumstances must not damage |monthly| +10493|AAAAAAAANPICAAAA|2452670|2452699|DEPARTMENT|98|17|Peacefully particular claims might grab local arguments. Liberal, critical step|monthly| +10494|AAAAAAAAOPICAAAA|2452670|2452699|DEPARTMENT|98|18|Oral, heavy organisations try also mental awards. Remarkable, appropriate skil|monthly| +10495|AAAAAAAAPPICAAAA|2452670|2452699|DEPARTMENT|98|19|Urban measures see new attitudes. Months replace again recently open subjects. Other folk|monthly| +10496|AAAAAAAAAAJCAAAA|2452670|2452699|DEPARTMENT|98|20|Models destroy inextricably important lengths. Long|monthly| +10497|AAAAAAAABAJCAAAA|2452670|2452699|DEPARTMENT|98|21|Once simple ages used to become with a measurements. Narrow, old|monthly| +10498|AAAAAAAACAJCAAAA|2452670|2452699|DEPARTMENT|98|22|Effects should not pass departments. Edges can find then val|monthly| +10499|AAAAAAAADAJCAAAA||2452699|DEPARTMENT|98|23|Results fix so words. Months cannot find publicly. Consequences must not re|monthly| +10500|AAAAAAAAEAJCAAAA|2452670|2452699|DEPARTMENT|98|24|Well political laws can grow of course completely english guests. Old |monthly| +10501|AAAAAAAAFAJCAAAA|2452670|2452699|DEPARTMENT|98|25|Marginally major areas gauge conservative forms. Moral products shall put years. Manufacturing, |monthly| +10502|AAAAAAAAGAJCAAAA|2452670|2452699|DEPARTMENT|98|26|Industrial glasses shall encourage soon various trousers. Then top minutes would develop. Spec|monthly| +10503|AAAAAAAAHAJCAAAA|2452670|2452699|DEPARTMENT|98|27|Patients ensure actually apparent, big experiments. Stable flowers encourage so black p|monthly| +10504|AAAAAAAAIAJCAAAA|2452670|2452699|DEPARTMENT|98|28|Both honest developments bother certain, local questions. However little countries ough|monthly| +10505|AAAAAAAAJAJCAAAA|2452670|2452699|DEPARTMENT|98|29|Crucial, german circumstances might not allow. Onl|monthly| +10506|AAAAAAAAKAJCAAAA|2452670|2452699|DEPARTMENT|98|30|Yet tender participants want uncomfortably dead, exciting horse|monthly| +10507|AAAAAAAALAJCAAAA|2452670|2452699|DEPARTMENT|98|31|Dry advantages appeal only expensive processes. Top o|monthly| +10508|AAAAAAAAMAJCAAAA|2452670|2452699|DEPARTMENT|98|32|Men can hold particularly other worlds. Poems could see however kinds. Small|monthly| +10509|AAAAAAAANAJCAAAA|2452670|2452699|DEPARTMENT|98|33|Substantially new activities might not fight howev|monthly| +10510|AAAAAAAAOAJCAAAA|2452670|2452699|DEPARTMENT|98|34|Just new requirements say dangerously general things. Quantities repeat|monthly| +10511|AAAAAAAAPAJCAAAA|2452670|2452699|DEPARTMENT|98|35|Perfectly excellent situations look now yet other weeks. New, secret difficulties cha|monthly| +10512|AAAAAAAAABJCAAAA|2452670|2452699|DEPARTMENT|98|36|Alternative, difficult estates used to think. Then alternative principles p|monthly| +10513|AAAAAAAABBJCAAAA|2452670|2452699|DEPARTMENT|98|37|Now whole standards use now at the languages; objectives ought to admit sometime|monthly| +10514|AAAAAAAACBJCAAAA|2452670|2452699|DEPARTMENT|98|38|Whole, whole dishes will not appear also in a jobs. Large, local frie|monthly| +10515|AAAAAAAADBJCAAAA|2452670|2452699|DEPARTMENT|98|39|Cards hold sure; secrets avoid clearly forwards protective skills; once geograph|monthly| +10516|AAAAAAAAEBJCAAAA|2452670|2452699|DEPARTMENT|98|40|Away political years will stay as lawyers. Also wet members emerge different, diff|monthly| +10517|AAAAAAAAFBJCAAAA|2452670|2452699|DEPARTMENT|98|41|Elements must believe to a plates. Well different animals care per annum g|monthly| +10518|AAAAAAAAGBJCAAAA|2452670|2452699|DEPARTMENT|98|42|Official, little floors figure services. Times scr|monthly| +10519|AAAAAAAAHBJCAAAA|2452670|2452699|DEPARTMENT|98|43|Values drive. Public persons must secure now contempora|monthly| +10520|AAAAAAAAIBJCAAAA|2452670|2452699|DEPARTMENT|98|44|Active miners get please vague chiefs. Difficult costs split later. Satisfied words swi|monthly| +10521|AAAAAAAAJBJCAAAA|2452670|2452699|DEPARTMENT|98|45|Also high savings take just. Smooth payments may not mean more cold |monthly| +10522|AAAAAAAAKBJCAAAA|2452670|2452699|DEPARTMENT|98|46|Again alone audiences should repay differently studies. Suddenly o|monthly| +10523|AAAAAAAALBJCAAAA|2452670|2452699|DEPARTMENT|98|47|Biological users tell now local, left options. Procedures must not expect other companie|monthly| +10524|AAAAAAAAMBJCAAAA|2452670|2452699|DEPARTMENT|98|48|Serious, important points take. Overseas, international ideas lose ag|monthly| +10525|AAAAAAAANBJCAAAA|2452670|2452699|DEPARTMENT|98|49|Urban, good clouds deepen centrally sacred, old artis|monthly| +10526|AAAAAAAAOBJCAAAA|2452670|2452699|DEPARTMENT|98|50|Unique parts cannot want so american models. Professional, small methods wi|monthly| +10527|AAAAAAAAPBJCAAAA|2452670|2452699|DEPARTMENT|98|51|Actually clear techniques cope often from a quantities; true resul|monthly| +10528|AAAAAAAAACJCAAAA|2452670|2452699|DEPARTMENT|98|52|Pale, electronic foods should publish at all strong dimensions. Dry, |monthly| +10529|AAAAAAAABCJCAAAA|2452670|2452699|DEPARTMENT|98|53|Clear programmes sell british eyes; heavily englis|monthly| +10530|AAAAAAAACCJCAAAA|2452670|2452699|DEPARTMENT|98|54|Stories may attempt until a rounds. As personal colleagues concede inner particles. Black, |monthly| +10531|AAAAAAAADCJCAAAA|2452670|2452699|DEPARTMENT|98|55|Papers cause always national, past councillors; histo|monthly| +10532|AAAAAAAAECJCAAAA|2452670|2452699|DEPARTMENT|98|56|Different, necessary conditions shall see. National respects write well. Controversial, publi|monthly| +10533|AAAAAAAAFCJCAAAA|2452670|2452699|DEPARTMENT|98|57|New, tired angles may win most best good kings. Rather various articles shall want i|monthly| +10534|AAAAAAAAGCJCAAAA|2452670|2452699|DEPARTMENT|98|58|Full, poor countries find elsewhere stories. Goods must see never stu|monthly| +10535|AAAAAAAAHCJCAAAA|2452670|2452699|DEPARTMENT|98|59|Benefits must walk yesterday by the languages. Able, simple contacts bring as in a versions. Rare |monthly| +10536|AAAAAAAAICJCAAAA|2452670|2452699|DEPARTMENT|98|60|Other, warm days go further american exercises; specific women ought to contin|monthly| +10537|AAAAAAAAJCJCAAAA|2452670|2452699|DEPARTMENT|98|61|Servants will rival responsibilities. Other, principal reforms shall acce|monthly| +10538|AAAAAAAAKCJCAAAA|2452670|2452699|DEPARTMENT|98|62|Rare, other times may not sink at all social, double cups. Men preven|monthly| +10539|AAAAAAAALCJCAAAA|2452670|2452699|DEPARTMENT|98|63|Scientists cost also still subtle companies. Furthermo|monthly| +10540|AAAAAAAAMCJCAAAA|2452670|2452699|DEPARTMENT|98|64|Main, private centres leave books. Implications give still particular expert|monthly| +10541|AAAAAAAANCJCAAAA|2452670|2452699|DEPARTMENT|98|65|Slight years come only great, proposed rocks; public, elderly |monthly| +10542|AAAAAAAAOCJCAAAA|2452670|2452699|DEPARTMENT|98|66|Major women answer however doctors. Sufficient values may ensure |monthly| +10543|AAAAAAAAPCJCAAAA|2452670|2452699|DEPARTMENT|98|67|Most technical contracts follow broadly main hands; symptoms would tell at all modes. Typical, majo|monthly| +10544|AAAAAAAAADJCAAAA|2452670|2452699|DEPARTMENT|98|68|Natural books should not mean too financial, important cases. Always grey faces must move also ano|monthly| +10545|AAAAAAAABDJCAAAA|2452670|2452699|DEPARTMENT|98|69|Happy, outside structures damage wide in the tales. Various loans come unable, special areas; conte|monthly| +10546|AAAAAAAACDJCAAAA|2452670|2452699|DEPARTMENT|98|70|Hard difficult books establish finally right independent co|monthly| +10547|AAAAAAAADDJCAAAA|2452670|2452699|DEPARTMENT|98|71|Long functions come particularly warm appearances. Regularly true |monthly| +10548|AAAAAAAAEDJCAAAA|2452670|2452699|DEPARTMENT|98|72|General spirits might share routinely prime materials. Local forces can realize never ear|monthly| +10549|AAAAAAAAFDJCAAAA|2452670|2452699|DEPARTMENT|98|73|Supposed moments know hardly genuine, golden executives; beautiful methods |monthly| +10550|AAAAAAAAGDJCAAAA|2452670|2452699|DEPARTMENT|98|74|Red sheets cost. Now medical parties used to turn as minor, public wo|monthly| +10551|AAAAAAAAHDJCAAAA|2452670|2452699|DEPARTMENT|98|75|Points should not observe out a days. Eligible, relative measures break far into a years. Casu|monthly| +10552|AAAAAAAAIDJCAAAA|2452670|2452699|DEPARTMENT|98|76|Products find consequently open, chief elections. Consistently spirit|monthly| +10553|AAAAAAAAJDJCAAAA|2452670|2452699|DEPARTMENT|98|77|As necessary changes run especially more fast patterns|monthly| +10554|AAAAAAAAKDJCAAAA|2452670|2452699|DEPARTMENT|98|78|Social spots encourage successfully. New sections speak still relat|monthly| +10555|AAAAAAAALDJCAAAA|2452670|2452699|DEPARTMENT|98|79|Popular members like in a looks. Forces should brin|monthly| +10556|AAAAAAAAMDJCAAAA|2452670|2452699|DEPARTMENT|98|80|Here mental chains shall not allocate between the times. Ne|monthly| +10557|AAAAAAAANDJCAAAA|2452670|2452699|DEPARTMENT|98|81|Mainly civil years shall mean parents. Minds take reportedly most |monthly| +10558|AAAAAAAAODJCAAAA|2452670|2452699|DEPARTMENT|98|82|Sometimes comfortable tears take still on a participants|monthly| +10559|AAAAAAAAPDJCAAAA|2452670|2452699|DEPARTMENT|98|83|Occasionally final assumptions need absolutely special|monthly| +10560|AAAAAAAAAEJCAAAA|2452670|2452699|DEPARTMENT|98|84|Problems must not create guilty cuts. Cheerfully young doctors should establish vivid|monthly| +10561|AAAAAAAABEJCAAAA|2452670|2452699|DEPARTMENT|98|85|Kings think events. Activities work all as advanced exhib|monthly| +10562|AAAAAAAACEJCAAAA|2452670|2452699|DEPARTMENT|98|86|Other parents would occur only electrical countries. Christian dimensions might use more th|monthly| +10563|AAAAAAAADEJCAAAA|2452670|2452699|DEPARTMENT|98|87|Turkish, likely techniques would not acquire detailed spots; good, capi|monthly| +10564|AAAAAAAAEEJCAAAA|2452670|2452699|DEPARTMENT|98|88|Exclusive, good eggs round public so detailed fears; states cannot discuss well. So other|monthly| +10565|AAAAAAAAFEJCAAAA|2452670|2452699|DEPARTMENT|98|89|Sure other stages provide. Soviet, medium legs used to burn presumably for a feet. Young be|monthly| +10566|AAAAAAAAGEJCAAAA|2452670|2452699|DEPARTMENT|98|90|Judicial, total eyes cannot make then elected, friendly hands. Books must not ease over a supporte|monthly| +10567|AAAAAAAAHEJCAAAA|2452670|2452699|DEPARTMENT|98|91|For long great companies ought to see as. However large ni|monthly| +10568|AAAAAAAAIEJCAAAA|2452670|2452699|DEPARTMENT|98|92|Very young floors should not learn great pictures. |monthly| +10569|AAAAAAAAJEJCAAAA|2452670||DEPARTMENT|98|||| +10570|AAAAAAAAKEJCAAAA|2452670|2452699|DEPARTMENT|98|94|Very, large titles should enable; otherwise tall visitors mur|monthly| +10571|AAAAAAAALEJCAAAA|2452670|2452699|DEPARTMENT|98|95|Secondary, unable germans must use. Absent, old parents may seem previous curtains.|monthly| +10572|AAAAAAAAMEJCAAAA|2452670|2452699|DEPARTMENT|98|96|Able, deep representatives exceed times. Just united falls expect existing, different |monthly| +10573|AAAAAAAANEJCAAAA|2452670|2452699|DEPARTMENT|98|97|Ordinary, other pounds shall hold around by a studies. Cars lift alternatively. Following terms |monthly| +10574|AAAAAAAAOEJCAAAA|2452670|2452699|DEPARTMENT|98|98|Ways would not see however; even other employees shall help here big |monthly| +10575|AAAAAAAAPEJCAAAA|2452670|2452699|DEPARTMENT|98|99|All political moments used to take so outside legal designs. Also new arts rectify obv|monthly| +10576|AAAAAAAAAFJCAAAA|2452670|2452699|DEPARTMENT|98|100|Electoral sons may follow only. Major, economic days need at a grounds. Neve|monthly| +10577|AAAAAAAABFJCAAAA|2452670|2452699|DEPARTMENT|98|101|Relationships imagine quite certain, full terms. Southern shops lend back, othe|monthly| +10578|AAAAAAAACFJCAAAA|2452670|2452699|DEPARTMENT|98|102|Drivers maintain into the products. Windows eat somewhere |monthly| +10579|AAAAAAAADFJCAAAA|2452670|2452699|DEPARTMENT|98|103|Only effective times mean exactly especially international heads. Strange, existing cha|monthly| +10580|AAAAAAAAEFJCAAAA|2452670|2452699|DEPARTMENT|98|104|Books explore buildings. Patients might remember very with a b|monthly| +10581|AAAAAAAAFFJCAAAA|2452670|2452699|DEPARTMENT|98|105|Aspirations can keep more circumstances. Creative, subsequent mice could take |monthly| +10582|AAAAAAAAGFJCAAAA|2452670|2452699|DEPARTMENT|98|106|Reduced properties enjoy soon. Potential, indian songs should make with a objectives. Littl|monthly| +10583|AAAAAAAAHFJCAAAA|2452670|2452699|DEPARTMENT|98|107|Previous, different days should protect complex letters|monthly| +10584|AAAAAAAAIFJCAAAA|2452670|2452699|DEPARTMENT|98|108|Personal, slim groups would know downstairs new skills. True, other partner|monthly| +10585|AAAAAAAAJFJCAAAA|2452700|2452729|DEPARTMENT|99|1|Australian, good sales should not drive. Knees promote rare|monthly| +10586|AAAAAAAAKFJCAAAA|2452700|2452729|DEPARTMENT|99|2|Economic, natural tensions result subsequently ever certain areas. Only casual texts|monthly| +10587|AAAAAAAALFJCAAAA|2452700|2452729|DEPARTMENT|99|3|Today living parties will limit relations. Hidden minutes wal|monthly| +10588|AAAAAAAAMFJCAAAA|2452700|2452729|DEPARTMENT|99|4|Differences regain pale persons. Elaborate goods could produce that curtains. Red doctors believe|monthly| +10589|AAAAAAAANFJCAAAA|2452700|2452729|DEPARTMENT|99|5|Words show desperate years. Typically modern details must not face other, abl|monthly| +10590|AAAAAAAAOFJCAAAA|2452700|2452729|DEPARTMENT|99|6|Properties limit therefore territories; old, creative breasts play onl|monthly| +10591|AAAAAAAAPFJCAAAA|2452700|2452729|DEPARTMENT|99|7|Familiar, main letters might distinguish things. True only poems examine at present sour|monthly| +10592|AAAAAAAAAGJCAAAA|2452700|2452729|DEPARTMENT|99|8|Afterwards very players tackle. So famous nations may no|monthly| +10593|AAAAAAAABGJCAAAA|2452700|2452729|DEPARTMENT|99|9|Animals go so different, suitable years. Simple centres make much for the sales. Lights bother soc|monthly| +10594|AAAAAAAACGJCAAAA|2452700|2452729|DEPARTMENT|99|10|Conditions feel hopefully lexical words. Other authorities |monthly| +10595|AAAAAAAADGJCAAAA|2452700|2452729|DEPARTMENT|99|11|Also difficult others bring to a firms. Probably p|monthly| +10596|AAAAAAAAEGJCAAAA|2452700|2452729|DEPARTMENT|99|12|More social changes write only for a mountains. Opportunities improve. So c|monthly| +10597|AAAAAAAAFGJCAAAA|2452700|2452729|DEPARTMENT|99|13|Parents may affect very about existing trees. Mostly big policies alleviate throu|monthly| +10598|AAAAAAAAGGJCAAAA|2452700|2452729|DEPARTMENT|99|14|Just reluctant words lead harder. Legal studies shall pay at the terms. Active agent|monthly| +10599|AAAAAAAAHGJCAAAA|2452700|2452729|DEPARTMENT|99|15|Groups force. Glad, secondary cases lead. Thanks may not identify movements. Her|monthly| +10600|AAAAAAAAIGJCAAAA|2452700|2452729|DEPARTMENT|99|16|Especially effective services used to hurt however on|monthly| +10601|AAAAAAAAJGJCAAAA|2452700|2452729|DEPARTMENT|99|17|Partners would not make well interesting men. New years may not start all |monthly| +10602|AAAAAAAAKGJCAAAA|2452700|2452729|DEPARTMENT|99|18|Possible pieces would exercise even likely, disabled ga|monthly| +10603|AAAAAAAALGJCAAAA|2452700|2452729|DEPARTMENT|99|19|Principles live thereby instantly public hands. Warm blacks can harm atlantic, ch|monthly| +10604|AAAAAAAAMGJCAAAA|2452700|2452729|DEPARTMENT|99|20|Possible matches cut far into a customs. Unusual statements will not cost alway|monthly| +10605|AAAAAAAANGJCAAAA|2452700|2452729|DEPARTMENT|99|21|Workers could not come domestic, cold services. European, dark details should not enable mainly o|monthly| +10606|AAAAAAAAOGJCAAAA|2452700|2452729|DEPARTMENT|99|22|Intimate hearts deny mathematical, firm developments. Occasionally eng|monthly| +10607|AAAAAAAAPGJCAAAA|2452700|2452729|DEPARTMENT|99|23|Important conditions cannot feel all main offices. Expensive|monthly| +10608|AAAAAAAAAHJCAAAA|2452700|2452729|DEPARTMENT|99|24|Historical children may require pleasant, bright s|monthly| +10609|AAAAAAAABHJCAAAA|2452700|2452729|DEPARTMENT|99|25|Other, historic tories cannot consider all right local centres. Rather stupid i|monthly| +10610|AAAAAAAACHJCAAAA|2452700|2452729|DEPARTMENT|99|||| +10611|AAAAAAAADHJCAAAA|2452700|2452729|DEPARTMENT|99|27|Above social pieces may mind square, proposed hundr|monthly| +10612|AAAAAAAAEHJCAAAA|2452700|2452729|DEPARTMENT|99|28|Severe estates arrange always important minutes. Of course difficult difficulties survive there h|monthly| +10613|AAAAAAAAFHJCAAAA|2452700|2452729|DEPARTMENT|99|29|Cells must sit too other feet. Simple, small colleag|monthly| +10614|AAAAAAAAGHJCAAAA|2452700|2452729|DEPARTMENT|99|30|Narrow sales mark always. Fond, political interests used to refer strong|monthly| +10615|AAAAAAAAHHJCAAAA|2452700|2452729|DEPARTMENT|99|31|Mutual maps may not express in a women. Conversely dead sales might share |monthly| +10616|AAAAAAAAIHJCAAAA|2452700|2452729|DEPARTMENT|99|32|Again bad figures should move for a requirements. Protective, general days |monthly| +10617|AAAAAAAAJHJCAAAA|2452700|2452729|DEPARTMENT|99|33|Marks work necessary parameters. Twice local estates c|monthly| +10618|AAAAAAAAKHJCAAAA|2452700|2452729|DEPARTMENT|99|34|Rural parents might raise necessary companies. Experts record in a sorts. Briti|monthly| +10619|AAAAAAAALHJCAAAA|2452700|2452729|DEPARTMENT|99|35|Terms match enough here special levels. Then guilty o|monthly| +10620|AAAAAAAAMHJCAAAA|2452700|2452729|DEPARTMENT|99|36|New articles look for a parties; essential payments used to know ne|monthly| +10621|AAAAAAAANHJCAAAA|2452700|2452729|DEPARTMENT|99|37|Significant novels bring activities. Light, concerned|monthly| +10622|AAAAAAAAOHJCAAAA|2452700|2452729|DEPARTMENT|99|38|Able, strong eyes must not know in particular hence private exceptions. Patients feel c|monthly| +10623|AAAAAAAAPHJCAAAA|2452700|2452729|DEPARTMENT|99|39|British, german parents eat particularly very, good p|monthly| +10624|AAAAAAAAAIJCAAAA|2452700|2452729|DEPARTMENT|99|40|Outdoor, domestic universities can charge similarly specified writers. Necessary actions vary i|monthly| +10625|AAAAAAAABIJCAAAA|2452700|2452729|DEPARTMENT|99|41|Enquiries will not improve simply also human tasks. Gradually wooden years slide families. S|monthly| +10626|AAAAAAAACIJCAAAA|2452700|2452729|DEPARTMENT|99|42|Also actual customers take. Respective processes help |monthly| +10627|AAAAAAAADIJCAAAA|2452700|2452729|DEPARTMENT|99|43|Female, new things trust voluntary, urban women. Soo|monthly| +10628|AAAAAAAAEIJCAAAA|2452700|2452729|DEPARTMENT|99|44|Openly other sales must not escape within the services. Manufacturing officials sh|monthly| +10629|AAAAAAAAFIJCAAAA|2452700|2452729|DEPARTMENT|99|45|Fresh months would not sweep. Italian planes adhere easily over raw benefits. Then legal levels s|monthly| +10630|AAAAAAAAGIJCAAAA|2452700|2452729|DEPARTMENT|99|46|Labour, local twins speak. Very necessary reactions set well political un|monthly| +10631|AAAAAAAAHIJCAAAA|||DEPARTMENT||47|Political, alone groups could release quickly possible, recent authoritie|monthly| +10632|AAAAAAAAIIJCAAAA|2452700|2452729|DEPARTMENT|99|48|Basic years should not spend very current fields. M|monthly| +10633|AAAAAAAAJIJCAAAA|2452700|2452729|DEPARTMENT|99|49|Conventional processes used to want very; golden reforms support too. Elderly mi|monthly| +10634|AAAAAAAAKIJCAAAA|2452700|2452729|DEPARTMENT|99|50|Films may not save by a dogs. Systems issue together shallow, firm|monthly| +10635|AAAAAAAALIJCAAAA|2452700|2452729|DEPARTMENT|99|51|More great lines ought to like then willing natural blues. Quickly true statements may |monthly| +10636|AAAAAAAAMIJCAAAA|2452700|2452729|DEPARTMENT|99|52|So waiting services could depend different forms. Key sites must not shift public, peculiar event|monthly| +10637|AAAAAAAANIJCAAAA|2452700|2452729|DEPARTMENT|99|53|Significantly british houses ought to deliver. General products must offer blue, other|monthly| +10638|AAAAAAAAOIJCAAAA|2452700|2452729|DEPARTMENT|99|54|Also fundamental levels would make then criminal cars. Thus|monthly| +10639|AAAAAAAAPIJCAAAA|2452700|2452729|DEPARTMENT|99|55|Human costs go open, indian costs. Other months manu|monthly| +10640|AAAAAAAAAJJCAAAA|2452700|2452729|DEPARTMENT|99|56|Journals shall want almost really married benefits; european, urgent negotiations kn|monthly| +10641|AAAAAAAABJJCAAAA|2452700|2452729|DEPARTMENT|99|57|Troubles try conspicuously vulnerable representatives. Rules see at l|monthly| +10642|AAAAAAAACJJCAAAA|2452700|2452729|DEPARTMENT|99|58|Also brief governments accept there white, importan|monthly| +10643|AAAAAAAADJJCAAAA|2452700|2452729|DEPARTMENT|99|59|Owners know. Women perform therefore until a years. Single, major com|monthly| +10644|AAAAAAAAEJJCAAAA|2452700|2452729|DEPARTMENT|99|60|Now peculiar elections should not depend students. Inc, stable democrats sha|monthly| +10645|AAAAAAAAFJJCAAAA|2452700|2452729|DEPARTMENT|99|61|Often notable activities spawn so; friends must base great activities; there fu|monthly| +10646|AAAAAAAAGJJCAAAA|2452700|2452729|DEPARTMENT|99|62|Once concerned offices ought to take with the members. Lips say similarly rough programmes. Ibid.|monthly| +10647|AAAAAAAAHJJCAAAA|2452700|2452729|DEPARTMENT|99|63|Extremely ethnic products shall not explain skills. As domestic preferences should|monthly| +10648|AAAAAAAAIJJCAAAA|2452700|2452729|DEPARTMENT|99|64|New, industrial pilots ride linguistic, lucky powers. Good so|monthly| +10649|AAAAAAAAJJJCAAAA|2452700|2452729|DEPARTMENT|99|65|Profoundly european terms exclude all at a businessmen; aware requirements cannot consider r|monthly| +10650|AAAAAAAAKJJCAAAA|2452700|2452729|DEPARTMENT|99|66|Personal, statutory characteristics should sell even regional s|monthly| +10651|AAAAAAAALJJCAAAA|2452700|2452729|DEPARTMENT|99|67|More than great positions keep up to in a waves. Old books foster inside more certain val|monthly| +10652|AAAAAAAAMJJCAAAA|2452700|2452729|DEPARTMENT|99|68|Elections get most like afraid miles. Fellow, broken r|monthly| +10653|AAAAAAAANJJCAAAA|2452700|2452729|DEPARTMENT|99|69|Economic, satisfied systems will not take small acts; imports could raise even possib|monthly| +10654|AAAAAAAAOJJCAAAA|2452700|2452729|DEPARTMENT|99|70|At least heavy voices operate highly overseas missiles; times rally for the time being linea|monthly| +10655|AAAAAAAAPJJCAAAA|2452700|2452729|DEPARTMENT|99|71|Left, ruling groups ought to register already privately|monthly| +10656|AAAAAAAAAKJCAAAA|2452700|2452729|DEPARTMENT|99|72|Women spend again. Bloody systems visit only far recent positions. Important, com|monthly| +10657|AAAAAAAABKJCAAAA|2452700|2452729|DEPARTMENT|99|73|Benefits could move functional students. Slow managers figure more gastric, special years. Flowe|monthly| +10658|AAAAAAAACKJCAAAA|2452700|2452729|DEPARTMENT|99|74|Shares may indulge home. Public arms shall not seem at once here local solutions. Different, chemic|monthly| +10659|AAAAAAAADKJCAAAA|2452700|2452729|DEPARTMENT|99|75|Separate, material cuts used to achieve too taxes. Affairs would supp|monthly| +10660|AAAAAAAAEKJCAAAA|2452700|2452729|DEPARTMENT|99|76|Efforts marry established adults. Less western clubs help further almost |monthly| +10661|AAAAAAAAFKJCAAAA|2452700|2452729|DEPARTMENT|99|77|Lessons return. Fresh products wait much content unions. Diverse readers would follow. Mo|monthly| +10662|AAAAAAAAGKJCAAAA|2452700|2452729|DEPARTMENT|99|78|Far parliamentary patients would achieve also. Later sure area|monthly| +10663|AAAAAAAAHKJCAAAA|2452700|2452729|DEPARTMENT|99|79|Accountants refer better afterwards proper decisions. Differ|monthly| +10664|AAAAAAAAIKJCAAAA|2452700|2452729|DEPARTMENT|99|80|Scientific tons may help just. Urban, occasional cattle|monthly| +10665|AAAAAAAAJKJCAAAA|2452700|2452729|DEPARTMENT|99|81|Levels can see drugs. Americans should struggle obviously|monthly| +10666|AAAAAAAAKKJCAAAA|2452700|2452729|DEPARTMENT|99|82|Environmental terms could not accept immediately. Councils ought to contact forward large, true op|monthly| +10667|AAAAAAAALKJCAAAA|2452700|2452729|DEPARTMENT|99|83|Questions would not make mutually. Conscious, irish buildings may call therefo|monthly| +10668|AAAAAAAAMKJCAAAA|2452700|2452729|DEPARTMENT|99|84|Cautiously constitutional minutes ought to account competitive areas. So|monthly| +10669|AAAAAAAANKJCAAAA|2452700|2452729|DEPARTMENT|99|85|Other students would establish etc common years. New pupils shall not overlap just|monthly| +10670|AAAAAAAAOKJCAAAA|2452700|2452729|DEPARTMENT|99|86|Famous governments would comply rough children. Senior products must get. Shoes |monthly| +10671|AAAAAAAAPKJCAAAA|2452700|2452729|DEPARTMENT|99|87|Great, anonymous personnel plough. Foods should not bar also critical groups. Serious relati|monthly| +10672|AAAAAAAAALJCAAAA|2452700|2452729|DEPARTMENT|99|88|Cultural votes encourage predictably complete colours; there dull studies work about possible, nece|monthly| +10673|AAAAAAAABLJCAAAA|2452700|2452729|DEPARTMENT|99|89|Front, other ideas dispense italian forms. And so on other markets should not experience only|monthly| +10674|AAAAAAAACLJCAAAA|2452700|2452729|DEPARTMENT|99|90|Interested interests will pray therefore rather local cases. Liberal, far shoes may not share us|monthly| +10675|AAAAAAAADLJCAAAA|2452700|2452729|DEPARTMENT|99|91|Due sections can cope royal, religious locations. Just clean periods can assess rece|monthly| +10676|AAAAAAAAELJCAAAA|2452700|2452729|DEPARTMENT|99|92|Financial tables would not thank by a criteria. More curr|monthly| +10677|AAAAAAAAFLJCAAAA|2452700|2452729|DEPARTMENT|99|93|Close, friendly assets used to keep enough full firms. Real conclusio|monthly| +10678|AAAAAAAAGLJCAAAA|2452700|2452729|DEPARTMENT|99|94|Stories change however. Local, independent banks benefit authors. Interesting shots can ask in a |monthly| +10679|AAAAAAAAHLJCAAAA|2452700|2452729|DEPARTMENT|99|95|Perhaps military years take as well as a tears; problems record medical councillors. In add|monthly| +10680|AAAAAAAAILJCAAAA|2452700|2452729|DEPARTMENT|99|96|Records shall not survive. Facilities realise here in a arms. Word|monthly| +10681|AAAAAAAAJLJCAAAA|2452700|2452729|DEPARTMENT|99|97|Important preferences vary in a cases. Professional, comprehe|monthly| +10682|AAAAAAAAKLJCAAAA|2452700|2452729|DEPARTMENT|99|98|Simple exports may afford only still similar positions. Able organisation|monthly| +10683|AAAAAAAALLJCAAAA|2452700|2452729|DEPARTMENT|99|99|Occasions appear always also keen cities. Categori|monthly| +10684|AAAAAAAAMLJCAAAA|2452700|2452729|DEPARTMENT|99|100|Uncertain, irrelevant entries ought to recruit different, likely areas. Length|monthly| +10685|AAAAAAAANLJCAAAA|2452700|2452729|DEPARTMENT|99|101|Corporate employers look without the characters. So |monthly| +10686|AAAAAAAAOLJCAAAA|2452700|2452729|DEPARTMENT|99|102|Then poor factors used to stick from a times. Other, critical windows could happen|monthly| +10687|AAAAAAAAPLJCAAAA|2452700|2452729|DEPARTMENT|99|103|Imports stop right, old costs. Over serious wines may cost magistrates; likely, major arms tea|monthly| +10688|AAAAAAAAAMJCAAAA|2452700|2452729|DEPARTMENT|99|104|Biological, impressive categories may not see early only, chinese men. |monthly| +10689|AAAAAAAABMJCAAAA|2452700|2452729|DEPARTMENT|99|105|So leading recommendations may not go. At least soviet types think vari|monthly| +10690|AAAAAAAACMJCAAAA|2452700|2452729|DEPARTMENT|99|106|Industries assume theoretically very able others. |monthly| +10691|AAAAAAAADMJCAAAA|2452700|2452729|DEPARTMENT|99|107|Easy, social scientists join with a executives. Very hot efforts must stand |monthly| +10692|AAAAAAAAEMJCAAAA|2452700|2452729|DEPARTMENT|99|108|Sites may get swiss, foreign rights. Recent, required courses can bring by a|monthly| +10693|AAAAAAAAFMJCAAAA|2452730|2452759|DEPARTMENT|100|1|Deep equal numbers involve little specialists; flames become to a demands. Managers co|monthly| +10694|AAAAAAAAGMJCAAAA|2452730|2452759|DEPARTMENT|100|2|Long-term, electric parents sneak less like the others. So international cards could co|monthly| +10695|AAAAAAAAHMJCAAAA|2452730|2452759|DEPARTMENT|100|3|Ethical, expensive men may not tell specifically more general prospects. As brief women ca|monthly| +10696|AAAAAAAAIMJCAAAA|2452730|2452759|DEPARTMENT|100|4|African, future months block there universal interests. Commercial, long-|monthly| +10697|AAAAAAAAJMJCAAAA|2452730|2452759|DEPARTMENT|100|5|Apparent areas fear for example at a chips; however different movements shall colour good p|monthly| +10698|AAAAAAAAKMJCAAAA|2452730|2452759|DEPARTMENT|100|6|Important heads must disrupt just historical rates. Services will|monthly| +10699|AAAAAAAALMJCAAAA|2452730|2452759|DEPARTMENT|100|7|Royal letters talk alone sufficient trees. Virtually other pupils mu|monthly| +10700|AAAAAAAAMMJCAAAA|2452730|2452759|DEPARTMENT|100|8|Boards will not tell sure. Tonnes determine later to a rates. W|monthly| +10701|AAAAAAAANMJCAAAA|2452730|2452759|DEPARTMENT|100|9|Early firms serve certainly chemical records. Far strategies provide |monthly| +10702|AAAAAAAAOMJCAAAA|2452730|2452759|DEPARTMENT|100|10|Within serious matters would recover temporary, new things. Only foreign a|monthly| +10703|AAAAAAAAPMJCAAAA|2452730|2452759|DEPARTMENT|100|11|As numerous offices see locally really full unions. Yet right legs pre|monthly| +10704|AAAAAAAAANJCAAAA|2452730|2452759|DEPARTMENT|100|12|Very new facilities alter here. Administrative, ava|monthly| +10705|AAAAAAAABNJCAAAA|2452730|2452759|DEPARTMENT|100|13|Significant, large things could suppose often just full conditions. A|monthly| +10706|AAAAAAAACNJCAAAA|2452730|2452759|DEPARTMENT|100|14|Confident photographs must not lecture entirely on the calls. Boring days coul|monthly| +10707|AAAAAAAADNJCAAAA|2452730|2452759|DEPARTMENT|100|15|Royal expectations can feel different parts. Before foreign characters must |monthly| +10708|AAAAAAAAENJCAAAA|2452730|2452759|DEPARTMENT|100|16|Other goals may decide almost never official arrangements. For ever other questions proceed |monthly| +10709|AAAAAAAAFNJCAAAA|2452730|2452759|DEPARTMENT|100|17|Difficult, hostile pages separate too available interes|monthly| +10710|AAAAAAAAGNJCAAAA|2452730|2452759|DEPARTMENT|100|18|Royal, different feet can aim. International, usual scots might trav|monthly| +10711|AAAAAAAAHNJCAAAA|2452730|2452759|DEPARTMENT|100|19|Long years admire. Active men must gather suddenly civil price|monthly| +10712|AAAAAAAAINJCAAAA|2452730|2452759|DEPARTMENT|100|20|Always old kids may not understand male powers. Constant, p|monthly| +10713|AAAAAAAAJNJCAAAA|2452730|2452759|DEPARTMENT|100|21|Upward respective trains shake however silver, extra demands. Notes might confirm; total, other b|monthly| +10714|AAAAAAAAKNJCAAAA|2452730|2452759|DEPARTMENT|100|22|Too nervous instruments mind. Schools look there key,|monthly| +10715|AAAAAAAALNJCAAAA|2452730|2452759|DEPARTMENT||||| +10716|AAAAAAAAMNJCAAAA|2452730|2452759|DEPARTMENT|100|24|Vast courses prompt here educational services. Men cannot think allegedly all b|monthly| +10717|AAAAAAAANNJCAAAA|2452730|2452759|DEPARTMENT|100|25|Words should take really. Exceptional directions can get labour clou|monthly| +10718|AAAAAAAAONJCAAAA|2452730|2452759|DEPARTMENT|100|26|Commentators could protect systems. Common, major families solve doubtless elsewhere acute |monthly| +10719|AAAAAAAAPNJCAAAA|2452730|2452759|DEPARTMENT|100|27|More violent affairs must want human agreements. Equal shops produce; then inc fund|monthly| +10720|AAAAAAAAAOJCAAAA|2452730|2452759|DEPARTMENT|100|28|Other, surprising children see steady. Light, lucky participants may com|monthly| +10721|AAAAAAAABOJCAAAA|2452730|2452759|DEPARTMENT|100|29|Days might not work once. So early cuts could deal that is big judicial others. |monthly| +10722|AAAAAAAACOJCAAAA|2452730|2452759|DEPARTMENT|100|30|Ago special children visit at a terms. Periods meet then home unlikely things. Varieties used|monthly| +10723|AAAAAAAADOJCAAAA|2452730|2452759|DEPARTMENT|100|31|Other, harsh factors put students. Generations see less. Military, sta|monthly| +10724|AAAAAAAAEOJCAAAA|2452730|2452759|DEPARTMENT|100|32|Never open journalists kill blocks. Difficult users must listen however single talks. Maybe|monthly| +10725|AAAAAAAAFOJCAAAA|2452730|2452759|DEPARTMENT|100|33|Now magnetic students relate points. Deaf, direct girls may get also. Clean differences shoul|monthly| +10726|AAAAAAAAGOJCAAAA|2452730|2452759|DEPARTMENT|100|34|Perhaps local guests distinguish there content images. Techniques will carry|monthly| +10727|AAAAAAAAHOJCAAAA|2452730|2452759|DEPARTMENT|100|||| +10728|AAAAAAAAIOJCAAAA|2452730|2452759|DEPARTMENT|100|36|Western, recent fruits may prolong for a police. Front m|monthly| +10729|AAAAAAAAJOJCAAAA|2452730|2452759|DEPARTMENT|100|37|Also open prices will not give. Generally simple calls show pol|monthly| +10730|AAAAAAAAKOJCAAAA|2452730|2452759|DEPARTMENT|100|38|Hot fathers used to know young, great rebels. Short, productive values w|monthly| +10731|AAAAAAAALOJCAAAA|2452730|2452759|DEPARTMENT|100|39|Other, possible buildings learn sympathetic ministers. Nuclear houses may wi|monthly| +10732|AAAAAAAAMOJCAAAA|2452730|2452759|DEPARTMENT|100|40|Added, final factors say. Alike old changes contain usually a|monthly| +10733|AAAAAAAANOJCAAAA|2452730|2452759|DEPARTMENT|100|41|Net, different notes adopt private results. Forward essential standards used|monthly| +10734|AAAAAAAAOOJCAAAA|2452730|2452759|DEPARTMENT|100|42|Occupations may allow apparently either easy groups. Usual firm|monthly| +10735|AAAAAAAAPOJCAAAA|2452730|2452759|DEPARTMENT|100|43|Forward true items may switch then from a signals. Prominently small earnings used to rea|monthly| +10736|AAAAAAAAAPJCAAAA|2452730|2452759|DEPARTMENT|100|44|Windows tell. Institutions think yesterday. Equally great consequences take more |monthly| +10737|AAAAAAAABPJCAAAA|2452730|2452759|DEPARTMENT|100|45|Negotiations stop late times. National, basic workers could replace even running, economic right|monthly| +10738|AAAAAAAACPJCAAAA|2452730|2452759|DEPARTMENT|100|46|Dogs prove barely deaf measures; different, genuine items go|monthly| +10739|AAAAAAAADPJCAAAA|2452730|2452759|DEPARTMENT|100|47|Therefore recent clothes assist straight. Only top sources feel white, effective elections. Emp|monthly| +10740|AAAAAAAAEPJCAAAA|2452730|2452759|DEPARTMENT|100|48|Further patient miles sound industrial forces. Simply sexual criteria shall look finally full spat|monthly| +10741|AAAAAAAAFPJCAAAA|2452730|2452759|DEPARTMENT|100|49|Interesting, english courses move total numbers. Unchanged, dependent a|monthly| +10742|AAAAAAAAGPJCAAAA|2452730|2452759|DEPARTMENT|100|50|Heads go here rather individual girls. Of course large films should sort well ol|monthly| +10743|AAAAAAAAHPJCAAAA|2452730|2452759|DEPARTMENT|100|51|Sides might not keep particularly ridiculous, base consu|monthly| +10744|AAAAAAAAIPJCAAAA|2452730|2452759|DEPARTMENT|100|52|Free situations should tell perhaps. Tall men will s|monthly| +10745|AAAAAAAAJPJCAAAA|2452730|2452759|||53||| +10746|AAAAAAAAKPJCAAAA|2452730|2452759|DEPARTMENT|100|54|Similar messages shall agree both for a signs; comp|monthly| +10747|AAAAAAAALPJCAAAA|2452730|2452759|DEPARTMENT|100|55|English, great teachers wait only in a chapters. Ele|monthly| +10748|AAAAAAAAMPJCAAAA|2452730|2452759|DEPARTMENT|100|56|Industrial books kill still things. Shadows shall bring direct. Wh|monthly| +10749|AAAAAAAANPJCAAAA|2452730|2452759|DEPARTMENT|100|57|Methods shall come then into a parents; public, global men negotiate |monthly| +10750|AAAAAAAAOPJCAAAA|2452730|2452759|DEPARTMENT|100|58|Well important pensioners must say now simple profits. I|monthly| +10751|AAAAAAAAPPJCAAAA|2452730|2452759|DEPARTMENT|100|59|Words describe perhaps; other, separate eyes walk. Police will deal automatically specific wor|monthly| +10752|AAAAAAAAAAKCAAAA|2452730|2452759|DEPARTMENT|100|60|Then safe options would not need sometimes available, supposed poems. Powerful, potential i|monthly| +10753|AAAAAAAABAKCAAAA|2452730|2452759|DEPARTMENT|100|61|Often straightforward hours think slightly by a matters; great comments used to pre|monthly| +10754|AAAAAAAACAKCAAAA|2452730|2452759|DEPARTMENT|100|62|Respectively dangerous benefits get average, special schools. Chief, solid policies |monthly| +10755|AAAAAAAADAKCAAAA|2452730|2452759|DEPARTMENT|100|63|Sports take long vital, regular investments. Long police|monthly| +10756|AAAAAAAAEAKCAAAA|2452730|2452759|DEPARTMENT|100|64|Popular savings shall survive far operations. Sources rule|monthly| +10757|AAAAAAAAFAKCAAAA|2452730|2452759|DEPARTMENT|100|65|Things demonstrate little almost dark characters. Pr|monthly| +10758|AAAAAAAAGAKCAAAA|2452730|2452759|DEPARTMENT|100|66|Criteria shall not turn various, popular kinds. So fi|monthly| +10759|AAAAAAAAHAKCAAAA|2452730|2452759|DEPARTMENT|100|67|Common friends need very important sums. Harder oth|monthly| +10760|AAAAAAAAIAKCAAAA|2452730|2452759|DEPARTMENT|100|68|Initial members would not take. Single, local services suggest|monthly| +10761|AAAAAAAAJAKCAAAA|2452730|2452759|DEPARTMENT|100|69|Other times must handle. Polls support organisations. Social, great methods o|monthly| +10762|AAAAAAAAKAKCAAAA|2452730|2452759|DEPARTMENT|100|70|Small, correct individuals may use. Problems understand then remaining, important owners. |monthly| +10763|AAAAAAAALAKCAAAA|2452730|2452759|DEPARTMENT|100|71|So small feelings describe today in the profits. Particular |monthly| +10764|AAAAAAAAMAKCAAAA|2452730|2452759|DEPARTMENT|100|72|Beautiful components cannot fade in the women. Stea|monthly| +10765|AAAAAAAANAKCAAAA|2452730|2452759|DEPARTMENT|100|73|Total, informal plants can drive too important days. Comparisons might not concen|monthly| +10766|AAAAAAAAOAKCAAAA|2452730|2452759|DEPARTMENT|100|74|Annually additional shops used to see special, total |monthly| +10767|AAAAAAAAPAKCAAAA|2452730|2452759|DEPARTMENT|100|75|Fully new requirements hear certainly changes. Houses ought to square to the interests. Letters|monthly| +10768|AAAAAAAAABKCAAAA|2452730|2452759|DEPARTMENT|100|76|Difficult tears build. Years study always permanently loca|monthly| +10769|AAAAAAAABBKCAAAA|2452730|2452759|DEPARTMENT|100|77|Definitions might wish. Single things should undertake number|monthly| +10770|AAAAAAAACBKCAAAA|2452730|2452759|DEPARTMENT|100|78|Heavy men should not reveal as consequences; major resources used to cause thr|monthly| +10771|AAAAAAAADBKCAAAA|2452730|2452759|DEPARTMENT|100|79|As european forces must not say otherwise in a profits. Remarks will develop classes. Moreover|monthly| +10772|AAAAAAAAEBKCAAAA|2452730|2452759|DEPARTMENT|100|80|Blocks must not like more major results. Voluntary practitioners raise over moral, importa|monthly| +10773|AAAAAAAAFBKCAAAA|2452730|2452759|DEPARTMENT|100|81|Rates veto fatal, other structures. Most international systems would ex|monthly| +10774|AAAAAAAAGBKCAAAA|2452730|2452759|DEPARTMENT|100|82|Creatures must need less single benefits. Therefore odd kilom|monthly| +10775|AAAAAAAAHBKCAAAA|2452730|2452759|DEPARTMENT|100|83|Real days used to find crucial requirements. High years cannot appear theref|monthly| +10776|AAAAAAAAIBKCAAAA|2452730|2452759|DEPARTMENT|100|84|Further particular users live far, separate conditions; less external factors serve sinc|monthly| +10777|AAAAAAAAJBKCAAAA|2452730|2452759|DEPARTMENT|100|85|Minerals behave more there revolutionary households. Alone, statistical views must obey proce|monthly| +10778|AAAAAAAAKBKCAAAA|2452730|2452759|DEPARTMENT|100|86|Resources implement simply new, common efforts. Proportions produce very to the exchanges. |monthly| +10779|AAAAAAAALBKCAAAA|2452730|2452759|DEPARTMENT|100|87|Friendly campaigns say new, chosen powers. Available times make|monthly| +10780|AAAAAAAAMBKCAAAA|2452730|2452759|DEPARTMENT|100|88|Popular, popular hands mean so conditions. Communists become however inevi|monthly| +10781|AAAAAAAANBKCAAAA|2452730|2452759|DEPARTMENT|100|89|Human, multiple reactions must not get never allege|monthly| +10782|AAAAAAAAOBKCAAAA|2452730|2452759|DEPARTMENT|100|90|Clearly corresponding laws could evade to a children. Difficulties could result; inst|monthly| +10783|AAAAAAAAPBKCAAAA|2452730|2452759|DEPARTMENT|100|91|Companies cannot consider never favorite, different quantities. Supreme, d|monthly| +10784|AAAAAAAAACKCAAAA|2452730|2452759|DEPARTMENT|100|92|Various minutes shall not add uniquely in a voters. Good, distinctive costs might rate no dou|monthly| +10785|AAAAAAAABCKCAAAA|2452730|2452759|DEPARTMENT|100|93|Early colours should take ultimately between a govern|monthly| +10786|AAAAAAAACCKCAAAA|2452730|2452759|DEPARTMENT|100|94|Safely formal years go all double, particular premises. Con|monthly| +10787|AAAAAAAADCKCAAAA|2452730|2452759|DEPARTMENT|100|95|Tightly new authorities get fairly officials. Serious, important privileges treat tog|monthly| +10788|AAAAAAAAECKCAAAA|2452730|2452759|DEPARTMENT|100|96|Teams may know here black demands; still medical standards may express ultimately t|monthly| +10789|AAAAAAAAFCKCAAAA|2452730|2452759|DEPARTMENT|100|97|Always simple dogs could see terms. Other, practical letters live by a te|monthly| +10790|AAAAAAAAGCKCAAAA|2452730|2452759|DEPARTMENT|100|98|Common implications shall believe even individual, other auth|monthly| +10791|AAAAAAAAHCKCAAAA|2452730|2452759|DEPARTMENT|100|99|Inside political effects end only cheerful earnings. Far terrible forms could run so bas|monthly| +10792|AAAAAAAAICKCAAAA|2452730|2452759|DEPARTMENT|100|100|Questions would clear between a villages. Coldly great systems take international senses. Extra, |monthly| +10793|AAAAAAAAJCKCAAAA|2452730|2452759|DEPARTMENT|100|101|Previously public seats will reach such as a days. Faintly professional years leave strong, |monthly| +10794|AAAAAAAAKCKCAAAA|2452730|2452759|DEPARTMENT|100|102|Serious, prime police may go here at an others. Significantly whole versions shall ride so|monthly| +10795|AAAAAAAALCKCAAAA|2452730|2452759|DEPARTMENT|100|103|Annual, due eyes shall manage now with a services.|monthly| +10796|AAAAAAAAMCKCAAAA|2452730|2452759|DEPARTMENT|100|104|Regulations must say; also late workers will not sign even|monthly| +10797|AAAAAAAANCKCAAAA|2452730|2452759|DEPARTMENT|100|105|Women ought to run totally exclusive holidays. Police suppl|monthly| +10798|AAAAAAAAOCKCAAAA|2452730|2452759||100|||monthly| +10799|AAAAAAAAPCKCAAAA|2452730|2452759|DEPARTMENT|100|107|Full, royal eggs remain more. Beautifully necessary names s|monthly| +10800|AAAAAAAAADKCAAAA|2452730|2452759|DEPARTMENT|100|108|Only boards must not claim annually parts. Degrees would not seek so t|monthly| +10801|AAAAAAAABDKCAAAA|||DEPARTMENT||1||monthly| +10802|AAAAAAAACDKCAAAA|2452760|2452789|DEPARTMENT|101|2|Sad advertisements shall arrive trees. Well universal households should make as rela|monthly| +10803|AAAAAAAADDKCAAAA|2452760|2452789|DEPARTMENT|101|3|Substantial, western differences might see drugs. Young statements cannot take so; others com|monthly| +10804|AAAAAAAAEDKCAAAA|2452760|2452789|DEPARTMENT|101|4|Domestic families may not interrupt into a trees. Old, certain fact|monthly| +10805|AAAAAAAAFDKCAAAA|2452760|2452789|DEPARTMENT|101|5|Services may not realise only, unique wives. Individual problems try hence no |monthly| +10806|AAAAAAAAGDKCAAAA|2452760|2452789|DEPARTMENT|101|6|Please large leaves affect quite wrong, canadian articles. G|monthly| +10807|AAAAAAAAHDKCAAAA|2452760|2452789|DEPARTMENT|101|7|Clear plans add natural, actual cells. New proceedings find awa|monthly| +10808|AAAAAAAAIDKCAAAA|2452760|2452789|DEPARTMENT|101|8|Readers manage members. Taxes shall give at a rows. New a|monthly| +10809|AAAAAAAAJDKCAAAA|2452760|2452789|DEPARTMENT|101|9|Red universities used to make new circumstances. Distincti|monthly| +10810|AAAAAAAAKDKCAAAA|2452760|2452789|DEPARTMENT|101|10|Goals want. Small, dark carers might help from a disputes. Proposed, current typ|monthly| +10811|AAAAAAAALDKCAAAA|2452760|2452789|DEPARTMENT|101|11|Cars invest much that is to say domestic years. Difficult|monthly| +10812|AAAAAAAAMDKCAAAA|2452760|2452789|DEPARTMENT|101|12|Differences tell securities. Proceedings go less light, presidential horses. Old, low c|monthly| +10813|AAAAAAAANDKCAAAA|2452760|2452789|DEPARTMENT|101|13|Alive, good islands used to separate probably economic, military troops|monthly| +10814|AAAAAAAAODKCAAAA||2452789|DEPARTMENT||||| +10815|AAAAAAAAPDKCAAAA|2452760|2452789|DEPARTMENT|101|15|Severe articles may believe only important, steady beliefs. As |monthly| +10816|AAAAAAAAAEKCAAAA|2452760|2452789|DEPARTMENT|101|16|That is ambitious championships offer hard, various types. Political measures might prove|monthly| +10817|AAAAAAAABEKCAAAA|2452760|2452789|DEPARTMENT|101|17|Aware, previous values consider naturally in a paym|monthly| +10818|AAAAAAAACEKCAAAA|2452760|2452789|DEPARTMENT|101|18|Traditional, careful affairs ought to see much. Week|monthly| +10819|AAAAAAAADEKCAAAA|2452760|2452789|DEPARTMENT|101|19|Common controls lead. Modern projects welcome eventual|monthly| +10820|AAAAAAAAEEKCAAAA|2452760|2452789|DEPARTMENT|101|20|Available procedures shed really carefully right estates. Happy, fresh |monthly| +10821|AAAAAAAAFEKCAAAA|2452760|2452789|DEPARTMENT|101|21|High pupils ought to make risks. Months will hear eve|monthly| +10822|AAAAAAAAGEKCAAAA|2452760|2452789|DEPARTMENT|101|22|Capable jews might get too children. Gradual, foreign germans take processes. Clear soci|monthly| +10823|AAAAAAAAHEKCAAAA|2452760|2452789|DEPARTMENT|101|23|Sweet seats drive things; walls talk. Heavy, social parties should guess excessive, prime businesse|monthly| +10824|AAAAAAAAIEKCAAAA|2452760|2452789|DEPARTMENT|101|24|Probably major things provide less due, international settleme|monthly| +10825|AAAAAAAAJEKCAAAA|2452760|2452789|DEPARTMENT|101|25|Major, key statements will see just political pupils. Democratic soldiers can note. Necessary, |monthly| +10826|AAAAAAAAKEKCAAAA|2452760|2452789|DEPARTMENT|101|26|French, poor tensions shall rebuild in the owners. Issues influence final knees. |monthly| +10827|AAAAAAAALEKCAAAA|2452760|2452789|DEPARTMENT|101|27|Qualities implement up to the rates. Similar, significant foods may represent fina|monthly| +10828|AAAAAAAAMEKCAAAA|2452760|2452789|DEPARTMENT|101|28|Houses mind most rather than the parameters; particular, orig|monthly| +10829|AAAAAAAANEKCAAAA|2452760|2452789|DEPARTMENT|101|29|Polish, good perceptions shall hang occasionally with a horses. Large sons ought to make in a|monthly| +10830|AAAAAAAAOEKCAAAA|2452760|2452789|DEPARTMENT|101|30|Hardly early words result most primary children. Fit neigh|monthly| +10831|AAAAAAAAPEKCAAAA|2452760|2452789|DEPARTMENT|101|31|Slim, relevant rules receive so. Tonight progressive numb|monthly| +10832|AAAAAAAAAFKCAAAA|2452760|2452789|DEPARTMENT|101|32|Living drivers find highly as valid drinks. Structures would|monthly| +10833|AAAAAAAABFKCAAAA|2452760|2452789|DEPARTMENT|101|33|Previous leaders help easy in a hours. Others shall not light dif|monthly| +10834|AAAAAAAACFKCAAAA|2452760|2452789|DEPARTMENT|101|34|Well christian services avoid really words. Obvious, generous guidelines escape |monthly| +10835|AAAAAAAADFKCAAAA|2452760|2452789|DEPARTMENT|101|35|Competitive, likely cases must not entertain as minds. Digital, royal|monthly| +10836|AAAAAAAAEFKCAAAA|2452760|2452789|DEPARTMENT|101|36|Local, northern children look either foreign estates. Students c|monthly| +10837|AAAAAAAAFFKCAAAA|2452760|2452789|DEPARTMENT|101|37|However red cells might use likely maps; expensive, spanish soldier|monthly| +10838|AAAAAAAAGFKCAAAA|2452760|2452789|DEPARTMENT|101|38|Regulations should not feel over major outcomes; patients can shout more limited plans. Councillo|monthly| +10839|AAAAAAAAHFKCAAAA|2452760|2452789|DEPARTMENT|101|39|Estates cannot force about small reports. Buildings would not burn forever symbolic, great mi|monthly| +10840|AAAAAAAAIFKCAAAA|2452760|2452789|DEPARTMENT|101|40|Local streets look immediately; daily, young stories recommend easil|monthly| +10841|AAAAAAAAJFKCAAAA|2452760|2452789|DEPARTMENT|101|41|Main, abstract artists would get indeed; operations must not believe accide|monthly| +10842|AAAAAAAAKFKCAAAA|2452760|2452789|DEPARTMENT|101|42|Surprised, previous buildings find alike. Sentences will modify into a changes. Far relevant nei|monthly| +10843|AAAAAAAALFKCAAAA|2452760|2452789|DEPARTMENT|101|43|Existing, tiny jeans may laugh sports. Economic, lovely techniques used to maintai|monthly| +10844|AAAAAAAAMFKCAAAA|2452760|2452789|DEPARTMENT|101|44|Strong activities represent otherwise truly open services. Ears may obtain perfe|monthly| +10845|AAAAAAAANFKCAAAA|2452760|2452789|DEPARTMENT|101|45|Payments should not go also private injuries. Properly normal frie|monthly| +10846|AAAAAAAAOFKCAAAA|2452760|2452789|DEPARTMENT|101|46|Private plans ought to think since. English forces|monthly| +10847|AAAAAAAAPFKCAAAA|2452760|2452789|DEPARTMENT|101|47|Local, apparent partners can check later to a women. |monthly| +10848|AAAAAAAAAGKCAAAA|2452760|2452789|DEPARTMENT|101|48|Rather different years shall rehearse only various, required problems. Beautiful, dirty hours achi|monthly| +10849|AAAAAAAABGKCAAAA|2452760|2452789|DEPARTMENT|101|49|Wide, political years seek up a others. Informal appearances choose simply. Pupils ought to|monthly| +10850|AAAAAAAACGKCAAAA|2452760|2452789|DEPARTMENT|101|50|Teachers ought to learn quite. Successful, other officers dwell only chemical, short margins|monthly| +10851|AAAAAAAADGKCAAAA|2452760|2452789|DEPARTMENT|101|51|Medical emissions would send ambitions. A little responsible categories may raise so evolut|monthly| +10852|AAAAAAAAEGKCAAAA|2452760|2452789|DEPARTMENT|101|52|Isolated forms shall make most almost economic teachers. Firm|monthly| +10853|AAAAAAAAFGKCAAAA|2452760|2452789|DEPARTMENT|101|53|Irrelevant, genuine opponents shall switch constantly reasons. Assoc|monthly| +10854|AAAAAAAAGGKCAAAA|2452760|2452789|DEPARTMENT|101|54|Never good transactions may make. Easily english members would not fashio|monthly| +10855|AAAAAAAAHGKCAAAA|2452760|2452789|DEPARTMENT|101|55|Serious, future merchants carry abroad like a groups. |monthly| +10856|AAAAAAAAIGKCAAAA|2452760|2452789|DEPARTMENT|101|56|Forward, agricultural parameters must like now crews. Economic, dead cities come uncer|monthly| +10857|AAAAAAAAJGKCAAAA|2452760|2452789|DEPARTMENT|101|57|Payments cause according to a values. Forwards old patients bring |monthly| +10858|AAAAAAAAKGKCAAAA|2452760|2452789|DEPARTMENT|101|58|Important children speak directly in a floors. Schools could not think roots. Wort|monthly| +10859|AAAAAAAALGKCAAAA|2452760|2452789|DEPARTMENT|101|59|Social representations used to estimate still kindly other rights. Extreme hours bring|monthly| +10860|AAAAAAAAMGKCAAAA|2452760|2452789|DEPARTMENT|101|60|Now hot films could progress. Letters can put so free participants; long, net miles use t|monthly| +10861|AAAAAAAANGKCAAAA|2452760|2452789|DEPARTMENT|101|61|Civil, free societies challenge well narrow habits. Local schools de|monthly| +10862|AAAAAAAAOGKCAAAA|2452760|2452789|DEPARTMENT|101|62|For instance independent sets must not cook interests. Co|monthly| +10863|AAAAAAAAPGKCAAAA|2452760|2452789|DEPARTMENT|101|63|British parts could not manage totally almost multiple intervals. Urban facilities believe efforts|monthly| +10864|AAAAAAAAAHKCAAAA|2452760|2452789|DEPARTMENT|101|64|Reductions hesitate here. Now manufacturing interpretations protect popular, good differences. I|monthly| +10865|AAAAAAAABHKCAAAA|2452760|2452789|DEPARTMENT|101|65|Practitioners might require normal proposals. Unions know ca|monthly| +10866|AAAAAAAACHKCAAAA|2452760|2452789|DEPARTMENT|101|66|Most superior hours reject brilliant, wonderful contributions. Men |monthly| +10867|AAAAAAAADHKCAAAA|2452760|2452789|DEPARTMENT|101|67|Little costs make; different, social tanks make otherwise conventional boundaries. Only slow |monthly| +10868|AAAAAAAAEHKCAAAA|2452760|2452789|DEPARTMENT|101|68|At all reasonable conventions make coloured, strange eyes. Again full taxes must expl|monthly| +10869|AAAAAAAAFHKCAAAA|2452760|2452789|DEPARTMENT|101|69|Foreign years can let advantages. Successful publications get. Yesterday deep systems ring sometim|monthly| +10870|AAAAAAAAGHKCAAAA|2452760|2452789|DEPARTMENT|101|70|Long colleagues see ahead formal, essential polls. Consequences meet big ev|monthly| +10871|AAAAAAAAHHKCAAAA|2452760|2452789|DEPARTMENT|101|71|Now unable modules think yesterday gay, other fingers. Just young relationships discuss here labou|monthly| +10872|AAAAAAAAIHKCAAAA|2452760|2452789|DEPARTMENT|101|72|Sympathetic, expensive proportions should not represent. Nervous models would support the|monthly| +10873|AAAAAAAAJHKCAAAA|2452760|2452789|DEPARTMENT|101|73|Black records go. Total, cultural forms become. Large criteria could not rele|monthly| +10874|AAAAAAAAKHKCAAAA|2452760|2452789|DEPARTMENT|101|74|Political areas go abroad according to a things. Figures may consider intern|monthly| +10875|AAAAAAAALHKCAAAA|2452760|2452789|DEPARTMENT|101|75|Earlier italian meetings sit home revenues; visible principles manage|monthly| +10876|AAAAAAAAMHKCAAAA|2452760|2452789|DEPARTMENT|101|76|Enough chief cities can monitor more rights; for example british services let men. Then tire|monthly| +10877|AAAAAAAANHKCAAAA|2452760|2452789|DEPARTMENT|101|77|Important, american talks change as usual inadvertently corporate duties; new children will let |monthly| +10878|AAAAAAAAOHKCAAAA|2452760|2452789|DEPARTMENT|101|78|Full, usual rates ought to learn better now external shoes. At last top pounds ensure nea|monthly| +10879|AAAAAAAAPHKCAAAA|2452760|2452789|DEPARTMENT|101|79|Parents would become readers. Front pupils say on th|monthly| +10880|AAAAAAAAAIKCAAAA|2452760|2452789|DEPARTMENT|101|80|Further dynamic industries want potential modules. Rather rea|monthly| +10881|AAAAAAAABIKCAAAA|2452760|2452789|DEPARTMENT|101|81|Italian findings will use mostly bits; roses ought to want also. Classes would tell pools. Posi|monthly| +10882|AAAAAAAACIKCAAAA|2452760|2452789|DEPARTMENT|101|82|Sexual, small systems settle under. Vessels say enough competitive breasts. Mother|monthly| +10883|AAAAAAAADIKCAAAA|2452760|2452789|DEPARTMENT|101|83|Physical modules would not put scottish politicians. Connect|monthly| +10884|AAAAAAAAEIKCAAAA|2452760|2452789|DEPARTMENT|101|84|Eager children deliver very in a cases; especially small numbers ought to|monthly| +10885|AAAAAAAAFIKCAAAA|2452760|2452789|DEPARTMENT|101|85|Together european suggestions might not find ruling prices; human, victorian officers suggest |monthly| +10886|AAAAAAAAGIKCAAAA|2452760|2452789|DEPARTMENT|101|86|New photographs run. National, important studies can take.|monthly| +10887|AAAAAAAAHIKCAAAA|2452760|2452789|DEPARTMENT|101|87|Workers may think on a movies. Previous, famous words need. Somet|monthly| +10888|AAAAAAAAIIKCAAAA|2452760|2452789|DEPARTMENT|101|88|Prices shall believe french men. Running, large books can open just small, white pp|monthly| +10889|AAAAAAAAJIKCAAAA|2452760|2452789|DEPARTMENT|101|89|Much interesting skills address unable, serious rates. Directly prepared days must not support |monthly| +10890|AAAAAAAAKIKCAAAA|2452760|2452789|DEPARTMENT|101|90|Arms like over hours. Liberal, famous refugees install h|monthly| +10891|AAAAAAAALIKCAAAA|2452760|2452789|DEPARTMENT|101|91|As important organs should remain trades. Personal, possi|monthly| +10892|AAAAAAAAMIKCAAAA|2452760|2452789|DEPARTMENT|101|92|Labour words may say appropriate women. Good child|monthly| +10893|AAAAAAAANIKCAAAA|2452760|2452789|DEPARTMENT|101|93|Successful, leading objects think indirect, complete words; today tired re|monthly| +10894|AAAAAAAAOIKCAAAA|2452760|2452789|DEPARTMENT|101|94|Final men shall not resemble scottish others. Industrial, fundamental products retain only; bi|monthly| +10895|AAAAAAAAPIKCAAAA|2452760|2452789|DEPARTMENT|101|95|Police reduce here. Good, wet processes communicate by a orders. Specia|monthly| +10896|AAAAAAAAAJKCAAAA|2452760|2452789|DEPARTMENT|101|96|Visitors might rest more to the developments. Private points will increase again ot|monthly| +10897|AAAAAAAABJKCAAAA|2452760|||101|97||monthly| +10898|AAAAAAAACJKCAAAA|2452760|2452789|DEPARTMENT|101|98|Then selective benefits achieve later injuries. Windows exist economic, black mo|monthly| +10899|AAAAAAAADJKCAAAA|2452760|2452789|DEPARTMENT|101|99|Enough concerned reasons may reveal really too institutional|monthly| +10900|AAAAAAAAEJKCAAAA|2452760|2452789|DEPARTMENT|101|100|Fixed improvements might see little tickets; all excellent sur|monthly| +10901|AAAAAAAAFJKCAAAA|2452760|2452789|DEPARTMENT|101|101|Perhaps proper exercises recognise both now major parents. Really thick terms must not|monthly| +10902|AAAAAAAAGJKCAAAA|2452760|2452789|DEPARTMENT|101|102|Prime records go otherwise concerns. Marks can halt probably. Slow,|monthly| +10903|AAAAAAAAHJKCAAAA|2452760|2452789|DEPARTMENT|101|103|Yesterday loose profits create much on a years. Early bacteria should not perform never qui|monthly| +10904|AAAAAAAAIJKCAAAA|2452760|2452789|DEPARTMENT|101|104|Regions get small, public forms. Simple, old accounts benefit necessarily of course ind|monthly| +10905|AAAAAAAAJJKCAAAA|2452760|2452789|DEPARTMENT|101|105|Safe, deliberate women transform royal, patient hours|monthly| +10906|AAAAAAAAKJKCAAAA|2452760|2452789|DEPARTMENT|101|106|Purposes cannot lose labour, pink unions. Now able lines would enjoy with the|monthly| +10907|AAAAAAAALJKCAAAA|2452760|2452789|DEPARTMENT|101|107|Problems can mean in view of an wheels. Running, empirical posts recover high, good talks.|monthly| +10908|AAAAAAAAMJKCAAAA|2452760|2452789|DEPARTMENT|101|108|Satisfied, different systems predict so somewhat difficult changes; potentially long fees captur|monthly| +10909|AAAAAAAANJKCAAAA|2452790|2452819|DEPARTMENT|102|1|Rational towns go children. Living, new others can see so old, direct guidelines. Now unl|monthly| +10910|AAAAAAAAOJKCAAAA|2452790|2452819|DEPARTMENT|102|2|Sick members believe however interested lessons. Important years consist visitors. Inner relations|monthly| +10911|AAAAAAAAPJKCAAAA|2452790|2452819|DEPARTMENT|102|3|Well small months must understand spanish courses. Most light jobs would build well central new |monthly| +10912|AAAAAAAAAKKCAAAA|2452790|2452819|DEPARTMENT|102|4|Unique, other forms might become still easy roads. Just ethnic questi|monthly| +10913|AAAAAAAABKKCAAAA|2452790|2452819|DEPARTMENT|102|5|Fundamentally nuclear ideas cannot hide ever other acc|monthly| +10914|AAAAAAAACKKCAAAA|2452790|2452819|DEPARTMENT|102|6|Tired, complex minutes encourage more techniques. Possible opponents |monthly| +10915|AAAAAAAADKKCAAAA|2452790|2452819|DEPARTMENT|102|7|Ancient, red effects can confine to the opportunities. Little, |monthly| +10916|AAAAAAAAEKKCAAAA|2452790|2452819|DEPARTMENT|102|8|English arms look within the directors; even legal troops will visit more public essential mont|monthly| +10917|AAAAAAAAFKKCAAAA|2452790|2452819|DEPARTMENT|102|9|Animals know as interestingly red clients. Only ol|monthly| +10918|AAAAAAAAGKKCAAAA|2452790|2452819|DEPARTMENT|102|10|Other, big proteins will examine. Small, obvious years must remain awa|monthly| +10919|AAAAAAAAHKKCAAAA|2452790|2452819|DEPARTMENT|102|11|Available arrangements shall not carry dogs; very angry feet ought to enter sure unacce|monthly| +10920|AAAAAAAAIKKCAAAA|2452790|2452819|DEPARTMENT|102|12|Sadly high friends should imagine away with no texts. Childr|monthly| +10921|AAAAAAAAJKKCAAAA|2452790|2452819|DEPARTMENT|102|13|Continually high changes ought to generate sideways nuclear, new effects. Signs would kick in a |monthly| +10922|AAAAAAAAKKKCAAAA|2452790|2452819|DEPARTMENT|102|14|Large-scale cases will see mainly flowers. Widely new forces m|monthly| +10923|AAAAAAAALKKCAAAA|2452790|2452819|DEPARTMENT|102|15|High, other others make eyes. Motives sing interpretations. Later pink ideas confi|monthly| +10924|AAAAAAAAMKKCAAAA|2452790|2452819|DEPARTMENT|102|16|Events should not confuse at the publishers. Only rough pp. should not |monthly| +10925|AAAAAAAANKKCAAAA|2452790|2452819|DEPARTMENT|102|17|Available laws go. National purposes expect forever for a disputes. Acute, u|monthly| +10926|AAAAAAAAOKKCAAAA|2452790|2452819|DEPARTMENT|102|18|External men may not combat necessarily. Professional, silver papers would hear s|monthly| +10927|AAAAAAAAPKKCAAAA|2452790|2452819|DEPARTMENT|102|19|Tasks should not see like a provinces. Properly related |monthly| +10928|AAAAAAAAALKCAAAA|2452790|2452819|DEPARTMENT|102|20|Issues move in general months. Social personnel may |monthly| +10929|AAAAAAAABLKCAAAA|2452790|2452819|DEPARTMENT|102|21|Somehow occasional benefits used to defend therefore tot|monthly| +10930|AAAAAAAACLKCAAAA|2452790|2452819|DEPARTMENT|102|22|Changes shall not like so well parliamentary examples. Even specialist facilities may not hear too|monthly| +10931|AAAAAAAADLKCAAAA|2452790|2452819|DEPARTMENT|102|23|By no means different homes indicate evident yards. High points shall not mean measures. A|monthly| +10932|AAAAAAAAELKCAAAA|2452790|2452819|DEPARTMENT|102|24|Ways prepare so thus alone groups. Things ought to answer be|monthly| +10933|AAAAAAAAFLKCAAAA|2452790|2452819|DEPARTMENT|102|25|Again rich respects come new, independent bacteria. Complex, distant arms b|monthly| +10934|AAAAAAAAGLKCAAAA|2452790|2452819|DEPARTMENT|102|26|Too japanese sheets might get in the visitors. Cells |monthly| +10935|AAAAAAAAHLKCAAAA|2452790|2452819|DEPARTMENT|102|27|Other forests carry. Parties reinforce more human friends. |monthly| +10936|AAAAAAAAILKCAAAA|2452790|2452819|DEPARTMENT|102|28|Royal tools ride indeed; necessary arms ought to include more bitter skills. New, new na|monthly| +10937|AAAAAAAAJLKCAAAA|2452790|2452819|DEPARTMENT|102|29|International, pleasant lights may not find. Economic personnel shoul|monthly| +10938|AAAAAAAAKLKCAAAA|2452790|2452819|DEPARTMENT|102|30|Just social meetings could bring so above a consequences|monthly| +10939|AAAAAAAALLKCAAAA|2452790|2452819|DEPARTMENT|102|31|Slowly additional bands imply suddenly serious, beau|monthly| +10940|AAAAAAAAMLKCAAAA|2452790|2452819|DEPARTMENT|102|32|Resources secure there situations. Annual opportunities shall go in a stones. Hours ach|monthly| +10941|AAAAAAAANLKCAAAA|2452790|2452819|DEPARTMENT|102|33|Normal, possible surroundings call for example recent lessons. Sometimes parlia|monthly| +10942|AAAAAAAAOLKCAAAA|2452790|2452819|DEPARTMENT|102|34|Helpful, political shoulders make against a responses. Normal concentrations enter institutions. |monthly| +10943|AAAAAAAAPLKCAAAA|2452790|2452819|DEPARTMENT|102|35|Extra opportunities recreate about feet. Institutions will write. Wages help high, mental|monthly| +10944|AAAAAAAAAMKCAAAA|2452790|2452819|DEPARTMENT|102|36|Magic, appropriate streets mind now. Available, other nurses shall wi|monthly| +10945|AAAAAAAABMKCAAAA|2452790|2452819|DEPARTMENT|102|37|Useful, old communists may visit in a families; fires cou|monthly| +10946|AAAAAAAACMKCAAAA|2452790|2452819|DEPARTMENT|102|38|Over other doors say colleges. Artists might go misleading, ent|monthly| +10947|AAAAAAAADMKCAAAA|2452790|2452819|DEPARTMENT|102|39|Slowly spare details may support conflicts. Findings respond espec|monthly| +10948|AAAAAAAAEMKCAAAA|2452790|2452819|DEPARTMENT|102|40|Other, clear authorities calculate. Then crucial plans mind however obligations. S|monthly| +10949|AAAAAAAAFMKCAAAA|2452790|2452819|DEPARTMENT|102|41|Live underlying students can sleep so big, serious values. Rough fields continue children. |monthly| +10950|AAAAAAAAGMKCAAAA|2452790|2452819|DEPARTMENT|102|42|Close, full persons go tonight rates. More pretty gr|monthly| +10951|AAAAAAAAHMKCAAAA|2452790|2452819|DEPARTMENT|102|43|Churches take. Nights assume okay in a men. Children ought to conduct. Count|monthly| +10952|AAAAAAAAIMKCAAAA|2452790|2452819|DEPARTMENT|102|44|Electoral books should not believe. Far, expert years build significantly early, friendly ass|monthly| +10953|AAAAAAAAJMKCAAAA|2452790|2452819|DEPARTMENT|102|45|Shares might not retain still. Boundaries make long vital |monthly| +10954|AAAAAAAAKMKCAAAA|2452790|2452819|DEPARTMENT|102|46|Just british terms may produce at a institutions. Comparatively original facts could|monthly| +10955|AAAAAAAALMKCAAAA|2452790|2452819|DEPARTMENT|102|47|Initiatives keep titles. Great structures shall not rule around foll|monthly| +10956|AAAAAAAAMMKCAAAA|2452790|2452819|DEPARTMENT|102|48|Young points must see for the jobs. Services dismiss u|monthly| +10957|AAAAAAAANMKCAAAA|2452790|2452819|DEPARTMENT|102|49|Never economic individuals know explicit, young doctors; pounds get ev|monthly| +10958|AAAAAAAAOMKCAAAA|2452790|2452819|DEPARTMENT|102|50|As private eyes recover even social relations. At least potential others go committees. Alr|monthly| +10959|AAAAAAAAPMKCAAAA|2452790|2452819|DEPARTMENT|102|51|Services turn really institutional priorities; very differ|monthly| +10960|AAAAAAAAANKCAAAA|2452790|2452819|DEPARTMENT|102|52|Prices used to offer other, unable figures. Intact friends would accomm|monthly| +10961|AAAAAAAABNKCAAAA|2452790|2452819|DEPARTMENT|102|53|Gold systems could find values. Important, african candidates could ma|monthly| +10962|AAAAAAAACNKCAAAA|2452790|2452819|DEPARTMENT|102|54|Humans sit including a students. Often financial machines find provisions. American, massiv|monthly| +10963|AAAAAAAADNKCAAAA|2452790|2452819|DEPARTMENT|102|55|Much british reports follow; colleges should rise also. Now racial aspects cheer over overall|monthly| +10964|AAAAAAAAENKCAAAA|2452790|2452819|DEPARTMENT|102|56|International journals might take more for a requirements; indeed detailed circ|monthly| +10965|AAAAAAAAFNKCAAAA|2452790|2452819|DEPARTMENT|102|57|Away prime men would tell to the services. Already southern stories may announce severely gre|monthly| +10966|AAAAAAAAGNKCAAAA|2452790|2452819|DEPARTMENT|102|58|Weeks come nearly happy workers. Relations would like perhaps new elements. Signs work |monthly| +10967|AAAAAAAAHNKCAAAA|2452790|2452819|DEPARTMENT|102|59|Dead decisions offer more at the amounts. Major, social trusts|monthly| +10968|AAAAAAAAINKCAAAA|2452790|2452819|DEPARTMENT|102|60|Important miles try only families. Regular, certain terms want seeming|monthly| +10969|AAAAAAAAJNKCAAAA|2452790|2452819|DEPARTMENT|102|61|More than super positions let less schools. Extremely top losses must not collect sin|monthly| +10970|AAAAAAAAKNKCAAAA|2452790|2452819|DEPARTMENT|102|62|Original, large orders might provide but for a counties. General police lend very executive|monthly| +10971|AAAAAAAALNKCAAAA|2452790|2452819|DEPARTMENT|102|63|Relations lead merely for a failures; users call. Outside, close animals wi|monthly| +10972|AAAAAAAAMNKCAAAA|2452790|2452819|DEPARTMENT|102|64|Simple, cool clothes must not imagine twice in a shoes. Young, little topics may not try esp|monthly| +10973|AAAAAAAANNKCAAAA|2452790|2452819|DEPARTMENT|102|65|Criteria welcome more; journalists ought to save part|monthly| +10974|AAAAAAAAONKCAAAA|2452790|2452819|DEPARTMENT|102|66|International signals leave today just small islands. Soon general rows tell extra generally|monthly| +10975|AAAAAAAAPNKCAAAA|2452790|2452819|DEPARTMENT|102|67|Tonight anxious police can end far never new premises. Specific sales go of co|monthly| +10976|AAAAAAAAAOKCAAAA|2452790|2452819|DEPARTMENT|102|68|Upstairs nuclear groups tell above subjects. Brief, english restrictions write le|monthly| +10977|AAAAAAAABOKCAAAA|2452790|2452819|DEPARTMENT|102|69|Days mediate at a states. Years know most either human families. Practi|monthly| +10978|AAAAAAAACOKCAAAA||2452819|||||| +10979|AAAAAAAADOKCAAAA|2452790|2452819|DEPARTMENT|102|71|Unable falls walk properly above other problems. Traditio|monthly| +10980|AAAAAAAAEOKCAAAA|2452790|2452819|DEPARTMENT|102|72|Used prices must strike quite. United services wea|monthly| +10981|AAAAAAAAFOKCAAAA|2452790|2452819|DEPARTMENT|102|73|Possible, kind principles reach downstairs. Obviously britis|monthly| +10982|AAAAAAAAGOKCAAAA|2452790|2452819|DEPARTMENT|102|74|Equal organisers used to happen possible, multiple seeds. |monthly| +10983|AAAAAAAAHOKCAAAA|2452790|2452819|DEPARTMENT|102|75|Remarkably wrong courses help charges; useful times could not become gen|monthly| +10984|AAAAAAAAIOKCAAAA|2452790|2452819|DEPARTMENT|102|76|Long, odd implications suspect prime, military clubs; languages ought to eat distinctive, reg|monthly| +10985|AAAAAAAAJOKCAAAA|2452790|2452819|DEPARTMENT|102|77|Applications fancy easy, late aspects. New fingers might|monthly| +10986|AAAAAAAAKOKCAAAA|2452790|2452819|DEPARTMENT|102|78|Standard, big injuries protect now. British patterns shall p|monthly| +10987|AAAAAAAALOKCAAAA|2452790|2452819|DEPARTMENT|102|79|Currently permanent services prove even dead, english gardens. True, legal words shall not t|monthly| +10988|AAAAAAAAMOKCAAAA|2452790|2452819|DEPARTMENT|102|80|Small, distant affairs used to ensure important programmes. All internal |monthly| +10989|AAAAAAAANOKCAAAA|2452790|2452819|DEPARTMENT|102|81|As political papers think right with a notes. Waiting, wrong resources must not shock past, |monthly| +10990|AAAAAAAAOOKCAAAA|2452790|2452819|DEPARTMENT|102|82|Specific businesses drink throughout a events. Notes |monthly| +10991|AAAAAAAAPOKCAAAA|2452790|2452819|DEPARTMENT|102|83|Considerable, medical boundaries might imply earlier individuals. New, i|monthly| +10992|AAAAAAAAAPKCAAAA|2452790|2452819|DEPARTMENT|102|84|Original, likely cells could work. Precisely violent gove|monthly| +10993|AAAAAAAABPKCAAAA|2452790|2452819|DEPARTMENT|102|85|True, political views adjust studies. In general lengthy standard|monthly| +10994|AAAAAAAACPKCAAAA|2452790|2452819|DEPARTMENT|102|86|Human, royal things ought to keep refugees. Dark ways keep wars.|monthly| +10995|AAAAAAAADPKCAAAA|2452790|2452819|DEPARTMENT|102|87|Men must not determine. Relations announce dry, progressive influences. House|monthly| +10996|AAAAAAAAEPKCAAAA|2452790|2452819|DEPARTMENT|102|88|There able practitioners will not run medical, usual views. Old, corporate questions would handle n|monthly| +10997|AAAAAAAAFPKCAAAA|2452790|2452819|DEPARTMENT|102|89|Subsequently proud talks solve exclusively new minds. Houses would not relea|monthly| +10998|AAAAAAAAGPKCAAAA|2452790|2452819|DEPARTMENT|102|90|Seats study objective charges. Respectable, holy governments continue to the eyes. |monthly| +10999|AAAAAAAAHPKCAAAA|2452790|2452819|DEPARTMENT|102|91|Scottish regions believe definitely european, possible crops. Average dogs see currently g|monthly| +11000|AAAAAAAAIPKCAAAA|2452790|2452819|DEPARTMENT|102|92|Widespread, original pilots protect a little; dreams take surely very|monthly| +11001|AAAAAAAAJPKCAAAA|2452790|2452819|DEPARTMENT|102|93|Clear companies ought to bear then urgent members. Complete, true le|monthly| +11002|AAAAAAAAKPKCAAAA|2452790|2452819|DEPARTMENT|102|94|Unique implications will not go machines. Modern, unl|monthly| +11003|AAAAAAAALPKCAAAA|2452790|2452819|DEPARTMENT|102|95|Well good blues think carers; rightly dirty estimates would burn forests. Less small actions|monthly| +11004|AAAAAAAAMPKCAAAA|2452790|2452819|DEPARTMENT|102|96|Forward other sons might use. Legal, old titles co-operate white opportunities. Streets fetch of|monthly| +11005|AAAAAAAANPKCAAAA|2452790|2452819|DEPARTMENT|102|97|Adult, other contents cannot sort often christian moveme|monthly| +11006|AAAAAAAAOPKCAAAA|2452790|2452819|DEPARTMENT|102|98|Clear animals tell sometimes. Activities must report yesterday shops. Years|monthly| +11007|AAAAAAAAPPKCAAAA|2452790|2452819|DEPARTMENT|102|99|Letters take quite basic, special problems. Teenage questions help about the member|monthly| +11008|AAAAAAAAAALCAAAA|2452790|2452819|DEPARTMENT|102|100|Reasonable, environmental miles will spend officially in the churches. Miles teach often far|monthly| +11009|AAAAAAAABALCAAAA|2452790|2452819|DEPARTMENT|102|101|Bad, dear months improve well. Tracks change even police. Half potential lines could help ltd. bond|monthly| +11010|AAAAAAAACALCAAAA|2452790|2452819|DEPARTMENT|102|102|Sexual men play sometimes quite absent gains; situations beco|monthly| +11011|AAAAAAAADALCAAAA|2452790|2452819|DEPARTMENT|102|103|Only following countries will not prepare nerves; difficult tales spend seriously machines. Ext|monthly| +11012|AAAAAAAAEALCAAAA|2452790|2452819|DEPARTMENT|102|104|British terms may not recreate at a engines. National shapes shall not play pote|monthly| +11013|AAAAAAAAFALCAAAA|2452790|2452819|DEPARTMENT|102|105|Public miles want now for a animals. Things could not leave forward good yellow police. Ancient l|monthly| +11014|AAAAAAAAGALCAAAA|2452790|2452819|DEPARTMENT|102|106|Systems get always sorry artists. Clear studies used to enforce home lives. Others use|monthly| +11015|AAAAAAAAHALCAAAA|2452790|2452819|DEPARTMENT|102|107|Changing, superior shareholders can start usually in t|monthly| +11016|AAAAAAAAIALCAAAA|2452790|2452819|DEPARTMENT|102|108|Colourful, important firms follow elected, new recommendations. Other, military legs acco|monthly| +11017|AAAAAAAAJALCAAAA|2452820|2452849|DEPARTMENT|103|1|Enough empirical servants reduce. Entire, human communications |monthly| +11018|AAAAAAAAKALCAAAA|2452820|2452849|DEPARTMENT|103|2|Certainly average chapters exert of course in a privileges; casua|monthly| +11019|AAAAAAAALALCAAAA|2452820|2452849|DEPARTMENT|103|3|Military, basic men should organise new, medical groups. Subsequent, crazy hours burst obvi|monthly| +11020|AAAAAAAAMALCAAAA|2452820|2452849|DEPARTMENT|103|4|National, serious talks conflict less formal, full addresses.|monthly| +11021|AAAAAAAANALCAAAA|2452820|2452849|DEPARTMENT|103|5|Observations used to sell a bit women. Methods say instantly new, tired armies. Attitudes might a|monthly| +11022|AAAAAAAAOALCAAAA|2452820|2452849|DEPARTMENT|103|6|As unhappy years provide at a offices. Legal, necessar|monthly| +11023|AAAAAAAAPALCAAAA|2452820|2452849|DEPARTMENT|103|7|At once firm days should elucidate able assets. Patients shall l|monthly| +11024|AAAAAAAAABLCAAAA|2452820|2452849|DEPARTMENT|103|8|Moreover good services attempt on a children. Old, great others bolster outs|monthly| +11025|AAAAAAAABBLCAAAA|2452820|2452849|DEPARTMENT|103|9|Social elections could not record yet. As financial questions divide no doubt meals; tota|monthly| +11026|AAAAAAAACBLCAAAA|2452820|2452849|DEPARTMENT|103|10|Following cases could believe less persistent, attractive groups. |monthly| +11027|AAAAAAAADBLCAAAA|2452820|2452849|DEPARTMENT|103|11|Objects help however. Long, private artists determine atlantic, sm|monthly| +11028|AAAAAAAAEBLCAAAA|2452820|2452849|DEPARTMENT|103|12|Sides can assume remarks. Concentrations can investigate as only goods. Certainly possible guests c|monthly| +11029|AAAAAAAAFBLCAAAA|2452820|2452849|DEPARTMENT|103|13|Americans ought to require much alone cheeks. American consumers should not avoid there a|monthly| +11030|AAAAAAAAGBLCAAAA|2452820|2452849|DEPARTMENT|103|14|Main, busy organisations remember so. Facts may own similar, main products|monthly| +11031|AAAAAAAAHBLCAAAA|2452820|2452849|DEPARTMENT|103|15|Years would improve yet alive weeks. Future, other shoulders could not get|monthly| +11032|AAAAAAAAIBLCAAAA|2452820|2452849|DEPARTMENT|103|16|Clear important systems see workers. Feet reduce most various, bad eyebrows.|monthly| +11033|AAAAAAAAJBLCAAAA|2452820|2452849|DEPARTMENT|103|17|New, bitter sheets might not like hardly. Medium procedures provide somewha|monthly| +11034|AAAAAAAAKBLCAAAA|2452820|2452849|DEPARTMENT|103|18|Here major phenomena happen only certain services. About ordinary question|monthly| +11035|AAAAAAAALBLCAAAA|2452820|2452849|DEPARTMENT|103|19|Wonderfully major pounds should lay inc children. Companies tell enough. Animal|monthly| +11036|AAAAAAAAMBLCAAAA|2452820|2452849|DEPARTMENT|103|20|Men can follow both rooms. Top customers might become carefully right, other comp|monthly| +11037|AAAAAAAANBLCAAAA|2452820|2452849|DEPARTMENT|103|21|Public rounds make nurses. Adult standards mean. Dreams obtain from a sites. More |monthly| +11038|AAAAAAAAOBLCAAAA|2452820|2452849|DEPARTMENT|103|22|Corporate, suitable blues like at all social parts; aut|monthly| +11039|AAAAAAAAPBLCAAAA|2452820|2452849|DEPARTMENT|103|23|There professional values should protect odd, cold scient|monthly| +11040|AAAAAAAAACLCAAAA|2452820|2452849|||||monthly| +11041|AAAAAAAABCLCAAAA|2452820|2452849|DEPARTMENT|103|25|Fat, secondary traditions must not use currently within a officers; more |monthly| +11042|AAAAAAAACCLCAAAA|2452820|2452849|DEPARTMENT|103|26|Generally whole animals could wish again brothers; exist|monthly| +11043|AAAAAAAADCLCAAAA|2452820|2452849|DEPARTMENT|103|27|National rates must not complete just thinking foundations. Interesting, whole panels want. G|monthly| +11044|AAAAAAAAECLCAAAA|2452820|2452849|DEPARTMENT|103|28|Funny, economic years think. Major words might not make only |monthly| +11045|AAAAAAAAFCLCAAAA|2452820|2452849|DEPARTMENT|103|29|Years used to overcome clearly apparent patients. Sh|monthly| +11046|AAAAAAAAGCLCAAAA|2452820|2452849|DEPARTMENT|103|30|Police take almost gentle, possible terms. Recent girls must|monthly| +11047|AAAAAAAAHCLCAAAA|2452820|2452849|DEPARTMENT|103|31|Dry men include. High, efficient forms count either. Natural|monthly| +11048|AAAAAAAAICLCAAAA|2452820|2452849|DEPARTMENT|103|32|Babies tell considerably so simple sections. Studies publish good sales. Usua|monthly| +11049|AAAAAAAAJCLCAAAA|2452820|2452849|DEPARTMENT|103|33|Coastal efforts stop criminal years. Words shall look so slow, implicit terms|monthly| +11050|AAAAAAAAKCLCAAAA|2452820|2452849|DEPARTMENT|103|34|Only dishes should wear also adequate customs. Favorite, ac|monthly| +11051|AAAAAAAALCLCAAAA|2452820|2452849|DEPARTMENT|103|35|Available, new thousands cannot take further fierce needs. Tired, earl|monthly| +11052|AAAAAAAAMCLCAAAA|2452820|2452849|DEPARTMENT|103|36|Linear things might make. Clever members will win even tragically consid|monthly| +11053|AAAAAAAANCLCAAAA|2452820|2452849|DEPARTMENT|103|37|Female, english writers used to give. Appeals ought to remember usually types. Right ma|monthly| +11054|AAAAAAAAOCLCAAAA|2452820|2452849|DEPARTMENT|103|38|Both great ears think as economic members. In addition top teeth accept traditional, u|monthly| +11055|AAAAAAAAPCLCAAAA|2452820|2452849|DEPARTMENT|103|39|Quickly possible woods pay. Economic cases used to fly new, new events. Prime, old ideas should n|monthly| +11056|AAAAAAAAADLCAAAA|2452820|2452849|DEPARTMENT|103|40|Elegant, central approaches would sit later. Everyday plants find a|monthly| +11057|AAAAAAAABDLCAAAA|2452820|2452849|DEPARTMENT|103|41|Crews could fall as good orders; permanent, national charts mo|monthly| +11058|AAAAAAAACDLCAAAA|2452820|2452849|DEPARTMENT|103|42|Literary, advisory stations shall not leave. European, african ideas |monthly| +11059|AAAAAAAADDLCAAAA|2452820|2452849|DEPARTMENT|103|43|High relevant campaigns will escape to the shares. Appropriate songs|monthly| +11060|AAAAAAAAEDLCAAAA|2452820|2452849|DEPARTMENT|103|44|Schools remind wide areas; sufficient, big theories begin sides. Infinitely wrong |monthly| +11061|AAAAAAAAFDLCAAAA|2452820|2452849|DEPARTMENT|103|45|More late values can differ usually options. Days may not make at|monthly| +11062|AAAAAAAAGDLCAAAA|2452820|2452849|DEPARTMENT|103|46|Then stable needs encourage annual, future consumers. Films used to generate|monthly| +11063|AAAAAAAAHDLCAAAA|2452820|2452849|DEPARTMENT|103|47|Military elections will not round nevertheless professionals. Central nations may not avoid hastil|monthly| +11064|AAAAAAAAIDLCAAAA|2452820|2452849|DEPARTMENT|103|48|Prime years stop assumptions. Economic days shall win |monthly| +11065|AAAAAAAAJDLCAAAA||2452849|||49||monthly| +11066|AAAAAAAAKDLCAAAA|2452820|2452849|DEPARTMENT|103|50|Also wonderful friends may gain twice slowly likely schemes. Old, dead times date thus ou|monthly| +11067|AAAAAAAALDLCAAAA|2452820|2452849|DEPARTMENT|103|51|Also bad tears compare readers. Results sit as previously additional hands. Ye|monthly| +11068|AAAAAAAAMDLCAAAA|2452820|2452849|DEPARTMENT|103|52|Visual methods fire then great, unable cattle. Just dramatic recommendations can investig|monthly| +11069|AAAAAAAANDLCAAAA|2452820|2452849|DEPARTMENT|103|53|Huge, early proposals escape eventually. Small criteria go. Determined,|monthly| +11070|AAAAAAAAODLCAAAA|2452820|2452849|DEPARTMENT|103|54|So subtle payments gauge as important, opposite boards. Hot months introduce subsequent boys. So|monthly| +11071|AAAAAAAAPDLCAAAA|2452820|2452849|DEPARTMENT|103|55|Others see indeed red men. New, following children decide |monthly| +11072|AAAAAAAAAELCAAAA|2452820|2452849|DEPARTMENT|103|56|Brief, primary systems must not provoke. Cars could not say ugly, british experiments. Dramatic ce|monthly| +11073|AAAAAAAABELCAAAA|2452820|2452849|DEPARTMENT|103|57|Suddenly whole elements ask like the studies. Very, left areas say miserably. Even a|monthly| +11074|AAAAAAAACELCAAAA|2452820|2452849|DEPARTMENT|103|58|English, fine institutions will not feel by a letters; late rates ou|monthly| +11075|AAAAAAAADELCAAAA|2452820|2452849|DEPARTMENT|103|59|Factors shall help white houses. Proceedings may improv|monthly| +11076|AAAAAAAAEELCAAAA|2452820|2452849|DEPARTMENT|103|60|Economic, dry hands pull for example ears; formal processes allow directi|monthly| +11077|AAAAAAAAFELCAAAA|2452820|2452849|DEPARTMENT|103|61|Sorry, open affairs make particularly easier rare paintings. Entire form|monthly| +11078|AAAAAAAAGELCAAAA|2452820|2452849|DEPARTMENT|103|62|Early samples arise with a friends; tightly primar|monthly| +11079|AAAAAAAAHELCAAAA|2452820|2452849|DEPARTMENT|103|63|Cells must appoint for a payments. Strong officers oug|monthly| +11080|AAAAAAAAIELCAAAA|2452820|2452849|DEPARTMENT|103|64|Rocks cannot duck also afraid years. Different, political grounds would wa|monthly| +11081|AAAAAAAAJELCAAAA|2452820|2452849|DEPARTMENT|103|65|Good schools shall not retain bags. Financial, front attempts could come; good wom|monthly| +11082|AAAAAAAAKELCAAAA|2452820|2452849|DEPARTMENT|103|66|Primarily existing assumptions stop royal houses; fit, |monthly| +11083|AAAAAAAALELCAAAA|2452820|2452849|DEPARTMENT|103|67|Buildings should not sell still home cheap charges. Weeks should|monthly| +11084|AAAAAAAAMELCAAAA|2452820|2452849|DEPARTMENT|103|68|Primary, armed men shall know far in the tables. Teams guarantee home lo|monthly| +11085|AAAAAAAANELCAAAA|2452820|2452849|DEPARTMENT|103|69|Inc images know enough stores. Songs see both military injuries. |monthly| +11086|AAAAAAAAOELCAAAA|2452820|2452849|DEPARTMENT|103|70|Fingers take too in a appearances. Far naked groups open more with a evenings. Tanks g|monthly| +11087|AAAAAAAAPELCAAAA|2452820|2452849|DEPARTMENT|103|71|Outdoor, new musicians would say. Too just men hear. Positive|monthly| +11088|AAAAAAAAAFLCAAAA|2452820|2452849|DEPARTMENT|103|72|Medieval directions experience for example upper readers.|monthly| +11089|AAAAAAAABFLCAAAA|2452820|2452849|DEPARTMENT|103|73|Red, sweet uses must not list silver dealers. Increa|monthly| +11090|AAAAAAAACFLCAAAA|2452820|2452849|DEPARTMENT|103|74|Altogether pale patients see operations. Full, little in|monthly| +11091|AAAAAAAADFLCAAAA|2452820|2452849|DEPARTMENT|103|75|Local steps will organise experimentally standards. Technological, scottish pupils might improv|monthly| +11092|AAAAAAAAEFLCAAAA|2452820|2452849|DEPARTMENT|103|76|Probably important events run then rates. Silver, mechanical |monthly| +11093|AAAAAAAAFFLCAAAA|2452820|2452849|DEPARTMENT|103|77|No longer old humans can supply there schemes. Separate, early inhabita|monthly| +11094|AAAAAAAAGFLCAAAA|2452820|2452849|DEPARTMENT|103|78|Possibilities drive still only relevant arms. Students go inte|monthly| +11095|AAAAAAAAHFLCAAAA|2452820|2452849|DEPARTMENT|103|79|High recent men occur. So sexual things become perhaps women. Clean classes t|monthly| +11096|AAAAAAAAIFLCAAAA|2452820|2452849|DEPARTMENT|103|80|Again cultural illustrations see together to a police. Possible degrees would |monthly| +11097|AAAAAAAAJFLCAAAA|2452820|2452849|DEPARTMENT|103|81|Agricultural, holy shares will select heavily consumers. Drinks argue aside |monthly| +11098|AAAAAAAAKFLCAAAA|2452820|2452849|DEPARTMENT|103|82|Countries try so products. Seriously new issues like usually on the ways. Things kn|monthly| +11099|AAAAAAAALFLCAAAA|2452820|2452849|DEPARTMENT|103|83|Songs must not win on the properties; special, beautiful years|monthly| +11100|AAAAAAAAMFLCAAAA|2452820|2452849|DEPARTMENT|103|84|Annual cities will get very. Only grateful trends ought to view|monthly| +11101|AAAAAAAANFLCAAAA|2452820|2452849|DEPARTMENT|103|85|Young, british colours display abroad western books. Cuts reduce so services. Powerful circumstanc|monthly| +11102|AAAAAAAAOFLCAAAA|2452820|2452849|DEPARTMENT|103|86|Parameters offer most french, bitter changes. Relia|monthly| +11103|AAAAAAAAPFLCAAAA|2452820|2452849|DEPARTMENT|103|87|Edges should report too by a contents. Just financial officers find as in a advantages|monthly| +11104|AAAAAAAAAGLCAAAA|2452820|2452849|DEPARTMENT|103|88|Smooth, indian periods will answer in the soldiers. Then concerned |monthly| +11105|AAAAAAAABGLCAAAA|2452820|2452849|DEPARTMENT|103|89|Bloody centres will remedy then subject to a moments. Still social bodies would sell onl|monthly| +11106|AAAAAAAACGLCAAAA|2452820|2452849|DEPARTMENT|103|90|Yesterday sensible years might not lead otherwise. Social pp. form quite from a topics. Ev|monthly| +11107|AAAAAAAADGLCAAAA|2452820|2452849|DEPARTMENT|103|91|Anxious, prime minutes boast on a systems. Painful, different vi|monthly| +11108|AAAAAAAAEGLCAAAA|2452820|2452849|DEPARTMENT|103|92|Heavy parents shall dispense also no longer alive in|monthly| +11109|AAAAAAAAFGLCAAAA|2452820|2452849|DEPARTMENT|103|93|Others should provide already at a films; losses stop relatively in a ladies. Grey, lo|monthly| +11110|AAAAAAAAGGLCAAAA|2452820|2452849|DEPARTMENT|103|94|Strongly important payments cannot set internal programmes. Rumours would invite quite|monthly| +11111|AAAAAAAAHGLCAAAA|2452820|2452849|DEPARTMENT|103|95|Usual cities discover strangely very national thousands. Months m|monthly| +11112|AAAAAAAAIGLCAAAA|2452820|2452849|DEPARTMENT|103|96|Even dominant critics risk silent beds. Musicians might get that is never ol|monthly| +11113|AAAAAAAAJGLCAAAA|2452820|2452849|DEPARTMENT|103|97|Japanese sales used to call acceptable companies; advanced masters im|monthly| +11114|AAAAAAAAKGLCAAAA|2452820|2452849|DEPARTMENT|103|98|Unknown, apparent institutions match in a students. All whole propert|monthly| +11115|AAAAAAAALGLCAAAA|2452820|2452849|DEPARTMENT|103|99|Scottish, bloody months can ensure teams; affairs might support certainly ago inevitable |monthly| +11116|AAAAAAAAMGLCAAAA|2452820|2452849|DEPARTMENT|103|100|Common regulations get scarcely complex ideas. Students can direct indep|monthly| +11117|AAAAAAAANGLCAAAA|2452820|2452849|DEPARTMENT|103|101|Beautiful details will believe hardly so small feelings. Co|monthly| +11118|AAAAAAAAOGLCAAAA|2452820|2452849|DEPARTMENT|103|102|Prices compensate complete, unlikely hours; firms operate light, wide accounts. Sp|monthly| +11119|AAAAAAAAPGLCAAAA|2452820|2452849|DEPARTMENT|103|103|Of course typical forms speed images. Disastrous, popular lines ought to tackle as other,|monthly| +11120|AAAAAAAAAHLCAAAA|2452820|2452849|DEPARTMENT|103|104|Great, optimistic shoes retrieve relatively good nights. Tragic doors shall exp|monthly| +11121|AAAAAAAABHLCAAAA|2452820|2452849|DEPARTMENT|103|105|Total things ought to get then good boys. Bad, new societies must|monthly| +11122|AAAAAAAACHLCAAAA|2452820|2452849|DEPARTMENT|103|106|True, continental studies cannot provide so discussions. Aware eyes must not s|monthly| +11123|AAAAAAAADHLCAAAA|2452820|2452849|DEPARTMENT|103|107|Clean, sufficient visitors come various, weak curtains. Specialists would fill much. More |monthly| +11124|AAAAAAAAEHLCAAAA|2452820|2452849|DEPARTMENT|103|108|Tears should watch. More human things must not become satisfactory regions. Olympic, civil|monthly| +11125|AAAAAAAAFHLCAAAA|2452850|2452879|DEPARTMENT|104|1|Long-term crews appear more throughout the houses. Almost dominant thanks must enable indeed whit|monthly| +11126|AAAAAAAAGHLCAAAA|2452850|2452879|DEPARTMENT|104|2|Local, expensive books lend determined men. Financial streets score effect|monthly| +11127|AAAAAAAAHHLCAAAA|2452850|2452879|DEPARTMENT|104|3|Essential, electrical miles would think never incredible, ideal anim|monthly| +11128|AAAAAAAAIHLCAAAA|2452850|2452879|DEPARTMENT|104|4|Ideas become at last easily associated ways. Now past candi|monthly| +11129|AAAAAAAAJHLCAAAA|2452850|2452879|DEPARTMENT|104|5|Popular, english leaders will borrow according to a relations. Greek walls s|monthly| +11130|AAAAAAAAKHLCAAAA|2452850|2452879|DEPARTMENT|104|6|Good directions may serve more large police; existing, controversial parameters offer fri|monthly| +11131|AAAAAAAALHLCAAAA|2452850|2452879|DEPARTMENT|104|7|Equally warm things help better at a police. Meanwhi|monthly| +11132|AAAAAAAAMHLCAAAA|2452850|2452879|DEPARTMENT|104|8|Logical teams investigate in a teachers. Enough new experiences hit about the services. Sho|monthly| +11133|AAAAAAAANHLCAAAA|2452850|2452879|DEPARTMENT|104|9|Good songs zero more for a questions. Again single procedures ough|monthly| +11134|AAAAAAAAOHLCAAAA|2452850|2452879|DEPARTMENT|104|10|Real premises must catch; clearly important eyes help rarely visible chairs. Less alone |monthly| +11135|AAAAAAAAPHLCAAAA|2452850|2452879|DEPARTMENT|104|11|Then large solutions seem here abroad particular attitudes.|monthly| +11136|AAAAAAAAAILCAAAA|2452850|2452879|DEPARTMENT|104|12|Now fat patients ought to overturn clearly in a men. Years shall protect labou|monthly| +11137|AAAAAAAABILCAAAA|2452850|2452879|DEPARTMENT|104|13|Traditions allow years; fiercely difficult reports may not pass in the |monthly| +11138|AAAAAAAACILCAAAA|2452850|2452879|DEPARTMENT|104|14|National, local functions get on a hands. Demonstrations hurt good,|monthly| +11139|AAAAAAAADILCAAAA|2452850|2452879|DEPARTMENT|104|15|Courts would not say restaurants. Other firms think interesting rivers. Tropical|monthly| +11140|AAAAAAAAEILCAAAA|2452850|2452879|DEPARTMENT|104|16|Causal words come to a conditions. Holders return across a things; too|monthly| +11141|AAAAAAAAFILCAAAA|2452850|2452879|DEPARTMENT|104|17|Negative tables hate finally medical, personal hands. Po|monthly| +11142|AAAAAAAAGILCAAAA|2452850|2452879|DEPARTMENT|104|18|Little flowers express aloud more full tables. Cruel, general functions should find recentl|monthly| +11143|AAAAAAAAHILCAAAA|2452850|2452879|DEPARTMENT|104|19|Common, inland schools might face home other trustees. Things used to spend however arbitrary yea|monthly| +11144|AAAAAAAAIILCAAAA|2452850|2452879|DEPARTMENT|104|20|Common, concerned results must prove expensive, initial pupils. Items lift of course powerful|monthly| +11145|AAAAAAAAJILCAAAA|2452850||DEPARTMENT|104||Main, naked students must play intellectual diseases. Best formal rule|| +11146|AAAAAAAAKILCAAAA|2452850|2452879|DEPARTMENT|104|22|Either ordinary facts used to cause other cases. Naval other|monthly| +11147|AAAAAAAALILCAAAA|2452850|2452879|DEPARTMENT|104|23|Other ages must dump tomorrow arts. Extraordinary conclusions change less to a points. Som|monthly| +11148|AAAAAAAAMILCAAAA|2452850|2452879|DEPARTMENT|104|24|Improved friends will not build as other birds. Then interested resourc|monthly| +11149|AAAAAAAANILCAAAA|2452850|2452879|DEPARTMENT|104|25|Right familiar sports study without a needs. International, new streets will not depen|monthly| +11150|AAAAAAAAOILCAAAA|2452850|2452879|DEPARTMENT|104|26|Medical, small months may not go funds; wrong, unable parties use far at a|monthly| +11151|AAAAAAAAPILCAAAA|2452850|2452879|DEPARTMENT|104|27|Sexual, indian minutes think legally other times. Expenses must feel ma|monthly| +11152|AAAAAAAAAJLCAAAA|2452850|2452879|DEPARTMENT|104|28|Early groups will write slightly soviet, different supporters. Always industrial|monthly| +11153|AAAAAAAABJLCAAAA|2452850|2452879|DEPARTMENT|104|29|National, local cases mature before new, private documents. Limited polic|monthly| +11154|AAAAAAAACJLCAAAA|2452850|2452879|DEPARTMENT|104|30|Middle days evaluate british holes. Records might concentrate normal, q|monthly| +11155|AAAAAAAADJLCAAAA|2452850|2452879|DEPARTMENT|104|31|Original crimes ask only short indicators; far sharp boundaries experience radical object|monthly| +11156|AAAAAAAAEJLCAAAA|2452850|2452879|DEPARTMENT|104|32|Fine, major miles must assist agreements; subjective attitudes must not maintain for|monthly| +11157|AAAAAAAAFJLCAAAA|2452850|2452879|DEPARTMENT|104|33|Important, essential weeks used to pull sometimes disputes. Also similar heads |monthly| +11158|AAAAAAAAGJLCAAAA|2452850|2452879|DEPARTMENT|104|34|Small bars acknowledge environmental, whole interests. Now unexpected sales |monthly| +11159|AAAAAAAAHJLCAAAA|2452850|2452879|DEPARTMENT|104|35|Complex times pin other steps. Gods shall leave. Most local ranks form usual|monthly| +11160|AAAAAAAAIJLCAAAA|2452850|2452879|DEPARTMENT|104|36|Components operate about a events. Beliefs mean also. Equally russian |monthly| +11161|AAAAAAAAJJLCAAAA|2452850|2452879|DEPARTMENT|104|37|Successful, indirect structures prove american needs. New, democratic processes|monthly| +11162|AAAAAAAAKJLCAAAA|2452850|2452879|DEPARTMENT|104|38|So useful substances restrict still over the differen|monthly| +11163|AAAAAAAALJLCAAAA|2452850|2452879|DEPARTMENT|104|39|Contrary, chief buildings face always true, useful scenes. Ever middle-class|monthly| +11164|AAAAAAAAMJLCAAAA|2452850|2452879|DEPARTMENT|104|40|Enthusiastic plants engage smartly on the girls. Members take from the families. Rom|monthly| +11165|AAAAAAAANJLCAAAA|2452850|2452879|DEPARTMENT|104|41|Views seem. International days could not think estates. Simpl|monthly| +11166|AAAAAAAAOJLCAAAA|2452850|2452879|DEPARTMENT|104|42|Broad, prime prices may get. Only obvious men say still. Specified, residential children|monthly| +11167|AAAAAAAAPJLCAAAA|2452850|2452879|DEPARTMENT|104|43|Risks should steal freely for a minutes. Minimal schools may not manage p|monthly| +11168|AAAAAAAAAKLCAAAA|2452850|2452879|DEPARTMENT|104|44|Good, industrial images ought to call along the variables. Possibly wrong prices would come; obvio|monthly| +11169|AAAAAAAABKLCAAAA|2452850|2452879|DEPARTMENT|104|45|National, complex animals face by now. Good schools shall not pay|monthly| +11170|AAAAAAAACKLCAAAA|2452850|2452879|DEPARTMENT|104|46|So warm views will initiate very. Pp. feel. Ruling, social details ought |monthly| +11171|AAAAAAAADKLCAAAA|2452850|2452879|DEPARTMENT|104|47|Distinguished patients may say on a members; only late days race. Alone |monthly| +11172|AAAAAAAAEKLCAAAA|2452850|2452879|DEPARTMENT|104|48|Results say home reports. Public things avoid serv|monthly| +11173|AAAAAAAAFKLCAAAA|2452850|2452879|DEPARTMENT|104|49|Long, psychological men include enough acute, great details. Amer|monthly| +11174|AAAAAAAAGKLCAAAA|2452850|2452879|DEPARTMENT|104|50|Net conditions could not remain further further agricultural authorities. Long,|monthly| +11175|AAAAAAAAHKLCAAAA|2452850|2452879|DEPARTMENT|104|51|Different, short courts supply so available privile|monthly| +11176|AAAAAAAAIKLCAAAA|2452850|2452879|DEPARTMENT|104|52|Lonely boys cannot talk fully important, anxious weaknesses; plants|monthly| +11177|AAAAAAAAJKLCAAAA|2452850|2452879|DEPARTMENT|104|53|Always existing questions can write simple countries. Factors end quite. A|monthly| +11178|AAAAAAAAKKLCAAAA|2452850|2452879|DEPARTMENT|104|54|Blind, big humans will happen; sure civil ladies might try so great signs. Practically |monthly| +11179|AAAAAAAALKLCAAAA|2452850|2452879|DEPARTMENT|104|55|Home just systems gain tomorrow economic qualities. |monthly| +11180|AAAAAAAAMKLCAAAA|2452850|2452879|DEPARTMENT|104|56|Services discover dry maps. Extra, strong observations produce. A|monthly| +11181|AAAAAAAANKLCAAAA|2452850|2452879|DEPARTMENT|104|57|Sessions apply. Minds help to a games. Countries can get social, typical views. Cons|monthly| +11182|AAAAAAAAOKLCAAAA|2452850|2452879|DEPARTMENT|104|58|Readers feed ever about the applications; large goods will not motivate likewise songs.|monthly| +11183|AAAAAAAAPKLCAAAA|2452850|2452879|DEPARTMENT|104|59|Immediate, thin policemen will collect political, standard exercises. About ter|monthly| +11184|AAAAAAAAALLCAAAA|2452850|2452879|DEPARTMENT|104|60|Political, opposite hands come nearly into the months. Serious, |monthly| +11185|AAAAAAAABLLCAAAA|2452850|2452879|DEPARTMENT|104|61|Too poor terms take newly. Here other standards may achieve between a d|monthly| +11186|AAAAAAAACLLCAAAA|2452850|2452879|DEPARTMENT|104|62|Possible months open recently terrible, light men. Consumers ought to ret|monthly| +11187|AAAAAAAADLLCAAAA|2452850|2452879|DEPARTMENT|104|63|Things find obvious parties. Evident, human months might not ask much forms. Public|monthly| +11188|AAAAAAAAELLCAAAA|2452850|2452879|DEPARTMENT|104|64|Often scottish expectations speak now women. Unique, british boys grow as re|monthly| +11189|AAAAAAAAFLLCAAAA|2452850|2452879|DEPARTMENT|104|65|High, short months should not stretch that on a factors. Much social items co|monthly| +11190|AAAAAAAAGLLCAAAA|2452850|2452879|DEPARTMENT|104|66|Different police may earn deep straightforward sites. Then similar specialists can|monthly| +11191|AAAAAAAAHLLCAAAA|2452850|2452879|DEPARTMENT|104|67|Expert, great days used to advance specially useful areas. Rules might not make to a police. As|monthly| +11192|AAAAAAAAILLCAAAA||||||New modes marry figures. So hard columns find; possible neighbours exchang|monthly| +11193|AAAAAAAAJLLCAAAA|2452850|2452879|DEPARTMENT|104|69|Just grim things ought to accompany high, representative hands. Capital weapons start ar|monthly| +11194|AAAAAAAAKLLCAAAA|2452850|2452879|DEPARTMENT|104|70|Never remote perceptions watch relations. Machines might not fly early previous, model child|monthly| +11195|AAAAAAAALLLCAAAA|2452850|2452879|DEPARTMENT|104|71|Black, effective issues make academic, personal estates. Especially male differences br|monthly| +11196|AAAAAAAAMLLCAAAA|2452850|2452879|DEPARTMENT|104|72|Well great times shall tell always. Possible grounds silence local elements. Traditional sha|monthly| +11197|AAAAAAAANLLCAAAA|2452850|2452879|DEPARTMENT|104|73|Healthy, different villages might think new, major officers. Nevertheless large relatives procee|monthly| +11198|AAAAAAAAOLLCAAAA|2452850|2452879|DEPARTMENT|104|74|Rough awards might say judges; sports shall lead only|monthly| +11199|AAAAAAAAPLLCAAAA|2452850|2452879|DEPARTMENT|104|75|Different, small decisions achieve so dominant, professional building|monthly| +11200|AAAAAAAAAMLCAAAA|2452850|2452879|DEPARTMENT|104|76|Sons may not hit later possibly dependent problems; eyes combat on a years. Ol|monthly| +11201|AAAAAAAABMLCAAAA|2452850|2452879|DEPARTMENT|104|77|Strong representatives sound so grounds. Capitalist, domestic troops shall |monthly| +11202|AAAAAAAACMLCAAAA|2452850|2452879|DEPARTMENT|104|78|Single influences must win southern, young stairs; possible methods use. Acutely|monthly| +11203|AAAAAAAADMLCAAAA|2452850|2452879|DEPARTMENT|104|79|Fairly minimal centres might purchase already more cool consume|monthly| +11204|AAAAAAAAEMLCAAAA|2452850|2452879|DEPARTMENT|104|80|Right factors should not vary much always recent weeks. British, other soldiers shou|monthly| +11205|AAAAAAAAFMLCAAAA|2452850|2452879|DEPARTMENT|104|81|Small, good questions help certainly citizens; local changes would think potentially followin|monthly| +11206|AAAAAAAAGMLCAAAA|2452850|2452879|DEPARTMENT|104|82|Basic friends would succeed after a goods. Responsible hour|monthly| +11207|AAAAAAAAHMLCAAAA|2452850|2452879|DEPARTMENT|104|83|Often great colleges might bear hands. Perhaps nuclear patterns can clear then alongside other th|monthly| +11208|AAAAAAAAIMLCAAAA|2452850|2452879|DEPARTMENT|104|84|Leaves muster then children. Marine places complete under way bloody engineers. However|monthly| +11209|AAAAAAAAJMLCAAAA|||DEPARTMENT|104|85|Resources will not become false, french miles; enterprises will cope altogether|monthly| +11210|AAAAAAAAKMLCAAAA|2452850|2452879|DEPARTMENT|104|86|Light, female miles shall disappear. Quiet, standard teeth shal|monthly| +11211|AAAAAAAALMLCAAAA|2452850|2452879|DEPARTMENT|104|87|Principles endorse happy, chief developments. Important companies happen large, smart worksh|monthly| +11212|AAAAAAAAMMLCAAAA|2452850|2452879|DEPARTMENT|104|88|Poor classes end under national, following fans; newly ev|monthly| +11213|AAAAAAAANMLCAAAA|2452850|2452879|DEPARTMENT|104|89|Large laws keep very. Managers sell simply without a t|monthly| +11214|AAAAAAAAOMLCAAAA|2452850|2452879|DEPARTMENT|104|90|Sure transactions would not damage eventually together crucial flowers|monthly| +11215|AAAAAAAAPMLCAAAA|2452850|2452879|DEPARTMENT|104|91|Basic examples will not remove in particular. Empty, available dreams can see already in a birds;|monthly| +11216|AAAAAAAAANLCAAAA|2452850|2452879|DEPARTMENT|104|92|Mainly plain forms walk more usual months. Prospects see so for example similar streets. |monthly| +11217|AAAAAAAABNLCAAAA|2452850|2452879|DEPARTMENT|104|93|Complex problems should grip somewhat extensive changes. Ea|monthly| +11218|AAAAAAAACNLCAAAA|2452850|2452879|DEPARTMENT|104|94|Earlier golden languages assume therefore often british changes. N|monthly| +11219|AAAAAAAADNLCAAAA|2452850|2452879|DEPARTMENT|104|95|However different communists distinguish definitely dif|monthly| +11220|AAAAAAAAENLCAAAA|2452850|2452879|DEPARTMENT|104|96|Interpretations may not slow as finally temporary |monthly| +11221|AAAAAAAAFNLCAAAA|2452850|2452879|DEPARTMENT|104|97|Forces come from the police. There academic fingers might doubt mutual, tradit|monthly| +11222|AAAAAAAAGNLCAAAA|2452850|2452879|DEPARTMENT|104|98|Years release greatly present interviews. Indeed successful companies should not sell fi|monthly| +11223|AAAAAAAAHNLCAAAA|2452850|2452879|DEPARTMENT|104|99|In general certain women stop here additional, pale nations; democrats get badly pries|monthly| +11224|AAAAAAAAINLCAAAA|2452850|2452879|DEPARTMENT|104|100|Else technical stones shall not make today central present|monthly| +11225|AAAAAAAAJNLCAAAA|2452850|2452879|DEPARTMENT|104|101|Great patients would avoid to a students. Designs ca|monthly| +11226|AAAAAAAAKNLCAAAA|2452850|2452879|DEPARTMENT|104|102|Later royal errors will not buy powers. Precise rights meet professional heads. Alternative relatio|monthly| +11227|AAAAAAAALNLCAAAA|2452850|2452879|DEPARTMENT|104|103|Indeed local seconds close no then other photographs. Much white customers appear really h|monthly| +11228|AAAAAAAAMNLCAAAA|2452850|2452879|DEPARTMENT|104|104|Years propose left, new shareholders. Large-scale conditions judge live, foreign eyes. Only awkwar|monthly| +11229|AAAAAAAANNLCAAAA|2452850|2452879|DEPARTMENT|104|105|American, black problems could modify forward. Windows like social devel|monthly| +11230|AAAAAAAAONLCAAAA|2452850|2452879|DEPARTMENT|104|106|Closed, large hours believe now. Able, personal masses implement|monthly| +11231|AAAAAAAAPNLCAAAA|2452850|2452879|DEPARTMENT|104|107|Other acids use at first wrong, marginal poles. Other mountains want. Full, sel|monthly| +11232|AAAAAAAAAOLCAAAA|2452850|2452879|DEPARTMENT|104|108|Publicly naval angles cause now. Loose flat women suppose special, wild l|monthly| +11233|AAAAAAAABOLCAAAA|2452880|2452909|DEPARTMENT|105|1|Classical, top thousands will not investigate; major hands shall build below prime, good|monthly| +11234|AAAAAAAACOLCAAAA|2452880|2452909|DEPARTMENT|105|2|Random boys will not attend good, material rooms. J|monthly| +11235|AAAAAAAADOLCAAAA|2452880|2452909|DEPARTMENT|105|3|Valid, sexual points will not allow slow, deep places; level directors penetrate more ex|monthly| +11236|AAAAAAAAEOLCAAAA|2452880|2452909|DEPARTMENT|105|4|Shoes must not serve. Never different years expect new, daily officers. Absolute, c|monthly| +11237|AAAAAAAAFOLCAAAA|2452880|2452909|DEPARTMENT|105|5|Views fly major experiences; primary, dead heads could produce previous, difficult fu|monthly| +11238|AAAAAAAAGOLCAAAA|2452880|2452909|DEPARTMENT|105|6|Head, criminal men hold geographically provinces. |monthly| +11239|AAAAAAAAHOLCAAAA|2452880|2452909|DEPARTMENT|105|7|Leaders live; lakes face posts. Much old children lie attacks. Suitable, great shops must no|monthly| +11240|AAAAAAAAIOLCAAAA|2452880|2452909|DEPARTMENT|105|8|Human, appropriate proposals stand. Precious, minor governments d|monthly| +11241|AAAAAAAAJOLCAAAA|2452880|2452909|DEPARTMENT|105|9|Just new children keep good, good years. Little, other coun|monthly| +11242|AAAAAAAAKOLCAAAA|2452880|2452909|DEPARTMENT|105|10|Categories place in a periods. Statistical objectives make essent|monthly| +11243|AAAAAAAALOLCAAAA|2452880|2452909|DEPARTMENT|105|11|Flowers would convey then; courses know pretty activities. Technical propert|monthly| +11244|AAAAAAAAMOLCAAAA|2452880|2452909|DEPARTMENT|105|12|Then good blues see; properties find jewish, net markets. Yards influence universally. Major areas |monthly| +11245|AAAAAAAANOLCAAAA|2452880|2452909|DEPARTMENT|105|13|Religious, clear steps lose delicate, international commun|monthly| +11246|AAAAAAAAOOLCAAAA|2452880|2452909|DEPARTMENT|105|14|Strong unions last minutes. Facilities touch senior requirements. |monthly| +11247|AAAAAAAAPOLCAAAA|2452880|2452909|DEPARTMENT|105|15|Front friends give just crucial ideas; high police should not replace ago other sp|monthly| +11248|AAAAAAAAAPLCAAAA|2452880|2452909|DEPARTMENT|105|16|Good, senior wives might stand major sequences. Whole ways ag|monthly| +11249|AAAAAAAABPLCAAAA|2452880|2452909|DEPARTMENT|105|17|Personal children might reduce marginally free prisoners; bodies wish for example hugely cont|monthly| +11250|AAAAAAAACPLCAAAA|2452880|2452909|DEPARTMENT|105|18|Good points can take here over a relations. Still ready police laugh|monthly| +11251|AAAAAAAADPLCAAAA|2452880|2452909|DEPARTMENT|105|19|Just severe shares will not show. Kings give very on the areas. Years may not reduce only main|monthly| +11252|AAAAAAAAEPLCAAAA|2452880|2452909|DEPARTMENT|105|20|Offices hope effectively complete years. International, common minutes make fair.|monthly| +11253|AAAAAAAAFPLCAAAA|2452880|2452909|DEPARTMENT|105|21|Less than competitive questions would not say only legal jewish circumstances; |monthly| +11254|AAAAAAAAGPLCAAAA|2452880|2452909|DEPARTMENT|105|22|So adult problems will confuse specifically obvious rights|monthly| +11255|AAAAAAAAHPLCAAAA|2452880|2452909|DEPARTMENT|105|23|Dollars establish in a degrees. Very successful years might not see n|monthly| +11256|AAAAAAAAIPLCAAAA|2452880|2452909|DEPARTMENT|105|24|Purposes ought to search vital, international surroundings; random, artistic el|monthly| +11257|AAAAAAAAJPLCAAAA|2452880|2452909|DEPARTMENT|105|25|Never standard suppliers would not become even good, final terms. Again sma|monthly| +11258|AAAAAAAAKPLCAAAA|2452880|2452909|DEPARTMENT|105|26|Then important goods could add shortly electrical wrong|monthly| +11259|AAAAAAAALPLCAAAA|2452880|2452909|DEPARTMENT|105|27|Sure, central doors answer like a arrangements; as chief holders must keep only metropol|monthly| +11260|AAAAAAAAMPLCAAAA|2452880|2452909|DEPARTMENT|105|28|Basic years would persist awfully there different parame|monthly| +11261|AAAAAAAANPLCAAAA|2452880|2452909|DEPARTMENT|105|29|Residents drop years. Years should improve actively always awfu|monthly| +11262|AAAAAAAAOPLCAAAA|2452880|2452909|DEPARTMENT|105|30|Benefits go primary lessons; silent cases cannot occur much acres.|monthly| +11263|AAAAAAAAPPLCAAAA|2452880|2452909|DEPARTMENT|105|31|Thanks could not promote times. Wages shall prefer quite different orga|monthly| +11264|AAAAAAAAAAMCAAAA|2452880|2452909|DEPARTMENT|105|32|Issues should continue in order today principal problems. Specially attrac|monthly| +11265|AAAAAAAABAMCAAAA|2452880|2452909|DEPARTMENT|105|33|As modern facts save however japanese, good countries. Empty cars used to regain as a talks. Ra|monthly| +11266|AAAAAAAACAMCAAAA|2452880|2452909|DEPARTMENT|105|34|Electoral, true years strike national cuts; cheeks wo|monthly| +11267|AAAAAAAADAMCAAAA|2452880|2452909|DEPARTMENT|105|35|Running, key facilities used to hesitate now inherent authorities. Spiritual, l|monthly| +11268|AAAAAAAAEAMCAAAA|2452880|2452909|DEPARTMENT|105|36|Here changing plans must look all so linguistic pupils. Heavy, basic insects may provide molecu|monthly| +11269|AAAAAAAAFAMCAAAA|2452880|2452909|DEPARTMENT|105|37|Ill broad workers spend quite pale incidents. Proportions can undergo |monthly| +11270|AAAAAAAAGAMCAAAA|2452880|2452909|DEPARTMENT|105|38|Negative interests can cut. Hitherto dangerous components press significantly foreign, uppe|monthly| +11271|AAAAAAAAHAMCAAAA|2452880|2452909|DEPARTMENT|105|39|Social houses could build hard, substantial periods. Services must come awful, defensive lim|monthly| +11272|AAAAAAAAIAMCAAAA|2452880|2452909|DEPARTMENT|105|40|Sharply military ministers ought to suggest. Members know quicker. Interests reduce al|monthly| +11273|AAAAAAAAJAMCAAAA|2452880|2452909|DEPARTMENT|105|41|Extensive, general rooms ask hardly with a women. So protective rules protect by a visitors|monthly| +11274|AAAAAAAAKAMCAAAA|2452880|2452909|DEPARTMENT|105|42|Assistant, doubtful committees should change small societies; educational, old others gi|monthly| +11275|AAAAAAAALAMCAAAA|2452880|2452909|DEPARTMENT|105|43|Parties shall not stimulate just operations. Often great qualities cannot announce acutely particul|monthly| +11276|AAAAAAAAMAMCAAAA|2452880|2452909|DEPARTMENT|105|44|Then complex classes know; good efforts lead now flexible, armed situations. In order foreign books|monthly| +11277|AAAAAAAANAMCAAAA|2452880|2452909|DEPARTMENT|105|45|Still national relationships ease so substantial eyes. As temporary threats allow then to|monthly| +11278|AAAAAAAAOAMCAAAA|2452880|2452909|DEPARTMENT|105|46|Wrong minutes may discuss seemingly turkish, big patients;|monthly| +11279|AAAAAAAAPAMCAAAA|2452880|2452909|DEPARTMENT|105|47|Proposals preserve to a inches. Australian proceedi|monthly| +11280|AAAAAAAAABMCAAAA|2452880|2452909|DEPARTMENT|105|48|Powerful acres come. Women leave now discussions. Social thanks should |monthly| +11281|AAAAAAAABBMCAAAA|2452880|2452909|DEPARTMENT|105|49|Just cold bones find labour, high professionals. Other images|monthly| +11282|AAAAAAAACBMCAAAA|2452880|2452909|DEPARTMENT|105|50|Easy girls should curb decidedly services; officials go ne|monthly| +11283|AAAAAAAADBMCAAAA|2452880|2452909|DEPARTMENT|105|51|Women must not want closely at a persons. Specific man|monthly| +11284|AAAAAAAAEBMCAAAA|2452880|2452909|DEPARTMENT|105|52|Peoples used to give perhaps fixed factors. Facilities may manage privately special years. Sudden|monthly| +11285|AAAAAAAAFBMCAAAA|2452880|2452909|DEPARTMENT|105|53|Red, small communities feel there now common pp.. Public, extra services watch unique, ma|monthly| +11286|AAAAAAAAGBMCAAAA|||DEPARTMENT|105||Local arts look financial, good machines. Decisions may achieve well social measu|monthly| +11287|AAAAAAAAHBMCAAAA|2452880|2452909|DEPARTMENT|105|55|Windows must let with a requirements; crazy, simple varieties could not tear substantially ri|monthly| +11288|AAAAAAAAIBMCAAAA|2452880|2452909|DEPARTMENT|105|56|Men would forget pieces. Respondents play; police like more insects. D|monthly| +11289|AAAAAAAAJBMCAAAA|2452880|2452909|DEPARTMENT|105|57|Large months kill at the pounds. Royal taxes shall not follow unknown, evolutionary i|monthly| +11290|AAAAAAAAKBMCAAAA|2452880|2452909|DEPARTMENT|105|58|Appropriate minutes get cautiously things. Other diseases expect ever for th|monthly| +11291|AAAAAAAALBMCAAAA|2452880|2452909|DEPARTMENT|105|59|Procedures show there new years. Here necessary insects shoot clean concerned leaves; eve|monthly| +11292|AAAAAAAAMBMCAAAA|2452880|2452909|DEPARTMENT|105|60|Accurate places may say standards. Tough communications used to combat almo|monthly| +11293|AAAAAAAANBMCAAAA|2452880|2452909|DEPARTMENT|105|61|Later traditional games would salvage yet. Just possible buildings beli|monthly| +11294|AAAAAAAAOBMCAAAA|2452880|2452909|DEPARTMENT|105|62|Heavy boys walk that is shallow, cautious subjects. Just l|monthly| +11295|AAAAAAAAPBMCAAAA|2452880|2452909|DEPARTMENT|105|63|Resources fear with the opportunities. Warm records must not care at all clear carers. Scores ret|monthly| +11296|AAAAAAAAACMCAAAA|2452880|2452909|DEPARTMENT|105|64|Nearly normal years add then public regulations; eggs apply places. Golden matters used to take |monthly| +11297|AAAAAAAABCMCAAAA|2452880|2452909|||65|Therefore minor characteristics cannot panic sometimes more l|| +11298|AAAAAAAACCMCAAAA|2452880|2452909|DEPARTMENT|105|66|Great, external fields should not know past famous, universal shows. Narrow|monthly| +11299|AAAAAAAADCMCAAAA|2452880|2452909|DEPARTMENT|105|67|There new laboratories explain enough schools. Authorities give so ru|monthly| +11300|AAAAAAAAECMCAAAA|2452880|2452909|DEPARTMENT|105|68|Guns ought to tell pleased effects. High needs should intensify again financ|monthly| +11301|AAAAAAAAFCMCAAAA|2452880|2452909|DEPARTMENT|105|69|Languages can sustain. Other services used to confirm flat long-term dogs.|monthly| +11302|AAAAAAAAGCMCAAAA|2452880|2452909|DEPARTMENT|105|70|Separate police save sweet, large workers. Different children should introduce too t|monthly| +11303|AAAAAAAAHCMCAAAA|2452880|2452909|DEPARTMENT|105|71|Ashamed, demanding hours look as by a trades. Safe, y|monthly| +11304|AAAAAAAAICMCAAAA|2452880|2452909|DEPARTMENT|105|72|Human figures draw today neighbours. Presidential wages c|monthly| +11305|AAAAAAAAJCMCAAAA|2452880|2452909|DEPARTMENT|105|73|Easy difficulties must not come often. Primitive, white toys preach as attra|monthly| +11306|AAAAAAAAKCMCAAAA|2452880|2452909|DEPARTMENT|105|74|Ancient publishers shall not accept; private women could sa|monthly| +11307|AAAAAAAALCMCAAAA|2452880|2452909|DEPARTMENT|105|75|Rights could return. Similarly similar interests will maintain too e|monthly| +11308|AAAAAAAAMCMCAAAA|2452880|2452909|DEPARTMENT|105|76|Actors fall and so on to a muscles. Dead images would not attend alway|monthly| +11309|AAAAAAAANCMCAAAA|2452880|2452909|DEPARTMENT|105|77|Arrangements tell then true trains. Modest, real actions might not drink part|monthly| +11310|AAAAAAAAOCMCAAAA|2452880|2452909|DEPARTMENT|105|78|Central, tough stores will want towards a eggs. Soon high complaints cannot sound anyway into a way|monthly| +11311|AAAAAAAAPCMCAAAA|2452880|2452909|DEPARTMENT|105|79|Crimes may look just hidden, male votes. Shares must get ago; hours can|monthly| +11312|AAAAAAAAADMCAAAA|2452880|2452909|DEPARTMENT|105|80|Options might not head in a methods; reasonable men allow good month|monthly| +11313|AAAAAAAABDMCAAAA|2452880|2452909|DEPARTMENT|105|81|Notes must seem goals. Still illegal eyes sustain social branches. Prime good|monthly| +11314|AAAAAAAACDMCAAAA|2452880|2452909|DEPARTMENT|105|82|New complaints cannot lie human claims. Various gods start best|monthly| +11315|AAAAAAAADDMCAAAA|2452880|2452909|DEPARTMENT|105|83|Video-taped experiences launch very authorities. Perfect, intense members promo|monthly| +11316|AAAAAAAAEDMCAAAA|2452880|2452909|DEPARTMENT|105|84|Different, widespread versions see french, existing affairs; ancient children mig|monthly| +11317|AAAAAAAAFDMCAAAA|2452880|2452909|DEPARTMENT|105|85|Conditions comment much small policies. Initial, special police might not see ago good partners; di|monthly| +11318|AAAAAAAAGDMCAAAA|2452880|2452909|DEPARTMENT|105|86|Other problems know with a documents; suspicious prizes test great factors. Over gre|monthly| +11319|AAAAAAAAHDMCAAAA|2452880|2452909|DEPARTMENT|105|87|Too minor tenants may receive easily too elegant regions. Requests should not get of course |monthly| +11320|AAAAAAAAIDMCAAAA|2452880|2452909|DEPARTMENT|105|88|Inappropriate gaps indicate possible others. Very sp|monthly| +11321|AAAAAAAAJDMCAAAA|2452880|2452909|DEPARTMENT|105|89|Now industrial arms go hopefully universal things; parents behave more effective|monthly| +11322|AAAAAAAAKDMCAAAA|2452880|2452909|DEPARTMENT|105|90|Average, large costs teach economic, ideal thoughts. Popular spirits used |monthly| +11323|AAAAAAAALDMCAAAA|2452880|2452909|DEPARTMENT|105|91|Upper chiefs used to lead also adequately brilliant sons. Colonia|monthly| +11324|AAAAAAAAMDMCAAAA|2452880|2452909|DEPARTMENT|105|92|Organic, local objectives take. Much new questions may|monthly| +11325|AAAAAAAANDMCAAAA|2452880|2452909|DEPARTMENT|105|93|Southern, clear reactions ought to see. Cognitive, busy persons will ans|monthly| +11326|AAAAAAAAODMCAAAA|2452880|2452909|DEPARTMENT|105|94|Substances ask. Signs should cease seriously. Free, good days make quickly|monthly| +11327|AAAAAAAAPDMCAAAA|2452880|2452909|DEPARTMENT|105|95|Very aware magistrates see then just, immediate circumstances. Great, national effects sense cle|monthly| +11328|AAAAAAAAAEMCAAAA|2452880|2452909|DEPARTMENT|105|96|Very popular losses shall not use desirable, specified communications. Residents may run newly |monthly| +11329|AAAAAAAABEMCAAAA|2452880|2452909|DEPARTMENT|105|97|Pupils should make just genuine transactions. Like, other lines may produce also|monthly| +11330|AAAAAAAACEMCAAAA|2452880|2452909|DEPARTMENT|105|98|Public students will not reply single, famous arms. Else local women must not drink a little a|monthly| +11331|AAAAAAAADEMCAAAA|2452880|2452909|DEPARTMENT|105|99|Private subjects would know thus busy, able leaders; practical,|monthly| +11332|AAAAAAAAEEMCAAAA|2452880|2452909|DEPARTMENT|105|100|Shops drop contemporary legs. Parts repair sometimes. Chinese facilities approach. Pure, fast types|monthly| +11333|AAAAAAAAFEMCAAAA|2452880|2452909|DEPARTMENT|105|101|About intense councils will hit over seats. Financial teeth can build here mental, early situat|monthly| +11334|AAAAAAAAGEMCAAAA|2452880|2452909|DEPARTMENT|105|102|Authorities might not agree since a players. Young hands may not escape|monthly| +11335|AAAAAAAAHEMCAAAA|2452880|2452909|DEPARTMENT|105|103|Important troops forgive interested villages; restrictions sh|monthly| +11336|AAAAAAAAIEMCAAAA|2452880|2452909|DEPARTMENT|105|104|Catholic positions present less direct complaints. Old applications could compete over |monthly| +11337|AAAAAAAAJEMCAAAA|2452880|2452909|DEPARTMENT|105|105|Small, future pieces could discuss academic factors. Complete, fixed features control open, small|monthly| +11338|AAAAAAAAKEMCAAAA|2452880|2452909|DEPARTMENT|105|106|Years hear deeply dead, tough dishes; appropriate, dead |monthly| +11339|AAAAAAAALEMCAAAA|2452880|2452909|DEPARTMENT|105|107|Only services should buy others. Claims can lose then impossible possibilities; forward numbers |monthly| +11340|AAAAAAAAMEMCAAAA|2452880|2452909|DEPARTMENT|105|108|So likely problems must help systems; old, sorry policies get deep hard directors; factors|monthly| +11341|AAAAAAAANEMCAAAA|2452910|2452939|DEPARTMENT|106|1|Though potential sorts go attractive seconds. Old, sexua|monthly| +11342|AAAAAAAAOEMCAAAA|2452910|2452939|DEPARTMENT|106|2|True, sexual prices might not command of course then complicated letters. Est|monthly| +11343|AAAAAAAAPEMCAAAA|2452910|2452939|DEPARTMENT|106|3|Immense techniques seek further. Shots will speak likewise quickl|monthly| +11344|AAAAAAAAAFMCAAAA|2452910|2452939|DEPARTMENT|106|4|Obvious managers cause right small classes. Seats will c|monthly| +11345|AAAAAAAABFMCAAAA|2452910|2452939|DEPARTMENT|106|5|New, only thanks will win judicial, available citizens. Materials develop rights. |monthly| +11346|AAAAAAAACFMCAAAA|2452910|2452939|DEPARTMENT|106|6|Safe, royal hours should find also urgent banks. Respon|monthly| +11347|AAAAAAAADFMCAAAA|2452910|2452939|DEPARTMENT|106|7|Forms complete children. Different examinations will not wish then beds. Wh|monthly| +11348|AAAAAAAAEFMCAAAA|2452910|2452939|DEPARTMENT|106|8|Great, human years used to encourage today uncomfortable hours. Single, public l|monthly| +11349|AAAAAAAAFFMCAAAA|2452910|2452939|DEPARTMENT|106|9|Small, practical rewards take long laws; far old days would switch other, othe|monthly| +11350|AAAAAAAAGFMCAAAA|2452910|2452939|DEPARTMENT|106|10|Good, key conditions associate relatively days. Mines could leave. Hours will not|monthly| +11351|AAAAAAAAHFMCAAAA|2452910|2452939|DEPARTMENT|106|11|Central wishes cannot acknowledge different experi|monthly| +11352|AAAAAAAAIFMCAAAA|2452910|2452939|DEPARTMENT|106|12|Ever accused girls make never personal parameters; new acts know tomorrow already exce|monthly| +11353|AAAAAAAAJFMCAAAA|2452910|2452939|DEPARTMENT|106|13|High, interesting fingers avoid flat more cultural g|monthly| +11354|AAAAAAAAKFMCAAAA|2452910|2452939|DEPARTMENT|106|14|Active powers will want here including the shoes; routinely european id|monthly| +11355|AAAAAAAALFMCAAAA|2452910|2452939|DEPARTMENT|106|15|Users might stop democratic, empty governments. There legitim|monthly| +11356|AAAAAAAAMFMCAAAA|2452910|2452939|DEPARTMENT|106|16|Here pale payments use. Words remove either all revolutionary achievements. Stories receive fi|monthly| +11357|AAAAAAAANFMCAAAA|2452910|2452939|DEPARTMENT|106|17|Authorities work ancient, vital readers. Lands must not speak perhaps around interior collections|monthly| +11358|AAAAAAAAOFMCAAAA|2452910|2452939|DEPARTMENT|106|18|Bodies demonstrate. New, military men cannot know ho|monthly| +11359|AAAAAAAAPFMCAAAA|2452910|2452939|DEPARTMENT|106|19|Effects give very straight automatic opportunities. Never relevant journal|monthly| +11360|AAAAAAAAAGMCAAAA|2452910|2452939|DEPARTMENT|106|20|American, collective men must not mix here. Very political yards would not mean languages. Co|monthly| +11361|AAAAAAAABGMCAAAA|2452910|2452939|DEPARTMENT|106|21|Attitudes apply resources. Over there formal novel|monthly| +11362|AAAAAAAACGMCAAAA|2452910|2452939|DEPARTMENT|106|22|Actual, white sorts can render rightly disabled, new breasts. There|monthly| +11363|AAAAAAAADGMCAAAA|2452910|2452939|DEPARTMENT|106|23|Therefore senior cases admit significant rights. Free events might imagine indeed minutes. Yet o|monthly| +11364|AAAAAAAAEGMCAAAA|2452910|2452939|||24||| +11365|AAAAAAAAFGMCAAAA|2452910|2452939|DEPARTMENT|106|25|Feelings shall not perform therefore benefits. Alone northern methods used to produce even from |monthly| +11366|AAAAAAAAGGMCAAAA|2452910|2452939|DEPARTMENT|106|26|Poor lakes may miss angry, dead mines. There singl|monthly| +11367|AAAAAAAAHGMCAAAA|2452910|2452939|DEPARTMENT|106|27|Human, national bodies result more. Families shall not reflect either misl|monthly| +11368|AAAAAAAAIGMCAAAA|2452910|2452939|DEPARTMENT|106|28|Interested, cheap lines make white walls. Thus american networks |monthly| +11369|AAAAAAAAJGMCAAAA|2452910|2452939|DEPARTMENT|106|29|Votes enable both then new machines. Dark, proper players show over e|monthly| +11370|AAAAAAAAKGMCAAAA|2452910|2452939|DEPARTMENT|106|30|Atomic buildings ask open aware, new leaves; also small areas t|monthly| +11371|AAAAAAAALGMCAAAA|2452910|2452939|DEPARTMENT|106|31|Equal, agricultural views shall not use so. Low small measur|monthly| +11372|AAAAAAAAMGMCAAAA|2452910|2452939|DEPARTMENT|106|32|Patient, international rules would create. Simply likely hands keep |monthly| +11373|AAAAAAAANGMCAAAA|2452910|2452939|DEPARTMENT|106|33|Identical, revolutionary facts could qualify further. There extraor|monthly| +11374|AAAAAAAAOGMCAAAA|2452910|2452939|DEPARTMENT|106|34|So other mammals would not co-ordinate frantically blank costs. Also distant friends|monthly| +11375|AAAAAAAAPGMCAAAA|2452910|2452939|DEPARTMENT|106|35|Industrial, big years should not defend. New modules know days. New, dark investors break. Eyes|monthly| +11376|AAAAAAAAAHMCAAAA|2452910|2452939|DEPARTMENT|106|36|Incentives get likely workers; traditional others ought to like backwards international experien|monthly| +11377|AAAAAAAABHMCAAAA|2452910|2452939|DEPARTMENT|106|37|New americans used to produce others. Otherwise expected members write just direct |monthly| +11378|AAAAAAAACHMCAAAA|2452910|2452939|DEPARTMENT|106|38|So capitalist activities give conveniently with the approaches; also separate sales may not sol|monthly| +11379|AAAAAAAADHMCAAAA|2452910|2452939|DEPARTMENT|106|39|Companies will stop visibly round others. Then fresh months |monthly| +11380|AAAAAAAAEHMCAAAA|2452910|2452939|DEPARTMENT|106|40|Underlying, major groups used to like so scottish ways; different, necessary fans get rece|monthly| +11381|AAAAAAAAFHMCAAAA|2452910|2452939|DEPARTMENT|106|41|Extra, typical prayers can work here molecular, direct wives. Formal roads show hardly indeed othe|monthly| +11382|AAAAAAAAGHMCAAAA|2452910|2452939|DEPARTMENT|106|42|Solar sales would alter in order underlying novels. Rules could not take e|monthly| +11383|AAAAAAAAHHMCAAAA|2452910|2452939|DEPARTMENT|106|43|Top, fair neighbours will make old, global coins. Lo|monthly| +11384|AAAAAAAAIHMCAAAA|2452910|2452939|DEPARTMENT|106|44|Policies sing partners. Prime, following wages shall bring wi|monthly| +11385|AAAAAAAAJHMCAAAA|2452910|2452939|DEPARTMENT|106|45|Keen, superior minutes would not expect only national, military implications; thoughtfull|monthly| +11386|AAAAAAAAKHMCAAAA|2452910|2452939|DEPARTMENT|106|46|Likely, junior programmes will apply; armed groups shall not give so ce|monthly| +11387|AAAAAAAALHMCAAAA|2452910|2452939|DEPARTMENT|106|47|At least full versions run too absolute, accurate teachers. Falls acknowledge conservative, g|monthly| +11388|AAAAAAAAMHMCAAAA|2452910|2452939|DEPARTMENT|106|48|Other, pleased matters work very old ways. Extremely russian experim|monthly| +11389|AAAAAAAANHMCAAAA|2452910|2452939|DEPARTMENT|106|49|Concerned ties act more old, serious doubts. Plans stay freshly|monthly| +11390|AAAAAAAAOHMCAAAA|2452910|2452939|DEPARTMENT|106|50|Right courses get surprising weapons; relatives could li|monthly| +11391|AAAAAAAAPHMCAAAA|2452910|2452939|DEPARTMENT|106|51|Years answer. Now wide forests can stand occasionally. Only churches present|monthly| +11392|AAAAAAAAAIMCAAAA|2452910|2452939|DEPARTMENT|106|52|Social, american votes may fill special, general weeks.|monthly| +11393|AAAAAAAABIMCAAAA|2452910|2452939|DEPARTMENT|106|53|General, safe communications become ships. Easier big men may disrupt by the gifts. Poor, bri|monthly| +11394|AAAAAAAACIMCAAAA|2452910|2452939|DEPARTMENT|106|54|Essentially small plants will not make. Available contracts go straight aga|monthly| +11395|AAAAAAAADIMCAAAA|2452910|2452939|DEPARTMENT|106|55|Light features quell here conditions. Dead rights may not get in |monthly| +11396|AAAAAAAAEIMCAAAA|||DEPARTMENT|106|||monthly| +11397|AAAAAAAAFIMCAAAA|2452910|2452939|DEPARTMENT|106|57|Slightly busy years might not see long standards. Just certain industries rest places. Fu|monthly| +11398|AAAAAAAAGIMCAAAA|2452910|2452939|DEPARTMENT|106|58|Other, distinct questions support soon loose female l|monthly| +11399|AAAAAAAAHIMCAAAA|2452910|2452939|DEPARTMENT|106|59|Buildings allow close pensions. Special men react. Well involved pers|monthly| +11400|AAAAAAAAIIMCAAAA|2452910|2452939|DEPARTMENT|106|60|Traditional, very contents take social, russian ques|monthly| +11401|AAAAAAAAJIMCAAAA|2452910|2452939|DEPARTMENT|106|61|Costs shall not love quite now environmental windows; so steep area|monthly| +11402|AAAAAAAAKIMCAAAA|2452910|2452939|DEPARTMENT|106|62|Difficult systems must not establish at first women. Certainly internatio|monthly| +11403|AAAAAAAALIMCAAAA|2452910||||63||| +11404|AAAAAAAAMIMCAAAA|2452910|2452939|DEPARTMENT|106|64|Possible, possible resources will not go clearly practical parties|monthly| +11405|AAAAAAAANIMCAAAA|2452910|2452939|DEPARTMENT|106|65|Bad, large stations seem normally full, environmen|monthly| +11406|AAAAAAAAOIMCAAAA|2452910|2452939|DEPARTMENT|106|66|Very real services shall enter limited studies. Northern schools may not see along no america|monthly| +11407|AAAAAAAAPIMCAAAA|2452910|2452939|DEPARTMENT|106|67|Military, intense considerations sustain even on a names. Dead, gr|monthly| +11408|AAAAAAAAAJMCAAAA|2452910|2452939|DEPARTMENT|106|68|Worthwhile, social police assume in a connections. New, lucky firms offer|monthly| +11409|AAAAAAAABJMCAAAA|2452910|2452939|DEPARTMENT|106|69|Public, black studies can think most developing, normal rela|monthly| +11410|AAAAAAAACJMCAAAA|2452910|2452939|DEPARTMENT|106|70|Books may not fall sometimes on a factors. Given, only minutes draw partly commonly light ch|monthly| +11411|AAAAAAAADJMCAAAA|2452910|2452939|DEPARTMENT|106|71|Everywhere eligible thousands pay somewhere. Too mean areas r|monthly| +11412|AAAAAAAAEJMCAAAA|2452910|2452939|DEPARTMENT|106|72|At present great properties consider upwards more old daug|monthly| +11413|AAAAAAAAFJMCAAAA|2452910|2452939|DEPARTMENT|106|73|Democratic girls must bite in between. Suddenly good records use obviously countries. Probab|monthly| +11414|AAAAAAAAGJMCAAAA|2452910|2452939|DEPARTMENT|106|74|Foreign, powerful examples try onwards whole members. Can|monthly| +11415|AAAAAAAAHJMCAAAA|2452910|2452939|DEPARTMENT|106|75|All young principles hear autonomous, informal sales; economic, inter|monthly| +11416|AAAAAAAAIJMCAAAA|2452910|2452939|DEPARTMENT|106|76|Usually happy ways may proceed wild computers. Again military words could not recover mor|monthly| +11417|AAAAAAAAJJMCAAAA|2452910|2452939|DEPARTMENT|106|77|Theoretical consequences should not argue. Animals cos|monthly| +11418|AAAAAAAAKJMCAAAA|2452910|2452939|DEPARTMENT|106|78|Current, likely elements ought to see. Authorities used to m|monthly| +11419|AAAAAAAALJMCAAAA|2452910|2452939|DEPARTMENT|106|79|Confidential ministers maintain often. Boys should manage wrong, e|monthly| +11420|AAAAAAAAMJMCAAAA|2452910|2452939|DEPARTMENT|106|80|Assessments think quietly perhaps international commentators. New, distinct girls should not rece|monthly| +11421|AAAAAAAANJMCAAAA|2452910|2452939|DEPARTMENT|106|81|Important, other policies keep enough; practical, other pup|monthly| +11422|AAAAAAAAOJMCAAAA|2452910|2452939|DEPARTMENT|106|82|Big values may end very pounds. Blue, special forces might re|monthly| +11423|AAAAAAAAPJMCAAAA|2452910|2452939|DEPARTMENT|106|83|Versions forgive together empty, young relatives; supporters cannot conf|monthly| +11424|AAAAAAAAAKMCAAAA|2452910|2452939|DEPARTMENT|106|84|As new tools apply in order french, recent times. Following, soft pictures|monthly| +11425|AAAAAAAABKMCAAAA|2452910|2452939|DEPARTMENT|106|85|Normal, small horses can confirm new, human characters. Required, other ad|monthly| +11426|AAAAAAAACKMCAAAA|2452910|2452939|DEPARTMENT|106|86|American dogs correspond by far different employers. Days keep fiercely probably eastern cars|monthly| +11427|AAAAAAAADKMCAAAA|2452910|2452939|DEPARTMENT|106|87|Things shall overwhelm accurately small, fine processes; growing, specific women might eat|monthly| +11428|AAAAAAAAEKMCAAAA|2452910|2452939|DEPARTMENT|106|88|Things might show just often social boys. Today separate alternatives lea|monthly| +11429|AAAAAAAAFKMCAAAA|2452910|2452939|DEPARTMENT|106|89|Various, local funds could take originally months. Vegetables bind partic|monthly| +11430|AAAAAAAAGKMCAAAA|2452910|2452939|DEPARTMENT|106|90|Sure, planned schools squeeze now different, likely women. Far, certain |monthly| +11431|AAAAAAAAHKMCAAAA|2452910|2452939|DEPARTMENT|106|91|Variable obligations ought to say religious, simple members. Clubs explain|monthly| +11432|AAAAAAAAIKMCAAAA|2452910|2452939|DEPARTMENT|106|92|In order small trees find most hundreds. About significant |monthly| +11433|AAAAAAAAJKMCAAAA|2452910|2452939|DEPARTMENT|106|93|Equal companies cover further also recent principles; also slim governments exercise hom|monthly| +11434|AAAAAAAAKKMCAAAA|2452910|2452939|DEPARTMENT|106|94|Important pupils defuse substantially cultural, abstract models; pale priorities s|monthly| +11435|AAAAAAAALKMCAAAA|2452910|2452939|DEPARTMENT|106|95|True months might not seek white circumstances. Authorities |monthly| +11436|AAAAAAAAMKMCAAAA|2452910|2452939|DEPARTMENT|106|96|Glorious groups represent now dead laws. Special cards can prevent how|monthly| +11437|AAAAAAAANKMCAAAA|2452910|2452939|DEPARTMENT|106|97|At once combined pp. raise very sports. Physical requests secure now old, social vessels; spiritu|monthly| +11438|AAAAAAAAOKMCAAAA|2452910|2452939|DEPARTMENT|106|98|Other pieces may realize possible, available games. Systems keep english, gl|monthly| +11439|AAAAAAAAPKMCAAAA|2452910|2452939|DEPARTMENT|106|99|Parties would keep vast books. Accurate years can join perhaps social principl|monthly| +11440|AAAAAAAAALMCAAAA|2452910|2452939|DEPARTMENT|106|100|Conscious schools will explore of course more than right gardens. Giant partners find most f|monthly| +11441|AAAAAAAABLMCAAAA|2452910|2452939|DEPARTMENT|106|101|Different, good responses shall agree services. Influential, controversial things embark copie|monthly| +11442|AAAAAAAACLMCAAAA|2452910|2452939|DEPARTMENT|106|102|Total, clear days keep activities. As regular police could not buy b|monthly| +11443|AAAAAAAADLMCAAAA|2452910||||||monthly| +11444|AAAAAAAAELMCAAAA|2452910|2452939|DEPARTMENT|106|104|Whole channels would not take in a holidays. Reall|monthly| +11445|AAAAAAAAFLMCAAAA|2452910|2452939|DEPARTMENT|106|105|Special hours will take possible relations. Available incomes used t|monthly| +11446|AAAAAAAAGLMCAAAA|2452910|2452939|DEPARTMENT|106|106|Brave differences float such as a issues. Hard comfortable firm|monthly| +11447|AAAAAAAAHLMCAAAA|2452910|2452939|DEPARTMENT|106|107|Of course new peasants should learn closely likely popular resources. Questions pa|monthly| +11448|AAAAAAAAILMCAAAA|2452910|2452939|DEPARTMENT|106|108|Times may phone in a children. Teachers cannot enter with a t|monthly| +11449|AAAAAAAAJLMCAAAA|2452940|2452969|DEPARTMENT|107|1|Conventional surveys used to believe heads. Ties cannot share violently at all urban students.|monthly| +11450|AAAAAAAAKLMCAAAA|2452940|2452969|DEPARTMENT|107|2|Naval, dependent forms will cope in a incentives. Grateful laws breed so into th|monthly| +11451|AAAAAAAALLMCAAAA|2452940|2452969|DEPARTMENT|107|3|As rough languages used to share traditional cars. Well pre|monthly| +11452|AAAAAAAAMLMCAAAA|2452940|2452969|DEPARTMENT|107|4|New, new indians could not want more only police. Patients give ne|monthly| +11453|AAAAAAAANLMCAAAA|2452940|2452969|DEPARTMENT|107|5|New rates might not dance. Feelings must afford conditions. Wide planes seem now|monthly| +11454|AAAAAAAAOLMCAAAA|2452940|2452969|DEPARTMENT|107|6|Full colleagues might not appeal at a colleagues. |monthly| +11455|AAAAAAAAPLMCAAAA|2452940|2452969|DEPARTMENT|107|7|Easily dark rocks must create blue lives. Thanks used to become so good children. White shows sha|monthly| +11456|AAAAAAAAAMMCAAAA|2452940|2452969|DEPARTMENT|107|8|Problems express in particular members. Bold comments get conditions. Relevant aci|monthly| +11457|AAAAAAAABMMCAAAA|2452940|2452969|DEPARTMENT|107|9|Theoretical, personal forms must not give purposes. Usually long children sho|monthly| +11458|AAAAAAAACMMCAAAA|2452940|2452969|DEPARTMENT|107|10|Rural, local children shall not appear. Interests ought to sit glad, married years; biological,|monthly| +11459|AAAAAAAADMMCAAAA|2452940|2452969|DEPARTMENT|107|11|Questions shall not become similar, right boots. Effi|monthly| +11460|AAAAAAAAEMMCAAAA|2452940|2452969|DEPARTMENT|107|12|Joint gifts take figures. Strongly electric mountains become. Benefic|monthly| +11461|AAAAAAAAFMMCAAAA|2452940|2452969|DEPARTMENT|107|13|Months vary also in a rumours; british genes see so sufficient courts. N|monthly| +11462|AAAAAAAAGMMCAAAA|2452940|2452969|DEPARTMENT|107|14|Human actions would argue. Large discussions control hard gra|monthly| +11463|AAAAAAAAHMMCAAAA|2452940|2452969|DEPARTMENT|107|15|Examples should shape from a matters. More original facts would ans|monthly| +11464|AAAAAAAAIMMCAAAA|2452940|2452969|DEPARTMENT|107|16|Main, true features might know at all birds. Horribly economic friends |monthly| +11465|AAAAAAAAJMMCAAAA|2452940|2452969|DEPARTMENT|107|17|Actual, important areas must not know really major ends. Professiona|monthly| +11466|AAAAAAAAKMMCAAAA|2452940|2452969|DEPARTMENT|107|18|Around new countries ought to submit too expensive reluctant others. Children|monthly| +11467|AAAAAAAALMMCAAAA|2452940|2452969|DEPARTMENT|107|19|Of course good lines will hear around today high parents. Small, formal securities look initially a|monthly| +11468|AAAAAAAAMMMCAAAA|2452940|2452969|DEPARTMENT|107|20|Also emotional employees must jump away. Well scottish movies ask so sweet, sufficient || +11469|AAAAAAAANMMCAAAA|2452940|2452969|DEPARTMENT|107|21|Operators could prove yesterday contracts; quite national countries ought to talk t|monthly| +11470|AAAAAAAAOMMCAAAA|2452940|2452969|DEPARTMENT|107|22|Direct obligations ought to view no longer concepts. Gentle, physical police may not|monthly| +11471|AAAAAAAAPMMCAAAA|2452940|2452969|DEPARTMENT|107|23|Benefits invest in the problems. Items should not know now simple factors. Parliamentary n|monthly| +11472|AAAAAAAAANMCAAAA|2452940|2452969|DEPARTMENT|107|24|Tonight huge chapters topple american goods. Circumstances |monthly| +11473|AAAAAAAABNMCAAAA|2452940|2452969|DEPARTMENT|107|25|Days used to carry more ahead of a supplies. Whole friends see quite personal parts. Main, vita|monthly| +11474|AAAAAAAACNMCAAAA|2452940|2452969|DEPARTMENT|107|26|There present fingers carry complete friends. Top v|monthly| +11475|AAAAAAAADNMCAAAA|2452940|2452969|DEPARTMENT|107|27|Signs see later international costs; reports shall add perfectly bags. Always left pr|monthly| +11476|AAAAAAAAENMCAAAA|2452940|2452969|DEPARTMENT|107|28|Central prices wind thereby. Gods could make already; games can trace alre|monthly| +11477|AAAAAAAAFNMCAAAA|2452940|2452969|DEPARTMENT|107|29|Large associations cannot need only british, miserable things; other cases shall not|monthly| +11478|AAAAAAAAGNMCAAAA|2452940|2452969|DEPARTMENT|107|30|As public provisions reduce so. Substantial, other limits used to deliver larg|monthly| +11479|AAAAAAAAHNMCAAAA|2452940|2452969|DEPARTMENT|107|31|Home unknown ideas know much really old consequences. |monthly| +11480|AAAAAAAAINMCAAAA|2452940|2452969|DEPARTMENT|107|32|Foods drive rare years. Specifically only men should not conform among a visitors. Share|monthly| +11481|AAAAAAAAJNMCAAAA|2452940|2452969|DEPARTMENT|107|33|Particular, economic visitors progress added results. Pp. might take living jeans. Histo|monthly| +11482|AAAAAAAAKNMCAAAA|2452940|2452969|DEPARTMENT|107|34|Determined, large blacks feel close strong communications. Original, old concessions purchase pu|monthly| +11483|AAAAAAAALNMCAAAA|2452940|2452969||107||So industrial conditions go. Public cases avoid so; onwards national policies may not play; |monthly| +11484|AAAAAAAAMNMCAAAA|2452940|2452969|DEPARTMENT|107|36|Inc papers could not owe currently earlier small pri|monthly| +11485|AAAAAAAANNMCAAAA|2452940|2452969|DEPARTMENT|107|37|Gastric, future rates may cross more only republics. Sharp, |monthly| +11486|AAAAAAAAONMCAAAA|2452940|2452969|DEPARTMENT|107|38|British, white years will not take therefore true men. Alternative studies lie. Traditions used to|monthly| +11487|AAAAAAAAPNMCAAAA|2452940|2452969|DEPARTMENT|107|39|Tiny questions ask for the tests. Walls take with a groups. D|monthly| +11488|AAAAAAAAAOMCAAAA|2452940|2452969|DEPARTMENT|107|40|Other, early nations start only democratic workers. Sorry, central friends defend however. Long |monthly| +11489|AAAAAAAABOMCAAAA|2452940|2452969|DEPARTMENT|107|41|Parallel unpleasant fees should consider hands. Soon main scores establish further cultura|monthly| +11490|AAAAAAAACOMCAAAA|2452940|2452969|DEPARTMENT|107|42|Special companies remember as potentially able sales. Global, ruling|monthly| +11491|AAAAAAAADOMCAAAA|2452940|2452969|DEPARTMENT|107|43|In short difficult consequences must draw well years; old, long pupils ought to develop |monthly| +11492|AAAAAAAAEOMCAAAA|2452940|2452969|DEPARTMENT|107|44|Away personal crimes would take. Solicitors decide major seconds. Very international years clear ev|monthly| +11493|AAAAAAAAFOMCAAAA|2452940|2452969|DEPARTMENT|107|45|Scottish, young instructions share as the doors. Really nice times may clear carefully joi|monthly| +11494|AAAAAAAAGOMCAAAA|2452940|2452969|DEPARTMENT|107|46|Great, atomic walls stress soon human, european parties. Unions could not see l|monthly| +11495|AAAAAAAAHOMCAAAA|2452940|2452969|DEPARTMENT|107|47|Famous considerations travel anyway painfully psychological years. |monthly| +11496|AAAAAAAAIOMCAAAA|2452940|2452969|DEPARTMENT|107|48|Other, huge strings could sound much together constitutional numbers. Awful, violent authorities s|monthly| +11497|AAAAAAAAJOMCAAAA|2452940|2452969|DEPARTMENT|107|49|English, important conditions make wrong things. United pupil|monthly| +11498|AAAAAAAAKOMCAAAA|2452940|2452969|DEPARTMENT|107|50|Here sensitive transactions cannot start possible, inc prisoners. National, defensive|monthly| +11499|AAAAAAAALOMCAAAA|2452940|2452969|DEPARTMENT|107|51|Rather competitive subsidies take technically growing weeks. Current chan|monthly| +11500|AAAAAAAAMOMCAAAA|2452940|2452969|DEPARTMENT|107|52|Civil problems support only teachers. Male, usual questions can |monthly| +11501|AAAAAAAANOMCAAAA|2452940|2452969|DEPARTMENT|107|53|Naturally vital sons will find relative, unable yea|monthly| +11502|AAAAAAAAOOMCAAAA|2452940|2452969|DEPARTMENT|107|54|Soldiers might develop readily national hands. Days shall tell already|monthly| +11503|AAAAAAAAPOMCAAAA|2452940|2452969|DEPARTMENT|107|55|Liable kids rely also. Typical eyes meet too more politic|monthly| +11504|AAAAAAAAAPMCAAAA|2452940|2452969|DEPARTMENT|107|56|Things might take historically to a managers. Men might|monthly| +11505|AAAAAAAABPMCAAAA|2452940|2452969|DEPARTMENT|107|57|Together english parts carry occasionally top lips. Contrary, african hands may not finance|monthly| +11506|AAAAAAAACPMCAAAA|2452940|2452969|DEPARTMENT|107|58|Other, difficult funds would not escape substantial, large ways. There old trees say. Good others d|monthly| +11507|AAAAAAAADPMCAAAA|2452940|2452969|DEPARTMENT|107|59|Systems should consider nervous elections. Almost close kinds satisfy important,|monthly| +11508|AAAAAAAAEPMCAAAA|2452940|2452969|DEPARTMENT|107|60|Years used to worsen humans. Other paintings must discredit valid others. Too new applic|monthly| +11509|AAAAAAAAFPMCAAAA|2452940|2452969|DEPARTMENT|107|61|Disabled, entire claims find redundant areas. New weeks buil|monthly| +11510|AAAAAAAAGPMCAAAA|2452940|2452969|DEPARTMENT|107|62|African settings would hold ago equivalent colleges. Specific, strange gains nee|monthly| +11511|AAAAAAAAHPMCAAAA|2452940|2452969|DEPARTMENT|107|63|Characters should like minutes; financial individuals allow yet it|monthly| +11512|AAAAAAAAIPMCAAAA|2452940|2452969|DEPARTMENT|107|64|Others face young effects. Daily, dramatic services try also cards. Econ|monthly| +11513|AAAAAAAAJPMCAAAA|2452940|2452969|DEPARTMENT|107|65|Alone simple children might not convey mostly hours. So necessa|monthly| +11514|AAAAAAAAKPMCAAAA|2452940|2452969|DEPARTMENT|107|66|Really small weapons will stay so large operational s|monthly| +11515|AAAAAAAALPMCAAAA|2452940|2452969|DEPARTMENT|107|67|Decisive institutions could not marry even only common days. So minor standards work r|monthly| +11516|AAAAAAAAMPMCAAAA|2452940|2452969|DEPARTMENT|107|68|Names would not feel separate pages. Particular ways would not become in a mat|monthly| +11517|AAAAAAAANPMCAAAA|2452940|2452969|DEPARTMENT|107|69|Overall favorite prices consider in a rates. Dreams can make. Electrical,|monthly| +11518|AAAAAAAAOPMCAAAA|2452940|2452969|DEPARTMENT|107|70|Regularly young pictures shall conduct fully estimated light arguments. Soci|monthly| +11519|AAAAAAAAPPMCAAAA|2452940|2452969|DEPARTMENT|107|71|Lonely years used to see here patients. Previous women will ensure from the ef|monthly| +11520|AAAAAAAAAANCAAAA|2452940|2452969|DEPARTMENT|107|72|Public workers ought to want almost still high visito|monthly| +11521|AAAAAAAABANCAAAA|2452940|2452969|DEPARTMENT|107|73|Machines become separately. Local, aggressive moments want originally lit|monthly| +11522|AAAAAAAACANCAAAA|2452940|2452969|DEPARTMENT|107|74|Smooth, asian systems say; strong benefits put important, technical doctors. Very other con|monthly| +11523|AAAAAAAADANCAAAA|2452940|2452969|DEPARTMENT|107|75|All objective police used to make more. Delightful elem|monthly| +11524|AAAAAAAAEANCAAAA|2452940|2452969|DEPARTMENT|107|76|White guests may work lovely, minute variations. Appa|monthly| +11525|AAAAAAAAFANCAAAA|2452940|2452969|DEPARTMENT|107|77|Terms should not make great authorities. Large stones shall not get crowds. For|monthly| +11526|AAAAAAAAGANCAAAA|2452940|2452969|DEPARTMENT|107|78|Always white allies travel so labour details. Successful principles will sm|monthly| +11527|AAAAAAAAHANCAAAA|2452940|2452969|DEPARTMENT|107|79|Historic details might not like clear customs. Active talks may tell. Other minutes s|monthly| +11528|AAAAAAAAIANCAAAA|2452940|2452969|DEPARTMENT|107|80|Very special appeals enter creatures; previous, deaf parents let careful, fit days. Inc|monthly| +11529|AAAAAAAAJANCAAAA|2452940|2452969|DEPARTMENT|107|81|Large, important conservatives take persons. Just visible m|monthly| +11530|AAAAAAAAKANCAAAA|2452940|2452969|DEPARTMENT|107|82|Formal dates may come. Matters get exactly more labour individu|monthly| +11531|AAAAAAAALANCAAAA|2452940|2452969|DEPARTMENT|107|83|Actually likely women will not ensure apparent, cheerful |monthly| +11532|AAAAAAAAMANCAAAA|2452940|2452969|DEPARTMENT|107|84|Public carers will rise however up to able tactics; new firms alert used, statutory po|monthly| +11533|AAAAAAAANANCAAAA|2452940|2452969|DEPARTMENT|107|85|Ways set great, great shows. Never close parties could make later competitive great facilit|monthly| +11534|AAAAAAAAOANCAAAA|2452940|2452969|DEPARTMENT|107|86|Areas could not keep to a guidelines. Present, long wings acquire labour, golden months. Free |monthly| +11535|AAAAAAAAPANCAAAA|2452940|2452969|DEPARTMENT|107|87|Leaders shall not give early without a years; services sp|monthly| +11536|AAAAAAAAABNCAAAA|2452940|2452969|DEPARTMENT|107|88|Here informal standards affect under a sorts; normally spectacular proteins serve here ta|monthly| +11537|AAAAAAAABBNCAAAA|2452940|2452969|DEPARTMENT|107|89|Tory, medical parents deliver awkwardly married scenes. Polish qualifications must lov|monthly| +11538|AAAAAAAACBNCAAAA|2452940|2452969|DEPARTMENT|107|90|Silver pupils used to stay social crimes. Attractive advantages get so likely pupils.|monthly| +11539|AAAAAAAADBNCAAAA|2452940|2452969|DEPARTMENT|107|91|Statistically previous events work actually schools. Available teeth must withdraw y|monthly| +11540|AAAAAAAAEBNCAAAA|2452940|2452969|DEPARTMENT|107|92|Pupils solve. As casual units help even available me|monthly| +11541|AAAAAAAAFBNCAAAA|2452940|2452969|DEPARTMENT|107|93|Current tables must fuck later. Relationships try reliable, wide powers|monthly| +11542|AAAAAAAAGBNCAAAA|2452940|2452969|DEPARTMENT|107|94|Studies characterize after a items. In addition pol|monthly| +11543|AAAAAAAAHBNCAAAA|2452940|2452969|DEPARTMENT|107|95|Accurate, occasional leaders change old numbers. Lives bring equally s|monthly| +11544|AAAAAAAAIBNCAAAA|2452940|2452969|DEPARTMENT|107|96|Typical, difficult police happen certainly well single costs. For|monthly| +11545|AAAAAAAAJBNCAAAA|2452940|2452969|DEPARTMENT|107|97|Alone regulations could see. Mature, high films might attempt. Tall, top years welcome neverthele|monthly| +11546|AAAAAAAAKBNCAAAA|2452940|2452969|DEPARTMENT|107|98|Powerful, extra schools look thinly during the feet; only tall reasons supply|monthly| +11547|AAAAAAAALBNCAAAA|2452940|2452969|DEPARTMENT|107|99|Clear, able pilots would not help most best appropriate miles. Models ought to make impro|monthly| +11548|AAAAAAAAMBNCAAAA|2452940|2452969|DEPARTMENT|107|100|Allegedly major eyes interpret high. As large houses should not know terms. Carers explain much |monthly| +11549|AAAAAAAANBNCAAAA|2452940|2452969|DEPARTMENT|107|101|Only fine youngsters would not believe plain. Ministers can |monthly| +11550|AAAAAAAAOBNCAAAA|2452940|2452969|DEPARTMENT|107|102|Specific legs avoid predominantly linear stations. Shoes mind for exam|monthly| +11551|AAAAAAAAPBNCAAAA|2452940|2452969|DEPARTMENT|107|103|Too political minutes come sooner with a references. Types kill against the expectations. Impor|monthly| +11552|AAAAAAAAACNCAAAA|2452940|2452969|DEPARTMENT|107|104|Bright, jewish feelings help. Grounds turn really. Slightly likely points|monthly| +11553|AAAAAAAABCNCAAAA|2452940|2452969|DEPARTMENT|107|105|Rapid, wrong years go as. Young structures make fin|monthly| +11554|AAAAAAAACCNCAAAA|2452940|2452969|DEPARTMENT|107|106|Also human cars might not establish later with a results. Numbers oppose origina|monthly| +11555|AAAAAAAADCNCAAAA|2452940|2452969|DEPARTMENT|107|107|So local championships can punish other, whole effects. Positive, silver|monthly| +11556|AAAAAAAAECNCAAAA|2452940|2452969|DEPARTMENT|107|108|Groups shall not take simple responses; necessary, various observ|monthly| +11557|AAAAAAAAFCNCAAAA|2452970|2452999|DEPARTMENT|108|1|Far, new children hear very there black relations. Soon concerned issues a|monthly| +11558|AAAAAAAAGCNCAAAA|2452970|2452999|DEPARTMENT|108|2|Beneficial, thin women count at first out of a cars. Private, |monthly| +11559|AAAAAAAAHCNCAAAA|2452970|2452999|DEPARTMENT|108|3|Large recordings would not face giant, new degrees. Good, different uni|monthly| +11560|AAAAAAAAICNCAAAA|2452970|2452999|DEPARTMENT|108|4|Even following rocks ought to solve other artists. Animals con|monthly| +11561|AAAAAAAAJCNCAAAA|2452970|2452999|DEPARTMENT|108|5|Drawings treat only calmly popular categories. Small, common girls could spend as positive samples|monthly| +11562|AAAAAAAAKCNCAAAA|2452970|2452999|DEPARTMENT|108|6|Greek years should not feel existing, harsh chapters. Friends|monthly| +11563|AAAAAAAALCNCAAAA|2452970|2452999|DEPARTMENT|108|7|Upwards full companies ought to send at the teachers. Better top contents make symbolic, fine |monthly| +11564|AAAAAAAAMCNCAAAA|2452970|2452999|DEPARTMENT|108|8|Relations must think never old speeches. Possible, high counci|monthly| +11565|AAAAAAAANCNCAAAA|2452970|2452999|DEPARTMENT|108|9|Too curious conditions become rural times. Scottish |monthly| +11566|AAAAAAAAOCNCAAAA|2452970|2452999|DEPARTMENT|108|10|Possible police prevent on a words; bonds operate f|monthly| +11567|AAAAAAAAPCNCAAAA|2452970|2452999|DEPARTMENT|108|11|Auditors share slightly similar sciences. Legs turn graduall|monthly| +11568|AAAAAAAAADNCAAAA|2452970|2452999|DEPARTMENT|108|12|Careful prices remain lastly willing, successful times; just exac|monthly| +11569|AAAAAAAABDNCAAAA|2452970|2452999|DEPARTMENT|108|13|Real directors would set communities. Also small effects make front weeks. Mentall|monthly| +11570|AAAAAAAACDNCAAAA|2452970|2452999|DEPARTMENT|108|14|True individuals serve more unknown years. Beautiful, |monthly| +11571|AAAAAAAADDNCAAAA|2452970|2452999|DEPARTMENT|108|15|Ridiculous, ashamed times may not separate never public proposals. Fi|monthly| +11572|AAAAAAAAEDNCAAAA|2452970|2452999|DEPARTMENT|108|16|Relatively regular systems shall not disagree present, intimate matte|monthly| +11573|AAAAAAAAFDNCAAAA|2452970|2452999|DEPARTMENT|108|17|Subsequently general dreams object as. Now french proceedings admir|monthly| +11574|AAAAAAAAGDNCAAAA|2452970|2452999|DEPARTMENT|108|18|Special, administrative borders turn about. Especially month|monthly| +11575|AAAAAAAAHDNCAAAA|2452970|2452999|DEPARTMENT|108|19|Relationships include marginal stories. Authorities begin largely imperial visitors. Polic|monthly| +11576|AAAAAAAAIDNCAAAA|2452970|2452999|DEPARTMENT|108|20|High, massive skills used to see police; unions could see major, whole years. |monthly| +11577|AAAAAAAAJDNCAAAA|2452970|2452999|DEPARTMENT|108|21|Other, polish papers will want respectively single counties. Records would purchase really per |monthly| +11578|AAAAAAAAKDNCAAAA|2452970|2452999|DEPARTMENT|108|22|Sole, new forces can equip. As pregnant sources imitate parallel; japanese, legal|monthly| +11579|AAAAAAAALDNCAAAA|2452970|2452999|DEPARTMENT|108|23|Possible novels wear normally sound stages. Young, full pubs might avoid then; bright, respon|monthly| +11580|AAAAAAAAMDNCAAAA|2452970|2452999|DEPARTMENT|108|24|Private, wild peoples might co-ordinate just thus intellectual rewards; unlikely years should know|monthly| +11581|AAAAAAAANDNCAAAA|2452970|2452999|||25|Great tons close certainly shows. Particular, french conditions get to a ages. Markets re|| +11582|AAAAAAAAODNCAAAA|2452970|2452999|DEPARTMENT|108|26|Traditional articles would not know reasonably false red increases. Goods go most in|monthly| +11583|AAAAAAAAPDNCAAAA|2452970|2452999|DEPARTMENT|108|27|Safe men could not trade from a ears; major shoulders should put |monthly| +11584|AAAAAAAAAENCAAAA|2452970|2452999|DEPARTMENT|108|28|Games reduce; huge premises ought to go at last unexpe|monthly| +11585|AAAAAAAABENCAAAA|2452970|2452999|DEPARTMENT|108|29|Material words could not change in situ very final guidelines. Volumes can put new, full times.|monthly| +11586|AAAAAAAACENCAAAA|2452970|2452999|DEPARTMENT|108|30|Then visual allegations could swell hardly back important associations. Victoria|monthly| +11587|AAAAAAAADENCAAAA|2452970|2452999|DEPARTMENT|108|31|Minor, able women comply. However new chains file. Then fine holidays sh|monthly| +11588|AAAAAAAAEENCAAAA|2452970|2452999|DEPARTMENT|108|32|Needs get slightly crucial patients. Circumstances agree medical, good years. Later present cost|monthly| +11589|AAAAAAAAFENCAAAA|2452970|2452999|DEPARTMENT|108|33|Common, electric variations say; pale companies get away alone changes.|monthly| +11590|AAAAAAAAGENCAAAA|2452970|2452999|DEPARTMENT|108|34|Underlying needs might describe heavy, permanent rights; units a|monthly| +11591|AAAAAAAAHENCAAAA|2452970|2452999|DEPARTMENT|108|35|And so on severe years leave at least suitable groups. Miles charge merely. Ea|monthly| +11592|AAAAAAAAIENCAAAA|2452970|2452999|DEPARTMENT|108|36|Essential, narrow properties might not know too on|monthly| +11593|AAAAAAAAJENCAAAA|2452970|2452999|DEPARTMENT|108|37|Items match hardly expectations. Secondary babies should not launch later existing co|monthly| +11594|AAAAAAAAKENCAAAA|2452970|2452999|DEPARTMENT|108|38|Russian, real sources get for the risks. Foreign companies b|monthly| +11595|AAAAAAAALENCAAAA|2452970|2452999|DEPARTMENT|108|39|Small, intellectual factors must seem tomorrow annual levels. Enough recent organisers indic|monthly| +11596|AAAAAAAAMENCAAAA|2452970|2452999|DEPARTMENT|108|40|Projects may not examine into a needs. Differently happy glasses wou|monthly| +11597|AAAAAAAANENCAAAA|2452970|2452999|DEPARTMENT|108|41|Major, pink tourists ought to determine much good,|monthly| +11598|AAAAAAAAOENCAAAA|2452970|2452999|DEPARTMENT|108|42|Never bad rooms mean quickly national boards. Black gro|monthly| +11599|AAAAAAAAPENCAAAA|2452970|2452999|DEPARTMENT|108|43|Angry, similar polls enforce activities. Implications p|monthly| +11600|AAAAAAAAAFNCAAAA|2452970|2452999|DEPARTMENT|108|44|Technical, young workers cover confidently early companies. High cities|monthly| +11601|AAAAAAAABFNCAAAA|2452970|2452999|DEPARTMENT|108|45|Natural, british participants must not sneak earnestly major leaders. Militar|monthly| +11602|AAAAAAAACFNCAAAA|2452970|2452999|DEPARTMENT|108|46|Really legal days would not cover bc near a members. Tests may|monthly| +11603|AAAAAAAADFNCAAAA|2452970|2452999|DEPARTMENT|108|47|Teachers will provide little, new cups; white, special metres slow comprehensive terms.|monthly| +11604|AAAAAAAAEFNCAAAA|2452970|2452999|DEPARTMENT|108|48|British, dramatic guests must love in the practices. Afraid times see patients. Oth|monthly| +11605|AAAAAAAAFFNCAAAA|2452970|2452999|DEPARTMENT|108|49|Thoughts buy however. Years materialise only other years. Bir|monthly| +11606|AAAAAAAAGFNCAAAA|2452970|2452999|DEPARTMENT|108|50|Critical, local features ought to take so natural pensioners. Well-known se|monthly| +11607|AAAAAAAAHFNCAAAA|2452970|2452999|DEPARTMENT|108|51|As willing rivals would not reduce at all already considerable tensions. Never primary cour|monthly| +11608|AAAAAAAAIFNCAAAA|2452970|2452999|DEPARTMENT|108|52|Tasks let through a taxes. Distinct, wrong situation|monthly| +11609|AAAAAAAAJFNCAAAA|2452970|2452999|DEPARTMENT|108|53|Ashamed times kill incredibly also unchanged oppor|monthly| +11610|AAAAAAAAKFNCAAAA|2452970|2452999|DEPARTMENT|108|54|Minor assets help far from in a communities. Possible, medical relatives used to constitut|monthly| +11611|AAAAAAAALFNCAAAA|2452970|2452999|DEPARTMENT|108|55|Just giant men widen. Personal, long eyes ought to send. Shoulders work heavy, black considerat|monthly| +11612|AAAAAAAAMFNCAAAA|2452970|2452999|DEPARTMENT|108|56|Clothes must make so long, amazing schools; other stan|monthly| +11613|AAAAAAAANFNCAAAA|2452970|2452999|DEPARTMENT|108|57|Shy, conscious reforms might go workers. Apparent guns warm peculiarly for the feet. Problems re|monthly| +11614|AAAAAAAAOFNCAAAA|2452970|2452999|DEPARTMENT|108|58|Beautiful times might not think still; resulting, large techniques would op|monthly| +11615|AAAAAAAAPFNCAAAA|2452970|2452999|DEPARTMENT|108|59|Services reserve international, broken rooms. Fine charges carry in a wom|monthly| +11616|AAAAAAAAAGNCAAAA|2452970|2452999|DEPARTMENT|108|60|Views could exercise economically bold practices. Here bitter employers would arrive here at a c|monthly| +11617|AAAAAAAABGNCAAAA|2452970|2452999|DEPARTMENT|108|61|Streets must see independently widespread, whole men. Degrees could labour today yellow positi|monthly| +11618|AAAAAAAACGNCAAAA|2452970|2452999|DEPARTMENT|108|62|So illegal problems shall hope barely. Normally minim|monthly| +11619|AAAAAAAADGNCAAAA|2452970|2452999|DEPARTMENT|108|63|Economically annual processes absorb nicely members. Given, appropria|monthly| +11620|AAAAAAAAEGNCAAAA|2452970|2452999|DEPARTMENT|108|64|Films grab elsewhere. Sophisticated opinions yield yet commercial years. Answers read|monthly| +11621|AAAAAAAAFGNCAAAA|2452970|2452999|DEPARTMENT|108|65|Royal services may ride sources. Increasingly quiet complaints receive working, gay eyes|monthly| +11622|AAAAAAAAGGNCAAAA|2452970|2452999|DEPARTMENT|108|66|Blind, naked hands may feel documents. American, very children could move french|monthly| +11623|AAAAAAAAHGNCAAAA|2452970|2452999|DEPARTMENT|108|67|Years last thus red, new classes; contacts suppose s|monthly| +11624|AAAAAAAAIGNCAAAA|2452970|2452999|DEPARTMENT|108|68|Readily black lines block horses. Ordinary cases wish just. St|monthly| +11625|AAAAAAAAJGNCAAAA|2452970|2452999|DEPARTMENT|108|69|Leading goods must need very animals. Cases would compel potentially things; colonial, nuclea|monthly| +11626|AAAAAAAAKGNCAAAA|2452970|2452999|DEPARTMENT|108|70|Even tiny arrangements swallow private systems. Terms will cha|monthly| +11627|AAAAAAAALGNCAAAA|2452970|2452999|DEPARTMENT|108|71|Key, effective powers could remember effects. Payments make usually ethnic, natio|monthly| +11628|AAAAAAAAMGNCAAAA|2452970|2452999|DEPARTMENT|108|72|About new protests used to fly all for a counties. As|monthly| +11629|AAAAAAAANGNCAAAA|2452970|2452999|DEPARTMENT|108|73|Also vital firms should receive americans; new circumstances get from a years. Just abl|monthly| +11630|AAAAAAAAOGNCAAAA|2452970|2452999|DEPARTMENT|108|74|Afraid, simple words leave both weak photographs. Strategic bands must see; just major women wou|monthly| +11631|AAAAAAAAPGNCAAAA|2452970|2452999|DEPARTMENT|108|75|Further present women would clear short, impossible participants. As superior show|monthly| +11632|AAAAAAAAAHNCAAAA|2452970|2452999|DEPARTMENT|108|76|Subsequent, boring tenants might get sections. About mad students play activities. Frank|monthly| +11633|AAAAAAAABHNCAAAA|2452970|2452999|DEPARTMENT|108|77|Issues could make white, independent prices; very familiar development|monthly| +11634|AAAAAAAACHNCAAAA|2452970|2452999|DEPARTMENT|108|78|Brown trusts must not meet highly available, free nations; of course n|monthly| +11635|AAAAAAAADHNCAAAA|2452970|2452999|DEPARTMENT|108|79|Applications might not avoid well particular years. Deaf rules cause clearly neighbou|monthly| +11636|AAAAAAAAEHNCAAAA|2452970|2452999|DEPARTMENT|108|80|Now busy families should venture however new questions. Soon western |monthly| +11637|AAAAAAAAFHNCAAAA|2452970|2452999|DEPARTMENT|108|81|Likely contracts would negotiate as prices. Lines search now for the man|monthly| +11638|AAAAAAAAGHNCAAAA|2452970|2452999|DEPARTMENT|108|82|Banks build about used schools. Existing, economic days can benefit briskly at least fam|monthly| +11639|AAAAAAAAHHNCAAAA|2452970|2452999|DEPARTMENT|108|83|Even confident documents make also local governments. Moreover common interactions need |monthly| +11640|AAAAAAAAIHNCAAAA|2452970|2452999|DEPARTMENT|108|84|Still left thoughts might know today unusual visible groups;|monthly| +11641|AAAAAAAAJHNCAAAA|2452970|2452999|DEPARTMENT|108|85|Appropriate, new things must ring international concessions; popular, |monthly| +11642|AAAAAAAAKHNCAAAA|2452970|2452999|DEPARTMENT|108|86|Years shall not earn both early goals. Old questions might lie therefore subjects;|monthly| +11643|AAAAAAAALHNCAAAA|2452970|2452999|DEPARTMENT|108|87|Previously statistical texts might not arise on a na|monthly| +11644|AAAAAAAAMHNCAAAA|2452970|2452999|DEPARTMENT|108|88|Rich, great masses ought to buy models. Long, natural systems may not devise lou|monthly| +11645|AAAAAAAANHNCAAAA|2452970|2452999|DEPARTMENT|108|89|Important members provide about fast private groups. Q|monthly| +11646|AAAAAAAAOHNCAAAA|2452970|2452999|DEPARTMENT|108|90|Regular families marry to the eyes. Equally special considerations pull i|monthly| +11647|AAAAAAAAPHNCAAAA|2452970|2452999|DEPARTMENT|108|91|Soviet, different words will not beat for the parts. Se|monthly| +11648|AAAAAAAAAINCAAAA|2452970|2452999|DEPARTMENT|108|92|Economic members sit from a quantities. Matters exceed in a findings; per|monthly| +11649|AAAAAAAABINCAAAA|2452970|2452999|DEPARTMENT|108|93|Products refuse other streets. Public girls should not repay|monthly| +11650|AAAAAAAACINCAAAA|2452970|2452999|DEPARTMENT|108|94|Useful levels shall not see directly new, english businesses. Indu|monthly| +11651|AAAAAAAADINCAAAA|2452970|2452999|DEPARTMENT|108|95|Always particular notes see organs. Possibilities reverse |monthly| +11652|AAAAAAAAEINCAAAA|2452970|2452999|DEPARTMENT|108|96|Specific forests seem of course. Then literary beds support admittedly young years. Possib|monthly| +11653|AAAAAAAAFINCAAAA|2452970|2452999|DEPARTMENT|108|97|Curious, soft opinions watch perhaps others. Able, extra girls must talk acute |monthly| +11654|AAAAAAAAGINCAAAA|2452970|2452999|DEPARTMENT|108|98|Terms can miss furiously in a methods. Never small ski|monthly| +11655|AAAAAAAAHINCAAAA|2452970|2452999|DEPARTMENT|108|99|Nationwide brief products accept both rural, technical circumstan|monthly| +11656|AAAAAAAAIINCAAAA|2452970|2452999|DEPARTMENT|108|100|Spots used to make regularly secret ways. Real, conventional funds fit then other dif|monthly| +11657|AAAAAAAAJINCAAAA|2452970|2452999|DEPARTMENT|108|101|Funds lose. Forward unlikely difficulties shall take actual, common backs. Reall|monthly| +11658|AAAAAAAAKINCAAAA|2452970|2452999|DEPARTMENT|108|102|Emissions must learn more personal, attractive shares. Legitimate, atomic |monthly| +11659|AAAAAAAALINCAAAA|2452970|2452999|DEPARTMENT|108|103|Commercial subjects show together boys. Ruling, boring parents give southern pressures. Therefore|monthly| +11660|AAAAAAAAMINCAAAA|2452970|2452999|DEPARTMENT|108|104|Blue delegates work about free prices. Heavy, other figu|monthly| +11661|AAAAAAAANINCAAAA|2452970|2452999|DEPARTMENT|108|105|Pupils plead only present, conventional others. Other types should no|monthly| +11662|AAAAAAAAOINCAAAA|2452970|2452999|DEPARTMENT|108|106|Away young men stop gently. Different opportunities might no|monthly| +11663|AAAAAAAAPINCAAAA|2452970|2452999|DEPARTMENT|108|107|Clearly total maps should speed tight multiple domestic relations. Soviet d|monthly| +11664|AAAAAAAAAJNCAAAA|2452970|2452999|DEPARTMENT|108|108|Legal seats shall resist in a refugees. Normal groups may find features. P|monthly| +11665|AAAAAAAABJNCAAAA|2453005|2453186|DEPARTMENT|109|1|Workers talk by the images; services combine now; levels walk fa|bi-annual| +11666|AAAAAAAACJNCAAAA|2453005|2453186|DEPARTMENT|109|2|Vehicles must tell blacks. Altogether linguistic offices come. Correct sites understand|bi-annual| +11667|AAAAAAAADJNCAAAA|2453005|2453186|DEPARTMENT|109|3|Foreign, good companies accept very simple clothes. Usually financial children mus|bi-annual| +11668|AAAAAAAAEJNCAAAA|2453005|2453186|DEPARTMENT|109|4|Affairs will not guarantee already. Total owners go implicitly as no meals; officers will take|bi-annual| +11669|AAAAAAAAFJNCAAAA|2453005|2453186|DEPARTMENT|109|5|Outer principles understand houses. Old, full parts recognize young patterns. Special, evident |bi-annual| +11670|AAAAAAAAGJNCAAAA|2453005|2453186|DEPARTMENT|109|6|Often central workers need high, early earnings. Poor, red years allow; other, proposed chairs s|bi-annual| +11671|AAAAAAAAHJNCAAAA|2453005|2453186|DEPARTMENT|109|7|Days sign however heroes. Institutions should not |bi-annual| +11672|AAAAAAAAIJNCAAAA|2453005|2453186|DEPARTMENT|109|8|New, visible workers ought to support perhaps on a activities; existing rates can pay weak, america|bi-annual| +11673|AAAAAAAAJJNCAAAA|2453005|2453186|DEPARTMENT|109|9|Clubs will ask. Net floors recover directors. Main shareholders would g|bi-annual| +11674|AAAAAAAAKJNCAAAA|2453005|2453186|DEPARTMENT|109|10|Certain, prime statements die purposes; scientific, new children could carry now|bi-annual| +11675|AAAAAAAALJNCAAAA|2453005|2453186|DEPARTMENT|109|11|Events ask beyond ago annual villages. Quite free boxes become false n|bi-annual| +11676|AAAAAAAAMJNCAAAA|2453005|2453186|DEPARTMENT|109|12|More severe cars may offer for a resources. Jobs bat there; only fixed shoulders ought to|bi-annual| +11677|AAAAAAAANJNCAAAA|2453005|2453186|DEPARTMENT|109|13|Future, other rules could retain almost; characters like and so on attempts. Visitors reach very o|bi-annual| +11678|AAAAAAAAOJNCAAAA|2453005|2453186|DEPARTMENT|109|14|Factors keep different players. Whole miles shall not understand upwards major, financial decisi|bi-annual| +11679|AAAAAAAAPJNCAAAA|2453005|2453186|DEPARTMENT|109|15|Future, different judges might take as residential words. Deep roots must not ca|bi-annual| +11680|AAAAAAAAAKNCAAAA|2453005|2453186|DEPARTMENT|109|16|Systems win words; parts will become prime positions. Other, european soldiers might fail simply lo|bi-annual| +11681|AAAAAAAABKNCAAAA|2453005|2453186|DEPARTMENT|109|17|Wings must give controls. Tonnes form fairly by a points. Enoug|bi-annual| +11682|AAAAAAAACKNCAAAA|2453005|2453186|DEPARTMENT|109|18|Individual elements go ministers. More silly remains carry only. Running parent|bi-annual| +11683|AAAAAAAADKNCAAAA|2453005|2453186|DEPARTMENT|109|19|Other affairs would not say indeed now standard profits. Nevertheless increased wages|bi-annual| +11684|AAAAAAAAEKNCAAAA|2453005|2453186|DEPARTMENT|109|20|Available hours come from the views. Able years canno|bi-annual| +11685|AAAAAAAAFKNCAAAA|2453005|2453186|DEPARTMENT|109|21|Special, large needs survive other, likely difficu|bi-annual| +11686|AAAAAAAAGKNCAAAA|2453005|2453186|DEPARTMENT|109|22|Interviews unlock always such as a children. Now possible workers distingu|bi-annual| +11687|AAAAAAAAHKNCAAAA|2453005|2453186|DEPARTMENT|109|23|Actually alone glasses should speak in the bedrooms. Constant thousands beat abou|bi-annual| +11688|AAAAAAAAIKNCAAAA|2453005|2453186|DEPARTMENT|109|24|Therefore other women shall please popular programs; other, |bi-annual| +11689|AAAAAAAAJKNCAAAA|2453005|2453186|DEPARTMENT|109|25|Deep, industrial companies look with a sports. Designs touch out |bi-annual| +11690|AAAAAAAAKKNCAAAA|2453005|2453186|DEPARTMENT|109|26|Social miles can clear most deep women. Nice judges can make a|bi-annual| +11691|AAAAAAAALKNCAAAA|2453005|2453186|DEPARTMENT|109|27|Economic women used to grow rules; likely hopes must try at least cultural patients. |bi-annual| +11692|AAAAAAAAMKNCAAAA|2453005|2453186|DEPARTMENT|109|28|Old, young shareholders go rationally with the floors. Most beautiful offers keep by a we|bi-annual| +11693|AAAAAAAANKNCAAAA|2453005|2453186|DEPARTMENT|109|29|Straight afraid services can oblige quite original, responsible processes. Youn|bi-annual| +11694|AAAAAAAAOKNCAAAA|2453005|2453186|DEPARTMENT|109|30|Rather other pilots must apply. Well thin years used |bi-annual| +11695|AAAAAAAAPKNCAAAA|2453005|2453186|DEPARTMENT|109|31|Forms think particularly from a years. Settlements used to get yesterday national demands. Ri|bi-annual| +11696|AAAAAAAAALNCAAAA|2453005|2453186|DEPARTMENT|109|32|Just old things will explain powerful, long members. Unduly neat objects may supply m|bi-annual| +11697|AAAAAAAABLNCAAAA|2453005|2453186|DEPARTMENT|109|33|More available events can see quite tasks. Unknown, foreign reports|bi-annual| +11698|AAAAAAAACLNCAAAA|2453005|2453186|DEPARTMENT|109|34|Faces ought to solve so as independent, disabled children. Days return. Even |bi-annual| +11699|AAAAAAAADLNCAAAA|2453005|2453186|DEPARTMENT|109|35|Great months accept rather fair, urgent others. Main, main action|bi-annual| +11700|AAAAAAAAELNCAAAA|2453005|2453186|DEPARTMENT|109|36|Figures can hear new teeth. Over ideal restrictions co|bi-annual| +11701|AAAAAAAAFLNCAAAA|2453005|2453186|DEPARTMENT|109|37|Available, nice motives can see parties. Easy hands must take too. Very large hou|bi-annual| +11702|AAAAAAAAGLNCAAAA|2453005|2453186|DEPARTMENT|109|38|Adequate feet find now. Sides change civil ways. The|bi-annual| +11703|AAAAAAAAHLNCAAAA|2453005|2453186|DEPARTMENT|109|39|Periods shall not come activities. Other, warm contracts|bi-annual| +11704|AAAAAAAAILNCAAAA|2453005|2453186|DEPARTMENT|109|40|Small penalties could accept following properties. Main nerves introduce bene|bi-annual| +11705|AAAAAAAAJLNCAAAA|2453005|2453186|DEPARTMENT|109|41|Dry eyes should support absolute children. Good, double councils fix. Waiting, personal st|bi-annual| +11706|AAAAAAAAKLNCAAAA|2453005|2453186|DEPARTMENT|109|42|Well harsh regions tell technical conclusions. Old fields|bi-annual| +11707|AAAAAAAALLNCAAAA|2453005|2453186|DEPARTMENT|109|43|New, good cars must like strange, black occasions. Even commercial tenants would |bi-annual| +11708|AAAAAAAAMLNCAAAA|2453005|2453186|DEPARTMENT|109|44|Independent lines postpone. Firmly deep foods shall stan|bi-annual| +11709|AAAAAAAANLNCAAAA|2453005|2453186|DEPARTMENT|109|45|Already straight results should refine good styles. However various teeth lo|bi-annual| +11710|AAAAAAAAOLNCAAAA|2453005|2453186|DEPARTMENT|109|46|Schools forget frantically on a questions. Distinctiv|bi-annual| +11711|AAAAAAAAPLNCAAAA|2453005|2453186|DEPARTMENT|109|47|Terms would not get capable, professional economies; too other powers see less military, |bi-annual| +11712|AAAAAAAAAMNCAAAA|2453005|2453186|DEPARTMENT|109|48|British pounds should build however. English, seri|bi-annual| +11713|AAAAAAAABMNCAAAA|2453005|2453186|DEPARTMENT|109|49|Sexual, dramatic blocks comply. Perfect, wide details shall make even girls. |bi-annual| +11714|AAAAAAAACMNCAAAA|2453005|2453186|DEPARTMENT|109|50|Special, equal trees examine across the years. Boats mus|bi-annual| +11715|AAAAAAAADMNCAAAA|2453005|2453186|DEPARTMENT|109|51|Neighbouring delegates care respectively aback military parties.|bi-annual| +11716|AAAAAAAAEMNCAAAA|2453005|2453186|DEPARTMENT|109|52|External, certain areas give at least views. Wet services use forms. Strange i|bi-annual| +11717|AAAAAAAAFMNCAAAA|2453005|2453186|DEPARTMENT|109|53|Further essential systems could offer clear aspects. Sp|bi-annual| +11718|AAAAAAAAGMNCAAAA|2453005|2453186|DEPARTMENT|109|54|Somewhat present views might get councils; other, big relations think only out|bi-annual| diff --git a/spark-connector/spark-common/src/test/resources/data/tpcds/catalog_returns/catalog_returns.dat b/spark-connector/spark-common/src/test/resources/data/tpcds/catalog_returns/catalog_returns.dat new file mode 100644 index 00000000000..e4e52990a3c --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/data/tpcds/catalog_returns/catalog_returns.dat @@ -0,0 +1,1440 @@ +2450926|45816|17368|14601|797995|6189|9583|14601|797995|4703|9583|1|106|2|2|30|2|47|3888.31|233.29|4121.60|91.23|1348.90|3577.24|186.64|124.43|1673.42| +2450946|74710|6295|14601|797995|6189|9583|82809|665550|991|14832|1|17|2|5|6|2|49|2490.18|99.60|2589.78|52.54|1867.39|323.72|931.57|1234.89|2019.53| +2451065|71104|3391|25383|3755|2480|5652|2311|700704|5571|12485|4|7|13|2|1|4|12|64.32|4.50|68.82|22.97|78.60|1.28|55.47|7.57|106.07| +2450954|28638|13114|76801|90299|3797|42495|71681|1168758|7154|35197|2|9|12|3|6|5|20|829.40|49.76|879.16|60.00|308.00|622.05|176.24|31.11|417.76| +2451023|44538|3614|31351|528859|5567|23122|33027|737861|54|2647|1|78|12|5|16|6|6|403.08|36.27|439.35|4.58|199.44|354.71|27.57|20.80|240.29| +2450992|10904|6656|31351|528859|5567|23122|48008|1289384|1410|27436|1|54|7|3|4|6|7|23.17|1.85|25.02|19.85|7.07|6.25|9.64|7.28|28.77| +2451016|60162|12422|77674||||14994|||4335||2||||9|3|413.94|0.00||||318.73|35.22|59.99|185.36| +2450926|30343|14980|41731|1046123|5715|32107|88927|865466|4658|2697|1|65|14|3|5|11|17|1283.67|51.34|1335.01|71.97|58.31|449.28|183.56|650.83|181.62| +2451058|53881|8335|41731|1046123|5715|32107|28053|1827904|2936|21968|1|42|11|5|17|11|9|29.79|1.19|30.98|91.16|59.67|0.00|20.85|8.94|152.02| +2451039|11891|1940|36748|83584|2773|8362|53188|1816862|2974|34286|4|11|2|1|13|12|61|2119.75|84.79|2204.54|30.32|271.45|1441.43|529.08|149.24|386.56| +2451035|37877|2389|36022|1502404|6833|19662|14294|1764552|4145|34491|1|18|3|1|17|13|27|54.81|0.00|54.81|7.04|146.07|12.05|40.19|2.57|153.11| +2450867|64669|13732|36022|1502404|6833|19662|16573|1482438|6726|38614|1|43|18|1|34|13|18|100.44|5.02|105.46|70.47|196.92|9.03|82.26|9.15|272.41| +2450952|64639|2128|24957|189998|3839|7327|95061|1639056|2195|3287|2|56|17|1|3|16|40|407.60|32.60|440.20|2.01|296.40|362.76|19.72|25.12|331.01| +2450940|15515|9586|24957|189998|3839|7327|51292|1218513|1645|32377|2|75|13|5|2|16|44|2157.76|43.15|2200.91|46.29|1280.84|1834.09|51.78|271.89|1370.28| +2450986|44014|5185|82003|599432|3708|27937|26895|1075366|3615|41511|4|9|12|1|2|18|10|339.60|30.56|370.16|25.06|157.70|196.96|45.64|97.00|213.32| +2450997|16008|4051|82003|599432|3708|27937|95809|452924|4311|28451|4|33|5|2|7|18|44|2618.44|235.65|2854.09|83.74|5117.64|968.82|65.98|1583.64|5437.03| +2450984|38620|11774|27389|27132|4828|48171|87287|326203|512|12421|4|100|8|2|31|19|44|6201.80|186.05|6387.85|29.08|1162.48|5891.71|173.65|136.44|1377.61| +2451067|22065|5074|75751|1766792|2633|25674|10993|517281|2882|33283|4|100|6|2|23|21|50|1654.00|82.70|1736.70|83.07|1034.00|1223.96|391.33|38.71|1199.77| +2450980|71718|9415|11242|541252|6015|7978|65550|640677|6379|23728|2|30|14|1|3|22|3|12.87|0.90|13.77|84.18|124.62|0.90|9.45|2.52|209.70| +2450997|73516|6014|11242|541252|6015|7978|46346|125760|4274|3988|2|45|3|2|11|22|30|2822.70|28.22|2850.92|25.25|1266.60|1072.62|122.50|1627.58|1320.07| +2450863|71183|11512|11242|541252|6015|7978|42328|1893150|6361|49172|2|21|18|4|20|22|2|64.64|5.17|69.81|66.56|77.08|1.93|52.04|10.67|148.81| +2450911|10436|8155|44703|600082|6661|6087|52371|178127|666|20805|4|73|9|3|27|23|8|85.92|1.71|87.63|0.94|50.00|65.29|13.61|7.02|52.65| +2451055|64648|7186|55764|1379528|6492|31056|97473|487719|1868|12817|1|61|11|5|31|26|17|1603.95|128.31|1732.26|92.88|1262.08|914.25|517.27|172.43|1483.27| +2450900|35973|15116|62042|643025|4008|23791|69519|445572|6601|46383|2|52|3|2|19|28|43|793.35|63.46|856.81|97.48|582.65|476.01|193.57|123.77|743.59| +2451065|54503|9790|24262|511666|7070|29274|13422|1020642|2223|43357|2|82|13|4|5|31|28|1098.16|98.83|1196.99|51.07|299.32|680.85|208.65|208.66|449.22| +2450983|7690|15460|70388|174162|3089|31663|66393|1678181|4527|42073|4|22|5|1|17|32|13|419.25|37.73|456.98|27.33|259.87|234.78|184.47|0.00|324.93| +2450896|74103|25|87486|1309701|1516|15378|43488|1596473|703|13286|1|35|9|4|9|33|1|7.80|0.31|8.11|57.79|7.22|3.51|2.35|1.94|65.32| +2450972|83387|6829|12919|717893|914|3888|60899|733587|4279|6571|1|54|7|4|1|35|32|5100.48|306.02|5406.50|83.98|377.60|5100.48|0.00|0.00|767.60| +2450927|36423|9283|53988|467550|5112|40321|99221|1235634|3167|11500|2|86|19|2|15|36|2|88.24|0.88|89.12|6.73|33.22|1.76|78.69|7.79|40.83| +2450966|47666|12661|53988|467550|5112|40321|21075|1897118|4028|22538|2|52|1|1|26|36|69|1440.72|100.85|1541.57|2.22|0.00|720.36|36.01|684.35|103.07| +2450952|75037|5635|76817|785079|1788|5250|2849|3803|6272|16388|2|7|9|3|34|37|14|377.58|30.20|407.78|95.44|684.32|332.27|19.93|25.38|809.96| +2450875|20595|262|50061|1286495|4976|25457|48250|349487|5628|15816|2|103|13|4|2|38|68|9568.28|669.77|10238.05|71.11|2192.32|2105.02|3433.09|4030.17|2933.20| +2451000|37487|11186|88898|1785329|2560|38991|4876|1456966|2056|7099|1|26|15|5|3|39|39|4710.03|0.00|4710.03|53.59|2302.56|3815.12|635.38|259.53|2356.15| +2450953|37170|4394|26283|157402|4874|43146|45523|1865664|3345|5426|4|74|9|5|31|40|40|1108.40|55.42|1163.82|77.46|408.80|1030.81|76.03|1.56|541.68| +2450962|65871|6902|7159|1915909|7059|18962|81338|488417|5969|9168|2|72|11|3|15|42|10|815.70|65.25|880.95|24.41|674.50|40.78|23.24|751.68|764.16| +2451026|77649|11450|53183|1623252|3721|25390|64385|187670|4723|40129|1|58|8|1|34|44|16|405.60|16.22|421.82|22.13|89.44|324.48|55.16|25.96|127.79| +2450932|31349|7153|53183|1623252|3721|25390|10971|1481620|29|44038|1|108|11|5|19|44|20|1854.60|37.09|1891.69|51.41|927.20|1075.66|646.52|132.42|1015.70| +2451047|54565|7582|64305|527683|6708|40486|38217|252890|3009|22728|2|98|18|2|21|45|1|9.89|0.09|9.98|5.19|16.60|5.43|2.14|2.32|21.88| +2450844|74844|625|64305|527683|6708|40486|59422|1717248|6779|39510|2|27|10|1|24|45|38|1472.50|103.07|1575.57|84.52|17.10|265.05|386.38|821.07|204.69| +2450935|54638|2722|86871|830328|6585|16842|69007|853051|4346|35994|1|78|12|5|35|46|50|1329.50|79.77|1409.27|30.45|52.00|1076.89|166.72|85.89|162.22| +2450953|36814|3464|86871|830328|6585|16842|83246|1490672|3850|46581|1|36|8|4|12|46|70|6902.00|207.06|7109.06|42.78|1401.40|5107.48|1740.68|53.84|1651.24| +2450991|18610|11164|95722|553976|3877|33331|96648|1555212|401|39143|1|74|10|2|6|47|49|3888.64|0.00|3888.64|43.74|3295.74|2605.38|667.29|615.97|3339.48| +2450953|52092|10774|48375|1166269|3010|28504|60422|1491876|7131|32339|2|43|10|5|18|48|10|892.10|26.76|918.86|75.24|1040.80|811.81|72.26|8.03|1142.80| +2450977|56863|2596|73786|196899|2059|33219|36658|678147|7063|39932|2|53|13|2|33|52|14|288.82|5.77|294.59|84.41|151.76|66.42|204.60|17.80|241.94| +2451022|23109|12164|73786|196899|2059|33219|44627|1013398|1201|43404|2|84|7|5|3|52|33|292.05|5.84|297.89|69.38|15.18|122.66|64.36|105.03|90.40| +2450911|46513|2887|85643|1592501|4159|46664|3188|1308684|1449|44567|1|11|2|2|12|55|1|21.25|0.85|22.10|94.74|2.39|17.21|0.20|3.84|97.98| +2450966|71731|15134|29127|1346606|6038|14270|16897|1493518|3358|16445|1|92|12|4|29|56|4|6.04|0.06|6.10|23.32|260.68|2.53|0.42|3.09|284.06| +2450927|44105|11264|42732|677128|5571|31567|25757|1779943|862|23028|2|34|5|1|20|57|34|3039.60|60.79|3100.39|61.21|719.78|1215.84|291.80|1531.96|841.78| +2450926|15203|13765|42732|677128|5571|31567|66948|89333|5045|23957|2|85|6|3|5|57|21|82.95|4.14|87.09|57.08|62.16|47.28|15.33|20.34|123.38| +2451022|45152|10760|31940|1030863|6139|11062|36177|42517|4863|6762|2|92|20|1|12|58|37|509.86|0.00|509.86|78.85|963.11|163.15|41.60|305.11|1041.96| +2450890|69632|17647|363|683276|4942|29055|46719|1594118|4137|26203|4|30|20|1|10|62|14|346.64|10.39|357.03|95.78|186.20|336.24|6.13|4.27|292.37| +2451025|37030|352|363|683276|4942|29055|96393|1322943|2774|37452|4|47|19|5|3|62|11|126.94|0.00|126.94|71.55|112.75|49.50|56.53|20.91|184.30| +2450900|69296|782|363|683276|4942|29055|20799|1828157|6005|49453|4|44|3|1|14|62|33|218.79|17.50|236.29|23.13|464.64|94.07|115.98|8.74|505.27| +2450844|46270|985|84758|1148029|2853|18124|98919|1126300|3007|31314|4|5|2|4|1|63|54|2701.08|162.06|2863.14|3.08|2132.46|594.23|1896.16|210.69|2297.60| +2450967|37706|9398|84758|1148029|2853|18124|50122|1718117|3509|45748|4|105|5|4|8|63|8|171.20|6.84|178.04|83.12|29.52|20.54|58.75|91.91|119.48| +2450968|42160|1255|55621|1225267|960|15053|62750|1014769|971|38373|1|93|19|1|28|64|11|434.17|26.05|460.22|55.21|248.05|347.33|86.84|0.00|329.31| +2450905|54924|242|6336|785982|3228|32874|55856|1134477|4252|47980|4|78|18|3|26|66|3|181.65|9.08|190.73|98.27|0.00|56.31|82.72|42.62|107.35| +2450916|19990|3325|6336|785982|3228|32874|41131|1110002|2615|24075|4|64|8|1|16|66|23|458.85|36.70|495.55|35.46|305.90|293.66|165.19|0.00|378.06| +2450900|68157|14866|47214|1227752|1090|17181|48436|1862046|4849|5205|2|102|13|1|27|67|42|7.14|0.57|7.71|30.38|3.78|2.07|1.41|3.66|34.73| +2450929|17521|8614|47214|1227752|1090|17181|83304|1378959|4311|2910|2|63|13|2|26|67|39|721.11|64.89|786.00|45.74|484.38|7.21|35.69|678.21|595.01| +2450994|60949|4375|47214|1227752|1090|17181|99612|1195468|3695|45522|2|17|10|2|9|67|28|182.00|12.74|194.74|11.87|49.28|127.40|3.27|51.33|73.89| +2450888|27281|14731|84907|163011|3515|44819|66807|15782|1929|45629|2|79|3|3|15|68|14|732.90|0.00|732.90|24.07|586.32|622.96|107.74|2.20|610.39| +2451013|72342|12235|78376|1694169|6315|23708|60985|398527|5033|3092|4|42|10|1|2|71|18|547.02|38.29|585.31|58.57|40.14|54.70|443.08|49.24|137.00| +2450969|20519|926|94024|589162|653|20144|58744|1117994|786|29531|1|1|13|3|33|73|64|2634.24|210.73|2844.97|91.85|1114.24|2370.81|134.34|129.09|1416.82| +2450919|86370|5200|23448|1738817|4139|42661|37880|698232|4263|1494|2|48|14|5|33|74|5|125.20|10.01|135.21|23.50|262.90|1.25|97.92|26.03|296.41| +2450917|4395|8540|99543|1096330|7070|45457|77572|900814|578|7542|2|76|7|1|25|75|3|101.04|4.04|105.08|46.13|74.91|83.86|6.35|10.83|125.08| +2450964|60432|13642|99543|1096330|7070|45457|42637|1169478|2888|44651|2|48|14|2|27|75|2|117.34|9.38|126.72|89.84|50.88|28.16|8.91|80.27|150.10| +2450891|43293|10210|77674|1087861|448|24635|99074|1540251|1673|3448|1|80|20|5|2|77|3|20.61|0.82|21.43|44.76|18.42|18.34|1.43|0.84|64.00| +2450946|82263|478|77674|||24635|||6513||1|89|||35|77|10|839.90|50.39|||425.70|||196.55|| +2450902|62016|6748|77674|1087861|448|24635|17247|1603603|380|10757|1|39|15|2|9|77|36|197.28|9.86|207.14|39.53|146.52|71.02|47.97|78.29|195.91| +2450933|34973|6506|31641|1516877|960|44596|49008|1059029|5992|46108|4|44|3|1|2|80|40|3567.60|0.00|3567.60|96.68|2583.60|3103.81|398.85|64.94|2680.28| +2450874|73899|11869|31641|1516877|960|44596|28063|1206262|2807|29752|4|63|7|2|22|80|52|2883.40|0.00|2883.40|55.61|1086.80|634.34|787.17|1461.89|1142.41| +2450931|54159|10453|44942|1441792|4242|13139|93208|109514|3643|44619|1|32|13|2|23|83|2|129.04|5.16|134.20|98.47|52.36|43.87|44.28|40.89|155.99| +2450992|34791|16831|44942|1441792|4242|13139|66137|1385649|6797|39192|1|97|3|1|16|83|12|406.92|28.48|435.40|81.31|197.64|236.01|111.09|59.82|307.43| +2450975|51666|14752|82224|47350|5629|3508|75515|274687|2921|6755|4|55|4|5|9|86|17|1077.29|43.09|1120.38|11.03|1179.97|441.68|603.82|31.79|1234.09| +2450862|82535|10946|82224|47350|5629|3508|73133|1494396|7123|45403|4|25|17|4|19|86|70|5176.50|207.06|5383.56|45.50|1293.60|5072.97|92.14|11.39|1546.16| +2450978|76680|11173|82224|47350|5629|3508|15617|398869|6573|22277|4|16|2|1|28|86|4|623.60|24.94|648.54|12.61|95.24|56.12|380.21|187.27|132.79| +2451022|73402|8354|45977|1492959|5894|36829|49464|1002026|1035|31997|2|67|17|1|30|89|15|763.80|0.00|763.80|4.63|361.80|114.57|253.19|396.04|366.43| +2450950|48909|8245|95024|552511|1551|29838|45912|1698770|416|8462|4|84|7|3|4|92|72|5989.68|539.07|6528.75|69.34|5682.24|1497.42|4402.41|89.85|6290.65| +2450822|76961|17521|95024|552511|1551|29838|95397|1886218|6642|8219|4|81|17|4|21|92|3|159.63|7.98|167.61|75.26|32.46|0.00|1.59|158.04|115.70| +2451025|38828|1544|57204|1098811|5665|18872|57204|1098811|1191|18872|4|77|10|3|16|93|11|648.56|38.91|687.47|70.54|108.02|142.68|318.70|187.18|217.47| +2451028|40911|16708|80093|1871653|1166|23126|55834|469308|147|48633|1|13|11|5|26|95|54|3159.00|31.59|3190.59|1.50|519.48|1200.42|1468.93|489.65|552.57| +2450977|31416|8476|69951|1504890|3382|48721|69951|1504890|429|48721|4|107|11|3|27|97|33|0.00|0.00|0.00|83.44|2611.62|0.00|0.00|0.00|2695.06| +2450995|61592|12518|38659|400535|811|21887|768|714707|1762|417|2|89|15|5|33|99|15|91.65|2.74|94.39|68.20|40.80|52.24|24.82|14.59|111.74| +2450878|69684|13334|38659|400535|811|21887|35553|1079964|6880|10596|2|91|4|5|11|99|26|48.10|1.44|49.54|6.32|214.24|23.56|1.47|23.07|222.00| +2450981|48520|11029|69651|896753|655|47357|31147|609116|985|11701|1|10|15|4|13|100|13|352.69|14.10|366.79|52.43|4.29|193.97|80.94|77.78|70.82| +2450931|36885|9385|53623|351944|218|30139|20172|1026902|1465|22075|4|36|11|1|19|101|21|119.49|0.00|119.49|72.97|2033.01|46.60|69.24|3.65|2105.98| +2450986|3015|7489|53623|351944|218|30139|86469|1395145|4004|29216|4|54|12|4|33|101|10|290.10|17.40|307.50|90.31|77.30|168.25|52.39|69.46|185.01| +2450941|66374|12769|9410|698752|4094|17836|8924|615789|900|35164|1|33|10|4|10|102|33|1898.82|151.90|2050.72|94.21|976.47|1253.22|464.83|180.77|1222.58| +2450891|4412|2036|9410|698752|4094|17836|9410|698752|881|17836|1|26|14|4|19|102|1|10.39|0.41|10.80|49.12|14.84|10.39|0.00|0.00|64.37| +2450836|29430|17198|80111|1835190|5916|46195|81957|1480884|1910|24936|1|106|14|2|28|103|9|61.11|3.66|64.77|74.43|48.87|8.55|33.63|18.93|126.96| +2450889|62646|6896|44648|421422|6643|4217|89102|175256|5750|31044|1|84|6|5|4|104|26|70.46|4.22|74.68|4.70|264.94|56.36|2.96|11.14|273.86| +2450876|36699|16612|44648|421422|6643|4217|32261|1383491|1012|20133|1|20|9|5|9|104|5|5.30|0.05|5.35|39.83|39.80|4.45|0.84|0.01|79.68| +2450898|23430|4663|53007|621140|4826|2959|10210|248712|4060|47808|4|24|12|5|23|105|12|1153.32|92.26|1245.58|89.75|29.52|542.06|48.90|562.36|211.53| +2450853|77970|1651|95481|663720|3159|18551|35438|519045|5508|23171|2|21|20|3|16|107|21|1340.22|26.80|1367.02|64.34|1235.01|1018.56|192.99|128.67|1326.15| +2450946|59967|6583|52770|285795|5220|16645|99902|92062|5189|29916|4|18|17|2|4|108|42|776.16|69.85|846.01|58.69|161.70|69.85|289.58|416.73|290.24| +2450956|73597|8701|24615|947475|288|31148|14782|17546|4351|36709|1|3|10|3|28|109|35|6849.15|273.96|7123.11|21.77|3311.70|4040.99|673.95|2134.21|3607.43| +2450888|72030|1822|48450|1173022|2394|37366|22518|458253|950|20847|4|51|18|3|25|110|50|403.00|32.24|435.24|81.15|76.00|120.90|104.37|177.73|189.39| +2450945|71329|12322|48450|1173022|2394|37366|22518|1494517|158|8199|4|58|2|3|24|110|11|840.40|58.82|899.22|69.08|336.16|344.56|476.00|19.84|464.06| +2450994|27412|6319|71121|1030555|7142|33376|71121|1030555|4570|33376|2|13|16|4|34|111|30|539.10|37.73|576.83|98.54|183.00|285.72|15.20|238.18|319.27| +2450952|30227|6901|40439|1182114|2636|5511|40439|1182114|6027|5511|1|3|19|2|35|112|3|74.28|2.22|76.50|60.26|53.28|33.42|9.39|31.47|115.76| +2450958|57471|13598|15288|1495357|3426|47942|55885|1340537|2164|12591|2|21|5|2|7|113|14|322.84|3.22|326.06|80.22|52.92|25.82|38.61|258.41|136.36| +2450991|16966|4340|52964|1699458|1599|35293|3738|861001|3466|43931|2|47|19|4|14|114|17|396.78|23.80|420.58|6.18|44.03|111.09|28.56|257.13|74.01| +2450888|24260|2260|52964|1699458|1599|35293|54788|1171364|2815|10666|2|2|19|5|26|114|11|797.61|23.92|821.53|3.78|167.42|374.87|257.87|164.87|195.12| +2450955|53747|3364|92771|1405329|2687|29292|92126|680999|1221|43090|1|84|3|2|14|116|22|2207.70|0.00|2207.70|55.09|1590.82|1346.69|206.64|654.37|1645.91| +2450897|38527|4975|3116|829297|5878|28859|65|63942|3359|6823|4|16|8|2|16|118|16|38.24|1.14|39.38|21.50|79.68|30.97|4.87|2.40|102.32| +2450934|58120|13610|3116|829297|5878|28859|59876|765345|3294|21643|4|62|10|1|15|118|11|1182.06|47.28|1229.34|47.46|160.05|780.15|269.27|132.64|254.79| +2450930|61667|6085|60418|1185578|4338|15098|34439|508528|3664|35953|4|77|20|4|28|119|30|660.60|0.00|660.60|14.90|124.50|6.60|340.08|313.92|139.40| +2450925|34737|6079|60418|1185578|4338|15098|79385|1264795|509|31823|4|38|5|4|28|119|7|78.33|3.91|82.24|34.18|104.44|17.23|4.27|56.83|142.53| +2450959|24968|908|30925|1164796|5067|6938|93685|704329|213|37744|2|81|3|2|13|120|10|356.10|10.68|366.78|1.81|366.60|39.17|301.08|15.85|379.09| +2450902|84800|1153|48640|760948|6056|38364|30486|579178|4690|47402|2|25|18|5|12|121|1|85.66|1.71|87.37|30.15|35.69|45.39|18.12|22.15|67.55| +2450911|66979|10094|48640|760948|6056|38364|48640|760948|6947|38364|2|64|1|1|6|121|13|331.76|19.90|351.66|98.51|117.00|155.92|26.37|149.47|235.41| +2451011|44715|5428|57876|1264678|5880|49882|67|1311165|2364|23963|2|30|14|3|19|123|36|167.04|15.03|182.07|86.77|34.20|98.55|19.17|49.32|136.00| +2450992|28648|7819|57876|1264678|5880|49882|33190|1495722|5328|42534|2|51|12|3|4|123|37|7.77|0.23|8.00|7.10|51.06|0.54|3.75|3.48|58.39| +2451012|48386|11236|15744|1377262|2780|11498|13219|1645979|1982|34338|4|16|6|2|28|124|49|2446.57|24.46|2471.03|9.50|176.40|2348.70|34.25|63.62|210.36| +2450913|64496|7432|15744|1377262|2780|11498|15744|1377262|246|11498|4|3|14|1|28|124|1|1.69|0.00|1.69|89.97|5.09|0.62|0.42|0.65|95.06| +2450990|62613|10438|68170|1197355|5293|40831|68170|1197355|3127|40831|2|55|3|2|1|125|8|992.56|89.33|1081.89|11.20|620.32|754.34|161.98|76.24|720.85| +2450890|18755|17440|231|1339159|6887|39415|77319|976527|758|17436|2|94|16|4|30|126|53|6723.58|403.41|7126.99|46.01|4745.62|5177.15|1500.03|46.40|5195.04| +2450953|60580|4498|231|1339159|6887|39415|22582|535862|502|14695|2|86|8|3|16|126|29|5047.45|100.94|5148.39|91.85|1381.27|3684.63|1076.62|286.20|1574.06| +2451048|75092|13954|231|1339159|6887|39415|55234|1603543|4520|39144|2|2|15|2|6|126|7|56.70|3.40|60.10|22.48|83.51|3.96|35.33|17.41|109.39| +2451025|35663|11041|74038|992741|4597|22853|8674|1439824|2017|41810|4|54|15|3|6|128|15|1978.20|158.25|2136.45|88.88|694.95|118.69|595.04|1264.47|942.08| +2450864|59498|3892|74038|992741|4597|22853|70519|569236|599|46362|4|6|18|1|4|128|72|0.00|0.00|0.00|86.15|4844.16|0.00|0.00|0.00|4930.31| +2450864|67407|757|74038|992741|4597|22853|35140|558739|6736|38733|4|6|11|3|35|128|72|1302.48|52.09|1354.57|26.30|180.72|1094.08|122.95|85.45|259.11| +2450951|5307|11618|74038|992741|4597|22853|64998|1771302|4482|31262|4|16|15|5|35|128|56|8238.16|576.67|8814.83|81.96|3142.16|3130.50|1787.68|3319.98|3800.79| +2450945|46431|12283|59708|366991|227|37510|71758|865964|232|22500|2|105|4|2|15|130|11|370.48|18.52|389.00|39.92|211.75|62.98|209.10|98.40|270.19| +2450922|75326|9188|59708|366991|227|37510|26974|543054|4129|36004|2|78|2|2|7|130|20|1177.60|82.43|1260.03|16.62|836.20|459.26|172.40|545.94|935.25| +2450872|11494|11884|18036|422690|1391|4492|85963|499623|1072|36521|1|104|1|5|4|132|39|3530.67|0.00|3530.67|92.22|516.36|35.30|2796.29|699.08|608.58| +2450850|69105|9080|60796|1675763|683|40003|91608|1723136|3416|7696|4|42|16|4|29|134|41|118.49|7.10|125.59|65.64|672.40|17.77|69.49|31.23|745.14| +2450902|10749|17590|37695|192413|1978|31896|25597|1107166|6372|34778|4|30|9|5|31|135|27|0.00|0.00|0.00|33.96|2788.83|0.00|0.00|0.00|2822.79| +2450892|80932|11020|56896|970825|4401|4730|24596|1464350|4145|31493|1|51|11|4|24|136|11|838.75|16.77|855.52|91.08|31.02|402.60|252.96|183.19|138.87| +2450909|73156|6550|10000|1248263|5726|49189|10000|1248263|3201|49189|2|23|20|3|23|137|43|515.57|46.40|561.97|15.73|753.36|433.07|51.15|31.35|815.49| +2451010|41160|8122|23571|1153135|746|14710|81230|854596|4921|4046|4|43|7|2|27|139|6|83.10|3.32|86.42|54.43|144.66|39.88|11.23|31.99|202.41| +2450934|35772|10618|23571|1153135|746|14710|61624|110060|1963|31019|4|50|14|5|1|139|55|1103.30|99.29|1202.59|29.34|4514.95|518.55|52.62|532.13|4643.58| +2450929|51612|3583|55100|1141712|4771|28983|61188|1639824|2705|45186|1|4|2|3|24|140|13|159.38|0.00|159.38|15.59|107.51|100.40|43.64|15.34|123.10| +2450960|84159|3970|25767|1128606|2645|29084|57015|976299|4766|15233|1|30|8|5|19|141|2|9.44|0.37|9.81|91.24|1.76|5.75|0.95|2.74|93.37| +2451048|39612|15853|64193|1454158|3676|13594|23670|1214826|4787|15148|2|89|11|2|18|143|47|1610.69|48.32|1659.01|89.10|3097.77|1111.37|329.55|169.77|3235.19| +2450969|71165|8426|64193|1454158|3676|13594|64193|1454158|4913|13594|2|90|8|2|24|143|15|549.45|21.97|571.42|37.95|631.80|153.84|328.35|67.26|691.72| +2450903|26989|10945|64193|1454158|3676|13594|2299|78230|5110|22743|2|42|16|1|9|143|2|50.94|3.05|53.99|73.40|22.64|28.01|2.75|20.18|99.09| +2450950|35107|6172|82523|179338|5753|11768|27532|1462125|2375|41025|2|8|15|3|13|144|46|86.94|6.08|93.02|68.05|43.24|24.34|48.82|13.78|117.37| +2451003|75850|10430|82523|179338|5753|11768|97015|1490804|2120|13836|2|91|10|1|25|144|1|59.14|2.95|62.09|10.94|13.96|19.51|23.77|15.86|27.85| +2450918|40682|13360|8849|1789452|6055|24705|65126|1909658|1019|30333|2|43|15|4|25|145|64|3504.00|210.24|3714.24|31.00|3020.80|2137.44|95.65|1270.91|3262.04| +2450978|48565|17326|8849|1789452|6055|24705|61720|33080|3597|47314|2|94|19|1|28|145|7|383.67|3.83|387.50|3.37|133.84|72.89|93.23|217.55|141.04| +2451041|30719|6127|23362|1730166|4970|25774|31646|1684228|6579|34483|2|54|9|5|4|146|2|222.30|4.44|226.74|93.42|23.70|88.92|32.01|101.37|121.56| +2451008|29416|2918|85927|1791050|4173|24896|33086|1802309|2738|23167|4|30|7|1|3|147|50|663.50|19.90|683.40|69.57|632.50|119.43|500.54|43.53|721.97| +2450987|67405|9565|98227|1014475|7170|23018|47804|1500885|7142|24999|4|57|3|5|7|150|4|152.24|9.13|161.37|94.19|86.04|103.52|34.10|14.62|189.36| +2451047|36831|14612|98227|1014475|7170|23018|74102|1487588|7097|16959|4|66|5|4|6|150|4|204.08|10.20|214.28|87.35|40.32|55.10|89.38|59.60|137.87| +2450836|78294|12092|98227|1014475|7170|23018|57425|759563|33|32856|4|66|19|4|32|150|20|702.60|7.02|709.62|36.30|154.40|330.22|264.38|108.00|197.72| +2450943|31204|5944|63742|1759443|5897|30798|63742|1759443|2058|30798|4|85|14|3|27|151|34|143.82|7.19|151.01|91.33|342.04|25.88|28.30|89.64|440.56| +2450963|60642|5156|63742|1759443|5897|30798|97037|1274589|712|12082|4|66|18|1|24|151|23|236.21|9.44|245.65|30.29|28.75|9.44|167.80|58.97|68.48| +2450966|5468|17377|6138|1335450|7162|31674|25719|1437425|2444|49416|2|10|19|1|20|155|32|3060.16|122.40|3182.56|66.14|474.88|1254.66|956.91|848.59|663.42| +2450921|57107|4112|95770|1561127|4437|24194|80385|1558197|5788|26132|4|57|11|2|21|156|14|135.94|10.87|146.81|22.68|89.46|55.73|36.09|44.12|123.01| +2450907|75135|13564|24511|1114265|1972|42479|18120|276934|6276|37218|1|93|17|4|6|160|33|800.91|64.07|864.98|39.91|600.60|512.58|72.08|216.25|704.58| +2450975|78047|4093|24511|1114265|1972|42479|44427|413519|5371|33576|1|19|5|2|12|160|4|165.72|6.62|172.34|62.29|93.64|33.14|124.62|7.96|162.55| +2450942|15930|308|5043|613370|4450|16377|5043|613370|6839|16377|1|93|9|3|12|163|10|987.10|49.35|1036.45|4.79|438.70|651.48|234.93|100.69|492.84| +2450997|36437|12104|5043|613370|4450|16377|58629|1159580|1012|8091|1|16|14|3|5|163|25|1248.50|49.94|1298.44|14.52|1305.25|312.12|571.19|365.19|1369.71| +2451086|79749|1933|5043|613370|4450|16377|27690|1057798|405|19812|1|14|6|3|20|163|7|391.79|23.50|415.29|5.37|587.65|152.79|31.07|207.93|616.52| +2450899|54532|13060|66034|1091101|4968|12482|99430|576638|1343|38495|2|4|17|2|9|164|25|125.50|6.27|131.77|27.82|712.00|51.45|60.72|13.33|746.09| +2450980|61443|13906|66034|1091101|4968|12482|66034|1091101|4421|12482|2|15|17|1|15|164|27|1358.10|81.48|1439.58|69.16|98.55|923.50|126.03|308.57|249.19| +2450921|61665|9146|88367|768291|5368|38092|599|810399|5990|1534|2|12|20|1|20|165|19|357.58|17.87|375.45|82.37|790.40|35.75|12.87|308.96|890.64| +2450997|51020|4978|88367|768291|5368|38092|32096|916632|7086|1140|2|55|2|1|32|165|16|1297.92|64.89|1362.81|89.60|469.92|142.77|404.30|750.85|624.41| +2451010|71998|15448|97693|353845|6444|36196|78136|691497|3427|16557|4|28|14|4|24|168|3|7.68|0.53|8.21|39.11|14.73|3.76|3.68|0.24|54.37| +2451015|25831|2404|61755|1704613|6318|4423|21523|343243|2971|38746|2|107|17|5|28|171|15|213.60|14.95|228.55|5.79|327.45|134.56|30.03|49.01|348.19| +2450908|65192|6268|10475|1830665|2408|49798|8362|514330|416|17255|4|58|12|1|7|173|41|5791.25|57.91|5849.16|19.96|658.05|4922.56|356.16|512.53|735.92| +2450832|38932|4702|87171|1861135|5946|35957|74425|540067|7007|47194|1|60|1|1|13|174|8|975.44|78.03|1053.47|33.48|426.72|224.35|315.45|435.64|538.23| +2450974|39463|13780|60519|1858010|3148|32639|83566|531368|1109|18497|2|40|20|3|14|176|8|36.80|1.10|37.90|65.80|26.48|16.56|1.61|18.63|93.38| +2450963|44356|1327|60519|1858010|3148|32639|87812|1069865|5480|35095|2|35|20|5|20|176|18|2295.36|0.00|2295.36|77.55|2100.06|1101.77|871.32|322.27|2177.61| +2450896|14407|4802|80474|411959|6997|36835|37805|1539061|4658|7055|2|7|13|5|7|177|70|597.80|53.80|651.60|47.80|315.70|89.67|259.14|248.99|417.30| +2450967|69712|6505|80474|411959|6997|36835|22008|1725344|7070|40349|2|15|20|1|30|177|2|51.28|0.51|51.79|30.38|28.22|39.48|9.91|1.89|59.11| +2450878|51622|2851|80474|411959|6997|36835|44943|1480531|5353|35422|2|74|15|4|6|177|63|360.99|14.43|375.42|50.62|155.61|267.13|38.48|55.38|220.66| +2451017|35949|10327|8338|596319|3053|44234|8338|596319|1505|44234|4|12|3|4|20|178|36|2994.12|179.64|3173.76|17.29|520.56|2425.23|347.02|221.87|717.49| +2450902|45222|6535|8338|596319|3053|44234|78053|1478230|6130|40030|4|95|10|5|5|178|1|215.37|19.38|234.75|69.89|51.50|17.22|172.39|25.76|140.77| +2450920|33873|14474|12556|1841322|6535|44101||581137|3128|||55|12|3|4|180|53||55.04|||15.37|48.92|523.31||| +2450959|58880|9484|12556|1841322|6535|44101|2845|1622444|1693|44542|2|48|16|2|21|180|10|594.20|47.53|641.73|70.32|81.40|172.31|329.07|92.82|199.25| +2450975|35622|7921|12556|1841322|6535|44101|67795|178648|6781|16914|2|10|6|4|7|180|22|2866.60|171.99|3038.59|32.43|608.08|630.65|1050.89|1185.06|812.50| +2450960|75127|7402|9542|557045|548|48051|54187|297659|514|28922|2|56|13|1|1|181|87|12661.98|506.47|13168.45|45.89|5341.80|10129.58|2101.89|430.51|5894.16| +2451033|63412|8116|14349|469749|5596|13396|73649|394915|178|3841|1|89|16|4|35|182|32|2362.24|70.86|2433.10|99.52|1049.92|2362.24|0.00|0.00|1220.30| +2450911|67075|4664|72026|1105070|3361|36805|92362|1663489|5066|37443|2|46|14|1|1|183|34|1231.48|73.88|1305.36|92.09|355.98|406.38|610.57|214.53|521.95| +2451024|86006|565|72026|1105070|3361|36805|72026|1105070|4467|36805|2|51|2|4|6|183|51|2744.31|54.88|2799.19|54.51|2286.84|1893.57|195.67|655.07|2396.23| +2450904|81582|2593|42045|804013|3934|18302|42315|285086|6360|9987|4|11|2|4|8|184|35|98.70|2.96|101.66|43.45|65.80|62.18|25.19|11.33|112.21| +2450939|39141|16588|88713|1642254|1748|3242|21592|1396033|4650|48467|2|44|18|5|2|187|31|1007.19|0.00|1007.19|47.45|118.42|110.79|573.69|322.71|165.87| +2450992|73910|6964|21064|1800220|2129|26485|97200|450967|1930|2140|2|71|3|1|31|188|76|1694.04|0.00|1694.04|27.77|338.20|271.04|156.53|1266.47|365.97| +2451025|40035|3530|4746|82447|5395|38296|61033|1708250|1618|48186|4|94|3|1|4|193|9|223.20|4.46|227.66|97.24|235.98|214.27|3.66|5.27|337.68| +2450885|61088|14300|83746|1206772|5445|23271|8611|1770114|5121|22928|4|59|3|5|34|194|34|2564.96|102.59|2667.55|84.54|1424.94|333.44|1963.73|267.79|1612.07| +2450951|82221|13142|1503|1498303|3550|28350|20404|1524135|2702|3669|4|13|17|4|26|197|63|384.93|0.00|384.93|72.68|18.90|123.17|52.35|209.41|91.58| +2450991|39004|1942|1503|1498303|3550|28350|86725|1208880|5121|28162|4|9|5|1|22|197|31|512.74|10.25|522.99|28.93|328.91|5.12|258.88|248.74|368.09| +2450853|44535|8035|1503|1498303|3550|28350|77234|1069597|1395|3244|4|29|19|5|31|197|4|713.32|28.53|741.85|16.09|291.44|299.59|74.47|339.26|336.06| +2451031|61742|10022|1503|1498303|3550|28350|34475|1146931|1109|16887|4|58|1|4|2|197|22|402.60|24.15|426.75|63.72|97.02|273.76|28.34|100.50|184.89| +2450912|58560|8518|13515|366988|3308|29581|39489|547908|6339|6817|1|47|4|5|8|202|9|1082.70|10.82|1093.52|80.85|1107.27|952.77|62.36|67.57|1198.94| +2450897|32887|17149|48049|4546|2290|29587|79396|446259|4841|26976|2|84|8|5|5|203|37|1110.37|66.62|1176.99|13.05|266.03|999.33|15.54|95.50|345.70| +2451009|37711|15991|19741|1803520|3746|22772|62769|825279|2501|17770|4|83|3|5|2|204|23|348.22|17.41|365.63|9.14|312.57|201.96|21.93|124.33|339.12| +2450945|16051|2596|19741|1803520|3746|22772|15457|659450|3473|29049|4|77|18|1|27|204|23|27.14|1.08|28.22|81.70|62.33|8.68|5.53|12.93|145.11| +2450903|39982|50|57376|926874|1619|43164|37470|4439|6568|3450|4|87|10|2|26|205|12|1330.80|0.00|1330.80|55.15|1021.32|1317.49|6.65|6.66|1076.47| +2451040|74657|13399|57376|926874|1619|43164|9482|920851|5111|171|4|21|4|4|14|205|10|17.40|0.52|17.92|2.16|367.00|16.00|0.96|0.44|369.68| +2450960|58511|8947|54871|1402036|2801|41613|49841|1113866|2630|33394|1|81|19|3|32|207|8|158.56|6.34|164.90|3.29|54.16|142.70|14.74|1.12|63.79| +2450891|69149|5833|62974|1443645|1916|16540|62974|1443645|6092|16540|4|21|13|5|27|208|2|95.02|0.00|95.02|54.64|33.18|27.55|37.10|30.37|87.82| +2450993|42943|14383|17674|1289493|6912|34731|3010|250875|7093|18958|1|31|6|4|34|209|20|1778.80|0.00|1778.80|84.72|1016.40|1156.22|174.32|448.26|1101.12| +2451054|15246|10742|17674|1289493|6912||15028|588910|3249|48388||76||2|6|209|||||||24.64||36.49|84.32| +2450930|86050|12391|17674|1289493|6912|34731|72906|1272828|6180|2379|1|76|9|1|15|209|27|1613.79|32.27|1646.06|1.49|274.05|1565.37|34.86|13.56|307.81| +2450900|70761|5812|3169|473061|3400|28939|4673|924987|4813|35792|2|108|3|2|18|211|34|765.68|22.97|788.65|71.82|598.40|91.88|20.21|653.59|693.19| +2451026|30728|10870|82128|1657260|4392|34324|28403|612962|3173|32584|1|52|17|3|14|212|44|4471.28|268.27|4739.55|98.57|1277.32|4471.28|0.00|0.00|1644.16| +2450985|77709|10130|82128|1657260|4392|34324|75687|270059|1799|30739|1|77|19|4|34|212|20|547.40|10.94|558.34|27.63|1197.60|432.44|111.51|3.45|1236.17| +2451065|43706|11509|72270|1389914|4358|45602|34968|1916312|3586|460|1|23|2|1|20|213|24|2418.00|120.90|2538.90|25.10|1302.00|2393.82|10.63|13.55|1448.00| +2450915|52592|757|72270|1389914|4358|45602|65689|186644|4804|164|1|82|4|4|24|213|41|143.91|10.07|153.98|85.02|1105.36|40.29|12.43|91.19|1200.45| +2450984|32707|3086|14269|712149|6029|38309|53111|582235|5589|42096|4|8|4|1|20|216|8|193.12|0.00|193.12|73.27|304.96|96.56|83.04|13.52|378.23| +2451024|48578|5650|93344|1413580|6263|31746|77788|711632|2882|40979|1|48|9|1|21|217|2|162.86|0.00|162.86|9.12|51.90|104.23|58.63|0.00|61.02| +2450951|36876|5840|13396|464619|3395|24218|13396|464619|558|24218|1|59|8|2|26|218|4|310.04|0.00|310.04|96.30|107.16|294.53|2.79|12.72|203.46| +2450894|17759|17630|13396|464619|3395|24218|96122|449380|6154|30928|1|62|3|2|34|218|10|540.90|32.45|573.35|91.21|263.70|416.49|104.50|19.91|387.36| +2451065|24572|4438|12388|639272|6956|23440|82836|194910|4204|47945|4|17|1|5|6|221|16|1080.32|97.22|1177.54|72.34|303.84|399.71|13.61|667.00|473.40| +2450943|35278|9550|12388|639272|6956|23440|29399|680828|3699|34523|4|55|2|1|10|221|14|60.06|5.40|65.46|36.78|67.62|10.81|27.58|21.67|109.80| +2451011|39418|16333|50462|1644862|5662|33662|962|1251339|4628|45117|1|58|4|1|25|222|6|123.24|11.09|134.33|13.15|23.40|98.59|22.18|2.47|47.64| +2451070|12818|14749|30073|675903||44716|96655|1304895|||1|||2|23|224|9|||81.35||20.25|74.57|0.66||| +2450854|77621|14467|13231|545744|3057|19523|72916|1157732|2779|11426|2|64|14|1|3|225|28|664.16|6.64|670.80|88.88|863.24|398.49|108.92|156.75|958.76| +2451038|23267|11488|49302||2728||||2485|14549|1|65||||226|6||2.79|142.77|95.18|87.84|53.19|||185.81| +2450835|6037|9344||||4194|1918||||1||12|4||226|36|||318.61||||11.76|184.32|| +2450877|40674|4292|85888|275638|4235|23585|86226|678704|157|18857|2|69|16|1|14|233|3|24.54|0.00|24.54|51.05|18.87|16.93|4.71|2.90|69.92| +2450976|76627|949|70981|749658|782|31906|38310|398814|6363|4359|4|40|13|2|26|234|52|480.48|19.21|499.69|13.07|37.44|480.48|0.00|0.00|69.72| +2450882|3935|4405|69234|811199|6764|7882|46619|252148|5161|49327|2|74|6|2|20|236|41|1899.53|37.99|1937.52|14.13|573.18|1462.63|21.84|415.06|625.30| +2450906|31434|17882|30364|406226|41|27503|48093|1094078|564|28650|4|84|14|5|24|237|55|1608.75|32.17|1640.92|95.55|965.25|128.70|1228.44|251.61|1092.97| +2451039|81044|5752|20400|31049|3745|18794|21932|680771|657|5743|1|3|14|5|6|239|11|11.11|0.22|11.33|22.26|42.13|9.88|0.55|0.68|64.61| +2451080|47804|4096|20400|31049|3745|18794|20400|31049|3339|18794|1|53|17|5|21|239|22|1260.60|12.60|1273.20|89.27|392.04|12.60|661.44|586.56|493.91| +2450974|40031|11960|84942|1313482|4869|36656|26837|1023406|7099|29239|4|66|18|1|1|240|10|1730.00|69.20|1799.20|47.71|988.60|588.20|890.60|251.20|1105.51| +2451055|51811|4921|19375|126238|6701|39564|48592|1906067|5648|35229|2|78|19|5|31|241|18|12.24|0.73|12.97|47.64|306.72|4.52|2.85|4.87|355.09| +2451050|25916|5318|25877|1885715|2706|21344|87205|1706185|3072|20359|4|106|7|4|6|242|34|2695.52|0.00|2695.52|13.63|2695.52|997.34|33.96|1664.22|2709.15| +2450992|80438|55|25877|1885715|2706|21344|3392|495522|2407|22492|4|107|14|4|1|242|2|55.18|0.55|55.73|44.86|43.94|36.97|14.38|3.83|89.35| +2451006|51545|12446|86637|1117483|3529|41975|17621|1112491|4702|22160|4|104|20|3|19|243|54|1537.38|122.99|1660.37|92.74|0.00|691.82|109.92|735.64|215.73| +2450905|46350|1040|63479|1175934|3236|6891|44311|401288|4109|130|2|61|20|1|7|245|14|251.58|22.64|274.22|53.73|180.88|246.54|2.26|2.78|257.25| +2450865|65155|9955|30606|944193|627|36214|43267|280915|3514|37713|4|60|17|1|20|246|21|1432.83|114.62|1547.45|73.69|618.03|1318.20|93.99|20.64|806.34| +2450901|15583|13472|30606|944193|627|36214|66613|1711184|1633|37329|4|67|16|2|1|246|9|722.34|7.22|729.56|41.33|73.26|556.20|136.23|29.91|121.81| +2450921|73540|4540|30606|944193|627|36214|28624|1111993|5528|37016|4|91|14|4|12|246|4|122.96|6.14|129.10|43.87|385.28|72.54|8.57|41.85|435.29| +2450898|66947|10789|46684|619168|6852|37736|41062|1327905|4191|48477|1|34|15|1|12|247|15|268.20|13.41|281.61|2.41|152.40|48.27|90.17|129.76|168.22| +2450860|19647|17239|46684|619168|6852|37736|79216|60539|4117|18448|1|99|15|3|1|247|1|22.99|0.45|23.44|30.08|28.10|20.23|0.44|2.32|58.63| +2450991|29522|16790|46684|619168|6852|37736|13226|1632216|85|47907|1|6|20|2|1|247|6|285.84|20.00|305.84|95.10|101.04|237.24|15.55|33.05|216.14| +2451030|40067|17068|52030|285427|4871|13868|13217|1553730|2665|36800|2|31|18|3|23|248|23|1115.27|100.37|1215.64|46.70|371.68|713.77|389.45|12.05|518.75| +2450970|35457|5668|12871|1351573|1475|38977|81850|897522|3203|11684|1|88|1|2|20|249|9|514.89|41.19|556.08|60.17|253.62|262.59|30.27|222.03|354.98| +2450986|71544|9590|5121|1048724|7080|20283|5121|1048724|5168|20283|4|35|19|5|16|250|34|2067.20|41.34|2108.54|69.58|2907.00|413.44|82.68|1571.08|3017.92| +2451084|59621|3049|5121|1048724|7080|20283|52270|1199231|1662|893|4|99|6|2|33|250|15|739.80|51.78|791.58|11.72|113.70|244.13|223.05|272.62|177.20| +2450908|6213|15901|20301|975317|380|7843|33171|964221|4723|44425|4|58|18|5|7|251|1|8.81|0.26|9.07|88.49|8.81|5.37|1.89|1.55|97.56| +2450927|54952|5242|16687|691102|3564|42826|62994|851463|1328|32793|4|80|15|5|5|254|63|6191.01|309.55|6500.56|6.42|1019.34|3157.41|970.75|2062.85|1335.31| +2451029|72213|14716|1230|177318|7006|38294|24478|593864|1454|37044|1|63|7|2|10|255|10|664.50|59.80|724.30|49.33|158.70|651.21|4.91|8.38|267.83| +2450971|58659|12902|1230|177318|7006|38294|17129|708358|6400|8442|1|91|8|3|20|255|76|4617.76|369.42|4987.18|89.66|4757.60|2447.41|759.62|1410.73|5216.68| +2450916|78880|3242|81551|500953|3555|42557|11491|72401|5919|22284|4|42|5|3|16|258|35|2423.05|72.69|2495.74|82.14|1930.25|654.22|707.53|1061.30|2085.08| +2451001|57850|1490|97602|1596919|1389|42306|79474|290822|5808|19547|1|29|5|1|27|260|8|256.88|7.70|264.58|82.03|225.52|218.34|17.34|21.20|315.25| +2450963|25029|3410|97602|1596919|1389|42306|52326|164478|5819|19160|1|62|10|1|29|260|17|652.80|32.64|685.44|26.33|842.35|0.00|150.14|502.66|901.32| +2450898|82595|11864|97602|1596919|1389|42306|74761|1357430|2432|20187|1|41|19|4|21|260|9|1087.92|87.03|1174.95|64.15|467.73|0.00|805.06|282.86|618.91| +2450907|29946|10252|34782|603476|170|22464|34782|603476|2701|22464|4|77|9|4|26|261|67|4884.30|48.84|4933.14|37.01|1258.93|3858.59|625.68|400.03|1344.78| +2450926|17447|5410|34782|603476|170|22464|34734|1193176|3456|45736|4|77|9|4|4|261|2|144.84|10.13|154.97|96.09|1.52|102.83|27.72|14.29|107.74| +2450981|54700|9460|57249|1091835|21|12607|73294|41497|4231|32165|2|108|20|4|28|263|18|2047.68|61.43|2109.11|15.45|729.54|532.39|1212.23|303.06|806.42| +2450885|9165|8732|67220|302457|94|38978|48950|1039817|5268|16672|1|80|18|4|35|265|1|5.91|0.35|6.26|88.71|4.30|3.07|0.02|2.82|93.36| +2451009|27794|14962|67220|302457|94|38978|43174|1137868|5943|5618|1|9|9|4|9|265|52|8411.00|84.11|8495.11|38.31|772.20|2943.85|546.71|4920.44|894.62| +2450958|33486|12700|79499|650598|897|25596|6200|795592|3498|20604|4|101|16|2|10|266|40|54.00|1.62|55.62|22.07|162.00|10.80|6.48|36.72|185.69| +2450987|9536|442|17910|291359|6408|41057|88299|100727|2158|38389|2|27|2|3|4|268|14|143.64|12.92|156.56|84.71|91.42|14.36|50.41|78.87|189.05| +2451035|74092|17852|17910|291359|6408|41057|16716|1083622|81|43740|2|40|17|4|28|268|18|2053.44|20.53|2073.97|25.93|876.78|554.42|614.59|884.43|923.24| +2450943|35041|9968|64300|161275|1097|44519|76744|1669239|6928|14170|1|99|3|2|29|269|51|4092.24|81.84|4174.08|37.10|776.73|2127.96|1158.92|805.36|895.67| +2451016|56009|8239|31043|416983|2123|11663|87670|1039536|5490|31891|1|104|13|4|3|272|11|394.35|7.88|402.23|37.58|101.64|173.51|214.21|6.63|147.10| +2450865|35794|8413|37025|1642777|2129|6234|50054|1918479|5214|41149|4|46|14|4|11|275|54|159.84|0.00|159.84|2.91|854.28|151.84|2.24|5.76|857.19| +2450847|71291|116|21392|1315015|6685|6708|60592|187385|2252|49166|1|59|8|2|1|276|15|690.30|55.22|745.52|79.03|372.30|662.68|20.99|6.63|506.55| +2450893|77291|7171|45571|1176172|1173|12174|23868|760121|612|35903|2|71|16|2|20|277|5|149.70|2.99|152.69|21.00|22.05|79.34|31.66|38.70|46.04| +2450950|66473|9739|45571|1176172|1173|12174|37014|532437|3342|10247|2|92|16|1|26|277|21|102.06|3.06|105.12|53.81|510.72|47.96|3.78|50.32|567.59| +2450943|79826|5276|45571|1176172|1173|12174|54446|1321279|6887|17695|2|5|19|4|2|277|53|7483.07|149.66|7632.73|98.61|3741.27|5238.14|179.59|2065.34|3989.54| +2451053|85004|6976|27866|459487|261|46682|23502|86058|6607|17909|4|31|15|4|16|278|30|1476.00|14.76|1490.76|61.38|773.10|324.72|115.12|1036.16|849.24| +2450984|14403|260|91015|680277|2339|17554|26921|1165428|5520|37612|4|18|15|4|21|280|4|190.44|0.00|190.44|49.47|76.68|26.66|16.37|147.41|126.15| +2450952|85763|1120|91015|680277|2339|17554|39253|950685|3622|28559|4|50|10|2|16|280|9|86.40|0.86|87.26|15.97|119.16|12.09|42.35|31.96|135.99| +2450987|18762|15958|38011|1694812|3613|16014|31543|131691|3967|27796|4|49|17|1|32|281|19|2325.60|23.25|2348.85|33.59|608.19|0.00|2116.29|209.31|665.03| +2451059|82002|3949|826|662289|4999|20462|37972|508254|5905|1556|2|94|2|3|4|282|30|3417.30|102.51|3519.81|16.02|1761.60|478.42|2674.38|264.50|1880.13| +2450889|54897|10370|7376|753683|6743|18203|25985|1322954|5763|41325|1|98|5|4|21|283|2|17.40|1.39|18.79|33.89|1.36|17.40|0.00|0.00|36.64| +2451029|41286|10960|18433|547933|3833|28933|3460|282734|198|38462|2|7|14|5|19|285|25|2159.00|107.95|2266.95|42.79|392.50|1986.28|141.63|31.09|543.24| +2450874|36583|7390|18433|547933|3833|28933|18433|547933|1308|28933|2|77|12|3|14|285|11|93.17|5.59|98.76|79.00|31.02|46.58|40.99|5.60|115.61| +2450923|61645|5386|18433|547933|3833|28933|30875|465594|5534|28321|2|40|15|5|7|285|3|244.71|2.44|247.15|39.82|38.37|193.32|34.94|16.45|80.63| +2451023|51241|1711|6881|322995|2309|44776|28418|1914062|6338|26661|4|11|3|4|7|292|60|48.60|0.48|49.08|65.04|121.80|35.47|8.79|4.34|187.32| +2450861|6731|17624|6881|322995|2309|44776|51095|1858442|4495|15301|4|99|20|3|24|292|7|639.59|38.37|677.96|72.78|155.33|31.97|559.01|48.61|266.48| +2450881|59295|10084|41132|725050|5768|21239|5050|1361236|3339|38642|1|41|11|1|24|293|68|668.44|60.15|728.59|10.07|48.28|401.06|211.23|56.15|118.50| +2450857|7141|5342|38096|1290146|2996|246|77640|64066|4904|5389|4|25|10|5|33|294|9|647.46|25.89|673.35|36.98|397.08|161.86|87.40|398.20|459.95| +2450894|25331|12571|38096|1290146|2996|246|27961|1603386|6860|22381|4|46|10|1|25|294|3|37.53|2.25|39.78|9.47|165.12|24.39|12.61|0.53|176.84| +2451013|63991|4366|73320|1246531|6942|22153|84553|37777|386|17842|4|12|16|2|1|297|6|306.12|0.00|306.12|5.01|443.88|116.32|1.89|187.91|448.89| +2450903|55727|4508|81187|203761|1088|45262|58221|629267|5130|48993|2|80|2|3|1|299|21|162.33|14.60|176.93|52.08|63.00|58.43|71.69|32.21|129.68| +2450915|71049|13076|67190|1584943|1543|37296|73942|1906464|694|45371|1|52|8|5|10|302|20|180.00|14.40|194.40|10.38|180.00|36.00|99.36|44.64|204.78| +2450974|13091|697|56296|1177356|4016|3076|81803|70390|1886|4229|2|52|17|5|21|303|33|3495.03|69.90|3564.93|13.35|1933.47|1502.86|1872.63|119.54|2016.72| +2450964|61750|7834|46159|1722970|2628|17039|61729|247172|1458|47744|4|72|15|1|22|304|17|114.07|4.56|118.63|42.75|634.27|83.27|9.54|21.26|681.58| +2450954|14277|12034|22956|1318645|5087|8329|62784|797867|5636|37005|4|36|19|3|22|308|1|81.61|7.34|88.95|53.99|68.55|52.23|18.80|10.58|129.88| +2450919|64263|2941|51251|343390|4329|2515|90562|738956|4193|27830|2|30|8|5|2|309|19|1559.14|62.36|1621.50|19.87|237.69|1309.67|236.99|12.48|319.92| +2451011|67049|8689|36304|1101290|3876|12062|42696|1453683|6628|15526|4|11|14|2|32|311|47|423.47|4.23|427.70|3.80|1990.92|29.64|3.93|389.90|1998.95| +2451018|55917|5125|12901|704020|5539|43858|97012|1201338|2354|3817|2|96|5|1|27|312|86|2368.44|23.68|2392.12|32.02|492.78|734.21|621.00|1013.23|548.48| +2450983|69888|8620|12901|704020|5539|43858|14034|196471|1305|49739|2|81|7|1|35|312|7|148.89|11.91|160.80|65.83|14.84|8.93|11.19|128.77|92.58| +2450998|52760|5845|53021|1243034|5047|34660|30227|1165111|6823|40888|2|86|10|2|27|313|22|1429.56|57.18|1486.74|19.85|175.78|1358.08|12.86|58.62|252.81| +2450991|52236|8944|53021|1243034|5047|34660|28252|1657467|5018|9016|2|18|16|2|10|313|23|653.20|39.19|692.39|7.20|1900.26|653.20|0.00|0.00|1946.65| +2450998|72410|7213|53021|1243034|5047|34660|64141|374079|4097|38515|2|15|9|2|20|313|39|933.27|46.66|979.93|34.57|357.24|223.98|148.95|560.34|438.47| +2451003|53458|9817|13676|959078|6080|9882|89966|737241|3570|10299|2|14|19|3|20|314|2|129.06|6.45|135.51|97.26|20.38|80.01|5.88|43.17|124.09| +2450881|22962|17212|77811|500200|6589|20978|97626|1247347|3788|5782|1|11|4|3|19|315|6|32.52|2.27|34.79|26.97|69.54|22.11|0.72|9.69|98.78| +2450834|41822|13795|66424|1852514|6157|14200|12139|173955|4393|46831|2|11|5|3|26|317|8|617.60|6.17|623.77|29.59|107.68|438.49|3.58|175.53|143.44| +2450955|54879|11042|66424|1852514|6157|14200|99061|1905966|6617|7715|2|84|2|3|31|317|3|30.75|1.84|32.59|8.58|14.22|8.30|18.40|4.05|24.64| +2451012|65635|1585|66424|1852514|6157|14200|30277|1833810|301|14345|2|61|11|2|18|317|9|1055.52|84.44|1139.96|66.72|544.77|73.88|343.57|638.07|695.93| +2450932|54713|10034|17800|1800817|4281|29745|17590|448696|2771|6446|1|108|18|3|5|318|2|156.82|3.13|159.95|3.82|117.04|83.11|39.06|34.65|123.99| +2451029|39337|10846|59512|289503|6247|2191|40690|70038|4755|3919|4|89|4|3|35|321|7|35.70|2.49|38.19|30.54|0.00|7.49|10.15|18.06|33.03| +2451066|74329|12040|68260|1493373|3639|45552|65464|1196442|6954|44219|1|101|8|3|20|322|14|902.86|81.25|984.11|56.53|1171.24|812.57|90.29|0.00|1309.02| +2450894|75894|934|41010|372208|6253|33051|68609|1031406|6922|26085|1|8|13|4|20|323|6|19.62|1.37|20.99|47.49|87.00|6.08|1.21|12.33|135.86| +2450979|34619|17983|89170|813258|4993|36328|32266|646941|2650|16806|2|43|14|5|21|324|61|2218.57|0.00|2218.57|12.89|316.59|1974.52|239.16|4.89|329.48| +2450848|49728|5713|89170|813258|4993|36328|70289|1552809|2789|42369|2|37|9|4|34|324|12|752.04|7.52|759.56|96.84|309.60|406.10|10.37|335.57|413.96| +2450928|36092|8798|75118|892053|4635|46017|26919|69454|2427|32247|1|103|20|5|18|325|2|17.34|0.00|17.34|65.70|72.30|11.27|1.76|4.31|138.00| +2450964|6510|14725|3104|993947|7190|22433|86310|1889260|5937|38567|1|18|4|1|26|327|4|38.88|3.49|42.37|46.70|43.28|34.21|0.93|3.74|93.47| +2450969|46280|14383|30042|268188|5534|26080|29601|1661842|112|11582|2|47|5|3|33|328|9|333.63|16.68|350.31|63.36|333.63|326.95|5.74|0.94|413.67| +2450959|34067|4622|77740|1629007|2227|10647|11420|75711|6553|33395|4|24|14|1|33|330|11|137.06|0.00|137.06|70.28|731.17|127.46|5.37|4.23|801.45| +2451061|80899|9038|32133|179214|1848|10129|40766|1315438|435|9479|2|66|17|1|8|331|47|861.98|25.85|887.83|57.48|348.74|836.12|23.27|2.59|432.07| +2450874|26264|11941|32133|179214|1848|10129|89061|1597124|5988|35699|2|69|18|2|28|331|45|4399.20|263.95|4663.15|8.30|939.15|3607.34|277.15|514.71|1211.40| +2450963|19948|1580|30027|551027|1627|43027|16201|690504|1376|29048|2|45|4|1|3|334|8|2.96|0.08|3.04|25.91|46.96|1.89|0.24|0.83|72.95| +2450990|77941|4093|30027|551027|1627|43027|71590|1585411|3726|38088|2|107|13|5|25|334|14|348.88|17.44|366.32|44.06|242.34|307.01|41.03|0.84|303.84| +2450900|19617|4729|33008|1425948|5735|27475|94644|944280|6088|35449|2|69|13|4|27|336|21|46.41|1.39|47.80|84.19|34.86|2.78|16.14|27.49|120.44| +2450873|28326|8744|57282|106418|2754|41937|33680|1480969|5319|6822|1|11|18|3|31|337|1|2.79|0.16|2.95|29.30|5.80|0.19|1.17|1.43|35.26| +2451020|18848|2779|81884|317321|7158|32148|23448|208675|6400|39725|2|90|13|3|18|338|1|34.55|2.76|37.31|81.89|21.03|16.23|13.19|5.13|105.68| +2450910|17288|14365|81884|317321|7158|32148|52434|100776|6323|47517|2|97|5|2|21|338|29|24.65|0.98|25.63|63.04|76.56|11.09|5.69|7.87|140.58| +2451039|56175|13021|17030|1101773|2516|859|17030|1101773|2367|859|1|51|18|4|7|339|12|70.32|6.32|76.64|1.26|6.96|18.98|50.31|1.03|14.54| +2451004|30593|11464|12673|1636541|7162|26230|82254|1457019|1193|38967|2|17|2|1|23|340|36|1515.96|60.63|1576.59|98.85|298.08|1455.32|4.85|55.79|457.56| +2451000|38515|4159|12673|1636541|7162|26230|26990|1068485|5160|40282|2|72|14|4|14|340|73|894.25|71.54|965.79|3.38|148.92|330.87|315.49|247.89|223.84| +2450905|46388|15025|97910|449922|5581|43993|91217|1646373|6423|17673|4|90|16|1|21|341|17|2612.39|78.37|2690.76|91.78|1199.52|391.85|1398.94|821.60|1369.67| +2450946|17106|1189|97910|449922|5581|43993|37889|1814638|5654|29517|4|3|12|3|9|341|88|1874.40|18.74|1893.14|29.05|3749.68|262.41|96.71|1515.28|3797.47| +2450934|69482|7462|28274||4098|10410||||48281|4|12||5||343||362.90||377.41|47.61|||169.62|127.96|757.82| +2450917|40505|16528|93222|1556974|1526|14925|11657|246245|2175|19305|1|96|1|1|11|345|27|173.07|13.84|186.91|31.66|157.95|64.03|7.63|101.41|203.45| +2450946|82096|7948|4658|1509815|3772|10082|78821|9643|1190|45937|1|46|18|5|32|346|31|919.15|27.57|946.72|79.76|735.32|661.78|213.61|43.76|842.65| +2451008|68765|6133|77547|244881|1462|5196|91032|604533|423|29160|4|5|17|4|9|348|47|3422.07|171.10|3593.17|44.46|1777.54|1745.25|637.19|1039.63|1993.10| +2450888|64445|8005|77547|244881|1462|5196|68342|1629489|1024|13806|4|29|18|3|25|348|9|101.97|6.11|108.08|40.45|448.92|91.77|7.85|2.35|495.48| +2451000|3957|626|29624|280947|6223|26346|79138|813819|3883|31547|1|24|12|4|18|349|8|420.32|12.60|432.92|1.95|194.80|306.83|47.66|65.83|209.35| +2450919|40704|13033|50856|1547754|2252|5903|93801|932173|2226|29926|2|19|12|2|35|352|41|480.11|19.20|499.31|40.81|95.94|345.67|102.17|32.27|155.95| +2451019|71839|32|67671|1211509|6547|37585|54216|55061|1470|326|2|66|20|4|30|353|8|745.20|44.71|789.91|59.21|235.92|685.58|26.82|32.80|339.84| +2450964|64288|5776|67671|1211509|6547|37585|61716|1278009|7199|43542|2|18|11|4|27|353|25|2125.00|21.25|2146.25|89.04|0.00|1466.25|46.11|612.64|110.29| +2451069|59050|7304|67671|1211509|6547|37585|46606|737355|3102|20049|2|1|10|2|24|353|38|2397.80|167.84|2565.64|67.64|245.86|527.51|224.43|1645.86|481.34| +2451009|54017|295|30393|123740|4734|33681|35841|1270350|878|47806|4|30|10|3|11|357|1|115.63|9.25|124.88|63.91|103.83|83.25|18.45|13.93|176.99| +2450980|263|17842|30393|123740|4734|33681|30393|123740|3619|33681|4|29|14|1|20|357|59|1322.78|105.82|1428.60|20.87|1469.69|119.05|409.26|794.47|1596.38| +2451050|29590|1465|29877|1732666|2655|19844|32924|649436|4141|27693|1|97|15|2|29|358|13|123.11|3.69|126.80|82.51|82.03|110.79|1.84|10.48|168.23| +2450928|47119|68|77219|533750|5034|36318|44230|1442785|4593|33201|2|86|5|3|1|360|10|1720.40|120.42|1840.82|49.13|325.90|1393.52|222.27|104.61|495.45| +2450944|77397|17662|77219|533750|5034|36318|87432|877259|6451|11996|2|66|5|2|2|360|9|21.06|1.47|22.53|95.69|32.40|9.05|6.96|5.05|129.56| +2451071|73862|8494|17680|1025907|2134|31608|4120|587803|2820|28684|2|51|16|5|19|362|68|2153.56|0.00|2153.56|99.65|2512.60|1205.99|701.20|246.37|2612.25| +2450946|55718|14980|66323|1046402|881|39713|93406|496293|890|17887|4|80|18|5|7|363|13|376.61|15.06|391.67|7.11|148.33|64.02|115.65|196.94|170.50| +2450937|46728|4858||||||||||||5|3|364||46.38|||39.73|30.12|32.46|4.45||73.56| +2451017|40556|13201|27411|535585|5359|25533|9320|1629396|4764|10132|4|95|9|2|5|366|90|5233.50|314.01|5547.51|65.25|5607.90|1465.38|3768.12|0.00|5987.16| +2450955|66615|6638|99707|1691205|4656|5354|99639|993069|2223|15777|1|65|6|5|16|367|51|3841.32|192.06|4033.38|31.49|1646.28|2496.85|537.78|806.69|1869.83| +2450941|17325|5372|22315|395529|1190|46804|22315|395529|529|46804|1|1|13|4|35|368|12|460.80|9.21|470.01|52.15|115.20|50.68|233.76|176.36|176.56| +2451022|36400|457|22315|395529|1190|46804|54091|1455441|3765|44136|1|20|15|4|3|368|45|2189.70|21.89|2211.59|56.58|1901.70|2167.80|9.41|12.49|1980.17| +2450903|1103|8756|22315|395529|1190|46804|6264|1595720|5346|21819|1|24|4|2|3|368|35|942.20|56.53|998.73|19.69|488.95|301.50|179.39|461.31|565.17| +2450880|21489|16942|70508|1510540|172|13004|74771|834135|4705|14828|1|49|17|2|10|372|2|88.72|2.66|91.38|23.57|73.94|32.82|36.89|19.01|100.17| +2451015|81048|2074|21541|52468|3842|34369|21541|52468|5748|34369|4|100|7|2|10|373|12|185.40|12.97|198.37|55.74|320.40|27.81|141.83|15.76|389.11| +2450990|36436|214|85437|677689|3094|33699|44268|500323|3267|45411|4|39|3|2|26|374|67|4176.11|250.56|4426.67|57.04|5055.82|1628.68|764.22|1783.21|5363.42| +2450873|71725|17642|58488|1543501|67|28680|74977|465085|3637|30189|4|24|18|5|32|376|43|4754.08|427.86|5181.94|68.13|735.73|1283.60|312.34|3158.14|1231.72| +2450954|3625|2686|87700|1858847|3594|23094|84095|1796396|993|13190|1|43|8|5|22|378|8|53.36|1.06|54.42|35.35|72.80|34.15|10.18|9.03|109.21| +2450965|33407|15328|29445|849227|656|5638|33213|556164|6715|44419|4|80|18|4|3|379|19|104.69|8.37|113.06|54.55|71.44|99.45|0.41|4.83|134.36| +2450944|78025|12656|80172|1082466|6360|39101|51937|798531|1904|30477|2|84|16|4|27|380|63|2337.30|0.00|2337.30|31.50|1314.81|1355.63|68.71|912.96|1346.31| +2450973|45750|5738|44118|1712118|471|44471|76482|1238132|3767|41449|4|82|8|5|12|382|24|1569.60|78.48|1648.08|87.32|618.24|706.32|215.82|647.46|784.04| +2451007|79244|5359|83626|1148028|6830|13279|72399|1013755|3350|27698|2|90|9|2|32|383|51|70.38|0.00|70.38|97.08|1236.75|14.07|42.79|13.52|1333.83| +2450843|48030|10814|12291|898839|6987|44735|368|1268168|525|40435|1|34|19|1|18|387|13|791.05|55.37|846.42|77.32|428.48|704.03|63.52|23.50|561.17| +2450999|48756|10729|12291|898839|6987|44735|35702|1263000|5310|26067|1|44|4|4|21|387|35|2378.95|23.78|2402.73|62.70|1631.35|1689.05|269.06|420.84|1717.83| +2450983|70782|3235|4992|1216416|2240|25311|35187|1841127|669|32211|1|26|17|4|1|388|3|107.82|8.62|116.44|93.24|107.82|81.94|24.06|1.82|209.68| +2450887|37824|17300|68316|276864|2659|4054|18935|968952|1555|45005|4|9|1|1|29|389|10|774.30|7.74|782.04|96.15|692.80|410.37|291.14|72.79|796.69| +2450863|85998|14347|39161|342553|345|26537|95897|1585135|5692|20202|4|100|6|1|3|392|56|324.24|25.93|350.17|41.48|504.56|213.99|101.43|8.82|571.97| +2451052|44684|2905|57937|97090|1396|46949|30815|500395|739|5530|2|108|9|5|24|393|4|168.88|5.06|173.94|7.68|101.32|136.79|13.15|18.94|114.06| +2450892|31575|6901|94515|508804|693|8982|12134|572314|321|16328|4|94|2|4|21|397|10|684.90|47.94|732.84|62.35|330.20|561.61|25.89|97.40|440.49| +2451059|9211|13082|94515|508804|693|8982|21370|197549|5286|48976|4|29|7|4|1|397|28|371.84|11.15|382.99|37.65|28.84|219.38|45.73|106.73|77.64| +2451014|45487|4226|33565|253452|3292|11179|72012|36468|229|29543|2|34|6|2|18|398|24|1384.56|13.84|1398.40|78.96|547.92|539.97|33.78|810.81|640.72| +2450868|64224|15940|33565|253452|3292|11179|11998|718313|6709|23152|2|91|14|2|20|398|30|4804.20|144.12|4948.32|27.38|2771.40|480.42|2205.12|2118.66|2942.90| +2450847|34429|10058|77033|688403|2173|11143|42132|1783016|2900|28031|2|21|10|4|13|400|2|291.38|17.48|308.86|60.29|0.00|116.55|71.68|103.15|77.77| +2450958|42507|15400|34528|1919328|1728|20928|70425|411265|4299|38180|4|83|9|3|4|401|7|183.26|10.99|194.25|79.66|398.44|148.44|24.02|10.80|489.09| +2451004|4891|16|34528|1919328|1728|20928|34528|1919328|7123|20928|4|19|18|2|4|401|39|461.37|18.45|479.82|22.43|269.10|129.18|119.58|212.61|309.98| +2450923|69743|11059||1919328|1728|||1879225|3599|4726||74||3||401||492.16|||36.19|||99.81||210.28| +2450949|83106|9410|93265|32325|2585|4845|26756|952636|4848|22260|1|78|16|3|20|402|4|162.68|11.38|174.06|22.62|73.36|24.40|33.18|105.10|107.36| +2450876|47081|17065|17284|862922|4113|35304|85216|689785|2498|24411|4|108|16|4|30|408|52|399.88|7.99|407.87|44.06|960.44|35.98|76.41|287.49|1012.49| +2450916|25218|4615|47965|1065872|2532|39|88957|810204|4498|31192|1|80|1|2|26|410|28|118.72|8.31|127.03|5.88|415.52|33.24|5.98|79.50|429.71| +2450876|52572|6622|35806|1598934|2509|29684|29401|1414905|107|5309|2|58|15|2|9|412|21|1987.44|158.99|2146.43|37.10|128.10|1848.31|86.26|52.87|324.19| +2451013|23008|16189|58863|1851147|6631|25668|15272|1677126|4592|10458|1|108|9|5|16|414|38|758.48|68.26|826.74|7.16|9.88|235.12|261.68|261.68|85.30| +2450933|67927|13466|23085|1266596|1754|30865|92024|1398274|3693|10312|1|65|15|4|21|415|7|226.10|18.08|244.18|90.47|85.96|33.91|3.84|188.35|194.51| +2451012|77742|6436|75716|165120|124|18681|4688|630043|6241|3639|1|20|13|1|29|418|29|2791.83|195.42|2987.25|93.83|1381.56|1228.40|547.20|1016.23|1670.81| +2450972|53585|3620|75716|165120|124|18681|75318|640820|1807|16747|1|9|8|5|32|418|23|3119.03|124.76|3243.79|66.87|1660.60|1653.08|73.29|1392.66|1852.23| +2450965|41150|9457|33562|1803936|1157|6331|28007|271786|3774|24915|2|96|5|3|14|419|39|3064.62|245.16|3309.78|17.18|2331.81|367.75|1537.21|1159.66|2594.15| +2451090|28605|10592|92253|911772|5338|44857|63597|1100874|6199|33571|1|95|13|3|15|420|9|308.52|27.76|336.28|89.96|201.69|277.66|22.83|8.03|319.41| +2450898|49882|12757|92253|911772|5338|44857|56020|1166850|6670|45690|1|103|10|5|20|420|55|5453.25|0.00|5453.25|81.60|2489.30|5126.05|271.57|55.63|2570.90| +2450977|49388|8617|9057|1646191|3772|7353|23921|1041382|938|1694|2|77|15|1|25|421|14|42.14|0.42|42.56|66.88|161.28|34.13|7.68|0.33|228.58| +2450933|69295|1198|9057|1646191|3772|7353|4057|1896815|5357|32699|2|86|11|3|30|421|45|941.85|65.92|1007.77|43.75|546.30|828.82|82.51|30.52|655.97| +2450871|51855|14576||1183003|1121|||1695891|6467|||91|18|4|30|422|8|191.76|||38.11|191.76|67.11|118.41|6.24|229.87| +2450968|52480|10981|79638|1183003|1121|44886|87526|1492320|2834|6948|4|30|3|2|9|422|36|964.44|19.28|983.72|3.35|137.52|125.37|511.83|327.24|160.15| +2450921|69715|9121|2132|1631362|6545|8128|67542|1734917|2086|29608|1|90|8|2|34|430|82|3268.52|294.16|3562.68|91.77|847.06|2516.76|488.64|263.12|1232.99| +2450997|39289|3085|78582|1866571|5760|48549|93951|765817|1723|34829|1|31|14|5|11|431|34|2108.68|42.17|2150.85|55.36|1581.68|738.03|1055.40|315.25|1679.21| +2451002|50835|13495|78582|1866571|5760|48549|73165|1361165|2679|20546|1|88|1|3|12|431|1|30.01|0.90|30.91|50.29|10.42|17.40|8.82|3.79|61.61| +2450923|34659|10924|52340|147438|2889|22740|74186|477385|3554|26123|2|59|10|4|13|436|17|927.35|46.36|973.71|77.22|302.77|398.76|417.58|111.01|426.35| +2450891|28696|3922|74490|1259629|768|1507|46428|1407978|3862|48546|1|20|5|4|35|437|10|575.50|28.77|604.27|71.02|121.10|529.46|10.12|35.92|220.89| +2450976|56829|13180|74490|1259629|768|1507|5308|1859445|5615|49832|1|38|16|3|18|437|58|3413.30|0.00|3413.30|27.61|3413.30|989.85|1793.35|630.10|3440.91| +2450905|47794|5032|55796|54720|6091|3015|29654|409934|5476|39418|1|85|11|5|2|439|40|3409.60|102.28|3511.88|80.50|528.00|716.01|565.65|2127.94|710.78| +2450947|47527|15554|55796|54720|6091|3015|55796|54720|1224|3015|1|85|2|5|1|439|2|20.46|1.63|22.09|54.77|1.70|19.64|0.65|0.17|58.10| +2450891|57176|11686|49674|420206|5938|26423|98743|1652314|192|28470|4|79|4|2|20|441|20|2173.00|108.65|2281.65|48.96|185.40|1868.78|12.16|292.06|343.01| +2450863|919|5852|61705|1379797|6689|48028|83806|915904|2399|14894|1|67|19|4|24|442|1|36.98|1.10|38.08|9.32|51.12|23.66|5.59|7.73|61.54| +2451017|72429|14977|61705|1379797|6689|48028|13812|1059619|3379|22739|1|46|6|1|16|442|27|3020.22|120.80|3141.02|57.79|177.66|2990.01|14.50|15.71|356.25| +2450939|27728|17575|61705|1379797|6689|48028|95595|1120311|3523|35535|1|33|11|3|10|442|3|82.23|2.46|84.69|56.58|30.84|20.55|19.73|41.95|89.88| +2450864|82172|15007|61705|1379797|6689|48028|2725|1913194|1679|42917|1|4|18|1|6|442|35|166.25|4.98|171.23|71.62|51.45|166.25|0.00|0.00|128.05| +2450989|61281|12266|61705|1379797|6689|48028|70293|881022|1300|39178|1|32|12|5|23|442|4|40.44|1.21|41.65|82.02|53.36|5.25|28.15|7.04|136.59| +2451044|60942|14203|89254|1469988|4322|25103|60054|352712|1924|13983|1|28|6|5|27|444|34|6806.12|204.18|7010.30|31.70|3130.72|2177.95|3378.56|1249.61|3366.60| +2450987|59803|3217||196410|||28149|530116|4824|||33|16||5|447|||||54.76|17.94|586.10|||| +2451063|9355|12170|42337|1020263||10562||1020263|1539|10562|||18||11|448|||0.85|||126.30|69.90|9.05||215.94| +2451035|83121|5156|88458|368476|2094|26159|39321|741810|31|11452|1|22|2|3|14|451|44|4820.64|0.00|4820.64|21.21|1168.64|1639.01|572.69|2608.94|1189.85| +2450866|35838|12212|17394|635181|5321|31108|76648|7845|881|35517|4|105|18|4|25|452|9|245.79|17.20|262.99|97.75|35.10|196.63|12.78|36.38|150.05| +2450980|44083|10238|41009|692127|5198|25116|32726|1022384|3211|33191|2|87|5|2|17|461|60|7551.00|226.53|7777.53|8.41|3547.80|755.10|4077.54|2718.36|3782.74| +2450897|73405|3007|41009|692127|5198|25116|54906|1331018|5693|17921|2|100|19|4|29|461|15|2376.00|71.28|2447.28|45.47|58.65|879.12|1287.31|209.57|175.40| +2450912|51135|238|41009|692127|5198|25116|19803|550731|578|24825|2|83|6|3|20|461|74|3090.98|30.90|3121.88|46.08|4415.58|463.64|2548.51|78.83|4492.56| +2450886|38053|17252|21479|829253|3427|19648|21479|829253|4973|19648|4|58|20|2|22|463|16|220.96|2.20|223.16|48.55|105.12|53.03|154.49|13.44|155.87| +2450955|63117|13945|70912|1654773|587|13648|18867|967425|3024|30270|4|103|7|1|27|465|12|434.88|21.74|456.62|19.17|579.84|230.48|130.81|73.59|620.75| +2451069|51350|8593|24924|627193|3862|5331|76256|1471823|3136|4849|2|49|15|3|11|466|19|329.84|23.08|352.92|90.76|879.70|323.24|5.54|1.06|993.54| +2451058|6514|17386|60178|1093007|589|30618|47143|1777434|4978|43322|1|76|1|5|9|470|19|1909.69|0.00|1909.69|26.89|514.14|95.48|1814.21|0.00|541.03| +2450901|72843|2788|2767|743612|904|13796|67850|341863|1568|19273|1|9|8|3|18|474|1|6.90|0.27|7.17|88.93|22.10|2.13|0.66|4.11|111.30| +2451055|70023|10570|70478|1556959|687|2015|20091|1682712|5733|3154|1|84|17|5|11|475|1|5.02|0.20|5.22|99.92|2.19|4.81|0.16|0.05|102.31| +2450975|61685|5858|21537|674128|6272|35663|58994|393220|3438|48409|2|17|16|2|20|478|61|4822.66|337.58|5160.24|27.80|1583.56|144.67|1075.93|3602.06|1948.94| +2451025|4879|3170|64897|1706749|2201|37653|61930|1469273|6307|43388|4|75|11|5|6|481|55|4076.60|326.12|4402.72|93.84|1132.45|2731.32|1264.56|80.72|1552.41| +2450861|46382|7688|64897|1706749|2201|37653|19160|1893379|3344|38062|4|5|10|3|23|481|11|671.88|0.00|671.88|55.23|116.82|215.00|438.60|18.28|172.05| +2450982|72986|12925|4829|414213|4397|26981|78720|1860876|7092|42061|4|71|15|1|20|482|57|5401.89|324.11|5726.00|51.41|3629.19|3781.32|599.61|1020.96|4004.71| +2450938|41266|4753|4829|414213|4397|26981|3717|1454664|5233|22249|4|42|16|5|9|482|3|0.00|0.00|0.00|43.23|87.00|0.00|0.00|0.00|130.23| +2451073|60367|8270|46125|255298|1271|41197|46125|255298|4367|41197|1|62|11|4|30|483|8|52.40|4.71|57.11|78.59|78.64|47.16|3.56|1.68|161.94| +2450891|54314|10|46125|255298|1271|41197|62711|1703610|709|43008|1|92|5|4|13|483|39|1604.46|16.04|1620.50|12.22|641.55|978.72|137.66|488.08|669.81| +2450918|42361|5686|68773|739779|6785|24591|43644|473541|5461|3381|1|93|1|1|13|485|17|247.01|19.76|266.77|68.65|213.35|4.94|174.29|67.78|301.76| +2450983|75518|15829|51127|337141|4308|18275|40851|217259|121|23736|2|55|2|2|19|490|2|54.54|0.00|54.54|92.18|9.68|13.63|21.27|19.64|101.86| +2451017|43407|6002|18473|466006|4739|37119|55632|182580|6246|19912|4|84|10|1|27|491|2|324.80|6.49|331.29|46.13|29.18|178.64|97.92|48.24|81.80| +2450970|32640|16522|88978|1854110|5995|22727|55501|1357269|1923|40130|2|24|11|3|27|493|23|1184.50|23.69|1208.19|55.34|0.00|118.45|426.42|639.63|79.03| +2450974|64628|8797|83228|162271|6561|35604|4462|1339529|4934|5986|2|20|15|2|27|494|47|4836.77|48.36|4885.13|48.72|302.21|4740.03|54.17|42.57|399.29| +2451026|61215|2227|2563|231129|2095|39861|2563|231129|7092|39861|2|77|2|1|5|495|4|22.36|0.00|22.36|95.32|73.84|9.16|11.88|1.32|169.16| +2450891|23765|13276|39102|1089078|7054|20230|3389|42565|5587|29809|2|78|8|5|32|496|43|3489.88|279.19|3769.07|57.06|669.08|2477.81|455.43|556.64|1005.33| +2450947|35914|14410|39102|1089078|7054|20230|52203|774484|950|27016|2|8|8|4|20|496|36|91.08|1.82|92.90|4.00|245.16|7.28|19.27|64.53|250.98| +2450906|64640|4418|95466|391930|6441|47752|96763|1801004|3358|32112|1|26|14|3|2|497|3|213.18|8.52|221.70|12.60|124.02|102.32|4.43|106.43|145.14| +2450871|30058|14185|44990|90870|5103|16583|26671|899162|2289|36169|1|77|3|5|17|498|3|683.46|20.50|703.96|54.50|15.69|82.01|402.97|198.48|90.69| +2451033|21864|12830|44990|90870|5103|16583|87362|1849310|5605|6347|1|19|12|2|2|498|24|1516.56|15.16|1531.72|74.18|900.48|272.98|248.71|994.87|989.82| +2450893|58376|13681||90870||16583||439528|699||1||16|5||498||12.64|1.01|13.65|68.45||9.60|0.33|2.71|| +2451058|39660|14116|52750|427630|2157|27484|63354|959091|4382|49226|1|102|12|3|32|503|21|500.64|0.00|500.64|91.12|1656.06|415.53|11.91|73.20|1747.18| +2450903|69069|14644|23717|115786|6702|2818|40589|1496797|1736|45028|4|77|18|2|19|505|5|414.20|8.28|422.48|92.01|119.05|265.08|20.87|128.25|219.34| +2450984|56451|10081|23717|115786|6702|2818|23717|115786|2284|2818|4|80|20|4|33|505|40|304.00|15.20|319.20|45.82|372.80|12.16|288.92|2.92|433.82| +2451079|69689|16382|27049|294741|1633|33325|37427|1672763|4934|19505|4|9|18|4|15|506|2|94.58|3.78|98.36|37.53|121.18|7.56|19.14|67.88|162.49| +2451013|84145|15332|91195|1162616|3990|43811|51467|612052|757|25062|1|86|3|3|11|507|28|266.56|21.32|287.88|51.04|199.92|138.61|62.69|65.26|272.28| +2450946|34778|5948|53653|1906982|2711|29640|78855|1856253|6576|48099|2|57|1|4|27|508|19|434.53|30.41|464.94|19.62|806.93|312.86|99.76|21.91|856.96| +2450896|62526|17356|532|1491942|2199|37656|83797|213501|1260|16219|2|13|3|2|3|510|64|8609.92|688.79|9298.71|29.70|3904.64|6887.93|223.85|1498.14|4623.13| +2451008|36496|6616|79146|911227|2508|23836|14731|647376|311|20493|1|25|5|5|25|512|1|179.48|1.79|181.27|19.68|40.33|14.35|146.96|18.17|61.80| +2451005|51878|5738|30858|1495803|7148|42540|43905|251030|770|34863|2|83|9|2|5|513|36|15.12|0.30|15.42|49.73|50.40|0.30|6.07|8.75|100.43| +2450987|79177|7604|30858|1495803|7148|42540|94450|502006|3369|36332|2|9|14|5|13|513|14|1900.64|152.05|2052.69|5.98|618.24|988.33|72.98|839.33|776.27| +2450912|15702|4837|30858|1495803|7148|42540|60389|637201|6958|49515|2|91|13|5|27|513|2|106.52|1.06|107.58|11.46|136.94|69.23|21.62|15.67|149.46| +2450870|58643|11540|91408|1548837|4666|495|59919|674995|4359|28923|1|30|15|4|13|514|27|517.32|41.38|558.70|55.24|191.43|372.47|5.79|139.06|288.05| +2450935|31408|4027|91408|1548837|4666|495|20457|140458|2348|4638|1|53|8|5|25|514|10|230.40|11.52|241.92|36.05|134.70|221.18|2.67|6.55|182.27| +2450967|60305|17378|55929|1899717|658|44046|97776|1151482|1443|3804|2|12|14|5|11|519|29|3175.79|222.30|3398.09|51.66|1928.21|3017.00|1.58|157.21|2202.17| +2450903|47248|4279|39466|495142|6018|3294|37084|979215|7051|6087|2|8|2|5|1|521|24|1315.92|78.95|1394.87|17.04|375.84|289.50|61.58|964.84|471.83| +2451012|68118|170|39466|495142|6018|3294|89212|1123607|4708|2609|2|20|5|1|22|521|2|51.02|0.51|51.53|82.42|74.22|4.59|40.39|6.04|157.15| +2451087|46711|1226|39551|1686167|2830|33846|61797|295579|1101|38576|4|94|8|5|35|522|46|7099.18|283.96|7383.14|18.87|3301.88|6744.22|53.24|301.72|3604.71| +2451066|52316|7100|39551|1686167|2830|33846|36208|350264|6284|21551|4|53|12|4|24|522|5|158.90|4.76|163.66|76.99|34.95|57.20|42.71|58.99|116.70| +2450874|77533|12898|46648|1167714|6380|693|52468|836110|6573|49789|1|75|11|2|20|525|2|134.58|4.03|138.61|7.10|33.18|10.76|12.38|111.44|44.31| +2450888|42163|4135|78368|738771|3174|47977|32322|824979|6615|37498|1|101|7|5|28|526|1|66.09|5.28|71.37|70.35|17.79|34.36|5.39|26.34|93.42| +2450981|70993|13484|38224|1912875|5126|20577|37154|1067397|2483|9922|2|37|13|4|13|527|11|180.84|5.42|186.26|53.93|215.82|168.18|2.91|9.75|275.17| +2450957|52700|12758|42886|1066140|4594|33048|90479|78482|6414|15546|1|33|15|2|4|529|79|899.81|8.99|908.80|69.22|423.44|809.82|52.19|37.80|501.65| +2450935|57645|10669|42886|1066140|4594|33048|12231|1723840|464|11088|1|81|13|5|4|529|7|142.52|5.70|148.22|30.56|94.99|132.54|2.39|7.59|131.25| +2450996|74493|6980|82410|116099|7035|46724|91142|885222|5491|24913|1|24|13|5|34|531|2|5.94|0.17|6.11|6.34|2.44|3.56|1.33|1.05|8.95| +2450912|35729|12514|51008|1476051|5894|28184|93734|1381361|5750|28939|1|61|13|2|31|532|10|758.50|45.51|804.01|61.77|409.10|743.33|8.95|6.22|516.38| +2450989|51363|4873|51008|1476051|5894|28184|77595|1758099|1254|48809|1|55|14|4|14|532|9|390.42|27.32|417.74|34.98|9.72|97.60|143.48|149.34|72.02| +2451066|68617|7153|45391|496784|1777|14723|58802|573138|6746|24801|4|74|12|2|30|534|16|238.24|16.67|254.91|21.69|1497.60|76.23|77.76|84.25|1535.96| +2451031|74209|13826|60396|1877215|3634|48453|76652|937478|4529|14427|2|46|7|5|6|536|2|3.70|0.07|3.77|5.23|10.04|2.62|0.28|0.80|15.34| +2450971|59224|15568|33965|754099|633|37967|86650|1027313|2061|20409|4|89|20|3|1|538|9|42.12|1.68|43.80|76.78|326.61|30.74|0.22|11.16|405.07| +2450973|49280|10540|13844|1511052|6307|1115|51990|1827394|6270|40346|4|88|14|2|3|539|8|350.96|28.07|379.03|0.81|274.16|122.83|180.22|47.91|303.04| +2450901|11267|7118||||38063|36588|||||75|3|2||541||||371.78|80.34|846.90||||930.92| +2450959|16791|1972|30023|1718154|1132|38063|48897|1466555|5563|2171|1|23|2|5|15|541|17|433.33|4.33|437.66|97.66|26.52|21.66|98.80|312.87|128.51| +2451014|34963|10076|||658|42780|||||||15|||543|42|1651.02|115.57||||||373.47|375.79| +2451038|74059|15553|59023|885359|4942|20478|2849|836002|5477|10999|4|22|5|2|19|544|23|141.91|0.00|141.91|1.20|2318.17|127.71|5.11|9.09|2319.37| +2451027|67454|14768|86307|815605|2503|16954|84206|1686191|4072|23953|2|44|16|2|3|545|19|1494.16|119.53|1613.69|90.25|235.03|1150.50|178.70|164.96|444.81| +2450885|51094|11056|86307|815605|2503|16954|74140|1029489|1691|48293|2|88|19|4|26|545|3|93.48|2.80|96.28|36.64|42.96|80.39|5.89|7.20|82.40| +2450915|55836|8816|99961|1095242|4923|3804|96709|330991|2985|979|4|13|14|3|2|547|10|49.20|0.00|49.20|3.88|196.90|1.47|27.20|20.53|200.78| +2451015|74146|9356|99961|1095242|4923|3804|22000|831212|5359|18259|4|84|6|1|6|547|1|12.04|0.72|12.76|10.43|50.19|11.43|0.32|0.29|61.34| +2450901|58985|7006|19600|1462369|7185|45954|4766|1361661|6469|6430|2|34|20|1|24|548|48|2094.24|104.71|2198.95|36.41|837.60|83.76|1990.37|20.11|978.72| +2450947|16626|10801|3738|116116|1741|7319|28216|1811584|314|11044|1|89|14|4|6|549|3|100.35|7.02|107.37|99.88|9.39|17.05|62.47|20.83|116.29| +2450881|25901|9949|36026|1435872|5765|2011|98709|83461|5530|38399|4|78|13|2|1|551|7|194.39|15.55|209.94|24.98|67.55|56.37|99.37|38.65|108.08| +2450871|21490|5546|56373|1378326|726|13926|41212|1273370|878|35186|1|47|16|4|25|552|49|5922.14|0.00|5922.14|17.27|2402.96|3020.29|1073.68|1828.17|2420.23| +2450945|46723|8450|67566|1577176|1539|19102|55260|386313|6711|36756|2|93|4|2|22|553|50|2880.00|28.80|2908.80|24.96|960.00|835.20|286.27|1758.53|1013.76| +2450910|60203|14504|76140|602470|2400|46730|50643|1225156|572|44388|4|106|9|3|14|554|23|1928.78|115.72|2044.50|10.30|1195.31|1408.00|187.48|333.30|1321.33| +2450988|71555|6620|76140|602470|2400|46730|7371|1851551|299|31447|4|90|11|4|2|554|22|23.76|2.13|25.89|69.69|103.18|8.31|5.56|9.89|175.00| +2450911|56298|14698|48861|1670483|458|14480|33221|604721|6387|14406|2|57|17|1|2|555|41|232.06|0.00|232.06|46.59|224.27|25.52|179.68|26.86|270.86| +2451018|79072|11752|90789|201767|5545|19370|44462|56525|176|4580|4|45|6|2|11|559|45|7303.95|0.00|7303.95|65.34|1275.30|2994.61|2413.23|1896.11|1340.64| +2450993|55410|14438|25250|436779|308|13837|50443|607348|4958|22615|1|83|6|3|20|560|46|551.08|11.02|562.10|4.89|3386.98|479.43|35.10|36.55|3402.89| +2450934|75987|7112|23156|693776|1196|7816|23156|693776|4181|7816|1|100|6|4|20|563|30|4428.00|132.84|4560.84|91.09|89.40|132.84|2233.48|2061.68|313.33| +2450941|21609|15730|17662|372808|401|24747|77471|1383687|3251|10815|2|80|14|3|12|564|43|3236.18|32.36|3268.54|71.90|2376.61|2944.92|163.10|128.16|2480.87| +2450986|73640|9278|11072|732087|4655|25270|68226|1100902|5199|28296|4|6|7|4|26|568|65|4844.45|48.44|4892.89|18.82|1428.05|2809.78|1871.89|162.78|1495.31| +2451026|54147|368|57245|1387816|5587|25358|51371|1423194|6957|27920|4|77|16|4|19|570|25|2230.00|178.40|2408.40|63.00|1517.50|914.30|1157.81|157.89|1758.90| +2450923|35546|4072|57245|1387816|5587|25358|51957|793107|6767|41627|4|52|20|5|20|570|42|1675.80|67.03|1742.83|67.05|1634.22|418.95|1018.04|238.81|1768.30| +2450963|57219|17533|8133|1436981|7076|4371|46434|1539624|4656|13688|1|75|4|1|23|571|14|684.74|61.62|746.36|43.61|530.18|568.33|103.60|12.81|635.41| +2451047|71747|10520|31681|36270|6812|19754|77615|293735|4051|39614|1|89|9|5|31|572|1|24.51|0.73|25.24|30.28|0.00|14.46|5.52|4.53|31.01| +2451050|74999|12877|31681|36270|6812|19754|90490|538684|581|8078|1|106|9|1|13|572|13|124.54|7.47|132.01|1.84|384.15|46.07|45.51|32.96|393.46| +2450983|45877|8408|9545|882645|6498|40398|12833|682311|7073|25082|4|9|2|1|18|573|33|4724.28|330.69|5054.97|47.13|2113.32|2125.92|2078.68|519.68|2491.14| +2450858|48430|6388|94041|1458793|792|7144|88004|135267|5609|18304|2|85|3|3|10|575|1|10.19|0.00|10.19|49.39|40.78|7.84|0.82|1.53|90.17| +2450933|40341|7172|86548|1590578|5808|41085|34225|812520|2174|45475|2|62|4|5|19|579|2|23.60|0.94|24.54|17.90|58.08|23.36|0.21|0.03|76.92| +2450919|44418|1693|99155|850445|935|28225|81324|646464|3751|35885|2|81|20|2|11|580|79|3611.88|72.23|3684.11|61.86|419.49|3178.45|416.09|17.34|553.58| +2450958|13909|12169|90408|268724|4640|13309|97887|171524|632|34587|4|78|5|5|15|581|8|938.08|28.14|966.22|45.35|299.36|741.08|187.15|9.85|372.85| +2450997|66508|17612|87959|1722125|5891|38457|3878|1448311|2604|24497|2|66|9|2|23|583|33|1118.70|0.00|1118.70|16.11|796.29|626.47|359.32|132.91|812.40| +2450932|32808|7790|23554|694859|117|1422|95506|1475258|679|22853|1|60|9|5|19|584|66|7825.62|547.79|8373.41|53.59|1304.16|4695.37|93.90|3036.35|1905.54| +2450998|26663|17449|23554|694859|117|1422|87868|807843|177|41311|1|4|20|4|23|584|72|47.52|0.47|47.99|69.83|1094.40|23.28|4.84|19.40|1164.70| +2450946|80941|9211|10077|733640|4403|43166|7874|815865|1080|25895|4|80|20|5|15|585|16|218.72|2.18|220.90|24.80|1375.20|142.16|11.48|65.08|1402.18| +2450885|41182|11786|58659|1436296|333|4770|58659|1436296|6722|4770|1|17|8|3|31|586|11|220.33|6.60|226.93|51.02|220.33|110.16|15.42|94.75|277.95| +2451076|46288|8407|36969|1843719|2469|22772|68592|1614912|6137|24609|4|31|18|5|30|587|11|1168.86|58.44|1227.30|60.76|637.56|701.31|369.36|98.19|756.76| +2450999|67913|15271|59348|814917|2486|23255|47019|633778|3840|15502|1|14|8|5|12|588|25|271.50|16.29|287.79|18.85|165.25|198.19|0.73|72.58|200.39| +2450994|32845|11647|74062|1199853|6091|25529|46186|367494|1653|7365|4|16|2|1|35|590|90|15059.70|1054.17|16113.87|86.29|5722.20|7529.85|2033.05|5496.80|6862.66| +2451037|70182|1801|16687|996663|3039|27015|17256|1252804|2883|29362|1|58|12|1|20|591|18|165.06|14.85|179.91|51.54|454.14|28.06|105.49|31.51|520.53| +2451031|72176|11012|51032|1782756|5680|11404|53174|1017112|6756|22400|2|7|19|5|14|592|16|39.04|3.51|42.55|5.72|195.84|23.42|5.77|9.85|205.07| +2450867|42784|8737|51070|1733058|1893|12681|15997|599141|3442|21743|2|5|6|2|5|594|15|852.45|0.00|852.45|82.41|0.00|306.88|169.12|376.45|82.41| +2450900|71526|1513|85806|501933|2860|27387|85806|501933|6117|27387|4|34|11|1|3|596|52|766.48|7.66|774.14|71.38|527.80|712.82|33.26|20.40|606.84| +2450966|68663|14846|23265|703401|6384|39767||||15059|2|||5|21|598|4|||185.76||117.84|69.20||103.89|| +2450904|68632|7958|1747|399434|3074|17961|74224|1698316|5717|5118|4|60|1|4|5|599|19|711.74|28.46|740.20|2.59|271.13|156.58|210.96|344.20|302.18| +2450994|68615|16258|||3867|13200|||||4||||27|600|41|||0.00|||0.00|0.00||361.00| +2451086|75295|13934|20384|239781|4778|27775|82611|1322145|3366|26634|4|15|11|4|32|601|21|164.85|3.29|168.14|36.36|47.88|121.98|3.42|39.45|87.53| +2451047|49775|2354|37748|1812536|3677|865|74090|745349|2340|16178|1|58|11|2|18|602|22|5134.14|256.70|5390.84|99.57|1410.42|3183.16|643.82|1307.16|1766.69| +2450946|32476|6049|51154|1210607|813|41866|23594|1386945|3148|25351|2|56|20|3|21|603|23|1249.82|99.98|1349.80|82.65|83.26|649.90|497.93|101.99|265.89| +2451016|25421|17774|62343|1602550|804|11058|29563|1860821|286|13704|1|55|12|5|15|605|9|657.27|32.86|690.13|22.86|200.34|565.25|28.52|63.50|256.06| +2451044|5101|295|98459|3822|1632|36995|98459|3822|3226|36995|4|36|14|1|23|606|33|2349.27|46.98|2396.25|59.59|1315.71|281.91|165.38|1901.98|1422.28| +2450931|75704|3680|98459|3822|1632|36995|98459|3822|6410|36995|4|102|12|1|10|606|45|682.65|40.95|723.60|55.95|272.70|505.16|115.36|62.13|369.60| +2450904|6443|2500|98751|1126621|6044|35467|28279|682903|6830|16357|2|6|18|5|32|607|14|92.68|3.70|96.38|69.94|85.82|7.41|69.92|15.35|159.46| +2450874|28039|14272|8462|758983|3904|13272|3865|683419|4720|19974|1|68|17|1|7|609|56|0.00|0.00|0.00|51.54|462.00|0.00|0.00|0.00|513.54| +2451063|52976|15148|34007|661676|2498|32967|52598|1234619|3376|39733|1|89|11|3|17|611|31|82.15|3.28|85.43|51.50|14.57|36.96|16.26|28.93|69.35| +2450960|59423|2156|27313|876564|4568|36172|1365|306047|4733|44219|4|26|17|3|14|614|10|30.70|2.14|32.84|45.85|446.20|24.86|1.92|3.92|494.19| +2450998|66269|15976|88733|202813|5693|35373|94594|910438|5645|22099|1|64|1|3|2|615|3|34.65|3.11|37.76|44.77|0.00|31.87|2.64|0.14|47.88| +2450908|73695|5941|88733|202813|5693|35373|49973|610180|4092|31957|1|42|12|2|34|615|1|33.32|1.99|35.31|58.14|4.21|4.99|9.91|18.42|64.34| +2450888|69246|17761|88733|202813|5693|35373|70211|206234|824|45061|1|26|11|3|23|615|9|2102.04|105.10|2207.14|70.32|281.97|1282.24|508.27|311.53|457.39| +2451020|29265|14578|88733|202813|5693|35373|70956|1767751|1474|21550|1|88|19|2|5|615|4|250.28|5.00|255.28|31.17|135.28|147.66|56.44|46.18|171.45| +2450990|73905|10789|54405|1596524|243|24762|3142|1070269|1176|6436|2|54|17|4|31|617|20|1026.00|82.08|1108.08|69.03|513.00|943.92|27.08|55.00|664.11| +2450992|62219|16954|54405|1596524|243|24762|42699|1225258|637|30816|2|100|7|1|15|617|15|590.40|5.90|596.30|72.92|257.25|495.93|83.13|11.34|336.07| +2451015|31284|3160|25474|417052|4983|43314|37865|1237732|3734|33783|1|98|3|2|24|618|7|286.09|5.72|291.81|12.12|43.96|123.01|102.74|60.34|61.80| +2451004|37418|11714|58932|246261|5143|34072|9584|1913299|2655|29258|1|61|20|3|1|621|2|182.48|10.94|193.42|30.73|136.02|67.51|41.38|73.59|177.69| +2450913|43045|5560|8200|706755|3710|31465|76438|367304|6426|27101|1|26|7|4|19|622|17|154.19|9.25|163.44|95.78|101.15|78.63|3.77|71.79|206.18| +2451040|40216|15979|5470|1742209|3395|10934|47084|714489|4413|37490|2|86|8|2|9|623|11|274.34|0.00|274.34|52.25|47.30|123.45|27.16|123.73|99.55| +2451048|50994|7642|76576|1594628|2727|34779|73875|944824|3358|47939|2|24|2|1|32|624|31|295.74|2.95|298.69|30.19|76.26|198.14|74.17|23.43|109.40| +2450950|35055|6844|14634|1260589|1391|24546|14634|1260589|1716|24546|1|100|20|3|19|625|10|929.00|9.29|938.29|1.58|33.70|473.79|250.36|204.85|44.57| +2450887|44318|6115|84586|870126|2066|27606|93238|511386|7142|24898|2|96|19|5|21|626|77|5511.66|440.93|5952.59|82.74|363.44|3472.34|1896.56|142.76|887.11| +2450973|42068|17137|84586|870126|2066|27606|1922|1068589|3111|28833|2|3|6|3|5|626|10|269.50|10.78|280.28|24.64|0.00|123.97|120.78|24.75|35.42| +2450938|73493|13078|84586|870126|2066|27606|76282|1684117|1620|47123|2|47|19|2|30|626|3|235.59|7.06|242.65|22.26|80.97|47.11|98.00|90.48|110.29| +2451071|57705|8692|48589|1604968|5700|49632|44458|778211|4135|14906|1|48|5|4|23|630|2|34.14|2.04|36.18|9.90|7.80|11.94|5.55|16.65|19.74| +2450895|60912|1978|48589|1604968|5700|49632|45888|1204967|5681|38842|1|85|18|4|20|630|29|691.94|6.91|698.85|48.13|172.84|325.21|3.66|363.07|227.88| +2450967|44742|11372|70320|1060015|4463|958|61290|1439451|415|37379|2|75|11|5|34|631|14|1805.16|54.15|1859.31|47.65|667.10|848.42|86.10|870.64|768.90| +2451088|15385|3334|97133|660711|642|18620|66247|1099278|5692|36106|4|94|1|5|4|632|13|1793.22|125.52|1918.74|36.51|35.75|950.40|345.55|497.27|197.78| +2450997|34953|11932|49438|192152|819|27133|40035|714381|6819|36457|4|1|4|2|8|635|11|108.79|5.43|114.22|50.31|166.87|32.63|51.78|24.38|222.61| +2450943|25603|11527|49438|192152|819|27133|49438|192152|4119|27133|4|76|18|1|6|635|8|599.20|23.96|623.16|90.81|153.84|233.68|237.58|127.94|268.61| +2451053|35114|1496|49438|192152|819|27133|41483|1657828|1094|44313|4|36|17|1|7|635|22|1292.50|0.00|1292.50|52.09|69.74|517.00|434.28|341.22|121.83| +2450910|48505|170|||||60473|485533||39283||||1||637|19||54.59||99.17|||108.92|409.77|1273.81| +2450940|39923|3175|40057|1627957|3857|32557|12930|618572|6455|13138|1|6|14|2|25|638|15|279.60|2.79|282.39|2.71|1677.75|232.06|34.70|12.84|1683.25| +2450913|65075|124|40057|1627957|3857|32557|16191|1646410|1880|9397|1|63|14|4|6|638|10|1296.70|77.80|1374.50|4.30|375.30|1076.26|105.81|114.63|457.40| +2450958|70721|17180|14887|317894|6501|21508|58596|279825|4704|37630|4|6|14|3|24|640|1|19.04|0.38|19.42|51.43|85.71|15.23|2.70|1.11|137.52| +2451020|7896|5360|10532|717113|5647|7381|91241|332547|3957|30167|4|36|17|2|34|641|82|4302.54|129.07|4431.61|17.71|3346.42|2710.60|1496.42|95.52|3493.20| +2450999|70570|3268|10532|717113|5647|7381|50241|822269|3639|22256|4|31|13|3|10|641|32|2203.84|44.07|2247.91|94.11|1469.12|947.65|163.30|1092.89|1607.30| +2450975|76263|7645|45006|688700|7147|44441|30357|354121|3045|10322|4|50|13|2|14|644|31|1175.83|0.00|1175.83|43.81|1359.66|317.47|412.01|446.35|1403.47| +2451019|43768|17491|69066|1390255|5044|16986|94100|1011253|297|12494|2|34|6|5|4|645|47|3536.28|0.00|3536.28|67.33|2613.67|2227.85|798.14|510.29|2681.00| +2451010|60946|6466|70095|60237|779|3321|70095|60237|4797|3321|4|69|19|3|32|647|8|536.56|21.46|558.02|8.68|34.24|21.46|51.51|463.59|64.38| +2450949|44933|2528|68050|591716|5782|39001|69902|794651|5504|40357|4|55|9|2|10|648|1|18.91|0.94|19.85|50.60|37.83|5.67|0.39|12.85|89.37| +2450858|68836|9404|17546|226874|5355|3483|48755|830451|4781|19911|1|68|8|3|28|650|1|0.00|0.00|0.00|31.41|7.88|0.00|0.00|0.00|39.29| +2450904|27443|16804|71331|1343465|3599|27733|60863|1905890|1356|17975|1|58|6|2|33|652|3|174.24|8.71|182.95|26.69|88.77|10.45|54.05|109.74|124.17| +2450976|74648|14738|71331|1343465|3599|27733|50446|1335642|3292|11295|1|81|18|1|11|652|50|954.00|47.70|1001.70|77.49|1319.00|877.68|48.08|28.24|1444.19| +2450921|79835|4706|38011|392726|5841|36956|36507|1352569|3981|48640|2|83|10|4|13|653|3|612.03|55.08|667.11|85.62|253.47|159.12|199.28|253.63|394.17| +2450881|38904|9307|1261|546514|5814|12667|29289|1169283|5542|19001|4|9|11|5|9|654|5|18.65|1.49|20.14|13.43|133.65|0.55|6.87|11.23|148.57| +2451031|34212|16664|42448|747837|3273|2662|22426|487984|6720|26703|1|101|18|1|17|655|7|1182.30|0.00|1182.30|54.02|48.72|874.90|144.47|162.93|102.74| +2451017|30580|13759|78392|128253|2114|21128|59113|134701|236|23818|4|82|10|5|30|656|8|6.48|0.00|6.48|38.91|5.84|0.97|1.21|4.30|44.75| +2451003|75727|11186|10741|1224930|1166|11802|36606|510168|2691|23214|1|45|3|3|16|657|31|495.69|19.82|515.51|74.33|671.46|54.52|269.11|172.06|765.61| +2451026|48699|2323|15291|19699|2507|5362|41691|1026128|4836|46344|2|54|15|3|6|659|31|813.75|16.27|830.02|12.28|705.25|179.02|253.89|380.84|733.80| +2450959|55965|15482|17972|175320|3515|32910|86185|179300|3089|14043|1|64|15|3|8|661|22|3202.54|192.15|3394.69|13.76|555.50|1985.57|730.18|486.79|761.41| +2450894|43397|2540|2366|1509504|2689|13027|92053|678187|2617|19000|4|61|3|3|31|662|77|1572.34|62.89|1635.23|50.11|3386.46|974.85|555.66|41.83|3499.46| +2450940|62338|8660|2366|1509504|2689|13027|82851|394495|330|7365|4|9|10|5|10|662|81|1182.60|82.78|1265.38|77.44|295.65|567.64|282.88|332.08|455.87| +2450929|43769|7429|87150|1839393|1236|22279|83157|194858|1508|39358|1|8|15|2|22|663|2|76.08|6.08|82.16|7.72|2.92|43.36|19.63|13.09|16.72| +2451003|57955|12530|87150|1839393|1236|22279|75321|407582|3102|20175|1|3|15|3|33|663|11|540.98|48.68|589.66|35.62|247.39|135.24|263.73|142.01|331.69| +2450958|6136|17098||986318|2361|26004|59799||4491||||20||28|665|53||0.00||46.29|||||| +2450923|41907|12604|86845|1555917|6236|41755|86845|1555917|237|41755|2|34|3|3|31|666|41|2026.22|162.09|2188.31|51.05|1608.02|40.52|436.85|1548.85|1821.16| +2450957|30915|13129|86845|1555917|6236|41755|86845|1555917|6250|41755|2|23|10|3|13|666|3|209.13|18.82|227.95|56.23|68.13|106.65|94.28|8.20|143.18| +2451006|63543|5632|86845|1555917|6236|41755|80173|1348131|6802|1920|2|29|20|1|4|666|42|156.24|14.06|170.30|33.46|211.68|103.11|0.53|52.60|259.20| +2450990|46140|7189|86845|1555917|6236|41755|4904|1730428|6950|13025|2|74|10|5|15|666|42|0.00|0.00|0.00|91.50|1067.22|0.00|0.00|0.00|1158.72| +2450909|10970|12313|58248|1002451|6654|43257|95573|157478|3182|38086|1|18|11|5|19|669|8|1206.64|48.26|1254.90|63.42|515.28|892.91|103.53|210.20|626.96| +2450981|17738|4502|81923|1884003|4130|15810|23632|1907726|2581|5436|1|79|14|4|29|670|9|710.46|7.10|717.56|19.77|33.03|547.05|81.70|81.71|59.90| +2450917|12788|8942|98366|428777|3635|26093|28749|603423|6955|31734|4|92|4|1|20|671|3|10.26|0.10|10.36|25.35|100.23|3.38|0.55|6.33|125.68| +2451054|25370|10694|28403|896708|5813|28918|18301|748983|2186|25789|1|78|5|3|26|672|14|154.84|10.83|165.67|68.82|663.74|144.00|3.46|7.38|743.39| +2450963|21179|1216|28403|896708|5813|28918|10106|1147708|6587|17466|1|85|10|4|25|672|1|20.79|1.87|22.66|34.63|47.52|11.43|7.58|1.78|84.02| +2451090|56973|188|90155|1524928|6454|34027|97419|287041|6002|1010|4|49|13|1|22|673|5|257.60|5.15|262.75|1.66|42.30|216.38|2.88|38.34|49.11| +2450900|69167|13258|70994|266911|2828|33192|8992|947812|2758|30551|4|28|3|4|2|674|45|1904.40|38.08|1942.48|94.11|981.00|876.02|791.85|236.53|1113.19| +2450859|45748|12188|69122|1466163|4004|35045|83981|1425761|3627|14340|2|21|5|3|4|675|18|1192.86|95.42|1288.28|68.58|391.68|1157.07|31.13|4.66|555.68| +2450952|67150|62|96781|138582|3936|15737|18616|1101399|6915|31|4|19|19|5|26|676|32|255.68|17.89|273.57|69.27|724.80|43.46|106.11|106.11|811.96| +2450969|74405|15976|40969|613740|2158|10976|97109|614899|5357|39462|4|53|10|5|10|677|51|167.28|1.67|168.95|4.39|382.50|6.69|83.50|77.09|388.56| +2451022|78749|6793|64879|862971|7016|21108|50370|1322023|5544|712|4|15|9|5|28|678|12|281.04|5.62|286.66|69.25|237.84|98.36|115.08|67.60|312.71| +2450881|73555|8842|7114|446054|194|26781|2654|788502|1514|10479|1|29|8|3|5|683|10|1893.70|0.00|1893.70|95.27|130.60|1287.71|284.81|321.18|225.87| +2450888|36803|10303|18697|506427|5004|20734|41662|636451|247|22843|2|92|11|2|18|685|3|120.51|6.02|126.53|38.25|22.59|83.15|2.61|34.75|66.86| +2450878|78810|6151|18697|506427|5004|20734|62509|1146247|978|4109|2|78|9|5|26|685|12|1920.96|0.00|1920.96|9.71|99.96|1632.81|221.87|66.28|109.67| +2450935|69507|980|52296|892630|1764|18498|49265|890208|6019|21030|1|8|16|2|3|688|26|959.92|9.59|969.51|14.88|489.84|748.73|118.26|92.93|514.31| +2451022|27719|248|11205|1484509|6966|47470|97922|1283816|4978|14493|1|78|12|2|34|690|1|111.34|6.68|118.02|79.17|13.49|15.58|23.94|71.82|99.34| +2450897|75210|9284|3358|1898802|6246|16537|88853|43258|1295|23332|2|2|3|5|12|691|36|3300.12|66.00|3366.12|72.22|687.24|1056.03|448.81|1795.28|825.46| +2450967|28765|952|3358|1898802|6246|16537|26686|1208147|2460|17173|2|41|5|3|29|691|4|78.92|7.10|86.02|32.39|33.60|30.77|31.29|16.86|73.09| +2451070|39646|11060|3358|1898802|6246|16537|38743|341501|2030|16559|2|87|19|3|25|691|9|815.31|0.00|815.31|45.53|31.68|8.15|500.43|306.73|77.21| +2451072|73595|12376|3358|1898802|6246|16537|56407|1887828|3732|28389|2|25|14|3|17|691|32|2653.76|132.68|2786.44|92.10|373.12|451.13|2158.57|44.06|597.90| +2451016|72862|11569|65163|1704330|2697|47064|59947|716700|2910|9520|2|102|17|2|27|692|82|8252.48|247.57|8500.05|32.94|5501.38|6436.93|1633.99|181.56|5781.89| +2450921|12241|14551|65163|1704330|2697|47064|93351|1206753|464|13775|2|43|8|4|30|692|31|1789.63|125.27|1914.90|2.56|280.55|733.74|232.29|823.60|408.38| +2451087|31600|6982|65163|1704330|2697|47064|15568|1417128|5335|20742|2|71|3|5|27|692|23|693.68|6.93|700.61|82.48|273.70|326.02|33.08|334.58|363.11| +2450920|73964|880|45555|1511570|432|23647|21842|1005052|2719|6786|4|14|20|2|19|695|53|4896.67|244.83|5141.50|61.22|2338.36|440.70|2361.66|2094.31|2644.41| +2450983|48020|8948|45555|1511570|432|23647|13799|243930|1061|20992|4|107|5|1|22|695|52|3685.24|147.40|3832.64|10.36|0.00|2432.25|25.05|1227.94|157.76| +2450998|73107|8878|5677|380280|4130|49133|14767|1815750|3536|40522|1|20|13|4|18|696|18|513.00|41.04|554.04|30.02|307.80|384.75|17.95|110.30|378.86| +2450921|67332|14504|5677|380280|4130|49133|93415|1269289|933|49377|1|51|9|1|20|696|27|8.10|0.48|8.58|90.65|211.14|7.69|0.17|0.24|302.27| +2450907|63230|15337|80825|882502|2932|22962|86150|1152376|6733|17090|4|81|1|1|28|698|23|2886.04|28.86|2914.90|71.98|620.54|865.81|1191.93|828.30|721.38| +2450910|59553|11074|80825|882502|2932|22962|25472|1523030|3347|27711|4|55|9|1|28|698|7|10.08|0.20|10.28|18.13|218.12|8.46|0.89|0.73|236.45| +2450981|45037|1484|74038|1724230|4822|39414|75819|1593294|3345|5796|4|26|2|3|22|701|66|8246.04|659.68|8905.72|59.76|2834.04|5030.08|996.94|2219.02|3553.48| +2451012|76937|9226|74038|1724230|4822|39414|16582|828739|2604|47622|4|68|10|5|13|701|14|991.76|79.34|1071.10|33.92|0.00|9.91|864.02|117.83|113.26| +2451065|66514|14642|74184|677752|967|30935|48425|321451|4809|12167|2|19|1|4|32|706|35|1059.10|52.95|1112.05|31.42|436.10|74.13|462.93|522.04|520.47| +2451003|72033|11077|74184|677752|967|30935|74184|677752|6389|30935|2|15|9|4|29|706|53|1921.25|134.48|2055.73|85.00|924.85|1171.96|269.74|479.55|1144.33| +2451017|65511|16372|74184|677752|967|30935|31375|1568501|1984|9067|2|7|16|4|32|706|43|3879.46|271.56|4151.02|2.44|1016.09|3297.54|395.70|186.22|1290.09| +2450891|72882|9055|38179|1868225|671|13564|57948|1789268|3994|273|1|48|17|1|35|710|47|96.82|3.87|100.69|52.58|30.55|30.98|1.31|64.53|87.00| +2450933|42868|3032|38179|1868225|671|||||||83|4|5|3|710|34|1390.26||1459.77||271.66|861.96|163.77|364.53|416.94| +2450851|67570|17870|47722|1586478|2834|48037|84095|209758|5773|43388|4|104|13|3|3|714|80|4474.40|313.20|4787.60|38.05|1804.00|939.62|1449.25|2085.53|2155.25| +2450904|57851|17752|65614|1525283|2999|9068|8979|671042|360|42125|2|100|6|4|17|715|19|2166.00|194.94|2360.94|37.94|1287.82|108.30|1070.00|987.70|1520.70| +2451009|45652|4270|55540|1671343|746|8596|55096|354364|2484|14557|2|84|10|3|28|716|2|72.26|0.00|72.26|96.13|38.98|46.96|22.77|2.53|135.11| +2451010|50289|12872|5683|1412335|6187|18439|5683|1412335|17|18439|2|105|6|5|19|717|21|698.46|6.98|705.44|45.30|582.12|537.81|48.19|112.46|634.40| +2450957|75219|15098|5683|1412335|6187|18439|35283|653707|1808|12709|2|58|19|5|28|717|10|125.30|1.25|126.55|54.84|114.90|116.52|7.46|1.32|170.99| +2450991|14695|4730|47923|1609316|662|18008|1028|381672|3489|39706|1|35|17|4|11|718|5|19.35|1.74|21.09|77.98|9.15|2.90|6.41|10.04|88.87| +2450953|36098|8230|74671|714296|1568|8840|57403|93795|1631|25420|2|89|1|2|6|720|74|3773.26|339.59|4112.85|55.25|1206.94|679.18|2103.97|990.11|1601.78| +2450941|64857|10963|74671|714296|1568|8840|83884|543073|4590|28907|2|12|5|4|1|720|42|173.04|3.46|176.50|77.24|76.02|5.19|28.53|139.32|156.72| +2450884|65240|10525|42348|883577|2853|6129|97885|853709|98|34487|1|60|14|3|17|723|17|943.50|75.48|1018.98|2.45|490.62|207.57|242.85|493.08|568.55| +2450942|29683|15931|30190|304794|4198|7602|17873|992643|5191|40092|4|37|9|2|20|724|7|143.01|1.43|144.44|92.24|128.73|137.28|2.06|3.67|222.40| +2450904|47875|14989|24777|1286846|562|35878|69929|2591|6028|5465|4|89|16|3|28|728|1|30.06|1.80|31.86|30.38|14.24|6.31|7.36|16.39|46.42| +2451022|56990|7708|41830|1775341|6452|41163|77970|1034220|5618|37369|1|38|9|5|21|729|55|2292.40|137.54|2429.94|57.28|1432.75|1398.36|178.80|715.24|1627.57| +2450988|60101|8653|46340|1886425|3757|30242|8262|1175118|6693|49915|4|11|1|1|33|730|3|278.19|5.56|283.75|10.03|73.80|80.67|49.38|148.14|89.39| +2450972|18180|7226|46340|1886425|3757|30242|46340|1886425|3089|30242|4|72|4|1|10|730|12|1595.64|63.82|1659.46|30.59|32.16|414.86|519.54|661.24|126.57| +2450963|30418|14528|8528|1577695|462|2829|8528|1577695|6915|2829|2|22|18|4|8|731|22|2998.16|59.96|3058.12|37.40|687.06|119.92|287.82|2590.42|784.42| +2450915|55133|14443|37254|1637535|1369|10450|70257|1802793|7194|14348|2|33|4|3|24|733|6|432.24|0.00|432.24|16.18|42.60|259.34|27.66|145.24|58.78| +2450990|79407|9392|37254|1637535|1369|10450|84548|413044|7011|28931|2|81|5|5|27|733|48|2799.84|0.00|2799.84|23.19|1370.88|391.97|1805.90|601.97|1394.07| +2450924|77836|12950|92932|1208759|3433|17707|14140|716742|80|29818|2|94|19|2|20|736|2|90.86|0.00|90.86|14.60|64.90|69.05|0.87|20.94|79.50| +2450943|23420|2258|76796|1017669|3342|21368|76796|1017669|3757|21368|1|62|9|2|5|743|47|67.68|0.00|67.68|15.69|182.36|31.13|10.23|26.32|198.05| +2450936|27876|7192|96263|1610005|2947|10689|34734|1063664|4213|14409|2|95|3|4|25|747|1|107.93|9.71|117.64|16.11|62.96|58.28|18.37|31.28|88.78| +2450962|63990|97|96263|1610005|2947|10689|96263|1610005|3579|10689|2|8|10|3|9|747|76|8259.68|743.37|9003.05|52.74|2922.20|495.58|5279.58|2484.52|3718.31| +2450993|53889|11155|10008|457070|4932|11194|2985|490885|6214|41096|2|68|3|2|16|752|29|5350.79|428.06|5778.85|55.59|1854.84|4548.17|658.14|144.48|2338.49| +2450948|65333|8030|10008|457070|4932|11194|63138|542618|4309|400|2|4|13|4|20|752|4|644.92|19.34|664.26|80.30|345.84|554.63|31.60|58.69|445.48| +2450907|82416|16870|16103|270419|3488|1004|16103|270419|864|1004|1|1|10|3|28|753|19|486.40|34.04|520.44|24.41|352.26|175.10|286.39|24.91|410.71| +2450913|80001|3272|16103|270419|3488|1004|37078|1466001|2010|9572|1|81|18|5|27|753|10|1074.60|85.96|1160.56|97.08|58.40|784.45|249.52|40.63|241.44| +2450959|23894|12172|8426|92656|2133|10010|58573|1562359|1518|16724|2|73|15|1|27|754|4|19.76|1.58|21.34|15.81|59.28|3.35|13.29|3.12|76.67| +2450982|4813|1492|62737|1516003|3669|34135|29218|1348516|658|36047|4|24|7|4|33|756|57|3172.62|285.53|3458.15|33.12|1569.21|1078.69|1633.26|460.67|1887.86| +2450998|75365|9446|41350|970915|2433|16398|22202|270654|542|17583|4|22|20|1|24|757|13|519.61|36.37|555.98|13.82|108.16|483.23|6.91|29.47|158.35| +2450982|41018|12968|83778|1757939|1700|35861|37145|1279961|3185|6009|4|93|1|5|4|759|4|489.44|9.78|499.22|63.01|191.52|92.99|130.82|265.63|264.31| +2450894|14332|7177|83778|1757939|1700|35861|8049|313403|635|33867|4|24|16|3|5|759|10|530.60|31.83|562.43|80.49|170.20|111.42|62.87|356.31|282.52| +2450896|18847|13046|7734|1561164|1347|25930|7734|1561164|5344|25930|1|32|1|2|33|763|68|346.80|10.40|357.20|18.48|1457.92|204.61|82.47|59.72|1486.80| +2450908|6142|17287|7734|1561164|1347|25930|7734|1561164|3860|25930|1|46|18|1|12|763|36|2205.72|176.45|2382.17|13.08|341.64|1323.43|652.89|229.40|531.17| +2451056|69947|9271|9628|1058037|1293|4949|21318|732521|5184|47494|2|88|17|2|10|764|13|110.89|7.76|118.65|16.78|34.06|73.18|16.59|21.12|58.60| +2450919|82539|2089|9628|1058037|1293|4949|42067|1706327|66|18558|2|84|8|5|17|764|15|1178.10|47.12|1225.22|86.44|814.20|895.35|138.54|144.21|947.76| +2450938|62326|7603|30655|1811895|7088|2281|82048|1357313|3380|21447|1|8|8|5|14|765|30|121.80|10.96|132.76|11.60|182.70|93.78|7.00|21.02|205.26| +2450892|43714|6052|30655|1811895|7088|2281|9613|715819|5417|9062|1|25|3|3|16|765|71|6483.72|0.00|6483.72|51.74|2570.91|5511.16|505.73|466.83|2622.65| +2450922|10238|10021|11518|889905|7045|13432|76757|614559|3747|26886|1|9|17|1|32|767|37|4624.26|323.69|4947.95|38.90|495.43|4624.26|0.00|0.00|858.02| +2450955|66096|9064|81045|1543942|1239|1336||1410145||||49|18|2||768|14||10.02||||263.89|14.03||| +2451043|7590|3244|88469|388921|3773|41825|19913|1316464|2477|42890|4|9|16|4|17|770|1|90.55|2.71|93.26|22.79|57.85|65.19|13.69|11.67|83.35| +2451033|29339|3313|74595|1027282|2722|6209|97661|89528|3425|42169|1|86|14|1|30|771|25|2071.50|165.72|2237.22|34.70|906.25|1097.89|720.47|253.14|1106.67| +2451024|11137|12556|76752|457086|620|13754|76752|457086|775|13754|2|52|9|4|21|773|36|858.60|60.10|918.70|54.13|0.00|480.81|22.66|355.13|114.23| +2450894|35464|15499|76752|457086|620|13754|67440|105241|3145|5496|2|11|2|3|29|773|24|2546.64|25.46|2572.10|54.50|214.56|2470.24|44.31|32.09|294.52| +2451009|65738|4180|76752|457086|620|13754|95723|1808208|1867|14326|2|98|3|2|18|773|28|77.56|3.87|81.43|20.60|77.56|76.78|0.56|0.22|102.03| +2450863|43485|13654|76752|457086|620|13754|14587|949905|1706|25907|2|80|4|5|21|773|7|238.70|14.32|253.02|93.76|122.71|176.63|54.00|8.07|230.79| +2450908|81925|12364|2941|505613|1438|12510|17868|1628341|7|26|2|84|10|4|25|776|41|2211.95|176.95|2388.90|30.68|807.29|1592.60|253.93|365.42|1014.92| +2451008|72748|8821|48512|216157|3802|33847|57974|490712|536|43913|1|48|11|5|22|777|1|40.35|3.63|43.98|25.22|28.36|20.98|6.97|12.40|57.21| +2451071|57946|16093|91773|246789|6605|32821|81675|1012473|2740|18207|4|94|1|1|33|779|13|1662.57|133.00|1795.57|11.75|983.45|216.13|1055.90|390.54|1128.20| +2450981|65084|11377|90918|396421|3971|31921|17482|880022|4213|8851|1|44|11|2|20|780|15|85.65|7.70|93.35|78.99|162.75|21.41|32.76|31.48|249.44| +2450913|75242|4705|41838|427299|3960|16221|84330|1091192|5847|28502|1|76|14|4|19|782|11|355.19|7.10|362.29|37.93|159.39|39.07|97.99|218.13|204.42| +2451009|74414|17024|41838|427299|3960|16221|31262|1738199|487|1175|1|6|13|1|11|782|10|790.60|0.00|790.60|48.65|236.30|719.44|64.04|7.12|284.95| +2451036|77718|2560|20130|1776167|6604|45041|12982|250203|2923|46490|1|81|4|1|28|784|49|493.92|14.81|508.73|58.91|1289.68|301.29|148.32|44.31|1363.40| +2450958|64398|3182|8539|889851|3163|24875|35936|705011|5687|14316|2|86|17|1|17|786|2|13.44|0.26|13.70|85.10|5.62|0.53|0.25|12.66|90.98| +2450955|57866|5233|8539|889851|3163|24875|57676|398081|4465|44896|2|33|9|5|11|786|20|879.00|17.58|896.58|80.03|156.20|685.62|125.69|67.69|253.81| +2450899|47452|9794|98486|1917630|3621|26765|86744|949351|2253|17108|2|61|7|4|16|788|3|107.67|0.00|107.67|48.49|18.72|0.00|93.67|14.00|67.21| +2451050|27278|2768|19329|164721|4513|41552|56390|1615693|6684|4428|4|73|1|2|8|792|16|90.40|4.52|94.92|5.04|875.04|55.14|5.64|29.62|884.60| +2451035|80002|7822|19329|164721|4513|41552|91639|51903|4755|43654|4|20|4|1|30|792|7|232.40|2.32|234.72|13.69|269.08|53.45|84.10|94.85|285.09| +2450940|53986|5938|69670|1824622|727|45279|92819|480919|3304|27336|1|17|11|1|6|793|13|372.06|3.72|375.78|22.21|343.46|152.54|158.05|61.47|369.39| +2451048|63206|11432|69670|1824622|727|45279|24461|1600183|6821|43059|1|16|20|5|13|793|33|761.97|30.47|792.44|77.81|200.31|693.39|7.54|61.04|308.59| +2450998|79098|9418|28100|242325|6950|7575|72629|1037038|1066|43894|2|19|5|5|18|794|70|8026.20|481.57|8507.77|42.12|6391.70|3692.05|3033.90|1300.25|6915.39| +2450993|27072|2852|43545|1394763|1981|17599|57343|1561149|4392|14835|4|55|4|2|30|796|61|4528.03|316.96|4844.99|91.51|397.11|2852.65|234.55|1440.83|805.58| +2451000|52259|2224|88149|1536816|5130|44597|57632|1015774|768|27315|2|1|10|4|1|797|3|15.24|0.60|15.84|45.94|7.98|9.14|4.27|1.83|54.52| +2451015|70874|860|79433|1406132|5278|30824|72701|33733|5334|38535|4|45|18|4|35|799|47|652.36|32.61|684.97|5.74|924.02|561.02|24.66|66.68|962.37| +2450984|43041|6403|79433|1406132|5278|30824|451|1790456|732|29408|4|38|3|4|25|799|13|843.05|8.43|851.48|90.86|135.98|219.19|461.65|162.21|235.27| +2450964|43866|9919|39826|1876330|3587|8891|91358|267182|5539|16696|1|48|7|4|6|801|28|1902.88|76.11|1978.99|53.84|52.64|1427.16|161.74|313.98|182.59| +2450952|27747|1354|39826|1876330|3587|8891|85539|1225617|5046|13275|1|92|15|4|28|801|35|571.90|5.71|577.61|96.03|985.25|337.42|154.75|79.73|1086.99| +2451032|9394|13690|39826|1876330|3587|8891|21315|337650|6088|23373|1|25|8|3|18|801|43|5642.03|451.36|6093.39|99.45|194.36|4231.52|437.25|973.26|745.17| +2450898|78842|17060|39826|1876330|3587|8891|76335|1082366|274|35735|1|22|5|1|7|801|11|923.01|83.07|1006.08|35.11|389.18|239.98|266.38|416.65|507.36| +2451031|31894|8822|7328|393061|4747|44033|47208|1320543|3181|37772|4|3|19|5|18|802|37|1780.44|71.21|1851.65|98.70|667.48|1673.61|22.43|84.40|837.39| +2451023|17016|11818|82603|1213282|3161|48240|11542|1326731|3859|634|2|103|12|5|25|803|69|3340.98|100.22|3441.20|31.05|89.70|835.24|601.37|1904.37|220.97| +2451020|79157|2410|82603|1213282|3161|48240|76822|1208298|3806|48561|2|102|19|2|32|803|41|970.88|9.70|980.58|32.77|172.20|543.69|162.33|264.86|214.67| +2450938|46879|8521|||4855|||||10161|||15|1||806|12|748.32|0.00|||||33.75|48.57|| +2450939|44314|12751|39264|1220299|2581|30016|43644|748707|2000|6940|4|48|13|3|13|808|3|27.39|1.91|29.30|33.82|14.43|23.28|0.24|3.87|50.16| +2451007|79107|464|40915||3285|10470|3876|1547378|4199|||7||3|17|809|10|||||||||| +2450953|1998|17548|40915|124500|3285|10470|97413|1366996|5315|16834|1|77|18|4|3|809|21|207.69|18.69|226.38|8.95|63.21|62.30|11.63|133.76|90.85| +2451015|29601|7021||||10470||572913||7654|||2||8|809||213.65||222.19||103.05||49.39|95.90|| +2451002|62873|16039|40915|124500|3285|10470|38829|1214891|423|42355|1|41|4|1|11|809|30|588.60|0.00|588.60|79.27|153.30|329.61|46.61|212.38|232.57| +2450912|68554|17596|88311|1487383|6455|11480|60519|1118914|4090|2866|4|42|14|5|6|811|31|556.45|33.38|589.83|55.43|147.56|417.33|15.30|123.82|236.37| +2451016|71315|11216|10776|207141|3153|16718|94016|271186|1507|31381|2|107|10|4|8|812|7|52.01|2.08|54.09|63.27|10.92|40.56|4.00|7.45|76.27| +2450989|42532|11779|5130|584501|5472|11290|10905|1164152|5041|3977|1|75|18|4|15|815|44|76.12|3.04|79.16|30.17|311.96|9.13|36.84|30.15|345.17| +2451018|17298|7756|5130|584501|5472|11290|46683|1478473|3307|46141|1|73|3|2|31|815|58|1980.12|178.21|2158.33|97.92|0.00|574.23|759.18|646.71|276.13| +2450907|37838|4276|3739|419811|3083|1908|90970|1525986|2482|2178|2|29|19|5|6|816|3|4.65|0.18|4.83|25.08|1.80|1.86|2.12|0.67|27.06| +2450929|60639|4742|74411|578576|3988|27800|69484|92595|5107|41172|4|97|14|4|16|817|40|122.80|8.59|131.39|92.18|21.60|94.55|4.80|23.45|122.37| +2451007|44915|482|74411|578576|3988|27800|61671|227458|4403|42195|4|99|4|1|11|817|25|3863.25|193.16|4056.41|2.97|1159.00|1236.24|2049.06|577.95|1355.13| +2450977|81132|16610|42146|1242808|4717|4179|73724|167743|6838|13533|1|85|4|4|14|818|7|552.30|49.70|602.00|83.87|160.79|276.15|190.54|85.61|294.36| +2450925|37509|9050|42146|1242808|4717|4179|19569|466972|6836|31547|1|30|13|4|35|818|32|65.60|1.31|66.91|84.64|29.12|26.24|16.53|22.83|115.07| +2451000|43691|1610|||||28127|424205|||4|||||820|1||0.66||97.51|53.17||2.78||151.34| +2450999|74674|15823|53817|1112487|3157|41380|76253|1660181|6990|8599|2|102|19|3|10|821|8|95.76|7.66|103.42|1.98|73.68|47.88|6.70|41.18|83.32| +2450993|65602|7837|53817|1112487|3157|41380|56219|1263903|5165|18427|2|6|13|2|19|821|6|846.00|42.30|888.30|95.85|423.00|769.86|39.59|36.55|561.15| +2450875|57906|2665|87999|162328|6610|45739|68891|1832747|1944|28341|1|69|5|2|34|822|12|753.72|0.00|753.72|97.69|439.68|565.29|24.49|163.94|537.37| +2450977|6767|6514|87999|162328|6610|45739|94371|626867|7176|21838|1|50|18|5|14|822|15|66.00|0.66|66.66|73.35|10.05|47.52|8.13|10.35|84.06| +2450928|47964|16270|94362|1503502|6689|1029|19335|1376006|4311|33816|2|41|5|5|25|824|15|176.70|7.06|183.76|40.29|54.30|70.68|43.46|62.56|101.65| +2450998|57852|4778|80395|467584|5173|43515|80395|467584|7136|43515|1|73|15|3|31|826|12|173.16|0.00|173.16|25.59|311.64|95.23|28.05|49.88|337.23| +2451091|22290|14690|24837|1143027|6370|24560|57764|1907949|4273|8950|2|107|8|5|17|827|8|476.96|9.53|486.49|38.36|93.20|19.07|219.78|238.11|141.09| +2451026|81070|4888|25842|289093|344|3948|34880|868042|3002|21562|2|76|2|2|7|828|36|214.92|8.59|223.51|47.21|5.04|4.29|56.87|153.76|60.84| +2450923|40614|17366|94578|1161589|6200|5611|14182|938824|1977|20377|1|57|17|2|32|829|17|2607.29|78.21|2685.50|13.65|1271.09|2450.85|53.18|103.26|1362.95| +2450969|55232|13984|94578|1161589|6200|5611|23576|1756465|1335|21052|1|46|2|2|12|829|3|225.93|6.77|232.70|35.90|152.82|6.77|184.09|35.07|195.49| +2450979|43567|15472|94578|1161589|6200|5611|25904|1902542|1812|9482|1|72|19|4|34|829|23|705.41|14.10|719.51|95.02|302.22|331.54|18.69|355.18|411.34| +2451061|33759|4207|45817|1604859|6301|28096|15649|105581|1701|34668|2|11|15|5|30|833|1|177.91|0.00|177.91|51.94|117.77|55.15|35.60|87.16|169.71| +2451012|6197|1411|45817|1604859|6301|28096|92625|1784596|3633|9117|2|27|8|5|17|833|27|268.11|21.44|289.55|30.97|211.41|53.62|2.14|212.35|263.82| +2450931|9232|3694|85391|1585529|2820|12111|11226|1289428|6168|18908|1|3|8|3|28|835|83|428.28|29.97|458.25|24.72|1836.79|269.81|155.30|3.17|1891.48| +2451012|26384|14882|85391|1585529|2820|12111|15651|744822|6324|46979|1|5|11|1|18|835|38|1185.22|0.00|1185.22|45.84|2142.82|568.90|277.34|338.98|2188.66| +2451070|76090|7465|85391|1585529|2820|12111|21818|605318|6871|30424|1|14|10|3|1|835|18|664.38|19.93|684.31|35.86|450.00|279.03|315.98|69.37|505.79| +2450903|16498|5354|13192|1611087|4982|47630|63654|1911522|4284|17446|2|33|12|4|11|837|23|2854.53|142.72|2997.25|81.44|436.54|114.18|657.68|2082.67|660.70| +2451038|80954|3794|77053|1905510|14|18071|44714|105112|3268|7071|2|35|19|5|1|838|7|31.92|2.23|34.15|2.39|17.08|9.57|13.63|8.72|21.70| +2450887|45661|12808|41614|1704036|4058|15280|67618|1573027|5657|10287|2|88|4|4|9|839|24|1208.64|0.00|1208.64|57.65|682.56|398.85|380.60|429.19|740.21| +2450998|32278|15097|41614|1704036|4058|15280|42631|900969|2594|43866|2|34|8|5|30|839|14|804.16|40.20|844.36|95.37|41.16|627.24|61.92|115.00|176.73| +2450949|71850|5812|87529|1892767|5558|45549|30717|557982|4162|30342|4|25|9|4|34|840|16|975.68|19.51|995.19|23.33|477.44|341.48|114.15|520.05|520.28| +2450971|76365|8503|20456|1078775|6694|9413|9145|53844|4214|12984|4|81|7|2|27|841|89|2362.06|47.24|2409.30|2.64|2214.32|259.82|2102.24|0.00|2264.20| +2450985|65867|17150|20456|1078775|6694|9413|62929|1025633|3167|23101|4|4|16|5|34|841|9|102.69|9.24|111.93|65.72|76.95|78.04|10.10|14.55|151.91| +2450870|43565|14926|39069|1908951|880|43162|90332|228982|844|33553|2|90|6|5|22|844|42|1561.56|109.30|1670.86|59.67|187.32|1202.40|129.29|229.87|356.29| +2450873|78678|15691|39069|1908951|880|43162|62207|680385|734|1483|2|98|17|1|35|844|44|3799.84|75.99|3875.83|43.53|3392.84|1139.95|1808.72|851.17|3512.36| +2450937|51401|9745|31059|1291663|3467|40471|||||4|||1|35|846|60|77.40||81.27|26.26||||18.12|| +2451063|36105|10234|9658|419572|5839|12106|90954|1484941|5025|24309|1|38|14|3|2|849|7|82.67|4.13|86.80|94.68|65.59|32.24|46.89|3.54|164.40| +2450999|62543|6121|87473|798465|7057|14896|35703|1018672|406|49340|1|9|9|3|12|850|45|3433.95|103.01|3536.96|90.82|1485.00|2644.14|236.94|552.87|1678.83| +2450954|8633|11623|87473|798465|7057|14896|73901|1685573|4517|21061|1|108|13|2|1|850|53|2967.47|29.67|2997.14|15.05|470.11|830.89|1751.99|384.59|514.83| +2451000|7738|7280|10311|794543|2775|36207|10938|1199783|3032|9034|1|96|14|1|15|852|67|11584.30|579.21|12163.51|79.05|2864.92|5907.99|5392.49|283.82|3523.18| +2450921|69961|11191|97945|156176|1254|14285|60493|1582973|5614|17055|4|2|11|2|13|856|38|5099.22|203.96|5303.18|84.64|2463.54|4079.37|795.48|224.37|2752.14| +2450984|56896|11275|97945|156176|1254|14285|32201|945292|1046|36000|4|41|9|3|14|856|51|5540.13|0.00|5540.13|63.35|2003.79|2936.26|364.54|2239.33|2067.14| +2450926|45878|13160|52933|3581|5782|42830|54688|772172|1317|10509|2|74|18|5|7|857|22|398.64|35.87|434.51|42.66|144.10|27.90|341.08|29.66|222.63| +2451025|74444|6496|52933|3581|5782|42830|45379|1419582|7060|23338|2|34|12|5|3|857|17|634.95|6.34|641.29|12.26|340.17|520.65|30.86|83.44|358.77| +2450998|52999|12424|52933|3581|5782|42830|182|1058939|1319|24852|2|83|5|3|18|857|16|358.08|32.22|390.30|45.66|1026.88|229.17|34.80|94.11|1104.76| +2450971|74852|13792|52933|3581|5782|42830|52933|3581|3274|42830|2|49|6|1|29|857|31|522.97|10.45|533.42|26.37|1680.82|224.87|181.84|116.26|1717.64| +2451040|1495|8048|71757|1238493|76|1612|51826|31488|1567|15199|2|44|15|2|24|858|58|1771.90|124.03|1895.93|48.51|217.50|1293.48|191.36|287.06|390.04| +2450924|68547|12559|50393|963528|4310|18692|16525|1454159|3976|49346|1|101|10|2|32|859|8|362.72|32.64|395.36|83.29|7.04|174.10|90.53|98.09|122.97| +2450968|75894|3290|23250|1342816|3982|25948|39355|1656695|3699|7503|1|7|15|2|11|860|5|399.95|23.99|423.94|25.35|195.10|363.95|5.40|30.60|244.44| +2450861|40621|11950|23250|1342816|3982|25948|76694|1247812|4066|38320|1|12|9|5|29|860|27|1791.99|125.43|1917.42|46.84|876.15|268.79|1294.72|228.48|1048.42| +2450954|68151|9734|84180|875404|5828|12299|7035|1358655|3359|4910|1|31|9|3|27|861|46|1196.00|95.68|1291.68|93.94|1058.00|406.64|370.99|418.37|1247.62| +2450975|2789|7435|22844|1614810|6776|14295|804|1282286|1798|910|2|1|6|3|5|862|27|2331.18|69.93|2401.11|36.62|1671.30|1002.40|1222.47|106.31|1777.85| +2450924|61993|7111|34433|214161|736|6064|54059|139758|424|3137|1|23|15|3|19|863|33|413.16|28.92|442.08|44.22|173.25|375.97|36.44|0.75|246.39| +2451033|44444|1891|33070|570828|3786|1144|30390|35665|3974|31436|4|71|9|1|23|864|17|61.37|3.68|65.05|21.79|88.06|12.27|37.31|11.79|113.53| +2451037|29390|6844|14915|406237|3159|14081|9845|763020|6822|46671|2|18|17|1|8|865|46|4476.26|313.33|4789.59|54.32|344.08|2417.18|1585.49|473.59|711.73| +2450862|65682|11126|89530|277583|6436|37336|44082|446846|2822|40845|4|67|20|3|4|866|1|40.33|3.62|43.95|60.88|8.69|23.79|6.28|10.26|73.19| +2451039|28344|12254|64679|710529|5979|8229|64679|710529|5241|8229|4|7|13|4|2|867|3|69.45|0.00|69.45|63.04|40.86|10.41|49.00|10.04|103.90| +2450887|74478|10844|80916|1632834|1552|3870|59927|798921|4456|34391|2|38|3|3|33|871|22|836.66|75.29|911.95|79.79|772.20|200.79|591.35|44.52|927.28| +2451085|72437|2090|80916|1632834|1552|3870|54362|356998|253|22308|2|51|13|3|33|871|4|171.20|10.27|181.47|77.06|146.16|11.98|135.33|23.89|233.49| +2450950|16682|5791|93330|794798|3466|13287|78225|16100|2188|18629|2|30|19|5|3|872|21|18.48|0.18|18.66|20.83|36.96|0.18|1.28|17.02|57.97| +2450862|27470|12560|93330|794798|3466|13287|15296|1209280|7079|6525|2|70|16|2|6|872|5|140.90|1.40|142.30|10.18|73.25|138.08|2.42|0.40|84.83| +2450941|65586|12218|93330|794798|3466|13287|98307|991088|6851|18214|2|90|3|1|14|872|33|625.35|0.00|625.35|58.14|179.85|37.52|276.28|311.55|237.99| +2450985|72250|554|63431|1684133|6082|26031|17423|961371|314|38010|2|31|19|2|3|873|96|67.20|4.70|71.90|66.84|134.40|39.64|9.92|17.64|205.94| +2450846|54193|2902|63431|1684133|6082|26031|16646|1087901|2231|43408|2|66|5|5|5|873|28|3603.60|36.03|3639.63|69.30|1201.20|2918.91|267.02|417.67|1306.53| +2450991|74170|9946|88089|980039|5589|13892|18606|1281958|936|3514|1|34|14|2|3|874|35|4016.25|240.97|4257.22|79.78|1468.25|1285.20|2403.32|327.73|1789.00| +2451017|45413|14881|30221|1625222|4223|19624|70117|19869|473|43430|2|99|2|1|30|876|43|3716.92|148.67|3865.59|77.10|1821.05|2118.64|1182.72|415.56|2046.82| +2450971|28361|17548|15804|662025|1046|45314|73075|1087165|2723|27834|1|59|1|2|33|877|27|4054.86|243.29|4298.15|12.83|643.41|364.93|516.59|3173.34|899.53| +2451013|73492|15020|15804|662025|1046|45314|80288|47406|5542|29231|1|31|12|3|14|877|47|258.03|2.58|260.61|18.11|553.19|131.59|30.34|96.10|573.88| +2450949|11625|10957|15804|662025|1046|45314|70058|182500|6892|49637|1|14|6|3|15|877|14|218.12|8.72|226.84|29.86|100.66|191.94|15.70|10.48|139.24| +2451000|38289|4393|3296|1251132|568|2757|41254|1127511|6618|6078|2|7|12|2|4|879|66|345.84|27.66|373.50|37.93|183.48|0.00|100.29|245.55|249.07| +2451016|72085|9458|3296|1251132|568|2757|5421|1493100|3170|37687|2|11|4|1|31|879|28|913.92|45.69|959.61|79.24|787.92|868.22|26.04|19.66|912.85| +2450939|36637|2962|13677|1178248|6372|37743|80593|1512600|4833|25066|1|28|5|3|13|880|8|291.52|20.40|311.92|44.65|118.72|11.66|11.19|268.67|183.77| +2450897|80827|16664|42130|1723917|1351|1938|76085|1823581|2372|34410|1|39|10|3|14|881|7|472.85|14.18|487.03|32.29|71.33|378.28|51.06|43.51|117.80| +2450891|48608|6278|8072|985247|3222|11997|33507|1426946|5366|13339|4|79|12|5|35|882|51|814.98|16.29|831.27|24.86|499.29|562.33|149.06|103.59|540.44| +2450996|75602|14440|20337|207201|6065|16129|20337|207201|254|16129|1|93|20|3|33|884|43|780.88|7.80|788.68|62.52|678.97|273.30|284.24|223.34|749.29| +2451047|70150|4111|56945|1249960|6175|10790|45661|104724|1610|33343|1|27|8|4|7|889|1|0.21|0.01|0.22|3.44|0.42|0.16|0.02|0.03|3.87| +2450996|68521|16456|56945|1249960|6175|10790|37088|1487074|6513|44799|1|33|10|1|27|889|28|25.20|0.25|25.45|16.50|2.52|15.12|0.60|9.48|19.27| +2451063|50249|6784|56945|1249960|6175|10790|96727|1578444|4387|5483|1|4|5|3|4|889|76|992.56|0.00|992.56|26.87|3055.20|456.57|53.59|482.40|3082.07| +2451064|28163|1684|78942|1799568|6947|16326|50960|62243|1157|47671|1|17|20|1|15|890|43|1075.43|86.03|1161.46|62.83|477.73|709.78|98.72|266.93|626.59| +2451031|78333|12238|78942|1799568|6947|16326|92973|680180|3461|1589|1|12|12|3|19|890|4|112.96|5.64|118.60|99.71|77.64|22.59|82.23|8.14|182.99| +2451041|22979|9164|78942|1799568|6947|16326|85|687829|3574|15366|1|105|15|2|27|890|35|267.05|8.01|275.06|82.61|179.55|237.67|20.27|9.11|270.17| +2450975|43679|17443|9988|376508|5428|21995|93992|770490|1176|26262|4|103|20|1|7|894|4|91.32|1.82|93.14|68.07|88.60|51.13|22.10|18.09|158.49| +2450989|78356|1891|9988|376508|5428|21995|80991|1614339|4526|3560|4|15|15|2|31|894|36|3918.24|274.27|4192.51|98.63|803.52|2977.86|413.76|526.62|1176.42| +2450961|67777|5594|98698|61881|3464|47447|52835|447699|3841|35630|4|12|16|3|15|895|3|41.79|1.67|43.46|85.11|24.36|30.08|3.63|8.08|111.14| +2451053|33929|7288|33172|1857997|6022|14047|62033|1629872|5215|31358|4|50|7|5|13|897|63|299.88|20.99|320.87|96.68|699.93|131.94|85.64|82.30|817.60| +2451027|3040|148|33172|1857997|6022|14047|79287|371681|4793|39058|4|37|7|4|21|897|55|2971.65|237.73|3209.38|61.41|585.75|237.73|1667.69|1066.23|884.89| +2451033|33793|11911|45509|130790|1671|27752|77602|543574|7161|21148|4|86|2|4|16|898|54|6197.04|557.73|6754.77|45.62|1549.26|6073.09|53.29|70.66|2152.61| +2451031|70458|9962|45509|130790|1671|27752|95844|1662196|3180|40964|4|84|9|3|17|898|14|40.74|0.81|41.55|78.67|713.44|10.18|12.22|18.34|792.92| +2450866|31700|13783|43127|191577|4474|18124|54240|1729253|6229|19958|1|2|19|3|32|900|23|78.20|6.25|84.45|13.09|49.45|75.85|1.85|0.50|68.79| +2451025|26601|11131|14676|1704466|5103|26093|79387|1163900|6079|27858|1|104|15|4|3|901|77|182.49|0.00|182.49|82.04|936.32|140.51|39.88|2.10|1018.36| +2450907|84730|14434|95504||888||13499|||30685|4|36|9|5|8|903||448.96|26.93|475.89|38.71||130.19|||| +2450972|22700|7177|95504|346596|888|47980|53195|1053227|3190|30753|4|66|9|2|15|903|8|99.84|4.99|104.83|8.89|49.92|42.93|48.37|8.54|63.80| +2451028|21714|511|69625|884802|3179|21556|6827|497239|1843|26047|1|5|12|4|13|907|28|3975.16|198.75|4173.91|21.00|1516.76|1272.05|1729.99|973.12|1736.51| +2450904|44180|16400|85499|555565|1631|6944|41352|1203589|397|6005|1|66|16|4|17|909|50|19.00|1.71|20.71|24.56|5.50|0.76|10.94|7.30|31.77| +2450848|57260|4864|32960|1851088|5663|2238|74479|763341|2551|38844|2|91|9|5|18|911|3|50.01|1.50|51.51|47.45|53.58|23.50|18.82|7.69|102.53| +2450944|45961|187|60572|578785|1445|30458|60572|578785|179|30458|2|75|11|4|2|912|26|892.32|53.53|945.85|76.10|2047.24|615.70|24.89|251.73|2176.87| +2451030|28279|13927|84815|1098986|710|41281|64843|639909|4608|25354|1|92|12|3|2|913|16|486.40|29.18|515.58|59.11|709.28|350.20|130.75|5.45|797.57| +2450972|32651|14332|66151|1578428|5552|30229|51586|1134368|675|40629|2|37|17|4|6|914|25|97.00|6.79|103.79|65.10|16.75|70.81|24.35|1.84|88.64| +2451024|85236|5824|40296|779212|5328|5844|14267|1784214|433|47405|4|43|15|4|3|915|52|4147.00|248.82|4395.82|96.55|2008.24|41.47|2873.87|1231.66|2353.61| +2450997|70127|14632|95990|1834161|3179|43350|62757|1484888|4035|15982|1|14|16|3|5|919|12|243.96|14.63|258.59|20.79|91.44|197.60|16.68|29.68|126.86| +2450960|55434|13444|67363|126060|1557|49454|47503|1515957|7188|5619|1|42|19|5|31|921|3|46.29|2.77|49.06|68.04|52.92|6.48|39.81|0.00|123.73| +2450892|43940|15592|10395|138483|2571|47412|82985|1806252|3914|1976|1|26|15|2|34|922|47|2093.38|188.40|2281.78|81.04|758.11|690.81|659.20|743.37|1027.55| +2451015|68370|7126|10395|138483|2571|47412|7177|180337|4604|1671|1|2|5|2|9|922|53|307.93|15.39|323.32|30.06|182.85|160.12|59.12|88.69|228.30| +2450987|71317|13742|10395|138483|2571|47412|10395|138483|2899|47412|1|71|18|1|19|922|3|0.96|0.00|0.96|77.03|13.08|0.92|0.00|0.04|90.11| +2450998|44815|8102|10395|138483|2571|47412|96667|1807624|6820|17969|1|31|10|4|25|922|4|473.80|37.90|511.70|96.70|149.04|151.61|106.32|215.87|283.64| +2450917|40599|8020|20004|982448|2139|37030|37987|1148599|5610|26221|2|77|8|4|7|923|50|3136.50|0.00|3136.50|78.74|1601.00|2038.72|823.33|274.45|1679.74| +2450966|44715|10624|20004|982448|2139|37030|8632|489827|4971|38162|2|108|1|5|4|923|15|206.55|8.26|214.81|55.25|413.25|136.32|58.29|11.94|476.76| +2450884|79660|15379|20004|982448|2139|37030|61836|259148|3315|41929|2|46|11|3|27|923|21|1203.51|48.14|1251.65|61.56|200.55|300.87|794.32|108.32|310.25| +2451020|77827|8156|75763|225590|3417|4844|48458|1883599|4806|20860|4|7|8|5|14|927|1|256.49|12.82|269.31|83.67|18.13|41.03|135.73|79.73|114.62| +2450908|42085|16765|82845|1111015|3985|39355|22396|903992|584|37176|4|51|12|4|26|928|36|1798.56|53.95|1852.51|40.90|2589.84|737.40|976.26|84.90|2684.69| +2450907|33148|451|82845|1111015|3985|39355|82845|1111015|5408|39355|4|81|15|5|20|928|41|2352.17|70.56|2422.73|73.84|1102.49|1364.25|286.49|701.43|1246.89| +2450903|9217|2986|82845|1111015|3985|39355|31311|958323|3572|42421|4|31|15|4|25|928|7|317.52|28.57|346.09|60.58|27.58|174.63|98.59|44.30|116.73| +2451061|71860|6049|43664|606314|7011|13661|59340|856026|5985|45544|2|10|1|5|24|929|46|7158.52|572.68|7731.20|89.44|2334.04|5726.81|787.44|644.27|2996.16| +2450920|18026|14461|91792|551651|1910|49816|58533|536687|5758|21182|2|44|1|5|34|930|12|367.56|0.00|367.56|69.84|1286.76|297.72|25.84|44.00|1356.60| +2450959|59805|16522|39767|986531|142|6906|20766|433285|6190|14295|4|13|17|4|29|934|4|18.84|0.37|19.21|92.96|150.96|16.01|2.71|0.12|244.29| +2451009|28242|13562|54683|901748|366|30231|32054|1687470|1632|42547|2|89|6|1|33|935|51|1657.50|99.45|1756.95|26.55|517.65|16.57|574.32|1066.61|643.65| +2450977|29313|6332|54683|901748|366|30231|87041|1026395|2672|48949|2|24|7|1|8|935|32|294.72|5.89|300.61|41.40|122.24|206.30|84.88|3.54|169.53| +2450967|34075|5539|92193|503106|3266|1826|42654|1684055|6317|16579|4|33|5|2|9|936|64|3437.44|68.74|3506.18|98.31|1050.24|240.62|479.52|2717.30|1217.29| +2450931|32565|8962|92193|503106|3266|1826|6124|576491|4734|28577|4|42|6|4|5|936|42|1783.74|53.51|1837.25|67.85|371.28|321.07|1126.25|336.42|492.64| +2450936|35349|3446|43946|476430|1714|49845|38367|374911|2757|12603|2|60|1|2|33|937|3|128.31|0.00|128.31|64.25|46.95|0.00|26.94|101.37|111.20| +2450921|45281|440|43946|476430|1714|49845|50249|745145|5507|7116|2|105|9|3|5|937|26|872.56|61.07|933.63|46.85|663.00|540.98|162.47|169.11|770.92| +2450892|75670|8050|7993|362940|4640|49187|15016|1839692|5041|49237|2|11|4|5|24|939|12|321.00|9.63|330.63|19.87|238.08|112.35|175.26|33.39|267.58| +2450901|64248|988|88656|520316|6376|31236|97744|307703|6510|164|2|17|12|4|6|941|3|212.52|19.12|231.64|91.05|52.47|167.89|3.12|41.51|162.64| +2451056|70180|13435|9675|1364205|1488|30418|68097|516789|301|31413|4|36|15|2|19|943|27|1365.93|0.00|1365.93|21.94|390.15|273.18|841.41|251.34|412.09| +2450882|42941|4561|326|727569|1659|37749|90514|776948|2109|35670|1|50|13|4|12|953|46|542.34|16.27|558.61|76.31|30.36|178.97|167.15|196.22|122.94| +2450943|22352|4669|326|727569|1659|37749|18300|134837|6952|20314|1|11|13|3|35|953|18|430.20|30.11|460.31|27.06|35.82|86.04|58.50|285.66|92.99| +2450993|48105|3674|4276|1879261|7046|22431|32094|126553|7009|46618|2|102|6|2|13|954|16|125.44|8.78|134.22|68.14|49.44|11.28|35.38|78.78|126.36| +2450889|56253|7909|4276|1879261|7046|22431|50275|512476|5093|23710|2|105|1|3|20|954|13|25.35|0.76|26.11|44.48|4.16|0.00|5.83|19.52|49.40| +2450991|49451|1582|26461|274540|6972|47404|26461|274540|7018|47404|4|79|18|3|20|957|60|70.20|0.70|70.90|27.55|125.40|61.07|3.28|5.85|153.65| +2451000|65459|11678|50487|1668416|5192|21121|50487|1668416|4457|21121|2|102|18|5|32|961|8|1965.92|157.27|2123.19|67.50|83.60|1710.35|163.56|92.01|308.37| +2451090|56758|8497|50487|1668416|5192|21121|84580|1337977|3569|17161|2|28|10|2|24|961|38|389.12|11.67|400.79|63.84|549.48|276.27|59.81|53.04|624.99| +2451013|67366|9182|21451||1523|29959||69524||15885|4|||4|33|963||59.71||||25.69|||0.00|113.29| +2450926|75754|11332|86824|1604453|3682|39711|38192|1119666|4392|46318|2|65|16|2|23|964|6|27.60|2.20|29.80|84.60|169.02|19.04|4.45|4.11|255.82| +2450996|43168|2077|86824|1604453|3682|39711|5234|812671|668|5912|2|69|4|3|12|964|36|407.52|16.30|423.82|31.54|162.72|354.54|3.17|49.81|210.56| +2451044|10338|12139|86824|1604453|3682|39711|86824|1604453|4421|39711|2|83|5|1|20|964|49|1742.93|52.28|1795.21|55.14|657.09|139.43|1090.38|513.12|764.51| +2451016|21560|1354|52574|1806862|4350|19791|49455|1076635|6558|13234|2|74|16|2|25|965|3|230.61|16.14|246.75|77.44|225.99|16.14|197.31|17.16|319.57| +2450877|1422|4766|91478|179235|6639|29243|5674|206864|5494|16477|1|62|20|5|5|967|33|1009.14|80.73|1089.87|94.68|170.61|645.84|123.52|239.78|346.02| +2450981|47884|5398|21984|281933|7129|6678|21984|281933|98|6678|1|35|3|4|17|968|20|115.20|10.36|125.56|72.48|106.40|86.40|23.90|4.90|189.24| +2450960|63873|1030|3179|1353628|877|20526|37656|1864729|4474|41819|4|4|14|1|21|970|29|3270.62|130.82|3401.44|21.28|557.38|1635.31|572.35|1062.96|709.48| +2450944|80798|16021|45847|1333790||26076|40893|1492006|6113|||||1||972|25|2388.75||2508.18||322.75|453.86|||497.92| +2450982|76284|12632|45847|1333790|133|26076|12399|957045|4607|46169|4|9|8|5|10|972|3|6.15|0.49|6.64|11.08|265.11|3.87|1.82|0.46|276.68| +2451041|17273|13510|6201|1886344|4887|17030|33583|292941|1823|31905|2|97|18|5|27|973|65|3540.55|106.21|3646.76|20.02|2057.25|1734.86|1227.86|577.83|2183.48| +2450949|71748|6062|6201|1886344|4887|17030|6201|1886344|1131|17030|2|86|9|2|10|973|44|1544.40|123.55|1667.95|2.32|1544.40|1127.41|216.83|200.16|1670.27| +2450946|63722|3667|6201|1886344|4887|17030|49878|1315612|1168|38324|2|103|12|3|4|973|29|1301.81|13.01|1314.82|97.07|325.38|755.04|535.83|10.94|435.46| +2451018|43315|11728|6510|1803602|5941|33833|20354|544041|6909|39330|2|70|14|5|13|975|3|131.64|5.26|136.90|50.18|12.51|42.12|64.45|25.07|67.95| +2451029|64186|12394|6397|349204|6411|32065|68659|6126|4105|21740|4|1|3|4|35|976|8|477.60|38.20|515.80|1.24|73.44|410.73|11.36|55.51|112.88| +2450932|12184|15424|6397|349204|6411|32065|68659|6126|920|21740|4|62|4|5|31|976|3|39.72|3.17|42.89|2.24|43.59|9.13|11.01|19.58|49.00| +2450979|47535|6218|53917|1087231|6945|12659|98062|1279291|360|17382|4|83|2|4|3|977|61|2449.76|122.48|2572.24|48.49|3292.17|1788.32|377.02|284.42|3463.14| +2450947|4259|15022|69759|247101|2043|185|69759|247101|3390|185|1|28|8|1|4|979|66|7495.62|299.82|7795.44|55.33|2950.20|6446.23|996.92|52.47|3305.35| +2451022|20079|493|64942|1168988|3434|25880|44279|770464|5541|11771|4|56|8|3|5|980|1|77.81|3.89|81.70|48.34|70.61|3.11|15.68|59.02|122.84| +2450901|66210|2827|27283|901789|2695|14801|46009|715061|2843|672|4|6|5|4|21|981|1|25.41|1.01|26.42|80.79|20.45|15.75|3.09|6.57|102.25| +2450958|68800|17302|12753|284271|1742|43260|44828|85516|4017|20118|1|82|8|4|5|982|11|1979.01|178.11|2157.12|32.59|842.16|79.16|151.98|1747.87|1052.86| +2450881|5685|307|16675|857649|1023|18444|49361|163869|876|27483|4|73|4|1|26|987|20|17.80|0.17|17.97|94.49|151.20|5.87|10.61|1.32|245.86| +2450899|16116|17828|14317|542556|4748|8187|91278|970195|7168|47341|1|79|7|5|3|988|23|847.78|16.95|864.73|77.17|258.06|50.86|462.21|334.71|352.18| +2450935|33775|10855|83215|1020606|4844|22635|83215|1020606|1376|22635|1|72|15|3|31|989|11|29.92|0.00|29.92|1.85|26.62|8.07|17.48|4.37|28.47| +2450941|27056|1064|92843|1330107|6571|40682|35038|991275|4778|43081|4|89|10|3|26|990|55|603.35|6.03|609.38|17.11|603.35|156.87|62.50|383.98|626.49| +2450956|46043|12889|92843|1330107|6571|40682|46520|1244505|5583|2261|4|108|8|1|22|990|26|678.86|13.57|692.43|89.98|699.92|386.95|5.83|286.08|803.47| +2450978|25978|7912|90829|375387|2345|4903|98012|1184434|6021|34808|2|66|9|1|13|992|33|2360.49|188.83|2549.32|2.57|753.39|896.98|1141.53|321.98|944.79| +2450962|41037|10264|99324|336044|3964|14284|26265|529619|6028|17284|1|84|18|4|23|993|77|5992.91|59.92|6052.83|89.18|2724.26|4854.25|79.70|1058.96|2873.36| +2451074|62359|13292|99324|336044|3964|14284|40254|488035|6093|19751|1|31|19|2|12|993|12|2488.32|124.41|2612.73|17.52|253.92|1692.05|660.90|135.37|395.85| +2450953|41194|14966|99324|336044|3964|14284|11298|1715081|6009|9690|1|82|14|5|23|993|33|768.90|30.75|799.65|21.25|723.69|115.33|228.74|424.83|775.69| +2451004|41725|3331|99324|336044|3964|14284|6903|1271684|631|15225|1|76|7|1|35|993|50|4546.50|318.25|4864.75|73.82|203.50|1636.74|2531.49|378.27|595.57| +2450973|71667|7030|24802|96704|3006|14061|8980|1688507|872|884|2|3|19|1|11|994|75|10272.00|513.60|10785.60|92.48|0.00|7190.40|215.71|2865.89|606.08| +2450938|68531|7124|22079|1396025|1971|317|20330|622918|1118|42518|1|22|18|5|10|995|5|141.35|0.00|141.35|7.87|14.95|45.23|47.09|49.03|22.82| +2451017|47719|6637|53751|1649527|3350|41173|37178|1582772|507|1971|4|56|18|4|27|996|52|4728.36|94.56|4822.92|96.81|1454.44|2742.44|1350.42|635.50|1645.81| +2450975|80843|7684|61995|1525377|4312|17694|68310|124053|7104|36508|1|14|1|5|11|997|16|1391.04|0.00|1391.04|97.90|894.24|236.47|865.92|288.65|992.14| +2450859|31165|9106|54496|1877464|7185|46506|22197|1460911|514|48470|2|69|18|4|27|998|63|23.94|1.19|25.13|75.92|163.17|10.29|8.73|4.92|240.28| +2451033|50631|464|54496|1877464|7185|46506|54496|1877464|4478|46506|2|7|11|1|21|998|2|192.58|7.70|200.28|23.65|20.48|48.14|131.44|13.00|51.83| +2450855|5299|10906|68957|1593114|6518|34275|66099|219113|3831|45749|1|6|10|3|8|1000|25|268.75|8.06|276.81|25.50|38.25|104.81|154.10|9.84|71.81| +2450956|67177|15320|46333|755203|3273|36143|40604|1592015|7120|35072|1|81|1|3|10|1001|26|576.42|5.76|582.18|64.43|515.58|432.31|27.38|116.73|585.77| +2450936|61223|4930|46333|755203|3273|36143|23996|1564535|5225|16268|1|38|18|3|26|1001|1|30.07|1.20|31.27|83.89|9.12|15.63|6.78|7.66|94.21| +2450978|46322|730|46333|755203|3273|36143|6804|1196502|5732|4962|1|93|5|5|5|1001|3|191.19|1.91|193.10|68.64|253.20|61.18|58.50|71.51|323.75| +2450994|32091|3002|15695|1655190|1085|45380|71188|1279333|3735|22090|1|44|2|4|9|1002|20|978.80|29.36|1008.16|59.47|484.40|332.79|613.70|32.31|573.23| +2450892|32195|6856|47003|1878148|2893|40038|83049|847984|2226|26821|4|65|20|5|35|1009|6|859.20|34.36|893.56|92.52|368.22|489.74|136.70|232.76|495.10| +2451032|273|12250|45781|1912573|2565|49357|38541|181958|607|44456|2|94|10|4|8|1014|27|3511.35|105.34|3616.69|33.43|330.48|2071.69|345.51|1094.15|469.25| +2450978|64717|9469|19461|840029|6197|15965|43736|1820576|4250|39971|4|13|14|5|1|1015|20|1261.80|75.70|1337.50|61.80|822.80|1236.56|14.13|11.11|960.30| +2451036|63163|3142|40359|1736021|6530|42639|38143|1437091|4863|40235|2|14|11|3|5|1016|54|396.36|19.81|416.17|5.61|7.56|166.47|186.21|43.68|32.98| +2450986|70150|7832|40359|1736021|6530|42639|79225|1746129|1641|35153|2|41|16|3|15|1016|16|266.24|23.96|290.20|79.54|163.84|133.12|58.57|74.55|267.34| +2450981|49294|5725|40408|1012152|4696|46393|38418|1162369|5005|21641|4|22|9|5|26|1018|43|964.92|86.84|1051.76|0.54|1236.68|0.00|337.72|627.20|1324.06| +2450935|76513|17884|40408|1012152|4696|46393|61224|1067188|2957|11973|4|94|7|1|5|1018|5|24.85|0.24|25.09|99.66|34.15|9.94|5.66|9.25|134.05| +2451054|57401|982|52582|1785501|6067|46586|67391|62588|2626|43864|4|16|18|1|7|1019|8|1057.60|31.72|1089.32|24.35|252.56|243.24|464.18|350.18|308.63| +2451056|20135|10420|53761|1644698|1235|42619|14729|1843263|2738|10966|1|78|4|4|27|1020|1|43.41|3.90|47.31|44.92|17.22|38.20|2.29|2.92|66.04| +2451087|26170|3703|53761|1644698|1235|42619|53761|1644698|1664|42619|1|82|2|3|9|1020|10|379.90|22.79|402.69|63.94|85.60|159.55|63.90|156.45|172.33| +2450948|71789|2336|53761|1644698|1235|42619|8778|382126|2655|33984|1|34|4|3|15|1020|51|45.90|3.21|49.11|4.12|18.36|38.09|4.76|3.05|25.69| +2451014|66422|16942|46323|1569350|2777|20204|46323|1569350|6512|20204|4|62|10|3|14|1022|7|367.43|25.72|393.15|63.88|72.17|128.60|224.50|14.33|161.77| +2450936|45290|2458|65503|581078|7053|10228|62777|1195490|391|32092|2|108|15|1|2|1024|22|3412.20|136.48|3548.68|97.19|1226.72|1126.02|914.47|1371.71|1460.39| +2451052|14878|13099|57415|223727|5592|25104|29047|178945|2338|12131|1|25|4|2|7|1025|9|249.48|0.00|249.48|0.98|92.61|244.49|1.29|3.70|93.59| +2450977|52800|13948|76242|354452|4215|32025|66335|1825247|5468|35242|2|76|11|2|21|1026|38|553.28|11.06|564.34|11.62|234.08|88.52|357.86|106.90|256.76| +2450992|45455|350|88480|950000|4767|18334|14527|376103|6687|32071|2|72|3|2|21|1027|4|438.60|39.47|478.07|27.29|290.36|153.51|105.48|179.61|357.12| +2450940|82456|9301|88480|950000|4767|18334|62783|1570565|1189|966|2|107|18|2|29|1027|48|3563.04|178.15|3741.19|13.91|1303.20|890.76|1255.97|1416.31|1495.26| +2451028|23499|11941|45378|1231749|1320|4491|6257|1264672|2856|6640|1|70|5|4|23|1028|14|1158.08|104.22|1262.30|62.21|42.84|359.00|399.54|399.54|209.27| +2450919|45374|11888|73862|1133402|2542|18082|3148|658359|6400|32041|1|55|4|5|16|1029|44|1257.08|0.00|1257.08|6.80|314.16|1055.94|42.23|158.91|320.96| +2450960|76370|10292|44261|1913461|2308|39908|44261|1913461|4294|39908|1|68|5|1|17|1030|39|327.60|3.27|330.87|88.25|209.43|212.94|27.51|87.15|300.95| +2451082|46876|7576|44261|1913461|2308|39908|4140|567245|344|27043|1|77|20|4|28|1030|7|667.59|46.73|714.32|78.78|240.31|100.13|289.40|278.06|365.82| +2450989|79963|11641|12863|30717|4571|32425|64662|785495|5014|19780|4|75|4|3|27|1031|18|142.20|9.95|152.15|73.25|38.70|142.20|0.00|0.00|121.90| +2451037|75080|17396|237|916222|5807|33345|17371|1166389|6072|4508|4|46|3|1|17|1034|6|728.34|58.26|786.60|6.91|448.20|386.02|130.08|212.24|513.37| +2450915|82257|6907|61384|1868737|1243|18149|7136|1504217|1304|21991|4|90|12|4|1|1035|59|3513.45|316.21|3829.66|67.06|1391.81|3021.56|427.94|63.95|1775.08| +2450878|67157|7550|61384|1868737|1243|18149|97668|334294|4554|5661|4|44|1|1|31|1035|11|1173.04|23.46|1196.50|36.50|395.89|1173.04|0.00|0.00|455.85| +2450921|58913|673|61384|1868737|1243|18149|73000|||11837||80||||1035|82||38.90||90.34|1028.28|155.63|2801.45|933.82|| +2450855|72879|16807|61384|1868737|1243|18149|91426|1883765|1552|22586|4|43|8|5|14|1035|25|1948.00|97.40|2045.40|79.97|1066.75|331.16|1228.79|388.05|1244.12| +2450893|62511|15586|8354|1262564|1127|36137|39109|1277689|3454|47619|2|23|14|4|14|1038|8|674.48|6.74|681.22|52.06|50.88|586.79|18.41|69.28|109.68| +2450991|12542|9343||1262564||||1613310|||2|21||||1038||||747.66||||160.74||| +2451015|28314|16592|||2764||||2346|10203|2||13||8|1040||||530.50|80.02|889.02|||353.13|974.29| +2450962|14155|7628|||2764||||3906|||||1||1040|12|347.64|13.90||6.93|236.40|20.85|179.73|147.06|| +2451027|73425|787|10228|1064878|1481|13731|59089|1038165|6893|17974|2|66|4|5|11|1041|2|63.86|5.74|69.60|12.70|18.12|60.66|1.63|1.57|36.56| +2450932|62263|9058|10228|1064878|1481|13731|87646|151579|2974|17522|2|19|13|3|11|1041|19|193.80|15.50|209.30|64.33|1135.63|34.88|66.74|92.18|1215.46| +2450987|66419|2800|10228|1064878|1481|13731|69913|236109|959|43856|2|88|20|5|17|1041|61|597.80|47.82|645.62|83.77|4270.00|400.52|53.26|144.02|4401.59| +2450988|43356|1942|10228|1064878|1481|13731|70443|276705|3387|38422|2|4|14|1|26|1041|60|5889.00|58.89|5947.89|50.69|2218.80|1884.48|360.40|3644.12|2328.38| +2450990|30759|17404|80548|1039723|6451|33226|80548|1039723|5723|33226|4|88|2|2|20|1042|8|126.00|6.30|132.30|49.95|252.00|108.36|7.58|10.06|308.25| +2450967|26317|9902|76564|1326909|3301|30093|54704|636024|3051|32078|2|27|6|3|27|1043|12|108.00|2.16|110.16|35.16|63.24|45.36|25.05|37.59|100.56| +2450942|46677|5540|23830|794912|3241|17523|19489|47468|747|12873|1|33|4|5|19|1044|6|447.30|8.94|456.24|25.13|379.80|178.92|21.47|246.91|413.87| +2450948|10350|7444|23830|794912|3241|17523|8366|1637218|1854|27337|1|107|6|2|35|1044|26|2166.32|0.00|2166.32|63.46|1743.56|649.89|318.45|1197.98|1807.02| +2450886|40623|6500|98695|1603663|3031|49552|54698|459260|5788|25069|2|48|6|4|15|1045|4|632.04|6.32|638.36|10.54|57.44|6.32|481.80|143.92|74.30| +2450877|49111|16915|98695|1603663|3031|49552|52988|406457|625|19193|2|24|4|2|10|1045|2|4.66|0.23|4.89|39.70|81.76|4.47|0.03|0.16|121.69| +2451021|54331|11396|40849|103255|4414|45620|55179|937154|3104|32607|1|33|3|1|11|1046|43|6290.04|0.00|6290.04|45.73|967.50|2201.51|1390.10|2698.43|1013.23| +2450879|45829|9802|17327|1775401|1922|28796|31867|526343|3945|42700|2|95|12|2|22|1047|11|30.80|2.46|33.26|76.88|94.93|27.41|1.38|2.01|174.27| +2451010|68790|5887|17327|1775401|1922|28796|82623|1671373|1887|32801|2|21|16|5|24|1047|6|196.86|7.87|204.73|54.37|309.42|17.71|73.45|105.70|371.66| +2451076|6757|14137|35888|1091991|2494|3750|72956|1682401|6918|46635|4|32|13|5|22|1048|15|471.60|4.71|476.31|61.45|141.45|198.07|268.05|5.48|207.61| +2450871|45232|182|35888|1091991|2494|3750|92810|1720649|5307|43965|4|60|12|5|26|1048|29|2700.77|81.02|2781.79|49.33|119.77|1080.30|1231.55|388.92|250.12| +2451042|40476|10168|21094|1143823|6552|27681|30590|997362|802|11842|2|79|2|2|2|1051|16|16.64|0.99|17.63|90.16|3.52|4.16|3.86|8.62|94.67| +2451051|1690|9490|23194|847605|3569|17980|9629|729392|2156|3320|2|78|19|3|7|1054|44|2407.24|216.65|2623.89|88.15|579.48|1636.92|670.17|100.15|884.28| +2450903|36337|14984|23194|847605|3569|17980|47565|1209095|5037|23426|2|54|6|5|4|1054|33|1863.18|18.63|1881.81|62.57|1621.62|1043.38|606.65|213.15|1702.82| +2450991|59801|5113|45274|1477875|4876|35077|30842|1467870|670|38670|2|91|17|2|27|1058|6|95.94|8.63|104.57|88.75|186.60|37.41|12.29|46.24|283.98| +2450904|58689|5785|45274|1477875|4876|35077|60437|1505636|11|49939|2|69|17|5|10|1058|32|1104.00|44.16|1148.16|66.51|1283.84|386.40|409.03|308.57|1394.51| +2451016|80246|15049|24379|478983|4787|28991|55251|107328|5501|25274|2|41|13|1|4|1059|23|780.39|7.80|788.19|72.66|198.03|554.07|101.84|124.48|278.49| +2450990|74314|2134|24379|478983|4787|28991|60835|266780|3279|28131|2|63|14|1|20|1059|36|2755.44|137.77|2893.21|74.05|1937.52|2535.00|55.11|165.33|2149.34| +2450850|69483|9248|59489|895759|29|11499|92767|1313026|2910|5194|2|77|19|3|7|1061|28|444.08|13.32|457.40|67.41|444.08|395.23|39.08|9.77|524.81| +2451067|65844|14281||||669|36995|1056055|||||14||33|1064|||1.85|187.77||0.00|||0.00|| +2450924|21472|13132|82074|771849|3671|30646|30946|1320813|1386|25512|1|71|4|2|30|1065|27|2278.26|205.04|2483.30|16.37|730.08|1389.73|888.53|0.00|951.49| +2450994|43343|4705|82074|771849|3671|30646|60480|1552758|7|22809|1|106|2|1|28|1065|31|846.61|59.26|905.87|70.22|470.27|347.11|404.59|94.91|599.75| +2451002|68834|1340|47904|987188|5672|23756|78732|464768|4692|38216|1|86|1|5|9|1066|32|2448.96|195.91|2644.87|81.61|791.04|2106.10|318.85|24.01|1068.56| +2451063|9707|529|81733|56110|6487|32064|87962|1482697|992|10087|4|63|12|2|27|1067|2|108.52|9.76|118.28|94.28|49.50|13.02|57.30|38.20|153.54| +2451050|20393|7435|81733|56110|6487|32064|94155|1388607|7076|3545|4|15|5|2|1|1067|32|193.28|15.46|208.74|78.22|26.56|69.58|68.03|55.67|120.24| +2451000|75747|8402|95676|605596|1116|32636|68629|210251|6749|30400|4|5|7|5|18|1068|20|638.80|51.10|689.90|77.61|133.60|364.11|87.90|186.79|262.31| +2450973|57193|10132|25259|1782980|6654|23575|17040|1909422|5950|4925|1|21|16|4|14|1069|41|255.84|12.79|268.63|18.69|398.11|253.28|1.68|0.88|429.59| +2450962|47539|5275|49220|217873|479|6685|81578|1313576|86|29749|2|13|19|4|17|1070|67|1721.23|137.69|1858.92|74.65|2993.56|1566.31|27.88|127.04|3205.90| +2450928|41889|8152|24830|572201|1172|30143|61084|702982|1045|308|4|7|17|5|28|1072|28|2269.12|158.83|2427.95|63.97|1525.72|1974.13|138.64|156.35|1748.52| +2451035|59842|13724|24830|572201|1172|30143|54544|1550907|1798|31489|4|7|9|5|16|1072|2|21.62|1.29|22.91|1.58|5.40|6.26|11.05|4.31|8.27| +2450932|58661|17074|24830|572201|1172|30143|56907|601639|6332|31425|4|76|11|4|19|1072|35|801.15|48.06|849.21|42.59|961.45|576.82|199.65|24.68|1052.10| +2450965|31161|4069|63864|1624496|2281|9713|65161|998091|1134|18930|1|89|5|3|34|1073|33|0.00|0.00|0.00|97.81|1680.36|0.00|0.00|0.00|1778.17| +2451037|74754|10346|63864|1624496|2281|9713|5728|1559226|595|42317|1|16|3|2|34|1073|16|41.44|2.90|44.34|45.32|1.60|16.57|1.98|22.89|49.82| +2450919|56263|16570|43604|727574|4344|25867|40847|857558|4913|27021|2|55|12|5|5|1075|42|3879.12|232.74|4111.86|38.11|562.80|465.49|2594.35|819.28|833.65| +2450964|72967|10220|43604|727574|4344|25867|3472|1827238|4944|38203|2|3|20|4|8|1075|4|50.08|3.50|53.58|47.21|109.28|11.51|3.85|34.72|159.99| +2450997|61437|6433|28125|455482|4886|48690|60020|682690|6594|46098|4|75|3|2|21|1080|35|59.50|3.57|63.07|27.36|89.25|54.74|2.52|2.24|120.18| +2451089|74439|11324|28125|455482|4886|48690|69493|1465575|6319|22310|4|9|9|3|9|1080|91|6219.85|310.99|6530.84|85.37|2892.89|6095.45|115.69|8.71|3289.25| +2450998|78578|1466|85391|1894708|5978|15295|85391|1894708|6195|15295|4|78|10|4|21|1083|1|103.30|3.09|106.39|38.93|40.58|102.26|0.73|0.31|82.60| +2451036|79869|2194|8559|726683|2054|45778|8559|726683|5412|45778|2|1|13|3|35|1086|3|60.18|3.61|63.79|29.99|19.14|26.47|9.10|24.61|52.74| +2451038|56703|2816|36220|32920|2067|7614|62503|50892|2465|4885|1|55|18|3|6|1088|34|743.58|22.30|765.88|3.77|285.26|431.27|28.10|284.21|311.33| +2450952|71960|2384|36220|32920|2067|7614|15382|436796|2147|45145|1|68|12|3|21|1088|12|129.12|7.74|136.86|83.42|59.64|67.14|8.67|53.31|150.80| +2451018|68596|16054|36220|32920|2067|7614|16380|80509|759|1856|1|18|13|3|5|1088|61|5585.77|223.43|5809.20|19.02|848.51|2625.31|2279.55|680.91|1090.96| +2451043|57318|6038|87119|502030|6894|25405|18242|1352596|3386|3423|1|51|20|5|30|1089|29|865.07|60.55|925.62|25.94|26.10|605.54|23.35|236.18|112.59| +2450954|78230|9830|87119|502030|6894|25405|57117|1844961|4803|15398|1|34|10|2|2|1089|68|1442.96|28.85|1471.81|53.19|1952.28|1341.95|76.76|24.25|2034.32| +2450908|54277|1300|31862|543632|3755|2725|16982|307934|996|22011|4|8|17|2|9|1091|35|3092.95|154.64|3247.59|75.17|924.70|2319.71|479.40|293.84|1154.51| +2451009|66072|2410|73449|1024495|2741|45787|50016|1156372|3082|43792|2|9|15|5|20|1092|52|608.40|18.25|626.65|38.10|254.28|346.78|75.86|185.76|310.63| +2451045|74673|9032|73449|1024495|2741|45787|72058|269534|1114|17094|2|38|14|4|12|1092|23|851.00|68.08|919.08|46.78|1063.98|587.19|21.10|242.71|1178.84| +2450941|25771|16714|73449|1024495|2741|45787|82371|1292324|4944|3964|2|104|4|4|19|1092|71|8912.63|267.37|9180.00|7.06|4455.96|8021.36|525.84|365.43|4730.39| +2450921|45483|6392|71742|1802546|903|18460|59588|100367|3225|46483|4|33|10|1|34|1093|21|428.40|0.00|428.40|3.56|9.45|8.56|146.94|272.90|13.01| +2450969|23274|16502|91603|720990|2824|13011|92798|1717258|2758|25458|1|22|5|5|5|1094|6|428.58|38.57|467.15|57.03|175.68|72.85|60.47|295.26|271.28| +2451050|77100|602|92607|229509|11|5666|73060|1127416|3416|18969|1|54|2|2|22|1097|11|16.28|1.46|17.74|4.21|3.30|3.90|4.20|8.18|8.97| +2450937|71517|5659|74072|863650|1675|2500|79825|895436|2376|916|2|5|3|3|18|1098|23|3480.82|34.80|3515.62|58.66|710.24|870.20|2114.60|496.02|803.70| +2450886|3532|8245|49556|431484|6659|38987|80789|626050|855|12507|4|6|9|4|1|1100|2|46.96|1.87|48.83|72.12|10.66|15.49|14.79|16.68|84.65| +2450922|79615|2443|49556|431484|6659|38987|5574|1344543|1895|33647|4|9|16|3|23|1100|3|9.03|0.81|9.84|76.33|72.30|1.35|5.14|2.54|149.44| +2450970|40552|5044|34406|13468|883|49145|27432|1059836|1113|21143|2|64|14|4|6|1102|23|1867.37|130.71|1998.08|80.53|488.98|952.35|256.20|658.82|700.22| +2451079|74409|8059|34406|13468|883|49145|5612|217843|4931|21666|2|83|17|5|20|1102|7|53.34|1.60|54.94|61.97|9.52|27.73|2.04|23.57|73.09| +2450853|40015|17770|57059|1858554|49|30697|33082|1279918|6977|34036|2|101|13|4|18|1103|19|380.38|7.60|387.98|69.56|31.35|300.50|12.78|67.10|108.51| +2451060|30851|10015|89040|150902|2811|25473|77817|1212640|5115|46390|4|96|9|4|26|1104|32|2043.84|183.94|2227.78|98.79|973.12|960.60|10.83|1072.41|1255.85| +2450982|67263|16406|79228|1004777|726|47475|79228|1004777|1883|47475|2|106|9|2|2|1105|3|83.85|4.19|88.04|49.03|40.41|45.27|32.40|6.18|93.63| +2451037|80186|16891|79228|1004777|726|47475|44511|393832|5560|36441|2|102|11|4|15|1105|24|41.76|2.92|44.68|7.55|816.72|26.72|0.30|14.74|827.19| +2450942|65900|8536|5130|1247646|2609|19125|58842|1862082|7121|46960|1|34|17|3|27|1107|17|0.00|0.00|0.00|97.67|335.41|0.00|0.00|0.00|433.08| +2451018|22650|7462|5836|1363181|5326|20671|6164|1108859|4313|4567|2|23|16|5|4|1108|21|366.87|11.00|377.87|35.01|64.68|183.43|97.22|86.22|110.69| +2450963|82716|15139|69892|1813816|5293|2370|19727|1871084|5731|45531|2|62|18|5|15|1110|16|452.48|27.14|479.62|2.73|140.32|294.11|152.03|6.34|170.19| +2451043|22671|4408|36556|1303125|5694|36710|26163|1794073|5208|33896|4|43|14|2|12|1112|11|87.45|5.24|92.69|56.51|71.17|83.07|3.28|1.10|132.92| +2451018|65651|16558|71630|1856196|1562|37328|25467|1911306|2166|34226|2|28|20|5|35|1113|23|1554.11|15.54|1569.65|2.84|660.79|1336.53|182.76|34.82|679.17| +2451032|44780|13712|20922|1340857|792|24727|74795|1218538|6620|45549|2|77|2|1|23|1115|71|286.13|14.30|300.43|22.01|386.24|111.59|89.01|85.53|422.55| +2450929|49291|13466|20922|1340857|792|24727|73353|171627|3504|22581|2|21|5|2|14|1115|18|204.84|12.29|217.13|41.51|93.96|53.25|139.46|12.13|147.76| +2450977|31589|13156|11358|309877|3149|21268|11358|309877|4384|21268|1|49|1|2|16|1116|21|61.74|2.46|64.20|37.84|68.25|39.51|6.66|15.57|108.55| +2450911|27094|7072|64168|1761588|3361|1581|43604|29679|6449|11219|4|40|16|1|21|1117|59|3316.39|33.16|3349.55|25.34|1752.89|1392.88|326.99|1596.52|1811.39| +2450979|33197|10162|99314|1676072|4830|48341|6909|1324461|3515|21769|1|14|17|1|10|1118|36|3558.96|142.35|3701.31|31.61|1933.20|1672.71|471.56|1414.69|2107.16| +2450981|60981|868|99314|1676072|4830|48341|31634|1873932|869|11806|1|31|4|1|33|1118|1|3.42|0.00|3.42|24.52|1.21|2.18|0.07|1.17|25.73| +2450984|78511|4849|91581|1783562|343|3971|84225|495790|4442|44341|4|103|18|5|34|1120|11|832.26|24.96|857.22|65.69|475.53|715.74|46.60|69.92|566.18| +2451049|25126|14276|17975|1430496|2664|44785|77795|677473|1464|3455|2|73|15|1|7|1122|7|30.24|1.20|31.44|77.57|31.99|12.70|16.83|0.71|110.76| +2450988|10006|7394|45617|81271|1725|42132|45617|81271|5452|42132|2|39|12|5|15|1123|25|112.75|7.89|120.64|82.00|135.25|47.35|39.89|25.51|225.14| +2450901|71956|17959|66169|1339116|5310|39104|66169|1339116|2187|39104|4|36|2|4|31|1124|8|333.28|16.66|349.94|35.41|140.72|259.95|59.39|13.94|192.79| +2450999|71551|1411|71325|970401|677|14153|47364|1379468|1936|30804|1|15|8|1|34|1125|1|104.28|3.12|107.40|10.87|97.77|49.01|38.68|16.59|111.76| +2451051|55672|11533|97677|760773|1469|11365|55488|404505|7092|27326|4|30|8|5|29|1127|9|513.45|35.94|549.39|28.22|74.61|456.97|7.34|49.14|138.77| +2451007|77789|15871|97677|760773|1469|11365|7250|674291|6553|26015|4|64|8|5|17|1127|11|1333.86|40.01|1373.87|44.58|613.47|867.00|406.16|60.70|698.06| +2450971|81975|9004|57452|1335134|463|18192|83246|1618850|2020|437|2|81|14|4|26|1131|34|3034.50|242.76|3277.26|81.18|812.60|2063.46|320.44|650.60|1136.54| +2451023|8517|9817|45620|1165605|7190|11928|52597|825417|4004|24238|4|82|6|3|22|1133|15|614.55|43.01|657.56|18.54|34.05|227.38|54.20|332.97|95.60| +2450966|32758|12886|87294|641840|1186|42132|24552|1419600|4973|22746|1|80|18|1|34|1134|4|531.68|21.26|552.94|13.36|113.48|398.76|9.30|123.62|148.10| +2450910|77925|3406|80933|1701305|2030|18402|46648|361777|2175|36973|2|48|20|3|16|1135|57|5743.89|516.95|6260.84|73.08|3257.55|4537.67|591.04|615.18|3847.58| +2451078|50706|15964|80933|1701305|2030|18402|11769|529922|6584|44093|2|94|19|1|23|1135|2|119.08|10.71|129.79|37.30|59.54|77.40|25.42|16.26|107.55| +2450999|32281|6314|80933|1701305|2030|18402|29860|1700686|7112|41538|2|38|16|5|21|1135|2|335.52|16.77|352.29|74.63|33.54|288.54|40.40|6.58|124.94| +2451011|27265|7150|3003|1128295|5234|18173|92272|1214004|5760|32635|4|43|11|2|23|1136|33|107.58|6.45|114.03|55.13|968.55|44.10|12.69|50.79|1030.13| +2450961|58833|15472|12443|1226745|5047|3349|19216|688488|2019|39197|1|9|17|3|19|1138|15|1920.00|115.20|2035.20|78.13|994.20|1171.20|321.98|426.82|1187.53| +2451079|49788|2404|12443|1226745|5047|3349|77932|266876|1594|7112|1|2|6|4|22|1138|17|447.44|17.89|465.33|2.61|1246.44|17.89|176.11|253.44|1266.94| +2451025|66799|6296|79238|1601|5211|774|82856|1481497|2023|12102|2|55|13|4|28|1139|74|12057.56|120.57|12178.13|71.21|5754.98|2532.08|5048.50|4476.98|5946.76| +2451019|17407|12344|79238|1601|5211|774|24809|1212951|6696|12888|2|42|17|2|9|1139|5|200.45|10.02|210.47|1.94|0.00|194.43|2.88|3.14|11.96| +2451036|51275|6346|58841|388062|4930|1751|49937|620233|5679|34325|1|28|3|4|5|1141|4|219.36|17.54|236.90|74.55|171.40|59.22|124.90|35.24|263.49| +2451016|73307|6268|58841|388062|4930|1751|24780|243350|3382|29014|1|39|3|5|22|1141|25|1728.75|51.86|1780.61|26.28|414.75|34.57|1067.33|626.85|492.89| +2451084|59744|13639|58841|388062||||561607|6254|31654|1||||28|1141|22|219.34|||35.60|5.50|||58.48|| +2451035|59894|9602|57821|884924|2874|21577|88946|885530|1445|48560|4|20|5|5|17|1143|41|2943.80|176.62|3120.42|16.01|1655.99|1501.33|1081.85|360.62|1848.62| +2450937|67317|16094|57821|884924|2874|21577|57821|884924|483|21577|4|62|14|5|1|1143|4|59.04|0.59|59.63|47.51|118.12|44.28|1.18|13.58|166.22| +2450912|66817|14738|48997|1187760|4923|25733|30375|350262|4867|43472|2|85|1|1|26|1144|32|3666.24|256.63|3922.87|3.80|2933.12|586.59|2094.16|985.49|3193.55| +2450992|67677|14840|87638|812929|4267|5558|89300|630098|5763|1828|4|9|3|3|28|1145|2|124.50|2.49|126.99|42.01|69.40|9.96|82.46|32.08|113.90| +2450876|64967|8563|87638|812929|4267|5558|46460|1451791|3792|11393|4|53|9|3|24|1145|32|2857.28|200.00|3057.28|41.50|2118.08|800.03|1069.77|987.48|2359.58| +2451016|28954|7690|89085|74150|4815|39880|56089|395104|3939|41574|2|13|19|4|24|1146|76|2553.60|178.75|2732.35|11.66|3346.28|740.54|725.22|1087.84|3536.69| +2450962|69439|9202|13346|717670|394|19118|78570|435745|4790|19435|1|39|17|2|17|1148|18|118.62|4.74|123.36|95.83|245.34|102.01|12.12|4.49|345.91| +2450937|68441|16111|34680|395209|2585|14314|34680|395209|1756|14314|1|106|7|3|9|1149|10|87.50|4.37|91.87|47.01|125.10|70.00|8.22|9.28|176.48| +2450956|55696|6937|63624|1051619|3167|20715|72763|803049|154|24106|1|28|8|3|18|1150|33|920.37|18.40|938.77|20.72|342.21|27.61|883.83|8.93|381.33| +2451035|62888|16978|26685|1138136|3187|22591|83853|298141|3494|38447|2|65|13|3|6|1152|55|3063.50|0.00|3063.50|1.65|293.70|980.32|1208.24|874.94|295.35| +2450965|34283|13555|26685||3187|22591|4769||2547||2|||1||1152|41|||||426.81|145.66|988.33|686.82|| +2451095|71382|8372|19720|237784|201|15018|54135|1864870|3244|36418|4|4|7|3|2|1153|38|41.04|1.23|42.27|33.66|1.14|9.02|13.44|18.58|36.03| +2450910|58560|4372|35488|560485|329|20526|30881|1435213|935|20257|4|41|16|1|8|1154|5|138.35|2.76|141.11|69.61|109.50|48.42|26.07|63.86|181.87| +2451015|6851|12154|81682|1461958|5081|14557|91162|1718109|5333|5804|2|40|3|2|31|1155|20|333.60|26.68|360.28|76.96|483.60|246.86|83.27|3.47|587.24| +2451067|21875|13190|27316|1107831|7099|34414|27316|1107831|4998|34414|2|104|17|5|8|1156|55|7194.55|287.78|7482.33|19.74|0.00|3093.65|410.09|3690.81|307.52| +2450894|76542|11893|97238|527357|3076|7995|80802|1711223|221|14419|4|14|13|1|24|1157|12|11.52|0.69|12.21|71.60|39.84|9.56|0.94|1.02|112.13| +2450906|65465|16177|90759|200103|3047|34391|42100|858117|3703|23689|4|31|20|1|20|1158|78|2113.02|63.39|2176.41|23.58|918.84|485.99|260.32|1366.71|1005.81| +2451057|23615|1322|90759|200103|3047|34391|24405|1481618|1752|32239|4|48|1|5|4|1158|13|0.00|0.00|0.00|50.83|145.08|0.00|0.00|0.00|195.91| +2451030|24489|17197|94573|271231|1489|42500|94573|271231|2269|42500|4|88|19|5|4|1159|15|165.45|3.30|168.75|96.43|294.30|153.86|7.30|4.29|394.03| +2450885|80918|7945|32375|4093|4611|43129|34560|301686|1883|33|2|14|17|2|4|1160|36|93.96|5.63|99.59|81.89|6.48|48.85|43.75|1.36|94.00| +2451002|35947|13456|32375|4093|4611|43129|68457|353257|2741|17425|2|90|3|2|34|1160|28|357.00|3.57|360.57|75.83|136.36|274.89|5.74|76.37|215.76| +2451007|41120|15688|3099|572726|1906|37533|65585|676467|3047|49227|1|81|7|2|22|1161|9|359.73|10.79|370.52|13.57|160.74|287.78|49.64|22.31|185.10| +2450982|68493|16298|3099|572726|1906|37533|56355|509591|2853|46515|1|66|1|3|5|1161|13|2383.42|23.83|2407.25|37.68|1175.33|1001.03|41.47|1340.92|1236.84| +2450956|82377|15892|3099|572726|1906|37533|82507|396909|297|36885|1|105|16|4|17|1161|8|92.48|4.62|97.10|59.80|85.60|49.93|0.42|42.13|150.02| +2451052|40811|17258|3837|1596225|4613|17401|34027|1145237|6881|125|2|73|17|1|30|1162|2|21.26|1.06|22.32|80.41|5.62|5.31|10.84|5.11|87.09| +2451033|11908|12454|||1369|15191|31084||||4|||2|28|1164|||38.29|1314.73|99.01||995.62|131.98|148.84|397.78| +2451039|49166|13094|54026|1018806|5539|45519|98609|735648|1024|46753|4|53|14|5|16|1166|20|522.60|36.58|559.18|32.95|107.20|397.17|76.51|48.92|176.73| +2450966|70941|10268|54026|1018806|5539|45519|54026|1018806|3655|45519|4|102|16|1|23|1166|24|717.60|21.52|739.12|34.47|287.04|86.11|378.89|252.60|343.03| +2450882|66064|2936|54026|1018806|5539|45519|63886|1020287|3831|39375|4|11|1|5|26|1166|64|9326.08|466.30|9792.38|73.10|942.08|746.08|2917.20|5662.80|1481.48| +2450940|15628|13472|74537|363780|1470|5913|1095|537075|3797|9719|4|20|12|4|1|1167|18|979.92|78.39|1058.31|87.45|257.22|725.14|81.52|173.26|423.06| +2450925|67675|7970|74537|363780|1470|5913|55409|26461|5897|16533|4|40|14|5|21|1167|16|768.16|15.36|783.52|31.43|880.48|253.49|77.20|437.47|927.27| +2450881|41878|9454|60710|888796|6129|9015|14897|681604|750|39096|4|53|14|5|21|1170|2|54.96|0.54|55.50|75.50|3.50|32.97|18.03|3.96|79.54| +2450945|47680|1694|58256|946792|6881|43417|98360|697416|4795|28927|4|78|10|2|29|1172|10|1030.30|51.51|1081.81|43.85|249.70|61.81|920.06|48.43|345.06| +2450863|35988|2024|46521|13834|3900|24766|96233|1316743|67|7391|2|31|16|5|14|1175|24|70.56|2.82|73.38|94.45|282.24|19.05|49.96|1.55|379.51| +2450996|20293|17839|53093|225027|2114|36848|11887|433400|4384|21321|1|32|6|5|14|1176|18|356.40|28.51|384.91|49.62|19.80|235.22|113.90|7.28|97.93| +2451030|16256|8906|53093|225027|2114|36848|40121|498090|1569|6648|1|70|18|2|11|1176|11|7.37|0.14|7.51|24.43|9.13|2.21|3.35|1.81|33.70| +2450973|71914|15380|56889|234668|4447|45826|44220|1108145|996|37800|1|53|6|2|21|1178|47|7566.53|302.66|7869.19|21.71|3004.24|2723.95|2711.84|2130.74|3328.61| +2451046|65134|17536|97004|566411|6218|39672|97004|566411|1020|39672|1|41|14|1|6|1179|2|0.00|0.00|0.00|50.40|150.68|0.00|0.00|0.00|201.08| +2450893|47850|10072|32539|42584|4629|27521|99288|1603963|2481|34199|2|29|20|3|26|1180|25|1018.50|71.29|1089.79|39.99|417.25|162.96|94.10|761.44|528.53| +2450905|75594|11438|32539|42584|4629|27521|86481|606411|5155|42948|2|83|4|3|5|1180|9|0.00|0.00|0.00|13.27|56.70|0.00|0.00|0.00|69.97| +2450900|68790|8731|4099|1420721|6943|33965|84430|1476562|1915|45221|1|4|6|4|20|1181|10|1122.50|56.12|1178.62|27.40|420.90|471.45|520.84|130.21|504.42| +2451074|33744|15560|28035|1700773|2311|44696|91137|1911674|4951|43828|4|22|19|1|1|1187|19|1664.97|133.19|1798.16|59.37|326.04|366.29|805.18|493.50|518.60| +2450985|46958|10994|86651|1455044|3884|677|31201|1237622|3557|23892|2|81|17|4|9|1188|16|843.84|50.63|894.47|25.69|1557.76|42.19|521.07|280.58|1634.08| +2451042|8520|2462|10067|585882|4097|39112|95117|1029125|2644|32963|4|97|13|2|26|1189|32|846.40|42.32|888.72|8.86|757.44|702.51|46.04|97.85|808.62| +2450978|33327|4064|10067|585882|4097|39112|34095|595785|6194|24650|4|38|17|1|30|1189|55|333.30|29.99|363.29|70.74|444.40|109.98|174.18|49.14|545.13| +2451013|38246|13630|46172|1171808|1444|37927|61885|367872|5122|325|4|93|1|1|4|1190|1|31.61|0.94|32.55|55.69|18.65|20.86|0.43|10.32|75.28| +2451013|66847|15784|19432|600466|3900|48134|91788|66222|26|32983|2|55|8|2|34|1191|14|299.32|8.97|308.29|8.95|149.66|59.86|189.17|50.29|167.58| +2450979|57081|17818|19432|600466|3900|48134|48154|1616279|6699|15072|2|77|5|3|18|1191|5|46.85|1.87|48.72|82.53|164.10|38.88|4.62|3.35|248.50| +2451042|69190|12118|29421|1901348|2428|755|9673|1398108|6210|6759|2|82|5|2|21|1192|19|660.25|52.82|713.07|12.18|571.33|250.89|32.74|376.62|636.33| +2450909|72770|16000|7836|1816693|3597|15701|873|280093|2178|36216|1|68|10|3|31|1193|9|23.40|2.10|25.50|15.69|90.27|6.31|9.05|8.04|108.06| +2450896|34347|17636|80413|837823|2880|25937|80413|837823|1503|25937|1|61|6|5|24|1194|80|11051.20|994.60|12045.80|60.26|4911.20|5415.08|3438.03|2198.09|5966.06| +2450969|33938|17830|34829|1872554|2279|20404|92344|1152227|335|22090|1|44|1|1|9|1195|44|3171.96|285.47|3457.43|98.93|731.72|380.63|2428.45|362.88|1116.12| +2450986|281|16124|34829|1872554|2279|20404|26769|1606385|4530|11075|1|1|3|3|3|1195|45|2122.20|0.00|2122.20|9.26|0.00|127.33|1735.53|259.34|9.26| +2450863|44911|1459|34829|1872554|2279|20404|71460|1461849|348|31647|1|17|17|1|10|1195|35|1106.70|0.00|1106.70|73.36|455.70|1062.43|5.31|38.96|529.06| +2451015|73387|277|75391|1908512|2880|13601|29203|1690737|3700|17463|4|85|12|5|13|1196|3|44.46|2.66|47.12|33.42|66.72|13.33|5.29|25.84|102.80| +2451088|44714|7772|75391|1908512|2880|13601|75391|1908512|4661|13601|4|59|6|2|25|1196|62|5124.92|358.74|5483.66|84.70|1295.80|2716.20|1397.05|1011.67|1739.24| +2451082|82302|2014|75391|1908512|2880|13601|75391|1908512|3741|13601|4|97|19|5|21|1196|17|544.34|48.99|593.33|93.86|916.81|59.87|329.43|155.04|1059.66| +2450993|27428|12931|76998|1784201|204|27360|3095|1453719|6439|22078|4|80|9|3|24|1197|3|0.00|0.00|0.00|75.32|4.71|0.00|0.00|0.00|80.03| +2450914|64095|7346|89292|1663986|6680|10221|28808|860353|342|48731|1|25|5|4|33|1198|2|64.24|0.64|64.88|60.04|14.26|10.27|11.33|42.64|74.94| +2451009|74746|5011|89292|1663986|6680|10221|97344|1034563|2204|39845|1|7|1|2|18|1198|46|677.58|60.98|738.56|4.99|1434.74|535.28|103.87|38.43|1500.71| +2450850|53739|13502|79472|223358|6444|6330|6450|1487587|5043|12899|4|39|20|1|2|1200|19|29.45|2.65|32.10|60.41|361.57|14.43|13.51|1.51|424.63| +2450989|44339|16882|8833|691839|1245|45451|8833|691839|2404|45451|4|106|12|2|29|1203|22|544.50|10.89|555.39|78.51|0.00|435.60|69.69|39.21|89.40| +2451040|78766|15952|8833|691839|1245|45451|84747|1267795|5065|21582|4|60|13|5|17|1203|4|64.88|5.19|70.07|79.90|239.64|62.28|0.59|2.01|324.73| +2450939|28582|2560|93564|1734798|32|24866|52259|105708|1979|17050|2|33|16|2|28|1204|26|359.84|0.00|359.84|90.35|428.48|226.69|70.56|62.59|518.83| +2450947|78472|9656|||32||||4238|32227|2|49|13|||1204|25|||4426.59|20.27|509.00|2440.83||898.98|818.86| +2451070|37238|2461|48817|589534|5098|40615|43820|86564|4930|36849|2|81|20|1|7|1206|14|37.24|3.35|40.59|87.94|111.72|6.70|20.15|10.39|203.01| +2450949|45997|1555|18426|153783|6740|4050|45536|1137571|5993|10862|2|102|2|4|33|1207|62|2862.54|114.50|2977.04|94.54|524.52|200.37|266.21|2395.96|733.56| +2451003|59572|116|34985|527286|2434|15182|9071|397565|5138|3111|1|14|4|2|17|1210|57|5559.78|222.39|5782.17|18.90|2849.43|3002.28|690.52|1866.98|3090.72| +2451048|47274|10732|34985|527286|2434|15182|95783|478615|1779|1790|1|74|3|4|23|1210|4|496.84|14.90|511.74|27.84|133.96|298.10|141.10|57.64|176.70| +2450999|60911|17623|93138|493433|2081|3576|67140|510384|830|31676|2|77|7|1|12|1211|28|1082.76|64.96|1147.72|32.18|183.68|887.86|107.19|87.71|280.82| +2450909|9398|7540|93138|493433|2081|3576|71438|1577329|1348|5814|2|37|18|2|21|1211|12|57.12|4.56|61.68|64.48|0.00|44.55|5.53|7.04|69.04| +2451003|25802|272|93138|493433|2081|3576|55044|1217573|1701|1876|2|12|14|4|19|1211|48|4854.72|436.92|5291.64|72.62|2157.60|2767.19|292.25|1795.28|2667.14| +2450936|67268|9188|60959|436229|4652|39875|44242|1754727|3410|48446|1|22|8|1|14|1214|74|96.94|0.96|97.90|76.65|535.76|77.55|11.44|7.95|613.37| +2450922|74554|13822|37048|1014828|161|7941|94960|116620|3151|8929|1|4|3|1|34|1215|24|1438.56|14.38|1452.94|31.56|561.84|848.75|406.96|182.85|607.78| +2450937|58691|17656|62831|1241586|6294|35802|62831|1241586|4094|35802|4|45|9|1|28|1216|20|2102.80|63.08|2165.88|59.48|402.00|1135.51|38.69|928.60|524.56| +2450969|68676|14954|98409|716056|1703|31350|54137|1510424|2230|17189|2|45|13|2|23|1217|9|781.11|70.29|851.40|80.99|691.02|335.87|151.38|293.86|842.30| +2450959|26570|517|98409|716056|1703|31350|13521|1340860|957|37101|2|93|19|1|20|1217|43|279.50|22.36|301.86|57.89|1071.56|231.98|47.52|0.00|1151.81| +2451035|37072|1606|28016|764596|5176|26203|95109|1196669|6611|41400|4|108|20|2|16|1218|60|6661.20|599.50|7260.70|8.22|1687.20|3263.98|203.83|3193.39|2294.92| +2450879|70345|6646|42388|657479|4217|42108|84269|1410072|1271|47223|2|12|6|3|15|1220|15|1759.05|87.95|1847.00|30.76|511.65|17.59|261.21|1480.25|630.36| +2450953|79100|13879|42388|657479|4217|42108|37756|575101|3981|16861|2|87|14|4|29|1220|5|20.40|1.83|22.23|82.97|163.55|6.93|6.60|6.87|248.35| +2451025|51151|17665|33246|513084|2969|6007|22304|440098|1977|1456|2|91|20|4|23|1224|21|1842.75|147.42|1990.17|92.09|1140.72|350.12|1223.95|268.68|1380.23| +2451006|73743|4670|58430|963774|7165|25356|8079|404015|5986|16710|4|57|2|1|25|1225|30|1231.20|12.31|1243.51|48.12|183.30|701.78|232.94|296.48|243.73| +2450994|71600|7219|58430|963774|7165|25356|2685|645498|1376|26901|4|104|2|5|10|1225|23|5891.68|58.91|5950.59|73.07|1761.34|4831.17|540.86|519.65|1893.32| +2451067|63448|10898|10719|718228|4490|48752|10719|718228|5750|48752|4|20|15|1|29|1226|2|239.62|4.79|244.41|67.61|157.24|112.62|34.29|92.71|229.64| +2451030|61008|10694|23437|356393|1349|41105|81765|1873924|1885|49846|4|16|9|5|9|1228|34|1006.40|30.19|1036.59|46.69|265.54|452.88|481.56|71.96|342.42| +2451035|44714|10357|58283|1867374|4112|47250|50882|716296|3151|33253|1|36|5|5|17|1229|72|8084.88|161.69|8246.57|60.61|8444.16|5416.86|1414.05|1253.97|8666.46| +2451026|38583|16003|58283|1867374|4112|47250|80771|608807|1714|32221|1|43|16|3|27|1229|91|9432.15|188.64|9620.79|74.53|4999.54|1226.17|6893.02|1312.96|5262.71| +2450899|30169|16324|19161|921780|2799|34665|65272|681209|3476|38496|4|86|3|3|35|1230|9|1286.28|51.45|1337.73|91.76|643.14|1093.33|123.48|69.47|786.35| +2450975|60721|2186|19161|921780|2799|34665|19161|921780|992|34665|4|45|2|3|17|1230|67|5037.73|403.01|5440.74|99.56|5037.73|3476.03|905.78|655.92|5540.30| +2450917|35810|2005|14196|1409023|6603|4630|14196|1409023|5968|4630|1|84|20|1|9|1231|23|1167.48|105.07|1272.55|86.01|259.44|186.79|902.23|78.46|450.52| +2450978|27740|8506|14196|1409023|6603|4630|8864|1562388|3314|33840|1|49|18|4|30|1231|41|150.88|10.56|161.44|50.91|70.11|104.10|9.35|37.43|131.58| +2451030|48011|2260|14196|1409023|6603|4630|14196|1409023|1363|4630|1|60|2|2|20|1231|37|2369.85|142.19|2512.04|48.19|666.37|900.54|1278.29|191.02|856.75| +2451023|56112|2984|64266|1799275|4684|8046|959|1405433|4077|40368|2|28|14|5|19|1233|5|598.05|35.88|633.93|43.16|299.00|568.14|12.56|17.35|378.04| +2450974|64911|7054|64266|1799275|4684|8046|29799|564066|6509|40505|2|21|7|5|31|1233|13|2002.52|0.00|2002.52|56.15|572.13|1161.46|302.78|538.28|628.28| +2451010|73565|1201|73876|822319|2762|17958|71513|255355|2132|20156|4|44|1|5|27|1234|12|148.44|8.90|157.34|28.57|556.80|62.34|34.44|51.66|594.27| +2451016|44845|2344|94976|1511195|4567|45186|36821|1474951|3630|22262|2|53|5|2|4|1236|31|2873.70|258.63|3132.33|9.29|651.31|344.84|480.48|2048.38|919.23| +2450952|71890|2330|42840|852368|2343|39471|69314|516435|407|9626|4|56|2|1|33|1237|15|2103.60|21.03|2124.63|14.60|435.15|1703.91|87.93|311.76|470.78| +2450862|3634|12346|42840|852368|2343|39471|3342|488842|771|49100|4|57|7|1|3|1237|19|1666.30|133.30|1799.60|95.10|584.63|1516.33|98.98|50.99|813.03| +2451050|70145|4418|44901|1831180|2612|30491|83934|31272|2011|9503|4|71|13|5|14|1239|1|2.74|0.24|2.98|73.68|13.74|0.73|0.98|1.03|87.66| +2450907|28913|17138|33000|246466|285|16504|8786|643751|5800|43049|2|16|9|1|15|1240|34|529.38|10.58|539.96|32.15|804.44|248.80|89.78|190.80|847.17| +2451010|74572|3505|83332|553796|3013|16277|45778|341744|4485|6826|2|36|2|1|2|1242|25|1948.00|19.48|1967.48|51.54|441.75|1051.92|53.76|842.32|512.77| +2450953|60946|12550|9692|1316718|897|47123|4874|869225|3217|30856|2|81|6|5|19|1244|17|527.17|5.27|532.44|38.44|366.18|26.35|65.10|435.72|409.89| +2451086|25153|4654|9692|1316718|897|47123|89462|977675|2677|4879|2|73|18|5|12|1244|17|214.88|6.44|221.32|1.45|2.38|30.08|121.96|62.84|10.27| +2450936|69259|14986|66300|426399|545|33444|92829|1654476|7115|27707|1|10|10|2|1|1245|28|1325.52|39.76|1365.28|24.47|773.08|861.58|83.50|380.44|837.31| +2450926|23905|2284|66300|426399|545|33444|66300|426399|4345|33444|1|83|16|2|23|1245|1|7.83|0.31|8.14|55.48|6.19|5.16|0.40|2.27|61.98| +2450928|31101|931|||3258|||1371620|4565|2310||||3||1247|||97.66|||||||| +2451020|6786|6292|89492|1150992|3245|10745|18756|1622402|895|41788|2|51|7|5|33|1248|6|197.28|17.75|215.03|68.54|34.62|142.04|32.59|22.65|120.91| +2450899|75150|7918|13402|1531474|2346|18818|27189|541471|2621|2971|4|87|14|4|34|1254|37|1726.79|17.26|1744.05|50.86|62.16|932.46|492.48|301.85|130.28| +2451090|26682|15037|9998|1717443|3735|49580|92284|767210|555|1947|1|97|3|3|15|1255|6|243.72|2.43|246.15|84.64|87.00|70.67|100.36|72.69|174.07| +2450933|11260|6130|18733|1702429|4878|21727|87248|1208837|3888|2939|1|106|19|5|1|1256|12|1342.92|120.86|1463.78|44.75|584.76|1020.61|54.79|267.52|750.37| +2450957|73654|7621|25596|923164|379|21994|59141|1197022|1852|19082|4|103|12|2|22|1257|53|4951.79|396.14|5347.93|84.98|449.97|4407.09|457.54|87.16|931.09| +2450937|56168|7876|41614|1401624|5987|15197|30696|847329|3611|42693|1|55|19|1|28|1263|36|494.28|24.71|518.99|62.41|1424.88|400.36|29.11|64.81|1512.00| +2451002|68957|4333|49526|700262|3798|20134|89342|481569|4757|20345|4|34|2|5|3|1265|1|22.83|0.22|23.05|37.03|13.04|7.76|14.16|0.91|50.29| +2451051|61104|12472|49526|700262|3798|20134|49526|700262|2358|20134|4|65|6|3|25|1265|2|65.40|4.57|69.97|74.86|52.32|25.50|13.16|26.74|131.75| +2451028|69150|1021|73684|67357|5477|15150|73276|757544|4408|15719|1|47|5|4|15|1266|29|283.33|2.83|286.16|24.25|566.66|215.33|22.44|45.56|593.74| +2450949|80839|7190|21175|489957|4339|9921|91570|1409821|5570|44566|1|45|13|2|1|1267|30|0.00|0.00|0.00|32.60|188.10|0.00|0.00|0.00|220.70| +2450988|73331|2863|21175|489957|4339|9921|50876|28721|5504|18334|1|69|10|5|3|1267|19|526.68|42.13|568.81|87.20|189.43|31.60|133.67|361.41|318.76| +2451047|40349|10321|21175|489957|4339|9921|23550|868625|1354|23330|1|23|4|5|19|1267|22|1156.98|57.84|1214.82|34.83|2142.58|404.94|293.29|458.75|2235.25| +2450935|72919|9164|55928|1498390|4052|30067|27862|1313485|979|5273|4|23|15|4|33|1268|2|37.66|2.25|39.91|88.53|28.80|9.03|21.75|6.88|119.58| +2451007|76081|7526|55928|1498390|4052|30067|69876|530632|995|48911|4|78|13|1|22|1268|29|3.48|0.31|3.79|42.53|39.44|2.12|0.61|0.75|82.28| +2451012|54011|6244|57862|810046|5430|49167|5817|867701|2120|24186|1|64|10|4|5|1269|6|181.86|0.00|181.86|98.40|38.94|129.12|12.13|40.61|137.34| +2451049|75174|17480|57862|810046|5430|49167|99820|813012|385|46558|1|45|4|2|5|1269|27|2177.28|130.63|2307.91|28.48|674.46|1153.95|767.49|255.84|833.57| +2451063|39123|8299|57862|810046|5430|49167|||||1|102|||24|1269|||2.15||||90.59|||1297.41| +2450978|11000|12860|57862|810046|5430|49167|17958|828099|842|3185|1|76|1|4|7|1269|3|0.00|0.00|0.00|45.32|130.68|0.00|0.00|0.00|176.00| +2450952|23935|5350|98729|257727|4372|2170|70855|574794|3962|7483|4|105|6|2|18|1273|1|26.98|1.88|28.86|21.80|21.46|8.09|2.45|16.44|45.14| +2450944|38877|9595|76862|694508|601|37847|96465|655470|6810|26950|2|71|10|5|25|1275|7|270.41|13.52|283.93|8.94|122.64|37.85|186.04|46.52|145.10| +2450919|45927|12289|45933|1257095||33866|||6386|||||5||1278|2||5.94|124.92||40.36||6.90||| +2451013|29816|14839|54955|1135706|4457|22808|19635|621472|7101|41577|1|17|18|4|23|1279|41|7842.48|235.27|8077.75|96.69|2546.10|5568.16|1410.07|864.25|2878.06| +2451037|59148|6085|2290|1823495|5947|44752|2290|1823495|3369|44752|4|28|18|2|26|1281|1|53.63|3.21|56.84|84.08|41.90|40.22|5.36|8.05|129.19| +2450952|71897|17965|66616|2269|2275|13528|80587|267964|6375|30386|1|77|19|5|11|1283|66|2192.52|0.00|2192.52|74.51|3253.80|1929.41|155.23|107.88|3328.31| +2451028|43228|12922|66616|2269|2275|13528|47888|1528865|6482|32899|1|16|17|4|31|1283|24|892.08|8.92|901.00|59.42|611.04|330.06|376.55|185.47|679.38| +2451040|44011|6674|73655|1436656|4457|9058|33207|1669893|4930|44014|4|21|6|5|14|1284|6|451.86|13.55|465.41|6.82|193.62|275.63|135.69|40.54|213.99| +2451000|49299|1171|3838|1745443|5095|3547|3838|1745443|661|3547|4|29|9|1|23|1287|34|140.42|9.82|150.24|21.37|120.36|63.18|19.31|57.93|151.55| +2450897|41816|6673|46228|379645|2662|16079|11314|519458|7168|26478|1|67|15|5|32|1288|1|3.57|0.24|3.81|96.71|16.42|2.96|0.33|0.28|113.37| +2450986|69132|3760|39730|517825|2673|48321|26237|597051|1155|10506|1|97|12|3|19|1289|40|3574.40|107.23|3681.63|80.14|1307.60|1965.92|1222.44|386.04|1494.97| +2450944|19147|12278|39730|517825|2673|48321|20549|158431|3339|21847|1|58|4|5|33|1289|18|350.28|0.00|350.28|69.92|475.38|105.08|215.77|29.43|545.30| +2451065|16690|1759|72223|1505516|6009|43302|3051|912365|5756|49147|1|21|7|3|14|1290|12|400.80|12.02|412.82|43.84|116.88|84.16|25.33|291.31|172.74| +2450938|40919|6676|51579|246557||40207|77862|1882667||15113|1|||2|25|1293|17|||387.84|23.54|6.12||50.61||40.95| +2450958|34885|8270|48112|1579244|4776|21508|79624|1879574|5438|26455|4|36|13|5|14|1294|3|146.34|0.00|146.34|50.79|106.44|24.87|82.59|38.88|157.23| +2451062|74038|14762|||3844||91384||5054|44230|2||18|4|10|1295|||7.48|90.64|51.96|463.68|83.16|||523.12| +2450991|4455|1639|98395|1530896|3844|49145|11610|697912|7124|21136|2|108|19|3|35|1295|2|213.52|10.67|224.19|68.06|62.92|149.46|25.62|38.44|141.65| +2450984|26408|13588|||5668||14219|1552071|5174|4277|2|44|19|2||1296|||||||309.28|||| +2451044|56665|16744|96255|1536385|5668|23398|13658|222455|631|37960|2|47|8|4|28|1296|11|475.75|0.00|475.75|96.21|134.20|14.27|424.56|36.92|230.41| +2450983|66486|16844|66965|1041626|3840|24901|13946|662145|741|34537|1|69|5|1|22|1297|26|568.36|51.15|619.51|6.14|405.86|426.27|58.25|83.84|463.15| +2450908|80994|9560|41447|1911723|5199|31428|38748|897308|1061|40414|1|49|6|4|12|1299|40|380.40|19.02|399.42|82.41|111.60|213.02|130.55|36.83|213.03| +2450886|65708|8936|41447|1911723|5199|31428|41447|1911723|2284|31428|1|101|4|4|29|1299|95|6298.50|188.95|6487.45|47.75|524.40|818.80|273.98|5205.72|761.10| +2450915|74013|5995|41447|1911723|5199|31428|40046|1738859|2185|27111|1|33|9|5|31|1299|40|1047.60|83.80|1131.40|60.13|245.60|817.12|101.41|129.07|389.53| +2451068|62660|11527|69373|1518495|6817|47939|62962|140268|729|12743|2|12|6|5|21|1302|35|4899.65|244.98|5144.63|99.00|2624.65|4115.70|125.43|658.52|2968.63| +2451055|72141|8560||1518495|||14715|||21402|2|15||4||1302|||||83.61||25.30|2.27||| +2450942|45753|11224|69373|1518495|6817|47939|39101|524571|2242|45960|2|70|20|3|29|1302|2|91.20|8.20|99.40|1.10|67.80|17.32|14.03|59.85|77.10| +2451056|11315|4387|91399|1494734|5269|40251|83003|987430|397|29811|4|28|19|1|20|1303|1|34.99|1.04|36.03|61.66|63.63|10.84|11.83|12.32|126.33| +2450960|17853|1768|57457|376470|2683|32449|44508|221469|162|45702|1|9|1|4|19|1304|2|38.94|0.38|39.32|69.10|16.54|19.47|7.78|11.69|86.02| +2450962|26848|4864|57457|376470|2683|32449|79113|1768833|2214|33595|1|92|5|1|32|1304|7|423.78|0.00|423.78|99.43|105.91|381.40|14.40|27.98|205.34| +2450968|70083|12178|47937|548418|6499|40180|48356|72134|6161|20188|1|6|9|3|35|1305|74|1889.22|170.02|2059.24|65.25|1816.70|1813.65|17.38|58.19|2051.97| +2450927|70777|14131|47937|548418|6499|40180|92004|426857|4660|26063|1|3|12|5|18|1305|17|1676.20|0.00|1676.20|85.09|259.08|1139.81|16.09|520.30|344.17| +2451004|22203|8923|46048|448606|3964|41275|77504|1355295|2841|38787|1|9|18|4|10|1308|20|1326.40|13.26|1339.66|5.72|189.40|1220.28|43.50|62.62|208.38| +2450970|14575|7801|73832|185159|933|43460|32183|1387925|3824|7723|2|44|13|2|6|1309|39|1006.59|0.00|1006.59|72.75|3020.16|312.04|625.09|69.46|3092.91| +2451006|43230|8341|30864|891534|6251|32521|68031|989152|7015|34925|2|93|1|4|12|1310|13|2186.73|65.60|2252.33|3.35|766.48|1224.56|173.19|788.98|835.43| +2451003|29804|2348|47852|732228|5404|23380|42671|43394|3675|10356|2|8|12|3|27|1311|10|29.70|1.18|30.88|60.54|106.90|16.92|1.40|11.38|168.62| +2451068|49154|13868|1340|567001|7062|45076|33016|1077826|646|20266|1|59|13|5|10|1313|14|2404.08|120.20|2524.28|35.24|223.02|2187.71|45.43|170.94|378.46| +2450958|47982|3757|37666|1790910|4154|29751|2205|1629901|3591|26434|4|75|4|2|12|1316|41|347.27|20.83|368.10|46.65|215.66|222.25|32.50|92.52|283.14| +2450945|38561|7756|40690|1551176|862|12548|55838||||4|||||1317|||||||505.64|954.75||| +2451038|82782|7396|40690|1551176|862|12548|5182|1774740|3139|23938|4|91|15|4|10|1317|31|876.99|17.53|894.52|66.16|219.17|640.20|56.82|179.97|302.86| +2451049|28362|1174|14702|779215|6128|12641|66088|1861025|1019|3013|2|26|15|4|33|1318|1|241.94|0.00|241.94|21.80|22.92|215.32|23.42|3.20|44.72| +2451041|63765|6097|14702|779215|6128|12641|64335|1681739|5707|15628|2|83|12|3|1|1318|2|53.62|3.75|57.37|31.70|23.82|35.38|18.05|0.19|59.27| +2450876|67103|11590|19222|82919|6263|4407|63366|1203964|1731|5098|4|95|13|5|9|1319|20|1613.20|129.05|1742.25|82.78|881.60|258.11|840.15|514.94|1093.43| +2450930|21876|8257|19222|82919|6263|4407|92434|239557|7038|23672|4|26|1|2|6|1319|29|2527.35|126.36|2653.71|48.52|552.74|2021.88|207.24|298.23|727.62| +2451025|48952|7273|77344|811691|4885|32832|53792|151018|4345|30919|4|63|14|3|12|1320|21|623.07|18.69|641.76|1.70|74.13|517.14|72.03|33.90|94.52| +2450916|42798|5197|61624|1505919|1414|909|28954|1414333|128|40323|2|26|14|4|22|1321|19|2577.16|231.94|2809.10|26.17|604.39|180.40|1749.63|647.13|862.50| +2451054|82413|7132|61788|1122599|210|17821|47156|1040908|2562|32216|1|101|18|2|1|1322|13|1429.35|114.34|1543.69|81.40|459.94|185.81|795.86|447.68|655.68| +2451077|73265|5692|57915|451051|187|30523|63956|1309197|1083|35016|4|99|8|1|12|1326|19|106.78|0.00|106.78|68.26|1202.89|50.18|29.43|27.17|1271.15| +2451053|62990|12718|57915|451051|187|30523|17053|1076589|2950|6864|4|48|20|1|22|1326|26|888.68|26.66|915.34|99.99|308.88|302.15|222.88|363.65|435.53| +2450889|57787|7538|88040|902463|2039|12062|20939|1052111|6673|1988|2|21|14|1|3|1327|45|261.45|13.07|274.52|13.48|261.45|31.37|18.40|211.68|288.00| +2450980|32929|13262|16838|1711797|1956|40115|37668|341802|5776|32150|1|102|12|4|33|1328|14|1585.22|63.40|1648.62|37.22|1688.54|1315.73|264.10|5.39|1789.16| +2450892|50188|10465|77698|1288847|3996|30745|92327|1572735|3122|37556|4|16|11|5|29|1330|9|269.19|13.45|282.64|11.20|340.02|115.75|121.21|32.23|364.67| +2450956|63867|7222|91643|1601324|2205|23886|48182|5939|3566|17946|2|16|13|5|20|1331|29|3043.55|121.74|3165.29|1.83|708.76|1673.95|342.40|1027.20|832.33| +2450951|76377|12332|85338|1371209|7127|10198|63855|890691|3126|21561|2|86|6|1|11|1333|20|138.40|6.92|145.32|80.95|73.80|106.56|28.65|3.19|161.67| +2450922|38305|10724|85338|1371209|7127|10198|90965|148397|7152|4660|2|72|11|4|27|1333|32|6686.40|267.45|6953.85|20.72|1215.68|4279.29|120.35|2286.76|1503.85| +2451077|70283|7220|80309|205819|576|48086|29404|1450111|4330|29349|2|66|20|4|28|1335|22|317.02|19.02|336.04|63.89|152.68|177.53|108.80|30.69|235.59| +2450936|51019|17056|75917|114928|2386|1797|46250|50472|6407|25142|4|64|14|4|24|1336|55|7823.75|234.71|8058.46|40.35|105.60|6415.47|253.49|1154.79|380.66| +2450928|19306|5485|88232|1597817|6155|17293|41439|1039619|1804|10789|4|90|4|4|14|1340|18|311.94|28.07|340.01|75.47|119.34|237.07|17.96|56.91|222.88| +2450965|47498|5594|81349|1682534|4872|10|88361|1528265|6981|49344|1|2|12|3|6|1341|21|752.43|37.62|790.05|85.89|633.57|75.24|514.66|162.53|757.08| +2451035|67634|2143|||4872|10|40455|1876651||45936|1|96|14|5|6|1341|||0.00|488.05|48.14|3348.41|453.88||8.89|| +2450888|29188|14414|86310|25681|1852|25976|77526|552660|4469|10678|2|95|12|3|12|1342|70|1612.80|129.02|1741.82|23.78|94.50|354.81|490.61|767.38|247.30| +2450925|60470|17090|49147|1280351|5955|21559|90701|1166109|3155|26554|2|69|5|1|25|1344|85|43.35|1.30|44.65|97.26|61.20|37.71|2.48|3.16|159.76| +2450939|29667|8611|99355|635427|299|4818|88167|1803454|5489|4324|2|51|2|2|22|1345|66|247.50|2.47|249.97|81.56|432.96|143.55|83.16|20.79|516.99| +2451069|39427|10646|47978|693538|2298|3458|97170|68107|5554|43848|4|23|16|5|7|1346|37|152.44|12.19|164.63|18.85|167.61|112.80|6.73|32.91|198.65| +2450948|42560|1780|47504|1096808|4512|5816|47504|1096808|6963|5816|4|53|12|4|3|1347|17|516.46|25.82|542.28|56.81|258.23|154.93|350.68|10.85|340.86| +2451079|25358|5986|47504|1096808|4512|5816|47504|1096808|3143|5816|4|16|14|5|21|1347|19|330.79|29.77|360.56|81.81|485.83|320.86|9.83|0.10|597.41| +2450983|70743|16678|61505|647591|5677|18163|37399|1640387|4346|35952|2|9|10|1|27|1349|1|82.07|3.28|85.35|71.65|49.52|59.09|9.88|13.10|124.45| +2450942|38665|15632|61505|647591|5677|18163|17919|1166817|220|13223|2|58|19|2|30|1349|66|204.60|4.09|208.69|16.85|1845.36|59.33|126.38|18.89|1866.30| +2451067|61318|7495|61505|647591|5677|18163|86471|260240|5290|2340|2|27|3|3|28|1349|64|152.32|1.52|153.84|94.31|111.36|70.06|56.75|25.51|207.19| +2451085|15569|13078|3005|6891|3224|31510|9679|1715351|5650|45196|1|40|20|1|25|1350|12|1542.72|30.85|1573.57|11.82|917.28|401.10|742.05|399.57|959.95| +2451023|47013|9763|3005|6891|3224|31510|74026|1860140|2483|48073|1|33|9|2|25|1350|7|106.89|1.06|107.95|73.21|8.68|33.13|23.60|50.16|82.95| +2450867|45211|3394|22347|442043|139|20588|64439|1652739|6415|28538|2|59|13|1|29|1351|8|16.64|1.33|17.97|70.17|325.68|7.15|2.56|6.93|397.18| +2451021|72300|13916|45792|227919|1246|34573|6693|817455|2734|6413|1|46|10|4|23|1352|78|10305.36|824.42|11129.78|51.69|3820.44|9790.09|412.21|103.06|4696.55| +2451105|57569|7568|45792|227919|1246|34573|29707|289731|6525|27719|1|54|14|1|20|1352|5|65.55|1.31|66.86|82.35|85.05|52.44|4.85|8.26|168.71| +2450929|64882|2758|9018|25626|987|40348|22282|1321852|4316|5580|1|88|12|1|35|1354|27|53.73|4.29|58.02|94.02|42.93|52.11|1.49|0.13|141.24| +2451016|30608|17647|25828|1111165|4502|2639|87583|897727|5531|44535|1|32|7|1|18|1355|63|1328.67|66.43|1395.10|40.51|362.25|704.19|62.44|562.04|469.19| +2451057|29509|1789|39066|1689535|804|22073|11470|894243|1987|12531|1|44|10|4|8|1358|23|564.19|11.28|575.47|18.71|711.39|344.15|169.43|50.61|741.38| +2451013|26031|5356|87978|1163640|1255|18870|87978|1163640|6299|18870|2|62|9|5|16|1361|32|1808.96|0.00|1808.96|17.45|542.40|1031.10|85.56|692.30|559.85| +2450995|27963|10874|93909|212063|5064|4418|63673|243965|1943|25874|4|53|14|2|18|1362|51|1585.08|15.85|1600.93|76.64|877.20|1046.15|334.13|204.80|969.69| +2451035|62424|9494|52818|677385|352|44119|52618|957802|1603|41004|4|45|14|5|21|1363|4|91.56|2.74|94.30|47.00|213.64|12.81|9.45|69.30|263.38| +2451020|60480|12602|52818|677385|352|44119|12307|325581|485|27342|4|3|20|4|21|1363|17|405.45|20.27|425.72|19.56|12.92|328.41|60.09|16.95|52.75| +2450908|6695|11882|70003|813665|1327|14636|39191|1446628|6751|25321|4|4|10|3|16|1364|61|1469.49|0.00|1469.49|24.42|46.97|470.23|149.88|849.38|71.39| +2451012|71219|17023|82811|886612|2060|42708|9423|1196680|1459|44638|1|89|9|3|32|1365|95|1656.80|66.27|1723.07|95.56|1066.85|1474.55|32.80|149.45|1228.68| +2450875|66280|14929|23267|119298|5729|4207|17719|592894|6084|45674|4|31|11|3|3|1367|58|46.40|0.46|46.86|2.53|441.96|30.62|11.67|4.11|444.95| +2450923|56872|11530|23267|119298|5729|4207|80186|471456|2409|23362|4|71|1|4|14|1367|40|1282.40|12.82|1295.22|46.76|524.40|269.30|932.05|81.05|583.98| +2450995|70530|1988|23267|119298|5729|4207|66190|1788852|1040|42381|4|75|18|4|27|1367|13|389.09|23.34|412.43|89.03|139.88|307.38|58.83|22.88|252.25| +2450952|5300|17728|34014|922967|5873|9626|21423|899034|4406|14178|1|42|14|1|3|1369|2|236.94|18.95|255.89|31.52|35.28|23.69|57.57|155.68|85.75| +2451007|46159|16670|74030|323322|614|28306|75262|877262|1948|13434|1|66|5|1|7|1370|15|969.30|67.85|1037.15|11.79|138.45|872.37|79.48|17.45|218.09| +2451002|62830|8947|48124|1197603|5835|15714|32296|1735729|3969|22656|4|29|17|2|14|1371|23|876.99|0.00|876.99|58.67|1962.82|464.80|160.75|251.44|2021.49| +2451005|36002|1660|35298|964753|2208|31263|35298|964753|4414|31263|1|64|14|4|24|1373|7|390.11|15.60|405.71|49.01|205.52|15.60|138.56|235.95|270.13| +2450950|37885|17498|49242|806558|2274|40790|3513|1301404|2456|47908|1|96|17|4|26|1374|54|11146.68|445.86|11592.54|94.44|2913.30|5127.47|4514.40|1504.81|3453.60| +2451020|32994|10960|72904|1059263||14287|44574||||2|70|17|||1379|||||||1297.69|102.63||| +2450966|55066|8533|43082|1591714|7193|225|68103|1421171|6657|14096|4|9|7|3|29|1380|16|432.00|0.00|432.00|51.90|394.40|241.92|87.43|102.65|446.30| +2451039|28855|15058|43082|1591714|7193|225|43829|28253|2956|32506|4|21|15|5|3|1380|1|1.83|0.07|1.90|93.34|2.02|0.64|0.24|0.95|95.43| +2450962|68709|386|57149|1246692|235|8578|57149|1246692|1177|8578|4|37|12|1|18|1381|48|720.96|43.25|764.21|31.15|720.96|569.55|86.30|65.11|795.36| +2450948|6266|14923|53574|1138180|2433|9039|40601|340573|2799|7778|4|63|16|5|21|1383|49|490.00|44.10|534.10|49.49|172.97|235.20|231.86|22.94|266.56| +2450973|13845|10831|82031|1394703|5775|32494|26556|209383|1911|24439|1|77|16|4|25|1385|58|1076.48|32.29|1108.77|29.95|1839.18|850.41|76.86|149.21|1901.42| +2450987|5646|15259|25691|44352|4260|10921|57157|853964|2344|15524|4|66|11|1|21|1388|2|4.76|0.33|5.09|55.27|9.06|3.47|1.22|0.07|64.66| +2450941|57203|12064|25691|44352|4260|10921|6163|185073|7147|7974|4|92|9|3|25|1388|33|1240.14|99.21|1339.35|87.53|1717.32|334.83|135.79|769.52|1904.06| +2451026|17086|10942|24107|1479518|6576|47187|79339|362442|2595|48395|1|77|15|5|8|1389|5|492.40|4.92|497.32|18.38|283.75|329.90|125.12|37.38|307.05| +2451023|11912|2191||||47187|||||1|8||||1389||||2513.16|||1105.79|||| +2451018|68591|2473|34941|921766|591|13816|81636|596428||12198||||||1390|10|187.40||202.39||76.40||||183.43| +2450893|29488|4486|34941|921766|591|13816|29713|1157767|728|3689|1|107|5|4|35|1390|58|944.82|66.13|1010.95|5.60|687.30|869.23|53.66|21.93|759.03| +2450935|44822|2438|49474|1124985|6896|37207|44135|1711739|5837|37135|4|75|16|4|2|1392|25|223.00|17.84|240.84|35.11|405.50|200.70|12.71|9.59|458.45| +2450963|33527|6787|52842|826996|6750|44457|50866|145068|1341|24061|1|69|7|5|23|1394|13|310.05|0.00|310.05|97.36|0.00|170.52|139.53|0.00|97.36| +2451083|8811|17228|52842|826996|6750|44457|38925|833339|2331|21323|1|95|19|1|3|1394|52|2378.48|142.70|2521.18|47.37|1532.44|380.55|299.68|1698.25|1722.51| +2450946|60511|1402|||||19657|||1691||19||5||1395|34||1.28||43.05||17.94||78.28|| +2450901|43955|1880|91724|1004775|226|32830|91724|1004775|5636|32830|2|2|16|1|26|1396|8|429.92|17.19|447.11|80.55|205.60|429.92|0.00|0.00|303.34| +2450972|32733|1048|2367|1215323|5526|32482|26724|1718075|3636|28797|2|29|20|1|2|1397|57|3190.29|287.12|3477.41|13.95|0.00|3126.48|34.45|29.36|301.07| +2450965|44740|7528|5004|188280|6403|37279|33099|1143933|904|8228|4|49|14|5|26|1398|15|800.10|48.00|848.10|45.12|457.20|528.06|59.84|212.20|550.32| +2450984|53389|4864|14964|862374|5431|43241|42688|353128|39|21797|1|46|17|5|16|1399|30|1308.30|26.16|1334.46|38.26|250.80|693.39|553.41|61.50|315.22| +2450897|25992|4096|56408|1266377|4793|15962|56130|1286163|3460|13157|2|88|15|2|19|1409|6|78.00|3.12|81.12|12.53|170.88|7.02|48.26|22.72|186.53| +2451028|35798|2980|45067|1718012|3757|22702|52959|447575|3525|33028|2|56|18|3|8|1411|26|272.22|24.49|296.71|8.92|616.98|269.49|0.90|1.83|650.39| +2451091|49568|7153|25729|193596|3416|25636|25729|193596|5611|25636|1|57|8|4|10|1412|29|4101.76|287.12|4388.88|60.83|1935.75|2584.10|394.59|1123.07|2283.70| +2451028|12529|16892|96553|1812010|2314|9371|26482|936416|5762|32708|2|91|6|5|11|1413|13|864.89|0.00|864.89|17.33|386.10|717.85|95.57|51.47|403.43| +2451102|34594|5263|7826|970311|6749|30434|37226|1737304|6101|27298|4|33|15|2|14|1414|15|674.85|47.23|722.08|55.45|648.90|546.62|82.06|46.17|751.58| +2450948|62069|3826|7826|970311|6749|30434|7826|970311|10|30434|4|46|2|5|13|1414|54|3019.14|0.00|3019.14|96.41|1277.10|392.48|2547.86|78.80|1373.51| +2450970|59366|4615|7826|970311|6749|30434|33647|704390|926|40662|4|64|10|1|22|1414|24|576.72|46.13|622.85|53.80|218.64|5.76|148.44|422.52|318.57| +2451077|43796|7861|7826|970311|6749|30434|64923|1323721|5113|12152|4|87|15|2|9|1414|27|1979.37|118.76|2098.13|95.10|296.73|1979.37|0.00|0.00|510.59| +2450908|67829|12457|66361|1561966|3171|5576|47442|1227080|4518|41203|2|71|17|5|23|1416|6|169.08|13.52|182.60|20.36|128.10|45.65|44.43|79.00|161.98| +2450976|22325|2260|66361|1561966|3171|5576|78932|1311017|5638|259|2|69|2|3|16|1416|20|299.40|8.98|308.38|69.12|178.00|266.46|12.84|20.10|256.10| +2450985|73392|1279|66361|1561966|3171|5576|86610|1809415|2497|3932|2|90|18|3|25|1416|5|43.45|3.91|47.36|25.98|13.90|3.91|19.37|20.17|43.79| +2450910|32468|11132|52551|902991|1078|18318|42842|553413|3045|30230|2|92|5|2|2|1418|11|15.95|0.31|16.26|44.36|63.91|6.53|5.93|3.49|108.58| +2450994|38404|3724|52551|902991|1078|18318|2920|51048|6735|48822|2|53|10|4|6|1418|37|290.08|17.40|307.48|22.84|231.99|147.94|54.01|88.13|272.23| +2450974|28816|3250|26559|963847|1935|4470|50522|1646558|7168|22178|4|64|15|5|5|1419|3|46.20|4.15|50.35|51.35|103.95|7.39|28.71|10.10|159.45| +2450923|18116|16334|83714|816493|2072|7651|14934|94545|5423|39854|4|18|14|5|26|1420|13|1495.13|134.56|1629.69|54.21|811.59|224.26|38.12|1232.75|1000.36| +2451017|63279|854|75844|139779|4514|4849|18873|568834|2641|43248|1|52|9|1|28|1422|67|544.71|49.02|593.73|65.28|4359.69|266.90|119.45|158.36|4473.99| +2450923|47720|11774|75844|139779|4514|4849|45349|1656561|6900|5239|1|73|3|2|13|1422|5|543.90|5.43|549.33|17.06|188.70|456.87|83.54|3.49|211.19| +2451019|43614|2198|95307|1757433|7112|26838|79750|729940|1881|31869|4|6|19|2|18|1423|3|119.94|1.19|121.13|18.67|11.97|44.37|25.69|49.88|31.83| +2451014|16230|9022|95307|1757433|7112|26838|21402|917020|2959|7651|4|39|1|3|17|1423|4|16.80|1.51|18.31|84.46|9.68|7.89|0.35|8.56|95.65| +2450949|5944|17578|95307|1757433|7112|26838|46040|758519|6735|28504|4|94|18|2|1|1423|6|205.50|4.11|209.61|82.44|308.28|156.18|10.35|38.97|394.83| +2451006|64738|925|54322|171714|6706|33298|38386|1668513|1284|46102|1|6|2|3|31|1427|72|9500.40|0.00|9500.40|26.91|3826.08|7220.30|250.81|2029.29|3852.99| +2450912|44759|15622|48262|1345331|800|30269|69962|1839687|357|37827|4|10|4|4|26|1428|42|254.10|15.24|269.34|61.93|441.84|2.54|173.57|77.99|519.01| +2450991|64689|4879||1345331|800||47613||||4|78|6||31|1428||7.52|0.45||26.00|3.30|||0.11|| +2450997|8443|17672|48262|1345331|800|30269|93032|1122489|2393|30297|4|26|9|3|32|1428|20|797.40|31.89|829.29|43.04|667.60|406.67|296.95|93.78|742.53| +2450864|31289|12734|48262|1345331|800|30269|51633|1589467|6623|23779|4|14|20|1|32|1428|22|7.70|0.61|8.31|24.81|88.66|1.77|5.04|0.89|114.08| +2450969|61286|3800|6224|1120096|3215|31887|83249|800607|596|2985|4|51|19|3|32|1432|6|131.10|11.79|142.89|54.96|149.88|20.97|73.78|36.35|216.63| +2450936|70280|7141|6224|1120096|3215|31887|26404|1019793|2158|20876|4|8|17|2|15|1432|4|103.48|2.06|105.54|59.87|50.92|36.21|12.10|55.17|112.85| +2450959|10067|6206|63790|135934|6478|48622|8164|1191239|2506|34973|1|100|7|4|6|1434|58|13799.36|689.96|14489.32|83.11|6485.56|13661.36|75.90|62.10|7258.63| +2450950|72141|16628|63790|135934|6478|48622|68441|1052354|4527|15947|1|81|6|1|4|1434|25|46.50|0.93|47.43|80.52|465.25|24.18|18.52|3.80|546.70| +2450924|70073|14690|93218|463296|3327|47758|17155|798882|3136|38637|1|35|6|5|9|1435|63|1101.24|66.07|1167.31|6.51|282.87|1101.24|0.00|0.00|355.45| +2450938|62335|13393|99925|847142|2359|17129|76687|957109|6839|169|4|27|6|1|21|1436|91|1590.68|79.53|1670.21|60.60|521.43|747.61|758.76|84.31|661.56| +2451001|48025|11062|99925|847142|2359|17129|82402|1868203|7048|42293|4|9|8|3|8|1436|11|438.46|35.07|473.53|86.82|85.47|135.92|45.38|257.16|207.36| +2450935|15779|9788|75788|310469|2750|3384|39878|626588|6890|10792|4|36|5|2|19|1437|4|287.44|11.49|298.93|92.39|167.64|209.83|59.75|17.86|271.52| +2450859|52462|8056|55707|1298790|3473|5756|52963|1830297|2391|14838|1|94|20|2|26|1439|49|1088.29|43.53|1131.82|73.13|1231.37|544.14|315.60|228.55|1348.03| +2450917|58591|3332|31566|1509584|6355|29173|89566|89941|3681|8221|2|79|1|1|26|1442|18|1753.38|87.66|1841.04|3.96|1134.54|753.95|669.61|329.82|1226.16| +2450919|68451|9212|70759|1142853|547|17041|12824|483029|3053|31524|1|65|10|1|27|1443|9|81.00|6.48|87.48|60.63|359.01|0.00|40.50|40.50|426.12| +2451064|67285|4000|70759|1142853|547|17041|11788|177082|1988|25647|1|82|18|2|28|1443|15|2082.30|104.11|2186.41|88.37|631.65|1061.97|459.14|561.19|824.13| +2450898|29894|5524|23021|66630|5086|36295|75538|1524626|5623|21467|4|82|12|1|10|1446|22|247.72|2.47|250.19|2.46|284.90|205.60|16.00|26.12|289.83| +2451015|84886|16696|23021|66630|5086|36295|25018|1890242|4221|15400|4|103|10|4|3|1446|5|212.25|0.00|212.25|47.78|74.90|191.02|7.64|13.59|122.68| +2451000|74473|3476|7996|811700|4651|41955|97038|884152|7089|24026|4|42|17|4|9|1448|14|501.06|5.01|506.07|56.50|657.72|175.37|185.64|140.05|719.23| +2450937|2173|8474|41661|1374345|3029|21760|92171|598375|5898|7868|4|42|17|5|23|1449|15|1859.40|92.97|1952.37|96.55|247.80|762.35|1097.05|0.00|437.32| +2450982|63943|8215|95408|897186|6964|44342|93347|1734352|7014|8076|2|3|2|4|11|1450|14|128.10|6.40|134.50|47.96|801.22|94.79|17.65|15.66|855.58| +2450925|77389|6358|95408|897186|6964|44342|59879|565670|3181|29892|2|88|5|1|7|1450|51|5407.53|108.15|5515.68|17.53|1886.49|4596.40|186.55|624.58|2012.17| +2450921|43173|2254|95408|897186|6964|44342|39203|877240|2366|48245|2|93|9|5|24|1450|32|1718.72|154.68|1873.40|82.46|0.00|292.18|313.83|1112.71|237.14| +2450970|68117|15458|94631|330701|2771|10088|33597|1649374|6273|35972|2|7|3|2|23|1451|10|55.30|4.97|60.27|10.38|41.50|49.21|5.05|1.04|56.85| +2451075|8131|15014|52219|572013|160|19154|94399|1054476|1533|14990|2|23|8|5|32|1456|27|292.41|0.00|292.41|37.94|987.39|160.82|19.73|111.86|1025.33| +2450968|27751|4064||482023|4485|42194||57774|3141|||||1||1460|40|163.20||||||16.56|40.56|952.19| +2450966|39197|6922|88720|484215|6557|16499|29287|38572|6521|22423|4|42|3|2|21|1461|31|1253.95|62.69|1316.64|9.45|248.00|288.40|936.58|28.97|320.14| +2450922|45545|5048|984|1761548|1759|40723|10442|385989|3504|48666|4|47|18|5|5|1462|43|780.02|39.00|819.02|38.51|8.60|343.20|327.61|109.21|86.11| +2450937|17156|5342|53672|1405316|3807|44251|62056|154523|6943|13716|2|48|9|4|2|1463|1|16.24|0.16|16.40|39.37|22.49|15.26|0.40|0.58|62.02| +2450992|33378|17366|36586|1346210|6234|48705|37637|1574021|4216|48811|4|4|16|4|14|1464|9|190.80|7.63|198.43|11.41|52.02|83.95|12.82|94.03|71.06| +2450889|53742|4519|36586|1346210|6234|48705|35631|1313105|1135|7212|4|105|2|4|19|1464|39|148.98|10.42|159.40|32.70|65.13|41.71|50.41|56.86|108.25| +2450911|74121|10993|63963|374562|1608|13054|82273|190318|3626|25334|1|75|11|4|35|1465|4|9.20|0.09|9.29|2.94|95.16|4.32|0.87|4.01|98.19| +2450951|44609|14077|94340|272761|5582|12403|51309|96359|3800|41|2|42|4|3|24|1466|85|6695.45|267.81|6963.26|82.17|0.00|3280.77|1980.51|1434.17|349.98| +2451078|42941|16502|94340|272761|5582|12403|96527|1822382|1651|42567|2|30|8|4|13|1466|9|408.06|32.64|440.70|51.17|519.39|395.81|10.78|1.47|603.20| +2450967|9118|1615|36677|497732|2340|10195|87937|336380|3744|24708|4|1|20|4|23|1467|59|1666.16|149.95|1816.11|1.58|97.94|1282.94|206.93|176.29|249.47| +2450971|32594|7072|27111|412624|6937|32450|17227|580471|6576|43881|1|3|3|3|18|1470|30|885.60|0.00|885.60|95.99|140.70|442.80|150.55|292.25|236.69| +2451040|68380|9547|27111|412624|6937|32450|27111|412624|2509|32450|1|27|19|5|24|1470|27|2251.26|67.53|2318.79|16.22|1113.48|135.07|1121.58|994.61|1197.23| +2450970|74477|5092|83065|570648|278|9861|42566|1378958|5016|27874|2|71|1|2|5|1473|12|2083.08|41.66|2124.74|15.49|21.36|1770.61|24.99|287.48|78.51| +2451009|74249|15892|83065|570648|278|9861|19905|391563|5427|37338|2|33|17|1|7|1473|67|2241.82|0.00|2241.82|61.06|448.23|1591.69|390.07|260.06|509.29| +2450887|69016|15139|94300|1185920|3587|8854|45810|536557|1541|572|1|34|11|2|31|1474|29|5750.41|57.50|5807.91|95.72|1856.87|1092.57|3539.95|1117.89|2010.09| +2451005|64647|10694|60875|1345834|40|34199|92790|345020|6698|5176|4|63|10|3|11|1475|6|179.04|10.74|189.78|11.69|588.42|23.27|20.25|135.52|610.85| +2450961|64075|7039|5045|1443948|5604|19660|79244|757001|1171|8941|4|46|11|1|21|1477|16|152.48|9.14|161.62|93.98|136.00|56.41|47.07|49.00|239.12| +2451017|18404|6668|5045|1443948|5604|19660|43828|323201|1597|1240|4|98|2|5|1|1477|70|2958.20|147.91|3106.11|98.04|3026.80|1597.42|231.33|1129.45|3272.75| +2450961|26198|6488|87879|78131|1983|49835|56854|170307|2617|7327|2|40|17|2|32|1478|2|259.32|7.77|267.09|75.18|156.86|132.25|116.90|10.17|239.81| +2450999|67180|11764|87846|1270191|6489|48787|87846|1270191|6064|48787|2|99|20|4|15|1479|24|1731.84|121.22|1853.06|84.88|2771.04|190.50|1479.68|61.66|2977.14| +2451083|50384|13414|87846|1270191|6489|48787|49437|1798142|361|28227|2|7|11|5|6|1479|15|57.45|1.14|58.59|80.49|15.00|14.93|35.29|7.23|96.63| +2451057|39781|3442|65517|608961|1205|19849|27145|361530|770|39610|2|34|17|2|24|1480|31|2243.47|157.04|2400.51|11.63|1062.68|1233.90|878.32|131.25|1231.35| +2451039|67491|15148|65517|608961|1205|19849|57315|1155749|72|23595|2|2|10|3|26|1480|5|116.60|0.00|116.60|48.11|31.25|36.14|62.75|17.71|79.36| +2451027|63852|11491|10536|1471645|4307|12616|10536|1471645|4800|12616|2|103|3|2|6|1481|47|1707.51|0.00|1707.51|61.94|39.48|409.80|661.83|635.88|101.42| +2450927|62435|5155|71827|149426|7025|15824|72396|144092|6339|18539|2|32|13|3|28|1483|17|1711.39|0.00|1711.39|92.07|1275.68|1317.77|251.91|141.71|1367.75| +2451017|77311|4564|71827|149426|7025|15824|13424|297490|2176|38509|2|32|14|2|15|1483|2|9.28|0.46|9.74|16.76|32.50|8.81|0.19|0.28|49.72| +2450896|13537|5497|54778|51865|5399|12486|66923|1748568|5838|38308|4|82|18|3|25|1484|7|227.78|20.50|248.28|49.81|0.00|13.66|49.24|164.88|70.31| +2451047|68242|7483|5804|115467|4330|28393|73849|368143|4390|2637|1|43|18|5|27|1485|25|32.50|1.95|34.45|57.82|65.00|6.17|12.37|13.96|124.77| +2450929|18693|7382|5497|1498498|6252|453|83099|1515931|6770|39609|4|40|20|4|13|1486|12|900.84|27.02|927.86|13.52|26.04|252.23|259.44|389.17|66.58| +2450885|32899|7556|5497|1498498|6252|453|83583|1188727|2495|30263|4|67|1|4|17|1486|10|1344.50|0.00|1344.50|29.75|112.00|1089.04|33.20|222.26|141.75| +2451012|39430|9002|83368|1236773|578|28783|81780|36259|5456|37053|2|72|4|3|35|1488|5|40.10|1.60|41.70|40.97|28.00|32.08|7.13|0.89|70.57| +2450986|22793|12440|42732|1337154|3129|16304|40836|719321|5640|10712|2|26|15|2|11|1490|22|1721.72|0.00|1721.72|41.50|1076.02|1566.76|116.22|38.74|1117.52| +2451078|63726|5761|42732|1337154|||5994||||||20|5|10|1490|7|605.22|36.31||||||71.67|388.47| +2450905|63954|15544|21475|1167551|3674|14197|77347|84988|652|10269|4|67|1|3|27|1492|81|319.95|22.39|342.34|45.08|1642.68|291.15|27.07|1.73|1710.15| +2451052|6141|9184|52424|1243522|6267|22965|23034|1475228|500|40172|1|36|8|5|27|1495|34|3275.56|65.51|3341.07|95.44|1853.34|3144.53|44.55|86.48|2014.29| +2451083|41172|9361|85997|1101667|2137|4207|85997|1101667|5342|4207|2|44|13|1|22|1496|30|1445.70|57.82|1503.52|94.72|435.60|1402.32|40.34|3.04|588.14| +2450924|72467|8000|80922|583951|6180|29609|61823|702041|1725|35809|1|67|15|4|35|1497|10|273.00|19.11|292.11|41.97|122.40|177.45|93.63|1.92|183.48| +2451029|77211|15284|93056|895729|2|2275|29882|1549095|928|14008|1|38|14|2|12|1498|43|2310.39|161.72|2472.11|66.24|990.29|508.28|720.84|1081.27|1218.25| +2450927|46287|10544|6922|92849|4729|49856|2420|907142|4842|19295|4|48|8|3|3|1499|22|199.98|1.99|201.97|32.08|186.12|55.99|59.03|84.96|220.19| +2450941|35793|11221|85886|1805495|304|33913|56215|316043|236|48429|1|43|8|1|27|1500|5|120.80|2.41|123.21|75.11|183.05|67.64|21.26|31.90|260.57| +2450921|77137|7592|85886|1805495|304|33913|68030|532157|6391|19825|1|90|20|4|23|1500|67|4608.93|322.62|4931.55|27.01|373.19|3963.67|219.38|425.88|722.82| +2451008|35756|1888|71220|1324221|4069|8317|6980|165884|1401|12565|4|19|3|5|7|1501|4|120.04|2.40|122.44|61.83|64.00|24.00|16.32|79.72|128.23| +2450994|55824|4489|71220|1324221|4069|8317|401|1389055|5495|29182|4|82|3|5|2|1501|3|89.64|1.79|91.43|19.50|245.01|25.99|10.18|53.47|266.30| +2451067|71464|6218|96919|507526|180|17234|92095|1265850|812|27327|4|105|13|5|32|1502|63|3392.55|237.47|3630.02|28.41|5026.14|3358.62|0.33|33.60|5292.02| +2450943|44318|10324|96919|507526|180|17234|96919|507526|1627|17234|4|80|11|2|23|1502|13|71.50|2.86|74.36|15.74|143.00|45.76|4.89|20.85|161.60| +2451064|35087|3668|48284|303488|4292|21896|64371|1870839|3652|9665|4|19|11|4|27|1504|31|962.55|48.12|1010.67|77.10|569.78|664.15|244.68|53.72|695.00| +2451092|44310|8596|85052|100347|5595|39643|42477|1406850|5128|15806|1|23|7|5|33|1506|2|201.12|4.02|205.14|7.77|69.04|20.11|39.82|141.19|80.83| +2450970|55080|692|85052|100347|5595|39643|86715|1025135|6764|33593|1|58|20|4|13|1506|31|76.57|3.82|80.39|13.05|18.60|3.82|53.10|19.65|35.47| +2450986|67319|10576|58716|1539409|902|37195|21289|1901522|6424|27726|4|28|12|1|8|1507|46|1420.02|127.80|1547.82|8.04|596.16|326.60|207.74|885.68|732.00| +2450990|47019|1972|97599|1441194|3989|31937|29814|783128|5397|25219|1|50|12|4|26|1509|8|808.16|48.48|856.64|17.00|121.20|72.73|58.83|676.60|186.68| +2450877|72225|7852|93405|1606790|4975|17960|74296|44333|6842|46904|1|24|4|1|20|1512|10|211.20|0.00|211.20|14.75|51.50|202.75|6.42|2.03|66.25| +2451018|16257|6025|54078|1686051|6377|41550|15129|1091760|2420|8280|2|57|20|5|5|1514|11|48.62|3.88|52.50|60.66|66.44|42.29|5.69|0.64|130.98| +2451031|3217|1142|54078|1686051|6377|41550|49251|286614|3187|22560|2|28|12|5|11|1514|3|9.72|0.48|10.20|18.24|41.31|6.51|0.93|2.28|60.03| +2450973|33336|7387|54496|515485|6074|44263|92773|1052768|5303|41391|2|52|15|3|11|1516|7|566.44|11.32|577.76|43.23|86.38|118.95|120.82|326.67|140.93| +2450870|49279|16627|46467|1043783|7146|32462|33543|606205|512|20372|1|59|19|4|14|1517|16|469.28|14.07|483.35|89.57|844.64|220.56|216.38|32.34|948.28| +2450986|76689|13789|92253|1801821|2589|49357|92253|1801821|6726|49357|4|49|1|5|7|1518|14|1267.00|63.35|1330.35|78.96|662.76|506.80|304.08|456.12|805.07| +2450877|58042|12742|57661|21459|5704|27549|68879|1136104|5360|40616|2|16|11|2|33|1525|2|263.04|0.00|263.04|68.63|0.00|210.43|5.26|47.35|68.63| +2451033|60021|17281|41734|979631|2328|44625|34688|415659|5200|16788|4|62|18|4|15|1527|54|2529.36|202.34|2731.70|33.37|1918.62|1972.90|506.37|50.09|2154.33| +2450986|52260|7063|41734|979631|2328|44625|46864|411051|5343|26788|4|47|17|4|29|1527|50|12616.50|883.15|13499.65|7.79|4994.00|8453.05|2581.33|1582.12|5884.94| +2451008|59476|13174|34177|919860|3143|6826|6267|454592|3375|4005|4|81|2|4|19|1529|11|1102.75|55.13|1157.88|31.91|893.64|871.17|132.00|99.58|980.68| +2451036|32443|11348|16661|798618|5375|3332|15160|1015015|5175|48488|1|38|14|4|32|1531|9|174.69|0.00|174.69|25.70|88.29|97.82|2.30|74.57|113.99| +2450884|18244|14062|16661|798618|5375|3332|57364|862606|2628|16603|1|1|8|5|5|1531|12|1351.08|13.51|1364.59|90.09|675.48|621.49|423.16|306.43|779.08| +2450946|66947|16789|58402|96002|2|7202|71946|1148824|6305|14186|2|2|2|3|4|1533|19|277.40|11.09|288.49|44.34|181.64|180.31|53.39|43.70|237.07| +2450958|73642|4664|2245|1838273|701|33129|38642|1835947|1616|3332|2|53|6|5|24|1534|48|2449.44|122.47|2571.91|31.67|2040.96|857.30|827.91|764.23|2195.10| +2451008|80927|8083|5823|1198428|7033|36438|8459|1774614|3109|36804|1|40|15|5|13|1538|10|212.10|14.84|226.94|0.56|96.40|163.31|19.51|29.28|111.80| +2450980|30204|17270|73661|427979|3097|14262|58454|1902629|3299|6369|2|104|19|2|7|1540|2|1.10|0.08|1.18|63.94|2.66|0.64|0.12|0.34|66.68| +2450969|48564|9718|73661|427979|3097|14262|36578|1559885|3163|12088|2|61|16|1|31|1540|3|214.83|15.03|229.86|66.62|7.95|55.85|89.02|69.96|89.60| +2450930|17491|7651|20986|800999|3059|8672|62909|1815277|3740|18956|1|39|18|4|27|1541|9|260.64|5.21|265.85|23.96|15.03|187.66|19.70|53.28|44.20| +2451011|33989|3373|20986|800999|3059|8672|2778|518393|3935|30677|1|24|9|1|26|1541|32|792.96|55.50|848.46|53.12|134.72|539.21|114.18|139.57|243.34| +2451018|44345|16352|78604|1200428|1899|11770|98845|1643459|6330|26001|2|33|14|5|28|1542|14|279.16|8.37|287.53|48.24|148.82|80.95|77.30|120.91|205.43| +2450925|80091|16622|78604|1200428|1899|11770|78604|1200428|2410|11770|2|27|8|4|2|1542|27|1655.10|82.75|1737.85|69.37|609.66|645.48|131.25|878.37|761.78| +2450981|76422|5782|78604|1200428|1899|11770|87381|50432|6183|46734|2|50|14|5|13|1542|48|3109.44|217.66|3327.10|15.84|1190.88|1834.56|446.20|828.68|1424.38| +2451021|55796|2707|20220|1171571|522||20220|1171571|1097|4626|||8|||1546|59||4.80|244.93||||93.65|62.44|| +2450935|71256|2594|87208|1658065|922|32626|62787|1917614|4404|5594|4|99|11|1|8|1547|40|246.00|4.92|250.92|51.24|723.20|179.58|32.54|33.88|779.36| +2451033|64295|3050|35065|390459|4253|49647||||33872|4|61|1|1||1548|||123.19|||3872.04|||3535.75|| +2451056|20389|1934|1009|1420574|1386|2951|40017|225840|1990|42140|2|102|10|5|20|1549|9|112.77|1.12|113.89|50.62|171.09|49.61|56.21|6.95|222.83| +2451062|72546|8518|1009|1420574|1386|2951|7133|978805|3704|3050|2|32|14|3|19|1549|49|499.80|34.98|534.78|63.18|288.61|274.89|47.23|177.68|386.77| +2450924|74734|17204|95318|1112622|5479|15983|43574|1626971|4875|8379|1|7|18|2|11|1552|28|683.76|13.67|697.43|99.07|166.32|280.34|330.80|72.62|279.06| +2451063|41667|3188|40501|998301|901|23501|57464|545630|6086|46542|4|9|3|4|35|1553|12|650.76|52.06|702.82|71.90|1449.48|605.20|42.82|2.74|1573.44| +2450945|69682|6664|40501|998301|901|23501|68216|1405307|5901|20095|4|43|17|1|15|1553|26|495.04|19.80|514.84|22.90|610.74|371.28|17.32|106.44|653.44| +2451011|62223|2581|35042|1346955|6868|34781|35042|1346955|6044|34781|4|93|8|4|5|1554|35|1252.65|0.00|1252.65|96.42|1962.80|1089.80|100.96|61.89|2059.22| +2451051|60063|6034|87084|1304332|780|21228|51507|334083|5327|17724|4|25|2|1|18|1555|24|341.52|23.90|365.42|89.83|160.08|286.87|6.55|48.10|273.81| +2451014|14015|13424|70429||5065||22763|1316488||||90||||1556||11.60|||71.30|16.16|1.74|||87.80| +2451011|60329|16388|70429|1196947|5065|25583|91297|214920|963|33806|1|9|2|5|28|1556|74|8821.54|264.64|9086.18|95.68|449.92|2558.24|1440.55|4822.75|810.24| +2450954|52759|11503|70429|1196947|5065|25583|70429|1196947|4988|25583|1|41|19|2|3|1556|12|822.00|65.76|887.76|64.69|278.04|558.96|205.17|57.87|408.49| +2450872|74729|946|83697|1304691|4885|44232|41114|577836|1997|21758|2|72|8|4|15|1557|1|5.14|0.20|5.34|78.35|0.93|0.51|1.57|3.06|79.48| +2451036|68121|2012|83697|1304691|4885|44232|52014|398686|6284|4282|2|44|9|2|35|1557|20|1223.40|48.93|1272.33|21.22|280.20|330.31|232.20|660.89|350.35| +2450995|12258|17809|83697|1304691|4885|44232|95041|1161538|2114|15937|2|41|5|3|9|1557|30|2506.20|175.43|2681.63|47.92|350.70|2130.27|349.61|26.32|574.05| +2451089|54371|4856|20531|45400|669|38738|85101|819373|4102|32831|4|10|18|2|29|1558|3|18.84|0.18|19.02|77.87|0.69|5.46|4.95|8.43|78.74| +2450979|41283|9812|20531|45400|669|38738|74002|1175606|620|13987|4|104|2|1|23|1558|9|394.02|35.46|429.48|15.50|117.36|374.31|15.76|3.95|168.32| +2451029|11039|3862|20531|45400|669|38738|68318|846028|5343|45505|4|84|11|1|13|1558|46|1406.22|98.43|1504.65|11.89|140.30|98.43|1216.24|91.55|250.62| +2450907|48607|15026|64550|1873775|1753|24578|36418|1332039|1873|40507|4|96|2|1|15|1561|13|473.59|28.41|502.00|11.85|69.16|194.17|215.15|64.27|109.42| +2451003|38006|1114|77195|804732|3469|35853|77195|804732|3217|35853|4|26|8|1|22|1562|43|2868.10|258.12|3126.22|82.92|385.71|602.30|2129.85|135.95|726.75| +2450946|71475|10726|77195|804732|3469|35853|19870|1429295|4961|15427|4|20|12|3|16|1562|70|896.00|0.00|896.00|98.94|5019.70|663.04|97.84|135.12|5118.64| +2450909|27949|12320|39832|944590|2595|35353|73546|209505|1656|6607|4|20|7|3|24|1565|13|311.09|21.77|332.86|31.49|29.25|74.66|101.66|134.77|82.51| +2450942|70458|5150|10260|1057228|1443|2011|99702|864537|1864|8391|2|16|5|5|7|1566|15|95.25|8.57|103.82|59.19|32.70|61.91|24.00|9.34|100.46| +2450863|65112|9146|10260|1057228|1443|2011|2427|575174|6863|46552|2|65|6|5|34|1566|26|1108.38|0.00|1108.38|59.62|411.06|288.17|377.29|442.92|470.68| +2450939|73502|2420|70543|546488|4033|33531|59556|25854|1042|47477|1|74|15|3|28|1570|24|313.68|15.68|329.36|9.75|0.00|43.91|70.14|199.63|25.43| +2450962|56722|9190|70543|546488|4033|33531|70543|546488|3471|33531|1|74|18|3|18|1570|30|1561.20|31.22|1592.42|72.21|1163.10|405.91|739.38|415.91|1266.53| +2450948|75221|7663|60635|128966|2897|34381|86147|591779|1575|40571|1|30|6|4|15|1571|11|490.82|44.17|534.99|39.32|672.65|436.82|31.86|22.14|756.14| +2451094|64513|9428|98675|1716247|1819|20591|98675|1716247|1610|20591|4|95|15|1|24|1573|94|1658.16|132.65|1790.81|66.98|216.20|82.90|866.39|708.87|415.83| +2450923|47452|9268|98675|1716247|1819|20591|61566|736009|970|14378|4|32|8|1|32|1573|54|2143.80|0.00|2143.80|7.92|344.52|2036.61|87.89|19.30|352.44| +2451024|58724|11113|78053|451276|852|38875|78053|451276|3763|38875|4|59|14|2|13|1574|15|117.30|1.17|118.47|61.65|385.65|16.42|2.01|98.87|448.47| +2451027|43051|4915|78053|451276|852|38875|1590|978233|6355|77|4|98|16|1|23|1574|57|603.63|6.03|609.66|82.08|230.85|585.52|14.66|3.45|318.96| +2450920|44510|12616|11524|1109028|6179|1683|461|693034|1208|14982|1|66|4|5|14|1577|41|387.86|7.75|395.61|44.47|211.56|100.84|287.02|0.00|263.78| +2450954|28819|5422|11524|1109028|6179|1683|11524|1109028|3692|1683|1|27|7|4|13|1577|1|42.29|1.26|43.55|29.27|50.74|28.75|4.19|9.35|81.27| +2451055|37827|15319|60184|133163|2142|45921|4577|1819651|1621|48344|4|108|7|5|5|1578|4|28.72|0.28|29.00|19.99|41.96|1.43|9.82|17.47|62.23| +2450947|39544|10274|56433|1049598|2410|7622|28560|1604596|6947|37651|4|97|4|3|19|1579|64|12172.16|0.00|12172.16|72.20|2808.96|12050.43|58.43|63.30|2881.16| +2450876|44911|7102|56433|1049598|2410|7622|65756|286850|3236|14375|4|40|6|2|1|1579|10|802.50|24.07|826.57|75.79|0.00|473.47|161.22|167.81|99.86| +2451026|70551|16322|56433|1049598|2410|7622|80044|1064756|2265|45821|4|75|9|5|6|1579|29|182.41|10.94|193.35|94.03|304.21|158.69|15.18|8.54|409.18| +2451024|12493|7276|36841|617205|3522|21086|94986|1672109|4666|12470|4|100|15|2|16|1580|8|123.28|1.23|124.51|6.04|100.80|107.25|2.56|13.47|108.07| +2450939|43091|9554|18731|820334|3537|34740|70249|1911455|891|37480|1|98|1|5|11|1582|9|492.12|24.60|516.72|7.84|30.24|201.76|81.30|209.06|62.68| +2450930|71031|1747|18731|820334|3537|34740|53691|1444748|5925|37855|1|26|12|2|14|1582|5|146.20|11.69|157.89|6.13|29.20|137.42|6.67|2.11|47.02| +2451048|75240|11798|18731|820334|3537|34740|43573|1267842|3525|1208|1|11|10|3|1|1582|54|4659.12|0.00|4659.12|46.92|931.50|1397.73|2609.11|652.28|978.42| +2450992|67882|944|32420||||34812||1365|4429|4|42|15|||1583|11||2.94|61.79|36.82|33.99|44.13||6.19|73.75| +2451004|74535|13432|15435|553733|4384|4235|15435|553733|6962|4235|1|51|18|2|16|1584|4|413.52|16.54|430.06|25.01|22.96|223.30|186.41|3.81|64.51| +2450949|78429|5911|99995|330678|5714|6397|74652|178801|6217|49233|1|93|1|4|15|1588|20|475.60|0.00|475.60|80.02|249.60|90.36|73.19|312.05|329.62| +2450871|81020|12544|99995|330678|5714|6397|50054|64380|2604|41228|1|78|14|3|4|1588|3|3.27|0.13|3.40|21.49|26.19|2.12|0.29|0.86|47.81| +2451076|69182|16214|84599|1749148|544|17940|84967|1454251|3116|24734|1|81|15|3|18|1589|1|36.79|2.57|39.36|50.00|25.21|12.87|10.52|13.40|77.78| +2451012|62528|16462|40584|526729|6474|38219|2288|909269|4577|27485|2|34|16|4|12|1590|37|3157.58|284.18|3441.76|87.49|851.37|63.15|2475.54|618.89|1223.04| +2450959|58631|1621|41060|654707|4001|41295|48097|101123|3561|44799|1|5|20|5|3|1596|38|531.24|26.56|557.80|69.10|962.92|132.81|286.86|111.57|1058.58| +2450941|34514|12632|82262|1170568|2521|27227|3543|892237|7155|2026|2|50|20|1|29|1598|5|486.00|24.30|510.30|88.12|10.65|442.26|41.55|2.19|123.07| +2450985|67517|10660|82262|1170568|2521|27227|25033|1895627|2467|12860|2|81|19|4|22|1598|10|1065.60|74.59|1140.19|51.58|395.30|447.55|333.74|284.31|521.47| +2451112|43595|15721|82262|1170568|2521|27227|79422|615865|3562|34859|2|43|3|5|32|1598|3|113.34|6.80|120.14|63.29|33.51|85.00|15.02|13.32|103.60| +2450858|16999|8702|51641|1662694|3347|46400|10487|1326672|3580|12488|4|8|6|4|27|1599|38|57.38|0.00|57.38|61.52|690.08|33.28|23.37|0.73|751.60| +2450955|70798|3238|20616|494731|6799|114|12695|791847|4824|49354|2|7|5|5|12|1600|2|195.60|0.00|195.60|49.24|4.64|119.31|27.46|48.83|53.88| +2450940|59081|10525|92581|926027|2673|9319|92581|926027|2824|9319|1|60|12|1|8|1602|21|2109.87|21.09|2130.96|18.38|1020.81|126.59|753.64|1229.64|1060.28| +2450939|18254|3817|36918|1179441|2317|20040|69166|1766066|5487|22508|1|62|18|4|30|1604|44|1104.84|77.33|1182.17|93.84|0.00|1027.50|23.20|54.14|171.17| +2450965|36206|4549|11093|350565|84|12756|11093|350565|120|12756|2|24|9|3|12|1605|23|827.54|16.55|844.09|50.75|401.12|736.51|71.00|20.03|468.42| +2450968|63465|5923|94468|1237249|6077|45658|24793|1621427|1631|32682|4|102|14|3|32|1606|77|823.13|41.15|864.28|56.84|1881.11|411.56|370.41|41.16|1979.10| +2450962|63118|70|94468|1237249|6077|45658|16550|1041581|2166|42798|4|98|14|1|21|1606|11|135.52|4.06|139.58|23.07|117.48|60.98|52.92|21.62|144.61| +2451027|58043|6577|13016|854073|5177|18634|75373|1827543|4410|20924|2|94|12|4|28|1607|5|237.10|21.33|258.43|90.87|333.70|23.71|4.26|209.13|445.90| +2451030|73357|5875|16823|344241|3659|18677|19737|1018713|4098|21149|2|60|17|5|8|1609|20|3594.80|71.89|3666.69|19.35|1761.40|1258.18|1588.90|747.72|1852.64| +2451089|65421|6847|16823|344241|3659|18677|78153|932168|6070|33572|2|94|11|1|18|1609|33|103.95|9.35|113.30|27.43|416.13|99.79|3.78|0.38|452.91| +2450993|19075|15638|16823||||19737||||||||32|1609|11|344.85|6.89||89.96|484.99|37.93|171.87|135.05|581.84| +2450947|51103|8444|16823|344241|3659|18677|79977|1503610|4640|42423|2|54|6|1|1|1609|22|572.00|34.32|606.32|46.59|117.70|74.36|144.31|353.33|198.61| +2450934|32065|2228|84307|1548472|2237|11602|47988|989770|5169|21768|2|108|5|4|33|1610|21|619.08|30.95|650.03|59.18|324.24|49.52|250.60|318.96|414.37| +2450954|69150|12068|84307|1548472|2237|11602|51075|1753189|5716|35043|2|88|11|3|5|1610|9|29.88|2.68|32.56|13.16|1.80|23.30|3.35|3.23|17.64| +2451004|27911|3206|51691|1776713|2182|22404|48629|1473860|3528|29243|1|68|1|1|1|1611|98|15097.88|754.89|15852.77|82.14|5304.74|4529.36|951.16|9617.36|6141.77| +2450870|71920|14962|46896|1069975|3454|28486|1353|444270|914|28758|2|90|10|4|30|1613|19|596.60|47.72|644.32|6.85|504.83|566.77|12.82|17.01|559.40| +2451012|54068|7972|69778|1573544|5310|5429|53688|1828913|507|32101|4|58|10|1|25|1616|43|1232.81|24.65|1257.46|70.27|328.52|36.98|167.41|1028.42|423.44| +2450920|84863|9769|67468|1839254|6687|27273|30645|1661534|3639|18944|4|86|12|2|5|1617|3|1.68|0.05|1.73|70.53|1.11|0.80|0.62|0.26|71.69| +2451041|58389|1903|67468|1839254|6687|27273|14135|1033285|1959|36586|4|103|1|5|12|1617|60|3585.60|107.56|3693.16|65.54|1476.00|3442.17|58.80|84.63|1649.10| +2451098|84274|3565|37740|1014948|6556|26964|51846|226375|3784|22840|4|55|8|4|15|1621|18|1767.96|141.43|1909.39|86.99|311.94|1432.04|245.22|90.70|540.36| +2450968|47814|3943|37740|1014948|6556|26964|21636|361494|5971|36048|4|60|19|1|28|1621|24|263.76|2.63|266.39|46.18|68.88|158.25|43.25|62.26|117.69| +2450914|51723|784|18857|1038053|5696|12492|7046|1616384|2173|11162|1|83|2|1|20|1624|38|3641.54|145.66|3787.20|82.07|1560.66|3240.97|244.34|156.23|1788.39| +2451103|77106|9020|49076|998991|106|45221|49076|998991|5961|45221|1|6|13|1|34|1629|11|2359.50|70.78|2430.28|32.06|1038.18|1156.15|348.97|854.38|1141.02| +2451029|16181|751|80012|1324106|2600|5494|70676|1019571|2054|26984|4|107|7|2|25|1634|47|1257.72|75.46|1333.18|26.70|294.69|402.47|846.69|8.56|396.85| +2450921|36387|4630|80012|1324106|2600|5494|44405|1848236|5617|45798|4|41|14|3|24|1634|29|2165.14|86.60|2251.74|56.26|1160.87|497.98|383.44|1283.72|1303.73| +2450942|72873|13201|6626|717293|313|4533|93198|922804|1931|1811|2|44|11|1|3|1635|1|6.49|0.51|7.00|96.45|1.62|3.69|2.46|0.34|98.58| +2451037|80856|9158|53011|947097|3936|24022|20701|1621649|239|34829|2|46|12|2|7|1636|23|5165.80|154.97|5320.77|40.79|593.63|2841.19|581.15|1743.46|789.39| +2450919|45976|2020|404|1588097|6990|29883|24965|270853|5359|31465|1|83|3|4|30|1638|64|1920.64|153.65|2074.29|54.32|1038.08|1440.48|321.70|158.46|1246.05| +2450949|35876|16501|56309|1008658|1007|44956|88643|1852074|5484|22447|4|187|3|1|31|1640|28|265.44|21.23|286.67|63.69|139.72|204.38|3.05|58.01|224.64| +2450864|43575|1294|38864|131370|3523|18829|63300|166215|5424|41386|1|83|19|3|10|1641|9|465.39|9.30|474.69|87.97|355.32|335.08|83.39|46.92|452.59| +2450887|14261|5533|87745|846794|3443|20092|71878|241942|1462|25382|4|39|15|5|35|1642|30|3965.10|118.95|4084.05|69.51|1875.30|1506.73|1204.60|1253.77|2063.76| +2450941|64877|8530|76573|1531125|877|49829|82009|555747|4203|9106|1|26|1|2|33|1643|36|173.88|10.43|184.31|84.72|572.04|52.16|114.41|7.31|667.19| +2451044|68996|2563|76573|1531125|877|49829|29216|1409649|3992|47535|1|69|11|1|19|1643|48|4386.72|0.00|4386.72|77.92|540.48|2456.56|1447.62|482.54|618.40| +2450970|61785|12410|76573|1531125|877|49829|64381|1562148|3878|31655|1|111|19|1|11|1643|23|1818.84|163.69|1982.53|29.58|509.22|1218.62|414.15|186.07|702.49| +2450900|55406|2182|15004|1309874|7144|28061|59473|1573177|1875|41373|4|102|1|4|14|1644|26|3587.74|143.50|3731.24|38.55|533.26|1291.58|1699.15|597.01|715.31| +2450930|84839|9808|15004|1309874|7144|28061|20993|1093026|2534|23995|4|33|5|1|27|1644|26|794.56|0.00|794.56|60.33|115.96|23.83|30.82|739.91|176.29| +2451090|57752|9842|14147|695622|3497|34619|64280|1024932|1714|18049|2|160|1|3|23|1646|2|240.28|2.40|242.68|96.27|20.22|76.88|1.63|161.77|118.89| +2450995|83174|16753|3183|714853|5723|32193|80722|924950|965|1027|2|14|12|4|14|1647|23|79.58|4.77|84.35|95.03|251.16|50.93|14.61|14.04|350.96| +2450890|41512|11740|62509|748475|2441|42007|27831|346768|6690|17059|2|40|12|4|33|1649|16|1393.12|83.58|1476.70|27.48|1321.60|766.21|43.88|583.03|1432.66| +2450907|74870|15571|62509|748475|2441|42007|47854|1505139|1471|14671|2|124|13|5|2|1649|16|928.80|83.59|1012.39|13.36|408.64|529.41|43.93|355.46|505.59| +2450996|37915|12086|53930|681027|3677|46374|1766|375061|3339|18417|4|23|11|4|35|1651|37|133.94|9.37|143.31|96.69|187.59|108.49|20.10|5.35|293.65| +2451028|17145|5072|53930|681027|3677|46374|51993|458438|6972|19106|4|176|8|2|1|1651|36|1534.68|46.04|1580.72|64.46|60.48|1396.55|113.26|24.87|170.98| +2451022|56950|4189|53930|681027|3677|46374|44208|365220|5246|24072|4|21|10|1|2|1651|2|19.16|1.14|20.30|98.65|2.54|0.57|11.71|6.88|102.33| +2450988|43024|2944|45471|1126782|6046|40510|15211|1465533|6939|23545|4|164|10|1|7|1655|58|847.38|59.31|906.69|71.79|734.28|601.63|238.37|7.38|865.38| +2451025|45378|6388|37891|823743|4795|47047|22613|894689|6806|43276|4|187|20|2|2|1656|71|1761.51|105.69|1867.20|41.89|2641.91|1409.20|73.98|278.33|2789.49| +2450874|31133|4471|12612|1233715|2818|22721|78226|1415280|6331|48229|2|98|16|2|10|1657|2|20.26|1.21|21.47|50.59|31.50|15.80|4.10|0.36|83.30| +2451047|50750|12721|12612|1233715|2818|22721|90284|384354|2031|18955|2|98|6|3|20|1657|38|1922.42|153.79|2076.21|4.78|529.34|1576.38|342.57|3.47|687.91| +2450926|41737|1832|54908|775295|1635|9622|41445|954204|3029|38254|4|172|16|1|21|1660|1|7.35|0.51|7.86|18.12|13.88|6.02|0.45|0.88|32.51| +2450960|16310|7075|52369|831832|2048|11511|4955|444843|1951|22659|4|53|10|3|1|1666|31|10.85|0.43|11.28|89.79|4.34|5.31|0.22|5.32|94.56| +2450916|72784|17000|4976|1173504|4079|607|4502|1693801|5087|6373|2|83|11|2|31|1670|1|73.19|0.00|73.19|71.24|49.54|0.00|66.60|6.59|120.78| +2451072|83781|14674|4976|1173504|4079|607|34839|153516|3306|38343|2|21|7|4|24|1670|35|392.35|7.84|400.19|64.22|1897.00|376.65|8.16|7.54|1969.06| +2450984|37555|1021|4946|847087|428|10169|77596|854234|784|38534|1|11|11|1|33|1671|3|88.32|0.00|88.32|63.47|56.01|44.16|7.50|36.66|119.48| +2450888|36768|12328|66387|1486419|5651|9283|25380|1081902|4472|22242|2|18|5|5|30|1672|1|65.88|1.97|67.85|41.17|6.33|14.49|25.18|26.21|49.47| +2450930|71551|15662|66387|1486419|5651|9283|60408|109889|2275|37461|2|138|19|5|31|1672|51|3328.26|199.69|3527.95|45.68|831.81|2096.80|960.53|270.93|1077.18| +2450962|44481|2504|68855|1551786|5917|7248|69249|169959|6051|34096|1|20|18|3|8|1675|83|712.97|0.00|712.97|30.81|1059.08|655.93|2.28|54.76|1089.89| +2450969|48691|6860|3254|1305572|5937|13902|28932|328139|7075|3964|4|184|18|4|3|1676|6|898.62|53.91|952.53|6.25|561.60|638.02|190.23|70.37|621.76| +2450904|44199|8650|73264|156773|5082|8991|64673|1865237|1256|49628|4|178|18|5|14|1677|35|884.80|70.78|955.58|46.97|60.90|424.70|262.25|197.85|178.65| +2450974|71025|8246|25336|1475574|5012|20497|25336|1475574|1081|20497|4|26|3|5|34|1679|1|147.84|13.30|161.14|0.50|36.96|82.79|43.58|21.47|50.76| +2451027|60447|9368|7560|636272|6584|22496|55322|1009446|2994|24095|2|83|18|1|27|1681|33|6739.59|134.79|6874.38|50.49|1754.94|269.58|4852.50|1617.51|1940.22| +2451024|56055|907|11067|1559073|3879|25885|95353|1487700|207|2667|1|53|8|5|30|1683|6|1125.24|11.25|1136.49|25.82|174.00|33.75|1036.91|54.58|211.07| +2450913|40730|3878|18862|74631|1247|22616|8259|207719|3061|26850|2|114|3|5|4|1686|30|1422.90|99.60|1522.50|75.81|512.10|313.03|455.04|654.83|687.51| +2451031|26190|3544|18862|74631|1247|22616|25518|370284|520|49556|2|73|10|1|17|1686|2|70.32|0.00|70.32|60.65|26.48|54.84|9.90|5.58|87.13| +2451000|74598|2671|69926|251212|2898|8984|69926|251212|3783|8984|4|14|4|1|8|1691|1|135.07|2.70|137.77|79.29|56.87|33.76|78.00|23.31|138.86| +2450939|77677|6566|17017|1371860|3503|21593|38828|668639|5256|24273|1|3|17|2|4|1693|5|534.80|26.74|561.54|90.12|25.85|475.97|7.64|51.19|142.71| +2450909|69409|7969|78672|826419|4919|34219|23455|906042|2210|16731|2|21|8|4|22|1694|22|265.54|5.31|270.85|37.71|404.14|132.77|110.19|22.58|447.16| diff --git a/spark-connector/spark-common/src/test/resources/data/tpcds/catalog_sales/catalog_sales.dat b/spark-connector/spark-common/src/test/resources/data/tpcds/catalog_sales/catalog_sales.dat new file mode 100644 index 00000000000..98984790036 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/data/tpcds/catalog_sales/catalog_sales.dat @@ -0,0 +1,14415 @@ +2450815|38212|2450886|62153|1822764|5775|19986|62153|1822764|5775|19986|4|62|3|4|16930|196|1|47|27.70|44.32|42.99|62.51|2020.53|1301.90|2083.04|101.02|0.00|1041.52|2020.53|2121.55|3062.05|3163.07|718.63| +2450815|38212|2450846|62153|1822764|5775|19986|62153|1822764|5775|19986|4|31|8|2|6020|270|1|20|87.55|260.89|153.92|2139.40|3078.40|1751.00|5217.80|71.41|1292.92|1356.60|1785.48|1856.89|3142.08|3213.49|34.48| +2450815|38212|2450868|62153|1822764|5775|19986|62153|1822764|5775|19986|4|76|2|2|16198|97|1|19|69.86|88.72|29.27|1129.55|556.13|1327.34|1685.68|33.36|0.00|168.53|556.13|589.49|724.66|758.02|-771.21| +2450815|38212|2450851|62153|1822764|5775|19986|62153|1822764|5775|19986|4|89|15|2|6200|284|1|50|70.00|205.10|188.69|820.50|9434.50|3500.00|10255.00|377.38|0.00|4307.00|9434.50|9811.88|13741.50|14118.88|5934.50| +2450815|29485|2450904|14601|797995|6189|9583|14601|797995|6189|9583|1|64|18|3|13831|176|2|56|67.54|166.82|18.35|8314.32|1027.60|3782.24|9341.92|0.00|0.00|3736.32|1027.60|1027.60|4763.92|4763.92|-2754.64| +2450815|29485|2450890|14601|797995|6189|9583|14601|797995|6189|9583|1|75|8|1|9248|278|2|88|20.08|60.03|20.41|3486.56|1796.08|1767.04|5282.64|13.82|1598.51|1056.00|197.57|211.39|1253.57|1267.39|-1569.47| +2450815|29485|2450849|14601|797995|6189|9583|14601|797995|6189|9583|1|39|4|3|15668|207|2|31|40.88|51.91|6.22|1416.39|192.82|1267.28|1609.21|11.56|0.00|321.78|192.82|204.38|514.60|526.16|-1074.46| +2450815|29485|2450889|14601|797995|6189|9583|14601|797995|6189|9583|1|49|8|2|15992|162|2|100|49.56|137.77|5.51|13226.00|551.00|4956.00|13777.00|0.00|0.00|3306.00|551.00|551.00|3857.00|3857.00|-4405.00| +2450815|29485|2450868|14601|797995|6189|9583|14601|797995|6189|9583|1|69|17|5|12248|72|2|30|72.82|88.84|71.07|533.10|2132.10|2184.60|2665.20|21.32|0.00|133.20|2132.10|2153.42|2265.30|2286.62|-52.50| +2450815|29485|2450831|14601|797995|6189|9583|14601|797995|6189|9583|1|64|1|3|8770|52|2|40|94.56|277.06|2.77|10971.60|110.80|3782.40|11082.40|2.21|0.00|664.80|110.80|113.01|775.60|777.81|-3671.60| +2450815|29485|2450839|14601|797995|6189|9583|14601|797995|6189|9583|1|108|8|4|13858|131|2|40|25.96|54.51|15.80|1548.40|632.00|1038.40|2180.40|22.75|176.96|1090.00|455.04|477.79|1545.04|1567.79|-583.36| +2450815|29485|2450825|14601|797995|6189|9583|14601|797995|6189|9583|1|14|20|3|6235|296|2|45|5.95|12.97|8.81|187.20|396.45|267.75|583.65|3.96|0.00|17.10|396.45|400.41|413.55|417.51|128.70| +2450815|29485|2450837|14601|797995|6189|9583|14601|797995|6189|9583|1|13|7|1|8707|271|2|34|43.69|105.29|82.12|787.78|2792.08|1485.46|3579.86|55.84|0.00|930.58|2792.08|2847.92|3722.66|3778.50|1306.62| +2450815|29485|2450822|14601|797995|6189|9583|14601|797995|6189|9583|1|106|16|5|17368|123|2|59|78.90|84.42|82.73|99.71|4881.07|4655.10|4980.78|292.86|0.00|1444.32|4881.07|5173.93|6325.39|6618.25|225.97| +2450815|29485|2450895|14601|797995|6189|9583|14601|797995|6189|9583|1|17|19|4|6295|131|2|94|41.36|105.88|50.82|5175.64|4777.08|3887.84|9952.72|191.08|0.00|497.26|4777.08|4968.16|5274.34|5465.42|889.24| +2450815|29485|2450892|14601|797995|6189|9583|14601|797995|6189|9583|1|74|18|2|6230|290|2|38|26.96|51.22|35.85|584.06|1362.30|1024.48|1946.36|108.98|0.00|642.20|1362.30|1471.28|2004.50|2113.48|337.82| +2450815|29485|2450862|14601|797995|6189|9583|14601|797995|6189|9583|1|23|16|5|14458|272|2|45|9.54|23.37|21.50|84.15|967.50|429.30|1051.65|21.76|725.62|294.30|241.88|263.64|536.18|557.94|-187.42| +2450815|29485|2450834|14601|797995|6189|9583|14601|797995|6189|9583|1|63|5|3|17671|127|2|41|39.04|108.92|75.15|1384.57|3081.15|1600.64|4465.72|215.68|0.00|1964.72|3081.15|3296.83|5045.87|5261.55|1480.51| +2450815|10687|2450864|67572|437897|6622|46147|67572|437897|6622|46147|1|28|5|5|5971|170|3|99|75.88|178.31|156.91|2118.60|15534.09|7512.12|17652.69|1398.06|0.00|6884.46|15534.09|16932.15|22418.55|23816.61|8021.97| +2450815|10687|2450905|67572|437897|6622|46147|67572|437897|6622|46147|1|47|10|5|10790|147|3|90|34.57|89.53|68.04|1934.10|6123.60|3111.30|8057.70|428.65|0.00|1449.90|6123.60|6552.25|7573.50|8002.15|3012.30| +2450815|10687|2450846|67572|437897|6622|46147|67572|437897|6622|46147|1|7|2|1|512|14|3|75|8.44|14.43|6.34|606.75|475.50|633.00|1082.25|33.28|0.00|10.50|475.50|508.78|486.00|519.28|-157.50| +2450815|10687|2450900|67572|437897|6622|46147|67572|437897|6622|46147|1|67|7|4|12622|162|3|45|45.24|59.26|53.33|266.85|2399.85|2035.80|2666.70|191.98|0.00|479.70|2399.85|2591.83|2879.55|3071.53|364.05| +2450815|10687|2450893|67572|437897|6622|46147|67572|437897|6622|46147|1|31|4|4|1358|116|3|33|89.04|217.25|52.14|5448.63|1720.62|2938.32|7169.25|68.82|0.00|3584.46|1720.62|1789.44|5305.08|5373.90|-1217.70| +2450815|10687|2450904|67572|437897|6622|46147|67572|437897|6622|46147|1|52|18|4|7906|121|3|25|72.16|98.13|29.43|1717.50|735.75|1804.00|2453.25|29.43|0.00|834.00|735.75|765.18|1569.75|1599.18|-1068.25| +2450815|10687|2450820|67572|437897|6622|46147|67572|437897|6622|46147|1|77|7|1|2476|216|3|56|12.62|28.39|20.72|429.52|1160.32|706.72|1589.84|4.87|1079.09|0.00|81.23|86.10|81.23|86.10|-625.49| +||2450835||3755||5652||3755|2480|5652|4||||10574|76|4|33|8.80|23.49||77.55|||775.17|34.88|0.00||||976.47|1011.35|407.22| +2450815|74512|2450888|25383|3755|2480|5652|25383|3755|2480|5652|4|107|13|2|5104|280|4|63|65.85|181.74|23.62|9961.56|1488.06|4148.55|11449.62|29.76|0.00|3663.45|1488.06|1517.82|5151.51|5181.27|-2660.49| +2450815|74512|2450825|25383|3755|2480|5652|25383|3755|2480|5652|4|88|13|1|9727|254|4|90|47.29|55.32|7.19|4331.70|647.10|4256.10|4978.80|38.82|0.00|1542.60|647.10|685.92|2189.70|2228.52|-3609.00| +2450815|74512|2450870|25383|3755|2480|5652|25383|3755|2480|5652|4|53|8|5|4772|6|4|68|51.19|135.65|82.74|3597.88|5626.32|3480.92|9224.20|337.57|0.00|922.08|5626.32|5963.89|6548.40|6885.97|2145.40| +2450815|74512|2450891|25383|3755|2480|5652|25383|3755|2480|5652|4|7|17|3|3391|53|4|12|35.46|59.57|5.36|650.52|64.32|425.52|714.84|4.50|0.00|250.08|64.32|68.82|314.40|318.90|-361.20| +2450815|74512|2450897|25383|3755|2480|5652|25383|3755|2480|5652|4|23|16|5|11180|38|4|79|49.20|118.57|92.48|2061.11|7305.92|3886.80|9367.03|219.17|0.00|3371.72|7305.92|7525.09|10677.64|10896.81|3419.12| +2450815|53201|2450856|76801|90299|3797|42495|76801|90299|3797|42495|2|29|20|1|8168|215|5|24|33.13|86.13|18.94|1612.56|454.56|795.12|2067.12|9.09|0.00|330.72|454.56|463.65|785.28|794.37|-340.56| +2450815|53201|2450886|76801|90299|3797|42495|76801|90299|3797|42495|2|90|2|5|716|204|5|88|61.57|91.12|87.47|321.20|7697.36|5418.16|8018.56|307.89|0.00|3046.56|7697.36|8005.25|10743.92|11051.81|2279.20| +2450815|53201|2450830|76801|90299|3797|42495|76801|90299|3797|42495|2|105|3|1|2323|245|5|33|79.68|165.73|97.78|2242.35|3226.74|2629.44|5469.09|64.53|0.00|273.24|3226.74|3291.27|3499.98|3564.51|597.30| +2450815|53201|2450895|76801|90299|3797|42495|76801|90299|3797|42495|2|40|18|5|14947|251|5|7|30.89|78.46|15.69|439.39|109.83|216.23|549.22|5.49|0.00|241.64|109.83|115.32|351.47|356.96|-106.40| +2450815|53201|2450817|76801|90299|3797|42495|76801|90299|3797|42495|2|78|16|1|4250|271|5|50|22.47|28.53|24.53|200.00|1226.50|1123.50|1426.50|85.85|0.00|256.50|1226.50|1312.35|1483.00|1568.85|103.00| +2450815|53201|2450820|76801|90299|3797|42495|76801|90299|3797|42495|2|44|12|5|3656|91|5|24|91.40|95.05|38.02|1368.72|912.48|2193.60|2281.20|54.74|0.00|638.64|912.48|967.22|1551.12|1605.86|-1281.12| +2450815|53201|2450846|76801|90299|3797|42495|76801|90299|3797|42495|2|69|14|5|17242|87|5|84|24.03|29.79|17.57|1026.48|1475.88|2018.52|2502.36|10.92|383.72|650.16|1092.16|1103.08|1742.32|1753.24|-926.36| +2450815|53201|2450833|76801|90299|3797|42495|76801|90299|3797|42495|2|16|10|1|12572|242|5|2|51.27|111.76|103.93|15.66|207.86|102.54|223.52|0.00|0.00|93.86|207.86|207.86|301.72|301.72|105.32| +2450815|53201|2450852|76801|90299|3797|42495|76801|90299|3797|42495|2|95|9|4|9589|99|5|17|38.62|106.59|68.21|652.46|1159.57|656.54|1812.03|33.04|57.97|199.24|1101.60|1134.64|1300.84|1333.88|445.06| +2450815|53201|2450860|76801|90299|3797|42495|76801|90299|3797|42495|2|9|16|2|13114|34|5|34|42.47|118.49|41.47|2618.68|1409.98|1443.98|4028.66|52.45|535.79|1571.14|874.19|926.64|2445.33|2497.78|-569.79| +2450815|53201|2450880|76801|90299|3797|42495|76801|90299|3797|42495|2|98|13|1|9466|120|5|7|67.40|138.17|56.64|570.71|396.48|471.80|967.19|23.78|0.00|483.56|396.48|420.26|880.04|903.82|-75.32| +2450815|47169|2450855|31351|528859|5567|23122|31351|528859|5567|23122|1|47|18|4|1909|103|6|21|41.74|85.14|6.81|1644.93|143.01|876.54|1787.94|7.15|0.00|697.20|143.01|150.16|840.21|847.36|-733.53| +2450815|47169|2450841|31351|528859|5567|23122|31351|528859|5567|23122|1|6|19|3|10135|70|6|43|79.58|210.09|44.11|7137.14|1896.73|3421.94|9033.87|4.55|1782.92|361.20|113.81|118.36|475.01|479.56|-3308.13| +2450815|47169|2450872|31351|528859|5567|23122|31351|528859|5567|23122|1|56|2|3|15902|142|6|57|21.60|26.35|24.24|120.27|1381.68|1231.20|1501.95|0.00|0.00|600.78|1381.68|1381.68|1982.46|1982.46|150.48| +2450815|47169|2450828|31351|528859|5567|23122|31351|528859|5567|23122|1|34|15|2|943|181|6|12|83.45|207.79|91.42|1396.44|1097.04|1001.40|2493.48|65.82|0.00|1072.08|1097.04|1162.86|2169.12|2234.94|95.64| +2450815|47169|2450864|31351|528859|5567|23122|31351|528859|5567|23122|1|81|18|5|12055|192|6|75|63.50|133.35|17.33|8701.50|1299.75|4762.50|10001.25|13.25|857.83|1399.50|441.92|455.17|1841.42|1854.67|-4320.58| +2450815|47169|2450871|31351|528859|5567|23122|31351|528859|5567|23122|1|78|13|3|3614|150|6|32|33.46|69.26|67.18|66.56|2149.76|1070.72|2216.32|193.47|0.00|221.44|2149.76|2343.23|2371.20|2564.67|1079.04| +2450815|47169|2450841|31351|528859|5567|23122|31351|528859|5567|23122|1|27|4|1|442|205|6|30|38.50|57.75|25.41|970.20|762.30|1155.00|1732.50|38.11|0.00|796.80|762.30|800.41|1559.10|1597.21|-392.70| +2450815|47169|2450849|31351|528859|5567|23122|31351|528859|5567|23122|1|62|18|5|17852|132|6|53|9.92|25.69|9.24|871.85|489.72|525.76|1361.57|15.08|112.63|122.43|377.09|392.17|499.52|514.60|-148.67| +2450815|47169|2450873|31351|528859|5567|23122|31351|528859|5567|23122|1|54|11|2|6656|55|6|56|1.84|5.35|3.31|114.24|185.36|103.04|299.60|14.82|0.00|104.72|185.36|200.18|290.08|304.90|82.32| +2450815|64129|2450878|16427|192069|5664|34106|16427|192069|5664|34106|1|89|8|1|1894|2|7|48|18.55|52.49|49.34|151.20|2368.32|890.40|2519.52|88.10|165.78|604.32|2202.54|2290.64|2806.86|2894.96|1312.14| +2450815|64129|2450855|16427|192069|5664|34106|16427|192069|5664|34106|1|27|2|1|1723|286|7|69|93.62|245.28|208.48|2539.20|14385.12|6459.78|16924.32|0.00|1006.95|7954.32|13378.17|13378.17|21332.49|21332.49|6918.39| +2450815|64129|2450818|16427|192069|5664|34106|16427|192069|5664|34106|1|65|6|4|9802|255|7|22|40.21|95.29|4.76|1991.66|104.72|884.62|2096.38|0.00|0.00|1027.18|104.72|104.72|1131.90|1131.90|-779.90| +2450815|64129|2450828|16427|192069|5664|34106|16427|192069|5664|34106|1|103|7|3|12013|75|7|26|10.72|19.29|4.82|376.22|125.32|278.72|501.54|5.01|0.00|55.12|125.32|130.33|180.44|185.45|-153.40| +2450815|34427|2450863|42814|271788|6362|38536|42814|271788|6362|38536|1|53|13|4|14618|241|8|46|47.13|76.35|64.13|562.12|2949.98|2167.98|3512.10|146.90|501.49|350.98|2448.49|2595.39|2799.47|2946.37|280.51| +2450815|34427|2450883|42814|271788|6362|38536|42814|271788|6362|38536|1|41|18|5|6139|224|8|83|64.16|150.13|48.04|8473.47|3987.32|5325.28|12460.79|159.49|0.00|3862.82|3987.32|4146.81|7850.14|8009.63|-1337.96| +2450815|34427|2450905|42814|271788|6362|38536|42814|271788|6362|38536|1|45|5|1|2860|61|8|15|72.17|199.18|161.33|567.75|2419.95|1082.55|2987.70|217.79|0.00|1045.65|2419.95|2637.74|3465.60|3683.39|1337.40| +2450815|34427|2450884|42814|271788|6362|38536|42814|271788|6362|38536|1|3|3|5|1826|209|8|27|90.20|183.10|3.66|4844.88|98.82|2435.40|4943.70|3.91|33.59|2273.94|65.23|69.14|2339.17|2343.08|-2370.17| +2450815|34427|2450882|42814|271788|6362|38536|42814|271788|6362|38536|1|53|2|1|13105|26|8|1|66.72|166.80|106.75|60.05|106.75|66.72|166.80|0.00|0.00|0.00|106.75|106.75|106.75|106.75|40.03| +2450815|34427|2450838|42814|271788|6362|38536|42814|271788|6362|38536|1|21|2|1|3140|204|8|62|2.33|3.86|0.57|203.98|35.34|144.46|239.32|0.00|0.00|18.60|35.34|35.34|53.94|53.94|-109.12| +2450815|34427|2450873|42814|271788|6362|38536|42814|271788|6362|38536|1|16|16|2|6950|248|8|49|51.74|83.81|4.19|3901.38|205.31|2535.26|4106.69|10.26|0.00|862.40|205.31|215.57|1067.71|1077.97|-2329.95| +2450815|34427|2450884|42814|271788|6362|38536|42814|271788|6362|38536|1|71|19|3|7360|204|8|57|34.01|77.54|40.32|2121.54|2298.24|1938.57|4419.78|0.00|0.00|1104.66|2298.24|2298.24|3402.90|3402.90|359.67| +2450815|34427|2450845|42814|271788|6362|38536|42814|271788|6362|38536|1|67|4|3|2236|194|8|29|6.77|12.45|3.61|256.36|104.69|196.33|361.05|1.04|0.00|53.94|104.69|105.73|158.63|159.67|-91.64| +2450815|34427|2450903|42814|271788|6362|38536|42814|271788|6362|38536|1|6|20|5|8197|195|8|4|6.26|14.83|5.04|39.16|20.16|25.04|59.32|0.00|0.00|11.84|20.16|20.16|32.00|32.00|-4.88| +2450815|45097|2450875|77674|804458|4395|23579|77674|804458|4395|23579|1|2|12|1|12422|126|9|6|48.56|143.73|137.98|34.50|827.88|291.36|862.38|0.00|0.00|155.22|827.88|827.88|983.10|983.10|536.52| +2450815|45097|2450856|77674|804458|4395|23579|77674|804458|4395|23579|1|1|12|1|17894|121|9|38|62.96|64.84|42.14|862.60|1601.32|2392.48|2463.92|41.31|224.18|1059.44|1377.14|1418.45|2436.58|2477.89|-1015.34| +2450815|45097|2450887|77674|804458|4395|23579|77674|804458|4395|23579|1|14|19|2|10538|113|9|23|57.35|108.39|80.20|648.37|1844.60|1319.05|2492.97|0.00|0.00|1096.87|1844.60|1844.60|2941.47|2941.47|525.55| +2450815|45097|2450870|77674|804458|4395|23579|77674|804458|4395|23579|1|52|12|5|1801|213|9|5|33.22|55.14|18.19|184.75|90.95|166.10|275.70|5.45|0.00|74.40|90.95|96.40|165.35|170.80|-75.15| +2450815|45097|2450823|77674|804458|4395|23579|77674|804458|4395|23579|1|94|14|4|268|42|9|72|95.14|268.29|233.41|2511.36|16805.52|6850.08|19316.88|840.27|0.00|2704.32|16805.52|17645.79|19509.84|20350.11|9955.44| +2450815|45097|2450845|77674|804458|4395|23579|77674|804458|4395|23579|1|61|12|3|13519|75|9|44|97.85|172.21|92.99|3485.68|4091.56|4305.40|7577.24|0.00|0.00|3409.56|4091.56|4091.56|7501.12|7501.12|-213.84| +2450815|80602|2450866|89276|23666|1703|9340|89276|23666|1703|9340|1|36|7|1|2029|120|10|74|74.83|184.83|103.50|6018.42|7659.00|5537.42|13677.42|689.31|0.00|1230.62|7659.00|8348.31|8889.62|9578.93|2121.58| +2450815|80602|2450852|89276|23666|1703|9340|89276|23666|1703|9340|1|43|6|2|6238|1|10|22|76.14|203.29|168.73|760.32|3712.06|1675.08|4472.38|222.72|0.00|402.38|3712.06|3934.78|4114.44|4337.16|2036.98| +2450815|80602|2450900|89276|23666|1703|9340|89276|23666|1703|9340|1|50|9|4|17014|211|10|76|43.24|74.80|63.58|852.72|4832.08|3286.24|5684.80|289.92|0.00|56.24|4832.08|5122.00|4888.32|5178.24|1545.84| +2450815|80602|2450828|89276|23666|1703|9340|89276|23666|1703|9340|1|25|10|1|5704|105|10|55|21.09|48.29|34.28|770.55|1885.40|1159.95|2655.95|131.97|0.00|796.40|1885.40|2017.37|2681.80|2813.77|725.45| +2450815|80602|2450882|89276|23666|1703|9340|89276|23666|1703|9340|1|54|18|2|11908|125|10|90|98.27|142.49|2.84|12568.50|255.60|8844.30|12824.10|15.33|0.00|127.80|255.60|270.93|383.40|398.73|-8588.70| +2450815|80602|2450846|89276|23666|1703|9340|89276|23666|1703|9340|1|24|5|1|9469|61|10|95|4.18|12.12|7.39|449.35|702.05|397.10|1151.40|42.12|0.00|91.20|702.05|744.17|793.25|835.37|304.95| +2450815|80602|2450857|89276|23666|1703|9340|89276|23666|1703|9340|1|33|3|5|9104|276|10|17|67.47|185.54|124.31|1040.91|2113.27|1146.99|3154.18|64.24|507.18|346.80|1606.09|1670.33|1952.89|2017.13|459.10| +2450815|80602|2450818|89276|23666|1703|9340|89276|23666|1703|9340|1|90|17|2|1778|244|10|63|84.32|222.60|115.75|6731.55|7292.25|5312.16|14023.80|291.69|0.00|3786.30|7292.25|7583.94|11078.55|11370.24|1980.09| +2450815|80602|2450889|89276|23666|1703|9340|89276|23666|1703|9340|1|93|3|1|10105|64|10|4|87.55|95.42|5.72|358.80|22.88|350.20|381.68|0.22|0.00|83.96|22.88|23.10|106.84|107.06|-327.32| +2450815|64115|2450880|41731|1046123|5715|32107|41731|1046123|5715|32107|1|10|10|4|2192|80|11|81|45.42|129.44|50.48|6395.76|4088.88|3679.02|10484.64|367.99|0.00|2621.16|4088.88|4456.87|6710.04|7078.03|409.86| +2450815|64115|2450873|41731|1046123|5715|32107|41731|1046123|5715|32107|1|65|9|1|14980|152|11|30|93.79|114.42|75.51|1167.30|2265.30|2813.70|3432.60|31.71|1472.44|308.70|792.86|824.57|1101.56|1133.27|-2020.84| +2450815|64115|2450866|41731|1046123|5715|32107|41731|1046123|5715|32107|1|95|14|2|9296|15|11|33|54.16|107.23|27.87|2618.88|919.71|1787.28|3538.59|45.98|0.00|1698.51|919.71|965.69|2618.22|2664.20|-867.57| +2450815|64115|2450826|41731|1046123|5715|32107|41731|1046123|5715|32107|1|68|13|3|12254|118|11|76|27.04|64.35|7.72|4303.88|586.72|2055.04|4890.60|11.73|0.00|1515.44|586.72|598.45|2102.16|2113.89|-1468.32| +2450815|64115|2450885|41731|1046123|5715|32107|41731|1046123|5715|32107|1|71|6|1|9679|282|11|5|60.75|83.83|19.28|322.75|96.40|303.75|419.15|3.85|0.00|62.85|96.40|100.25|159.25|163.10|-207.35| +2450815|64115|2450828|41731|1046123|5715|32107|41731|1046123|5715|32107|1|65|9|3|14626|161|11|78|49.40|143.75|103.50|3139.50|8073.00|3853.20|11212.50|403.65|0.00|4260.36|8073.00|8476.65|12333.36|12737.01|4219.80| +2450815|64115|2450895|41731|1046123|5715|32107|41731|1046123|5715|32107|1|42|17|5|8335|26|11|32|14.57|30.15|3.31|858.88|105.92|466.24|964.80|4.23|0.00|231.36|105.92|110.15|337.28|341.51|-360.32| +2450815|64115|2450838|41731|1046123|5715|32107|41731|1046123|5715|32107|1|72|20|4|4534|125|11|11|14.54|27.77|1.38|290.29|15.18|159.94|305.47|0.60|0.00|9.13|15.18|15.78|24.31|24.91|-144.76| +2450815|64115|2450831|41731|1046123|5715|32107|41731|1046123|5715|32107|1|41|3|2|3034|96|11|56|96.00|270.72|97.45|9703.12|5457.20|5376.00|15160.32|36.01|4256.61|4699.52|1200.59|1236.60|5900.11|5936.12|-4175.41| +2450815|64115|2450872|41731|1046123|5715|32107|41731|1046123|5715|32107|1|40|9|2|2818|180|11|43|1.71|4.85|1.06|162.97|45.58|73.53|208.55|3.19|0.00|14.19|45.58|48.77|59.77|62.96|-27.95| +2450815|64115|2450852|41731|1046123|5715|32107|41731|1046123|5715|32107|1|35|11|5|5152|67|11|86|73.25|139.90|8.39|11309.86|721.54|6299.50|12031.40|28.86|0.00|239.94|721.54|750.40|961.48|990.34|-5577.96| +2450815|81128|2450857|36748|83584|2773|8362|36748|83584|2773|8362|4|58|12|4|3517|88|12|75|59.96|82.74|77.77|372.75|5832.75|4497.00|6205.50|291.63|0.00|434.25|5832.75|6124.38|6267.00|6558.63|1335.75| +2450815|81128|2450866|36748|83584|2773|8362|36748|83584|2773|8362|4|84|5|5|8389|182|12|40|86.12|120.56|114.53|241.20|4581.20|3444.80|4822.40|274.87|0.00|1108.80|4581.20|4856.07|5690.00|5964.87|1136.40| +2450815|81128|2450843|36748|83584|2773|8362|36748|83584|2773|8362|4|21|6|5|1627|118|12|41|57.21|159.04|98.60|2478.04|4042.60|2345.61|6520.64|0.00|0.00|1956.11|4042.60|4042.60|5998.71|5998.71|1696.99| +2450815|81128|2450876|36748|83584|2773|8362|36748|83584|2773|8362|4|8|1|3|16466|144|12|79|63.39|148.33|93.44|4336.31|7381.76|5007.81|11718.07|180.85|3764.69|1756.96|3617.07|3797.92|5374.03|5554.88|-1390.74| +2450815|81128|2450829|36748|83584|2773|8362|36748|83584|2773|8362|4|37|18|3|14326|60|12|89|74.58|146.17|62.85|7415.48|5593.65|6637.62|13009.13|335.61|0.00|4683.18|5593.65|5929.26|10276.83|10612.44|-1043.97| +2450815|81128|2450889|36748|83584|2773|8362|36748|83584|2773|8362|4|11|16|3|1940|84|12|84|80.29|89.12|34.75|4567.08|2919.00|6744.36|7486.08|116.76|0.00|2394.84|2919.00|3035.76|5313.84|5430.60|-3825.36| +2450815|31835|2450852|36022|1502404|6833|19662|36022|1502404|6833|19662|1|35|2|5|14164|294|13|51|43.78|114.26|22.85|4661.91|1165.35|2232.78|5827.26|23.30|0.00|2563.77|1165.35|1188.65|3729.12|3752.42|-1067.43| +2450815|31835|2450878|36022|1502404|6833|19662|36022|1502404|6833|19662|1|18|9|2|2389|153|13|100|13.66|16.93|2.03|1490.00|203.00|1366.00|1693.00|0.00|0.00|220.00|203.00|203.00|423.00|423.00|-1163.00| +2450815|31835|2450835|36022|1502404|6833|19662|36022|1502404|6833|19662|1|43|4|4|13732|132|13|25|11.88|22.33|5.58|418.75|139.50|297.00|558.25|6.06|18.13|83.50|121.37|127.43|204.87|210.93|-175.63| +2450815|31835|2450904|36022|1502404|6833|19662|36022|1502404|6833|19662|1|6|16|4|12577|48|13|81|44.40|94.12|66.82|2211.30|5412.42|3596.40|7623.72|162.37|0.00|304.56|5412.42|5574.79|5716.98|5879.35|1816.02| +2450815|31835|2450895|36022|1502404|6833|19662|36022|1502404|6833|19662|1|49|7|2|3056|10|13|65|34.85|66.56|8.65|3764.15|562.25|2265.25|4326.40|50.60|0.00|1254.50|562.25|612.85|1816.75|1867.35|-1703.00| +2450815|31835|2450876|36022|1502404|6833|19662|36022|1502404|6833|19662|1|64|13|4|4778|279|13|98|40.95|104.83|72.33|3185.00|7088.34|4013.10|10273.34|85.06|5387.13|2773.40|1701.21|1786.27|4474.61|4559.67|-2311.89| +2450815|31835|2450857|36022|1502404|6833|19662|36022|1502404|6833|19662|1|36|18|1|2216|154|13|95|15.77|20.50|12.91|721.05|1226.45|1498.15|1947.50|73.58|0.00|506.35|1226.45|1300.03|1732.80|1806.38|-271.70| +2450815|31835|2450890|36022|1502404|6833|19662|36022|1502404|6833|19662|1|11|3|4|286|43|13|83|84.67|94.83|41.72|4408.13|3462.76|7027.61|7870.89|69.25|0.00|943.71|3462.76|3532.01|4406.47|4475.72|-3564.85| +2450815|31835|2450860|36022|1502404|6833|19662|36022|1502404|6833|19662|1|91|8|1|10066|9|13|52|27.17|79.06|26.08|2754.96|1356.16|1412.84|4111.12|40.68|0.00|1562.08|1356.16|1396.84|2918.24|2958.92|-56.68| +2450815|31835|2450822|36022|1502404|6833|19662|36022|1502404|6833|19662|1|23|7|1|13628|203|13|61|19.60|43.12|43.12|0.00|2630.32|1195.60|2630.32|52.60|0.00|841.19|2630.32|2682.92|3471.51|3524.11|1434.72| +2450815|31835|2450838|36022|1502404|6833|19662|36022|1502404|6833|19662|1|62|11|1|14527|92|13|67|79.67|89.23|29.44|4005.93|1972.48|5337.89|5978.41|138.07|0.00|2152.04|1972.48|2110.55|4124.52|4262.59|-3365.41| +2450815|77322|2450847|19226|1917174|722|39470|19226|1917174|722|39470|4|57|7|1|9190|286|14|58|85.87|146.83|82.22|3747.38|4768.76|4980.46|8516.14|286.12|0.00|4172.52|4768.76|5054.88|8941.28|9227.40|-211.70| +2450815|77322|2450865|19226|1917174|722|39470|19226|1917174|722|39470|4|30|4|2|14902|41|14|72|79.34|194.38|66.08|9237.60|4757.76|5712.48|13995.36|380.62|0.00|4338.00|4757.76|5138.38|9095.76|9476.38|-954.72| +2450815|77322|2450820|19226|1917174|722|39470|19226|1917174|722|39470|4|40|2|4|6170|198|14|37|38.63|104.30|86.56|656.38|3202.72|1429.31|3859.10|32.02|0.00|192.77|3202.72|3234.74|3395.49|3427.51|1773.41| +2450815|77322|2450905|19226|1917174|722|39470|19226|1917174|722|39470|4|82|3|1|8917|227|14|16|18.27|42.75|19.66|369.44|314.56|292.32|684.00|6.29|0.00|54.72|314.56|320.85|369.28|375.57|22.24| +2450815|39455|2450855|86950|800085|1573|23461|86950|800085|1573|23461|1|94|11|1|4372|38|15|17|8.33|11.32|1.13|173.23|19.21|141.61|192.44|1.34|0.00|32.64|19.21|20.55|51.85|53.19|-122.40| +2450815|39455|2450844|86950|800085|1573|23461|86950|800085|1573|23461|1|94|16|3|8632|164|15|6|78.30|149.55|64.30|511.50|385.80|469.80|897.30|34.72|0.00|421.68|385.80|420.52|807.48|842.20|-84.00| +2450815|39455|2450849|86950|800085|1573|23461|86950|800085|1573|23461|1|74|1|4|2044|178|15|40|55.49|77.13|64.01|524.80|2560.40|2219.60|3085.20|9.21|2406.77|616.80|153.63|162.84|770.43|779.64|-2065.97| +2450815|39455|2450837|86950|800085|1573|23461|86950|800085|1573|23461|1|38|2|3|2398|242|15|14|67.49|120.80|14.49|1488.34|202.86|944.86|1691.20|4.05|0.00|185.92|202.86|206.91|388.78|392.83|-742.00| +2450815|39455|2450858|86950|800085|1573|23461|86950|800085|1573|23461|1|30|14|4|12016|132|15|82|27.65|61.93|56.35|457.56|4620.70|2267.30|5078.26|134.46|138.62|2284.52|4482.08|4616.54|6766.60|6901.06|2214.78| +2450815|39455|2450872|86950|800085|1573|23461|86950|800085|1573|23461|1|66|6|1|1906|299|15|81|92.67|222.40|175.69|3783.51|14230.89|7506.27|18014.40|426.92|0.00|900.72|14230.89|14657.81|15131.61|15558.53|6724.62| +2450815|39455|2450871|86950|800085|1573|23461|86950|800085|1573|23461|1|79|13|2|13951|51|15|86|82.76|201.93|193.85|694.88|16671.10|7117.36|17365.98|1500.39|0.00|8682.56|16671.10|18171.49|25353.66|26854.05|9553.74| +2450815|39455|2450875|86950|800085|1573|23461|86950|800085|1573|23461|1|69|20|1|4088|291|15|76|84.54|157.24|144.66|956.08|10994.16|6425.04|11950.24|659.64|0.00|4062.96|10994.16|11653.80|15057.12|15716.76|4569.12| +2450815|39455|2450885|86950|800085|1573|23461|86950|800085|1573|23461|1|94|18|1|17374|26|15|1|64.25|160.62|96.37|64.25|96.37|64.25|160.62|1.92|0.00|33.73|96.37|98.29|130.10|132.02|32.12| +2450815|52486|2450857|24957|189998|3839|7327|24957|189998|3839|7327|2|55|11|5|17794|141|16|26|84.94|174.12|31.34|3712.28|814.84|2208.44|4527.12|57.20|179.26|905.32|635.58|692.78|1540.90|1598.10|-1572.86| +2450815|52486|2450828|24957|189998|3839|7327|24957|189998|3839|7327|2|56|15|2|2128|291|16|69|15.66|23.17|10.19|895.62|703.11|1080.54|1598.73|56.24|0.00|191.82|703.11|759.35|894.93|951.17|-377.43| +2450815|52486|2450868|24957|189998|3839|7327|24957|189998|3839|7327|2|103|3|2|6103|95|16|38|60.61|85.46|45.29|1526.46|1721.02|2303.18|3247.48|34.42|0.00|0.00|1721.02|1755.44|1721.02|1755.44|-582.16| +2450815|52486|2450846|24957|189998|3839|7327|24957|189998|3839|7327|2|62|7|1|12268|174|16|76|67.07|179.74|149.18|2322.56|11337.68|5097.32|13660.24|907.01|0.00|2321.80|11337.68|12244.69|13659.48|14566.49|6240.36| +2450815|52486|2450880|24957|189998|3839|7327|24957|189998|3839|7327|2|34|7|1|848|212|16|2|71.37|106.34|42.53|127.62|85.06|142.74|212.68|0.85|0.00|106.34|85.06|85.91|191.40|192.25|-57.68| +2450815|52486|2450838|24957|189998|3839|7327|24957|189998|3839|7327|2|7|16|1|11071|256|16|48|53.49|136.93|73.94|3023.52|3549.12|2567.52|6572.64|0.00|0.00|1380.00|3549.12|3549.12|4929.12|4929.12|981.60| +2450815|52486|2450858|24957|189998|3839|7327|24957|189998|3839|7327|2|50|13|1|17056|282|16|86|74.44|140.69|75.97|5565.92|6533.42|6401.84|12099.34|0.00|0.00|604.58|6533.42|6533.42|7138.00|7138.00|131.58| +2450815|52486|2450888|24957|189998|3839|7327|24957|189998|3839|7327|2|75|5|3|9586|7|16|61|62.05|153.26|49.04|6357.42|2991.44|3785.05|9348.86|59.82|0.00|2243.58|2991.44|3051.26|5235.02|5294.84|-793.61| +2450815|52486|2450901|24957|189998|3839|7327|24957|189998|3839|7327|2|1|19|3|14528|121|16|59|45.40|120.76|42.26|4631.50|2493.34|2678.60|7124.84|0.00|0.00|3206.06|2493.34|2493.34|5699.40|5699.40|-185.26| +2450815|15952|2450875|99796|1235795|6547|44499|99796|1235795|6547|44499|4|9|7|5|7258|206|17|100|95.79|144.64|78.10|6654.00|7810.00|9579.00|14464.00|156.20|0.00|3037.00|7810.00|7966.20|10847.00|11003.20|-1769.00| +2450815|15952|2450859|99796|1235795|6547|44499|99796|1235795|6547|44499|4|19|11|2|295|83|17|26|51.87|128.63|29.58|2575.30|769.08|1348.62|3344.38|7.69|0.00|601.90|769.08|776.77|1370.98|1378.67|-579.54| +2450815|15952|2450818|99796|1235795|6547|44499|99796|1235795|6547|44499|4|30|16|2|17842|227|17|14|48.22|116.69|4.66|1568.42|65.24|675.08|1633.66|5.87|0.00|702.38|65.24|71.11|767.62|773.49|-609.84| +2450815|15952|2450820|99796|1235795|6547|44499|99796|1235795|6547|44499|4|95|13|1|5383|42|17|4|11.25|18.78|14.27|18.04|57.08|45.00|75.12|2.28|0.00|12.00|57.08|59.36|69.08|71.36|12.08| +2450815|29616|2450853|82003|599432|3708|27937|82003|599432|3708|27937|4|32|5|1|2188|66|18|45|30.67|52.13|45.35|305.10|2040.75|1380.15|2345.85|122.44|0.00|656.55|2040.75|2163.19|2697.30|2819.74|660.60| +2450815|29616|2450839|82003|599432|3708|27937|82003|599432|3708|27937|4|42|4|4|7531|191|18|17|13.80|40.98|16.80|411.06|285.60|234.60|696.66|22.84|0.00|13.77|285.60|308.44|299.37|322.21|51.00| +2450815|29616|2450850|82003|599432|3708|27937|82003|599432|3708|27937|4|3|5|3|17353|294|18|43|53.09|92.90|44.59|2077.33|1917.37|2282.87|3994.70|134.21|0.00|119.54|1917.37|2051.58|2036.91|2171.12|-365.50| +2450815|29616|2450843|82003|599432|3708|27937|82003|599432|3708|27937|4|56|2|3|12124|142|18|13|34.96|40.90|17.17|308.49|223.21|454.48|531.70|2.23|0.00|148.85|223.21|225.44|372.06|374.29|-231.27| +2450815|29616|2450902|82003|599432|3708|27937|82003|599432|3708|27937|4|64|11|4|9842|76|18|15|79.76|124.42|116.95|112.05|1754.25|1196.40|1866.30|70.17|0.00|242.55|1754.25|1824.42|1996.80|2066.97|557.85| +2450815|29616|2450828|82003|599432|3708|27937|82003|599432|3708|27937|4|69|19|3|362|40|18|35|27.85|70.46|30.29|1405.95|1060.15|974.75|2466.10|0.00|0.00|567.00|1060.15|1060.15|1627.15|1627.15|85.40| +2450815|29616|2450833|82003|599432|3708|27937|82003|599432|3708|27937|4|76|4|1|3856|23|18|55|10.63|11.37|1.70|531.85|93.50|584.65|625.35|4.67|0.00|6.05|93.50|98.17|99.55|104.22|-491.15| +2450815|29616|2450880|82003|599432|3708|27937|82003|599432|3708|27937|4|9|11|3|5185|166|18|18|57.78|60.66|33.96|480.60|611.28|1040.04|1091.88|55.01|0.00|305.64|611.28|666.29|916.92|971.93|-428.76| +2450815|29616|2450894|82003|599432|3708|27937|82003|599432|3708|27937|4|34|11|2|4526|170|18|92|74.73|95.65|90.86|440.68|8359.12|6875.16|8799.80|250.77|0.00|87.40|8359.12|8609.89|8446.52|8697.29|1483.96| +2450815|29616|2450841|82003|599432|3708|27937|82003|599432|3708|27937|4|25|7|4|13591|138|18|19|17.59|19.34|5.60|261.06|106.40|334.21|367.46|1.06|0.00|154.28|106.40|107.46|260.68|261.74|-227.81| +2450815|29616|2450880|82003|599432|3708|27937|82003|599432|3708|27937|4|33|4|1|4051|103|18|67|91.70|270.51|59.51|14137.00|3987.17|6143.90|18124.17|46.65|3468.83|180.90|518.34|564.99|699.24|745.89|-5625.56| +2450815|43898|2450843|27389|27132|4828|48171|27389|27132|4828|48171|4|89|2|1|10330|289|19|98|37.42|108.89|60.97|4696.16|5975.06|3667.16|10671.22|537.75|0.00|3521.14|5975.06|6512.81|9496.20|10033.95|2307.90| +2450815|43898|2450844|27389|27132|4828|48171|27389|27132|4828|48171|4|94|12|4|12655|11|19|91|55.74|63.54|4.44|5378.10|404.04|5072.34|5782.14|20.20|0.00|172.90|404.04|424.24|576.94|597.14|-4668.30| +2450815|43898|2450869|27389|27132|4828|48171|27389|27132|4828|48171|4|8|10|3|7480|227|19|26|4.56|11.76|11.05|18.46|287.30|118.56|305.76|20.11|0.00|73.32|287.30|307.41|360.62|380.73|168.74| +2450815|43898|2450899|27389|27132|4828|48171|27389|27132|4828|48171|4|22|3|5|5218|199|19|38|38.76|95.34|68.64|1014.60|2608.32|1472.88|3622.92|234.74|0.00|0.00|2608.32|2843.06|2608.32|2843.06|1135.44| +2450815|43898|2450828|27389|27132|4828|48171|27389|27132|4828|48171|4|100|16|5|11774|66|19|83|85.12|176.19|140.95|2924.92|11698.85|7064.96|14623.77|350.96|0.00|1315.55|11698.85|12049.81|13014.40|13365.36|4633.89| +2450815|43898|2450864|27389|27132|4828|48171|27389|27132|4828|48171|4|18|11|3|13340|277|19|38|14.39|28.92|11.56|659.68|439.28|546.82|1098.96|8.78|0.00|362.52|439.28|448.06|801.80|810.58|-107.54| +2450815|40538|2450897|24357|27634|1311|11435|24357|27634|1311|11435|1|85|1|1|2312|229|20|50|76.90|103.81|40.48|3166.50|2024.00|3845.00|5190.50|161.92|0.00|1609.00|2024.00|2185.92|3633.00|3794.92|-1821.00| +2450815|40538|2450891|24357|27634|1311|11435|24357|27634|1311|11435|1|62|14|5|8462|288|20|77|59.49|150.50|88.79|4751.67|6836.83|4580.73|11588.50|478.57|0.00|2201.43|6836.83|7315.40|9038.26|9516.83|2256.10| +2450815|40538|2450834|24357|27634|1311|11435|24357|27634|1311|11435|1|63|19|1|3008|245|20|68|31.01|72.25|35.40|2505.80|2407.20|2108.68|4913.00|72.21|0.00|2210.68|2407.20|2479.41|4617.88|4690.09|298.52| +2450815|40538|2450858|24357|27634|1311|11435|24357|27634|1311|11435|1|65|8|3|12944|279|20|39|91.84|142.35|11.38|5107.83|443.82|3581.76|5551.65|22.90|62.13|499.59|381.69|404.59|881.28|904.18|-3200.07| +2450815|40538|2450847|24357|27634|1311|11435|24357|27634|1311|11435|1|39|15|2|13963|156|20|20|41.60|50.75|28.42|446.60|568.40|832.00|1015.00|17.05|0.00|477.00|568.40|585.45|1045.40|1062.45|-263.60| +2450815|40538|2450857|24357|27634|1311|11435|24357|27634|1311|11435|1|14|16|1|3670|69|20|3|40.97|81.12|29.20|155.76|87.60|122.91|243.36|7.88|0.00|94.89|87.60|95.48|182.49|190.37|-35.31| +2450815|40538|2450828|24357|27634|1311|11435|24357|27634|1311|11435|1|61|7|4|11227|99|20|56|39.64|60.25|25.30|1957.20|1416.80|2219.84|3374.00|85.00|0.00|539.84|1416.80|1501.80|1956.64|2041.64|-803.04| +2450815|40538|2450853|24357|27634|1311|11435|24357|27634|1311|11435|1|31|7|5|12910|180|20|78|32.85|32.85|6.89|2024.88|537.42|2562.30|2562.30|16.12|0.00|845.52|537.42|553.54|1382.94|1399.06|-2024.88| +2450815|62858|2450895|75751|1766792|2633|25674|75751|1766792|2633|25674|4|100|14|5|5074|77|21|51|80.32|82.72|33.08|2531.64|1687.08|4096.32|4218.72|84.35|0.00|210.63|1687.08|1771.43|1897.71|1982.06|-2409.24| +2450815|62858|2450830|75751|1766792|2633|25674|75751|1766792|2633|25674|4|45|10|3|9152|186|21|66|61.43|163.40|140.52|1510.08|9274.32|4054.38|10784.40|80.68|1205.66|3558.72|8068.66|8149.34|11627.38|11708.06|4014.28| +2450815|62858|2450894|75751|1766792|2633|25674|75751|1766792|2633|25674|4|74|4|4|10184|91|21|37|2.83|8.20|8.03|6.29|297.11|104.71|303.40|23.76|0.00|8.88|297.11|320.87|305.99|329.75|192.40| +2450815|62858|2450891|75751|1766792|2633|25674|75751|1766792|2633|25674|4|75|13|2|43|57|21|49|68.10|190.68|131.56|2896.88|6446.44|3336.90|9343.32|386.78|0.00|933.94|6446.44|6833.22|7380.38|7767.16|3109.54| +2450815|62858|2450861|75751|1766792|2633|25674|75751|1766792|2633|25674|4|42|13|3|1213|286|21|76|10.60|24.91|24.41|38.00|1855.16|805.60|1893.16|30.05|1521.23|340.48|333.93|363.98|674.41|704.46|-471.67| +2450815|62858|2450884|75751|1766792|2633|25674|75751|1766792|2633|25674|4|90|7|1|17380|73|21|28|27.16|54.86|2.74|1459.36|76.72|760.48|1536.08|3.83|0.00|445.20|76.72|80.55|521.92|525.75|-683.76| +2450815|62858|2450891|75751|1766792|2633|25674|75751|1766792|2633|25674|4|104|19|2|14474|50|21|51|7.01|7.92|6.96|48.96|354.96|357.51|403.92|0.00|0.00|189.72|354.96|354.96|544.68|544.68|-2.55| +2450815|62858|2450857|75751|1766792|2633|25674|75751|1766792|2633|25674|4|47|16|2|9484|83|21|64|95.78|132.17|62.11|4483.84|3975.04|6129.92|8458.88|79.50|0.00|3045.12|3975.04|4054.54|7020.16|7099.66|-2154.88| +2450815|79044|2450821|11242|541252|6015|7978|11242|541252|6015|7978|2|30|7|4|9415|80|22|7|57.77|143.26|4.29|972.79|30.03|404.39|1002.82|2.10|0.00|360.99|30.03|32.13|391.02|393.12|-374.36| +2450815|79044|2450850|11242|541252|6015|7978|11242|541252|6015|7978|2|45|7|1|6014|49|22|49|77.33|120.63|94.09|1300.46|4610.41|3789.17|5910.87|46.10|0.00|1595.93|4610.41|4656.51|6206.34|6252.44|821.24| +2450815|79044|2450891|11242|541252|6015|7978|11242|541252|6015|7978|2|40|10|4|8452|16|22|94|43.63|67.19|49.04|1706.10|4609.76|4101.22|6315.86|276.58|0.00|3157.46|4609.76|4886.34|7767.22|8043.80|508.54| +2450815|79044|2450905|11242|541252|6015|7978|11242|541252|6015|7978|2|34|8|3|4186|293|22|67|13.03|38.56|35.86|180.90|2402.62|873.01|2583.52|48.05|1441.57|1007.01|961.05|1009.10|1968.06|2016.11|88.04| +2450815|79044|2450823|11242|541252|6015|7978|11242|541252|6015|7978|2|21|12|4|11512|20|22|6|56.52|124.34|32.32|552.12|193.92|339.12|746.04|15.51|0.00|246.18|193.92|209.43|440.10|455.61|-145.20| +2450815|79044|2450880|11242|541252|6015|7978|11242|541252|6015|7978|2|77|7|3|13442|165|22|69|97.02|195.98|92.11|7167.03|6355.59|6694.38|13522.62|444.89|0.00|4056.51|6355.59|6800.48|10412.10|10856.99|-338.79| +2450815|79044|2450887|11242|541252|6015|7978|11242|541252|6015|7978|2|21|11|4|1501|102|22|1|78.01|78.01|49.92|28.09|49.92|78.01|78.01|0.00|41.93|15.60|7.99|7.99|23.59|23.59|-70.02| +2450815|79044|2450821|11242|541252|6015|7978|11242|541252|6015|7978|2|58|15|3|16498|61|22|98|9.97|15.65|11.11|444.92|1088.78|977.06|1533.70|54.43|0.00|122.50|1088.78|1143.21|1211.28|1265.71|111.72| +2450815|79044|2450892|11242|541252|6015|7978|11242|541252|6015|7978|2|100|16|1|12604|176|22|38|3.42|5.67|0.62|191.90|23.56|129.96|215.46|0.94|0.00|36.48|23.56|24.50|60.04|60.98|-106.40| +2450815|79044|2450894|11242|541252|6015|7978|11242|541252|6015|7978|2|5|4|2|10573|131|22|2|57.25|62.40|31.82|61.16|63.64|114.50|124.80|3.18|0.00|61.14|63.64|66.82|124.78|127.96|-50.86| +2450815|79044|2450848|11242|541252|6015|7978|11242|541252|6015|7978|2|70|6|2|13129|176|22|7|35.80|36.87|36.87|0.00|258.09|250.60|258.09|5.16|0.00|87.71|258.09|263.25|345.80|350.96|7.49| +2450815|79044|2450857|11242|541252|6015|7978|11242|541252|6015|7978|2|22|1|2|13778|134|22|4|33.82|68.99|33.80|140.76|135.20|135.28|275.96|0.81|114.92|121.40|20.28|21.09|141.68|142.49|-115.00| +2450815|79044|2450832|11242|541252|6015|7978|11242|541252|6015|7978|2|102|16|2|4204|244|22|11|29.36|67.23|37.64|325.49|414.04|322.96|739.53|19.75|194.59|36.96|219.45|239.20|256.41|276.16|-103.51| +2450815|79044|2450848|11242|541252|6015|7978|11242|541252|6015|7978|2|75|17|4|5632|243|22|60|87.75|253.59|114.11|8368.80|6846.60|5265.00|15215.40|597.70|205.39|151.80|6641.21|7238.91|6793.01|7390.71|1376.21| +2450815|6756|2450903|44703|600082|6661|6087|93499|1627645|1424|10050|4|6|15|4|10387|252|23|13|24.25|59.17|17.75|538.46|230.75|315.25|769.21|6.92|0.00|99.97|230.75|237.67|330.72|337.64|-84.50| +2450815|6756|2450879|44703|600082|6661|6087|93499|1627645|1424|10050|4|40|20|3|8192|179|23|25|56.76|158.36|133.02|633.50|3325.50|1419.00|3959.00|0.00|0.00|475.00|3325.50|3325.50|3800.50|3800.50|1906.50| +2450815|6756|2450875|44703|600082|6661|6087|93499|1627645|1424|10050|4|68|4|5|7240|122|23|32|63.89|125.86|52.86|2336.00|1691.52|2044.48|4027.52|16.91|0.00|362.24|1691.52|1708.43|2053.76|2070.67|-352.96| +2450815|6756|2450886|44703|600082|6661|6087|93499|1627645|1424|10050|4|2|18|2|5236|195|23|22|86.40|101.95|70.34|695.42|1547.48|1900.80|2242.90|139.27|0.00|740.08|1547.48|1686.75|2287.56|2426.83|-353.32| +2450815|6756|2450829|44703|600082|6661|6087|93499|1627645|1424|10050|4|55|17|3|8653|149|23|39|20.20|59.38|26.72|1273.74|1042.08|787.80|2315.82|0.00|31.26|1111.50|1010.82|1010.82|2122.32|2122.32|223.02| +2450815|6756|2450837|44703|600082|6661|6087|93499|1627645|1424|10050|4|73|16|3|2104|202|23|75|74.14|146.05|65.72|6024.75|4929.00|5560.50|10953.75|147.87|0.00|5148.00|4929.00|5076.87|10077.00|10224.87|-631.50| +2450815|6756|2450852|44703|600082|6661|6087|93499|1627645|1424|10050|4|17|1|5|6736|68|23|34|57.65|103.77|31.13|2469.76|1058.42|1960.10|3528.18|52.92|0.00|811.24|1058.42|1111.34|1869.66|1922.58|-901.68| +2450815|6756|2450885|44703|600082|6661|6087|93499|1627645|1424|10050|4|74|4|3|7426|208|23|98|17.90|18.25|0.36|1753.22|35.28|1754.20|1788.50|0.00|28.22|733.04|7.06|7.06|740.10|740.10|-1747.14| +2450815|6756|2450878|44703|600082|6661|6087|93499|1627645|1424|10050|4|73|15|3|8155|171|23|18|10.31|13.60|10.74|51.48|193.32|185.58|244.80|3.86|0.00|53.82|193.32|197.18|247.14|251.00|7.74| +2450815|6756|2450904|44703|600082|6661|6087|93499|1627645|1424|10050|4|7|12|1|6476|144|23|93|2.90|8.70|2.43|583.11|225.99|269.70|809.10|20.33|0.00|48.36|225.99|246.32|274.35|294.68|-43.71| +2450815|6756|2450891|44703|600082|6661|6087|93499|1627645|1424|10050|4|28|17|4|7226|278|23|97|46.81|139.02|31.97|10383.85|3101.09|4540.57|13484.94|248.08|0.00|3910.07|3101.09|3349.17|7011.16|7259.24|-1439.48| +2450815|6756|2450845|44703|600082|6661|6087|93499|1627645|1424|10050|4|45|7|1|8200|21|23|86|70.53|107.91|85.24|1949.62|7330.64|6065.58|9280.26|219.91|0.00|4639.70|7330.64|7550.55|11970.34|12190.25|1265.06| +2450815|6756|2450860|44703|600082|6661|6087|93499|1627645|1424|10050|4|31|20|4|7874|19|23|78|30.12|77.40|36.37|3200.34|2836.86|2349.36|6037.20|15.88|2439.69|301.86|397.17|413.05|699.03|714.91|-1952.19| +2450815|64186|2450837|58260|1431849|5438|42627|58260|1431849|5438|42627|1|67|5|3|11518|166|24|41|3.83|7.92|7.76|6.56|318.16|157.03|324.72|19.08|0.00|116.85|318.16|337.24|435.01|454.09|161.13| +2450815|64186|2450839|58260|1431849|5438|42627|58260|1431849|5438|42627|1|21|12|2|4262|90|24|3|25.19|59.19|29.59|88.80|88.77|75.57|177.57|0.84|4.43|88.77|84.34|85.18|173.11|173.95|8.77| +2450815|64186|2450826|58260|1431849|5438|42627|58260|1431849|5438|42627|1|79|17|2|106|96|24|99|10.54|22.34|1.11|2101.77|109.89|1043.46|2211.66|2.19|0.00|618.75|109.89|112.08|728.64|730.83|-933.57| +2450815|64186|2450868|58260|1431849|5438|42627|58260|1431849|5438|42627|1|43|16|5|15860|180|24|13|9.22|25.26|3.53|282.49|45.89|119.86|328.38|0.45|0.00|22.88|45.89|46.34|68.77|69.22|-73.97| +2450815|78270|2450845|73381|352036|738|10593|73381|352036|738|10593|2|50|16|2|15151|144|25|67|49.23|84.67|58.42|1758.75|3914.14|3298.41|5672.89|156.56|0.00|1644.85|3914.14|4070.70|5558.99|5715.55|615.73| +2450815|78270|2450896|73381|352036|738|10593|73381|352036|738|10593|2|78|7|2|4930|271|25|86|31.01|38.45|34.60|331.10|2975.60|2666.86|3306.70|238.04|0.00|165.12|2975.60|3213.64|3140.72|3378.76|308.74| +2450815|78270|2450888|73381|352036|738|10593|73381|352036|738|10593|2|95|14|2|16526|230|25|62|36.02|54.03|39.98|871.10|2478.76|2233.24|3349.86|24.78|0.00|1674.62|2478.76|2503.54|4153.38|4178.16|245.52| +2450815|78270|2450867|73381|352036|738|10593|73381|352036|738|10593|2|76|5|1|2002|275|25|4|31.53|90.17|68.52|86.60|274.08|126.12|360.68|5.48|0.00|28.84|274.08|279.56|302.92|308.40|147.96| +2450815|78270|2450886|73381|352036|738|10593|73381|352036|738|10593|2|89|11|5|730|254|25|27|45.85|60.98|26.22|938.52|707.94|1237.95|1646.46|28.31|0.00|526.77|707.94|736.25|1234.71|1263.02|-530.01| +2450815|75465|2450851|55764|1379528|6492|31056|55764|1379528|6492|31056|1|91|15|4|1256|102|26|19|93.76|111.57|103.76|148.39|1971.44|1781.44|2119.83|98.57|0.00|211.85|1971.44|2070.01|2183.29|2281.86|190.00| +2450815|75465|2450886|55764|1379528|6492|31056|55764|1379528|6492|31056|1|85|8|1|12379|195|26|81|19.49|23.19|17.85|432.54|1445.85|1578.69|1878.39|14.45|0.00|168.48|1445.85|1460.30|1614.33|1628.78|-132.84| +2450815|75465|2450877|55764|1379528|6492|31056|55764|1379528|6492|31056|1|61|18|1|7186|112|26|20|60.90|154.68|94.35|1206.60|1887.00|1218.00|3093.60|150.96|0.00|1020.80|1887.00|2037.96|2907.80|3058.76|669.00| +2450815|75465|2450824|55764|1379528|6492|31056|55764|1379528|6492|31056|1|26|14|2|10906|141|26|58|1.76|3.94|1.02|169.36|59.16|102.08|228.52|0.00|0.00|81.78|59.16|59.16|140.94|140.94|-42.92| +2450815|55294|2450819|65067|54728|6342|46803|65067|54728|6342|46803|1|8|10|3|7916|229|27|63|57.27|102.51|30.75|4520.88|1937.25|3608.01|6458.13|0.00|0.00|258.30|1937.25|1937.25|2195.55|2195.55|-1670.76| +2450815|55294|2450892|65067|54728|6342|46803|65067|54728|6342|46803|1|59|4|3|10270|179|27|58|97.19|185.63|5.56|10444.06|322.48|5637.02|10766.54|19.34|0.00|107.30|322.48|341.82|429.78|449.12|-5314.54| +2450815|55294|2450865|65067|54728|6342|46803|65067|54728|6342|46803|1|60|19|2|7543|146|27|73|64.60|98.19|84.44|1003.75|6164.12|4715.80|7167.87|123.28|0.00|3009.79|6164.12|6287.40|9173.91|9297.19|1448.32| +2450815|55294|2450894|65067|54728|6342|46803|65067|54728|6342|46803|1|62|3|4|10192|79|27|67|67.68|176.64|51.22|8403.14|3431.74|4534.56|11834.88|0.00|0.00|5798.85|3431.74|3431.74|9230.59|9230.59|-1102.82| +2450815|55294|2450834|65067|54728|6342|46803|65067|54728|6342|46803|1|95|19|3|16924|274|27|85|97.34|162.55|89.40|6217.75|7599.00|8273.90|13816.75|0.00|0.00|4835.65|7599.00|7599.00|12434.65|12434.65|-674.90| +2450815|55294|2450830|65067|54728|6342|46803|65067|54728|6342|46803|1|78|14|3|13178|269|27|45|15.98|39.63|15.05|1106.10|677.25|719.10|1783.35|27.09|0.00|124.65|677.25|704.34|801.90|828.99|-41.85| +2450815|55294|2450871|65067|54728|6342|46803|65067|54728|6342|46803|1|105|8|1|16916|177|27|76|74.75|147.25|54.48|7050.52|4140.48|5681.00|11191.00|41.40|0.00|1454.64|4140.48|4181.88|5595.12|5636.52|-1540.52| +2450815|55294|2450903|65067|54728|6342|46803|65067|54728|6342|46803|1|68|8|5|9163|71|27|82|43.43|90.76|29.95|4986.42|2455.90|3561.26|7442.32|55.99|1522.65|2604.32|933.25|989.24|3537.57|3593.56|-2628.01| +2450815|55294|2450848|65067|54728|6342|46803|65067|54728|6342|46803|1|7|12|4|1294|80|27|80|38.23|107.80|31.26|6123.20|2500.80|3058.40|8624.00|0.00|0.00|3535.20|2500.80|2500.80|6036.00|6036.00|-557.60| +2450815|55294|2450844|65067|54728|6342|46803|65067|54728|6342|46803|1|78|8|5|17746|290|27|74|27.26|34.07|0.34|2496.02|25.16|2017.24|2521.18|0.00|0.00|75.48|25.16|25.16|100.64|100.64|-1992.08| +2450815|55294|2450903|65067|54728|6342|46803|65067|54728|6342|46803|1|45|10|3|7028|172|27|90|32.66|67.93|43.47|2201.40|3912.30|2939.40|6113.70|312.98|0.00|2016.90|3912.30|4225.28|5929.20|6242.18|972.90| +2450815|55294|2450903|65067|54728|6342|46803|65067|54728|6342|46803|1|87|17|5|10807|169|27|52|20.86|48.81|31.23|914.16|1623.96|1084.72|2538.12|16.56|1071.81|837.20|552.15|568.71|1389.35|1405.91|-532.57| +2450815|69862|2450832|62042|643025|4008|23791|62042|643025|4008|23791|2|25|7|4|6178|287|28|28|96.96|151.25|119.48|889.56|3345.44|2714.88|4235.00|200.72|0.00|338.80|3345.44|3546.16|3684.24|3884.96|630.56| +2450815|69862|2450826|62042|643025|4008|23791|62042|643025|4008|23791|2|9|13|2|17690|35|28|24|2.69|7.55|5.05|60.00|121.20|64.56|181.20|4.65|4.84|74.16|116.36|121.01|190.52|195.17|51.80| +2450815|69862|2450824|62042|643025|4008|23791|62042|643025|4008|23791|2|69|3|1|16951|5|28|47|45.83|85.70|16.28|3262.74|765.16|2154.01|4027.90|22.95|0.00|1611.16|765.16|788.11|2376.32|2399.27|-1388.85| +2450815|69862|2450885|62042|643025|4008|23791|62042|643025|4008|23791|2|34|3|1|12919|289|28|38|82.23|231.88|39.41|7313.86|1497.58|3124.74|8811.44|27.55|119.80|2555.12|1377.78|1405.33|3932.90|3960.45|-1746.96| +2450815|69862|2450872|62042|643025|4008|23791|62042|643025|4008|23791|2|46|7|1|2374|187|28|33|53.59|106.64|7.46|3272.94|246.18|1768.47|3519.12|14.77|0.00|1231.56|246.18|260.95|1477.74|1492.51|-1522.29| +2450815|69862|2450856|62042|643025|4008|23791|62042|643025|4008|23791|2|91|13|2|10597|92|28|22|2.72|7.15|3.50|80.30|77.00|59.84|157.30|1.54|0.00|29.70|77.00|78.54|106.70|108.24|17.16| +2450815|69862|2450880|62042|643025|4008|23791|62042|643025|4008|23791|2|10|10|5|16274|250|28|81|62.84|71.63|2.86|5570.37|231.66|5090.04|5802.03|13.89|0.00|1218.24|231.66|245.55|1449.90|1463.79|-4858.38| +2450815|69862|2450846|62042|643025|4008|23791|62042|643025|4008|23791|2|52|5|3|15116|47|28|72|11.22|28.83|18.45|747.36|1328.40|807.84|2075.76|106.27|0.00|165.60|1328.40|1434.67|1494.00|1600.27|520.56| +2450815|75740|2450856|50856|693791|7126|46861|50856|693791|7126|46861|2|64|19|2|15650|91|29|87|60.66|99.48|69.63|2596.95|6057.81|5277.42|8654.76|363.46|0.00|951.78|6057.81|6421.27|7009.59|7373.05|780.39| +2450815|75740|2450853|50856|693791|7126|46861|50856|693791|7126|46861|2|2|2|1|4501|16|29|22|97.93|114.57|60.72|1184.70|1335.84|2154.46|2520.54|53.43|0.00|201.52|1335.84|1389.27|1537.36|1590.79|-818.62| +|||50856|693791|||50856|693791|7126||2||11||4327|157|29|45|86.01||112.73||5072.85||||0.00|4973.40|5072.85||10046.25|10046.25|1202.40| +2450815|75740|2450837|50856|693791|7126|46861|50856|693791|7126|46861|2|26|18|5|13868|164|29|95|17.17|17.34|16.99|33.25|1614.05|1631.15|1647.30|71.66|419.65|543.40|1194.40|1266.06|1737.80|1809.46|-436.75| +2450815|75740|2450817|50856|693791|7126|46861|50856|693791|7126|46861|2|67|18|2|17648|25|29|29|22.88|32.26|10.64|626.98|308.56|663.52|935.54|0.00|0.00|280.43|308.56|308.56|588.99|588.99|-354.96| +2450815|75740|2450887|50856|693791|7126|46861|50856|693791|7126|46861|2|67|7|2|7993|214|29|28|52.83|99.32|14.89|2364.04|416.92|1479.24|2780.96|11.04|195.95|55.44|220.97|232.01|276.41|287.45|-1258.27| +2450815|75740|2450884|50856|693791|7126|46861|50856|693791|7126|46861|2|53|5|1|10417|56|29|17|6.63|11.86|7.82|68.68|132.94|112.71|201.62|10.63|0.00|56.44|132.94|143.57|189.38|200.01|20.23| +2450815|75740|2450818|50856|693791|7126|46861|50856|693791|7126|46861|2|80|19|2|4297|102|29|9|15.76|16.07|2.41|122.94|21.69|141.84|144.63|0.42|7.59|46.26|14.10|14.52|60.36|60.78|-127.74| +2450815|75740|2450825|50856|693791|7126|46861|50856|693791|7126|46861|2|77|15|5|16166|175|29|61|81.35|139.92|120.33|1194.99|7340.13|4962.35|8535.12|246.62|4257.27|2901.77|3082.86|3329.48|5984.63|6231.25|-1879.49| +2450815|75740|2450877|50856|693791|7126|46861|50856|693791|7126|46861|2|12|18|3|12763|98|29|39|18.56|19.67|8.65|429.78|337.35|723.84|767.13|10.12|0.00|161.07|337.35|347.47|498.42|508.54|-386.49| +2450815|75740|2450862|50856|693791|7126|46861|50856|693791|7126|46861|2|16|10|2|10948|148|29|68|7.82|20.01|15.80|286.28|1074.40|531.76|1360.68|32.23|0.00|530.40|1074.40|1106.63|1604.80|1637.03|542.64| +2450815|69238|2450878|73036|1717426|1016|40653|73036|1717426|1016|40653|1|11|12|3|1646|259|30|71|98.32|139.61|4.18|9615.53|296.78|6980.72|9912.31|26.71|0.00|4262.13|296.78|323.49|4558.91|4585.62|-6683.94| +2450815|69238|2450820|73036|1717426|1016|40653|73036|1717426|1016|40653|1|75|8|4|4999|61|30|58|74.37|121.96|79.27|2476.02|4597.66|4313.46|7073.68|183.90|0.00|636.26|4597.66|4781.56|5233.92|5417.82|284.20| +2450815|69238|2450851|73036|1717426|1016|40653|73036|1717426|1016|40653|1|7|20|5|2518|75|30|27|81.48|174.36|48.82|3389.58|1318.14|2199.96|4707.72|5.66|751.33|2024.19|566.81|572.47|2591.00|2596.66|-1633.15| +2450815|69238|2450904|73036|1717426|1016|40653|73036|1717426|1016|40653|1|43|7|4|4754|221|30|25|57.40|138.90|65.28|1840.50|1632.00|1435.00|3472.50|48.96|0.00|555.50|1632.00|1680.96|2187.50|2236.46|197.00| +2450815|57336|2450881|24262|511666|7070|29274|24262|511666|7070|29274|2|41|6|5|3596|22|31|57|56.60|95.08|57.99|2114.13|3305.43|3226.20|5419.56|231.38|0.00|1842.24|3305.43|3536.81|5147.67|5379.05|79.23| +2450815|57336|2450872|24262|511666|7070|29274|24262|511666|7070|29274|2|26|7|5|3529|295|31|19|89.23|195.41|117.24|1485.23|2227.56|1695.37|3712.79|111.37|0.00|1522.09|2227.56|2338.93|3749.65|3861.02|532.19| +2450815|57336|2450854|24262|511666|7070|29274|24262|511666|7070|29274|2|38|7|1|7012|192|31|89|4.88|14.39|11.51|256.32|1024.39|434.32|1280.71|71.70|0.00|319.51|1024.39|1096.09|1343.90|1415.60|590.07| +2450815|57336|2450865|24262|511666|7070|29274|24262|511666|7070|29274|2|21|19|1|2620|154|31|48|4.94|8.89|5.60|157.92|268.80|237.12|426.72|5.37|0.00|72.48|268.80|274.17|341.28|346.65|31.68| +2450815|57336|2450874|24262|511666|7070|29274|24262|511666|7070|29274|2|51|8|2|15014|33|31|15|23.44|29.76|8.92|312.60|133.80|351.60|446.40|4.14|50.84|53.55|82.96|87.10|136.51|140.65|-268.64| +2450815|57336|2450889|24262|511666|7070|29274|24262|511666|7070|29274|2|49|5|2|10204|204|31|49|61.32|147.16|138.33|432.67|6778.17|3004.68|7210.84|610.03|0.00|1442.07|6778.17|7388.20|8220.24|8830.27|3773.49| +2450815|57336|2450884|24262|511666|7070|29274|24262|511666|7070|29274|2|74|3|3|17116|261|31|87|64.78|66.72|8.67|5050.35|754.29|5635.86|5804.64|45.25|0.00|1218.87|754.29|799.54|1973.16|2018.41|-4881.57| +2450815|57336|2450872|24262|511666|7070|29274|24262|511666|7070|29274|2|6|9|5|322|185|31|31|69.05|179.53|109.51|2170.62|3394.81|2140.55|5565.43|169.74|0.00|2170.31|3394.81|3564.55|5565.12|5734.86|1254.26| +2450815|57336|2450817|24262|511666|7070|29274|24262|511666|7070|29274|2|65|11|3|11312|250|31|50|54.57|157.16|136.72|1022.00|6836.00|2728.50|7858.00|205.08|0.00|2828.50|6836.00|7041.08|9664.50|9869.58|4107.50| +2450815|57336|2450905|24262|511666|7070|29274|24262|511666|7070|29274|2|86|10|3|13766|246|31|31|29.50|31.86|7.00|770.66|217.00|914.50|987.66|6.51|0.00|157.79|217.00|223.51|374.79|381.30|-697.50| +2450815|57336|2450848|24262|511666|7070|29274|24262|511666|7070|29274|2|19|18|2|2858|298|31|50|64.29|85.50|11.11|3719.50|555.50|3214.50|4275.00|5.55|0.00|171.00|555.50|561.05|726.50|732.05|-2659.00| +2450815|57336|2450891|24262|511666|7070|29274|24262|511666|7070|29274|2|82|3|1|9790|135|31|44|18.96|39.62|39.22|17.60|1725.68|834.24|1743.28|155.31|0.00|261.36|1725.68|1880.99|1987.04|2142.35|891.44| +2450815|57336|2450900|24262|511666|7070|29274|24262|511666|7070|29274|2|14|1|2|5990|274|31|85|14.99|39.87|32.69|610.30|2778.65|1274.15|3388.95|194.50|0.00|1253.75|2778.65|2973.15|4032.40|4226.90|1504.50| +|25034||70388|174162||31663|||||4|||5|12536||32|5|66.34|184.42|121.71||608.55|331.70||||368.80|608.55||977.35|1032.11|276.85| +2450815|25034|2450830|70388|174162|3089|31663|70388|174162|3089|31663|4|50|18|4|17188|138|32|21|60.96|156.66|150.39|131.67|3158.19|1280.16|3289.86|0.00|0.00|1282.89|3158.19|3158.19|4441.08|4441.08|1878.03| +2450815|25034|2450874|70388|174162|3089|31663|70388|174162|3089|31663|4|22|7|1|15460|91|32|78|53.31|64.50|32.25|2515.50|2515.50|4158.18|5031.00|226.39|0.00|1911.78|2515.50|2741.89|4427.28|4653.67|-1642.68| +2450815|25034|2450882|70388|174162|3089|31663|70388|174162|3089|31663|4|66|18|2|3829|285|32|17|71.61|104.55|18.81|1457.58|319.77|1217.37|1777.35|22.38|0.00|533.12|319.77|342.15|852.89|875.27|-897.60| +2450815|25034|2450868|70388|174162|3089|31663|70388|174162|3089|31663|4|82|11|5|17194|90|32|85|46.96|71.37|4.99|5642.30|424.15|3991.60|6066.45|25.44|0.00|2001.75|424.15|449.59|2425.90|2451.34|-3567.45| +2450815|25034|2450823|70388|174162|3089|31663|70388|174162|3089|31663|4|97|9|1|12907|95|32|2|20.89|55.14|9.92|90.44|19.84|41.78|110.28|1.38|0.00|9.92|19.84|21.22|29.76|31.14|-21.94| +2450815|25034|2450856|70388|174162|3089|31663|70388|174162|3089|31663|4|44|20|2|7340|196|32|76|36.17|90.78|34.49|4278.04|2621.24|2748.92|6899.28|183.48|0.00|2138.64|2621.24|2804.72|4759.88|4943.36|-127.68| +2450815|25034|2450883|70388|174162|3089|31663|70388|174162|3089|31663|4|47|6|3|2131|142|32|67|80.10|125.75|123.23|168.84|8256.41|5366.70|8425.25|330.25|0.00|0.00|8256.41|8586.66|8256.41|8586.66|2889.71| +2450815|25034|2450844|70388|174162|3089|31663|70388|174162|3089|31663|4|64|20|5|13663|32|32|71|92.83|123.46|104.94|1314.92|7450.74|6590.93|8765.66|149.01|0.00|3417.94|7450.74|7599.75|10868.68|11017.69|859.81| +2450815|25034|2450865|70388|174162|3089|31663|70388|174162|3089|31663|4|97|15|4|3470|37|32|28|28.42|30.40|15.20|425.60|425.60|795.76|851.20|12.76|0.00|238.28|425.60|438.36|663.88|676.64|-370.16| +2450815|25034|2450852|70388|174162|3089|31663|70388|174162|3089|31663|4|36|2|4|5162|235|32|13|88.71|122.41|3.67|1543.62|47.71|1153.23|1591.33|3.81|0.00|381.81|47.71|51.52|429.52|433.33|-1105.52| +2450815|56643|2450904|87486|1309701|1516|15378|41720|1257055|70|8332|1|102|3|1|10141|266|33|64|25.65|56.94|51.81|328.32|3315.84|1641.60|3644.16|33.15|0.00|1311.36|3315.84|3348.99|4627.20|4660.35|1674.24| +2450815|56643|2450876|87486|1309701|1516|15378|41720|1257055|70|8332|1|4|13|4|1270|108|33|22|74.71|198.72|5.96|4240.72|131.12|1643.62|4371.84|3.93|0.00|655.60|131.12|135.05|786.72|790.65|-1512.50| +2450815|56643|2450863|87486|1309701|1516|15378|41720|1257055|70|8332|1|35|12|5|25|86|33|1|11.25|28.91|7.80|21.11|7.80|11.25|28.91|0.31|0.00|0.00|7.80|8.11|7.80|8.11|-3.45| +2450815|56643|2450883|87486|1309701|1516|15378|41720|1257055|70|8332|1|35|16|3|17428|90|33|77|45.59|61.54|22.76|2986.06|1752.52|3510.43|4738.58|70.10|0.00|1752.52|1752.52|1822.62|3505.04|3575.14|-1757.91| +2450815|56643|2450886|87486|1309701|1516|15378|41720|1257055|70|8332|1|72|1|1|15463|19|33|22|15.03|23.89|19.58|94.82|430.76|330.66|525.58|38.76|0.00|57.64|430.76|469.52|488.40|527.16|100.10| +2450815|56643|2450880|87486|1309701|1516|15378|41720|1257055|70|8332|1|81|8|3|9667|45|33|53|36.21|48.52|1.94|2468.74|102.82|1919.13|2571.56|0.90|91.50|462.69|11.32|12.22|474.01|474.91|-1907.81| +2450815|56643|2450890|87486|1309701|1516|15378|41720|1257055|70|8332|1|26|10|3|17932|244|33|72|9.09|17.90|16.46|103.68|1185.12|654.48|1288.80|106.66|0.00|154.08|1185.12|1291.78|1339.20|1445.86|530.64| +2450815|56643|2450833|87486|1309701|1516|15378|41720|1257055|70|8332|1|23|19|3|1489|158|33|30|84.40|107.18|4.28|3087.00|128.40|2532.00|3215.40|11.55|0.00|321.30|128.40|139.95|449.70|461.25|-2403.60| +2450815|56643|2450835|87486|1309701|1516|15378|41720|1257055|70|8332|1|15|3|4|511|38|33|11|15.71|38.80|13.96|273.24|153.56|172.81|426.80|6.49|81.38|174.90|72.18|78.67|247.08|253.57|-100.63| +2450815|56643|2450819|87486|1309701|1516|15378|41720|1257055|70|8332|1|11|15|1|56|203|33|30|75.75|103.02|62.84|1205.40|1885.20|2272.50|3090.60|140.82|320.48|1174.20|1564.72|1705.54|2738.92|2879.74|-707.78| +2450815|56643|2450841|87486|1309701|1516|15378|41720|1257055|70|8332|1|45|16|5|9934|281|33|67|39.29|112.76|80.05|2191.57|5363.35|2632.43|7554.92|375.43|0.00|2719.53|5363.35|5738.78|8082.88|8458.31|2730.92| +2450815|56643|2450867|87486|1309701|1516|15378|41720|1257055|70|8332|1|16|1|3|1207|125|33|35|55.01|73.71|12.53|2141.30|438.55|1925.35|2579.85|8.77|0.00|438.55|438.55|447.32|877.10|885.87|-1486.80| +2450815|56643|2450890|87486|1309701|1516|15378|41720|1257055|70|8332|1|75|14|5|15490|14|33|95|90.49|257.89|224.36|3185.35|21314.20|8596.55|24499.55|1278.85|0.00|10534.55|21314.20|22593.05|31848.75|33127.60|12717.65| +2450815|56643|2450857|87486|1309701|1516|15378|41720|1257055|70|8332|1|6|15|5|373|147|33|61|80.78|132.47|18.54|6949.73|1130.94|4927.58|8080.67|0.00|0.00|2746.83|1130.94|1130.94|3877.77|3877.77|-3796.64| +2450815|73498|2450854|86755|449756|2357|19358|86755|449756|2357|19358|1|44|13|2|16730|35|34|76|71.21|154.52|89.62|4932.40|6811.12|5411.96|11743.52|476.77|0.00|4931.64|6811.12|7287.89|11742.76|12219.53|1399.16| +2450815|73498|2450881|86755|449756|2357|19358|86755|449756|2357|19358|1|65|4|5|17266|109|34|67|17.57|44.10|43.65|30.15|2924.55|1177.19|2954.70|204.71|0.00|442.87|2924.55|3129.26|3367.42|3572.13|1747.36| +2450815|73498|2450890|86755|449756|2357|19358|86755|449756|2357|19358|1|28|1|5|6974|217|34|96|12.37|13.73|0.54|1266.24|51.84|1187.52|1318.08|2.07|0.00|157.44|51.84|53.91|209.28|211.35|-1135.68| +2450815|73498|2450821|86755|449756|2357|19358|86755|449756|2357|19358|1|46|1|5|9238|268|34|92|3.67|3.85|2.46|127.88|226.32|337.64|354.20|6.78|0.00|52.44|226.32|233.10|278.76|285.54|-111.32| +2450815|73498|2450853|86755|449756|2357|19358|86755|449756|2357|19358|1|42|14|1|10279|220|34|43|68.60|129.65|110.20|836.35|4738.60|2949.80|5574.95|426.47|0.00|2675.89|4738.60|5165.07|7414.49|7840.96|1788.80| +2450815|73498|2450844|86755|449756|2357|19358|86755|449756|2357|19358|1|49|6|3|12056|217|34|13|86.69|245.33|225.70|255.19|2934.10|1126.97|3189.29|264.06|0.00|605.93|2934.10|3198.16|3540.03|3804.09|1807.13| +2450815|73498|2450881|86755|449756|2357|19358|86755|449756|2357|19358|1|50|16|4|7297|241|34|37|42.51|73.11|67.99|189.44|2515.63|1572.87|2705.07|0.00|0.00|459.54|2515.63|2515.63|2975.17|2975.17|942.76| +2450815|73498|2450896|86755|449756|2357|19358|86755|449756|2357|19358|1|91|13|5|2932|26|34|30|9.58|17.05|3.23|414.60|96.90|287.40|511.50|7.75|0.00|143.10|96.90|104.65|240.00|247.75|-190.50| +2450815|73498|2450831|86755|449756|2357|19358|86755|449756|2357|19358|1|7|13|1|6289|274|34|62|57.21|75.51|24.91|3137.20|1544.42|3547.02|4681.62|15.44|0.00|468.10|1544.42|1559.86|2012.52|2027.96|-2002.60| +2450815|73498|2450873|86755|449756|2357|19358|86755|449756|2357|19358|1|10|15|4|14294|237|34|15|61.72|85.79|10.29|1132.50|154.35|925.80|1286.85|9.26|0.00|141.45|154.35|163.61|295.80|305.06|-771.45| +2450815|73498|2450821|86755|449756|2357|19358|86755|449756|2357|19358|1|107|12|1|16756|22|34|34|61.66|85.70|69.41|553.86|2359.94|2096.44|2913.80|188.79|0.00|1019.66|2359.94|2548.73|3379.60|3568.39|263.50| +2450815|73498|2450882|86755|449756|2357|19358|86755|449756|2357|19358|1|75|12|4|7564|181|34|60|79.51|226.60|47.58|10741.20|2854.80|4770.60|13596.00|114.19|0.00|5166.00|2854.80|2968.99|8020.80|8134.99|-1915.80| +2450815|69327|2450825|12919|717893|914|3888|12919|717893|914|3888|1|20|2|4|11071|38|35|71|72.13|80.06|9.60|5002.66|681.60|5121.23|5684.26|61.34|0.00|2159.82|681.60|742.94|2841.42|2902.76|-4439.63| +2450815|69327|2450897|12919|717893|914|3888|12919|717893|914|3888|1|82|12|1|17056|291|35|82|96.96|139.62|71.20|5610.44|5838.40|7950.72|11448.84|233.53|0.00|4922.46|5838.40|6071.93|10760.86|10994.39|-2112.32| +2450815|69327|2450819|12919|717893|914|3888|12919|717893|914|3888|1|26|11|4|9586|125|35|93|12.45|28.01|10.64|1615.41|989.52|1157.85|2604.93|59.37|0.00|78.12|989.52|1048.89|1067.64|1127.01|-168.33| +2450815|69327|2450842|12919|717893|914|3888|12919|717893|914|3888|1|4|14|5|14528|132|35|32|87.76|139.53|103.25|1160.96|3304.00|2808.32|4464.96|165.20|0.00|1473.28|3304.00|3469.20|4777.28|4942.48|495.68| +2450815|69327|2450847|12919|717893|914|3888|12919|717893|914|3888|1|9|4|1|6988|120|35|75|47.28|84.15|48.80|2651.25|3660.00|3546.00|6311.25|36.60|0.00|2902.50|3660.00|3696.60|6562.50|6599.10|114.00| +2450815|69327|2450869|12919|717893|914|3888|12919|717893|914|3888|1|84|15|4|6010|280|35|99|89.30|166.09|16.60|14799.51|1643.40|8840.70|16442.91|98.60|0.00|4274.82|1643.40|1742.00|5918.22|6016.82|-7197.30| +2450815|69327|2450826|12919|717893|914|3888|12919|717893|914|3888|1|56|1|4|2587|80|35|30|29.35|51.94|23.37|857.10|701.10|880.50|1558.20|63.09|0.00|779.10|701.10|764.19|1480.20|1543.29|-179.40| +2450815|69327|2450904|12919|717893|914|3888|12919|717893|914|3888|1|8|17|1|4958|169|35|8|36.75|45.93|5.51|323.36|44.08|294.00|367.44|0.88|14.54|10.96|29.54|30.42|40.50|41.38|-264.46| +2450815|69327|2450869|12919|717893|914|3888|12919|717893|914|3888|1|27|19|5|2326|157|35|54|79.25|221.10|88.44|7163.64|4775.76|4279.50|11939.40|191.03|0.00|4775.76|4775.76|4966.79|9551.52|9742.55|496.26| +2450815|69327|2450830|12919|717893|914|3888|12919|717893|914|3888|1|56|3|5|6955|267|35|8|30.04|34.84|2.43|259.28|19.44|240.32|278.72|1.16|0.00|22.24|19.44|20.60|41.68|42.84|-220.88| +2450815|69327|2450839|12919|717893|914|3888|12919|717893|914|3888|1|33|13|3|9127|18|35|93|75.21|197.05|23.64|16127.13|2198.52|6994.53|18325.65|71.23|417.71|4397.97|1780.81|1852.04|6178.78|6250.01|-5213.72| +2450815|69327|2450843|12919|717893|914|3888|12919|717893|914|3888|1|81|15|2|17828|141|35|94|26.84|77.29|7.72|6539.58|725.68|2522.96|7265.26|0.00|0.00|72.38|725.68|725.68|798.06|798.06|-1797.28| +2450815|69327|2450880|12919|717893|914|3888|12919|717893|914|3888|1|54|3|5|6829|167|35|61|97.42|196.78|159.39|2280.79|9722.79|5942.62|12003.58|58.33|8750.51|5401.55|972.28|1030.61|6373.83|6432.16|-4970.34| +2450815|69327|2450863|12919|717893|914|3888|12919|717893|914|3888|1|101|5|2|8504|33|35|26|70.52|131.87|9.23|3188.64|239.98|1833.52|3428.62|2.15|23.99|1199.90|215.99|218.14|1415.89|1418.04|-1617.53| +2450815|42120|2450873|53988|467550|5112|40321|53988|467550|5112|40321|2|46|10|3|8630|49|36|49|67.35|192.62|109.79|4058.67|5379.71|3300.15|9438.38|0.00|0.00|3019.87|5379.71|5379.71|8399.58|8399.58|2079.56| +2450815|42120|2450904|53988|467550|5112|40321|53988|467550|5112|40321|2|86|4|4|9283|294|36|33|26.17|57.31|44.12|435.27|1455.96|863.61|1891.23|14.55|0.00|529.32|1455.96|1470.51|1985.28|1999.83|592.35| +2450815|42120|2450863|53988|467550|5112|40321|53988|467550|5112|40321|2|30|10|2|15034|259|36|98|63.27|104.39|46.97|5627.16|4603.06|6200.46|10230.22|250.40|1472.97|2659.72|3130.09|3380.49|5789.81|6040.21|-3070.37| +2450815|42120|2450897|53988|467550|5112|40321|53988|467550|5112|40321|2|52|19|4|12661|275|36|98|46.67|83.53|20.88|6139.70|2046.24|4573.66|8185.94|143.23|0.00|1391.60|2046.24|2189.47|3437.84|3581.07|-2527.42| +2450815|42120|2450866|53988|467550|5112|40321|53988|467550|5112|40321|2|92|17|3|17356|115|36|45|28.00|63.28|63.28|0.00|2847.60|1260.00|2847.60|56.95|0.00|1395.00|2847.60|2904.55|4242.60|4299.55|1587.60| +2450815|78803|2450892|76817|785079|1788|5250|76817|785079|1788|5250|2|7|9|3|5635|143|37|14|94.18|168.58|26.97|1982.54|377.58|1318.52|2360.12|30.20|0.00|566.30|377.58|407.78|943.88|974.08|-940.94| +|78803||76817||1788|||785079|1788|5250||57||5|15110|86|37|5|30.47|62.46||262.35||||2.49|0.00|74.95|||124.90|127.39|-102.40| +2450815|78803|2450873|76817|785079|1788|5250|76817|785079|1788|5250|2|13|17|3|5170|233|37|37|6.70|11.18|4.47|248.27|165.39|247.90|413.66|9.92|0.00|20.35|165.39|175.31|185.74|195.66|-82.51| +2450815|78803|2450853|76817|785079|1788|5250|76817|785079|1788|5250|2|61|16|5|4508|183|37|5|15.92|28.33|26.34|9.95|131.70|79.60|141.65|7.90|0.00|0.00|131.70|139.60|131.70|139.60|52.10| +2450815|78803|2450882|76817|785079|1788|5250|76817|785079|1788|5250|2|39|11|3|17797|32|37|81|49.46|116.72|0.00|9454.32|0.00|4006.26|9454.32|0.00|0.00|3025.35|0.00|0.00|3025.35|3025.35|-4006.26| +2450815|21970|2450833|50061|1286495|4976|25457|50061|1286495|4976|25457|2|103|1|4|262|238|38|88|98.38|146.58|140.71|516.56|12382.48|8657.44|12899.04|866.77|0.00|6062.32|12382.48|13249.25|18444.80|19311.57|3725.04| +2450815|21970|2450840|50061|1286495|4976|25457|50061|1286495|4976|25457|2|48|12|5|8026|91|38|76|86.00|138.46|31.84|8103.12|2419.84|6536.00|10522.96|96.79|0.00|630.80|2419.84|2516.63|3050.64|3147.43|-4116.16| +2450815|21970|2450882|50061|1286495|4976|25457|50061|1286495|4976|25457|2|68|19|5|10802|207|38|12|15.77|41.94|17.19|297.00|206.28|189.24|503.28|5.69|16.50|110.64|189.78|195.47|300.42|306.11|0.54| +2450815|21970|2450850|50061|1286495|4976|25457|50061|1286495|4976|25457|2|86|19|2|1406|141|38|5|2.95|3.89|3.30|2.95|16.50|14.75|19.45|0.33|0.00|1.35|16.50|16.83|17.85|18.18|1.75| +2450815|21970|2450823|50061|1286495|4976|25457|50061|1286495|4976|25457|2|15|12|5|10046|131|38|54|25.77|27.57|19.57|432.00|1056.78|1391.58|1488.78|95.11|0.00|178.20|1056.78|1151.89|1234.98|1330.09|-334.80| +2450815|21970|2450825|50061|1286495|4976|25457|50061|1286495|4976|25457|2|53|13|4|14509|226|38|21|15.57|30.82|16.95|291.27|355.95|326.97|647.22|0.00|0.00|278.25|355.95|355.95|634.20|634.20|28.98| +2450815|21970|2450824|50061|1286495|4976|25457|50061|1286495|4976|25457|2|18|2|3|1669|247|38|100|8.47|21.17|19.89|128.00|1989.00|847.00|2117.00|119.34|0.00|889.00|1989.00|2108.34|2878.00|2997.34|1142.00| +2450815|83055|2450883|||2560||||2560|38991|1|105|||1102|162|39|80||||881.60||785.60|1602.40|||160.00||771.25|880.80|931.25|-64.80| +2450815|83055|2450893|88898|1785329|2560|38991|88898|1785329|2560|38991|1|99|14|5|2170|106|39|12|92.40|195.88|144.95|611.16|1739.40|1108.80|2350.56|52.18|0.00|1081.20|1739.40|1791.58|2820.60|2872.78|630.60| +2450815|83055|2450866|88898|1785329|2560|38991|88898|1785329|2560|38991|1|2|3|2|112|94|39|13|27.62|59.38|31.47|362.83|409.11|359.06|771.94|20.45|0.00|123.50|409.11|429.56|532.61|553.06|50.05| +2450815|83055|2450838|88898|1785329|2560|38991|88898|1785329|2560|38991|1|61|20|4|4220|37|39|34|80.08|150.55|117.42|1126.42|3992.28|2722.72|5118.70|0.00|0.00|153.34|3992.28|3992.28|4145.62|4145.62|1269.56| +2450815|83055|2450832|88898|1785329|2560|38991|88898|1785329|2560|38991|1|90|5|3|11036|24|39|97|67.74|183.57|95.45|8547.64|9258.65|6570.78|17806.29|92.58|0.00|3204.88|9258.65|9351.23|12463.53|12556.11|2687.87| +2450815|83055|2450840|88898|1785329|2560|38991|88898|1785329|2560|38991|1|54|18|2|1429|260|39|51|46.12|113.45|103.23|521.22|5264.73|2352.12|5785.95|473.82|0.00|809.88|5264.73|5738.55|6074.61|6548.43|2912.61| +2450815|83055|2450828|88898|1785329|2560|38991|88898|1785329|2560|38991|1|36|12|5|8890|183|39|40|71.92|181.23|106.92|2972.40|4276.80|2876.80|7249.20|299.37|0.00|2464.40|4276.80|4576.17|6741.20|7040.57|1400.00| +2450815|83055|2450856|88898|1785329|2560|38991|88898|1785329|2560|38991|1|6|8|1|5689|223|39|8|95.05|152.08|118.62|267.68|948.96|760.40|1216.64|41.84|351.11|133.76|597.85|639.69|731.61|773.45|-162.55| +2450815|83055|2450892|88898|1785329|2560|38991|88898|1785329|2560|38991|1|50|20|1|6896|273|39|55|97.79|196.55|82.55|6270.00|4540.25|5378.45|10810.25|227.01|0.00|1729.20|4540.25|4767.26|6269.45|6496.46|-838.20| +2450815|83055|2450874|88898|1785329|2560|38991|88898|1785329|2560|38991|1|107|2|5|13441|42|39|50|11.78|24.73|23.24|74.50|1162.00|589.00|1236.50|29.28|185.92|445.00|976.08|1005.36|1421.08|1450.36|387.08| +2450815|83055|2450905|88898|1785329|2560|38991|88898|1785329|2560|38991|1|39|8|2|3235|241|39|27|56.60|92.25|54.42|1021.41|1469.34|1528.20|2490.75|88.16|0.00|797.04|1469.34|1557.50|2266.38|2354.54|-58.86| +2450815|83055|2450864|88898|1785329|2560|38991|88898|1785329|2560|38991|1|43|11|2|110|26|39|72|67.09|162.35|63.31|7130.88|4558.32|4830.48|11689.20|109.39|911.66|3039.12|3646.66|3756.05|6685.78|6795.17|-1183.82| +2450815|83055|2450883|88898|1785329|2560|38991|88898|1785329|2560|38991|1|5|6|2|16400|93|39|18|69.64|169.22|157.37|213.30|2832.66|1253.52|3045.96|113.30|0.00|548.10|2832.66|2945.96|3380.76|3494.06|1579.14| +2450815|83055|2450891|88898|1785329|2560|38991|88898|1785329|2560|38991|1|26|18|3|11186|38|39|49|98.67|134.19|120.77|657.58|5917.73|4834.83|6575.31|0.00|0.00|131.32|5917.73|5917.73|6049.05|6049.05|1082.90| +2450815|75521|2450904|26283|157402|4874|43146|26283|157402|4874|43146|4|13|11|2|5719|14|40|67|27.15|59.18|14.79|2974.13|990.93|1819.05|3965.06|22.29|545.01|1189.25|445.92|468.21|1635.17|1657.46|-1373.13| +2450815|75521|2450854|26283|157402|4874|43146|26283|157402|4874|43146|4|92|18|4|16831|179|40|37|8.58|22.47|6.51|590.52|240.87|317.46|831.39|9.63|0.00|91.39|240.87|250.50|332.26|341.89|-76.59| +2450815|75521|2450875|26283|157402|4874|43146|26283|157402|4874|43146|4|52|13|3|14632|181|40|34|38.71|77.42|65.03|421.26|2211.02|1316.14|2632.28|0.00|0.00|947.58|2211.02|2211.02|3158.60|3158.60|894.88| +2450815|75521|2450889|26283|157402|4874|43146|26283|157402|4874|43146|4|107|18|3|6686|45|40|1|40.85|103.75|75.73|28.02|75.73|40.85|103.75|1.51|0.00|8.30|75.73|77.24|84.03|85.54|34.88| +2450815|75521|2450845|26283|157402|4874|43146|26283|157402|4874|43146|4|2|7|3|12202|112|40|76|74.02|133.97|103.15|2342.32|7839.40|5625.52|10181.72|504.85|627.15|3054.44|7212.25|7717.10|10266.69|10771.54|1586.73| +2450815|75521|2450835|26283|157402|4874|43146|26283|157402|4874|43146|4|74|5|5|4394|38|40|63|14.16|32.99|27.71|332.64|1745.73|892.08|2078.37|6.11|1623.52|61.74|122.21|128.32|183.95|190.06|-769.87| +2450815|75521|2450896|26283|157402|4874|43146|26283|157402|4874|43146|4|67|4|2|15496|120|40|58|50.32|95.10|56.10|2262.00|3253.80|2918.56|5515.80|162.69|0.00|110.20|3253.80|3416.49|3364.00|3526.69|335.24| +2450815||2450887||157402|4874||26283|||||61||2|17686||40|86|91.64||238.70||20528.20|7881.04|23327.50|480.36|15190.86||5337.34|5817.70|14901.40|15381.76|-2543.70| +2450815|38383|2450854|6161|549009|7057|14352|6161|549009|7057|14352|2|30|2|2|15589|78|41|65|50.23|58.76|9.40|3208.40|611.00|3264.95|3819.40|24.44|0.00|1641.90|611.00|635.44|2252.90|2277.34|-2653.95| +2450815|38383|2450864|6161|549009|7057|14352|6161|549009|7057|14352|2|34|6|2|10879|179|41|14|84.31|181.26|117.81|888.30|1649.34|1180.34|2537.64|16.49|0.00|0.00|1649.34|1665.83|1649.34|1665.83|469.00| +2450815|38383|2450838|6161|549009|7057|14352|6161|549009|7057|14352|2|3|2|2|14810|22|41|88|88.24|120.00|86.40|2956.80|7603.20|7765.12|10560.00|472.15|2356.99|633.60|5246.21|5718.36|5879.81|6351.96|-2518.91| +2450815|38383|2450878|6161|549009|7057|14352|6161|549009|7057|14352|2|76|4|5|11209|36|41|42|65.22|138.91|16.66|5134.50|699.72|2739.24|5834.22|18.89|69.97|116.34|629.75|648.64|746.09|764.98|-2109.49| +2450815|38383|2450853|6161|549009|7057|14352|6161|549009|7057|14352|2|45|18|1|6998|47|41|89|15.91|36.75|30.13|589.18|2681.57|1415.99|3270.75|241.34|0.00|654.15|2681.57|2922.91|3335.72|3577.06|1265.58| +2450815|45569|2450827|7159|1915909|7059|18962|7159|1915909|7059|18962|2|59|2|1|7066|5|42|10|5.00|13.45|2.15|113.00|21.50|50.00|134.50|0.43|0.00|63.20|21.50|21.93|84.70|85.13|-28.50| +2450815|45569|2450845|7159|1915909|7059|18962|7159|1915909|7059|18962|2|18|15|4|7693|202|42|32|14.61|15.48|1.54|446.08|49.28|467.52|495.36|2.46|0.00|108.80|49.28|51.74|158.08|160.54|-418.24| +2450815|45569|2450879|7159|1915909|7059|18962|7159|1915909|7059|18962|2|72|20|1|6902|128|42|69|74.00|156.88|81.57|5196.39|5628.33|5106.00|10824.72|450.26|0.00|4654.05|5628.33|6078.59|10282.38|10732.64|522.33| +2450815|45569|2450889|7159|1915909|7059|18962|7159|1915909|7059|18962|2|32|11|4|9496|238|42|2|24.13|44.15|38.85|10.60|77.70|48.26|88.30|0.49|27.97|16.76|49.73|50.22|66.49|66.98|1.47| +2450815|45569|2450851|7159|1915909|7059|18962|7159|1915909|7059|18962|2|107|9|4|6100|126|42|41|19.58|40.72|21.98|768.34|901.18|802.78|1669.52|63.08|0.00|483.80|901.18|964.26|1384.98|1448.06|98.40| +2450815|45569|2450903|7159|1915909|7059|18962|7159|1915909|7059|18962|2|57|15|5|10687|125|42|88|29.29|42.47|14.01|2504.48|1232.88|2577.52|3737.36|12.32|0.00|1008.48|1232.88|1245.20|2241.36|2253.68|-1344.64| +2450815|36865|2450856|16476|49790|4257|31571|16476|49790|4257|31571|1|32|13|2|5458|258|43|68|13.45|36.71|2.20|2346.68|149.60|914.60|2496.28|10.47|0.00|199.24|149.60|160.07|348.84|359.31|-765.00| +2450815|36865|2450868|16476|49790|4257|31571|16476|49790|4257|31571|1|78|8|1|10648|115|43|69|97.14|108.79|44.60|4429.11|3077.40|6702.66|7506.51|123.09|0.00|750.03|3077.40|3200.49|3827.43|3950.52|-3625.26| +2450815|36865|2450887|16476|49790|4257|31571|16476|49790|4257|31571|1|79|14|5|938|69|43|55|88.73|131.32|52.52|4334.00|2888.60|4880.15|7222.60|259.97|0.00|1733.05|2888.60|3148.57|4621.65|4881.62|-1991.55| +2450815|36865|2450883|16476|49790|4257|31571|16476|49790|4257|31571|1|103|18|2|4850|203|43|84|16.99|23.44|23.44|0.00|1968.96|1427.16|1968.96|0.00|1929.58|255.36|39.38|39.38|294.74|294.74|-1387.78| +2450815|36865|2450836|16476|49790|4257|31571|16476|49790|4257|31571|1|57|4|4|7612|37|43|48|74.98|209.19|79.49|6225.60|3815.52|3599.04|10041.12|267.08|0.00|100.32|3815.52|4082.60|3915.84|4182.92|216.48| +2450815|36865|2450891|16476|49790|4257|31571|16476|49790|4257|31571|1|25|18|3|3889|96|43|83|39.11|95.81|13.41|6839.20|1113.03|3246.13|7952.23|55.65|0.00|3975.70|1113.03|1168.68|5088.73|5144.38|-2133.10| +2450815|36865|2450872|16476|49790|4257|31571|16476|49790|4257|31571|1|61|20|4|5101|271|43|52|21.35|43.34|10.40|1712.88|540.80|1110.20|2253.68|11.19|416.41|135.20|124.39|135.58|259.59|270.78|-985.81| +2450815|36865|2450817|16476|49790|4257|31571|16476|49790|4257|31571|1|25|9|4|15067|63|43|59|79.77|112.47|107.97|265.50|6370.23|4706.43|6635.73|191.10|0.00|2255.57|6370.23|6561.33|8625.80|8816.90|1663.80| +2450815|36865|2450857|16476|49790|4257|31571|16476|49790|4257|31571|1|76|17|5|6094|99|43|14|28.09|73.87|65.74|113.82|920.36|393.26|1034.18|27.61|0.00|392.98|920.36|947.97|1313.34|1340.95|527.10| +2450815|36865|2450826|16476|49790|4257|31571|16476|49790|4257|31571|1|83|1|4|7225|250|43|79|18.16|54.48|41.94|990.66|3313.26|1434.64|4303.92|113.31|1424.70|257.54|1888.56|2001.87|2146.10|2259.41|453.92| +2450815||2450886||49790|4257||16476||4257||||16|1|1054|190|43|||119.26|71.55|1526.72|2289.60||3816.32|0.00|0.00|||2289.60|3014.40|3014.40|514.56| +2450815|36865|2450846|16476|49790|4257|31571|16476|49790|4257|31571|1|69|11|4|4711|30|43|51|2.21|2.96|2.07|45.39|105.57|112.71|150.96|2.11|0.00|60.18|105.57|107.68|165.75|167.86|-7.14| +2450815|36865|2450899|16476|49790|4257|31571|16476|49790|4257|31571|1|61|1|3|6292|192|43|43|82.42|159.89|99.13|2612.68|4262.59|3544.06|6875.27|298.38|0.00|2131.08|4262.59|4560.97|6393.67|6692.05|718.53| +2450815|59287|2450821|53183|1623252|3721|25390|53183|1623252|3721|25390|1|28|10|3|7435|49|44|2|89.84|244.36|102.63|283.46|205.26|179.68|488.72|6.15|0.00|244.36|205.26|211.41|449.62|455.77|25.58| +2450815|59287|2450881|53183|1623252|3721|25390|53183|1623252|3721|25390|1|58|15|4|11450|195|44|52|34.52|37.28|25.35|620.36|1318.20|1795.04|1938.56|52.72|0.00|678.08|1318.20|1370.92|1996.28|2049.00|-476.84| +2450815|59287|2450884|53183|1623252|3721|25390|53183|1623252|3721|25390|1|108|2|4|7153|157|44|33|67.01|115.92|92.73|765.27|3060.09|2211.33|3825.36|61.20|0.00|1147.41|3060.09|3121.29|4207.50|4268.70|848.76| +2450815|59287|2450878|53183|1623252|3721|25390|53183|1623252|3721|25390|1|42|17|2|7084|160|44|70|73.07|124.21|105.57|1304.80|7389.90|5114.90|8694.70|295.59|0.00|2086.70|7389.90|7685.49|9476.60|9772.19|2275.00| +2450815|59287|2450845|53183|1623252|3721|25390|53183|1623252|3721|25390|1|45|8|3|7496|144|44|77|98.65|199.27|0.00|15343.79|0.00|7596.05|15343.79|0.00|0.00|7518.28|0.00|0.00|7518.28|7518.28|-7596.05| +2450815|59287|2450903|53183|1623252|3721|25390|53183|1623252|3721|25390|1|32|8|1|11635|134|44|30|9.78|13.10|2.22|326.40|66.60|293.40|393.00|2.66|0.00|184.50|66.60|69.26|251.10|253.76|-226.80| +2450815|59287|2450892|53183|1623252|3721|25390|53183|1623252|3721|25390|1|95|13|1|11570|223|44|55|46.80|65.98|39.58|1452.00|2176.90|2574.00|3628.90|195.92|0.00|1306.25|2176.90|2372.82|3483.15|3679.07|-397.10| +2450815|14339|2450881|64305|527683|6708|40486|64305|527683|6708|40486|2|89|8|2|5569|5|45|31|95.47|286.41|171.84|3551.67|5327.04|2959.57|8878.71|245.04|426.16|1953.31|4900.88|5145.92|6854.19|7099.23|1941.31| +2450815|14339|2450842|64305|527683|6708|40486|64305|527683|6708|40486|2|8|5|1|9086|165|45|52|85.75|150.06|33.01|6086.60|1716.52|4459.00|7803.12|154.48|0.00|468.00|1716.52|1871.00|2184.52|2339.00|-2742.48| +2450815|14339|2450820|64305|527683|6708|40486|64305|527683|6708|40486|2|37|13|1|1634|57|45|45|60.16|73.39|62.38|495.45|2807.10|2707.20|3302.55|168.42|0.00|264.15|2807.10|2975.52|3071.25|3239.67|99.90| +2450815|14339|2450871|64305|527683|6708|40486|64305|527683|6708|40486|2|98|17|3|7582|42|45|1|24.54|35.33|9.89|25.44|9.89|24.54|35.33|0.05|4.25|12.71|5.64|5.69|18.35|18.40|-18.90| +2450815|14339|2450896|64305|527683|6708|40486|64305|527683|6708|40486|2|18|7|1|6754|27|45|69|69.08|92.56|41.65|3512.79|2873.85|4766.52|6386.64|16.09|2672.68|2107.26|201.17|217.26|2308.43|2324.52|-4565.35| +2450815|14339|2450903|64305|527683|6708|40486|64305|527683|6708|40486|2|9|3|3|11828|221|45|89|28.49|78.91|70.22|773.41|6249.58|2535.61|7022.99|437.47|0.00|2457.29|6249.58|6687.05|8706.87|9144.34|3713.97| +2450815|14339|2450858|64305|527683|6708|40486|64305|527683|6708|40486|2|43|15|4|11614|159|45|69|27.70|30.47|0.00|2102.43|0.00|1911.30|2102.43|0.00|0.00|251.85|0.00|0.00|251.85|251.85|-1911.30| +2450815|14339|2450839|64305|527683|6708|40486|64305|527683|6708|40486|2|27|17|1|625|240|45|96|17.74|45.59|38.75|656.64|3720.00|1703.04|4376.64|62.49|2827.20|1794.24|892.80|955.29|2687.04|2749.53|-810.24| +2450815|56496|2450838|86871|830328|6585|16842|85678|1179356|60|7117|1|78|15|3|2722|126|46|83|20.86|52.15|26.59|2121.48|2206.97|1731.38|4328.45|132.41|0.00|1860.86|2206.97|2339.38|4067.83|4200.24|475.59| +2450815|56496|2450884|86871|830328|6585|16842|85678|1179356|60|7117|1|78|18|1|16660|218|46|57|25.00|50.50|45.45|287.85|2590.65|1425.00|2878.50|0.00|0.00|1439.25|2590.65|2590.65|4029.90|4029.90|1165.65| +2450815|56496|2450873|86871|830328|6585|16842|85678|1179356|60|7117|1|24|16|4|3218|115|46|83|55.95|133.16|51.93|6742.09|4310.19|4643.85|11052.28|52.58|1680.97|2210.29|2629.22|2681.80|4839.51|4892.09|-2014.63| +2450815|56496|2450847|86871|830328|6585|16842|85678|1179356|60|7117|1|57|2|1|14776|106|46|57|23.47|23.70|16.59|405.27|945.63|1337.79|1350.90|28.36|0.00|513.00|945.63|973.99|1458.63|1486.99|-392.16| +2450815|56496|2450877|86871|830328|6585|16842|85678|1179356|60|7117|1|15|3|5|5986|191|46|65|53.08|106.16|49.89|3657.55|3242.85|3450.20|6900.40|0.00|0.00|68.90|3242.85|3242.85|3311.75|3311.75|-207.35| +2450815|56496|2450867|86871|830328|6585|16842|85678|1179356|60|7117|1|79|10|1|10933|14|46|63|8.80|9.24|8.13|69.93|512.19|554.40|582.12|35.85|0.00|122.22|512.19|548.04|634.41|670.26|-42.21| +2450815|56496|2450858|86871|830328|6585|16842|85678|1179356|60|7117|1|85|11|1|13136|6|46|7|42.99|80.39|33.76|326.41|236.32|300.93|562.73|16.54|0.00|129.36|236.32|252.86|365.68|382.22|-64.61| +2450815|56496|2450857|86871|830328|6585|16842|85678|1179356|60|7117|1|51|10|5|3314|210|46|78|10.68|13.35|5.07|645.84|395.46|833.04|1041.30|8.06|126.54|468.00|268.92|276.98|736.92|744.98|-564.12| +2450815|56496|2450861|86871|830328|6585|16842|85678|1179356|60|7117|1|71|7|5|12172|278|46|26|67.43|140.25|95.37|1166.88|2479.62|1753.18|3646.50|74.38|0.00|1823.12|2479.62|2554.00|4302.74|4377.12|726.44| +2450815|56496|2450848|86871|830328|6585|16842|85678|1179356|60|7117|1|15|17|4|5506|278|46|33|28.17|63.66|24.82|1281.72|819.06|929.61|2100.78|24.57|0.00|1008.15|819.06|843.63|1827.21|1851.78|-110.55| +2450815|56496|2450833|86871|830328|6585|16842|85678|1179356|60|7117|1|26|13|4|8986|62|46|10|15.57|39.23|8.23|310.00|82.30|155.70|392.30|3.29|0.00|129.40|82.30|85.59|211.70|214.99|-73.40| +2450815|56496|2450894|86871|830328|6585|16842|85678|1179356|60|7117|1|91|8|1|11924|82|46|17|81.55|215.29|159.31|951.66|2708.27|1386.35|3659.93|113.74|433.32|1134.41|2274.95|2388.69|3409.36|3523.10|888.60| +2450815|56496|2450847|86871|830328|6585|16842|85678|1179356|60|7117|1|36|8|2|3464|155|46|78|78.61|154.07|98.60|4326.66|7690.80|6131.58|12017.46|230.72|0.00|0.00|7690.80|7921.52|7690.80|7921.52|1559.22| +2450815|56496|2450837|86871|830328|6585|16842|85678|1179356|60|7117|1|19|5|4|8204|87|46|99|8.53|22.26|16.91|529.65|1674.09|844.47|2203.74|133.92|0.00|1079.10|1674.09|1808.01|2753.19|2887.11|829.62| +2450815|45024|2450866|95722|553976|3877|33331|95722|553976|3877|33331|1|76|2|4|10837|182|47|14|30.67|75.75|49.99|360.64|699.86|429.38|1060.50|0.00|0.00|413.56|699.86|699.86|1113.42|1113.42|270.48| +2450815|45024|2450887|95722|553976|3877|33331|95722|553976|3877|33331|1|47|7|4|15676|3|47|17|94.84|160.27|46.47|1934.60|789.99|1612.28|2724.59|71.09|0.00|817.36|789.99|861.08|1607.35|1678.44|-822.29| +2450815|45024|2450854|95722|553976|3877|33331|95722|553976|3877|33331|1|101|1|3|9601|100|47|11|88.66|232.28|104.52|1405.36|1149.72|975.26|2555.08|57.48|0.00|536.47|1149.72|1207.20|1686.19|1743.67|174.46| +2450815|45024|2450824|95722|553976|3877|33331|95722|553976|3877|33331|1|74|3|4|11164|214|47|52|88.50|134.52|79.36|2868.32|4126.72|4602.00|6995.04|0.00|0.00|3147.56|4126.72|4126.72|7274.28|7274.28|-475.28| +2450815|45024|2450835|95722|553976|3877|33331|95722|553976|3877|33331|1|106|13|5|13933|147|47|86|30.41|62.94|1.88|5251.16|161.68|2615.26|5412.84|3.23|0.00|2110.44|161.68|164.91|2272.12|2275.35|-2453.58| +2450815|45024|2450850|95722|553976|3877|33331|95722|553976|3877|33331|1|90|18|1|8863|275|47|21|38.17|62.98|42.82|423.36|899.22|801.57|1322.58|0.00|0.00|370.23|899.22|899.22|1269.45|1269.45|97.65| +2450815|45024|2450861|95722|553976|3877|33331|95722|553976|3877|33331|1|3|10|1|14270|38|47|48|29.10|71.87|15.81|2690.88|758.88|1396.80|3449.76|68.29|0.00|34.08|758.88|827.17|792.96|861.25|-637.92| +2450815|45024|2450873|95722|553976|3877|33331|95722|553976|3877|33331|1|22|4|3|14779|26|47|6|9.33|25.84|8.52|103.92|51.12|55.98|155.04|4.08|0.00|49.56|51.12|55.20|100.68|104.76|-4.86| +2450815|45024|2450871|95722|553976|3877|33331|95722|553976|3877|33331|1|31|7|2|5869|65|47|66|56.85|102.33|20.46|5403.42|1350.36|3752.10|6753.78|108.02|0.00|2228.16|1350.36|1458.38|3578.52|3686.54|-2401.74| +2450815|45024|2450842|95722|553976|3877|33331|95722|553976|3877|33331|1|105|6|4|12188|232|47|3|63.82|132.74|54.42|234.96|163.26|191.46|398.22|8.16|0.00|191.13|163.26|171.42|354.39|362.55|-28.20| +2450815|68086|2450898|48375|1166269|3010|28504|48375|1166269|3010|28504|2|52|12|2|12656|183|48|58|16.99|44.34|23.50|1208.72|1363.00|985.42|2571.72|15.12|858.69|51.04|504.31|519.43|555.35|570.47|-481.11| +2450815|68086|2450882|48375|1166269|3010|28504|48375|1166269|3010|28504|2|42|20|4|16600|229|48|72|29.87|68.10|3.40|4658.40|244.80|2150.64|4903.20|0.19|239.90|1568.88|4.90|5.09|1573.78|1573.97|-2145.74| +2450815|68086|2450847|48375|1166269|3010|28504|48375|1166269|3010|28504|2|77|8|3|17035|213|48|43|58.50|60.25|22.29|1632.28|958.47|2515.50|2590.75|0.00|0.00|699.18|958.47|958.47|1657.65|1657.65|-1557.03| +2450815|68086|2450833|48375|1166269|3010|28504|48375|1166269|3010|28504|2|80|14|4|2809|295|48|63|31.78|34.64|4.50|1898.82|283.50|2002.14|2182.32|17.01|0.00|589.05|283.50|300.51|872.55|889.56|-1718.64| +2450815|68086|2450860|48375|1166269|3010|28504|48375|1166269|3010|28504|2|82|6|2|16306|268|48|5|8.53|10.49|3.04|37.25|15.20|42.65|52.45|0.30|0.00|9.95|15.20|15.50|25.15|25.45|-27.45| +2450815|68086|2450883|48375|1166269|3010|28504|48375|1166269|3010|28504|2|56|6|3|950|20|48|67|14.00|23.80|1.90|1467.30|127.30|938.00|1594.60|8.91|0.00|63.65|127.30|136.21|190.95|199.86|-810.70| +2450815|68086|2450863|48375|1166269|3010|28504|48375|1166269|3010|28504|2|99|6|4|9794|222|48|9|73.32|104.11|45.80|524.79|412.20|659.88|936.99|37.09|0.00|262.35|412.20|449.29|674.55|711.64|-247.68| +2450815|68086|2450895|48375|1166269|3010|28504|48375|1166269|3010|28504|2|12|7|4|12955|110|48|4|19.48|25.51|4.08|85.72|16.32|77.92|102.04|0.16|0.00|17.32|16.32|16.48|33.64|33.80|-61.60| +2450815|68086|2450890|48375|1166269|3010|28504|48375|1166269|3010|28504|2|25|13|2|13624|255|48|22|4.18|12.16|7.90|93.72|173.80|91.96|267.52|10.42|0.00|58.74|173.80|184.22|232.54|242.96|81.84| +2450815|68086|2450862|48375|1166269|3010|28504|48375|1166269|3010|28504|2|43|4|1|10774|33|48|13|99.13|297.39|89.21|2706.34|1159.73|1288.69|3866.07|34.79|0.00|425.23|1159.73|1194.52|1584.96|1619.75|-128.96| +2450815|68086|2450831|48375|1166269|3010|28504|48375|1166269|3010|28504|2|57|17|5|5428|99|48|9|62.41|149.15|2.98|1315.53|26.82|561.69|1342.35|1.07|0.00|335.52|26.82|27.89|362.34|363.41|-534.87| +2450815|68086|2450886|48375|1166269|3010|28504|48375|1166269|3010|28504|2|75|11|3|4504|45|48|54|29.69|50.76|42.63|439.02|2302.02|1603.26|2741.04|207.18|0.00|849.42|2302.02|2509.20|3151.44|3358.62|698.76| +2450815|68086|2450869|48375|1166269|3010|28504|48375|1166269|3010|28504|2|75|18|1|7819|285|48|76|8.91|13.45|12.10|102.60|919.60|677.16|1022.20|0.00|0.00|275.88|919.60|919.60|1195.48|1195.48|242.44| +2450815|68086|2450877|48375|1166269|3010|28504|48375|1166269|3010|28504|2|73|6|1|4174|174|48|66|12.75|18.87|7.92|722.70|522.72|841.50|1245.42|19.34|135.90|361.02|386.82|406.16|747.84|767.18|-454.68| +2450815|66515|2450886|28373|1195487|3754|3268|28373|1195487|3754|3268|2|2|4|2|16636|216|49|53|69.35|179.61|7.18|9138.79|380.54|3675.55|9519.33|30.44|0.00|2379.70|380.54|410.98|2760.24|2790.68|-3295.01| +2450815|66515|2450883|28373|1195487|3754|3268|28373|1195487|3754|3268|2|105|18|5|541|209|49|26|49.24|65.48|33.39|834.34|868.14|1280.24|1702.48|60.76|0.00|851.24|868.14|928.90|1719.38|1780.14|-412.10| +2450815|66515|2450884|28373|1195487|3754|3268|28373|1195487|3754|3268|2|2|19|5|17804|202|49|3|10.63|23.17|0.23|68.82|0.69|31.89|69.51|0.02|0.00|34.05|0.69|0.71|34.74|34.76|-31.20| +2450815|66515|2450884|28373|1195487|3754|3268|28373|1195487|3754|3268|2|13|16|5|17623|265|49|74|1.59|1.68|1.56|8.88|115.44|117.66|124.32|2.30|0.00|20.72|115.44|117.74|136.16|138.46|-2.22| +2450815|66515|2450845|28373|1195487|3754|3268|28373|1195487|3754|3268|2|50|8|1|7540|15|49|98|35.28|72.32|18.80|5244.96|1842.40|3457.44|7087.36|0.00|0.00|2550.94|1842.40|1842.40|4393.34|4393.34|-1615.04| +2450815|66515|2450894|28373|1195487|3754|3268|28373|1195487|3754|3268|2|27|18|3|6787|287|49|95|67.47|109.30|87.44|2076.70|8306.80|6409.65|10383.50|747.61|0.00|3945.35|8306.80|9054.41|12252.15|12999.76|1897.15| +2450815|66515|2450892|28373|1195487|3754|3268|28373|1195487|3754|3268|2|71|18|1|272|167|49|5|61.54|94.77|86.24|42.65|431.20|307.70|473.85|25.87|0.00|80.55|431.20|457.07|511.75|537.62|123.50| +2450815|66515|2450846|28373|1195487|3754|3268|28373|1195487|3754|3268|2|82|16|1|17228|13|49|98|73.57|210.41|44.18|16290.54|4329.64|7209.86|20620.18|173.18|0.00|618.38|4329.64|4502.82|4948.02|5121.20|-2880.22| +2450815|66515|2450859|28373|1195487|3754|3268|28373|1195487|3754|3268|2|63|12|1|14248|219|49|87|52.17|142.42|91.14|4461.36|7929.18|4538.79|12390.54|388.53|2378.75|5079.93|5550.43|5938.96|10630.36|11018.89|1011.64| +2450815|66515|2450818|28373|1195487|3754|3268|28373|1195487|3754|3268|2|72|20|2|15620|233|49|56|3.41|5.52|3.36|120.96|188.16|190.96|309.12|15.05|0.00|95.76|188.16|203.21|283.92|298.97|-2.80| +2450815|66515|2450863|28373|1195487|3754|3268|28373|1195487|3754|3268|2|88|15|5|6661|122|49|22|22.07|55.61|7.78|1052.26|171.16|485.54|1223.42|6.84|0.00|587.18|171.16|178.00|758.34|765.18|-314.38| +2450815|66515|2450840|28373|1195487|3754|3268|28373|1195487|3754|3268|2|98|6|2|7183|161|49|82|84.55|197.84|148.38|4055.72|12167.16|6933.10|16222.88|973.37|0.00|7624.36|12167.16|13140.53|19791.52|20764.89|5234.06| +2450815|60246|2450874|34061|340556|651|42746|34061|340556|651|42746|4|4|15|5|15127|125|50|78|18.12|36.05|28.47|591.24|2220.66|1413.36|2811.90|22.20|0.00|758.94|2220.66|2242.86|2979.60|3001.80|807.30| +2450815|60246|2450844|34061|340556|651|42746|34061|340556|651|42746|4|15|3|2|1081|285|50|97|59.61|127.56|127.56|0.00|12373.32|5782.17|12373.32|247.46|0.00|370.54|12373.32|12620.78|12743.86|12991.32|6591.15| +2450815|60246|2450893|34061|340556|651|42746|34061|340556|651|42746|4|21|3|3|1412|81|50|2|19.58|37.39|25.05|24.68|50.10|39.16|74.78|2.50|0.00|29.16|50.10|52.60|79.26|81.76|10.94| +2450815|60246|2450882|34061|340556|651|42746|34061|340556|651|42746|4|90|4|1|5953|169|50|2|92.73|218.84|83.15|271.38|166.30|185.46|437.68|8.31|0.00|140.04|166.30|174.61|306.34|314.65|-19.16| +2450815|60246|2450878|34061|340556|651|42746|34061|340556|651|42746|4|26|12|3|10168|271|50|31|9.75|13.06|7.70|166.16|238.70|302.25|404.86|9.54|0.00|202.43|238.70|248.24|441.13|450.67|-63.55| +2450815|60246|2450861|34061|340556|651|42746|34061|340556|651|42746|4|89|19|1|3286|293|50|93|32.18|60.17|59.56|56.73|5539.08|2992.74|5595.81|332.34|0.00|1901.85|5539.08|5871.42|7440.93|7773.27|2546.34| +2450815|60246|2450830|34061|340556|651|42746|34061|340556|651|42746|4|26|17|3|15691|13|50|93|78.48|102.80|93.54|861.18|8699.22|7298.64|9560.40|782.92|0.00|764.46|8699.22|9482.14|9463.68|10246.60|1400.58| +2450815|60246|2450893|34061|340556|651|42746|34061|340556|651|42746|4|69|13|4|7603|101|50|76|35.09|91.93|11.95|6078.48|908.20|2666.84|6986.68|81.73|0.00|768.36|908.20|989.93|1676.56|1758.29|-1758.64| +2450815|60246|2450818|34061|340556|651|42746|34061|340556|651|42746|4|55|19|4|11284|44|50|85|6.02|16.91|2.53|1222.30|215.05|511.70|1437.35|4.30|0.00|516.80|215.05|219.35|731.85|736.15|-296.65| +2450815|33125|2450853|91731|87099|6020|36541|91731|87099|6020|36541|1|87|4|1|3788|240|51|69|19.92|41.63|8.32|2298.39|574.08|1374.48|2872.47|9.47|258.33|143.52|315.75|325.22|459.27|468.74|-1058.73| +2450815|33125|2450838|91731|87099|6020|36541|91731|87099|6020|36541|1|32|19|3|5192|4|51|88|50.90|112.48|93.35|1683.44|8214.80|4479.20|9898.24|657.18|0.00|3761.12|8214.80|8871.98|11975.92|12633.10|3735.60| +2450815|33125|2450901|91731|87099|6020|36541|91731|87099|6020|36541|1|105|14|4|8654|132|51|28|13.71|30.84|5.55|708.12|155.40|383.88|863.52|1.55|0.00|397.04|155.40|156.95|552.44|553.99|-228.48| +2450815|33125|2450855|91731|87099|6020|36541|91731|87099|6020|36541|1|61|14|4|9998|116|51|95|19.54|30.67|17.48|1253.05|1660.60|1856.30|2913.65|149.45|0.00|494.95|1660.60|1810.05|2155.55|2305.00|-195.70| +2450815|33125|2450863|91731|87099|6020|36541|91731|87099|6020|36541|1|25|12|2|8996|154|51|79|80.22|166.05|124.53|3280.08|9837.87|6337.38|13117.95|590.27|0.00|655.70|9837.87|10428.14|10493.57|11083.84|3500.49| +2450815|11499|2450845|73786|196899|2059|33219|73786|196899|2059|33219|2|89|19|3|11152|193|52|86|80.84|140.66|2.81|11855.10|241.66|6952.24|12096.76|2.41|0.00|1087.90|241.66|244.07|1329.56|1331.97|-6710.58| +2450815|11499|2450835|73786|196899|2059|33219|73786|196899|2059|33219|2|24|18|1|15734|214|52|79|72.16|183.28|34.82|11728.34|2750.78|5700.64|14479.12|247.57|0.00|2606.21|2750.78|2998.35|5356.99|5604.56|-2949.86| +2450815|11499|2450883|73786|196899|2059|33219|73786|196899|2059|33219|2|26|10|4|5425|253|52|69|38.00|80.94|69.60|782.46|4802.40|2622.00|5584.86|96.04|0.00|502.32|4802.40|4898.44|5304.72|5400.76|2180.40| +2450815|11499|2450842|73786|196899|2059|33219|73786|196899|2059|33219|2|53|11|2|2596|256|52|17|17.67|34.98|20.63|243.95|350.71|300.39|594.66|7.01|0.00|154.53|350.71|357.72|505.24|512.25|50.32| +2450815|11499|2450865|73786|196899|2059|33219|73786|196899|2059|33219|2|1|7|4|10426|92|52|39|99.74|104.72|48.17|2205.45|1878.63|3889.86|4084.08|131.50|0.00|1551.81|1878.63|2010.13|3430.44|3561.94|-2011.23| +2450815|11499|2450862|73786|196899|2059|33219|73786|196899|2059|33219|2|7|3|3|9577|125|52|95|35.83|78.82|5.51|6964.45|523.45|3403.85|7487.90|20.93|0.00|2245.80|523.45|544.38|2769.25|2790.18|-2880.40| +2450815|11499|2450825|73786|196899|2059|33219|73786|196899|2059|33219|2|59|18|1|4874|158|52|58|74.84|109.26|6.55|5957.18|379.90|4340.72|6337.08|11.39|0.00|63.22|379.90|391.29|443.12|454.51|-3960.82| +2450815|11499|2450888|73786|196899|2059|33219|73786|196899|2059|33219|2|50|7|4|6434|46|52|57|82.14|225.06|54.01|9749.85|3078.57|4681.98|12828.42|123.14|0.00|4874.64|3078.57|3201.71|7953.21|8076.35|-1603.41| +2450815|11499|2450820|73786|196899|2059|33219|73786|196899|2059|33219|2|49|20|1|12487|201|52|24|29.99|89.67|85.18|107.76|2044.32|719.76|2152.08|163.54|0.00|451.92|2044.32|2207.86|2496.24|2659.78|1324.56| +2450815|11499|2450905|73786|196899|2059|33219|73786|196899|2059|33219|2|84|4|3|12164|183|52|82|8.46|11.50|8.85|217.30|725.70|693.72|943.00|14.51|0.00|377.20|725.70|740.21|1102.90|1117.41|31.98| +2450815|11499|2450904|73786|196899|2059|33219|73786|196899|2059|33219|2|104|2|2|13130|209|52|68|5.60|9.96|4.98|338.64|338.64|380.80|677.28|23.70|0.00|229.84|338.64|362.34|568.48|592.18|-42.16| +2450816|74666|2450897|67148|368370|2392|12461|67148|368370|2392|12461|4|40|13|4|9295|226|53|83|65.75|186.07|137.69|4015.54|11428.27|5457.25|15443.81|530.27|4799.87|3397.19|6628.40|7158.67|10025.59|10555.86|1171.15| +2450816|||67148|368370|2392|||368370||12461|4||||3634||53|65|||73.51|590.85|4778.15|2767.70|||0.00|0.00||4921.49|4778.15|4921.49|2010.45| +2450816|74666|2450850|67148|368370|2392|12461|67148|368370|2392|12461|4|48|11|1|10714|105|53|97|87.00|227.07|36.33|18501.78|3524.01|8439.00|22025.79|76.11|2572.52|4404.77|951.49|1027.60|5356.26|5432.37|-7487.51| +2450816|74666|2450836|67148|368370|2392|12461|67148|368370|2392|12461|4|27|8|4|12079|253|53|23|25.90|59.82|32.90|619.16|756.70|595.70|1375.86|0.00|0.00|577.76|756.70|756.70|1334.46|1334.46|161.00| +2450816|74666|2450822|67148|368370|2392|12461|67148|368370|2392|12461|4|22|16|5|15754|272|53|4|73.55|213.29|38.39|699.60|153.56|294.20|853.16|4.60|0.00|264.44|153.56|158.16|418.00|422.60|-140.64| +2450816|74666|2450869|67148|368370|2392|12461|67148|368370|2392|12461|4|51|10|2|9247|112|53|63|36.38|67.30|16.82|3180.24|1059.66|2291.94|4239.90|74.17|0.00|1610.91|1059.66|1133.83|2670.57|2744.74|-1232.28| +2450816|74666|2450824|67148|368370|2392|12461|67148|368370|2392|12461|4|83|9|2|10556|233|53|93|46.44|129.10|2.58|11766.36|239.94|4318.92|12006.30|7.19|0.00|2881.14|239.94|247.13|3121.08|3128.27|-4078.98| +2450816|74666|2450903|67148|368370|2392|12461|67148|368370|2392|12461|4|7|11|5|8284|6|53|4|66.12|130.25|110.71|78.16|442.84|264.48|521.00|13.28|0.00|229.24|442.84|456.12|672.08|685.36|178.36| +2450816|74666|2450903|67148|368370|2392|12461|67148|368370|2392|12461|4|48|20|1|3214|60|53|92|83.97|115.03|87.42|2540.12|8042.64|7725.24|10582.76|80.42|0.00|846.40|8042.64|8123.06|8889.04|8969.46|317.40| +2450816|4319|2450859|57853|112132|6411|30290|57853|112132|6411|30290|4|68|12|5|6476|204|54|26|91.11|262.39|191.54|1842.10|4980.04|2368.86|6822.14|298.80|0.00|1978.34|4980.04|5278.84|6958.38|7257.18|2611.18| +2450816|4319|2450894|57853|112132|6411|30290|57853|112132|6411|30290|4|40|7|1|7226|106|54|12|26.63|51.39|45.73|67.92|548.76|319.56|616.68|5.48|0.00|265.08|548.76|554.24|813.84|819.32|229.20| +2450816|4319|2450859|57853|112132|6411|30290|57853|112132|6411|30290|4|57|5|5|8200|283|54|69|92.27|156.85|119.20|2597.85|8224.80|6366.63|10822.65|246.74|0.00|2164.53|8224.80|8471.54|10389.33|10636.07|1858.17| +2450816|4319|2450897|57853|112132|6411|30290|57853|112132|6411|30290|4|86|11|1|7874|236|54|95|16.80|32.42|26.90|524.40|2555.50|1596.00|3079.90|6.64|2223.28|831.25|332.22|338.86|1163.47|1170.11|-1263.78| +2450816|4319|2450891|57853|112132|6411|30290|57853|112132|6411|30290|4|77|10|2|4118|256|54|87|96.28|180.04|163.83|1410.27|14253.21|8376.36|15663.48|570.12|0.00|3601.80|14253.21|14823.33|17855.01|18425.13|5876.85| +2450816|4319|2450833|57853|112132|6411|30290|57853|112132|6411|30290|4|75|3|3|16064|81|54|93|11.33|26.39|11.08|1423.83|1030.44|1053.69|2454.27|51.52|0.00|465.93|1030.44|1081.96|1496.37|1547.89|-23.25| +2450816|44905||85643|||46664|||3685||1|72||3|8098|48|55|||104.80|81.74|1937.04|||8803.20|223.15|||||7103.97|7327.12|1190.37| +2450816|44905|2450904|85643|1592501|4159|46664|338|156827|3685|33790|1|26|13|1|11665|268|55|5|96.79|173.25|147.26|129.95|736.30|483.95|866.25|44.17|0.00|259.85|736.30|780.47|996.15|1040.32|252.35| +2450816|44905|2450850|85643|1592501|4159|46664|338|156827|3685|33790|1|41|2|3|3907|265|55|99|48.31|141.06|132.59|838.53|13126.41|4782.69|13964.94|0.00|0.00|2513.61|13126.41|13126.41|15640.02|15640.02|8343.72| +2450816|44905|2450823|85643|1592501|4159|46664|338|156827|3685|33790|1|31|20|4|1891|22|55|55|47.04|95.02|75.06|1097.80|4128.30|2587.20|5226.10|206.41|0.00|627.00|4128.30|4334.71|4755.30|4961.71|1541.10| +2450816|44905|2450870|85643|1592501|4159|46664|338|156827|3685|33790|1|39|14|4|15439|201|55|51|27.94|65.65|22.32|2209.83|1138.32|1424.94|3348.15|56.91|0.00|1171.47|1138.32|1195.23|2309.79|2366.70|-286.62| +2450816|44905|2450838|85643|1592501|4159|46664|338|156827|3685|33790|1|69|11|2|17827|144|55|1|95.71|200.03|26.00|174.03|26.00|95.71|200.03|2.34|0.00|84.01|26.00|28.34|110.01|112.35|-69.71| +2450816|44905|2450862|85643|1592501|4159|46664|338|156827|3685|33790|1|50|5|3|7142|1|55|43|84.46|86.14|37.90|2074.32|1629.70|3631.78|3704.02|114.07|0.00|962.77|1629.70|1743.77|2592.47|2706.54|-2002.08| +2450816|44905|2450889|85643|1592501|4159|46664|338|156827|3685|33790|1|11|14|5|2887|215|55|3|18.08|26.57|21.25|15.96|63.75|54.24|79.71|2.55|0.00|32.67|63.75|66.30|96.42|98.97|9.51| +2450816|44905|2450842|85643|1592501|4159|46664|338|156827|3685|33790|1|51|6|1|8570|52|55|2|16.10|26.88|22.04|9.68|44.08|32.20|53.76|0.00|15.86|2.68|28.22|28.22|30.90|30.90|-3.98| +2450816|44905|2450848|85643|1592501|4159|46664|338|156827|3685|33790|1|52|2|4|3259|193|55|42|36.26|82.67|53.73|1215.48|2256.66|1522.92|3472.14|203.09|0.00|590.10|2256.66|2459.75|2846.76|3049.85|733.74| +2450816|83041|2450862|29127|1346606|6038|14270|29127|1346606|6038|14270|1|92|10|5|15134|297|56|16|98.43|151.58|1.51|2401.12|24.16|1574.88|2425.28|0.24|0.00|24.16|24.16|24.40|48.32|48.56|-1550.72| +2450816|83041|2450876|29127|1346606|6038|14270|29127|1346606|6038|14270|1|51|13|3|17128|130|56|12|73.40|148.26|72.64|907.44|871.68|880.80|1779.12|69.73|0.00|320.16|871.68|941.41|1191.84|1261.57|-9.12| +2450816|83041|2450883|29127|1346606|6038|14270|29127|1346606|6038|14270|1|13|5|4|11612|147|56|71|28.61|50.63|40.50|719.23|2875.50|2031.31|3594.73|172.53|0.00|1258.12|2875.50|3048.03|4133.62|4306.15|844.19| +2450816|83041|2450838|29127|1346606|6038|14270|29127|1346606|6038|14270|1|32|1|3|6841|134|56|100|4.68|10.99|4.50|649.00|450.00|468.00|1099.00|40.50|0.00|494.00|450.00|490.50|944.00|984.50|-18.00| +2450816|83041|2450868|29127|1346606|6038|14270|29127|1346606|6038|14270|1|50|15|1|16726|38|56|19|9.10|12.55|4.14|159.79|78.66|172.90|238.45|0.00|0.00|14.25|78.66|78.66|92.91|92.91|-94.24| +2450816|83041|2450868|29127|1346606|6038|14270|29127|1346606|6038|14270|1|59|13|5|16555|142|56|78|10.64|10.85|8.89|152.88|693.42|829.92|846.30|0.00|693.42|194.22|0.00|0.00|194.22|194.22|-829.92| +2450816|83041|2450855|29127|1346606|6038|14270|29127|1346606|6038|14270|1|68|2|2|13750|150|56|97|27.76|38.86|32.64|603.34|3166.08|2692.72|3769.42|126.64|0.00|414.19|3166.08|3292.72|3580.27|3706.91|473.36| +2450816|4728|2450825|42732|677128|5571|31567|68187|226006|2802|22445|2|24|12|1|14396|116|57|3|88.27|215.37|36.61|536.28|109.83|264.81|646.11|5.49|0.00|25.83|109.83|115.32|135.66|141.15|-154.98| +|||42732|677128|5571||68187|||22445||||5|1018|254|57|42||130.27|113.33|||1968.12||||||4997.85|7440.72|7678.71|2791.74| +2450816|4728|2450873|42732|677128|5571|31567|68187|226006|2802|22445|2|88|7|2|16045|85|57|3|66.32|133.96|53.58|241.14|160.74|198.96|401.88|1.35|126.98|160.74|33.76|35.11|194.50|195.85|-165.20| +2450816|4728|2450852|42732|677128|5571|31567|68187|226006|2802|22445|2|34|14|3|3862|293|57|66|23.31|67.13|56.38|709.50|3721.08|1538.46|4430.58|334.89|0.00|2170.74|3721.08|4055.97|5891.82|6226.71|2182.62| +2450816|4728|2450822|42732|677128|5571|31567|68187|226006|2802|22445|2|34|3|4|11264|107|57|86|46.87|117.64|89.40|2428.64|7688.40|4030.82|10117.04|99.94|2690.94|1922.10|4997.46|5097.40|6919.56|7019.50|966.64| +2450816|4728|2450872|42732|677128|5571|31567|68187|226006|2802|22445|2|27|16|2|9289|251|57|6|42.36|83.02|73.88|54.84|443.28|254.16|498.12|0.00|0.00|109.56|443.28|443.28|552.84|552.84|189.12| +2450816|4728|2450885|42732|677128|5571|31567|68187|226006|2802|22445|2|63|3|1|12934|250|57|40|19.19|23.79|11.89|476.00|475.60|767.60|951.60|19.02|0.00|256.80|475.60|494.62|732.40|751.42|-292.00| +2450816|4728|2450843|42732|677128|5571|31567|68187|226006|2802|22445|2|7|3|2|9340|66|57|36|84.31|108.75|77.21|1135.44|2779.56|3035.16|3915.00|222.36|0.00|587.16|2779.56|3001.92|3366.72|3589.08|-255.60| +2450816|4728|2450853|42732|677128|5571|31567|68187|226006|2802|22445|2|62|15|5|16640|17|57|47|18.48|32.70|5.55|1276.05|260.85|868.56|1536.90|7.82|0.00|537.68|260.85|268.67|798.53|806.35|-607.71| +2450816|4728|2450846|42732|677128|5571|31567|68187|226006|2802|22445|2|85|3|3|13765|78|57|37|89.13|98.93|3.95|3514.26|146.15|3297.81|3660.41|7.30|0.00|1756.76|146.15|153.45|1902.91|1910.21|-3151.66| +2450816|4728|2450837|42732|677128|5571|31567|68187|226006|2802|22445|2|31|3|3|8152|121|57|46|67.23|129.08|5.16|5700.32|237.36|3092.58|5937.68|7.12|0.00|534.06|237.36|244.48|771.42|778.54|-2855.22| +2450816|42807|2450902|31940|1030863|6139|11062|31940|1030863|6139|11062|2|9|19|5|352|290|58|1|34.47|79.28|18.23|61.05|18.23|34.47|79.28|1.27|0.00|6.34|18.23|19.50|24.57|25.84|-16.24| +2450816|42807|2450864|31940|1030863|6139|11062|31940|1030863|6139|11062|2|64|18|4|782|19|58|60|51.80|144.52|82.37|3729.00|4942.20|3108.00|8671.20|197.68|0.00|693.60|4942.20|5139.88|5635.80|5833.48|1834.20| +2450816|42807|2450825|31940|1030863|6139|11062|31940|1030863|6139|11062|2|54|12|1|11486|284|58|2|13.38|24.35|0.73|47.24|1.46|26.76|48.70|0.02|0.00|14.60|1.46|1.48|16.06|16.08|-25.30| +2450816|42807|2450867|31940|1030863|6139|11062|31940|1030863|6139|11062|2|56|16|2|7123|181|58|3|80.78|208.41|164.64|131.31|493.92|242.34|625.23|39.51|0.00|106.26|493.92|533.43|600.18|639.69|251.58| +2450816|42807|2450895|31940|1030863|6139|11062|31940|1030863|6139|11062|2|23|13|2|8888|77|58|80|87.35|172.95|39.77|10654.40|3181.60|6988.00|13836.00|159.08|0.00|276.00|3181.60|3340.68|3457.60|3616.68|-3806.40| +2450816|42807|2450828|31940|1030863|6139|11062|31940|1030863|6139|11062|2|41|17|1|8668|257|58|84|71.52|83.67|24.26|4990.44|2037.84|6007.68|7028.28|0.00|0.00|2740.92|2037.84|2037.84|4778.76|4778.76|-3969.84| +2450816|42807|2450890|31940|1030863|6139|11062|31940|1030863|6139|11062|2|92|1|4|10760|23|58|50|98.82|153.17|13.78|6969.50|689.00|4941.00|7658.50|0.00|0.00|76.50|689.00|689.00|765.50|765.50|-4252.00| +2450816|42807|2450873|31940|1030863|6139|11062|31940|1030863|6139|11062|2|81|4|1|8341|25|58|94|19.85|53.99|49.67|406.08|4668.98|1865.90|5075.06|420.20|0.00|1268.06|4668.98|5089.18|5937.04|6357.24|2803.08| +2450816|42807|2450894|31940|1030863|6139|11062|31940|1030863|6139|11062|2|13|4|1|3302|85|58|56|94.17|124.30|119.32|278.88|6681.92|5273.52|6960.80|0.00|6147.36|69.44|534.56|534.56|604.00|604.00|-4738.96| +2450816|42807|2450870|31940|1030863|6139|11062|31940|1030863|6139|11062|2|68|2|2|12229|54|58|69|57.38|75.16|32.31|2956.65|2229.39|3959.22|5186.04|89.17|0.00|881.13|2229.39|2318.56|3110.52|3199.69|-1729.83| +2450816|42807|2450856|31940|1030863|6139|11062|31940|1030863|6139|11062|2|24|11|4|9202|79|58|59|73.18|182.95|142.70|2374.75|8419.30|4317.62|10794.05|84.19|0.00|2050.84|8419.30|8503.49|10470.14|10554.33|4101.68| +2450816|48800|2450832|83021|835564|2860|29403|83021|835564|2860|29403|2|102|13|2|17293|82|59|81|26.64|69.53|57.01|1014.12|4617.81|2157.84|5631.93|230.89|0.00|1070.01|4617.81|4848.70|5687.82|5918.71|2459.97| +2450816|48800|2450836|83021|835564|2860|29403|83021|835564|2860|29403|2|74|5|3|2650|6|59|40|63.02|86.33|37.12|1968.40|1484.80|2520.80|3453.20|59.39|0.00|103.20|1484.80|1544.19|1588.00|1647.39|-1036.00| +2450816|48800|2450837|83021|835564|2860|29403|83021|835564|2860|29403|2|2|7|3|16129|200|59|81|19.97|33.15|21.87|913.68|1771.47|1617.57|2685.15|19.13|1293.17|1288.71|478.30|497.43|1767.01|1786.14|-1139.27| +2450816|48800|2450835|83021|835564|2860|29403|83021|835564|2860|29403|2|7|11|3|4520|240|59|68|67.48|176.12|107.43|4670.92|7305.24|4588.64|11976.16|368.18|2045.46|2514.64|5259.78|5627.96|7774.42|8142.60|671.14| +2450816|79982|2450905|72650|445332|5661|17543|72650|445332|5661|17543|2|40|9|4|9938|272|60|100|21.02|46.66|35.46|1120.00|3546.00|2102.00|4666.00|141.84|0.00|1026.00|3546.00|3687.84|4572.00|4713.84|1444.00| +2450816|79982|2450866|72650|445332|5661|17543|72650|445332|5661|17543|2|75|4|5|8290|70|60|100|50.81|131.59|75.00|5659.00|7500.00|5081.00|13159.00|525.00|0.00|526.00|7500.00|8025.00|8026.00|8551.00|2419.00| +2450816|79982|2450871|72650|445332|5661|17543|72650|445332|5661|17543|2|59|1|2|6242|174|60|97|78.19|162.63|138.23|2366.80|13408.31|7584.43|15775.11|134.08|0.00|630.50|13408.31|13542.39|14038.81|14172.89|5823.88| +2450816|79982|2450840|72650|445332|5661|17543|72650|445332|5661|17543|2|100|1|3|17252|144|60|50|85.48|164.97|67.63|4867.00|3381.50|4274.00|8248.50|0.00|0.00|3794.00|3381.50|3381.50|7175.50|7175.50|-892.50| +2450816|29157|2450866|38665|1571317|2722|1327|38665|1571317|2722|1327|2|103|19|3|5222|209|61|64|51.04|83.70|5.85|4982.40|374.40|3266.56|5356.80|3.74|0.00|267.52|374.40|378.14|641.92|645.66|-2892.16| +2450816|29157|2450880|38665|1571317|2722|1327|38665|1571317|2722|1327|2|106|1|5|14857|28|61|86|6.17|7.58|1.51|522.02|129.86|530.62|651.88|11.68|0.00|266.60|129.86|141.54|396.46|408.14|-400.76| +2450816|29157|2450877|38665|1571317|2722|1327|38665|1571317|2722|1327|2|105|20|5|8408|5|61|96|43.88|50.90|35.12|1514.88|3371.52|4212.48|4886.40|236.00|0.00|48.00|3371.52|3607.52|3419.52|3655.52|-840.96| +2450816|29157|2450829|38665|1571317|2722|1327|38665|1571317|2722|1327|2|25|3|5|356|176|61|95|94.26|190.40|108.52|7778.60|10309.40|8954.70|18088.00|269.07|7319.67|1265.40|2989.73|3258.80|4255.13|4524.20|-5964.97| +2450816|29157|2450829|38665|1571317|2722|1327|38665|1571317|2722|1327|2|45|10|2|9217|66|61|63|6.79|18.06|2.88|956.34|181.44|427.77|1137.78|7.25|0.00|170.10|181.44|188.69|351.54|358.79|-246.33| +2450816|29157|2450841|38665|1571317|2722|1327|38665|1571317|2722|1327|2|74|6|5|17486|58|61|87|26.34|66.11|52.88|1151.01|4600.56|2291.58|5751.57|414.05|0.00|2357.70|4600.56|5014.61|6958.26|7372.31|2308.98| +2450816|29157|2450898|38665|1571317|2722|1327|38665|1571317|2722|1327|2|73|17|3|7664|64|61|29|39.50|42.26|41.83|12.47|1213.07|1145.50|1225.54|0.00|0.00|416.44|1213.07|1213.07|1629.51|1629.51|67.57| +2450816|29157|2450872|38665|1571317|2722|1327|38665|1571317|2722|1327|2|20|4|3|7378|69|61|23|48.30|71.48|1.42|1611.38|32.66|1110.90|1644.04|0.95|0.97|772.57|31.69|32.64|804.26|805.21|-1079.21| +2450816|29157|2450852|38665|1571317|2722|1327|38665|1571317|2722|1327|2|25|15|2|10459|176|61|82|6.17|10.55|10.33|18.04|847.06|505.94|865.10|25.41|0.00|337.02|847.06|872.47|1184.08|1209.49|341.12| +2450816|29157|2450860|38665|1571317|2722|1327|38665|1571317|2722|1327|2|57|14|3|2494|4|61|49|46.44|67.33|8.07|2903.74|395.43|2275.56|3299.17|27.68|0.00|263.62|395.43|423.11|659.05|686.73|-1880.13| +2450816|65060|2450818|363|683276|4942|29055|363|683276|4942|29055|4|81|20|4|16364|144|62|63|84.05|126.07|85.72|2542.05|5400.36|5295.15|7942.41|108.00|0.00|2700.18|5400.36|5508.36|8100.54|8208.54|105.21| +2450816|65060|2450819|363|683276|4942|29055|363|683276|4942|29055|4|86|10|3|17722|77|62|57|83.74|85.41|84.55|49.02|4819.35|4773.18|4868.37|192.77|0.00|2044.59|4819.35|5012.12|6863.94|7056.71|46.17| +2450816|65060|2450828|363|683276|4942|29055|363|683276|4942|29055|4|30|5|3|17647|22|62|71|36.70|45.87|24.76|1498.81|1757.96|2605.70|3256.77|52.73|0.00|1302.14|1757.96|1810.69|3060.10|3112.83|-847.74| +2450816|65060|2450877|363|683276|4942|29055|363|683276|4942|29055|4|47|11|4|352|266|62|15|43.33|64.12|11.54|788.70|173.10|649.95|961.80|0.00|0.00|298.05|173.10|173.10|471.15|471.15|-476.85| +2450816|65060|2450834|363|683276|4942|29055|363|683276|4942|29055|4|44|19|4|782|118|62|88|27.09|41.44|6.63|3063.28|583.44|2383.92|3646.72|46.67|0.00|364.32|583.44|630.11|947.76|994.43|-1800.48| +2450816|65060|2450862|363|683276|4942|29055|363|683276|4942|29055|4|42|10|1|11486|8|62|29|41.24|114.23|38.83|2186.60|1126.07|1195.96|3312.67|67.56|0.00|827.95|1126.07|1193.63|1954.02|2021.58|-69.89| +2450816|65060|2450836|363|683276|4942|29055|363|683276|4942|29055|4|30|1|5|7123|33|62|43|63.02|73.73|61.19|539.22|2631.17|2709.86|3170.39|117.87|947.22|665.64|1683.95|1801.82|2349.59|2467.46|-1025.91| +2450816|65060|2450846|363|683276|4942|29055|363|683276|4942|29055|4|50|16|2|8888|267|62|77|75.43|101.07|5.05|7393.54|388.85|5808.11|7782.39|3.88|0.00|0.00|388.85|392.73|388.85|392.73|-5419.26| +2450816|65060|2450894|363|683276|4942|29055|363|683276|4942|29055|4|14|8|3|8668|226|62|11|22.08|54.75|53.10|18.15|584.10|242.88|602.25|46.72|0.00|210.76|584.10|630.82|794.86|841.58|341.22| +2450816|78774|2450856|84758|1148029|2853|18124|84758|1148029|2853|18124|4|27|6|3|10978|47|63|74|60.08|138.18|135.41|204.98|10020.34|4445.92|10225.32|0.00|8116.47|4396.34|1903.87|1903.87|6300.21|6300.21|-2542.05| +2450816|78774|2450896|84758|1148029|2853|18124|84758|1148029|2853|18124|4|106|1|5|2830|78|63|90|10.43|14.81|0.74|1266.30|66.60|938.70|1332.90|3.99|0.00|292.50|66.60|70.59|359.10|363.09|-872.10| +2450816|78774|2450818|84758|1148029|2853|18124|84758|1148029|2853|18124|4|5|10|3|985|44|63|87|96.10|263.31|50.02|18556.23|4351.74|8360.70|22907.97|261.10|0.00|11224.74|4351.74|4612.84|15576.48|15837.58|-4008.96| +2450816|78774|2450874|84758|1148029|2853|18124|84758|1148029|2853|18124|4|91|16|4|8704|273|63|55|40.80|115.05|62.12|2911.15|3416.60|2244.00|6327.75|59.79|2562.45|2404.05|854.15|913.94|3258.20|3317.99|-1389.85| +2450816|78774|2450853|84758|1148029|2853|18124|84758|1148029|2853|18124|4|57|13|2|17342|133|63|45|74.49|213.04|57.52|6998.40|2588.40|3352.05|9586.80|20.96|1889.53|2971.80|698.87|719.83|3670.67|3691.63|-2653.18| +2450816|78774|2450891|84758|1148029|2853|18124|84758|1148029|2853|18124|4|76|15|3|4786|98|63|63|94.46|265.43|148.64|7357.77|9364.32|5950.98|16722.09|719.18|374.57|1337.49|8989.75|9708.93|10327.24|11046.42|3038.77| +2450816|78774|2450906|84758|1148029|2853|18124|84758|1148029|2853|18124|4|68|15|5|10867|223|63|4|11.40|17.78|16.35|5.72|65.40|45.60|71.12|2.61|0.00|7.80|65.40|68.01|73.20|75.81|19.80| +2450816|78774|2450887|84758|1148029|2853|18124|84758|1148029|2853|18124|4|105|11|2|9398|234|63|11|49.21|73.81|21.40|576.51|235.40|541.31|811.91|9.41|0.00|113.63|235.40|244.81|349.03|358.44|-305.91| +2450816|78774|2450827|84758|1148029|2853|18124|84758|1148029|2853|18124|4|61|15|1|10132|165|63|7|67.77|146.38|102.46|307.44|717.22|474.39|1024.66|14.34|0.00|440.58|717.22|731.56|1157.80|1172.14|242.83| +2450816|78774|2450891|84758|1148029|2853|18124|84758|1148029|2853|18124|4|90|19|2|9646|180|63|93|10.22|29.43|12.06|1615.41|1121.58|950.46|2736.99|11.21|0.00|848.16|1121.58|1132.79|1969.74|1980.95|171.12| +2450816|78774|2450903|84758|1148029|2853|18124|84758|1148029|2853|18124|4|10|15|1|13448|223|63|43|36.53|70.86|36.13|1493.39|1553.59|1570.79|3046.98|62.14|0.00|761.53|1553.59|1615.73|2315.12|2377.26|-17.20| +2450816|78774|2450838|84758|1148029|2853|18124|84758|1148029|2853|18124|4|85|19|3|13972|18|63|73|56.18|118.53|88.89|2163.72|6488.97|4101.14|8652.69|324.44|0.00|1557.09|6488.97|6813.41|8046.06|8370.50|2387.83| +2450816|78774|2450841|84758|1148029|2853|18124|84758|1148029|2853|18124|4|49|15|2|1544|43|63|95|67.74|159.86|150.26|912.00|14274.70|6435.30|15186.70|0.00|0.00|3796.20|14274.70|14274.70|18070.90|18070.90|7839.40| +2450816|5478|2450822|55621|1225267|960|15053|55621|1225267|960|15053|1|49|4|1|5180|90|64|8|40.95|54.05|39.99|112.48|319.92|327.60|432.40|2.49|236.74|181.60|83.18|85.67|264.78|267.27|-244.42| +2450816|5478|2450861|55621|1225267|960|15053|55621|1225267|960|15053|1|102|10|4|14344|122|64|43|76.53|202.03|103.03|4257.00|4430.29|3290.79|8687.29|398.72|0.00|1910.92|4430.29|4829.01|6341.21|6739.93|1139.50| +2450816|5478|2450822|55621|1225267|960|15053|55621|1225267|960|15053|1|41|7|3|12986|260|64|48|97.61|136.65|5.46|6297.12|262.08|4685.28|6559.20|10.48|0.00|1836.48|262.08|272.56|2098.56|2109.04|-4423.20| +2450816|5478|2450865|55621|1225267|960|15053|55621|1225267|960|15053|1|46|13|5|4819|142|64|88|63.32|139.93|135.73|369.60|11944.24|5572.16|12313.84|200.66|8599.85|1969.44|3344.39|3545.05|5313.83|5514.49|-2227.77| +2450816|5478|2450855|55621|1225267|960|15053|55621|1225267|960|15053|1|43|16|5|9418|250|64|3|2.23|2.85|0.17|8.04|0.51|6.69|8.55|0.02|0.00|2.28|0.51|0.53|2.79|2.81|-6.18| +2450816|5478|2450855|55621|1225267|960|15053|55621|1225267|960|15053|1|26|20|4|16870|119|64|55|56.33|87.87|46.57|2271.50|2561.35|3098.15|4832.85|102.45|0.00|289.85|2561.35|2663.80|2851.20|2953.65|-536.80| +2450816|5478|2450820|55621|1225267|960|15053|55621|1225267|960|15053|1|93|15|5|1255|240|64|28|70.49|70.49|39.47|868.56|1105.16|1973.72|1973.72|66.30|0.00|947.24|1105.16|1171.46|2052.40|2118.70|-868.56| +2450816|5478|2450864|55621|1225267|960|15053|55621|1225267|960|15053|1|23|13|4|9752|257|64|56|18.97|53.49|27.27|1468.32|1527.12|1062.32|2995.44|11.30|962.08|1467.76|565.04|576.34|2032.80|2044.10|-497.28| +2450816|5478|2450886|55621|1225267|960|15053|55621|1225267|960|15053|1|101|20|4|3272|111|64|34|14.00|32.90|24.01|302.26|816.34|476.00|1118.60|8.16|0.00|525.64|816.34|824.50|1341.98|1350.14|340.34| +2450816|59988|2450818|10055|1667005|6355|39352|10055|1667005|6355|39352|4|63|15|5|2755|250|65|44|75.74|82.55|9.90|3196.60|435.60|3332.56|3632.20|34.84|0.00|1307.24|435.60|470.44|1742.84|1777.68|-2896.96| +2450816|59988|2450820|10055|1667005|6355|39352|10055|1667005|6355|39352|4|34|19|5|8096|9|65|50|30.86|39.50|1.97|1876.50|98.50|1543.00|1975.00|0.67|75.84|158.00|22.66|23.33|180.66|181.33|-1520.34| +2450816|59988|2450847|10055|1667005|6355|39352|10055|1667005|6355|39352|4|70|18|4|15664|52|65|52|71.28|73.41|55.79|916.24|2901.08|3706.56|3817.32|203.07|0.00|1259.44|2901.08|3104.15|4160.52|4363.59|-805.48| +2450816|59988|2450827|10055|1667005|6355|39352|10055|1667005|6355|39352|4|94|2|4|3121|206|65|20|32.45|86.31|83.72|51.80|1674.40|649.00|1726.20|0.00|0.00|811.20|1674.40|1674.40|2485.60|2485.60|1025.40| +2450816|39433|2450883|6336|785982|3228|32874|6336|785982|3228|32874|4|83|15|5|13288|53|66|94|85.52|253.99|172.71|7640.32|16234.74|8038.88|23875.06|811.73|0.00|5729.30|16234.74|17046.47|21964.04|22775.77|8195.86| +2450816|39433|2450868|6336|785982|3228|32874|6336|785982|3228|32874|4|78|17|2|242|170|66|75|95.79|104.41|60.55|3289.50|4541.25|7184.25|7830.75|227.06|0.00|1252.50|4541.25|4768.31|5793.75|6020.81|-2643.00| +2450816|39433|2450853|6336|785982|3228|32874|6336|785982|3228|32874|4|49|3|4|3124|162|66|30|83.39|118.41|46.17|2167.20|1385.10|2501.70|3552.30|96.95|0.00|461.70|1385.10|1482.05|1846.80|1943.75|-1116.60| +2450816|39433|2450853|6336|785982|3228|32874|6336|785982|3228|32874|4|64|15|2|3325|246|66|77|75.94|166.30|19.95|11268.95|1536.15|5847.38|12805.10|122.89|0.00|2944.48|1536.15|1659.04|4480.63|4603.52|-4311.23| +2450816|39433|2450874|6336|785982|3228|32874|6336|785982|3228|32874|4|5|16|2|946|51|66|24|85.55|122.33|88.07|822.24|2113.68|2053.20|2935.92|21.13|0.00|146.64|2113.68|2134.81|2260.32|2281.45|60.48| +2450816|39433|2450846|6336|785982|3228|32874|6336|785982|3228|32874|4|6|13|5|2012|187|66|11|31.70|60.23|56.01|46.42|616.11|348.70|662.53|36.22|98.57|245.08|517.54|553.76|762.62|798.84|168.84| +2450816|39433|2450828|6336|785982|3228|32874|6336|785982|3228|32874|4|90|12|4|17809|87|66|75|31.58|31.58|25.57|450.75|1917.75|2368.50|2368.50|19.17|0.00|117.75|1917.75|1936.92|2035.50|2054.67|-450.75| +2450816|39433|2450859|6336|785982|3228|32874|6336|785982|3228|32874|4|57|11|4|2233|165|66|6|73.69|82.53|56.12|158.46|336.72|442.14|495.18|3.36|0.00|9.90|336.72|340.08|346.62|349.98|-105.42| +2450816|39433|2450820|6336|785982|3228|32874|6336|785982|3228|32874|4|64|8|4|5350|48|66|25|22.14|63.54|26.68|921.50|667.00|553.50|1588.50|46.69|0.00|603.50|667.00|713.69|1270.50|1317.19|113.50| +2450816|18155|2450850|47214|1227752|1090|17181|47214|1227752|1090|17181|2|5|12|5|17632|247|67|60|2.31|3.92|1.88|122.40|112.80|138.60|235.20|10.15|0.00|4.20|112.80|122.95|117.00|127.15|-25.80| +2450816|18155|2450888|47214|1227752|1090|17181|47214|1227752|1090|17181|2|108|6|5|11936|76|67|56|47.66|110.09|5.50|5857.04|308.00|2668.96|6165.04|27.72|0.00|1170.96|308.00|335.72|1478.96|1506.68|-2360.96| +2450816|18155|2450853|47214|1227752|1090|17181|47214|1227752|1090|17181|2|60|13|2|19|61|67|46|40.69|93.99|6.57|4021.32|302.22|1871.74|4323.54|12.08|0.00|1426.46|302.22|314.30|1728.68|1740.76|-1569.52| +2450816|18155|2450840|47214|1227752|1090|17181|47214|1227752|1090|17181|2|40|18|1|10244|61|67|62|91.96|263.92|237.52|1636.80|14726.24|5701.52|16363.04|147.26|0.00|7363.12|14726.24|14873.50|22089.36|22236.62|9024.72| +2450816|18155|2450890|47214|1227752|1090|17181|47214|1227752|1090|17181|2|68|12|4|17234|96|67|93|74.96|111.69|14.51|9037.74|1349.43|6971.28|10387.17|94.46|0.00|3219.66|1349.43|1443.89|4569.09|4663.55|-5621.85| +2450816|18155|2450827|47214|1227752|1090|17181|47214|1227752|1090|17181|2|69|3|4|5908|147|67|18|12.59|25.80|10.83|269.46|194.94|226.62|464.40|13.64|0.00|208.98|194.94|208.58|403.92|417.56|-31.68| +2450816|18155|2450838|47214|1227752|1090|17181|47214|1227752|1090|17181|2|68|12|2|5269|277|67|34|79.93|201.42|159.12|1438.20|5410.08|2717.62|6848.28|291.60|1244.31|1917.26|4165.77|4457.37|6083.03|6374.63|1448.15| +2450816|18155|2450827|47214|1227752|1090|17181|47214|1227752|1090|17181|2|53|14|3|2258|204|67|76|53.04|143.73|34.49|8302.24|2621.24|4031.04|10923.48|209.69|0.00|1529.12|2621.24|2830.93|4150.36|4360.05|-1409.80| +2450816|18155|2450829|47214|1227752|1090|17181|47214|1227752|1090|17181|2|102|12|1|14866|211|67|97|1.09|1.91|0.17|168.78|16.49|105.73|185.27|1.31|0.00|6.79|16.49|17.80|23.28|24.59|-89.24| +2450816|18155|2450821|47214|1227752|1090|17181|47214|1227752|1090|17181|2|63|15|4|8614|134|67|48|24.44|27.61|18.49|437.76|887.52|1173.12|1325.28|79.87|0.00|52.80|887.52|967.39|940.32|1020.19|-285.60| +2450816|18155|2450884|47214|1227752|1090|17181|47214|1227752|1090|17181|2|13|19|4|6692|87|67|7|61.70|71.57|4.29|470.96|30.03|431.90|500.99|2.40|0.00|120.19|30.03|32.43|150.22|152.62|-401.87| +2450816|18155|2450888|47214|1227752|1090|17181|47214|1227752|1090|17181|2|18|5|5|4015|38|67|33|6.98|10.60|1.06|314.82|34.98|230.34|349.80|2.79|0.00|17.49|34.98|37.77|52.47|55.26|-195.36| +2450816|18155|2450847|47214|1227752|1090|17181|47214|1227752|1090|17181|2|14|13|2|3211|285|67|74|93.83|188.59|52.80|10048.46|3907.20|6943.42|13955.66|0.00|0.00|2092.72|3907.20|3907.20|5999.92|5999.92|-3036.22| +2450816|18155|2450883|47214|1227752|1090|17181|47214|1227752|1090|17181|2|17|12|3|4375|285|67|87|2.95|6.78|6.50|24.36|565.50|256.65|589.86|39.58|0.00|212.28|565.50|605.08|777.78|817.36|308.85| +2450816||2450864|84907|163011||44819|84907||3515|||34|||5581||68|||100.64||1648.53||||7.96|2117.07||||276.75|284.71|-1683.39| +2450816|19837|2450872|84907|163011|3515|44819|84907|163011|3515|44819|2|47|18|5|14252|156|68|84|15.18|19.43|11.26|686.28|945.84|1275.12|1632.12|18.91|0.00|782.88|945.84|964.75|1728.72|1747.63|-329.28| +2450816|19837|2450887|84907|163011|3515|44819|84907|163011|3515|44819|2|55|3|3|13276|83|68|66|77.14|199.02|173.14|1708.08|11427.24|5091.24|13135.32|571.36|0.00|3020.82|11427.24|11998.60|14448.06|15019.42|6336.00| +2450816|19837||84907|163011|3515|||163011||44819|2|||2|14410|100|68|20|61.58|115.15||69.20|2233.80||||0.00|621.80|2233.80||2855.60|3056.64|1002.20| +2450816|19837|2450849|84907|163011|3515|44819|84907|163011|3515|44819|2|22|20|5|14756|283|68|45|93.08|202.91|22.32|8126.55|1004.40|4188.60|9130.95|30.13|0.00|4017.60|1004.40|1034.53|5022.00|5052.13|-3184.20| +2450816|19837|2450854|84907|163011|3515|44819|84907|163011|3515|44819|2|91|17|4|8332|63|68|88|13.05|38.10|3.04|3085.28|267.52|1148.40|3352.80|0.00|0.00|938.08|267.52|267.52|1205.60|1205.60|-880.88| +2450816|19837|2450851|84907|163011|3515|44819|84907|163011|3515|44819|2|17|9|3|9338|204|68|22|43.48|115.65|56.66|1297.78|1246.52|956.56|2544.30|49.86|0.00|814.00|1246.52|1296.38|2060.52|2110.38|289.96| +2450816|19837|2450839|84907|163011|3515|44819|84907|163011|3515|44819|2|79|9|2|14731|288|68|52|99.73|149.59|52.35|5056.48|2722.20|5185.96|7778.68|0.00|0.00|3655.60|2722.20|2722.20|6377.80|6377.80|-2463.76| +2450816|19837|2450868|84907|163011|3515|44819|84907|163011|3515|44819|2|28|9|4|4076|142|68|28|24.77|47.80|37.76|281.12|1057.28|693.56|1338.40|21.14|0.00|147.00|1057.28|1078.42|1204.28|1225.42|363.72| +2450816|19837|2450827|84907|163011|3515|44819|84907|163011|3515|44819|2|32|19|3|12316|176|68|51|28.83|80.14|24.04|2861.10|1226.04|1470.33|4087.14|24.52|0.00|1307.64|1226.04|1250.56|2533.68|2558.20|-244.29| +2450816|19837|2450850|84907|163011|3515|44819|84907|163011|3515|44819|2|30|16|1|15970|151|68|56|10.60|12.61|4.66|445.20|260.96|593.60|706.16|15.65|0.00|282.24|260.96|276.61|543.20|558.85|-332.64| +2450816|67838|2450838|76229|905828|1027|31426|76229|905828|1027|31426|2|25|18|1|17054|61|69|46|62.29|75.37|64.06|520.26|2946.76|2865.34|3467.02|147.33|0.00|1109.06|2946.76|3094.09|4055.82|4203.15|81.42| +2450816|67838|2450894|76229|905828|1027|31426|76229|905828|1027|31426|2|96|6|1|2126|181|69|12|46.60|67.10|59.04|96.72|708.48|559.20|805.20|14.02|474.68|48.24|233.80|247.82|282.04|296.06|-325.40| +2450816|67838|2450843|76229|905828|1027|31426|76229|905828|1027|31426|2|68|17|2|12038|15|69|98|69.38|140.84|128.16|1242.64|12559.68|6799.24|13802.32|1004.77|0.00|1379.84|12559.68|13564.45|13939.52|14944.29|5760.44| +2450816|67838|2450881|76229|905828|1027|31426|76229|905828|1027|31426|2|100|10|3|5498|257|69|36|31.08|84.84|26.30|2107.44|946.80|1118.88|3054.24|36.92|208.29|855.00|738.51|775.43|1593.51|1630.43|-380.37| +2450816|46228|2450823|93701|1864664|3274|8284|93701|1864664|3274|8284|1|62|17|1|16616|142|70|54|81.46|193.06|108.11|4587.30|5837.94|4398.84|10425.24|350.27|0.00|104.22|5837.94|6188.21|5942.16|6292.43|1439.10| +2450816|46228|2450859|93701|1864664|3274|8284|93701|1864664|3274|8284|1|51|8|2|9010|15|70|95|53.93|159.09|14.31|13754.10|1359.45|5123.35|15113.55|95.16|0.00|3929.20|1359.45|1454.61|5288.65|5383.81|-3763.90| +2450816|46228|2450850|93701|1864664|3274|8284|93701|1864664|3274|8284|1|60|15|4|16178|284|70|5|81.62|137.12|38.39|493.65|191.95|408.10|685.60|3.83|0.00|164.50|191.95|195.78|356.45|360.28|-216.15| +2450816|46228|2450862|93701|1864664|3274|8284|93701|1864664|3274|8284|1|23|2|1|16528|69|70|88|81.72|120.94|113.68|638.88|10003.84|7191.36|10642.72|800.30|0.00|1808.40|10003.84|10804.14|11812.24|12612.54|2812.48| +2450816|40136|2450834|78376|1694169|6315|23708|78376|1694169|6315|23708|4|89|16|2|14770|167|71|29|35.55|99.18|27.77|2070.89|805.33|1030.95|2876.22|48.31|0.00|316.10|805.33|853.64|1121.43|1169.74|-225.62| +2450816|40136|2450847|78376|1694169|6315|23708|78376|1694169|6315|23708|4|60|19|5|6949|46|71|26|42.17|46.80|39.31|194.74|1022.06|1096.42|1216.80|61.32|0.00|182.52|1022.06|1083.38|1204.58|1265.90|-74.36| +2450816|40136|2450899|78376|1694169|6315|23708|78376|1694169|6315|23708|4|65|9|3|8930|21|71|67|50.38|78.08|26.54|3453.18|1778.18|3375.46|5231.36|142.25|0.00|2405.97|1778.18|1920.43|4184.15|4326.40|-1597.28| +2450816|40136|2450857|78376|1694169|6315|23708|78376|1694169|6315|23708|4|5|2|1|7327|93|71|15|43.09|64.63|22.62|630.15|339.30|646.35|969.45|13.57|0.00|232.65|339.30|352.87|571.95|585.52|-307.05| +2450816|40136|2450819|78376|1694169|6315|23708|78376|1694169|6315|23708|4|78|2|2|17485|37|71|25|84.69|93.15|72.65|512.50|1816.25|2117.25|2328.75|163.46|0.00|675.25|1816.25|1979.71|2491.50|2654.96|-301.00| +2450816|40136|2450873|78376|1694169|6315|23708|78376|1694169|6315|23708|4|32|3|5|163|114|71|2|58.89|172.54|120.77|103.54|241.54|117.78|345.08|12.07|0.00|138.02|241.54|253.61|379.56|391.63|123.76| +2450816|40136|2450851|78376|1694169|6315|23708|78376|1694169|6315|23708|4|42|16|2|12235|172|71|39|35.20|44.70|30.39|558.09|1185.21|1372.80|1743.30|82.96|0.00|0.00|1185.21|1268.17|1185.21|1268.17|-187.59| +2450816|54492|2450905|50173|1158199|5778|6204|50173|1158199|5778|6204|2|84|6|2|1069|124|72|47|73.86|96.75|12.57|3956.46|590.79|3471.42|4547.25|41.35|0.00|1500.24|590.79|632.14|2091.03|2132.38|-2880.63| +2450816|54492|2450828|50173|1158199|5778|6204|50173|1158199|5778|6204|2|92|13|1|2702|215|72|15|34.55|68.40|12.31|841.35|184.65|518.25|1026.00|16.61|0.00|92.25|184.65|201.26|276.90|293.51|-333.60| +2450816|54492|2450855|50173|1158199|5778|6204|50173|1158199|5778|6204|2|9|20|2|5834|279|72|47|93.29|259.34|241.18|853.52|11335.46|4384.63|12188.98|566.77|0.00|3047.01|11335.46|11902.23|14382.47|14949.24|6950.83| +2450816|54492|2450823|50173|1158199|5778|6204|50173|1158199|5778|6204|2|25|14|5|7598|104|72|32|73.46|178.50|21.42|5026.56|685.44|2350.72|5712.00|0.00|0.00|2284.80|685.44|685.44|2970.24|2970.24|-1665.28| +2450816|54492|2450892|50173|1158199|5778|6204|50173|1158199|5778|6204|2|76|10|5|8935|65|72|98|98.94|209.75|98.58|10894.66|9660.84|9696.12|20555.50|772.86|0.00|5755.54|9660.84|10433.70|15416.38|16189.24|-35.28| +2450816|54492|2450887|50173|1158199|5778|6204|50173|1158199|5778|6204|2|33|15|2|17977|33|72|19|57.78|101.11|50.55|960.64|960.45|1097.82|1921.09|9.60|0.00|95.95|960.45|970.05|1056.40|1066.00|-137.37| +2450816|54492|2450826|50173|1158199|5778|6204|50173|1158199|5778|6204|2|44|20|2|13183|37|72|43|53.11|103.56|23.81|3429.25|1023.83|2283.73|4453.08|92.14|0.00|1335.58|1023.83|1115.97|2359.41|2451.55|-1259.90| +2450816|54492|2450899|50173|1158199|5778|6204|50173|1158199|5778|6204|2|97|18|1|10658|36|72|13|45.58|110.30|55.15|716.95|716.95|592.54|1433.90|64.52|0.00|401.44|716.95|781.47|1118.39|1182.91|124.41| +2450816|43353|2450879|94024|589162|653|20144|26578|1078773|4664|27402|1|1|7|2|926|129|73|83|35.66|79.16|41.16|3154.00|3416.28|2959.78|6570.28|273.30|0.00|1708.14|3416.28|3689.58|5124.42|5397.72|456.50| +2450816|43353|2450894|94024|589162|653|20144|26578|1078773|4664|27402|1|68|20|4|17590|142|73|59|18.78|39.43|35.88|209.45|2116.92|1108.02|2326.37|105.84|0.00|977.04|2116.92|2222.76|3093.96|3199.80|1008.90| +2450816|43353|2450840|94024|589162|653|20144|26578|1078773|4664|27402|1|70|6|3|17629|195|73|40|52.20|74.64|54.48|806.40|2179.20|2088.00|2985.60|43.58|0.00|895.60|2179.20|2222.78|3074.80|3118.38|91.20| +2450816|43353|2450852|94024|589162|653|20144|26578|1078773|4664|27402|1|56|14|4|11095|127|73|8|47.96|65.22|6.52|469.60|52.16|383.68|521.76|1.56|0.00|125.20|52.16|53.72|177.36|178.92|-331.52| +2450816|43353|2450853|94024|589162|653|20144|26578|1078773|4664|27402|1|25|18|5|14653|90|73|44|73.96|215.96|213.80|95.04|9407.20|3254.24|9502.24|658.50|0.00|3990.80|9407.20|10065.70|13398.00|14056.50|6152.96| +2450816|43353|2450839|94024|589162|653|20144|26578|1078773|4664|27402|1|45|15|2|685|18|73|54|5.84|9.40|6.58|152.28|355.32|315.36|507.60|28.78|35.53|203.04|319.79|348.57|522.83|551.61|4.43| +2450816|71358|2450859|23448|1738817|4139|42661|23448|1738817|4139|42661|2|92|2|5|12266|102|74|90|49.50|94.54|85.08|851.40|7657.20|4455.00|8508.60|0.00|0.00|3402.90|7657.20|7657.20|11060.10|11060.10|3202.20| +2450816|71358|2450880|23448|1738817|4139|42661|23448|1738817|4139|42661|2|59|6|1|3475|131|74|66|73.67|182.70|54.81|8440.74|3617.46|4862.22|12058.20|72.34|0.00|4460.94|3617.46|3689.80|8078.40|8150.74|-1244.76| +2450816|71358|2450831|23448|1738817|4139|42661|23448|1738817|4139|42661|2|40|7|2|13969|289|74|85|74.46|153.38|108.89|3781.65|9255.65|6329.10|13037.30|740.45|0.00|1303.05|9255.65|9996.10|10558.70|11299.15|2926.55| +2450816|71358|2450889|23448|1738817|4139|42661|23448|1738817|4139|42661|2|33|1|1|12469|84|74|38|45.53|66.01|12.54|2031.86|476.52|1730.14|2508.38|28.59|0.00|50.16|476.52|505.11|526.68|555.27|-1253.62| +2450816|71358|2450837|23448|1738817|4139|42661|23448|1738817|4139|42661|2|96|16|3|10108|257|74|93|5.58|15.67|11.43|394.32|1062.99|518.94|1457.31|53.14|0.00|131.13|1062.99|1116.13|1194.12|1247.26|544.05| +2450816|71358|2450870|23448|1738817|4139|42661|23448|1738817|4139|42661|2|43|2|3|16132|295|74|5|67.16|144.39|54.86|447.65|274.30|335.80|721.95|8.22|0.00|353.75|274.30|282.52|628.05|636.27|-61.50| +2450816|71358|2450839|23448|1738817|4139|42661|23448|1738817|4139|42661|2|31|2|2|1724|182|74|89|75.59|135.30|74.41|5419.21|6622.49|6727.51|12041.70|331.12|0.00|3130.13|6622.49|6953.61|9752.62|10083.74|-105.02| +2450816|71358|2450887|23448|1738817|4139|42661|23448|1738817|4139|42661|2|86|15|5|15346|255|74|70|5.84|14.89|14.74|10.50|1031.80|408.80|1042.30|51.59|0.00|395.50|1031.80|1083.39|1427.30|1478.89|623.00| +2450816|71358|2450850|23448|1738817|4139|42661|23448|1738817|4139|42661|2|92|10|4|8258|12|74|68|99.80|100.79|43.33|3907.28|2946.44|6786.40|6853.72|0.00|1001.78|479.40|1944.66|1944.66|2424.06|2424.06|-4841.74| +2450816|71358|2450827|23448|1738817|4139|42661|23448|1738817|4139|42661|2|7|7|3|1760|157|74|95|10.94|15.97|1.11|1411.70|105.45|1039.30|1517.15|1.05|0.00|439.85|105.45|106.50|545.30|546.35|-933.85| +2450816|71358|2450900|23448|1738817|4139|42661|23448|1738817|4139|42661|2|92|10|5|4616|202|74|60|59.60|109.06|57.80|3075.60|3468.00|3576.00|6543.60|104.04|0.00|3075.00|3468.00|3572.04|6543.00|6647.04|-108.00| +2450816|71358|2450884|23448|1738817|4139|42661|23448|1738817|4139|42661|2|48|6|1|5200|101|74|13|85.76|250.41|25.04|2929.81|325.52|1114.88|3255.33|26.04|0.00|781.17|325.52|351.56|1106.69|1132.73|-789.36| +2450816|71358|2450872|23448|1738817|4139|42661|23448|1738817|4139|42661|2|68|5|1|15916|50|74|61|11.53|11.53|7.84|225.09|478.24|703.33|703.33|19.12|0.00|351.36|478.24|497.36|829.60|848.72|-225.09| +2450816|39236|2450830|99543|1096330|7070|45457|99543|1096330|7070|45457|2|76|15|4|8540|147|75|37|19.69|58.08|33.68|902.80|1246.16|728.53|2148.96|49.84|0.00|794.76|1246.16|1296.00|2040.92|2090.76|517.63| +2450816|39236|2450876|99543|1096330|7070|45457|99543|1096330|7070|45457|2|29|19|4|6740|2|75|38|32.40|43.41|19.53|907.44|742.14|1231.20|1649.58|0.00|0.00|478.04|742.14|742.14|1220.18|1220.18|-489.06| +2450816|39236|2450825|99543|1096330|7070|45457|99543|1096330|7070|45457|2|44|1|2|14962|219|75|17|43.11|45.69|12.79|559.30|217.43|732.87|776.73|15.22|0.00|163.03|217.43|232.65|380.46|395.68|-515.44| +2450816|39236|2450855|99543|1096330|7070|45457|99543|1096330|7070|45457|2|31|19|2|598|79|75|38|41.73|91.38|10.05|3090.54|381.90|1585.74|3472.44|30.55|0.00|937.46|381.90|412.45|1319.36|1349.91|-1203.84| +2450816|39236|2450901|99543|1096330|7070|45457|99543|1096330|7070|45457|2|81|14|4|15910|184|75|19|98.72|191.51|178.10|254.79|3383.90|1875.68|3638.69|33.83|0.00|1164.32|3383.90|3417.73|4548.22|4582.05|1508.22| +2450816|39236|2450882|99543|1096330|7070|45457|99543|1096330|7070|45457|2|39|17|2|15850|82|75|35|98.24|218.09|128.67|3129.70|4503.45|3438.40|7633.15|90.06|0.00|1144.85|4503.45|4593.51|5648.30|5738.36|1065.05| +2450816|39236|2450859|99543|1096330|7070|45457|99543|1096330|7070|45457|2|103|16|3|7160|25|75|11|7.34|18.57|18.19|4.18|200.09|80.74|204.27|18.00|0.00|42.79|200.09|218.09|242.88|260.88|119.35| +2450816|39236|2450901|99543|1096330|7070|45457|99543|1096330|7070|45457|2|72|5|4|16342|225|75|51|45.91|58.76|48.77|509.49|2487.27|2341.41|2996.76|0.00|0.00|299.37|2487.27|2487.27|2786.64|2786.64|145.86| +2450816|39236|2450860|99543|1096330|7070|45457|99543|1096330|7070|45457|2|90|14|2|3361|91|75|20|86.89|232.86|114.10|2375.20|2282.00|1737.80|4657.20|182.56|0.00|1071.00|2282.00|2464.56|3353.00|3535.56|544.20| +2450816|39236|2450822|99543|1096330|7070|45457|99543|1096330|7070|45457|2|48|14|5|13642|245|75|13|53.56|70.69|58.67|156.26|762.71|696.28|918.97|53.69|91.52|459.42|671.19|724.88|1130.61|1184.30|-25.09| +2450816|39236|2450870|99543|1096330|7070|45457|99543|1096330|7070|45457|2|18|2|3|1288|118|75|6|16.50|21.78|2.61|115.02|15.66|99.00|130.68|1.09|0.00|28.74|15.66|16.75|44.40|45.49|-83.34| +2450816|39236|2450819|99543|1096330|7070|45457|99543|1096330|7070|45457|2|12|11|4|11342|180|75|21|63.89|145.66|40.78|2202.48|856.38|1341.69|3058.86|34.25|0.00|948.15|856.38|890.63|1804.53|1838.78|-485.31| +2450816|73406|2450843|24798|1875635|5225|29262|24798|1875635|5225|29262|2|37|17|1|4495|277|76|80|11.61|29.60|2.36|2179.20|188.80|928.80|2368.00|0.00|0.00|804.80|188.80|188.80|993.60|993.60|-740.00| +2450816|73406|2450837|24798|1875635|5225|29262|24798|1875635|5225|29262|2|31|8|1|12400|240|76|86|27.66|72.74|39.27|2878.42|3377.22|2378.76|6255.64|33.77|0.00|1188.52|3377.22|3410.99|4565.74|4599.51|998.46| +2450816|73406|2450850|24798|1875635|5225|29262|24798|1875635|5225|29262|2|55|8|2|3482|168|76|50|19.85|26.99|19.97|351.00|998.50|992.50|1349.50|69.89|0.00|53.50|998.50|1068.39|1052.00|1121.89|6.00| +2450816|73406|2450856|24798|1875635|5225|29262|24798|1875635|5225|29262|2|60|16|5|4444|55|76|75|71.57|146.00|11.68|10074.00|876.00|5367.75|10950.00|17.52|0.00|2299.50|876.00|893.52|3175.50|3193.02|-4491.75| +2450816|73406|2450863|24798|1875635|5225|29262|24798|1875635|5225|29262|2|89|4|4|1466|282|76|18|19.36|45.88|3.21|768.06|57.78|348.48|825.84|2.88|0.00|313.74|57.78|60.66|371.52|374.40|-290.70| +2450816|58966|2450824|77674|1087861|448|24635|77674|1087861|448|24635|1|48|12|2|2179|84|77|69|36.04|84.33|33.73|3491.40|2327.37|2486.76|5818.77|116.36|0.00|1919.58|2327.37|2443.73|4246.95|4363.31|-159.39| +2450816|58966|2450851|77674|1087861|448|24635|77674|1087861|448|24635|1|96|17|1|6475|149|77|17|43.83|113.08|6.78|1807.10|115.26|745.11|1922.36|10.37|0.00|422.79|115.26|125.63|538.05|548.42|-629.85| +2450816|58966|2450884|77674|1087861|448|24635|77674|1087861|448|24635|1|89|17|4|3877|275|77|36|9.18|26.98|4.85|796.68|174.60|330.48|971.28|5.23|0.00|242.64|174.60|179.83|417.24|422.47|-155.88| +2450816|58966|2450830|77674|1087861|448|24635|77674|1087861|448|24635|1|80|16|1|10210|236|77|22|21.00|24.57|6.87|389.40|151.14|462.00|540.54|6.04|0.00|59.40|151.14|157.18|210.54|216.58|-310.86| +2450816|58966|2450819|77674|1087861|448|24635|77674|1087861|448|24635|1|39|8|2|5215|244|77|33|34.03|53.42|3.20|1657.26|105.60|1122.99|1762.86|0.00|0.00|334.62|105.60|105.60|440.22|440.22|-1017.39| +2450816|58966|2450840|77674|1087861|448|24635|77674|1087861|448|24635|1|6|7|3|13562|145|77|99|36.64|95.99|69.11|2661.12|6841.89|3627.36|9503.01|547.35|0.00|4370.85|6841.89|7389.24|11212.74|11760.09|3214.53| +2450816|58966|2450830|77674|1087861|448|24635|77674|1087861|448|24635|1|89|13|1|478|54|77|39|71.47|115.06|83.99|1211.73|3275.61|2787.33|4487.34|196.53|0.00|1839.63|3275.61|3472.14|5115.24|5311.77|488.28| +2450816|58966|2450859|77674|1087861|448|24635|77674|1087861|448|24635|1|89|7|3|277|200|77|60|23.57|49.26|21.18|1684.80|1270.80|1414.20|2955.60|38.12|0.00|265.80|1270.80|1308.92|1536.60|1574.72|-143.40| +2450816|58966|2450848|77674|1087861|448|24635|77674|1087861|448|24635|1|39|9|4|6748|151|77|79|5.70|15.67|5.48|805.01|432.92|450.30|1237.93|21.64|0.00|507.18|432.92|454.56|940.10|961.74|-17.38| +2450816|58966|2450893|77674|1087861|448|24635|77674|1087861|448|24635|1|64|19|4|14986|182|77|69|89.88|97.96|3.91|6489.45|269.79|6201.72|6759.24|2.69|0.00|2432.94|269.79|272.48|2702.73|2705.42|-5931.93| +2450816|58966|2450890|77674|1087861|448|24635|77674|1087861|448|24635|1|39|17|2|7772|267|77|62|27.98|74.98|65.98|558.00|4090.76|1734.76|4648.76|0.00|0.00|1812.88|4090.76|4090.76|5903.64|5903.64|2356.00| +2450816|58966|2450855|77674|1087861|448|24635|77674|1087861|448|24635|1|27|19|3|2284|216|77|11|7.10|18.74|9.55|101.09|105.05|78.10|206.14|2.52|73.53|37.07|31.52|34.04|68.59|71.11|-46.58| +2450816|24702|2450905|53783|126523|4510|29650|53783|126523|4510|29650|1|45|18|3|9478|120|78|31|16.90|50.70|24.84|801.66|770.04|523.90|1571.70|30.80|0.00|754.23|770.04|800.84|1524.27|1555.07|246.14| +2450816|24702|2450844|53783|126523|4510|29650|53783|126523|4510|29650|1|11|17|1|17998|142|78|31|64.42|97.27|35.01|1930.06|1085.31|1997.02|3015.37|86.82|0.00|391.84|1085.31|1172.13|1477.15|1563.97|-911.71| +2450816|24702|2450895|53783|126523|4510|29650|53783|126523|4510|29650|1|62|15|1|8210|188|78|15|37.18|84.77|27.97|852.00|419.55|557.70|1271.55|3.86|323.05|165.30|96.50|100.36|261.80|265.66|-461.20| +2450816|24702|2450884|53783|126523|4510|29650|53783|126523|4510|29650|1|1|9|5|4813|34|78|39|9.43|22.82|20.76|80.34|809.64|367.77|889.98|72.86|0.00|364.65|809.64|882.50|1174.29|1247.15|441.87| +2450816|24702|2450824|53783|126523|4510|29650|53783|126523|4510|29650|1|65|13|3|8188|230|78|8|98.53|146.80|39.63|857.36|317.04|788.24|1174.40|9.51|0.00|23.44|317.04|326.55|340.48|349.99|-471.20| +2450816|24702|2450870|53783|126523|4510|29650|53783|126523|4510|29650|1|23|19|4|13708|264|78|27|76.98|149.34|65.70|2258.28|1773.90|2078.46|4032.18|124.17|0.00|685.26|1773.90|1898.07|2459.16|2583.33|-304.56| +2450816|24702|2450847|53783|126523|4510|29650|53783|126523|4510|29650|1|16|5|2|2320|216|78|62|28.54|49.65|40.71|554.28|2524.02|1769.48|3078.30|0.00|252.40|399.90|2271.62|2271.62|2671.52|2671.52|502.14| +2450816|24702|2450896|53783|126523|4510|29650|53783|126523|4510|29650|1|31|14|5|17929|100|78|18|34.45|63.73|21.03|768.60|378.54|620.10|1147.14|22.71|0.00|493.20|378.54|401.25|871.74|894.45|-241.56| +2450816|10322|2450847|85933|1051132|1931|17177|85933|1051132|1931|17177|2|98|19|3|11000|277|79|20|40.78|77.88|36.60|825.60|732.00|815.60|1557.60|14.64|0.00|420.40|732.00|746.64|1152.40|1167.04|-83.60| +2450816|10322|2450852|85933|1051132|1931|17177|85933|1051132|1931|17177|2|50|7|4|5737|128|79|32|92.42|242.14|152.54|2867.20|4881.28|2957.44|7748.48|146.43|0.00|2634.24|4881.28|5027.71|7515.52|7661.95|1923.84| +2450816|10322|2450906|85933|1051132|1931|17177|85933|1051132|1931|17177|2|46|7|5|5966|33|79|52|54.49|65.38|3.92|3195.92|203.84|2833.48|3399.76|2.93|57.07|917.80|146.77|149.70|1064.57|1067.50|-2686.71| +2450816|10322|2450843|85933|1051132|1931|17177|85933|1051132|1931|17177|2|93|1|4|13297|264|79|55|88.09|109.23|49.15|3304.40|2703.25|4844.95|6007.65|216.26|0.00|2282.50|2703.25|2919.51|4985.75|5202.01|-2141.70| +2450816|54649|2450883|31641|1516877|960|44596|31641|1516877|960|44596|4|21|18|4|194|73|80|85|72.96|105.79|14.81|7733.30|1258.85|6201.60|8992.15|88.11|0.00|3326.90|1258.85|1346.96|4585.75|4673.86|-4942.75| +2450816|54649|2450881|31641|1516877|960|44596|31641|1516877|960|44596|4|44|13|3|6506|68|80|44|55.52|153.79|89.19|2842.40|3924.36|2442.88|6766.76|0.00|353.19|67.32|3571.17|3571.17|3638.49|3638.49|1128.29| +2450816|54649|2450831|31641|1516877|960|44596|31641|1516877|960|44596|4|49|4|3|4330|133|80|27|61.09|102.63|45.15|1551.96|1219.05|1649.43|2771.01|0.00|0.00|858.87|1219.05|1219.05|2077.92|2077.92|-430.38| +2450816|54649|2450833|31641|1516877|960|44596|31641|1516877|960|44596|4|72|16|3|1546|34|80|55|9.16|18.32|14.47|211.75|795.85|503.80|1007.60|33.66|421.80|0.00|374.05|407.71|374.05|407.71|-129.75| +2450816|54649|2450846|31641|1516877|960|44596|31641|1516877|960|44596|4|99|3|3|10285|115|80|50|96.00|275.52|173.57|5097.50|8678.50|4800.00|13776.00|433.92|0.00|3168.00|8678.50|9112.42|11846.50|12280.42|3878.50| +2450816|54649|2450896|31641|1516877|960|44596|31641|1516877|960|44596|4|9|14|3|12118|188|80|85|92.59|186.10|33.49|12971.85|2846.65|7870.15|15818.50|28.46|0.00|316.20|2846.65|2875.11|3162.85|3191.31|-5023.50| +2450816|54649|2450864|31641|1516877|960|44596|31641|1516877|960|44596|4|95|10|1|1531|88|80|69|94.00|136.30|106.31|2069.31|7335.39|6486.00|9404.70|220.06|0.00|187.68|7335.39|7555.45|7523.07|7743.13|849.39| +2450816|54649|2450874|31641|1516877|960|44596|31641|1516877|960|44596|4|11|12|2|13058|277|80|76|5.34|9.71|3.39|480.32|257.64|405.84|737.96|5.48|74.71|199.12|182.93|188.41|382.05|387.53|-222.91| +2450816|54649|2450865|31641|1516877|960|44596|31641|1516877|960|44596|4|87|5|3|8038|183|80|37|81.83|139.92|118.93|776.63|4400.41|3027.71|5177.04|88.00|0.00|776.26|4400.41|4488.41|5176.67|5264.67|1372.70| +2450816|54649|2450864|31641|1516877|960|44596|31641|1516877|960|44596|4|27|14|3|14245|64|80|74|53.98|156.00|132.60|1731.60|9812.40|3994.52|11544.00|294.37|0.00|923.52|9812.40|10106.77|10735.92|11030.29|5817.88| +2450816|54649|2450877|31641|1516877|960|44596|31641|1516877|960|44596|4|33|14|4|2852|287|80|55|30.97|47.07|10.82|1993.75|595.10|1703.35|2588.85|5.95|0.00|517.55|595.10|601.05|1112.65|1118.60|-1108.25| +2450816|54649|2450856|31641|1516877|960|44596|31641|1516877|960|44596|4|56|13|1|12058|180|80|2|20.96|50.09|35.56|29.06|71.12|41.92|100.18|0.00|0.00|21.02|71.12|71.12|92.14|92.14|29.20| +2450816|54649|2450825|31641|1516877|960|44596|31641|1516877|960|44596|4|63|1|4|11869|23|80|61|39.70|90.91|55.45|2163.06|3382.45|2421.70|5545.51|0.00|0.00|886.94|3382.45|3382.45|4269.39|4269.39|960.75| +2450816|54649|2450888|31641|1516877|960|44596|31641|1516877|960|44596|4|1|5|3|5734|275|80|21|40.81|47.74|13.84|711.90|290.64|857.01|1002.54|17.43|0.00|360.78|290.64|308.07|651.42|668.85|-566.37| +2450816|61626|2450839|89468|1294986|3704|44022|89468|1294986|3704|44022|1|3|14|5|13633|217|81|73|72.22|83.05|58.13|1819.16|4243.49|5272.06|6062.65|381.91|0.00|1212.53|4243.49|4625.40|5456.02|5837.93|-1028.57| +2450816|61626|2450881|89468|1294986|3704|44022|89468|1294986|3704|44022|1|43|13|1|14953|83|81|60|17.72|39.16|21.92|1034.40|1315.20|1063.20|2349.60|58.00|670.75|704.40|644.45|702.45|1348.85|1406.85|-418.75| +2450816|61626|2450896|89468|1294986|3704|44022|89468|1294986|3704|44022|1|32|3|2|12643|225|81|9|91.72|264.15|171.69|832.14|1545.21|825.48|2377.35|77.26|0.00|451.62|1545.21|1622.47|1996.83|2074.09|719.73| +2450816|61626|2450835|89468|1294986|3704|44022|89468|1294986|3704|44022|1|31|1|3|8674|9|81|30|73.19|185.90|9.29|5298.30|278.70|2195.70|5577.00|8.36|0.00|780.60|278.70|287.06|1059.30|1067.66|-1917.00| +2450816|61626|2450839|89468|1294986|3704|44022|89468|1294986|3704|44022|1|88|19|5|7642|154|81|82|16.75|19.09|11.07|657.64|907.74|1373.50|1565.38|49.83|354.01|218.94|553.73|603.56|772.67|822.50|-819.77| +2450816|61626|2450819|89468|1294986|3704|44022|89468|1294986|3704|44022|1|21|17|1|14311|38|81|37|96.08|283.43|65.18|8075.25|2411.66|3554.96|10486.91|120.58|0.00|2097.16|2411.66|2532.24|4508.82|4629.40|-1143.30| +2450816|61626|2450833|89468|1294986|3704|44022|89468|1294986|3704|44022|1|32|4|5|11161|172|81|75|46.13|127.78|30.66|7284.00|2299.50|3459.75|9583.50|183.96|0.00|191.25|2299.50|2483.46|2490.75|2674.71|-1160.25| +2450816|61626|2450826|89468|1294986|3704|44022|89468|1294986|3704|44022|1|61|7|5|16258|207|81|50|62.17|64.03|4.48|2977.50|224.00|3108.50|3201.50|15.68|0.00|1152.50|224.00|239.68|1376.50|1392.18|-2884.50| +2450816|61626|2450875|89468|1294986|3704|44022|89468|1294986|3704|44022|1|46|14|1|784|43|81|26|12.59|18.88|0.18|486.20|4.68|327.34|490.88|0.42|0.00|39.26|4.68|5.10|43.94|44.36|-322.66| +2450816|72252|2450874|28045|1279899|4553|37207|66749|1878868|6722|32129|4|82|19|1|15811|175|82|22|13.71|24.26|6.79|384.34|149.38|301.62|533.72|5.97|0.00|111.98|149.38|155.35|261.36|267.33|-152.24| +2450816|72252|2450893|28045|1279899|4553|37207|66749|1878868|6722|32129|4|93|5|4|17018|75|82|9|1.97|4.94|4.59|3.15|41.31|17.73|44.46|1.23|0.00|15.48|41.31|42.54|56.79|58.02|23.58| +2450816|72252|2450865|28045|1279899|4553|37207|66749|1878868|6722|32129|4|49|6|5|13315|186|82|34|46.52|126.06|45.38|2743.12|1542.92|1581.68|4286.04|108.00|0.00|1628.60|1542.92|1650.92|3171.52|3279.52|-38.76| +2450816|72252|2450825|28045|1279899|4553|37207|66749|1878868|6722|32129|4|28|3|5|6055|269|82|60|80.81|138.18|122.98|912.00|7378.80|4848.60|8290.80|442.72|0.00|3813.60|7378.80|7821.52|11192.40|11635.12|2530.20| +2450816|72252|2450901|28045|1279899|4553|37207|66749|1878868|6722|32129|4|84|9|4|980|45|82|19|58.25|168.34|104.37|1215.43|1983.03|1106.75|3198.46|79.32|0.00|1119.29|1983.03|2062.35|3102.32|3181.64|876.28| +2450816|72252|2450824|28045|1279899|4553|37207|66749|1878868|6722|32129|4|35|5|5|1399|14|82|99|14.50|31.17|28.05|308.88|2776.95|1435.50|3085.83|55.53|0.00|925.65|2776.95|2832.48|3702.60|3758.13|1341.45| +2450816|72252|2450871|28045|1279899|4553|37207|66749|1878868|6722|32129|4|97|6|4|5611|243|82|51|34.12|52.20|3.13|2502.57|159.63|1740.12|2662.20|3.19|0.00|1331.10|159.63|162.82|1490.73|1493.92|-1580.49| +2450816|72252|2450879|28045|1279899|4553|37207|66749|1878868|6722|32129|4|12|1|3|12415|122|82|56|59.08|101.02|26.26|4186.56|1470.56|3308.48|5657.12|73.52|0.00|2488.64|1470.56|1544.08|3959.20|4032.72|-1837.92| +2450816|72252|2450863||1279899|||66749|1878868|6722|||38|5|3|2252||82||76.85||||5081.44||5293.40|254.07||1587.88|5081.44||6669.32|6923.39|2929.64| +2450816|72252|2450850|28045|1279899|4553|37207|66749|1878868|6722|32129|4|6|1|1|8536|157|82|5|13.20|16.23|4.86|56.85|24.30|66.00|81.15|0.00|0.00|21.90|24.30|24.30|46.20|46.20|-41.70| +2450816|64482|2450900|44942|1441792|4242|13139|44942|1441792|4242|13139|1|51|18|2|4003|292|83|9|64.53|121.31|21.83|895.32|196.47|580.77|1091.79|3.47|80.55|305.64|115.92|119.39|421.56|425.03|-464.85| +2450816|64482|2450877|44942|1441792|4242|13139|44942|1441792|4242|13139|1|28|12|2|11194|283|83|8|29.72|48.14|15.40|261.92|123.20|237.76|385.12|3.69|0.00|76.96|123.20|126.89|200.16|203.85|-114.56| +2450816|64482|2450881|44942|1441792|4242|13139|44942|1441792|4242|13139|1|106|12|4|9694|240|83|98|89.51|209.45|75.40|13136.90|7389.20|8771.98|20526.10|221.67|0.00|1641.50|7389.20|7610.87|9030.70|9252.37|-1382.78| +2450816|64482|2450860|44942|1441792|4242|13139|44942|1441792|4242|13139|1|98|14|5|5461|207|83|93|84.55|181.78|61.80|11158.14|5747.40|7863.15|16905.54|229.89|0.00|3719.07|5747.40|5977.29|9466.47|9696.36|-2115.75| +2450816|64482|2450888|44942|1441792|4242|13139|44942|1441792|4242|13139|1|32|5|2|10453|123|83|11|87.41|93.52|64.52|319.00|709.72|961.51|1028.72|28.38|0.00|123.42|709.72|738.10|833.14|861.52|-251.79| +2450816|64482|2450837|44942|1441792|4242|13139|44942|1441792|4242|13139|1|3|19|1|5719|137|83|81|73.09|108.17|104.92|263.25|8498.52|5920.29|8761.77|424.92|0.00|1051.38|8498.52|8923.44|9549.90|9974.82|2578.23| +2450816|64482|2450901|44942|1441792|4242|13139|44942|1441792|4242|13139|1|97|16|2|16831|64|83|26|32.41|96.90|33.91|1637.74|881.66|842.66|2519.40|61.71|0.00|151.06|881.66|943.37|1032.72|1094.43|39.00| +2450816|64482|2450856|44942|1441792|4242|13139|44942|1441792|4242|13139|1|62|18|3|14632|276|83|63|41.36|54.18|9.75|2799.09|614.25|2605.68|3413.34|1.22|552.82|1365.21|61.43|62.65|1426.64|1427.86|-2544.25| +2450816|64482|2450842|44942|1441792|4242|13139|44942|1441792|4242|13139|1|74|1|5|6686|138|83|73|3.20|9.50|3.42|443.84|249.66|233.60|693.50|17.47|0.00|235.79|249.66|267.13|485.45|502.92|16.06| +2450816|64482|2450821|44942|1441792|4242|13139|44942|1441792|4242|13139|1|57|6|5|12202|278|83|6|86.19|258.57|43.95|1287.72|263.70|517.14|1551.42|15.82|0.00|62.04|263.70|279.52|325.74|341.56|-253.44| +2450816|64482|2450839|44942|1441792|4242|13139|44942|1441792|4242|13139|1|95|6|4|4394|233|83|21|2.93|3.92|2.27|34.65|47.67|61.53|82.32|1.90|0.00|35.28|47.67|49.57|82.95|84.85|-13.86| +2450816|78988|2450853|96524|1053401|7078|13555|96524|1053401|7078|13555|1|3|6|2|1633|289|84|34|61.22|110.19|47.38|2135.54|1610.92|2081.48|3746.46|80.54|0.00|1873.06|1610.92|1691.46|3483.98|3564.52|-470.56| +2450816|78988|2450833|96524|1053401|7078|13555|96524|1053401|7078|13555|1|58|19|1|17995|152|84|48|90.34|208.68|191.98|801.60|9215.04|4336.32|10016.64|737.20|0.00|3305.28|9215.04|9952.24|12520.32|13257.52|4878.72| +2450816|78988|2450897|96524|1053401|7078|13555|96524|1053401|7078|13555|1|79|3|3|1844|13|84|76|57.21|57.78|46.22|878.56|3512.72|4347.96|4391.28|316.14|0.00|482.60|3512.72|3828.86|3995.32|4311.46|-835.24| +2450816|78988|2450820|96524|1053401|7078|13555|96524|1053401|7078|13555|1|63|5|5|8806|25|84|21|71.21|211.49|0.00|4441.29|0.00|1495.41|4441.29|0.00|0.00|1243.41|0.00|0.00|1243.41|1243.41|-1495.41| +2450816|78988|2450886|96524|1053401|7078|13555|96524|1053401|7078|13555|1|22|2|5|16135|78|84|49|68.69|92.04|84.67|361.13|4148.83|3365.81|4509.96|82.97|0.00|1262.73|4148.83|4231.80|5411.56|5494.53|783.02| +2450816|78988|2450904|96524|1053401|7078|13555|96524|1053401|7078|13555|1|22|3|2|3152|225|84|85|43.17|101.01|58.58|3606.55|4979.30|3669.45|8585.85|0.00|0.00|3090.60|4979.30|4979.30|8069.90|8069.90|1309.85| +2450816|33879|||861546|1684||||1684|||59|5||6032|65|85|95|64.16|116.12|||7169.65|6095.20|11031.40|0.00|0.00||7169.65||9927.50|9927.50|1074.45| +2450816|33879|2450828|9408|861546|1684|18222|9408|861546|1684|18222|1|97|19|3|7066|16|85|37|65.56|70.14|40.68|1090.02|1505.16|2425.72|2595.18|90.30|0.00|882.08|1505.16|1595.46|2387.24|2477.54|-920.56| +2450816|33879|2450856|9408|861546|1684|18222|9408|861546|1684|18222|1|83|15|2|7693|83|85|29|20.69|60.00|20.40|1148.40|591.60|600.01|1740.00|12.24|183.39|713.40|408.21|420.45|1121.61|1133.85|-191.80| +2450816|33879|2450837|9408|861546|1684|18222|9408|861546|1684|18222|1|19|3|5|6902|165|85|12|80.49|144.07|2.88|1694.28|34.56|965.88|1728.84|0.69|0.00|380.28|34.56|35.25|414.84|415.53|-931.32| +2450816|33879|2450821|9408|861546|1684|18222|9408|861546|1684|18222|1|21|19|1|9496|285|85|55|99.82|293.47|8.80|15656.85|484.00|5490.10|16140.85|4.84|0.00|2743.40|484.00|488.84|3227.40|3232.24|-5006.10| +2450816|33879|2450860|9408|861546|1684|18222|9408|861546|1684|18222|1|37|9|2|6100|255|85|33|47.74|83.06|54.81|932.25|1808.73|1575.42|2740.98|108.52|0.00|383.46|1808.73|1917.25|2192.19|2300.71|233.31| +2450816|33879|2450846|9408|861546|1684|18222|9408|861546|1684|18222|1|95|15|2|10687|121|85|81|42.67|58.88|16.48|3434.40|1334.88|3456.27|4769.28|106.79|0.00|1764.18|1334.88|1441.67|3099.06|3205.85|-2121.39| +2450816|46528|2450834|82224|47350|5629|3508|82224|47350|5629|3508|4|1|6|2|2329|15|86|52|81.64|199.20|87.64|5801.12|4557.28|4245.28|10358.40|364.58|0.00|3935.88|4557.28|4921.86|8493.16|8857.74|312.00| +2450816|46528|2450822|82224|47350|5629|3508|82224|47350|5629|3508|4|84|3|5|14053|137|86|9|22.41|36.08|16.23|178.65|146.07|201.69|324.72|4.38|0.00|64.89|146.07|150.45|210.96|215.34|-55.62| +2450816|46528|2450823|82224|47350|5629|3508|82224|47350|5629|3508|4|55|8|2|14752|153|86|92|55.48|150.90|63.37|8052.76|5830.04|5104.16|13882.80|62.96|4255.92|2637.64|1574.12|1637.08|4211.76|4274.72|-3530.04| +2450816|46528|2450889|82224|47350|5629|3508|82224|47350|5629|3508|4|104|15|2|16988|19|86|89|76.38|203.93|142.75|5445.02|12704.75|6797.82|18149.77|651.75|5463.04|7622.85|7241.71|7893.46|14864.56|15516.31|443.89| +2450816|46528|2450850|82224|47350|5629|3508|82224|47350|5629|3508|4|25|13|5|10946|195|86|83|77.23|132.06|73.95|4823.13|6137.85|6410.09|10960.98|245.51|0.00|5150.98|6137.85|6383.36|11288.83|11534.34|-272.24| +2450816|46528|2450827|82224|47350|5629|3508|82224|47350|5629|3508|4|16|16|2|11173|146|86|36|74.16|216.54|155.90|2183.04|5612.40|2669.76|7795.44|224.49|0.00|77.76|5612.40|5836.89|5690.16|5914.65|2942.64| +2450816|46528|2450904|82224|47350|5629|3508|82224|47350|5629|3508|4|92|20|3|7354|102|86|82|86.20|258.60|193.95|5301.30|15903.90|7068.40|21205.20|7.95|15108.70|0.00|795.20|803.15|795.20|803.15|-6273.20| +2450816|46528|2450887|82224|47350|5629|3508|82224|47350|5629|3508|4|103|6|2|15050|155|86|31|84.83|206.13|158.72|1469.71|4920.32|2629.73|6390.03|49.20|0.00|1533.57|4920.32|4969.52|6453.89|6503.09|2290.59| +2450816|46528|2450904|82224|47350|5629|3508|82224|47350|5629|3508|4|54|15|1|4688|191|86|34|26.64|30.36|21.55|299.54|732.70|905.76|1032.24|21.98|0.00|268.26|732.70|754.68|1000.96|1022.94|-173.06| +2450816|46528|2450819|82224|47350|5629|3508|82224|47350|5629|3508|4|89|12|3|13597|90|86|40|26.62|59.89|59.29|24.00|2371.60|1064.80|2395.60|7.82|1588.97|958.00|782.63|790.45|1740.63|1748.45|-282.17| +2450816|46528|2450882|82224|47350|5629|3508|82224|47350|5629|3508|4|24|20|1|4513|158|86|59|99.17|194.37|114.67|4702.30|6765.53|5851.03|11467.83|608.89|0.00|4701.71|6765.53|7374.42|11467.24|12076.13|914.50| +2450816|46528|2450901|82224|47350|5629|3508|82224|47350|5629|3508|4|106|17|3|16420|42|86|50|74.92|83.16|60.70|1123.00|3035.00|3746.00|4158.00|30.35|0.00|374.00|3035.00|3065.35|3409.00|3439.35|-711.00| +2450816|46528|2450882|82224|47350|5629|3508|82224|47350|5629|3508|4|24|20|1|8656|158|86|26|55.45|141.39|80.59|1580.80|2095.34|1441.70|3676.14|75.43|209.53|404.30|1885.81|1961.24|2290.11|2365.54|444.11| +2450816|43948|2450887|26969|870772|1775|2778|26969|870772|1775|2778|4|68|8|5|8917|187|87|14|52.15|153.32|111.92|579.60|1566.88|730.10|2146.48|62.67|0.00|407.82|1566.88|1629.55|1974.70|2037.37|836.78| +2450816|43948|2450869|26969|870772|1775|2778|26969|870772|1775|2778|4|31|9|1|6524|114|87|9|37.74|61.51|28.90|293.49|260.10|339.66|553.59|7.80|0.00|265.68|260.10|267.90|525.78|533.58|-79.56| +2450816|43948|2450826|26969|870772|1775|2778|26969|870772|1775|2778|4|74|15|5|11734|79|87|77|27.35|68.37|45.12|1790.25|3474.24|2105.95|5264.49|138.96|0.00|631.40|3474.24|3613.20|4105.64|4244.60|1368.29| +2450816|43948|2450844|26969|870772|1775|2778|26969|870772|1775|2778|4|38|5|5|14194|114|87|69|67.52|152.59|82.39|4843.80|5684.91|4658.88|10528.71|2.84|5628.06|104.88|56.85|59.69|161.73|164.57|-4602.03| +2450816|43948|2450904|26969|870772|1775|2778|26969|870772|1775|2778|4|27|12|2|1201|200|87|16|82.38|209.24|96.25|1807.84|1540.00|1318.08|3347.84|30.80|0.00|1004.32|1540.00|1570.80|2544.32|2575.12|221.92| +2450816|43948|2450879|26969|870772|1775|2778|26969|870772|1775|2778|4|11|18|1|5240|162|87|55|89.58|215.88|105.78|6055.50|5817.90|4926.90|11873.40|116.35|0.00|2849.55|5817.90|5934.25|8667.45|8783.80|891.00| +2450816|43948|2450901|26969|870772|1775|2778|26969|870772|1775|2778|4|43|6|1|3952|244|87|2|6.91|14.23|1.28|25.90|2.56|13.82|28.46|0.10|0.00|13.94|2.56|2.66|16.50|16.60|-11.26| +2450816|43948|2450847|26969|870772|1775|2778|26969|870772|1775|2778|4|81|13|3|9868|216|87|58|40.86|64.55|41.95|1310.80|2433.10|2369.88|3743.90|72.99|0.00|748.78|2433.10|2506.09|3181.88|3254.87|63.22| +2450816|69936|2450884|86800|965403|1159|32115|86800|965403|1159|32115|1|56|1|3|13654|26|88|23|46.39|46.85|10.77|829.84|247.71|1066.97|1077.55|14.86|0.00|420.21|247.71|262.57|667.92|682.78|-819.26| +2450816|69936|2450905|86800|965403|1159|32115|86800|965403|1159|32115|1|5|13|4|2998|200|88|45|11.40|28.27|5.37|1030.50|241.65|513.00|1272.15|0.00|0.00|241.65|241.65|241.65|483.30|483.30|-271.35| +2450816|69936|2450852|86800|965403|1159|32115|86800|965403|1159|32115|1|99|3|2|11356|232|88|69|24.92|47.59|36.64|755.55|2528.16|1719.48|3283.71|176.97|0.00|360.87|2528.16|2705.13|2889.03|3066.00|808.68| +2450816|69936|2450897|86800|965403|1159|32115|86800|965403|1159|32115|1|86|17|1|17725|139|88|53|21.72|50.39|22.17|1495.66|1175.01|1151.16|2670.67|105.75|0.00|1201.51|1175.01|1280.76|2376.52|2482.27|23.85| +2450816|69936|2450832|86800|965403|1159|32115|86800|965403|1159|32115|1|85|9|5|7276|95|88|87|55.63|121.27|118.84|211.41|10339.08|4839.81|10550.49|827.12|0.00|1793.07|10339.08|11166.20|12132.15|12959.27|5499.27| +2450816|81222|2450895|45977|1492959|5894|36829|45977|1492959|5894|36829|2|12|15|4|4814|195|89|77|4.52|13.51|12.83|52.36|987.91|348.04|1040.27|29.63|0.00|405.02|987.91|1017.54|1392.93|1422.56|639.87| +2450816|81222|2450869|45977|1492959|5894|36829|45977|1492959|5894|36829|2|85|12|3|13252|194|89|35|52.31|103.05|48.43|1911.70|1695.05|1830.85|3606.75|135.60|0.00|648.90|1695.05|1830.65|2343.95|2479.55|-135.80| +2450816|81222|2450832|45977|1492959|5894|36829|45977|1492959|5894|36829|2|9|8|3|5041|65|89|60|80.35|116.50|43.10|4404.00|2586.00|4821.00|6990.00|181.02|0.00|1467.60|2586.00|2767.02|4053.60|4234.62|-2235.00| +2450816|81222|2450893|45977|1492959|5894|36829|45977|1492959|5894|36829|2|32|18|5|1792|288|89|34|40.89|77.69|59.04|634.10|2007.36|1390.26|2641.46|140.51|0.00|1082.90|2007.36|2147.87|3090.26|3230.77|617.10| +2450816|81222|2450895|45977|1492959|5894|36829|45977|1492959|5894|36829|2|33|14|2|2372|298|89|24|26.48|47.92|16.29|759.12|390.96|635.52|1150.08|3.90|0.00|540.48|390.96|394.86|931.44|935.34|-244.56| +2450816|81222|2450902|45977|1492959|5894|36829|45977|1492959|5894|36829|2|67|16|5|8354|46|89|38|29.14|53.61|50.92|102.22|1934.96|1107.32|2037.18|0.00|0.00|81.32|1934.96|1934.96|2016.28|2016.28|827.64| +2450816|81222|2450904|45977|1492959|5894|36829|45977|1492959|5894|36829|2|58|3|5|16532|46|89|76|42.72|66.21|9.26|4328.20|703.76|3246.72|5031.96|14.07|0.00|2012.48|703.76|717.83|2716.24|2730.31|-2542.96| +2450816|81222|2450885|45977|1492959|5894|36829|45977|1492959|5894|36829|2|98|18|2|5318|182|89|53|86.44|150.40|117.31|1753.77|6217.43|4581.32|7971.20|248.69|0.00|1514.21|6217.43|6466.12|7731.64|7980.33|1636.11| +2450816|36623|2450838|71252|873708|1764|13020|71252|873708|1764|13020|1|50|15|1|11830|297|90|28|78.29|108.82|7.61|2833.88|213.08|2192.12|3046.96|8.52|0.00|1157.80|213.08|221.60|1370.88|1379.40|-1979.04| +2450816|36623|2450833|71252|873708|1764|13020|71252|873708|1764|13020|1|30|17|2|17359|210|90|9|6.46|18.73|8.80|89.37|79.20|58.14|168.57|5.54|0.00|16.83|79.20|84.74|96.03|101.57|21.06| +2450816|36623|2450841|71252|873708|1764|13020|71252|873708|1764|13020|1|23|17|2|15748|67|90|41|90.78|177.02|37.17|5733.85|1523.97|3721.98|7257.82|45.71|0.00|3047.94|1523.97|1569.68|4571.91|4617.62|-2198.01| +2450816|36623|2450828|71252|873708|1764|13020|71252|873708|1764|13020|1|10|13|1|1892|199|90|12|76.18|134.07|26.81|1287.12|321.72|914.16|1608.84|6.43|0.00|498.72|321.72|328.15|820.44|826.87|-592.44| +2450816|36623|2450855|71252|873708|1764|13020|71252|873708|1764|13020|1|45|9|2|14557|287|90|65|53.76|115.04|19.55|6206.85|1270.75|3494.40|7477.60|25.41|0.00|2392.65|1270.75|1296.16|3663.40|3688.81|-2223.65| +2450816|36623|2450884|71252|873708|1764|13020|71252|873708|1764|13020|1|38|14|4|17965|183|90|24|11.73|24.39|5.12|462.48|122.88|281.52|585.36|1.22|0.00|64.32|122.88|124.10|187.20|188.42|-158.64| +2450816|36623|2450838|71252|873708|1764|13020|71252|873708|1764|13020|1|24|2|1|12922|207|90|40|71.26|78.38|69.75|345.20|2790.00|2850.40|3135.20|195.30|0.00|1254.00|2790.00|2985.30|4044.00|4239.30|-60.40| +2450816|36623|2450836|71252|873708|1764|13020|71252|873708|1764|13020|1|82|2|4|5773|138|90|92|31.05|89.73|34.09|5118.88|3136.28|2856.60|8255.16|219.53|0.00|164.68|3136.28|3355.81|3300.96|3520.49|279.68| +2450816|36623|2450904|71252|873708|1764|13020|71252|873708|1764|13020|1|77|9|4|7724|71|90|53|68.99|87.61|84.98|139.39|4503.94|3656.47|4643.33|63.05|3603.15|371.00|900.79|963.84|1271.79|1334.84|-2755.68| +2450816|36623|2450857|71252|873708|1764|13020|71252|873708|1764|13020|1|22|4|3|13360|111|90|36|13.50|34.83|4.17|1103.76|150.12|486.00|1253.88|6.00|0.00|74.88|150.12|156.12|225.00|231.00|-335.88| +2450816|36623|2450867|71252|873708|1764|13020|71252|873708|1764|13020|1|106|12|2|10570|150|90|85|72.60|161.17|156.33|411.40|13288.05|6171.00|13699.45|398.64|3322.01|1643.90|9966.04|10364.68|11609.94|12008.58|3795.04| +2450816|36623|2450904|71252|873708|1764|13020|71252|873708|1764|13020|1|85|15|5|15932|5|90|88|43.82|74.05|62.20|1042.80|5473.60|3856.16|6516.40|109.47|0.00|1107.04|5473.60|5583.07|6580.64|6690.11|1617.44| +2450816|36623|2450887|71252|873708|1764|13020|71252|873708|1764|13020|1|43|2|2|3799|52|90|86|49.45|55.87|37.43|1585.84|3218.98|4252.70|4804.82|193.13|0.00|191.78|3218.98|3412.11|3410.76|3603.89|-1033.72| +2450816|42701|2450905|32188|266653|3071|31136|32188|266653|3071|31136|2|64|18|3|572|163|91|83|9.21|20.53|3.69|1397.72|306.27|764.43|1703.99|0.00|0.00|34.03|306.27|306.27|340.30|340.30|-458.16| +2450816|42701|2450819|32188|266653|3071|31136|32188|266653|3071|31136|2|100|16|4|853|20|91|83|14.07|32.22|25.45|561.91|2112.35|1167.81|2674.26|84.49|1056.17|989.36|1056.18|1140.67|2045.54|2130.03|-111.63| +2450816|42701|2450851|32188|266653|3071|31136|32188|266653|3071|31136|2|35|13|3|10508|3|91|12|60.80|105.79|47.60|698.28|571.20|729.60|1269.48|51.40|0.00|444.24|571.20|622.60|1015.44|1066.84|-158.40| +2450816|42701|2450898|32188|266653|3071|31136|32188|266653|3071|31136|2|76|16|5|16564|13|91|99|98.91|160.23|120.17|3965.94|11896.83|9792.09|15862.77|217.71|4639.76|2855.16|7257.07|7474.78|10112.23|10329.94|-2535.02| +2450816|42701|2450835|32188|266653|3071|31136|32188|266653|3071|31136|2|32|6|2|14137|94|91|31|96.48|239.27|66.99|5340.68|2076.69|2990.88|7417.37|186.90|0.00|2150.78|2076.69|2263.59|4227.47|4414.37|-914.19| +2450816|42701|2450834|32188|266653|3071|31136|32188|266653|3071|31136|2|61|15|5|11104|121|91|64|4.65|7.02|0.35|426.88|22.40|297.60|449.28|1.79|0.00|165.76|22.40|24.19|188.16|189.95|-275.20| +2450816|42701|2450842|32188|266653|3071|31136|32188|266653|3071|31136|2|57|15|2|15508|241|91|21|63.21|96.07|29.78|1392.09|625.38|1327.41|2017.47|25.01|0.00|120.96|625.38|650.39|746.34|771.35|-702.03| +2450816|42701|2450872|32188|266653|3071|31136|32188|266653|3071|31136|2|63|7|4|2944|246|91|46|68.15|196.95|82.71|5255.04|3804.66|3134.90|9059.70|33.86|418.51|543.26|3386.15|3420.01|3929.41|3963.27|251.25| +2450816|42701|2450861|32188|266653|3071|31136|32188|266653|3071|31136|2|38|8|5|8818|213|91|66|29.94|62.27|37.36|1644.06|2465.76|1976.04|4109.82|197.26|0.00|1397.22|2465.76|2663.02|3862.98|4060.24|489.72| +2450816|42701|2450870|32188|266653|3071|31136|32188|266653|3071|31136|2|40|10|1|6175|165|91|80|8.90|20.38|9.37|880.80|749.60|712.00|1630.40|22.48|0.00|684.00|749.60|772.08|1433.60|1456.08|37.60| +2450816|42701|2450893|32188|266653|3071|31136|32188|266653|3071|31136|2|36|2|2|15908|254|91|31|30.45|66.68|48.67|558.31|1508.77|943.95|2067.08|60.35|0.00|930.00|1508.77|1569.12|2438.77|2499.12|564.82| +2450816|73755|2450884|95024|552511|1551|29838|95024|552511|1551|29838|4|84|16|2|8245|201|92|83|78.43|213.32|83.19|10800.79|6904.77|6509.69|17705.56|621.42|0.00|6019.16|6904.77|7526.19|12923.93|13545.35|395.08| +2450816|73755|2450833|95024|552511|1551|29838|95024|552511|1551|29838|4|41|19|1|14576|237|92|31|8.25|24.75|14.60|314.65|452.60|255.75|767.25|4.52|0.00|122.76|452.60|457.12|575.36|579.88|196.85| +2450816|73755|2450848|95024|552511|1551|29838|95024|552511|1551|29838|4|94|14|1|15391|51|92|83|95.89|180.27|165.84|1197.69|13764.72|7958.87|14962.41|550.58|0.00|2692.52|13764.72|14315.30|16457.24|17007.82|5805.85| +2450816|73755|2450848|95024|552511|1551|29838|95024|552511|1551|29838|4|39|14|1|5845|227|92|19|15.75|34.96|27.26|146.30|517.94|299.25|664.24|20.71|0.00|59.66|517.94|538.65|577.60|598.31|218.69| +2450816|73755|2450818|95024|552511|1551|29838|95024|552511|1551|29838|4|81|11|2|17521|124|92|6|48.23|90.19|53.21|221.88|319.26|289.38|541.14|15.96|0.00|194.76|319.26|335.22|514.02|529.98|29.88| +2450816|73755|2450893|95024|552511|1551|29838|95024|552511|1551|29838|4|92|17|3|10981|30|92|75|33.28|99.84|68.88|2322.00|5166.00|2496.00|7488.00|309.96|0.00|74.25|5166.00|5475.96|5240.25|5550.21|2670.00| +2450816|73755|2450853|95024|552511|1551|29838|95024|552511|1551|29838|4|75|1|3|15796|43|92|74|75.63|91.51|79.61|880.60|5891.14|5596.62|6771.74|412.37|0.00|1421.54|5891.14|6303.51|7312.68|7725.05|294.52| +2450816|57653|2450845|57204|1098811|5665|18872|57204|1098811|5665|18872|4|9|11|4|9398|95|93|90|23.01|63.27|56.31|626.40|5067.90|2070.90|5694.30|304.07|0.00|227.70|5067.90|5371.97|5295.60|5599.67|2997.00| +2450816|57653|2450894|57204|1098811|5665|18872|57204|1098811|5665|18872|4|28|7|5|10132|26|93|82|81.54|155.74|104.34|4214.80|8555.88|6686.28|12770.68|85.55|0.00|6129.50|8555.88|8641.43|14685.38|14770.93|1869.60| +2450816|57653|2450823|57204|1098811|5665|18872|57204|1098811|5665|18872|4|56|15|4|9646|78|93|79|53.50|61.52|8.61|4179.89|680.19|4226.50|4860.08|6.80|340.09|1068.87|340.10|346.90|1408.97|1415.77|-3886.40| +2450816|57653|2450827|57204|1098811|5665|18872|57204|1098811|5665|18872|4|83|4|3|13448|43|93|94|80.72|85.56|7.70|7318.84|723.80|7587.68|8042.64|21.71|0.00|2090.56|723.80|745.51|2814.36|2836.07|-6863.88| +2450816|57653|2450904|57204|1098811|5665|18872|57204|1098811|5665|18872|4|94|15|3|13972|163|93|16|81.50|187.45|11.24|2819.36|179.84|1304.00|2999.20|7.19|0.00|479.84|179.84|187.03|659.68|666.87|-1124.16| +2450816|57653|2450878|57204|1098811|5665|18872|57204|1098811|5665|18872|4|77|12|4|1544|81|93|18|59.35|81.90|58.96|412.92|1061.28|1068.30|1474.20|63.67|0.00|442.26|1061.28|1124.95|1503.54|1567.21|-7.02| +2450816|57653|2450892|57204|1098811|5665|18872|57204|1098811|5665|18872|4|65|6|2|1280|149|93|48|17.96|49.03|39.71|447.36|1906.08|862.08|2353.44|95.30|0.00|23.52|1906.08|2001.38|1929.60|2024.90|1044.00| +2450816|33151|2450868|96466|1840163|3060|29157|96466|1840163|3060|29157|4|18|10|5|12812|242|94|78|82.22|110.17|103.55|516.36|8076.90|6413.16|8593.26|484.61|0.00|1374.36|8076.90|8561.51|9451.26|9935.87|1663.74| +2450816|33151|2450895|96466|1840163|3060|29157|96466|1840163|3060|29157|4|90|19|2|12061|291|94|39|5.85|9.65|3.86|225.81|150.54|228.15|376.35|4.87|96.34|127.92|54.20|59.07|182.12|186.99|-173.95| +2450816|33151|2450863|96466|1840163|3060|29157|96466|1840163|3060|29157|4|28|14|5|9262|237|94|98|47.59|81.37|6.50|7337.26|637.00|4663.82|7974.26|0.00|528.71|796.74|108.29|108.29|905.03|905.03|-4555.53| +2450816|33151|2450832|96466|1840163|3060|29157|96466|1840163|3060|29157|4|103|14|1|9344|63|94|22|34.60|36.33|7.99|623.48|175.78|761.20|799.26|3.02|24.60|79.86|151.18|154.20|231.04|234.06|-610.02| +2450816|33151|2450886|96466|1840163|3060|29157|96466|1840163|3060|29157|4|39|13|4|1|261|94|45|41.31|85.92|5.15|3634.65|231.75|1858.95|3866.40|4.63|0.00|1353.15|231.75|236.38|1584.90|1589.53|-1627.20| +2450816|33151|2450860|96466|1840163|3060|29157|96466|1840163|3060|29157|4|2|20|1|1316|120|94|51|2.51|3.51|1.47|104.04|74.97|128.01|179.01|2.24|0.00|69.36|74.97|77.21|144.33|146.57|-53.04| +2450816|33151|2450860|96466|1840163|3060|29157|96466|1840163|3060|29157|4|5|3|2|13960|271|94|71|62.37|118.50|46.21|5132.59|3280.91|4428.27|8413.50|0.00|0.00|83.78|3280.91|3280.91|3364.69|3364.69|-1147.36| +2450816|71270|2450895|80093|1871653|1166|23126|80093|1871653|1166|23126|1|17|16|2|15320|14|95|36|20.83|58.94|4.12|1973.52|148.32|749.88|2121.84|4.44|0.00|1060.92|148.32|152.76|1209.24|1213.68|-601.56| +2450816|71270|2450890|80093|1871653|1166|23126|80093|1871653|1166|23126|1|41|5|3|17437|124|95|2|88.06|105.67|22.19|166.96|44.38|176.12|211.34|2.66|0.00|0.00|44.38|47.04|44.38|47.04|-131.74| +2450816|71270|2450880|80093|1871653|1166|23126|80093|1871653|1166|23126|1|13|14|3|16708|159|95|83|65.54|74.06|58.50|1291.48|4855.50|5439.82|6146.98|48.55|0.00|368.52|4855.50|4904.05|5224.02|5272.57|-584.32| +2450816|71270|2450866|80093|1871653|1166|23126|80093|1871653|1166|23126|1|18|2|4|3988|190|95|11|49.99|102.47|27.66|822.91|304.26|549.89|1127.17|0.00|0.00|202.84|304.26|304.26|507.10|507.10|-245.63| +2450816|71270|2450858|80093|1871653|1166|23126|80093|1871653|1166|23126|1|41|1|1|15151|206|95|83|47.99|137.73|41.31|8002.86|3428.73|3983.17|11431.59|0.00|0.00|685.58|3428.73|3428.73|4114.31|4114.31|-554.44| +2450816|71270|2450831|80093|1871653|1166|23126|80093|1871653|1166|23126|1|14|4|3|4930|90|95|72|71.67|146.92|19.09|9203.76|1374.48|5160.24|10578.24|0.00|0.00|2115.36|1374.48|1374.48|3489.84|3489.84|-3785.76| +2450816|17125|2450851|45224|69216|3161|47553|45224|69216|3161|47553|2|24|6|5|12031|102|96|71|80.77|90.46|12.66|5523.80|898.86|5734.67|6422.66|53.93|0.00|2761.19|898.86|952.79|3660.05|3713.98|-4835.81| +2450816|17125|2450893|45224|69216|3161|47553|45224|69216|3161|47553|2|103|20|2|7868|7|96|56|63.66|134.32|80.59|3008.88|4513.04|3564.96|7521.92|315.91|0.00|2181.20|4513.04|4828.95|6694.24|7010.15|948.08| +2450816|17125|2450890|45224|69216|3161|47553|45224|69216|3161|47553|2|17|3|1|10819|190|96|92|46.00|119.14|19.06|9207.36|1753.52|4232.00|10960.88|87.67|0.00|3945.88|1753.52|1841.19|5699.40|5787.07|-2478.48| +2450816|17125|2450875|45224|69216|3161|47553|45224|69216|3161|47553|2|22|2|4|17608|283|96|66|33.55|76.82|33.03|2890.14|2179.98|2214.30|5070.12|19.18|1940.18|405.24|239.80|258.98|645.04|664.22|-1974.50| +2450816|17125|2450823|45224|69216|3161|47553|45224|69216|3161|47553|2|76|17|2|1484|174|96|23|89.70|183.88|73.55|2537.59|1691.65|2063.10|4229.24|0.00|0.00|0.00|1691.65|1691.65|1691.65|1691.65|-371.45| +2450816|80799|2450872|69951|1504890|3382|48721|69951|1504890|3382|48721|4|15|3|5|10444|126|97|4|53.44|63.59|43.24|81.40|172.96|213.76|254.36|8.87|74.37|106.80|98.59|107.46|205.39|214.26|-115.17| +2450816|80799|2450828|69951|1504890|3382|48721|69951|1504890|3382|48721|4|19|18|3|7124|205|97|89|54.93|103.26|55.76|4227.50|4962.64|4888.77|9190.14|446.63|0.00|0.00|4962.64|5409.27|4962.64|5409.27|73.87| +2450816|80799|2450884|69951|1504890|3382|48721|69951|1504890|3382|48721|4|75|11|5|4570|203|97|67|46.03|134.86|41.80|6235.02|2800.60|3084.01|9035.62|56.01|0.00|2077.67|2800.60|2856.61|4878.27|4934.28|-283.41| +2450816|80799|2450897|69951|1504890|3382|48721|69951|1504890|3382|48721|4|107|14|4|8476|91|97|65|69.67|197.86|0.00|12860.90|0.00|4528.55|12860.90|0.00|0.00|4243.85|0.00|0.00|4243.85|4243.85|-4528.55| +2450816|80799|2450849|69951|1504890|3382|48721|69951|1504890|3382|48721|4|11|12|3|535|170|97|6|32.61|73.37|39.61|202.56|237.66|195.66|440.22|0.00|0.00|123.24|237.66|237.66|360.90|360.90|42.00| +2450816|80799||||3382||69951|1504890|3382||||13|1|3139||97||65.17|||||4952.92||46.91|459.19|531.24|781.89||1313.13|1360.04|-4171.03| +2450816|71437|2450866|89392|192830|4268|30906|89392|192830|4268|30906|4|10|12|1|15802|156|98|28|26.02|26.54|24.94|44.80|698.32|728.56|743.12|19.55|307.26|163.24|391.06|410.61|554.30|573.85|-337.50| +2450816|71437|2450894|89392|192830|4268|30906|89392|192830|4268|30906|4|30|17|5|4831|171|98|63|16.12|41.42|12.01|1852.83|756.63|1015.56|2609.46|30.26|0.00|990.99|756.63|786.89|1747.62|1777.88|-258.93| +2450816|71437|2450851|89392|192830|4268|30906|89392|192830|4268|30906|4|67|2|2|16027|14|98|3|13.10|19.12|15.86|9.78|47.58|39.30|57.36|3.33|0.00|18.90|47.58|50.91|66.48|69.81|8.28| +2450816|71437|2450852|89392|192830|4268|30906|89392|192830|4268|30906|4|88|17|3|7892|293|98|39|89.92|237.38|218.38|741.00|8516.82|3506.88|9257.82|766.51|0.00|184.86|8516.82|9283.33|8701.68|9468.19|5009.94| +2450816|56423|2450893|38659|400535|811|21887|38659|400535|811|21887|2|25|14|2|12598|72|99|12|19.42|57.48|31.61|310.44|379.32|233.04|689.76|18.96|0.00|89.64|379.32|398.28|468.96|487.92|146.28| +2450816|56423|2450885|38659|400535|811|21887|38659|400535|811|21887|2|86|9|5|17914|97|99|17|95.13|195.01|23.40|2917.37|397.80|1617.21|3315.17|15.75|135.25|1558.05|262.55|278.30|1820.60|1836.35|-1354.66| +2450816|56423|2450840|38659|400535|811|21887|38659|400535|811|21887|2|48|2|4|13936|133|99|5|1.46|3.41|1.05|11.80|5.25|7.30|17.05|0.42|0.00|5.25|5.25|5.67|10.50|10.92|-2.05| +2450816|56423|2450867|38659|400535|811|21887|38659|400535|811|21887|2|54|9|2|15061|68|99|56|43.15|87.59|47.29|2256.80|2648.24|2416.40|4905.04|238.34|0.00|48.72|2648.24|2886.58|2696.96|2935.30|231.84| +2450816|56423|2450905|38659|400535|811|21887|38659|400535|811|21887|2|89|19|5|12518|137|99|36|4.68|9.40|6.11|118.44|219.96|168.48|338.40|6.59|0.00|60.84|219.96|226.55|280.80|287.39|51.48| +2450816|56423|2450848|38659|400535|811|21887|38659|400535|811|21887|2|28|8|4|8380|101|99|55|30.03|48.04|19.21|1585.65|1056.55|1651.65|2642.20|52.82|0.00|1135.75|1056.55|1109.37|2192.30|2245.12|-595.10| +2450816|56423|2450860|38659|400535|811|21887|38659|400535|811|21887|2|58|12|4|11488|95|99|42|68.20|116.62|66.47|2106.30|2791.74|2864.40|4898.04|83.75|0.00|538.44|2791.74|2875.49|3330.18|3413.93|-72.66| +2450816|56423|2450859|38659|400535|811|21887|38659|400535|811|21887|2|15|16|5|12901|139|99|40|70.24|148.90|110.18|1548.80|4407.20|2809.60|5956.00|176.28|0.00|2084.40|4407.20|4583.48|6491.60|6667.88|1597.60| +2450816|56423|2450849|38659|400535|811|21887|38659|400535|811|21887|2|16|12|5|10480|261|99|47|96.25|103.95|16.63|4104.04|781.61|4523.75|4885.65|62.52|0.00|1221.06|781.61|844.13|2002.67|2065.19|-3742.14| +2450816|56423|2450880|38659|400535|811|21887|38659|400535|811|21887|2|37|19|4|1220|96|99|70|58.34|91.01|15.47|5287.80|1082.90|4083.80|6370.70|97.46|0.00|1847.30|1082.90|1180.36|2930.20|3027.66|-3000.90| +2450816|56423|2450846|38659|400535|811|21887|38659|400535|811|21887|2|53|9|2|1990|181|99|46|84.54|226.56|74.76|6982.80|3438.96|3888.84|10421.76|309.50|0.00|2292.64|3438.96|3748.46|5731.60|6041.10|-449.88| +2450816|56423|2450832|38659|400535|811|21887|38659|400535|811|21887|2|91|10|3|13334|106|99|87|12.38|16.83|1.85|1303.26|160.95|1077.06|1464.21|4.82|0.00|731.67|160.95|165.77|892.62|897.44|-916.11| +2450816|56423|2450822|38659|400535|811|21887|38659|400535|811|21887|2|59|5|2|12812|261|99|30|31.94|32.25|5.48|803.10|164.40|958.20|967.50|0.32|147.96|193.50|16.44|16.76|209.94|210.26|-941.76| +2450816|56423|2450824|38659|400535|811|21887|38659|400535|811|21887|2|51|4|4|12061|271|99|49|65.41|120.35|91.46|1415.61|4481.54|3205.09|5897.15|89.63|0.00|1356.32|4481.54|4571.17|5837.86|5927.49|1276.45| +2450816|45041|2450865|69651|896753|655|47357|69651|896753|655|47357|1|75|19|5|1291|133|100|76|91.19|190.58|184.86|434.72|14049.36|6930.44|14484.08|983.45|0.00|6372.60|14049.36|15032.81|20421.96|21405.41|7118.92| +2450816|45041|2450841|69651|896753|655|47357|69651|896753|655|47357|1|56|16|2|6722|66|100|20|39.62|117.27|31.66|1712.20|633.20|792.40|2345.40|44.32|0.00|375.20|633.20|677.52|1008.40|1052.72|-159.20| +2450816|45041|2450877|69651|896753|655|47357|69651|896753|655|47357|1|10|11|4|11029|191|100|86|16.92|33.50|27.13|547.82|2333.18|1455.12|2881.00|93.32|0.00|1238.40|2333.18|2426.50|3571.58|3664.90|878.06| +2450816|45041|2450846|69651|896753|655|47357|69651|896753|655|47357|1|34|18|3|7381|213|100|10|68.23|118.03|10.62|1074.10|106.20|682.30|1180.30|0.00|24.42|70.80|81.78|81.78|152.58|152.58|-600.52| +2450816|45041|2450829|69651|896753|655|47357|69651|896753|655|47357|1|9|7|5|14540|286|100|92|90.09|179.27|43.02|12535.00|3957.84|8288.28|16492.84|0.00|0.00|2143.60|3957.84|3957.84|6101.44|6101.44|-4330.44| +2450816|45041|2450830|69651|896753|655|47357|69651|896753|655|47357|1|67|11|2|5821|164|100|3|19.32|21.63|20.33|3.90|60.99|57.96|64.89|0.60|0.00|27.90|60.99|61.59|88.89|89.49|3.03| +2450816|38521|2450859|53623|351944|218|30139|53623|351944|218|30139|4|26|3|3|6862|26|101|11|11.24|17.75|5.50|134.75|60.50|123.64|195.25|1.81|0.00|13.64|60.50|62.31|74.14|75.95|-63.14| +2450816|38521|2450906|53623|351944|218|30139|53623|351944|218|30139|4|2|2|4|406|218|101|38|25.14|69.88|60.79|345.42|2310.02|955.32|2655.44|46.20|0.00|0.00|2310.02|2356.22|2310.02|2356.22|1354.70| +2450816|38521|2450873|53623|351944|218|30139|53623|351944|218|30139|4|16|12|5|6325|211|101|39|13.33|20.66|13.22|290.16|515.58|519.87|805.74|36.09|0.00|88.53|515.58|551.67|604.11|640.20|-4.29| +2450816|38521|2450852|53623|351944|218|30139|53623|351944|218|30139|4|24|18|2|16345|275|101|61|12.82|34.35|10.99|1424.96|670.39|782.02|2095.35|13.40|0.00|41.48|670.39|683.79|711.87|725.27|-111.63| +2450816|38521|2450883|53623|351944|218|30139|53623|351944|218|30139|4|36|19|2|9385|80|101|35|97.85|284.74|5.69|9766.75|199.15|3424.75|9965.90|0.00|0.00|4285.05|199.15|199.15|4484.20|4484.20|-3225.60| +2450816|38521|2450822|53623|351944|218|30139|53623|351944|218|30139|4|31|13|1|10567|182|101|85|21.34|62.95|18.25|3799.50|1551.25|1813.90|5350.75|139.61|0.00|2086.75|1551.25|1690.86|3638.00|3777.61|-262.65| +2450816|38521|2450836|53623|351944|218|30139|53623|351944|218|30139|4|16|11|1|12199|211|101|10|87.13|257.90|190.84|670.60|1908.40|871.30|2579.00|95.42|0.00|1263.70|1908.40|2003.82|3172.10|3267.52|1037.10| +2450816|38521|2450818|53623|351944|218|30139|53623|351944|218|30139|4|54|9|5|7489|180|101|32|26.32|64.48|29.01|1135.04|928.32|842.24|2063.36|55.69|0.00|515.84|928.32|984.01|1444.16|1499.85|86.08| +2450816|57288|2450847|9410|698752|4094|17836|9410|698752|4094|17836|1|102|11|5|5800|150|102|35|62.89|113.20|99.61|475.65|3486.35|2201.15|3962.00|7.32|3381.75|1188.60|104.60|111.92|1293.20|1300.52|-2096.55| +2450816|57288|2450879|9410|698752|4094|17836|9410|698752|4094|17836|1|33|17|2|12769|232|102|55|55.93|82.21|57.54|1356.85|3164.70|3076.15|4521.55|217.73|443.05|2124.65|2721.65|2939.38|4846.30|5064.03|-354.50| +2450816|57288|2450837|9410|698752|4094|17836|9410|698752|4094|17836|1|57|20|4|3691|182|102|15|63.38|129.29|20.68|1629.15|310.20|950.70|1939.35|4.46|86.85|833.85|223.35|227.81|1057.20|1061.66|-727.35| +2450816|57288|2450849|9410|698752|4094|17836|9410|698752|4094|17836|1|105|12|2|9958|59|102|32|59.71|131.36|49.91|2606.40|1597.12|1910.72|4203.52|31.94|0.00|966.72|1597.12|1629.06|2563.84|2595.78|-313.60| +2450816|57288|2450846|9410|698752|4094|17836|9410|698752|4094|17836|1|66|7|2|986|238|102|89|53.39|100.90|50.45|4490.05|4490.05|4751.71|8980.10|134.70|0.00|2783.03|4490.05|4624.75|7273.08|7407.78|-261.66| +2450816|57288|2450876|9410|698752|4094|17836|9410|698752|4094|17836|1|6|11|5|196|135|102|10|40.68|115.93|20.86|950.70|208.60|406.80|1159.30|2.08|0.00|197.00|208.60|210.68|405.60|407.68|-198.20| +2450816|57288|2450833|9410|698752|4094|17836|9410|698752|4094|17836|1|73|5|1|15901|215|102|79|20.36|31.96|17.57|1136.81|1388.03|1608.44|2524.84|13.88|0.00|24.49|1388.03|1401.91|1412.52|1426.40|-220.41| +2450816|57288|2450852|9410|698752|4094|17836|9410|698752|4094|17836|1|26|20|5|2036|5|102|1|47.13|49.48|10.39|39.09|10.39|47.13|49.48|0.41|0.00|9.40|10.39|10.80|19.79|20.20|-36.74| +2450816|57288|2450864|9410|698752|4094|17836|9410|698752|4094|17836|1|41|19|3|7082|263|102|28|53.67|55.28|43.67|325.08|1222.76|1502.76|1547.84|12.22|0.00|510.72|1222.76|1234.98|1733.48|1745.70|-280.00| +2450816|38973|2450860|80111|1835190|5916|46195|80111|1835190|5916|46195|1|5|9|5|14737|234|103|10|28.21|59.52|53.56|59.60|535.60|282.10|595.20|21.42|0.00|130.90|535.60|557.02|666.50|687.92|253.50| +2450816|38973|2450891|80111|1835190|5916|46195|80111|1835190|5916|46195|1|71|9|5|8233|234|103|40|38.64|103.55|103.55|0.00|4142.00|1545.60|4142.00|331.36|0.00|745.20|4142.00|4473.36|4887.20|5218.56|2596.40| +2450816|38973|2450846|80111|1835190|5916|46195|80111|1835190|5916|46195|1|95|1|5|7891|160|103|39|7.81|21.78|3.04|730.86|118.56|304.59|849.42|3.55|0.00|101.79|118.56|122.11|220.35|223.90|-186.03| +2450816|38973|2450831|80111|1835190|5916|46195|80111|1835190|5916|46195|1|106|18|5|17198|201|103|26|66.64|135.94|6.79|3357.90|176.54|1732.64|3534.44|7.30|54.72|0.00|121.82|129.12|121.82|129.12|-1610.82| +2450817|76478|2450878|44648|421422|6643|4217|44648|421422|6643|4217|1|48|6|5|5689|28|104|84|96.71|98.64|9.86|7457.52|828.24|8123.64|8285.76|49.69|0.00|2816.52|828.24|877.93|3644.76|3694.45|-7295.40| +2450817|76478|2450837|44648|421422|6643|4217|44648|421422|6643|4217|1|84|2|3|6896|145|104|45|11.99|22.66|2.71|897.75|121.95|539.55|1019.70|7.31|0.00|265.05|121.95|129.26|387.00|394.31|-417.60| +2450817|76478|2450903|44648|421422|6643|4217|44648|421422|6643|4217|1|26|8|2|13441|113|104|55|53.75|124.16|54.63|3824.15|3004.65|2956.25|6828.80|0.00|0.00|2799.50|3004.65|3004.65|5804.15|5804.15|48.40| +2450817|76478|2450904|44648|421422|6643|4217|44648|421422|6643|4217|1|46|12|3|3235|226|104|5|2.53|5.06|4.19|4.35|20.95|12.65|25.30|0.00|16.13|4.80|4.82|4.82|9.62|9.62|-7.83| +2450817|76478|2450900|44648|421422|6643|4217|44648|421422|6643|4217|1|30|5|4|110|5|104|64|94.22|203.51|65.12|8856.96|4167.68|6030.08|13024.64|166.70|0.00|1692.80|4167.68|4334.38|5860.48|6027.18|-1862.40| +2450817|76478|2450881|44648|421422|6643|4217|44648|421422|6643|4217|1|21|4|5|16400|283|104|93|84.57|114.16|99.31|1381.05|9235.83|7865.01|10616.88|171.78|3509.61|0.00|5726.22|5898.00|5726.22|5898.00|-2138.79| +2450817|76478|2450825|44648|421422|6643|4217|44648|421422|6643|4217|1|68|15|4|11186|147|104|67|54.30|77.64|6.98|4734.22|467.66|3638.10|5201.88|9.35|0.00|1560.43|467.66|477.01|2028.09|2037.44|-3170.44| +2450817|76478|2450822|44648|421422|6643|4217|44648|421422|6643|4217|1|20|15|3|16612|169|104|48|27.09|53.09|1.06|2497.44|50.88|1300.32|2548.32|0.50|0.00|203.52|50.88|51.38|254.40|254.90|-1249.44| +2450817|40065|2450848|53007|621140|4826|2959|53007|621140|4826|2959|4|24|17|5|4663|141|105|27|43.24|123.23|96.11|732.24|2594.97|1167.48|3327.21|207.59|0.00|1497.15|2594.97|2802.56|4092.12|4299.71|1427.49| +2450817|40065|2450841|53007|621140|4826|2959|53007|621140|4826|2959|4|70|2|2|17066|63|105|41|33.59|44.33|16.40|1145.13|672.40|1377.19|1817.53|20.17|0.00|326.77|672.40|692.57|999.17|1019.34|-704.79| +2450817|40065|2450823|53007|621140|4826|2959|53007|621140|4826|2959|4|95|4|2|16328|16|105|88|55.77|144.44|102.55|3686.32|9024.40|4907.76|12710.72|270.73|0.00|253.44|9024.40|9295.13|9277.84|9548.57|4116.64| +2450817|40065|2450851|53007|621140|4826|2959|53007|621140|4826|2959|4|83|8|5|6199|114|105|43|88.41|90.17|15.32|3218.55|658.76|3801.63|3877.31|6.58|0.00|1163.15|658.76|665.34|1821.91|1828.49|-3142.87| +2450817|40065|2450825|53007|621140|4826|2959|53007|621140|4826|2959|4|90|12|4|6097|131|105|9|85.26|138.12|60.77|696.15|546.93|767.34|1243.08|49.22|0.00|0.00|546.93|596.15|546.93|596.15|-220.41| +2450817|40065|2450858|53007|621140|4826|2959|53007|621140|4826|2959|4|38|12|1|2122|51|105|97|10.44|26.51|14.84|1131.99|1439.48|1012.68|2571.47|54.12|86.36|514.10|1353.12|1407.24|1867.22|1921.34|340.44| +2450817|40663|2450887|89308|430805|5549|31046|89308|430805|5549|31046|1|5|18|3|11420|288|106|83|88.29|193.35|152.74|3370.63|12677.42|7328.07|16048.05|380.32|0.00|4332.60|12677.42|13057.74|17010.02|17390.34|5349.35| +2450817|40663|2450903|89308|430805|5549|31046|89308|430805|5549|31046|1|16|17|5|7747|55|106|90|94.11|143.04|131.59|1030.50|11843.10|8469.90|12873.60|592.15|0.00|3861.90|11843.10|12435.25|15705.00|16297.15|3373.20| +2450817|40663|2450830|89308|430805|5549|31046|89308|430805|5549|31046|1|33|10|5|9799|73|106|79|80.12|144.21|30.28|9000.47|2392.12|6329.48|11392.59|95.68|0.00|3303.78|2392.12|2487.80|5695.90|5791.58|-3937.36| +2450817|40663|2450866|89308|430805|5549|31046|89308|430805|5549|31046|1|21|9|1|16327|183|106|93|39.36|49.59|23.80|2398.47|2213.40|3660.48|4611.87|199.20|0.00|1474.98|2213.40|2412.60|3688.38|3887.58|-1447.08| +2450817|40663|2450853|89308|430805|5549|31046|89308|430805|5549|31046|1|58|6|2|10040|30|106|2|73.66|169.41|147.38|44.06|294.76|147.32|338.82|26.52|0.00|44.04|294.76|321.28|338.80|365.32|147.44| +2450817|40663|2450846|89308|430805|5549|31046|89308|430805|5549|31046|1|68|14|1|3298|205|106|85|61.24|121.86|35.33|7355.05|3003.05|5205.40|10358.10|30.63|2492.53|827.90|510.52|541.15|1338.42|1369.05|-4694.88| +2450817|40663|2450869|89308|430805|5549|31046|89308|430805|5549|31046|1|64|20|1|1364|155|106|35|36.33|65.75|48.65|598.50|1702.75|1271.55|2301.25|51.08|0.00|736.40|1702.75|1753.83|2439.15|2490.23|431.20| +2450817|40663|2450903|89308|430805|5549|31046|89308|430805|5549|31046|1|49|12|4|10496|108|106|6|8.10|15.06|10.24|28.92|61.44|48.60|90.36|1.22|0.00|3.60|61.44|62.66|65.04|66.26|12.84| +2450817|40663|2450823|89308|430805|5549|31046|89308|430805|5549|31046|1|88|7|2|10468|208|106|4|38.33|79.34|56.33|92.04|225.32|153.32|317.36|0.00|0.00|0.00|225.32|225.32|225.32|225.32|72.00| +2450817|40663|2450861|89308|430805|5549|31046|89308|430805|5549|31046|1|1|4|3|12398|136|106|85|9.73|26.85|5.37|1825.80|456.45|827.05|2282.25|27.38|0.00|935.00|456.45|483.83|1391.45|1418.83|-370.60| +2450817|40663|2450859|89308|430805|5549|31046|89308|430805|5549|31046|1|57|11|3|14005|224|106|67|30.53|56.48|10.16|3103.44|680.72|2045.51|3784.16|54.45|0.00|377.88|680.72|735.17|1058.60|1113.05|-1364.79| +2450817|40663|2450856|89308|430805|5549|31046|89308|430805|5549|31046|1|60|16|4|15469|192|106|20|97.60|196.17|66.69|2589.60|1333.80|1952.00|3923.40|120.04|0.00|902.20|1333.80|1453.84|2236.00|2356.04|-618.20| +2450817|40663|2450900|89308|430805|5549|31046|89308|430805|5549|31046|1|96|17|1|7300|14|106|13|11.10|17.98|11.14|88.92|144.82|144.30|233.74|13.03|0.00|114.53|144.82|157.85|259.35|272.38|0.52| +2450817|71385|2450863|95481|663720|3159|18551|95481|663720|3159|18551|2|82|14|2|7036|223|107|3|30.31|76.68|71.31|16.11|213.93|90.93|230.04|19.25|0.00|48.30|213.93|233.18|262.23|281.48|123.00| +2450817|71385|2450882|95481|663720|3159|18551|95481|663720|3159|18551|2|53|12|3|16358|128|107|14|2.50|4.00|2.28|24.08|31.92|35.00|56.00|2.87|0.00|28.00|31.92|34.79|59.92|62.79|-3.08| +2450817|71385|2450905|95481|663720|3159|18551|95481|663720|3159|18551|2|25|6|5|3824|11|107|35|42.24|60.82|12.16|1703.10|425.60|1478.40|2128.70|13.61|153.21|276.50|272.39|286.00|548.89|562.50|-1206.01| +2450817|71385|2450840|95481|663720|3159|18551|95481|663720|3159|18551|2|84|18|3|6488|233|107|39|94.26|258.27|43.90|8360.43|1712.10|3676.14|10072.53|17.12|0.00|1712.10|1712.10|1729.22|3424.20|3441.32|-1964.04| +2450817|71385|2450902|95481|663720|3159|18551|95481|663720|3159|18551|2|37|16|5|2608|281|107|63|86.97|104.36|34.43|4405.59|2169.09|5479.11|6574.68|151.83|0.00|2432.43|2169.09|2320.92|4601.52|4753.35|-3310.02| +2450817|71385|2450863|95481|663720|3159|18551|95481|663720|3159|18551|2|65|10|5|187|242|107|28|31.80|58.19|34.91|651.84|977.48|890.40|1629.32|29.32|0.00|390.88|977.48|1006.80|1368.36|1397.68|87.08| +2450817|71385|2450858|95481|663720|3159|18551|95481|663720|3159|18551|2|1|12|5|11131|183|107|21|46.94|96.69|49.31|994.98|1035.51|985.74|2030.49|41.42|0.00|324.87|1035.51|1076.93|1360.38|1401.80|49.77| +2450817|71385|2450893|95481|663720|3159|18551|95481|663720|3159|18551|2|28|4|2|5455|86|107|45|24.36|65.04|18.86|2078.10|848.70|1096.20|2926.80|42.43|0.00|1199.70|848.70|891.13|2048.40|2090.83|-247.50| +2450817|71385|2450847|95481|663720|3159|18551|95481|663720|3159|18551|2|78|18|2|17366|13|107|27|93.38|94.31|37.72|1527.93|1018.44|2521.26|2546.37|20.36|0.00|458.19|1018.44|1038.80|1476.63|1496.99|-1502.82| +2450817|71385|2450821|95481|663720|3159|18551|95481|663720|3159|18551|2|21|11|2|1651|290|107|97|89.39|125.14|63.82|5948.04|6190.54|8670.83|12138.58|123.81|0.00|5219.57|6190.54|6314.35|11410.11|11533.92|-2480.29| +2450817|71385|2450878|95481|663720|3159|18551|95481|663720|3159|18551|2|42|18|1|4519|139|107|31|90.55|96.88|23.25|2282.53|720.75|2807.05|3003.28|28.83|0.00|360.22|720.75|749.58|1080.97|1109.80|-2086.30| +2450817|71385|2450887|95481|663720|3159|18551|95481|663720|3159|18551|2|95|4|4|10009|101|107|83|64.29|91.29|52.94|3183.05|4394.02|5336.07|7577.07|7.03|4218.25|3257.75|175.77|182.80|3433.52|3440.55|-5160.30| +2450817|74108|2450860|52770|285795|5220|16645|52770|285795|5220|16645|4|70|18|5|14005|232|108|80|49.32|121.32|100.69|1650.40|8055.20|3945.60|9705.60|724.96|0.00|2037.60|8055.20|8780.16|10092.80|10817.76|4109.60| +2450817|74108|2450832|52770|285795|5220|16645|52770|285795|5220|16645|4|97|3|2|15469|250|108|6|40.23|67.18|34.93|193.50|209.58|241.38|403.08|2.09|0.00|76.56|209.58|211.67|286.14|288.23|-31.80| +2450817|74108|2450907|52770|285795|5220|16645|52770|285795|5220|16645|4|66|6|4|7300|3|108|19|78.14|208.63|22.94|3528.11|435.86|1484.66|3963.97|21.79|0.00|1189.02|435.86|457.65|1624.88|1646.67|-1048.80| +2450817|74108|2450847|52770|285795|5220|16645|52770|285795|5220|16645|4|18|4|3|6583|39|108|54|11.41|25.67|18.48|388.26|997.92|616.14|1386.18|89.81|0.00|262.98|997.92|1087.73|1260.90|1350.71|381.78| +2450817|70549|2450829|24615|947475|288|31148|24615|947475|288|31148|1|3|15|5|8701|156|109|41|83.68|215.05|195.69|793.76|8023.29|3430.88|8817.05|320.93|0.00|3879.42|8023.29|8344.22|11902.71|12223.64|4592.41| +2450817|70549|2450895|24615|947475|288|31148|24615|947475|288|31148|1|78|13|1|6928|198|109|48|79.80|229.02|139.70|4287.36|6705.60|3830.40|10992.96|469.39|0.00|659.52|6705.60|7174.99|7365.12|7834.51|2875.20| +2450817|70549|2450839|24615|947475|288|31148|24615|947475|288|31148|1|79|4|1|4621|257|109|26|33.51|89.13|52.58|950.30|1367.08|871.26|2317.38|82.02|0.00|509.60|1367.08|1449.10|1876.68|1958.70|495.82| +2450817|70549|2450868|24615|947475|288|31148|24615|947475|288|31148|1|12|6|5|5935|253|109|58|4.17|12.25|0.61|675.12|35.38|241.86|710.50|0.46|19.81|134.56|15.57|16.03|150.13|150.59|-226.29| +2450817|70549|2450890|24615|947475|288|31148|24615|947475|288|31148|1|107|8|4|11386|263|109|26|36.19|82.51|12.37|1823.64|321.62|940.94|2145.26|16.08|0.00|107.12|321.62|337.70|428.74|444.82|-619.32| +2450817|70549|2450840|24615|947475|288|31148|24615|947475|288|31148|1|59|13|1|2582|190|109|61|29.09|41.59|16.22|1547.57|989.42|1774.49|2536.99|0.98|969.63|913.17|19.79|20.77|932.96|933.94|-1754.70| +2450817|70549|2450903|24615|947475|288|31148|24615|947475|288|31148|1|102|5|2|15550|93|109|40|71.86|99.16|37.68|2459.20|1507.20|2874.40|3966.40|120.57|0.00|436.00|1507.20|1627.77|1943.20|2063.77|-1367.20| +||2450836|||2394||48450|1173022|||4|20|4|1|8692||110||39.34|117.62|||||||||2189.94||2480.45|2480.45|1442.48| +2450817|50770|2450855|48450|1173022|2394|37366|48450|1173022|2394|37366|4|10|18|4|1978|81|110|69|22.87|64.72|58.24|447.12|4018.56|1578.03|4465.68|200.92|0.00|1830.57|4018.56|4219.48|5849.13|6050.05|2440.53| +2450817|50770|2450856|48450|1173022|2394|37366|48450|1173022|2394|37366|4|40|14|2|9094|150|110|79|43.22|127.49|52.27|5942.38|4129.33|3414.38|10071.71|123.87|0.00|4733.68|4129.33|4253.20|8863.01|8986.88|714.95| +2450817|50770|2450839|48450|1173022|2394|37366|48450|1173022|2394|37366|4|25|10|2|13702|228|110|63|57.66|163.75|13.10|9490.95|825.30|3632.58|10316.25|24.75|0.00|2991.24|825.30|850.05|3816.54|3841.29|-2807.28| +2450817|50770|2450882|48450|1173022|2394|37366|48450|1173022|2394|37366|4|62|4|4|7471|108|110|61|81.97|190.99|164.25|1631.14|10019.25|5000.17|11650.39|100.19|0.00|5591.87|10019.25|10119.44|15611.12|15711.31|5019.08| +2450817|50770|2450829|48450|1173022|2394|37366|48450|1173022|2394|37366|4|53|5|2|6934|14|110|22|50.78|54.33|10.86|956.34|238.92|1117.16|1195.26|4.77|0.00|11.88|238.92|243.69|250.80|255.57|-878.24| +2450817|50770|2450872|48450|1173022|2394|37366|48450|1173022|2394|37366|4|51|2|2|1822|155|110|92|4.77|8.49|8.06|39.56|741.52|438.84|781.08|59.32|0.00|179.40|741.52|800.84|920.92|980.24|302.68| +2450817|50770|2450866|48450|1173022|2394|37366|48450|1173022|2394|37366|4|10|18|1|12070|162|110|92|90.32|144.51|98.26|4255.00|9039.92|8309.44|13294.92|90.39|0.00|5184.20|9039.92|9130.31|14224.12|14314.51|730.48| +2450817|50770|2450823|48450|1173022|2394|37366|48450|1173022|2394|37366|4|93|13|2|5182|16|110|74|81.22|84.46|43.91|3000.70|3249.34|6010.28|6250.04|194.96|0.00|2624.78|3249.34|3444.30|5874.12|6069.08|-2760.94| +2450817|50770|2450885|48450|1173022|2394|37366|48450|1173022|2394|37366|4|47|5|5|5500|93|110|56|52.03|137.35|34.33|5769.12|1922.48|2913.68|7691.60|19.22|0.00|768.88|1922.48|1941.70|2691.36|2710.58|-991.20| +2450817|50770|2450871|48450|1173022|2394|37366|48450|1173022|2394|37366|4|58|19|4|12322|12|110|26|59.37|138.92|76.40|1625.52|1986.40|1543.62|3611.92|94.55|635.64|1805.96|1350.76|1445.31|3156.72|3251.27|-192.86| +2450817|62461|2450895|71121|1030555|7142|33376|71121|1030555|7142|33376|2|33|4|4|17125|237|111|8|16.89|34.62|8.65|207.76|69.20|135.12|276.96|3.46|0.00|116.32|69.20|72.66|185.52|188.98|-65.92| +2450817|62461|2450872|71121|1030555|7142|33376|71121|1030555|7142|33376|2|2|13|4|14773|239|111|58|93.82|257.06|5.14|14611.36|298.12|5441.56|14909.48|0.00|0.00|1192.48|298.12|298.12|1490.60|1490.60|-5143.44| +2450817|62461|2450849|71121|1030555|7142|33376|71121|1030555|7142|33376|2|64|6|5|4190|263|111|30|3.72|5.39|3.66|51.90|109.80|111.60|161.70|9.88|0.00|35.40|109.80|119.68|145.20|155.08|-1.80| +2450817|62461|2450824|71121|1030555|7142|33376|71121|1030555|7142|33376|2|54|11|1|1375|244|111|14|76.40|85.56|74.43|155.82|1042.02|1069.60|1197.84|83.36|0.00|407.26|1042.02|1125.38|1449.28|1532.64|-27.58| +2450817|62461|2450881|71121|1030555|7142|33376|71121|1030555|7142|33376|2|90|11|4|3362|186|111|58|7.96|13.13|13.13|0.00|761.54|461.68|761.54|0.00|0.00|327.12|761.54|761.54|1088.66|1088.66|299.86| +2450817|62461|2450886|71121|1030555|7142|33376|71121|1030555|7142|33376|2|103|9|5|11563|295|111|54|77.87|102.78|69.89|1776.06|3774.06|4204.98|5550.12|150.96|0.00|776.52|3774.06|3925.02|4550.58|4701.54|-430.92| +2450817|62461|2450897|71121|1030555|7142|33376|71121|1030555|7142|33376|2|108|12|4|5413|290|111|82|54.79|118.34|5.91|9219.26|484.62|4492.78|9703.88|14.53|0.00|582.20|484.62|499.15|1066.82|1081.35|-4008.16| +2450817|62461|2450827|71121|1030555|7142|33376|71121|1030555|7142|33376|2|40|11|5|12050|46|111|48|88.94|90.71|58.96|1524.00|2830.08|4269.12|4354.08|113.20|0.00|1654.08|2830.08|2943.28|4484.16|4597.36|-1439.04| +2450817|62461|2450900|71121|1030555|7142|33376|71121|1030555|7142|33376|2|54|7|2|10666|189|111|38|19.52|41.77|33.83|301.72|1285.54|741.76|1587.26|52.45|411.37|476.14|874.17|926.62|1350.31|1402.76|132.41| +2450817|62461|2450856|71121|1030555|7142|33376|71121|1030555|7142|33376|2|71|1|3|5654|29|111|40|25.84|45.99|10.57|1416.80|422.80|1033.60|1839.60|19.53|97.24|606.80|325.56|345.09|932.36|951.89|-708.04| +2450817|62461|2450879|71121|1030555|7142|33376|71121|1030555|7142|33376|2|33|7|5|2618|1|111|20|94.84|277.88|200.07|1556.20|4001.40|1896.80|5557.60|280.09|0.00|778.00|4001.40|4281.49|4779.40|5059.49|2104.60| +2450817|62461|2450841|71121|1030555|7142|33376|71121|1030555|7142|33376|2|13|3|2|6319|88|111|66|33.59|35.94|17.97|1186.02|1186.02|2216.94|2372.04|83.02|0.00|877.14|1186.02|1269.04|2063.16|2146.18|-1030.92| +2450817|62461|2450858|71121|1030555|7142|33376|71121|1030555|7142|33376|2|25|14|4|2996|54|111|17|52.09|66.15|46.30|337.45|787.10|885.53|1124.55|39.35|0.00|191.08|787.10|826.45|978.18|1017.53|-98.43| +2450817|62461|2450906|71121|1030555|7142|33376|71121|1030555|7142|33376|2|77|5|3|13153|17|111|33|23.49|35.00|23.80|369.60|785.40|775.17|1155.00|70.68|0.00|473.55|785.40|856.08|1258.95|1329.63|10.23| +2450817|67659|2450878|40439|1182114|2636|5511|40439|1182114|2636|5511|1|46|4|2|3508|8|112|41|97.64|133.76|86.94|1919.62|3564.54|4003.24|5484.16|71.29|0.00|1809.74|3564.54|3635.83|5374.28|5445.57|-438.70| +2450817|67659|2450833|40439|1182114|2636|5511|40439|1182114|2636|5511|1|104|9|2|1264|163|112|74|58.98|81.39|70.80|783.66|5239.20|4364.52|6022.86|157.17|0.00|1023.42|5239.20|5396.37|6262.62|6419.79|874.68| +2450817|67659|2450879|40439|1182114|2636|5511|40439|1182114|2636|5511|1|24|16|5|4225|178|112|96|35.63|46.67|18.66|2688.96|1791.36|3420.48|4480.32|143.30|0.00|716.16|1791.36|1934.66|2507.52|2650.82|-1629.12| +2450817|67659|2450884|40439|1182114|2636|5511|40439|1182114|2636|5511|1|38|14|3|6814|281|112|80|71.13|108.11|55.13|4238.40|4410.40|5690.40|8648.80|0.00|0.00|1729.60|4410.40|4410.40|6140.00|6140.00|-1280.00| +2450817|67659|2450882|40439|1182114|2636|5511|40439|1182114|2636|5511|1|34|16|2|1630|256|112|94|45.79|59.52|49.99|895.82|4699.06|4304.26|5594.88|187.96|0.00|55.46|4699.06|4887.02|4754.52|4942.48|394.80| +2450817|67659|2450826|40439|1182114|2636|5511|40439|1182114|2636|5511|1|38|7|1|14188|115|112|44|24.61|31.74|19.67|531.08|865.48|1082.84|1396.56|43.27|0.00|195.36|865.48|908.75|1060.84|1104.11|-217.36| +2450817|67659|2450841|40439|1182114|2636|5511|40439|1182114|2636|5511|1|27|6|4|5482|234|112|45|99.80|265.46|180.51|3822.75|8122.95|4491.00|11945.70|324.91|0.00|4897.35|8122.95|8447.86|13020.30|13345.21|3631.95| +2450817|67659|2450865|40439|1182114|2636|5511|40439|1182114|2636|5511|1|68|19|3|17527|17|112|79|37.71|70.14|37.87|2549.33|2991.73|2979.09|5541.06|149.58|0.00|2105.35|2991.73|3141.31|5097.08|5246.66|12.64| +2450817|67659|2450825|40439|1182114|2636|5511|40439|1182114|2636|5511|1|49|5|3|8623|279|112|9|66.92|94.35|33.02|551.97|297.18|602.28|849.15|23.77|0.00|93.33|297.18|320.95|390.51|414.28|-305.10| +2450817|67659|2450857|40439|1182114|2636|5511|40439|1182114|2636|5511|1|13|14|5|10750|260|112|63|80.94|182.11|83.77|6195.42|5277.51|5099.22|11472.93|72.82|1636.02|4588.92|3641.49|3714.31|8230.41|8303.23|-1457.73| +2450817|67659|2450824|40439|1182114|2636|5511|40439|1182114|2636|5511|1|3|7|2|6901|215|112|40|33.23|53.83|24.76|1162.80|990.40|1329.20|2153.20|5.64|802.22|150.40|188.18|193.82|338.58|344.22|-1141.02| +2450817|67659|2450905|40439|1182114|2636|5511|40439|1182114|2636|5511|1|105|8|1|4798|79|112|91|24.03|32.44|28.22|384.02|2568.02|2186.73|2952.04|179.76|0.00|1327.69|2568.02|2747.78|3895.71|4075.47|381.29| +2450817|67659|2450858|40439|1182114|2636|5511|40439|1182114|2636|5511|1|61|7|2|13082|237|112|65|68.68|136.67|117.53|1244.10|7639.45|4464.20|8883.55|687.55|0.00|1865.50|7639.45|8327.00|9504.95|10192.50|3175.25| +2450817|67659|2450860|40439|1182114|2636|5511|40439|1182114|2636|5511|1|50|2|3|12226|205|112|66|73.35|81.41|17.09|4245.12|1127.94|4841.10|5373.06|11.27|563.97|0.00|563.97|575.24|563.97|575.24|-4277.13| +2450817|10722|2450900|15288|1495357|3426|47942|15288|1495357|3426|47942|2|49|12|4|5743|66|113|37|33.80|42.92|40.34|95.46|1492.58|1250.60|1588.04|104.48|0.00|794.02|1492.58|1597.06|2286.60|2391.08|241.98| +2450817|10722|2450874|15288|1495357|3426|47942|15288|1495357|3426|47942|2|92|5|5|1796|293|113|70|53.33|137.59|72.92|4526.90|5104.40|3733.10|9631.30|255.22|0.00|2407.30|5104.40|5359.62|7511.70|7766.92|1371.30| +2450817|10722|2450882|15288|1495357|3426|47942|15288|1495357|3426|47942|2|21|11|2|13598|244|113|27|14.41|34.43|23.06|306.99|622.62|389.07|929.61|6.22|0.00|306.72|622.62|628.84|929.34|935.56|233.55| +2450817|10722|2450903|15288|1495357|3426|47942|15288|1495357|3426|47942|2|103|17|4|12397|61|113|61|51.16|78.27|17.21|3724.66|1049.81|3120.76|4774.47|0.00|0.00|2243.58|1049.81|1049.81|3293.39|3293.39|-2070.95| +2450817|10722|2450845|15288|1495357|3426|47942|15288|1495357|3426|47942|2|92|8|5|15470|17|113|96|12.44|36.82|34.61|212.16|3322.56|1194.24|3534.72|99.67|0.00|671.04|3322.56|3422.23|3993.60|4093.27|2128.32| +2450817|10722|2450856|15288|1495357|3426|47942|15288|1495357|3426|47942|2|1|8|1|11464|270|113|18|36.56|37.65|22.96|264.42|413.28|658.08|677.70|4.13|0.00|81.18|413.28|417.41|494.46|498.59|-244.80| +2450817|10722|2450821|15288|1495357|3426|47942|15288|1495357|3426|47942|2|78|17|5|12650|59|113|97|38.72|98.34|81.62|1621.84|7917.14|3755.84|9538.98|79.17|0.00|4769.49|7917.14|7996.31|12686.63|12765.80|4161.30| +2450817|10722|2450893|15288|1495357|3426|47942|15288|1495357|3426|47942|2|30|3|5|4159|247|113|86|25.92|64.28|38.56|2211.92|3316.16|2229.12|5528.08|165.80|0.00|773.14|3316.16|3481.96|4089.30|4255.10|1087.04| +2450817|28496|2450823|52964|1699458|1599|35293|52964|1699458|1599|35293|2|9|3|1|1030|71|114|22|85.08|203.34|63.03|3086.82|1386.66|1871.76|4473.48|27.73|0.00|0.00|1386.66|1414.39|1386.66|1414.39|-485.10| +2450817|28496|2450887|52964|1699458|1599|35293|52964|1699458|1599|35293|2|47|12|3|4340|45|114|40|33.01|43.24|23.34|796.00|933.60|1320.40|1729.60|56.01|0.00|449.60|933.60|989.61|1383.20|1439.21|-386.80| +2450817|28496|2450819|52964|1699458|1599|35293|52964|1699458|1599|35293|2|27|19|1|6535|39|114|57|94.83|280.69|154.37|7200.24|8799.09|5405.31|15999.33|450.51|3167.67|319.77|5631.42|6081.93|5951.19|6401.70|226.11| +2450817|28496|2450856|52964|1699458|1599|35293|52964|1699458|1599|35293|2|63|2|3|15466|129|114|64|27.49|58.82|52.93|376.96|3387.52|1759.36|3764.48|0.00|0.00|752.64|3387.52|3387.52|4140.16|4140.16|1628.16| +2450817|28496|2450822|52964|1699458|1599|35293|52964|1699458|1599|35293|2|37|1|4|12457|224|114|91|86.35|117.43|65.76|4701.97|5984.16|7857.85|10686.13|239.36|0.00|2457.00|5984.16|6223.52|8441.16|8680.52|-1873.69| +2450817|28496|2450866|52964|1699458|1599|35293|52964|1699458|1599|35293|2|45|11|4|2005|280|114|34|58.24|165.98|127.80|1298.12|4345.20|1980.16|5643.32|260.71|0.00|1805.74|4345.20|4605.91|6150.94|6411.65|2365.04| +2450817|28496|2450905|52964|1699458|1599|35293|52964|1699458|1599|35293|2|1|11|5|8506|265|114|30|18.54|53.39|24.02|881.10|720.60|556.20|1601.70|57.64|0.00|224.10|720.60|778.24|944.70|1002.34|164.40| +2450817|28496|2450841|52964|1699458|1599|35293|52964|1699458|1599|35293|2|40|15|5|14392|157|114|64|23.39|50.05|1.00|3139.20|64.00|1496.96|3203.20|5.12|0.00|1601.28|64.00|69.12|1665.28|1670.40|-1432.96| +2450817|28496|2450904|52964|1699458|1599|35293|52964|1699458|1599|35293|2|63|14|3|14330|9|114|80|91.19|101.22|30.36|5668.80|2428.80|7295.20|8097.60|170.01|0.00|2104.80|2428.80|2598.81|4533.60|4703.61|-4866.40| +2450817|28496|2450854|52964|1699458|1599|35293|52964|1699458|1599|35293|2|2|8|4|2260|164|114|25|33.66|89.53|72.51|425.50|1812.75|841.50|2238.25|54.38|0.00|402.75|1812.75|1867.13|2215.50|2269.88|971.25| +2450817|28970|2450883|46613|532851|7089|37327|46613|532851|7089|37327|2|14|7|4|9788|80|115|69|79.45|166.84|131.80|2417.76|9094.20|5482.05|11511.96|818.47|0.00|460.23|9094.20|9912.67|9554.43|10372.90|3612.15| +2450817|28970|2450836|46613|532851|7089|37327|46613|532851|7089|37327|2|68|3|4|5140|28|115|45|26.60|33.78|26.68|319.50|1200.60|1197.00|1520.10|72.03|0.00|45.45|1200.60|1272.63|1246.05|1318.08|3.60| +2450817|28970|2450902|46613|532851|7089|37327|46613|532851|7089|37327|2|83|18|1|16630|277|115|23|14.02|31.12|5.91|579.83|135.93|322.46|715.76|2.93|99.22|143.06|36.71|39.64|179.77|182.70|-285.75| +2450817|28970|2450853|46613|532851|7089|37327|46613|532851|7089|37327|2|58|18|1|1741|129|115|35|15.47|34.03|5.44|1000.65|190.40|541.45|1191.05|17.13|0.00|392.70|190.40|207.53|583.10|600.23|-351.05| +2450817|28970|2450823|46613|532851|7089|37327|46613|532851|7089|37327|2|97|11|2|16513|22|115|3|30.41|74.20|67.52|20.04|202.56|91.23|222.60|18.23|0.00|13.35|202.56|220.79|215.91|234.14|111.33| +2450817|28970|2450883|46613|532851|7089|37327|46613|532851|7089|37327|2|102|15|1|9355|73|115|70|31.49|92.58|50.91|2916.90|3563.70|2204.30|6480.60|0.00|0.00|1749.30|3563.70|3563.70|5313.00|5313.00|1359.40| +2450817|35308|2450858|92771|1405329|2687|29292|92771|1405329|2687|29292|1|64|6|3|370|159|116|56|56.56|124.99|42.49|4620.00|2379.44|3167.36|6999.44|0.00|1879.75|2029.44|499.69|499.69|2529.13|2529.13|-2667.67| +2450817|35308|2450891|92771|1405329|2687|29292|92771|1405329|2687|29292|1|84|9|2|3364|213|116|96|59.51|147.58|100.35|4534.08|9633.60|5712.96|14167.68|0.00|2890.08|2832.96|6743.52|6743.52|9576.48|9576.48|1030.56| +2450817|35308|2450851|92771|1405329|2687|29292|92771|1405329|2687|29292|1|70|12|4|7675|82|116|22|53.47|89.29|50.00|864.38|1100.00|1176.34|1964.38|77.00|0.00|707.08|1100.00|1177.00|1807.08|1884.08|-76.34| +2450817|35308|2450849|92771|1405329|2687|29292|92771|1405329|2687|29292|1|69|18|5|9904|165|116|27|86.95|130.42|7.82|3310.20|211.14|2347.65|3521.34|19.00|0.00|1161.81|211.14|230.14|1372.95|1391.95|-2136.51| +2450817|46732|2450832|60732|1117497|215|35380|60732|1117497|215|35380|4|28|7|3|17356|115|117|59|9.28|25.61|1.79|1405.38|105.61|547.52|1510.99|5.28|0.00|588.82|105.61|110.89|694.43|699.71|-441.91| +2450817|46732|2450856|60732|1117497|215|35380|60732|1117497|215|35380|4|106|2|5|10741|133|117|1|42.49|95.17|42.82|52.35|42.82|42.49|95.17|1.71|0.00|2.85|42.82|44.53|45.67|47.38|0.33| +2450817|46732|2450871|60732|1117497|215|35380|60732|1117497|215|35380|4|108|10|1|14236|165|117|93|84.62|225.93|29.37|18280.08|2731.41|7869.66|21011.49|0.00|0.00|7773.87|2731.41|2731.41|10505.28|10505.28|-5138.25| +2450817|46732|2450833|60732|1117497|215|35380|60732|1117497|215|35380|4|23|6|3|1300|146|117|23|88.15|141.92|58.18|1926.02|1338.14|2027.45|3264.16|99.95|227.48|652.74|1110.66|1210.61|1763.40|1863.35|-916.79| +2450817|46732|2450828|60732|1117497|215|35380|60732|1117497|215|35380|4|20|6|1|9763|171|117|19|27.27|52.08|32.81|366.13|623.39|518.13|989.52|37.40|0.00|435.29|623.39|660.79|1058.68|1096.08|105.26| +2450817|46732|2450902|60732|1117497|215|35380|60732|1117497|215|35380|4|10|8|3|14317|263|117|13|63.44|66.61|54.62|155.87|710.06|824.72|865.93|56.80|0.00|164.45|710.06|766.86|874.51|931.31|-114.66| +2450817|46732|2450845|60732|1117497|215|35380|60732|1117497|215|35380|4|105|7|1|10492|254|117|9|22.56|24.81|11.41|120.60|102.69|203.04|223.29|9.24|0.00|28.98|102.69|111.93|131.67|140.91|-100.35| +2450817|46732|2450847|60732|1117497|215|35380|60732|1117497|215|35380|4|106|5|2|727|238|117|100|45.76|117.14|16.39|10075.00|1639.00|4576.00|11714.00|49.17|0.00|3279.00|1639.00|1688.17|4918.00|4967.17|-2937.00| +2450817|46732|2450875|60732|1117497|215|35380|60732|1117497|215|35380|4|105|9|2|14860|103|117|2|53.35|66.15|37.70|56.90|75.40|106.70|132.30|2.26|0.00|27.78|75.40|77.66|103.18|105.44|-31.30| +2450817|46732|2450899|60732|1117497|215|35380|60732|1117497|215|35380|4|24|11|5|10939|93|117|87|62.60|175.90|86.19|7804.77|7498.53|5446.20|15303.30|26.99|6598.70|1988.82|899.83|926.82|2888.65|2915.64|-4546.37| +2450817|46732|2450820|60732|1117497|215|35380|60732|1117497|215|35380|4|78|7|2|5722|227|117|27|98.84|175.93|160.09|427.68|4322.43|2668.68|4750.11|389.01|0.00|807.30|4322.43|4711.44|5129.73|5518.74|1653.75| +2450817|46732|2450878|60732|1117497|215|35380|60732|1117497|215|35380|4|106|10|1|12340|41|117|95|22.06|59.12|1.77|5448.25|168.15|2095.70|5616.40|4.69|116.02|2245.80|52.13|56.82|2297.93|2302.62|-2043.57| +2450817|75750|2450889|3116|829297|5878|28859|3116|829297|5878|28859|4|79|15|4|3392|87|118|82|91.30|178.03|49.84|10511.58|4086.88|7486.60|14598.46|286.08|0.00|6714.98|4086.88|4372.96|10801.86|11087.94|-3399.72| +2450817|75750|2450830|3116|829297|5878|28859|3116|829297|5878|28859|4|4|6|2|4768|240|118|62|60.28|124.77|49.90|4641.94|3093.80|3737.36|7735.74|247.50|0.00|76.88|3093.80|3341.30|3170.68|3418.18|-643.56| +2450817|75750|2450873|3116|829297|5878|28859|3116|829297|5878|28859|4|82|19|2|8438|71|118|97|36.78|54.80|40.55|1382.25|3933.35|3567.66|5315.60|39.33|0.00|1647.06|3933.35|3972.68|5580.41|5619.74|365.69| +2450817|75750|2450905|3116|829297|5878|28859|3116|829297|5878|28859|4|48|9|3|4906|276|118|81|87.62|222.55|135.75|7030.80|10995.75|7097.22|18026.55|0.00|0.00|4506.03|10995.75|10995.75|15501.78|15501.78|3898.53| +2450817|75750|2450853|3116|829297|5878|28859|3116|829297|5878|28859|4|17|13|5|6793|49|118|76|81.81|117.80|3.53|8684.52|268.28|6217.56|8952.80|21.46|0.00|3759.72|268.28|289.74|4028.00|4049.46|-5949.28| +2450817|75750|2450827|3116|829297|5878|28859|3116|829297|5878|28859|4|85|18|5|3622|255|118|21|58.11|59.85|50.27|201.18|1055.67|1220.31|1256.85|31.67|0.00|62.79|1055.67|1087.34|1118.46|1150.13|-164.64| +2450817|75750|2450888|3116|829297|5878|28859|3116|829297|5878|28859|4|26|10|4|9926|239|118|16|41.16|51.45|12.34|625.76|197.44|658.56|823.20|15.63|23.69|337.44|173.75|189.38|511.19|526.82|-484.81| +2450817|75750|2450833|3116|829297|5878|28859|3116|829297|5878|28859|4|16|18|5|4975|157|118|34|11.07|19.92|2.39|596.02|81.26|376.38|677.28|2.43|0.00|284.24|81.26|83.69|365.50|367.93|-295.12| +2450817|75750|2450829|3116|829297|5878|28859|3116|829297|5878|28859|4|58|3|3|637|93|118|58|43.43|121.60|55.93|3808.86|3243.94|2518.94|7052.80|124.56|129.75|3032.24|3114.19|3238.75|6146.43|6270.99|595.25| +2450817|75750|2450885|3116|829297|5878|28859|3116|829297|5878|28859|4|72|9|5|9781|282|118|72|48.35|73.49|47.03|1905.12|3386.16|3481.20|5291.28|67.72|0.00|899.28|3386.16|3453.88|4285.44|4353.16|-95.04| +2450817|75750|2450900|3116|829297|5878|28859|3116|829297|5878|28859|4|62|9|5|13610|267|118|20|55.97|111.94|107.46|89.60|2149.20|1119.40|2238.80|85.96|0.00|694.00|2149.20|2235.16|2843.20|2929.16|1029.80| +2450817|46544|2450891|60418|1185578|4338|15098|60418|1185578|4338|15098|4|78|9|3|1148|56|119|24|22.89|64.54|26.46|913.92|635.04|549.36|1548.96|0.00|0.00|309.60|635.04|635.04|944.64|944.64|85.68| +2450817|46544|2450821|60418|1185578|4338|15098|60418|1185578|4338|15098|4|77|11|5|6085|113|119|77|23.53|24.47|22.02|188.65|1695.54|1811.81|1884.19|0.00|0.00|451.99|1695.54|1695.54|2147.53|2147.53|-116.27| +2450817|46544|2450886|60418|1185578|4338|15098|60418|1185578|4338|15098|4|52|19|3|4370|238|119|79|84.40|122.38|74.65|3770.67|5897.35|6667.60|9668.02|235.89|0.00|1643.20|5897.35|6133.24|7540.55|7776.44|-770.25| +2450817|46544|2450831|60418|1185578|4338|15098|60418|1185578|4338|15098|4|46|5|2|12412|208|119|69|81.35|152.12|120.17|2204.55|8291.73|5613.15|10496.28|580.42|0.00|5142.57|8291.73|8872.15|13434.30|14014.72|2678.58| +2450817|46544|2450823|60418|1185578|4338|15098|60418|1185578|4338|15098|4|38|9|2|6079|45|119|16|21.15|31.09|11.19|318.40|179.04|338.40|497.44|8.95|0.00|9.92|179.04|187.99|188.96|197.91|-159.36| +2450817|46544|2450819|60418|||||1185578|||4|62||4|12662|70|119|37|37.97||11.11|3325.93|411.07|1404.89|||0.00|1494.80||443.95|1905.87|1938.75|-993.82| +2450817|55814|2450866|30925|1164796|5067|6938|30925|1164796|5067|6938|2|104|16|5|10117|95|120|44|69.07|168.53|60.67|4745.84|2669.48|3039.08|7415.32|133.47|0.00|1482.80|2669.48|2802.95|4152.28|4285.75|-369.60| +2450817|55814|2450831|30925|1164796|5067|6938|30925|1164796|5067|6938|2|39|8|2|16952|235|120|1|20.50|45.92|43.62|2.30|43.62|20.50|45.92|3.05|0.00|3.21|43.62|46.67|46.83|49.88|23.12| +2450817|55814|2450857|30925|1164796|5067|6938|30925|1164796|5067|6938|2|40|6|4|8410|25|120|73|65.84|168.55|64.04|7629.23|4674.92|4806.32|12304.15|373.99|0.00|5782.33|4674.92|5048.91|10457.25|10831.24|-131.40| +2450817|55814|2450905|30925|1164796|5067|6938|30925|1164796|5067|6938|2|90|11|1|8006|169|120|99|96.55|144.82|114.40|3011.58|11325.60|9558.45|14337.18|792.79|0.00|5447.97|11325.60|12118.39|16773.57|17566.36|1767.15| +2450817|55814|2450896|30925|1164796|5067|6938|30925|1164796|5067|6938|2|12|9|3|15752|85|120|95|26.83|28.97|2.02|2560.25|191.90|2548.85|2752.15|0.00|0.00|329.65|191.90|191.90|521.55|521.55|-2356.95| +2450817|55814|2450867|30925|1164796|5067|6938|30925|1164796|5067|6938|2|6|20|2|12098|156|120|57|13.44|18.00|10.80|410.40|615.60|766.08|1026.00|55.40|0.00|389.88|615.60|671.00|1005.48|1060.88|-150.48| +2450817|55814|2450862|30925|1164796|5067|6938|30925|1164796|5067|6938|2|76|17|1|2270|284|120|95|67.00|123.28|89.99|3162.55|8549.05|6365.00|11711.60|341.96|0.00|585.20|8549.05|8891.01|9134.25|9476.21|2184.05| +2450817|55814|2450868|30925|1164796|5067|6938|30925|1164796|5067|6938|2|6|7|1|2836|74|120|14|58.73|126.26|74.49|724.78|1042.86|822.22|1767.64|62.57|0.00|565.60|1042.86|1105.43|1608.46|1671.03|220.64| +2450817|55814|2450872|30925|1164796|5067|6938|30925|1164796|5067|6938|2|81|5|1|908|158|120|56|67.15|104.75|35.61|3871.84|1994.16|3760.40|5866.00|59.82|0.00|938.56|1994.16|2053.98|2932.72|2992.54|-1766.24| +2450817|55814|2450830|30925|1164796|5067|6938|30925|1164796|5067|6938|2|51|16|4|10153|217|120|24|39.55|69.60|2.78|1603.68|66.72|949.20|1670.40|1.33|0.00|835.20|66.72|68.05|901.92|903.25|-882.48| +2450817|55814|2450895|30925|1164796|5067|6938|30925|1164796|5067|6938|2|67|14|3|14032|228|120|82|58.57|98.98|52.45|3815.46|4300.90|4802.74|8116.36|301.06|0.00|3814.64|4300.90|4601.96|8115.54|8416.60|-501.84| +2450817|49847|2450859|48640|760948|6056|38364|48640|760948|6056|38364|2|4|20|2|3400|163|121|50|31.19|65.49|3.92|3078.50|196.00|1559.50|3274.50|7.29|13.72|1604.50|182.28|189.57|1786.78|1794.07|-1377.22| +2450817|49847|2450865|48640|760948|6056|38364|48640|760948|6056|38364|2|25|20|2|1153|297|121|67|37.65|89.23|85.66|239.19|5739.22|2522.55|5978.41|114.78|0.00|716.90|5739.22|5854.00|6456.12|6570.90|3216.67| +2450817|49847|2450840|48640|760948|6056|38364|48640|760948|6056|38364|2|64|14|2|10094|239|121|53|66.73|150.14|25.52|6604.86|1352.56|3536.69|7957.42|81.15|0.00|3660.18|1352.56|1433.71|5012.74|5093.89|-2184.13| +||||||38364|48640|||38364|2|56|||12232|10|121|72|||20.81||1498.32|1116.72|2724.48||0.00||1498.32||2260.80|2260.80|381.60| +2450817|81787|2450849|49591|1384831|2024|30864|49591|1384831|2024|30864|4|60|20|5|12181|192|122|85|75.74|209.79|90.20|10165.15|7667.00|6437.90|17832.15|29.13|6210.27|1604.80|1456.73|1485.86|3061.53|3090.66|-4981.17| +2450817|81787|2450835|49591|1384831|2024|30864|49591|1384831|2024|30864|4|55|13|3|949|300|122|68|50.08|129.20|125.32|263.84|8521.76|3405.44|8785.60|766.95|0.00|2196.40|8521.76|9288.71|10718.16|11485.11|5116.32| +2450817|81787|2450890|49591|1384831|2024|30864|49591|1384831|2024|30864|4|64|5|5|14629|206|122|76|64.16|165.53|148.97|1258.56|11321.72|4876.16|12580.28|566.08|0.00|1760.92|11321.72|11887.80|13082.64|13648.72|6445.56| +2450817|81787|2450845|49591|1384831|2024|30864|49591|1384831|2024|30864|4|75|3|4|15610|21|122|38|75.68|136.98|90.40|1770.04|3435.20|2875.84|5205.24|0.00|0.00|2550.56|3435.20|3435.20|5985.76|5985.76|559.36| +2450817|81787|2450868|49591|1384831|2024|30864|49591|1384831|2024|30864|4|43|10|4|10063|120|122|88|91.29|257.43|15.44|21295.12|1358.72|8033.52|22653.84|122.28|0.00|2944.48|1358.72|1481.00|4303.20|4425.48|-6674.80| +2450817|81787|2450865|49591|1384831|2024|30864|49591|1384831|2024|30864|4|7|18|2|15692|292|122|87|81.25|124.31|93.23|2703.96|8111.01|7068.75|10814.97|243.33|0.00|864.78|8111.01|8354.34|8975.79|9219.12|1042.26| +2450817|81787|2450879|49591|1384831|2024|30864|49591|1384831|2024|30864|4|84|3|4|12784|284|122|94|16.10|43.14|27.60|1460.76|2594.40|1513.40|4055.16|103.77|0.00|2027.58|2594.40|2698.17|4621.98|4725.75|1081.00| +2450817|81787|2450862|49591|1384831|2024|30864|49591|1384831|2024|30864|4|1|4|4|15772|297|122|15|94.71|279.39|223.51|838.20|3352.65|1420.65|4190.85|134.10|0.00|586.65|3352.65|3486.75|3939.30|4073.40|1932.00| +2450817|81787|2450854|49591|1384831|2024|30864|49591|1384831|2024|30864|4|93|3|1|13376|100|122|41|55.57|157.26|7.86|6125.40|322.26|2278.37|6447.66|6.38|3.22|2256.64|319.04|325.42|2575.68|2582.06|-1959.33| +2450817|81787|2450819|49591|1384831|2024|30864|49591|1384831|2024|30864|4|38|13|2|17522|22|122|50|15.75|30.08|30.08|0.00|1504.00|787.50|1504.00|15.04|0.00|105.00|1504.00|1519.04|1609.00|1624.04|716.50| +2450817|65337|2450867|57876|1264678|5880|49882|57876|1264678|5880|49882|2|75|3|4|10774|174|123|58|71.91|185.52|50.09|7854.94|2905.22|4170.78|10760.16|87.15|0.00|4626.66|2905.22|2992.37|7531.88|7619.03|-1265.56| +2450817|65337|2450904|57876|1264678|5880|49882|57876|1264678|5880|49882|2|30|7|3|5428|285|123|42|11.57|11.91|4.64|305.34|194.88|485.94|500.22|17.53|0.00|239.82|194.88|212.41|434.70|452.23|-291.06| +2450817|65337|2450869|57876|1264678|5880|49882|57876|1264678|5880|49882|2|18|1|5|4504|86|123|48|12.79|34.91|17.45|838.08|837.60|613.92|1675.68|0.00|0.00|418.56|837.60|837.60|1256.16|1256.16|223.68| +2450817|65337|2450890|57876|1264678|5880|49882|57876|1264678|5880|49882|2|51|2|5|7819|71|123|76|5.69|7.28|0.21|537.32|15.96|432.44|553.28|0.47|0.00|187.72|15.96|16.43|203.68|204.15|-416.48| +2450817|65337|2450851|57876|1264678|5880|49882|57876|1264678|5880|49882|2|50|9|4|4174|92|123|3|45.78|63.17|42.95|60.66|128.85|137.34|189.51|7.73|0.00|7.56|128.85|136.58|136.41|144.14|-8.49| +2450817|65337|2450858|57876|1264678|5880|49882|57876|1264678|5880|49882|2|19|13|4|6727|201|123|12|60.86|177.71|58.64|1428.84|703.68|730.32|2132.52|19.63|485.53|895.56|218.15|237.78|1113.71|1133.34|-512.17| +2450817|65337|2450822|57876|1264678|5880|49882|57876|1264678|5880|49882|2|97|5|1|15547|230|123|78|47.98|68.61|39.10|2301.78|3049.80|3742.44|5351.58|147.91|91.49|2407.86|2958.31|3106.22|5366.17|5514.08|-784.13| +2450817|65337|2450905|57876|1264678|5880|49882|57876|1264678|5880|49882|2|29|17|5|7888|259|123|40|65.19|125.16|46.30|3154.40|1852.00|2607.60|5006.40|129.64|0.00|1952.40|1852.00|1981.64|3804.40|3934.04|-755.60| +2450817|64435|2450877|15744|1377262|2780|11498|15744|1377262|2780|11498|4|16|11|2|11236|239|124|61|23.19|51.48|49.93|94.55|3045.73|1414.59|3140.28|30.45|0.00|1507.31|3045.73|3076.18|4553.04|4583.49|1631.14| +2450817|64435|2450855|15744|1377262|2780|11498|15744|1377262|2780|11498|4|54|17|4|3472|283|124|68|64.74|85.45|29.05|3835.20|1975.40|4402.32|5810.60|39.50|0.00|2498.32|1975.40|2014.90|4473.72|4513.22|-2426.92| +2450817|64435|2450829|15744|1377262|2780|11498|15744|1377262|2780|11498|4|2|7|1|15614|64|124|25|8.66|21.56|21.56|0.00|539.00|216.50|539.00|3.44|495.88|220.75|43.12|46.56|263.87|267.31|-173.38| +2450817|64435|2450875|15744|1377262|2780|11498|15744|1377262|2780|11498|4|90|13|5|2438|114|124|43|64.21|120.07|54.03|2839.72|2323.29|2761.03|5163.01|46.46|0.00|1755.26|2323.29|2369.75|4078.55|4125.01|-437.74| +2450817|64435|2450881|15744|1377262|2780|11498|15744|1377262|2780|11498|4|3|11|2|7432|225|124|5|68.50|84.94|1.69|416.25|8.45|342.50|424.70|0.00|4.39|135.90|4.06|4.06|139.96|139.96|-338.44| +2450817|64435|2450824|15744|1377262|2780|11498|15744|1377262|2780|11498|4|33|17|5|15854|178|124|52|39.47|44.99|27.44|912.60|1426.88|2052.44|2339.48|71.34|0.00|303.68|1426.88|1498.22|1730.56|1801.90|-625.56| +2450817|64435|2450907|15744|1377262|2780|11498|15744|1377262|2780|11498|4|4|19|1|14486|200|124|19|93.48|196.30|172.74|447.64|3282.06|1776.12|3729.70|32.82|0.00|559.36|3282.06|3314.88|3841.42|3874.24|1505.94| +2450817|21961|2450835|68170|1197355|5293|40831|68170|1197355|5293|40831|2|46|11|5|7190|183|125|30|95.74|197.22|51.27|4378.50|1538.10|2872.20|5916.60|26.45|876.71|1952.40|661.39|687.84|2613.79|2640.24|-2210.81| +2450817|21961|2450884|68170|1197355|5293|40831|68170|1197355|5293|40831|2|64|19|3|3931|87|125|88|13.34|20.01|7.00|1144.88|616.00|1173.92|1760.88|43.12|0.00|105.60|616.00|659.12|721.60|764.72|-557.92| +2450817|21961|2450872|68170|1197355|5293|40831|68170|1197355|5293|40831|2|32|18|1|13261|167|125|16|79.62|160.03|1.60|2534.88|25.60|1273.92|2560.48|0.25|0.00|640.00|25.60|25.85|665.60|665.85|-1248.32| +2450817|21961|2450855|68170|1197355|5293|40831|68170|1197355|5293|40831|2|73|1|2|2863|142|125|1|35.95|98.86|61.29|37.57|61.29|35.95|98.86|2.20|24.51|4.94|36.78|38.98|41.72|43.92|0.83| +2450817|21961|2450901|68170|1197355|5293|40831|68170|1197355|5293|40831|2|11|14|2|10321|201|125|27|80.66|207.29|26.94|4869.45|727.38|2177.82|5596.83|65.46|0.00|2518.56|727.38|792.84|3245.94|3311.40|-1450.44| +2450817|21961|2450852|68170|1197355|5293|40831|68170|1197355|5293|40831|2|71|17|5|12256|63|125|6|13.64|20.32|18.69|9.78|112.14|81.84|121.92|0.00|49.34|45.06|62.80|62.80|107.86|107.86|-19.04| +2450817|21961|2450839|68170|1197355|5293|40831|68170|1197355|5293|40831|2|20|11|3|2764|299|125|12|32.96|38.89|22.55|196.08|270.60|395.52|466.68|16.23|0.00|107.28|270.60|286.83|377.88|394.11|-124.92| +2450817|21961|2450883|68170|1197355|5293|40831|68170|1197355|5293|40831|2|75|5|1|3878|201|125|6|79.52|163.81|111.39|314.52|668.34|477.12|982.86|0.00|507.93|88.44|160.41|160.41|248.85|248.85|-316.71| +2450817|21961|2450852|68170|1197355|5293|40831|68170|1197355|5293|40831|2|107|8|3|15865|158|125|20|6.16|6.89|1.30|111.80|26.00|123.20|137.80|1.30|0.00|45.40|26.00|27.30|71.40|72.70|-97.20| +2450817|21961|2450863|68170|1197355|5293|40831|68170|1197355|5293|40831|2|95|4|3|14386|270|125|67|35.80|38.66|9.27|1969.13|621.09|2398.60|2590.22|37.26|0.00|336.34|621.09|658.35|957.43|994.69|-1777.51| +2450817|21961|2450842|68170|1197355|5293|40831|68170|1197355|5293|40831|2|97|14|5|14806|180|125|10|67.94|134.52|0.00|1345.20|0.00|679.40|1345.20|0.00|0.00|403.50|0.00|0.00|403.50|403.50|-679.40| +2450817|21961|2450840|68170|1197355|5293|40831|68170|1197355|5293|40831|2|62|2|2|2308|284|125|88|42.65|127.95|34.54|8220.08|3039.52|3753.20|11259.60|243.16|0.00|1238.16|3039.52|3282.68|4277.68|4520.84|-713.68| +2450817|21961|2450852|68170|1197355|5293|40831|68170|1197355|5293|40831|2|45|5|1|3544|175|125|59|2.57|3.64|1.45|129.21|85.55|151.63|214.76|1.71|0.00|57.82|85.55|87.26|143.37|145.08|-66.08| +2450817|21961|2450851|68170|1197355|5293|40831|68170|1197355|5293|40831|2|55|4|1|10438|175|125|78|68.75|193.87|124.07|5444.40|9677.46|5362.50|15121.86|870.97|0.00|2872.74|9677.46|10548.43|12550.20|13421.17|4314.96| +2450817|67504|2450835|231|1339159|6887|39415|231|1339159|6887|39415|2|14|7|3|6530|267|126|17|93.25|242.45|145.47|1648.66|2472.99|1585.25|4121.65|49.45|0.00|41.14|2472.99|2522.44|2514.13|2563.58|887.74| +2450817|67504|2450855|231|1339159|6887|39415|231|1339159|6887|39415|2|94|4|2|17440|90|126|93|68.09|186.56|126.86|5552.10|11797.98|6332.37|17350.08|707.87|0.00|4857.39|11797.98|12505.85|16655.37|17363.24|5465.61| +2450817|67504|2450903|231|1339159|6887|39415|231|1339159|6887|39415|2|19|10|2|11866|21|126|43|23.08|23.31|2.56|892.25|110.08|992.44|1002.33|1.67|89.16|110.08|20.92|22.59|131.00|132.67|-971.52| +2450817|67504|2450852|231|1339159|6887|39415|231|1339159|6887|39415|2|63|12|2|9658|106|126|16|20.08|34.73|12.85|350.08|205.60|321.28|555.68|12.33|0.00|16.64|205.60|217.93|222.24|234.57|-115.68| +2450817|67504|2450820|231|1339159|6887|39415|231|1339159|6887|39415|2|94|19|5|16120|166|126|93|41.71|64.23|13.48|4719.75|1253.64|3879.03|5973.39|100.29|0.00|1911.15|1253.64|1353.93|3164.79|3265.08|-2625.39| +2450817|67504|2450876|231|1339159|6887|39415|231|1339159|6887|39415|2|32|16|2|17348|276|126|74|3.96|5.54|2.21|246.42|163.54|293.04|409.96|11.44|0.00|130.98|163.54|174.98|294.52|305.96|-129.50| +2450817|67504|2450901|231|1339159|6887|39415|231|1339159|6887|39415|2|86|8|3|4498|53|126|100|63.18|183.22|174.05|917.00|17405.00|6318.00|18322.00|121.83|11313.25|732.00|6091.75|6213.58|6823.75|6945.58|-226.25| +2450817|67504|2450855|231|1339159|6887|39415|231|1339159|6887|39415|2|69|12|4|15343|268|126|26|27.60|63.20|56.24|180.96|1462.24|717.60|1643.20|73.11|0.00|82.16|1462.24|1535.35|1544.40|1617.51|744.64| +2450817|67504|2450842|231|1339159|6887|39415|231|1339159|6887|39415|2|67|20|1|17924|20|126|22|54.99|92.38|30.48|1361.80|670.56|1209.78|2032.36|46.93|0.00|914.54|670.56|717.49|1585.10|1632.03|-539.22| +2450817|67504|2450891|231|1339159|6887|39415|231|1339159|6887|39415|2|2|18|5|13954|67|126|21|25.69|42.64|8.10|725.34|170.10|539.49|895.44|10.20|0.00|187.95|170.10|180.30|358.05|368.25|-369.39| +2450817|67504|2450833|231|1339159|6887|39415|231|1339159|6887|39415|2|44|7|5|17036|44|126|44|98.85|234.27|67.93|7318.96|2988.92|4349.40|10307.88|49.31|2002.57|4638.48|986.35|1035.66|5624.83|5674.14|-3363.05| +2450817|1409|2450822|44366|910495|3377|38259|62441|725670|2552|2234|1|107|10|2|3620|27|127|31|22.23|36.23|7.60|887.53|235.60|689.13|1123.13|16.49|0.00|303.18|235.60|252.09|538.78|555.27|-453.53| +2450817|1409|2450865|44366|910495|3377|38259|62441|725670|2552|2234|1|59|18|5|11224|24|127|35|95.52|126.08|16.39|3839.15|573.65|3343.20|4412.80|32.12|114.73|352.80|458.92|491.04|811.72|843.84|-2884.28| +2450817|1409|2450863|44366|910495|3377|38259|62441|725670|2552|2234|1|62|9|1|10723|140|127|63|47.65|126.27|97.22|1830.15|6124.86|3001.95|7955.01|0.00|0.00|2147.67|6124.86|6124.86|8272.53|8272.53|3122.91| +2450817|1409|2450855|44366|910495|3377|38259|62441|725670|2552|2234|1|31|7|2|12595|291|127|28|17.80|50.73|42.61|227.36|1193.08|498.40|1420.44|83.51|0.00|638.96|1193.08|1276.59|1832.04|1915.55|694.68| +2450817|1409|2450891|44366|910495|3377|38259|62441|725670|2552|2234|1|56|8|3|17570|228|127|70|94.97|116.81|47.89|4824.40|3352.30|6647.90|8176.70|234.66|0.00|2125.90|3352.30|3586.96|5478.20|5712.86|-3295.60| +2450817|2524|2450865|74038|992741|4597|22853|74038|992741|4597|22853|4|102|15|3|1756|107|128|42|18.77|44.67|13.40|1313.34|562.80|788.34|1876.14|50.65|0.00|769.02|562.80|613.45|1331.82|1382.47|-225.54| +2450817|2524|2450853|74038|992741|4597|22853|74038|992741|4597|22853|4|54|20|5|11041|96|128|32|84.87|178.22|131.88|1482.88|4220.16|2715.84|5703.04|337.61|0.00|2851.52|4220.16|4557.77|7071.68|7409.29|1504.32| +2450817|2524|2450845|74038|992741|4597|22853|74038|992741|4597|22853|4|6|11|3|3892|222|128|82|73.14|134.57|0.00|11034.74|0.00|5997.48|11034.74|0.00|0.00|2758.48|0.00|0.00|2758.48|2758.48|-5997.48| +2450817|2524|2450888|74038|992741|4597|22853|74038|992741|4597|22853|4|19|20|3|121|233|128|14|2.28|3.48|2.61|12.18|36.54|31.92|48.72|2.19|0.00|14.00|36.54|38.73|50.54|52.73|4.62| +2450817|2524|2450902|74038|992741|4597|22853|74038|992741|4597|22853|4|15|4|2|11422|170|128|6|64.32|68.82|53.67|90.90|322.02|385.92|412.92|16.10|0.00|173.40|322.02|338.12|495.42|511.52|-63.90| +2450817|2524|2450850|74038|992741|4597|22853|74038|992741|4597|22853|4|91|17|5|16159|29|128|94|54.61|97.20|76.78|1919.48|7217.32|5133.34|9136.80|0.00|0.00|1827.36|7217.32|7217.32|9044.68|9044.68|2083.98| +2450817|2524|2450881|74038|992741|4597|22853|74038|992741|4597|22853|4|51|18|3|11509|204|128|100|36.20|76.74|6.13|7061.00|613.00|3620.00|7674.00|25.01|300.37|0.00|312.63|337.64|312.63|337.64|-3307.37| +2450817|2524|2450831|74038|992741|4597|22853|74038|992741|4597|22853|4|42|12|3|17599|75|128|16|2.15|2.32|1.90|6.72|30.40|34.40|37.12|1.68|6.38|16.96|24.02|25.70|40.98|42.66|-10.38| +2450817|2524|2450856|74038|992741|4597|22853|74038|992741|4597|22853|4|77|10|2|3112|144|128|20|55.74|70.23|30.19|800.80|603.80|1114.80|1404.60|18.11|0.00|154.40|603.80|621.91|758.20|776.31|-511.00| +2450817|2524|2450832|74038|992741|4597|22853|74038|992741|4597|22853|4|6|3|2|757|185|128|84|12.25|22.90|18.09|404.04|1519.56|1029.00|1923.60|60.78|0.00|845.88|1519.56|1580.34|2365.44|2426.22|490.56| +2450817|2524|2450838|74038|992741|4597|22853|74038|992741|4597|22853|4|16|2|2|11618|117|128|81|63.46|151.66|147.11|368.55|11915.91|5140.26|12284.46|834.11|0.00|5895.99|11915.91|12750.02|17811.90|18646.01|6775.65| +2450817|22746|2450833|35557|1428629|101|2326|35557|1428629|101|2326|2|38|8|4|913|248|129|41|10.59|26.05|20.31|235.34|832.71|434.19|1068.05|0.00|0.00|330.87|832.71|832.71|1163.58|1163.58|398.52| +2450817|22746|2450885|35557|1428629|101|2326|35557|1428629|101|2326|2|17|2|3|11876|60|129|30|87.81|199.32|115.60|2511.60|3468.00|2634.30|5979.60|34.68|0.00|2929.80|3468.00|3502.68|6397.80|6432.48|833.70| +2450817|22746|2450867|35557|1428629|101|2326|35557|1428629|101|2326|2|22|6|1|13759|186|129|53|86.75|202.99|2.02|10651.41|107.06|4597.75|10758.47|1.07|0.00|429.83|107.06|108.13|536.89|537.96|-4490.69| +2450817|22746|2450841|35557|1428629|101|2326|35557|1428629|101|2326|2|55|15|5|6164|217|129|15|4.60|5.70|2.73|44.55|40.95|69.00|85.50|1.34|24.16|25.65|16.79|18.13|42.44|43.78|-52.21| +2450817|22746|2450823|35557|1428629|101|2326|35557|1428629|101|2326|2|6|4|4|15712|286|129|19|94.64|148.58|49.03|1891.45|931.57|1798.16|2823.02|7.45|186.31|338.58|745.26|752.71|1083.84|1091.29|-1052.90| +2450817|22746|2450878|35557|1428629|101|2326|35557|1428629|101|2326|2|65|20|5|12985|21|129|72|51.25|130.17|91.11|2812.32|6559.92|3690.00|9372.24|131.19|0.00|2061.36|6559.92|6691.11|8621.28|8752.47|2869.92| +2450817|22746|2450855|35557|1428629|101|2326|35557|1428629|101|2326|2|14|9|2|17104|206|129|37|30.36|85.61|32.53|1963.96|1203.61|1123.32|3167.57|7.58|1119.35|633.44|84.26|91.84|717.70|725.28|-1039.06| +2450817|35814|2450846|59708|366991|227|37510|59708|366991|227|37510|2|105|16|4|12283|280|130|92|75.20|96.25|33.68|5756.44|3098.56|6918.40|8855.00|154.92|0.00|3806.96|3098.56|3253.48|6905.52|7060.44|-3819.84| +2450817|35814|2450893|59708|366991|227|37510|59708|366991|227|37510|2|12|15|5|1573|300|130|34|67.73|138.84|20.82|4012.68|707.88|2302.82|4720.56|14.15|0.00|1416.10|707.88|722.03|2123.98|2138.13|-1594.94| +2450817|35814|2450890|59708|366991|227|37510|59708|366991|227|37510|2|80|15|5|4234|109|130|19|27.18|76.37|64.91|217.74|1233.29|516.42|1451.03|24.66|0.00|667.47|1233.29|1257.95|1900.76|1925.42|716.87| +2450817|35814|2450843|59708|366991|227|37510|59708|366991|227|37510|2|12|4|2|8251|276|130|26|18.51|41.09|37.39|96.20|972.14|481.26|1068.34|68.04|0.00|512.72|972.14|1040.18|1484.86|1552.90|490.88| +2450817|35814|2450870|59708|366991|227|37510|59708|366991|227|37510|2|78|5|4|9188|117|130|83|35.12|85.34|58.88|2196.18|4887.04|2914.96|7083.22|342.09|0.00|2903.34|4887.04|5229.13|7790.38|8132.47|1972.08| +2450817|35814|2450831|59708|366991|227|37510|59708|366991|227|37510|2|18|8|5|4802|217|130|74|5.71|10.84|10.18|48.84|753.32|422.54|802.16|21.69|210.92|344.84|542.40|564.09|887.24|908.93|119.86| +2450817|35814|2450902|59708|366991|227|37510|59708|366991|227|37510|2|89|8|5|14098|124|130|9|79.44|177.15|40.74|1227.69|366.66|714.96|1594.35|18.33|0.00|334.80|366.66|384.99|701.46|719.79|-348.30| +2450817|35814|2450835|59708|366991|227|37510|59708|366991|227|37510|2|108|20|2|1369|161|130|81|30.07|76.37|38.18|3093.39|3092.58|2435.67|6185.97|278.33|0.00|1669.41|3092.58|3370.91|4761.99|5040.32|656.91| +2450817|35814|2450819|59708|366991|227|37510|59708|366991|227|37510|2|11|16|2|6505|103|130|53|70.65|167.44|3.34|8697.30|177.02|3744.45|8874.32|5.41|116.83|2484.64|60.19|65.60|2544.83|2550.24|-3684.26| +2450817|47716|2450834|42669|55491|1560|40429|42669|55491|1560|40429|2|35|5|5|17392|282|131|31|23.92|66.97|18.75|1494.82|581.25|741.52|2076.07|52.31|0.00|747.10|581.25|633.56|1328.35|1380.66|-160.27| +2450817|47716|2450884|42669|55491|1560|40429|42669|55491|1560|40429|2|48|17|1|7744|286|131|69|18.78|41.69|28.34|921.15|1955.46|1295.82|2876.61|136.88|0.00|834.21|1955.46|2092.34|2789.67|2926.55|659.64| +2450817|47716|2450894|42669|55491|1560|40429|42669|55491|1560|40429|2|77|2|1|15007|242|131|96|30.78|64.02|55.05|861.12|5284.80|2954.88|6145.92|0.00|0.00|1105.92|5284.80|5284.80|6390.72|6390.72|2329.92| +2450817|47716|2450820|42669|55491|1560|40429|42669|55491|1560|40429|2|100|12|2|12266|140|131|27|96.85|197.57|120.51|2080.62|3253.77|2614.95|5334.39|162.68|0.00|1973.70|3253.77|3416.45|5227.47|5390.15|638.82| +2450817|47716|2450877|42669|55491|1560|40429|42669|55491|1560|40429|2|82|3|3|3475|184|131|60|58.48|173.68|158.04|938.40|9482.40|3508.80|10420.80|284.47|0.00|3230.40|9482.40|9766.87|12712.80|12997.27|5973.60| +2450817|47716|2450860|42669|55491|1560|40429|42669|55491|1560|40429|2|69|10|4|13969|204|131|69|25.88|37.52|22.51|1035.69|1553.19|1785.72|2588.88|139.78|0.00|776.25|1553.19|1692.97|2329.44|2469.22|-232.53| +2450817|47716|2450825|42669|55491|1560|40429|42669|55491|1560|40429|2|39|16|4|12469|283|131|11|71.07|94.52|19.84|821.48|218.24|781.77|1039.72|17.45|0.00|405.46|218.24|235.69|623.70|641.15|-563.53| +2450817|47716|2450905|42669|55491|1560|40429|42669|55491|1560|40429|2|69|6|1|10108|231|131|13|60.21|76.46|61.16|198.90|795.08|782.73|993.98|55.65|0.00|327.99|795.08|850.73|1123.07|1178.72|12.35| +2450817|68618|2450851|18036|422690|1391|4492|18036|422690|1391|4492|1|9|14|3|1282|154|132|36|85.18|119.25|96.59|815.76|3477.24|3066.48|4293.00|278.17|0.00|1502.28|3477.24|3755.41|4979.52|5257.69|410.76| +2450817|68618|2450848|18036|422690|1391|4492|18036|422690|1391|4492|1|36|16|5|10987|30|132|85|76.77|164.28|4.92|13545.60|418.20|6525.45|13963.80|25.09|0.00|4747.25|418.20|443.29|5165.45|5190.54|-6107.25| +2450817|68618|2450849|18036|422690|1391|4492|18036|422690|1391|4492|1|104|4|5|11884|98|132|76|80.01|110.41|90.53|1510.88|6880.28|6080.76|8391.16|0.00|0.00|2685.08|6880.28|6880.28|9565.36|9565.36|799.52| +2450817|68618|2450821|18036|422690|1391|4492|18036|422690|1391|4492|1|8|2|3|9664|108|132|76|17.19|51.05|40.32|815.48|3064.32|1306.44|3879.80|153.21|0.00|1241.08|3064.32|3217.53|4305.40|4458.61|1757.88| +2450817|81700|2450823|46455|1497524|3346|36815|46455|1497524|3346|36815|4|8|1|2|17047|272|133|61|29.49|73.43|6.60|4076.63|402.60|1798.89|4479.23|20.13|0.00|1612.23|402.60|422.73|2014.83|2034.96|-1396.29| +2450817|81700|2450840|46455|1497524|3346|36815|46455|1497524|3346|36815|4|90|1|5|607|222|133|29|24.61|63.49|58.41|147.32|1693.89|713.69|1841.21|67.75|0.00|202.42|1693.89|1761.64|1896.31|1964.06|980.20| +2450817|81700|2450874|46455|1497524|3346|36815|46455|1497524|3346|36815|4|62|1|2|9820|258|133|43|50.10|146.29|122.88|1006.63|5283.84|2154.30|6290.47|158.51|0.00|817.43|5283.84|5442.35|6101.27|6259.78|3129.54| +2450817|81700|2450849|46455|1497524|3346|36815|46455|1497524|3346|36815|4|78|20|5|2461|71|133|47|82.96|219.01|194.91|1132.70|9160.77|3899.12|10293.47|549.64|0.00|4425.99|9160.77|9710.41|13586.76|14136.40|5261.65| +2450817|81700|2450839|46455|1497524|3346|36815|46455|1497524|3346|36815|4|101|17|5|13543|148|133|55|50.30|104.62|85.78|1036.20|4717.90|2766.50|5754.10|0.00|0.00|805.20|4717.90|4717.90|5523.10|5523.10|1951.40| +2450817|81700|2450828|46455|1497524|3346|36815|46455|1497524|3346|36815|4|3|18|5|17371|289|133|50|31.42|44.30|2.21|2104.50|110.50|1571.00|2215.00|4.42|0.00|376.50|110.50|114.92|487.00|491.42|-1460.50| +2450817|81700|2450853|46455|1497524|3346|36815|46455|1497524|3346|36815|4|4|3|1|3308|185|133|65|16.93|36.56|17.18|1259.70|1116.70|1100.45|2376.40|100.50|0.00|1092.65|1116.70|1217.20|2209.35|2309.85|16.25| +2450817|70564|2450883|60796|1675763|683|40003|60796|1675763|683|40003|4|87|1|4|9172|14|134|23|46.54|89.35|81.30|185.15|1869.90|1070.42|2055.05|18.69|0.00|904.13|1869.90|1888.59|2774.03|2792.72|799.48| +2450817|70564|2450868|60796|1675763|683|40003|60796|1675763|683|40003|4|12|15|4|15758|47|134|19|93.13|230.96|55.43|3335.07|1053.17|1769.47|4388.24|21.06|0.00|2150.23|1053.17|1074.23|3203.40|3224.46|-716.30| +2450817|70564|2450899|60796|1675763|683|40003|60796|1675763|683|40003|4|13|12|5|15616|226|134|67|59.09|101.63|23.37|5243.42|1565.79|3959.03|6809.21|62.63|0.00|3268.26|1565.79|1628.42|4834.05|4896.68|-2393.24| +2450817|70564|2450857|60796|1675763|683|40003|60796|1675763|683|40003|4|62|9|3|17500|86|134|30|60.31|118.81|95.04|713.10|2851.20|1809.30|3564.30|256.60|0.00|35.40|2851.20|3107.80|2886.60|3143.20|1041.90| +2450817|70564|2450819|60796|1675763|683|40003|60796|1675763|683|40003|4|51|1|5|12877|21|134|70|46.80|55.69|20.04|2495.50|1402.80|3276.00|3898.30|112.22|0.00|1169.00|1402.80|1515.02|2571.80|2684.02|-1873.20| +2450817|70564|2450832|60796|1675763|683|40003|60796|1675763|683|40003|4|1|15|4|3220|233|134|52|56.28|69.22|44.99|1259.96|2339.48|2926.56|3599.44|163.76|0.00|863.72|2339.48|2503.24|3203.20|3366.96|-587.08| +2450817|70564|2450870|60796|1675763|683|40003|60796|1675763|683|40003|4|98|19|4|14090|154|134|23|82.71|238.20|92.89|3342.13|2136.47|1902.33|5478.60|149.55|0.00|1698.32|2136.47|2286.02|3834.79|3984.34|234.14| +2450817|70564|2450822|60796|1675763|683|40003|60796|1675763|683|40003|4|42|1|1|9080|61|134|90|63.48|96.48|2.89|8423.10|260.10|5713.20|8683.20|15.60|0.00|1997.10|260.10|275.70|2257.20|2272.80|-5453.10| +2450817|70564|2450839|60796|1675763|683|40003|60796|1675763|683|40003|4|1|17|2|9769|99|134|72|72.84|194.48|122.52|5181.12|8821.44|5244.48|14002.56|88.21|0.00|2800.08|8821.44|8909.65|11621.52|11709.73|3576.96| +2450817|70564|2450873|60796|1675763|683|40003|60796|1675763|683|40003|4|2|4|2|8218|51|134|32|10.12|10.32|8.46|59.52|270.72|323.84|330.24|5.41|0.00|145.28|270.72|276.13|416.00|421.41|-53.12| +2450817|70564|2450900|60796|1675763|683|40003|60796|1675763|683|40003|4|57|7|5|3062|174|134|53|15.36|36.71|13.21|1245.50|700.13|814.08|1945.63|49.00|0.00|544.31|700.13|749.13|1244.44|1293.44|-113.95| +2450817|70564|2450823|60796|1675763|683|40003|60796|1675763|683|40003|4|47|19|5|6673|155|134|59|4.27|5.33|1.22|242.49|71.98|251.93|314.47|0.64|59.02|65.49|12.96|13.60|78.45|79.09|-238.97| +2450817|70564|2450832|60796|1675763|683|40003|60796|1675763|683|40003|4|12|5|2|15446|99|134|17|48.28|75.79|29.55|786.08|502.35|820.76|1288.43|20.09|0.00|309.06|502.35|522.44|811.41|831.50|-318.41| +2450817|70564|2450835|60796|1675763|683|40003|60796|1675763|683|40003|4|36|11|1|1903|11|134|58|2.01|4.26|3.66|34.80|212.28|116.58|247.08|1.69|169.82|123.54|42.46|44.15|166.00|167.69|-74.12| +2450817|54723|2450866|37695|192413|1978|31896|37695|192413|1978|31896|4|102|14|5|5102|165|135|52|6.46|18.47|17.54|48.36|912.08|335.92|960.44|18.24|0.00|144.04|912.08|930.32|1056.12|1074.36|576.16| +2450817|54723|2450851|37695|192413|1978|31896|37695|192413|1978|31896|4|32|17|2|9986|159|135|62|69.45|103.48|28.97|4619.62|1796.14|4305.90|6415.76|35.92|0.00|897.76|1796.14|1832.06|2693.90|2729.82|-2509.76| +2450817|54723|2450868|37695|192413|1978|31896|37695|192413|1978|31896|4|8|20|5|9236|210|135|75|7.51|19.45|10.30|686.25|772.50|563.25|1458.75|23.17|0.00|583.50|772.50|795.67|1356.00|1379.17|209.25| +2450817|54723|2450894|37695|192413|1978|31896|37695|192413|1978|31896|4|23|1|2|17806|211|135|91|95.23|222.83|53.47|15411.76|4865.77|8665.93|20277.53|389.26|0.00|7907.90|4865.77|5255.03|12773.67|13162.93|-3800.16| +2450817|54723|2450882|37695|192413|1978|31896|37695|192413|1978|31896|4|97|18|1|4255|24|135|46|42.54|51.89|28.53|1074.56|1312.38|1956.84|2386.94|0.00|0.00|453.10|1312.38|1312.38|1765.48|1765.48|-644.46| +2450817|54723|2450901|37695|192413|1978|31896|37695|192413|1978|31896|4|14|5|2|926|81|135|57|84.28|246.09|219.02|1542.99|12484.14|4803.96|14027.13|374.52|0.00|981.54|12484.14|12858.66|13465.68|13840.20|7680.18| +2450817|54723|2450880|37695|192413|1978|31896|37695|192413|1978|31896|4|30|16|2|17590|15|135|33|89.98|251.94|0.00|8314.02|0.00|2969.34|8314.02|0.00|0.00|165.99|0.00|0.00|165.99|165.99|-2969.34| +2450817|54723|2450903|37695|192413|1978|31896|37695|192413|1978|31896|4|76|20|4|17629|36|135|27|10.39|12.67|10.76|51.57|290.52|280.53|342.09|23.24|0.00|82.08|290.52|313.76|372.60|395.84|9.99| +2450817|54723|2450869|37695|192413|1978|31896|37695|192413|1978|31896|4|94|1|1|11095|145|135|70|64.14|80.81|67.88|905.10|4751.60|4489.80|5656.70|112.13|1948.15|2149.00|2803.45|2915.58|4952.45|5064.58|-1686.35| +2450817|54723|2450870|37695|192413|1978|31896|37695|192413|1978|31896|4|67|2|2|14653|294|135|75|48.71|143.69|90.52|3987.75|6789.00|3653.25|10776.75|271.56|0.00|1185.00|6789.00|7060.56|7974.00|8245.56|3135.75| +2450817|55162|2450897|56896|970825|4401|4730|56896|970825|4401|4730|1|35|15|5|664|13|136|79|90.17|133.45|24.02|8644.97|1897.58|7123.43|10542.55|170.78|0.00|2635.44|1897.58|2068.36|4533.02|4703.80|-5225.85| +2450817|55162|2450829|56896|970825|4401|4730|56896|970825|4401|4730|1|35|8|4|13804|235|136|74|95.37|257.49|195.69|4573.20|14481.06|7057.38|19054.26|78.19|11874.46|4763.38|2606.60|2684.79|7369.98|7448.17|-4450.78| +2450817|55162|2450821|56896|970825|4401|4730|56896|970825|4401|4730|1|51|18|5|11020|176|136|78|52.89|94.14|76.25|1395.42|5947.50|4125.42|7342.92|118.95|0.00|2936.70|5947.50|6066.45|8884.20|9003.15|1822.08| +2450817|55162|2450890|56896|970825|4401|4730|56896|970825|4401|4730|1|17|10|2|2224|224|136|9|2.10|5.20|1.71|31.41|15.39|18.90|46.80|0.30|0.00|0.00|15.39|15.69|15.39|15.69|-3.51| +2450817|55162|2450850|56896|970825|4401|4730|56896|970825|4401|4730|1|64|3|3|13552|252|136|11|41.38|41.79|14.62|298.87|160.82|455.18|459.69|2.60|30.55|160.82|130.27|132.87|291.09|293.69|-324.91| +2450817|35420|2450885|10000|1248263|5726|49189|10000|1248263|5726|49189|2|82|15|3|10582|146|137|45|43.80|92.85|59.42|1504.35|2673.90|1971.00|4178.25|80.21|0.00|1378.80|2673.90|2754.11|4052.70|4132.91|702.90| +2450817|35420|2450893|10000|1248263|5726|49189|10000|1248263|5726|49189|2|62|8|3|14671|16|137|30|88.29|96.23|67.36|866.10|2020.80|2648.70|2886.90|181.87|0.00|461.70|2020.80|2202.67|2482.50|2664.37|-627.90| +2450817|35420|2450890|10000|1248263|5726|49189|10000|1248263|5726|49189|2|7|20|4|1208|95|137|54|12.88|29.36|26.71|143.10|1442.34|695.52|1585.44|0.00|346.16|380.16|1096.18|1096.18|1476.34|1476.34|400.66| +2450817|35420|2450835|10000|1248263|5726|49189|10000|1248263|5726|49189|2|23|1|1|6550|82|137|79|83.11|92.25|11.99|6340.54|947.21|6565.69|7287.75|85.24|0.00|2404.76|947.21|1032.45|3351.97|3437.21|-5618.48| +2450817|63345|2450848|3084|629483|329|19928|3084|629483|329|19928|1|95|4|4|15187|117|138|23|56.89|137.67|27.53|2533.22|633.19|1308.47|3166.41|25.32|0.00|728.18|633.19|658.51|1361.37|1386.69|-675.28| +2450817|63345|2450854|3084|629483|329|19928|3084|629483|329|19928|1|53|11|2|6160|66|138|90|93.75|193.12|38.62|13905.00|3475.80|8437.50|17380.80|69.51|0.00|7647.30|3475.80|3545.31|11123.10|11192.61|-4961.70| +2450817|63345|2450841|3084|629483|329|19928|3084|629483|329|19928|1|17|2|5|12448|126|138|53|94.15|112.98|28.24|4491.22|1496.72|4989.95|5987.94|14.96|0.00|2754.41|1496.72|1511.68|4251.13|4266.09|-3493.23| +2450817|63345|2450838|3084|629483|329|19928|3084|629483|329|19928|1|39|9|2|15002|8|138|65|19.88|24.05|11.78|797.55|765.70|1292.20|1563.25|20.44|84.22|734.50|681.48|701.92|1415.98|1436.42|-610.72| +2450817|63345|2450830|3084|629483|329|19928|3084|629483|329|19928|1|2|16|2|5047|286|138|72|28.65|39.82|1.59|2752.56|114.48|2062.80|2867.04|3.43|0.00|401.04|114.48|117.91|515.52|518.95|-1948.32| +2450817|63345|2450824|3084|629483|329|19928|3084|629483|329|19928|1|96|8|5|1711|141|138|52|31.22|42.77|23.52|1001.00|1223.04|1623.44|2224.04|48.92|0.00|800.28|1223.04|1271.96|2023.32|2072.24|-400.40| +2450817|63345|2450903|3084|629483|329|19928|3084|629483|329|19928|1|7|2|5|6487|106|138|66|90.51|231.70|222.43|611.82|14680.38|5973.66|15292.20|734.01|0.00|2293.50|14680.38|15414.39|16973.88|17707.89|8706.72| +2450817|63345|2450823|3084|629483|329|19928|3084|629483|329|19928|1|96|2|3|10123|250|138|36|88.57|240.91|26.50|7718.76|954.00|3188.52|8672.76|85.86|0.00|867.24|954.00|1039.86|1821.24|1907.10|-2234.52| +2450817|63345|2450856|3084|629483|329|19928|3084|629483|329|19928|1|34|12|2|4357|235|138|49|65.54|140.91|63.40|3797.99|3106.60|3211.46|6904.59|93.19|0.00|2485.28|3106.60|3199.79|5591.88|5685.07|-104.86| +2450817|63345|2450824|3084|629483|329|19928|3084|629483|329|19928|1|80|2|2|17624|96|138|23|1.05|1.53|1.14|8.97|26.22|24.15|35.19|2.35|0.00|5.06|26.22|28.57|31.28|33.63|2.07| +2450817|63345|2450883|3084|629483|329|19928|3084|629483|329|19928|1|18|9|4|910|296|138|83|31.12|75.62|19.66|4644.68|1631.78|2582.96|6276.46|146.86|0.00|2886.74|1631.78|1778.64|4518.52|4665.38|-951.18| +2450817|64670|2450853|23571|1153135|746|14710|23571|1153135|746|14710|4|28|15|5|5821|263|139|45|48.26|90.72|60.78|1347.30|2735.10|2171.70|4082.40|0.00|0.00|1592.10|2735.10|2735.10|4327.20|4327.20|563.40| +2450817|64670|2450868|23571|1153135|746|14710|23571|1153135|746|14710|4|99|7|3|6002|255|139|28|44.09|129.62|102.39|762.44|2866.92|1234.52|3629.36|200.68|0.00|1741.88|2866.92|3067.60|4608.80|4809.48|1632.40| +2450817|64670|2450819|23571|1153135|746|14710|23571|1153135|746|14710|4|10|8|1|6008|159|139|94|67.97|195.75|187.92|736.02|17664.48|6389.18|18400.50|1589.80|0.00|2391.36|17664.48|19254.28|20055.84|21645.64|11275.30| +2450817|64670|2450840|23571|1153135|746|14710|23571|1153135|746|14710|4|43|5|1|8122|103|139|68|46.22|51.30|13.85|2546.60|941.80|3142.96|3488.40|4.89|819.36|278.80|122.44|127.33|401.24|406.13|-3020.52| +2450817|64670|2450839|23571|1153135|746|14710|23571|1153135|746|14710|4|98|9|5|15718|185|139|90|85.88|170.04|88.42|7345.80|7957.80|7729.20|15303.60|85.94|6525.39|1530.00|1432.41|1518.35|2962.41|3048.35|-6296.79| +2450817|64670|2450888|23571|1153135|746|14710|23571|1153135|746|14710|4|45|9|3|15001|128|139|22|64.30|77.16|70.98|135.96|1561.56|1414.60|1697.52|37.47|936.93|763.84|624.63|662.10|1388.47|1425.94|-789.97| +2450817|64670|2450885|23571|1153135|746|14710|23571|1153135|746|14710|4|50|2|3|10618|252|139|94|97.56|182.43|20.06|15262.78|1885.64|9170.64|17148.42|169.70|0.00|1200.38|1885.64|2055.34|3086.02|3255.72|-7285.00| +2450817|64670|2450866|23571|1153135|746|14710|23571|1153135|746|14710|4|79|4|3|13363|251|139|21|43.35|125.28|117.76|157.92|2472.96|910.35|2630.88|74.18|0.00|78.75|2472.96|2547.14|2551.71|2625.89|1562.61| +2450817|64670|2450830|23571|1153135|746|14710|23571|1153135|746|14710|4|70|14|2|5780|135|139|33|4.21|12.25|11.51|24.42|379.83|138.93|404.25|30.38|0.00|104.94|379.83|410.21|484.77|515.15|240.90| +2450817|64670|2450887|23571|1153135|746|14710|23571|1153135|746|14710|4|103|7|3|10670|270|139|66|87.83|107.15|57.86|3253.14|3818.76|5796.78|7071.90|114.56|0.00|2899.38|3818.76|3933.32|6718.14|6832.70|-1978.02| +2450817|64670|2450883|23571|1153135|746|14710|23571|1153135|746|14710|4|64|18|1|8719|170|139|69|91.18|203.33|30.49|11925.96|2103.81|6291.42|14029.77|42.07|0.00|1823.67|2103.81|2145.88|3927.48|3969.55|-4187.61| +2450817|17737|2450819|55100|1141712|4771|28983|55100|1141712|4771|28983|1|41|2|3|11408|98|140|40|4.39|7.72|3.08|185.60|123.20|175.60|308.80|0.76|85.00|151.20|38.20|38.96|189.40|190.16|-137.40| +2450817|17737|2450882|55100|1141712|4771|28983|55100|1141712|4771|28983|1|45|15|5|1030|210|140|89|90.78|182.46|12.77|15102.41|1136.53|8079.42|16238.94|90.92|0.00|6820.07|1136.53|1227.45|7956.60|8047.52|-6942.89| +2450817|17737|2450838|55100|1141712|4771|28983|55100|1141712|4771|28983|1|45|7|2|4340|209|140|55|76.74|80.57|50.75|1640.10|2791.25|4220.70|4431.35|139.56|0.00|442.75|2791.25|2930.81|3234.00|3373.56|-1429.45| +2450817|17737|2450906|55100|1141712|4771|28983|55100|1141712|4771|28983|1|58|7|3|6535|213|140|10|73.78|128.37|60.33|680.40|603.30|737.80|1283.70|36.19|0.00|372.20|603.30|639.49|975.50|1011.69|-134.50| +2450817|17737|2450852|55100|1141712|4771|28983|55100|1141712|4771|28983|1|53|18|5|15466|266|140|67|83.23|117.35|11.73|7076.54|785.91|5576.41|7862.45|23.57|0.00|550.07|785.91|809.48|1335.98|1359.55|-4790.50| +2450817|17737|2450873|55100|1141712|4771|28983|55100|1141712|4771|28983|1|43|12|3|12457|52|140|27|45.77|47.14|36.29|292.95|979.83|1235.79|1272.78|0.00|0.00|598.05|979.83|979.83|1577.88|1577.88|-255.96| +2450817|17737|2450864|55100|1141712|4771|28983|55100|1141712|4771|28983|1|63|15|3|2005|99|140|13|10.79|12.40|0.86|150.02|11.18|140.27|161.20|0.11|0.00|43.42|11.18|11.29|54.60|54.71|-129.09| +2450817|17737|2450862|55100|1141712|4771|28983|55100|1141712|4771|28983|1|4|6|5|8506|133|140|87|62.80|165.16|46.24|10346.04|4022.88|5463.60|14368.92|160.91|0.00|4597.95|4022.88|4183.79|8620.83|8781.74|-1440.72| +2450817|17737|2450869|55100|1141712|4771|28983|55100|1141712|4771|28983|1|79|16|5|14392|113|140|100|34.56|48.38|16.44|3194.00|1644.00|3456.00|4838.00|32.88|0.00|387.00|1644.00|1676.88|2031.00|2063.88|-1812.00| +2450817|17737|2450885|55100|1141712|4771|28983|55100|1141712|4771|28983|1|106|15|2|14330|284|140|79|57.77|66.43|57.79|682.56|4565.41|4563.83|5247.97|136.96|0.00|2413.45|4565.41|4702.37|6978.86|7115.82|1.58| +2450817|17737|2450854|55100|1141712|4771|28983|55100|1141712|4771|28983|1|89|7|2|2260|239|140|14|99.92|158.87|123.91|489.44|1734.74|1398.88|2224.18|126.28|156.12|978.60|1578.62|1704.90|2557.22|2683.50|179.74| +2450817|17737|2450850|55100|1141712|4771|28983|55100|1141712|4771|28983|1|4|15|3|3583|55|140|62|14.39|30.65|12.26|1140.18|760.12|892.18|1900.30|0.00|0.00|284.58|760.12|760.12|1044.70|1044.70|-132.06| +2450817|17737|2450889|55100|1141712|4771|28983|55100|1141712|4771|28983|1|6|20|3|1232|298|140|71|1.50|3.55|2.69|61.06|190.99|106.50|252.05|0.00|0.00|72.42|190.99|190.99|263.41|263.41|84.49| +2450817|15899|2450846|25767|1128606|2645|29084|25767|1128606|2645|29084|1|74|15|2|16280|83|141|52|28.06|67.90|22.40|2366.00|1164.80|1459.12|3530.80|81.53|0.00|247.00|1164.80|1246.33|1411.80|1493.33|-294.32| +2450817|15899|2450896|25767|1128606|2645|29084|25767|1128606|2645|29084|1|27|13|5|14803|260|141|64|77.09|97.13|82.56|932.48|5283.84|4933.76|6216.32|105.67|0.00|2858.88|5283.84|5389.51|8142.72|8248.39|350.08| +2450817|15899|2450907|25767|1128606|2645|29084|25767|1128606|2645|29084|1|30|7|2|3970|222|141|23|11.14|29.52|4.72|570.40|108.56|256.22|678.96|4.34|0.00|325.68|108.56|112.90|434.24|438.58|-147.66| +2450817|15899|2450839|25767|1128606|2645|29084|25767|1128606|2645|29084|1|56|14|3|6178|147|141|53|80.41|168.05|10.08|8372.41|534.24|4261.73|8906.65|26.71|0.00|3116.93|534.24|560.95|3651.17|3677.88|-3727.49| +2450817|15899|2450849|25767|1128606|2645|29084|25767|1128606|2645|29084|1|3|6|3|17690|222|141|71|93.72|189.31|7.57|12903.54|537.47|6654.12|13441.01|48.37|0.00|3225.53|537.47|585.84|3763.00|3811.37|-6116.65| +2450817|15899|2450837|25767|1128606|2645|29084|25767|1128606|2645|29084|1|39|17|2|16951|197|141|83|17.99|45.87|18.34|2284.99|1522.22|1493.17|3807.21|91.33|0.00|684.75|1522.22|1613.55|2206.97|2298.30|29.05| +2450817|15899|2450881|25767|1128606|2645|29084|25767|1128606|2645|29084|1|104|18|1|12919|295|141|37|67.52|175.55|26.33|5521.14|974.21|2498.24|6495.35|58.45|0.00|2013.54|974.21|1032.66|2987.75|3046.20|-1524.03| +2450817|15899|2450824|25767|1128606|2645|29084|25767|1128606|2645|29084|1|90|9|2|2374|3|141|8|39.50|96.38|46.26|400.96|370.08|316.00|771.04|29.60|0.00|154.16|370.08|399.68|524.24|553.84|54.08| +2450817|15899|2450877|25767|1128606|2645|29084|25767|1128606|2645|29084|1|87|6|4|10597|25|141|26|53.59|74.49|72.25|58.24|1878.50|1393.34|1936.74|37.57|0.00|232.18|1878.50|1916.07|2110.68|2148.25|485.16| +2450817|15899|2450871|25767|1128606|2645|29084|25767|1128606|2645|29084|1|76|16|3|16274|152|141|41|72.23|203.68|73.32|5344.76|3006.12|2961.43|8350.88|180.36|0.00|1586.29|3006.12|3186.48|4592.41|4772.77|44.69| +2450817|17789|2450850|66201|84080|6312|16944|86158|1179855|1287|40366|2|44|6|1|8012|261|142|9|97.12|168.98|143.63|228.15|1292.67|874.08|1520.82|77.56|0.00|288.90|1292.67|1370.23|1581.57|1659.13|418.59| +2450817|17789|2450833|66201|84080|6312|16944|86158|1179855|1287|40366|2|62|7|1|4633|37|142|12|34.85|58.19|5.81|628.56|69.72|418.20|698.28|6.27|0.00|125.64|69.72|75.99|195.36|201.63|-348.48| +2450817|17789|2450831|66201|84080|6312|16944|86158|1179855|1287|40366|2|25|1|2|4916|79|142|49|28.64|29.78|17.27|612.99|846.23|1403.36|1459.22|8.46|0.00|641.90|846.23|854.69|1488.13|1496.59|-557.13| +2450817|17789|2450904|66201|84080|6312|16944|86158|1179855|1287|40366|2|68|2|5|9346|268|142|28|59.60|67.34|22.89|1244.60|640.92|1668.80|1885.52|20.25|134.59|131.88|506.33|526.58|638.21|658.46|-1162.47| +2450817|17789|2450873|66201|84080|6312|16944|86158|1179855|1287|40366|2|43|13|5|820|100|142|85|63.06|99.63|17.93|6944.50|1524.05|5360.10|8468.55|121.92|0.00|2116.50|1524.05|1645.97|3640.55|3762.47|-3836.05| +2450817|17789|2450892|66201|84080|6312|16944|86158|1179855|1287|40366|2|8|9|3|5480|121|142|95|77.30|136.04|36.73|9434.45|3489.35|7343.50|12923.80|139.57|0.00|387.60|3489.35|3628.92|3876.95|4016.52|-3854.15| +2450817|17789|2450887|66201|84080|6312|16944|86158|1179855|1287|40366|2|99|4|3|15031|44|142|81|72.64|153.27|125.68|2234.79|10180.08|5883.84|12414.87|305.40|0.00|2233.98|10180.08|10485.48|12414.06|12719.46|4296.24| +2450817|17789|2450901|66201|84080|6312|16944|86158|1179855|1287|40366|2|44|12|5|16658|18|142|36|44.08|46.28|24.52|783.36|882.72|1586.88|1666.08|52.96|0.00|33.12|882.72|935.68|915.84|968.80|-704.16| +2450817|38095|2450875|64193|1454158|3676|13594|64193|1454158|3676|13594|2|81|6|2|3007|189|143|34|77.75|219.25|67.96|5143.86|2310.64|2643.50|7454.50|23.10|0.00|3279.98|2310.64|2333.74|5590.62|5613.72|-332.86| +2450817|38095|2450840|64193|1454158|3676|13594|64193|1454158|3676|13594|2|69|9|1|3748|262|143|81|23.16|68.09|53.79|1158.30|4356.99|1875.96|5515.29|348.55|0.00|2315.79|4356.99|4705.54|6672.78|7021.33|2481.03| +2450817|38095|2450834|64193|1454158|3676|13594|64193|1454158|3676|13594|2|37|12|5|238|138|143|78|16.58|49.07|35.82|1033.50|2793.96|1293.24|3827.46|167.63|0.00|1530.36|2793.96|2961.59|4324.32|4491.95|1500.72| +2450817|38095|2450896|64193|1454158|3676|13594|64193|1454158|3676|13594|2|89|16|3|15853|282|143|81|93.16|263.64|34.27|18578.97|2775.87|7545.96|21354.84|83.27|0.00|8327.61|2775.87|2859.14|11103.48|11186.75|-4770.09| +2450817|38095|2450851|64193|1454158|3676|13594|64193|1454158|3676|13594|2|90|14|1|8426|246|143|21|69.64|183.15|36.63|3076.92|769.23|1462.44|3846.15|30.76|0.00|576.87|769.23|799.99|1346.10|1376.86|-693.21| +2450817|38095|2450851|64193|1454158|3676|13594|64193|1454158|3676|13594|2|42|16|5|10945|270|143|22|97.95|283.07|25.47|5667.20|560.34|2154.90|6227.54|19.16|240.94|1494.46|319.40|338.56|1813.86|1833.02|-1835.50| +2450817|38095|2450889|64193|1454158|3676|13594|64193|1454158|3676|13594|2|85|19|5|12007|142|143|51|58.25|93.78|29.07|3300.21|1482.57|2970.75|4782.78|0.00|0.00|2104.26|1482.57|1482.57|3586.83|3586.83|-1488.18| +2450817|38095|2450847|64193|1454158|3676|13594|64193|1454158|3676|13594|2|2|18|1|16412|139|143|81|25.75|29.87|12.54|1403.73|1015.74|2085.75|2419.47|27.42|467.24|895.05|548.50|575.92|1443.55|1470.97|-1537.25| +2450817|38095|2450846|64193|1454158|3676|13594|64193|1454158|3676|13594|2|17|12|2|2210|300|143|77|1.66|2.98|1.37|123.97|105.49|127.82|229.46|3.16|0.00|40.81|105.49|108.65|146.30|149.46|-22.33| +2450817|38095|2450819|64193|1454158|3676|13594|64193|1454158|3676|13594|2|106|4|2|1880|16|143|93|83.00|89.64|21.51|6336.09|2000.43|7719.00|8336.52|60.01|0.00|2500.77|2000.43|2060.44|4501.20|4561.21|-5718.57| +2450817|38095|2450877|64193|1454158|3676|13594|64193|1454158|3676|13594|2|40|16|2|16360|107|143|79|46.28|60.62|41.82|1485.20|3303.78|3656.12|4788.98|264.30|0.00|1484.41|3303.78|3568.08|4788.19|5052.49|-352.34| +2450817|75147|2450902|82523|179338|5753|11768|94042|105798|213|17428|2|8|10|5|6172|240|144|79|43.50|47.41|1.89|3596.08|149.31|3436.50|3745.39|10.45|0.00|74.26|149.31|159.76|223.57|234.02|-3287.19| +2450817|75147|2450895|82523|179338|5753|11768|94042|105798|213|17428|2|91|10|1|10430|191|144|24|37.00|82.14|59.14|552.00|1419.36|888.00|1971.36|59.61|227.09|197.04|1192.27|1251.88|1389.31|1448.92|304.27| +2450817|75147|2450865|82523|179338|5753|11768|94042|105798|213|17428|2|70|4|3|4078|9|144|26|13.15|24.59|0.24|633.10|6.24|341.90|639.34|0.49|0.00|38.22|6.24|6.73|44.46|44.95|-335.66| +2450817|75147|2450862|82523|179338|5753|11768|94042|105798|213|17428|2|50|6|5|10490|94|144|77|24.93|62.82|45.23|1354.43|3482.71|1919.61|4837.14|208.96|0.00|1450.68|3482.71|3691.67|4933.39|5142.35|1563.10| +2450817|63901|2450863|8849|1789452|6055|24705|8849|1789452|6055|24705|2|26|16|5|7724|5|145|80|98.87|234.32|222.60|937.60|17808.00|7909.60|18745.60|712.32|0.00|3186.40|17808.00|18520.32|20994.40|21706.72|9898.40| +2450817|63901|2450871|8849|1789452|6055|24705|8849|1789452|6055|24705|2|43|2|4|13360|220|145|76|77.38|188.80|54.75|10187.80|4161.00|5880.88|14348.80|249.66|0.00|2008.68|4161.00|4410.66|6169.68|6419.34|-1719.88| +2450817|63901|2450850|8849|1789452|6055|24705|8849|1789452|6055|24705|2|43|9|5|10570|19|145|48|91.45|176.49|37.06|6692.64|1778.88|4389.60|8471.52|106.73|0.00|1185.60|1778.88|1885.61|2964.48|3071.21|-2610.72| +2450817|63901|2450905|8849|1789452|6055|24705|8849|1789452|6055|24705|2|77|12|2|15932|244|145|17|53.94|131.07|5.24|2139.11|89.08|916.98|2228.19|2.67|0.00|757.52|89.08|91.75|846.60|849.27|-827.90| +2450817|63901|2450876|8849|1789452|6055|24705|8849|1789452|6055|24705|2|10|16|2|3799|279|145|77|95.24|163.81|40.95|9460.22|3153.15|7333.48|12613.37|283.78|0.00|3026.87|3153.15|3436.93|6180.02|6463.80|-4180.33| +2450817|63901|2450890|8849|1789452|6055|24705|8849|1789452|6055|24705|2|88|19|4|17272|94|145|34|62.54|65.66|60.40|178.84|2053.60|2126.36|2232.44|112.12|451.79|892.84|1601.81|1713.93|2494.65|2606.77|-524.55| +2450817|63901|2450863|8849|1789452|6055|24705|8849|1789452|6055|24705|2|94|18|5|17326|232|145|16|52.46|127.47|54.81|1162.56|876.96|839.36|2039.52|8.76|0.00|305.92|876.96|885.72|1182.88|1191.64|37.60| +2450817|63901|2450859|8849|1789452|6055|24705|8849|1789452|6055|24705|2|38|14|2|16219|204|145|13|67.63|116.32|67.46|635.18|876.98|879.19|1512.16|17.53|0.00|15.08|876.98|894.51|892.06|909.59|-2.21| +2450817|63901|2450874|8849|1789452|6055|24705|8849|1789452|6055|24705|2|38|18|1|10045|75|145|58|30.61|73.77|37.62|2096.70|2181.96|1775.38|4278.66|109.09|0.00|85.26|2181.96|2291.05|2267.22|2376.31|406.58| +2450817|85899|2450869|23362|1730166|4970|25774|85290|457382|2586|28990|2|105|4|3|7045|210|146|67|73.66|83.97|51.22|2194.25|3431.74|4935.22|5625.99|205.90|0.00|2643.82|3431.74|3637.64|6075.56|6281.46|-1503.48| +2450817|85899|2450849|23362|1730166|4970|25774|85290|457382|2586|28990|2|94|5|2|7063|106|146|13|32.93|58.94|14.14|582.40|183.82|428.09|766.22|2.02|143.37|15.21|40.45|42.47|55.66|57.68|-387.64| +2450817|85899|2450871|23362|1730166|4970|25774|85290|457382|2586|28990|2|32|7|1|3478|104|146|70|24.27|45.87|16.05|2087.40|1123.50|1698.90|3210.90|42.13|280.87|31.50|842.63|884.76|874.13|916.26|-856.27| +|85899|2450849||1730166|4970|||457382|2586||||||2612||146||23.27||7.83|3145.80||1396.20||18.79|0.00|||488.59|2096.40|2115.19|-926.40| +2450817|85899|2450900|23362|1730166|4970|25774|85290|457382|2586|28990|2|53|7|5|8302|79|146|96|51.72|130.33|1.30|12386.88|124.80|4965.12|12511.68|1.24|0.00|1875.84|124.80|126.04|2000.64|2001.88|-4840.32| +2450817|85899|2450898|23362|1730166|4970|25774|85290|457382|2586|28990|2|54|17|2|6127|172|146|6|69.58|148.20|111.15|222.30|666.90|417.48|889.20|1.46|593.54|409.02|73.36|74.82|482.38|483.84|-344.12| +2450817|85899|2450841|23362|1730166|4970|25774|85290|457382|2586|28990|2|64|2|2|9607|210|146|86|91.37|178.17|71.26|9194.26|6128.36|7857.82|15322.62|306.41|0.00|765.40|6128.36|6434.77|6893.76|7200.17|-1729.46| +2450817|85899|2450903|23362|1730166|4970|25774|85290|457382|2586|28990|2|58|8|3|3758|135|146|14|41.61|48.68|0.48|674.80|6.72|582.54|681.52|0.47|0.00|136.22|6.72|7.19|142.94|143.41|-575.82| +2450817|75948|2450871|85927|1791050|4173|24896|85927|1791050|4173|24896|4|72|12|1|16396|112|147|54|54.56|79.65|43.80|1935.90|2365.20|2946.24|4301.10|23.65|0.00|1075.14|2365.20|2388.85|3440.34|3463.99|-581.04| +2450817|75948|2450880|85927|1791050|4173|24896|85927|1791050|4173|24896|4|74|16|4|2552|192|147|10|21.87|39.58|2.77|368.10|27.70|218.70|395.80|1.93|0.00|27.70|27.70|29.63|55.40|57.33|-191.00| +2450817|75948|2450884|85927|1791050|4173|24896|85927|1791050|4173|24896|4|4|8|2|1744|125|147|95|17.77|38.02|36.11|181.45|3430.45|1688.15|3611.90|137.21|0.00|1552.30|3430.45|3567.66|4982.75|5119.96|1742.30| +2450817|75948|2450879|85927|1791050|4173|24896|85927|1791050|4173|24896|4|32|2|1|9025|267|147|23|93.62|177.87|149.41|654.58|3436.43|2153.26|4091.01|137.45|0.00|613.64|3436.43|3573.88|4050.07|4187.52|1283.17| +2450817|75948|2450902|85927|1791050|4173|24896|85927|1791050|4173|24896|4|30|4|5|2918|104|147|100|12.55|30.87|13.27|1760.00|1327.00|1255.00|3087.00|39.81|0.00|493.00|1327.00|1366.81|1820.00|1859.81|72.00| +2450817|16479|2450905|29687|384404|5968|32332|29687|384404|5968|32332|1|101|19|5|1702|104|148|15|76.86|215.97|144.69|1069.20|2170.35|1152.90|3239.55|43.40|0.00|356.25|2170.35|2213.75|2526.60|2570.00|1017.45| +2450817|16479|2450839|29687|384404|5968|32332|29687|384404|5968|32332|1|60|10|3|15422|135|148|74|39.19|94.05|47.02|3480.22|3479.48|2900.06|6959.70|278.35|0.00|486.92|3479.48|3757.83|3966.40|4244.75|579.42| +2450817|16479|2450880|29687|384404|5968|32332|29687|384404|5968|32332|1|9|16|4|3968|3|148|85|2.56|7.55|5.43|180.20|461.55|217.60|641.75|27.69|0.00|5.95|461.55|489.24|467.50|495.19|243.95| +2450817|16479|2450872|29687|384404|5968|32332|29687|384404|5968|32332|1|13|18|1|6385|265|148|53|50.32|56.86|33.54|1235.96|1777.62|2666.96|3013.58|88.88|0.00|1476.58|1777.62|1866.50|3254.20|3343.08|-889.34| +2450817|16479|2450894|29687|384404|5968|32332|29687|384404|5968|32332|1|86|6|4|7640|32|148|8|15.19|45.41|15.43|239.84|123.44|121.52|363.28|2.46|0.00|178.00|123.44|125.90|301.44|303.90|1.92| +2450817|16479|2450862|29687|384404|5968|32332|29687|384404|5968|32332|1|84|11|5|10333|26|148|25|63.75|89.88|29.66|1505.50|741.50|1593.75|2247.00|29.66|0.00|112.25|741.50|771.16|853.75|883.41|-852.25| +2450817|16479|2450893|29687|384404|5968|32332|29687|384404|5968|32332|1|94|17|3|1712|78|148|78|72.26|172.70|10.36|12662.52|808.08|5636.28|13470.60|64.64|0.00|3367.26|808.08|872.72|4175.34|4239.98|-4828.20| +2450817|16479|2450853|29687|384404|5968|32332|29687|384404|5968|32332|1|35|6|3|13916|267|148|14|16.36|26.99|11.60|215.46|162.40|229.04|377.86|8.12|0.00|185.08|162.40|170.52|347.48|355.60|-66.64| +2450817|16479|2450882|29687|384404|5968|32332|29687|384404|5968|32332|1|23|13|1|14764|282|148|5|36.39|82.60|21.47|305.65|107.35|181.95|413.00|0.00|98.76|169.30|8.59|8.59|177.89|177.89|-173.36| +2450817|16479|2450848|29687|384404|5968|32332|29687|384404|5968|32332|1|47|9|4|16406|172|148|19|82.41|99.71|26.92|1383.01|511.48|1565.79|1894.49|5.11|0.00|644.10|511.48|516.59|1155.58|1160.69|-1054.31| +2450817|80167|2450830|86122|1510244|4413|13335|86122|1510244|4413|13335|2|104|17|5|4292|68|149|100|96.36|174.41|27.90|14651.00|2790.00|9636.00|17441.00|139.50|0.00|3488.00|2790.00|2929.50|6278.00|6417.50|-6846.00| +2450817|80167|2450907|86122|1510244|4413|13335|86122|1510244|4413|13335|2|29|4|1|4682|243|149|96|93.90|241.32|139.96|9730.56|13436.16|9014.40|23166.72|134.36|0.00|6485.76|13436.16|13570.52|19921.92|20056.28|4421.76| +2450817|80167|2450869|86122|1510244|4413|13335|86122|1510244|4413|13335|2|85|16|4|11905|57|149|100|87.32|247.11|59.30|18781.00|5930.00|8732.00|24711.00|225.34|1423.20|6424.00|4506.80|4732.14|10930.80|11156.14|-4225.20| +2450817|80167|2450870|86122|1510244|4413|13335|86122|1510244|4413|13335|2|74|4|5|17110|100|149|16|20.63|44.97|3.14|669.28|50.24|330.08|719.52|2.00|0.00|273.28|50.24|52.24|323.52|325.52|-279.84| +2450817|80167|2450906|86122|1510244|4413|13335|86122|1510244|4413|13335|2|41|20|2|10861|157|149|6|90.56|109.57|97.51|72.36|585.06|543.36|657.42|40.95|0.00|91.98|585.06|626.01|677.04|717.99|41.70| +2450817|80167|2450839|86122|1510244|4413|13335|86122|1510244|4413|13335|2|4|12|3|10513|13|149|12|38.13|109.05|37.07|863.76|444.84|457.56|1308.60|40.03|0.00|588.84|444.84|484.87|1033.68|1073.71|-12.72| +2450817|80167|2450876|86122|1510244|4413|13335|86122|1510244|4413|13335|2|29|11|2|9772|269|149|70|6.13|16.73|9.36|515.90|655.20|429.10|1171.10|32.76|0.00|81.90|655.20|687.96|737.10|769.86|226.10| +2450817|37574|2450838|98227|1014475|7170|23018|98227|1014475|7170|23018|4|104|2|4|13390|53|150|12|96.76|190.61|120.08|846.36|1440.96|1161.12|2287.32|115.27|0.00|91.44|1440.96|1556.23|1532.40|1647.67|279.84| +2450817|37574|2450864|98227|1014475|7170|23018|98227|1014475|7170|23018|4|57|12|4|15086|258|150|21|90.66|228.46|27.41|4222.05|575.61|1903.86|4797.66|35.05|74.82|1727.04|500.79|535.84|2227.83|2262.88|-1403.07| +2450817|37574|2450880|98227|1014475|7170|23018|98227|1014475|7170|23018|4|50|5|5|9584|48|150|13|52.45|52.45|36.19|211.38|470.47|681.85|681.85|42.34|0.00|54.47|470.47|512.81|524.94|567.28|-211.38| +2450817|37574|2450863|98227|1014475|7170|23018|98227|1014475|7170|23018|4|69|19|2|10370|111|150|73|73.00|171.55|97.78|5385.21|7137.94|5329.00|12523.15|71.37|0.00|625.61|7137.94|7209.31|7763.55|7834.92|1808.94| +2450817|37574|2450838|98227|1014475|7170|23018|98227|1014475|7170|23018|4|87|2|3|3574|137|150|35|95.62|154.90|105.33|1734.95|3686.55|3346.70|5421.50|51.61|2654.31|2114.35|1032.24|1083.85|3146.59|3198.20|-2314.46| +2450817|37574|2450878|98227|1014475|7170|23018|98227|1014475|7170|23018|4|7|15|2|919|226|150|56|8.48|22.47|18.42|226.80|1031.52|474.88|1258.32|72.20|0.00|465.36|1031.52|1103.72|1496.88|1569.08|556.64| +2450817|37574|2450861|98227|1014475|7170|23018|98227|1014475|7170|23018|4|57|11|5|9565|121|150|35|35.21|82.74|38.06|1563.80|1332.10|1232.35|2895.90|79.92|0.00|1332.10|1332.10|1412.02|2664.20|2744.12|99.75| +2450817|37574|2450901|98227|1014475|7170|23018|98227|1014475|7170|23018|4|66|7|3|14612|195|150|50|58.75|59.33|51.02|415.50|2551.00|2937.50|2966.50|127.55|0.00|207.50|2551.00|2678.55|2758.50|2886.05|-386.50| +2450817|37574|2450821|98227|1014475|7170|23018|98227|1014475|7170|23018|4|66|18|3|12092|41|150|40|36.79|70.26|35.13|1405.20|1405.20|1471.60|2810.40|14.05|0.00|702.40|1405.20|1419.25|2107.60|2121.65|-66.40| +2450817|37574|2450875|98227|1014475|7170|23018|98227|1014475|7170|23018|4|61|13|4|649|225|150|39|81.42|100.96|5.04|3740.88|196.56|3175.38|3937.44|3.93|0.00|1929.33|196.56|200.49|2125.89|2129.82|-2978.82| +2450817|84836|2450848|63742|1759443|5897|30798|63742|1759443|5897|30798|4|26|2|1|11470|169|151|50|37.22|45.03|40.97|203.00|2048.50|1861.00|2251.50|0.00|0.00|337.50|2048.50|2048.50|2386.00|2386.00|187.50| +2450817|84836|2450897|63742|1759443|5897|30798|63742|1759443|5897|30798|4|46|14|2|1444|110|151|74|1.91|4.04|1.73|170.94|128.02|141.34|298.96|1.28|0.00|0.00|128.02|129.30|128.02|129.30|-13.32| +2450817|84836|2450873|63742|1759443|5897|30798|63742|1759443|5897|30798|4|85|17|2|5944|65|151|41|17.90|26.49|4.23|912.66|173.43|733.90|1086.09|4.50|83.24|542.84|90.19|94.69|633.03|637.53|-643.71| +2450817|84836|2450879|63742|1759443|5897|30798|63742|1759443|5897|30798|4|74|16|4|9916|118|151|94|3.28|5.70|2.85|267.90|267.90|308.32|535.80|0.75|249.14|192.70|18.76|19.51|211.46|212.21|-289.56| +2450817|84836|2450832|63742|1759443|5897|30798|63742|1759443|5897|30798|4|66|10|2|5156|298|151|32|11.44|25.05|10.27|472.96|328.64|366.08|801.60|11.83|32.86|400.64|295.78|307.61|696.42|708.25|-70.30| +2450817|84836|2450819|63742|1759443|5897|30798|63742|1759443|5897|30798|4|41|16|1|8774|141|151|88|57.56|57.56|56.40|102.08|4963.20|5065.28|5065.28|75.44|1191.16|0.00|3772.04|3847.48|3772.04|3847.48|-1293.24| +2450817|66442|2450830|58945|1447314|4930|49299|58945|1447314|4930|49299|1|9|15|4|14320|20|152|10|8.65|18.59|3.71|148.80|37.10|86.50|185.90|0.74|0.00|24.10|37.10|37.84|61.20|61.94|-49.40| +2450817|66442|2450861|58945|1447314|4930|49299|58945|1447314|4930|49299|1|84|17|4|7316|58|152|71|29.21|35.34|33.21|151.23|2357.91|2073.91|2509.14|141.47|0.00|802.30|2357.91|2499.38|3160.21|3301.68|284.00| +2450817|66442|2450890|58945|1447314|4930|49299|58945|1447314|4930|49299|1|16|1|1|6859|228|152|53|51.83|124.39|36.07|4680.96|1911.71|2746.99|6592.67|38.23|0.00|2966.41|1911.71|1949.94|4878.12|4916.35|-835.28| +2450817|66442|2450828|58945|1447314|4930|49299|58945|1447314|4930|49299|1|49|20|4|7963|139|152|38|46.38|101.10|10.11|3457.62|384.18|1762.44|3841.80|7.68|0.00|576.08|384.18|391.86|960.26|967.94|-1378.26| +2450817|42109|2450876|61429|1133714|3246|1931|61429|1133714|3246|1931|1|23|18|1|9088|113|153|15|50.06|91.10|25.50|984.00|382.50|750.90|1366.50|15.26|164.47|491.85|218.03|233.29|709.88|725.14|-532.87| +2450817|42109|2450821|61429|1133714|3246|1931|61429|1133714|3246|1931|1|2|3|5|12788|277|153|13|15.48|24.76|12.62|157.82|164.06|201.24|321.88|0.19|144.37|157.69|19.69|19.88|177.38|177.57|-181.55| +2450817|42109|2450901|61429|1133714|3246|1931|61429|1133714|3246|1931|1|28|15|2|16666|266|153|7|21.61|62.23|16.17|322.42|113.19|151.27|435.61|9.05|0.00|139.37|113.19|122.24|252.56|261.61|-38.08| +2450817|42109|2450882|61429|1133714|3246|1931|61429|1133714|3246|1931|1|26|15|4|16330|215|153|41|85.26|133.85|29.44|4280.81|1207.04|3495.66|5487.85|108.63|0.00|1756.03|1207.04|1315.67|2963.07|3071.70|-2288.62| +2450817|42109|2450884|61429|1133714|3246|1931|61429|1133714|3246|1931|1|32|8|3|9092|192|153|31|52.87|124.24|42.24|2542.00|1309.44|1638.97|3851.44|91.66|0.00|1540.39|1309.44|1401.10|2849.83|2941.49|-329.53| +2450817|42109|2450899|61429|1133714|3246|1931|61429|1133714|3246|1931|1|81|6|4|17038|27|153|1|5.38|11.67|4.90|6.77|4.90|5.38|11.67|0.08|2.69|3.73|2.21|2.29|5.94|6.02|-3.17| +2450817|42109|2450876|61429|1133714|3246|1931|61429|1133714|3246|1931|1|44|15|3|14228|49|153|15|50.97|87.15|86.27|13.20|1294.05|764.55|1307.25|90.58|0.00|91.50|1294.05|1384.63|1385.55|1476.13|529.50| +2450817|42109|2450907|61429|1133714|3246|1931|61429|1133714|3246|1931|1|51|8|2|9982|47|153|93|84.16|132.13|104.38|2580.75|9707.34|7826.88|12288.09|47.56|9027.82|5528.85|679.52|727.08|6208.37|6255.93|-7147.36| +2450817|42109|2450861|61429|1133714|3246|1931|61429|1133714|3246|1931|1|60|19|1|15694|195|153|91|44.04|76.18|30.47|4159.61|2772.77|4007.64|6932.38|33.82|1081.38|3049.41|1691.39|1725.21|4740.80|4774.62|-2316.25| +2450817|42109|2450907|61429|1133714|3246|1931|61429|1133714|3246|1931|1|7|8|4|236|98|153|21|64.04|162.66|107.35|1161.51|2254.35|1344.84|3415.86|135.26|0.00|1536.99|2254.35|2389.61|3791.34|3926.60|909.51| +2450817|42109|2450878|61429|1133714|3246|1931|61429|1133714|3246|1931|1|88|7|3|11245|112|153|44|98.99|202.92|0.00|8928.48|0.00|4355.56|8928.48|0.00|0.00|3481.72|0.00|0.00|3481.72|3481.72|-4355.56| +2450817|42109|2450907|61429|1133714|3246|1931|61429|1133714|3246|1931|1|12|13|4|7706|118|153|46|31.33|36.65|6.59|1382.76|303.14|1441.18|1685.90|6.06|0.00|741.52|303.14|309.20|1044.66|1050.72|-1138.04| +2450817|14471|2450872|90839|1856128|5064|36000|90839|1856128|5064|36000|2|28|3|2|12712|58|154|82|9.83|28.11|8.15|1636.72|668.30|806.06|2305.02|26.73|0.00|691.26|668.30|695.03|1359.56|1386.29|-137.76| +2450817|14471|2450835|90839|1856128|5064|36000|90839|1856128|5064|36000|2|38|13|1|9416|277|154|45|24.63|62.80|27.63|1582.65|1243.35|1108.35|2826.00|62.16|0.00|621.45|1243.35|1305.51|1864.80|1926.96|135.00| +2450817|14471|2450894|90839|1856128|5064|36000|90839|1856128|5064|36000|2|75|18|2|8174|50|154|24|48.32|95.19|88.52|160.08|2124.48|1159.68|2284.56|0.00|212.44|639.60|1912.04|1912.04|2551.64|2551.64|752.36| +2450817|14471|2450882|90839|1856128|5064|36000|90839|1856128|5064|36000|2|54|5|1|12454|39|154|55|5.23|8.31|1.32|384.45|72.60|287.65|457.05|0.00|0.00|109.45|72.60|72.60|182.05|182.05|-215.05| +2450817|14471|2450874|90839|1856128|5064|36000|90839|1856128|5064|36000|2|21|3|2|12421|20|154|36|36.66|100.08|10.00|3242.88|360.00|1319.76|3602.88|28.80|0.00|1585.08|360.00|388.80|1945.08|1973.88|-959.76| +2450817|14471|2450872|90839|1856128|5064|36000|90839|1856128|5064|36000|2|34|9|2|4279|171|154|89|61.36|73.63|49.33|2162.70|4390.37|5461.04|6553.07|351.22|0.00|1244.22|4390.37|4741.59|5634.59|5985.81|-1070.67| +2450817|14471|2450869|90839|1856128|5064|36000|90839|1856128|5064|36000|2|24|3|1|12772|13|154|28|79.28|231.49|203.71|777.84|5703.88|2219.84|6481.72|285.19|0.00|1361.08|5703.88|5989.07|7064.96|7350.15|3484.04| +2450817|14471|2450897|90839|1856128|5064|36000|90839|1856128|5064|36000|2|59|15|5|8995|118|154|55|62.12|88.21|61.74|1455.85|3395.70|3416.60|4851.55|237.69|0.00|1212.75|3395.70|3633.39|4608.45|4846.14|-20.90| +2450817|14471|2450823|90839|1856128|5064|36000|90839|1856128|5064|36000|2|61|10|4|8228|59|154|1|43.66|116.57|38.46|78.11|38.46|43.66|116.57|0.00|20.76|6.99|17.70|17.70|24.69|24.69|-25.96| +2450817|14471|2450854|90839|1856128|5064|36000|90839|1856128|5064|36000|2|80|6|2|6908|102|154|94|86.44|117.55|27.03|8508.88|2540.82|8125.36|11049.70|13.72|1854.79|2982.62|686.03|699.75|3668.65|3682.37|-7439.33| +2450817|14471|2450822|90839|1856128|5064|36000|90839|1856128|5064|36000|2|48|17|5|8066|289|154|23|88.46|168.07|146.22|502.55|3363.06|2034.58|3865.61|33.63|0.00|1932.69|3363.06|3396.69|5295.75|5329.38|1328.48| +2450817|14471|2450866|90839|1856128|5064|36000|90839|1856128|5064|36000|2|63|6|2|170|79|154|90|77.22|168.33|94.26|6666.30|8483.40|6949.80|15149.70|678.67|0.00|7423.20|8483.40|9162.07|15906.60|16585.27|1533.60| +2450818|66389|2450866|6138|1335450|7162|31674|6138|1335450|7162|31674|2|33|17|4|15223|200|155|39|29.21|67.18|18.81|1886.43|733.59|1139.19|2620.02|22.00|0.00|418.86|733.59|755.59|1152.45|1174.45|-405.60| +2450818|66389|2450868|6138|1335450|7162|31674|6138|1335450|7162|31674|2|20|7|1|3481|81|155|24|93.66|251.00|2.51|5963.76|60.24|2247.84|6024.00|3.61|0.00|120.48|60.24|63.85|180.72|184.33|-2187.60| +2450818|66389|2450875|6138|1335450|7162|31674|6138|1335450|7162|31674|2|20|16|2|7688|277|155|70|97.85|107.63|7.53|7007.00|527.10|6849.50|7534.10|36.89|0.00|2184.70|527.10|563.99|2711.80|2748.69|-6322.40| +2450818|66389|2450895|6138|1335450|7162|31674|6138|1335450|7162|31674|2|49|3|4|10309|134|155|64|49.20|103.81|87.20|1063.04|5580.80|3148.80|6643.84|167.42|0.00|2524.16|5580.80|5748.22|8104.96|8272.38|2432.00| +2450818|66389|2450891|6138|1335450|7162|31674|6138|1335450|7162|31674|2|20|10|5|11377|17|155|55|87.44|169.63|145.88|1306.25|8023.40|4809.20|9329.65|240.70|0.00|3171.85|8023.40|8264.10|11195.25|11435.95|3214.20| +2450818|66389|2450822|6138|1335450|7162|31674|6138|1335450|7162|31674|2|11|18|5|14146|31|155|75|95.72|259.40|98.57|12062.25|7392.75|7179.00|19455.00|443.56|0.00|4863.75|7392.75|7836.31|12256.50|12700.06|213.75| +2450818|66389|2450881|6138|1335450|7162|31674|6138|1335450|7162|31674|2|59|16|5|13594|53|155|4|97.80|153.54|49.13|417.64|196.52|391.20|614.16|1.96|0.00|233.36|196.52|198.48|429.88|431.84|-194.68| +2450818|66389|2450874|6138|1335450|7162|31674|6138|1335450|7162|31674|2|10|4|2|17377|210|155|100|60.18|164.89|95.63|6926.00|9563.00|6018.00|16489.00|382.52|0.00|7090.00|9563.00|9945.52|16653.00|17035.52|3545.00| +2450818|66389|2450836|6138|1335450|7162|31674|6138|1335450|7162|31674|2|29|7|4|10546|134|155|72|16.03|37.67|19.96|1275.12|1437.12|1154.16|2712.24|28.74|0.00|813.60|1437.12|1465.86|2250.72|2279.46|282.96| +2450818|66389|2450886|6138|1335450|7162|31674|6138|1335450|7162|31674|2|100|7|4|13802|208|155|57|55.27|96.72|13.54|4741.26|771.78|3150.39|5513.04|6.94|77.17|991.80|694.61|701.55|1686.41|1693.35|-2455.78| +2450818|36262|2450872|95770|1561127|4437|24194|95770|1561127|4437|24194|4|19|4|3|11413|83|156|88|72.00|190.08|159.66|2676.96|14050.08|6336.00|16727.04|983.50|0.00|668.80|14050.08|15033.58|14718.88|15702.38|7714.08| +2450818|36262|2450868|95770|1561127|4437|24194|95770|1561127|4437|24194|4|56|13|1|17224|205|156|50|68.90|90.94|16.36|3729.00|818.00|3445.00|4547.00|24.54|0.00|1864.00|818.00|842.54|2682.00|2706.54|-2627.00| +2450818|36262|2450837|95770|1561127|4437|24194|95770|1561127|4437|24194|4|34|11|4|3454|181|156|44|60.26|144.02|48.96|4182.64|2154.24|2651.44|6336.88|6.03|2003.44|1837.44|150.80|156.83|1988.24|1994.27|-2500.64| +2450818|36262|2450882|95770|1561127|4437|24194|95770|1561127|4437|24194|4|69|14|2|2720|16|156|21|14.86|43.68|0.00|917.28|0.00|312.06|917.28|0.00|0.00|265.86|0.00|0.00|265.86|265.86|-312.06| +2450818|36262|2450895|95770||||95770|||24194||57|11||4112|73|156|33|16.50||||||843.81|7.43||109.56|92.93||202.49|209.92|-451.57| +2450818|36262|2450901|95770|1561127|4437|24194|95770|1561127|4437|24194|4|85|1|2|14186|291|156|94|57.84|78.66|3.93|7024.62|369.42|5436.96|7394.04|10.04|243.81|3253.34|125.61|135.65|3378.95|3388.99|-5311.35| +2450818|36262|2450823|95770|1561127|4437|24194|95770|1561127|4437|24194|4|6|4|3|14905|61|156|50|40.86|106.64|78.91|1386.50|3945.50|2043.00|5332.00|315.64|0.00|2506.00|3945.50|4261.14|6451.50|6767.14|1902.50| +2450818|36262|2450823|95770|1561127|4437|24194|95770|1561127|4437|24194|4|21|6|3|9523|7|156|76|55.08|90.33|61.42|2197.16|4667.92|4186.08|6865.08|420.11|0.00|1990.44|4667.92|5088.03|6658.36|7078.47|481.84| +2450818|44871|2450878|68393|1526337|3834|14178|68393|1526337|3834|14178|4|63|1|4|934|197|157|1|98.17|154.12|107.88|46.24|107.88|98.17|154.12|6.47|0.00|7.70|107.88|114.35|115.58|122.05|9.71| +2450818|44871|2450824|68393|1526337|3834|14178|68393|1526337|3834|14178|4|96|12|4|12854|241|157|44|3.34|8.35|2.42|260.92|106.48|146.96|367.40|8.51|0.00|69.52|106.48|114.99|176.00|184.51|-40.48| +2450818|44871|2450883|68393|1526337|3834|14178|68393|1526337|3834|14178|4|36|15|2|13406|198|157|94|56.06|61.10|48.88|1148.68|4594.72|5269.64|5743.40|91.89|0.00|516.06|4594.72|4686.61|5110.78|5202.67|-674.92| +2450818|44871|2450878|68393|1526337|3834|14178|68393|1526337|3834|14178|4|71|18|5|16282|101|157|59|88.57|96.54|48.27|2847.93|2847.93|5225.63|5695.86|94.83|1794.19|1024.83|1053.74|1148.57|2078.57|2173.40|-4171.89| +2450818|44871|2450896|68393|1526337|3834|14178|68393|1526337|3834|14178|4|57|16|2|3506|89|157|26|63.89|93.27|85.80|194.22|2230.80|1661.14|2425.02|111.54|0.00|339.30|2230.80|2342.34|2570.10|2681.64|569.66| +2450818|44871|2450853|68393|1526337|3834|14178|68393|1526337|3834|14178|4|70|11|1|12632|43|157|57|83.76|165.00|128.70|2069.10|7335.90|4774.32|9405.00|366.79|0.00|1975.05|7335.90|7702.69|9310.95|9677.74|2561.58| +2450818|44871|2450829|68393|1526337|3834|14178|68393|1526337|3834|14178|4|82|6|5|10660|180|157|76|47.79|74.07|15.55|4447.52|1181.80|3632.04|5629.32|106.36|0.00|2082.40|1181.80|1288.16|3264.20|3370.56|-2450.24| +2450818|44871|2450861|68393|1526337|3834|14178|68393|1526337|3834|14178|4|91|15|5|614|246|157|76|28.46|71.43|31.42|3040.76|2387.92|2162.96|5428.68|119.39|0.00|867.92|2387.92|2507.31|3255.84|3375.23|224.96| +2450818|44871|2450874|68393|1526337|3834|14178|68393|1526337|3834|14178|4|33|16|5|15721|241|157|81|8.36|14.63|9.80|391.23|793.80|677.16|1185.03|15.87|0.00|248.67|793.80|809.67|1042.47|1058.34|116.64| +2450818|38093|2450896|48307|1090899|1138|10530|48307|1090899|1138|10530|2|38|12|2|7864|164|158|66|21.48|54.34|2.17|3443.22|143.22|1417.68|3586.44|11.45|0.00|788.70|143.22|154.67|931.92|943.37|-1274.46| +2450818|38093|2450829|48307|1090899|1138|10530|48307|1090899|1138|10530|2|42|2|5|14384|173|158|40|64.68|155.23|77.61|3104.80|3104.40|2587.20|6209.20|0.00|0.00|2669.60|3104.40|3104.40|5774.00|5774.00|517.20| +2450818|38093|2450839|48307|1090899|1138|10530|48307|1090899|1138|10530|2|86|1|3|17156|286|158|26|87.66|136.74|58.79|2026.70|1528.54|2279.16|3555.24|30.57|0.00|1777.62|1528.54|1559.11|3306.16|3336.73|-750.62| +2450818|38093|2450825|48307|1090899|1138|10530|48307|1090899|1138|10530|2|61|15|2|8090|205|158|67|46.47|96.65|28.02|4598.21|1877.34|3113.49|6475.55|71.33|93.86|2136.63|1783.48|1854.81|3920.11|3991.44|-1330.01| +2450818|38093|2450865|48307|1090899|1138|10530|48307|1090899|1138|10530|2|12|6|5|17113|131|158|79|95.48|267.34|155.05|8870.91|12248.95|7542.92|21119.86|68.59|11391.52|2534.32|857.43|926.02|3391.75|3460.34|-6685.49| +2450818|38093|2450875|48307|1090899|1138|10530|48307|1090899|1138|10530|2|89|15|4|2150|250|158|24|18.12|23.01|12.88|243.12|309.12|434.88|552.24|12.36|0.00|248.40|309.12|321.48|557.52|569.88|-125.76| +2450818|38093|2450854|48307|1090899|1138|10530|48307|1090899|1138|10530|2|95|5|1|6404|148|158|35|75.52|175.96|86.22|3140.90|3017.70|2643.20|6158.60|0.00|0.00|1724.10|3017.70|3017.70|4741.80|4741.80|374.50| +||||1090899|1138|10530|48307|1090899|1138||2|18||3|15056||158|24|||92.28||2214.72|1269.60||||1234.32|||3449.04|3648.36|945.12| +2450818|40071|2450833|96893|868838|1230|40775|96893|868838|1230|40775|4|95|12|2|9292|106|159|45|31.39|82.86|50.54|1454.40|2274.30|1412.55|3728.70|68.22|0.00|447.30|2274.30|2342.52|2721.60|2789.82|861.75| +2450818|40071|2450879|96893|868838|1230|40775|96893|868838|1230|40775|4|28|12|5|7988|89|159|93|49.74|82.56|78.43|384.09|7293.99|4625.82|7678.08|583.51|0.00|1842.33|7293.99|7877.50|9136.32|9719.83|2668.17| +2450818|40071|2450880|96893|868838|1230|40775|96893|868838|1230|40775|4|91|19|5|5618|184|159|68|60.86|83.98|77.26|456.96|5253.68|4138.48|5710.64|157.61|0.00|2398.36|5253.68|5411.29|7652.04|7809.65|1115.20| +2450818|40071|2450873|96893|868838|1230|40775|96893|868838|1230|40775|4|56|20|5|7813|170|159|13|71.83|103.43|97.22|80.73|1263.86|933.79|1344.59|18.95|947.89|605.02|315.97|334.92|920.99|939.94|-617.82| +2450818|58650|2450860|24511|1114265|1972|42479|24511|1114265|1972|42479|1|23|7|4|15538|6|160|37|13.02|26.56|11.15|570.17|412.55|481.74|982.72|28.87|0.00|206.09|412.55|441.42|618.64|647.51|-69.19| +2450818|58650|2450853|24511|1114265|1972|42479|24511|1114265|1972|42479|1|95|16|1|1472|185|160|90|3.65|5.87|4.93|84.60|443.70|328.50|528.30|31.05|0.00|237.60|443.70|474.75|681.30|712.35|115.20| +2450818|58650|2450869|24511|1114265|1972|42479|24511|1114265|1972|42479|1|93|6|5|13564|298|160|90|52.28|67.44|24.27|3885.30|2184.30|4705.20|6069.60|174.74|0.00|1517.40|2184.30|2359.04|3701.70|3876.44|-2520.90| +2450818|58650|2450881|24511|1114265|1972|42479|24511|1114265|1972|42479|1|36|1|3|1714|85|160|83|57.34|122.13|96.48|2128.95|8007.84|4759.22|10136.79|480.47|0.00|1013.43|8007.84|8488.31|9021.27|9501.74|3248.62| +2450818|58650|2450865|24511|1114265|1972|42479|24511|1114265|1972|42479|1|104|13|3|7718|106|160|87|29.96|35.35|19.08|1415.49|1659.96|2606.52|3075.45|33.19|0.00|737.76|1659.96|1693.15|2397.72|2430.91|-946.56| +2450818|58650|2450838|24511|1114265|1972|42479|24511|1114265|1972|42479|1|19|10|1|4093|273|160|37|60.45|180.14|41.43|5132.27|1532.91|2236.65|6665.18|61.31|0.00|133.20|1532.91|1594.22|1666.11|1727.42|-703.74| +2450818|70080|2450892|46125|359731|537|48143|46125|359731|537|48143|2|18|6|5|1148|146|161|2|43.16|90.20|13.53|153.34|27.06|86.32|180.40|0.00|0.00|73.96|27.06|27.06|101.02|101.02|-59.26| +2450818|70080|2450887|46125|359731|537|48143|46125|359731|537|48143|2|21|17|3|6085|186|161|87|79.74|130.77|6.53|10808.88|568.11|6937.38|11376.99|49.59|17.04|910.02|551.07|600.66|1461.09|1510.68|-6386.31| +2450818|70080|2450872|46125|359731|537|48143|46125|359731|537|48143|2|36|11|5|4370|270|161|53|44.34|49.21|6.88|2243.49|364.64|2350.02|2608.13|1.02|350.05|1277.83|14.59|15.61|1292.42|1293.44|-2335.43| +2450818|70080|2450897|46125|359731|537|48143|46125|359731|537|48143|2|82|9|1|12412|45|161|15|11.83|23.06|23.06|0.00|345.90|177.45|345.90|17.29|0.00|51.75|345.90|363.19|397.65|414.94|168.45| +2450818|70080|2450885|46125|359731|537|48143|46125|359731|537|48143|2|8|20|2|6079|101|161|66|51.48|113.77|44.37|4580.40|2928.42|3397.68|7508.82|117.13|0.00|149.82|2928.42|3045.55|3078.24|3195.37|-469.26| +2450818|61699|2450901|67220|1094840|413|39633|67220|1094840|413|39633|4|8|5|3|6062|161|162|45|2.34|4.93|0.49|199.80|22.05|105.30|221.85|1.54|0.00|32.85|22.05|23.59|54.90|56.44|-83.25| +2450818|61699|2450833|67220|1094840|413|39633|67220|1094840|413|39633|4|17|15|3|1922|112|162|32|57.57|154.28|152.73|49.60|4887.36|1842.24|4936.96|293.24|0.00|1086.08|4887.36|5180.60|5973.44|6266.68|3045.12| +2450818|61699|2450839|67220|1094840|413|39633|67220|1094840|413|39633|4|59|1|5|3667|7|162|46|18.45|41.69|24.59|786.60|1131.14|848.70|1917.74|11.31|0.00|766.82|1131.14|1142.45|1897.96|1909.27|282.44| +2450818|61699|2450898|67220|1094840|413|39633|67220|1094840|413|39633|4|80|6|3|3232|264|162|51|39.31|72.33|14.46|2951.37|737.46|2004.81|3688.83|44.24|0.00|405.45|737.46|781.70|1142.91|1187.15|-1267.35| +2450818|61699|2450903|67220|1094840|413|39633|67220|1094840|413|39633|4|79|14|3|10052|223|162|86|12.45|33.61|15.12|1590.14|1300.32|1070.70|2890.46|65.01|0.00|924.50|1300.32|1365.33|2224.82|2289.83|229.62| +2450818|53312|2450827|5043|613370|4450|16377|5043|613370|4450|16377|1|8|13|2|10951|107|163|43|50.30|146.87|27.90|5115.71|1199.70|2162.90|6315.41|47.98|0.00|757.66|1199.70|1247.68|1957.36|2005.34|-963.20| +2450818|53312|2450826|5043|613370|4450|16377|5043|613370|4450|16377|1|74|20|4|11125|10|163|39|91.04|115.62|47.40|2660.58|1848.60|3550.56|4509.18|36.97|0.00|2074.02|1848.60|1885.57|3922.62|3959.59|-1701.96| +2450818|53312|2450891|5043|613370|4450|16377|5043|613370|4450|16377|1|65|1|3|6025|22|163|79|26.41|69.98|31.49|3040.71|2487.71|2086.39|5528.42|49.75|0.00|1934.71|2487.71|2537.46|4422.42|4472.17|401.32| +2450818|53312|2450867|5043|613370|4450|16377|5043|613370|4450|16377|1|19|2|1|550|67|163|85|61.06|90.97|30.02|5180.75|2551.70|5190.10|7732.45|127.58|0.00|463.25|2551.70|2679.28|3014.95|3142.53|-2638.40| +2450818|53312|2450872|5043|613370|4450|16377|5043|613370|4450|16377|1|102|12|2|12806|165|163|8|29.25|34.80|33.40|11.20|267.20|234.00|278.40|12.15|93.52|128.00|173.68|185.83|301.68|313.83|-60.32| +2450818|53312|2450829|5043|613370|4450|16377|5043|613370|4450|16377|1|62|15|4|4376|224|163|50|53.06|54.65|25.68|1448.50|1284.00|2653.00|2732.50|115.56|0.00|1065.50|1284.00|1399.56|2349.50|2465.06|-1369.00| +2450818|53312|2450901|5043|613370|4450|16377|5043|613370|4450|16377|1|103|9|4|15512|129|163|26|1.57|2.15|1.61|14.04|41.86|40.82|55.90|0.83|0.00|8.84|41.86|42.69|50.70|51.53|1.04| +2450818|53312|2450859|5043|613370|4450|16377|5043|613370|4450|16377|1|97|5|2|1142|298|163|17|25.16|43.52|34.81|148.07|591.77|427.72|739.84|53.25|0.00|81.26|591.77|645.02|673.03|726.28|164.05| +2450818|53312|2450897|5043|613370|4450|16377|5043|613370|4450|16377|1|97|15|5|11116|133|163|66|31.89|89.92|23.37|4392.30|1542.42|2104.74|5934.72|61.69|771.21|2729.76|771.21|832.90|3500.97|3562.66|-1333.53| +2450818|53312|2450844|5043|613370|4450|16377|5043|613370|4450|16377|1|93|17|4|308|265|163|39|54.79|99.71|98.71|39.00|3849.69|2136.81|3888.69|192.48|0.00|0.00|3849.69|4042.17|3849.69|4042.17|1712.88| +2450818|53312|2450851|5043|613370|4450|16377|5043|613370|4450|16377|1|16|3|3|12104|299|163|77|57.62|113.51|49.94|4894.89|3845.38|4436.74|8740.27|153.81|0.00|2184.49|3845.38|3999.19|6029.87|6183.68|-591.36| +2450818|53312|2450873|5043|613370|4450|16377|5043|613370|4450|16377|1|29|20|2|10744|30|163|75|87.05|237.64|35.64|15150.00|2673.00|6528.75|17823.00|240.57|0.00|5703.00|2673.00|2913.57|8376.00|8616.57|-3855.75| +2450818|53312|2450906|5043|613370|4450|16377|5043|613370|4450|16377|1|14|10|4|1933|225|163|7|68.06|174.91|55.97|832.58|391.79|476.42|1224.37|2.11|356.52|391.79|35.27|37.38|427.06|429.17|-441.15| +2450818|61349|2450888|66034|1091101|4968|12482|66034|1091101|4968|12482|2|29|6|3|16513|128|164|1|99.85|195.70|162.43|33.27|162.43|99.85|195.70|8.12|0.00|80.23|162.43|170.55|242.66|250.78|62.58| +2450818|61349|2450853|66034|1091101|4968|12482|66034|1091101|4968|12482|2|24|14|1|9355|44|164|17|100.00|222.00|186.48|603.84|3170.16|1700.00|3774.00|253.61|0.00|1509.60|3170.16|3423.77|4679.76|4933.37|1470.16| +2450818|61349|2450870|66034|1091101|4968|12482|66034|1091101|4968|12482|2|90|7|2|16892|153|164|66|7.07|17.18|0.51|1100.22|33.66|466.62|1133.88|1.68|0.00|22.44|33.66|35.34|56.10|57.78|-432.96| +2450818|61349|2450905|66034|1091101|4968|12482|66034|1091101|4968|12482|2|31|13|3|14479|288|164|77|88.98|106.77|69.40|2877.49|5343.80|6851.46|8221.29|160.31|0.00|2876.72|5343.80|5504.11|8220.52|8380.83|-1507.66| +2450818|61349|2450824|66034|1091101|4968|12482|66034|1091101|4968|12482|2|74|1|4|12730|120|164|90|86.10|104.18|79.17|2250.90|7125.30|7749.00|9376.20|356.26|0.00|2156.40|7125.30|7481.56|9281.70|9637.96|-623.70| +2450818|61349|2450828|66034|1091101|4968|12482|66034|1091101|4968|12482|2|102|12|5|15913|74|164|53|32.92|49.70|40.75|474.35|2159.75|1744.76|2634.10|151.18|0.00|1053.64|2159.75|2310.93|3213.39|3364.57|414.99| +2450818|61349|2450860|66034|1091101|4968|12482|66034|1091101|4968|12482|2|4|5|2|13060|221|164|36|64.19|167.53|5.02|5850.36|180.72|2310.84|6031.08|9.03|0.00|1688.40|180.72|189.75|1869.12|1878.15|-2130.12| +2450818|61349|2450907|66034|1091101|4968|12482|66034|1091101|4968|12482|2|40|3|1|15292|39|164|53|74.92|98.14|7.85|4785.37|416.05|3970.76|5201.42|33.28|0.00|1664.20|416.05|449.33|2080.25|2113.53|-3554.71| +2450818|61349|2450834|66034|1091101|4968|12482|66034|1091101|4968|12482|2|15|3|2|13906|62|164|28|45.06|91.47|50.30|1152.76|1408.40|1261.68|2561.16|84.50|0.00|50.96|1408.40|1492.90|1459.36|1543.86|146.72| +2450818|61349|2450896|66034|1091101|4968|12482|66034|1091101|4968|12482|2|91|11|4|10747|33|164|72|20.69|30.41|5.47|1795.68|393.84|1489.68|2189.52|27.56|0.00|196.56|393.84|421.40|590.40|617.96|-1095.84| +2450818|61349|2450888|66034|1091101|4968|12482|66034|1091101|4968|12482|2|1|3|4|12686|159|164|66|16.10|27.20|18.49|574.86|1220.34|1062.60|1795.20|61.01|0.00|430.32|1220.34|1281.35|1650.66|1711.67|157.74| +2450818|61349|2450851|66034|1091101|4968|12482|66034|1091101|4968|12482|2|5|20|1|10340|23|164|11|58.69|117.96|87.29|337.37|960.19|645.59|1297.56|50.69|115.22|337.26|844.97|895.66|1182.23|1232.92|199.38| +2450818|61349|2450863|66034|1091101|4968|12482|66034|1091101|4968|12482|2|15|3|3|17410|300|164|60|68.48|154.76|117.61|2229.00|7056.60|4108.80|9285.60|0.00|0.00|3064.20|7056.60|7056.60|10120.80|10120.80|2947.80| +2450818|51699|2450848|88367|768291|5368|38092|88367|768291|5368|38092|2|97|9|3|5150|55|165|31|65.72|139.32|90.55|1511.87|2807.05|2037.32|4318.92|140.35|0.00|431.83|2807.05|2947.40|3238.88|3379.23|769.73| +2450818|51699|2450829|88367|768291|5368|38092|88367|768291|5368|38092|2|39|3|2|16738|65|165|78|87.85|143.19|110.25|2569.32|8599.50|6852.30|11168.82|85.99|0.00|5025.54|8599.50|8685.49|13625.04|13711.03|1747.20| +2450818|51699|2450842|88367|768291|5368|38092|88367|768291|5368|38092|2|98|4|1|14176|192|165|7|66.23|188.09|45.14|1000.65|315.98|463.61|1316.63|25.27|0.00|236.95|315.98|341.25|552.93|578.20|-147.63| +2450818|51699|2450849|88367|768291|5368|38092|88367|768291|5368|38092|2|12|4|4|9146|162|165|20|65.18|99.07|18.82|1605.00|376.40|1303.60|1981.40|18.82|0.00|257.40|376.40|395.22|633.80|652.62|-927.20| +2450818|51699|2450889|88367|768291|5368|38092|88367|768291|5368|38092|2|44|10|2|4381|166|165|29|3.64|4.11|0.61|101.50|17.69|105.56|119.19|0.08|13.62|28.42|4.07|4.15|32.49|32.57|-101.49| +2450818|51699|2450852|88367|768291|5368|38092|88367|768291|5368|38092|2|98|15|2|14816|181|165|60|99.38|222.61|48.97|10418.40|2938.20|5962.80|13356.60|264.43|0.00|934.80|2938.20|3202.63|3873.00|4137.43|-3024.60| +2450818|51699|2450854|88367|768291|5368|38092|88367|768291|5368|38092|2|53|13|4|12547|283|165|22|7.24|7.24|6.51|16.06|143.22|159.28|159.28|8.59|0.00|47.74|143.22|151.81|190.96|199.55|-16.06| +2450818|51699|2450835|88367|768291|5368|38092|88367|768291|5368|38092|2|8|9|5|16568|149|165|96|59.36|132.96|81.10|4978.56|7785.60|5698.56|12764.16|6.22|7162.75|509.76|622.85|629.07|1132.61|1138.83|-5075.71| +2450818|51699|2450821|88367|768291|5368|38092|88367|768291|5368|38092|2|51|16|4|5674|244|165|47|60.40|156.43|103.24|2499.93|4852.28|2838.80|7352.21|0.00|0.00|3675.87|4852.28|4852.28|8528.15|8528.15|2013.48| +2450818|51699|2450884|88367|768291|5368|38092|88367|768291|5368|38092|2|43|17|2|17734|182|165|98|4.44|9.94|5.36|448.84|525.28|435.12|974.12|21.01|0.00|487.06|525.28|546.29|1012.34|1033.35|90.16| +2450818|51699|2450887|88367|768291|5368|38092|88367|768291|5368|38092|2|105|5|2|7816|234|165|21|59.03|149.93|37.48|2361.45|787.08|1239.63|3148.53|53.44|23.61|1385.16|763.47|816.91|2148.63|2202.07|-476.16| +2450818|51699|2450821|88367|768291|5368|38092|88367|768291|5368|38092|2|91|8|3|3196|6|165|2|20.61|44.72|14.31|60.82|28.62|41.22|89.44|1.71|0.00|16.98|28.62|30.33|45.60|47.31|-12.60| +2450818|51699|2450887|88367|768291|5368|38092|88367|768291|5368|38092|2|55|19|4|4978|209|165|85|80.39|139.87|81.12|4993.75|6895.20|6833.15|11888.95|344.76|0.00|3447.60|6895.20|7239.96|10342.80|10687.56|62.05| +2450818|51699|2450849|88367|768291|5368|38092|88367|768291|5368|38092|2|56|4|3|9316|111|165|87|99.29|202.55|105.32|8459.01|9162.84|8638.23|17621.85|183.25|0.00|2643.06|9162.84|9346.09|11805.90|11989.15|524.61| +2450818|26465|2450886|83708|1007544|4533|46369|20820|1549994|2183|40819|4|66|2|3|8608|112|166|30|5.24|10.53|6.10|132.90|183.00|157.20|315.90|4.09|124.44|6.30|58.56|62.65|64.86|68.95|-98.64| +2450818|26465|2450825|83708|1007544|4533|46369|20820|1549994|2183|40819|4|58|8|1|7324|21|166|94|92.81|156.84|128.60|2654.56|12088.40|8724.14|14742.96|967.07|0.00|3095.42|12088.40|13055.47|15183.82|16150.89|3364.26| +2450818|26465|2450835|83708|1007544|4533|46369|20820|1549994|2183|40819|4|79|4|4|8887|244|166|21|44.67|81.29|54.46|563.43|1143.66|938.07|1707.09|68.61|0.00|307.23|1143.66|1212.27|1450.89|1519.50|205.59| +2450818|26465|2450833|83708|1007544|4533|46369|20820|1549994|2183|40819|4|105|11|5|14512|10|166|48|21.28|21.91|5.25|799.68|252.00|1021.44|1051.68|17.64|0.00|483.36|252.00|269.64|735.36|753.00|-769.44| +2450818|26465|2450858|83708|1007544|4533|46369|20820|1549994|2183|40819|4|28|19|3|15352|263|166|97|81.90|104.01|11.44|8979.29|1109.68|7944.30|10088.97|11.09|0.00|2320.24|1109.68|1120.77|3429.92|3441.01|-6834.62| +2450818|26465|2450905|83708|1007544|4533|46369|20820|1549994|2183|40819|4|91|1|1|2656|94|166|93|9.09|25.36|17.24|755.16|1603.32|845.37|2358.48|64.13|0.00|612.87|1603.32|1667.45|2216.19|2280.32|757.95| +2450818|26465|2450846|83708|1007544|4533|46369|20820|1549994|2183|40819|4|75|2|2|13225|176|166|57|14.02|15.00|4.95|572.85|282.15|799.14|855.00|5.64|0.00|111.15|282.15|287.79|393.30|398.94|-516.99| +2450818|26465|2450898|83708|1007544|4533|46369|20820|1549994|2183|40819|4|21|6|3|13930|123|166|41|14.62|30.99|5.26|1054.93|215.66|599.42|1270.59|19.40|0.00|457.15|215.66|235.06|672.81|692.21|-383.76| +2450818|26465|2450905|83708|1007544|4533|46369|20820|1549994|2183|40819|4|19|16|4|586|5|166|61|32.70|92.21|63.62|1743.99|3880.82|1994.70|5624.81|155.23|0.00|1743.38|3880.82|4036.05|5624.20|5779.43|1886.12| +2450818|26465|2450846|83708|1007544|4533|46369|20820|1549994|2183|40819|4|90|17|2|17317|220|166|30|21.03|50.89|23.40|824.70|702.00|630.90|1526.70|49.14|0.00|0.00|702.00|751.14|702.00|751.14|71.10| +2450818|26465|2450867|83708|1007544|4533|46369|20820|1549994|2183|40819|4|95|17|1|1837|1|166|70|56.46|110.66|26.55|5887.70|1858.50|3952.20|7746.20|93.66|687.64|1858.50|1170.86|1264.52|3029.36|3123.02|-2781.34| +2450818|26465|2450830|83708|1007544|4533|46369|20820|1549994|2183|40819|4|36|10|2|1762|100|166|10|87.16|93.26|81.13|121.30|811.30|871.60|932.60|73.01|0.00|298.40|811.30|884.31|1109.70|1182.71|-60.30| +2450818|26465|2450840|83708|1007544|4533|46369|20820|1549994|2183|40819|4|75|18|5|11860|134|166|79|92.74|231.85|32.45|15752.60|2563.55|7326.46|18316.15|205.08|0.00|2197.78|2563.55|2768.63|4761.33|4966.41|-4762.91| +2450818|43579|2450841|52073|1034112|1751|4190|52073|1034112|1751|4190|1|73|6|1|12608|1|167|87|39.15|97.87|31.31|5790.72|2723.97|3406.05|8514.69|81.71|0.00|1958.37|2723.97|2805.68|4682.34|4764.05|-682.08| +2450818|43579|2450875|52073|1034112|1751|4190|52073|1034112|1751|4190|1|79|2|2|17617|182|167|34|54.06|97.30|82.70|496.40|2811.80|1838.04|3308.20|196.82|0.00|1289.96|2811.80|3008.62|4101.76|4298.58|973.76| +2450818|43579|2450907|52073|1034112|1751|4190|52073|1034112|1751|4190|1|53|14|1|16156|9|167|49|14.83|17.20|13.41|185.71|657.09|726.67|842.80|39.42|0.00|159.74|657.09|696.51|816.83|856.25|-69.58| +2450818|43579|2450872|52073|1034112|1751|4190|52073|1034112|1751|4190|1|56|1|1|3674|266|167|72|4.01|4.33|0.47|277.92|33.84|288.72|311.76|0.00|33.84|133.92|0.00|0.00|133.92|133.92|-288.72| +2450818|43579|2450897|52073|1034112|1751|4190|52073|1034112|1751|4190|1|98|9|2|1381|3|167|47|36.04|68.47|11.63|2671.48|546.61|1693.88|3218.09|38.26|0.00|385.87|546.61|584.87|932.48|970.74|-1147.27| +2450818|43579|2450840|52073|1034112|1751|4190|52073|1034112|1751|4190|1|62|8|4|7909|248|167|33|68.07|95.97|52.78|1425.27|1741.74|2246.31|3167.01|8.36|1323.72|791.67|418.02|426.38|1209.69|1218.05|-1828.29| +2450818|43579|2450849|52073|1034112|1751|4190|52073|1034112|1751|4190|1|38|16|3|13843|177|167|2|26.30|71.27|9.26|124.02|18.52|52.60|142.54|1.48|0.00|71.26|18.52|20.00|89.78|91.26|-34.08| +2450818|43579|2450864|52073|1034112|1751|4190|52073|1034112|1751|4190|1|71|18|4|5911|98|167|16|41.41|69.15|44.94|387.36|719.04|662.56|1106.40|0.00|0.00|199.04|719.04|719.04|918.08|918.08|56.48| +2450818|23833|2450825|97693|353845|6444|36196|97693|353845|6444|36196|4|6|12|2|13064|42|168|2|76.31|117.51|0.00|235.02|0.00|152.62|235.02|0.00|0.00|112.80|0.00|0.00|112.80|112.80|-152.62| +2450818|23833|2450858|97693|353845|6444|36196|97693|353845|6444|36196|4|71|5|3|4466|251|168|24|33.94|77.72|38.08|951.36|913.92|814.56|1865.28|45.69|0.00|167.76|913.92|959.61|1081.68|1127.37|99.36| +2450818|23833|2450905|97693|353845|6444|36196|97693|353845|6444|36196|4|28|6|2|15448|182|168|19|4.23|11.16|2.56|163.40|48.64|80.37|212.04|3.40|0.00|67.83|48.64|52.04|116.47|119.87|-31.73| +2450818|23833|2450906|97693|353845|6444|36196|97693|353845|6444|36196|4|22|12|4|1705|28|168|73|18.78|31.55|30.60|69.35|2233.80|1370.94|2303.15|89.35|0.00|575.24|2233.80|2323.15|2809.04|2898.39|862.86| +2450818|23833|2450825|97693|353845|6444|36196|97693|353845|6444|36196|4|34|19|4|13316|96|168|24|81.72|232.08|13.92|5235.84|334.08|1961.28|5569.92|13.36|0.00|1893.60|334.08|347.44|2227.68|2241.04|-1627.20| +2450818|23833|2450852|97693|353845|6444|36196|97693|353845|6444|36196|4|29|11|3|17114|65|168|31|75.71|180.94|164.65|504.99|5104.15|2347.01|5609.14|204.16|0.00|2580.13|5104.15|5308.31|7684.28|7888.44|2757.14| +2450818|23833|2450889|97693|353845|6444|36196|97693|353845|6444|36196|4|45|4|1|12247|201|168|7|59.83|144.78|138.98|40.60|972.86|418.81|1013.46|38.91|0.00|121.59|972.86|1011.77|1094.45|1133.36|554.05| +2450818|23833|2450882|97693|353845|6444|36196|97693|353845|6444|36196|4|30|19|4|2402|119|168|15|35.34|68.20|41.60|399.00|624.00|530.10|1023.00|24.96|0.00|327.30|624.00|648.96|951.30|976.26|93.90| +2450818|23833|2450860|97693|353845|6444|36196|97693|353845|6444|36196|4|23|17|3|8474|138|168|27|12.24|18.60|5.58|351.54|150.66|330.48|502.20|7.53|0.00|245.97|150.66|158.19|396.63|404.16|-179.82| +2450818|23833|2450840|97693|353845|6444|36196|97693|353845|6444|36196|4|51|17|2|13514|282|168|59|11.77|21.06|7.37|807.71|434.83|694.43|1242.54|8.69|0.00|211.22|434.83|443.52|646.05|654.74|-259.60| +2450818|14609|2450891|49067|1108213|3806|20552|49067|1108213|3806|20552|2|11|6|3|9836|110|169|9|67.82|97.66|2.92|852.66|26.28|610.38|878.94|2.10|0.00|123.03|26.28|28.38|149.31|151.41|-584.10| +2450818|14609|2450897|49067|1108213|3806|20552|49067|1108213|3806|20552|2|25|16|2|16160|233|169|53|77.11|98.70|20.72|4132.94|1098.16|4086.83|5231.10|38.21|461.22|836.87|636.94|675.15|1473.81|1512.02|-3449.89| +2450818|14609|2450837|49067|1108213|3806|20552|49067|1108213|3806|20552|2|98|17|3|8515|202|169|7|97.72|109.44|10.94|689.50|76.58|684.04|766.08|1.48|2.29|0.00|74.29|75.77|74.29|75.77|-609.75| +2450818|14609|2450897|49067|1108213|3806|20552|49067|1108213|3806|20552|2|42|18|3|3985|8|169|23|92.18|148.40|136.52|273.24|3139.96|2120.14|3413.20|156.99|0.00|614.33|3139.96|3296.95|3754.29|3911.28|1019.82| +2450818|14609|2450864|49067|1108213|3806|20552|49067|1108213|3806|20552|2|95|5|1|6670|140|169|2|36.60|91.50|90.58|1.84|181.16|73.20|183.00|5.43|0.00|7.32|181.16|186.59|188.48|193.91|107.96| +2450818|14609|2450879|49067|1108213|3806|20552|49067|1108213|3806|20552|2|32|10|1|11606|173|169|53|84.01|107.53|21.50|4559.59|1139.50|4452.53|5699.09|56.97|0.00|2735.33|1139.50|1196.47|3874.83|3931.80|-3313.03| +2450818|14609|2450896|49067|1108213|3806|20552|49067|1108213|3806|20552|2|26|11|1|12824|58|169|25|49.55|145.67|112.16|837.75|2804.00|1238.75|3641.75|168.24|0.00|1056.00|2804.00|2972.24|3860.00|4028.24|1565.25| +2450818|14609|2450876|49067|1108213|3806|20552|49067|1108213|3806|20552|2|26|7|3|13010|130|169|76|8.78|20.10|17.68|183.92|1343.68|667.28|1527.60|40.31|0.00|656.64|1343.68|1383.99|2000.32|2040.63|676.40| +2450818|14609|2450884|49067|1108213|3806|20552|49067|1108213|3806|20552|2|43|16|5|5066|298|169|9|49.42|67.70|31.14|329.04|280.26|444.78|609.30|19.61|0.00|249.75|280.26|299.87|530.01|549.62|-164.52| +2450818|14609|2450856|49067|1108213|3806|20552|49067|1108213|3806|20552|2|48|18|3|4238|83|169|28|96.36|241.86|12.09|6433.56|338.52|2698.08|6772.08|14.01|104.94|2505.44|233.58|247.59|2739.02|2753.03|-2464.50| +2450818|14609|2450888|49067|1108213|3806|20552|49067|1108213|3806|20552|2|31|2|3|3068|67|169|9|77.19|203.78|99.85|935.37|898.65|694.71|1834.02|17.97|0.00|421.74|898.65|916.62|1320.39|1338.36|203.94| +2450818|80183|2450833|89256|597544|4232|48520|89256|597544|4232|48520|1|76|6|1|17179|191|170|35|1.37|2.04|0.48|54.60|16.80|47.95|71.40|1.17|0.00|12.60|16.80|17.97|29.40|30.57|-31.15| +2450818|80183|2450858|89256|597544|4232|48520|89256|597544|4232|48520|1|3|11|4|6721|224|170|96|7.20|9.28|3.52|552.96|337.92|691.20|890.88|20.27|0.00|408.96|337.92|358.19|746.88|767.15|-353.28| +2450818|80183|2450826|89256|597544|4232|48520|89256|597544|4232|48520|1|68|13|2|16918|73|170|100|6.89|11.50|9.89|161.00|989.00|689.00|1150.00|39.56|0.00|11.00|989.00|1028.56|1000.00|1039.56|300.00| +2450818|80183|2450861|89256|597544|4232|48520|89256|597544|4232|48520|1|62|17|2|922|235|170|70|79.51|141.52|28.30|7925.40|1981.00|5565.70|9906.40|39.62|0.00|3367.70|1981.00|2020.62|5348.70|5388.32|-3584.70| +2450818|80183|2450877|89256|597544|4232|48520|89256|597544|4232|48520|1|36|10|4|13030|154|170|28|82.43|184.64|127.40|1602.72|3567.20|2308.04|5169.92|0.00|0.00|1861.16|3567.20|3567.20|5428.36|5428.36|1259.16| +2450818|80183|2450828|89256|597544|4232|48520|89256|597544|4232|48520|1|103|2|3|13244|283|170|54|94.19|200.62|16.04|9967.32|866.16|5086.26|10833.48|17.32|0.00|3141.18|866.16|883.48|4007.34|4024.66|-4220.10| +2450818|80183|2450852|89256|597544|4232|48520|89256|597544|4232|48520|1|13|5|1|12844|157|170|45|66.67|68.00|31.96|1621.80|1438.20|3000.15|3060.00|57.52|0.00|612.00|1438.20|1495.72|2050.20|2107.72|-1561.95| +2450818|80183|2450837|89256|597544|4232|48520|89256|597544|4232|48520|1|68|14|5|16624|247|170|79|4.26|11.07|2.32|691.25|183.28|336.54|874.53|14.66|0.00|270.97|183.28|197.94|454.25|468.91|-153.26| +2450818|80183|2450846|89256|597544|4232|48520|89256|597544|4232|48520|1|65|5|1|12850|28|170|54|45.60|91.65|88.90|148.50|4800.60|2462.40|4949.10|384.04|0.00|841.32|4800.60|5184.64|5641.92|6025.96|2338.20| +2450818|42639|2450886|61755|1704613|6318|4423|16877|1539786|2291|39196|2|107|19|2|2404|290|171|47|51.60|94.94|14.24|3792.90|669.28|2425.20|4462.18|46.84|0.00|356.73|669.28|716.12|1026.01|1072.85|-1755.92| +2450818|42639|2450856|61755|1704613|6318|4423|16877|1539786|2291|39196|2|76|18|4|2593|113|171|89|25.90|59.31|49.82|844.61|4433.98|2305.10|5278.59|44.33|0.00|1318.98|4433.98|4478.31|5752.96|5797.29|2128.88| +2450818|42639|2450908|61755|1704613|6318|4423|16877|1539786|2291|39196|2|2|16|4|9169|13|171|59|72.77|99.69|61.80|2235.51|3646.20|4293.43|5881.71|33.54|1968.94|2881.56|1677.26|1710.80|4558.82|4592.36|-2616.17| +2450818|42639|2450823|61755|1704613|6318|4423|16877|1539786|2291|39196|2|53|13|5|12332|159|171|57|2.98|3.36|1.91|82.65|108.87|169.86|191.52|3.26|0.00|95.76|108.87|112.13|204.63|207.89|-60.99| +2450818|49203|2450858|98828|427554|280|12253|98828|427554|280|12253|1|28|8|5|8650|194|172|88|25.32|48.36|23.21|2213.20|2042.48|2228.16|4255.68|61.27|0.00|255.20|2042.48|2103.75|2297.68|2358.95|-185.68| +2450818|49203|2450894|98828|427554|280|12253|98828|427554|280|12253|1|76|15|3|9529|151|172|69|79.74|165.85|16.58|10299.63|1144.02|5502.06|11443.65|57.20|0.00|4577.46|1144.02|1201.22|5721.48|5778.68|-4358.04| +2450818|49203|2450894|98828|427554|280|12253|98828|427554|280|12253|1|43|17|2|1562|144|172|90|81.69|91.49|89.66|164.70|8069.40|7352.10|8234.10|161.38|0.00|1810.80|8069.40|8230.78|9880.20|10041.58|717.30| +2450818|49203|2450866|98828|427554|280|12253|98828|427554|280|12253|1|56|18|4|15241|138|172|78|60.06|153.75|116.85|2878.20|9114.30|4684.68|11992.50|523.16|1640.57|3717.48|7473.73|7996.89|11191.21|11714.37|2789.05| +2450818|49203|2450860|98828|427554|280|12253|98828|427554|280|12253|1|60|4|2|7237|208|172|55|75.84|214.62|68.67|8027.25|3776.85|4171.20|11804.10|113.30|0.00|5193.65|3776.85|3890.15|8970.50|9083.80|-394.35| +2450818|49203|2450884|98828|427554|280|12253|98828|427554|280|12253|1|44|9|5|1118|227|172|58|81.06|201.83|18.16|10652.86|1053.28|4701.48|11706.14|1.89|990.08|1872.82|63.20|65.09|1936.02|1937.91|-4638.28| +2450818|49203|2450849|98828|427554|280|12253|98828|427554|280|12253|1|3|11|1|12601|121|172|7|27.17|48.36|13.54|243.74|94.78|190.19|338.52|5.68|0.00|40.60|94.78|100.46|135.38|141.06|-95.41| +2450818|49203|2450867|98828|427554|280|12253|98828|427554|280|12253|1|26|20|3|15296|57|172|61|19.63|22.77|7.96|903.41|485.56|1197.43|1388.97|0.00|0.00|541.68|485.56|485.56|1027.24|1027.24|-711.87| +2450818|28390|2450853|10475|1830665|2408|49798|10475|1830665|2408|49798|4|25|18|2|12884|55|173|17|97.22|169.16|123.48|776.56|2099.16|1652.74|2875.72|62.97|0.00|891.31|2099.16|2162.13|2990.47|3053.44|446.42| +2450818|28390|2450868|10475|1830665|2408|49798|10475|1830665|2408|49798|4|4|20|5|14035|125|173|35|57.11|87.94|51.88|1262.10|1815.80|1998.85|3077.90|127.10|0.00|276.85|1815.80|1942.90|2092.65|2219.75|-183.05| +2450818|28390|2450891|10475|1830665|2408|49798|10475|1830665|2408|49798|4|52|10|4|17792|2|173|38|57.71|98.68|91.77|262.58|3487.26|2192.98|3749.84|174.36|0.00|1799.68|3487.26|3661.62|5286.94|5461.30|1294.28| +2450818|28390|2450834|10475|1830665|2408|49798|10475|1830665|2408|49798|4|41|6|1|14758|181|173|52|20.43|36.97|9.98|1403.48|518.96|1062.36|1922.44|5.18|0.00|172.64|518.96|524.14|691.60|696.78|-543.40| +2450818|28390|2450847|10475|1830665|2408|49798|10475|1830665|2408|49798|4|22|2|1|17389|57|173|79|56.18|119.66|118.46|94.80|9358.34|4438.22|9453.14|561.50|0.00|1323.25|9358.34|9919.84|10681.59|11243.09|4920.12| +2450818|28390|2450863|10475|1830665|2408|49798|10475|1830665|2408|49798|4|84|17|1|1490|186|173|99|64.98|165.04|66.01|9803.97|6534.99|6433.02|16338.96|261.39|0.00|4901.49|6534.99|6796.38|11436.48|11697.87|101.97| +2450818|28390|2450893|10475|1830665|2408|49798|10475|1830665|2408|49798|4|15|15|2|3410|106|173|84|35.03|95.28|50.49|3762.36|4241.16|2942.52|8003.52|381.70|0.00|3201.24|4241.16|4622.86|7442.40|7824.10|1298.64| +2450818|28390|2450837|10475|1830665|2408|49798|10475|1830665|2408|49798|4|75|19|3|3520|83|173|24|49.35|49.35|44.41|118.56|1065.84|1184.40|1184.40|63.95|0.00|592.08|1065.84|1129.79|1657.92|1721.87|-118.56| +2450818|28390|2450880|10475|1830665|2408|49798|10475|1830665|2408|49798|4|96|5|2|2527|42|173|63|32.01|48.97|34.27|926.10|2159.01|2016.63|3085.11|129.54|0.00|215.46|2159.01|2288.55|2374.47|2504.01|142.38| +2450818|28390|2450841|10475|1830665|2408|49798|10475|1830665|2408|49798|4|105|15|2|6346|83|173|84|55.27|97.27|49.60|4004.28|4166.40|4642.68|8170.68|291.64|0.00|3022.32|4166.40|4458.04|7188.72|7480.36|-476.28| +2450818|28390|2450834|10475|1830665|2408|49798|10475|1830665|2408|49798|4|56|18|1|3092|240|173|100|2.80|8.17|2.69|548.00|269.00|280.00|817.00|2.69|0.00|318.00|269.00|271.69|587.00|589.69|-11.00| +2450818|28390|2450857|10475|1830665|2408|49798|10475|1830665|2408|49798|4|58|6|1|6268|18|173|74|91.21|160.52|141.25|1425.98|10452.50|6749.54|11878.48|104.52|0.00|1900.32|10452.50|10557.02|12352.82|12457.34|3702.96| +2450818|16831|2450820|87171|1861135|5946|35957|87171|1861135|5946|35957|1|60|12|2|4702|99|174|12|71.36|190.53|121.93|823.20|1463.16|856.32|2286.36|117.05|0.00|1120.20|1463.16|1580.21|2583.36|2700.41|606.84| +2450818|16831|2450863|87171|1861135|5946|35957|87171|1861135|5946|35957|1|16|8|4|11044|88|174|96|43.76|87.52|63.88|2269.44|6132.48|4200.96|8401.92|490.59|0.00|251.52|6132.48|6623.07|6384.00|6874.59|1931.52| +2450818|16831|2450829|87171|1861135|5946|35957|87171|1861135|5946|35957|1|33|3|3|8065|222|174|86|86.54|222.40|113.42|9372.28|9754.12|7442.44|19126.40|93.63|7413.13|3442.58|2340.99|2434.62|5783.57|5877.20|-5101.45| +2450818|16831|2450891|87171|1861135|5946|35957|87171|1861135|5946|35957|1|44|11|5|14641|222|174|7|40.74|43.99|10.99|231.00|76.93|285.18|307.93|1.53|0.00|89.25|76.93|78.46|166.18|167.71|-208.25| +2450818|16831|2450822|87171|1861135|5946|35957|87171|1861135|5946|35957|1|73|17|2|13652|227|174|18|82.22|91.26|31.94|1067.76|574.92|1479.96|1642.68|17.24|0.00|328.50|574.92|592.16|903.42|920.66|-905.04| +2450818|16831|2450821|87171|1861135|5946|35957|87171|1861135|5946|35957|1|105|11|5|4675|106|174|17|70.96|106.44|67.05|669.63|1139.85|1206.32|1809.48|56.99|0.00|325.55|1139.85|1196.84|1465.40|1522.39|-66.47| +2450818|16831|2450876|87171|1861135|5946|35957|87171|1861135|5946|35957|1|25|6|1|7222|268|174|61|62.02|84.34|16.86|4116.28|1028.46|3783.22|5144.74|30.85|0.00|2572.37|1028.46|1059.31|3600.83|3631.68|-2754.76| +2450818|16831|2450889|87171|1861135|5946|35957|87171|1861135|5946|35957|1|105|6|1|4822|1|174|72|60.97|174.37|52.31|8788.32|3766.32|4389.84|12554.64|0.00|0.00|4393.44|3766.32|3766.32|8159.76|8159.76|-623.52| +2450818|16831|2450836|87171|1861135|5946|35957|87171|1861135|5946|35957|1|89|14|1|17833|68|174|4|95.94|156.38|115.72|162.64|462.88|383.76|625.52|41.65|0.00|112.56|462.88|504.53|575.44|617.09|79.12| +2450818|33975|2450866|56053|1631763|4626|29136|56053|1631763|4626|29136|4|41|15|1|4684|83|175|20|50.51|86.87|3.47|1668.00|69.40|1010.20|1737.40|3.33|2.77|573.20|66.63|69.96|639.83|643.16|-943.57| +2450818|33975|2450887|56053|1631763|4626|29136|56053|1631763|4626|29136|4|57|12|2|12805|84|175|19|43.44|116.41|57.04|1128.03|1083.76|825.36|2211.79|54.18|0.00|1061.53|1083.76|1137.94|2145.29|2199.47|258.40| +2450818|33975|2450848|56053|1631763|4626|29136|56053|1631763|4626|29136|4|40|11|5|5312|226|175|93|27.14|32.02|7.04|2323.14|654.72|2524.02|2977.86|52.37|0.00|1041.60|654.72|707.09|1696.32|1748.69|-1869.30| +2450818|33975|2450837|56053|1631763|4626|29136|56053|1631763|4626|29136|4|93|6|5|409|227|175|32|3.22|7.59|7.59|0.00|242.88|103.04|242.88|2.42|0.00|70.40|242.88|245.30|313.28|315.70|139.84| +2450818|33975|2450896|56053|1631763|4626|29136|56053|1631763|4626|29136|4|39|14|5|9644|87|175|87|53.72|100.45|73.32|2360.31|6378.84|4673.64|8739.15|510.30|0.00|1659.96|6378.84|6889.14|8038.80|8549.10|1705.20| +2450818|33975|2450821|56053|1631763|4626|29136|56053|1631763|4626|29136|4|91|20|2|4024|297|175|87|82.85|146.64|39.59|9313.35|3444.33|7207.95|12757.68|241.10|0.00|4974.66|3444.33|3685.43|8418.99|8660.09|-3763.62| +2450818|33975|2450885|56053|1631763|4626|29136|56053|1631763|4626|29136|4|67|18|2|14677|75|175|82|84.05|166.41|49.92|9552.18|4093.44|6892.10|13645.62|245.60|0.00|1364.48|4093.44|4339.04|5457.92|5703.52|-2798.66| +2450818|70706|2450830|60519|1858010|3148|32639|60519|1858010|3148|32639|2|40|18|1|13780|8|176|20|12.28|18.42|4.60|276.40|92.00|245.60|368.40|2.76|0.00|29.40|92.00|94.76|121.40|124.16|-153.60| +2450818|70706|2450834|60519|1858010|3148|32639|60519|1858010|3148|32639|2|22|14|4|9445|179|176|45|75.21|195.54|183.80|528.30|8271.00|3384.45|8799.30|165.42|0.00|1495.80|8271.00|8436.42|9766.80|9932.22|4886.55| +2450818|70706|2450899|60519|1858010|3148|32639|60519|1858010|3148|32639|2|107|16|3|15548|219|176|43|52.87|91.99|78.19|593.40|3362.17|2273.41|3955.57|302.59|0.00|553.41|3362.17|3664.76|3915.58|4218.17|1088.76| +2450818|70706|2450861|60519|1858010|3148|32639|60519|1858010|3148|32639|2|64|12|2|15619|32|176|56|33.01|53.47|48.12|299.60|2694.72|1848.56|2994.32|29.91|1697.67|149.52|997.05|1026.96|1146.57|1176.48|-851.51| +2450818|70706|2450851|60519|1858010|3148|32639|60519|1858010|3148|32639|2|11|3|3|17731|115|176|66|80.97|122.26|86.80|2340.36|5728.80|5344.02|8069.16|54.99|5041.34|2985.18|687.46|742.45|3672.64|3727.63|-4656.56| +2450818|70706|2450829|60519|1858010|3148|32639|60519|1858010|3148|32639|2|25|19|1|8791|250|176|3|93.30|239.78|31.17|625.83|93.51|279.90|719.34|3.74|0.00|352.47|93.51|97.25|445.98|449.72|-186.39| +2450818|70706|2450867|60519|1858010|3148|32639|60519|1858010|3148|32639|2|47|13|3|11522|292|176|45|3.86|5.94|4.39|69.75|197.55|173.70|267.30|13.82|0.00|53.10|197.55|211.37|250.65|264.47|23.85| +2450818|70706|2450904|60519|1858010|3148|32639|60519|1858010|3148|32639|2|50|5|1|5432|143|176|19|81.09|103.79|8.30|1814.31|157.70|1540.71|1972.01|14.19|0.00|966.15|157.70|171.89|1123.85|1138.04|-1383.01| +2450818|70706|2450832|60519|1858010|3148|32639|60519|1858010|3148|32639|2|45|9|1|3262|192|176|14|82.08|240.49|182.77|808.08|2558.78|1149.12|3366.86|25.58|0.00|976.36|2558.78|2584.36|3535.14|3560.72|1409.66| +2450818|70706|2450839|60519|1858010|3148|32639|60519|1858010|3148|32639|2|35|17|2|1327|90|176|29|98.31|271.33|127.52|4170.49|3698.08|2850.99|7868.57|0.00|2884.50|2281.72|813.58|813.58|3095.30|3095.30|-2037.41| +2450818|75805|2450827|80474|411959|6997|36835|80474|411959|6997|36835|2|17|8|5|4234|226|177|80|57.06|136.94|101.33|2848.80|8106.40|4564.80|10955.20|648.51|0.00|4491.20|8106.40|8754.91|12597.60|13246.11|3541.60| +2450818|75805|2450823|80474|411959|6997|36835|80474|411959|6997|36835|2|24|10|1|8251|101|177|84|79.67|184.03|160.10|2010.12|13448.40|6692.28|15458.52|0.00|0.00|5565.00|13448.40|13448.40|19013.40|19013.40|6756.12| +2450818|75805|2450887|80474|411959|6997|36835|80474|411959|6997|36835|2|50|18|5|9188|91|177|68|91.62|172.24|167.07|351.56|11360.76|6230.16|11712.32|568.03|0.00|819.40|11360.76|11928.79|12180.16|12748.19|5130.60| +2450818|75805|2450859|80474|411959|6997|36835|80474|411959|6997|36835|2|7|14|2|4802|39|177|74|9.60|9.60|8.54|78.44|631.96|710.40|710.40|56.87|0.00|184.26|631.96|688.83|816.22|873.09|-78.44| +2450818|75805|2450829|80474|411959|6997|36835|80474|411959|6997|36835|2|65|13|2|14098|197|177|84|59.69|168.92|141.89|2270.52|11918.76|5013.96|14189.28|595.93|0.00|141.12|11918.76|12514.69|12059.88|12655.81|6904.80| +2450818|75805|2450839|80474|411959|6997|36835|80474|411959|6997|36835|2|68|5|1|1369|53|177|39|16.63|43.23|14.69|1113.06|572.91|648.57|1685.97|51.56|0.00|421.20|572.91|624.47|994.11|1045.67|-75.66| +2450818|75805|2450896|80474|411959|6997|36835|80474|411959|6997|36835|2|15|6|2|6505|152|177|47|11.76|28.81|25.64|148.99|1205.08|552.72|1354.07|12.05|0.00|609.12|1205.08|1217.13|1814.20|1826.25|652.36| +2450818|75805|2450850|80474|411959|6997|36835|80474|411959|6997|36835|2|74|10|1|2851|261|177|76|4.42|13.03|5.73|554.80|435.48|335.92|990.28|6.61|269.99|425.60|165.49|172.10|591.09|597.70|-170.43| +2450818|76925|2450852|8338|596319|3053|44234|8338|596319|3053|44234|4|44|2|3|7522|252|178|16|56.02|112.04|63.86|770.88|1021.76|896.32|1792.64|20.43|0.00|107.52|1021.76|1042.19|1129.28|1149.71|125.44| +2450818|76925|2450886|8338|596319|3053|44234|8338|596319|3053|44234|4|89|4|4|3254|24|178|89|94.09|129.84|67.51|5547.37|6008.39|8374.01|11555.76|480.67|0.00|230.51|6008.39|6489.06|6238.90|6719.57|-2365.62| +2450818|76925|2450892|8338|596319|3053|44234|8338|596319|3053|44234|4|12|6|3|10327|47|178|80|85.29|180.81|83.17|7811.20|6653.60|6823.20|14464.80|399.21|0.00|433.60|6653.60|7052.81|7087.20|7486.41|-169.60| +2450818|76925|2450823|8338|596319|3053|44234|8338|596319|3053|44234|4|87|11|2|17840|57|178|7|92.48|122.07|6.10|811.79|42.70|647.36|854.49|0.38|38.43|410.13|4.27|4.65|414.40|414.78|-643.09| +2450818|76925|2450897|8338|596319|3053|44234|8338|596319|3053|44234|4|85|4|4|11408|154|178|14|21.31|38.35|28.76|134.26|402.64|298.34|536.90|2.25|370.42|5.32|32.22|34.47|37.54|39.79|-266.12| +2450818|76925|2450897|8338|596319|3053|44234|8338|596319|3053|44234|4|64|15|5|1030|215|178|25|39.80|91.93|36.77|1379.00|919.25|995.00|2298.25|1.93|726.20|413.50|193.05|194.98|606.55|608.48|-801.95| +2450818|76925|2450876|8338|596319|3053|44234|8338|596319|3053|44234|4|22|12|1|4340|147|178|9|19.93|31.48|27.70|34.02|249.30|179.37|283.32|6.98|162.04|90.63|87.26|94.24|177.89|184.87|-92.11| +2450818|76925|2450873|8338|596319|3053|44234|8338|596319|3053|44234|4|95|17|2|6535|92|178|16|88.34|234.10|215.37|299.68|3445.92|1413.44|3745.60|310.13|0.00|1648.00|3445.92|3756.05|5093.92|5404.05|2032.48| +2450818|76925|2450848|8338|596319|3053|44234|8338|596319|3053|44234|4|9|18|1|15466|294|178|10|87.70|208.72|20.87|1878.50|208.70|877.00|2087.20|18.78|0.00|500.90|208.70|227.48|709.60|728.38|-668.30| +2450818|76925|2450864|8338|596319|3053|44234|8338|596319|3053|44234|4|2|2|2|12457|180|178|84|31.30|77.31|76.53|65.52|6428.52|2629.20|6494.04|128.57|0.00|1688.40|6428.52|6557.09|8116.92|8245.49|3799.32| +2450818|81545|2450853|69877|486575|6473|39971|69877|486575|6473|39971|1|26|9|1|2287|112|179|23|97.87|228.03|13.68|4930.05|314.64|2251.01|5244.69|2.64|248.56|2464.91|66.08|68.72|2530.99|2533.63|-2184.93| +2450818|81545|2450894|69877|486575|6473|39971|69877|486575|6473|39971|1|92|11|2|6704|160|179|30|27.30|39.31|11.79|825.60|353.70|819.00|1179.30|0.00|0.00|235.80|353.70|353.70|589.50|589.50|-465.30| +2450818|81545|2450851|69877|486575|6473|39971|69877|486575|6473|39971|1|8|17|1|8980|41|179|37|34.97|61.54|4.92|2094.94|182.04|1293.89|2276.98|16.38|0.00|614.57|182.04|198.42|796.61|812.99|-1111.85| +2450818|81545|2450845|69877|486575|6473|39971|69877|486575|6473|39971|1|81|20|3|17680|299|179|13|12.51|30.39|24.00|83.07|312.00|162.63|395.07|15.60|0.00|75.01|312.00|327.60|387.01|402.61|149.37| +2450818|81545|2450878|69877|486575|6473|39971|69877|486575|6473|39971|1|30|5|4|3736|9|179|84|86.76|191.73|13.42|14978.04|1127.28|7287.84|16105.32|0.00|349.45|7246.68|777.83|777.83|8024.51|8024.51|-6510.01| +2450818|81545|2450904|69877|486575|6473|39971|69877|486575|6473|39971|1|106|10|1|4480|41|179|58|4.46|6.69|2.54|240.70|147.32|258.68|388.02|13.25|0.00|143.26|147.32|160.57|290.58|303.83|-111.36| +2450818|81545|2450876|69877|486575|6473|39971|69877|486575|6473|39971|1|27|6|5|94|142|179|24|68.94|79.97|29.58|1209.36|709.92|1654.56|1919.28|63.89|0.00|114.96|709.92|773.81|824.88|888.77|-944.64| +2450818|81545|2450884|69877|486575|6473|39971|69877|486575|6473|39971|1|80|1|5|8161|121|179|76|24.38|24.86|7.95|1285.16|604.20|1852.88|1889.36|24.16|0.00|471.96|604.20|628.36|1076.16|1100.32|-1248.68| +2450818|44203|2450849|12556|1841322|6535|44101|12556|1841322|6535|44101|2|1|11|2|12151|104|180|97|41.49|100.40|58.23|4090.49|5648.31|4024.53|9738.80|220.28|1242.62|3895.52|4405.69|4625.97|8301.21|8521.49|381.16| +2450818|44203|2450896|12556|1841322|6535|44101|12556|1841322|6535|44101|2|95|14|3|5074|68|180|83|62.07|117.31|42.23|6231.64|3505.09|5151.81|9736.73|210.30|0.00|389.27|3505.09|3715.39|3894.36|4104.66|-1646.72| +2450818|44203|2450877|12556|1841322|6535|44101|12556|1841322|6535|44101|2|27|8|1|9152|187|180|37|52.22|69.45|6.25|2338.40|231.25|1932.14|2569.65|18.50|0.00|513.93|231.25|249.75|745.18|763.68|-1700.89| +2450818|44203|2450906|12556|1841322|6535|44101|12556|1841322|6535|44101|2|36|4|1|10184|137|180|29|65.68|182.59|178.93|106.14|5188.97|1904.72|5295.11|155.66|0.00|1376.63|5188.97|5344.63|6565.60|6721.26|3284.25| +2450818|44203|2450873|12556|1841322|6535|44101|12556|1841322|6535|44101|2|35|7|2|43|209|180|76|45.98|123.68|55.65|5170.28|4229.40|3494.48|9399.68|380.64|0.00|1785.24|4229.40|4610.04|6014.64|6395.28|734.92| +2450818|44203|2450856|12556|1841322|6535|44101|12556|1841322|6535|44101|2|85|14|1|1213|139|180|16|22.12|53.08|28.13|399.20|450.08|353.92|849.28|27.00|0.00|110.40|450.08|477.08|560.48|587.48|96.16| +2450818|44203|2450844|12556|1841322|6535|44101|12556|1841322|6535|44101|2|36|16|3|17380|5|180|18|85.61|235.42|233.06|42.48|4195.08|1540.98|4237.56|41.95|0.00|42.30|4195.08|4237.03|4237.38|4279.33|2654.10| +2450818|44203|2450879|12556|1841322|6535|44101|12556|1841322|6535|44101|2|55|7|1|14474|207|180|86|7.86|14.61|11.54|264.02|992.44|675.96|1256.46|89.31|0.00|426.56|992.44|1081.75|1419.00|1508.31|316.48| +2450818|44203|2450837|12556|1841322|6535|44101|12556|1841322|6535|44101|2|48|3|5|9484|214|180|69|68.41|81.40|59.42|1516.62|4099.98|4720.29|5616.60|327.99|0.00|1235.10|4099.98|4427.97|5335.08|5663.07|-620.31| +2450818|44203|2450898|12556|1841322|6535|44101|12556|1841322|6535|44101|2|5|4|4|6484|108|180|62|41.00|41.41|1.65|2465.12|102.30|2542.00|2567.42|9.20|0.00|898.38|102.30|111.50|1000.68|1009.88|-2439.70| +2450818|44203|2450902|12556|1841322|6535|44101|12556|1841322|6535|44101|2|10|18|5|7921|282|180|46|72.32|131.62|130.30|60.72|5993.80|3326.72|6054.52|359.62|0.00|1029.02|5993.80|6353.42|7022.82|7382.44|2667.08| +2450818|44203|2450900|12556|1841322|6535|44101|12556|1841322|6535|44101|2|99|9|5|15511|70|180|20|44.51|60.08|25.23|697.00|504.60|890.20|1201.60|35.32|0.00|468.60|504.60|539.92|973.20|1008.52|-385.60| +2450818|64825|2450855|9542|557045|548|48051|9542|557045|548|48051|2|86|8|1|11653|219|181|78|14.46|25.88|18.37|585.78|1432.86|1127.88|2018.64|42.98|0.00|726.18|1432.86|1475.84|2159.04|2202.02|304.98| +2450818|64825|2450834|9542|557045|548|48051|9542|557045|548|48051|2|56|9|4|7402|272|181|92|80.36|227.41|145.54|7532.04|13389.68|7393.12|20921.72|535.58|0.00|2092.08|13389.68|13925.26|15481.76|16017.34|5996.56| +2450818|64825|2450896|9542|557045|548|48051|9542|557045|548|48051|2|47|3|5|4600|237|181|47|42.93|53.66|32.73|983.71|1538.31|2017.71|2522.02|76.91|0.00|377.88|1538.31|1615.22|1916.19|1993.10|-479.40| +2450818|64825|2450898|9542|557045|548|48051|9542|557045|548|48051|2|53|10|3|6619|243|181|47|76.86|132.19|42.30|4224.83|1988.10|3612.42|6212.93|19.88|0.00|372.71|1988.10|2007.98|2360.81|2380.69|-1624.32| +2450818|20732|2450881|14349|469749|5596|13396|14349|469749|5596|13396|1|27|19|2|10072|241|182|22|19.56|34.81|18.10|367.62|398.20|430.32|765.82|27.87|0.00|107.14|398.20|426.07|505.34|533.21|-32.12| +2450818|20732|2450879|14349|469749|5596|13396|14349|469749|5596|13396|1|31|7|2|11438|30|182|59|70.91|194.29|104.91|5273.42|6189.69|4183.69|11463.11|0.00|0.00|1145.78|6189.69|6189.69|7335.47|7335.47|2006.00| +2450818|20732|2450861|14349|469749|5596|13396|14349|469749|5596|13396|1|25|17|2|16858|223|182|19|90.75|137.03|54.81|1562.18|1041.39|1724.25|2603.57|62.48|0.00|234.27|1041.39|1103.87|1275.66|1338.14|-682.86| +2450818|20732|2450869|14349|469749|5596|13396|14349|469749|5596|13396|1|29|18|3|14761|60|182|94|71.31|111.24|60.06|4810.92|5645.64|6703.14|10456.56|282.28|0.00|3345.46|5645.64|5927.92|8991.10|9273.38|-1057.50| +2450818|20732|2450857|14349|469749|5596|13396|14349|469749|5596|13396|1|65|20|4|4816|89|182|64|4.28|7.70|6.31|88.96|403.84|273.92|492.80|2.10|298.84|49.28|105.00|107.10|154.28|156.38|-168.92| +2450818|20732|2450821|14349|469749|5596|13396|14349|469749|5596|13396|1|97|19|4|826|147|182|92|72.92|118.85|26.14|8529.32|2404.88|6708.64|10934.20|24.04|0.00|2951.36|2404.88|2428.92|5356.24|5380.28|-4303.76| +2450818|20732|2450821|14349|469749|5596|13396|14349|469749|5596|13396|1|9|8|5|5698|117|182|13|29.26|71.97|71.97|0.00|935.61|380.38|935.61|9.35|0.00|177.71|935.61|944.96|1113.32|1122.67|555.23| +2450818|20732|2450862|14349|469749|5596|13396|14349|469749|5596|13396|1|14|1|2|15236|90|182|25|41.84|55.22|3.31|1297.75|82.75|1046.00|1380.50|4.96|0.00|55.00|82.75|87.71|137.75|142.71|-963.25| +2450818|20732|2450874|14349|469749|5596|13396|14349|469749|5596|13396|1|20|9|4|10636|176|182|77|52.53|59.88|11.97|3689.07|921.69|4044.81|4610.76|82.95|0.00|783.09|921.69|1004.64|1704.78|1787.73|-3123.12| +2450818|20732|2450862|14349|469749|5596|13396|14349|469749|5596|13396|1|49|10|2|16712|29|182|86|11.71|25.29|0.75|2110.44|64.50|1007.06|2174.94|1.29|0.00|869.46|64.50|65.79|933.96|935.25|-942.56| +2450818|20732|2450895|14349|469749|5596|13396|14349|469749|5596|13396|1|11|7|1|3752|214|182|77|15.72|27.03|24.59|187.88|1893.43|1210.44|2081.31|34.83|1022.45|166.32|870.98|905.81|1037.30|1072.13|-339.46| +2450818|20732|2450838|14349|469749|5596|13396|14349|469749|5596|13396|1|50|10|4|14713|2|182|70|28.09|70.78|23.35|3320.10|1634.50|1966.30|4954.60|32.69|0.00|1733.90|1634.50|1667.19|3368.40|3401.09|-331.80| +2450818|20732|2450871|14349|469749|5596|13396|14349|469749|5596|13396|1|36|6|4|17240|179|182|91|79.80|102.94|101.91|93.73|9273.81|7261.80|9367.54|370.95|0.00|3559.01|9273.81|9644.76|12832.82|13203.77|2012.01| +2450818|20732|2450881|14349|469749|5596|13396|14349|469749|5596|13396|1|89|6|5|8116|30|182|33|66.97|117.19|73.82|1431.21|2436.06|2210.01|3867.27|73.08|0.00|966.57|2436.06|2509.14|3402.63|3475.71|226.05| +2450818|37909|2450860|72026|1105070|3361|36805|72026|1105070|3361|36805|2|6|13|5|15488|144|183|28|6.36|6.74|2.08|130.48|58.24|178.08|188.72|0.44|36.10|5.60|22.14|22.58|27.74|28.18|-155.94| +2450818|37909|2450878|72026|1105070|3361|36805|72026|1105070|3361|36805|2|84|1|5|2311|109|183|97|70.96|149.01|128.14|2024.39|12429.58|6883.12|14453.97|372.88|0.00|6792.91|12429.58|12802.46|19222.49|19595.37|5546.46| +2450818|37909|2450850|72026|1105070|3361|36805|72026|1105070|3361|36805|2|46|6|1|4664|123|183|37|15.48|43.65|36.22|274.91|1340.14|572.76|1615.05|80.40|0.00|646.02|1340.14|1420.54|1986.16|2066.56|767.38| +2450818|37909|2450865|72026|1105070|3361|36805|72026|1105070|3361|36805|2|51|17|1|565|64|183|81|83.75|128.13|53.81|6019.92|4358.61|6783.75|10378.53|87.17|0.00|4151.25|4358.61|4445.78|8509.86|8597.03|-2425.14| +2450818|37909|2450846|72026|1105070|3361|36805|72026|1105070|3361|36805|2|76|10|5|7369|55|183|82|41.71|61.73|13.58|3948.30|1113.56|3420.22|5061.86|33.40|0.00|252.56|1113.56|1146.96|1366.12|1399.52|-2306.66| +2450818|56639|2450820|42045|804013|3934|18302|42045|804013|3934|18302|4|85|17|4|1004|25|184|100|47.55|51.82|19.69|3213.00|1969.00|4755.00|5182.00|177.21|0.00|725.00|1969.00|2146.21|2694.00|2871.21|-2786.00| +2450818|56639|2450842|42045|804013|3934|18302|42045|804013|3934|18302|4|32|5|1|2404|122|184|43|89.78|112.22|12.34|4294.84|530.62|3860.54|4825.46|4.35|313.06|433.87|217.56|221.91|651.43|655.78|-3642.98| +2450818|56639|2450860|42045|804013|3934|18302|42045|804013|3934|18302|4|11|18|5|2593|166|184|44|74.14|94.15|2.82|4018.52|124.08|3262.16|4142.60|3.72|0.00|2029.72|124.08|127.80|2153.80|2157.52|-3138.08| +2450818|56639|2450835|42045|804013|3934|18302|42045|804013|3934|18302|4|24|2|2|9169|260|184|87|87.52|151.40|122.63|2502.99|10668.81|7614.24|13171.80|533.44|0.00|5268.72|10668.81|11202.25|15937.53|16470.97|3054.57| +2450818|56639|2450834|42045|804013|3934|18302|42045|804013|3934|18302|4|87|2|5|12332|192|184|59|43.55|127.16|127.16|0.00|7502.44|2569.45|7502.44|450.14|0.00|2100.40|7502.44|7952.58|9602.84|10052.98|4932.99| +2450818|56639|2450887|42045|804013|3934|18302|42045|804013|3934|18302|4|101|17|4|3685|45|184|20|90.02|212.44|212.44|0.00|4248.80|1800.40|4248.80|84.97|0.00|1147.00|4248.80|4333.77|5395.80|5480.77|2448.40| +2450818|55540|2450870|99856|98428|3400|772|99856|98428|3400|772|4|100|3|4|2038|55|185|49|42.92|54.07|52.98|53.41|2596.02|2103.08|2649.43|207.68|0.00|1245.09|2596.02|2803.70|3841.11|4048.79|492.94| +2450818|55540|2450892|99856|98428|3400|772|99856|98428|3400|772|4|69|10|3|9961|7|185|99|91.19|120.37|58.98|6077.61|5839.02|9027.81|11916.63|0.00|0.00|1429.56|5839.02|5839.02|7268.58|7268.58|-3188.79| +2450818|55540|2450900|99856|98428|3400|772|99856|98428|3400|772|4|17|4|1|15572|25|185|36|6.36|7.63|2.82|173.16|101.52|228.96|274.68|5.30|42.63|87.84|58.89|64.19|146.73|152.03|-170.07| +2450818|55540|2450873|99856|98428|3400|772|99856|98428|3400|772|4|101|5|1|13844|163|185|63|28.20|53.29|30.90|1410.57|1946.70|1776.60|3357.27|77.86|0.00|1409.94|1946.70|2024.56|3356.64|3434.50|170.10| +2450818|55540|2450863|99856|98428|3400|772|99856|98428|3400|772|4|85|8|2|3184|100|185|31|21.42|58.47|29.23|906.44|906.13|664.02|1812.57|27.18|0.00|761.05|906.13|933.31|1667.18|1694.36|242.11| +2450818|55540|2450846|99856|98428|3400|772|99856|98428|3400|772|4|102|14|5|6800|102|185|95|20.28|32.04|28.83|304.95|2738.85|1926.60|3043.80|246.49|0.00|1034.55|2738.85|2985.34|3773.40|4019.89|812.25| +2450818|35643|2450887|27990|569558|3126|8694|27990|569558|3126|8694|2|69|3|3|7732|75|186|58|3.75|11.25|3.60|443.70|208.80|217.50|652.50|14.61|0.00|32.48|208.80|223.41|241.28|255.89|-8.70| +2450818|35643|2450825|27990|569558|3126|8694|27990|569558|3126|8694|2|62|17|4|7786|264|186|6|29.28|57.09|26.83|181.56|160.98|175.68|342.54|3.91|30.58|130.14|130.40|134.31|260.54|264.45|-45.28| +2450818|35643|2450885|27990|569558|3126|8694|27990|569558|3126|8694|2|25|14|3|9079|128|186|41|80.74|178.43|58.88|4901.55|2414.08|3310.34|7315.63|96.56|0.00|3072.54|2414.08|2510.64|5486.62|5583.18|-896.26| +2450818|35643|2450831|27990|569558|3126|8694|27990|569558|3126|8694|2|49|15|1|15980|255|186|6|60.36|159.35|78.08|487.62|468.48|362.16|956.10|4.68|0.00|191.22|468.48|473.16|659.70|664.38|106.32| +2450818|35643|2450888|27990|569558|3126|8694|27990|569558|3126|8694|2|71|18|5|3511|259|186|87|70.83|114.03|91.22|1984.47|7936.14|6162.21|9920.61|317.44|0.00|2181.96|7936.14|8253.58|10118.10|10435.54|1773.93| +2450818|35643|2450869|27990|569558|3126|8694|27990|569558|3126|8694|2|18|18|4|15115|49|186|92|44.64|51.33|22.07|2691.92|2030.44|4106.88|4722.36|162.43|0.00|46.92|2030.44|2192.87|2077.36|2239.79|-2076.44| +2450818|35643|2450906|27990|569558|3126|8694|27990|569558|3126|8694|2|32|10|4|9625|111|186|95|70.33|164.57|161.27|313.50|15320.65|6681.35|15634.15|153.20|0.00|2344.60|15320.65|15473.85|17665.25|17818.45|8639.30| +2450818|44226|2450876|88713|1642254|1748|3242|88713|1642254|1748|3242|2|15|4|1|14455|215|187|16|79.06|81.43|15.47|1055.36|247.52|1264.96|1302.88|14.85|0.00|195.36|247.52|262.37|442.88|457.73|-1017.44| +2450818|44226|2450821|88713|1642254|1748|3242|88713|1642254|1748|3242|2|44|12|3|16588|215|187|36|34.88|47.78|32.49|550.44|1169.64|1255.68|1720.08|0.00|0.00|704.88|1169.64|1169.64|1874.52|1874.52|-86.04| +2450818|44226|2450822|88713|1642254|1748|3242|88713|1642254|1748|3242|2|50|4|5|12676|252|187|7|43.00|118.25|14.19|728.42|99.33|301.00|827.75|3.97|0.00|206.92|99.33|103.30|306.25|310.22|-201.67| +2450818|44226|2450851|88713|1642254|1748|3242|88713|1642254|1748|3242|2|11|9|4|8773|292|187|71|25.41|36.33|15.62|1470.41|1109.02|1804.11|2579.43|66.54|0.00|489.90|1109.02|1175.56|1598.92|1665.46|-695.09| +2450818|44226|2450900|88713|1642254|1748|3242|88713|1642254|1748|3242|2|105|20|4|958|130|187|35|41.15|107.81|33.42|2603.65|1169.70|1440.25|3773.35|35.09|0.00|981.05|1169.70|1204.79|2150.75|2185.84|-270.55| +2450818|30908|2450861|21064|1800220|2129|26485|21064|1800220|2129|26485|2|92|15|5|2704|138|188|95|37.83|86.25|57.78|2704.65|5489.10|3593.85|8193.75|494.01|0.00|1311.00|5489.10|5983.11|6800.10|7294.11|1895.25| +2450818|30908|2450877|21064|1800220|2129|26485|21064|1800220|2129|26485|2|71|8|1|6964|39|188|79|52.33|111.46|22.29|7044.43|1760.91|4134.07|8805.34|0.00|1743.30|616.20|17.61|17.61|633.81|633.81|-4116.46| +2450818|30908|2450834|21064|1800220|2129|26485|21064|1800220|2129|26485|2|87|20|4|2275|300|188|48|88.07|132.98|86.43|2234.40|4148.64|4227.36|6383.04|82.97|0.00|2936.16|4148.64|4231.61|7084.80|7167.77|-78.72| +2450818|30908|2450858|21064|1800220|2129|26485|21064|1800220|2129|26485|2|19|13|3|2990|68|188|31|13.79|22.89|12.58|319.61|389.98|427.49|709.59|15.59|0.00|305.04|389.98|405.57|695.02|710.61|-37.51| +2450818|30908|2450873|21064|1800220|2129|26485|21064|1800220|2129|26485|2|3|17|1|2060|7|188|7|59.05|124.59|62.29|436.10|436.03|413.35|872.13|34.88|0.00|26.11|436.03|470.91|462.14|497.02|22.68| +2450818|30908|2450907|21064|1800220|2129|26485|21064|1800220|2129|26485|2|40|9|1|10339|17|188|61|94.55|152.22|95.89|3436.13|5849.29|5767.55|9285.42|292.46|0.00|3435.52|5849.29|6141.75|9284.81|9577.27|81.74| +2450818|30908|2450889|21064|1800220|2129|26485|21064|1800220|2129|26485|2|43|6|1|15886|93|188|76|30.56|59.28|49.79|721.24|3784.04|2322.56|4505.28|227.04|0.00|405.08|3784.04|4011.08|4189.12|4416.16|1461.48| +2450818|30908|2450901|21064|1800220|2129|26485|21064|1800220|2129|26485|2|95|12|3|11188|259|188|12|64.93|154.53|137.53|204.00|1650.36|779.16|1854.36|66.01|0.00|686.04|1650.36|1716.37|2336.40|2402.41|871.20| +2450818|30908|2450845|21064|1800220|2129|26485|21064|1800220|2129|26485|2|108|3|3|9829|15|188|23|6.62|11.51|3.22|190.67|74.06|152.26|264.73|6.66|0.00|92.46|74.06|80.72|166.52|173.18|-78.20| +2450818|30908|2450845|21064|1800220|2129|26485|21064|1800220|2129|26485|2|51|19|3|10778|163|188|75|25.91|76.69|3.06|5522.25|229.50|1943.25|5751.75|11.47|0.00|1494.75|229.50|240.97|1724.25|1735.72|-1713.75| +2450818|30908|2450821|21064|1800220|2129|26485|21064|1800220|2129|26485|2|29|6|2|5552|149|188|100|92.14|274.57|151.01|12356.00|15101.00|9214.00|27457.00|253.69|10872.72|8786.00|4228.28|4481.97|13014.28|13267.97|-4985.72| +2450818|30908|2450864|21064|1800220|2129|26485|21064|1800220|2129|26485|2|7|17|1|1981|109|188|73|20.43|55.77|6.69|3582.84|488.37|1491.39|4071.21|39.06|0.00|0.00|488.37|527.43|488.37|527.43|-1003.02| +2450818|30908|2450856|21064|1800220|2129|26485|21064|1800220|2129|26485|2|34|14|1|8230|148|188|7|59.39|125.90|55.39|493.57|387.73|415.73|881.30|3.87|290.79|423.01|96.94|100.81|519.95|523.82|-318.79| +2450818|65052|2450891|89219|1656939|4659|4379|29441|1165419|5139|41259|4|76|16|1|10604|101|189|35|7.26|16.84|2.18|513.10|76.30|254.10|589.40|6.86|0.00|235.55|76.30|83.16|311.85|318.71|-177.80| +2450818|65052|2450837|89219|1656939|4659|4379|29441|1165419|5139|41259|4|84|1|4|12853|135|189|26|5.75|13.68|10.53|81.90|273.78|149.50|355.68|6.57|191.64|156.26|82.14|88.71|238.40|244.97|-67.36| +2450818|65052|2450824|89219|1656939|4659|4379|29441|1165419|5139|41259|4|15|14|5|6518|152|189|38|19.77|45.86|43.56|87.40|1655.28|751.26|1742.68|33.10|0.00|296.02|1655.28|1688.38|1951.30|1984.40|904.02| +2450818|65052|2450833|89219|1656939|4659|4379|29441|1165419|5139|41259|4|41|10|1|3104|1|189|40|10.12|25.09|24.83|10.40|993.20|404.80|1003.60|79.45|0.00|140.40|993.20|1072.65|1133.60|1213.05|588.40| +2450818|65052|2450832|89219|1656939|4659|4379|29441|1165419|5139|41259|4|100|7|5|14599|229|189|46|58.00|109.62|13.15|4437.62|604.90|2668.00|5042.52|24.19|0.00|1663.82|604.90|629.09|2268.72|2292.91|-2063.10| +2450818|65052|2450886|89219|1656939|4659|4379|29441|1165419|5139|41259|4|53|1|4|1094|156|189|26|35.15|72.40|26.78|1186.12|696.28|913.90|1882.40|62.66|0.00|922.22|696.28|758.94|1618.50|1681.16|-217.62| +2450818|73536|2450867|601|1565266|331|30196|601|1565266|331|30196|1|51|5|5|11558|155|190|17|22.57|45.81|3.66|716.55|62.22|383.69|778.77|5.59|0.00|225.76|62.22|67.81|287.98|293.57|-321.47| +2450818|73536|2450826|601|1565266|331|30196|601|1565266|331|30196|1|8|9|3|2684|199|190|74|32.21|54.43|8.16|3423.98|603.84|2383.54|4027.82|36.23|0.00|402.56|603.84|640.07|1006.40|1042.63|-1779.70| +2450818|73536|2450902|601|1565266|331|30196|601|1565266|331|30196|1|27|7|3|6536|90|190|96|12.74|21.02|1.26|1896.96|120.96|1223.04|2017.92|7.25|0.00|20.16|120.96|128.21|141.12|148.37|-1102.08| +2450818|73536|2450853|601|1565266|331|30196|601|1565266|331|30196|1|7|14|5|7358|42|190|45|28.50|62.13|20.50|1873.35|922.50|1282.50|2795.85|33.94|73.80|726.75|848.70|882.64|1575.45|1609.39|-433.80| +2450818|73536|2450896|601|1565266|331|30196|601|1565266|331|30196|1|83|11|1|13468|76|190|31|75.19|112.03|95.22|521.11|2951.82|2330.89|3472.93|113.35|1062.65|69.44|1889.17|2002.52|1958.61|2071.96|-441.72| +2450818|73536|2450852|601|1565266|331|30196|601|1565266|331|30196|1|9|13|3|4286|62|190|84|43.03|94.23|21.67|6095.04|1820.28|3614.52|7915.32|18.20|0.00|1344.84|1820.28|1838.48|3165.12|3183.32|-1794.24| +2450818|1011|2450821|45202|1667372|1542|42465|45202|1667372|1542|42465|4|79|10|3|1690|296|191|29|47.88|109.16|108.06|31.90|3133.74|1388.52|3165.64|0.00|0.00|379.61|3133.74|3133.74|3513.35|3513.35|1745.22| +2450818|1011|2450883|45202|1667372|1542|42465|45202|1667372|1542|42465|4|9|20|3|7687|126|191|35|80.61|117.69|36.48|2842.35|1276.80|2821.35|4119.15|51.07|0.00|1894.55|1276.80|1327.87|3171.35|3222.42|-1544.55| +2450818|1011|2450867|45202|1667372|1542|42465|45202|1667372|1542|42465|4|69|17|2|16294|112|191|99|54.65|86.34|77.70|855.36|7692.30|5410.35|8547.66|615.38|0.00|1708.74|7692.30|8307.68|9401.04|10016.42|2281.95| +2450818|1011|2450856|45202|1667372|1542|42465|45202|1667372|1542|42465|4|92|13|2|5743|47|191|23|51.83|94.33|33.01|1410.36|759.23|1192.09|2169.59|7.44|652.93|954.50|106.30|113.74|1060.80|1068.24|-1085.79| +2450818|1011|2450877|45202|1667372|1542|42465|45202|1667372|1542|42465|4|11|7|1|1796|122|191|100|93.58|131.01|113.97|1704.00|11397.00|9358.00|13101.00|569.85|0.00|4192.00|11397.00|11966.85|15589.00|16158.85|2039.00| +2450818|1011|2450850|45202|1667372|1542|42465|45202|1667372|1542|42465|4|87|16|1|13598|123|191|84|77.55|207.05|134.58|6087.48|11304.72|6514.20|17392.20|452.18|0.00|3652.32|11304.72|11756.90|14957.04|15409.22|4790.52| +2450818|1011|2450821|45202|1667372|1542|42465|45202|1667372|1542|42465|4|46|15|1|12397|7|191|49|44.27|116.87|75.96|2004.59|3722.04|2169.23|5726.63|37.22|0.00|2290.26|3722.04|3759.26|6012.30|6049.52|1552.81| +2450818|1011|2450845|45202|1667372|1542|42465|45202|1667372|1542|42465|4|18|4|4|15470|194|191|72|21.91|46.88|13.12|2430.72|944.64|1577.52|3375.36|28.33|0.00|944.64|944.64|972.97|1889.28|1917.61|-632.88| +2450818|66455|2450844|52920|1153264|3561|27058|52920|1153264|3561|27058|2|9|11|4|17848|117|192|63|67.40|73.46|52.15|1342.53|3285.45|4246.20|4627.98|131.41|0.00|601.02|3285.45|3416.86|3886.47|4017.88|-960.75| +2450818|66455|2450883|52920|1153264|3561|27058|52920|1153264|3561|27058|2|11|6|1|871|64|192|96|10.94|16.62|4.32|1180.80|414.72|1050.24|1595.52|33.17|0.00|31.68|414.72|447.89|446.40|479.57|-635.52| +2450818|66455|2450822|52920|1153264|3561|27058|52920|1153264|3561|27058|2|24|17|2|4759|126|192|72|44.63|124.96|8.74|8367.84|629.28|3213.36|8997.12|8.81|453.08|2968.56|176.20|185.01|3144.76|3153.57|-3037.16| +2450818|66455|2450844|52920|1153264|3561|27058|52920|1153264|3561|27058|2|29|1|1|14242|281|192|68|44.43|77.75|72.30|370.60|4916.40|3021.24|5287.00|393.31|0.00|739.84|4916.40|5309.71|5656.24|6049.55|1895.16| +2450818|66455|2450841|52920|1153264|3561|27058|52920|1153264|3561|27058|2|56|3|4|11792|208|192|75|91.80|250.61|115.28|10149.75|8646.00|6885.00|18795.75|605.22|0.00|3570.75|8646.00|9251.22|12216.75|12821.97|1761.00| +2450818|66455|2450882|52920|1153264|3561|27058|52920|1153264|3561|27058|2|98|12|2|172|137|192|22|79.78|102.11|1.02|2223.98|22.44|1755.16|2246.42|2.01|0.00|1123.10|22.44|24.45|1145.54|1147.55|-1732.72| +2450818|66455|2450877|52920|1153264|3561|27058|52920|1153264|3561|27058|2|80|2|5|6169|28|192|53|93.78|110.66|8.85|5395.93|469.05|4970.34|5864.98|23.45|0.00|2580.57|469.05|492.50|3049.62|3073.07|-4501.29| +2450818|66455|2450844|52920|1153264|3561|27058|52920|1153264|3561|27058|2|66|17|5|7376|29|192|50|32.91|53.64|4.29|2467.50|214.50|1645.50|2682.00|8.58|0.00|134.00|214.50|223.08|348.50|357.08|-1431.00| +2450818|66455|2450865|52920|1153264|3561|27058|52920|1153264|3561|27058|2|82|12|4|8198|93|192|45|28.27|32.79|2.29|1372.50|103.05|1272.15|1475.55|7.21|0.00|339.30|103.05|110.26|442.35|449.56|-1169.10| +2450818|66455|2450905|52920|1153264|3561|27058|52920|1153264|3561|27058|2|27|17|5|2824|177|192|92|62.77|69.04|0.00|6351.68|0.00|5774.84|6351.68|0.00|0.00|825.24|0.00|0.00|825.24|825.24|-5774.84| +2450818|66455|2450886|52920|1153264|3561|27058|52920|1153264|3561|27058|2|97|17|4|8110|298|192|70|12.44|30.97|7.74|1626.10|541.80|870.80|2167.90|21.67|0.00|324.80|541.80|563.47|866.60|888.27|-329.00| +2450818|66455|2450855|52920|1153264|3561|27058|52920|1153264|3561|27058|2|56|3|1|10310|91|192|8|93.80|270.14|62.13|1664.08|497.04|750.40|2161.12|0.00|0.00|259.28|497.04|497.04|756.32|756.32|-253.36| +2450818|66455|2450862|52920|1153264|3561|27058|52920|1153264|3561|27058|2|100|6|5|9350|129|192|54|42.06|66.03|34.33|1711.80|1853.82|2271.24|3565.62|74.15|0.00|285.12|1853.82|1927.97|2138.94|2213.09|-417.42| +||2450879|52920||3561||52920|||||94|||355||192|1||32.97||30.67||17.92||0.06||||2.36|10.54|10.60|-15.62| +2450818|47736|2450893|4746|82447|5395|38296|4746|82447|5395|38296|4|88|20|3|10924|184|193|9|61.04|139.17|87.67|463.50|789.03|549.36|1252.53|71.01|0.00|162.81|789.03|860.04|951.84|1022.85|239.67| +2450818|47736|2450902|4746|82447|5395|38296|4746|82447|5395|38296|4|94|5|5|3530|157|193|29|27.26|70.87|24.80|1336.03|719.20|790.54|2055.23|14.38|0.00|821.86|719.20|733.58|1541.06|1555.44|-71.34| +2450818|47736|2450845|4746|82447|5395|38296|4746|82447|5395|38296|4|65|17|3|2245|155|193|79|60.76|116.65|108.48|645.43|8569.92|4800.04|9215.35|47.99|3770.76|3317.21|4799.16|4847.15|8116.37|8164.36|-0.88| +2450818|47736|2450865|4746|82447|5395|38296|4746|82447|5395|38296|4|103|20|1|1495|156|193|51|48.80|134.68|36.36|5014.32|1854.36|2488.80|6868.68|18.54|0.00|1442.28|1854.36|1872.90|3296.64|3315.18|-634.44| +2450818|47736|2450889|4746|82447|5395|38296|4746|82447|5395|38296|4|62|17|5|15565|129|193|69|79.82|138.08|99.41|2668.23|6859.29|5507.58|9527.52|480.15|0.00|3524.52|6859.29|7339.44|10383.81|10863.96|1351.71| +2450818|47736|2450903|4746|82447|5395|38296|4746|82447|5395|38296|4|33|3|1|4450|260|193|23|81.73|178.98|21.47|3622.73|493.81|1879.79|4116.54|29.62|0.00|1564.23|493.81|523.43|2058.04|2087.66|-1385.98| +2450818|47736|2450836|4746|82447|5395|38296|4746|82447|5395|38296|4|17|11|3|8084|207|193|35|79.87|92.64|71.33|745.85|2496.55|2795.45|3242.40|99.86|0.00|1523.90|2496.55|2596.41|4020.45|4120.31|-298.90| +2450818|47736|2450848|4746|82447|5395|38296|4746|82447|5395|38296|4|11|19|5|6589|120|193|79|44.18|59.64|7.15|4146.71|564.85|3490.22|4711.56|39.53|0.00|1884.15|564.85|604.38|2449.00|2488.53|-2925.37| +2450818|47736|2450859|4746|82447|5395|38296|4746|82447|5395|38296|4|66|18|3|5164|141|193|44|82.74|172.92|133.14|1750.32|5858.16|3640.56|7608.48|0.00|0.00|227.92|5858.16|5858.16|6086.08|6086.08|2217.60| +2450818|48496|2450875|83746|1206772|5445|23271|83746|1206772|5445|23271|4|16|17|4|8066|25|194|59|78.25|79.03|27.66|3030.83|1631.94|4616.75|4662.77|65.27|0.00|1864.99|1631.94|1697.21|3496.93|3562.20|-2984.81| +2450818|48496|2450890|83746|1206772|5445|23271|83746|1206772|5445|23271|4|58|2|2|170|272|194|65|15.05|40.93|40.93|0.00|2660.45|978.25|2660.45|133.02|0.00|1170.00|2660.45|2793.47|3830.45|3963.47|1682.20| +2450818|48496|2450880|83746|1206772|5445|23271|83746|1206772|5445|23271|4|19|14|2|14378|295|194|55|42.60|108.20|42.19|3630.55|2320.45|2343.00|5951.00|69.61|0.00|2975.50|2320.45|2390.06|5295.95|5365.56|-22.55| +2450818|48496|2450821|83746|1206772|5445|23271|83746|1206772|5445|23271|4|3|19|4|602|69|194|91|12.06|35.33|26.49|804.44|2410.59|1097.46|3215.03|24.10|0.00|1124.76|2410.59|2434.69|3535.35|3559.45|1313.13| +2450818|48496|2450900|83746|1206772|5445|23271|83746|1206772|5445|23271|4|96|7|3|6350|95|194|75|25.04|69.11|64.27|363.00|4820.25|1878.00|5183.25|96.40|0.00|1503.00|4820.25|4916.65|6323.25|6419.65|2942.25| +2450818|48496|2450851|83746|1206772|5445|23271|83746|1206772|5445|23271|4|11|2|4|4726|151|194|99|90.29|204.95|200.85|405.90|19884.15|8938.71|20290.05|918.64|4573.35|1419.66|15310.80|16229.44|16730.46|17649.10|6372.09| +2450818|48496|2450852|83746|1206772|5445|23271|83746|1206772|5445|23271|4|5|19|2|13771|221|194|35|92.05|252.21|12.61|8386.00|441.35|3221.75|8827.35|0.00|0.00|1412.25|441.35|441.35|1853.60|1853.60|-2780.40| +2450818|48496|2450879|83746|1206772|5445|23271|83746|1206772|5445|23271|4|50|7|3|5824|77|194|84|36.52|40.53|12.96|2315.88|1088.64|3067.68|3404.52|33.20|424.56|1123.08|664.08|697.28|1787.16|1820.36|-2403.60| +2450818|48496|2450847|83746|1206772|5445|23271|83746|1206772|5445|23271|4|59|8|5|14300|139|194|83|59.71|93.14|75.44|1469.10|6261.52|4955.93|7730.62|250.46|0.00|695.54|6261.52|6511.98|6957.06|7207.52|1305.59| +2450818|48496|2450821|83746|1206772|5445|23271|83746|1206772|5445|23271|4|51|18|5|1508|80|194|87|97.89|139.98|113.38|2314.20|9864.06|8516.43|12178.26|98.64|0.00|4140.33|9864.06|9962.70|14004.39|14103.03|1347.63| +2450818|48496|2450895|83746|1206772|5445|23271|83746|1206772|5445|23271|4|70|10|1|8878|273|194|16|68.58|102.87|8.22|1514.40|131.52|1097.28|1645.92|1.57|111.79|444.32|19.73|21.30|464.05|465.62|-1077.55| +2450818|48496|2450853|83746|1206772|5445|23271|83746|1206772|5445|23271|4|50|13|1|14504|45|194|47|96.26|112.62|23.65|4181.59|1111.55|4524.22|5293.14|44.46|0.00|1693.41|1111.55|1156.01|2804.96|2849.42|-3412.67| +2450818|48496|2450864|83746|1206772|5445|23271|83746|1206772|5445|23271|4|95|12|1|1165|86|194|93|23.92|67.69|0.67|6232.86|62.31|2224.56|6295.17|4.98|0.00|3083.88|62.31|67.29|3146.19|3151.17|-2162.25| +2450818|48496|2450830|83746|1206772|5445|23271|83746|1206772|5445|23271|4|87|7|2|7039|62|194|82|97.16|282.73|36.75|20170.36|3013.50|7967.12|23183.86|271.21|0.00|11591.52|3013.50|3284.71|14605.02|14876.23|-4953.62| +2450818|||97928|||7000||1274654|||2||6|5|2767|197|195|40||33.12||||571.20|||0.00|211.60|||1046.00|1121.09|263.20| +2450818|34415|2450888|97928|1274654|3027|7000|97928|1274654|3027|7000|2|8|7|5|11419|166|195|74|8.20|17.71|17.71|0.00|1310.54|606.80|1310.54|91.73|0.00|615.68|1310.54|1402.27|1926.22|2017.95|703.74| +2450818|34415|2450890|97928|1274654|3027|7000|97928|1274654|3027|7000|2|52|18|3|4441|110|195|29|48.27|85.92|15.46|2043.34|448.34|1399.83|2491.68|31.38|0.00|548.10|448.34|479.72|996.44|1027.82|-951.49| +2450818|34415|2450856|97928|1274654|3027|7000|97928|1274654|3027|7000|2|15|4|2|5176|193|195|83|6.94|7.21|5.33|156.04|442.39|576.02|598.43|35.03|53.08|59.76|389.31|424.34|449.07|484.10|-186.71| +2450818|34415|2450827|97928|1274654|3027|7000|97928|1274654|3027|7000|2|89|20|3|3914|25|195|37|14.00|23.66|7.33|604.21|271.21|518.00|875.42|2.68|2.71|122.47|268.50|271.18|390.97|393.65|-249.50| +2450818|17576|2450881|24082|11357|5479|22801|24082|11357|5479|22801|2|45|10|3|3260|246|196|39|42.82|108.33|71.49|1436.76|2788.11|1669.98|4224.87|83.64|0.00|2112.24|2788.11|2871.75|4900.35|4983.99|1118.13| +2450818|17576|2450872|24082|11357|5479|22801|24082|11357|5479|22801|2|40|20|1|638|181|196|76|83.68|222.58|26.70|14886.88|2029.20|6359.68|16916.08|182.62|0.00|3044.56|2029.20|2211.82|5073.76|5256.38|-4330.48| +2450818|17576|2450855|24082|11357|5479|22801|24082|11357|5479|22801|2|47|16|4|2920|60|196|45|77.59|196.30|23.55|7773.75|1059.75|3491.55|8833.50|40.05|392.10|883.35|667.65|707.70|1551.00|1591.05|-2823.90| +2450818|17576|2450873|24082|11357|5479|22801|24082|11357|5479|22801|2|19|4|4|12938|243|196|13|68.54|170.66|8.53|2107.69|110.89|891.02|2218.58|4.43|0.00|244.01|110.89|115.32|354.90|359.33|-780.13| +2450818|71845|2450899|1503|1498303|3550|28350|77234|1069597|3197|3244|4|71|7|4|2800|208|197|40|57.09|75.92|48.58|1093.60|1943.20|2283.60|3036.80|174.88|0.00|273.20|1943.20|2118.08|2216.40|2391.28|-340.40| +2450818|71845|2450860|1503|1498303|3550|28350|77234|1069597|3197|3244|4|13|14|5|13142|297|197|86|2.78|6.11|6.11|0.00|525.46|239.08|525.46|0.00|0.00|20.64|525.46|525.46|546.10|546.10|286.38| +2450818|71845|2450829|1503|1498303|3550|28350|77234|1069597|3197|3244|4|107|8|4|40|51|197|99|15.10|16.15|9.85|623.70|975.15|1494.90|1598.85|39.00|0.00|510.84|975.15|1014.15|1485.99|1524.99|-519.75| +2450818|71845|2450851|1503|1498303|3550|28350|77234|1069597|3197|3244|4|81|10|3|2206|264|197|48|31.03|47.47|11.39|1731.84|546.72|1489.44|2278.56|0.00|202.28|409.92|344.44|344.44|754.36|754.36|-1145.00| +2450818|71845|2450903|1503|1498303|3550|28350|77234|1069597|3197|3244|4|9|16|4|1942|187|197|31|14.80|31.22|16.54|455.08|512.74|458.80|967.82|10.25|0.00|48.36|512.74|522.99|561.10|571.35|53.94| +2450818|71845|2450886|1503|1498303|3550|28350|77234|1069597|3197|3244|4|88|4|2|10498|261|197|93|5.15|6.48|1.10|500.34|102.30|478.95|602.64|0.81|20.46|294.81|81.84|82.65|376.65|377.46|-397.11| +2450818|71845|2450824|1503|1498303|3550|28350|77234|1069597|3197|3244|4|29|17|4|8035|20|197|21|79.90|191.76|178.33|282.03|3744.93|1677.90|4026.96|149.79|0.00|563.64|3744.93|3894.72|4308.57|4458.36|2067.03| +2450818|71845|2450838|1503|1498303|3550|28350|77234|1069597|3197|3244|4|64|13|2|10630|95|197|48|25.50|27.03|2.16|1193.76|103.68|1224.00|1297.44|1.45|67.39|77.76|36.29|37.74|114.05|115.50|-1187.71| +2450818|71845|2450901|1503|1498303|3550|28350|77234|1069597|3197|3244|4|58|1|4|10022|19|197|64|21.63|22.06|18.30|240.64|1171.20|1384.32|1411.84|70.27|0.00|423.04|1171.20|1241.47|1594.24|1664.51|-213.12| +2450818|71845|2450873|1503|1498303|3550|28350|77234|1069597|3197|3244|4|83|3|5|16538|111|197|64|20.30|49.12|2.45|2986.88|156.80|1299.20|3143.68|10.97|0.00|314.24|156.80|167.77|471.04|482.01|-1142.40| +2450818|71845|2450893|1503|1498303|3550|28350|77234|1069597|3197|3244|4|75|9|3|1021|91|197|24|38.77|53.11|45.14|191.28|1083.36|930.48|1274.64|75.83|0.00|382.32|1083.36|1159.19|1465.68|1541.51|152.88| +2450818|66045|2450830|39758|1488272|339|15606|12192|581787|1854|39168|2|71|15|2|5638|271|198|86|16.37|44.69|42.00|231.34|3612.00|1407.82|3843.34|87.77|686.28|998.46|2925.72|3013.49|3924.18|4011.95|1517.90| +2450818|66045|2450857|39758|1488272|339|15606|12192|581787|1854|39168|2|22|8|2|2732|73|198|41|57.34|83.71|67.80|652.31|2779.80|2350.94|3432.11|83.39|0.00|411.64|2779.80|2863.19|3191.44|3274.83|428.86| +2450818|66045|2450834|39758|1488272|339|15606|12192|581787|1854|39168|2|16|9|4|11906|134|198|25|72.09|72.81|50.96|546.25|1274.00|1802.25|1820.25|12.74|0.00|236.50|1274.00|1286.74|1510.50|1523.24|-528.25| +2450818|66045|2450852|39758|1488272|339|15606|12192|581787|1854|39168|2|12|8|2|514|14|198|2|57.86|72.32|58.57|27.50|117.14|115.72|144.64|2.34|0.00|11.56|117.14|119.48|128.70|131.04|1.42| +2450818|66045|2450837|39758|1488272|339|15606|12192|581787|1854|39168|2|75|18|4|12190|26|198|85|85.80|207.63|159.87|4059.60|13588.95|7293.00|17648.55|271.77|0.00|4411.50|13588.95|13860.72|18000.45|18272.22|6295.95| +2450818|66045|2450844|39758|1488272|339|15606|12192|581787|1854|39168|2|3|1|3|9322|182|198|71|63.40|102.07|16.33|6087.54|1159.43|4501.40|7246.97|34.78|0.00|3043.06|1159.43|1194.21|4202.49|4237.27|-3341.97| +2450818|66045|2450827|39758|1488272|339|15606|12192|581787|1854|39168|2|18|6|5|11815|149|198|1|64.61|166.04|146.11|19.93|146.11|64.61|166.04|8.76|0.00|23.24|146.11|154.87|169.35|178.11|81.50| +2450818|66045|2450840|39758|1488272|339|15606|12192|581787|1854|39168|2|25|3|2|2528|190|198|50|4.57|11.47|3.78|384.50|189.00|228.50|573.50|1.89|0.00|206.00|189.00|190.89|395.00|396.89|-39.50| +2450818|41437|2450898|4975|1551063|4351|19239|4975|1551063|4351|19239|2|96|2|4|5074|110|199|7|48.12|135.21|56.78|549.01|397.46|336.84|946.47|7.94|0.00|425.88|397.46|405.40|823.34|831.28|60.62| +2450818|41437|2450821|4975|1551063|4351|19239|4975|1551063|4351|19239|2|3|15|2|9152|188|199|79|76.65|186.25|78.22|8534.37|6179.38|6055.35|14713.75|52.52|5128.88|2206.47|1050.50|1103.02|3256.97|3309.49|-5004.85| +2450818|41437|2450880|4975|1551063|4351|19239|4975|1551063|4351|19239|2|50|13|3|10184|11|199|17|1.17|1.17|1.11|1.02|18.87|19.89|19.89|0.75|0.00|9.69|18.87|19.62|28.56|29.31|-1.02| +2450818|41437|2450908|4975|1551063|4351|19239|4975|1551063|4351|19239|2|7|10|5|43|300|199|45|88.67|198.62|61.57|6167.25|2770.65|3990.15|8937.90|138.53|0.00|3574.80|2770.65|2909.18|6345.45|6483.98|-1219.50| +2450818|41437|2450850|4975|1551063|4351|19239|4975|1551063|4351|19239|2|9|1|1|1213|299|199|11|84.00|247.80|89.20|1744.60|981.20|924.00|2725.80|88.30|0.00|1281.06|981.20|1069.50|2262.26|2350.56|57.20| +2450818|41437|2450881|4975|1551063|4351|19239|4975|1551063|4351|19239|2|38|18|1|17380|110|199|78|24.77|49.78|16.92|2563.08|1319.76|1932.06|3882.84|39.59|0.00|387.66|1319.76|1359.35|1707.42|1747.01|-612.30| +2450818|41437|2450833|4975|1551063|4351|19239|4975|1551063|4351|19239|2|56|20|3|14474|136|199|12|14.04|20.07|12.84|86.76|154.08|168.48|240.84|13.86|0.00|16.80|154.08|167.94|170.88|184.74|-14.40| +2450818|41437|2450895|4975|1551063|4351|19239|4975|1551063|4351|19239|2|14|19|4|9484|237|199|81|14.18|16.16|11.47|379.89|929.07|1148.58|1308.96|37.16|0.00|51.84|929.07|966.23|980.91|1018.07|-219.51| +2450818|41437|2450855|4975|1551063|4351|19239|4975|1551063|4351|19239|2|83|6|1|6484|254|199|38|45.25|115.84|19.69|3653.70|748.22|1719.50|4401.92|67.33|0.00|1980.56|748.22|815.55|2728.78|2796.11|-971.28| +2450818|44739|2450903|41151|119382|2860|5891|41151|119382|2860|5891|2|11|8|4|11155|211|200|22|69.00|153.87|49.23|2302.08|1083.06|1518.00|3385.14|21.66|0.00|1590.82|1083.06|1104.72|2673.88|2695.54|-434.94| +2450818|44739|2450839|41151|119382|2860|5891|41151|119382|2860|5891|2|11|13|1|8030|186|200|67|11.82|30.49|25.30|347.73|1695.10|791.94|2042.83|67.80|0.00|837.50|1695.10|1762.90|2532.60|2600.40|903.16| +2450818|44739|2450871|41151|119382|2860|5891|41151|119382|2860|5891|2|40|8|2|15964|25|200|7|86.05|104.12|74.96|204.12|524.72|602.35|728.84|1.78|346.31|204.05|178.41|180.19|382.46|384.24|-423.94| +2450818|44739|2450897|41151|119382|2860|5891|41151|119382|2860|5891|2|59|20|4|6314|22|200|39|47.26|80.81|27.47|2080.26|1071.33|1843.14|3151.59|42.85|0.00|1039.74|1071.33|1114.18|2111.07|2153.92|-771.81| +2450818|44739|2450820|41151|119382|2860|5891|41151|119382|2860|5891|2|1|19|4|7294|8|200|80|27.07|60.09|45.66|1154.40|3652.80|2165.60|4807.20|109.58|0.00|1874.40|3652.80|3762.38|5527.20|5636.78|1487.20| +2450818|68744|2450821|32713|1383881|3496|44264|32713|1383881|3496|44264|4|46|13|1|15788|152|201|42|49.32|135.63|36.62|4158.42|1538.04|2071.44|5696.46|92.28|0.00|1139.04|1538.04|1630.32|2677.08|2769.36|-533.40| +2450818|68744|2450884|32713|1383881|3496|44264|32713|1383881|3496|44264|4|34|16|2|6607|9|201|89|29.01|73.97|56.95|1514.78|5068.55|2581.89|6583.33|50.68|0.00|1777.33|5068.55|5119.23|6845.88|6896.56|2486.66| +2450818|68744|2450833|32713|1383881|3496|44264|32713|1383881|3496|44264|4|79|20|5|11588|284|201|33|78.36|135.56|28.46|3534.30|939.18|2585.88|4473.48|46.95|0.00|178.86|939.18|986.13|1118.04|1164.99|-1646.70| +2450818|68744|2450860|32713|1383881|3496|44264|32713|1383881|3496|44264|4|28|13|2|11278|160|201|69|17.11|47.22|5.66|2867.64|390.54|1180.59|3258.18|31.63|39.05|162.84|351.49|383.12|514.33|545.96|-829.10| +2450818|68744|2450852|32713|1383881|3496|44264|32713|1383881|3496|44264|4|85|16|4|2677|75|201|39|87.55|225.00|128.25|3773.25|5001.75|3414.45|8775.00|50.01|0.00|1404.00|5001.75|5051.76|6405.75|6455.76|1587.30| +2450818|68744|2450899|32713|1383881|3496|44264|32713|1383881|3496|44264|4|99|7|5|14524|275|201|18|78.30|188.70|150.96|679.32|2717.28|1409.40|3396.60|54.34|0.00|916.92|2717.28|2771.62|3634.20|3688.54|1307.88| +2450818|68744|2450830|32713|1383881|3496|44264|32713|1383881|3496|44264|4|99|20|1|13622|290|201|36|92.02|273.29|185.83|3148.56|6689.88|3312.72|9838.44|535.19|0.00|983.52|6689.88|7225.07|7673.40|8208.59|3377.16| +2450818|68744|2450868|32713|1383881|3496|44264|32713|1383881|3496|44264|4|36|15|1|2425|173|201|38|79.25|164.84|93.95|2693.82|3570.10|3011.50|6263.92|142.80|0.00|2254.92|3570.10|3712.90|5825.02|5967.82|558.60| +2450818|52409|2450853|13515|366988|3308|29581|13515|366988|3308|29581|1|91|9|5|16766|236|202|99|49.65|95.32|36.22|5850.90|3585.78|4915.35|9436.68|71.71|0.00|3019.50|3585.78|3657.49|6605.28|6676.99|-1329.57| +2450818|52409|2450866|13515|366988|3308|29581|13515|366988|3308|29581|1|14|10|2|16222|246|202|44|42.46|87.89|36.91|2243.12|1624.04|1868.24|3867.16|43.84|1136.82|618.64|487.22|531.06|1105.86|1149.70|-1381.02| +2450818|52409|2450844|13515|366988|3308|29581|13515|366988|3308|29581|1|88|12|4|499|77|202|25|5.82|16.52|7.10|235.50|177.50|145.50|413.00|3.05|24.85|152.75|152.65|155.70|305.40|308.45|7.15| +2450818|52409|2450868|13515|366988|3308|29581|13515|366988|3308|29581|1|57|7|1|9061|261|202|65|49.88|109.23|74.27|2272.40|4827.55|3242.20|7099.95|386.20|0.00|3478.80|4827.55|5213.75|8306.35|8692.55|1585.35| +2450818|52409|2450896|13515|366988|3308|29581|13515|366988|3308|29581|1|34|20|1|10195|213|202|62|34.94|71.62|37.24|2131.56|2308.88|2166.28|4440.44|161.62|0.00|843.20|2308.88|2470.50|3152.08|3313.70|142.60| +2450818|52409|2450824|13515|366988|3308|29581|13515|366988|3308|29581|1|47|2|4|481|183|202|21|4.68|5.14|2.00|65.94|42.00|98.28|107.94|1.68|0.00|2.10|42.00|43.68|44.10|45.78|-56.28| +2450818|52409|2450859|13515|366988|3308|29581|13515|366988|3308|29581|1|29|5|1|17720|196|202|60|30.33|55.20|14.90|2418.00|894.00|1819.80|3312.00|55.78|196.68|927.00|697.32|753.10|1624.32|1680.10|-1122.48| +2450818|52409|2450875|13515|366988|3308|29581|13515|366988|3308|29581|1|42|4|5|1934|75|202|66|26.97|69.85|62.86|461.34|4148.76|1780.02|4610.10|124.46|0.00|783.42|4148.76|4273.22|4932.18|5056.64|2368.74| +2450818|52409|2450902|13515|366988|3308|29581|13515|366988|3308|29581|1|47|11|2|8518|113|202|14|91.75|273.41|120.30|2143.54|1684.20|1284.50|3827.74|16.84|0.00|765.52|1684.20|1701.04|2449.72|2466.56|399.70| +2450818|52409|2450855|13515|366988|3308|29581|13515|366988|3308|29581|1|7|4|5|3739|109|202|84|60.08|88.91|72.01|1419.60|6048.84|5046.72|7468.44|120.97|0.00|597.24|6048.84|6169.81|6646.08|6767.05|1002.12| +2450818|52409|2450862|13515|366988|3308|29581|13515|366988|3308|29581|1|10|6|4|13909|178|202|71|75.43|162.17|69.73|6563.24|4950.83|5355.53|11514.07|99.01|0.00|3108.38|4950.83|5049.84|8059.21|8158.22|-404.70| +2450818|52409|2450876|13515|366988|3308|29581|13515|366988|3308|29581|1|6|7|3|3178|175|202|30|38.85|71.87|44.55|819.60|1336.50|1165.50|2156.10|7.48|1242.94|86.10|93.56|101.04|179.66|187.14|-1071.94| +2450818|66251|2450907|48049|4546|2290|29587|48049|4546|2290|29587|2|97|14|4|5870|206|203|4|58.05|81.27|17.06|256.84|68.24|232.20|325.08|6.14|0.00|126.76|68.24|74.38|195.00|201.14|-163.96| +2450818|66251|2450880|48049|4546|2290|29587|48049|4546|2290|29587|2|14|1|5|9514|234|203|33|30.14|75.95|25.06|1679.37|826.98|994.62|2506.35|66.15|0.00|400.95|826.98|893.13|1227.93|1294.08|-167.64| +2450818|66251|2450858|48049|4546|2290|29587|48049|4546|2290|29587|2|72|14|3|4963|56|203|93|4.46|11.46|1.26|948.60|117.18|414.78|1065.78|4.68|0.00|84.63|117.18|121.86|201.81|206.49|-297.60| +||2450876||4546|2290|29587|48049||2290|29587||12||4|15805|174|203||66.27||47.95||1054.90|||73.84|||1054.90|1128.74|1511.40|1585.24|-403.04| +2450818|66251|2450905|48049|4546|2290|29587|48049|4546|2290|29587|2|43|14|1|14128|253|203|66|62.18|86.43|50.12|2396.46|3307.92|4103.88|5704.38|99.23|0.00|227.70|3307.92|3407.15|3535.62|3634.85|-795.96| +2450818|66251|2450884|48049|4546|2290|29587|48049|4546|2290|29587|2|10|11|4|9925|262|203|44|95.77|203.03|174.60|1250.92|7682.40|4213.88|8933.32|76.82|0.00|3841.20|7682.40|7759.22|11523.60|11600.42|3468.52| +2450818|66251|2450892|48049|4546|2290|29587|48049|4546|2290|29587|2|32|2|5|691|69|203|80|28.90|52.02|10.40|3329.60|832.00|2312.00|4161.60|16.97|549.12|332.80|282.88|299.85|615.68|632.65|-2029.12| +2450818|66251|2450822|48049|4546|2290|29587|48049|4546|2290|29587|2|40|8|3|17786|300|203|26|72.73|113.45|6.80|2772.90|176.80|1890.98|2949.70|0.54|121.99|353.86|54.81|55.35|408.67|409.21|-1836.17| +2450818|66251|2450857|48049|4546|2290|29587|48049|4546|2290|29587|2|61|1|4|16094|88|203|68|62.70|164.27|116.63|3239.52|7930.84|4263.60|11170.36|317.23|0.00|2569.04|7930.84|8248.07|10499.88|10817.11|3667.24| +2450818|66251|2450821|48049|4546|2290|29587|48049|4546|2290|29587|2|12|10|1|3706|231|203|15|91.59|239.96|28.79|3167.55|431.85|1373.85|3599.40|38.86|0.00|251.85|431.85|470.71|683.70|722.56|-942.00| +2450818|66251|2450890|48049|4546|2290|29587|48049|4546|2290|29587|2|84|7|2|17149|17|203|89|18.29|31.27|30.01|112.14|2670.89|1627.81|2783.03|70.51|1495.69|1307.41|1175.20|1245.71|2482.61|2553.12|-452.61| +2450818|66251|2450831|48049|4546|2290|29587|48049|4546|2290|29587|2|43|9|4|14518|269|203|11|93.33|220.25|15.41|2253.24|169.51|1026.63|2422.75|11.86|0.00|1017.50|169.51|181.37|1187.01|1198.87|-857.12| +2450818|66251|2450903|48049|4546|2290|29587|48049|4546|2290|29587|2|84|5|5|12802|49|203|41|22.43|55.17|45.79|384.58|1877.39|919.63|2261.97|0.00|0.00|225.91|1877.39|1877.39|2103.30|2103.30|957.76| +2450818|66251|2450835|48049|4546|2290|29587|48049|4546|2290|29587|2|104|3|2|11089|292|203|30|35.50|71.00|66.74|127.80|2002.20|1065.00|2130.00|160.17|0.00|873.30|2002.20|2162.37|2875.50|3035.67|937.20| +2450818|71701|2450908|19741|1803520|3746|22772|19741|1803520|3746|22772|4|63|4|1|10106|257|204|95|83.54|174.59|90.78|7961.95|8624.10|7936.30|16586.05|603.68|0.00|5141.40|8624.10|9227.78|13765.50|14369.18|687.80| +2450818|71701|2450884|19741|1803520|3746|22772|19741|1803520|3746|22772|4|28|3|1|13756|172|204|97|91.63|167.68|142.52|2440.52|13824.44|8888.11|16264.96|0.00|0.00|7318.65|13824.44|13824.44|21143.09|21143.09|4936.33| +2450818|71701|2450825|19741|1803520|3746|22772|19741|1803520|3746|22772|4|36|1|3|13405|187|204|13|39.03|74.54|13.41|794.69|174.33|507.39|969.02|8.71|0.00|232.44|174.33|183.04|406.77|415.48|-333.06| +2450818|71701|2450895|19741|1803520|3746|22772|19741|1803520|3746|22772|4|83|7|5|15991|41|204|27|15.69|30.90|15.14|425.52|408.78|423.63|834.30|20.43|0.00|200.07|408.78|429.21|608.85|629.28|-14.85| +2450818|71701|2450846|19741|1803520|3746|22772|19741|1803520|3746|22772|4|45|16|2|10345|159|204|25|45.34|48.06|2.40|1141.50|60.00|1133.50|1201.50|4.20|0.00|264.25|60.00|64.20|324.25|328.45|-1073.50| +2450818|71701|2450880|19741|1803520|3746|22772|19741|1803520|3746|22772|4|36|17|5|6794|114|204|96|62.15|164.07|139.45|2363.52|13387.20|5966.40|15750.72|669.36|0.00|2677.44|13387.20|14056.56|16064.64|16734.00|7420.80| +2450818|71701|2450884|19741|1803520|3746|22772|19741|1803520|3746|22772|4|48|7|5|11152|168|204|4|42.12|94.77|5.68|356.36|22.72|168.48|379.08|0.68|0.00|178.16|22.72|23.40|200.88|201.56|-145.76| +2450818|71701|2450848|19741|1803520|3746|22772|19741|1803520|3746|22772|4|72|15|3|15734|287|204|20|69.96|168.60|64.06|2090.80|1281.20|1399.20|3372.00|89.68|0.00|472.00|1281.20|1370.88|1753.20|1842.88|-118.00| +2450818|71701|2450888|19741|1803520|3746|22772|19741|1803520|3746|22772|4|100|4|4|5425|290|204|70|16.93|32.33|8.08|1697.50|565.60|1185.10|2263.10|16.96|0.00|746.20|565.60|582.56|1311.80|1328.76|-619.50| +2450818|71701|2450828|19741|1803520|3746|22772|19741|1803520|3746|22772|4|77|11|2|2596|7|204|51|3.34|8.48|1.18|372.30|60.18|170.34|432.48|2.40|0.00|159.63|60.18|62.58|219.81|222.21|-110.16| +2450818|71701|2450862|19741|1803520|3746|22772|19741|1803520|3746|22772|4|45|15|4|10426|203|204|100|72.34|113.57|64.73|4884.00|6473.00|7234.00|11357.00|388.38|0.00|5337.00|6473.00|6861.38|11810.00|12198.38|-761.00| +2450818|52597|2450874|57376|926874|1619|43164|57376|926874|1619|43164|4|87|6|5|50|166|205|78|95.53|257.93|110.90|11468.34|8650.20|7451.34|20118.54|0.00|0.00|1005.42|8650.20|8650.20|9655.62|9655.62|1198.86| +2450818|52597|2450873|57376|926874|1619|43164|57376|926874|1619|43164|4|67|17|2|11210|177|205|52|47.33|58.68|36.96|1129.44|1921.92|2461.16|3051.36|0.00|0.00|1403.48|1921.92|1921.92|3325.40|3325.40|-539.24| +2450818|52597|2450902|57376|926874|1619|43164|57376|926874|1619|43164|4|83|19|3|15274|251|205|3|49.60|122.51|79.63|128.64|238.89|148.80|367.53|2.91|93.16|66.15|145.73|148.64|211.88|214.79|-3.07| +2450818|52597|2450895|57376|926874|1619|43164|57376|926874|1619|43164|4|21|1|4|13399|193|205|13|95.00|174.80|1.74|2249.78|22.62|1235.00|2272.40|0.67|0.00|68.12|22.62|23.29|90.74|91.41|-1212.38| +2450818|52597|2450828|57376|926874|1619|43164|57376|926874|1619|43164|4|108|2|5|7010|261|205|81|73.76|76.71|13.04|5157.27|1056.24|5974.56|6213.51|4.85|813.30|621.27|242.94|247.79|864.21|869.06|-5731.62| +2450818|52597|2450907|57376|926874|1619|43164|57376|926874|1619|43164|4|88|11|5|3310|295|205|51|77.14|143.48|58.82|4317.66|2999.82|3934.14|7317.48|269.98|0.00|658.41|2999.82|3269.80|3658.23|3928.21|-934.32| +2450819|63502|2450821|91288|1337798|1555|39712|91288|1337798|1555|39712|4|101|10|1|11395|240|206|18|12.52|28.67|26.66|36.18|479.88|225.36|516.06|38.39|0.00|5.04|479.88|518.27|484.92|523.31|254.52| +2450819|63502|2450860|91288|1337798|1555|39712|91288|1337798|1555|39712|4|36|19|5|4294|233|206|24|58.99|130.95|120.47|251.52|2891.28|1415.76|3142.80|202.38|0.00|597.12|2891.28|3093.66|3488.40|3690.78|1475.52| +2450819|63502|2450899|91288|1337798|1555|39712|91288|1337798|1555|39712|4|74|7|5|11767|67|206|23|99.09|233.85|137.97|2205.24|3173.31|2279.07|5378.55|0.00|793.32|1721.09|2379.99|2379.99|4101.08|4101.08|100.92| +2450819|63502|2450905|91288|1337798|1555|39712|91288|1337798|1555|39712|4|44|16|1|8569|26|206|48|48.54|112.61|41.66|3405.60|1999.68|2329.92|5405.28|35.99|1599.74|1837.44|399.94|435.93|2237.38|2273.37|-1929.98| +2450819|63502|2450902|91288|1337798|1555|39712|91288|1337798|1555|39712|4|9|3|3|8764|61|206|66|80.05|148.09|139.20|586.74|9187.20|5283.30|9773.94|826.84|0.00|2149.62|9187.20|10014.04|11336.82|12163.66|3903.90| +2450819|63502|2450838|91288|1337798|1555|39712|91288|1337798|1555|39712|4|49|9|3|15374|106|206|74|20.38|37.09|27.07|741.48|2003.18|1508.12|2744.66|20.03|0.00|273.80|2003.18|2023.21|2276.98|2297.01|495.06| +2450819|63502|2450867|91288|1337798|1555|39712|91288|1337798|1555|39712|4|19|4|2|6943|18|206|26|81.26|111.32|2.22|2836.60|57.72|2112.76|2894.32|1.15|0.00|839.28|57.72|58.87|897.00|898.15|-2055.04| +2450819|63502|2450902|91288|1337798|1555|39712|91288|1337798|1555|39712|4|37|13|2|11350|150|206|91|44.08|96.53|57.91|3514.42|5269.81|4011.28|8784.23|316.18|0.00|789.88|5269.81|5585.99|6059.69|6375.87|1258.53| +2450819|63502|2450903|91288|1337798|1555|39712|91288|1337798|1555|39712|4|71|10|3|13885|51|206|26|5.39|10.83|0.21|276.12|5.46|140.14|281.58|0.21|0.00|73.06|5.46|5.67|78.52|78.73|-134.68| +2450819|63502|2450906|91288|1337798|1555|39712|91288|1337798|1555|39712|4|83|12|4|8893|188|206|80|89.00|131.72|39.51|7376.80|3160.80|7120.00|10537.60|94.82|0.00|2423.20|3160.80|3255.62|5584.00|5678.82|-3959.20| +2450819|63502|2450861|91288|1337798|1555|39712|91288|1337798|1555|39712|4|60|8|5|3676|112|206|32|54.35|135.87|59.78|2434.88|1912.96|1739.20|4347.84|153.03|0.00|1391.04|1912.96|2065.99|3304.00|3457.03|173.76| +2450819|60341|2450907|54871|1402036|2801|41613|54871|1402036|2801|41613|1|81|7|1|8947|99|207|12|19.04|24.18|19.82|52.32|237.84|228.48|290.16|9.51|0.00|78.24|237.84|247.35|316.08|325.59|9.36| +2450819|60341|2450893|54871|1402036|2801|41613|54871|1402036|2801|41613|1|16|19|5|9553|172|207|57|19.45|55.82|3.34|2991.36|190.38|1108.65|3181.74|0.00|0.00|1272.24|190.38|190.38|1462.62|1462.62|-918.27| +2450819|60341|2450826|54871|1402036|2801|41613|54871|1402036|2801|41613|1|78|3|5|16036|193|207|64|41.33|69.02|41.41|1767.04|2650.24|2645.12|4417.28|37.10|1722.65|2208.64|927.59|964.69|3136.23|3173.33|-1717.53| +2450819|60341|2450849|54871|1402036|2801|41613|54871|1402036|2801|41613|1|92|12|4|14539|48|207|67|71.25|183.82|33.08|10099.58|2216.36|4773.75|12315.94|155.14|0.00|369.17|2216.36|2371.50|2585.53|2740.67|-2557.39| +2450819|60341|2450832|54871|1402036|2801|41613|54871|1402036|2801|41613|1|95|11|4|5575|273|207|11|35.60|53.04|38.71|157.63|425.81|391.60|583.44|0.00|0.00|40.81|425.81|425.81|466.62|466.62|34.21| +2450819|60341|2450894|54871|1402036|2801|41613|54871|1402036|2801|41613|1|95|13|3|7078|33|207|99|45.44|70.88|35.44|3508.56|3508.56|4498.56|7017.12|140.34|0.00|1543.41|3508.56|3648.90|5051.97|5192.31|-990.00| +2450819|60341|2450839|54871|1402036|2801|41613|54871|1402036|2801|41613|1|15|8|4|5137|238|207|25|98.61|135.09|25.66|2735.75|641.50|2465.25|3377.25|32.07|0.00|67.50|641.50|673.57|709.00|741.07|-1823.75| +2450819|60341|2450856|54871|1402036|2801|41613|54871|1402036|2801|41613|1|26|3|4|3349|221|207|14|30.00|68.70|52.21|230.86|730.94|420.00|961.80|10.52|467.80|0.00|263.14|273.66|263.14|273.66|-156.86| +2450819|60341|2450905|54871|1402036|2801|41613|54871|1402036|2801|41613|1|38|14|4|10930|203|207|71|36.88|97.36|31.15|4700.91|2211.65|2618.48|6912.56|164.54|154.81|2211.65|2056.84|2221.38|4268.49|4433.03|-561.64| +2450819|60341|2450906|54871|1402036|2801|41613|54871|1402036|2801|41613|1|32|14|4|15862|102|207|30|81.56|125.60|116.80|264.00|3504.00|2446.80|3768.00|33.28|175.20|1205.70|3328.80|3362.08|4534.50|4567.78|882.00| +2450819|60341|2450822|54871|1402036|2801|41613|54871|1402036|2801|41613|1|52|20|5|12766|278|207|30|89.17|264.83|259.53|159.00|7785.90|2675.10|7944.90|155.71|0.00|635.40|7785.90|7941.61|8421.30|8577.01|5110.80| +2450819|23771|2450859|62974|1443645|1916|16540|62974|1443645|1916|16540|4|55|5|5|11042|154|208|89|86.77|100.65|20.13|7166.28|1791.57|7722.53|8957.85|125.40|0.00|895.34|1791.57|1916.97|2686.91|2812.31|-5930.96| +2450819|23771|2450832|62974|1443645|1916|16540|62974|1443645|1916|16540|4|87|5|2|1597|293|208|31|19.84|38.88|13.99|771.59|433.69|615.04|1205.28|11.27|208.17|325.19|225.52|236.79|550.71|561.98|-389.52| +2450819|23771|2450826|62974|1443645|1916|16540|62974|1443645|1916|16540|4|74|5|5|1585|163|208|41|68.79|135.51|43.36|3778.15|1777.76|2820.39|5555.91|0.00|0.00|2555.53|1777.76|1777.76|4333.29|4333.29|-1042.63| +2450819|23771|2450879|62974|1443645|1916|16540|62974|1443645|1916|16540|4|28|1|3|12700|109|208|57|16.52|21.80|7.63|807.69|434.91|941.64|1242.60|39.14|0.00|86.64|434.91|474.05|521.55|560.69|-506.73| +2450819|23771|2450825|62974|1443645|1916|16540|62974|1443645|1916|16540|4|21|16|3|5833|255|208|2|51.31|75.42|47.51|55.82|95.02|102.62|150.84|0.00|0.00|72.40|95.02|95.02|167.42|167.42|-7.60| +2450819|23771|2450898|62974|1443645|1916|16540|62974|1443645|1916|16540|4|28|7|3|2875|35|208|3|70.40|209.79|56.64|459.45|169.92|211.20|629.37|6.79|0.00|251.73|169.92|176.71|421.65|428.44|-41.28| +2450819|23771|2450863|62974|1443645|1916|16540|62974|1443645|1916|16540|4|88|7|1|16543|149|208|12|74.32|170.93|54.69|1394.88|656.28|891.84|2051.16|59.06|0.00|779.40|656.28|715.34|1435.68|1494.74|-235.56| +2450819|46154|2450865|17674|1289493|6912|34731|17674|1289493|6912|34731|1|36|14|4|16784|222|209|66|26.17|48.67|9.73|2570.04|642.18|1727.22|3212.22|44.95|0.00|224.40|642.18|687.13|866.58|911.53|-1085.04| +2450819|46154|2450841|17674|1289493|6912|34731|17674|1289493|6912|34731|1|31|7|1|14383|224|209|100|89.17|254.13|88.94|16519.00|8894.00|8917.00|25413.00|0.00|0.00|12452.00|8894.00|8894.00|21346.00|21346.00|-23.00| +2450819|46154|2450858|17674|1289493|6912|34731|17674|1289493|6912|34731|1|79|3|2|4796|233|209|27|58.77|165.14|143.67|579.69|3879.09|1586.79|4458.78|193.95|0.00|802.44|3879.09|4073.04|4681.53|4875.48|2292.30| +2450819|46154|2450841|17674|1289493|6912|34731|17674|1289493|6912|34731|1|49|2|2|13051|244|209|95|86.47|102.03|23.46|7464.15|2228.70|8214.65|9692.85|133.72|0.00|3295.55|2228.70|2362.42|5524.25|5657.97|-5985.95| +2450819|46154|2450850|17674|1289493|6912|34731|17674|1289493|6912|34731|1|24|1|2|3568|92|209|29|24.76|62.39|54.27|235.48|1573.83|718.04|1809.31|62.95|0.00|578.84|1573.83|1636.78|2152.67|2215.62|855.79| +2450819|46154|2450907|17674|1289493|6912|34731|17674|1289493|6912|34731|1|76|15|1|10742|188|209|7|20.35|30.11|18.96|78.05|132.72|142.45|210.77|3.98|0.00|56.84|132.72|136.70|189.56|193.54|-9.73| +2450819|46154|2450867|17674|1289493|6912|34731|17674|1289493|6912|34731|1|72|5|3|3787|6|209|38|50.15|87.76|17.55|2667.98|666.90|1905.70|3334.88|6.66|0.00|1467.18|666.90|673.56|2134.08|2140.74|-1238.80| +2450819|46154|2450880|17674|1289493|6912|34731|17674|1289493|6912|34731|1|76|12|5|12391|159|209|39|65.58|112.79|59.77|2067.78|2331.03|2557.62|4398.81|2.33|2214.47|175.89|116.56|118.89|292.45|294.78|-2441.06| +2450819|46154|2450884|17674|1289493|6912|34731|17674|1289493|6912|34731|1|53|20|5|14470|76|209|25|4.63|8.14|2.36|144.50|59.00|115.75|203.50|0.68|41.89|93.50|17.11|17.79|110.61|111.29|-98.64| +2450819|46154|2450852|17674|1289493|6912|34731|17674|1289493|6912|34731|1|9|4|5|1204|58|209|54|98.93|205.77|189.30|889.38|10222.20|5342.22|11111.58|204.44|0.00|3999.78|10222.20|10426.64|14221.98|14426.42|4879.98| +2450819|46154|2450861|17674|1289493|6912|34731|17674|1289493|6912|34731|1|69|5|1|6598|23|209|47|42.31|112.54|25.88|4073.02|1216.36|1988.57|5289.38|72.98|0.00|2485.83|1216.36|1289.34|3702.19|3775.17|-772.21| +2450819|46154|2450875|17674|1289493|6912|34731|17674|1289493|6912|34731|1|71|6|4|17618|165|209|92|6.15|16.54|13.56|274.16|1247.52|565.80|1521.68|37.42|0.00|151.80|1247.52|1284.94|1399.32|1436.74|681.72| +2450819|46154|2450878|17674|1289493|6912|34731|17674|1289493|6912|34731|1|42|19|3|14020|175|209|76|18.21|41.15|29.62|876.28|2251.12|1383.96|3127.40|0.00|0.00|1219.04|2251.12|2251.12|3470.16|3470.16|867.16| +2450819|22767|2450907|10722|220803|4931|7059|84823|1295715|3043|30371|1|76|6|5|1466|138|210|5|90.40|106.67|40.53|330.70|202.65|452.00|533.35|18.23|0.00|16.00|202.65|220.88|218.65|236.88|-249.35| +2450819|22767|2450830|10722|220803|4931|7059|84823|1295715|3043|30371|1|102|14|3|6082|199|210|58|47.38|59.69|34.62|1454.06|2007.96|2748.04|3462.02|40.15|0.00|311.46|2007.96|2048.11|2319.42|2359.57|-740.08| +2450819|22767|2450890|10722|220803|4931|7059|84823|1295715|3043|30371|1|97|2|5|15829|276|210|45|96.61|278.23|116.85|7262.10|5258.25|4347.45|12520.35|157.74|0.00|5258.25|5258.25|5415.99|10516.50|10674.24|910.80| +2450819|22767|2450854|10722|220803|4931|7059|84823|1295715|3043|30371|1|92|1|5|3440|61|210|38|58.49|143.30|127.53|599.26|4846.14|2222.62|5445.40|339.22|0.00|2395.90|4846.14|5185.36|7242.04|7581.26|2623.52| +2450819|22767|2450866|10722|220803|4931|7059|84823|1295715|3043|30371|1|74|17|3|5527|195|210|43|86.58|129.87|76.62|2289.75|3294.66|3722.94|5584.41|32.94|0.00|2010.25|3294.66|3327.60|5304.91|5337.85|-428.28| +2450819|22767|2450843|10722|220803|4931|7059|84823|1295715|3043|30371|1|53|17|1|17443|241|210|78|31.25|90.62|43.49|3676.14|3392.22|2437.50|7068.36|12.21|2171.02|2191.02|1221.20|1233.41|3412.22|3424.43|-1216.30| +2450819|22767|2450860|10722|220803|4931|7059|84823|1295715|3043|30371|1|29|12|1|14566|282|210|28|70.79|163.52|81.76|2289.28|2289.28|1982.12|4578.56|160.24|0.00|2151.80|2289.28|2449.52|4441.08|4601.32|307.16| +2450819|22767|2450822|10722|220803|4931|7059|84823|1295715|3043|30371|1|33|12|2|9259|97|210|7|2.76|2.87|0.31|17.92|2.17|19.32|20.09|0.10|0.00|5.18|2.17|2.27|7.35|7.45|-17.15| +2450819|22767|2450861|10722|220803|4931|7059|84823|1295715|3043|30371|1|107|7|1|7478|212|210|7|58.60|93.76|25.31|479.15|177.17|410.20|656.32|10.63|0.00|216.58|177.17|187.80|393.75|404.38|-233.03| +2450819|22767|2450859|10722|220803|4931|7059|84823|1295715|3043|30371|1|28|20|2|5965|289|210|6|2.77|7.03|3.58|20.70|21.48|16.62|42.18|1.93|0.00|4.20|21.48|23.41|25.68|27.61|4.86| +2450819|22767|2450902|10722|220803|4931|7059|84823|1295715|3043|30371|1|69|2|1|8098|20|210|53|44.89|70.02|6.30|3377.16|333.90|2379.17|3711.06|3.33|0.00|1372.70|333.90|337.23|1706.60|1709.93|-2045.27| +2450819|22767|2450849|10722|220803|4931|7059|84823|1295715|3043|30371|1|41|4|5|11665|31|210|96|84.72|139.78|44.72|9125.76|4293.12|8133.12|13418.88|105.61|772.76|804.48|3520.36|3625.97|4324.84|4430.45|-4612.76| +2450819|22767|2450882|10722|220803|4931|7059|84823|1295715|3043|30371|1|72|15|3|3907|31|210|16|30.78|46.17|11.08|561.44|177.28|492.48|738.72|4.52|26.59|295.36|150.69|155.21|446.05|450.57|-341.79| +2450819|78554|2450897|3169|473061|3400|28939|3169|473061|3400|28939|2|38|12|1|5122|156|211|12|24.85|30.81|26.80|48.12|321.60|298.20|369.72|3.21|0.00|88.68|321.60|324.81|410.28|413.49|23.40| +2450819|78554|2450843|3169|473061|3400|28939|3169|473061|3400|28939|2|81|6|3|16729|259|211|41|40.21|72.78|53.12|806.06|2177.92|1648.61|2983.98|152.45|0.00|835.17|2177.92|2330.37|3013.09|3165.54|529.31| +2450819|78554|2450909|3169|473061|3400|28939|3169|473061|3400|28939|2|38|9|5|10220|90|211|89|61.18|95.44|32.44|5607.00|2887.16|5445.02|8494.16|259.84|0.00|1613.57|2887.16|3147.00|4500.73|4760.57|-2557.86| +2450819|78554|2450881|3169|473061|3400|28939|3169|473061|3400|28939|2|74|14|1|11833|133|211|25|50.62|87.57|36.77|1270.00|919.25|1265.50|2189.25|36.77|0.00|1050.75|919.25|956.02|1970.00|2006.77|-346.25| +2450819|78554|2450860|3169|473061|3400|28939|3169|473061|3400|28939|2|35|3|2|16102|52|211|54|68.19|182.74|131.57|2763.18|7104.78|3682.26|9867.96|0.00|0.00|2960.28|7104.78|7104.78|10065.06|10065.06|3422.52| +2450819|78554|2450870|3169|473061|3400|28939|3169|473061|3400|28939|2|38|4|5|8432|237|211|75|70.53|135.41|111.03|1828.50|8327.25|5289.75|10155.75|0.00|0.00|4468.50|8327.25|8327.25|12795.75|12795.75|3037.50| +2450819|78554|2450889|3169|473061|3400|28939|3169|473061|3400|28939|2|72|10|5|9229|282|211|81|38.43|39.19|19.20|1619.19|1555.20|3112.83|3174.39|30.48|31.10|1142.10|1524.10|1554.58|2666.20|2696.68|-1588.73| +2450819|78554|2450852|3169|473061|3400|28939|3169|473061|3400|28939|2|16|4|3|12502|76|211|19|59.91|111.43|109.20|42.37|2074.80|1138.29|2117.17|186.73|0.00|995.03|2074.80|2261.53|3069.83|3256.56|936.51| +2450819|78554|2450890|3169|473061|3400|28939|3169|473061|3400|28939|2|59|4|4|3866|287|211|11|27.40|32.60|6.84|283.36|75.24|301.40|358.60|6.77|0.00|157.74|75.24|82.01|232.98|239.75|-226.16| +2450819|78554|2450873|3169|473061|3400|28939|3169|473061|3400|28939|2|4|9|1|11260|272|211|47|59.49|142.77|45.68|4563.23|2146.96|2796.03|6710.19|0.00|1846.38|1341.85|300.58|300.58|1642.43|1642.43|-2495.45| +2450819|78554|2450885|3169|473061|3400|28939|3169|473061|3400|28939|2|108|3|2|5812|244|211|41|37.25|70.40|22.52|1963.08|923.32|1527.25|2886.40|20.22|249.29|461.66|674.03|694.25|1135.69|1155.91|-853.22| +2450819|78554|2450874|3169|473061|3400|28939|3169|473061|3400|28939|2|54|10|2|16933|189|211|68|82.65|149.59|73.29|5188.40|4983.72|5620.20|10172.12|348.86|0.00|4882.40|4983.72|5332.58|9866.12|10214.98|-636.48| +2450819|78554|2450902|3169|473061|3400|28939|3169|473061|3400|28939|2|66|3|2|9596|270|211|30|57.19|152.69|77.87|2244.60|2336.10|1715.70|4580.70|163.52|0.00|366.30|2336.10|2499.62|2702.40|2865.92|620.40| +2450819|78554|2450898|3169|473061|3400|28939|3169|473061|3400|28939|2|96|16|3|14191|182|211|54|47.23|117.13|91.36|1391.58|4933.44|2550.42|6325.02|2.96|4884.10|1264.68|49.34|52.30|1314.02|1316.98|-2501.08| +2450819|24878|2450852|82128|1657260|4392|34324|82128|1657260|4392|34324|1|18|16|2|6529|269|212|62|4.31|11.37|1.70|599.54|105.40|267.22|704.94|1.11|49.53|6.82|55.87|56.98|62.69|63.80|-211.35| +2450819|24878|2450840|82128|1657260|4392|34324|82128|1657260|4392|34324|1|1|19|4|10292|239|212|47|80.73|134.01|62.98|3338.41|2960.06|3794.31|6298.47|148.00|0.00|566.82|2960.06|3108.06|3526.88|3674.88|-834.25| +||2450896|82128|1657260|4392|34324||1657260|4392||1||||6211|146|212|||||851.88|7663.20||8515.08|229.89|||||9451.28|9681.17|4824.84| +2450819|24878|2450823|82128|1657260|4392|34324|82128|1657260|4392|34324|1|82|1|1|7093|159|212|98|8.79|25.40|2.54|2240.28|248.92|861.42|2489.20|9.95|0.00|248.92|248.92|258.87|497.84|507.79|-612.50| +2450819|24878|2450908|82128|1657260|4392|34324|82128|1657260|4392|34324|1|52|2|3|10870|169|212|53|76.57|181.47|101.62|4232.05|5385.86|4058.21|9617.91|274.67|807.87|4231.52|4577.99|4852.66|8809.51|9084.18|519.78| +2450819|24878|2450890|82128|1657260|4392|34324|82128|1657260|4392|34324|1|107|4|3|7576|3|212|82|50.24|141.67|29.75|9177.44|2439.50|4119.68|11616.94|73.18|0.00|2439.50|2439.50|2512.68|4879.00|4952.18|-1680.18| +2450819|24878|2450882|82128|1657260|4392|34324|82128|1657260|4392|34324|1|102|19|4|15566|234|212|50|92.44|183.95|167.39|828.00|8369.50|4622.00|9197.50|0.00|3264.10|183.50|5105.40|5105.40|5288.90|5288.90|483.40| +2450819|24878|2450886|82128|1657260|4392|34324|82128|1657260|4392|34324|1|6|9|2|1876|212|212|34|9.39|14.64|0.73|472.94|24.82|319.26|497.76|1.22|11.16|94.52|13.66|14.88|108.18|109.40|-305.60| +2450819|24878|2450863|82128|1657260|4392|34324|82128|1657260|4392|34324|1|85|16|4|7081|74|212|81|9.62|26.35|25.29|85.86|2048.49|779.22|2134.35|40.96|0.00|938.79|2048.49|2089.45|2987.28|3028.24|1269.27| +2450819|24878|2450903|82128|1657260|4392|34324|82128|1657260|4392|34324|1|5|1|4|280|235|212|70|3.26|9.78|6.74|212.80|471.80|228.20|684.60|28.30|0.00|156.80|471.80|500.10|628.60|656.90|243.60| +2450819|24878|2450894|82128|1657260|4392|34324|82128|1657260|4392|34324|1|77|15|1|10130|276|212|85|85.98|171.10|27.37|12217.05|2326.45|7308.30|14543.50|46.52|0.00|5671.20|2326.45|2372.97|7997.65|8044.17|-4981.85| +2450819||2450845|||4392||||4392||||12||566|87|212|65|12.45|||46.15||809.25||37.32|254.09|45.50|||667.61|704.93|-187.14| +2450819|24878|2450881|82128|1657260|4392|34324|82128|1657260|4392|34324|1|72|8|4|14450|268|212|71|26.38|29.80|27.41|169.69|1946.11|1872.98|2115.80|136.22|0.00|465.05|1946.11|2082.33|2411.16|2547.38|73.13| +2450819|14555|2450903|72270|1389914|4358|45602|72270|1389914|4358|45602|1|23|8|2|11509|250|213|26|48.20|129.17|100.75|738.92|2619.50|1253.20|3358.42|130.97|0.00|1578.20|2619.50|2750.47|4197.70|4328.67|1366.30| +2450819|14555|2450884|72270|1389914|4358|45602|72270|1389914|4358|45602|1|2|3|3|17599|223|213|17|19.91|43.20|31.53|198.39|536.01|338.47|734.40|3.85|407.36|359.72|128.65|132.50|488.37|492.22|-209.82| +2450819|14555|2450884|72270|1389914|4358|45602|72270|1389914|4358|45602|1|72|4|3|3112|100|213|3|83.33|129.16|5.16|372.00|15.48|249.99|387.48|1.23|0.00|185.97|15.48|16.71|201.45|202.68|-234.51| +2450819|14555|2450898|72270|1389914|4358|45602|72270|1389914|4358|45602|1|82|3|2|757|204|213|70|36.18|58.61|3.51|3857.00|245.70|2532.60|4102.70|17.19|0.00|1353.80|245.70|262.89|1599.50|1616.69|-2286.90| +2450819|14555|2450902|72270|1389914|4358|45602|72270|1389914|4358|45602|1|54|9|1|11618|229|213|78|60.44|96.09|18.25|6071.52|1423.50|4714.32|7495.02|0.00|0.00|0.00|1423.50|1423.50|1423.50|1423.50|-3290.82| +2450819|14555|2450884|72270|1389914|4358|45602|72270|1389914|4358|45602|1|73|17|1|5998|36|213|11|18.87|30.38|24.60|63.58|270.60|207.57|334.18|13.53|0.00|136.95|270.60|284.13|407.55|421.08|63.03| +2450819|27963|2450826|2084|1915233|735|27084|2084|1915233|735|27084|4|21|11|2|5173|179|214|1|94.44|198.32|142.79|55.53|142.79|94.44|198.32|1.42|0.00|91.22|142.79|144.21|234.01|235.43|48.35| +2450819|27963|2450840|2084|1915233|735|27084|2084|1915233|735|27084|4|76|15|1|8698|135|214|43|45.04|86.47|72.63|595.12|3123.09|1936.72|3718.21|187.38|0.00|892.25|3123.09|3310.47|4015.34|4202.72|1186.37| +2450819|27963|2450887|2084|1915233|735|27084|2084|1915233|735|27084|4|101|9|3|15100|283|214|85|4.66|13.00|12.87|11.05|1093.95|396.10|1105.00|54.69|0.00|375.70|1093.95|1148.64|1469.65|1524.34|697.85| +2450819|27963|2450898|2084|1915233|735|27084|2084|1915233|735|27084|4|66|1|3|10477|245|214|91|94.91|265.74|63.77|18379.27|5803.07|8636.81|24182.34|33.07|2495.32|4835.74|3307.75|3340.82|8143.49|8176.56|-5329.06| +2450819|27963|2450891|2084|1915233|735|27084|2084|1915233|735|27084|4|53|19|3|7915|37|214|99|17.27|36.26|9.06|2692.80|896.94|1709.73|3589.74|80.72|0.00|35.64|896.94|977.66|932.58|1013.30|-812.79| +2450819|27963|2450845|2084|1915233|735|27084|2084|1915233|735|27084|4|69|11|5|2690|133|214|7|99.02|258.44|15.50|1700.58|108.50|693.14|1809.08|3.25|0.00|379.89|108.50|111.75|488.39|491.64|-584.64| +2450819|27963|2450907|2084|1915233|735|27084|2084|1915233|735|27084|4|69|15|5|595|15|214|7|19.86|30.78|8.31|157.29|58.17|139.02|215.46|4.65|0.00|45.22|58.17|62.82|103.39|108.04|-80.85| +2450819|4118|2450864|82568|1709232|1049|28513|82568|1709232|1049|28513|4|7|10|4|17042|135|215|28|55.54|71.64|45.13|742.28|1263.64|1555.12|2005.92|63.18|0.00|942.76|1263.64|1326.82|2206.40|2269.58|-291.48| +2450819|4118|2450851|82568|1709232|1049|28513|82568|1709232|1049|28513|4|42|5|3|10964|7|215|35|78.49|87.90|0.00|3076.50|0.00|2747.15|3076.50|0.00|0.00|399.70|0.00|0.00|399.70|399.70|-2747.15| +2450819|4118|2450861|82568|1709232|1049|28513|82568|1709232|1049|28513|4|44|19|2|9472|228|215|74|62.88|79.85|78.25|118.40|5790.50|4653.12|5908.90|0.00|0.00|2658.82|5790.50|5790.50|8449.32|8449.32|1137.38| +2450819|4118|2450833|82568|1709232|1049|28513|82568|1709232|1049|28513|4|105|9|5|5690|282|215|9|27.21|79.45|2.38|693.63|21.42|244.89|715.05|0.21|0.00|28.53|21.42|21.63|49.95|50.16|-223.47| +2450819|4118|2450881|82568|1709232|1049|28513|82568|1709232|1049|28513|4|71|5|5|11390|282|215|30|78.97|114.50|43.51|2129.70|1305.30|2369.10|3435.00|52.21|0.00|343.50|1305.30|1357.51|1648.80|1701.01|-1063.80| +2450819|49015|2450864|14269|712149|6029|38309|14269|712149|6029|38309|4|107|17|1|15724|105|216|88|41.63|69.93|51.04|1662.32|4491.52|3663.44|6153.84|4.04|4446.60|1783.76|44.92|48.96|1828.68|1832.72|-3618.52| +2450819|49015|2450894|14269|712149|6029|38309|14269|712149|6029|38309|4|95|19|3|16580|133|216|33|57.16|118.89|10.70|3570.27|353.10|1886.28|3923.37|2.33|275.41|627.66|77.69|80.02|705.35|707.68|-1808.59| +2450819|49015|2450846|14269|712149|6029|38309|14269|712149|6029|38309|4|8|7|3|3086|125|216|8|43.23|127.09|24.14|823.60|193.12|345.84|1016.72|0.00|0.00|182.96|193.12|193.12|376.08|376.08|-152.72| +2450819|49015|2450849|14269|712149|6029|38309|14269|712149|6029|38309|4|12|17|3|1228|197|216|75|66.29|169.03|43.94|9381.75|3295.50|4971.75|12677.25|296.59|0.00|2408.25|3295.50|3592.09|5703.75|6000.34|-1676.25| +2450819|49015|2450871|14269|712149|6029|38309|14269|712149|6029|38309|4|13|1|1|8095|67|216|67|23.05|43.56|14.37|1955.73|962.79|1544.35|2918.52|86.65|0.00|554.09|962.79|1049.44|1516.88|1603.53|-581.56| +2450819|49015|2450856|14269|712149|6029|38309|14269|712149|6029|38309|4|63|13|4|15922|81|216|3|62.02|104.81|83.84|62.91|251.52|186.06|314.43|0.00|0.00|81.75|251.52|251.52|333.27|333.27|65.46| +2450819|49015|2450821|14269|712149|6029|38309|14269|712149|6029|38309|4|3|8|3|7048|240|216|17|34.30|97.75|23.46|1262.93|398.82|583.10|1661.75|0.00|0.00|299.03|398.82|398.82|697.85|697.85|-184.28| +2450819|49015|2450845|14269|712149|6029|38309|14269|712149|6029|38309|4|75|6|5|7249|286|216|79|33.92|98.02|87.23|852.41|6891.17|2679.68|7743.58|463.08|1102.58|77.42|5788.59|6251.67|5866.01|6329.09|3108.91| +2450819|49015|2450899|14269|712149|6029|38309|14269|712149|6029|38309|4|58|13|5|8080|287|216|33|62.82|85.43|12.81|2396.46|422.73|2073.06|2819.19|0.00|0.00|1324.95|422.73|422.73|1747.68|1747.68|-1650.33| +2450819|49015|2450834|14269|712149|6029|38309|14269|712149|6029|38309|4|52|20|2|16063|238|216|35|39.69|115.49|115.49|0.00|4042.15|1389.15|4042.15|242.52|0.00|403.90|4042.15|4284.67|4446.05|4688.57|2653.00| +2450819|54269|2450837|93344|1413580|6263|31746|93344|1413580|6263|31746|1|1|9|4|8662|146|217|84|41.70|53.79|26.35|2304.96|2213.40|3502.80|4518.36|2.65|2124.86|948.36|88.54|91.19|1036.90|1039.55|-3414.26| +2450819|54269|2450826|93344|1413580|6263|31746|93344|1413580|6263|31746|1|61|2|5|4735|63|217|68|93.56|272.25|209.63|4258.16|14254.84|6362.08|18513.00|1140.38|0.00|4257.48|14254.84|15395.22|18512.32|19652.70|7892.76| +2450819|54269|2450894|93344|1413580|6263|31746|93344|1413580|6263|31746|1|60|18|3|13159|300|217|5|56.73|148.63|31.21|587.10|156.05|283.65|743.15|11.79|24.96|274.95|131.09|142.88|406.04|417.83|-152.56| +2450819|54269|2450904|93344|1413580|6263|31746|93344|1413580|6263|31746|1|48|9|5|5650|12|217|13|31.40|89.49|81.43|104.78|1058.59|408.20|1163.37|0.00|0.00|535.08|1058.59|1058.59|1593.67|1593.67|650.39| +2450819|54269|2450882|93344|1413580|6263|31746|93344|1413580|6263|31746|1|64|1|3|2452|22|217|89|40.09|100.62|0.00|8955.18|0.00|3568.01|8955.18|0.00|0.00|1343.01|0.00|0.00|1343.01|1343.01|-3568.01| +2450819|29037|2450865|13396|464619|3395|24218|13396|464619|3395|24218|1|50|6|5|10534|200|218|31|23.38|52.83|7.39|1408.64|229.09|724.78|1637.73|16.03|0.00|245.52|229.09|245.12|474.61|490.64|-495.69| +2450819|29037|2450836|13396|464619|3395|24218|13396|464619|3395|24218|1|85|13|5|15667|48|218|43|49.15|63.89|44.72|824.31|1922.96|2113.45|2747.27|2.69|1884.50|1043.61|38.46|41.15|1082.07|1084.76|-2074.99| +2450819|29037|2450835|13396|464619|3395|24218|13396|464619|3395|24218|1|26|2|2|3164|242|218|54|80.43|155.22|52.77|5532.30|2849.58|4343.22|8381.88|142.47|0.00|1257.12|2849.58|2992.05|4106.70|4249.17|-1493.64| +2450819|29037|2450861|13396|464619|3395|24218|13396|464619|3395|24218|1|59|1|4|5840|12|218|40|92.92|95.70|77.51|727.60|3100.40|3716.80|3828.00|0.00|0.00|688.80|3100.40|3100.40|3789.20|3789.20|-616.40| +2450819|29037|2450871|13396|464619|3395|24218|13396|464619|3395|24218|1|23|3|2|4586|275|218|56|15.19|36.60|20.13|922.32|1127.28|850.64|2049.60|16.90|845.46|389.20|281.82|298.72|671.02|687.92|-568.82| +2450819|29037|2450878|13396|464619|3395|24218|13396|464619|3395|24218|1|62|16|1|17630|39|218|17|41.49|67.62|54.09|230.01|919.53|705.33|1149.54|1.10|901.13|241.40|18.40|19.50|259.80|260.90|-686.93| +2450819|18643|2450906|91633|28147|3861|14375|91633|28147|3861|14375|2|82|13|1|3379|95|219|18|17.14|47.13|41.94|93.42|754.92|308.52|848.34|67.94|0.00|220.50|754.92|822.86|975.42|1043.36|446.40| +2450819|18643|2450862|91633|28147|3861|14375|91633|28147|3861|14375|2|43|3|1|3692|124|219|39|91.30|103.16|35.07|2655.51|1367.73|3560.70|4023.24|78.50|246.19|1850.55|1121.54|1200.04|2972.09|3050.59|-2439.16| +2450819|18643|2450825|91633|28147|3861|14375|91633|28147|3861|14375|2|59|5|1|13070|161|219|49|11.50|30.36|25.80|223.44|1264.20|563.50|1487.64|0.75|1238.91|520.38|25.29|26.04|545.67|546.42|-538.21| +2450819|18643|2450900|91633|28147|3861|14375|91633|28147|3861|14375|2|62|7|3|9368|230|219|8|25.91|34.46|19.29|121.36|154.32|207.28|275.68|9.25|0.00|107.44|154.32|163.57|261.76|271.01|-52.96| +2450819|18643|2450872|91633|28147|3861|14375|91633|28147|3861|14375|2|87|11|2|13240|62|219|76|41.82|50.60|5.06|3461.04|384.56|3178.32|3845.60|15.38|0.00|769.12|384.56|399.94|1153.68|1169.06|-2793.76| +2450819|18643|2450828|91633|28147|3861|14375|91633|28147|3861|14375|2|17|20|2|5089|23|219|95|32.99|50.14|39.10|1048.80|3714.50|3134.05|4763.30|260.01|0.00|2048.20|3714.50|3974.51|5762.70|6022.71|580.45| +2450819|18643|2450882|91633|28147|3861|14375|91633|28147|3861|14375|2|65|8|2|9194|293|219|82|83.00|180.11|156.69|1920.44|12848.58|6806.00|14769.02|642.42|0.00|4873.26|12848.58|13491.00|17721.84|18364.26|6042.58| +2450819|18643|2450837|91633|28147|3861|14375|91633|28147|3861|14375|2|93|19|4|349|111|219|15|44.87|65.95|52.76|197.85|791.40|673.05|989.25|63.31|0.00|336.30|791.40|854.71|1127.70|1191.01|118.35| +2450819|18643|2450828|91633|28147|3861|14375|91633|28147|3861|14375|2|97|2|1|7246|229|219|63|34.36|89.67|32.28|3615.57|2033.64|2164.68|5649.21|183.02|0.00|1581.30|2033.64|2216.66|3614.94|3797.96|-131.04| +2450819|1176|2450833|67261|833906|5351|43596|67261|833906|5351|43596|4|4|16|5|11191|75|220|21|99.77|141.67|22.66|2499.21|475.86|2095.17|2975.07|33.31|0.00|148.68|475.86|509.17|624.54|657.85|-1619.31| +2450819|1176|2450844|67261|833906|5351|43596|67261|833906|5351|43596|4|16|11|4|488|20|220|41|40.92|117.84|15.31|4203.73|627.71|1677.72|4831.44|50.21|0.00|2222.20|627.71|677.92|2849.91|2900.12|-1050.01| +2450819|1176|2450844|67261|833906|5351|43596|67261|833906|5351|43596|4|94|3|5|11275|102|220|89|36.31|98.03|76.46|1919.73|6804.94|3231.59|8724.67|340.24|0.00|4187.45|6804.94|7145.18|10992.39|11332.63|3573.35| +2450819|1176|2450843|67261|833906|5351|43596|67261|833906|5351|43596|4|56|11|3|11842|158|220|1|55.99|83.98|71.38|12.60|71.38|55.99|83.98|0.00|57.10|38.63|14.28|14.28|52.91|52.91|-41.71| +2450819|1176|2450870|67261|833906|5351|43596|67261|833906|5351|43596|4|53|19|5|6865|69|220|57|61.81|110.63|11.06|5675.49|630.42|3523.17|6305.91|56.73|0.00|377.91|630.42|687.15|1008.33|1065.06|-2892.75| +2450819|1176|2450847|67261|833906|5351|43596|67261|833906|5351|43596|4|98|15|1|13381|148|220|78|8.50|24.39|19.26|400.14|1502.28|663.00|1902.42|135.20|0.00|570.18|1502.28|1637.48|2072.46|2207.66|839.28| +2450819|1176|2450885|67261|833906|5351|43596|67261|833906|5351|43596|4|5|18|5|15880|127|220|37|4.94|7.85|1.80|223.85|66.60|182.78|290.45|5.99|0.00|116.18|66.60|72.59|182.78|188.77|-116.18| +2450819|1176|2450875|67261|833906|5351|43596|67261|833906|5351|43596|4|54|11|5|11890|13|220|62|45.58|98.45|35.44|3906.62|2197.28|2825.96|6103.90|87.89|0.00|121.52|2197.28|2285.17|2318.80|2406.69|-628.68| +2450819|1176|2450906|67261|833906|5351|43596|67261|833906|5351|43596|4|65|4|2|15382|206|220|79|61.71|155.50|49.76|8353.46|3931.04|4875.09|12284.50|314.48|0.00|613.83|3931.04|4245.52|4544.87|4859.35|-944.05| +2450819|1176|2450856|67261|833906|5351|43596|67261|833906|5351|43596|4|37|18|3|16880|177|220|64|10.07|19.23|2.88|1046.40|184.32|644.48|1230.72|5.52|0.00|467.20|184.32|189.84|651.52|657.04|-460.16| +|||67261|833906|5351|43596|67261||5351||4|33|3|1|10090|32|220|24|||||1702.56|1184.40|2155.44|0.51|||||999.32|999.83|-1133.32| +2450819|17873|2450897|12388|639272|6956|23440|12388|639272|6956|23440|4|26|20|1|4232|16|221|79|12.82|32.17|12.86|1525.49|1015.94|1012.78|2541.43|10.15|0.00|914.82|1015.94|1026.09|1930.76|1940.91|3.16| +2450819|17873|2450872|12388|639272|6956|23440|12388|639272|6956|23440|4|72|14|3|15794|133|221|99|48.51|93.13|38.18|5440.05|3779.82|4802.49|9219.87|75.59|0.00|1659.24|3779.82|3855.41|5439.06|5514.65|-1022.67| +2450819|17873|2450889|12388|639272|6956|23440|12388|639272|6956|23440|4|17|16|5|4438|6|221|21|54.11|105.51|67.52|797.79|1417.92|1136.31|2215.71|127.61|0.00|354.48|1417.92|1545.53|1772.40|1900.01|281.61| +2450819|17873|2450909|12388|639272|6956|23440|12388|639272|6956|23440|4|55|18|1|9550|167|221|15|10.66|13.43|4.29|137.10|64.35|159.90|201.45|5.79|0.00|3.90|64.35|70.14|68.25|74.04|-95.55| +2450819|17873|2450830|12388|639272|6956|23440|12388|639272|6956|23440|4|12|13|5|2714|81|221|97|54.99|92.93|25.09|6580.48|2433.73|5334.03|9014.21|146.02|0.00|2072.89|2433.73|2579.75|4506.62|4652.64|-2900.30| +2450819|17873|2450825|12388|639272|6956|23440|12388|639272|6956|23440|4|4|11|4|9712|115|221|72|48.92|97.84|81.20|1198.08|5846.40|3522.24|7044.48|467.71|0.00|2113.20|5846.40|6314.11|7959.60|8427.31|2324.16| +2450819|17873|2450872|12388|639272|6956|23440|12388|639272|6956|23440|4|94|5|3|5566|192|221|74|80.29|140.50|82.89|4263.14|6133.86|5941.46|10397.00|61.33|0.00|1767.12|6133.86|6195.19|7900.98|7962.31|192.40| +2450819|17873|2450896|12388|639272|6956|23440|12388|639272|6956|23440|4|85|17|2|8803|112|221|92|84.32|153.46|15.34|12707.04|1411.28|7757.44|14118.32|112.90|0.00|2116.92|1411.28|1524.18|3528.20|3641.10|-6346.16| +2450819|17873|2450829|12388|639272|6956|23440|12388|639272|6956|23440|4|89|18|2|10150|55|221|50|63.97|178.47|167.76|535.50|8388.00|3198.50|8923.50|671.04|0.00|3123.00|8388.00|9059.04|11511.00|12182.04|5189.50| +2450819|17873|2450882|12388|639272|6956|23440|12388|639272|6956|23440|4|22|14|1|1231|89|221|90|26.13|70.81|9.20|5544.90|828.00|2351.70|6372.90|24.84|0.00|190.80|828.00|852.84|1018.80|1043.64|-1523.70| +2450819|17873|2450835|12388|639272|6956|23440|12388|639272|6956|23440|4|75|18|2|340|237|221|88|37.82|44.62|24.54|1767.04|2159.52|3328.16|3926.56|64.78|0.00|313.28|2159.52|2224.30|2472.80|2537.58|-1168.64| +2450819|17873|2450845|12388|639272|6956|23440|12388|639272|6956|23440|4|9|10|3|2731|162|221|96|39.03|70.64|29.66|3934.08|2847.36|3746.88|6781.44|61.50|797.26|3050.88|2050.10|2111.60|5100.98|5162.48|-1696.78| +2450819|68138|2450824|50462|1644862|5662|33662|50462|1644862|5662|33662|1|62|17|1|3326|244|222|14|87.31|115.24|99.10|225.96|1387.40|1222.34|1613.36|41.62|0.00|548.52|1387.40|1429.02|1935.92|1977.54|165.06| +2450819|68138|2450898|50462|1644862|5662|33662|50462|1644862|5662|33662|1|58|8|1|16333|223|222|16|20.34|20.54|20.54|0.00|328.64|325.44|328.64|29.57|0.00|45.92|328.64|358.21|374.56|404.13|3.20| +2450819|68138|2450884|50462|1644862|5662|33662|50462|1644862|5662|33662|1|73|14|2|4730|113|222|5|6.41|17.81|3.20|73.05|16.00|32.05|89.05|0.32|0.00|37.40|16.00|16.32|53.40|53.72|-16.05| +2450819|68138|2450860|50462|1644862|5662|33662|50462|1644862|5662|33662|1|90|8|2|12973|1|222|63|12.40|15.12|14.51|38.43|914.13|781.20|952.56|0.00|0.00|238.14|914.13|914.13|1152.27|1152.27|132.93| +2450819|68138|2450823|50462|1644862|5662|33662|50462|1644862|5662|33662|1|89|6|5|3580|216|222|63|57.80|155.48|60.63|5975.55|3819.69|3641.40|9795.24|267.37|0.00|4407.48|3819.69|4087.06|8227.17|8494.54|178.29| +2450819|68138|2450824|50462|1644862|5662|33662|50462|1644862|5662|33662|1|99|10|4|7336|60|222|72|60.28|176.62|42.38|9665.28|3051.36|4340.16|12716.64|213.59|0.00|3560.40|3051.36|3264.95|6611.76|6825.35|-1288.80| +2450819|68138|2450894|50462|1644862|5662|33662|50462|1644862|5662|33662|1|77|1|2|17959|37|222|21|88.49|179.63|52.09|2678.34|1093.89|1858.29|3772.23|4.59|1017.31|1018.50|76.58|81.17|1095.08|1099.67|-1781.71| +2450819|66294|2450887|81509|119226|3343|34260|81509|119226|3343|34260|1|105|3|2|4886|290|223|14|32.24|35.78|31.48|60.20|440.72|451.36|500.92|39.66|0.00|215.32|440.72|480.38|656.04|695.70|-10.64| +2450819|66294|2450871|81509|119226|3343|34260|81509|119226|3343|34260|1|86|12|2|5842|150|223|14|65.44|128.91|46.40|1155.14|649.60|916.16|1804.74|25.98|0.00|613.48|649.60|675.58|1263.08|1289.06|-266.56| +2450819|66294|2450859|81509|119226|3343|34260|81509|119226|3343|34260|1|4|7|1|6208|112|223|20|19.17|21.85|8.95|258.00|179.00|383.40|437.00|16.11|0.00|96.00|179.00|195.11|275.00|291.11|-204.40| +2450819|66294|2450904|81509|119226|3343|34260|81509|119226|3343|34260|1|102|7|5|13798|210|223|85|21.03|23.13|4.16|1612.45|353.60|1787.55|1966.05|24.75|0.00|157.25|353.60|378.35|510.85|535.60|-1433.95| +2450819|66294|2450873|81509|119226|3343|34260|81509|119226|3343|34260|1|80|19|4|8108|276|223|31|59.90|143.16|143.16|0.00|4437.96|1856.90|4437.96|0.00|0.00|576.91|4437.96|4437.96|5014.87|5014.87|2581.06| +2450819|66294|2450899|81509|119226|3343|34260|81509|119226|3343|34260|1|59|12|4|271|241|223|62|69.73|207.09|138.75|4237.08|8602.50|4323.26|12839.58|603.89|1892.55|4365.42|6709.95|7313.84|11075.37|11679.26|2386.69| +2450819|66294|2450861|81509|119226|3343|34260|81509|119226|3343|34260|1|78|6|4|11374|20|223|88|24.43|30.29|19.68|933.68|1731.84|2149.84|2665.52|22.51|606.14|612.48|1125.70|1148.21|1738.18|1760.69|-1024.14| +2450819|66294|2450841|81509|119226|3343|34260|81509|119226|3343|34260|1|33|19|5|3880|76|223|45|91.64|235.51|181.34|2437.65|8160.30|4123.80|10597.95|391.69|326.41|3179.25|7833.89|8225.58|11013.14|11404.83|3710.09| +2450819|66294|2450850|81509|119226|3343|34260|81509|119226|3343|34260|1|22|20|5|1916|93|223|76|67.47|101.87|39.72|4723.40|3018.72|5127.72|7742.12|120.74|0.00|154.28|3018.72|3139.46|3173.00|3293.74|-2109.00| +2450819|66294|2450908|81509|119226|3343|34260|81509|119226|3343|34260|1|22|17|4|17851|86|223|50|75.84|178.98|132.44|2327.00|6622.00|3792.00|8949.00|132.44|0.00|1700.00|6622.00|6754.44|8322.00|8454.44|2830.00| +2450819|66294|2450864|81509|119226|3343|34260|81509|119226|3343|34260|1|54|10|3|13850|234|223|15|6.48|18.85|17.90|14.25|268.50|97.20|282.75|13.42|0.00|110.25|268.50|281.92|378.75|392.17|171.30| +2450819|66294|2450890|81509|119226|3343|34260|81509|119226|3343|34260|1|56|13|4|2281|229|223|57|28.39|66.71|24.68|2395.71|1406.76|1618.23|3802.47|2.25|1294.21|684.00|112.55|114.80|796.55|798.80|-1505.68| +2450819|66294|2450839|81509|119226|3343|34260|81509|119226|3343|34260|1|47|9|3|5083|74|223|53|33.20|98.27|28.49|3698.34|1509.97|1759.60|5208.31|105.69|0.00|885.10|1509.97|1615.66|2395.07|2500.76|-249.63| +2450819|72837|2450900|30073|675903|6886|44716|30073|675903|6886|44716|1|40|15|1|14749|106|224|21|12.12|32.23|8.37|501.06|175.77|254.52|676.83|14.06|0.00|40.53|175.77|189.83|216.30|230.36|-78.75| +2450819|72837|2450899|30073|675903|6886|44716|30073|675903|6886|44716|1|103|1|1|14977|56|224|34|77.69|125.85|103.19|770.44|3508.46|2641.46|4278.90|2.10|3438.29|213.86|70.17|72.27|284.03|286.13|-2571.29| +2450819|72837|2450887|30073|675903|6886|44716|30073|675903|6886|44716|1|26|18|4|301|126|224|56|85.07|225.43|169.07|3156.16|9467.92|4763.92|12624.08|757.43|0.00|3282.16|9467.92|10225.35|12750.08|13507.51|4704.00| +2450819|72837|2450861|30073|675903|6886|44716|30073|675903|6886|44716|1|20|20|5|12523|112|224|100|39.14|61.84|11.74|5010.00|1174.00|3914.00|6184.00|11.74|0.00|1051.00|1174.00|1185.74|2225.00|2236.74|-2740.00| +2450819|72837||30073||6886||30073|675903|6886|44716|||11|1|17575|60|224|99|||13.36|6944.85||5863.77||105.81||2562.12|1322.64||3884.76|3990.57|-4541.13| +2450819|72837|2450831|30073|675903|6886|44716|30073|675903|6886|44716|1|26|13|5|17392|224|224|68|52.14|142.34|56.93|5807.88|3871.24|3545.52|9679.12|25.55|3445.40|4548.52|425.84|451.39|4974.36|4999.91|-3119.68| +2450819|72837|2450838|30073|675903|6886|44716|30073|675903|6886|44716|1|8|14|2|7744|43|224|8|8.94|16.18|10.84|42.72|86.72|71.52|129.44|3.43|29.48|53.04|57.24|60.67|110.28|113.71|-14.28| +2450819|72837|2450855|30073|675903|6886|44716|30073|675903|6886|44716|1|106|1|4|15007|48|224|26|56.66|140.51|56.20|2192.06|1461.20|1473.16|3653.26|43.83|0.00|109.46|1461.20|1505.03|1570.66|1614.49|-11.96| +2450819|72837|2450874|30073|675903|6886|44716|30073|675903|6886|44716|1|35|11|3|12266|260|224|6|67.16|152.45|65.55|521.40|393.30|402.96|914.70|12.19|149.45|246.96|243.85|256.04|490.81|503.00|-159.11| +2450819|72837|2450877|30073|675903|6886|44716|30073|675903|6886|44716|1|65|9|2|3475|13|224|38|79.31|138.79|92.98|1740.78|3533.24|3013.78|5274.02|247.32|0.00|1002.06|3533.24|3780.56|4535.30|4782.62|519.46| +2450819|72837|2450905|30073|675903|6886|44716|30073|675903|6886|44716|1|102|8|4|13969|43|224|83|34.34|55.28|1.10|4496.94|91.30|2850.22|4588.24|3.65|0.00|183.43|91.30|94.95|274.73|278.38|-2758.92| +2450819|9679|2450862|13231|545744|3057|19523|13231|545744|3057|19523|2|36|12|4|15271|200|225|37|66.74|110.12|85.89|896.51|3177.93|2469.38|4074.44|63.55|0.00|366.67|3177.93|3241.48|3544.60|3608.15|708.55| +2450819|9679|2450851|13231|545744|3057|19523|13231|545744|3057|19523|2|26|12|4|14918|143|225|42|78.24|158.82|146.11|533.82|6136.62|3286.08|6670.44|368.19|0.00|3201.66|6136.62|6504.81|9338.28|9706.47|2850.54| +2450819|9679|2450838|13231|545744|3057|19523|13231|545744|3057|19523|2|59|13|4|15440|51|225|69|14.49|20.57|2.46|1249.59|169.74|999.81|1419.33|11.88|0.00|652.74|169.74|181.62|822.48|834.36|-830.07| +2450819|9679|2450905|13231|545744|3057|19523|13231|545744|3057|19523|2|69|7|3|9850|63|225|54|49.73|129.79|51.91|4205.52|2803.14|2685.42|7008.66|56.06|0.00|1331.64|2803.14|2859.20|4134.78|4190.84|117.72| +2450819|9679|2450831|13231|545744|3057|19523|13231|545744|3057|19523|2|64|6|1|14467|184|225|30|62.26|79.07|23.72|1660.50|711.60|1867.80|2372.10|6.26|85.39|23.70|626.21|632.47|649.91|656.17|-1241.59| +2450819|9679|2450897|13231|545744|3057|19523|13231|545744|3057|19523|2|105|16|2|16240|174|225|58|24.28|43.21|18.14|1454.06|1052.12|1408.24|2506.18|10.52|0.00|1127.52|1052.12|1062.64|2179.64|2190.16|-356.12| +2450819|9679|2450899|13231|545744|3057|19523|13231|545744|3057|19523|2|29|12|5|14222|163|225|46|18.19|39.47|18.55|962.32|853.30|836.74|1815.62|34.13|0.00|326.60|853.30|887.43|1179.90|1214.03|16.56| +2450819|29066|2450888|49302|236815|2728|4194|49302|236815|2728|4194|1|88|18|4|8380|164|226|50|23.70|41.23|29.27|598.00|1463.50|1185.00|2061.50|58.54|0.00|329.50|1463.50|1522.04|1793.00|1851.54|278.50| +2450819|29066|2450873|49302|236815|2728|4194|49302|236815|2728|4194|1|65|12|1|11488|40|226|87|21.89|45.75|23.33|1950.54|2029.71|1904.43|3980.25|40.59|0.00|198.36|2029.71|2070.30|2228.07|2268.66|125.28| +2450819|29066|2450846|49302|236815|2728|4194|49302|236815|2728|4194|1|108|2|3|12901|265|226|82|77.19|187.57|31.88|12766.58|2614.16|6329.58|15380.74|19.08|705.82|1691.66|1908.34|1927.42|3600.00|3619.08|-4421.24| +2450819|29066|2450835|49302|236815|2728|4194|49302|236815|2728|4194|1|31|18|5|10480|168|226|18|39.87|55.81|15.62|723.42|281.16|717.66|1004.58|5.62|0.00|80.28|281.16|286.78|361.44|367.06|-436.50| +2450819|29066|2450833|49302|236815|2728|4194|49302|236815|2728|4194|1|93|19|3|1220|256|226|4|67.69|92.05|3.68|353.48|14.72|270.76|368.20|1.03|0.00|22.08|14.72|15.75|36.80|37.83|-256.04| +2450819|29066|2450867|49302|236815|2728|4194|49302|236815|2728|4194|1|7|12|3|1990|17|226|42|27.82|70.94|36.17|1460.34|1519.14|1168.44|2979.48|54.68|911.48|387.24|607.66|662.34|994.90|1049.58|-560.78| +2450819|29066|2450894|49302|236815|2728|4194|49302|236815|2728|4194|1|29|11|3|13334|115|226|79|17.36|29.33|27.57|139.04|2178.03|1371.44|2317.07|65.34|0.00|393.42|2178.03|2243.37|2571.45|2636.79|806.59| +2450819|29066|2450870|49302|236815|2728|4194|49302|236815|2728|4194|1|72|2|4|12812|115|226|11|38.66|76.54|36.73|437.91|404.03|425.26|841.94|12.60|246.45|303.05|157.58|170.18|460.63|473.23|-267.68| +2450819|29066|2450821|49302|236815|2728|4194|49302|236815|2728|4194|1|19|10|5|12061|10|226|11|57.45|135.00|71.55|697.95|787.05|631.95|1485.00|15.74|0.00|727.65|787.05|802.79|1514.70|1530.44|155.10| +2450819|29066|2450906|49302|236815|2728|4194|49302|236815|2728|4194|1|67|18|3|9262|151|226|61|34.20|79.34|75.37|242.17|4597.57|2086.20|4839.74|137.92|0.00|145.18|4597.57|4735.49|4742.75|4880.67|2511.37| +2450819|29066|2450826|49302|236815|2728|4194|49302|236815|2728|4194|1|49|3|3|9344|141|226|39|9.49|12.52|8.51|156.39|331.89|370.11|488.28|13.27|0.00|43.68|331.89|345.16|375.57|388.84|-38.22| +2450819|29066|2450843|49302|236815|2728|4194|49302|236815|2728|4194|1|78|2|3|1|290|226|60|70.85|78.64|77.85|47.40|4671.00|4251.00|4718.40|233.55|0.00|46.80|4671.00|4904.55|4717.80|4951.35|420.00| +2450819|35206|2450881|32324|1823050|6129|55|8474|905060|6986|1265|1|34|19|2|13159|223|227|53|86.04|230.58|23.05|10999.09|1221.65|4560.12|12220.74|109.94|0.00|366.23|1221.65|1331.59|1587.88|1697.82|-3338.47| +2450819|35206|2450880|32324|1823050|6129|55|8474|905060|6986|1265|1|65|9|1|5650|215|227|99|34.73|95.85|86.26|949.41|8539.74|3438.27|9489.15|341.58|0.00|1517.67|8539.74|8881.32|10057.41|10398.99|5101.47| +2450819|35206|2450864|32324|1823050|6129|55|8474|905060|6986|1265|1|79|15|1|2452|282|227|66|57.36|91.77|61.48|1999.14|4057.68|3785.76|6056.82|324.61|0.00|2664.42|4057.68|4382.29|6722.10|7046.71|271.92| +2450819|35206|2450838|32324|1823050|6129|55|8474|905060|6986|1265|1|84|4|2|13231|220|227|41|36.05|40.37|20.58|811.39|843.78|1478.05|1655.17|0.67|776.27|297.66|67.51|68.18|365.17|365.84|-1410.54| +2450819|35206|2450904|32324|1823050|6129|55|8474|905060|6986|1265|1|13|17|4|10165|138|227|46|97.85|271.04|132.80|6359.04|6108.80|4501.10|12467.84|427.61|0.00|1495.92|6108.80|6536.41|7604.72|8032.33|1607.70| +2450819|22668|2450851|14783|1111526|1869|5812|14783|1111526|1869|5812|4|78|11|2|9326|278|228|6|4.85|11.34|10.43|5.46|62.58|29.10|68.04|0.00|62.58|1.32|0.00|0.00|1.32|1.32|-29.10| +2450819|22668|2450828|14783|1111526|1869|5812|14783|1111526|1869|5812|4|66|16|1|11846|212|228|46|55.41|148.49|89.09|2732.40|4098.14|2548.86|6830.54|327.85|0.00|2185.46|4098.14|4425.99|6283.60|6611.45|1549.28| +2450819|22668|2450831|14783|1111526|1869|5812|14783|1111526|1869|5812|4|24|4|5|7069|62|228|31|59.48|67.80|52.88|462.52|1639.28|1843.88|2101.80|16.39|0.00|861.49|1639.28|1655.67|2500.77|2517.16|-204.60| +2450819|22668|2450896|14783|1111526|1869|5812|14783|1111526|1869|5812|4|87|18|2|7778|9|228|58|97.74|147.58|23.61|7190.26|1369.38|5668.92|8559.64|0.00|0.00|770.24|1369.38|1369.38|2139.62|2139.62|-4299.54| +2450819|22668|2450880|14783|1111526|1869|5812|14783|1111526|1869|5812|4|58|15|4|1591|196|228|5|31.21|78.02|14.82|316.00|74.10|156.05|390.10|5.92|0.00|109.20|74.10|80.02|183.30|189.22|-81.95| +2450819|22668|2450860|14783|1111526|1869|5812|14783|1111526|1869|5812|4|31|3|3|4678|101|228|35|41.61|43.69|6.11|1315.30|213.85|1456.35|1529.15|4.27|0.00|320.95|213.85|218.12|534.80|539.07|-1242.50| +2450819|25109|2450886|23867|1247789|511|9233|23867|1247789|511|9233|1|56|8|2|727|33|229|72|25.57|36.05|26.67|675.36|1920.24|1841.04|2595.60|134.41|0.00|830.16|1920.24|2054.65|2750.40|2884.81|79.20| +2450819|25109|2450842|23867|1247789|511|9233|23867|1247789|511|9233|1|62|3|2|14860|193|229|20|88.87|155.52|108.86|933.20|2177.20|1777.40|3110.40|130.63|0.00|155.40|2177.20|2307.83|2332.60|2463.23|399.80| +2450819|25109|2450890|23867|1247789|511|9233|23867|1247789|511|9233|1|65|8|5|10939|190|229|87|99.09|203.13|93.43|9543.90|8128.41|8620.83|17672.31|81.28|0.00|5300.91|8128.41|8209.69|13429.32|13510.60|-492.42| +2450819|25109|2450898|23867|1247789|511|9233|23867|1247789|511|9233|1|4|1|3|5722|94|229|79|64.27|130.46|130.46|0.00|10306.34|5077.33|10306.34|309.19|0.00|1236.35|10306.34|10615.53|11542.69|11851.88|5229.01| +2450819|17198|2450867|7746|1322910|2074|20838|7746|1322910|2074|20838|2|24|11|3|4198|85|230|10|8.45|22.30|11.59|107.10|115.90|84.50|223.00|4.63|0.00|6.60|115.90|120.53|122.50|127.13|31.40| +2450819|17198|2450892|7746|1322910|2074|20838|7746|1322910|2074|20838|2|19|17|5|206|263|230|95|75.87|135.04|0.00|12828.80|0.00|7207.65|12828.80|0.00|0.00|3720.20|0.00|0.00|3720.20|3720.20|-7207.65| +2450819|17198|2450903|7746|1322910|2074|20838|7746|1322910|2074|20838|2|12|11|5|17182|195|230|56|61.42|99.50|39.80|3343.20|2228.80|3439.52|5572.00|133.72|0.00|1504.16|2228.80|2362.52|3732.96|3866.68|-1210.72| +2450819|17198|2450871|7746|1322910|2074|20838|7746|1322910|2074|20838|2|42|19|2|3175|12|230|23|93.20|114.63|6.87|2478.48|158.01|2143.60|2636.49|0.39|150.10|1186.34|7.91|8.30|1194.25|1194.64|-2135.69| +2450819|17198|2450825|7746|1322910|2074|20838|7746|1322910|2074|20838|2|49|10|2|15379|164|230|76|43.53|53.54|27.84|1953.20|2115.84|3308.28|4069.04|169.26|0.00|1342.16|2115.84|2285.10|3458.00|3627.26|-1192.44| +2450819|17198|2450888|7746|1322910|2074|20838|7746|1322910|2074|20838|2|33|1|3|124|94|230|14|32.29|67.16|25.52|582.96|357.28|452.06|940.24|14.29|0.00|0.00|357.28|371.57|357.28|371.57|-94.78| +2450819|17198|2450840|7746|1322910|2074|20838|7746|1322910|2074|20838|2|94|17|1|4582|288|230|99|70.52|139.62|48.86|8985.24|4837.14|6981.48|13822.38|338.59|0.00|3731.31|4837.14|5175.73|8568.45|8907.04|-2144.34| +2450819|17198|2450829|7746|1322910|2074|20838|7746|1322910|2074|20838|2|34|9|4|10736|126|230|26|83.90|208.91|18.80|4942.86|488.80|2181.40|5431.66|8.11|83.09|923.26|405.71|413.82|1328.97|1337.08|-1775.69| +2450819|17198|2450876|7746|1322910|2074|20838|7746|1322910|2074|20838|2|24|5|1|1864|233|230|75|6.76|17.23|17.23|0.00|1292.25|507.00|1292.25|38.76|0.00|322.50|1292.25|1331.01|1614.75|1653.51|785.25| +2450819|17198|2450856|7746|1322910|2074|20838|7746|1322910|2074|20838|2|103|15|2|2011|242|230|100|44.96|46.30|0.46|4584.00|46.00|4496.00|4630.00|2.76|0.00|1666.00|46.00|48.76|1712.00|1714.76|-4450.00| +2450819|17198|2450869|7746|1322910|2074|20838|7746|1322910|2074|20838|2|98|9|1|3740|145|230|22|47.43|131.38|69.63|1358.50|1531.86|1043.46|2890.36|0.00|428.92|115.50|1102.94|1102.94|1218.44|1218.44|59.48| +2450819|26586|2450828|18012|1327705|598|46291|18012|1327705|598|46291|1|13|2|4|6002|88|231|7|75.76|133.33|54.66|550.69|382.62|530.32|933.31|34.43|0.00|354.62|382.62|417.05|737.24|771.67|-147.70| +2450819|26586|2450833|18012|1327705|598|46291|18012|1327705|598|46291|1|61|4|3|6008|4|231|53|56.98|88.88|77.32|612.68|4097.96|3019.94|4710.64|245.87|0.00|1789.81|4097.96|4343.83|5887.77|6133.64|1078.02| +2450819|26586|2450906|18012|1327705|598|46291|18012|1327705|598|46291|1|53|2|2|8122|10|231|82|5.62|12.58|3.39|753.58|277.98|460.84|1031.56|19.45|0.00|505.12|277.98|297.43|783.10|802.55|-182.86| +2450819|26586|2450832|18012|1327705|598|46291|18012|1327705|598|46291|1|3|14|5|15718|145|231|55|66.70|97.38|78.87|1018.05|4337.85|3668.50|5355.90|86.75|0.00|1553.20|4337.85|4424.60|5891.05|5977.80|669.35| +2450819|26586|2450856|18012|1327705|598|46291|18012|1327705|598|46291|1|24|11|1|15001|116|231|81|15.21|39.54|17.39|1794.15|1408.59|1232.01|3202.74|28.17|0.00|1184.22|1408.59|1436.76|2592.81|2620.98|176.58| +2450819|26586|2450844|18012|1327705|598|46291|18012|1327705|598|46291|1|9|3|3|10618|213|231|67|88.32|149.26|7.46|9500.60|499.82|5917.44|10000.42|29.98|0.00|999.64|499.82|529.80|1499.46|1529.44|-5417.62| +2450819|26586|2450833|18012|1327705|598|46291|18012|1327705|598|46291|1|108|4|5|13363|64|231|62|16.98|20.37|8.75|720.44|542.50|1052.76|1262.94|16.27|0.00|189.10|542.50|558.77|731.60|747.87|-510.26| +2450819|26586|2450821|18012|1327705|598|46291|18012|1327705|598|46291|1|13|13|1|5780|227|231|90|66.04|98.39|1.96|8678.70|176.40|5943.60|8855.10|10.58|0.00|3807.00|176.40|186.98|3983.40|3993.98|-5767.20| +2450819|26586|2450844|18012|1327705|598|46291|18012|1327705|598|46291|1|12|10|1|10670|95|231|5|14.37|16.38|16.21|0.85|81.05|71.85|81.90|3.24|0.00|39.30|81.05|84.29|120.35|123.59|9.20| +2450819|37430|2450891|64532|1869320|155|26543|64532|1869320|155|26543|1|47|8|3|6700|152|232|6|45.45|80.44|19.30|366.84|115.80|272.70|482.64|6.94|0.00|202.68|115.80|122.74|318.48|325.42|-156.90| +2450819|37430|2450892|64532|1869320|155|26543|64532|1869320|155|26543|1|59|11|5|6604|238|232|64|78.45|233.78|231.44|149.76|14812.16|5020.80|14961.92|170.34|11405.36|5535.36|3406.80|3577.14|8942.16|9112.50|-1614.00| +2450819|37430|2450882|64532|1869320|155|26543|64532|1869320|155|26543|1|17|18|5|12025|251|232|8|90.84|176.22|58.15|944.56|465.20|726.72|1409.76|13.95|0.00|155.04|465.20|479.15|620.24|634.19|-261.52| +2450819|37430|2450863|64532|1869320|155|26543|64532|1869320|155|26543|1|25|5|2|5749|51|232|20|85.15|190.73|158.30|648.60|3166.00|1703.00|3814.60|221.62|0.00|1792.80|3166.00|3387.62|4958.80|5180.42|1463.00| +2450819|37430|2450859|64532|1869320|155|26543|64532|1869320|155|26543|1|5|14|5|13117|79|232|5|82.28|115.19|1.15|570.20|5.75|411.40|575.95|0.11|0.00|40.30|5.75|5.86|46.05|46.16|-405.65| +2450819|37430|2450878|64532|1869320|155|26543|64532|1869320|155|26543|1|106|20|1|5860|33|232|100|6.14|7.55|5.28|227.00|528.00|614.00|755.00|5.28|0.00|120.00|528.00|533.28|648.00|653.28|-86.00| +2450819|37430|2450867|64532|1869320|155|26543|64532|1869320|155|26543|1|80|18|4|3490|253|232|99|39.69|42.46|10.19|3194.73|1008.81|3929.31|4203.54|46.40|80.70|1344.42|928.11|974.51|2272.53|2318.93|-3001.20| +2450819|37430|2450831|64532|1869320|155|26543|64532|1869320|155|26543|1|59|17|2|8017|230|232|10|53.90|76.53|60.45|160.80|604.50|539.00|765.30|12.09|302.25|198.90|302.25|314.34|501.15|513.24|-236.75| +2450819|37430|2450821|64532|1869320|155|26543|64532|1869320|155|26543|1|49|9|4|3242|35|232|87|48.05|50.93|39.21|1019.64|3411.27|4180.35|4430.91|170.56|0.00|1417.23|3411.27|3581.83|4828.50|4999.06|-769.08| +2450819|43138|2450881|85888|275638|4235|23585|98800|320332|5729|46279|2|77|16|4|16798|282|233|85|32.79|36.06|10.09|2207.45|857.65|2787.15|3065.10|2.91|566.04|980.05|291.61|294.52|1271.66|1274.57|-2495.54| +2450819|43138|2450881|85888|275638|4235|23585|98800|320332|5729|46279|2|97|1|1|17996|51|233|18|70.44|107.06|25.69|1464.66|462.42|1267.92|1927.08|17.38|245.08|346.86|217.34|234.72|564.20|581.58|-1050.58| +2450819|43138|2450900|85888|275638|4235|23585|98800|320332|5729|46279|2|13|18|2|6253|42|233|85|59.60|149.00|58.11|7725.65|4939.35|5066.00|12665.00|0.00|0.00|1393.15|4939.35|4939.35|6332.50|6332.50|-126.65| +2450819|43138|2450851|85888|275638|4235|23585|98800|320332|5729|46279|2|34|17|3|15856|263|233|7|15.61|46.51|13.48|231.21|94.36|109.27|325.57|1.88|0.00|13.02|94.36|96.24|107.38|109.26|-14.91| +2450819|43138|2450871|85888|275638|4235|23585|98800|320332|5729|46279|2|69|13|2|4292|10|233|24|24.04|62.98|8.18|1315.20|196.32|576.96|1511.52|0.00|0.00|105.60|196.32|196.32|301.92|301.92|-380.64| +2450819|43138|2450903|85888|275638|4235|23585|98800|320332|5729|46279|2|50|14|4|4682|249|233|51|78.88|193.25|156.53|1872.72|7983.03|4022.88|9855.75|159.66|0.00|4336.53|7983.03|8142.69|12319.56|12479.22|3960.15| +2450819|78314|2450893|70981|749658|782|31906|70981|749658|782|31906|4|84|17|3|14620|291|234|50|96.02|194.92|128.64|3314.00|6432.00|4801.00|9746.00|514.56|0.00|4775.50|6432.00|6946.56|11207.50|11722.06|1631.00| +2450819|78314|2450821|70981|749658|782|31906|70981|749658|782|31906|4|73|7|3|14894|116|234|82|5.77|7.67|6.97|57.40|571.54|473.14|628.94|51.43|0.00|269.78|571.54|622.97|841.32|892.75|98.40| +2450819|78314|2450865|70981|749658|782|31906|70981|749658|782|31906|4|35|11|5|12181|200|234|35|41.77|54.30|26.06|988.40|912.10|1461.95|1900.50|16.60|674.95|170.80|237.15|253.75|407.95|424.55|-1224.80| +2450819|78314|2450884|70981|749658|782|31906|70981|749658|782|31906|4|40|15|4|949|49|234|66|12.29|24.33|9.24|995.94|609.84|811.14|1605.78|24.39|0.00|192.06|609.84|634.23|801.90|826.29|-201.30| +2450819|78314|2450867|70981|749658|782|31906|70981|749658|782|31906|4|101|11|4|14629|170|234|53|26.79|79.83|7.98|3808.05|422.94|1419.87|4230.99|0.00|207.24|1734.69|215.70|215.70|1950.39|1950.39|-1204.17| +2450819|78314|2450892|70981|749658|782|31906|70981|749658|782|31906|4|35|6|1|15610|271|234|57|52.60|101.51|43.64|3298.59|2487.48|2998.20|5786.07|124.37|0.00|57.57|2487.48|2611.85|2545.05|2669.42|-510.72| +2450819|78314|2450854|70981|749658|782|31906|70981|749658|782|31906|4|91|12|5|10063|33|234|62|74.98|95.97|51.82|2737.30|3212.84|4648.76|5950.14|289.15|0.00|1189.78|3212.84|3501.99|4402.62|4691.77|-1435.92| +2450819|78314|2450882|70981|749658|782|31906|70981|749658|782|31906|4|59|6|5|15692|166|234|87|84.85|163.76|4.91|13819.95|427.17|7381.95|14247.12|12.81|0.00|6838.20|427.17|439.98|7265.37|7278.18|-6954.78| +2450819|78314|2450904|70981|749658|782|31906|70981|749658|782|31906|4|37|7|3|12784|27|234|79|58.36|98.04|81.37|1316.93|6428.23|4610.44|7745.16|0.00|0.00|2245.97|6428.23|6428.23|8674.20|8674.20|1817.79| +2450819|78314|2450840|70981|749658|782|31906|70981|749658|782|31906|4|89|14|1|15772|50|234|58|9.13|27.02|25.93|63.22|1503.94|529.54|1567.16|60.15|0.00|187.92|1503.94|1564.09|1691.86|1752.01|974.40| +2450819|78314|2450832|70981|749658|782|31906|70981|749658|782|31906|4|99|1|5|13376|113|234|40|73.59|105.23|64.19|1641.60|2567.60|2943.60|4209.20|102.70|0.00|210.40|2567.60|2670.30|2778.00|2880.70|-376.00| +2450819|78314|2450837|70981|749658|782|31906|70981|749658|782|31906|4|29|19|2|17522|23|234|47|54.92|97.75|68.42|1378.51|3215.74|2581.24|4594.25|192.94|0.00|1883.29|3215.74|3408.68|5099.03|5291.97|634.50| +2450819|78314|2450889|70981|749658|782|31906|70981|749658|782|31906|4|48|10|5|16250|22|234|81|16.37|38.79|3.10|2890.89|251.10|1325.97|3141.99|5.02|0.00|784.89|251.10|256.12|1035.99|1041.01|-1074.87| +2450819|22566|2450883|9065|1881700|2335|36170|9065|1881700|2335|36170|4|74|9|5|14983|220|235|28|35.68|107.04|13.91|2607.64|389.48|999.04|2997.12|3.89|0.00|149.80|389.48|393.37|539.28|543.17|-609.56| +2450819|22566|2450849|9065|1881700|2335|36170|9065|1881700|2335|36170|4|24|4|5|1513|20|235|87|56.11|113.90|66.06|4162.08|5747.22|4881.57|9909.30|50.57|3218.44|1684.32|2528.78|2579.35|4213.10|4263.67|-2352.79| +2450819|22566|2450884|9065|1881700|2335|36170|9065|1881700|2335|36170|4|39|7|1|8431|204|235|50|94.05|263.34|52.66|10534.00|2633.00|4702.50|13167.00|78.99|0.00|4345.00|2633.00|2711.99|6978.00|7056.99|-2069.50| +2450819|22566|2450868|9065|1881700|2335|36170|9065|1881700|2335|36170|4|98|14|4|3746|54|235|98|53.83|136.72|123.04|1340.64|12057.92|5275.34|13398.56|0.00|0.00|6565.02|12057.92|12057.92|18622.94|18622.94|6782.58| +2450819|22566|2450884|9065|1881700|2335|36170|9065|1881700|2335|36170|4|95|15|3|2228|234|235|80|90.77|210.58|160.04|4043.20|12803.20|7261.60|16846.40|896.22|0.00|3537.60|12803.20|13699.42|16340.80|17237.02|5541.60| +2450819|22566|2450865|9065|1881700|2335|36170|9065|1881700|2335|36170|4|96|13|4|3229|93|235|32|90.98|235.63|122.52|3619.52|3920.64|2911.36|7540.16|78.41|0.00|1206.40|3920.64|3999.05|5127.04|5205.45|1009.28| +2450819|22566|2450836|9065|1881700|2335|36170|9065|1881700|2335|36170|4|85|12|4|12649|278|235|37|39.04|108.14|59.47|1800.79|2200.39|1444.48|4001.18|19.80|1804.31|1400.08|396.08|415.88|1796.16|1815.96|-1048.40| +2450819|22566|2450907|9065|1881700|2335|36170|9065|1881700|2335|36170|4|63|16|5|12068|64|235|82|76.59|94.97|0.94|7710.46|77.08|6280.38|7787.54|0.37|39.31|3659.66|37.77|38.14|3697.43|3697.80|-6242.61| +2450819|22566|2450886|9065|1881700|2335|36170|9065|1881700|2335|36170|4|21|18|1|2300|217|235|34|49.45|94.94|18.98|2582.64|645.32|1681.30|3227.96|51.62|0.00|1323.28|645.32|696.94|1968.60|2020.22|-1035.98| +2450819|22566|2450853|9065|1881700|2335|36170|9065|1881700|2335|36170|4|72|18|5|12022|16|235|21|74.63|191.05|168.12|481.53|3530.52|1567.23|4012.05|0.00|0.00|1644.93|3530.52|3530.52|5175.45|5175.45|1963.29| +2450819|22566|2450844|9065|1881700|2335|36170|9065|1881700|2335|36170|4|29|18|4|6962|287|235|63|51.22|102.95|35.00|4280.85|2205.00|3226.86|6485.85|22.05|0.00|2594.34|2205.00|2227.05|4799.34|4821.39|-1021.86| +2450819|22566|2450838|9065|1881700|2335|36170|9065|1881700|2335|36170|4|89|11|4|16544|265|235|74|45.70|102.36|71.65|2272.54|5302.10|3381.80|7574.64|477.18|0.00|1438.56|5302.10|5779.28|6740.66|7217.84|1920.30| +2450819|22566|2450847|9065|1881700|2335|36170|9065|1881700|2335|36170|4|16|12|4|9241|85|235|89|68.24|166.50|0.00|14818.50|0.00|6073.36|14818.50|0.00|0.00|4741.92|0.00|0.00|4741.92|4741.92|-6073.36| +2450819|22566|2450848|9065|1881700|2335|36170|9065|1881700|2335|36170|4|12|10|3|7705|264|235|65|95.46|218.60|179.25|2557.75|11651.25|6204.90|14209.00|1048.61|0.00|2273.05|11651.25|12699.86|13924.30|14972.91|5446.35| +2450819|61279|2450867|69234|811199|6764|7882|37771|1056565|1730|3648|2|21|7|5|15031|106|236|89|75.97|171.69|161.38|917.59|14362.82|6761.33|15280.41|1149.02|0.00|5348.01|14362.82|15511.84|19710.83|20859.85|7601.49| +2450819|61279|2450897|69234|811199|6764|7882|37771|1056565|1730|3648|2|60|19|3|16658|164|236|19|66.99|154.74|71.18|1587.64|1352.42|1272.81|2940.06|108.19|0.00|999.59|1352.42|1460.61|2352.01|2460.20|79.61| +2450819|61279|2450838|69234|811199|6764|7882|37771|1056565|1730|3648|2|15|11|4|7105|178|236|57|53.41|76.37|29.78|2655.63|1697.46|3044.37|4353.09|0.00|0.00|609.33|1697.46|1697.46|2306.79|2306.79|-1346.91| +2450819|61279|2450893|69234|811199|6764|7882|37771|1056565|1730|3648|2|21|2|4|1114|70|236|40|90.52|95.95|37.42|2341.20|1496.80|3620.80|3838.00|119.74|0.00|767.60|1496.80|1616.54|2264.40|2384.14|-2124.00| +2450819|61279|2450822|69234|811199|6764|7882|37771|1056565|1730|3648|2|75|13|3|10726|219|236|23|50.48|147.40|106.12|949.44|2440.76|1161.04|3390.20|170.85|0.00|644.00|2440.76|2611.61|3084.76|3255.61|1279.72| +2450819|61279|2450844|69234|811199|6764|7882|37771|1056565|1730|3648|2|47|19|2|5347|271|236|85|79.10|126.56|124.02|215.90|10541.70|6723.50|10757.60|527.08|0.00|0.00|10541.70|11068.78|10541.70|11068.78|3818.20| +2450819|61279|2450902|69234|811199|6764|7882|37771|1056565|1730|3648|2|52|18|4|10033|249|236|49|72.65|122.77|0.00|6015.73|0.00|3559.85|6015.73|0.00|0.00|2646.49|0.00|0.00|2646.49|2646.49|-3559.85| +2450819|61279|2450887|69234|811199|6764|7882|37771|1056565|1730|3648|2|8|19|3|17524|58|236|11|34.81|75.53|74.77|8.36|822.47|382.91|830.83|49.34|0.00|315.70|822.47|871.81|1138.17|1187.51|439.56| +2450819|61279|2450876|69234|811199|6764|7882|37771|1056565|1730|3648|2|74|2|3|4405|72|236|42|47.26|87.43|46.33|1726.20|1945.86|1984.92|3672.06|38.91|0.00|587.16|1945.86|1984.77|2533.02|2571.93|-39.06| +2450819|59029|2450849|30364|406226|41|27503|95041|183180|5442|34904|4|29|17|3|8498|8|237|14|35.50|80.94|63.94|238.00|895.16|497.00|1133.16|80.56|0.00|101.92|895.16|975.72|997.08|1077.64|398.16| +2450819|59029|2450904|30364|406226|41|27503|95041|183180|5442|34904|4|101|16|2|12790|34|237|50|39.77|84.71|44.89|1991.00|2244.50|1988.50|4235.50|89.78|0.00|720.00|2244.50|2334.28|2964.50|3054.28|256.00| +2450819|59029|2450899|30364|406226|41|27503|95041|183180|5442|34904|4|51|7|5|15464|66|237|98|27.94|61.74|50.00|1150.52|4900.00|2738.12|6050.52|392.00|0.00|2964.50|4900.00|5292.00|7864.50|8256.50|2161.88| +2450819|59029|2450830|30364|406226|41|27503|95041|183180|5442|34904|4|78|8|4|6412|253|237|67|53.74|84.90|45.84|2617.02|3071.28|3600.58|5688.30|214.98|0.00|2730.25|3071.28|3286.26|5801.53|6016.51|-529.30| +2450819|59029|2450858|30364|406226|41|27503|95041|183180|5442|34904|4|13|3|5|4870|92|237|7|96.96|279.24|228.97|351.89|1602.79|678.72|1954.68|48.08|0.00|449.54|1602.79|1650.87|2052.33|2100.41|924.07| +2450819|59029|2450878|30364|406226|41|27503|95041|183180|5442|34904|4|78|1|3|6142|213|237|28|12.56|31.77|21.28|293.72|595.84|351.68|889.56|13.82|250.25|329.00|345.59|359.41|674.59|688.41|-6.09| +2450819|59029|2450855|30364|406226|41|27503|95041|183180|5442|34904|4|101|20|3|5056|36|237|43|73.26|120.87|18.13|4417.82|779.59|3150.18|5197.41|0.00|0.00|1559.18|779.59|779.59|2338.77|2338.77|-2370.59| +2450819|59029|2450897|30364|406226|41|27503|95041|183180|5442|34904|4|66|2|4|4750|13|237|89|54.81|109.62|94.27|1366.15|8390.03|4878.09|9756.18|503.40|0.00|1950.88|8390.03|8893.43|10340.91|10844.31|3511.94| +2450819|59029|2450888|30364|406226|41|27503|95041|183180|5442|34904|4|84|19|3|17882|78|237|75|47.81|48.76|29.25|1463.25|2193.75|3585.75|3657.00|43.87|0.00|1206.75|2193.75|2237.62|3400.50|3444.37|-1392.00| +2450819|59029|2450834|30364|406226|41|27503|95041|183180|5442|34904|4|79|13|4|16768|171|237|4|1.52|2.88|0.00|11.52|0.00|6.08|11.52|0.00|0.00|1.84|0.00|0.00|1.84|1.84|-6.08| +2450819|59029|2450885|30364|406226|41|27503|95041|183180|5442|34904|4|83|17|2|8134|104|237|87|28.75|78.48|43.94|3004.98|3822.78|2501.25|6827.76|229.36|0.00|3208.56|3822.78|4052.14|7031.34|7260.70|1321.53| +2450819|61541|2450875|7116|455178|1287|41796|7116|455178|1287|41796|1|75|18|5|14642|35|238|94|74.56|108.11|67.02|3862.46|6299.88|7008.64|10162.34|62.99|0.00|2133.80|6299.88|6362.87|8433.68|8496.67|-708.76| +2450819|61541|2450841|7116|455178|1287|41796|7116|455178|1287|41796|1|7|9|1|11077|85|238|68|4.96|5.15|1.69|235.28|114.92|337.28|350.20|5.74|0.00|114.92|114.92|120.66|229.84|235.58|-222.36| +2450819|61541|2450900|7116|455178|1287|41796|7116|455178|1287|41796|1|105|15|5|1897|86|238|6|94.29|189.52|168.67|125.10|1012.02|565.74|1137.12|40.48|0.00|557.16|1012.02|1052.50|1569.18|1609.66|446.28| +2450819|61541|2450858|7116|455178|1287|41796|7116|455178|1287|41796|1|33|10|3|16372|262|238|57|47.19|96.73|76.41|1158.24|4355.37|2689.83|5513.61|43.55|0.00|1322.97|4355.37|4398.92|5678.34|5721.89|1665.54| +2450819|18001|2450829|20400|31049|3745|18794|20400|31049|3745|18794|1|11|13|4|6910|224|239|86|68.96|186.19|139.64|4003.30|12009.04|5930.56|16012.34|240.18|0.00|7525.00|12009.04|12249.22|19534.04|19774.22|6078.48| +2450819|18001|2450895|20400|31049|3745|18794|20400|31049|3745|18794|1|3|19|5|5752|116|239|12|7.83|20.20|1.01|230.28|12.12|93.96|242.40|0.24|0.00|31.44|12.12|12.36|43.56|43.80|-81.84| +2450819|18001|2450904|20400|31049|3745|18794|20400|31049|3745|18794|1|1|20|2|9910|79|239|23|83.17|168.00|47.04|2782.08|1081.92|1912.91|3864.00|97.37|0.00|463.68|1081.92|1179.29|1545.60|1642.97|-830.99| +2450819|18001|2450903|20400|31049|3745|18794|20400|31049|3745|18794|1|53|17|1|4096|99|239|91|21.88|63.67|57.30|579.67|5214.30|1991.08|5793.97|5.73|4640.72|1390.48|573.58|579.31|1964.06|1969.79|-1417.50| +2450819|18001|2450827|20400|31049|3745|18794|20400|31049|3745|18794|1|24|14|1|11974|153|239|41|69.39|70.08|34.33|1465.75|1407.53|2844.99|2873.28|84.45|0.00|603.11|1407.53|1491.98|2010.64|2095.09|-1437.46| +2450819|18001|2450828|20400|31049|3745|18794|20400|31049|3745|18794|1|9|4|3|14654|71|239|52|58.59|67.37|41.76|1331.72|2171.52|3046.68|3503.24|130.29|0.00|525.20|2171.52|2301.81|2696.72|2827.01|-875.16| +2450819|18001|2450884|20400|31049|3745|18794|20400|31049|3745|18794|1|102|9|5|7942|225|239|37|30.31|65.77|2.63|2336.18|97.31|1121.47|2433.49|8.75|0.00|705.59|97.31|106.06|802.90|811.65|-1024.16| +2450819|18001|2450856|20400|31049|3745|18794|20400|31049|3745|18794|1|44|13|1|12751|44|239|50|4.21|12.50|5.87|331.50|293.50|210.50|625.00|26.41|0.00|168.50|293.50|319.91|462.00|488.41|83.00| +2450819|18001|2450863|20400|31049|3745|18794|20400|31049|3745|18794|1|28|6|1|9622|235|239|96|92.58|169.42|121.98|4554.24|11710.08|8887.68|16264.32|234.20|0.00|6830.40|11710.08|11944.28|18540.48|18774.68|2822.40| +2450819|18001|2450866|20400|31049|3745|18794|20400|31049|3745|18794|1|74|7|4|12082|121|239|42|94.35|212.28|106.14|4457.88|4457.88|3962.70|8915.76|133.73|0.00|267.12|4457.88|4591.61|4725.00|4858.73|495.18| +2450819|18001|2450856|20400|31049|3745|18794|20400|31049|3745|18794|1|86|12|1|8468|244|239|68|31.39|90.71|36.28|3701.24|2467.04|2134.52|6168.28|74.01|0.00|1048.56|2467.04|2541.05|3515.60|3589.61|332.52| +2450819|18001|2450855|20400|31049|3745|18794|20400|31049|3745|18794|1|74|9|4|15536|139|239|81|18.57|42.71|14.52|2283.39|1176.12|1504.17|3459.51|35.28|0.00|1452.33|1176.12|1211.40|2628.45|2663.73|-328.05| +2450819|18001|2450871|20400|31049|3745|18794|20400|31049|3745|18794|1|62|2|2|12308|134|239|76|48.90|132.51|80.83|3927.68|6143.08|3716.40|10070.76|307.15|0.00|1409.80|6143.08|6450.23|7552.88|7860.03|2426.68| +2450819|18001|2450824|20400|31049|3745|18794|20400|31049|3745|18794|1|89|17|2|10862|197|239|22|74.84|127.97|35.83|2027.08|788.26|1646.48|2815.34|8.51|693.66|1323.08|94.60|103.11|1417.68|1426.19|-1551.88| +2450819|6588|2450849|84942|1313482|4869|36656|84942|1313482|4869|36656|4|3|4|4|17810|70|240|50|73.34|158.41|57.02|5069.50|2851.00|3667.00|7920.50|0.00|0.00|2217.50|2851.00|2851.00|5068.50|5068.50|-816.00| +2450819|6588|2450835|84942|1313482|4869|36656|84942|1313482|4869|36656|4|20|11|5|13774|141|240|33|8.36|20.06|1.20|622.38|39.60|275.88|661.98|0.79|0.00|198.33|39.60|40.39|237.93|238.72|-236.28| +2450819|6588|2450832|84942|1313482|4869|36656|84942|1313482|4869|36656|4|17|12|1|4222|281|240|87|51.78|86.47|11.24|6545.01|977.88|4504.86|7522.89|68.45|0.00|2106.27|977.88|1046.33|3084.15|3152.60|-3526.98| +2450819|6588|2450843|84942|1313482|4869|36656|84942|1313482|4869|36656|4|25|17|5|9205|176|240|12|17.48|42.12|30.32|141.60|363.84|209.76|505.44|14.55|0.00|227.40|363.84|378.39|591.24|605.79|154.08| +2450819|6588|2450894|84942|1313482|4869|36656|84942|1313482|4869|36656|4|45|8|4|4456|233|240|13|9.97|13.55|10.70|37.05|139.10|129.61|176.15|1.39|0.00|77.48|139.10|140.49|216.58|217.97|9.49| +2450819|6588|2450856|84942|1313482|4869|36656|84942|1313482|4869|36656|4|49|2|2|7201|167|240|24|85.13|115.77|112.29|83.52|2694.96|2043.12|2778.48|188.64|0.00|0.00|2694.96|2883.60|2694.96|2883.60|651.84| +2450819|6588|2450824|84942|1313482|4869|36656|84942|1313482|4869|36656|4|108|17|4|8812|148|240|35|64.74|165.08|112.25|1849.05|3928.75|2265.90|5777.80|0.00|0.00|404.25|3928.75|3928.75|4333.00|4333.00|1662.85| +2450819|6588|2450863|84942|1313482|4869|36656|84942|1313482|4869|36656|4|66|7|3|11960|65|240|39|77.14|205.96|173.00|1285.44|6747.00|3008.46|8032.44|269.88|0.00|1927.77|6747.00|7016.88|8674.77|8944.65|3738.54| +2450819|76507|2450863|19375|126238|6701|39564|19375|126238|6701|39564|2|92|2|3|7510|252|241|58|95.31|199.19|23.90|10166.82|1386.20|5527.98|11553.02|97.03|0.00|2772.40|1386.20|1483.23|4158.60|4255.63|-4141.78| +2450819|76507|2450855|19375|126238|6701|39564|19375|126238|6701|39564|2|17|2|5|13726|38|241|11|12.61|25.85|9.04|184.91|99.44|138.71|284.35|1.39|64.63|76.67|34.81|36.20|111.48|112.87|-103.90| +2450819|76507|2450904|19375|126238|6701|39564|19375|126238|6701|39564|2|10|19|5|16246|139|241|28|16.16|22.94|10.09|359.80|282.52|452.48|642.32|8.47|0.00|308.28|282.52|290.99|590.80|599.27|-169.96| +2450819|76507|2450908|19375|126238|6701|39564|19375|126238|6701|39564|2|97|15|5|10897|173|241|50|86.67|117.87|41.25|3831.00|2062.50|4333.50|5893.50|123.75|0.00|1001.50|2062.50|2186.25|3064.00|3187.75|-2271.00| +2450819|76507|2450834|19375|126238|6701|39564|19375|126238|6701|39564|2|34|14|5|14030|234|241|5|60.47|104.61|25.10|397.55|125.50|302.35|523.05|6.27|0.00|125.50|125.50|131.77|251.00|257.27|-176.85| +2450819|76507|2450868|19375|126238|6701|39564|19375|126238|6701|39564|2|37|14|2|9361|186|241|7|96.41|112.79|28.19|592.20|197.33|674.87|789.53|7.89|0.00|363.16|197.33|205.22|560.49|568.38|-477.54| +2450819|76507|2450890|19375|126238|6701|39564|19375|126238|6701|39564|2|108|13|5|6697|83|241|64|84.58|85.42|69.19|1038.72|4428.16|5413.12|5466.88|161.18|398.53|1803.52|4029.63|4190.81|5833.15|5994.33|-1383.49| +2450819|76507|2450829|19375|126238|6701|39564|19375|126238|6701|39564|2|18|4|3|16300|13|241|10|48.16|130.03|124.82|52.10|1248.20|481.60|1300.30|24.96|0.00|572.10|1248.20|1273.16|1820.30|1845.26|766.60| +2450819|76507|2450877|19375|126238|6701|39564|19375|126238|6701|39564|2|26|14|4|11480|229|241|93|38.32|71.65|54.45|1599.60|5063.85|3563.76|6663.45|303.83|0.00|1332.69|5063.85|5367.68|6396.54|6700.37|1500.09| +2450819|76507|2450895|19375|126238|6701|39564|19375|126238|6701|39564|2|78|3|5|4921|117|241|80|14.51|34.09|0.68|2672.80|54.40|1160.80|2727.20|3.23|0.54|81.60|53.86|57.09|135.46|138.69|-1106.94| +2450819|68702|2450887|25877|1885715|2706|21344|25877|1885715|2706|21344|4|106|5|2|5318|186|242|68|92.09|193.38|79.28|7758.80|5391.04|6262.12|13149.84|0.00|0.00|6179.84|5391.04|5391.04|11570.88|11570.88|-871.08| +2450819|68702|2450878|25877|1885715|2706|21344|25877|1885715|2706|21344|4|62|2|2|3760|84|242|1|9.94|12.62|0.12|12.50|0.12|9.94|12.62|0.00|0.00|0.75|0.12|0.12|0.87|0.87|-9.82| +2450819|68702|2450830|25877|1885715|2706|21344|25877|1885715|2706|21344|4|78|15|2|12278|221|242|68|49.83|115.10|88.62|1800.64|6026.16|3388.44|7826.80|189.82|2229.67|234.60|3796.49|3986.31|4031.09|4220.91|408.05| +2450819|68702|2450877|25877|1885715|2706|21344|25877|1885715|2706|21344|4|47|19|5|13657|173|242|23|99.06|146.60|36.65|2528.85|842.95|2278.38|3371.80|50.57|0.00|1517.31|842.95|893.52|2360.26|2410.83|-1435.43| +2450819|68702|2450858|25877|1885715|2706|21344|25877|1885715|2706|21344|4|57|16|3|3772|51|242|80|63.58|122.07|118.40|293.60|9472.00|5086.40|9765.60|278.47|4830.72|2148.00|4641.28|4919.75|6789.28|7067.75|-445.12| +2450819|68702|2450865|25877|1885715|2706|21344|25877|1885715|2706|21344|4|92|1|1|11668|201|242|46|69.37|165.79|129.31|1678.08|5948.26|3191.02|7626.34|187.37|2200.85|3278.88|3747.41|3934.78|7026.29|7213.66|556.39| +2450819|68702|2450844|25877|1885715|2706|21344|25877|1885715|2706|21344|4|101|10|4|668|279|242|44|32.80|51.82|27.98|1048.96|1231.12|1443.20|2280.08|0.00|0.00|113.96|1231.12|1231.12|1345.08|1345.08|-212.08| +2450819|68702|2450905|25877|1885715|2706|21344|25877|1885715|2706|21344|4|67|20|1|14887|214|242|83|80.13|217.15|152.00|5407.45|12616.00|6650.79|18023.45|504.64|0.00|4145.02|12616.00|13120.64|16761.02|17265.66|5965.21| +2450819|68702|2450832|25877|1885715|2706|21344|25877|1885715|2706|21344|4|6|2|3|16615|265|242|12|86.32|251.19|231.09|241.20|2773.08|1035.84|3014.28|92.89|915.11|1235.76|1857.97|1950.86|3093.73|3186.62|822.13| +2450819|68702|2450873|25877|1885715|2706|21344|25877|1885715|2706|21344|4|107|6|3|55|45|242|3|50.60|51.10|27.59|70.53|82.77|151.80|153.30|0.82|0.00|70.50|82.77|83.59|153.27|154.09|-69.03| +2450819|68702|2450867|25877|1885715|2706|21344|25877|1885715|2706|21344|4|97|15|4|6050|35|242|51|44.59|76.24|33.54|2177.70|1710.54|2274.09|3888.24|136.84|0.00|699.72|1710.54|1847.38|2410.26|2547.10|-563.55| +2450819|68702|2450876|25877|1885715|2706|21344|25877|1885715|2706|21344|4|90|11|2|1334|277|242|85|62.53|81.91|63.88|1532.55|5429.80|5315.05|6962.35|325.78|0.00|2297.55|5429.80|5755.58|7727.35|8053.13|114.75| +2450819|16034|2450839|86637|1117483|3529|41975|86637|1117483|3529|41975|4|40|11|4|13454|138|243|93|74.55|75.29|40.65|3221.52|3780.45|6933.15|7001.97|226.82|0.00|209.25|3780.45|4007.27|3989.70|4216.52|-3152.70| +2450819|16034|2450868|86637|1117483|3529|41975|86637|1117483|3529|41975|4|104|4|4|12446|37|243|92|99.04|113.89|28.47|7858.64|2619.24|9111.68|10477.88|209.53|0.00|4610.12|2619.24|2828.77|7229.36|7438.89|-6492.44| +2450819|16034|2450907|86637|1117483|3529|41975|86637|1117483|3529|41975|4|107|19|3|8749|233|243|62|79.65|169.65|132.32|2314.46|8203.84|4938.30|10518.30|426.60|2871.34|104.78|5332.50|5759.10|5437.28|5863.88|394.20| +2450819|16034|2450854|86637|1117483|3529|41975|86637|1117483|3529|41975|4|7|6|1|11440|117|243|49|68.87|202.47|105.28|4762.31|5158.72|3374.63|9921.03|0.00|0.00|2876.79|5158.72|5158.72|8035.51|8035.51|1784.09| +2450819|16034|2450878|86637|1117483|3529|41975|86637|1117483|3529|41975|4|108|6|1|199|211|243|73|16.54|28.11|8.99|1395.76|656.27|1207.42|2052.03|0.00|0.00|327.77|656.27|656.27|984.04|984.04|-551.15| +2450819|16034|2450894|86637|1117483|3529|41975|86637|1117483|3529|41975|4|34|5|2|2498|281|243|49|59.27|141.06|31.03|5391.47|1520.47|2904.23|6911.94|45.61|0.00|3248.21|1520.47|1566.08|4768.68|4814.29|-1383.76| +2450819|16034|2450851|86637|1117483|3529|41975|86637|1117483|3529|41975|4|64|17|5|4207|186|243|98|14.32|15.32|2.45|1261.26|240.10|1403.36|1501.36|2.88|192.08|705.60|48.02|50.90|753.62|756.50|-1355.34| +2450819|16034|2450835|86637|1117483|3529|41975|86637|1117483|3529|41975|4|5|15|3|3289|13|243|91|86.44|165.96|51.44|10421.32|4681.04|7866.04|15102.36|0.00|0.00|2718.17|4681.04|4681.04|7399.21|7399.21|-3185.00| +2450819|16034|2450885|86637|1117483|3529|41975|86637|1117483|3529|41975|4|55|10|2|11048|153|243|48|39.11|102.07|68.38|1617.12|3282.24|1877.28|4899.36|295.40|0.00|1763.52|3282.24|3577.64|5045.76|5341.16|1404.96| +2450819|16034|2450857|86637|1117483|3529|41975|86637|1117483|3529|41975|4|38|20|3|4942|281|243|57|97.85|254.41|241.68|725.61|13775.76|5577.45|14501.37|964.30|0.00|0.00|13775.76|14740.06|13775.76|14740.06|8198.31| +2450819|16034|2450900|86637|1117483|3529|41975|86637|1117483|3529|41975|4|34|12|5|4952|258|243|19|45.97|56.08|1.68|1033.60|31.92|873.43|1065.52|1.59|0.00|330.22|31.92|33.51|362.14|363.73|-841.51| +2450819|16034|2450831|86637|1117483|3529|41975|86637|1117483|3529|41975|4|59|10|4|1411|264|243|53|6.21|16.82|2.35|766.91|124.55|329.13|891.46|9.96|0.00|142.57|124.55|134.51|267.12|277.08|-204.58| +2450819|16034|2450858|86637|1117483|3529|41975|86637|1117483|3529|41975|4|57|2|5|17773|84|243|67|73.90|76.11|76.11|0.00|5099.37|4951.30|5099.37|203.97|2549.68|917.23|2549.69|2753.66|3466.92|3670.89|-2401.61| +2450819|50916|2450884|93373|1637389|2205|27821|93373|1637389|2205|27821|1|16|16|2|5503|66|244|63|52.24|83.06|13.28|4396.14|836.64|3291.12|5232.78|66.93|0.00|2511.18|836.64|903.57|3347.82|3414.75|-2454.48| +2450819|50916|2450875|93373|1637389|2205|27821|93373|1637389|2205|27821|1|15|13|3|775|211|244|9|26.04|33.85|2.03|286.38|18.27|234.36|304.65|1.64|0.00|15.21|18.27|19.91|33.48|35.12|-216.09| +2450819|50916|2450852|93373|1637389|2205|27821|93373|1637389|2205|27821|1|98|13|4|17989|203|244|57|97.40|197.72|92.92|5973.60|5296.44|5551.80|11270.04|423.71|0.00|3042.66|5296.44|5720.15|8339.10|8762.81|-255.36| +2450819|50916|2450846|93373|1637389|2205|27821|93373|1637389|2205|27821|1|3|1|4|889|54|244|3|97.00|228.92|201.44|82.44|604.32|291.00|686.76|1.93|555.97|281.55|48.35|50.28|329.90|331.83|-242.65| +2450819|50916|2450849|93373|1637389|2205|27821|93373|1637389|2205|27821|1|56|20|5|17197|200|244|98|15.13|35.25|13.39|2142.28|1312.22|1482.74|3454.50|0.00|0.00|1243.62|1312.22|1312.22|2555.84|2555.84|-170.52| +2450819|50916|2450825|93373|1637389|2205|27821|93373|1637389|2205|27821|1|86|10|2|14164|148|244|58|66.90|157.21|103.75|3100.68|6017.50|3880.20|9118.18|248.52|2467.17|91.06|3550.33|3798.85|3641.39|3889.91|-329.87| +2450819|50916|2450899|93373|1637389|2205|27821|93373|1637389|2205|27821|1|70|5|2|2389|11|244|21|26.87|80.07|32.02|1009.05|672.42|564.27|1681.47|10.08|168.10|706.02|504.32|514.40|1210.34|1220.42|-59.95| +2450819|50916|2450868|93373|1637389|2205|27821|93373|1637389|2205|27821|1|9|5|1|13732|197|244|20|4.29|11.79|4.48|146.20|89.60|85.80|235.80|3.44|40.32|89.60|49.28|52.72|138.88|142.32|-36.52| +2450819|50916|2450855|93373|1637389|2205|27821|93373|1637389|2205|27821|1|19|11|3|12577|287|244|49|38.04|89.01|2.67|4230.66|130.83|1863.96|4361.49|2.61|0.00|2006.06|130.83|133.44|2136.89|2139.50|-1733.13| +2450819|50916|2450833|93373|1637389|2205|27821|93373|1637389|2205|27821|1|103|18|5|3056|291|244|83|60.88|144.89|86.93|4810.68|7215.19|5053.04|12025.87|292.93|3030.37|480.57|4184.82|4477.75|4665.39|4958.32|-868.22| +2450819|50916|2450830|93373|1637389|2205|27821|93373|1637389|2205|27821|1|32|6|3|4778|218|244|6|32.71|86.68|36.40|301.68|218.40|196.26|520.08|4.36|0.00|244.38|218.40|222.76|462.78|467.14|22.14| +2450819|50916|2450902|93373|1637389|2205|27821|93373|1637389|2205|27821|1|64|1|4|2216|37|244|50|70.05|114.18|99.33|742.50|4966.50|3502.50|5709.00|0.00|0.00|1141.50|4966.50|4966.50|6108.00|6108.00|1464.00| +2450819|77945|2450880|63479|1175934|3236|6891|63479|1175934|3236|6891|2|50|3|4|4276|206|245|59|27.32|68.84|0.68|4021.44|40.12|1611.88|4061.56|0.61|32.49|1096.22|7.63|8.24|1103.85|1104.46|-1604.25| +2450819|77945|2450842|63479|1175934|3236|6891|63479|1175934|3236|6891|2|61|14|4|1040|253|245|42|20.66|28.09|17.97|425.04|754.74|867.72|1179.78|67.92|0.00|117.60|754.74|822.66|872.34|940.26|-112.98| +2450819|77945|2450879|63479|1175934|3236|6891|63479|1175934|3236|6891|2|92|12|1|160|224|245|16|3.98|5.77|3.05|43.52|48.80|63.68|92.32|2.92|0.00|0.80|48.80|51.72|49.60|52.52|-14.88| +2450819|77945|2450839|63479|1175934|3236|6891|63479|1175934|3236|6891|2|99|5|4|2332|295|245|38|15.69|39.69|31.75|301.72|1206.50|596.22|1508.22|36.19|0.00|467.40|1206.50|1242.69|1673.90|1710.09|610.28| +2450819|77945|2450891|63479|1175934|3236|6891|63479|1175934|3236|6891|2|31|17|5|12856|254|245|38|22.03|40.31|13.30|1026.38|505.40|837.14|1531.78|20.21|0.00|520.60|505.40|525.61|1026.00|1046.21|-331.74| +2450819|77945|2450829|63479|1175934|3236|6891|63479|1175934|3236|6891|2|32|9|1|10663|203|245|15|27.88|45.72|4.11|624.15|61.65|418.20|685.80|0.48|53.63|192.00|8.02|8.50|200.02|200.50|-410.18| +2450819|77945|2450889|63479|1175934|3236|6891|63479|1175934|3236|6891|2|87|2|1|13204|64|245|18|98.61|270.19|86.46|3307.14|1556.28|1774.98|4863.42|31.12|0.00|1215.72|1556.28|1587.40|2772.00|2803.12|-218.70| +2450819|77945|2450898|63479|1175934|3236|6891|63479|1175934|3236|6891|2|10|16|4|3284|134|245|94|78.05|96.00|61.44|3248.64|5775.36|7336.70|9024.00|106.26|4447.02|541.44|1328.34|1434.60|1869.78|1976.04|-6008.36| +2450819|77945|2450869|63479|1175934|3236|6891|63479|1175934|3236|6891|2|5|9|5|6577|132|245|71|49.08|117.79|53.00|4600.09|3763.00|3484.68|8363.09|261.90|489.19|668.82|3273.81|3535.71|3942.63|4204.53|-210.87| +2450819|77945|2450823|63479|1175934|3236|6891|63479|1175934|3236|6891|2|76|20|5|17200|73|245|50|59.36|120.50|34.94|4278.00|1747.00|2968.00|6025.00|1.39|1607.24|421.50|139.76|141.15|561.26|562.65|-2828.24| +2450819|77945|2450909|63479|1175934|3236|6891|63479|1175934|3236|6891|2|108|8|5|1681|41|245|60|43.52|117.93|79.01|2335.20|4740.60|2611.20|7075.80|426.65|0.00|2263.80|4740.60|5167.25|7004.40|7431.05|2129.40| +2450819|77945|2450898|63479|1175934|3236|6891|63479|1175934|3236|6891|2|32|18|4|11437|276|245|67|96.81|187.81|129.58|3901.41|8681.86|6486.27|12583.27|434.09|0.00|6039.38|8681.86|9115.95|14721.24|15155.33|2195.59| +2450819|77945|2450885|63479|1175934|3236|6891|63479|1175934|3236|6891|2|101|2|1|8828|137|245|25|50.46|109.49|105.11|109.50|2627.75|1261.50|2737.25|52.55|0.00|383.00|2627.75|2680.30|3010.75|3063.30|1366.25| +2450819|77945|2450895|63479|1175934|3236|6891|63479|1175934|3236|6891|2|39|16|1|13135|156|245|10|14.39|26.62|14.64|119.80|146.40|143.90|266.20|11.71|0.00|101.10|146.40|158.11|247.50|259.21|2.50| +2450819|12690|2450854|30606|944193|627|36214|30606|944193|627|36214|4|83|10|4|1328|54|246|16|50.14|58.16|56.41|28.00|902.56|802.24|930.56|27.07|0.00|316.32|902.56|929.63|1218.88|1245.95|100.32| +2450819|12690|2450861|30606|944193|627|36214|30606|944193|627|36214|4|68|13|2|11716|167|246|47|30.61|63.66|53.47|478.93|2513.09|1438.67|2992.02|100.52|0.00|627.92|2513.09|2613.61|3141.01|3241.53|1074.42| +2450819|12690|2450831|30606|944193|627|36214|30606|944193|627|36214|4|36|12|2|8455|30|246|12|86.19|132.73|66.36|796.44|796.32|1034.28|1592.76|7.96|0.00|47.76|796.32|804.28|844.08|852.04|-237.96| +2450819|12690|2450883|30606|944193|627|36214|30606|944193|627|36214|4|37|20|3|3550|65|246|33|6.10|17.87|14.83|100.32|489.39|201.30|589.71|34.25|0.00|288.75|489.39|523.64|778.14|812.39|288.09| +2450819|12690|2450888|30606|944193|627|36214|30606|944193|627|36214|4|46|18|1|8642|202|246|87|1.93|3.24|0.00|281.88|0.00|167.91|281.88|0.00|0.00|84.39|0.00|0.00|84.39|84.39|-167.91| +2450819|12690|2450856|30606|944193|627|36214|30606|944193|627|36214|4|60|17|4|9955|188|246|34|72.72|133.80|68.23|2229.38|2319.82|2472.48|4549.20|85.36|1252.70|1137.30|1067.12|1152.48|2204.42|2289.78|-1405.36| +2450819|12690|2450837|30606|944193|627|36214|30606|944193|627|36214|4|67|6|2|13472|187|246|14|93.06|116.32|80.26|504.84|1123.64|1302.84|1628.48|11.23|0.00|569.94|1123.64|1134.87|1693.58|1704.81|-179.20| +2450819|12690|2450862|30606|944193|627|36214|30606|944193|627|36214|4|49|12|3|7232|195|246|36|24.71|68.19|13.63|1964.16|490.68|889.56|2454.84|0.00|0.00|417.24|490.68|490.68|907.92|907.92|-398.88| +2450819|12690|2450887|30606|944193|627|36214|30606|944193|627|36214|4|43|9|5|8344|69|246|1|24.17|58.00|36.54|21.46|36.54|24.17|58.00|1.82|0.00|17.98|36.54|38.36|54.52|56.34|12.37| +2450819|12690|2450872|30606|944193|627|36214|30606|944193|627|36214|4|91|17|2|4540|14|246|8|81.65|204.94|30.74|1393.60|245.92|653.20|1639.52|4.91|147.55|573.76|98.37|103.28|672.13|677.04|-554.83| +2450819|12690|2450874|30606|944193|627|36214|30606|944193|627|36214|4|24|9|3|11323|31|246|18|30.88|48.17|1.44|841.14|25.92|555.84|867.06|0.40|5.70|294.66|20.22|20.62|314.88|315.28|-535.62| +2450819|12690|2450856|30606|944193|627|36214|30606|944193|627|36214|4|34|10|1|580|174|246|55|59.51|166.62|63.31|5682.05|3482.05|3273.05|9164.10|69.64|0.00|641.30|3482.05|3551.69|4123.35|4192.99|209.00| +2450819|12690|2450861|30606|944193|627|36214|30606|944193|627|36214|4|10|2|4|7970|182|246|71|74.69|202.40|188.23|1006.07|13364.33|5302.99|14370.40|502.49|7083.09|4023.57|6281.24|6783.73|10304.81|10807.30|978.25| +2450819|12690|2450871|30606|944193|627|36214|30606|944193|627|36214|4|94|20|4|4|132|246|54|2.56|4.78|0.90|209.52|48.60|138.24|258.12|4.37|0.00|51.30|48.60|52.97|99.90|104.27|-89.64| +2450819|61094|2450874|46684|619168|6852|37736|46684|619168|6852|37736|1|34|15|1|10789|246|247|24|20.22|40.64|17.88|546.24|429.12|485.28|975.36|21.45|0.00|58.32|429.12|450.57|487.44|508.89|-56.16| +2450819|61094|2450908|46684|619168|6852|37736|46684|619168|6852|37736|1|49|14|2|12818|61|247|93|22.63|34.85|21.25|1264.80|1976.25|2104.59|3241.05|0.00|1818.15|291.09|158.10|158.10|449.19|449.19|-1946.49| +2450819|61094|2450826|46684|619168|6852|37736|46684|619168|6852|37736|1|49|8|2|16954|241|247|41|68.84|75.03|51.77|953.66|2122.57|2822.44|3076.23|26.32|806.57|553.50|1316.00|1342.32|1869.50|1895.82|-1506.44| +2450819|61094|2450822|46684|619168|6852|37736|46684|619168|6852|37736|1|43|12|4|14404|8|247|46|89.11|234.35|222.63|539.12|10240.98|4099.06|10780.10|0.00|7680.73|4312.04|2560.25|2560.25|6872.29|6872.29|-1538.81| +2450819|61094|2450826|46684|619168|6852|37736|46684|619168|6852|37736|1|99|6|1|17239|143|247|34|47.85|127.75|22.99|3561.84|781.66|1626.90|4343.50|15.63|0.00|607.92|781.66|797.29|1389.58|1405.21|-845.24| +2450819|61094|2450830|46684|619168|6852|37736|46684|619168|6852|37736|1|43|13|1|14023|109|247|55|81.88|83.51|43.42|2204.95|2388.10|4503.40|4593.05|0.00|0.00|1147.85|2388.10|2388.10|3535.95|3535.95|-2115.30| +2450819|61094|2450875|46684|619168|6852|37736|46684|619168|6852|37736|1|52|16|4|8512|69|247|96|43.77|120.36|58.97|5893.44|5661.12|4201.92|11554.56|56.61|0.00|3350.40|5661.12|5717.73|9011.52|9068.13|1459.20| +2450819|61094|2450899|46684|619168|6852|37736|46684|619168|6852|37736|1|65|18|5|5326|154|247|31|29.03|40.64|33.32|226.92|1032.92|899.93|1259.84|20.65|0.00|617.21|1032.92|1053.57|1650.13|1670.78|132.99| +2450819|61094|2450828|46684|619168|6852|37736|46684|619168|6852|37736|1|6|11|3|16790|58|247|28|52.35|58.10|47.64|292.88|1333.92|1465.80|1626.80|90.57|40.01|666.96|1293.91|1384.48|1960.87|2051.44|-171.89| +2450819|61094|2450845|46684|619168|6852|37736|46684|619168|6852|37736|1|84|4|4|12644|182|247|55|25.38|69.28|7.62|3391.30|419.10|1395.90|3810.40|1.46|272.41|532.95|146.69|148.15|679.64|681.10|-1249.21| +2450819|61094|2450881|46684|619168|6852|37736|46684|619168|6852|37736|1|39|1|5|5635|120|247|87|18.84|51.99|29.11|1990.56|2532.57|1639.08|4523.13|126.62|0.00|1356.33|2532.57|2659.19|3888.90|4015.52|893.49| +2450819|70702|2450845|52030|285427|4871|13868|52030|285427|4871|13868|2|77|5|5|6590|290|248|53|28.37|64.96|50.66|757.90|2684.98|1503.61|3442.88|214.79|0.00|791.82|2684.98|2899.77|3476.80|3691.59|1181.37| +2450819|70702|2450851|52030|285427|4871|13868|52030|285427|4871|13868|2|18|17|5|10120|159|248|86|61.99|181.63|112.61|5935.72|9684.46|5331.14|15620.18|581.06|0.00|467.84|9684.46|10265.52|10152.30|10733.36|4353.32| +2450819|70702|2450830|52030|285427|4871|13868|52030|285427|4871|13868|2|63|16|2|5386|122|248|91|56.86|160.91|94.93|6004.18|8638.63|5174.26|14642.81|518.31|0.00|4832.10|8638.63|9156.94|13470.73|13989.04|3464.37| +2450819|70702|2450877|52030|285427|4871|13868|52030|285427|4871|13868|2|31|6|1|17068|168|248|55|18.69|52.14|48.49|200.75|2666.95|1027.95|2867.70|240.02|0.00|114.40|2666.95|2906.97|2781.35|3021.37|1639.00| +2450819|70702|2450829|52030|285427|4871|13868|52030|285427|4871|13868|2|40|19|3|10514|227|248|79|33.08|66.16|41.68|1933.92|3292.72|2613.32|5226.64|131.70|0.00|1097.31|3292.72|3424.42|4390.03|4521.73|679.40| +2450819|70702|2450846|52030|285427|4871|13868|52030|285427|4871|13868|2|39|20|5|6254|10|248|6|77.20|97.27|90.46|40.86|542.76|463.20|583.62|38.64|59.70|145.86|483.06|521.70|628.92|667.56|19.86| +2450819|70702|2450847|52030|285427|4871|13868|52030|285427|4871|13868|2|26|13|3|10876|56|248|74|77.97|89.66|9.86|5905.20|729.64|5769.78|6634.84|65.66|0.00|1591.74|729.64|795.30|2321.38|2387.04|-5040.14| +2450819|70702|2450837|52030|285427|4871|13868|52030|285427|4871|13868|2|24|6|2|14767|130|248|80|16.02|27.71|23.83|310.40|1906.40|1281.60|2216.80|152.51|0.00|620.00|1906.40|2058.91|2526.40|2678.91|624.80| +2450819|44476|2450872|12871|1351573|1475|38977|12871|1351573|1475|38977|1|64|1|2|12872|171|249|99|74.68|106.79|17.08|8881.29|1690.92|7393.32|10572.21|16.90|0.00|4228.29|1690.92|1707.82|5919.21|5936.11|-5702.40| +2450819|44476|2450899|12871|1351573|1475|38977|12871|1351573|1475|38977|1|23|7|2|2515|37|249|6|48.71|74.52|66.32|49.20|397.92|292.26|447.12|15.91|0.00|89.40|397.92|413.83|487.32|503.23|105.66| +2450819|44476|2450846|12871|1351573|1475|38977|12871|1351573|1475|38977|1|62|10|1|15098|232|249|88|39.34|98.35|71.79|2337.28|6317.52|3461.92|8654.80|314.61|1073.97|3201.44|5243.55|5558.16|8444.99|8759.60|1781.63| +2450819|44476|2450851|12871|1351573|1475|38977|12871|1351573|1475|38977|1|88|9|1|5668|190|249|23|59.31|85.40|57.21|648.37|1315.83|1364.13|1964.20|105.26|0.00|903.44|1315.83|1421.09|2219.27|2324.53|-48.30| +2450819|58238|2450888|5121|1048724|7080|20283|5121|1048724|7080|20283|4|35|11|2|9590|79|250|100|77.24|190.01|60.80|12921.00|6080.00|7724.00|19001.00|121.60|0.00|4370.00|6080.00|6201.60|10450.00|10571.60|-1644.00| +2450819|58238|2450849|5121|1048724|7080|20283|5121|1048724|7080|20283|4|93|6|2|3194|219|250|71|17.08|23.05|8.98|998.97|637.58|1212.68|1636.55|57.38|0.00|48.99|637.58|694.96|686.57|743.95|-575.10| +2450819||2450880|5121|1048724|7080|20283||1048724||||28||3|1832|287|250||25.87|||8.10|783.30|776.10|||0.00||783.30||1044.30|1106.96|7.20| +2450819|58238|2450875|5121|1048724|7080|20283|5121|1048724|7080|20283|4|67|14|3|1570|111|250|95|95.33|151.57|81.84|6624.35|7774.80|9056.35|14399.15|544.23|0.00|6335.55|7774.80|8319.03|14110.35|14654.58|-1281.55| +2450819|58238|2450852|5121|1048724|7080|20283|5121|1048724|7080|20283|4|2|19|5|16126|192|250|50|84.36|162.81|92.80|3500.50|4640.00|4218.00|8140.50|46.40|0.00|569.50|4640.00|4686.40|5209.50|5255.90|422.00| +2450819|58238|2450906|5121|1048724|7080|20283|5121|1048724|7080|20283|4|99|1|4|3049|173|250|88|37.87|63.24|49.32|1224.96|4340.16|3332.56|5565.12|303.81|0.00|1557.60|4340.16|4643.97|5897.76|6201.57|1007.60| +2450819|58238|2450885|5121|1048724|7080|20283|5121|1048724|7080|20283|4|74|5|3|15175|6|250|38|3.70|8.47|7.87|22.80|299.06|140.60|321.86|0.00|0.00|31.92|299.06|299.06|330.98|330.98|158.46| +2450819|58238|2450886|5121|1048724|7080|20283|5121|1048724|7080|20283|4|106|3|1|2362|165|250|81|90.94|95.48|27.68|5491.80|2242.08|7366.14|7733.88|22.42|0.00|772.74|2242.08|2264.50|3014.82|3037.24|-5124.06| +2450819|58238|2450864|5121|1048724|7080|20283|5121|1048724|7080|20283|4|44|1|2|6415|107|250|91|34.49|94.84|40.78|4919.46|3710.98|3138.59|8630.44|28.94|2263.69|3020.29|1447.29|1476.23|4467.58|4496.52|-1691.30| +2450819|58238|2450833|5121|1048724|7080|20283|5121|1048724|7080|20283|4|33|4|4|8564|129|250|56|94.58|246.85|120.95|7050.40|6773.20|5296.48|13823.60|541.85|0.00|6358.80|6773.20|7315.05|13132.00|13673.85|1476.72| +2450819|58238|2450821|5121|1048724|7080|20283|5121|1048724|7080|20283|4|77|20|4|16930|287|250|64|42.88|113.63|56.81|3636.48|3635.84|2744.32|7272.32|218.15|0.00|290.56|3635.84|3853.99|3926.40|4144.55|891.52| +2450819|58238|2450870|5121|1048724|7080|20283|5121|1048724|7080|20283|4|35|13|4|6020|297|250|90|90.34|151.77|145.69|547.20|13112.10|8130.60|13659.30|917.84|0.00|5736.60|13112.10|14029.94|18848.70|19766.54|4981.50| +2450819|79144|2450860|20301|975317|380|7843|20301|975317|380|7843|4|59|5|2|3691|171|251|84|40.59|67.78|57.61|854.28|4839.24|3409.56|5693.52|387.13|0.00|2504.88|4839.24|5226.37|7344.12|7731.25|1429.68| +2450819|79144|2450869|20301|975317|380|7843|20301|975317|380|7843|4|9|15|5|9958|268|251|77|12.19|24.01|6.72|1331.33|517.44|938.63|1848.77|25.87|0.00|535.92|517.44|543.31|1053.36|1079.23|-421.19| +2450819|79144|2450873|20301|975317|380|7843|20301|975317|380|7843|4|18|20|3|986|217|251|14|89.94|142.10|122.20|278.60|1710.80|1259.16|1989.40|34.21|0.00|457.52|1710.80|1745.01|2168.32|2202.53|451.64| +2450819|79144|2450893|20301|975317|380|7843|20301|975317|380|7843|4|86|13|4|196|60|251|77|16.72|16.72|3.84|991.76|295.68|1287.44|1287.44|20.69|0.00|140.91|295.68|316.37|436.59|457.28|-991.76| +2450819|79144|2450857|20301|975317|380|7843|20301|975317|380|7843|4|58|11|3|15901|194|251|69|39.92|73.45|8.81|4460.16|607.89|2754.48|5068.05|18.23|0.00|1215.78|607.89|626.12|1823.67|1841.90|-2146.59| +2450819|79144|2450864|20301|975317|380|7843|20301|975317|380|7843|4|48|5|1|2036|108|251|74|83.30|216.58|164.60|3846.52|12180.40|6164.20|16026.92|103.53|10109.73|1922.52|2070.67|2174.20|3993.19|4096.72|-4093.53| +2450819|79144|2450867||975317|380|7843|20301|975317|||4||11|4|7082||251|75||93.18||||||9.76||279.00|||418.50|428.26|-2559.00| +2450819|79144|2450905|20301|975317|380|7843|20301|975317|380|7843|4|89|4|2|7660|276|251|20|75.54|210.00|102.90|2142.00|2058.00|1510.80|4200.00|82.32|0.00|462.00|2058.00|2140.32|2520.00|2602.32|547.20| +2450819|79144|2450875|20301|975317|380|7843|20301|975317|380|7843|4|38|4|1|10820|275|251|46|91.88|159.87|71.94|4044.78|3309.24|4226.48|7354.02|132.37|661.84|1764.56|2647.40|2779.77|4411.96|4544.33|-1579.08| +2450819|79144|2450890|20301|975317|380|7843|20301|975317|380|7843|4|105|2|2|1804|149|251|75|21.16|56.28|6.75|3714.75|506.25|1587.00|4221.00|30.37|0.00|970.50|506.25|536.62|1476.75|1507.12|-1080.75| +2450819|79144|2450839|20301|975317|380|7843|20301|975317|380|7843|4|25|20|3|11248|200|251|39|33.20|57.76|15.59|1644.63|608.01|1294.80|2252.64|18.24|0.00|292.50|608.01|626.25|900.51|918.75|-686.79| +2450819|79144|2450882|20301|975317|380|7843|20301|975317|380|7843|4|78|3|1|9136|288|251|79|19.34|30.94|9.90|1662.16|782.10|1527.86|2444.26|70.38|0.00|391.05|782.10|852.48|1173.15|1243.53|-745.76| +2450819|79144|2450885|20301|975317|380|7843|20301|975317|380|7843|4|99|2|5|14422|242|251|44|65.11|115.89|108.93|306.24|4792.92|2864.84|5099.16|95.85|0.00|1631.52|4792.92|4888.77|6424.44|6520.29|1928.08| +2450819|79144|2450901|20301|975317|380|7843|20301|975317|380|7843|4|55|2|3|16447|60|251|22|10.39|14.64|12.88|38.72|283.36|228.58|322.08|22.66|0.00|32.12|283.36|306.02|315.48|338.14|54.78| +2450819|76574|2450909|60494|1112565|1083|9154|60494|1112565|1083|9154|4|38|12|2|15043|37|252|67|96.14|167.28|125.46|2801.94|8405.82|6441.38|11207.76|3.36|8321.76|1568.47|84.06|87.42|1652.53|1655.89|-6357.32| +2450819|76574|2450887|60494|1112565|1083|9154|60494|1112565|1083|9154|4|66|13|1|1654|185|252|70|40.73|72.49|4.34|4770.50|303.80|2851.10|5074.30|27.34|0.00|2536.80|303.80|331.14|2840.60|2867.94|-2547.30| +2450819|76574|2450906|60494|1112565|1083|9154|60494|1112565|1083|9154|4|57|14|2|337|210|252|65|7.28|17.03|2.04|974.35|132.60|473.20|1106.95|0.00|94.14|497.90|38.46|38.46|536.36|536.36|-434.74| +2450819|76574|2450889|60494|1112565|1083|9154|60494|1112565|1083|9154|4|31|18|1|14570|204|252|76|93.78|254.14|0.00|19314.64|0.00|7127.28|19314.64|0.00|0.00|1931.16|0.00|0.00|1931.16|1931.16|-7127.28| +2450819|37617|2450903|56019|1385881|1790|46499|56019|1385881|1790|46499|2|76|2|4|13318|38|253|57|19.79|25.52|19.65|334.59|1120.05|1128.03|1454.64|0.00|0.00|683.43|1120.05|1120.05|1803.48|1803.48|-7.98| +2450819|37617|2450855|56019|1385881|1790|46499|56019|1385881|1790|46499|2|92|13|2|14936|135|253|42|58.24|83.86|3.35|3381.42|140.70|2446.08|3522.12|11.25|0.00|1232.70|140.70|151.95|1373.40|1384.65|-2305.38| +2450819|37617|2450892|56019|1385881|1790|46499|56019|1385881|1790|46499|2|50|14|2|2959|111|253|68|98.59|264.22|219.30|3054.56|14912.40|6704.12|17966.96|447.37|0.00|2694.84|14912.40|15359.77|17607.24|18054.61|8208.28| +2450819|37617|2450854|56019|1385881|1790|46499|56019|1385881|1790|46499|2|73|10|1|13795|13|253|74|93.02|108.83|81.62|2013.54|6039.88|6883.48|8053.42|483.19|0.00|885.78|6039.88|6523.07|6925.66|7408.85|-843.60| +2450819|85794|2450905|16687|691102|3564|42826|16687|691102|3564|42826|4|12|16|1|932|191|254|24|14.06|27.97|14.54|322.32|348.96|337.44|671.28|13.95|0.00|308.64|348.96|362.91|657.60|671.55|11.52| +2450819|85794|2450885|16687|691102|3564|42826|16687|691102|3564|42826|4|80|3|5|5242|4|254|67|88.94|115.62|98.27|1162.45|6584.09|5958.98|7746.54|329.20|0.00|2014.02|6584.09|6913.29|8598.11|8927.31|625.11| +2450819|85794|2450860|16687|691102|3564|42826|16687|691102|3564|42826|4|67|3|1|11749|80|254|44|1.20|1.58|0.11|64.68|4.84|52.80|69.52|0.04|0.00|10.12|4.84|4.88|14.96|15.00|-47.96| +2450819|85794|2450865|16687|691102|3564|42826|16687|691102|3564|42826|4|36|9|5|16846|140|254|44|94.46|270.15|183.70|3803.80|8082.80|4156.24|11886.60|0.00|0.00|3684.56|8082.80|8082.80|11767.36|11767.36|3926.56| +2450819|85794|2450830|16687|691102|3564|42826|16687|691102|3564|42826|4|101|14|4|11572|15|254|93|11.95|16.96|0.50|1530.78|46.50|1111.35|1577.28|3.72|0.00|362.70|46.50|50.22|409.20|412.92|-1064.85| +2450819|35617|2450905|1230|177318|7006|38294|77913|1062470|2558|21046|1|9|1|1|5803|71|255|72|80.47|219.68|105.44|8225.28|7591.68|5793.84|15816.96|607.33|0.00|474.48|7591.68|8199.01|8066.16|8673.49|1797.84| +2450819|35617|2450846|1230|177318|7006|38294|77913|1062470|2558|21046|1|16|12|4|6877|218|255|61|70.98|173.19|46.76|7712.23|2852.36|4329.78|10564.59|171.14|0.00|2217.96|2852.36|3023.50|5070.32|5241.46|-1477.42| +2450819|35617|2450852|1230|177318|7006|38294|77913|1062470|2558|21046|1|76|10|4|16670|182|255|57|98.73|265.58|188.56|4390.14|10747.92|5627.61|15138.06|967.31|0.00|3632.61|10747.92|11715.23|14380.53|15347.84|5120.31| +2450819|35617|2450903|1230|177318|7006|38294|77913|1062470|2558|21046|1|63|9|4|14716|53|255|70|41.33|99.19|66.45|2291.80|4651.50|2893.10|6943.30|418.63|0.00|2638.30|4651.50|5070.13|7289.80|7708.43|1758.40| +2450819|35617|2450862|1230|177318|7006|38294|77913|1062470|2558|21046|1|99|6|5|16072|33|255|15|7.44|16.07|11.40|70.05|171.00|111.60|241.05|7.54|63.27|113.25|107.73|115.27|220.98|228.52|-3.87| +2450819|35617|2450889|1230|177318|7006|38294|77913|1062470|2558|21046|1|108|8|1|12584|87|255|99|45.97|122.28|13.45|10774.17|1331.55|4551.03|12105.72|79.89|0.00|3147.21|1331.55|1411.44|4478.76|4558.65|-3219.48| +2450819|35617|2450876|1230|177318|7006|38294|77913|1062470|2558|21046|1|7|10|3|274|231|255|84|97.85|266.15|218.24|4024.44|18332.16|8219.40|22356.60|82.49|10082.68|5365.08|8249.48|8331.97|13614.56|13697.05|30.08| +2450819|35617|2450882|1230|177318|7006|38294|77913|1062470|2558|21046|1|80|19|3|10772|18|255|66|55.68|147.55|69.34|5161.86|4576.44|3674.88|9738.30|45.76|0.00|681.12|4576.44|4622.20|5257.56|5303.32|901.56| +2450819|35617|2450898|1230|177318|7006|38294|77913|1062470|2558|21046|1|33|15|3|7262|72|255|14|86.25|245.81|117.98|1789.62|1651.72|1207.50|3441.34|16.51|0.00|103.18|1651.72|1668.23|1754.90|1771.41|444.22| +2450819|35617|2450873|1230|177318|7006|38294|77913|1062470|2558|21046|1|91|14|2|12902|110|255|83|70.82|184.13|60.76|10239.71|5043.08|5878.06|15282.79|403.44|0.00|458.16|5043.08|5446.52|5501.24|5904.68|-834.98| +2450819|28983|2450835|71876|829928|4780|16032|71876|829928|4780|16032|2|12|2|1|3169|179|256|67|8.14|21.57|21.13|29.48|1415.71|545.38|1445.19|28.31|0.00|433.49|1415.71|1444.02|1849.20|1877.51|870.33| +2450819|28983|2450884|71876|829928|4780|16032|71876|829928|4780|16032|2|91|19|2|8590|88|256|20|93.15|201.20|24.14|3541.20|482.80|1863.00|4024.00|0.00|0.00|1569.20|482.80|482.80|2052.00|2052.00|-1380.20| +2450819|28983|2450860|71876|829928|4780|16032|71876|829928|4780|16032|2|93|13|2|4964|54|256|20|37.43|57.26|16.60|813.20|332.00|748.60|1145.20|0.00|0.00|229.00|332.00|332.00|561.00|561.00|-416.60| +2450819|28983|2450867|71876|829928|4780|16032|71876|829928|4780|16032|2|59|3|5|14926|155|256|42|45.41|82.64|8.26|3123.96|346.92|1907.22|3470.88|24.28|0.00|485.52|346.92|371.20|832.44|856.72|-1560.30| +2450819|28983|2450831|71876|829928|4780|16032|71876|829928|4780|16032|2|65|18|4|2774|226|256|40|28.24|40.10|30.47|385.20|1218.80|1129.60|1604.00|24.37|0.00|80.00|1218.80|1243.17|1298.80|1323.17|89.20| +2450819|28983|2450884|71876|829928|4780|16032|71876|829928|4780|16032|2|19|19|3|5449|233|256|59|45.50|116.02|90.49|1506.27|5338.91|2684.50|6845.18|106.77|0.00|2326.96|5338.91|5445.68|7665.87|7772.64|2654.41| +2450819|28983|2450876|71876|829928|4780|16032|71876|829928|4780|16032|2|18|12|3|12709|52|256|91|28.00|63.28|8.85|4953.13|805.35|2548.00|5758.48|7.89|16.10|2533.44|789.25|797.14|3322.69|3330.58|-1758.75| +2450819|28983|2450897|71876|829928|4780|16032|71876|829928|4780|16032|2|106|15|1|15127|250|256|54|73.01|81.77|60.50|1148.58|3267.00|3942.54|4415.58|32.67|0.00|353.16|3267.00|3299.67|3620.16|3652.83|-675.54| +2450819|28983|2450835|71876|829928|4780|16032|71876|829928|4780|16032|2|18|4|2|1081|186|256|7|50.40|126.00|64.26|432.18|449.82|352.80|882.00|8.99|0.00|343.98|449.82|458.81|793.80|802.79|97.02| +2450819|28983|2450835|71876|829928|4780|16032|71876|829928|4780|16032|2|40|6|5|1412|158|256|31|59.02|176.46|65.29|3446.27|2023.99|1829.62|5470.26|94.92|667.91|765.70|1356.08|1451.00|2121.78|2216.70|-473.54| +2450819|28983|2450840|71876|829928|4780|16032|71876|829928|4780|16032|2|51|14|5|5953|9|256|67|72.81|125.23|117.71|503.84|7886.57|4878.27|8390.41|473.19|0.00|1593.93|7886.57|8359.76|9480.50|9953.69|3008.30| +2450820|30549|2450883|77247|1260786|2278|29017|77247|1260786|2278|29017|4|22|14|1|13478|2|257|86|14.86|27.04|8.11|1627.98|697.46|1277.96|2325.44|0.00|0.00|348.30|697.46|697.46|1045.76|1045.76|-580.50| +2450820|30549|2450871|77247|1260786|2278|29017|77247|1260786|2278|29017|4|81|18|4|16639|151|257|59|99.02|182.19|103.84|4622.65|6126.56|5842.18|10749.21|490.12|0.00|214.76|6126.56|6616.68|6341.32|6831.44|284.38| +2450820|30549|2450894|77247|1260786|2278|29017|77247|1260786|2278|29017|4|27|16|2|12328|201|257|84|11.28|15.45|6.95|714.00|583.80|947.52|1297.80|23.35|0.00|622.44|583.80|607.15|1206.24|1229.59|-363.72| +2450820|30549|2450831|77247|1260786|2278|29017|77247|1260786|2278|29017|4|99|18|4|14366|124|257|21|94.19|234.53|72.70|3398.43|1526.70|1977.99|4925.13|91.60|0.00|492.45|1526.70|1618.30|2019.15|2110.75|-451.29| +2450820|30549|2450884|77247|1260786|2278|29017|77247|1260786|2278|29017|4|41|16|4|1357|293|257|30|75.18|161.63|53.33|3249.00|1599.90|2255.40|4848.90|47.99|0.00|969.60|1599.90|1647.89|2569.50|2617.49|-655.50| +2450820|30549|2450886|77247|1260786|2278|29017|77247|1260786|2278|29017|4|104|4|5|15662|179|257|2|73.86|199.42|123.64|151.56|247.28|147.72|398.84|0.00|0.00|83.74|247.28|247.28|331.02|331.02|99.56| +2450820|30549|2450833|77247|1260786|2278|29017|77247|1260786|2278|29017|4|69|4|5|10174|267|257|20|21.62|40.21|37.39|56.40|747.80|432.40|804.20|0.00|0.00|305.40|747.80|747.80|1053.20|1053.20|315.40| +2450820|30549|2450871|77247|1260786|2278|29017|77247|1260786|2278|29017|4|84|9|4|10304|237|257|48|73.63|127.37|3.82|5930.40|183.36|3534.24|6113.76|7.33|0.00|855.84|183.36|190.69|1039.20|1046.53|-3350.88| +2450820|30549|2450854|77247|1260786|2278|29017|77247|1260786|2278|29017|4|94|8|4|5827|25|257|15|30.49|45.43|4.08|620.25|61.20|457.35|681.45|5.50|0.00|156.60|61.20|66.70|217.80|223.30|-396.15| +2450820|41498|2450882|81551|500953|3555|42557|81551|500953|3555|42557|4|90|3|1|14266|284|258|48|10.55|19.72|3.54|776.64|169.92|506.40|946.56|6.79|0.00|151.20|169.92|176.71|321.12|327.91|-336.48| +2450820|41498|2450888|81551|500953|3555|42557|81551|500953|3555|42557|4|3|20|1|6700|226|258|49|54.21|103.54|40.38|3094.84|1978.62|2656.29|5073.46|78.15|415.51|1623.37|1563.11|1641.26|3186.48|3264.63|-1093.18| +2450820|41498|2450835|81551|500953|3555|42557|81551|500953|3555|42557|4|1|16|1|6604|53|258|76|89.90|233.74|14.02|16698.72|1065.52|6832.40|17764.24|21.31|0.00|709.84|1065.52|1086.83|1775.36|1796.67|-5766.88| +2450820|41498|2450888|81551|500953|3555|42557|81551|500953|3555|42557|4|10|14|3|12025|208|258|53|5.89|13.60|4.76|468.52|252.28|312.17|720.80|0.00|0.00|86.39|252.28|252.28|338.67|338.67|-59.89| +2450820|41498|2450829|81551|500953|3555|42557|81551|500953|3555|42557|4|71|8|2|5749|274|258|24|75.97|149.66|92.78|1365.12|2226.72|1823.28|3591.84|200.40|0.00|538.56|2226.72|2427.12|2765.28|2965.68|403.44| +2450820|41498|2450867|81551|500953|3555|42557|81551|500953|3555|42557|4|53|2|4|13117|196|258|86|36.63|81.68|57.99|2037.34|4987.14|3150.18|7024.48|299.22|0.00|2668.58|4987.14|5286.36|7655.72|7954.94|1836.96| +2450820|41498|2450870|81551|500953|3555|42557|81551|500953|3555|42557|4|108|6|2|5860|216|258|6|66.15|163.39|52.28|666.66|313.68|396.90|980.34|3.13|0.00|235.26|313.68|316.81|548.94|552.07|-83.22| +2450820|41498|2450869|81551|500953|3555|42557|81551|500953|3555|42557|4|37|20|5|3490|200|258|70|31.71|94.17|67.80|1845.90|4746.00|2219.70|6591.90|94.92|0.00|263.20|4746.00|4840.92|5009.20|5104.12|2526.30| +2450820|41498|2450900|81551|500953|3555|42557|81551|500953|3555|42557|4|37|3|1|8017|127|258|66|72.18|207.15|49.71|10391.04|3280.86|4763.88|13671.90|262.46|0.00|819.72|3280.86|3543.32|4100.58|4363.04|-1483.02| +2450820|41498|2450870|81551|500953|3555|42557|81551|500953|3555|42557|4|42|5|1|3242|85|258|66|94.64|117.35|69.23|3175.92|4569.18|6246.24|7745.10|121.99|502.60|696.96|4066.58|4188.57|4763.54|4885.53|-2179.66| +2450820|41498|2450830|81551|500953|3555|42557|81551|500953|3555|42557|4|14|5|5|2869|22|258|95|8.42|20.96|7.96|1235.00|756.20|799.90|1991.20|2.72|665.45|736.25|90.75|93.47|827.00|829.72|-709.15| +2450820|46939|2450838|95760|1203899|2438|36930|95760|1203899|2438|36930|4|8|4|1|9841|223|259|66|38.15|92.70|87.13|367.62|5750.58|2517.90|6118.20|172.51|0.00|2814.24|5750.58|5923.09|8564.82|8737.33|3232.68| +2450820|46939|2450891|95760|1203899|2438|36930|95760|1203899|2438|36930|4|76|3|4|17203|36|259|29|17.72|33.84|9.13|716.59|264.77|513.88|981.36|7.94|0.00|205.90|264.77|272.71|470.67|478.61|-249.11| +2450820|46939|2450896|95760|1203899|2438|36930|95760|1203899|2438|36930|4|84|16|5|11068|300|259|88|20.34|61.02|35.39|2255.44|3114.32|1789.92|5369.76|155.71|0.00|483.12|3114.32|3270.03|3597.44|3753.15|1324.40| +2450820|46939|2450904|95760|1203899|2438|36930|95760|1203899|2438|36930|4|23|19|4|17821|219|259|99|77.53|217.85|71.89|14450.04|7117.11|7675.47|21567.15|498.19|0.00|8841.69|7117.11|7615.30|15958.80|16456.99|-558.36| +2450820|46939|2450888|95760|1203899|2438|36930|95760|1203899|2438|36930|4|101|12|2|4684|40|259|77|67.59|156.13|62.45|7213.36|4808.65|5204.43|12022.01|96.17|0.00|3485.79|4808.65|4904.82|8294.44|8390.61|-395.78| +2450820|46939|2450857|95760|1203899|2438|36930|95760|1203899|2438|36930|4|47|7|5|12805|148|259|55|50.79|89.89|82.69|396.00|4547.95|2793.45|4943.95|50.93|2001.09|48.95|2546.86|2597.79|2595.81|2646.74|-246.59| +2450820|46939|2450890|95760|1203899|2438|36930|95760|1203899|2438|36930|4|1|16|5|5312|141|259|37|1.03|1.58|0.90|25.16|33.30|38.11|58.46|0.28|28.63|9.62|4.67|4.95|14.29|14.57|-33.44| +2450820|85589|2450835|97602|1596919|1389|42306|97602|1596919|1389|42306|1|99|4|3|17389|275|260|66|69.88|76.16|65.49|704.22|4322.34|4612.08|5026.56|389.01|0.00|1960.20|4322.34|4711.35|6282.54|6671.55|-289.74| +2450820|85589|2450834|97602|1596919|1389|42306|97602|1596919|1389|42306|1|29|2|5|1490|112|260|62|62.17|78.33|32.11|2865.64|1990.82|3854.54|4856.46|59.72|0.00|1359.66|1990.82|2050.54|3350.48|3410.20|-1863.72| +2450820|85589|2450827|97602|1596919|1389|42306|97602|1596919|1389|42306|1|62|4|1|3410|123|260|81|66.97|123.89|38.40|6924.69|3110.40|5424.57|10035.09|155.52|0.00|3913.11|3110.40|3265.92|7023.51|7179.03|-2314.17| +2450820|85589|2450884|97602|1596919|1389|42306|97602|1596919|1389|42306|1|57|5|2|3520|150|260|8|5.89|12.89|1.93|87.68|15.44|47.12|103.12|1.38|0.00|10.24|15.44|16.82|25.68|27.06|-31.68| +2450820|85589|2450838|97602|1596919|1389|42306|97602|1596919|1389|42306|1|79|7|4|2527|270|260|16|75.02|190.55|146.72|701.28|2347.52|1200.32|3048.80|52.11|610.35|914.56|1737.17|1789.28|2651.73|2703.84|536.85| +2450820|85589|2450870|97602|1596919|1389|42306|97602|1596919|1389|42306|1|83|7|2|6346|264|260|13|96.42|140.77|63.34|1006.59|823.42|1253.46|1830.01|41.17|0.00|695.37|823.42|864.59|1518.79|1559.96|-430.04| +2450820|85589|2450841|97602|1596919|1389|42306|97602|1596919|1389|42306|1|18|8|2|3092|122|260|94|41.32|98.34|4.91|8782.42|461.54|3884.08|9243.96|0.00|0.00|2218.40|461.54|461.54|2679.94|2679.94|-3422.54| +2450820|85589|2450889|97602|1596919|1389|42306|97602|1596919|1389|42306|1|68|1|4|6268|10|260|27|51.82|56.48|27.11|792.99|731.97|1399.14|1524.96|65.87|0.00|259.20|731.97|797.84|991.17|1057.04|-667.17| +2450820|85589|2450836|97602|1596919|1389|42306|97602|1596919|1389|42306|1|19|9|2|3382|150|260|23|96.44|273.88|219.10|1259.94|5039.30|2218.12|6299.24|43.33|705.50|818.80|4333.80|4377.13|5152.60|5195.93|2115.68| +2450820|85589|2450853|97602|1596919|1389|42306|97602|1596919|1389|42306|1|40|18|2|15931|50|260|52|86.35|172.70|119.16|2784.08|6196.32|4490.20|8980.40|247.85|0.00|1975.48|6196.32|6444.17|8171.80|8419.65|1706.12| +2450820|85589|2450864|97602|1596919|1389|42306|97602|1596919|1389|42306|1|73|3|4|1066|89|260|41|94.65|190.24|97.02|3822.02|3977.82|3880.65|7799.84|318.22|0.00|311.60|3977.82|4296.04|4289.42|4607.64|97.17| +2450820|85589|2450843|97602|1596919|1389|42306|97602|1596919|1389|42306|1|22|7|5|15088|181|260|18|9.00|22.77|20.94|32.94|376.92|162.00|409.86|3.76|0.00|122.94|376.92|380.68|499.86|503.62|214.92| +2450820|85589|2450859|97602|1596919|1389|42306|97602|1596919|1389|42306|1|41|3|5|11864|72|260|9|48.16|120.88|120.88|0.00|1087.92|433.44|1087.92|87.03|0.00|337.23|1087.92|1174.95|1425.15|1512.18|654.48| +2450820|60271|2450883|34782|603476|170|22464|34782|603476|170|22464|4|101|4|1|4054|38|261|78|69.03|135.29|54.11|6332.04|4220.58|5384.34|10552.62|253.23|0.00|4642.56|4220.58|4473.81|8863.14|9116.37|-1163.76| +2450820|60271|2450863|34782|603476|170|22464|34782|603476|170|22464|4|15|2|2|6868|142|261|78|94.93|213.59|76.89|10662.60|5997.42|7404.54|16660.02|80.96|5097.80|5497.44|899.62|980.58|6397.06|6478.02|-6504.92| +2450820|60271|2450836|34782|603476|170|22464|34782|603476|170|22464|4|77|18|5|10252|91|261|94|57.82|75.16|72.90|212.44|6852.60|5435.08|7065.04|68.52|0.00|3532.52|6852.60|6921.12|10385.12|10453.64|1417.52| +2450820|60271|2450849|34782|603476|170|22464|34782|603476|170|22464|4|77|5|4|5410|205|261|27|32.17|76.24|72.42|103.14|1955.34|868.59|2058.48|21.90|1642.48|329.13|312.86|334.76|641.99|663.89|-555.73| +2450820|60271|2450882|34782|603476|170|22464|34782|603476|170|22464|4|58|3|5|11812|118|261|37|48.34|100.06|47.02|1962.48|1739.74|1788.58|3702.22|17.39|0.00|1369.74|1739.74|1757.13|3109.48|3126.87|-48.84| +2450820|60271|2450867|34782|603476|170|22464|34782|603476|170|22464|4|35|16|3|3128|165|261|62|19.41|41.34|16.94|1512.80|1050.28|1203.42|2563.08|73.51|0.00|486.70|1050.28|1123.79|1536.98|1610.49|-153.14| +2450820|60271|2450867|34782|603476|170|22464|34782|603476|170|22464|4|88|1|5|10936|52|261|13|96.04|230.49|53.01|2307.24|689.13|1248.52|2996.37|13.78|0.00|149.76|689.13|702.91|838.89|852.67|-559.39| +2450820|60271|2450831|34782|603476|170|22464|34782|603476|170|22464|4|68|11|2|15040|289|261|72|82.88|185.65|94.68|6549.84|6816.96|5967.36|13366.80|0.00|0.00|3474.72|6816.96|6816.96|10291.68|10291.68|849.60| +2450820|60271|2450872|34782|603476|170|22464|34782|603476|170|22464|4|26|18|4|10574|199|261|43|10.56|26.40|25.08|56.76|1078.44|454.08|1135.20|0.00|0.00|499.23|1078.44|1078.44|1577.67|1577.67|624.36| +2450820|60271|2450835|34782|603476|170|22464|34782|603476|170|22464|4|49|8|3|5104|253|261|45|21.12|56.39|0.00|2537.55|0.00|950.40|2537.55|0.00|0.00|684.90|0.00|0.00|684.90|684.90|-950.40| +2450820|70313|2450859|68197|1482037|4677|47717|68197|1482037|4677|47717|1|14|13|5|4870|54|262|31|95.39|95.39|57.23|1182.96|1774.13|2957.09|2957.09|0.00|0.00|236.53|1774.13|1774.13|2010.66|2010.66|-1182.96| +2450820|70313|2450850|68197|1482037|4677|47717|68197|1482037|4677|47717|1|76|8|4|6142|216|262|5|16.53|24.29|6.31|89.90|31.55|82.65|121.45|0.63|0.00|19.40|31.55|32.18|50.95|51.58|-51.10| +2450820|70313|2450873|68197|1482037|4677|47717|68197|1482037|4677|47717|1|8|20|2|5056|213|262|99|32.98|77.83|35.02|4238.19|3466.98|3265.02|7705.17|104.00|0.00|1155.33|3466.98|3570.98|4622.31|4726.31|201.96| +2450820|70313|2450888|68197|1482037|4677|47717|68197|1482037|4677|47717|1|26|2|1|4750|198|262|74|65.38|70.61|60.01|784.40|4440.74|4838.12|5225.14|222.03|0.00|1096.68|4440.74|4662.77|5537.42|5759.45|-397.38| +2450820|70313|2450827|68197|1482037|4677|47717|68197|1482037|4677|47717|1|98|2|1|17882|169|262|90|39.94|68.29|34.82|3012.30|3133.80|3594.60|6146.10|188.02|0.00|1658.70|3133.80|3321.82|4792.50|4980.52|-460.80| +2450820|70313|2450889|68197|1482037|4677|47717|68197|1482037|4677|47717|1|70|14|3|16768|218|262|10|38.55|90.97|79.14|118.30|791.40|385.50|909.70|7.91|0.00|354.70|791.40|799.31|1146.10|1154.01|405.90| +2450820|63961|2450859|57249|1091835|21|12607|57249|1091835|21|12607|2|36|11|5|712|73|263|18|8.10|11.50|10.12|24.84|182.16|145.80|207.00|1.82|0.00|18.54|182.16|183.98|200.70|202.52|36.36| +2450820|63961|2450857|57249|1091835|21|12607|57249|1091835|21|12607|2|33|18|2|7981|266|263|100|70.45|120.46|45.77|7469.00|4577.00|7045.00|12046.00|183.08|0.00|3131.00|4577.00|4760.08|7708.00|7891.08|-2468.00| +2450820|63961|2450865|57249|1091835|21|12607|57249|1091835|21|12607|2|78|18|1|5740|139|263|79|24.05|29.82|16.10|1083.88|1271.90|1899.95|2355.78|5.34|1004.80|494.54|267.10|272.44|761.64|766.98|-1632.85| +2450820|63961|2450840|57249|1091835|21|12607|57249|1091835|21|12607|2|36|13|2|16921|7|263|32|73.17|73.90|46.55|875.20|1489.60|2341.44|2364.80|104.27|0.00|1135.04|1489.60|1593.87|2624.64|2728.91|-851.84| +2450820|63961|2450890|57249|1091835|21|12607|57249|1091835|21|12607|2|36|18|5|13634|31|263|36|93.09|261.58|5.23|9228.60|188.28|3351.24|9416.88|7.62|103.55|3954.96|84.73|92.35|4039.69|4047.31|-3266.51| +2450820|63961|2450857|57249|1091835|21|12607|57249|1091835|21|12607|2|81|7|3|16414|90|263|25|48.90|83.61|4.18|1985.75|104.50|1222.50|2090.25|0.83|83.60|585.25|20.90|21.73|606.15|606.98|-1201.60| +2450820|63961|2450839|57249|1091835|21|12607|57249|1091835|21|12607|2|107|3|4|5702|38|263|95|6.14|15.35|0.00|1458.25|0.00|583.30|1458.25|0.00|0.00|524.40|0.00|0.00|524.40|524.40|-583.30| +2450820|63961|2450880|57249|1091835|21|12607|57249|1091835|21|12607|2|65|11|3|10522|244|263|94|33.79|65.89|55.34|991.70|5201.96|3176.26|6193.66|208.07|0.00|2600.98|5201.96|5410.03|7802.94|8011.01|2025.70| +2450820|63961|2450841|57249|1091835|21|12607|57249|1091835|21|12607|2|11|6|2|8584|267|263|54|34.73|86.47|26.80|3222.18|1447.20|1875.42|4669.38|14.47|0.00|186.30|1447.20|1461.67|1633.50|1647.97|-428.22| +2450820|63961|2450854|57249|1091835|21|12607|57249|1091835|21|12607|2|108|16|3|9460|190|263|24|61.11|130.77|113.76|408.24|2730.24|1466.64|3138.48|81.90|0.00|721.68|2730.24|2812.14|3451.92|3533.82|1263.60| +2450820|63961|2450851|57249|1091835|21|12607|57249|1091835|21|12607|2|47|13|4|4747|191|263|78|15.59|24.47|14.43|783.12|1125.54|1216.02|1908.66|45.02|0.00|648.18|1125.54|1170.56|1773.72|1818.74|-90.48| +2450820|29863|2450837|4234|800665|649|20680|4234|800665|649|20680|4|61|9|3|17936|75|264|86|32.09|70.27|9.13|5258.04|785.18|2759.74|6043.22|15.70|0.00|1631.42|785.18|800.88|2416.60|2432.30|-1974.56| +2450820|29863|2450838|4234|800665|649|20680|4234|800665|649|20680|4|74|9|5|14299|235|264|53|87.45|146.91|20.56|6696.55|1089.68|4634.85|7786.23|87.17|0.00|2491.53|1089.68|1176.85|3581.21|3668.38|-3545.17| +2450820|29863|2450904|4234|800665|649|20680|4234|800665|649|20680|4|80|1|5|16252|157|264|34|83.13|160.44|112.30|1636.76|3818.20|2826.42|5454.96|267.27|0.00|1363.74|3818.20|4085.47|5181.94|5449.21|991.78| +2450820|29863|2450884|4234|800665|649|20680|4234|800665|649|20680|4|49|11|4|13621|77|264|69|67.55|137.12|126.15|756.93|8704.35|4660.95|9461.28|435.21|0.00|378.12|8704.35|9139.56|9082.47|9517.68|4043.40| +2450820|29863|2450868|4234|800665|649|20680|4234|800665|649|20680|4|87|15|2|9698|83|264|99|99.57|219.05|208.09|1085.04|20600.91|9857.43|21685.95|519.14|13184.58|9324.81|7416.33|7935.47|16741.14|17260.28|-2441.10| +2450820|29863|2450908|4234|800665|649|20680|4234|800665|649|20680|4|54|13|3|3358|27|264|16|19.19|37.80|24.57|211.68|393.12|307.04|604.80|31.44|0.00|72.48|393.12|424.56|465.60|497.04|86.08| +2450820|29863|2450825|4234|800665|649|20680|4234|800665|649|20680|4|60|2|1|10456|219|264|6|95.90|187.96|116.53|428.58|699.18|575.40|1127.76|11.11|328.61|248.10|370.57|381.68|618.67|629.78|-204.83| +2450820|29863|2450834|4234|800665|649|20680|4234|800665|649|20680|4|39|10|1|15938|201|264|50|11.78|24.97|16.23|437.00|811.50|589.00|1248.50|32.46|0.00|624.00|811.50|843.96|1435.50|1467.96|222.50| +2450820|29863|2450869|4234|800665|649|20680|4234|800665|649|20680|4|75|16|5|4424|205|264|55|14.96|39.64|34.48|283.80|1896.40|822.80|2180.20|37.92|0.00|21.45|1896.40|1934.32|1917.85|1955.77|1073.60| +2450820|29863|2450910|4234|800665|649|20680|4234|800665|649|20680|4|14|5|5|12416|267|264|28|84.01|199.10|49.77|4181.24|1393.56|2352.28|5574.80|41.80|0.00|2285.64|1393.56|1435.36|3679.20|3721.00|-958.72| +2450820|29863|2450881|4234|800665|649|20680|4234|800665|649|20680|4|29|10|2|6490|133|264|58|19.78|26.30|4.20|1281.80|243.60|1147.24|1525.40|17.05|0.00|411.80|243.60|260.65|655.40|672.45|-903.64| +2450820|29863|2450854|4234|800665|649|20680|4234|800665|649|20680|4|32|19|5|17452|158|264|6|83.16|204.57|128.87|454.20|773.22|498.96|1227.42|38.66|0.00|257.70|773.22|811.88|1030.92|1069.58|274.26| +2450820|29863|2450887|4234|800665|649|20680|4234|800665|649|20680|4|89|5|1|4882|220|264|40|10.17|29.59|25.44|166.00|1017.60|406.80|1183.60|20.35|0.00|496.80|1017.60|1037.95|1514.40|1534.75|610.80| +2450820|29863|2450853|4234|800665|649|20680|4234|800665|649|20680|4|100|13|3|8275|173|264|12|40.54|95.26|31.43|765.96|377.16|486.48|1143.12|22.62|0.00|240.00|377.16|399.78|617.16|639.78|-109.32| +2450820|38288|2450822|67220|302457|94|38978|50804|68299|336|40420|1|14|14|5|17084|240|265|55|10.63|12.11|3.75|459.80|206.25|584.65|666.05|10.31|0.00|146.30|206.25|216.56|352.55|362.86|-378.40| +2450820|38288|2450845|67220|302457|94|38978|50804|68299|336|40420|1|63|7|5|17474|138|265|80|23.61|60.20|18.06|3371.20|1444.80|1888.80|4816.00|130.03|0.00|770.40|1444.80|1574.83|2215.20|2345.23|-444.00| +2450820|38288|2450822|67220|302457|94|38978|50804|68299|336|40420|1|75|16|1|1147|177|265|85|54.26|118.82|114.06|404.60|9695.10|4612.10|10099.70|431.43|1066.46|4140.35|8628.64|9060.07|12768.99|13200.42|4016.54| +2450820|38288|2450881|67220|302457|94|38978|50804|68299|336|40420|1|80|12|4|8732|13|265|4|6.08|17.93|5.91|48.08|23.64|24.32|71.72|1.41|0.00|15.76|23.64|25.05|39.40|40.81|-0.68| +2450820|38288|2450848|67220|302457|94|38978|50804|68299|336|40420|1|53|6|5|4852|106|265|83|41.90|62.43|16.85|3783.14|1398.55|3477.70|5181.69|41.95|0.00|465.63|1398.55|1440.50|1864.18|1906.13|-2079.15| +2450820|38288|2450832|67220|302457|94|38978|50804|68299|336|40420|1|8|5|2|2708|43|265|61|72.01|213.14|136.40|4681.14|8320.40|4392.61|13001.54|83.20|0.00|5460.11|8320.40|8403.60|13780.51|13863.71|3927.79| +2450820|38288|2450865|67220|302457|94|38978|50804|68299|336|40420|1|44|17|5|8540|21|265|60|84.14|189.31|87.08|6133.80|5224.80|5048.40|11358.60|261.24|0.00|1249.20|5224.80|5486.04|6474.00|6735.24|176.40| +2450820|38288|2450876|67220|302457|94|38978|50804|68299|336|40420|1|64|17|5|6740|16|265|61|17.27|19.86|2.78|1041.88|169.58|1053.47|1211.46|8.47|0.00|157.38|169.58|178.05|326.96|335.43|-883.89| +2450820|38288|2450882|67220|302457|94|38978|50804|68299|336|40420|1|9|6|2|14962|186|265|69|64.48|165.06|161.75|228.39|11160.75|4449.12|11389.14|111.60|0.00|2163.84|11160.75|11272.35|13324.59|13436.19|6711.63| +2450820|38288|2450901|67220|302457|94|38978|50804|68299|336|40420|1|45|14|4|598|251|265|83|5.87|8.98|6.37|216.63|528.71|487.21|745.34|47.58|0.00|342.79|528.71|576.29|871.50|919.08|41.50| +2450820|38288|2450837|67220|302457|94|38978|50804|68299|336|40420|1|98|7|4|15910|153|265|26|19.99|47.97|25.42|586.30|660.92|519.74|1247.22|13.21|0.00|511.16|660.92|674.13|1172.08|1185.29|141.18| +2450820|38474|2450904|79499|650598|897|25596|79499|650598|897|25596|4|68|1|4|1585|113|266|4|90.34|161.70|42.04|478.64|168.16|361.36|646.80|6.72|0.00|103.48|168.16|174.88|271.64|278.36|-193.20| +2450820|38474|2450891|79499|650598|897|25596|79499|650598|897|25596|4|101|19|2|12700|196|266|41|50.47|135.25|1.35|5489.90|55.35|2069.27|5545.25|0.83|27.67|1663.37|27.68|28.51|1691.05|1691.88|-2041.59| +2450820|38474|2450833|79499|650598|897|25596|79499|650598|897|25596|4|5|8|3|5833|16|266|44|52.04|77.53|20.93|2490.40|920.92|2289.76|3411.32|27.62|0.00|136.40|920.92|948.54|1057.32|1084.94|-1368.84| +2450820|38474|2450856|79499|650598|897|25596|79499|650598|897|25596|4|1|6|3|2875|71|266|93|71.39|131.35|53.85|7207.50|5008.05|6639.27|12215.55|400.64|0.00|854.67|5008.05|5408.69|5862.72|6263.36|-1631.22| +2450820|38474|2450831|79499|650598|897|25596|79499|650598|897|25596|4|99|17|2|16543|224|266|17|20.84|62.31|61.06|21.25|1038.02|354.28|1059.27|31.14|0.00|158.78|1038.02|1069.16|1196.80|1227.94|683.74| +2450820|38474|2450858|79499|650598|897|25596|79499|650598|897|25596|4|8|11|1|12739|24|266|19|9.44|25.29|20.23|96.14|384.37|179.36|480.51|19.64|103.77|211.28|280.60|300.24|491.88|511.52|101.24| +2450820|38474|2450853|79499|650598|897|25596|79499|650598|897|25596|4|7|4|1|1003|100|266|23|56.46|153.00|22.95|2991.15|527.85|1298.58|3519.00|10.55|263.92|1055.70|263.93|274.48|1319.63|1330.18|-1034.65| +2450820|38474|2450878|79499|650598|897|25596|79499|650598|897|25596|4|37|19|3|12154|61|266|41|49.62|64.50|36.12|1163.58|1480.92|2034.42|2644.50|27.84|784.88|264.45|696.04|723.88|960.49|988.33|-1338.38| +2450820|41804|2450904|27686|845864|6089|6667|27686|845864|6089|6667|2|74|17|3|2708|101|267|69|75.92|119.95|98.35|1490.40|6786.15|5238.48|8276.55|542.89|0.00|2151.42|6786.15|7329.04|8937.57|9480.46|1547.67| +2450820|41804|2450888|27686|845864|6089|6667|27686|845864|6089|6667|2|15|14|1|8540|244|267|73|12.00|13.08|4.57|621.23|333.61|876.00|954.84|16.68|0.00|37.96|333.61|350.29|371.57|388.25|-542.39| +2450820|41804|2450825|27686|845864|6089|6667|27686|845864|6089|6667|2|12|16|4|6740|47|267|42|91.93|234.42|131.27|4332.30|5513.34|3861.06|9845.64|0.00|0.00|984.48|5513.34|5513.34|6497.82|6497.82|1652.28| +2450820|41804|2450876|27686|845864|6089|6667|27686|845864|6089|6667|2|27|13|4|14962|186|267|71|88.73|149.95|82.47|4791.08|5855.37|6299.83|10646.45|292.76|0.00|2767.58|5855.37|6148.13|8622.95|8915.71|-444.46| +2450820|46710|2450861|17910|291359|6408|41057|17910|291359|6408|41057|2|23|1|2|12055|274|268|48|6.57|18.39|12.68|274.08|608.64|315.36|882.72|0.00|0.00|202.56|608.64|608.64|811.20|811.20|293.28| +2450820|46710|2450878|17910|291359|6408|41057|17910|291359|6408|41057|2|99|13|2|3614|70|268|39|32.74|41.57|14.13|1070.16|551.07|1276.86|1621.23|24.68|198.38|194.22|352.69|377.37|546.91|571.59|-924.17| +2450820|46710|2450846|17910|291359|6408|41057|17910|291359|6408|41057|2|27|8|5|442|139|268|41|28.82|31.12|10.26|855.26|420.66|1181.62|1275.92|37.85|0.00|407.95|420.66|458.51|828.61|866.46|-760.96| +2450820|46710|2450860|17910|291359|6408|41057|17910|291359|6408|41057|2|40|6|3|17852|253|268|27|87.21|128.19|114.08|380.97|3080.16|2354.67|3461.13|30.80|0.00|1453.41|3080.16|3110.96|4533.57|4564.37|725.49| +2450820|71696|2450894|64300|161275|1097|44519|64300|161275|1097|44519|1|99|5|4|9968|198|269|93|40.63|101.57|80.24|1983.69|7462.32|3778.59|9446.01|149.24|0.00|3777.66|7462.32|7611.56|11239.98|11389.22|3683.73| +2450820|71696|2450878|64300|161275|1097|44519|64300|161275|1097|44519|1|98|20|5|16028|257|269|75|37.32|45.15|31.15|1050.00|2336.25|2799.00|3386.25|0.00|0.00|405.75|2336.25|2336.25|2742.00|2742.00|-462.75| +2450820|71696|2450900|64300|161275|1097|44519|64300|161275|1097|44519|1|24|18|1|13088|262|269|99|9.83|18.87|6.60|1214.73|653.40|973.17|1868.13|52.27|0.00|784.08|653.40|705.67|1437.48|1489.75|-319.77| +2450820|71696|2450823|64300|161275|1097|44519|64300|161275|1097|44519|1|49|14|5|17191|262|269|25|21.41|29.11|7.27|546.00|181.75|535.25|727.75|3.94|125.40|0.00|56.35|60.29|56.35|60.29|-478.90| +2450820|71696|2450895|64300|161275|1097|44519|64300|161275|1097|44519|1|42|4|5|5510|222|269|26|24.15|40.08|1.20|1010.88|31.20|627.90|1042.08|1.87|0.00|135.46|31.20|33.07|166.66|168.53|-596.70| +2450820|71696|2450876|64300|161275|1097|44519|64300|161275|1097|44519|1|25|19|2|7270|220|269|93|21.64|47.60|17.61|2789.07|1637.73|2012.52|4426.80|147.39|0.00|1549.38|1637.73|1785.12|3187.11|3334.50|-374.79| +2450820|71696|2450891|64300|161275|1097|44519|64300|161275|1097|44519|1|42|2|4|9230|255|269|29|80.11|179.44|177.64|52.20|5151.56|2323.19|5203.76|9.78|4172.76|51.91|978.80|988.58|1030.71|1040.49|-1344.39| +2450820|71696|2450869|64300|161275|1097|44519|64300|161275|1097|44519|1|91|1|1|12148|4|269|6|63.83|97.65|24.41|439.44|146.46|382.98|585.90|11.71|0.00|164.04|146.46|158.17|310.50|322.21|-236.52| +2450820|71696|2450876|64300|161275|1097|44519|64300|161275|1097|44519|1|96|16|3|13166|95|269|50|50.73|76.60|19.91|2834.50|995.50|2536.50|3830.00|6.47|348.42|1838.00|647.08|653.55|2485.08|2491.55|-1889.42| +2450820|71696|2450905|64300|161275|1097|44519|64300|161275|1097|44519|1|88|13|4|7549|210|269|11|25.54|36.26|26.46|107.80|291.06|280.94|398.86|14.84|43.65|0.00|247.41|262.25|247.41|262.25|-33.53| +2450820|71696|2450879|64300|161275|1097|44519|64300|161275|1097|44519|1|62|19|4|5485|49|269|72|71.00|105.08|68.30|2648.16|4917.60|5112.00|7565.76|310.79|1032.69|1739.52|3884.91|4195.70|5624.43|5935.22|-1227.09| +2450820|71696|2450865|64300|161275|1097|44519|64300|161275|1097|44519|1|35|9|5|8062|174|269|57|57.21|109.27|91.78|996.93|5231.46|3260.97|6228.39|104.62|0.00|2117.55|5231.46|5336.08|7349.01|7453.63|1970.49| +2450820|71696|2450834|64300|161275|1097|44519|64300|161275|1097|44519|1|69|1|4|4237|22|269|3|35.19|59.82|7.77|156.15|23.31|105.57|179.46|0.00|5.36|80.73|17.95|17.95|98.68|98.68|-87.62| +2450820|71894|2450851|76307|509275|5890|6058|76307|509275|5890|6058|4|2|14|1|13240|38|270|91|25.61|53.78|38.72|1370.46|3523.52|2330.51|4893.98|69.06|70.47|1663.48|3453.05|3522.11|5116.53|5185.59|1122.54| +2450820|71894|2450896|76307|509275|5890|6058|76307|509275|5890|6058|4|87|1|5|5089|53|270|58|28.83|84.76|25.42|3441.72|1474.36|1672.14|4916.08|0.00|191.66|2015.50|1282.70|1282.70|3298.20|3298.20|-389.44| +2450820|71894|2450895|76307|509275|5890|6058|76307|509275|5890|6058|4|100|18|4|9194|131|270|34|18.07|42.64|34.96|261.12|1188.64|614.38|1449.76|0.00|0.00|57.80|1188.64|1188.64|1246.44|1246.44|574.26| +2450820|71894|2450886|76307|509275|5890|6058|76307|509275|5890|6058|4|79|3|2|349|185|270|33|53.25|157.62|141.85|520.41|4681.05|1757.25|5201.46|26.68|2012.85|780.12|2668.20|2694.88|3448.32|3475.00|910.95| +2450820|71894|2450851|76307|509275|5890|6058|76307|509275|5890|6058|4|85|14|4|7246|172|270|34|25.68|33.64|27.24|217.60|926.16|873.12|1143.76|64.83|0.00|205.70|926.16|990.99|1131.86|1196.69|53.04| +2450820|60635|2450896|4231|874318|4852|33739|4231|874318|4852|33739|1|9|5|5|17917|158|271|17|90.85|243.47|177.73|1117.58|3021.41|1544.45|4138.99|151.07|0.00|1779.73|3021.41|3172.48|4801.14|4952.21|1476.96| +2450820|60635|2450825|4231|874318|4852|33739|4231|874318|4852|33739|1|8|3|4|2966|165|271|76|80.19|181.22|67.05|8676.92|5095.80|6094.44|13772.72|254.79|0.00|3717.92|5095.80|5350.59|8813.72|9068.51|-998.64| +2450820|60635|2450904|4231|874318|4852|33739|4231|874318|4852|33739|1|70|10|5|6616|175|271|35|31.49|61.72|11.10|1771.70|388.50|1102.15|2160.20|19.42|0.00|928.55|388.50|407.92|1317.05|1336.47|-713.65| +2450820|60635|2450899|4231|874318|4852|33739|4231|874318|4852|33739|1|50|8|2|16604|260|271|64|49.20|138.25|48.38|5751.68|3096.32|3148.80|8848.00|12.07|1888.75|4335.36|1207.57|1219.64|5542.93|5555.00|-1941.23| +2450820|30703|2450880|31043|416983|2123|11663|31043|416983|2123|11663|1|55|9|3|4732|198|272|55|99.72|116.67|32.66|4620.55|1796.30|5484.60|6416.85|71.85|0.00|1925.00|1796.30|1868.15|3721.30|3793.15|-3688.30| +2450820|30703|2450826|31043|416983|2123|11663|31043|416983|2123|11663|1|20|5|2|9266|251|272|24|51.27|65.11|7.16|1390.80|171.84|1230.48|1562.64|15.46|0.00|593.76|171.84|187.30|765.60|781.06|-1058.64| +2450820|30703|2450894|31043|416983|2123|11663|31043|416983|2123|11663|1|31|18|2|11167|10|272|10|76.69|128.07|103.73|243.40|1037.30|766.90|1280.70|10.37|0.00|153.60|1037.30|1047.67|1190.90|1201.27|270.40| +2450820|30703|2450894|31043|416983|2123|11663|31043|416983|2123|11663|1|70|4|4|11599|296|272|87|38.34|99.30|48.65|4406.55|4232.55|3335.58|8639.10|253.95|0.00|4319.55|4232.55|4486.50|8552.10|8806.05|896.97| +2450820|30703|2450891|31043|416983|2123|11663|31043|416983|2123|11663|1|35|6|5|5905|40|272|11|81.56|133.75|133.75|0.00|1471.25|897.16|1471.25|44.13|0.00|661.98|1471.25|1515.38|2133.23|2177.36|574.09| +2450820|30703|2450852|31043|416983|2123|11663|31043|416983|2123|11663|1|62|6|2|50|237|272|21|61.13|78.85|67.02|248.43|1407.42|1283.73|1655.85|112.59|0.00|629.16|1407.42|1520.01|2036.58|2149.17|123.69| +2450820|30703|2450848|31043|416983|2123|11663|31043|416983|2123|11663|1|100|3|1|11210|117|272|60|94.20|149.77|10.48|8357.40|628.80|5652.00|8986.20|18.86|0.00|4402.80|628.80|647.66|5031.60|5050.46|-5023.20| +2450820|30703|2450840|31043|416983|2123|11663|31043|416983|2123|11663|1|37|13|4|15274|194|272|3|57.18|169.25|140.47|86.34|421.41|171.54|507.75|6.57|257.06|192.93|164.35|170.92|357.28|363.85|-7.19| +2450820|30703|2450836|31043|416983|2123|11663|31043|416983|2123|11663|1|49|17|5|13399|278|272|73|72.63|206.99|68.30|10124.37|4985.90|5301.99|15110.27|448.73|0.00|1208.15|4985.90|5434.63|6194.05|6642.78|-316.09| +2450820|30703|2450884|31043|416983|2123|11663|31043|416983|2123|11663|1|78|14|4|7010|254|272|28|2.86|6.89|3.37|98.56|94.36|80.08|192.92|0.00|46.23|90.44|48.13|48.13|138.57|138.57|-31.95| +2450820|30703|2450838|31043|416983|2123|11663|31043|416983|2123|11663|1|68|2|5|3310|86|272|57|19.46|47.09|47.09|0.00|2684.13|1109.22|2684.13|214.73|0.00|1261.41|2684.13|2898.86|3945.54|4160.27|1574.91| +2450820|30703|2450860|31043|416983|2123|11663|31043|416983|2123|11663|1|104|14|3|8239|119|272|81|29.34|36.96|35.85|89.91|2903.85|2376.54|2993.76|58.07|0.00|688.50|2903.85|2961.92|3592.35|3650.42|527.31| +2450820|54584|2450873|80517|872412|2707|33002|80517|872412|2707|33002|1|98|2|4|9706|229|273|98|28.42|72.75|23.28|4848.06|2281.44|2785.16|7129.50|182.51|0.00|427.28|2281.44|2463.95|2708.72|2891.23|-503.72| +2450820|54584|2450889|80517|872412|2707|33002|80517|872412|2707|33002|1|104|3|3|15722|71|273|87|33.04|37.33|1.11|3151.14|96.57|2874.48|3247.71|2.89|0.00|843.90|96.57|99.46|940.47|943.36|-2777.91| +2450820|54584|2450846|80517|872412|2707|33002|80517|872412|2707|33002|1|33|3|2|6632|246|273|74|28.23|74.24|22.27|3845.78|1647.98|2089.02|5493.76|3.13|1334.86|1482.96|313.12|316.25|1796.08|1799.21|-1775.90| +2450820|54584|2450842|80517|872412|2707|33002|80517|872412|2707|33002|1|8|2|1|2482|152|273|26|65.77|111.80|46.95|1686.10|1220.70|1710.02|2906.80|12.20|0.00|1249.82|1220.70|1232.90|2470.52|2482.72|-489.32| +2450820|54584|2450882|80517|872412|2707|33002|80517|872412|2707|33002|1|80|14|2|10603|185|273|89|46.87|67.02|15.41|4593.29|1371.49|4171.43|5964.78|109.71|0.00|834.82|1371.49|1481.20|2206.31|2316.02|-2799.94| +2450820|54584|2450878|80517|872412|2707|33002|80517|872412|2707|33002|1|71|14|5|16105|175|273|63|75.64|76.39|10.69|4139.10|673.47|4765.32|4812.57|47.14|0.00|1154.79|673.47|720.61|1828.26|1875.40|-4091.85| +2450820|54584|2450842|80517|872412|2707|33002|80517|872412|2707|33002|1|67|20|3|9880|234|273|32|18.25|32.85|25.29|241.92|809.28|584.00|1051.20|48.55|0.00|315.20|809.28|857.83|1124.48|1173.03|225.28| +2450820|54584|2450881|80517|872412|2707|33002|80517|872412|2707|33002|1|88|14|2|1099|153|273|53|99.04|157.47|119.67|2003.40|6342.51|5249.12|8345.91|380.55|0.00|2419.98|6342.51|6723.06|8762.49|9143.04|1093.39| +2450820|54584|2450852|80517|872412|2707|33002|80517|872412|2707|33002|1|5|5|1|14774|56|273|54|56.99|88.90|21.33|3648.78|1151.82|3077.46|4800.60|19.35|668.05|1824.12|483.77|503.12|2307.89|2327.24|-2593.69| +2450820|54584|2450830|80517|872412|2707|33002|80517|872412|2707|33002|1|64|13|5|5906|51|273|91|67.76|133.48|17.35|10567.83|1578.85|6166.16|12146.68|17.68|1357.81|2064.79|221.04|238.72|2285.83|2303.51|-5945.12| +2450820|54584|2450895|80517|872412|2707|33002|80517|872412|2707|33002|1|55|8|5|2156|148|273|31|83.32|204.96|122.97|2541.69|3812.07|2582.92|6353.76|190.60|0.00|889.39|3812.07|4002.67|4701.46|4892.06|1229.15| +2450820|54584|2450895|80517|872412|2707|33002|80517|872412|2707|33002|1|84|5|1|6326|89|273|41|26.21|63.95|62.67|52.48|2569.47|1074.61|2621.95|231.25|0.00|1258.29|2569.47|2800.72|3827.76|4059.01|1494.86| +2450820|54584|2450839|80517|872412|2707|33002|80517|872412|2707|33002|1|20|6|3|13526|277|273|50|62.03|115.37|36.91|3923.00|1845.50|3101.50|5768.50|55.36|0.00|461.00|1845.50|1900.86|2306.50|2361.86|-1256.00| +2450820|54584|2450840|80517|872412|2707|33002|80517|872412|2707|33002|1|36|20|3|15410|96|273|92|98.82|275.70|129.57|13443.96|11920.44|9091.44|25364.40|667.54|2384.08|2028.60|9536.36|10203.90|11564.96|12232.50|444.92| +2450820|80814|2450872|86421|1082121|574|13874|86421|1082121|574|13874|4|61|10|1|16837|121|274|22|11.79|30.41|23.41|154.00|515.02|259.38|669.02|0.00|118.45|307.56|396.57|396.57|704.13|704.13|137.19| +2450820|80814|2450847|86421|1082121|574|13874|86421|1082121|574|13874|4|86|7|1|10780|84|274|27|55.69|62.92|8.80|1461.24|237.60|1503.63|1698.84|21.38|0.00|560.52|237.60|258.98|798.12|819.50|-1266.03| +2450820|80814|2450833|86421|1082121|574|13874|86421|1082121|574|13874|4|71|19|3|3679|263|274|87|73.25|190.45|179.02|994.41|15574.74|6372.75|16569.15|630.77|2959.20|4805.01|12615.54|13246.31|17420.55|18051.32|6242.79| +2450820|80814|2450839|86421|1082121|574|13874|86421|1082121|574|13874|4|14|4|2|12542|13|274|58|70.15|159.94|155.14|278.40|8998.12|4068.70|9276.52|449.90|0.00|3431.86|8998.12|9448.02|12429.98|12879.88|4929.42| +2450820|80814|2450883|86421|1082121|574|13874|86421|1082121|574|13874|4|32|4|2|12286|131|274|58|30.43|47.77|30.09|1025.44|1745.22|1764.94|2770.66|34.90|0.00|526.06|1745.22|1780.12|2271.28|2306.18|-19.72| +2450820|80814|2450883|86421|1082121|574|13874|86421|1082121|574|13874|4|7|13|5|13753|152|274|91|9.24|16.63|5.15|1044.68|468.65|840.84|1513.33|14.05|0.00|332.15|468.65|482.70|800.80|814.85|-372.19| +2450820|80814|2450878|86421|1082121|574|13874|86421|1082121|574|13874|4|34|7|2|17857|42|274|50|96.01|238.10|173.81|3214.50|8690.50|4800.50|11905.00|434.52|0.00|5238.00|8690.50|9125.02|13928.50|14363.02|3890.00| +2450820|80814|2450883|86421|1082121|574|13874|86421|1082121|574|13874|4|51|19|1|16843|217|274|96|33.29|68.91|6.20|6020.16|595.20|3195.84|6615.36|53.56|0.00|2513.28|595.20|648.76|3108.48|3162.04|-2600.64| +2450820|80814|2450827|86421|1082121|574|13874|86421|1082121|574|13874|4|74|16|4|5504|283|274|18|33.53|77.45|20.13|1031.76|362.34|603.54|1394.10|0.87|347.84|334.44|14.50|15.37|348.94|349.81|-589.04| +2450820|80814|2450853|86421|1082121|574|13874|86421|1082121|574|13874|4|34|1|1|17900|208|274|62|33.08|85.34|85.34|0.00|5291.08|2050.96|5291.08|264.55|0.00|793.60|5291.08|5555.63|6084.68|6349.23|3240.12| +2450820|42556|2450904|37025|1642777|2129|6234|37025|1642777|2129|6234|4|94|17|2|4447|262|275|60|93.67|215.44|43.08|10341.60|2584.80|5620.20|12926.40|13.95|1189.00|2455.80|1395.80|1409.75|3851.60|3865.55|-4224.40| +2450820|42556|2450896|37025|1642777|2129|6234|37025|1642777|2129|6234|4|23|6|5|10234|13|275|6|29.54|66.76|4.67|372.54|28.02|177.24|400.56|0.20|25.77|60.06|2.25|2.45|62.31|62.51|-174.99| +2450820|42556|2450871|37025|1642777|2129|6234|37025|1642777|2129|6234|4|2|9|5|7441|85|275|22|24.87|50.98|40.78|224.40|897.16|547.14|1121.56|44.85|0.00|527.12|897.16|942.01|1424.28|1469.13|350.02| +2450820|42556|2450896|37025|1642777|2129|6234|37025|1642777|2129|6234|4|43|17|5|11102|34|275|71|48.85|51.78|5.69|3272.39|403.99|3468.35|3676.38|16.15|0.00|1359.65|403.99|420.14|1763.64|1779.79|-3064.36| +2450820|42556|2450823|37025|1642777|2129|6234|37025|1642777|2129|6234|4|46|17|1|8413|89|275|70|11.82|32.97|2.96|2100.70|207.20|827.40|2307.90|0.00|0.00|1061.20|207.20|207.20|1268.40|1268.40|-620.20| +2450820|42556|2450844|37025|1642777|2129|6234|37025|1642777|2129|6234|4|37|16|5|10606|36|275|32|14.99|31.32|31.32|0.00|1002.24|479.68|1002.24|10.02|0.00|440.96|1002.24|1012.26|1443.20|1453.22|522.56| +2450820|42556|2450906|37025|1642777|2129|6234|37025|1642777|2129|6234|4|40|19|4|961|234|275|64|51.28|63.07|5.67|3673.60|362.88|3281.92|4036.48|10.88|0.00|524.16|362.88|373.76|887.04|897.92|-2919.04| +2450820|42556|2450861|37025|1642777|2129|6234|37025|1642777|2129|6234|4|27|4|1|5305|8|275|91|29.18|72.36|39.07|3029.39|3555.37|2655.38|6584.76|0.00|0.00|394.94|3555.37|3555.37|3950.31|3950.31|899.99| +2450820|42556|2450871|37025|1642777|2129|6234|37025|1642777|2129|6234|4|87|19|5|12637|197|275|49|47.56|66.10|1.32|3174.22|64.68|2330.44|3238.90|0.64|0.00|712.46|64.68|65.32|777.14|777.78|-2265.76| +2450820|42556|2450892|37025|1642777|2129|6234|37025|1642777|2129|6234|4|66|19|4|5707|57|275|42|14.45|30.63|30.01|26.04|1260.42|606.90|1286.46|0.00|0.00|179.76|1260.42|1260.42|1440.18|1440.18|653.52| +2450820|74419|2450874|21392|1315015|6685|6708|21392|1315015|6685|6708|1|50|13|2|12752|268|276|60|78.14|154.71|80.44|4456.20|4826.40|4688.40|9282.60|48.26|0.00|835.20|4826.40|4874.66|5661.60|5709.86|138.00| +2450820|74419|2450875|21392|1315015|6685|6708|21392|1315015|6685|6708|1|45|18|4|8960|76|276|3|92.83|229.29|188.01|123.84|564.03|278.49|687.87|11.28|0.00|27.51|564.03|575.31|591.54|602.82|285.54| +2450820|74419|2450859|21392|1315015|6685|6708|21392|1315015|6685|6708|1|88|15|4|5476|35|276|97|13.35|19.22|15.76|335.62|1528.72|1294.95|1864.34|0.00|0.00|819.65|1528.72|1528.72|2348.37|2348.37|233.77| +2450820|74419|2450840|21392|1315015|6685|6708|21392|1315015|6685|6708|1|59|7|1|116|81|276|67|45.36|51.71|46.02|381.23|3083.34|3039.12|3464.57|246.66|0.00|519.25|3083.34|3330.00|3602.59|3849.25|44.22| +2450820|74419|2450848|21392|1315015|6685|6708|21392|1315015|6685|6708|1|107|11|1|1015|158|276|45|43.25|107.26|65.42|1882.80|2943.90|1946.25|4826.70|206.07|0.00|48.15|2943.90|3149.97|2992.05|3198.12|997.65| +2450820|74419|2450881|21392|1315015|6685|6708|21392|1315015|6685|6708|1|36|10|2|15304|281|276|3|87.74|185.13|164.76|61.11|494.28|263.22|555.39|0.00|0.00|72.18|494.28|494.28|566.46|566.46|231.06| +2450820|74419|2450841|21392|1315015|6685|6708|21392|1315015|6685|6708|1|51|14|3|4114|286|276|49|5.65|9.15|4.48|228.83|219.52|276.85|448.35|8.42|8.78|223.93|210.74|219.16|434.67|443.09|-66.11| +2450820|74419|2450899|21392|1315015|6685|6708|21392|1315015|6685|6708|1|105|2|4|3224|51|276|72|45.34|100.20|57.11|3102.48|4111.92|3264.48|7214.40|205.59|0.00|2885.76|4111.92|4317.51|6997.68|7203.27|847.44| +2450820|74419|2450857|21392|1315015|6685|6708|21392|1315015|6685|6708|1|58|18|1|6734|20|276|23|56.11|76.30|6.86|1597.12|157.78|1290.53|1754.90|0.23|153.04|701.96|4.74|4.97|706.70|706.93|-1285.79| +2450820|74419|2450844|21392|1315015|6685|6708|21392|1315015|6685|6708|1|70|14|4|15385|191|276|36|46.92|84.45|5.06|2858.04|182.16|1689.12|3040.20|3.64|0.00|121.32|182.16|185.80|303.48|307.12|-1506.96| +2450820|74419|2450859|21392|1315015|6685|6708|21392|1315015|6685|6708|1|64|11|1|10732|141|276|12|95.35|266.98|37.37|2755.32|448.44|1144.20|3203.76|0.00|201.79|1505.76|246.65|246.65|1752.41|1752.41|-897.55| +2450820|76279|2450858|45571|1176172|1173|12174|45571|1176172|1173|12174|2|33|16|4|17630|76|277|47|7.43|13.07|7.18|276.83|337.46|349.21|614.29|30.37|0.00|54.99|337.46|367.83|392.45|422.82|-11.75| +2450820|76279|2450848|45571|1176172|1173|12174|45571|1176172|1173|12174|2|71|18|5|7171|80|277|41|27.18|31.52|29.94|64.78|1227.54|1114.38|1292.32|24.55|0.00|348.91|1227.54|1252.09|1576.45|1601.00|113.16| +|76279|2450881||1176172||12174|45571||||2|106|13|1|6118|257|277||50.24|88.42|37.13|2820.95|2042.15|||183.79|0.00||2042.15|2225.94|3938.55|4122.34|-721.05| +2450820|76279|2450890|45571|1176172|1173|12174|45571|1176172|1173|12174|2|19|1|1|13123|14|277|9|6.07|9.34|9.24|0.90|83.16|54.63|84.06|2.49|0.00|0.81|83.16|85.65|83.97|86.46|28.53| +2450820|76279|2450862|45571|1176172|1173|12174|45571|1176172|1173|12174|2|73|7|3|17126|261|277|86|35.90|106.98|84.51|1932.42|7267.86|3087.40|9200.28|436.07|0.00|3127.82|7267.86|7703.93|10395.68|10831.75|4180.46| +2450820|76279|2450852|45571|1176172|1173|12174|45571|1176172|1173|12174|2|92|17|5|9739|216|277|45|32.94|69.50|4.86|2908.80|218.70|1482.30|3127.50|6.56|0.00|250.20|218.70|225.26|468.90|475.46|-1263.60| +2450820|76279|2450908|45571|1176172|1173|12174|45571|1176172|1173|12174|2|5|3|2|5276|13|277|86|99.42|261.47|141.19|10344.08|12142.34|8550.12|22486.42|242.84|0.00|9893.44|12142.34|12385.18|22035.78|22278.62|3592.22| +2450820|62542|2450827|27866|459487|261|46682|27866|459487|261|46682|4|55|15|2|14728|8|278|60|85.16|97.08|39.80|3436.80|2388.00|5109.60|5824.80|23.88|0.00|1572.60|2388.00|2411.88|3960.60|3984.48|-2721.60| +2450820|62542|2450879|27866|459487|261|46682|27866|459487|261|46682|4|67|7|3|13093|20|278|66|13.51|26.61|15.43|737.88|1018.38|891.66|1756.26|71.28|0.00|69.96|1018.38|1089.66|1088.34|1159.62|126.72| +2450820|62542|2450859|27866|459487|261|46682|27866|459487|261|46682|4|42|5|1|16603|298|278|32|21.44|36.23|0.00|1159.36|0.00|686.08|1159.36|0.00|0.00|138.88|0.00|0.00|138.88|138.88|-686.08| +2450820|62542|2450827|27866|459487|261|46682|27866|459487|261|46682|4|19|11|4|1076|208|278|46|79.55|224.33|201.89|1032.24|9286.94|3659.30|10319.18|371.47|0.00|1341.36|9286.94|9658.41|10628.30|10999.77|5627.64| +2450820|62542|2450874|27866|459487|261|46682|27866|459487|261|46682|4|35|16|3|8605|127|278|40|99.91|169.84|163.04|272.00|6521.60|3996.40|6793.60|391.29|0.00|2649.20|6521.60|6912.89|9170.80|9562.09|2525.20| +2450820|62542|2450878|27866|459487|261|46682|27866|459487|261|46682|4|31|13|5|6976|204|278|41|78.89|234.30|49.20|7589.10|2017.20|3234.49|9606.30|20.17|0.00|3266.06|2017.20|2037.37|5283.26|5303.43|-1217.29| +2450820|62992|2450886|72216|644272|2375|7278|98404|1260684|4387|15243|1|6|19|5|17966|190|279|15|47.07|120.49|107.23|198.90|1608.45|706.05|1807.35|80.42|0.00|849.45|1608.45|1688.87|2457.90|2538.32|902.40| +2450820|62992|2450870|72216|644272|2375|7278|98404|1260684|4387|15243|1|92|17|1|6344|219|279|24|66.67|136.00|62.56|1762.56|1501.44|1600.08|3264.00|40.08|165.15|261.12|1336.29|1376.37|1597.41|1637.49|-263.79| +2450820|62992|2450905|72216|644272|2375|7278|98404|1260684|4387|15243|1|100|13|4|7627|46|279|89|35.22|92.62|63.90|2556.08|5687.10|3134.58|8243.18|0.00|0.00|741.37|5687.10|5687.10|6428.47|6428.47|2552.52| +2450820|62992|2450910|72216|644272|2375|7278|98404|1260684|4387|15243|1|66|14|5|15145|46|279|20|87.90|156.46|79.79|1533.40|1595.80|1758.00|3129.20|2.23|1563.88|344.20|31.92|34.15|376.12|378.35|-1726.08| +2450820|62992|2450870|72216|644272|2375|7278|98404|1260684|4387|15243|1|4|19|5|13012|248|279|37|52.30|114.53|107.65|254.56|3983.05|1935.10|4237.61|318.64|0.00|762.57|3983.05|4301.69|4745.62|5064.26|2047.95| +2450820|74638|2450845|91015|680277|2339|17554|91015|680277|2339|17554|4|5|6|3|5198|149|280|66|84.56|125.99|25.19|6652.80|1662.54|5580.96|8315.34|133.00|0.00|3991.02|1662.54|1795.54|5653.56|5786.56|-3918.42| +2450820|74638|2450839|91015|680277|2339|17554|91015|680277|2339|17554|4|23|11|3|10694|242|280|22|21.55|55.59|26.68|636.02|586.96|474.10|1222.98|5.86|0.00|550.22|586.96|592.82|1137.18|1143.04|112.86| +2450820|74638|2450886|91015|680277|2339|17554|91015|680277|2339|17554|4|106|12|1|4369|158|280|15|28.53|72.18|67.12|75.90|1006.80|427.95|1082.70|90.61|0.00|498.00|1006.80|1097.41|1504.80|1595.41|578.85| +2450820|74638|2450866|91015|680277|2339|17554|91015|680277|2339|17554|4|9|1|3|16996|62|280|100|22.04|62.15|21.13|4102.00|2113.00|2204.00|6215.00|126.78|0.00|1988.00|2113.00|2239.78|4101.00|4227.78|-91.00| +2450820|74638|2450864|91015|680277|2339|17554|91015|680277|2339|17554|4|18|3|2|260|28|280|8|53.77|66.13|47.61|148.16|380.88|430.16|529.04|0.00|0.00|264.48|380.88|380.88|645.36|645.36|-49.28| +2450820|74638|2450843|91015|680277|2339|17554|91015|680277|2339|17554|4|23|16|4|1141|118|280|65|91.79|275.37|66.08|13603.85|4295.20|5966.35|17899.05|386.56|0.00|7875.40|4295.20|4681.76|12170.60|12557.16|-1671.15| +2450820|74638|2450897|91015|680277|2339|17554|91015|680277|2339|17554|4|50|5|5|1120|70|280|24|12.18|33.12|9.60|564.48|230.40|292.32|794.88|2.30|0.00|111.12|230.40|232.70|341.52|343.82|-61.92| +2450820|74638|2450910|91015|680277|2339|17554|91015|680277|2339|17554|4|66|9|1|6232|95|280|78|33.45|77.93|31.17|3647.28|2431.26|2609.10|6078.54|0.00|1167.00|2613.00|1264.26|1264.26|3877.26|3877.26|-1344.84| +2450820|74638|2450905|91015|680277|2339|17554|91015|680277|2339|17554|4|16|20|4|1216|289|280|70|97.23|287.80|207.21|5641.30|14504.70|6806.10|20146.00|0.00|0.00|1208.20|14504.70|14504.70|15712.90|15712.90|7698.60| +2450820|82319|2450838|38011|1694812|3613|16014|38011|1694812|3613|16014|4|79|3|4|850|104|281|17|29.19|55.75|2.23|909.84|37.91|496.23|947.75|0.89|7.96|236.81|29.95|30.84|266.76|267.65|-466.28| +2450820|82319|2450842|38011|1694812|3613|16014|38011|1694812|3613|16014|4|6|16|4|9494|240|281|16|52.29|144.32|64.94|1270.08|1039.04|836.64|2309.12|62.34|0.00|877.44|1039.04|1101.38|1916.48|1978.82|202.40| +2450820|82319|2450850|38011|1694812|3613|16014|38011|1694812|3613|16014|4|44|18|1|5125|146|281|55|98.40|276.50|11.06|14599.20|608.30|5412.00|15207.50|54.74|0.00|5322.35|608.30|663.04|5930.65|5985.39|-4803.70| +2450820|82319|2450857|38011|1694812|3613|16014|38011|1694812|3613|16014|4|22|19|3|12602|282|281|76|98.97|150.43|64.68|6517.00|4915.68|7521.72|11432.68|393.25|0.00|5029.68|4915.68|5308.93|9945.36|10338.61|-2606.04| +2450820|82319|2450882|38011|1694812|3613|16014|38011|1694812|3613|16014|4|23|9|5|8620|220|281|83|45.63|122.74|101.87|1732.21|8455.21|3787.29|10187.42|760.96|0.00|4584.09|8455.21|9216.17|13039.30|13800.26|4667.92| +2450820|82319|2450835|38011|1694812|3613|16014|38011|1694812|3613|16014|4|90|16|3|15055|65|281|64|26.05|47.93|16.77|1994.24|1073.28|1667.20|3067.52|21.46|0.00|122.24|1073.28|1094.74|1195.52|1216.98|-593.92| +2450820|82319|2450870|38011|1694812|3613|16014|38011|1694812|3613|16014|4|79|2|4|1795|3|281|96|65.41|180.53|74.01|10225.92|7104.96|6279.36|17330.88|71.04|0.00|3639.36|7104.96|7176.00|10744.32|10815.36|825.60| +2450820|82319|2450850|38011|1694812|3613|16014|38011|1694812|3613|16014|4|44|9|2|16945|201|281|21|65.45|141.37|19.79|2553.18|415.59|1374.45|2968.77|8.31|0.00|29.61|415.59|423.90|445.20|453.51|-958.86| +2450820|82319|2450889|38011|1694812|3613|16014|38011|1694812|3613|16014|4|62|19|4|11551|44|281|67|91.15|159.51|86.13|4916.46|5770.71|6107.05|10687.17|173.12|0.00|4808.59|5770.71|5943.83|10579.30|10752.42|-336.34| +2450820|82319|2450874|38011|1694812|3613|16014|38011|1694812|3613|16014|4|49|10|3|15958|4|281|23|76.55|188.31|122.40|1515.93|2815.20|1760.65|4331.13|28.15|0.00|43.24|2815.20|2843.35|2858.44|2886.59|1054.55| +2450820|82319|2450896|38011|1694812|3613|16014|38011|1694812|3613|16014|4|36|1|2|15847|286|281|4|63.31|138.01|59.34|314.68|237.36|253.24|552.04|21.36|0.00|220.80|237.36|258.72|458.16|479.52|-15.88| +2450820|82319|2450838|38011|1694812|3613|16014|38011|1694812|3613|16014|4|92|13|3|712|115|281|61|63.80|84.85|84.85|0.00|5175.85|3891.80|5175.85|419.24|517.58|465.43|4658.27|5077.51|5123.70|5542.94|766.47| +2450820|59275|2450895|826|662289|4999|20462|826|662289|4999|20462|2|25|13|4|4280|259|282|44|43.83|77.14|16.97|2647.48|746.68|1928.52|3394.16|44.80|0.00|1187.56|746.68|791.48|1934.24|1979.04|-1181.84| +2450820|59275|2450828|826|662289|4999|20462|826|662289|4999|20462|2|45|13|5|2617|39|282|45|43.55|113.66|18.18|4296.60|818.10|1959.75|5114.70|15.38|433.59|766.80|384.51|399.89|1151.31|1166.69|-1575.24| +2450820|59275|2450897|826|662289|4999|20462|826|662289|4999|20462|2|100|19|2|7429|33|282|65|3.62|9.99|0.79|598.00|51.35|235.30|649.35|3.59|0.00|324.35|51.35|54.94|375.70|379.29|-183.95| +2450820|59275|2450877|826|662289|4999|20462|826|662289|4999|20462|2|16|19|3|2164|140|282|63|54.38|157.15|12.57|9108.54|791.91|3425.94|9900.45|31.67|0.00|1088.64|791.91|823.58|1880.55|1912.22|-2634.03| +||2450824|||||826||4999|20462||||5|12530|34|282|||167.08|||11728.80|||234.57|0.00|3307.50||11963.37|15036.30|15270.87|5808.60| +2450820|59275|2450885|826|662289|4999|20462|826|662289|4999|20462|2|100|6|2|6644|175|282|40|25.34|52.45|44.05|336.00|1762.00|1013.60|2098.00|105.72|0.00|797.20|1762.00|1867.72|2559.20|2664.92|748.40| +2450820|59275|2450903|826|662289|4999|20462|826|662289|4999|20462|2|94|12|3|3949|190|282|89|46.24|117.44|113.91|314.17|10137.99|4115.36|10452.16|304.13|0.00|1880.57|10137.99|10442.12|12018.56|12322.69|6022.63| +2450820|59275|2450837|826|662289|4999|20462|826|662289|4999|20462|2|87|15|4|3673|204|282|19|16.09|21.88|18.81|58.33|357.39|305.71|415.72|4.82|303.78|153.71|53.61|58.43|207.32|212.14|-252.10| +2450820|59275|2450899|826|662289|4999|20462|826|662289|4999|20462|2|96|7|2|1196|9|282|17|98.75|101.71|66.11|605.20|1123.87|1678.75|1729.07|42.25|67.43|120.87|1056.44|1098.69|1177.31|1219.56|-622.31| +2450820|59275|2450894|826|662289|4999|20462|826|662289|4999|20462|2|35|6|5|5336|206|282|76|73.90|155.92|116.94|2962.48|8887.44|5616.40|11849.92|266.62|0.00|1895.44|8887.44|9154.06|10782.88|11049.50|3271.04| +2450820|59275|2450864|826|662289|4999|20462|826|662289|4999|20462|2|24|3|3|7696|105|282|87|15.72|41.50|15.77|2238.51|1371.99|1367.64|3610.50|109.75|0.00|1552.08|1371.99|1481.74|2924.07|3033.82|4.35| +2450820|49102|2450878|7376|753683|6743|18203|7376|753683|6743|18203|1|105|2|1|8428|282|283|73|89.06|89.95|39.57|3677.74|2888.61|6501.38|6566.35|144.43|0.00|1706.74|2888.61|3033.04|4595.35|4739.78|-3612.77| +2450820|49102|2450872|7376|753683|6743|18203|7376|753683|6743|18203|1|16|13|5|13390|9|283|88|51.69|103.38|3.10|8824.64|272.80|4548.72|9097.44|0.00|120.03|1636.80|152.77|152.77|1789.57|1789.57|-4395.95| +2450820|49102|2450850|7376|753683|6743|18203|7376|753683|6743|18203|1|53|2|3|15086|42|283|25|87.74|216.71|67.18|3738.25|1679.50|2193.50|5417.75|100.77|0.00|1300.25|1679.50|1780.27|2979.75|3080.52|-514.00| +2450820|49102|2450870|7376|753683|6743|18203|7376|753683|6743|18203|1|5|10|5|9584|232|283|81|51.75|110.22|84.86|2054.16|6873.66|4191.75|8927.82|274.94|0.00|3927.69|6873.66|7148.60|10801.35|11076.29|2681.91| +2450820|49102|2450879|7376|753683|6743|18203|7376|753683|6743|18203|1|98|4|4|10370|167|283|16|11.01|11.45|8.70|44.00|139.20|176.16|183.20|11.13|0.00|60.32|139.20|150.33|199.52|210.65|-36.96| +2450820|49102|2450888|7376|753683|6743|18203|7376|753683|6743|18203|1|88|16|2|3574|49|283|37|75.93|138.19|82.91|2045.36|3067.67|2809.41|5113.03|276.09|0.00|408.85|3067.67|3343.76|3476.52|3752.61|258.26| +2450820|49102|2450839|7376|753683|6743|18203|7376|753683|6743|18203|1|92|13|2|919|128|283|83|46.03|50.63|27.84|1891.57|2310.72|3820.49|4202.29|184.85|0.00|1259.94|2310.72|2495.57|3570.66|3755.51|-1509.77| +2450820|49102|2450862|7376|753683|6743|18203|7376|753683|6743|18203|1|101|10|2|9565|97|283|24|47.21|54.76|28.47|630.96|683.28|1133.04|1314.24|47.82|0.00|538.80|683.28|731.10|1222.08|1269.90|-449.76| +2450820|49102|2450854|7376|753683|6743|18203|7376|753683|6743|18203|1|34|4|3|14612|171|283|27|91.66|158.57|114.17|1198.80|3082.59|2474.82|4281.39|246.60|0.00|427.95|3082.59|3329.19|3510.54|3757.14|607.77| +2450820|73888|2450840|98595|1199774|5400|10179|98595|1199774|5400|10179|2|61|10|1|16975|141|284|30|87.56|136.59|2.73|4015.80|81.90|2626.80|4097.70|5.73|0.00|450.60|81.90|87.63|532.50|538.23|-2544.90| +2450820|73888|2450904|98595|1199774|5400|10179|98595|1199774|5400|10179|2|23|9|2|17464|194|284|7|91.11|264.21|243.07|147.98|1701.49|637.77|1849.47|153.13|0.00|166.39|1701.49|1854.62|1867.88|2021.01|1063.72| +2450820|73888|2450869|98595|1199774|5400|10179|98595|1199774|5400|10179|2|73|10|5|838|146|284|31|10.65|24.49|12.24|379.75|379.44|330.15|759.19|22.76|0.00|280.86|379.44|402.20|660.30|683.06|49.29| +2450820|73888|2450828|98595|1199774|5400|10179|98595|1199774|5400|10179|2|93|6|2|6457|39|284|49|18.17|35.61|3.20|1588.09|156.80|890.33|1744.89|2.74|117.60|610.54|39.20|41.94|649.74|652.48|-851.13| +2450820|73888|2450856|98595|1199774|5400|10179|98595|1199774|5400|10179|2|93|2|3|5054|247|284|40|91.91|177.38|172.05|213.20|6882.00|3676.40|7095.20|619.38|0.00|780.40|6882.00|7501.38|7662.40|8281.78|3205.60| +2450820|73888|2450829|98595|1199774|5400|10179|98595|1199774|5400|10179|2|16|15|1|9464|213|284|12|59.60|160.32|11.22|1789.20|134.64|715.20|1923.84|2.69|0.00|153.84|134.64|137.33|288.48|291.17|-580.56| +2450820|73888|2450834|98595|1199774|5400|10179|98595|1199774|5400|10179|2|4|18|5|1750|173|284|35|26.48|54.01|18.90|1228.85|661.50|926.80|1890.35|6.61|0.00|548.10|661.50|668.11|1209.60|1216.21|-265.30| +2450820|73888|2450860|98595|1199774|5400|10179|98595|1199774|5400|10179|2|17|5|2|9811|21|284|6|16.98|44.48|9.78|208.20|58.68|101.88|266.88|4.69|0.00|24.00|58.68|63.37|82.68|87.37|-43.20| +2450820|73888|2450840|98595|1199774|5400|10179|98595|1199774|5400|10179|2|25|2|1|2209|136|284|72|88.72|230.67|11.53|15778.08|830.16|6387.84|16608.24|24.90|0.00|1660.32|830.16|855.06|2490.48|2515.38|-5557.68| +2450820|73888|2450870|98595|1199774|5400|10179|98595|1199774|5400|10179|2|104|8|1|898|262|284|75|54.39|160.45|3.20|11793.75|240.00|4079.25|12033.75|2.40|0.00|1804.50|240.00|242.40|2044.50|2046.90|-3839.25| +2450820|73888|2450906|98595|1199774|5400|10179|98595|1199774|5400|10179|2|102|8|4|9433|159|284|12|16.50|41.74|15.44|315.60|185.28|198.00|500.88|11.11|0.00|175.20|185.28|196.39|360.48|371.59|-12.72| +2450820|73888|2450873|98595|1199774|5400|10179|98595|1199774|5400|10179|2|41|13|2|3182|215|284|96|77.74|121.27|29.10|8848.32|2793.60|7463.04|11641.92|167.61|0.00|5704.32|2793.60|2961.21|8497.92|8665.53|-4669.44| +2450820|73888|2450840|98595|1199774|5400|10179|98595|1199774|5400|10179|2|55|6|4|5233|223|284|66|89.89|174.38|6.97|11049.06|460.02|5932.74|11509.08|3.86|331.21|4603.50|128.81|132.67|4732.31|4736.17|-5803.93| +2450820|52855|2450873|18433|547933|3833|28933|18433|547933|3833|28933|2|64|5|4|11332|137|285|72|15.04|45.12|15.34|2144.16|1104.48|1082.88|3248.64|0.00|0.00|324.72|1104.48|1104.48|1429.20|1429.20|21.60| +2450820|52855|2450858|18433|547933|3833|28933|18433|547933|3833|28933|2|16|3|4|13546|271|285|70|6.02|11.55|2.07|663.60|144.90|421.40|808.50|13.04|0.00|347.20|144.90|157.94|492.10|505.14|-276.50| +2450820|52855|2450904|18433|547933|3833|28933|18433|547933|3833|28933|2|44|1|3|2077|55|285|67|27.07|73.08|16.80|3770.76|1125.60|1813.69|4896.36|67.53|0.00|1419.73|1125.60|1193.13|2545.33|2612.86|-688.09| +2450820|52855|2450906|18433|547933|3833|28933|18433|547933|3833|28933|2|65|8|5|12139|26|285|32|64.65|154.51|7.72|4697.28|247.04|2068.80|4944.32|14.82|0.00|1136.96|247.04|261.86|1384.00|1398.82|-1821.76| +2450820|52855|2450902|18433|547933|3833|28933|18433|547933|3833|28933|2|7|15|2|10960|126|285|43|69.18|157.03|86.36|3038.81|3713.48|2974.74|6752.29|185.67|0.00|2498.30|3713.48|3899.15|6211.78|6397.45|738.74| +2450820|52855|2450854|18433|547933|3833|28933|18433|547933|3833|28933|2|13|17|1|14536|167|285|97|69.11|198.34|144.78|5195.32|14043.66|6703.67|19238.98|561.74|0.00|9619.49|14043.66|14605.40|23663.15|24224.89|7339.99| +2450820|52855|2450910|18433|547933|3833|28933|18433|547933|3833|28933|2|91|13|5|11197|222|285|87|24.17|70.57|23.28|4114.23|2025.36|2102.79|6139.59|182.28|0.00|2087.13|2025.36|2207.64|4112.49|4294.77|-77.43| +2450820|52855|2450869|18433|547933|3833|28933|18433|547933|3833|28933|2|77|18|2|7390|258|285|68|51.18|141.25|8.47|9029.04|575.96|3480.24|9605.00|34.55|0.00|2016.88|575.96|610.51|2592.84|2627.39|-2904.28| +2450820|52855|2450883|18433|547933|3833|28933|18433|547933|3833|28933|2|100|4|1|6590|195|285|41|25.66|57.22|11.44|1876.98|469.04|1052.06|2346.02|9.38|0.00|281.26|469.04|478.42|750.30|759.68|-583.02| +2450820|52855|2450877|18433|547933|3833|28933|18433|547933|3833|28933|2|45|1|5|10120|202|285|7|41.78|48.88|12.70|253.26|88.90|292.46|342.16|1.77|0.00|171.08|88.90|90.67|259.98|261.75|-203.56| +2450820|52855|2450837|18433|547933|3833|28933|18433|547933|3833|28933|2|40|8|2|5386|15|285|69|90.89|159.96|81.57|5408.91|5628.33|6271.41|11037.24|56.28|0.00|440.91|5628.33|5684.61|6069.24|6125.52|-643.08| +2450820|52855|2450822|18433|547933|3833|28933|18433|547933|3833|28933|2|100|12|3|17068|142|285|89|55.15|66.18|14.55|4595.07|1294.95|4908.35|5890.02|77.69|0.00|412.07|1294.95|1372.64|1707.02|1784.71|-3613.40| +2450820|45796|2450848|51998|361102|5053|40604|51998|361102|5053|40604|2|12|7|4|7238|83|286|35|16.06|25.53|5.87|688.10|205.45|562.10|893.55|4.00|125.32|196.35|80.13|84.13|276.48|280.48|-481.97| +2450820|45796||51998|361102|5053||51998|361102|5053|||107|14||14|39|286||37.24|||||2122.68||66.91|0.00|605.91||1405.27|1944.27|2011.18|-784.32| +2450820|45796|2450863|51998|361102|5053|40604|51998|361102|5053|40604|2|69|11|3|382|140|286|96|76.52|200.48|78.18|11740.80|7505.28|7345.92|19246.08|675.47|0.00|8275.20|7505.28|8180.75|15780.48|16455.95|159.36| +2450820|45796|2450835|51998|361102|5053|40604|51998|361102|5053|40604|2|5|15|4|8383|287|286|22|30.71|81.68|0.00|1796.96|0.00|675.62|1796.96|0.00|0.00|503.14|0.00|0.00|503.14|503.14|-675.62| +2450820|45796|2450879|51998|361102|5053|40604|51998|361102|5053|40604|2|63|10|5|17920|72|286|32|38.42|111.80|65.96|1466.88|2110.72|1229.44|3577.60|84.42|0.00|1073.28|2110.72|2195.14|3184.00|3268.42|881.28| +2450820|45796|2450874|51998|361102|5053|40604|51998|361102|5053|40604|2|57|20|4|1214|184|286|94|70.98|132.73|13.27|11229.24|1247.38|6672.12|12476.62|37.42|0.00|4491.32|1247.38|1284.80|5738.70|5776.12|-5424.74| +2450820|78979|2450826|62092|1669220|348|6676|62092|1669220|348|6676|1|37|3|4|1579|76|287|21|2.79|7.44|0.59|143.85|12.39|58.59|156.24|0.74|0.00|78.12|12.39|13.13|90.51|91.25|-46.20| +2450820|78979|2450824|62092|1669220|348|6676|62092|1669220|348|6676|1|26|20|5|446|260|287|95|89.08|267.24|88.18|17010.70|8377.10|8462.60|25387.80|335.08|0.00|7869.80|8377.10|8712.18|16246.90|16581.98|-85.50| +2450820|78979|2450872|62092|1669220|348|6676|62092|1669220|348|6676|1|48|11|2|14839|186|287|62|30.54|32.98|8.57|1513.42|531.34|1893.48|2044.76|15.94|0.00|163.06|531.34|547.28|694.40|710.34|-1362.14| +2450820|78979|2450870|62092|1669220|348|6676|62092|1669220|348|6676|1|42|14|2|14858|275|287|58|31.96|59.12|41.97|994.70|2434.26|1853.68|3428.96|219.08|0.00|68.44|2434.26|2653.34|2502.70|2721.78|580.58| +2450820||||1669220||6676||1669220||||77|5|5|13717||287|26||67.59||17.68|1739.66|976.30||139.17|||||2337.14|2476.31|763.36| +2450820|78979|2450861|62092|1669220|348|6676|62092|1669220|348|6676|1|47|17|4|4108|141|287|81|37.23|82.65|77.69|401.76|6292.89|3015.63|6694.65|188.78|0.00|1003.59|6292.89|6481.67|7296.48|7485.26|3277.26| +2450820|75472|2450866|60497|130995|4693|14744|60497|130995|4693|14744|1|11|20|3|7588|156|288|53|72.61|96.57|82.08|767.97|4350.24|3848.33|5118.21|348.01|0.00|2046.86|4350.24|4698.25|6397.10|6745.11|501.91| +2450820|75472|2450830|60497|130995|4693|14744|60497|130995|4693|14744|1|16|14|1|9940|225|288|41|98.54|272.95|234.73|1567.02|9623.93|4040.14|11190.95|769.91|0.00|783.10|9623.93|10393.84|10407.03|11176.94|5583.79| +2450820|75472|2450870|60497|130995|4693|14744|60497|130995|4693|14744|1|43|19|5|12352|129|288|65|82.07|222.40|71.16|9830.60|4625.40|5334.55|14456.00|138.76|0.00|2312.70|4625.40|4764.16|6938.10|7076.86|-709.15| +2450820|75472|2450899|60497|130995|4693|14744|60497|130995|4693|14744|1|79|20|2|12536|225|288|87|39.26|47.50|27.55|1735.65|2396.85|3415.62|4132.50|191.74|0.00|1693.89|2396.85|2588.59|4090.74|4282.48|-1018.77| +2450820|75472|2450827|60497|130995|4693|14744|60497|130995|4693|14744|1|77|5|3|17188|29|288|7|64.03|147.26|104.55|298.97|731.85|448.21|1030.82|4.75|636.70|206.15|95.15|99.90|301.30|306.05|-353.06| +2450820|75472|2450872|60497|130995|4693|14744|60497|130995|4693|14744|1|13|17|2|15460|139|288|11|87.66|203.37|10.16|2125.31|111.76|964.26|2237.07|5.58|0.00|290.73|111.76|117.34|402.49|408.07|-852.50| +2450820|75472|2450841|60497|130995|4693|14744|60497|130995|4693|14744|1|19|19|5|3829|82|288|57|50.94|151.80|54.64|5538.12|3114.48|2903.58|8652.60|52.32|2460.43|3374.40|654.05|706.37|4028.45|4080.77|-2249.53| +2450820|75472|2450877|60497|130995|4693|14744|60497|130995|4693|14744|1|28|7|5|17194|161|288|80|37.89|97.75|78.20|1564.00|6256.00|3031.20|7820.00|563.04|0.00|2502.40|6256.00|6819.04|8758.40|9321.44|3224.80| +2450820|75472|2450888|60497|130995|4693|14744|60497|130995|4693|14744|1|9|5|2|12907|156|288|20|41.11|75.23|19.55|1113.60|391.00|822.20|1504.60|23.46|0.00|466.40|391.00|414.46|857.40|880.86|-431.20| +2450820|75472|2450904|60497|130995|4693|14744|60497|130995|4693|14744|1|100|19|1|7340|61|288|59|7.09|18.29|17.37|54.28|1024.83|418.31|1079.11|40.99|0.00|182.90|1024.83|1065.82|1207.73|1248.72|606.52| +2450820|77649|2450837|6806|1801130|2301|31072|6806|1801130|2301|31072|2|39|16|4|17122|150|289|35|4.53|6.38|0.00|223.30|0.00|158.55|223.30|0.00|0.00|111.65|0.00|0.00|111.65|111.65|-158.55| +2450820|77649|2450879|6806|1801130|2301|31072|6806|1801130|2301|31072|2|93|18|1|9634|58|289|59|41.96|50.35|33.73|980.58|1990.07|2475.64|2970.65|59.70|0.00|474.95|1990.07|2049.77|2465.02|2524.72|-485.57| +2450820|77649|2450910|6806|1801130|2301|31072|6806|1801130|2301|31072|2|9|16|2|14335|91|289|54|54.69|113.20|97.35|855.90|5256.90|2953.26|6112.80|75.69|3995.24|183.06|1261.66|1337.35|1444.72|1520.41|-1691.60| +2450820|77649|2450908|6806|1801130|2301|31072|6806|1801130|2301|31072|2|18|6|2|13886|57|289|3|41.47|90.81|69.92|62.67|209.76|124.41|272.43|18.87|0.00|81.72|209.76|228.63|291.48|310.35|85.35| +2450820|45821|2450867|61298|732996|1494|16392|61298|732996|1494|16392|1|99|5|5|6061|48|290|27|92.13|133.58|46.75|2344.41|1262.25|2487.51|3606.66|25.24|0.00|144.18|1262.25|1287.49|1406.43|1431.67|-1225.26| +2450820|45821|2450853|61298|732996|1494|16392|61298|732996|1494|16392|1|40|11|1|17917|141|290|15|5.75|12.76|8.54|63.30|128.10|86.25|191.40|5.12|0.00|24.75|128.10|133.22|152.85|157.97|41.85| +2450820|45821|2450908|61298|732996|1494|16392|61298|732996|1494|16392|1|84|4|1|2966|7|290|40|54.85|150.28|106.69|1743.60|4267.60|2194.00|6011.20|341.40|0.00|2224.00|4267.60|4609.00|6491.60|6833.00|2073.60| +2450820|45821|2450878|61298|732996|1494|16392|61298|732996|1494|16392|1|24|4|4|6616|27|290|29|34.68|35.02|16.45|538.53|477.05|1005.72|1015.58|14.31|0.00|152.25|477.05|491.36|629.30|643.61|-528.67| +2450820|45821|2450901|61298|732996|1494|16392|61298|732996|1494|16392|1|8|11|4|16604|250|290|89|75.94|199.72|129.81|6221.99|11553.09|6758.66|17775.08|62.38|5314.42|6043.10|6238.67|6301.05|12281.77|12344.15|-519.99| +2450820|45821|2450894|61298|732996|1494|16392|61298|732996|1494|16392|1|28|9|3|16826|233|290|57|7.89|10.33|2.89|424.08|164.73|449.73|588.81|14.82|0.00|87.78|164.73|179.55|252.51|267.33|-285.00| +2450820|45821|2450835|61298|732996|1494|16392|61298|732996|1494|16392|1|56|10|2|32|177|290|2|78.25|96.24|59.66|73.16|119.32|156.50|192.48|5.96|0.00|86.60|119.32|125.28|205.92|211.88|-37.18| +2450820|45821|2450847|61298|732996|1494|16392|61298|732996|1494|16392|1|71|10|3|2756|111|290|76|43.13|55.20|19.32|2726.88|1468.32|3277.88|4195.20|0.00|0.00|1258.56|1468.32|1468.32|2726.88|2726.88|-1809.56| +2450820|67600|2450892|32785|1724704|270|14589|32785|1724704|270|14589|2|77|13|3|4088|135|291|29|12.92|22.22|6.44|457.62|186.76|374.68|644.38|1.86|0.00|160.95|186.76|188.62|347.71|349.57|-187.92| +2450820|67600|2450860|32785|1724704|270|14589|32785|1724704|270|14589|2|58|12|5|17374|105|291|41|45.71|62.16|18.64|1784.32|764.24|1874.11|2548.56|0.00|0.00|789.66|764.24|764.24|1553.90|1553.90|-1109.87| +2450820|67600|2450899|32785|1724704|270|14589|32785|1724704|270|14589|2|86|14|5|5644|285|291|81|55.25|140.88|118.33|1826.55|9584.73|4475.25|11411.28|766.77|0.00|798.66|9584.73|10351.50|10383.39|11150.16|5109.48| +2450820|67600|2450852|32785|1724704|270|14589|32785|1724704|270|14589|2|36|16|1|15703|131|291|35|25.04|39.56|30.06|332.50|1052.10|876.40|1384.60|63.12|0.00|678.30|1052.10|1115.22|1730.40|1793.52|175.70| +2450820|67600|2450822|32785|1724704|270|14589|32785|1724704|270|14589|2|50|12|4|188|281|291|44|33.43|60.17|55.95|185.68|2461.80|1470.92|2647.48|49.23|0.00|317.68|2461.80|2511.03|2779.48|2828.71|990.88| +2450820|67600|2450860|32785|1724704|270|14589|32785|1724704|270|14589|2|52|10|3|14263|184|291|3|75.14|186.34|26.08|480.78|78.24|225.42|559.02|6.25|0.00|22.35|78.24|84.49|100.59|106.84|-147.18| +2450820|67600|2450860|32785|1724704|270|14589|32785|1724704|270|14589|2|84|5|2|12434|273|291|93|56.01|56.01|30.24|2396.61|2812.32|5208.93|5208.93|77.33|1265.54|364.56|1546.78|1624.11|1911.34|1988.67|-3662.15| +2450820|67600|2450832|32785|1724704|270|14589|32785|1724704|270|14589|2|48|10|5|5167|89|291|79|60.12|64.32|43.73|1626.61|3454.67|4749.48|5081.28|0.00|2867.37|711.00|587.30|587.30|1298.30|1298.30|-4162.18| +2450820|67600|2450861|32785|1724704|270|14589|32785|1724704|270|14589|2|30|19|4|15640|99|291|12|36.89|73.41|48.45|299.52|581.40|442.68|880.92|52.32|0.00|88.08|581.40|633.72|669.48|721.80|138.72| +2450820|67600|2450851|32785|1724704|270|14589|32785|1724704|270|14589|2|29|18|3|12529|33|291|37|6.55|14.54|12.64|70.30|467.68|242.35|537.98|19.08|229.16|0.00|238.52|257.60|238.52|257.60|-3.83| +2450820|34990|2450839|6881|322995|2309|44776|6881|322995|2309|44776|4|28|13|4|6160|108|292|80|99.35|296.06|0.00|23684.80|0.00|7948.00|23684.80|0.00|0.00|236.80|0.00|0.00|236.80|236.80|-7948.00| +2450820|34990|2450878|6881|322995|2309|44776|6881|322995|2309|44776|4|46|15|1|12448|227|292|93|12.94|36.49|27.00|882.57|2511.00|1203.42|3393.57|200.88|0.00|677.97|2511.00|2711.88|3188.97|3389.85|1307.58| +2450820|34990|2450831|6881|322995|2309|44776|6881|322995|2309|44776|4|83|7|1|15002|27|292|18|25.31|53.40|37.91|278.82|682.38|455.58|961.20|27.29|0.00|374.76|682.38|709.67|1057.14|1084.43|226.80| +2450820|34990|2450877|6881|322995|2309|44776|6881|322995|2309|44776|4|58|18|4|5047|255|292|66|46.48|94.35|66.04|1868.46|4358.64|3067.68|6227.10|130.75|0.00|1120.68|4358.64|4489.39|5479.32|5610.07|1290.96| +2450820|34990|2450853|6881|322995|2309|44776|6881|322995|2309|44776|4|11|17|4|1711|247|292|88|40.74|40.74|0.81|3513.84|71.28|3585.12|3585.12|0.71|0.00|286.00|71.28|71.99|357.28|357.99|-3513.84| +2450820|34990|2450864|6881|322995|2309|44776|6881|322995|2309|44776|4|13|19|4|6487|252|292|67|8.18|13.25|4.63|577.54|310.21|548.06|887.75|3.10|0.00|8.71|310.21|313.31|318.92|322.02|-237.85| +2450820|34990|2450871|6881|322995|2309|44776|6881|322995|2309|44776|4|7|6|5|10123|138|292|29|42.46|100.63|81.51|554.48|2363.79|1231.34|2918.27|165.46|0.00|525.19|2363.79|2529.25|2888.98|3054.44|1132.45| +2450820|34990|2450884|6881|322995|2309|44776|6881|322995|2309|44776|4|6|9|3|4357|3|292|15|14.07|32.92|8.23|370.35|123.45|211.05|493.80|1.23|0.00|14.70|123.45|124.68|138.15|139.38|-87.60| +2450820|34990|2450829|6881|322995|2309|44776|6881|322995|2309|44776|4|99|11|1|17624|158|292|8|75.46|130.54|91.37|313.36|730.96|603.68|1044.32|43.85|0.00|250.56|730.96|774.81|981.52|1025.37|127.28| +2450820|34990|2450883|6881|322995|2309|44776|6881|322995|2309|44776|4|41|15|2|910|286|292|58|30.43|51.42|17.99|1938.94|1043.42|1764.94|2982.36|10.43|0.00|506.92|1043.42|1053.85|1550.34|1560.77|-721.52| +2450820|34990|2450832|6881|322995|2309|44776|6881|322995|2309|44776|4|4|4|2|1537|136|292|3|23.31|50.81|8.63|126.54|25.89|69.93|152.43|1.29|0.00|27.42|25.89|27.18|53.31|54.60|-44.04| +2450820|34990|2450887|6881|322995|2309|44776|6881|322995|2309|44776|4|58|10|5|12475|145|292|56|24.52|36.04|28.11|444.08|1574.16|1373.12|2018.24|31.48|0.00|362.88|1574.16|1605.64|1937.04|1968.52|201.04| +2450820|10584|2450845|41132|725050|5768|21239|41132|725050|5768|21239|1|6|13|3|1078|129|293|95|67.25|141.89|32.63|10379.70|3099.85|6388.75|13479.55|99.81|1673.91|5930.85|1425.94|1525.75|7356.79|7456.60|-4962.81| +2450820|10584|2450849|41132|725050|5768|21239|41132|725050|5768|21239|1|9|3|3|14614|23|293|36|3.26|6.03|2.41|130.32|86.76|117.36|217.08|1.73|0.00|101.88|86.76|88.49|188.64|190.37|-30.60| +2450820|10584|2450838|41132|725050|5768|21239|41132|725050|5768|21239|1|72|3|3|13184|13|293|43|59.30|151.21|117.94|1430.61|5071.42|2549.90|6502.03|354.99|0.00|1040.17|5071.42|5426.41|6111.59|6466.58|2521.52| +2450820|10584|2450827|41132|725050|5768|21239|41132|725050|5768|21239|1|3|9|1|11425|264|293|86|44.33|69.59|12.52|4908.02|1076.72|3812.38|5984.74|43.06|0.00|1675.28|1076.72|1119.78|2752.00|2795.06|-2735.66| +2450820|10584|2450895|41132|725050|5768|21239|41132|725050|5768|21239|1|67|11|2|1675|294|293|72|81.18|157.48|42.51|8277.84|3060.72|5844.96|11338.56|30.60|0.00|2154.24|3060.72|3091.32|5214.96|5245.56|-2784.24| +2450820|10584|2450844|41132|725050|5768|21239|41132|725050|5768|21239|1|67|10|3|9877|109|293|41|17.73|35.99|29.87|250.92|1224.67|726.93|1475.59|0.00|0.00|324.31|1224.67|1224.67|1548.98|1548.98|497.74| +2450820|10584|2450861|41132|725050|5768|21239|41132|725050|5768|21239|1|40|4|4|4933|257|293|76|86.84|127.65|86.80|3104.60|6596.80|6599.84|9701.40|593.71|0.00|4170.88|6596.80|7190.51|10767.68|11361.39|-3.04| +2450820|10584|2450857|41132|725050|5768|21239|41132|725050|5768|21239|1|21|7|2|5974|290|293|36|87.92|218.04|180.97|1334.52|6514.92|3165.12|7849.44|428.03|1759.02|3610.44|4755.90|5183.93|8366.34|8794.37|1590.78| +2450820|10584|2450830|41132|725050|5768|21239|41132|725050|5768|21239|1|83|4|2|15776|113|293|4|73.23|83.48|42.57|163.64|170.28|292.92|333.92|8.51|0.00|123.52|170.28|178.79|293.80|302.31|-122.64| +2450820|10584|2450883|41132|725050|5768|21239|41132|725050|5768|21239|1|44|7|5|3085|146|293|97|91.46|231.39|189.73|4041.02|18403.81|8871.62|22444.83|1104.22|0.00|8304.17|18403.81|19508.03|26707.98|27812.20|9532.19| +2450820|10584|2450833|41132|725050|5768|21239|41132|725050|5768|21239|1|41|15|2|10084|65|293|74|6.03|11.99|9.83|159.84|727.42|446.22|887.26|65.46|0.00|203.50|727.42|792.88|930.92|996.38|281.20| +2450820|23056|2450828|38096|1290146|2996|246|38096|1290146|2996|246|4|63|10|3|9454|271|294|36|76.76|175.78|86.13|3227.40|3100.68|2763.36|6328.08|26.04|496.10|2278.08|2604.58|2630.62|4882.66|4908.70|-158.78| +2450820|23056|2450833|38096|1290146|2996|246|38096|1290146|2996|246|4|2|7|3|7639|276|294|70|47.23|65.64|28.22|2619.40|1975.40|3306.10|4594.80|59.26|0.00|1745.80|1975.40|2034.66|3721.20|3780.46|-1330.70| +2450820|23056|2450856|38096|1290146|2996|246|38096|1290146|2996|246|4|77|12|5|6679|246|294|13|88.36|150.21|16.52|1737.97|214.76|1148.68|1952.73|8.59|0.00|663.91|214.76|223.35|878.67|887.26|-933.92| +2450820|23056|2450833|38096|1290146|2996|246|38096|1290146|2996|246|4|25|16|4|5342|127|294|67|33.31|95.93|71.94|1607.33|4819.98|2231.77|6427.31|192.79|0.00|1027.78|4819.98|5012.77|5847.76|6040.55|2588.21| +2450820|23056|2450879|38096|1290146|2996|246|38096|1290146|2996|246|4|34|18|2|7453|259|294|37|26.22|46.93|28.62|677.47|1058.94|970.14|1736.41|13.34|614.18|260.11|444.76|458.10|704.87|718.21|-525.38| +2450820|23056|2450890|38096|1290146|2996|246|38096|1290146|2996|246|4|7|5|3|17926|155|294|17|72.00|200.88|58.25|2424.71|990.25|1224.00|3414.96|29.70|0.00|717.06|990.25|1019.95|1707.31|1737.01|-233.75| +2450820|23056|2450852|38096|1290146|2996|246|38096|1290146|2996|246|4|78|5|3|4432|37|294|57|66.66|173.98|40.01|7636.29|2280.57|3799.62|9916.86|91.22|0.00|396.15|2280.57|2371.79|2676.72|2767.94|-1519.05| +2450820|23056|2450879|38096|1290146|2996|246|38096|1290146|2996|246|4|10|10|1|12026|90|294|37|6.49|15.96|4.78|413.66|176.86|240.13|590.52|0.84|134.41|46.99|42.45|43.29|89.44|90.28|-197.68| +2450820|23056|2450862|38096|1290146|2996|246|38096|1290146|2996|246|4|46|14|2|12571|138|294|50|71.09|125.11|12.51|5630.00|625.50|3554.50|6255.50|37.53|0.00|1313.50|625.50|663.03|1939.00|1976.53|-2929.00| +2450820|23056|2450889|38096|1290146|2996|246|38096|1290146|2996|246|4|73|14|2|12481|144|294|41|97.54|272.13|125.17|6025.36|5131.97|3999.14|11157.33|359.23|0.00|5578.46|5131.97|5491.20|10710.43|11069.66|1132.83| +2450820|23056|2450834|38096|1290146|2996|246|38096|1290146|2996|246|4|6|1|1|8971|268|294|94|61.75|74.71|14.19|5688.88|1333.86|5804.50|7022.74|120.04|0.00|140.06|1333.86|1453.90|1473.92|1593.96|-4470.64| +2450820|39585|2450831|1360|1320662|5564|20866|70567|945070|4019|30521|4|93|8|1|13465|225|295|47|44.41|62.17|11.19|2396.06|525.93|2087.27|2921.99|30.60|15.77|963.97|510.16|540.76|1474.13|1504.73|-1577.11| +2450820|39585|2450866|1360|1320662|5564|20866|70567|945070|4019|30521|4|5|10|2|3338|88|295|41|10.31|12.99|0.51|511.68|20.91|422.71|532.59|1.04|0.00|116.85|20.91|21.95|137.76|138.80|-401.80| +2450820|39585|2450904|1360|1320662|5564|20866|70567|945070|4019|30521|4|83|2|2|16180|96|295|5|75.91|148.78|71.41|386.85|357.05|379.55|743.90|32.13|0.00|141.30|357.05|389.18|498.35|530.48|-22.50| +2450820|39585|2450844|1360|1320662|5564|20866|70567|945070|4019|30521|4|81|18|1|16885|9|295|30|33.69|63.67|36.29|821.40|1088.70|1010.70|1910.10|21.77|0.00|725.70|1088.70|1110.47|1814.40|1836.17|78.00| +2450820|39585|2450880|1360|1320662|5564|20866|70567|945070|4019|30521|4|103|16|5|2302|298|295|50|34.12|69.60|18.79|2540.50|939.50|1706.00|3480.00|9.39|0.00|278.00|939.50|948.89|1217.50|1226.89|-766.50| +|40430||78261|551175|4889||||4889|||90|||2692|183|296|57|17.49|48.62||637.83||996.93|2771.34|||1191.30|618.72||1810.02|1810.02|-378.21| +2450820|40430|2450891|78261|551175|4889|38603|78261|551175|4889|38603|1|106|17|5|7849|89|296|62|49.41|69.66|59.90|605.12|3713.80|3063.42|4318.92|44.19|3082.45|1943.08|631.35|675.54|2574.43|2618.62|-2432.07| +2450820|40430|2450844|78261|551175|4889|38603|78261|551175|4889|38603|1|52|16|4|392|79|296|5|46.99|136.74|64.26|362.40|321.30|234.95|683.70|3.21|0.00|198.25|321.30|324.51|519.55|522.76|86.35| +2450820|40430|2450826|78261|551175|4889|38603|78261|551175|4889|38603|1|62|1|2|14965|117|296|92|19.00|19.00|7.98|1013.84|734.16|1748.00|1748.00|14.68|0.00|699.20|734.16|748.84|1433.36|1448.04|-1013.84| +2450820|40430|2450844|78261|551175|4889|38603|78261|551175|4889|38603|1|31|11|1|4273|140|296|27|38.51|82.79|13.24|1877.85|357.48|1039.77|2235.33|25.02|0.00|514.08|357.48|382.50|871.56|896.58|-682.29| +2450820|40430|2450893|78261|551175|4889|38603|78261|551175|4889|38603|1|1|4|5|17893|257|296|65|81.42|128.64|27.01|6605.95|1755.65|5292.30|8361.60|35.11|0.00|2173.60|1755.65|1790.76|3929.25|3964.36|-3536.65| +2450820|40430|2450889|78261|551175|4889|38603|78261|551175|4889|38603|1|2|20|5|5960|282|296|23|70.54|104.39|28.18|1752.83|648.14|1622.42|2400.97|32.40|0.00|648.14|648.14|680.54|1296.28|1328.68|-974.28| +2450820|66646|2450898|73320|1246531|6942|22153|73320|1246531|6942|22153|4|93|2|3|6830|3|297|36|69.97|89.56|68.06|774.00|2450.16|2518.92|3224.16|24.50|0.00|966.96|2450.16|2474.66|3417.12|3441.62|-68.76| +2450820|66646|2450841|73320|1246531|6942|22153|73320|1246531|6942|22153|4|59|19|5|1642|139|297|41|12.54|22.44|9.87|515.37|404.67|514.14|920.04|4.04|0.00|321.85|404.67|408.71|726.52|730.56|-109.47| +2450820|66646|2450877|73320|1246531|6942|22153|73320|1246531|6942|22153|4|101|3|5|12866|152|297|25|20.06|20.06|8.42|291.00|210.50|501.50|501.50|6.31|0.00|10.00|210.50|216.81|220.50|226.81|-291.00| +2450820|66646|2450902|73320|1246531|6942|22153|73320|1246531|6942|22153|4|35|15|1|14092|36|297|43|34.92|60.76|37.06|1019.10|1593.58|1501.56|2612.68|143.42|0.00|287.24|1593.58|1737.00|1880.82|2024.24|92.02| +2450820|66646|2450859|73320|1246531|6942|22153|73320|1246531|6942|22153|4|56|9|5|2914|215|297|87|94.92|112.95|50.82|5405.31|4421.34|8258.04|9826.65|309.49|0.00|1277.16|4421.34|4730.83|5698.50|6007.99|-3836.70| +2450820|66646|2450903|73320|1246531|6942|22153|73320|1246531|6942|22153|4|93|6|2|13837|102|297|47|31.02|54.90|7.68|2219.34|360.96|1457.94|2580.30|3.60|0.00|851.17|360.96|364.56|1212.13|1215.73|-1096.98| +2450820|66646|2450905|73320|1246531|6942|22153|73320|1246531|6942|22153|4|54|2|1|13220|74|297|92|51.53|98.93|42.53|5188.80|3912.76|4740.76|9101.56|0.00|0.00|272.32|3912.76|3912.76|4185.08|4185.08|-828.00| +2450820|66646|2450828|73320|1246531|6942|22153|73320|1246531|6942|22153|4|81|5|4|13940|217|297|96|25.33|57.75|33.49|2328.96|3215.04|2431.68|5544.00|225.05|0.00|1774.08|3215.04|3440.09|4989.12|5214.17|783.36| +2450820|66646|2450864|73320|1246531|6942|22153|73320|1246531|6942|22153|4|65|7|4|11378|175|297|100|23.89|30.34|4.55|2579.00|455.00|2389.00|3034.00|30.71|113.75|424.00|341.25|371.96|765.25|795.96|-2047.75| +2450820|66646|2450874|73320|1246531|6942|22153|73320|1246531|6942|22153|4|12|10|4|4366|90|297|21|88.28|255.12|51.02|4286.10|1071.42|1853.88|5357.52|0.00|0.00|1875.09|1071.42|1071.42|2946.51|2946.51|-782.46| +2450820|66646|2450822|73320|1246531|6942|22153|73320|1246531|6942|22153|4|55|17|3|3646|242|297|24|67.31|175.00|126.00|1176.00|3024.00|1615.44|4200.00|90.72|0.00|126.00|3024.00|3114.72|3150.00|3240.72|1408.56| +2450820|66646|2450822|73320|1246531|6942|22153|73320|1246531|6942|22153|4|23|15|4|14260|110|297|29|67.02|102.54|0.00|2973.66|0.00|1943.58|2973.66|0.00|0.00|178.35|0.00|0.00|178.35|178.35|-1943.58| +2450820|23551|2450845|77718|319293|68|10843|77718|319293|68|10843|2|58|11|2|5299|56|298|21|99.19|228.13|52.46|3689.07|1101.66|2082.99|4790.73|66.09|0.00|383.25|1101.66|1167.75|1484.91|1551.00|-981.33| +2450820|23551|2450885|77718|319293|68|10843|77718|319293|68|10843|2|1|13|5|17878|110|298|86|13.06|24.81|2.23|1941.88|191.78|1123.16|2133.66|17.26|0.00|191.78|191.78|209.04|383.56|400.82|-931.38| +2450820|23551|2450879|77718|319293|68|10843|77718|319293|68|10843|2|83|2|2|9890|17|298|72|97.68|108.42|100.83|546.48|7259.76|7032.96|7806.24|508.18|0.00|3044.16|7259.76|7767.94|10303.92|10812.10|226.80| +2450820|23551|2450868|77718|319293|68|10843|77718|319293|68|10843|2|43|14|5|13057|110|298|2|18.79|40.96|29.08|23.76|58.16|37.58|81.92|1.16|0.00|7.36|58.16|59.32|65.52|66.68|20.58| +2450820|23551|2450870|77718|319293|68|10843|77718|319293|68|10843|2|8|10|2|2396|24|298|35|21.89|35.02|14.35|723.45|502.25|766.15|1225.70|0.00|0.00|294.00|502.25|502.25|796.25|796.25|-263.90| +2450820|23551|2450833|77718|319293|68|10843|77718|319293|68|10843|2|1|4|3|11725|233|298|87|20.61|51.73|48.62|270.57|4229.94|1793.07|4500.51|211.49|0.00|1079.67|4229.94|4441.43|5309.61|5521.10|2436.87| +2450820|23551|2450873|77718|319293|68|10843|77718|319293|68|10843|2|78|16|5|17984|245|298|20|37.94|65.63|42.00|472.60|840.00|758.80|1312.60|42.00|0.00|210.00|840.00|882.00|1050.00|1092.00|81.20| +2450820|23551|2450879|77718|319293|68|10843|77718|319293|68|10843|2|80|7|1|14288|114|298|85|90.37|136.45|136.45|0.00|11598.25|7681.45|11598.25|463.93|0.00|2899.35|11598.25|12062.18|14497.60|14961.53|3916.80| +2450820|23551|2450901|77718|319293|68|10843|77718|319293|68|10843|2|76|8|2|700|78|298|43|75.38|174.88|150.39|1053.07|6466.77|3241.34|7519.84|258.67|0.00|3684.67|6466.77|6725.44|10151.44|10410.11|3225.43| +2450820|23551|2450883|77718|319293|68|10843|77718|319293|68|10843|2|47|12|2|10201|57|298|38|19.14|54.16|51.45|102.98|1955.10|727.32|2058.08|39.10|0.00|782.04|1955.10|1994.20|2737.14|2776.24|1227.78| +2450820|23551|2450898|77718|319293|68|10843|77718|319293|68|10843|2|52|3|5|7885|209|298|17|94.08|153.35|85.87|1147.16|1459.79|1599.36|2606.95|58.39|0.00|573.41|1459.79|1518.18|2033.20|2091.59|-139.57| +2450820|74498|2450901|81187|203761|1088|45262|81187|203761|1088|45262|2|108|8|1|5326|57|299|38|57.83|97.73|45.93|1968.40|1745.34|2197.54|3713.74|157.08|0.00|1151.02|1745.34|1902.42|2896.36|3053.44|-452.20| +2450820|74498|2450910|81187|203761|1088|45262|81187|203761|1088|45262|2|71|17|3|16790|98|299|17|78.30|223.93|109.72|1941.57|1865.24|1331.10|3806.81|18.65|0.00|75.99|1865.24|1883.89|1941.23|1959.88|534.14| +2450820|74498|2450865|81187|203761|1088|45262|81187|203761|1088|45262|2|30|7|1|12644|89|299|56|71.51|193.79|93.01|5643.68|5208.56|4004.56|10852.24|260.42|0.00|2495.92|5208.56|5468.98|7704.48|7964.90|1204.00| +2450820|74498|2450903|81187|203761|1088|45262|81187|203761|1088|45262|2|83|5|1|5635|279|299|43|51.45|71.00|4.97|2839.29|213.71|2212.35|3053.00|10.68|0.00|702.19|213.71|224.39|915.90|926.58|-1998.64| +2450820|74498|2450898|81187|203761|1088|45262|81187|203761|1088|45262|2|67|9|5|15110|264|299|18|58.75|112.80|23.68|1604.16|426.24|1057.50|2030.40|34.09|0.00|223.20|426.24|460.33|649.44|683.53|-631.26| +2450820|74498|2450867|81187|203761|1088|45262|81187|203761|1088|45262|2|20|17|5|5170|169|299|98|98.25|101.19|0.00|9916.62|0.00|9628.50|9916.62|0.00|0.00|1784.58|0.00|0.00|1784.58|1784.58|-9628.50| +2450820|74498|2450873|81187|203761|1088|45262|81187|203761|1088|45262|2|80|11|1|4508|217|299|63|14.92|42.96|7.73|2219.49|486.99|939.96|2706.48|43.82|0.00|243.18|486.99|530.81|730.17|773.99|-452.97| +2450820|25334|2450832|33387|1261293|2799|23105|33387|1261293|2799|23105|4|86|6|5|7933|119|300|20|27.49|34.91|33.86|21.00|677.20|549.80|698.20|47.40|0.00|118.60|677.20|724.60|795.80|843.20|127.40| +2450820|25334|2450882|33387|1261293|2799|23105|33387|1261293|2799|23105|4|71|6|3|12920|105|300|100|43.52|78.33|73.63|470.00|7363.00|4352.00|7833.00|73.63|0.00|2036.00|7363.00|7436.63|9399.00|9472.63|3011.00| +2450820|25334|2450909|33387|1261293|2799|23105|33387|1261293|2799|23105|4|18|19|3|8626|219|300|69|86.55|132.42|97.99|2375.67|6761.31|5971.95|9136.98|0.00|0.00|456.78|6761.31|6761.31|7218.09|7218.09|789.36| +2450820|25334|2450904|33387|1261293|2799|23105|33387|1261293|2799|23105|4|103|14|3|3326|22|300|88|98.04|147.06|141.17|518.32|12422.96|8627.52|12941.28|745.37|0.00|4011.04|12422.96|13168.33|16434.00|17179.37|3795.44| +2450820|25334|2450853|33387|1261293|2799|23105|33387|1261293|2799|23105|4|17|1|3|16333|32|300|34|95.36|112.52|13.50|3366.68|459.00|3242.24|3825.68|4.86|215.73|879.58|243.27|248.13|1122.85|1127.71|-2998.97| +2450820|25334|2450906|33387|1261293|2799|23105|33387|1261293|2799|23105|4|23|17|3|4730|148|300|89|24.11|64.61|60.73|345.32|5404.97|2145.79|5750.29|162.14|0.00|862.41|5404.97|5567.11|6267.38|6429.52|3259.18| +2450820|25334|2450867|33387|1261293|2799|23105|33387|1261293|2799|23105|4|60|14|5|12973|92|300|61|83.15|108.09|12.97|5802.32|791.17|5072.15|6593.49|6.17|688.31|2439.39|102.86|109.03|2542.25|2548.42|-4969.29| +2450820|25334|2450896|33387|1261293|2799|23105|33387|1261293|2799|23105|4|31|16|2|3580|24|300|75|12.48|17.84|11.59|468.75|869.25|936.00|1338.00|69.54|0.00|521.25|869.25|938.79|1390.50|1460.04|-66.75| +2450820|43771|2450843|61396|429891|6739|20034|61396|429891|6739|20034|1|82|12|4|14854|65|301|62|26.16|55.72|23.95|1969.74|1484.90|1621.92|3454.64|42.31|74.24|655.96|1410.66|1452.97|2066.62|2108.93|-211.26| +2450820|43771|2450887|61396|429891|6739|20034|61396|429891|6739|20034|1|18|17|3|10568|299|301|77|63.53|114.35|0.00|8804.95|0.00|4891.81|8804.95|0.00|0.00|2200.66|0.00|0.00|2200.66|2200.66|-4891.81| +2450820|43771|2450841|61396|429891|6739|20034|61396|429891|6739|20034|1|82|18|2|16010|12|301|62|55.18|98.22|36.34|3836.56|2253.08|3421.16|6089.64|157.71|0.00|2435.36|2253.08|2410.79|4688.44|4846.15|-1168.08| +2450820|43771|2450879|61396|429891|6739|20034|61396|429891|6739|20034|1|97|12|2|14920|104|301|14|34.65|71.72|28.68|602.56|401.52|485.10|1004.08|36.13|0.00|341.32|401.52|437.65|742.84|778.97|-83.58| +2450820|43771|2450878|61396|429891|6739|20034|61396|429891|6739|20034|1|76|2|3|17438|55|301|84|82.30|246.90|143.20|8710.80|12028.80|6913.20|20739.60|240.57|0.00|621.60|12028.80|12269.37|12650.40|12890.97|5115.60| +2450820|43771|2450889|61396|429891|6739|20034|61396|429891|6739|20034|1|18|3|5|9020|175|301|63|8.71|16.28|11.39|308.07|717.57|548.73|1025.64|43.05|0.00|91.98|717.57|760.62|809.55|852.60|168.84| +2450820|43771|2450857|61396|429891|6739|20034|61396|429891|6739|20034|1|95|2|3|14924|33|301|4|12.14|28.89|21.37|30.08|85.48|48.56|115.56|6.83|0.00|16.16|85.48|92.31|101.64|108.47|36.92| +2450820|43771|2450897|61396|429891|6739|20034|61396|429891|6739|20034|1|86|18|5|9775|244|301|98|33.10|91.68|25.67|6468.98|2515.66|3243.80|8984.64|100.62|0.00|179.34|2515.66|2616.28|2695.00|2795.62|-728.14| +2450820|43771|2450902|61396|429891|6739|20034|61396|429891|6739|20034|1|88|4|3|4021|158|301|23|40.70|49.24|41.36|181.24|951.28|936.10|1132.52|9.32|818.10|452.87|133.18|142.50|586.05|595.37|-802.92| +2450820|49841|2450827|67190|1584943|1543|37296|67190|1584943|1543|37296|1|52|17|2|13076|114|302|63|17.84|52.98|9.00|2770.74|567.00|1123.92|3337.74|30.84|181.44|466.83|385.56|416.40|852.39|883.23|-738.36| +2450820|49841|2450883|67190|1584943|1543|37296|67190|1584943|1543|37296|1|66|14|3|1057|264|302|11|31.87|63.74|13.38|553.96|147.18|350.57|701.14|4.41|0.00|308.44|147.18|151.59|455.62|460.03|-203.39| +2450820|49841|2450907|67190|1584943|1543|37296|67190|1584943|1543|37296|1|14|12|1|3805|236|302|69|9.24|21.15|12.90|569.25|890.10|637.56|1459.35|17.80|0.00|364.32|890.10|907.90|1254.42|1272.22|252.54| +2450820|49841|2450857|67190|1584943|1543|37296|67190|1584943|1543|37296|1|13|8|3|7018|206|302|38|59.10|70.92|36.16|1320.88|1374.08|2245.80|2694.96|96.18|0.00|430.92|1374.08|1470.26|1805.00|1901.18|-871.72| +2450820|49841|2450863|67190|1584943|1543|37296|67190|1584943|1543|37296|1|40|9|3|5584|96|302|11|49.33|129.24|91.76|412.28|1009.36|542.63|1421.64|59.35|161.49|597.08|847.87|907.22|1444.95|1504.30|305.24| +2450820|49841|2450876|67190|1584943|1543|37296|67190|1584943|1543|37296|1|88|8|5|11794|76|302|19|68.78|123.11|72.63|959.12|1379.97|1306.82|2339.09|0.00|607.18|233.89|772.79|772.79|1006.68|1006.68|-534.03| +2450820|49841|2450833|67190|1584943|1543|37296|67190|1584943|1543|37296|1|96|1|5|11107|214|302|17|84.90|212.25|82.77|2201.16|1407.09|1443.30|3608.25|56.28|0.00|938.06|1407.09|1463.37|2345.15|2401.43|-36.21| +2450820|49841|2450830|67190|1584943|1543|37296|67190|1584943|1543|37296|1|95|8|1|17300|195|302|83|51.27|62.03|12.40|4119.29|1029.20|4255.41|5148.49|41.16|0.00|0.00|1029.20|1070.36|1029.20|1070.36|-3226.21| +2450820|42485|2450894|56296|1177356|4016|3076|56296|1177356|4016|3076|2|47|17|3|6799|65|303|68|93.93|216.97|119.33|6639.52|8114.44|6387.24|14753.96|649.15|0.00|7229.08|8114.44|8763.59|15343.52|15992.67|1727.20| +2450820|42485|2450845|56296|1177356|4016|3076|56296|1177356|4016|3076|2|69|18|1|6836|20|303|6|77.85|191.51|84.26|643.50|505.56|467.10|1149.06|0.00|0.00|471.06|505.56|505.56|976.62|976.62|38.46| +2450820|42485|2450902|56296|1177356|4016|3076|56296|1177356|4016|3076|2|32|20|1|8941|287|303|5|52.09|63.54|22.23|206.55|111.15|260.45|317.70|8.89|0.00|88.95|111.15|120.04|200.10|208.99|-149.30| +2450820|42485|2450841|56296|1177356|4016|3076|56296|1177356|4016|3076|2|85|13|4|9554|228|303|13|95.48|188.09|135.42|684.71|1760.46|1241.24|2445.17|158.44|0.00|146.64|1760.46|1918.90|1907.10|2065.54|519.22| +2450820|42485|2450834|56296|1177356|4016|3076|56296|1177356|4016|3076|2|4|8|1|7957|127|303|38|50.04|53.04|29.17|907.06|1108.46|1901.52|2015.52|0.99|1075.20|685.14|33.26|34.25|718.40|719.39|-1868.26| +2450820|42485|2450841|56296|1177356|4016|3076|56296|1177356|4016|3076|2|39|16|4|4801|178|303|36|6.91|18.93|12.30|238.68|442.80|248.76|681.48|12.22|137.26|61.20|305.54|317.76|366.74|378.96|56.78| +2450820|42485|2450857|56296|1177356|4016|3076|56296|1177356|4016|3076|2|16|2|4|7898|135|303|18|91.89|151.61|69.74|1473.66|1255.32|1654.02|2728.98|75.31|0.00|436.50|1255.32|1330.63|1691.82|1767.13|-398.70| +2450820|42485|2450859|56296|1177356|4016|3076|56296|1177356|4016|3076|2|74|2|1|8858|214|303|92|33.53|87.17|53.17|3128.00|4891.64|3084.76|8019.64|303.77|1516.40|1362.52|3375.24|3679.01|4737.76|5041.53|290.48| +2450820|42485|2450837|56296|1177356|4016|3076|56296|1177356|4016|3076|2|65|11|4|1747|260|303|40|2.20|3.56|1.31|90.00|52.40|88.00|142.40|1.35|29.86|9.60|22.54|23.89|32.14|33.49|-65.46| +2450820||2450898|56296|1177356||3076|56296|1177356|4016||2||||11798|165|303|47|68.97|157.25|119.51|1773.78|5616.97|3241.59|||0.00||||6947.07|7003.23|2375.38| +2450820|42485|2450897|56296|1177356|4016|3076|56296|1177356|4016|3076|2|52|14|3|697|3|303|70|93.51|225.35|105.91|8360.80|7413.70|6545.70|15774.50|148.27|0.00|3943.10|7413.70|7561.97|11356.80|11505.07|868.00| +2450820|42485|2450895|56296|1177356|4016|3076|56296|1177356|4016|3076|2|13|1|4|6694|198|303|65|18.19|21.82|5.89|1035.45|382.85|1182.35|1418.30|15.62|187.59|439.40|195.26|210.88|634.66|650.28|-987.09| +2450820|42485|2450884|56296|1177356|4016|3076|56296|1177356|4016|3076|2|39|13|3|2942|184|303|48|78.32|191.10|34.39|7522.08|1650.72|3759.36|9172.80|33.01|0.00|2017.92|1650.72|1683.73|3668.64|3701.65|-2108.64| +2450820|42485|2450868|56296|1177356|4016|3076|56296|1177356|4016|3076|2|66|20|5|7291|187|303|5|97.72|236.48|205.73|153.75|1028.65|488.60|1182.40|10.28|0.00|425.65|1028.65|1038.93|1454.30|1464.58|540.05| +2450820|7403|2450856|46159|1722970|2628|17039|46159|1722970|2628|17039|4|64|7|3|15295|125|304|57|7.73|10.28|4.11|351.69|234.27|440.61|585.96|4.68|0.00|257.64|234.27|238.95|491.91|496.59|-206.34| +2450820|7403|2450883|46159|1722970|2628|17039|46159|1722970|2628|17039|4|98|10|1|6946|298|304|49|50.88|95.65|74.60|1031.45|3655.40|2493.12|4686.85|109.66|0.00|1358.77|3655.40|3765.06|5014.17|5123.83|1162.28| +2450820|7403|2450884|46159|1722970|2628|17039|46159|1722970|2628|17039|4|87|13|2|9838|30|304|33|25.93|35.78|25.40|342.54|838.20|855.69|1180.74|8.38|0.00|11.55|838.20|846.58|849.75|858.13|-17.49| +2450820|7403|2450863|46159|1722970|2628|17039|46159|1722970|2628|17039|4|86|20|3|313|296|304|86|58.74|144.50|72.25|6213.50|6213.50|5051.64|12427.00|62.13|0.00|3728.10|6213.50|6275.63|9941.60|10003.73|1161.86| +2450820|7403|2450856|46159|1722970|2628|17039|46159|1722970|2628|17039|4|66|19|5|13034|68|304|24|71.09|154.97|9.29|3496.32|222.96|1706.16|3719.28|17.83|0.00|1822.32|222.96|240.79|2045.28|2063.11|-1483.20| +2450820|7403|2450880|46159|1722970|2628|17039|46159|1722970|2628|17039|4|91|3|2|16993|16|304|61|2.21|3.46|1.83|99.43|111.63|134.81|211.06|0.00|0.00|81.74|111.63|111.63|193.37|193.37|-23.18| +2450820|7403|2450861|46159|1722970|2628|17039|46159|1722970|2628|17039|4|72|10|2|7834|203|304|44|51.83|74.63|6.71|2988.48|295.24|2280.52|3283.72|10.62|29.52|1116.28|265.72|276.34|1382.00|1392.62|-2014.80| +2450820|7403|2450874|46159|1722970|2628|17039|46159|1722970|2628|17039|4|3|6|3|11320|297|304|39|60.24|125.29|72.66|2052.57|2833.74|2349.36|4886.31|141.68|0.00|2149.68|2833.74|2975.42|4983.42|5125.10|484.38| +2450820|7403|2450865|46159|1722970|2628|17039|46159|1722970|2628|17039|4|20|10|3|6226|101|304|5|87.23|111.65|107.18|22.35|535.90|436.15|558.25|10.71|0.00|55.80|535.90|546.61|591.70|602.41|99.75| +2450820|7403|2450822|46159|1722970|2628|17039|46159|1722970|2628|17039|4|81|2|4|15010|20|304|70|30.12|73.49|23.51|3498.60|1645.70|2108.40|5144.30|65.82|0.00|771.40|1645.70|1711.52|2417.10|2482.92|-462.70| +2450820|65392|2450909|87867|1728955|2396|25084|87867|1728955|2396|25084|4|13|17|1|8509|47|305|84|55.78|111.00|51.06|5034.96|4289.04|4685.52|9324.00|171.56|0.00|4009.32|4289.04|4460.60|8298.36|8469.92|-396.48| +2450820|65392|2450866|87867|1728955|2396|25084|87867|1728955|2396|25084|4|22|3|1|11530|100|305|44|51.08|100.11|41.04|2599.08|1805.76|2247.52|4404.84|112.13|559.78|1277.32|1245.98|1358.11|2523.30|2635.43|-1001.54| +2450820|65392|2450906|87867|1728955|2396|25084|87867|1728955|2396|25084|4|82|4|5|4436|247|305|22|62.76|185.76|105.88|1757.36|2329.36|1380.72|4086.72|23.29|0.00|1266.76|2329.36|2352.65|3596.12|3619.41|948.64| +2450820|65392|2450871|87867|1728955|2396|25084|87867|1728955|2396|25084|4|98|8|1|17461|124|305|6|12.17|25.55|22.99|15.36|137.94|73.02|153.30|1.37|0.00|38.28|137.94|139.31|176.22|177.59|64.92| +2450820|22752|2450897|53252|1900823|7147|1918|53252|1900823|7147|1918|4|49|2|4|16693|102|306|13|98.66|284.14|241.51|554.19|3139.63|1282.58|3693.82|125.58|0.00|1662.18|3139.63|3265.21|4801.81|4927.39|1857.05| +2450820|22752|2450897|53252|1900823|7147|1918|53252|1900823|7147|1918|4|29|8|2|7274|250|306|16|98.37|239.03|100.39|2218.24|1606.24|1573.92|3824.48|16.06|0.00|191.20|1606.24|1622.30|1797.44|1813.50|32.32| +2450820|22752|2450866|53252|1900823|7147|1918|53252|1900823|7147|1918|4|15|12|1|5290|192|306|21|96.83|136.53|57.34|1662.99|1204.14|2033.43|2867.13|72.24|0.00|1060.71|1204.14|1276.38|2264.85|2337.09|-829.29| +2450820|22752|2450844|53252|1900823|7147|1918|53252|1900823|7147|1918|4|13|9|3|10759|299|306|70|40.08|72.54|15.23|4011.70|1066.10|2805.60|5077.80|63.96|0.00|1116.50|1066.10|1130.06|2182.60|2246.56|-1739.50| +2450820|22752|2450880|53252|1900823|7147|1918|53252|1900823|7147|1918|4|98|1|3|2200|10|306|16|75.20|89.48|68.00|343.68|1088.00|1203.20|1431.68|10.88|0.00|443.68|1088.00|1098.88|1531.68|1542.56|-115.20| +2450820|62504|2450852|84662|510238|3014|36637|84662|510238|3014|36637|4|10|4|4|16738|104|307|60|16.05|41.56|29.50|723.60|1770.00|963.00|2493.60|123.90|0.00|149.40|1770.00|1893.90|1919.40|2043.30|807.00| +2450820|62504|2450823|84662|510238|3014|36637|84662|510238|3014|36637|4|85|3|4|14176|61|307|36|69.42|127.03|33.02|3384.36|1188.72|2499.12|4573.08|106.98|0.00|777.24|1188.72|1295.70|1965.96|2072.94|-1310.40| +2450820|62504|2450880|84662|510238|3014|36637|84662|510238|3014|36637|4|19|12|4|9146|106|307|45|95.17|206.51|0.00|9292.95|0.00|4282.65|9292.95|0.00|0.00|1300.95|0.00|0.00|1300.95|1300.95|-4282.65| +2450820|62504|2450897|84662|510238|3014|36637|84662|510238|3014|36637|4|25|15|2|4381|111|307|67|65.06|172.40|31.03|9471.79|2079.01|4359.02|11550.80|68.19|1226.61|3349.33|852.40|920.59|4201.73|4269.92|-3506.62| +2450820|62504|2450832|84662|510238|3014|36637|84662|510238|3014|36637|4|106|13|5|14816|86|307|48|44.37|73.21|41.72|1511.52|2002.56|2129.76|3514.08|31.64|420.53|456.48|1582.03|1613.67|2038.51|2070.15|-547.73| +2450821|66839|2450864|22956|1318645|5087|8329|22956|1318645|5087|8329|4|103|8|2|7706|55|308|55|48.87|129.01|1.29|7024.60|70.95|2687.85|7095.55|1.10|34.05|0.00|36.90|38.00|36.90|38.00|-2650.95| +2450821|66839|2450869|22956|1318645|5087|8329|22956|1318645|5087|8329|4|36|4|4|12034|12|308|40|70.36|163.23|81.61|3264.80|3264.40|2814.40|6529.20|293.79|0.00|3003.20|3264.40|3558.19|6267.60|6561.39|450.00| +2450821|66839|2450891|22956|1318645|5087|8329|22956|1318645|5087|8329|4|44|8|2|490|125|308|95|53.89|160.05|83.22|7298.85|7905.90|5119.55|15204.75|0.00|5138.83|912.00|2767.07|2767.07|3679.07|3679.07|-2352.48| +2450821|66839|2450840|22956|1318645|5087|8329|22956|1318645|5087|8329|4|104|16|4|374|160|308|55|64.58|185.99|172.97|716.10|9513.35|3551.90|10229.45|0.00|0.00|3682.25|9513.35|9513.35|13195.60|13195.60|5961.45| +2450821|66839|2450848|22956|1318645|5087|8329|22956|1318645|5087|8329|4|21|4|1|439|222|308|80|46.54|114.95|71.26|3495.20|5700.80|3723.20|9196.00|342.04|0.00|1563.20|5700.80|6042.84|7264.00|7606.04|1977.60| +2450821|64411|2450851|51251|343390|4329|2515|51251|343390|4329|2515|2|19|3|5|14911|54|309|67|35.23|67.64|2.02|4396.54|135.34|2360.41|4531.88|1.86|41.95|1766.79|93.39|95.25|1860.18|1862.04|-2267.02| +2450821|64411|2450828|51251|343390|4329|2515|51251|343390|4329|2515|2|7|13|5|12194|204|309|22|87.14|158.59|30.13|2826.12|662.86|1917.08|3488.98|53.02|0.00|453.42|662.86|715.88|1116.28|1169.30|-1254.22| +2450821|64411|2450891|51251|343390|4329|2515|51251|343390|4329|2515|2|74|3|3|3994|297|309|25|69.15|150.74|63.31|2185.75|1582.75|1728.75|3768.50|47.48|0.00|414.50|1582.75|1630.23|1997.25|2044.73|-146.00| +2450821|64411|2450861|51251|343390|4329|2515|51251|343390|4329|2515|2|30|10|1|2941|296|309|28|46.52|139.09|82.06|1596.84|2297.68|1302.56|3894.52|91.90|0.00|1440.88|2297.68|2389.58|3738.56|3830.46|995.12| +2450821|64411|2450886|51251|343390|4329|2515|51251|343390|4329|2515|2|27|17|5|3664|286|309|16|76.09|194.79|15.58|2867.36|249.28|1217.44|3116.64|2.49|0.00|1246.56|249.28|251.77|1495.84|1498.33|-968.16| +2450821|64411|2450839|51251|343390|4329|2515|51251|343390|4329|2515|2|17|16|5|7681|3|309|92|35.18|103.07|4.12|9103.40|379.04|3236.56|9482.44|25.58|94.76|4740.76|284.28|309.86|5025.04|5050.62|-2952.28| +2450821|64411|2450836|51251|343390|4329|2515|51251|343390|4329|2515|2|12|9|5|16792|74|309|9|44.11|73.22|0.73|652.41|6.57|396.99|658.98|0.26|0.00|296.46|6.57|6.83|303.03|303.29|-390.42| +2450821|64411|2450886|51251|343390|4329|2515|51251|343390|4329|2515|2|56|15|1|998|181|309|31|55.51|93.81|65.66|872.65|2035.46|1720.81|2908.11|183.19|0.00|1221.40|2035.46|2218.65|3256.86|3440.05|314.65| +2450821|64411|2450842|51251|343390|4329|2515|51251|343390|4329|2515|2|19|14|4|8032|300|309|68|83.69|236.84|4.73|15783.48|321.64|5690.92|16105.12|9.64|0.00|5797.68|321.64|331.28|6119.32|6128.96|-5369.28| +2450821|64411|2450885|51251|343390|4329|2515|51251|343390|4329|2515|2|25|3|4|9518|166|309|72|37.33|89.59|28.66|4386.96|2063.52|2687.76|6450.48|108.95|701.59|966.96|1361.93|1470.88|2328.89|2437.84|-1325.83| +2450821|41316|2450907|85703|1667536|3769|28802|85703|1667536|3769|28802|2|72|7|5|13522|123|310|88|15.93|46.51|15.81|2701.60|1391.28|1401.84|4092.88|13.91|0.00|449.68|1391.28|1405.19|1840.96|1854.87|-10.56| +2450821|41316|2450909|85703|1667536|3769|28802|85703|1667536|3769|28802|2|69|7|3|8701|82|310|70|68.10|123.94|74.36|3470.60|5205.20|4767.00|8675.80|52.05|0.00|259.70|5205.20|5257.25|5464.90|5516.95|438.20| +2450821|41316|2450869|85703|1667536|3769|28802|85703|1667536|3769|28802|2|98|8|2|6928|152|310|73|59.01|86.15|43.07|3144.84|3144.11|4307.73|6288.95|0.00|0.00|1383.35|3144.11|3144.11|4527.46|4527.46|-1163.62| +2450821|41316|2450873|85703|1667536|3769|28802|85703|1667536|3769|28802|2|103|16|3|4621|157|310|97|36.57|62.53|36.89|2487.08|3578.33|3547.29|6065.41|35.78|0.00|423.89|3578.33|3614.11|4002.22|4038.00|31.04| +2450821|41316|2450845|85703|1667536|3769|28802|85703|1667536|3769|28802|2|98|1|1|5935|108|310|15|87.36|172.97|148.75|363.30|2231.25|1310.40|2594.55|148.60|580.12|726.45|1651.13|1799.73|2377.58|2526.18|340.73| +2450821|41316|2450840|85703|1667536|3769|28802|85703|1667536|3769|28802|2|38|20|4|11386|104|310|50|8.69|9.73|7.58|107.50|379.00|434.50|486.50|7.58|0.00|150.50|379.00|386.58|529.50|537.08|-55.50| +2450821|41316|2450867|85703|1667536|3769|28802|85703|1667536|3769|28802|2|47|17|4|2582|119|310|27|57.47|94.25|23.56|1908.63|636.12|1551.69|2544.75|44.52|0.00|534.33|636.12|680.64|1170.45|1214.97|-915.57| +2450821|41316|2450893|85703|1667536|3769|28802|85703|1667536|3769|28802|2|72|12|2|15550|142|310|47|57.22|82.96|53.92|1364.88|2534.24|2689.34|3899.12|228.08|0.00|272.60|2534.24|2762.32|2806.84|3034.92|-155.10| +2450821|29889|2450841|36304|1101290|3876|12062|36304|1101290|3876|12062|4|53|1|4|8458|202|311|57|68.55|196.05|182.32|782.61|10392.24|3907.35|11174.85|45.72|9249.09|5140.26|1143.15|1188.87|6283.41|6329.13|-2764.20| +2450821|29889|2450833|36304|1101290|3876|12062|36304|1101290|3876|12062|4|107|10|4|7420|261|311|1|81.19|211.90|129.25|82.65|129.25|81.19|211.90|5.17|0.00|93.23|129.25|134.42|222.48|227.65|48.06| +2450821|29889|2450823|36304|1101290|3876|12062|36304|1101290|3876|12062|4|22|10|5|11212|243|311|46|83.12|235.22|103.49|6059.58|4760.54|3823.52|10820.12|142.81|0.00|4652.44|4760.54|4903.35|9412.98|9555.79|937.02| +2450821|29889|2450863|36304|1101290|3876|12062|36304|1101290|3876|12062|4|31|14|3|9692|14|311|90|98.93|98.93|38.58|5431.50|3472.20|8903.70|8903.70|173.61|0.00|2047.50|3472.20|3645.81|5519.70|5693.31|-5431.50| +2450821|29889|2450834|36304|1101290|3876|12062|36304|1101290|3876|12062|4|84|2|3|14018|235|311|38|92.23|197.37|104.60|3525.26|3974.80|3504.74|7500.06|198.74|0.00|599.64|3974.80|4173.54|4574.44|4773.18|470.06| +2450821|29889|2450903|36304|1101290|3876|12062|36304|1101290|3876|12062|4|48|11|5|2539|253|311|57|71.17|151.59|21.22|7431.09|1209.54|4056.69|8640.63|108.85|0.00|4320.03|1209.54|1318.39|5529.57|5638.42|-2847.15| +2450821|29889|2450891|36304|1101290|3876|12062|36304|1101290|3876|12062|4|11|14|1|8689|175|311|47|75.12|90.14|9.01|3813.11|423.47|3530.64|4236.58|4.23|0.00|1482.38|423.47|427.70|1905.85|1910.08|-3107.17| +2450821|29889|2450908|36304|1101290|3876|12062|36304|1101290|3876|12062|4|69|10|3|17911|166|311|92|80.09|189.01|166.32|2087.48|15301.44|7368.28|17388.92|153.01|0.00|2955.96|15301.44|15454.45|18257.40|18410.41|7933.16| +2450821|29889|2450876|36304|1101290|3876|12062|36304|1101290|3876|12062|4|30|8|3|1084|196|311|7|75.69|211.93|180.14|222.53|1260.98|529.83|1483.51|88.26|0.00|59.29|1260.98|1349.24|1320.27|1408.53|731.15| +2450821|29889|2450829|36304|1101290|3876|12062|36304|1101290|3876|12062|4|89|16|2|10297|32|311|48|36.60|61.48|50.41|531.36|2419.68|1756.80|2951.04|96.78|0.00|1091.52|2419.68|2516.46|3511.20|3607.98|662.88| +2450821|29889|2450843|36304|1101290|3876|12062|36304|1101290|3876|12062|4|97|2|4|5594|156|311|79|4.14|11.55|7.96|283.61|628.84|327.06|912.45|25.15|0.00|136.67|628.84|653.99|765.51|790.66|301.78| +2450821|29889|2450832|36304|1101290|3876|12062|36304|1101290|3876|12062|4|18|16|3|17954|27|311|96|19.80|55.44|37.69|1704.00|3618.24|1900.80|5322.24|99.50|1628.20|957.12|1990.04|2089.54|2947.16|3046.66|89.24| +2450821|43224|2450880|12901|704020|5539|43858|12901|704020|5539|43858|2|50|3|4|850|71|312|88|95.54|272.28|133.41|12220.56|11740.08|8407.52|23960.64|46.96|9392.06|478.72|2348.02|2394.98|2826.74|2873.70|-6059.50| +2450821|43224|2450887|12901|704020|5539|43858|12901|704020|5539|43858|2|37|3|5|9494|271|312|76|4.66|5.49|4.17|100.32|316.92|354.16|417.24|25.35|0.00|112.48|316.92|342.27|429.40|454.75|-37.24| +2450821|43224|2450910|12901|704020|5539|43858|12901|704020|5539|43858|2|96|6|1|5125|259|312|87|49.47|57.38|27.54|2596.08|2395.98|4303.89|4992.06|23.95|0.00|1696.50|2395.98|2419.93|4092.48|4116.43|-1907.91| +2450821|43224|2450878|12901|704020|5539|43858|12901|704020|5539|43858|2|4|5|5|12602|90|312|48|44.07|91.22|72.97|876.00|3502.56|2115.36|4378.56|175.12|0.00|744.00|3502.56|3677.68|4246.56|4421.68|1387.20| +2450821|43224|2450866|12901|704020|5539|43858|12901|704020|5539|43858|2|81|15|1|8620|230|312|11|78.22|106.37|21.27|936.10|233.97|860.42|1170.07|18.71|0.00|514.80|233.97|252.68|748.77|767.48|-626.45| +2450821|43224|2450832|12901|704020|5539|43858|12901|704020|5539|43858|2|78|11|3|15055|224|312|6|78.64|208.39|81.27|762.72|487.62|471.84|1250.34|43.88|0.00|0.00|487.62|531.50|487.62|531.50|15.78| +2450821|43224|2450869|12901|704020|5539|43858|12901|704020|5539|43858|2|87|13|3|1795|263|312|42|64.05|140.91|116.95|1006.32|4911.90|2690.10|5918.22|343.83|0.00|295.68|4911.90|5255.73|5207.58|5551.41|2221.80| +2450821|43224|2450907|12901|704020|5539|43858|12901|704020|5539|43858|2|94|13|1|16945|184|312|23|95.15|158.90|109.64|1132.98|2521.72|2188.45|3654.70|75.65|0.00|1388.74|2521.72|2597.37|3910.46|3986.11|333.27| +2450821|43224|2450826|12901|704020|5539|43858|12901|704020|5539|43858|2|105|18|3|11551|211|312|53|20.44|55.18|2.20|2807.94|116.60|1083.32|2924.54|10.49|0.00|1140.56|116.60|127.09|1257.16|1267.65|-966.72| +2450821|43224|2450875|12901|704020|5539|43858|12901|704020|5539|43858|2|66|8|2|15958|55|312|28|56.25|61.87|11.75|1403.36|329.00|1575.00|1732.36|23.03|0.00|103.88|329.00|352.03|432.88|455.91|-1246.00| +2450821|43224|2450890|12901|704020|5539|43858|12901|704020|5539|43858|2|78|15|3|15847|144|312|99|49.06|56.90|36.41|2028.51|3604.59|4856.94|5633.10|181.67|576.73|1070.19|3027.86|3209.53|4098.05|4279.72|-1829.08| +2450821|69105|2450833|53021|1243034|5047|34660|41946|1485039|4652|26218|2|86|19|1|5845|268|313|49|76.32|99.97|64.98|1714.51|3184.02|3739.68|4898.53|127.36|0.00|2302.02|3184.02|3311.38|5486.04|5613.40|-555.66| +2450821|69105|2450835|53021|1243034|5047|34660|41946|1485039|4652|26218|2|38|19|2|17521|59|313|72|41.26|99.02|14.85|6060.24|1069.20|2970.72|7129.44|8.98|620.13|1995.84|449.07|458.05|2444.91|2453.89|-2521.65| +2450821|69105|2450876|53021|1243034|5047|34660|41946|1485039|4652|26218|2|30|17|5|10981|51|313|3|9.89|10.28|8.94|4.02|26.82|29.67|30.84|0.53|0.00|5.55|26.82|27.35|32.37|32.90|-2.85| +2450821|69105|2450896|53021|1243034|5047|34660|41946|1485039|4652|26218|2|97|1|5|15796|178|313|44|64.14|135.33|129.91|238.48|5716.04|2822.16|5954.52|228.64|0.00|59.40|5716.04|5944.68|5775.44|6004.08|2893.88| +2450821|69105|2450907|53021|1243034|5047|34660|41946|1485039|4652|26218|2|47|14|4|8752|121|313|1|63.21|175.72|84.34|91.38|84.34|63.21|175.72|5.90|0.00|22.84|84.34|90.24|107.18|113.08|21.13| +2450821|69105|2450834|53021|1243034|5047|34660|41946|1485039|4652|26218|2|29|2|4|2443|205|313|50|82.76|185.38|46.34|6952.00|2317.00|4138.00|9269.00|115.85|0.00|3336.50|2317.00|2432.85|5653.50|5769.35|-1821.00| +2450821|69105|2450870|53021|1243034|5047|34660|41946|1485039|4652|26218|2|10|13|4|15380|166|313|75|98.09|173.61|95.48|5859.75|7161.00|7356.75|13020.75|572.88|0.00|6510.00|7161.00|7733.88|13671.00|14243.88|-195.75| +2450821|69105|2450830|53021|1243034|5047|34660|41946|1485039|4652|26218|2|18|12|2|8944|89|313|29|86.65|258.21|28.40|6664.49|823.60|2512.85|7488.09|23.22|436.50|2395.98|387.10|410.32|2783.08|2806.30|-2125.75| +2450821|69105|2450870|53021|1243034|5047|34660|41946|1485039|4652|26218|2|15|2|1|7213|247|313|81|8.75|25.46|23.93|123.93|1938.33|708.75|2062.26|96.91|0.00|123.12|1938.33|2035.24|2061.45|2158.36|1229.58| +2450821|69105|2450881|53021|1243034|5047|34660|41946|1485039|4652|26218|2|40|18|4|13094|164|313|99|61.45|121.05|25.42|9467.37|2516.58|6083.55|11983.95|115.76|201.32|5392.53|2315.26|2431.02|7707.79|7823.55|-3768.29| +2450821|45522|2450823|13676|959078|6080|9882|13676|959078|6080|9882|2|94|4|3|15553|44|314|75|54.68|86.39|67.38|1425.75|5053.50|4101.00|6479.25|72.77|1414.98|1813.50|3638.52|3711.29|5452.02|5524.79|-462.48| +2450821|45522|2450829|13676|959078|6080|9882|13676|959078|6080|9882|2|76|8|4|6944|257|314|86|20.48|39.73|3.97|3075.36|341.42|1761.28|3416.78|30.72|0.00|1058.66|341.42|372.14|1400.08|1430.80|-1419.86| +2450821|45522|2450908|13676|959078|6080|9882|13676|959078|6080|9882|2|14|15|2|9817|214|314|3|30.77|84.92|64.53|61.17|193.59|92.31|254.76|9.67|0.00|81.51|193.59|203.26|275.10|284.77|101.28| +2450821|45522|2450844|13676|959078|6080|9882|13676|959078|6080|9882|2|56|11|1|6172|115|314|50|53.40|59.80|13.75|2302.50|687.50|2670.00|2990.00|13.75|0.00|1046.50|687.50|701.25|1734.00|1747.75|-1982.50| +2450821|45522|2450859|13676|959078|6080|9882|13676|959078|6080|9882|2|90|13|1|10430|297|314|31|60.37|159.98|126.38|1041.60|3917.78|1871.47|4959.38|195.88|0.00|2082.89|3917.78|4113.66|6000.67|6196.55|2046.31| +2450821|45522|2450877|13676|959078|6080|9882|13676|959078|6080|9882|2|45|13|1|4078|175|314|25|82.29|199.96|39.99|3999.25|999.75|2057.25|4999.00|89.97|0.00|1399.50|999.75|1089.72|2399.25|2489.22|-1057.50| +2450821|45522|2450856|13676|959078|6080|9882|13676|959078|6080|9882|2|85|7|2|10490|267|314|94|74.47|128.08|8.96|11197.28|842.24|7000.18|12039.52|0.00|648.52|0.00|193.72|193.72|193.72|193.72|-6806.46| +2450821|45522|2450835|13676|959078|6080|9882|13676|959078|6080|9882|2|74|1|1|11770|125|314|10|42.34|44.03|8.80|352.30|88.00|423.40|440.30|7.92|0.00|88.00|88.00|95.92|176.00|183.92|-335.40| +2450821|73124|2450890|77811|500200|6589|20978|88764|594556|3345|28534|1|90|5|4|1684|254|315|72|79.98|164.75|18.12|10557.36|1304.64|5758.56|11862.00|39.13|0.00|3439.44|1304.64|1343.77|4744.08|4783.21|-4453.92| +2450821|73124|2450842|77811|500200|6589|20978|88764|594556|3345|28534|1|47|19|1|13222|128|315|75|56.76|164.03|44.28|8981.25|3321.00|4257.00|12302.25|298.89|0.00|4059.00|3321.00|3619.89|7380.00|7678.89|-936.00| +2450821|73124|2450834|77811|500200|6589|20978|88764|594556|3345|28534|1|11|14|5|17212|5|315|6|14.78|24.68|5.42|115.56|32.52|88.68|148.08|1.45|11.70|72.54|20.82|22.27|93.36|94.81|-67.86| +2450821|73124|2450870|77811|500200|6589|20978|88764|594556|3345|28534|1|43|14|5|12238|282|315|32|50.99|93.31|87.71|179.20|2806.72|1631.68|2985.92|140.33|0.00|1492.80|2806.72|2947.05|4299.52|4439.85|1175.04| +2450821|73124|2450844|77811|500200|6589|20978|88764|594556|3345|28534|1|41|11|5|10274|128|315|84|66.54|149.71|100.30|4150.44|8425.20|5589.36|12575.64|674.01|0.00|4903.92|8425.20|9099.21|13329.12|14003.13|2835.84| +2450821|73124|2450888|77811|500200|6589|20978|88764|594556|3345|28534|1|82|8|1|7102|79|315|70|66.75|184.89|122.02|4400.90|8541.40|4672.50|12942.30|0.00|0.00|1552.60|8541.40|8541.40|10094.00|10094.00|3868.90| +2450821|73124|2450896|77811|500200|6589|20978|88764|594556|3345|28534|1|97|14|5|9164|117|315|94|93.14|137.84|126.81|1036.82|11920.14|8755.16|12956.96|0.00|0.00|1295.32|11920.14|11920.14|13215.46|13215.46|3164.98| +2450821|73124|2450874|77811|500200|6589|20978|88764|594556|3345|28534|1|70|18|1|7526|234|315|27|81.61|170.56|11.93|4283.01|322.11|2203.47|4605.12|3.22|0.00|276.21|322.11|325.33|598.32|601.54|-1881.36| +2450821|73124|2450839|77811|500200|6589|20978|88764|594556|3345|28534|1|7|14|2|3098|172|315|53|77.61|211.87|144.07|3593.40|7635.71|4113.33|11229.11|305.42|0.00|3143.96|7635.71|7941.13|10779.67|11085.09|3522.38| +2450821|73124|2450908|77811|500200|6589|20978|88764|594556|3345|28534|1|21|2|2|1765|116|315|60|35.66|41.36|34.32|422.40|2059.20|2139.60|2481.60|82.36|0.00|818.40|2059.20|2141.56|2877.60|2959.96|-80.40| +2450821|73124|2450892|77811|500200|6589|20978|88764|594556|3345|28534|1|99|12|1|1321|135|315|26|42.45|104.00|60.32|1135.68|1568.32|1103.70|2704.00|125.46|0.00|1108.64|1568.32|1693.78|2676.96|2802.42|464.62| +2450821|73124|2450854|77811|500200|6589|20978|88764|594556|3345|28534|1|45|10|1|11872|243|315|84|1.30|3.36|0.36|252.00|30.24|109.20|282.24|0.17|12.39|50.40|17.85|18.02|68.25|68.42|-91.35| +2450821|73124|2450852|77811|500200|6589|20978|88764|594556|3345|28534|1|20|12|5|16322|86|315|55|48.06|95.63|10.51|4681.60|578.05|2643.30|5259.65|1.85|531.80|578.05|46.25|48.10|624.30|626.15|-2597.05| +2450821|68470|2450876|90299|578378|2904|47383|3351|620388|114|12593|4|13|15|3|12244|197|316|2|5.23|9.04|7.50|3.08|15.00|10.46|18.08|0.75|0.00|5.60|15.00|15.75|20.60|21.35|4.54| +2450821|68470|2450885|90299|578378|2904|47383|3351|620388|114|12593|4|91|4|3|13454|236|316|13|38.59|66.76|26.03|529.49|338.39|501.67|867.88|16.91|0.00|112.71|338.39|355.30|451.10|468.01|-163.28| +2450821|68470|2450869|90299|578378|2904|47383|3351|620388|114|12593|4|72|15|5|12446|125|316|92|67.47|105.92|96.38|877.68|8866.96|6207.24|9744.64|181.77|5231.50|3117.88|3635.46|3817.23|6753.34|6935.11|-2571.78| +2450821|68470|2450834|90299|578378|2904|47383|3351|620388|114|12593|4|105|19|3|8749|36|316|47|24.94|45.64|28.75|793.83|1351.25|1172.18|2145.08|108.10|0.00|42.77|1351.25|1459.35|1394.02|1502.12|179.07| +2450821|68470|2450840|90299|578378|2904|47383|3351|620388|114|12593|4|57|17|3|11440|207|316|12|43.29|51.08|43.92|85.92|527.04|519.48|612.96|47.43|0.00|239.04|527.04|574.47|766.08|813.51|7.56| +2450821|68470|2450829|90299|578378|2904|47383|3351|620388|114|12593|4|59|18|5|199|202|316|93|89.85|228.21|93.56|12522.45|8701.08|8356.05|21223.53|87.01|0.00|8701.08|8701.08|8788.09|17402.16|17489.17|345.03| +2450821|68470|2450880|90299|578378|2904|47383|3351|620388|114|12593|4|26|7|3|2498|157|316|35|34.52|40.04|32.43|266.35|1135.05|1208.20|1401.40|102.15|0.00|658.35|1135.05|1237.20|1793.40|1895.55|-73.15| +2450821|68470|2450903|90299|578378|2904|47383|3351|620388|114|12593|4|11|16|4|4207|125|316|14|31.57|77.34|68.05|130.06|952.70|441.98|1082.76|76.31|104.79|497.98|847.91|924.22|1345.89|1422.20|405.93| +2450821|68470|2450861|90299|578378|2904|47383|3351|620388|114|12593|4|7|7|1|3289|119|316|8|95.59|195.95|152.84|344.88|1222.72|764.72|1567.60|48.90|0.00|172.40|1222.72|1271.62|1395.12|1444.02|458.00| +2450821|68470|2450911|90299|578378|2904|47383|3351|620388|114|12593|4|9|19|4|11048|165|316|62|15.51|19.85|19.85|0.00|1230.70|961.62|1230.70|61.53|0.00|295.12|1230.70|1292.23|1525.82|1587.35|269.08| +2450821|68470|2450831|90299|578378|2904|47383|3351|620388|114|12593|4|5|2|2|4942|64|316|28|83.81|103.08|95.86|202.16|2684.08|2346.68|2886.24|187.88|0.00|750.40|2684.08|2871.96|3434.48|3622.36|337.40| +2450821|68470|2450834|90299|578378|2904|47383|3351|620388|114|12593|4|79|9|2|4952|108|316|44|96.87|205.36|80.09|5511.88|3523.96|4262.28|9035.84|105.71|0.00|542.08|3523.96|3629.67|4066.04|4171.75|-738.32| +2450821|68470|2450834|90299|578378|2904|47383|3351|620388|114|12593|4|2|6|1|1411|104|316|89|38.01|98.82|85.97|1143.65|7651.33|3382.89|8794.98|382.56|0.00|2286.41|7651.33|8033.89|9937.74|10320.30|4268.44| +2450821|22391|2450850|66424|1852514|6157|14200|66424|1852514|6157|14200|2|16|1|4|14936|255|317|33|40.39|48.87|40.56|274.23|1338.48|1332.87|1612.71|80.30|0.00|580.47|1338.48|1418.78|1918.95|1999.25|5.61| +2450821|22391|2450911|66424|1852514|6157|14200|66424|1852514|6157|14200|2|104|20|3|2959|87|317|59|86.93|174.72|125.79|2886.87|7421.61|5128.87|10308.48|39.33|3488.15|4844.49|3933.46|3972.79|8777.95|8817.28|-1195.41| +2450821|22391|2450827|66424|1852514|6157|14200|66424|1852514|6157|14200|2|11|20|4|13795|66|317|48|31.95|89.77|77.20|603.36|3705.60|1533.60|4308.96|7.41|2964.48|602.88|741.12|748.53|1344.00|1351.41|-792.48| +2450821|22391|2450825|66424|1852514|6157|14200|66424|1852514|6157|14200|2|20|9|4|12388|57|317|5|83.69|179.93|57.57|611.80|287.85|418.45|899.65|10.36|80.59|278.85|207.26|217.62|486.11|496.47|-211.19| +2450821|22391|2450829|66424|1852514|6157|14200|66424|1852514|6157|14200|2|84|17|1|11042|181|317|3|6.32|12.82|10.25|7.71|30.75|18.96|38.46|1.84|0.00|2.67|30.75|32.59|33.42|35.26|11.79| +2450821|22391|2450853|66424|1852514|6157|14200|66424|1852514|6157|14200|2|39|2|3|1597|7|317|41|24.94|26.68|8.53|744.15|349.73|1022.54|1093.88|0.00|255.30|470.27|94.43|94.43|564.70|564.70|-928.11| +2450821|22391|2450852|66424|1852514|6157|14200|66424|1852514|6157|14200|2|61|2|5|1585|88|317|40|71.93|189.17|117.28|2875.60|4691.20|2877.20|7566.80|375.29|0.00|1967.20|4691.20|5066.49|6658.40|7033.69|1814.00| +2450821|22391|2450888|66424|1852514|6157|14200|66424|1852514|6157|14200|2|105|19|4|12700|127|317|54|25.55|70.77|32.55|2063.88|1757.70|1379.70|3821.58|140.61|0.00|878.58|1757.70|1898.31|2636.28|2776.89|378.00| +2450821|56693|2450830|17800|1800817|4281|29745|17800|1800817|4281|29745|1|51|16|1|3836|88|318|44|65.37|122.89|4.91|5191.12|216.04|2876.28|5407.16|19.44|0.00|594.44|216.04|235.48|810.48|829.92|-2660.24| +2450821|56693|2450874|17800|1800817|4281|29745|17800|1800817|4281|29745|1|107|17|3|4334|122|318|61|63.65|175.03|73.51|6192.72|4484.11|3882.65|10676.83|313.88|0.00|3843.61|4484.11|4797.99|8327.72|8641.60|601.46| +2450821|56693|2450861|17800|1800817|4281|29745|17800|1800817|4281|29745|1|20|2|4|15904|229|318|54|93.71|103.08|1.03|5510.70|55.62|5060.34|5566.32|2.78|0.00|667.44|55.62|58.40|723.06|725.84|-5004.72| +2450821|56693|2450831|17800|1800817|4281|29745|17800|1800817|4281|29745|1|30|18|1|1514|289|318|51|37.20|77.00|69.30|392.70|3534.30|1897.20|3927.00|70.68|0.00|1531.53|3534.30|3604.98|5065.83|5136.51|1637.10| +2450821|56693|2450833|17800|1800817|4281|29745|17800|1800817|4281|29745|1|84|16|3|6472|247|318|96|18.89|45.14|20.31|2383.68|1949.76|1813.44|4333.44|90.46|818.89|1256.64|1130.87|1221.33|2387.51|2477.97|-682.57| +2450821|56693|2450881|17800|1800817|4281|29745|17800|1800817|4281|29745|1|108|9|1|10034|84|318|4|91.44|117.04|78.41|154.52|313.64|365.76|468.16|6.27|0.00|88.92|313.64|319.91|402.56|408.83|-52.12| +2450821|56693|2450888|17800|1800817|4281|29745|17800|1800817|4281|29745|1|34|20|1|15229|173|318|38|25.59|28.40|23.57|183.54|895.66|972.42|1079.20|8.95|0.00|269.80|895.66|904.61|1165.46|1174.41|-76.76| +2450821|56693|2450889|17800|1800817|4281|29745|17800|1800817|4281|29745|1|48|19|1|2182|213|318|66|22.97|41.11|34.94|407.22|2306.04|1516.02|2713.26|92.24|0.00|271.26|2306.04|2398.28|2577.30|2669.54|790.02| +2450821|56693|2450838|17800|1800817|4281|29745|17800|1800817|4281|29745|1|94|14|5|16450|161|318|18|10.73|28.64|0.28|510.48|5.04|193.14|515.52|0.16|1.71|41.22|3.33|3.49|44.55|44.71|-189.81| +2450821|69456|2450910|57088|483275|7062|41602|57088|483275|7062|41602|1|47|6|2|5839|153|319|1|32.44|86.29|79.38|6.91|79.38|32.44|86.29|3.96|0.00|6.90|79.38|83.34|86.28|90.24|46.94| +2450821|69456|2450826|57088|483275|7062|41602|57088|483275|7062|41602|1|104|14|2|8491|196|319|25|82.40|172.21|120.54|1291.75|3013.50|2060.00|4305.25|271.21|0.00|1420.50|3013.50|3284.71|4434.00|4705.21|953.50| +2450821|69456|2450891|57088|483275|7062|41602|57088|483275|7062|41602|1|83|1|1|15112|44|319|20|80.66|90.33|46.06|885.40|921.20|1613.20|1806.60|55.27|0.00|379.20|921.20|976.47|1300.40|1355.67|-692.00| +2450821|69456|2450894|57088|483275|7062|41602|57088|483275|7062|41602|1|77|8|1|3392|210|319|77|94.52|276.94|188.31|6824.51|14499.87|7278.04|21324.38|95.69|4929.95|1705.55|9569.92|9665.61|11275.47|11371.16|2291.88| +2450821|69456|2450877|57088|483275|7062|41602|57088|483275|7062|41602|1|22|15|4|4768|225|319|18|23.53|47.29|18.44|519.30|331.92|423.54|851.22|3.31|0.00|425.52|331.92|335.23|757.44|760.75|-91.62| +2450821|69456|2450825|57088|483275|7062|41602|57088|483275|7062|41602|1|57|1|2|8438|109|319|68|31.49|63.60|23.53|2724.76|1600.04|2141.32|4324.80|55.68|208.00|0.00|1392.04|1447.72|1392.04|1447.72|-749.28| +2450821|69456|2450849|57088|483275|7062|41602|57088|483275|7062|41602|1|42|5|2|4906|210|319|19|92.31|94.15|3.76|1717.41|71.44|1753.89|1788.85|2.14|0.00|733.40|71.44|73.58|804.84|806.98|-1682.45| +2450821|69456|2450871|57088|483275|7062|41602|57088|483275|7062|41602|1|62|17|3|6793|85|319|25|96.55|207.58|168.13|986.25|4203.25|2413.75|5189.50|84.06|0.00|415.00|4203.25|4287.31|4618.25|4702.31|1789.50| +2450821|69456|2450830|57088|483275|7062|41602|57088|483275|7062|41602|1|69|4|2|3622|290|319|21|68.38|99.15|70.39|603.96|1478.19|1435.98|2082.15|14.78|0.00|333.06|1478.19|1492.97|1811.25|1826.03|42.21| +2450821|69456|2450830|57088|483275|7062|41602|57088|483275|7062|41602|1|12|11|3|9926|290|319|60|81.39|198.59|148.94|2979.00|8936.40|4883.40|11915.40|357.45|0.00|4051.20|8936.40|9293.85|12987.60|13345.05|4053.00| +2450821|67672|2450844|23589|1411705|1774|35843|23715|1228|6944|24613|4|77|18|2|2239|192|320|49|7.63|10.98|7.24|183.26|354.76|373.87|538.02|3.54|0.00|123.48|354.76|358.30|478.24|481.78|-19.11| +2450821|67672|2450867|23589|1411705|1774|35843|23715|1228|6944|24613|4|71|8|5|14305|186|320|15|77.33|114.44|60.65|806.85|909.75|1159.95|1716.60|54.58|0.00|600.75|909.75|964.33|1510.50|1565.08|-250.20| +2450821|67672|2450829|23589|1411705|1774|35843|23715|1228|6944|24613|4|94|16|5|6214|237|320|62|2.04|2.06|1.40|40.92|86.80|126.48|127.72|5.20|0.00|49.60|86.80|92.00|136.40|141.60|-39.68| +2450821|67672|2450859|23589|1411705|1774|35843|23715|1228|6944|24613|4|14|13|1|7258|189|320|68|40.02|69.23|39.46|2024.36|2683.28|2721.36|4707.64|53.66|0.00|1459.28|2683.28|2736.94|4142.56|4196.22|-38.08| +2450821|67672|2450904|23589|1411705|1774|35843|23715|1228|6944|24613|4|70|6|3|295|67|320|45|74.64|183.61|66.09|5288.40|2974.05|3358.80|8262.45|59.48|0.00|991.35|2974.05|3033.53|3965.40|4024.88|-384.75| +2450821|61523|2450903|59512|289503|6247|2191|59512|289503|6247|2191|4|89|10|4|10846|195|321|59|4.28|12.15|5.10|415.95|300.90|252.52|716.85|21.06|0.00|315.06|300.90|321.96|615.96|637.02|48.38| +2450821|61523|2450842|59512|289503|6247|2191|59512|289503|6247|2191|4|61|13|4|8665|246|321|73|37.29|86.88|33.01|3932.51|2409.73|2722.17|6342.24|72.29|0.00|1395.03|2409.73|2482.02|3804.76|3877.05|-312.44| +2450821|61523|2450908|59512|289503|6247|2191|59512|289503|6247|2191|4|64|20|4|2414|125|321|13|24.28|50.74|31.96|244.14|415.48|315.64|659.62|12.46|0.00|263.77|415.48|427.94|679.25|691.71|99.84| +2450821|61523|2450893|59512|289503|6247|2191|59512|289503|6247|2191|4|55|4|1|5330|64|321|30|97.53|187.25|108.60|2359.50|3258.00|2925.90|5617.50|97.74|0.00|2527.80|3258.00|3355.74|5785.80|5883.54|332.10| +2450821|26417|2450858|68260|1493373|3639|45552|68260|1493373|3639|45552|1|63|4|1|16922|293|322|63|22.44|28.94|3.76|1586.34|236.88|1413.72|1823.22|2.55|194.24|273.42|42.64|45.19|316.06|318.61|-1371.08| +2450821|26417|2450886|68260|1493373|3639|45552|68260|1493373|3639|45552|1|101|8|3|12040|160|322|71|74.49|174.30|64.49|7796.51|4578.79|5288.79|12375.30|412.09|0.00|3588.34|4578.79|4990.88|8167.13|8579.22|-710.00| +2450821|26417|2450862|68260|1493373|3639|45552|68260|1493373|3639|45552|1|45|13|3|10990|118|322|38|55.17|76.68|49.07|1049.18|1864.66|2096.46|2913.84|0.00|0.00|728.46|1864.66|1864.66|2593.12|2593.12|-231.80| +2450821|26417|2450883|68260|1493373|3639|45552|68260|1493373|3639|45552|1|63|10|2|7003|201|322|50|55.76|139.95|60.17|3989.00|3008.50|2788.00|6997.50|270.76|0.00|2659.00|3008.50|3279.26|5667.50|5938.26|220.50| +2450821|26417|2450895|68260|1493373|3639|45552|68260|1493373|3639|45552|1|44|12|3|16936|185|322|75|88.74|200.55|84.23|8724.00|6317.25|6655.50|15041.25|442.20|0.00|2256.00|6317.25|6759.45|8573.25|9015.45|-338.25| +2450821|26417|2450860|68260|1493373|3639|45552|68260|1493373|3639|45552|1|12|12|2|17479|160|322|16|70.40|162.62|55.29|1717.28|884.64|1126.40|2601.92|23.53|548.47|468.32|336.17|359.70|804.49|828.02|-790.23| +2450821|26417|2450845|68260|1493373|3639|45552|68260|1493373|3639|45552|1|85|5|5|14648|158|322|99|53.91|155.79|18.69|13572.90|1850.31|5337.09|15423.21|14.80|370.06|4471.83|1480.25|1495.05|5952.08|5966.88|-3856.84| +2450821|26417|2450848|68260|1493373|3639|45552|68260|1493373|3639|45552|1|4|9|4|1909|192|322|45|62.60|148.36|124.62|1068.30|5607.90|2817.00|6676.20|168.23|0.00|2470.05|5607.90|5776.13|8077.95|8246.18|2790.90| +2450821|26417|2450825|68260|1493373|3639|45552|68260|1493373|3639|45552|1|59|11|4|10135|54|322|91|95.84|287.52|224.26|5756.66|20407.66|8721.44|26164.32|204.07|0.00|1830.92|20407.66|20611.73|22238.58|22442.65|11686.22| +2450821|41116|2450832|41010|372208|6253|33051|41010|372208|6253|33051|1|7|18|4|17278|86|323|5|18.30|25.80|21.93|19.35|109.65|91.50|129.00|7.67|0.00|42.55|109.65|117.32|152.20|159.87|18.15| +2450821|41116|2450832|41010|372208|6253|33051|41010|372208|6253|33051|1|54|12|3|9676|142|323|51|64.50|101.26|29.36|3666.90|1497.36|3289.50|5164.26|0.00|1152.96|413.10|344.40|344.40|757.50|757.50|-2945.10| +2450821|41116|2450856|41010|372208|6253|33051|41010|372208|6253|33051|1|20|10|1|5608|22|323|69|47.05|125.15|77.59|3281.64|5353.71|3246.45|8635.35|481.83|0.00|863.19|5353.71|5835.54|6216.90|6698.73|2107.26| +2450821|41116|2450891|41010|372208|6253|33051|41010|372208|6253|33051|1|29|2|2|16021|164|323|33|88.69|98.44|44.29|1786.95|1461.57|2926.77|3248.52|87.69|0.00|1039.50|1461.57|1549.26|2501.07|2588.76|-1465.20| +2450821|41116|2450835|41010|372208|6253|33051|41010|372208|6253|33051|1|30|20|1|6427|225|323|76|34.01|97.94|60.72|2828.72|4614.72|2584.76|7443.44|276.88|0.00|2456.32|4614.72|4891.60|7071.04|7347.92|2029.96| +2450821|41116|2450883|41010|372208|6253|33051|41010|372208|6253|33051|1|70|6|1|10399|228|323|61|88.37|165.25|133.85|1915.40|8164.85|5390.57|10080.25|571.53|0.00|503.86|8164.85|8736.38|8668.71|9240.24|2774.28| +2450821|41116|2450867|41010|372208|6253|33051|41010|372208|6253|33051|1|8|1|2|934|62|323|11|32.28|46.80|3.27|478.83|35.97|355.08|514.80|2.51|0.00|247.06|35.97|38.48|283.03|285.54|-319.11| +2450821|41116|2450859|41010|372208|6253|33051|41010|372208|6253|33051|1|13|16|3|12854|231|323|63|84.66|194.71|103.19|5765.76|6500.97|5333.58|12266.73|585.08|0.00|1716.75|6500.97|7086.05|8217.72|8802.80|1167.39| +2450821|56717|2450868|89170|813258|4993|36328|89170|813258|4993|36328|2|72|10|1|12962|131|324|46|85.81|99.53|90.57|412.16|4166.22|3947.26|4578.38|374.95|0.00|1510.64|4166.22|4541.17|5676.86|6051.81|218.96| +2450821|56717|2450906|89170|813258|4993|36328|89170|813258|4993|36328|2|105|9|4|7606|269|324|40|72.26|74.42|56.55|714.80|2262.00|2890.40|2976.80|67.86|0.00|624.80|2262.00|2329.86|2886.80|2954.66|-628.40| +2450821|56717|2450905|89170|813258|4993|36328|89170|813258|4993|36328|2|20|3|4|26|90|324|28|73.01|165.00|41.25|3465.00|1155.00|2044.28|4620.00|80.85|0.00|2310.00|1155.00|1235.85|3465.00|3545.85|-889.28| +2450821|56717|2450862|89170|813258|4993|36328|89170|813258|4993|36328|2|43|11|4|17983|35|324|71|39.61|64.96|36.37|2029.89|2582.27|2812.31|4612.16|0.00|0.00|2213.78|2582.27|2582.27|4796.05|4796.05|-230.04| +2450821|56717|2450885|89170|813258|4993|36328|89170|813258|4993|36328|2|13|12|2|12494|279|324|10|36.62|57.49|50.01|74.80|500.10|366.20|574.90|45.00|0.00|281.70|500.10|545.10|781.80|826.80|133.90| +2450821|56717|2450837|89170|813258|4993|36328|89170|813258|4993|36328|2|44|4|5|14992|162|324|87|62.48|168.69|131.57|3229.44|11446.59|5435.76|14676.03|915.72|0.00|3668.79|11446.59|12362.31|15115.38|16031.10|6010.83| +2450821|56717|2450841|89170|813258|4993|36328|89170|813258|4993|36328|2|37|8|1|5713|158|324|83|39.43|73.73|62.67|917.98|5201.61|3272.69|6119.59|8.84|4317.33|2997.96|884.28|893.12|3882.24|3891.08|-2388.41| +2450821|56717|2450893|89170|813258|4993|36328|89170|813258|4993|36328|2|107|20|3|11689|73|324|23|54.31|99.93|76.94|528.77|1769.62|1249.13|2298.39|44.59|1274.12|666.31|495.50|540.09|1161.81|1206.40|-753.63| +2450821|56717|2450904|89170|813258|4993|36328|89170|813258|4993|36328|2|64|12|5|11728|206|324|73|59.67|146.19|5.84|10245.55|426.32|4355.91|10671.87|4.26|0.00|2881.31|426.32|430.58|3307.63|3311.89|-3929.59| +2450821|56717|2450866|89170|813258|4993|36328|89170|813258|4993|36328|2|10|6|1|5726|156|324|35|90.91|111.81|72.67|1369.90|2543.45|3181.85|3913.35|0.76|2467.14|1721.65|76.31|77.07|1797.96|1798.72|-3105.54| +2450821|28936|2450865|75118|892053|4635|46017|75118|892053|4635|46017|1|30|17|2|9632|130|325|61|60.29|172.42|53.45|7257.17|3260.45|3677.69|10517.62|65.20|0.00|1471.93|3260.45|3325.65|4732.38|4797.58|-417.24| +2450821|28936|2450860|75118|892053|4635|46017|75118|892053|4635|46017|1|45|12|5|9292|100|325|49|69.75|143.68|106.32|1830.64|5209.68|3417.75|7040.32|104.19|0.00|3027.22|5209.68|5313.87|8236.90|8341.09|1791.93| +2450821|28936|2450838|75118|892053|4635|46017|75118|892053|4635|46017|1|46|11|1|7988|126|325|82|96.16|139.43|18.12|9947.42|1485.84|7885.12|11433.26|14.85|0.00|2972.50|1485.84|1500.69|4458.34|4473.19|-6399.28| +2450821|28936|2450889|75118|892053|4635|46017|75118|892053|4635|46017|1|32|15|3|5618|67|325|15|70.55|178.49|124.94|803.25|1874.10|1058.25|2677.35|74.96|0.00|615.75|1874.10|1949.06|2489.85|2564.81|815.85| +2450821|28936|2450857|75118|892053|4635|46017|75118|892053|4635|46017|1|11|12|5|7813|208|325|9|2.67|4.91|3.28|14.67|29.52|24.03|44.19|0.30|25.68|1.71|3.84|4.14|5.55|5.85|-20.19| +2450821|28936|2450894|75118|892053|4635|46017|75118|892053|4635|46017|1|85|10|1|5408|61|325|41|54.13|109.88|104.38|225.50|4279.58|2219.33|4505.08|107.84|2482.15|1261.16|1797.43|1905.27|3058.59|3166.43|-421.90| +2450821|28936|2450873|75118|892053|4635|46017|75118|892053|4635|46017|1|51|16|5|1658|68|325|85|26.74|38.50|0.77|3207.05|65.45|2272.90|3272.50|0.65|0.00|1603.10|65.45|66.10|1668.55|1669.20|-2207.45| +2450821|28936|2450860|75118|892053|4635|46017|75118|892053|4635|46017|1|103|15|3|8798|182|325|3|66.03|144.60|8.67|407.79|26.01|198.09|433.80|0.00|0.00|160.50|26.01|26.01|186.51|186.51|-172.08| +2450821|28936|2450833|75118|892053|4635|46017|75118|892053|4635|46017|1|72|14|3|8830|155|325|38|88.89|243.55|199.71|1665.92|7588.98|3377.82|9254.90|303.55|0.00|277.40|7588.98|7892.53|7866.38|8169.93|4211.16| +2450821|28936|2450901|75118|892053|4635|46017|75118|892053|4635|46017|1|75|19|4|4280|6|325|77|96.95|280.18|268.97|863.17|20710.69|7465.15|21573.86|0.00|2485.28|3451.14|18225.41|18225.41|21676.55|21676.55|10760.26| +2450821|28936|2450892|75118|892053|4635|46017|75118|892053|4635|46017|1|37|17|1|2617|1|325|12|39.51|53.33|13.86|473.64|166.32|474.12|639.96|9.28|11.64|25.56|154.68|163.96|180.24|189.52|-319.44| +2450821|28936|2450856|75118|892053|4635|46017|75118|892053|4635|46017|1|69|19|4|7429|152|325|70|47.69|96.33|86.69|674.80|6068.30|3338.30|6743.10|485.46|0.00|1820.00|6068.30|6553.76|7888.30|8373.76|2730.00| +2450821|28936|2450903|75118|892053|4635|46017|75118|892053|4635|46017|1|63|13|5|2164|160|325|77|94.37|254.79|38.21|16676.66|2942.17|7266.49|19618.83|117.68|0.00|7650.72|2942.17|3059.85|10592.89|10710.57|-4324.32| +2450821|46262|2450890|36318|1032797|1723|27802|36318|1032797|1723|27802|4|34|1|4|3703|65|326|23|99.67|252.16|5.04|5683.76|115.92|2292.41|5799.68|3.47|0.00|637.79|115.92|119.39|753.71|757.18|-2176.49| +2450821|46262|2450892|36318|1032797|1723|27802|36318|1032797|1723|27802|4|92|20|3|1888|204|326|15|98.13|228.64|198.91|445.95|2983.65|1471.95|3429.60|119.34|0.00|788.70|2983.65|3102.99|3772.35|3891.69|1511.70| +2450821|46262|2450868|36318|1032797|1723|27802|36318|1032797|1723|27802|4|59|4|2|12010|99|326|28|87.03|101.82|81.45|570.36|2280.60|2436.84|2850.96|45.61|0.00|627.20|2280.60|2326.21|2907.80|2953.41|-156.24| +2450821|46262|2450859|36318|1032797|1723|27802|36318|1032797|1723|27802|4|37|18|4|4489|24|326|5|61.61|114.59|68.75|229.20|343.75|308.05|572.95|6.87|0.00|114.55|343.75|350.62|458.30|465.17|35.70| +2450821|46262|2450842|36318|1032797|1723|27802|36318|1032797|1723|27802|4|67|13|3|2336|201|326|78|44.02|59.42|10.69|3800.94|833.82|3433.56|4634.76|66.70|0.00|184.86|833.82|900.52|1018.68|1085.38|-2599.74| +2450821|46262|2450901|36318|1032797|1723|27802|36318|1032797|1723|27802|4|48|17|2|13436|182|326|17|79.01|192.78|92.53|1704.25|1573.01|1343.17|3277.26|141.57|0.00|1343.51|1573.01|1714.58|2916.52|3058.09|229.84| +2450821|46262|2450863|36318|1032797|1723|27802|36318|1032797|1723|27802|4|78|19|2|6283|205|326|92|62.01|186.03|117.19|6333.28|10781.48|5704.92|17114.76|0.00|6900.14|684.48|3881.34|3881.34|4565.82|4565.82|-1823.58| +2450821|33107|2450845|3104|993947|7190|22433|3104|993947|7190|22433|1|8|4|1|9562|28|327|78|82.72|181.98|52.77|10078.38|4116.06|6452.16|14194.44|2.46|3869.09|4541.94|246.97|249.43|4788.91|4791.37|-6205.19| +2450821|33107|2450899|3104|993947|7190|22433|3104|993947|7190|22433|1|18|9|2|6380|177|327|29|3.38|4.42|3.18|35.96|92.22|98.02|128.18|5.53|0.00|39.73|92.22|97.75|131.95|137.48|-5.80| +2450821|33107|2450886|3104|993947|7190|22433|3104|993947|7190|22433|1|84|6|3|16285|175|327|99|83.36|146.71|136.44|1016.73|13507.56|8252.64|14524.29|810.45|0.00|5518.26|13507.56|14318.01|19025.82|19836.27|5254.92| +2450821|33107|2450895|3104|993947|7190|22433|3104|993947|7190|22433|1|3|18|2|12206|62|327|71|6.85|13.70|3.28|739.82|232.88|486.35|972.70|9.31|0.00|437.36|232.88|242.19|670.24|679.55|-253.47| +2450821|33107|2450888|3104|993947|7190|22433|3104|993947|7190|22433|1|104|13|2|17834|263|327|77|49.34|125.32|10.02|8878.10|771.54|3799.18|9649.64|0.00|0.00|2798.18|771.54|771.54|3569.72|3569.72|-3027.64| +2450821|33107|2450855|3104|993947|7190|22433|3104|993947|7190|22433|1|18|15|4|14725|40|327|61|18.12|22.10|9.72|755.18|592.92|1105.32|1348.10|53.36|0.00|350.14|592.92|646.28|943.06|996.42|-512.40| +2450821|33107|2450890|3104|993947|7190|22433|3104|993947|7190|22433|1|51|11|4|16888|100|327|7|66.21|131.75|104.08|193.69|728.56|463.47|922.25|43.71|0.00|0.00|728.56|772.27|728.56|772.27|265.09| +2450821|33107|2450877|3104|993947|7190|22433|3104|993947|7190|22433|1|27|12|1|5792|64|327|65|9.63|21.66|13.86|507.00|900.90|625.95|1407.90|13.51|630.63|379.60|270.27|283.78|649.87|663.38|-355.68| +2450821|33107|2450895|3104|993947|7190|22433|3104|993947|7190|22433|1|105|12|1|5227|52|327|46|97.98|254.74|38.21|9960.38|1757.66|4507.08|11718.04|55.36|966.71|5859.02|790.95|846.31|6649.97|6705.33|-3716.13| +2450821|33107|2450880|3104|993947|7190|22433|3104|993947|7190|22433|1|81|13|1|11461|155|327|65|73.66|208.45|37.52|11110.45|2438.80|4787.90|13549.25|48.77|0.00|677.30|2438.80|2487.57|3116.10|3164.87|-2349.10| +2450821|33107|2450903|3104|993947|7190|22433|3104|993947|7190|22433|1|25|1|5|3586|138|327|5|65.89|177.90|17.79|800.55|88.95|329.45|889.50|2.66|0.00|391.35|88.95|91.61|480.30|482.96|-240.50| +2450821|33107|2450871|3104|993947|7190|22433|3104|993947|7190|22433|1|77|17|1|16927|207|327|38|17.22|41.32|4.95|1382.06|188.10|654.36|1570.16|3.76|0.00|643.72|188.10|191.86|831.82|835.58|-466.26| +2450821|33107|2450889|3104|993947|7190|22433|3104|993947|7190|22433|1|103|11|1|7148|97|327|97|72.82|217.00|82.46|13050.38|7998.62|7063.54|21049.00|399.93|0.00|7998.62|7998.62|8398.55|15997.24|16397.17|935.08| +2450821|33107|2450903|3104|993947|7190|22433|3104|993947|7190|22433|1|2|3|1|1808|209|327|64|1.28|3.60|0.86|175.36|55.04|81.92|230.40|3.85|0.00|85.12|55.04|58.89|140.16|144.01|-26.88| +2450821|67116|2450894|30042|268188|5534|26080|30042|268188|5534|26080|2|39|5|3|11192|268|328|20|82.31|150.62|122.00|572.40|2440.00|1646.20|3012.40|146.40|0.00|1265.20|2440.00|2586.40|3705.20|3851.60|793.80| +2450821|67116|2450864|30042|268188|5534|26080|30042|268188|5534|26080|2|94|19|2|10681|130|328|19|56.26|120.95|62.89|1103.14|1194.91|1068.94|2298.05|83.64|0.00|827.26|1194.91|1278.55|2022.17|2105.81|125.97| +2450821|67116|2450877|30042|268188|5534|26080|30042|268188|5534|26080|2|67|8|4|16784|93|328|11|91.10|197.68|126.51|782.87|1391.61|1002.10|2174.48|13.91|0.00|456.61|1391.61|1405.52|1848.22|1862.13|389.51| +2450821|67116|2450873|30042|268188|5534|26080|30042|268188|5534|26080|2|47|14|4|14383|263|328|18|62.02|75.66|37.07|694.62|667.26|1116.36|1361.88|6.00|547.15|136.08|120.11|126.11|256.19|262.19|-996.25| +2450821|67116|2450910|30042|268188|5534|26080|30042|268188|5534|26080|2|101|7|5|4796|66|328|64|15.37|25.20|17.89|467.84|1144.96|983.68|1612.80|22.89|0.00|225.28|1144.96|1167.85|1370.24|1393.13|161.28| +2450821|67116||30042|268188|||30042|268188||26080|2|||3|13051||328|68|2.50|6.75|6.68||454.24||459.00|27.25||||481.49|596.36|623.61|284.24| +2450821|67116|2450838|30042|268188|5534|26080|30042|268188|5534|26080|2|89|18|5|3568|288|328|77|68.85|172.12|127.36|3446.52|9806.72|5301.45|13253.24|294.20|0.00|5566.33|9806.72|10100.92|15373.05|15667.25|4505.27| +2450821|63416|2450829|63834|1688101|5615|41082|63834|1688101|5615|41082|2|14|20|3|6511|214|329|98|93.56|123.49|81.50|4115.02|7987.00|9168.88|12102.02|319.48|0.00|4356.10|7987.00|8306.48|12343.10|12662.58|-1181.88| +2450821|63416|2450882|63834|1688101|5615|41082|63834|1688101|5615|41082|2|38|7|1|7999|247|329|54|98.94|100.91|86.78|763.02|4686.12|5342.76|5449.14|4.21|4264.36|2397.60|421.76|425.97|2819.36|2823.57|-4921.00| +2450821|63416|2450840|63834|1688101|5615|41082|63834|1688101|5615|41082|2|19|4|2|1918|205|329|60|31.81|79.84|6.38|4407.60|382.80|1908.60|4790.40|3.82|0.00|2347.20|382.80|386.62|2730.00|2733.82|-1525.80| +2450821|63416|2450889|63834|1688101|5615|41082|63834|1688101|5615|41082|2|104|18|4|11050|1|329|57|8.66|12.99|5.32|437.19|303.24|493.62|740.43|18.19|0.00|117.99|303.24|321.43|421.23|439.42|-190.38| +2450821|63416|2450842|63834|1688101|5615|41082|63834|1688101|5615|41082|2|35|15|2|10424|41|329|32|42.39|123.77|51.98|2297.28|1663.36|1356.48|3960.64|0.00|0.00|871.04|1663.36|1663.36|2534.40|2534.40|306.88| +2450821|63416|2450895|63834|1688101|5615|41082|63834|1688101|5615|41082|2|18|7|5|12019|71|329|29|77.22|213.12|42.62|4944.50|1235.98|2239.38|6180.48|61.79|0.00|2224.88|1235.98|1297.77|3460.86|3522.65|-1003.40| +2450821|63416|2450897|63834|1688101|5615|41082|63834|1688101|5615|41082|2|60|1|3|2215|151|329|71|41.07|52.15|7.82|3147.43|555.22|2915.97|3702.65|11.10|0.00|1110.44|555.22|566.32|1665.66|1676.76|-2360.75| +2450821|35771|2450889|77740|1629007|2227|10647|77740|1629007|2227|10647|4|21|10|2|16459|36|330|45|44.87|61.92|57.58|195.30|2591.10|2019.15|2786.40|155.46|0.00|835.65|2591.10|2746.56|3426.75|3582.21|571.95| +2450821|35771|2450878|77740|1629007|2227|10647|77740|1629007|2227|10647|4|41|14|1|14914|10|330|11|77.12|179.68|111.40|751.08|1225.40|848.32|1976.48|24.50|0.00|889.35|1225.40|1249.90|2114.75|2139.25|377.08| +2450821|35771|2450886|77740|1629007|2227|10647|77740|1629007|2227|10647|4|24|4|5|4622|59|330|18|62.95|138.49|12.46|2268.54|224.28|1133.10|2492.82|0.00|219.79|722.88|4.49|4.49|727.37|727.37|-1128.61| +2450821|35771|2450835|77740|1629007|2227|10647|77740|1629007|2227|10647|4|71|7|4|3715|252|330|23|97.47|253.42|200.20|1224.06|4604.60|2241.81|5828.66|46.04|0.00|2622.69|4604.60|4650.64|7227.29|7273.33|2362.79| +2450821|35771|2450835|77740|1629007|2227|10647|77740|1629007|2227|10647|4|82|19|3|17138|44|330|36|74.08|185.20|61.11|4467.24|2199.96|2666.88|6667.20|131.99|0.00|399.96|2199.96|2331.95|2599.92|2731.91|-466.92| +2450821|35771|2450847|77740|1629007|2227|10647|77740|1629007|2227|10647|4|1|2|3|1594|115|330|39|2.69|6.85|5.89|37.44|229.71|104.91|267.15|4.59|0.00|117.39|229.71|234.30|347.10|351.69|124.80| +2450821|35771|2450900|77740|1629007|2227|10647|77740|1629007|2227|10647|4|69|7|3|14251|126|330|86|64.93|143.49|22.95|10366.44|1973.70|5583.98|12340.14|177.63|0.00|1357.08|1973.70|2151.33|3330.78|3508.41|-3610.28| +2450821|35771|2450855|77740|1629007|2227|10647|77740|1629007|2227|10647|4|102|15|1|1006|201|330|40|47.01|92.60|37.96|2185.60|1518.40|1880.40|3704.00|30.36|0.00|1666.80|1518.40|1548.76|3185.20|3215.56|-362.00| +2450821|35771|2450857|77740|1629007|2227|10647|77740|1629007|2227|10647|4|8|2|3|2696|195|330|30|87.49|187.22|87.99|2976.90|2639.70|2624.70|5616.60|184.77|0.00|2078.10|2639.70|2824.47|4717.80|4902.57|15.00| +2450821|69642|2450848|32133|179214|1848|10129|39012|760691|925|42006|2|38|1|5|14245|26|331|16|77.51|165.09|161.78|52.96|2588.48|1240.16|2641.44|0.00|414.15|845.12|2174.33|2174.33|3019.45|3019.45|934.17| +2450821|69642|2450847|32133|179214|1848|10129|39012|760691|925|42006|2|50|5|1|2852|191|331|23|27.09|70.70|36.76|780.62|845.48|623.07|1626.10|16.90|0.00|471.50|845.48|862.38|1316.98|1333.88|222.41| +2450821|69642|2450882|32133|179214|1848|10129|39012|760691|925|42006|2|39|10|5|12058|3|331|33|15.61|38.71|30.19|281.16|996.27|515.13|1277.43|29.88|0.00|0.00|996.27|1026.15|996.27|1026.15|481.14| +2450821|69642|2450888|32133|179214|1848|10129|39012|760691|925|42006|2|59|15|4|11869|80|331|23|30.10|35.81|20.41|354.20|469.43|692.30|823.63|28.16|0.00|16.33|469.43|497.59|485.76|513.92|-222.87| +2450821|69642|2450871|32133|179214|1848|10129|39012|760691|925|42006|2|55|11|3|5734|165|331|41|47.93|65.18|55.40|400.98|2271.40|1965.13|2672.38|181.71|0.00|320.62|2271.40|2453.11|2592.02|2773.73|306.27| +2450821|69642|2450877|32133|179214|1848|10129|39012|760691|925|42006|2|42|8|1|1012|32|331|13|38.21|74.89|38.94|467.35|506.22|496.73|973.57|0.00|0.00|116.74|506.22|506.22|622.96|622.96|9.49| +2450821|69642|2450910|32133|179214|1848|10129|39012|760691|925|42006|2|47|6|4|12698|247|331|94|68.79|140.33|57.53|7783.20|5407.82|6466.26|13191.02|432.62|0.00|1714.56|5407.82|5840.44|7122.38|7555.00|-1058.44| +2450821|69642|2450830|32133|179214|1848|10129|39012|760691|925|42006|2|36|5|5|2110|175|331|50|8.05|21.25|2.12|956.50|106.00|402.50|1062.50|7.27|2.12|244.00|103.88|111.15|347.88|355.15|-298.62| +2450821|69642|2450863|32133|179214|1848|10129|39012|760691|925|42006|2|108|1|3|12685|74|331|84|88.59|195.78|184.03|987.00|15458.52|7441.56|16445.52|463.75|0.00|0.00|15458.52|15922.27|15458.52|15922.27|8016.96| +2450821|69642|2450848|32133|179214|1848|10129|39012|760691|925|42006|2|54|10|4|9265|204|331|48|20.83|29.78|28.29|71.52|1357.92|999.84|1429.44|108.63|0.00|585.60|1357.92|1466.55|1943.52|2052.15|358.08| +2450821|69642|2450894|32133|179214|1848|10129|39012|760691|925|42006|2|66|5|4|9038|242|331|80|19.16|43.68|18.34|2027.20|1467.20|1532.80|3494.40|1.32|1423.18|1257.60|44.02|45.34|1301.62|1302.94|-1488.78| +2450821|69642|2450823|32133|179214|1848|10129|39012|760691|925|42006|2|69|8|4|11941|109|331|65|42.58|109.85|97.76|785.85|6354.40|2767.70|7140.25|381.26|0.00|1927.25|6354.40|6735.66|8281.65|8662.91|3586.70| +2450821|69642|2450865|32133|179214|1848|10129|39012|760691|925|42006|2|24|4|2|5554|216|331|76|74.49|186.96|44.87|10798.84|3410.12|5661.24|14208.96|68.20|0.00|3410.12|3410.12|3478.32|6820.24|6888.44|-2251.12| +2450821|69642|2450841|32133|179214|1848|10129|39012|760691|925|42006|2|57|8|2|11636|122|331|19|20.77|23.05|19.13|74.48|363.47|394.63|437.95|29.07|0.00|100.70|363.47|392.54|464.17|493.24|-31.16| +2450821|18637|2450877|87635|1726043|5251|41612|87635|1726043|5251|41612|2|91|13|4|4126|106|332|90|82.21|157.02|98.92|5229.00|8902.80|7398.90|14131.80|356.11|0.00|3108.60|8902.80|9258.91|12011.40|12367.51|1503.90| +2450821|18637|2450888|87635|1726043|5251|41612|87635|1726043|5251|41612|2|38|9|2|15316|148|332|79|60.40|66.44|17.27|3884.43|1364.33|4771.60|5248.76|0.00|1077.82|1993.96|286.51|286.51|2280.47|2280.47|-4485.09| +2450821|18637|2450902|87635|1726043|5251|41612|87635|1726043|5251|41612|2|48|14|5|11611|135|332|61|72.43|147.75|134.45|811.30|8201.45|4418.23|9012.75|246.04|0.00|3514.82|8201.45|8447.49|11716.27|11962.31|3783.22| +2450821|18637|2450879|87635|1726043|5251|41612|87635|1726043|5251|41612|2|58|3|5|12667|52|332|60|56.27|159.80|139.02|1246.80|8341.20|3376.20|9588.00|173.49|6172.48|1917.60|2168.72|2342.21|4086.32|4259.81|-1207.48| +2450821|18637|2450906|87635|1726043|5251|41612|87635|1726043|5251|41612|2|13|7|4|11101|296|332|83|17.47|43.50|3.91|3285.97|324.53|1450.01|3610.50|22.06|48.67|902.21|275.86|297.92|1178.07|1200.13|-1174.15| +2450821|18637|2450860|87635|1726043|5251|41612|87635|1726043|5251|41612|2|76|11|2|4006|77|332|44|96.08|234.43|75.01|7014.48|3300.44|4227.52|10314.92|83.17|1221.16|1340.68|2079.28|2162.45|3419.96|3503.13|-2148.24| +2450821|18637|2450906|87635|1726043|5251|41612|87635|1726043|5251|41612|2|104|3|4|17293|122|332|19|44.15|52.53|28.36|459.23|538.84|838.85|998.07|26.94|0.00|69.73|538.84|565.78|608.57|635.51|-300.01| +2450821|18637|2450872|87635|1726043|5251|41612|87635|1726043|5251|41612|2|37|18|1|2650|210|332|70|85.37|236.47|210.45|1821.40|14731.50|5975.90|16552.90|147.31|0.00|7117.60|14731.50|14878.81|21849.10|21996.41|8755.60| +2450821|18637|2450867|87635|1726043|5251|41612|87635|1726043|5251|41612|2|40|11|3|16129|295|332|91|6.73|19.98|4.99|1364.09|454.09|612.43|1818.18|1.27|422.30|144.69|31.79|33.06|176.48|177.75|-580.64| +2450821|18637|2450883|87635|1726043|5251|41612|87635|1726043|5251|41612|2|43|20|1|4520|126|332|64|14.21|28.84|17.01|757.12|1088.64|909.44|1845.76|32.65|0.00|737.92|1088.64|1121.29|1826.56|1859.21|179.20| +2450821|18637|2450866|87635|1726043|5251|41612|87635|1726043|5251|41612|2|59|16|2|10646|120|332|59|15.33|28.66|12.89|930.43|760.51|904.47|1690.94|9.73|638.82|33.63|121.69|131.42|155.32|165.05|-782.78| +2450821|46253|2450909|91110|1274142|6327|20112|91110|1274142|6327|20112|4|95|14|2|10364|13|333|73|26.20|68.64|64.52|300.76|4709.96|1912.60|5010.72|94.19|0.00|2204.60|4709.96|4804.15|6914.56|7008.75|2797.36| +2450821|46253|2450908|91110|1274142|6327|20112|91110|1274142|6327|20112|4|75|10|1|16297|26|333|53|3.13|4.38|1.70|142.04|90.10|165.89|232.14|0.49|80.18|53.00|9.92|10.41|62.92|63.41|-155.97| +2450821|46253|2450843|91110|1274142|6327|20112|91110|1274142|6327|20112|4|107|5|3|5008|1|333|59|97.08|172.80|53.56|7035.16|3160.04|5727.72|10195.20|221.20|0.00|4485.77|3160.04|3381.24|7645.81|7867.01|-2567.68| +2450821|46253|2450830|91110|1274142|6327|20112|91110|1274142|6327|20112|4|78|14|2|6967|63|333|82|79.07|120.18|88.93|2562.50|7292.26|6483.74|9854.76|0.00|0.00|4927.38|7292.26|7292.26|12219.64|12219.64|808.52| +2450821|46253|2450856|91110|1274142|6327|20112|91110|1274142|6327|20112|4|65|17|3|17294|198|333|60|80.05|186.51|110.04|4588.20|6602.40|4803.00|11190.60|396.14|0.00|2125.80|6602.40|6998.54|8728.20|9124.34|1799.40| +||2450842||1274142|6327|20112|91110|1274142|6327|20112|4|41|||3445||333||||154.52|1520.40|||||0.00||4326.56||4501.84|4761.43|2267.72| +2450821|46253|2450859|91110|1274142|6327|20112|91110|1274142|6327|20112|4|79|3|5|10525|262|333|18|69.87|70.56|56.44|254.16|1015.92|1257.66|1270.08|30.47|0.00|533.34|1015.92|1046.39|1549.26|1579.73|-241.74| +2450821|46253|2450864|91110|1274142|6327|20112|91110|1274142|6327|20112|4|55|8|2|1048|78|333|23|13.32|25.44|17.55|181.47|403.65|306.36|585.12|12.10|0.00|257.37|403.65|415.75|661.02|673.12|97.29| +2450821|46253|2450911|91110|1274142|6327|20112|91110|1274142|6327|20112|4|75|14|5|12691|201|333|25|89.88|115.94|98.54|435.00|2463.50|2247.00|2898.50|172.44|0.00|550.50|2463.50|2635.94|3014.00|3186.44|216.50| +2450821|46253|2450836|91110|1274142|6327|20112|91110|1274142|6327|20112|4|49|1|2|9415|129|333|8|80.51|108.68|2.17|852.08|17.36|644.08|869.44|0.34|0.00|426.00|17.36|17.70|443.36|443.70|-626.72| +2450821|46253|2450836|91110|1274142|6327|20112|91110|1274142|6327|20112|4|17|20|2|6014|101|333|68|46.69|47.15|11.31|2437.12|769.08|3174.92|3206.20|46.14|0.00|993.48|769.08|815.22|1762.56|1808.70|-2405.84| +|||91110||6327|||||||||3|8452||333|4||109.40|||323.80|||||218.80|323.80|327.03|542.60|545.83|-40.88| +2450821|46253|2450829|91110|1274142|6327|20112|91110|1274142|6327|20112|4|23|8|3|4186|272|333|46|72.48|205.84|137.91|3124.78|6343.86|3334.08|9468.64|0.00|0.00|662.40|6343.86|6343.86|7006.26|7006.26|3009.78| +2450821|46253|2450899|91110|1274142|6327|20112|91110|1274142|6327|20112|4|11|1|3|11512|192|333|40|15.21|22.05|4.41|705.60|176.40|608.40|882.00|0.52|149.94|158.40|26.46|26.98|184.86|185.38|-581.94| +2450821|81787|2450875|30027|551027|1627|43027|90566|1812627|5627|10674|2|104|19|1|16093|15|334|45|94.83|153.62|89.09|2903.85|4009.05|4267.35|6912.90|205.26|1443.25|2488.50|2565.80|2771.06|5054.30|5259.56|-1701.55| +2450821|81787|2450851|30027|551027|1627|43027|90566|1812627|5627|10674|2|81|17|2|3350|189|334|8|49.61|116.08|88.22|222.88|705.76|396.88|928.64|0.00|0.00|148.56|705.76|705.76|854.32|854.32|308.88| +2450821|81787|2450903|30027|551027|1627|43027|90566|1812627|5627|10674|2|45|9|3|1580|178|334|31|14.37|18.96|0.37|576.29|11.47|445.47|587.76|0.16|5.96|199.64|5.51|5.67|205.15|205.31|-439.96| +2450821|81787|2450841|30027|551027|1627|43027|90566|1812627|5627|10674|2|31|20|4|15538|177|334|28|78.09|82.77|68.69|394.24|1923.32|2186.52|2317.56|40.38|576.99|996.52|1346.33|1386.71|2342.85|2383.23|-840.19| +2450821|81787|2450842|30027|551027|1627|43027|90566|1812627|5627|10674|2|102|8|3|1472|80|334|20|97.71|232.54|60.46|3441.60|1209.20|1954.20|4650.80|96.73|0.00|232.40|1209.20|1305.93|1441.60|1538.33|-745.00| +2450821|81787|2450893|30027|551027|1627|43027|90566|1812627|5627|10674|2|9|15|5|13564|153|334|75|5.91|10.46|6.48|298.50|486.00|443.25|784.50|19.44|0.00|313.50|486.00|505.44|799.50|818.94|42.75| +2450821|81787|2450856|30027|551027|1627|43027|90566|1812627|5627|10674|2|27|18|1|1714|240|334|66|2.19|4.02|3.85|11.22|254.10|144.54|265.32|5.08|0.00|36.96|254.10|259.18|291.06|296.14|109.56| +2450821|81787|2450882|30027|551027|1627|43027|90566|1812627|5627|10674|2|76|15|5|7718|90|334|67|21.02|62.42|8.73|3597.23|584.91|1408.34|4182.14|40.94|0.00|125.29|584.91|625.85|710.20|751.14|-823.43| +2450821|81787|2450832|30027|551027|1627|43027|90566|1812627|5627|10674|2|107|3|4|4093|91|334|21|56.76|69.24|24.92|930.72|523.32|1191.96|1454.04|26.16|0.00|189.00|523.32|549.48|712.32|738.48|-668.64| +2450821|81787|2450834|30027|551027|1627|43027|90566|1812627|5627|10674|2|14|9|1|16723|50|334|78|52.31|108.28|76.87|2449.98|5995.86|4080.18|8445.84|419.71|0.00|1350.96|5995.86|6415.57|7346.82|7766.53|1915.68| +2450821|81787|2450886|30027|551027|1627|43027|90566|1812627|5627|10674|2|73|7|2|16244|189|334|2|24.86|41.76|37.58|8.36|75.16|49.72|83.52|2.25|0.00|26.72|75.16|77.41|101.88|104.13|25.44| +2450821|81787|2450899|30027|551027|1627|43027|90566|1812627|5627|10674|2|18|6|1|11065|188|334|67|36.24|96.03|2.88|6241.05|192.96|2428.08|6434.01|3.85|0.00|192.96|192.96|196.81|385.92|389.77|-2235.12| +2450821|81787|2450859|30027|551027|1627|43027|90566|1812627|5627|10674|2|43|17|2|15392|92|334|86|41.74|120.21|70.92|4238.94|6099.12|3589.64|10338.06|304.95|0.00|723.26|6099.12|6404.07|6822.38|7127.33|2509.48| +2450821|81787|2450888|30027|551027|1627|43027|90566|1812627|5627|10674|2|7|8|3|14791|3|334|32|98.73|137.23|123.50|439.36|3952.00|3159.36|4391.36|140.69|434.72|1580.80|3517.28|3657.97|5098.08|5238.77|357.92| +2450821|72115|2450894|78376|245157|7138|27166|78376|245157|7138|27166|2|80|15|2|12704|259|335|11|22.62|43.65|36.66|76.89|403.26|248.82|480.15|16.13|0.00|81.62|403.26|419.39|484.88|501.01|154.44| +2450821|72115|2450827|78376|245157|7138|27166|78376|245157|7138|27166|2|65|12|3|10580|189|335|91|20.37|42.16|8.01|3107.65|728.91|1853.67|3836.56|29.15|0.00|1610.70|728.91|758.06|2339.61|2368.76|-1124.76| +2450821|72115|2450885|78376|245157|7138|27166|78376|245157|7138|27166|2|91|11|5|13262|283|335|56|46.24|110.97|5.54|5904.08|310.24|2589.44|6214.32|15.20|6.20|1926.40|304.04|319.24|2230.44|2245.64|-2285.40| +2450821|72115|2450826|78376|245157|7138|27166|78376|245157|7138|27166|2|37|17|5|14293|70|335|44|84.51|206.20|96.91|4808.76|4264.04|3718.44|9072.80|298.48|0.00|3719.76|4264.04|4562.52|7983.80|8282.28|545.60| +2450821|72115|2450886|78376|245157|7138|27166|78376|245157|7138|27166|2|102|16|3|3277|40|335|47|1.79|5.28|4.11|54.99|193.17|84.13|248.16|15.45|0.00|76.61|193.17|208.62|269.78|285.23|109.04| +2450821|72115|2450824|78376|245157|7138|27166|78376|245157|7138|27166|2|106|1|5|16378|20|335|20|64.32|105.48|52.74|1054.80|1054.80|1286.40|2109.60|1.26|1012.60|1012.60|42.20|43.46|1054.80|1056.06|-1244.20| +2450821|72115|2450885|78376|245157|7138|27166|78376|245157|7138|27166|2|38|9|2|502|173|335|99|32.04|55.74|6.68|4856.94|661.32|3171.96|5518.26|59.51|0.00|2372.04|661.32|720.83|3033.36|3092.87|-2510.64| +2450821|72115|2450840|78376|245157|7138|27166|78376|245157|7138|27166|2|46|10|3|8392|181|335|91|22.57|57.32|57.32|0.00|5216.12|2053.87|5216.12|128.31|3077.51|2190.37|2138.61|2266.92|4328.98|4457.29|84.74| +2450821|72115|2450909|78376|245157|7138|27166|78376|245157|7138|27166|2|28|3|4|578|58|335|66|12.93|18.48|6.46|793.32|426.36|853.38|1219.68|4.26|0.00|23.76|426.36|430.62|450.12|454.38|-427.02| +2450821|72115|2450830|78376|245157|7138|27166|78376|245157|7138|27166|2|79|13|1|4756|157|335|54|38.40|81.79|67.06|795.42|3621.24|2073.60|4416.66|217.27|0.00|839.16|3621.24|3838.51|4460.40|4677.67|1547.64| +2450821|72115|2450909|78376|245157|7138|27166|78376|245157|7138|27166|2|102|12|5|14144|76|335|30|96.15|261.52|28.76|6982.80|862.80|2884.50|7845.60|34.51|0.00|313.80|862.80|897.31|1176.60|1211.11|-2021.70| +2450821|72115|2450904|78376|245157|7138|27166|78376|245157|7138|27166|2|108|8|1|17782|204|335|10|29.92|38.89|3.88|350.10|38.80|299.20|388.90|0.77|0.00|15.50|38.80|39.57|54.30|55.07|-260.40| +2450821|72115|2450887|78376|245157|7138|27166|78376|245157|7138|27166|2|90|4|4|8716|131|335|22|88.80|237.09|42.67|4277.24|938.74|1953.60|5215.98|84.48|0.00|625.90|938.74|1023.22|1564.64|1649.12|-1014.86| +2450821|72115|2450834|78376|245157|7138|27166|78376|245157|7138|27166|2|89|13|3|11392|160|335|35|41.63|49.53|35.16|502.95|1230.60|1457.05|1733.55|49.22|0.00|242.55|1230.60|1279.82|1473.15|1522.37|-226.45| +2450821|79439|2450829|33008|1425948|5735|27475|62423|1021449|4789|26176|2|7|2|3|11180|59|336|21|6.69|12.57|7.79|100.38|163.59|140.49|263.97|1.63|0.00|105.42|163.59|165.22|269.01|270.64|23.10| +2450821|79439|2450849|33008|1425948|5735|27475|62423|1021449|4789|26176|2|69|8|4|4729|273|336|29|3.70|3.88|2.21|48.43|64.09|107.30|112.52|1.92|0.00|30.16|64.09|66.01|94.25|96.17|-43.21| +2450821|79439|2450861|33008|1425948|5735|27475|62423|1021449|4789|26176|2|26|20|3|4492|140|336|64|5.77|16.96|11.53|347.52|737.92|369.28|1085.44|22.13|0.00|531.84|737.92|760.05|1269.76|1291.89|368.64| +2450821|79439|2450836|33008|1425948|5735|27475|62423|1021449|4789|26176|2|95|20|2|10076|64|336|100|87.30|252.29|222.01|3028.00|22201.00|8730.00|25229.00|377.41|3330.15|4288.00|18870.85|19248.26|23158.85|23536.26|10140.85| +2450821|79439|2450823|33008|1425948|5735|27475|62423|1021449|4789|26176|2|24|12|3|16990|168|336|67|85.81|123.56|53.13|4718.81|3559.71|5749.27|8278.52|106.79|0.00|1821.06|3559.71|3666.50|5380.77|5487.56|-2189.56| +2450821|79439|2450828|33008|1425948|5735|27475|62423|1021449|4789|26176|2|30|13|5|15019|149|336|89|87.98|141.64|18.41|10967.47|1638.49|7830.22|12605.96|114.69|0.00|0.00|1638.49|1753.18|1638.49|1753.18|-6191.73| +2450821|79439|2450889|33008|1425948|5735|27475|62423|1021449|4789|26176|2|18|16|4|11978|63|336|73|56.31|156.54|14.08|10399.58|1027.84|4110.63|11427.42|51.39|0.00|1142.45|1027.84|1079.23|2170.29|2221.68|-3082.79| +2450821|79439|2450907|33008|1425948|5735|27475|62423|1021449|4789|26176|2|57|14|4|10364|139|336|89|65.70|146.51|105.48|3651.67|9387.72|5847.30|13039.39|844.89|0.00|782.31|9387.72|10232.61|10170.03|11014.92|3540.42| +2450821|79439|2450858|33008|1425948|5735|27475|62423|1021449|4789|26176|2|85|10|1|16297|13|336|96|68.01|148.26|108.22|3843.84|10389.12|6528.96|14232.96|519.45|0.00|2561.28|10389.12|10908.57|12950.40|13469.85|3860.16| +2450821|79439|2450827|33008|1425948|5735|27475|62423|1021449|4789|26176|2|106|1|2|5008|129|336|94|24.25|41.95|19.71|2090.56|1852.74|2279.50|3943.30|74.10|0.00|1813.26|1852.74|1926.84|3666.00|3740.10|-426.76| +2450821|79439|2450854|33008|1425948|5735|27475|62423|1021449|4789|26176|2|107|9|1|6967|288|336|83|4.13|7.10|2.13|412.51|176.79|342.79|589.30|0.00|0.00|141.10|176.79|176.79|317.89|317.89|-166.00| +2450821|79439|2450901|33008|1425948|5735|27475|62423|1021449|4789|26176|2|20|20|1|17294|123|336|78|77.00|130.90|28.79|7964.58|2245.62|6006.00|10210.20|179.64|0.00|3266.64|2245.62|2425.26|5512.26|5691.90|-3760.38| +2450821|55559|2450824|57282|106418|2754|41937|57282|106418|2754|41937|1|69|9|2|11392|185|337|93|79.56|125.70|1.25|11573.85|116.25|7399.08|11690.10|10.46|0.00|4558.86|116.25|126.71|4675.11|4685.57|-7282.83| +2450821|55559|2450839|57282|106418|2754|41937|57282|106418|2754|41937|1|16|14|2|596|88|337|100|13.47|36.77|9.92|2685.00|992.00|1347.00|3677.00|79.36|0.00|845.00|992.00|1071.36|1837.00|1916.36|-355.00| +2450821|55559|2450910|57282|106418|2754|41937|57282|106418|2754|41937|1|78|6|3|343|17|337|59|88.53|146.07|119.77|1551.70|7066.43|5223.27|8618.13|353.32|0.00|258.42|7066.43|7419.75|7324.85|7678.17|1843.16| +2450821|55559|2450851|57282|106418|2754|41937|57282|106418|2754|41937|1|58|2|2|15434|223|337|92|93.93|158.74|88.89|6426.20|8177.88|8641.56|14604.08|163.55|0.00|2628.44|8177.88|8341.43|10806.32|10969.87|-463.68| +2450821|55559|2450896|57282|106418|2754|41937|57282|106418|2754|41937|1|89|17|3|12091|162|337|37|29.94|74.55|17.89|2096.42|661.93|1107.78|2758.35|6.61|0.00|192.77|661.93|668.54|854.70|861.31|-445.85| +2450821|55559|2450892|57282|106418|2754|41937|57282|106418|2754|41937|1|72|13|4|16970|8|337|99|4.49|6.91|3.38|349.47|334.62|444.51|684.09|0.00|294.46|136.62|40.16|40.16|176.78|176.78|-404.35| +2450821|55559|2450848|57282|106418|2754|41937|57282|106418|2754|41937|1|11|2|1|8744|91|337|12|10.75|21.50|2.79|224.52|33.48|129.00|258.00|2.00|0.00|2.52|33.48|35.48|36.00|38.00|-95.52| +2450821|55559|2450848|57282|106418|2754|41937|57282|106418|2754|41937|1|104|20|5|7196|40|337|38|2.17|5.31|1.16|157.70|44.08|82.46|201.78|2.20|0.00|14.06|44.08|46.28|58.14|60.34|-38.38| +2450821|55559|2450836|57282|106418|2754|41937|57282|106418|2754|41937|1|9|1|5|6398|25|337|51|44.63|71.40|39.27|1638.63|2002.77|2276.13|3641.40|80.11|0.00|946.56|2002.77|2082.88|2949.33|3029.44|-273.36| +2450821|46025|2450901|81884|317321|7158|32148|81884|317321|7158|32148|2|90|16|1|2779|171|338|93|42.09|50.08|34.55|1444.29|3213.15|3914.37|4657.44|257.05|0.00|93.00|3213.15|3470.20|3306.15|3563.20|-701.22| +2450821|46025|2450850|81884|317321|7158|32148|81884|317321|7158|32148|2|104|5|4|15338|20|338|48|78.64|111.66|56.94|2626.56|2733.12|3774.72|5359.68|136.65|0.00|2089.92|2733.12|2869.77|4823.04|4959.69|-1041.60| +2450821|46025|2450905|81884|317321|7158|32148|81884|317321|7158|32148|2|45|5|4|7141|94|338|40|34.14|46.08|41.93|166.00|1677.20|1365.60|1843.20|0.67|1643.65|700.40|33.55|34.22|733.95|734.62|-1332.05| +2450821|46025|2450823|81884|317321|7158|32148|81884|317321|7158|32148|2|97|16|3|14365|262|338|76|3.91|8.52|0.85|582.92|64.60|297.16|647.52|0.51|51.68|310.08|12.92|13.43|323.00|323.51|-284.24| +2450821|37941|2450902|17030|1101773|2516|859|17030|1101773|2516|859|1|86|17|5|14132|214|339|64|87.55|112.93|72.27|2602.24|4625.28|5603.20|7227.52|231.26|0.00|2601.60|4625.28|4856.54|7226.88|7458.14|-977.92| +2450821|37941|2450837|17030|1101773|2516|859|17030|1101773|2516|859|1|99|15|4|3532|82|339|92|59.06|145.28|15.98|11895.60|1470.16|5433.52|13365.76|8.82|1293.74|5612.92|176.42|185.24|5789.34|5798.16|-5257.10| +2450821|37941|2450899|17030|1101773|2516|859|17030|1101773|2516|859|1|9|9|3|1507|63|339|62|92.69|268.80|244.60|1500.40|15165.20|5746.78|16665.60|1213.21|0.00|5499.40|15165.20|16378.41|20664.60|21877.81|9418.42| +2450821|37941|2450885|17030|1101773|2516|859|17030|1101773|2516|859|1|34|8|3|1345|278|339|35|21.13|57.89|23.73|1195.60|830.55|739.55|2026.15|8.30|0.00|344.40|830.55|838.85|1174.95|1183.25|91.00| +2450821|37941|2450869|17030|1101773|2516|859|17030|1101773|2516|859|1|51|14|2|13021|276|339|12|12.79|19.56|5.86|164.40|70.32|153.48|234.72|0.69|62.58|46.92|7.74|8.43|54.66|55.35|-145.74| +2450821|67980|2450911|12673|1636541|7162|26230|12673|1636541|7162|26230|2|54|12|4|1690|187|340|31|72.37|131.71|115.90|490.11|3592.90|2243.47|4083.01|107.78|0.00|1877.98|3592.90|3700.68|5470.88|5578.66|1349.43| +2450821|67980|2450900|12673|1636541|7162|26230|12673|1636541|7162|26230|2|70|12|4|7687|271|340|100|4.08|6.97|6.06|91.00|606.00|408.00|697.00|6.06|0.00|348.00|606.00|612.06|954.00|960.06|198.00| +2450821|67980|2450881|12673|1636541|7162|26230|12673|1636541|7162|26230|2|101|10|3|16294|149|340|23|82.44|129.43|25.88|2381.65|595.24|1896.12|2976.89|29.76|0.00|1220.38|595.24|625.00|1815.62|1845.38|-1300.88| +2450821|67980|2450892|12673|1636541|7162|26230|12673|1636541|7162|26230|2|15|3|5|5743|172|340|86|13.23|15.08|3.92|959.76|337.12|1137.78|1296.88|23.59|0.00|258.86|337.12|360.71|595.98|619.57|-800.66| +2450821|67980|2450907|12673|1636541|7162|26230|12673|1636541|7162|26230|2|80|20|2|1796|169|340|86|21.92|58.96|54.24|405.92|4664.64|1885.12|5070.56|0.00|0.00|2433.80|4664.64|4664.64|7098.44|7098.44|2779.52| +2450821|67980|2450878|12673|1636541|7162|26230|12673|1636541|7162|26230|2|23|7|2|13598|207|340|70|16.15|36.33|15.25|1475.60|1067.50|1130.50|2543.10|10.67|0.00|1169.70|1067.50|1078.17|2237.20|2247.87|-63.00| +2450821|67980|2450849|12673|1636541|7162|26230|12673|1636541|7162|26230|2|21|19|3|12397|278|340|4|11.55|12.58|6.29|25.16|25.16|46.20|50.32|0.75|0.00|15.08|25.16|25.91|40.24|40.99|-21.04| +2450821|67980|2450835|12673|1636541|7162|26230|12673|1636541|7162|26230|2|92|9|2|15470|283|340|45|88.96|105.86|4.23|4573.35|190.35|4003.20|4763.70|3.88|93.27|380.70|97.08|100.96|477.78|481.66|-3906.12| +2450821|67980|2450870|12673|1636541|7162|26230|12673|1636541|7162|26230|2|17|5|4|11464|293|340|47|40.38|69.04|42.11|1265.71|1979.17|1897.86|3244.88|79.16|0.00|162.15|1979.17|2058.33|2141.32|2220.48|81.31| +2450821|67980|2450856|12673|1636541|7162|26230|12673|1636541|7162|26230|2|17|9|1|12650|116|340|41|9.32|26.74|6.41|833.53|262.81|382.12|1096.34|0.00|0.00|218.94|262.81|262.81|481.75|481.75|-119.31| +2450821|67980|2450880|12673|1636541|7162|26230|12673|1636541|7162|26230|2|72|18|2|4159|223|340|88|22.11|29.18|12.25|1489.84|1078.00|1945.68|2567.84|86.24|0.00|949.52|1078.00|1164.24|2027.52|2113.76|-867.68| +2450821|45273|2450832|97910|449922|5581|43993|97910|449922|5581|43993|4|40|4|2|15212|247|341|28|20.30|58.46|56.70|49.28|1587.60|568.40|1636.88|79.38|0.00|409.08|1587.60|1666.98|1996.68|2076.06|1019.20| +2450821|45273|2450906|97910|449922|5581|43993|97910|449922|5581|43993|4|57|19|5|13100|194|341|51|14.46|39.04|13.27|1314.27|676.77|737.46|1991.04|0.00|0.00|179.01|676.77|676.77|855.78|855.78|-60.69| +2450821|45273|2450857|97910|449922|5581|43993|97910|449922|5581|43993|4|105|12|4|11882|88|341|6|73.43|209.27|69.05|841.32|414.30|440.58|1255.62|8.28|0.00|577.56|414.30|422.58|991.86|1000.14|-26.28| +2450821|45273|2450892|97910|449922|5581|43993|97910|449922|5581|43993|4|89|12|1|5626|234|341|80|93.38|249.32|249.32|0.00|19945.60|7470.40|19945.60|418.85|11568.44|7180.00|8377.16|8796.01|15557.16|15976.01|906.76| +2450821|45273|2450872|97910|449922|5581|43993|97910|449922|5581|43993|4|90|1|2|15025|197|341|45|93.34|156.81|153.67|141.30|6915.15|4200.30|7056.45|207.45|0.00|1904.85|6915.15|7122.60|8820.00|9027.45|2714.85| +2450821|45273|2450840|97910|449922|5581|43993|97910|449922|5581|43993|4|87|3|1|17176|13|341|76|56.90|57.46|37.92|1485.04|2881.92|4324.40|4366.96|115.27|0.00|2183.48|2881.92|2997.19|5065.40|5180.67|-1442.48| +2450821|45273|2450894|97910|449922|5581|43993|97910|449922|5581|43993|4|19|3|1|3919|79|341|6|39.42|81.59|23.66|347.58|141.96|236.52|489.54|0.00|0.00|14.64|141.96|141.96|156.60|156.60|-94.56| +2450821|45273|2450855|97910|449922|5581|43993|97910|449922|5581|43993|4|3|10|1|1189|274|341|94|74.77|213.09|21.30|18028.26|2002.20|7028.38|20030.46|0.80|1922.11|7611.18|80.09|80.89|7691.27|7692.07|-6948.29| +2450821|45273|2450845|97910|449922|5581|43993|97910|449922|5581|43993|4|93|14|2|11659|70|341|65|27.15|38.82|20.18|1211.60|1311.70|1764.75|2523.30|13.11|0.00|958.75|1311.70|1324.81|2270.45|2283.56|-453.05| +2450821|45273|2450838|97910|449922|5581|43993|97910|449922|5581|43993|4|47|9|5|8914|300|341|12|5.92|7.69|5.92|21.24|71.04|71.04|92.28|2.84|0.00|8.28|71.04|73.88|79.32|82.16|0.00| +2450821|45273|2450891|97910|449922|5581|43993|97910|449922|5581|43993|4|58|1|3|2239|22|341|80|46.76|129.05|34.84|7536.80|2787.20|3740.80|10324.00|222.97|0.00|4748.80|2787.20|3010.17|7536.00|7758.97|-953.60| +2450821|45273|2450825|97910|449922|5581|43993|97910|449922|5581|43993|4|40|18|5|14305|149|341|81|68.86|117.06|30.43|7017.03|2464.83|5577.66|9481.86|172.53|0.00|1516.32|2464.83|2637.36|3981.15|4153.68|-3112.83| +2450821|45273|2450867|97910|449922|5581|43993|97910|449922|5581|43993|4|86|18|4|6214|89|341|89|81.70|133.17|94.55|3437.18|8414.95|7271.30|11852.13|504.89|0.00|1540.59|8414.95|8919.84|9955.54|10460.43|1143.65| +2450821|75432|2450897|29519|1112782|398|36861|29519|1112782|398|36861|4|87|18|4|12356|98|342|42|94.88|168.88|40.53|5390.70|1702.26|3984.96|7092.96|0.00|1463.94|141.54|238.32|238.32|379.86|379.86|-3746.64| +2450821|75432|2450911|29519|1112782|398|36861|29519|1112782|398|36861|4|10|7|2|4544|23|342|83|94.19|136.57|45.06|7595.33|3739.98|7817.77|11335.31|224.39|0.00|4080.28|3739.98|3964.37|7820.26|8044.65|-4077.79| +2450821|75432|2450868|29519|1112782|398|36861|29519|1112782|398|36861|4|105|14|1|10036|212|342|18|1.91|5.30|1.80|63.00|32.40|34.38|95.40|0.32|0.00|16.20|32.40|32.72|48.60|48.92|-1.98| +2450821|75432|2450841|29519|1112782|398|36861|29519|1112782|398|36861|4|7|14|3|15109|8|342|86|2.07|2.17|0.78|119.54|67.08|178.02|186.62|0.55|39.57|67.08|27.51|28.06|94.59|95.14|-150.51| +2450821|75432|2450853|29519|1112782|398|36861|29519|1112782|398|36861|4|51|6|1|13741|215|342|65|49.71|88.98|60.50|1851.20|3932.50|3231.15|5783.70|39.32|0.00|2197.65|3932.50|3971.82|6130.15|6169.47|701.35| +2450821|75432|2450830|29519|1112782|398|36861|29519|1112782|398|36861|4|92|3|3|1417|126|342|12|83.80|243.85|185.32|702.36|2223.84|1005.60|2926.20|200.14|0.00|1346.04|2223.84|2423.98|3569.88|3770.02|1218.24| +2450821|74798|2450833|28274|1707386|4098|10410|57273|1851834|5346|3258|4|75|18|3|7484|120|343|58|43.08|68.06|40.83|1579.34|2368.14|2498.64|3947.48|71.04|0.00|1894.28|2368.14|2439.18|4262.42|4333.46|-130.50| +2450821|74798|2450835|28274|1707386|4098|10410|57273|1851834|5346|3258|4|20|1|2|12380|234|343|84|86.50|176.46|142.93|2816.52|12006.12|7266.00|14822.64|960.48|0.00|3112.20|12006.12|12966.60|15118.32|16078.80|4740.12| +2450821|74798|2450833|28274|1707386|4098|10410|57273|1851834|5346|3258|4|102|12|5|10688|161|343|35|63.42|159.81|51.13|3803.80|1789.55|2219.70|5593.35|143.16|0.00|2349.20|1789.55|1932.71|4138.75|4281.91|-430.15| +2450821|74798|2450903|28274|1707386|4098|10410|57273|1851834|5346|3258|4|108|9|2|643|115|343|87|54.18|89.93|21.58|5946.45|1877.46|4713.66|7823.91|0.00|0.00|77.43|1877.46|1877.46|1954.89|1954.89|-2836.20| +2450821|74798|2450873|28274|1707386|4098|10410|57273|1851834|5346|3258|4|37|5|3|433|202|343|92|67.14|135.62|31.19|9607.56|2869.48|6176.88|12477.04|58.82|1692.99|747.96|1176.49|1235.31|1924.45|1983.27|-5000.39| +2450821|74798|2450826|28274|1707386|4098|10410|57273|1851834|5346|3258|4|83|5|5|6878|255|343|12|73.62|164.17|164.17|0.00|1970.04|883.44|1970.04|98.50|0.00|886.44|1970.04|2068.54|2856.48|2954.98|1086.60| +2450821|74798|2450894|28274|1707386|4098|10410|57273|1851834|5346|3258|4|7|4|5|12578|170|343|38|20.88|38.21|28.27|377.72|1074.26|793.44|1451.98|32.22|0.00|159.60|1074.26|1106.48|1233.86|1266.08|280.82| +2450821|74798|2450898|28274|1707386|4098|10410|57273|1851834|5346|3258|4|101|18|4|2371|257|343|15|34.23|53.74|17.19|548.25|257.85|513.45|806.10|15.47|0.00|185.40|257.85|273.32|443.25|458.72|-255.60| +2450821|74798|2450903|28274|1707386|4098|10410|57273|1851834|5346|3258|4|12|18|3|7462|107|343|54|55.40|151.24|36.29|6207.30|1959.66|2991.60|8166.96|53.30|627.09|2123.28|1332.57|1385.87|3455.85|3509.15|-1659.03| +2450821|74798|2450894|28274|1707386|4098|10410|57273|1851834|5346|3258|4|95|14|4|4351|129|343|81|28.25|59.04|14.16|3635.28|1146.96|2288.25|4782.24|80.28|0.00|1482.30|1146.96|1227.24|2629.26|2709.54|-1141.29| +2450821|68827|2450876|65538|562105|272|47239|7232|1593620|6187|23154|4|66|17|3|6164|5|344|30|46.38|130.79|82.39|1452.00|2471.70|1391.40|3923.70|24.71|0.00|1922.40|2471.70|2496.41|4394.10|4418.81|1080.30| +2450821|68827|2450903|65538|562105|272|47239|7232|1593620|6187|23154|4|15|9|5|15712|124|344|98|26.83|30.31|4.24|2554.86|415.52|2629.34|2970.38|0.37|403.05|920.22|12.47|12.84|932.69|933.06|-2616.87| +2450821|68827|2450844|65538|562105|272|47239|7232|1593620|6187|23154|4|14|17|4|12985|106|344|76|24.08|60.68|9.10|3920.08|691.60|1830.08|4611.68|13.83|0.00|1475.16|691.60|705.43|2166.76|2180.59|-1138.48| +2450821|68827|2450838|65538|562105|272|47239|7232|1593620|6187|23154|4|106|20|4|17104|40|344|92|40.31|45.55|0.91|4106.88|83.72|3708.52|4190.60|2.41|53.58|1591.60|30.14|32.55|1621.74|1624.15|-3678.38| +2450821|68827|2450853|65538|562105|272|47239|7232|1593620|6187|23154|4|31|9|5|3157|132|344|71|65.65|139.17|115.51|1679.86|8201.21|4661.15|9881.07|328.04|0.00|3260.32|8201.21|8529.25|11461.53|11789.57|3540.06| +2450821|68827|2450864|65538|562105|272|47239|7232|1593620|6187|23154|4|71|8|2|9448|89|344|15|62.16|116.86|19.86|1455.00|297.90|932.40|1752.90|5.09|241.29|473.25|56.61|61.70|529.86|534.95|-875.79| +2450821|68827|2450842|65538|562105|272|47239|7232|1593620|6187|23154|4|18|13|5|6248|182|344|52|83.45|163.56|76.87|4507.88|3997.24|4339.40|8505.12|79.94|0.00|3657.16|3997.24|4077.18|7654.40|7734.34|-342.16| +2450821|41843|2450826|93222|1556974|1526|14925|93222|1556974|1526|14925|1|71|1|1|9010|44|345|78|17.78|18.13|9.42|679.38|734.76|1386.84|1414.14|3.15|418.81|635.70|315.95|319.10|951.65|954.80|-1070.89| +2450821|41843|2450904|93222|1556974|1526|14925|93222|1556974|1526|14925|1|73|17|4|16178|115|345|53|46.48|84.12|53.83|1605.37|2852.99|2463.44|4458.36|114.11|0.00|133.56|2852.99|2967.10|2986.55|3100.66|389.55| +2450821|41843|2450887|93222|1556974|1526|14925|93222|1556974|1526|14925|1|96|7|4|16528|106|345|31|5.60|13.94|6.41|233.43|198.71|173.60|432.14|15.89|0.00|99.20|198.71|214.60|297.91|313.80|25.11| +2450821|41843|2450875|93222|1556974|1526|14925|93222|1556974|1526|14925|1|93|18|4|3241|299|345|35|34.76|40.32|14.11|917.35|493.85|1216.60|1411.20|19.75|0.00|620.90|493.85|513.60|1114.75|1134.50|-722.75| +2450821|41843|2450830|93222|1556974|1526|14925|93222|1556974|1526|14925|1|92|16|3|4784|267|345|36|10.38|23.04|2.07|754.92|74.52|373.68|829.44|0.98|25.33|124.20|49.19|50.17|173.39|174.37|-324.49| +2450821|41843|2450827|93222|1556974|1526|14925|93222|1556974|1526|14925|1|15|11|1|5002|158|345|34|20.71|54.05|5.40|1654.10|183.60|704.14|1837.70|3.67|0.00|330.48|183.60|187.27|514.08|517.75|-520.54| +2450821|41843|2450844|93222|1556974|1526|14925|93222|1556974|1526|14925|1|88|17|3|4936|217|345|35|47.22|80.27|12.84|2360.05|449.40|1652.70|2809.45|31.45|0.00|112.35|449.40|480.85|561.75|593.20|-1203.30| +2450821|41843|2450875|93222|1556974|1526|14925|93222|1556974|1526|14925|1|17|5|3|4382|226|345|15|36.48|89.37|17.87|1072.50|268.05|547.20|1340.55|21.44|0.00|562.95|268.05|289.49|831.00|852.44|-279.15| +2450821|41843|2450904|93222|1556974|1526|14925|93222|1556974|1526|14925|1|72|5|2|17491|123|345|1|23.32|47.80|15.77|32.03|15.77|23.32|47.80|0.94|0.00|20.55|15.77|16.71|36.32|37.26|-7.55| +2450821|41843|2450898|93222|1556974|1526|14925|93222|1556974|1526|14925|1|63|8|4|17275|125|345|13|82.68|125.67|91.73|441.22|1192.49|1074.84|1633.71|31.83|131.17|408.33|1061.32|1093.15|1469.65|1501.48|-13.52| +2450821|41843|2450894|93222|1556974|1526|14925|93222|1556974|1526|14925|1|69|3|3|314|114|345|31|35.55|45.14|43.33|56.11|1343.23|1102.05|1399.34|80.59|0.00|349.68|1343.23|1423.82|1692.91|1773.50|241.18| +|41843||93222|||14925|93222|||14925|1||12||8173||345|50|12.14|||664.50||607.00|1444.50|18.25|475.80||||564.20|582.45|-302.80| +2450821|41843|2450832|93222|1556974|1526|14925|93222|1556974|1526|14925|1|13|11|1|5876|58|345|50|78.74|127.55|113.51|702.00|5675.50|3937.00|6377.50|113.51|0.00|1084.00|5675.50|5789.01|6759.50|6873.01|1738.50| +2450821|41843|2450907|93222|1556974|1526|14925|93222|1556974|1526|14925|1|51|11|2|12385|154|345|89|96.32|153.14|52.06|8996.12|4633.34|8572.48|13629.46|278.00|0.00|5860.65|4633.34|4911.34|10493.99|10771.99|-3939.14| +2450821|30875|2450839|4658|1509815|3772|10082|4658|1509815|3772|10082|1|69|15|2|5954|23|346|12|96.48|160.15|115.30|538.20|1383.60|1157.76|1921.80|9.96|1217.56|384.36|166.04|176.00|550.40|560.36|-991.72| +2450821|30875|2450877|4658|1509815|3772|10082|4658|1509815|3772|10082|1|7|7|3|3790|209|346|84|13.61|35.52|6.39|2446.92|536.76|1143.24|2983.68|48.30|0.00|89.04|536.76|585.06|625.80|674.10|-606.48| +2450821|30875|2450883|4658|1509815|3772|10082|4658|1509815|3772|10082|1|87|5|2|11710|65|346|12|59.38|64.72|62.13|31.08|745.56|712.56|776.64|37.27|0.00|108.72|745.56|782.83|854.28|891.55|33.00| +2450821|30875|2450825|4658|1509815|3772|10082|4658|1509815|3772|10082|1|79|15|3|12697|22|346|16|55.99|141.65|32.57|1745.28|521.12|895.84|2266.40|5.21|0.00|249.28|521.12|526.33|770.40|775.61|-374.72| +2450821|30875|2450880|4658|1509815|3772|10082|4658|1509815|3772|10082|1|46|5|1|7948|20|346|81|85.71|148.27|29.65|9608.22|2401.65|6942.51|12009.87|72.04|0.00|3482.19|2401.65|2473.69|5883.84|5955.88|-4540.86| +2450821|30875|2450886|4658|1509815|3772|10082|4658|1509815|3772|10082|1|106|12|4|5878|78|346|4|97.87|150.71|61.79|355.68|247.16|391.48|602.84|4.94|0.00|0.00|247.16|252.10|247.16|252.10|-144.32| +2450821|35454|2450906|77447|96213|132|23251|77447|96213|132|23251|2|42|20|5|2540|196|347|48|54.66|135.55|84.04|2472.48|4033.92|2623.68|6506.40|161.35|0.00|1431.36|4033.92|4195.27|5465.28|5626.63|1410.24| +2450821|35454|2450845|77447|96213|132|23251|77447|96213|132|23251|2|85|4|1|1016|162|347|22|8.49|22.92|7.10|348.04|156.20|186.78|504.24|0.00|0.00|25.08|156.20|156.20|181.28|181.28|-30.58| +2450821|35454|2450851|77447|96213|132|23251|77447|96213|132|23251|2|5|3|2|11182|254|347|27|58.38|173.38|140.43|889.65|3791.61|1576.26|4681.26|303.32|0.00|2106.54|3791.61|4094.93|5898.15|6201.47|2215.35| +2450821|35454|2450858|77447|96213|132|23251|77447|96213|132|23251|2|38|6|2|2456|130|347|41|62.29|180.64|75.86|4295.98|3110.26|2553.89|7406.24|0.00|0.00|3184.47|3110.26|3110.26|6294.73|6294.73|556.37| +2450821|35454|2450885|77447|96213|132|23251|77447|96213|132|23251|2|101|20|2|2018|172|347|24|84.61|155.68|140.11|373.68|3362.64|2030.64|3736.32|134.50|0.00|1419.60|3362.64|3497.14|4782.24|4916.74|1332.00| +2450821|35454|2450884|77447|96213|132|23251|77447|96213|132|23251|2|67|5|2|8660|81|347|54|73.59|142.02|56.80|4601.88|3067.20|3973.86|7669.08|276.04|0.00|690.12|3067.20|3343.24|3757.32|4033.36|-906.66| +2450821|34823|2450897|77547|244881|1462|5196|77547|244881|1462|5196|4|50|19|4|3178|256|348|53|23.34|60.91|23.75|1969.48|1258.75|1237.02|3228.23|0.00|0.00|1323.41|1258.75|1258.75|2582.16|2582.16|21.73| +2450821|34823|2450887|77547|244881|1462|5196|77547|244881|1462|5196|4|102|4|4|3278|236|348|96|81.48|232.21|25.54|19840.32|2451.84|7822.08|22292.16|122.59|0.00|3789.12|2451.84|2574.43|6240.96|6363.55|-5370.24| +2450821|34823|2450890|77547|244881|1462|5196|77547|244881|1462|5196|4|69|9|3|10288|96|348|37|66.93|97.04|24.26|2692.86|897.62|2476.41|3590.48|35.90|0.00|1723.09|897.62|933.52|2620.71|2656.61|-1578.79| +2450821|34823|2450908|77547|244881|1462|5196|77547|244881|1462|5196|4|5|4|2|6133|13|348|94|37.09|94.57|72.81|2045.44|6844.14|3486.46|8889.58|342.20|0.00|0.00|6844.14|7186.34|6844.14|7186.34|3357.68| +2450821|34823|2450824|77547|244881|1462|5196|77547|244881|1462|5196|4|21|10|3|3355|101|348|97|93.63|220.03|107.81|10885.34|10457.57|9082.11|21342.91|0.00|0.00|2987.60|10457.57|10457.57|13445.17|13445.17|1375.46| +2450821|34823|2450829|77547|244881|1462|5196|77547|244881|1462|5196|4|77|3|2|8215|5|348|48|29.58|38.15|26.32|567.84|1263.36|1419.84|1831.20|113.70|0.00|347.52|1263.36|1377.06|1610.88|1724.58|-156.48| +2450821|34823|2450852|77547|244881|1462|5196|77547|244881|1462|5196|4|29|12|1|8005|67|348|50|66.69|113.37|11.33|5102.00|566.50|3334.50|5668.50|33.99|0.00|170.00|566.50|600.49|736.50|770.49|-2768.00| +2450821|34823|2450907|77547|244881|1462|5196|77547|244881|1462|5196|4|11|12|5|6416|133|348|35|27.66|48.68|19.95|1005.55|698.25|968.10|1703.80|48.87|0.00|834.75|698.25|747.12|1533.00|1581.87|-269.85| +2450821|59564|2450873|29624|280947|6223|26346|29624|280947|6223|26346|1|54|5|4|12883|117|349|12|12.70|34.79|27.83|83.52|333.96|152.40|417.48|0.00|0.00|33.36|333.96|333.96|367.32|367.32|181.56| +2450821|59564|2450841|29624|280947|6223|26346|29624|280947|6223|26346|1|46|1|3|7000|297|349|47|96.21|191.45|189.53|90.24|8907.91|4521.87|8998.15|534.47|0.00|1079.59|8907.91|9442.38|9987.50|10521.97|4386.04| +2450821|59564|2450881|29624|280947|6223|26346|29624|280947|6223|26346|1|15|14|2|13966|270|349|32|77.08|127.95|47.34|2579.52|1514.88|2466.56|4094.40|60.59|0.00|982.40|1514.88|1575.47|2497.28|2557.87|-951.68| +2450821|59564|2450843|29624|280947|6223|26346|29624|280947|6223|26346|1|12|12|2|7052|284|349|37|12.71|28.59|15.15|497.28|560.55|470.27|1057.83|3.02|526.91|21.09|33.64|36.66|54.73|57.75|-436.63| +2450821|59564|2450828|29624|280947|6223|26346|29624|280947|6223|26346|1|24|11|5|626|201|349|8|53.40|128.16|52.54|604.96|420.32|427.20|1025.28|7.81|159.72|512.64|260.60|268.41|773.24|781.05|-166.60| +2450821|59564|2450865|29624|280947|6223|26346|29624|280947|6223|26346|1|7|9|3|10255|38|349|16|34.11|62.42|45.56|269.76|728.96|545.76|998.72|43.73|0.00|99.84|728.96|772.69|828.80|872.53|183.20| +2450821|44866|2450853|60915|328208|1101|46347|60915|328208|1101|46347|4|14|11|4|2468|65|350|18|81.49|90.45|61.50|521.10|1107.00|1466.82|1628.10|33.21|0.00|276.66|1107.00|1140.21|1383.66|1416.87|-359.82| +2450821|44866|2450902|60915|328208|1101|46347|60915|328208|1101|46347|4|10|9|2|16004|260|350|62|86.22|180.19|79.28|6256.42|4915.36|5345.64|11171.78|268.37|442.38|3351.10|4472.98|4741.35|7824.08|8092.45|-872.66| +2450821|44866|2450862|60915|328208|1101|46347|60915|328208|1101|46347|4|29|18|4|11221|1|350|66|8.02|11.86|7.94|258.72|524.04|529.32|782.76|5.24|0.00|116.82|524.04|529.28|640.86|646.10|-5.28| +2450821|44866|2450854|60915|328208|1101|46347|60915|328208|1101|46347|4|8|3|4|15976|294|350|24|39.88|82.55|34.67|1149.12|832.08|957.12|1981.20|16.64|0.00|752.64|832.08|848.72|1584.72|1601.36|-125.04| +2450821|52990|2450839|37624|918173|5122|2071|37624|918173|5122|2071|4|55|17|3|9418|67|351|85|75.00|126.00|18.90|9103.50|1606.50|6375.00|10710.00|16.06|0.00|1927.80|1606.50|1622.56|3534.30|3550.36|-4768.50| +2450821|52990|2450838|37624|918173|5122|2071|37624|918173|5122|2071|4|38|11|2|16870|147|351|11|93.71|245.52|238.15|81.07|2619.65|1030.81|2700.72|0.00|0.00|864.16|2619.65|2619.65|3483.81|3483.81|1588.84| +2450821|52990|2450859|37624|918173|5122|2071|37624|918173|5122|2071|4|68|14|3|1255|46|351|4|9.06|19.84|9.32|42.08|37.28|36.24|79.36|3.35|0.00|3.96|37.28|40.63|41.24|44.59|1.04| +2450821|52990|2450897|37624|918173|5122|2071|37624|918173|5122|2071|4|92|14|3|9752|74|351|6|55.43|59.31|45.07|85.44|270.42|332.58|355.86|11.68|75.71|128.10|194.71|206.39|322.81|334.49|-137.87| +2450821|52990|2450838|37624|918173|5122|2071|37624|918173|5122|2071|4|63|1|4|3272|75|351|34|98.76|216.28|4.32|7206.64|146.88|3357.84|7353.52|13.21|0.00|1102.96|146.88|160.09|1249.84|1263.05|-3210.96| +2450821|52990|2450829|37624|918173|5122|2071|37624|918173|5122|2071|4|18|19|3|4988|284|351|62|33.88|65.72|50.60|937.44|3137.20|2100.56|4074.64|0.00|0.00|1914.56|3137.20|3137.20|5051.76|5051.76|1036.64| +2450821|52990|2450875|37624|918173|5122|2071|37624|918173|5122|2071|4|49|5|2|7843|128|351|51|30.60|78.33|47.78|1558.05|2436.78|1560.60|3994.83|60.91|1218.39|519.18|1218.39|1279.30|1737.57|1798.48|-342.21| +2450821|52990|2450865|37624|918173|5122|2071|37624|918173|5122|2071|4|1|12|3|536|272|351|51|50.01|53.51|13.91|2019.60|709.41|2550.51|2729.01|7.09|0.00|272.85|709.41|716.50|982.26|989.35|-1841.10| +2450821|52990|2450831|37624|918173|5122|2071|37624|918173|5122|2071|4|98|14|5|10466|23|351|63|7.56|12.77|9.06|233.73|570.78|476.28|804.51|11.41|0.00|0.00|570.78|582.19|570.78|582.19|94.50| +2450821|52990|2450908|37624|918173|5122|2071|37624|918173|5122|2071|4|48|11|2|9682|114|351|83|17.52|33.98|15.29|1551.27|1269.07|1454.16|2820.34|0.00|0.00|224.93|1269.07|1269.07|1494.00|1494.00|-185.09| +2450821|46011|2450830|50856|1547754|2252|5903|50856|1547754|2252|5903|2|94|7|2|2782|180|352|73|36.46|66.72|37.36|2143.28|2727.28|2661.58|4870.56|218.18|0.00|1801.64|2727.28|2945.46|4528.92|4747.10|65.70| +|||50856||2252||50856||||2||19|5|13033||352||39.05|78.10|11.71|3518.67|620.63|2069.65||24.82||||645.45|1448.49|1473.31|-1449.02| +2450821|46011|2450826|50856|1547754|2252|5903|50856|1547754|2252|5903|2|103|18|4|1819|296|352|81|86.88|239.78|220.59|1554.39|17867.79|7037.28|19422.18|250.14|14294.23|6603.12|3573.56|3823.70|10176.68|10426.82|-3463.72| +2450821|46011|2450858|50856|1547754|2252|5903|50856|1547754|2252|5903|2|12|6|5|760|190|352|44|68.80|156.86|123.91|1449.80|5452.04|3027.20|6901.84|82.87|3380.26|137.72|2071.78|2154.65|2209.50|2292.37|-955.42| +2450821|82647|2450853|67671|1211509|6547|37585|55303|1014508|2346|47784|2|66|12|3|32|46|353|13|91.33|155.26|93.15|807.43|1210.95|1187.29|2018.38|72.65|0.00|766.87|1210.95|1283.60|1977.82|2050.47|23.66| +2450821|82647|2450866|67671|1211509|6547|37585|55303|1014508|2346|47784|2|43|19|2|2756|135|353|80|30.28|42.08|39.97|168.80|3197.60|2422.40|3366.40|287.78|0.00|33.60|3197.60|3485.38|3231.20|3518.98|775.20| +2450821|82647|2450883|67671|1211509|6547|37585|55303|1014508|2346|47784|2|108|7|4|13528|188|353|39|60.05|144.72|127.35|677.43|4966.65|2341.95|5644.08|347.66|0.00|1072.11|4966.65|5314.31|6038.76|6386.42|2624.70| +2450821|82647|2450911|67671|1211509|6547|37585|55303|1014508|2346|47784|2|2|5|3|1297|87|353|72|17.40|44.89|25.58|1390.32|1841.76|1252.80|3232.08|73.67|0.00|936.72|1841.76|1915.43|2778.48|2852.15|588.96| +2450821|82647|2450883|67671|1211509|6547|37585|55303|1014508|2346|47784|2|96|19|2|5332|240|353|36|36.80|57.04|24.52|1170.72|882.72|1324.80|2053.44|35.30|0.00|328.32|882.72|918.02|1211.04|1246.34|-442.08| +2450821|82647|2450870|67671|1211509|6547|37585|55303|1014508|2346|47784|2|50|5|5|1058|33|353|100|81.20|193.25|0.00|19325.00|0.00|8120.00|19325.00|0.00|0.00|2125.00|0.00|0.00|2125.00|2125.00|-8120.00| +2450821|82647|2450826|67671|1211509|6547|37585|55303|1014508|2346|47784|2|54|8|3|4435|58|353|61|13.07|32.67|2.28|1853.79|139.08|797.27|1992.87|6.95|0.00|816.79|139.08|146.03|955.87|962.82|-658.19| +2450821|82647|2450890|67671|1211509|6547|37585|55303|1014508|2346|47784|2|40|9|3|5080|266|353|83|91.85|225.95|42.93|15190.66|3563.19|7623.55|18753.85|249.42|0.00|8063.45|3563.19|3812.61|11626.64|11876.06|-4060.36| +2450821|82647|2450834|67671|1211509|6547|37585|55303|1014508|2346|47784|2|18|13|4|5776|69|353|99|45.62|110.40|85.00|2514.60|8415.00|4516.38|10929.60|84.15|0.00|2403.72|8415.00|8499.15|10818.72|10902.87|3898.62| +2450821|82647|2450911|67671|1211509|6547|37585|55303|1014508|2346|47784|2|1|19|5|7304|32|353|69|69.15|80.90|63.10|1228.20|4353.90|4771.35|5582.10|85.33|3134.80|2791.05|1219.10|1304.43|4010.15|4095.48|-3552.25| +2450821|82647|2450824|67671|1211509|6547|37585|55303|1014508|2346|47784|2|87|3|3|1556|218|353|76|46.32|56.04|33.06|1746.48|2512.56|3520.32|4259.04|226.13|0.00|1873.40|2512.56|2738.69|4385.96|4612.09|-1007.76| +2450821|50281|2450907|63155|753853|6151|6002|63155|753853|6151|6002|2|75|8|4|9106|158|354|93|1.81|3.09|2.81|26.04|261.33|168.33|287.37|23.51|0.00|122.76|261.33|284.84|384.09|407.60|93.00| +2450821|50281|2450896|63155|753853|6151|6002|63155|753853|6151|6002|2|97|18|2|1042|167|354|75|37.83|99.49|41.78|4328.25|3133.50|2837.25|7461.75|125.34|0.00|3432.00|3133.50|3258.84|6565.50|6690.84|296.25| +2450821|50281|2450823|63155|753853|6151|6002|63155|753853|6151|6002|2|36|4|5|11713|170|354|58|75.12|94.65|8.51|4996.12|493.58|4356.96|5489.70|4.93|0.00|713.40|493.58|498.51|1206.98|1211.91|-3863.38| +2450821|50281|2450877|63155|753853|6151|6002|63155|753853|6151|6002|2|67|13|5|16501|112|354|46|12.18|33.61|13.10|943.46|602.60|560.28|1546.06|6.02|0.00|618.24|602.60|608.62|1220.84|1226.86|42.32| +2450821|50281|2450911|63155|753853|6151|6002|63155|753853|6151|6002|2|96|14|2|464|245|354|55|18.04|36.26|6.16|1655.50|338.80|992.20|1994.30|27.10|0.00|378.40|338.80|365.90|717.20|744.30|-653.40| +2450821|50281|2450860|63155|753853|6151|6002|63155|753853|6151|6002|2|47|14|5|11683|9|354|35|81.84|189.86|79.74|3854.20|2790.90|2864.40|6645.10|195.36|0.00|132.65|2790.90|2986.26|2923.55|3118.91|-73.50| +2450821|50281|2450864|63155|753853|6151|6002|63155|753853|6151|6002|2|27|17|3|16144|173|354|64|66.46|99.02|89.11|634.24|5703.04|4253.44|6337.28|114.06|0.00|380.16|5703.04|5817.10|6083.20|6197.26|1449.60| +2450821|38039|2450908|33728|1443345|4162|49779|33728|1443345|4162|49779|4|58|9|5|14354|173|355|14|66.28|91.46|63.10|397.04|883.40|927.92|1280.44|53.00|0.00|51.10|883.40|936.40|934.50|987.50|-44.52| +2450821|38039|2450826|33728|1443345|4162|49779|33728|1443345|4162|49779|4|98|12|4|12103|98|355|27|74.79|207.91|31.18|4771.71|841.86|2019.33|5613.57|56.82|210.46|2020.68|631.40|688.22|2652.08|2708.90|-1387.93| +2450821|38039|2450824|33728|1443345|4162|49779|33728|1443345|4162|49779|4|57|20|3|7292|99|355|42|59.05|73.81|8.11|2759.40|340.62|2480.10|3100.02|0.00|0.00|1518.72|340.62|340.62|1859.34|1859.34|-2139.48| +2450821|38039|2450878|33728|1443345|4162|49779|33728|1443345|4162|49779|4|8|11|5|3643|253|355|78|87.57|132.23|130.90|103.74|10210.20|6830.46|10313.94|20.42|9801.79|1340.04|408.41|428.83|1748.45|1768.87|-6422.05| +2450821|38039|2450890|33728|1443345|4162|49779|33728|1443345|4162|49779|4|1|2|4|3716|63|355|46|23.02|48.80|41.96|314.64|1930.16|1058.92|2244.80|115.80|0.00|561.20|1930.16|2045.96|2491.36|2607.16|871.24| +2450821|38039|2450893|33728|1443345|4162|49779|33728|1443345|4162|49779|4|42|6|4|15103|240|355|38|80.36|187.23|149.78|1423.10|5691.64|3053.68|7114.74|398.41|0.00|1565.22|5691.64|6090.05|7256.86|7655.27|2637.96| +2450821|38039|2450852|33728|1443345|4162|49779|33728|1443345|4162|49779|4|26|2|3|6811|278|355|47|31.16|86.62|24.25|2931.39|1139.75|1464.52|4071.14|0.00|0.00|1180.17|1139.75|1139.75|2319.92|2319.92|-324.77| +2450821|9453|2450839|29126|975426|2526|44873|29126|975426|2526|44873|2|41|13|2|5800|23|356|78|46.62|81.58|8.97|5663.58|699.66|3636.36|6363.24|0.00|0.00|2799.42|699.66|699.66|3499.08|3499.08|-2936.70| +2450821|9453|2450883|29126|975426|2526|44873|29126|975426|2526|44873|2|23|4|2|12769|128|356|82|92.52|223.89|136.57|7160.24|11198.74|7586.64|18358.98|671.92|0.00|5139.76|11198.74|11870.66|16338.50|17010.42|3612.10| +2450821|9453|2450883|29126|975426|2526|44873|29126|975426|2526|44873|2|85|10|4|3691|20|356|14|25.11|62.77|20.71|588.84|289.94|351.54|878.78|5.79|0.00|43.82|289.94|295.73|333.76|339.55|-61.60| +2450821|9453|2450876|29126|975426|2526|44873|29126|975426|2526|44873|2|16|7|2|9958|138|356|54|64.38|159.01|3.18|8414.82|171.72|3476.52|8586.54|10.30|0.00|2232.36|171.72|182.02|2404.08|2414.38|-3304.80| +2450821|9453|2450840|29126|975426|2526|44873|29126|975426|2526|44873|2|83|7|2|986|230|356|38|27.01|45.37|42.19|120.84|1603.22|1026.38|1724.06|144.28|0.00|310.08|1603.22|1747.50|1913.30|2057.58|576.84| +2450821|15409|2450908|30393|123740|4734|33681|30393|123740|4734|33681|4|4|15|3|8914|139|357|28|92.30|272.28|92.57|5031.88|2591.96|2584.40|7623.84|181.43|0.00|2973.04|2591.96|2773.39|5565.00|5746.43|7.56| +2450821|15409|2450855|30393|123740|4734|33681|30393|123740|4734|33681|4|73|17|4|2239|115|357|25|50.65|120.04|106.83|330.25|2670.75|1266.25|3001.00|156.23|934.76|1020.25|1735.99|1892.22|2756.24|2912.47|469.74| +2450821|15409|2450867|30393|123740|4734|33681|30393|123740|4734|33681|4|45|20|4|14305|114|357|76|87.10|209.04|204.85|318.44|15568.60|6619.60|15887.04|467.05|0.00|3812.16|15568.60|16035.65|19380.76|19847.81|8949.00| +2450821|15409|2450837|30393|123740|4734|33681|30393|123740|4734|33681|4|19|7|4|6214|219|357|7|63.53|76.87|61.49|107.66|430.43|444.71|538.09|25.82|0.00|247.52|430.43|456.25|677.95|703.77|-14.28| +2450821|15409|2450836|30393|123740|4734|33681|30393|123740|4734|33681|4|17|9|2|7258|236|357|74|82.48|188.87|98.21|6708.84|7267.54|6103.52|13976.38|37.79|6322.75|0.00|944.79|982.58|944.79|982.58|-5158.73| +2450821|15409|2450870|30393|123740|4734|33681|30393|123740|4734|33681|4|30|7|3|295|296|357|92|84.89|235.99|115.63|11073.12|10637.96|7809.88|21711.08|851.03|0.00|3255.88|10637.96|11488.99|13893.84|14744.87|2828.08| +2450821|15409|2450854|30393|123740|4734|33681|30393|123740|4734|33681|4|29|2|5|17842|143|357|65|39.36|83.04|22.42|3940.30|1457.30|2558.40|5397.60|116.58|0.00|1565.20|1457.30|1573.88|3022.50|3139.08|-1101.10| +2450821|15409|2450900|30393|123740|4734|33681|30393|123740|4734|33681|4|41|3|1|5383|189|357|85|82.86|99.43|80.53|1606.50|6845.05|7043.10|8451.55|136.90|0.00|168.30|6845.05|6981.95|7013.35|7150.25|-198.05| +2450821|15409|2450867|30393|123740|4734|33681|30393|123740|4734|33681|4|22|4|5|9715|230|357|35|21.63|24.22|19.13|178.15|669.55|757.05|847.70|37.36|46.86|109.90|622.69|660.05|732.59|769.95|-134.36| +2450821|15409|2450855|30393|123740|4734|33681|30393|123740|4734|33681|4|69|7|4|3680|200|357|99|10.63|13.81|5.38|834.57|532.62|1052.37|1367.19|47.93|0.00|436.59|532.62|580.55|969.21|1017.14|-519.75| +2450821|35886|2450887|29877|1732666|2655|19844|29877|1732666|2655|19844|1|93|19|4|2410|261|358|33|96.59|133.29|69.31|2111.34|2287.23|3187.47|4398.57|68.61|0.00|1451.34|2287.23|2355.84|3738.57|3807.18|-900.24| +2450821|35886|2450896|29877|1732666|2655|19844|29877|1732666|2655|19844|1|38|11|2|1177|193|358|79|22.37|39.37|3.54|2830.57|279.66|1767.23|3110.23|0.00|279.66|372.88|0.00|0.00|372.88|372.88|-1767.23| +2450821|35886|2450892|29877|1732666|2655|19844|29877|1732666|2655|19844|1|97|5|3|1465|235|358|98|10.22|21.05|9.47|1134.84|928.06|1001.56|2062.90|27.84|0.00|144.06|928.06|955.90|1072.12|1099.96|-73.50| +2450821|35886|2450831|29877|1732666|2655|19844|29877|1732666|2655|19844|1|95|11|4|9032|297|358|74|66.31|72.94|41.57|2321.38|3076.18|4906.94|5397.56|153.80|0.00|1349.02|3076.18|3229.98|4425.20|4579.00|-1830.76| +2450821|35886|2450870|29877|1732666|2655|19844|29877|1732666|2655|19844|1|56|1|5|9406|26|358|68|60.26|103.64|47.67|3805.96|3241.56|4097.68|7047.52|129.66|0.00|3382.32|3241.56|3371.22|6623.88|6753.54|-856.12| +2450822|70030|2450849|78525|544217|309|44801|78525|544217|309|44801|4|76|2|1|1051|156|359|34|50.91|96.21|0.00|3271.14|0.00|1730.94|3271.14|0.00|0.00|1079.16|0.00|0.00|1079.16|1079.16|-1730.94| +2450822|70030|2450828|78525|544217|309|44801|78525|544217|309|44801|4|81|16|1|2749|113|359|19|83.54|169.58|89.87|1514.49|1707.53|1587.26|3222.02|0.00|0.00|64.41|1707.53|1707.53|1771.94|1771.94|120.27| +2450822|70030|2450867|78525|544217|309|44801|78525|544217|309|44801|4|54|2|4|4910|113|359|97|37.83|39.72|27.00|1233.84|2619.00|3669.51|3852.84|235.71|0.00|1926.42|2619.00|2854.71|4545.42|4781.13|-1050.51| +2450822|70030|2450829|78525|544217|309|44801|78525|544217|309|44801|4|69|16|5|8942|140|359|86|25.69|28.51|6.55|1888.56|563.30|2209.34|2451.86|13.80|366.14|882.36|197.16|210.96|1079.52|1093.32|-2012.18| +2450822|70030|2450861|78525|544217|309|44801|78525|544217|309|44801|4|50|10|5|11854|205|359|55|86.53|133.25|113.26|1099.45|6229.30|4759.15|7328.75|373.75|0.00|3444.10|6229.30|6603.05|9673.40|10047.15|1470.15| +2450822|64973|2450897|77219|533750|5034|36318|77219|533750|5034|36318|2|82|11|4|17638|275|360|12|46.22|113.23|27.17|1032.72|326.04|554.64|1358.76|13.92|127.15|54.24|198.89|212.81|253.13|267.05|-355.75| +2450822|64973|2450828|77219|533750|5034|36318|77219|533750|5034|36318|2|81|3|5|127|252|360|39|8.46|21.65|0.86|810.81|33.54|329.94|844.35|0.44|11.06|143.52|22.48|22.92|166.00|166.44|-307.46| +2450822|64973|2450838|77219|533750|5034|36318|77219|533750|5034|36318|2|86|4|3|68|270|360|13|72.44|181.10|172.04|117.78|2236.52|941.72|2354.30|156.55|0.00|494.39|2236.52|2393.07|2730.91|2887.46|1294.80| +2450822|64973|2450904|77219|533750|5034|36318|77219|533750|5034|36318|2|35|16|2|16279|219|360|11|56.55|115.92|103.16|140.36|1134.76|622.05|1275.12|102.12|0.00|318.78|1134.76|1236.88|1453.54|1555.66|512.71| +2450822|64973|2450848|77219|533750|5034|36318|77219|533750|5034|36318|2|50|4|2|16760|256|360|64|18.13|53.12|5.31|3059.84|339.84|1160.32|3399.68|0.00|30.58|1427.84|309.26|309.26|1737.10|1737.10|-851.06| +2450822|64973|2450892|77219|533750|5034|36318|77219|533750|5034|36318|2|67|10|2|8272|201|360|12|55.73|107.55|76.36|374.28|916.32|668.76|1290.60|13.74|687.24|180.60|229.08|242.82|409.68|423.42|-439.68| +2450822|64973|2450895|77219|533750|5034|36318|77219|533750|5034|36318|2|66|5|3|17662|276|360|22|2.85|7.83|2.34|120.78|51.48|62.70|172.26|3.60|0.00|67.10|51.48|55.08|118.58|122.18|-11.22| +2450822|64973|2450856|77219|533750|5034|36318|77219|533750|5034|36318|2|41|4|4|2413|136|360|54|55.32|146.59|131.93|791.64|7124.22|2987.28|7915.86|213.72|0.00|632.88|7124.22|7337.94|7757.10|7970.82|4136.94| +2450822|64973|2450826|77219|533750|5034|36318|77219|533750|5034|36318|2|108|5|4|10693|220|360|36|94.68|156.22|89.04|2418.48|3205.44|3408.48|5623.92|32.05|0.00|1349.64|3205.44|3237.49|4555.08|4587.13|-203.04| +2450822|19638|2450884|3023|594473|6276|926|3023|594473|6276|926|4|63|3|4|3536|83|361|70|7.63|12.28|6.50|404.60|455.00|534.10|859.60|0.63|423.15|343.70|31.85|32.48|375.55|376.18|-502.25| +2450822|19638|2450880|3023|594473|6276|926|3023|594473|6276|926|4|91|18|3|9356|228|361|90|73.51|112.47|93.35|1720.80|8401.50|6615.90|10122.30|0.00|0.00|2429.10|8401.50|8401.50|10830.60|10830.60|1785.60| +2450822|19638|2450898|3023|594473|6276|926|3023|594473|6276|926|4|43|2|4|6961|200|361|13|98.59|148.87|22.33|1645.02|290.29|1281.67|1935.31|0.00|0.00|890.24|290.29|290.29|1180.53|1180.53|-991.38| +2450822|19638|2450867|3023|594473|6276|926|3023|594473|6276|926|4|38|12|1|4399|16|361|1|64.70|131.34|51.22|80.12|51.22|64.70|131.34|4.09|0.00|49.90|51.22|55.31|101.12|105.21|-13.48| +2450822|19638|2450893|3023|594473|6276|926|3023|594473|6276|926|4|108|2|2|10006|204|361|67|91.80|258.87|98.37|10753.50|6590.79|6150.60|17344.29|329.53|0.00|5549.61|6590.79|6920.32|12140.40|12469.93|440.19| +2450822|19638|2450853|3023|594473|6276|926|3023|594473|6276|926|4|9|12|5|12313|229|361|36|67.31|127.21|45.79|2931.12|1648.44|2423.16|4579.56|115.39|0.00|1144.80|1648.44|1763.83|2793.24|2908.63|-774.72| +2450822|19638|2450851|3023|594473|6276|926|3023|594473|6276|926|4|63|18|3|14230|165|361|58|97.52|179.43|116.62|3642.98|6763.96|5656.16|10406.94|202.91|0.00|103.82|6763.96|6966.87|6867.78|7070.69|1107.80| +2450822|19638|2450902|3023|594473|6276|926|3023|594473|6276|926|4|47|12|3|4597|169|361|17|48.77|137.53|41.25|1636.76|701.25|829.09|2338.01|0.00|259.46|1122.17|441.79|441.79|1563.96|1563.96|-387.30| +2450822|19638|2450903|3023|594473|6276|926|3023|594473|6276|926|4|81|12|5|16645|43|361|56|57.68|107.86|50.69|3201.52|2838.64|3230.08|6040.16|85.15|0.00|1268.40|2838.64|2923.79|4107.04|4192.19|-391.44| +2450822|19638|2450839|3023|594473|6276|926|3023|594473|6276|926|4|4|15|4|16201|93|361|32|36.07|90.89|3.63|2792.32|116.16|1154.24|2908.48|5.80|0.00|639.68|116.16|121.96|755.84|761.64|-1038.08| +2450822|19638|2450906|3023|594473|6276|926|3023|594473|6276|926|4|71|20|2|8404|39|361|92|38.39|74.47|39.46|3220.92|3630.32|3531.88|6851.24|290.42|0.00|2465.60|3630.32|3920.74|6095.92|6386.34|98.44| +2450822|19638|2450869|3023|594473|6276|926|3023|594473|6276|926|4|4|9|5|1462|189|361|1|99.57|140.39|101.08|39.31|101.08|99.57|140.39|6.14|24.25|19.65|76.83|82.97|96.48|102.62|-22.74| +2450822|45334|2450826|17680|1025907|2134|31608|17680|1025907|2134|31608|2|41|4|3|7945|236|362|78|81.30|213.81|32.07|14175.72|2501.46|6341.40|16677.18|75.04|0.00|999.96|2501.46|2576.50|3501.42|3576.46|-3839.94| +2450822|45334|2450840|17680|1025907|2134|31608|17680|1025907|2134|31608|2|86|3|2|3616|230|362|80|72.07|190.26|157.91|2588.00|12632.80|5765.60|15220.80|34.10|11495.84|2282.40|1136.96|1171.06|3419.36|3453.46|-4628.64| +2450822|45334|2450912|17680|1025907|2134|31608|17680|1025907|2134|31608|2|51|8|1|8494|274|362|98|36.51|87.98|31.67|5518.38|3103.66|3577.98|8622.04|0.00|0.00|2155.02|3103.66|3103.66|5258.68|5258.68|-474.32| +2450822|45334|2450844|17680|1025907|2134|31608|17680|1025907|2134|31608|2|59|7|4|1318|183|362|36|10.24|30.41|27.06|120.60|974.16|368.64|1094.76|68.19|0.00|87.48|974.16|1042.35|1061.64|1129.83|605.52| +2450822|39044|2450844|66323|1046402|881|39713|66323|1046402|881|39713|4|80|5|2|14980|76|363|26|29.87|43.90|28.97|388.18|753.22|776.62|1141.40|30.12|0.00|456.56|753.22|783.34|1209.78|1239.90|-23.40| +2450822|39044|2450906|66323|1046402|881|39713|66323|1046402|881|39713|4|5|4|2|9296|58|363|80|30.22|42.00|36.12|470.40|2889.60|2417.60|3360.00|0.00|0.00|470.40|2889.60|2889.60|3360.00|3360.00|472.00| +2450822|39044|2450826|66323|1046402|881|39713|66323|1046402|881|39713|4|9|14|5|12254|94|363|76|99.32|152.95|148.36|348.84|11275.36|7548.32|11624.20|456.65|6201.44|464.36|5073.92|5530.57|5538.28|5994.93|-2474.40| +2450822|39044|2450885|66323|1046402|881|39713|66323|1046402|881|39713|4|24|8|2|9679|106|363|95|32.47|40.26|9.25|2945.95|878.75|3084.65|3824.70|79.08|0.00|1032.65|878.75|957.83|1911.40|1990.48|-2205.90| +2450822|39044|2450901|66323|1046402|881|39713|66323|1046402|881|39713|4|28|4|2|14626|227|363|19|68.90|183.27|18.32|3134.05|348.08|1309.10|3482.13|0.00|149.67|104.31|198.41|198.41|302.72|302.72|-1110.69| +2450822|39044|2450875|66323|1046402|881|39713|66323|1046402|881|39713|4|7|15|4|8335|114|363|31|25.53|28.59|26.87|53.32|832.97|791.43|886.29|4.33|616.39|159.34|216.58|220.91|375.92|380.25|-574.85| +2450822|39044|2450909|66323|1046402|881|39713|66323|1046402|881|39713|4|84|18|5|4534|278|363|28|85.72|166.29|53.21|3166.24|1489.88|2400.16|4656.12|74.49|0.00|325.92|1489.88|1564.37|1815.80|1890.29|-910.28| +2450822|36944|2450846|41095|1288029|5810|3544|41095|1288029|5810|3544|4|15|19|2|2071|81|364|59|40.49|106.89|63.06|2585.97|3720.54|2388.91|6306.51|0.00|744.10|567.58|2976.44|2976.44|3544.02|3544.02|587.53| +2450822|36944|2450899|41095|1288029|5810|3544|41095|1288029|5810|3544|4|45|3|4|5779|74|364|70|22.13|27.21|13.87|933.80|970.90|1549.10|1904.70|87.38|0.00|342.30|970.90|1058.28|1313.20|1400.58|-578.20| +2450822|36944|2450898|41095|1288029|5810|3544|41095|1288029|5810|3544|4|39|15|4|8114|241|364|2|91.42|170.95|6.83|328.24|13.66|182.84|341.90|0.40|0.00|133.34|13.66|14.06|147.00|147.40|-169.18| +2450822|36944|2450910|41095|1288029|5810|3544|41095|1288029|5810|3544|4|17|4|2|16112|258|364|95|19.94|44.06|7.93|3432.35|753.35|1894.30|4185.70|15.06|0.00|292.60|753.35|768.41|1045.95|1061.01|-1140.95| +2450822|36944|2450884|41095|1288029|5810|3544|41095|1288029|5810|3544|4|72|19|2|15818|191|364|78|94.73|227.35|186.42|3192.54|14540.76|7388.94|17733.30|290.81|0.00|2836.86|14540.76|14831.57|17377.62|17668.43|7151.82| +2450822|36944|2450862|41095|1288029|5810|3544|41095|1288029|5810|3544|4|89|4|4|8920|51|364|46|73.02|219.06|135.81|3829.50|6247.26|3358.92|10076.76|187.41|0.00|4232.00|6247.26|6434.67|10479.26|10666.67|2888.34| +2450822|36944|2450874|41095|1288029|5810|3544|41095|1288029|5810|3544|4|39|19|5|9670|4|364|24|80.54|82.15|44.36|906.96|1064.64|1932.96|1971.60|85.17|0.00|749.04|1064.64|1149.81|1813.68|1898.85|-868.32| +2450822|36944|2450830|41095|1288029|5810|3544|41095|1288029|5810|3544|4|71|19|4|11707|119|364|15|88.02|157.55|148.09|141.90|2221.35|1320.30|2363.25|177.70|0.00|543.45|2221.35|2399.05|2764.80|2942.50|901.05| +2450822|36944|2450855|41095|1288029|5810|3544|41095|1288029|5810|3544|4|67|6|3|13981|232|364|25|4.42|9.59|7.48|52.75|187.00|110.50|239.75|7.48|0.00|88.50|187.00|194.48|275.50|282.98|76.50| +2450822|36944|2450850|41095|1288029|5810|3544|41095|1288029|5810|3544|4|50|20|1|13700|176|364|47|91.02|168.38|133.02|1661.92|6251.94|4277.94|7913.86|62.51|0.00|3165.45|6251.94|6314.45|9417.39|9479.90|1974.00| +2450822|36944|2450902|41095|1288029|5810|3544|41095|1288029|5810|3544|4|97|19|4|13213|29|364|61|31.11|89.90|40.45|3016.45|2467.45|1897.71|5483.90|49.34|0.00|2741.95|2467.45|2516.79|5209.40|5258.74|569.74| +2450822|36944|2450883|41095|1288029|5810|3544|41095|1288029|5810|3544|4|25|4|5|4858|194|364|7|4.40|12.89|7.73|36.12|54.11|30.80|90.23|4.32|0.00|45.08|54.11|58.43|99.19|103.51|23.31| +2450822|36944|2450877|41095|1288029|5810|3544|41095|1288029|5810|3544|4|99|5|4|15848|94|364|72|16.20|35.15|26.01|658.08|1872.72|1166.40|2530.80|9.36|1685.44|733.68|187.28|196.64|920.96|930.32|-979.12| +2450822|36944|2450904|41095|1288029|5810|3544|41095|1288029|5810|3544|4|15|19|2|10483|239|364|31|77.98|93.57|21.52|2233.55|667.12|2417.38|2900.67|0.00|0.00|1217.99|667.12|667.12|1885.11|1885.11|-1750.26| +2450822|14522|2450889|61120|774313|3506|10299|61120|774313|3506|10299|2|102|6|3|14728|5|365|20|6.48|8.42|7.99|8.60|159.80|129.60|168.40|9.58|0.00|84.20|159.80|169.38|244.00|253.58|30.20| +2450822|14522|2450839|61120|774313|3506|10299|61120|774313|3506|10299|2|100|20|3|13093|24|365|72|24.52|64.24|41.11|1665.36|2959.92|1765.44|4625.28|266.39|0.00|1202.40|2959.92|3226.31|4162.32|4428.71|1194.48| +2450822|14522|2450888|61120|774313|3506|10299|61120|774313|3506|10299|2|48|7|4|16603|183|365|97|3.93|10.88|5.65|507.31|548.05|381.21|1055.36|10.96|0.00|506.34|548.05|559.01|1054.39|1065.35|166.84| +2450822|14522|2450885|61120|774313|3506|10299|61120|774313|3506|10299|2|31|14|4|1076|189|365|6|52.55|109.30|103.83|32.82|622.98|315.30|655.80|0.00|0.00|236.04|622.98|622.98|859.02|859.02|307.68| +2450822|14522|2450829|61120|774313|3506|10299|61120|774313|3506|10299|2|106|3|2|8605|170|365|76|60.31|177.91|37.36|10681.80|2839.36|4583.56|13521.16|227.14|0.00|1081.48|2839.36|3066.50|3920.84|4147.98|-1744.20| +2450822|14522|2450882|61120|774313|3506|10299|61120|774313|3506|10299|2|60|9|1|6976|101|365|74|71.82|107.01|99.51|555.00|7363.74|5314.68|7918.74|147.27|0.00|79.18|7363.74|7511.01|7442.92|7590.19|2049.06| +2450822|14522|2450869|61120|774313|3506|10299|61120|774313|3506|10299|2|63|10|5|11317|27|365|65|61.51|86.11|18.08|4421.95|1175.20|3998.15|5597.15|35.25|0.00|2126.80|1175.20|1210.45|3302.00|3337.25|-2822.95| +2450822|14522|2450891|61120|774313|3506|10299|61120|774313|3506|10299|2|59|4|5|17155|244|365|45|61.07|169.77|33.95|6111.90|1527.75|2748.15|7639.65|30.55|0.00|840.15|1527.75|1558.30|2367.90|2398.45|-1220.40| +2450822|14522|2450901|61120|774313|3506|10299|61120|774313|3506|10299|2|91|1|3|10640|119|365|22|98.98|135.60|132.88|59.84|2923.36|2177.56|2983.20|204.63|0.00|1282.60|2923.36|3127.99|4205.96|4410.59|745.80| +2450822|14522|2450859|61120|774313|3506|10299|61120|774313|3506|10299|2|8|2|4|15308|173|365|78|67.93|124.99|102.49|1755.00|7994.22|5298.54|9749.22|399.71|0.00|974.22|7994.22|8393.93|8968.44|9368.15|2695.68| +2450822|14522|2450849|61120|774313|3506|10299|61120|774313|3506|10299|2|45|3|4|11120|223|365|19|40.29|69.70|18.81|966.91|357.39|765.51|1324.30|0.00|0.00|410.40|357.39|357.39|767.79|767.79|-408.12| +2450822|14522|2450862|61120|774313|3506|10299|61120|774313|3506|10299|2|23|10|5|15736|187|365|29|53.15|151.47|134.80|483.43|3909.20|1541.35|4392.63|39.09|0.00|351.19|3909.20|3948.29|4260.39|4299.48|2367.85| +2450822|14522|2450829|61120|774313|3506|10299|61120|774313|3506|10299|2|33|2|5|12350|86|365|55|93.16|143.46|80.33|3472.15|4418.15|5123.80|7890.30|397.63|0.00|3234.55|4418.15|4815.78|7652.70|8050.33|-705.65| +2450822|45868|2450857|27411|535585|5359|25533|27411|535585|5359|25533|4|69|6|4|9914|30|366|27|17.39|27.12|17.62|256.50|475.74|469.53|732.24|28.54|0.00|58.32|475.74|504.28|534.06|562.60|6.21| +2450822|45868|2450892|27411|535585|5359|25533|27411|535585|5359|25533|4|38|9|3|2516|146|366|84|59.65|134.80|33.70|8492.40|2830.80|5010.60|11323.20|226.46|0.00|339.36|2830.80|3057.26|3170.16|3396.62|-2179.80| +2450822|45868|2450836|27411|535585|5359|25533|27411|535585|5359|25533|4|82|12|2|16075|37|366|39|36.30|99.82|71.87|1090.05|2802.93|1415.70|3892.98|0.00|0.00|272.22|2802.93|2802.93|3075.15|3075.15|1387.23| +2450822|45868|2450837|27411|535585|5359|25533|27411|535585|5359|25533|4|95|8|3|13201|266|366|97|48.08|138.47|58.15|7791.04|5640.55|4663.76|13431.59|338.43|0.00|3223.31|5640.55|5978.98|8863.86|9202.29|976.79| +2450822|45868|2450912|27411|535585|5359|25533|27411|535585|5359|25533|4|37|2|2|8414|12|366|52|20.68|51.49|11.32|2088.84|588.64|1075.36|2677.48|0.00|0.00|963.56|588.64|588.64|1552.20|1552.20|-486.72| +2450822|45868|2450852|27411|535585|5359|25533|27411|535585|5359|25533|4|56|20|4|440|109|366|89|65.19|156.45|34.41|10861.56|3062.49|5801.91|13924.05|244.99|0.00|5291.05|3062.49|3307.48|8353.54|8598.53|-2739.42| +2450822|42848|2450862|99707|1691205|4656|5354|99707|1691205|4656|5354|1|52|1|5|16009|90|367|81|77.63|103.24|34.06|5603.58|2758.86|6288.03|8362.44|79.17|1627.72|2090.61|1131.14|1210.31|3221.75|3300.92|-5156.89| +2450822|42848|2450849|99707|1691205|4656|5354|99707|1691205|4656|5354|1|75|3|4|14227|196|367|55|79.36|112.69|94.65|992.20|5205.75|4364.80|6197.95|208.23|0.00|2230.80|5205.75|5413.98|7436.55|7644.78|840.95| +2450822|42848|2450860|99707|1691205|4656|5354|99707|1691205|4656|5354|1|72|16|2|6655|32|367|82|87.18|152.56|141.88|875.76|11634.16|7148.76|12509.92|930.73|0.00|2876.56|11634.16|12564.89|14510.72|15441.45|4485.40| +2450822|42848|2450887|99707|1691205|4656|5354|99707|1691205|4656|5354|1|50|12|4|10442|183|367|65|23.41|26.21|22.01|273.00|1430.65|1521.65|1703.65|54.07|529.34|187.20|901.31|955.38|1088.51|1142.58|-620.34| +2450822|42848|2450838|99707|1691205|4656|5354|99707|1691205|4656|5354|1|7|1|1|9272|106|367|45|76.57|157.73|143.53|639.00|6458.85|3445.65|7097.85|64.58|0.00|1206.45|6458.85|6523.43|7665.30|7729.88|3013.20| +2450822|42848|2450846|99707|1691205|4656|5354|99707|1691205|4656|5354|1|65|2|1|6638|187|367|57|57.85|107.60|75.32|1839.96|4293.24|3297.45|6133.20|214.66|0.00|1839.96|4293.24|4507.90|6133.20|6347.86|995.79| +2450822|42848|2450826|99707|1691205|4656|5354|99707|1691205|4656|5354|1|34|20|1|16873|26|367|100|37.39|109.55|77.78|3177.00|7778.00|3739.00|10955.00|77.78|0.00|2957.00|7778.00|7855.78|10735.00|10812.78|4039.00| +2450822|63183|2450863|22315|395529|1190|46804|22315|395529|1190|46804|1|77|2|3|12727|145|368|62|98.90|229.44|128.48|6259.52|7965.76|6131.80|14225.28|121.87|3903.22|3982.88|4062.54|4184.41|8045.42|8167.29|-2069.26| +2450822|63183|2450857|22315|395529|1190|46804|22315|395529|1190|46804|1|72|1|2|11660|40|368|47|39.79|115.78|37.04|3700.78|1740.88|1870.13|5441.66|156.67|0.00|163.09|1740.88|1897.55|1903.97|2060.64|-129.25| +2450822|63183|2450890|22315|395529|1190|46804|22315|395529|1190|46804|1|93|16|2|4666|281|368|50|13.17|26.60|4.52|1104.00|226.00|658.50|1330.00|0.00|0.00|425.50|226.00|226.00|651.50|651.50|-432.50| +2450822|63183|2450861|22315|395529|1190|46804|22315|395529|1190|46804|1|56|6|4|5558|127|368|34|74.13|221.64|214.99|226.10|7309.66|2520.42|7535.76|365.48|0.00|0.00|7309.66|7675.14|7309.66|7675.14|4789.24| +2450822|63183|2450892|22315|395529|1190|46804|22315|395529|1190|46804|1|57|6|5|16946|216|368|24|26.93|70.01|33.60|873.84|806.40|646.32|1680.24|0.00|0.00|134.40|806.40|806.40|940.80|940.80|160.08| +2450822|63183|2450848|22315|395529|1190|46804|22315|395529|1190|46804|1|27|2|3|1129|160|368|75|39.81|109.07|80.71|2127.00|6053.25|2985.75|8180.25|161.62|665.85|2126.25|5387.40|5549.02|7513.65|7675.27|2401.65| +2450822|63183|2450892|22315|395529|1190|46804|22315|395529|1190|46804|1|8|6|2|11239|2|368|6|20.10|44.62|24.54|120.48|147.24|120.60|267.72|2.94|0.00|24.06|147.24|150.18|171.30|174.24|26.64| +2450822|63183|2450870|22315|395529|1190|46804|22315|395529|1190|46804|1|1|9|5|5372|89|368|16|60.61|80.00|38.40|665.60|614.40|969.76|1280.00|12.28|0.00|448.00|614.40|626.68|1062.40|1074.68|-355.36| +2450822|63183|2450837|22315|395529|1190|46804|22315|395529|1190|46804|1|94|16|2|17714|269|368|97|12.06|24.60|7.38|1670.34|715.86|1169.82|2386.20|57.26|0.00|1168.85|715.86|773.12|1884.71|1941.97|-453.96| +2450822|63183|2450892|22315|395529|1190|46804|22315|395529|1190|46804|1|20|10|4|457|110|368|59|94.17|128.07|48.66|4685.19|2870.94|5556.03|7556.13|4.01|2469.00|453.12|401.94|405.95|855.06|859.07|-5154.09| +2450822|63183|2450847|22315|395529|1190|46804|22315|395529|1190|46804|1|11|6|1|5306|16|368|98|41.67|55.00|45.10|970.20|4419.80|4083.66|5390.00|265.18|0.00|1185.80|4419.80|4684.98|5605.60|5870.78|336.14| +2450822|63183|2450830|22315|395529|1190|46804|22315|395529|1190|46804|1|85|15|2|1555|153|368|12|45.30|58.89|53.00|70.68|636.00|543.60|706.68|50.88|0.00|56.52|636.00|686.88|692.52|743.40|92.40| +2450822|63183|2450878|22315|395529|1190|46804|22315|395529|1190|46804|1|24|6|5|8756|196|368|35|32.77|34.08|26.92|250.60|942.20|1146.95|1192.80|56.53|0.00|202.65|942.20|998.73|1144.85|1201.38|-204.75| +2450822|63183|2450857|22315|395529|1190|46804|22315|395529|1190|46804|1|16|12|2|14822|254|368|1|8.52|12.52|7.38|5.14|7.38|8.52|12.52|0.00|0.00|2.50|7.38|7.38|9.88|9.88|-1.14| +2450822|28410|2450879|84106|573180|1454|12928|84106|573180|1454|12928|1|75|6|4|12028|227|369|55|1.73|3.40|1.87|84.15|102.85|95.15|187.00|7.19|0.00|65.45|102.85|110.04|168.30|175.49|7.70| +2450822|28410|2450889|84106|573180|1454|12928|84106|573180|1454|12928|1|58|19|5|14798|199|369|14|75.28|143.78|11.50|1851.92|161.00|1053.92|2012.92|8.00|46.69|181.16|114.31|122.31|295.47|303.47|-939.61| +2450822|28410|2450875|84106|573180|1454|12928|84106|573180|1454|12928|1|56|15|1|13678|18|369|33|60.71|152.38|25.90|4173.84|854.70|2003.43|5028.54|25.64|0.00|653.40|854.70|880.34|1508.10|1533.74|-1148.73| +2450822|28410|2450846|84106|573180|1454|12928|84106|573180|1454|12928|1|92|6|4|9847|130|369|90|90.66|199.45|63.82|12206.70|5743.80|8159.40|17950.50|459.50|0.00|1435.50|5743.80|6203.30|7179.30|7638.80|-2415.60| +2450822|28410|2450853|84106|573180|1454|12928|84106|573180|1454|12928|1|41|4|5|7978|170|369|25|38.28|101.05|24.25|1920.00|606.25|957.00|2526.25|18.18|0.00|1136.75|606.25|624.43|1743.00|1761.18|-350.75| +2450822|28410|2450858|84106|573180|1454|12928|84106|573180|1454|12928|1|22|16|2|2842|276|369|31|35.03|85.12|60.43|765.39|1873.33|1085.93|2638.72|74.93|0.00|791.43|1873.33|1948.26|2664.76|2739.69|787.40| +2450822|28410|2450828|84106|573180|1454|12928|84106|573180|1454|12928|1|37|9|2|5282|183|369|57|6.29|16.79|10.74|344.85|612.18|358.53|957.03|18.36|0.00|104.88|612.18|630.54|717.06|735.42|253.65| +2450822|28410|2450892|84106|573180|1454|12928|84106|573180|1454|12928|1|25|19|5|17707|22|369|5|4.83|7.58|1.81|28.85|9.05|24.15|37.90|0.45|0.00|18.15|9.05|9.50|27.20|27.65|-15.10| +2450822|28410|2450899|84106|573180|1454|12928|84106|573180|1454|12928|1|98|13|5|14482|204|369|63|71.89|101.36|101.36|0.00|6385.68|4529.07|6385.68|574.71|0.00|1979.46|6385.68|6960.39|8365.14|8939.85|1856.61| +2450822|74538|2450876|87162|1001396|3630|21017|87162|1001396|3630|21017|4|79|13|1|2498|57|370|85|68.95|186.85|20.55|14135.50|1746.75|5860.75|15882.25|157.20|0.00|4605.30|1746.75|1903.95|6352.05|6509.25|-4114.00| +2450822|74538|2450903|87162|1001396|3630|21017|87162|1001396|3630|21017|4|103|5|3|4207|6|370|53|48.17|64.54|53.56|581.94|2838.68|2553.01|3420.62|0.00|0.00|1402.38|2838.68|2838.68|4241.06|4241.06|285.67| +2450822|74538|2450859|87162|1001396|3630|21017|87162|1001396|3630|21017|4|48|14|2|3289|125|370|4|90.23|112.78|77.81|139.88|311.24|360.92|451.12|9.33|0.00|203.00|311.24|320.57|514.24|523.57|-49.68| +||2450895||||21017||||||59|||11048|121|370|77||||||||11.68|1891.42||233.78||658.82|670.50|-3645.48| +2450822|74538|2450875|87162|1001396|3630|21017|87162|1001396|3630|21017|4|58|3|1|4942|15|370|65|62.56|142.63|39.93|6675.50|2595.45|4066.40|9270.95|0.00|2154.22|3337.10|441.23|441.23|3778.33|3778.33|-3625.17| +2450822|74538|2450899|87162|1001396|3630|21017|87162|1001396|3630|21017|4|44|8|1|4952|82|370|5|20.47|50.15|23.06|135.45|115.30|102.35|250.75|10.37|0.00|37.60|115.30|125.67|152.90|163.27|12.95| +2450822|74538|2450841|87162|1001396|3630|21017|87162|1001396|3630|21017|4|17|17|5|1411|198|370|16|44.30|65.56|47.85|283.36|765.60|708.80|1048.96|35.83|367.48|335.52|398.12|433.95|733.64|769.47|-310.68| +2450822|74538|2450841|87162|1001396|3630|21017|87162|1001396|3630|21017|4|12|2|2|17773|29|370|52|28.44|51.76|3.10|2530.32|161.20|1478.88|2691.52|8.06|0.00|322.92|161.20|169.26|484.12|492.18|-1317.68| +2450822|44700|2450889|46845|356328|5011|48894|46845|356328|5011|48894|4|55|11|5|7642|225|371|60|88.40|177.68|67.51|6610.20|4050.60|5304.00|10660.80|81.01|0.00|959.40|4050.60|4131.61|5010.00|5091.01|-1253.40| +2450822|44700|2450848|46845|356328|5011|48894|46845|356328|5011|48894|4|67|3|1|14311|194|371|97|48.83|97.66|95.70|190.12|9282.90|4736.51|9473.02|649.80|0.00|1799.35|9282.90|9932.70|11082.25|11732.05|4546.39| +2450822|44700|2450825|46845|356328|5011|48894|46845|356328|5011|48894|4|36|18|4|11161|178|371|60|28.74|54.31|38.56|945.00|2313.60|1724.40|3258.60|69.40|0.00|553.80|2313.60|2383.00|2867.40|2936.80|589.20| +2450822|44700|2450908|46845|356328|5011|48894|46845|356328|5011|48894|4|43|20|2|16258|153|371|93|94.98|104.47|28.20|7093.11|2622.60|8833.14|9715.71|157.35|0.00|1262.94|2622.60|2779.95|3885.54|4042.89|-6210.54| +2450822|50004|2450853|70508|1510540|172|13004|70508|1510540|172|13004|1|56|13|1|12272|62|372|19|64.46|161.15|146.64|275.69|2786.16|1224.74|3061.85|139.30|0.00|61.18|2786.16|2925.46|2847.34|2986.64|1561.42| +2450822|50004|2450910|70508|1510540|172|13004|70508|1510540|172|13004|1|66|20|3|7202|27|372|29|34.45|44.44|42.66|51.62|1237.14|999.05|1288.76|86.59|0.00|90.19|1237.14|1323.73|1327.33|1413.92|238.09| +2450822|50004|2450847|70508|1510540|172|13004|70508|1510540|172|13004|1|51|16|3|13682|295|372|27|78.34|142.57|35.64|2887.11|962.28|2115.18|3849.39|57.73|0.00|1809.00|962.28|1020.01|2771.28|2829.01|-1152.90| +2450822|50004|2450831|70508|1510540|172|13004|70508|1510540|172|13004|1|49|19|2|16942|71|372|14|31.20|73.94|44.36|414.12|621.04|436.80|1035.16|18.63|0.00|424.34|621.04|639.67|1045.38|1064.01|184.24| +2450822|50004|2450858|70508|1510540|172|13004|70508|1510540|172|13004|1|21|11|4|8053|243|372|69|95.79|168.59|77.55|6281.76|5350.95|6609.51|11632.71|107.01|0.00|0.00|5350.95|5457.96|5350.95|5457.96|-1258.56| +2450822|50004|2450894|70508|1510540|172|13004|70508|1510540|172|13004|1|44|3|2|11018|192|372|73|19.75|29.03|26.70|170.09|1949.10|1441.75|2119.19|136.43|0.00|974.55|1949.10|2085.53|2923.65|3060.08|507.35| +2450822|50004|2450862|70508|1510540|172|13004|70508|1510540|172|13004|1|80|5|4|6832|130|372|18|6.84|15.80|7.42|150.84|133.56|123.12|284.40|6.67|0.00|136.44|133.56|140.23|270.00|276.67|10.44| +2450822|50004|2450904|70508|1510540|172|13004|70508|1510540|172|13004|1|26|18|5|3577|22|372|19|99.73|143.61|132.12|218.31|2510.28|1894.87|2728.59|67.77|1154.72|954.94|1355.56|1423.33|2310.50|2378.27|-539.31| +2450822|76651|2450874|21541|52468|3842|34369|21541|52468|3842|34369|4|102|19|2|5132|120|373|88|13.70|40.82|16.32|2156.00|1436.16|1205.60|3592.16|129.25|0.00|1580.48|1436.16|1565.41|3016.64|3145.89|230.56| +2450822|76651|2450903|21541|52468|3842|34369|21541|52468|3842|34369|4|16|6|2|15140|247|373|78|81.05|147.51|60.47|6789.12|4716.66|6321.90|11505.78|0.00|0.00|4602.00|4716.66|4716.66|9318.66|9318.66|-1605.24| +2450822|76651|2450894|21541|52468|3842|34369|21541|52468|3842|34369|4|54|13|4|1276|149|373|77|1.25|2.76|0.02|210.98|1.54|96.25|212.52|0.00|0.18|50.82|1.36|1.36|52.18|52.18|-94.89| +2450822|76651|2450855|21541|52468|3842|34369|21541|52468|3842|34369|4|80|6|2|5564|239|373|8|70.52|74.04|30.35|349.52|242.80|564.16|592.32|9.71|0.00|41.44|242.80|252.51|284.24|293.95|-321.36| +2450822|76651|2450872|21541|52468|3842|34369|21541|52468|3842|34369|4|76|17|5|10316|113|373|12|58.94|171.51|29.15|1708.32|349.80|707.28|2058.12|13.99|0.00|349.80|349.80|363.79|699.60|713.59|-357.48| +2450822|76651|2450908|21541|52468|3842|34369|21541|52468|3842|34369|4|58|12|1|8185|4|373|63|80.26|211.08|145.64|4122.72|9175.32|5056.38|13298.04|642.27|0.00|1861.65|9175.32|9817.59|11036.97|11679.24|4118.94| +2450822|76651|2450879|21541|52468|3842|34369|21541|52468|3842|34369|4|8|6|4|9952|283|373|34|23.52|69.85|23.74|1567.74|807.16|799.68|2374.90|0.00|0.00|997.22|807.16|807.16|1804.38|1804.38|7.48| +2450822|76651|2450854|21541|52468|3842|34369|21541|52468|3842|34369|4|100|17|3|2074|174|373|33|33.15|70.27|15.45|1809.06|509.85|1093.95|2318.91|35.68|0.00|672.21|509.85|545.53|1182.06|1217.74|-584.10| +2450822|76651|2450859|21541|52468|3842|34369|21541|52468|3842|34369|4|47|4|2|3418|270|373|57|10.89|22.10|20.11|113.43|1146.27|620.73|1259.70|68.77|0.00|478.23|1146.27|1215.04|1624.50|1693.27|525.54| +2450822|76651|2450840|21541|52468|3842|34369|21541|52468|3842|34369|4|39|10|3|13370|246|373|65|33.93|96.02|94.09|125.45|6115.85|2205.45|6241.30|366.95|0.00|1872.00|6115.85|6482.80|7987.85|8354.80|3910.40| +2450822|76651|2450890|21541|52468|3842|34369|21541|52468|3842|34369|4|47|5|5|8362|242|373|19|75.01|104.26|50.04|1030.18|950.76|1425.19|1980.94|19.01|0.00|376.20|950.76|969.77|1326.96|1345.97|-474.43| +2450822|76651|2450911|21541|52468|3842|34369|21541|52468|3842|34369|4|61|3|5|16309|212|373|96|3.27|4.08|3.99|8.64|383.04|313.92|391.68|26.81|0.00|19.20|383.04|409.85|402.24|429.05|69.12| +2450822|69384|2450848|85437|677689|3094|33699|85437|677689|3094|33699|4|83|19|3|1742|51|374|70|50.02|113.54|81.74|2226.00|5721.80|3501.40|7947.80|277.50|171.65|158.90|5550.15|5827.65|5709.05|5986.55|2048.75| +2450822|69384|2450891|85437|677689|3094|33699|85437|677689|3094|33699|4|39|2|3|214|215|374|76|73.24|164.05|62.33|7730.72|4737.08|5566.24|12467.80|284.22|0.00|4612.44|4737.08|5021.30|9349.52|9633.74|-829.16| +2450822|69384|2450854|85437|677689|3094|33699|85437|677689|3094|33699|4|27|3|1|2845|2|374|8|63.02|171.41|143.98|219.44|1151.84|504.16|1371.28|3.22|1071.21|479.92|80.63|83.85|560.55|563.77|-423.53| +2450822|69384|2450885|85437|677689|3094|33699|85437|677689|3094|33699|4|31|7|1|14506|86|374|72|64.59|76.21|67.82|604.08|4883.04|4650.48|5487.12|439.47|0.00|383.76|4883.04|5322.51|5266.80|5706.27|232.56| +2450822|69384|2450883|85437|677689|3094|33699|85437|677689|3094|33699|4|75|15|3|1730|150|374|69|62.96|170.62|155.26|1059.84|10712.94|4344.24|11772.78|642.77|0.00|4591.26|10712.94|11355.71|15304.20|15946.97|6368.70| +2450822|69384|2450882|85437|677689|3094|33699|85437|677689|3094|33699|4|1|2|3|5020|35|374|59|72.85|184.31|123.48|3588.97|7285.32|4298.15|10874.29|364.26|0.00|4023.21|7285.32|7649.58|11308.53|11672.79|2987.17| +2450822|69384|2450903|85437|677689|3094|33699|85437|677689|3094|33699|4|78|8|4|13880|109|374|32|80.38|240.33|100.93|4460.80|3229.76|2572.16|7690.56|0.00|0.00|1845.44|3229.76|3229.76|5075.20|5075.20|657.60| +2450822|69384|2450897|85437|677689|3094|33699|85437|677689|3094|33699|4|75|5|4|12272|50|374|54|85.38|245.89|83.60|8763.66|4514.40|4610.52|13278.06|45.14|0.00|3452.22|4514.40|4559.54|7966.62|8011.76|-96.12| +2450822|69384|2450854|85437|677689|3094|33699|85437|677689|3094|33699|4|7|10|4|7202|87|374|32|56.16|157.24|69.18|2817.92|2213.76|1797.12|5031.68|132.82|0.00|1861.44|2213.76|2346.58|4075.20|4208.02|416.64| +2450822|69384|2450908|85437|677689|3094|33699|85437|677689|3094|33699|4|95|8|5|13682|154|374|22|92.61|187.07|170.23|370.48|3745.06|2037.42|4115.54|149.80|0.00|411.40|3745.06|3894.86|4156.46|4306.26|1707.64| +2450822|69384|2450836|85437|677689|3094|33699|85437|677689|3094|33699|4|6|13|3|16942|211|374|50|30.50|42.09|30.30|589.50|1515.00|1525.00|2104.50|136.35|0.00|736.50|1515.00|1651.35|2251.50|2387.85|-10.00| +2450822|69384|2450899|85437|677689|3094|33699|85437|677689|3094|33699|4|21|8|1|8053|46|374|91|84.70|101.64|12.19|8139.95|1109.29|7707.70|9249.24|41.04|288.41|2497.04|820.88|861.92|3317.92|3358.96|-6886.82| +2450822|69384|2450874|85437|677689|3094|33699|85437|677689|3094|33699|4|99|13|3|11018|140|374|48|22.77|30.05|5.40|1183.20|259.20|1092.96|1442.40|2.59|0.00|576.96|259.20|261.79|836.16|838.75|-833.76| +2450822|69384|2450892|85437|677689|3094|33699|85437|677689|3094|33699|4|88|18|1|6832|167|374|40|63.74|188.03|75.21|4512.80|3008.40|2549.60|7521.20|90.25|0.00|2256.00|3008.40|3098.65|5264.40|5354.65|458.80| +2450822|6485|2450846|77527|323130|4733|45089|77527|323130|4733|45089|1|99|11|4|3721|40|375|70|95.02|229.94|227.64|161.00|15934.80|6651.40|16095.80|1434.13|0.00|7725.90|15934.80|17368.93|23660.70|25094.83|9283.40| +2450822|6485|2450834|77527|323130|4733|45089|77527|323130|4733|45089|1|72|13|5|9098|139|375|59|11.04|19.54|1.56|1060.82|92.04|651.36|1152.86|0.92|0.00|368.75|92.04|92.96|460.79|461.71|-559.32| +2450822|6485|2450890|77527|323130|4733|45089|77527|323130|4733|45089|1|35|9|2|16736|218|375|22|95.29|202.01|14.14|4133.14|311.08|2096.38|4444.22|11.19|87.10|399.96|223.98|235.17|623.94|635.13|-1872.40| +2450822|6485|2450875|77527|323130|4733|45089|77527|323130|4733|45089|1|68|18|2|13777|195|375|15|94.67|147.68|110.76|553.80|1661.40|1420.05|2215.20|0.00|0.00|310.05|1661.40|1661.40|1971.45|1971.45|241.35| +2450822|6485|2450870|77527|323130|4733|45089|77527|323130|4733|45089|1|97|15|2|928|179|375|57|89.17|260.37|145.80|6530.49|8310.60|5082.69|14841.09|664.84|0.00|6975.09|8310.60|8975.44|15285.69|15950.53|3227.91| +2450822|6485|2450855|77527|323130|4733|45089|77527|323130|4733|45089|1|97|3|5|17072|219|375|71|28.32|40.78|37.51|232.17|2663.21|2010.72|2895.38|41.54|1970.77|115.73|692.44|733.98|808.17|849.71|-1318.28| +2450822|26683|2450891|58488|1543501|67|28680|58488|1543501|67|28680|4|35|4|1|3613|180|376|58|37.00|68.45|41.07|1588.04|2382.06|2146.00|3970.10|23.82|0.00|1309.64|2382.06|2405.88|3691.70|3715.52|236.06| +2450822|26683|2450845|58488|1543501|67|28680|58488|1543501|67|28680|4|78|10|4|13760|240|376|20|14.17|23.94|9.81|282.60|196.20|283.40|478.80|10.71|43.16|239.40|153.04|163.75|392.44|403.15|-130.36| +2450822|26683|2450846|58488|1543501|67|28680|58488|1543501|67|28680|4|52|12|3|15259|202|376|75|46.58|81.04|34.84|3465.00|2613.00|3493.50|6078.00|52.26|0.00|2977.50|2613.00|2665.26|5590.50|5642.76|-880.50| +2450822|26683|2450875|58488|1543501|67|28680|58488|1543501|67|28680|4|86|6|4|10381|237|376|46|88.80|204.24|104.16|4603.68|4791.36|4084.80|9395.04|103.97|3306.03|3945.88|1485.33|1589.30|5431.21|5535.18|-2599.47| +2450822|26683|2450852|58488|1543501|67|28680|58488|1543501|67|28680|4|75|19|2|17404|93|376|67|66.05|81.24|51.99|1959.75|3483.33|4425.35|5443.08|0.00|0.00|1033.81|3483.33|3483.33|4517.14|4517.14|-942.02| +2450822|26683|2450864|58488|1543501|67|28680|58488|1543501|67|28680|4|37|4|1|5356|270|376|30|70.72|163.36|40.84|3675.60|1225.20|2121.60|4900.80|49.00|0.00|1470.00|1225.20|1274.20|2695.20|2744.20|-896.40| +2450822|26683|2450829|58488|1543501|67|28680|58488|1543501|67|28680|4|36|4|3|3637|79|376|28|79.72|172.19|68.87|2892.96|1928.36|2232.16|4821.32|36.25|1022.03|337.40|906.33|942.58|1243.73|1279.98|-1325.83| +2450822|26683|2450877|58488|1543501|67|28680|58488|1543501|67|28680|4|75|20|3|12064|99|376|96|61.43|77.40|24.76|5053.44|2376.96|5897.28|7430.40|118.84|0.00|1782.72|2376.96|2495.80|4159.68|4278.52|-3520.32| +2450822|26683|2450892|58488|1543501|67|28680|58488|1543501|67|28680|4|58|10|2|3223|153|376|75|97.53|134.59|126.51|606.00|9488.25|7314.75|10094.25|853.94|0.00|1816.50|9488.25|10342.19|11304.75|12158.69|2173.50| +2450822|26683|2450845|58488|1543501|67|28680|58488|1543501|67|28680|4|2|7|2|14480|155|376|9|57.85|145.78|144.32|13.14|1298.88|520.65|1312.02|77.93|0.00|459.18|1298.88|1376.81|1758.06|1835.99|778.23| +2450822|26683|2450841|58488|1543501|67|28680|58488|1543501|67|28680|4|24|6|3|17642|270|376|56|55.31|131.63|110.56|1179.92|6191.36|3097.36|7371.28|234.03|3590.98|2506.00|2600.38|2834.41|5106.38|5340.41|-496.98| +2450822|26683|2450838|58488|1543501|67|28680|58488|1543501|67|28680|4|58|15|3|15262|4|376|56|12.33|15.28|0.30|838.88|16.80|690.48|855.68|0.03|16.29|0.00|0.51|0.54|0.51|0.54|-689.97| +2450822|26683|2450900|58488|1543501|67|28680|58488|1543501|67|28680|4|26|18|2|14491|150|376|67|42.56|53.20|30.85|1497.45|2066.95|2851.52|3564.40|82.67|0.00|712.88|2066.95|2149.62|2779.83|2862.50|-784.57| +2450822|26683|2450881|58488|1543501|67|28680|58488|1543501|67|28680|4|78|1|5|11662|171|376|74|98.80|187.72|144.54|3195.32|10695.96|7311.20|13891.28|427.83|0.00|5834.16|10695.96|11123.79|16530.12|16957.95|3384.76| +2450822|65871|2450833|17400|792439|3925|35364|17400|792439|3925|35364|4|62|2|3|17329|255|377|30|65.40|157.61|31.52|3782.70|945.60|1962.00|4728.30|19.66|453.88|2080.20|491.72|511.38|2571.92|2591.58|-1470.28| +2450822|65871|2450891|17400|792439|3925|35364|17400|792439|3925|35364|4|102|9|1|13600|125|377|70|17.94|21.52|1.50|1401.40|105.00|1255.80|1506.40|2.10|0.00|301.00|105.00|107.10|406.00|408.10|-1150.80| +2450822|65871|2450851|17400|792439|3925|35364|17400|792439|3925|35364|4|85|14|5|5384|229|377|1|21.55|45.68|1.82|43.86|1.82|21.55|45.68|0.16|0.00|19.18|1.82|1.98|21.00|21.16|-19.73| +2450822|65871|2450881|17400|792439|3925|35364|17400|792439|3925|35364|4|20|2|4|16432|163|377|39|97.31|125.52|47.69|3035.37|1859.91|3795.09|4895.28|167.39|0.00|587.34|1859.91|2027.30|2447.25|2614.64|-1935.18| +2450822|65871|2450833|17400|792439|3925|35364|17400|792439|3925|35364|4|36|15|3|11560|46|377|65|62.59|179.63|53.88|8173.75|3502.20|4068.35|11675.95|280.17|0.00|2685.15|3502.20|3782.37|6187.35|6467.52|-566.15| +2450822|65871|2450889|17400|792439|3925|35364|17400|792439|3925|35364|4|97|8|2|9874|159|377|52|40.85|95.58|43.01|2733.64|2236.52|2124.20|4970.16|111.82|0.00|297.96|2236.52|2348.34|2534.48|2646.30|112.32| +2450822|65871|2450911|17400|792439|3925|35364|17400|792439|3925|35364|4|22|5|3|9700|300|377|9|20.12|52.11|39.08|117.27|351.72|181.08|468.99|24.62|0.00|23.40|351.72|376.34|375.12|399.74|170.64| +2450822|65871|2450856|17400|792439|3925|35364|17400|792439|3925|35364|4|97|19|2|16562|255|377|13|32.26|73.23|62.97|133.38|818.61|419.38|951.99|16.37|0.00|333.19|818.61|834.98|1151.80|1168.17|399.23| +2450822|65871|2450849|17400|792439|3925|35364|17400|792439|3925|35364|4|67|18|5|15841|184|377|70|94.61|161.78|82.50|5549.60|5775.00|6622.70|11324.60|291.06|1617.00|1245.30|4158.00|4449.06|5403.30|5694.36|-2464.70| +2450822|17404|2450865|87700|1858847|3594|23094|24601|74988|6935|14835|1|37|3|5|13622|161|378|23|93.47|258.91|183.82|1727.07|4227.86|2149.81|5954.93|376.70|42.27|1965.12|4185.59|4562.29|6150.71|6527.41|2035.78| +2450822|17404|2450883|87700|1858847|3594|23094|24601|74988|6935|14835|1|85|14|5|2425|287|378|30|38.51|107.44|1.07|3191.10|32.10|1155.30|3223.20|2.24|0.00|257.70|32.10|34.34|289.80|292.04|-1123.20| +2450822|17404|2450862|87700|1858847|3594|23094|24601|74988|6935|14835|1|62|7|4|12310|133|378|44|18.10|20.63|1.65|835.12|72.60|796.40|907.72|3.63|0.00|253.88|72.60|76.23|326.48|330.11|-723.80| +2450822|17404|2450900|87700|1858847|3594|23094|24601|74988|6935|14835|1|19|1|2|12433|7|378|57|99.13|230.97|39.26|10927.47|2237.82|5650.41|13165.29|44.75|0.00|6187.35|2237.82|2282.57|8425.17|8469.92|-3412.59| +2450822|17404|2450912|87700|1858847|3594|23094|24601|74988|6935|14835|1|18|19|5|1426|95|378|10|53.30|121.52|114.22|73.00|1142.20|533.00|1215.20|9.13|685.32|182.20|456.88|466.01|639.08|648.21|-76.12| +2450822|17404|2450891|87700|1858847|3594|23094|24601|74988|6935|14835|1|97|6|1|6256|21|378|30|96.27|148.25|105.25|1290.00|3157.50|2888.10|4447.50|31.57|0.00|177.90|3157.50|3189.07|3335.40|3366.97|269.40| +2450822|17404|2450855|87700|1858847|3594|23094|24601|74988|6935|14835|1|25|9|4|4993|109|378|67|95.82|165.76|43.09|8218.89|2887.03|6419.94|11105.92|115.48|0.00|2665.26|2887.03|3002.51|5552.29|5667.77|-3532.91| +2450822|17404|2450824|87700|1858847|3594|23094|24601|74988|6935|14835|1|43|4|2|2686|241|378|86|19.21|30.35|6.67|2036.48|573.62|1652.06|2610.10|2.40|453.15|1173.90|120.47|122.87|1294.37|1296.77|-1531.59| +2450822|26286|2450912|29445|849227|656|5638|29445|849227|656|5638|4|78|12|3|13357|258|379|70|69.68|138.66|8.31|9124.50|581.70|4877.60|9706.20|29.08|0.00|1067.50|581.70|610.78|1649.20|1678.28|-4295.90| +2450822|26286|2450839|29445|849227|656|5638|29445|849227|656|5638|4|103|15|2|14324|253|379|75|32.91|87.86|43.05|3360.75|3228.75|2468.25|6589.50|193.72|0.00|987.75|3228.75|3422.47|4216.50|4410.22|760.50| +2450822|26286|2450841|29445|849227|656|5638|29445|849227|656|5638|4|51|9|5|1636|95|379|60|18.56|19.85|10.52|559.80|631.20|1113.60|1191.00|18.93|0.00|511.80|631.20|650.13|1143.00|1161.93|-482.40| +2450822|26286|2450858|29445|849227|656|5638|29445|849227|656|5638|4|2|17|1|10376|114|379|40|32.42|95.96|91.16|192.00|3646.40|1296.80|3838.40|218.78|0.00|460.40|3646.40|3865.18|4106.80|4325.58|2349.60| +2450822|26286|2450845|29445|849227|656|5638|29445|849227|656|5638|4|87|14|5|15518|185|379|70|72.86|146.44|120.08|1845.20|8405.60|5100.20|10250.80|244.60|252.16|3689.70|8153.44|8398.04|11843.14|12087.74|3053.24| +2450822|26286|2450861|29445|849227|656|5638|29445|849227|656|5638|4|34|20|3|8737|294|379|56|73.81|96.69|3.86|5198.48|216.16|4133.36|5414.64|15.13|0.00|432.88|216.16|231.29|649.04|664.17|-3917.20| +2450822|26286|2450829|29445|849227|656|5638|29445|849227|656|5638|4|27|3|1|7357|262|379|90|43.88|54.85|34.55|1827.00|3109.50|3949.20|4936.50|217.66|0.00|1925.10|3109.50|3327.16|5034.60|5252.26|-839.70| +2450822|26286|2450827|29445|849227|656|5638|29445|849227|656|5638|4|53|1|2|6121|252|379|7|91.76|154.15|89.40|453.25|625.80|642.32|1079.05|18.77|0.00|409.99|625.80|644.57|1035.79|1054.56|-16.52| +2450822|26286|2450885|29445|849227|656|5638|29445|849227|656|5638|4|80|8|5|15328|177|379|29|8.01|13.45|5.51|230.26|159.79|232.29|390.05|12.78|0.00|116.87|159.79|172.57|276.66|289.44|-72.50| +2450822|26286|2450825|29445|849227|656|5638|29445|849227|656|5638|4|38|4|1|15139|47|379|52|39.52|104.33|45.90|3038.36|2386.80|2055.04|5425.16|119.34|0.00|1898.52|2386.80|2506.14|4285.32|4404.66|331.76| +2450822|14705|2450829|80172|1082466|6360|39101|80172|1082466|6360|39101|2|2|14|4|5026|120|380|59|45.03|67.54|54.03|797.09|3187.77|2656.77|3984.86|0.00|2008.29|437.78|1179.48|1179.48|1617.26|1617.26|-1477.29| +2450822|14705|2450895|80172|1082466|6360|39101|80172|1082466|6360|39101|2|18|18|1|12559|267|380|61|79.68|101.19|72.85|1728.74|4443.85|4860.48|6172.59|399.94|0.00|2468.67|4443.85|4843.79|6912.52|7312.46|-416.63| +2450822|14705|2450861|80172|1082466|6360|39101|80172|1082466|6360|39101|2|54|3|4|6838|89|380|74|96.79|123.89|101.58|1650.94|7516.92|7162.46|9167.86|300.67|0.00|1191.40|7516.92|7817.59|8708.32|9008.99|354.46| +2450822|14705|2450845|80172|1082466|6360|39101|80172|1082466|6360|39101|2|84|6|1|12656|21|380|87|68.61|115.95|37.10|6859.95|3227.70|5969.07|10087.65|0.00|0.00|2420.34|3227.70|3227.70|5648.04|5648.04|-2741.37| +2450822|67372|2450860|58887|232830|2326|37869|58887|232830|2326|37869|4|70|9|3|1334|13|381|79|60.09|156.23|128.10|2222.27|10119.90|4747.11|12342.17|202.39|0.00|1974.21|10119.90|10322.29|12094.11|12296.50|5372.79| +2450822|67372|2450840|58887|232830|2326|37869|58887|232830|2326|37869|4|61|6|3|15890|100|381|66|45.42|57.22|52.07|339.90|3436.62|2997.72|3776.52|269.43|68.73|906.18|3367.89|3637.32|4274.07|4543.50|370.17| +2450822|67372|2450855|58887|232830|2326|37869|58887|232830|2326|37869|4|70|5|5|9796|25|381|77|28.39|34.63|19.73|1147.30|1519.21|2186.03|2666.51|45.57|0.00|1305.92|1519.21|1564.78|2825.13|2870.70|-666.82| +2450822|67372|2450826|58887|232830|2326|37869|58887|232830|2326|37869|4|86|2|4|15008|141|381|51|95.50|155.66|24.90|6668.76|1269.90|4870.50|7938.66|114.29|0.00|3889.77|1269.90|1384.19|5159.67|5273.96|-3600.60| +2450822|67372|2450905|58887|232830|2326|37869|58887|232830|2326|37869|4|53|11|4|5656|210|381|69|57.36|118.73|98.54|1393.11|6799.26|3957.84|8192.37|543.94|0.00|3932.31|6799.26|7343.20|10731.57|11275.51|2841.42| +2450822|67372|2450871|58887|232830|2326|37869|58887|232830|2326|37869|4|23|13|4|6463|228|381|97|83.94|97.37|83.73|1323.08|8121.81|8142.18|9444.89|324.87|0.00|94.09|8121.81|8446.68|8215.90|8540.77|-20.37| +2450822|67372|2450897|58887|232830|2326|37869|58887|232830|2326|37869|4|19|1|3|13651|85|381|99|35.59|62.63|11.27|5084.64|1115.73|3523.41|6200.37|0.00|0.00|743.49|1115.73|1115.73|1859.22|1859.22|-2407.68| +2450822|67372|2450870|58887|232830|2326|37869|58887|232830|2326|37869|4|77|6|5|17210|180|381|39|36.22|51.79|33.14|727.35|1292.46|1412.58|2019.81|30.24|956.42|282.75|336.04|366.28|618.79|649.03|-1076.54| +2450822|61915|2450880|44118|1712118|471|44471|44118|1712118|471|44471|4|87|4|4|12404|243|382|71|49.85|93.71|64.65|2063.26|4590.15|3539.35|6653.41|229.50|0.00|2328.09|4590.15|4819.65|6918.24|7147.74|1050.80| +2450822|61915|2450830|44118|1712118|471|44471|44118|1712118|471|44471|4|54|15|3|11848|61|382|22|60.05|61.25|9.80|1131.90|215.60|1321.10|1347.50|4.57|101.33|80.74|114.27|118.84|195.01|199.58|-1206.83| +2450822|61915|2450894|44118|1712118|471|44471|44118|1712118|471|44471|4|82|9|5|5738|20|382|33|82.59|198.21|65.40|4382.73|2158.20|2725.47|6540.93|107.91|0.00|2681.58|2158.20|2266.11|4839.78|4947.69|-567.27| +2450822|61915|2450830|44118|1712118|471|44471|44118|1712118|471|44471|4|104|15|4|7604|122|382|96|24.17|58.24|29.12|2795.52|2795.52|2320.32|5591.04|0.00|0.00|2067.84|2795.52|2795.52|4863.36|4863.36|475.20| +2450822|61915|2450872|44118|1712118|471|44471|44118|1712118|471|44471|4|90|8|2|3542|220|382|19|35.89|50.96|27.00|455.24|513.00|681.91|968.24|10.26|0.00|174.23|513.00|523.26|687.23|697.49|-168.91| +2450822|61915|2450826|44118|1712118|471|44471|44118|1712118|471|44471|4|86|18|3|4336|47|382|74|41.77|46.78|4.21|3150.18|311.54|3090.98|3461.72|5.29|205.61|968.66|105.93|111.22|1074.59|1079.88|-2985.05| +2450822|61915|2450843|44118|1712118|471|44471|44118|1712118|471|44471|4|97|20|4|4837|198|382|61|62.24|97.09|94.17|178.12|5744.37|3796.64|5922.49|114.88|0.00|2605.31|5744.37|5859.25|8349.68|8464.56|1947.73| +2450822|61915|2450885|44118|1712118|471|44471|44118|1712118|471|44471|4|2|16|3|6973|154|382|36|49.77|96.05|90.28|207.72|3250.08|1791.72|3457.80|162.50|0.00|1417.68|3250.08|3412.58|4667.76|4830.26|1458.36| +2450822|61915|2450901|44118|1712118|471|44471|44118|1712118|471|44471|4|47|4|5|6145|168|382|62|65.32|107.77|48.49|3675.36|3006.38|4049.84|6681.74|53.51|330.70|1937.50|2675.68|2729.19|4613.18|4666.69|-1374.16| +2450822|61915|2450845|44118|1712118|471|44471|44118|1712118|471|44471|4|61|20|3|16783|221|382|37|93.77|158.47|125.19|1231.36|4632.03|3469.49|5863.39|46.32|0.00|175.75|4632.03|4678.35|4807.78|4854.10|1162.54| +2450822|62884|2450857|83626|1148028|6830|13279|83626|1148028|6830|13279|2|55|11|4|15188|242|383|82|9.89|14.24|5.55|712.58|455.10|810.98|1167.68|31.85|0.00|361.62|455.10|486.95|816.72|848.57|-355.88| +2450822|62884|2450859|83626|1148028|6830|13279|83626|1148028|6830|13279|2|90|13|5|5359|234|383|89|55.44|69.30|1.38|6044.88|122.82|4934.16|6167.70|0.00|0.00|986.12|122.82|122.82|1108.94|1108.94|-4811.34| +2450822|62884|2450875|83626|1148028|6830|13279|83626|1148028|6830|13279|2|4|7|2|15710|81|383|69|75.06|192.15|128.74|4375.29|8883.06|5179.14|13258.35|799.47|0.00|6363.87|8883.06|9682.53|15246.93|16046.40|3703.92| +2450822|62884|2450902|83626|1148028|6830|13279|83626|1148028|6830|13279|2|103|2|1|15679|11|383|52|18.92|45.78|2.74|2238.08|142.48|983.84|2380.56|0.69|132.50|523.64|9.98|10.67|533.62|534.31|-973.86| +2450822|74450|2450829|60871|129502|1333|38717|60871|129502|1333|38717|1|39|15|3|2881|172|384|23|46.39|66.80|19.37|1090.89|445.51|1066.97|1536.40|8.91|0.00|537.74|445.51|454.42|983.25|992.16|-621.46| +2450822|74450|2450886|60871|129502|1333|38717|60871|129502|1333|38717|1|3|7|4|9352|175|384|10|43.94|102.81|74.02|287.90|740.20|439.40|1028.10|14.80|0.00|102.80|740.20|755.00|843.00|857.80|300.80| +2450822|74450|2450895|60871|129502|1333|38717|60871|129502|1333|38717|1|57|14|4|5078|84|384|95|3.34|3.50|0.14|319.20|13.30|317.30|332.50|0.26|0.00|26.60|13.30|13.56|39.90|40.16|-304.00| +2450822|74450|2450867|60871|129502|1333|38717|60871|129502|1333|38717|1|9|3|5|17422|50|384|87|82.16|226.76|47.61|15586.05|4142.07|7147.92|19728.12|138.75|1366.88|5917.74|2775.19|2913.94|8692.93|8831.68|-4372.73| +2450822|74450|2450879|60871|129502|1333|38717|60871|129502|1333|38717|1|14|20|4|7879|277|384|40|46.59|89.91|82.71|288.00|3308.40|1863.60|3596.40|231.58|0.00|1042.80|3308.40|3539.98|4351.20|4582.78|1444.80| +2450822|74450|2450831|60871|129502|1333|38717|60871|129502|1333|38717|1|104|4|1|4844|183|384|9|6.99|11.32|3.28|72.36|29.52|62.91|101.88|0.88|0.00|13.23|29.52|30.40|42.75|43.63|-33.39| +2450822|74450|2450868|60871|129502|1333|38717|60871|129502|1333|38717|1|43|7|4|12128|227|384|27|26.50|68.90|2.06|1804.68|55.62|715.50|1860.30|2.22|0.00|409.05|55.62|57.84|464.67|466.89|-659.88| +2450822|74450|2450826|60871|129502|1333|38717|60871|129502|1333|38717|1|27|6|1|17444|298|384|77|2.74|6.30|4.41|145.53|339.57|210.98|485.10|3.39|0.00|194.04|339.57|342.96|533.61|537.00|128.59| +|74450|2450837|60871|||38717|60871|129502||38717||12|||1672|291|384|||||4404.36|3751.56|4687.41|||1688.20|||2083.99|4183.43|4204.06|-2624.05| +2450822|74450|2450892|60871|129502|1333|38717|60871|129502|1333|38717|1|2|13|3|8762|176|384|4|73.07|151.98|107.90|176.32|431.60|292.28|607.92|0.64|366.86|273.56|64.74|65.38|338.30|338.94|-227.54| +2450822|69964|2450907|54730|1702870|5410|30350|54730|1702870|5410|30350|1|57|4|3|16700|68|385|5|99.87|112.85|39.49|366.80|197.45|499.35|564.25|15.79|0.00|62.05|197.45|213.24|259.50|275.29|-301.90| +2450822|69964|2450860|54730|1702870|5410|30350|54730|1702870|5410|30350|1|59|2|2|8524|71|385|94|48.64|145.92|36.48|10287.36|3429.12|4572.16|13716.48|88.47|480.07|4526.10|2949.05|3037.52|7475.15|7563.62|-1623.11| +2450822|69964|2450861|54730|1702870|5410|30350|54730|1702870|5410|30350|1|95|7|1|10003|7|385|91|1.29|3.17|2.44|66.43|222.04|117.39|288.47|8.88|0.00|51.87|222.04|230.92|273.91|282.79|104.65| +2450822|69964|2450830|54730|1702870|5410|30350|54730|1702870|5410|30350|1|17|10|2|5036|68|385|67|58.46|133.87|111.11|1524.92|7444.37|3916.82|8969.29|556.09|1265.54|3318.51|6178.83|6734.92|9497.34|10053.43|2262.01| +2450822|69964|2450874|54730|1702870|5410|30350|54730|1702870|5410|30350|1|77|7|5|232|259|385|68|35.24|69.07|61.47|516.80|4179.96|2396.32|4696.76|41.79|0.00|751.40|4179.96|4221.75|4931.36|4973.15|1783.64| +2450822|69964|2450851|54730|1702870|5410|30350|54730|1702870|5410|30350|1|31|13|4|13364|186|385|47|9.24|19.21|9.79|442.74|460.13|434.28|902.87|18.40|0.00|288.58|460.13|478.53|748.71|767.11|25.85| +2450822|69964|2450865|54730|1702870|5410|30350|54730|1702870|5410|30350|1|4|10|2|1604|156|385|50|38.17|83.97|30.22|2687.50|1511.00|1908.50|4198.50|105.77|0.00|2015.00|1511.00|1616.77|3526.00|3631.77|-397.50| +2450822|69964|2450904|54730|1702870|5410|30350|54730|1702870|5410|30350|1|101|17|3|4522|146|385|44|49.89|63.85|47.88|702.68|2106.72|2195.16|2809.40|0.00|0.00|1179.64|2106.72|2106.72|3286.36|3286.36|-88.44| +2450822|69964|2450836|54730|1702870|5410|30350|54730|1702870|5410|30350|1|52|18|1|4339|247|385|2|61.03|104.97|5.24|199.46|10.48|122.06|209.94|0.73|0.00|37.78|10.48|11.21|48.26|48.99|-111.58| +2450822|69964|2450899|54730|1702870|5410|30350|54730|1702870|5410|30350|1|9|1|3|15596|135|385|78|74.25|220.52|46.30|13589.16|3611.40|5791.50|17200.56|72.22|0.00|5676.06|3611.40|3683.62|9287.46|9359.68|-2180.10| +2450822|69964|2450863|54730|1702870|5410|30350|54730|1702870|5410|30350|1|17|12|2|2440|199|385|64|79.28|156.97|14.12|9142.40|903.68|5073.92|10046.08|45.18|0.00|1305.60|903.68|948.86|2209.28|2254.46|-4170.24| +2450822|69964|2450876|54730|1702870|5410|30350|54730|1702870|5410|30350|1|37|6|5|6307|108|385|86|6.45|7.93|6.81|96.32|585.66|554.70|681.98|11.71|0.00|108.36|585.66|597.37|694.02|705.73|30.96| +2450822|69964|2450824|54730|1702870|5410|30350|54730|1702870|5410|30350|1|19|5|5|4082|125|385|47|32.99|96.33|44.31|2444.94|2082.57|1550.53|4527.51|62.47|0.00|678.68|2082.57|2145.04|2761.25|2823.72|532.04| +2450822|69964|2450863|54730|1702870|5410|30350|54730|1702870|5410|30350|1|36|12|4|9826|136|385|72|11.36|13.29|8.63|335.52|621.36|817.92|956.88|43.49|0.00|315.36|621.36|664.85|936.72|980.21|-196.56| +2450822|62591|2450903|58903|1551839|6822|22605|58903|1551839|6822|22605|1|103|19|2|16676|173|386|3|74.65|127.65|22.97|314.04|68.91|223.95|382.95|2.75|0.00|141.69|68.91|71.66|210.60|213.35|-155.04| +2450822|62591|2450835|58903|1551839|6822|22605|58903|1551839|6822|22605|1|99|1|4|3434|216|386|26|31.81|68.07|14.29|1398.28|371.54|827.06|1769.82|29.72|0.00|867.10|371.54|401.26|1238.64|1268.36|-455.52| +2450822|62591|2450864|58903|1551839|6822|22605|58903|1551839|6822|22605|1|87|10|4|12514|169|386|31|84.02|152.07|80.59|2215.88|2498.29|2604.62|4714.17|0.00|0.00|2309.81|2498.29|2498.29|4808.10|4808.10|-106.33| +2450822|62591|2450841|58903|1551839|6822|22605|58903|1551839|6822|22605|1|69|1|3|4873|70|386|65|61.25|168.43|143.16|1642.55|9305.40|3981.25|10947.95|837.48|0.00|1422.85|9305.40|10142.88|10728.25|11565.73|5324.15| +2450822|62591|2450833|58903|1551839|6822|22605|58903|1551839|6822|22605|1|72|8|4|10102|113|386|21|55.55|87.21|33.13|1135.68|695.73|1166.55|1831.41|62.61|0.00|109.83|695.73|758.34|805.56|868.17|-470.82| +2450822|62591|2450835|58903|1551839|6822|22605|58903|1551839|6822|22605|1|17|5|1|15859|94|386|53|95.61|223.72|26.84|10434.64|1422.52|5067.33|11857.16|85.35|0.00|2964.29|1422.52|1507.87|4386.81|4472.16|-3644.81| +2450822|71221|2450855|12291|898839|6987|44735|12291|898839|6987|44735|1|68|9|1|1156|160|387|9|43.37|78.06|32.78|407.52|295.02|390.33|702.54|17.70|0.00|252.90|295.02|312.72|547.92|565.62|-95.31| +2450822|71221|2450833|12291|898839|6987|44735|12291|898839|6987|44735|1|34|18|5|10814|22|387|65|44.49|126.79|60.85|4286.10|3955.25|2891.85|8241.35|5.53|3876.14|3625.70|79.11|84.64|3704.81|3710.34|-2812.74| +2450822|71221|2450879|12291|898839|6987|44735|12291|898839|6987|44735|1|40|3|4|15844|198|387|61|38.17|50.76|49.74|62.22|3034.14|2328.37|3096.36|151.70|0.00|525.82|3034.14|3185.84|3559.96|3711.66|705.77| +2450822|71221|2450863|12291|898839|6987|44735|12291|898839|6987|44735|1|70|8|1|16684|14|387|84|31.44|77.97|44.44|2816.52|3732.96|2640.96|6549.48|111.98|0.00|0.00|3732.96|3844.94|3732.96|3844.94|1092.00| +2450822|71221|2450898|12291|898839|6987|44735|12291|898839|6987|44735|1|14|6|1|16286|223|387|18|57.32|113.49|19.29|1695.60|347.22|1031.76|2042.82|6.94|0.00|735.30|347.22|354.16|1082.52|1089.46|-684.54| +2450822|71221|2450896|12291|898839|6987|44735|12291|898839|6987|44735|1|19|6|4|11431|247|387|68|21.96|45.45|2.72|2905.64|184.96|1493.28|3090.60|7.39|0.00|1359.32|184.96|192.35|1544.28|1551.67|-1308.32| +2450822|71221|2450880|12291|898839|6987|44735|12291|898839|6987|44735|1|11|13|5|4429|70|387|87|34.82|43.87|42.11|153.12|3663.57|3029.34|3816.69|72.53|36.63|113.97|3626.94|3699.47|3740.91|3813.44|597.60| +2450822|71221|2450839|12291|898839|6987|44735|12291|898839|6987|44735|1|44|2|5|10729|123|387|45|45.38|97.11|67.97|1311.30|3058.65|2042.10|4369.95|9.17|2141.05|1485.45|917.60|926.77|2403.05|2412.22|-1124.50| +2450822|68043|2450872|4992|1216416|2240|25311|4992|1216416|2240|25311|1|3|9|1|694|275|388|80|89.76|110.40|13.24|7772.80|1059.20|7180.80|8832.00|95.32|0.00|4416.00|1059.20|1154.52|5475.20|5570.52|-6121.60| +2450822|68043|2450876|4992|1216416|2240|25311|4992|1216416|2240|25311|1|100|17|4|17960|146|388|98|45.17|84.91|67.92|1665.02|6656.16|4426.66|8321.18|0.00|0.00|2412.76|6656.16|6656.16|9068.92|9068.92|2229.50| +2450822|68043|2450846|4992|1216416|2240|25311|4992|1216416|2240|25311|1|75|16|4|1102|206|388|88|31.72|63.75|0.63|5554.56|55.44|2791.36|5610.00|1.10|0.00|953.04|55.44|56.54|1008.48|1009.58|-2735.92| +2450822|68043|2450891|4992|1216416|2240|25311|4992|1216416|2240|25311|1|62|9|2|2170|148|388|55|58.93|175.61|136.97|2125.20|7533.35|3241.15|9658.55|301.33|0.00|2897.40|7533.35|7834.68|10430.75|10732.08|4292.20| +2450822|68043|2450898|4992|1216416|2240|25311|4992|1216416|2240|25311|1|70|5|2|112|103|388|94|13.80|35.46|23.40|1133.64|2199.60|1297.20|3333.24|21.99|0.00|132.54|2199.60|2221.59|2332.14|2354.13|902.40| +2450822|68043|2450829|4992|1216416|2240|25311|4992|1216416|2240|25311|1|26|13|5|4220|244|388|50|14.06|22.49|16.64|292.50|832.00|703.00|1124.50|0.00|0.00|438.50|832.00|832.00|1270.50|1270.50|129.00| +2450822|68043|2450887|4992|1216416|2240|25311|4992|1216416|2240|25311|1|40|3|5|11036|86|388|85|63.31|160.17|41.64|10075.05|3539.40|5381.35|13614.45|212.36|0.00|1769.70|3539.40|3751.76|5309.10|5521.46|-1841.95| +2450822|68043|2450911|4992|1216416|2240|25311|4992|1216416|2240|25311|1|64|8|3|1429|253|388|44|43.26|125.02|80.01|1980.44|3520.44|1903.44|5500.88|36.61|2605.12|2145.00|915.32|951.93|3060.32|3096.93|-988.12| +2450822|68043|2450878|4992|1216416|2240|25311|4992|1216416|2240|25311|1|31|2|4|8890|76|388|1|66.93|108.42|45.53|62.89|45.53|66.93|108.42|0.91|0.00|1.08|45.53|46.44|46.61|47.52|-21.40| +2450822|68043|2450885|4992|1216416|2240|25311|4992|1216416|2240|25311|1|104|14|1|5689|61|388|52|80.02|92.82|21.34|3716.96|1109.68|4161.04|4826.64|33.29|0.00|868.40|1109.68|1142.97|1978.08|2011.37|-3051.36| +2450822|68043|2450846|4992|1216416|2240|25311|4992|1216416|2240|25311|1|15|16|4|6896|173|388|30|23.45|47.36|25.57|653.70|767.10|703.50|1420.80|30.68|0.00|28.20|767.10|797.78|795.30|825.98|63.60| +2450822|68043|2450887|4992|1216416|2240|25311|4992|1216416|2240|25311|1|25|20|1|13441|58|388|56|61.79|100.09|88.07|673.12|4931.92|3460.24|5605.04|14.30|3501.66|336.00|1430.26|1444.56|1766.26|1780.56|-2029.98| +2450822|68043|2450895|4992|1216416|2240|25311|4992|1216416|2240|25311|1|26|4|4|3235|2|388|6|88.80|163.39|35.94|764.70|215.64|532.80|980.34|10.17|88.41|107.82|127.23|137.40|235.05|245.22|-405.57| +2450822|29659|2450828|68316|276864|2659|4054|68316|276864|2659|4054|4|9|12|2|17300|215|389|74|76.32|203.77|77.43|9349.16|5729.82|5647.68|15078.98|57.29|0.00|5427.90|5729.82|5787.11|11157.72|11215.01|82.14| +2450822|29659|2450877|68316|276864|2659|4054|68316|276864|2659|4054|4|22|3|2|1790|120|389|61|99.65|183.35|146.68|2236.87|8947.48|6078.65|11184.35|357.89|0.00|2124.63|8947.48|9305.37|11072.11|11430.00|2868.83| +2450822|29659|2450874|68316|276864|2659|4054|68316|276864|2659|4054|4|77|8|2|2174|250|389|22|76.52|143.85|86.31|1265.88|1898.82|1683.44|3164.70|3.79|1519.05|759.44|379.77|383.56|1139.21|1143.00|-1303.67| +2450822|29659|2450872|68316|276864|2659|4054|68316|276864|2659|4054|4|33|17|3|12727|175|389|98|36.91|44.29|15.50|2821.42|1519.00|3617.18|4340.42|91.14|0.00|259.70|1519.00|1610.14|1778.70|1869.84|-2098.18| +2450822|29659|2450835|68316|276864|2659|4054|68316|276864|2659|4054|4|84|10|3|11660|171|389|43|69.68|91.97|39.54|2254.49|1700.22|2996.24|3954.71|136.01|0.00|197.37|1700.22|1836.23|1897.59|2033.60|-1296.02| +2450822|29659|2450891|68316|276864|2659|4054|68316|276864|2659|4054|4|83|4|2|4666|97|389|1|95.01|195.72|138.96|56.76|138.96|95.01|195.72|12.50|0.00|82.20|138.96|151.46|221.16|233.66|43.95| +2450822|29659|2450840|68316|276864|2659|4054|68316|276864|2659|4054|4|25|2|5|5558|241|389|66|47.43|54.07|10.27|2890.80|677.82|3130.38|3568.62|1.83|494.80|1284.36|183.02|184.85|1467.38|1469.21|-2947.36| +2450822|29659|2450904|68316|276864|2659|4054|68316|276864|2659|4054|4|27|1|1|16946|37|389|72|46.82|134.37|49.71|6095.52|3579.12|3371.04|9674.64|250.53|0.00|1740.96|3579.12|3829.65|5320.08|5570.61|208.08| +2450822|72617|2450908|20319|87894|269|4597|20319|87894|269|4597|4|92|4|4|5680|1|390|3|8.16|15.50|10.07|16.29|30.21|24.48|46.50|1.51|0.00|12.54|30.21|31.72|42.75|44.26|5.73| +2450822|72617|2450833|20319|87894|269|4597|20319|87894|269|4597|4|75|1|2|5149|223|390|92|71.06|135.72|74.64|5619.36|6866.88|6537.52|12486.24|68.66|0.00|3370.88|6866.88|6935.54|10237.76|10306.42|329.36| +2450822|72617|2450845|20319|87894|269|4597|20319|87894|269|4597|4|15|9|2|338|188|390|18|73.84|189.03|9.45|3232.44|170.10|1329.12|3402.54|13.60|0.00|1088.64|170.10|183.70|1258.74|1272.34|-1159.02| +2450822|72617|2450871|20319|87894|269|4597|20319|87894|269|4597|4|86|6|3|3428|135|390|29|68.05|186.45|175.26|324.51|5082.54|1973.45|5407.05|254.12|0.00|2000.42|5082.54|5336.66|7082.96|7337.08|3109.09| +2450822|72617|2450876|20319|87894|269|4597|20319|87894|269|4597|4|54|17|2|16969|29|390|48|79.51|225.80|76.77|7153.44|3684.96|3816.48|10838.40|147.39|0.00|3359.52|3684.96|3832.35|7044.48|7191.87|-131.52| +2450822|72617|2450878|20319|87894|269|4597|20319|87894|269|4597|4|102|9|5|14048|132|390|26|7.01|16.40|10.66|149.24|277.16|182.26|426.40|7.20|133.03|46.80|144.13|151.33|190.93|198.13|-38.13| +2450822|72617|2450879|20319|87894|269|4597|20319|87894|269|4597|4|76|8|1|9728|59|390|25|64.55|118.12|20.08|2451.00|502.00|1613.75|2953.00|20.08|0.00|324.75|502.00|522.08|826.75|846.83|-1111.75| +2450822|72617|2450890|20319|87894|269|4597|20319|87894|269|4597|4|42|14|5|772|160|390|91|64.58|180.82|155.50|2304.12|14150.50|5876.78|16454.62|141.50|0.00|4277.91|14150.50|14292.00|18428.41|18569.91|8273.72| +2450822|72617|2450883|20319|87894|269|4597|20319|87894|269|4597|4|29|7|3|1838|265|390|71|12.04|32.38|7.77|1747.31|551.67|854.84|2298.98|20.85|319.96|827.15|231.71|252.56|1058.86|1079.71|-623.13| +2450822|72617|2450910|20319|87894|269|4597|20319|87894|269|4597|4|38|20|3|2822|169|390|68|75.13|191.58|118.77|4951.08|8076.36|5108.84|13027.44|403.81|0.00|5601.16|8076.36|8480.17|13677.52|14081.33|2967.52| +2450822|72617|2450830|20319|87894|269|4597|20319|87894|269|4597|4|57|11|4|13366|263|390|75|60.17|167.27|41.81|9409.50|3135.75|4512.75|12545.25|0.00|0.00|1630.50|3135.75|3135.75|4766.25|4766.25|-1377.00| +2450822|72617|2450858|20319|87894|269|4597|20319|87894|269|4597|4|26|1|4|12874|272|390|86|65.66|137.88|102.03|3083.10|8774.58|5646.76|11857.68|526.47|0.00|2845.74|8774.58|9301.05|11620.32|12146.79|3127.82| +2450822|55668|2450865|82333|341522|3558|28747|82333|341522|3558|28747|2|11|6|4|2065|89|391|73|37.78|96.71|41.58|4024.49|3035.34|2757.94|7059.83|212.47|0.00|493.48|3035.34|3247.81|3528.82|3741.29|277.40| +2450822|55668|2450897|82333|341522|3558|28747|82333|341522|3558|28747|2|23|2|4|6937|74|391|90|89.15|96.28|81.83|1300.50|7364.70|8023.50|8665.20|73.64|0.00|865.80|7364.70|7438.34|8230.50|8304.14|-658.80| +2450822|55668|2450838|82333|341522|3558|28747|82333|341522|3558|28747|2|68|5|2|7406|264|391|42|76.10|213.08|125.71|3669.54|5279.82|3196.20|8949.36|369.58|0.00|2595.18|5279.82|5649.40|7875.00|8244.58|2083.62| +2450822|55668|2450836|82333|341522|3558|28747|82333|341522|3558|28747|2|98|16|5|15193|35|391|80|42.78|57.75|46.77|878.40|3741.60|3422.40|4620.00|37.41|0.00|184.80|3741.60|3779.01|3926.40|3963.81|319.20| +2450822|55668|2450868|82333|341522|3558|28747|82333|341522|3558|28747|2|92|3|5|12979|224|391|10|1.09|1.52|0.63|8.90|6.30|10.90|15.20|0.50|0.00|5.00|6.30|6.80|11.30|11.80|-4.60| +2450822|55668|2450889|82333|341522|3558|28747|82333|341522|3558|28747|2|36|5|1|15284|171|391|21|63.23|89.15|9.80|1666.35|205.80|1327.83|1872.15|8.23|0.00|131.04|205.80|214.03|336.84|345.07|-1122.03| +2450822|44055|2450891|39161|342553|345|26537|86678|1496121|1113|9705|4|86|1|3|8722|245|392|62|38.73|90.24|75.80|895.28|4699.60|2401.26|5594.88|46.99|0.00|2629.42|4699.60|4746.59|7329.02|7376.01|2298.34| +2450822|44055|2450899|39161|342553|345|26537|86678|1496121|1113|9705|4|34|9|2|16136|160|392|61|43.04|105.44|4.21|6175.03|256.81|2625.44|6431.84|5.34|190.03|450.18|66.78|72.12|516.96|522.30|-2558.66| +2450822|44055|2450876|39161|342553|345|26537|86678|1496121|1113|9705|4|22|2|5|1340|144|392|54|76.39|101.59|72.12|1591.38|3894.48|4125.06|5485.86|60.75|2881.91|658.26|1012.57|1073.32|1670.83|1731.58|-3112.49| +2450822|44055|2450864|39161|342553|345|26537|86678|1496121|1113|9705|4|6|20|4|9973|27|392|26|7.85|17.81|16.56|32.50|430.56|204.10|463.06|12.91|0.00|166.66|430.56|443.47|597.22|610.13|226.46| +2450822|44055|2450877|39161|342553|345|26537|86678|1496121|1113|9705|4|95|12|1|7450|277|392|95|89.32|133.08|91.82|3919.70|8722.90|8485.40|12642.60|357.63|1570.12|4424.15|7152.78|7510.41|11576.93|11934.56|-1332.62| +2450822|44055|2450885|39161|342553|345|26537|86678|1496121|1113|9705|4|20|1|1|11617|66|392|37|24.77|70.59|63.53|261.22|2350.61|916.49|2611.83|31.02|1833.47|287.12|517.14|548.16|804.26|835.28|-399.35| +2450822|44055|2450887|39161|342553|345|26537|86678|1496121|1113|9705|4|87|13|5|13720|157|392|77|26.13|48.34|25.13|1787.17|1935.01|2012.01|3722.18|58.05|0.00|446.60|1935.01|1993.06|2381.61|2439.66|-77.00| +2450822|44055|2450833|39161|342553|345|26537|86678|1496121|1113|9705|4|91|7|2|7006|272|392|12|96.36|158.99|54.05|1259.28|648.60|1156.32|1907.88|58.37|0.00|724.92|648.60|706.97|1373.52|1431.89|-507.72| +2450822|44055|2450850|39161|342553|345|26537|86678|1496121|1113|9705|4|100|17|1|14347|212|392|62|15.34|21.47|5.79|972.16|358.98|951.08|1331.14|28.71|0.00|319.30|358.98|387.69|678.28|706.99|-592.10| +2450822|44055|2450868|39161|342553|345|26537|86678|1496121|1113|9705|4|24|16|5|12592|29|392|10|52.51|92.94|53.90|390.40|539.00|525.10|929.40|5.39|0.00|232.30|539.00|544.39|771.30|776.69|13.90| +2450822|44055|2450869|39161|342553|345|26537|86678|1496121|1113|9705|4|10|19|2|6757|233|392|90|25.88|27.69|16.33|1022.40|1469.70|2329.20|2492.10|102.87|0.00|273.60|1469.70|1572.57|1743.30|1846.17|-859.50| +2450822|44055|2450839|39161|342553|345|26537|86678|1496121|1113|9705|4|50|5|2|11035|256|392|33|22.56|37.22|7.07|994.95|233.31|744.48|1228.26|18.66|0.00|24.42|233.31|251.97|257.73|276.39|-511.17| +2450822|44055|2450901|39161|342553|345|26537|86678|1496121|1113|9705|4|77|4|5|10796|117|392|9|90.67|152.32|114.24|342.72|1028.16|816.03|1370.88|30.84|0.00|0.00|1028.16|1059.00|1028.16|1059.00|212.13| +2450822|44055|2450909|39161|342553|345|26537|86678|1496121|1113|9705|4|38|12|4|10444|230|392|16|72.81|160.91|125.50|566.56|2008.00|1164.96|2574.56|77.30|461.84|1287.20|1546.16|1623.46|2833.36|2910.66|381.20| +2450822|43961|2450898|57937|97090|1396|46949|13340|1157608|3161|9867|2|108|17|1|2905|116|393|31|61.56|120.65|42.22|2431.33|1308.82|1908.36|3740.15|39.26|0.00|1869.92|1308.82|1348.08|3178.74|3218.00|-599.54| +2450822|43961|2450879|57937|97090|1396|46949|13340|1157608|3161|9867|2|57|17|2|11776|39|393|26|18.69|53.64|50.95|69.94|1324.70|485.94|1394.64|33.91|476.89|83.46|847.81|881.72|931.27|965.18|361.87| +2450822|43961|2450905|57937|97090|1396|46949|13340|1157608|3161|9867|2|12|18|3|16262|80|393|68|96.91|233.55|98.09|9211.28|6670.12|6589.88|15881.40|66.70|0.00|4604.96|6670.12|6736.82|11275.08|11341.78|80.24| +2450822|43961|2450845|57937|97090|1396|46949|13340|1157608|3161|9867|2|6|11|2|3766|117|393|44|62.45|165.49|157.21|364.32|6917.24|2747.80|7281.56|37.35|3181.93|363.88|3735.31|3772.66|4099.19|4136.54|987.51| +2450822|43961|2450836|57937|97090|1396|46949|13340|1157608|3161|9867|2|77|19|3|8242|124|393|36|24.78|62.19|28.60|1209.24|1029.60|892.08|2238.84|61.77|0.00|66.96|1029.60|1091.37|1096.56|1158.33|137.52| +2450822|43961|2450850|57937|97090|1396|46949|13340|1157608|3161|9867|2|99|18|3|5329|155|393|88|6.77|8.80|1.76|619.52|154.88|595.76|774.40|10.84|0.00|316.80|154.88|165.72|471.68|482.52|-440.88| +2450822|43961|2450831|57937|97090|1396|46949|13340|1157608|3161|9867|2|47|14|4|13604|217|393|73|39.29|104.11|78.08|1900.19|5699.84|2868.17|7600.03|284.99|0.00|607.36|5699.84|5984.83|6307.20|6592.19|2831.67| +2450822|78550|2450841|75744|282140|2489|10038|75744|282140|2489|10038|1|28|10|2|3662|243|394|68|15.44|45.85|27.51|1247.12|1870.68|1049.92|3117.80|49.38|1253.35|405.28|617.33|666.71|1022.61|1071.99|-432.59| +2450822|78550|2450846|75744|282140|2489|10038|75744|282140|2489|10038|1|74|20|1|14452|173|394|15|53.63|84.73|26.26|877.05|393.90|804.45|1270.95|1.37|256.03|38.10|137.87|139.24|175.97|177.34|-666.58| +2450822|78550|2450908|75744|282140|2489|10038|75744|282140|2489|10038|1|30|18|3|5468|48|394|42|79.19|213.81|83.38|5478.06|3501.96|3325.98|8980.02|123.26|1961.09|1167.18|1540.87|1664.13|2708.05|2831.31|-1785.11| +2450822|78550|2450889|75744|282140|2489|10038|75744|282140|2489|10038|1|80|8|3|7658|110|394|99|29.31|71.51|60.78|1062.27|6017.22|2901.69|7079.49|240.68|0.00|1698.84|6017.22|6257.90|7716.06|7956.74|3115.53| +2450822|44428|2450875|53367|1408640|6313|19986|53367|1408640|6313|19986|4|41|7|3|17666|56|395|73|24.46|50.14|29.08|1537.38|2122.84|1785.58|3660.22|169.82|0.00|951.19|2122.84|2292.66|3074.03|3243.85|337.26| +2450822|44428|2450829|53367|1408640|6313|19986|53367|1408640|6313|19986|4|87|20|1|17401|12|395|46|76.07|111.06|87.73|1073.18|4035.58|3499.22|5108.76|80.71|0.00|868.48|4035.58|4116.29|4904.06|4984.77|536.36| +2450822|44428|2450866|53367|1408640|6313|19986|53367|1408640|6313|19986|4|26|12|1|16267|241|395|32|26.80|53.60|4.82|1560.96|154.24|857.60|1715.20|10.79|0.00|205.76|154.24|165.03|360.00|370.79|-703.36| +2450822|44428|2450841|53367|1408640|6313|19986|53367|1408640|6313|19986|4|46|15|4|11059|139|395|86|87.66|109.57|13.14|8292.98|1130.04|7538.76|9423.02|79.10|0.00|1224.64|1130.04|1209.14|2354.68|2433.78|-6408.72| +2450822|44428|2450832|53367|1408640|6313|19986|53367|1408640|6313|19986|4|100|7|2|5894|178|395|42|54.25|155.69|28.02|5362.14|1176.84|2278.50|6538.98|11.76|0.00|784.56|1176.84|1188.60|1961.40|1973.16|-1101.66| +2450822|44428|2450833|53367|1408640|6313|19986|53367|1408640|6313|19986|4|54|1|3|3808|55|395|85|1.85|2.46|0.49|167.45|41.65|157.25|209.10|1.24|0.00|62.05|41.65|42.89|103.70|104.94|-115.60| +2450822|44428|2450848|53367|1408640|6313|19986|53367|1408640|6313|19986|4|78|11|2|12554|174|395|45|83.32|247.46|89.08|7127.10|4008.60|3749.40|11135.70|200.43|0.00|779.40|4008.60|4209.03|4788.00|4988.43|259.20| +2450822|44428|2450864|53367|1408640|6313|19986|53367|1408640|6313|19986|4|9|15|4|4609|75|395|27|52.10|149.52|0.00|4037.04|0.00|1406.70|4037.04|0.00|0.00|242.19|0.00|0.00|242.19|242.19|-1406.70| +2450822|44428|2450908|53367|1408640|6313|19986|53367|1408640|6313|19986|4|51|17|3|16234|90|395|43|6.87|13.94|3.48|449.78|149.64|295.41|599.42|8.97|0.00|275.63|149.64|158.61|425.27|434.24|-145.77| +2450822|44428|2450864|53367|1408640|6313|19986|53367|1408640|6313|19986|4|42|4|4|14900|92|395|96|17.40|51.50|32.44|1829.76|3114.24|1670.40|4944.00|0.00|3114.24|1680.96|0.00|0.00|1680.96|1680.96|-1670.40| +2450822|44428|2450889|53367|1408640|6313|19986|53367|1408640|6313|19986|4|10|9|1|2434|128|395|3|58.58|87.28|54.98|96.90|164.94|175.74|261.84|0.00|94.01|26.16|70.93|70.93|97.09|97.09|-104.81| +2450822|44428|2450886|53367|1408640|6313|19986|53367|1408640|6313|19986|4|81|14|5|5980|177|395|5|76.44|77.96|45.99|159.85|229.95|382.20|389.80|13.68|34.49|74.05|195.46|209.14|269.51|283.19|-186.74| +2450822|44428|2450898|53367|1408640|6313|19986|53367|1408640|6313|19986|4|105|5|2|8539|251|395|39|82.43|172.27|87.85|3292.38|3426.15|3214.77|6718.53|0.00|0.00|2888.73|3426.15|3426.15|6314.88|6314.88|211.38| +2450822|44428|2450847|53367|1408640|6313|19986|53367|1408640|6313|19986|4|103|16|4|7610|58|395|54|19.71|26.60|8.77|962.82|473.58|1064.34|1436.40|9.47|0.00|200.88|473.58|483.05|674.46|683.93|-590.76| +2450822|64897|2450874|79977|1128919|2661|50|79977|1128919|2661|50|2|75|7|5|11204|17|396|100|25.62|34.84|1.04|3380.00|104.00|2562.00|3484.00|2.08|0.00|313.00|104.00|106.08|417.00|419.08|-2458.00| +2450822|64897|2450826|79977|1128919|2661|50|79977|1128919|2661|50|2|55|3|5|15979|69|396|55|39.17|57.97|6.37|2838.00|350.35|2154.35|3188.35|28.02|0.00|1179.20|350.35|378.37|1529.55|1557.57|-1804.00| +2450822|64897|2450855|79977|1128919|2661|50|79977|1128919|2661|50|2|81|8|4|17467|277|396|88|33.66|50.15|27.08|2030.16|2383.04|2962.08|4413.20|190.64|0.00|2074.16|2383.04|2573.68|4457.20|4647.84|-579.04| +2450822|64897|2450860|79977|1128919|2661|50|79977|1128919|2661|50|2|100|6|1|9770|87|396|1|24.69|71.84|25.86|45.98|25.86|24.69|71.84|0.51|0.00|30.89|25.86|26.37|56.75|57.26|1.17| +2450822|64897|2450824|79977|1128919|2661|50|79977|1128919|2661|50|2|53|17|5|2305|124|396|41|36.24|72.84|72.11|29.93|2956.51|1485.84|2986.44|118.26|0.00|1373.50|2956.51|3074.77|4330.01|4448.27|1470.67| +2450822|64897|2450900|79977|1128919|2661|50|79977|1128919|2661|50|2|104|15|2|14414|131|396|19|11.01|14.75|8.40|120.65|159.60|209.19|280.25|11.17|0.00|137.18|159.60|170.77|296.78|307.95|-49.59| +2450822|64897|2450870|79977|1128919|2661|50|79977|1128919|2661|50|2|105|5|2|12130|293|396|83|7.11|14.36|6.46|655.70|536.18|590.13|1191.88|3.37|498.64|309.59|37.54|40.91|347.13|350.50|-552.59| +2450822|64897|2450847|79977|1128919|2661|50|79977|1128919|2661|50|2|40|15|4|17390|71|396|100|71.36|160.56|51.37|10919.00|5137.00|7136.00|16056.00|308.22|0.00|3371.00|5137.00|5445.22|8508.00|8816.22|-1999.00| +2450822|64897|2450889|79977|1128919|2661|50|79977|1128919|2661|50|2|66|13|4|4646|49|396|42|78.19|100.86|36.30|2711.52|1524.60|3283.98|4236.12|121.96|0.00|2075.64|1524.60|1646.56|3600.24|3722.20|-1759.38| +2450822|64897|2450827|79977|1128919|2661|50|79977|1128919|2661|50|2|15|20|1|15589|23|396|38|59.31|77.10|44.71|1230.82|1698.98|2253.78|2929.80|67.95|0.00|849.30|1698.98|1766.93|2548.28|2616.23|-554.80| +2450822|64897|2450880|79977|1128919|2661|50|79977|1128919|2661|50|2|42|19|1|10879|123|396|43|31.86|80.92|67.16|591.68|2887.88|1369.98|3479.56|115.51|0.00|591.25|2887.88|3003.39|3479.13|3594.64|1517.90| +2450822|18849|2450908|94515|508804|693|8982|94515|508804|693|8982|4|61|17|2|10750|291|397|98|92.69|200.21|184.19|1569.96|18050.62|9083.62|19620.58|693.14|6498.22|5885.88|11552.40|12245.54|17438.28|18131.42|2468.78| +2450822|18849|2450836|94515|508804|693|8982|94515|508804|693|8982|4|94|13|3|6901|177|397|13|97.08|122.32|68.49|699.79|890.37|1262.04|1590.16|62.32|0.00|317.98|890.37|952.69|1208.35|1270.67|-371.67| +2450822|18849|2450842|94515|508804|693|8982|94515|508804|693|8982|4|35|18|3|4798|136|397|10|26.23|55.08|42.41|126.70|424.10|262.30|550.80|38.16|0.00|187.20|424.10|462.26|611.30|649.46|161.80| +2450822|18849|2450881|94515|508804|693|8982|94515|508804|693|8982|4|29|10|3|13082|234|397|40|10.03|20.76|13.28|299.20|531.20|401.20|830.40|15.93|0.00|415.20|531.20|547.13|946.40|962.33|130.00| +2450822|18849|2450880|94515|508804|693|8982|94515|508804|693|8982|4|76|12|3|12226|208|397|12|90.28|104.72|30.36|892.32|364.32|1083.36|1256.64|3.64|0.00|477.48|364.32|367.96|841.80|845.44|-719.04| +2450822|18849|2450829|94515|508804|693|8982|94515|508804|693|8982|4|53|5|4|3463|209|397|76|32.68|64.70|34.29|2311.16|2606.04|2483.68|4917.20|104.24|0.00|98.04|2606.04|2710.28|2704.08|2808.32|122.36| +2450822|18849|2450851|94515|508804|693|8982|94515|508804|693|8982|4|15|20|3|1900|121|397|22|61.50|153.75|119.92|744.26|2638.24|1353.00|3382.50|79.14|0.00|135.30|2638.24|2717.38|2773.54|2852.68|1285.24| +2450822|71908|2450903|33565|253452|3292|11179|33565|253452|3292|11179|2|28|20|1|5065|43|398|64|91.83|185.49|137.26|3086.72|8784.64|5877.12|11871.36|498.96|2547.54|2611.20|6237.10|6736.06|8848.30|9347.26|359.98| +2450822|71908|2450899|33565|253452|3292|11179|33565|253452|3292|11179|2|34|12|5|4226|34|398|64|45.70|120.19|57.69|4000.00|3692.16|2924.80|7692.16|25.47|1144.56|846.08|2547.60|2573.07|3393.68|3419.15|-377.20| +2450822|71908|2450829|33565|253452|3292|11179|33565|253452|3292|11179|2|47|11|2|7561|2|398|58|64.28|95.13|51.37|2538.08|2979.46|3728.24|5517.54|238.35|0.00|1268.46|2979.46|3217.81|4247.92|4486.27|-748.78| +2450822|71908|2450838|33565|253452|3292|11179|33565|253452|3292|11179|2|4|20|1|2158|218|398|76|25.99|40.54|29.99|801.80|2279.24|1975.24|3081.04|0.00|0.00|30.40|2279.24|2279.24|2309.64|2309.64|304.00| +2450822|71908|2450910|33565|253452|3292|11179|33565|253452|3292|11179|2|10|16|5|1226|143|398|52|99.01|203.96|97.90|5515.12|5090.80|5148.52|10605.92|152.72|0.00|3499.60|5090.80|5243.52|8590.40|8743.12|-57.72| +2450822|71908|2450853|33565|253452|3292|11179|33565|253452|3292|11179|2|27|4|4|12302|151|398|74|21.23|56.47|40.65|1170.68|3008.10|1571.02|4178.78|0.00|0.00|1670.92|3008.10|3008.10|4679.02|4679.02|1437.08| +2450822|71908|2450845|33565|253452|3292|11179|33565|253452|3292|11179|2|45|5|5|7100|62|398|64|47.22|113.32|16.99|6165.12|1087.36|3022.08|7252.48|76.11|0.00|652.16|1087.36|1163.47|1739.52|1815.63|-1934.72| +2450822|71908|2450857|33565|253452|3292|11179|33565|253452|3292|11179|2|91|2|1|15940|257|398|47|77.77|205.31|160.14|2122.99|7526.58|3655.19|9649.57|124.18|3386.96|3955.99|4139.62|4263.80|8095.61|8219.79|484.43| +2450822|71908|2450902|33565|253452|3292|11179|33565|253452|3292|11179|2|22|10|3|6854|290|398|24|68.02|78.90|70.22|208.32|1685.28|1632.48|1893.60|84.26|0.00|227.04|1685.28|1769.54|1912.32|1996.58|52.80| +2450822|71908|2450872|33565|253452|3292|11179|33565|253452|3292|11179|2|52|3|3|6074|238|398|71|20.50|29.72|24.37|379.85|1730.27|1455.50|2110.12|51.90|0.00|927.97|1730.27|1782.17|2658.24|2710.14|274.77| +2450822|71908|2450894|33565|253452|3292|11179|33565|253452|3292|11179|2|58|20|5|5443|125|398|38|40.47|42.49|22.09|775.20|839.42|1537.86|1614.62|67.15|0.00|484.12|839.42|906.57|1323.54|1390.69|-698.44| +2450822|71908|2450864|33565|253452|3292|11179|33565|253452|3292|11179|2|71|5|3|13250|244|398|24|15.93|18.47|17.36|26.64|416.64|382.32|443.28|4.79|320.81|84.00|95.83|100.62|179.83|184.62|-286.49| +2450822|62149|2450842|46107|1261866|3672|631|46107|1261866|3672|631|2|68|19|2|14902|46|399|9|67.21|199.61|15.96|1652.85|143.64|604.89|1796.49|10.05|0.00|736.56|143.64|153.69|880.20|890.25|-461.25| +2450822|62149|2450909|46107|1261866|3672|631|46107|1261866|3672|631|2|15|4|1|6170|68|399|1|29.28|41.28|30.13|11.15|30.13|29.28|41.28|0.30|0.00|9.08|30.13|30.43|39.21|39.51|0.85| +2450822|62149|2450841|46107|1261866|3672|631|46107|1261866|3672|631|2|44|2|5|8917|258|399|86|18.21|20.39|4.07|1403.52|350.02|1566.06|1753.54|14.00|0.00|69.66|350.02|364.02|419.68|433.68|-1216.04| +2450822|62149|2450890|46107|1261866|3672|631|46107|1261866|3672|631|2|7|16|1|6524|19|399|81|76.77|155.07|71.33|6782.94|5777.73|6218.37|12560.67|115.55|0.00|5652.18|5777.73|5893.28|11429.91|11545.46|-440.64| +2450822|62149|2450908|46107|1261866|3672|631|46107|1261866|3672|631|2|48|8|5|11734|279|399|96|12.56|37.05|0.74|3485.76|71.04|1205.76|3556.80|0.71|0.00|1671.36|71.04|71.75|1742.40|1743.11|-1134.72| +2450822|62149|2450886|46107|1261866|3672|631|46107|1261866|3672|631|2|73|16|2|14194|9|399|82|65.61|103.66|31.09|5950.74|2549.38|5380.02|8500.12|152.96|0.00|3824.48|2549.38|2702.34|6373.86|6526.82|-2830.64| +2450822|62149|2450868|46107|1261866|3672|631|46107|1261866|3672|631|2|92|2|1|1201|150|399|68|44.38|94.08|32.92|4158.88|2238.56|3017.84|6397.44|0.00|447.71|2942.36|1790.85|1790.85|4733.21|4733.21|-1226.99| +2450822|62149|2450833|46107|1261866|3672|631|46107|1261866|3672|631|2|34|8|4|5240|21|399|69|85.75|159.49|100.47|4072.38|6932.43|5916.75|11004.81|207.97|0.00|2751.03|6932.43|7140.40|9683.46|9891.43|1015.68| +2450822|62149|2450831|46107|1261866|3672|631|46107|1261866|3672|631|2|22|3|5|3952|81|399|10|40.56|121.68|52.32|693.60|523.20|405.60|1216.80|0.00|0.00|535.30|523.20|523.20|1058.50|1058.50|117.60| +2450822|62149|2450866|46107|1261866|3672|631|46107|1261866|3672|631|2|102|9|4|9868|279|399|26|93.41|221.38|57.55|4259.58|1496.30|2428.66|5755.88|44.88|0.00|172.64|1496.30|1541.18|1668.94|1713.82|-932.36| +2450822|||46107|1261866|3672|631||1261866|||2|77|3||10810||399|5|15.20|||||76.00|||0.00|21.95||40.29|61.45|62.24|-36.50| +2450822|62149|2450886|46107|1261866|3672|631|46107|1261866|3672|631|2|4|4|2|15577|80|399|44|76.60|139.41|89.22|2208.36|3925.68|3370.40|6134.04|0.00|0.00|122.32|3925.68|3925.68|4048.00|4048.00|555.28| +2450822|62149|2450870|46107|1261866|3672|631|46107|1261866|3672|631|2|45|4|1|2167|69|399|32|57.88|169.58|108.53|1953.60|3472.96|1852.16|5426.56|104.18|0.00|868.16|3472.96|3577.14|4341.12|4445.30|1620.80| +2450822|62149|2450858|46107|1261866|3672|631|46107|1261866|3672|631|2|52|19|3|12115|121|399|85|96.22|178.96|16.10|13843.10|1368.50|8178.70|15211.60|95.79|0.00|4714.95|1368.50|1464.29|6083.45|6179.24|-6810.20| +2450822|45360|2450906|77033|688403|2173|11143|77033|688403|2173|11143|2|53|9|3|1036|188|400|99|7.66|13.32|7.05|620.73|697.95|758.34|1318.68|55.83|0.00|342.54|697.95|753.78|1040.49|1096.32|-60.39| +2450822|45360|2450900|77033|688403|2173|11143|77033|688403|2173|11143|2|20|20|4|4406|269|400|84|44.17|118.81|103.36|1297.80|8682.24|3710.28|9980.04|694.57|0.00|3492.72|8682.24|9376.81|12174.96|12869.53|4971.96| +2450822|45360|2450874|77033|688403|2173|11143|77033|688403|2173|11143|2|99|20|3|12758|186|400|42|62.12|173.31|142.11|1310.40|5968.62|2609.04|7279.02|0.00|2208.38|2256.24|3760.24|3760.24|6016.48|6016.48|1151.20| +2450822|45360|2450910|77033|688403|2173|11143|77033|688403|2173|11143|2|43|16|3|13723|285|400|49|6.57|17.34|1.04|798.70|50.96|321.93|849.66|1.01|0.00|93.10|50.96|51.97|144.06|145.07|-270.97| +2450822|45360|2450893|77033|688403|2173|11143|77033|688403|2173|11143|2|81|12|2|10669|86|400|7|47.37|113.21|50.94|435.89|356.58|331.59|792.47|28.52|0.00|166.39|356.58|385.10|522.97|551.49|24.99| +2450822|45360|2450828|77033|688403|2173|11143|77033|688403|2173|11143|2|21|12|2|10058|148|400|6|65.42|167.47|145.69|130.68|874.14|392.52|1004.82|14.16|638.12|291.36|236.02|250.18|527.38|541.54|-156.50| +2450822|45360|2450840|77033|688403|2173|11143|77033|688403|2173|11143|2|104|16|4|1024|170|400|70|86.86|105.10|57.80|3311.00|4046.00|6080.20|7357.00|121.38|0.00|2427.60|4046.00|4167.38|6473.60|6594.98|-2034.20| +2450822|25555|2450894|34528|1919328|1728|20928|34528|1919328|1728|20928|4|84|19|3|692|228|401|80|24.83|39.47|13.41|2084.80|1072.80|1986.40|3157.60|10.72|0.00|536.00|1072.80|1083.52|1608.80|1619.52|-913.60| +2450822|25555|2450874|34528|1919328|1728|20928|34528|1919328|1728|20928|4|5|7|4|13024|40|401|43|65.14|96.40|60.73|1533.81|2611.39|2801.02|4145.20|52.22|0.00|1658.08|2611.39|2663.61|4269.47|4321.69|-189.63| +2450822|25555|2450911|34528|1919328|1728|20928|34528|1919328|1728|20928|4|38|2|1|9758|17|401|76|72.06|200.32|138.22|4719.60|10504.72|5476.56|15224.32|525.23|0.00|2283.04|10504.72|11029.95|12787.76|13312.99|5028.16| +2450822|25555|2450824|34528|1919328|1728|20928|34528|1919328|1728|20928|4|92|9|4|304|229|401|73|69.46|126.41|83.43|3137.54|6090.39|5070.58|9227.93|243.61|0.00|1476.06|6090.39|6334.00|7566.45|7810.06|1019.81| +2450822|25555|2450909|34528|1919328|1728|20928|34528|1919328|1728|20928|4|62|1|3|1768|202|401|68|3.97|9.76|8.00|119.68|544.00|269.96|663.68|7.83|152.32|79.56|391.68|399.51|471.24|479.07|121.72| +2450822|25555|2450912|34528|1919328|1728|20928|34528|1919328|1728|20928|4|106|10|5|4864|230|401|35|22.48|31.92|6.70|882.70|234.50|786.80|1117.20|4.59|168.84|111.65|65.66|70.25|177.31|181.90|-721.14| +2450822|25555|2450910|34528|1919328|1728|20928|34528|1919328|1728|20928|4|73|10|4|6584|161|401|37|12.99|38.58|37.42|42.92|1384.54|480.63|1427.46|55.38|0.00|371.11|1384.54|1439.92|1755.65|1811.03|903.91| +2450822|25555|2450907|34528|1919328|1728|20928|34528|1919328|1728|20928|4|83|12|2|15400|210|401|9|86.25|113.85|26.18|789.03|235.62|776.25|1024.65|14.13|0.00|368.82|235.62|249.75|604.44|618.57|-540.63| +|||||||||1728|20928|4|19|8||16|33|401|90|47.63|||7808.40|||8873.10|||3548.70||631.15|4155.58|4179.85|-3679.82| +2450822|25555|2450890|34528|1919328|1728|20928|34528|1919328|1728|20928|4|51|3|4|17666|16|401|75|85.27|165.42|115.79|3722.25|8684.25|6395.25|12406.50|554.92|2518.43|5210.25|6165.82|6720.74|11376.07|11930.99|-229.43| +2450822|25555|2450897|34528|1919328|1728|20928|34528|1919328|1728|20928|4|81|14|4|17401|206|401|16|39.10|70.77|14.86|894.56|237.76|625.60|1132.32|4.75|0.00|328.32|237.76|242.51|566.08|570.83|-387.84| +2450822|25555|2450849|34528|1919328|1728|20928|34528|1919328|1728|20928|4|55|13|5|16267|137|401|60|5.60|8.12|4.54|214.80|272.40|336.00|487.20|19.61|54.48|243.60|217.92|237.53|461.52|481.13|-118.08| +2450822|25555|2450856|34528|1919328|1728|20928|34528|1919328|1728|20928|4|74|13|2|11059|10|401|6|61.74|135.21|123.04|73.02|738.24|370.44|811.26|66.44|0.00|292.02|738.24|804.68|1030.26|1096.70|367.80| +2450822|25555|2450909|34528|1919328|1728|20928|34528|1919328|1728|20928|4|58|10|2|5894|18|401|61|65.34|119.57|110.00|583.77|6710.00|3985.74|7293.77|603.90|0.00|2188.07|6710.00|7313.90|8898.07|9501.97|2724.26| +2450822|79303|2450879|93265|32325|2585|4845|93265|32325|2585|4845|1|52|18|5|1232|74|402|73|90.78|135.26|108.20|1975.38|7898.60|6626.94|9873.98|0.00|947.83|3356.54|6950.77|6950.77|10307.31|10307.31|323.83| +2450822|79303|2450863|93265|32325|2585|4845|93265|32325|2585|4845|1|21|4|3|1582|269|402|29|63.80|66.35|51.75|423.40|1500.75|1850.20|1924.15|0.00|1080.54|403.97|420.21|420.21|824.18|824.18|-1429.99| +2450822|79303|2450880|93265|32325|2585|4845|93265|32325|2585|4845|1|71|19|4|1279|47|402|17|54.41|112.62|13.51|1684.87|229.67|924.97|1914.54|4.59|0.00|536.01|229.67|234.26|765.68|770.27|-695.30| +2450822|79303|2450885|93265|32325|2585|4845|93265|32325|2585|4845|1|20|7|1|8425|220|402|10|66.22|118.53|114.97|35.60|1149.70|662.20|1185.30|34.49|0.00|509.60|1149.70|1184.19|1659.30|1693.79|487.50| +2450822|79303|2450895|93265|32325|2585|4845|93265|32325|2585|4845|1|78|3|1|9410|62|402|65|79.76|79.76|40.67|2540.85|2643.55|5184.40|5184.40|185.04|0.00|1296.10|2643.55|2828.59|3939.65|4124.69|-2540.85| +2450822|58313|2450843|6453|1847130|3807|43284|6453|1847130|3807|43284|1|97|5|5|8312|205|403|22|28.92|64.49|39.98|539.22|879.56|636.24|1418.78|39.58|439.78|368.72|439.78|479.36|808.50|848.08|-196.46| +2450822|58313|2450840|6453|1847130|3807|43284|6453|1847130|3807|43284|1|77|16|3|1180|282|403|38|74.78|79.26|5.54|2801.36|210.52|2841.64|3011.88|16.84|0.00|1385.10|210.52|227.36|1595.62|1612.46|-2631.12| +2450822|58313|2450850|6453|1847130|3807|43284|6453|1847130|3807|43284|1|24|14|1|15454|243|403|4|17.20|32.33|20.04|49.16|80.16|68.80|129.32|3.11|2.40|2.56|77.76|80.87|80.32|83.43|8.96| +2450822|58313|2450876|6453|1847130|3807|43284|6453|1847130|3807|43284|1|42|10|5|4213|131|403|39|75.92|100.21|86.18|547.17|3361.02|2960.88|3908.19|134.44|0.00|390.78|3361.02|3495.46|3751.80|3886.24|400.14| +2450822|58313|2450880|6453|1847130|3807|43284|6453|1847130|3807|43284|1|27|8|2|3934|200|403|47|25.97|68.56|4.11|3029.15|193.17|1220.59|3222.32|7.72|0.00|1191.92|193.17|200.89|1385.09|1392.81|-1027.42| +2450822|58313|2450857|6453|1847130|3807|43284|6453|1847130|3807|43284|1|43|16|3|16558|192|403|3|59.31|94.89|80.65|42.72|241.95|177.93|284.67|0.00|0.00|65.46|241.95|241.95|307.41|307.41|64.02| +2450822|58313|2450863|6453|1847130|3807|43284|6453|1847130|3807|43284|1|98|7|4|16651|135|403|7|38.10|60.19|36.11|168.56|252.77|266.70|421.33|12.63|0.00|0.00|252.77|265.40|252.77|265.40|-13.93| +2450822|58313|2450894|6453|1847130|3807|43284|6453|1847130|3807|43284|1|17|13|2|8696|268|403|51|44.81|48.39|20.32|1431.57|1036.32|2285.31|2467.89|93.26|0.00|1209.21|1036.32|1129.58|2245.53|2338.79|-1248.99| +2450822|58313|2450833|6453|1847130|3807|43284|6453|1847130|3807|43284|1|10|4|1|15478|155|403|94|1.26|3.56|0.89|250.98|83.66|118.44|334.64|1.67|0.00|43.24|83.66|85.33|126.90|128.57|-34.78| +2450822|76346|2450886|63192|1451183|7174|11165|63192|1451183|7174|11165|2|52|10|5|17011|190|404|40|60.76|80.81|58.18|905.20|2327.20|2430.40|3232.40|69.81|0.00|64.40|2327.20|2397.01|2391.60|2461.41|-103.20| +2450822|76346|2450828|63192|1451183|7174|11165|63192|1451183|7174|11165|2|107|3|3|1390|181|404|57|36.58|65.84|52.01|788.31|2964.57|2085.06|3752.88|148.22|0.00|0.00|2964.57|3112.79|2964.57|3112.79|879.51| +2450822|76346|2450854|63192|1451183|7174|11165|63192|1451183|7174|11165|2|37|18|2|14929|127|404|91|15.52|40.04|27.62|1130.22|2513.42|1412.32|3643.64|150.80|0.00|1821.82|2513.42|2664.22|4335.24|4486.04|1101.10| +2450822|76346|2450866|63192|1451183|7174|11165|63192|1451183|7174|11165|2|7|12|2|17132|80|404|68|50.17|86.79|25.16|4190.84|1710.88|3411.56|5901.72|34.21|0.00|2891.36|1710.88|1745.09|4602.24|4636.45|-1700.68| +2450822|76346|2450844|63192|1451183|7174|11165|63192|1451183|7174|11165|2|73|19|3|8509|186|404|63|5.13|8.77|6.48|144.27|408.24|323.19|552.51|4.08|0.00|54.81|408.24|412.32|463.05|467.13|85.05| +2450822|76346|2450876|63192|1451183|7174|11165|63192|1451183|7174|11165|2|108|1|2|11530|10|404|14|84.71|182.97|106.12|1075.90|1485.68|1185.94|2561.58|59.42|0.00|1152.62|1485.68|1545.10|2638.30|2697.72|299.74| +2450822|77696|2450881|26206|1602497|1541|39832|55311|392367|3411|18902|2|26|12|2|964|132|405|4|61.87|175.09|24.51|602.32|98.04|247.48|700.36|1.96|0.00|203.08|98.04|100.00|301.12|303.08|-149.44| +2450822|77696|2450868|26206|1602497|1541|39832|55311|392367|3411|18902|2|52|4|2|754|196|405|50|88.39|131.70|59.26|3622.00|2963.00|4419.50|6585.00|177.78|0.00|2173.00|2963.00|3140.78|5136.00|5313.78|-1456.50| +2450822|77696|2450896|26206|1602497|1541|39832|55311|392367|3411|18902|2|13|9|2|10202|214|405|5|2.32|4.03|0.20|19.15|1.00|11.60|20.15|0.08|0.00|3.20|1.00|1.08|4.20|4.28|-10.60| +2450822|77696|2450894|26206|1602497|1541|39832|55311|392367|3411|18902|2|74|15|4|3794|109|405|8|82.29|209.01|127.49|652.16|1019.92|658.32|1672.08|0.00|0.00|334.40|1019.92|1019.92|1354.32|1354.32|361.60| +2450822|77696|2450900|26206|1602497|1541|39832|55311|392367|3411|18902|2|23|14|5|1126|199|405|73|90.50|162.90|40.72|8919.14|2972.56|6606.50|11891.70|0.00|0.00|2021.37|2972.56|2972.56|4993.93|4993.93|-3633.94| +2450822|77696|2450899|26206|1602497|1541|39832|55311|392367|3411|18902|2|27|17|4|13202|196|405|28|57.07|84.46|43.07|1158.92|1205.96|1597.96|2364.88|48.23|0.00|1087.80|1205.96|1254.19|2293.76|2341.99|-392.00| +2450822|77696|2450900|26206|1602497|1541|39832|55311|392367|3411|18902|2|37|7|2|13174|265|405|74|83.60|178.06|117.51|4480.70|8695.74|6186.40|13176.44|782.61|0.00|5533.72|8695.74|9478.35|14229.46|15012.07|2509.34| +2450822|77696|2450862|26206|1602497|1541|39832|55311|392367|3411|18902|2|43|8|3|8932|116|405|56|7.65|16.52|5.12|638.40|286.72|428.40|925.12|5.59|100.35|351.12|186.37|191.96|537.49|543.08|-242.03| +2450822|77696|2450880|26206|1602497|1541|39832|55311|392367|3411|18902|2|8|12|4|3452|298|405|93|77.49|136.38|36.82|9259.08|3424.26|7206.57|12683.34|273.94|0.00|633.33|3424.26|3698.20|4057.59|4331.53|-3782.31| +2450822|61598|2450880|11433|506447|5461|20428|11433|506447|5461|20428|1|72|19|1|9004|123|406|93|61.12|73.34|46.20|2524.02|4296.60|5684.16|6820.62|70.89|1933.47|1567.98|2363.13|2434.02|3931.11|4002.00|-3321.03| +2450822|61598|2450868|11433|506447|5461|20428|11433|506447|5461|20428|1|108|5|1|12884|118|406|50|99.48|292.47|254.44|1901.50|12722.00|4974.00|14623.50|254.44|0.00|5118.00|12722.00|12976.44|17840.00|18094.44|7748.00| +2450822|61598|2450836|11433|506447|5461|20428|11433|506447|5461|20428|1|71|17|5|14035|245|406|74|97.22|267.35|120.30|10881.70|8902.20|7194.28|19783.90|801.19|0.00|7517.66|8902.20|9703.39|16419.86|17221.05|1707.92| +2450822|61598|2450856|11433|506447|5461|20428|11433|506447|5461|20428|1|22|4|5|17792|149|406|75|29.51|46.33|10.19|2710.50|764.25|2213.25|3474.75|30.57|0.00|660.00|764.25|794.82|1424.25|1454.82|-1449.00| +2450822|61598|2450832|11433|506447|5461|20428|11433|506447|5461|20428|1|3|13|3|14758|283|406|46|49.70|56.65|43.05|625.60|1980.30|2286.20|2605.90|158.42|0.00|1120.10|1980.30|2138.72|3100.40|3258.82|-305.90| +2450822|61598|2450833|11433|506447|5461|20428|11433|506447|5461|20428|1|99|19|4|17389|297|406|14|23.60|64.42|44.44|279.72|622.16|330.40|901.88|43.55|0.00|27.02|622.16|665.71|649.18|692.73|291.76| +2450822|61598|2450829|11433|506447|5461|20428|11433|506447|5461|20428|1|50|17|4|1490|193|406|72|44.17|56.53|22.04|2483.28|1586.88|3180.24|4070.16|15.86|0.00|854.64|1586.88|1602.74|2441.52|2457.38|-1593.36| +2450822|61598|2450882|11433|506447|5461|20428|11433|506447|5461|20428|1|87|7|3|3410|152|406|97|8.14|13.67|6.83|663.48|662.51|789.58|1325.99|46.37|0.00|582.97|662.51|708.88|1245.48|1291.85|-127.07| +2450822|61598|2450879|11433|506447|5461|20428|11433|506447|5461|20428|1|48|5|1|3520|103|406|54|37.90|89.06|43.63|2453.22|2356.02|2046.60|4809.24|94.24|0.00|624.78|2356.02|2450.26|2980.80|3075.04|309.42| +2450822|61598|2450905|11433|506447|5461|20428|11433|506447|5461|20428|1|44|20|2|2527|65|406|43|66.95|77.66|53.58|1035.44|2303.94|2878.85|3339.38|134.78|806.37|1636.15|1497.57|1632.35|3133.72|3268.50|-1381.28| +2450822|61598|2450833|11433|506447|5461|20428|11433|506447|5461|20428|1|93|8|4|6346|278|406|85|79.27|111.77|58.12|4560.25|4940.20|6737.95|9500.45|247.01|0.00|3989.90|4940.20|5187.21|8930.10|9177.11|-1797.75| +2450822|61598|2450909|11433|506447|5461|20428|11433|506447|5461|20428|1|75|9|1|3092|4|406|71|4.17|9.50|6.55|209.45|465.05|296.07|674.50|9.06|162.76|323.76|302.29|311.35|626.05|635.11|6.22| +2450822|74100|2450897|21653|1301444|1682|1120|35383|585149|587|1625|4|18|8|1|4786|104|407|64|87.76|261.52|91.53|10879.36|5857.92|5616.64|16737.28|527.21|0.00|1338.88|5857.92|6385.13|7196.80|7724.01|241.28| +2450822|74100|2450834|21653|1301444|1682|1120|35383|585149|587|1625|4|106|15|4|10867|200|407|92|68.05|146.98|102.88|4057.20|9464.96|6260.60|13522.16|283.94|0.00|6761.08|9464.96|9748.90|16226.04|16509.98|3204.36| +2450822|74100|2450874|21653|1301444|1682|1120|35383|585149|587|1625|4|78|2|2|9398|292|407|68|95.06|164.45|74.00|6150.60|5032.00|6464.08|11182.60|201.28|0.00|4584.56|5032.00|5233.28|9616.56|9817.84|-1432.08| +2450822|74100|2450865|21653|1301444|1682|1120|35383|585149|587|1625|4|3|17|3|10132|298|407|94|13.66|15.16|3.48|1097.92|327.12|1284.04|1425.04|29.44|0.00|270.72|327.12|356.56|597.84|627.28|-956.92| +2450822|74100|2450848|21653|1301444|1682|1120|35383|585149|587|1625|4|92|4|2|9646|204|407|100|94.11|110.10|78.17|3193.00|7817.00|9411.00|11010.00|312.68|0.00|4954.00|7817.00|8129.68|12771.00|13083.68|-1594.00| +2450822|74100|2450848|21653|1301444|1682|1120|35383|585149|587|1625|4|104|11|3|13448|227|407|70|27.95|39.40|13.79|1792.70|965.30|1956.50|2758.00|57.91|0.00|551.60|965.30|1023.21|1516.90|1574.81|-991.20| +2450822|74100|2450833|21653|1301444|1682|1120|35383|585149|587|1625|4|63|20|2|13972|168|407|64|25.04|36.30|23.23|836.48|1486.72|1602.56|2323.20|0.00|0.00|1045.12|1486.72|1486.72|2531.84|2531.84|-115.84| +2450822|74441|2450875|17284|862922|4113|35304|17284|862922|4113|35304|4|5|2|3|6428|267|408|23|72.44|89.82|35.92|1239.70|826.16|1666.12|2065.86|66.09|0.00|495.65|826.16|892.25|1321.81|1387.90|-839.96| +2450822|74441|2450867|17284|862922|4113|35304|17284|862922|4113|35304|4|49|19|3|10196|166|408|75|94.12|102.59|13.33|6694.50|999.75|7059.00|7694.25|69.98|0.00|923.25|999.75|1069.73|1923.00|1992.98|-6059.25| +2450822|74441|2450838|17284|862922|4113|35304|17284|862922|4113|35304|4|108|5|4|17065|261|408|57|56.59|76.96|7.69|3948.39|438.33|3225.63|4386.72|3.33|271.76|1930.02|166.57|169.90|2096.59|2099.92|-3059.06| +2450822|74441|2450911|17284|862922|4113|35304|17284|862922|4113|35304|4|44|10|4|9878|207|408|27|39.54|40.33|8.06|871.29|217.62|1067.58|1088.91|15.23|0.00|10.80|217.62|232.85|228.42|243.65|-849.96| +2450822|74441|2450882|17284|862922|4113|35304|17284|862922|4113|35304|4|107|9|3|7136|104|408|100|66.39|136.09|20.41|11568.00|2041.00|6639.00|13609.00|163.28|0.00|3130.00|2041.00|2204.28|5171.00|5334.28|-4598.00| +2450822|74441|2450848|17284|862922|4113|35304|17284|862922|4113|35304|4|46|12|2|12394|150|408|81|17.07|34.31|3.43|2501.28|277.83|1382.67|2779.11|0.00|0.00|332.91|277.83|277.83|610.74|610.74|-1104.84| +2450822|74441|2450842|17284|862922|4113|35304|17284|862922|4113|35304|4|41|15|1|1904|271|408|78|46.23|127.59|57.41|5474.04|4477.98|3605.94|9952.02|304.50|671.69|3383.64|3806.29|4110.79|7189.93|7494.43|200.35| +2450822|74441|2450833|17284|862922|4113|35304|17284|862922|4113|35304|4|13|20|2|1088|101|408|1|15.01|20.41|17.55|2.86|17.55|15.01|20.41|0.35|0.00|0.61|17.55|17.90|18.16|18.51|2.54| +2450822|74441|2450901|17284|862922|4113|35304|17284|862922|4113|35304|4|33|4|4|9055|240|408|21|66.67|181.34|152.32|609.42|3198.72|1400.07|3808.14|95.96|0.00|304.50|3198.72|3294.68|3503.22|3599.18|1798.65| +2450822|74441|2450849|17284|862922|4113|35304|17284|862922|4113|35304|4|35|6|2|8338|154|408|63|37.19|106.36|104.23|134.19|6566.49|2342.97|6700.68|0.00|0.00|2613.24|6566.49|6566.49|9179.73|9179.73|4223.52| +|74441|2450876|||4113|35304|17284|||35304|||12|2|15424|139|408|||24.93||||908.46|2443.14|71.79|||||1709.12|1780.91|288.12| +2450822|74441|2450832|17284|862922|4113|35304|17284|862922|4113|35304|4|64|5|4|17071|27|408|34|77.21|78.75|45.67|1124.72|1552.78|2625.14|2677.50|67.08|807.44|1311.72|745.34|812.42|2057.06|2124.14|-1879.80| +2450822|74441|2450851|17284|862922|4113|35304|17284|862922|4113|35304|4|91|18|1|13243|12|408|37|50.89|122.64|8.58|4220.22|317.46|1882.93|4537.68|15.87|0.00|634.92|317.46|333.33|952.38|968.25|-1565.47| +2450822|73221|2450877|92243|1725488|5133|35578|92243|1725488|5133|35578|1|104|4|4|3566|234|409|20|82.57|150.27|141.25|180.40|2825.00|1651.40|3005.40|35.03|1073.50|1262.20|1751.50|1786.53|3013.70|3048.73|100.10| +2450822|73221|2450868|92243|1725488|5133|35578|92243|1725488|5133|35578|1|88|3|1|17941|233|409|9|60.08|129.17|43.91|767.34|395.19|540.72|1162.53|27.66|0.00|581.22|395.19|422.85|976.41|1004.07|-145.53| +2450822|73221|2450835|92243|1725488|5133|35578|92243|1725488|5133|35578|1|53|8|5|6613|290|409|43|20.81|52.85|26.42|1136.49|1136.06|894.83|2272.55|45.44|0.00|590.82|1136.06|1181.50|1726.88|1772.32|241.23| +2450822|73221|2450858|92243|1725488|5133|35578|92243|1725488|5133|35578|1|11|3|3|2545|298|409|83|13.70|23.83|13.10|890.59|1087.30|1137.10|1977.89|53.06|424.04|375.16|663.26|716.32|1038.42|1091.48|-473.84| +2450822|73221|2450897|92243|1725488|5133|35578|92243|1725488|5133|35578|1|31|18|3|458|119|409|17|3.83|8.19|1.63|111.52|27.71|65.11|139.23|0.55|0.00|36.04|27.71|28.26|63.75|64.30|-37.40| +2450822|73221|2450852|92243|1725488|5133|35578|92243|1725488|5133|35578|1|57|14|3|17282|109|409|76|95.21|285.63|14.28|20622.60|1085.28|7235.96|21707.88|10.85|0.00|10419.60|1085.28|1096.13|11504.88|11515.73|-6150.68| +2450822|73221|2450856|92243|1725488|5133|35578|92243|1725488|5133|35578|1|77|9|1|2314|40|409|79|77.43|220.67|211.84|697.57|16735.36|6116.97|17432.93|1338.82|0.00|1219.76|16735.36|18074.18|17955.12|19293.94|10618.39| +2450822|73221|2450862|92243|1725488|5133|35578|92243|1725488|5133|35578|1|42|9|5|7624|18|409|93|91.21|186.98|115.92|6608.58|10780.56|8482.53|17389.14|107.80|0.00|0.00|10780.56|10888.36|10780.56|10888.36|2298.03| +2450822|73221|2450849|92243|1725488|5133|35578|92243|1725488|5133|35578|1|70|17|3|3247|1|409|58|96.68|125.68|123.16|146.16|7143.28|5607.44|7289.44|65.71|5500.32|728.48|1642.96|1708.67|2371.44|2437.15|-3964.48| +2450822|73221|2450889|92243|1725488|5133|35578|92243|1725488|5133|35578|1|18|5|5|8776|47|409|69|3.34|3.64|0.87|191.13|60.03|230.46|251.16|3.60|0.00|125.58|60.03|63.63|185.61|189.21|-170.43| +2450822|73221|2450877|92243|1725488|5133|35578|92243|1725488|5133|35578|1|74|9|3|14521|198|409|6|53.06|150.15|63.06|522.54|378.36|318.36|900.90|18.91|0.00|0.00|378.36|397.27|378.36|397.27|60.00| +2450822|73221|2450912|92243|1725488|5133|35578|92243|1725488|5133|35578|1|34|4|1|11228|12|409|39|85.09|117.42|4.69|4396.47|182.91|3318.51|4579.38|1.82|0.00|366.21|182.91|184.73|549.12|550.94|-3135.60| +2450823|35287|2450895|47965|1065872|2532|39|47965|1065872|2532|39|1|89|17|1|5263|46|410|70|18.26|29.94|5.68|1698.20|397.60|1278.20|2095.80|35.78|0.00|460.60|397.60|433.38|858.20|893.98|-880.60| +2450823|35287|2450886|47965|1065872|2532|39|47965|1065872|2532|39|1|32|19|2|3826|98|410|33|60.44|80.38|54.65|849.09|1803.45|1994.52|2652.54|18.03|0.00|1060.95|1803.45|1821.48|2864.40|2882.43|-191.07| +2450823|35287|2450854|47965|1065872|2532|39|47965|1065872|2532|39|1|3|17|3|8827|152|410|38|95.20|175.16|117.35|2196.78|4459.30|3617.60|6656.08|133.77|0.00|1264.64|4459.30|4593.07|5723.94|5857.71|841.70| +2450823|35287|2450909|47965|1065872|2532|39|47965|1065872|2532|39|1|5|9|3|1376|285|410|7|47.28|111.10|35.55|528.85|248.85|330.96|777.70|22.39|0.00|373.24|248.85|271.24|622.09|644.48|-82.11| +2450823|35287|2450901|47965|1065872|2532|39|47965|1065872|2532|39|1|31|8|1|16510|34|410|96|8.27|22.49|4.72|1705.92|453.12|793.92|2159.04|31.71|0.00|971.52|453.12|484.83|1424.64|1456.35|-340.80| +2450823|35287|2450856|47965|1065872|2532|39|47965|1065872|2532|39|1|68|16|4|16849|30|410|87|46.81|100.17|65.11|3050.22|5664.57|4072.47|8714.79|283.22|0.00|3921.09|5664.57|5947.79|9585.66|9868.88|1592.10| +2450823|35287|2450855|47965|1065872|2532|39|47965|1065872|2532|39|1|80|4|3|4615|116|410|32|35.35|70.70|4.24|2126.72|135.68|1131.20|2262.40|9.49|0.00|1063.04|135.68|145.17|1198.72|1208.21|-995.52| +2450823|35287|2450906|47965|1065872|2532|39|47965|1065872|2532|39|1|24|18|5|7861|281|410|75|60.66|137.69|31.66|7952.25|2374.50|4549.50|10326.75|166.21|0.00|3614.25|2374.50|2540.71|5988.75|6154.96|-2175.00| +2450823|35287|2450848|47965|1065872|2532|39|47965|1065872|2532|39|1|48|16|4|6038|110|410|95|57.40|68.88|59.23|916.75|5626.85|5453.00|6543.60|153.61|3432.37|2420.60|2194.48|2348.09|4615.08|4768.69|-3258.52| +2450823|35287|2450825|47965|1065872|2532|39|47965|1065872|2532|39|1|49|2|5|988|24|410|86|25.88|68.58|29.48|3362.60|2535.28|2225.68|5897.88|152.11|0.00|1474.04|2535.28|2687.39|4009.32|4161.43|309.60| +2450823|35287|2450872|47965|1065872|2532|39|47965|1065872|2532|39|1|64|5|2|5038|259|410|29|25.98|69.36|36.06|965.70|1045.74|753.42|2011.44|14.43|324.17|864.78|721.57|736.00|1586.35|1600.78|-31.85| +2450823|35287|2450872|47965|1065872|2532|39|47965|1065872|2532|39|1|95|7|5|12374|8|410|54|65.79|133.55|89.47|2380.32|4831.38|3552.66|7211.70|434.82|0.00|3533.22|4831.38|5266.20|8364.60|8799.42|1278.72| +2450823|35287|2450857|47965|1065872|2532|39|47965|1065872|2532|39|1|67|7|2|1814|260|410|17|44.36|129.08|0.00|2194.36|0.00|754.12|2194.36|0.00|0.00|877.71|0.00|0.00|877.71|877.71|-754.12| +2450823|35287|2450856|47965|1065872|2532|39|47965|1065872|2532|39|1|26|12|4|9830|295|410|17|70.19|99.66|55.80|745.62|948.60|1193.23|1694.22|2.56|863.22|694.62|85.38|87.94|780.00|782.56|-1107.85| +2450823|30587|2450854|47856|1909175|1247|31766|47856|1909175|1247|31766|2|103|4|1|17695|62|411|38|13.01|15.22|10.80|167.96|410.40|494.38|578.36|8.20|0.00|28.88|410.40|418.60|439.28|447.48|-83.98| +2450823|30587|2450851|47856|1909175|1247|31766|47856|1909175|1247|31766|2|11|13|5|16216|197|411|67|70.92|157.44|37.78|8017.22|2531.26|4751.64|10548.48|25.31|0.00|210.38|2531.26|2556.57|2741.64|2766.95|-2220.38| +2450823|30587|2450836|47856|1909175|1247|31766|47856|1909175|1247|31766|2|3|12|3|2701|167|411|7|82.60|121.42|41.28|560.98|288.96|578.20|849.94|5.34|182.04|271.95|106.92|112.26|378.87|384.21|-471.28| +2450823|30587|2450845|47856|1909175|1247|31766|47856|1909175|1247|31766|2|20|11|1|10972|181|411|83|16.71|32.91|14.15|1557.08|1174.45|1386.93|2731.53|46.97|0.00|901.38|1174.45|1221.42|2075.83|2122.80|-212.48| +2450823|30587|2450903|47856|1909175|1247|31766|47856|1909175|1247|31766|2|93|11|5|3064|276|411|30|18.78|33.80|21.63|365.10|648.90|563.40|1014.00|12.97|0.00|283.80|648.90|661.87|932.70|945.67|85.50| +2450823|30587|2450840|47856|1909175|1247|31766|47856|1909175|1247|31766|2|20|12|5|3188|137|411|67|12.91|33.82|7.77|1745.35|520.59|864.97|2265.94|14.52|36.44|1064.63|484.15|498.67|1548.78|1563.30|-380.82| +2450823|30587|2450850|47856|1909175|1247|31766|47856|1909175|1247|31766|2|50|1|1|6664|236|411|20|91.55|253.59|215.55|760.80|4311.00|1831.00|5071.80|243.14|258.66|2535.80|4052.34|4295.48|6588.14|6831.28|2221.34| +2450823|30587|2450859|47856|1909175|1247|31766|47856|1909175|1247|31766|2|12|3|5|17288|197|411|35|7.45|12.21|11.11|38.50|388.85|260.75|427.35|3.88|0.00|21.35|388.85|392.73|410.20|414.08|128.10| +2450823|30587|2450837|47856|1909175|1247|31766|47856|1909175|1247|31766|2|42|6|3|15812|112|411|18|34.06|34.06|2.38|570.24|42.84|613.08|613.08|3.42|0.00|128.70|42.84|46.26|171.54|174.96|-570.24| +2450823|30587|2450825|47856|1909175|1247|31766|47856|1909175|1247|31766|2|70|1|3|284|120|411|11|84.26|244.35|39.09|2257.86|429.99|926.86|2687.85|25.79|0.00|725.67|429.99|455.78|1155.66|1181.45|-496.87| +2450823|30587|2450866|47856|1909175|1247|31766|47856|1909175|1247|31766|2|95|9|4|4838|239|411|41|81.21|204.64|204.64|0.00|8390.24|3329.61|8390.24|167.80|0.00|1258.29|8390.24|8558.04|9648.53|9816.33|5060.63| +2450823|30587|2450892|47856|1909175|1247|31766|47856|1909175|1247|31766|2|17|11|2|5774|196|411|19|79.63|191.90|36.46|2953.36|692.74|1512.97|3646.10|34.63|0.00|1312.52|692.74|727.37|2005.26|2039.89|-820.23| +2450823|30587|2450878|47856|1909175|1247|31766|47856|1909175|1247|31766|2|32|10|1|17395|68|411|25|12.24|24.96|11.98|324.50|299.50|306.00|624.00|11.98|0.00|93.50|299.50|311.48|393.00|404.98|-6.50| +2450823|30587|2450857|47856|1909175|1247|31766|47856|1909175|1247|31766|2|65|20|5|9853|82|411|92|86.25|108.67|31.51|7098.72|2898.92|7935.00|9997.64|217.99|173.93|1199.68|2724.99|2942.98|3924.67|4142.66|-5210.01| +2450823|55187|2450837|35806|1598934|2509|29684|35806|1598934|2509|29684|2|58|14|1|6622|267|412|93|47.12|101.77|94.64|663.09|8801.52|4382.16|9464.61|704.12|0.00|2933.22|8801.52|9505.64|11734.74|12438.86|4419.36| +2450823|55187|2450872|35806|1598934|2509|29684|35806|1598934|2509|29684|2|54|1|4|17701|75|412|76|56.20|60.69|20.63|3044.56|1567.88|4271.20|4612.44|94.07|0.00|322.24|1567.88|1661.95|1890.12|1984.19|-2703.32| +2450823|55187|2450888|35806|1598934|2509|29684|35806|1598934|2509|29684|2|4|16|3|10951|285|412|54|39.09|66.06|27.08|2104.92|1462.32|2110.86|3567.24|95.19|102.36|1533.60|1359.96|1455.15|2893.56|2988.75|-750.90| +2450823|55187|2450853|35806|1598934|2509|29684|35806|1598934|2509|29684|2|28|9|4|11125|40|412|28|93.66|279.10|64.19|6017.48|1797.32|2622.48|7814.80|67.39|449.33|1093.96|1347.99|1415.38|2441.95|2509.34|-1274.49| +2450823|55187|2450846|35806|1598934|2509|29684|35806|1598934|2509|29684|2|88|9|4|6025|134|412|4|1.23|1.91|1.69|0.88|6.76|4.92|7.64|0.40|0.00|0.96|6.76|7.16|7.72|8.12|1.84| +2450823|55187|2450874|35806|1598934|2509|29684|35806|1598934|2509|29684|2|26|3|2|550|223|412|21|37.82|76.39|49.65|561.54|1042.65|794.22|1604.19|62.55|0.00|513.24|1042.65|1105.20|1555.89|1618.44|248.43| +2450823|55187|2450862|35806|1598934|2509|29684|35806|1598934|2509|29684|2|68|1|5|12806|13|412|37|27.65|31.24|28.11|115.81|1040.07|1023.05|1155.88|72.80|0.00|57.72|1040.07|1112.87|1097.79|1170.59|17.02| +2450823|55187|2450831|35806|1598934|2509|29684|35806|1598934|2509|29684|2|24|14|3|4376|182|412|94|82.52|141.93|93.67|4536.44|8804.98|7756.88|13341.42|704.39|0.00|6136.32|8804.98|9509.37|14941.30|15645.69|1048.10| +2450823|55187|2450906|35806|1598934|2509|29684|35806|1598934|2509|29684|2|24|11|3|15512|190|412|81|94.58|232.66|183.80|3957.66|14887.80|7660.98|18845.46|595.51|0.00|4145.58|14887.80|15483.31|19033.38|19628.89|7226.82| +2450823|55187|2450838|35806|1598934|2509|29684|35806|1598934|2509|29684|2|76|13|5|1142|116|412|73|18.48|36.22|7.24|2115.54|528.52|1349.04|2644.06|36.99|0.00|608.09|528.52|565.51|1136.61|1173.60|-820.52| +2450823|55187|2450866|35806|1598934|2509|29684|35806|1598934|2509|29684|2|35|11|2|11116|55|412|9|12.50|17.75|17.21|4.86|154.89|112.50|159.75|6.19|0.00|63.90|154.89|161.08|218.79|224.98|42.39| +2450823|55187|2450875|35806|1598934|2509|29684|35806|1598934|2509|29684|2|16|19|1|308|218|412|93|63.30|87.35|63.76|2193.87|5929.68|5886.90|8123.55|533.67|0.00|1461.96|5929.68|6463.35|7391.64|7925.31|42.78| +2450823|55187|2450825|35806|1598934|2509|29684|35806|1598934|2509|29684|2|89|3|4|12104|126|412|69|71.92|185.55|111.33|5121.18|7681.77|4962.48|12802.95|307.27|0.00|0.00|7681.77|7989.04|7681.77|7989.04|2719.29| +2450823|55187|2450850|35806|1598934|2509|29684|35806|1598934|2509|29684|2|39|20|1|10744|63|412|42|22.26|39.17|21.15|756.84|888.30|934.92|1645.14|6.04|586.27|657.72|302.03|308.07|959.75|965.79|-632.89| +2450823|74463|2450837|87571|1528653|6535|26064|87571|1528653|6535|26064|4|59|11|4|17434|67|413|71|26.79|38.30|19.91|1305.69|1413.61|1902.09|2719.30|0.00|0.00|352.87|1413.61|1413.61|1766.48|1766.48|-488.48| +2450823|74463|2450829|87571|1528653|6535|26064|87571|1528653|6535|26064|4|27|5|4|12478|19|413|51|46.72|47.18|2.35|2286.33|119.85|2382.72|2406.18|4.31|71.91|168.30|47.94|52.25|216.24|220.55|-2334.78| +2450823|74463|2450880|87571|1528653|6535|26064|87571|1528653|6535|26064|4|87|12|2|9139|99|413|82|9.94|12.72|2.92|803.60|239.44|815.08|1043.04|4.78|0.00|51.66|239.44|244.22|291.10|295.88|-575.64| +2450823|74463|2450864|87571|1528653|6535|26064|87571|1528653|6535|26064|4|85|12|5|7166|233|413|9|78.96|170.55|69.92|905.67|629.28|710.64|1534.95|18.87|0.00|752.04|629.28|648.15|1381.32|1400.19|-81.36| +2450823|74463|2450832|87571|1528653|6535|26064|87571|1528653|6535|26064|4|30|6|5|11731|270|413|4|68.21|171.20|99.29|287.64|397.16|272.84|684.80|15.88|0.00|260.20|397.16|413.04|657.36|673.24|124.32| +2450823|74463|2450883|87571|1528653|6535|26064|87571|1528653|6535|26064|4|35|15|4|10064|159|413|31|66.52|152.99|35.18|3652.11|1090.58|2062.12|4742.69|43.62|0.00|2039.18|1090.58|1134.20|3129.76|3173.38|-971.54| +2450823|74463|2450874|87571|1528653|6535|26064|87571|1528653|6535|26064|4|22|15|3|1267|92|413|68|77.60|81.48|77.40|277.44|5263.20|5276.80|5540.64|473.68|0.00|2659.48|5263.20|5736.88|7922.68|8396.36|-13.60| +2450823|74463|2450905|87571|1528653|6535|26064|87571|1528653|6535|26064|4|29|1|1|11017|68|413|49|72.16|103.91|7.27|4735.36|356.23|3535.84|5091.59|21.37|0.00|1069.18|356.23|377.60|1425.41|1446.78|-3179.61| +2450823|74463|2450885|87571|1528653|6535|26064|87571|1528653|6535|26064|4|56|5|3|6110|104|413|93|44.63|59.80|7.77|4838.79|722.61|4150.59|5561.40|7.22|0.00|2780.70|722.61|729.83|3503.31|3510.53|-3427.98| +2450823|74463|2450843|87571|1528653|6535|26064|87571|1528653|6535|26064|4|60|13|2|6290|128|413|54|62.17|115.63|53.18|3372.30|2871.72|3357.18|6244.02|258.45|0.00|1560.60|2871.72|3130.17|4432.32|4690.77|-485.46| +2450823|74463|2450912|87571|1528653|6535|26064|87571|1528653|6535|26064|4|39|18|4|8786|11|413|19|68.69|172.41|144.82|524.21|2751.58|1305.11|3275.79|137.57|0.00|655.12|2751.58|2889.15|3406.70|3544.27|1446.47| +2450823|74463|2450850|87571|1528653|6535|26064|87571|1528653|6535|26064|4|74|14|4|202|26|413|45|94.54|178.68|94.70|3779.10|4261.50|4254.30|8040.60|298.30|0.00|3939.75|4261.50|4559.80|8201.25|8499.55|7.20| +2450823|9786|2450913|58863|1851147|6631|25668|58863|1851147|6631|25668|1|108|14|5|10414|245|414|14|93.15|267.34|21.38|3443.44|299.32|1304.10|3742.76|2.99|0.00|1309.84|299.32|302.31|1609.16|1612.15|-1004.78| +2450823|9786|2450875|58863|1851147|6631|25668|58863|1851147|6631|25668|1|80|8|5|8599|21|414|19|80.78|168.02|95.77|1372.75|1819.63|1534.82|3192.38|145.57|0.00|1564.08|1819.63|1965.20|3383.71|3529.28|284.81| +2450823|9786|2450842|58863|1851147|6631|25668|58863|1851147|6631|25668|1|41|11|5|17308|21|414|78|62.11|79.50|50.08|2294.76|3906.24|4844.58|6201.00|156.24|0.00|2045.94|3906.24|4062.48|5952.18|6108.42|-938.34| +2450823|9786|2450847|58863|1851147|6631|25668|58863|1851147|6631|25668|1|108|3|1|16189|72|414|66|10.51|26.27|19.96|416.46|1317.36|693.66|1733.82|118.56|0.00|554.40|1317.36|1435.92|1871.76|1990.32|623.70| +2450823|9786|2450851|58863|1851147|6631|25668|58863|1851147|6631|25668|1|72|8|5|11900|39|414|89|94.14|174.15|64.43|9765.08|5734.27|8378.46|15499.35|88.88|3956.64|7284.65|1777.63|1866.51|9062.28|9151.16|-6600.83| +2450823|9786|2450876|58863|1851147|6631|25668|58863|1851147|6631|25668|1|24|1|4|12274|228|414|11|83.16|158.00|115.34|469.26|1268.74|914.76|1738.00|0.00|0.00|225.94|1268.74|1268.74|1494.68|1494.68|353.98| +2450823|9786|2450879|58863|1851147|6631|25668|58863|1851147|6631|25668|1|29|15|2|10954|32|414|81|33.43|92.26|32.29|4857.57|2615.49|2707.83|7473.06|209.23|0.00|672.30|2615.49|2824.72|3287.79|3497.02|-92.34| +2450823|9786|2450871|58863|1851147|6631|25668|58863|1851147|6631|25668|1|50|8|5|6637|246|414|94|47.37|99.95|46.97|4980.12|4415.18|4452.78|9395.30|44.15|0.00|2160.12|4415.18|4459.33|6575.30|6619.45|-37.60| +2450823|9786|2450825|58863|1851147|6631|25668|58863|1851147|6631|25668|1|96|8|1|10069|52|414|7|44.18|85.26|24.72|423.78|173.04|309.26|596.82|15.57|0.00|47.74|173.04|188.61|220.78|236.35|-136.22| +|9786|2450876|58863||6631|25668||1851147|6631||1||20||6523|241|414|98|51.96|92.48|45.31|4622.66|4440.38|5092.08|9063.04|||180.32|||4620.70|4709.50|-651.70| +2450823|9786|2450837|58863|1851147|6631|25668|58863|1851147|6631|25668|1|52|5|2|1843|233|414|18|3.87|9.86|4.04|104.76|72.72|69.66|177.48|3.05|21.81|23.04|50.91|53.96|73.95|77.00|-18.75| +2450823|9786|2450841|58863|1851147|6631|25668|58863|1851147|6631|25668|1|88|19|2|17347|209|414|58|20.26|38.69|17.79|1212.20|1031.82|1175.08|2244.02|82.54|0.00|22.04|1031.82|1114.36|1053.86|1136.40|-143.26| +2450823|9786|2450913|58863|1851147|6631|25668|58863|1851147|6631|25668|1|74|13|2|12493|67|414|42|13.78|26.04|5.20|875.28|218.40|578.76|1093.68|2.75|80.80|284.34|137.60|140.35|421.94|424.69|-441.16| +2450823|56540|2450899|23085|1266596|1754|30865|23085|1266596|1754|30865|1|49|2|1|190|72|415|75|85.01|136.86|54.74|6159.00|4105.50|6375.75|10264.50|0.00|3653.89|4618.50|451.61|451.61|5070.11|5070.11|-5924.14| +2450823|56540|2450868|23085|1266596|1754|30865|23085|1266596|1754|30865|1|65|1|5|13466|141|415|69|39.92|45.50|32.30|910.80|2228.70|2754.48|3139.50|178.29|0.00|282.21|2228.70|2406.99|2510.91|2689.20|-525.78| +2450823|56540|2450847|23085|1266596|1754|30865|23085|1266596|1754|30865|1|82|9|4|1852|161|415|83|34.44|92.29|84.90|613.37|7046.70|2858.52|7660.07|0.00|0.00|1225.08|7046.70|7046.70|8271.78|8271.78|4188.18| +2450823|56540|2450900|23085|1266596|1754|30865|23085|1266596|1754|30865|1|20|6|3|370|149|415|43|85.26|135.56|127.42|350.02|5479.06|3666.18|5829.08|54.79|0.00|58.05|5479.06|5533.85|5537.11|5591.90|1812.88| +2450823|56540|2450840|23085|1266596|1754|30865|23085|1266596|1754|30865|1|16|8|1|3364|104|415|95|55.37|123.47|88.89|3285.10|8444.55|5260.15|11729.65|506.67|0.00|1055.45|8444.55|8951.22|9500.00|10006.67|3184.40| +2450823|56540|2450907|23085|1266596|1754|30865|23085|1266596|1754|30865|1|36|7|2|7675|222|415|21|48.05|65.82|29.61|760.41|621.81|1009.05|1382.22|55.96|0.00|55.23|621.81|677.77|677.04|733.00|-387.24| +2450823|56540|2450861|23085|1266596|1754|30865|23085|1266596|1754|30865|1|4|18|2|9904|123|415|40|83.78|215.31|88.27|5081.60|3530.80|3351.20|8612.40|176.54|0.00|3875.20|3530.80|3707.34|7406.00|7582.54|179.60| +2450823|11418|2450905|20435|143143|6298|45406|82635|94069|2377|1132|4|32|15|3|46|276|416|55|38.93|101.21|60.72|2226.95|3339.60|2141.15|5566.55|66.12|1135.46|1335.95|2204.14|2270.26|3540.09|3606.21|62.99| +2450823|11418|2450871|20435|143143|6298|45406|82635|94069|2377|1132|4|21|4|2|17306|103|416|36|79.67|144.99|2.89|5115.60|104.04|2868.12|5219.64|5.20|0.00|312.84|104.04|109.24|416.88|422.08|-2764.08| +2450823|11418|2450838|20435|143143|6298|45406|82635|94069|2377|1132|4|58|19|5|6481|42|416|99|24.38|41.68|30.00|1156.32|2970.00|2413.62|4126.32|148.50|0.00|948.42|2970.00|3118.50|3918.42|4066.92|556.38| +2450823|11418|2450845|20435|143143|6298|45406|82635|94069|2377|1132|4|39|9|5|9314|131|416|27|80.00|169.60|106.84|1694.52|2884.68|2160.00|4579.20|144.23|0.00|503.55|2884.68|3028.91|3388.23|3532.46|724.68| +2450823|11418|2450844|20435|143143|6298|45406|82635|94069|2377|1132|4|46|18|4|14068|96|416|20|46.71|94.35|32.07|1245.60|641.40|934.20|1887.00|32.07|0.00|320.60|641.40|673.47|962.00|994.07|-292.80| +2450823|11418|2450845|20435|143143|6298|45406|82635|94069|2377|1132|4|60|16|4|6112|155|416|91|1.63|3.16|0.41|250.25|37.31|148.33|287.56|1.86|0.00|37.31|37.31|39.17|74.62|76.48|-111.02| +2450823|11418|2450827|20435|143143|6298|45406|82635|94069|2377|1132|4|55|2|2|1486|234|416|51|35.64|90.16|21.63|3495.03|1103.13|1817.64|4598.16|11.03|0.00|183.60|1103.13|1114.16|1286.73|1297.76|-714.51| +2450823|11418|2450879|20435|143143|6298|45406|82635|94069|2377|1132|4|63|1|1|12196|49|416|43|41.16|87.67|54.35|1432.76|2337.05|1769.88|3769.81|0.00|1893.01|753.79|444.04|444.04|1197.83|1197.83|-1325.84| +2450823|11418|2450842|20435|143143|6298|45406|82635|94069|2377|1132|4|9|8|4|6788|191|416|46|26.57|34.00|9.18|1141.72|422.28|1222.22|1564.00|29.55|0.00|281.52|422.28|451.83|703.80|733.35|-799.94| +2450823|11418|2450863|20435|143143|6298|45406|82635|94069|2377|1132|4|13|12|4|16522|126|416|14|17.24|28.44|18.48|139.44|258.72|241.36|398.16|12.93|0.00|171.08|258.72|271.65|429.80|442.73|17.36| +2450823|69360|2450907|98061|997472|530|45988|98061|997472|530|45988|4|37|3|3|17114|223|417|71|27.59|37.79|12.47|1797.72|885.37|1958.89|2683.09|35.41|0.00|375.59|885.37|920.78|1260.96|1296.37|-1073.52| +2450823|69360|2450910|98061|997472|530|45988|98061|997472|530|45988|4|21|15|1|12247|32|417|86|63.07|151.36|118.06|2863.80|10153.16|5424.02|13016.96|609.18|0.00|5596.88|10153.16|10762.34|15750.04|16359.22|4729.14| +2450823|69360|2450913|98061|997472|530|45988|98061|997472|530|45988|4|61|18|4|2402|264|417|39|9.05|10.86|4.12|262.86|160.68|352.95|423.54|9.64|0.00|50.70|160.68|170.32|211.38|221.02|-192.27| +2450823|69360|2450836|98061|997472|530|45988|98061|997472|530|45988|4|74|8|3|8474|158|417|19|75.32|88.87|23.99|1232.72|455.81|1431.08|1688.53|9.11|0.00|354.54|455.81|464.92|810.35|819.46|-975.27| +2450823|69360|2450869|98061|997472|530|45988|98061|997472|530|45988|4|86|17|2|13514|160|417|96|90.94|271.00|37.94|22373.76|3642.24|8730.24|26016.00|90.32|1384.05|9886.08|2258.19|2348.51|12144.27|12234.59|-6472.05| +2450823|69360|2450834|98061|997472|530|45988|98061|997472|530|45988|4|103|6|4|4736|185|417|58|36.28|105.93|19.06|5038.46|1105.48|2104.24|6143.94|55.27|0.00|1658.80|1105.48|1160.75|2764.28|2819.55|-998.76| +2450823|15771|2450897|75716|165120|124|18681|61364|160242|46|20203|1|76|1|4|104|275|418|20|5.27|10.22|1.83|167.80|36.60|105.40|204.40|1.83|0.00|16.20|36.60|38.43|52.80|54.63|-68.80| +2450823|15771|2450904|75716|165120|124|18681|61364|160242|46|20203|1|60|4|1|7670|245|418|2|21.50|33.32|5.99|54.66|11.98|43.00|66.64|0.61|1.79|3.32|10.19|10.80|13.51|14.12|-32.81| +2450823|15771|2450868|75716|165120|124|18681|61364|160242|46|20203|1|24|20|5|11527|149|418|64|52.79|147.28|73.64|4712.96|4712.96|3378.56|9425.92|94.25|0.00|3675.52|4712.96|4807.21|8388.48|8482.73|1334.40| +2450823|15771|2450881|75716|165120|124|18681|61364|160242|46|20203|1|2|18|1|13480|286|418|47|54.88|100.43|15.06|4012.39|707.82|2579.36|4720.21|16.13|304.36|991.23|403.46|419.59|1394.69|1410.82|-2175.90| +2450823|15771|2450904|75716|165120|124|18681|61364|160242|46|20203|1|20|2|4|6436|284|418|30|47.72|99.25|96.27|89.40|2888.10|1431.60|2977.50|202.16|0.00|1071.90|2888.10|3090.26|3960.00|4162.16|1456.50| +2450823|15771|2450849|75716|165120|124|18681|61364|160242|46|20203|1|57|9|2|4472|142|418|71|42.19|99.14|84.26|1056.48|5982.46|2995.49|7038.94|179.47|0.00|1899.96|5982.46|6161.93|7882.42|8061.89|2986.97| +||||165120|124|18681|61364|160242|46|||97||4|9412|90|418|78|34.69||15.48|5502.90|||6710.34||0.00|201.24|1207.44|1279.88|1408.68|1481.12|-1498.38| +2450823|15771|2450906|75716|165120|124|18681|61364|160242|46|20203|1|108|19|2|1496|45|418|40|34.42|65.05|17.56|1899.60|702.40|1376.80|2602.00|49.16|0.00|572.40|702.40|751.56|1274.80|1323.96|-674.40| +2450823|15771|2450881|75716|165120|124|18681|61364|160242|46|20203|1|84|9|2|8560|216|418|94|41.22|45.34|29.47|1491.78|2770.18|3874.68|4261.96|193.91|0.00|2002.20|2770.18|2964.09|4772.38|4966.29|-1104.50| +2450823|15771|2450853|75716|165120|124|18681|61364|160242|46|20203|1|9|17|1|3620|227|418|37|95.72|176.12|135.61|1498.87|5017.57|3541.64|6516.44|200.70|0.00|781.81|5017.57|5218.27|5799.38|6000.08|1475.93| +2450823|50091|2450882|33562|1803936|1157|6331|33562|1803936|1157|6331|2|38|5|3|3686|202|419|97|79.16|183.65|104.68|7660.09|10153.96|7678.52|17814.05|241.66|6701.61|6768.66|3452.35|3694.01|10221.01|10462.67|-4226.17| +2450823|50091|2450830|33562|1803936|1157|6331|33562|1803936|1157|6331|2|102|12|3|8792|149|419|99|32.56|79.77|3.98|7503.21|394.02|3223.44|7897.23|3.94|0.00|552.42|394.02|397.96|946.44|950.38|-2829.42| +2450823|50091|2450897|33562|1803936|1157|6331|33562|1803936|1157|6331|2|106|5|4|15134|37|419|38|5.53|5.97|0.95|190.76|36.10|210.14|226.86|1.08|0.00|108.68|36.10|37.18|144.78|145.86|-174.04| +2450823|50091|2450895|33562|1803936|1157|6331|33562|1803936|1157|6331|2|84|18|4|17128|222|419|98|43.07|80.97|73.68|714.42|7220.64|4220.86|7935.06|361.03|0.00|1269.10|7220.64|7581.67|8489.74|8850.77|2999.78| +2450823|50091|2450856|33562|1803936|1157|6331|33562|1803936|1157|6331|2|93|8|1|11612|53|419|90|9.39|9.39|7.98|126.90|718.20|845.10|845.10|14.36|0.00|219.60|718.20|732.56|937.80|952.16|-126.90| +2450823|50091|2450902|33562|1803936|1157|6331|33562|1803936|1157|6331|2|9|9|3|6841|86|419|32|69.11|188.67|154.70|1087.04|4950.40|2211.52|6037.44|297.02|0.00|603.52|4950.40|5247.42|5553.92|5850.94|2738.88| +2450823|50091|2450884|33562|1803936|1157|6331|33562|1803936|1157|6331|2|6|1|4|16726|40|419|81|43.45|81.25|26.00|4475.25|2106.00|3519.45|6581.25|98.77|694.98|3092.58|1411.02|1509.79|4503.60|4602.37|-2108.43| +2450823|50091|2450867|33562|1803936|1157|6331|33562|1803936|1157|6331|2|103|5|3|16555|171|419|94|41.75|123.16|39.41|7872.50|3704.54|3924.50|11577.04|148.18|0.00|2430.84|3704.54|3852.72|6135.38|6283.56|-219.96| +2450823|50091|2450829|33562|1803936|1157|6331|33562|1803936|1157|6331|2|78|20|2|13750|150|419|85|82.22|172.66|1.72|14529.90|146.20|6988.70|14676.10|4.38|0.00|6310.40|146.20|150.58|6456.60|6460.98|-6842.50| +2450823|50091|2450857|33562|1803936|1157|6331|33562|1803936|1157|6331|2|98|20|1|8365|126|419|39|29.78|89.04|5.34|3264.30|208.26|1161.42|3472.56|10.64|56.23|416.52|152.03|162.67|568.55|579.19|-1009.39| +2450823|50091|2450851|33562|1803936|1157|6331|33562|1803936|1157|6331|2|32|17|1|5563|165|419|64|35.28|92.43|68.39|1538.56|4376.96|2257.92|5915.52|218.84|0.00|650.24|4376.96|4595.80|5027.20|5246.04|2119.04| +2450823|50091|2450834|33562|1803936|1157|6331|33562|1803936|1157|6331|2|20|13|2|15364|243|419|45|45.58|111.67|73.70|1708.65|3316.50|2051.10|5025.15|33.16|0.00|401.85|3316.50|3349.66|3718.35|3751.51|1265.40| +2450823|50091|2450860|33562|1803936|1157|6331|33562|1803936|1157|6331|2|96|20|5|9457|293|419|42|96.52|170.84|78.58|3874.92|3300.36|4053.84|7175.28|264.02|0.00|3515.82|3300.36|3564.38|6816.18|7080.20|-753.48| +2450823|50091|2450860|33562|1803936|1157|6331|33562|1803936|1157|6331|2|45|3|4|8516|144|419|67|82.62|180.11|18.01|10860.70|1206.67|5535.54|12067.37|0.00|0.00|2895.74|1206.67|1206.67|4102.41|4102.41|-4328.87| +2450823|25546|2450912|92253|911772|5338|44857|70340|1764342|5908|37427|1|95|15|2|10592|4|420|53|51.12|131.88|34.28|5172.80|1816.84|2709.36|6989.64|163.51|0.00|1956.76|1816.84|1980.35|3773.60|3937.11|-892.52| +2450823|25546|2450856|92253|911772|5338|44857|70340|1764342|5908|37427|1|62|10|5|1333|190|420|17|92.61|246.34|231.55|251.43|3936.35|1574.37|4187.78|39.36|0.00|1172.49|3936.35|3975.71|5108.84|5148.20|2361.98| +2450823|25546|2450839|92253|911772|5338|44857|70340|1764342|5908|37427|1|103|4|3|12757|261|420|61|98.43|215.56|99.15|7101.01|6048.15|6004.23|13149.16|0.00|1935.40|5917.00|4112.75|4112.75|10029.75|10029.75|-1891.48| +2450823|25546|2450879|92253|911772|5338|44857|70340|1764342|5908|37427|1|35|14|1|46|290|420|46|58.59|137.68|23.40|5256.88|1076.40|2695.14|6333.28|96.87|0.00|442.98|1076.40|1173.27|1519.38|1616.25|-1618.74| +2450823|25546|2450853|92253|911772|5338|44857|70340|1764342|5908|37427|1|3|2|4|17306|172|420|88|48.00|52.32|51.79|46.64|4557.52|4224.00|4604.16|45.57|0.00|1288.32|4557.52|4603.09|5845.84|5891.41|333.52| +2450823|25546|2450829|92253|911772|5338|44857|70340|1764342|5908|37427|1|86|18|4|6481|148|420|5|74.44|118.35|31.95|432.00|159.75|372.20|591.75|0.00|0.00|71.00|159.75|159.75|230.75|230.75|-212.45| +2450823|25546|2450840|92253|911772|5338|44857|70340|1764342|5908|37427|1|108|3|3|9314|108|420|94|30.66|74.81|26.18|4571.22|2460.92|2882.04|7032.14|147.65|0.00|2671.48|2460.92|2608.57|5132.40|5280.05|-421.12| +2450823|25546|2450836|92253|911772|5338|44857|70340|1764342|5908|37427|1|36|17|4|14068|298|420|52|39.18|115.58|57.79|3005.08|3005.08|2037.36|6010.16|93.75|1833.09|2463.76|1171.99|1265.74|3635.75|3729.50|-865.37| +2450823|25546|2450879|92253|911772|5338|44857|70340|1764342|5908|37427|1|87|10|5|6112|11|420|42|22.16|23.48|5.87|739.62|246.54|930.72|986.16|22.18|0.00|29.40|246.54|268.72|275.94|298.12|-684.18| +2450823|27545|2450875|9057|1646191|3772|7353|9057|1646191|3772|7353|2|30|3|4|16040|131|421|75|93.39|106.46|2.12|7825.50|159.00|7004.25|7984.50|1.59|0.00|1916.25|159.00|160.59|2075.25|2076.84|-6845.25| +2450823|27545|2450884|9057|1646191|3772|7353|9057|1646191|3772|7353|2|68|5|2|7118|164|421|61|18.36|44.98|44.98|0.00|2743.78|1119.96|2743.78|0.00|0.00|877.79|2743.78|2743.78|3621.57|3621.57|1623.82| +2450823|27545|2450862|9057|1646191|3772|7353|9057|1646191|3772|7353|2|77|8|5|8617|151|421|61|10.17|27.45|3.01|1490.84|183.61|620.37|1674.45|1.83|0.00|535.58|183.61|185.44|719.19|721.02|-436.76| +2450823|27545|2450868|9057|1646191|3772|7353|9057|1646191|3772|7353|2|42|12|1|646|12|421|94|50.86|121.04|113.77|683.38|10694.38|4780.84|11377.76|427.77|0.00|5005.50|10694.38|11122.15|15699.88|16127.65|5913.54| +2450823|27545|2450908|9057|1646191|3772|7353|9057|1646191|3772|7353|2|72|7|1|2377|289|421|1|15.72|46.68|27.07|19.61|27.07|15.72|46.68|0.46|17.86|12.60|9.21|9.67|21.81|22.27|-6.51| +2450823|27545|2450901|9057|1646191|3772|7353|9057|1646191|3772|7353|2|10|2|3|3823|199|421|62|12.42|16.76|10.22|405.48|633.64|770.04|1039.12|20.14|297.81|280.24|335.83|355.97|616.07|636.21|-434.21| +2450823|27545|2450895|9057|1646191|3772|7353|9057|1646191|3772|7353|2|63|8|5|11692|87|421|85|28.33|46.46|17.19|2487.95|1461.15|2408.05|3949.10|116.89|0.00|1737.40|1461.15|1578.04|3198.55|3315.44|-946.90| +2450823|27545|2450897|9057|1646191|3772|7353|9057|1646191|3772|7353|2|32|5|3|3493|279|421|100|22.92|42.86|16.28|2658.00|1628.00|2292.00|4286.00|146.52|0.00|1757.00|1628.00|1774.52|3385.00|3531.52|-664.00| +2450823|27545|2450909|9057|1646191|3772|7353|9057|1646191|3772|7353|2|95|4|3|2846|48|421|40|94.97|168.09|67.23|4034.40|2689.20|3798.80|6723.60|188.24|0.00|3227.20|2689.20|2877.44|5916.40|6104.64|-1109.60| +2450823|27545|2450863|9057|1646191|3772|7353|9057|1646191|3772|7353|2|50|3|3|1972|170|421|24|42.24|108.55|106.37|52.32|2552.88|1013.76|2605.20|102.11|0.00|599.04|2552.88|2654.99|3151.92|3254.03|1539.12| +2450823|27545|2450903|9057|1646191|3772|7353|9057|1646191|3772|7353|2|86|11|1|1198|124|421|88|29.70|41.87|20.93|1842.72|1841.84|2613.60|3684.56|128.92|0.00|1804.88|1841.84|1970.76|3646.72|3775.64|-771.76| +2450823|27545|2450888|9057|1646191|3772|7353|9057|1646191|3772|7353|2|28|16|1|4825|164|421|9|25.30|70.84|35.42|318.78|318.78|227.70|637.56|3.44|261.39|197.64|57.39|60.83|255.03|258.47|-170.31| +2450823|27545|2450876|9057|1646191|3772|7353|9057|1646191|3772|7353|2|82|4|3|14024|11|421|75|72.54|189.32|0.00|14199.00|0.00|5440.50|14199.00|0.00|0.00|5111.25|0.00|0.00|5111.25|5111.25|-5440.50| +2450823|38104|2450912|79638|1183003|1121|44886|16644|713169|3734|43099|4|11|10|1|13729|238|422|17|29.30|33.98|26.16|132.94|444.72|498.10|577.66|40.02|0.00|207.91|444.72|484.74|652.63|692.65|-53.38| +2450823|38104|2450865|79638|1183003|1121|44886|16644|713169|3734|43099|4|56|13|2|9008|268|422|73|63.11|176.07|24.64|11054.39|1798.72|4607.03|12853.11|17.98|0.00|3469.69|1798.72|1816.70|5268.41|5286.39|-2808.31| +2450823|38104|2450858|79638|1183003|1121|44886|16644|713169|3734|43099|4|85|7|1|16426|36|422|100|87.16|152.53|18.30|13423.00|1830.00|8716.00|15253.00|41.17|457.50|762.00|1372.50|1413.67|2134.50|2175.67|-7343.50| +2450823|38104|2450888|79638|1183003|1121|44886|16644|713169|3734|43099|4|64|19|5|8245|116|422|53|1.77|2.15|1.07|57.24|56.71|93.81|113.95|1.70|0.00|4.24|56.71|58.41|60.95|62.65|-37.10| +2450823|38104|2450858|79638|1183003|1121|44886|16644|713169|3734|43099|4|91|17|3|14576|145|422|36|84.84|133.19|23.97|3931.92|862.92|3054.24|4794.84|0.00|0.00|623.16|862.92|862.92|1486.08|1486.08|-2191.32| +2450823|38104|2450883|79638|1183003|1121|44886|16644|713169|3734|43099|4|28|19|1|15391|40|422|60|63.11|120.54|59.06|3688.80|3543.60|3786.60|7232.40|283.48|0.00|1735.20|3543.60|3827.08|5278.80|5562.28|-243.00| +2450823|38104|2450860|79638|1183003|1121|44886|16644|713169|3734|43099|4|65|8|4|5845|143|422|27|41.34|63.25|32.89|819.72|888.03|1116.18|1707.75|35.52|0.00|136.62|888.03|923.55|1024.65|1060.17|-228.15| +2450823|38104|2450890|79638|1183003|1121|44886|16644|713169|3734|43099|4|21|18|1|17521|146|422|100|22.51|36.69|10.64|2605.00|1064.00|2251.00|3669.00|53.20|0.00|36.00|1064.00|1117.20|1100.00|1153.20|-1187.00| +2450823|38104|2450857|79638|1183003|1121|44886|16644|713169|3734|43099|4|30|2|2|10981|190|422|36|17.79|47.85|26.79|758.16|964.44|640.44|1722.60|19.28|0.00|51.48|964.44|983.72|1015.92|1035.20|324.00| +2450823|46250|2450850|58775|838279|1783|20440|10275|1587401|2905|13562|1|95|9|4|3430|102|423|7|22.25|40.71|17.09|165.34|119.63|155.75|284.97|1.19|0.00|76.93|119.63|120.82|196.56|197.75|-36.12| +2450823|46250|2450847|58775|838279|1783|20440|10275|1587401|2905|13562|1|27|6|1|6979|227|423|51|42.24|119.96|15.59|5322.87|795.09|2154.24|6117.96|47.70|0.00|2814.18|795.09|842.79|3609.27|3656.97|-1359.15| +2450823|46250|2450868|58775|838279|1783|20440|10275|1587401|2905|13562|1|29|8|2|9088|55|423|21|4.01|4.61|1.24|70.77|26.04|84.21|96.81|0.26|0.00|42.42|26.04|26.30|68.46|68.72|-58.17| +2450823|46250|2450886|58775|838279|1783|20440|10275|1587401|2905|13562|1|14|2|4|12788|57|423|45|46.01|126.52|84.76|1879.20|3814.20|2070.45|5693.40|228.85|0.00|1423.35|3814.20|4043.05|5237.55|5466.40|1743.75| +2450823|46250|2450890|58775|838279|1783|20440|10275|1587401|2905|13562|1|24|7|2|16666|158|423|37|10.83|17.00|12.41|169.83|459.17|400.71|629.00|41.32|0.00|301.92|459.17|500.49|761.09|802.41|58.46| +2450823|46250|2450869|58775|838279|1783|20440|10275|1587401|2905|13562|1|93|13|4|16330|59|423|72|34.94|87.35|0.87|6226.56|62.64|2515.68|6289.20|5.01|0.00|565.92|62.64|67.65|628.56|633.57|-2453.04| +2450823|46250|2450884|58775|838279|1783|20440|10275|1587401|2905|13562|1|102|8|5|9092|214|423|11|92.88|119.81|73.08|514.03|803.88|1021.68|1317.91|16.07|0.00|316.25|803.88|819.95|1120.13|1136.20|-217.80| +2450823|46250|2450830|58775|838279|1783|20440|10275|1587401|2905|13562|1|4|17|5|17038|50|423|19|21.11|46.01|19.78|498.37|375.82|401.09|874.19|3.75|0.00|113.62|375.82|379.57|489.44|493.19|-25.27| +2450823|46250|2450905|58775|838279|1783|20440|10275|1587401|2905|13562|1|20|1|3|14228|134|423|87|73.00|92.71|0.92|7985.73|80.04|6351.00|8065.77|2.74|40.82|321.90|39.22|41.96|361.12|363.86|-6311.78| +2450823|46250|2450857|58775|838279|1783|20440|10275|1587401|2905|13562|1|102|9|3|9982|276|423|63|96.61|271.47|228.03|2736.72|14365.89|6086.43|17102.61|462.58|7757.58|1025.64|6608.31|7070.89|7633.95|8096.53|521.88| +2450823|46250|2450875|58775|838279|1783|20440|10275|1587401|2905|13562|1|20|1|4|15694|138|423|24|70.29|152.52|114.39|915.12|2745.36|1686.96|3660.48|164.72|0.00|1207.92|2745.36|2910.08|3953.28|4118.00|1058.40| +2450823|46250|2450832|58775|838279|1783|20440|10275|1587401|2905|13562|1|33|4|2|236|273|423|96|80.23|201.37|167.13|3287.04|16044.48|7702.08|19331.52|0.00|0.00|6765.12|16044.48|16044.48|22809.60|22809.60|8342.40| +2450823|46250|2450848|58775|838279|1783|20440|10275|1587401|2905|13562|1|66|7|3|11245|141|423|50|2.45|2.47|1.03|72.00|51.50|122.50|123.50|0.51|0.00|49.00|51.50|52.01|100.50|101.01|-71.00| +2450823||2450894|62100||4921|||672134|||2||5||13964|239|424|66|||||988.68|||29.66||494.34||1018.34|1483.02|1512.68|-1250.70| +2450823|68337|2450906|62100|672134|4921|10155|62100|672134|4921|10155|2|47|20|1|8257|197|424|97|22.48|46.75|44.88|181.39|4353.36|2180.56|4534.75|104.48|1741.34|815.77|2612.02|2716.50|3427.79|3532.27|431.46| +2450823|68337|2450850|62100|672134|4921|10155|62100|672134|4921|10155|2|22|9|3|11276|36|424|61|29.23|59.62|17.88|2546.14|1090.68|1783.03|3636.82|54.20|316.29|872.30|774.39|828.59|1646.69|1700.89|-1008.64| +2450823|68337|2450828|62100|672134|4921|10155|62100|672134|4921|10155|2|61|9|1|7856|190|424|98|17.55|18.25|1.09|1681.68|106.82|1719.90|1788.50|2.13|0.00|89.18|106.82|108.95|196.00|198.13|-1613.08| +2450823|32241|2450891|36669|1647473|6277|28234|36669|1647473|6277|28234|2|40|5|3|15418|145|425|47|71.89|148.81|142.85|280.12|6713.95|3378.83|6994.07|537.11|0.00|2028.05|6713.95|7251.06|8742.00|9279.11|3335.12| +2450823|32241|2450843|36669|1647473|6277|28234|36669|1647473|6277|28234|2|59|3|2|4840|207|425|43|44.97|107.47|83.82|1016.95|3604.26|1933.71|4621.21|36.04|0.00|2310.39|3604.26|3640.30|5914.65|5950.69|1670.55| +2450823|32241|2450837|36669|1647473|6277|28234|36669|1647473|6277|28234|2|60|9|1|11653|141|425|32|17.36|46.52|8.83|1206.08|282.56|555.52|1488.64|5.65|0.00|252.80|282.56|288.21|535.36|541.01|-272.96| +2450823|32241|2450879|36669|1647473|6277|28234|36669|1647473|6277|28234|2|23|16|5|7402|68|425|65|27.30|33.57|4.36|1898.65|283.40|1774.50|2182.05|22.67|0.00|1025.05|283.40|306.07|1308.45|1331.12|-1491.10| +2450823|32241|2450840|36669|1647473|6277|28234|36669|1647473|6277|28234|2|12|7|2|4600|203|425|6|89.18|167.65|160.94|40.26|965.64|535.08|1005.90|67.59|0.00|492.84|965.64|1033.23|1458.48|1526.07|430.56| +2450823|32241|2450841|36669|1647473|6277|28234|36669|1647473|6277|28234|2|49|19|4|6619|297|425|51|94.79|160.19|92.91|3431.28|4738.41|4834.29|8169.69|379.07|0.00|1061.82|4738.41|5117.48|5800.23|6179.30|-95.88| +2450823|32241|2450899|36669|1647473|6277|28234|36669|1647473|6277|28234|2|4|12|4|7822|194|425|6|3.26|3.42|1.88|9.24|11.28|19.56|20.52|0.83|2.03|4.26|9.25|10.08|13.51|14.34|-10.31| +2450823|32241|2450900|36669|1647473|6277|28234|36669|1647473|6277|28234|2|16|10|3|14014|224|425|98|11.82|24.70|18.03|653.66|1766.94|1158.36|2420.60|40.28|1095.50|48.02|671.44|711.72|719.46|759.74|-486.92| +2450823|32241|2450847|36669|1647473|6277|28234|36669|1647473|6277|28234|2|1|11|3|16879|257|425|88|98.15|283.65|226.92|4992.24|19968.96|8637.20|24961.20|1186.15|199.68|11731.28|19769.28|20955.43|31500.56|32686.71|11132.08| +2450823|32241|2450857|36669|1647473|6277|28234|36669|1647473|6277|28234|2|14|17|5|3091|186|425|53|61.20|64.87|64.22|34.45|3403.66|3243.60|3438.11|34.03|0.00|1031.38|3403.66|3437.69|4435.04|4469.07|160.06| +2450823|32241|2450848|36669|1647473|6277|28234|36669|1647473|6277|28234|2|68|13|3|262|65|425|3|1.55|2.82|0.73|6.27|2.19|4.65|8.46|0.08|0.00|2.34|2.19|2.27|4.53|4.61|-2.46| +2450823|32241|2450873|36669|1647473|6277|28234|36669|1647473|6277|28234|2|67|13|2|8026|70|425|2|5.72|15.84|6.17|19.34|12.34|11.44|31.68|0.74|0.00|2.52|12.34|13.08|14.86|15.60|0.90| +2450823|32241|2450840|36669|1647473|6277|28234|36669|1647473|6277|28234|2|75|6|2|10802|294|425|89|27.45|34.86|21.96|1148.10|1954.44|2443.05|3102.54|97.72|0.00|898.90|1954.44|2052.16|2853.34|2951.06|-488.61| +2450823|36246|2450899|60353|1256875|150|49872|60353|1256875|150|49872|1|103|6|5|12553|20|426|41|42.97|79.92|65.53|589.99|2686.73|1761.77|3276.72|0.00|0.00|720.78|2686.73|2686.73|3407.51|3407.51|924.96| +2450823|36246|2450892|60353|1256875|150|49872|60353|1256875|150|49872|1|16|11|3|2324|152|426|33|78.64|187.94|71.41|3845.49|2356.53|2595.12|6202.02|212.08|0.00|61.71|2356.53|2568.61|2418.24|2630.32|-238.59| +2450823|36246|2450871|60353|1256875|150|49872|60353|1256875|150|49872|1|59|6|1|5695|90|426|97|51.04|87.27|79.41|762.42|7702.77|4950.88|8465.19|462.16|0.00|3809.19|7702.77|8164.93|11511.96|11974.12|2751.89| +2450823|36246|2450845|60353|1256875|150|49872|60353|1256875|150|49872|1|27|14|4|2893|7|426|66|66.69|98.70|85.86|847.44|5666.76|4401.54|6514.20|283.33|0.00|2084.28|5666.76|5950.09|7751.04|8034.37|1265.22| +2450823|36246|2450866|60353|1256875|150|49872|60353|1256875|150|49872|1|69|6|4|7030|217|426|99|51.71|88.94|20.45|6780.51|2024.55|5119.29|8805.06|0.00|0.00|703.89|2024.55|2024.55|2728.44|2728.44|-3094.74| +2450823|36246|2450835|60353|1256875|150|49872|60353|1256875|150|49872|1|43|9|5|2644|67|426|63|17.15|36.35|13.81|1420.02|870.03|1080.45|2290.05|52.20|0.00|503.37|870.03|922.23|1373.40|1425.60|-210.42| +2450823|36246|2450911|60353|1256875|150|49872|60353|1256875|150|49872|1|15|11|3|3430|164|426|94|36.15|83.50|81.83|156.98|7692.02|3398.10|7849.00|153.84|0.00|862.92|7692.02|7845.86|8554.94|8708.78|4293.92| +2450823|33548|2450835|32701|573916|2684|22299|32701|573916|2684|22299|4|75|3|2|13430|87|427|85|61.93|165.97|21.57|12274.00|1833.45|5264.05|14107.45|0.00|0.00|2962.25|1833.45|1833.45|4795.70|4795.70|-3430.60| +2450823|33548|2450880|32701|573916|2684|22299|32701|573916|2684|22299|4|60|5|3|8881|251|427|23|33.35|99.38|99.38|0.00|2285.74|767.05|2285.74|182.85|0.00|868.48|2285.74|2468.59|3154.22|3337.07|1518.69| +2450823|33548|2450856|32701|573916|2684|22299|32701|573916|2684|22299|4|8|12|1|1483|31|427|43|98.18|124.68|79.79|1930.27|3430.97|4221.74|5361.24|68.61|0.00|1232.81|3430.97|3499.58|4663.78|4732.39|-790.77| +2450823|33548|2450866|32701|573916|2684|22299|32701|573916|2684|22299|4|51|14|2|86|108|427|25|54.24|99.25|77.41|546.00|1935.25|1356.00|2481.25|96.76|0.00|1017.25|1935.25|2032.01|2952.50|3049.26|579.25| +2450823|33548|2450863|32701|573916|2684|22299|32701|573916|2684|22299|4|40|18|1|13066|5|427|75|41.93|57.44|24.69|2456.25|1851.75|3144.75|4308.00|148.14|0.00|1636.50|1851.75|1999.89|3488.25|3636.39|-1293.00| +2450823|33548|2450913|32701|573916|2684|22299|32701|573916|2684|22299|4|42|15|5|5464|114|427|87|84.36|242.95|104.46|12048.63|9088.02|7339.32|21136.65|19.08|8815.37|2958.87|272.65|291.73|3231.52|3250.60|-7066.67| +2450823|33548|2450898|32701|573916|2684|22299|32701|573916|2684|22299|4|61|12|4|16514|226|427|87|94.05|212.55|178.54|2958.87|15532.98|8182.35|18491.85|621.31|0.00|6286.62|15532.98|16154.29|21819.60|22440.91|7350.63| +2450823|33548|2450859|32701|573916|2684|22299|32701|573916|2684|22299|4|74|14|2|5005|80|427|18|4.95|6.68|2.20|80.64|39.60|89.10|120.24|2.37|0.00|22.68|39.60|41.97|62.28|64.65|-49.50| +2450823|33548|2450893|32701|573916|2684|22299|32701|573916|2684|22299|4|82|6|4|7844|288|427|19|4.53|11.86|8.65|60.99|164.35|86.07|225.34|14.79|0.00|15.77|164.35|179.14|180.12|194.91|78.28| +2450823|33548|2450862|32701|573916|2684|22299|32701|573916|2684|22299|4|86|17|4|12343|225|427|86|7.31|8.40|1.00|636.40|86.00|628.66|722.40|1.78|26.66|288.96|59.34|61.12|348.30|350.08|-569.32| +2450823|66635|2450894|37086|1694647|6608|14969|37086|1694647|6608|14969|1|24|14|5|5230|121|428|4|18.46|26.58|0.53|104.20|2.12|73.84|106.32|0.02|1.16|10.60|0.96|0.98|11.56|11.58|-72.88| +2450823|66635|2450910|37086|1694647|6608|14969|37086|1694647|6608|14969|1|40|6|2|15988|212|428|50|74.79|160.79|120.59|2010.00|6029.50|3739.50|8039.50|241.18|0.00|723.50|6029.50|6270.68|6753.00|6994.18|2290.00| +2450823|66635|2450835|37086|1694647|6608|14969|37086|1694647|6608|14969|1|47|16|1|5629|70|428|55|59.97|137.93|117.24|1137.95|6448.20|3298.35|7586.15|193.44|0.00|1289.20|6448.20|6641.64|7737.40|7930.84|3149.85| +2450823|66635|2450831|37086|1694647|6608|14969|37086|1694647|6608|14969|1|94|5|5|14840|143|428|84|94.37|201.95|137.32|5428.92|11534.88|7927.08|16963.80|461.39|0.00|3223.08|11534.88|11996.27|14757.96|15219.35|3607.80| +2450823|66635|2450840|37086|1694647|6608|14969|37086|1694647|6608|14969|1|69|12|5|8563|65|428|83|21.69|32.53|20.16|1026.71|1673.28|1800.27|2699.99|8.53|819.90|1160.34|853.38|861.91|2013.72|2022.25|-946.89| +2450823|66635|2450846|37086|1694647|6608|14969|37086|1694647|6608|14969|1|53|10|1|6680|267|428|4|24.09|72.02|38.89|132.52|155.56|96.36|288.08|12.44|0.00|103.68|155.56|168.00|259.24|271.68|59.20| +2450823|70684|2450911|29019|1524654|3136|33618|29019|1524654|3136|33618|2|102|1|4|17269|15|429|34|4.54|10.12|7.59|86.02|258.06|154.36|344.08|12.90|0.00|103.02|258.06|270.96|361.08|373.98|103.70| +2450823|70684|2450903|29019|1524654|3136|33618|29019|1524654|3136|33618|2|87|13|4|15091|268|429|87|21.92|35.29|13.76|1873.11|1197.12|1907.04|3070.23|71.82|0.00|1319.79|1197.12|1268.94|2516.91|2588.73|-709.92| +2450823|70684|2450862|29019|1524654|3136|33618|29019|1524654|3136|33618|2|29|13|5|11330|204|429|25|35.19|43.63|39.26|109.25|981.50|879.75|1090.75|9.81|0.00|490.75|981.50|991.31|1472.25|1482.06|101.75| +2450823|70684|2450834|29019|1524654|3136|33618|29019|1524654|3136|33618|2|64|6|5|14074|23|429|93|66.96|193.51|187.70|540.33|17456.10|6227.28|17996.43|1221.92|0.00|2519.37|17456.10|18678.02|19975.47|21197.39|11228.82| +2450823|70684|2450890|29019|1524654|3136|33618|29019|1524654|3136|33618|2|78|17|5|13558|8|429|8|27.91|70.89|0.70|561.52|5.60|223.28|567.12|0.05|0.00|124.72|5.60|5.65|130.32|130.37|-217.68| +2450823|70684|2450866|29019|1524654|3136|33618|29019|1524654|3136|33618|2|82|20|3|7106|264|429|42|85.55|148.00|63.64|3543.12|2672.88|3593.10|6216.00|133.64|0.00|1989.12|2672.88|2806.52|4662.00|4795.64|-920.22| +2450823|34117|2450906|2132|1631362|6545|8128|2132|1631362|6545|8128|1|52|5|4|17384|265|430|27|64.63|76.90|63.05|373.95|1702.35|1745.01|2076.30|51.07|0.00|747.36|1702.35|1753.42|2449.71|2500.78|-42.66| +2450823|34117|2450861|2132|1631362|6545|8128|2132|1631362|6545|8128|1|56|11|2|3802|26|430|20|9.84|25.19|3.27|438.40|65.40|196.80|503.80|2.01|15.04|55.40|50.36|52.37|105.76|107.77|-146.44| +2450823|34117|2450871|2132|1631362|6545|8128|2132|1631362|6545|8128|1|90|9|3|9121|136|430|84|26.85|73.83|39.86|2853.48|3348.24|2255.40|6201.72|24.10|3080.38|1736.28|267.86|291.96|2004.14|2028.24|-1987.54| +2450823|34117|2450873|2132|1631362|6545|8128|2132|1631362|6545|8128|1|18|15|5|13246|50|430|41|81.46|118.93|38.05|3316.08|1560.05|3339.86|4876.13|1.87|1513.24|1657.63|46.81|48.68|1704.44|1706.31|-3293.05| +2450823|34117|2450869|2132|1631362|6545|8128|2132|1631362|6545|8128|1|79|4|4|4036|149|430|67|85.16|160.95|67.59|6255.12|4528.53|5705.72|10783.65|362.28|0.00|2479.67|4528.53|4890.81|7008.20|7370.48|-1177.19| +2450823|34117|2450863|2132|1631362|6545|8128|2132|1631362|6545|8128|1|24|2|4|17269|7|430|94|75.53|200.15|28.02|16180.22|2633.88|7099.82|18814.10|184.37|0.00|5643.76|2633.88|2818.25|8277.64|8462.01|-4465.94| +2450823|34117|2450865|2132|1631362|6545|8128|2132|1631362|6545|8128|1|4|12|2|15091|142|430|78|46.80|101.08|28.30|5676.84|2207.40|3650.40|7884.24|198.66|0.00|2207.40|2207.40|2406.06|4414.80|4613.46|-1443.00| +2450823|74558|2450849|78582|1866571|5760|48549|78582|1866571|5760|48549|1|6|16|2|15776|174|431|35|28.59|40.31|3.22|1298.15|112.70|1000.65|1410.85|1.72|93.54|521.85|19.16|20.88|541.01|542.73|-981.49| +2450823|74558|2450853|78582|1866571|5760|48549|78582|1866571|5760|48549|1|31|16|5|3085|131|431|57|93.14|172.30|62.02|6285.96|3535.14|5308.98|9821.10|70.70|0.00|2062.26|3535.14|3605.84|5597.40|5668.10|-1773.84| +2450823|74558|2450868|78582|1866571|5760|48549|78582|1866571|5760|48549|1|88|13|3|10084|235|431|44|20.80|32.65|30.36|100.76|1335.84|915.20|1436.60|120.22|0.00|703.56|1335.84|1456.06|2039.40|2159.62|420.64| +2450823|74558|2450842|78582|1866571|5760|48549|78582|1866571|5760|48549|1|83|13|5|2257|266|431|46|59.00|76.11|38.05|1750.76|1750.30|2714.00|3501.06|140.02|0.00|735.08|1750.30|1890.32|2485.38|2625.40|-963.70| +2450823|74558|2450897|78582|1866571|5760|48549|78582|1866571|5760|48549|1|108|18|3|14113|256|431|46|21.84|40.62|12.59|1289.38|579.14|1004.64|1868.52|0.00|0.00|504.16|579.14|579.14|1083.30|1083.30|-425.50| +2450823|74558|2450908|78582|1866571|5760|48549|78582|1866571|5760|48549|1|19|8|5|3176|229|431|15|38.89|53.27|28.23|375.60|423.45|583.35|799.05|8.46|0.00|343.50|423.45|431.91|766.95|775.41|-159.90| +2450823|74558|2450897|78582|1866571|5760|48549|78582|1866571|5760|48549|1|83|9|2|15644|16|431|18|33.78|45.94|38.13|140.58|686.34|608.04|826.92|20.59|0.00|256.32|686.34|706.93|942.66|963.25|78.30| +2450823|74558|2450844|78582|1866571|5760|48549|78582|1866571|5760|48549|1|65|12|4|13585|73|431|62|42.32|56.70|20.41|2249.98|1265.42|2623.84|3515.40|101.23|0.00|316.20|1265.42|1366.65|1581.62|1682.85|-1358.42| +2450823|74558|2450910|78582|1866571|5760|48549|78582|1866571|5760|48549|1|50|2|3|10171|173|431|75|92.80|111.36|57.90|4009.50|4342.50|6960.00|8352.00|81.63|260.55|166.50|4081.95|4163.58|4248.45|4330.08|-2878.05| +2450823|74558|2450855|78582|1866571|5760|48549|78582|1866571|5760|48549|1|9|8|1|2360|165|431|33|83.34|250.02|175.01|2475.33|5775.33|2750.22|8250.66|0.00|0.00|1402.50|5775.33|5775.33|7177.83|7177.83|3025.11| +2450823|74558|2450861|78582|1866571|5760|48549|78582|1866571|5760|48549|1|88|4|3|13495|68|431|2|11.53|31.59|30.01|3.16|60.02|23.06|63.18|1.80|0.00|13.26|60.02|61.82|73.28|75.08|36.96| +2450823|74558|2450903|78582|1866571|5760|48549|78582|1866571|5760|48549|1|38|11|3|16730|257|431|53|57.58|164.67|74.10|4800.21|3927.30|3051.74|8727.51|93.47|2592.01|2792.57|1335.29|1428.76|4127.86|4221.33|-1716.45| +2450823|74558|2450875|78582|1866571|5760|48549|78582|1866571|5760|48549|1|92|4|4|17266|56|431|85|68.40|201.78|56.49|12349.65|4801.65|5814.00|17151.30|384.13|0.00|7718.00|4801.65|5185.78|12519.65|12903.78|-1012.35| +2450823|70353|2450839|65115|339988|1614|1640|65115|339988|1614|1640|4|20|1|2|10885|109|432|69|47.66|95.32|1.90|6445.98|131.10|3288.54|6577.08|9.17|0.00|328.44|131.10|140.27|459.54|468.71|-3157.44| +2450823|70353|2450894|65115|339988|1614|1640|65115|339988|1614|1640|4|50|5|4|17896|27|432|95|69.90|99.25|52.60|4431.75|4997.00|6640.50|9428.75|249.85|0.00|1602.65|4997.00|5246.85|6599.65|6849.50|-1643.50| +2450823|70353|2450871|65115|339988|1614|1640|65115|339988|1614|1640|4|16|6|2|11335|239|432|9|10.90|14.38|10.06|38.88|90.54|98.10|129.42|1.81|0.00|50.40|90.54|92.35|140.94|142.75|-7.56| +2450823|70353|2450865|65115|339988|1614|1640|65115|339988|1614|1640|4|29|5|3|16648|11|432|65|86.16|112.00|103.04|582.40|6697.60|5600.40|7280.00|602.78|0.00|2984.80|6697.60|7300.38|9682.40|10285.18|1097.20| +2450823|70353|2450867|65115|339988|1614|1640|65115|339988|1614|1640|4|40|6|3|4987|249|432|7|99.13|222.05|88.82|932.61|621.74|693.91|1554.35|43.52|0.00|0.00|621.74|665.26|621.74|665.26|-72.17| +2450823|70353|2450827|65115|339988|1614|1640|65115|339988|1614|1640|4|47|6|5|2048|203|432|88|3.33|4.86|3.35|132.88|294.80|293.04|427.68|2.94|0.00|166.32|294.80|297.74|461.12|464.06|1.76| +2450823|28576|2450843|20458|1831809|4760|35711|25389|232413|6164|22715|2|44|13|2|6766|267|433|33|51.87|76.24|56.41|654.39|1861.53|1711.71|2515.92|102.19|725.99|50.16|1135.54|1237.73|1185.70|1287.89|-576.17| +2450823|28576|2450856|20458|1831809|4760|35711|25389|232413|6164|22715|2|47|6|3|6652|2|433|4|28.86|46.46|22.76|94.80|91.04|115.44|185.84|2.73|0.00|7.40|91.04|93.77|98.44|101.17|-24.40| +2450823|28576|2450888|20458|1831809|4760|35711|25389|232413|6164|22715|2|68|6|2|13696|61|433|46|13.28|25.76|0.77|1149.54|35.42|610.88|1184.96|0.13|30.81|391.00|4.61|4.74|395.61|395.74|-606.27| +2450823|28576|2450892|20458|1831809|4760|35711|25389|232413|6164|22715|2|50|4|4|10394|159|433|20|27.47|80.48|66.79|273.80|1335.80|549.40|1609.60|13.35|0.00|144.80|1335.80|1349.15|1480.60|1493.95|786.40| +2450823|28576|2450868|20458|1831809|4760|35711|25389|232413|6164|22715|2|41|3|4|13888|89|433|85|36.44|61.94|48.93|1105.85|4159.05|3097.40|5264.90|166.36|0.00|1368.50|4159.05|4325.41|5527.55|5693.91|1061.65| +2450823|68801|2450868|70517|895664|2411|4311|70517|895664|2411|4311|4|77|15|3|11557|81|434|89|90.73|214.12|203.41|953.19|18103.49|8074.97|19056.68|181.03|0.00|8765.61|18103.49|18284.52|26869.10|27050.13|10028.52| +2450823|68801|2450883|70517|895664|2411|4311|70517|895664|2411|4311|4|65|8|1|3536|124|434|8|75.11|212.56|146.66|527.20|1173.28|600.88|1700.48|27.45|258.12|84.96|915.16|942.61|1000.12|1027.57|314.28| +2450823|68801|2450862|70517|895664|2411|4311|70517|895664|2411|4311|4|75|10|1|9356|58|434|58|80.00|181.60|170.70|632.20|9900.60|4640.00|10532.80|792.04|0.00|631.62|9900.60|10692.64|10532.22|11324.26|5260.60| +2450823|68801|2450842|70517|895664|2411|4311|70517|895664|2411|4311|4|62|8|3|6961|7|434|43|4.01|7.49|6.96|22.79|299.28|172.43|322.07|26.93|0.00|18.92|299.28|326.21|318.20|345.13|126.85| +2450823|68801|2450859|70517|895664|2411|4311|70517|895664|2411|4311|4|48|17|3|4399|17|434|15|17.09|33.83|24.69|137.10|370.35|256.35|507.45|25.92|0.00|197.85|370.35|396.27|568.20|594.12|114.00| +2450823|68801|2450836|70517|895664|2411|4311|70517|895664|2411|4311|4|79|16|2|10006|277|434|99|45.43|129.02|58.05|7026.03|5746.95|4497.57|12772.98|4.02|5344.66|5491.53|402.29|406.31|5893.82|5897.84|-4095.28| +2450823|68801|2450844|70517|895664|2411|4311|70517|895664|2411|4311|4|77|11|5|12313|198|434|73|60.90|71.25|25.65|3328.80|1872.45|4445.70|5201.25|0.00|74.89|363.54|1797.56|1797.56|2161.10|2161.10|-2648.14| +2450823|68801|2450865|70517|895664|2411|4311|70517|895664|2411|4311|4|55|1|5|14230|251|434|100|79.75|116.43|27.94|8849.00|2794.00|7975.00|11643.00|0.00|0.00|5821.00|2794.00|2794.00|8615.00|8615.00|-5181.00| +2450823|47986|2450831|24461|1103607|7200|23546|24461|1103607|7200|23546|2|8|18|4|9748|161|435|63|79.95|221.46|219.24|139.86|13812.12|5036.85|13951.98|276.24|0.00|3906.00|13812.12|14088.36|17718.12|17994.36|8775.27| +2450823|47986|2450858|24461|1103607|7200|23546|24461|1103607|7200|23546|2|108|5|2|4966|264|435|52|3.00|8.76|5.34|177.84|277.68|156.00|455.52|5.55|0.00|204.88|277.68|283.23|482.56|488.11|121.68| +2450823|47986|2450867|24461|1103607|7200|23546|24461|1103607|7200|23546|2|45|11|4|17557|148|435|46|22.65|63.42|12.68|2334.04|583.28|1041.90|2917.32|23.33|0.00|379.04|583.28|606.61|962.32|985.65|-458.62| +2450823|47986|2450884|24461|1103607|7200|23546|24461|1103607|7200|23546|2|98|9|1|5996|25|435|39|65.79|138.81|124.92|541.71|4871.88|2565.81|5413.59|48.71|0.00|2652.39|4871.88|4920.59|7524.27|7572.98|2306.07| +2450823|47986|2450891|24461|1103607|7200|23546|24461|1103607|7200|23546|2|55|15|4|16964|183|435|63|84.92|128.22|117.96|646.38|7431.48|5349.96|8077.86|72.08|222.94|888.30|7208.54|7280.62|8096.84|8168.92|1858.58| +2450823|47986|2450825|24461|1103607|7200|23546|24461|1103607|7200|23546|2|31|10|2|6826|139|435|53|27.64|28.74|21.55|381.07|1142.15|1464.92|1523.22|4.79|662.44|197.69|479.71|484.50|677.40|682.19|-985.21| +2450823|47986|2450847|24461|1103607|7200|23546|24461|1103607|7200|23546|2|48|10|3|7459|211|435|67|94.70|244.32|97.72|9822.20|6547.24|6344.90|16369.44|327.36|0.00|4746.95|6547.24|6874.60|11294.19|11621.55|202.34| +2450823|47986|2450868|24461|1103607|7200|23546|24461|1103607|7200|23546|2|84|9|1|1130|239|435|53|7.20|14.90|4.76|537.42|252.28|381.60|789.70|10.09|0.00|173.31|252.28|262.37|425.59|435.68|-129.32| +2450823|47986|2450859|24461|1103607|7200|23546|24461|1103607|7200|23546|2|105|2|5|9598|266|435|25|51.65|97.10|89.33|194.25|2233.25|1291.25|2427.50|156.32|0.00|558.25|2233.25|2389.57|2791.50|2947.82|942.00| +2450823|47986|2450908|24461|1103607|7200|23546|24461|1103607|7200|23546|2|71|7|5|5438|5|435|36|55.68|160.35|32.07|4618.08|1154.52|2004.48|5772.60|11.54|0.00|1616.04|1154.52|1166.06|2770.56|2782.10|-849.96| +2450823|47986|2450846|24461|1103607|7200|23546|24461|1103607|7200|23546|2|78|1|1|3859|73|435|81|52.84|103.03|13.39|7260.84|1084.59|4280.04|8345.43|16.48|260.30|2336.04|824.29|840.77|3160.33|3176.81|-3455.75| +2450823|47986|2450838|24461|1103607|7200|23546|24461|1103607|7200|23546|2|93|16|2|12895|226|435|48|45.35|72.56|36.28|1741.44|1741.44|2176.80|3482.88|156.72|0.00|1079.52|1741.44|1898.16|2820.96|2977.68|-435.36| +2450823|47986|2450902|24461|1103607|7200|23546|24461|1103607|7200|23546|2|19|5|2|9500|211|435|38|42.81|124.14|0.00|4717.32|0.00|1626.78|4717.32|0.00|0.00|188.48|0.00|0.00|188.48|188.48|-1626.78| +2450823|70793|2450865|52340|147438|2889|22740|52340|147438|2889|22740|2|34|5|4|5059|232|436|32|46.92|87.27|41.88|1452.48|1340.16|1501.44|2792.64|26.80|0.00|558.40|1340.16|1366.96|1898.56|1925.36|-161.28| +2450823|70793|2450905|52340|147438|2889|22740|52340|147438|2889|22740|2|9|2|3|13664|132|436|45|70.79|106.89|96.20|481.05|4329.00|3185.55|4810.05|346.32|0.00|47.70|4329.00|4675.32|4376.70|4723.02|1143.45| +2450823|70793|2450906|52340|147438|2889|22740|52340|147438|2889|22740|2|30|8|1|817|270|436|14|57.37|110.15|102.43|108.08|1434.02|803.18|1542.10|47.17|760.03|215.88|673.99|721.16|889.87|937.04|-129.19| +2450823|70793|2450828|52340|147438|2889|22740|52340|147438|2889|22740|2|55|6|5|7930|98|436|11|19.89|39.97|24.78|167.09|272.58|218.79|439.67|22.32|24.53|145.09|248.05|270.37|393.14|415.46|29.26| +2450823|70793|2450888|52340|147438|2889|22740|52340|147438|2889|22740|2|38|8|1|6040|132|436|63|42.90|84.08|70.62|847.98|4449.06|2702.70|5297.04|400.41|0.00|2436.21|4449.06|4849.47|6885.27|7285.68|1746.36| +2450823|70793|2450902|52340|147438|2889|22740|52340|147438|2889|22740|2|22|19|5|5962|112|436|17|78.64|125.03|110.02|255.17|1870.34|1336.88|2125.51|56.11|0.00|935.17|1870.34|1926.45|2805.51|2861.62|533.46| +2450823|70793|2450861|52340|147438|2889|22740|52340|147438|2889|22740|2|59|8|5|10924|194|436|37|27.29|55.67|54.55|41.44|2018.35|1009.73|2059.79|100.91|0.00|370.74|2018.35|2119.26|2389.09|2490.00|1008.62| +2450823|39035|2450884|74490|1259629|768|1507|74490|1259629|768|1507|1|95|17|1|14282|56|437|100|11.39|24.26|4.36|1990.00|436.00|1139.00|2426.00|4.36|0.00|679.00|436.00|440.36|1115.00|1119.36|-703.00| +2450823|39035|2450881|74490|1259629|768|1507|74490|1259629|768|1507|1|60|19|1|3770|263|437|44|33.54|94.24|85.75|373.56|3773.00|1475.76|4146.56|37.73|0.00|828.96|3773.00|3810.73|4601.96|4639.69|2297.24| +2450823|39035|2450874|74490|1259629|768|1507|74490|1259629|768|1507|1|82|13|3|15278|75|437|99|36.51|97.84|23.48|7361.64|2324.52|3614.49|9686.16|0.00|1952.59|2324.52|371.93|371.93|2696.45|2696.45|-3242.56| +2450823|39035|2450878|74490|1259629|768|1507|74490|1259629|768|1507|1|10|1|4|1339|122|437|63|9.66|15.55|6.99|539.28|440.37|608.58|979.65|35.22|0.00|293.58|440.37|475.59|733.95|769.17|-168.21| +2450823|39035|2450838|74490|1259629|768|1507|74490|1259629|768|1507|1|20|18|5|3922|129|437|64|93.50|151.47|57.55|6010.88|3683.20|5984.00|9694.08|184.16|0.00|3780.48|3683.20|3867.36|7463.68|7647.84|-2300.80| +2450823|39035|2450857|74490|1259629|768|1507|74490|1259629|768|1507|1|51|2|1|11954|189|437|46|94.54|173.00|173.00|0.00|7958.00|4348.84|7958.00|397.90|0.00|1909.92|7958.00|8355.90|9867.92|10265.82|3609.16| +2450823|39035|2450864|74490|1259629|768|1507|74490|1259629|768|1507|1|1|15|5|5030|25|437|85|93.02|228.82|171.61|4862.85|14586.85|7906.70|19449.70|1021.07|0.00|9530.20|14586.85|15607.92|24117.05|25138.12|6680.15| +2450823|39035|2450896|74490|1259629|768|1507|74490|1259629|768|1507|1|54|13|5|7156|109|437|45|8.21|20.03|2.40|793.35|108.00|369.45|901.35|4.32|0.00|441.45|108.00|112.32|549.45|553.77|-261.45| +2450823|39035|2450848|74490|1259629|768|1507|74490|1259629|768|1507|1|78|10|3|2557|230|437|10|31.62|51.22|1.53|496.90|15.30|316.20|512.20|0.00|3.82|199.70|11.48|11.48|211.18|211.18|-304.72| +2450823|39035|2450853|74490|1259629|768|1507|74490|1259629|768|1507|1|47|8|2|628|47|437|55|83.10|90.57|78.79|647.90|4333.45|4570.50|4981.35|86.66|0.00|696.85|4333.45|4420.11|5030.30|5116.96|-237.05| +2450823|39035|2450841|74490|1259629|768|1507|74490|1259629|768|1507|1|90|2|3|8101|75|437|61|3.02|6.00|3.42|157.38|208.62|184.22|366.00|6.25|0.00|40.26|208.62|214.87|248.88|255.13|24.40| +2450823|39035|2450870|74490|1259629|768|1507|74490|1259629|768|1507|1|38|3|2|13180|62|437|99|87.86|202.95|58.85|14265.90|5826.15|8698.14|20092.05|0.00|5709.62|199.98|116.53|116.53|316.51|316.51|-8581.61| +2450823||2450830||||1507|74490|1259629||1507|1|||4|8488||437||||14.65|||1183.14|||||395.55||444.96|476.60|-787.59| +2450823|68618|2450846|31455|1135084|6713|45142|31455|1135084|6713|45142|1|102|18|4|4963|4|438|28|56.84|109.70|84.46|706.72|2364.88|1591.52|3071.60|94.59|0.00|307.16|2364.88|2459.47|2672.04|2766.63|773.36| +2450823|68618|2450832|31455|1135084|6713|45142|31455|1135084|6713|45142|1|99|3|2|15805|265|438|39|19.86|39.91|6.38|1307.67|248.82|774.54|1556.49|7.46|0.00|669.24|248.82|256.28|918.06|925.52|-525.72| +2450823|68618|2450887|31455|1135084|6713|45142|31455|1135084|6713|45142|1|25|7|3|14128|48|438|27|21.60|27.43|6.30|570.51|170.10|583.20|740.61|11.90|0.00|325.62|170.10|182.00|495.72|507.62|-413.10| +2450823|68618|2450881|31455|1135084|6713|45142|31455|1135084|6713|45142|1|105|15|1|9925|218|438|34|23.44|33.28|12.64|701.76|429.76|796.96|1131.52|34.38|0.00|486.54|429.76|464.14|916.30|950.68|-367.20| +2450823|68618|2450854|31455|1135084|6713|45142|31455|1135084|6713|45142|1|20|19|5|691|225|438|74|89.14|184.51|27.67|11606.16|2047.58|6596.36|13653.74|61.42|0.00|409.22|2047.58|2109.00|2456.80|2518.22|-4548.78| +2450823|68618|2450862|31455|1135084|6713|45142|31455|1135084|6713|45142|1|55|12|4|17786|124|438|47|75.85|109.22|28.39|3799.01|1334.33|3564.95|5133.34|40.02|0.00|0.00|1334.33|1374.35|1334.33|1374.35|-2230.62| +2450823|68618|2450904|31455|1135084|6713|45142|31455|1135084|6713|45142|1|79|6|1|16094|110|438|73|67.00|71.69|36.56|2564.49|2668.88|4891.00|5233.37|87.54|1574.63|1883.40|1094.25|1181.79|2977.65|3065.19|-3796.75| +2450823|23168|2450898|55796|54720|6091|3015|55796|54720|6091|3015|1|85|19|3|5032|52|439|49|40.02|120.06|85.24|1706.18|4176.76|1960.98|5882.94|125.30|0.00|882.00|4176.76|4302.06|5058.76|5184.06|2215.78| +2450823|23168|2450885|55796|54720|6091|3015|55796|54720|6091|3015|1|50|16|2|5497|126|439|88|51.67|62.00|29.14|2891.68|2564.32|4546.96|5456.00|0.00|641.08|982.08|1923.24|1923.24|2905.32|2905.32|-2623.72| +2450823|23168|2450906|55796|54720|6091|3015|55796|54720|6091|3015|1|14|9|1|9832|190|439|48|40.06|54.08|2.16|2492.16|103.68|1922.88|2595.84|0.00|0.00|648.96|103.68|103.68|752.64|752.64|-1819.20| +2450823|23168|2450849|55796|54720|6091|3015|55796|54720|6091|3015|1|85|9|2|15554|125|439|49|5.08|10.66|10.23|21.07|501.27|248.92|522.34|0.40|496.25|46.55|5.02|5.42|51.57|51.97|-243.90| +2450823|23168|2450866|55796|54720|6091|3015|55796|54720|6091|3015|1|51|3|4|4454|209|439|83|51.77|51.77|48.14|301.29|3995.62|4296.91|4296.91|159.82|0.00|1074.02|3995.62|4155.44|5069.64|5229.46|-301.29| +2450823|23168|2450848|55796|54720|6091|3015|55796|54720|6091|3015|1|14|12|1|6808|109|439|94|84.92|136.72|127.14|900.52|11951.16|7982.48|12851.68|956.09|0.00|5911.66|11951.16|12907.25|17862.82|18818.91|3968.68| +2450823|23168|2450827|55796|54720|6091|3015|55796|54720|6091|3015|1|77|14|3|12760|269|439|81|94.69|134.45|106.21|2287.44|8603.01|7669.89|10890.45|30.97|7054.46|1197.18|1548.55|1579.52|2745.73|2776.70|-6121.34| +2450823|23168|2450831|55796|54720|6091|3015|55796|54720|6091|3015|1|53|17|3|10520|93|439|26|84.49|125.04|86.27|1008.02|2243.02|2196.74|3251.04|201.87|0.00|910.26|2243.02|2444.89|3153.28|3355.15|46.28| +2450823|23168|2450867|55796|54720|6091|3015|55796|54720|6091|3015|1|16|13|4|7394|217|439|91|64.37|156.41|6.25|13664.56|568.75|5857.67|14233.31|39.81|0.00|2418.78|568.75|608.56|2987.53|3027.34|-5288.92| +2450823|23168|2450875|55796|54720|6091|3015|55796|54720|6091|3015|1|5|19|4|14707|54|439|86|35.28|50.45|29.26|1822.34|2516.36|3034.08|4338.70|0.00|0.00|910.74|2516.36|2516.36|3427.10|3427.10|-517.72| +2450823|23168|2450830|55796|54720|6091|3015|55796|54720|6091|3015|1|8|20|3|4573|10|439|7|46.41|116.48|67.55|342.51|472.85|324.87|815.36|28.37|0.00|228.27|472.85|501.22|701.12|729.49|147.98| +2450823|23168|2450860|55796|54720|6091|3015|55796|54720|6091|3015|1|70|14|2|9172|145|439|65|58.72|128.59|39.86|5767.45|2590.90|3816.80|8358.35|119.70|595.90|1838.20|1995.00|2114.70|3833.20|3952.90|-1821.80| +2450823|68181|2450871|45588|780770|5199|8381|45588|780770|5199|8381|1|75|6|3|15964|169|440|59|20.01|48.82|48.33|28.91|2851.47|1180.59|2880.38|0.00|0.00|690.89|2851.47|2851.47|3542.36|3542.36|1670.88| +2450823|68181|2450827|45588|780770|5199|8381|45588|780770|5199|8381|1|33|14|2|6314|90|440|53|15.62|36.55|17.90|988.45|948.70|827.86|1937.15|1.70|920.23|329.13|28.47|30.17|357.60|359.30|-799.39| +2450823|68181|2450844|45588|780770|5199|8381|45588|780770|5199|8381|1|64|15|3|7294|37|440|69|26.93|60.05|24.62|2444.67|1698.78|1858.17|4143.45|50.96|0.00|1366.89|1698.78|1749.74|3065.67|3116.63|-159.39| +2450823|68181|2450856|45588|780770|5199|8381|45588|780770|5199|8381|1|16|12|4|17600|268|440|7|4.30|8.55|0.42|56.91|2.94|30.10|59.85|0.20|0.00|25.69|2.94|3.14|28.63|28.83|-27.16| +|68181|2450868|45588|780770|5199||45588|780770|||1||7||2653||440||31.39||||5284.00|3139.00|8663.00|211.36|0.00|||5495.36|8835.00|9046.36|2145.00| +2450823|68181|2450859|45588|780770|5199|8381|45588|780770|5199|8381|1|14|15|2|1606|258|440|81|65.70|190.53|129.56|4938.57|10494.36|5321.70|15432.93|0.00|0.00|1388.34|10494.36|10494.36|11882.70|11882.70|5172.66| +||2450825|45588|780770|5199|8381||780770|||1||4||5912||440|54|34.04|||4383.18||1838.16||||||239.80|1475.82|1485.04|-1607.58| +2450823|68181|2450894|45588|780770|5199|8381|45588|780770|5199|8381|1|104|18|5|11504|82|440|87|42.68|52.06|42.16|861.30|3667.92|3713.16|4529.22|73.35|0.00|1720.86|3667.92|3741.27|5388.78|5462.13|-45.24| +2450823|68181|2450887|45588|780770|5199|8381|45588|780770|5199|8381|1|35|5|3|9778|238|440|72|26.69|76.60|53.62|1654.56|3860.64|1921.68|5515.20|116.20|2200.56|110.16|1660.08|1776.28|1770.24|1886.44|-261.60| +2450823|68181|2450891|45588|780770|5199|8381|45588|780770|5199|8381|1|65|9|5|1291|38|440|55|95.35|151.60|121.28|1667.60|6670.40|5244.25|8338.00|66.70|0.00|3752.10|6670.40|6737.10|10422.50|10489.20|1426.15| +2450823|17274|2450845|49674|420206|5938|26423|49674|420206|5938|26423|4|82|12|5|7183|106|441|30|73.69|87.69|78.04|289.50|2341.20|2210.70|2630.70|140.47|0.00|789.00|2341.20|2481.67|3130.20|3270.67|130.50| +2450823|17274|2450827|49674|420206|5938|26423|49674|420206|5938|26423|4|75|11|4|403|252|441|73|16.97|41.06|13.13|2038.89|958.49|1238.81|2997.38|38.33|0.00|838.77|958.49|996.82|1797.26|1835.59|-280.32| +2450823|17274|2450849|49674|420206|5938|26423|49674|420206|5938|26423|4|63|12|1|4724|222|441|87|32.55|94.06|32.92|5319.18|2864.04|2831.85|8183.22|85.92|0.00|981.36|2864.04|2949.96|3845.40|3931.32|32.19| +2450823|17274|2450888|49674|420206|5938|26423|49674|420206|5938|26423|4|55|13|4|634|289|441|27|76.46|176.62|33.55|3862.89|905.85|2064.42|4768.74|54.35|0.00|0.00|905.85|960.20|905.85|960.20|-1158.57| +2450823|17274|2450836|49674|420206|5938|26423|49674|420206|5938|26423|4|50|7|4|10228|296|441|68|57.02|72.41|60.10|837.08|4086.80|3877.36|4923.88|25.33|2819.89|393.72|1266.91|1292.24|1660.63|1685.96|-2610.45| +2450823|17274|2450840|49674|420206|5938|26423|49674|420206|5938|26423|4|29|12|1|4069|156|441|10|37.10|83.10|2.49|806.10|24.90|371.00|831.00|0.49|0.00|349.00|24.90|25.39|373.90|374.39|-346.10| +2450823|17274|2450840|49674|420206|5938|26423|49674|420206|5938|26423|4|85|5|1|7144|179|441|62|40.77|85.61|66.77|1168.08|4139.74|2527.74|5307.82|82.79|0.00|1167.46|4139.74|4222.53|5307.20|5389.99|1612.00| +2450823|17274|2450854|49674|420206|5938|26423|49674|420206|5938|26423|4|79|16|5|11686|272|441|76|50.77|132.50|108.65|1812.60|8257.40|3858.52|10070.00|412.87|0.00|2416.80|8257.40|8670.27|10674.20|11087.07|4398.88| +2450823|17274|2450838|49674|420206|5938|26423|49674|420206|5938|26423|4|78|14|1|9382|110|441|6|63.25|99.93|43.96|335.82|263.76|379.50|599.58|7.91|0.00|293.76|263.76|271.67|557.52|565.43|-115.74| +2450823|17274|2450900|49674|420206|5938|26423|49674|420206|5938|26423|4|19|10|3|10615|189|441|27|32.17|52.75|13.18|1068.39|355.86|868.59|1424.25|28.46|0.00|427.14|355.86|384.32|783.00|811.46|-512.73| +2450823|17274|2450860|49674|420206|5938|26423|49674|420206|5938|26423|4|49|18|3|10267|195|441|83|71.86|84.79|40.69|3660.30|3377.27|5964.38|7037.57|236.40|0.00|3025.35|3377.27|3613.67|6402.62|6639.02|-2587.11| +2450823|17274|2450838|49674|420206|5938|26423|49674|420206|5938|26423|4|70|16|1|7850|76|441|22|6.11|11.36|11.36|0.00|249.92|134.42|249.92|12.49|0.00|39.82|249.92|262.41|289.74|302.23|115.50| +2450823|17274|2450828|49674|420206|5938|26423|49674|420206|5938|26423|4|104|6|2|10346|126|441|74|52.84|68.69|37.09|2338.40|2744.66|3910.16|5083.06|219.57|0.00|863.58|2744.66|2964.23|3608.24|3827.81|-1165.50| +2450823|17274|2450876|49674|420206|5938|26423|49674|420206|5938|26423|4|91|3|3|11894|258|441|37|33.64|58.53|4.09|2014.28|151.33|1244.68|2165.61|3.02|0.00|519.48|151.33|154.35|670.81|673.83|-1093.35| +2450823|59408|2450835|61705|1379797|6689|48028|61705|1379797|6689|48028|1|67|1|1|5852|66|442|43|86.33|108.77|36.98|3086.97|1590.14|3712.19|4677.11|47.70|0.00|888.38|1590.14|1637.84|2478.52|2526.22|-2122.05| +2450823|59408|2450868|61705|1379797|6689|48028|61705|1379797|6689|48028|1|75|1|1|14749|68|442|23|55.10|63.91|16.61|1087.90|382.03|1267.30|1469.93|19.10|0.00|558.44|382.03|401.13|940.47|959.57|-885.27| +2450823|59408|2450908|61705|1379797|6689|48028|61705|1379797|6689|48028|1|46|19|1|14977|55|442|97|87.39|219.34|111.86|10425.56|10850.42|8476.83|21275.98|434.01|0.00|1914.78|10850.42|11284.43|12765.20|13199.21|2373.59| +2450823|59408|2450866|61705|1379797|6689|48028|61705|1379797|6689|48028|1|74|17|5|301|258|442|15|99.47|226.79|49.89|2653.50|748.35|1492.05|3401.85|49.24|44.90|1088.55|703.45|752.69|1792.00|1841.24|-788.60| +2450823|59408|2450835|61705|1379797|6689|48028|61705|1379797|6689|48028|1|16|15|3|12523|49|442|82|87.61|110.38|22.07|7241.42|1809.74|7184.02|9051.16|18.09|0.00|451.82|1809.74|1827.83|2261.56|2279.65|-5374.28| +2450823|59408|2450908|61705|1379797|6689|48028|61705|1379797|6689|48028|1|33|18|5|17575|296|442|35|37.25|68.54|27.41|1439.55|959.35|1303.75|2398.90|28.78|0.00|911.40|959.35|988.13|1870.75|1899.53|-344.40| +2450823|59408|2450849|61705|1379797|6689|48028|61705|1379797|6689|48028|1|41|13|5|17392|62|442|49|23.81|70.95|9.93|2989.98|486.57|1166.69|3476.55|21.79|214.09|1042.72|272.48|294.27|1315.20|1336.99|-894.21| +2450823|59408|2450826|61705|1379797|6689|48028|61705|1379797|6689|48028|1|100|2|4|7744|31|442|42|13.67|24.87|14.17|449.40|595.14|574.14|1044.54|19.99|309.47|125.16|285.67|305.66|410.83|430.82|-288.47| +2450823|59408|2450842|61705|1379797|6689|48028|61705|1379797|6689|48028|1|4|17|5|15007|202|442|42|2.74|4.90|4.75|6.30|199.50|115.08|205.80|5.98|0.00|14.28|199.50|205.48|213.78|219.76|84.42| +2450823|59408|2450884|61705|1379797|6689|48028|61705|1379797|6689|48028|1|32|1|3|12266|109|442|7|28.89|40.44|10.11|212.31|70.77|202.23|283.08|2.12|0.00|138.67|70.77|72.89|209.44|211.56|-131.46| +2450823|59408|2450843|61705|1379797|6689|48028|61705|1379797|6689|48028|1|85|6|4|3475|53|442|30|92.89|195.99|188.15|235.20|5644.50|2786.70|5879.70|56.44|0.00|1822.50|5644.50|5700.94|7467.00|7523.44|2857.80| +2450823|50202|2450865|15042|1826724|1959|46794|15042|1826724|1959|46794|2|10|2|1|14606|261|443|91|76.36|92.39|24.94|6137.95|2269.54|6948.76|8407.49|22.46|22.69|3866.59|2246.85|2269.31|6113.44|6135.90|-4701.91| +2450823|50202|2450854|15042|1826724|1959|46794|15042|1826724|1959|46794|2|4|11|2|11348|294|443|19|18.77|23.83|1.42|425.79|26.98|356.63|452.77|1.61|0.00|185.63|26.98|28.59|212.61|214.22|-329.65| +2450823|50202|2450913|15042|1826724|1959|46794|15042|1826724|1959|46794|2|69|18|4|4892|79|443|48|41.66|86.65|6.93|3826.56|332.64|1999.68|4159.20|19.95|0.00|582.24|332.64|352.59|914.88|934.83|-1667.04| +2450823|50202|2450896|15042|1826724|1959|46794|15042|1826724|1959|46794|2|34|6|2|14062|259|443|100|95.88|121.76|41.39|8037.00|4139.00|9588.00|12176.00|11.58|3849.27|852.00|289.73|301.31|1141.73|1153.31|-9298.27| +2450823|50202|2450868|15042|1826724|1959|46794|15042|1826724|1959|46794|2|60|1|2|7424|214|443|70|51.98|85.24|26.42|4117.40|1849.40|3638.60|5966.80|147.95|0.00|1789.90|1849.40|1997.35|3639.30|3787.25|-1789.20| +2450823|50202|2450829|15042|1826724|1959|46794|15042|1826724|1959|46794|2|75|1|4|7351|13|443|76|83.57|119.50|7.17|8537.08|544.92|6351.32|9082.00|10.89|0.00|2179.68|544.92|555.81|2724.60|2735.49|-5806.40| +2450823|63023|2450908|89254|1469988|4322|25103|89254|1469988|4322|25103|1|5|6|3|16048|149|444|62|32.45|83.07|48.18|2163.18|2987.16|2011.90|5150.34|179.22|0.00|1596.50|2987.16|3166.38|4583.66|4762.88|975.26| +2450823|63023|2450864|89254|1469988|4322|25103|89254|1469988|4322|25103|1|18|12|5|10366|183|444|95|69.62|89.11|18.71|6688.00|1777.45|6613.90|8465.45|106.64|0.00|2454.80|1777.45|1884.09|4232.25|4338.89|-4836.45| +2450823|63023|2450874|89254|1469988|4322|25103|89254|1469988|4322|25103|1|28|10|2|14203|188|444|100|88.97|200.18|200.18|0.00|20018.00|8897.00|20018.00|276.24|10809.72|6806.00|9208.28|9484.52|16014.28|16290.52|311.28| +2450823|63023|2450837|89254|1469988|4322|25103|89254|1469988|4322|25103|1|107|12|3|13460|273|444|77|80.85|104.29|23.98|6183.87|1846.46|6225.45|8030.33|0.00|36.92|1043.35|1809.54|1809.54|2852.89|2852.89|-4415.91| +2450823|60183|2450850|90747|846956|318|30880|905|361651|1860|6822|4|100|13|2|4117|43|445|90|94.24|203.55|101.77|9160.20|9159.30|8481.60|18319.50|100.75|4121.68|3296.70|5037.62|5138.37|8334.32|8435.07|-3443.98| +2450823|60183|2450879|90747|846956|318|30880|905|361651|1860|6822|4|21|20|5|15590|33|445|73|4.03|6.16|4.74|103.66|346.02|294.19|449.68|18.65|79.58|26.28|266.44|285.09|292.72|311.37|-27.75| +2450823|60183|2450826|90747|846956|318|30880|905|361651|1860|6822|4|34|2|4|17908|273|445|63|97.85|202.54|131.65|4466.07|8293.95|6164.55|12760.02|663.51|0.00|0.00|8293.95|8957.46|8293.95|8957.46|2129.40| +2450823|60183|2450908|90747|846956|318|30880|905|361651|1860|6822|4|93|15|5|17588|25|445|33|16.77|48.29|21.73|876.48|717.09|553.41|1593.57|34.42|28.68|159.06|688.41|722.83|847.47|881.89|135.00| +2450823|60183|2450832|90747|846956|318|30880|905|361651|1860|6822|4|22|13|2|15919|160|445|42|19.03|20.74|0.41|853.86|17.22|799.26|871.08|1.54|0.00|86.94|17.22|18.76|104.16|105.70|-782.04| +2450823|60183|2450864|90747|846956|318|30880|905|361651|1860|6822|4|44|14|1|16975|272|445|94|4.57|6.80|4.21|243.46|395.74|429.58|639.20|9.10|213.69|281.06|182.05|191.15|463.11|472.21|-247.53| +2450823|54289|2450846|72926|1603018|5557|49249|72926|1603018|5557|49249|1|24|12|2|11269|209|446|25|51.06|64.84|8.42|1410.50|210.50|1276.50|1621.00|6.31|0.00|486.25|210.50|216.81|696.75|703.06|-1066.00| +2450823|54289|2450895|72926|1603018|5557|49249|72926|1603018|5557|49249|1|84|4|1|15817|122|446|97|23.06|31.59|17.69|1348.30|1715.93|2236.82|3064.23|154.43|0.00|918.59|1715.93|1870.36|2634.52|2788.95|-520.89| +2450823|54289|2450846|72926|1603018|5557|49249|72926|1603018|5557|49249|1|90|13|5|17332|197|446|61|58.10|155.70|79.40|4654.30|4843.40|3544.10|9497.70|17.43|3099.77|4083.95|1743.63|1761.06|5827.58|5845.01|-1800.47| +2450823|54289|2450884|72926|1603018|5557|49249|72926|1603018|5557|49249|1|85|9|3|16370|121|446|89|74.13|85.24|36.65|4324.51|3261.85|6597.57|7586.36|18.59|1402.59|3261.85|1859.26|1877.85|5121.11|5139.70|-4738.31| +2450823|54289|2450874|72926|1603018|5557|49249|72926|1603018|5557|49249|1|16|12|2|4490|277|446|99|91.89|125.88|35.24|8973.36|3488.76|9097.11|12462.12|34.88|0.00|2866.05|3488.76|3523.64|6354.81|6389.69|-5608.35| +2450823|54289|2450836|72926|1603018|5557|49249|72926|1603018|5557|49249|1|39|16|1|10318|1|446|79|33.21|64.09|8.33|4405.04|658.07|2623.59|5063.11|13.16|0.00|252.80|658.07|671.23|910.87|924.03|-1965.52| +2450823|54289|2450896|72926|1603018|5557|49249|72926|1603018|5557|49249|1|107|14|5|9254|162|446|23|29.18|85.20|78.38|156.86|1802.74|671.14|1959.60|54.08|0.00|783.84|1802.74|1856.82|2586.58|2640.66|1131.60| +2450823|82900|2450833|43421|196410|4952|13094|43421|196410|4952|13094|2|5|13|1|14408|277|447|32|81.88|203.06|160.41|1364.80|5133.12|2620.16|6497.92|153.99|0.00|129.92|5133.12|5287.11|5263.04|5417.03|2512.96| +2450823|82900|2450867|43421|196410|4952|13094|43421|196410|4952|13094|2|102|8|4|15506|128|447|74|44.41|78.60|41.65|2734.30|3082.10|3286.34|5816.40|123.28|0.00|2558.92|3082.10|3205.38|5641.02|5764.30|-204.24| +2450823|82900|2450890|43421|196410|4952|13094|43421|196410|4952|13094|2|53|4|4|12184|36|447|23|26.39|62.54|37.52|575.46|862.96|606.97|1438.42|17.25|0.00|575.23|862.96|880.21|1438.19|1455.44|255.99| +2450823|82900|2450861|43421|196410|4952|13094|43421|196410|4952|13094|2|33|7|5|3217|176|447|67|53.93|138.06|63.50|4995.52|4254.50|3613.31|9250.02|170.18|0.00|3514.82|4254.50|4424.68|7769.32|7939.50|641.19| +2450823|82900|2450834|43421|196410|4952|13094|43421|196410|4952|13094|2|93|13|5|4087|109|447|44|20.87|61.98|55.78|272.80|2454.32|918.28|2727.12|98.17|0.00|708.84|2454.32|2552.49|3163.16|3261.33|1536.04| +2450823|82900|2450892|43421|196410|4952|13094|43421|196410|4952|13094|2|61|19|3|866|219|447|81|66.09|180.42|135.31|3653.91|10960.11|5353.29|14614.02|657.60|0.00|6867.99|10960.11|11617.71|17828.10|18485.70|5606.82| +2450823|82900|2450885|43421|196410|4952|13094|43421|196410|4952|13094|2|24|10|3|52|239|447|37|93.23|165.94|96.24|2578.90|3560.88|3449.51|6139.78|320.47|0.00|859.51|3560.88|3881.35|4420.39|4740.86|111.37| +2450823|82900|2450858|43421|196410|4952|13094|43421|196410|4952|13094|2|91|6|4|4363|178|447|70|76.54|218.90|218.90|0.00|15323.00|5357.80|15323.00|1225.84|0.00|6129.20|15323.00|16548.84|21452.20|22678.04|9965.20| +2450823|39358|2450858|42337|1020263|236|10562|42337|1020263|236|10562|2|59|12|2|4201|5|448|17|29.63|44.44|33.33|188.87|566.61|503.71|755.48|33.99|0.00|52.87|566.61|600.60|619.48|653.47|62.90| +2450823|39358|2450852|42337|1020263|236|10562|42337|1020263|236|10562|2|43|7|1|1070|24|448|66|46.89|123.78|122.54|81.84|8087.64|3094.74|8169.48|0.00|0.00|2532.42|8087.64|8087.64|10620.06|10620.06|4992.90| +2450823|39358|2450892|42337|1020263|236|10562|42337|1020263|236|10562|2|12|15|2|722|144|448|32|78.19|163.41|55.55|3451.52|1777.60|2502.08|5229.12|88.88|0.00|52.16|1777.60|1866.48|1829.76|1918.64|-724.48| +2450823|39358|2450896|42337|1020263|236|10562|42337|1020263|236|10562|2|24|10|3|2047|111|448|5|48.80|133.22|14.65|592.85|73.25|244.00|666.10|0.00|0.00|93.25|73.25|73.25|166.50|166.50|-170.75| +2450823|39358|2450898|42337|1020263|236|10562|42337|1020263|236|10562|2|107|4|3|2600|241|448|99|85.96|232.09|155.50|7582.41|15394.50|8510.04|22976.91|104.68|10160.37|2757.15|5234.13|5338.81|7991.28|8095.96|-3275.91| +2450823|39358|2450838|42337|1020263|236|10562|42337|1020263|236|10562|2|44|10|5|6568|248|448|4|70.52|96.61|22.22|297.56|88.88|282.08|386.44|0.00|0.00|3.84|88.88|88.88|92.72|92.72|-193.20| +2450823|39358|2450850|42337|1020263|236|10562|42337|1020263|236|10562|2|40|2|1|157|296|448|89|99.29|265.10|167.01|8730.01|14863.89|8836.81|23593.90|148.63|0.00|5426.33|14863.89|15012.52|20290.22|20438.85|6027.08| +2450823|39358|2450898|42337|1020263|236|10562|42337|1020263|236|10562|2|84|8|3|14708|60|448|6|92.08|165.74|102.75|377.94|616.50|552.48|994.44|12.33|0.00|109.38|616.50|628.83|725.88|738.21|64.02| +2450823|39358|2450842|42337|1020263|236|10562|42337|1020263|236|10562|2|91|4|3|6260|266|448|37|48.52|118.87|93.90|923.89|3474.30|1795.24|4398.19|104.22|0.00|1231.36|3474.30|3578.52|4705.66|4809.88|1679.06| +2450823|39358|2450903|42337|1020263|236|10562|42337|1020263|236|10562|2|55|19|2|12170|96|448|13|26.43|63.16|17.05|599.43|221.65|343.59|821.08|2.21|0.00|139.49|221.65|223.86|361.14|363.35|-121.94| +2450823|39358|2450895|42337|1020263|236|10562|42337|1020263|236|10562|2|28|5|4|14290|254|448|10|35.16|66.10|54.20|119.00|542.00|351.60|661.00|16.26|0.00|290.80|542.00|558.26|832.80|849.06|190.40| +2450823|39358|2450892|42337|1020263|236|10562|42337|1020263|236|10562|2|95|19|2|8281|81|448|49|81.14|125.76|60.36|3204.60|2957.64|3975.86|6162.24|207.03|0.00|862.40|2957.64|3164.67|3820.04|4027.07|-1018.22| +2450823|62146|2450880|41264|1067447|6877|43507|41264|1067447|6877|43507|2|103|6|4|7141|278|449|64|68.71|179.33|112.97|4247.04|7230.08|4397.44|11477.12|650.70|0.00|5049.60|7230.08|7880.78|12279.68|12930.38|2832.64| +2450823|62146|2450828|41264|1067447|6877|43507|41264|1067447|6877|43507|2|54|7|4|14365|216|449|47|8.35|17.03|13.45|168.26|632.15|392.45|800.41|25.28|0.00|7.99|632.15|657.43|640.14|665.42|239.70| +2450823|62146|2450829|41264|1067447|6877|43507|41264|1067447|6877|43507|2|65|12|1|4741|252|449|13|56.28|90.61|28.08|812.89|365.04|731.64|1177.93|32.85|0.00|259.09|365.04|397.89|624.13|656.98|-366.60| +2450823|62146|2450856|41264|1067447|6877|43507|41264|1067447|6877|43507|2|1|4|3|13849|51|449|62|90.67|106.08|102.89|197.78|6379.18|5621.54|6576.96|6.37|6251.59|1380.74|127.59|133.96|1508.33|1514.70|-5493.95| +2450823|68579|2450847|18799|878735|6671|17054|18799|878735|6671|17054|1|7|9|3|10702|191|450|41|87.60|209.36|25.12|7553.84|1029.92|3591.60|8583.76|61.79|0.00|4120.09|1029.92|1091.71|5150.01|5211.80|-2561.68| +2450823|68579|2450894|18799|878735|6671|17054|18799|878735|6671|17054|1|18|2|4|8012|231|450|44|71.14|165.75|4.97|7074.32|218.68|3130.16|7293.00|8.74|0.00|583.44|218.68|227.42|802.12|810.86|-2911.48| +2450823|68579|2450910|18799|878735|6671|17054|18799|878735|6671|17054|1|55|3|4|4633|23|450|6|30.28|88.41|83.98|26.58|503.88|181.68|530.46|5.03|0.00|21.18|503.88|508.91|525.06|530.09|322.20| +2450823|68579|2450850|18799|878735|6671|17054|18799|878735|6671|17054|1|28|8|2|4916|48|450|90|48.11|136.63|9.56|11436.30|860.40|4329.90|12296.70|14.45|499.03|6147.90|361.37|375.82|6509.27|6523.72|-3968.53| +2450823|68579|2450871|18799|878735|6671|17054|18799|878735|6671|17054|1|54|11|1|9346|20|450|63|88.61|153.29|1.53|9560.88|96.39|5582.43|9657.27|1.21|79.03|2027.97|17.36|18.57|2045.33|2046.54|-5565.07| +2450823|76025|2450849|88458|368476|2094|26159|88458|368476|2094|26159|1|29|6|2|5944|169|451|6|76.99|130.88|11.77|714.66|70.62|461.94|785.28|4.23|0.00|337.62|70.62|74.85|408.24|412.47|-391.32| +2450823|76025|2450883|88458|368476|2094|26159|88458|368476|2094|26159|1|61|9|5|9916|115|451|54|2.75|3.46|1.90|84.24|102.60|148.50|186.84|0.00|0.00|82.08|102.60|102.60|184.68|184.68|-45.90| +2450823|76025|2450897|88458|368476|2094|26159|88458|368476|2094|26159|1|22|7|1|5156|298|451|57|97.08|110.67|109.56|63.27|6244.92|5533.56|6308.19|0.00|1498.78|2018.37|4746.14|4746.14|6764.51|6764.51|-787.42| +2450823|76025|2450878|88458|368476|2094|26159|88458|368476|2094|26159|1|16|8|2|8774|83|451|17|33.07|41.33|19.42|372.47|330.14|562.19|702.61|6.60|0.00|126.31|330.14|336.74|456.45|463.05|-232.05| +2450823|27039|2450839|17394|635181|5321|31108|17394|635181|5321|31108|4|105|1|2|12212|289|452|60|42.24|97.57|27.31|4215.60|1638.60|2534.40|5854.20|114.70|0.00|409.20|1638.60|1753.30|2047.80|2162.50|-895.80| +2450823|27039|2450842|17394|635181|5321|31108|17394|635181|5321|31108|4|86|13|3|14407|228|452|21|63.74|105.17|13.67|1921.50|287.07|1338.54|2208.57|8.61|0.00|287.07|287.07|295.68|574.14|582.75|-1051.47| +2450823|27039|2450829|17394|635181|5321|31108|17394|635181|5321|31108|4|75|8|4|8360|41|452|99|4.13|11.68|1.86|972.18|184.14|408.87|1156.32|7.36|0.00|149.49|184.14|191.50|333.63|340.99|-224.73| +2450823|27039|2450852|17394|635181|5321|31108|17394|635181|5321|31108|4|70|18|5|10795|174|452|22|77.27|215.58|68.98|3225.20|1517.56|1699.94|4742.76|30.35|0.00|379.28|1517.56|1547.91|1896.84|1927.19|-182.38| +2450823|27039|2450838|17394|635181|5321|31108|17394|635181|5321|31108|4|100|7|2|8068|80|452|19|16.32|36.72|31.21|104.69|592.99|310.08|697.68|17.78|0.00|62.70|592.99|610.77|655.69|673.47|282.91| +2450823|27039|2450845|17394|635181|5321|31108|17394|635181|5321|31108|4|8|14|4|13939|116|452|5|87.08|181.99|180.17|9.10|900.85|435.40|909.95|45.04|0.00|454.95|900.85|945.89|1355.80|1400.84|465.45| +2450823|27039|2450873|17394|635181|5321|31108|17394|635181|5321|31108|4|101|6|4|1567|267|452|37|48.65|53.02|40.82|451.40|1510.34|1800.05|1961.74|75.51|0.00|902.06|1510.34|1585.85|2412.40|2487.91|-289.71| +2450823|27039|2450901|17394|635181|5321|31108|17394|635181|5321|31108|4|57|15|3|1528|136|452|95|21.76|53.31|19.72|3191.05|1873.40|2067.20|5064.45|74.93|0.00|151.05|1873.40|1948.33|2024.45|2099.38|-193.80| +2450823|30597|2450897|49072|1847667|3862|42810|49072|1847667|3862|42810|2|15|20|2|15482|68|453|67|23.89|40.13|37.32|188.27|2500.44|1600.63|2688.71|25.00|0.00|375.87|2500.44|2525.44|2876.31|2901.31|899.81| +2450823|30597|2450837|49072|1847667|3862|42810|49072|1847667|3862|42810|2|43|19|1|9835|102|453|19|87.97|131.95|46.18|1629.63|877.42|1671.43|2507.05|17.54|0.00|952.66|877.42|894.96|1830.08|1847.62|-794.01| +2450823|30597|2450888|49072|1847667|3862|42810|49072|1847667|3862|42810|2|92|16|4|16436|58|453|36|65.94|123.96|112.80|401.76|4060.80|2373.84|4462.56|0.00|0.00|133.56|4060.80|4060.80|4194.36|4194.36|1686.96| +2450823|30597|2450890|49072|1847667|3862|42810|49072|1847667|3862|42810|2|74|11|4|14941|92|453|95|43.71|82.61|50.39|3060.90|4787.05|4152.45|7847.95|134.99|287.22|1647.30|4499.83|4634.82|6147.13|6282.12|347.38| +2450823|26073|2450832|34947|1760064|734|11004|34947|1760064|734|11004|4|38|5|3|5288|55|454|72|3.22|5.79|4.45|96.48|320.40|231.84|416.88|22.42|0.00|79.20|320.40|342.82|399.60|422.02|88.56| +2450823|26073|2450872|34947|1760064|734|11004|34947|1760064|734|11004|4|37|20|4|16772|215|454|51|38.75|39.13|14.08|1277.55|718.08|1976.25|1995.63|5.74|430.84|418.71|287.24|292.98|705.95|711.69|-1689.01| +2450823|26073|2450861|34947|1760064|734|11004|34947|1760064|734|11004|4|93|16|5|8758|60|454|100|92.84|200.53|128.33|7220.00|12833.00|9284.00|20053.00|1026.64|0.00|7018.00|12833.00|13859.64|19851.00|20877.64|3549.00| +2450823|26073|2450845|34947|1760064|734|11004|34947|1760064|734|11004|4|59|10|4|4201|290|454|20|44.43|82.19|37.80|887.80|756.00|888.60|1643.80|60.48|0.00|131.40|756.00|816.48|887.40|947.88|-132.60| +2450823|26073|2450882|34947|1760064|734|11004|34947|1760064|734|11004|4|97|13|5|1070|39|454|45|1.43|3.76|3.12|28.80|140.40|64.35|169.20|5.30|64.58|67.50|75.82|81.12|143.32|148.62|11.47| +2450823|26073|2450838|34947|1760064|734|11004|34947|1760064|734|11004|4|58|3|4|722|236|454|56|30.09|57.17|44.59|704.48|2497.04|1685.04|3201.52|174.79|0.00|1568.56|2497.04|2671.83|4065.60|4240.39|812.00| +2450823|26073|2450871|34947|1760064|734|11004|34947|1760064|734|11004|4|28|5|5|2047|63|454|69|30.47|77.08|59.35|1223.37|4095.15|2102.43|5318.52|163.80|0.00|2127.27|4095.15|4258.95|6222.42|6386.22|1992.72| +2450823|26073|2450828|34947|1760064|734|11004|34947|1760064|734|11004|4|17|19|2|2600|64|454|11|67.34|74.74|43.34|345.40|476.74|740.74|822.14|11.25|195.46|386.32|281.28|292.53|667.60|678.85|-459.46| +2450823|26073|2450910|34947|1760064|734|11004|34947|1760064|734|11004|4|49|19|4|6568|64|454|47|49.45|80.10|52.86|1280.28|2484.42|2324.15|3764.70|24.84|0.00|112.80|2484.42|2509.26|2597.22|2622.06|160.27| +2450823|26073|2450842|34947|1760064|734|11004|34947|1760064|734|11004|4|58|11|5|157|49|454|42|52.87|125.30|12.53|4736.34|526.26|2220.54|5262.60|31.57|0.00|1947.12|526.26|557.83|2473.38|2504.95|-1694.28| +2450823|26073|2450843|34947|1760064|734|11004|34947|1760064|734|11004|4|38|14|2|14708|187|454|48|19.63|27.87|7.24|990.24|347.52|942.24|1337.76|3.47|0.00|641.76|347.52|350.99|989.28|992.75|-594.72| +2450823|27143|2450856|99171|88549|3880|458|70581|1190414|145|46723|4|40|2|4|12940|179|455|30|95.24|207.62|176.47|934.50|5294.10|2857.20|6228.60|264.70|0.00|747.30|5294.10|5558.80|6041.40|6306.10|2436.90| +2450823|27143|2450911|99171|88549|3880|458|70581|1190414|145|46723|4|84|7|3|2212|188|455|9|6.84|13.06|9.40|32.94|84.60|61.56|117.54|4.23|0.00|15.21|84.60|88.83|99.81|104.04|23.04| +2450823|27143|2450872|99171|88549|3880|458|70581|1190414|145|46723|4|85|18|3|13828|183|455|96|89.86|127.60|109.73|1715.52|10534.08|8626.56|12249.60|632.04|0.00|5512.32|10534.08|11166.12|16046.40|16678.44|1907.52| +2450823|27143|2450830|99171|88549|3880|458|70581|1190414|145|46723|4|91|14|3|4192|86|455|39|85.32|187.70|150.16|1464.06|5856.24|3327.48|7320.30|231.90|58.56|3074.37|5797.68|6029.58|8872.05|9103.95|2470.20| +2450823|27143|2450845|99171|88549|3880|458|70581|1190414|145|46723|4|105|7|4|1028|217|455|17|58.31|107.29|98.70|146.03|1677.90|991.27|1823.93|151.01|0.00|54.57|1677.90|1828.91|1732.47|1883.48|686.63| +2450823|27143|2450875|99171|88549|3880|458|70581|1190414|145|46723|4|41|17|4|3788|233|455|47|60.45|90.67|18.13|3409.38|852.11|2841.15|4261.49|17.12|281.19|1022.72|570.92|588.04|1593.64|1610.76|-2270.23| +2450823|54098|2450831|23604|1405648|3339|34583|23604|1405648|3339|34583|4|10|16|5|12218|177|456|49|28.73|71.82|50.27|1055.95|2463.23|1407.77|3519.18|20.69|1428.67|281.26|1034.56|1055.25|1315.82|1336.51|-373.21| +2450823|54098|2450855|23604|1405648|3339|34583|23604|1405648|3339|34583|4|9|19|1|10408|239|456|7|87.07|97.51|74.10|163.87|518.70|609.49|682.57|36.30|0.00|279.79|518.70|555.00|798.49|834.79|-90.79| +2450823|54098|2450842|23604|1405648|3339|34583|23604|1405648|3339|34583|4|76|12|3|8833|145|456|18|56.02|62.74|15.05|858.42|270.90|1008.36|1129.32|24.38|0.00|316.08|270.90|295.28|586.98|611.36|-737.46| +2450823|54098|2450890|23604|1405648|3339|34583|23604|1405648|3339|34583|4|51|18|3|1574|218|456|73|18.93|54.32|33.67|1507.45|2457.91|1381.89|3965.36|196.63|0.00|197.83|2457.91|2654.54|2655.74|2852.37|1076.02| +2450823|54098|2450904|23604|1405648|3339|34583|23604|1405648|3339|34583|4|78|2|2|7484|70|456|14|65.58|97.05|95.10|27.30|1331.40|918.12|1358.70|66.57|0.00|624.96|1331.40|1397.97|1956.36|2022.93|413.28| +2450823|54098|2450902|23604|1405648|3339|34583|23604|1405648|3339|34583|4|23|12|4|12380|21|456|45|62.25|118.89|29.72|4012.65|1337.40|2801.25|5350.05|79.44|13.37|588.15|1324.03|1403.47|1912.18|1991.62|-1477.22| +2450823|54098|2450885|23604|1405648|3339|34583|23604|1405648|3339|34583|4|66|17|5|10688|161|456|84|66.04|192.83|69.41|10367.28|5830.44|5547.36|16197.72|291.52|0.00|2267.16|5830.44|6121.96|8097.60|8389.12|283.08| +2450823|54098|2450913|23604|1405648|3339|34583|23604|1405648|3339|34583|4|29|9|4|643|62|456|87|6.69|10.43|3.02|644.67|262.74|582.03|907.41|5.25|0.00|144.42|262.74|267.99|407.16|412.41|-319.29| +2450823|61596|2450835|3215|410591|3214|13390|23706|1713789|2765|15741|4|100|4|5|1288|194|457|100|96.35|238.94|9.55|22939.00|955.00|9635.00|23894.00|57.30|0.00|10752.00|955.00|1012.30|11707.00|11764.30|-8680.00| +2450823|61596|2450887|3215|410591|3214|13390|23706|1713789|2765|15741|4|35|16|5|11342|176|457|20|87.95|249.77|52.45|3946.40|1049.00|1759.00|4995.40|31.47|0.00|149.80|1049.00|1080.47|1198.80|1230.27|-710.00| +2450823|61596|2450911|3215|410591|3214|13390|23706|1713789|2765|15741|4|93|17|3|15788|254|457|65|80.58|95.89|25.89|4550.00|1682.85|5237.70|6232.85|0.00|0.00|2804.75|1682.85|1682.85|4487.60|4487.60|-3554.85| +2450823|61596|2450841|3215|410591|3214|13390|23706|1713789|2765|15741|4|98|8|1|6607|242|457|52|52.63|92.62|34.26|3034.72|1781.52|2736.76|4816.24|121.85|427.56|770.12|1353.96|1475.81|2124.08|2245.93|-1382.80| +2450823|61596|2450857|3215|410591|3214|13390|23706|1713789|2765|15741|4|88|17|5|11588|71|457|14|37.52|107.68|44.14|889.56|617.96|525.28|1507.52|55.61|0.00|602.98|617.96|673.57|1220.94|1276.55|92.68| +2450823|61596|2450832|3215|410591|3214|13390|23706|1713789|2765|15741|4|29|16|3|11278|25|457|35|73.87|81.25|7.31|2587.90|255.85|2585.45|2843.75|10.23|0.00|1336.30|255.85|266.08|1592.15|1602.38|-2329.60| +2450823|61596|2450881|3215|410591|3214|13390|23706|1713789|2765|15741|4|15|7|4|2677|36|457|26|43.63|68.06|62.61|141.70|1627.86|1134.38|1769.56|113.95|0.00|566.02|1627.86|1741.81|2193.88|2307.83|493.48| +2450823|61596|2450876|3215|410591|3214|13390|23706|1713789|2765|15741|4|63|2|5|14524|146|457|15|60.86|96.76|29.99|1001.55|449.85|912.90|1451.40|17.99|0.00|333.75|449.85|467.84|783.60|801.59|-463.05| +2450823|61596|2450912|3215|410591|3214|13390|23706|1713789|2765|15741|4|53|15|1|13622|270|457|73|1.11|2.73|2.59|10.22|189.07|81.03|199.29|13.76|17.01|19.71|172.06|185.82|191.77|205.53|91.03| +2450823|61596|2450860|3215|410591|3214|13390|23706|1713789|2765|15741|4|85|8|3|2425|197|457|42|51.09|136.92|87.62|2070.60|3680.04|2145.78|5750.64|11.77|3091.23|2357.46|588.81|600.58|2946.27|2958.04|-1556.97| +2450823|61596|2450873|3215|410591|3214|13390|23706|1713789|2765|15741|4|61|8|3|12310|35|457|64|11.30|27.34|15.85|735.36|1014.40|723.20|1749.76|50.72|0.00|664.32|1014.40|1065.12|1678.72|1729.44|291.20| +2450823|61596|2450836|3215|410591|3214|13390|23706|1713789|2765|15741|4|80|12|2|12433|215|457|44|44.84|46.63|37.77|389.84|1661.88|1972.96|2051.72|73.28|847.55|1025.64|814.33|887.61|1839.97|1913.25|-1158.63| +2450823|63822|2450852|82822|913678|4887|43743|82822|913678|4887|43743|4|81|2|2|10876|212|458|26|19.66|51.90|17.64|890.76|458.64|511.16|1349.40|4.58|0.00|188.76|458.64|463.22|647.40|651.98|-52.52| +2450823|63822|2450863|82822|913678|4887|43743|82822|913678|4887|43743|4|45|18|5|14767|19|458|37|18.67|32.11|31.78|12.21|1175.86|690.79|1188.07|47.03|0.00|82.88|1175.86|1222.89|1258.74|1305.77|485.07| +2450823|63822|2450864|82822|913678|4887|43743|82822|913678|4887|43743|4|66|10|4|7033|230|458|72|55.09|100.26|75.19|1805.04|5413.68|3966.48|7218.72|324.82|0.00|2237.76|5413.68|5738.50|7651.44|7976.26|1447.20| +2450823|63822|2450904|82822|913678|4887|43743|82822|913678|4887|43743|4|105|19|1|6356|78|458|57|45.22|120.73|60.36|3441.09|3440.52|2577.54|6881.61|68.81|0.00|2408.25|3440.52|3509.33|5848.77|5917.58|862.98| +2450823|63822|2450885|82822|913678|4887|43743|82822|913678|4887|43743|4|88|4|3|4135|11|458|16|37.56|111.92|0.00|1790.72|0.00|600.96|1790.72|0.00|0.00|590.88|0.00|0.00|590.88|590.88|-600.96| +2450823|63822|2450872|82822|913678|4887|43743|82822|913678|4887|43743|4|76|15|3|3082|297|458|8|11.62|24.75|7.92|134.64|63.36|92.96|198.00|3.16|0.00|29.68|63.36|66.52|93.04|96.20|-29.60| +2450823|63822|2450875|82822|913678|4887|43743|82822|913678|4887|43743|4|47|9|3|7363|3|458|4|10.47|17.90|8.23|38.68|32.92|41.88|71.60|0.32|0.00|6.44|32.92|33.24|39.36|39.68|-8.96| +2450823|63822|2450840|82822|913678|4887|43743|82822|913678|4887|43743|4|87|15|2|1924|45|458|89|78.41|88.60|19.49|6150.79|1734.61|6978.49|7885.40|17.34|0.00|3390.01|1734.61|1751.95|5124.62|5141.96|-5243.88| +2450823|25853|2450834|64750|1229894|6238|4982|64750|1229894|6238|4982|2|103|2|4|4436|45|459|61|72.97|132.80|88.97|2673.63|5427.17|4451.17|8100.80|217.08|0.00|1215.12|5427.17|5644.25|6642.29|6859.37|976.00| +2450823|25853|2450905|64750|1229894|6238|4982|64750|1229894|6238|4982|2|22|13|1|17461|212|459|64|50.34|86.58|84.84|111.36|5429.76|3221.76|5541.12|325.78|0.00|221.44|5429.76|5755.54|5651.20|5976.98|2208.00| +2450823|25853|2450830|64750|1229894|6238|4982|64750|1229894|6238|4982|2|17|3|4|12946|116|459|61|40.70|89.94|82.74|439.20|5047.14|2482.70|5486.34|10.59|3987.24|493.49|1059.90|1070.49|1553.39|1563.98|-1422.80| +2450823|25853|2450900|64750|1229894|6238|4982|64750|1229894|6238|4982|2|86|20|1|1988|89|459|49|83.24|238.06|185.68|2566.62|9098.32|4078.76|11664.94|727.86|0.00|4549.16|9098.32|9826.18|13647.48|14375.34|5019.56| +2450823|25853|2450859|64750|1229894|6238|4982|64750|1229894|6238|4982|2|72|9|5|16280|2|459|53|18.26|53.31|39.98|706.49|2118.94|967.78|2825.43|105.94|0.00|677.87|2118.94|2224.88|2796.81|2902.75|1151.16| +2450823|25853|2450848|64750|1229894|6238|4982|64750|1229894|6238|4982|2|22|9|3|14803|46|459|88|16.34|27.28|19.36|696.96|1703.68|1437.92|2400.64|153.33|0.00|167.20|1703.68|1857.01|1870.88|2024.21|265.76| +2450823|25853|2450834|64750|1229894|6238|4982|64750|1229894|6238|4982|2|35|3|5|3970|144|459|56|50.75|131.95|55.41|4286.24|3102.96|2842.00|7389.20|217.20|0.00|1256.08|3102.96|3320.16|4359.04|4576.24|260.96| +2450823|2564|2450844|93866|148823|133|290|93866|148823|133|290|1|76|1|1|17320|69|460|89|75.37|137.92|66.20|6383.08|5891.80|6707.93|12274.88|530.26|0.00|4786.42|5891.80|6422.06|10678.22|11208.48|-816.13| +2450823|2564|2450901|93866|148823|133|290|93866|148823|133|290|1|47|20|1|4538|90|460|10|15.64|29.40|17.05|123.50|170.50|156.40|294.00|10.23|0.00|141.10|170.50|180.73|311.60|321.83|14.10| +2450823|2564|2450905|93866|148823|133|290|93866|148823|133|290|1|107|9|3|1862|290|460|32|86.48|90.80|21.79|2208.32|697.28|2767.36|2905.60|11.01|146.42|1220.16|550.86|561.87|1771.02|1782.03|-2216.50| +2450823|2564|2450857|93866|148823|133|290|93866|148823|133|290|1|43|11|3|7130|249|460|96|31.68|83.31|47.48|3439.68|4558.08|3041.28|7997.76|182.32|0.00|1199.04|4558.08|4740.40|5757.12|5939.44|1516.80| +2450823|2564|2450912|93866|148823|133|290|93866|148823|133|290|1|107|3|4|3548|286|460|35|48.99|62.21|39.19|805.70|1371.65|1714.65|2177.35|0.00|0.00|892.50|1371.65|1371.65|2264.15|2264.15|-343.00| +2450823|2564|2450901|93866|148823|133|290|93866|148823|133|290|1|72|2|5|14173|184|460|97|89.12|265.57|39.83|21896.78|3863.51|8644.64|25760.29|77.27|0.00|4121.53|3863.51|3940.78|7985.04|8062.31|-4781.13| +2450823|2564|2450868|93866|148823|133|290|93866|148823|133|290|1|67|13|5|1438|104|460|97|46.65|74.17|74.17|0.00|7194.49|4525.05|7194.49|70.50|3669.18|71.78|3525.31|3595.81|3597.09|3667.59|-999.74| +2450824|29065|2450856|41009|692127|5198|25116|41009|692127|5198|25116|2|43|10|2|16814|284|461|53|66.21|196.64|169.11|1459.09|8962.83|3509.13|10421.92|627.39|0.00|4376.74|8962.83|9590.22|13339.57|13966.96|5453.70| +2450824|29065|2450860|41009|692127|5198|25116|41009|692127|5198|25116|2|25|18|1|6889|276|461|69|63.39|64.02|46.73|1193.01|3224.37|4373.91|4417.38|257.94|0.00|220.80|3224.37|3482.31|3445.17|3703.11|-1149.54| +2450824|29065|2450872|41009|692127|5198|25116|41009|692127|5198|25116|2|87|9|1|10238|170|461|95|56.16|151.63|125.85|2449.10|11955.75|5335.20|14404.85|358.67|0.00|431.30|11955.75|12314.42|12387.05|12745.72|6620.55| +2450824|29065|2450912|41009|692127|5198|25116|41009|692127|5198|25116|2|19|1|1|9253|136|461|85|20.82|41.22|40.39|70.55|3433.15|1769.70|3503.70|240.32|0.00|1436.50|3433.15|3673.47|4869.65|5109.97|1663.45| +2450824|29065|2450851|41009|692127|5198|25116|41009|692127|5198|25116|2|59|6|4|4900|292|461|12|20.62|28.66|18.91|117.00|226.92|247.44|343.92|0.00|0.00|51.48|226.92|226.92|278.40|278.40|-20.52| +2450824|29065|2450899|41009|692127|5198|25116|41009|692127|5198|25116|2|80|15|4|410|139|461|11|74.36|194.82|192.87|21.45|2121.57|817.96|2143.02|84.86|0.00|214.28|2121.57|2206.43|2335.85|2420.71|1303.61| +2450824|29065|2450892|41009|692127|5198|25116|41009|692127|5198|25116|2|64|2|2|9142|163|461|79|81.19|199.72|89.87|8678.15|7099.73|6414.01|15777.88|567.97|0.00|6783.73|7099.73|7667.70|13883.46|14451.43|685.72| +2450824|29065|2450892|41009|692127|5198|25116|41009|692127|5198|25116|2|100|13|5|3007|255|461|31|68.86|195.56|158.40|1151.96|4910.40|2134.66|6062.36|114.90|1080.28|2424.82|3830.12|3945.02|6254.94|6369.84|1695.46| +2450824|29065|2450861|41009|692127|5198|25116|41009|692127|5198|25116|2|106|9|2|3748|48|461|87|45.42|92.65|38.91|4675.38|3385.17|3951.54|8060.55|169.25|0.00|3787.98|3385.17|3554.42|7173.15|7342.40|-566.37| +2450824|29065|2450860|41009|692127|5198|25116|41009|692127|5198|25116|2|83|14|5|238|147|461|78|72.78|119.35|41.77|6051.24|3258.06|5676.84|9309.30|9.77|2280.64|4467.84|977.42|987.19|5445.26|5455.03|-4699.42| +2450824|29065|2450870|41009|692127|5198|25116|41009|692127|5198|25116|2|78|17|4|15853|223|461|37|51.72|90.51|33.48|2110.11|1238.76|1913.64|3348.87|99.10|0.00|0.00|1238.76|1337.86|1238.76|1337.86|-674.88| +2450824|29065|2450826|41009|692127|5198|25116|41009|692127|5198|25116|2|11|13|2|8426|77|461|54|5.87|13.91|13.77|7.56|743.58|316.98|751.14|0.00|0.00|172.26|743.58|743.58|915.84|915.84|426.60| +2450824|29065|2450856|41009|692127|5198|25116|41009|692127|5198|25116|2|72|14|4|10945|82|461|81|86.53|132.39|109.88|1823.31|8900.28|7008.93|10723.59|356.01|0.00|5039.82|8900.28|9256.29|13940.10|14296.11|1891.35| +2450824|29065|2450864|41009|692127|5198|25116|41009|692127|5198|25116|2|21|12|2|12007|274|461|50|83.25|166.50|114.88|2581.00|5744.00|4162.50|8325.00|229.76|0.00|1581.50|5744.00|5973.76|7325.50|7555.26|1581.50| +2450824|30763|2450872|39457|925139|3174|28009|39457|925139|3174|28009|2|20|8|4|7066|118|462|23|46.78|58.00|6.96|1173.92|160.08|1075.94|1334.00|0.00|0.00|653.66|160.08|160.08|813.74|813.74|-915.86| +2450824|30763|2450887|39457|925139|3174|28009|39457|925139|3174|28009|2|98|2|3|7693|273|462|38|88.92|161.83|105.18|2152.70|3996.84|3378.96|6149.54|39.96|0.00|553.28|3996.84|4036.80|4550.12|4590.08|617.88| +2450824|30763|2450913|39457|925139|3174|28009|39457|925139|3174|28009|2|13|18|5|6902|234|462|84|57.42|75.22|14.29|5118.12|1200.36|4823.28|6318.48|48.01|0.00|1832.04|1200.36|1248.37|3032.40|3080.41|-3622.92| +2450824|30763|2450874|39457|925139|3174|28009|39457|925139|3174|28009|2|45|13|1|9496|90|462|45|82.88|219.63|122.99|4348.80|5534.55|3729.60|9883.35|110.69|0.00|4545.90|5534.55|5645.24|10080.45|10191.14|1804.95| +2450824|30763|2450875|39457|925139|3174|28009|39457|925139|3174|28009|2|93|18|1|6100|110|462|53|99.63|277.96|61.15|11490.93|3240.95|5280.39|14731.88|0.00|0.00|2062.23|3240.95|3240.95|5303.18|5303.18|-2039.44| +2450824|30763|2450839|39457|925139|3174|28009|39457|925139|3174|28009|2|3|3|5|10687|24|462|65|45.12|105.58|7.39|6382.35|480.35|2932.80|6862.70|28.82|0.00|2744.95|480.35|509.17|3225.30|3254.12|-2452.45| +2450824|30763|2450876|39457|925139|3174|28009|39457|925139|3174|28009|2|43|6|3|10858|259|462|33|13.19|27.43|17.82|317.13|588.06|435.27|905.19|29.40|0.00|235.29|588.06|617.46|823.35|852.75|152.79| +2450824|63619|2450855|21479|829253|3427|19648|21479|829253|3427|19648|4|28|18|5|17395|130|463|63|38.58|78.31|37.58|2565.99|2367.54|2430.54|4933.53|23.67|0.00|1627.92|2367.54|2391.21|3995.46|4019.13|-63.00| +2450824|63619|2450835|21479|829253|3427|19648|21479|829253|3427|19648|4|72|5|2|9853|30|463|35|9.64|22.36|13.41|313.25|469.35|337.40|782.60|28.16|0.00|70.35|469.35|497.51|539.70|567.86|131.95| +2450824|63619|2450887|21479|829253|3427|19648|21479|829253|3427|19648|4|39|10|2|4976|299|463|5|35.80|82.69|9.92|363.85|49.60|179.00|413.45|0.22|45.13|78.55|4.47|4.69|83.02|83.24|-174.53| +2450824|63619|2450859|21479|829253|3427|19648|21479|829253|3427|19648|4|107|7|1|12904|257|463|77|57.04|62.17|32.32|2298.45|2488.64|4392.08|4787.09|52.26|1617.61|0.00|871.03|923.29|871.03|923.29|-3521.05| +2450824|63619|2450892|21479|829253|3427|19648|21479|829253|3427|19648|4|13|17|1|12706|99|463|45|99.67|213.29|83.18|5854.95|3743.10|4485.15|9598.05|299.44|0.00|1823.40|3743.10|4042.54|5566.50|5865.94|-742.05| +2450824|63619|2450882|21479|829253|3427|19648|21479|829253|3427|19648|4|88|15|4|17414|212|463|46|12.53|28.81|9.21|901.60|423.66|576.38|1325.26|16.94|0.00|582.82|423.66|440.60|1006.48|1023.42|-152.72| +2450824|63619|2450835|21479|829253|3427|19648|21479|829253|3427|19648|4|63|14|5|1912|87|463|48|89.11|253.07|73.39|8624.64|3522.72|4277.28|12147.36|246.59|0.00|2672.16|3522.72|3769.31|6194.88|6441.47|-754.56| +2450824|63619|2450859|21479|829253|3427|19648|21479|829253|3427|19648|4|85|7|1|8144|147|463|73|27.17|69.28|66.50|202.94|4854.50|1983.41|5057.44|227.19|1067.99|808.84|3786.51|4013.70|4595.35|4822.54|1803.10| +2450824|63619|2450852|21479|829253|3427|19648|21479|829253|3427|19648|4|90|8|1|9938|277|463|25|26.11|26.89|14.78|302.75|369.50|652.75|672.25|0.00|0.00|26.75|369.50|369.50|396.25|396.25|-283.25| +2450824|63619|2450847|21479|829253|3427|19648|21479|829253|3427|19648|4|15|8|4|8290|205|463|1|84.24|177.74|88.87|88.87|88.87|84.24|177.74|0.88|0.00|69.31|88.87|89.75|158.18|159.06|4.63| +2450824|63619|2450912|21479|829253|3427|19648|21479|829253|3427|19648|4|86|18|5|6242|2|463|76|96.92|283.00|200.93|6237.32|15270.68|7365.92|21508.00|458.12|0.00|6452.40|15270.68|15728.80|21723.08|22181.20|7904.76| +2450824|63619|2450862|21479|829253|3427|19648|21479|829253|3427|19648|4|58|3|5|17252|32|463|82|8.57|21.93|13.81|665.84|1132.42|702.74|1798.26|11.32|0.00|395.24|1132.42|1143.74|1527.66|1538.98|429.68| +2450824|63619|2450839|21479|829253|3427|19648|21479|829253|3427|19648|4|77|13|4|778|284|463|72|13.26|15.11|7.55|544.32|543.60|954.72|1087.92|32.61|0.00|97.20|543.60|576.21|640.80|673.41|-411.12| +2450824|16904|2450849|36410|956476|6942|3808|36410|956476|6942|3808|2|14|5|5|15730|6|464|36|1.48|2.87|0.11|99.36|3.96|53.28|103.32|0.27|0.00|25.56|3.96|4.23|29.52|29.79|-49.32| +2450824|16904|2450904|36410|956476|6942|3808|36410|956476|6942|3808|2|30|19|2|17192|53|464|17|71.99|146.85|41.11|1797.58|698.87|1223.83|2496.45|55.90|0.00|224.57|698.87|754.77|923.44|979.34|-524.96| +2450824|16904|2450883|36410|956476|6942|3808|36410|956476|6942|3808|2|91|13|1|3133|280|464|32|91.19|238.00|21.42|6930.56|685.44|2918.08|7616.00|1.91|637.45|1904.00|47.99|49.90|1951.99|1953.90|-2870.09| +2450824|16904|2450884|36410|956476|6942|3808|36410|956476|6942|3808|2|15|8|3|9235|95|464|59|31.34|89.31|57.15|1897.44|3371.85|1849.06|5269.29|233.66|775.52|0.00|2596.33|2829.99|2596.33|2829.99|747.27| +2450824|16904|2450840|36410|956476|6942|3808|36410|956476|6942|3808|2|39|16|1|8671|234|464|59|97.42|230.88|223.95|408.87|13213.05|5747.78|13621.92|924.91|0.00|6674.67|13213.05|14137.96|19887.72|20812.63|7465.27| +2450824|16904|2450855|36410|956476|6942|3808|36410|956476|6942|3808|2|8|13|2|5896|138|464|2|28.29|42.43|22.06|40.74|44.12|56.58|84.86|1.76|0.00|11.88|44.12|45.88|56.00|57.76|-12.46| +2450824|28026|2450878|70912|1654773|587|13648|70912|1654773|587|13648|4|99|7|1|8074|111|465|3|14.78|38.42|29.58|26.52|88.74|44.34|115.26|4.43|0.00|11.52|88.74|93.17|100.26|104.69|44.40| +2450824|28026|2450831|70912|1654773|587|13648|70912|1654773|587|13648|4|62|8|1|5077|44|465|27|50.79|140.18|109.34|832.68|2952.18|1371.33|3784.86|90.92|1653.22|1854.36|1298.96|1389.88|3153.32|3244.24|-72.37| +2450824|28026|2450902|70912|1654773|587|13648|70912|1654773|587|13648|4|66|8|5|8047|27|465|67|92.56|211.03|80.19|8766.28|5372.73|6201.52|14139.01|268.63|0.00|4382.47|5372.73|5641.36|9755.20|10023.83|-828.79| +2450824|28026|2450884|70912|1654773|587|13648|70912|1654773|587|13648|4|40|1|2|2347|142|465|47|60.50|95.59|29.63|3100.12|1392.61|2843.50|4492.73|41.77|0.00|2246.13|1392.61|1434.38|3638.74|3680.51|-1450.89| +2450824|28026|2450879|70912|1654773|587|13648|70912|1654773|587|13648|4|103|2|4|13945|153|465|71|56.45|120.80|36.24|6003.76|2573.04|4007.95|8576.80|128.65|0.00|3258.90|2573.04|2701.69|5831.94|5960.59|-1434.91| +2450824|28026|2450911|70912|1654773|587|13648|70912|1654773|587|13648|4|75|18|2|14516|140|465|82|67.02|95.16|90.40|390.32|7412.80|5495.64|7803.12|74.12|0.00|2418.18|7412.80|7486.92|9830.98|9905.10|1917.16| +2450824|50494|2450902|24924|627193|3862|5331|24924|627193|3862|5331|2|80|19|2|17929|198|466|31|1.44|2.24|0.49|54.25|15.19|44.64|69.44|0.30|7.59|22.63|7.60|7.90|30.23|30.53|-37.04| +2450824|50494|2450910|24924|627193|3862|5331|24924|627193|3862|5331|2|49|12|3|8593|142|466|19|88.36|115.75|17.36|1869.41|329.84|1678.84|2199.25|23.08|0.00|307.80|329.84|352.92|637.64|660.72|-1349.00| +2450824|50494|2450848|24924|627193|3862|5331|24924|627193|3862|5331|2|96|17|5|17605|246|466|23|22.73|50.00|22.50|632.50|517.50|522.79|1150.00|10.35|0.00|563.50|517.50|527.85|1081.00|1091.35|-5.29| +2450824|50494|2450828|24924|627193|3862|5331|24924|627193|3862|5331|2|69|3|1|13570|59|466|44|74.12|138.60|63.75|3293.40|2805.00|3261.28|6098.40|140.25|0.00|1707.20|2805.00|2945.25|4512.20|4652.45|-456.28| +2450824|50494|2450847|24924|627193|3862|5331|24924|627193|3862|5331|2|13|5|1|4348|126|466|72|40.17|40.97|37.28|265.68|2684.16|2892.24|2949.84|161.04|0.00|87.84|2684.16|2845.20|2772.00|2933.04|-208.08| +2450824|50494|2450867|24924|627193|3862|5331|24924|627193|3862|5331|2|78|13|5|11078|187|466|60|97.68|134.79|20.21|6874.80|1212.60|5860.80|8087.40|12.12|0.00|2668.80|1212.60|1224.72|3881.40|3893.52|-4648.20| +2450824|50494|2450890|24924|627193|3862|5331|24924|627193|3862|5331|2|44|3|4|8420|269|466|77|47.42|128.50|118.22|791.56|9102.94|3651.34|9894.50|273.08|0.00|2968.35|9102.94|9376.02|12071.29|12344.37|5451.60| +2450824|30991|2450857|69300|705602|7151|7853|69300|705602|7151|7853|4|93|18|3|16774|143|467|61|51.44|93.10|89.37|227.53|5451.57|3137.84|5679.10|294.38|545.15|1646.39|4906.42|5200.80|6552.81|6847.19|1768.58| +2450824|30991|2450874|69300|705602|7151|7853|69300|705602|7151|7853|4|46|4|1|16442|163|467|61|19.02|48.31|20.29|1709.22|1237.69|1160.22|2946.91|12.37|0.00|87.84|1237.69|1250.06|1325.53|1337.90|77.47| +2450824|30991|2450907|69300|705602|7151|7853|69300|705602|7151|7853|4|12|19|4|12320|186|467|14|14.40|23.61|19.83|52.92|277.62|201.60|330.54|8.32|0.00|122.22|277.62|285.94|399.84|408.16|76.02| +2450824|30991|2450861|69300|705602|7151|7853|69300|705602|7151|7853|4|52|10|4|8708|27|467|14|29.63|29.92|29.32|8.40|410.48|414.82|418.88|7.63|155.98|37.66|254.50|262.13|292.16|299.79|-160.32| +2450824|30991|2450912|69300|705602|7151|7853|69300|705602|7151|7853|4|74|3|1|9535|184|467|26|33.29|57.59|21.88|928.46|568.88|865.54|1497.34|39.82|0.00|419.12|568.88|608.70|988.00|1027.82|-296.66| +2450824|30991|2450841|69300|705602|7151|7853|69300|705602|7151|7853|4|20|11|4|10832|205|467|73|71.87|153.80|93.81|4379.27|6848.13|5246.51|11227.40|410.88|0.00|4827.49|6848.13|7259.01|11675.62|12086.50|1601.62| +2450824|30991|2450904|69300|705602|7151|7853|69300|705602|7151|7853|4|20|4|5|8023|105|467|38|17.05|26.76|12.04|559.36|457.52|647.90|1016.88|19.44|68.62|386.08|388.90|408.34|774.98|794.42|-259.00| +2450824|30991|2450886|69300|705602|7151|7853|69300|705602|7151|7853|4|76|8|1|16753|287|467|40|46.09|49.77|9.95|1592.80|398.00|1843.60|1990.80|17.91|199.00|517.60|199.00|216.91|716.60|734.51|-1644.60| +2450824|52272|2450837|35196|328413|3230|21247|35196|328413|3230|21247|2|71|7|1|9295|45|468|29|89.95|189.79|37.95|4403.36|1100.55|2608.55|5503.91|66.03|0.00|2256.49|1100.55|1166.58|3357.04|3423.07|-1508.00| +2450824|52272|2450910|35196|328413|3230|21247|35196|328413|3230|21247|2|104|7|5|3634|196|468|57|69.27|129.53|103.62|1476.87|5906.34|3948.39|7383.21|472.50|0.00|2805.54|5906.34|6378.84|8711.88|9184.38|1957.95| +2450824|52272|2450896|35196|328413|3230|21247|35196|328413|3230|21247|2|28|8|2|10714|32|468|45|31.26|50.95|25.47|1146.60|1146.15|1406.70|2292.75|0.00|252.15|549.90|894.00|894.00|1443.90|1443.90|-512.70| +2450824|52272|2450841|35196|328413|3230|21247|35196|328413|3230|21247|2|75|2|4|12079|203|468|71|83.24|216.42|127.68|6300.54|9065.28|5910.04|15365.82|90.65|0.00|921.58|9065.28|9155.93|9986.86|10077.51|3155.24| +2450824|70266|2450895|57115|1368341|1967|34840|57115|1368341|1967|34840|2|92|14|2|2815|226|469|40|37.41|67.33|63.96|134.80|2558.40|1496.40|2693.20|204.67|0.00|161.20|2558.40|2763.07|2719.60|2924.27|1062.00| +2450824|70266|2450849|57115|1368341|1967|34840|57115|1368341|1967|34840|2|51|6|3|17284|49|469|59|85.03|102.88|49.38|3156.50|2913.42|5016.77|6069.92|21.85|2476.40|1881.51|437.02|458.87|2318.53|2340.38|-4579.75| +2450824|70266|2450867|57115|1368341|1967|34840|57115|1368341|1967|34840|2|34|13|3|2479|202|469|61|55.55|56.10|45.44|650.26|2771.84|3388.55|3422.10|55.43|0.00|991.86|2771.84|2827.27|3763.70|3819.13|-616.71| +2450824|70266|2450829|57115|1368341|1967|34840|57115|1368341|1967|34840|2|98|9|2|6823|101|469|33|93.85|152.03|110.98|1354.65|3662.34|3097.05|5016.99|36.62|0.00|752.40|3662.34|3698.96|4414.74|4451.36|565.29| +2450824|70266|2450843|57115|1368341|1967|34840|57115|1368341|1967|34840|2|92|13|3|11536|50|469|78|9.99|11.88|3.56|648.96|277.68|779.22|926.64|11.10|0.00|46.02|277.68|288.78|323.70|334.80|-501.54| +2450824|70266|2450901|57115|1368341|1967|34840|57115|1368341|1967|34840|2|53|10|2|16828|154|469|87|29.66|56.94|29.60|2378.58|2575.20|2580.42|4953.78|25.75|0.00|147.90|2575.20|2600.95|2723.10|2748.85|-5.22| +2450824|70266|2450905|57115|1368341|1967|34840|57115|1368341|1967|34840|2|97|10|3|9391|57|469|74|21.69|27.11|20.33|501.72|1504.42|1605.06|2006.14|60.17|0.00|441.04|1504.42|1564.59|1945.46|2005.63|-100.64| +2450824|43418|2450894|60178|1093007|589|30618|60178|1093007|589|30618|1|92|6|2|10423|85|470|77|62.49|126.22|2.52|9524.90|194.04|4811.73|9718.94|17.46|0.00|4081.77|194.04|211.50|4275.81|4293.27|-4617.69| +2450824|43418|2450883|60178|1093007|589|30618|60178|1093007|589|30618|1|26|5|3|11576|19|470|76|52.93|97.39|1.94|7254.20|147.44|4022.68|7401.64|11.79|0.00|1998.04|147.44|159.23|2145.48|2157.27|-3875.24| +2450824|43418|2450911|60178|1093007|589|30618|60178|1093007|589|30618|1|66|9|4|2125|296|470|88|40.89|78.09|25.76|4605.04|2266.88|3598.32|6871.92|158.68|0.00|480.48|2266.88|2425.56|2747.36|2906.04|-1331.44| +2450824|43418|2450887|60178|1093007|589|30618|60178|1093007|589|30618|1|76|18|4|17386|241|470|29|68.18|128.86|100.51|822.15|2914.79|1977.22|3736.94|0.00|0.00|410.93|2914.79|2914.79|3325.72|3325.72|937.57| +2450824|43418|2450836|60178|1093007|589|30618|60178|1093007|589|30618|1|67|7|5|14167|223|470|78|89.06|154.96|117.76|2901.60|9185.28|6946.68|12086.88|642.96|0.00|3504.54|9185.28|9828.24|12689.82|13332.78|2238.60| +2450824|41867|2450859|6432|1156090|1748|34253|6432|1156090|1748|34253|1|31|16|3|1402|21|471|57|63.44|87.54|80.53|399.57|4590.21|3616.08|4989.78|229.51|0.00|598.50|4590.21|4819.72|5188.71|5418.22|974.13| +2450824|41867|2450849|6432|1156090|1748|34253|6432|1156090|1748|34253|1|17|7|1|10354|11|471|24|2.81|4.52|4.52|0.00|108.48|67.44|108.48|0.00|83.52|32.40|24.96|24.96|57.36|57.36|-42.48| +2450824|41867|2450828|6432|1156090|1748|34253|6432|1156090|1748|34253|1|33|2|4|2716|21|471|55|9.33|12.50|0.50|660.00|27.50|513.15|687.50|0.55|0.00|295.35|27.50|28.05|322.85|323.40|-485.65| +2450824|41867|2450905|6432|1156090|1748|34253|6432|1156090|1748|34253|1|68|4|3|6724|137|471|15|9.19|14.61|12.85|26.40|192.75|137.85|219.15|1.92|0.00|89.85|192.75|194.67|282.60|284.52|54.90| +2450824|41867|2450866|6432|1156090|1748|34253|6432|1156090|1748|34253|1|27|9|5|9044|140|471|29|27.44|53.78|4.84|1419.26|140.36|795.76|1559.62|8.42|0.00|670.48|140.36|148.78|810.84|819.26|-655.40| +2450824|41867|2450849|6432|1156090|1748|34253|6432|1156090|1748|34253|1|36|7|1|1324|289|471|13|15.59|46.30|33.33|168.61|433.29|202.67|601.90|25.99|0.00|102.31|433.29|459.28|535.60|561.59|230.62| +2450824|41867|2450885|6432|1156090|1748|34253|6432|1156090|1748|34253|1|51|14|1|5110|43|471|84|93.96|210.47|115.75|7956.48|9723.00|7892.64|17679.48|194.46|0.00|5303.76|9723.00|9917.46|15026.76|15221.22|1830.36| +2450824|41867|2450828|6432|1156090|1748|34253|6432|1156090|1748|34253|1|39|4|3|10351|279|471|64|52.10|137.02|45.21|5875.84|2893.44|3334.40|8769.28|57.86|0.00|3068.80|2893.44|2951.30|5962.24|6020.10|-440.96| +2450824|41867|2450866|6432|1156090|1748|34253|6432|1156090|1748|34253|1|37|2|2|11533|82|471|45|19.22|35.94|19.40|744.30|873.00|864.90|1617.30|17.46|0.00|0.00|873.00|890.46|873.00|890.46|8.10| +2450824|41867|2450896|6432|1156090|1748|34253|6432|1156090|1748|34253|1|58|5|3|14089|124|471|50|22.82|55.45|33.82|1081.50|1691.00|1141.00|2772.50|16.91|0.00|277.00|1691.00|1707.91|1968.00|1984.91|550.00| +2450824|41867|2450909|6432|1156090|1748|34253|6432|1156090|1748|34253|1|45|17|2|15871|299|471|35|76.91|177.66|56.85|4228.35|1989.75|2691.85|6218.10|79.59|0.00|1865.15|1989.75|2069.34|3854.90|3934.49|-702.10| +2450824|41867|2450913|6432|1156090|1748|34253|6432|1156090|1748|34253|1|13|16|4|391|130|471|54|21.20|46.00|36.34|521.64|1962.36|1144.80|2484.00|97.13|19.62|74.52|1942.74|2039.87|2017.26|2114.39|797.94| +2450824|41867|2450890|6432|1156090|1748|34253|6432|1156090|1748|34253|1|37|15|2|4579|93|471|33|11.47|16.51|11.39|168.96|375.87|378.51|544.83|0.00|0.00|163.35|375.87|375.87|539.22|539.22|-2.64| +2450824|41867|2450877|6432|1156090|1748|34253|6432|1156090|1748|34253|1|61|10|1|7771|103|471|40|41.71|44.21|21.66|902.00|866.40|1668.40|1768.40|13.86|173.28|247.20|693.12|706.98|940.32|954.18|-975.28| +2450824|60473|2450881|31428|1143902|5176|48450|28392|59845|319|4793|2|53|8|1|11176|97|472|21|29.54|88.62|5.31|1749.51|111.51|620.34|1861.02|2.26|79.17|614.04|32.34|34.60|646.38|648.64|-588.00| +2450824|60473|2450911|31428|1143902|5176|48450|28392|59845|319|4793|2|95|9|3|10264|68|472|58|51.94|138.67|18.02|6997.70|1045.16|3012.52|8042.86|52.25|0.00|1206.40|1045.16|1097.41|2251.56|2303.81|-1967.36| +2450824|60473|2450839|31428|1143902|5176|48450|28392|59845|319|4793|2|66|12|5|13048|41|472|44|77.46|117.73|69.46|2123.88|3056.24|3408.24|5180.12|61.12|0.00|362.56|3056.24|3117.36|3418.80|3479.92|-352.00| +2450824|60473|2450878|31428|1143902|5176|48450|28392|59845|319|4793|2|55|12|2|13292|271|472|94|65.00|94.90|14.23|7582.98|1337.62|6110.00|8920.60|107.00|0.00|4192.40|1337.62|1444.62|5530.02|5637.02|-4772.38| +2450824|60473|2450891|31428|1143902|5176|48450|28392|59845|319|4793|2|65|2|2|553|227|472|48|37.01|40.71|17.09|1133.76|820.32|1776.48|1954.08|8.20|0.00|703.20|820.32|828.52|1523.52|1531.72|-956.16| +2450824|60473|2450912|31428|1143902|5176|48450|28392|59845|319|4793|2|26|14|1|14966|110|472|18|23.82|57.88|22.57|635.58|406.26|428.76|1041.84|0.00|0.00|312.48|406.26|406.26|718.74|718.74|-22.50| +2450824|60473|2450902|31428|1143902|5176|48450|28392|59845|319|4793|2|27|18|1|1771|251|472|3|32.48|87.69|68.39|57.90|205.17|97.44|263.07|14.07|4.10|126.27|201.07|215.14|327.34|341.41|103.63| +2450824|60473|2450848|31428|1143902|5176|48450|28392|59845|319|4793|2|1|18|2|9991|245|472|65|15.91|17.66|13.95|241.15|906.75|1034.15|1147.90|13.96|707.26|11.05|199.49|213.45|210.54|224.50|-834.66| +2450824|60473|2450896|31428|1143902|5176|48450|28392|59845|319|4793|2|90|3|2|3650|33|472|75|29.11|53.27|30.36|1718.25|2277.00|2183.25|3995.25|91.08|0.00|79.50|2277.00|2368.08|2356.50|2447.58|93.75| +2450824|31892|2450837|5844|599983|6475|30614|38971|1080798|1337|24229|1|75|19|1|3799|140|473|38|50.34|99.16|76.35|866.78|2901.30|1912.92|3768.08|261.11|0.00|376.58|2901.30|3162.41|3277.88|3538.99|988.38| +2450824|31892|2450851|5844|599983|6475|30614|38971|1080798|1337|24229|1|67|15|3|17272|214|473|47|94.30|263.09|139.43|5812.02|6553.21|4432.10|12365.23|131.06|0.00|6182.38|6553.21|6684.27|12735.59|12866.65|2121.11| +2450824|31892|2450914|5844|599983|6475|30614|38971|1080798|1337|24229|1|106|17|3|17326|195|473|1|91.14|173.16|58.87|114.29|58.87|91.14|173.16|2.94|0.00|74.45|58.87|61.81|133.32|136.26|-32.27| +2450824|31892|2450910|5844|599983|6475|30614|38971|1080798|1337|24229|1|107|7|3|16219|72|473|8|44.50|121.04|1.21|958.64|9.68|356.00|968.32|0.23|6.29|251.76|3.39|3.62|255.15|255.38|-352.61| +2450824|31892|2450906|5844|599983|6475|30614|38971|1080798|1337|24229|1|48|16|1|10045|253|473|39|38.54|56.65|19.26|1458.21|751.14|1503.06|2209.35|22.53|0.00|441.87|751.14|773.67|1193.01|1215.54|-751.92| +2450824|31892|2450889|5844|599983|6475|30614|38971|1080798|1337|24229|1|28|20|1|7132|183|473|22|91.93|160.87|28.95|2902.24|636.90|2022.46|3539.14|57.32|0.00|1203.18|636.90|694.22|1840.08|1897.40|-1385.56| +2450824|31892|2450857|5844|599983|6475|30614|38971|1080798|1337|24229|1|100|17|2|1160|119|473|40|57.42|106.80|53.40|2136.00|2136.00|2296.80|4272.00|64.08|0.00|85.20|2136.00|2200.08|2221.20|2285.28|-160.80| +2450824|31892|2450864|5844|599983|6475|30614|38971|1080798|1337|24229|1|76|19|4|14821|89|473|72|64.31|164.63|161.33|237.60|11615.76|4630.32|11853.36|464.63|0.00|236.88|11615.76|12080.39|11852.64|12317.27|6985.44| +2450824|28849|2450905|2767|743612|904|13796|2767|743612|904|13796|1|31|13|3|6688|248|474|17|83.97|121.75|114.44|124.27|1945.48|1427.49|2069.75|0.00|0.00|496.74|1945.48|1945.48|2442.22|2442.22|517.99| +2450824|28849|2450863|2767|743612|904|13796|2767|743612|904|13796|1|97|14|3|427|91|474|85|97.71|233.52|53.70|15284.70|4564.50|8305.35|19849.20|410.80|0.00|4167.55|4564.50|4975.30|8732.05|9142.85|-3740.85| +2450824|28849|2450832|2767|743612|904|13796|2767|743612|904|13796|1|28|7|2|2935|277|474|95|7.97|16.97|11.03|564.30|1047.85|757.15|1612.15|73.34|0.00|724.85|1047.85|1121.19|1772.70|1846.04|290.70| +2450824|28849|2450834|2767|743612|904|13796|2767|743612|904|13796|1|85|2|1|6643|99|474|32|73.20|160.30|40.07|3847.36|1282.24|2342.40|5129.60|0.00|0.00|666.56|1282.24|1282.24|1948.80|1948.80|-1060.16| +2450824|28849|2450827|2767|743612|904|13796|2767|743612|904|13796|1|9|5|4|2788|145|474|35|65.47|138.14|6.90|4593.40|241.50|2291.45|4834.90|9.66|0.00|1740.55|241.50|251.16|1982.05|1991.71|-2049.95| +2450824|28849|2450841|2767|743612|904|13796|2767|743612|904|13796|1|33|17|4|4328|136|474|44|63.99|175.97|10.55|7278.48|464.20|2815.56|7742.68|27.85|0.00|1625.80|464.20|492.05|2090.00|2117.85|-2351.36| +2450824|28849|2450834|2767|743612|904|13796|2767|743612|904|13796|1|95|1|4|12704|41|474|24|41.16|120.59|68.73|1244.64|1649.52|987.84|2894.16|0.00|0.00|260.40|1649.52|1649.52|1909.92|1909.92|661.68| +2450824|28849|2450864|2767|743612|904|13796|2767|743612|904|13796|1|41|7|2|10580|179|474|99|22.56|57.30|33.80|2326.50|3346.20|2233.44|5672.70|160.61|1338.48|1814.67|2007.72|2168.33|3822.39|3983.00|-225.72| +2450824|28849|2450830|2767|743612|904|13796|2767|743612|904|13796|1|16|18|5|13262|6|474|51|47.54|122.17|84.29|1931.88|4298.79|2424.54|6230.67|186.99|558.84|1370.37|3739.95|3926.94|5110.32|5297.31|1315.41| +2450824|28849|2450861|2767|743612|904|13796|2767|743612|904|13796|1|35|20|1|14293|123|474|18|56.17|114.02|15.96|1765.08|287.28|1011.06|2052.36|17.23|0.00|636.12|287.28|304.51|923.40|940.63|-723.78| +2450824|28849|2450872|2767|743612|904|13796|2767|743612|904|13796|1|78|8|1|3277|196|474|46|98.70|213.19|123.65|4118.84|5687.90|4540.20|9806.74|56.87|0.00|0.00|5687.90|5744.77|5687.90|5744.77|1147.70| +2450824|28849|2450850|2767|743612|904|13796|2767|743612|904|13796|1|63|5|1|16378|251|474|27|48.62|85.57|33.37|1409.40|900.99|1312.74|2310.39|27.02|0.00|1039.50|900.99|928.01|1940.49|1967.51|-411.75| +2450824|28849|2450846|2767|743612|904|13796|2767|743612|904|13796|1|70|1|3|502|96|474|47|62.06|147.08|75.01|3387.29|3525.47|2916.82|6912.76|0.00|0.00|138.18|3525.47|3525.47|3663.65|3663.65|608.65| +2450824|28849|2450907|2767|743612|904|13796|2767|743612|904|13796|1|73|18|4|8392|55|474|19|44.61|126.69|40.54|1636.85|770.26|847.59|2407.11|1.61|608.50|770.26|161.76|163.37|932.02|933.63|-685.83| +2450824|33695|2450912|70478|1556959|687|2015|70478|1556959|687|2015|1|48|20|1|7724|83|475|76|57.67|77.27|69.54|587.48|5285.04|4382.92|5872.52|105.70|0.00|2466.20|5285.04|5390.74|7751.24|7856.94|902.12| +2450824|33695|2450861|70478|1556959|687|2015|70478|1556959|687|2015|1|93|19|5|13360|120|475|69|85.10|222.96|122.62|6923.46|8460.78|5871.90|15384.24|253.82|0.00|4153.11|8460.78|8714.60|12613.89|12867.71|2588.88| +2450824|33695|2450893|70478|1556959|687|2015|70478|1556959|687|2015|1|84|3|1|10570|7|475|32|3.83|7.85|5.02|90.56|160.64|122.56|251.20|6.42|0.00|7.36|160.64|167.06|168.00|174.42|38.08| +2450824|33695|2450829|70478|1556959|687|2015|70478|1556959|687|2015|1|70|15|1|15932|175|475|9|70.25|128.55|60.41|613.26|543.69|632.25|1156.95|0.00|0.00|370.17|543.69|543.69|913.86|913.86|-88.56| +2450824|53412|2450855|42890|1119364|2238|17112|42890|1119364|2238|17112|4|47|2|2|9424|217|476|84|89.65|156.88|112.95|3690.12|9487.80|7530.60|13177.92|660.35|1233.41|526.68|8254.39|8914.74|8781.07|9441.42|723.79| +2450824|53412|2450896|42890|1119364|2238|17112|42890|1119364|2238|17112|4|96|13|2|9214|176|476|46|38.13|74.73|48.57|1203.36|2234.22|1753.98|3437.58|111.71|0.00|584.20|2234.22|2345.93|2818.42|2930.13|480.24| +2450824|53412|2450835|42890|1119364|2238|17112|42890|1119364|2238|17112|4|38|6|3|3212|142|476|38|93.18|205.92|123.55|3130.06|4694.90|3540.84|7824.96|0.00|0.00|1173.44|4694.90|4694.90|5868.34|5868.34|1154.06| +2450824|53412|2450885|42890|1119364|2238|17112|42890|1119364|2238|17112|4|65|2|3|6470|57|476|12|35.75|68.99|11.72|687.24|140.64|429.00|827.88|11.25|0.00|173.76|140.64|151.89|314.40|325.65|-288.36| +2450824|53412|2450844|42890|1119364|2238|17112|42890|1119364|2238|17112|4|35|15|4|5494|294|476|17|9.61|24.79|3.71|358.36|63.07|163.37|421.43|5.04|0.00|206.38|63.07|68.11|269.45|274.49|-100.30| +2450824|53412|2450910|42890|1119364|2238|17112|42890|1119364|2238|17112|4|12|16|5|10942|73|476|93|74.96|131.92|64.64|6257.04|6011.52|6971.28|12268.56|60.11|0.00|121.83|6011.52|6071.63|6133.35|6193.46|-959.76| +2450824|53412|2450879|42890|1119364|2238|17112|42890|1119364|2238|17112|4|83|15|3|8846|8|476|28|70.47|99.36|61.60|1057.28|1724.80|1973.16|2782.08|17.24|0.00|1196.16|1724.80|1742.04|2920.96|2938.20|-248.36| +2450824|43905|2450861|74195|1380887|4379|3871|74195|1380887|4379|3871|2|13|16|4|11359|186|477|21|38.42|69.15|33.19|755.16|696.99|806.82|1452.15|6.96|0.00|86.94|696.99|703.95|783.93|790.89|-109.83| +2450824|43905|2450875|74195|1380887|4379|3871|74195|1380887|4379|3871|2|101|13|2|10124|282|477|92|93.51|251.54|193.68|5323.12|17818.56|8602.92|23141.68|178.18|0.00|693.68|17818.56|17996.74|18512.24|18690.42|9215.64| +2450824|43905|2450878|74195|1380887|4379|3871|74195|1380887|4379|3871|2|105|2|2|7280|186|477|76|51.15|148.33|28.18|9131.40|2141.68|3887.40|11273.08|42.83|0.00|1803.48|2141.68|2184.51|3945.16|3987.99|-1745.72| +2450824|43905|2450829|74195|1380887|4379|3871|74195|1380887|4379|3871|2|83|14|5|655|10|477|92|88.13|233.54|154.13|7305.72|14179.96|8107.96|21485.68|336.06|2977.79|8593.72|11202.17|11538.23|19795.89|20131.95|3094.21| +2450824|43905|2450877|74195|1380887|4379|3871|74195|1380887|4379|3871|2|6|13|1|11137|2|477|56|13.50|26.05|18.23|437.92|1020.88|756.00|1458.80|71.46|0.00|451.92|1020.88|1092.34|1472.80|1544.26|264.88| +2450824|43905|2450879|74195|1380887|4379|3871|74195|1380887|4379|3871|2|15|15|1|2119|156|477|24|92.63|244.54|122.27|2934.48|2934.48|2223.12|5868.96|146.72|0.00|528.00|2934.48|3081.20|3462.48|3609.20|711.36| +2450824|43905|2450908|74195|1380887|4379|3871|74195|1380887|4379|3871|2|48|9|1|7579|300|477|62|59.63|142.51|27.07|7157.28|1678.34|3697.06|8835.62|33.56|0.00|176.70|1678.34|1711.90|1855.04|1888.60|-2018.72| +2450824|43905|2450890|74195|1380887|4379|3871|74195|1380887|4379|3871|2|70|12|5|17692|171|477|35|5.60|11.36|11.36|0.00|397.60|196.00|397.60|7.95|0.00|87.15|397.60|405.55|484.75|492.70|201.60| +2450824|43905|2450886|74195|1380887|4379|3871|74195|1380887|4379|3871|2|69|16|2|11839|172|477|24|87.22|253.81|124.36|3106.80|2984.64|2093.28|6091.44|59.69|0.00|1218.24|2984.64|3044.33|4202.88|4262.57|891.36| +2450824|43905|2450910|74195|1380887|4379|3871|74195|1380887|4379|3871|2|61|11|5|11962|88|477|42|46.28|128.19|103.83|1023.12|4360.86|1943.76|5383.98|43.60|0.00|968.94|4360.86|4404.46|5329.80|5373.40|2417.10| +2450824|62527|2450841|21537|674128|6272|35663|21537|674128|6272|35663|2|34|13|1|2974|240|478|89|95.77|158.02|107.45|4500.73|9563.05|8523.53|14063.78|463.80|286.89|3796.74|9276.16|9739.96|13072.90|13536.70|752.63| +2450824|62527|2450879|21537|674128|6272|35663|21537|674128|6272|35663|2|66|2|4|12062|90|478|30|20.36|30.13|2.41|831.60|72.30|610.80|903.90|2.89|0.00|325.20|72.30|75.19|397.50|400.39|-538.50| +2450824|62527|2450843|21537|674128|6272|35663|21537|674128|6272|35663|2|17|1|2|5858|294|478|73|90.09|118.01|79.06|2843.35|5771.38|6576.57|8614.73|0.00|5771.38|3273.32|0.00|0.00|3273.32|3273.32|-6576.57| +2450824|62527|2450838|21537|674128|6272|35663|21537|674128|6272|35663|2|50|15|5|14464|204|478|31|82.23|167.74|38.58|4003.96|1195.98|2549.13|5199.94|0.00|358.79|1143.90|837.19|837.19|1981.09|1981.09|-1711.94| +2450824|62527|2450887|21537|674128|6272|35663|21537|674128|6272|35663|2|91|18|2|740|77|478|16|67.46|144.36|46.19|1570.72|739.04|1079.36|2309.76|44.34|0.00|877.60|739.04|783.38|1616.64|1660.98|-340.32| +2450824|62527|2450857|21537|674128|6272|35663|21537|674128|6272|35663|2|8|8|1|1772|297|478|93|81.92|136.80|95.76|3816.72|8905.68|7618.56|12722.40|534.34|0.00|4961.55|8905.68|9440.02|13867.23|14401.57|1287.12| +2450824|62527|2450847|21537|674128|6272|35663|21537|674128|6272|35663|2|75|4|1|15517|20|478|38|94.17|116.77|5.83|4215.72|221.54|3578.46|4437.26|8.86|0.00|1730.52|221.54|230.40|1952.06|1960.92|-3356.92| +2450824|81238|2450848|78818|692976|3934|32892|78818|692976|3934|32892|4|26|18|2|6463|87|479|35|41.05|68.96|59.99|313.95|2099.65|1436.75|2413.60|63.82|1301.78|579.25|797.87|861.69|1377.12|1440.94|-638.88| +2450824|81238|2450826|78818|692976|3934|32892|78818|692976|3934|32892|4|100|14|3|13651|91|479|17|66.79|104.86|88.08|285.26|1497.36|1135.43|1782.62|74.86|0.00|213.86|1497.36|1572.22|1711.22|1786.08|361.93| +2450824|81238|2450902|78818|692976|3934|32892|78818|692976|3934|32892|4|24|10|4|17210|277|479|35|2.56|7.34|6.75|20.65|236.25|89.60|256.90|9.45|0.00|123.20|236.25|245.70|359.45|368.90|146.65| +2450824|81238|2450854|78818|692976|3934|32892|78818|692976|3934|32892|4|23|15|2|11750|241|479|40|62.03|101.10|39.42|2467.20|1576.80|2481.20|4044.00|126.14|0.00|849.20|1576.80|1702.94|2426.00|2552.14|-904.40| +2450824|81238|2450867|78818|692976|3934|32892|78818|692976|3934|32892|4|85|8|1|10516|70|479|28|5.13|9.95|9.45|14.00|264.60|143.64|278.60|0.00|0.00|5.32|264.60|264.60|269.92|269.92|120.96| +2450824|81238|2450895|78818|692976|3934|32892|78818|692976|3934|32892|4|86|11|1|11294|96|479|23|65.75|100.59|42.24|1342.05|971.52|1512.25|2313.57|0.00|0.00|1087.21|971.52|971.52|2058.73|2058.73|-540.73| +2450824|74429|2450833|13201|1061660|2566|7472|13201|1061660|2566|7472|1|91|3|1|17798|169|480|23|48.64|125.97|120.93|115.92|2781.39|1118.72|2897.31|21.69|2058.22|1100.78|723.17|744.86|1823.95|1845.64|-395.55| +2450824|74429|2450844|13201|1061660|2566|7472|13201|1061660|2566|7472|1|14|8|2|13616|267|480|34|73.25|100.35|50.17|1706.12|1705.78|2490.50|3411.90|102.34|0.00|34.00|1705.78|1808.12|1739.78|1842.12|-784.72| +2450824|74429|2450851|13201|1061660|2566|7472|13201|1061660|2566|7472|1|88|17|5|15445|89|480|66|28.87|58.02|41.77|1072.50|2756.82|1905.42|3829.32|57.34|2040.04|1684.32|716.78|774.12|2401.10|2458.44|-1188.64| +2450824|74429|2450904|13201|1061660|2566|7472|13201|1061660|2566|7472|1|79|18|5|10441|149|480|42|27.33|34.98|10.84|1013.88|455.28|1147.86|1469.16|4.55|0.00|146.58|455.28|459.83|601.86|606.41|-692.58| +2450824|74429|2450912|13201|1061660|2566|7472|13201|1061660|2566|7472|1|105|11|3|14008|189|480|70|54.99|70.38|7.03|4434.50|492.10|3849.30|4926.60|39.36|0.00|1182.30|492.10|531.46|1674.40|1713.76|-3357.20| +2450824|71633|2450848|64897|1706749|2201|37653|64897|1706749|2201|37653|4|80|11|2|12992|294|481|39|32.65|52.56|38.36|553.80|1496.04|1273.35|2049.84|44.88|0.00|594.36|1496.04|1540.92|2090.40|2135.28|222.69| +2450824|71633|2450891|64897|1706749|2201|37653|64897|1706749|2201|37653|4|106|3|1|5251|144|481|70|25.84|27.39|2.19|1764.00|153.30|1808.80|1917.30|3.06|0.00|919.80|153.30|156.36|1073.10|1076.16|-1655.50| +2450824|71633|2450857|64897|1706749|2201|37653|64897|1706749|2201|37653|4|75|9|4|3170|119|481|81|98.05|205.90|74.12|10674.18|6003.72|7942.05|16677.90|480.29|0.00|332.91|6003.72|6484.01|6336.63|6816.92|-1938.33| +2450824|71633|2450904|64897|1706749|2201|37653|64897|1706749|2201|37653|4|13|11|1|12049|234|481|38|64.48|99.94|62.96|1405.24|2392.48|2450.24|3797.72|23.92|0.00|797.24|2392.48|2416.40|3189.72|3213.64|-57.76| +2450824|71633|2450855|64897|1706749|2201|37653|64897|1706749|2201|37653|4|61|14|1|11222|128|481|50|13.03|16.67|15.00|83.50|750.00|651.50|833.50|45.00|0.00|0.00|750.00|795.00|750.00|795.00|98.50| +2450824|71633|2450912|64897|1706749|2201|37653|64897|1706749|2201|37653|4|53|8|1|16369|57|481|46|21.63|53.85|32.84|966.46|1510.64|994.98|2477.10|135.95|0.00|1238.32|1510.64|1646.59|2748.96|2884.91|515.66| +2450824|71633|2450886|64897|1706749|2201|37653|64897|1706749|2201|37653|4|54|12|5|13327|144|481|49|49.53|124.81|8.73|5687.92|427.77|2426.97|6115.69|29.94|0.00|1223.04|427.77|457.71|1650.81|1680.75|-1999.20| +2450824|71633|2450888|64897|1706749|2201|37653|64897|1706749|2201|37653|4|47|8|3|14413|209|481|4|48.94|134.58|103.62|123.84|414.48|195.76|538.32|2.40|294.28|242.24|120.20|122.60|362.44|364.84|-75.56| +2450824|71633|2450835|64897|1706749|2201|37653|64897|1706749|2201|37653|4|34|16|3|15223|69|481|99|6.52|14.21|13.35|85.14|1321.65|645.48|1406.79|68.99|555.09|351.45|766.56|835.55|1118.01|1187.00|121.08| +2450824|71633|2450850|64897|1706749|2201|37653|64897|1706749|2201|37653|4|89|4|3|3481|174|481|25|83.25|146.52|30.76|2894.00|769.00|2081.25|3663.00|23.07|0.00|989.00|769.00|792.07|1758.00|1781.07|-1312.25| +2450824|71633|2450841|64897|1706749|2201|37653|64897|1706749|2201|37653|4|5|2|3|7688|228|481|82|70.26|132.79|61.08|5880.22|5008.56|5761.32|10888.78|0.00|0.00|0.00|5008.56|5008.56|5008.56|5008.56|-752.76| +2450824|71633|2450862|64897|1706749|2201|37653|64897|1706749|2201|37653|4|4|11|2|10309|165|481|33|25.86|39.82|29.46|341.88|972.18|853.38|1314.06|87.49|0.00|564.96|972.18|1059.67|1537.14|1624.63|118.80| +2450824|69212|2450851|4829|414213|4397|26981|4829|414213|4397|26981|4|55|14|3|16741|28|482|89|46.51|119.99|110.39|854.40|9824.71|4139.39|10679.11|687.72|0.00|4271.11|9824.71|10512.43|14095.82|14783.54|5685.32| +2450824|69212|2450844|4829|414213|4397|26981|4829|414213|4397|26981|4|71|4|3|12925|111|482|65|59.47|148.08|94.77|3465.15|6160.05|3865.55|9625.20|369.60|0.00|1058.20|6160.05|6529.65|7218.25|7587.85|2294.50| +2450824|69212|2450867|4829|414213|4397|26981|4829|414213|4397|26981|4|67|1|4|17095|110|482|90|11.99|30.45|27.40|274.50|2466.00|1079.10|2740.50|15.53|1948.14|602.10|517.86|533.39|1119.96|1135.49|-561.24| +2450824|69212|2450884|4829|414213|4397|26981|4829|414213|4397|26981|4|42|1|2|4753|71|482|8|74.27|207.21|0.00|1657.68|0.00|594.16|1657.68|0.00|0.00|663.04|0.00|0.00|663.04|663.04|-594.16| +2450824|69212|2450844|4829|414213|4397|26981|4829|414213|4397|26981|4|75|20|2|7514|64|482|50|54.30|67.87|51.58|814.50|2579.00|2715.00|3393.50|206.32|0.00|1289.50|2579.00|2785.32|3868.50|4074.82|-136.00| +2450824|69212|2450832|4829|414213|4397|26981|4829|414213|4397|26981|4|75|7|3|13108|93|482|42|26.88|44.62|32.57|506.10|1367.94|1128.96|1874.04|13.67|0.00|599.34|1367.94|1381.61|1967.28|1980.95|238.98| +2450824|69212|2450856|4829|414213|4397|26981|4829|414213|4397|26981|4|29|2|2|5959|110|482|76|16.44|32.05|26.92|389.88|2045.92|1249.44|2435.80|61.37|0.00|900.60|2045.92|2107.29|2946.52|3007.89|796.48| +2450824|69212|2450898|4829|414213|4397|26981|4829|414213|4397|26981|4|25|1|2|5638|220|482|96|23.21|48.50|12.12|3492.48|1163.52|2228.16|4656.00|58.17|0.00|325.44|1163.52|1221.69|1488.96|1547.13|-1064.64| +2450824|69212|2450837|4829|414213|4397|26981|4829|414213|4397|26981|4|91|19|1|2732|83|482|92|53.78|117.77|47.10|6501.64|4333.20|4947.76|10834.84|216.66|0.00|1299.96|4333.20|4549.86|5633.16|5849.82|-614.56| +2450824|69212|2450899|4829|414213|4397|26981|4829|414213|4397|26981|4|46|14|3|11906|54|482|6|24.28|65.55|42.60|137.70|255.60|145.68|393.30|1.12|143.13|11.76|112.47|113.59|124.23|125.35|-33.21| +2450824|69212|2450894|4829|414213|4397|26981|4829|414213|4397|26981|4|68|3|4|514|255|482|80|84.19|224.78|173.08|4136.00|13846.40|6735.20|17982.40|560.77|2630.81|0.00|11215.59|11776.36|11215.59|11776.36|4480.39| +2450824|69212|2450908|4829|414213|4397|26981|4829|414213|4397|26981|4|21|5|5|12190|17|482|20|37.22|67.74|49.45|365.80|989.00|744.40|1354.80|0.00|0.00|433.40|989.00|989.00|1422.40|1422.40|244.60| +2450824|69212|2450859|4829|414213|4397|26981|4829|414213|4397|26981|4|52|5|3|9322|231|482|89|92.54|223.02|104.81|10520.69|9328.09|8236.06|19848.78|746.24|0.00|4961.75|9328.09|10074.33|14289.84|15036.08|1092.03| +2450824|69212|2450862|4829|414213|4397|26981|4829|414213|4397|26981|4|8|11|4|11815|42|482|40|94.98|159.56|89.35|2808.40|3574.00|3799.20|6382.40|250.18|0.00|2872.00|3574.00|3824.18|6446.00|6696.18|-225.20| +2450824|80763|2450875|46125|255298|1271|41197|46125|255298|1271|41197|1|106|1|2|8690|224|483|25|56.59|160.14|155.33|120.25|3883.25|1414.75|4003.50|77.66|0.00|1120.75|3883.25|3960.91|5004.00|5081.66|2468.50| +2450824|80763|2450911|46125|255298|1271|41197|46125|255298|1271|41197|1|62|9|3|8270|126|483|19|8.19|23.42|6.55|320.53|124.45|155.61|444.98|11.20|0.00|129.01|124.45|135.65|253.46|264.66|-31.16| +2450824|80763|2450870|46125|255298|1271|41197|46125|255298|1271|41197|1|95|15|1|10004|38|483|26|6.89|14.95|2.09|334.36|54.34|179.14|388.70|3.26|0.00|166.92|54.34|57.60|221.26|224.52|-124.80| +2450824|80763|2450902|46125|255298|1271|41197|46125|255298|1271|41197|1|60|2|3|13870|80|483|47|24.43|35.17|11.25|1124.24|528.75|1148.21|1652.99|5.28|0.00|793.36|528.75|534.03|1322.11|1327.39|-619.46| +2450824|80763|2450875|46125|255298|1271|41197|46125|255298|1271|41197|1|12|2|5|13168|260|483|44|35.88|88.26|58.25|1320.44|2563.00|1578.72|3883.44|205.04|0.00|931.92|2563.00|2768.04|3494.92|3699.96|984.28| +2450824|80763|2450828|46125|255298|1271|41197|46125|255298|1271|41197|1|92|9|2|10|241|483|58|99.38|205.71|41.14|9545.06|2386.12|5764.04|11931.18|23.86|0.00|2743.98|2386.12|2409.98|5130.10|5153.96|-3377.92| +2450824|80763|2450859|46125|255298|1271|41197|46125|255298|1271|41197|1|56|18|2|12550|90|483|25|68.88|121.22|46.06|1879.00|1151.50|1722.00|3030.50|23.03|0.00|697.00|1151.50|1174.53|1848.50|1871.53|-570.50| +2450824|65685|2450838|40064|1720768|3872|26976|40064|1720768|3872|26976|2|74|8|3|10777|263|484|29|12.47|17.33|15.59|50.46|452.11|361.63|502.57|36.16|0.00|115.42|452.11|488.27|567.53|603.69|90.48| +2450824|65685|2450845|40064|1720768|3872|26976|40064|1720768|3872|26976|2|67|1|2|2635|117|484|18|34.04|44.25|39.38|87.66|708.84|612.72|796.50|42.53|0.00|278.64|708.84|751.37|987.48|1030.01|96.12| +2450824|65685|2450907|40064|1720768|3872|26976|40064|1720768|3872|26976|2|5|15|4|9650|100|484|3|26.18|30.89|10.81|60.24|32.43|78.54|92.67|2.27|0.00|37.05|32.43|34.70|69.48|71.75|-46.11| +2450824|65685|2450859|40064|1720768|3872|26976|40064|1720768|3872|26976|2|11|14|2|9320|271|484|80|55.28|107.79|25.86|6554.40|2068.80|4422.40|8623.20|165.50|0.00|172.00|2068.80|2234.30|2240.80|2406.30|-2353.60| +2450824|65685|2450843|40064|1720768|3872|26976|40064|1720768|3872|26976|2|89|19|2|14104|168|484|62|12.30|25.70|22.35|207.70|1385.70|762.60|1593.40|41.57|0.00|190.96|1385.70|1427.27|1576.66|1618.23|623.10| +2450824|65685|2450875|40064|1720768|3872|26976|40064|1720768|3872|26976|2|106|9|4|12716|50|484|16|94.57|170.22|170.22|0.00|2723.52|1513.12|2723.52|163.41|0.00|599.04|2723.52|2886.93|3322.56|3485.97|1210.40| +2450824|65685|2450911|40064|1720768|3872|26976|40064|1720768|3872|26976|2|71|3|5|14174|276|484|7|38.08|104.72|91.10|95.34|637.70|266.56|733.04|11.73|344.35|307.86|293.35|305.08|601.21|612.94|26.79| +2450824|77880|2450837|68773|739779|6785|24591|68773|739779|6785|24591|1|65|9|4|15946|269|485|80|3.98|6.40|2.56|307.20|204.80|318.40|512.00|12.28|0.00|204.80|204.80|217.08|409.60|421.88|-113.60| +2450824|77880|2450884|68773|739779|6785|24591|68773|739779|6785|24591|1|93|7|3|5686|235|485|18|12.47|33.04|14.53|333.18|261.54|224.46|594.72|20.92|0.00|255.60|261.54|282.46|517.14|538.06|37.08| +2450824|77880|2450869|68773|739779|6785|24591|68773|739779|6785|24591|1|27|3|2|10207|64|485|36|5.66|16.01|4.16|426.60|149.76|203.76|576.36|7.48|0.00|201.60|149.76|157.24|351.36|358.84|-54.00| +2450824|77880|2450871|68773|739779|6785|24591|68773|739779|6785|24591|1|44|2|3|12817|33|485|78|19.12|49.90|21.45|2219.10|1673.10|1491.36|3892.20|33.46|0.00|816.66|1673.10|1706.56|2489.76|2523.22|181.74| +2450824|77880|2450832|68773|739779|6785|24591|68773|739779|6785|24591|1|84|19|1|1858|103|485|49|48.38|90.95|75.48|758.03|3698.52|2370.62|4456.55|36.98|0.00|1158.36|3698.52|3735.50|4856.88|4893.86|1327.90| +2450824|77880|2450866|68773|739779|6785|24591|68773|739779|6785|24591|1|13|2|2|10582|117|485|71|77.90|175.27|98.15|5475.52|6968.65|5530.90|12444.17|217.42|4250.87|1990.84|2717.78|2935.20|4708.62|4926.04|-2813.12| +2450824|77880|2450893|68773|739779|6785|24591|68773|739779|6785|24591|1|78|16|3|14671|114|485|9|16.99|28.20|9.87|164.97|88.83|152.91|253.80|7.10|0.00|101.52|88.83|95.93|190.35|197.45|-64.08| +2450824|77880|2450871|68773|739779|6785|24591|68773|739779|6785|24591|1|60|8|1|1208|59|485|72|97.49|190.10|70.33|8623.44|5063.76|7019.28|13687.20|405.10|0.00|5474.88|5063.76|5468.86|10538.64|10943.74|-1955.52| +2450824|77880|2450888|68773|739779|6785|24591|68773|739779|6785|24591|1|77|17|4|6550|193|485|93|95.94|246.56|78.89|15593.31|7336.77|8922.42|22930.08|436.53|1100.51|6191.01|6236.26|6672.79|12427.27|12863.80|-2686.16| +2450824|77880|2450842|68773|739779|6785|24591|68773|739779|6785|24591|1|83|16|4|7903|121|485|50|61.52|112.58|24.76|4391.00|1238.00|3076.00|5629.00|0.00|0.00|2251.50|1238.00|1238.00|3489.50|3489.50|-1838.00| +2450824|77880|2450831|68773|739779|6785|24591|68773|739779|6785|24591|1|68|8|3|11582|275|485|62|8.64|15.03|8.71|391.84|540.02|535.68|931.86|32.40|0.00|186.00|540.02|572.42|726.02|758.42|4.34| +2450824|38007|2450901|90142|1266772|5802|11279|52819|122186|3616|28246|4|61|11|3|3844|84|486|94|25.14|71.64|40.83|2896.14|3838.02|2363.16|6734.16|191.90|0.00|1009.56|3838.02|4029.92|4847.58|5039.48|1474.86| +2450824|38007|2450872|90142|1266772|5802|11279|52819|122186|3616|28246|4|89|8|1|5402|45|486|16|99.95|242.87|216.15|427.52|3458.40|1599.20|3885.92|242.08|0.00|816.00|3458.40|3700.48|4274.40|4516.48|1859.20| +2450824|38007|2450901|90142|1266772|5802|11279|52819|122186|3616|28246|4|54|7|5|7702|131|486|6|70.88|91.43|2.74|532.14|16.44|425.28|548.58|1.15|0.00|208.44|16.44|17.59|224.88|226.03|-408.84| +2450824|38007|2450887|90142|1266772|5802|11279|52819|122186|3616|28246|4|49|19|3|14012|194|486|84|69.76|117.89|106.10|990.36|8912.40|5859.84|9902.76|356.49|0.00|296.52|8912.40|9268.89|9208.92|9565.41|3052.56| +2450824|38007|2450881|90142|1266772|5802|11279|52819|122186|3616|28246|4|10|18|2|8840|245|486|6|14.08|34.91|31.41|21.00|188.46|84.48|209.46|2.45|139.46|4.14|49.00|51.45|53.14|55.59|-35.48| +2450824|38007|2450841|90142|1266772|5802|11279|52819|122186|3616|28246|4|44|5|5|7216|50|486|31|43.39|43.39|39.05|134.54|1210.55|1345.09|1345.09|84.73|0.00|564.82|1210.55|1295.28|1775.37|1860.10|-134.54| +2450824|38007|2450842|90142|1266772|5802|11279|52819|122186|3616|28246|4|26|11|3|4471|154|486|85|87.86|249.52|72.36|15058.60|6150.60|7468.10|21209.20|184.51|0.00|8695.50|6150.60|6335.11|14846.10|15030.61|-1317.50| +2450824|38007|2450864|90142|1266772|5802|11279|52819|122186|3616|28246|4|71|9|5|11545|25|486|62|81.62|232.61|188.41|2740.40|11681.42|5060.44|14421.82|467.25|0.00|2307.02|11681.42|12148.67|13988.44|14455.69|6620.98| +2450824|38007|2450828|90142|1266772|5802|11279|52819|122186|3616|28246|4|54|16|5|16573|118|486|96|76.17|219.36|39.48|17268.48|3790.08|7312.32|21058.56|162.97|530.61|9476.16|3259.47|3422.44|12735.63|12898.60|-4052.85| +2450824|38007|2450841|90142|1266772|5802|11279|52819|122186|3616|28246|4|70|14|1|12721|213|486|76|64.42|147.52|25.07|9306.20|1905.32|4895.92|11211.52|5.71|1333.72|4932.40|571.60|577.31|5504.00|5509.71|-4324.32| +2450824|38007|2450853|90142|1266772|5802|11279|52819|122186|3616|28246|4|20|18|4|10400|104|486|98|99.28|262.09|188.70|7192.22|18492.60|9729.44|25684.82|184.92|0.00|9759.82|18492.60|18677.52|28252.42|28437.34|8763.16| +2450824|38007|2450912|90142|1266772|5802|11279|52819|122186|3616|28246|4|82|15|1|1708|241|486|59|94.02|160.77|160.77|0.00|9485.43|5547.18|9485.43|758.83|0.00|2560.60|9485.43|10244.26|12046.03|12804.86|3938.25| +2450824|23669|2450910|83242|855084|1679|35474|83242|855084|1679|35474|4|19|4|4|9398|79|487|23|57.88|108.23|18.39|2066.32|422.97|1331.24|2489.29|16.91|0.00|373.29|422.97|439.88|796.26|813.17|-908.27| +2450824|23669|2450866|83242|855084|1679|35474|83242|855084|1679|35474|4|5|3|5|10132|91|487|71|98.94|257.24|205.79|3652.95|14611.09|7024.74|18264.04|0.00|0.00|8948.84|14611.09|14611.09|23559.93|23559.93|7586.35| +2450824|23669|2450899|83242|855084|1679|35474|83242|855084|1679|35474|4|70|5|2|9646|19|487|38|62.22|170.48|158.54|453.72|6024.52|2364.36|6478.24|97.59|4397.89|1878.34|1626.63|1724.22|3504.97|3602.56|-737.73| +2450824|23669|2450862|83242|855084|1679|35474|83242|855084|1679|35474|4|8|3|1|13448|27|487|11|57.31|135.25|17.58|1294.37|193.38|630.41|1487.75|5.80|0.00|595.10|193.38|199.18|788.48|794.28|-437.03| +2450824|23669|2450880|83242|855084|1679|35474|83242|855084|1679|35474|4|1|8|5|13972|168|487|25|85.78|174.13|114.92|1480.25|2873.00|2144.50|4353.25|258.57|0.00|522.25|2873.00|3131.57|3395.25|3653.82|728.50| +2450824|23669|2450906|83242|855084|1679|35474|83242|855084|1679|35474|4|64|12|1|1544|287|487|22|60.60|153.92|29.24|2742.96|643.28|1333.20|3386.24|25.73|0.00|609.40|643.28|669.01|1252.68|1278.41|-689.92| +2450824|23669|2450829|83242|855084|1679|35474|83242|855084|1679|35474|4|69|19|3|1280|104|487|32|87.76|236.95|165.86|2274.88|5307.52|2808.32|7582.40|106.15|0.00|1667.84|5307.52|5413.67|6975.36|7081.51|2499.20| +2450824|23669|2450892|83242|855084|1679|35474|83242|855084|1679|35474|4|25|16|5|8510|283|487|57|91.94|230.76|59.99|9733.89|3419.43|5240.58|13153.32|0.00|0.00|3024.99|3419.43|3419.43|6444.42|6444.42|-1821.15| +2450824|23669|2450853|83242|855084|1679|35474|83242|855084|1679|35474|4|1|13|2|16598|166|487|76|66.08|198.24|63.43|10245.56|4820.68|5022.08|15066.24|48.20|0.00|7533.12|4820.68|4868.88|12353.80|12402.00|-201.40| +2450824|32735|2450891|31561|1010283|2605|527|31561|1010283|2605|527|2|29|9|2|13777|90|488|55|24.00|70.08|21.72|2659.80|1194.60|1320.00|3854.40|95.56|0.00|1464.65|1194.60|1290.16|2659.25|2754.81|-125.40| +2450824|32735|2450854|31561|1010283|2605|527|31561|1010283|2605|527|2|40|14|4|928|47|488|43|40.95|96.23|62.54|1448.67|2689.22|1760.85|4137.89|65.34|1963.13|372.38|726.09|791.43|1098.47|1163.81|-1034.76| +2450824|32735|2450906|31561|1010283|2605|527|31561|1010283|2605|527|2|89|20|1|17072|254|488|94|99.36|217.59|174.07|4090.88|16362.58|9339.84|20453.46|163.62|0.00|5317.58|16362.58|16526.20|21680.16|21843.78|7022.74| +2450824|32735|2450846|31561|1010283|2605|527|31561|1010283|2605|527|2|97|2|3|14221|236|488|99|22.65|67.04|45.58|2124.54|4512.42|2242.35|6636.96|69.94|3113.56|3318.48|1398.86|1468.80|4717.34|4787.28|-843.49| +2450824|32735|2450872|31561|1010283|2605|527|31561|1010283|2605|527|2|7|1|1|16069|284|488|38|4.36|7.84|0.47|280.06|17.86|165.68|297.92|0.23|6.07|104.12|11.79|12.02|115.91|116.14|-153.89| +2450824|32735|2450838|31561|1010283|2605|527|31561|1010283|2605|527|2|62|1|5|5473|60|488|60|61.57|161.31|100.01|3678.00|6000.60|3694.20|9678.60|300.03|0.00|3580.80|6000.60|6300.63|9581.40|9881.43|2306.40| +2450824|32735|2450854|31561|1010283|2605|527|31561|1010283|2605|527|2|34|20|3|16657|272|488|31|29.75|65.45|53.66|365.49|1663.46|922.25|2028.95|116.44|0.00|567.92|1663.46|1779.90|2231.38|2347.82|741.21| +2450824|32735|2450912|31561|1010283|2605|527|31561|1010283|2605|527|2|47|12|2|17020|32|488|7|90.60|135.90|97.84|266.42|684.88|634.20|951.30|7.60|431.47|85.61|253.41|261.01|339.02|346.62|-380.79| +2450824|32735|2450881|31561|1010283|2605|527|31561|1010283|2605|527|2|54|5|4|12674|95|488|51|37.63|61.71|59.24|125.97|3021.24|1919.13|3147.21|30.21|0.00|1447.38|3021.24|3051.45|4468.62|4498.83|1102.11| +2450824|32735|2450846|31561|1010283|2605|527|31561|1010283|2605|527|2|72|7|5|4954|114|488|71|46.72|100.44|72.31|1997.23|5134.01|3317.12|7131.24|410.72|0.00|998.26|5134.01|5544.73|6132.27|6542.99|1816.89| +2450824|32735|2450896|31561|1010283|2605|527|31561|1010283|2605|527|2|63|5|4|17632|80|488|48|1.68|4.75|1.28|166.56|61.44|80.64|228.00|4.91|0.00|100.32|61.44|66.35|161.76|166.67|-19.20| +2450824|32735|2450906|31561|1010283|2605|527|31561|1010283|2605|527|2|6|19|3|11936|201|488|6|65.50|96.28|71.24|150.24|427.44|393.00|577.68|29.92|0.00|75.06|427.44|457.36|502.50|532.42|34.44| +2450824|32735|2450904|31561|1010283|2605|527|31561|1010283|2605|527|2|106|16|3|19|205|488|23|96.09|165.27|153.70|266.11|3535.10|2210.07|3801.21|70.70|0.00|1216.24|3535.10|3605.80|4751.34|4822.04|1325.03| +2450824|32735|2450829|31561|1010283|2605|527|31561|1010283|2605|527|2|63|9|1|10244|80|488|55|63.35|86.15|55.99|1658.80|3079.45|3484.25|4738.25|246.35|0.00|141.90|3079.45|3325.80|3221.35|3467.70|-404.80| +2450824|73147|2450888|57701|507478|8|22985|57701|507478|8|22985|4|18|1|5|11512|265|489|19|90.08|196.37|78.54|2238.77|1492.26|1711.52|3731.03|14.92|0.00|746.13|1492.26|1507.18|2238.39|2253.31|-219.26| +2450824|73147|2450838|57701|507478|8|22985|57701|507478|8|22985|4|26|15|4|13442|259|489|20|64.53|132.93|18.61|2286.40|372.20|1290.60|2658.60|29.77|0.00|1169.60|372.20|401.97|1541.80|1571.57|-918.40| +2450824|73147|2450910|57701|507478|8|22985|57701|507478|8|22985|4|28|12|5|1501|255|489|73|7.70|13.16|1.05|884.03|76.65|562.10|960.68|5.36|0.00|460.63|76.65|82.01|537.28|542.64|-485.45| +2450824|73147|2450890|57701|507478|8|22985|57701|507478|8|22985|4|62|6|4|16498|245|489|37|55.30|113.91|26.19|3245.64|969.03|2046.10|4214.67|48.45|0.00|800.68|969.03|1017.48|1769.71|1818.16|-1077.07| +2450824|73147|2450878|57701|507478|8|22985|57701|507478|8|22985|4|30|3|1|12604|49|489|25|3.47|6.21|4.34|46.75|108.50|86.75|155.25|8.68|0.00|12.25|108.50|117.18|120.75|129.43|21.75| +2450824|73147|2450874|57701|507478|8|22985|57701|507478|8|22985|4|2|18|2|10573|16|489|74|91.28|262.88|110.40|11283.52|8169.60|6754.72|19453.12|80.87|81.69|2723.20|8087.91|8168.78|10811.11|10891.98|1333.19| +2450824|73147|2450854|57701|507478|8|22985|57701|507478|8|22985|4|42|5|5|13129|213|489|9|17.74|44.88|4.03|367.65|36.27|159.66|403.92|2.17|0.00|28.26|36.27|38.44|64.53|66.70|-123.39| +2450824|73147|2450882|57701|507478|8|22985|57701|507478|8|22985|4|16|4|1|13778|113|489|28|61.57|102.82|83.28|547.12|2331.84|1723.96|2878.96|139.91|0.00|1324.12|2331.84|2471.75|3655.96|3795.87|607.88| +2450824|68002|2450849|51127|337141|4308|18275|51127|337141|4308|18275|2|55|4|1|15829|183|490|20|31.40|60.60|27.27|666.60|545.40|628.00|1212.00|0.00|0.00|84.80|545.40|545.40|630.20|630.20|-82.60| +2450824|68002|2450827|51127|337141|4308|18275|51127|337141|4308|18275|2|66|13|2|3440|147|490|85|33.00|84.15|32.81|4363.90|2788.85|2805.00|7152.75|77.80|1924.30|285.60|864.55|942.35|1150.15|1227.95|-1940.45| +2450824|68002|2450902|51127|337141|4308|18275|51127|337141|4308|18275|2|106|6|5|5527|101|490|17|35.30|67.42|44.49|389.81|756.33|600.10|1146.14|0.00|0.00|389.64|756.33|756.33|1145.97|1145.97|156.23| +2450824|68002|2450828|51127|337141|4308|18275|51127|337141|4308|18275|2|33|10|4|17443|127|490|86|73.39|193.01|92.64|8631.82|7967.04|6311.54|16598.86|239.01|0.00|497.94|7967.04|8206.05|8464.98|8703.99|1655.50| +2450824|68002|2450851|51127|337141|4308|18275|51127|337141|4308|18275|2|41|14|2|14566|96|490|47|56.63|100.23|26.05|3486.46|1224.35|2661.61|4710.81|110.19|0.00|706.41|1224.35|1334.54|1930.76|2040.95|-1437.26| +2450824|68002|2450878|51127|337141|4308|18275|51127|337141|4308|18275|2|69|18|2|9259|103|490|78|24.80|26.04|22.13|304.98|1726.14|1934.40|2031.12|103.56|0.00|913.38|1726.14|1829.70|2639.52|2743.08|-208.26| +2450824|68002|2450905|51127|337141|4308|18275|51127|337141|4308|18275|2|53|18|3|7478|281|490|61|11.60|23.66|11.83|721.63|721.63|707.60|1443.26|57.73|0.00|259.25|721.63|779.36|980.88|1038.61|14.03| +2450824|68002|2450910|51127|337141|4308|18275|51127|337141|4308|18275|2|74|9|2|5965|243|490|99|69.56|124.51|109.56|1480.05|10846.44|6886.44|12326.49|829.75|1626.96|1355.31|9219.48|10049.23|10574.79|11404.54|2333.04| +2450824|68002|2450884|51127|337141|4308|18275|51127|337141|4308|18275|2|88|10|4|8098|190|490|18|53.33|137.59|13.75|2229.12|247.50|959.94|2476.62|14.85|0.00|817.20|247.50|262.35|1064.70|1079.55|-712.44| +2450824|68002|2450840|51127|337141|4308|18275|51127|337141|4308|18275|2|41|19|5|11665|135|490|1|12.57|27.15|0.54|26.61|0.54|12.57|27.15|0.01|0.25|5.43|0.29|0.30|5.72|5.73|-12.28| +2450824|68002|2450854|51127|337141|4308|18275|51127|337141|4308|18275|2|97|19|1|3907|90|490|90|34.93|40.51|26.33|1276.20|2369.70|3143.70|3645.90|28.43|947.88|1166.40|1421.82|1450.25|2588.22|2616.65|-1721.88| +2450824|68002|2450840|51127|337141|4308|18275|51127|337141|4308|18275|2|13|7|3|1891|292|490|44|75.09|168.20|134.56|1480.16|5920.64|3303.96|7400.80|236.82|0.00|443.96|5920.64|6157.46|6364.60|6601.42|2616.68| +2450824|68002|2450876|51127|337141|4308|18275|51127|337141|4308|18275|2|57|10|1|15439|91|490|41|66.08|137.44|43.98|3831.86|1803.18|2709.28|5635.04|54.09|0.00|2141.02|1803.18|1857.27|3944.20|3998.29|-906.10| +2450824|28116|2450851|18473|466006|4739|37119|47978|1212585|4118|451|4|14|18|4|17600|116|491|75|10.83|24.80|9.42|1153.50|706.50|812.25|1860.00|21.19|0.00|204.00|706.50|727.69|910.50|931.69|-105.75| +2450824|28116|2450901|18473|466006|4739|37119|47978|1212585|4118|451|4|21|11|5|2653|86|491|29|10.21|15.62|7.81|226.49|226.49|296.09|452.98|11.32|0.00|203.58|226.49|237.81|430.07|441.39|-69.60| +2450824|28116|2450882|18473|466006|4739|37119|47978|1212585|4118|451|4|30|13|2|1606|172|491|48|48.11|70.24|49.87|977.76|2393.76|2309.28|3371.52|95.75|0.00|404.16|2393.76|2489.51|2797.92|2893.67|84.48| +2450824|28116|2450870|18473|466006|4739|37119|47978|1212585|4118|451|4|100|17|4|5912|16|491|89|73.78|123.95|94.20|2647.75|8383.80|6566.42|11031.55|0.00|0.00|2757.22|8383.80|8383.80|11141.02|11141.02|1817.38| +2450824|28116|2450875|18473|466006|4739|37119|47978|1212585|4118|451|4|105|18|3|11504|71|491|62|1.42|3.81|0.60|199.02|37.20|88.04|236.22|1.11|0.00|70.68|37.20|38.31|107.88|108.99|-50.84| +2450824|28116|2450829|18473|466006|4739|37119|47978|1212585|4118|451|4|88|6|5|9778|85|491|72|21.03|49.84|43.36|466.56|3121.92|1514.16|3588.48|62.43|0.00|143.28|3121.92|3184.35|3265.20|3327.63|1607.76| +2450824|28116|2450872|18473|466006|4739|37119|47978|1212585|4118|451|4|50|9|3|1291|228|491|66|2.80|2.88|0.08|184.80|5.28|184.80|190.08|0.00|0.00|41.58|5.28|5.28|46.86|46.86|-179.52| +2450824|28116|2450852|18473|466006|4739|37119|47978|1212585|4118|451|4|31|11|3|6722|10|491|19|69.48|179.25|132.64|885.59|2520.16|1320.12|3405.75|126.00|0.00|1191.87|2520.16|2646.16|3712.03|3838.03|1200.04| +2450824|28116|2450841|18473|466006|4739|37119|47978|1212585|4118|451|4|13|4|3|11029|103|491|85|24.48|65.11|37.76|2324.75|3209.60|2080.80|5534.35|192.57|0.00|884.85|3209.60|3402.17|4094.45|4287.02|1128.80| +2450824|28116|2450873|18473|466006|4739|37119|47978|1212585|4118|451|4|15|10|2|7381|241|491|16|81.41|171.77|48.09|1978.88|769.44|1302.56|2748.32|15.38|0.00|796.96|769.44|784.82|1566.40|1581.78|-533.12| +2450824|28116|2450842|18473|466006|4739|37119|47978|1212585|4118|451|4|21|19|1|14540|105|491|57|86.14|172.28|99.92|4124.52|5695.44|4909.98|9819.96|227.81|0.00|4909.98|5695.44|5923.25|10605.42|10833.23|785.46| +2450824|28116|2450910|18473|466006|4739|37119|47978|1212585|4118|451|4|15|10|3|5821|8|491|11|96.72|189.57|147.86|458.81|1626.46|1063.92|2085.27|97.58|0.00|333.63|1626.46|1724.04|1960.09|2057.67|562.54| +2450824|28116|2450899|18473|466006|4739|37119|47978|1212585|4118|451|4|84|12|1|6002|185|491|24|86.08|182.48|162.40|481.92|3897.60|2065.92|4379.52|77.95|0.00|1488.96|3897.60|3975.55|5386.56|5464.51|1831.68| +2450824|22837|2450884|49264|180974|1484|27994|49264|180974|1484|27994|2|59|1|5|8707|295|492|97|73.27|142.14|61.12|7858.94|5928.64|7107.19|13787.58|67.58|4802.19|4136.08|1126.45|1194.03|5262.53|5330.11|-5980.74| +2450824|22837|2450878|49264|180974|1484|27994|49264|180974|1484|27994|2|34|15|1|17368|67|492|16|43.55|53.56|18.74|557.12|299.84|696.80|856.96|26.98|0.00|402.72|299.84|326.82|702.56|729.54|-396.96| +2450824|22837|2450851|49264|180974|1484|27994|49264|180974|1484|27994|2|58|13|4|6295|1|492|95|90.41|104.87|94.38|996.55|8966.10|8588.95|9962.65|192.77|5110.67|1294.85|3855.43|4048.20|5150.28|5343.05|-4733.52| +2450824|22837|2450859|49264|180974|1484|27994|49264|180974|1484|27994|2|70|1|1|6230|196|492|60|19.14|46.12|36.43|581.40|2185.80|1148.40|2767.20|174.86|0.00|138.00|2185.80|2360.66|2323.80|2498.66|1037.40| +2450824|22837|2450905|49264|180974|1484|27994|49264|180974|1484|27994|2|41|4|1|14458|213|492|48|38.57|72.89|21.13|2484.48|1014.24|1851.36|3498.72|30.42|0.00|1259.52|1014.24|1044.66|2273.76|2304.18|-837.12| +2450824|66211|2450874|88978|1854110|5995|22727|88978|1854110|5995|22727|2|7|9|1|6788|110|493|93|48.31|108.21|19.47|8252.82|1810.71|4492.83|10063.53|22.45|688.06|2112.96|1122.65|1145.10|3235.61|3258.06|-3370.18| +2450824|66211|2450911|88978|1854110|5995|22727|88978|1854110|5995|22727|2|24|7|5|16522|236|493|24|27.10|58.53|51.50|168.72|1236.00|650.40|1404.72|24.72|0.00|42.00|1236.00|1260.72|1278.00|1302.72|585.60| +2450824|66211|2450868|88978|1854110|5995|22727|88978|1854110|5995|22727|2|10|7|4|14338|235|493|19|39.50|86.11|77.49|163.78|1472.31|750.50|1636.09|76.85|191.40|507.11|1280.91|1357.76|1788.02|1864.87|530.41| +2450824|66211|2450898|88978|1854110|5995|22727|88978|1854110|5995|22727|2|17|7|2|10982|214|493|42|30.60|69.76|16.04|2256.24|673.68|1285.20|2929.92|4.85|511.99|644.28|161.69|166.54|805.97|810.82|-1123.51| +2450824|66211|2450867|88978|1854110|5995|22727|88978|1854110|5995|22727|2|51|1|2|15685|97|493|71|64.70|174.69|174.69|0.00|12402.99|4593.70|12402.99|372.08|0.00|2852.07|12402.99|12775.07|15255.06|15627.14|7809.29| +2450824|66211|2450899|88978|1854110|5995|22727|88978|1854110|5995|22727|2|99|6|5|9220|118|493|59|31.02|56.76|45.97|636.61|2712.23|1830.18|3348.84|244.10|0.00|1473.23|2712.23|2956.33|4185.46|4429.56|882.05| +2450824|66211|2450873|88978|1854110|5995|22727|88978|1854110|5995|22727|2|22|11|3|12811|27|493|14|51.32|89.29|7.14|1150.10|99.96|718.48|1250.06|0.99|0.00|212.38|99.96|100.95|312.34|313.33|-618.52| +2450824|66211|2450848|88978|1854110|5995|22727|88978|1854110|5995|22727|2|44|4|3|2080|69|493|41|92.53|170.25|6.81|6701.04|279.21|3793.73|6980.25|2.34|44.67|3210.71|234.54|236.88|3445.25|3447.59|-3559.19| +2450824|66211|2450858|88978|1854110|5995|22727|88978|1854110|5995|22727|2|30|2|4|14534|193|493|58|45.36|125.19|41.31|4865.04|2395.98|2630.88|7261.02|37.37|1773.02|3412.14|622.96|660.33|4035.10|4072.47|-2007.92| +2450824|66211|2450839|88978|1854110|5995|22727|88978|1854110|5995|22727|2|77|9|4|6778|259|493|30|30.58|47.39|39.80|227.70|1194.00|917.40|1421.70|0.00|0.00|14.10|1194.00|1194.00|1208.10|1208.10|276.60| +2450824|66211|2450864|88978|1854110|5995|22727|88978|1854110|5995|22727|2|12|20|2|7250|250|493|77|98.87|251.12|110.49|10828.51|8507.73|7612.99|19336.24|340.30|0.00|0.00|8507.73|8848.03|8507.73|8848.03|894.74| +2450824|66211|2450904|88978|1854110|5995|22727|88978|1854110|5995|22727|2|100|17|2|16610|242|493|30|61.25|66.76|50.73|480.90|1521.90|1837.50|2002.80|60.87|0.00|200.10|1521.90|1582.77|1722.00|1782.87|-315.60| +2450824|66211|2450862|88978|1854110|5995|22727|88978|1854110|5995|22727|2|42|18|3|9502|73|493|11|1.35|2.63|0.13|27.50|1.43|14.85|28.93|0.11|0.00|12.43|1.43|1.54|13.86|13.97|-13.42| +2450824|71485|2450827|83228|162271|6561|35604|83228|162271|6561|35604|2|9|6|1|17074|295|494|73|16.53|44.13|3.97|2931.68|289.81|1206.69|3221.49|23.18|0.00|965.79|289.81|312.99|1255.60|1278.78|-916.88| +2450824|71485|2450877|83228|162271|6561|35604|83228|162271|6561|35604|2|48|6|2|2674|144|494|81|42.76|121.01|39.93|6567.48|3234.33|3463.56|9801.81|32.34|0.00|1862.19|3234.33|3266.67|5096.52|5128.86|-229.23| +2450824|71485|2450854|83228|162271|6561|35604|83228|162271|6561|35604|2|88|3|3|16576|36|494|51|71.24|76.22|16.76|3032.46|854.76|3633.24|3887.22|68.38|0.00|116.28|854.76|923.14|971.04|1039.42|-2778.48| +2450824|71485|2450899|83228|162271|6561|35604|83228|162271|6561|35604|2|84|20|5|434|242|494|83|96.34|184.00|49.68|11148.56|4123.44|7996.22|15272.00|371.10|0.00|2901.68|4123.44|4494.54|7025.12|7396.22|-3872.78| +2450824|71485|2450872|83228|162271|6561|35604|83228|162271|6561|35604|2|57|11|3|8728|150|494|48|48.51|51.90|51.90|0.00|2491.20|2328.48|2491.20|174.38|0.00|946.56|2491.20|2665.58|3437.76|3612.14|162.72| +2450824|71485|2450883|83228|162271|6561|35604|83228|162271|6561|35604|2|21|17|3|12265|60|494|60|29.65|81.24|53.61|1657.80|3216.60|1779.00|4874.40|32.16|0.00|340.80|3216.60|3248.76|3557.40|3589.56|1437.60| +2450824|71485|2450895|83228|162271|6561|35604|83228|162271|6561|35604|2|93|17|2|14140|45|494|90|69.74|139.48|76.71|5649.30|6903.90|6276.60|12553.20|345.19|0.00|3263.40|6903.90|7249.09|10167.30|10512.49|627.30| +2450824|71485|2450901|83228|162271|6561|35604|83228|162271|6561|35604|2|49|8|1|9685|59|494|96|60.44|119.06|17.85|9716.16|1713.60|5802.24|11429.76|119.95|0.00|2400.00|1713.60|1833.55|4113.60|4233.55|-4088.64| +2450824|71485|2450848|83228|162271|6561|35604|83228|162271|6561|35604|2|38|3|5|15626|114|494|12|29.75|80.02|57.61|268.92|691.32|357.00|960.24|13.82|0.00|86.40|691.32|705.14|777.72|791.54|334.32| +2450824|71485|2450850|83228|162271|6561|35604|83228|162271|6561|35604|2|59|10|5|5930|224|494|96|62.32|95.34|40.99|5217.60|3935.04|5982.72|9152.64|39.35|0.00|2562.24|3935.04|3974.39|6497.28|6536.63|-2047.68| +2450824|71485|2450841|83228|162271|6561|35604|83228|162271|6561|35604|2|20|14|4|8797|62|494|55|80.91|214.41|102.91|6132.50|5660.05|4450.05|11792.55|56.60|0.00|707.30|5660.05|5716.65|6367.35|6423.95|1210.00| +2450824|75467|2450898|2563|231129|2095|39861|2563|231129|2095|39861|2|6|5|4|17408|254|495|7|62.56|101.34|39.52|432.74|276.64|437.92|709.38|16.59|0.00|49.63|276.64|293.23|326.27|342.86|-161.28| +2450824|75467|2450872|2563|231129|2095|39861|2563|231129|2095|39861|2|86|7|5|12097|159|495|40|96.82|128.77|99.15|1184.80|3966.00|3872.80|5150.80|47.59|3371.10|1905.60|594.90|642.49|2500.50|2548.09|-3277.90| +2450824|75467|2450858|2563|231129|2095|39861|2563|231129|2095|39861|2|37|7|5|8446|69|495|52|96.91|260.68|70.38|9895.60|3659.76|5039.32|13555.36|146.39|0.00|2846.48|3659.76|3806.15|6506.24|6652.63|-1379.56| +2450824|75467|2450879|2563|231129|2095|39861|2563|231129|2095|39861|2|46|17|2|1592|95|495|68|3.15|7.43|0.52|469.88|35.36|214.20|505.24|2.47|0.00|115.60|35.36|37.83|150.96|153.43|-178.84| +2450824|75467|2450842|2563|231129|2095|39861|2563|231129|2095|39861|2|12|16|2|12361|127|495|94|61.27|129.89|123.39|611.00|11598.66|5759.38|12209.66|0.00|4987.42|0.00|6611.24|6611.24|6611.24|6611.24|851.86| +2450824|75467|2450848|2563|231129|2095|39861|2563|231129|2095|39861|2|77|19|5|2227|250|495|13|30.25|55.96|5.59|654.81|72.67|393.25|727.48|0.00|0.00|87.23|72.67|72.67|159.90|159.90|-320.58| +2450824|67759|2450883|39102|1089078|7054|20230|39102|1089078|7054|20230|2|34|11|3|5581|150|496|80|86.60|190.52|15.24|14022.40|1219.20|6928.00|15241.60|24.50|402.33|7468.00|816.87|841.37|8284.87|8309.37|-6111.13| +2450824|67759|2450879|39102|1089078|7054|20230|39102|1089078|7054|20230|2|86|12|5|14252|184|496|63|67.15|138.32|24.89|7146.09|1568.07|4230.45|8714.16|141.12|0.00|2004.03|1568.07|1709.19|3572.10|3713.22|-2662.38| +2450824|67759|2450870|39102|1089078|7054|20230|39102|1089078|7054|20230|2|78|4|3|13276|296|496|43|57.91|111.18|81.16|1290.86|3489.88|2490.13|4780.74|279.19|0.00|1864.48|3489.88|3769.07|5354.36|5633.55|999.75| +2450824|67759|2450907|39102|1089078|7054|20230|39102|1089078|7054|20230|2|8|12|4|14410|71|496|38|8.25|19.47|2.53|643.72|96.14|313.50|739.86|1.26|32.68|266.00|63.46|64.72|329.46|330.72|-250.04| +2450824|67759|2450860|39102|1089078|7054|20230|39102|1089078|7054|20230|2|46|3|4|14756|263|496|95|94.98|201.35|159.06|4017.55|15110.70|9023.10|19128.25|151.10|0.00|190.95|15110.70|15261.80|15301.65|15452.75|6087.60| +2450824|68997|2450892|95466|391930|6441|47752|38363|852727|3238|45349|1|22|20|4|4267|202|497|5|94.02|130.68|28.74|509.70|143.70|470.10|653.40|0.00|0.00|45.70|143.70|143.70|189.40|189.40|-326.40| +2450824|68997|2450859|95466|391930|6441|47752|38363|852727|3238|45349|1|94|20|2|15266|139|497|18|18.97|56.15|5.05|919.80|90.90|341.46|1010.70|5.45|0.00|100.98|90.90|96.35|191.88|197.33|-250.56| +2450824|68997|2450882|95466|391930|6441|47752|38363|852727|3238|45349|1|99|6|4|968|123|497|63|21.89|52.75|42.72|631.89|2691.36|1379.07|3323.25|53.82|0.00|1528.38|2691.36|2745.18|4219.74|4273.56|1312.29| +2450824||||391930||47752||852727|||1|17|5|4|10012|100|497||4.07|||298.08|||386.64|||65.34|||153.90|155.67|-131.22| +2450824|68997|2450905|95466|391930|6441|47752|38363|852727|3238|45349|1|74|1|5|3847|117|497|46|55.66|79.03|57.69|981.64|2653.74|2560.36|3635.38|53.07|0.00|1744.78|2653.74|2706.81|4398.52|4451.59|93.38| +2450824|68997|2450903|95466|391930|6441|47752|38363|852727|3238|45349|1|17|17|3|9512|195|497|65|36.64|39.93|26.75|856.70|1738.75|2381.60|2595.45|104.32|0.00|856.05|1738.75|1843.07|2594.80|2699.12|-642.85| +2450824|68997|2450845|95466|391930|6441|47752|38363|852727|3238|45349|1|68|8|5|4342|154|497|96|48.50|144.04|116.67|2627.52|11200.32|4656.00|13827.84|112.00|0.00|6913.92|11200.32|11312.32|18114.24|18226.24|6544.32| +2450824|68997|2450874|95466|391930|6441|47752|38363|852727|3238|45349|1|22|1|4|10250|262|497|83|67.38|117.24|25.79|7590.35|2140.57|5592.54|9730.92|0.00|171.24|2919.11|1969.33|1969.33|4888.44|4888.44|-3623.21| +2450824|68997|2450887|95466|391930|6441|47752|38363|852727|3238|45349|1|73|4|4|7558|180|497|74|23.97|34.75|10.77|1774.52|796.98|1773.78|2571.50|7.96|0.00|1156.62|796.98|804.94|1953.60|1961.56|-976.80| +2450824|68997|2450863|95466|391930|6441|47752|38363|852727|3238|45349|1|19|6|3|1342|94|497|4|9.52|13.23|12.17|4.24|48.68|38.08|52.92|2.92|0.00|17.44|48.68|51.60|66.12|69.04|10.60| +2450824|68997|2450850|95466|391930|6441|47752|38363|852727|3238|45349|1|2|2|4|12952|120|497|87|98.52|98.52|21.67|6685.95|1885.29|8571.24|8571.24|18.85|0.00|3427.80|1885.29|1904.14|5313.09|5331.94|-6685.95| +2450824|68997|2450850|95466|391930|6441|47752|38363|852727|3238|45349|1|26|7|5|4418|169|497|83|68.00|129.20|71.06|4825.62|5897.98|5644.00|10723.60|231.20|117.95|1608.54|5780.03|6011.23|7388.57|7619.77|136.03| +2450824|73704|2450880|44990|90870|5103|16583|44990|90870|5103|16583|1|4|10|2|4052|244|498|35|83.59|242.41|169.68|2545.55|5938.80|2925.65|8484.35|296.94|0.00|1781.50|5938.80|6235.74|7720.30|8017.24|3013.15| +2450824|73704|2450846|44990|90870|5103|16583|44990|90870|5103|16583|1|77|8|3|14185|113|498|4|96.99|261.87|227.82|136.20|911.28|387.96|1047.48|27.33|0.00|167.56|911.28|938.61|1078.84|1106.17|523.32| +2450824|73704|2450884|44990|90870|5103|16583|44990|90870|5103|16583|1|59|5|1|5311|291|498|62|6.90|12.62|5.17|461.90|320.54|427.80|782.44|19.23|0.00|391.22|320.54|339.77|711.76|730.99|-107.26| +2450824|73704|2450862|44990|90870|5103|16583|44990|90870|5103|16583|1|9|7|2|12001|249|498|58|61.95|76.81|21.50|3207.98|1247.00|3593.10|4454.98|49.88|0.00|1826.42|1247.00|1296.88|3073.42|3123.30|-2346.10| +2450824|73704|2450860|44990|90870|5103|16583|44990|90870|5103|16583|1|63|12|1|7346|129|498|78|93.24|251.74|52.86|15512.64|4123.08|7272.72|19635.72|123.69|0.00|5497.44|4123.08|4246.77|9620.52|9744.21|-3149.64| +2450824|73704|2450844|44990|90870|5103|16583|44990|90870|5103|16583|1|9|6|4|15224|139|498|72|89.80|226.29|108.61|8472.96|7819.92|6465.60|16292.88|78.19|0.00|8146.08|7819.92|7898.11|15966.00|16044.19|1354.32| +2450824|73704|2450830|44990|90870|5103|16583|44990|90870|5103|16583|1|64|14|5|16172|97|498|17|18.01|25.03|11.26|234.09|191.42|306.17|425.51|6.20|88.05|161.67|103.37|109.57|265.04|271.24|-202.80| +2450824|73704|2450899|44990|90870|5103|16583|44990|90870|5103|16583|1|93|1|4|5011|20|498|27|76.15|146.96|79.35|1825.47|2142.45|2056.05|3967.92|107.12|0.00|356.94|2142.45|2249.57|2499.39|2606.51|86.40| +2450824|73704|2450912|44990|90870|5103|16583|44990|90870|5103|16583|1|19|20|4|12830|212|498|35|84.04|197.49|63.19|4700.50|2211.65|2941.40|6912.15|2.65|1946.25|2488.15|265.40|268.05|2753.55|2756.20|-2676.00| +2450824|73704|2450880|44990|90870|5103|16583|44990|90870|5103|16583|1|81|4|1|9560|227|498|43|96.91|274.25|85.01|8137.32|3655.43|4167.13|11792.75|36.55|0.00|4481.03|3655.43|3691.98|8136.46|8173.01|-511.70| +2450824|73704|2450841|44990|90870|5103|16583|44990|90870|5103|16583|1|66|4|1|13681|37|498|41|39.56|39.56|1.58|1557.18|64.78|1621.96|1621.96|2.38|34.98|97.17|29.80|32.18|126.97|129.35|-1592.16| +2450824|73704|2450866|44990|90870|5103|16583|44990|90870|5103|16583|1|37|12|2|7862|137|498|9|86.82|103.31|98.14|46.53|883.26|781.38|929.79|17.66|0.00|37.17|883.26|900.92|920.43|938.09|101.88| +2450824|35453|2450899|34034|947895|2909|1170|34034|947895|2909|1170|4|31|9|1|15956|21|499|98|4.36|12.42|3.60|864.36|352.80|427.28|1217.16|0.00|0.00|182.28|352.80|352.80|535.08|535.08|-74.48| +2450824|35453|2450882|34034|947895|2909|1170|34034|947895|2909|1170|4|75|19|2|10843|64|499|46|62.17|92.01|82.80|423.66|3808.80|2859.82|4232.46|114.26|0.00|253.92|3808.80|3923.06|4062.72|4176.98|948.98| +2450824|35453|2450909|34034|947895|2909|1170|34034|947895|2909|1170|4|57|19|4|16381|279|499|100|61.53|158.74|36.51|12223.00|3651.00|6153.00|15874.00|219.06|0.00|7937.00|3651.00|3870.06|11588.00|11807.06|-2502.00| +2450824|35453|2450852|34034|947895|2909|1170|34034|947895|2909|1170|4|11|15|3|10075|188|499|67|78.07|175.65|40.39|9062.42|2706.13|5230.69|11768.55|162.36|0.00|4001.24|2706.13|2868.49|6707.37|6869.73|-2524.56| +2450824|35453|2450865|34034|947895|2909|1170|34034|947895|2909|1170|4|21|14|2|13394|183|499|78|80.07|93.68|76.81|1315.86|5991.18|6245.46|7307.04|299.55|0.00|1315.08|5991.18|6290.73|7306.26|7605.81|-254.28| +2450824|35453|2450900|34034|947895|2909|1170|34034|947895|2909|1170|4|10|16|3|9566|199|499|76|38.80|65.57|44.58|1595.24|3388.08|2948.80|4983.32|203.28|0.00|49.40|3388.08|3591.36|3437.48|3640.76|439.28| +2450824|35453|2450912|34034|947895|2909|1170|34034|947895|2909|1170|4|107|7|5|9019|38|499|35|47.19|117.03|107.66|327.95|3768.10|1651.65|4096.05|188.40|0.00|2006.90|3768.10|3956.50|5775.00|5963.40|2116.45| +2450824|35453|2450883|34034|947895|2909|1170|34034|947895|2909|1170|4|40|8|1|13354|83|499|18|96.96|162.89|141.71|381.24|2550.78|1745.28|2932.02|229.57|0.00|820.80|2550.78|2780.35|3371.58|3601.15|805.50| +2450824|44195|2450890|6333|239380|6027|23874|6333|239380|6027|23874|4|61|14|2|16216|55|500|82|8.78|12.46|0.49|981.54|40.18|719.96|1021.72|2.81|0.00|91.84|40.18|42.99|132.02|134.83|-679.78| +2450824|44195|2450861|6333|239380|6027|23874|6333|239380|6027|23874|4|94|2|1|2701|259|500|20|72.24|202.27|50.56|3034.20|1011.20|1444.80|4045.40|10.11|0.00|889.80|1011.20|1021.31|1901.00|1911.11|-433.60| +2450824|44195|2450853|6333|239380|6027|23874|6333|239380|6027|23874|4|71|6|4|10972|285|500|21|24.09|49.38|13.33|757.05|279.93|505.89|1036.98|0.00|0.00|134.61|279.93|279.93|414.54|414.54|-225.96| +2450824|44195|2450828|6333|239380|6027|23874|6333|239380|6027|23874|4|102|19|3|3064|116|500|47|9.49|20.40|17.13|153.69|805.11|446.03|958.80|56.35|0.00|28.67|805.11|861.46|833.78|890.13|359.08| +2450824|45551|2450877|40487|403510|1380|8403|40487|403510|1380|8403|4|20|6|2|6907|4|501|47|39.49|66.73|4.67|2916.82|219.49|1856.03|3136.31|8.77|0.00|438.98|219.49|228.26|658.47|667.24|-1636.54| +2450824|45551|2450899|40487|403510|1380|8403|40487|403510|1380|8403|4|72|6|3|7550|91|501|21|54.86|69.12|33.17|754.95|696.57|1152.06|1451.52|48.75|0.00|653.10|696.57|745.32|1349.67|1398.42|-455.49| +2450824|45551|2450829|40487|403510|1380|8403|40487|403510|1380|8403|4|27|18|1|4651|162|501|52|47.96|57.07|31.95|1306.24|1661.40|2493.92|2967.64|66.45|0.00|1068.08|1661.40|1727.85|2729.48|2795.93|-832.52| +2450824|45551|2450826|40487|403510|1380|8403|40487|403510|1380|8403|4|8|11|3|15907|65|501|34|36.80|51.15|28.13|782.68|956.42|1251.20|1739.10|38.25|0.00|156.40|956.42|994.67|1112.82|1151.07|-294.78| +2450824|45551|2450897|40487|403510|1380|8403|40487|403510|1380|8403|4|97|9|1|8528|145|501|10|12.66|20.88|5.42|154.60|54.20|126.60|208.80|1.05|33.06|14.60|21.14|22.19|35.74|36.79|-105.46| +2450824|45551|2450887|40487|403510|1380|8403|40487|403510|1380|8403|4|92|4|3|673|215|501|72|29.21|30.37|10.93|1399.68|786.96|2103.12|2186.64|62.95|0.00|743.04|786.96|849.91|1530.00|1592.95|-1316.16| +2450824|45551|2450903|40487|403510|1380|8403|40487|403510|1380|8403|4|71|5|5|17626|282|501|36|4.38|8.76|4.81|142.20|173.16|157.68|315.36|0.00|0.00|88.20|173.16|173.16|261.36|261.36|15.48| +2450824|45551|2450870|40487|403510|1380|8403|40487|403510|1380|8403|4|13|10|5|16807|5|501|91|86.33|198.55|57.57|12829.18|5238.87|7856.03|18068.05|2.09|5186.48|3613.61|52.39|54.48|3666.00|3668.09|-7803.64| +2450824|45551|2450897|40487|403510|1380|8403|40487|403510|1380|8403|4|60|18|3|3619|85|501|88|6.26|10.76|8.39|208.56|738.32|550.88|946.88|66.44|0.00|179.52|738.32|804.76|917.84|984.28|187.44| +2450824|45551|2450833|40487|403510|1380|8403|40487|403510|1380|8403|4|29|3|1|794|26|501|4|55.31|164.82|84.05|323.08|336.20|221.24|659.28|10.08|0.00|309.84|336.20|346.28|646.04|656.12|114.96| +2450824|45551|2450893|40487|403510|1380|8403|40487|403510|1380|8403|4|98|12|4|12692|128|501|98|78.58|216.09|187.99|2753.80|18423.02|7700.84|21176.82|552.69|0.00|3811.22|18423.02|18975.71|22234.24|22786.93|10722.18| +2450824|45551|2450858|40487|403510|1380|8403|40487|403510|1380|8403|4|39|4|2|3926|293|501|45|49.25|110.81|96.40|648.45|4338.00|2216.25|4986.45|303.66|0.00|847.35|4338.00|4641.66|5185.35|5489.01|2121.75| +2450824|5876|2450831|2157|881110|1216|14522|2157|881110|1216|14522|4|17|2|5|661|116|502|43|81.95|135.21|32.45|4418.68|1395.35|3523.85|5814.03|83.72|0.00|2267.39|1395.35|1479.07|3662.74|3746.46|-2128.50| +2450824|5876|2450867|2157|881110|1216|14522|2157|881110|1216|14522|4|59|12|5|469|246|502|64|44.93|124.00|9.92|7301.12|634.88|2875.52|7936.00|50.79|0.00|2856.96|634.88|685.67|3491.84|3542.63|-2240.64| +2450824|5876|2450912|2157|881110|1216|14522|2157|881110|1216|14522|4|68|10|3|17890|81|502|38|12.97|29.96|1.79|1070.46|68.02|492.86|1138.48|3.40|0.00|352.64|68.02|71.42|420.66|424.06|-424.84| +2450824|5876|2450842|2157|881110|1216|14522|2157|881110|1216|14522|4|92|15|4|482|176|502|77|16.94|36.75|19.84|1302.07|1527.68|1304.38|2829.75|30.55|0.00|424.27|1527.68|1558.23|1951.95|1982.50|223.30| +2450824|64261|2450876|52750|427630|2157|27484|52750|427630|2157|27484|1|61|14|3|15392|300|503|34|49.38|54.81|51.52|111.86|1751.68|1678.92|1863.54|35.03|0.00|297.84|1751.68|1786.71|2049.52|2084.55|72.76| +2450824|64261|2450857|52750|427630|2157|27484|52750|427630|2157|27484|1|59|15|1|14791|200|503|32|26.56|71.18|32.03|1252.80|1024.96|849.92|2277.76|10.24|0.00|1070.40|1024.96|1035.20|2095.36|2105.60|175.04| +||2450827|52750|427630|2157|27484|52750||||||18||5486|127|503|||74.62|7.46|||||||155.80|||297.54|297.54|-1020.49| +2450824|64261|2450862|52750|427630|2157|27484|52750|427630|2157|27484|1|54|2|1|13196|159|503|57|15.35|27.78|10.83|966.15|617.31|874.95|1583.46|12.34|0.00|442.89|617.31|629.65|1060.20|1072.54|-257.64| +2450824|64261|2450827|52750|427630|2157|27484|52750|427630|2157|27484|1|67|11|4|10460|78|503|44|8.19|18.09|0.54|772.20|23.76|360.36|795.96|0.95|0.00|119.24|23.76|24.71|143.00|143.95|-336.60| +2450824|64261|2450887|52750|427630|2157|27484|52750|427630|2157|27484|1|102|3|1|14116|282|503|39|62.17|183.40|23.84|6222.84|929.76|2424.63|7152.60|0.00|0.00|0.00|929.76|929.76|929.76|929.76|-1494.87| +2450824|64261|2450904|52750|427630|2157|27484|52750|427630|2157|27484|1|101|20|2|9428|239|503|90|13.50|25.65|13.33|1108.80|1199.70|1215.00|2308.50|95.97|0.00|369.00|1199.70|1295.67|1568.70|1664.67|-15.30| +2450824|64261|2450890|52750|427630|2157|27484|52750|427630|2157|27484|1|99|2|4|9268|184|503|22|67.29|86.13|15.50|1553.86|341.00|1480.38|1894.86|30.69|0.00|568.26|341.00|371.69|909.26|939.95|-1139.38| +2450824|64261|2450850|52750|427630|2157|27484|52750|427630|2157|27484|1|62|2|3|2666|240|503|84|13.83|38.86|24.09|1240.68|2023.56|1161.72|3264.24|11.73|849.89|1566.60|1173.67|1185.40|2740.27|2752.00|11.95| +2450824|64261|2450870|52750|427630|2157|27484|52750|427630|2157|27484|1|86|10|5|2116|53|503|76|48.18|104.55|61.68|3258.12|4687.68|3661.68|7945.80|0.00|0.00|1589.16|4687.68|4687.68|6276.84|6276.84|1026.00| +2450824|64261|2450864|52750|427630|2157|27484|52750|427630|2157|27484|1|13|7|4|1882|78|503|74|25.37|64.69|61.45|239.76|4547.30|1877.38|4787.06|181.89|0.00|1053.02|4547.30|4729.19|5600.32|5782.21|2669.92| +2450824|64261|2450868|52750|427630|2157|27484|52750|427630|2157|27484|1|30|5|5|3686|182|503|70|56.02|127.72|8.94|8314.60|625.80|3921.40|8940.40|56.32|0.00|268.10|625.80|682.12|893.90|950.22|-3295.60| +2450824|64261|2450853|52750|427630|2157|27484|52750|427630|2157|27484|1|103|18|5|8792|248|503|53|86.08|100.71|78.55|1174.48|4163.15|4562.24|5337.63|83.26|0.00|1067.42|4163.15|4246.41|5230.57|5313.83|-399.09| +2450824|75112|2450839|39739|638521|5750|19332|39739|638521|5750|19332|2|80|9|4|9616|181|504|8|37.43|48.28|46.83|11.60|374.64|299.44|386.24|7.49|0.00|185.36|374.64|382.13|560.00|567.49|75.20| +2450824|75112|2450849|39739|638521|5750|19332|39739|638521|5750|19332|2|107|16|2|5474|67|504|54|60.21|107.17|30.00|4167.18|1620.00|3251.34|5787.18|64.80|0.00|2893.32|1620.00|1684.80|4513.32|4578.12|-1631.34| +2450824|75112|2450863|39739|638521|5750|19332|39739|638521|5750|19332|2|76|11|2|11623|197|504|53|82.49|160.03|107.22|2798.93|5682.66|4371.97|8481.59|397.78|0.00|1611.20|5682.66|6080.44|7293.86|7691.64|1310.69| +2450824|75112|2450886|39739|638521|5750|19332|39739|638521|5750|19332|2|21|5|2|2536|98|504|99|17.37|48.46|46.52|192.06|4605.48|1719.63|4797.54|46.05|0.00|1390.95|4605.48|4651.53|5996.43|6042.48|2885.85| +2450824|75112|2450866|39739|638521|5750|19332|39739|638521|5750|19332|2|75|19|2|4699|15|504|17|94.94|233.55|37.36|3335.23|635.12|1613.98|3970.35|12.70|0.00|1191.02|635.12|647.82|1826.14|1838.84|-978.86| +2450824|75112|2450892|39739|638521|5750|19332|39739|638521|5750|19332|2|60|19|3|11881|284|504|50|44.91|67.36|41.08|1314.00|2054.00|2245.50|3368.00|164.32|0.00|1448.00|2054.00|2218.32|3502.00|3666.32|-191.50| +2450824|75112|2450889|39739|638521|5750|19332|39739|638521|5750|19332|2|48|9|1|13576|260|504|26|1.24|3.17|2.59|15.08|67.34|32.24|82.42|4.71|0.00|37.70|67.34|72.05|105.04|109.75|35.10| +2450824|62384|2450864|23717|115786|6702|2818|23717|115786|6702|2818|4|80|5|4|16465|35|505|55|81.97|96.72|69.63|1489.95|3829.65|4508.35|5319.60|219.82|689.33|957.00|3140.32|3360.14|4097.32|4317.14|-1368.03| +|62384|2450897|23717|||2818|23717|115786|||4|77||4|14644||505||79.66||82.84|1076.92||4142.32|||0.00|2692.04||4393.83|6999.72|7085.87|165.36| +2450824|62384|2450901|23717|115786|6702|2818|23717|115786|6702|2818|4|10|3|1|10189|150|505|92|64.22|161.83|3.23|14591.20|297.16|5908.24|14888.36|20.80|0.00|1488.56|297.16|317.96|1785.72|1806.52|-5611.08| +2450824|62384|2450835|23717|115786|6702|2818|23717|115786|6702|2818|4|4|15|4|11602|95|505|29|78.96|236.09|165.26|2054.07|4792.54|2289.84|6846.61|143.77|0.00|2806.91|4792.54|4936.31|7599.45|7743.22|2502.70| +2450824|62384|2450834|23717|115786|6702|2818|23717|115786|6702|2818|4|31|12|2|7015|82|505|25|5.78|14.50|5.36|228.50|134.00|144.50|362.50|3.64|88.44|123.25|45.56|49.20|168.81|172.45|-98.94| +2450824|62384|2450868|23717|115786|6702|2818|23717|115786|6702|2818|4|31|9|5|17330|182|505|87|9.97|12.06|0.00|1049.22|0.00|867.39|1049.22|0.00|0.00|524.61|0.00|0.00|524.61|524.61|-867.39| +2450824|62384|2450851|23717|115786|6702|2818|23717|115786|6702|2818|4|80|3|1|10081|210|505|40|15.53|24.53|7.60|677.20|304.00|621.20|981.20|15.20|0.00|235.20|304.00|319.20|539.20|554.40|-317.20| +2450824|62384|2450830|23717|115786|6702|2818|23717|115786|6702|2818|4|104|18|2|15764|59|505|59|36.04|59.82|32.90|1588.28|1941.10|2126.36|3529.38|174.69|0.00|176.41|1941.10|2115.79|2117.51|2292.20|-185.26| +2450824|62384|2450883|23717|115786|6702|2818|23717|115786|6702|2818|4|40|5|3|10882|57|505|78|41.60|79.04|32.40|3637.92|2527.20|3244.80|6165.12|126.36|0.00|1170.78|2527.20|2653.56|3697.98|3824.34|-717.60| +2450824|62384|2450891|23717|115786|6702|2818|23717|115786|6702|2818|4|106|15|4|3874|180|505|95|87.59|216.34|188.21|2672.35|17879.95|8321.05|20552.30|1609.19|0.00|8015.15|17879.95|19489.14|25895.10|27504.29|9558.90| +2450824|62384|2450867|23717|115786|6702|2818|23717|115786|6702|2818|4|4|13|3|8839|255|505|78|10.43|30.03|4.20|2014.74|327.60|813.54|2342.34|16.38|0.00|538.20|327.60|343.98|865.80|882.18|-485.94| +2450824|62384|2450894|23717|115786|6702|2818|23717|115786|6702|2818|4|4|17|2|1742|72|505|25|31.90|53.27|38.88|359.75|972.00|797.50|1331.75|77.76|0.00|399.50|972.00|1049.76|1371.50|1449.26|174.50| +2450824|5861|2450901|27049|294741|1633|33325|27049|294741|1633|33325|4|37|17|3|15583|290|506|38|30.18|73.03|38.70|1304.54|1470.60|1146.84|2775.14|14.70|0.00|638.02|1470.60|1485.30|2108.62|2123.32|323.76| +2450824|5861|2450883|27049|294741|1633|33325|27049|294741|1633|33325|4|39|4|4|15836|32|506|14|9.04|12.83|8.21|64.68|114.94|126.56|179.62|6.89|0.00|64.54|114.94|121.83|179.48|186.37|-11.62| +2450824|5861|2450870|27049|294741|1633|33325|27049|294741|1633|33325|4|53|5|4|17455|197|506|29|95.19|121.84|112.09|282.75|3250.61|2760.51|3533.36|260.04|0.00|671.06|3250.61|3510.65|3921.67|4181.71|490.10| +2450824|5861|2450910|27049|294741|1633|33325|27049|294741|1633|33325|4|9|5|5|16382|186|506|5|52.41|147.79|47.29|502.50|236.45|262.05|738.95|9.45|0.00|169.95|236.45|245.90|406.40|415.85|-25.60| +2450824|5861|2450882|27049|294741|1633|33325|27049|294741|1633|33325|4|30|18|3|9091|143|506|12|49.05|104.47|50.14|651.96|601.68|588.60|1253.64|30.08|0.00|75.12|601.68|631.76|676.80|706.88|13.08| +2450824|5861|2450914|27049|294741|1633|33325|27049|294741|1633|33325|4|38|18|5|1855|193|506|65|17.70|35.40|12.74|1472.90|828.10|1150.50|2301.00|49.68|0.00|436.80|828.10|877.78|1264.90|1314.58|-322.40| +2450824|5861|2450843|27049|294741|1633|33325|27049|294741|1633|33325|4|52|13|4|2401|199|506|77|3.81|8.38|4.52|297.22|348.04|293.37|645.26|27.84|0.00|12.32|348.04|375.88|360.36|388.20|54.67| +2450824||||294741||||294741|1633||||11|5|769|21|506|20|5.31||13.95||279.00|106.20||22.32|||||322.00|344.32|172.80| +2450824|5861|2450849|27049|294741|1633|33325|27049|294741|1633|33325|4|23|16|4|14564|3|506|97|23.61|29.51|13.57|1546.18|1316.29|2290.17|2862.47|26.32|0.00|428.74|1316.29|1342.61|1745.03|1771.35|-973.88| +2450824|5861|2450842|27049|294741|1633|33325|27049|294741|1633|33325|4|17|12|1|7858|159|506|100|23.54|47.08|16.47|3061.00|1647.00|2354.00|4708.00|0.00|0.00|1082.00|1647.00|1647.00|2729.00|2729.00|-707.00| +2450824|5861|2450904|27049|294741|1633|33325|27049|294741|1633|33325|4|92|15|3|8296|255|506|10|6.38|9.05|0.81|82.40|8.10|63.80|90.50|0.64|0.00|18.10|8.10|8.74|26.20|26.84|-55.70| +2450824|5861|2450848|27049|294741|1633|33325|27049|294741|1633|33325|4|74|3|1|538|238|506|10|35.10|53.00|46.64|63.60|466.40|351.00|530.00|13.99|0.00|174.90|466.40|480.39|641.30|655.29|115.40| +2450824|5861|2450881|27049|294741|1633|33325|27049|294741|1633|33325|4|38|1|1|4018|118|506|42|93.35|202.56|131.66|2977.80|5529.72|3920.70|8507.52|218.97|1880.10|170.10|3649.62|3868.59|3819.72|4038.69|-271.08| +2450824|65749|2450889|91195|1162616|3990|43811|91195|1162616|3990|43811|1|23|4|4|14578|232|507|1|11.68|20.44|5.11|15.33|5.11|11.68|20.44|0.25|0.00|8.38|5.11|5.36|13.49|13.74|-6.57| +2450824|65749|2450843|91195|1162616|3990|43811|91195|1162616|3990|43811|1|2|5|5|11366|15|507|14|79.52|96.21|12.50|1171.94|175.00|1113.28|1346.94|3.50|0.00|282.80|175.00|178.50|457.80|461.30|-938.28| +2450824|65749|2450902|91195|1162616|3990|43811|91195|1162616|3990|43811|1|94|12|5|6196|152|507|1|24.46|44.02|10.12|33.90|10.12|24.46|44.02|0.33|1.82|2.20|8.30|8.63|10.50|10.83|-16.16| +2450824|65749|2450908|91195|1162616|3990|43811|91195|1162616|3990|43811|1|86|7|3|15332|170|507|59|37.11|79.41|9.52|4123.51|561.68|2189.49|4685.19|44.93|0.00|2295.69|561.68|606.61|2857.37|2902.30|-1627.81| +2450824|65749|2450833|91195|1162616|3990|43811|91195|1162616|3990|43811|1|74|11|2|6034|230|507|3|20.46|57.08|2.28|164.40|6.84|61.38|171.24|0.54|0.00|75.33|6.84|7.38|82.17|82.71|-54.54| +2450824|65749|2450893|91195|1162616|3990|43811|91195|1162616|3990|43811|1|6|7|1|5264|145|507|55|29.83|85.01|58.65|1449.80|3225.75|1640.65|4675.55|225.80|0.00|1589.50|3225.75|3451.55|4815.25|5041.05|1585.10| +2450824|65749|2450856|91195|1162616|3990|43811|91195|1162616|3990|43811|1|42|13|2|8722|118|507|71|46.51|139.53|80.92|4161.31|5745.32|3302.21|9906.63|172.35|0.00|990.45|5745.32|5917.67|6735.77|6908.12|2443.11| +2450824|66187|2450877|53653|1906982|2711|29640|53653|1906982|2711|29640|2|70|6|4|16354|28|508|12|91.64|201.60|197.56|48.48|2370.72|1099.68|2419.20|88.90|592.68|628.92|1778.04|1866.94|2406.96|2495.86|678.36| +2450824|66187|2450828|53653|1906982|2711|29640|53653|1906982|2711|29640|2|57|13|2|5948|20|508|38|78.54|163.36|22.87|5338.62|869.06|2984.52|6207.68|60.83|0.00|61.94|869.06|929.89|931.00|991.83|-2115.46| +2450824|66187|2450907|53653|1906982|2711|29640|53653|1906982|2711|29640|2|8|12|5|7976|145|508|70|57.99|171.65|34.33|9612.40|2403.10|4059.30|12015.50|216.27|0.00|1561.70|2403.10|2619.37|3964.80|4181.07|-1656.20| +2450824|66187|2450877|53653|1906982|2711|29640|53653|1906982|2711|29640|2|103|7|2|15205|49|508|24|92.06|256.84|82.18|4191.84|1972.32|2209.44|6164.16|39.44|0.00|2958.72|1972.32|2011.76|4931.04|4970.48|-237.12| +2450824|66187|2450904|53653|1906982|2711|29640|53653|1906982|2711|29640|2|4|12|5|8965|129|508|47|39.56|72.39|55.01|816.86|2585.47|1859.32|3402.33|15.51|1809.82|1394.49|775.65|791.16|2170.14|2185.65|-1083.67| +2450824|66187|2450907|53653|1906982|2711|29640|53653|1906982|2711|29640|2|91|17|2|10216|210|508|11|48.05|84.56|19.44|716.32|213.84|528.55|930.16|10.69|0.00|362.67|213.84|224.53|576.51|587.20|-314.71| +2450824|65719|2450828|33421|710068|315|1762|33421|710068|315|1762|4|42|11|2|3853|145|509|49|44.97|85.89|66.99|926.10|3282.51|2203.53|4208.61|89.28|2166.45|1472.94|1116.06|1205.34|2589.00|2678.28|-1087.47| +2450824|65719|2450906|33421|710068|315|1762|33421|710068|315|1762|4|4|18|5|8746|142|509|94|76.43|100.12|15.01|8000.34|1410.94|7184.42|9411.28|112.87|0.00|3575.76|1410.94|1523.81|4986.70|5099.57|-5773.48| +2450824|65719|2450829|33421|710068|315|1762|33421|710068|315|1762|4|93|5|5|1096|198|509|66|65.79|181.58|181.58|0.00|11984.28|4342.14|11984.28|838.89|0.00|5152.62|11984.28|12823.17|17136.90|17975.79|7642.14| +2450824|65719|2450905|33421|710068|315|1762|33421|710068|315|1762|4|70|11|2|10315|16|509|43|34.99|80.12|4.80|3238.76|206.40|1504.57|3445.16|2.06|0.00|1481.35|206.40|208.46|1687.75|1689.81|-1298.17| +2450824|65719|2450826|33421|710068|315|1762|33421|710068|315|1762|4|12|6|4|2191|95|509|24|91.26|169.74|59.40|2648.16|1425.60|2190.24|4073.76|99.79|0.00|529.44|1425.60|1525.39|1955.04|2054.83|-764.64| +2450824|65719|2450904|33421|710068|315|1762|33421|710068|315|1762|4|94|3|4|14809|164|509|87|95.17|209.37|64.90|12568.89|5646.30|8279.79|18215.19|2.82|5589.83|1274.55|56.47|59.29|1331.02|1333.84|-8223.32| +2450824|44286|2450856|532|1491942|2199|37656|44356|1516170|1180|15037|2|56|10|1|15034|154|510|67|99.40|286.27|14.31|18221.32|958.77|6659.80|19180.09|9.58|0.00|8438.65|958.77|968.35|9397.42|9407.00|-5701.03| +2450824|44286|2450846|532|1491942|2199|37656|44356|1516170|1180|15037|2|65|12|2|12661|284|510|86|12.90|31.47|25.80|487.62|2218.80|1109.40|2706.42|177.50|0.00|486.76|2218.80|2396.30|2705.56|2883.06|1109.40| +2450824|44286|2450852|532|1491942|2199|37656|44356|1516170|1180|15037|2|13|18|3|17356|111|510|70|59.26|156.44|134.53|1533.70|9417.10|4148.20|10950.80|459.55|3672.66|5475.40|5744.44|6203.99|11219.84|11679.39|1596.24| +2450824|44286|2450864|532|1491942|2199|37656|44356|1516170|1180|15037|2|69|7|2|10741|174|510|39|30.29|71.78|55.27|643.89|2155.53|1181.31|2799.42|86.22|0.00|1259.70|2155.53|2241.75|3415.23|3501.45|974.22| +2450824|44286|2450849|532|1491942|2199|37656|44356|1516170|1180|15037|2|55|8|4|14236|91|510|93|36.65|40.68|14.64|2421.72|1361.52|3408.45|3783.24|0.00|0.00|567.30|1361.52|1361.52|1928.82|1928.82|-2046.93| +2450824|44286|2450867|532|1491942|2199|37656|44356|1516170|1180|15037|2|66|13|5|1300|182|510|98|88.28|183.62|124.86|5758.48|12236.28|8651.44|17994.76|856.53|0.00|1438.64|12236.28|13092.81|13674.92|14531.45|3584.84| +2450824|44286|2450832|532|1491942|2199|37656|44356|1516170|1180|15037|2|14|15|2|9763|188|510|15|40.66|86.19|61.19|375.00|917.85|609.90|1292.85|82.60|0.00|336.00|917.85|1000.45|1253.85|1336.45|307.95| +2450824|44286|2450839|532|1491942|2199|37656|44356|1516170|1180|15037|2|17|2|5|14317|206|510|96|42.51|85.87|29.19|5441.28|2802.24|4080.96|8243.52|112.08|0.00|2802.24|2802.24|2914.32|5604.48|5716.56|-1278.72| +2450824|44286|2450852|532|1491942|2199|37656|44356|1516170|1180|15037|2|26|18|2|10492|190|510|11|86.05|97.23|40.83|620.40|449.13|946.55|1069.53|4.49|0.00|342.21|449.13|453.62|791.34|795.83|-497.42| +2450824|44286|2450845|532|1491942|2199|37656|44356|1516170|1180|15037|2|4|7|5|727|64|510|93|40.91|60.54|36.32|2252.46|3377.76|3804.63|5630.22|270.22|0.00|393.39|3377.76|3647.98|3771.15|4041.37|-426.87| +2450824|44286|2450871|532|1491942|2199|37656|44356|1516170|1180|15037|2|9|13|4|14860|1|510|45|45.73|76.82|54.54|1002.60|2454.30|2057.85|3456.90|122.71|0.00|172.80|2454.30|2577.01|2627.10|2749.81|396.45| +2450824|44286|2450864|532|1491942|2199|37656|44356|1516170|1180|15037|2|74|18|1|10939|181|510|31|85.97|222.66|97.97|3865.39|3037.07|2665.07|6902.46|151.85|0.00|2760.86|3037.07|3188.92|5797.93|5949.78|372.00| +2450824|44286|2450834|532|1491942|2199|37656|44356|1516170|1180|15037|2|43|15|1|5722|260|510|96|67.70|108.32|40.07|6552.00|3846.72|6499.20|10398.72|76.93|0.00|1247.04|3846.72|3923.65|5093.76|5170.69|-2652.48| +2450824|15021|2450890|12144|453957|1817|16077|12144|453957|1817|16077|1|98|3|3|6496|150|511|17|59.27|136.32|57.25|1344.19|973.25|1007.59|2317.44|77.86|0.00|231.71|973.25|1051.11|1204.96|1282.82|-34.34| +2450824|15021|2450909|12144|453957|1817|16077|12144|453957|1817|16077|1|29|5|5|10579|275|511|15|35.52|81.69|24.50|857.85|367.50|532.80|1225.35|0.07|363.82|343.05|3.68|3.75|346.73|346.80|-529.12| +2450824|15021|2450844|12144|453957|1817|16077|12144|453957|1817|16077|1|2|17|4|4894|20|511|50|57.75|162.27|94.11|3408.00|4705.50|2887.50|8113.50|105.87|2588.02|973.50|2117.48|2223.35|3090.98|3196.85|-770.02| +2450824|15021|2450829|12144|453957|1817|16077|12144|453957|1817|16077|1|21|6|2|12424|37|511|9|46.47|111.06|29.98|729.72|269.82|418.23|999.54|0.00|186.17|99.90|83.65|83.65|183.55|183.55|-334.58| +2450824|15021|2450854|12144|453957|1817|16077|12144|453957|1817|16077|1|35|8|3|13792|18|511|47|93.00|273.42|259.74|642.96|12207.78|4371.00|12850.74|122.07|0.00|2569.96|12207.78|12329.85|14777.74|14899.81|7836.78| +2450824|15021|2450892|12144|453957|1817|16077|12144|453957|1817|16077|1|91|15|3|5666|167|511|76|36.99|83.22|18.30|4933.92|1390.80|2811.24|6324.72|112.65|139.08|3162.36|1251.72|1364.37|4414.08|4526.73|-1559.52| +2450824|15021|2450905|12144|453957|1817|16077|12144|453957|1817|16077|1|33|3|1|5947|252|511|8|87.98|129.33|71.13|465.60|569.04|703.84|1034.64|0.00|0.00|103.44|569.04|569.04|672.48|672.48|-134.80| +2450824|15021|2450841|12144|453957|1817|16077|12144|453957|1817|16077|1|47|19|5|2383|90|511|80|34.91|84.48|27.87|4528.80|2229.60|2792.80|6758.40|0.00|1382.35|67.20|847.25|847.25|914.45|914.45|-1945.55| +2450824|15021|2450838|12144|453957|1817|16077|12144|453957|1817|16077|1|53|14|4|7504|254|511|87|36.63|73.26|43.95|2549.97|3823.65|3186.81|6373.62|267.65|0.00|2421.21|3823.65|4091.30|6244.86|6512.51|636.84| +2450824|15021|2450858|12144|453957|1817|16077|12144|453957|1817|16077|1|39|18|5|1036|33|511|71|97.86|268.13|150.15|8376.58|10660.65|6948.06|19037.23|639.63|0.00|6662.64|10660.65|11300.28|17323.29|17962.92|3712.59| +2450824|15021|2450866|12144|453957|1817|16077|12144|453957|1817|16077|1|94|11|4|4406|174|511|40|70.70|162.61|123.58|1561.20|4943.20|2828.00|6504.40|49.43|0.00|2211.20|4943.20|4992.63|7154.40|7203.83|2115.20| +2450824|15021|2450880|12144|453957|1817|16077|12144|453957|1817|16077|1|83|11|2|12758|212|511|37|18.94|54.73|4.37|1863.32|161.69|700.78|2025.01|12.93|0.00|951.64|161.69|174.62|1113.33|1126.26|-539.09| +2450824|15021|2450871|12144|453957|1817|16077|12144|453957|1817|16077|1|88|5|2|13723|41|511|86|89.80|167.02|15.03|13071.14|1292.58|7722.80|14363.72|38.77|0.00|1005.34|1292.58|1331.35|2297.92|2336.69|-6430.22| +2450824|15021|2450886|12144|453957|1817|16077|12144|453957|1817|16077|1|72|1|4|10669|150|511|27|5.02|5.37|3.38|53.73|91.26|135.54|144.99|5.47|0.00|66.69|91.26|96.73|157.95|163.42|-44.28| +2450825|49952|2450843|79146|911227|2508|23836|79146|911227|2508|23836|1|58|10|5|14672|282|512|67|17.65|48.71|0.97|3198.58|64.99|1182.55|3263.57|1.29|0.00|783.23|64.99|66.28|848.22|849.51|-1117.56| +2450825|49952|2450865|79146|911227|2508|23836|79146|911227|2508|23836|1|64|3|4|14818|83|512|13|93.71|192.10|124.86|874.12|1623.18|1218.23|2497.30|16.23|1298.54|1173.64|324.64|340.87|1498.28|1514.51|-893.59| +2450825|49952|2450882|79146|911227|2508|23836|79146|911227|2508|23836|1|95|2|1|13426|204|512|91|99.53|169.20|79.52|8160.88|7236.32|9057.23|15397.20|289.45|0.00|0.00|7236.32|7525.77|7236.32|7525.77|-1820.91| +2450825|49952|2450842|79146|911227|2508|23836|79146|911227|2508|23836|1|104|18|4|6061|184|512|89|28.39|53.65|15.02|3438.07|1336.78|2526.71|4774.85|40.10|0.00|858.85|1336.78|1376.88|2195.63|2235.73|-1189.93| +2450825|49952|2450899|79146|911227|2508|23836|79146|911227|2508|23836|1|96|6|2|17917|21|512|39|34.03|80.99|41.30|1547.91|1610.70|1327.17|3158.61|96.64|0.00|1452.75|1610.70|1707.34|3063.45|3160.09|283.53| +2450825|49952|2450837|79146|911227|2508|23836|79146|911227|2508|23836|1|4|17|4|2966|178|512|67|88.06|120.64|55.49|4365.05|3717.83|5900.02|8082.88|148.71|0.00|2020.72|3717.83|3866.54|5738.55|5887.26|-2182.19| +2450825|49952|2450846|79146|911227|2508|23836|79146|911227|2508|23836|1|25|10|2|6616|108|512|76|80.67|201.67|179.48|1686.44|13640.48|6130.92|15326.92|136.40|0.00|2451.76|13640.48|13776.88|16092.24|16228.64|7509.56| +2450825|49952|2450850|79146|911227|2508|23836|79146|911227|2508|23836|1|69|17|2|16604|292|512|31|5.62|16.74|0.50|503.44|15.50|174.22|518.94|0.77|0.00|46.50|15.50|16.27|62.00|62.77|-158.72| +2450825|49952|2450853|79146|911227|2508|23836|79146|911227|2508|23836|1|3|10|2|16826|134|512|2|88.44|199.87|137.91|123.92|275.82|176.88|399.74|13.79|0.00|127.90|275.82|289.61|403.72|417.51|98.94| +2450825|49952|2450878|79146|911227|2508|23836|79146|911227|2508|23836|1|72|6|1|32|26|512|36|13.92|36.33|24.34|431.64|876.24|501.12|1307.88|61.68|105.14|392.04|771.10|832.78|1163.14|1224.82|269.98| +2450825|40221|2450901|30858|1495803|7148|42540|30858|1495803|7148|42540|2|105|9|2|8713|235|513|33|83.04|249.12|209.26|1315.38|6905.58|2740.32|8220.96|63.53|5317.29|3123.78|1588.29|1651.82|4712.07|4775.60|-1152.03| +2450825|40221|2450840|30858|1495803|7148|42540|30858|1495803|7148|42540|2|101|4|2|16057|26|513|80|6.00|17.88|6.07|944.80|485.60|480.00|1430.40|0.00|0.00|472.00|485.60|485.60|957.60|957.60|5.60| +2450825|40221|2450858|30858|1495803|7148|42540|30858|1495803|7148|42540|2|10|7|4|11821|98|513|62|50.36|102.23|47.02|3423.02|2915.24|3122.32|6338.26|204.06|0.00|1520.86|2915.24|3119.30|4436.10|4640.16|-207.08| +2450825|40221|2450850|30858|1495803|7148|42540|30858|1495803|7148|42540|2|31|5|1|12404|75|513|28|17.05|18.24|1.82|459.76|50.96|477.40|510.72|4.58|0.00|229.60|50.96|55.54|280.56|285.14|-426.44| +2450825|40221|2450827|30858|1495803|7148|42540|30858|1495803|7148|42540|2|50|1|1|11848|127|513|80|95.23|137.13|54.85|6582.40|4388.00|7618.40|10970.40|219.40|0.00|2412.80|4388.00|4607.40|6800.80|7020.20|-3230.40| +2450825|40221|2450870|30858|1495803|7148|42540|30858|1495803|7148|42540|2|83|14|4|5738|84|513|46|3.33|4.69|0.42|196.42|19.32|153.18|215.74|0.34|1.93|88.32|17.39|17.73|105.71|106.05|-135.79| +2450825|40221|2450883|30858|1495803|7148|42540|30858|1495803|7148|42540|2|9|12|3|7604|287|513|52|71.12|163.57|135.76|1446.12|7059.52|3698.24|8505.64|564.76|0.00|1870.96|7059.52|7624.28|8930.48|9495.24|3361.28| +2450825|40221|2450877|30858|1495803|7148|42540|30858|1495803|7148|42540|2|76|14|3|3542|270|513|48|80.38|180.05|1.80|8556.00|86.40|3858.24|8642.40|3.45|0.00|172.80|86.40|89.85|259.20|262.65|-3771.84| +2450825|40221|2450853|30858|1495803|7148|42540|30858|1495803|7148|42540|2|63|20|4|4336|172|513|78|91.05|222.16|6.66|16809.00|519.48|7101.90|17328.48|20.77|0.00|3118.44|519.48|540.25|3637.92|3658.69|-6582.42| +2450825|40221|2450879|30858|1495803|7148|42540|30858|1495803|7148|42540|2|91|2|3|4837|282|513|39|77.96|190.22|53.26|5341.44|2077.14|3040.44|7418.58|20.77|0.00|964.08|2077.14|2097.91|3041.22|3061.99|-963.30| +2450825|40221|2450892|30858|1495803|7148|42540|30858|1495803|7148|42540|2|13|12|3|6973|235|513|85|87.89|139.74|115.98|2019.60|9858.30|7470.65|11877.90|295.74|0.00|3325.20|9858.30|10154.04|13183.50|13479.24|2387.65| +2450825|40221|2450848|30858|1495803|7148|42540|30858|1495803|7148|42540|2|20|8|4|6145|46|513|58|10.16|21.23|14.64|382.22|849.12|589.28|1231.34|24.45|543.43|541.72|305.69|330.14|847.41|871.86|-283.59| +2450825|43788|2450844|91408|1548837|4666|495|74509|1048153|3982|12611|1|44|17|1|10472|169|514|78|33.73|95.79|46.93|3811.08|3660.54|2630.94|7471.62|109.81|0.00|447.72|3660.54|3770.35|4108.26|4218.07|1029.60| +2450825|43788|2450887|91408|1548837|4666|495|74509|1048153|3982|12611|1|75|13|3|3904|223|514|56|86.95|193.02|193.02|0.00|10809.12|4869.20|10809.12|135.11|8106.84|2702.00|2702.28|2837.39|5404.28|5539.39|-2166.92| +2450825|43788|2450897|91408|1548837|4666|495|74509|1048153|3982|12611|1|4|4|5|979|45|514|52|52.97|151.49|49.99|5278.00|2599.48|2754.44|7877.48|19.49|649.87|1023.88|1949.61|1969.10|2973.49|2992.98|-804.83| +2450825|43788|2450885|91408|1548837|4666|495|74509|1048153|3982|12611|1|76|13|4|16375|178|514|79|27.39|79.15|0.00|6252.85|0.00|2163.81|6252.85|0.00|0.00|2188.30|0.00|0.00|2188.30|2188.30|-2163.81| +2450825|43788|2450835|91408|1548837|4666|495|74509|1048153|3982|12611|1|30|17|1|11540|42|514|37|23.90|70.98|19.16|1917.34|708.92|884.30|2626.26|56.71|0.00|814.00|708.92|765.63|1522.92|1579.63|-175.38| +2450825|43788|2450859|91408|1548837|4666|495|74509|1048153|3982|12611|1|44|12|4|487|12|514|85|77.58|209.46|85.87|10505.15|7298.95|6594.30|17804.10|0.00|0.00|1067.60|7298.95|7298.95|8366.55|8366.55|704.65| +2450825|43788|2450840|91408|1548837|4666|495|74509|1048153|3982|12611|1|93|16|5|2599|183|514|39|5.75|6.38|4.21|84.63|164.19|224.25|248.82|8.20|0.00|76.83|164.19|172.39|241.02|249.22|-60.06| +2450825|43788|2450908|91408|1548837|4666|495|74509|1048153|3982|12611|1|53|17|4|4027|55|514|21|16.27|35.46|23.04|260.82|483.84|341.67|744.66|24.19|0.00|148.89|483.84|508.03|632.73|656.92|142.17| +2450825|43788|2450900|91408|1548837|4666|495|74509|1048153|3982|12611|1|50|9|1|16540|121|514|7|49.23|143.75|56.06|613.83|392.42|344.61|1006.25|25.07|113.80|462.84|278.62|303.69|741.46|766.53|-65.99| +2450825|43788|2450834|91408|1548837|4666|495|74509|1048153|3982|12611|1|16|2|3|9541|285|514|39|79.97|82.36|28.82|2088.06|1123.98|3118.83|3212.04|0.00|0.00|1316.64|1123.98|1123.98|2440.62|2440.62|-1994.85| +2450825|43788|2450897|91408|1548837|4666|495|74509|1048153|3982|12611|1|5|2|3|14792|61|514|18|5.62|15.67|2.35|239.76|42.30|101.16|282.06|1.69|0.00|14.04|42.30|43.99|56.34|58.03|-58.86| +2450825|71460|2450835|44074|923589|4257|1325|44074|923589|4257|1325|1|105|2|3|8696|38|515|93|72.92|82.39|6.59|7049.40|612.87|6781.56|7662.27|18.38|0.00|382.23|612.87|631.25|995.10|1013.48|-6168.69| +2450825|71460|2450901|44074|923589|4257|1325|44074|923589|4257|1325|1|9|16|2|15478|152|515|90|50.08|76.12|11.41|5823.90|1026.90|4507.20|6850.80|12.93|842.05|890.10|184.85|197.78|1074.95|1087.88|-4322.35| +2450825|71460|2450862|44074|923589|4257|1325|44074|923589|4257|1325|1|57|13|1|7994|132|515|9|22.32|28.34|9.63|168.39|86.67|200.88|255.06|0.00|0.00|35.64|86.67|86.67|122.31|122.31|-114.21| +2450825|71460|2450881|44074|923589|4257|1325|44074|923589|4257|1325|1|84|5|4|11518|225|515|40|20.61|47.81|7.17|1625.60|286.80|824.40|1912.40|1.83|195.02|248.40|91.78|93.61|340.18|342.01|-732.62| +2450825|71460|2450838|44074|923589|4257|1325|44074|923589|4257|1325|1|94|2|4|4262|298|515|75|89.90|171.70|75.54|7212.00|5665.50|6742.50|12877.50|56.65|0.00|1674.00|5665.50|5722.15|7339.50|7396.15|-1077.00| +2450825|71460|2450895|44074|923589|4257|1325|44074|923589|4257|1325|1|46|1|2|106|61|515|73|91.51|243.41|4.86|17414.15|354.78|6680.23|17768.93|28.38|0.00|2309.72|354.78|383.16|2664.50|2692.88|-6325.45| +2450825|71460|2450880|44074|923589|4257|1325|44074|923589|4257|1325|1|11|9|2|15860|98|515|99|53.28|64.46|33.51|3064.05|3317.49|5274.72|6381.54|165.87|0.00|2998.71|3317.49|3483.36|6316.20|6482.07|-1957.23| +2450825|57106|2450896|77890|1849662|6234|12406|77890|1849662|6234|12406|2|59|13|4|17617|292|516|83|82.91|218.05|209.32|724.59|17373.56|6881.53|18098.15|1216.14|0.00|3981.51|17373.56|18589.70|21355.07|22571.21|10492.03| +2450825|57106|2450835|77890|1849662|6234|12406|77890|1849662|6234|12406|2|1|19|4|16156|42|516|16|82.50|173.25|24.25|2384.00|388.00|1320.00|2772.00|31.04|0.00|526.56|388.00|419.04|914.56|945.60|-932.00| +2450825|57106|2450866|77890|1849662|6234|12406|77890|1849662|6234|12406|2|71|11|3|3674|209|516|23|62.81|77.25|23.17|1243.84|532.91|1444.63|1776.75|0.00|0.00|53.13|532.91|532.91|586.04|586.04|-911.72| +2450825|57106|2450847|77890|1849662|6234|12406|77890|1849662|6234|12406|2|60|1|2|1381|185|516|93|61.25|101.67|30.50|6618.81|2836.50|5696.25|9455.31|226.92|0.00|472.44|2836.50|3063.42|3308.94|3535.86|-2859.75| +2450825|57106|2450864|77890|1849662|6234|12406|77890|1849662|6234|12406|2|60|15|3|7909|144|516|29|54.31|61.91|6.19|1615.88|179.51|1574.99|1795.39|10.77|0.00|879.57|179.51|190.28|1059.08|1069.85|-1395.48| +2450825|57106|2450828|77890|1849662|6234|12406|77890|1849662|6234|12406|2|69|3|5|13843|65|516|99|25.98|62.35|58.60|371.25|5801.40|2572.02|6172.65|348.08|0.00|1789.92|5801.40|6149.48|7591.32|7939.40|3229.38| +2450825|57106|2450884|77890|1849662|6234|12406|77890|1849662|6234|12406|2|61|19|3|5911|298|516|81|20.40|49.98|45.98|324.00|3724.38|1652.40|4048.38|260.70|0.00|1497.69|3724.38|3985.08|5222.07|5482.77|2071.98| +2450825|57106|2450880|77890|1849662|6234|12406|77890|1849662|6234|12406|2|90|11|5|12544|213|516|50|74.94|122.15|25.65|4825.00|1282.50|3747.00|6107.50|12.82|0.00|366.00|1282.50|1295.32|1648.50|1661.32|-2464.50| +2450825|3113|2450886|4112|1902421|4730|19039|4112|1902421|4730|19039|2|16|6|5|10909|61|517|6|30.36|36.73|22.40|85.98|134.40|182.16|220.38|3.27|52.41|48.48|81.99|85.26|130.47|133.74|-100.17| +2450825|3113|2450892|4112|1902421|4730|19039|4112|1902421|4730|19039|2|79|1|5|14044|102|517|71|4.17|12.17|4.13|570.84|293.23|296.07|864.07|0.00|0.00|276.19|293.23|293.23|569.42|569.42|-2.84| +2450825|3113|2450855|4112|1902421|4730|19039|4112|1902421|4730|19039|2|54|19|1|10099|187|517|84|49.78|136.89|93.08|3680.04|7818.72|4181.52|11498.76|31.27|6254.97|114.24|1563.75|1595.02|1677.99|1709.26|-2617.77| +2450825|3113|2450830|4112|1902421|4730|19039|4112|1902421|4730|19039|2|3|13|2|11948|278|517|73|89.42|218.18|202.90|1115.44|14811.70|6527.66|15927.14|133.30|1481.17|4777.85|13330.53|13463.83|18108.38|18241.68|6802.87| +2450825|68213|2450856|19121|1175588|2102|37369|19121|1175588|2102|37369|2|33|12|4|10933|36|518|81|27.62|59.38|11.87|3848.31|961.47|2237.22|4809.78|16.44|778.79|1683.18|182.68|199.12|1865.86|1882.30|-2054.54| +2450825|68213|2450831|19121|1175588|2102|37369|19121|1175588|2102|37369|2|95|12|2|13136|260|518|6|16.57|21.20|1.27|119.58|7.62|99.42|127.20|0.15|0.00|43.20|7.62|7.77|50.82|50.97|-91.80| +2450825|68213|2450831|19121|1175588|2102|37369|19121|1175588|2102|37369|2|67|12|3|3314|203|518|75|61.54|77.54|58.93|1395.75|4419.75|4615.50|5815.50|220.98|0.00|2151.00|4419.75|4640.73|6570.75|6791.73|-195.75| +2450825|68213|2450875|19121|1175588|2102|37369|19121|1175588|2102|37369|2|68|12|4|12172|59|518|2|60.11|176.12|112.71|126.82|225.42|120.22|352.24|11.27|0.00|126.80|225.42|236.69|352.22|363.49|105.20| +2450825|68213|2450876|19121|1175588|2102|37369|19121|1175588|2102|37369|2|94|5|5|5506|50|518|49|34.48|45.16|30.70|708.54|1504.30|1689.52|2212.84|38.35|225.64|176.89|1278.66|1317.01|1455.55|1493.90|-410.86| +2450825|68213|2450827|19121|1175588|2102|37369|19121|1175588|2102|37369|2|83|8|4|8986|52|518|15|62.78|109.86|49.43|906.45|741.45|941.70|1647.90|59.31|0.00|675.60|741.45|800.76|1417.05|1476.36|-200.25| +2450825|68213|2450895|19121|1175588|2102|37369|19121|1175588|2102|37369|2|64|5|1|11924|149|518|36|97.28|124.51|90.89|1210.32|3272.04|3502.08|4482.36|229.04|0.00|1792.80|3272.04|3501.08|5064.84|5293.88|-230.04| +2450825|68213|2450906|19121|1175588|2102|37369|19121|1175588|2102|37369|2|49|8|4|3464|245|518|20|67.68|103.55|71.44|642.20|1428.80|1353.60|2071.00|0.00|300.04|372.60|1128.76|1128.76|1501.36|1501.36|-224.84| +2450825|68213|2450861|19121|1175588|2102|37369|19121|1175588|2102|37369|2|107|13|3|8204|253|518|2|67.29|162.84|102.58|120.52|205.16|134.58|325.68|0.00|0.00|127.00|205.16|205.16|332.16|332.16|70.58| +2450825|68213|2450878|19121|1175588|2102|37369|19121|1175588|2102|37369|2|29|19|4|4885|185|518|29|14.37|25.57|23.52|59.45|682.08|416.73|741.53|13.71|225.08|229.68|457.00|470.71|686.68|700.39|40.27| +2450825|68213|2450851|19121|1175588|2102|37369|19121|1175588|2102|37369|2|22|15|4|12823|172|518|26|52.32|124.52|102.10|582.92|2654.60|1360.32|3237.52|79.63|0.00|1165.32|2654.60|2734.23|3819.92|3899.55|1294.28| +2450825|68213|2450862|19121|1175588|2102|37369|19121|1175588|2102|37369|2|102|10|1|8684|133|518|5|23.84|54.11|41.66|62.25|208.30|119.20|270.55|0.00|0.00|56.80|208.30|208.30|265.10|265.10|89.10| +2450825|68213|2450886|19121|1175588|2102|37369|19121|1175588|2102|37369|2|43|7|2|13294|9|518|19|25.44|37.39|14.58|433.39|277.02|483.36|710.41|22.16|0.00|234.27|277.02|299.18|511.29|533.45|-206.34| +2450825|68213|2450881|19121|1175588|2102|37369|19121|1175588|2102|37369|2|102|16|2|3248|57|518|18|2.55|6.57|6.57|0.00|118.26|45.90|118.26|1.18|0.00|44.82|118.26|119.44|163.08|164.26|72.36| +2450825|71180|2450911|55929|1899717|658|44046|55929|1899717|658|44046|2|29|5|1|7120|268|519|48|50.96|105.99|52.99|2544.00|2543.52|2446.08|5087.52|76.30|0.00|1322.40|2543.52|2619.82|3865.92|3942.22|97.44| +2450825|71180|2450871|55929|1899717|658|44046|55929|1899717|658|44046|2|105|6|1|13669|107|519|72|47.99|82.54|59.42|1664.64|4278.24|3455.28|5942.88|49.20|3294.24|1425.60|984.00|1033.20|2409.60|2458.80|-2471.28| +2450825|71180|2450834|55929|1899717|658|44046|55929|1899717|658|44046|2|3|2|3|6880|121|519|26|42.81|107.45|75.21|838.24|1955.46|1113.06|2793.70|78.21|0.00|1145.30|1955.46|2033.67|3100.76|3178.97|842.40| +2450825|71180|2450890|55929|1899717|658|44046|55929|1899717|658|44046|2|57|19|1|728|193|519|82|88.37|254.50|218.87|2921.66|17947.34|7246.34|20869.00|0.00|17947.34|7721.12|0.00|0.00|7721.12|7721.12|-7246.34| +2450825|71180|2450846|55929|1899717|658|44046|55929|1899717|658|44046|2|8|19|4|6373|244|519|10|52.10|64.60|12.27|523.30|122.70|521.00|646.00|0.00|0.00|232.50|122.70|122.70|355.20|355.20|-398.30| +2450825|71180|2450894|55929|1899717|658|44046|55929|1899717|658|44046|2|12|1|2|17378|56|519|61|78.86|195.57|109.51|5249.66|6680.11|4810.46|11929.77|341.35|1803.62|2862.73|4876.49|5217.84|7739.22|8080.57|66.03| +2450825|66672|2450887|30136|1251609|3482|8508|30136|1251609|3482|8508|1|41|13|5|9608|74|520|20|3.37|7.75|6.82|18.60|136.40|67.40|155.00|4.09|0.00|77.40|136.40|140.49|213.80|217.89|69.00| +2450825|66672|2450854|30136|1251609|3482|8508|30136|1251609|3482|8508|1|3|18|4|13009|274|520|2|88.33|92.74|70.48|44.52|140.96|176.66|185.48|2.81|0.00|42.66|140.96|143.77|183.62|186.43|-35.70| +2450825|66672|2450914|30136|1251609|3482|8508|30136|1251609|3482|8508|1|65|12|4|12967|108|520|63|32.69|34.65|11.43|1462.86|720.09|2059.47|2182.95|64.80|0.00|108.99|720.09|784.89|829.08|893.88|-1339.38| +2450825|66672|2450860|30136|1251609|3482|8508|30136|1251609|3482|8508|1|36|11|3|13826|163|520|14|3.76|4.47|0.67|53.20|9.38|52.64|62.58|0.36|0.18|21.84|9.20|9.56|31.04|31.40|-43.44| +2450825|66672|2450895|30136|1251609|3482|8508|30136|1251609|3482|8508|1|7|19|5|5416|213|520|65|75.09|81.09|72.98|527.15|4743.70|4880.85|5270.85|237.18|0.00|526.50|4743.70|4980.88|5270.20|5507.38|-137.15| +2450825|66672|2450862|30136|1251609|3482|8508|30136|1251609|3482|8508|1|80|4|1|15697|127|520|5|44.48|119.20|54.83|321.85|274.15|222.40|596.00|5.48|0.00|95.35|274.15|279.63|369.50|374.98|51.75| +2450825|35397|2450881|39466|495142|6018|3294|39466|495142|6018|3294|2|103|16|2|9416|211|521|86|97.85|284.74|128.13|13468.46|11019.18|8415.10|24487.64|220.38|0.00|1223.78|11019.18|11239.56|12242.96|12463.34|2604.08| +2450825|35397|2450851|39466|495142|6018|3294|39466|495142|6018|3294|2|93|6|5|8174|262|521|62|53.33|101.32|68.89|2010.66|4271.18|3306.46|6281.84|42.71|0.00|2889.20|4271.18|4313.89|7160.38|7203.09|964.72| +2450825|35397|2450839|39466|495142|6018|3294|39466|495142|6018|3294|2|59|17|4|12454|43|521|12|10.89|21.01|5.04|191.64|60.48|130.68|252.12|0.00|0.00|100.80|60.48|60.48|161.28|161.28|-70.20| +2450825|35397|2450888|39466|495142|6018|3294|39466|495142|6018|3294|2|20|20|4|12421|146|521|86|11.87|12.58|7.04|476.44|605.44|1020.82|1081.88|0.00|0.00|508.26|605.44|605.44|1113.70|1113.70|-415.38| +2450825|35397|2450870|39466|495142|6018|3294|39466|495142|6018|3294|2|8|1|5|4279|181|521|24|23.90|71.22|54.83|393.36|1315.92|573.60|1709.28|78.95|0.00|512.64|1315.92|1394.87|1828.56|1907.51|742.32| +2450825|35397|2450861|39466|495142|6018|3294|39466|495142|6018|3294|2|52|3|3|12772|210|521|19|64.46|65.74|32.87|624.53|624.53|1224.74|1249.06|27.79|68.69|512.05|555.84|583.63|1067.89|1095.68|-668.90| +2450825|35397|2450833|39466|495142|6018|3294|39466|495142|6018|3294|2|81|9|2|8995|63|521|36|19.85|20.04|11.62|303.12|418.32|714.60|721.44|20.91|0.00|187.56|418.32|439.23|605.88|626.79|-296.28| +2450825|35397|2450886|39466|495142|6018|3294|39466|495142|6018|3294|2|68|9|4|8228|42|521|43|88.65|125.88|55.38|3031.50|2381.34|3811.95|5412.84|166.69|0.00|2110.87|2381.34|2548.03|4492.21|4658.90|-1430.61| +2450825|35397|2450878|39466||6018|3294||495142||||12||1|6908||521||15.21||||||1691.20||0.00|||1657.20|1775.20|1775.20|1048.80| +2450825|35397|2450904|39466|495142|6018|3294|39466|495142|6018|3294|2|66|7|2|8066|86|521|87|29.13|40.78|11.82|2519.52|1028.34|2534.31|3547.86|41.13|0.00|531.57|1028.34|1069.47|1559.91|1601.04|-1505.97| +2450825|35397|2450866|39466|495142|6018|3294|39466|495142|6018|3294|2|20|3|5|170|299|521|2|69.66|77.32|25.51|103.62|51.02|139.32|154.64|0.36|14.28|55.66|36.74|37.10|92.40|92.76|-102.58| +2450825|35397|2450829|39466|495142|6018|3294|39466|495142|6018|3294|2|61|10|3|14378|74|521|50|23.18|50.06|35.04|751.00|1752.00|1159.00|2503.00|41.17|928.56|1026.00|823.44|864.61|1849.44|1890.61|-335.56| +2450825|35397|2450843|39466|495142|6018|3294|39466|495142|6018|3294|2|80|8|2|602|45|521|83|38.44|94.17|45.20|4064.51|3751.60|3190.52|7816.11|337.64|0.00|1562.89|3751.60|4089.24|5314.49|5652.13|561.08| +2450825|35397|2450869|39466|495142|6018|3294|39466|495142|6018|3294|2|52|6|4|6350|72|521|93|66.81|177.04|83.20|8727.12|7737.60|6213.33|16464.72|0.00|0.00|5268.45|7737.60|7737.60|13006.05|13006.05|1524.27| +2450825|70818|2450908|39551|1686167|2830|33846|39551|1686167|2830|33846|4|94|14|4|1226|6|522|51|73.25|179.46|154.33|1281.63|7870.83|3735.75|9152.46|314.83|0.00|4393.14|7870.83|8185.66|12263.97|12578.80|4135.08| +2450825|70818|2450869|39551|1686167|2830|33846|39551|1686167|2830|33846|4|101|20|2|12302|261|522|90|41.90|105.58|74.96|2755.80|6746.40|3771.00|9502.20|269.85|0.00|569.70|6746.40|7016.25|7316.10|7585.95|2975.40| +||2450894||1686167|2830|||1686167|||4||||7100||522||54.81|63.57|31.78||572.04|986.58|||0.00|503.46|||1075.50|1092.66|-414.54| +2450825|70818|2450829|39551|1686167|2830|33846|39551|1686167|2830|33846|4|16|20|5|15940|204|522|22|6.47|11.45|8.35|68.20|183.70|142.34|251.90|9.18|0.00|62.92|183.70|192.88|246.62|255.80|41.36| +2450825|70818|2450902|39551|1686167|2830|33846|39551|1686167|2830|33846|4|43|8|3|6854|283|522|4|78.22|139.23|50.12|356.44|200.48|312.88|556.92|4.00|0.00|16.68|200.48|204.48|217.16|221.16|-112.40| +2450825|70818|2450908|39551|1686167|2830|33846|39551|1686167|2830|33846|4|85|11|2|6074|149|522|94|36.86|45.33|5.43|3750.60|510.42|3464.84|4261.02|5.10|0.00|1021.78|510.42|515.52|1532.20|1537.30|-2954.42| +2450825|70818|2450864|39551|1686167|2830|33846|39551|1686167|2830|33846|4|87|12|4|5443|268|522|83|2.13|4.11|1.35|229.08|112.05|176.79|341.13|2.24|0.00|57.27|112.05|114.29|169.32|171.56|-64.74| +2450825|70818|2450860|39551|1686167|2830|33846|39551|1686167|2830|33846|4|32|18|5|13250|228|522|32|82.73|210.13|149.19|1950.08|4774.08|2647.36|6724.16|334.18|0.00|1479.04|4774.08|5108.26|6253.12|6587.30|2126.72| +2450825|70818|2450893|39551|1686167|2830|33846|39551|1686167|2830|33846|4|90|17|2|14510|129|522|74|56.65|137.09|69.91|4971.32|5173.34|4192.10|10144.66|465.60|0.00|506.90|5173.34|5638.94|5680.24|6145.84|981.24| +2450825|70818|2450913|39551|1686167|2830|33846|39551|1686167|2830|33846|4|40|16|1|14923|192|522|27|9.51|21.39|13.47|213.84|363.69|256.77|577.53|29.09|0.00|109.62|363.69|392.78|473.31|502.40|106.92| +2450825|70818|2450858|39551|1686167|2830|33846|39551|1686167|2830|33846|4|24|11|5|17494|226|522|52|17.44|27.38|17.24|527.28|896.48|906.88|1423.76|35.85|0.00|683.28|896.48|932.33|1579.76|1615.61|-10.40| +2450825|70818|2450878|39551|1686167|2830|33846|39551|1686167|2830|33846|4|39|17|3|5006|292|522|50|50.71|120.18|72.10|2404.00|3605.00|2535.50|6009.00|180.25|0.00|721.00|3605.00|3785.25|4326.00|4506.25|1069.50| +2450825|12083|2450893|26149|1029782|4615|35048|26149|1029782|4615|35048|4|32|19|4|6746|160|523|45|70.09|91.11|43.73|2132.10|1967.85|3154.05|4099.95|5.90|1377.49|1311.75|590.36|596.26|1902.11|1908.01|-2563.69| +2450825|12083|2450887|26149|1029782|4615|35048|26149|1029782|4615|35048|4|31|4|4|14180|247|523|2|21.72|36.48|19.69|33.58|39.38|43.44|72.96|0.39|0.00|6.56|39.38|39.77|45.94|46.33|-4.06| +2450825|12083|2450843|26149|1029782|4615|35048|26149|1029782|4615|35048|4|10|12|1|10598|30|523|92|80.46|186.66|175.46|1030.40|16142.32|7402.32|17172.72|161.42|0.00|686.32|16142.32|16303.74|16828.64|16990.06|8740.00| +2450825|12083|2450855|26149|1029782|4615|35048|26149|1029782|4615|35048|4|70|10|5|12362|85|523|83|52.55|134.00|6.70|10565.90|556.10|4361.65|11122.00|27.80|0.00|889.76|556.10|583.90|1445.86|1473.66|-3805.55| +2450825|12083|2450869|26149|1029782|4615|35048|26149|1029782|4615|35048|4|70|17|1|14246|194|523|44|64.96|154.60|95.85|2585.00|4217.40|2858.24|6802.40|0.00|0.00|2516.80|4217.40|4217.40|6734.20|6734.20|1359.16| +2450825|12083|2450831|26149|1029782|4615|35048|26149|1029782|4615|35048|4|14|5|5|3202|100|523|74|48.31|116.42|90.80|1895.88|6719.20|3574.94|8615.08|67.19|0.00|1980.98|6719.20|6786.39|8700.18|8767.37|3144.26| +2450825|12083|2450848|26149|1029782|4615|35048|26149|1029782|4615|35048|4|91|20|3|13990|117|523|35|61.95|71.86|20.12|1810.90|704.20|2168.25|2515.10|21.12|0.00|704.20|704.20|725.32|1408.40|1429.52|-1464.05| +2450825|12083|2450831|26149|1029782|4615|35048|26149|1029782|4615|35048|4|81|14|2|13531|35|523|57|39.63|110.17|56.18|3077.43|3202.26|2258.91|6279.69|64.04|0.00|1381.11|3202.26|3266.30|4583.37|4647.41|943.35| +2450825|12083|2450891|26149|1029782|4615|35048|26149|1029782|4615|35048|4|13|1|1|14755|205|523|65|88.99|187.76|133.30|3539.90|8664.50|5784.35|12204.40|109.17|7104.89|4637.10|1559.61|1668.78|6196.71|6305.88|-4224.74| +2450825|12083|2450905|26149|1029782|4615|35048|26149|1029782|4615|35048|4|33|8|3|682|203|523|24|47.31|112.59|43.91|1648.32|1053.84|1135.44|2702.16|63.23|0.00|432.24|1053.84|1117.07|1486.08|1549.31|-81.60| +2450825|12083|2450913|26149|1029782|4615|35048|26149|1029782|4615|35048|4|1|2|3|5572|188|523|46|27.91|59.72|3.58|2582.44|164.68|1283.86|2747.12|4.94|0.00|494.04|164.68|169.62|658.72|663.66|-1119.18| +2450825|12083|2450840|26149|1029782|4615|35048|26149|1029782|4615|35048|4|64|6|3|10472|165|523|5|96.60|161.32|22.58|693.70|112.90|483.00|806.60|6.77|0.00|209.70|112.90|119.67|322.60|329.37|-370.10| +2450825|36081|2450863|32016|414067|4918|10169|32016|414067|4918|10169|1|21|15|1|13822|200|524|40|89.29|249.11|129.53|4783.20|5181.20|3571.60|9964.40|310.87|0.00|2291.60|5181.20|5492.07|7472.80|7783.67|1609.60| +2450825|36081|2450888|32016|414067|4918|10169|32016|414067|4918|10169|1|86|9|2|11852|285|524|63|32.74|40.92|11.04|1882.44|695.52|2062.62|2577.96|48.68|0.00|180.18|695.52|744.20|875.70|924.38|-1367.10| +2450825|36081|2450854|32016|414067|4918|10169|32016|414067|4918|10169|1|41|19|3|1124|209|524|79|4.69|13.27|11.54|136.67|911.66|370.51|1048.33|0.00|0.00|324.69|911.66|911.66|1236.35|1236.35|541.15| +2450825|36081|2450887|32016|414067|4918|10169|32016|414067|4918|10169|1|15|16|5|12775|28|524|42|98.48|293.47|111.51|7642.32|4683.42|4136.16|12325.74|374.67|0.00|2218.44|4683.42|5058.09|6901.86|7276.53|547.26| +2450825|36081|2450887|32016|414067|4918|10169|32016|414067|4918|10169|1|12|14|5|15704|167|524|3|85.35|215.08|202.17|38.73|606.51|256.05|645.24|0.00|0.00|167.76|606.51|606.51|774.27|774.27|350.46| +2450825|36081|2450846|32016|414067|4918|10169|32016|414067|4918|10169|1|18|5|1|10792|40|524|86|25.08|49.15|27.52|1860.18|2366.72|2156.88|4226.90|23.66|0.00|718.10|2366.72|2390.38|3084.82|3108.48|209.84| +2450825|36081|2450865|32016|414067|4918|10169|32016|414067|4918|10169|1|36|1|2|4618|38|524|78|44.89|115.81|103.07|993.72|8039.46|3501.42|9033.18|723.55|0.00|4245.54|8039.46|8763.01|12285.00|13008.55|4538.04| +2450825|36081|2450850|32016|414067|4918|10169|32016|414067|4918|10169|1|37|8|4|9974|141|524|46|16.88|31.90|6.38|1173.92|293.48|776.48|1467.40|8.80|0.00|132.02|293.48|302.28|425.50|434.30|-483.00| +2450825|36081|2450839|32016|414067|4918|10169|32016|414067|4918|10169|1|3|6|5|10276|26|524|50|47.77|114.64|11.46|5159.00|573.00|2388.50|5732.00|0.00|0.00|2350.00|573.00|573.00|2923.00|2923.00|-1815.50| +2450825|36081|2450901|32016|414067|4918|10169|32016|414067|4918|10169|1|91|16|3|2335|259|524|66|4.81|7.55|6.34|79.86|418.44|317.46|498.30|20.92|0.00|243.54|418.44|439.36|661.98|682.90|100.98| +2450825|36081|2450890|32016|414067|4918|10169|32016|414067|4918|10169|1|31|8|4|17161|93|524|79|50.81|116.35|2.32|9008.37|183.28|4013.99|9191.65|0.00|0.00|1378.55|183.28|183.28|1561.83|1561.83|-3830.71| +2450825|36081|2450845|32016|414067|4918|10169|32016|414067|4918|10169|1|10|1|2|11656|137|524|32|92.06|202.53|135.69|2138.88|4342.08|2945.92|6480.96|260.52|0.00|388.80|4342.08|4602.60|4730.88|4991.40|1396.16| +2450825|36081|2450896|32016|414067|4918|10169|32016|414067|4918|10169|1|80|20|3|13264|14|524|28|62.27|83.44|55.07|794.36|1541.96|1743.56|2336.32|92.51|0.00|864.36|1541.96|1634.47|2406.32|2498.83|-201.60| +2450825|7060|2450905|46648|1167714|6380|693|46648|1167714|6380|693|1|34|20|1|7399|195|525|92|72.11|168.01|119.28|4483.16|10973.76|6634.12|15456.92|768.16|0.00|5409.60|10973.76|11741.92|16383.36|17151.52|4339.64| +2450825|7060|2450843|46648|1167714|6380|693|46648|1167714|6380|693|1|76|10|3|806|147|525|63|5.96|12.99|3.89|573.30|245.07|375.48|818.37|19.60|0.00|286.02|245.07|264.67|531.09|550.69|-130.41| +2450825|7060|2450901|46648|1167714|6380|693|46648|1167714|6380|693|1|18|19|1|2288|240|525|38|46.82|50.09|43.57|247.76|1655.66|1779.16|1903.42|49.66|0.00|342.38|1655.66|1705.32|1998.04|2047.70|-123.50| +2450825|7060|2450900|46648|1167714|6380|693|46648|1167714|6380|693|1|17|5|1|1237|143|525|64|30.06|49.59|24.79|1587.20|1586.56|1923.84|3173.76|79.32|0.00|1332.48|1586.56|1665.88|2919.04|2998.36|-337.28| +2450825|7060|2450878|46648|1167714|6380|693|46648|1167714|6380|693|1|36|15|3|6181|34|525|51|18.28|50.08|32.05|919.53|1634.55|932.28|2554.08|32.85|539.40|408.51|1095.15|1128.00|1503.66|1536.51|162.87| +2450825|7060|2450852|46648|1167714|6380|693|46648|1167714|6380|693|1|101|15|3|12445|97|525|22|24.94|45.88|11.01|767.14|242.22|548.68|1009.36|5.52|104.15|262.24|138.07|143.59|400.31|405.83|-410.61| +2450825|7060|2450860|46648|1167714|6380|693|46648|1167714|6380|693|1|75|2|5|12898|2|525|13|88.64|92.18|67.29|323.57|874.77|1152.32|1198.34|16.79|314.91|227.63|559.86|576.65|787.49|804.28|-592.46| +2450825|7060|2450882|46648|1167714|6380|693|46648|1167714|6380|693|1|14|3|2|5138|165|525|58|76.57|168.45|82.54|4982.78|4787.32|4441.06|9770.10|0.00|0.00|3028.18|4787.32|4787.32|7815.50|7815.50|346.26| +2450825|7060|2450858|46648|1167714|6380|693|46648|1167714|6380|693|1|2|11|2|10934|234|525|62|30.54|66.88|58.85|497.86|3648.70|1893.48|4146.56|0.00|0.00|787.40|3648.70|3648.70|4436.10|4436.10|1755.22| +2450825|40879|2450879|78368|738771|3174|47977|59344|295230|4433|24436|1|101|20|3|4135|109|526|19|49.85|127.11|66.09|1159.38|1255.71|947.15|2415.09|100.45|0.00|965.96|1255.71|1356.16|2221.67|2322.12|308.56| +2450825|40879|2450914|78368|738771|3174|47977|59344|295230|4433|24436|1|84|11|2|3082|199|526|7|56.80|106.21|20.17|602.28|141.19|397.60|743.47|0.00|0.00|200.69|141.19|141.19|341.88|341.88|-256.41| +2450825|40879|2450842|78368|738771|3174|47977|59344|295230|4433|24436|1|15|13|4|7363|46|526|23|98.95|132.59|118.00|335.57|2714.00|2275.85|3049.57|189.98|0.00|792.81|2714.00|2903.98|3506.81|3696.79|438.15| +2450825|40879|2450863|78368|738771|3174|47977|59344|295230|4433|24436|1|30|18|5|1924|34|526|62|28.27|74.06|70.35|230.02|4361.70|1752.74|4591.72|392.55|0.00|0.00|4361.70|4754.25|4361.70|4754.25|2608.96| +2450825|40879|2450841|78368|738771|3174|47977|59344|295230|4433|24436|1|40|20|3|12565|291|526|65|32.52|64.06|42.92|1374.10|2789.80|2113.80|4163.90|55.79|0.00|1790.10|2789.80|2845.59|4579.90|4635.69|676.00| +2450825|40879|2450878|78368|738771|3174|47977|59344|295230|4433|24436|1|32|6|5|7375|223|526|95|84.03|172.26|25.83|13910.85|2453.85|7982.85|16364.70|147.23|0.00|3108.40|2453.85|2601.08|5562.25|5709.48|-5529.00| +2450825|40879|2450827|78368|738771|3174|47977|59344|295230|4433|24436|1|43|12|2|547|77|526|90|84.22|97.69|30.28|6066.90|2725.20|7579.80|8792.10|32.70|2316.42|2813.40|408.78|441.48|3222.18|3254.88|-7171.02| +2450825|56818|2450893|38224|1912875|5126|20577|27144|1285879|6930|30381|2|89|10|1|2065|21|527|89|55.59|155.65|38.91|10389.86|3462.99|4947.51|13852.85|34.62|0.00|1800.47|3462.99|3497.61|5263.46|5298.08|-1484.52| +2450825|56818|2450877|38224|1912875|5126|20577|27144|1285879|6930|30381|2|6|9|3|6937|93|527|62|34.37|90.04|16.20|4578.08|1004.40|2130.94|5582.48|60.26|0.00|1730.42|1004.40|1064.66|2734.82|2795.08|-1126.54| +2450825|56818|2450893|38224|1912875|5126|20577|27144|1285879|6930|30381|2|73|19|2|7406|85|527|63|51.40|115.65|27.75|5537.70|1748.25|3238.20|7285.95|87.41|0.00|2258.55|1748.25|1835.66|4006.80|4094.21|-1489.95| +2450825|56818|2450855|38224|1912875|5126|20577|27144|1285879|6930|30381|2|34|4|3|15193|292|527|35|37.18|62.09|8.07|1890.70|282.45|1301.30|2173.15|14.12|0.00|347.55|282.45|296.57|630.00|644.12|-1018.85| +2450825|56818|2450898|38224|1912875|5126|20577|27144|1285879|6930|30381|2|104|5|5|12979|126|527|23|77.77|220.86|150.18|1625.64|3454.14|1788.71|5079.78|34.54|0.00|1676.24|3454.14|3488.68|5130.38|5164.92|1665.43| +2450825|56818|2450866|38224|1912875|5126|20577|27144|1285879|6930|30381|2|55|16|4|15284|108|527|84|79.84|169.26|13.54|13080.48|1137.36|6706.56|14217.84|45.49|0.00|3127.32|1137.36|1182.85|4264.68|4310.17|-5569.20| +2450825|56818|2450827|38224|1912875|5126|20577|27144|1285879|6930|30381|2|89|5|1|4474|163|527|40|8.77|17.45|3.66|551.60|146.40|350.80|698.00|13.17|0.00|55.60|146.40|159.57|202.00|215.17|-204.40| +2450825|56818|2450915|38224|1912875|5126|20577|27144|1285879|6930|30381|2|37|16|1|13484|8|527|81|29.15|53.05|16.44|2965.41|1331.64|2361.15|4297.05|39.94|0.00|643.95|1331.64|1371.58|1975.59|2015.53|-1029.51| +2450825|56818|2450850|38224|1912875|5126|20577|27144|1285879|6930|30381|2|89|14|3|1874|244|527|77|27.73|57.12|19.42|2902.90|1495.34|2135.21|4398.24|74.76|0.00|2110.57|1495.34|1570.10|3605.91|3680.67|-639.87| +2450825|56818|2450832|38224|1912875|5126|20577|27144|1285879|6930|30381|2|12|3|2|16484|195|527|51|93.80|273.89|235.54|1955.85|12012.54|4783.80|13968.39|1081.12|0.00|278.97|12012.54|13093.66|12291.51|13372.63|7228.74| +2450825|42111|2450905|37246|722984|1075|4366|37246|722984|1075|4366|4|11|19|2|9787|186|528|55|64.47|190.18|186.37|209.55|10250.35|3545.85|10459.90|0.00|0.00|1778.15|10250.35|10250.35|12028.50|12028.50|6704.50| +2450825|42111|2450827|37246|722984|1075|4366|37246|722984|1075|4366|4|26|6|2|3508|156|528|84|67.87|127.59|21.69|8895.60|1821.96|5701.08|10717.56|18.21|0.00|4500.72|1821.96|1840.17|6322.68|6340.89|-3879.12| +2450825|42111|2450848|37246|722984|1075|4366|37246|722984|1075|4366|4|36|1|4|1264|47|528|8|88.31|168.67|48.91|958.08|391.28|706.48|1349.36|15.65|0.00|350.80|391.28|406.93|742.08|757.73|-315.20| +2450825|42111|2450887|37246|722984|1075|4366|37246|722984|1075|4366|4|1|16|1|4225|194|528|86|30.95|63.75|45.90|1535.10|3947.40|2661.70|5482.50|0.00|0.00|1808.58|3947.40|3947.40|5755.98|5755.98|1285.70| +2450825|42111|2450901|37246|722984|1075|4366|37246|722984|1075|4366|4|102|6|3|6814|173|528|58|77.29|91.97|68.97|1334.00|4000.26|4482.82|5334.26|240.01|0.00|2239.96|4000.26|4240.27|6240.22|6480.23|-482.56| +2450825|42111|2450878|37246|722984|1075|4366|37246|722984|1075|4366|4|44|9|3|1630|243|528|60|54.11|148.26|32.61|6939.00|1956.60|3246.60|8895.60|156.52|0.00|978.00|1956.60|2113.12|2934.60|3091.12|-1290.00| +2450825|42111|2450872|37246|722984|1075|4366|37246|722984|1075|4366|4|86|2|3|14188|263|528|34|73.44|165.24|109.05|1910.46|3707.70|2496.96|5618.16|0.00|0.00|1067.26|3707.70|3707.70|4774.96|4774.96|1210.74| +2450825|47624|2450898|42886|1066140|4594|33048|42886|1066140|4594|33048|1|33|2|4|12758|278|529|87|14.90|22.35|11.39|953.52|990.93|1296.30|1944.45|9.90|0.00|718.62|990.93|1000.83|1709.55|1719.45|-305.37| +2450825|47624|2450890|42886|1066140|4594|33048|42886|1066140|4594|33048|1|31|6|5|13723|58|529|80|28.95|69.19|53.96|1218.40|4316.80|2316.00|5535.20|86.33|0.00|1992.00|4316.80|4403.13|6308.80|6395.13|2000.80| +2450825|47624|2450843|42886|1066140|4594|33048|42886|1066140|4594|33048|1|81|10|2|10669|58|529|47|29.93|75.42|20.36|2587.82|956.92|1406.71|3544.74|38.27|0.00|637.79|956.92|995.19|1594.71|1632.98|-449.79| +2450825|47624|2450872|42886|1066140|4594|33048|42886|1066140|4594|33048|1|67|4|4|10058|173|529|27|81.96|182.77|56.65|3405.24|1529.55|2212.92|4934.79|24.62|1177.75|295.92|351.80|376.42|647.72|672.34|-1861.12| +2450825|47624|2450862|42886|1066140|4594|33048|42886|1066140|4594|33048|1|46|18|4|1024|290|529|93|57.95|159.94|70.37|8330.01|6544.41|5389.35|14874.42|196.33|0.00|3866.94|6544.41|6740.74|10411.35|10607.68|1155.06| +2450825|47624|2450906|42886|1066140|4594|33048|42886|1066140|4594|33048|1|16|11|2|6277|152|529|97|49.59|102.65|37.98|6272.99|3684.06|4810.23|9957.05|73.68|0.00|2986.63|3684.06|3757.74|6670.69|6744.37|-1126.17| +2450825|47624|2450915|42886|1066140|4594|33048|42886|1066140|4594|33048|1|58|15|5|5617|172|529|99|90.74|149.72|118.27|3113.55|11708.73|8983.26|14822.28|117.08|0.00|1778.04|11708.73|11825.81|13486.77|13603.85|2725.47| +2450825|47624|2450861|42886|1066140|4594|33048|42886|1066140|4594|33048|1|8|20|2|12463|291|529|39|67.13|179.90|129.52|1964.82|5051.28|2618.07|7016.10|202.05|0.00|69.81|5051.28|5253.33|5121.09|5323.14|2433.21| +2450825|47624|2450844|42886|1066140|4594|33048|42886|1066140|4594|33048|1|45|8|1|1243|54|529|48|95.26|206.71|157.09|2381.76|7540.32|4572.48|9922.08|527.82|0.00|1091.04|7540.32|8068.14|8631.36|9159.18|2967.84| +2450825|67975|2450881|51292|333170|4648|926|51292|333170|4648|926|2|38|12|1|4898|231|530|42|54.93|110.40|72.86|1576.68|3060.12|2307.06|4636.80|275.41|0.00|324.24|3060.12|3335.53|3384.36|3659.77|753.06| +2450825|67975|2450913|51292|333170|4648|926|51292|333170|4648|926|2|98|11|5|8636|129|530|17|83.37|175.91|84.43|1555.16|1435.31|1417.29|2990.47|28.70|0.00|956.93|1435.31|1464.01|2392.24|2420.94|18.02| +2450825|67975|2450830|51292|333170|4648|926|51292|333170|4648|926|2|101|4|1|16490|55|530|66|29.38|50.23|8.53|2752.20|562.98|1939.08|3315.18|22.51|0.00|1458.60|562.98|585.49|2021.58|2044.09|-1376.10| +2450825|67975|2450915|51292|333170|4648|926|51292|333170|4648|926|2|52|13|1|17134|235|530|8|23.06|26.28|4.46|174.56|35.68|184.48|210.24|0.14|28.54|27.28|7.14|7.28|34.42|34.56|-177.34| +2450825|67975|2450846|51292|333170|4648|926|51292|333170|4648|926|2|82|7|1|17743|222|530|89|86.94|209.52|146.66|5594.54|13052.74|7737.66|18647.28|783.16|0.00|9136.74|13052.74|13835.90|22189.48|22972.64|5315.08| +2450825|67975|2450886|51292|333170|4648|926|51292|333170|4648|926|2|11|18|1|16621|110|530|11|78.42|131.74|73.77|637.67|811.47|862.62|1449.14|64.91|0.00|666.60|811.47|876.38|1478.07|1542.98|-51.15| +2450825|4139|2450861|82410|116099|7035|46724|82410|116099|7035|46724|1|50|9|1|15481|282|531|45|12.19|22.30|21.18|50.40|953.10|548.55|1003.50|9.53|0.00|501.75|953.10|962.63|1454.85|1464.38|404.55| +2450825|4139|2450909|82410|116099|7035|46724|82410|116099|7035|46724|1|74|2|4|7766|126|531|13|21.02|40.77|6.11|450.58|79.43|273.26|530.01|3.17|0.00|74.10|79.43|82.60|153.53|156.70|-193.83| +2450825|4139|2450870|82410|116099|7035|46724|82410|116099|7035|46724|1|24|18|3|6980|196|531|36|2.54|6.47|2.97|126.00|106.92|91.44|232.92|3.20|0.00|46.44|106.92|110.12|153.36|156.56|15.48| +2450825|4139|2450888|82410|116099|7035|46724|82410|116099|7035|46724|1|3|6|4|15514|238|531|8|92.99|106.00|78.44|220.48|627.52|743.92|848.00|25.10|0.00|364.64|627.52|652.62|992.16|1017.26|-116.40| +2450825|4139|2450913|82410|116099|7035|46724|82410|116099|7035|46724|1|81|18|3|4462|240|531|9|30.13|82.85|17.39|589.14|156.51|271.17|745.65|12.52|0.00|283.32|156.51|169.03|439.83|452.35|-114.66| +2450825|4139|2450911|82410|116099|7035|46724|82410|116099|7035|46724|1|22|7|3|2806|147|531|49|75.66|80.95|31.57|2419.62|1546.93|3707.34|3966.55|61.87|0.00|1110.34|1546.93|1608.80|2657.27|2719.14|-2160.41| +2450825|78780|2450909|51008|1476051|5894|28184|51008|1476051|5894|28184|1|48|1|5|3434|148|532|49|90.95|269.21|61.91|10157.70|3033.59|4456.55|13191.29|121.34|0.00|1054.97|3033.59|3154.93|4088.56|4209.90|-1422.96| +2450825|78780|2450870|51008|1476051|5894|28184|51008|1476051|5894|28184|1|61|15|5|12514|40|532|53|70.44|85.23|75.85|497.14|4020.05|3733.32|4517.19|62.71|2974.83|1038.80|1045.22|1107.93|2084.02|2146.73|-2688.10| +2450825|78780|2450843|51008|1476051|5894|28184|51008|1476051|5894|28184|1|55|2|1|4873|167|532|9|47.57|108.45|43.38|585.63|390.42|428.13|976.05|27.32|0.00|263.52|390.42|417.74|653.94|681.26|-37.71| +2450825|78780|2450911|51008|1476051|5894|28184|51008|1476051|5894|28184|1|82|11|3|10102|297|532|84|71.28|76.98|48.49|2393.16|4073.16|5987.52|6466.32|81.46|0.00|1357.44|4073.16|4154.62|5430.60|5512.06|-1914.36| +2450825|78780|2450864|51008|1476051|5894|28184|51008|1476051|5894|28184|1|98|10|2|15859|109|532|5|63.83|103.40|70.31|165.45|351.55|319.15|517.00|21.09|0.00|129.25|351.55|372.64|480.80|501.89|32.40| +2450825|78780|2450872|51008|1476051|5894|28184|51008|1476051|5894|28184|1|74|8|3|11002|219|532|62|35.73|78.96|48.95|1860.62|3034.90|2215.26|4895.52|242.79|0.00|2153.88|3034.90|3277.69|5188.78|5431.57|819.64| +2450825|78780|2450859|51008|1476051|5894|28184|51008|1476051|5894|28184|1|55|17|4|10054|128|532|51|20.34|58.37|46.11|625.26|2351.61|1037.34|2976.87|47.03|0.00|89.25|2351.61|2398.64|2440.86|2487.89|1314.27| +2450825|69335|2450903|91268|1359911|4201|49644|91268|1359911|4201|49644|4|49|15|2|11270|139|533|81|92.54|95.31|9.53|6948.18|771.93|7495.74|7720.11|0.00|0.00|2547.45|771.93|771.93|3319.38|3319.38|-6723.81| +2450825|69335|2450899|91268|1359911|4201|49644|91268|1359911|4201|49644|4|59|19|3|15092|284|533|71|87.60|220.75|86.09|9560.86|6112.39|6219.60|15673.25|305.61|0.00|3134.65|6112.39|6418.00|9247.04|9552.65|-107.21| +2450825|69335|2450838|91268|1359911|4201|49644|91268|1359911|4201|49644|4|68|17|5|2548|194|533|40|85.86|251.56|163.51|3522.00|6540.40|3434.40|10062.40|196.21|0.00|4427.20|6540.40|6736.61|10967.60|11163.81|3106.00| +2450825|69335|2450874|91268|1359911|4201|49644|91268|1359911|4201|49644|4|24|1|2|16681|124|533|34|40.64|96.72|69.63|921.06|2367.42|1381.76|3288.48|71.02|0.00|295.80|2367.42|2438.44|2663.22|2734.24|985.66| +2450825|69335|2450877|91268|1359911|4201|49644|91268|1359911|4201|49644|4|81|4|2|9704|58|533|51|41.29|93.72|31.86|3154.86|1624.86|2105.79|4779.72|9.09|714.93|525.30|909.93|919.02|1435.23|1444.32|-1195.86| +2450825|69335|2450905|91268|1359911|4201|49644|91268|1359911|4201|49644|4|74|14|3|2680|84|533|9|46.06|62.18|15.54|419.76|139.86|414.54|559.62|4.19|0.00|33.57|139.86|144.05|173.43|177.62|-274.68| +2450825|69335|2450882|91268|1359911|4201|49644|91268|1359911|4201|49644|4|28|20|2|6190|126|533|88|99.90|295.70|56.18|21077.76|4943.84|8791.20|26021.60|0.00|0.00|3642.32|4943.84|4943.84|8586.16|8586.16|-3847.36| +2450825|69335|2450912|91268|1359911|4201|49644|91268|1359911|4201|49644|4|41|2|5|6406|161|533|83|12.67|15.58|1.09|1202.67|90.47|1051.61|1293.14|4.52|0.00|620.01|90.47|94.99|710.48|715.00|-961.14| +2450825|69335|2450861|91268|1359911|4201|49644|91268|1359911|4201|49644|4|72|10|2|2662|271|533|6|36.19|93.37|22.40|425.82|134.40|217.14|560.22|5.37|0.00|44.76|134.40|139.77|179.16|184.53|-82.74| +2450825|69335|2450907|91268|1359911|4201|49644|91268|1359911|4201|49644|4|82|6|2|13801|79|533|49|59.14|91.07|64.65|1294.58|3167.85|2897.86|4462.43|95.03|0.00|1338.68|3167.85|3262.88|4506.53|4601.56|269.99| +2450825|36235|2450890|45391|496784|1777|14723|45391|496784|1777|14723|4|74|3|1|7153|274|534|35|84.76|212.74|14.89|6924.75|521.15|2966.60|7445.90|36.48|0.00|2382.45|521.15|557.63|2903.60|2940.08|-2445.45| +2450825|36235|2450838|45391|496784|1777|14723|45391|496784|1777|14723|4|65|4|1|7084|173|534|13|30.05|79.03|30.82|626.73|400.66|390.65|1027.39|4.00|200.33|71.89|200.33|204.33|272.22|276.22|-190.32| +2450825|36235|2450913|45391|496784|1777|14723|45391|496784|1777|14723|4|107|6|2|7496|257|534|2|88.12|110.15|82.61|55.08|165.22|176.24|220.30|3.27|110.69|94.72|54.53|57.80|149.25|152.52|-121.71| +2450825|36235|2450900|45391|496784|1777|14723|45391|496784|1777|14723|4|92|10|2|11635|176|534|2|54.70|90.80|85.35|10.90|170.70|109.40|181.60|13.65|0.00|38.12|170.70|184.35|208.82|222.47|61.30| +2450825|36235|2450865|45391|496784|1777|14723|45391|496784|1777|14723|4|83|7|3|11570|65|534|35|71.68|147.66|90.07|2015.65|3152.45|2508.80|5168.10|0.00|0.00|1963.85|3152.45|3152.45|5116.30|5116.30|643.65| +2450825|36235|2450847|45391|496784|1777|14723|45391|496784|1777|14723|4|33|16|2|4591|145|534|16|63.01|98.92|96.94|31.68|1551.04|1008.16|1582.72|62.04|0.00|632.96|1551.04|1613.08|2184.00|2246.04|542.88| +2450825|36235|2450827|45391|496784|1777|14723|45391|496784|1777|14723|4|56|17|5|17407|32|534|97|93.25|126.82|107.79|1845.91|10455.63|9045.25|12301.54|941.00|0.00|2336.73|10455.63|11396.63|12792.36|13733.36|1410.38| +2450825|62807|2450904|64277|1460996|2962|14928|64277|1460996|2962|14928|4|26|3|3|6289|210|535|98|62.65|120.91|32.64|8650.46|3198.72|6139.70|11849.18|0.00|0.00|591.92|3198.72|3198.72|3790.64|3790.64|-2940.98| +2450825|62807|2450886|64277|1460996|2962|14928|64277|1460996|2962|14928|4|1|9|1|14294|261|535|28|18.41|53.20|17.55|998.20|491.40|515.48|1489.60|29.48|0.00|551.04|491.40|520.88|1042.44|1071.92|-24.08| +2450825|62807|2450851|64277|1460996|2962|14928|64277|1460996|2962|14928|4|2|10|2|16756|79|535|88|34.97|37.76|6.79|2725.36|597.52|3077.36|3322.88|47.80|0.00|763.84|597.52|645.32|1361.36|1409.16|-2479.84| +2450825|62807|2450851|64277|1460996|2962|14928|64277|1460996|2962|14928|4|55|13|5|7564|235|535|34|24.99|73.97|51.03|779.96|1735.02|849.66|2514.98|12.49|1318.61|427.38|416.41|428.90|843.79|856.28|-433.25| +2450825|62807|2450839|64277|1460996|2962|14928|64277|1460996|2962|14928|4|8|2|1|6364|233|535|77|83.73|226.90|47.64|13803.02|3668.28|6447.21|17471.30|256.77|0.00|1747.13|3668.28|3925.05|5415.41|5672.18|-2778.93| +2450825|62807|2450835|64277|1460996|2962|14928|64277|1460996|2962|14928|4|102|18|1|14998|132|535|54|43.39|103.26|42.33|3290.22|2285.82|2343.06|5576.04|22.85|0.00|1226.34|2285.82|2308.67|3512.16|3535.01|-57.24| +2450825|4887|2450838|60396|1877215|3634|48453|31914|824938|157|16176|2|99|11|2|9608|11|536|99|40.09|53.31|49.04|422.73|4854.96|3968.91|5277.69|291.29|0.00|2638.35|4854.96|5146.25|7493.31|7784.60|886.05| +2450825|4887|2450852|60396|1877215|3634|48453|31914|824938|157|16176|2|6|16|1|13009|67|536|31|14.60|34.89|32.44|75.95|1005.64|452.60|1081.59|60.33|0.00|75.64|1005.64|1065.97|1081.28|1141.61|553.04| +2450825|4887|2450853|60396|1877215|3634|48453|31914|824938|157|16176|2|81|2|3|12967|242|536|34|21.13|34.86|25.09|332.18|853.06|718.42|1185.24|68.24|0.00|533.12|853.06|921.30|1386.18|1454.42|134.64| +2450825|4887|2450895|60396|1877215|3634|48453|31914|824938|157|16176|2|46|20|1|13826|45|536|8|9.45|26.46|1.85|196.88|14.80|75.60|211.68|0.29|0.00|8.40|14.80|15.09|23.20|23.49|-60.80| +2450825|4887|2450861|60396|1877215|3634|48453|31914|824938|157|16176|2|50|13|2|5416|86|536|59|47.52|90.28|15.34|4421.46|905.06|2803.68|5326.52|8.68|36.20|1437.83|868.86|877.54|2306.69|2315.37|-1934.82| +2450825|46866|2450891|47408|1842621|3434|46647|47408|1842621|3434|46647|4|105|20|2|9376|194|537|59|6.96|19.00|17.48|89.68|1031.32|410.64|1121.00|0.82|948.81|392.35|82.51|83.33|474.86|475.68|-328.13| +2450825|46866|2450900|47408|1842621|3434|46647|47408|1842621|3434|46647|4|11|9|2|16732|23|537|54|11.34|30.50|22.87|412.02|1234.98|612.36|1647.00|0.00|0.00|444.42|1234.98|1234.98|1679.40|1679.40|622.62| +2450825|46866|2450838|47408|1842621|3434|46647|47408|1842621|3434|46647|4|12|10|4|9734|42|537|18|90.69|187.72|157.68|540.72|2838.24|1632.42|3378.96|28.38|0.00|1655.64|2838.24|2866.62|4493.88|4522.26|1205.82| +2450825|46866|2450915|47408|1842621|3434|46647|47408|1842621|3434|46647|4|38|20|5|11995|82|537|2|39.38|44.89|17.05|55.68|34.10|78.76|89.78|0.30|30.69|44.88|3.41|3.71|48.29|48.59|-75.35| +2450825|46866|2450831|47408|1842621|3434|46647|47408|1842621|3434|46647|4|42|1|5|16459|193|537|71|41.54|112.98|80.21|2326.67|5694.91|2949.34|8021.58|56.94|0.00|3609.64|5694.91|5751.85|9304.55|9361.49|2745.57| +2450825|46866|2450828|47408|1842621|3434|46647|47408|1842621|3434|46647|4|13|9|2|14914|196|537|62|45.59|100.75|84.63|999.44|5247.06|2826.58|6246.50|0.00|0.00|1499.16|5247.06|5247.06|6746.22|6746.22|2420.48| +2450825|63583|2450900|33965|754099|633|37967|33965|754099|633|37967|4|25|13|1|15556|224|538|59|14.30|26.59|1.06|1506.27|62.54|843.70|1568.81|3.75|0.00|501.50|62.54|66.29|564.04|567.79|-781.16| +2450825|63583|2450900|33965|754099|633|37967|33965|754099|633|37967|4|12|13|3|16214|235|538|72|82.68|248.04|233.15|1072.08|16786.80|5952.96|17858.88|839.34|0.00|5714.64|16786.80|17626.14|22501.44|23340.78|10833.84| +2450825|63583|2450853|33965|754099|633|37967|33965|754099|633|37967|4|33|20|3|10705|176|538|25|49.50|78.21|53.18|625.75|1329.50|1237.50|1955.25|66.47|0.00|234.50|1329.50|1395.97|1564.00|1630.47|92.00| +2450825|63583|2450876|33965|754099|633|37967|33965|754099|633|37967|4|56|19|2|9886|21|538|42|21.39|30.15|20.20|417.90|848.40|898.38|1266.30|76.35|0.00|189.84|848.40|924.75|1038.24|1114.59|-49.98| +2450825|63583|2450913|33965|754099|633|37967|33965|754099|633|37967|4|14|17|4|3883|293|538|9|10.63|14.35|1.14|118.89|10.26|95.67|129.15|0.71|0.00|19.35|10.26|10.97|29.61|30.32|-85.41| +2450825|63583|2450827|33965|754099|633|37967|33965|754099|633|37967|4|24|7|3|7676|6|538|11|73.88|172.87|12.10|1768.47|133.10|812.68|1901.57|0.00|0.00|646.47|133.10|133.10|779.57|779.57|-679.58| +2450825|63583|2450903|33965|754099|633|37967|33965|754099|633|37967|4|29|14|2|16340|118|538|18|31.63|81.28|42.26|702.36|760.68|569.34|1463.04|15.21|0.00|672.84|760.68|775.89|1433.52|1448.73|191.34| +2450825|63583|2450907|33965|754099|633|37967|33965|754099|633|37967|4|89|20|1|15568|1|538|77|76.02|117.07|4.68|8654.03|360.36|5853.54|9014.39|14.41|0.00|3695.23|360.36|374.77|4055.59|4070.00|-5493.18| +2450825|63583|2450910|33965|754099|633|37967|33965|754099|633|37967|4|12|19|3|11558|101|538|91|71.35|81.33|37.41|3996.72|3404.31|6492.85|7401.03|136.17|0.00|3255.98|3404.31|3540.48|6660.29|6796.46|-3088.54| +2450825|63583|2450856|33965|754099|633|37967|33965|754099|633|37967|4|22|16|2|2684|297|538|63|34.00|35.70|16.06|1237.32|1011.78|2142.00|2249.10|30.35|0.00|112.14|1011.78|1042.13|1123.92|1154.27|-1130.22| +2450825|63583|2450870|33965|754099|633|37967|33965|754099|633|37967|4|77|12|5|6536|258|538|92|5.70|15.10|6.19|819.72|569.48|524.40|1389.20|0.00|0.00|402.04|569.48|569.48|971.52|971.52|45.08| +2450825|63583|2450865|33965|754099|633|37967|33965|754099|633|37967|4|54|16|5|7358|282|538|71|92.25|160.51|101.12|4216.69|7179.52|6549.75|11396.21|358.97|0.00|3988.07|7179.52|7538.49|11167.59|11526.56|629.77| +2450825|63583|2450894|33965|754099|633|37967|33965|754099|633|37967|4|55|16|2|13468|232|538|51|72.30|79.53|39.76|2028.27|2027.76|3687.30|4056.03|162.22|0.00|607.92|2027.76|2189.98|2635.68|2797.90|-1659.54| +2450825|53202|2450902|13844|1511052|6307|1115|13844|1511052|6307|1115|4|50|13|4|17641|41|539|77|49.28|81.80|54.80|2079.00|4219.60|3794.56|6298.60|337.56|0.00|2644.95|4219.60|4557.16|6864.55|7202.11|425.04| +2450825|53202|2450827|13844|1511052|6307|1115|13844|1511052|6307|1115|4|17|2|2|17710|207|539|89|61.57|150.23|42.06|9627.13|3743.34|5479.73|13370.47|6.73|3631.03|3074.95|112.31|119.04|3187.26|3193.99|-5367.42| +2450825|53202|2450875|13844|1511052|6307|1115|13844|1511052|6307|1115|4|66|19|3|16909|260|539|35|6.11|7.75|6.12|57.05|214.20|213.85|271.25|12.85|0.00|97.65|214.20|227.05|311.85|324.70|0.35| +2450825|53202|2450845|13844|1511052|6307|1115|13844|1511052|6307|1115|4|88|11|4|10540|153|539|43|79.71|137.10|43.87|4008.89|1886.41|3427.53|5895.30|137.33|169.77|1827.50|1716.64|1853.97|3544.14|3681.47|-1710.89| +2450825|53202|2450915|13844|1511052|6307|1115|13844|1511052|6307|1115|4|75|13|4|254|293|539|88|56.81|139.18|68.19|6247.12|6000.72|4999.28|12247.84|540.06|0.00|4286.48|6000.72|6540.78|10287.20|10827.26|1001.44| +2450825|53202|2450852|13844|1511052|6307|1115|13844|1511052|6307|1115|4|38|3|1|7982|262|539|50|75.06|212.41|74.34|6903.50|3717.00|3753.00|10620.50|95.15|1338.12|4779.00|2378.88|2474.03|7157.88|7253.03|-1374.12| +2450825|53202|2450884|13844|1511052|6307|1115|13844|1511052|6307|1115|4|79|11|1|902|83|539|99|81.93|243.33|26.76|21440.43|2649.24|8111.07|24089.67|79.47|0.00|8912.97|2649.24|2728.71|11562.21|11641.68|-5461.83| +2450825|53202|2450833|13844|1511052|6307|1115|13844|1511052|6307|1115|4|42|7|2|3052|242|539|11|60.79|80.85|4.04|844.91|44.44|668.69|889.35|1.77|0.00|275.66|44.44|46.21|320.10|321.87|-624.25| +2450825|53202|2450867|13844|1511052|6307|1115|13844|1511052|6307|1115|4|102|15|5|17098|181|539|72|23.95|28.50|24.22|308.16|1743.84|1724.40|2052.00|52.31|0.00|266.40|1743.84|1796.15|2010.24|2062.55|19.44| +2450825|49316|2450861|61934|1037130|6726|9969|61934|1037130|6726|9969|2|59|1|3|9284|192|540|76|4.49|8.39|1.51|522.88|114.76|341.24|637.64|4.13|11.47|273.60|103.29|107.42|376.89|381.02|-237.95| +2450825|49316|2450900|61934|1037130|6726|9969|61934|1037130|6726|9969|2|59|18|4|952|82|540|90|36.35|39.62|4.35|3174.30|391.50|3271.50|3565.80|11.74|0.00|106.20|391.50|403.24|497.70|509.44|-2880.00| +2450825|49316|2450896|61934|1037130|6726|9969|61934|1037130|6726|9969|2|22|20|5|8443|209|540|75|85.56|195.07|95.58|7461.75|7168.50|6417.00|14630.25|501.79|0.00|6144.00|7168.50|7670.29|13312.50|13814.29|751.50| +2450825|49316|2450878|61934|1037130|6726|9969|61934|1037130|6726|9969|2|20|7|4|11060|51|540|63|31.19|73.92|17.74|3539.34|1117.62|1964.97|4656.96|22.35|0.00|1583.19|1117.62|1139.97|2700.81|2723.16|-847.35| +2450825|49316|2450829|61934|1037130|6726|9969|61934|1037130|6726|9969|2|36|6|4|445|8|540|80|9.96|25.59|5.88|1576.80|470.40|796.80|2047.20|9.40|0.00|839.20|470.40|479.80|1309.60|1319.00|-326.40| +2450825|49316|2450836|61934|1037130|6726|9969|61934|1037130|6726|9969|2|59|7|2|15997|20|540|49|24.95|36.17|14.10|1081.43|690.90|1222.55|1772.33|6.90|0.00|88.20|690.90|697.80|779.10|786.00|-531.65| +2450825|49316|2450906|61934|1037130|6726|9969|61934|1037130|6726|9969|2|43|12|2|12376|282|540|30|63.39|168.61|161.86|202.50|4855.80|1901.70|5058.30|388.46|0.00|2377.20|4855.80|5244.26|7233.00|7621.46|2954.10| +2450825|49316|2450891|61934|1037130|6726|9969|61934|1037130|6726|9969|2|92|17|3|4442|155|540|39|8.47|19.05|1.14|698.49|44.46|330.33|742.95|0.00|44.46|356.46|0.00|0.00|356.46|356.46|-330.33| +2450825|27808|2450885|30023|1718154|1132|38063|30023|1718154|1132|38063|1|75|10|4|7118|175|541|32|90.25|122.74|12.27|3535.04|392.64|2888.00|3927.68|3.10|82.45|0.00|310.19|313.29|310.19|313.29|-2577.81| +2450825|27808|2450844|30023|1718154|1132|38063|30023|1718154|1132|38063|1|26|17|5|8617|168|541|49|63.49|81.26|21.12|2946.86|1034.88|3111.01|3981.74|9.93|41.39|1194.13|993.49|1003.42|2187.62|2197.55|-2117.52| +2450825|27808|2450845|30023|1718154|1132|38063|30023|1718154|1132|38063|1|82|11|4|646|220|541|97|59.02|84.98|39.09|4451.33|3791.73|5724.94|8243.06|189.58|0.00|1565.58|3791.73|3981.31|5357.31|5546.89|-1933.21| +2450825|27808|2450865|30023|1718154|1132|38063|30023|1718154|1132|38063|1|56|13|3|2377|28|541|76|35.62|79.43|55.60|1811.08|4225.60|2707.12|6036.68|35.49|676.09|2474.56|3549.51|3585.00|6024.07|6059.56|842.39| +2450825|27808|2450894|30023|1718154|1132|38063|30023|1718154|1132|38063|1|64|20|5|3823|260|541|92|42.36|108.01|11.88|8843.96|1092.96|3897.12|9936.92|0.00|0.00|496.80|1092.96|1092.96|1589.76|1589.76|-2804.16| +2450825|27808|2450899|30023|1718154|1132|38063|30023|1718154|1132|38063|1|37|2|4|11692|116|541|77|27.14|32.29|28.09|323.40|2162.93|2089.78|2486.33|43.25|0.00|1242.78|2162.93|2206.18|3405.71|3448.96|73.15| +2450825|27808|2450833|30023|1718154|1132|38063|30023|1718154|1132|38063|1|58|7|3|3493|256|541|66|4.42|11.22|3.25|526.02|214.50|291.72|740.52|0.00|0.00|110.88|214.50|214.50|325.38|325.38|-77.22| +2450825|27808|2450843|30023|1718154|1132|38063|30023|1718154|1132|38063|1|45|20|1|2846|24|541|46|70.59|156.70|54.84|4685.56|2522.64|3247.14|7208.20|6.55|2194.69|1874.04|327.95|334.50|2201.99|2208.54|-2919.19| +2450825|27808|2450866|30023|1718154|1132|38063|30023|1718154|1132|38063|1|23|2|3|1972|78|541|54|18.65|52.03|25.49|1433.16|1376.46|1007.10|2809.62|13.76|0.00|1292.22|1376.46|1390.22|2668.68|2682.44|369.36| +2450825|8579|2450898|89138|1758789|1240|1738|89138|1758789|1240|1738|4|52|19|2|15202|268|542|8|29.60|29.60|19.24|82.88|153.92|236.80|236.80|1.53|0.00|33.12|153.92|155.45|187.04|188.57|-82.88| +2450825|8579|2450835|89138|1758789|1240|1738|89138|1758789|1240|1738|4|64|8|5|2263|99|542|58|47.70|138.33|51.18|5054.70|2968.44|2766.60|8023.14|0.00|0.00|882.18|2968.44|2968.44|3850.62|3850.62|201.84| +2450825|8579|2450843|89138|1758789|1240|1738|89138|1758789|1240|1738|4|33|12|4|10021|71|542|53|19.25|49.08|30.92|962.48|1638.76|1020.25|2601.24|114.71|0.00|571.87|1638.76|1753.47|2210.63|2325.34|618.51| +2450825|8579|2450870|89138|1758789|1240|1738|89138|1758789|1240|1738|4|65|5|5|4550|190|542|37|89.62|155.04|147.28|287.12|5449.36|3315.94|5736.48|490.44|0.00|0.00|5449.36|5939.80|5449.36|5939.80|2133.42| +2450825|8579|2450893|89138|1758789|1240|1738|89138|1758789|1240|1738|4|106|10|2|13126|25|542|33|49.54|63.41|49.45|460.68|1631.85|1634.82|2092.53|16.31|0.00|41.58|1631.85|1648.16|1673.43|1689.74|-2.97| +2450825|8579|2450839|89138|1758789|1240|1738|89138|1758789|1240|1738|4|71|10|5|11593|33|542|39|54.31|140.66|105.49|1371.63|4114.11|2118.09|5485.74|0.00|2303.90|548.34|1810.21|1810.21|2358.55|2358.55|-307.88| +2450825|8579|2450909|89138|1758789|1240|1738|89138|1758789|1240|1738|4|68|16|2|11380|54|542|98|6.05|9.13|6.57|250.88|643.86|592.90|894.74|51.50|0.00|339.08|643.86|695.36|982.94|1034.44|50.96| +2450825|8579|2450844|89138|1758789|1240|1738|89138|1758789|1240|1738|4|2|17|2|7519|67|542|9|56.93|163.95|24.59|1254.24|221.31|512.37|1475.55|19.91|0.00|339.30|221.31|241.22|560.61|580.52|-291.06| +2450825|8579|2450895|89138|1758789|1240|1738|89138|1758789|1240|1738|4|108|5|5|13000|188|542|11|75.85|225.27|47.30|1957.67|520.30|834.35|2477.97|26.01|0.00|569.91|520.30|546.31|1090.21|1116.22|-314.05| +2450825|8579|2450887|89138|1758789|1240|1738|89138|1758789|1240|1738|4|7|1|2|3386|178|542|79|12.25|18.00|2.34|1237.14|184.86|967.75|1422.00|14.78|0.00|113.76|184.86|199.64|298.62|313.40|-782.89| +2450825|61416|2450875|20414|1009736|658|42780|20414|1009736|658|42780|4|19|12|2|4492|111|543|3|59.38|113.41|102.06|34.05|306.18|178.14|340.23|18.37|0.00|122.46|306.18|324.55|428.64|447.01|128.04| +2450825|61416|2450858|20414|1009736|658|42780|20414|1009736|658|42780|4|100|15|4|10076|147|543|66|29.19|75.60|39.31|2395.14|2594.46|1926.54|4989.60|181.61|0.00|2195.16|2594.46|2776.07|4789.62|4971.23|667.92| +2450825|61416|2450868|20414|1009736|658|42780|20414|1009736|658|42780|4|24|20|3|16990|86|543|94|71.63|145.40|37.80|10114.40|3553.20|6733.22|13667.60|0.00|0.00|3143.36|3553.20|3553.20|6696.56|6696.56|-3180.02| +2450825|61416|2450910|20414|1009736|658|42780|20414|1009736|658|42780|4|86|11|5|15019|263|543|94|60.86|173.45|12.14|15163.14|1141.16|5720.84|16304.30|0.00|0.00|2608.50|1141.16|1141.16|3749.66|3749.66|-4579.68| +2450825|61416|2450859|20414|1009736|658|42780|20414|1009736|658|42780|4|59|9|4|11978|120|543|54|52.46|115.41|115.41|0.00|6232.14|2832.84|6232.14|0.00|0.00|2367.90|6232.14|6232.14|8600.04|8600.04|3399.30| +2450825|61416|2450870|20414|1009736|658|42780|20414|1009736|658|42780|4|32|3|2|10364|190|543|56|38.86|111.13|43.34|3796.24|2427.04|2176.16|6223.28|121.35|0.00|3111.36|2427.04|2548.39|5538.40|5659.75|250.88| +2450825|61416|2450828|20414|1009736|658|42780|20414|1009736|658|42780|4|89|11|3|16297|72|543|62|78.93|170.48|98.87|4439.82|6129.94|4893.66|10569.76|122.59|4597.45|4756.02|1532.49|1655.08|6288.51|6411.10|-3361.17| +2450825|76725|2450862|59023|885359|4942|20478|59023|885359|4942|20478|4|30|2|2|1186|181|544|9|84.36|190.65|175.39|137.34|1578.51|759.24|1715.85|142.06|0.00|497.52|1578.51|1720.57|2076.03|2218.09|819.27| +2450825|76725|2450872|59023|885359|4942|20478|59023|885359|4942|20478|4|87|1|5|14660|45|544|24|71.07|98.78|54.32|1067.04|1303.68|1705.68|2370.72|13.03|0.00|734.88|1303.68|1316.71|2038.56|2051.59|-402.00| +2450825|76725|2450843|59023|885359|4942|20478|59023|885359|4942|20478|4|39|4|3|5623|277|544|91|61.28|180.16|1.80|16230.76|163.80|5576.48|16394.56|4.65|47.50|1311.31|116.30|120.95|1427.61|1432.26|-5460.18| +2450825|76725|2450872|59023|885359|4942|20478|59023|885359|4942|20478|4|62|4|2|10237|250|544|17|74.02|151.00|140.43|179.69|2387.31|1258.34|2567.00|95.49|0.00|102.68|2387.31|2482.80|2489.99|2585.48|1128.97| +|76725|2450853||||20478|||4942|||51|9||12607|35|544||20.94||26.25|||167.52|477.36|||14.32|210.00||224.32|239.02|42.48| +2450825|76725|2450846|59023|885359|4942|20478|59023|885359|4942|20478|4|24|1|5|10642|222|544|24|10.30|23.58|5.89|424.56|141.36|247.20|565.92|2.82|0.00|277.20|141.36|144.18|418.56|421.38|-105.84| +2450825|76725|2450844|59023|885359|4942|20478|59023|885359|4942|20478|4|43|19|5|4576|204|544|49|17.06|39.40|21.27|888.37|1042.23|835.94|1930.60|31.26|0.00|269.99|1042.23|1073.49|1312.22|1343.48|206.29| +2450825|76725|2450835|59023|885359|4942|20478|59023|885359|4942|20478|4|19|18|4|4849|201|544|44|73.22|123.00|79.95|1894.20|3517.80|3221.68|5412.00|175.89|0.00|1785.96|3517.80|3693.69|5303.76|5479.65|296.12| +2450825|76725|2450868|59023|885359|4942|20478|59023|885359|4942|20478|4|65|17|2|12140|274|544|98|8.74|17.48|15.73|171.50|1541.54|856.52|1713.04|15.41|0.00|804.58|1541.54|1556.95|2346.12|2361.53|685.02| +2450825|76725|2450828|59023|885359|4942|20478|59023|885359|4942|20478|4|59|16|2|9475|169|544|93|28.27|67.28|21.52|4255.68|2001.36|2629.11|6257.04|35.62|220.14|2377.08|1781.22|1816.84|4158.30|4193.92|-847.89| +2450825|76725|2450880|59023|885359|4942|20478|59023|885359|4942|20478|4|22|7|3|15553|259|544|27|89.83|205.71|6.17|5387.58|166.59|2425.41|5554.17|0.00|0.00|2221.56|166.59|166.59|2388.15|2388.15|-2258.82| +2450825|76725|2450849|59023|885359|4942|20478|59023|885359|4942|20478|4|22|17|3|6944|100|544|16|96.46|201.60|64.51|2193.44|1032.16|1543.36|3225.60|12.07|629.61|257.92|402.55|414.62|660.47|672.54|-1140.81| +2450825|76725|2450847|59023|885359|4942|20478|59023|885359|4942|20478|4|71|1|5|9817|156|544|2|27.94|52.52|41.49|22.06|82.98|55.88|105.04|4.14|0.00|35.70|82.98|87.12|118.68|122.82|27.10| +2450825|45414|2450904|86307|815605|2503|16954|86307|815605|2503|16954|2|61|5|1|11300|88|545|85|87.29|176.32|104.02|6145.50|8841.70|7419.65|14987.20|795.75|0.00|2997.10|8841.70|9637.45|11838.80|12634.55|1422.05| +2450825|45414|2450890|86307|815605|2503|16954|86307|815605|2503|16954|2|106|9|1|11834|175|545|48|6.04|8.27|0.82|357.60|39.36|289.92|396.96|0.00|0.00|31.68|39.36|39.36|71.04|71.04|-250.56| +2450825|45414|2450887|86307|815605|2503|16954|86307|815605|2503|16954|2|44|12|2|14768|252|545|28|47.00|88.36|78.64|272.16|2201.92|1316.00|2474.08|176.15|0.00|1137.92|2201.92|2378.07|3339.84|3515.99|885.92| +2450825|45414|2450904|86307|815605|2503|16954|86307|815605|2503|16954|2|62|19|5|12166|119|545|72|84.91|254.73|40.75|15406.56|2934.00|6113.52|18340.56|117.36|0.00|3851.28|2934.00|3051.36|6785.28|6902.64|-3179.52| +2450825|45414|2450835|86307|815605|2503|16954|86307|815605|2503|16954|2|88|17|1|11056|67|545|36|53.32|84.24|31.16|1910.88|1121.76|1919.52|3032.64|33.65|0.00|1425.24|1121.76|1155.41|2547.00|2580.65|-797.76| +2450825|61803|2450901|16305|382365|6025|10885|16305|382365|6025|10885|1|34|2|5|10861|120|546|8|62.70|127.90|76.74|409.28|613.92|501.60|1023.20|5.89|24.55|368.32|589.37|595.26|957.69|963.58|87.77| +2450825|61803|2450858|16305|382365|6025|10885|16305|382365|6025|10885|1|32|16|5|10513|61|546|38|81.04|194.49|126.41|2587.04|4803.58|3079.52|7390.62|240.17|0.00|3621.40|4803.58|5043.75|8424.98|8665.15|1724.06| +2450825|61803|2450836|16305|382365|6025|10885|16305|382365|6025|10885|1|91|14|4|9772|81|546|62|11.20|27.21|10.06|1063.30|623.72|694.40|1687.02|49.89|0.00|438.34|623.72|673.61|1062.06|1111.95|-70.68| +2450825|61803|2450895|16305|382365|6025|10885|16305|382365|6025|10885|1|62|20|1|12680|210|546|52|37.38|56.07|30.83|1312.48|1603.16|1943.76|2915.64|112.22|0.00|845.52|1603.16|1715.38|2448.68|2560.90|-340.60| +2450825|32828|2450877|99961|1095242|4923|3804|99961|1095242|4923|3804|4|13|13|2|8816|175|547|24|80.45|82.05|4.92|1851.12|118.08|1930.80|1969.20|0.00|0.00|708.72|118.08|118.08|826.80|826.80|-1812.72| +2450825|32828|2450905|99961|1095242|4923|3804|99961|1095242|4923|3804|4|73|2|4|1502|29|547|10|92.79|209.70|138.40|713.00|1384.00|927.90|2097.00|14.39|1204.08|314.50|179.92|194.31|494.42|508.81|-747.98| +2450825|32828|2450897|99961|1095242|4923|3804|99961|1095242|4923|3804|4|75|11|5|3352|177|547|57|51.37|64.21|21.18|2452.71|1207.26|2928.09|3659.97|0.00|0.00|1207.26|1207.26|1207.26|2414.52|2414.52|-1720.83| +2450825|32828|2450898|99961|1095242|4923|3804|99961|1095242|4923|3804|4|63|16|5|4312|28|547|47|64.11|154.50|91.15|2977.45|4284.05|3013.17|7261.50|214.20|0.00|217.61|4284.05|4498.25|4501.66|4715.86|1270.88| +2450825|32828|2450839|99961|1095242|4923|3804|99961|1095242|4923|3804|4|6|20|5|13648|235|547|33|60.05|165.73|87.83|2570.70|2898.39|1981.65|5469.09|115.93|0.00|601.59|2898.39|3014.32|3499.98|3615.91|916.74| +2450825|32828|2450907|99961|1095242|4923|3804|99961|1095242|4923|3804|4|104|1|2|11620|7|547|73|67.39|109.17|79.69|2152.04|5817.37|4919.47|7969.41|232.69|0.00|1992.17|5817.37|6050.06|7809.54|8042.23|897.90| +2450825|32828|2450872|99961|1095242|4923|3804|99961|1095242|4923|3804|4|90|15|2|6469|83|547|11|71.65|174.10|50.48|1359.82|555.28|788.15|1915.10|11.10|0.00|344.63|555.28|566.38|899.91|911.01|-232.87| +2450825|32828|2450858|99961|1095242|4923|3804|99961|1095242|4923|3804|4|22|19|1|11557|175|547|49|82.83|197.96|45.53|7469.07|2230.97|4058.67|9700.04|4.46|2007.87|2715.58|223.10|227.56|2938.68|2943.14|-3835.57| +2450825|32828|2450832|99961|1095242|4923|3804|99961|1095242|4923|3804|4|81|2|2|3536|152|547|34|78.67|195.10|35.11|5439.66|1193.74|2674.78|6633.40|59.68|0.00|1591.88|1193.74|1253.42|2785.62|2845.30|-1481.04| +2450825|32828|2450912|99961|1095242|4923|3804|99961|1095242|4923|3804|4|84|14|5|9356|32|547|4|74.36|100.38|12.04|353.36|48.16|297.44|401.52|2.88|0.00|104.36|48.16|51.04|152.52|155.40|-249.28| +2450825|32828|2450835|99961|1095242|4923|3804|99961|1095242|4923|3804|4|66|9|4|6961|107|547|82|90.82|217.96|80.64|11260.24|6612.48|7447.24|17872.72|330.62|0.00|5182.40|6612.48|6943.10|11794.88|12125.50|-834.76| +2450825|47301|2450830|19600|1462369|7185|45954|19600|1462369|7185|45954|2|36|6|2|6034|176|548|48|76.16|132.51|104.68|1335.84|5024.64|3655.68|6360.48|401.97|0.00|1971.36|5024.64|5426.61|6996.00|7397.97|1368.96| +2450825|47301|2450843|19600|1462369|7185|45954|19600|1462369|7185|45954|2|26|17|1|5264|76|548|22|86.76|199.54|77.82|2677.84|1712.04|1908.72|4389.88|119.84|0.00|658.46|1712.04|1831.88|2370.50|2490.34|-196.68| +2450825|47301|2450876|19600|1462369|7185|45954|19600|1462369|7185|45954|2|11|20|2|8722|189|548|88|20.91|51.85|45.10|594.00|3968.80|1840.08|4562.80|238.12|0.00|1733.60|3968.80|4206.92|5702.40|5940.52|2128.72| +2450825|47301|2450891|19600|1462369|7185|45954|19600|1462369|7185|45954|2|101|7|1|16136|69|548|59|2.22|5.81|1.33|264.32|78.47|130.98|342.79|0.78|0.00|17.11|78.47|79.25|95.58|96.36|-52.51| +2450825|47301|2450842|19600|1462369|7185|45954|19600|1462369|7185|45954|2|25|10|4|1340|217|548|80|53.33|146.12|77.44|5494.40|6195.20|4266.40|11689.60|309.76|0.00|1402.40|6195.20|6504.96|7597.60|7907.36|1928.80| +2450825|47301|2450874|19600|1462369|7185|45954|19600|1462369|7185|45954|2|16|1|5|9973|5|548|66|10.24|29.38|17.62|776.16|1162.92|675.84|1939.08|11.62|0.00|77.22|1162.92|1174.54|1240.14|1251.76|487.08| +2450825|47301|2450853|19600|1462369|7185|45954|19600|1462369|7185|45954|2|103|13|2|7450|50|548|55|41.86|45.62|40.60|276.10|2233.00|2302.30|2509.10|111.65|0.00|125.40|2233.00|2344.65|2358.40|2470.05|-69.30| +2450825|47301|2450846|19600|1462369|7185|45954|19600|1462369|7185|45954|2|101|9|5|11617|2|548|65|89.15|126.59|10.12|7570.55|657.80|5794.75|8228.35|46.04|0.00|2386.15|657.80|703.84|3043.95|3089.99|-5136.95| +2450825|47301|2450849|19600|1462369|7185|45954|19600|1462369|7185|45954|2|97|2|1|13720|36|548|95|1.19|1.39|1.13|24.70|107.35|113.05|132.05|2.57|21.47|14.25|85.88|88.45|100.13|102.70|-27.17| +2450825|47301|2450873|19600|1462369|7185|45954|19600|1462369|7185|45954|2|34|19|1|7006|139|548|98|36.18|96.96|43.63|5226.34|4275.74|3545.64|9502.08|213.78|0.00|1329.86|4275.74|4489.52|5605.60|5819.38|730.10| +2450825|47301|2450834|19600|1462369|7185|45954|19600|1462369|7185|45954|2|33|17|5|14347|31|548|36|12.95|23.95|4.07|715.68|146.52|466.20|862.20|5.86|0.00|293.04|146.52|152.38|439.56|445.42|-319.68| +2450825|47301|2450843|19600|1462369|7185|45954|19600|1462369|7185|45954|2|51|11|4|12592|60|548|4|77.01|153.24|41.37|447.48|165.48|308.04|612.96|1.65|0.00|208.40|165.48|167.13|373.88|375.53|-142.56| +2450825|47301|2450841|19600|1462369|7185|45954|19600|1462369|7185|45954|2|16|5|2|6757|88|548|98|29.68|83.40|70.89|1225.98|6947.22|2908.64|8173.20|69.47|0.00|1960.98|6947.22|7016.69|8908.20|8977.67|4038.58| +2450825|4821|2450842|3738|116116|1741|7319|3738|116116|1741|7319|1|89|12|2|10801|282|549|5|18.54|52.28|33.45|94.15|167.25|92.70|261.40|7.96|53.52|117.60|113.73|121.69|231.33|239.29|21.03| +2450825|4821|2450830|3738|116116|1741|7319|3738|116116|1741|7319|1|9|16|3|8324|110|549|20|76.55|137.79|129.52|165.40|2590.40|1531.00|2755.80|181.32|0.00|771.60|2590.40|2771.72|3362.00|3543.32|1059.40| +2450825|4821|2450864|3738|116116|1741|7319|3738|116116|1741|7319|1|68|15|5|5258|118|549|43|19.99|28.98|7.82|909.88|336.26|859.57|1246.14|0.00|0.00|361.20|336.26|336.26|697.46|697.46|-523.31| +2450825|4821|2450899|3738|116116|1741|7319|3738|116116|1741|7319|1|18|7|2|7372|82|549|66|57.66|68.03|52.38|1032.90|3457.08|3805.56|4489.98|0.00|138.28|1750.98|3318.80|3318.80|5069.78|5069.78|-486.76| +2450825|4821|2450835|3738|116116|1741|7319|3738|116116|1741|7319|1|88|11|3|3704|232|549|26|4.33|8.74|2.79|154.70|72.54|112.58|227.24|1.16|43.52|97.50|29.02|30.18|126.52|127.68|-83.56| +2450825|4821|2450836|3738|116116|1741|7319|3738|116116|1741|7319|1|101|10|1|11851|226|549|81|41.63|58.28|25.06|2690.82|2029.86|3372.03|4720.68|101.49|0.00|1274.13|2029.86|2131.35|3303.99|3405.48|-1342.17| +2450825|4821|2450831|3738|116116|1741|7319|3738|116116|1741|7319|1|61|5|2|4052|64|549|63|36.42|88.13|37.89|3165.12|2387.07|2294.46|5552.19|214.83|0.00|1387.89|2387.07|2601.90|3774.96|3989.79|92.61| +2450825|4821|2450879|3738|116116|1741|7319|3738|116116|1741|7319|1|29|12|5|14185|246|549|88|54.32|133.08|78.51|4802.16|6908.88|4780.16|11711.04|18.65|6287.08|234.08|621.80|640.45|855.88|874.53|-4158.36| +2450825|4821|2450840|3738|116116|1741|7319|3738|116116|1741|7319|1|103|7|2|5311|83|549|44|17.03|19.41|6.98|546.92|307.12|749.32|854.04|0.00|304.04|204.60|3.08|3.08|207.68|207.68|-746.24| +2450825|4821|2450902|3738|116116|1741|7319|3738|116116|1741|7319|1|13|1|4|12001|50|549|81|54.65|128.42|30.82|7905.60|2496.42|4426.65|10402.02|99.85|0.00|4888.35|2496.42|2596.27|7384.77|7484.62|-1930.23| +2450825|4821|2450882|3738|116116|1741|7319|3738|116116|1741|7319|1|13|14|3|7346|146|549|59|5.41|15.36|8.29|417.13|489.11|319.19|906.24|4.89|0.00|398.25|489.11|494.00|887.36|892.25|169.92| +2450825|4821|2450852|3738|116116|1741|7319|3738|116116|1741|7319|1|87|6|5|15224|109|549|16|75.51|163.10|30.98|2113.92|495.68|1208.16|2609.60|44.61|0.00|443.52|495.68|540.29|939.20|983.81|-712.48| +2450825|11299|2450837|31044|1211264|284|8504|31044|1211264|284|8504|1|39|8|5|7630|57|550|100|31.79|46.41|33.87|1254.00|3387.00|3179.00|4641.00|292.63|135.48|1763.00|3251.52|3544.15|5014.52|5307.15|72.52| +2450825|11299|2450870|31044|1211264|284|8504|31044|1211264|284|8504|1|54|2|4|590|211|550|45|2.93|6.62|1.32|238.50|59.40|131.85|297.90|1.18|0.00|145.80|59.40|60.58|205.20|206.38|-72.45| +2450825|11299|2450902|31044|1211264|284|8504|31044|1211264|284|8504|1|102|1|4|1123|102|550|65|7.74|15.32|14.24|70.20|925.60|503.10|995.80|46.28|0.00|189.15|925.60|971.88|1114.75|1161.03|422.50| +2450825|11299|2450890|31044|1211264|284|8504|31044|1211264|284|8504|1|21|10|3|9013|226|550|78|98.30|224.12|127.74|7517.64|9963.72|7667.40|17481.36|99.63|0.00|6293.04|9963.72|10063.35|16256.76|16356.39|2296.32| +2450825|11299|2450910|31044|1211264|284|8504|31044|1211264|284|8504|1|10|16|5|8641|96|550|37|3.64|10.73|4.82|218.67|178.34|134.68|397.01|0.00|105.22|122.84|73.12|73.12|195.96|195.96|-61.56| +2450825|11299|2450884|31044|1211264|284|8504|31044|1211264|284|8504|1|104|15|2|5815|291|550|87|47.96|85.84|33.47|4556.19|2911.89|4172.52|7468.08|29.11|0.00|896.10|2911.89|2941.00|3807.99|3837.10|-1260.63| +2450825|11299|2450867|31044|1211264|284|8504|31044|1211264|284|8504|1|56|20|3|2096|142|550|74|45.73|48.01|40.80|533.54|3019.20|3384.02|3552.74|0.00|0.00|1740.48|3019.20|3019.20|4759.68|4759.68|-364.82| +2450825|11299|2450890|31044|1211264|284|8504|31044|1211264|284|8504|1|24|19|3|17515|74|550|1|98.19|165.94|159.30|6.64|159.30|98.19|165.94|0.00|82.83|1.65|76.47|76.47|78.12|78.12|-21.72| +2450825|11299|2450847|31044|1211264|284|8504|31044|1211264|284|8504|1|50|1|5|14875|21|550|99|3.93|8.17|2.77|534.60|274.23|389.07|808.83|1.20|153.56|145.53|120.67|121.87|266.20|267.40|-268.40| +2450825|75128|2450858|36026|1435872|5765|2011|36026|1435872|5765|2011|4|97|9|3|5144|196|551|83|79.29|124.48|33.60|7543.04|2788.80|6581.07|10331.84|223.10|0.00|2065.87|2788.80|3011.90|4854.67|5077.77|-3792.27| +2450825|75128|2450915|36026|1435872|5765|2011|36026|1435872|5765|2011|4|8|12|1|2486|281|551|27|58.60|76.76|56.03|559.71|1512.81|1582.20|2072.52|15.12|0.00|1036.26|1512.81|1527.93|2549.07|2564.19|-69.39| +2450825|75128|2450865|36026|1435872|5765|2011|36026|1435872|5765|2011|4|46|18|3|10612|175|551|64|68.94|99.27|66.51|2096.64|4256.64|4412.16|6353.28|383.09|0.00|317.44|4256.64|4639.73|4574.08|4957.17|-155.52| +2450825|75128|2450861|36026|1435872|5765|2011|36026|1435872|5765|2011|4|70|20|2|12008|257|551|83|39.64|43.60|5.66|3149.02|469.78|3290.12|3618.80|4.69|0.00|1049.12|469.78|474.47|1518.90|1523.59|-2820.34| +2450825|75128|2450872|36026|1435872|5765|2011|36026|1435872|5765|2011|4|78|16|3|9949|182|551|14|59.19|120.74|27.77|1301.58|388.78|828.66|1690.36|7.77|291.58|202.72|97.20|104.97|299.92|307.69|-731.46| +2450825|75128|2450897|36026|1435872|5765|2011|36026|1435872|5765|2011|4|22|16|2|13807|289|551|51|39.60|59.79|8.37|2622.42|426.87|2019.60|3049.29|22.28|179.28|456.96|247.59|269.87|704.55|726.83|-1772.01| +2450825|75128|2450857|36026|1435872|5765|2011|36026|1435872|5765|2011|4|53|1|1|8224|56|551|4|95.29|157.22|92.75|257.88|371.00|381.16|628.88|22.26|0.00|308.12|371.00|393.26|679.12|701.38|-10.16| +2450825|75128|2450840|36026|1435872|5765|2011|36026|1435872|5765|2011|4|7|6|1|4111|278|551|90|67.18|180.71|90.35|8132.40|8131.50|6046.20|16263.90|650.52|0.00|2439.00|8131.50|8782.02|10570.50|11221.02|2085.30| +2450825|75128|2450874|36026|1435872|5765|2011|36026|1435872|5765|2011|4|103|11|5|16456|122|551|72|11.94|20.41|6.12|1028.88|440.64|859.68|1469.52|4.40|0.00|58.32|440.64|445.04|498.96|503.36|-419.04| +2450825|75128|2450854|36026|1435872|5765|2011|36026|1435872|5765|2011|4|46|6|3|13150|104|551|72|50.34|71.48|4.28|4838.40|308.16|3624.48|5146.56|2.92|15.40|257.04|292.76|295.68|549.80|552.72|-3331.72| +2450825|22068|2450864|56373|1378326|726|13926|56373|1378326|726|13926|1|39|4|2|3950|161|552|88|63.49|104.12|69.76|3023.68|6138.88|5587.12|9162.56|0.00|0.00|3848.24|6138.88|6138.88|9987.12|9987.12|551.76| +2450825|22068|2450855|56373|1378326|726|13926|56373|1378326|726|13926|1|57|20|1|12187|50|552|29|10.81|11.24|6.18|146.74|179.22|313.49|325.96|1.50|129.03|25.81|50.19|51.69|76.00|77.50|-263.30| +2450825|22068|2450883|56373|1378326|726|13926|56373|1378326|726|13926|1|93|20|4|8395|273|552|95|73.54|219.14|149.01|6662.35|14155.95|6986.30|20818.30|141.55|0.00|3538.75|14155.95|14297.50|17694.70|17836.25|7169.65| +2450825|22068|2450906|56373|1378326|726|13926|56373|1378326|726|13926|1|2|5|5|14683|161|552|89|9.96|25.59|24.56|91.67|2185.84|886.44|2277.51|0.00|0.00|1092.92|2185.84|2185.84|3278.76|3278.76|1299.40| +2450825|22068|2450906|56373|1378326|726|13926|56373|1378326|726|13926|1|41|4|2|14746|146|552|25|83.29|150.75|78.39|1809.00|1959.75|2082.25|3768.75|117.58|0.00|1206.00|1959.75|2077.33|3165.75|3283.33|-122.50| +2450825|22068|2450849|56373|1378326|726|13926|56373|1378326|726|13926|1|30|14|3|13312|273|552|93|22.43|33.64|9.08|2284.08|844.44|2085.99|3128.52|59.11|0.00|1126.23|844.44|903.55|1970.67|2029.78|-1241.55| +2450825|22068|2450914|56373|1378326|726|13926|56373|1378326|726|13926|1|52|7|4|15751|36|552|19|36.49|90.86|82.68|155.42|1570.92|693.31|1726.34|15.70|1178.19|51.68|392.73|408.43|444.41|460.11|-300.58| +2450825|22068|2450843|56373|1378326|726|13926|56373|1378326|726|13926|1|17|7|2|14726|62|552|79|92.63|227.86|100.25|10081.19|7919.75|7317.77|18000.94|79.19|0.00|6300.25|7919.75|7998.94|14220.00|14299.19|601.98| +2450825|22068|2450874|56373|1378326|726|13926|56373|1378326|726|13926|1|86|13|3|14284|67|552|29|48.93|53.33|15.46|1098.23|448.34|1418.97|1546.57|13.45|0.00|386.57|448.34|461.79|834.91|848.36|-970.63| +2450825|22068|2450837|56373|1378326|726|13926|56373|1378326|726|13926|1|47|11|3|5546|275|552|49|76.49|175.16|120.86|2660.70|5922.14|3748.01|8582.84|0.00|0.00|3604.44|5922.14|5922.14|9526.58|9526.58|2174.13| +2450825|22068|2450851|56373|1378326|726|13926|56373|1378326|726|13926|1|97|6|1|16168|226|552|85|13.98|41.24|10.31|2629.05|876.35|1188.30|3505.40|43.81|0.00|1366.80|876.35|920.16|2243.15|2286.96|-311.95| +2450825|22068|2450870|56373|1378326|726|13926|56373|1378326|726|13926|1|75|20|3|9073|297|552|25|61.75|92.62|12.96|1991.50|324.00|1543.75|2315.50|19.44|0.00|833.50|324.00|343.44|1157.50|1176.94|-1219.75| +2450825|36888|2450873|67566|1577176|1539|19102|67566|1577176|1539|19102|2|32|5|5|15184|73|553|59|87.30|249.67|79.89|10017.02|4713.51|5150.70|14730.53|377.08|0.00|5007.92|4713.51|5090.59|9721.43|10098.51|-437.19| +2450825|36888|2450833|67566|1577176|1539|19102|67566|1577176|1539|19102|2|76|5|4|11983|52|553|3|37.93|48.92|32.28|49.92|96.84|113.79|146.76|6.77|0.00|27.87|96.84|103.61|124.71|131.48|-16.95| +2450825|36888|2450888|67566|1577176|1539|19102|67566|1577176|1539|19102|2|93|15|5|8450|275|553|97|52.94|147.70|57.60|8739.70|5587.20|5135.18|14326.90|55.87|0.00|4727.78|5587.20|5643.07|10314.98|10370.85|452.02| +2450825|36888|2450902|67566|1577176|1539|19102|67566|1577176|1539|19102|2|60|1|1|2329|236|553|64|50.79|90.40|30.73|3818.88|1966.72|3250.56|5785.60|39.33|0.00|2718.72|1966.72|2006.05|4685.44|4724.77|-1283.84| +2450825|36888|2450906|67566|1577176|1539|19102|67566|1577176|1539|19102|2|70|2|1|14053|246|553|58|2.27|3.85|2.07|103.24|120.06|131.66|223.30|0.56|63.63|15.08|56.43|56.99|71.51|72.07|-75.23| +2450825|36888|2450839|67566|1577176|1539|19102|67566|1577176|1539|19102|2|94|13|2|14752|218|553|4|30.44|55.09|12.67|169.68|50.68|121.76|220.36|4.05|0.00|35.24|50.68|54.73|85.92|89.97|-71.08| +2450825|36888|2450858|67566|1577176|1539|19102|67566|1577176|1539|19102|2|20|8|3|16988|76|553|15|59.61|100.74|57.42|649.80|861.30|894.15|1511.10|8.61|0.00|120.75|861.30|869.91|982.05|990.66|-32.85| +2450825|36888|2450896|67566|1577176|1539|19102|67566|1577176|1539|19102|2|57|20|2|10946|246|553|20|39.33|82.59|26.42|1123.40|528.40|786.60|1651.80|36.98|0.00|16.40|528.40|565.38|544.80|581.78|-258.20| +2450825|36888|2450911|67566|1577176|1539|19102|67566|1577176|1539|19102|2|98|19|4|11173|167|553|78|46.38|103.89|50.90|4133.22|3970.20|3617.64|8103.42|39.70|0.00|2836.08|3970.20|4009.90|6806.28|6845.98|352.56| +2450825|36888|2450871|67566|1577176|1539|19102|67566|1577176|1539|19102|2|88|11|2|7354|49|553|100|42.62|80.55|16.11|6444.00|1611.00|4262.00|8055.00|112.77|0.00|1047.00|1611.00|1723.77|2658.00|2770.77|-2651.00| +2450825|36888|2450882|67566|1577176|1539|19102|67566|1577176|1539|19102|2|105|14|3|15050|157|553|9|72.59|169.86|74.73|856.17|672.57|653.31|1528.74|47.07|0.00|672.57|672.57|719.64|1345.14|1392.21|19.26| +2450825|36888|2450893|67566|1577176|1539|19102|67566|1577176|1539|19102|2|63|16|5|4688|267|553|40|89.37|130.48|35.22|3810.40|1408.80|3574.80|5219.20|112.70|0.00|521.60|1408.80|1521.50|1930.40|2043.10|-2166.00| +2450825|36888|2450885|67566|1577176|1539|19102|67566|1577176|1539|19102|2|3|9|2|13597|65|553|61|97.31|213.10|8.52|12479.38|519.72|5935.91|12999.10|5.19|0.00|2729.75|519.72|524.91|3249.47|3254.66|-5416.19| +2450825|36888|2450911|67566|1577176|1539|19102|67566|1577176|1539|19102|2|84|16|3|4513|27|553|44|43.55|77.51|47.28|1330.12|2080.32|1916.20|3410.44|187.22|0.00|750.20|2080.32|2267.54|2830.52|3017.74|164.12| +2450825|49545|2450894|76140|602470|2400|46730|76140|602470|2400|46730|4|32|3|4|5824|284|554|91|59.79|121.37|95.88|2319.59|8725.08|5440.89|11044.67|523.50|0.00|3092.18|8725.08|9248.58|11817.26|12340.76|3284.19| +2450825|49545|2450836|76140|602470|2400|46730|76140|602470|2400|46730|4|83|3|1|14300|174|554|54|78.58|219.23|13.15|11128.32|710.10|4243.32|11838.42|35.50|0.00|828.36|710.10|745.60|1538.46|1573.96|-3533.22| +2450825|49545|2450894|76140|602470|2400|46730|76140|602470|2400|46730|4|91|3|5|1508|187|554|34|44.44|49.77|26.87|778.60|913.58|1510.96|1692.18|73.08|0.00|371.96|913.58|986.66|1285.54|1358.62|-597.38| +2450825|49545|2450902|76140|602470|2400|46730|76140|602470|2400|46730|4|96|18|5|8878|7|554|5|81.49|172.75|12.09|803.30|60.45|407.45|863.75|0.24|54.40|207.30|6.05|6.29|213.35|213.59|-401.40| +2450825|49545|2450900|76140|602470|2400|46730|76140|602470|2400|46730|4|106|20|5|14504|243|554|59|58.19|118.12|83.86|2021.34|4947.74|3433.21|6969.08|270.14|445.29|2996.61|4502.45|4772.59|7499.06|7769.20|1069.24| +2450825|49545|2450841|76140|602470|2400|46730|76140|602470|2400|46730|4|11|14|5|1165|32|554|47|71.95|174.83|87.41|4108.74|4108.27|3381.65|8217.01|369.74|0.00|3286.71|4108.27|4478.01|7394.98|7764.72|726.62| +2450825|49545|2450861|76140|602470|2400|46730|76140|602470|2400|46730|4|59|8|3|7039|81|554|12|38.23|73.40|36.70|440.40|440.40|458.76|880.80|0.00|0.00|273.00|440.40|440.40|713.40|713.40|-18.36| +2450825|49545|2450846|76140|602470|2400|46730|76140|602470|2400|46730|4|77|11|4|9688|238|554|5|31.72|52.33|45.52|34.05|227.60|158.60|261.65|0.00|0.00|70.60|227.60|227.60|298.20|298.20|69.00| +2450825|49545|2450853|76140|602470|2400|46730|76140|602470|2400|46730|4|90|8|3|6620|56|554|65|7.82|12.04|1.08|712.40|70.20|508.30|782.60|6.31|0.00|171.60|70.20|76.51|241.80|248.11|-438.10| +2450825|70137|2450884|48861|1670483|458|14480|48861|1670483|458|14480|2|52|13|2|3722|64|555|43|49.52|83.19|55.73|1180.78|2396.39|2129.36|3577.17|119.81|0.00|929.66|2396.39|2516.20|3326.05|3445.86|267.03| +2450825|70137|2450830|48861|1670483|458|14480|48861|1670483|458|14480|2|68|12|4|16339|108|555|9|72.35|148.31|123.09|226.98|1107.81|651.15|1334.79|66.46|0.00|80.01|1107.81|1174.27|1187.82|1254.28|456.66| +2450825|70137|2450899|48861|1670483|458|14480|48861|1670483|458|14480|2|31|2|3|380|217|555|99|64.42|161.69|42.03|11846.34|4160.97|6377.58|16007.31|374.48|0.00|0.00|4160.97|4535.45|4160.97|4535.45|-2216.61| +2450825|70137|2450875|48861|1670483|458|14480|48861|1670483|458|14480|2|54|10|4|4612|106|555|75|3.03|7.81|4.13|276.00|309.75|227.25|585.75|24.78|0.00|134.25|309.75|334.53|444.00|468.78|82.50| +2450825|70137|2450852|48861|1670483|458|14480|48861|1670483|458|14480|2|104|10|1|13432|23|555|27|67.36|150.21|46.56|2798.55|1257.12|1818.72|4055.67|62.85|0.00|446.04|1257.12|1319.97|1703.16|1766.01|-561.60| +2450825|70137|2450905|48861|1670483|458|14480|48861|1670483|458|14480|2|101|12|3|5602|186|555|5|35.01|103.27|39.24|320.15|196.20|175.05|516.35|1.96|0.00|25.80|196.20|198.16|222.00|223.96|21.15| +2450825|70137|2450910|48861|1670483|458|14480|48861|1670483|458|14480|2|2|1|4|16186|7|555|81|32.27|55.18|2.75|4246.83|222.75|2613.87|4469.58|4.45|0.00|1251.45|222.75|227.20|1474.20|1478.65|-2391.12| +2450825|70137|2450834|48861|1670483|458|14480|48861|1670483|458|14480|2|72|10|4|11302|146|555|68|46.92|85.39|58.06|1858.44|3948.08|3190.56|5806.52|84.09|1144.94|1567.40|2803.14|2887.23|4370.54|4454.63|-387.42| +2450825|70137|2450840|48861|1670483|458|14480|48861|1670483|458|14480|2|60|15|3|10087|74|555|58|89.24|188.29|171.34|983.10|9937.72|5175.92|10920.82|95.40|8745.19|2839.10|1192.53|1287.93|4031.63|4127.03|-3983.39| +2450825|70137|2450848|48861|1670483|458|14480|48861|1670483|458|14480|2|57|20|3|14698|11|555|47|6.36|18.88|5.66|621.34|266.02|298.92|887.36|0.00|0.00|328.06|266.02|266.02|594.08|594.08|-32.90| +2450825|58570|2450889|62953|1036758|1316|28321|62953|1036758|1316|28321|1|43|18|1|7220|187|556|45|76.90|183.02|49.41|6012.45|2223.45|3460.50|8235.90|133.40|0.00|3046.95|2223.45|2356.85|5270.40|5403.80|-1237.05| +2450825|58570|2450885|62953|1036758|1316|28321|62953|1036758|1316|28321|1|33|2|2|956|37|556|1|30.49|37.19|2.97|34.22|2.97|30.49|37.19|0.20|0.00|18.22|2.97|3.17|21.19|21.39|-27.52| +2450825|58570|2450887|62953|1036758|1316|28321|62953|1036758|1316|28321|1|25|16|2|14341|132|556|81|50.90|78.38|54.86|1905.12|4443.66|4122.90|6348.78|399.92|0.00|1142.10|4443.66|4843.58|5585.76|5985.68|320.76| +2450825|58570|2450890|62953|1036758|1316|28321|62953|1036758|1316|28321|1|52|8|4|1579|64|556|38|8.74|10.57|5.91|177.08|224.58|332.12|401.66|13.47|0.00|136.42|224.58|238.05|361.00|374.47|-107.54| +2450825|58570|2450892|62953|1036758|1316|28321|62953|1036758|1316|28321|1|91|3|5|446|131|556|4|17.71|33.11|15.56|70.20|62.24|70.84|132.44|0.37|24.89|51.64|37.35|37.72|88.99|89.36|-33.49| +2450825|58570|2450882|62953|1036758|1316|28321|62953|1036758|1316|28321|1|44|7|2|14839|175|556|78|8.68|25.17|4.78|1590.42|372.84|677.04|1963.26|11.93|74.56|450.84|298.28|310.21|749.12|761.05|-378.76| +2450825|58570|2450835|62953|1036758|1316|28321|62953|1036758|1316|28321|1|103|19|4|14858|214|556|56|43.03|108.43|7.59|5647.04|425.04|2409.68|6072.08|25.50|0.00|425.04|425.04|450.54|850.08|875.58|-1984.64| +2450825|58570|2450881|62953|1036758|1316|28321|62953|1036758|1316|28321|1|74|7|5|13717|236|556|18|1.67|1.67|0.00|30.06|0.00|30.06|30.06|0.00|0.00|10.44|0.00|0.00|10.44|10.44|-30.06| +2450825|58570|2450863|62953|1036758|1316|28321|62953|1036758|1316|28321|1|106|8|3|4108|41|556|96|65.08|95.01|11.40|8026.56|1094.40|6247.68|9120.96|32.83|0.00|2462.40|1094.40|1127.23|3556.80|3589.63|-5153.28| +2450825|58570|2450860|62953|1036758|1316|28321|62953|1036758|1316|28321|1|74|18|3|16444|176|556|6|83.14|229.46|199.63|178.98|1197.78|498.84|1376.76|107.80|0.00|385.44|1197.78|1305.58|1583.22|1691.02|698.94| +2450825|81898|2450858|85244|1816037|5677|40917|85244|1816037|5677|40917|2|68|4|1|1496|298|557|82|83.18|155.54|97.99|4719.10|8035.18|6820.76|12754.28|80.35|0.00|892.16|8035.18|8115.53|8927.34|9007.69|1214.42| +2450825|81898|2450834|85244|1816037|5677|40917|85244|1816037|5677|40917|2|69|16|4|8560|62|557|27|55.30|149.86|109.39|1092.69|2953.53|1493.10|4046.22|177.21|0.00|566.46|2953.53|3130.74|3519.99|3697.20|1460.43| +2450825|81898|2450885|85244|1816037|5677|40917|85244|1816037|5677|40917|2|37|13|4|3620|18|557|99|36.47|68.92|57.20|1160.28|5662.80|3610.53|6823.08|3.96|5606.17|3206.61|56.63|60.59|3263.24|3267.20|-3553.90| +2450825|81898|2450869|85244|1816037|5677|40917|85244|1816037|5677|40917|2|23|9|3|11224|295|557|57|30.58|52.59|25.24|1558.95|1438.68|1743.06|2997.63|35.96|719.34|898.89|719.34|755.30|1618.23|1654.19|-1023.72| +2450825|81898|2450865|85244|1816037|5677|40917|85244|1816037|5677|40917|2|37|7|2|10723|122|557|27|38.00|98.80|78.05|560.25|2107.35|1026.00|2667.60|126.44|0.00|26.46|2107.35|2233.79|2133.81|2260.25|1081.35| +2450825|81898|2450898|85244|1816037|5677|40917|85244|1816037|5677|40917|2|49|10|4|12595|237|557|38|79.04|112.23|10.10|3880.94|383.80|3003.52|4264.74|7.67|0.00|639.54|383.80|391.47|1023.34|1031.01|-2619.72| +2450825|81898|2450827|85244|1816037|5677|40917|85244|1816037|5677|40917|2|3|8|3|17570|268|557|43|8.90|11.74|8.68|131.58|373.24|382.70|504.82|29.85|0.00|247.25|373.24|403.09|620.49|650.34|-9.46| +2450825|81898|2450872|85244|1816037|5677|40917|85244|1816037|5677|40917|2|80|15|3|2614|131|557|81|44.69|123.79|32.18|7420.41|2606.58|3619.89|10026.99|234.59|0.00|4912.65|2606.58|2841.17|7519.23|7753.82|-1013.31| +2450825|81898|2450860|85244|1816037|5677|40917|85244|1816037|5677|40917|2|52|5|3|1069|95|557|97|57.86|170.68|88.75|7947.21|8608.75|5612.42|16555.96|0.00|0.00|7615.47|8608.75|8608.75|16224.22|16224.22|2996.33| +2450825|81898|2450899|85244|1816037|5677|40917|85244|1816037|5677|40917|2|83|7|1|2702|232|557|26|51.08|76.10|51.74|633.36|1345.24|1328.08|1978.60|121.07|0.00|811.20|1345.24|1466.31|2156.44|2277.51|17.16| +2450825|81898|2450885|85244|1816037|5677|40917|85244|1816037|5677|40917|2|105|20|1|5834|110|557|60|26.76|63.68|14.00|2980.80|840.00|1605.60|3820.80|16.80|0.00|1604.40|840.00|856.80|2444.40|2461.20|-765.60| +2450825|81898|2450838|85244|1816037|5677|40917|85244|1816037|5677|40917|2|90|18|1|7598|101|557|99|14.67|19.51|3.90|1545.39|386.10|1452.33|1931.49|15.44|0.00|289.08|386.10|401.54|675.18|690.62|-1066.23| +2450825|81898|2450862|85244|1816037|5677|40917|85244|1816037|5677|40917|2|107|12|5|8935|72|557|78|53.05|56.76|32.35|1903.98|2523.30|4137.90|4427.28|75.69|0.00|177.06|2523.30|2598.99|2700.36|2776.05|-1614.60| +2450825|81898|2450831|85244|1816037|5677|40917|85244|1816037|5677|40917|2|31|11|4|17977|270|557|31|80.34|102.03|99.98|63.55|3099.38|2490.54|3162.93|61.98|0.00|316.20|3099.38|3161.36|3415.58|3477.56|608.84| +2450825|52719|2450842|81730|1401522|867|11059|81730|1401522|867|11059|1|42|16|5|15985|112|558|59|35.64|47.75|8.59|2310.44|506.81|2102.76|2817.25|35.47|0.00|27.73|506.81|542.28|534.54|570.01|-1595.95| +2450825|52719|2450853|81730|1401522|867|11059|81730|1401522|867|11059|1|12|15|3|9967|146|558|83|77.31|176.26|167.44|732.06|13897.52|6416.73|14629.58|833.85|0.00|7314.79|13897.52|14731.37|21212.31|22046.16|7480.79| +2450825|52719|2450857|81730|1401522|867|11059|81730|1401522|867|11059|1|12|13|5|4124|77|558|39|76.05|120.15|39.64|3139.89|1545.96|2965.95|4685.85|32.46|463.78|374.79|1082.18|1114.64|1456.97|1489.43|-1883.77| +2450825|52719|2450833|81730|1401522|867|11059|81730|1401522|867|11059|1|1|8|4|1225|149|558|78|21.54|54.06|17.29|2868.06|1348.62|1680.12|4216.68|107.88|0.00|1433.64|1348.62|1456.50|2782.26|2890.14|-331.50| +2450825|52719|2450830|81730|1401522|867|11059|81730|1401522|867|11059|1|15|15|3|8848|149|558|42|80.01|128.81|34.77|3949.68|1460.34|3360.42|5410.02|58.41|0.00|0.00|1460.34|1518.75|1460.34|1518.75|-1900.08| +2450825|52719|2450883|81730|1401522|867|11059|81730|1401522|867|11059|1|10|14|4|12968|124|558|21|97.72|176.87|159.18|371.49|3342.78|2052.12|3714.27|267.42|0.00|965.58|3342.78|3610.20|4308.36|4575.78|1290.66| +2450825|52719|2450840|81730|1401522|867|11059|81730|1401522|867|11059|1|58|5|2|8600|153|558|21|84.84|194.28|44.68|3141.60|938.28|1781.64|4079.88|28.14|0.00|1101.45|938.28|966.42|2039.73|2067.87|-843.36| +2450825|52719|2450886|81730|1401522|867|11059|81730|1401522|867|11059|1|22|20|1|6166|284|558|24|18.11|53.60|10.72|1029.12|257.28|434.64|1286.40|20.58|0.00|578.88|257.28|277.86|836.16|856.74|-177.36| +2450825|52719|2450909|81730|1401522|867|11059|81730|1401522|867|11059|1|61|7|1|17152|259|558|1|14.14|33.08|23.15|9.93|23.15|14.14|33.08|0.92|0.00|6.28|23.15|24.07|29.43|30.35|9.01| +2450825|52719|2450881|81730|1401522|867|11059|81730|1401522|867|11059|1|91|1|3|13894|29|558|99|68.26|121.50|10.93|10946.43|1082.07|6757.74|12028.50|75.74|0.00|1803.78|1082.07|1157.81|2885.85|2961.59|-5675.67| +2450825|52719|2450913|81730|1401522|867|11059|81730|1401522|867|11059|1|26|9|5|14434|197|558|53|87.47|211.67|131.23|4263.32|6955.19|4635.91|11218.51|278.20|0.00|5272.44|6955.19|7233.39|12227.63|12505.83|2319.28| +2450825|52719|2450866|81730|1401522|867|11059|81730|1401522|867|11059|1|55|16|5|7177|72|558|65|5.76|12.67|5.19|486.20|337.35|374.40|823.55|30.36|0.00|139.75|337.35|367.71|477.10|507.46|-37.05| +2450825|52719|2450860|81730|1401522|867|11059|81730|1401522|867|11059|1|58|1|5|8612|16|558|93|91.74|259.62|171.34|8210.04|15934.62|8531.82|24144.66|731.39|7807.96|4345.89|8126.66|8858.05|12472.55|13203.94|-405.16| +2450825|32644|2450894|90789|201767|5545|19370|25899|915726|2704|12929|4|45|1|5|11752|160|559|58|98.72|257.65|162.31|5529.72|9413.98|5725.76|14943.70|0.00|0.00|5229.86|9413.98|9413.98|14643.84|14643.84|3688.22| +2450825|32644|2450849|90789|201767|5545|19370|25899|915726|2704|12929|4|59|17|2|7129|118|559|43|76.22|118.14|82.69|1524.35|3555.67|3277.46|5080.02|213.34|0.00|558.57|3555.67|3769.01|4114.24|4327.58|278.21| +2450825|32644|2450849|90789|201767|5545|19370|25899|915726|2704|12929|4|86|19|3|2794|269|559|40|34.65|42.61|11.50|1244.40|460.00|1386.00|1704.40|27.60|0.00|834.80|460.00|487.60|1294.80|1322.40|-926.00| +2450825|32644|2450863|90789|201767|5545|19370|25899|915726|2704|12929|4|30|5|1|9397|217|559|60|82.10|167.48|61.96|6331.20|3717.60|4926.00|10048.80|185.88|0.00|4320.60|3717.60|3903.48|8038.20|8224.08|-1208.40| +2450825|13971|2450882|25250|436779|308|13837|25250|436779|308|13837|1|21|11|3|9931|22|560|98|26.33|61.34|35.57|2525.46|3485.86|2580.34|6011.32|185.44|836.60|2403.94|2649.26|2834.70|5053.20|5238.64|68.92| +2450825|13971|2450864|25250|436779|308|13837|25250|436779|308|13837|1|91|4|1|3458|288|560|51|57.30|68.18|44.99|1182.69|2294.49|2922.30|3477.18|91.77|0.00|521.22|2294.49|2386.26|2815.71|2907.48|-627.81| +2450825|13971|2450856|25250|436779|308|13837|25250|436779|308|13837|1|104|17|2|5365|258|560|35|75.55|139.76|97.83|1467.55|3424.05|2644.25|4891.60|34.24|0.00|48.65|3424.05|3458.29|3472.70|3506.94|779.80| +2450825|13971|2450885|25250|436779|308|13837|25250|436779|308|13837|1|26|12|5|5588|298|560|98|55.17|136.26|8.17|12552.82|800.66|5406.66|13353.48|48.03|0.00|5741.82|800.66|848.69|6542.48|6590.51|-4606.00| +2450825|13971|2450863|25250|436779|308|13837|25250|436779|308|13837|1|15|16|1|3733|271|560|17|63.92|90.76|49.91|694.45|848.47|1086.64|1542.92|67.87|0.00|447.44|848.47|916.34|1295.91|1363.78|-238.17| +2450825|13971|2450868|25250|436779|308|13837|25250|436779|308|13837|1|83|17|3|14438|70|560|63|81.55|171.25|11.98|10034.01|754.74|5137.65|10788.75|15.09|0.00|4207.14|754.74|769.83|4961.88|4976.97|-4382.91| +2450825|13971|2450842|25250|436779|308|13837|25250|436779|308|13837|1|52|20|1|14110|111|560|31|29.82|78.42|76.85|48.67|2382.35|924.42|2431.02|47.64|0.00|486.08|2382.35|2429.99|2868.43|2916.07|1457.93| +2450825|13971|2450914|25250|436779|308|13837|25250|436779|308|13837|1|11|17|2|16099|60|560|49|25.52|54.86|26.88|1371.02|1317.12|1250.48|2688.14|11.85|131.71|295.47|1185.41|1197.26|1480.88|1492.73|-65.07| +2450825|13971|2450848|25250|436779|308|13837|25250|436779|308|13837|1|13|3|5|13154|64|560|57|53.48|68.45|52.02|936.51|2965.14|3048.36|3901.65|0.00|0.00|1638.18|2965.14|2965.14|4603.32|4603.32|-83.22| +2450825|48085|2450873|31891|156214|6137|39707|31891|156214|6137|39707|2|97|11|1|12962|86|561|89|98.35|123.92|29.74|8382.02|2646.86|8753.15|11028.88|238.21|0.00|3749.57|2646.86|2885.07|6396.43|6634.64|-6106.29| +2450825|48085|2450833|31891|156214|6137|39707|31891|156214|6137|39707|2|65|16|5|7606|20|561|33|4.36|6.62|1.39|172.59|45.87|143.88|218.46|4.12|0.00|98.01|45.87|49.99|143.88|148.00|-98.01| +2450825|48085|2450863|31891|156214|6137|39707|31891|156214|6137|39707|2|94|16|5|26|85|561|67|34.51|35.20|3.87|2099.11|259.29|2312.17|2358.40|21.46|20.74|235.84|238.55|260.01|474.39|495.85|-2073.62| +2450825|48085|2450875|31891|156214|6137|39707|31891|156214|6137|39707|2|3|7|5|17983|128|561|24|41.23|95.24|83.81|274.32|2011.44|989.52|2285.76|40.22|0.00|662.64|2011.44|2051.66|2674.08|2714.30|1021.92| +2450825|48085|2450857|31891|156214|6137|39707|31891|156214|6137|39707|2|42|19|3|12494|237|561|29|99.99|261.97|199.09|1823.52|5773.61|2899.71|7597.13|348.14|1905.29|987.45|3868.32|4216.46|4855.77|5203.91|968.61| +2450825|48085|2450844|31891|156214|6137|39707|31891|156214|6137|39707|2|20|15|1|14992|255|561|48|47.00|112.80|66.55|2220.00|3194.40|2256.00|5414.40|191.66|0.00|920.16|3194.40|3386.06|4114.56|4306.22|938.40| +2450825|48085|2450874|31891|156214|6137|39707|31891|156214|6137|39707|2|98|17|4|5713|2|561|6|9.87|22.60|6.32|97.68|37.92|59.22|135.60|0.14|30.71|48.78|7.21|7.35|55.99|56.13|-52.01| +2450825|16227|2450854|28142|282653|1164|7322|28142|282653|1164|7322|1|5|14|3|17086|86|562|78|58.98|126.21|74.46|4036.50|5807.88|4600.44|9844.38|464.63|0.00|1968.72|5807.88|6272.51|7776.60|8241.23|1207.44| +2450825|16227|2450900|28142|282653|1164|7322|28142|282653|1164|7322|1|103|3|2|2026|122|562|94|17.52|38.36|38.36|0.00|3605.84|1646.88|3605.84|216.35|0.00|0.00|3605.84|3822.19|3605.84|3822.19|1958.96| +2450825|16227|2450878|28142|282653|1164|7322|28142|282653|1164|7322|1|50|20|3|6068|87|562|6|12.03|31.03|30.71|1.92|184.26|72.18|186.18|0.07|182.41|39.06|1.85|1.92|40.91|40.98|-70.33| +2450825|16227|2450878|28142|282653|1164|7322|28142|282653|1164|7322|1|96|6|5|3181|174|562|55|40.41|42.83|39.83|165.00|2190.65|2222.55|2355.65|131.43|0.00|871.20|2190.65|2322.08|3061.85|3193.28|-31.90| +2450825|16227|2450882|28142|282653|1164|7322|28142|282653|1164|7322|1|107|8|1|13177|295|562|2|63.77|131.36|130.04|2.64|260.08|127.54|262.72|10.40|0.00|34.14|260.08|270.48|294.22|304.62|132.54| +2450825|16227|2450903|28142|282653|1164|7322|28142|282653|1164|7322|1|40|1|3|11426|209|562|55|48.84|60.56|50.26|566.50|2764.30|2686.20|3330.80|193.50|0.00|232.65|2764.30|2957.80|2996.95|3190.45|78.10| +2450825|16227|2450889|28142|282653|1164|7322|28142|282653|1164|7322|1|5|20|4|2768|285|562|68|4.46|5.26|2.36|197.20|160.48|303.28|357.68|4.81|0.00|38.76|160.48|165.29|199.24|204.05|-142.80| +2450825|16227|2450899|28142|282653|1164|7322|28142|282653|1164|7322|1|19|18|1|6098|294|562|63|56.58|148.23|54.84|5883.57|3454.92|3564.54|9338.49|69.09|0.00|2521.26|3454.92|3524.01|5976.18|6045.27|-109.62| +2450825|16227|2450874|28142|282653|1164|7322|28142|282653|1164|7322|1|27|16|2|15418|209|562|45|69.97|71.36|28.54|1926.90|1284.30|3148.65|3211.20|31.59|231.17|1091.70|1053.13|1084.72|2144.83|2176.42|-2095.52| +2450825|16227|2450862|28142|282653|1164|7322|28142|282653|1164|7322|1|50|20|5|4840|228|562|73|42.46|67.51|12.82|3992.37|935.86|3099.58|4928.23|65.51|0.00|1724.26|935.86|1001.37|2660.12|2725.63|-2163.72| +2450826|33891|2450877|23156|693776|1196|7816|23156|693776|1196|7816|1|52|8|1|17785|125|563|74|7.18|14.43|11.25|235.32|832.50|531.32|1067.82|8.32|0.00|63.64|832.50|840.82|896.14|904.46|301.18| +2450826|33891|2450831|23156|693776|1196|7816|23156|693776|1196|7816|1|61|11|3|10798|260|563|59|85.88|115.93|79.99|2120.46|4719.41|5066.92|6839.87|235.97|0.00|1367.62|4719.41|4955.38|6087.03|6323.00|-347.51| +2450826|33891|2450882|23156|693776|1196|7816|23156|693776|1196|7816|1|51|15|4|11740|69|563|16|43.71|71.68|70.96|11.52|1135.36|699.36|1146.88|36.33|681.21|103.20|454.15|490.48|557.35|593.68|-245.21| +2450826|33891|2450879|23156|693776|1196|7816|23156|693776|1196|7816|1|106|16|3|6596|57|563|53|69.16|200.56|146.40|2870.48|7759.20|3665.48|10629.68|0.00|0.00|1913.30|7759.20|7759.20|9672.50|9672.50|4093.72| +2450826|33891|2450904|23156|693776|1196|7816|23156|693776|1196|7816|1|72|2|1|15571|106|563|63|73.87|98.24|36.34|3899.70|2289.42|4653.81|6189.12|68.68|0.00|680.40|2289.42|2358.10|2969.82|3038.50|-2364.39| +2450826|33891|2450843|23156|693776|1196|7816|23156|693776|1196|7816|1|5|9|1|4568|68|563|63|9.75|25.35|21.80|223.65|1373.40|614.25|1597.05|27.46|0.00|79.38|1373.40|1400.86|1452.78|1480.24|759.15| +2450826|33891|2450844|23156|693776|1196|7816|23156|693776|1196|7816|1|100|1|5|7112|29|563|35|57.57|149.10|147.60|52.50|5166.00|2014.95|5218.50|154.98|0.00|2139.55|5166.00|5320.98|7305.55|7460.53|3151.05| +2450826|33891|2450866|23156|693776|1196|7816|23156|693776|1196|7816|1|31|6|5|4510|20|563|18|45.77|50.80|13.71|667.62|246.78|823.86|914.40|2.46|0.00|320.04|246.78|249.24|566.82|569.28|-577.08| +2450826|33891|2450832|23156|693776|1196|7816|23156|693776|1196|7816|1|61|1|3|5653|79|563|37|11.82|31.08|14.60|609.76|540.20|437.34|1149.96|5.40|0.00|517.26|540.20|545.60|1057.46|1062.86|102.86| +2450826|33891|2450863|23156|693776|1196|7816|23156|693776|1196|7816|1|48|9|2|1994|101|563|29|26.04|55.98|36.94|552.16|1071.26|755.16|1623.42|32.13|0.00|32.19|1071.26|1103.39|1103.45|1135.58|316.10| +2450826|33891|2450865|23156|693776|1196|7816|23156|693776|1196|7816|1|60|6|3|1268|171|563|85|72.90|122.47|84.50|3227.45|7182.50|6196.50|10409.95|502.77|0.00|2289.90|7182.50|7685.27|9472.40|9975.17|986.00| +2450826|33891|2450878|23156|693776|1196|7816|23156|693776|1196|7816|1|48|12|4|3640|189|563|95|69.46|76.40|3.82|6895.10|362.90|6598.70|7258.00|25.40|0.00|2104.25|362.90|388.30|2467.15|2492.55|-6235.80| +2450826|75553|2450848|17662|372808|401|24747|17662|372808|401|24747|2|80|12|3|15730|31|564|62|67.20|117.60|75.26|2625.08|4666.12|4166.40|7291.20|28.46|1819.78|1166.22|2846.34|2874.80|4012.56|4041.02|-1320.06| +2450826|75553|2450894|17662|372808|401|24747|17662|372808|401|24747|2|8|1|5|17192|42|564|92|88.73|105.58|45.39|5537.48|4175.88|8163.16|9713.36|39.25|250.55|1942.12|3925.33|3964.58|5867.45|5906.70|-4237.83| +2450826|75553|2450909|17662|372808|401|24747|17662|372808|401|24747|2|61|7|2|3133|215|564|6|8.62|24.99|12.49|75.00|74.94|51.72|149.94|6.74|0.00|58.44|74.94|81.68|133.38|140.12|23.22| +2450826|75553|2450859|17662|372808|401|24747|17662|372808|401|24747|2|27|10|5|9235|123|564|17|41.93|111.53|0.00|1896.01|0.00|712.81|1896.01|0.00|0.00|758.37|0.00|0.00|758.37|758.37|-712.81| +2450826|75553|2450907|17662|372808|401|24747|17662|372808|401|24747|2|57|20|3|8671|67|564|69|87.28|136.15|122.53|939.78|8454.57|6022.32|9394.35|253.63|0.00|2724.12|8454.57|8708.20|11178.69|11432.32|2432.25| +2450826|75553|2450897|17662|372808|401|24747|17662|372808|401|24747|2|62|11|4|5896|139|564|38|91.13|175.88|12.31|6215.66|467.78|3462.94|6683.44|18.71|0.00|1202.70|467.78|486.49|1670.48|1689.19|-2995.16| +2450826|75553|2450865|17662|372808|401|24747|17662|372808|401|24747|2|94|2|3|1372|155|564|27|91.18|207.89|49.89|4266.00|1347.03|2461.86|5613.03|13.47|0.00|1347.03|1347.03|1360.50|2694.06|2707.53|-1114.83| +2450826|45543|2450891|80115|1711198|4681|4164|80115|1711198|4681|4164|1|93|5|1|16148|66|565|61|21.62|39.99|23.19|1024.80|1414.59|1318.82|2439.39|26.31|976.06|1048.59|438.53|464.84|1487.12|1513.43|-880.29| +2450826|45543|2450879|80115|1711198|4681|4164|80115|1711198|4681|4164|1|90|5|5|17179|278|565|14|73.20|137.61|33.02|1464.26|462.28|1024.80|1926.54|27.73|0.00|924.70|462.28|490.01|1386.98|1414.71|-562.52| +2450826|45543|2450902|80115|1711198|4681|4164|80115|1711198|4681|4164|1|103|10|1|6721|169|565|5|1.02|2.93|1.11|9.10|5.55|5.10|14.65|0.05|0.00|4.65|5.55|5.60|10.20|10.25|0.45| +2450826|45543|2450906|80115|1711198|4681|4164|80115|1711198|4681|4164|1|46|15|1|16918|168|565|44|69.20|79.58|39.79|1750.76|1750.76|3044.80|3501.52|70.03|0.00|1155.44|1750.76|1820.79|2906.20|2976.23|-1294.04| +2450826|45543|2450881|80115|1711198|4681|4164|80115|1711198|4681|4164|1|33|19|3|922|6|565|98|54.64|124.57|13.70|10865.26|1342.60|5354.72|12207.86|40.27|0.00|3906.28|1342.60|1382.87|5248.88|5289.15|-4012.12| +2450826|45543|2450874|80115|1711198|4681|4164|80115|1711198|4681|4164|1|5|16|4|13030|300|565|66|41.33|80.18|36.88|2857.80|2434.08|2727.78|5291.88|48.68|0.00|370.26|2434.08|2482.76|2804.34|2853.02|-293.70| +2450826|45543|2450850|80115|1711198|4681|4164|80115|1711198|4681|4164|1|16|8|4|13244|101|565|97|59.16|146.12|130.04|1559.76|12613.88|5738.52|14173.64|35.31|9081.99|991.34|3531.89|3567.20|4523.23|4558.54|-2206.63| +2450826|45543|2450907|80115|1711198|4681|4164|80115|1711198|4681|4164|1|103|13|2|12844|178|565|69|1.11|2.13|1.17|66.24|80.73|76.59|146.97|0.80|0.00|26.22|80.73|81.53|106.95|107.75|4.14| +2450826|76428|2450843|71511|1468115|4719|6923|71511|1468115|4719|6923|4|31|17|1|8413|292|566|9|46.89|75.49|22.64|475.65|203.76|422.01|679.41|18.33|0.00|27.09|203.76|222.09|230.85|249.18|-218.25| +2450826|76428|2450838|71511|1468115|4719|6923|71511|1468115|4719|6923|4|40|11|1|10606|166|566|65|81.83|180.02|162.01|1170.65|10530.65|5318.95|11701.30|31.59|8951.05|5265.00|1579.60|1611.19|6844.60|6876.19|-3739.35| +2450826|76428|2450894|71511|1468115|4719|6923|71511|1468115|4719|6923|4|60|9|3|961|279|566|86|95.93|262.84|2.62|22378.92|225.32|8249.98|22604.24|6.75|0.00|10849.76|225.32|232.07|11075.08|11081.83|-8024.66| +2450826|76428|2450878|71511|1468115|4719|6923|71511|1468115|4719|6923|4|55|13|3|5305|147|566|74|72.54|151.60|110.66|3029.56|8188.84|5367.96|11218.40|302.16|4831.41|1121.84|3357.43|3659.59|4479.27|4781.43|-2010.53| +2450826|76428|2450901|71511|1468115|4719|6923|71511|1468115|4719|6923|4|48|16|5|12637|116|566|99|84.60|203.04|152.28|5025.24|15075.72|8375.40|20100.96|54.27|14472.69|5025.24|603.03|657.30|5628.27|5682.54|-7772.37| +2450826|76428|2450894|71511|1468115|4719|6923|71511|1468115|4719|6923|4|23|2|4|5707|290|566|3|50.41|85.69|56.55|87.42|169.65|151.23|257.07|2.44|128.93|69.39|40.72|43.16|110.11|112.55|-110.51| +2450826|76428|2450856|71511|1468115|4719|6923|71511|1468115|4719|6923|4|62|1|2|9595|7|566|67|81.04|100.48|4.01|6463.49|268.67|5429.68|6732.16|5.37|0.00|2827.40|268.67|274.04|3096.07|3101.44|-5161.01| +2450826|49401|2450898|21925|1016142|2806|45423|21925|1016142|2806|45423|2|91|11|2|15646|117|567|41|40.82|116.33|52.34|2623.59|2145.94|1673.62|4769.53|150.21|0.00|285.77|2145.94|2296.15|2431.71|2581.92|472.32| +2450826|49401|2450861|21925|1016142|2806|45423|21925|1016142|2806|45423|2|60|13|2|14198|254|567|36|21.60|25.27|21.47|136.80|772.92|777.60|909.72|38.64|0.00|36.36|772.92|811.56|809.28|847.92|-4.68| +2450826|49401|2450871|21925|1016142|2806|45423|21925|1016142|2806|45423|2|81|2|2|4624|165|567|47|78.85|110.39|83.89|1245.50|3942.83|3705.95|5188.33|91.47|2799.40|1711.74|1143.43|1234.90|2855.17|2946.64|-2562.52| +2450826|49401|2450842|21925|1016142|2806|45423|21925|1016142|2806|45423|2|6|10|5|13609|185|567|82|71.16|116.70|75.85|3349.70|6219.70|5835.12|9569.40|559.77|0.00|2200.88|6219.70|6779.47|8420.58|8980.35|384.58| +2450826|49401|2450841|21925|1016142|2806|45423|21925|1016142|2806|45423|2|79|6|4|11011|33|567|94|12.41|25.44|17.04|789.60|1601.76|1166.54|2391.36|62.78|816.89|47.00|784.87|847.65|831.87|894.65|-381.67| +2450826|49401|2450914|21925|1016142|2806|45423|21925|1016142|2806|45423|2|42|2|1|13822|251|567|40|44.14|119.17|104.86|572.40|4194.40|1765.60|4766.80|0.00|0.00|1096.00|4194.40|4194.40|5290.40|5290.40|2428.80| +2450826|49401|2450836|21925|1016142|2806|45423|21925|1016142|2806|45423|2|82|8|1|11852|170|567|99|47.00|131.60|5.26|12507.66|520.74|4653.00|13028.40|10.41|0.00|0.00|520.74|531.15|520.74|531.15|-4132.26| +2450826|73571|2450891|11072|732087|4655|25270|11072|732087|4655|25270|4|35|20|2|142|291|568|5|84.16|92.57|56.46|180.55|282.30|420.80|462.85|5.64|0.00|231.40|282.30|287.94|513.70|519.34|-138.50| +2450826|73571|2450886|11072|732087|4655|25270|11072|732087|4655|25270|4|98|15|4|12883|49|568|92|58.07|108.01|103.68|398.36|9538.56|5342.44|9936.92|667.69|0.00|4272.48|9538.56|10206.25|13811.04|14478.73|4196.12| +2450826|73571|2450876|11072|732087|4655|25270|11072|732087|4655|25270|4|97|5|1|7000|210|568|85|38.80|94.28|44.31|4247.45|3766.35|3298.00|8013.80|263.64|0.00|400.35|3766.35|4029.99|4166.70|4430.34|468.35| +2450826|73571|2450849|11072|732087|4655|25270|11072|732087|4655|25270|4|67|15|2|13966|121|568|91|62.56|116.36|116.36|0.00|10588.76|5692.96|10588.76|423.55|0.00|5082.35|10588.76|11012.31|15671.11|16094.66|4895.80| +2450826|73571|2450844|11072|732087|4655|25270|11072|732087|4655|25270|4|71|6|5|7052|230|568|54|9.01|21.44|21.22|11.88|1145.88|486.54|1157.76|22.91|0.00|300.78|1145.88|1168.79|1446.66|1469.57|659.34| +2450826|73571|2450866|11072|732087|4655|25270|11072|732087|4655|25270|4|81|13|3|626|263|568|93|76.41|195.60|111.49|7822.23|10368.57|7106.13|18190.80|0.00|0.00|544.98|10368.57|10368.57|10913.55|10913.55|3262.44| +2450826|73571|2450850|11072|732087|4655|25270|11072|732087|4655|25270|4|44|18|1|10255|98|568|67|58.16|79.67|77.27|160.80|5177.09|3896.72|5337.89|25.36|2640.31|1120.91|2536.78|2562.14|3657.69|3683.05|-1359.94| +2450826|73571|2450899|11072|732087|4655|25270|11072|732087|4655|25270|4|3|6|3|13147|40|568|67|20.51|56.40|47.37|605.01|3173.79|1374.17|3778.80|63.47|0.00|1397.62|3173.79|3237.26|4571.41|4634.88|1799.62| +2450826|73571|2450913|11072|732087|4655|25270|11072|732087|4655|25270|4|6|6|3|9278|154|568|67|71.32|95.56|74.53|1409.01|4993.51|4778.44|6402.52|49.93|0.00|767.82|4993.51|5043.44|5761.33|5811.26|215.07| +2450826|74638|2450866|92316|394648|5780|42512|92316|394648|5780|42512|4|3|6|2|4382|131|569|20|65.68|129.38|45.28|1682.00|905.60|1313.60|2587.60|45.28|0.00|1086.60|905.60|950.88|1992.20|2037.48|-408.00| +2450826|74638|2450854|92316|394648|5780|42512|92316|394648|5780|42512|4|50|20|5|17491|156|569|86|24.41|63.46|50.76|1092.20|4365.36|2099.26|5457.56|86.43|43.65|1910.06|4321.71|4408.14|6231.77|6318.20|2222.45| +2450826|74638|2450850|92316|394648|5780|42512|92316|394648|5780|42512|4|71|6|4|17275|212|569|78|40.55|110.70|60.88|3885.96|4748.64|3162.90|8634.60|237.43|0.00|2676.18|4748.64|4986.07|7424.82|7662.25|1585.74| +2450826|74638|2450883|92316|394648|5780|42512|92316|394648|5780|42512|4|104|4|3|314|9|569|74|20.73|57.21|33.75|1736.04|2497.50|1534.02|4233.54|74.92|0.00|677.10|2497.50|2572.42|3174.60|3249.52|963.48| +2450826|74638|2450848|92316|394648|5780|42512|92316|394648|5780|42512|4|34|15|1|8173|145|569|17|43.52|101.83|40.73|1038.70|692.41|739.84|1731.11|0.00|0.00|155.72|692.41|692.41|848.13|848.13|-47.43| +2450826|74638|2450838|92316|394648|5780|42512|92316|394648|5780|42512|4|78|11|1|5876|284|569|18|7.07|16.68|3.33|240.30|59.94|127.26|300.24|0.00|0.00|29.88|59.94|59.94|89.82|89.82|-67.32| +2450826|19174|2450899|57245|1387816|5587|25358|57245|1387816|5587|25358|4|77|12|5|368|48|570|31|62.89|123.89|89.20|1075.39|2765.20|1949.59|3840.59|221.21|0.00|499.10|2765.20|2986.41|3264.30|3485.51|815.61| +2450826|19174|2450887|57245|1387816|5587|25358|57245|1387816|5587|25358|4|108|18|3|13438|51|570|6|80.05|108.06|23.77|505.74|142.62|480.30|648.36|2.85|0.00|272.28|142.62|145.47|414.90|417.75|-337.68| +2450826|19174|2450875|57245|1387816|5587|25358|57245|1387816|5587|25358|4|69|19|4|5272|226|570|79|28.37|84.25|59.81|1930.76|4724.99|2241.23|6655.75|283.49|0.00|2861.38|4724.99|5008.48|7586.37|7869.86|2483.76| +2450826|19174|2450883|57245|1387816|5587|25358|57245|1387816|5587|25358|4|52|19|5|4072|5|570|98|70.76|99.77|39.90|5867.26|3910.20|6934.48|9777.46|156.40|0.00|3813.18|3910.20|4066.60|7723.38|7879.78|-3024.28| +2450826|19174|2450856|57245|1387816|5587|25358|57245|1387816|5587|25358|4|102|8|4|12403|86|570|78|66.93|105.08|63.04|3279.12|4917.12|5220.54|8196.24|98.34|0.00|1393.08|4917.12|5015.46|6310.20|6408.54|-303.42| +2450826|19174|2450912|57245|1387816|5587|25358|57245|1387816|5587|25358|4|102|6|1|10784|240|570|81|13.40|32.02|6.72|2049.30|544.32|1085.40|2593.62|9.63|223.17|751.68|321.15|330.78|1072.83|1082.46|-764.25| +2450826|19174|2450910|57245|1387816|5587|25358|57245|1387816|5587|25358|4|52|1|2|1718|206|570|65|99.22|215.30|90.42|8117.20|5877.30|6449.30|13994.50|528.95|0.00|5877.30|5877.30|6406.25|11754.60|12283.55|-572.00| +2450826|19174|2450883|57245|1387816|5587|25358|57245|1387816|5587|25358|4|17|14|3|11930|97|570|38|9.41|10.06|7.94|80.56|301.72|357.58|382.28|24.13|0.00|76.38|301.72|325.85|378.10|402.23|-55.86| +2450826|8769|2450909|8133|1436981|7076|4371|8133|1436981|7076|4371|1|65|11|3|1886|267|571|72|2.57|5.67|2.43|233.28|174.96|185.04|408.24|10.49|0.00|203.76|174.96|185.45|378.72|389.21|-10.08| +2450826|8769|2450877|8133|1436981|7076|4371|8133|1436981|7076|4371|1|102|16|3|17108|149|571|29|16.40|41.98|36.94|146.16|1071.26|475.60|1217.42|10.71|0.00|511.27|1071.26|1081.97|1582.53|1593.24|595.66| +2450826|8769|2450898|8133|1436981|7076|4371|8133|1436981|7076|4371|1|94|5|1|15715|147|571|27|57.68|95.74|16.27|2145.69|439.29|1557.36|2584.98|21.96|0.00|0.00|439.29|461.25|439.29|461.25|-1118.07| +2450826|8769|2450897|8133|1436981|7076|4371|8133|1436981|7076|4371|1|91|9|2|7790|175|571|62|30.88|63.30|54.43|549.94|3374.66|1914.56|3924.60|134.98|0.00|863.04|3374.66|3509.64|4237.70|4372.68|1460.10| +2450826|8769|2450885|8133|1436981|7076|4371|8133|1436981|7076|4371|1|67|7|4|16508|281|571|83|74.99|224.22|159.19|5397.49|13212.77|6224.17|18610.26|792.76|0.00|4466.23|13212.77|14005.53|17679.00|18471.76|6988.60| +2450826|8769|2450889|8133|1436981|7076|4371|8133|1436981|7076|4371|1|32|9|3|14437|141|571|5|8.59|14.68|13.06|8.10|65.30|42.95|73.40|5.87|0.00|28.60|65.30|71.17|93.90|99.77|22.35| +2450826|8769|2450839|8133|1436981|7076|4371|8133|1436981|7076|4371|1|3|18|4|1558|14|571|79|3.08|7.42|0.51|545.89|40.29|243.32|586.18|2.01|0.00|40.29|40.29|42.30|80.58|82.59|-203.03| +2450826|8769|2450871|8133|1436981|7076|4371|8133|1436981|7076|4371|1|65|10|5|17449|15|571|14|93.01|239.96|100.78|1948.52|1410.92|1302.14|3359.44|83.52|366.83|302.26|1044.09|1127.61|1346.35|1429.87|-258.05| +2450826|8769|2450865|8133|1436981|7076|4371|8133|1436981|7076|4371|1|69|3|1|11722|162|571|2|39.31|56.99|21.08|71.82|42.16|78.62|113.98|2.10|0.00|45.58|42.16|44.26|87.74|89.84|-36.46| +2450826|8769|2450888|8133|1436981|7076|4371|8133|1436981|7076|4371|1|75|10|1|17533|243|571|14|62.13|78.90|48.91|419.86|684.74|869.82|1104.60|61.62|0.00|331.38|684.74|746.36|1016.12|1077.74|-185.08| +2450826|8769|2450878|8133|1436981|7076|4371|8133|1436981|7076|4371|1|102|8|1|11755|119|571|39|32.01|57.29|45.25|469.56|1764.75|1248.39|2234.31|105.88|0.00|22.23|1764.75|1870.63|1786.98|1892.86|516.36| +2450826|8769|2450867|8133|1436981|7076|4371|8133|1436981|7076|4371|1|80|2|5|3668|262|571|19|91.21|95.77|81.40|273.03|1546.60|1732.99|1819.63|108.26|0.00|309.32|1546.60|1654.86|1855.92|1964.18|-186.39| +2450826|8769|2450834|8133|1436981|7076|4371|8133|1436981|7076|4371|1|39|2|4|7477|235|571|53|37.21|98.23|98.23|0.00|5206.19|1972.13|5206.19|104.12|0.00|832.63|5206.19|5310.31|6038.82|6142.94|3234.06| +2450826|23423|2450911|31681|36270|6812|19754|31681|36270|6812|19754|1|89|16|2|10520|91|572|13|54.84|163.42|24.51|1805.83|318.63|712.92|2124.46|0.76|293.13|637.26|25.50|26.26|662.76|663.52|-687.42| +2450826|23423|2450855|31681|36270|6812|19754|31681|36270|6812|19754|1|27|15|2|7394|226|572|47|41.35|42.17|39.63|119.38|1862.61|1943.45|1981.99|130.38|0.00|812.16|1862.61|1992.99|2674.77|2805.15|-80.84| +2450826|23423|2450887|31681|36270|6812|19754|31681|36270|6812|19754|1|26|3|1|14707|71|572|11|90.89|224.49|8.97|2370.72|98.67|999.79|2469.39|1.97|0.00|395.01|98.67|100.64|493.68|495.65|-901.12| +2450826|23423|2450886|31681|36270|6812|19754|31681|36270|6812|19754|1|67|2|3|4573|52|572|27|43.08|93.48|66.37|731.97|1791.99|1163.16|2523.96|48.74|1182.71|984.15|609.28|658.02|1593.43|1642.17|-553.88| +2450826|23423|2450833|31681|36270|6812|19754|31681|36270|6812|19754|1|37|3|3|9172|20|572|53|1.20|1.69|1.69|0.00|89.57|63.60|89.57|0.00|0.00|28.62|89.57|89.57|118.19|118.19|25.97| +2450826|23423|2450909|31681|36270|6812|19754|31681|36270|6812|19754|1|15|10|5|15758|25|572|10|76.99|170.14|25.52|1446.20|255.20|769.90|1701.40|9.59|15.31|255.20|239.89|249.48|495.09|504.68|-530.01| +2450826|23423|2450832|31681|36270|6812|19754|31681|36270|6812|19754|1|55|10|5|15616|188|572|58|29.96|33.55|7.38|1517.86|428.04|1737.68|1945.90|8.56|0.00|797.50|428.04|436.60|1225.54|1234.10|-1309.64| +2450826|23423|2450829|31681|36270|6812|19754|31681|36270|6812|19754|1|106|8|2|17500|213|572|34|54.80|94.25|28.27|2243.32|961.18|1863.20|3204.50|19.22|0.00|929.22|961.18|980.40|1890.40|1909.62|-902.02| +2450826|23423|2450916|31681|36270|6812|19754|31681|36270|6812|19754|1|106|11|2|12877|104|572|83|60.07|79.89|9.58|5835.73|795.14|4985.81|6630.87|47.70|0.00|1193.54|795.14|842.84|1988.68|2036.38|-4190.67| +2450826|23423|2450877|31681|36270|6812|19754|31681|36270|6812|19754|1|82|17|2|3220|62|572|80|45.21|70.97|14.19|4542.40|1135.20|3616.80|5677.60|0.00|0.00|2554.40|1135.20|1135.20|3689.60|3689.60|-2481.60| +2450826|23423|2450893|31681|36270|6812|19754|31681|36270|6812|19754|1|85|13|2|14090|42|572|57|79.69|152.20|33.48|6767.04|1908.36|4542.33|8675.40|133.58|0.00|1301.31|1908.36|2041.94|3209.67|3343.25|-2633.97| +2450826|23423|2450829|31681|36270|6812|19754|31681|36270|6812|19754|1|97|14|5|9080|185|572|76|97.19|276.99|229.90|3578.84|17472.40|7386.44|21051.24|698.89|0.00|4841.20|17472.40|18171.29|22313.60|23012.49|10085.96| +2450826|16554|2450877|9545|882645|6498|40398|9545|882645|6498|40398|4|5|13|2|15082|233|573|29|27.93|60.32|18.69|1207.27|542.01|809.97|1749.28|27.10|0.00|174.87|542.01|569.11|716.88|743.98|-267.96| +2450826|16554|2450878|9545|882645|6498|40398|9545|882645|6498|40398|4|13|17|5|17660|278|573|14|70.95|185.17|103.69|1140.72|1451.66|993.30|2592.38|29.03|0.00|1270.22|1451.66|1480.69|2721.88|2750.91|458.36| +2450826|16554|2450891|9545|882645|6498|40398|9545|882645|6498|40398|4|67|14|2|14533|61|573|79|1.56|2.09|0.08|158.79|6.32|123.24|165.11|0.31|0.00|80.58|6.32|6.63|86.90|87.21|-116.92| +2450826|16554|2450875|9545|882645|6498|40398|9545|882645|6498|40398|4|50|15|1|5222|290|573|34|25.52|71.20|58.38|435.88|1984.92|867.68|2420.80|19.45|1707.03|968.32|277.89|297.34|1246.21|1265.66|-589.79| +2450826|16554|2450861|9545|882645|6498|40398|9545|882645|6498|40398|4|9|20|2|14857|63|573|67|53.62|151.20|69.55|5470.55|4659.85|3592.54|10130.40|37.74|885.37|1114.21|3774.48|3812.22|4888.69|4926.43|181.94| +2450826|16554|2450881|9545|882645|6498|40398|9545|882645|6498|40398|4|9|4|3|8408|233|573|40|78.49|188.37|143.16|1808.40|5726.40|3139.60|7534.80|400.84|0.00|1280.80|5726.40|6127.24|7007.20|7408.04|2586.80| +2450826|23097|2450855|95121|713968|6062|18109|95121|713968|6062|18109|4|17|4|3|17920|31|574|85|5.88|13.81|7.59|528.70|645.15|499.80|1173.85|51.61|0.00|105.40|645.15|696.76|750.55|802.16|145.35| +2450826|23097|2450895|95121|713968|6062|18109|95121|713968|6062|18109|4|85|2|4|1214|182|574|25|21.33|53.53|31.58|548.75|789.50|533.25|1338.25|39.47|0.00|267.50|789.50|828.97|1057.00|1096.47|256.25| +2450826|23097|2450839|95121|713968|6062|18109|95121|713968|6062|18109|4|23|1|2|16922|120|574|22|50.30|141.84|32.62|2402.84|717.64|1106.60|3120.48|12.63|559.75|936.10|157.89|170.52|1093.99|1106.62|-948.71| +2450826|23097|2450913|95121|713968|6062|18109|95121|713968|6062|18109|4|99|11|5|12040|50|574|46|27.71|42.95|15.03|1284.32|691.38|1274.66|1975.70|62.22|0.00|908.50|691.38|753.60|1599.88|1662.10|-583.28| +2450826|67327|2450830|94041|1458793|792|7144|94041|1458793|792|7144|2|90|10|2|7363|169|575|43|82.15|134.72|37.72|4171.00|1621.96|3532.45|5792.96|32.43|0.00|2838.43|1621.96|1654.39|4460.39|4492.82|-1910.49| +2450826|67327|2450830|94041|1458793|792|7144|94041|1458793|792|7144|2|81|16|1|1924|211|575|19|89.18|198.87|87.50|2116.03|1662.50|1694.42|3778.53|0.00|0.00|1889.17|1662.50|1662.50|3551.67|3551.67|-31.92| +2450826|67327|2450916|94041|1458793|792|7144|94041|1458793|792|7144|2|77|18|3|12565|119|575|65|86.90|250.27|212.72|2440.75|13826.80|5648.50|16267.55|414.80|0.00|1301.30|13826.80|14241.60|15128.10|15542.90|8178.30| +2450826|67327|2450898|94041|1458793|792|7144|94041|1458793|792|7144|2|97|6|3|7375|13|575|5|38.81|78.39|57.22|105.85|286.10|194.05|391.95|22.88|0.00|137.15|286.10|308.98|423.25|446.13|92.05| +2450826|67327|2450894|94041|1458793|792|7144|94041|1458793|792|7144|2|43|17|4|547|62|575|51|65.95|166.85|63.40|5275.95|3233.40|3363.45|8509.35|97.00|0.00|3573.57|3233.40|3330.40|6806.97|6903.97|-130.05| +2450826|67327|2450848|94041|1458793|792|7144|94041|1458793|792|7144|2|85|16|1|6388|293|575|1|69.60|203.92|10.19|193.73|10.19|69.60|203.92|0.00|0.00|53.01|10.19|10.19|63.20|63.20|-59.41| +2450826|67327|2450905|94041|1458793|792|7144|94041|1458793|792|7144|2|29|6|5|16850|105|575|50|5.15|5.97|0.17|290.00|8.50|257.50|298.50|0.76|0.00|41.50|8.50|9.26|50.00|50.76|-249.00| +2450826|67327|2450897|94041|1458793|792|7144|94041|1458793|792|7144|2|41|2|5|9034|168|575|89|61.65|123.30|55.48|6035.98|4937.72|5486.85|10973.70|49.37|0.00|3511.05|4937.72|4987.09|8448.77|8498.14|-549.13| +2450826|67327|2450850|94041|1458793|792|7144|94041|1458793|792|7144|2|53|15|2|12085|32|575|57|57.92|163.91|154.07|560.88|8781.99|3301.44|9342.87|790.37|0.00|3269.52|8781.99|9572.36|12051.51|12841.88|5480.55| +2450826|67327|2450867|94041|1458793|792|7144|94041|1458793|792|7144|2|11|5|5|12793|147|575|39|58.93|156.75|137.94|733.59|5379.66|2298.27|6113.25|109.74|3550.57|1100.19|1829.09|1938.83|2929.28|3039.02|-469.18| +2450826|67327|2450844|94041|1458793|792|7144|94041|1458793|792|7144|2|75|19|5|5434|49|575|46|17.64|18.52|3.33|698.74|153.18|811.44|851.92|0.00|0.00|374.44|153.18|153.18|527.62|527.62|-658.26| +2450826|86229|2450889|90851|1484940|3829|3918|90851|1484940|3829|3918|2|20|8|5|124|198|576|27|5.99|14.85|10.84|108.27|292.68|161.73|400.95|14.63|0.00|11.88|292.68|307.31|304.56|319.19|130.95| +2450826|86229|2450886|90851|1484940|3829|3918|90851|1484940|3829|3918|2|34|20|1|4582|76|576|48|76.19|140.95|95.84|2165.28|4600.32|3657.12|6765.60|46.00|0.00|1217.76|4600.32|4646.32|5818.08|5864.08|943.20| +2450826|86229|2450844|90851|1484940|3829|3918|90851|1484940|3829|3918|2|42|15|3|10736|147|576|96|71.73|144.17|72.08|6920.64|6919.68|6886.08|13840.32|276.78|0.00|5951.04|6919.68|7196.46|12870.72|13147.50|33.60| +2450826|86229|2450838|90851|1484940|3829|3918|90851|1484940|3829|3918|2|10|13|3|1864|280|576|30|6.64|7.10|1.98|153.60|59.40|199.20|213.00|0.05|54.05|80.70|5.35|5.40|86.05|86.10|-193.85| +2450826|86229|2450829|90851|1484940|3829|3918|90851|1484940|3829|3918|2|79|18|5|2011|187|576|90|86.29|93.19|13.04|7213.50|1173.60|7766.10|8387.10|23.47|0.00|4109.40|1173.60|1197.07|5283.00|5306.47|-6592.50| +2450826|86229|2450890|90851|1484940|3829|3918|90851|1484940|3829|3918|2|29|13|1|3740|129|576|49|18.20|18.38|12.31|297.43|603.19|891.80|900.62|54.28|0.00|171.01|603.19|657.47|774.20|828.48|-288.61| +2450826|86229|2450907|90851|1484940|3829|3918|90851|1484940|3829|3918|2|75|18|1|11200|295|576|80|84.00|200.76|122.46|6264.00|9796.80|6720.00|16060.80|288.02|195.93|4496.80|9600.87|9888.89|14097.67|14385.69|2880.87| +2450826|86229|2450887|90851|1484940|3829|3918|90851|1484940|3829|3918|2|11|13|4|9128|34|576|89|94.22|263.81|65.95|17609.54|5869.55|8385.58|23479.09|0.00|0.00|8686.40|5869.55|5869.55|14555.95|14555.95|-2516.03| +2450826|86229|2450886|90851|1484940|3829|3918|90851|1484940|3829|3918|2|105|18|1|884|92|576|48|75.94|208.07|20.80|8988.96|998.40|3645.12|9987.36|58.10|29.95|599.04|968.45|1026.55|1567.49|1625.59|-2676.67| +2450826|86229|2450866|90851|1484940|3829|3918|90851|1484940|3829|3918|2|90|9|2|15260|299|576|82|77.64|182.45|145.96|2992.18|11968.72|6366.48|14960.90|119.68|0.00|5984.36|11968.72|12088.40|17953.08|18072.76|5602.24| +2450826|42693|2450904|60626|1668942|6058|14021|60626|1668942|6058|14021|2|10|1|5|16342|125|577|85|75.92|206.50|74.34|11233.60|6318.90|6453.20|17552.50|115.00|568.70|0.00|5750.20|5865.20|5750.20|5865.20|-703.00| +2450826|42693|2450880|60626|1668942|6058|14021|60626|1668942|6058|14021|2|23|11|5|3361|35|577|57|72.71|217.40|117.39|5700.57|6691.23|4144.47|12391.80|0.00|0.00|371.64|6691.23|6691.23|7062.87|7062.87|2546.76| +2450826|42693|2450914|60626|1668942|6058|14021|60626|1668942|6058|14021|2|104|2|1|13642|266|577|35|30.14|81.67|80.85|28.70|2829.75|1054.90|2858.45|28.29|0.00|1372.00|2829.75|2858.04|4201.75|4230.04|1774.85| +2450826|42693|2450914|60626|1668942|6058|14021|60626|1668942|6058|14021|2|25|13|5|1288|247|577|85|96.85|273.11|54.62|18571.65|4642.70|8232.25|23214.35|278.56|0.00|5570.90|4642.70|4921.26|10213.60|10492.16|-3589.55| +2450826|42693|2450839|60626|1668942|6058|14021|60626|1668942|6058|14021|2|85|5|2|11342|158|577|72|44.74|123.03|38.13|6112.80|2745.36|3221.28|8858.16|192.17|0.00|2479.68|2745.36|2937.53|5225.04|5417.21|-475.92| +2450826|42693|2450850|60626|1668942|6058|14021|60626|1668942|6058|14021|2|1|15|5|15788|180|577|60|60.27|111.49|16.72|5686.20|1003.20|3616.20|6689.40|24.57|652.08|3344.40|351.12|375.69|3695.52|3720.09|-3265.08| +2450826|42693|2450912|60626|1668942|6058|14021|60626|1668942|6058|14021|2|71|14|1|6607|45|577|2|16.11|34.63|15.58|38.10|31.16|32.22|69.26|1.24|0.00|33.92|31.16|32.40|65.08|66.32|-1.06| +2450826|38732|2450854|21039|1024557|6828|36746|21039|1024557|6828|36746|1|4|6|3|1912|52|578|52|91.93|173.74|12.16|8402.16|632.32|4780.36|9034.48|18.96|0.00|632.32|632.32|651.28|1264.64|1283.60|-4148.04| +2450826|38732|2450845|21039|1024557|6828|36746|21039|1024557|6828|36746|1|17|12|5|8144|253|578|45|27.06|60.61|0.60|2700.45|27.00|1217.70|2727.45|0.49|2.43|1199.70|24.57|25.06|1224.27|1224.76|-1193.13| +2450826|38732|2450840|21039|1024557|6828|36746|21039|1024557|6828|36746|1|58|1|2|9938|76|578|8|35.16|102.31|12.27|720.32|98.16|281.28|818.48|2.94|0.00|343.76|98.16|101.10|441.92|444.86|-183.12| +2450826|38732|2450896|21039|1024557|6828|36746|21039|1024557|6828|36746|1|71|2|5|8290|106|578|44|39.74|115.64|58.97|2493.48|2594.68|1748.56|5088.16|155.68|0.00|152.24|2594.68|2750.36|2746.92|2902.60|846.12| +2450826|38732|2450892|21039|1024557|6828|36746|21039|1024557|6828|36746|1|59|9|5|6242|248|578|19|20.07|58.40|23.94|654.74|454.86|381.33|1109.60|4.54|0.00|33.25|454.86|459.40|488.11|492.65|73.53| +2450826|38732|2450905|21039|1024557|6828|36746|21039|1024557|6828|36746|1|74|6|4|17252|39|578|31|46.56|77.75|18.66|1831.79|578.46|1443.36|2410.25|46.27|0.00|409.51|578.46|624.73|987.97|1034.24|-864.90| +2450826|38732|2450872|21039|1024557|6828|36746|21039|1024557|6828|36746|1|72|1|2|778|14|578|60|84.48|119.11|29.77|5360.40|1786.20|5068.80|7146.60|35.72|0.00|2429.40|1786.20|1821.92|4215.60|4251.32|-3282.60| +2450826|38732|2450852|21039|1024557|6828|36746|21039|1024557|6828|36746|1|4|1|3|11398|297|578|81|82.08|175.65|21.07|12520.98|1706.67|6648.48|14227.65|153.60|0.00|141.75|1706.67|1860.27|1848.42|2002.02|-4941.81| +2450826|38732|2450876|21039|1024557|6828|36746|21039|1024557|6828|36746|1|105|20|3|16687|25|578|34|42.59|68.99|64.85|140.76|2204.90|1448.06|2345.66|51.81|1168.59|891.14|1036.31|1088.12|1927.45|1979.26|-411.75| +2450826|38732|2450881|21039|1024557|6828|36746|21039|1024557|6828|36746|1|104|4|1|3592|57|578|20|38.49|91.22|51.99|784.60|1039.80|769.80|1824.40|41.59|0.00|547.20|1039.80|1081.39|1587.00|1628.59|270.00| +2450826|38732|2450846|21039|1024557|6828|36746|21039|1024557|6828|36746|1|7|8|1|13573|46|578|22|13.49|28.46|28.17|6.38|619.74|296.78|626.12|49.57|0.00|37.40|619.74|669.31|657.14|706.71|322.96| +2450826|38732|2450828|21039|1024557|6828|36746|21039|1024557|6828|36746|1|51|10|2|7400|169|578|32|69.92|175.49|31.58|4605.12|1010.56|2237.44|5615.68|10.10|0.00|112.00|1010.56|1020.66|1122.56|1132.66|-1226.88| +2450826|38732|2450883|21039|1024557|6828|36746|21039|1024557|6828|36746|1|98|10|2|11134|276|578|11|45.40|106.23|12.74|1028.39|140.14|499.40|1168.53|12.61|0.00|58.41|140.14|152.75|198.55|211.16|-359.26| +2450826|83670|2450890|86548|1590578|5808|41085|86548|1590578|5808|41085|2|61|1|2|518|217|579|57|80.04|216.90|86.76|7417.98|4945.32|4562.28|12363.30|445.07|0.00|6181.65|4945.32|5390.39|11126.97|11572.04|383.04| +2450826|83670|2450890|86548|1590578|5808|41085|86548|1590578|5808|41085|2|71|14|3|5551|215|579|25|69.57|75.83|58.38|436.25|1459.50|1739.25|1895.75|34.59|306.49|606.50|1153.01|1187.60|1759.51|1794.10|-586.24| +2450826|83670|2450896|86548|1590578|5808|41085|86548|1590578|5808|41085|2|33|6|2|10405|25|579|50|83.67|230.09|230.09|0.00|11504.50|4183.50|11504.50|345.13|0.00|1610.50|11504.50|11849.63|13115.00|13460.13|7321.00| +2450826|83670|2450870|86548|1590578|5808|41085|86548|1590578|5808|41085|2|2|1|4|7027|243|579|54|31.29|40.36|3.63|1983.42|196.02|1689.66|2179.44|3.92|0.00|43.20|196.02|199.94|239.22|243.14|-1493.64| +2450826|83670|2450866|86548|1590578|5808|41085|86548|1590578|5808|41085|2|8|2|5|3109|231|579|98|18.03|28.66|26.94|168.56|2640.12|1766.94|2808.68|105.60|0.00|814.38|2640.12|2745.72|3454.50|3560.10|873.18| +2450826|83670|2450881|86548|1590578|5808|41085|86548|1590578|5808|41085|2|106|3|3|6410|206|579|84|22.43|26.69|25.88|68.04|2173.92|1884.12|2241.96|130.43|0.00|604.80|2173.92|2304.35|2778.72|2909.15|289.80| +|83670||86548||||||5808|41085|2|62|19||7172||579|22|||11.80|||762.30||10.38|0.00||259.60||738.76|749.14|-502.70| +2450826|83670|2450853|86548|1590578|5808|41085|86548|1590578|5808|41085|2|104|8|2|14050|48|579|37|3.05|3.44|1.06|88.06|39.22|112.85|127.28|0.00|27.45|29.23|11.77|11.77|41.00|41.00|-101.08| +2450826|83670|2450905|86548|1590578|5808|41085|86548|1590578|5808|41085|2|14|20|1|4981|160|579|39|69.88|154.43|0.00|6022.77|0.00|2725.32|6022.77|0.00|0.00|60.06|0.00|0.00|60.06|60.06|-2725.32| +2450826|83670|2450891|86548|1590578|5808|41085|86548|1590578|5808|41085|2|69|10|5|9158|163|579|61|74.63|136.57|99.69|2249.68|6081.09|4552.43|8330.77|72.97|3648.65|3165.29|2432.44|2505.41|5597.73|5670.70|-2119.99| +2450826|83670|2450906|86548|1590578|5808|41085|86548|1590578|5808|41085|2|58|14|1|12086|65|579|14|11.20|33.37|27.69|79.52|387.66|156.80|467.18|31.01|0.00|121.38|387.66|418.67|509.04|540.05|230.86| +2450826|56138|2450893|99155|850445|935|28225|99155|850445|935|28225|2|81|9|3|1693|264|580|88|79.95|106.33|45.72|5333.68|4023.36|7035.60|9357.04|80.46|0.00|1122.00|4023.36|4103.82|5145.36|5225.82|-3012.24| +2450826|56138|2450901|99155|850445|935|28225|99155|850445|935|28225|2|2|14|4|10537|106|580|26|58.44|65.45|49.08|425.62|1276.08|1519.44|1701.70|12.76|0.00|595.40|1276.08|1288.84|1871.48|1884.24|-243.36| +2450826|56138|2450846|99155|850445|935|28225|99155|850445|935|28225|2|58|15|4|17794|159|580|76|23.52|30.34|28.51|139.08|2166.76|1787.52|2305.84|173.34|0.00|553.28|2166.76|2340.10|2720.04|2893.38|379.24| +2450826|56138|2450909|99155|850445|935|28225|99155|850445|935|28225|2|82|16|4|2128|170|580|12|36.38|106.22|13.80|1109.04|165.60|436.56|1274.64|8.84|18.21|433.32|147.39|156.23|580.71|589.55|-289.17| +2450826|56138|2450854|99155|850445|935|28225|99155|850445|935|28225|2|97|11|3|6103|271|580|93|4.94|8.74|3.67|471.51|341.31|459.42|812.82|0.92|331.07|292.02|10.24|11.16|302.26|303.18|-449.18| +2450826|56138|2450866|99155|850445|935|28225|99155|850445|935|28225|2|62|4|5|12268|280|580|81|16.67|34.00|22.10|963.90|1790.10|1350.27|2754.00|30.78|250.61|743.58|1539.49|1570.27|2283.07|2313.85|189.22| +2450826|56138|2450840|99155|850445|935|28225|99155|850445|935|28225|2|3|3|2|848|252|580|12|72.99|145.25|34.86|1324.68|418.32|875.88|1743.00|20.91|0.00|278.88|418.32|439.23|697.20|718.11|-457.56| +2450826|56138|2450916|99155|850445|935|28225|99155|850445|935|28225|2|76|10|4|11071|247|580|21|54.30|158.01|58.46|2090.55|1227.66|1140.30|3318.21|61.38|0.00|729.96|1227.66|1289.04|1957.62|2019.00|87.36| +2450826|56138|2450867|99155|850445|935|28225|99155|850445|935|28225|2|4|4|4|17056|147|580|32|20.52|48.42|9.68|1239.68|309.76|656.64|1549.44|9.29|0.00|123.84|309.76|319.05|433.60|442.89|-346.88| +2450826|76660|2450902|90408|268724|4640|13309|90408|268724|4640|13309|4|54|4|4|17612|89|581|46|89.10|124.74|84.82|1836.32|3901.72|4098.60|5738.04|351.15|0.00|2466.98|3901.72|4252.87|6368.70|6719.85|-196.88| +2450826|76660|2450842|90408|268724|4640|13309|90408|268724|4640|13309|4|23|13|1|8545|264|581|24|58.76|163.94|144.26|472.32|3462.24|1410.24|3934.56|138.48|0.00|826.08|3462.24|3600.72|4288.32|4426.80|2052.00| +2450826|76660|2450867|90408|268724|4640|13309|90408|268724|4640|13309|4|37|12|4|12484|36|581|32|13.32|38.62|22.01|531.52|704.32|426.24|1235.84|14.08|0.00|383.04|704.32|718.40|1087.36|1101.44|278.08| +2450826|76660|2450863|90408|268724|4640|13309|90408|268724|4640|13309|4|78|17|2|12169|149|581|11|45.20|124.75|117.26|82.39|1289.86|497.20|1372.25|38.69|0.00|82.28|1289.86|1328.55|1372.14|1410.83|792.66| +2450826|76660|2450841|90408|268724|4640|13309|90408|268724|4640|13309|4|37|4|2|13900|12|581|63|60.42|165.55|157.27|521.64|9908.01|3806.46|10429.65|15.85|9115.36|3337.11|792.65|808.50|4129.76|4145.61|-3013.81| +2450826|76660|2450907|90408|268724|4640|13309|90408|268724|4640|13309|4|85|6|1|398|102|581|70|39.36|115.71|76.36|2754.50|5345.20|2755.20|8099.70|213.80|0.00|3644.20|5345.20|5559.00|8989.40|9203.20|2590.00| +2450826|76660|2450828|90408|268724|4640|13309|90408|268724|4640|13309|4|36|13|2|4303|277|581|65|77.75|101.85|12.22|5825.95|794.30|5053.75|6620.25|0.00|0.00|529.10|794.30|794.30|1323.40|1323.40|-4259.45| +2450826|21986|2450894|23900|1636273|4199|40572|23900|1636273|4199|40572|2|28|9|2|16804|88|582|61|63.68|183.39|110.03|4474.96|6711.83|3884.48|11186.79|0.00|0.00|3467.85|6711.83|6711.83|10179.68|10179.68|2827.35| +2450826|21986|2450859|23900|1636273|4199|40572|23900|1636273|4199|40572|2|46|16|2|14738|239|582|58|83.23|134.83|74.15|3519.44|4300.70|4827.34|7820.14|344.05|0.00|3675.46|4300.70|4644.75|7976.16|8320.21|-526.64| +2450826|21986|2450851|23900|1636273|4199|40572|23900|1636273|4199|40572|2|104|10|1|5509|54|582|52|56.52|76.86|69.94|359.84|3636.88|2939.04|3996.72|181.84|0.00|1318.72|3636.88|3818.72|4955.60|5137.44|697.84| +2450826|21986|2450896|23900|1636273|4199|40572|23900|1636273|4199|40572|2|36|5|2|5620|38|582|90|35.15|41.47|21.56|1791.90|1940.40|3163.50|3732.30|97.02|0.00|1417.50|1940.40|2037.42|3357.90|3454.92|-1223.10| +2450826|21986|2450902|23900|1636273|4199|40572|23900|1636273|4199|40572|2|77|13|4|1105|176|582|71|81.60|174.62|24.44|10662.78|1735.24|5793.60|12398.02|156.17|0.00|1239.66|1735.24|1891.41|2974.90|3131.07|-4058.36| +2450826|21986|2450850|23900|1636273|4199|40572|23900|1636273|4199|40572|2|96|19|3|5900|223|582|61|14.13|32.07|12.50|1193.77|762.50|861.93|1956.27|20.43|251.62|742.98|510.88|531.31|1253.86|1274.29|-351.05| +2450826|21986|2450861|23900|1636273|4199|40572|23900|1636273|4199|40572|2|43|1|3|9892|24|582|85|50.80|66.54|5.32|5203.70|452.20|4318.00|5655.90|4.52|0.00|848.30|452.20|456.72|1300.50|1305.02|-3865.80| +2450826|21986|2450888|23900|1636273|4199|40572|23900|1636273|4199|40572|2|93|20|1|13825|283|582|99|11.56|27.05|23.80|321.75|2356.20|1144.44|2677.95|94.24|0.00|1338.48|2356.20|2450.44|3694.68|3788.92|1211.76| +2450826|21986|2450914|23900|1636273|4199|40572|23900|1636273|4199|40572|2|87|17|2|7022|78|582|89|36.85|104.28|91.76|1114.28|8166.64|3279.65|9280.92|489.99|0.00|3804.75|8166.64|8656.63|11971.39|12461.38|4886.99| +2450826|21986|2450910|23900|1636273|4199|40572|23900|1636273|4199|40572|2|56|14|4|5053|27|582|97|32.60|34.23|15.74|1793.53|1526.78|3162.20|3320.31|41.68|137.41|1426.87|1389.37|1431.05|2816.24|2857.92|-1772.83| +2450826|21986|2450840|23900|1636273|4199|40572|23900|1636273|4199|40572|2|17|7|3|8659|155|582|42|17.58|41.84|22.59|808.50|948.78|738.36|1757.28|9.48|0.00|825.72|948.78|958.26|1774.50|1783.98|210.42| +2450826|21986|2450914|23900|1636273|4199|40572|23900|1636273|4199|40572|2|47|9|2|13141|55|582|82|58.77|101.67|72.18|2418.18|5918.76|4819.14|8336.94|59.18|0.00|1917.16|5918.76|5977.94|7835.92|7895.10|1099.62| +2450826|21986|2450888|23900|1636273|4199|40572|23900|1636273|4199|40572|2|69|16|3|6853|209|582|11|97.09|114.56|29.78|932.58|327.58|1067.99|1260.16|9.82|0.00|428.45|327.58|337.40|756.03|765.85|-740.41| +2450826|21986|2450869|23900|1636273|4199|40572|23900|1636273|4199|40572|2|53|2|1|4261|128|582|83|61.20|100.98|24.23|6370.25|2011.09|5079.60|8381.34|160.88|0.00|1256.62|2011.09|2171.97|3267.71|3428.59|-3068.51| +2450826|60465|2450874|87959|1722125|5891|38457|87959|1722125|5891|38457|2|65|17|5|10112|129|583|81|2.82|3.60|3.27|26.73|264.87|228.42|291.60|10.59|0.00|63.99|264.87|275.46|328.86|339.45|36.45| +2450826|60465|2450876|87959|1722125|5891|38457|87959|1722125|5891|38457|2|78|7|3|12848|288|583|41|45.02|83.73|56.09|1133.24|2299.69|1845.82|3432.93|22.99|0.00|514.55|2299.69|2322.68|2814.24|2837.23|453.87| +2450826|60465|2450845|87959|1722125|5891|38457|87959|1722125|5891|38457|2|66|17|4|17612|293|583|52|31.40|57.46|33.90|1225.12|1762.80|1632.80|2987.92|0.00|0.00|985.92|1762.80|1762.80|2748.72|2748.72|130.00| +2450826|60465|2450911|87959|1722125|5891|38457|87959|1722125|5891|38457|2|11|17|3|8545|60|583|21|1.53|2.83|0.16|56.07|3.36|32.13|59.43|0.23|0.00|6.51|3.36|3.59|9.87|10.10|-28.77| +2450826|60465|2450857|87959|1722125|5891|38457|87959|1722125|5891|38457|2|8|7|5|12484|35|583|42|61.94|170.95|87.18|3518.34|3661.56|2601.48|7179.90|219.69|0.00|1579.20|3661.56|3881.25|5240.76|5460.45|1060.08| +2450826|60465|2450830|87959|1722125|5891|38457|87959|1722125|5891|38457|2|76|8|2|12169|280|583|90|15.87|40.94|20.87|1806.30|1878.30|1428.30|3684.60|75.13|0.00|662.40|1878.30|1953.43|2540.70|2615.83|450.00| +2450826|60465|2450844|87959|1722125|5891|38457|87959|1722125|5891|38457|2|46|6|2|13900|279|583|81|25.32|46.58|17.23|2377.35|1395.63|2050.92|3772.98|41.86|0.00|867.51|1395.63|1437.49|2263.14|2305.00|-655.29| +2450826|10408|2450838|23554|694859|117|1422|97926|319185|5690|16148|1|5|1|5|1886|130|584|90|73.56|189.78|58.83|11785.50|5294.70|6620.40|17080.20|105.89|0.00|7515.00|5294.70|5400.59|12809.70|12915.59|-1325.70| +2450826|10408|2450893|23554|694859|117|1422|97926|319185|5690|16148|1|10|8|2|17108|130|584|80|62.11|177.01|164.61|992.00|13168.80|4968.80|14160.80|1053.50|0.00|2973.60|13168.80|14222.30|16142.40|17195.90|8200.00| +2450826|10408|2450879|23554|694859|117|1422|97926|319185|5690|16148|1|59|1|2|15715|194|584|35|12.85|21.71|6.29|539.70|220.15|449.75|759.85|4.40|0.00|273.35|220.15|224.55|493.50|497.90|-229.60| +2450826|10408|2450906|23554|694859|117|1422|97926|319185|5690|16148|1|60|15|2|7790|149|584|85|59.35|131.75|118.57|1120.30|10078.45|5044.75|11198.75|705.49|0.00|2015.35|10078.45|10783.94|12093.80|12799.29|5033.70| +2450826|10408|2450831|23554|694859|117|1422|97926|319185|5690|16148|1|92|14|3|16508|158|584|42|36.98|95.40|48.65|1963.50|2043.30|1553.16|4006.80|81.73|0.00|280.14|2043.30|2125.03|2323.44|2405.17|490.14| +2450826|10408|2450876|23554|694859|117|1422|97926|319185|5690|16148|1|40|15|2|14437|83|584|76|46.17|72.02|69.13|219.64|5253.88|3508.92|5473.52|245.88|2521.86|218.88|2732.02|2977.90|2950.90|3196.78|-776.90| +2450826|10408|2450883|23554|694859|117|1422|97926|319185|5690|16148|1|18|8|3|1558|141|584|33|71.62|118.88|27.34|3020.82|902.22|2363.46|3923.04|45.11|0.00|0.00|902.22|947.33|902.22|947.33|-1461.24| +2450826|10408|2450899|23554|694859|117|1422|97926|319185|5690|16148|1|4|7|2|17449|95|584|85|28.49|66.09|0.66|5561.55|56.10|2421.65|5617.65|0.21|34.22|2190.45|21.88|22.09|2212.33|2212.54|-2399.77| +2450826|10408|2450864|23554|694859|117|1422|97926|319185|5690|16148|1|89|15|1|11722|284|584|87|24.61|26.82|12.60|1237.14|1096.20|2141.07|2333.34|43.84|0.00|839.55|1096.20|1140.04|1935.75|1979.59|-1044.87| +2450826|32137|2450873|10077|733640|4403|43166|10077|733640|4403|43166|4|89|15|5|3820|120|585|4|25.97|32.72|9.48|92.96|37.92|103.88|130.88|2.27|0.00|34.00|37.92|40.19|71.92|74.19|-65.96| +2450826|32137|2450856|10077|733640|4403|43166|10077|733640|4403|43166|4|80|12|2|9211|117|585|75|73.44|195.35|13.67|13626.00|1025.25|5508.00|14651.25|1.33|891.96|7325.25|133.29|134.62|7458.54|7459.87|-5374.71| +2450826|32137|2450867|10077|733640|4403|43166|10077|733640|4403|43166|4|57|2|2|17918|257|585|55|37.14|90.99|17.28|4054.05|950.40|2042.70|5004.45|73.75|28.51|449.90|921.89|995.64|1371.79|1445.54|-1120.81| +2450826|32137|2450904|10077|733640|4403|43166|10077|733640|4403|43166|4|19|15|2|11803|23|585|65|41.84|48.95|25.94|1495.65|1686.10|2719.60|3181.75|84.30|0.00|540.80|1686.10|1770.40|2226.90|2311.20|-1033.50| +2450826|32137|2450833|10077|733640|4403|43166|10077|733640|4403|43166|4|72|5|1|6244|200|585|25|20.48|34.61|19.38|380.75|484.50|512.00|865.25|29.07|0.00|69.00|484.50|513.57|553.50|582.57|-27.50| +2450826|32137|2450875|10077|733640|4403|43166|10077|733640|4403|43166|4|97|17|3|7924|130|585|33|30.25|39.93|1.19|1278.42|39.27|998.25|1317.69|0.39|0.00|368.94|39.27|39.66|408.21|408.60|-958.98| +2450826|32137|2450869|10077|733640|4403|43166|10077|733640|4403|43166|4|100|7|2|16486|118|585|58|55.08|159.73|99.03|3520.60|5743.74|3194.64|9264.34|114.87|0.00|277.82|5743.74|5858.61|6021.56|6136.43|2549.10| +2450826|32137|2450914|10077|733640|4403|43166|10077|733640|4403|43166|4|6|1|1|17480|254|585|100|1.25|3.00|2.31|69.00|231.00|125.00|300.00|4.62|0.00|48.00|231.00|235.62|279.00|283.62|106.00| +2450826|32137|2450889|10077|733640|4403|43166|10077|733640|4403|43166|4|53|4|3|2876|70|585|48|60.99|134.17|5.36|6182.88|257.28|2927.52|6440.16|12.86|0.00|2704.80|257.28|270.14|2962.08|2974.94|-2670.24| +2450826|54430|2450872|58659|1436296|333|4770|58659|1436296|333|4770|1|39|17|2|5935|70|586|85|4.30|9.67|3.28|543.15|278.80|365.50|821.95|5.57|0.00|139.40|278.80|284.37|418.20|423.77|-86.70| +2450826|54430|2450853|58659|1436296|333|4770|58659|1436296|333|4770|1|101|15|3|11386|280|586|82|42.40|104.30|37.54|5474.32|3078.28|3476.80|8552.60|246.26|0.00|1282.48|3078.28|3324.54|4360.76|4607.02|-398.52| +2450826|54430|2450861|58659|1436296|333|4770|58659|1436296|333|4770|1|107|7|4|2582|166|586|55|95.19|128.50|15.42|6219.40|848.10|5235.45|7067.50|16.96|0.00|2827.00|848.10|865.06|3675.10|3692.06|-4387.35| +2450826|54430|2450846|58659|1436296|333|4770|58659|1436296|333|4770|1|84|5|1|15550|163|586|74|31.58|66.63|45.97|1528.84|3401.78|2336.92|4930.62|68.03|0.00|936.10|3401.78|3469.81|4337.88|4405.91|1064.86| +2450826|54430|2450872|58659|1436296|333|4770|58659|1436296|333|4770|1|42|6|1|6104|103|586|3|69.26|76.87|74.56|6.93|223.68|207.78|230.61|15.65|0.00|20.73|223.68|239.33|244.41|260.06|15.90| +2450826|54430|2450857|58659|1436296|333|4770|58659|1436296|333|4770|1|64|13|5|10402|106|586|97|40.91|61.77|43.85|1738.24|4253.45|3968.27|5991.69|145.89|2169.25|1557.82|2084.20|2230.09|3642.02|3787.91|-1884.07| +2450826|54430|2450905|58659|1436296|333|4770|58659|1436296|333|4770|1|63|11|3|10594|112|586|59|61.27|181.97|76.42|6227.45|4508.78|3614.93|10736.23|180.35|0.00|643.69|4508.78|4689.13|5152.47|5332.82|893.85| +2450826|54430|2450837|58659|1436296|333|4770|58659|1436296|333|4770|1|17|10|3|11786|206|586|14|96.71|286.26|20.03|3727.22|280.42|1353.94|4007.64|0.67|257.98|1603.00|22.44|23.11|1625.44|1626.11|-1331.50| +2450826|54430|2450862|58659|1436296|333|4770|58659|1436296|333|4770|1|66|3|3|802|40|586|64|35.59|57.29|20.05|2383.36|1283.20|2277.76|3666.56|89.82|0.00|1393.28|1283.20|1373.02|2676.48|2766.30|-994.56| +2450826|54430|2450883|58659|1436296|333|4770|58659|1436296|333|4770|1|7|7|2|364|292|586|10|85.34|138.25|56.68|815.70|566.80|853.40|1382.50|0.00|0.00|414.70|566.80|566.80|981.50|981.50|-286.60| +2450826|54430|2450829|58659|1436296|333|4770|58659|1436296|333|4770|1|23|13|1|14930|127|586|21|18.77|20.27|10.33|208.74|216.93|394.17|425.67|8.67|0.00|187.11|216.93|225.60|404.04|412.71|-177.24| +2450826|43187|2450898|36969|1843719|2469|22772|36969|1843719|2469|22772|4|31|16|1|8407|65|587|18|55.71|161.00|106.26|985.32|1912.68|1002.78|2898.00|95.63|0.00|1246.14|1912.68|2008.31|3158.82|3254.45|909.90| +2450826|43187|2450850|36969|1843719|2469|22772|36969|1843719|2469|22772|4|73|3|1|12074|175|587|15|22.68|47.40|14.22|497.70|213.30|340.20|711.00|4.26|0.00|206.10|213.30|217.56|419.40|423.66|-126.90| +2450826|43187|2450844|36969|1843719|2469|22772|36969|1843719|2469|22772|4|74|16|5|12880|208|587|66|83.25|129.03|41.28|5791.50|2724.48|5494.50|8515.98|190.71|0.00|425.70|2724.48|2915.19|3150.18|3340.89|-2770.02| +2450826|43187|2450874|36969|1843719|2469|22772|36969|1843719|2469|22772|4|66|10|1|6070|65|587|22|61.26|142.73|141.30|31.46|3108.60|1347.72|3140.06|93.25|0.00|941.82|3108.60|3201.85|4050.42|4143.67|1760.88| +2450826|43187|2450864|36969|1843719|2469|22772|36969|1843719|2469|22772|4|47|6|2|10816|124|587|30|16.42|40.22|10.85|881.10|325.50|492.60|1206.60|26.04|0.00|253.20|325.50|351.54|578.70|604.74|-167.10| +2450826|43187|2450899|36969|1843719|2469|22772|36969|1843719|2469|22772|4|40|12|4|1162|286|587|18|91.90|230.66|122.24|1951.56|2200.32|1654.20|4151.88|66.00|0.00|1992.78|2200.32|2266.32|4193.10|4259.10|546.12| +2450826|43187|2450864|36969|1843719|2469|22772|36969|1843719|2469|22772|4|36|13|1|13588|222|587|74|8.77|18.41|17.85|41.44|1320.90|648.98|1362.34|26.41|0.00|353.72|1320.90|1347.31|1674.62|1701.03|671.92| +2450826|43187|2450878|36969|1843719|2469|22772|36969|1843719|2469|22772|4|54|15|2|16744|291|587|1|26.67|75.74|71.19|4.55|71.19|26.67|75.74|1.88|33.45|23.47|37.74|39.62|61.21|63.09|11.07| +2450826|43187|2450841|36969|1843719|2469|22772|36969|1843719|2469|22772|4|69|20|1|9574|297|587|91|90.12|208.17|151.96|5115.11|13828.36|8200.92|18943.47|414.85|0.00|3219.58|13828.36|14243.21|17047.94|17462.79|5627.44| +2450826||2450914|36969|1843719|2469|22772|||2469|22772|4|59|||4795||587|||||404.32||877.04||||126.16|2121.16|2206.00|2247.32|2332.16|1244.12| +2450826|43187|2450895|36969|1843719|2469|22772|36969|1843719|2469|22772|4|101|9|5|13606|85|587|66|58.06|98.12|77.51|1360.26|5115.66|3831.96|6475.92|153.46|0.00|64.68|5115.66|5269.12|5180.34|5333.80|1283.70| +2450826|43187|2450876|36969|1843719|2469|22772|36969|1843719|2469|22772|4|64|19|4|10561|242|587|89|25.81|43.10|13.79|2608.59|1227.31|2297.09|3835.90|73.63|0.00|1687.44|1227.31|1300.94|2914.75|2988.38|-1069.78| +2450826|43187|2450863|36969|1843719|2469|22772|36969|1843719|2469|22772|4|90|4|1|11672|208|587|74|13.84|26.98|7.55|1437.82|558.70|1024.16|1996.52|37.93|16.76|378.88|541.94|579.87|920.82|958.75|-482.22| +2450826|72024|2450831|59348|814917|2486|23255|59348|814917|2486|23255|1|36|19|1|14317|125|588|67|94.54|204.20|83.72|8072.16|5609.24|6334.18|13681.40|392.64|0.00|6293.31|5609.24|6001.88|11902.55|12295.19|-724.94| +2450826|72024|2450887|59348|814917|2486|23255|59348|814917|2486|23255|1|76|13|3|10492|256|588|81|72.34|173.61|118.05|4500.36|9562.05|5859.54|14062.41|764.96|0.00|5624.64|9562.05|10327.01|15186.69|15951.65|3702.51| +2450826|72024|2450888|59348|814917|2486|23255|59348|814917|2486|23255|1|24|18|2|727|202|588|68|34.22|94.78|85.30|644.64|5800.40|2326.96|6445.04|406.02|0.00|128.52|5800.40|6206.42|5928.92|6334.94|3473.44| +2450826|72024|2450840|59348|814917|2486|23255|59348|814917|2486|23255|1|92|2|5|14860|144|588|10|65.15|177.20|54.93|1222.70|549.30|651.50|1772.00|16.47|0.00|177.20|549.30|565.77|726.50|742.97|-102.20| +2450826|72024|2450868|59348|814917|2486|23255|59348|814917|2486|23255|1|22|9|3|10939|71|588|90|83.28|174.88|94.43|7240.50|8498.70|7495.20|15739.20|339.94|0.00|3462.30|8498.70|8838.64|11961.00|12300.94|1003.50| +2450826|72024|2450878|59348|814917|2486|23255|59348|814917|2486|23255|1|25|3|2|5722|35|588|5|25.37|53.53|6.42|235.55|32.10|126.85|267.65|1.28|0.00|80.25|32.10|33.38|112.35|113.63|-94.75| +2450826|72024|2450839|59348|814917|2486|23255|59348|814917|2486|23255|1|94|20|2|12340|29|588|34|69.43|190.23|41.85|5044.92|1422.90|2360.62|6467.82|113.83|0.00|2392.92|1422.90|1536.73|3815.82|3929.65|-937.72| +2450826|72024|2450832|59348|814917|2486|23255|59348|814917|2486|23255|1|93|12|1|17204|264|588|53|49.67|88.41|68.95|1031.38|3654.35|2632.51|4685.73|109.63|0.00|140.45|3654.35|3763.98|3794.80|3904.43|1021.84| +2450826|72024|2450865|59348|814917|2486|23255|59348|814917|2486|23255|1|23|12|4|10114|161|588|58|10.15|14.10|3.38|621.76|196.04|588.70|817.80|1.13|82.33|130.50|113.71|114.84|244.21|245.34|-474.99| +2450826|72024|2450879|59348|814917|2486|23255|59348|814917|2486|23255|1|6|6|4|1910|30|588|98|66.59|181.12|1.81|17572.38|177.38|6525.82|17749.76|1.77|0.00|709.52|177.38|179.15|886.90|888.67|-6348.44| +2450826|72024|2450900|59348|814917|2486|23255|59348|814917|2486|23255|1|32|12|3|5414|8|588|84|26.95|39.61|19.40|1697.64|1629.60|2263.80|3327.24|81.48|0.00|864.36|1629.60|1711.08|2493.96|2575.44|-634.20| +2450826|72024|2450882|59348|814917|2486|23255|59348|814917|2486|23255|1|14|14|3|15271|136|588|27|13.47|15.75|10.86|132.03|293.22|363.69|425.25|17.59|0.00|106.11|293.22|310.81|399.33|416.92|-70.47| +2450826|72024|2450886|59348|814917|2486|23255|59348|814917|2486|23255|1|17|2|1|14918|71|588|19|48.61|52.98|48.21|90.63|915.99|923.59|1006.62|82.43|0.00|463.03|915.99|998.42|1379.02|1461.45|-7.60| +2450826|72024|2450892|59348|814917|2486|23255|59348|814917|2486|23255|1|38|3|1|15440|32|588|88|75.32|159.67|130.92|2530.00|11520.96|6628.16|14050.96|576.04|0.00|6462.72|11520.96|12097.00|17983.68|18559.72|4892.80| +2450826|60930|2450842|29974|1521718|5462|38359|29974|1521718|5462|38359|4|72|16|3|5713|160|589|39|85.27|172.24|72.34|3896.10|2821.26|3325.53|6717.36|225.70|0.00|3291.21|2821.26|3046.96|6112.47|6338.17|-504.27| +2450826|60930|2450857|29974|1521718|5462|38359|29974|1521718|5462|38359|4|98|6|1|11689|124|589|63|42.54|125.06|103.79|1340.01|6538.77|2680.02|7878.78|0.00|0.00|3860.01|6538.77|6538.77|10398.78|10398.78|3858.75| +2450826|60930|2450859|29974|1521718|5462|38359|29974|1521718|5462|38359|4|32|3|4|11728|31|589|96|4.40|8.84|0.35|815.04|33.60|422.40|848.64|2.35|0.00|160.32|33.60|35.95|193.92|196.27|-388.80| +2450826|60930|2450881|29974|1521718|5462|38359|29974|1521718|5462|38359|4|94|5|1|5726|120|589|97|18.63|24.96|8.23|1622.81|798.31|1807.11|2421.12|63.86|0.00|749.81|798.31|862.17|1548.12|1611.98|-1008.80| +2450826|60930|2450833|29974|1521718|5462|38359|29974|1521718|5462|38359|4|76|2|2|14216|149|589|17|84.39|222.78|142.57|1363.57|2423.69|1434.63|3787.26|48.47|0.00|643.79|2423.69|2472.16|3067.48|3115.95|989.06| +2450826|60930|2450845|29974|1521718|5462|38359|29974|1521718|5462|38359|4|31|6|1|9016|171|589|48|46.74|86.00|61.92|1155.84|2972.16|2243.52|4128.00|0.00|0.00|1692.48|2972.16|2972.16|4664.64|4664.64|728.64| +2450826|60930|2450887|29974|1521718|5462|38359|29974|1521718|5462|38359|4|33|7|5|15068|129|589|19|21.68|59.40|5.34|1027.14|101.46|411.92|1128.60|0.00|0.00|11.21|101.46|101.46|112.67|112.67|-310.46| +2450826|60930|2450908|29974|1521718|5462|38359|29974|1521718|5462|38359|4|30|6|4|13345|76|589|93|97.25|165.32|59.51|9840.33|5534.43|9044.25|15374.76|110.68|0.00|5687.88|5534.43|5645.11|11222.31|11332.99|-3509.82| +2450826|60930|2450895|29974|1521718|5462|38359|29974|1521718|5462|38359|4|6|2|4|15247|143|589|41|59.77|179.31|17.93|6616.58|735.13|2450.57|7351.71|40.13|161.72|3161.10|573.41|613.54|3734.51|3774.64|-1877.16| +2450826|60930|2450877|29974|1521718|5462|38359|29974|1521718|5462|38359|4|12|15|5|11215|27|589|71|36.92|66.82|42.09|1755.83|2988.39|2621.32|4744.22|0.00|0.00|1517.98|2988.39|2988.39|4506.37|4506.37|367.07| +2450826|60930|2450892|29974|1521718|5462|38359|29974|1521718|5462|38359|4|58|14|2|8350|49|589|92|90.33|203.24|34.55|15519.48|3178.60|8310.36|18698.08|95.35|0.00|9349.04|3178.60|3273.95|12527.64|12622.99|-5131.76| +2450826|60930|2450848|29974|1521718|5462|38359|29974|1521718|5462|38359|4|55|1|3|11413|192|589|57|75.09|167.45|63.63|5917.74|3626.91|4280.13|9544.65|145.07|0.00|1049.37|3626.91|3771.98|4676.28|4821.35|-653.22| +2450826|60930|2450868|29974|1521718|5462|38359|29974|1521718|5462|38359|4|80|9|3|17224|228|589|55|5.85|10.88|9.79|59.95|538.45|321.75|598.40|1.93|489.98|197.45|48.47|50.40|245.92|247.85|-273.28| +2450826|44401|2450835|74062|1199853|6091|25529|74062|1199853|6091|25529|4|36|19|3|16786|36|590|81|8.67|13.09|8.24|392.85|667.44|702.27|1060.29|2.66|400.46|508.68|266.98|269.64|775.66|778.32|-435.29| +2450826|44401|2450864|74062|1199853|6091|25529|74062|1199853|6091|25529|4|40|9|2|1150|268|590|69|49.03|111.78|57.00|3779.82|3933.00|3383.07|7712.82|78.66|0.00|2544.72|3933.00|4011.66|6477.72|6556.38|549.93| +2450826|44401|2450889|74062|1199853|6091|25529|74062|1199853|6091|25529|4|55|15|1|11150|195|590|99|64.33|66.90|33.45|3311.55|3311.55|6368.67|6623.10|198.69|0.00|3112.56|3311.55|3510.24|6424.11|6622.80|-3057.12| +2450826|44401|2450892|74062|1199853|6091|25529|74062|1199853|6091|25529|4|36|13|3|17083|267|590|6|11.73|16.65|10.32|37.98|61.92|70.38|99.90|0.61|0.00|24.96|61.92|62.53|86.88|87.49|-8.46| +2450826|44401|2450884|74062|1199853|6091|25529|74062|1199853|6091|25529|4|100|18|1|10846|297|590|97|36.72|80.78|32.31|4701.59|3134.07|3561.84|7835.66|219.38|0.00|1174.67|3134.07|3353.45|4308.74|4528.12|-427.77| +2450826|44401|2450851|74062|1199853|6091|25529|74062|1199853|6091|25529|4|47|8|3|8665|283|590|73|85.61|179.78|111.46|4987.36|8136.58|6249.53|13123.94|162.73|0.00|4461.76|8136.58|8299.31|12598.34|12761.07|1887.05| +2450826|44401|2450899|74062|1199853|6091|25529|74062|1199853|6091|25529|4|68|15|1|2414|243|590|81|13.00|13.00|2.73|831.87|221.13|1053.00|1053.00|8.84|0.00|421.20|221.13|229.97|642.33|651.17|-831.87| +2450826|44401|2450902|74062|1199853|6091|25529|74062|1199853|6091|25529|4|89|13|3|5330|143|590|58|62.12|172.69|146.78|1502.78|8513.24|3602.96|10016.02|255.39|0.00|3705.62|8513.24|8768.63|12218.86|12474.25|4910.28| +2450826|44401|2450856|74062|1199853|6091|25529|74062|1199853|6091|25529|4|19|2|3|4298|240|590|65|53.52|132.19|0.00|8592.35|0.00|3478.80|8592.35|0.00|0.00|4037.80|0.00|0.00|4037.80|4037.80|-3478.80| +2450826|44401|2450908|74062|1199853|6091|25529|74062|1199853|6091|25529|4|28|2|3|296|83|590|83|9.75|22.62|13.79|732.89|1144.57|809.25|1877.46|23.34|560.83|112.05|583.74|607.08|695.79|719.13|-225.51| +2450826|44401|2450893|74062|1199853|6091|25529|74062|1199853|6091|25529|4|16|18|1|11647|84|590|90|77.47|167.33|167.33|0.00|15059.70|6972.30|15059.70|516.54|7680.44|6776.10|7379.26|7895.80|14155.36|14671.90|406.96| +2450826|44401|2450857|74062|1199853|6091|25529|74062|1199853|6091|25529|4|58|14|4|6019|141|590|75|92.17|262.68|102.44|12018.00|7683.00|6912.75|19701.00|0.00|4379.31|4925.25|3303.69|3303.69|8228.94|8228.94|-3609.06| +2450826|57130|2450842|16687|996663|3039|27015|16687|996663|3039|27015|1|3|3|2|14834|80|591|65|7.87|13.06|12.27|51.35|797.55|511.55|848.90|47.85|0.00|84.50|797.55|845.40|882.05|929.90|286.00| +2450826|57130|2450875|16687|996663|3039|27015|16687|996663|3039|27015|1|43|1|2|12422|270|591|36|18.28|53.74|49.44|154.80|1779.84|658.08|1934.64|36.30|872.12|212.76|907.72|944.02|1120.48|1156.78|249.64| +2450826|57130|2450882|16687|996663|3039|27015|16687|996663|3039|27015|1|14|7|5|17894|197|591|65|48.51|105.26|61.05|2873.65|3968.25|3153.15|6841.90|317.46|0.00|2189.20|3968.25|4285.71|6157.45|6474.91|815.10| +2450826|57130|2450886|16687|996663|3039|27015|16687|996663|3039|27015|1|66|6|3|10538|272|591|28|86.41|226.39|151.68|2091.88|4247.04|2419.48|6338.92|297.29|0.00|126.56|4247.04|4544.33|4373.60|4670.89|1827.56| +2450826|57130|2450886|16687|996663|3039|27015|16687|996663|3039|27015|1|58|13|1|1801|290|591|41|57.06|76.46|9.17|2758.89|375.97|2339.46|3134.86|33.83|0.00|564.16|375.97|409.80|940.13|973.96|-1963.49| +2450826|57130|2450895|16687|996663|3039|27015|16687|996663|3039|27015|1|64|15|1|268|8|591|96|75.11|98.39|79.69|1795.20|7650.24|7210.56|9445.44|612.01|0.00|2832.96|7650.24|8262.25|10483.20|11095.21|439.68| +2450826|57130|2450843|16687|996663|3039|27015|16687|996663|3039|27015|1|87|3|2|13519|115|591|25|39.46|71.02|51.13|497.25|1278.25|986.50|1775.50|89.47|0.00|35.50|1278.25|1367.72|1313.75|1403.22|291.75| +2450826|57130|2450870|16687|996663|3039|27015|16687|996663|3039|27015|1|83|5|1|2474|11|591|11|13.12|21.38|1.71|216.37|18.81|144.32|235.18|1.50|0.00|2.31|18.81|20.31|21.12|22.62|-125.51| +2450826|57130|2450828|16687|996663|3039|27015|16687|996663|3039|27015|1|6|19|5|11521|217|591|20|3.25|5.72|3.88|36.80|77.60|65.00|114.40|2.32|0.00|49.00|77.60|79.92|126.60|128.92|12.60| +2450826|57130|2450856|16687|996663|3039|27015|16687|996663|3039|27015|1|73|17|3|4232|275|591|83|30.92|60.60|16.36|3671.92|1357.88|2566.36|5029.80|40.73|0.00|150.23|1357.88|1398.61|1508.11|1548.84|-1208.48| +2450826|57130|2450853|16687|996663|3039|27015|16687|996663|3039|27015|1|75|3|1|15794|152|591|40|6.79|17.99|3.95|561.60|158.00|271.60|719.60|5.11|30.02|107.60|127.98|133.09|235.58|240.69|-143.62| +2450826|57130|2450844|16687|996663|3039|27015|16687|996663|3039|27015|1|6|1|4|4438|30|591|52|5.71|8.79|3.77|261.04|196.04|296.92|457.08|0.00|0.00|191.88|196.04|196.04|387.92|387.92|-100.88| +2450826|72493|2450908|51032|1782756|5680|11404|60754|1721205|3329|8253|2|26|18|2|2038|257|592|55|84.63|175.18|80.58|5203.00|4431.90|4654.65|9634.90|265.91|0.00|1252.35|4431.90|4697.81|5684.25|5950.16|-222.75| +2450826|72493|2450854|51032|1782756|5680|11404|60754|1721205|3329|8253|2|101|3|3|9961|154|592|59|3.73|10.70|8.02|158.12|473.18|220.07|631.30|18.92|0.00|258.42|473.18|492.10|731.60|750.52|253.11| +2450826|72493|2450846|51032|1782756|5680|11404|60754|1721205|3329|8253|2|35|3|5|15572|14|592|20|46.73|47.19|36.80|207.80|736.00|934.60|943.80|36.80|0.00|443.40|736.00|772.80|1179.40|1216.20|-198.60| +2450826|72493|2450828|51032|1782756|5680|11404|60754|1721205|3329|8253|2|33|1|5|13844|136|592|13|93.21|272.17|168.74|1344.59|2193.62|1211.73|3538.21|153.55|0.00|919.88|2193.62|2347.17|3113.50|3267.05|981.89| +2450826|72493|2450883|51032|1782756|5680|11404|60754|1721205|3329|8253|2|87|7|5|3184|196|592|51|94.66|188.37|41.44|7493.43|2113.44|4827.66|9606.87|63.40|0.00|4322.76|2113.44|2176.84|6436.20|6499.60|-2714.22| +2450826|72493|2450837|51032|1782756|5680|11404|60754|1721205|3329|8253|2|16|10|5|6800|142|592|13|77.54|228.74|118.94|1427.40|1546.22|1008.02|2973.62|77.31|0.00|1278.55|1546.22|1623.53|2824.77|2902.08|538.20| +2450826|72493|2450913|51032|1782756|5680|11404|60754|1721205|3329|8253|2|71|7|3|12878|277|592|12|56.19|103.38|64.09|471.48|769.08|674.28|1240.56|46.14|0.00|334.92|769.08|815.22|1104.00|1150.14|94.80| +2450826|72493|2450875|51032|1782756|5680|11404|60754|1721205|3329|8253|2|4|5|3|11968|104|592|51|46.64|94.21|61.23|1681.98|3122.73|2378.64|4804.71|218.59|0.00|2066.01|3122.73|3341.32|5188.74|5407.33|744.09| +2450826|72493|2450870|51032|1782756|5680|11404|60754|1721205|3329|8253|2|4|16|5|13192|85|592|41|90.82|103.53|15.52|3608.41|636.32|3723.62|4244.73|19.08|0.00|1655.17|636.32|655.40|2291.49|2310.57|-3087.30| +2450826|72493|2450896|51032|1782756|5680|11404|60754|1721205|3329|8253|2|7|4|2|11012|45|592|16|40.96|122.47|2.44|1920.48|39.04|655.36|1959.52|3.51|0.00|627.04|39.04|42.55|666.08|669.59|-616.32| +2450826|72493|2450906|51032|1782756|5680|11404|60754|1721205|3329|8253|2|89|7|2|6572|294|592|30|86.77|129.28|15.51|3413.10|465.30|2603.10|3878.40|9.30|0.00|1745.10|465.30|474.60|2210.40|2219.70|-2137.80| +2450826|72493|2450849|51032|1782756|5680|11404|60754|1721205|3329|8253|2|65|15|5|9691|168|592|89|46.63|83.93|55.39|2540.06|4929.71|4150.07|7469.77|212.96|2563.44|1717.70|2366.27|2579.23|4083.97|4296.93|-1783.80| +2450826|72493|2450873|51032|1782756|5680|11404|60754|1721205|3329|8253|2|30|2|5|8401|226|592|10|88.96|158.34|128.25|300.90|1282.50|889.60|1583.40|51.30|0.00|554.10|1282.50|1333.80|1836.60|1887.90|392.90| +2450826|16532|2450894|5505|42945|5985|39778|5505|42945|5985|39778|1|71|13|2|2108|185|593|93|88.54|262.07|65.51|18280.08|6092.43|8234.22|24372.51|115.14|4447.47|11454.81|1644.96|1760.10|13099.77|13214.91|-6589.26| +2450826|16532|2450871|5505|42945|5985|39778|5505|42945|5985|39778|1|65|7|5|8320|190|593|55|49.64|130.05|117.04|715.55|6437.20|2730.20|7152.75|321.86|0.00|2217.05|6437.20|6759.06|8654.25|8976.11|3707.00| +2450826|16532|2450858|5505|42945|5985|39778|5505|42945|5985|39778|1|75|17|3|14608|208|593|71|53.13|89.25|11.60|5513.15|823.60|3772.23|6336.75|71.90|24.70|2154.14|798.90|870.80|2953.04|3024.94|-2973.33| +2450826|16532|2450869|5505|42945|5985|39778|5505|42945|5985|39778|1|93|17|1|5042|89|593|39|17.33|50.95|22.92|1093.17|893.88|675.87|1987.05|53.63|0.00|218.40|893.88|947.51|1112.28|1165.91|218.01| +2450826|19864|2450868|51070|1733058|1893|12681|51070|1733058|1893|12681|2|12|10|1|10376|213|594|89|71.23|131.77|3.95|11375.98|351.55|6339.47|11727.53|31.63|0.00|1993.60|351.55|383.18|2345.15|2376.78|-5987.92| +2450826|19864|2450896|51070|1733058|1893|12681|51070|1733058|1893|12681|2|17|20|4|15518|87|594|21|72.55|146.55|46.89|2092.86|984.69|1523.55|3077.55|78.77|0.00|461.58|984.69|1063.46|1446.27|1525.04|-538.86| +2450826|19864|2450829|51070|1733058|1893|12681|51070|1733058|1893|12681|2|5|6|4|8737|183|594|54|69.66|94.73|56.83|2046.60|3068.82|3761.64|5115.42|0.00|1227.52|664.74|1841.30|1841.30|2506.04|2506.04|-1920.34| +2450826|19864|2450890|51070|1733058|1893|12681|51070|1733058|1893|12681|2|31|19|5|7357|226|594|66|43.81|54.32|0.54|3549.48|35.64|2891.46|3585.12|0.21|24.94|143.22|10.70|10.91|153.92|154.13|-2880.76| +2450826|70852|2450871|64461|1628807|4353|19852|64461|1628807|4353|19852|4|19|1|4|9835|57|595|15|59.44|130.76|99.37|470.85|1490.55|891.60|1961.40|134.14|0.00|470.70|1490.55|1624.69|1961.25|2095.39|598.95| +2450826|70852|2450907|64461|1628807|4353|19852|64461|1628807|4353|19852|4|80|2|1|16436|224|595|89|87.03|246.29|150.23|8549.34|13370.47|7745.67|21919.81|802.22|0.00|6575.32|13370.47|14172.69|19945.79|20748.01|5624.80| +2450826|70852|2450828|64461|1628807|4353|19852|64461|1628807|4353|19852|4|43|8|1|14941|154|595|68|76.73|174.17|94.05|5448.16|6395.40|5217.64|11843.56|511.63|0.00|2605.08|6395.40|6907.03|9000.48|9512.11|1177.76| +2450826|70852|2450828|64461|1628807|4353|19852|64461|1628807|4353|19852|4|94|10|2|10844|118|595|5|50.26|85.44|67.49|89.75|337.45|251.30|427.20|30.37|0.00|119.60|337.45|367.82|457.05|487.42|86.15| +2450826|70852|2450851|64461|1628807|4353|19852|64461|1628807|4353|19852|4|63|19|4|11083|256|595|97|43.79|129.18|129.18|0.00|12530.46|4247.63|12530.46|25.06|10024.36|2130.12|2506.10|2531.16|4636.22|4661.28|-1741.53| +2450826|70852|2450902|64461|1628807|4353|19852|64461|1628807|4353|19852|4|31|10|5|6193|143|595|66|45.63|111.33|0.00|7347.78|0.00|3011.58|7347.78|0.00|0.00|2498.10|0.00|0.00|2498.10|2498.10|-3011.58| +2450826|70852|2450840|64461|1628807|4353|19852|64461|1628807|4353|19852|4|97|4|2|6163|181|595|74|75.54|179.78|156.40|1730.12|11573.60|5589.96|13303.72|0.00|0.00|1064.12|11573.60|11573.60|12637.72|12637.72|5983.64| +2450826|70852|2450878|64461|1628807|4353|19852|64461|1628807|4353|19852|4|69|15|5|2090|204|595|6|48.87|121.68|43.80|467.28|262.80|293.22|730.08|7.88|0.00|277.38|262.80|270.68|540.18|548.06|-30.42| +2450826|70852|2450843|64461|1628807|4353|19852|64461|1628807|4353|19852|4|67|13|3|10393|13|595|40|4.56|10.16|9.24|36.80|369.60|182.40|406.40|15.63|195.88|89.20|173.72|189.35|262.92|278.55|-8.68| +2450826|70852|2450913|64461|1628807|4353|19852|64461|1628807|4353|19852|4|24|8|5|10712|2|595|85|93.55|280.65|72.96|17653.65|6201.60|7951.75|23855.25|62.01|0.00|3577.65|6201.60|6263.61|9779.25|9841.26|-1750.15| +2450826|70852|2450891|64461|1628807|4353|19852|64461|1628807|4353|19852|4|100|1|4|10504|253|595|44|85.92|250.02|220.01|1320.44|9680.44|3780.48|11000.88|484.02|0.00|4290.00|9680.44|10164.46|13970.44|14454.46|5899.96| +2450826|70852|2450861|64461|1628807|4353|19852|64461|1628807|4353|19852|4|59|4|1|10651|103|595|63|27.65|29.03|23.22|366.03|1462.86|1741.95|1828.89|48.12|775.31|164.43|687.55|735.67|851.98|900.10|-1054.40| +2450826|26413|2450858|85806|501933|2860|27387|85806|501933|2860|27387|4|87|8|3|16108|95|596|28|77.89|171.35|61.68|3070.76|1727.04|2180.92|4797.80|19.86|1329.82|1535.24|397.22|417.08|1932.46|1952.32|-1783.70| +2450826|26413|2450914|85806|501933|2860|27387|85806|501933|2860|27387|4|25|20|4|14881|242|596|31|35.09|88.42|57.47|959.45|1781.57|1087.79|2741.02|53.44|0.00|191.58|1781.57|1835.01|1973.15|2026.59|693.78| +2450826|26413|2450850|85806|501933|2860|27387|85806|501933|2860|27387|4|104|12|1|14983|24|596|48|86.91|203.36|134.21|3319.20|6442.08|4171.68|9761.28|0.00|0.00|2830.56|6442.08|6442.08|9272.64|9272.64|2270.40| +2450826|26413|2450872|85806|501933|2860|27387|85806|501933|2860|27387|4|34|12|4|1513|165|596|87|14.48|24.18|14.74|821.28|1282.38|1259.76|2103.66|10.38|243.65|630.75|1038.73|1049.11|1669.48|1679.86|-221.03| +2450826|33945|2450837|8048|1614451|2454|38010|8048|1614451|2454|38010|1|20|14|1|11834|100|597|51|31.07|34.17|4.10|1533.57|209.10|1584.57|1742.67|6.27|0.00|696.66|209.10|215.37|905.76|912.03|-1375.47| +2450826|33945|2450885|8048|1614451|2454|38010|8048|1614451|2454|38010|1|50|1|1|14768|93|597|55|22.91|42.61|23.86|1031.25|1312.30|1260.05|2343.55|65.61|0.00|773.30|1312.30|1377.91|2085.60|2151.21|52.25| +2450826|33945|2450871|8048|1614451|2454|38010|8048|1614451|2454|38010|1|50|16|3|12166|247|597|75|19.35|46.24|12.94|2497.50|970.50|1451.25|3468.00|52.79|310.56|970.50|659.94|712.73|1630.44|1683.23|-791.31| +2450826|33945|2450914|8048|1614451|2454|38010|8048|1614451|2454|38010|1|70|8|3|11056|190|597|14|51.80|61.64|54.85|95.06|767.90|725.20|862.96|53.75|0.00|60.34|767.90|821.65|828.24|881.99|42.70| +2450826|33945|2450859|8048|1614451|2454|38010|8048|1614451|2454|38010|1|79|9|1|16591|127|597|9|24.07|31.53|18.28|119.25|164.52|216.63|283.77|11.51|0.00|85.05|164.52|176.03|249.57|261.08|-52.11| +2450826|33945|2450912|8048|1614451|2454|38010|8048|1614451|2454|38010|1|106|8|4|13538|153|597|94|32.84|37.76|25.29|1172.18|2377.26|3086.96|3549.44|118.86|0.00|1241.74|2377.26|2496.12|3619.00|3737.86|-709.70| +2450826|33945|2450880|8048|1614451|2454|38010|8048|1614451|2454|38010|1|99|8|3|10208|193|597|25|27.64|82.09|65.67|410.50|1641.75|691.00|2052.25|0.00|229.84|246.25|1411.91|1411.91|1658.16|1658.16|720.91| +2450826|33945|2450845|8048|1614451|2454|38010|8048|1614451|2454|38010|1|72|19|4|13549|126|597|71|40.37|83.96|69.68|1013.88|4947.28|2866.27|5961.16|445.25|0.00|118.57|4947.28|5392.53|5065.85|5511.10|2081.01| +2450826|33945|2450870|8048|1614451|2454|38010|8048|1614451|2454|38010|1|23|14|2|14239|15|597|92|44.52|127.32|84.03|3982.68|7730.76|4095.84|11713.44|77.30|0.00|4684.64|7730.76|7808.06|12415.40|12492.70|3634.92| +2450826|38301|2450831|23265|703401|6384|39767|23265|703401|6384|39767|2|43|14|2|2200|64|598|53|94.59|242.15|188.87|2823.84|10010.11|5013.27|12833.95|355.35|2902.93|769.56|7107.18|7462.53|7876.74|8232.09|2093.91| +2450826|38301|2450881|23265|703401|6384|39767|23265|703401|6384|39767|2|44|4|5|7928|98|598|58|54.26|130.22|9.11|7024.38|528.38|3147.08|7552.76|47.55|0.00|2265.48|528.38|575.93|2793.86|2841.41|-2618.70| +2450826|38301|2450890|23265|703401|6384|39767|23265|703401|6384|39767|2|9|4|1|5755|15|598|83|93.14|224.46|109.98|9501.84|9128.34|7730.62|18630.18|0.00|3651.33|2049.27|5477.01|5477.01|7526.28|7526.28|-2253.61| +2450826|38301|2450861|23265|703401|6384|39767|23265|703401|6384|39767|2|58|4|3|12640|69|598|81|94.28|229.10|96.22|10763.28|7793.82|7636.68|18557.10|0.00|0.00|3896.91|7793.82|7793.82|11690.73|11690.73|157.14| +2450826|38301|2450849|23265|703401|6384|39767|23265|703401|6384|39767|2|89|3|4|13006|108|598|13|8.87|23.77|18.77|65.00|244.01|115.31|309.01|0.00|0.00|105.04|244.01|244.01|349.05|349.05|128.70| +2450826|38301|2450857|23265|703401|6384|39767|23265|703401|6384|39767|2|23|15|5|5060|2|598|68|9.63|12.42|10.43|135.32|709.24|654.84|844.56|2.55|666.68|16.32|42.56|45.11|58.88|61.43|-612.28| +2450826|38301|2450877|23265|703401|6384|39767|23265|703401|6384|39767|2|81|8|4|7492|139|598|74|29.23|33.90|27.12|501.72|2006.88|2163.02|2508.60|60.20|0.00|953.12|2006.88|2067.08|2960.00|3020.20|-156.14| +2450826|38301|2450838|23265|703401|6384|39767|23265|703401|6384|39767|2|9|12|4|14846|223|598|61|82.16|133.92|45.53|5391.79|2777.33|5011.76|8169.12|55.54|0.00|2368.63|2777.33|2832.87|5145.96|5201.50|-2234.43| +2450826|38301|2450895|23265|703401|6384|39767|23265|703401|6384|39767|2|9|6|4|14800|83|598|53|59.01|151.65|80.37|3777.84|4259.61|3127.53|8037.45|20.44|4004.03|240.62|255.58|276.02|496.20|516.64|-2871.95| +2450826|41856|2450881|1747|399434|3074|17961|1747|399434|3074|17961|4|83|5|1|982|35|599|45|24.52|35.30|15.88|873.90|714.60|1103.40|1588.50|0.00|0.00|94.95|714.60|714.60|809.55|809.55|-388.80| +2450826|41856|2450861|1747|399434|3074|17961|1747|399434|3074|17961|4|56|15|4|10156|142|599|75|10.28|23.13|10.87|919.50|815.25|771.00|1734.75|0.00|277.18|0.00|538.07|538.07|538.07|538.07|-232.93| +2450826|41856|2450882|1747|399434|3074|17961|1747|399434|3074|17961|4|60|10|2|7958|96|599|25|65.84|178.42|37.46|3524.00|936.50|1646.00|4460.50|37.46|0.00|1025.75|936.50|973.96|1962.25|1999.71|-709.50| +2450826|41856|2450900|1747|399434|3074|17961|1747|399434|3074|17961|4|11|13|3|7328|298|599|2|63.39|109.66|28.51|162.30|57.02|126.78|219.32|0.57|0.00|83.34|57.02|57.59|140.36|140.93|-69.76| +2450826|41856|2450913|1747|399434|3074|17961|1747|399434|3074|17961|4|26|18|3|8479|113|599|61|78.13|220.32|114.56|6451.36|6988.16|4765.93|13439.52|139.76|0.00|6719.76|6988.16|7127.92|13707.92|13847.68|2222.23| +2450826|80099|2450872|58801|539334|3867|13200|58801|539334|3867|13200|4|39|14|2|16258|260|600|52|15.13|29.20|0.00|1518.40|0.00|786.76|1518.40|0.00|0.00|607.36|0.00|0.00|607.36|607.36|-786.76| +2450826|80099|2450855|58801|539334|3867|13200|58801|539334|3867|13200|4|55|20|4|784|96|600|48|42.50|75.65|23.45|2505.60|1125.60|2040.00|3631.20|67.53|0.00|362.88|1125.60|1193.13|1488.48|1556.01|-914.40| +2450826|80099|2450842|58801|539334|3867|13200|58801|539334|3867|13200|4|73|13|3|1525|98|600|25|78.10|188.22|135.51|1317.75|3387.75|1952.50|4705.50|203.26|0.00|188.00|3387.75|3591.01|3575.75|3779.01|1435.25| +2450826|80099|2450871|58801|539334|3867|13200|58801|539334|3867|13200|4|66|5|4|9295|85|600|71|79.49|112.87|47.40|4648.37|3365.40|5643.79|8013.77|110.38|1985.58|1281.55|1379.82|1490.20|2661.37|2771.75|-4263.97| +2450826|80099|2450893|58801|539334|3867|13200|58801|539334|3867|13200|4|35|12|4|3634|148|600|72|81.37|222.95|140.45|5940.00|10112.40|5858.64|16052.40|303.37|0.00|320.40|10112.40|10415.77|10432.80|10736.17|4253.76| +2450826|80099|2450872|58801|539334|3867|13200|58801|539334|3867|13200|4|32|13|5|10714|200|600|22|88.26|245.36|144.76|2213.20|3184.72|1941.72|5397.92|0.00|222.93|1079.54|2961.79|2961.79|4041.33|4041.33|1020.07| +2450826|63216|2450879|20384|239781|4778|27775|20384|239781|4778|27775|4|57|7|5|12106|291|601|14|65.46|76.58|16.08|847.00|225.12|916.44|1072.12|8.82|4.50|482.44|220.62|229.44|703.06|711.88|-695.82| +2450826|63216|2450905|20384|239781|4778|27775|20384|239781|4778|27775|4|33|15|1|932|51|601|45|98.28|161.17|127.32|1523.25|5729.40|4422.60|7252.65|286.47|0.00|2030.40|5729.40|6015.87|7759.80|8046.27|1306.80| +2450826|63216|2450831|20384|239781|4778|27775|20384|239781|4778|27775|4|102|5|3|5242|136|601|98|64.65|177.78|35.55|13938.54|3483.90|6335.70|17422.44|104.51|0.00|6619.90|3483.90|3588.41|10103.80|10208.31|-2851.80| +2450826|63216|2450915|20384|239781|4778|27775|20384|239781|4778|27775|4|62|16|5|11749|145|601|90|20.85|23.35|20.31|273.60|1827.90|1876.50|2101.50|0.00|0.00|483.30|1827.90|1827.90|2311.20|2311.20|-48.60| +2450826|63216|2450898|20384|239781|4778|27775|20384|239781|4778|27775|4|106|2|1|16846|204|601|45|34.64|83.48|53.42|1352.70|2403.90|1558.80|3756.60|134.61|721.17|1690.20|1682.73|1817.34|3372.93|3507.54|123.93| +2450826|63216|2450865|20384|239781|4778|27775|20384|239781|4778|27775|4|100|11|4|11572|17|601|38|7.61|11.94|8.95|113.62|340.10|289.18|453.72|0.00|98.62|199.50|241.48|241.48|440.98|440.98|-47.70| +2450826|63216|2450910|20384|239781|4778|27775|20384|239781|4778|27775|4|40|10|5|2642|40|601|93|2.75|6.46|4.84|150.66|450.12|255.75|600.78|40.51|0.00|137.64|450.12|490.63|587.76|628.27|194.37| +2450826|63216|2450866|20384|239781|4778|27775|20384|239781|4778|27775|4|108|18|1|15394|43|601|41|41.68|76.69|70.55|251.74|2892.55|1708.88|3144.29|12.72|2256.18|94.30|636.37|649.09|730.67|743.39|-1072.51| +2450826|63216|2450896|20384|239781|4778|27775|20384|239781|4778|27775|4|10|14|1|13186|292|601|73|31.13|47.31|6.15|3004.68|448.95|2272.49|3453.63|13.46|0.00|1484.82|448.95|462.41|1933.77|1947.23|-1823.54| +2450826|63216|2450831|20384|239781|4778|27775|20384|239781|4778|27775|4|63|12|4|13814|273|601|48|56.97|127.61|26.79|4839.36|1285.92|2734.56|6125.28|11.57|1054.45|2817.60|231.47|243.04|3049.07|3060.64|-2503.09| +2450826|63216|2450842|20384|239781|4778|27775|20384|239781|4778|27775|4|104|10|4|15532|142|601|50|93.65|198.53|186.61|596.00|9330.50|4682.50|9926.50|839.74|0.00|2382.00|9330.50|10170.24|11712.50|12552.24|4648.00| +2450826|63216|2450868|20384|239781|4778|27775|20384|239781|4778|27775|4|3|8|3|16424|268|601|3|8.46|8.79|1.84|20.85|5.52|25.38|26.37|0.15|1.76|12.63|3.76|3.91|16.39|16.54|-21.62| +2450826|63216|2450874|20384|239781|4778|27775|20384|239781|4778|27775|4|66|10|3|3946|127|601|26|85.62|125.86|7.55|3076.06|196.30|2226.12|3272.36|1.96|0.00|817.96|196.30|198.26|1014.26|1016.22|-2029.82| +2450826|63216|2450915|20384|239781|4778|27775|20384|239781|4778|27775|4|15|7|5|13934|184|601|34|14.48|25.34|7.85|594.66|266.90|492.32|861.56|5.33|0.00|387.60|266.90|272.23|654.50|659.83|-225.42| +2450826|29095|2450909|37748|1812536|3677|865|37748|1812536|3677|865|1|58|18|4|2354|251|602|53|99.79|256.46|233.37|1223.77|12368.61|5288.87|13592.38|618.43|0.00|1630.81|12368.61|12987.04|13999.42|14617.85|7079.74| +2450826|29095|2450876|37748|1812536|3677|865|37748|1812536|3677|865|1|28|18|3|13816|117|602|54|22.00|24.64|8.37|878.58|451.98|1188.00|1330.56|36.15|0.00|518.40|451.98|488.13|970.38|1006.53|-736.02| +2450826|29095|2450888|37748|1812536|3677|865|37748|1812536|3677|865|1|62|19|4|16774|151|602|63|1.13|2.88|1.41|92.61|88.83|71.19|181.44|4.44|0.00|68.67|88.83|93.27|157.50|161.94|17.64| +2450826|29095|2450886|37748|1812536|3677|865|37748|1812536|3677|865|1|23|14|5|16442|195|602|90|15.87|47.45|9.49|3416.40|854.10|1428.30|4270.50|51.24|0.00|1024.20|854.10|905.34|1878.30|1929.54|-574.20| +2450826|29095|2450860|37748|1812536|3677|865|37748|1812536|3677|865|1|67|19|2|12320|153|602|30|53.67|154.03|95.49|1756.20|2864.70|1610.10|4620.90|85.94|0.00|1293.60|2864.70|2950.64|4158.30|4244.24|1254.60| +2450826|29095|2450863|37748|1812536|3677|865|37748|1812536|3677|865|1|53|3|4|8708|43|602|56|89.42|264.68|140.28|6966.40|7855.68|5007.52|14822.08|707.01|0.00|6224.96|7855.68|8562.69|14080.64|14787.65|2848.16| +2450826|38488|2450878|51154|1210607|813|41866|51154|1210607|813|41866|2|97|14|2|15370|203|603|1|35.39|53.43|40.60|12.83|40.60|35.39|53.43|2.43|10.15|7.48|30.45|32.88|37.93|40.36|-4.94| +2450826|38488|2450911|51154|1210607|813|41866|51154|1210607|813|41866|2|10|5|5|8638|278|603|38|43.79|87.14|62.74|927.20|2384.12|1664.02|3311.32|119.20|0.00|761.52|2384.12|2503.32|3145.64|3264.84|720.10| +2450826|38488|2450859|51154|1210607|813|41866|51154|1210607|813|41866|2|86|14|1|11485|90|603|39|33.84|55.15|1.10|2107.95|42.90|1319.76|2150.85|2.14|0.00|774.15|42.90|45.04|817.05|819.19|-1276.86| +2450826|38488|2450899|51154|1210607|813|41866|51154|1210607|813|41866|2|57|20|1|13952|263|603|2|23.08|60.00|4.20|111.60|8.40|46.16|120.00|0.67|0.00|39.60|8.40|9.07|48.00|48.67|-37.76| +2450826|38488|2450849|51154|1210607|813|41866|51154|1210607|813|41866|2|84|19|4|10348|296|603|13|51.65|106.39|45.74|788.45|594.62|671.45|1383.07|7.37|225.95|622.31|368.67|376.04|990.98|998.35|-302.78| +2450826|38488|2450829|51154|1210607|813|41866|51154|1210607|813|41866|2|14|8|3|13819|61|603|55|60.38|124.38|113.18|616.00|6224.90|3320.90|6840.90|497.99|0.00|1709.95|6224.90|6722.89|7934.85|8432.84|2904.00| +2450826|38488|2450848|51154|1210607|813|41866|51154|1210607|813|41866|2|56|18|4|6049|246|603|28|50.74|120.76|54.34|1859.76|1521.52|1420.72|3381.28|121.72|0.00|168.84|1521.52|1643.24|1690.36|1812.08|100.80| +2450826|38488|2450910|51154|1210607|813|41866|51154|1210607|813|41866|2|12|14|3|17026|240|603|23|32.21|36.39|20.01|376.74|460.23|740.83|836.97|11.78|165.68|117.07|294.55|306.33|411.62|423.40|-446.28| +2450826|38488|2450835|51154|1210607|813|41866|51154|1210607|813|41866|2|90|7|1|13016|156|603|76|42.50|56.52|19.21|2835.56|1459.96|3230.00|4295.52|102.19|0.00|987.24|1459.96|1562.15|2447.20|2549.39|-1770.04| +2450826|38488|2450835|51154|1210607|813|41866|51154|1210607|813|41866|2|89|13|5|10699|162|603|61|48.02|129.65|86.86|2610.19|5298.46|2929.22|7908.65|370.89|0.00|2926.17|5298.46|5669.35|8224.63|8595.52|2369.24| +2450826|38488|2450880|51154|1210607|813|41866|51154|1210607|813|41866|2|78|7|2|836|222|603|63|94.38|97.21|87.48|612.99|5511.24|5945.94|6124.23|0.00|0.00|1898.19|5511.24|5511.24|7409.43|7409.43|-434.70| +2450826|58521|2450901|97813|1181022|6984|37793|79300|194564|973|18982|1|75|15|5|17234|143|604|81|17.71|40.73|23.62|1385.91|1913.22|1434.51|3299.13|0.00|0.00|1418.31|1913.22|1913.22|3331.53|3331.53|478.71| +2450826|58521|2450898|97813|1181022|6984|37793|79300|194564|973|18982|1|89|5|2|5908|80|604|7|85.75|97.75|42.03|390.04|294.21|600.25|684.25|2.94|0.00|239.47|294.21|297.15|533.68|536.62|-306.04| +2450826|58521|2450883|97813|1181022|6984|37793|79300|194564|973|18982|1|67|9|2|5269|168|604|69|1.39|3.09|1.66|98.67|114.54|95.91|213.21|4.58|0.00|44.16|114.54|119.12|158.70|163.28|18.63| +2450826|58521|2450833|97813|1181022|6984|37793|79300|194564|973|18982|1|21|6|4|2258|79|604|52|19.17|43.70|9.61|1772.68|499.72|996.84|2272.40|0.00|349.80|22.36|149.92|149.92|172.28|172.28|-846.92| +2450826|58521|2450866|97813|1181022|6984|37793|79300|194564|973|18982|1|22|8|3|14866|16|604|67|43.18|57.42|37.89|1308.51|2538.63|2893.06|3847.14|152.31|0.00|538.01|2538.63|2690.94|3076.64|3228.95|-354.43| +2450826|62688|2450889|62343|1602550|804|11058|3130|910319|6973|3580|1|50|19|2|2719|280|605|84|38.88|114.69|71.10|3661.56|5972.40|3265.92|9633.96|537.51|0.00|4431.00|5972.40|6509.91|10403.40|10940.91|2706.48| +2450826|62688|2450828|62343|1602550|804|11058|3130|910319|6973|3580|1|102|7|3|1474|70|605|67|78.98|123.20|113.34|660.62|7593.78|5291.66|8254.40|303.75|0.00|1567.80|7593.78|7897.53|9161.58|9465.33|2302.12| +2450826|62688|2450871|62343|1602550|804|11058|3130|910319|6973|3580|1|97|10|2|6409|121|605|60|81.34|156.17|118.68|2249.40|7120.80|4880.40|9370.20|498.45|0.00|3373.20|7120.80|7619.25|10494.00|10992.45|2240.40| +2450826|62688|2450894|62343|1602550|804|11058|3130|910319|6973|3580|1|55|14|4|17774|137|605|42|58.60|89.07|73.03|673.68|3067.26|2461.20|3740.94|153.36|0.00|1870.26|3067.26|3220.62|4937.52|5090.88|606.06| +2450826|62688|2450834|62343|1602550|804|11058|3130|910319|6973|3580|1|50|3|3|4478|113|605|39|29.45|39.46|6.70|1277.64|261.30|1148.55|1538.94|5.22|0.00|446.16|261.30|266.52|707.46|712.68|-887.25| +2450826|62688|2450879|62343|1602550|804|11058|3130|910319|6973|3580|1|42|17|4|16058|81|605|74|18.04|24.35|9.98|1063.38|738.52|1334.96|1801.90|14.77|0.00|216.08|738.52|753.29|954.60|969.37|-596.44| +2450826|62688|2450862|62343|1602550|804|11058|3130|910319|6973|3580|1|104|17|3|12470|265|605|54|34.24|99.98|93.98|324.00|5074.92|1848.96|5398.92|168.99|3197.19|1133.46|1877.73|2046.72|3011.19|3180.18|28.77| +2450826|62688|2450899|62343|1602550|804|11058|3130|910319|6973|3580|1|38|17|3|7594|139|605|30|36.46|43.38|37.30|182.40|1119.00|1093.80|1301.40|22.38|0.00|208.20|1119.00|1141.38|1327.20|1349.58|25.20| +2450826|62688|2450846|62343|1602550|804|11058|3130|910319|6973|3580|1|30|4|3|4880|61|605|24|79.32|139.60|50.25|2144.40|1206.00|1903.68|3350.40|0.00|0.00|770.40|1206.00|1206.00|1976.40|1976.40|-697.68| +2450826|44094|2450859|98459|3822|1632|36995|98459|3822|1632|36995|4|15|11|3|11659|252|606|7|62.37|91.68|7.33|590.45|51.31|436.59|641.76|4.61|0.00|128.31|51.31|55.92|179.62|184.23|-385.28| +2450826|44094|2450898|98459|3822|1632|36995|98459|3822|1632|36995|4|103|1|5|8914|276|606|7|81.32|130.11|15.61|801.50|109.27|569.24|910.77|0.00|0.00|273.21|109.27|109.27|382.48|382.48|-459.97| +2450826|44094|2450829|98459|3822|1632|36995|98459|3822|1632|36995|4|89|17|1|2239|263|606|46|85.50|191.52|22.98|7752.84|1057.08|3933.00|8809.92|10.57|0.00|4228.32|1057.08|1067.65|5285.40|5295.97|-2875.92| +2450826|44094|2450855|98459|3822|1632|36995|98459|3822|1632|36995|4|81|2|3|14305|225|606|87|82.97|112.83|34.97|6773.82|3042.39|7218.39|9816.21|182.54|0.00|883.05|3042.39|3224.93|3925.44|4107.98|-4176.00| +2450826|44094|2450834|98459|3822|1632|36995|98459|3822|1632|36995|4|44|20|3|6214|188|606|30|73.05|155.59|49.78|3174.30|1493.40|2191.50|4667.70|44.80|0.00|1680.30|1493.40|1538.20|3173.70|3218.50|-698.10| +2450826|44094|2450854|98459|3822|1632|36995|98459|3822|1632|36995|4|28|6|3|7258|27|606|60|13.63|15.67|8.61|423.60|516.60|817.80|940.20|20.66|0.00|272.40|516.60|537.26|789.00|809.66|-301.20| +2450826|44094|2450874|98459|3822|1632|36995|98459|3822|1632|36995|4|36|17|1|295|210|606|44|82.55|94.93|71.19|1044.56|3132.36|3632.20|4176.92|62.64|0.00|918.72|3132.36|3195.00|4051.08|4113.72|-499.84| +2450826|44094|2450847|98459|3822|1632|36995|98459|3822|1632|36995|4|68|18|5|17842|192|606|90|14.66|40.31|4.43|3229.20|398.70|1319.40|3627.90|3.98|0.00|979.20|398.70|402.68|1377.90|1381.88|-920.70| +2450826|44094|2450905|98459|3822|1632|36995|98459|3822|1632|36995|4|69|12|4|5383|211|606|70|87.45|93.57|56.14|2620.10|3929.80|6121.50|6549.90|117.89|0.00|3143.70|3929.80|4047.69|7073.50|7191.39|-2191.70| +2450826|44094|2450902|98459|3822|1632|36995|98459|3822|1632|36995|4|45|14|1|9715|115|606|26|43.53|114.04|54.73|1542.06|1422.98|1131.78|2965.04|0.00|0.00|148.20|1422.98|1422.98|1571.18|1571.18|291.20| +2450826|44094|2450869|98459|3822|1632|36995|98459|3822|1632|36995|4|102|19|3|3680|10|606|54|46.62|101.16|15.17|4643.46|819.18|2517.48|5462.64|49.15|0.00|2403.54|819.18|868.33|3222.72|3271.87|-1698.30| +2450826|67665|2450864|98751|1126621|6044|35467|98751|1126621|6044|35467|2|66|2|1|3610|90|607|3|77.28|105.87|99.51|19.08|298.53|231.84|317.61|8.80|122.39|6.33|176.14|184.94|182.47|191.27|-55.70| +2450826|67665|2450839|98751|1126621|6044|35467|98751|1126621|6044|35467|2|6|19|5|2500|175|607|17|19.94|24.52|6.62|304.30|112.54|338.98|416.84|4.50|0.00|133.28|112.54|117.04|245.82|250.32|-226.44| +2450826|67665|2450871|98751|1126621|6044|35467|98751|1126621|6044|35467|2|64|18|2|6650|61|607|64|47.03|92.64|24.08|4387.84|1541.12|3009.92|5928.96|35.13|369.86|177.28|1171.26|1206.39|1348.54|1383.67|-1838.66| +2450826|67665|2450830|98751|1126621|6044|35467|98751|1126621|6044|35467|2|93|12|5|5374|277|607|70|86.29|209.68|4.19|14384.30|293.30|6040.30|14677.60|3.69|231.70|146.30|61.60|65.29|207.90|211.59|-5978.70| +2450826|72614|2450874|89877|704890|2750|18963|89877|704890|2750|18963|4|67|8|1|8708|274|608|80|15.56|33.14|20.87|981.60|1669.60|1244.80|2651.20|0.00|267.13|370.40|1402.47|1402.47|1772.87|1772.87|157.67| +2450826|72614|2450877|89877|704890|2750|18963|89877|704890|2750|18963|4|55|5|2|9535|78|608|39|94.84|243.73|82.86|6273.93|3231.54|3698.76|9505.47|290.83|0.00|1140.36|3231.54|3522.37|4371.90|4662.73|-467.22| +2450826|72614|2450900|89877|704890|2750|18963|89877|704890|2750|18963|4|76|20|3|10832|158|608|41|94.97|145.30|43.59|4170.11|1787.19|3893.77|5957.30|19.30|1572.72|2084.85|214.47|233.77|2299.32|2318.62|-3679.30| +2450826|72614|2450875|89877|704890|2750|18963|89877|704890|2750|18963|4|35|8|2|8023|3|608|40|93.43|93.43|34.56|2354.80|1382.40|3737.20|3737.20|96.76|0.00|410.80|1382.40|1479.16|1793.20|1889.96|-2354.80| +2450826|72614|2450894|89877|704890|2750|18963|89877|704890|2750|18963|4|89|6|1|16753|104|608|99|2.04|5.99|5.75|23.76|569.25|201.96|593.01|28.46|0.00|171.27|569.25|597.71|740.52|768.98|367.29| +2450826|72614|2450884|89877|704890|2750|18963|89877|704890|2750|18963|4|50|1|1|2038|147|608|58|96.50|165.98|114.52|2984.68|6642.16|5597.00|9626.84|531.37|0.00|2598.98|6642.16|7173.53|9241.14|9772.51|1045.16| +2450826|23208|2450906|8462|758983|3904|13272|8462|758983|3904|13272|1|31|10|1|4687|186|609|46|96.74|139.30|34.82|4806.08|1601.72|4450.04|6407.80|48.05|0.00|1730.06|1601.72|1649.77|3331.78|3379.83|-2848.32| +2450826|23208|2450901|8462|758983|3904|13272|8462|758983|3904|13272|1|55|15|2|17755|56|609|43|20.23|36.41|20.75|673.38|892.25|869.89|1565.63|62.45|0.00|594.69|892.25|954.70|1486.94|1549.39|22.36| +2450826|23208|2450856|8462|758983|3904|13272|8462|758983|3904|13272|1|84|16|3|15487|121|609|80|79.11|170.08|142.86|2177.60|11428.80|6328.80|13606.40|228.57|0.00|544.00|11428.80|11657.37|11972.80|12201.37|5100.00| +2450826|23208|2450854|8462|758983|3904|13272|8462|758983|3904|13272|1|68|11|4|14272|115|609|77|48.14|117.94|0.00|9081.38|0.00|3706.78|9081.38|0.00|0.00|2996.84|0.00|0.00|2996.84|2996.84|-3706.78| +2450826|23208|2450864|8462|758983|3904|13272|8462|758983|3904|13272|1|107|1|4|10586|195|609|24|71.90|74.05|41.46|782.16|995.04|1725.60|1777.20|79.60|0.00|462.00|995.04|1074.64|1457.04|1536.64|-730.56| +2450826|23208|2450860|8462|758983|3904|13272|8462|758983|3904|13272|1|57|11|3|8368|247|609|66|94.05|246.41|219.30|1789.26|14473.80|6207.30|16263.06|1302.64|0.00|5691.84|14473.80|15776.44|20165.64|21468.28|8266.50| +2450826|23208|2450855|8462|758983|3904|13272|8462|758983|3904|13272|1|36|1|1|3380|108|609|100|45.03|67.54|50.65|1689.00|5065.00|4503.00|6754.00|253.25|0.00|1823.00|5065.00|5318.25|6888.00|7141.25|562.00| +2450826|23208|2450896|8462|758983|3904|13272|8462|758983|3904|13272|1|37|8|5|16864|215|609|55|12.02|23.19|15.30|433.95|841.50|661.10|1275.45|25.24|0.00|0.00|841.50|866.74|841.50|866.74|180.40| +2450826|23208|2450864|8462|758983|3904|13272|8462|758983|3904|13272|1|46|12|1|7796|231|609|81|18.79|30.43|10.04|1651.59|813.24|1521.99|2464.83|73.19|0.00|665.01|813.24|886.43|1478.25|1551.44|-708.75| +2450826|23208|2450888|8462|758983|3904|13272|8462|758983|3904|13272|1|73|1|5|16676|231|609|75|24.79|49.58|7.43|3161.25|557.25|1859.25|3718.50|22.29|0.00|185.25|557.25|579.54|742.50|764.79|-1302.00| +2450826|23208|2450863|8462|758983|3904|13272|8462|758983|3904|13272|1|26|14|2|3434|253|609|56|80.19|117.87|66.00|2904.72|3696.00|4490.64|6600.72|110.88|0.00|1518.16|3696.00|3806.88|5214.16|5325.04|-794.64| +2450826|23208|2450873|8462|758983|3904|13272|8462|758983|3904|13272|1|38|14|1|12514|185|609|8|60.67|77.65|72.21|43.52|577.68|485.36|621.20|34.66|0.00|291.92|577.68|612.34|869.60|904.26|92.32| +2450826|23208|2450870|8462|758983|3904|13272|8462|758983|3904|13272|1|94|12|3|4873|200|609|83|35.96|40.27|12.48|2306.57|1035.84|2984.68|3342.41|20.71|0.00|1370.33|1035.84|1056.55|2406.17|2426.88|-1948.84| +2450826|1358|2450878|31707|1122864|3621|8425|31707|1122864|3621|8425|1|63|14|3|736|216|610|18|22.51|32.63|9.78|411.30|176.04|405.18|587.34|8.80|0.00|275.94|176.04|184.84|451.98|460.78|-229.14| +2450826|1358|2450850|31707|1122864|3621|8425|31707|1122864|3621|8425|1|34|7|4|3151|274|610|69|96.61|231.86|231.86|0.00|15998.34|6666.09|15998.34|959.90|0.00|5758.74|15998.34|16958.24|21757.08|22716.98|9332.25| +2450826|1358|2450829|31707|1122864|3621|8425|31707|1122864|3621|8425|1|108|1|3|15830|289|610|17|79.66|195.96|111.69|1432.59|1898.73|1354.22|3331.32|75.94|0.00|766.19|1898.73|1974.67|2664.92|2740.86|544.51| +2450826|1358|2450833|31707|1122864|3621|8425|31707|1122864|3621|8425|1|86|1|1|16228|205|610|53|36.90|61.99|34.09|1478.70|1806.77|1955.70|3285.47|126.47|0.00|426.65|1806.77|1933.24|2233.42|2359.89|-148.93| +2450826|1358|2450876|31707|1122864|3621|8425|31707|1122864|3621|8425|1|52|17|1|1378|4|610|90|44.53|73.47|33.79|3571.20|3041.10|4007.70|6612.30|30.41|0.00|792.90|3041.10|3071.51|3834.00|3864.41|-966.60| +2450826|1358|2450892|31707|1122864|3621|8425|31707|1122864|3621|8425|1|96|20|1|12406|277|610|70|81.58|209.66|50.31|11154.50|3521.70|5710.60|14676.20|211.30|0.00|6163.50|3521.70|3733.00|9685.20|9896.50|-2188.90| +2450826|1358|2450904|31707|1122864|3621|8425|31707|1122864|3621|8425|1|81|14|3|13214|89|610|98|39.80|105.86|91.03|1453.34|8920.94|3900.40|10374.28|624.46|0.00|3941.56|8920.94|9545.40|12862.50|13486.96|5020.54| +2450826|78175|2450872|34007|661676|2498|32967|34007|661676|2498|32967|1|15|20|4|8356|98|611|32|41.00|106.60|75.68|989.44|2421.76|1312.00|3411.20|193.74|0.00|1466.56|2421.76|2615.50|3888.32|4082.06|1109.76| +2450826|78175|2450835|34007|661676|2498|32967|34007|661676|2498|32967|1|62|17|5|2707|196|611|81|96.10|244.09|119.60|10083.69|9687.60|7784.10|19771.29|290.62|0.00|7710.39|9687.60|9978.22|17397.99|17688.61|1903.50| +2450826|78175|2450851|34007|661676|2498|32967|34007|661676|2498|32967|1|1|1|2|12733|273|611|61|5.77|6.40|3.64|168.36|222.04|351.97|390.40|6.66|0.00|34.77|222.04|228.70|256.81|263.47|-129.93| +2450826|78175|2450856|34007|661676|2498|32967|34007|661676|2498|32967|1|37|20|3|16718|275|611|11|64.54|67.76|0.00|745.36|0.00|709.94|745.36|0.00|0.00|22.33|0.00|0.00|22.33|22.33|-709.94| +2450826|78175|2450843|34007|661676|2498|32967|34007|661676|2498|32967|1|21|17|1|5545|4|611|70|69.85|165.54|3.31|11356.10|231.70|4889.50|11587.80|2.31|0.00|5561.50|231.70|234.01|5793.20|5795.51|-4657.80| +2450826|78175|2450913|34007|661676|2498|32967|34007|661676|2498|32967|1|38|15|1|17815|151|611|13|17.91|35.82|32.23|46.67|418.99|232.83|465.66|4.18|0.00|204.88|418.99|423.17|623.87|628.05|186.16| +2450826|78175|2450853|34007|661676|2498|32967|34007|661676|2498|32967|1|98|3|4|10975|61|611|20|44.09|111.98|68.30|873.60|1366.00|881.80|2239.60|95.62|0.00|649.40|1366.00|1461.62|2015.40|2111.02|484.20| +2450826|78175|2450915|34007|661676|2498|32967|34007|661676|2498|32967|1|89|11|3|15148|270|611|57|2.68|3.40|2.65|42.75|151.05|152.76|193.80|6.04|0.00|50.16|151.05|157.09|201.21|207.25|-1.71| +2450826|78175|2450913|34007|661676|2498|32967|34007|661676|2498|32967|1|2|15|2|4972|18|611|23|4.11|10.35|9.31|23.92|214.13|94.53|238.05|0.00|0.00|49.91|214.13|214.13|264.04|264.04|119.60| +2450826|58369|2450840|73473|725233|3840|40000|69045|351920|2080|33840|4|87|13|4|15830|259|612|87|85.84|138.20|98.12|3486.96|8536.44|7468.08|12023.40|0.00|0.00|6011.70|8536.44|8536.44|14548.14|14548.14|1068.36| +2450826|58369|2450909|73473|725233|3840|40000|69045|351920|2080|33840|4|92|8|5|16228|194|612|96|41.26|105.62|103.50|203.52|9936.00|3960.96|10139.52|695.52|0.00|506.88|9936.00|10631.52|10442.88|11138.40|5975.04| +2450826|58369|2450832|73473|725233|3840|40000|69045|351920|2080|33840|4|78|17|2|1378|87|612|44|73.65|120.78|4.83|5101.80|212.52|3240.60|5314.32|12.94|27.62|2497.44|184.90|197.84|2682.34|2695.28|-3055.70| +2450826|58369|2450882|73473|725233|3840|40000|69045|351920|2080|33840|4|70|5|3|12406|216|612|89|14.64|21.08|15.38|507.30|1368.82|1302.96|1876.12|82.12|0.00|449.45|1368.82|1450.94|1818.27|1900.39|65.86| +2450826|58369|2450829|73473|725233|3840|40000|69045|351920|2080|33840|4|61|15|5|13214|234|612|51|100.00|295.00|185.85|5566.65|9478.35|5100.00|15045.00|379.13|0.00|5566.65|9478.35|9857.48|15045.00|15424.13|4378.35| +2450826|58369|2450840|73473|725233|3840|40000|69045|351920|2080|33840|4|95|18|1|14359|191|612|66|31.80|75.04|3.00|4754.64|198.00|2098.80|4952.64|9.90|0.00|742.50|198.00|207.90|940.50|950.40|-1900.80| +2450826|58369|2450828|73473|725233|3840|40000|69045|351920|2080|33840|4|10|12|5|7154|71|612|54|27.94|63.14|18.94|2386.80|1022.76|1508.76|3409.56|51.13|0.00|476.82|1022.76|1073.89|1499.58|1550.71|-486.00| +2450826|58369|2450831|73473|725233|3840|40000|69045|351920|2080|33840|4|16|8|4|5317|249|612|18|76.83|172.09|18.92|2757.06|340.56|1382.94|3097.62|2.45|309.90|247.68|30.66|33.11|278.34|280.79|-1352.28| +2450826|58369|2450834|73473|725233|3840|40000|69045|351920|2080|33840|4|11|5|3|16840|48|612|45|26.13|48.07|9.61|1730.70|432.45|1175.85|2163.15|30.27|0.00|670.50|432.45|462.72|1102.95|1133.22|-743.40| +2450826|58369|2450859|73473|725233|3840|40000|69045|351920|2080|33840|4|20|12|5|658|227|612|44|49.61|140.89|9.86|5765.32|433.84|2182.84|6199.16|17.35|0.00|867.68|433.84|451.19|1301.52|1318.87|-1749.00| +2450826|58369|2450839|73473|725233|3840|40000|69045|351920|2080|33840|4|88|16|2|6376|126|612|97|18.14|20.31|13.40|670.27|1299.80|1759.58|1970.07|25.99|0.00|944.78|1299.80|1325.79|2244.58|2270.57|-459.78| +2450826|58369|2450872|73473|725233|3840|40000|69045|351920|2080|33840|4|34|1|3|9116|55|612|54|17.16|19.39|10.27|492.48|554.58|926.64|1047.06|22.18|0.00|104.22|554.58|576.76|658.80|680.98|-372.06| +2450826|58369|2450907|73473|725233|3840|40000|69045|351920|2080|33840|4|33|15|3|1603|119|612|18|88.48|114.13|28.53|1540.80|513.54|1592.64|2054.34|20.54|0.00|307.98|513.54|534.08|821.52|842.06|-1079.10| +2450826|49394|2450899|35076|90054|6985|37163|35076|90054|6985|37163|2|5|5|2|3092|168|613|77|82.52|227.75|81.99|11223.52|6313.23|6354.04|17536.75|126.26|0.00|7716.17|6313.23|6439.49|14029.40|14155.66|-40.81| +2450826|49394|2450910|35076|90054|6985|37163|35076|90054|6985|37163|2|85|12|1|6268|170|613|14|81.32|117.91|40.08|1089.62|561.12|1138.48|1650.74|22.44|0.00|231.00|561.12|583.56|792.12|814.56|-577.36| +2450826|49394|2450857|35076|90054|6985|37163|35076|90054|6985|37163|2|70|5|4|3382|230|613|27|40.32|67.73|39.96|749.79|1078.92|1088.64|1828.71|10.78|0.00|91.26|1078.92|1089.70|1170.18|1180.96|-9.72| +2450826|49394|2450898|35076|90054|6985|37163|35076|90054|6985|37163|2|44|19|2|15931|8|613|57|72.01|113.05|56.52|3222.21|3221.64|4104.57|6443.85|96.64|0.00|966.15|3221.64|3318.28|4187.79|4284.43|-882.93| +2450827|12371|2450840|27313|876564|4568|36172|27313|876564|4568|36172|4|64|5|1|14812|223|614|83|58.66|147.82|45.82|8466.00|3803.06|4868.78|12269.06|190.15|0.00|981.06|3803.06|3993.21|4784.12|4974.27|-1065.72| +2450827|12371|2450914|27313|876564|4568|36172|27313|876564|4568|36172|4|59|14|4|7256|272|614|15|71.45|90.02|50.41|594.15|756.15|1071.75|1350.30|7.56|0.00|634.50|756.15|763.71|1390.65|1398.21|-315.60| +2450827|12371|2450898|27313|876564|4568|36172|27313|876564|4568|36172|4|37|2|2|9706|39|614|5|67.86|166.25|39.90|631.75|199.50|339.30|831.25|0.00|0.00|49.85|199.50|199.50|249.35|249.35|-139.80| +2450827|12371|2450878|27313|876564|4568|36172|27313|876564|4568|36172|4|67|2|4|15722|260|614|12|22.98|24.12|22.67|17.40|272.04|275.76|289.44|5.44|0.00|63.60|272.04|277.48|335.64|341.08|-3.72| +2450827|12371|2450880|27313|876564|4568|36172|27313|876564|4568|36172|4|23|15|1|6632|30|614|50|22.35|31.73|14.27|873.00|713.50|1117.50|1586.50|57.08|0.00|713.50|713.50|770.58|1427.00|1484.08|-404.00| +2450827|12371|2450900|27313|876564|4568|36172|27313|876564|4568|36172|4|104|15|3|2482|197|614|14|97.66|229.50|229.50|0.00|3213.00|1367.24|3213.00|0.00|0.00|1028.16|3213.00|3213.00|4241.16|4241.16|1845.76| +2450827|12371|2450895|27313|876564|4568|36172|27313|876564|4568|36172|4|59|4|4|10603|205|614|93|45.93|123.09|54.15|6411.42|5035.95|4271.49|11447.37|352.51|0.00|3433.56|5035.95|5388.46|8469.51|8822.02|764.46| +2450827|12371|2450829|27313|876564|4568|36172|27313|876564|4568|36172|4|10|8|3|16105|9|614|62|94.75|280.46|33.65|15302.22|2086.30|5874.50|17388.52|2.50|1835.94|6780.94|250.36|252.86|7031.30|7033.80|-5624.14| +2450827|12371|2450887|27313|876564|4568|36172|27313|876564|4568|36172|4|22|3|2|9880|52|614|47|16.09|34.59|33.55|48.88|1576.85|756.23|1625.73|126.14|0.00|178.60|1576.85|1702.99|1755.45|1881.59|820.62| +2450827|12371|2450845|27313|876564|4568|36172|27313|876564|4568|36172|4|87|17|2|1099|252|614|38|22.92|63.48|15.87|1809.18|603.06|870.96|2412.24|30.15|0.00|458.28|603.06|633.21|1061.34|1091.49|-267.90| +2450827|12371|2450852|27313|876564|4568|36172|27313|876564|4568|36172|4|13|4|4|14774|181|614|69|72.33|130.19|65.09|4491.90|4491.21|4990.77|8983.11|134.73|0.00|2334.96|4491.21|4625.94|6826.17|6960.90|-499.56| +2450827|12371|2450912|27313|876564|4568|36172|27313|876564|4568|36172|4|94|12|5|5906|188|614|37|26.32|72.64|71.91|27.01|2660.67|973.84|2687.68|42.57|1596.40|429.94|1064.27|1106.84|1494.21|1536.78|90.43| +2450827|12371|2450855|27313|876564|4568|36172|27313|876564|4568|36172|4|26|17|5|2156|116|614|43|81.85|153.87|3.07|6484.40|132.01|3519.55|6616.41|9.24|0.00|2844.88|132.01|141.25|2976.89|2986.13|-3387.54| +2450827|12371|2450865|27313|876564|4568|36172|27313|876564|4568|36172|4|16|2|4|6326|68|614|67|13.24|19.19|8.63|707.52|578.21|887.08|1285.73|28.91|0.00|115.24|578.21|607.12|693.45|722.36|-308.87| +2450827|14275|2450887|88733|202813|5693|35373|88733|202813|5693|35373|1|64|5|4|15976|160|615|36|6.91|13.75|11.55|79.20|415.80|248.76|495.00|29.18|91.47|44.28|324.33|353.51|368.61|397.79|75.57| +2450827|14275|2450870|88733|202813|5693|35373|88733|202813|5693|35373|1|40|12|2|532|167|615|38|46.75|99.11|28.74|2674.06|1092.12|1776.50|3766.18|87.36|0.00|0.00|1092.12|1179.48|1092.12|1179.48|-684.38| +2450827|14275|2450890|88733|202813|5693|35373|88733|202813|5693|35373|1|23|10|2|17950|287|615|80|51.56|90.23|45.11|3609.60|3608.80|4124.80|7218.40|180.44|0.00|3536.80|3608.80|3789.24|7145.60|7326.04|-516.00| +2450827|14275|2450896|88733|202813|5693|35373|88733|202813|5693|35373|1|98|12|5|8132|114|615|35|16.82|42.72|16.23|927.15|568.05|588.70|1495.20|5.68|0.00|493.15|568.05|573.73|1061.20|1066.88|-20.65| +2450827|14275|2450861|88733|202813|5693|35373|88733|202813|5693|35373|1|101|10|3|7592|109|615|99|39.56|39.95|25.56|1424.61|2530.44|3916.44|3955.05|135.63|835.04|395.01|1695.40|1831.03|2090.41|2226.04|-2221.04| +2450827|14275|2450885|88733|202813|5693|35373|88733|202813|5693|35373|1|42|3|3|5941|232|615|5|30.41|38.31|33.32|24.95|166.60|152.05|191.55|9.99|0.00|63.20|166.60|176.59|229.80|239.79|14.55| +2450827|14275|2450883|88733|202813|5693|35373|88733|202813|5693|35373|1|18|15|4|7975|35|615|41|7.32|7.61|5.85|72.16|239.85|300.12|312.01|2.39|0.00|109.06|239.85|242.24|348.91|351.30|-60.27| +2450827|14275|2450856|88733|202813|5693|35373|88733|202813|5693|35373|1|13|5|4|11912|188|615|92|40.85|61.27|55.14|563.96|5072.88|3758.20|5636.84|456.55|0.00|2028.60|5072.88|5529.43|7101.48|7558.03|1314.68| +2450827|14275|2450854|88733|202813|5693|35373|88733|202813|5693|35373|1|79|12|4|4346|1|615|38|62.65|117.15|80.83|1380.16|3071.54|2380.70|4451.70|215.00|0.00|133.38|3071.54|3286.54|3204.92|3419.92|690.84| +2450827|14275|2450834|88733|202813|5693|35373|88733|202813|5693|35373|1|26|19|1|17761|283|615|48|98.90|284.83|233.56|2460.96|11210.88|4747.20|13671.84|190.58|7399.18|819.84|3811.70|4002.28|4631.54|4822.12|-935.50| +2450827|14275|2450873|88733|202813|5693|35373|88733|202813|5693|35373|1|69|6|5|14804|48|615|90|63.06|157.65|4.72|13763.70|424.80|5675.40|14188.50|8.49|0.00|6810.30|424.80|433.29|7235.10|7243.59|-5250.60| +2450827|14275|2450903|88733|202813|5693|35373|88733|202813|5693|35373|1|72|2|5|5804|53|615|48|85.26|133.85|66.92|3212.64|3212.16|4092.48|6424.80|96.36|0.00|321.12|3212.16|3308.52|3533.28|3629.64|-880.32| +2450827|14275|2450914|88733|202813|5693|35373|88733|202813|5693|35373|1|58|19|4|14590|292|615|39|49.58|88.25|35.30|2065.05|1376.70|1933.62|3441.75|41.30|0.00|447.33|1376.70|1418.00|1824.03|1865.33|-556.92| +2450827|14275|2450875|88733|202813|5693|35373|88733|202813|5693|35373|1|88|5|2|14578|40|615|31|86.73|169.12|62.57|3303.05|1939.67|2688.63|5242.72|38.79|0.00|2306.71|1939.67|1978.46|4246.38|4285.17|-748.96| +2450827|72842|2450842|49156|731314|6625|46783|49156|731314|6625|46783|1|25|13|3|13543|86|616|6|65.32|141.74|110.55|187.14|663.30|391.92|850.44|43.51|119.39|144.54|543.91|587.42|688.45|731.96|151.99| +2450827|72842|2450857|49156|731314|6625|46783|49156|731314|6625|46783|1|76|17|1|17371|191|616|53|2.66|5.00|1.80|169.60|95.40|140.98|265.00|6.67|0.00|66.25|95.40|102.07|161.65|168.32|-45.58| +2450827|72842|2450892|49156|731314|6625|46783|49156|731314|6625|46783|1|11|6|5|3308|292|616|54|62.12|139.77|127.19|679.32|6868.26|3354.48|7547.58|480.77|0.00|150.66|6868.26|7349.03|7018.92|7499.69|3513.78| +2450827|72842|2450878|49156|731314|6625|46783|49156|731314|6625|46783|1|12|14|2|16519|28|616|67|1.99|4.55|1.68|192.29|112.56|133.33|304.85|3.37|0.00|97.15|112.56|115.93|209.71|213.08|-20.77| +2450827|72842|2450862|49156|731314|6625|46783|49156|731314|6625|46783|1|53|10|2|8686|131|616|83|9.02|23.72|5.45|1516.41|452.35|748.66|1968.76|13.57|0.00|39.01|452.35|465.92|491.36|504.93|-296.31| +2450827|75836|2450850|54405|1596524|243|24762|54405|1596524|243|24762|2|54|5|2|10789|246|617|24|65.35|98.67|51.30|1136.88|1231.20|1568.40|2368.08|28.56|874.15|876.00|357.05|385.61|1233.05|1261.61|-1211.35| +2450827|75836|2450889|54405|1596524|243|24762|54405|1596524|243|24762|2|6|15|4|12818|39|617|2|92.73|190.09|152.07|76.04|304.14|185.46|380.18|21.28|0.00|110.24|304.14|325.42|414.38|435.66|118.68| +2450827|75836|2450831|54405|1596524|243|24762|54405|1596524|243|24762|2|100|10|2|16954|79|617|26|50.72|100.93|39.36|1600.82|1023.36|1318.72|2624.18|10.23|0.00|52.26|1023.36|1033.59|1075.62|1085.85|-295.36| +2450827|75836|2450893|54405|1596524|243|24762|54405|1596524|243|24762|2|93|17|1|14404|88|617|77|74.80|83.02|1.66|6264.72|127.82|5759.60|6392.54|8.94|0.00|255.64|127.82|136.76|383.46|392.40|-5631.78| +2450827|75836|2450865|54405|1596524|243|24762|54405|1596524|243|24762|2|40|3|5|17239|141|617|54|37.21|57.30|20.05|2011.50|1082.70|2009.34|3094.20|32.48|0.00|1237.68|1082.70|1115.18|2320.38|2352.86|-926.64| +2450827|75836|2450878|54405|1596524|243|24762|54405|1596524|243|24762|2|53|17|3|14023|115|617|22|3.08|6.62|0.39|137.06|8.58|67.76|145.64|0.08|0.00|8.58|8.58|8.66|17.16|17.24|-59.18| +2450827|35384|2450881|25474|417052|4983|43314|25474|417052|4983|43314|1|101|14|1|8062|134|618|51|30.35|56.45|1.12|2821.83|57.12|1547.85|2878.95|4.56|0.00|287.64|57.12|61.68|344.76|349.32|-1490.73| +2450827|35384|2450856|25474|417052|4983|43314|25474|417052|4983|43314|1|59|17|1|4237|137|618|2|70.62|127.82|1.27|253.10|2.54|141.24|255.64|0.07|0.00|127.82|2.54|2.61|130.36|130.43|-138.70| +2450827|35384|2450885|25474|417052|4983|43314|25474|417052|4983|43314|1|38|9|3|7621|48|618|6|2.74|6.71|3.22|20.94|19.32|16.44|40.26|0.38|0.00|12.06|19.32|19.70|31.38|31.76|2.88| +2450827|35384|2450891|25474|417052|4983|43314|25474|417052|4983|43314|1|93|9|1|7051|114|618|27|74.14|163.84|103.21|1637.01|2786.67|2001.78|4423.68|27.86|0.00|928.80|2786.67|2814.53|3715.47|3743.33|784.89| +2450827|35384|2450864|25474|417052|4983|43314|25474|417052|4983|43314|1|98|18|3|3160|281|618|7|56.96|78.60|40.87|264.11|286.09|398.72|550.20|5.72|0.00|236.53|286.09|291.81|522.62|528.34|-112.63| +2450827|35384|2450847|25474|417052|4983|43314|25474|417052|4983|43314|1|16|4|5|1072|270|618|40|57.16|65.73|44.03|868.00|1761.20|2286.40|2629.20|0.00|0.00|867.60|1761.20|1761.20|2628.80|2628.80|-525.20| +2450827|35384|2450842|25474|417052|4983|43314|25474|417052|4983|43314|1|82|2|4|16276|88|618|33|42.41|63.19|48.65|479.82|1605.45|1399.53|2085.27|0.00|0.00|437.58|1605.45|1605.45|2043.03|2043.03|205.92| +2450827|35384|2450890|25474|417052|4983|43314|25474|417052|4983|43314|1|48|11|5|10864|145|618|63|2.39|5.11|4.24|54.81|267.12|150.57|321.93|2.67|0.00|51.03|267.12|269.79|318.15|320.82|116.55| +2450827|68442|2450883|69953|873461|1769|36077|69953|873461|1769|36077|2|10|5|3|4958|84|619|42|94.87|113.84|56.92|2390.64|2390.64|3984.54|4781.28|23.90|0.00|716.94|2390.64|2414.54|3107.58|3131.48|-1593.90| +2450827|68442|2450876|69953|873461|1769|36077|69953|873461|1769|36077|2|87|5|2|2326|290|619|95|70.39|207.65|39.45|15979.00|3747.75|6687.05|19726.75|37.47|0.00|4142.00|3747.75|3785.22|7889.75|7927.22|-2939.30| +2450827|68442|2450896|69953|873461|1769|36077|69953|873461|1769|36077|2|36|17|1|6955|207|619|86|1.72|1.85|1.29|48.16|110.94|147.92|159.10|4.43|0.00|21.50|110.94|115.37|132.44|136.87|-36.98| +2450827|68442|2450909|69953|873461|1769|36077|69953|873461|1769|36077|2|34|5|3|9127|77|619|26|43.95|76.47|18.35|1511.12|477.10|1142.70|1988.22|33.39|0.00|59.54|477.10|510.49|536.64|570.03|-665.60| +2450827|68442|2450857|69953|873461|1769|36077|69953|873461|1769|36077|2|86|20|1|17828|263|619|29|56.93|83.11|0.83|2386.12|24.07|1650.97|2410.19|1.92|0.00|867.39|24.07|25.99|891.46|893.38|-1626.90| +2450827|68442|2450917|69953|873461|1769|36077|69953|873461|1769|36077|2|19|16|2|6829|56|619|57|51.28|90.25|57.76|1851.93|3292.32|2922.96|5144.25|164.61|0.00|1131.45|3292.32|3456.93|4423.77|4588.38|369.36| +2450827|74561|2450916|30249|1161762|5628|8694|30249|1161762|5628|8694|2|51|11|4|11083|135|620|55|44.19|58.77|28.79|1648.90|1583.45|2430.45|3232.35|53.83|506.70|1066.45|1076.75|1130.58|2143.20|2197.03|-1353.70| +2450827|74561|2450854|30249|1161762|5628|8694|30249|1161762|5628|8694|2|60|8|4|6193|288|620|56|3.26|4.20|1.68|141.12|94.08|182.56|235.20|4.70|0.00|61.04|94.08|98.78|155.12|159.82|-88.48| +2450827|74561|2450834|30249|1161762|5628|8694|30249|1161762|5628|8694|2|56|3|3|6163|100|620|9|69.51|75.07|60.80|128.43|547.20|625.59|675.63|20.02|213.40|337.77|333.80|353.82|671.57|691.59|-291.79| +2450827|74561|2450844|30249|1161762|5628|8694|30249|1161762|5628|8694|2|49|18|3|2090|78|620|97|44.29|112.93|83.56|2848.89|8105.32|4296.13|10954.21|194.52|4863.19|4491.10|3242.13|3436.65|7733.23|7927.75|-1054.00| +2450827|74561|2450889|30249|1161762|5628|8694|30249|1161762|5628|8694|2|58|12|1|10393|111|620|7|90.22|193.07|108.11|594.72|756.77|631.54|1351.49|60.54|0.00|445.97|756.77|817.31|1202.74|1263.28|125.23| +2450827|74561|2450832|30249|1161762|5628|8694|30249|1161762|5628|8694|2|33|6|5|10712|148|620|64|33.13|43.73|27.11|1063.68|1735.04|2120.32|2798.72|2.94|1440.08|1035.52|294.96|297.90|1330.48|1333.42|-1825.36| +2450827|74561|2450885|30249|1161762|5628|8694|30249|1161762|5628|8694|2|52|14|2|10504|60|620|50|91.04|97.41|83.77|682.00|4188.50|4552.00|4870.50|293.19|0.00|2289.00|4188.50|4481.69|6477.50|6770.69|-363.50| +2450827|74561|2450841|30249|1161762|5628|8694|30249|1161762|5628|8694|2|103|14|3|10651|296|620|36|2.24|3.87|0.03|138.24|1.08|80.64|139.32|0.09|0.00|6.84|1.08|1.17|7.92|8.01|-79.56| +2450827|74561|2450899|30249|1161762|5628|8694|30249|1161762|5628|8694|2|108|13|1|10382|291|620|77|37.31|66.78|65.44|103.18|5038.88|2872.87|5142.06|302.33|0.00|0.00|5038.88|5341.21|5038.88|5341.21|2166.01| +2450827|74561|2450846|30249|1161762|5628|8694|30249|1161762|5628|8694|2|46|10|2|5335|253|620|97|72.64|212.10|65.75|14195.95|6377.75|7046.08|20573.70|255.11|0.00|6789.03|6377.75|6632.86|13166.78|13421.89|-668.33| +2450827|74561|2450857|30249|1161762|5628|8694|30249|1161762|5628|8694|2|5|16|1|8492|126|620|17|80.97|141.69|140.27|24.14|2384.59|1376.49|2408.73|71.53|0.00|722.50|2384.59|2456.12|3107.09|3178.62|1008.10| +2450827|74561|2450841|30249|1161762|5628|8694|30249|1161762|5628|8694|2|97|1|4|11959|275|620|80|10.97|13.05|7.17|470.40|573.60|877.60|1044.00|11.47|0.00|250.40|573.60|585.07|824.00|835.47|-304.00| +2450827|74561|2450863|30249|1161762|5628|8694|30249|1161762|5628|8694|2|8|13|4|13138|71|620|48|84.21|236.63|179.83|2726.40|8631.84|4042.08|11358.24|172.63|0.00|3861.60|8631.84|8804.47|12493.44|12666.07|4589.76| +2450827|74561|2450903|30249|1161762|5628|8694|30249|1161762|5628|8694|2|58|12|1|9157|267|620|15|41.77|53.88|53.34|8.10|800.10|626.55|808.20|40.00|0.00|298.95|800.10|840.10|1099.05|1139.05|173.55| +2450827|16850|2450883|58932|246261|5143|34072|55141|28283|7165|28494|1|104|4|2|3998|46|621|64|15.36|44.69|42.00|172.16|2688.00|983.04|2860.16|134.40|0.00|1344.00|2688.00|2822.40|4032.00|4166.40|1704.96| +2450827|16850|2450857|58932|246261|5143|34072|55141|28283|7165|28494|1|7|16|1|7498|51|621|28|88.68|107.30|81.54|721.28|2283.12|2483.04|3004.40|36.98|433.79|1351.84|1849.33|1886.31|3201.17|3238.15|-633.71| +2450827|16850|2450867|58932|246261|5143|34072|55141|28283|7165|28494|1|20|13|2|10922|65|621|50|92.68|237.26|128.12|5457.00|6406.00|4634.00|11863.00|576.54|0.00|5575.50|6406.00|6982.54|11981.50|12558.04|1772.00| +2450827|16850|2450880|58932|246261|5143|34072|55141|28283|7165|28494|1|61|20|4|11714|97|621|3|57.21|165.90|91.24|223.98|273.72|171.63|497.70|16.42|0.00|29.85|273.72|290.14|303.57|319.99|102.09| +2450827|16850|2450860|58932|246261|5143|34072|55141|28283|7165|28494|1|47|1|2|7436|14|621|85|43.84|109.60|20.82|7546.30|1769.70|3726.40|9316.00|123.87|0.00|2049.35|1769.70|1893.57|3819.05|3942.92|-1956.70| +2450827|16850|2450913|58932|246261|5143|34072|55141|28283|7165|28494|1|58|10|1|2713|285|621|100|75.28|94.85|60.70|3415.00|6070.00|7528.00|9485.00|84.98|4856.00|4268.00|1214.00|1298.98|5482.00|5566.98|-6314.00| +2450827|16850|2450883|58932|246261|5143|34072|55141|28283|7165|28494|1|26|14|1|12506|239|621|5|44.94|61.11|23.22|189.45|116.10|224.70|305.55|6.96|0.00|100.80|116.10|123.06|216.90|223.86|-108.60| +2450827|16850|2450904|58932|246261|5143|34072|55141|28283|7165|28494|1|49|19|2|14282|168|621|68|1.17|2.08|1.78|20.40|121.04|79.56|141.44|1.21|0.00|60.52|121.04|122.25|181.56|182.77|41.48| +2450827|16850|2450855|58932|246261|5143|34072|55141|28283|7165|28494|1|61|5|3|3770|272|621|44|14.51|35.25|2.46|1442.76|108.24|638.44|1551.00|2.16|0.00|666.60|108.24|110.40|774.84|777.00|-530.20| +2450827|16850|2450845|58932|246261|5143|34072|55141|28283|7165|28494|1|1|18|3|15278|100|621|26|34.83|95.78|22.02|1917.76|572.52|905.58|2490.28|45.80|0.00|597.48|572.52|618.32|1170.00|1215.80|-333.06| +2450827|16850|2450887|58932|246261|5143|34072|55141|28283|7165|28494|1|97|5|5|1339|249|621|45|98.64|112.44|84.33|1264.95|3794.85|4438.80|5059.80|265.63|0.00|1872.00|3794.85|4060.48|5666.85|5932.48|-643.95| +2450827|16850|2450915|58932|246261|5143|34072|55141|28283|7165|28494|1|37|12|4|3922|41|621|31|2.24|3.42|2.18|38.44|67.58|69.44|106.02|0.00|0.00|49.60|67.58|67.58|117.18|117.18|-1.86| +2450827|35239|2450889|8200|706755|3710|31465|8200|706755|3710|31465|1|42|3|3|1184|142|622|64|88.22|236.42|61.46|11197.44|3933.44|5646.08|15130.88|51.92|2202.72|1058.56|1730.72|1782.64|2789.28|2841.20|-3915.36| +2450827|35239|2450851|8200|706755|3710|31465|8200|706755|3710|31465|1|67|11|5|15160|241|622|52|86.56|198.22|57.48|7318.48|2988.96|4501.12|10307.44|41.24|2301.49|927.16|687.47|728.71|1614.63|1655.87|-3813.65| +2450827|35239|2450857|8200|706755|3710|31465|8200|706755|3710|31465|1|8|14|5|3154|209|622|85|28.73|60.33|19.30|3487.55|1640.50|2442.05|5128.05|49.21|0.00|1743.35|1640.50|1689.71|3383.85|3433.06|-801.55| +2450827|35239|2450903|8200|706755|3710|31465|8200|706755|3710|31465|1|26|1|5|5560|3|622|36|8.91|13.54|9.07|160.92|326.52|320.76|487.44|19.59|0.00|73.08|326.52|346.11|399.60|419.19|5.76| +2450827|35239|2450888|8200|706755|3710|31465|8200|706755|3710|31465|1|57|9|4|11764|19|622|63|49.06|94.68|60.59|2147.67|3817.17|3090.78|5964.84|190.85|0.00|1670.13|3817.17|4008.02|5487.30|5678.15|726.39| +2450827|35239|2450839|8200|706755|3710|31465|8200|706755|3710|31465|1|19|2|3|13414|235|622|28|66.22|66.88|48.82|505.68|1366.96|1854.16|1872.64|109.35|0.00|823.76|1366.96|1476.31|2190.72|2300.07|-487.20| +2450827|57362|2450895|5470|1742209|3395|10934|5470|1742209|3395|10934|2|90|5|2|482|143|623|32|93.82|133.22|126.55|213.44|4049.60|3002.24|4263.04|242.97|0.00|1747.84|4049.60|4292.57|5797.44|6040.41|1047.36| +2450827|57362|2450863|5470|1742209|3395|10934|5470|1742209|3395|10934|2|45|5|1|4789|173|623|47|15.28|31.47|17.93|636.38|842.71|718.16|1479.09|25.28|0.00|458.25|842.71|867.99|1300.96|1326.24|124.55| +2450827|57362|2450907|5470|1742209|3395|10934|5470|1742209|3395|10934|2|20|19|3|11204|44|623|59|61.54|171.08|130.02|2422.54|7671.18|3630.86|10093.72|0.00|6520.50|2018.39|1150.68|1150.68|3169.07|3169.07|-2480.18| +2450827|57362|2450899|5470|1742209|3395|10934|5470|1742209|3395|10934|2|86|8|3|15979|253|623|58|80.39|86.01|24.94|3542.06|1446.52|4662.62|4988.58|0.00|0.00|1147.24|1446.52|1446.52|2593.76|2593.76|-3216.10| +2450827|57362|2450898|5470|1742209|3395|10934|5470|1742209|3395|10934|2|25|2|2|17467|211|623|9|58.11|112.73|32.69|720.36|294.21|522.99|1014.57|2.94|0.00|446.40|294.21|297.15|740.61|743.55|-228.78| +2450827|76938|2450886|76576|1594628|2727|34779|76576|1594628|2727|34779|2|107|9|5|17494|45|624|84|14.71|15.59|5.92|812.28|497.28|1235.64|1309.56|14.91|0.00|195.72|497.28|512.19|693.00|707.91|-738.36| +2450827|76938|2450886|76576|1594628|2727|34779|76576|1594628|2727|34779|2|2|10|1|5006|76|624|22|95.41|141.20|67.77|1615.46|1490.94|2099.02|3106.40|0.00|0.00|1180.30|1490.94|1490.94|2671.24|2671.24|-608.08| +2450827|76938|2450852|76576|1594628|2727|34779|76576|1594628|2727|34779|2|13|12|4|13633|83|624|75|3.77|8.48|2.03|483.75|152.25|282.75|636.00|10.65|0.00|82.50|152.25|162.90|234.75|245.40|-130.50| +2450827|76938|2450908|76576|1594628|2727|34779|76576|1594628|2727|34779|2|38|12|4|14953|21|624|66|61.99|168.61|101.16|4451.70|6676.56|4091.34|11128.26|600.89|0.00|2781.90|6676.56|7277.45|9458.46|10059.35|2585.22| +2450827|76938|2450868|76576|1594628|2727|34779|76576|1594628|2727|34779|2|78|16|1|12643|116|624|13|36.99|75.82|3.03|946.27|39.39|480.87|985.66|0.00|0.00|98.54|39.39|39.39|137.93|137.93|-441.48| +2450827|76938|2450841|76576|1594628|2727|34779|76576|1594628|2727|34779|2|103|10|4|8674|237|624|19|6.88|19.47|11.87|144.40|225.53|130.72|369.93|1.71|54.12|170.05|171.41|173.12|341.46|343.17|40.69| +2450827|76938|2450912|76576|1594628|2727|34779|76576|1594628|2727|34779|2|24|2|5|7642|222|624|53|4.79|10.72|9.54|62.54|505.62|253.87|568.16|5.05|0.00|142.04|505.62|510.67|647.66|652.71|251.75| +2450827|76938|2450906|76576|1594628|2727|34779|76576|1594628|2727|34779|2|98|17|3|14311|7|624|21|83.54|221.38|203.66|372.12|4276.86|1754.34|4648.98|299.38|0.00|278.88|4276.86|4576.24|4555.74|4855.12|2522.52| +2450827|76938|2450902|76576|1594628|2727|34779|76576|1594628|2727|34779|2|2|12|4|11161|18|624|32|16.76|16.76|14.24|80.64|455.68|536.32|536.32|0.45|410.11|128.64|45.57|46.02|174.21|174.66|-490.75| +2450827|76938|2450835|76576|1594628|2727|34779|76576|1594628|2727|34779|2|55|8|1|16258|137|624|99|7.44|16.36|4.90|1134.54|485.10|736.56|1619.64|29.10|0.00|631.62|485.10|514.20|1116.72|1145.82|-251.46| +2450827|76938|2450877|76576|1594628|2727|34779|76576|1594628|2727|34779|2|68|14|3|784|129|624|12|37.28|76.42|48.14|339.36|577.68|447.36|917.04|28.88|0.00|27.48|577.68|606.56|605.16|634.04|130.32| +2450827|76938|2450850|76576|1594628|2727|34779|76576|1594628|2727|34779|2|56|1|5|1525|231|624|95|74.55|127.48|93.06|3269.90|8840.70|7082.25|12110.60|0.00|2917.43|5328.55|5923.27|5923.27|11251.82|11251.82|-1158.98| +2450827|76938|2450884|76576|1594628|2727|34779|76576|1594628|2727|34779|2|85|16|2|9295|281|624|63|85.68|179.07|96.69|5189.94|6091.47|5397.84|11281.41|487.31|0.00|1578.78|6091.47|6578.78|7670.25|8157.56|693.63| +2450827|66374|2450877|14634|1260589|1391|24546|14634|1260589|1391|24546|1|6|11|2|16652|54|625|80|7.11|18.77|10.13|691.20|810.40|568.80|1501.60|64.83|0.00|555.20|810.40|875.23|1365.60|1430.43|241.60| +2450827|66374|2450882|14634|1260589|1391|24546|14634|1260589|1391|24546|1|47|18|2|15286|247|625|67|2.98|3.33|2.09|83.08|140.03|199.66|223.11|8.40|0.00|44.22|140.03|148.43|184.25|192.65|-59.63| +2450827|66374|2450887|14634|1260589|1391|24546|14634|1260589|1391|24546|1|2|20|5|9400|22|625|31|86.01|109.23|84.10|779.03|2607.10|2666.31|3386.13|61.78|547.49|338.52|2059.61|2121.39|2398.13|2459.91|-606.70| +2450827|66374|2450836|14634|1260589|1391|24546|14634|1260589|1391|24546|1|103|15|2|10999|171|625|51|57.83|163.65|67.09|4924.56|3421.59|2949.33|8346.15|171.07|0.00|2336.82|3421.59|3592.66|5758.41|5929.48|472.26| +2450827|66374|2450889|14634|1260589|1391|24546|14634|1260589|1391|24546|1|43|8|1|10562|39|625|78|10.41|23.52|14.81|679.38|1155.18|811.98|1834.56|11.55|0.00|439.92|1155.18|1166.73|1595.10|1606.65|343.20| +2450827|66374|2450874|14634|1260589|1391|24546|14634|1260589|1391|24546|1|74|6|2|11863|21|625|50|43.83|117.90|16.50|5070.00|825.00|2191.50|5895.00|8.25|0.00|589.50|825.00|833.25|1414.50|1422.75|-1366.50| +2450827|66374|2450878|14634|1260589|1391|24546|14634|1260589|1391|24546|1|100|1|2|6844|67|625|54|99.37|168.92|92.90|4105.08|5016.60|5365.98|9121.68|50.16|0.00|912.06|5016.60|5066.76|5928.66|5978.82|-349.38| +2450827|47137|2450846|84586|870126|2066|27606|72836|522686|3426|34966|2|94|16|2|2456|264|626|29|61.49|154.95|102.26|1528.01|2965.54|1783.21|4493.55|118.62|0.00|1842.08|2965.54|3084.16|4807.62|4926.24|1182.33| +2450827|47137|2450860|84586|870126|2066|27606|72836|522686|3426|34966|2|45|18|5|2018|144|626|62|32.14|77.77|27.21|3134.72|1687.02|1992.68|4821.74|134.96|0.00|2217.74|1687.02|1821.98|3904.76|4039.72|-305.66| +2450827|47137|2450904|84586|870126|2066|27606|72836|522686|3426|34966|2|60|1|4|8660|210|626|21|37.82|52.94|50.82|44.52|1067.22|794.22|1111.74|0.00|0.00|522.48|1067.22|1067.22|1589.70|1589.70|273.00| +2450827|47137|2450855|84586|870126|2066|27606|72836|522686|3426|34966|2|105|13|5|13852|96|626|19|65.28|93.35|70.01|443.46|1330.19|1240.32|1773.65|26.60|0.00|230.47|1330.19|1356.79|1560.66|1587.26|89.87| +2450827|47137|2450847|84586|870126|2066|27606|72836|522686|3426|34966|2|96|9|5|6115|102|626|81|78.67|78.67|71.58|574.29|5797.98|6372.27|6372.27|463.83|0.00|573.48|5797.98|6261.81|6371.46|6835.29|-574.29| +2450827|47137|2450861|84586|870126|2066|27606|72836|522686|3426|34966|2|3|4|5|17137|224|626|38|35.25|84.24|26.95|2177.02|1024.10|1339.50|3201.12|40.96|0.00|416.10|1024.10|1065.06|1440.20|1481.16|-315.40| +2450827|47137|2450833|84586|870126|2066|27606|72836|522686|3426|34966|2|47|7|5|13078|118|626|8|82.08|245.41|78.53|1335.04|628.24|656.64|1963.28|18.84|0.00|255.20|628.24|647.08|883.44|902.28|-28.40| +2450827|47137|2450902|84586|870126|2066|27606|72836|522686|3426|34966|2|50|6|2|8630|162|626|27|52.85|65.53|31.45|920.16|849.15|1426.95|1769.31|42.45|0.00|406.89|849.15|891.60|1256.04|1298.49|-577.80| +2450827|47137|2450833|84586|870126|2066|27606|72836|522686|3426|34966|2|94|13|3|9283|232|626|54|49.03|141.20|4.23|7396.38|228.42|2647.62|7624.80|0.00|0.00|76.14|228.42|228.42|304.56|304.56|-2419.20| +2450827|47137|2450903|84586|870126|2066|27606|72836|522686|3426|34966|2|87|11|2|15034|259|626|24|29.77|56.86|44.35|300.24|1064.40|714.48|1364.64|10.64|0.00|27.12|1064.40|1075.04|1091.52|1102.16|349.92| +2450827|46457|2450869|4393|598514|188|662|87290|1894457|131|1452|1|97|20|5|8212|270|627|16|77.91|101.28|65.83|567.20|1053.28|1246.56|1620.48|21.06|0.00|307.84|1053.28|1074.34|1361.12|1382.18|-193.28| +2450827|46457|2450859|4393|598514|188|662|87290|1894457|131|1452|1|39|19|5|12046|234|627|70|25.73|50.94|32.09|1319.50|2246.30|1801.10|3565.80|202.16|0.00|284.90|2246.30|2448.46|2531.20|2733.36|445.20| +2450827|46457|2450892|4393|598514|188|662|87290|1894457|131|1452|1|4|1|4|1873|252|627|99|7.84|18.97|11.00|789.03|1089.00|776.16|1878.03|98.01|0.00|769.23|1089.00|1187.01|1858.23|1956.24|312.84| +2450827|46457|2450895|4393|598514|188|662|87290|1894457|131|1452|1|53|11|4|241|22|627|6|26.99|80.97|48.58|194.34|291.48|161.94|485.82|26.23|0.00|82.56|291.48|317.71|374.04|400.27|129.54| +2450827|46457|2450836|4393|598514|188|662|87290|1894457|131|1452|1|50|7|4|16006|161|627|93|7.82|22.91|21.99|85.56|2045.07|727.26|2130.63|143.15|0.00|574.74|2045.07|2188.22|2619.81|2762.96|1317.81| +2450827|46457|2450844|4393|598514|188|662|87290|1894457|131|1452|1|83|3|3|5161|123|627|90|95.78|221.25|108.41|10155.60|9756.90|8620.20|19912.50|390.27|0.00|4380.30|9756.90|10147.17|14137.20|14527.47|1136.70| +2450827|46457|2450903|4393|598514|188|662|87290|1894457|131|1452|1|10|13|4|9590|46|627|99|80.70|90.38|46.99|4295.61|4652.01|7989.30|8947.62|46.52|0.00|3936.24|4652.01|4698.53|8588.25|8634.77|-3337.29| +2450827|46457|2450864|4393|598514|188|662|87290|1894457|131|1452|1|37|4|2|3194|165|627|55|23.09|46.18|20.78|1397.00|1142.90|1269.95|2539.90|0.00|0.00|939.40|1142.90|1142.90|2082.30|2082.30|-127.05| +2450827|46457|2450905|4393|598514|188|662|87290|1894457|131|1452|1|4|7|4|1832|198|627|86|18.80|43.42|26.05|1493.82|2240.30|1616.80|3734.12|22.40|0.00|1455.98|2240.30|2262.70|3696.28|3718.68|623.50| +2450827|46457|2450899|4393|598514|188|662|87290|1894457|131|1452|1|60|6|3|1570|184|627|23|18.35|20.18|6.05|324.99|139.15|422.05|464.14|1.39|0.00|106.72|139.15|140.54|245.87|247.26|-282.90| +2450827|46457|2450867|4393|598514|188|662|87290|1894457|131|1452|1|73|1|1|16126|286|627|83|52.90|149.70|56.88|7704.06|4721.04|4390.70|12425.10|377.68|0.00|248.17|4721.04|5098.72|4969.21|5346.89|330.34| +2450827|46457|2450840|4393|598514|188|662|87290|1894457|131|1452|1|11|16|2|3049|246|627|40|93.46|272.90|43.66|9169.60|1746.40|3738.40|10916.00|87.32|0.00|1528.00|1746.40|1833.72|3274.40|3361.72|-1992.00| +2450827|46457|2450902|4393|598514|188|662|87290|1894457|131|1452|1|51|9|4|15175|131|627|72|34.77|49.37|42.95|462.24|3092.40|2503.44|3554.64|53.18|432.93|1527.84|2659.47|2712.65|4187.31|4240.49|156.03| +2450827|46457|2450878|4393|598514|188|662|87290|1894457|131|1452|1|28|9|2|2362|285|627|35|18.22|34.25|14.72|683.55|515.20|637.70|1198.75|20.60|0.00|23.80|515.20|535.80|539.00|559.60|-122.50| +2450827|59834|2450897|39079|421457|635|1413|39079|421457|635|1413|4|13|9|2|163|235|628|95|91.24|223.53|73.76|14228.15|7007.20|8667.80|21235.35|280.28|0.00|0.00|7007.20|7287.48|7007.20|7287.48|-1660.60| +2450827|59834|2450899|39079|421457|635|1413|39079|421457|635|1413|4|96|13|2|12235|40|628|73|38.78|41.49|0.00|3028.77|0.00|2830.94|3028.77|0.00|0.00|423.40|0.00|0.00|423.40|423.40|-2830.94| +2450827|59834|2450860|39079|421457|635|1413|39079|421457|635|1413|4|5|13|1|706|300|628|53|91.29|169.79|56.03|6029.28|2969.59|4838.37|8998.87|0.00|0.00|2609.19|2969.59|2969.59|5578.78|5578.78|-1868.78| +2450827|59834|2450909|39079|421457|635|1413|39079|421457|635|1413|4|91|2|3|6466|270|628|32|59.19|79.31|63.44|507.84|2030.08|1894.08|2537.92|32.48|1624.06|659.84|406.02|438.50|1065.86|1098.34|-1488.06| +2450827|59834|2450854|39079|421457|635|1413|39079|421457|635|1413|4|14|1|4|17318|292|628|25|48.77|79.49|12.71|1669.50|317.75|1219.25|1987.25|4.44|228.78|19.75|88.97|93.41|108.72|113.16|-1130.28| +2450827|63507|2450852|97644|1581969|3447|25325|87008|211528|1006|17684|1|26|20|1|17072|205|629|66|72.83|118.71|32.05|5719.56|2115.30|4806.78|7834.86|21.15|0.00|626.34|2115.30|2136.45|2741.64|2762.79|-2691.48| +2450827|63507|2450830|97644|1581969|3447|25325|87008|211528|1006|17684|1|92|18|4|14221|214|629|19|59.10|125.29|20.04|1999.75|380.76|1122.90|2380.51|34.26|0.00|357.01|380.76|415.02|737.77|772.03|-742.14| +2450827|63507|2450835|97644|1581969|3447|25325|87008|211528|1006|17684|1|45|5|1|16069|53|629|42|86.97|249.60|154.75|3983.70|6499.50|3652.74|10483.20|584.95|0.00|4612.44|6499.50|7084.45|11111.94|11696.89|2846.76| +2450827|63507|2450903|97644|1581969|3447|25325|87008|211528|1006|17684|1|75|20|2|5473|291|629|73|12.92|33.46|17.73|1148.29|1294.29|943.16|2442.58|0.00|0.00|121.91|1294.29|1294.29|1416.20|1416.20|351.13| +2450827|63507|2450881|97644|1581969|3447|25325|87008|211528|1006|17684|1|36|3|1|16657|85|629|55|91.65|190.63|47.65|7863.90|2620.75|5040.75|10484.65|26.20|0.00|3774.10|2620.75|2646.95|6394.85|6421.05|-2420.00| +2450827|63507|2450861|97644|1581969|3447|25325|87008|211528|1006|17684|1|96|1|3|17020|152|629|69|5.49|14.98|10.93|279.45|754.17|378.81|1033.62|63.80|45.25|485.76|708.92|772.72|1194.68|1258.48|330.11| +2450827|63507|2450899|97644|1581969|3447|25325|87008|211528|1006|17684|1|54|9|3|12674|84|629|53|48.43|59.08|7.08|2756.00|375.24|2566.79|3131.24|7.50|0.00|187.62|375.24|382.74|562.86|570.36|-2191.55| +2450827|45399|2450905|48589|1604968|5700|49632|48589|1604968|5700|49632|1|48|17|1|8692|216|630|15|10.66|20.57|17.07|52.50|256.05|159.90|308.55|3.99|189.47|21.45|66.58|70.57|88.03|92.02|-93.32| +2450827|45399|2450832|48589|1604968|5700|49632|48589|1604968|5700|49632|1|85|11|2|1978|7|630|79|29.47|66.30|23.86|3352.76|1884.94|2328.13|5237.70|18.84|0.00|942.47|1884.94|1903.78|2827.41|2846.25|-443.19| +2450827|45399|2450855|48589|1604968|5700|49632|48589|1604968|5700|49632|1|57|11|2|9094|187|630|63|57.85|78.67|6.29|4559.94|396.27|3644.55|4956.21|31.70|0.00|594.72|396.27|427.97|990.99|1022.69|-3248.28| +2450827|45399|2450890|48589|1604968|5700|49632|48589|1604968|5700|49632|1|108|6|3|13702|132|630|35|43.76|75.26|24.83|1765.05|869.05|1531.60|2634.10|8.69|434.52|500.15|434.53|443.22|934.68|943.37|-1097.07| +2450827|45399|2450864|48589|1604968|5700|49632|48589|1604968|5700|49632|1|104|8|5|7471|236|630|78|11.67|18.55|15.39|246.48|1200.42|910.26|1446.90|96.03|0.00|289.38|1200.42|1296.45|1489.80|1585.83|290.16| +2450827|45399|2450864|48589|1604968|5700|49632|48589|1604968|5700|49632|1|25|13|3|6934|220|630|48|98.73|146.12|89.13|2735.52|4278.24|4739.04|7013.76|61.60|2738.07|2314.08|1540.17|1601.77|3854.25|3915.85|-3198.87| +2450827|45399|2450899|48589|1604968|5700|49632|48589|1604968|5700|49632|1|23|14|4|1822|292|630|23|9.18|16.24|14.45|41.17|332.35|211.14|373.52|0.00|269.20|70.84|63.15|63.15|133.99|133.99|-147.99| +2450827|45399|2450851|48589|1604968|5700|49632|48589|1604968|5700|49632|1|34|7|5|12070|134|630|62|74.28|207.24|78.75|7966.38|4882.50|4605.36|12848.88|390.60|0.00|4368.52|4882.50|5273.10|9251.02|9641.62|277.14| +2450827|45399|2450854|48589|1604968|5700|49632|48589|1604968|5700|49632|1|73|4|2|5182|45|630|78|86.54|231.06|226.43|361.14|17661.54|6750.12|18022.68|0.00|0.00|1621.62|17661.54|17661.54|19283.16|19283.16|10911.42| +2450827|45399|2450842|48589|1604968|5700|49632|48589|1604968|5700|49632|1|65|7|1|5500|94|630|10|86.44|187.57|90.03|975.40|900.30|864.40|1875.70|0.00|819.27|37.50|81.03|81.03|118.53|118.53|-783.37| +2450827|45399|2450831|48589|1604968|5700|49632|48589|1604968|5700|49632|1|43|14|3|12322|266|630|5|20.55|57.54|54.08|17.30|270.40|102.75|287.70|16.22|0.00|126.55|270.40|286.62|396.95|413.17|167.65| +2450827|45399|2450885|48589|1604968|5700|49632|48589|1604968|5700|49632|1|41|11|5|12439|43|630|65|50.52|149.53|79.25|4568.20|5151.25|3283.80|9719.45|360.58|0.00|777.40|5151.25|5511.83|5928.65|6289.23|1867.45| +2450827|45399|2450901|48589|1604968|5700|49632|48589|1604968|5700|49632|1|68|5|5|13280|6|630|16|50.43|150.28|36.06|1827.52|576.96|806.88|2404.48|11.53|0.00|865.60|576.96|588.49|1442.56|1454.09|-229.92| +2450827|72305|2450882|70320|1060015|4463|958|70320|1060015|4463|958|2|60|19|3|11746|231|631|44|25.21|54.20|8.13|2027.08|357.72|1109.24|2384.80|3.57|0.00|596.20|357.72|361.29|953.92|957.49|-751.52| +2450827|72305|2450849|70320|1060015|4463|958|70320|1060015|4463|958|2|98|20|4|13330|240|631|34|50.51|133.85|133.85|0.00|4550.90|1717.34|4550.90|24.57|2093.41|2184.16|2457.49|2482.06|4641.65|4666.22|740.15| +2450827|72305|2450892|70320|1060015|4463|958|70320|1060015|4463|958|2|67|16|2|15226|171|631|74|53.80|160.86|130.29|2262.18|9641.46|3981.20|11903.64|192.82|0.00|594.96|9641.46|9834.28|10236.42|10429.24|5660.26| +2450827|72305|2450844|70320|1060015|4463|958|70320|1060015|4463|958|2|51|3|3|15950|233|631|47|1.84|2.35|0.42|90.71|19.74|86.48|110.45|1.77|0.00|54.99|19.74|21.51|74.73|76.50|-66.74| +2450827|72305|2450910|70320|1060015|4463|958|70320|1060015|4463|958|2|75|16|2|11372|246|631|82|54.75|140.16|128.94|920.04|10573.08|4489.50|11493.12|317.19|0.00|5286.54|10573.08|10890.27|15859.62|16176.81|6083.58| +2450827|72305|2450863|70320|1060015|4463|958|70320|1060015|4463|958|2|87|7|5|11446|166|631|80|69.35|88.07|83.66|352.80|6692.80|5548.00|7045.60|74.95|2944.83|492.80|3747.97|3822.92|4240.77|4315.72|-1800.03| +2450827|72305|2450893|70320|1060015|4463|958|70320|1060015|4463|958|2|17|18|2|12781|240|631|53|65.07|191.95|99.81|4883.42|5289.93|3448.71|10173.35|105.79|0.00|610.03|5289.93|5395.72|5899.96|6005.75|1841.22| +2450827|72305|2450898|70320|1060015|4463|958|70320|1060015|4463|958|2|95|11|3|3950|282|631|96|40.95|71.25|24.93|4446.72|2393.28|3931.20|6840.00|95.73|0.00|1504.32|2393.28|2489.01|3897.60|3993.33|-1537.92| +2450827|72305|2450829|70320|1060015|4463|958|70320|1060015|4463|958|2|97|17|1|12187|6|631|83|53.47|107.47|59.10|4014.71|4905.30|4438.01|8920.01|343.37|0.00|1872.48|4905.30|5248.67|6777.78|7121.15|467.29| +2450827|72305|2450917|70320|1060015|4463|958|70320|1060015|4463|958|2|30|10|5|8395|104|631|8|98.22|288.76|228.12|485.12|1824.96|785.76|2310.08|72.99|0.00|646.80|1824.96|1897.95|2471.76|2544.75|1039.20| +2450827|72305|2450894|70320|1060015|4463|958|70320|1060015|4463|958|2|28|12|2|14683|62|631|46|31.67|88.67|56.74|1468.78|2610.04|1456.82|4078.82|104.40|0.00|570.86|2610.04|2714.44|3180.90|3285.30|1153.22| +2450827|72305|2450876|70320|1060015|4463|958|70320|1060015|4463|958|2|43|18|4|14746|46|631|29|83.53|126.96|97.75|847.09|2834.75|2422.37|3681.84|170.08|0.00|36.54|2834.75|3004.83|2871.29|3041.37|412.38| +2450827|77533|2450855|97133|660711|642|18620|11815|222929|2060|24838|4|7|18|2|17569|73|632|53|24.43|45.43|27.71|939.16|1468.63|1294.79|2407.79|102.80|0.00|192.39|1468.63|1571.43|1661.02|1763.82|173.84| +2450827|77533|2450913|97133|660711|642|18620|11815|222929|2060|24838|4|94|12|3|3334|118|632|23|82.11|137.94|137.94|0.00|3172.62|1888.53|3172.62|222.08|0.00|824.78|3172.62|3394.70|3997.40|4219.48|1284.09| +2450827|77533|2450895|97133|660711|642|18620|11815|222929|2060|24838|4|13|1|1|15142|53|632|67|21.51|53.99|0.53|3581.82|35.51|1441.17|3617.33|3.19|0.00|614.39|35.51|38.70|649.90|653.09|-1405.66| +2450827|77533|2450907|97133|660711|642|18620|11815|222929|2060|24838|4|34|10|1|15652|114|632|8|56.37|112.74|86.80|207.52|694.40|450.96|901.92|20.83|0.00|342.72|694.40|715.23|1037.12|1057.95|243.44| +2450827|77533|2450870|97133|660711|642|18620|11815|222929|2060|24838|4|99|9|3|15632|93|632|72|54.28|72.73|69.09|262.08|4974.48|3908.16|5236.56|0.00|0.00|1518.48|4974.48|4974.48|6492.96|6492.96|1066.32| +2450827|77533|2450833|97133|660711|642|18620|11815|222929|2060|24838|4|107|1|5|11806|8|632|90|75.57|95.97|5.75|8119.80|517.50|6801.30|8637.30|20.70|0.00|517.50|517.50|538.20|1035.00|1055.70|-6283.80| +2450827|77533|2450899|97133|660711|642|18620|11815|222929|2060|24838|4|13|13|2|7495|188|632|75|51.08|113.39|81.64|2381.25|6123.00|3831.00|8504.25|367.38|0.00|3656.25|6123.00|6490.38|9779.25|10146.63|2292.00| +2450827|77533|2450844|97133|660711|642|18620|11815|222929|2060|24838|4|79|11|4|8248|131|632|69|87.75|250.96|117.95|9177.69|8138.55|6054.75|17316.24|569.69|0.00|5540.70|8138.55|8708.24|13679.25|14248.94|2083.80| +2450827|51490|2450829|69880|812966|4499|30032|69880|812966|4499|30032|2|12|11|3|5494|232|633|11|89.11|132.77|83.64|540.43|920.04|980.21|1460.47|46.00|0.00|423.50|920.04|966.04|1343.54|1389.54|-60.17| +2450827|51490|2450889|69880|812966|4499|30032|69880|812966|4499|30032|2|57|19|5|10942|282|633|14|8.56|25.25|17.17|113.12|240.38|119.84|353.50|14.42|0.00|166.04|240.38|254.80|406.42|420.84|120.54| +2450827|51490|2450834|69880|812966|4499|30032|69880|812966|4499|30032|2|43|7|1|8846|260|633|59|78.55|200.30|116.17|4963.67|6854.03|4634.45|11817.70|616.86|0.00|3781.31|6854.03|7470.89|10635.34|11252.20|2219.58| +2450827|51490|2450835|69880|812966|4499|30032|69880|812966|4499|30032|2|65|10|2|3853|137|633|37|62.42|72.40|48.50|884.30|1794.50|2309.54|2678.80|30.68|771.63|214.23|1022.87|1053.55|1237.10|1267.78|-1286.67| +2450827|51490|2450872|69880|812966|4499|30032|69880|812966|4499|30032|2|40|14|3|8746|101|633|65|67.91|173.17|133.34|2588.95|8667.10|4414.15|11256.05|780.03|0.00|1688.05|8667.10|9447.13|10355.15|11135.18|4252.95| +2450827|51490|2450904|69880|812966|4499|30032|69880|812966|4499|30032|2|10|19|5|1096|54|633|6|84.84|177.31|129.43|287.28|776.58|509.04|1063.86|7.76|0.00|180.84|776.58|784.34|957.42|965.18|267.54| +2450827|51490|2450879|69880|812966|4499|30032|69880|812966|4499|30032|2|90|8|3|10315|50|633|84|42.36|87.68|45.59|3535.56|3829.56|3558.24|7365.12|0.00|0.00|2651.04|3829.56|3829.56|6480.60|6480.60|271.32| +2450827|51490|2450868|69880|812966|4499|30032|69880|812966|4499|30032|2|48|16|4|2191|55|633|43|76.49|118.55|37.93|3466.66|1630.99|3289.07|5097.65|81.54|0.00|815.28|1630.99|1712.53|2446.27|2527.81|-1658.08| +2450827|51490|2450914|69880|812966|4499|30032|69880|812966|4499|30032|2|93|6|5|14809|281|633|31|30.48|30.78|13.54|534.44|419.74|944.88|954.18|8.39|0.00|228.78|419.74|428.13|648.52|656.91|-525.14| +2450827|51490|2450835|69880|812966|4499|30032|69880|812966|4499|30032|2|81|5|4|5839|3|633|30|38.56|101.02|38.38|1879.20|1151.40|1156.80|3030.60|0.00|0.00|939.30|1151.40|1151.40|2090.70|2090.70|-5.40| +2450827|19450|2450840|79557|488316|5522|39481|49329|958646|3052|41811|2|81|14|3|6454|176|634|61|7.96|18.78|10.89|481.29|664.29|485.56|1145.58|19.92|0.00|22.57|664.29|684.21|686.86|706.78|178.73| +2450827|19450|2450876|79557|488316|5522|39481|49329|958646|3052|41811|2|79|18|3|4423|278|634|16|66.94|194.12|116.47|1242.40|1863.52|1071.04|3105.92|37.27|0.00|1180.16|1863.52|1900.79|3043.68|3080.95|792.48| +2450827|19450|2450872|79557|488316|5522|39481|49329|958646|3052|41811|2|31|2|5|16357|98|634|21|44.53|99.74|54.85|942.69|1151.85|935.13|2094.54|103.66|0.00|314.16|1151.85|1255.51|1466.01|1569.67|216.72| +2450827|19450|2450834|79557|488316|5522|39481|49329|958646|3052|41811|2|62|18|4|13106|121|634|15|93.53|211.37|114.13|1458.60|1711.95|1402.95|3170.55|102.71|0.00|1426.65|1711.95|1814.66|3138.60|3241.31|309.00| +2450827|19450|2450853|79557|488316|5522|39481|49329|958646|3052|41811|2|67|10|4|8206|150|634|18|60.77|133.08|125.09|143.82|2251.62|1093.86|2395.44|67.54|0.00|1149.66|2251.62|2319.16|3401.28|3468.82|1157.76| +2450827|45242|2450860|49438|192152|819|27133|49438|192152|819|27133|4|85|10|1|2146|17|635|59|18.72|28.26|1.97|1551.11|116.23|1104.48|1667.34|3.48|0.00|382.91|116.23|119.71|499.14|502.62|-988.25| +2450827|45242|2450892|49438|192152|819|27133|49438|192152|819|27133|4|10|3|1|1618|277|635|69|35.73|103.61|11.39|6363.18|785.91|2465.37|7149.09|0.00|0.00|2359.11|785.91|785.91|3145.02|3145.02|-1679.46| +2450827|45242|2450833|49438|192152|819|27133|49438|192152|819|27133|4|42|18|2|9386|242|635|7|80.36|97.23|43.75|374.36|306.25|562.52|680.61|0.00|0.00|34.02|306.25|306.25|340.27|340.27|-256.27| +2450827|45242|2450916|49438|192152|819|27133|49438|192152|819|27133|4|1|4|1|11932|32|635|43|33.15|65.96|9.89|2411.01|425.27|1425.45|2836.28|15.09|123.32|1333.00|301.95|317.04|1634.95|1650.04|-1123.50| +2450827|45242|2450866|49438|192152|819|27133|49438|192152|819|27133|4|90|1|4|13216|51|635|27|81.71|232.87|65.20|4527.09|1760.40|2206.17|6287.49|88.02|0.00|2577.69|1760.40|1848.42|4338.09|4426.11|-445.77| +2450827|45242|2450889|49438|192152|819|27133|49438|192152|819|27133|4|18|15|3|104|129|635|28|95.26|106.69|24.53|2300.48|686.84|2667.28|2987.32|41.21|0.00|1194.76|686.84|728.05|1881.60|1922.81|-1980.44| +2450827|45242|2450911|49438|192152|819|27133|49438|192152|819|27133|4|72|1|2|7670|299|635|95|92.57|264.75|45.00|20876.25|4275.00|8794.15|25151.25|50.01|3719.25|5281.05|555.75|605.76|5836.80|5886.81|-8238.40| +2450827|45242|2450910|49438|192152|819|27133|49438|192152|819|27133|4|76|9|4|11527|169|635|10|48.20|101.22|74.90|263.20|749.00|482.00|1012.20|29.96|0.00|354.20|749.00|778.96|1103.20|1133.16|267.00| +2450827|45242|2450838|49438|192152|819|27133|49438|192152|819|27133|4|52|12|2|13480|217|635|32|73.35|119.56|115.97|114.88|3711.04|2347.20|3825.92|111.33|0.00|918.08|3711.04|3822.37|4629.12|4740.45|1363.84| +2450827|45242|2450861|49438|192152|819|27133|49438|192152|819|27133|4|2|10|4|6436|258|635|10|97.55|158.03|158.03|0.00|1580.30|975.50|1580.30|94.81|0.00|647.90|1580.30|1675.11|2228.20|2323.01|604.80| +2450827|45242|2450860|49438|192152|819|27133|49438|192152|819|27133|4|81|3|2|4472|32|635|25|38.98|86.14|49.96|904.50|1249.00|974.50|2153.50|37.47|0.00|258.25|1249.00|1286.47|1507.25|1544.72|274.50| +2450827|45242|2450882|49438|192152|819|27133|49438|192152|819|27133|4|6|8|4|9412|294|635|29|92.86|119.78|27.54|2674.96|798.66|2692.94|3473.62|0.00|0.00|1250.48|798.66|798.66|2049.14|2049.14|-1894.28| +2450827|45242|2450906|49438|192152|819|27133|49438|192152|819|27133|4|36|13|3|1496|130|635|74|36.76|79.40|58.75|1528.10|4347.50|2720.24|5875.60|0.00|0.00|0.00|4347.50|4347.50|4347.50|4347.50|1627.26| +2450827|46560|2450897|80532|1095298|817|4736|80532|1095298|817|4736|4|43|6|3|463|187|636|61|3.55|8.20|2.21|365.39|134.81|216.55|500.20|2.61|4.04|84.79|130.77|133.38|215.56|218.17|-85.78| +2450827|46560|2450848|80532|1095298|817|4736|80532|1095298|817|4736|4|47|19|4|10894|79|636|7|96.16|135.58|82.70|370.16|578.90|673.12|949.06|23.15|0.00|379.61|578.90|602.05|958.51|981.66|-94.22| +2450827|46560|2450831|80532|1095298|817|4736|80532|1095298|817|4736|4|105|17|3|6334|230|636|90|18.77|40.54|12.56|2518.20|1130.40|1689.30|3648.60|90.43|0.00|181.80|1130.40|1220.83|1312.20|1402.63|-558.90| +2450827|46560|2450848|80532|1095298|817|4736|80532|1095298|817|4736|4|108|16|3|2623|80|636|29|13.98|14.11|11.42|78.01|331.18|405.42|409.19|29.80|0.00|122.67|331.18|360.98|453.85|483.65|-74.24| +2450827|46560|2450830|80532|1095298|817|4736|80532|1095298|817|4736|4|7|12|1|7826|287|636|86|81.47|137.68|132.17|473.86|11366.62|7006.42|11840.48|681.99|0.00|236.50|11366.62|12048.61|11603.12|12285.11|4360.20| +2450827|46560|2450886|||817|4736|||||4||||3253||636||58.20||19.55|3467.88|1485.80|4423.20|4953.68|10.84|401.16|2229.08|1084.64|1095.48|3313.72|3324.56|-3338.56| +|||72714|53079|4644||72714|53079|||1||||170|220|637|48|79.75||71.84||||||0.00|530.40||3586.25|3978.72|4116.65|-379.68| +2450827|72086|2450881|72714|53079|4644|33409|72714|53079|4644|33409|1|50|8|3|14378|163|637|23|61.02|114.71|57.35|1319.28|1319.05|1403.46|2638.33|0.00|0.00|0.00|1319.05|1319.05|1319.05|1319.05|-84.41| +2450827|72086|2450904|72714|53079|4644|33409|72714|53079|4644|33409|1|75|4|3|602|98|637|5|3.02|5.31|2.70|13.05|13.50|15.10|26.55|0.94|0.00|11.65|13.50|14.44|25.15|26.09|-1.60| +2450827|72086|2450912|72714|53079|4644|33409|72714|53079|4644|33409|1|49|15|5|6350|230|637|95|27.74|35.22|5.63|2811.05|534.85|2635.30|3345.90|13.15|315.56|468.35|219.29|232.44|687.64|700.79|-2416.01| +2450827|72086|2450898|72714|53079|4644|33409|72714|53079|4644|33409|1|59|18|1|4726|65|637|77|71.13|141.54|101.90|3052.28|7846.30|5477.01|10898.58|235.38|0.00|653.73|7846.30|8081.68|8500.03|8735.41|2369.29| +2450827|72086|2450881|72714|53079|4644|33409|72714|53079|4644|33409|1|51|17|2|13771|219|637|26|86.62|86.62|32.91|1396.46|855.66|2252.12|2252.12|3.16|539.06|179.92|316.60|319.76|496.52|499.68|-1935.52| +2450827|72086|2450842|72714|53079|4644|33409|72714|53079|4644|33409|1|101|6|3|5824|169|637|74|43.51|90.93|31.82|4374.14|2354.68|3219.74|6728.82|211.92|0.00|3094.68|2354.68|2566.60|5449.36|5661.28|-865.06| +2450827|72086|2450864|72714|53079|4644|33409|72714|53079|4644|33409|1|29|2|4|14300|258|637|38|13.49|31.70|26.62|193.04|1011.56|512.62|1204.60|19.72|617.05|493.62|394.51|414.23|888.13|907.85|-118.11| +2450827|72086|2450860|72714|53079|4644|33409|72714|53079|4644|33409|1|100|7|3|1508|181|637|38|94.13|154.37|26.24|4868.94|997.12|3576.94|5866.06|29.91|0.00|117.04|997.12|1027.03|1114.16|1144.07|-2579.82| +2450827|72086|2450895|72714|53079|4644|33409|72714|53079|4644|33409|1|77|9|5|8878|207|637|68|55.99|139.41|105.95|2275.28|7204.60|3807.32|9479.88|116.71|5907.77|2085.56|1296.83|1413.54|3382.39|3499.10|-2510.49| +2450827|65593|2450898|40057|1627957|3857|32557|40057|1627957|3857|32557|1|35|10|3|17182|266|638|61|3.41|8.93|8.39|32.94|511.79|208.01|544.73|10.23|0.00|21.35|511.79|522.02|533.14|543.37|303.78| +2450827|65593|2450830|40057|1627957|3857|32557|40057|1627957|3857|32557|1|6|14|4|3175|111|638|80|92.48|233.04|18.64|17152.00|1491.20|7398.40|18643.20|14.91|0.00|8575.20|1491.20|1506.11|10066.40|10081.31|-5907.20| +2450827|65593|2450889|40057|1627957|3857|32557|40057|1627957|3857|32557|1|80|4|2|15379|93|638|47|89.26|131.21|10.49|5673.84|493.03|4195.22|6166.87|0.00|0.00|2096.67|493.03|493.03|2589.70|2589.70|-3702.19| +2450827|65593|2450870|40057|1627957|3857|32557|40057|1627957|3857|32557|1|63|16|1|124|75|638|54|76.86|170.62|129.67|2211.30|7002.18|4150.44|9213.48|420.13|0.00|4237.92|7002.18|7422.31|11240.10|11660.23|2851.74| +2450827|65593|2450899|40057|1627957|3857|32557|40057|1627957|3857|32557|1|88|15|4|4582|202|638|2|48.70|70.12|44.87|50.50|89.74|97.40|140.24|1.79|0.00|53.28|89.74|91.53|143.02|144.81|-7.66| +2450827|65593|2450882|40057|1627957|3857|32557|40057|1627957|3857|32557|1|69|9|3|10736|16|638|13|25.84|58.65|45.74|167.83|594.62|335.92|762.45|23.78|0.00|373.49|594.62|618.40|968.11|991.89|258.70| +2450827|65593|2450889|40057|1627957|3857|32557|40057|1627957|3857|32557|1|7|11|5|1864|145|638|28|66.19|159.51|27.11|3707.20|759.08|1853.32|4466.28|22.77|0.00|178.64|759.08|781.85|937.72|960.49|-1094.24| +2450827|65593|2450838|40057|1627957|3857|32557|40057|1627957|3857|32557|1|102|7|5|2011|19|638|80|81.95|86.86|85.99|69.60|6879.20|6556.00|6948.80|137.58|0.00|1528.00|6879.20|7016.78|8407.20|8544.78|323.20| +2450827|65593|2450862|40057|1627957|3857|32557|40057|1627957|3857|32557|1|43|19|2|3740|290|638|6|87.19|122.93|109.40|81.18|656.40|523.14|737.58|1.83|610.45|147.48|45.95|47.78|193.43|195.26|-477.19| +2450827|65593|2450834|40057|1627957|3857|32557|40057|1627957|3857|32557|1|32|7|1|11200|179|638|73|67.74|175.44|149.12|1921.36|10885.76|4945.02|12807.12|108.85|0.00|255.50|10885.76|10994.61|11141.26|11250.11|5940.74| +2450827|61961|2450838|67119|1314714|3462|13010|67119|1314714|3462|13010|2|35|3|5|16585|297|639|43|8.63|25.28|22.49|119.97|967.07|371.09|1087.04|77.36|0.00|173.72|967.07|1044.43|1140.79|1218.15|595.98| +2450827|61961|2450911|67119|1314714|3462|13010|67119|1314714|3462|13010|2|9|11|1|15526|265|639|54|88.35|262.39|162.68|5384.34|8784.72|4770.90|14169.06|0.00|0.00|6517.26|8784.72|8784.72|15301.98|15301.98|4013.82| +2450827|61961|2450856|67119|1314714|3462|13010|67119|1314714|3462|13010|2|33|8|2|1154|87|639|71|94.94|284.82|19.93|18807.19|1415.03|6740.74|20222.22|7.35|679.21|5257.55|735.82|743.17|5993.37|6000.72|-6004.92| +2450827|61961|2450842|67119|1314714|3462|13010|67119|1314714|3462|13010|2|14|2|5|3991|132|639|65|55.22|74.54|18.63|3634.15|1210.95|3589.30|4845.10|12.10|0.00|290.55|1210.95|1223.05|1501.50|1513.60|-2378.35| +2450827|61961|2450905|67119|1314714|3462|13010|67119|1314714|3462|13010|2|102|6|1|13772|136|639|35|28.39|51.95|51.95|0.00|1818.25|993.65|1818.25|18.18|0.00|127.05|1818.25|1836.43|1945.30|1963.48|824.60| +2450827|61961|2450895|67119|1314714|3462|13010|67119|1314714|3462|13010|2|37|7|2|2692|266|639|48|2.77|6.64|3.12|168.96|149.76|132.96|318.72|2.99|0.00|79.68|149.76|152.75|229.44|232.43|16.80| +2450827|61961|2450904|67119|1314714|3462|13010|67119|1314714|3462|13010|2|53|15|1|7849|210|639|7|63.96|179.08|0.00|1253.56|0.00|447.72|1253.56|0.00|0.00|313.39|0.00|0.00|313.39|313.39|-447.72| +2450827|86017|2450899|14887|317894|6501|21508|14887|317894|6501|21508|4|23|9|3|9259|153|640|4|17.72|38.62|0.00|154.48|0.00|70.88|154.48|0.00|0.00|46.32|0.00|0.00|46.32|46.32|-70.88| +2450827|86017|2450849|14887|317894|6501|21508|14887|317894|6501|21508|4|55|12|3|7478|127|640|57|23.87|63.97|30.06|1932.87|1713.42|1360.59|3646.29|85.67|0.00|108.87|1713.42|1799.09|1822.29|1907.96|352.83| +2450827|86017|2450865|14887|317894|6501|21508|14887|317894|6501|21508|4|48|20|3|5965|274|640|44|6.69|12.84|2.82|440.88|124.08|294.36|564.96|0.00|75.68|191.84|48.40|48.40|240.24|240.24|-245.96| +2450827|86017|2450896|14887|317894|6501|21508|14887|317894|6501|21508|4|106|7|3|8098|100|640|51|50.99|65.77|57.87|402.90|2951.37|2600.49|3354.27|265.62|0.00|167.28|2951.37|3216.99|3118.65|3384.27|350.88| +2450827|86017|2450899|14887|317894|6501|21508|14887|317894|6501|21508|4|28|6|5|11665|193|640|100|95.92|143.88|142.44|144.00|14244.00|9592.00|14388.00|774.87|4558.08|4316.00|9685.92|10460.79|14001.92|14776.79|93.92| +2450827|86017|2450881|14887|317894|6501|21508|14887|317894|6501|21508|4|96|13|4|3907|28|640|81|81.98|186.91|175.69|908.82|14230.89|6640.38|15139.71|426.92|0.00|5904.09|14230.89|14657.81|20134.98|20561.90|7590.51| +2450827|86017|2450829|14887|317894|6501|21508|14887|317894|6501|21508|4|96|19|5|1891|201|640|12|91.34|254.83|56.06|2385.24|672.72|1096.08|3057.96|13.45|0.00|672.72|672.72|686.17|1345.44|1358.89|-423.36| +2450827|86017|2450849|14887|317894|6501|21508|14887|317894|6501|21508|4|11|20|2|15439|281|640|29|27.79|32.51|23.08|273.47|669.32|805.91|942.79|0.00|0.00|18.85|669.32|669.32|688.17|688.17|-136.59| +2450827|86017|2450829|14887|317894|6501|21508|14887|317894|6501|21508|4|91|13|5|17827|170|640|99|59.00|94.99|88.34|658.35|8745.66|5841.00|9404.01|787.10|0.00|751.41|8745.66|9532.76|9497.07|10284.17|2904.66| +2450827|86017|2450845|14887|317894|6501|21508|14887|317894|6501|21508|4|7|20|3|7142|107|640|2|85.31|206.45|187.86|37.18|375.72|170.62|412.90|12.32|67.62|16.50|308.10|320.42|324.60|336.92|137.48| +2450827|86017|2450895|14887|317894|6501|21508|14887|317894|6501|21508|4|10|13|3|2887|158|640|73|8.56|16.17|0.97|1109.60|70.81|624.88|1180.41|0.00|0.00|23.36|70.81|70.81|94.17|94.17|-554.07| +2450827|86017|2450900|14887|317894|6501|21508|14887|317894|6501|21508|4|9|10|5|8570|105|640|78|54.32|116.78|19.85|7560.54|1548.30|4236.96|9108.84|139.34|0.00|910.26|1548.30|1687.64|2458.56|2597.90|-2688.66| +2450827|86017|2450845|14887|317894|6501|21508|14887|317894|6501|21508|4|26|1|2|3259|146|640|18|53.81|99.54|23.88|1361.88|429.84|968.58|1791.72|4.29|0.00|412.02|429.84|434.13|841.86|846.15|-538.74| +2450827|86017|2450895|14887|317894|6501|21508|14887|317894|6501|21508|4|6|12|2|17180|102|640|6|77.12|190.48|19.04|1028.64|114.24|462.72|1142.88|2.28|0.00|205.68|114.24|116.52|319.92|322.20|-348.48| +2450827|37242|2450845|10532|717113|5647|7381|10532|717113|5647|7381|4|36|2|2|5360|152|641|83|97.18|145.77|52.47|7743.90|4355.01|8065.94|12098.91|130.65|0.00|6049.04|4355.01|4485.66|10404.05|10534.70|-3710.93| +2450827|37242|2450912|10532|717113|5647|7381|10532|717113|5647|7381|4|80|20|5|13507|17|641|84|81.03|222.02|179.83|3543.96|15105.72|6806.52|18649.68|755.28|0.00|2424.24|15105.72|15861.00|17529.96|18285.24|8299.20| +2450827|37242|2450846|10532|717113|5647|7381|10532|717113|5647|7381|4|31|1|1|3268|213|641|37|88.47|153.05|68.87|3114.66|2548.19|3273.39|5662.85|50.96|0.00|113.22|2548.19|2599.15|2661.41|2712.37|-725.20| +2450827|37242|2450867|10532|717113|5647|7381|10532|717113|5647|7381|4|42|4|2|15956|166|641|27|26.42|35.66|28.17|202.23|760.59|713.34|962.82|29.20|395.50|250.29|365.09|394.29|615.38|644.58|-348.25| +2450827|54499|2450830|54407|1793425|2043|16708|54407|1793425|2043|16708|2|15|3|5|1333|149|642|7|7.83|8.22|7.48|5.18|52.36|54.81|57.54|2.92|19.89|6.30|32.47|35.39|38.77|41.69|-22.34| +2450827|54499|2450831|54407|1793425|2043|16708|54407|1793425|2043|16708|2|104|1|5|12757|175|642|71|38.26|44.38|0.00|3150.98|0.00|2716.46|3150.98|0.00|0.00|1449.11|0.00|0.00|1449.11|1449.11|-2716.46| +2450827|54499|2450898|54407|1793425|2043|16708|54407|1793425|2043|16708|2|49|14|2|46|38|642|87|3.42|8.44|7.93|44.37|689.91|297.54|734.28|6.89|0.00|212.28|689.91|696.80|902.19|909.08|392.37| +2450827|54499|2450891|54407|1793425|2043|16708|54407|1793425|2043|16708|2|72|15|1|17306|190|642|67|52.17|89.73|39.48|3366.75|2645.16|3495.39|6011.91|25.12|2142.57|841.52|502.59|527.71|1344.11|1369.23|-2992.80| +2450827|54499|2450858|54407|1793425|2043|16708|54407|1793425|2043|16708|2|53|5|4|6481|82|642|5|65.70|94.60|38.78|279.10|193.90|328.50|473.00|5.81|0.00|42.55|193.90|199.71|236.45|242.26|-134.60| +2450827|54499|2450831|54407|1793425|2043|16708|54407|1793425|2043|16708|2|64|3|3|9314|168|642|94|24.21|70.69|4.24|6246.30|398.56|2275.74|6644.86|15.94|0.00|1660.98|398.56|414.50|2059.54|2075.48|-1877.18| +2450827|54499|2450839|54407|1793425|2043|16708|54407|1793425|2043|16708|2|6|16|3|14068|247|642|74|13.92|40.22|10.85|2173.38|802.90|1030.08|2976.28|64.23|0.00|386.28|802.90|867.13|1189.18|1253.41|-227.18| +2450827|54499|2450864|54407|1793425|2043|16708|54407|1793425|2043|16708|2|67|20|5|6112|176|642|49|84.69|203.25|132.11|3485.86|6473.39|4149.81|9959.25|198.08|3171.96|4481.54|3301.43|3499.51|7782.97|7981.05|-848.38| +2450827|60515|2450896|72258|690827|5396|3565|72258|690827|5396|3565|2|26|6|4|6691|105|643|38|38.10|71.62|15.04|2150.04|571.52|1447.80|2721.56|51.43|0.00|1306.06|571.52|622.95|1877.58|1929.01|-876.28| +2450827|60515|2450888|72258|690827|5396|3565|72258|690827|5396|3565|2|27|11|2|8113|104|643|22|60.29|85.61|8.56|1695.10|188.32|1326.38|1883.42|5.64|0.00|338.80|188.32|193.96|527.12|532.76|-1138.06| +2450827|60515|2450916|72258|690827|5396|3565|72258|690827|5396|3565|2|66|1|3|13408|257|643|26|43.60|117.72|35.31|2142.66|918.06|1133.60|3060.72|11.01|697.72|275.34|220.34|231.35|495.68|506.69|-913.26| +2450827|60515|2450899|72258|690827|5396|3565|72258|690827|5396|3565|2|83|4|2|15373|105|643|19|30.06|69.73|59.96|185.63|1139.24|571.14|1324.87|11.39|0.00|609.33|1139.24|1150.63|1748.57|1759.96|568.10| +2450827|60515|2450896|72258|690827|5396|3565|72258|690827|5396|3565|2|79|11|4|13838|67|643|3|4.72|13.59|12.91|2.04|38.73|14.16|40.77|1.03|12.78|2.85|25.95|26.98|28.80|29.83|11.79| +2450827|60515|2450905|72258|690827|5396|3565|72258|690827|5396|3565|2|36|5|1|12836|139|643|27|18.06|24.56|0.73|643.41|19.71|487.62|663.12|0.00|0.00|225.45|19.71|19.71|245.16|245.16|-467.91| +2450827|60515|2450896|72258|690827|5396|3565|72258|690827|5396|3565|2|34|9|2|6730|88|643|47|94.50|203.17|184.88|859.63|8689.36|4441.50|9548.99|260.68|0.00|1241.27|8689.36|8950.04|9930.63|10191.31|4247.86| +2450827|60515|2450880|72258|690827|5396|3565|72258|690827|5396|3565|2|1|13|1|3442|30|643|63|78.72|91.31|44.74|2933.91|2818.62|4959.36|5752.53|197.30|0.00|632.52|2818.62|3015.92|3451.14|3648.44|-2140.74| +2450827|60515|2450843|72258|690827|5396|3565|72258|690827|5396|3565|2|35|13|2|8356|230|643|5|27.61|30.92|25.35|27.85|126.75|138.05|154.60|8.89|27.88|58.70|98.87|107.76|157.57|166.46|-39.18| +2450827|60515|2450843|72258|690827|5396|3565|72258|690827|5396|3565|2|94|8|1|2707|87|643|28|12.26|15.57|13.54|56.84|379.12|343.28|435.96|18.95|0.00|174.16|379.12|398.07|553.28|572.23|35.84| +2450827|60515|2450869|72258|690827|5396|3565|72258|690827|5396|3565|2|33|15|3|12733|45|643|19|15.78|38.18|17.56|391.78|333.64|299.82|725.42|26.69|0.00|137.75|333.64|360.33|471.39|498.08|33.82| +2450827|60515|2450839|72258|690827|5396|3565|72258|690827|5396|3565|2|71|3|2|16718|68|643|87|5.90|15.10|11.77|289.71|1023.99|513.30|1313.70|40.95|0.00|511.56|1023.99|1064.94|1535.55|1576.50|510.69| +2450827|60515|2450843|72258|690827|5396|3565|72258|690827|5396|3565|2|80|16|3|5545|299|643|19|3.86|4.94|1.08|73.34|20.52|73.34|93.86|1.64|0.00|7.41|20.52|22.16|27.93|29.57|-52.82| +2450827|60515|2450894|72258|690827|5396|3565|72258|690827|5396|3565|2|51|5|2|17815|128|643|13|94.04|125.07|26.26|1284.53|341.38|1222.52|1625.91|0.00|0.00|130.00|341.38|341.38|471.38|471.38|-881.14| +2450827|36593|2450846|45006|688700|7147|44441|45006|688700|7147|44441|4|50|8|5|7645|99|644|46|74.10|118.56|37.93|3708.98|1744.78|3408.60|5453.76|0.00|0.00|1635.76|1744.78|1744.78|3380.54|3380.54|-1663.82| +2450827|36593|2450884|45006|688700|7147|44441|45006|688700|7147|44441|4|57|5|5|1111|231|644|59|22.46|55.92|17.89|2243.77|1055.51|1325.14|3299.28|94.99|0.00|725.70|1055.51|1150.50|1781.21|1876.20|-269.63| +2450827|36593|2450830|45006|688700|7147|44441|45006|688700|7147|44441|4|37|8|3|6031|144|644|49|91.02|183.86|38.61|7117.25|1891.89|4459.98|9009.14|113.51|0.00|4234.09|1891.89|2005.40|6125.98|6239.49|-2568.09| +2450827|36593|2450882|45006|688700|7147|44441|45006|688700|7147|44441|4|84|16|4|11578|199|644|78|64.24|91.86|28.47|4944.42|2220.66|5010.72|7165.08|66.61|0.00|1003.08|2220.66|2287.27|3223.74|3290.35|-2790.06| +2450827|36593|2450842|45006|688700|7147|44441|45006|688700|7147|44441|4|14|2|4|10280|112|644|45|60.69|143.22|101.68|1869.30|4575.60|2731.05|6444.90|91.51|0.00|2126.70|4575.60|4667.11|6702.30|6793.81|1844.55| +2450827|38389|2450911|69066|1390255|5044|16986|69066|1390255|5044|16986|2|64|7|1|4936|99|645|38|1.96|4.39|1.84|96.90|69.92|74.48|166.82|5.59|0.00|14.82|69.92|75.51|84.74|90.33|-4.56| +2450827|38389|2450879|69066|1390255|5044|16986|69066|1390255|5044|16986|2|39|2|2|4382|132|645|9|54.58|144.63|59.29|768.06|533.61|491.22|1301.67|20.49|21.34|26.01|512.27|532.76|538.28|558.77|21.05| +2450827|38389|2450870|69066|1390255|5044|16986|69066|1390255|5044|16986|2|34|12|3|17491|70|645|96|64.15|163.58|75.24|8480.64|7223.04|6158.40|15703.68|0.00|0.00|0.00|7223.04|7223.04|7223.04|7223.04|1064.64| +2450827|38389|2450862|69066|1390255|5044|16986|69066|1390255|5044|16986|2|8|16|2|17275|204|645|73|70.23|198.04|194.07|289.81|14167.11|5126.79|14456.92|991.69|0.00|1734.48|14167.11|15158.80|15901.59|16893.28|9040.32| +2450827|49588|2450901|70541|929853|365|7230|51373|1284254|1966|17678|4|77|8|2|11840|155|646|46|52.34|117.76|60.05|2654.66|2762.30|2407.64|5416.96|37.56|1823.11|2437.54|939.19|976.75|3376.73|3414.29|-1468.45| +2450827|49588|2450914|70541|929853|365|7230|51373|1284254|1966|17678|4|3|13|1|10387|105|646|80|57.78|147.91|69.51|6272.00|5560.80|4622.40|11832.80|0.00|0.00|4732.80|5560.80|5560.80|10293.60|10293.60|938.40| +2450827|49588|2450899|70541|929853|365|7230|51373|1284254|1966|17678|4|35|12|1|8192|296|646|77|15.92|20.05|5.41|1127.28|416.57|1225.84|1543.85|37.49|0.00|200.20|416.57|454.06|616.77|654.26|-809.27| +2450827|49588|2450879|70541|929853|365|7230|51373|1284254|1966|17678|4|17|15|3|7240|157|646|52|19.72|59.16|50.87|431.08|2645.24|1025.44|3076.32|132.26|0.00|984.36|2645.24|2777.50|3629.60|3761.86|1619.80| +2450827|49588|2450880|70541|929853|365|7230|51373|1284254|1966|17678|4|57|6|3|5236|296|646|96|80.37|139.84|104.88|3356.16|10068.48|7715.52|13424.64|906.16|0.00|5638.08|10068.48|10974.64|15706.56|16612.72|2352.96| +2450827|35847|2450852|70095|60237|779|3321|70095|60237|779|3321|4|27|16|5|6949|96|647|98|28.79|28.79|16.41|1213.24|1608.18|2821.42|2821.42|53.39|273.39|733.04|1334.79|1388.18|2067.83|2121.22|-1486.63| +2450827|35847|2450838|70095|60237|779|3321|70095|60237|779|3321|4|63|4|4|8930|281|647|91|49.44|145.35|45.05|9127.30|4099.55|4499.04|13226.85|245.97|0.00|0.00|4099.55|4345.52|4099.55|4345.52|-399.49| +2450827|35847|2450906|70095|60237|779|3321|70095|60237|779|3321|4|82|5|2|7327|181|647|51|28.48|75.47|27.16|2463.81|1385.16|1452.48|3848.97|0.00|0.00|1038.87|1385.16|1385.16|2424.03|2424.03|-67.32| +2450827|35847|2450862|70095|60237|779|3321|70095|60237|779|3321|4|55|5|2|17485|13|647|91|28.26|60.47|52.60|716.17|4786.60|2571.66|5502.77|382.92|0.00|1430.52|4786.60|5169.52|6217.12|6600.04|2214.94| +2450827|35847|2450893|70095|60237|779|3321|70095|60237|779|3321|4|105|14|5|163|21|647|22|60.37|86.93|29.55|1262.36|650.10|1328.14|1912.46|26.00|0.00|152.90|650.10|676.10|803.00|829.00|-678.04| +2450827|35847|2450914|70095|60237|779|3321|70095|60237|779|3321|4|57|9|3|12235|78|647|47|63.35|153.94|9.23|6801.37|433.81|2977.45|7235.18|17.70|212.56|3472.83|221.25|238.95|3694.08|3711.78|-2756.20| +2450827|35847|2450876|70095|60237|779|3321|70095|60237|779|3321|4|26|17|4|706|163|647|34|32.91|65.82|46.73|649.06|1588.82|1118.94|2237.88|63.55|0.00|469.88|1588.82|1652.37|2058.70|2122.25|469.88| +|||70095|60237|779|3321||60237|||4|69||1|6466||647||60.48|71.36||47.19|737.77||||0.00|54.89|737.77||792.66|822.17|72.49| +2450827|35847|2450841|70095|60237|779|3321|70095|60237|779|3321|4|33|20|3|17318|161|647|37|55.14|143.36|28.67|4243.53|1060.79|2040.18|5304.32|95.47|0.00|583.12|1060.79|1156.26|1643.91|1739.38|-979.39| +2450827|35847|2450907|70095|60237|779|3321|70095|60237|779|3321|4|15|13|1|14636|49|647|41|3.46|6.29|4.40|77.49|180.40|141.86|257.89|16.23|0.00|79.54|180.40|196.63|259.94|276.17|38.54| +2450827|35847|2450896|70095|60237|779|3321|70095|60237|779|3321|4|86|16|3|7922|185|647|88|79.12|126.59|35.44|8021.20|3118.72|6962.56|11139.92|211.76|93.56|3564.00|3025.16|3236.92|6589.16|6800.92|-3937.40| +2450827|35847|2450894|70095|60237|779|3321|70095|60237|779|3321|4|106|4|2|13190|253|647|63|18.03|19.47|17.13|147.42|1079.19|1135.89|1226.61|48.56|269.79|612.99|809.40|857.96|1422.39|1470.95|-326.49| +2450827|35847|2450861|70095|60237|779|3321|70095|60237|779|3321|4|8|19|4|5684|97|647|55|89.48|135.11|70.25|3567.30|3863.75|4921.40|7431.05|0.00|0.00|817.30|3863.75|3863.75|4681.05|4681.05|-1057.65| +2450827|78841|2450832|68050|591716|5782|39001|68050|591716|5782|39001|4|71|4|2|13108|36|648|80|4.64|8.16|0.73|594.40|58.40|371.20|652.80|0.00|0.00|38.40|58.40|58.40|96.80|96.80|-312.80| +2450827|78841|2450907|68050|591716|5782|39001|68050|591716|5782|39001|4|52|20|4|5959|90|648|63|22.21|61.29|34.32|1699.11|2162.16|1399.23|3861.27|151.35|0.00|887.67|2162.16|2313.51|3049.83|3201.18|762.93| +2450827|78841|2450844|68050|591716|5782|39001|68050|591716|5782|39001|4|54|8|1|5638|297|648|42|98.24|274.08|98.66|7367.64|4143.72|4126.08|11511.36|41.43|0.00|0.00|4143.72|4185.15|4143.72|4185.15|17.64| +2450827|78841|2450912|68050|591716|5782|39001|68050|591716|5782|39001|4|71|17|4|2732|8|648|74|85.91|103.09|13.40|6637.06|991.60|6357.34|7628.66|72.28|188.40|2288.08|803.20|875.48|3091.28|3163.56|-5554.14| +2450827|78841|2450859|68050|591716|5782|39001|68050|591716|5782|39001|4|35|20|2|11906|228|648|87|45.83|62.32|53.59|759.51|4662.33|3987.21|5421.84|0.00|4662.33|867.39|0.00|0.00|867.39|867.39|-3987.21| +2450827|78841|2450859|68050|591716|5782|39001|68050|591716|5782|39001|4|47|8|4|514|39|648|99|8.74|11.44|5.14|623.70|508.86|865.26|1132.56|35.62|0.00|520.74|508.86|544.48|1029.60|1065.22|-356.40| +2450827|78841|2450894|68050|591716|5782|39001|68050|591716|5782|39001|4|69|6|5|12190|102|648|95|64.52|123.23|32.03|8664.00|3042.85|6129.40|11706.85|60.85|0.00|702.05|3042.85|3103.70|3744.90|3805.75|-3086.55| +2450827|78841|2450895|68050|591716|5782|39001|68050|591716|5782|39001|4|67|3|2|9322|205|648|44|22.69|54.22|4.87|2171.40|214.28|998.36|2385.68|5.07|44.99|787.16|169.29|174.36|956.45|961.52|-829.07| +2450827|78841|2450845|68050|591716|5782|39001|68050|591716|5782|39001|4|21|3|1|11815|230|648|3|88.20|220.50|147.73|218.31|443.19|264.60|661.50|35.45|0.00|244.74|443.19|478.64|687.93|723.38|178.59| +2450827|78841|2450862|68050|591716|5782|39001|68050|591716|5782|39001|4|55|11|5|2528|98|648|4|79.72|90.08|18.91|284.68|75.64|318.88|360.32|3.78|0.00|162.12|75.64|79.42|237.76|241.54|-243.24| +2450827|72317|2450870|10951|448838|7078|38565|10951|448838|7078|38565|2|26|5|2|1286|158|649|14|11.22|31.30|15.65|219.10|219.10|157.08|438.20|1.20|194.99|109.48|24.11|25.31|133.59|134.79|-132.97| +2450827|72317|2450849|10951|448838|7078|38565|10951|448838|7078|38565|2|63|12|5|14258|274|649|32|64.07|148.64|19.32|4138.24|618.24|2050.24|4756.48|30.91|0.00|523.20|618.24|649.15|1141.44|1172.35|-1432.00| +2450827|72317|2450889|10951|448838|7078|38565|10951|448838|7078|38565|2|100|8|1|13087|240|649|75|75.73|121.92|113.38|640.50|8503.50|5679.75|9144.00|40.81|4421.82|2103.00|4081.68|4122.49|6184.68|6225.49|-1598.07| +2450827|72317|2450845|10951|448838|7078|38565|10951|448838|7078|38565|2|90|2|2|14734|192|649|26|76.68|184.03|40.48|3732.30|1052.48|1993.68|4784.78|39.78|389.41|1530.88|663.07|702.85|2193.95|2233.73|-1330.61| +2450827|72317|2450846|10951|448838|7078|38565|10951|448838|7078|38565|2|108|17|4|17812|3|649|4|83.98|199.03|143.30|222.92|573.20|335.92|796.12|22.92|0.00|23.88|573.20|596.12|597.08|620.00|237.28| +2450827|72317|2450844|10951|448838|7078|38565|10951|448838|7078|38565|2|28|2|4|4388|59|649|89|77.01|188.67|5.66|16287.89|503.74|6853.89|16791.63|0.00|0.00|6044.88|503.74|503.74|6548.62|6548.62|-6350.15| +2450827|72317|2450914|10951|448838|7078|38565|10951|448838|7078|38565|2|47|10|3|92|283|649|86|8.32|14.97|8.23|579.64|707.78|715.52|1287.42|0.00|0.00|128.14|707.78|707.78|835.92|835.92|-7.74| +2450827|72317|2450883|10951|448838|7078|38565|10951|448838|7078|38565|2|26|14|2|1132|128|649|67|40.55|94.88|87.28|509.20|5847.76|2716.85|6356.96|58.47|0.00|2479.00|5847.76|5906.23|8326.76|8385.23|3130.91| +2450827|72317|2450914|10951|448838|7078|38565|10951|448838|7078|38565|2|18|2|3|8282|225|649|6|45.05|74.33|60.20|84.78|361.20|270.30|445.98|3.61|0.00|49.02|361.20|364.81|410.22|413.83|90.90| +2450827|72317|2450840|10951|448838|7078|38565|10951|448838|7078|38565|2|48|1|4|151|147|649|19|60.24|112.04|112.04|0.00|2128.76|1144.56|2128.76|47.89|532.19|574.75|1596.57|1644.46|2171.32|2219.21|452.01| +2450827|43311|2450907|17546|226874|5355|3483|81688|1794492|620|12301|1|67|13|3|12886|90|650|80|68.36|184.57|31.37|12256.00|2509.60|5468.80|14765.60|25.09|0.00|5905.60|2509.60|2534.69|8415.20|8440.29|-2959.20| +2450827|43311|2450900|17546|226874|5355|3483|81688|1794492|620|12301|1|59|14|3|6013|170|650|82|18.85|25.44|1.52|1961.44|124.64|1545.70|2086.08|7.47|0.00|750.30|124.64|132.11|874.94|882.41|-1421.06| +2450827|43311|2450850|17546|226874|5355|3483|81688|1794492|620|12301|1|60|19|3|14659|3|650|68|35.71|58.92|49.49|641.24|3365.32|2428.28|4006.56|235.57|0.00|1362.04|3365.32|3600.89|4727.36|4962.93|937.04| +2450827|43311|2450893|17546|226874|5355|3483|81688|1794492|620|12301|1|22|15|2|1945|178|650|96|84.78|179.73|48.52|12596.16|4657.92|8138.88|17254.08|93.15|0.00|5520.96|4657.92|4751.07|10178.88|10272.03|-3480.96| +2450827|43311|2450875|17546|226874|5355|3483|81688|1794492|620|12301|1|102|20|1|11797|166|650|88|41.66|43.74|27.55|1424.72|2424.40|3666.08|3849.12|218.19|0.00|961.84|2424.40|2642.59|3386.24|3604.43|-1241.68| +2450827|43311|2450871|17546|226874|5355|3483|81688|1794492|620|12301|1|12|8|3|8978|171|650|59|27.28|32.73|3.92|1699.79|231.28|1609.52|1931.07|6.93|0.00|366.39|231.28|238.21|597.67|604.60|-1378.24| +2450827|43311|2450849|17546|226874|5355|3483|81688|1794492|620|12301|1|68|10|5|9404|122|650|1|21.25|27.20|0.00|27.20|0.00|21.25|27.20|0.00|0.00|6.80|0.00|0.00|6.80|6.80|-21.25| +2450827|43311|2450853|17546|226874|5355|3483|81688|1794492|620|12301|1|91|9|2|11233|183|650|2|62.71|153.63|110.61|86.04|221.22|125.42|307.26|3.05|119.45|113.68|101.77|104.82|215.45|218.50|-23.65| +2450827|43311|2450838|17546|226874|5355|3483|81688|1794492|620|12301|1|26|20|2|8804|289|650|54|82.40|115.36|5.76|5918.40|311.04|4449.60|6229.44|24.88|0.00|62.10|311.04|335.92|373.14|398.02|-4138.56| +2450827|43311|2450910|17546|226874|5355|3483|81688|1794492|620|12301|1|40|20|1|5344|221|650|32|78.89|131.74|131.74|0.00|4215.68|2524.48|4215.68|210.78|0.00|1011.52|4215.68|4426.46|5227.20|5437.98|1691.20| +2450827|43311|2450901|17546|226874|5355|3483|81688|1794492|620|12301|1|69|9|4|3013|282|650|93|67.75|155.14|43.43|10389.03|4038.99|6300.75|14428.02|40.38|0.00|4038.99|4038.99|4079.37|8077.98|8118.36|-2261.76| +2450827|23081|2450856|80917|1907353|5789|37425|80917|1907353|5789|37425|2|73|4|3|2533|8|651|91|23.65|25.30|19.98|484.12|1818.18|2152.15|2302.30|54.54|0.00|436.80|1818.18|1872.72|2254.98|2309.52|-333.97| +2450827|23081|2450851|80917|1907353|5789|37425|80917|1907353|5789|37425|2|62|18|1|2272|137|651|88|68.99|154.53|106.62|4216.08|9382.56|6071.12|13598.64|0.00|0.00|3943.28|9382.56|9382.56|13325.84|13325.84|3311.44| +2450827|23081|2450853|80917|1907353|5789|37425|80917|1907353|5789|37425|2|58|14|4|10112|4|651|62|63.07|143.16|51.53|5681.06|3194.86|3910.34|8875.92|10.54|2843.42|1597.12|351.44|361.98|1948.56|1959.10|-3558.90| +2450827|23081|2450830|80917|1907353|5789|37425|80917|1907353|5789|37425|2|64|7|1|12848|2|651|23|8.74|20.18|11.09|209.07|255.07|201.02|464.14|17.49|5.10|129.95|249.97|267.46|379.92|397.41|48.95| +2450827|23081|2450912|80917|1907353|5789|37425|80917|1907353|5789|37425|2|33|20|4|17612|77|651|11|35.46|49.99|48.49|16.50|533.39|390.06|549.89|26.66|0.00|159.39|533.39|560.05|692.78|719.44|143.33| +2450827|23081|2450896|80917|1907353|5789|37425|80917|1907353|5789|37425|2|50|19|2|8545|165|651|69|67.66|72.39|57.91|999.12|3995.79|4668.54|4994.91|319.66|0.00|2197.65|3995.79|4315.45|6193.44|6513.10|-672.75| +2450827|23081|2450917|80917|1907353|5789|37425|80917|1907353|5789|37425|2|53|11|5|12484|236|651|55|41.30|63.60|2.54|3358.30|139.70|2271.50|3498.00|2.79|0.00|1574.10|139.70|142.49|1713.80|1716.59|-2131.80| +2450827|23081|2450867|80917|1907353|5789|37425|80917|1907353|5789|37425|2|39|20|1|12169|251|651|68|24.55|38.05|24.35|931.60|1655.80|1669.40|2587.40|149.02|0.00|853.40|1655.80|1804.82|2509.20|2658.22|-13.60| +2450827|23081|2450870|80917|1907353|5789|37425|80917|1907353|5789|37425|2|71|8|5|13900|187|651|66|87.38|166.89|110.14|3745.50|7269.24|5767.08|11014.74|145.38|0.00|2533.08|7269.24|7414.62|9802.32|9947.70|1502.16| +2450827|23081|2450915|80917|1907353|5789|37425|80917|1907353|5789|37425|2|93|5|3|398|23|651|5|39.25|69.86|25.84|220.10|129.20|196.25|349.30|0.00|0.00|108.25|129.20|129.20|237.45|237.45|-67.05| +2450827|23081|2450852|80917|1907353|5789|37425|80917|1907353|5789|37425|2|19|19|3|4303|13|651|60|66.96|175.43|31.57|8631.60|1894.20|4017.60|10525.80|116.68|435.66|5157.60|1458.54|1575.22|6616.14|6732.82|-2559.06| +2450827|23081|2450905|80917|1907353|5789|37425|80917|1907353|5789|37425|2|7|17|3|15026|182|651|51|46.54|66.55|29.28|1900.77|1493.28|2373.54|3394.05|119.46|0.00|0.00|1493.28|1612.74|1493.28|1612.74|-880.26| +2450827||2450897|80917|1907353|5789|37425|80917|||37425|2||12||11038||651|30||151.15||90.90|4443.60|2190.60||133.30||861.30|||5304.90|5438.20|2253.00| +2450827|71477|2450846|71331|1343465|3599|27733|71331|1343465|3599|27733|1|79|4|5|5320|115|652|81|87.20|176.14|103.92|5849.82|8417.52|7063.20|14267.34|168.35|0.00|1854.09|8417.52|8585.87|10271.61|10439.96|1354.32| +2450827|71477|2450898|71331|1343465|3599|27733|71331|1343465|3599|27733|1|87|17|2|6106|240|652|41|21.50|51.60|32.50|783.10|1332.50|881.50|2115.60|119.92|0.00|613.36|1332.50|1452.42|1945.86|2065.78|451.00| +2450827|71477|2450854|71331|1343465|3599|27733|71331|1343465|3599|27733|1|58|17|2|16804|65|652|36|56.21|109.60|58.08|1854.72|2090.88|2023.56|3945.60|104.54|0.00|1775.52|2090.88|2195.42|3866.40|3970.94|67.32| +2450827|71477|2450874|71331|1343465|3599|27733|71331|1343465|3599|27733|1|81|10|2|14738|190|652|70|37.43|56.14|19.08|2594.20|1335.60|2620.10|3929.80|66.78|0.00|471.10|1335.60|1402.38|1806.70|1873.48|-1284.50| +2450827|71477|2450849|71331|1343465|3599|27733|71331|1343465|3599|27733|1|52|3|5|5509|47|652|18|51.41|152.68|129.77|412.38|2335.86|925.38|2748.24|163.51|0.00|1319.04|2335.86|2499.37|3654.90|3818.41|1410.48| +2450827|71477|2450866|71331|1343465|3599|27733|71331|1343465|3599|27733|1|52|1|2|5620|7|652|78|24.77|60.19|0.00|4694.82|0.00|1932.06|4694.82|0.00|0.00|281.58|0.00|0.00|281.58|281.58|-1932.06| +2450827|71477|2450842|71331|1343465|3599|27733|71331|1343465|3599|27733|1|57|12|1|1105|229|652|86|88.54|262.07|13.10|21411.42|1126.60|7614.44|22538.02|101.39|0.00|9691.34|1126.60|1227.99|10817.94|10919.33|-6487.84| +2450827|71477|2450880|71331|1343465|3599|27733|71331|1343465|3599|27733|1|70|20|5|5900|146|652|95|49.85|141.07|138.24|268.85|13132.80|4735.75|13401.65|472.78|5253.12|669.75|7879.68|8352.46|8549.43|9022.21|3143.93| +2450827|33923|2450903|38011|392726|5841|36956|38011|392726|5841|36956|2|70|12|1|1723|282|653|50|18.78|34.74|33.35|69.50|1667.50|939.00|1737.00|28.34|250.12|17.00|1417.38|1445.72|1434.38|1462.72|478.38| +2450827|33923|2450917|38011|392726|5841|36956|38011|392726|5841|36956|2|10|2|5|9802|9|653|97|48.02|91.71|74.28|1690.71|7205.16|4657.94|8895.87|360.25|0.00|4180.70|7205.16|7565.41|11385.86|11746.11|2547.22| +2450827|33923|2450865|38011|392726|5841|36956|38011|392726|5841|36956|2|90|16|4|12013|295|653|56|11.03|20.95|8.38|703.92|469.28|617.68|1173.20|42.23|0.00|328.16|469.28|511.51|797.44|839.67|-148.40| +2450827|33923|2450839|38011|392726|5841|36956|38011|392726|5841|36956|2|83|2|2|4706|90|653|60|88.07|206.08|204.01|124.20|12240.60|5284.20|12364.80|749.12|3916.99|618.00|8323.61|9072.73|8941.61|9690.73|3039.41| +2450827|33923|2450842|38011|392726|5841|36956|38011|392726|5841|36956|2|68|4|5|17602|10|653|22|90.03|108.93|17.42|2013.22|383.24|1980.66|2396.46|11.49|0.00|1054.24|383.24|394.73|1437.48|1448.97|-1597.42| +2450827|24978|2450881|1261|546514|5814|12667|1261|546514|5814|12667|4|35|20|2|9523|290|654|96|3.05|5.73|0.40|511.68|38.40|292.80|550.08|3.45|0.00|65.28|38.40|41.85|103.68|107.13|-254.40| +2450827|24978|2450889|1261|546514|5814|12667|1261|546514|5814|12667|4|101|5|2|14954|86|654|81|91.02|129.24|115.02|1151.82|9316.62|7372.62|10468.44|0.00|0.00|627.75|9316.62|9316.62|9944.37|9944.37|1944.00| +2450827|24978|2450908|1261|546514|5814|12667|1261|546514|5814|12667|4|28|8|2|517|37|654|16|24.73|33.13|16.56|265.12|264.96|395.68|530.08|3.86|71.53|196.00|193.43|197.29|389.43|393.29|-202.25| +2450827|24978|2450844|1261|546514|5814|12667|1261|546514|5814|12667|4|8|3|2|17005|227|654|10|43.47|110.84|64.28|465.60|642.80|434.70|1108.40|6.42|0.00|476.60|642.80|649.22|1119.40|1125.82|208.10| +2450827|24978|2450904|1261|546514|5814|12667|1261|546514|5814|12667|4|67|15|1|3427|238|654|82|60.74|61.34|33.12|2314.04|2715.84|4980.68|5029.88|70.06|380.21|2162.34|2335.63|2405.69|4497.97|4568.03|-2645.05| +2450827|24978|2450900|1261|546514|5814|12667|1261|546514|5814|12667|4|17|12|5|554|186|654|60|82.20|147.13|10.29|8210.40|617.40|4932.00|8827.80|24.69|0.00|2118.60|617.40|642.09|2736.00|2760.69|-4314.60| +2450827|24978|2450867|1261|546514|5814|12667|1261|546514|5814|12667|4|9|15|1|9307|79|654|11|32.56|62.18|3.73|642.95|41.03|358.16|683.98|3.28|0.00|116.27|41.03|44.31|157.30|160.58|-317.13| +2450827|24978|2450915|1261|546514|5814|12667|1261|546514|5814|12667|4|97|8|4|7352|277|654|20|85.95|191.66|101.57|1801.80|2031.40|1719.00|3833.20|142.19|0.00|1379.80|2031.40|2173.59|3411.20|3553.39|312.40| +2450827|24978|2450854|1261|546514|5814|12667|1261|546514|5814|12667|4|50|13|2|2902|130|654|21|29.61|73.13|19.01|1136.52|399.21|621.81|1535.73|6.18|275.45|399.21|123.76|129.94|522.97|529.15|-498.05| +2450827|24978|2450915|1261|546514|5814|12667|1261|546514|5814|12667|4|78|8|1|17482|265|654|43|5.02|7.27|1.01|269.18|43.43|215.86|312.61|3.04|0.00|96.75|43.43|46.47|140.18|143.22|-172.43| +2450827|24978|2450912|1261|546514|5814|12667|1261|546514|5814|12667|4|17|13|2|14642|126|654|22|50.38|105.29|69.49|787.60|1528.78|1108.36|2316.38|30.57|0.00|509.52|1528.78|1559.35|2038.30|2068.87|420.42| +2450827|24978|2450879|1261|546514|5814|12667|1261|546514|5814|12667|4|83|1|3|11077|277|654|18|17.85|37.12|5.56|568.08|100.08|321.30|668.16|5.00|0.00|73.44|100.08|105.08|173.52|178.52|-221.22| +2450827|24978|2450878|1261|546514|5814|12667|1261|546514|5814|12667|4|95|1|4|1897|70|654|10|64.57|69.08|53.88|152.00|538.80|645.70|690.80|23.27|247.84|62.10|290.96|314.23|353.06|376.33|-354.74| +2450827|36676|2450915|42448|747837|3273|2662|42448|747837|3273|2662|1|15|7|5|13279|109|655|98|82.90|225.48|162.34|6187.72|15909.32|8124.20|22097.04|795.46|0.00|3092.88|15909.32|16704.78|19002.20|19797.66|7785.12| +2450827|36676|2450877|42448|747837|3273|2662|42448|747837|3273|2662|1|49|19|4|3997|161|655|67|66.36|90.24|72.19|1209.35|4836.73|4446.12|6046.08|43.53|3385.71|0.00|1451.02|1494.55|1451.02|1494.55|-2995.10| +2450827|36676|2450861|42448|747837|3273|2662|42448|747837|3273|2662|1|101|9|4|16664|125|655|16|93.62|174.13|168.90|83.68|2702.40|1497.92|2786.08|0.00|1080.96|83.52|1621.44|1621.44|1704.96|1704.96|123.52| +2450827|36676|2450887|42448|747837|3273|2662|42448|747837|3273|2662|1|16|7|4|17953|185|655|32|30.44|39.57|29.67|316.80|949.44|974.08|1266.24|55.54|332.30|303.68|617.14|672.68|920.82|976.36|-356.94| +2450827|36676|2450847|42448|747837|3273|2662|42448|747837|3273|2662|1|80|1|5|16478|195|655|57|6.57|13.00|6.50|370.50|370.50|374.49|741.00|3.70|0.00|266.76|370.50|374.20|637.26|640.96|-3.99| +2450827|36676|2450906|42448|747837|3273|2662|42448|747837|3273|2662|1|106|19|4|9661|279|655|59|74.71|117.29|92.65|1453.76|5466.35|4407.89|6920.11|437.30|0.00|138.06|5466.35|5903.65|5604.41|6041.71|1058.46| +2450827|36676|2450886|42448|747837|3273|2662|42448|747837|3273|2662|1|104|10|2|1849|143|655|53|84.64|125.26|120.24|266.06|6372.72|4485.92|6638.78|254.90|0.00|2256.74|6372.72|6627.62|8629.46|8884.36|1886.80| +2450827|36676|2450874|42448|747837|3273|2662|42448|747837|3273|2662|1|1|2|1|2641|125|655|34|67.59|170.32|32.36|4690.64|1100.24|2298.06|5790.88|0.00|0.00|1158.04|1100.24|1100.24|2258.28|2258.28|-1197.82| +2450827|36676|2450863|42448|747837|3273|2662|42448|747837|3273|2662|1|26|14|4|2636|247|655|9|29.22|30.97|2.47|256.50|22.23|262.98|278.73|0.44|0.00|86.40|22.23|22.67|108.63|109.07|-240.75| +2450827|36676|2450860|42448|747837|3273|2662|42448|747837|3273|2662|1|93|2|1|1993|39|655|40|20.97|54.94|53.29|66.00|2131.60|838.80|2197.60|149.21|0.00|1098.80|2131.60|2280.81|3230.40|3379.61|1292.80| +2450827|84542|2450846|78392|128253|2114|21128|78392|128253|2114|21128|4|75|1|3|5108|13|656|5|34.29|72.69|26.89|229.00|134.45|171.45|363.45|1.34|0.00|61.75|134.45|135.79|196.20|197.54|-37.00| +2450827|84542|2450906|78392|128253|2114|21128|78392|128253|2114|21128|4|90|2|5|15412|106|656|99|48.12|63.99|60.79|316.80|6018.21|4763.88|6335.01|541.63|0.00|1899.81|6018.21|6559.84|7918.02|8459.65|1254.33| +2450827|84542|2450835|78392|128253|2114|21128|78392|128253|2114|21128|4|84|19|2|3871|190|656|16|14.32|36.65|10.62|416.48|169.92|229.12|586.40|0.00|39.08|64.48|130.84|130.84|195.32|195.32|-98.28| +2450827|84542|2450888|78392|128253|2114|21128|78392|128253|2114|21128|4|33|12|1|913|291|656|40|54.59|140.84|46.47|3774.80|1858.80|2183.60|5633.60|148.70|0.00|676.00|1858.80|2007.50|2534.80|2683.50|-324.80| +2450827|84542|2450867|78392|128253|2114|21128|78392|128253|2114|21128|4|43|19|5|11876|29|656|98|74.92|188.79|37.75|14801.92|3699.50|7342.16|18501.42|147.98|0.00|7399.98|3699.50|3847.48|11099.48|11247.46|-3642.66| +2450827|84542|2450903|78392|128253|2114|21128|78392|128253|2114|21128|4|82|12|1|13759|139|656|16|1.05|2.55|0.81|27.84|12.96|16.80|40.80|0.00|0.00|4.48|12.96|12.96|17.44|17.44|-3.84| +2450827|84542|2450837|78392|128253|2114|21128|78392|128253|2114|21128|4|83|1|2|6164|192|656|52|35.18|41.86|36.83|261.56|1915.16|1829.36|2176.72|33.70|1072.48|391.56|842.68|876.38|1234.24|1267.94|-986.68| +|84542||78392|128253|2114||78392|128253|||4||||15712|251|656|91|29.63|59.26|||5068.70|||253.43|0.00|592.41||5322.13|5661.11|5914.54|2372.37| +2450827|84542|2450889|78392|128253|2114|21128|78392|128253|2114|21128|4|80|6|3|12985|58|656|20|4.34|8.94|4.91|80.60|98.20|86.80|178.80|3.92|0.00|64.20|98.20|102.12|162.40|166.32|11.40| +2450827|84542|2450830|78392|128253|2114|21128|78392|128253|2114|21128|4|90|12|4|17104|208|656|95|10.13|11.85|3.55|788.50|337.25|962.35|1125.75|6.74|0.00|77.90|337.25|343.99|415.15|421.89|-625.10| +2450827|61304|2450874|10741|1224930|1166|11802|10741|1224930|1166|11802|1|20|5|3|13441|254|657|37|7.41|13.41|2.14|416.99|79.18|274.17|496.17|3.51|20.58|128.76|58.60|62.11|187.36|190.87|-215.57| +2450827|61304|2450893|10741|1224930|1166|11802|10741|1224930|1166|11802|1|14|16|1|3235|34|657|51|12.59|31.09|5.59|1300.50|285.09|642.09|1585.59|2.85|0.00|158.10|285.09|287.94|443.19|446.04|-357.00| +2450827|61304|2450859|10741|1224930|1166|11802|10741|1224930|1166|11802|1|28|16|1|110|86|657|51|37.16|46.82|21.53|1289.79|1098.03|1895.16|2387.82|54.90|0.00|930.75|1098.03|1152.93|2028.78|2083.68|-797.13| +2450827|61304|2450907|10741|1224930|1166|11802|10741|1224930|1166|11802|1|3|20|2|16400|36|657|58|84.93|170.70|39.26|7623.52|2277.08|4925.94|9900.60|204.93|0.00|2871.00|2277.08|2482.01|5148.08|5353.01|-2648.86| +2450827|61304|2450842||1224930|1166|11802||1224930||||45||2|11186||657||20.89|51.59||||||51.16|0.00|618.40|1279.20||1897.60|1948.76|-392.00| +2450827|61304|2450880|10741|1224930|1166|11802|10741|1224930|1166|11802|1|72|8|2|16612|149|657|83|49.18|50.16|35.61|1207.65|2955.63|4081.94|4163.28|0.00|0.00|166.00|2955.63|2955.63|3121.63|3121.63|-1126.31| +2450827|61304|2450842|10741|1224930|1166|11802|10741|1224930|1166|11802|1|69|18|5|31|159|657|37|51.44|126.02|73.09|1958.41|2704.33|1903.28|4662.74|108.17|0.00|1258.74|2704.33|2812.50|3963.07|4071.24|801.05| +2450827|61304|2450879|10741|1224930|1166|11802|10741|1224930|1166|11802|1|71|10|3|746|102|657|69|8.57|21.85|2.18|1357.23|150.42|591.33|1507.65|0.00|0.00|255.99|150.42|150.42|406.41|406.41|-440.91| +2450827|61304|2450836|10741|1224930|1166|11802|10741|1224930|1166|11802|1|4|7|2|7502|272|657|40|77.88|131.61|2.63|5159.20|105.20|3115.20|5264.40|6.31|0.00|1158.00|105.20|111.51|1263.20|1269.51|-3010.00| +2450827|61304|2450907|10741|1224930|1166|11802|10741|1224930|1166|11802|1|59|6|3|5677|154|657|40|54.92|57.66|56.50|46.40|2260.00|2196.80|2306.40|135.60|0.00|1130.00|2260.00|2395.60|3390.00|3525.60|63.20| +2450827|61304|2450890|10741|1224930|1166|11802|10741|1224930|1166|11802|1|22|10|2|10672|95|657|80|7.05|14.24|2.56|934.40|204.80|564.00|1139.20|0.00|0.00|318.40|204.80|204.80|523.20|523.20|-359.20| +2450827|82824|2450845|33590|1627100|7010|39273|33590|1627100|7010|39273|4|23|2|2|1171|63|658|20|48.78|60.48|4.83|1113.00|96.60|975.60|1209.60|6.76|0.00|120.80|96.60|103.36|217.40|224.16|-879.00| +2450827|82824|2450880|33590|1627100|7010|39273|33590|1627100|7010|39273|4|15|20|5|11701|60|658|16|90.87|125.40|84.01|662.24|1344.16|1453.92|2006.40|67.20|0.00|401.28|1344.16|1411.36|1745.44|1812.64|-109.76| +2450827|82824|2450909|33590|1627100|7010|39273|33590|1627100|7010|39273|4|26|9|2|7840|198|658|44|83.16|187.94|180.42|330.88|7938.48|3659.04|8269.36|25.40|5398.16|3307.48|2540.32|2565.72|5847.80|5873.20|-1118.72| +2450827|82824|2450831|33590|1627100|7010|39273|33590|1627100|7010|39273|4|90|15|4|14402|143|658|18|46.51|71.16|2.84|1229.76|51.12|837.18|1280.88|0.51|0.00|38.34|51.12|51.63|89.46|89.97|-786.06| +2450827|82824|2450851|33590|1627100|7010|39273|33590|1627100|7010|39273|4|29|11|4|3776|175|658|77|39.14|45.40|26.33|1468.39|2027.41|3013.78|3495.80|101.37|0.00|244.09|2027.41|2128.78|2271.50|2372.87|-986.37| +2450827|82824|2450911|33590|1627100|7010|39273|33590|1627100|7010|39273|4|70|13|1|16147|40|658|8|28.10|68.28|23.21|360.56|185.68|224.80|546.24|0.00|109.55|0.00|76.13|76.13|76.13|76.13|-148.67| +2450827|82824|2450884|33590|1627100|7010|39273|33590|1627100|7010|39273|4|57|8|2|13304|78|658|22|28.79|30.22|1.81|625.02|39.82|633.38|664.84|1.19|0.00|86.24|39.82|41.01|126.06|127.25|-593.56| +2450827|82824|2450837|33590|1627100|7010|39273|33590|1627100|7010|39273|4|42|13|3|11594|161|658|80|7.86|8.96|3.40|444.80|272.00|628.80|716.80|24.48|0.00|13.60|272.00|296.48|285.60|310.08|-356.80| +2450827|82824|2450857|33590|1627100|7010|39273|33590|1627100|7010|39273|4|80|10|2|16585|294|658|95|82.84|218.69|159.64|5609.75|15165.80|7869.80|20775.55|151.65|0.00|830.30|15165.80|15317.45|15996.10|16147.75|7296.00| +2450827|64075|2450892|15291|19699|2507|5362|15291|19699|2507|5362|2|66|8|1|5450|59|659|74|50.86|145.96|99.25|3456.54|7344.50|3763.64|10801.04|249.71|2350.24|1187.70|4994.26|5243.97|6181.96|6431.67|1230.62| +2450827|64075|2450897|15291|19699|2507|5362|15291|19699|2507|5362|2|48|1|5|16735|65|659|38|70.34|99.88|39.95|2277.34|1518.10|2672.92|3795.44|30.36|0.00|910.86|1518.10|1548.46|2428.96|2459.32|-1154.82| +2450827|64075|2450854|15291|19699|2507|5362|15291|19699|2507|5362|2|23|11|4|8168|168|659|18|24.74|35.87|34.43|25.92|619.74|445.32|645.66|37.18|0.00|148.50|619.74|656.92|768.24|805.42|174.42| +2450827|64075|2450907|15291|19699|2507|5362|15291|19699|2507|5362|2|19|15|3|716|151|659|59|84.14|118.63|21.35|5739.52|1259.65|4964.26|6999.17|31.36|214.14|2099.22|1045.51|1076.87|3144.73|3176.09|-3918.75| +2450827|64075|2450864|15291|19699|2507|5362|15291|19699|2507|5362|2|54|14|5|2323|224|659|52|35.80|58.35|26.25|1669.20|1365.00|1861.60|3034.20|15.28|600.60|0.00|764.40|779.68|764.40|779.68|-1097.20| +2450827|38202|2450906|50000|1852514|6228|39142|56779|674570|1884|7998|4|47|5|2|7516|147|660|19|35.54|64.68|49.15|295.07|933.85|675.26|1228.92|65.36|0.00|135.09|933.85|999.21|1068.94|1134.30|258.59| +2450827|38202|2450915|50000|1852514|6228|39142|56779|674570|1884|7998|4|4|17|4|3295|168|660|71|62.28|163.79|137.58|1860.91|9768.18|4421.88|11629.09|586.09|0.00|4419.04|9768.18|10354.27|14187.22|14773.31|5346.30| +2450827|38202|2450890|50000|1852514|6228|39142|56779|674570|1884|7998|4|75|19|2|8606|281|660|7|63.25|154.96|139.46|108.50|976.22|442.75|1084.72|87.85|0.00|249.48|976.22|1064.07|1225.70|1313.55|533.47| +2450827|38202|2450878|50000|1852514|6228|39142|56779|674570|1884|7998|4|90|2|1|8557|295|660|8|18.88|28.50|15.67|102.64|125.36|151.04|228.00|2.50|0.00|77.52|125.36|127.86|202.88|205.38|-25.68| +2450827|38202|2450862|50000|1852514|6228|39142|56779|674570|1884|7998|4|108|10|3|9901|142|660|27|99.37|224.57|130.25|2546.64|3516.75|2682.99|6063.39|35.16|0.00|2728.35|3516.75|3551.91|6245.10|6280.26|833.76| +2450827|38202|2450864|50000|1852514|6228|39142|56779|674570|1884|7998|4|76|9|2|12742|300|660|6|11.58|31.03|7.44|141.54|44.64|69.48|186.18|0.00|0.00|31.62|44.64|44.64|76.26|76.26|-24.84| +2450827|39012|2450879|17972|175320|3515|32910|86185|179300|5895|14043|1|74|15|5|9163|203|661|28|12.11|12.59|9.94|74.20|278.32|339.08|352.52|11.13|0.00|109.20|278.32|289.45|387.52|398.65|-60.76| +2450827|39012|2450829|17972|175320|3515|32910|86185|179300|5895|14043|1|80|4|5|1294|175|661|12|35.85|84.60|6.76|934.08|81.12|430.20|1015.20|3.24|0.00|345.12|81.12|84.36|426.24|429.48|-349.08| +2450827|39012|2450856|17972|175320|3515|32910|86185|179300|5895|14043|1|83|6|4|17746|50|661|62|32.98|70.24|33.01|2308.26|2046.62|2044.76|4354.88|0.00|0.00|652.86|2046.62|2046.62|2699.48|2699.48|1.86| +2450827|39012|2450895|17972|175320|3515|32910|86185|179300|5895|14043|1|86|10|4|7028|223|661|85|1.32|1.34|0.16|100.30|13.60|112.20|113.90|0.95|0.00|14.45|13.60|14.55|28.05|29.00|-98.60| +2450827|39012|2450835|17972|175320|3515|32910|86185|179300|5895|14043|1|27|20|3|10807|76|661|31|69.15|174.25|115.00|1836.75|3565.00|2143.65|5401.75|106.95|0.00|1728.56|3565.00|3671.95|5293.56|5400.51|1421.35| +2450827|39012|2450900|17972|175320|3515|32910|86185|179300|5895|14043|1|52|19|4|7774|149|661|56|61.00|165.31|137.20|1574.16|7683.20|3416.00|9257.36|460.99|0.00|832.72|7683.20|8144.19|8515.92|8976.91|4267.20| +2450827|39012|2450851|17972|175320|3515|32910|86185|179300|5895|14043|1|64|1|5|15482|296|661|33|73.54|148.55|145.57|98.34|4803.81|2426.82|4902.15|288.22|0.00|980.43|4803.81|5092.03|5784.24|6072.46|2376.99| +2450827|39012|2450896|17972|175320|3515|32910|86185|179300|5895|14043|1|75|10|1|9835|10|661|60|65.31|150.86|4.52|8780.40|271.20|3918.60|9051.60|16.27|0.00|3620.40|271.20|287.47|3891.60|3907.87|-3647.40| +2450827|39012|2450905|17972|175320|3515|32910|86185|179300|5895|14043|1|67|5|5|16436|8|661|12|79.45|98.51|68.95|354.72|827.40|953.40|1182.12|57.91|0.00|153.60|827.40|885.31|981.00|1038.91|-126.00| +2450827|39012|2450859|17972|175320|3515|32910|86185|179300|5895|14043|1|73|11|4|14941|202|661|22|5.59|7.93|6.74|26.18|148.28|122.98|174.46|13.34|0.00|41.80|148.28|161.62|190.08|203.42|25.30| +2450827|39012|2450848|17972|175320|3515|32910|86185|179300|5895|14043|1|9|11|3|10844|27|661|97|71.10|136.51|76.44|5826.79|7414.68|6896.70|13241.47|51.90|4819.54|3044.83|2595.14|2647.04|5639.97|5691.87|-4301.56| +2450827|39012|2450859|17972|175320|3515|32910|86185|179300|5895|14043|1|55|14|3|11083|194|661|97|75.11|155.47|40.42|11159.85|3920.74|7285.67|15080.59|117.62|0.00|1809.05|3920.74|4038.36|5729.79|5847.41|-3364.93| +2450827|39012|2450859|17972|175320|3515|32910|86185|179300|5895|14043|1|98|4|1|6193|17|661|17|40.80|120.76|2.41|2011.95|40.97|693.60|2052.92|0.40|0.00|738.99|40.97|41.37|779.96|780.36|-652.63| +2450827|39012|2450908|17972|175320|3515|32910|86185|179300|5895|14043|1|37|17|4|6163|113|661|11|20.62|22.26|6.67|171.49|73.37|226.82|244.86|2.72|5.13|2.42|68.24|70.96|70.66|73.38|-158.58| +2450827|64171|2450906|2366|1509504|2689|13027|2366|1509504|2689|13027|4|93|3|1|14920|186|662|6|92.66|260.37|177.05|499.92|1062.30|555.96|1562.22|74.36|0.00|702.96|1062.30|1136.66|1765.26|1839.62|506.34| +2450827|64171|2450907|2366|1509504|2689|13027|2366|1509504|2689|13027|4|93|4|4|17438|2|662|44|84.50|130.13|19.51|4867.28|858.44|3718.00|5725.72|60.09|0.00|2576.20|858.44|918.53|3434.64|3494.73|-2859.56| +2450827|64171|2450904|2366|1509504|2689|13027|2366|1509504|2689|13027|4|26|7|4|9020|288|662|70|21.35|51.02|23.97|1893.50|1677.90|1494.50|3571.40|134.23|0.00|1320.90|1677.90|1812.13|2998.80|3133.03|183.40| +2450827|64171|2450836|2366|1509504|2689|13027|2366|1509504|2689|13027|4|65|9|5|14924|177|662|58|22.02|47.34|7.10|2333.92|411.80|1277.16|2745.72|12.02|111.18|329.44|300.62|312.64|630.06|642.08|-976.54| +2450827|64171|2450912|2366|1509504|2689|13027|2366|1509504|2689|13027|4|17|11|1|9775|134|662|91|92.69|155.71|1.55|14028.56|141.05|8434.79|14169.61|0.00|5.64|283.01|135.41|135.41|418.42|418.42|-8299.38| +2450827|64171|2450903|2366|1509504|2689|13027|2366|1509504|2689|13027|4|54|15|2|4021|146|662|9|88.70|245.69|213.75|287.46|1923.75|798.30|2211.21|153.90|0.00|0.00|1923.75|2077.65|1923.75|2077.65|1125.45| +2450827|64171|2450890|2366|1509504|2689|13027|2366|1509504|2689|13027|4|61|6|4|2540|157|662|86|60.42|157.09|20.42|11753.62|1756.12|5196.12|13509.74|70.24|0.00|3377.22|1756.12|1826.36|5133.34|5203.58|-3440.00| +2450827|64171|2450840|2366|1509504|2689|13027|2366|1509504|2689|13027|4|56|12|4|1016|70|662|4|35.58|37.35|31.74|22.44|126.96|142.32|149.40|1.86|64.74|59.76|62.22|64.08|121.98|123.84|-80.10| +2450827|64171|2450887|2366|1509504|2689|13027|2366|1509504|2689|13027|4|22|11|1|11182|274|662|100|14.21|33.25|20.61|1264.00|2061.00|1421.00|3325.00|185.49|0.00|1629.00|2061.00|2246.49|3690.00|3875.49|640.00| +2450827|64171|2450902|2366|1509504|2689|13027|2366|1509504|2689|13027|4|105|3|3|2456|75|662|62|22.48|63.61|0.63|3904.76|39.06|1393.76|3943.82|1.17|0.00|1695.70|39.06|40.23|1734.76|1735.93|-1354.70| +2450827|64171|2450867|2366|1509504|2689|13027|2366|1509504|2689|13027|4|43|5|4|2018|212|662|58|70.25|174.22|59.23|6669.42|3435.34|4074.50|10104.76|240.47|0.00|1717.38|3435.34|3675.81|5152.72|5393.19|-639.16| +2450827|64171|2450912|2366|1509504|2689|13027|2366|1509504|2689|13027|4|9|16|5|8660|264|662|83|57.41|91.28|14.60|6364.44|1211.80|4765.03|7576.24|66.16|266.59|1136.27|945.21|1011.37|2081.48|2147.64|-3819.82| +2450827|22089|2450870|87150|1839393|1236|22279|87150|1839393|1236|22279|1|8|8|5|7429|25|663|20|46.60|73.16|38.04|702.40|760.80|932.00|1463.20|60.86|0.00|629.00|760.80|821.66|1389.80|1450.66|-171.20| +2450827|22089|2450899|87150|1839393|1236|22279|87150|1839393|1236|22279|1|17|19|4|2164|152|663|95|81.79|172.57|84.55|8361.90|8032.25|7770.05|16394.15|401.61|0.00|2130.85|8032.25|8433.86|10163.10|10564.71|262.20| +2450827|22089|2450882|87150|1839393|1236|22279|87150|1839393|1236|22279|1|3|13|3|12530|95|663|11|22.36|52.32|49.18|34.54|540.98|245.96|575.52|48.68|0.00|241.67|540.98|589.66|782.65|831.33|295.02| +2450827|22089|2450892|87150|1839393|1236|22279|87150|1839393|1236|22279|1|19|12|5|6644|23|663|28|45.01|86.41|65.67|580.72|1838.76|1260.28|2419.48|8.45|1415.84|435.40|422.92|431.37|858.32|866.77|-837.36| +2450827|22089|2450887|87150|1839393|1236|22279|87150|1839393|1236|22279|1|2|5|5|3949|164|663|47|36.78|106.66|35.19|3359.09|1653.93|1728.66|5013.02|33.07|0.00|2205.71|1653.93|1687.00|3859.64|3892.71|-74.73| +2450827||||798965|4925|1332||||||5||4|4543|139|664|62||114.20|83.36||5168.32|2821.00|||||5168.32||5663.70|5922.11|2347.32| +2450827|45719|2450891|24205|798965|4925|1332|24205|798965|4925|1332|2|11|12|5|9632|126|664|34|81.47|227.30|38.64|6414.44|1313.76|2769.98|7728.20|29.16|827.66|2395.64|486.10|515.26|2881.74|2910.90|-2283.88| +2450827|45719|2450862|24205|798965|4925|1332|24205|798965|4925|1332|2|105|3|4|9292|210|664|45|60.30|119.39|88.34|1397.25|3975.30|2713.50|5372.55|50.88|1431.10|859.50|2544.20|2595.08|3403.70|3454.58|-169.30| +2450827|45719|2450849|24205|798965|4925|1332|24205|798965|4925|1332|2|55|14|4|7988|205|664|85|15.20|29.48|12.38|1453.50|1052.30|1292.00|2505.80|42.09|0.00|550.80|1052.30|1094.39|1603.10|1645.19|-239.70| +2450827|45719|2450878|24205|798965|4925|1332|24205|798965|4925|1332|2|45|1|3|5618|223|664|48|96.98|110.55|8.84|4882.08|424.32|4655.04|5306.40|28.51|67.89|0.00|356.43|384.94|356.43|384.94|-4298.61| +2450827|45719|2450870|24205|798965|4925|1332|24205|798965|4925|1332|2|6|6|1|7813|114|664|92|69.70|145.67|36.41|10051.92|3349.72|6412.40|13401.64|180.88|1339.88|3751.76|2009.84|2190.72|5761.60|5942.48|-4402.56| +2450827|45719|2450843|24205|798965|4925|1332|24205|798965|4925|1332|2|34|3|3|5408|281|664|60|94.14|181.69|52.69|7740.00|3161.40|5648.40|10901.40|94.84|0.00|3815.40|3161.40|3256.24|6976.80|7071.64|-2487.00| +2450827|45719|2450874|24205|798965|4925|1332|24205|798965|4925|1332|2|3|11|4|1658|212|664|37|85.23|253.13|35.43|8054.90|1310.91|3153.51|9365.81|91.76|0.00|4027.08|1310.91|1402.67|5337.99|5429.75|-1842.60| +2450827|45719|2450912|24205|798965|4925|1332|24205|798965|4925|1332|2|13|20|2|8798|178|664|77|27.40|44.66|26.34|1410.64|2028.18|2109.80|3438.82|20.28|0.00|790.79|2028.18|2048.46|2818.97|2839.25|-81.62| +2450828|59799|2450853|49028|986318|2361|26004|49028|986318|2361|26004|2|40|3|5|254|153|665|43|62.68|184.90|134.97|2146.99|5803.71|2695.24|7950.70|136.38|3075.96|3816.25|2727.75|2864.13|6544.00|6680.38|32.51| +2450828|59799|2450915|49028|986318|2361|26004|49028|986318|2361|26004|2|19|17|4|7982|275|665|92|65.56|68.83|56.44|1139.88|5192.48|6031.52|6332.36|467.32|0.00|2152.80|5192.48|5659.80|7345.28|7812.60|-839.04| +2450828|59799|2450900|49028|986318|2361|26004|49028|986318|2361|26004|2|79|12|5|902|104|665|65|89.06|100.63|54.34|3008.85|3532.10|5788.90|6540.95|35.32|0.00|2484.95|3532.10|3567.42|6017.05|6052.37|-2256.80| +2450828|59799|2450832|49028|986318|2361|26004|49028|986318|2361|26004|2|86|13|5|3052|187|665|2|37.87|48.09|30.29|35.60|60.58|75.74|96.18|4.84|0.00|30.76|60.58|65.42|91.34|96.18|-15.16| +2450828|59799|2450861|49028|986318|2361|26004|49028|986318|2361|26004|2|60|10|3|17098|70|665|60|74.33|109.26|69.92|2360.40|4195.20|4459.80|6555.60|0.00|1132.70|0.00|3062.50|3062.50|3062.50|3062.50|-1397.30| +2450828|59799|2450839|49028|986318|2361|26004|49028|986318|2361|26004|2|44|4|2|136|145|665|21|32.79|62.62|39.45|486.57|828.45|688.59|1315.02|13.58|149.12|249.69|679.33|692.91|929.02|942.60|-9.26| +2450828|59799|2450842|49028|986318|2361|26004|49028|986318|2361|26004|2|53|13|5|1996|218|665|33|59.67|140.82|80.26|1998.48|2648.58|1969.11|4647.06|0.00|0.00|2230.47|2648.58|2648.58|4879.05|4879.05|679.47| +2450828|59799|2450877|49028|986318|2361|26004|49028|986318|2361|26004|2|87|20|2|12230|10|665|47|13.96|33.36|3.00|1426.92|141.00|656.12|1567.92|0.00|0.00|376.00|141.00|141.00|517.00|517.00|-515.12| +2450828|59799|2450874|49028|986318|2361|26004|49028|986318|2361|26004|2|104|12|3|14488|58|665|14|39.29|63.25|37.31|363.16|522.34|550.06|885.50|5.74|407.42|44.24|114.92|120.66|159.16|164.90|-435.14| +2450828|59799|2450865|49028|986318|2361|26004|49028|986318|2361|26004|2|16|7|2|205|162|665|25|29.02|69.93|56.64|332.25|1416.00|725.50|1748.25|70.80|0.00|699.25|1416.00|1486.80|2115.25|2186.05|690.50| +2450828|59799|2450876|49028|986318|2361|26004|49028|986318|2361|26004|2|81|16|1|6688|88|665|73|69.28|75.51|12.08|4630.39|881.84|5057.44|5512.23|70.54|0.00|1432.99|881.84|952.38|2314.83|2385.37|-4175.60| +2450828|59799|2450832|49028|986318|2361|26004|49028|986318|2361|26004|2|33|9|1|427|275|665|93|43.58|108.07|65.92|3919.95|6130.56|4052.94|10050.51|0.00|0.00|3316.38|6130.56|6130.56|9446.94|9446.94|2077.62| +2450828|59799|2450847|49028|986318|2361|26004|49028|986318|2361|26004|2|93|1|1|2935|48|665|55|34.74|80.94|78.51|133.65|4318.05|1910.70|4451.70|215.90|0.00|1290.85|4318.05|4533.95|5608.90|5824.80|2407.35| +2450828|59799|2450830|49028|986318|2361|26004|49028|986318|2361|26004|2|30|13|1|6643|54|665|82|53.85|154.54|131.35|1901.58|10770.70|4415.70|12672.28|107.70|0.00|2787.18|10770.70|10878.40|13557.88|13665.58|6355.00| +2450828|35565|2450882|86845|1555917|6236|41755|86845|1555917|6236|41755|2|62|19|4|13442|252|666|72|96.13|234.55|119.62|8274.96|8612.64|6921.36|16887.60|775.13|0.00|3546.00|8612.64|9387.77|12158.64|12933.77|1691.28| +2450828|35565|2450905|86845|1555917|6236|41755|86845|1555917|6236|41755|2|32|1|1|1501|296|666|70|41.53|81.39|51.27|2108.40|3588.90|2907.10|5697.30|43.06|2871.12|341.60|717.78|760.84|1059.38|1102.44|-2189.32| +2450828|35565|2450866|86845|1555917|6236|41755|86845|1555917|6236|41755|2|81|16|2|16498|216|666|12|1.30|1.62|0.68|11.28|8.16|15.60|19.44|0.48|0.00|8.88|8.16|8.64|17.04|17.52|-7.44| +2450828|35565|2450830|86845|1555917|6236|41755|86845|1555917|6236|41755|2|34|13|3|12604|85|666|66|45.88|78.45|49.42|1915.98|3261.72|3028.08|5177.70|260.93|0.00|0.00|3261.72|3522.65|3261.72|3522.65|233.64| +2450828|35565|2450838|86845|1555917|6236|41755|86845|1555917|6236|41755|2|85|15|3|10573|156|666|36|54.39|56.02|11.76|1593.36|423.36|1958.04|2016.72|16.93|0.00|181.44|423.36|440.29|604.80|621.73|-1534.68| +2450828|35565|2450857|86845|1555917|6236|41755|86845|1555917|6236|41755|2|23|7|2|13129|25|666|8|43.52|78.33|69.71|68.96|557.68|348.16|626.64|50.19|0.00|62.64|557.68|607.87|620.32|670.51|209.52| +2450828|35565|2450834|86845|1555917|6236|41755|86845|1555917|6236|41755|2|72|9|2|13778|78|666|36|62.42|144.19|17.30|4568.04|622.80|2247.12|5190.84|37.36|0.00|986.04|622.80|660.16|1608.84|1646.20|-1624.32| +2450828|35565|2450835|86845|1555917|6236|41755|86845|1555917|6236|41755|2|43|5|1|4204|230|666|69|77.62|211.90|112.30|6872.40|7748.70|5355.78|14621.10|154.97|0.00|3216.09|7748.70|7903.67|10964.79|11119.76|2392.92| +2450828|35565|2450862|86845|1555917|6236|41755|86845|1555917|6236|41755|2|29|15|2|5632|230|666|77|4.35|12.00|3.72|637.56|286.44|334.95|924.00|11.60|157.54|351.12|128.90|140.50|480.02|491.62|-206.05| +2450828|35565|2450865|86845|1555917|6236|41755|86845|1555917|6236|41755|2|20|16|2|10976|297|666|23|21.02|21.02|21.02|0.00|483.46|483.46|483.46|8.99|333.58|120.75|149.88|158.87|270.63|279.62|-333.58| +2450828|35565|2450839|86845|1555917|6236|41755|86845|1555917|6236|41755|2|74|13|3|7189|211|666|79|39.13|87.65|0.00|6924.35|0.00|3091.27|6924.35|0.00|0.00|2215.16|0.00|0.00|2215.16|2215.16|-3091.27| +2450828|35565|2450858|86845|1555917|6236|41755|86845|1555917|6236|41755|2|46|8|3|17695|273|666|1|84.58|239.36|4.78|234.58|4.78|84.58|239.36|0.00|0.00|114.89|4.78|4.78|119.67|119.67|-79.80| +2450828|35565|2450902|86845|1555917|6236|41755|86845|1555917|6236|41755|2|17|10|1|16216|165|666|62|74.97|141.69|126.10|966.58|7818.20|4648.14|8784.78|234.54|0.00|1141.42|7818.20|8052.74|8959.62|9194.16|3170.06| +|35565|2450879|86845|||41755|||6236||2|70|14||2701|57|666|||233.76||1105.96|8945.72|3490.31||626.20|||8945.72||13770.32|14396.52|5455.41| +2450828|49783|2450838|57266|1605100|3334|43568|57266|1605100|3334|43568|2|72|7|3|9778|32|667|96|55.94|58.73|28.77|2876.16|2761.92|5370.24|5638.08|11.04|2209.53|2198.40|552.39|563.43|2750.79|2761.83|-4817.85| +2450828|49783|2450882|57266|1605100|3334|43568|57266|1605100|3334|43568|2|33|10|5|1291|207|667|72|3.37|9.90|6.23|264.24|448.56|242.64|712.80|31.39|0.00|192.24|448.56|479.95|640.80|672.19|205.92| +2450828|49783|2450903|57266|1605100|3334|43568|57266|1605100|3334|43568|2|50|19|4|6722|234|667|7|7.39|15.81|0.00|110.67|0.00|51.73|110.67|0.00|0.00|52.01|0.00|0.00|52.01|52.01|-51.73| +2450828|49783|2450871|57266|1605100|3334|43568|57266|1605100|3334|43568|2|67|2|2|11029|113|667|13|33.23|72.10|44.70|356.20|581.10|431.99|937.30|46.48|0.00|393.64|581.10|627.58|974.74|1021.22|149.11| +2450828|49783|2450915|57266|1605100|3334|43568|57266|1605100|3334|43568|2|93|12|5|7381|41|667|3|17.58|38.32|27.59|32.19|82.77|52.74|114.96|3.31|0.00|20.67|82.77|86.08|103.44|106.75|30.03| +2450828|49783|2450840|57266|1605100|3334|43568|57266|1605100|3334|43568|2|89|14|5|14540|219|667|33|25.31|66.56|8.65|1911.03|285.45|835.23|2196.48|14.27|0.00|241.56|285.45|299.72|527.01|541.28|-549.78| +2450828|49783|2450831|57266|1605100|3334|43568|57266|1605100|3334|43568|2|103|2|5|5821|276|667|94|25.46|62.12|54.04|759.52|5079.76|2393.24|5839.28|457.17|0.00|2160.12|5079.76|5536.93|7239.88|7697.05|2686.52| +2450828|49783|2450912|57266|1605100|3334|43568|57266|1605100|3334|43568|2|90|3|5|6002|272|667|46|69.05|96.67|19.33|3557.64|889.18|3176.30|4446.82|35.56|0.00|622.38|889.18|924.74|1511.56|1547.12|-2287.12| +2450828|49783|2450856|57266|1605100|3334|43568|57266|1605100|3334|43568|2|31|20|3|6008|44|667|76|63.40|190.20|190.20|0.00|14455.20|4818.40|14455.20|358.48|5492.97|577.60|8962.23|9320.71|9539.83|9898.31|4143.83| +2450828|43447|2450911|78965|962577|6189|25401|78965|962577|6189|25401|1|8|4|2|4040|43|668|23|18.38|45.76|12.35|768.43|284.05|422.74|1052.48|11.36|0.00|526.24|284.05|295.41|810.29|821.65|-138.69| +2450828|43447|2450898|78965|962577|6189|25401|78965|962577|6189|25401|1|90|17|3|4177|125|668|81|47.45|100.59|16.09|6844.50|1303.29|3843.45|8147.79|117.29|0.00|2688.39|1303.29|1420.58|3991.68|4108.97|-2540.16| +2450828|43447|2450831|78965|962577|6189|25401|78965|962577|6189|25401|1|67|13|3|3898|125|668|53|33.81|40.91|0.81|2125.30|42.93|1791.93|2168.23|1.08|24.89|151.58|18.04|19.12|169.62|170.70|-1773.89| +2450828|43447|2450840|78965|962577|6189|25401|78965|962577|6189|25401|1|88|20|2|2293|168|668|63|16.94|50.82|12.70|2401.56|800.10|1067.22|3201.66|64.00|0.00|927.99|800.10|864.10|1728.09|1792.09|-267.12| +2450828|43447|2450894|78965|962577|6189|25401|78965|962577|6189|25401|1|15|17|1|3913|43|668|26|8.16|24.07|20.94|81.38|544.44|212.16|625.82|38.11|0.00|250.12|544.44|582.55|794.56|832.67|332.28| +2450828|43447|2450869|78965|962577|6189|25401|78965|962577|6189|25401|1|7|20|5|14066|37|668|55|11.67|14.82|8.15|366.85|448.25|641.85|815.10|26.89|0.00|195.25|448.25|475.14|643.50|670.39|-193.60| +2450828|43447|2450867|78965|962577|6189|25401|78965|962577|6189|25401|1|73|3|4|11822|81|668|38|75.16|139.04|63.95|2853.42|2430.10|2856.08|5283.52|72.90|0.00|369.74|2430.10|2503.00|2799.84|2872.74|-425.98| +2450828|37250|2450918|58248|1002451|6654|43257|58248|1002451|6654|43257|1|44|12|1|6961|97|669|76|46.84|72.60|65.34|551.76|4965.84|3559.84|5517.60|446.92|0.00|1434.12|4965.84|5412.76|6399.96|6846.88|1406.00| +2450828|37250|2450889|58248|1002451|6654|43257|58248|1002451|6654|43257|1|75|13|2|4399|78|669|85|54.91|57.65|14.41|3675.40|1224.85|4667.35|4900.25|12.24|0.00|1518.95|1224.85|1237.09|2743.80|2756.04|-3442.50| +2450828|37250|2450888|58248|1002451|6654|43257|58248|1002451|6654|43257|1|3|14|3|10006|106|669|88|2.64|7.02|5.61|124.08|493.68|232.32|617.76|4.93|0.00|123.20|493.68|498.61|616.88|621.81|261.36| +2450828|37250|2450841|58248|1002451|6654|43257|58248|1002451|6654|43257|1|18|16|5|12313|197|669|9|54.37|157.12|150.83|56.61|1357.47|489.33|1414.08|54.29|0.00|339.30|1357.47|1411.76|1696.77|1751.06|868.14| +2450828|37250|2450898|58248|1002451|6654|43257|58248|1002451|6654|43257|1|85|15|1|14230|109|669|49|19.93|25.90|19.42|317.52|951.58|976.57|1269.10|19.03|0.00|558.11|951.58|970.61|1509.69|1528.72|-24.99| +2450828|27555|2450856|81923|1884003|4130|15810|81923|1884003|4130|15810|1|86|7|2|9784|185|670|73|49.64|122.11|106.23|1159.24|7754.79|3623.72|8914.03|697.93|0.00|2495.87|7754.79|8452.72|10250.66|10948.59|4131.07| +2450828|27555|2450910|81923|1884003|4130|15810|81923|1884003|4130|15810|1|38|8|4|2647|214|670|43|39.81|75.63|68.82|292.83|2959.26|1711.83|3252.09|177.55|0.00|97.18|2959.26|3136.81|3056.44|3233.99|1247.43| +2450828|27555|2450842|81923|1884003|4130|15810|81923|1884003|4130|15810|1|9|1|4|16550|178|670|13|12.16|31.00|0.31|398.97|4.03|158.08|403.00|0.06|1.73|201.50|2.30|2.36|203.80|203.86|-155.78| +2450828|27555|2450877|81923|1884003|4130|15810|81923|1884003|4130|15810|1|80|7|5|13375|204|670|62|50.23|76.34|43.51|2035.46|2697.62|3114.26|4733.08|80.92|0.00|1514.04|2697.62|2778.54|4211.66|4292.58|-416.64| +2450828|27555|2450875|81923|1884003|4130|15810|81923|1884003|4130|15810|1|79|3|1|4502|250|670|36|99.78|183.59|78.94|3767.40|2841.84|3592.08|6609.24|28.41|0.00|2313.00|2841.84|2870.25|5154.84|5183.25|-750.24| +2450828|27555|2450836|81923|1884003|4130|15810|81923|1884003|4130|15810|1|3|14|2|1856|53|670|96|91.38|187.32|44.95|13667.52|4315.20|8772.48|17982.72|0.00|0.00|6652.80|4315.20|4315.20|10968.00|10968.00|-4457.28| +2450828|27555|2450887|81923|1884003|4130|15810|81923|1884003|4130|15810|1|61|18|3|8989|82|670|38|43.09|55.15|40.81|544.92|1550.78|1637.42|2095.70|139.57|0.00|712.50|1550.78|1690.35|2263.28|2402.85|-86.64| +2450828|27555|2450881|81923|1884003|4130|15810|81923|1884003|4130|15810|1|55|11|5|2906|88|670|38|67.77|93.52|57.04|1386.24|2167.52|2575.26|3553.76|30.77|628.58|532.76|1538.94|1569.71|2071.70|2102.47|-1036.32| +2450828|27555|2450914|81923|1884003|4130|15810|81923|1884003|4130|15810|1|71|16|3|3136|282|670|37|3.00|3.96|1.90|76.22|70.30|111.00|146.52|6.32|0.00|5.55|70.30|76.62|75.85|82.17|-40.70| +2450828|27555|2450886|81923|1884003|4130|15810|81923|1884003|4130|15810|1|27|6|5|15358|137|670|56|29.91|48.75|46.31|136.64|2593.36|1674.96|2730.00|233.40|0.00|1064.56|2593.36|2826.76|3657.92|3891.32|918.40| +2450828|27555|2450870|81923|1884003|4130|15810|81923|1884003|4130|15810|1|97|13|3|12688|128|670|7|6.93|8.80|8.36|3.08|58.52|48.51|61.60|4.68|0.00|3.64|58.52|63.20|62.16|66.84|10.01| +2450828|27555|2450878|81923|1884003|4130|15810|81923|1884003|4130|15810|1|22|3|5|2437|123|670|49|44.95|133.95|26.79|5250.84|1312.71|2202.55|6563.55|47.25|367.55|2231.46|945.16|992.41|3176.62|3223.87|-1257.39| +2450828|66014|2450873|98366|428777|3635|26093|98366|428777|3635|26093|4|13|4|1|14996|94|671|31|17.42|40.76|23.23|543.43|720.13|540.02|1263.56|43.20|0.00|62.93|720.13|763.33|783.06|826.26|180.11| +2450828|66014|2450851|98366|428777|3635|26093|98366|428777|3635|26093|4|30|3|2|9062|117|671|46|12.68|37.78|3.40|1581.48|156.40|583.28|1737.88|0.71|120.42|816.50|35.98|36.69|852.48|853.19|-547.30| +2450828|66014|2450843|98366|428777|3635|26093|98366|428777|3635|26093|4|84|15|4|1051|169|671|97|14.87|31.52|24.27|703.25|2354.19|1442.39|3057.44|0.00|0.00|1497.68|2354.19|2354.19|3851.87|3851.87|911.80| +2450828|66014|2450851|98366|428777|3635|26093|98366|428777|3635|26093|4|84|1|2|2749|59|671|47|59.15|141.96|2.83|6539.11|133.01|2780.05|6672.12|11.97|0.00|1000.63|133.01|144.98|1133.64|1145.61|-2647.04| +2450828|66014|2450873|98366|428777|3635|26093|98366|428777|3635|26093|4|35|4|2|4910|165|671|52|52.48|147.46|88.47|3067.48|4600.44|2728.96|7667.92|414.03|0.00|3527.16|4600.44|5014.47|8127.60|8541.63|1871.48| +2450828|66014|2450836|98366|428777|3635|26093|98366|428777|3635|26093|4|92|18|3|8942|175|671|6|44.62|85.67|3.42|493.50|20.52|267.72|514.02|0.20|0.00|51.36|20.52|20.72|71.88|72.08|-247.20| +2450828|66014|2450905|98366|428777|3635|26093|98366|428777|3635|26093|4|21|15|1|11854|146|671|70|87.65|211.23|78.15|9315.60|5470.50|6135.50|14786.10|437.64|0.00|6801.20|5470.50|5908.14|12271.70|12709.34|-665.00| +2450828|66014|2450840|98366|428777|3635|26093|98366|428777|3635|26093|4|85|11|3|14078|222|671|53|96.90|166.66|19.99|7773.51|1059.47|5135.70|8832.98|4.44|836.98|2031.49|222.49|226.93|2253.98|2258.42|-4913.21| +2450828|66014|2450871|98366|428777|3635|26093|98366|428777|3635|26093|4|21|12|1|4585|69|671|98|85.74|153.47|153.47|0.00|15040.06|8402.52|15040.06|1353.60|0.00|2856.70|15040.06|16393.66|17896.76|19250.36|6637.54| +2450828|66014|2450865|98366|428777|3635|26093|98366|428777|3635|26093|4|29|10|1|3146|240|671|30|63.78|148.60|126.31|668.70|3789.30|1913.40|4458.00|0.00|0.00|668.70|3789.30|3789.30|4458.00|4458.00|1875.90| +2450828|66014|2450901|98366|428777|3635|26093|98366|428777|3635|26093|4|100|15|3|7562|130|671|47|37.14|44.19|9.72|1620.09|456.84|1745.58|2076.93|4.38|310.65|1038.23|146.19|150.57|1184.42|1188.80|-1599.39| +2450828|66014|2450892|98366|428777|3635|26093|98366|428777|3635|26093|4|34|8|3|7792|35|671|51|73.00|170.82|51.24|6098.58|2613.24|3723.00|8711.82|127.00|496.51|4181.49|2116.73|2243.73|6298.22|6425.22|-1606.27| +2450828|66014|2450910|98366|428777|3635|26093|98366|428777|3635|26093|4|62|5|5|1330|170|671|34|21.53|27.77|26.65|38.08|906.10|732.02|944.18|16.12|99.67|9.18|806.43|822.55|815.61|831.73|74.41| +2450828|49138|2450912|28403|896708|5813|28918|28403|896708|5813|28918|1|47|19|2|2344|88|672|4|70.39|145.70|107.81|151.56|431.24|281.56|582.80|34.49|0.00|104.88|431.24|465.73|536.12|570.61|149.68| +2450828|49138|2450876|28403|896708|5813|28918|28403|896708|5813|28918|1|88|3|2|5212|7|672|11|89.04|196.77|165.28|346.39|1818.08|979.44|2164.47|109.08|0.00|714.23|1818.08|1927.16|2532.31|2641.39|838.64| +2450828|49138|2450876|28403|896708|5813|28918|28403|896708|5813|28918|1|61|19|2|5198|266|672|64|79.69|81.28|1.62|5098.24|103.68|5100.16|5201.92|1.97|64.28|675.84|39.40|41.37|715.24|717.21|-5060.76| +2450828|49138|2450885|28403|896708|5813|28918|28403|896708|5813|28918|1|78|15|1|10694|180|672|61|76.73|158.06|11.06|8967.00|674.66|4680.53|9641.66|47.22|0.00|771.04|674.66|721.88|1445.70|1492.92|-4005.87| +2450828|49138|2450880|28403|896708|5813|28918|28403|896708|5813|28918|1|14|5|3|4369|191|672|81|18.63|35.02|7.70|2212.92|623.70|1509.03|2836.62|0.00|0.00|1048.95|623.70|623.70|1672.65|1672.65|-885.33| +2450828|49138|2450886|28403|896708|5813|28918|28403|896708|5813|28918|1|99|1|2|16996|89|672|77|7.31|19.66|8.65|847.77|666.05|562.87|1513.82|39.96|0.00|484.33|666.05|706.01|1150.38|1190.34|103.18| +2450828|49138|2450912|28403|896708|5813|28918|28403|896708|5813|28918|1|17|10|4|260|220|672|51|7.29|20.04|5.41|746.13|275.91|371.79|1022.04|13.46|107.60|418.71|168.31|181.77|587.02|600.48|-203.48| +2450828|49138|2450835|28403|896708|5813|28918|28403|896708|5813|28918|1|16|7|3|1141|112|672|32|2.41|5.83|1.34|143.68|42.88|77.12|186.56|3.00|0.00|0.00|42.88|45.88|42.88|45.88|-34.24| +2450828|49138|2450912|28403|896708|5813|28918|28403|896708|5813|28918|1|49|4|3|1120|269|672|56|25.48|34.14|11.26|1281.28|630.56|1426.88|1911.84|0.00|0.00|0.00|630.56|630.56|630.56|630.56|-796.32| +2450828|49138|2450844|28403|896708|5813|28918|28403|896708|5813|28918|1|33|15|2|6232|17|672|84|55.27|132.09|68.68|5326.44|5769.12|4642.68|11095.56|519.22|0.00|3549.84|5769.12|6288.34|9318.96|9838.18|1126.44| +2450828|49138|2450881|28403|896708|5813|28918|28403|896708|5813|28918|1|85|15|1|1216|111|672|11|76.16|99.00|20.79|860.31|228.69|837.76|1089.00|20.58|0.00|457.38|228.69|249.27|686.07|706.65|-609.07| +2450828|49138|2450834|28403|896708|5813|28918|28403|896708|5813|28918|1|81|14|5|11023|109|672|19|75.50|93.62|24.34|1316.32|462.46|1434.50|1778.78|23.12|0.00|675.83|462.46|485.58|1138.29|1161.41|-972.04| +2450828|62057|2450894|90155|1524928|6454|34027|90155|1524928|6454|34027|4|87|5|1|2044|115|673|41|52.61|119.42|107.47|489.95|4406.27|2157.01|4896.22|132.18|0.00|929.88|4406.27|4538.45|5336.15|5468.33|2249.26| +2450828|62057|2450918|90155|1524928|6454|34027|90155|1524928|6454|34027|4|89|5|5|2398|182|673|90|43.87|107.92|43.16|5828.40|3884.40|3948.30|9712.80|38.84|0.00|1747.80|3884.40|3923.24|5632.20|5671.04|-63.90| +2450828|62057|2450875|90155|1524928|6454|34027|90155|1524928|6454|34027|4|64|1|1|12016|279|673|47|70.37|210.40|187.25|1088.05|8800.75|3307.39|9888.80|352.03|0.00|3065.34|8800.75|9152.78|11866.09|12218.12|5493.36| +2450828|62057|2450864|90155|1524928|6454|34027|90155|1524928|6454|34027|4|39|14|1|1906|170|673|82|19.12|30.20|6.64|1931.92|544.48|1567.84|2476.40|21.77|0.00|866.74|544.48|566.25|1411.22|1432.99|-1023.36| +2450828|62057|2450908|90155|1524928|6454|34027|90155|1524928|6454|34027|4|107|10|2|13951|88|673|88|87.73|140.36|9.82|11487.52|864.16|7720.24|12351.68|69.13|0.00|6175.84|864.16|933.29|7040.00|7109.13|-6856.08| +2450828|62057|2450831|90155|1524928|6454|34027|90155|1524928|6454|34027|4|56|5|2|4088|284|673|51|31.00|60.14|21.65|1962.99|1104.15|1581.00|3067.14|33.12|0.00|0.00|1104.15|1137.27|1104.15|1137.27|-476.85| +2450828|62057|2450877|90155|1524928|6454|34027|90155|1524928|6454|34027|4|98|10|5|17374|290|673|21|78.60|179.20|132.60|978.60|2784.60|1650.60|3763.20|167.07|0.00|940.80|2784.60|2951.67|3725.40|3892.47|1134.00| +2450828|62057|2450908|90155|1524928|6454|34027|90155|1524928|6454|34027|4|46|15|1|5644|153|673|61|43.18|118.31|5.91|6856.40|360.51|2633.98|7216.91|18.02|0.00|432.49|360.51|378.53|793.00|811.02|-2273.47| +2450828|62057|2450894|90155|1524928|6454|34027|90155|1524928|6454|34027|4|45|10|4|15703|30|673|83|35.54|93.47|85.05|698.86|7059.15|2949.82|7758.01|0.00|0.00|2327.32|7059.15|7059.15|9386.47|9386.47|4109.33| +2450828|62057|2450911|90155|1524928|6454|34027|90155|1524928|6454|34027|4|49|9|1|188|213|673|5|36.80|76.91|51.52|126.95|257.60|184.00|384.55|5.15|0.00|173.00|257.60|262.75|430.60|435.75|73.60| +2450828|62057|2450885|90155|1524928|6454|34027|90155|1524928|6454|34027|4|3|5|3|14263|119|673|84|34.62|36.69|20.17|1387.68|1694.28|2908.08|3081.96|67.77|0.00|1263.36|1694.28|1762.05|2957.64|3025.41|-1213.80| +2450828|62057|2450864|90155|1524928|6454|34027|90155|1524928|6454|34027|4|44|17|1|12434|267|673|2|80.70|159.78|78.29|162.98|156.58|161.40|319.56|0.00|0.00|143.80|156.58|156.58|300.38|300.38|-4.82| +2450828|62057|2450855|90155|1524928|6454|34027|90155|1524928|6454|34027|4|31|20|5|5167|195|673|85|11.33|32.85|31.86|84.15|2708.10|963.05|2792.25|73.11|270.81|474.30|2437.29|2510.40|2911.59|2984.70|1474.24| +2450828|62057|2450897|90155|1524928|6454|34027|90155|1524928|6454|34027|4|60|7|5|15640|139|673|23|9.36|20.12|2.41|407.33|55.43|215.28|462.76|3.88|0.00|217.35|55.43|59.31|272.78|276.66|-159.85| +2450828|75999|2450860|70994|266911|2828|33192|70994|266911|2828|33192|4|15|3|1|16874|195|674|93|15.27|41.22|30.91|958.83|2874.63|1420.11|3833.46|28.74|0.00|191.58|2874.63|2903.37|3066.21|3094.95|1454.52| +2450828|75999|2450839|70994|266911|2828|33192|70994|266911|2828|33192|4|28|19|1|13258|245|674|85|51.72|64.13|42.32|1853.85|3597.20|4396.20|5451.05|71.94|0.00|2397.85|3597.20|3669.14|5995.05|6066.99|-799.00| +2450828|75999|2450845|70994|266911|2828|33192|70994|266911|2828|33192|4|62|4|4|10756|174|674|94|52.69|146.47|24.89|11428.52|2339.66|4952.86|13768.18|45.62|1427.19|2615.08|912.47|958.09|3527.55|3573.17|-4040.39| +2450828|75999|2450841|70994|266911|2828|33192|70994|266911|2828|33192|4|82|12|2|5324|57|674|69|95.45|230.98|23.09|14344.41|1593.21|6586.05|15937.62|70.26|589.48|955.65|1003.73|1073.99|1959.38|2029.64|-5582.32| +2450828|75999|2450905|70994|266911|2828|33192|70994|266911|2828|33192|4|97|11|4|1894|140|674|81|46.82|90.83|1.81|7210.62|146.61|3792.42|7357.23|2.93|0.00|146.61|146.61|149.54|293.22|296.15|-3645.81| +2450828|73985|2450845|69122|1466163|4004|35045|69122|1466163|4004|35045|2|21|4|5|12188|249|675|34|43.77|98.92|66.27|1110.10|2253.18|1488.18|3363.28|102.74|968.86|672.52|1284.32|1387.06|1956.84|2059.58|-203.86| +2450828|73985|2450857|69122|1466163|4004|35045|69122|1466163|4004|35045|2|79|14|2|13958|168|675|54|67.74|191.70|115.02|4140.72|6211.08|3657.96|10351.80|310.55|0.00|3933.36|6211.08|6521.63|10144.44|10454.99|2553.12| +2450828|73985|2450891|69122|1466163|4004|35045|69122|1466163|4004|35045|2|46|8|5|14563|281|675|80|42.18|115.57|52.00|5085.60|4160.00|3374.40|9245.60|249.60|0.00|3605.60|4160.00|4409.60|7765.60|8015.20|785.60| +2450828|73985|2450856|69122|1466163|4004|35045|69122|1466163|4004|35045|2|107|5|5|7946|5|675|100|57.59|102.51|87.13|1538.00|8713.00|5759.00|10251.00|348.52|0.00|4407.00|8713.00|9061.52|13120.00|13468.52|2954.00| +2450828|73985|2450877|69122|1466163|4004|35045|69122|1466163|4004|35045|2|90|15|3|7147|152|675|2|36.59|47.56|10.46|74.20|20.92|73.18|95.12|0.41|0.00|25.68|20.92|21.33|46.60|47.01|-52.26| +2450828|73985|2450901|69122|1466163|4004|35045|69122|1466163|4004|35045|2|46|13|3|181|43|675|97|83.47|217.02|180.12|3579.30|17471.64|8096.59|21050.94|953.95|3843.76|2525.88|13627.88|14581.83|16153.76|17107.71|5531.29| +2450828|73985|2450891|69122|1466163|4004|35045|69122|1466163|4004|35045|2|81|14|5|5456|92|675|62|50.79|117.83|56.55|3799.36|3506.10|3148.98|7305.46|315.54|0.00|2410.56|3506.10|3821.64|5916.66|6232.20|357.12| +2450828|4690|2450854|96781|138582|3936|15737|96781|138582|3936|15737|4|97|3|4|4708|291|676|64|92.14|272.73|155.45|7505.92|9948.80|5896.96|17454.72|236.78|6566.20|2268.80|3382.60|3619.38|5651.40|5888.18|-2514.36| +2450828|4690|2450899|96781|138582|3936|15737|96781|138582|3936|15737|4|19|12|5|62|296|676|93|27.20|66.64|7.99|5454.45|743.07|2529.60|6197.52|52.01|0.00|3036.45|743.07|795.08|3779.52|3831.53|-1786.53| +2450828|4690|2450837|96781|138582|3936|15737|96781|138582|3936|15737|4|33|19|3|2035|40|676|71|9.23|27.59|26.76|58.93|1899.96|655.33|1958.89|14.44|1538.96|176.08|361.00|375.44|537.08|551.52|-294.33| +2450828|4690|2450853|96781|138582|3936|15737|96781|138582|3936|15737|4|17|10|2|421|32|676|60|17.17|29.70|9.80|1194.00|588.00|1030.20|1782.00|35.28|0.00|658.80|588.00|623.28|1246.80|1282.08|-442.20| +2450828|4690|2450877|96781|138582|3936|15737|96781|138582|3936|15737|4|56|7|4|109|158|676|77|66.37|120.12|26.42|7214.90|2034.34|5110.49|9249.24|20.34|0.00|2497.11|2034.34|2054.68|4531.45|4551.79|-3076.15| +2450828|4690|2450885|96781|138582|3936|15737|96781|138582|3936|15737|4|35|20|1|17396|135|676|53|65.19|74.96|37.48|1986.44|1986.44|3455.07|3972.88|119.18|0.00|1509.44|1986.44|2105.62|3495.88|3615.06|-1468.63| +2450828|4690|2450843|96781|138582|3936|15737|96781|138582|3936|15737|4|84|9|3|11564|103|676|68|47.82|69.33|51.30|1226.04|3488.40|3251.76|4714.44|69.76|0.00|518.16|3488.40|3558.16|4006.56|4076.32|236.64| +2450828|4690|2450862|96781|138582|3936|15737|96781|138582|3936|15737|4|105|8|5|16507|114|676|52|12.38|19.56|9.38|529.36|487.76|643.76|1017.12|7.56|336.55|477.88|151.21|158.77|629.09|636.65|-492.55| +2450828|4690|2450857|96781|138582|3936|15737|96781|138582|3936|15737|4|96|9|5|8000|195|676|10|62.29|148.87|31.26|1176.10|312.60|622.90|1488.70|21.88|0.00|238.10|312.60|334.48|550.70|572.58|-310.30| +2450828|4690|2450839|96781|138582|3936|15737|96781|138582|3936|15737|4|38|17|5|1312|168|676|70|67.19|120.27|25.25|6651.40|1767.50|4703.30|8418.90|36.58|1360.97|4125.10|406.53|443.11|4531.63|4568.21|-4296.77| +2450828|4690|2450851|96781|138582|3936|15737|96781|138582|3936|15737|4|80|3|4|3500|111|676|42|39.66|46.00|23.00|966.00|966.00|1665.72|1932.00|67.62|0.00|212.52|966.00|1033.62|1178.52|1246.14|-699.72| +2450828|4690|2450863|96781|138582|3936|15737|96781|138582|3936|15737|4|17|4|3|11566|44|676|68|28.35|46.21|8.31|2577.20|565.08|1927.80|3142.28|28.25|0.00|1225.36|565.08|593.33|1790.44|1818.69|-1362.72| +2450828|4690|2450839|96781|138582|3936|15737|96781|138582|3936|15737|4|70|20|3|5395|140|676|43|15.64|46.76|25.25|924.93|1085.75|672.52|2010.68|86.86|0.00|623.07|1085.75|1172.61|1708.82|1795.68|413.23| +2450828|40695|2450870|40969|613740|2158|10976|40969|613740|2158|10976|4|5|19|2|16004|67|677|83|73.25|139.17|44.53|7855.12|3695.99|6079.75|11551.11|0.00|0.00|693.05|3695.99|3695.99|4389.04|4389.04|-2383.76| +2450828|40695|2450889|40969|613740|2158|10976|40969|613740|2158|10976|4|17|7|3|11221|147|677|46|33.14|91.79|3.67|4053.52|168.82|1524.44|4222.34|6.07|101.29|337.64|67.53|73.60|405.17|411.24|-1456.91| +2450828|40695|2450856|40969|613740|2158|10976|40969|613740|2158|10976|4|53|4|4|15976|174|677|74|9.59|15.63|3.28|913.90|242.72|709.66|1156.62|2.42|0.00|219.04|242.72|245.14|461.76|464.18|-466.94| +2450828|40695|2450832|40969|613740|2158|10976|40969|613740|2158|10976|4|28|8|4|532|91|677|44|6.59|9.42|6.59|124.52|289.96|289.96|414.48|17.39|0.00|194.48|289.96|307.35|484.44|501.83|0.00| +2450828|40695|2450906|40969|613740|2158|10976|40969|613740|2158|10976|4|6|10|4|17950|105|677|21|12.01|19.09|16.22|60.27|340.62|252.21|400.89|23.84|0.00|31.92|340.62|364.46|372.54|396.38|88.41| +2450828|50987|2450907|64879|862971|7016|21108|64879|862971|7016|21108|4|42|17|3|4768|87|678|83|24.12|62.22|51.64|878.14|4286.12|2001.96|5164.26|131.15|2828.83|1497.32|1457.29|1588.44|2954.61|3085.76|-544.67| +2450828|50987|2450835|64879|862971|7016|21108|64879|862971|7016|21108|4|108|10|1|8438|188|678|17|48.56|107.80|63.60|751.40|1081.20|825.52|1832.60|32.43|0.00|128.18|1081.20|1113.63|1209.38|1241.81|255.68| +2450828|50987|2450862|64879|862971|7016|21108|64879|862971|7016|21108|4|26|8|1|4906|230|678|96|72.11|137.73|64.73|7008.00|6214.08|6922.56|13222.08|434.98|0.00|3305.28|6214.08|6649.06|9519.36|9954.34|-708.48| +2450828|50987|2450848|64879|862971|7016|21108|64879|862971|7016|21108|4|15|15|1|6793|243|678|82|46.93|60.07|23.42|3005.30|1920.44|3848.26|4925.74|8.83|1478.73|1871.24|441.71|450.54|2312.95|2321.78|-3406.55| +2450828|21727|2450910|83021|1760844|6667|2537|83021|1760844|6667|2537|2|37|4|2|12925|213|679|77|80.64|86.28|9.49|5912.83|730.73|6209.28|6643.56|21.92|0.00|3321.78|730.73|752.65|4052.51|4074.43|-5478.55| +2450828|21727|2450836|83021|1760844|6667|2537|83021|1760844|6667|2537|2|62|5|1|17095|158|679|7|83.28|109.09|0.00|763.63|0.00|582.96|763.63|0.00|0.00|91.63|0.00|0.00|91.63|91.63|-582.96| +2450828|21727|2450854|83021|1760844|6667|2537|83021|1760844|6667|2537|2|13|7|3|4753|225|679|81|19.77|32.22|3.86|2297.16|312.66|1601.37|2609.82|25.01|0.00|626.13|312.66|337.67|938.79|963.80|-1288.71| +2450828|21727|2450849|83021|1760844|6667|2537|83021|1760844|6667|2537|2|45|5|3|7514|267|679|23|60.75|78.97|66.33|290.72|1525.59|1397.25|1816.31|15.25|0.00|90.62|1525.59|1540.84|1616.21|1631.46|128.34| +2450828|21727|2450856|83021|1760844|6667|2537|83021|1760844|6667|2537|2|102|7|2|13108|20|679|23|91.21|206.13|142.22|1469.93|3271.06|2097.83|4740.99|228.97|0.00|426.65|3271.06|3500.03|3697.71|3926.68|1173.23| +2450828|21727|2450857|83021|1760844|6667|2537|83021|1760844|6667|2537|2|52|13|5|5959|250|679|37|20.10|58.08|41.23|623.45|1525.51|743.70|2148.96|122.04|0.00|902.43|1525.51|1647.55|2427.94|2549.98|781.81| +2450828|21727|2450857|83021|1760844|6667|2537|83021|1760844|6667|2537|2|99|12|3|5638|157|679|77|92.26|205.73|4.11|15524.74|316.47|7104.02|15841.21|3.16|0.00|6970.04|316.47|319.63|7286.51|7289.67|-6787.55| +2450828|42147|2450859|78608|94698|6435|36525|78608|94698|6435|36525|2|82|14|4|3818|292|680|85|82.83|136.66|125.72|929.90|10686.20|7040.55|11616.10|641.17|0.00|3832.65|10686.20|11327.37|14518.85|15160.02|3645.65| +2450828|42147|2450891|78608|94698|6435|36525|78608|94698|6435|36525|2|49|18|2|865|253|680|24|2.03|3.34|2.87|11.28|68.88|48.72|80.16|0.27|65.43|13.44|3.45|3.72|16.89|17.16|-45.27| +2450828|42147|2450849|78608|94698|6435|36525|78608|94698|6435|36525|2|38|18|1|3043|165|680|100|69.72|115.73|11.57|10416.00|1157.00|6972.00|11573.00|80.99|0.00|4166.00|1157.00|1237.99|5323.00|5403.99|-5815.00| +2450828|42147|2450888|78608|94698|6435|36525|78608|94698|6435|36525|2|14|18|3|4094|187|680|84|98.96|248.38|89.41|13353.48|7510.44|8312.64|20863.92|0.00|2703.75|5633.04|4806.69|4806.69|10439.73|10439.73|-3505.95| +2450828|42147|2450884|78608|94698|6435|36525|78608|94698|6435|36525|2|31|7|1|14350|52|680|39|13.78|26.18|1.83|949.65|71.37|537.42|1021.02|6.42|0.00|306.15|71.37|77.79|377.52|383.94|-466.05| +2450828|42147|2450847|78608|94698|6435|36525|78608|94698|6435|36525|2|104|1|4|6544|10|680|31|44.21|83.99|47.87|1119.72|1483.97|1370.51|2603.69|29.67|0.00|728.81|1483.97|1513.64|2212.78|2242.45|113.46| +2450828|42147|2450837|78608|94698|6435|36525|78608|94698|6435|36525|2|33|1|4|13412|85|680|97|77.21|90.33|71.36|1840.09|6921.92|7489.37|8762.01|484.53|0.00|0.00|6921.92|7406.45|6921.92|7406.45|-567.45| +2450828|42147|2450860|78608|94698|6435|36525|78608|94698|6435|36525|2|16|3|2|6796|165|680|47|64.68|112.54|29.26|3914.16|1375.22|3039.96|5289.38|13.61|921.39|1004.86|453.83|467.44|1458.69|1472.30|-2586.13| +2450828|42147|2450836|78608|94698|6435|36525|78608|94698|6435|36525|2|104|16|5|9571|3|680|10|39.55|51.41|24.67|267.40|246.70|395.50|514.10|0.00|189.95|113.10|56.75|56.75|169.85|169.85|-338.75| +2450828|42147|2450903|78608|94698|6435|36525|78608|94698|6435|36525|2|1|7|2|736|212|680|47|29.29|61.50|21.52|1879.06|1011.44|1376.63|2890.50|60.68|0.00|86.48|1011.44|1072.12|1097.92|1158.60|-365.19| +2450828|42147|2450884|78608|94698|6435|36525|78608|94698|6435|36525|2|86|5|5|3151|125|680|76|68.78|206.34|57.77|11291.32|4390.52|5227.28|15681.84|7.02|3688.03|1881.76|702.49|709.51|2584.25|2591.27|-4524.79| +2450828|42147|2450883|78608|94698|6435|36525|78608|94698|6435|36525|2|21|7|5|15830|234|680|96|79.48|163.72|27.83|13045.44|2671.68|7630.08|15717.12|133.58|0.00|3300.48|2671.68|2805.26|5972.16|6105.74|-4958.40| +2450828|42147|2450837|78608|94698|6435|36525|78608|94698|6435|36525|2|9|20|5|16228|147|680|97|45.72|61.72|8.02|5208.90|777.94|4434.84|5986.84|54.45|0.00|0.00|777.94|832.39|777.94|832.39|-3656.90| +2450828|47553|2450875|1708|522817|6326|19035|1708|522817|6326|19035|4|24|19|5|3596|80|681|47|79.69|130.69|7.84|5773.95|368.48|3745.43|6142.43|11.05|0.00|1105.44|368.48|379.53|1473.92|1484.97|-3376.95| +2450828|47553|2450852|1708|522817|6326|19035|1708|522817|6326|19035|4|25|10|3|3529|229|681|86|86.82|103.31|9.29|8085.72|798.94|7466.52|8884.66|55.92|0.00|2220.52|798.94|854.86|3019.46|3075.38|-6667.58| +2450828|47553|2450881|1708|522817|6326|19035|1708|522817|6326|19035|4|14|17|5|7012|137|681|49|37.18|107.07|8.56|4826.99|419.44|1821.82|5246.43|33.55|0.00|996.66|419.44|452.99|1416.10|1449.65|-1402.38| +2450828|47553|2450905|1708|522817|6326|19035|1708|522817|6326|19035|4|34|18|2|2620|230|681|44|46.16|61.39|34.99|1161.60|1539.56|2031.04|2701.16|46.18|0.00|351.12|1539.56|1585.74|1890.68|1936.86|-491.48| +2450828|47553|2450836|1708|522817|6326|19035|1708|522817|6326|19035|4|58|17|3|15014|170|681|59|22.75|63.01|58.59|260.78|3456.81|1342.25|3717.59|0.00|2108.65|1635.48|1348.16|1348.16|2983.64|2983.64|5.91| +2450828|47553|2450895|1708|522817|6326|19035|1708|522817|6326|19035|4|2|5|4|10204|186|681|90|43.46|83.00|45.65|3361.50|4108.50|3911.40|7470.00|287.59|0.00|2838.60|4108.50|4396.09|6947.10|7234.69|197.10| +2450828|47553|2450860|1708|522817|6326|19035|1708|522817|6326|19035|4|67|9|5|17116|170|681|77|60.12|64.92|15.58|3799.18|1199.66|4629.24|4998.84|7.67|431.87|499.73|767.79|775.46|1267.52|1275.19|-3861.45| +2450828|47553|2450871|1708|522817|6326|19035|1708|522817|6326|19035|4|75|2|3|322|75|681|77|67.61|120.34|55.35|5004.23|4261.95|5205.97|9266.18|298.33|0.00|1574.65|4261.95|4560.28|5836.60|6134.93|-944.02| +2450828|47553|2450884|1708|522817|6326|19035|1708|522817|6326|19035|4|57|19|4|11312|70|681|70|69.40|124.92|14.99|7695.10|1049.30|4858.00|8744.40|52.46|0.00|3409.70|1049.30|1101.76|4459.00|4511.46|-3808.70| +2450828|47553|2450869|1708|522817|6326|19035|1708|522817|6326|19035|4|16|11|1|13766|103|681|49|38.24|93.68|11.24|4039.56|550.76|1873.76|4590.32|11.01|0.00|550.76|550.76|561.77|1101.52|1112.53|-1323.00| +2450828|71031|2450908|70845|1301407|416|46578|70845|1301407|416|46578|2|58|13|5|4216|79|682|50|38.40|92.16|6.45|4285.50|322.50|1920.00|4608.00|3.22|0.00|138.00|322.50|325.72|460.50|463.72|-1597.50| +2450828|71031|2450847|70845|1301407|416|46578|70845|1301407|416|46578|2|30|6|3|7220|242|682|51|42.83|105.36|26.34|4030.02|1343.34|2184.33|5373.36|67.16|0.00|644.64|1343.34|1410.50|1987.98|2055.14|-840.99| +2450828|71031|2450851|70845|1301407|416|46578|70845|1301407|416|46578|2|39|8|4|956|300|682|42|74.35|120.44|57.81|2630.46|2428.02|3122.70|5058.48|194.24|0.00|960.96|2428.02|2622.26|3388.98|3583.22|-694.68| +2450828|71031|2450868|70845|1301407|416|46578|70845|1301407|416|46578|2|10|15|4|14341|215|682|32|24.89|60.23|17.46|1368.64|558.72|796.48|1927.36|0.00|0.00|153.92|558.72|558.72|712.64|712.64|-237.76| +2450828|71031|2450837|70845|1301407|416|46578|70845|1301407|416|46578|2|77|4|5|1579|232|682|60|66.52|86.47|78.68|467.40|4720.80|3991.20|5188.20|236.04|0.00|2490.00|4720.80|4956.84|7210.80|7446.84|729.60| +2450828|71031|2450873|70845|1301407|416|46578|70845|1301407|416|46578|2|83|16|5|446|121|682|80|86.10|134.31|38.94|7629.60|3115.20|6888.00|10744.80|9.96|2865.98|3975.20|249.22|259.18|4224.42|4234.38|-6638.78| +2450828|71031|2450830|70845|1301407|416|46578|70845|1301407|416|46578|2|93|9|4|14839|131|682|34|83.93|130.93|73.32|1958.74|2492.88|2853.62|4451.62|124.64|0.00|355.98|2492.88|2617.52|2848.86|2973.50|-360.74| +2450828|71031|2450840|70845|1301407|416|46578|70845|1301407|416|46578|2|104|17|5|14858|169|682|40|74.40|138.38|40.13|3930.00|1605.20|2976.00|5535.20|64.20|0.00|2269.20|1605.20|1669.40|3874.40|3938.60|-1370.80| +2450828|23653|2450836|7114|446054|194|26781|7114|446054|194|26781|1|69|11|3|14125|200|683|2|49.24|97.00|12.61|168.78|25.22|98.48|194.00|1.76|0.00|89.24|25.22|26.98|114.46|116.22|-73.26| +2450828|23653|2450844|7114|446054|194|26781|7114|446054|194|26781|1|54|1|5|8260|26|683|80|83.66|155.60|21.78|10705.60|1742.40|6692.80|12448.00|17.42|0.00|2364.80|1742.40|1759.82|4107.20|4124.62|-4950.40| +2450828|23653|2450862|7114|446054|194|26781|7114|446054|194|26781|1|30|16|5|13993|53|683|10|40.21|94.49|60.47|340.20|604.70|402.10|944.90|6.04|0.00|434.60|604.70|610.74|1039.30|1045.34|202.60| +2450828|23653|2450907|7114|446054|194|26781|7114|446054|194|26781|1|24|12|4|14500|59|683|42|69.34|205.93|30.88|7352.10|1296.96|2912.28|8649.06|51.87|0.00|1989.12|1296.96|1348.83|3286.08|3337.95|-1615.32| +2450828|23653|2450878|7114|446054|194|26781|7114|446054|194|26781|1|91|5|2|12748|176|683|46|41.43|104.40|26.10|3601.80|1200.60|1905.78|4802.40|60.03|0.00|191.82|1200.60|1260.63|1392.42|1452.45|-705.18| +2450828|23653|2450879|7114|446054|194|26781|7114|446054|194|26781|1|16|3|4|5515|9|683|76|90.16|202.86|81.14|9250.72|6166.64|6852.16|15417.36|431.66|0.00|3545.40|6166.64|6598.30|9712.04|10143.70|-685.52| +2450828|23653|2450838|7114|446054|194|26781|7114|446054|194|26781|1|29|19|2|8842|58|683|21|89.21|217.67|189.37|594.30|3976.77|1873.41|4571.07|0.00|119.30|365.61|3857.47|3857.47|4223.08|4223.08|1984.06| +2450828|23653|2450853|7114|446054|194|26781|7114|446054|194|26781|1|20|13|4|10429|28|683|39|53.34|78.40|72.91|214.11|2843.49|2080.26|3057.60|227.47|0.00|1375.92|2843.49|3070.96|4219.41|4446.88|763.23| +2450828|23653|2450852|7114|446054|194|26781|7114|446054|194|26781|1|102|14|4|8104|4|683|11|31.29|64.14|48.10|176.44|529.10|344.19|705.54|21.16|0.00|225.72|529.10|550.26|754.82|775.98|184.91| +2450828|23653|2450843|7114|446054|194|26781|7114|446054|194|26781|1|85|3|2|16597|226|683|77|3.50|6.26|3.94|178.64|303.38|269.50|482.02|16.26|100.11|211.75|203.27|219.53|415.02|431.28|-66.23| +2450828|23653|2450858|7114|446054|194|26781|7114|446054|194|26781|1|83|3|3|9115|171|683|17|78.32|179.35|112.99|1128.12|1920.83|1331.44|3048.95|153.66|0.00|1524.39|1920.83|2074.49|3445.22|3598.88|589.39| +2450828|23653|2450905|7114|446054|194|26781|7114|446054|194|26781|1|73|8|3|8018|116|683|92|69.39|199.14|35.84|15023.60|3297.28|6383.88|18320.88|98.91|0.00|1098.48|3297.28|3396.19|4395.76|4494.67|-3086.60| +2450828|23653|2450860|7114|446054|194|26781|7114|446054|194|26781|1|69|11|4|12625|43|683|11|9.48|16.40|11.48|54.12|126.28|104.28|180.40|2.29|93.44|45.10|32.84|35.13|77.94|80.23|-71.44| +2450828|8477|2450868|10032|1553908|2584|34860|10032|1553908|2584|34860|4|29|5|3|8527|21|684|1|54.87|102.05|101.02|1.03|101.02|54.87|102.05|1.13|84.85|5.10|16.17|17.30|21.27|22.40|-38.70| +2450828|8477|2450885|10032|1553908|2584|34860|10032|1553908|2584|34860|4|88|8|3|2584|216|684|74|27.68|76.39|74.09|170.20|5482.66|2048.32|5652.86|414.48|877.22|677.84|4605.44|5019.92|5283.28|5697.76|2557.12| +2450828|8477|2450916|10032|1553908|2584|34860|10032|1553908|2584|34860|4|95|18|4|14132|59|684|7|46.28|103.20|50.56|368.48|353.92|323.96|722.40|16.84|113.25|339.50|240.67|257.51|580.17|597.01|-83.29| +2450828|8477|2450869|10032|1553908|2584|34860|10032|1553908|2584|34860|4|32|8|2|3532|265|684|4|58.60|59.77|37.65|88.48|150.60|234.40|239.08|9.03|0.00|43.00|150.60|159.63|193.60|202.63|-83.80| +2450828|8477|2450878|10032|1553908|2584|34860|10032|1553908|2584|34860|4|101|9|5|1507|49|684|11|56.04|84.06|30.26|591.80|332.86|616.44|924.66|9.98|0.00|83.16|332.86|342.84|416.02|426.00|-283.58| +2450828|8477|2450864|10032|1553908|2584|34860|10032|1553908|2584|34860|4|37|4|1|1345|267|684|94|96.69|145.03|43.50|9543.82|4089.00|9088.86|13632.82|245.34|0.00|408.90|4089.00|4334.34|4497.90|4743.24|-4999.86| +2450828|8477|2450844|10032|1553908|2584|34860|10032|1553908|2584|34860|4|69|9|2|13021|188|684|44|17.21|45.77|38.44|322.52|1691.36|757.24|2013.88|25.37|845.68|201.08|845.68|871.05|1046.76|1072.13|88.44| +2450828|8477|2450837|10032|1553908|2584|34860|10032|1553908|2584|34860|4|67|4|1|13684|220|684|67|83.99|89.02|88.12|60.30|5904.04|5627.33|5964.34|531.36|0.00|2802.61|5904.04|6435.40|8706.65|9238.01|276.71| +2450828|8477|2450835|10032|1553908|2584|34860|10032|1553908|2584|34860|4|107|6|4|5086|269|684|50|60.78|177.47|157.94|976.50|7897.00|3039.00|8873.50|710.73|0.00|1597.00|7897.00|8607.73|9494.00|10204.73|4858.00| +2450828|8477|2450913|10032|1553908|2584|34860|10032|1553908|2584|34860|4|87|2|2|16195|45|684|45|88.58|248.02|39.68|9375.30|1785.60|3986.10|11160.90|17.85|0.00|4240.80|1785.60|1803.45|6026.40|6044.25|-2200.50| +2450828|8477|2450867|10032|1553908|2584|34860|10032|1553908|2584|34860|4|104|1|4|860|294|684|91|67.35|120.55|27.72|8447.53|2522.52|6128.85|10970.05|151.35|0.00|986.44|2522.52|2673.87|3508.96|3660.31|-3606.33| +2450828|8477|2450862|10032|1553908|2584|34860|10032|1553908|2584|34860|4|99|7|1|6403|99|684|52|44.63|97.73|32.25|3404.96|1677.00|2320.76|5081.96|33.54|0.00|965.12|1677.00|1710.54|2642.12|2675.66|-643.76| +2450828|15845|2450882|18697|506427|5004|20734|18697|506427|5004|20734|2|65|15|1|4099|103|685|78|88.06|206.94|76.56|10169.64|5971.68|6868.68|16141.32|358.30|0.00|3873.48|5971.68|6329.98|9845.16|10203.46|-897.00| +2450828|15845|2450891|18697|506427|5004|20734|18697|506427|5004|20734|2|19|3|4|15832|223|685|11|2.92|3.44|1.61|20.13|17.71|32.12|37.84|0.17|0.00|3.74|17.71|17.88|21.45|21.62|-14.41| +2450828|15845|2450915|18697|506427|5004|20734|18697|506427|5004|20734|2|77|14|5|7894|250|685|35|51.86|151.43|57.54|3286.15|2013.90|1815.10|5300.05|0.00|0.00|1324.75|2013.90|2013.90|3338.65|3338.65|198.80| +2450828|15845|2450891|18697|506427|5004|20734|18697|506427|5004|20734|2|56|11|2|15686|127|685|33|77.83|157.99|31.59|4171.20|1042.47|2568.39|5213.67|10.42|0.00|1459.59|1042.47|1052.89|2502.06|2512.48|-1525.92| +2450828|15845|2450903|18697|506427|5004|20734|18697|506427|5004|20734|2|3|13|5|118|31|685|98|3.80|3.91|2.50|138.18|245.00|372.40|383.18|2.45|0.00|45.08|245.00|247.45|290.08|292.53|-127.40| +|15845|2450909|18697|506427|5004||18697|||20734|||18||247|4|685|81|||94.94|||7846.47||538.30|0.00||7690.14||11012.76|11551.06|-156.33| +2450828|15845|2450853|18697|506427|5004|20734|18697|506427|5004|20734|2|92|17|5|10303|272|685|57|48.66|83.69|40.17|2480.64|2289.69|2773.62|4770.33|114.48|0.00|667.47|2289.69|2404.17|2957.16|3071.64|-483.93| +2450828|15845|2450871|18697|506427|5004|20734|18697|506427|5004|20734|2|8|9|5|134|145|685|29|93.88|270.37|102.74|4861.27|2979.46|2722.52|7840.73|59.58|0.00|2430.49|2979.46|3039.04|5409.95|5469.53|256.94| +2450828|15845|2450845|18697|506427|5004|20734|18697|506427|5004|20734|2|26|3|4|6343|236|685|71|76.90|180.71|70.47|7827.04|5003.37|5459.90|12830.41|50.03|0.00|641.13|5003.37|5053.40|5644.50|5694.53|-456.53| +2450828|15845|2450834|18697|506427|5004|20734|18697|506427|5004|20734|2|78|16|4|6151|123|685|22|71.57|166.75|160.08|146.74|3521.76|1574.54|3668.50|0.00|0.00|1430.66|3521.76|3521.76|4952.42|4952.42|1947.22| +2450828|15845|2450914|18697|506427|5004|20734|18697|506427|5004|20734|2|24|16|1|3412|288|685|61|55.36|131.20|26.24|6402.56|1600.64|3376.96|8003.20|32.01|0.00|4001.60|1600.64|1632.65|5602.24|5634.25|-1776.32| +2450828|15845|2450903|18697|506427|5004|20734|18697|506427|5004|20734|2|25|1|4|6772|8|685|44|53.16|137.15|97.37|1750.32|4284.28|2339.04|6034.60|385.58|0.00|542.96|4284.28|4669.86|4827.24|5212.82|1945.24| +2450828|15845|2450863|18697|506427|5004|20734|18697|506427|5004|20734|2|103|14|1|856|58|685|47|27.53|77.63|43.47|1605.52|2043.09|1293.91|3648.61|163.44|0.00|1386.03|2043.09|2206.53|3429.12|3592.56|749.18| +2450828|15845|2450840|18697|506427|5004|20734|18697|506427|5004|20734|2|69|8|5|8456|2|685|32|14.76|21.54|17.23|137.92|551.36|472.32|689.28|11.02|0.00|310.08|551.36|562.38|861.44|872.46|79.04| +2450828|18226|2450862|60008|889636|4064|12045|60008|889636|4064|12045|4|46|7|4|7606|266|686|82|60.34|144.21|23.07|9933.48|1891.74|4947.88|11825.22|0.00|0.00|4965.92|1891.74|1891.74|6857.66|6857.66|-3056.14| +2450828|18226|2450911|60008|889636|4064|12045|60008|889636|4064|12045|4|53|16|5|26|155|686|100|20.23|60.69|36.41|2428.00|3641.00|2023.00|6069.00|145.64|0.00|0.00|3641.00|3786.64|3641.00|3786.64|1618.00| +2450828|18226|2450913|60008|889636|4064|12045|60008|889636|4064|12045|4|27|20|4|17983|117|686|26|77.64|120.34|108.30|313.04|2815.80|2018.64|3128.84|84.47|0.00|250.12|2815.80|2900.27|3065.92|3150.39|797.16| +2450828|18226|2450910|60008|889636|4064|12045|60008|889636|4064|12045|4|105|19|3|12494|63|686|42|50.09|76.13|15.22|2558.22|639.24|2103.78|3197.46|20.77|223.73|863.10|415.51|436.28|1278.61|1299.38|-1688.27| +2450828|18226|2450882|60008|889636|4064|12045|60008|889636|4064|12045|4|74|3|2|14992|83|686|97|6.15|14.32|5.15|889.49|499.55|596.55|1389.04|39.96|0.00|541.26|499.55|539.51|1040.81|1080.77|-97.00| +2450828|41009|2450889|82741|84771|401|8478|82741|84771|401|8478|4|33|4|3|6464|59|687|85|30.56|55.31|6.08|4184.55|516.80|2597.60|4701.35|4.23|304.91|0.00|211.89|216.12|211.89|216.12|-2385.71| +2450828|41009|2450833|82741|84771|401|8478|82741|84771|401|8478|4|69|9|1|8486|182|687|38|57.19|61.76|19.76|1596.00|750.88|2173.22|2346.88|0.00|0.00|1126.32|750.88|750.88|1877.20|1877.20|-1422.34| +2450828|41009|2450868|82741|84771|401|8478|82741|84771|401|8478|4|48|20|1|7846|201|687|49|89.49|246.99|242.05|242.06|11860.45|4385.01|12102.51|948.83|0.00|5808.95|11860.45|12809.28|17669.40|18618.23|7475.44| +2450828|41009|2450849|82741|84771|401|8478|82741|84771|401|8478|4|89|15|4|12338|70|687|2|94.68|98.46|26.58|143.76|53.16|189.36|196.92|3.72|0.00|82.70|53.16|56.88|135.86|139.58|-136.20| +2450828|41009|2450888|82741|84771|401|8478|82741|84771|401|8478|4|33|13|4|931|120|687|49|21.78|40.72|39.49|60.27|1935.01|1067.22|1995.28|174.15|0.00|398.86|1935.01|2109.16|2333.87|2508.02|867.79| +2450828|41009|2450884|82741|84771|401|8478|82741|84771|401|8478|4|21|9|1|9997|35|687|25|9.79|24.57|24.57|0.00|614.25|244.75|614.25|7.67|460.68|61.25|153.57|161.24|214.82|222.49|-91.18| +2450828|41009|2450848|82741|84771|401|8478|82741|84771|401|8478|4|103|12|4|3014|93|687|15|52.58|101.47|17.24|1263.45|258.60|788.70|1522.05|23.27|0.00|365.25|258.60|281.87|623.85|647.12|-530.10| +2450828|41009|2450855|82741|84771|401|8478|82741|84771|401|8478|4|78|11|3|6320|205|687|59|35.60|46.28|34.71|682.63|2047.89|2100.40|2730.52|61.43|0.00|81.42|2047.89|2109.32|2129.31|2190.74|-52.51| +2450828|41009|2450903|82741|84771|401|8478|82741|84771|401|8478|4|84|11|2|8896|270|687|80|57.11|73.67|66.30|589.60|5304.00|4568.80|5893.60|265.20|0.00|1296.00|5304.00|5569.20|6600.00|6865.20|735.20| +2450828|41009|2450877|82741|84771|401|8478|82741|84771|401|8478|4|25|14|2|11695|8|687|10|94.91|104.40|62.64|417.60|626.40|949.10|1044.00|56.37|0.00|407.10|626.40|682.77|1033.50|1089.87|-322.70| +2450828|41009|2450836|82741|84771|401|8478|82741|84771|401|8478|4|104|16|4|15500|25|687|43|68.26|120.13|103.31|723.26|4442.33|2935.18|5165.59|310.96|0.00|1032.86|4442.33|4753.29|5475.19|5786.15|1507.15| +2450828|41009|2450889|82741|84771|401|8478|82741|84771|401|8478|4|20|12|2|11977|272|687|91|79.75|232.87|176.98|5085.99|16105.18|7257.25|21191.17|322.10|0.00|2754.57|16105.18|16427.28|18859.75|19181.85|8847.93| +2450828|41009|2450914|82741|84771|401|8478|82741|84771|401|8478|4|55|2|1|2059|154|687|46|23.99|39.58|12.66|1238.32|582.36|1103.54|1820.68|34.94|0.00|728.18|582.36|617.30|1310.54|1345.48|-521.18| +2450828|36366|2450871|52296|892630|1764|18498|52296|892630|1764|18498|1|55|14|5|16183|80|688|68|53.63|107.79|15.09|6303.60|1026.12|3646.84|7329.72|30.78|0.00|2564.96|1026.12|1056.90|3591.08|3621.86|-2620.72| +2450828|36366|2450853|52296|892630|1764|18498|52296|892630|1764|18498|1|24|9|4|15811|235|688|51|13.12|22.96|1.83|1077.63|93.33|669.12|1170.96|0.00|0.00|562.02|93.33|93.33|655.35|655.35|-575.79| +2450828|36366|2450884|52296|892630|1764|18498|52296|892630|1764|18498|1|53|10|1|17018|43|688|39|3.12|8.26|6.93|51.87|270.27|121.68|322.14|16.21|0.00|38.61|270.27|286.48|308.88|325.09|148.59| +2450828|36366|2450864|52296|892630|1764|18498|52296|892630|1764|18498|1|82|5|5|13315|55|688|37|99.12|134.80|22.91|4139.93|847.67|3667.44|4987.60|50.86|0.00|1097.05|847.67|898.53|1944.72|1995.58|-2819.77| +2450828|36366|2450842|52296|892630|1764|18498|52296|892630|1764|18498|1|105|1|4|6055|140|688|10|41.71|55.05|2.75|523.00|27.50|417.10|550.50|0.55|0.00|22.00|27.50|28.05|49.50|50.05|-389.60| +2450828|36366|2450901|52296|892630|1764|18498|52296|892630|1764|18498|1|8|4|4|980|124|688|42|21.17|37.68|36.92|31.92|1550.64|889.14|1582.56|15.50|0.00|632.94|1550.64|1566.14|2183.58|2199.08|661.50| +2450828|36366|2450917|52296|892630|1764|18498|52296|892630|1764|18498|1|62|8|4|1399|149|688|83|59.78|161.40|156.55|402.55|12993.65|4961.74|13396.20|909.55|0.00|5492.11|12993.65|13903.20|18485.76|19395.31|8031.91| +2450828|36366|2450848|52296|892630|1764|18498|52296|892630|1764|18498|1|27|11|3|5611|182|688|51|90.90|180.89|117.57|3229.32|5996.07|4635.90|9225.39|0.00|0.00|461.04|5996.07|5996.07|6457.11|6457.11|1360.17| +2450828|36366|2450836|52296|892630|1764|18498|52296|892630|1764|18498|1|4|8|2|12415|262|688|22|82.68|174.45|94.20|1765.50|2072.40|1818.96|3837.90|0.00|0.00|345.40|2072.40|2072.40|2417.80|2417.80|253.44| +2450828|36366|2450918|52296|892630|1764|18498|52296|892630|1764|18498|1|84|11|1|2252|101|688|8|31.98|73.87|18.46|443.28|147.68|255.84|590.96|8.86|0.00|41.36|147.68|156.54|189.04|197.90|-108.16| +2450828|36366|2450876|52296|892630|1764|18498|52296|892630|1764|18498|1|91|17|5|8536|227|688|71|85.94|249.22|209.34|2831.48|14863.14|6101.74|17694.62|148.63|0.00|5131.17|14863.14|15011.77|19994.31|20142.94|8761.40| +2450828|36366|2450896|52296|892630|1764|18498|52296|892630|1764|18498|1|84|4|4|15146|251|688|3|85.55|213.87|198.89|44.94|596.67|256.65|641.61|5.96|0.00|192.48|596.67|602.63|789.15|795.11|340.02| +2450828|84182|2450874|18521|1846889|6104|44919|18521|1846889|6104|44919|4|27|11|4|316|98|689|48|73.55|151.51|72.72|3781.92|3490.56|3530.40|7272.48|69.81|0.00|2763.36|3490.56|3560.37|6253.92|6323.73|-39.84| +2450828|84182|2450847|18521|1846889|6104|44919|18521|1846889|6104|44919|4|71|16|4|15157|152|689|67|56.96|157.77|44.17|7611.20|2959.39|3816.32|10570.59|151.52|1065.38|1268.31|1894.01|2045.53|3162.32|3313.84|-1922.31| +2450828|84182|2450873|18521|1846889|6104|44919|18521|1846889|6104|44919|4|75|4|5|4183|132|689|63|35.64|37.42|7.85|1862.91|494.55|2245.32|2357.46|0.00|0.00|1131.48|494.55|494.55|1626.03|1626.03|-1750.77| +2450828|84182|2450894|18521|1846889|6104|44919|18521|1846889|6104|44919|4|56|12|3|12746|300|689|6|79.85|217.19|45.60|1029.54|273.60|479.10|1303.14|8.20|0.00|651.54|273.60|281.80|925.14|933.34|-205.50| +2450828|84182|2450846|18521|1846889|6104|44919|18521|1846889|6104|44919|4|33|13|3|6262|253|689|43|57.64|75.50|3.77|3084.39|162.11|2478.52|3246.50|0.00|0.00|454.51|162.11|162.11|616.62|616.62|-2316.41| +2450828|84182|2450874|18521|1846889|6104|44919|18521|1846889|6104|44919|4|96|10|5|7486|164|689|50|29.41|84.11|73.17|547.00|3658.50|1470.50|4205.50|109.75|0.00|0.00|3658.50|3768.25|3658.50|3768.25|2188.00| +2450828|84182|2450918|18521|1846889|6104|44919|18521|1846889|6104|44919|4|38|2|2|6140|63|689|46|61.71|67.88|23.75|2029.98|1092.50|2838.66|3122.48|0.00|0.00|842.72|1092.50|1092.50|1935.22|1935.22|-1746.16| +2450828|84182|2450862|18521|1846889|6104|44919|18521|1846889|6104|44919|4|22|14|4|895|132|689|30|12.62|19.93|1.79|544.20|53.70|378.60|597.90|4.29|0.00|286.80|53.70|57.99|340.50|344.79|-324.90| +2450828|84182|2450906|18521|1846889|6104|44919|18521|1846889|6104|44919|4|18|5|2|16465|19|689|45|26.74|49.20|43.29|265.95|1948.05|1203.30|2214.00|58.44|0.00|110.70|1948.05|2006.49|2058.75|2117.19|744.75| +2450828|84182|2450895|18521|1846889|6104|44919|18521|1846889|6104|44919|4|98|20|2|14644|263|689|22|75.28|179.16|60.91|2601.50|1340.02|1656.16|3941.52|40.20|0.00|196.90|1340.02|1380.22|1536.92|1577.12|-316.14| +2450828|69417|2450900|11205|1484509|6966|47470|11205|1484509|6966|47470|1|61|19|1|10927|258|690|4|84.69|227.81|186.80|164.04|747.20|338.76|911.24|67.24|0.00|182.24|747.20|814.44|929.44|996.68|408.44| +2450828|69417|2450844|11205|1484509|6966|47470|11205|1484509|6966|47470|1|78|18|2|248|38|690|1|61.80|112.47|111.34|1.13|111.34|61.80|112.47|6.68|0.00|32.61|111.34|118.02|143.95|150.63|49.54| +2450828|69417|2450866|11205|1484509|6966|47470|11205|1484509|6966|47470|1|107|1|1|16070|211|690|81|25.54|65.38|48.38|1377.00|3918.78|2068.74|5295.78|313.50|0.00|1588.41|3918.78|4232.28|5507.19|5820.69|1850.04| +2450828|69417|2450837|11205|1484509|6966|47470|11205|1484509|6966|47470|1|19|11|5|4888|225|690|61|53.42|135.15|10.81|7584.74|659.41|3258.62|8244.15|39.56|0.00|1895.88|659.41|698.97|2555.29|2594.85|-2599.21| +2450828|69417|2450884|11205|1484509|6966|47470|11205|1484509|6966|47470|1|36|5|5|9746|288|690|71|23.80|57.35|18.35|2769.00|1302.85|1689.80|4071.85|91.19|0.00|40.47|1302.85|1394.04|1343.32|1434.51|-386.95| +2450828|69417|2450883|11205|1484509|6966|47470|11205|1484509|6966|47470|1|83|15|5|15646|241|690|96|83.76|249.60|37.44|20367.36|3594.24|8040.96|23961.60|100.63|2156.54|239.04|1437.70|1538.33|1676.74|1777.37|-6603.26| +2450828|61163|2450834|3358|1898802|6246|16537|3358|1898802|6246|16537|2|2|16|3|9284|227|691|37|57.88|127.33|91.67|1319.42|3391.79|2141.56|4711.21|67.83|0.00|2072.74|3391.79|3459.62|5464.53|5532.36|1250.23| +2450828|61163|2450862|3358|1898802|6246|16537|3358|1898802|6246|16537|2|41|19|2|952|158|691|38|23.73|36.54|19.73|638.78|749.74|901.74|1388.52|67.47|0.00|194.18|749.74|817.21|943.92|1011.39|-152.00| +2450828|61163|2450866|3358|1898802|6246|16537|3358|1898802|6246|16537|2|91|11|5|8443|198|691|55|77.22|223.16|60.25|8960.05|3313.75|4247.10|12273.80|265.10|0.00|4540.80|3313.75|3578.85|7854.55|8119.65|-933.35| +2450828|61163|2450913|3358|1898802|6246|16537|3358|1898802|6246|16537|2|87|14|3|11060|259|691|30|75.42|117.65|90.59|811.80|2717.70|2262.60|3529.50|0.00|0.00|1164.60|2717.70|2717.70|3882.30|3882.30|455.10| +2450828|61163|2450884|3358|1898802|6246|16537|3358|1898802|6246|16537|2|46|4|1|445|200|691|23|73.05|88.39|3.53|1951.78|81.19|1680.15|2032.97|4.45|31.66|996.13|49.53|53.98|1045.66|1050.11|-1630.62| +2450828|61163|2450868|3358|1898802|6246|16537|3358|1898802|6246|16537|2|14|9|4|15997|29|691|54|71.40|151.36|24.21|6866.10|1307.34|3855.60|8173.44|78.44|0.00|2370.06|1307.34|1385.78|3677.40|3755.84|-2548.26| +2450828|61163|2450904|3358|1898802|6246|16537|3358|1898802|6246|16537|2|25|17|2|12376|70|691|50|64.15|129.58|82.93|2332.50|4146.50|3207.50|6479.00|207.32|0.00|1490.00|4146.50|4353.82|5636.50|5843.82|939.00| +2450828|61163|2450870|3358|1898802|6246|16537|3358|1898802|6246|16537|2|73|6|5|4442|65|691|13|8.49|13.15|1.70|148.85|22.10|110.37|170.95|0.43|7.73|64.87|14.37|14.80|79.24|79.67|-96.00| +2450828|61163|2450838|3358|1898802|6246|16537|3358|1898802|6246|16537|2|76|15|3|4531|77|691|91|93.10|153.61|150.53|280.28|13698.23|8472.10|13978.51|821.89|0.00|5031.39|13698.23|14520.12|18729.62|19551.51|5226.13| +2450828|62884|2450912|65163|1704330|2697|47064|65163|1704330|2697|47064|2|88|2|2|10258|252|692|51|10.45|12.01|3.60|428.91|183.60|532.95|612.51|7.34|0.00|140.76|183.60|190.94|324.36|331.70|-349.35| +2450828|62884|2450857|65163|1704330|2697|47064|65163|1704330|2697|47064|2|64|8|4|16940|101|692|40|58.10|98.18|16.69|3259.60|667.60|2324.00|3927.20|44.86|106.81|863.60|560.79|605.65|1424.39|1469.25|-1763.21| +2450828|62884|2450867|65163|1704330|2697|47064|65163|1704330|2697|47064|2|66|12|4|2888|46|692|99|76.35|207.67|78.91|12747.24|7812.09|7558.65|20559.33|312.48|0.00|2877.93|7812.09|8124.57|10690.02|11002.50|253.44| +2450828|62884|2450859|65163|1704330|2697|47064|65163|1704330|2697|47064|2|102|8|4|11569|278|692|82|82.23|139.79|100.64|3210.30|8252.48|6742.86|11462.78|44.56|6767.03|3094.68|1485.45|1530.01|4580.13|4624.69|-5257.41| +2450828|62884|2450851|65163|1704330|2697|47064|65163|1704330|2697|47064|2|43|3|2|14551|145|692|39|41.17|113.21|57.73|2163.72|2251.47|1605.63|4415.19|157.60|0.00|882.96|2251.47|2409.07|3134.43|3292.03|645.84| +2450828|62884|2450912|65163|1704330|2697|47064|65163|1704330|2697|47064|2|71|2|5|6982|30|692|83|67.85|79.38|30.16|4085.26|2503.28|5631.55|6588.54|25.03|0.00|724.59|2503.28|2528.31|3227.87|3252.90|-3128.27| +2450828|62884|2450891|65163|1704330|2697|47064|65163|1704330|2697|47064|2|54|18|2|5419|147|692|24|26.79|35.89|17.94|430.80|430.56|642.96|861.36|21.52|0.00|103.20|430.56|452.08|533.76|555.28|-212.40| +2450828|62884|2450859|65163|1704330|2697|47064|65163|1704330|2697|47064|2|19|6|3|2740|200|692|19|82.20|207.14|45.57|3069.83|865.83|1561.80|3935.66|60.60|0.00|1023.15|865.83|926.43|1888.98|1949.58|-695.97| +2450828|74141|2450855|96882|1110636|4837|16591|96882|1110636|4837|16591|1|1|15|3|8785|21|693|22|91.14|166.78|1.66|3632.64|36.52|2005.08|3669.16|0.73|0.00|1210.66|36.52|37.25|1247.18|1247.91|-1968.56| +2450828|74141|2450892|96882|1110636|4837|16591|96882|1110636|4837|16591|1|80|6|2|8854|166|693|80|70.36|75.98|3.03|5836.00|242.40|5628.80|6078.40|0.00|0.00|1519.20|242.40|242.40|1761.60|1761.60|-5386.40| +2450828|74141|2450878|96882|1110636|4837|16591|96882|1110636|4837|16591|1|28|12|3|2449|102|693|1|72.23|136.51|25.93|110.58|25.93|72.23|136.51|0.00|0.00|16.38|25.93|25.93|42.31|42.31|-46.30| +2450828|74141|2450897|96882|1110636|4837|16591|96882|1110636|4837|16591|1|32|19|1|15058|115|693|62|94.62|146.66|127.59|1182.34|7910.58|5866.44|9092.92|395.52|0.00|90.52|7910.58|8306.10|8001.10|8396.62|2044.14| +2450828|74141|2450869|96882|1110636|4837|16591|96882|1110636|4837|16591|1|87|1|5|16264|208|693|37|15.73|18.40|8.28|374.44|306.36|582.01|680.80|21.44|0.00|149.48|306.36|327.80|455.84|477.28|-275.65| +2450828|74141|2450869|96882|1110636|4837|16591|96882|1110636|4837|16591|1|39|20|1|16387|251|693|48|6.41|6.79|5.97|39.36|286.56|307.68|325.92|14.32|0.00|139.68|286.56|300.88|426.24|440.56|-21.12| +2450828|74141|2450840|96882|1110636|4837|16591|96882|1110636|4837|16591|1|71|16|4|15898|268|693|38|94.03|201.22|96.58|3976.32|3670.04|3573.14|7646.36|0.00|0.00|1758.64|3670.04|3670.04|5428.68|5428.68|96.90| +2450828|74141|2450890|96882|1110636|4837|16591|96882|1110636|4837|16591|1|48|1|2|9652|140|693|63|7.57|19.30|2.31|1070.37|145.53|476.91|1215.90|0.00|138.25|583.38|7.28|7.28|590.66|590.66|-469.63| +2450828|74141|2450849|96882|1110636|4837|16591|96882|1110636|4837|16591|1|59|16|3|12289|72|693|18|63.03|187.82|52.58|2434.32|946.44|1134.54|3380.76|66.25|0.00|338.04|946.44|1012.69|1284.48|1350.73|-188.10| +2450828|74141|2450853|96882|1110636|4837|16591|96882|1110636|4837|16591|1|49|6|1|12440|166|693|53|86.92|137.33|10.98|6696.55|581.94|4606.76|7278.49|11.63|0.00|1892.10|581.94|593.57|2474.04|2485.67|-4024.82| +2450828|59521|2450863|79287|800603|2719|10835|79287|800603|2719|10835|4|99|12|2|448|30|694|78|24.39|59.99|31.79|2199.60|2479.62|1902.42|4679.22|198.36|0.00|280.02|2479.62|2677.98|2759.64|2958.00|577.20| +2450828|59521|2450832|79287|800603|2719|10835|79287|800603|2719|10835|4|88|6|4|4123|188|694|38|81.26|113.76|9.10|3977.08|345.80|3087.88|4322.88|17.29|0.00|1729.00|345.80|363.09|2074.80|2092.09|-2742.08| +2450828|59521|2450905|79287|800603|2719|10835|79287|800603|2719|10835|4|91|16|3|14696|228|694|31|80.92|111.66|24.56|2700.10|761.36|2508.52|3461.46|6.85|76.13|484.53|685.23|692.08|1169.76|1176.61|-1823.29| +2450828|59521|2450918|79287|800603|2719|10835|79287|800603|2719|10835|4|31|6|2|5929|84|694|35|76.64|102.69|77.01|898.80|2695.35|2682.40|3594.15|53.90|0.00|1401.40|2695.35|2749.25|4096.75|4150.65|12.95| +2450828|59521|2450860|79287|800603|2719|10835|79287|800603|2719|10835|4|65|15|5|3244|68|694|4|63.01|105.22|1.05|416.68|4.20|252.04|420.88|0.09|3.06|75.72|1.14|1.23|76.86|76.95|-250.90| +2450828|59521|2450830|79287|800603|2719|10835|79287|800603|2719|10835|4|4|5|4|10249|144|694|76|44.44|66.21|45.68|1560.28|3471.68|3377.44|5031.96|173.58|0.00|1811.08|3471.68|3645.26|5282.76|5456.34|94.24| +2450828|59521|2450918|79287|800603|2719|10835|79287|800603|2719|10835|4|73|5|4|6922|182|694|95|32.86|34.83|3.48|2978.25|330.60|3121.70|3308.85|13.22|0.00|1124.80|330.60|343.82|1455.40|1468.62|-2791.10| +2450828|59521|2450841|79287|800603|2719|10835|79287|800603|2719|10835|4|8|14|3|10988|8|694|25|13.63|22.21|16.65|139.00|416.25|340.75|555.25|12.48|0.00|0.00|416.25|428.73|416.25|428.73|75.50| +2450828|59521|2450889|79287|800603|2719|10835|79287|800603|2719|10835|4|74|18|1|11216|211|694|47|54.76|57.49|48.29|432.40|2269.63|2573.72|2702.03|90.78|0.00|972.43|2269.63|2360.41|3242.06|3332.84|-304.09| +2450828|59521|2450885|79287|800603|2719|10835|79287|800603|2719|10835|4|22|17|1|13630|131|694|28|52.24|110.74|62.01|1364.44|1736.28|1462.72|3100.72|81.43|572.97|1054.20|1163.31|1244.74|2217.51|2298.94|-299.41| +2450828|59521|2450857|79287|800603|2719|10835|79287|800603|2719|10835|4|98|3|3|17539|75|694|85|41.85|71.56|57.24|1217.20|4865.40|3557.25|6082.60|0.00|0.00|2554.25|4865.40|4865.40|7419.65|7419.65|1308.15| +2450828|59521|2450833|79287|800603|2719|10835|79287|800603|2719|10835|4|89|7|4|11920|293|694|67|15.46|35.40|32.92|166.16|2205.64|1035.82|2371.80|22.05|0.00|545.38|2205.64|2227.69|2751.02|2773.07|1169.82| +2450828|59521|2450898|79287|800603|2719|10835|79287|800603|2719|10835|4|107|7|1|1966|39|694|78|78.38|166.94|120.19|3646.50|9374.82|6113.64|13021.32|93.74|0.00|6510.66|9374.82|9468.56|15885.48|15979.22|3261.18| +2450828|59521|2450830|79287|800603|2719|10835|79287|800603|2719|10835|4|97|7|4|9506|270|694|15|56.99|120.81|16.91|1558.50|253.65|854.85|1812.15|22.82|0.00|869.70|253.65|276.47|1123.35|1146.17|-601.20| +2450828|72490|2450907|45555|1511570|432|23647|45555|1511570|432|23647|4|43|8|5|835|102|695|26|74.70|194.22|130.12|1666.60|3383.12|1942.20|5049.72|135.32|0.00|2171.26|3383.12|3518.44|5554.38|5689.70|1440.92| +2450828|72490|2450837|45555|1511570|432|23647|45555|1511570|432|23647|4|64|18|4|4046|271|695|22|72.07|160.71|65.89|2086.04|1449.58|1585.54|3535.62|101.47|0.00|1202.08|1449.58|1551.05|2651.66|2753.13|-135.96| +2450828|72490|2450833|45555|1511570|432|23647|45555|1511570|432|23647|4|34|10|1|12242|11|695|99|64.49|114.14|59.35|5424.21|5875.65|6384.51|11299.86|0.00|2702.79|4293.63|3172.86|3172.86|7466.49|7466.49|-3211.65| +2450828|72490|2450897|45555|1511570|432|23647|45555|1511570|432|23647|4|94|15|1|12094|246|695|46|37.65|68.14|67.45|31.74|3102.70|1731.90|3134.44|47.16|2513.18|814.66|589.52|636.68|1404.18|1451.34|-1142.38| +2450828|72490|2450905|45555|1511570|432|23647|45555|1511570|432|23647|4|44|3|4|10436|36|695|82|86.35|97.57|4.87|7601.40|399.34|7080.70|8000.74|35.94|0.00|1280.02|399.34|435.28|1679.36|1715.30|-6681.36| +2450828|72490|2450864|45555|1511570|432|23647|45555|1511570|432|23647|4|75|16|2|4639|194|695|77|39.64|118.52|84.14|2647.26|6478.78|3052.28|9126.04|194.36|0.00|1277.43|6478.78|6673.14|7756.21|7950.57|3426.50| +2450828|72490|2450893|45555|1511570|432|23647|45555|1511570|432|23647|4|14|16|1|880|101|695|80|63.26|137.90|92.39|3640.80|7391.20|5060.80|11032.00|369.56|0.00|1433.60|7391.20|7760.76|8824.80|9194.36|2330.40| +2450828|72490|2450882|45555|1511570|432|23647|45555|1511570|432|23647|4|97|12|2|1520|49|695|6|68.51|82.21|53.43|172.68|320.58|411.06|493.26|12.82|0.00|98.64|320.58|333.40|419.22|432.04|-90.48| +2450828|72490|2450849|45555|1511570|432|23647|45555|1511570|432|23647|4|95|15|1|13417|88|695|58|93.63|222.83|86.90|7883.94|5040.20|5430.54|12924.14|181.44|2772.11|4523.42|2268.09|2449.53|6791.51|6972.95|-3162.45| +2450828|72490|2450899|45555|1511570|432|23647|45555|1511570|432|23647|4|25|8|2|14134|11|695|94|24.30|54.43|26.67|2609.44|2506.98|2284.20|5116.42|100.27|0.00|1585.78|2506.98|2607.25|4092.76|4193.03|222.78| +2450828|72490|2450900|45555|1511570|432|23647|45555|1511570|432|23647|4|21|1|3|16822|147|695|66|32.03|48.04|36.03|792.66|2377.98|2113.98|3170.64|166.45|0.00|728.64|2377.98|2544.43|3106.62|3273.07|264.00| +2450828|72490|2450867|45555|1511570|432|23647|45555|1511570|432|23647|4|107|5|4|8948|228|695|59|62.17|93.25|70.87|1320.42|4181.33|3668.03|5501.75|167.25|0.00|990.02|4181.33|4348.58|5171.35|5338.60|513.30| +2450828|72051|2450898|5677|380280|4130|49133|5677|380280|4130|49133|1|99|18|2|14300|209|696|16|26.24|30.96|27.86|49.60|445.76|419.84|495.36|4.45|0.00|34.56|445.76|450.21|480.32|484.77|25.92| +2450828|72051|2450862|5677|380280|4130|49133|5677|380280|4130|49133|1|73|17|1|1508|29|696|84|55.87|107.82|91.64|1359.12|7697.76|4693.08|9056.88|76.97|0.00|2173.08|7697.76|7774.73|9870.84|9947.81|3004.68| +2450828|72051|2450876|5677|380280|4130|49133|5677|380280|4130|49133|1|20|14|4|8878|26|696|25|38.52|71.26|28.50|1069.00|712.50|963.00|1781.50|57.00|0.00|516.50|712.50|769.50|1229.00|1286.00|-250.50| +2450828|72051|2450905|5677|380280|4130|49133|5677|380280|4130|49133|1|51|15|3|14504|147|696|56|15.52|30.10|0.30|1668.80|16.80|869.12|1685.60|1.00|0.00|286.16|16.80|17.80|302.96|303.96|-852.32| +2450828|23333|2450846|1305|689725|4898|30518|36211|1556111|2484|13704|2|84|7|4|4976|115|697|92|48.42|142.83|17.13|11564.40|1575.96|4454.64|13140.36|0.00|819.49|919.08|756.47|756.47|1675.55|1675.55|-3698.17| +2450828|23333|2450859|1305|689725|4898|30518|36211|1556111|2484|13704|2|73|4|5|12904|217|697|20|64.18|183.55|170.70|257.00|3414.00|1283.60|3671.00|102.42|0.00|293.60|3414.00|3516.42|3707.60|3810.02|2130.40| +2450828|23333|2450903|1305|689725|4898|30518|36211|1556111|2484|13704|2|85|9|2|12706|241|697|78|21.34|29.87|19.71|792.48|1537.38|1664.52|2329.86|138.36|0.00|279.24|1537.38|1675.74|1816.62|1954.98|-127.14| +2450828|23333|2450876|1305|689725|4898|30518|36211|1556111|2484|13704|2|88|12|1|17414|247|697|84|6.70|8.97|2.06|580.44|173.04|562.80|753.48|1.76|143.62|308.28|29.42|31.18|337.70|339.46|-533.38| +2450828|23333|2450867|1305|689725|4898|30518|36211|1556111|2484|13704|2|87|20|5|1912|220|697|49|88.39|240.42|26.44|10485.02|1295.56|4331.11|11780.58|103.64|0.00|588.98|1295.56|1399.20|1884.54|1988.18|-3035.55| +2450828|23333|2450908|1305|689725|4898|30518|36211|1556111|2484|13704|2|82|11|2|8144|89|697|31|50.83|119.95|19.19|3123.56|594.89|1575.73|3718.45|16.06|59.48|446.09|535.41|551.47|981.50|997.56|-1040.32| +2450828|33668|2450837|80825|882502|2932|22962|80825|882502|2932|22962|4|81|19|1|15337|50|698|50|46.53|134.93|125.48|472.50|6274.00|2326.50|6746.50|49.56|1317.54|877.00|4956.46|5006.02|5833.46|5883.02|2629.96| +2450828|33668|2450859|80825|882502|2932|22962|80825|882502|2932|22962|4|55|5|2|11074|23|698|22|44.20|72.48|1.44|1562.88|31.68|972.40|1594.56|0.63|0.00|191.18|31.68|32.31|222.86|223.49|-940.72| +2450828|33668|2450915|80825|882502|2932|22962|80825|882502|2932|22962|4|86|1|5|6560|283|698|74|32.08|76.67|11.50|4822.58|851.00|2373.92|5673.58|76.59|0.00|2155.62|851.00|927.59|3006.62|3083.21|-1522.92| +2450828|33668|2450909|80825|882502|2932|22962|80825|882502|2932|22962|4|55|5|3|12841|214|698|69|8.65|20.76|6.64|974.28|458.16|596.85|1432.44|18.32|0.00|572.70|458.16|476.48|1030.86|1049.18|-138.69| +2450828|33668|2450840|80825|882502|2932|22962|80825|882502|2932|22962|4|101|18|2|13255|217|698|54|3.04|3.34|1.77|84.78|95.58|164.16|180.36|2.34|48.74|28.62|46.84|49.18|75.46|77.80|-117.32| +2450828|33668|2450890||882502|2932|||882502|||4||||6986||698|1||236.16|162.95|73.21|||236.16||0.00|16.53|||179.48|181.10|74.83| +2450828|33668|2450878|80825|882502|2932|22962|80825|882502|2932|22962|4|5|16|4|6710|270|698|9|62.11|164.59|90.52|666.63|814.68|558.99|1481.31|46.92|293.28|577.71|521.40|568.32|1099.11|1146.03|-37.59| +2450828|33668|2450877|80825|882502|2932|22962|80825|882502|2932|22962|4|91|5|5|5467|273|698|61|16.48|20.92|11.50|574.62|701.50|1005.28|1276.12|28.06|0.00|586.82|701.50|729.56|1288.32|1316.38|-303.78| +2450828|33668|2450909|80825|882502|2932|22962|80825|882502|2932|22962|4|108|8|1|5710|50|698|47|73.31|203.06|14.21|8875.95|667.87|3445.57|9543.82|13.35|0.00|4103.57|667.87|681.22|4771.44|4784.79|-2777.70| +2450828|33668|2450838|80825|882502|2932|22962|80825|882502|2932|22962|4|66|5|5|14906|160|698|90|90.36|98.49|79.77|1684.80|7179.30|8132.40|8864.10|0.00|7179.30|3190.50|0.00|0.00|3190.50|3190.50|-8132.40| +2450828|33668|2450916|80825|882502|2932|22962|80825|882502|2932|22962|4|14|12|2|16489|237|698|87|80.90|99.50|88.55|952.65|7703.85|7038.30|8656.50|693.34|0.00|173.13|7703.85|8397.19|7876.98|8570.32|665.55| +2450828|33668|2450881|80825|882502|2932|22962|80825|882502|2932|22962|4|39|9|3|3727|74|698|4|62.29|174.41|22.67|606.96|90.68|249.16|697.64|8.16|0.00|209.28|90.68|98.84|299.96|308.12|-158.48| +2450828|16404|2450831|89818|1091255|1492|26176|89818|1091255|1492|26176|4|8|7|2|2464|300|699|23|70.43|183.11|76.90|2442.83|1768.70|1619.89|4211.53|35.37|0.00|1684.52|1768.70|1804.07|3453.22|3488.59|148.81| +2450828|16404|2450904|89818|1091255|1492|26176|89818|1091255|1492|26176|4|45|7|2|15952|64|699|99|19.04|29.32|20.52|871.20|2031.48|1884.96|2902.68|142.20|0.00|1364.22|2031.48|2173.68|3395.70|3537.90|146.52| +2450828|16404|2450870|89818|1091255|1492|26176|89818|1091255|1492|26176|4|18|13|4|16363|226|699|80|92.60|195.38|152.39|3439.20|12191.20|7408.00|15630.40|432.78|3535.44|1250.40|8655.76|9088.54|9906.16|10338.94|1247.76| +2450828|16404|2450860|89818|1091255|1492|26176|89818|1091255|1492|26176|4|37|8|3|15349|78|699|87|37.10|55.27|38.68|1443.33|3365.16|3227.70|4808.49|33.65|0.00|816.93|3365.16|3398.81|4182.09|4215.74|137.46| +2450828|16404|2450865|89818|1091255|1492|26176|89818|1091255|1492|26176|4|38|8|2|2114|28|699|28|1.75|4.28|2.48|50.40|69.44|49.00|119.84|6.24|0.00|47.88|69.44|75.68|117.32|123.56|20.44| +2450828|16404|2450868|89818|1091255|1492|26176|89818|1091255|1492|26176|4|29|12|1|11698|7|699|74|9.88|17.68|7.42|759.24|549.08|731.12|1308.32|16.47|0.00|523.18|549.08|565.55|1072.26|1088.73|-182.04| +2450828|16404|2450844|89818|1091255|1492|26176|89818|1091255|1492|26176|4|43|14|4|4862|32|699|11|49.40|88.92|36.45|577.17|400.95|543.40|978.12|12.02|0.00|39.05|400.95|412.97|440.00|452.02|-142.45| +2450828|16404|2450848|89818|1091255|1492|26176|89818|1091255|1492|26176|4|1|8|1|14623|258|699|34|92.18|250.72|107.80|4859.28|3665.20|3134.12|8524.48|256.56|0.00|1363.74|3665.20|3921.76|5028.94|5285.50|531.08| +2450828|16404|2450872|89818|1091255|1492|26176|89818|1091255|1492|26176|4|21|7|2|15368|139|699|2|87.88|248.70|54.71|387.98|109.42|175.76|497.40|0.00|0.00|193.98|109.42|109.42|303.40|303.40|-66.34| +2450828|16404|2450839|89818|1091255|1492|26176|89818|1091255|1492|26176|4|85|13|5|5444|89|699|82|72.89|209.19|2.09|16982.20|171.38|5976.98|17153.58|13.71|0.00|6346.80|171.38|185.09|6518.18|6531.89|-5805.60| +2450828|44197|2450888|50040|883650|2860|6070|50040|883650|2860|6070|4|61|13|5|13756|87|700|9|72.22|97.49|95.54|17.55|859.86|649.98|877.41|3.09|550.31|412.38|309.55|312.64|721.93|725.02|-340.43| +2450828|44197|2450895|50040|883650|2860|6070|50040|883650|2860|6070|4|6|17|3|13405|195|700|82|56.95|72.89|31.34|3407.10|2569.88|4669.90|5976.98|170.64|436.87|1792.52|2133.01|2303.65|3925.53|4096.17|-2536.89| +2450828|44197|2450855|50040|883650|2860|6070|50040|883650|2860|6070|4|27|9|5|15991|233|700|76|3.89|8.44|7.00|109.44|532.00|295.64|641.44|26.60|0.00|147.44|532.00|558.60|679.44|706.04|236.36| +2450828|44197|2450879|50040|883650|2860|6070|50040|883650|2860|6070|4|50|15|3|10345|169|700|90|47.31|104.55|4.18|9033.30|376.20|4257.90|9409.50|18.81|0.00|2822.40|376.20|395.01|3198.60|3217.41|-3881.70| +2450828|44197|2450917|50040|883650|2860|6070|50040|883650|2860|6070|4|41|17|5|6794|94|700|18|40.41|112.33|104.46|141.66|1880.28|727.38|2021.94|37.60|0.00|930.06|1880.28|1917.88|2810.34|2847.94|1152.90| +2450828|10396|2450850|74038|1724230|4822|39414|74038|1724230|4822|39414|4|65|5|1|10819|258|701|18|58.43|114.52|80.16|618.48|1442.88|1051.74|2061.36|115.43|0.00|494.64|1442.88|1558.31|1937.52|2052.95|391.14| +2450828|10396|2450886|74038|1724230|4822|39414|74038|1724230|4822|39414|4|86|4|2|17608|92|701|97|20.56|24.05|4.32|1913.81|419.04|1994.32|2332.85|16.76|0.00|325.92|419.04|435.80|744.96|761.72|-1575.28| +2450828|10396|2450852|74038|1724230|4822|39414|74038|1724230|4822|39414|4|26|9|5|1484|48|701|79|59.98|130.15|124.94|411.59|9870.26|4738.42|10281.85|789.62|0.00|3495.75|9870.26|10659.88|13366.01|14155.63|5131.84| +2450828|10396|2450917|74038|1724230|4822|39414|74038|1724230|4822|39414|4|16|5|3|6176|98|701|66|35.00|103.95|19.75|5557.20|1303.50|2310.00|6860.70|87.98|325.87|1851.96|977.63|1065.61|2829.59|2917.57|-1332.37| +2450828|10396|2450908|74038|1724230|4822|39414|74038|1724230|4822|39414|4|68|12|2|9226|299|701|58|76.17|99.78|70.84|1678.52|4108.72|4417.86|5787.24|328.69|0.00|0.00|4108.72|4437.41|4108.72|4437.41|-309.14| +2450828|10396|2450913|74038|1724230|4822|39414|74038|1724230|4822|39414|4|67|17|2|13834|150|701|70|19.36|19.55|7.23|862.40|506.10|1355.20|1368.50|45.54|0.00|40.60|506.10|551.64|546.70|592.24|-849.10| +2450828|10396|2450884|74038|1724230|4822|39414|74038|1724230|4822|39414|4|76|17|3|13418|265|701|79|3.18|5.15|3.19|154.84|252.01|251.22|406.85|5.04|0.00|36.34|252.01|257.05|288.35|293.39|0.79| +2450828|57995|2450855|15159|12043|3327|35411|15159|12043|3327|35411|4|95|14|3|5902|179|702|35|99.32|154.93|97.60|2006.55|3416.00|3476.20|5422.55|239.12|0.00|1951.95|3416.00|3655.12|5367.95|5607.07|-60.20| +2450828|57995|2450839|15159|12043|3327|35411|15159|12043|3327|35411|4|42|8|2|2671|295|702|47|60.41|127.46|126.18|60.16|5930.46|2839.27|5990.62|24.90|3439.66|1377.57|2490.80|2515.70|3868.37|3893.27|-348.47| +2450828|57995|2450893|15159|12043|3327|35411|15159|12043|3327|35411|4|8|17|3|13820|156|702|51|4.66|10.81|10.05|38.76|512.55|237.66|551.31|10.25|0.00|203.49|512.55|522.80|716.04|726.29|274.89| +2450828|57995|2450916|15159|12043|3327|35411|15159|12043|3327|35411|4|68|13|5|7828|265|702|73|67.54|187.08|93.54|6828.42|6828.42|4930.42|13656.84|68.28|0.00|2867.44|6828.42|6896.70|9695.86|9764.14|1898.00| +2450828|57995|2450894|15159|12043|3327|35411|15159|12043|3327|35411|4|36|2|4|5341|228|702|71|9.52|11.99|10.55|102.24|749.05|675.92|851.29|37.45|0.00|127.09|749.05|786.50|876.14|913.59|73.13| +2450828|57995|2450891|15159|12043|3327|35411|15159|12043|3327|35411|4|41|9|5|4903|7|702|58|2.62|4.13|1.28|165.30|74.24|151.96|239.54|2.96|0.00|78.88|74.24|77.20|153.12|156.08|-77.72| +2450828|57995|2450859|15159|12043|3327|35411|15159|12043|3327|35411|4|96|9|2|992|71|702|7|60.38|78.49|29.82|340.69|208.74|422.66|549.43|0.00|208.74|0.00|0.00|0.00|0.00|0.00|-422.66| +2450828|57995|2450835|15159|12043|3327|35411|15159|12043|3327|35411|4|82|20|2|14287|112|702|92|82.88|152.49|135.71|1543.76|12485.32|7624.96|14029.08|1123.67|0.00|3507.04|12485.32|13608.99|15992.36|17116.03|4860.36| +2450828|57995|2450830|15159|12043|3327|35411|15159|12043|3327|35411|4|49|8|5|14200|258|702|79|60.27|103.66|92.25|901.39|7287.75|4761.33|8189.14|0.00|0.00|2046.89|7287.75|7287.75|9334.64|9334.64|2526.42| +2450828|57995|2450909|15159|12043|3327|35411|15159|12043|3327|35411|4|8|8|4|3226|191|702|30|36.59|46.83|34.65|365.40|1039.50|1097.70|1404.90|41.58|0.00|674.10|1039.50|1081.08|1713.60|1755.18|-58.20| +2450828|57995|2450918|15159|12043|3327|35411|15159|12043|3327|35411|4|44|14|2|8755|215|702|30|69.64|122.56|57.60|1948.80|1728.00|2089.20|3676.80|155.52|0.00|771.90|1728.00|1883.52|2499.90|2655.42|-361.20| +2450828|57995|2450890|15159|12043|3327|35411|15159|12043|3327|35411|4|85|3|2|17935|175|702|38|10.52|24.51|13.23|428.64|502.74|399.76|931.38|31.22|155.84|279.30|346.90|378.12|626.20|657.42|-52.86| +2450828|64142|2450893|62385|139480|4928|45223|62385|139480|4928|45223|4|1|19|2|6901|138|703|13|44.42|112.38|79.78|423.80|1037.14|577.46|1460.94|62.22|0.00|248.30|1037.14|1099.36|1285.44|1347.66|459.68| +2450828|64142|2450834|62385|139480|4928|45223|62385|139480|4928|45223|4|77|1|2|4798|239|703|17|31.71|73.25|43.95|498.10|747.15|539.07|1245.25|37.35|0.00|423.30|747.15|784.50|1170.45|1207.80|208.08| +2450828|64142|2450903|62385|139480|4928|45223|62385|139480|4928|45223|4|80|6|4|13082|232|703|27|22.62|45.24|15.38|806.22|415.26|610.74|1221.48|16.61|0.00|109.89|415.26|431.87|525.15|541.76|-195.48| +2450828|64142|2450843|62385|139480|4928|45223|62385|139480|4928|45223|4|14|8|4|12226|174|703|90|75.03|216.83|201.65|1366.20|18148.50|6752.70|19514.70|362.97|0.00|4488.30|18148.50|18511.47|22636.80|22999.77|11395.80| +2450828|18777|2450907|15549|525141|686|40678|15549|525141|686|40678|2|47|19|2|8578|96|704|58|67.99|162.49|25.99|7917.00|1507.42|3943.42|9424.42|60.29|0.00|3675.46|1507.42|1567.71|5182.88|5243.17|-2436.00| +2450828|18777|2450868|15549|525141|686|40678|15549|525141|686|40678|2|66|5|4|8050|165|704|32|48.70|133.43|50.70|2647.36|1622.40|1558.40|4269.76|110.16|48.67|42.56|1573.73|1683.89|1616.29|1726.45|15.33| +2450828|18777|2450885|15549|525141|686|40678|15549|525141|686|40678|2|47|16|5|9562|65|704|66|10.33|14.87|3.12|775.50|205.92|681.78|981.42|0.00|0.00|58.74|205.92|205.92|264.66|264.66|-475.86| +2450828|18777|2450881|15549|525141|686|40678|15549|525141|686|40678|2|26|7|3|6380|200|704|64|99.47|169.09|72.70|6168.96|4652.80|6366.08|10821.76|372.22|0.00|2272.00|4652.80|5025.02|6924.80|7297.02|-1713.28| +2450828|18777|2450843|15549|525141|686|40678|15549|525141|686|40678|2|5|15|2|16285|46|704|45|27.42|53.74|46.75|314.55|2103.75|1233.90|2418.30|147.26|0.00|23.85|2103.75|2251.01|2127.60|2274.86|869.85| +2450828|18777|2450875|15549|525141|686|40678|15549|525141|686|40678|2|23|8|4|12206|216|704|52|71.70|170.64|83.61|4525.56|4347.72|3728.40|8873.28|86.95|0.00|2572.96|4347.72|4434.67|6920.68|7007.63|619.32| +2450828|18777|2450844|15549|525141|686|40678|15549|525141|686|40678|2|82|8|1|17834|170|704|4|53.99|122.55|80.88|166.68|323.52|215.96|490.20|29.11|0.00|215.68|323.52|352.63|539.20|568.31|107.56| +2450828|18777|2450857|15549|525141|686|40678|15549|525141|686|40678|2|41|7|2|14725|179|704|43|68.00|146.88|96.94|2147.42|4168.42|2924.00|6315.84|333.47|0.00|3157.92|4168.42|4501.89|7326.34|7659.81|1244.42| +2450828|18777|2450834|15549|525141|686|40678|15549|525141|686|40678|2|67|2|5|16888|126|704|6|43.87|78.52|4.71|442.86|28.26|263.22|471.12|2.54|0.00|146.04|28.26|30.80|174.30|176.84|-234.96| +2450828|18777|2450838|15549|525141|686|40678|15549|525141|686|40678|2|50|10|5|5792|94|704|37|11.10|19.42|5.04|532.06|186.48|410.70|718.54|13.05|0.00|236.80|186.48|199.53|423.28|436.33|-224.22| +2450828|18777|2450872|15549|525141|686|40678|15549|525141|686|40678|2|18|4|5|5227|105|704|78|98.86|177.94|88.97|6939.66|6939.66|7711.08|13879.32|416.37|0.00|6800.82|6939.66|7356.03|13740.48|14156.85|-771.42| +2450828|18777|2450830|15549|525141|686|40678|15549|525141|686|40678|2|77|9|3|11461|14|704|26|80.06|81.66|24.49|1486.42|636.74|2081.56|2123.16|57.30|0.00|849.16|636.74|694.04|1485.90|1543.20|-1444.82| +2450828|18777|2450839|15549|525141|686|40678|15549|525141|686|40678|2|61|12|4|3586|175|704|22|81.00|136.08|114.30|479.16|2514.60|1782.00|2993.76|125.73|0.00|568.70|2514.60|2640.33|3083.30|3209.03|732.60| +2450828|68307|2450915|15514|144899|4331|22516|15514|144899|4331|22516|1|10|15|2|5596|278|705|54|45.56|130.75|87.60|2330.10|4730.40|2460.24|7060.50|0.00|0.00|3176.82|4730.40|4730.40|7907.22|7907.22|2270.16| +2450828|68307|2450910|15514|144899|4331|22516|15514|144899|4331|22516|1|44|7|3|6496|205|705|39|55.75|144.95|31.88|4409.73|1243.32|2174.25|5653.05|24.86|0.00|2204.67|1243.32|1268.18|3447.99|3472.85|-930.93| +2450828|68307|2450843|15514|144899|4331|22516|15514|144899|4331|22516|1|34|14|5|10579|206|705|36|11.91|24.77|12.63|437.04|454.68|428.76|891.72|18.18|0.00|329.76|454.68|472.86|784.44|802.62|25.92| +2450828|68307|2450863|15514|144899|4331|22516|15514|144899|4331|22516|1|20|14|5|4894|17|705|89|3.26|5.05|4.49|49.84|399.61|290.14|449.45|6.47|291.71|0.00|107.90|114.37|107.90|114.37|-182.24| +2450828|69641|2450917|74184|677752|967|30935|74184|677752|967|30935|2|104|16|4|17482|50|706|97|35.02|80.89|4.04|7454.45|391.88|3396.94|7846.33|27.43|0.00|3373.66|391.88|419.31|3765.54|3792.97|-3005.06| +2450828|69641|2450917|74184|677752|967|30935|74184|677752|967|30935|2|19|4|4|14642|209|706|81|66.43|178.03|30.26|11969.37|2451.06|5380.83|14420.43|122.55|0.00|2162.70|2451.06|2573.61|4613.76|4736.31|-2929.77| +2450828|69641|2450852|74184|677752|967|30935|74184|677752|967|30935|2|15|10|3|11077|119|706|72|33.24|67.14|36.25|2224.08|2610.00|2393.28|4834.08|182.70|0.00|2271.60|2610.00|2792.70|4881.60|5064.30|216.72| +2450828|69641|2450849|74184|677752|967|30935|74184|677752|967|30935|2|31|6|5|1897|155|706|41|68.45|88.98|15.12|3028.26|619.92|2806.45|3648.18|0.00|0.00|1422.70|619.92|619.92|2042.62|2042.62|-2186.53| +2450828|69641|2450908|74184|677752|967|30935|74184|677752|967|30935|2|7|5|4|16372|251|706|58|92.60|214.83|90.22|7227.38|5232.76|5370.80|12460.14|366.29|0.00|124.12|5232.76|5599.05|5356.88|5723.17|-138.04| +2450828|69641|2450873|74184|677752|967|30935|74184|677752|967|30935|2|32|20|4|15094|289|706|24|73.22|85.66|57.39|678.48|1377.36|1757.28|2055.84|0.00|0.00|596.16|1377.36|1377.36|1973.52|1973.52|-379.92| +2450828|69641|2450834|74184|677752|967|30935|74184|677752|967|30935|2|46|7|5|2558|284|706|2|36.78|54.80|51.51|6.58|103.02|73.56|109.60|0.00|44.29|28.48|58.73|58.73|87.21|87.21|-14.83| +2450828|69641|2450830|74184|677752|967|30935|74184|677752|967|30935|2|12|18|5|8419|22|706|86|7.94|14.29|12.86|122.98|1105.96|682.84|1228.94|66.35|0.00|196.08|1105.96|1172.31|1302.04|1368.39|423.12| +2450828|69641|2450843|74184|677752|967|30935|74184|677752|967|30935|2|9|3|4|350|268|706|69|12.50|26.37|3.42|1583.55|235.98|862.50|1819.53|11.79|0.00|472.65|235.98|247.77|708.63|720.42|-626.52| +2450828|40502|2450840|10435|780205|375|15792|10435|780205|375|15792|2|32|15|4|2230|123|707|40|43.02|69.69|5.57|2564.80|222.80|1720.80|2787.60|11.14|0.00|1031.20|222.80|233.94|1254.00|1265.14|-1498.00| +2450828|40502|2450905|10435|780205|375|15792|10435|780205|375|15792|2|74|9|4|4345|264|707|32|32.57|97.05|80.55|528.00|2577.60|1042.24|3105.60|25.77|0.00|1179.84|2577.60|2603.37|3757.44|3783.21|1535.36| +2450828|40502|2450841|10435|780205|375|15792|10435|780205|375|15792|2|40|17|3|8761|63|707|96|64.49|118.01|103.84|1360.32|9968.64|6191.04|11328.96|398.74|0.00|3511.68|9968.64|10367.38|13480.32|13879.06|3777.60| +2450828|40502|2450907|10435|780205|375|15792|10435|780205|375|15792|2|44|5|3|10358|279|707|11|9.30|23.34|13.07|112.97|143.77|102.30|256.74|2.87|0.00|0.00|143.77|146.64|143.77|146.64|41.47| +2450828|40502|2450885|10435|780205|375|15792|10435|780205|375|15792|2|48|6|4|16052|19|707|27|53.59|127.00|29.21|2640.33|788.67|1446.93|3429.00|70.98|0.00|994.41|788.67|859.65|1783.08|1854.06|-658.26| +2450828|40502|2450910|10435|780205|375|15792|10435|780205|375|15792|2|75|20|1|3524|25|707|89|86.69|215.85|21.58|17290.03|1920.62|7715.41|19210.65|96.03|0.00|2497.34|1920.62|2016.65|4417.96|4513.99|-5794.79| +2450828|40502|2450846|10435|780205|375|15792|10435|780205|375|15792|2|105|13|1|8857|72|707|61|50.40|115.92|9.27|6505.65|565.47|3074.40|7071.12|50.89|0.00|1201.70|565.47|616.36|1767.17|1818.06|-2508.93| +2450828|40502|2450859|10435|780205|375|15792|10435|780205|375|15792|2|17|15|2|13513|98|707|14|38.95|74.78|3.73|994.70|52.22|545.30|1046.92|1.04|0.00|94.22|52.22|53.26|146.44|147.48|-493.08| +2450828|40502|2450894|10435|780205|375|15792|10435|780205|375|15792|2|16|2|3|16786|175|707|90|30.90|46.04|16.57|2652.30|1491.30|2781.00|4143.60|29.82|0.00|1284.30|1491.30|1521.12|2775.60|2805.42|-1289.70| +2450828|40502|2450906|10435|780205|375|15792|10435|780205|375|15792|2|91|2|5|1150|66|707|40|85.24|156.84|122.33|1380.40|4893.20|3409.60|6273.60|293.59|0.00|878.00|4893.20|5186.79|5771.20|6064.79|1483.60| +2450828|40502|2450906|10435|780205|375|15792|10435|780205|375|15792|2|67|14|2|11150|168|707|28|26.29|43.37|21.25|619.36|595.00|736.12|1214.36|53.55|0.00|424.76|595.00|648.55|1019.76|1073.31|-141.12| +2450828|40502|2450884|10435|780205|375|15792|10435|780205|375|15792|2|14|17|1|17083|214|707|76|34.81|77.62|31.04|3540.08|2359.04|2645.56|5899.12|23.59|0.00|2005.64|2359.04|2382.63|4364.68|4388.27|-286.52| +2450828|40502|2450907|10435|780205|375|15792|10435|780205|375|15792|2|44|2|4|10846|8|707|65|77.75|183.49|146.79|2385.50|9541.35|5053.75|11926.85|263.34|2957.81|2504.45|6583.54|6846.88|9087.99|9351.33|1529.79| +2450828|59997|2450830|98103|44390|5077|34164|98103|44390|5077|34164|2|84|18|1|11396|185|708|93|19.69|49.61|5.95|4060.38|553.35|1831.17|4613.73|22.13|0.00|414.78|553.35|575.48|968.13|990.26|-1277.82| +2450828|59997|2450834|98103|44390|5077|34164|98103|44390|5077|34164|2|85|10|3|10222|209|708|43|63.23|125.82|98.13|1190.67|4219.59|2718.89|5410.26|42.19|0.00|1839.11|4219.59|4261.78|6058.70|6100.89|1500.70| +2450828|59997|2450838|98103|44390|5077|34164|98103|44390|5077|34164|2|103|5|5|3044|217|708|60|41.14|89.27|88.37|54.00|5302.20|2468.40|5356.20|0.00|265.11|428.40|5037.09|5037.09|5465.49|5465.49|2568.69| +2450828|59997|2450834|98103|44390|5077|34164|98103|44390|5077|34164|2|12|18|1|10423|47|708|3|35.21|45.77|9.61|108.48|28.83|105.63|137.31|2.01|0.00|65.88|28.83|30.84|94.71|96.72|-76.80| +2450828|59997|2450858|98103|44390|5077|34164|98103|44390|5077|34164|2|74|7|2|11576|125|708|35|56.69|121.31|8.49|3948.70|297.15|1984.15|4245.85|0.47|285.26|1103.90|11.89|12.36|1115.79|1116.26|-1972.26| +2450828|59997|2450917|98103|44390|5077|34164|98103|44390|5077|34164|2|107|16|2|2125|81|708|34|61.39|148.56|136.67|404.26|4646.78|2087.26|5051.04|92.93|0.00|2424.20|4646.78|4739.71|7070.98|7163.91|2559.52| +2450828|33623|2450836|21894|1806972|1203|30234|21894|1806972|1203|30234|1|106|2|1|4562|254|709|72|36.39|99.70|54.83|3230.64|3947.76|2620.08|7178.40|315.82|0.00|2871.36|3947.76|4263.58|6819.12|7134.94|1327.68| +2450828|33623|2450897|21894|1806972|1203|30234|21894|1806972|1203|30234|1|64|19|2|10892|164|709|30|75.16|113.49|90.79|681.00|2723.70|2254.80|3404.70|217.89|0.00|1566.00|2723.70|2941.59|4289.70|4507.59|468.90| +2450828|33623|2450885|21894|1806972|1203|30234|21894|1806972|1203|30234|1|52|14|3|1238|191|709|80|99.85|207.68|72.68|10800.00|5814.40|7988.00|16614.40|465.15|0.00|6645.60|5814.40|6279.55|12460.00|12925.15|-2173.60| +2450828|33623|2450839|21894|1806972|1203|30234|21894|1806972|1203|30234|1|78|1|4|16957|225|709|65|18.73|44.20|34.47|632.45|2240.55|1217.45|2873.00|22.40|0.00|1119.95|2240.55|2262.95|3360.50|3382.90|1023.10| +2450828|33623|2450908|21894|1806972|1203|30234|21894|1806972|1203|30234|1|54|20|1|9337|53|709|23|88.37|256.27|48.69|4774.34|1119.87|2032.51|5894.21|3.58|1075.07|2888.11|44.80|48.38|2932.91|2936.49|-1987.71| +2450828|41708|2450843|38179|1868225|671|13564|38179|1868225|671|13564|1|92|11|5|17065|255|710|58|56.21|95.55|29.62|3823.94|1717.96|3260.18|5541.90|103.07|0.00|1883.84|1717.96|1821.03|3601.80|3704.87|-1542.22| +2450828|||38179||671|||1868225|||1||5||9878||710|19||252.25||2204.76|2587.99||||0.00|239.59|2587.99|2820.90|2827.58|3060.49|758.67| +2450828|41708|2450916|38179|1868225|671|13564|38179|1868225|671|13564|1|45|12|5|7136|60|710|62|65.18|143.39|41.58|6312.22|2577.96|4041.16|8890.18|154.67|0.00|2133.42|2577.96|2732.63|4711.38|4866.05|-1463.20| +2450828|41708|2450881|38179|1868225|671|13564|38179|1868225|671|13564|1|105|6|5|12394|19|710|82|11.10|29.08|4.94|1979.48|405.08|910.20|2384.56|28.35|0.00|309.96|405.08|433.43|715.04|743.39|-505.12| +2450828|41708|2450849|38179|1868225|671|13564|38179|1868225|671|13564|1|73|5|5|1904|80|710|91|1.27|2.28|1.41|79.17|128.31|115.57|207.48|0.00|0.00|63.70|128.31|128.31|192.01|192.01|12.74| +2450828|41708|2450840|38179|1868225|671|13564|38179|1868225|671|13564|1|32|2|5|1088|204|710|18|29.04|83.63|30.94|948.42|556.92|522.72|1505.34|0.00|0.00|677.34|556.92|556.92|1234.26|1234.26|34.20| +2450828|41708|2450833|38179|1868225|671|13564|38179|1868225|671|13564|1|48|18|5|9055|264|710|89|4.04|8.24|2.06|550.02|183.34|359.56|733.36|7.33|0.00|109.47|183.34|190.67|292.81|300.14|-176.22| +2450828|41708|2450835|38179|1868225|671|13564|38179|1868225|671|13564|1|37|9|3|8338|69|710|10|13.61|14.15|2.54|116.10|25.40|136.10|141.50|1.01|0.00|69.30|25.40|26.41|94.70|95.71|-110.70| +2450828|41708|2450836|38179|1868225|671|13564|38179|1868225|671|13564|1|78|5|3|15424|44|710|23|24.02|54.52|31.07|539.35|714.61|552.46|1253.96|50.02|0.00|413.77|714.61|764.63|1128.38|1178.40|162.15| +2450828|41708|2450841|38179|1868225|671|13564|38179|1868225|671|13564|1|39|8|4|17071|164|710|98|14.96|38.44|30.36|791.84|2975.28|1466.08|3767.12|119.01|0.00|1129.94|2975.28|3094.29|4105.22|4224.23|1509.20| +2450828|41708|2450895|38179|1868225|671|13564|38179|1868225|671|13564|1|4|4|2|13243|141|710|88|29.90|66.97|42.86|2121.68|3771.68|2631.20|5893.36|339.45|0.00|647.68|3771.68|4111.13|4419.36|4758.81|1140.48| +2450828|41708|2450838|38179|1868225|671|13564|38179|1868225|671|13564|1|106|4|5|4231|105|710|56|46.96|140.88|25.35|6469.68|1419.60|2629.76|7889.28|14.19|0.00|2681.84|1419.60|1433.79|4101.44|4115.63|-1210.16| +2450828|41708|2450918|38179|1868225|671|13564|38179|1868225|671|13564|1|104|6|4|11605|262|710|44|77.70|86.24|31.04|2428.80|1365.76|3418.80|3794.56|54.63|0.00|1176.12|1365.76|1420.39|2541.88|2596.51|-2053.04| +2450828|41708|2450888|38179|1868225|671|13564|38179|1868225|671|13564|1|83|1|4|3032|293|710|50|34.07|47.01|40.89|306.00|2044.50|1703.50|2350.50|102.22|0.00|752.00|2044.50|2146.72|2796.50|2898.72|341.00| +2450828|27220|2450914|794|1842139|284|2029|794|1842139|284|2029|2|107|11|1|17132|274|711|34|39.49|99.51|12.93|2943.72|439.62|1342.66|3383.34|17.58|0.00|574.94|439.62|457.20|1014.56|1032.14|-903.04| +2450828|27220|2450899|794|1842139|284|2029|794|1842139|284|2029|2|108|2|2|8509|178|711|40|66.26|123.90|106.55|694.00|4262.00|2650.40|4956.00|383.58|0.00|446.00|4262.00|4645.58|4708.00|5091.58|1611.60| +2450828|27220|2450851|794|1842139|284|2029|794|1842139|284|2029|2|42|1|4|11530|270|711|10|37.93|57.65|8.07|495.80|80.70|379.30|576.50|2.42|0.00|149.80|80.70|83.12|230.50|232.92|-298.60| +2450828|27220|2450883|794|1842139|284|2029|794|1842139|284|2029|2|12|10|1|4436|118|711|56|61.57|97.89|77.33|1151.36|4330.48|3447.92|5481.84|216.52|0.00|2521.12|4330.48|4547.00|6851.60|7068.12|882.56| +2450828|5650|2450918|87352|895854|3|39752|87352|895854|3|39752|1|33|12|4|3121|276|712|98|82.11|182.28|78.38|10182.20|7681.24|8046.78|17863.44|307.24|0.00|5358.64|7681.24|7988.48|13039.88|13347.12|-365.54| +2450828|5650|2450833|87352|895854|3|39752|87352|895854|3|39752|1|54|3|4|16906|53|712|46|20.40|21.62|2.16|895.16|99.36|938.40|994.52|0.00|42.72|347.76|56.64|56.64|404.40|404.40|-881.76| +2450828|||87352|895854||39752|||3|||33||2|16354|267|712|49|||31.52||1544.48||1625.82||1220.13|292.53|324.35|337.32|616.88|629.85|-615.47| +2450828|5650|2450844|87352|895854|3|39752|87352|895854|3|39752|1|30|20|4|5948|7|712|19|87.65|136.73|27.34|2078.41|519.46|1665.35|2597.87|25.19|15.58|1272.81|503.88|529.07|1776.69|1801.88|-1161.47| +2450828|5650|2450918|87352|895854|3|39752|87352|895854|3|39752|1|44|14|1|7976|32|712|63|37.87|56.80|50.55|393.75|3184.65|2385.81|3578.40|127.38|0.00|929.88|3184.65|3312.03|4114.53|4241.91|798.84| +2450828|5650|2450852|87352|895854|3|39752|87352|895854|3|39752|1|36|7|3|15205|45|712|3|84.67|89.75|1.79|263.88|5.37|254.01|269.25|0.48|0.00|0.00|5.37|5.85|5.37|5.85|-248.64| +2450828|5650|2450848|87352|895854|3|39752|87352|895854|3|39752|1|62|3|1|8965|176|712|72|54.76|63.52|13.33|3613.68|959.76|3942.72|4573.44|73.70|38.39|1783.44|921.37|995.07|2704.81|2778.51|-3021.35| +2450828|5650|2450832|87352|895854|3|39752|87352|895854|3|39752|1|93|15|4|10216|195|712|50|40.73|87.56|66.54|1051.00|3327.00|2036.50|4378.00|199.62|0.00|1532.00|3327.00|3526.62|4859.00|5058.62|1290.50| +2450828|5650|2450912|87352|895854|3|39752|87352|895854|3|39752|1|82|7|1|16852|86|712|43|81.61|159.13|36.59|5269.22|1573.37|3509.23|6842.59|141.60|0.00|2668.58|1573.37|1714.97|4241.95|4383.55|-1935.86| +2450828|5650|2450877|87352|895854|3|39752|87352|895854|3|39752|1|25|2|2|11878|270|712|90|26.62|50.04|9.50|3648.60|855.00|2395.80|4503.60|59.85|0.00|1890.90|855.00|914.85|2745.90|2805.75|-1540.80| +2450828|5650|2450869|87352|895854|3|39752|87352|895854|3|39752|1|40|2|4|10633|178|712|59|40.70|112.33|106.71|331.58|6295.89|2401.30|6627.47|214.06|944.38|2650.87|5351.51|5565.57|8002.38|8216.44|2950.21| +2450828|5650|2450913|87352|895854|3|39752|87352|895854|3|39752|1|68|2|3|5407|277|712|51|49.58|133.37|74.68|2993.19|3808.68|2528.58|6801.87|266.60|0.00|1496.34|3808.68|4075.28|5305.02|5571.62|1280.10| +2450828|72906|2450865|69686|1435525|6611|42450|22216|1480928|2767|28253|4|58|13|1|10246|179|713|14|79.61|169.56|116.99|735.98|1637.86|1114.54|2373.84|16.37|0.00|189.84|1637.86|1654.23|1827.70|1844.07|523.32| +2450828|72906|2450840|69686|1435525|6611|42450|22216|1480928|2767|28253|4|99|9|3|3932|188|713|83|40.18|116.92|100.55|1358.71|8345.65|3334.94|9704.36|8.34|7511.08|3007.92|834.57|842.91|3842.49|3850.83|-2500.37| +2450828|72906|2450903|69686|1435525|6611|42450|22216|1480928|2767|28253|4|16|5|1|10519|43|713|79|44.60|70.91|33.32|2969.61|2632.28|3523.40|5601.89|57.91|1184.52|2296.53|1447.76|1505.67|3744.29|3802.20|-2075.64| +2450828|72906|2450886|69686|1435525|6611|42450|22216|1480928|2767|28253|4|61|6|2|13378|299|713|48|6.80|18.83|17.51|63.36|840.48|326.40|903.84|2.77|748.02|90.24|92.46|95.23|182.70|185.47|-233.94| +2450828|40476||47722|||48037|47722||2834||4|64||2|4526|172|714|||||932.40|14605.80||15538.20|||6991.80|||21597.60|21597.60|9356.40| +2450828|40476|2450851|47722|1586478|2834|48037|47722|1586478|2834|48037|4|95|20|5|13591|263|714|80|6.14|13.93|3.06|869.60|244.80|491.20|1114.40|2.44|0.00|88.80|244.80|247.24|333.60|336.04|-246.40| +2450828|40476|2450844|47722|1586478|2834|48037|47722|1586478|2834|48037|4|85|10|4|4051|65|714|94|71.81|185.26|142.65|4005.34|13409.10|6750.14|17414.44|1072.72|0.00|2263.52|13409.10|14481.82|15672.62|16745.34|6658.96| +2450828|40476|2450831|47722|1586478|2834|48037|47722|1586478|2834|48037|4|106|3|5|14908|272|714|53|29.22|62.82|52.14|566.04|2763.42|1548.66|3329.46|82.90|0.00|1131.55|2763.42|2846.32|3894.97|3977.87|1214.76| +2450828|40476|2450833|47722|1586478|2834|48037|47722|1586478|2834|48037|4|104|2|4|17870|163|714|91|52.45|90.21|55.93|3119.48|5089.63|4772.95|8209.11|242.26|1628.68|4022.20|3460.95|3703.21|7483.15|7725.41|-1312.00| +2450828|40476|2450851|47722|1586478|2834|48037|47722|1586478|2834|48037|4|49|1|3|9481|90|714|56|69.40|107.57|24.74|4638.48|1385.44|3886.40|6023.92|83.12|0.00|1445.36|1385.44|1468.56|2830.80|2913.92|-2500.96| +2450828|40476|2450896|47722|1586478|2834|48037|47722|1586478|2834|48037|4|100|13|2|13045|102|714|48|50.93|58.56|21.66|1771.20|1039.68|2444.64|2810.88|62.38|0.00|618.24|1039.68|1102.06|1657.92|1720.30|-1404.96| +2450828|48461|2450861|65614|1525283|2999|9068|65614|1525283|2999|9068|2|101|18|1|12482|19|715|11|64.93|150.63|105.44|497.09|1159.84|714.23|1656.93|69.59|0.00|530.20|1159.84|1229.43|1690.04|1759.63|445.61| +2450828|48461|2450875|65614|1525283|2999|9068|65614|1525283|2999|9068|2|85|14|4|14210|199|715|30|28.18|69.32|6.23|1892.70|186.90|845.40|2079.60|14.95|0.00|561.30|186.90|201.85|748.20|763.15|-658.50| +2450828|48461|2450842|65614|1525283|2999|9068|65614|1525283|2999|9068|2|3|2|5|3682|53|715|12|47.66|77.68|41.17|438.12|494.04|571.92|932.16|4.94|0.00|195.72|494.04|498.98|689.76|694.70|-77.88| +2450828|48461|2450835|65614|1525283|2999|9068|65614|1525283|2999|9068|2|48|16|1|9308|184|715|55|67.50|152.55|50.34|5621.55|2768.70|3712.50|8390.25|55.37|0.00|671.00|2768.70|2824.07|3439.70|3495.07|-943.80| +2450828|48461|2450899|65614|1525283|2999|9068|65614|1525283|2999|9068|2|103|3|1|11246|193|715|22|99.32|287.03|77.49|4609.88|1704.78|2185.04|6314.66|4.77|1636.58|315.70|68.20|72.97|383.90|388.67|-2116.84| +2450828|48461|2450874|65614|1525283|2999|9068|65614|1525283|2999|9068|2|100|5|4|17752|97|715|32|98.13|154.06|114.00|1281.92|3648.00|3140.16|4929.92|328.32|0.00|1035.20|3648.00|3976.32|4683.20|5011.52|507.84| +2450828|48461|2450899|65614|1525283|2999|9068|65614|1525283|2999|9068|2|47|13|1|15136|172|715|32|60.24|173.49|39.90|4274.88|1276.80|1927.68|5551.68|89.37|0.00|776.96|1276.80|1366.17|2053.76|2143.13|-650.88| +2450828|48461|2450877|65614|1525283|2999|9068|65614|1525283|2999|9068|2|74|13|5|13162|67|715|49|20.81|25.38|14.46|535.08|708.54|1019.69|1243.62|0.00|0.00|335.65|708.54|708.54|1044.19|1044.19|-311.15| +2450828|48461|2450869|65614|1525283|2999|9068|65614|1525283|2999|9068|2|15|11|4|6092|187|715|73|52.21|102.85|22.62|5856.79|1651.26|3811.33|7508.05|82.56|0.00|3228.06|1651.26|1733.82|4879.32|4961.88|-2160.07| +2450828|48461|2450847|65614|1525283|2999|9068|65614|1525283|2999|9068|2|35|7|5|10510|229|715|73|52.14|135.04|32.40|7492.72|2365.20|3806.22|9857.92|94.60|0.00|2563.03|2365.20|2459.80|4928.23|5022.83|-1441.02| +2450829|70079|2450838|55540|1671343|746|8596|55540|1671343|746|8596|2|27|12|5|823|205|716|28|22.96|63.82|40.20|661.36|1125.60|642.88|1786.96|56.28|0.00|0.00|1125.60|1181.88|1125.60|1181.88|482.72| +2450829|70079|2450909|55540|1671343|746|8596|55540|1671343|746|8596|2|10|12|4|3811|201|716|53|76.52|139.26|54.31|4502.35|2878.43|4055.56|7380.78|0.00|201.49|2140.14|2676.94|2676.94|4817.08|4817.08|-1378.62| +2450829|70079|2450893|55540|1671343|746|8596|55540|1671343|746|8596|2|84|4|2|4270|104|716|5|21.81|47.54|36.13|57.05|180.65|109.05|237.70|0.00|0.00|66.55|180.65|180.65|247.20|247.20|71.60| +2450829|70079|2450840|55540|1671343|746|8596|55540|1671343|746|8596|2|71|4|2|259|55|716|77|31.00|74.40|55.80|1432.20|4296.60|2387.00|5728.80|128.89|0.00|400.40|4296.60|4425.49|4697.00|4825.89|1909.60| +2450829|70079|2450901|55540|1671343|746|8596|55540|1671343|746|8596|2|103|14|2|16876|115|716|72|44.32|50.08|49.07|72.72|3533.04|3191.04|3605.76|123.65|1059.91|792.72|2473.13|2596.78|3265.85|3389.50|-717.91| +2450829|70079|2450843|55540|1671343|746|8596|55540|1671343|746|8596|2|80|20|4|2744|227|716|67|11.55|23.67|20.82|190.95|1394.94|773.85|1585.89|69.74|0.00|728.96|1394.94|1464.68|2123.90|2193.64|621.09| +2450829|76301|2450918|5683|1412335|6187|18439|5683|1412335|6187|18439|2|105|15|4|12872|267|717|56|59.75|138.62|33.26|5900.16|1862.56|3346.00|7762.72|14.52|409.76|3881.36|1452.80|1467.32|5334.16|5348.68|-1893.20| +2450829|76301|2450903|5683|1412335|6187|18439|5683|1412335|6187|18439|2|55|11|2|2515|61|717|63|65.23|112.84|13.54|6255.90|853.02|4109.49|7108.92|59.71|0.00|2559.06|853.02|912.73|3412.08|3471.79|-3256.47| +2450829|76301|2450880|5683|1412335|6187|18439|5683|1412335|6187|18439|2|58|2|3|15098|67|717|36|20.17|52.24|12.53|1429.56|451.08|726.12|1880.64|4.51|0.00|0.00|451.08|455.59|451.08|455.59|-275.04| +2450829|76301|2450884|5683|1412335|6187|18439|5683|1412335|6187|18439|2|4|1|1|5668|228|717|84|2.73|5.13|2.61|211.68|219.24|229.32|430.92|13.15|0.00|210.84|219.24|232.39|430.08|443.23|-10.08| +2450829|55090|2450858|47923|1609316|662|18008|47923|1609316|662|18008|1|47|10|2|16333|201|718|91|74.13|183.84|130.52|4852.12|11877.32|6745.83|16729.44|0.00|0.00|6691.23|11877.32|11877.32|18568.55|18568.55|5131.49| +2450829|55090|2450904|47923|1609316|662|18008|47923|1609316|662|18008|1|35|15|3|4730|92|718|64|4.37|5.24|3.87|87.68|247.68|279.68|335.36|5.12|190.71|12.80|56.97|62.09|69.77|74.89|-222.71| +2450829|55090|2450842|47923|1609316|662|18008|47923|1609316|662|18008|1|51|14|3|12973|44|718|17|26.89|62.11|21.73|686.46|369.41|457.13|1055.87|33.24|0.00|168.81|369.41|402.65|538.22|571.46|-87.72| +2450829|55090|2450884|47923|1609316|662|18008|47923|1609316|662|18008|1|57|17|3|3580|73|718|19|85.67|225.31|0.00|4280.89|0.00|1627.73|4280.89|0.00|0.00|1883.47|0.00|0.00|1883.47|1883.47|-1627.73| +2450829|55090|2450904|47923|1609316|662|18008|47923|1609316|662|18008|1|15|10|2|7336|66|718|12|12.12|27.87|9.19|224.16|110.28|145.44|334.44|9.92|0.00|43.44|110.28|120.20|153.72|163.64|-35.16| +2450829|55090|2450838|47923|1609316|662|18008|47923|1609316|662|18008|1|24|17|3|17959|63|718|67|3.11|3.38|0.16|215.74|10.72|208.37|226.46|0.10|0.00|62.98|10.72|10.82|73.70|73.80|-197.65| +2450829|55090|2450907|47923|1609316|662|18008|47923|1609316|662|18008|1|92|19|3|4453|73|718|92|67.66|97.43|76.96|1883.24|7080.32|6224.72|8963.56|212.40|0.00|1702.92|7080.32|7292.72|8783.24|8995.64|855.60| +2450829|55090|2450894|47923|1609316|662|18008|47923|1609316|662|18008|1|40|13|1|13744|95|718|82|36.18|85.38|17.07|5601.42|1399.74|2966.76|7001.16|69.98|0.00|1890.10|1399.74|1469.72|3289.84|3359.82|-1567.02| +2450829|55090|2450863|47923|1609316|662|18008|47923|1609316|662|18008|1|54|16|4|3962|13|718|92|83.58|111.99|15.67|8861.44|1441.64|7689.36|10303.08|9.65|475.74|2162.92|965.90|975.55|3128.82|3138.47|-6723.46| +2450829|55090|2450850|47923|1609316|662|18008|47923|1609316|662|18008|1|5|7|2|5186|64|718|66|28.06|62.85|35.82|1783.98|2364.12|1851.96|4148.10|23.64|0.00|1866.48|2364.12|2387.76|4230.60|4254.24|512.16| +2450829|55090|2450890|47923|1609316|662|18008|47923|1609316|662|18008|1|80|2|3|17050|31|718|70|39.28|98.20|64.81|2337.30|4536.70|2749.60|6874.00|90.73|0.00|1649.20|4536.70|4627.43|6185.90|6276.63|1787.10| +2450829|55090|2450887|47923|1609316|662|18008|47923|1609316|662|18008|1|17|13|2|572|123|718|57|35.12|41.79|13.79|1596.00|786.03|2001.84|2382.03|62.88|0.00|833.34|786.03|848.91|1619.37|1682.25|-1215.81| +2450829|55090|2450845|47923|1609316|662|18008|47923|1609316|662|18008|1|26|7|2|853|57|718|29|50.18|93.33|67.19|758.06|1948.51|1455.22|2706.57|155.88|0.00|1244.97|1948.51|2104.39|3193.48|3349.36|493.29| +2450829|55090|2450840|47923|1609316|662|18008|47923|1609316|662|18008|1|86|9|1|10508|22|718|71|55.89|78.80|23.64|3916.36|1678.44|3968.19|5594.80|34.91|805.65|1174.34|872.79|907.70|2047.13|2082.04|-3095.40| +2450829|65808|2450895|30774|1436698|5022|37746|30774|1436698|5022|37746|2|6|8|3|1760|6|719|76|61.46|157.33|102.26|4185.32|7771.76|4670.96|11957.08|108.80|6217.40|3706.52|1554.36|1663.16|5260.88|5369.68|-3116.60| +2450829|65808|2450891|30774|1436698|5022|37746|30774|1436698|5022|37746|2|5|4|4|4616|194|719|85|31.54|64.65|28.44|3077.85|2417.40|2680.90|5495.25|72.52|0.00|1208.70|2417.40|2489.92|3626.10|3698.62|-263.50| +2450829|65808|2450892|30774|1436698|5022|37746|30774|1436698|5022|37746|2|20|20|4|5200|249|719|18|28.21|29.05|24.40|83.70|439.20|507.78|522.90|30.74|0.00|214.38|439.20|469.94|653.58|684.32|-68.58| +2450829|65808|2450896|30774|1436698|5022|37746|30774|1436698|5022|37746|2|60|4|3|15916|91|719|19|12.01|24.26|9.70|276.64|184.30|228.19|460.94|3.68|0.00|216.60|184.30|187.98|400.90|404.58|-43.89| +2450829|65808|2450919|30774|1436698|5022|37746|30774|1436698|5022|37746|2|103|8|2|3487|185|719|41|35.53|86.33|84.60|70.93|3468.60|1456.73|3539.53|242.80|0.00|424.35|3468.60|3711.40|3892.95|4135.75|2011.87| +2450829|43226|2450863|74671|714296|1568|8840|74671|714296|1568|8840|2|6|8|4|1981|69|720|8|12.36|14.83|4.15|85.44|33.20|98.88|118.64|1.66|0.00|23.68|33.20|34.86|56.88|58.54|-65.68| +2450829|43226|2450903|74671|714296|1568|8840|74671|714296|1568|8840|2|89|1|5|8230|119|720|90|38.34|101.98|50.99|4589.10|4589.10|3450.60|9178.20|413.01|0.00|2110.50|4589.10|5002.11|6699.60|7112.61|1138.50| +2450829|43226|2450831|74671|714296|1568|8840|74671|714296|1568|8840|2|85|18|3|13402|47|720|45|65.69|162.91|34.21|5791.50|1539.45|2956.05|7330.95|46.18|0.00|2492.10|1539.45|1585.63|4031.55|4077.73|-1416.60| +2450829|43226|2450901|74671|714296|1568|8840|74671|714296|1568|8840|2|13|14|1|3496|273|720|97|18.31|32.04|25.31|652.81|2455.07|1776.07|3107.88|49.10|0.00|808.01|2455.07|2504.17|3263.08|3312.18|679.00| +2450829|43226|2450854|74671|714296|1568|8840|74671|714296|1568|8840|2|12|3|3|10963|103|720|69|14.36|16.51|4.12|854.91|284.28|990.84|1139.19|0.22|272.90|136.62|11.38|11.60|148.00|148.22|-979.46| +2450829|62776|2450864|46100|1558571|2689|39560|46100|1558571|2689|39560|4|53|7|3|7633|297|721|59|55.53|145.48|65.46|4721.18|3862.14|3276.27|8583.32|115.86|0.00|342.79|3862.14|3978.00|4204.93|4320.79|585.87| +2450829|62776|2450886|46100|1558571|2689|39560|46100|1558571|2689|39560|4|7|14|2|5893|248|721|81|60.39|109.30|6.55|8322.75|530.55|4891.59|8853.30|1.37|461.57|2478.60|68.98|70.35|2547.58|2548.95|-4822.61| +2450829|62776|2450858|46100|1558571|2689|39560|46100|1558571|2689|39560|4|45|6|4|184|83|721|66|38.93|87.20|72.37|978.78|4776.42|2569.38|5755.20|0.00|0.00|2647.26|4776.42|4776.42|7423.68|7423.68|2207.04| +2450829|62776|2450884|46100|1558571|2689|39560|46100|1558571|2689|39560|4|76|13|1|16213|195|721|72|35.87|36.22|22.81|965.52|1642.32|2582.64|2607.84|82.11|0.00|651.60|1642.32|1724.43|2293.92|2376.03|-940.32| +2450829|62776|2450841|46100|1558571|2689|39560|46100|1558571|2689|39560|4|66|5|5|4130|225|721|66|71.17|155.15|9.30|9626.10|613.80|4697.22|10239.90|4.29|552.42|4709.76|61.38|65.67|4771.14|4775.43|-4635.84| +2450829|62776|2450842|46100|1558571|2689|39560|46100|1558571|2689|39560|4|1|15|4|13735|135|721|96|15.53|39.75|21.06|1794.24|2021.76|1490.88|3816.00|40.43|0.00|1296.96|2021.76|2062.19|3318.72|3359.15|530.88| +2450829|62776|2450852|46100|1558571|2689|39560|46100|1558571|2689|39560|4|104|12|4|15727|111|721|74|91.60|236.32|47.26|13990.44|3497.24|6778.40|17487.68|0.00|1853.53|3497.24|1643.71|1643.71|5140.95|5140.95|-5134.69| +2450829|62776|2450875|46100|1558571|2689|39560|46100|1558571|2689|39560|4|2|16|3|10831|235|721|47|68.17|114.52|0.00|5382.44|0.00|3203.99|5382.44|0.00|0.00|0.00|0.00|0.00|0.00|0.00|-3203.99| +2450829|62776|2450903|46100|1558571|2689|39560|46100|1558571|2689|39560|4|25|10|2|10868|108|721|61|3.20|5.34|3.84|91.50|234.24|195.20|325.74|18.73|0.00|126.88|234.24|252.97|361.12|379.85|39.04| +2450829|62776|2450859|46100|1558571|2689|39560|46100|1558571|2689|39560|4|108|12|5|6397|186|721|8|34.73|35.07|14.37|165.60|114.96|277.84|280.56|9.19|0.00|5.60|114.96|124.15|120.56|129.75|-162.88| +2450829|68317|2450862|45036|998111|6386|44708|45036|998111|6386|44708|2|16|13|1|15385|224|722|27|16.14|41.47|41.47|0.00|1119.69|435.78|1119.69|44.78|0.00|514.89|1119.69|1164.47|1634.58|1679.36|683.91| +2450829|68317|2450836|45036|998111|6386|44708|45036|998111|6386|44708|2|34|8|1|10732|282|722|56|27.17|57.60|51.84|322.56|2903.04|1521.52|3225.60|0.00|0.00|580.16|2903.04|2903.04|3483.20|3483.20|1381.52| +2450829|68317|2450832|45036|998111|6386|44708|45036|998111|6386|44708|2|5|15|1|9787|191|722|86|76.62|95.00|74.10|1797.40|6372.60|6589.32|8170.00|0.00|191.17|1879.10|6181.43|6181.43|8060.53|8060.53|-407.89| +2450829|68317|2450843|45036|998111|6386|44708|45036|998111|6386|44708|2|105|6|5|3508|53|722|58|17.40|38.28|15.31|1332.26|887.98|1009.20|2220.24|62.15|0.00|865.36|887.98|950.13|1753.34|1815.49|-121.22| +2450829|68317|2450900|45036|998111|6386|44708|45036|998111|6386|44708|2|50|16|1|1264|269|722|39|40.43|72.77|10.18|2441.01|397.02|1576.77|2838.03|0.00|0.00|1220.31|397.02|397.02|1617.33|1617.33|-1179.75| +2450829|47755|2450885|42348|883577|2853|6129|42348|883577|2853|6129|1|86|2|1|16297|249|723|22|17.18|44.83|37.65|157.96|828.30|377.96|986.26|51.60|91.11|118.14|737.19|788.79|855.33|906.93|359.23| +2450829|47755|2450900|42348|883577|2853|6129|42348|883577|2853|6129|1|104|3|2|5008|131|723|85|42.78|121.92|12.19|9327.05|1036.15|3636.30|10363.20|0.00|269.39|4040.90|766.76|766.76|4807.66|4807.66|-2869.54| +2450829|47755|2450904|42348|883577|2853|6129|42348|883577|2853|6129|1|15|7|5|6967|241|723|43|4.27|10.24|7.88|101.48|338.84|183.61|440.32|0.00|0.00|34.83|338.84|338.84|373.67|373.67|155.23| +2450829|47755|2450865|42348|883577|2853|6129|42348|883577|2853|6129|1|91|12|1|17294|87|723|50|19.41|29.11|19.79|466.00|989.50|970.50|1455.50|9.30|524.43|349.00|465.07|474.37|814.07|823.37|-505.43| +2450829|47755|2450904|42348|883577|2853|6129|42348|883577|2853|6129|1|96|7|1|3445|59|723|49|82.77|134.08|18.77|5650.19|919.73|4055.73|6569.92|29.06|193.14|2956.17|726.59|755.65|3682.76|3711.82|-3329.14| +2450829|47755|2450842|42348|883577|2853|6129|42348|883577|2853|6129|1|60|3|1|10525|252|723|17|72.55|74.00|55.50|314.50|943.50|1233.35|1258.00|75.48|0.00|100.64|943.50|1018.98|1044.14|1119.62|-289.85| +2450829|56940|2450878|30190|304794|4198|7602|30190|304794|4198|7602|4|82|18|4|3382|55|724|46|19.11|45.86|18.80|1244.76|864.80|879.06|2109.56|60.53|0.00|653.66|864.80|925.33|1518.46|1578.99|-14.26| +2450829|56940|2450865|30190|304794|4198|7602|30190|304794|4198|7602|4|37|9|2|15931|38|724|22|47.97|102.17|20.43|1798.28|449.46|1055.34|2247.74|4.49|0.00|112.20|449.46|453.95|561.66|566.15|-605.88| +2450829|56940|2450894|30190|304794|4198|7602|30190|304794|4198|7602|4|7|12|3|1066|132|724|43|58.63|157.12|23.56|5743.08|1013.08|2521.09|6756.16|40.52|0.00|3107.61|1013.08|1053.60|4120.69|4161.21|-1508.01| +2450829|56940|2450832|30190|304794|4198|7602|30190|304794|4198|7602|4|45|15|1|15088|296|724|57|94.88|196.40|109.98|4925.94|6268.86|5408.16|11194.80|233.20|438.82|3246.15|5830.04|6063.24|9076.19|9309.39|421.88| +2450829|56940|2450873|30190|304794|4198|7602|30190|304794|4198|7602|4|2|7|2|11864|113|724|39|8.47|12.70|4.19|331.89|163.41|330.33|495.30|4.90|0.00|123.63|163.41|168.31|287.04|291.94|-166.92| +2450829|77051|2450883|50700|927007|2867|36374|50700|927007|2867|36374|4|21|11|3|4621|55|725|59|53.86|66.24|2.64|3752.40|155.76|3177.74|3908.16|4.67|0.00|1289.15|155.76|160.43|1444.91|1449.58|-3021.98| +2450829|77051|2450880|50700|927007|2867|36374|50700|927007|2867|36374|4|51|1|2|5935|50|725|7|96.13|124.96|4.99|839.79|34.93|672.91|874.72|3.14|0.00|349.86|34.93|38.07|384.79|387.93|-637.98| +2450829|77051|2450898|50700|927007|2867|36374|50700|927007|2867|36374|4|97|11|1|11386|156|725|1|10.49|29.79|27.40|2.39|27.40|10.49|29.79|0.82|0.00|2.68|27.40|28.22|30.08|30.90|16.91| +2450829|77051|2450845|50700|927007|2867|36374|50700|927007|2867|36374|4|65|12|5|2582|15|725|71|51.61|152.76|151.23|108.63|10737.33|3664.31|10845.96|644.23|0.00|2819.41|10737.33|11381.56|13556.74|14200.97|7073.02| +2450829|77051|2450915|50700|927007|2867|36374|50700|927007|2867|36374|4|18|20|5|15550|122|725|93|46.14|56.75|9.64|4381.23|896.52|4291.02|5277.75|0.00|0.00|52.08|896.52|896.52|948.60|948.60|-3394.50| +2450829|77051|2450855|50700|927007|2867|36374|50700|927007|2867|36374|4|54|17|2|6104|154|725|26|49.46|86.55|44.14|1102.66|1147.64|1285.96|2250.30|45.90|0.00|1012.44|1147.64|1193.54|2160.08|2205.98|-138.32| +2450829|77051|2450919|50700|927007|2867|36374|50700|927007|2867|36374|4|69|18|3|10402|196|725|9|40.93|92.91|73.39|175.68|660.51|368.37|836.19|6.60|0.00|217.35|660.51|667.11|877.86|884.46|292.14| +2450829|77051|2450878|50700|927007|2867|36374|50700|927007|2867|36374|4|60|2|1|10594|51|725|2|89.86|146.47|143.54|5.86|287.08|179.72|292.94|2.87|0.00|20.50|287.08|289.95|307.58|310.45|107.36| +2450829|77051|2450882|50700|927007|2867|36374|50700|927007|2867|36374|4|58|19|4|11786|21|725|86|46.93|113.10|35.06|6711.44|3015.16|4035.98|9726.60|60.30|0.00|1167.02|3015.16|3075.46|4182.18|4242.48|-1020.82| +2450829|77051|2450884|50700|927007|2867|36374|50700|927007|2867|36374|4|79|6|3|802|242|725|43|34.70|74.60|61.17|577.49|2630.31|1492.10|3207.80|184.12|0.00|512.99|2630.31|2814.43|3143.30|3327.42|1138.21| +2450829|77051|2450868|50700|927007|2867|36374|50700|927007|2867|36374|4|108|5|1|364|170|725|84|9.89|28.38|12.77|1311.24|1072.68|830.76|2383.92|96.54|0.00|643.44|1072.68|1169.22|1716.12|1812.66|241.92| +2450829|77051|2450866|50700|927007|2867|36374|50700|927007|2867|36374|4|76|13|2|14930|74|725|32|38.40|81.40|52.91|911.68|1693.12|1228.80|2604.80|33.86|0.00|1198.08|1693.12|1726.98|2891.20|2925.06|464.32| +2450829|44876|2450847|52283|1796255|3074|42693|52283|1796255|3074|42693|1|27|6|3|1310|139|726|32|98.36|119.01|11.90|3427.52|380.80|3147.52|3808.32|11.42|0.00|1180.48|380.80|392.22|1561.28|1572.70|-2766.72| +2450829|44876|2450867|52283|1796255|3074|42693|52283|1796255|3074|42693|1|32|3|1|2654|151|726|61|93.24|107.22|12.86|5755.96|784.46|5687.64|6540.42|7.84|0.00|1046.15|784.46|792.30|1830.61|1838.45|-4903.18| +2450829|44876|2450878|52283|1796255|3074|42693|52283|1796255|3074|42693|1|45|15|2|745|291|726|13|64.63|138.95|33.34|1372.93|433.42|840.19|1806.35|30.33|0.00|632.19|433.42|463.75|1065.61|1095.94|-406.77| +2450829|44876|2450897|52283|1796255|3074|42693|52283|1796255|3074|42693|1|99|17|5|16402|239|726|85|31.64|75.93|34.16|3550.45|2903.60|2689.40|6454.05|116.14|0.00|2774.40|2903.60|3019.74|5678.00|5794.14|214.20| +2450829|44876|2450887|52283|1796255|3074|42693|52283|1796255|3074|42693|1|33|1|4|12673|70|726|76|90.06|262.07|133.65|9759.92|10157.40|6844.56|19917.32|711.01|0.00|9759.16|10157.40|10868.41|19916.56|20627.57|3312.84| +2450829|44876|2450854|52283|1796255|3074|42693|52283|1796255|3074|42693|1|68|17|1|2264|219|726|81|48.47|90.63|68.87|1762.56|5578.47|3926.07|7341.03|167.35|0.00|3596.40|5578.47|5745.82|9174.87|9342.22|1652.40| +2450829|27971|2450854|18996|488227|5905|40736|18996|488227|5905|40736|1|100|18|2|4309|10|727|15|75.92|151.84|19.73|1981.65|295.95|1138.80|2277.60|8.87|0.00|22.65|295.95|304.82|318.60|327.47|-842.85| +2450829|27971|2450884|18996|488227|5905|40736|18996|488227|5905|40736|1|26|12|5|12259|243|727|51|9.89|17.90|6.44|584.46|328.44|504.39|912.90|9.85|0.00|373.83|328.44|338.29|702.27|712.12|-175.95| +2450829|27971|2450860|18996|488227|5905|40736|18996|488227|5905|40736|1|71|14|2|6499|154|727|17|87.09|241.23|130.26|1886.49|2214.42|1480.53|4100.91|132.86|0.00|122.91|2214.42|2347.28|2337.33|2470.19|733.89| +2450829|27971|2450894|18996|488227|5905|40736|18996|488227|5905|40736|1|40|7|4|15745|77|727|72|25.78|67.28|52.47|1066.32|3777.84|1856.16|4844.16|188.89|0.00|1743.84|3777.84|3966.73|5521.68|5710.57|1921.68| +2450829|27971|2450875|18996|488227|5905|40736|18996|488227|5905|40736|1|25|4|5|2917|154|727|1|65.24|115.47|28.86|86.61|28.86|65.24|115.47|0.51|11.83|40.41|17.03|17.54|57.44|57.95|-48.21| +2450829|27971|2450874|18996|488227|5905|40736|18996|488227|5905|40736|1|101|5|5|8738|202|727|95|22.03|30.18|11.16|1806.90|1060.20|2092.85|2867.10|10.60|0.00|57.00|1060.20|1070.80|1117.20|1127.80|-1032.65| +2450829|27971|2450916|18996|488227|5905|40736|18996|488227|5905|40736|1|15|5|2|4927|281|727|22|63.69|185.97|46.49|3068.56|1022.78|1401.18|4091.34|20.45|0.00|1800.04|1022.78|1043.23|2822.82|2843.27|-378.40| +2450829|27971|2450894|18996|488227|5905|40736|18996|488227|5905|40736|1|65|19|1|5920|195|727|17|55.29|136.56|75.10|1044.82|1276.70|939.93|2321.52|25.53|0.00|46.41|1276.70|1302.23|1323.11|1348.64|336.77| +2450829|27971|2450873|18996|488227|5905|40736|18996|488227|5905|40736|1|70|1|5|1258|135|727|86|52.86|111.53|15.61|8249.12|1342.46|4545.96|9591.58|0.00|805.47|2876.70|536.99|536.99|3413.69|3413.69|-4008.97| +2450829|38677|2450831|24777|1286846|562|35878|24777|1286846|562|35878|4|6|1|3|5179|72|728|98|77.73|157.79|58.38|9742.18|5721.24|7617.54|15463.42|457.69|0.00|4174.80|5721.24|6178.93|9896.04|10353.73|-1896.30| +2450829|38677|2450898|24777|1286846|562|35878|24777|1286846|562|35878|4|83|4|1|13090|225|728|60|75.64|133.12|70.55|3754.20|4233.00|4538.40|7987.20|211.65|0.00|319.20|4233.00|4444.65|4552.20|4763.85|-305.40| +2450829|38677|2450850|24777|1286846|562|35878|24777|1286846|562|35878|4|16|20|5|9542|235|728|63|60.26|141.61|26.90|7226.73|1694.70|3796.38|8921.43|67.78|0.00|1516.41|1694.70|1762.48|3211.11|3278.89|-2101.68| +2450829|38677|2450844|24777|1286846|562|35878|24777|1286846|562|35878|4|71|19|1|3860|166|728|60|46.09|72.82|45.14|1660.80|2708.40|2765.40|4369.20|0.00|0.00|1965.60|2708.40|2708.40|4674.00|4674.00|-57.00| +2450829|38677|2450871|24777|1286846|562|35878|24777|1286846|562|35878|4|61|17|3|15793|88|728|32|90.56|159.38|6.37|4896.32|203.84|2897.92|5100.16|18.34|0.00|2498.88|203.84|222.18|2702.72|2721.06|-2694.08| +2450829|38677|2450885|24777|1286846|562|35878|24777|1286846|562|35878|4|89|18|2|14989|290|728|24|94.20|158.25|30.06|3076.56|721.44|2260.80|3798.00|43.28|0.00|683.52|721.44|764.72|1404.96|1448.24|-1539.36| +2450829|38677|2450845|24777|1286846|562|35878|24777|1286846|562|35878|4|4|5|4|1628|285|728|38|61.77|177.89|90.72|3312.46|3447.36|2347.26|6759.82|34.47|0.00|1554.58|3447.36|3481.83|5001.94|5036.41|1100.10| +2450829|32104|2450885|41830|1775341|6452|41163|24827|209738|49|39960|1|70|17|3|2318|77|729|76|76.48|164.43|60.83|7873.60|4623.08|5812.48|12496.68|277.38|0.00|2873.56|4623.08|4900.46|7496.64|7774.02|-1189.40| +2450829|32104|2450874|41830|1775341|6452|41163|24827|209738|49|39960|1|38|12|1|7708|231|729|58|20.60|52.11|41.68|604.94|2417.44|1194.80|3022.38|145.04|0.00|634.52|2417.44|2562.48|3051.96|3197.00|1222.64| +2450829|32104|2450890|41830|1775341|6452|41163|24827|209738|49|39960|1|11|15|5|11926|148|729|95|70.03|116.24|83.69|3092.25|7950.55|6652.85|11042.80|79.50|0.00|4858.30|7950.55|8030.05|12808.85|12888.35|1297.70| +2450829|32104|2450860|41830|1775341|6452|41163|24827|209738|49|39960|1|98|8|5|2179|16|729|94|82.68|138.07|35.89|9604.92|3373.66|7771.92|12978.58|0.00|1855.51|3763.76|1518.15|1518.15|5281.91|5281.91|-6253.77| +2450829|32104|2450848|41830|1775341|6452|41163|24827|209738|49|39960|1|40|1|4|6475|65|729|18|87.57|157.62|0.00|2837.16|0.00|1576.26|2837.16|0.00|0.00|595.80|0.00|0.00|595.80|595.80|-1576.26| +2450829|32104|2450875|41830|1775341|6452|41163|24827|209738|49|39960|1|94|3|1|3877|21|729|44|25.06|63.15|18.94|1945.24|833.36|1102.64|2778.60|33.33|0.00|1166.88|833.36|866.69|2000.24|2033.57|-269.28| +2450829|32104|2450900|41830|1775341|6452|41163|24827|209738|49|39960|1|81|1|5|10210|263|729|38|98.63|128.21|102.56|974.70|3897.28|3747.94|4871.98|18.70|2026.58|2094.94|1870.70|1889.40|3965.64|3984.34|-1877.24| +2450829|32104|2450850|41830|1775341|6452|41163|24827|209738|49|39960|1|8|13|3|5215|84|729|65|23.40|50.07|49.06|65.65|3188.90|1521.00|3254.55|159.44|0.00|910.65|3188.90|3348.34|4099.55|4258.99|1667.90| +2450829|32104|2450908|41830|1775341|6452|41163|24827|209738|49|39960|1|13|13|2|13562|245|729|10|78.36|106.56|14.91|916.50|149.10|783.60|1065.60|0.00|0.00|298.30|149.10|149.10|447.40|447.40|-634.50| +2450829|32104|2450851|41830|1775341|6452|41163|24827|209738|49|39960|1|63|3|5|478|94|729|13|31.90|78.79|14.18|839.93|184.34|414.70|1024.27|5.53|0.00|286.78|184.34|189.87|471.12|476.65|-230.36| +2450829|32104|2450872|41830|1775341|6452|41163|24827|209738|49|39960|1|33|15|1|277|90|729|9|77.94|107.55|81.73|232.38|735.57|701.46|967.95|66.20|0.00|435.51|735.57|801.77|1171.08|1237.28|34.11| +2450829|32104|2450909|41830|1775341|6452|41163|24827|209738|49|39960|1|14|19|1|6748|82|729|10|18.85|35.24|2.11|331.30|21.10|188.50|352.40|0.60|10.97|91.60|10.13|10.73|101.73|102.33|-178.37| +2450829|32104|2450906|41830|1775341|6452|41163|24827|209738|49|39960|1|86|13|2|14986|242|729|59|96.51|161.17|67.69|5515.32|3993.71|5694.09|9509.03|239.62|0.00|1996.56|3993.71|4233.33|5990.27|6229.89|-1700.38| +2450829|32104|2450851|41830|1775341|6452|41163|24827|209738|49|39960|1|4|10|3|7772|176|729|97|13.10|36.41|18.56|1731.45|1800.32|1270.70|3531.77|32.04|198.03|0.00|1602.29|1634.33|1602.29|1634.33|331.59| +2450829|17836|2450867|46340|1886425|3757|30242|46340|1886425|3757|30242|4|36|17|4|5236|3|730|66|32.80|56.41|28.20|1861.86|1861.20|2164.80|3723.06|18.61|0.00|1265.22|1861.20|1879.81|3126.42|3145.03|-303.60| +2450829|17836|2450844|46340|1886425|3757|30242|46340|1886425|3757|30242|4|11|8|2|8653|231|730|86|61.85|94.63|92.73|163.40|7974.78|5319.10|8138.18|159.49|0.00|2197.30|7974.78|8134.27|10172.08|10331.57|2655.68| +2450829|17836|2450879|46340|1886425|3757|30242|46340|1886425|3757|30242|4|67|19|4|2104|87|730|46|80.63|114.49|12.59|4687.40|579.14|3708.98|5266.54|11.58|0.00|210.22|579.14|590.72|789.36|800.94|-3129.84| +2450829|17836|2450906|46340|1886425|3757|30242|46340|1886425|3757|30242|4|52|20|4|6736|27|730|57|34.78|77.55|37.99|2254.92|2165.43|1982.46|4420.35|108.27|0.00|574.56|2165.43|2273.70|2739.99|2848.26|182.97| +2450829|17836|2450831|46340|1886425|3757|30242|46340|1886425|3757|30242|4|62|19|5|7426|62|730|33|67.75|183.60|179.92|121.44|5937.36|2235.75|6058.80|29.68|2968.68|726.99|2968.68|2998.36|3695.67|3725.35|732.93| +2450829|17836|2450905|46340|1886425|3757|30242|46340|1886425|3757|30242|4|50|13|2|8155|70|730|62|45.99|66.68|62.01|289.54|3844.62|2851.38|4134.16|346.01|0.00|330.46|3844.62|4190.63|4175.08|4521.09|993.24| +2450829|17836|2450860|46340|1886425|3757|30242|46340|1886425|3757|30242|4|51|4|4|6476|61|730|88|11.07|14.61|0.29|1260.16|25.52|974.16|1285.68|0.51|0.00|166.32|25.52|26.03|191.84|192.35|-948.64| +2450829|17836|2450862|46340|1886425|3757|30242|46340|1886425|3757|30242|4|72|17|5|7226|291|730|13|99.50|134.32|132.97|17.55|1728.61|1293.50|1746.16|6.91|1555.74|174.59|172.87|179.78|347.46|354.37|-1120.63| +2450829|17836|2450843|46340|1886425|3757|30242|46340|1886425|3757|30242|4|18|15|4|8200|10|730|77|3.98|4.53|3.03|115.50|233.31|306.46|348.81|13.99|0.00|174.02|233.31|247.30|407.33|421.32|-73.15| +2450829|17836|2450914|46340|1886425|3757|30242|46340|1886425|3757|30242|4|84|20|4|7874|252|730|25|26.34|51.36|24.13|680.75|603.25|658.50|1284.00|12.06|0.00|487.75|603.25|615.31|1091.00|1103.06|-55.25| +2450829|40208|2450905|8528|1577695|462|2829|8528|1577695|462|2829|2|43|8|2|2128|137|731|79|33.33|42.32|13.96|2240.44|1102.84|2633.07|3343.28|44.11|0.00|33.18|1102.84|1146.95|1136.02|1180.13|-1530.23| +2450829|40208|2450843|8528|1577695|462|2829|8528|1577695|462|2829|2|15|18|4|6103|201|731|35|59.56|164.38|14.79|5235.65|517.65|2084.60|5753.30|0.00|0.00|114.80|517.65|517.65|632.45|632.45|-1566.95| +2450829|40208|2450906|8528|1577695|462|2829|8528|1577695|462|2829|2|34|5|5|12268|237|731|19|10.07|13.79|2.89|207.10|54.91|191.33|262.01|0.00|0.00|68.02|54.91|54.91|122.93|122.93|-136.42| +2450829|40208|2450877|8528|1577695|462|2829|8528|1577695|462|2829|2|45|16|1|848|151|731|22|43.37|109.29|76.50|721.38|1683.00|954.14|2404.38|67.32|0.00|1129.92|1683.00|1750.32|2812.92|2880.24|728.86| +2450829|40208|2450831|8528|1577695|462|2829|8528|1577695|462|2829|2|14|8|5|11071|66|731|52|66.34|179.78|71.91|5609.24|3739.32|3449.68|9348.56|0.00|3739.32|3458.52|0.00|0.00|3458.52|3458.52|-3449.68| +2450829|40208|2450867|8528|1577695|462|2829|8528|1577695|462|2829|2|82|16|3|17056|129|731|77|63.27|129.70|19.45|8489.25|1497.65|4871.79|9986.90|134.78|0.00|1897.28|1497.65|1632.43|3394.93|3529.71|-3374.14| +2450829|40208|2450875|8528|1577695|462|2829|8528|1577695|462|2829|2|61|2|1|9586|92|731|45|3.07|7.03|0.49|294.30|22.05|138.15|316.35|0.22|0.00|148.50|22.05|22.27|170.55|170.77|-116.10| +2450829|40208|2450898|8528|1577695|462|2829|8528|1577695|462|2829|2|22|11|5|14528|77|731|26|73.94|141.96|136.28|147.68|3543.28|1922.44|3690.96|70.86|0.00|1107.08|3543.28|3614.14|4650.36|4721.22|1620.84| +2450829|40208|2450886|8528|1577695|462|2829|8528|1577695|462|2829|2|69|2|2|6988|48|731|7|38.75|99.97|93.97|42.00|657.79|271.25|699.79|19.73|0.00|48.93|657.79|677.52|706.72|726.45|386.54| +2450829|40208|2450897|8528|1577695|462|2829|8528|1577695|462|2829|2|59|14|3|6010|34|731|30|91.89|211.34|48.60|4882.20|1458.00|2756.70|6340.20|102.06|0.00|760.80|1458.00|1560.06|2218.80|2320.86|-1298.70| +2450829|40090|2450881|47928|1574934|6740|14546|47928|1574934|6740|14546|2|81|12|5|7489|272|732|35|88.61|245.44|83.44|5670.00|2920.40|3101.35|8590.40|82.35|175.22|1202.60|2745.18|2827.53|3947.78|4030.13|-356.17| +2450829|40090|2450899|47928|1574934|6740|14546|47928|1574934|6740|14546|2|40|14|3|14444|290|732|75|31.14|47.02|40.90|459.00|3067.50|2335.50|3526.50|0.00|0.00|0.00|3067.50|3067.50|3067.50|3067.50|732.00| +2450829|40090|2450863|47928|1574934|6740|14546|47928|1574934|6740|14546|2|11|17|5|518|10|732|4|53.67|103.04|31.94|284.40|127.76|214.68|412.16|11.49|0.00|61.80|127.76|139.25|189.56|201.05|-86.92| +2450829|40090|2450915|47928|1574934|6740|14546|47928|1574934|6740|14546|2|39|3|4|5551|124|732|6|84.13|105.16|45.21|359.70|271.26|504.78|630.96|18.98|0.00|113.52|271.26|290.24|384.78|403.76|-233.52| +2450829|40090|2450895|47928|1574934|6740|14546|47928|1574934|6740|14546|2|10|13|2|10405|258|732|41|1.86|4.91|4.02|36.49|164.82|76.26|201.31|1.64|0.00|100.45|164.82|166.46|265.27|266.91|88.56| +2450829|40090|2450842|47928|1574934|6740|14546|47928|1574934|6740|14546|2|68|5|3|7027|81|732|94|56.47|59.29|6.52|4960.38|612.88|5308.18|5573.26|6.12|0.00|1504.00|612.88|619.00|2116.88|2123.00|-4695.30| +2450829|40090|2450868|47928|1574934|6740|14546|47928|1574934|6740|14546|2|39|7|4|3109|225|732|84|97.81|258.21|123.94|11278.68|10410.96|8216.04|21689.64|0.00|0.00|8675.52|10410.96|10410.96|19086.48|19086.48|2194.92| +2450829|40090|2450869|47928|1574934|6740|14546|47928|1574934|6740|14546|2|72|7|1|6410|226|732|53|40.75|83.13|27.43|2952.10|1453.79|2159.75|4405.89|87.22|0.00|836.87|1453.79|1541.01|2290.66|2377.88|-705.96| +2450829|67944|2450871|37254|1637535|1369|10450|37254|1637535|1369|10450|2|88|1|2|7972|137|733|81|19.59|25.27|3.28|1781.19|265.68|1586.79|2046.87|23.91|0.00|1002.78|265.68|289.59|1268.46|1292.37|-1321.11| +2450829|67944|2450869|37254|1637535|1369|10450|37254|1637535|1369|10450|2|33|17|2|14443|124|733|49|49.26|101.47|72.04|1442.07|3529.96|2413.74|4972.03|0.00|0.00|596.33|3529.96|3529.96|4126.29|4126.29|1116.22| +2450829|67944|2450849|37254|1637535|1369|10450|37254|1637535|1369|10450|2|38|19|1|16747|191|733|33|84.01|246.98|46.92|6601.98|1548.36|2772.33|8150.34|46.45|0.00|1222.32|1548.36|1594.81|2770.68|2817.13|-1223.97| +2450829|67944|2450910|37254|1637535|1369|10450|37254|1637535|1369|10450|2|4|20|2|10240|79|733|32|58.81|106.44|39.38|2145.92|1260.16|1881.92|3406.08|25.20|0.00|1668.80|1260.16|1285.36|2928.96|2954.16|-621.76| +2450829|67944|2450875|37254|1637535|1369|10450|37254|1637535|1369|10450|2|89|8|3|14686|133|733|70|77.07|114.83|31.00|5868.10|2170.00|5394.90|8038.10|195.30|0.00|2813.30|2170.00|2365.30|4983.30|5178.60|-3224.90| +2450829|67944|2450918|37254|1637535|1369|10450|37254|1637535|1369|10450|2|15|11|2|11293|93|733|22|57.93|111.22|20.01|2006.62|440.22|1274.46|2446.84|39.61|0.00|391.38|440.22|479.83|831.60|871.21|-834.24| +2450829|67944|2450893|37254|1637535|1369|10450|37254|1637535|1369|10450|2|101|10|3|3578|132|733|92|48.15|114.59|68.75|4217.28|6325.00|4429.80|10542.28|0.00|0.00|2213.52|6325.00|6325.00|8538.52|8538.52|1895.20| +2450829|67944|2450868|37254|1637535|1369|10450|37254|1637535|1369|10450|2|103|7|4|13705|161|733|79|52.09|139.60|26.52|8933.32|2095.08|4115.11|11028.40|188.55|0.00|2646.50|2095.08|2283.63|4741.58|4930.13|-2020.03| +2450829|67944|2450912|37254|1637535|1369|10450|37254|1637535|1369|10450|2|81|8|4|9392|117|733|51|53.78|60.77|58.33|124.44|2974.83|2742.78|3099.27|0.00|0.00|340.68|2974.83|2974.83|3315.51|3315.51|232.05| +2450829|24448|2450893|97586|1315931|1476|8221|70882|824958|3303|39695|2|42|12|1|901|147|734|72|59.21|129.07|74.86|3903.12|5389.92|4263.12|9293.04|31.80|2209.86|3809.52|3180.06|3211.86|6989.58|7021.38|-1083.06| +2450829|24448|2450909|97586|1315931|1476|8221|70882|824958|3303|39695|2|12|4|4|16894|234|734|31|98.71|160.89|17.69|4439.20|548.39|3060.01|4987.59|16.45|0.00|199.33|548.39|564.84|747.72|764.17|-2511.62| +2450829|24448|2450840|97586|1315931|1476|8221|70882|824958|3303|39695|2|40|19|3|1402|112|734|55|98.29|224.10|2.24|12202.30|123.20|5405.95|12325.50|1.23|0.00|492.80|123.20|124.43|616.00|617.23|-5282.75| +2450829|24448|2450851|97586|1315931|1476|8221|70882|824958|3303|39695|2|88|2|5|10354|108|734|39|34.73|54.52|4.36|1956.24|170.04|1354.47|2126.28|8.50|0.00|21.06|170.04|178.54|191.10|199.60|-1184.43| +2450829|24448|2450855|97586|1315931|1476|8221|70882|824958|3303|39695|2|37|4|4|2716|231|734|70|14.10|21.43|2.57|1320.20|179.90|987.00|1500.10|0.70|156.51|569.80|23.39|24.09|593.19|593.89|-963.61| +2450829|24448|2450887|97586|1315931|1476|8221|70882|824958|3303|39695|2|54|18|5|6724|66|734|95|60.55|165.30|34.71|12406.05|3297.45|5752.25|15703.50|131.89|0.00|470.25|3297.45|3429.34|3767.70|3899.59|-2454.80| +2450829|24448|2450834|97586|1315931|1476|8221|70882|824958|3303|39695|2|103|10|3|9044|198|734|11|66.52|70.51|10.57|659.34|116.27|731.72|775.61|0.00|0.00|54.23|116.27|116.27|170.50|170.50|-615.45| +2450829|24448|2450893|97586|1315931|1476|8221|70882|824958|3303|39695|2|57|20|5|1324|293|734|66|28.96|29.24|11.98|1139.16|790.68|1911.36|1929.84|71.16|0.00|713.46|790.68|861.84|1504.14|1575.30|-1120.68| +2450829|24448|2450874|97586|1315931|1476|8221|70882|824958|3303|39695|2|41|13|3|5110|156|734|9|36.32|82.44|70.07|111.33|630.63|326.88|741.96|44.14|0.00|304.20|630.63|674.77|934.83|978.97|303.75| +2450829|24448|2450842|97586|1315931|1476|8221|70882|824958|3303|39695|2|69|13|4|10351|129|734|63|67.28|191.74|159.14|2053.80|10025.82|4238.64|12079.62|100.25|0.00|2898.63|10025.82|10126.07|12924.45|13024.70|5787.18| +2450829|24448|2450875|97586|1315931|1476|8221|70882|824958|3303|39695|2|58|3|5|11533|259|734|33|8.18|23.14|23.14|0.00|763.62|269.94|763.62|68.72|0.00|290.07|763.62|832.34|1053.69|1122.41|493.68| +2450829|50084|2450894|90582|725090|3598|17706|90582|725090|3598|17706|2|46|11|4|1522|8|735|100|54.45|102.36|33.77|6859.00|3377.00|5445.00|10236.00|33.77|0.00|2354.00|3377.00|3410.77|5731.00|5764.77|-2068.00| +2450829|50084|2450853|90582|725090|3598|17706|90582|725090|3598|17706|2|58|7|4|16202|97|735|57|45.72|73.60|5.15|3901.65|293.55|2606.04|4195.20|26.41|0.00|1216.38|293.55|319.96|1509.93|1536.34|-2312.49| +2450829|50084|2450837|90582|725090|3598|17706|90582|725090|3598|17706|2|22|3|2|3820|295|735|11|24.12|32.07|18.60|148.17|204.60|265.32|352.77|12.27|0.00|63.47|204.60|216.87|268.07|280.34|-60.72| +2450829|50084|2450891|90582|725090|3598|17706|90582|725090|3598|17706|2|13|3|3|9211|266|735|55|78.09|206.15|45.35|8844.00|2494.25|4294.95|11338.25|34.91|1995.40|5101.80|498.85|533.76|5600.65|5635.56|-3796.10| +||2450900|90582||||||3598||2||13||17918|18|735||34.02|||0.00|6939.40|2891.70|6939.40||0.00||||7286.20|7910.74|4047.70| +2450829|50084|2450879|90582|725090|3598|17706|90582|725090|3598|17706|2|27|17|3|11803|32|735|25|2.49|5.35|0.64|117.75|16.00|62.25|133.75|0.48|0.00|4.00|16.00|16.48|20.00|20.48|-46.25| +2450829|50084|2450858|90582|725090|3598|17706|90582|725090|3598|17706|2|68|5|1|6244|98|735|81|17.91|22.74|2.72|1621.62|220.32|1450.71|1841.94|19.82|0.00|331.29|220.32|240.14|551.61|571.43|-1230.39| +2450829|50084|2450903|90582|725090|3598|17706|90582|725090|3598|17706|2|18|13|1|7924|260|735|99|66.98|68.31|25.95|4193.64|2569.05|6631.02|6762.69|154.14|0.00|743.49|2569.05|2723.19|3312.54|3466.68|-4061.97| +2450829|50084|2450856|90582|725090|3598|17706|90582|725090|3598|17706|2|91|14|5|16486|278|735|13|28.82|46.97|2.81|574.08|36.53|374.66|610.61|1.82|0.00|12.09|36.53|38.35|48.62|50.44|-338.13| +2450829|50084|2450886|90582|725090|3598|17706|90582|725090|3598|17706|2|42|8|2|17480|107|735|73|22.76|49.61|37.20|905.93|2715.60|1661.48|3621.53|190.09|0.00|1339.55|2715.60|2905.69|4055.15|4245.24|1054.12| +2450829|50084|2450917|90582|725090|3598|17706|90582|725090|3598|17706|2|49|2|3|2876|215|735|90|54.42|120.81|119.60|108.90|10764.00|4897.80|10872.90|655.52|1399.32|2934.90|9364.68|10020.20|12299.58|12955.10|4466.88| +2450829|73611|2450853|92932|1208759|3433|17707|92932|1208759|3433|17707|2|67|15|3|11311|36|736|41|52.33|148.09|77.00|2914.69|3157.00|2145.53|6071.69|94.71|0.00|2125.03|3157.00|3251.71|5282.03|5376.74|1011.47| +2450829|73611|2450859|92932|1208759|3433|17707|92932|1208759|3433|17707|2|94|18|3|12950|14|736|17|24.87|64.91|45.43|331.16|772.31|422.79|1103.47|0.00|0.00|77.18|772.31|772.31|849.49|849.49|349.52| +2450829|73611|2450908|92932|1208759|3433|17707|92932|1208759|3433|17707|2|9|10|4|7657|30|736|92|56.65|105.93|5.29|9258.88|486.68|5211.80|9745.56|34.06|0.00|3020.36|486.68|520.74|3507.04|3541.10|-4725.12| +2450829|73611|2450907|92932|1208759|3433|17707|92932|1208759|3433|17707|2|65|8|2|5770|240|736|68|91.20|233.47|135.41|6668.08|9207.88|6201.60|15875.96|552.47|0.00|1111.12|9207.88|9760.35|10319.00|10871.47|3006.28| +2450829|73611|2450843|92932|1208759|3433|17707|92932|1208759|3433|17707|2|52|10|4|11510|85|736|91|26.42|53.89|12.39|3776.50|1127.49|2404.22|4903.99|22.54|0.00|489.58|1127.49|1150.03|1617.07|1639.61|-1276.73| +2450829|73611|2450834|92932|1208759|3433|17707|92932|1208759|3433|17707|2|68|3|2|6076|78|736|52|85.05|149.68|40.41|5682.04|2101.32|4422.60|7783.36|84.05|0.00|1478.36|2101.32|2185.37|3579.68|3663.73|-2321.28| +2450829|73611|2450858|92932|1208759|3433|17707|92932|1208759|3433|17707|2|81|15|5|15530|277|736|15|26.37|57.48|26.44|465.60|396.60|395.55|862.20|19.83|0.00|379.35|396.60|416.43|775.95|795.78|1.05| +2450829|73611|2450852|92932|1208759|3433|17707|92932|1208759|3433|17707|2|1|15|1|967|33|736|92|82.50|170.77|100.75|6441.84|9269.00|7590.00|15710.84|834.21|0.00|5812.56|9269.00|10103.21|15081.56|15915.77|1679.00| +2450829|73611|2450898|92932|1208759|3433|17707|92932|1208759|3433|17707|2|51|18|1|6862|7|736|2|15.36|17.97|16.17|3.60|32.34|30.72|35.94|1.29|0.00|7.90|32.34|33.63|40.24|41.53|1.62| +2450829|52965|2450873|49948|1054986|4824|25015|49948|1054986|4824|25015|4|92|4|1|3295|279|737|100|1.63|2.75|2.11|64.00|211.00|163.00|275.00|0.00|97.06|129.00|113.94|113.94|242.94|242.94|-49.06| +2450829|52965|2450859|49948|1054986|4824|25015|49948|1054986|4824|25015|4|47|12|3|8606|263|737|73|63.17|64.43|41.23|1693.60|3009.79|4611.41|4703.39|28.29|1595.18|564.29|1414.61|1442.90|1978.90|2007.19|-3196.80| +2450829|52965|2450853|49948|1054986|4824|25015|49948|1054986|4824|25015|4|65|8|2|8557|39|737|91|67.51|153.24|15.32|12550.72|1394.12|6143.41|13944.84|0.00|195.17|4322.50|1198.95|1198.95|5521.45|5521.45|-4944.46| +2450829|52965|2450863|49948|1054986|4824|25015|49948|1054986|4824|25015|4|105|16|1|9901|186|737|92|18.75|33.00|1.65|2884.20|151.80|1725.00|3036.00|12.14|0.00|1184.04|151.80|163.94|1335.84|1347.98|-1573.20| +|||49948|||25015|49948|||||50|17||12742||737|51|60.66|131.63||||3093.66|||0.00|3154.86|||8323.71|8530.46|2075.19| +2450829|52965|2450906|49948|1054986|4824|25015|49948|1054986|4824|25015|4|13|1|4|11288|226|737|11|75.94|188.33|145.01|476.52|1595.11|835.34|2071.63|0.00|0.00|538.56|1595.11|1595.11|2133.67|2133.67|759.77| +2450829|17208|2450897|7373|1742054|7135|169|7373|1742054|7135|169|2|43|11|1|13712|163|738|30|43.10|65.51|53.71|354.00|1611.30|1293.00|1965.30|48.33|0.00|176.70|1611.30|1659.63|1788.00|1836.33|318.30| +2450829|17208|2450888|7373|1742054|7135|169|7373|1742054|7135|169|2|91|11|3|190|42|738|61|28.28|74.09|34.82|2395.47|2124.02|1725.08|4519.49|16.99|424.80|722.85|1699.22|1716.21|2422.07|2439.06|-25.86| +2450829|17208|2450852|7373|1742054|7135|169|7373|1742054|7135|169|2|44|9|3|13466|16|738|73|38.43|50.72|46.15|333.61|3368.95|2805.39|3702.56|0.00|0.00|1517.67|3368.95|3368.95|4886.62|4886.62|563.56| +2450829|17208|2450889|7373|1742054|7135|169|7373|1742054|7135|169|2|99|16|3|1852|114|738|37|65.67|76.17|63.22|479.15|2339.14|2429.79|2818.29|210.52|0.00|169.09|2339.14|2549.66|2508.23|2718.75|-90.65| +2450829|72174|2450898|62410|1005146|7082|48218|62410|1005146|7082|48218|1|48|13|5|5570|294|739|71|10.89|23.41|2.34|1495.97|166.14|773.19|1662.11|11.62|0.00|498.42|166.14|177.76|664.56|676.18|-607.05| +2450829|72174|2450916|62410|1005146|7082|48218|62410|1005146|7082|48218|1|81|15|3|12829|294|739|33|98.08|174.58|50.62|4090.68|1670.46|3236.64|5761.14|31.57|1319.66|2016.30|350.80|382.37|2367.10|2398.67|-2885.84| +2450829|72174|2450841|62410|1005146|7082|48218|62410|1005146|7082|48218|1|32|18|1|3868|145|739|42|65.74|149.88|146.88|126.00|6168.96|2761.08|6294.96|17.27|4441.65|1699.32|1727.31|1744.58|3426.63|3443.90|-1033.77| +2450829|72174|2450851|62410|1005146|7082|48218|62410|1005146|7082|48218|1|22|19|4|4360|156|739|53|74.23|97.24|44.73|2783.03|2370.69|3934.19|5153.72|165.94|0.00|566.57|2370.69|2536.63|2937.26|3103.20|-1563.50| +2450829|72174|2450833|62410|1005146|7082|48218|62410|1005146|7082|48218|1|6|14|5|9524|215|739|13|27.87|58.80|45.86|168.22|596.18|362.31|764.40|6.67|429.24|336.31|166.94|173.61|503.25|509.92|-195.37| +2450829|72174|2450848|62410|1005146|7082|48218|62410|1005146|7082|48218|1|103|19|5|14353|44|739|92|56.64|147.83|128.61|1768.24|11832.12|5210.88|13600.36|276.87|2603.06|4215.44|9229.06|9505.93|13444.50|13721.37|4018.18| +2450829|46981|2450905|56461|901222|4383|16297|56461|901222|4383|16297|1|45|5|4|10837|44|740|66|61.48|169.68|13.57|10303.26|895.62|4057.68|11198.88|8.95|0.00|2015.64|895.62|904.57|2911.26|2920.21|-3162.06| +2450829|46981|2450835|56461|901222|4383|16297|56461|901222|4383|16297|1|77|10|1|15676|106|740|27|9.73|19.36|2.90|444.42|78.30|262.71|522.72|7.04|0.00|120.15|78.30|85.34|198.45|205.49|-184.41| +2450829|46981|2450845|56461|901222|4383|16297|56461|901222|4383|16297|1|56|14|1|9601|35|740|1|15.45|16.68|4.17|12.51|4.17|15.45|16.68|0.05|1.33|4.50|2.84|2.89|7.34|7.39|-12.61| +2450829|46981|2450879|56461|901222|4383|16297|56461|901222|4383|16297|1|61|11|1|11164|66|740|23|17.05|44.67|21.88|524.17|503.24|392.15|1027.41|20.12|0.00|421.13|503.24|523.36|924.37|944.49|111.09| +2450829|46981|2450831|56461|901222|4383|16297|56461|901222|4383|16297|1|77|11|3|13933|279|740|61|14.38|19.26|1.15|1104.71|70.15|877.18|1174.86|0.00|0.00|58.56|70.15|70.15|128.71|128.71|-807.03| +2450829|46981|2450890|56461|901222|4383|16297|56461|901222|4383|16297|1|56|9|4|8863|84|740|1|25.32|36.46|2.55|33.91|2.55|25.32|36.46|0.15|0.00|1.45|2.55|2.70|4.00|4.15|-22.77| +2450829|46981|2450860|56461|901222|4383|16297|56461|901222|4383|16297|1|4|10|4|14270|247|740|82|99.35|130.14|37.74|7576.80|3094.68|8146.70|10671.48|30.94|0.00|2454.26|3094.68|3125.62|5548.94|5579.88|-5052.02| +2450829|46981|2450836|56461|901222|4383|16297|56461|901222|4383|16297|1|6|16|2|14779|266|740|71|41.19|100.09|64.05|2558.84|4547.55|2924.49|7106.39|409.27|0.00|3410.84|4547.55|4956.82|7958.39|8367.66|1623.06| +2450829|46981|2450831|56461|901222|4383|16297|56461|901222|4383|16297|1|17|20|5|5869|270|740|13|90.51|149.34|10.45|1805.57|135.85|1176.63|1941.42|1.35|101.88|562.90|33.97|35.32|596.87|598.22|-1142.66| +2450829|46981|2450885|56461|901222|4383|16297|56461|901222|4383|16297|1|58|7|1|12188|24|740|55|17.14|45.24|30.76|796.40|1691.80|942.70|2488.20|135.34|0.00|870.65|1691.80|1827.14|2562.45|2697.79|749.10| +2450829|46981|2450890|56461|901222|4383|16297|56461|901222|4383|16297|1|83|9|4|13958|289|740|56|95.16|269.30|215.44|3016.16|12064.64|5328.96|15080.80|284.72|4946.50|2563.68|7118.14|7402.86|9681.82|9966.54|1789.18| +2450829|62211|2450865|35217|694629|3994|2206|35217|694629|3994|2206|2|3|8|4|16897|153|741|11|39.92|51.09|37.29|151.80|410.19|439.12|561.99|13.78|180.48|5.61|229.71|243.49|235.32|249.10|-209.41| +2450829|62211|2450840|35217|694629|3994|2206|35217|694629|3994|2206|2|77|20|4|61|105|741|11|24.39|55.36|12.17|475.09|133.87|268.29|608.96|1.92|37.48|121.77|96.39|98.31|218.16|220.08|-171.90| +2450829|62211|2450885|35217|694629|3994|2206|35217|694629|3994|2206|2|58|9|3|9788|183|741|50|14.32|32.93|8.23|1235.00|411.50|716.00|1646.50|16.46|0.00|806.50|411.50|427.96|1218.00|1234.46|-304.50| +2450829|62211|2450869|35217|694629|3994|2206|35217|694629|3994|2206|2|104|8|5|5140|39|741|37|73.23|213.09|12.78|7411.47|472.86|2709.51|7884.33|33.10|0.00|2680.65|472.86|505.96|3153.51|3186.61|-2236.65| +2450829|62211|2450876|35217|694629|3994|2206|35217|694629|3994|2206|2|65|10|5|16630|5|741|60|43.47|53.03|42.95|604.80|2577.00|2608.20|3181.80|14.43|2370.84|922.20|206.16|220.59|1128.36|1142.79|-2402.04| +2450829|62211|2450911|35217|694629|3994|2206|35217|694629|3994|2206|2|12|10|4|1741|79|741|29|16.95|40.68|31.73|259.55|920.17|491.55|1179.72|55.21|0.00|412.67|920.17|975.38|1332.84|1388.05|428.62| +2450829|62211|2450904|35217|694629|3994|2206|35217|694629|3994|2206|2|34|15|2|16513|45|741|11|48.55|79.62|31.84|525.58|350.24|534.05|875.82|0.00|255.67|61.27|94.57|94.57|155.84|155.84|-439.48| +2450829|62211|2450907|35217|694629|3994|2206|35217|694629|3994|2206|2|50|19|4|9355|176|741|86|68.87|95.72|26.80|5927.12|2304.80|5922.82|8231.92|161.33|0.00|4033.40|2304.80|2466.13|6338.20|6499.53|-3618.02| +2450829|62211|2450912|35217|694629|3994|2206|35217|694629|3994|2206|2|4|18|3|16892|210|741|93|26.49|33.64|29.93|345.03|2783.49|2463.57|3128.52|222.67|0.00|374.79|2783.49|3006.16|3158.28|3380.95|319.92| +2450829|62211|2450850|35217|694629|3994|2206|35217|694629|3994|2206|2|83|16|1|14479|176|741|77|31.58|53.68|34.89|1446.83|2686.53|2431.66|4133.36|8.05|2417.87|1901.13|268.66|276.71|2169.79|2177.84|-2163.00| +2450829|62211|2450859|35217|694629|3994|2206|35217|694629|3994|2206|2|79|17|1|12730|31|741|81|64.60|100.13|41.05|4785.48|3325.05|5232.60|8110.53|299.25|0.00|3811.86|3325.05|3624.30|7136.91|7436.16|-1907.55| +2450829|62211|2450845|35217|694629||2206|||3994|2206||77||5|15913||741|66||43.78|41.59|144.54|2744.94|||109.79|||||3235.98|3345.77|856.02| +2450829|62211|2450831|35217|694629|3994|2206|35217|694629|3994|2206|2|78|5|1|13060|13|741|84|15.63|29.54|28.94|50.40|2430.96|1312.92|2481.36|194.47|0.00|1116.36|2430.96|2625.43|3547.32|3741.79|1118.04| +2450829|62211|2450877|35217|694629|3994|2206|35217|694629|3994|2206|2|14|20|5|15292|30|741|20|55.78|109.32|41.54|1355.60|830.80|1115.60|2186.40|66.46|0.00|131.00|830.80|897.26|961.80|1028.26|-284.80| +2450829|33403|2450885|46363|1299147|4378|39209|46363|1299147|4378|39209|4|103|15|5|16759|265|742|64|77.27|197.03|33.49|10466.56|2143.36|4945.28|12609.92|192.90|0.00|2773.76|2143.36|2336.26|4917.12|5110.02|-2801.92| +2450829|33403|2450864|46363|1299147|4378|39209|46363|1299147|4378|39209|4|2|17|3|15244|227|742|60|44.12|116.03|68.45|2854.80|4107.00|2647.20|6961.80|0.00|0.00|556.80|4107.00|4107.00|4663.80|4663.80|1459.80| +2450829|33403|2450915|46363|1299147|4378|39209|46363|1299147|4378|39209|4|36|19|2|13675|173|742|43|94.73|251.03|135.55|4965.64|5828.65|4073.39|10794.29|116.57|0.00|4317.63|5828.65|5945.22|10146.28|10262.85|1755.26| +2450829|33403|2450914|46363|1299147|4378|39209|46363|1299147|4378|39209|4|62|12|4|6445|45|742|60|72.55|196.61|108.13|5308.80|6487.80|4353.00|11796.60|417.81|519.02|3420.60|5968.78|6386.59|9389.38|9807.19|1615.78| +2450829|33403|2450913|46363|1299147|4378|39209|46363|1299147|4378|39209|4|102|6|5|5809|160|742|61|75.57|200.26|46.05|9406.81|2809.05|4609.77|12215.86|140.45|0.00|1709.83|2809.05|2949.50|4518.88|4659.33|-1800.72| +2450829|33403|2450865|46363|1299147|4378|39209|46363|1299147|4378|39209|4|25|5|5|16321|182|742|46|65.69|189.18|1.89|8615.34|86.94|3021.74|8702.28|6.08|0.00|3654.70|86.94|93.02|3741.64|3747.72|-2934.80| +2450829|33403|2450860|46363|1299147|4378|39209|46363|1299147|4378|39209|4|86|16|3|9277|59|742|41|40.66|56.51|41.81|602.70|1714.21|1667.06|2316.91|117.25|411.41|694.95|1302.80|1420.05|1997.75|2115.00|-364.26| +2450829|33403|2450853|46363|1299147|4378|39209|46363|1299147|4378|39209|4|8|4|3|3992|211|742|73|34.73|77.44|28.65|3561.67|2091.45|2535.29|5653.12|188.23|0.00|1582.64|2091.45|2279.68|3674.09|3862.32|-443.84| +2450829|33403|2450868|46363|1299147|4378|39209|46363|1299147|4378|39209|4|13|3|1|11917|32|742|12|40.78|121.11|61.76|712.20|741.12|489.36|1453.32|21.93|466.90|145.32|274.22|296.15|419.54|441.47|-215.14| +2450829|69941|2450848|76796|1017669|3342|21368|76796|1017669|3342|21368|1|104|12|5|19|11|743|85|80.80|99.38|52.67|3970.35|4476.95|6868.00|8447.30|89.53|0.00|4138.65|4476.95|4566.48|8615.60|8705.13|-2391.05| +2450829|69941|2450851|76796|1017669|3342|21368|76796|1017669|3342|21368|1|51|7|4|10244|230|743|4|78.91|203.58|171.00|130.32|684.00|315.64|814.32|41.04|0.00|399.00|684.00|725.04|1083.00|1124.04|368.36| +2450829|69941|2450878|76796|1017669|3342|21368|76796|1017669|3342|21368|1|3|14|2|17234|18|743|13|86.43|249.78|107.40|1850.94|1396.20|1123.59|3247.14|41.88|0.00|0.00|1396.20|1438.08|1396.20|1438.08|272.61| +2450829|69941|2450856|76796|1017669|3342|21368|76796|1017669|3342|21368|1|74|15|4|5908|270|743|45|13.58|36.93|20.68|731.25|930.60|611.10|1661.85|46.53|0.00|697.95|930.60|977.13|1628.55|1675.08|319.50| +2450829|69941|2450861|76796|1017669|3342|21368|76796|1017669|3342|21368|1|34|13|1|5269|2|743|45|48.30|89.35|37.52|2332.35|1688.40|2173.50|4020.75|50.65|0.00|683.10|1688.40|1739.05|2371.50|2422.15|-485.10| +2450829|69941|2450846|76796|1017669|3342|21368|76796|1017669|3342|21368|1|62|13|2|2258|117|743|58|6.00|14.40|1.44|751.68|83.52|348.00|835.20|0.00|0.00|141.52|83.52|83.52|225.04|225.04|-264.48| +2450829|69941|2450855|76796|1017669|3342|21368|76796|1017669|3342|21368|1|5|17|2|14866|295|743|20|1.24|3.07|0.18|57.80|3.60|24.80|61.40|0.07|0.00|30.60|3.60|3.67|34.20|34.27|-21.20| +2450829|67357|2450837|29493|38684|5828|38972|10588|1324060|1251|40395|4|32|20|3|13012|166|744|72|87.00|107.88|93.85|1010.16|6757.20|6264.00|7767.36|405.43|0.00|3106.80|6757.20|7162.63|9864.00|10269.43|493.20| +2450829|67357|2450891|29493|38684|5828|38972|10588|1324060|1251|40395|4|43|11|4|9920|183|744|64|35.93|90.54|7.24|5331.20|463.36|2299.52|5794.56|4.63|0.00|2607.36|463.36|467.99|3070.72|3075.35|-1836.16| +2450829|67357|2450886|29493|38684|5828|38972|10588|1324060|1251|40395|4|99|19|2|13064|165|744|67|94.15|134.63|129.24|361.13|8659.08|6308.05|9020.21|519.54|0.00|0.00|8659.08|9178.62|8659.08|9178.62|2351.03| +2450829|67357|2450832|29493|38684|5828|38972|10588|1324060|1251|40395|4|89|2|3|4466|117|744|99|93.14|186.28|7.45|17704.17|737.55|9220.86|18441.72|14.75|0.00|2766.06|737.55|752.30|3503.61|3518.36|-8483.31| +2450829|67357|2450832|29493|38684|5828|38972|10588|1324060|1251|40395|4|16|20|1|15448|139|744|90|9.52|16.27|10.90|483.30|981.00|856.80|1464.30|88.29|0.00|365.40|981.00|1069.29|1346.40|1434.69|124.20| +||2450875||38684|||10588||1251||4|4|20|4|1705|155|744|39||44.85||315.12||||114.72||804.57|1434.03||2238.60|2353.32|-141.96| +2450829|67357|2450899|29493|38684|5828|38972|10588|1324060|1251|40395|4|108|4|4|13316|99|744|77|80.27|222.34|195.65|2055.13|15065.05|6180.79|17120.18|1054.55|0.00|6847.61|15065.05|16119.60|21912.66|22967.21|8884.26| +2450829|49534|2450876|43076|1439791|4506|11174|43076|1439791|4506|11174|1|104|17|1|15499|82|745|82|73.82|200.79|8.03|15806.32|658.46|6053.24|16464.78|59.26|0.00|164.00|658.46|717.72|822.46|881.72|-5394.78| +2450829|49534|2450903|43076|1439791|4506|11174|43076|1439791|4506|11174|1|14|6|1|10474|36|745|69|92.68|197.40|94.75|7082.85|6537.75|6394.92|13620.60|0.00|0.00|6810.30|6537.75|6537.75|13348.05|13348.05|142.83| +2450829|49534|2450917|43076|1439791|4506|11174|43076|1439791|4506|11174|1|60|4|4|4180|256|745|5|86.33|246.90|153.07|469.15|765.35|431.65|1234.50|53.57|0.00|580.20|765.35|818.92|1345.55|1399.12|333.70| +2450829|49534|2450882|43076|1439791|4506|11174|43076|1439791|4506|11174|1|19|6|5|1454|216|745|52|73.80|99.63|23.91|3937.44|1243.32|3837.60|5180.76|23.00|783.29|362.44|460.03|483.03|822.47|845.47|-3377.57| +2450829|49534|2450880|43076|1439791|4506|11174|43076|1439791|4506|11174|1|53|13|5|13654|181|745|18|5.20|11.49|9.76|31.14|175.68|93.60|206.82|6.32|96.62|70.20|79.06|85.38|149.26|155.58|-14.54| +2450829|49534|2450870|43076|1439791|4506|11174|43076|1439791|4506|11174|1|34|11|4|2998|123|745|13|94.84|97.68|24.42|952.38|317.46|1232.92|1269.84|3.17|0.00|418.99|317.46|320.63|736.45|739.62|-915.46| +2450829|49534|2450874|43076|1439791|4506|11174|43076|1439791|4506|11174|1|60|19|3|11356|123|745|25|64.63|93.06|52.11|1023.75|1302.75|1615.75|2326.50|39.08|0.00|69.75|1302.75|1341.83|1372.50|1411.58|-313.00| +2450829|49534|2450861|43076|1439791|4506|11174|43076|1439791|4506|11174|1|87|13|2|17725|173|745|95|63.30|65.83|58.58|688.75|5565.10|6013.50|6253.85|0.00|0.00|1750.85|5565.10|5565.10|7315.95|7315.95|-448.40| +2450829|68828|2450877|75562|925970|25|18080|75562|925970|25|18080|4|84|14|3|15367|219|746|90|43.97|87.06|54.84|2899.80|4935.60|3957.30|7835.40|98.71|0.00|469.80|4935.60|5034.31|5405.40|5504.11|978.30| +2450829|68828|2450902|75562|925970|25|18080|75562|925970|25|18080|4|39|17|3|3902|212|746|84|13.04|27.38|19.71|644.28|1655.64|1095.36|2299.92|66.22|0.00|827.40|1655.64|1721.86|2483.04|2549.26|560.28| +2450829|68828|2450877|75562|925970|25|18080|75562|925970|25|18080|4|16|19|2|10178|157|746|7|82.60|163.54|35.97|892.99|251.79|578.20|1144.78|20.14|0.00|11.41|251.79|271.93|263.20|283.34|-326.41| +2450829|68828|2450850|75562|925970|25|18080|75562|925970|25|18080|4|102|1|3|11098|104|746|65|29.38|64.63|21.97|2772.90|1428.05|1909.70|4200.95|71.40|0.00|1175.85|1428.05|1499.45|2603.90|2675.30|-481.65| +2450829|68828|2450834|75562|925970|25|18080|75562|925970|25|18080|4|97|15|1|15097|273|746|72|90.43|156.44|62.57|6758.64|4505.04|6510.96|11263.68|135.15|0.00|2590.56|4505.04|4640.19|7095.60|7230.75|-2005.92| +2450829|68828|2450914|75562|925970|25|18080|75562|925970|25|18080|4|58|11|4|3781|222|746|51|31.12|85.26|3.41|4174.35|173.91|1587.12|4348.26|3.47|0.00|86.70|173.91|177.38|260.61|264.08|-1413.21| +2450829|40641|2450899|96263|1610005|2947|10689|96263|1610005|2947|10689|2|90|3|4|9596|64|747|23|82.52|137.80|81.30|1299.50|1869.90|1897.96|3169.40|93.49|0.00|94.99|1869.90|1963.39|1964.89|2058.38|-28.06| +2450829|40641|2450871|96263|1610005|2947|10689|96263|1610005|2947|10689|2|55|12|3|14191|212|747|93|34.07|47.69|41.01|621.24|3813.93|3168.51|4435.17|305.11|0.00|1951.14|3813.93|4119.04|5765.07|6070.18|645.42| +2450829|40641|2450913|96263|1610005|2947|10689|96263|1610005|2947|10689|2|1|6|4|1249|18|747|80|56.65|163.15|127.25|2872.00|10180.00|4532.00|13052.00|916.20|0.00|1827.20|10180.00|11096.20|12007.20|12923.40|5648.00| +2450829|40641|2450861|96263|1610005|2947|10689|96263|1610005|2947|10689|2|36|14|5|2762|147|747|52|1.96|2.17|0.73|74.88|37.96|101.92|112.84|0.63|29.98|15.60|7.98|8.61|23.58|24.21|-93.94| +2450829|40641|2450842|96263|1610005|2947|10689|96263|1610005|2947|10689|2|95|19|3|7192|61|747|14|96.61|128.49|107.93|287.84|1511.02|1352.54|1798.86|135.99|0.00|341.74|1511.02|1647.01|1852.76|1988.75|158.48| +2450829|40641|2450832|96263|1610005|2947|10689|96263|1610005|2947|10689|2|7|8|5|6182|75|747|27|16.25|23.72|11.62|326.70|313.74|438.75|640.44|6.27|0.00|0.00|313.74|320.01|313.74|320.01|-125.01| +2450829|40641|2450838|96263|1610005|2947|10689|96263|1610005|2947|10689|2|22|8|3|5942|254|747|25|36.12|88.85|3.55|2132.50|88.75|903.00|2221.25|5.32|0.00|777.25|88.75|94.07|866.00|871.32|-814.25| +2450829|40641|2450850|96263|1610005|2947|10689|96263|1610005|2947|10689|2|35|18|3|17572|193|747|48|1.73|3.52|0.35|152.16|16.80|83.04|168.96|0.84|0.00|57.12|16.80|17.64|73.92|74.76|-66.24| +2450829|40641|2450894|96263|1610005|2947|10689|96263|1610005|2947|10689|2|8|20|4|97|214|747|99|59.93|167.20|108.68|5793.48|10759.32|5933.07|16552.80|968.33|0.00|7117.11|10759.32|11727.65|17876.43|18844.76|4826.25| +2450829|40641|2450881|96263|1610005|2947|10689|96263|1610005|2947|10689|2|88|14|4|16114|2|747|86|82.76|198.62|141.02|4953.60|12127.72|7117.36|17081.32|485.10|0.00|8198.38|12127.72|12612.82|20326.10|20811.20|5010.36| +2450829|40641|2450853|96263|1610005|2947|10689|96263|1610005|2947|10689|2|96|11|5|3001|250|747|19|56.03|66.67|0.00|1266.73|0.00|1064.57|1266.73|0.00|0.00|63.27|0.00|0.00|63.27|63.27|-1064.57| +2450829|40641|2450838|96263|1610005|2947|10689|96263|1610005|2947|10689|2|108|7|1|13693|164|747|84|69.78|192.59|78.96|9544.92|6632.64|5861.52|16177.56|265.30|0.00|5014.80|6632.64|6897.94|11647.44|11912.74|771.12| +2450829|40641|2450848|96263|1610005|2947|10689|96263|1610005|2947|10689|2|98|16|5|10621|255|747|23|75.73|124.19|12.41|2570.94|285.43|1741.79|2856.37|8.56|0.00|571.09|285.43|293.99|856.52|865.08|-1456.36| +2450829|40641|2450881|96263|1610005|2947|10689|96263|1610005|2947|10689|2|90|20|4|613|263|747|51|92.53|139.72|27.94|5700.78|1424.94|4719.03|7125.72|28.49|0.00|1923.72|1424.94|1453.43|3348.66|3377.15|-3294.09| +2450829|35456|2450839|80688|332763|1191|32066|80688|332763|1191|32066|4|92|19|4|8908|122|748|43|49.40|75.58|47.61|1202.71|2047.23|2124.20|3249.94|4.50|1596.83|1397.07|450.40|454.90|1847.47|1851.97|-1673.80| +2450829|35456|2450834|80688|332763|1191|32066|80688|332763|1191|32066|4|54|14|3|385|40|748|38|65.56|173.73|46.90|4819.54|1782.20|2491.28|6601.74|124.75|0.00|263.72|1782.20|1906.95|2045.92|2170.67|-709.08| +2450829|35456|2450852|80688|332763|1191|32066|80688|332763|1191|32066|4|108|14|5|800|67|748|49|71.02|92.32|0.92|4478.60|45.08|3479.98|4523.68|0.38|25.69|588.00|19.39|19.77|607.39|607.77|-3460.59| +2450829|35456|2450893|80688|332763|1191|32066|80688|332763|1191|32066|4|75|6|3|7762|70|748|61|70.64|119.38|68.04|3131.74|4150.44|4309.04|7282.18|207.52|0.00|3349.51|4150.44|4357.96|7499.95|7707.47|-158.60| +2450829|35456|2450863|80688|332763|1191|32066|80688|332763|1191|32066|4|11|8|5|12838|223|748|36|94.48|205.96|166.82|1409.04|6005.52|3401.28|7414.56|360.33|0.00|3188.16|6005.52|6365.85|9193.68|9554.01|2604.24| +2450829|35456|2450846|80688|332763|1191|32066|80688|332763|1191|32066|4|65|8|2|601|287|748|77|66.81|167.02|26.72|10803.10|2057.44|5144.37|12860.54|164.59|0.00|2057.44|2057.44|2222.03|4114.88|4279.47|-3086.93| +2450829|35456|2450903|80688|332763|1191|32066|80688|332763|1191|32066|4|14|20|1|2965|13|748|78|96.71|123.78|35.89|6855.42|2799.42|7543.38|9654.84|251.94|0.00|1350.96|2799.42|3051.36|4150.38|4402.32|-4743.96| +2450829|35456|2450903|80688|332763|1191|32066|80688|332763|1191|32066|4|92|12|5|4228|131|748|70|34.54|42.82|26.12|1169.00|1828.40|2417.80|2997.40|127.98|0.00|1408.40|1828.40|1956.38|3236.80|3364.78|-589.40| +2450829|35456|2450900|80688|332763|1191|32066|80688|332763|1191|32066|4|88|11|3|679|107|748|54|55.53|121.61|13.37|5844.96|721.98|2998.62|6566.94|36.09|0.00|524.88|721.98|758.07|1246.86|1282.95|-2276.64| +2450829|35456|2450843|80688|332763|1191|32066|80688|332763|1191|32066|4|52|6|5|9662|85|748|79|29.85|36.41|25.85|834.24|2042.15|2358.15|2876.39|132.94|142.95|1380.13|1899.20|2032.14|3279.33|3412.27|-458.95| +2450829|35456|2450894|80688|332763|1191|32066|80688|332763|1191|32066|4|28|20|1|496|181|748|76|22.99|30.11|7.52|1716.84|571.52|1747.24|2288.36|28.57|0.00|1121.00|571.52|600.09|1692.52|1721.09|-1175.72| +2450829|35456|2450902|80688|332763|1191|32066|80688|332763|1191|32066|4|84|6|5|229|52|748|72|45.63|88.97|32.02|4100.40|2305.44|3285.36|6405.84|0.00|0.00|191.52|2305.44|2305.44|2496.96|2496.96|-979.92| +2450829|35456|2450858|80688|332763|1191|32066|80688|332763|1191|32066|4|3|5|3|9608|7|748|15|92.73|140.94|109.93|465.15|1648.95|1390.95|2114.10|65.95|0.00|443.85|1648.95|1714.90|2092.80|2158.75|258.00| +2450829|62556|2450853|49361|81440|5966|26845|49361|81440|5966|26845|4|78|18|4|1945|41|749|88|50.38|141.06|69.11|6331.60|6081.68|4433.44|12413.28|57.16|3223.29|5709.44|2858.39|2915.55|8567.83|8624.99|-1575.05| +2450829|62556|2450889|49361|81440|5966|26845|49361|81440|5966|26845|4|106|14|4|11797|7|749|88|37.86|73.82|24.36|4352.48|2143.68|3331.68|6496.16|150.05|0.00|1364.00|2143.68|2293.73|3507.68|3657.73|-1188.00| +2450829|62556|2450839|49361|81440|5966|26845|49361|81440|5966|26845|4|46|18|2|8978|175|749|72|71.04|130.00|44.20|6177.60|3182.40|5114.88|9360.00|254.59|0.00|2433.60|3182.40|3436.99|5616.00|5870.59|-1932.48| +2450829|62556|2450886|49361|81440|5966|26845|49361|81440|5966|26845|4|94|15|3|9404|104|749|96|51.97|55.08|15.97|3754.56|1533.12|4989.12|5287.68|107.31|0.00|1427.52|1533.12|1640.43|2960.64|3067.95|-3456.00| +2450829|62556|2450851|49361|81440|5966|26845|49361|81440|5966|26845|4|69|11|2|11233|89|749|91|35.06|42.42|12.72|2702.70|1157.52|3190.46|3860.22|0.00|335.68|1002.82|821.84|821.84|1824.66|1824.66|-2368.62| +2450829|62556|2450853|49361|81440|5966|26845|49361|81440|5966|26845|4|53|1|4|8804|192|749|8|47.12|97.06|73.76|186.40|590.08|376.96|776.48|23.60|0.00|217.36|590.08|613.68|807.44|831.04|213.12| +2450829|62556|2450838|49361|81440|5966|26845|49361|81440|5966|26845|4|35|4|5|5344|145|749|46|21.19|24.15|15.69|389.16|721.74|974.74|1110.90|43.30|0.00|99.82|721.74|765.04|821.56|864.86|-253.00| +2450829|64667|2450860|1907|929594|2881|37368|1907|929594|2881|37368|4|8|10|3|10153|209|750|14|43.71|76.92|19.99|797.02|279.86|611.94|1076.88|13.99|0.00|301.42|279.86|293.85|581.28|595.27|-332.08| +2450829|64667|2450910|1907|929594|2881|37368|1907|929594|2881|37368|4|74|6|4|14032|255|750|47|98.49|230.46|64.52|7799.18|3032.44|4629.03|10831.62|121.29|0.00|1624.32|3032.44|3153.73|4656.76|4778.05|-1596.59| +2450829|64667|2450858|1907|929594|2881|37368|1907|929594|2881|37368|4|55|14|4|2140|12|750|54|56.89|83.62|22.57|3296.70|1218.78|3072.06|4515.48|109.69|0.00|1851.12|1218.78|1328.47|3069.90|3179.59|-1853.28| +2450829|64667|2450891|1907|929594|2881|37368|1907|929594|2881|37368|4|39|3|1|4129|130|750|88|28.50|33.06|9.58|2066.24|843.04|2508.00|2909.28|8.43|0.00|814.00|843.04|851.47|1657.04|1665.47|-1664.96| +2450829|64667|2450858|1907|929594|2881|37368|1907|929594|2881|37368|4|42|11|5|9067|111|750|23|60.91|119.99|19.19|2318.40|441.37|1400.93|2759.77|30.89|0.00|965.77|441.37|472.26|1407.14|1438.03|-959.56| +2450829|64667|2450894|1907|929594|2881|37368|1907|929594|2881|37368|4|88|16|1|13856|28|750|89|51.02|101.52|40.60|5421.88|3613.40|4540.78|9035.28|0.00|0.00|4246.19|3613.40|3613.40|7859.59|7859.59|-927.38| +2450829|64667|2450875|1907|929594|2881|37368|1907|929594|2881|37368|4|75|10|4|2269|5|750|13|31.27|79.73|7.97|932.88|103.61|406.51|1036.49|6.02|17.61|269.36|86.00|92.02|355.36|361.38|-320.51| +2450829|64667|2450846|1907|929594|2881|37368|1907|929594|2881|37368|4|47|7|3|8974|276|750|100|19.09|44.47|31.12|1335.00|3112.00|1909.00|4447.00|0.00|0.00|1867.00|3112.00|3112.00|4979.00|4979.00|1203.00| +2450829|64667|2450883|1907|929594|2881|37368|1907|929594|2881|37368|4|106|8|4|9176|222|750|70|69.74|153.42|49.09|7303.10|3436.30|4881.80|10739.40|68.72|0.00|1932.70|3436.30|3505.02|5369.00|5437.72|-1445.50| +2450829|64667|2450846|1907|929594|2881|37368|1907|929594|2881|37368|4|27|4|2|10141|50|750|8|47.20|89.20|9.81|635.12|78.48|377.60|713.60|4.70|0.00|356.80|78.48|83.18|435.28|439.98|-299.12| +2450829|31257|2450841|66225|501378|531|4484|66225|501378|531|4484|1|102|7|2|9482|162|751|8|61.62|154.66|41.75|903.28|334.00|492.96|1237.28|26.72|0.00|457.76|334.00|360.72|791.76|818.48|-158.96| +2450829|31257|2450910|66225|501378|531|4484|66225|501378|531|4484|1|40|16|1|15535|78|751|99|55.03|84.74|33.89|5034.15|3355.11|5447.97|8389.26|14.09|2885.39|3019.50|469.72|483.81|3489.22|3503.31|-4978.25| +2450829|31257|2450913|66225|501378|531|4484|66225|501378|531|4484|1|101|6|5|9943|265|751|74|79.35|133.30|106.64|1972.84|7891.36|5871.90|9864.20|315.65|0.00|3748.10|7891.36|8207.01|11639.46|11955.11|2019.46| +2450829|31257|2450917|66225|501378|531|4484|66225|501378|531|4484|1|40|17|3|5612|150|751|37|36.89|98.12|85.36|472.12|3158.32|1364.93|3630.44|31.58|0.00|907.61|3158.32|3189.90|4065.93|4097.51|1793.39| +2450829|31257|2450833|66225|501378|531|4484|66225|501378|531|4484|1|24|18|5|15013|106|751|74|67.63|85.89|80.73|381.84|5974.02|5004.62|6355.86|59.74|0.00|508.38|5974.02|6033.76|6482.40|6542.14|969.40| +2450829|71075|2450869|10008|457070|4932|11194|10008|457070|4932|11194|2|80|9|3|13456|129|752|15|32.58|71.67|62.35|139.80|935.25|488.70|1075.05|28.05|0.00|53.70|935.25|963.30|988.95|1017.00|446.55| +2450829|71075|2450884|10008|457070|4932|11194|10008|457070|4932|11194|2|11|12|3|3080|32|752|38|97.82|253.35|98.80|5872.90|3754.40|3717.16|9627.30|75.08|0.00|1925.46|3754.40|3829.48|5679.86|5754.94|37.24| +2450829|71075|2450868|10008|457070|4932|11194|10008|457070|4932|11194|2|56|3|4|4244|215|752|47|23.99|45.82|37.11|409.37|1744.17|1127.53|2153.54|156.97|0.00|236.88|1744.17|1901.14|1981.05|2138.02|616.64| +2450829|71075|2450907|10008|457070|4932|11194|10008|457070|4932|11194|2|95|14|4|3116|261|752|35|35.50|74.90|21.72|1861.30|760.20|1242.50|2621.50|15.96|228.06|104.65|532.14|548.10|636.79|652.75|-710.36| +2450829|71075|2450910|10008|457070|4932|11194|10008|457070|4932|11194|2|60|11|3|9223|172|752|4|10.65|31.95|25.87|24.32|103.48|42.60|127.80|6.08|2.06|6.36|101.42|107.50|107.78|113.86|58.82| +2450829|71075|2450866|10008|457070|4932|11194|10008|457070|4932|11194|2|78|20|1|3406|193|752|41|14.70|41.45|23.21|747.84|951.61|602.70|1699.45|66.61|0.00|560.47|951.61|1018.22|1512.08|1578.69|348.91| +2450829|71075|2450887|10008|457070|4932|11194|10008|457070|4932|11194|2|68|4|3|11155|75|752|39|92.49|246.02|184.51|2398.89|7195.89|3607.11|9594.78|437.51|1727.01|3357.90|5468.88|5906.39|8826.78|9264.29|1861.77| +2450829|71075|2450897|10008|457070|4932|11194|10008|457070|4932|11194|2|4|20|4|8030|122|752|70|93.85|233.68|161.23|5071.50|11286.10|6569.50|16357.60|240.39|3272.96|7851.20|8013.14|8253.53|15864.34|16104.73|1443.64| +2450829|71075|2450904|10008|457070|4932|11194|10008|457070|4932|11194|2|94|8|3|15964|148|752|31|59.22|146.86|27.90|3687.76|864.90|1835.82|4552.66|51.89|0.00|546.22|864.90|916.79|1411.12|1463.01|-970.92| +2450829|19355|2450847|16103|270419|3488|1004|16103|270419|3488|1004|1|1|19|2|16870|110|753|25|29.24|44.15|25.60|463.75|640.00|731.00|1103.75|44.80|0.00|529.75|640.00|684.80|1169.75|1214.55|-91.00| +2450829|19355|2450875|16103|270419|3488|1004|16103|270419|3488|1004|1|32|3|5|1255|289|753|42|46.39|71.90|64.71|301.98|2717.82|1948.38|3019.80|135.89|0.00|90.30|2717.82|2853.71|2808.12|2944.01|769.44| +2450829|19355|2450853|16103|270419|3488|1004|16103|270419|3488|1004|1|82|1|4|9752|183|753|19|21.53|21.74|20.87|16.53|396.53|409.07|413.06|1.11|341.01|45.41|55.52|56.63|100.93|102.04|-353.55| +2450829|19355|2450905|16103|270419|3488|1004|16103|270419|3488|1004|1|81|7|3|3272|90|753|23|46.54|116.81|107.46|215.05|2471.58|1070.42|2686.63|47.45|1878.40|1262.70|593.18|640.63|1855.88|1903.33|-477.24| +2450829|19355|2450917|16103|270419|3488|1004|16103|270419|3488|1004|1|75|13|2|4988|40|753|22|56.42|85.75|43.73|924.44|962.06|1241.24|1886.50|77.92|96.20|528.22|865.86|943.78|1394.08|1472.00|-375.38| +2450829|19355|2450899|16103|270419|3488|1004|16103|270419|3488|1004|1|82|3|5|7843|122|753|9|86.60|197.44|140.18|515.34|1261.62|779.40|1776.96|42.89|189.24|319.77|1072.38|1115.27|1392.15|1435.04|292.98| +2450829|19355|2450861|16103|270419|3488|1004|16103|270419|3488|1004|1|104|2|5|536|161|753|24|47.06|48.47|13.57|837.60|325.68|1129.44|1163.28|29.31|0.00|58.08|325.68|354.99|383.76|413.07|-803.76| +2450829|19355|2450849|16103|270419|3488|1004|16103|270419|3488|1004|1|51|8|1|10466|292|753|53|62.27|144.46|76.56|3598.70|4057.68|3300.31|7656.38|284.03|0.00|3675.02|4057.68|4341.71|7732.70|8016.73|757.37| +2450829|36536|2450843|8426|92656|2133|10010|8426|92656|2133|10010|2|13|14|3|3314|5|754|47|53.18|69.66|1.39|3208.69|65.33|2499.46|3274.02|1.95|0.00|228.89|65.33|67.28|294.22|296.17|-2434.13| +2450829|36536|2450844|8426|92656|2133|10010|8426|92656|2133|10010|2|73|12|1|12172|114|754|5|46.01|82.35|4.94|387.05|24.70|230.05|411.75|1.97|0.00|189.40|24.70|26.67|214.10|216.07|-205.35| +2450829|36536|2450887|8426|92656|2133|10010|8426|92656|2133|10010|2|91|18|2|5506|232|754|24|68.45|111.57|54.66|1365.84|1311.84|1642.80|2677.68|104.94|0.00|642.48|1311.84|1416.78|1954.32|2059.26|-330.96| +2450829|36536|2450911|8426|92656|2133|10010|8426|92656|2133|10010|2|46|9|3|8986|284|754|10|74.40|207.57|91.33|1162.40|913.30|744.00|2075.70|18.26|0.00|20.70|913.30|931.56|934.00|952.26|169.30| +2450829|36536|2450858|8426|92656|2133|10010|8426|92656|2133|10010|2|94|1|3|11924|157|754|66|18.21|54.63|0.00|3605.58|0.00|1201.86|3605.58|0.00|0.00|900.90|0.00|0.00|900.90|900.90|-1201.86| +2450829|15082|2450856|99926|1180984|1242|22653|99926|1180984|1242|22653|1|101|1|4|614|107|755|92|82.00|241.90|133.04|10015.12|12239.68|7544.00|22254.80|979.17|0.00|9346.28|12239.68|13218.85|21585.96|22565.13|4695.68| +2450829|15082|2450856|99926|1180984|1242|22653|99926|1180984|1242|22653|1|21|1|3|15721|139|755|2|92.74|155.80|82.57|146.46|165.14|185.48|311.60|11.55|0.00|62.32|165.14|176.69|227.46|239.01|-20.34| +2450829|15082|2450868|99926|1180984|1242|22653|99926|1180984|1242|22653|1|79|11|1|14312|151|755|48|30.91|69.23|69.23|0.00|3323.04|1483.68|3323.04|115.64|1395.67|99.36|1927.37|2043.01|2026.73|2142.37|443.69| +2450829|15082|2450904|99926|1180984|1242|22653|99926|1180984|1242|22653|1|57|12|2|2660|16|755|74|26.10|46.45|26.94|1443.74|1993.56|1931.40|3437.30|19.93|0.00|996.78|1993.56|2013.49|2990.34|3010.27|62.16| +2450829|15082|2450845|99926|1180984|1242|22653|99926|1180984|1242|22653|1|58|12|2|16087|243|755|32|29.94|33.23|9.96|744.64|318.72|958.08|1063.36|25.49|0.00|127.36|318.72|344.21|446.08|471.57|-639.36| +2450829|15082|2450890|99926|1180984|1242|22653|99926|1180984|1242|22653|1|96|16|5|5803|18|755|51|61.60|93.01|57.66|1802.85|2940.66|3141.60|4743.51|147.03|0.00|1612.62|2940.66|3087.69|4553.28|4700.31|-200.94| +2450829|15082|2450836|99926|1180984|1242|22653|99926|1180984|1242|22653|1|87|6|5|6877|231|755|13|8.31|15.87|7.61|107.38|98.93|108.03|206.31|0.00|50.45|94.90|48.48|48.48|143.38|143.38|-59.55| +2450829|15082|2450871|99926|1180984|1242|22653|99926|1180984|1242|22653|1|5|6|5|16670|208|755|100|10.86|12.05|8.67|338.00|867.00|1086.00|1205.00|17.34|0.00|24.00|867.00|884.34|891.00|908.34|-219.00| +2450829|15082|2450900|99926|1180984|1242|22653|99926|1180984|1242|22653|1|8|4|2|14716|101|755|52|28.80|80.64|34.67|2390.44|1802.84|1497.60|4193.28|34.25|90.14|1299.48|1712.70|1746.95|3012.18|3046.43|215.10| +2450829|15082|2450888|99926|1180984|1242|22653|99926|1180984|1242|22653|1|64|3|2|16072|118|755|87|62.29|112.12|25.78|7511.58|2242.86|5419.23|9754.44|43.06|1525.14|2242.86|717.72|760.78|2960.58|3003.64|-4701.51| +2450829|15082|2450913|99926|1180984|1242|22653|99926|1180984|1242|22653|1|45|19|3|12584|78|755|72|32.73|96.88|95.91|69.84|6905.52|2356.56|6975.36|414.33|0.00|2162.16|6905.52|7319.85|9067.68|9482.01|4548.96| +2450829|77034|2450865|62737|1516003|3669|34135|62737|1516003|3669|34135|4|19|17|4|15037|210|756|35|40.74|117.33|95.03|780.50|3326.05|1425.90|4106.55|83.81|1929.10|451.50|1396.95|1480.76|1848.45|1932.26|-28.95| +2450829|77034|2450906|62737|1516003|3669|34135|62737|1516003|3669|34135|4|5|4|3|6392|28|756|76|91.57|208.77|194.15|1111.12|14755.40|6959.32|15866.52|0.00|0.00|7457.12|14755.40|14755.40|22212.52|22212.52|7796.08| +2450829|77034|2450883|62737|1516003|3669|34135|62737|1516003|3669|34135|4|55|2|2|11632|40|756|90|30.12|66.86|2.00|5837.40|180.00|2710.80|6017.40|12.60|0.00|480.60|180.00|192.60|660.60|673.20|-2530.80| +2450829|77034|2450888|62737|1516003|3669|34135|62737|1516003|3669|34135|4|73|18|2|2638|77|756|94|78.84|127.72|68.96|5523.44|6482.24|7410.96|12005.68|388.93|0.00|2160.12|6482.24|6871.17|8642.36|9031.29|-928.72| +2450829|77034|2450885|62737|1516003|3669|34135|62737|1516003|3669|34135|4|92|7|4|5884|288|756|16|59.94|62.33|44.87|279.36|717.92|959.04|997.28|35.89|0.00|339.04|717.92|753.81|1056.96|1092.85|-241.12| +2450829|77034|2450873|62737|1516003|3669|34135|62737|1516003|3669|34135|4|100|18|4|4045|35|756|59|52.10|83.36|83.36|0.00|4918.24|3073.90|4918.24|245.91|0.00|48.97|4918.24|5164.15|4967.21|5213.12|1844.34| +2450829|77034|2450875|62737|1516003|3669|34135|62737|1516003|3669|34135|4|21|5|3|253|162|756|53|49.90|77.34|64.96|656.14|3442.88|2644.70|4099.02|137.71|0.00|1229.60|3442.88|3580.59|4672.48|4810.19|798.18| +2450829|77034|2450854|62737|1516003|3669|34135|62737|1516003|3669|34135|4|24|1|2|1492|136|756|86|35.42|59.85|55.66|360.34|4786.76|3046.12|5147.10|430.80|0.00|1646.90|4786.76|5217.56|6433.66|6864.46|1740.64| +2450829|77034|2450841|62737|1516003|3669|34135|62737|1516003|3669|34135|4|79|15|5|14932|257|756|83|8.80|25.08|20.31|395.91|1685.73|730.40|2081.64|33.71|0.00|790.99|1685.73|1719.44|2476.72|2510.43|955.33| +2450829|77034|2450870|62737|1516003|3669|34135|62737|1516003|3669|34135|4|48|7|2|9134|285|756|70|10.08|14.61|4.23|726.60|296.10|705.60|1022.70|5.92|0.00|398.30|296.10|302.02|694.40|700.32|-409.50| +2450829|77034|2450914|62737|1516003|3669|34135|62737|1516003|3669|34135|4|49|13|5|9865|24|756|15|33.20|91.30|15.52|1136.70|232.80|498.00|1369.50|11.64|0.00|397.05|232.80|244.44|629.85|641.49|-265.20| +2450829|77034|2450890|62737|1516003|3669|34135|62737|1516003|3669|34135|4|13|3|1|10448|247|756|85|32.87|42.07|2.52|3361.75|214.20|2793.95|3575.95|10.71|0.00|893.35|214.20|224.91|1107.55|1118.26|-2579.75| +2450829|77034|2450847|62737|1516003|3669|34135|62737|1516003|3669|34135|4|5|7|1|17221|232|756|29|57.80|162.99|13.03|4348.84|377.87|1676.20|4726.71|0.00|0.00|661.49|377.87|377.87|1039.36|1039.36|-1298.33| +2450829|77034|2450844|62737|1516003|3669|34135|62737|1516003|3669|34135|4|17|7|3|176|60|756|88|66.06|81.91|31.94|4397.36|2810.72|5813.28|7208.08|139.13|28.10|71.28|2782.62|2921.75|2853.90|2993.03|-3030.66| +2450829|80225|2450916|41350|970915|2433|16398|41350|970915|2433|16398|4|37|7|5|9962|164|757|100|52.50|75.60|47.62|2798.00|4762.00|5250.00|7560.00|142.86|0.00|3250.00|4762.00|4904.86|8012.00|8154.86|-488.00| +2450829|80225|2450835|41350|970915|2433|16398|41350|970915|2433|16398|4|47|15|2|3451|296|757|44|80.83|132.56|124.60|350.24|5482.40|3556.52|5832.64|54.82|0.00|2332.88|5482.40|5537.22|7815.28|7870.10|1925.88| +2450829|80225|2450845|41350|970915|2433|16398|41350|970915|2433|16398|4|45|2|2|3403|120|757|91|87.57|232.93|79.19|13990.34|7206.29|7968.87|21196.63|576.50|0.00|8478.47|7206.29|7782.79|15684.76|16261.26|-762.58| +2450829|80225|2450847|41350|970915|2433|16398|41350|970915|2433|16398|4|19|1|2|8629|237|757|91|58.77|138.69|12.48|11485.11|1135.68|5348.07|12620.79|37.25|670.05|882.70|465.63|502.88|1348.33|1385.58|-4882.44| +2450829|80225|2450834|41350|970915|2433|16398|41350|970915|2433|16398|4|22|14|5|9446|202|757|14|80.47|166.57|39.97|1772.40|559.58|1126.58|2331.98|39.17|0.00|23.24|559.58|598.75|582.82|621.99|-567.00| +2450829|80225|2450919|41350|970915|2433|16398|41350|970915|2433|16398|4|83|20|3|14401|214|757|84|31.63|36.05|20.90|1272.60|1755.60|2656.92|3028.20|7.02|1580.04|272.16|175.56|182.58|447.72|454.74|-2481.36| +2450829|80225|2450868|41350|970915|2433|16398|41350|970915|2433|16398|4|53|10|3|4274|54|757|31|23.36|60.03|51.02|279.31|1581.62|724.16|1860.93|47.44|0.00|130.20|1581.62|1629.06|1711.82|1759.26|857.46| +2450829|80225|2450833|41350|970915|2433|16398|41350|970915|2433|16398|4|17|2|4|3854|213|757|24|85.73|219.46|96.56|2949.60|2317.44|2057.52|5267.04|185.39|0.00|2580.72|2317.44|2502.83|4898.16|5083.55|259.92| +2450829|80225|2450834|41350|970915|2433|16398|41350|970915|2433|16398|4|68|9|1|10903|205|757|75|88.10|253.72|71.04|13701.00|5328.00|6607.50|19029.00|319.68|0.00|8181.75|5328.00|5647.68|13509.75|13829.43|-1279.50| +2450829|78117|2450899|54079|78975|5118|10014|54079|78975|5118|10014|1|81|2|3|10694|285|758|53|82.24|240.96|207.22|1788.22|10982.66|4358.72|12770.88|988.43|0.00|0.00|10982.66|11971.09|10982.66|11971.09|6623.94| +2450829|78117|2450880|54079|78975|5118|10014|54079|78975|5118|10014|1|86|10|1|4369|198|758|86|31.64|42.08|37.87|362.06|3256.82|2721.04|3618.88|130.27|0.00|0.00|3256.82|3387.09|3256.82|3387.09|535.78| +2450829|78117|2450837|54079|78975|5118|10014|54079|78975|5118|10014|1|29|8|1|16996|89|758|37|57.40|157.27|42.46|4247.97|1571.02|2123.80|5818.99|0.00|0.00|2560.03|1571.02|1571.02|4131.05|4131.05|-552.78| +2450829|78117|2450867|54079|78975|5118|10014|54079|78975|5118|10014|1|85|9|2|260|113|758|77|80.21|110.68|66.40|3409.56|5112.80|6176.17|8522.36|357.89|0.00|1618.54|5112.80|5470.69|6731.34|7089.23|-1063.37| +2450829|73163|2450892|83778|1757939|1700|35861|83778|1757939|1700|35861|4|93|10|4|12968|202|759|12|76.44|133.00|122.36|127.68|1468.32|917.28|1596.00|29.36|0.00|111.72|1468.32|1497.68|1580.04|1609.40|551.04| +2450829|73163|2450894|83778|1757939|1700|35861|83778|1757939|1700|35861|4|94|9|4|8600|58|759|52|82.85|134.21|116.76|907.40|6071.52|4308.20|6978.92|546.43|0.00|2163.20|6071.52|6617.95|8234.72|8781.15|1763.32| +2450829|73163|2450874|83778|1757939|1700|35861|83778|1757939|1700|35861|4|58|9|3|6166|186|759|82|94.76|157.30|51.90|8642.80|4255.80|7770.32|12898.60|297.90|0.00|5029.88|4255.80|4553.70|9285.68|9583.58|-3514.52| +2450829|73163|2450894|83778|1757939|1700|35861|83778|1757939|1700|35861|4|95|14|3|17152|111|759|10|14.76|30.70|17.19|135.10|171.90|147.60|307.00|15.47|0.00|24.50|171.90|187.37|196.40|211.87|24.30| +2450829|73163|2450831|83778|1757939|1700|35861|83778|1757939|1700|35861|4|21|6|1|13894|48|759|66|8.35|14.77|11.22|234.30|740.52|551.10|974.82|37.02|0.00|272.58|740.52|777.54|1013.10|1050.12|189.42| +2450829|73163|2450919|83778|1757939|1700|35861|83778|1757939|1700|35861|4|60|17|4|14434|103|759|39|44.53|99.30|82.41|658.71|3213.99|1736.67|3872.70|0.00|0.00|425.88|3213.99|3213.99|3639.87|3639.87|1477.32| +2450829|73163|2450848|83778|1757939|1700|35861|83778|1757939|1700|35861|4|24|15|2|7177|15|759|69|77.03|100.13|53.06|3247.83|3661.14|5315.07|6908.97|219.66|0.00|2417.76|3661.14|3880.80|6078.90|6298.56|-1653.93| +2450829|73163|2450832|83778|1757939|1700|35861|83778|1757939|1700|35861|4|88|7|3|8612|282|759|72|50.10|109.71|59.24|3633.84|4265.28|3607.20|7899.12|213.26|0.00|2448.72|4265.28|4478.54|6714.00|6927.26|658.08| +2450829|73163|2450851|83778|1757939|1700|35861|83778|1757939|1700|35861|4|69|13|2|17887|45|759|51|41.39|119.61|27.51|4697.10|1403.01|2110.89|6100.11|70.15|0.00|914.94|1403.01|1473.16|2317.95|2388.10|-707.88| +2450829|73163|2450859|83778|1757939|1700|35861|83778|1757939|1700|35861|4|27|11|2|5197|139|759|38|77.31|82.72|33.08|1886.32|1257.04|2937.78|3143.36|75.42|0.00|534.28|1257.04|1332.46|1791.32|1866.74|-1680.74| +2450829|59345|2450888|48014|438668|5322|46729|48014|438668|5322|46729|2|51|5|3|3931|56|760|92|43.43|58.63|22.86|3290.84|2103.12|3995.56|5393.96|147.21|0.00|2480.32|2103.12|2250.33|4583.44|4730.65|-1892.44| +2450829|59345|2450885|48014|438668|5322|46729|48014|438668|5322|46729|2|50|3|2|13261|80|760|14|63.59|183.77|95.56|1234.94|1337.84|890.26|2572.78|107.02|0.00|360.08|1337.84|1444.86|1697.92|1804.94|447.58| +2450829|59345|2450854|48014|438668|5322|46729|48014|438668|5322|46729|2|81|14|1|2863|62|760|22|27.79|76.14|35.02|904.64|770.44|611.38|1675.08|0.00|423.74|334.84|346.70|346.70|681.54|681.54|-264.68| +2450829|59345|2450882|48014|438668|5322|46729|48014|438668|5322|46729|2|91|7|4|10321|237|760|22|60.52|65.96|26.38|870.76|580.36|1331.44|1451.12|5.80|0.00|319.22|580.36|586.16|899.58|905.38|-751.08| +2450829|59345|2450906|48014|438668|5322|46729|48014|438668|5322|46729|2|20|6|1|12256|144|760|3|9.65|27.02|15.67|34.05|47.01|28.95|81.06|2.82|0.00|38.88|47.01|49.83|85.89|88.71|18.06| +2450829|59345|2450867|48014|438668|5322|46729|48014|438668|5322|46729|2|68|12|3|2764|82|760|11|50.25|150.24|34.55|1272.59|380.05|552.75|1652.64|9.80|53.20|247.83|326.85|336.65|574.68|584.48|-225.90| +2450829|59345|2450870|48014|438668|5322|46729|48014|438668|5322|46729|2|62|16|4|3878|260|760|65|61.21|127.92|112.56|998.40|7316.40|3978.65|8314.80|585.31|0.00|3408.60|7316.40|7901.71|10725.00|11310.31|3337.75| +2450829|59345|2450902|48014|438668|5322|46729|48014|438668|5322|46729|2|85|14|2|15865|88|760|34|98.28|249.63|174.74|2546.26|5941.16|3341.52|8487.42|475.29|0.00|3988.88|5941.16|6416.45|9930.04|10405.33|2599.64| +2450829|59345|2450851|48014|438668|5322|46729|48014|438668|5322|46729|2|59|3|2|14386|193|760|27|73.97|105.03|94.52|283.77|2552.04|1997.19|2835.81|76.56|0.00|481.95|2552.04|2628.60|3033.99|3110.55|554.85| +2450829|59345|2450919|48014|438668|5322|46729|48014|438668|5322|46729|2|26|12|2|14806|264|760|31|92.12|137.25|34.31|3191.14|1063.61|2855.72|4254.75|0.00|0.00|42.47|1063.61|1063.61|1106.08|1106.08|-1792.11| +2450829|23772|2450878|42299|394286|226|26213|42299|394286|226|26213|1|41|14|3|5566|49|761|76|59.67|112.17|84.12|2131.80|6393.12|4534.92|8524.92|575.38|0.00|3324.24|6393.12|6968.50|9717.36|10292.74|1858.20| +2450829|23772|2450887|42299|394286|226|26213|42299|394286|226|26213|1|86|7|4|8803|55|761|55|75.50|178.93|168.19|590.70|9250.45|4152.50|9841.15|832.54|0.00|3050.30|9250.45|10082.99|12300.75|13133.29|5097.95| +2450829|23772|2450919|42299|394286|226|26213|42299|394286|226|26213|1|106|2|1|10150|183|761|43|18.08|43.21|28.08|650.59|1207.44|777.44|1858.03|108.66|0.00|687.14|1207.44|1316.10|1894.58|2003.24|430.00| +2450829|23772|2450894|42299|394286|226|26213|42299|394286|226|26213|1|19|15|4|1231|2|761|25|48.35|134.41|68.54|1646.75|1713.50|1208.75|3360.25|0.00|0.00|571.00|1713.50|1713.50|2284.50|2284.50|504.75| +2450829|23772|2450872|42299|394286|226|26213|42299|394286|226|26213|1|61|19|1|340|285|761|31|78.88|164.85|75.83|2759.62|2350.73|2445.28|5110.35|188.05|0.00|306.59|2350.73|2538.78|2657.32|2845.37|-94.55| +2450829|23772|2450834|42299|394286|226|26213|42299|394286|226|26213|1|5|3|2|2731|207|761|88|38.62|105.43|59.04|4082.32|5195.52|3398.56|9277.84|207.82|0.00|2875.84|5195.52|5403.34|8071.36|8279.18|1796.96| +2450829|23772|2450913|42299|394286|226|26213|42299|394286|226|26213|1|6|11|2|8983|184|761|36|39.37|51.57|23.20|1021.32|835.20|1417.32|1856.52|33.40|0.00|594.00|835.20|868.60|1429.20|1462.60|-582.12| +2450829|23772|2450891|42299|394286|226|26213|42299|394286|226|26213|1|34|12|1|5479|217|761|21|96.36|106.95|53.47|1123.08|1122.87|2023.56|2245.95|22.45|0.00|965.58|1122.87|1145.32|2088.45|2110.90|-900.69| +2450829|64813|2450883|96148|1907359|4970|36981|96148|1907359|4970|36981|1|103|16|4|5779|160|762|62|97.14|189.42|13.25|10922.54|821.50|6022.68|11744.04|73.93|0.00|5284.26|821.50|895.43|6105.76|6179.69|-5201.18| +2450829|64813|2450883|96148|1907359|4970|36981|96148|1907359|4970|36981|1|3|9|3|8114|177|762|48|51.29|123.09|107.08|768.48|5139.84|2461.92|5908.32|205.59|0.00|1476.96|5139.84|5345.43|6616.80|6822.39|2677.92| +2450829|64813|2450861|96148|1907359|4970|36981|96148|1907359|4970|36981|1|58|3|3|16112|30|762|66|71.49|111.52|97.02|957.00|6403.32|4718.34|7360.32|320.16|0.00|3385.14|6403.32|6723.48|9788.46|10108.62|1684.98| +2450829|64813|2450880|96148|1907359|4970|36981|96148|1907359|4970|36981|1|51|16|2|15818|189|762|25|77.07|181.88|105.49|1909.75|2637.25|1926.75|4547.00|11.86|2505.38|636.50|131.87|143.73|768.37|780.23|-1794.88| +2450829|64813|2450834|96148|1907359|4970|36981|96148|1907359|4970|36981|1|96|13|1|8920|120|762|51|64.23|89.92|35.96|2751.96|1833.96|3275.73|4585.92|12.47|1522.18|550.29|311.78|324.25|862.07|874.54|-2963.95| +2450829|64813|2450854|96148|1907359|4970|36981|96148|1907359|4970|36981|1|17|20|2|9670|134|762|53|9.48|9.48|9.29|10.07|492.37|502.44|502.44|24.61|0.00|49.82|492.37|516.98|542.19|566.80|-10.07| +2450829|64813|2450848|96148|1907359|4970|36981|96148|1907359|4970|36981|1|101|19|1|11707|190|762|62|40.18|55.44|33.81|1341.06|2096.22|2491.16|3437.28|125.77|0.00|308.76|2096.22|2221.99|2404.98|2530.75|-394.94| +2450829|64813|2450897|96148|1907359|4970|36981|96148|1907359|4970|36981|1|29|15|2|13981|214|762|27|80.90|129.44|81.54|1293.30|2201.58|2184.30|3494.88|154.11|0.00|1607.58|2201.58|2355.69|3809.16|3963.27|17.28| +2450829|25783|2450858|7734|1561164|1347|25930|7734|1561164|1347|25930|1|4|12|3|9148|125|763|8|71.47|208.69|64.69|1152.00|517.52|571.76|1669.52|16.97|305.33|667.76|212.19|229.16|879.95|896.92|-359.57| +2450829|25783|2450833|7734|1561164|1347|25930|7734|1561164|1347|25930|1|32|3|1|13046|115|763|94|54.60|102.10|5.10|9118.00|479.40|5132.40|9597.40|2.30|402.69|1535.02|76.71|79.01|1611.73|1614.03|-5055.69| +2450829|25783|2450834|7734|1561164|1347|25930|7734|1561164|1347|25930|1|88|5|3|14473|11|763|12|97.98|116.59|106.09|126.00|1273.08|1175.76|1399.08|101.84|0.00|27.96|1273.08|1374.92|1301.04|1402.88|97.32| +2450829|25783|2450850|7734|1561164|1347|25930|7734|1561164|1347|25930|1|46|10|2|17287|131|763|51|46.15|86.30|61.27|1276.53|3124.77|2353.65|4401.30|249.98|0.00|1760.52|3124.77|3374.75|4885.29|5135.27|771.12| +2450829|25783|2450878|7734|1561164|1347|25930|7734|1561164|1347|25930|1|76|13|5|5378|102|763|42|43.36|85.41|63.20|932.82|2654.40|1821.12|3587.22|238.89|0.00|0.00|2654.40|2893.29|2654.40|2893.29|833.28| +2450829|25783|2450858|7734|1561164|1347|25930|7734|1561164|1347|25930|1|30|1|5|11887|199|763|99|99.92|264.78|137.68|12582.90|13630.32|9892.08|26213.22|703.32|1908.24|2359.17|11722.08|12425.40|14081.25|14784.57|1830.00| +2450829|25783|2450912|7734|1561164|1347|25930|7734|1561164|1347|25930|1|52|20|2|14635|236|763|28|24.59|53.85|5.38|1357.16|150.64|688.52|1507.80|6.02|0.00|120.40|150.64|156.66|271.04|277.06|-537.88| +2450829|25783|2450832|7734|1561164|1347|25930|7734|1561164|1347|25930|1|19|10|1|13099|168|763|2|41.15|63.78|44.64|38.28|89.28|82.30|127.56|1.78|0.00|2.54|89.28|91.06|91.82|93.60|6.98| +2450829|25783|2450887|7734|1561164|1347|25930|7734|1561164|1347|25930|1|37|7|1|11494|104|763|8|34.34|44.98|16.19|230.32|129.52|274.72|359.84|10.36|0.00|39.52|129.52|139.88|169.04|179.40|-145.20| +2450829|25783|2450903|7734|1561164|1347|25930|7734|1561164|1347|25930|1|68|18|3|15200|43|763|95|90.44|243.28|128.93|10863.25|12248.35|8591.80|23111.60|248.64|8696.32|9937.95|3552.03|3800.67|13489.98|13738.62|-5039.77| +2450829|25783|2450838|7734|1561164|1347|25930|7734|1561164|1347|25930|1|32|7|2|4808|82|763|85|37.74|102.65|79.04|2006.85|6718.40|3207.90|8725.25|335.92|0.00|2355.35|6718.40|7054.32|9073.75|9409.67|3510.50| +2450829|25783|2450874|7734|1561164|1347|25930|7734|1561164|1347|25930|1|81|2|4|17173|290|763|46|89.32|253.66|43.12|9684.84|1983.52|4108.72|11668.36|19.83|0.00|3967.04|1983.52|2003.35|5950.56|5970.39|-2125.20| +2450829|25783|2450831|7734|1561164|1347|25930|7734|1561164|1347|25930|1|86|2|4|3122|11|763|55|13.86|15.24|0.15|829.95|8.25|762.30|838.20|0.00|0.00|242.55|8.25|8.25|250.80|250.80|-754.05| +2450829|46265|2450866|9628|1058037|1293|4949|9628|1058037|1293|4949|2|30|17|2|16642|180|764|68|26.39|67.82|35.26|2214.08|2397.68|1794.52|4611.76|119.88|0.00|0.00|2397.68|2517.56|2397.68|2517.56|603.16| +2450829|46265|2450831|9628|1058037|1293|4949|9628|1058037|1293|4949|2|34|20|4|9068|130|764|98|56.73|152.60|117.50|3439.80|11515.00|5559.54|14954.80|115.15|0.00|4635.40|11515.00|11630.15|16150.40|16265.55|5955.46| +2450829|46265|2450848|9628|1058037|1293|4949|9628|1058037|1293|4949|2|54|5|3|5246|194|764|48|1.68|2.90|2.05|40.80|98.40|80.64|139.20|3.93|0.00|0.00|98.40|102.33|98.40|102.33|17.76| +2450829|46265|2450915|9628|1058037|1293|4949|9628|1058037|1293|4949|2|88|1|3|9271|10|764|24|4.90|13.13|8.53|110.40|204.72|117.60|315.12|14.33|0.00|132.24|204.72|219.05|336.96|351.29|87.12| +2450829|46265|2450880|9628|1058037|1293|4949|9628|1058037|1293|4949|2|83|5|3|6764|104|764|4|90.66|223.93|203.77|80.64|815.08|362.64|895.72|65.20|0.00|214.96|815.08|880.28|1030.04|1095.24|452.44| +2450829|46265|2450850|9628|1058037|1293|4949|9628|1058037|1293|4949|2|84|13|4|2089|140|764|73|74.05|115.51|78.54|2698.81|5733.42|5405.65|8432.23|229.33|0.00|3962.44|5733.42|5962.75|9695.86|9925.19|327.77| +2450829|46265|2450867|9628|1058037|1293|4949|9628|1058037|1293|4949|2|103|6|3|9196|14|764|97|8.50|19.80|14.85|480.15|1440.45|824.50|1920.60|57.61|0.00|153.26|1440.45|1498.06|1593.71|1651.32|615.95| +2450829|46265|2450860|9628|1058037|1293|4949|9628|1058037|1293|4949|2|88|7|4|15674|105|764|77|31.21|52.74|11.60|3167.78|893.20|2403.17|4060.98|8.93|0.00|934.01|893.20|902.13|1827.21|1836.14|-1509.97| +2450829|46265|2450842|9628|1058037|1293|4949|9628|1058037|1293|4949|2|89|16|1|3662|214|764|50|19.74|55.46|5.54|2496.00|277.00|987.00|2773.00|2.88|132.96|443.50|144.04|146.92|587.54|590.42|-842.96| +2450829|46265|2450887|9628|1058037|1293|4949|9628|1058037|1293|4949|2|36|6|4|14452|165|764|21|56.03|86.84|65.99|437.85|1385.79|1176.63|1823.64|124.72|0.00|784.14|1385.79|1510.51|2169.93|2294.65|209.16| +2450829|46265|2450856|9628|1058037|1293|4949|9628|1058037|1293|4949|2|70|18|1|5468|216|764|50|62.25|103.95|96.67|364.00|4833.50|3112.50|5197.50|386.68|0.00|831.50|4833.50|5220.18|5665.00|6051.68|1721.00| +2450829|46265|2450862|9628|1058037|1293|4949|9628|1058037|1293|4949|2|88|10|3|7658|258|764|17|96.80|124.87|98.64|445.91|1676.88|1645.60|2122.79|0.00|905.51|848.98|771.37|771.37|1620.35|1620.35|-874.23| +2450829|46265|2450836|9628|1058037|1293|4949|9628|1058037|1293|4949|2|75|9|5|13516|190|764|17|50.35|143.49|2.86|2390.71|48.62|855.95|2439.33|0.87|34.03|804.95|14.59|15.46|819.54|820.41|-841.36| +2450829|63043|2450860|30655|1811895|7088|2281|91527|1890714|2707|19900|1|40|19|3|3286|127|765|59|13.15|37.08|25.95|656.67|1531.05|775.85|2187.72|61.24|0.00|196.47|1531.05|1592.29|1727.52|1788.76|755.20| +2450829|63043|2450864|30655|1811895|7088|2281|91527|1890714|2707|19900|1|18|3|3|15691|191|765|23|87.53|232.82|30.26|4658.88|695.98|2013.19|5354.86|27.83|0.00|2195.35|695.98|723.81|2891.33|2919.16|-1317.21| +2450829|63043|2450831|30655|1811895|7088|2281|91527|1890714|2707|19900|1|8|5|3|7603|268|765|30|7.11|15.64|4.06|347.40|121.80|213.30|469.20|3.39|84.04|210.90|37.76|41.15|248.66|252.05|-175.54| +2450829|63043|2450872|30655|1811895|7088|2281|91527|1890714|2707|19900|1|72|3|3|11284|1|765|36|56.73|156.57|73.58|2987.64|2648.88|2042.28|5636.52|158.93|0.00|2818.08|2648.88|2807.81|5466.96|5625.89|606.60| +2450829|63043|2450867|30655|1811895|7088|2281|91527|1890714|2707|19900|1|33|3|4|16646|31|765|29|43.48|96.52|71.42|727.90|2071.18|1260.92|2799.08|82.84|0.00|447.76|2071.18|2154.02|2518.94|2601.78|810.26| +2450829|63043|2450913|30655|1811895|7088|2281|91527|1890714|2707|19900|1|5|18|2|3937|88|765|8|85.70|164.54|157.95|52.72|1263.60|685.60|1316.32|9.60|783.43|144.72|480.17|489.77|624.89|634.49|-205.43| +2450829|63043|2450884|30655|1811895|7088|2281|91527|1890714|2707|19900|1|81|8|4|524|251|765|23|32.78|93.42|67.26|601.68|1546.98|753.94|2148.66|15.46|0.00|537.05|1546.98|1562.44|2084.03|2099.49|793.04| +2450829|63043|2450881|30655|1811895|7088|2281|91527|1890714|2707|19900|1|25|14|1|6052|38|765|99|58.32|157.46|91.32|6547.86|9040.68|5773.68|15588.54|0.00|0.00|1558.26|9040.68|9040.68|10598.94|10598.94|3267.00| +2450829|63043|2450893|30655|1811895|7088|2281|91527|1890714|2707|19900|1|2|9|2|6194|243|765|85|64.22|161.19|56.41|8906.30|4794.85|5458.70|13701.15|0.00|0.00|1232.50|4794.85|4794.85|6027.35|6027.35|-663.85| +2450829|49915|2450881|72479|1137276|4473|19270|72479|1137276|4473|19270|4|20|17|1|12277|272|766|40|50.83|149.94|97.46|2099.20|3898.40|2033.20|5997.60|155.93|0.00|1019.20|3898.40|4054.33|4917.60|5073.53|1865.20| +2450829|49915|2450913|72479|1137276|4473|19270|72479|1137276|4473|19270|4|10|2|3|8186|218|766|40|21.69|37.08|30.40|267.20|1216.00|867.60|1483.20|46.20|291.84|44.40|924.16|970.36|968.56|1014.76|56.56| +2450829|49915|2450841|72479|1137276|4473|19270|72479|1137276|4473|19270|4|92|10|2|14869|148|766|6|61.96|88.60|75.31|79.74|451.86|371.76|531.60|22.59|0.00|69.06|451.86|474.45|520.92|543.51|80.10| +2450829|49915|2450909|72479|1137276|4473|19270|72479|1137276|4473|19270|4|13|5|1|16778|36|766|68|91.40|207.47|184.64|1552.44|12555.52|6215.20|14107.96|125.55|0.00|987.36|12555.52|12681.07|13542.88|13668.43|6340.32| +2450829|49915|2450903|72479|1137276|4473|19270|72479|1137276|4473|19270|4|48|6|1|6706|259|766|13|54.36|109.80|73.56|471.12|956.28|706.68|1427.40|47.81|0.00|613.73|956.28|1004.09|1570.01|1617.82|249.60| +2450830|47388|2450896|11518|889905|7045|13432|76757|614559|6099|26886|1|45|4|5|3416|223|767|39|59.29|118.58|24.90|3653.52|971.10|2312.31|4624.62|29.13|0.00|1109.55|971.10|1000.23|2080.65|2109.78|-1341.21| +2450830|47388|2450841|11518|889905|7045|13432|76757|614559|6099|26886|1|53|10|2|4628|126|767|91|75.34|199.65|127.77|6541.08|11627.07|6855.94|18168.15|0.00|5464.72|7811.44|6162.35|6162.35|13973.79|13973.79|-693.59| +2450830|47388|2450910|11518|889905|7045|13432|76757|614559|6099|26886|1|85|16|4|2068|118|767|93|20.29|37.13|33.41|345.96|3107.13|1886.97|3453.09|45.98|1957.49|1381.05|1149.64|1195.62|2530.69|2576.67|-737.33| +2450830|47388|2450838|11518|889905|7045|13432|76757|614559|6099|26886|1|3|15|1|6626|25|767|97|19.10|38.58|33.56|486.94|3255.32|1852.70|3742.26|97.65|0.00|747.87|3255.32|3352.97|4003.19|4100.84|1402.62| +2450830|47388|2450893|11518|889905|7045|13432|76757|614559|6099|26886|1|15|17|5|14719|4|767|58|18.50|23.49|22.78|41.18|1321.24|1073.00|1362.42|105.69|0.00|571.88|1321.24|1426.93|1893.12|1998.81|248.24| +2450830|47388|2450842|11518|889905|7045|13432|76757|614559|6099|26886|1|104|16|1|11054|129|767|48|94.76|255.85|117.69|6631.68|5649.12|4548.48|12280.80|508.42|0.00|3684.00|5649.12|6157.54|9333.12|9841.54|1100.64| +2450830|47388|2450862|11518|889905|7045|13432|76757|614559|6099|26886|1|12|15|3|15044|287|767|13|14.68|41.25|8.66|423.67|112.58|190.84|536.25|0.00|0.00|53.56|112.58|112.58|166.14|166.14|-78.26| +2450830|47388|2450880|11518|889905|7045|13432|76757|614559|6099|26886|1|9|16|5|4042|136|767|53|51.64|63.51|8.89|2894.86|471.17|2736.92|3366.03|18.84|0.00|1312.28|471.17|490.01|1783.45|1802.29|-2265.75| +2450830|47388|2450904|11518|889905|7045|13432|76757|614559|6099|26886|1|86|1|5|12220|241|767|99|42.40|58.93|17.67|4084.74|1749.33|4197.60|5834.07|52.47|0.00|2216.61|1749.33|1801.80|3965.94|4018.41|-2448.27| +2450830|47388|2450888|11518|889905|7045|13432|76757|614559|6099|26886|1|39|15|3|1405|47|767|73|9.20|17.02|4.08|944.62|297.84|671.60|1242.46|7.50|110.20|248.20|187.64|195.14|435.84|443.34|-483.96| +2450830|47388|2450873|11518|889905|7045|13432|76757|614559|6099|26886|1|2|13|4|17611|158|767|77|29.29|63.26|27.83|2728.11|2142.91|2255.33|4871.02|85.71|0.00|1071.07|2142.91|2228.62|3213.98|3299.69|-112.42| +2450830|47388|2450889|11518|889905|7045|13432|76757|614559|6099|26886|1|36|5|3|15202|227|767|26|60.61|134.55|43.05|2379.00|1119.30|1575.86|3498.30|44.77|0.00|139.88|1119.30|1164.07|1259.18|1303.95|-456.56| +2450830|47388|2450912|11518|889905|7045|13432|76757|614559|6099|26886|1|26|15|3|2263|171|767|60|90.36|247.58|86.65|9655.80|5199.00|5421.60|14854.80|259.95|0.00|1633.80|5199.00|5458.95|6832.80|7092.75|-222.60| +2450830|47388|2450859|11518|889905|7045|13432|76757|614559|6099|26886|1|9|11|5|10021|235|767|93|95.79|223.19|124.98|9133.53|11623.14|8908.47|20756.67|813.61|0.00|8302.11|11623.14|12436.75|19925.25|20738.86|2714.67| +2450830|46993|2450882|81045|1543942|1239|1336|81045|1543942|1239|1336|4|47|12|1|3364|57|768|98|35.07|66.63|43.30|2286.34|4243.40|3436.86|6529.74|169.73|0.00|2415.70|4243.40|4413.13|6659.10|6828.83|806.54| +2450830|46993|2450884|81045|1543942|1239|1336|81045|1543942|1239|1336|4|21|4|1|7675|224|768|9|3.69|4.94|4.54|3.60|40.86|33.21|44.46|1.22|0.00|11.07|40.86|42.08|51.93|53.15|7.65| +2450830|46993|2450908|81045|1543942|1239|1336|81045|1543942|1239|1336|4|33|12|5|9904|138|768|45|87.32|165.03|70.96|4233.15|3193.20|3929.40|7426.35|63.86|0.00|2524.95|3193.20|3257.06|5718.15|5782.01|-736.20| +2450830|46993|2450901|81045|1543942|1239|1336|81045|1543942|1239|1336|4|24|2|3|3367|248|768|94|61.71|82.69|35.55|4431.16|3341.70|5800.74|7772.86|133.66|0.00|1320.70|3341.70|3475.36|4662.40|4796.06|-2459.04| +2450830|46993|2450848|81045|1543942|1239|1336|81045|1543942|1239|1336|4|49|19|1|9064|93|768|45|77.79|183.58|23.86|7187.40|1073.70|3500.55|8261.10|22.86|311.37|1651.95|762.33|785.19|2414.28|2437.14|-2738.22| +2450830|46993|2450890|81045|1543942|1239|1336|81045|1543942|1239|1336|4|7|10|2|14864|255|768|13|39.12|76.28|57.97|238.03|753.61|508.56|991.64|30.14|150.72|317.20|602.89|633.03|920.09|950.23|94.33| +2450830|46993|2450834|81045|1543942|1239|1336|81045|1543942|1239|1336|4|83|5|4|12032|152|768|50|83.74|224.42|96.50|6396.00|4825.00|4187.00|11221.00|289.50|0.00|897.50|4825.00|5114.50|5722.50|6012.00|638.00| +2450830|46993|2450846|81045|1543942|1239|1336|81045|1543942|1239|1336|4|17|9|4|11084|14|768|85|19.43|56.92|29.59|2323.05|2515.15|1651.55|4838.20|201.21|0.00|918.85|2515.15|2716.36|3434.00|3635.21|863.60| +2450830|46993|2450868|81045|1543942|1239|1336|81045|1543942|1239|1336|4|62|13|1|7567|105|768|87|34.32|90.26|37.00|4633.62|3219.00|2985.84|7852.62|64.38|0.00|1805.25|3219.00|3283.38|5024.25|5088.63|233.16| +2450830|73412|2450860|116|678083|5650|31617|46643|1549951|7118|40285|1|12|1|3|6511|13|769|3|44.35|92.69|85.27|22.26|255.81|133.05|278.07|15.34|0.00|16.68|255.81|271.15|272.49|287.83|122.76| +2450830|73412|2450865|116|678083|5650|31617|46643|1549951|7118|40285|1|7|16|5|7999|98|769|74|30.26|65.05|53.34|866.54|3947.16|2239.24|4813.70|118.41|0.00|1203.24|3947.16|4065.57|5150.40|5268.81|1707.92| +2450830|73412|2450901|116|678083|5650|31617|46643|1549951|7118|40285|1|3|1|5|1918|30|769|1|31.93|60.98|26.83|34.15|26.83|31.93|60.98|1.34|0.00|25.61|26.83|28.17|52.44|53.78|-5.10| +2450830|73412|2450837|116|678083|5650|31617|46643|1549951|7118|40285|1|77|17|3|11050|245|769|31|5.60|7.39|7.31|2.48|226.61|173.60|229.09|18.12|0.00|91.45|226.61|244.73|318.06|336.18|53.01| +2450830|65024|2450907|88469|388921|3773|41825|88469|388921|3773|41825|4|24|7|5|14696|86|770|28|66.27|113.32|80.45|920.36|2252.60|1855.56|3172.96|13.51|901.04|1300.88|1351.56|1365.07|2652.44|2665.95|-504.00| +2450830|65024|2450857|88469|388921|3773|41825|88469|388921|3773|41825|4|69|5|5|5929|200|770|99|54.88|69.14|11.06|5749.92|1094.94|5433.12|6844.86|54.74|0.00|2394.81|1094.94|1149.68|3489.75|3544.49|-4338.18| +2450830|65024|2450882|88469|388921|3773|41825|88469|388921|3773|41825|4|9|8|5|3244|4|770|28|86.15|251.55|90.55|4508.00|2535.40|2412.20|7043.40|76.06|0.00|633.64|2535.40|2611.46|3169.04|3245.10|123.20| +2450830|65024|2450868|88469|388921|3773|41825|88469|388921|3773|41825|4|3|9|3|10249|48|770|9|65.24|66.54|31.27|317.43|281.43|587.16|598.86|0.00|95.68|185.58|185.75|185.75|371.33|371.33|-401.41| +2450830|65024|2450884|88469|388921|3773|41825|88469|388921|3773|41825|4|23|18|2|6922|55|770|42|23.85|44.36|34.60|409.92|1453.20|1001.70|1863.12|53.76|377.83|391.02|1075.37|1129.13|1466.39|1520.15|73.67| +2450830|65024|2450897|88469|388921|3773|41825|88469|388921|3773|41825|4|106|2|3|10988|15|770|70|26.18|36.12|10.11|1820.70|707.70|1832.60|2528.40|28.30|0.00|1187.90|707.70|736.00|1895.60|1923.90|-1124.90| +2450830|65024|2450914|88469|388921|3773|41825|88469|388921|3773|41825|4|24|4|1|11216|57|770|96|83.73|169.97|56.09|10932.48|5384.64|8038.08|16317.12|0.00|2476.93|325.44|2907.71|2907.71|3233.15|3233.15|-5130.37| +2450830|65024|2450852|88469|388921|3773|41825|88469|388921|3773|41825|4|26|18|1|13630|194|770|56|23.42|33.95|1.01|1844.64|56.56|1311.52|1901.20|0.00|0.00|722.40|56.56|56.56|778.96|778.96|-1254.96| +2450830|18461|2450893|74595|1027282|2722|6209|74595|1027282|2722|6209|1|68|6|5|7418|51|771|8|32.19|71.78|29.42|338.88|235.36|257.52|574.24|0.00|0.00|45.92|235.36|235.36|281.28|281.28|-22.16| +2450830|18461|2450863|74595|1027282|2722|6209|74595|1027282|2722|6209|1|21|20|3|15770|35|771|7|6.97|17.35|17.35|0.00|121.45|48.79|121.45|3.64|0.00|53.41|121.45|125.09|174.86|178.50|72.66| +2450830|18461|2450913|74595|1027282|2722|6209|74595|1027282|2722|6209|1|96|2|4|3344|13|771|19|25.87|40.09|3.20|700.91|60.80|491.53|761.71|1.21|0.00|60.80|60.80|62.01|121.60|122.81|-430.73| +2450830|18461|2450854|74595|1027282|2722|6209|74595|1027282|2722|6209|1|71|10|2|12916|173|771|73|54.34|119.54|8.36|8116.14|610.28|3966.82|8726.42|0.00|0.00|1744.70|610.28|610.28|2354.98|2354.98|-3356.54| +2450830|18461|2450894|74595|1027282|2722|6209|74595|1027282|2722|6209|1|22|13|5|1166|267|771|50|83.61|96.15|67.30|1442.50|3365.00|4180.50|4807.50|100.95|0.00|2259.50|3365.00|3465.95|5624.50|5725.45|-815.50| +2450830|18461|2450904|74595|1027282|2722|6209|74595|1027282|2722|6209|1|86|2|1|3313|230|771|50|61.66|172.64|82.86|4489.00|4143.00|3083.00|8632.00|331.44|0.00|1726.00|4143.00|4474.44|5869.00|6200.44|1060.00| +2450830|18461|2450836|74595|1027282|2722|6209|74595|1027282|2722|6209|1|89|16|4|16334|175|771|95|14.85|26.28|13.92|1174.20|1322.40|1410.75|2496.60|105.79|0.00|798.00|1322.40|1428.19|2120.40|2226.19|-88.35| +2450830|18461|2450898|74595|1027282|2722|6209|74595|1027282|2722|6209|1|82|18|2|16525|60|771|60|85.18|160.13|4.80|9319.80|288.00|5110.80|9607.80|14.40|0.00|3843.00|288.00|302.40|4131.00|4145.40|-4822.80| +2450830|42578|2450833|45299|1089429|2359|2489|45299|1089429|2359|2489|4|55|3|1|6706|251|772|9|90.74|139.73|106.19|301.86|955.71|816.66|1257.57|86.01|0.00|289.17|955.71|1041.72|1244.88|1330.89|139.05| +2450830|42578|2450885|45299|1089429|2359|2489|45299|1089429|2359|2489|4|76|1|5|12872|38|772|87|74.19|221.82|179.67|3667.05|15631.29|6454.53|19298.34|1094.19|0.00|5402.70|15631.29|16725.48|21033.99|22128.18|9176.76| +2450830|42578|2450835|45299|1089429|2359|2489|45299|1089429|2359|2489|4|13|3|4|2515|108|772|38|32.73|70.04|53.93|612.18|2049.34|1243.74|2661.52|143.45|0.00|478.80|2049.34|2192.79|2528.14|2671.59|805.60| +2450830|42578|2450865|45299|1089429|2359|2489|45299|1089429|2359|2489|4|6|18|3|15098|125|772|38|23.54|65.20|52.16|495.52|1982.08|894.52|2477.60|138.74|0.00|965.96|1982.08|2120.82|2948.04|3086.78|1087.56| +2450830|42578|2450880|45299|1089429|2359|2489|45299|1089429|2359|2489|4|76|3|5|5668|19|772|51|11.24|33.15|31.16|101.49|1589.16|573.24|1690.65|63.56|0.00|338.13|1589.16|1652.72|1927.29|1990.85|1015.92| +2450830|42578|2450876|45299|1089429|2359|2489|45299|1089429|2359|2489|4|93|8|1|6830|177|772|66|93.72|273.66|62.94|13907.52|4154.04|6185.52|18061.56|0.00|0.00|2347.62|4154.04|4154.04|6501.66|6501.66|-2031.48| +2450830|57164|2450909|76752|457086|620|13754|76752|457086|620|13754|2|52|7|2|12556|54|773|39|32.73|88.37|23.85|2516.28|930.15|1276.47|3446.43|65.11|0.00|1619.67|930.15|995.26|2549.82|2614.93|-346.32| +2450830|57164|2450919|76752|457086|620|13754|76752|457086|620|13754|2|17|12|1|3061|58|773|87|6.18|15.01|14.85|13.92|1291.95|537.66|1305.87|90.43|0.00|91.35|1291.95|1382.38|1383.30|1473.73|754.29| +2450830|57164|2450833|76752|457086|620|13754|76752|457086|620|13754|2|11|2|3|15499|126|773|67|54.64|127.85|106.11|1456.58|7109.37|3660.88|8565.95|71.09|0.00|1113.54|7109.37|7180.46|8222.91|8294.00|3448.49| +2450830|57164|2450917|76752|457086|620|13754|76752|457086|620|13754|2|65|10|3|10474|47|773|89|14.94|31.37|14.11|1536.14|1255.79|1329.66|2791.93|62.78|0.00|1339.45|1255.79|1318.57|2595.24|2658.02|-73.87| +2450830|57164|2450875|76752|457086|620|13754|76752|457086|620|13754|2|98|8|1|4180|222|773|84|52.50|92.40|2.77|7528.92|232.68|4410.00|7761.60|11.51|2.32|1241.52|230.36|241.87|1471.88|1483.39|-4179.64| +2450830|57164|2450863|76752|457086|620|13754|76752|457086|620|13754|2|66|15|3|1454|243|773|32|6.16|9.24|1.66|242.56|53.12|197.12|295.68|4.24|0.00|35.20|53.12|57.36|88.32|92.56|-144.00| +2450830|57164|2450859|76752|457086|620|13754|76752|457086|620|13754|2|80|1|4|13654|153|773|32|59.42|97.44|34.10|2026.88|1091.20|1901.44|3118.08|65.47|0.00|1496.64|1091.20|1156.67|2587.84|2653.31|-810.24| +2450830|57164|2450849|76752|457086|620|13754|76752|457086|620|13754|2|74|13|2|2998|63|773|100|82.32|190.98|126.04|6494.00|12604.00|8232.00|19098.00|0.00|2646.84|5538.00|9957.16|9957.16|15495.16|15495.16|1725.16| +2450830|57164|2450919|76752|457086|620|13754|76752|457086|620|13754|2|44|15|4|11356|77|773|67|16.44|19.39|16.48|194.97|1104.16|1101.48|1299.13|22.08|0.00|285.42|1104.16|1126.24|1389.58|1411.66|2.68| +2450830|57164|2450907|76752|457086|620|13754|76752|457086|620|13754|2|21|17|1|17725|247|773|90|15.81|47.27|25.52|1957.50|2296.80|1422.90|4254.30|0.00|0.00|1276.20|2296.80|2296.80|3573.00|3573.00|873.90| +2450830|57164|2450896|76752|457086|620|13754|76752|457086|620|13754|2|56|18|4|7276|116|773|80|7.54|13.72|11.66|164.80|932.80|603.20|1097.60|0.00|0.00|208.00|932.80|932.80|1140.80|1140.80|329.60| +2450830|57164|2450837|76752|457086|620|13754|76752|457086|620|13754|2|65|9|1|13706|68|773|39|90.49|167.40|145.63|849.03|5679.57|3529.11|6528.60|227.18|0.00|913.77|5679.57|5906.75|6593.34|6820.52|2150.46| +2450830|79123|2450844|50113|116535|5804|32626|50113|116535|5804|32626|1|77|12|4|13048|293|774|92|86.04|120.45|90.33|2771.04|8310.36|7915.68|11081.40|415.51|0.00|2770.12|8310.36|8725.87|11080.48|11495.99|394.68| +2450830|79123|2450857|50113|116535|5804|32626|50113|116535|5804|32626|1|35|3|3|13292|49|774|11|97.43|139.32|9.75|1425.27|107.25|1071.73|1532.52|9.65|0.00|321.75|107.25|116.90|429.00|438.65|-964.48| +||2450850||116535||32626|||5804|32626|||15|5|553|282|774|96|29.43||66.20||6355.20||8362.56|156.33|3749.56||2605.64||4862.60|5018.93|-219.64| +2450830|79123|2450896|50113|116535|5804|32626|50113|116535|5804|32626|1|49|4|2|14966|151|774|83|51.77|133.56|25.37|8979.77|2105.71|4296.91|11085.48|21.05|0.00|3769.03|2105.71|2126.76|5874.74|5895.79|-2191.20| +2450830|79123|2450882|50113|116535|5804|32626|50113|116535|5804|32626|1|6|5|2|1771|104|774|79|66.75|148.85|72.93|5997.68|5761.47|5273.25|11759.15|230.45|0.00|3880.48|5761.47|5991.92|9641.95|9872.40|488.22| +2450830|79123|2450875|50113|116535|5804|32626|50113|116535|5804|32626|1|90|7|5|9991|173|774|73|74.50|185.50|135.41|3656.57|9884.93|5438.50|13541.50|138.38|7907.94|2979.13|1976.99|2115.37|4956.12|5094.50|-3461.51| +2450830|79123|2450898|50113|116535|5804|32626|50113|116535|5804|32626|1|106|7|1|3650|144|774|94|39.71|115.95|27.82|8284.22|2615.08|3732.74|10899.30|130.75|0.00|4903.98|2615.08|2745.83|7519.06|7649.81|-1117.66| +2450830|79123|2450860|50113|116535|5804|32626|50113|116535|5804|32626|1|81|18|3|1108|38|774|2|82.53|213.75|14.96|397.58|29.92|165.06|427.50|1.79|0.00|188.10|29.92|31.71|218.02|219.81|-135.14| +2450830|38341|2450915|70398|255759|3520|21681|70398|255759|3520|21681|2|76|9|1|8755|295|775|84|81.94|231.07|94.73|11452.56|7957.32|6882.96|19409.88|79.57|0.00|194.04|7957.32|8036.89|8151.36|8230.93|1074.36| +2450830|38341|2450872|70398|255759|3520|21681|70398|255759|3520|21681|2|44|20|3|17935|231|775|56|54.73|122.59|40.45|4599.84|2265.20|3064.88|6865.04|47.11|1087.29|1098.16|1177.91|1225.02|2276.07|2323.18|-1886.97| +2450830|38341|2450902|70398|255759|3520|21681|70398|255759|3520|21681|2|71|12|5|13490|150|775|25|84.87|204.53|147.26|1431.75|3681.50|2121.75|5113.25|184.07|0.00|357.75|3681.50|3865.57|4039.25|4223.32|1559.75| +2450830|38341|2450841|70398|255759|3520|21681|70398|255759|3520|21681|2|107|20|5|17029|131|775|88|66.52|193.57|176.14|1533.84|15500.32|5853.76|17034.16|775.01|0.00|1702.80|15500.32|16275.33|17203.12|17978.13|9646.56| +2450830|38341|2450876|70398|255759|3520|21681|70398|255759|3520|21681|2|14|10|4|2342|203|775|80|63.53|98.47|37.41|4884.80|2992.80|5082.40|7877.60|29.92|0.00|787.20|2992.80|3022.72|3780.00|3809.92|-2089.60| +2450830|38341|2450833|70398|255759|3520|21681|70398|255759|3520|21681|2|7|4|4|3940|75|775|65|73.70|82.54|62.73|1287.65|4077.45|4790.50|5365.10|40.77|0.00|1502.15|4077.45|4118.22|5579.60|5620.37|-713.05| +2450830|38341|2450884|70398|255759|3520|21681|70398|255759|3520|21681|2|55|6|4|15784|32|775|41|32.22|70.56|22.57|1967.59|925.37|1321.02|2892.96|12.21|314.62|1070.10|610.75|622.96|1680.85|1693.06|-710.27| +2450830|38341|2450846|70398|255759|3520|21681|70398|255759|3520|21681|2|13|18|4|9620|122|775|32|79.34|136.46|39.57|3100.48|1266.24|2538.88|4366.72|0.00|0.00|1266.24|1266.24|1266.24|2532.48|2532.48|-1272.64| +2450830|38341|2450838|70398|255759|3520|21681|70398|255759|3520|21681|2|42|13|4|7996|121|775|90|74.00|207.94|174.66|2995.20|15719.40|6660.00|18714.60|1257.55|0.00|186.30|15719.40|16976.95|15905.70|17163.25|9059.40| +2450830|38341|2450839|70398|255759|3520|21681|70398|255759|3520|21681|2|22|4|1|17818|123|775|10|30.95|86.04|47.32|387.20|473.20|309.50|860.40|9.46|0.00|275.30|473.20|482.66|748.50|757.96|163.70| +2450830|38341|2450878|70398|255759|3520|21681|70398|255759|3520|21681|2|84|14|4|12586|136|775|60|21.12|34.21|33.86|21.00|2031.60|1267.20|2052.60|20.31|0.00|923.40|2031.60|2051.91|2955.00|2975.31|764.40| +2450830|38341|2450896|70398|255759|3520|21681|70398|255759|3520|21681|2|30|13|1|1100|94|775|46|86.03|97.21|20.41|3532.80|938.86|3957.38|4471.66|28.16|0.00|491.74|938.86|967.02|1430.60|1458.76|-3018.52| +2450830|38341|2450911|70398|255759|3520|21681|70398|255759|3520|21681|2|48|18|4|7447|199|775|23|30.53|47.62|14.76|755.78|339.48|702.19|1095.26|30.55|0.00|186.07|339.48|370.03|525.55|556.10|-362.71| +2450830|33556|2450868|2941|505613|1438|12510|2941|505613|1438|12510|2|72|4|5|7658|108|776|33|88.35|108.67|95.62|430.65|3155.46|2915.55|3586.11|0.00|0.00|143.22|3155.46|3155.46|3298.68|3298.68|239.91| +2450830|33556|2450840|2941|505613|1438|12510|2941|505613|1438|12510|2|46|10|5|13516|39|776|96|77.68|205.85|174.97|2964.48|16797.12|7457.28|19761.60|839.85|0.00|7311.36|16797.12|17636.97|24108.48|24948.33|9339.84| +2450830|33556|2450841|2941|505613|1438|12510|2941|505613|1438|12510|2|84|19|5|12364|292|776|70|72.58|85.64|53.95|2218.30|3776.50|5080.60|5994.80|302.12|0.00|0.00|3776.50|4078.62|3776.50|4078.62|-1304.10| +2450830|33556|2450910|2941|505613|1438|12510|2941|505613|1438|12510|2|18|7|4|17768|24|776|71|58.67|119.10|10.71|7695.69|760.41|4165.57|8456.10|53.22|0.00|2367.14|760.41|813.63|3127.55|3180.77|-3405.16| +2450830|33556|2450866|2941|505613|1438|12510|2941|505613|1438|12510|2|3|10|4|1576|285|776|27|16.83|41.90|17.17|667.71|463.59|454.41|1131.30|10.19|259.61|441.18|203.98|214.17|645.16|655.35|-250.43| +2450830|33556|2450844|2941|505613|1438|12510|2941|505613|1438|12510|2|100|2|4|17641|136|776|49|70.79|146.53|24.91|5959.38|1220.59|3468.71|7179.97|109.85|0.00|3230.57|1220.59|1330.44|4451.16|4561.01|-2248.12| +2450830|33556|2450879|2941|505613|1438|12510|2941|505613|1438|12510|2|34|4|1|17710|52|776|68|91.29|190.79|146.90|2984.52|9989.20|6207.72|12973.72|599.35|0.00|4281.28|9989.20|10588.55|14270.48|14869.83|3781.48| +2450830|2415|2450861|48512|216157|3802|33847|34694|1258737|1582|47627|1|48|8|5|8821|166|777|52|88.69|109.08|40.35|3573.96|2098.20|4611.88|5672.16|52.87|1510.70|907.40|587.50|640.37|1494.90|1547.77|-4024.38| +2450830|2415|2450865|48512|216157|3802|33847|34694|1258737|1582|47627|1|32|13|2|8366|200|777|6|53.48|55.61|50.60|30.06|303.60|320.88|333.66|24.28|0.00|140.10|303.60|327.88|443.70|467.98|-17.28| +2450830|2415|2450879|48512|216157|3802|33847|34694|1258737|1582|47627|1|20|14|1|4912|110|777|23|95.10|100.80|0.00|2318.40|0.00|2187.30|2318.40|0.00|0.00|417.22|0.00|0.00|417.22|417.22|-2187.30| +2450830|2415|2450877|48512|216157|3802|33847|34694|1258737|1582|47627|1|90|2|4|14942|213|777|35|31.57|49.56|28.24|746.20|988.40|1104.95|1734.60|40.03|187.79|710.85|800.61|840.64|1511.46|1551.49|-304.34| +2450830|2415|2450866|48512|216157|3802|33847|34694|1258737|1582|47627|1|41|19|5|13027|173|777|2|15.30|28.45|0.56|55.78|1.12|30.60|56.90|0.10|0.00|27.30|1.12|1.22|28.42|28.52|-29.48| +2450830|2415|2450885|48512|216157|3802|33847|34694|1258737|1582|47627|1|107|16|4|1159|57|777|41|58.01|158.94|114.43|1824.91|4691.63|2378.41|6516.54|375.33|0.00|2410.80|4691.63|5066.96|7102.43|7477.76|2313.22| +2450830|2415|2450847|48512|216157|3802|33847|34694|1258737|1582|47627|1|40|10|1|10018|160|777|56|1.97|2.34|1.82|29.12|101.92|110.32|131.04|9.17|0.00|53.20|101.92|111.09|155.12|164.29|-8.40| +2450830|2415|2450834|48512|216157|3802|33847|34694|1258737|1582|47627|1|33|19|1|17398|262|777|67|73.73|140.08|81.24|3942.28|5443.08|4939.91|9385.36|272.15|0.00|1689.07|5443.08|5715.23|7132.15|7404.30|503.17| +2450830|2415|2450869|48512|216157|3802|33847|34694|1258737|1582|47627|1|16|2|3|9988|104|777|19|1.67|3.03|1.96|20.33|37.24|31.73|57.57|2.97|0.00|26.41|37.24|40.21|63.65|66.62|5.51| +2450830|2415|2450895|48512|216157|3802|33847|34694|1258737|1582|47627|1|105|3|4|6266|290|777|47|7.06|9.03|1.08|373.65|50.76|331.82|424.41|1.32|36.03|211.97|14.73|16.05|226.70|228.02|-317.09| +2450830|2415|2450884|48512|216157|3802|33847|34694|1258737|1582|47627|1|88|9|2|5462|48|777|80|99.12|165.53|82.76|6621.60|6620.80|7929.60|13242.40|463.45|0.00|3045.60|6620.80|7084.25|9666.40|10129.85|-1308.80| +||||||42441||1612458|6073||||||8110||778|54|49.97|||5692.68|2105.46|||||623.70|2105.46||2729.16|2876.54|-592.92| +2450830|74592|2450852|47643|1612458|6073|42441|47643|1612458|6073|42441|2|107|6|3|10310|189|778|68|75.99|129.94|111.74|1237.60|7598.32|5167.32|8835.92|531.88|0.00|1943.44|7598.32|8130.20|9541.76|10073.64|2431.00| +2450830|74592|2450913|47643|1612458|6073|42441|47643|1612458|6073|42441|2|102|3|3|9350|6|778|70|11.34|29.59|25.15|310.80|1760.50|793.80|2071.30|4.92|1637.26|310.10|123.24|128.16|433.34|438.26|-670.56| +2450830|74592|2450900|47643|1612458|6073|42441|47643|1612458|6073|42441|2|80|12|5|355|161|778|21|85.21|185.75|70.58|2418.57|1482.18|1789.41|3900.75|15.56|1259.85|546.00|222.33|237.89|768.33|783.89|-1567.08| +2450830|56792|2450858|91773|246789|6605|32821|91773|246789|6605|32821|4|57|1|3|2806|35|779|10|91.49|228.72|141.80|869.20|1418.00|914.90|2287.20|70.90|0.00|686.10|1418.00|1488.90|2104.10|2175.00|503.10| +2450830|56792|2450868|91773|246789|6605|32821|91773|246789|6605|32821|4|60|6|5|9895|28|779|94|24.30|32.07|23.73|783.96|2230.62|2284.20|3014.58|111.53|0.00|572.46|2230.62|2342.15|2803.08|2914.61|-53.58| +2450830|56792|2450917|91773|246789|6605|32821|91773|246789|6605|32821|4|13|15|4|5426|114|779|58|95.02|198.59|37.73|9329.88|2188.34|5511.16|11518.22|43.76|0.00|4261.26|2188.34|2232.10|6449.60|6493.36|-3322.82| +2450830|56792|2450920|91773|246789|6605|32821|91773|246789|6605|32821|4|94|10|4|16093|129|779|17|81.88|180.13|127.89|888.08|2174.13|1391.96|3062.21|173.93|0.00|1500.42|2174.13|2348.06|3674.55|3848.48|782.17| +2450830|23074|2450859|90918|396421|3971|31921|90918|396421|3971|31921|1|53|14|5|3481|94|780|24|41.02|63.58|34.33|702.00|823.92|984.48|1525.92|65.91|0.00|579.84|823.92|889.83|1403.76|1469.67|-160.56| +2450830|23074|2450915|90918|396421|3971|31921|90918|396421|3971|31921|1|7|7|1|7688|83|780|12|52.04|118.65|78.30|484.20|939.60|624.48|1423.80|0.00|0.00|370.08|939.60|939.60|1309.68|1309.68|315.12| +2450830|23074|2450894|90918|396421|3971|31921|90918|396421|3971|31921|1|46|12|1|10309|217|780|37|64.84|81.69|35.12|1723.09|1299.44|2399.08|3022.53|116.94|0.00|725.20|1299.44|1416.38|2024.64|2141.58|-1099.64| +2450830|23074|2450873|90918|396421|3971|31921|90918|396421|3971|31921|1|44|18|2|11377|123|780|15|24.63|57.14|5.71|771.45|85.65|369.45|857.10|7.70|0.00|317.10|85.65|93.35|402.75|410.45|-283.80| +2450830|23074|2450834|90918|396421|3971|31921|90918|396421|3971|31921|1|47|20|1|14146|266|780|92|46.97|136.21|132.12|376.28|12155.04|4321.24|12531.32|121.55|0.00|4887.04|12155.04|12276.59|17042.08|17163.63|7833.80| +2450830|23074|2450894|90918|396421|3971|31921|90918|396421|3971|31921|1|7|17|3|13594|290|780|95|55.14|156.04|4.68|14379.20|444.60|5238.30|14823.80|26.67|0.00|1926.60|444.60|471.27|2371.20|2397.87|-4793.70| +2450830|23074|2450899|90918|396421|3971|31921|90918|396421|3971|31921|1|21|16|3|17377|36|780|13|26.20|72.31|39.77|423.02|517.01|340.60|940.03|46.53|0.00|197.34|517.01|563.54|714.35|760.88|176.41| +2450830|23074|2450905|90918|396421|3971|31921|90918|396421|3971|31921|1|84|9|1|10546|51|780|71|28.51|78.97|7.10|5102.77|504.10|2024.21|5606.87|15.12|0.00|1738.08|504.10|519.22|2242.18|2257.30|-1520.11| +2450830|83673|2450856|98695|977788|4434|20327|94190|242866|712|43405|4|22|3|4|16768|233|781|58|95.46|105.00|10.50|5481.00|609.00|5536.68|6090.00|32.39|146.16|913.50|462.84|495.23|1376.34|1408.73|-5073.84| +2450830|83673|2450857|98695|977788|4434|20327|94190|242866|712|43405|4|37|12|4|8134|266|781|56|94.47|143.59|110.56|1849.68|6191.36|5290.32|8041.04|0.00|0.00|2572.64|6191.36|6191.36|8764.00|8764.00|901.04| +2450830|83673|2450832|98695|977788|4434|20327|94190|242866|712|43405|4|15|3|1|2368|297|781|21|82.77|200.30|54.08|3070.62|1135.68|1738.17|4206.30|4.76|658.69|420.63|476.99|481.75|897.62|902.38|-1261.18| +2450830|83673|2450918|98695|977788|4434|20327|94190|242866|712|43405|4|33|4|4|16318|12|781|12|12.12|29.45|10.30|229.80|123.60|145.44|353.40|8.65|0.00|116.52|123.60|132.25|240.12|248.77|-21.84| +2450830|60182|2450867|41838|427299|3960|16221|41838|427299|3960|16221|1|3|16|1|15356|221|782|1|8.09|9.62|3.75|5.87|3.75|8.09|9.62|0.22|0.00|2.98|3.75|3.97|6.73|6.95|-4.34| +2450830|60182|2450847|41838|427299|3960|16221|41838|427299|3960|16221|1|65|3|1|9154|54|782|51|71.05|71.76|20.09|2635.17|1024.59|3623.55|3659.76|20.49|0.00|328.95|1024.59|1045.08|1353.54|1374.03|-2598.96| +2450830|60182|2450909|41838|427299|3960|16221|41838|427299|3960|16221|1|76|18|4|4705|105|782|13|32.15|65.90|32.29|436.93|419.77|417.95|856.70|2.09|314.82|42.77|104.95|107.04|147.72|149.81|-313.00| +2450830|60182|2450834|41838|427299|3960|16221|41838|427299|3960|16221|1|5|4|5|17360|4|782|5|84.17|244.93|7.34|1187.95|36.70|420.85|1224.65|2.56|0.00|293.90|36.70|39.26|330.60|333.16|-384.15| +2450830|60182|2450885|41838|427299|3960|16221|41838|427299|3960|16221|1|6|5|3|17024|201|782|21|41.59|81.51|79.06|51.45|1660.26|873.39|1711.71|0.00|265.64|530.46|1394.62|1394.62|1925.08|1925.08|521.23| +2450830|60182|2450879|41838|427299|3960|16221|41838|427299|3960|16221|1|108|6|2|8548|149|782|53|79.78|97.33|4.86|4900.91|257.58|4228.34|5158.49|7.18|177.73|308.99|79.85|87.03|388.84|396.02|-4148.49| +2450830|60182|2450863|41838|427299|3960|16221|41838|427299|3960|16221|1|50|16|2|3466|6|782|97|50.70|94.30|24.51|6769.63|2377.47|4917.90|9147.10|0.00|0.00|2744.13|2377.47|2377.47|5121.60|5121.60|-2540.43| +2450830|60182|2450873|41838|427299|3960|16221|41838|427299|3960|16221|1|85|14|5|9754|261|782|6|13.51|37.28|19.01|109.62|114.06|81.06|223.68|2.28|0.00|0.00|114.06|116.34|114.06|116.34|33.00| +2450830|61476|2450911|83082|221177|72|33414|83082|221177|72|33414|2|60|10|1|12620|25|783|41|32.82|39.71|31.76|325.95|1302.16|1345.62|1628.11|33.33|885.46|276.75|416.70|450.03|693.45|726.78|-928.92| +2450830|61476|2450887|83082|221177|72|33414|83082|221177|72|33414|2|48|18|2|6080|185|783|8|66.36|181.16|94.20|695.68|753.60|530.88|1449.28|7.53|0.00|318.80|753.60|761.13|1072.40|1079.93|222.72| +2450830|61476|2450884|83082|221177|72|33414|83082|221177|72|33414|2|54|1|2|2569|169|783|13|70.33|115.34|64.59|659.75|839.67|914.29|1499.42|8.39|0.00|629.72|839.67|848.06|1469.39|1477.78|-74.62| +2450830|61476|2450904|83082|221177|72|33414|83082|221177|72|33414|2|42|12|3|4762|257|783|30|28.28|80.88|5.66|2256.60|169.80|848.40|2426.40|3.39|0.00|194.10|169.80|173.19|363.90|367.29|-678.60| +2450830|61476|2450837|83082|221177|72|33414|83082|221177|72|33414|2|102|5|2|15406|240|783|18|62.28|155.07|65.12|1619.10|1172.16|1121.04|2791.26|46.88|0.00|558.18|1172.16|1219.04|1730.34|1777.22|51.12| +2450830|61476|2450834|83082|221177|72|33414|83082|221177|72|33414|2|33|20|5|6206|80|783|82|93.14|257.06|257.06|0.00|21078.92|7637.48|21078.92|843.15|0.00|2528.88|21078.92|21922.07|23607.80|24450.95|13441.44| +2450830|61476|2450901|83082|221177|72|33414|83082|221177|72|33414|2|51|8|5|14815|54|783|90|8.69|14.59|2.18|1116.90|196.20|782.10|1313.10|3.41|139.30|235.80|56.90|60.31|292.70|296.11|-725.20| +2450830|45595|2450912|20130|1776167|6604|45041|20130|1776167|6604|45041|1|77|4|1|4736|218|784|36|64.57|111.06|77.74|1199.52|2798.64|2324.52|3998.16|0.00|475.76|1878.84|2322.88|2322.88|4201.72|4201.72|-1.64| +2450830|45595|2450916|20130|1776167|6604|45041|20130|1776167|6604|45041|1|81|7|2|2560|237|784|59|29.17|56.00|10.08|2709.28|594.72|1721.03|3304.00|17.84|0.00|859.04|594.72|612.56|1453.76|1471.60|-1126.31| +2450830|45595|2450832|20130|1776167|6604|45041|20130|1776167|6604|45041|1|103|20|3|17270|85|784|61|80.14|143.45|64.55|4812.90|3937.55|4888.54|8750.45|170.89|1496.26|699.67|2441.29|2612.18|3140.96|3311.85|-2447.25| +2450830|45595|2450914|20130|1776167|6604|45041|20130|1776167|6604|45041|1|71|15|1|9656|8|784|97|87.50|214.37|92.17|11853.40|8940.49|8487.50|20793.89|200.26|3933.81|9356.62|5006.68|5206.94|14363.30|14563.56|-3480.82| +2450830|45595|2450891|20130|1776167|6604|45041|20130|1776167|6604|45041|1|79|12|1|6298|116|784|67|40.81|68.96|2.75|4436.07|184.25|2734.27|4620.32|5.52|0.00|276.71|184.25|189.77|460.96|466.48|-2550.02| +2450830|45595|2450884|20130|1776167|6604|45041|20130|1776167|6604|45041|1|42|16|5|9718|255|784|83|31.48|60.44|13.90|3862.82|1153.70|2612.84|5016.52|11.53|0.00|1203.50|1153.70|1165.23|2357.20|2368.73|-1459.14| +2450830|45595|2450863|20130|1776167|6604|45041|20130|1776167|6604|45041|1|43|17|5|11782|251|784|100|1.53|4.49|3.99|50.00|399.00|153.00|449.00|3.99|0.00|161.00|399.00|402.99|560.00|563.99|246.00| +2450830|45595|2450873|20130|1776167|6604|45041|20130|1776167|6604|45041|1|96|19|2|14368|278|784|14|29.99|54.28|10.85|608.02|151.90|419.86|759.92|3.03|0.00|243.04|151.90|154.93|394.94|397.97|-267.96| +2450830|45595|2450856|20130|1776167|6604|45041|20130|1776167|6604|45041|1|72|4|1|2062|53|784|66|48.26|87.35|24.45|4151.40|1613.70|3185.16|5765.10|129.09|0.00|633.60|1613.70|1742.79|2247.30|2376.39|-1571.46| +2450830|50299|2450848|96681|1890847|3766|37885|96681|1890847|3766|37885|2|79|4|4|16216|52|785|35|95.88|154.36|97.24|1999.20|3403.40|3355.80|5402.60|154.85|306.30|1566.60|3097.10|3251.95|4663.70|4818.55|-258.70| +2450830|50299|2450843|96681|1890847|3766|37885|96681|1890847|3766|37885|2|38|10|4|2701|251|785|94|69.32|161.51|72.67|8350.96|6830.98|6516.08|15181.94|68.30|0.00|1062.20|6830.98|6899.28|7893.18|7961.48|314.90| +2450830|50299|2450863|96681|1890847|3766|37885|96681|1890847|3766|37885|2|44|11|4|10972|265|785|36|35.12|38.98|12.86|940.32|462.96|1264.32|1403.28|18.51|0.00|659.52|462.96|481.47|1122.48|1140.99|-801.36| +2450830|50299|2450890|96681|1890847|3766|37885|96681|1890847|3766|37885|2|82|5|1|3064|37|785|63|10.81|26.91|21.52|339.57|1355.76|681.03|1695.33|19.52|867.68|728.91|488.08|507.60|1216.99|1236.51|-192.95| +2450830|64993|2450865|8539|889851|3163|24875|8539|889851|3163|24875|2|86|9|5|3182|130|786|4|6.02|9.09|6.72|9.48|26.88|24.08|36.36|0.53|0.00|9.08|26.88|27.41|35.96|36.49|2.80| +2450830|64993|2450890|8539|889851|3163|24875|8539|889851|3163|24875|2|33|20|1|5233|93|786|84|32.56|48.84|43.95|410.76|3691.80|2735.04|4102.56|73.83|0.00|1722.84|3691.80|3765.63|5414.64|5488.47|956.76| +2450830|64993|2450906|8539|889851|3163|24875|8539|889851|3163|24875|2|78|2|1|9782|12|786|17|91.86|133.19|90.56|724.71|1539.52|1561.62|2264.23|30.79|0.00|1086.81|1539.52|1570.31|2626.33|2657.12|-22.10| +2450830|64993|2450869|8539|889851|3163|24875|8539|889851|3163|24875|2|23|17|3|1756|115|786|95|6.20|14.07|2.67|1083.00|253.65|589.00|1336.65|17.75|0.00|186.20|253.65|271.40|439.85|457.60|-335.35| +2450830|64993|2450832|8539|889851|3163|24875|8539|889851|3163|24875|2|66|2|4|11041|55|786|20|53.66|77.80|76.24|31.20|1524.80|1073.20|1556.00|137.23|0.00|606.80|1524.80|1662.03|2131.60|2268.83|451.60| +2450830|64993|2450897|8539|889851|3163|24875|8539|889851|3163|24875|2|48|7|5|3892|86|786|15|72.04|78.52|58.10|306.30|871.50|1080.60|1177.80|69.72|0.00|270.75|871.50|941.22|1142.25|1211.97|-209.10| +2450830|64993|2450849|8539|889851|3163|24875|8539|889851|3163|24875|2|48|16|5|121|6|786|84|81.19|83.62|28.43|4635.96|2388.12|6819.96|7024.08|0.00|167.16|1053.36|2220.96|2220.96|3274.32|3274.32|-4599.00| +2450830|64993|2450883|8539|889851|3163|24875|8539|889851|3163|24875|2|23|3|2|11422|95|786|99|68.24|166.50|79.92|8571.42|7912.08|6755.76|16483.50|712.08|0.00|823.68|7912.08|8624.16|8735.76|9447.84|1156.32| +2450830|64993|2450858|8539|889851|3163|24875|8539|889851|3163|24875|2|1|17|5|16159|169|786|26|75.14|219.40|30.71|4905.94|798.46|1953.64|5704.40|39.92|0.00|1768.26|798.46|838.38|2566.72|2606.64|-1155.18| +2450830|64993|2450912|8539|889851|3163|24875|8539|889851|3163|24875|2|38|15|2|11509|96|786|27|40.07|86.15|6.03|2163.24|162.81|1081.89|2326.05|8.14|0.00|1046.52|162.81|170.95|1209.33|1217.47|-919.08| +2450830|64993|2450870|8539|889851|3163|24875|8539|889851|3163|24875|2|57|15|5|17599|272|786|14|51.10|113.95|19.37|1324.12|271.18|715.40|1595.30|18.98|0.00|0.00|271.18|290.16|271.18|290.16|-444.22| +2450830|64993|2450842|8539|889851|3163|24875|8539|889851|3163|24875|2|104|16|2|3112|88|786|91|33.09|59.89|44.91|1363.18|4086.81|3011.19|5449.99|163.47|0.00|544.18|4086.81|4250.28|4630.99|4794.46|1075.62| +2450830|64993|2450896|8539|889851|3163|24875|8539|889851|3163|24875|2|16|5|2|757|259|786|78|63.47|88.85|56.86|2495.22|4435.08|4950.66|6930.30|399.15|0.00|2425.02|4435.08|4834.23|6860.10|7259.25|-515.58| +2450830|64993|2450864|8539|889851|3163|24875|8539|889851|3163|24875|2|51|18|1|11618|108|786|2|47.17|125.47|116.68|17.58|233.36|94.34|250.94|7.00|0.00|100.36|233.36|240.36|333.72|340.72|139.02| +2450830|36453|2450849|40025|1316193|4008|10176|40025|1316193|4008|10176|1|7|2|3|12640|293|787|1|86.95|251.28|12.56|238.72|12.56|86.95|251.28|1.00|0.00|110.56|12.56|13.56|123.12|124.12|-74.39| +2450830|36453|2450872|40025|1316193|4008|10176|40025|1316193|4008|10176|1|45|3|5|13006|104|787|51|96.29|129.02|76.12|2697.90|3882.12|4910.79|6580.02|38.82|0.00|1250.01|3882.12|3920.94|5132.13|5170.95|-1028.67| +2450830|36453|2450870|40025|1316193|4008|10176|40025|1316193|4008|10176|1|66|19|2|5060|146|787|57|35.93|59.28|16.00|2466.96|912.00|2048.01|3378.96|45.60|0.00|0.00|912.00|957.60|912.00|957.60|-1136.01| +2450830|36453|2450854|40025|1316193|4008|10176|40025|1316193|4008|10176|1|17|17|5|7492|94|787|45|90.25|218.40|163.80|2457.00|7371.00|4061.25|9828.00|73.71|0.00|3341.25|7371.00|7444.71|10712.25|10785.96|3309.75| +2450830|36453|2450907|40025|1316193|4008|10176|40025|1316193|4008|10176|1|13|11|3|14846|25|787|88|89.68|238.54|19.08|19312.48|1679.04|7891.84|20991.52|117.53|0.00|3568.40|1679.04|1796.57|5247.44|5364.97|-6212.80| +2450830|36453|2450864|40025|1316193|4008|10176|40025|1316193|4008|10176|1|107|18|3|14800|273|787|58|20.75|48.34|24.65|1374.02|1429.70|1203.50|2803.72|14.29|0.00|0.00|1429.70|1443.99|1429.70|1443.99|226.20| +2450830|55826|2450862|98486|1917630|3621|26765|98486|1917630|3621|26765|2|9|2|4|950|261|788|5|18.61|38.33|36.79|7.70|183.95|93.05|191.65|9.19|0.00|24.90|183.95|193.14|208.85|218.04|90.90| +2450830|55826|2450850|98486|1917630|3621|26765|98486|1917630|3621|26765|2|61|16|2|9794|2|788|54|46.45|78.03|35.89|2275.56|1938.06|2508.30|4213.62|0.00|0.00|463.32|1938.06|1938.06|2401.38|2401.38|-570.24| +2450830|55826|2450916|98486|1917630|3621|26765|98486|1917630|3621|26765|2|99|18|4|12955|31|788|45|69.41|97.17|29.15|3060.90|1311.75|3123.45|4372.65|104.94|0.00|1136.70|1311.75|1416.69|2448.45|2553.39|-1811.70| +2450830|55826|2450861|98486|1917630|3621|26765|98486|1917630|3621|26765|2|34|17|4|13624|254|788|13|10.07|20.24|11.94|107.90|155.22|130.91|263.12|10.86|0.00|41.99|155.22|166.08|197.21|208.07|24.31| +2450830|63825|2450861|73885|653384|1330|21629|73885|653384|1330|21629|4|47|10|2|10987|199|789|39|10.35|26.18|12.30|541.32|479.70|403.65|1021.02|33.57|0.00|142.74|479.70|513.27|622.44|656.01|76.05| +2450830|63825|2450863|73885|653384|1330|21629|73885|653384|1330|21629|4|1|9|3|11884|59|789|85|75.76|173.49|65.92|9143.45|5603.20|6439.60|14746.65|56.03|4202.40|5456.15|1400.80|1456.83|6856.95|6912.98|-5038.80| +2450830|63825|2450867|73885|653384|1330|21629|73885|653384|1330|21629|4|30|20|2|9664|39|789|42|41.23|45.35|11.33|1428.84|475.86|1731.66|1904.70|42.82|0.00|742.56|475.86|518.68|1218.42|1261.24|-1255.80| +2450830|63825|2450905|73885|653384|1330|21629|73885|653384|1330|21629|4|3|1|1|7700|195|789|17|26.99|27.79|25.01|47.26|425.17|458.83|472.43|5.95|276.36|113.22|148.81|154.76|262.03|267.98|-310.02| +2450830|63825|2450851|73885|653384|1330|21629|73885|653384|1330|21629|4|86|5|3|13946|181|789|50|67.35|113.14|62.22|2546.00|3111.00|3367.50|5657.00|0.00|0.00|2375.50|3111.00|3111.00|5486.50|5486.50|-256.50| +2450830|63825|2450857|73885|653384|1330|21629|73885|653384|1330|21629|4|21|10|1|14336|283|789|36|67.09|194.56|126.46|2451.60|4552.56|2415.24|7004.16|91.05|0.00|910.44|4552.56|4643.61|5463.00|5554.05|2137.32| +2450830|63825|2450898|73885|653384|1330|21629|73885|653384|1330|21629|4|1|1|3|10088|47|789|30|99.22|120.05|40.81|2377.20|1224.30|2976.60|3601.50|30.85|195.88|1728.60|1028.42|1059.27|2757.02|2787.87|-1948.18| +2450830|63825|2450873|73885|653384|1330|21629|73885|653384|1330|21629|4|10|6|1|15914|143|789|11|45.99|99.33|0.00|1092.63|0.00|505.89|1092.63|0.00|0.00|174.79|0.00|0.00|174.79|174.79|-505.89| +2450830|63825|2450861|73885|653384|1330|21629|73885|653384|1330|21629|4|33|15|4|10978|250|789|17|48.10|93.31|42.92|856.63|729.64|817.70|1586.27|36.48|0.00|95.03|729.64|766.12|824.67|861.15|-88.06| +2450830|15712|2450861|42668|1507764|1307|43603|42668|1507764|1307|43603|1|12|15|1|7351|267|790|80|34.64|37.06|8.15|2312.80|652.00|2771.20|2964.80|6.52|0.00|1126.40|652.00|658.52|1778.40|1784.92|-2119.20| +2450830|15712|2450862|42668|1507764|1307|43603|42668|1507764|1307|43603|1|101|20|4|12392|277|790|45|55.72|61.29|49.03|551.70|2206.35|2507.40|2758.05|40.37|860.47|441.00|1345.88|1386.25|1786.88|1827.25|-1161.52| +2450830|15712|2450906|42668|1507764|1307|43603|42668|1507764|1307|43603|1|61|18|2|8954|10|790|43|84.85|208.73|83.49|5385.32|3590.07|3648.55|8975.39|30.15|2584.85|4308.17|1005.22|1035.37|5313.39|5343.54|-2643.33| +2450830|15712|2450877|42668|1507764|1307|43603|42668|1507764|1307|43603|1|3|5|3|15457|34|790|72|96.20|216.45|194.80|1558.80|14025.60|6926.40|15584.40|981.79|0.00|5922.00|14025.60|15007.39|19947.60|20929.39|7099.20| +2450830|41124|2450832|48210|1712628|2646|7417|48210|1712628|2646|7417|2|98|3|1|2638|220|791|77|32.37|46.28|1.38|3457.30|106.26|2492.49|3563.56|1.06|0.00|1496.11|106.26|107.32|1602.37|1603.43|-2386.23| +2450830|41124|2450873|48210|1712628|2646|7417|48210|1712628|2646|7417|2|102|19|1|5884|195|791|30|97.03|174.65|54.14|3615.30|1624.20|2910.90|5239.50|64.96|0.00|942.90|1624.20|1689.16|2567.10|2632.06|-1286.70| +2450830|41124|2450834|48210|1712628|2646|7417|48210|1712628|2646|7417|2|48|15|2|4045|39|791|80|41.42|96.09|76.87|1537.60|6149.60|3313.60|7687.20|368.97|0.00|768.00|6149.60|6518.57|6917.60|7286.57|2836.00| +2450830|41124|2450914|48210|1712628|2646|7417|48210|1712628|2646|7417|2|27|11|4|253|57|791|67|86.86|259.71|218.15|2784.52|14616.05|5819.62|17400.57|727.87|2484.72|6438.03|12131.33|12859.20|18569.36|19297.23|6311.71| +2450830|41124|2450856|48210|1712628|2646|7417|48210|1712628|2646|7417|2|98|1|2|1492|119|791|65|47.16|93.37|29.87|4127.50|1941.55|3065.40|6069.05|77.66|0.00|546.00|1941.55|2019.21|2487.55|2565.21|-1123.85| +2450830|41124|2450887|48210|1712628|2646|7417|48210|1712628|2646|7417|2|81|2|1|14932|145|791|27|72.92|184.48|66.41|3187.89|1793.07|1968.84|4980.96|143.44|0.00|198.99|1793.07|1936.51|1992.06|2135.50|-175.77| +2450830|41124|2450915|48210|1712628|2646|7417|48210|1712628|2646|7417|2|54|5|5|9134|85|791|14|85.02|200.64|52.16|2078.72|730.24|1190.28|2808.96|7.30|0.00|926.94|730.24|737.54|1657.18|1664.48|-460.04| +2450830|41124|2450879|48210|1712628|2646|7417|48210|1712628|2646|7417|2|13|4|3|9865|161|791|92|54.22|112.23|84.17|2581.52|7743.64|4988.24|10325.16|542.05|0.00|5162.12|7743.64|8285.69|12905.76|13447.81|2755.40| +2450830|41124|2450848|48210|1712628|2646|7417|48210|1712628|2646|7417|2|21|6|1|10448|243|791|74|34.32|88.88|61.32|2039.44|4537.68|2539.68|6577.12|272.26|0.00|3025.12|4537.68|4809.94|7562.80|7835.06|1998.00| +2450830|72532|2450908|19329|164721|4513|41552|19329|164721|4513|41552|4|73|5|1|2768|131|792|40|68.84|188.62|5.65|7318.80|226.00|2753.60|7544.80|11.30|0.00|1508.80|226.00|237.30|1734.80|1746.10|-2527.60| +2450830|72532|2450852|19329|164721|4513|41552|19329|164721|4513|41552|4|1|11|5|6098|61|792|77|34.67|74.88|73.38|115.50|5650.26|2669.59|5765.76|452.02|0.00|1498.42|5650.26|6102.28|7148.68|7600.70|2980.67| +2450830|72532|2450843|19329|164721|4513|41552|19329|164721|4513|41552|4|62|16|3|15418|262|792|24|24.32|33.56|1.67|765.36|40.08|583.68|805.44|2.80|0.00|168.96|40.08|42.88|209.04|211.84|-543.60| +2450830|72532|2450901|19329|164721|4513|41552|19329|164721|4513|41552|4|103|15|3|4840|211|792|13|10.46|23.43|18.97|57.98|246.61|135.98|304.59|17.26|0.00|15.21|246.61|263.87|261.82|279.08|110.63| +2450830|72532|2450913|19329|164721|4513|41552|19329|164721|4513|41552|4|104|12|5|11653|279|792|1|46.58|80.58|67.68|12.90|67.68|46.58|80.58|0.00|0.00|11.28|67.68|67.68|78.96|78.96|21.10| +2450830|72532|2450898|19329|164721|4513|41552|19329|164721|4513|41552|4|91|4|5|7402|10|792|13|90.16|264.16|105.66|2060.50|1373.58|1172.08|3434.08|82.41|0.00|1339.26|1373.58|1455.99|2712.84|2795.25|201.50| +2450830|72532|2450870|19329|164721|4513|41552|19329|164721|4513|41552|4|103|10|5|4600|232|792|33|63.77|162.61|78.05|2790.48|2575.65|2104.41|5366.13|77.26|0.00|2575.65|2575.65|2652.91|5151.30|5228.56|471.24| +2450830|72532|2450905|19329|164721|4513|41552|19329|164721|4513|41552|4|96|20|1|6619|196|792|19|50.81|57.41|45.92|218.31|872.48|965.39|1090.79|61.07|0.00|436.24|872.48|933.55|1308.72|1369.79|-92.91| +2450830|72532|2450909|19329|164721|4513|41552|19329|164721|4513|41552|4|20|2|1|7822|9|792|91|70.76|174.77|33.20|12882.87|3021.20|6439.16|15904.07|30.21|0.00|953.68|3021.20|3051.41|3974.88|4005.09|-3417.96| +2450830|72532|2450884|19329|164721|4513|41552|19329|164721|4513|41552|4|83|16|5|14014|148|792|40|14.59|34.87|21.61|530.40|864.40|583.60|1394.80|51.86|0.00|530.00|864.40|916.26|1394.40|1446.26|280.80| +2450830|72532|2450902|19329|164721|4513|41552|19329|164721|4513|41552|4|71|18|1|16879|263|792|16|56.29|69.23|64.38|77.60|1030.08|900.64|1107.68|41.20|0.00|121.76|1030.08|1071.28|1151.84|1193.04|129.44| +2450830|72532|2450835|19329|164721|4513|41552|19329|164721|4513|41552|4|64|15|1|3091|175|792|9|24.76|26.24|16.53|87.39|148.77|222.84|236.16|5.95|0.00|40.14|148.77|154.72|188.91|194.86|-74.07| +2450830|43446|2450905|69670|1824622|727|45279|69670|1824622|727|45279|1|78|17|2|13273|152|793|25|56.47|99.38|18.88|2012.50|472.00|1411.75|2484.50|14.16|0.00|670.75|472.00|486.16|1142.75|1156.91|-939.75| +2450830|43446|2450907|69670|1824622|727|45279|69670|1824622|727|45279|1|76|12|5|13369|53|793|55|30.22|52.58|21.55|1706.65|1185.25|1662.10|2891.90|33.18|521.51|1329.90|663.74|696.92|1993.64|2026.82|-998.36| +2450830|43446|2450894|69670|1824622|727|45279|69670|1824622|727|45279|1|33|9|5|16474|82|793|78|67.45|118.03|42.49|5892.12|3314.22|5261.10|9206.34|231.99|0.00|828.36|3314.22|3546.21|4142.58|4374.57|-1946.88| +2450830|43446|2450835|69670|1824622|727|45279|69670|1824622|727|45279|1|17|14|3|5938|98|793|19|92.52|220.19|28.62|3639.83|543.78|1757.88|4183.61|5.43|0.00|1505.94|543.78|549.21|2049.72|2055.15|-1214.10| +2450830|43446|2450843|69670|1824622|727|45279|69670|1824622|727|45279|1|31|18|2|11389|128|793|14|81.36|178.99|150.35|400.96|2104.90|1139.04|2505.86|21.04|0.00|1227.80|2104.90|2125.94|3332.70|3353.74|965.86| +2450830|43446|2450914|69670|1824622|727|45279|69670|1824622|727|45279|1|83|7|5|13730|106|793|76|82.15|112.54|19.13|7099.16|1453.88|6243.40|8553.04|116.31|0.00|684.00|1453.88|1570.19|2137.88|2254.19|-4789.52| +2450830|43446|2450839|69670|1824622|727|45279|69670|1824622|727|45279|1|48|10|5|3649|103|793|60|23.47|65.48|0.65|3889.80|39.00|1408.20|3928.80|1.24|7.80|78.00|31.20|32.44|109.20|110.44|-1377.00| +2450830|43446|2450903|69670|1824622|727|45279|69670|1824622|727|45279|1|16|3|1|11432|225|793|36|81.58|121.55|23.09|3544.56|831.24|2936.88|4375.80|33.24|0.00|1531.44|831.24|864.48|2362.68|2395.92|-2105.64| +2450830|16628|2450887|28100|242325|6950|7575|28100|242325|6950|7575|2|19|7|2|9418|238|794|71|77.50|212.35|114.66|6935.99|8140.86|5502.50|15076.85|488.45|0.00|904.54|8140.86|8629.31|9045.40|9533.85|2638.36| +2450830|16628|2450905|28100|242325|6950|7575|28100|242325|6950|7575|2|36|20|4|16870|14|794|100|72.26|127.90|28.13|9977.00|2813.00|7226.00|12790.00|0.00|0.00|2302.00|2813.00|2813.00|5115.00|5115.00|-4413.00| +2450830|16628|2450859|28100|242325|6950|7575|28100|242325|6950|7575|2|37|1|2|1255|293|794|13|24.44|35.19|20.41|192.14|265.33|317.72|457.47|2.65|0.00|150.93|265.33|267.98|416.26|418.91|-52.39| +2450830|16628|2450895|28100|242325|6950|7575|28100|242325|6950|7575|2|88|1|1|9752|203|794|58|79.92|110.28|26.46|4861.56|1534.68|4635.36|6396.24|30.69|0.00|1151.30|1534.68|1565.37|2685.98|2716.67|-3100.68| +2450830|16628|2450892|28100|242325|6950|7575|28100|242325|6950|7575|2|96|19|1|3272|198|794|28|35.35|48.42|17.43|867.72|488.04|989.80|1355.76|8.39|278.18|108.36|209.86|218.25|318.22|326.61|-779.94| +2450830|16628|2450889|28100|242325|6950|7575|28100|242325|6950|7575|2|65|10|2|4988|239|794|45|19.02|29.86|16.42|604.80|738.90|855.90|1343.70|11.67|155.16|282.15|583.74|595.41|865.89|877.56|-272.16| +2450830|16628|2450906|28100|242325|6950|7575|28100|242325|6950|7575|2|84|7|3|7843|193|794|47|52.55|56.22|1.12|2589.70|52.64|2469.85|2642.34|1.57|0.00|105.28|52.64|54.21|157.92|159.49|-2417.21| +2450830|16628|2450845|28100|242325|6950|7575|28100|242325|6950|7575|2|108|11|4|536|52|794|70|43.73|102.32|82.87|1361.50|5800.90|3061.10|7162.40|348.05|0.00|1862.00|5800.90|6148.95|7662.90|8010.95|2739.80| +2450830|16628|2450910|28100|242325|6950|7575|28100|242325|6950|7575|2|73|16|1|10466|278|794|47|67.84|156.03|106.10|2346.71|4986.70|3188.48|7333.41|249.33|0.00|3153.23|4986.70|5236.03|8139.93|8389.26|1798.22| +2450830|16628|2450848|28100|242325|6950|7575|28100|242325|6950|7575|2|64|10|4|9682|6|794|33|21.43|51.21|0.51|1673.10|16.83|707.19|1689.93|0.71|8.91|0.00|7.92|8.63|7.92|8.63|-699.27| +2450830|16628|2450914|28100|242325|6950|7575|28100|242325|6950|7575|2|10|9|3|34|168|794|88|80.98|99.60|9.96|7888.32|876.48|7126.24|8764.80|0.00|0.00|3681.04|876.48|876.48|4557.52|4557.52|-6249.76| +2450830|52246|2450880|30301|1548689|1124|25959|30301|1548689|1124|25959|4|14|2|4|3151|186|795|98|19.52|46.26|37.00|907.48|3626.00|1912.96|4533.48|126.18|1522.92|2220.68|2103.08|2229.26|4323.76|4449.94|190.12| +2450830|52246|2450832|30301|1548689|1124|25959|30301|1548689|1124|25959|4|101|5|2|15830|204|795|29|91.99|113.14|39.59|2132.95|1148.11|2667.71|3281.06|14.46|941.45|229.39|206.66|221.12|436.05|450.51|-2461.05| +2450830|52246|2450872|30301|1548689|1124|25959|30301|1548689|1124|25959|4|14|11|3|16228|54|795|83|69.82|113.10|108.57|375.99|9011.31|5795.06|9387.30|540.67|0.00|4223.87|9011.31|9551.98|13235.18|13775.85|3216.25| +2450830|52246|2450869|30301|1548689|1124|25959|30301|1548689|1124|25959|4|50|17|1|1378|118|795|13|8.03|9.87|0.19|125.84|2.47|104.39|128.31|0.19|0.00|49.92|2.47|2.66|52.39|52.58|-101.92| +2450830|52246|2450910|30301|1548689|1124|25959|30301|1548689|1124|25959|4|20|14|1|12406|107|795|64|92.35|139.44|87.84|3302.40|5621.76|5910.40|8924.16|56.21|0.00|3390.72|5621.76|5677.97|9012.48|9068.69|-288.64| +2450830|52246|2450914|30301|1548689|1124|25959|30301|1548689|1124|25959|4|40|10|4|13214|17|795|76|17.24|50.16|19.56|2325.60|1486.56|1310.24|3812.16|0.00|0.00|533.52|1486.56|1486.56|2020.08|2020.08|176.32| +2450830|52246|2450899|30301|1548689|1124|25959|30301|1548689|1124|25959|4|83|9|1|14359|150|795|61|66.45|169.44|89.80|4858.04|5477.80|4053.45|10335.84|0.00|0.00|2583.96|5477.80|5477.80|8061.76|8061.76|1424.35| +2450830|52246|2450889|30301|1548689|1124|25959|30301|1548689|1124|25959|4|33|9|5|7154|136|795|44|46.02|61.66|36.99|1085.48|1627.56|2024.88|2713.04|33.85|1204.39|705.32|423.17|457.02|1128.49|1162.34|-1601.71| +2450830|52246|2450871|30301|1548689|1124|25959|30301|1548689|1124|25959|4|75|5|5|5317|282|795|55|50.29|98.56|65.04|1843.60|3577.20|2765.95|5420.80|137.36|143.08|2384.80|3434.12|3571.48|5818.92|5956.28|668.17| +2450830|35636|2450906|43545|1394763|1981|17599|43545|1394763|1981|17599|4|103|12|5|1531|81|796|46|79.24|165.61|163.95|76.36|7541.70|3645.04|7618.06|0.00|2337.92|2133.02|5203.78|5203.78|7336.80|7336.80|1558.74| +2450830|||||||43545||||4|22|||13058||796|82|41.14||77.14|3558.80|6325.48|3373.48|9884.28|0.00|5376.65||948.83||4605.21|4605.21|-2424.65| +2450830|35636|2450847|43545|1394763|1981|17599|43545|1394763|1981|17599|4|59|16|4|8038|110|796|93|96.56|99.45|55.69|4069.68|5179.17|8980.08|9248.85|0.00|0.00|4068.75|5179.17|5179.17|9247.92|9247.92|-3800.91| +2450830|35636|2450902|43545|1394763|1981|17599|43545|1394763|1981|17599|4|22|20|1|14245|100|796|9|77.04|127.11|55.92|640.71|503.28|693.36|1143.99|4.42|392.55|331.74|110.73|115.15|442.47|446.89|-582.63| +2450830|35636|2450875|43545|1394763|1981|17599|43545|1394763|1981|17599|4|55|17|1|2852|14|796|98|45.86|130.24|74.23|5488.98|7274.54|4494.28|12763.52|458.29|727.45|5488.00|6547.09|7005.38|12035.09|12493.38|2052.81| +2450830|63707|2450874|88149|1536816|5130|44597|88149|1536816|5130|44597|2|1|18|5|2224|253|797|75|2.85|7.83|5.08|206.25|381.00|213.75|587.25|15.24|0.00|69.75|381.00|396.24|450.75|465.99|167.25| +2450830|63707|2450833|88149|1536816|5130|44597|88149|1536816|5130|44597|2|82|11|4|13552|112|797|19|72.01|203.78|116.15|1664.97|2206.85|1368.19|3871.82|88.27|0.00|1626.02|2206.85|2295.12|3832.87|3921.14|838.66| +2450830|63707|2450862|88149|1536816|5130|44597|88149|1536816|5130|44597|2|52|11|4|17518|228|797|98|24.64|73.42|71.95|144.06|7051.10|2414.72|7195.16|141.02|0.00|359.66|7051.10|7192.12|7410.76|7551.78|4636.38| +2450830|63707|2450848|88149|1536816|5130|44597|88149|1536816|5130|44597|2|26|2|2|14218|227|797|74|5.43|9.77|5.27|333.00|389.98|401.82|722.98|3.89|0.00|158.36|389.98|393.87|548.34|552.23|-11.84| +2450830|63707|2450895|88149|1536816|5130|44597|88149|1536816|5130|44597|2|16|10|3|1652|130|797|10|58.27|167.23|153.85|133.80|1538.50|582.70|1672.30|84.46|600.01|217.30|938.49|1022.95|1155.79|1240.25|355.79| +2450830|43671|2450891|74945|891966|987|18761|2589|362356|177|4751|4|98|15|2|2509|171|798|20|88.71|133.06|1.33|2634.60|26.60|1774.20|2661.20|2.12|0.00|1330.60|26.60|28.72|1357.20|1359.32|-1747.60| +2450830|43671|2450832|74945|891966|987|18761|2589|362356|177|4751|4|17|7|3|11491|213|798|14|51.09|101.66|4.06|1366.40|56.84|715.26|1423.24|4.54|0.00|184.94|56.84|61.38|241.78|246.32|-658.42| +2450830|43671|2450894|74945|891966|987|18761|2589|362356|177|4751|4|99|9|3|8372|261|798|45|22.14|55.79|15.62|1807.65|702.90|996.30|2510.55|42.17|0.00|752.85|702.90|745.07|1455.75|1497.92|-293.40| +2450830|43671|2450881|74945|891966|987|18761|2589|362356|177|4751|4|58|5|4|6592|179|798|13|19.19|27.63|8.28|251.55|107.64|249.47|359.19|4.30|0.00|107.64|107.64|111.94|215.28|219.58|-141.83| +2450830|43671|2450906|74945|891966|987|18761|2589|362356|177|4751|4|71|3|5|8305|26|798|44|8.36|10.45|6.06|193.16|266.64|367.84|459.80|21.33|0.00|114.84|266.64|287.97|381.48|402.81|-101.20| +2450830|43671|2450913|74945|891966|987|18761|2589|362356|177|4751|4|99|7|5|14863|73|798|76|65.56|108.82|22.85|6533.72|1736.60|4982.56|8270.32|156.29|0.00|330.60|1736.60|1892.89|2067.20|2223.49|-3245.96| +2450830|43671|2450844|74945|891966|987|18761|2589|362356|177|4751|4|84|8|3|7516|113|798|2|79.14|227.13|147.63|159.00|295.26|158.28|454.26|23.62|0.00|95.38|295.26|318.88|390.64|414.26|136.98| +2450830|43671|2450916|74945|891966|987|18761|2589|362356|177|4751|4|90|14|3|3295|266|798|95|12.20|14.27|0.42|1315.75|39.90|1159.00|1355.65|3.59|0.00|257.45|39.90|43.49|297.35|300.94|-1119.10| +2450830|66802|2450861|79433|1406132|5278|30824|14333|1186716|262|10561|4|50|13|4|13021|76|799|43|4.42|12.50|1.75|462.25|75.25|190.06|537.50|6.02|0.00|172.00|75.25|81.27|247.25|253.27|-114.81| +2450830|66802|2450899|79433|1406132|5278|30824|14333|1186716|262|10561|4|103|2|3|13684|97|799|3|11.42|18.50|16.46|6.12|49.38|34.26|55.50|0.00|0.00|21.63|49.38|49.38|71.01|71.01|15.12| +2450830|66802|2450853|79433|1406132|5278|30824|14333|1186716|262|10561|4|41|18|4|5086|129|799|10|9.60|23.61|18.41|52.00|184.10|96.00|236.10|0.00|47.86|63.70|136.24|136.24|199.94|199.94|40.24| +2450830|66802|2450868|79433|1406132|5278|30824|14333|1186716|262|10561|4|12|20|3|16195|257|799|67|60.80|152.60|48.83|6952.59|3271.61|4073.60|10224.20|98.14|0.00|3987.17|3271.61|3369.75|7258.78|7356.92|-801.99| +2450830|66802|2450907|79433|1406132|5278|30824|14333|1186716|262|10561|4|45|8|2|860|167|799|62|51.42|115.69|13.88|6312.22|860.56|3188.04|7172.78|43.02|0.00|3586.08|860.56|903.58|4446.64|4489.66|-2327.48| +2450830|66802|2450862|79433|1406132|5278|30824|14333|1186716|262|10561|4|38|7|4|6403|108|799|72|49.58|104.61|64.85|2862.72|4669.20|3569.76|7531.92|41.55|513.61|1129.68|4155.59|4197.14|5285.27|5326.82|585.83| +2450830|66802|2450901|79433|1406132|5278|30824|14333|1186716|262|10561|4|20|4|3|9112|9|799|12|47.15|128.24|91.05|446.28|1092.60|565.80|1538.88|76.48|0.00|630.84|1092.60|1169.08|1723.44|1799.92|526.80| +2450830|66802|2450918|79433|1406132|5278|30824|14333|1186716|262|10561|4|107|16|5|1352|172|799|16|26.18|47.90|5.74|674.56|91.84|418.88|766.40|5.51|0.00|252.80|91.84|97.35|344.64|350.15|-327.04| +2450830|66802|2450902|79433|1406132|5278|30824|14333|1186716|262|10561|4|55|16|4|7273|173|799|71|11.13|22.03|6.60|1095.53|468.60|790.23|1564.13|4.68|0.00|265.54|468.60|473.28|734.14|738.82|-321.63| +2450830|66802|2450833|79433|1406132|5278|30824|14333|1186716|262|10561|4|92|17|2|16417|64|799|60|90.25|213.89|166.83|2823.60|10009.80|5415.00|12833.40|800.78|0.00|3336.60|10009.80|10810.58|13346.40|14147.18|4594.80| +2450830|41232|2450914|57226|66382|3091|49047|57226|66382|3091|49047|2|24|2|2|3085|63|800|96|74.78|111.42|86.90|2353.92|8342.40|7178.88|10696.32|83.42|0.00|3208.32|8342.40|8425.82|11550.72|11634.14|1163.52| +2450830|41232|2450850|57226|66382|3091|49047|57226|66382|3091|49047|2|1|20|1|10084|117|800|30|15.89|45.28|5.43|1195.50|162.90|476.70|1358.40|3.25|0.00|244.50|162.90|166.15|407.40|410.65|-313.80| +2450830|41232|2450860|57226|66382|3091|49047|57226|66382|3091|49047|2|101|8|3|2257|298|800|23|61.35|88.95|40.91|1104.92|940.93|1411.05|2045.85|37.63|0.00|593.17|940.93|978.56|1534.10|1571.73|-470.12| +2450830|41232|2450854|57226|66382|3091|49047|57226|66382|3091|49047|2|3|16|5|14113|120|800|67|43.53|94.89|45.54|3306.45|3051.18|2916.51|6357.63|244.09|0.00|253.93|3051.18|3295.27|3305.11|3549.20|134.67| +2450830|41232|2450885|57226|66382|3091|49047|57226|66382|3091|49047|2|55|20|4|3176|45|800|63|26.50|29.41|29.11|18.90|1833.93|1669.50|1852.83|55.01|0.00|55.44|1833.93|1888.94|1889.37|1944.38|164.43| +2450830|41232|2450863|57226|66382|3091|49047|57226|66382|3091|49047|2|87|8|1|15644|269|800|16|6.79|12.90|3.99|142.56|63.84|108.64|206.40|3.19|0.00|24.64|63.84|67.03|88.48|91.67|-44.80| +2450830|41232|2450874|57226|66382|3091|49047|57226|66382|3091|49047|2|34|7|4|13585|193|800|41|19.39|50.41|32.26|744.15|1322.66|794.99|2066.81|13.22|0.00|0.00|1322.66|1335.88|1322.66|1335.88|527.67| +2450830|41232|2450854|57226|66382|3091|49047|57226|66382|3091|49047|2|9|13|4|10171|136|800|9|79.10|103.62|20.72|746.10|186.48|711.90|932.58|14.91|0.00|205.11|186.48|201.39|391.59|406.50|-525.42| +2450830|41232|2450905|57226|66382|3091|49047|57226|66382|3091|49047|2|78|1|2|2360|50|800|15|58.59|111.32|64.56|701.40|968.40|878.85|1669.80|15.88|174.31|534.30|794.09|809.97|1328.39|1344.27|-84.76| +2450830|66088|2450834|39826|1876330|3587|8891|39826|1876330|3587|8891|1|6|7|4|4171|38|801|41|80.22|120.33|48.13|2960.20|1973.33|3289.02|4933.53|103.60|493.33|147.60|1480.00|1583.60|1627.60|1731.20|-1809.02| +2450830|66088|2450897|39826|1876330|3587|8891|39826|1876330|3587|8891|1|108|20|4|14972|20|801|90|92.93|179.35|35.87|12913.20|3228.30|8363.70|16141.50|133.97|548.81|5326.20|2679.49|2813.46|8005.69|8139.66|-5684.21| +2450830|66088|2450900|39826|1876330|3587|8891|39826|1876330|3587|8891|1|48|18|3|9919|113|801|45|65.55|94.39|67.96|1189.35|3058.20|2949.75|4247.55|52.60|1743.17|1698.75|1315.03|1367.63|3013.78|3066.38|-1634.72| +2450830|66088|2450915|39826|1876330|3587|8891|39826|1876330|3587|8891|1|92|3|4|1354|246|801|85|64.88|90.83|16.34|6331.65|1388.90|5514.80|7720.55|6.66|722.22|1620.95|666.68|673.34|2287.63|2294.29|-4848.12| +2450830|66088|2450864|39826|1876330|3587|8891|39826|1876330|3587|8891|1|57|20|2|5210|49|801|43|97.17|262.35|76.08|8009.61|3271.44|4178.31|11281.05|294.42|0.00|1917.37|3271.44|3565.86|5188.81|5483.23|-906.87| +2450830|66088|2450835|39826|1876330|3587|8891|39826|1876330|3587|8891|1|72|13|4|16288|23|801|56|37.82|88.12|37.89|2812.88|2121.84|2117.92|4934.72|0.00|0.00|1924.16|2121.84|2121.84|4046.00|4046.00|3.92| +2450830|66088|2450884|39826|1876330|3587|8891|39826|1876330|3587|8891|1|25|14|2|13690|173|801|57|70.81|150.82|131.21|1117.77|7478.97|4036.17|8596.74|598.31|0.00|2406.54|7478.97|8077.28|9885.51|10483.82|3442.80| +2450830|66088|2450891|39826|1876330|3587|8891|39826|1876330|3587|8891|1|22|4|2|17060|265|801|12|94.49|101.10|83.91|206.28|1006.92|1133.88|1213.20|90.62|0.00|36.36|1006.92|1097.54|1043.28|1133.90|-126.96| +2450830|66088|2450862|39826|1876330|3587|8891|39826|1876330|3587|8891|1|11|17|3|2984|22|801|53|30.70|91.17|60.17|1643.00|3189.01|1627.10|4832.01|287.01|0.00|1207.87|3189.01|3476.02|4396.88|4683.89|1561.91| +2450830|66088|2450892|39826|1876330|3587|8891|39826|1876330|3587|8891|1|5|2|4|7054|290|801|31|40.78|108.06|74.56|1038.50|2311.36|1264.18|3349.86|46.22|0.00|468.72|2311.36|2357.58|2780.08|2826.30|1047.18| +2450830|66088|2450856|39826|1876330|3587|8891|39826|1876330|3587|8891|1|25|3|3|9626|100|801|13|74.68|217.31|47.80|2203.63|621.40|970.84|2825.03|55.92|0.00|960.44|621.40|677.32|1581.84|1637.76|-349.44| +2450830|66088|2450844|39826|1876330|3587|8891|39826|1876330|3587|8891|1|16|12|3|9908|157|801|69|87.43|251.79|251.79|0.00|17373.51|6032.67|17373.51|868.67|0.00|6948.99|17373.51|18242.18|24322.50|25191.17|11340.84| +2450830|46718|2450904|7328|393061|4747|44033|7328|393061|4747|44033|4|72|16|4|8359|103|802|5|85.78|137.24|20.58|583.30|102.90|428.90|686.20|0.00|0.00|13.70|102.90|102.90|116.60|116.60|-326.00| +2450830|46718|2450859|7328|393061|4747|44033|7328|393061|4747|44033|4|93|13|2|5120|44|802|50|58.67|111.47|33.44|3901.50|1672.00|2933.50|5573.50|83.60|0.00|2619.50|1672.00|1755.60|4291.50|4375.10|-1261.50| +2450830|46718|2450889|7328|393061|4747|44033|7328|393061|4747|44033|4|34|19|1|2317|280|802|77|13.10|14.27|14.12|11.55|1087.24|1008.70|1098.79|7.50|337.04|21.56|750.20|757.70|771.76|779.26|-258.50| +2450830|46718|2450875|7328|393061|4747|44033|7328|393061|4747|44033|4|78|1|5|686|45|802|9|97.70|257.92|188.28|626.76|1694.52|879.30|2321.28|16.94|0.00|580.32|1694.52|1711.46|2274.84|2291.78|815.22| +2450830|46718|2450910|7328|393061|4747|44033|7328|393061|4747|44033|4|3|11|5|8822|171|802|84|43.47|75.20|48.12|2274.72|4042.08|3651.48|6316.80|161.68|0.00|2968.56|4042.08|4203.76|7010.64|7172.32|390.60| +2450830|46718|2450887|7328|393061|4747|44033|7328|393061|4747|44033|4|78|9|1|3037|68|802|31|47.72|63.46|14.59|1514.97|452.29|1479.32|1967.26|22.61|0.00|137.64|452.29|474.90|589.93|612.54|-1027.03| +2450830|46718|2450880|7328|393061|4747|44033|7328|393061|4747|44033|4|4|2|3|4150|193|802|26|68.26|73.03|22.63|1310.40|588.38|1774.76|1898.78|0.00|588.38|56.94|0.00|0.00|56.94|56.94|-1774.76| +2450830|6858|2450838|82603|1213282|3161|48240|82603|1213282|3161|48240|2|1|10|5|4081|48|803|63|1.07|1.54|1.49|3.15|93.87|67.41|97.02|0.00|0.00|45.36|93.87|93.87|139.23|139.23|26.46| +2450830|6858|2450902|82603|1213282|3161|48240|82603|1213282|3161|48240|2|70|17|2|5090|175|803|34|84.69|100.78|68.53|1096.50|2330.02|2879.46|3426.52|93.20|0.00|34.00|2330.02|2423.22|2364.02|2457.22|-549.44| +2450830|6858|2450875|82603|1213282|3161|48240|82603|1213282|3161|48240|2|15|17|3|15766|225|803|59|39.10|81.71|76.80|289.69|4531.20|2306.90|4820.89|271.87|0.00|2121.05|4531.20|4803.07|6652.25|6924.12|2224.30| +2450830|6858|2450880|82603|1213282|3161|48240|82603|1213282|3161|48240|2|66|8|4|11119|101|803|7|55.45|103.13|36.09|469.28|252.63|388.15|721.91|8.18|47.99|144.34|204.64|212.82|348.98|357.16|-183.51| +2450830|6858|2450868|82603|1213282|3161|48240|82603|1213282|3161|48240|2|103|12|2|11818|21|803|89|60.04|65.44|48.42|1514.78|4309.38|5343.56|5824.16|129.28|0.00|523.32|4309.38|4438.66|4832.70|4961.98|-1034.18| +2450830|6858|2450901|82603|1213282|3161|48240|82603|1213282|3161|48240|2|77|18|5|9379|96|803|11|79.27|189.45|143.98|500.17|1583.78|871.97|2083.95|79.18|0.00|437.58|1583.78|1662.96|2021.36|2100.54|711.81| +2450830|6858|2450853|82603|1213282|3161|48240|82603|1213282|3161|48240|2|102|19|3|2410|46|803|91|25.99|38.20|23.68|1321.32|2154.88|2365.09|3476.20|21.54|0.00|485.94|2154.88|2176.42|2640.82|2662.36|-210.21| +2450830|6858|2450894|82603|1213282|3161|48240|82603|1213282|3161|48240|2|20|12|2|1177|227|803|29|37.96|97.93|72.46|738.63|2101.34|1100.84|2839.97|42.02|0.00|283.91|2101.34|2143.36|2385.25|2427.27|1000.50| +2450830|17970|2450881|41035|1872124|4366|18655|41035|1872124|4366|18655|2|13|19|5|9373|63|804|5|97.97|220.43|147.68|363.75|738.40|489.85|1102.15|7.38|0.00|297.55|738.40|745.78|1035.95|1043.33|248.55| +2450830|17970|2450836|41035|1872124|4366|18655|41035|1872124|4366|18655|2|70|11|2|16111|107|804|15|94.11|271.97|152.30|1795.05|2284.50|1411.65|4079.55|22.84|0.00|367.05|2284.50|2307.34|2651.55|2674.39|872.85| +2450830|17970|2450912|41035|1872124|4366|18655|41035|1872124|4366|18655|2|67|17|5|4945|298|804|3|50.29|89.01|60.52|85.47|181.56|150.87|267.03|5.08|54.46|93.45|127.10|132.18|220.55|225.63|-23.77| +2450830|17970|2450843|41035|1872124|4366|18655|41035|1872124|4366|18655|2|64|7|5|5983|143|804|92|11.24|23.04|18.20|445.28|1674.40|1034.08|2119.68|97.28|284.64|931.96|1389.76|1487.04|2321.72|2419.00|355.68| +2450830|17970|2450862|41035|1872124|4366|18655|41035|1872124|4366|18655|2|8|14|3|15320|134|804|28|46.22|116.93|36.24|2259.32|1014.72|1294.16|3274.04|71.03|0.00|360.08|1014.72|1085.75|1374.80|1445.83|-279.44| +2450830|17970|2450884|41035|1872124|4366|18655|41035|1872124|4366|18655|2|11|3|1|17437|152|804|30|52.15|151.75|6.07|4370.40|182.10|1564.50|4552.50|10.92|0.00|500.70|182.10|193.02|682.80|693.72|-1382.40| +2450830|17970|2450853|41035|1872124|4366|18655|41035|1872124|4366|18655|2|45|10|1|16708|113|804|97|63.81|125.06|16.25|10554.57|1576.25|6189.57|12130.82|0.00|0.00|4609.44|1576.25|1576.25|6185.69|6185.69|-4613.32| +2450830|46776|2450866|93019|1067722|1978|39081|93019|1067722|1978|39081|2|103|17|5|15416|218|805|19|17.29|30.60|21.11|180.31|401.09|328.51|581.40|36.09|0.00|168.53|401.09|437.18|569.62|605.71|72.58| +2450830|46776|2450917|93019|1067722|1978|39081|93019|1067722|1978|39081|2|27|18|1|12223|285|805|10|74.66|167.98|109.18|588.00|1091.80|746.60|1679.80|10.91|0.00|705.50|1091.80|1102.71|1797.30|1808.21|345.20| +2450830|46776|2450861|93019|1067722|1978|39081|93019|1067722|1978|39081|2|9|11|2|9896|24|805|58|67.69|130.64|37.88|5380.08|2197.04|3926.02|7577.12|109.85|0.00|454.14|2197.04|2306.89|2651.18|2761.03|-1728.98| +2450830|46776|2450838|93019|1067722|1978|39081|93019|1067722|1978|39081|2|66|15|5|12820|212|805|63|68.67|164.80|98.88|4152.96|6229.44|4326.21|10382.40|62.29|0.00|207.27|6229.44|6291.73|6436.71|6499.00|1903.23| +2450830|46776|2450836|93019|1067722|1978|39081|93019|1067722|1978|39081|2|14|20|2|3764|184|805|32|70.14|77.15|59.40|568.00|1900.80|2244.48|2468.80|57.02|0.00|765.12|1900.80|1957.82|2665.92|2722.94|-343.68| +2450830|46776|2450878|93019|1067722|1978|39081|93019|1067722|1978|39081|2|68|7|1|3074|264|805|94|13.32|14.51|14.21|28.20|1335.74|1252.08|1363.94|106.85|0.00|109.04|1335.74|1442.59|1444.78|1551.63|83.66| +2450830|46776|2450839|93019|1067722|1978|39081|93019|1067722|1978|39081|2|13|17|3|13525|277|805|69|88.10|96.91|20.35|5282.64|1404.15|6078.90|6686.79|42.12|0.00|2272.86|1404.15|1446.27|3677.01|3719.13|-4674.75| +2450830|46776|2450920|93019|1067722|1978|39081|93019|1067722|1978|39081|2|29|9|4|15542|229|805|81|26.37|39.55|35.99|288.36|2915.19|2135.97|3203.55|145.75|0.00|1057.05|2915.19|3060.94|3972.24|4117.99|779.22| +2450830|46776|2450878|93019|1067722|1978|39081|93019|1067722|1978|39081|2|20|14|5|11824|40|805|32|31.47|62.94|49.09|443.20|1570.88|1007.04|2014.08|141.37|0.00|825.60|1570.88|1712.25|2396.48|2537.85|563.84| +2450830|46776|2450907|93019|1067722|1978|39081|93019|1067722|1978|39081|2|33|18|5|3055|232|805|26|68.65|153.08|41.33|2905.50|1074.58|1784.90|3980.08|53.72|0.00|238.68|1074.58|1128.30|1313.26|1366.98|-710.32| +2450830|46776|2450860|93019|1067722|1978|39081|93019|1067722|1978|39081|2|88|19|1|15106|86|805|75|43.68|95.22|95.22|0.00|7141.50|3276.00|7141.50|142.83|0.00|2928.00|7141.50|7284.33|10069.50|10212.33|3865.50| +2450830|46776|2450889|93019|1067722|1978|39081|93019|1067722|1978|39081|2|63|19|3|11395|39|805|60|18.13|25.56|21.72|230.40|1303.20|1087.80|1533.60|65.16|0.00|122.40|1303.20|1368.36|1425.60|1490.76|215.40| +2450830|86298|2450912|41161|1860608|4855|4302|41161|1860608|4855|4302|4|95|14|5|3685|275|806|68|47.60|100.91|58.52|2882.52|3979.36|3236.80|6861.88|358.14|0.00|2195.72|3979.36|4337.50|6175.08|6533.22|742.56| +2450830|86298||||4855||41161|1860608||||15||2|15280|27|806|18|||14.67||264.06||2032.38|0.13|261.41|507.96|2.65|2.78|510.61|510.74|-884.93| +2450830|86298|2450903|41161|1860608|4855|4302|41161|1860608|4855|4302|4|78|15|3|8521|61|806|61|66.84|145.04|62.36|5043.48|3803.96|4077.24|8847.44|0.00|0.00|1061.40|3803.96|3803.96|4865.36|4865.36|-273.28| +2450830|86298|2450903|41161|1860608|4855|4302|41161|1860608|4855|4302|4|91|1|2|2408|144|806|11|41.55|115.92|10.43|1160.39|114.73|457.05|1275.12|0.59|99.81|165.66|14.92|15.51|180.58|181.17|-442.13| +2450830|86298|2450906|41161|1860608|4855|4302|41161|1860608|4855|4302|4|46|8|4|5371|118|806|65|14.54|17.15|7.88|602.55|512.20|945.10|1114.75|30.73|0.00|356.20|512.20|542.93|868.40|899.13|-432.90| +2450830|86298|2450910|41161|1860608|4855|4302|41161|1860608|4855|4302|4|71|20|5|3958|133|806|52|57.55|136.96|32.87|5412.68|1709.24|2992.60|7121.92|4.44|1264.83|2421.12|444.41|448.85|2865.53|2869.97|-2548.19| +2450830|72729|2450871|32273|470140|5607|19121|32273|470140|5607|19121|2|101|8|1|4591|295|807|42|22.65|24.91|0.74|1015.14|31.08|951.30|1046.22|1.24|0.00|522.90|31.08|32.32|553.98|555.22|-920.22| +2450830|72729|2450880|32273|470140|5607|19121|32273|470140|5607|19121|2|54|3|2|17407|84|807|64|14.95|37.67|35.78|120.96|2289.92|956.80|2410.88|91.59|0.00|240.64|2289.92|2381.51|2530.56|2622.15|1333.12| +2450830|72729|2450904|32273|470140|5607|19121|32273|470140|5607|19121|2|85|9|2|14372|213|807|12|24.81|64.50|30.96|402.48|371.52|297.72|774.00|18.57|0.00|325.08|371.52|390.09|696.60|715.17|73.80| +2450830|72729|2450913|32273|470140|5607|19121|32273|470140|5607|19121|2|77|19|3|12535|166|807|89|67.70|190.91|166.09|2208.98|14782.01|6025.30|16990.99|147.82|0.00|5607.00|14782.01|14929.83|20389.01|20536.83|8756.71| +2450830|72729|2450898|32273|470140|5607|19121|32273|470140|5607|19121|2|8|9|4|13612|82|807|96|80.29|208.75|2.08|19840.32|199.68|7707.84|20040.00|9.68|5.99|3206.40|193.69|203.37|3400.09|3409.77|-7514.15| +2450830|72729|2450852|32273|470140|5607|19121|32273|470140|5607|19121|2|40|7|2|5764|235|807|62|8.10|22.03|9.25|792.36|573.50|502.20|1365.86|0.00|0.00|218.24|573.50|573.50|791.74|791.74|71.30| +2450830|72729|2450918|32273|470140|5607|19121|32273|470140|5607|19121|2|3|3|1|13444|2|807|78|89.49|237.14|92.48|11283.48|7213.44|6980.22|18496.92|216.40|0.00|4438.98|7213.44|7429.84|11652.42|11868.82|233.22| +2450830|75892|2450859|39264|1220299|2581|30016|39264|1220299|2581|30016|4|63|10|5|9910|123|808|29|27.94|79.07|33.20|1330.23|962.80|810.26|2293.03|28.88|0.00|1100.55|962.80|991.68|2063.35|2092.23|152.54| +2450830|75892|2450918|39264|1220299|2581|30016|39264|1220299|2581|30016|4|64|5|3|4096|281|808|93|39.47|70.65|37.44|3088.53|3481.92|3670.71|6570.45|243.73|0.00|196.23|3481.92|3725.65|3678.15|3921.88|-188.79| +2450830|75892|2450850|39264|1220299|2581|30016|39264|1220299|2581|30016|4|8|16|1|11974|134|808|82|68.40|156.63|144.09|1028.28|11815.38|5608.80|12843.66|236.30|0.00|0.00|11815.38|12051.68|11815.38|12051.68|6206.58| +2450830|75892|2450868|39264|1220299|2581|30016|39264|1220299|2581|30016|4|65|6|2|14654|26|808|67|94.64|170.35|131.16|2625.73|8787.72|6340.88|11413.45|263.63|0.00|4336.91|8787.72|9051.35|13124.63|13388.26|2446.84| +2450830|75892|2450909|39264|1220299|2581|30016|39264|1220299|2581|30016|4|1|5|4|7942|114|808|44|53.58|115.19|27.64|3852.20|1216.16|2357.52|5068.36|36.48|0.00|1520.20|1216.16|1252.64|2736.36|2772.84|-1141.36| +2450830|75892|2450842|39264|1220299|2581|30016|39264|1220299|2581|30016|4|48|18|3|12751|248|808|8|5.06|12.34|9.13|25.68|73.04|40.48|98.72|1.43|52.58|2.96|20.46|21.89|23.42|24.85|-20.02| +2450830|75892|2450850|39264|1220299|2581|30016|39264|1220299|2581|30016|4|73|4|5|9622|13|808|100|55.69|82.97|82.97|0.00|8297.00|5569.00|8297.00|248.91|0.00|912.00|8297.00|8545.91|9209.00|9457.91|2728.00| +2450830|75892|2450917|39264|1220299|2581|30016|39264|1220299|2581|30016|4|79|5|3|12082|290|808|93|66.70|136.73|94.34|3942.27|8773.62|6203.10|12715.89|126.34|7194.36|1016.49|1579.26|1705.60|2595.75|2722.09|-4623.84| +2450830|75892|2450878|39264|1220299|2581|30016|39264|1220299|2581|30016|4|58|2|5|8468|74|808|69|53.31|122.61|67.43|3807.42|4652.67|3678.39|8460.09|223.32|930.53|2452.95|3722.14|3945.46|6175.09|6398.41|43.75| +2450830|75892|2450897|39264|1220299|2581|30016|39264|1220299|2581|30016|4|45|1|3|15536|237|808|75|51.46|104.46|59.54|3369.00|4465.50|3859.50|7834.50|223.27|0.00|939.75|4465.50|4688.77|5405.25|5628.52|606.00| +2450830|75892|2450891|39264|1220299|2581|30016|39264|1220299|2581|30016|4|99|13|1|12308|235|808|10|18.28|28.33|16.43|119.00|164.30|182.80|283.30|6.57|0.00|42.40|164.30|170.87|206.70|213.27|-18.50| +2450830|75892|2450910|39264|1220299|2581|30016|39264|1220299|2581|30016|4|41|14|2|10862|88|808|91|62.38|140.97|94.44|4234.23|8594.04|5676.58|12828.27|85.94|0.00|2693.60|8594.04|8679.98|11287.64|11373.58|2917.46| +2450830|75892|2450861|39264|1220299|2581|30016|39264|1220299|2581|30016|4|103|5|2|12970|19|808|40|21.31|28.55|3.71|993.60|148.40|852.40|1142.00|0.59|140.98|548.00|7.42|8.01|555.42|556.01|-844.98| +2450830|75892|2450883|39264|1220299|2581|30016|39264|1220299|2581|30016|4|73|6|1|5440|259|808|50|95.33|116.30|22.09|4710.50|1104.50|4766.50|5815.00|11.04|0.00|2849.00|1104.50|1115.54|3953.50|3964.54|-3662.00| +2450830|49963|2450838|40915|124500|3285|10470|99969|1601593|6778|35916|1|7|5|1|464|174|809|26|21.11|63.11|60.58|65.78|1575.08|548.86|1640.86|31.50|0.00|147.42|1575.08|1606.58|1722.50|1754.00|1026.22| +2450830|49963|2450902|40915|124500|3285|10470|99969|1601593|6778|35916|1|95|19|3|11683|16|809|59|81.09|158.93|81.05|4594.92|4781.95|4784.31|9376.87|187.45|2104.05|656.08|2677.90|2865.35|3333.98|3521.43|-2106.41| +2450830|49963|2450886|40915|124500|3285|10470|99969|1601593|6778|35916|1|9|9|3|16144|157|809|50|44.24|74.76|17.19|2878.50|859.50|2212.00|3738.00|0.00|0.00|1046.50|859.50|859.50|1906.00|1906.00|-1352.50| +2450830|49963|2450895|40915|124500|3285|10470|99969|1601593|6778|35916|1|65|10|2|12842|199|809|77|6.83|20.49|9.01|883.96|693.77|525.91|1577.73|0.41|672.95|314.93|20.82|21.23|335.75|336.16|-505.09| +2450830|49963|2450833|40915|124500|3285|10470|99969|1601593|6778|35916|1|77|12|5|17548|128|809|77|20.79|43.03|9.89|2551.78|761.53|1600.83|3313.31|68.53|0.00|1490.72|761.53|830.06|2252.25|2320.78|-839.30| +2450830|49963|2450859|40915|124500|3285|10470|99969|1601593|6778|35916|1|77|3|4|17584|13|809|25|45.80|76.02|12.16|1596.50|304.00|1145.00|1900.50|18.48|72.96|95.00|231.04|249.52|326.04|344.52|-913.96| +2450830|49963|2450918|40915|124500|3285|10470|99969|1601593|6778|35916|1|46|19|4|15020|246|809|70|84.61|238.60|133.61|7349.30|9352.70|5922.70|16702.00|467.63|0.00|4676.00|9352.70|9820.33|14028.70|14496.33|3430.00| +2450830|49963|2450863|40915|124500|3285|10470|99969|1601593|6778|35916|1|15|14|5|1394|100|809|76|1.52|4.16|2.95|91.96|224.20|115.52|316.16|13.45|0.00|135.28|224.20|237.65|359.48|372.93|108.68| +2450830|49963|2450849|40915|124500|3285|10470|99969|1601593|6778|35916|1|102|17|3|15493|132|809|39|62.06|78.19|72.71|213.72|2835.69|2420.34|3049.41|170.14|0.00|975.78|2835.69|3005.83|3811.47|3981.61|415.35| +2450830|49963|2450837|40915|124500|3285|10470|99969|1601593|6778|35916|1|103|19|4|10957|154|809|89|85.44|256.32|33.32|19847.00|2965.48|7604.16|22812.48|177.92|0.00|0.00|2965.48|3143.40|2965.48|3143.40|-4638.68| +2450830|49963|2450912|40915|124500|3285|10470|99969|1601593|6778|35916|1|37|3|3|7021|95|809|46|34.68|50.28|42.73|347.30|1965.58|1595.28|2312.88|78.62|0.00|46.00|1965.58|2044.20|2011.58|2090.20|370.30| +2450830|49963|2450887|40915|124500|3285|10470|99969|1601593|6778|35916|1|87|3|4|4219|194|809|18|88.27|139.46|36.25|1857.78|652.50|1588.86|2510.28|32.62|0.00|752.94|652.50|685.12|1405.44|1438.06|-936.36| +2450830|49963|2450896|40915|124500|3285|10470|99969|1601593|6778|35916|1|41|6|3|16039|13|809|44|49.89|85.31|19.62|2890.36|863.28|2195.16|3753.64|0.00|0.00|1276.00|863.28|863.28|2139.28|2139.28|-1331.88| +2450830|26796|2450866|77584|630704|6224|31791|77584|630704|6224|31791|1|47|19|3|4970|289|810|92|71.05|126.46|73.34|4887.04|6747.28|6536.60|11634.32|0.00|0.00|581.44|6747.28|6747.28|7328.72|7328.72|210.68| +2450830|26796|2450859|77584|630704|6224|31791|77584|630704|6224|31791|1|76|20|5|9004|37|810|57|70.84|195.51|5.86|10810.05|334.02|4037.88|11144.07|10.02|0.00|2785.59|334.02|344.04|3119.61|3129.63|-3703.86| +2450830|26796|2450855|77584|630704|6224|31791|77584|630704|6224|31791|1|4|10|1|12884|64|810|87|82.78|177.14|141.71|3082.41|12328.77|7201.86|15411.18|863.01|0.00|1540.77|12328.77|13191.78|13869.54|14732.55|5126.91| +2450830|26796|2450859|77584|630704|6224|31791|77584|630704|6224|31791|1|48|19|5|14035|290|810|40|39.85|74.12|25.20|1956.80|1008.00|1594.00|2964.80|90.72|0.00|1393.20|1008.00|1098.72|2401.20|2491.92|-586.00| +2450830|26796|2450856|77584|630704|6224|31791|77584|630704|6224|31791|1|43|10|2|17792|271|810|61|97.61|210.83|137.03|4501.80|8358.83|5954.21|12860.63|83.58|0.00|4758.00|8358.83|8442.41|13116.83|13200.41|2404.62| +2450830|26796|2450845|77584|630704|6224|31791|77584|630704|6224|31791|1|67|10|2|14758|201|810|11|27.12|60.20|8.42|569.58|92.62|298.32|662.20|1.85|0.00|26.40|92.62|94.47|119.02|120.87|-205.70| +2450830|26796|2450914|77584|630704|6224|31791|77584|630704|6224|31791|1|102|17|3|17389|138|810|77|40.18|117.72|100.06|1359.82|7704.62|3093.86|9064.44|693.41|0.00|4078.69|7704.62|8398.03|11783.31|12476.72|4610.76| +2450830|26796|2450855|77584|630704|6224|31791|77584|630704|6224|31791|1|72|2|4|1490|51|810|1|26.56|58.96|12.38|46.58|12.38|26.56|58.96|0.37|0.00|16.50|12.38|12.75|28.88|29.25|-14.18| +2450830|26796|2450914|77584|630704|6224|31791|77584|630704|6224|31791|1|86|2|4|3410|78|810|94|60.20|150.50|12.04|13015.24|1131.76|5658.80|14147.00|11.31|0.00|423.94|1131.76|1143.07|1555.70|1567.01|-4527.04| +2450830|26796|2450906|77584|630704|6224|31791|77584|630704|6224|31791|1|96|15|3|3520|216|810|82|73.72|89.93|65.64|1991.78|5382.48|6045.04|7374.26|269.12|0.00|2285.34|5382.48|5651.60|7667.82|7936.94|-662.56| +2450830|29930|2450838|88311|1487383|6455|11480|88311|1487383|6455|11480|4|91|11|2|8269|279|811|84|35.78|82.65|3.30|6665.40|277.20|3005.52|6942.60|0.00|124.74|1596.00|152.46|152.46|1748.46|1748.46|-2853.06| +2450830|29930|2450913|88311|1487383|6455|11480|88311|1487383|6455|11480|4|84|18|2|571|72|811|11|31.47|94.09|69.62|269.17|765.82|346.17|1034.99|61.26|0.00|175.89|765.82|827.08|941.71|1002.97|419.65| +2450830|29930|2450888|88311|1487383|6455|11480|88311|1487383|6455|11480|4|76|12|5|6220|9|811|75|90.43|236.92|52.12|13860.00|3909.00|6782.25|17769.00|351.81|0.00|3020.25|3909.00|4260.81|6929.25|7281.06|-2873.25| +2450830|29930|2450836|88311|1487383|6455|11480|88311|1487383|6455|11480|4|42|7|4|17596|148|811|100|24.60|36.65|17.95|1870.00|1795.00|2460.00|3665.00|107.70|0.00|879.00|1795.00|1902.70|2674.00|2781.70|-665.00| +2450830|29930|2450851|88311|1487383|6455|11480|88311|1487383|6455|11480|4|28|16|3|11384|194|811|82|47.63|52.39|11.52|3351.34|944.64|3905.66|4295.98|0.00|0.00|1890.10|944.64|944.64|2834.74|2834.74|-2961.02| +2450830|29930|2450849|88311|1487383|6455|11480|88311|1487383|6455|11480|4|84|5|4|17545|49|811|96|48.03|70.60|50.12|1966.08|4811.52|4610.88|6777.60|240.57|0.00|2981.76|4811.52|5052.09|7793.28|8033.85|200.64| +2450830|29930|2450832|88311|1487383|6455|11480|88311|1487383|6455|11480|4|9|15|3|11354|97|811|48|48.95|83.21|54.08|1398.24|2595.84|2349.60|3994.08|168.21|726.83|1277.76|1869.01|2037.22|3146.77|3314.98|-480.59| +2450830|29930|2450877|88311|1487383|6455|11480|88311|1487383|6455|11480|4|82|2|4|1664|178|811|50|59.93|71.31|35.65|1783.00|1782.50|2996.50|3565.50|89.12|0.00|1140.50|1782.50|1871.62|2923.00|3012.12|-1214.00| +2450830|29930|2450845|88311|1487383|6455|11480|88311|1487383|6455|11480|4|90|19|3|13532|6|811|48|86.84|249.23|119.63|6220.80|5742.24|4168.32|11963.04|114.84|0.00|2870.88|5742.24|5857.08|8613.12|8727.96|1573.92| +2450830|29930|2450850|88311|1487383|6455|11480|88311|1487383|6455|11480|4|13|18|4|14782|21|811|83|33.82|76.77|66.78|829.17|5542.74|2807.06|6371.91|0.00|0.00|3057.72|5542.74|5542.74|8600.46|8600.46|2735.68| +2450830|29930|2450859|88311|1487383|6455|11480|88311|1487383|6455|11480|4|27|4|2|16477|155|811|56|43.00|91.16|70.19|1174.32|3930.64|2408.00|5104.96|275.14|0.00|203.84|3930.64|4205.78|4134.48|4409.62|1522.64| +2450830|9217|2450874|10776|207141|3153|16718|10776|207141|3153|16718|2|103|12|1|6922|94|812|24|90.25|259.01|62.16|4724.40|1491.84|2166.00|6216.24|0.00|0.00|1429.68|1491.84|1491.84|2921.52|2921.52|-674.16| +2450830|9217|2450896|10776|207141|3153|16718|10776|207141|3153|16718|2|42|3|4|10988|30|812|81|41.50|75.94|12.90|5106.24|1044.90|3361.50|6151.14|0.00|1044.90|2583.09|0.00|0.00|2583.09|2583.09|-3361.50| +2450830|9217|2450909|10776|207141|3153|16718|10776|207141|3153|16718|2|107|16|3|11216|102|812|8|6.31|8.26|7.43|6.64|59.44|50.48|66.08|2.37|0.00|31.04|59.44|61.81|90.48|92.85|8.96| +2450830|9217|2450875|10776|207141|3153|16718|10776|207141|3153|16718|2|69|8|4|13630|179|812|41|65.92|183.25|168.59|601.06|6912.19|2702.72|7513.25|138.24|0.00|525.62|6912.19|7050.43|7437.81|7576.05|4209.47| +2450830|9217|2450872|10776|207141|3153|16718|10776|207141|3153|16718|2|20|3|1|17539|82|812|47|88.18|203.69|57.03|6893.02|2680.41|4144.46|9573.43|105.07|1179.38|2871.70|1501.03|1606.10|4372.73|4477.80|-2643.43| +2450830|65271|2450838|31045|743699|800|23854|18247|1267209|3863|19717|1|70|4|4|3896|116|813|57|59.27|135.72|109.93|1470.03|6266.01|3378.39|7736.04|0.00|0.00|2939.49|6266.01|6266.01|9205.50|9205.50|2887.62| +2450830|65271|2450874|31045|743699|800|23854|18247|1267209|3863|19717|1|40|3|2|12679|109|813|2|75.37|186.16|128.45|115.42|256.90|150.74|372.32|3.90|208.08|156.36|48.82|52.72|205.18|209.08|-101.92| +2450830|65271|2450907|31045|743699|800|23854|18247|1267209|3863|19717|1|31|6|3|17218|164|813|9|18.95|54.38|20.12|308.34|181.08|170.55|489.42|3.04|79.67|244.71|101.41|104.45|346.12|349.16|-69.14| +2450830|65271|2450919|31045|743699|800|23854|18247|1267209|3863|19717|1|105|14|4|10576|239|813|2|78.50|153.86|20.00|267.72|40.00|157.00|307.72|2.40|0.00|40.00|40.00|42.40|80.00|82.40|-117.00| +2450830|65271|2450853|31045|743699|800|23854|18247|1267209|3863|19717|1|41|19|2|14354|285|813|64|23.52|40.21|22.11|1158.40|1415.04|1505.28|2573.44|84.90|0.00|385.92|1415.04|1499.94|1800.96|1885.86|-90.24| +2450830|65271|2450862|31045|743699|800|23854|18247|1267209|3863|19717|1|107|8|3|12103|265|813|30|57.59|90.41|52.43|1139.40|1572.90|1727.70|2712.30|6.29|943.74|894.90|629.16|635.45|1524.06|1530.35|-1098.54| +2450830|65271|2450898|31045|743699|800|23854|18247|1267209|3863|19717|1|66|6|3|7292|200|813|26|19.85|29.97|12.58|452.14|327.08|516.10|779.22|9.81|0.00|54.34|327.08|336.89|381.42|391.23|-189.02| +2450830|65271|2450896|31045|743699|800|23854|18247|1267209|3863|19717|1|18|11|5|3643|103|813|72|7.57|14.98|11.38|259.20|819.36|545.04|1078.56|49.16|0.00|64.08|819.36|868.52|883.44|932.60|274.32| +2450830|69751|2450838|16544|673742|2940|26585|16544|673742|2940|26585|2|90|10|4|9604|17|814|34|49.91|90.83|35.42|1883.94|1204.28|1696.94|3088.22|48.17|0.00|1266.16|1204.28|1252.45|2470.44|2518.61|-492.66| +2450830|69751|2450911|16544|673742|2940|26585|16544|673742|2940|26585|2|42|7|1|10486|39|814|62|80.78|83.20|69.05|877.30|4281.10|5008.36|5158.40|82.19|3253.63|1495.44|1027.47|1109.66|2522.91|2605.10|-3980.89| +2450830|69751|2450876|16544|673742|2940|26585|16544|673742|2940|26585|2|93|15|1|12652|88|814|37|97.29|135.23|124.41|400.34|4603.17|3599.73|5003.51|230.15|0.00|1300.55|4603.17|4833.32|5903.72|6133.87|1003.44| +2450830|69751|2450874|16544|673742|2940|26585|16544|673742|2940|26585|2|29|15|1|11149|299|814|21|28.66|64.77|1.29|1333.08|27.09|601.86|1360.17|1.08|0.00|408.03|27.09|28.17|435.12|436.20|-574.77| +2450830|69751|2450860|16544|673742|2940|26585|16544|673742|2940|26585|2|102|20|1|3961|35|814|31|57.88|143.54|27.27|3604.37|845.37|1794.28|4449.74|36.35|118.35|1423.83|727.02|763.37|2150.85|2187.20|-1067.26| +2450830|69751|2450911|16544|673742|2940|26585|16544|673742|2940|26585|2|74|10|2|7918|49|814|93|50.83|86.41|38.88|4420.29|3615.84|4727.19|8036.13|216.95|0.00|1365.24|3615.84|3832.79|4981.08|5198.03|-1111.35| +2450830|69751|2450888|16544|673742|2940|26585|16544|673742|2940|26585|2|9|15|3|8680|257|814|13|13.58|19.82|10.70|118.56|139.10|176.54|257.66|9.73|0.00|126.23|139.10|148.83|265.33|275.06|-37.44| +2450830|69751|2450846|16544|673742|2940|26585|16544|673742|2940|26585|2|61|3|4|16948|44|814|14|34.93|91.86|74.40|244.44|1041.60|489.02|1286.04|62.49|0.00|25.62|1041.60|1104.09|1067.22|1129.71|552.58| +2450830|69751|2450894|16544|673742|2940|26585|16544|673742|2940|26585|2|62|5|3|11072|97|814|80|27.11|79.97|24.79|4414.40|1983.20|2168.80|6397.60|59.49|0.00|2814.40|1983.20|2042.69|4797.60|4857.09|-185.60| +2450830|69751|2450836|16544|673742|2940|26585|16544|673742|2940|26585|2|79|5|4|17830|18|814|42|82.74|153.89|32.31|5106.36|1357.02|3475.08|6463.38|94.99|0.00|3037.44|1357.02|1452.01|4394.46|4489.45|-2118.06| +2450830|46493|2450836|5130|584501|5472|11290|5130|584501|5472|11290|1|43|13|4|15104|161|815|18|93.46|108.41|5.42|1853.82|97.56|1682.28|1951.38|3.90|0.00|58.50|97.56|101.46|156.06|159.96|-1584.72| +2450830|46493|2450865|5130|584501|5472|11290|5130|584501|5472|11290|1|75|15|4|11779|137|815|85|6.98|17.31|1.73|1324.30|147.05|593.30|1471.35|5.88|0.00|544.00|147.05|152.93|691.05|696.93|-446.25| +2450830|46493|2450861|5130|584501|5472|11290|5130|584501|5472|11290|1|100|2|3|17446|257|815|24|90.66|197.63|120.55|1849.92|2893.20|2175.84|4743.12|173.59|0.00|1897.20|2893.20|3066.79|4790.40|4963.99|717.36| +2450830|46493|2450858|5130|584501|5472|11290|5130|584501|5472|11290|1|13|14|4|9130|202|815|2|50.88|152.64|7.63|290.02|15.26|101.76|305.28|0.76|0.00|143.48|15.26|16.02|158.74|159.50|-86.50| +2450830|46493|2450888|5130|584501|5472|11290|5130|584501|5472|11290|1|84|19|4|98|84|815|74|89.16|177.42|85.16|6827.24|6301.84|6597.84|13129.08|252.07|0.00|2362.82|6301.84|6553.91|8664.66|8916.73|-296.00| +2450830|46493|2450911|5130|584501|5472|11290|5130|584501|5472|11290|1|73|14|3|7756|233|815|87|58.90|110.14|34.14|6612.00|2970.18|5124.30|9582.18|240.58|297.01|4791.09|2673.17|2913.75|7464.26|7704.84|-2451.13| +2450830|46493|2450865|5130|584501|5472|11290|5130|584501|5472|11290|1|103|11|4|5216|40|815|26|33.08|57.89|1.73|1460.16|44.98|860.08|1505.14|0.44|0.00|692.12|44.98|45.42|737.10|737.54|-815.10| +2450830|20078|2450866|3739|419811|3083|1908|3739|419811|3083|1908|2|27|15|1|8204|256|816|20|65.41|187.07|7.48|3591.80|149.60|1308.20|3741.40|10.47|0.00|1758.40|149.60|160.07|1908.00|1918.47|-1158.60| +2450830|20078|2450865|3739|419811|3083|1908|3739|419811|3083|1908|2|48|9|2|4885|241|816|87|6.08|17.69|9.02|754.29|784.74|528.96|1539.03|44.73|39.23|538.53|745.51|790.24|1284.04|1328.77|216.55| +2450830|20078|2450860|3739|419811|3083|1908|3739|419811|3083|1908|2|45|5|1|12823|165|816|81|58.03|110.25|4.41|8573.04|357.21|4700.43|8930.25|0.00|0.00|267.30|357.21|357.21|624.51|624.51|-4343.22| +2450830|20078|2450903|3739|419811|3083|1908|3739|419811|3083|1908|2|46|5|5|8684|300|816|97|49.27|118.24|59.12|5734.64|5734.64|4779.19|11469.28|0.00|0.00|1834.27|5734.64|5734.64|7568.91|7568.91|955.45| +2450830|20078|2450835|3739|419811|3083|1908|3739|419811|3083|1908|2|1|6|1|13294|204|816|35|97.60|292.80|243.02|1742.30|8505.70|3416.00|10248.00|168.41|6634.44|1024.80|1871.26|2039.67|2896.06|3064.47|-1544.74| +2450830|20078|2450844|3739|419811|3083|1908|3739|419811|3083|1908|2|35|11|1|3248|123|816|92|85.70|133.69|64.17|6395.84|5903.64|7884.40|12299.48|118.07|0.00|3074.64|5903.64|6021.71|8978.28|9096.35|-1980.76| +2450830|20078|2450863|3739|419811|3083|1908|3739|419811|3083|1908|2|29|6|1|4276|273|816|27|2.68|3.53|1.55|53.46|41.85|72.36|95.31|1.67|0.00|20.79|41.85|43.52|62.64|64.31|-30.51| +2450830|20078|2450852|3739|419811|3083|1908|3739|419811|3083|1908|2|107|20|5|1040|60|816|76|29.90|39.16|34.06|387.60|2588.56|2272.40|2976.16|21.74|2044.96|208.24|543.60|565.34|751.84|773.58|-1728.80| +2450830|20078|2450906|3739|419811|3083|1908|3739|419811|3083|1908|2|32|4|5|160|4|816|86|45.84|63.71|2.54|5260.62|218.44|3942.24|5479.06|10.92|0.00|821.30|218.44|229.36|1039.74|1050.66|-3723.80| +2450830|20078|2450835|3739|419811|3083|1908|3739|419811|3083|1908|2|28|10|2|2332|276|816|67|1.31|2.81|0.95|124.62|63.65|87.77|188.27|1.90|0.00|86.43|63.65|65.55|150.08|151.98|-24.12| +2450830|20078|2450891|3739|419811|3083|1908|3739|419811|3083|1908|2|73|12|4|12856|150|816|41|12.68|35.50|8.87|1091.83|363.67|519.88|1455.50|0.00|0.00|596.55|363.67|363.67|960.22|960.22|-156.21| +2450830|20078|2450893|3739|419811|3083|1908|3739|419811|3083|1908|2|82|19|1|10663|250|816|88|80.01|151.21|25.70|11044.88|2261.60|7040.88|13306.48|0.00|0.00|3059.76|2261.60|2261.60|5321.36|5321.36|-4779.28| +2450830|74670|2450876|74411|578576|3988|27800|30341|1248224|1989|10201|4|51|16|1|6274|158|817|100|78.38|205.35|172.49|3286.00|17249.00|7838.00|20535.00|103.49|13799.20|4312.00|3449.80|3553.29|7761.80|7865.29|-4388.20| +2450830|74670|2450871|74411|578576|3988|27800|30341|1248224|1989|10201|4|97|2|2|4742|251|817|65|2.94|5.40|3.07|151.45|199.55|191.10|351.00|13.96|0.00|27.95|199.55|213.51|227.50|241.46|8.45| +2450830|74670|2450847|74411|578576|3988|27800|30341|1248224|1989|10201|4|17|5|1|3523|262|817|83|80.71|213.07|59.65|12733.86|4950.95|6698.93|17684.81|0.00|0.00|3890.21|4950.95|4950.95|8841.16|8841.16|-1747.98| +2450830|74670|2450844|74411|578576|3988|27800|30341|1248224|1989|10201|4|61|11|5|661|226|817|79|15.72|19.49|0.97|1463.08|76.63|1241.88|1539.71|0.05|71.26|169.06|5.37|5.42|174.43|174.48|-1236.51| +2450830|74670|2450847|74411|578576|3988|27800|30341|1248224|1989|10201|4|74|4|4|469|54|817|11|99.01|202.97|180.64|245.63|1987.04|1089.11|2232.67|18.08|1728.72|1093.95|258.32|276.40|1352.27|1370.35|-830.79| +2450830|74670|2450914|74411|578576|3988|27800|30341|1248224|1989|10201|4|86|2|5|17890|156|817|40|48.17|113.68|96.62|682.40|3864.80|1926.80|4547.20|37.10|2937.24|500.00|927.56|964.66|1427.56|1464.66|-999.24| +2450830|74670|2450886|74411|578576|3988|27800|30341|1248224|1989|10201|4|99|2|5|482|223|817|48|92.82|171.71|154.53|824.64|7417.44|4455.36|8242.08|370.87|0.00|1648.32|7417.44|7788.31|9065.76|9436.63|2962.08| +2450830|74670|2450879|74411|578576|3988|27800|30341|1248224|1989|10201|4|71|13|2|4789|147|817|50|98.16|266.99|53.39|10680.00|2669.50|4908.00|13349.50|53.39|0.00|1735.00|2669.50|2722.89|4404.50|4457.89|-2238.50| +2450831|73238|2450888|42146|1242808|4717|4179|42146|1242808|4717|4179|1|69|2|2|7250|200|818|14|5.68|12.04|10.47|21.98|146.58|79.52|168.56|13.19|0.00|33.60|146.58|159.77|180.18|193.37|67.06| +2450831|73238|2450877|42146|1242808|4717|4179|42146|1242808|4717|4179|1|85|3|4|16610|209|818|46|69.85|99.88|78.90|965.08|3629.40|3213.10|4594.48|326.64|0.00|1791.70|3629.40|3956.04|5421.10|5747.74|416.30| +2450831|73238|2450852|42146|1242808|4717|4179|42146|1242808|4717|4179|1|71|17|5|9502|206|818|53|72.25|194.35|87.45|5665.70|4634.85|3829.25|10300.55|417.13|0.00|3296.07|4634.85|5051.98|7930.92|8348.05|805.60| +2450831|73238|2450883|42146|1242808|4717|4179|42146|1242808|4717|4179|1|47|18|5|6494|189|818|98|49.37|102.68|78.03|2415.70|7646.94|4838.26|10062.64|85.64|6423.42|3119.34|1223.52|1309.16|4342.86|4428.50|-3614.74| +2450831|73238|2450894|42146|1242808|4717|4179|42146|1242808|4717|4179|1|38|2|1|4700|248|818|14|52.73|64.33|9.64|765.66|134.96|738.22|900.62|2.69|0.00|8.96|134.96|137.65|143.92|146.61|-603.26| +2450831|73238|2450906|42146|1242808|4717|4179|42146|1242808|4717|4179|1|30|10|5|9050|29|818|36|1.20|2.23|2.05|6.48|73.80|43.20|80.28|1.47|0.00|29.52|73.80|75.27|103.32|104.79|30.60| +2450831|73238|2450920|42146|1242808|4717|4179|42146|1242808|4717|4179|1|30|2|2|404|162|818|99|23.15|57.64|48.41|913.77|4792.59|2291.85|5706.36|25.88|4505.03|341.55|287.56|313.44|629.11|654.99|-2004.29| +2450831|73238|2450846|42146|1242808|4717|4179|42146|1242808|4717|4179|1|74|4|3|400|284|818|85|70.67|178.79|94.75|7143.40|8053.75|6006.95|15197.15|483.22|0.00|607.75|8053.75|8536.97|8661.50|9144.72|2046.80| +2450831|73238|2450864|42146|1242808|4717|4179|42146|1242808|4717|4179|1|73|1|4|15716|186|818|16|21.25|63.53|60.35|50.88|965.60|340.00|1016.48|9.65|0.00|10.08|965.60|975.25|975.68|985.33|625.60| +2450831|73238|2450918|42146|1242808|4717|4179|42146|1242808|4717|4179|1|50|6|2|13238|264|818|84|53.30|139.11|108.50|2571.24|9114.00|4477.20|11685.24|364.56|0.00|3505.32|9114.00|9478.56|12619.32|12983.88|4636.80| +2450831|73238|2450919|42146|1242808|4717|4179|42146|1242808|4717|4179|1|9|4|1|17510|47|818|39|52.71|68.52|53.44|588.12|2084.16|2055.69|2672.28|0.00|0.00|721.50|2084.16|2084.16|2805.66|2805.66|28.47| +2450831|73238|2450918|42146|1242808|4717|4179|42146|1242808|4717|4179|1|83|10|1|9499|37|818|3|45.31|125.96|47.86|234.30|143.58|135.93|377.88|10.05|0.00|102.00|143.58|153.63|245.58|255.63|7.65| +2450831|26170|2450845|27248|21584|5167|26703|27248|21584|5167|26703|4|108|14|1|4856|19|819|97|54.60|145.23|114.73|2958.50|11128.81|5296.20|14087.31|0.00|3116.06|4648.24|8012.75|8012.75|12660.99|12660.99|2716.55| +2450831|26170|2450887|27248|21584|5167|26703|27248|21584|5167|26703|4|10|3|4|9812|20|819|13|17.87|31.45|27.04|57.33|351.52|232.31|408.85|0.00|0.00|73.58|351.52|351.52|425.10|425.10|119.21| +2450831|26170|2450841|27248||5167|26703|27248||5167||4||2|3|16834||819||83.24|198.94||||4328.48||0.00|0.00|3103.36|3310.32|3310.32|6413.68|6413.68|-1018.16| +2450831|26170|2450905|27248|21584|5167|26703|27248|21584|5167|26703|4|4|4|2|12037|113|819|34|82.30|83.12|72.31|367.54|2458.54|2798.20|2826.08|122.92|0.00|875.84|2458.54|2581.46|3334.38|3457.30|-339.66| +2450831|26170|2450901|27248|21584|5167|26703|27248|21584|5167|26703|4|99|20|5|14396|120|819|1|71.89|74.04|56.27|17.77|56.27|71.89|74.04|1.12|0.00|11.84|56.27|57.39|68.11|69.23|-15.62| +2450831|26170|2450903|27248|21584|5167|26703|27248|21584|5167|26703|4|82|12|1|1018|247|819|100|75.07|204.94|178.29|2665.00|17829.00|7507.00|20494.00|356.58|0.00|4508.00|17829.00|18185.58|22337.00|22693.58|10322.00| +2450831|26170|2450881|27248|21584|5167|26703|27248|21584|5167|26703|4|53|5|4|16045|244|819|71|66.04|192.17|172.95|1364.62|12279.45|4688.84|13644.07|0.00|0.00|272.64|12279.45|12279.45|12552.09|12552.09|7590.61| +2450831|26170|2450874|27248|21584|5167|26703|27248|21584|5167|26703|4|97|12|5|3862|64|819|82|56.97|138.43|71.98|5448.90|5902.36|4671.54|11351.26|118.04|0.00|3631.78|5902.36|6020.40|9534.14|9652.18|1230.82| +2450831|26170|2450862|27248|21584|5167|26703|27248|21584|5167|26703|4|85|6|5|11264|131|819|39|36.00|77.40|41.02|1418.82|1599.78|1404.00|3018.60|127.98|0.00|452.79|1599.78|1727.76|2052.57|2180.55|195.78| +2450831|63563|2450891|84670|488914|3205|34296|74505|1866078|4369|45013|4|88|17|3|15331|297|820|4|71.23|127.50|21.67|423.32|86.68|284.92|510.00|0.00|0.00|76.48|86.68|86.68|163.16|163.16|-198.24| +2450831|63563|2450921|84670|488914|3205|34296|74505|1866078|4369|45013|4|60|3|5|2017|211|820|3|19.09|26.34|23.44|8.70|70.32|57.27|79.02|6.32|0.00|15.78|70.32|76.64|86.10|92.42|13.05| +2450831|63563|2450887|84670|488914|3205|34296|74505|1866078|4369|45013|4|40|16|3|5398|6|820|10|82.08|92.75|4.63|881.20|46.30|820.80|927.50|0.92|0.00|426.60|46.30|47.22|472.90|473.82|-774.50| +2450831|63563|2450845|84670|488914|3205|34296|74505|1866078|4369|45013|4|62|4|2|17248|123|820|81|79.08|196.90|106.32|7336.98|8611.92|6405.48|15948.90|775.07|0.00|7176.60|8611.92|9386.99|15788.52|16563.59|2206.44| +2450831|63563|2450903|84670|488914|3205|34296|74505|1866078|4369|45013|4|82|11|4|811|147|820|15|2.89|6.47|1.48|74.85|22.20|43.35|97.05|0.72|7.77|34.80|14.43|15.15|49.23|49.95|-28.92| +2450831|63563|2450867|84670|488914|3205|34296|74505|1866078|4369|45013|4|15|18|2|7072|293|820|12|83.82|132.43|27.81|1255.44|333.72|1005.84|1589.16|0.66|300.34|397.20|33.38|34.04|430.58|431.24|-972.46| +2450831|63563|2450878|84670|488914|3205|34296|74505|1866078|4369|45013|4|35|20|1|12452|40|820|73|80.52|114.33|52.59|4507.02|3839.07|5877.96|8346.09|76.78|0.00|83.22|3839.07|3915.85|3922.29|3999.07|-2038.89| +2450831|63563|2450918|84670|488914|3205|34296|74505|1866078|4369|45013|4|3|6|5|1610|45|820|22|35.45|106.35|7.44|2176.02|163.68|779.90|2339.70|14.73|0.00|654.94|163.68|178.41|818.62|833.35|-616.22| +2450831|56046|2450889|53817|1112487|3157|41380|53817|1112487|3157|41380|2|102|17|1|15823|97|821|19|20.75|30.71|11.97|356.06|227.43|394.25|583.49|18.19|0.00|17.48|227.43|245.62|244.91|263.10|-166.82| +2450831|56046|2450878|53817|1112487|3157|41380|53817|1112487|3157|41380|2|38|13|5|11329|179|821|96|63.31|131.05|117.94|1258.56|11322.24|6077.76|12580.80|1019.00|0.00|2263.68|11322.24|12341.24|13585.92|14604.92|5244.48| +2450831|56046|2450837|53817|1112487|3157|41380|53817|1112487|3157|41380|2|6|16|4|7837|173|821|30|67.53|176.25|141.00|1057.50|4230.00|2025.90|5287.50|211.50|0.00|264.30|4230.00|4441.50|4494.30|4705.80|2204.10| +2450831|56046|2450883|53817|1112487|3157|41380|53817|1112487|3157|41380|2|94|4|2|12014|36|821|61|48.41|126.35|73.28|3237.27|4470.08|2953.01|7707.35|68.39|2190.33|3159.80|2279.75|2348.14|5439.55|5507.94|-673.26| +2450831|56046|2450875|53817|1112487|3157|41380|53817|1112487|3157|41380|2|105|11|5|13694|31|821|94|21.48|54.55|45.82|820.62|4307.08|2019.12|5127.70|129.21|0.00|1742.76|4307.08|4436.29|6049.84|6179.05|2287.96| +2450831|63263|2450849|87999|162328|6610|45739|87999|162328|6610|45739|1|35|17|4|2636|248|822|73|30.26|81.09|78.65|178.12|5741.45|2208.98|5919.57|0.00|1952.09|59.13|3789.36|3789.36|3848.49|3848.49|1580.38| +2450831|63263|2450897|87999|162328|6610|45739|87999|162328|6610|45739|1|52|8|2|1993|248|822|77|78.14|222.69|162.56|4630.01|12517.12|6016.78|17147.13|125.17|0.00|3943.17|12517.12|12642.29|16460.29|16585.46|6500.34| +2450831|63263|2450861|87999|162328|6610|45739|87999|162328|6610|45739|1|69|10|5|2665|103|822|38|87.24|174.48|62.81|4243.46|2386.78|3315.12|6630.24|0.00|0.00|397.48|2386.78|2386.78|2784.26|2784.26|-928.34| +2450831|63263|2450910|87999|162328|6610|45739|87999|162328|6610|45739|1|94|6|2|17312|227|822|12|96.96|178.40|155.20|278.40|1862.40|1163.52|2140.80|22.90|1098.81|920.52|763.59|786.49|1684.11|1707.01|-399.93| +2450831|63263|2450846|87999|162328|6610|45739|87999|162328|6610|45739|1|83|1|3|11080|78|822|66|10.54|15.70|0.94|974.16|62.04|695.64|1036.20|1.24|0.00|279.18|62.04|63.28|341.22|342.46|-633.60| +2450831|63263|2450919|87999|162328|6610|45739|87999|162328|6610|45739|1|50|12|4|6514|96|822|18|11.61|16.95|4.40|225.90|79.20|208.98|305.10|0.79|0.00|106.74|79.20|79.99|185.94|186.73|-129.78| +2450831|53949|2450915|21624|1829381|5138|32895|21624|1829381|5138|32895|2|105|14|2|17485|25|823|63|63.94|156.65|9.39|9277.38|591.57|4028.22|9868.95|0.00|0.00|4045.86|591.57|591.57|4637.43|4637.43|-3436.65| +2450831|53949|2450839|21624|1829381|5138|32895|21624|1829381|5138|32895|2|52|3|5|163|14|823|4|56.69|140.59|53.42|348.68|213.68|226.76|562.36|1.70|170.94|281.16|42.74|44.44|323.90|325.60|-184.02| +2450831|53949|2450868|21624|1829381|5138|32895|21624|1829381|5138|32895|2|35|9|3|12235|166|823|87|12.80|22.91|7.78|1316.31|676.86|1113.60|1993.17|6.76|0.00|398.46|676.86|683.62|1075.32|1082.08|-436.74| +2450831|53949|2450835|21624|1829381|5138|32895|21624|1829381|5138|32895|2|89|14|3|706|211|823|39|86.93|206.89|169.64|1452.75|6615.96|3390.27|8068.71|529.27|0.00|322.53|6615.96|7145.23|6938.49|7467.76|3225.69| +2450831|53949|2450855|21624|1829381|5138|32895|21624|1829381|5138|32895|2|55|19|1|6466|25|823|40|39.52|67.97|67.29|27.20|2691.60|1580.80|2718.80|11.30|2530.10|1141.60|161.50|172.80|1303.10|1314.40|-1419.30| +2450831|53949|2450868|21624|1829381|5138|32895|21624|1829381|5138|32895|2|75|2|2|17318|244|823|45|5.94|7.00|1.33|255.15|59.85|267.30|315.00|0.00|0.00|18.90|59.85|59.85|78.75|78.75|-207.45| +2450831|45116|2450914|94362|1503502|6689|1029|94362|1503502|6689|1029|2|72|20|5|5374|202|824|3|20.81|54.52|38.16|49.08|114.48|62.43|163.56|1.14|0.00|68.67|114.48|115.62|183.15|184.29|52.05| +2450831|45116|2450905|94362|1503502|6689|1029|94362|1503502|6689|1029|2|41|19|1|16270|222|824|21|66.67|90.67|11.78|1656.69|247.38|1400.07|1904.07|9.89|0.00|57.12|247.38|257.27|304.50|314.39|-1152.69| +2450831|45116|2450912|94362|1503502|6689|1029|94362|1503502|6689|1029|2|1|11|3|15556|151|824|82|65.40|124.91|38.72|7067.58|3175.04|5362.80|10242.62|190.50|0.00|3584.22|3175.04|3365.54|6759.26|6949.76|-2187.76| +|45116|2450890|94362|||||1503502|6689||2|84||2|16214|209|824|56|73.03||104.28|1112.72||4089.68|||0.00||5839.68|5898.07|6048.00|6106.39|1750.00| +2450831|45116|2450908|94362|1503502|6689|1029|94362|1503502|6689|1029|2|54|14|1|10705|125|824|29|10.29|22.02|1.98|581.16|57.42|298.41|638.58|0.00|0.00|229.68|57.42|57.42|287.10|287.10|-240.99| +2450831|45116|2450870|94362|1503502|6689|1029|94362|1503502|6689|1029|2|2|11|2|9886|94|824|84|47.84|115.77|47.46|5738.04|3986.64|4018.56|9724.68|318.93|0.00|1264.20|3986.64|4305.57|5250.84|5569.77|-31.92| +2450831|71239|2450902|47438|1065538|3638|6891|47438|1065538|3638|6891|4|48|20|3|3340|11|825|37|88.23|132.34|48.96|3085.06|1811.52|3264.51|4896.58|18.11|0.00|1762.68|1811.52|1829.63|3574.20|3592.31|-1452.99| +2450831|71239|2450881|47438|1065538|3638|6891|47438|1065538|3638|6891|4|92|14|3|13351|293|825|23|90.43|264.95|219.90|1036.15|5057.70|2079.89|6093.85|354.03|0.00|1035.92|5057.70|5411.73|6093.62|6447.65|2977.81| +2450831|71239|2450881|47438|1065538|3638|6891|47438|1065538|3638|6891|4|55|14|4|16034|24|825|85|39.28|97.02|95.07|165.75|8080.95|3338.80|8246.70|375.76|565.66|1071.85|7515.29|7891.05|8587.14|8962.90|4176.49| +2450831|71239|2450849|47438|1065538|3638|6891|47438|1065538|3638|6891|4|64|8|3|6454|173|825|27|7.86|16.42|15.59|22.41|420.93|212.22|443.34|7.15|63.13|57.51|357.80|364.95|415.31|422.46|145.58| +2450831|71239|2450898|47438|1065538|3638|6891|47438|1065538|3638|6891|4|81|20|4|4423|50|825|64|27.04|45.96|4.13|2677.12|264.32|1730.56|2941.44|21.14|0.00|735.36|264.32|285.46|999.68|1020.82|-1466.24| +2450831|71239|2450876|47438|1065538|3638|6891|47438|1065538|3638|6891|4|70|2|4|16357|153|825|46|94.71|107.02|99.52|345.00|4577.92|4356.66|4922.92|274.67|0.00|1919.58|4577.92|4852.59|6497.50|6772.17|221.26| +2450831|71239|2450853|47438|1065538|3638|6891|47438|1065538|3638|6891|4|6|6|2|13106|82|825|11|49.89|127.71|125.15|28.16|1376.65|548.79|1404.81|18.72|440.52|604.01|936.13|954.85|1540.14|1558.86|387.34| +2450831|71239|2450902|47438|1065538|3638|6891|47438|1065538|3638|6891|4|108|4|2|8206|240|825|13|8.47|10.58|2.22|108.68|28.86|110.11|137.54|0.09|19.62|21.97|9.24|9.33|31.21|31.30|-100.87| +2450831|71239|2450894|47438|1065538|3638|6891|47438|1065538|3638|6891|4|96|2|1|9166|74|825|73|63.99|186.21|87.51|7205.10|6388.23|4671.27|13593.33|447.17|0.00|0.00|6388.23|6835.40|6388.23|6835.40|1716.96| +2450831|71239|2450891|47438|1065538|3638|6891|47438|1065538|3638|6891|4|43|9|5|13462|64|825|47|17.52|35.91|12.56|1097.45|590.32|823.44|1687.77|2.65|537.19|708.76|53.13|55.78|761.89|764.54|-770.31| +2450831|71239|2450890|47438|1065538|3638|6891|47438|1065538|3638|6891|4|44|15|4|1190|126|825|94|89.31|200.05|74.01|11847.76|6956.94|8395.14|18804.70|200.36|4452.44|1880.00|2504.50|2704.86|4384.50|4584.86|-5890.64| +2450831|71239|2450852|47438|1065538|3638|6891|47438|1065538|3638|6891|4|45|18|2|13076|222|825|67|63.27|127.80|53.67|4966.71|3595.89|4239.09|8562.60|98.16|323.63|2568.78|3272.26|3370.42|5841.04|5939.20|-966.83| +2450831|70696|2450860|80395|467584|5173|43515|80395|467584|5173|43515|1|107|6|4|17197|227|826|73|95.54|104.13|42.69|4485.12|3116.37|6974.42|7601.49|12.46|2804.73|684.01|311.64|324.10|995.65|1008.11|-6662.78| +2450831|70696|2450881|80395|467584|5173|43515|80395|467584|5173|43515|1|33|5|2|14164|204|826|98|33.02|93.44|48.58|4396.28|4760.84|3235.96|9157.12|285.65|0.00|3936.66|4760.84|5046.49|8697.50|8983.15|1524.88| +2450831|70696|2450900|80395|467584|5173|43515|80395|467584|5173|43515|1|81|19|5|2389|12|826|4|97.77|210.20|115.61|378.36|462.44|391.08|840.80|0.00|161.85|218.60|300.59|300.59|519.19|519.19|-90.49| +2450831|70696|2450881|80395|467584|5173|43515|80395|467584|5173|43515|1|93|13|2|13732|185|826|70|93.45|214.00|149.80|4494.00|10486.00|6541.50|14980.00|734.02|0.00|7340.20|10486.00|11220.02|17826.20|18560.22|3944.50| +2450831|70696|2450913|80395|467584|5173|43515|80395|467584|5173|43515|1|89|14|4|12577|132|826|17|96.18|178.89|100.17|1338.24|1702.89|1635.06|3041.13|0.00|0.00|821.10|1702.89|1702.89|2523.99|2523.99|67.83| +2450831|70696|2450875|80395|467584|5173|43515|80395|467584|5173|43515|1|45|7|4|3056|70|826|88|40.64|64.61|60.73|341.44|5344.24|3576.32|5685.68|427.53|0.00|511.28|5344.24|5771.77|5855.52|6283.05|1767.92| +2450831|70696|2450840|80395|467584|5173|43515|80395|467584|5173|43515|1|73|20|5|4778|298|826|41|62.20|72.15|14.43|2366.52|591.63|2550.20|2958.15|0.00|0.00|739.23|591.63|591.63|1330.86|1330.86|-1958.57| +2450831|70696|2450884|80395|467584|5173|43515|80395|467584|5173|43515|1|8|20|3|2216|201|826|85|21.82|43.42|28.22|1292.00|2398.70|1854.70|3690.70|76.75|863.53|1291.15|1535.17|1611.92|2826.32|2903.07|-319.53| +2450831|70696|2450864|80395|467584|5173|43515|80395|467584|5173|43515|1|18|20|2|286|149|826|15|23.30|59.88|31.73|422.25|475.95|349.50|898.20|11.13|104.70|242.40|371.25|382.38|613.65|624.78|21.75| +2450831|32902|2450845|24837|1143027|6370|24560|24787|1063693|2683|18426|2|95|17|4|13580|215|827|41|14.65|30.91|21.32|393.19|874.12|600.65|1267.31|52.44|0.00|227.96|874.12|926.56|1102.08|1154.52|273.47| +2450831|32902|2450877|24837|1143027|6370|24560|24787|1063693|2683|18426|2|10|12|2|17354|246|827|7|52.91|106.34|103.14|22.40|721.98|370.37|744.38|7.21|0.00|178.64|721.98|729.19|900.62|907.83|351.61| +2450831|32902|2450915|24837|1143027|6370|24560|24787|1063693|2683|18426|2|107|16|2|14690|146|827|73|51.15|68.54|59.62|651.16|4352.26|3733.95|5003.42|87.04|0.00|2151.31|4352.26|4439.30|6503.57|6590.61|618.31| +2450831|32902|2450860|24837|1143027|6370|24560|24787|1063693|2683|18426|2|88|16|5|3598|165|827|14|21.16|28.14|3.37|346.78|47.18|296.24|393.96|1.88|0.00|189.00|47.18|49.06|236.18|238.06|-249.06| +2450831|43739|2450900|25842|289093|344|3948|25842|289093|344|3948|2|5|1|1|248|221|828|45|32.08|87.57|25.39|2798.10|1142.55|1443.60|3940.65|11.42|0.00|1851.75|1142.55|1153.97|2994.30|3005.72|-301.05| +2450831|43739|2450909|25842|289093|344|3948|25842|289093|344|3948|2|85|7|5|16070|66|828|11|98.61|194.26|7.77|2051.39|85.47|1084.71|2136.86|4.27|0.00|534.16|85.47|89.74|619.63|623.90|-999.24| +2450831|43739|2450902|25842|289093|344|3948|25842|289093|344|3948|2|76|13|2|4888|115|828|98|6.05|14.58|5.97|843.78|585.06|592.90|1428.84|23.40|0.00|142.10|585.06|608.46|727.16|750.56|-7.84| +2450831|43739|2450853|25842|289093|344|3948|25842|289093|344|3948|2|44|12|2|9746|68|828|93|39.85|110.38|97.13|1232.25|9033.09|3706.05|10265.34|455.26|1445.29|1231.32|7587.80|8043.06|8819.12|9274.38|3881.75| +2450831|14338|2450896|94578|1161589|6200|5611|94578|1161589|6200|5611|1|57|9|4|17366|142|829|25|72.90|191.72|153.37|958.75|3834.25|1822.50|4793.00|115.02|0.00|1629.50|3834.25|3949.27|5463.75|5578.77|2011.75| +2450831|14338|2450838|94578|1161589|6200|5611|94578|1161589|6200|5611|1|31|7|4|1651|184|829|57|40.19|116.14|81.29|1986.45|4633.53|2290.83|6619.98|370.68|0.00|2647.65|4633.53|5004.21|7281.18|7651.86|2342.70| +2450831|14338|2450878|94578|1161589|6200|5611|94578|1161589|6200|5611|1|61|18|3|4519|255|829|41|74.69|106.05|98.62|304.63|4043.42|3062.29|4348.05|283.03|0.00|1782.68|4043.42|4326.45|5826.10|6109.13|981.13| +2450831|14338|2450885|94578|1161589|6200|5611|94578|1161589|6200|5611|1|12|11|2|10009|17|829|70|28.13|81.01|34.02|3289.30|2381.40|1969.10|5670.70|71.44|0.00|623.70|2381.40|2452.84|3005.10|3076.54|412.30| +2450831|14338|2450879|94578|1161589|6200|5611|94578|1161589|6200|5611|1|46|8|2|13984|109|829|37|90.05|221.52|75.31|5409.77|2786.47|3331.85|8196.24|83.59|0.00|3524.25|2786.47|2870.06|6310.72|6394.31|-545.38| +2450831|14338|2450876|94578|1161589|6200|5611|94578|1161589|6200|5611|1|65|5|1|6130|178|829|43|47.25|115.76|38.20|3335.08|1642.60|2031.75|4977.68|9.85|1149.82|2289.32|492.78|502.63|2782.10|2791.95|-1538.97| +2450831|14338|2450863|94578|1161589|6200|5611|94578|1161589|6200|5611|1|32|12|1|10813|82|829|35|78.78|139.44|115.73|829.85|4050.55|2757.30|4880.40|0.00|0.00|0.00|4050.55|4050.55|4050.55|4050.55|1293.25| +2450831|14338|2450869|94578|1161589|6200|5611|94578|1161589|6200|5611|1|72|1|2|15472|66|829|39|61.39|62.61|30.67|1245.66|1196.13|2394.21|2441.79|23.92|0.00|1147.38|1196.13|1220.05|2343.51|2367.43|-1198.08| +2450831|14338|2450921|94578|1161589|6200|5611|94578|1161589|6200|5611|1|37|20|1|11290|250|829|75|58.49|91.82|21.11|5303.25|1583.25|4386.75|6886.50|94.99|0.00|2340.75|1583.25|1678.24|3924.00|4018.99|-2803.50| +2450831|14338|2450847|94578|1161589|6200|5611|94578|1161589|6200|5611|1|99|3|3|12724|135|829|100|21.34|27.95|1.67|2628.00|167.00|2134.00|2795.00|6.68|0.00|1257.00|167.00|173.68|1424.00|1430.68|-1967.00| +2450831|14338|2450843|94578|1161589|6200|5611|94578|1161589|6200|5611|1|46|16|3|1004|275|829|78|19.98|53.54|17.66|2798.64|1377.48|1558.44|4176.12|123.97|0.00|1460.94|1377.48|1501.45|2838.42|2962.39|-180.96| +2450831|14338|2450853|94578|1161589|6200|5611|94578|1161589|6200|5611|1|34|17|1|2404|217|829|56|50.74|82.19|45.20|2071.44|2531.20|2841.44|4602.64|177.18|0.00|2070.88|2531.20|2708.38|4602.08|4779.26|-310.24| +2450831||||||37942|||1963||||17||17902|118|830|42|24.13|37.40||||1013.46|||0.00|62.58|||1240.68|1240.68|164.64| +2450831|58980|2450845|68852|1458031|1963|37942|68852|1458031|1963|37942|2|73|6|1|2546|115|830|53|57.02|111.18|53.36|3064.46|2828.08|3022.06|5892.54|197.96|0.00|2062.23|2828.08|3026.04|4890.31|5088.27|-193.98| +2450831|58980|2450869|68852|1458031|1963|37942|68852|1458031|1963|37942|2|44|4|3|5728|291|830|9|15.68|30.10|30.10|0.00|270.90|141.12|270.90|16.25|0.00|86.67|270.90|287.15|357.57|373.82|129.78| +2450831|58980|2450881|68852|1458031|1963|37942|68852|1458031|1963|37942|2|16|15|2|776|6|830|67|96.46|149.51|79.24|4708.09|5309.08|6462.82|10017.17|0.00|53.09|1702.47|5255.99|5255.99|6958.46|6958.46|-1206.83| +2450831|58980|2450852|68852|1458031|1963|37942|68852|1458031|1963|37942|2|15|16|2|16249|4|830|94|28.38|60.44|22.96|3523.12|2158.24|2667.72|5681.36|36.90|1748.17|908.98|410.07|446.97|1319.05|1355.95|-2257.65| +2450831|58980|2450844|68852|1458031|1963|37942|68852|1458031|1963|37942|2|73|10|1|16520|141|830|7|26.16|56.76|45.97|75.53|321.79|183.12|397.32|22.52|0.00|87.36|321.79|344.31|409.15|431.67|138.67| +2450831|14996|2450834|64256|1274354|3205|30103|64256|1274354|3205|30103|2|26|8|3|15968|124|831|7|5.24|6.91|1.24|39.69|8.68|36.68|48.37|0.00|2.08|22.68|6.60|6.60|29.28|29.28|-30.08| +2450831|14996|2450834|64256|1274354|3205|30103|64256|1274354|3205|30103|2|52|7|1|14842|26|831|64|75.99|104.86|1.04|6644.48|66.56|4863.36|6711.04|0.39|53.24|3221.12|13.32|13.71|3234.44|3234.83|-4850.04| +2450831|14996|2450909|64256|1274354|3205|30103|64256|1274354|3205|30103|2|95|14|3|16810|8|831|73|11.06|23.66|17.50|449.68|1277.50|807.38|1727.18|63.87|0.00|690.58|1277.50|1341.37|1968.08|2031.95|470.12| +2450831|14996|2450897|64256|1274354|3205|30103|64256|1274354|3205|30103|2|76|19|5|326|52|831|39|85.97|251.89|171.28|3143.79|6679.92|3352.83|9823.71|333.99|0.00|982.02|6679.92|7013.91|7661.94|7995.93|3327.09| +2450831|14996|2450891|64256|1274354|3205|30103|64256|1274354|3205|30103|2|36|9|4|1543|142|831|26|33.90|63.39|45.64|461.50|1186.64|881.40|1648.14|94.93|0.00|626.08|1186.64|1281.57|1812.72|1907.65|305.24| +2450831|14996|2450908|64256|1274354|3205|30103|64256|1274354|3205|30103|2|99|2|3|7754|37|831|90|68.01|194.50|180.88|1225.80|16279.20|6120.90|17505.00|162.79|0.00|2625.30|16279.20|16441.99|18904.50|19067.29|10158.30| +2450831|14996|2450878|64256|1274354|3205|30103|64256|1274354|3205|30103|2|19|1|3|1022|242|831|39|3.65|4.08|0.65|133.77|25.35|142.35|159.12|1.52|0.00|0.00|25.35|26.87|25.35|26.87|-117.00| +2450831|14996|2450909|64256|1274354|3205|30103|64256|1274354|3205|30103|2|107|9|5|17696|28|831|83|1.68|1.94|0.44|124.50|36.52|139.44|161.02|1.82|0.00|36.52|36.52|38.34|73.04|74.86|-102.92| +2450831|14996|2450888|64256|1274354|3205|30103|64256|1274354|3205|30103|2|48|5|2|9100|111|831|42|29.26|82.22|5.75|3211.74|241.50|1228.92|3453.24|0.00|0.00|172.62|241.50|241.50|414.12|414.12|-987.42| +2450831|14996|2450884|64256|1274354|3205|30103|64256|1274354|3205|30103|2|38|19|5|4354|282|831|28|29.56|78.62|29.87|1365.00|836.36|827.68|2201.36|41.81|0.00|572.32|836.36|878.17|1408.68|1450.49|8.68| +2450831|14996|2450896|64256|1274354|3205|30103|64256|1274354|3205|30103|2|50|7|1|2896|166|831|71|9.91|18.13|6.88|798.75|488.48|703.61|1287.23|43.96|0.00|166.85|488.48|532.44|655.33|699.29|-215.13| +2450831|41940|2450921|12491|1700427|3563|15099|12491|1700427|3563|15099|2|48|10|3|566|43|832|40|56.81|78.39|21.94|2258.00|877.60|2272.40|3135.60|26.32|0.00|376.00|877.60|903.92|1253.60|1279.92|-1394.80| +2450831|41940|2450910|12491|1700427|3563|15099|12491|1700427|3563|15099|2|14|3|5|14450|152|832|71|56.51|157.66|37.83|8507.93|2685.93|4012.21|11193.86|0.00|0.00|4700.91|2685.93|2685.93|7386.84|7386.84|-1326.28| +2450831|41940|2450913|12491|1700427|3563|15099|12491|1700427|3563|15099|2|24|4|3|7723|233|832|89|42.74|76.93|51.54|2259.71|4587.06|3803.86|6846.77|275.22|0.00|821.47|4587.06|4862.28|5408.53|5683.75|783.20| +2450831|41940|2450919|12491|1700427|3563|15099|12491|1700427|3563|15099|2|77|6|5|13568|203|832|58|16.68|30.35|4.85|1479.00|281.30|967.44|1760.30|11.25|0.00|615.96|281.30|292.55|897.26|908.51|-686.14| +2450831|41940|2450867|12491|1700427|3563|15099|12491|1700427|3563|15099|2|2|20|5|14371|155|832|97|10.69|24.58|12.53|1168.85|1215.41|1036.93|2384.26|60.77|0.00|738.17|1215.41|1276.18|1953.58|2014.35|178.48| +2450831|52289|2450866|45817|1604859|6301|28096|45817|1604859|6301|28096|2|26|18|3|11440|285|833|50|62.69|170.51|71.61|4945.00|3580.50|3134.50|8525.50|71.61|0.00|2898.50|3580.50|3652.11|6479.00|6550.61|446.00| +2450831|52289|2450862|45817|1604859|6301|28096|45817|1604859|6301|28096|2|31|1|3|199|112|833|64|32.34|46.89|43.13|240.64|2760.32|2069.76|3000.96|55.20|0.00|119.68|2760.32|2815.52|2880.00|2935.20|690.56| +2450831|52289|2450861|45817|1604859|6301|28096|45817|1604859|6301|28096|2|69|17|1|2498|259|833|74|5.82|10.65|6.17|331.52|456.58|430.68|788.10|31.96|0.00|15.54|456.58|488.54|472.12|504.08|25.90| +2450831|52289|2450903|45817|1604859|6301|28096|45817|1604859|6301|28096|2|11|16|1|4207|72|833|1|87.62|250.59|177.91|72.68|177.91|87.62|250.59|0.00|0.00|105.24|177.91|177.91|283.15|283.15|90.29| +2450831|52289|2450900|45817|1604859|6301|28096|45817|1604859|6301|28096|2|1|5|2|3289|112|833|26|4.76|9.09|0.27|229.32|7.02|123.76|236.34|0.00|0.00|118.04|7.02|7.02|125.06|125.06|-116.74| +2450831|52289|2450898|45817|1604859|6301|28096|45817|1604859|6301|28096|2|48|20|5|11048|73|833|84|90.97|155.55|37.33|9930.48|3135.72|7641.48|13066.20|125.42|0.00|3658.20|3135.72|3261.14|6793.92|6919.34|-4505.76| +2450831|52289|2450915|45817|1604859|6301|28096|45817|1604859|6301|28096|2|94|10|2|4942|226|833|55|94.10|106.33|40.40|3626.15|2222.00|5175.50|5848.15|133.32|0.00|642.95|2222.00|2355.32|2864.95|2998.27|-2953.50| +2450831|52289|2450840|45817|1604859|6301|28096|45817|1604859|6301|28096|2|26|6|4|4952|10|833|17|32.89|66.76|66.09|11.39|1123.53|559.13|1134.92|89.88|0.00|238.17|1123.53|1213.41|1361.70|1451.58|564.40| +2450831|52289|2450892|45817|1604859|6301|28096|45817|1604859|6301|28096|2|27|14|1|1411|116|833|39|18.20|19.11|9.93|358.02|387.27|709.80|745.29|30.98|0.00|312.78|387.27|418.25|700.05|731.03|-322.53| +2450831|53993|2450891|25949|463443|937|36431|25949|463443|937|36431|1|91|20|5|14008|165|834|23|23.33|32.42|30.79|37.49|708.17|536.59|745.66|56.65|0.00|268.41|708.17|764.82|976.58|1033.23|171.58| +2450831|53993|2450920|25949|463443|937|36431|25949|463443|937|36431|1|42|7|2|17120|206|834|96|48.94|93.47|19.62|7089.60|1883.52|4698.24|8973.12|113.01|0.00|3588.48|1883.52|1996.53|5472.00|5585.01|-2814.72| +2450831|53993|2450896|25949|463443|937|36431|25949|463443|937|36431|1|42|5|3|1219|297|834|76|28.34|34.57|6.22|2154.60|472.72|2153.84|2627.32|0.00|316.72|1024.48|156.00|156.00|1180.48|1180.48|-1997.84| +2450831|53993|2450836|25949|463443|937|36431|25949|463443|937|36431|1|61|8|1|6086|172|834|73|99.19|123.98|110.34|995.72|8054.82|7240.87|9050.54|241.64|0.00|3891.63|8054.82|8296.46|11946.45|12188.09|813.95| +2450831|29872|2450906|85391|1585529|2820|12111|85391|1585529|2820|12111|1|6|1|1|2380|161|835|29|48.27|95.57|67.85|803.88|1967.65|1399.83|2771.53|35.81|177.08|692.81|1790.57|1826.38|2483.38|2519.19|390.74| +2450831|29872|2450839|85391|1585529|2820|12111|85391|1585529|2820|12111|1|3|16|1|3694|168|835|97|35.31|73.79|5.16|6657.11|500.52|3425.07|7157.63|35.03|0.00|356.96|500.52|535.55|857.48|892.51|-2924.55| +2450831|29872|2450863|85391|1585529|2820|12111|85391|1585529|2820|12111|1|34|2|5|6004|61|835|68|1.49|4.24|1.39|193.80|94.52|101.32|288.32|6.61|0.00|37.40|94.52|101.13|131.92|138.53|-6.80| +2450831|29872|2450903|85391|1585529|2820|12111|85391|1585529|2820|12111|1|5|8|1|14882|281|835|46|44.44|119.98|31.19|4084.34|1434.74|2044.24|5519.08|0.00|774.75|1489.94|659.99|659.99|2149.93|2149.93|-1384.25| +2450831|29872|2450910|85391|1585529|2820|12111|85391|1585529|2820|12111|1|14|10|5|7465|168|835|32|45.98|119.08|36.91|2629.44|1181.12|1471.36|3810.56|35.43|0.00|1295.36|1181.12|1216.55|2476.48|2511.91|-290.24| +2450831|29872|2450837|85391|1585529|2820|12111|85391|1585529|2820|12111|1|52|13|1|14060|225|835|50|65.08|156.84|87.83|3450.50|4391.50|3254.00|7842.00|351.32|0.00|2901.50|4391.50|4742.82|7293.00|7644.32|1137.50| +2450831|29872|2450915|85391|1585529|2820|12111|85391|1585529|2820|12111|1|74|17|4|3848|4|835|38|12.76|14.41|5.61|334.40|213.18|484.88|547.58|4.47|149.22|251.56|63.96|68.43|315.52|319.99|-420.92| +2450831|29872|2450886|85391|1585529|2820|12111|85391|1585529|2820|12111|1|22|19|1|13171|210|835|74|13.30|21.81|0.43|1582.12|31.82|984.20|1613.94|1.14|19.09|548.34|12.73|13.87|561.07|562.21|-971.47| +2450831|29872|2450862|85391|1585529|2820|12111|85391|1585529|2820|12111|1|40|12|5|10214|109|835|88|95.68|111.94|47.01|5713.84|4136.88|8419.84|9850.72|41.36|0.00|2659.36|4136.88|4178.24|6796.24|6837.60|-4282.96| +2450831|29872|2450868|85391|1585529|2820|12111|85391|1585529|2820|12111|1|91|18|3|17462|188|835|27|45.50|60.06|52.25|210.87|1410.75|1228.50|1621.62|70.53|0.00|113.40|1410.75|1481.28|1524.15|1594.68|182.25| +2450831|21522|2450844|92836|1229168|700|44232|56915|331296|3628|29113|4|44|18|3|6892|178|836|24|57.20|82.94|68.01|358.32|1632.24|1372.80|1990.56|18.11|1028.31|378.00|603.93|622.04|981.93|1000.04|-768.87| +2450831|21522|2450876|92836|1229168|700|44232|56915|331296|3628|29113|4|85|5|3|12214|172|836|81|36.37|90.92|56.37|2798.55|4565.97|2945.97|7364.52|228.29|0.00|72.90|4565.97|4794.26|4638.87|4867.16|1620.00| +2450831|21522|2450893|92836|1229168|700|44232|56915|331296|3628|29113|4|43|16|5|1039|95|836|61|4.30|6.79|3.12|223.87|190.32|262.30|414.19|0.00|0.00|32.94|190.32|190.32|223.26|223.26|-71.98| +2450831|21522|2450907|92836|1229168|700|44232|56915|331296|3628|29113|4|59|9|3|6520|71|836|82|69.98|209.24|184.13|2059.02|15098.66|5738.36|17157.68|1207.89|0.00|5147.14|15098.66|16306.55|20245.80|21453.69|9360.30| +2450831|21522|2450920|92836|1229168|700|44232|56915|331296|3628|29113|4|47|2|5|6601|275|836|94|92.35|113.59|51.11|5873.12|4804.34|8680.90|10677.46|0.00|0.00|1174.06|4804.34|4804.34|5978.40|5978.40|-3876.56| +2450831|21522|2450887|92836|1229168|700|44232|56915|331296|3628|29113|4|98|7|5|12794|194|836|6|22.22|36.66|18.33|109.98|109.98|133.32|219.96|3.29|0.00|32.94|109.98|113.27|142.92|146.21|-23.34| +2450831|77497|2450857|13192|1611087|4982|47630|40759|539373|6868|4763|2|41|2|4|8815|128|837|9|64.86|192.63|169.51|208.08|1525.59|583.74|1733.67|45.76|0.00|225.36|1525.59|1571.35|1750.95|1796.71|941.85| +2450831|77497|2450898|13192|1611087|4982|47630|40759|539373|6868|4763|2|18|7|5|1519|24|837|90|58.24|105.41|16.86|7969.50|1517.40|5241.60|9486.90|0.00|0.00|1991.70|1517.40|1517.40|3509.10|3509.10|-3724.20| +2450831|77497|2450864|13192|1611087|4982|47630|40759|539373|6868|4763|2|93|17|1|11162|4|837|75|67.12|165.11|161.80|248.25|12135.00|5034.00|12383.25|1092.15|0.00|3962.25|12135.00|13227.15|16097.25|17189.40|7101.00| +2450831|77497|2450875|13192|1611087|4982|47630|40759|539373|6868|4763|2|33|14|2|5354|290|837|33|52.34|146.02|124.11|723.03|4095.63|1727.22|4818.66|204.78|0.00|433.62|4095.63|4300.41|4529.25|4734.03|2368.41| +2450831|77497|2450872|13192|1611087|4982|47630|40759|539373|6868|4763|2|101|4|1|2866|180|837|54|96.82|273.03|43.68|12384.90|2358.72|5228.28|14743.62|141.52|0.00|5160.24|2358.72|2500.24|7518.96|7660.48|-2869.56| +2450831|77497|2450846|13192|1611087|4982|47630|40759|539373|6868|4763|2|70|10|1|2533|158|837|17|14.43|41.99|0.83|699.72|14.11|245.31|713.83|1.12|0.00|228.31|14.11|15.23|242.42|243.54|-231.20| +2450831|77497|2450872|13192|1611087|4982|47630|40759|539373|6868|4763|2|35|19|2|2272|273|837|3|12.78|21.21|8.90|36.93|26.70|38.34|63.63|1.33|0.00|26.07|26.70|28.03|52.77|54.10|-11.64| +2450831|77497|2450886|13192|1611087|4982|47630|40759|539373|6868|4763|2|18|14|5|10112|280|837|41|82.82|160.67|89.97|2898.70|3688.77|3395.62|6587.47|221.32|0.00|922.09|3688.77|3910.09|4610.86|4832.18|293.15| +2450831|77770|2450880|77053|1905510|14|18071|77053|1905510|14|18071|2|35|10|1|3794|78|838|19|6.94|10.61|4.56|114.95|86.64|131.86|201.59|5.15|12.99|80.56|73.65|78.80|154.21|159.36|-58.21| +2450831|77770|2450907|77053|1905510|14|18071|77053|1905510|14|18071|2|79|5|1|1126|153|838|59|71.79|95.48|54.42|2422.54|3210.78|4235.61|5633.32|224.75|0.00|675.55|3210.78|3435.53|3886.33|4111.08|-1024.83| +2450831|77770|2450841|77053|1905510|14|18071|77053|1905510|14|18071|2|50|17|4|13202|273|838|54|80.56|202.20|165.80|1965.60|8953.20|4350.24|10918.80|26.85|8505.54|1965.06|447.66|474.51|2412.72|2439.57|-3902.58| +2450831|77770|2450903|77053|1905510|14|18071|77053|1905510|14|18071|2|79|8|3|13174|292|838|45|57.58|168.70|42.17|5693.85|1897.65|2591.10|7591.50|0.00|0.00|2732.85|1897.65|1897.65|4630.50|4630.50|-693.45| +2450831|77770|2450841|77053|1905510|14|18071|77053|1905510|14|18071|2|64|16|2|8932|102|838|66|33.86|63.99|5.75|3843.84|379.50|2234.76|4223.34|1.44|307.39|1224.30|72.11|73.55|1296.41|1297.85|-2162.65| +2450831|77770|2450882|77053|1905510|14|18071|77053|1905510|14|18071|2|96|16|2|3452|296|838|38|13.46|19.24|4.81|548.34|182.78|511.48|731.12|3.65|0.00|43.70|182.78|186.43|226.48|230.13|-328.70| +2450831|77770|2450846|77053|1905510|14|18071|77053|1905510|14|18071|2|31|6|5|14038|58|838|63|58.54|61.46|17.82|2749.32|1122.66|3688.02|3871.98|67.35|0.00|580.23|1122.66|1190.01|1702.89|1770.24|-2565.36| +2450831|77770|2450910|77053|1905510|14|18071|77053|1905510|14|18071|2|101|13|4|1561|41|838|96|45.56|67.88|20.36|4561.92|1954.56|4373.76|6516.48|2.73|1915.46|2280.00|39.10|41.83|2319.10|2321.83|-4334.66| +2450831|77770|2450872|77053|1905510|14|18071|77053|1905510|14|18071|2|32|9|2|9284|284|838|93|67.79|94.90|93.00|176.70|8649.00|6304.47|8825.70|691.92|0.00|2911.83|8649.00|9340.92|11560.83|12252.75|2344.53| +2450831|77770|2450873|77053|1905510|14|18071|77053|1905510|14|18071|2|52|17|1|952|7|838|4|10.67|13.76|11.14|10.48|44.56|42.68|55.04|4.01|0.00|11.00|44.56|48.57|55.56|59.57|1.88| +2450831|77770|2450868|77053|1905510|14|18071|77053|1905510|14|18071|2|52|5|1|8443|69|838|29|25.20|60.22|39.14|611.32|1135.06|730.80|1746.38|56.75|0.00|401.65|1135.06|1191.81|1536.71|1593.46|404.26| +2450831|77770|2450900|77053|1905510|14|18071|77053|1905510|14|18071|2|50|19|3|11060|156|838|55|47.95|49.38|37.52|652.30|2063.60|2637.25|2715.90|0.00|0.00|244.20|2063.60|2063.60|2307.80|2307.80|-573.65| +2450831|77770|2450865|77053|1905510|14|18071|77053|1905510|14|18071|2|90|4|5|445|118|838|24|89.51|247.94|116.53|3153.84|2796.72|2148.24|5950.56|223.73|0.00|952.08|2796.72|3020.45|3748.80|3972.53|648.48| +2450831|44606|2450858|41614|1704036|4058|15280|41614|1704036|4058|15280|2|88|8|3|12808|278|839|92|46.66|59.25|50.36|817.88|4633.12|4292.72|5451.00|0.00|0.00|108.56|4633.12|4633.12|4741.68|4741.68|340.40| +2450831|44606|2450849|41614|1704036|4058|15280|41614|1704036|4058|15280|2|95|18|3|1850|140|839|71|83.02|90.49|83.25|514.04|5910.75|5894.42|6424.79|118.21|0.00|384.82|5910.75|6028.96|6295.57|6413.78|16.33| +2450831|44606|2450846|41614|1704036|4058|15280|41614|1704036|4058|15280|2|76|1|2|15038|94|839|73|12.97|26.58|2.65|1746.89|193.45|946.81|1940.34|1.93|0.00|154.76|193.45|195.38|348.21|350.14|-753.36| +2450831|44606|2450896|41614|1704036|4058|15280|41614|1704036|4058|15280|2|38|20|4|15367|172|839|28|76.89|149.93|38.98|3106.60|1091.44|2152.92|4198.04|5.67|523.89|1343.16|567.55|573.22|1910.71|1916.38|-1585.37| +2450831|44606|2450910|41614|1704036|4058|15280|41614|1704036|4058|15280|2|39|4|5|3902|132|839|22|54.74|99.07|52.50|1024.54|1155.00|1204.28|2179.54|57.75|0.00|108.90|1155.00|1212.75|1263.90|1321.65|-49.28| +2450831|44606|2450915|41614|1704036|4058|15280|41614|1704036|4058|15280|2|12|16|1|10178|18|839|74|96.37|188.88|147.32|3075.44|10901.68|7131.38|13977.12|436.06|0.00|1397.12|10901.68|11337.74|12298.80|12734.86|3770.30| +2450831|44606|2450910|41614|1704036|4058|15280|41614|1704036|4058|15280|2|83|4|4|11098|59|839|85|86.59|91.78|72.50|1638.80|6162.50|7360.15|7801.30|369.75|0.00|3822.45|6162.50|6532.25|9984.95|10354.70|-1197.65| +2450831|44606|2450851|41614|1704036|4058|15280|41614|1704036|4058|15280|2|34|6|3|15097|285|839|19|52.24|73.65|57.44|307.99|1091.36|992.56|1399.35|54.56|0.00|321.67|1091.36|1145.92|1413.03|1467.59|98.80| +2450831|44606|2450894|41614|1704036|4058|15280|41614|1704036|4058|15280|2|19|6|4|3781|61|839|28|7.31|13.01|4.29|244.16|120.12|204.68|364.28|7.20|0.00|18.20|120.12|127.32|138.32|145.52|-84.56| +2450831|60186|2450848|87529|1892767|5558|45549|87529|1892767|5558|45549|4|32|17|2|12502|177|840|73|20.39|43.22|27.66|1135.88|2019.18|1488.47|3155.06|60.57|0.00|1577.53|2019.18|2079.75|3596.71|3657.28|530.71| +2450831|60186|2450891|87529|1892767|5558|45549|87529|1892767|5558|45549|4|34|9|4|3866|131|840|62|30.75|69.80|40.48|1817.84|2509.76|1906.50|4327.60|5.52|1957.61|389.36|552.15|557.67|941.51|947.03|-1354.35| +2450831|60186|2450881|87529|1892767|5558|45549|87529|1892767|5558|45549|4|76|20|2|11260|146|840|82|14.96|27.97|7.55|1674.44|619.10|1226.72|2293.54|24.76|0.00|962.68|619.10|643.86|1581.78|1606.54|-607.62| +2450831|60186|2450882|87529|1892767|5558|45549|87529|1892767|5558|45549|4|25|6|3|5812|35|840|47|47.88|129.75|60.98|3232.19|2866.06|2250.36|6098.25|57.32|0.00|2073.17|2866.06|2923.38|4939.23|4996.55|615.70| +2450831|60186|2450917|87529|1892767|5558|45549|87529|1892767|5558|45549|4|10|19|4|16933|184|840|71|95.88|255.04|38.25|15392.09|2715.75|6807.48|18107.84|124.92|217.26|5613.26|2498.49|2623.41|8111.75|8236.67|-4308.99| +2450831|74458|2450843|20456|1078775|6694|9413|75533|945204|6723|11042|4|103|8|3|9028|258|841|81|65.16|115.98|57.99|4697.19|4697.19|5277.96|9394.38|422.74|0.00|375.03|4697.19|5119.93|5072.22|5494.96|-580.77| +2450831|74458|2450843|20456|1078775|6694|9413|75533|945204|6723|11042|4|30|13|2|10766|68|841|65|79.32|212.57|12.75|12988.30|828.75|5155.80|13817.05|74.58|0.00|3315.65|828.75|903.33|4144.40|4218.98|-4327.05| +2450831|74458|2450916|20456|1078775|6694|9413|75533|945204|6723|11042|4|64|6|2|12508|246|841|18|97.22|265.41|148.62|2102.22|2675.16|1749.96|4777.38|15.78|1096.81|716.58|1578.35|1594.13|2294.93|2310.71|-171.61| +2450831|74458|2450852|20456|1078775|6694|9413|75533|945204|6723|11042|4|81|17|1|8503|101|841|94|22.95|55.30|26.54|2703.44|2494.76|2157.30|5198.20|49.89|0.00|1350.78|2494.76|2544.65|3845.54|3895.43|337.46| +2450831|74458|2450838|20456|1078775|6694|9413|75533|945204|6723|11042|4|10|1|2|1964|255|841|99|35.55|40.88|38.01|284.13|3762.99|3519.45|4047.12|338.66|0.00|1618.65|3762.99|4101.65|5381.64|5720.30|243.54| +2450831|74458|2450890|20456|1078775|6694|9413|75533|945204|6723|11042|4|4|7|5|17150|194|841|19|35.89|57.06|11.41|867.35|216.79|681.91|1084.14|19.51|0.00|357.58|216.79|236.30|574.37|593.88|-465.12| +2450831|54016|2450856|37484|1006453|2622|391|37484|1006453|2622|391|1|35|8|1|12415|262|842|44|16.74|21.92|2.63|848.76|115.72|736.56|964.48|0.00|0.00|443.52|115.72|115.72|559.24|559.24|-620.84| +2450831|54016|2450867|37484|1006453|2622|391|37484|1006453|2622|391|1|107|14|4|2252|142|842|47|90.76|213.28|0.00|10024.16|0.00|4265.72|10024.16|0.00|0.00|501.02|0.00|0.00|501.02|501.02|-4265.72| +2450831|54016|2450888|37484|1006453|2622|391|37484|1006453|2622|391|1|37|6|1|8536|268|842|24|97.56|188.29|146.86|994.32|3524.64|2341.44|4518.96|105.73|0.00|180.72|3524.64|3630.37|3705.36|3811.09|1183.20| +2450831|54016|2450879|37484|1006453|2622|391|37484|1006453|2622|391|1|26|7|2|15146|264|842|92|22.73|59.77|57.37|220.80|5278.04|2091.16|5498.84|422.24|0.00|219.88|5278.04|5700.28|5497.92|5920.16|3186.88| +2450831|54016|2450889|37484|1006453|2622|391|37484|1006453|2622|391|1|104|10|5|17839|208|842|56|95.72|182.82|127.97|3071.60|7166.32|5360.32|10237.92|143.32|0.00|1535.52|7166.32|7309.64|8701.84|8845.16|1806.00| +2450831|54016|2450836|37484|1006453|2622|391|37484|1006453|2622|391|1|35|17|1|8872|49|842|3|71.28|97.65|78.12|58.59|234.36|213.84|292.95|0.49|185.14|82.02|49.22|49.71|131.24|131.73|-164.62| +2450831|54016|2450889|37484|1006453|2622|391|37484|1006453|2622|391|1|75|19|3|11641|259|842|98|86.23|169.01|148.72|1988.42|14574.56|8450.54|16562.98|301.69|4518.11|4637.36|10056.45|10358.14|14693.81|14995.50|1605.91| +2450831|54016|2450904|37484|1006453|2622|391|37484|1006453|2622|391|1|99|7|2|38|203|842|28|37.74|56.23|45.54|299.32|1275.12|1056.72|1574.44|56.61|331.53|267.40|943.59|1000.20|1210.99|1267.60|-113.13| +2450831|74476|2450887|22423|28418|6413|14008|22423|28418|6413|14008|2|42|2|4|7177|255|843|57|17.34|28.78|4.89|1361.73|278.73|988.38|1640.46|19.51|0.00|229.14|278.73|298.24|507.87|527.38|-709.65| +2450831|74476|2450884|22423|28418|6413|14008|22423|28418|6413|14008|2|10|18|4|8612|8|843|21|54.26|158.98|139.90|400.68|2937.90|1139.46|3338.58|235.03|0.00|467.25|2937.90|3172.93|3405.15|3640.18|1798.44| +2450831|74476|2450842|22423|28418|6413|14008|22423|28418|6413|14008|2|100|8|2|17887|75|843|71|53.98|111.19|51.14|4263.55|3630.94|3832.58|7894.49|36.30|0.00|2210.23|3630.94|3667.24|5841.17|5877.47|-201.64| +2450831|74476|2450903|22423|28418|6413|14008|22423|28418|6413|14008|2|58|8|3|5197|64|843|14|94.59|214.71|92.32|1713.46|1292.48|1324.26|3005.94|90.47|0.00|1292.48|1292.48|1382.95|2584.96|2675.43|-31.78| +2450831|74476|2450847|22423|28418|6413|14008|22423|28418|6413|14008|2|44|3|1|10501|224|843|78|76.81|228.12|109.49|9253.14|8540.22|5991.18|17793.36|427.01|0.00|7828.86|8540.22|8967.23|16369.08|16796.09|2549.04| +2450831|74476|2450887|22423|28418|6413|14008|22423|28418|6413|14008|2|61|11|5|12145|140|843|79|54.03|141.55|89.17|4138.02|7044.43|4268.37|11182.45|563.55|0.00|4696.55|7044.43|7607.98|11740.98|12304.53|2776.06| +2450831|74476|2450914|22423|28418|6413|14008|22423|28418|6413|14008|2|90|1|3|16720|272|843|81|93.81|215.76|144.55|5768.01|11708.55|7598.61|17476.56|468.34|0.00|1048.14|11708.55|12176.89|12756.69|13225.03|4109.94| +2450831|74476|2450864|22423|28418|6413|14008|22423|28418|6413|14008|2|18|10|1|4138|223|843|30|52.26|56.44|9.03|1422.30|270.90|1567.80|1693.20|18.96|0.00|304.50|270.90|289.86|575.40|594.36|-1296.90| +2450831|74476|2450834|22423|28418|6413|14008|22423|28418|6413|14008|2|97|4|3|13808|29|843|55|23.62|25.03|11.51|743.60|633.05|1299.10|1376.65|37.98|0.00|688.05|633.05|671.03|1321.10|1359.08|-666.05| +2450831|74476|2450876|22423|28418|6413|14008|22423|28418|6413|14008|2|13|20|2|230|255|843|63|83.83|124.90|4.99|7554.33|314.37|5281.29|7868.70|22.00|0.00|2674.98|314.37|336.37|2989.35|3011.35|-4966.92| +2450831|74476|2450850|22423|28418|6413|14008|22423|28418|6413|14008|2|16|4|4|1748|24|843|69|37.52|100.55|5.02|6591.57|346.38|2588.88|6937.95|20.78|0.00|3052.56|346.38|367.16|3398.94|3419.72|-2242.50| +2450831|74476|2450897|22423|28418|6413|14008|22423|28418|6413|14008|2|37|9|4|14042|169|843|96|23.61|59.26|22.51|3528.00|2160.96|2266.56|5688.96|86.87|713.11|2388.48|1447.85|1534.72|3836.33|3923.20|-818.71| +2450831|74476|2450920|22423|28418|6413|14008|22423|28418|6413|14008|2|7|13|2|7238|50|843|50|49.74|132.30|105.84|1323.00|5292.00|2487.00|6615.00|423.36|0.00|1653.50|5292.00|5715.36|6945.50|7368.86|2805.00| +2450831|75367|2450900|39069|1908951|880|43162|29438|787326|6008|20737|2|9|16|5|3169|216|844|64|34.33|65.91|0.00|4218.24|0.00|2197.12|4218.24|0.00|0.00|1011.84|0.00|0.00|1011.84|1011.84|-2197.12| +2450831|75367|2450918|39069|1908951|880|43162|29438|787326|6008|20737|2|21|6|2|8590|80|844|48|98.87|144.35|49.07|4573.44|2355.36|4745.76|6928.80|164.87|0.00|553.92|2355.36|2520.23|2909.28|3074.15|-2390.40| +2450831|75367|2450867|39069|1908951|880|43162|29438|787326|6008|20737|2|96|10|4|4964|148|844|7|27.94|33.24|25.59|53.55|179.13|195.58|232.68|0.00|0.00|37.17|179.13|179.13|216.30|216.30|-16.45| +2450831|75367|2450843|39069|1908951|880|43162|29438|787326|6008|20737|2|90|6|1|14926|21|844|85|50.25|74.37|37.18|3161.15|3160.30|4271.25|6321.45|221.22|0.00|2085.90|3160.30|3381.52|5246.20|5467.42|-1110.95| +2450831|75367|2450835|39069|1908951|880|43162|29438|787326|6008|20737|2|56|13|2|2774|280|844|86|84.69|157.52|148.06|813.56|12733.16|7283.34|13546.72|381.99|0.00|6366.58|12733.16|13115.15|19099.74|19481.73|5449.82| +2450831|75367|2450890|39069|1908951|880|43162|29438|787326|6008|20737|2|63|16|3|5449|130|844|87|4.16|8.48|4.74|325.38|412.38|361.92|737.76|28.86|0.00|43.50|412.38|441.24|455.88|484.74|50.46| +2450831|75367|2450866|39069|1908951|880|43162|29438|787326|6008|20737|2|103|15|5|12709|112|844|27|27.95|76.58|11.48|1757.70|309.96|754.65|2067.66|21.69|0.00|454.68|309.96|331.65|764.64|786.33|-444.69| +2450831|75367|2450903|39069|1908951|880|43162|29438|787326|6008|20737|2|100|18|3|15127|183|844|29|11.60|25.28|21.23|117.45|615.67|336.40|733.12|12.31|0.00|336.98|615.67|627.98|952.65|964.96|279.27| +2450831|75367|2450916|39069|1908951|880|43162|29438|787326|6008|20737|2|57|1|5|1081|68|844|26|23.51|23.74|15.66|210.08|407.16|611.26|617.24|24.42|0.00|185.12|407.16|431.58|592.28|616.70|-204.10| +2450831|75367|2450898|39069|1908951|880|43162|29438|787326|6008|20737|2|19|12|5|1412|276|844|42|78.10|177.28|129.41|2010.54|5435.22|3280.20|7445.76|54.35|0.00|3499.44|5435.22|5489.57|8934.66|8989.01|2155.02| +2450831|75367|2450863|39069|1908951|880|43162|29438|787326|6008|20737|2|22|20|1|5953|278|844|53|62.46|136.16|84.41|2742.75|4473.73|3310.38|7216.48|0.00|1878.96|1804.12|2594.77|2594.77|4398.89|4398.89|-715.61| +2450831|75367|2450872|39069|1908951|880|43162|29438|787326|6008|20737|2|10|18|3|10168|259|844|6|77.47|123.95|114.03|59.52|684.18|464.82|743.70|1.09|656.81|215.64|27.37|28.46|243.01|244.10|-437.45| +2450831|75367|2450833|39069|1908951|880|43162|29438|787326|6008|20737|2|10|19|1|3286|288|844|48|93.67|214.50|158.73|2676.96|7619.04|4496.16|10296.00|73.14|6399.99|3912.48|1219.05|1292.19|5131.53|5204.67|-3277.11| +2450831|75367|2450856|39069|1908951|880|43162|29438|787326|6008|20737|2|98|12|2|15691|295|844|85|69.47|154.22|86.36|5768.10|7340.60|5904.95|13108.70|124.79|1101.09|3670.30|6239.51|6364.30|9909.81|10034.60|334.56| +2450831|35741|2450875|62896|1134848|2000|35552|62896|1134848|2000|35552|4|36|2|2|13052|72|845|87|45.19|80.89|39.63|3589.62|3447.81|3931.53|7037.43|310.30|0.00|69.60|3447.81|3758.11|3517.41|3827.71|-483.72| +2450831|35741|2450875|62896|1134848|2000|35552|62896|1134848|2000|35552|4|3|15|2|7255|229|845|55|66.54|197.62|197.62|0.00|10869.10|3659.70|10869.10|495.63|4673.71|217.25|6195.39|6691.02|6412.64|6908.27|2535.69| +2450831|35741|2450834|62896|1134848|2000|35552|62896|1134848|2000|35552|4|107|4|1|7070|168|845|26|31.11|89.59|4.47|2213.12|116.22|808.86|2329.34|10.45|0.00|512.20|116.22|126.67|628.42|638.87|-692.64| +2450831|35741|2450845|62896|1134848|2000|35552|62896|1134848|2000|35552|4|18|9|1|12980|229|845|49|33.73|60.37|35.61|1213.24|1744.89|1652.77|2958.13|122.14|0.00|177.38|1744.89|1867.03|1922.27|2044.41|92.12| +2450831|35741|2450843|62896|1134848|2000|35552|62896|1134848|2000|35552|4|80|19|5|16292|19|845|64|75.57|178.34|108.78|4451.84|6961.92|4836.48|11413.76|173.35|1183.52|3537.92|5778.40|5951.75|9316.32|9489.67|941.92| +2450831|35741|2450842|62896|1134848|2000|35552|62896|1134848|2000|35552|4|76|11|1|14732|204|845|10|57.62|118.69|108.00|106.90|1080.00|576.20|1186.90|97.20|0.00|154.20|1080.00|1177.20|1234.20|1331.40|503.80| +2450831|35741|2450895|62896|1134848|2000|35552|62896|1134848|2000|35552|4|51|11|4|10610|78|845|28|20.10|36.58|34.75|51.24|973.00|562.80|1024.24|9.73|0.00|255.92|973.00|982.73|1228.92|1238.65|410.20| +2450831|2869|2450862|31059|1291663|3467|40471|31059|1291663|3467|40471|4|12|10|4|12862|279|846|90|46.58|104.80|101.65|283.50|9148.50|4192.20|9432.00|35.67|5580.58|4527.00|3567.92|3603.59|8094.92|8130.59|-624.28| +2450831|2869|2450880|31059|1291663|3467|40471|31059|1291663|3467|40471|4|52|17|2|13435|62|846|82|23.98|64.98|57.83|586.30|4742.06|1966.36|5328.36|121.39|3224.60|851.98|1517.46|1638.85|2369.44|2490.83|-448.90| +2450831|2869|2450861|31059|1291663|3467|40471|31059|1291663|3467|40471|4|95|13|5|9745|289|846|61|55.48|129.26|1.29|7806.17|78.69|3384.28|7884.86|3.93|0.00|3390.38|78.69|82.62|3469.07|3473.00|-3305.59| +2450831|2869|2450881|31059|1291663|3467|40471|31059|1291663|3467|40471|4|106|18|5|15184|116|846|7|78.89|100.97|54.52|325.15|381.64|552.23|706.79|11.44|0.00|261.45|381.64|393.08|643.09|654.53|-170.59| +2450831|2869|2450921|31059|1291663|3467|40471|31059|1291663|3467|40471|4|12|2|4|11983|56|846|73|46.26|119.81|19.16|7347.45|1398.68|3376.98|8746.13|125.88|0.00|1049.01|1398.68|1524.56|2447.69|2573.57|-1978.30| +2450831|2869|2450864|31059|1291663|3467|40471|31059|1291663|3467|40471|4|46|2|2|8450|169|846|63|2.82|6.45|0.64|366.03|40.32|177.66|406.35|3.62|0.00|105.21|40.32|43.94|145.53|149.15|-137.34| +2450831|2869|2450854|31059|1291663|3467|40471|31059|1291663|3467|40471|4|57|16|3|2329|164|846|64|46.97|112.72|76.64|2309.12|4904.96|3006.08|7214.08|196.19|0.00|1731.20|4904.96|5101.15|6636.16|6832.35|1898.88| +2450831|2869|2450865|31059|1291663|3467|40471|31059|1291663|3467|40471|4|34|4|4|14053|63|846|94|75.02|107.27|74.01|3126.44|6956.94|7051.88|10083.38|11.13|6817.80|4032.60|139.14|150.27|4171.74|4182.87|-6912.74| +2450831|2869|2450895|31059|1291663|3467|40471|31059|1291663|3467|40471|4|98|16|3|14752|283|846|31|91.44|93.26|23.31|2168.45|722.61|2834.64|2891.06|0.00|0.00|606.98|722.61|722.61|1329.59|1329.59|-2112.03| +2450831|2869|2450874|31059|1291663|3467|40471|31059|1291663|3467|40471|4|55|1|2|16988|181|846|95|1.29|3.87|0.00|367.65|0.00|122.55|367.65|0.00|0.00|47.50|0.00|0.00|47.50|47.50|-122.55| +2450831|2869|2450868|31059|1291663|3467|40471|31059|1291663|3467|40471|4|50|16|2|10946|34|846|93|6.27|18.68|16.25|225.99|1511.25|583.11|1737.24|14.50|1027.65|52.08|483.60|498.10|535.68|550.18|-99.51| +2450831|70024|2450889|79745|23403|4214|23872|79745|23403|4214|23872|4|93|1|4|16102|95|847|64|86.62|158.51|17.43|9029.12|1115.52|5543.68|10144.64|18.74|647.00|4057.60|468.52|487.26|4526.12|4544.86|-5075.16| +2450831|70024|2450915|79745|23403|4214|23872|79745|23403|4214|23872|4|11|5|3|8432|127|847|53|35.24|99.02|4.95|4985.71|262.35|1867.72|5248.06|7.87|0.00|996.93|262.35|270.22|1259.28|1267.15|-1605.37| +2450831|70024|2450909|79745|23403|4214|23872|79745|23403|4214|23872|4|17|9|1|9229|267|847|73|47.01|139.14|33.39|7719.75|2437.47|3431.73|10157.22|170.62|0.00|2640.41|2437.47|2608.09|5077.88|5248.50|-994.26| +2450831|70024|2450918|79745|23403|4214|23872|79745|23403|4214|23872|4|92|5|4|12502|5|847|70|88.10|99.55|21.90|5435.50|1533.00|6167.00|6968.50|15.33|0.00|1114.40|1533.00|1548.33|2647.40|2662.73|-4634.00| +2450831|70024|2450908|79745|23403|4214|23872|79745|23403|4214|23872|4|3|13|2|3866|12|847|5|84.21|247.57|180.72|334.25|903.60|421.05|1237.85|63.25|0.00|259.90|903.60|966.85|1163.50|1226.75|482.55| +2450831|70024|2450918|79745|23403|4214|23872|79745|23403|4214|23872|4|60|17|4|11260|180|847|55|88.22|110.27|74.98|1940.95|4123.90|4852.10|6064.85|293.62|453.62|2061.95|3670.28|3963.90|5732.23|6025.85|-1181.82| +2450831|70024|2450902|79745|23403|4214|23872|79745|23403|4214|23872|4|75|1|3|5812|177|847|80|20.33|29.88|18.82|884.80|1505.60|1626.40|2390.40|45.16|0.00|549.60|1505.60|1550.76|2055.20|2100.36|-120.80| +2450831|70024|2450856|79745|23403|4214|23872|79745|23403|4214|23872|4|85|17|3|16933|34|847|70|70.38|197.76|23.73|12182.10|1661.10|4926.60|13843.20|66.44|0.00|3875.90|1661.10|1727.54|5537.00|5603.44|-3265.50| +2450831|66264|2450862|22259|750843|580|48364|22259|750843|580|48364|1|71|16|5|5882|231|848|43|84.60|217.42|102.18|4955.32|4393.74|3637.80|9349.06|2.63|4305.86|4487.48|87.88|90.51|4575.36|4577.99|-3549.92| +2450831|66264|2450917|22259|750843|580|48364|22259|750843|580|48364|1|105|2|1|4760|24|848|66|70.37|188.59|118.81|4605.48|7841.46|4644.42|12446.94|313.65|0.00|497.64|7841.46|8155.11|8339.10|8652.75|3197.04| +2450831|66264|2450898|22259|750843|580|48364|22259|750843|580|48364|1|42|8|3|5512|245|848|63|94.56|161.69|27.48|8455.23|1731.24|5957.28|10186.47|0.69|1713.92|2954.07|17.32|18.01|2971.39|2972.08|-5939.96| +2450831|66264|2450919|22259|750843|580|48364|22259|750843|580|48364|1|27|8|5|11158|109|848|40|72.10|129.78|111.61|726.80|4464.40|2884.00|5191.20|223.22|0.00|1505.20|4464.40|4687.62|5969.60|6192.82|1580.40| +2450831|66264|2450879|22259|750843|580|48364|22259|750843|580|48364|1|94|2|5|11198|131|848|31|4.52|5.96|1.96|124.00|60.76|140.12|184.76|4.25|0.00|58.90|60.76|65.01|119.66|123.91|-79.36| +2450831|66264|2450900|22259|750843|580|48364|22259|750843|580|48364|1|8|20|4|7330|170|848|36|34.25|78.43|61.17|621.36|2202.12|1233.00|2823.48|66.06|0.00|1044.36|2202.12|2268.18|3246.48|3312.54|969.12| +2450831|66264|2450897|22259|750843|580|48364|22259|750843|580|48364|1|23|13|3|7972|18|848|91|57.49|90.25|47.83|3860.22|4352.53|5231.59|8212.75|13.05|4091.37|2791.88|261.16|274.21|3053.04|3066.09|-4970.43| +2450831|66264|2450887|22259|750843|580|48364|22259|750843|580|48364|1|20|15|1|14443|150|848|24|1.05|1.14|0.22|22.08|5.28|25.20|27.36|0.14|2.90|12.96|2.38|2.52|15.34|15.48|-22.82| +2450831|66264|2450852|22259|750843|580|48364|22259|750843|580|48364|1|32|14|1|16747|49|848|80|60.50|151.25|55.96|7623.20|4476.80|4840.00|12100.00|0.00|0.00|5324.00|4476.80|4476.80|9800.80|9800.80|-363.20| +2450831|77630|2450890|9658|419572|5839|12106|9658|419572|5839|12106|1|38|1|3|10234|126|849|8|18.52|20.37|11.81|68.48|94.48|148.16|162.96|4.72|0.00|65.12|94.48|99.20|159.60|164.32|-53.68| +2450831|77630|2450867|9658|419572|5839|12106|9658|419572|5839|12106|1|85|2|4|7441|285|849|55|36.81|82.45|20.61|3401.20|1133.55|2024.55|4534.75|22.67|0.00|2222.00|1133.55|1156.22|3355.55|3378.22|-891.00| +2450831|77630|2450907|9658|419572|5839|12106|9658|419572|5839|12106|1|92|18|3|11102|183|849|33|16.16|32.64|12.72|657.36|419.76|533.28|1077.12|12.59|0.00|75.24|419.76|432.35|495.00|507.59|-113.52| +2450831|77630|2450850|9658|419572|5839|12106|9658|419572|5839|12106|1|45|10|3|8413|114|849|11|87.13|230.89|226.27|50.82|2488.97|958.43|2539.79|224.00|0.00|939.62|2488.97|2712.97|3428.59|3652.59|1530.54| +2450831|77630|2450919|9658|419572|5839|12106|9658|419572|5839|12106|1|14|15|2|10606|278|849|15|99.59|187.22|170.37|252.75|2555.55|1493.85|2808.30|119.60|1226.66|814.35|1328.89|1448.49|2143.24|2262.84|-164.96| +2450831|77630|2450904|9658|419572|5839|12106|9658|419572|5839|12106|1|71|1|4|961|47|849|58|43.16|62.58|48.81|798.66|2830.98|2503.28|3629.64|113.23|0.00|35.96|2830.98|2944.21|2866.94|2980.17|327.70| +2450831|77630|2450862|9658|419572|5839|12106|9658|419572|5839|12106|1|31|15|1|5305|300|849|40|58.93|143.78|106.39|1495.60|4255.60|2357.20|5751.20|170.22|0.00|287.20|4255.60|4425.82|4542.80|4713.02|1898.40| +2450831|77630|2450850|9658|419572|5839|12106|9658|419572|5839|12106|1|103|18|1|12637|117|849|70|60.17|129.96|3.89|8824.90|272.30|4211.90|9097.20|21.78|0.00|0.00|272.30|294.08|272.30|294.08|-3939.60| +2450831|77630|2450920|9658|419572|5839|12106|9658|419572|5839|12106|1|45|7|4|5707|265|849|41|70.75|104.71|34.55|2876.56|1416.55|2900.75|4293.11|26.77|1034.08|128.74|382.47|409.24|511.21|537.98|-2518.28| +2450831|77630|2450905|9658|419572|5839|12106|9658|419572|5839|12106|1|17|9|2|9595|215|849|97|20.27|26.95|24.25|261.90|2352.25|1966.19|2614.15|161.83|329.31|1280.40|2022.94|2184.77|3303.34|3465.17|56.75| +2450831|68274|2450909|87473|798465|7057|14896|87473|798465|7057|14896|1|9|4|5|6121|253|850|73|94.19|206.27|76.31|9487.08|5570.63|6875.87|15057.71|167.11|0.00|3011.25|5570.63|5737.74|8581.88|8748.99|-1305.24| +2450831|68274|2450879|87473|798465|7057|14896|87473|798465|7057|14896|1|69|9|3|15328|101|850|28|68.21|195.08|158.01|1037.96|4424.28|1909.88|5462.24|398.18|0.00|2403.24|4424.28|4822.46|6827.52|7225.70|2514.40| +2450831|68274|2450916|87473|798465|7057|14896|87473|798465|7057|14896|1|83|14|1|15139|221|850|88|48.89|125.64|92.97|2874.96|8181.36|4302.32|11056.32|122.72|6136.02|0.00|2045.34|2168.06|2045.34|2168.06|-2256.98| +2450831|68274|2450913|87473|798465|7057|14896|87473|798465|7057|14896|1|70|17|4|8714|52|850|1|81.30|240.64|156.41|84.23|156.41|81.30|240.64|9.38|0.00|60.16|156.41|165.79|216.57|225.95|75.11| +2450831|68274|2450920|87473|798465|7057|14896|87473|798465|7057|14896|1|13|10|2|14374|1|850|98|97.88|189.88|168.99|2047.22|16561.02|9592.24|18608.24|1020.15|3809.03|7814.52|12751.99|13772.14|20566.51|21586.66|3159.75| +2450831|68274|2450856|87473|798465|7057|14896|87473|798465|7057|14896|1|1|16|3|9616|115|850|3|32.07|43.61|27.03|49.74|81.09|96.21|130.83|0.00|0.00|20.91|81.09|81.09|102.00|102.00|-15.12| +2450831|68274|2450902|87473|798465|7057|14896|87473|798465|7057|14896|1|23|8|1|5474|280|850|87|21.35|63.62|10.81|4594.47|940.47|1857.45|5534.94|0.00|0.00|331.47|940.47|940.47|1271.94|1271.94|-916.98| +2450831|68274|2450917|87473|798465|7057|14896|87473|798465|7057|14896|1|108|4|3|11623|163|850|94|51.74|68.29|55.99|1156.20|5263.06|4863.56|6419.26|52.63|0.00|2181.74|5263.06|5315.69|7444.80|7497.43|399.50| +2450831|68274|2450916|87473|798465|7057|14896|87473|798465|7057|14896|1|82|11|2|2536|231|850|82|50.13|68.67|64.54|338.66|5292.28|4110.66|5630.94|317.53|0.00|2364.88|5292.28|5609.81|7657.16|7974.69|1181.62| +2450831|68274|2450858|87473|798465|7057|14896|87473|798465|7057|14896|1|100|19|2|4699|6|850|23|69.50|181.39|132.41|1126.54|3045.43|1598.50|4171.97|172.98|883.17|1126.31|2162.26|2335.24|3288.57|3461.55|563.76| +2450831|68274|2450894|87473|798465|7057|14896|87473|798465|7057|14896|1|36|9|2|11881|33|850|32|22.27|47.65|1.42|1479.36|45.44|712.64|1524.80|2.27|0.00|335.36|45.44|47.71|380.80|383.07|-667.20| +2450831|65325|2450879|8498|246883|3315|147|8498|246883|3315|147|1|59|20|3|6010|259|851|39|65.64|116.18|6.97|4259.19|271.83|2559.96|4531.02|19.02|0.00|2219.88|271.83|290.85|2491.71|2510.73|-2288.13| +2450831|65325|2450914|8498|246883|3315|147|8498|246883|3315|147|1|78|18|1|2587|59|851|22|9.32|10.25|3.28|153.34|72.16|205.04|225.50|1.44|0.00|65.34|72.16|73.60|137.50|138.94|-132.88| +2450831|65325|2450855|8498|246883|3315|147|8498|246883|3315|147|1|81|7|1|4958|149|851|21|82.96|237.26|173.19|1345.47|3636.99|1742.16|4982.46|218.21|0.00|597.87|3636.99|3855.20|4234.86|4453.07|1894.83| +2450831|65325|2450901|8498|246883|3315|147|8498|246883|3315|147|1|102|17|2|2326|70|851|9|85.05|253.44|2.53|2258.19|22.77|765.45|2280.96|1.41|7.05|136.80|15.72|17.13|152.52|153.93|-749.73| +2450831|65325|2450911|8498|246883|3315|147|8498|246883|3315|147|1|92|19|5|6955|72|851|20|90.05|169.29|148.97|406.40|2979.40|1801.00|3385.80|268.14|0.00|1455.80|2979.40|3247.54|4435.20|4703.34|1178.40| +2450831|65325|2450851|8498|246883|3315|147|8498|246883|3315|147|1|1|9|4|9127|124|851|39|79.18|141.73|82.20|2321.67|3205.80|3088.02|5527.47|32.05|0.00|2376.66|3205.80|3237.85|5582.46|5614.51|117.78| +2450831|65325|2450895|8498|246883|3315|147|8498|246883|3315|147|1|75|10|1|17828|90|851|95|57.80|129.47|90.62|3690.75|8608.90|5491.00|12299.65|258.26|0.00|2951.65|8608.90|8867.16|11560.55|11818.81|3117.90| +2450831|65325|2450877|8498|246883|3315|147|8498|246883|3315|147|1|71|9|4|6829|292|851|46|33.52|89.83|0.00|4132.18|0.00|1541.92|4132.18|0.00|0.00|1694.18|0.00|0.00|1694.18|1694.18|-1541.92| +2450831|65325|2450887|8498|246883|3315|147|8498|246883|3315|147|1|54|7|4|8504|18|851|43|53.71|150.38|117.29|1422.87|5043.47|2309.53|6466.34|3.02|4892.16|646.29|151.31|154.33|797.60|800.62|-2158.22| +2450831|65325|2450901|8498|246883|3315|147|8498|246883|3315|147|1|35|20|1|1448|211|851|12|35.17|60.14|10.22|599.04|122.64|422.04|721.68|9.81|0.00|14.40|122.64|132.45|137.04|146.85|-299.40| +2450831|23229|2450914|10311|794543|2775|36207|10311|794543|2775|36207|1|12|3|2|8485|103|852|75|16.66|45.98|26.20|1483.50|1965.00|1249.50|3448.50|157.20|0.00|516.75|1965.00|2122.20|2481.75|2638.95|715.50| +2450831|23229|2450897|10311|794543|2775|36207|10311|794543|2775|36207|1|17|3|3|12160|106|852|53|67.09|128.14|10.25|6248.17|543.25|3555.77|6791.42|43.46|0.00|339.20|543.25|586.71|882.45|925.91|-3012.52| +2450831|23229|2450842|10311|794543|2775|36207|10311|794543|2775|36207|1|83|19|4|12067|41|852|62|15.48|32.35|2.58|1845.74|159.96|959.76|2005.70|4.79|0.00|60.14|159.96|164.75|220.10|224.89|-799.80| +2450831|23229|2450866|10311|794543|2775|36207|10311|794543|2775|36207|1|79|16|5|1828|36|852|18|43.20|54.86|19.20|641.88|345.60|777.60|987.48|31.10|0.00|345.60|345.60|376.70|691.20|722.30|-432.00| +2450831|23229|2450906|10311|794543|2775|36207|10311|794543|2775|36207|1|19|1|5|11359|213|852|94|44.08|59.06|11.81|4441.50|1110.14|4143.52|5551.64|0.00|910.31|2497.58|199.83|199.83|2697.41|2697.41|-3943.69| +2450831|23229|2450897|10311|794543|2775|36207|10311|794543|2775|36207|1|27|20|5|10124|18|852|64|69.20|145.32|100.27|2883.20|6417.28|4428.80|9300.48|0.00|0.00|3534.08|6417.28|6417.28|9951.36|9951.36|1988.48| +2450831|23229|2450837|10311|794543|2775|36207|10311|794543|2775|36207|1|96|19|2|7280|46|852|76|94.86|185.92|172.90|989.52|13140.40|7209.36|14129.92|657.02|0.00|3108.40|13140.40|13797.42|16248.80|16905.82|5931.04| +2450831|23229|2450881|10311|794543|2775|36207|10311|794543|2775|36207|1|65|13|3|655|124|852|29|89.66|200.83|8.03|5591.20|232.87|2600.14|5824.07|20.95|0.00|756.90|232.87|253.82|989.77|1010.72|-2367.27| +2450831|23229|2450914|10311|794543|2775|36207|10311|794543|2775|36207|1|37|5|1|11137|135|852|32|75.71|217.28|115.15|3268.16|3684.80|2422.72|6952.96|184.24|0.00|138.88|3684.80|3869.04|3823.68|4007.92|1262.08| +2450831|23229|2450871|10311|794543|2775|36207|10311|794543|2775|36207|1|84|15|3|2119|137|852|68|62.31|181.94|176.48|371.28|12000.64|4237.08|12371.92|840.04|0.00|5690.92|12000.64|12840.68|17691.56|18531.60|7763.56| +2450831|23229|2450903|10311|794543|2775|36207|10311|794543|2775|36207|1|50|16|3|7579|215|852|88|49.93|113.84|23.90|7914.72|2103.20|4393.84|10017.92|84.12|0.00|4607.68|2103.20|2187.32|6710.88|6795.00|-2290.64| +2450831|29482|2450859|95527|1543750|2373|36596|95527|1543750|2373|36596|4|85|16|3|17557|42|853|28|9.24|18.57|12.25|176.96|343.00|258.72|519.96|24.01|0.00|244.16|343.00|367.01|587.16|611.17|84.28| +2450831|29482|2450839|95527|1543750|2373|36596|95527|1543750|2373|36596|4|28|2|4|5996|206|853|25|69.72|164.53|164.53|0.00|4113.25|1743.00|4113.25|329.06|0.00|1357.25|4113.25|4442.31|5470.50|5799.56|2370.25| +2450831|29482|2450863|95527|1543750|2373|36596|95527|1543750|2373|36596|4|6|7|4|16964|63|853|89|23.62|52.43|16.25|3220.02|1446.25|2102.18|4666.27|5.49|896.67|1026.17|549.58|555.07|1575.75|1581.24|-1552.60| +2450831|29482|2450897|95527|1543750|2373|36596|95527|1543750|2373|36596|4|93|13|1|6826|12|853|78|92.95|163.59|1.63|12632.88|127.14|7250.10|12760.02|0.00|0.00|2041.26|127.14|127.14|2168.40|2168.40|-7122.96| +2450831|29482|2450849|95527|1543750|2373|36596|95527|1543750|2373|36596|4|71|20|2|7459|169|853|37|12.14|31.56|27.77|140.23|1027.49|449.18|1167.72|30.82|0.00|23.31|1027.49|1058.31|1050.80|1081.62|578.31| +2450831|29482|2450906|95527|1543750|2373|36596|95527|1543750|2373|36596|4|26|10|4|1130|12|853|7|90.44|166.40|139.77|186.41|978.39|633.08|1164.80|0.00|0.00|163.03|978.39|978.39|1141.42|1141.42|345.31| +2450831|29482|2450908|95527|1543750|2373|36596|95527|1543750|2373|36596|4|82|9|1|9598|120|853|6|79.52|171.76|15.45|937.86|92.70|477.12|1030.56|4.63|0.00|164.88|92.70|97.33|257.58|262.21|-384.42| +2450831|29482|2450908|95527|1543750|2373|36596|95527|1543750|2373|36596|4|104|12|3|5438|85|853|71|57.63|103.15|46.41|4028.54|3295.11|4091.73|7323.65|263.60|0.00|1244.63|3295.11|3558.71|4539.74|4803.34|-796.62| +2450831|29482|2450873|95527|1543750|2373|36596|95527|1543750|2373|36596|4|65|15|4|3859|150|853|90|9.78|24.25|10.67|1222.20|960.30|880.20|2182.50|19.20|0.00|937.80|960.30|979.50|1898.10|1917.30|80.10| +2450831|29482|2450844|95527|1543750|2373|36596|95527|1543750|2373|36596|4|14|20|1|12895|203|853|85|98.88|197.76|170.07|2353.65|14455.95|8404.80|16809.60|1156.47|0.00|6555.20|14455.95|15612.42|21011.15|22167.62|6051.15| +2450831|37021|2450851|61485|736653|6221|3436|61485|736653|6221|3436|4|51|15|1|13561|165|854|33|36.29|64.59|62.65|64.02|2067.45|1197.57|2131.47|186.07|0.00|809.82|2067.45|2253.52|2877.27|3063.34|869.88| +2450831|37021|2450893|61485|736653|6221|3436|61485|736653|6221|3436|4|72|9|2|17659|131|854|50|50.48|60.57|9.69|2544.00|484.50|2524.00|3028.50|0.00|0.00|454.00|484.50|484.50|938.50|938.50|-2039.50| +2450831|37021|2450843|61485|736653|6221|3436|61485|736653|6221|3436|4|15|15|1|15655|142|854|48|29.22|34.77|11.12|1135.20|533.76|1402.56|1668.96|32.02|0.00|767.52|533.76|565.78|1301.28|1333.30|-868.80| +2450831|37021|2450838|61485|736653|6221|3436|61485|736653|6221|3436|4|25|5|5|5252|100|854|61|48.96|141.98|93.70|2945.08|5715.70|2986.56|8660.78|99.45|743.04|1385.31|4972.66|5072.11|6357.97|6457.42|1986.10| +2450831|19147|2450833|64456|1234475|6894|14913|64456|1234475|6894|14913|2|95|17|2|16753|49|855|29|24.94|66.58|48.60|521.42|1409.40|723.26|1930.82|84.56|0.00|945.98|1409.40|1493.96|2355.38|2439.94|686.14| +2450831|19147|2450918|64456|1234475|6894|14913|64456|1234475|6894|14913|2|4|15|3|2038|226|855|77|67.11|150.32|84.17|5093.55|6481.09|5167.47|11574.64|542.46|453.67|3934.70|6027.42|6569.88|9962.12|10504.58|859.95| +2450831|19147|2450897|64456|1234475|6894|14913|64456|1234475|6894|14913|2|76|2|1|9961|56|855|1|29.52|85.60|27.39|58.21|27.39|29.52|85.60|2.19|0.00|17.12|27.39|29.58|44.51|46.70|-2.13| +2450831|19147|2450921|64456|1234475|6894|14913|64456|1234475|6894|14913|2|81|10|2|15572|286|855|91|74.46|174.23|111.50|5708.43|10146.50|6775.86|15854.93|304.39|0.00|7134.40|10146.50|10450.89|17280.90|17585.29|3370.64| +2450831|19147|2450863|64456|1234475|6894|14913|64456|1234475|6894|14913|2|102|14|2|13844|181|855|35|91.78|273.50|123.07|5265.05|4307.45|3212.30|9572.50|12.92|4092.07|191.45|215.38|228.30|406.83|419.75|-2996.92| +2450831|19147|2450848|64456|1234475|6894|14913|64456|1234475|6894|14913|2|19|15|4|3184|102|855|50|68.03|80.27|73.04|361.50|3652.00|3401.50|4013.50|36.52|0.00|1204.00|3652.00|3688.52|4856.00|4892.52|250.50| +2450831|19147|2450912|64456|1234475|6894|14913|64456|1234475|6894|14913|2|103|16|1|6800|48|855|18|74.78|186.20|80.06|1910.52|1441.08|1346.04|3351.60|14.41|0.00|201.06|1441.08|1455.49|1642.14|1656.55|95.04| +2450831|19147|2450907|64456|1234475|6894|14913|64456|1234475|6894|14913|2|37|4|3|12878|95|855|97|23.05|24.89|7.71|1666.46|747.87|2235.85|2414.33|12.78|605.77|1158.18|142.10|154.88|1300.28|1313.06|-2093.75| +2450831|19147|2450837|64456|1234475|6894|14913|64456|1234475|6894|14913|2|97|19|3|11968|259|855|43|26.50|40.81|22.03|807.54|947.29|1139.50|1754.83|37.89|0.00|666.50|947.29|985.18|1613.79|1651.68|-192.21| +2450831|19147|2450870|64456|1234475|6894|14913|64456|1234475|6894|14913|2|64|7|2|13192|27|855|50|7.39|12.56|4.39|408.50|219.50|369.50|628.00|10.97|0.00|106.50|219.50|230.47|326.00|336.97|-150.00| +2450831|19147|2450864|64456|1234475|6894|14913|64456|1234475|6894|14913|2|8|10|1|11012|17|855|11|65.00|66.30|23.20|474.10|255.20|715.00|729.30|5.10|0.00|138.49|255.20|260.30|393.69|398.79|-459.80| +2450831|54978|2450901|97945|156176|1254|14285|97945|156176|1254|14285|4|82|19|5|14740|102|856|4|68.90|134.35|49.70|338.60|198.80|275.60|537.40|9.94|0.00|107.48|198.80|208.74|306.28|316.22|-76.80| +2450831|54978|2450903|97945|156176|1254|14285|97945|156176|1254|14285|4|39|18|2|15313|7|856|100|68.57|196.79|179.07|1772.00|17907.00|6857.00|19679.00|716.28|0.00|5313.00|17907.00|18623.28|23220.00|23936.28|11050.00| +2450831|54978|2450869|97945|156176|1254|14285|97945|156176|1254|14285|4|2|18|5|11191|134|856|48|89.22|150.78|134.19|796.32|6441.12|4282.56|7237.44|257.64|0.00|1447.20|6441.12|6698.76|7888.32|8145.96|2158.56| +2450831|54978|2450896|97945|156176|1254|14285|97945|156176|1254|14285|4|105|4|3|488|153|856|35|92.25|205.71|20.57|6479.90|719.95|3228.75|7199.85|14.39|0.00|0.00|719.95|734.34|719.95|734.34|-2508.80| +2450831|54978|2450866|97945|156176|1254|14285|97945|156176|1254|14285|4|41|5|1|11275|13|856|52|97.53|231.14|108.63|6370.52|5648.76|5071.56|12019.28|0.00|0.00|2043.08|5648.76|5648.76|7691.84|7691.84|577.20| +2450831|54978|2450877|97945|156176|1254|14285|97945|156176|1254|14285|4|43|14|3|11842|212|856|14|7.14|9.21|6.81|33.60|95.34|99.96|128.94|1.90|0.00|20.58|95.34|97.24|115.92|117.82|-4.62| +2450831|54978|2450882|97945|156176|1254|14285|97945|156176|1254|14285|4|73|13|3|6865|188|856|8|33.93|57.34|51.03|50.48|408.24|271.44|458.72|28.57|0.00|77.92|408.24|436.81|486.16|514.73|136.80| +2450831|26060|2450870|52933|3581|5782|42830|52933|3581|5782|42830|2|74|10|2|13160|59|857|31|12.48|21.84|18.12|115.32|561.72|386.88|677.04|50.55|0.00|318.06|561.72|612.27|879.78|930.33|174.84| +2450831|26060|2450902|52933|3581|5782|42830|52933|3581|5782|42830|2|40|1|4|5596|215|857|2|44.59|60.64|57.00|7.28|114.00|89.18|121.28|5.70|0.00|15.76|114.00|119.70|129.76|135.46|24.82| +2450831|26060|2450876|52933|3581|5782|42830|52933|3581|5782|42830|2|34|7|2|6496|300|857|21|24.26|66.71|37.35|616.56|784.35|509.46|1400.91|7.84|0.00|55.86|784.35|792.19|840.21|848.05|274.89| +2450831|26060|2450837|52933|3581|5782|42830|52933|3581|5782|42830|2|95|15|3|10579|244|857|12|62.59|93.88|40.36|642.24|484.32|751.08|1126.56|38.74|0.00|33.72|484.32|523.06|518.04|556.78|-266.76| +2450831|26060|2450885|52933|3581|5782|42830|52933|3581|5782|42830|2|56|4|4|4894|84|857|39|96.25|179.98|134.98|1755.00|5264.22|3753.75|7019.22|421.13|0.00|0.00|5264.22|5685.35|5264.22|5685.35|1510.47| +2450831|26060|2450888|52933|3581|5782|42830|52933|3581|5782|42830|2|83|7|4|12424|110|857|70|52.01|149.26|22.38|8881.60|1566.60|3640.70|10448.20|140.99|0.00|1880.20|1566.60|1707.59|3446.80|3587.79|-2074.10| +2450831|26060|2450851|52933|3581|5782|42830|52933|3581|5782|42830|2|49|13|1|13792|35|857|56|41.70|120.51|16.87|5803.84|944.72|2335.20|6748.56|18.89|0.00|539.84|944.72|963.61|1484.56|1503.45|-1390.48| +2450831|26060|2450901|52933|3581|5782|42830|52933|3581|5782|42830|2|99|7|2|5666|215|857|48|66.67|158.67|3.17|7464.00|152.16|3200.16|7616.16|10.65|0.00|304.32|152.16|162.81|456.48|467.13|-3048.00| +2450831|26060|2450873|52933|3581|5782|42830|52933|3581|5782|42830|2|24|9|4|5947|236|857|63|9.88|11.95|10.87|68.04|684.81|622.44|752.85|0.00|0.00|217.98|684.81|684.81|902.79|902.79|62.37| +2450831|26060|2450890|52933|3581|5782|42830|52933|3581|5782|42830|2|17|15|2|2383|109|857|69|57.33|162.24|113.56|3358.92|7835.64|3955.77|11194.56|705.20|0.00|4589.19|7835.64|8540.84|12424.83|13130.03|3879.87| +2450831|10748|2450839|71757|1238493|76|1612|71757|1238493|76|1612|2|71|11|3|11684|54|858|25|98.41|262.75|0.00|6568.75|0.00|2460.25|6568.75|0.00|0.00|2167.50|0.00|0.00|2167.50|2167.50|-2460.25| +2450831|10748|2450857|71757|1238493|76|1612|71757|1238493|76|1612|2|78|9|4|3982|112|858|16|88.40|183.87|75.38|1735.84|1206.08|1414.40|2941.92|0.00|0.00|1412.00|1206.08|1206.08|2618.08|2618.08|-208.32| +2450831|10748|2450879|71757|1238493|76|1612|71757|1238493|76|1612|2|32|12|5|6739|108|858|96|20.91|21.53|7.32|1364.16|702.72|2007.36|2066.88|63.24|0.00|206.40|702.72|765.96|909.12|972.36|-1304.64| +2450831|10748|2450856|71757|1238493|76|1612|71757|1238493|76|1612|2|35|4|1|16531|152|858|15|25.45|44.28|11.95|484.95|179.25|381.75|664.20|1.79|0.00|139.35|179.25|181.04|318.60|320.39|-202.50| +2450831|10748|2450885|71757|1238493|76|1612|71757|1238493|76|1612|2|90|17|5|7987|31|858|13|75.66|172.50|5.17|2175.29|67.21|983.58|2242.50|0.67|0.00|493.35|67.21|67.88|560.56|561.23|-916.37| +2450831|10748|2450880|71757|1238493|76|1612|71757|1238493|76|1612|2|37|17|1|14647|167|858|16|86.20|226.70|142.82|1342.08|2285.12|1379.20|3627.20|159.95|0.00|834.24|2285.12|2445.07|3119.36|3279.31|905.92| +2450831|10748|2450875|71757|1238493|76|1612|71757|1238493|76|1612|2|40|19|2|10100|203|858|25|85.85|193.16|94.64|2463.00|2366.00|2146.25|4829.00|0.00|0.00|2028.00|2366.00|2366.00|4394.00|4394.00|219.75| +2450831|10748|2450881|71757|1238493|76|1612|71757|1238493|76|1612|2|71|20|3|2222|259|858|79|48.29|114.44|48.06|5244.02|3796.74|3814.91|9040.76|227.80|0.00|4067.71|3796.74|4024.54|7864.45|8092.25|-18.17| +2450831|10748|2450909|71757|1238493|76|1612|71757|1238493|76|1612|2|5|13|4|715|198|858|11|63.13|157.19|40.86|1279.63|449.46|694.43|1729.09|31.46|0.00|207.46|449.46|480.92|656.92|688.38|-244.97| +2450831|10748|2450909|71757|1238493|76|1612|71757|1238493|76|1612|2|44|1|2|8048|275|858|70|22.06|53.60|30.55|1613.50|2138.50|1544.20|3752.00|38.92|1582.49|412.30|556.01|594.93|968.31|1007.23|-988.19| +2450831|10748|2450901|71757|1238493|76|1612|71757|1238493|76|1612|2|25|13|3|10786|13|858|54|34.41|98.41|8.85|4836.24|477.90|1858.14|5314.14|33.45|0.00|212.22|477.90|511.35|690.12|723.57|-1380.24| +2450831|48995|2450846|50393|963528|4310|18692|50393|963528|4310|18692|1|106|6|3|286|230|859|34|56.41|161.33|75.82|2907.34|2577.88|1917.94|5485.22|51.55|0.00|1919.64|2577.88|2629.43|4497.52|4549.07|659.94| +2450831|48995|2450877|50393|963528|4310|18692|50393|963528|4310|18692|1|40|13|3|10066|275|859|98|23.82|51.45|9.26|4134.62|907.48|2334.36|5042.10|81.67|0.00|49.98|907.48|989.15|957.46|1039.13|-1426.88| +2450831|48995|2450892|50393|963528|4310|18692|50393|963528|4310|18692|1|35|11|3|13628|45|859|81|4.02|7.67|2.30|434.97|186.30|325.62|621.27|1.86|0.00|298.08|186.30|188.16|484.38|486.24|-139.32| +2450831|48995|2450905|50393|963528|4310|18692|50393|963528|4310|18692|1|79|12|1|14527|5|859|41|81.83|99.83|67.88|1309.95|2783.08|3355.03|4093.03|222.64|0.00|450.18|2783.08|3005.72|3233.26|3455.90|-571.95| +2450831|48995|2450835|50393|963528|4310|18692|50393|963528|4310|18692|1|103|8|5|13003|177|859|2|26.99|52.36|51.83|1.06|103.66|53.98|104.72|2.03|2.07|51.30|101.59|103.62|152.89|154.92|47.61| +2450831|48995|2450846|50393|963528|4310|18692|50393|963528|4310|18692|1|80|2|1|5026|57|859|56|93.94|144.66|98.36|2592.80|5508.16|5260.64|8100.96|110.16|0.00|161.84|5508.16|5618.32|5670.00|5780.16|247.52| +2450831|48995|2450836|50393|963528|4310|18692|50393|963528|4310|18692|1|101|19|5|12559|38|859|33|66.36|88.92|45.34|1438.14|1496.22|2189.88|2934.36|134.65|0.00|0.00|1496.22|1630.87|1496.22|1630.87|-693.66| +2450831|48995|2450872|50393|963528|4310|18692|50393|963528|4310|18692|1|4|13|4|6838|95|859|53|54.43|72.93|48.13|1314.40|2550.89|2884.79|3865.29|153.05|0.00|1352.56|2550.89|2703.94|3903.45|4056.50|-333.90| +2450831|48995|2450888|50393|963528|4310|18692|50393|963528|4310|18692|1|51|6|1|12656|258|859|49|57.19|145.26|85.70|2918.44|4199.30|2802.31|7117.74|41.99|0.00|1992.83|4199.30|4241.29|6192.13|6234.12|1396.99| +2450831|37025|2450893|23250|1342816|3982|25948|23250|1342816|3982|25948|1|7|16|3|3290|261|860|14|92.04|97.56|79.99|245.98|1119.86|1288.56|1365.84|67.19|0.00|682.92|1119.86|1187.05|1802.78|1869.97|-168.70| +2450831|37025|2450866|23250|1342816|3982|25948|23250|1342816|3982|25948|1|33|5|1|5936|274|860|39|52.42|148.87|31.26|4586.79|1219.14|2044.38|5805.93|85.33|0.00|1857.57|1219.14|1304.47|3076.71|3162.04|-825.24| +2450831|37025|2450856|23250|1342816|3982|25948|23250|1342816|3982|25948|1|12|18|5|11950|186|860|56|35.29|73.75|66.37|413.28|3716.72|1976.24|4130.00|260.17|0.00|123.76|3716.72|3976.89|3840.48|4100.65|1740.48| +2450831|37025|2450904|23250|1342816|3982|25948|23250|1342816|3982|25948|1|90|9|2|13627|261|860|50|47.69|76.30|46.54|1488.00|2327.00|2384.50|3815.00|23.27|0.00|1602.00|2327.00|2350.27|3929.00|3952.27|-57.50| +2450831|37025|2450844|23250|1342816|3982|25948|23250|1342816|3982|25948|1|69|14|3|16220|234|860|44|73.72|151.86|110.85|1804.44|4877.40|3243.68|6681.84|341.41|0.00|2471.92|4877.40|5218.81|7349.32|7690.73|1633.72| +2450831|37025|2450850|23250|1342816|3982|25948|23250|1342816|3982|25948|1|108|20|3|1738|267|860|76|43.91|108.01|65.88|3201.88|5006.88|3337.16|8208.76|350.48|0.00|2790.72|5006.88|5357.36|7797.60|8148.08|1669.72| +2450831|37025|2450835|23250|1342816|3982|25948|23250|1342816|3982|25948|1|60|17|5|12949|211|860|54|53.47|56.14|47.71|455.22|2576.34|2887.38|3031.56|47.40|1983.78|1060.56|592.56|639.96|1653.12|1700.52|-2294.82| +2450831|37025|2450872|23250|1342816|3982|25948|23250|1342816|3982|25948|1|104|5|4|2672|240|860|9|25.40|64.77|11.65|478.08|104.85|228.60|582.93|2.09|0.00|116.55|104.85|106.94|221.40|223.49|-123.75| +2450831|37025|2450898|23250|1342816|3982|25948|23250|1342816|3982|25948|1|64|13|2|14606|123|860|24|90.01|95.41|62.01|801.60|1488.24|2160.24|2289.84|59.52|0.00|572.40|1488.24|1547.76|2060.64|2120.16|-672.00| +2450831|59726|2450842|84180|875404|5828|12299|97284|1905547|6371|5242|1|104|8|2|12682|220|861|56|31.48|39.66|36.48|178.08|2042.88|1762.88|2220.96|40.85|0.00|554.96|2042.88|2083.73|2597.84|2638.69|280.00| +2450831|59726|2450833|84180|875404|5828|12299|97284|1905547|6371|5242|1|63|13|2|15290|36|861|100|76.77|103.63|31.08|7255.00|3108.00|7677.00|10363.00|217.56|0.00|1347.00|3108.00|3325.56|4455.00|4672.56|-4569.00| +2450831|59726|2450869|84180|875404|5828|12299|97284|1905547|6371|5242|1|37|12|4|9376|188|861|24|6.92|8.30|6.30|48.00|151.20|166.08|199.20|1.51|0.00|69.60|151.20|152.71|220.80|222.31|-14.88| +2450831|59726|2450900|84180|875404|5828|12299|97284|1905547|6371|5242|1|24|5|1|16732|233|861|36|56.37|130.21|32.55|3515.76|1171.80|2029.32|4687.56|0.00|0.00|1546.56|1171.80|1171.80|2718.36|2718.36|-857.52| +|||84180|||12299|97284||||1|31|10|5|9734|167|861|50|||26.00|3700.00||||||2250.00||1404.00|3550.00|3654.00|-1989.50| +2450831|59726|2450874|84180|875404|5828|12299|97284|1905547|6371|5242|1|22|2|3|11995|242|861|36|90.53|112.25|14.59|3515.76|525.24|3259.08|4041.00|22.06|210.09|1778.04|315.15|337.21|2093.19|2115.25|-2943.93| +2450831|59726|2450893|84180|875404|5828|12299|97284|1905547|6371|5242|1|22|9|3|16459|285|861|80|2.57|5.83|1.86|317.60|148.80|205.60|466.40|8.92|0.00|195.20|148.80|157.72|344.00|352.92|-56.80| +2450831|59726|2450860|84180|875404|5828|12299|97284|1905547|6371|5242|1|31|18|5|14914|275|861|50|57.58|93.27|57.82|1772.50|2891.00|2879.00|4663.50|57.82|0.00|2331.50|2891.00|2948.82|5222.50|5280.32|12.00| +2450831|50846|2450874|22844|1614810|6776|14295|22844|1614810|6776|14295|2|68|14|1|7618|130|862|71|4.28|7.78|1.16|470.02|82.36|303.88|552.38|3.29|0.00|115.73|82.36|85.65|198.09|201.38|-221.52| +2450831||||1614810|||22844|1614810|6776|14295||||1|1052||862|||159.80|145.41|1338.27||||||||14064.05|14117.40|14658.32|6947.10| +2450831|50846|2450877|22844|1614810|6776|14295|22844|1614810|6776|14295|2|33|3|2|7279|183|862|14|57.49|86.80|77.25|133.70|1081.50|804.86|1215.20|32.44|0.00|315.84|1081.50|1113.94|1397.34|1429.78|276.64| +2450831|50846|2450894|22844|1614810|6776|14295|22844|1614810|6776|14295|2|15|7|3|3700|73|862|18|72.14|76.46|25.23|922.14|454.14|1298.52|1376.28|13.62|0.00|674.28|454.14|467.76|1128.42|1142.04|-844.38| +2450831|50846|2450889|22844|1614810|6776|14295|22844|1614810|6776|14295|2|57|14|4|331|67|862|36|8.52|24.62|22.40|79.92|806.40|306.72|886.32|0.00|0.00|398.52|806.40|806.40|1204.92|1204.92|499.68| +2450831|50846|2450897|22844|1614810|6776|14295|22844|1614810|6776|14295|2|22|12|1|8800|208|862|25|14.58|32.22|5.79|660.75|144.75|364.50|805.50|13.02|0.00|144.75|144.75|157.77|289.50|302.52|-219.75| +2450831|50846|2450913|22844|1614810|6776|14295|22844|1614810|6776|14295|2|51|1|2|529|199|862|68|9.16|11.08|4.21|467.16|286.28|622.88|753.44|0.00|0.00|308.72|286.28|286.28|595.00|595.00|-336.60| +2450831|50846|2450880|22844|1614810|6776|14295|22844|1614810|6776|14295|2|53|2|2|8371|107|862|27|16.45|31.41|5.96|687.15|160.92|444.15|848.07|9.65|0.00|415.53|160.92|170.57|576.45|586.10|-283.23| +2450831|50846|2450870|22844|1614810|6776|14295|22844|1614810|6776|14295|2|84|6|3|15613|271|862|15|66.92|85.65|36.82|732.45|552.30|1003.80|1284.75|11.04|0.00|411.00|552.30|563.34|963.30|974.34|-451.50| +2450831|50846|2450898|22844|1614810|6776|14295|22844|1614810|6776|14295|2|87|18|3|11809|211|862|39|68.11|152.56|59.49|3629.73|2320.11|2656.29|5949.84|139.20|0.00|2260.83|2320.11|2459.31|4580.94|4720.14|-336.18| +2450831|50846|2450918|22844|1614810|6776|14295|22844|1614810|6776|14295|2|1|6|1|7435|285|862|29|92.57|162.92|86.34|2220.82|2503.86|2684.53|4724.68|75.11|0.00|1417.23|2503.86|2578.97|3921.09|3996.20|-180.67| +2450831|35761|2450884|34433|214161|736|6064|34433|214161|736|6064|1|88|5|1|11552|170|863|86|56.36|126.24|103.51|1954.78|8901.86|4846.96|10856.64|178.03|0.00|108.36|8901.86|9079.89|9010.22|9188.25|4054.90| +2450831|35761|2450847|34433|214161|736|6064|34433|214161|736|6064|1|23|10|1|7111|228|863|34|12.36|13.47|12.52|32.30|425.68|420.24|457.98|29.79|0.00|59.50|425.68|455.47|485.18|514.97|5.44| +2450831|35761|2450861|||736|||||||40|||10420|32|863||79.49|85.84|84.98|5.16||476.94||17.74|66.28|221.46|443.60||665.06|682.80|-33.34| +2450831|35761|2450921|34433|214161|736|6064|34433|214161|736|6064|1|32|9|1|3703|127|863|71|42.38|77.97|49.90|1992.97|3542.90|3008.98|5535.87|146.32|1452.58|2712.20|2090.32|2236.64|4802.52|4948.84|-918.66| +2450831|35761|2450887|34433|214161|736|6064|34433|214161|736|6064|1|67|2|1|1888|211|863|62|71.76|129.88|59.74|4348.68|3703.88|4449.12|8052.56|175.93|185.19|482.98|3518.69|3694.62|4001.67|4177.60|-930.43| +2450831|35761|2450919|34433|214161|736|6064|34433|214161|736|6064|1|6|11|4|12010|111|863|4|65.89|102.12|22.46|318.64|89.84|263.56|408.48|7.18|0.00|183.80|89.84|97.02|273.64|280.82|-173.72| +2450831|35761|2450906|34433|214161|736|6064|34433|214161|736|6064|1|7|9|5|4489|19|863|12|26.25|34.65|2.77|382.56|33.24|315.00|415.80|1.32|0.00|16.56|33.24|34.56|49.80|51.12|-281.76| +2450831|35761|2450885|34433|214161|736|6064|34433|214161|736|6064|1|1|17|2|2336|82|863|27|74.19|119.44|10.74|2934.90|289.98|2003.13|3224.88|20.29|0.00|1450.98|289.98|310.27|1740.96|1761.25|-1713.15| +2450831|35761|2450875|34433|214161|736|6064|34433|214161|736|6064|1|97|2|2|13436|107|863|25|13.84|37.50|27.75|243.75|693.75|346.00|937.50|46.62|111.00|0.00|582.75|629.37|582.75|629.37|236.75| +2450831|35761|2450854|34433|214161|736|6064|34433|214161|736|6064|1|89|12|4|6283|5|863|88|49.68|69.55|61.89|674.08|5446.32|4371.84|6120.40|0.00|0.00|2937.44|5446.32|5446.32|8383.76|8383.76|1074.48| +2450831|35761|2450887|34433|214161|736|6064|34433|214161|736|6064|1|43|17|3|5818|189|863|75|89.61|175.63|128.20|3557.25|9615.00|6720.75|13172.25|559.59|288.45|2238.75|9326.55|9886.14|11565.30|12124.89|2605.80| +2450831|35761|2450835|34433|214161|736|6064|34433|214161|736|6064|1|68|16|4|914|75|863|26|54.11|142.85|112.85|780.00|2934.10|1406.86|3714.10|29.34|0.00|1039.74|2934.10|2963.44|3973.84|4003.18|1527.24| +2450831|50025|2450834|33070|570828|3786|1144|99268|702660|2018|8576|4|73|16|3|15829|29|864|35|35.19|85.15|6.81|2741.90|238.35|1231.65|2980.25|14.30|0.00|1340.85|238.35|252.65|1579.20|1593.50|-993.30| +2450831|50025|2450865|33070|570828|3786|1144|99268|702660|2018|8576|4|64|2|2|3440|295|864|53|53.59|87.35|25.33|3287.06|1342.49|2840.27|4629.55|107.39|0.00|508.80|1342.49|1449.88|1851.29|1958.68|-1497.78| +2450831|50025|2450885|33070|570828|3786|1144|99268|702660|2018|8576|4|15|14|3|5527|283|864|17|7.13|20.60|9.06|196.18|154.02|121.21|350.20|3.08|0.00|97.92|154.02|157.10|251.94|255.02|32.81| +2450831|50025|2450834|33070|570828|3786|1144|99268|702660|2018|8576|4|35|6|5|17443|273|864|20|84.54|243.47|197.21|925.20|3944.20|1690.80|4869.40|315.53|0.00|1509.40|3944.20|4259.73|5453.60|5769.13|2253.40| +2450831|50025|2450860|33070|570828|3786|1144|99268|702660|2018|8576|4|41|6|2|14566|120|864|28|57.31|108.88|87.10|609.84|2438.80|1604.68|3048.64|121.94|0.00|243.88|2438.80|2560.74|2682.68|2804.62|834.12| +2450831|50025|2450837|33070|570828|3786|1144|99268|702660|2018|8576|4|70|10|3|9259|176|864|71|34.82|36.90|28.04|629.06|1990.84|2472.22|2619.90|39.81|0.00|1178.60|1990.84|2030.65|3169.44|3209.25|-481.38| +2450831|50025|2450848|33070|570828|3786|1144|99268|702660|2018|8576|4|12|15|4|7478|157|864|94|60.97|79.26|13.47|6184.26|1266.18|5731.18|7450.44|113.95|0.00|2905.54|1266.18|1380.13|4171.72|4285.67|-4465.00| +2450831|50025|2450874|33070|570828|3786|1144|99268|702660|2018|8576|4|28|19|3|5965|85|864|44|5.63|6.47|5.04|62.92|221.76|247.72|284.68|2.21|0.00|91.08|221.76|223.97|312.84|315.05|-25.96| +2450831|50025|2450871|33070|570828|3786|1144|99268|702660|2018|8576|4|3|15|1|8098|53|864|71|25.43|34.83|27.16|544.57|1928.36|1805.53|2472.93|57.85|0.00|815.79|1928.36|1986.21|2744.15|2802.00|122.83| +2450831|50025|2450917|33070|570828|3786|1144|99268|702660|2018|8576|4|73|9|2|11665|163|864|38|93.62|220.00|184.80|1337.60|7022.40|3557.56|8360.00|210.67|0.00|1504.80|7022.40|7233.07|8527.20|8737.87|3464.84| +2450831|50025|2450875|33070|570828|3786|1144|99268|702660|2018|8576|4|93|11|5|3907|24|864|99|20.30|37.55|28.53|892.98|2824.47|2009.70|3717.45|169.46|0.00|1486.98|2824.47|2993.93|4311.45|4480.91|814.77| +2450831|50025|2450856|33070|570828|3786|1144|99268|702660|2018|8576|4|71|7|5|1891|33|864|22|7.27|15.70|3.61|265.98|79.42|159.94|345.40|4.76|0.00|3.30|79.42|84.18|82.72|87.48|-80.52| +2450831|50025|2450853|33070|570828|3786|1144|99268|702660|2018|8576|4|98|20|3|15439|42|864|11|3.51|9.40|6.67|30.03|73.37|38.61|103.40|6.60|0.00|29.92|73.37|79.97|103.29|109.89|34.76| +2450831|50025|2450859|33070|570828|3786|1144|99268|702660|2018|8576|4|40|19|1|17827|115|864|60|6.91|11.33|3.17|489.60|190.20|414.60|679.80|11.41|0.00|237.60|190.20|201.61|427.80|439.21|-224.40| +2450831|77219|2450835|14915|406237|3159|14081|14915|406237|3159|14081|2|87|4|1|10999|117|865|8|49.76|96.53|89.77|54.08|718.16|398.08|772.24|57.45|0.00|131.28|718.16|775.61|849.44|906.89|320.08| +2450831|77219|2450850|14915|406237|3159|14081|14915|406237|3159|14081|2|37|6|3|10562|229|865|59|3.66|5.56|1.89|216.53|111.51|215.94|328.04|8.92|0.00|84.96|111.51|120.43|196.47|205.39|-104.43| +2450831|77219|2450856|14915|406237|3159|14081|14915|406237|3159|14081|2|66|5|1|11863|88|865|66|33.14|57.99|2.89|3636.60|190.74|2187.24|3827.34|11.44|0.00|152.46|190.74|202.18|343.20|354.64|-1996.50| +2450831|77219|2450884|14915|406237|3159|14081|14915|406237|3159|14081|2|18|8|4|6844|159|865|61|84.11|149.71|97.31|3196.40|5935.91|5130.71|9132.31|328.25|1246.54|2008.73|4689.37|5017.62|6698.10|7026.35|-441.34| +2450831|77219|2450857|14915|406237|3159|14081|14915|406237|3159|14081|2|20|4|5|7699|85|865|9|8.82|17.72|11.34|57.42|102.06|79.38|159.48|6.52|29.59|63.72|72.47|78.99|136.19|142.71|-6.91| +2450831|77219|2450838|14915|406237|3159|14081|14915|406237|3159|14081|2|90|7|1|16682|149|865|56|25.94|53.17|40.94|684.88|2292.64|1452.64|2977.52|91.70|0.00|952.56|2292.64|2384.34|3245.20|3336.90|840.00| +2450831|52695|2450899|89530|277583|6436|37336|89530|277583|6436|37336|4|69|16|2|7795|51|866|8|44.56|45.00|1.80|345.60|14.40|356.48|360.00|1.00|0.00|93.60|14.40|15.40|108.00|109.00|-342.08| +2450831|52695|2450834|89530|277583|6436|37336|89530|277583|6436|37336|4|96|14|4|4969|87|866|86|15.43|33.94|24.77|788.62|2130.22|1326.98|2918.84|42.60|0.00|700.04|2130.22|2172.82|2830.26|2872.86|803.24| +2450831|52695|2450834|89530|277583|6436|37336|89530|277583|6436|37336|4|55|6|2|15430|82|866|34|70.71|183.13|87.90|3237.82|2988.60|2404.14|6226.42|59.77|0.00|2490.50|2988.60|3048.37|5479.10|5538.87|584.46| +2450831|52695|2450854|89530|277583|6436|37336|89530|277583|6436|37336|4|67|12|3|11126|230|866|60|67.59|79.08|40.33|2325.00|2419.80|4055.40|4744.80|217.78|0.00|1850.40|2419.80|2637.58|4270.20|4487.98|-1635.60| +2450831|35667|2450910|64679|710529|5979|8229|64679|710529|5979|8229|4|17|7|5|12487|281|867|47|10.09|17.45|0.69|787.72|32.43|474.23|820.15|0.32|0.00|262.26|32.43|32.75|294.69|295.01|-441.80| +2450831|35667|2450846|64679|710529|5979|8229|64679|710529|5979|8229|4|81|7|5|12164|206|867|89|70.60|148.96|141.51|663.05|12594.39|6283.40|13257.44|377.83|0.00|3711.30|12594.39|12972.22|16305.69|16683.52|6310.99| +2450831|35667|2450842|64679|710529|5979|8229|64679|710529|5979|8229|4|87|12|2|13130|148|867|37|3.01|3.76|0.00|139.12|0.00|111.37|139.12|0.00|0.00|58.09|0.00|0.00|58.09|58.09|-111.37| +2450831|35667|2450872|64679|710529|5979|8229|64679|710529|5979|8229|4|26|14|3|2192|112|867|73|84.94|231.88|48.69|13372.87|3554.37|6200.62|16927.24|71.08|0.00|5077.88|3554.37|3625.45|8632.25|8703.33|-2646.25| +2450831|35667|2450917|64679|710529|5979|8229|64679|710529|5979|8229|4|26|13|1|14980|194|867|32|35.89|102.64|88.27|459.84|2824.64|1148.48|3284.48|169.47|0.00|656.64|2824.64|2994.11|3481.28|3650.75|1676.16| +2450831|35667|2450892|64679|710529|5979|8229|64679|710529|5979|8229|4|50|17|3|9296|53|867|44|46.37|110.82|21.05|3949.88|926.20|2040.28|4876.08|24.54|435.31|1413.72|490.89|515.43|1904.61|1929.15|-1549.39| +2450831|35667|2450867|64679|710529|5979|8229|64679|710529|5979|8229|4|7|16|5|12254|151|867|12|43.38|68.10|23.15|539.40|277.80|520.56|817.20|0.00|0.00|335.04|277.80|277.80|612.84|612.84|-242.76| +2450831|35667|2450917|64679|710529|5979|8229|64679|710529|5979|8229|4|59|3|3|9679|258|867|98|13.45|28.24|7.90|1993.32|774.20|1318.10|2767.52|19.82|526.45|746.76|247.75|267.57|994.51|1014.33|-1070.35| +2450831|35667|2450915|64679|710529|5979|8229|64679|710529|5979|8229|4|50|19|1|14626|225|867|24|95.93|139.09|44.50|2270.16|1068.00|2302.32|3338.16|96.12|0.00|567.36|1068.00|1164.12|1635.36|1731.48|-1234.32| +2450831|49018|2450843|90339|191531|1523|27915|90339|191531|1523|27915|1|7|19|1|15442|129|868|90|2.43|2.47|1.18|116.10|106.20|218.70|222.30|3.18|0.00|50.40|106.20|109.38|156.60|159.78|-112.50| +2450831|49018|2450868|90339|191531|1523|27915|90339|191531|1523|27915|1|82|10|1|7810|265|868|10|93.95|130.59|9.14|1214.50|91.40|939.50|1305.90|7.31|0.00|391.70|91.40|98.71|483.10|490.41|-848.10| +2450831|49018|2450872|90339|191531|1523|27915|90339|191531|1523|27915|1|37|10|3|11218|14|868|44|84.69|218.50|6.55|9325.80|288.20|3726.36|9614.00|2.88|0.00|4037.88|288.20|291.08|4326.08|4328.96|-3438.16| +2450831|49018|2450905|90339|191531|1523|27915|90339|191531|1523|27915|1|25|13|4|15800|156|868|39|2.40|2.88|0.60|88.92|23.40|93.60|112.32|0.23|0.00|35.88|23.40|23.63|59.28|59.51|-70.20| +2450831|49018|2450834|90339|191531|1523|27915|90339|191531|1523|27915|1|88|20|4|5768|169|868|1|97.50|205.72|34.97|170.75|34.97|97.50|205.72|1.39|0.00|76.11|34.97|36.36|111.08|112.47|-62.53| +2450832|27387|2450850|75009|1233316|423|39930|75009|1233316|423|39930|4|12|3|1|10448|115|869|78|97.53|232.12|132.30|7785.96|10319.40|7607.34|18105.36|451.99|2786.23|5250.18|7533.17|7985.16|12783.35|13235.34|-74.17| +2450832|27387|2450852|75009|1233316|423|39930|75009|1233316|423|39930|4|69|18|3|17221|70|869|78|79.08|84.61|46.53|2970.24|3629.34|6168.24|6599.58|181.46|0.00|3101.28|3629.34|3810.80|6730.62|6912.08|-2538.90| +2450832|27387|2450908|75009|1233316|423|39930|75009|1233316|423|39930|4|81|6|5|176|152|869|76|39.18|63.86|57.47|485.64|4367.72|2977.68|4853.36|131.03|0.00|873.24|4367.72|4498.75|5240.96|5371.99|1390.04| +2450832|27387|2450897|75009|1233316|423|39930|75009|1233316|423|39930|4|34|18|1|13988|285|869|9|98.90|197.80|156.26|373.86|1406.34|890.10|1780.20|112.50|0.00|302.58|1406.34|1518.84|1708.92|1821.42|516.24| +2450832|27387|2450846|75009|1233316|423|39930|75009|1233316|423|39930|4|54|4|1|7789|155|869|79|58.64|98.51|4.92|7393.61|388.68|4632.56|7782.29|3.88|0.00|1322.46|388.68|392.56|1711.14|1715.02|-4243.88| +2450832|27387|2450903|75009|1233316|423|39930|75009|1233316|423|39930|4|85|2|1|4939|61|869|77|51.02|86.22|19.83|5112.03|1526.91|3928.54|6638.94|15.26|0.00|1061.83|1526.91|1542.17|2588.74|2604.00|-2401.63| +2450832|27387|2450906|75009|1233316|423|39930|75009|1233316|423|39930|4|23|17|3|12458|254|869|12|86.79|247.35|93.99|1840.32|1127.88|1041.48|2968.20|67.67|0.00|1365.36|1127.88|1195.55|2493.24|2560.91|86.40| +2450832|27387|2450859|75009|1233316|423|39930|75009|1233316|423|39930|4|94|4|3|2203|128|869|26|61.45|107.53|56.99|1314.04|1481.74|1597.70|2795.78|14.81|0.00|1146.08|1481.74|1496.55|2627.82|2642.63|-115.96| +2450832|27387|2450899|75009|1233316|423|39930|75009|1233316|423|39930|4|101|19|5|5570|29|869|94|53.72|117.10|51.52|6164.52|4842.88|5049.68|11007.40|48.42|0.00|219.96|4842.88|4891.30|5062.84|5111.26|-206.80| +2450832|27387|2450922|75009|1233316|423|39930|75009|1233316|423|39930|4|52|8|3|12829|65|869|80|45.25|54.75|8.76|3679.20|700.80|3620.00|4380.00|35.04|0.00|744.00|700.80|735.84|1444.80|1479.84|-2919.20| +2450832|27387|2450861|75009|1233316|423|39930|75009|1233316|423|39930|4|8|18|3|3868|122|869|84|82.72|136.48|107.81|2408.28|9056.04|6948.48|11464.32|452.80|0.00|457.80|9056.04|9508.84|9513.84|9966.64|2107.56| +2450832|27387|2450901|75009|1233316|423|39930|75009|1233316|423|39930|4|24|17|4|4360|283|869|98|10.82|28.13|16.31|1158.36|1598.38|1060.36|2756.74|111.88|0.00|27.44|1598.38|1710.26|1625.82|1737.70|538.02| +2450832|27387|2450875|75009|1233316|423|39930|75009|1233316|423|39930|4|59|6|1|9524|156|869|100|33.92|87.17|35.73|5144.00|3573.00|3392.00|8717.00|214.38|0.00|261.00|3573.00|3787.38|3834.00|4048.38|181.00| +2450832|58842|2450921|2065|374304|1743|17135|83732|1224413|1452|32091|2|13|17|3|1330|133|870|6|51.53|82.96|60.56|134.40|363.36|309.18|497.76|14.53|0.00|199.08|363.36|377.89|562.44|576.97|54.18| +2450832|58842|2450877|2065|374304|1743|17135|83732|1224413|1452|32091|2|57|1|3|907|115|870|48|14.29|31.00|30.38|29.76|1458.24|685.92|1488.00|5.24|1399.91|624.96|58.33|63.57|683.29|688.53|-627.59| +2450832|58842|2450913|2065|374304|1743|17135|83732|1224413|1452|32091|2|20|19|4|11396|245|870|46|85.76|242.70|140.76|4689.24|6474.96|3944.96|11164.20|517.99|0.00|4130.34|6474.96|6992.95|10605.30|11123.29|2530.00| +2450832|58842|2450858|2065|374304|1743|17135|83732|1224413|1452|32091|2|102|13|5|10222|282|870|98|77.59|110.17|76.01|3347.68|7448.98|7603.82|10796.66|0.00|0.00|3346.70|7448.98|7448.98|10795.68|10795.68|-154.84| +2450832|58842|2450843|2065|374304|1743|17135|83732|1224413|1452|32091|2|92|12|5|3044|104|870|26|61.60|65.91|29.00|959.66|754.00|1601.60|1713.66|22.62|0.00|496.86|754.00|776.62|1250.86|1273.48|-847.60| +2450832|58842|2450898|2065|374304|1743|17135|83732|1224413|1452|32091|2|75|8|1|10423|165|870|6|60.44|71.92|69.76|12.96|418.56|362.64|431.52|25.11|0.00|38.82|418.56|443.67|457.38|482.49|55.92| +2450832|58842|2450838|2065|374304|1743|17135|83732|1224413|1452|32091|2|82|5|3|11576|197|870|86|4.48|11.51|5.40|525.46|464.40|385.28|989.86|18.11|162.54|325.94|301.86|319.97|627.80|645.91|-83.42| +2450832|58842|2450853|2065|374304|1743|17135|83732|1224413|1452|32091|2|87|10|2|2125|194|870|45|99.59|147.39|85.48|2785.95|3846.60|4481.55|6632.55|115.39|0.00|2453.85|3846.60|3961.99|6300.45|6415.84|-634.95| +2450832|58842|2450899|2065|374304|1743|17135|83732|1224413|1452|32091|2|81|17|4|17386|130|870|63|56.95|85.99|76.53|595.98|4821.39|3587.85|5417.37|337.49|0.00|2708.37|4821.39|5158.88|7529.76|7867.25|1233.54| +2450832|11585|2450876|80916|1632834|1552|3870|80916|1632834|1552|3870|2|22|6|1|16436|298|871|95|8.51|24.16|11.11|1239.75|1055.45|808.45|2295.20|84.43|0.00|642.20|1055.45|1139.88|1697.65|1782.08|247.00| +2450832|11585|2450902|80916|1632834|1552|3870|80916|1632834|1552|3870|2|29|8|3|14941|133|871|72|28.31|55.20|46.92|596.16|3378.24|2038.32|3974.40|236.47|0.00|714.96|3378.24|3614.71|4093.20|4329.67|1339.92| +2450832|11585|2450861|80916|1632834|1552|3870|80916|1632834|1552|3870|2|38|5|5|10844|285|871|61|72.42|146.28|38.03|6603.25|2319.83|4417.62|8923.08|37.58|1902.26|891.82|417.57|455.15|1309.39|1346.97|-4000.05| +2450832|11585|2450868|80916|1632834|1552|3870|80916|1632834|1552|3870|2|23|10|3|11083|12|871|70|58.44|120.38|19.26|7078.40|1348.20|4090.80|8426.60|67.41|0.00|2106.30|1348.20|1415.61|3454.50|3521.91|-2742.60| +2450832|11585|2450868|80916|1632834|1552|3870|80916|1632834|1552|3870|2|82|4|2|6193|53|871|17|23.24|39.50|30.81|147.73|523.77|395.08|671.50|20.95|0.00|127.50|523.77|544.72|651.27|672.22|128.69| +2450832|11585|2450856|80916|1632834|1552|3870|80916|1632834|1552|3870|2|102|20|4|6163|300|871|72|5.68|10.67|7.04|261.36|506.88|408.96|768.24|15.20|0.00|214.56|506.88|522.08|721.44|736.64|97.92| +2450832|11585|2450921|80916|1632834|1552|3870|80916|1632834|1552|3870|2|51|3|5|2090|11|871|7|77.34|104.40|42.80|431.20|299.60|541.38|730.80|17.97|0.00|328.86|299.60|317.57|628.46|646.43|-241.78| +2450832|11585|2450875|80916|1632834|1552|3870|80916|1632834|1552|3870|2|16|8|5|10393|191|871|10|84.20|157.45|92.89|645.60|928.90|842.00|1574.50|0.00|0.00|488.00|928.90|928.90|1416.90|1416.90|86.90| +2450832|11585|2450892|80916|1632834|1552|3870|80916|1632834|1552|3870|2|55|15|5|10712|103|871|42|31.36|40.76|24.86|667.80|1044.12|1317.12|1711.92|0.00|0.00|804.30|1044.12|1044.12|1848.42|1848.42|-273.00| +2450832|11585|2450844|80916|1632834|1552|3870|80916|1632834|1552|3870|2|82|13|1|10504|2|871|32|33.97|68.95|21.37|1522.56|683.84|1087.04|2206.40|13.67|0.00|132.16|683.84|697.51|816.00|829.67|-403.20| +2450832|11585|2450863|80916|1632834|1552|3870|80916|1632834|1552|3870|2|72|11|5|10651|180|871|65|23.11|43.21|40.61|169.00|2639.65|1502.15|2808.65|0.00|0.00|814.45|2639.65|2639.65|3454.10|3454.10|1137.50| +2450832|11585|2450845|80916|1632834|1552|3870|80916|1632834|1552|3870|2|67|18|4|10382|245|871|20|87.40|144.21|86.52|1153.80|1730.40|1748.00|2884.20|64.02|449.90|1153.60|1280.50|1344.52|2434.10|2498.12|-467.50| +2450832|11585|2450856|80916|1632834|1552|3870|80916|1632834|1552|3870|2|8|1|2|5335|265|871|40|67.38|97.02|44.62|2096.00|1784.80|2695.20|3880.80|56.40|374.80|776.00|1410.00|1466.40|2186.00|2242.40|-1285.20| +2450832|18467|2450846|93330|794798|3466|13287|93330|794798|3466|13287|2|96|11|4|12931|81|872|57|73.73|151.14|77.08|4221.42|4393.56|4202.61|8614.98|87.87|0.00|2670.45|4393.56|4481.43|7064.01|7151.88|190.95| +2450832|18467|2450862|93330|794798|3466|13287|93330|794798|3466|13287|2|30|7|3|5791|18|872|66|28.55|29.40|0.88|1882.32|58.08|1884.30|1940.40|0.58|0.00|213.18|58.08|58.66|271.26|271.84|-1826.22| +2450832|18467|2450902|93330|794798|3466|13287|93330|794798|3466|13287|2|77|3|1|650|172|872|94|40.37|41.58|0.00|3908.52|0.00|3794.78|3908.52|0.00|0.00|742.60|0.00|0.00|742.60|742.60|-3794.78| +2450832|18467|2450871|93330|794798|3466|13287|93330|794798|3466|13287|2|23|14|1|6610|37|872|30|39.25|47.10|1.41|1370.70|42.30|1177.50|1413.00|2.18|5.92|438.00|36.38|38.56|474.38|476.56|-1141.12| +2450832|18467|2450841|93330|794798|3466|13287|93330|794798|3466|13287|2|70|9|1|12560|15|872|5|32.97|56.37|28.18|140.95|140.90|164.85|281.85|1.40|0.00|140.90|140.90|142.30|281.80|283.20|-23.95| +2450832|18467|2450840|93330|794798|3466|13287|93330|794798|3466|13287|2|14|18|3|14108|293|872|48|46.32|136.18|59.91|3660.96|2875.68|2223.36|6536.64|86.27|1437.84|2941.44|1437.84|1524.11|4379.28|4465.55|-785.52| +2450832|18467|2450885|93330|794798|3466|13287|93330|794798|3466|13287|2|63|6|1|2840|119|872|88|78.07|220.93|30.93|16720.00|2721.84|6870.16|19441.84|81.65|0.00|777.04|2721.84|2803.49|3498.88|3580.53|-4148.32| +2450832|18467|2450905|93330|794798|3466|13287|93330|794798|3466|13287|2|32|5|3|1510|26|872|67|84.80|164.51|59.22|7054.43|3967.74|5681.60|11022.17|277.74|0.00|5069.89|3967.74|4245.48|9037.63|9315.37|-1713.86| +2450832|18467|2450836|93330|794798|3466|13287|93330|794798|3466|13287|2|26|18|1|17062|192|872|68|19.29|48.61|34.51|958.80|2346.68|1311.72|3305.48|0.00|0.00|98.60|2346.68|2346.68|2445.28|2445.28|1034.96| +2450832|18467|2450910|93330|794798|3466|13287|93330|794798|3466|13287|2|90|15|1|12218|186|872|33|16.76|25.97|18.95|231.66|625.35|553.08|857.01|0.00|625.35|282.81|0.00|0.00|282.81|282.81|-553.08| +2450832|18467|2450864|93330|794798|3466|13287|93330|794798|3466|13287|2|9|19|2|10408|270|872|73|28.57|82.56|18.98|4641.34|1385.54|2085.61|6026.88|69.27|0.00|1385.54|1385.54|1454.81|2771.08|2840.35|-700.07| +2450832|24386|2450914|63431|1684133|6082|26031|63431|1684133|6082|26031|2|31|4|2|554|186|873|100|10.72|11.68|0.70|1098.00|70.00|1072.00|1168.00|4.90|0.00|315.00|70.00|74.90|385.00|389.90|-1002.00| +2450832|24386|2450883|63431|1684133|6082|26031|63431|1684133|6082|26031|2|16|18|5|9307|275|873|46|63.85|64.48|19.98|2047.00|919.08|2937.10|2966.08|82.71|0.00|1393.80|919.08|1001.79|2312.88|2395.59|-2018.02| +2450832|24386|2450847|63431|1684133|6082|26031|63431|1684133|6082|26031|2|76|14|1|7352|210|873|79|99.37|202.71|38.51|12971.80|3042.29|7850.23|16014.09|114.39|1612.41|319.95|1429.88|1544.27|1749.83|1864.22|-6420.35| +2450832|24386|2450838|63431|1684133|6082|26031|63431|1684133|6082|26031|2|66|11|4|2902|40|873|76|99.31|286.01|128.70|11955.56|9781.20|7547.56|21736.76|97.81|0.00|9781.20|9781.20|9879.01|19562.40|19660.21|2233.64| +2450832|24386|2450846|63431|1684133|6082|26031|63431|1684133|6082|26031|2|101|8|2|17482|205|873|96|31.70|87.49|20.99|6384.00|2015.04|3043.20|8399.04|64.88|1088.12|3275.52|926.92|991.80|4202.44|4267.32|-2116.28| +2450832|24386|2450919|63431|1684133|6082|26031|63431|1684133|6082|26031|2|28|11|5|14642|136|873|90|38.68|52.60|11.57|3692.70|1041.30|3481.20|4734.00|41.65|0.00|1514.70|1041.30|1082.95|2556.00|2597.65|-2439.90| +2450832|24386|2450918|63431|1684133|6082|26031|63431|1684133|6082|26031|2|50|7|5|11077|89|873|54|54.61|117.41|5.87|6023.16|316.98|2948.94|6340.14|12.67|0.00|2789.64|316.98|329.65|3106.62|3119.29|-2631.96| +2450832|79920|2450856|88089|980039|5589|13892|44805|225792|942|42845|1|44|5|3|15806|92|874|42|38.94|84.11|57.19|1130.64|2401.98|1635.48|3532.62|120.09|0.00|1094.94|2401.98|2522.07|3496.92|3617.01|766.50| +2450832|79920|2450879|88089|980039|5589|13892|44805|225792|942|42845|1|74|3|5|13846|242|874|28|37.87|98.08|79.44|521.92|2224.32|1060.36|2746.24|0.00|2001.88|137.20|222.44|222.44|359.64|359.64|-837.92| +2450832|79920|2450889|88089|980039|5589|13892|44805|225792|942|42845|1|34|4|3|9946|164|874|41|79.10|123.39|114.75|354.24|4704.75|3243.10|5058.99|282.28|0.00|1517.41|4704.75|4987.03|6222.16|6504.44|1461.65| +2450832|79920|2450846|88089|980039|5589|13892|44805|225792|942|42845|1|41|19|3|17078|99|874|73|93.50|141.18|96.00|3298.14|7008.00|6825.50|10306.14|140.16|0.00|2885.69|7008.00|7148.16|9893.69|10033.85|182.50| +2450832|70419|2450880|44526|1200071|4416|20714|44526|1200071|4416|20714|1|89|6|3|9622|104|875|91|88.20|226.67|102.00|11344.97|9282.00|8026.20|20626.97|649.74|0.00|9075.43|9282.00|9931.74|18357.43|19007.17|1255.80| +2450832|70419|2450847|44526|1200071|4416|20714|44526|1200071|4416|20714|1|1|9|5|12082|207|875|63|62.53|111.30|51.19|3786.93|3224.97|3939.39|7011.90|20.31|2934.72|2804.76|290.25|310.56|3095.01|3115.32|-3649.14| +2450832|70419|2450895|44526|1200071|4416|20714|44526|1200071|4416|20714|1|96|7|1|8468|269|875|74|38.95|51.41|1.02|3728.86|75.48|2882.30|3804.34|5.28|0.00|0.00|75.48|80.76|75.48|80.76|-2806.82| +2450832|70419|2450868|44526|1200071|4416|20714|44526|1200071|4416|20714|1|4|1|4|15536|77|875|77|41.85|60.26|49.41|835.45|3804.57|3222.45|4640.02|76.09|0.00|324.17|3804.57|3880.66|4128.74|4204.83|582.12| +2450832|70419|2450877|44526|1200071|4416|20714|44526|1200071|4416|20714|1|23|20|4|12308|124|875|93|29.86|49.86|28.91|1948.35|2688.63|2776.98|4636.98|48.39|1882.04|324.57|806.59|854.98|1131.16|1179.55|-1970.39| +2450832|32718|2450849|30221|1625222|4223|19624|30221|1625222|4223|19624|2|28|3|2|13028|35|876|36|2.11|5.29|1.79|126.00|64.44|75.96|190.44|5.79|0.00|85.68|64.44|70.23|150.12|155.91|-11.52| +2450832|32718|2450887|30221|1625222|4223|19624|30221|1625222|4223|19624|2|14|5|5|14665|285|876|32|54.18|100.77|1.00|3192.64|32.00|1733.76|3224.64|2.88|0.00|1483.20|32.00|34.88|1515.20|1518.08|-1701.76| +2450832|32718|2450841|30221|1625222|4223|19624|30221|1625222|4223|19624|2|12|4|4|13228|294|876|34|29.61|78.46|57.27|720.46|1947.18|1006.74|2667.64|175.24|0.00|0.00|1947.18|2122.42|1947.18|2122.42|940.44| +2450832|32718|2450878|30221|1625222|4223|19624|30221|1625222|4223|19624|2|10|6|2|16108|142|876|62|39.23|75.32|39.91|2195.42|2474.42|2432.26|4669.84|24.74|0.00|1354.08|2474.42|2499.16|3828.50|3853.24|42.16| +2450832|32718|2450893|30221|1625222|4223|19624|30221|1625222|4223|19624|2|99|4|4|14881|79|876|95|51.15|86.44|86.44|0.00|8211.80|4859.25|8211.80|328.47|0.00|410.40|8211.80|8540.27|8622.20|8950.67|3352.55| +2450832|64888|2450917|15804|662025|1046|45314|15804|662025|1046|45314|1|36|11|4|12842|118|877|91|52.39|80.15|53.70|2406.95|4886.70|4767.49|7293.65|0.00|1075.07|2698.15|3811.63|3811.63|6509.78|6509.78|-955.86| +2450832|64888|2450922|15804|662025|1046|45314|15804|662025|1046|45314|1|59|6|5|17548|27|877|81|94.60|238.39|150.18|7145.01|12164.58|7662.60|19309.59|729.87|0.00|5792.31|12164.58|12894.45|17956.89|18686.76|4501.98| +2450832|64888|2450868|15804|662025|1046|45314|15804|662025|1046|45314|1|51|5|5|17584|234|877|16|93.20|95.99|47.03|783.36|752.48|1491.20|1535.84|45.14|0.00|460.64|752.48|797.62|1213.12|1258.26|-738.72| +2450832|64888|2450860|15804|662025|1046|45314|15804|662025|1046|45314|1|31|20|5|15020|12|877|91|55.70|78.53|5.49|6646.64|499.59|5068.70|7146.23|4.99|0.00|2572.57|499.59|504.58|3072.16|3077.15|-4569.11| +2450832|64888|2450866|15804|662025|1046|45314|15804|662025|1046|45314|1|63|7|4|1394|25|877|5|34.69|38.85|24.47|71.90|122.35|173.45|194.25|8.56|0.00|44.65|122.35|130.91|167.00|175.56|-51.10| +2450832|64888|2450876|15804|662025|1046|45314|15804|662025|1046|45314|1|102|7|1|15493|39|877|63|79.23|92.69|90.83|117.18|5722.29|4991.49|5839.47|343.33|0.00|291.69|5722.29|6065.62|6013.98|6357.31|730.80| +2450832|64888|2450892|15804|662025|1046|45314|15804|662025|1046|45314|1|14|2|2|10957|187|877|17|59.66|119.91|15.58|1773.61|264.86|1014.22|2038.47|10.59|0.00|183.43|264.86|275.45|448.29|458.88|-749.36| +2450832|64888|2450886|15804|662025|1046|45314|15804|662025|1046|45314|1|108|14|5|7021|132|877|1|54.22|105.18|36.81|68.37|36.81|54.22|105.18|3.31|0.00|17.88|36.81|40.12|54.69|58.00|-17.41| +2450832|64888|2450899|15804|662025|1046|45314|15804|662025|1046|45314|1|36|2|2|4219|79|877|88|9.89|10.97|1.31|850.08|115.28|870.32|965.36|7.28|24.20|221.76|91.08|98.36|312.84|320.12|-779.24| +2450832|64888|2450907|15804|662025|1046|45314|15804|662025|1046|45314|1|6|7|2|16039|27|877|19|48.45|133.23|6.66|2404.83|126.54|920.55|2531.37|6.32|0.00|303.62|126.54|132.86|430.16|436.48|-794.01| +2450832|64888|2450852|15804|662025|1046|45314|15804|662025|1046|45314|1|76|15|1|4378|98|877|85|62.41|88.62|39.87|4143.75|3388.95|5304.85|7532.70|16.26|2982.27|2108.85|406.68|422.94|2515.53|2531.79|-4898.17| +2450832|70734|2450917|46405|108023|1641|36106|46405|108023|1641|36106|2|10|16|5|9662|178|878|9|90.38|226.85|22.68|1837.53|204.12|813.42|2041.65|6.12|0.00|796.23|204.12|210.24|1000.35|1006.47|-609.30| +2450832|70734|2450895|46405|108023|1641|36106|46405|108023|1641|36106|2|95|15|2|496|46|878|7|51.65|132.74|91.59|288.05|641.13|361.55|929.18|13.78|365.44|65.03|275.69|289.47|340.72|354.50|-85.86| +2450832|70734|2450904|46405|108023|1641|36106|46405|108023|1641|36106|2|77|5|3|229|198|878|91|47.74|82.11|34.48|4334.33|3137.68|4344.34|7472.01|0.00|1474.70|1419.60|1662.98|1662.98|3082.58|3082.58|-2681.36| +2450832|70734|2450896|46405|108023|1641|36106|46405|108023|1641|36106|2|13|16|5|9608|265|878|20|82.35|106.23|19.12|1742.20|382.40|1647.00|2124.60|3.82|0.00|616.00|382.40|386.22|998.40|1002.22|-1264.60| +2450832|71487|2450917|3296|1251132|568|2757|24720|158829|3465|13254|2|7|17|2|4393|24|879|86|9.22|11.15|5.24|508.26|450.64|792.92|958.90|36.05|0.00|287.24|450.64|486.69|737.88|773.93|-342.28| +2450832|71487|2450921|3296|1251132|568|2757|24720|158829|3465|13254|2|98|19|4|11305|236|879|67|15.41|22.80|22.11|46.23|1481.37|1032.47|1527.60|11.55|1288.79|442.87|192.58|204.13|635.45|647.00|-839.89| +2450832|71487|2450848|3296|1251132|568|2757|24720|158829|3465|13254|2|6|10|2|14884|198|879|61|85.18|169.50|35.59|8168.51|2170.99|5195.98|10339.50|21.70|0.00|4135.80|2170.99|2192.69|6306.79|6328.49|-3024.99| +2450832|71487|2450849|3296|1251132|568|2757|24720|158829|3465|13254|2|58|12|1|11726|196|879|70|10.17|20.84|18.13|189.70|1269.10|711.90|1458.80|25.38|0.00|203.70|1269.10|1294.48|1472.80|1498.18|557.20| +2450832|71487|2450837|3296|1251132|568|2757|24720|158829|3465|13254|2|11|11|4|9458|62|879|54|66.62|112.58|32.64|4316.76|1762.56|3597.48|6079.32|88.12|0.00|1215.54|1762.56|1850.68|2978.10|3066.22|-1834.92| +2450832|71487|2450842|3296|1251132|568|2757|24720|158829|3465|13254|2|67|9|1|15230|230|879|41|88.27|94.44|62.33|1316.51|2555.53|3619.07|3872.04|229.99|0.00|38.54|2555.53|2785.52|2594.07|2824.06|-1063.54| +2450832|71487|2450911|3296|1251132|568|2757|24720|158829|3465|13254|2|84|2|2|15220|13|879|51|55.21|60.17|56.55|184.62|2884.05|2815.71|3068.67|7.49|2134.19|552.33|749.86|757.35|1302.19|1309.68|-2065.85| +2450832|71487|2450846|3296|1251132|568|2757|24720|158829|3465|13254|2|88|12|1|13210|223|879|17|90.92|189.11|122.92|1125.23|2089.64|1545.64|3214.87|65.19|1003.02|1253.75|1086.62|1151.81|2340.37|2405.56|-459.02| +2450832|71487|2450846|3296|1251132|568|2757|24720|158829|3465|13254|2|105|7|3|8227|43|879|55|95.78|224.12|224.12|0.00|12326.60|5267.90|12326.60|613.86|2095.52|2218.70|10231.08|10844.94|12449.78|13063.64|4963.18| +2450832|71487|2450883|3296|1251132|568|2757|24720|158829|3465|13254|2|40|3|4|11506|243|879|83|81.45|162.90|50.49|9330.03|4190.67|6760.35|13520.70|125.72|0.00|2298.27|4190.67|4316.39|6488.94|6614.66|-2569.68| +2450832|71487|2450893|3296|1251132|568|2757|24720|158829|3465|13254|2|44|1|3|15121|214|879|84|89.73|215.35|191.66|1989.96|16099.44|7537.32|18089.40|804.97|0.00|2713.20|16099.44|16904.41|18812.64|19617.61|8562.12| +2450832|71487|2450879|3296|1251132|568|2757|24720|158829|3465|13254|2|85|19|1|12997|10|879|27|20.94|51.30|3.07|1302.21|82.89|565.38|1385.10|0.11|71.28|82.89|11.61|11.72|94.50|94.61|-553.77| +2450832|71487|2450876|3296|1251132|568|2757|24720|158829|3465|13254|2|76|14|5|8551|27|879|82|29.56|66.80|0.00|5477.60|0.00|2423.92|5477.60|0.00|0.00|601.88|0.00|0.00|601.88|601.88|-2423.92| +2450832|71487|2450860|3296|1251132|568|2757|24720|158829|3465|13254|2|66|15|1|2246|1|879|42|15.41|31.89|21.68|428.82|910.56|647.22|1339.38|63.73|0.00|589.26|910.56|974.29|1499.82|1563.55|263.34| +2450832|34615|2450836|13677|1178248|6372|37743|13677|1178248|6372|37743|1|70|17|5|5317|56|880|92|41.11|41.93|12.15|2739.76|1117.80|3782.12|3857.56|67.06|0.00|1812.40|1117.80|1184.86|2930.20|2997.26|-2664.32| +2450832|34615|2450881|13677|1178248|6372|37743|13677|1178248|6372|37743|1|101|19|5|16840|174|880|71|69.86|163.47|22.88|9981.89|1624.48|4960.06|11606.37|48.73|0.00|927.97|1624.48|1673.21|2552.45|2601.18|-3335.58| +2450832|34615|2450835|13677|1178248|6372|37743|13677|1178248|6372|37743|1|55|11|1|658|191|880|67|94.58|274.28|189.25|5697.01|12679.75|6336.86|18376.76|253.59|0.00|1102.15|12679.75|12933.34|13781.90|14035.49|6342.89| +2450832|34615|2450895|13677|1178248|6372|37743|13677|1178248|6372|37743|1|65|6|5|6376|258|880|38|33.88|70.80|33.27|1426.14|1264.26|1287.44|2690.40|37.92|0.00|618.64|1264.26|1302.18|1882.90|1920.82|-23.18| +2450832|34615|2450903|13677|1178248|6372|37743|13677|1178248|6372|37743|1|5|9|5|9116|253|880|7|45.81|122.77|34.37|618.80|240.59|320.67|859.39|0.00|180.44|60.13|60.15|60.15|120.28|120.28|-260.52| +2450832|34615|2450876|13677|1178248|6372|37743|13677|1178248|6372|37743|1|18|18|2|1603|12|880|11|96.45|189.04|94.52|1039.72|1039.72|1060.95|2079.44|31.19|0.00|603.02|1039.72|1070.91|1642.74|1673.93|-21.23| +2450832|34615|2450868|13677|1178248|6372|37743|13677|1178248|6372|37743|1|93|6|4|2834|264|880|64|40.55|109.07|81.80|1745.28|5235.20|2595.20|6980.48|418.81|0.00|3141.12|5235.20|5654.01|8376.32|8795.13|2640.00| +2450832|34615|2450845|13677|1178248|6372|37743|13677|1178248|6372|37743|1|39|12|1|6898|244|880|57|58.60|77.35|40.22|2116.41|2292.54|3340.20|4408.95|183.40|0.00|1498.53|2292.54|2475.94|3791.07|3974.47|-1047.66| +2450832|34615|2450845|13677|1178248|6372|37743|13677|1178248|6372|37743|1|94|18|5|9937|293|880|50|24.52|61.30|2.45|2942.50|122.50|1226.00|3065.00|8.57|0.00|766.00|122.50|131.07|888.50|897.07|-1103.50| +2450832|34615|2450871|13677|1178248|6372|37743|13677|1178248|6372|37743|1|28|14|4|2962|23|880|25|23.68|44.99|36.44|213.75|911.00|592.00|1124.75|63.77|0.00|551.00|911.00|974.77|1462.00|1525.77|319.00| +2450832|34615|2450841|13677|1178248|6372|37743|13677|1178248|6372|37743|1|14|5|1|9931|175|880|29|85.35|228.73|139.52|2587.09|4046.08|2475.15|6633.17|121.38|0.00|2321.45|4046.08|4167.46|6367.53|6488.91|1570.93| +2450832|34615|2450844|13677|1178248|6372|37743|13677|1178248|6372|37743|1|14|14|5|3458|120|880|62|47.22|59.96|28.78|1933.16|1784.36|2927.64|3717.52|160.59|0.00|1189.16|1784.36|1944.95|2973.52|3134.11|-1143.28| +2450832|34615|2450867|13677|1178248|6372|37743|13677|1178248|6372|37743|1|56|15|5|5365|222|880|82|60.51|171.84|5.15|13668.58|422.30|4961.82|14090.88|4.22|0.00|5776.90|422.30|426.52|6199.20|6203.42|-4539.52| +2450832|63230|2450861|42130|1723917|1351|1938|42130|1723917|1351|1938|1|64|7|5|6820|194|881|49|43.84|81.98|14.75|3294.27|722.75|2148.16|4017.02|14.45|0.00|1044.19|722.75|737.20|1766.94|1781.39|-1425.41| +2450832|63230|2450863|42130|1723917|1351|1938|42130|1723917|1351|1938|1|71|12|4|13279|78|881|54|85.83|226.59|54.38|9299.34|2936.52|4634.82|12235.86|146.82|0.00|5628.42|2936.52|3083.34|8564.94|8711.76|-1698.30| +2450832|63230|2450878|42130|1723917|1351|1938|42130|1723917|1351|1938|1|102|16|3|3997|94|881|3|62.86|69.77|36.28|100.47|108.84|188.58|209.31|7.61|0.00|0.00|108.84|116.45|108.84|116.45|-79.74| +2450832|63230|2450867|42130|1723917|1351|1938|42130|1723917|1351|1938|1|39|15|2|16664|115|881|12|98.82|127.47|67.55|719.04|810.60|1185.84|1529.64|24.31|0.00|244.68|810.60|834.91|1055.28|1079.59|-375.24| +2450832|63230|2450916|42130|1723917|1351|1938|42130|1723917|1351|1938|1|26|7|3|17953|223|881|66|9.41|25.50|18.36|471.24|1211.76|621.06|1683.00|48.47|0.00|774.18|1211.76|1260.23|1985.94|2034.41|590.70| +2450832|63230|2450837|42130|1723917|1351|1938|42130|1723917|1351|1938|1|32|1|5|16478|84|881|15|85.01|216.77|62.86|2308.65|942.90|1275.15|3251.55|65.62|122.57|422.70|820.33|885.95|1243.03|1308.65|-454.82| +2450832|63230|2450907|42130|1723917|1351|1938|42130|1723917|1351|1938|1|4|20|2|9661|106|881|73|59.27|73.49|14.69|4292.40|1072.37|4326.71|5364.77|32.17|0.00|2574.71|1072.37|1104.54|3647.08|3679.25|-3254.34| +2450832|63230|2450873|42130|1723917|1351|1938|42130|1723917|1351|1938|1|75|16|2|1849|152|881|64|46.93|65.23|24.13|2630.40|1544.32|3003.52|4174.72|92.65|0.00|417.28|1544.32|1636.97|1961.60|2054.25|-1459.20| +2450832|63230|2450909|42130|1723917|1351|1938|42130|1723917|1351|1938|1|57|3|2|2641|50|881|15|72.69|125.75|21.37|1565.70|320.55|1090.35|1886.25|6.41|0.00|811.05|320.55|326.96|1131.60|1138.01|-769.80| +2450832|63230|2450877|42130|1723917|1351|1938|42130|1723917|1351|1938|1|65|17|2|2636|264|881|98|1.76|2.18|1.61|55.86|157.78|172.48|213.64|0.55|102.55|69.58|55.23|55.78|124.81|125.36|-117.25| +2450832|63230|2450893|42130|1723917|1351|1938|42130|1723917|1351|1938|1|30|16|2|1993|114|881|8|20.91|50.18|24.58|204.80|196.64|167.28|401.44|1.96|0.00|84.24|196.64|198.60|280.88|282.84|29.36| +2450832|63230|2450834|42130|1723917|1351|1938|42130|1723917|1351|1938|1|39|10|3|2665|274|881|21|24.10|53.50|48.15|112.35|1011.15|506.10|1123.50|30.33|0.00|393.12|1011.15|1041.48|1404.27|1434.60|505.05| +2450832|63230|2450887|42130|1723917|1351|1938|42130|1723917|1351|1938|1|73|10|3|17312|140|881|85|73.25|104.74|46.08|4986.10|3916.80|6226.25|8902.90|39.16|0.00|1958.40|3916.80|3955.96|5875.20|5914.36|-2309.45| +2450832|76913|2450840|8072|985247|3222|11997|8072|985247|3222|11997|4|105|20|5|3754|193|882|30|53.30|102.86|46.28|1697.40|1388.40|1599.00|3085.80|111.07|0.00|894.60|1388.40|1499.47|2283.00|2394.07|-210.60| +2450832|76913|2450909|8072|985247|3222|11997|8072|985247|3222|11997|4|90|11|5|14785|206|882|84|48.98|129.30|34.91|7928.76|2932.44|4114.32|10861.20|117.29|0.00|3692.64|2932.44|3049.73|6625.08|6742.37|-1181.88| +2450832|76913|2450897|8072|985247|3222|11997|8072|985247|3222|11997|4|18|4|3|13582|1|882|60|31.33|84.59|61.75|1370.40|3705.00|1879.80|5075.40|222.30|0.00|2334.60|3705.00|3927.30|6039.60|6261.90|1825.20| +2450832|76913|2450887|8072|985247|3222|11997|8072|985247|3222|11997|4|34|4|1|8390|162|882|67|45.35|69.38|0.69|4602.23|46.23|3038.45|4648.46|0.92|0.00|557.44|46.23|47.15|603.67|604.59|-2992.22| +2450832|76913|2450839|8072|985247|3222|11997|8072|985247|3222|11997|4|26|13|5|11168|124|882|5|70.69|98.25|95.30|14.75|476.50|353.45|491.25|6.48|395.49|171.90|81.01|87.49|252.91|259.39|-272.44| +2450832|76913|2450864|8072|985247|3222|11997|8072|985247|3222|11997|4|59|18|3|11368|181|882|67|77.50|213.12|210.98|143.38|14135.66|5192.50|14279.04|141.35|0.00|1998.61|14135.66|14277.01|16134.27|16275.62|8943.16| +2450832|76913|||985247|3222|11997|8072|||||79|10||6278|12|882|57|31.25|||2028.06||1781.25|2938.92||464.53|1234.05|||1680.38|1689.30|-1334.92| +2450832|76913|2450902|8072|985247|3222|11997|8072|985247|3222|11997|4|35|12|4|9193|223|882|43|62.04|102.98|38.10|2789.84|1638.30|2667.72|4428.14|32.76|0.00|2125.49|1638.30|1671.06|3763.79|3796.55|-1029.42| +2450832|74799|2450850|68864|1811066|4468|7070|19419|1165474|2876|24678|1|63|9|5|548|265|883|5|95.20|261.80|112.57|746.15|562.85|476.00|1309.00|22.51|0.00|235.60|562.85|585.36|798.45|820.96|86.85| +2450832|74799|2450849|68864|1811066|4468|7070|19419|1165474|2876|24678|1|60|1|1|2704|61|883|67|51.17|68.56|32.22|2434.78|2158.74|3428.39|4593.52|151.11|0.00|505.18|2158.74|2309.85|2663.92|2815.03|-1269.65| +2450832|74799|2450835|68864|1811066|4468|7070|19419|1165474|2876|24678|1|52|10|4|6964|185|883|61|72.05|149.14|14.91|8188.03|909.51|4395.05|9097.54|63.66|0.00|636.23|909.51|973.17|1545.74|1609.40|-3485.54| +2450832|74799|2450900|68864|1811066|4468|7070|19419|1165474|2876|24678|1|94|18|4|2275|161|883|78|19.09|47.34|11.83|2769.78|922.74|1489.02|3692.52|46.13|0.00|737.88|922.74|968.87|1660.62|1706.75|-566.28| +2450832|74799|2450856|68864|1811066|4468|7070|19419|1165474|2876|24678|1|32|6|4|2990|80|883|64|28.96|74.13|48.18|1660.80|3083.52|1853.44|4744.32|277.51|0.00|2134.40|3083.52|3361.03|5217.92|5495.43|1230.08| +2450832|74799|2450835|68864|1811066|4468|7070|19419|1165474|2876|24678|1|89|11|4|2060|288|883|62|2.51|4.96|2.52|151.28|156.24|155.62|307.52|12.49|0.00|21.08|156.24|168.73|177.32|189.81|0.62| +2450832|74799|2450861|68864|1811066|4468|7070|19419|1165474|2876|24678|1|11|19|2|10339|96|883|84|56.36|105.39|89.58|1328.04|7524.72|4734.24|8852.76|279.16|3536.61|2389.80|3988.11|4267.27|6377.91|6657.07|-746.13| +2450832|74799|2450861|68864|1811066|4468|7070|19419|1165474|2876|24678|1|66|1|1|15886|41|883|38|35.89|46.65|42.45|159.60|1613.10|1363.82|1772.70|96.78|0.00|478.42|1613.10|1709.88|2091.52|2188.30|249.28| +2450832|74799|2450892|68864|1811066|4468|7070|19419|1165474|2876|24678|1|12|8|2|11188|92|883|11|58.70|64.57|14.85|546.92|163.35|645.70|710.27|3.88|107.81|319.55|55.54|59.42|375.09|378.97|-590.16| +2450832|40870|2450913|20337|207201|6065|16129|20337|207201|6065|16129|1|46|20|3|3952|241|884|87|30.57|67.25|4.70|5441.85|408.90|2659.59|5850.75|0.00|314.85|642.93|94.05|94.05|736.98|736.98|-2565.54| +2450832|40870|2450848|20337||||20337|207201|6065|16129|1||10|1|9868|225|884|||214.60|193.14||6953.04||7725.60|35.46|||3546.06|3581.52|5013.78|5049.24|776.94| +2450832|40870|2450883|20337|207201|6065|16129|20337|207201|6065|16129|1|102|6|2|10810|3|884|8|49.94|114.36|26.30|704.48|210.40|399.52|914.88|0.00|0.00|210.40|210.40|210.40|420.80|420.80|-189.12| +2450832|40870|2450852|20337|207201|6065|16129|20337|207201|6065|16129|1|20|16|5|15577|96|884|23|12.76|18.62|14.70|90.16|338.10|293.48|428.26|3.38|0.00|4.14|338.10|341.48|342.24|345.62|44.62| +2450832|40870|2450860|20337|207201|6065|16129|20337|207201|6065|16129|1|1|15|5|2167|22|884|17|88.58|144.38|41.87|1742.67|711.79|1505.86|2454.46|49.82|0.00|24.48|711.79|761.61|736.27|786.09|-794.07| +2450832|40870|2450865|20337|207201|6065|16129|20337|207201|6065|16129|1|46|15|4|12115|216|884|22|47.88|64.15|5.77|1284.36|126.94|1053.36|1411.30|10.15|0.00|409.20|126.94|137.09|536.14|546.29|-926.42| +2450832|40870|2450875|20337|207201|6065|16129|20337|207201|6065|16129|1|93|13|5|14440|121|884|61|32.10|39.48|18.16|1300.52|1107.76|1958.10|2408.28|11.07|0.00|168.36|1107.76|1118.83|1276.12|1287.19|-850.34| +2450832|40870|2450846|20337|207201|6065|16129|20337|207201|6065|16129|1|16|10|2|12937|88|884|9|41.82|50.60|13.66|332.46|122.94|376.38|455.40|9.83|0.00|22.77|122.94|132.77|145.71|155.54|-253.44| +2450832|40870|2450900|20337|207201|6065|16129|20337|207201|6065|16129|1|53|5|2|394|73|884|45|2.21|5.06|2.07|134.55|93.15|99.45|227.70|1.86|0.00|106.65|93.15|95.01|199.80|201.66|-6.30| +2450832|40870|2450919|20337|207201|6065|16129|20337|207201|6065|16129|1|35|1|4|2882|220|884|51|18.80|47.00|26.32|1054.68|1342.32|958.80|2397.00|67.11|0.00|71.91|1342.32|1409.43|1414.23|1481.34|383.52| +2450832|40870|2450901|20337|207201|6065|16129|20337|207201|6065|16129|1|48|15|1|9373|83|884|75|69.89|126.50|29.09|7305.75|2181.75|5241.75|9487.50|109.08|0.00|1992.00|2181.75|2290.83|4173.75|4282.83|-3060.00| +2450832|37608|2450901|4610|1569141|4072|5803|4610|1569141|4072|5803|1|43|16|3|6649|74|885|1|32.06|70.53|36.67|33.86|36.67|32.06|70.53|0.14|33.73|2.82|2.94|3.08|5.76|5.90|-29.12| +2450832|37608|2450912|4610|1569141|4072|5803|4610|1569141|4072|5803|1|79|1|2|1952|46|885|13|55.35|83.57|72.70|141.31|945.10|719.55|1086.41|37.80|0.00|488.80|945.10|982.90|1433.90|1471.70|225.55| +2450832|37608|2450914|4610|1569141|4072|5803|4610|1569141|4072|5803|1|64|2|3|14360|35|885|92|13.76|17.33|16.81|47.84|1546.52|1265.92|1594.36|15.77|1283.61|685.40|262.91|278.68|948.31|964.08|-1003.01| +2450832|37608|2450893|4610|1569141|4072|5803|4610|1569141|4072|5803|1|105|13|1|11311|75|885|41|17.42|40.06|32.04|328.82|1313.64|714.22|1642.46|78.81|0.00|98.40|1313.64|1392.45|1412.04|1490.85|599.42| +2450832|37608|2450917|4610|1569141|4072|5803|4610|1569141|4072|5803|1|104|8|3|12950|98|885|100|77.23|108.89|29.40|7949.00|2940.00|7723.00|10889.00|117.60|0.00|1197.00|2940.00|3057.60|4137.00|4254.60|-4783.00| +2450832|51535|2450860|62467|670952|6284|42416|62467|670952|6284|42416|1|93|16|4|2299|227|886|2|37.17|91.43|21.94|138.98|43.88|74.34|182.86|1.75|0.00|56.68|43.88|45.63|100.56|102.31|-30.46| +2450832|51535|2450841|62467|670952|6284|42416|62467|670952|6284|42416|1|30|1|4|16634|270|886|71|22.69|42.20|27.43|1048.67|1947.53|1610.99|2996.20|38.95|0.00|1138.13|1947.53|1986.48|3085.66|3124.61|336.54| +2450832|51535|2450862|62467|670952|6284|42416|62467|670952|6284|42416|1|90|1|3|1999|179|886|19|89.50|240.75|185.37|1052.22|3522.03|1700.50|4574.25|246.54|0.00|365.94|3522.03|3768.57|3887.97|4134.51|1821.53| +2450832|51535|2450840|62467|670952|6284|42416|62467|670952|6284|42416|1|97|15|3|12889|267|886|45|41.00|120.54|96.43|1084.95|4339.35|1845.00|5424.30|86.78|0.00|2386.35|4339.35|4426.13|6725.70|6812.48|2494.35| +2450832|51535|2450902|62467|670952|6284|42416|62467|670952|6284|42416|1|73|18|2|1702|239|886|57|31.16|58.26|57.09|66.69|3254.13|1776.12|3320.82|57.59|1334.19|929.67|1919.94|1977.53|2849.61|2907.20|143.82| +2450832|51535|2450910|62467|670952|6284|42416|62467|670952|6284|42416|1|90|4|2|15422|208|886|48|98.57|209.95|155.36|2620.32|7457.28|4731.36|10077.60|393.74|894.87|2720.64|6562.41|6956.15|9283.05|9676.79|1831.05| +2450832|75314|2450836|2186|863195|2604|21213|2186|863195|2604|21213|4|53|10|3|13226|134|887|16|99.90|151.84|21.25|2089.44|340.00|1598.40|2429.44|23.80|0.00|97.12|340.00|363.80|437.12|460.92|-1258.40| +2450832|75314|2450880|2186|863195|2604|21213|2186|863195|2604|21213|4|20|20|3|12847|237|887|48|22.73|55.68|13.36|2031.36|641.28|1091.04|2672.64|6.41|0.00|1309.44|641.28|647.69|1950.72|1957.13|-449.76| +2450832|75314|2450850|2186|863195|2604|21213|2186|863195|2604|21213|4|21|10|4|6439|187|887|94|47.09|74.40|48.36|2447.76|4545.84|4426.46|6993.60|0.00|0.00|3076.62|4545.84|4545.84|7622.46|7622.46|119.38| +2450832|75314|2450905|2186|863195|2604|21213|2186|863195|2604|21213|4|74|6|4|15562|157|887|66|22.32|52.00|32.76|1269.84|2162.16|1473.12|3432.00|172.97|0.00|823.68|2162.16|2335.13|2985.84|3158.81|689.04| +2450832|75314|2450883|2186|863195|2604|21213|2186|863195|2604|21213|4|16|4|4|13300|151|887|77|4.76|5.85|3.39|189.42|261.03|366.52|450.45|20.88|0.00|139.37|261.03|281.91|400.40|421.28|-105.49| +2450832|75314|2450841|2186|863195|2604|21213|2186|863195|2604|21213|4|78|16|4|5023|122|887|49|67.36|96.32|29.85|3257.03|1462.65|3300.64|4719.68|106.62|277.90|2123.66|1184.75|1291.37|3308.41|3415.03|-2115.89| +2450832|75314|2450837|2186|863195|2604|21213|2186|863195|2604|21213|4|89|2|4|6991|106|887|68|49.97|58.96|13.56|3087.20|922.08|3397.96|4009.28|14.38|442.59|1723.80|479.49|493.87|2203.29|2217.67|-2918.47| +2450832|75314|2450838|2186|863195|2604|21213|2186|863195|2604|21213|4|59|10|3|16405|69|887|30|99.95|133.93|103.12|924.30|3093.60|2998.50|4017.90|247.48|0.00|120.30|3093.60|3341.08|3213.90|3461.38|95.10| +2450832|75314|2450919|2186|863195|2604|21213|2186|863195|2604|21213|4|89|14|4|2755|65|887|11|41.28|108.15|4.32|1142.13|47.52|454.08|1189.65|0.11|41.81|594.77|5.71|5.82|600.48|600.59|-448.37| +2450832|76595|2450889|65912|1868603|2047|338|65912|1868603|2047|338|4|21|9|4|6025|40|888|19|11.10|27.30|17.74|181.64|337.06|210.90|518.70|6.74|0.00|103.74|337.06|343.80|440.80|447.54|126.16| +2450832|76595|2450870|65912|1868603|2047|338|65912|1868603|2047|338|4|16|3|5|550|192|888|60|44.61|132.93|50.51|4945.20|3030.60|2676.60|7975.80|0.00|0.00|1435.20|3030.60|3030.60|4465.80|4465.80|354.00| +|||65912||||||2047||4||4|1|12806|99|888||||5.75|656.28|34.50||||0.00|186.48||35.19|220.98|221.67|-481.02| +2450832|76595|2450862|65912|1868603|2047|338|65912|1868603|2047|338|4|74|8|3|4376|141|888|26|94.81|182.03|176.56|142.22|4590.56|2465.06|4732.78|275.43|0.00|1561.56|4590.56|4865.99|6152.12|6427.55|2125.50| +2450832|76595|2450871|65912|1868603|2047|338|65912|1868603|2047|338|4|52|3|5|15512|86|888|20|77.79|232.59|125.59|2140.00|2511.80|1555.80|4651.80|175.82|0.00|744.20|2511.80|2687.62|3256.00|3431.82|956.00| +2450832|76595|2450891|65912|1868603|2047|338|65912|1868603|2047|338|4|85|10|5|1142|22|888|64|93.07|189.86|113.91|4860.80|7290.24|5956.48|12151.04|437.41|0.00|4738.56|7290.24|7727.65|12028.80|12466.21|1333.76| +2450832|39399|2450862|56945|1249960|6175|10790|56945|1249960|6175|10790|1|69|19|2|8224|64|889|35|58.81|76.45|70.33|214.20|2461.55|2058.35|2675.75|123.07|0.00|1123.50|2461.55|2584.62|3585.05|3708.12|403.20| +2450832|39399|2450904|56945|1249960|6175|10790|56945|1249960|6175|10790|1|27|8|4|4111|239|889|9|1.14|1.64|0.21|12.87|1.89|10.26|14.76|0.11|0.00|3.06|1.89|2.00|4.95|5.06|-8.37| +2450832|39399|2450848|56945|1249960|6175|10790|56945|1249960|6175|10790|1|33|15|3|16456|77|889|30|1.62|3.02|0.90|63.60|27.00|48.60|90.60|0.27|0.00|8.10|27.00|27.27|35.10|35.37|-21.60| +2450832|39399|2450869|56945|1249960|6175|10790|56945|1249960|6175|10790|1|10|3|3|13150|34|889|4|73.87|158.82|30.17|514.60|120.68|295.48|635.28|8.44|0.00|165.16|120.68|129.12|285.84|294.28|-174.80| +2450832|39399|2450901|56945|1249960|6175|10790|56945|1249960|6175|10790|1|4|5|1|6784|129|889|77|43.89|100.50|13.06|6732.88|1005.62|3379.53|7738.50|0.00|0.00|77.00|1005.62|1005.62|1082.62|1082.62|-2373.91| +2450832|39399|2450877|56945|1249960|6175|10790|56945|1249960|6175|10790|1|39|16|2|2530|38|889|54|30.39|88.73|57.67|1677.24|3114.18|1641.06|4791.42|93.42|0.00|2347.38|3114.18|3207.60|5461.56|5554.98|1473.12| +2450832|39399|2450834|56945|1249960|6175|10790|56945|1249960|6175|10790|1|99|11|4|16994|35|889|99|86.67|132.60|125.97|656.37|12471.03|8580.33|13127.40|623.55|0.00|5643.99|12471.03|13094.58|18115.02|18738.57|3890.70| +2450832|39399|2450857|56945|1249960|6175|10790|56945|1249960|6175|10790|1|21|19|1|9994|253|889|85|46.18|120.99|2.41|10079.30|204.85|3925.30|10284.15|1.47|180.26|719.10|24.59|26.06|743.69|745.16|-3900.71| +2450832|39399|2450905|56945|1249960|6175|10790|56945|1249960|6175|10790|1|28|15|2|15877|253|889|94|85.06|132.69|132.69|0.00|12472.86|7995.64|12472.86|498.91|0.00|4863.56|12472.86|12971.77|17336.42|17835.33|4477.22| +2450832|39399|2450863|56945|1249960|6175|10790|56945|1249960|6175|10790|1|102|17|4|7412|70|889|40|47.58|128.46|43.67|3391.60|1746.80|1903.20|5138.40|52.40|0.00|1181.60|1746.80|1799.20|2928.40|2980.80|-156.40| +2450832|39399|2450863|56945|1249960|6175|10790|56945|1249960|6175|10790|1|100|11|2|17276|204|889|83|2.45|6.37|0.50|487.21|41.50|203.35|528.71|0.83|0.00|263.94|41.50|42.33|305.44|306.27|-161.85| +2450832|75809|2450859|78942|1799568|6947|16326|835|770331|4110|38289|1|26|16|2|9334|125|890|96|4.14|8.40|6.21|210.24|596.16|397.44|806.40|5.96|0.00|322.56|596.16|602.12|918.72|924.68|198.72| +2450832|75809|2450920|78942|1799568|6947|16326|835|770331|4110|38289|1|17|2|3|1684|112|890|77|21.79|25.27|25.01|20.02|1925.77|1677.83|1945.79|89.35|808.82|777.70|1116.95|1206.30|1894.65|1984.00|-560.88| +2450832|75809|2450842|78942|1799568|6947|16326|835|770331|4110|38289|1|5|13|5|13222|22|890|85|89.78|125.69|110.60|1282.65|9401.00|7631.30|10683.65|470.05|0.00|854.25|9401.00|9871.05|10255.25|10725.30|1769.70| +2450832|75809|2450890|78942|1799568|6947|16326|835|770331|4110|38289|1|18|10|1|17212|273|890|22|5.04|9.32|6.05|71.94|133.10|110.88|205.04|9.31|0.00|47.08|133.10|142.41|180.18|189.49|22.22| +2450832|75809|2450873|78942|1799568|6947|16326|835|770331|4110|38289|1|12|7|5|12238|148|890|4|76.75|88.26|28.24|240.08|112.96|307.00|353.04|5.64|0.00|14.12|112.96|118.60|127.08|132.72|-194.04| +2450832|75809|2450867|78942|1799568|6947|16326|835|770331|4110|38289|1|24|3|4|10274|11|890|94|38.39|72.94|51.78|1989.04|4867.32|3608.66|6856.36|243.36|0.00|1507.76|4867.32|5110.68|6375.08|6618.44|1258.66| +2450832|75809|2450921|78942|1799568|6947|16326|835|770331|4110|38289|1|16|6|3|7102|135|890|66|52.32|85.80|6.86|5210.04|452.76|3453.12|5662.80|4.52|0.00|2265.12|452.76|457.28|2717.88|2722.40|-3000.36| +2450832|75809|2450896|78942|1799568|6947|16326|835|770331|4110|38289|1|105|8|4|9164|11|890|59|4.49|11.40|7.63|222.43|450.17|264.91|672.60|13.10|13.50|208.27|436.67|449.77|644.94|658.04|171.76| +2450832|75809|2450838|78942|1799568|6947|16326|835|770331|4110|38289|1|63|3|2|7526|113|890|33|6.92|7.12|0.92|204.60|30.36|228.36|234.96|2.42|0.00|74.91|30.36|32.78|105.27|107.69|-198.00| +2450832|75809|2450905|78942|1799568|6947|16326|835|770331|4110|38289|1|73|10|3|3098|74|890|17|29.06|43.00|42.14|14.62|716.38|494.02|731.00|5.22|193.42|36.55|522.96|528.18|559.51|564.73|28.94| +2450832|75809|2450898|78942|1799568|6947|16326|835|770331|4110|38289|1|29|18|4|1765|163|890|57|99.62|204.22|57.18|8381.28|3259.26|5678.34|11640.54|89.95|260.74|1512.78|2998.52|3088.47|4511.30|4601.25|-2679.82| +2450832|75809|2450919|78942|1799568|6947|16326|835|770331|4110|38289|1|86|16|5|1321|258|890|50|89.94|172.68|155.41|863.50|7770.50|4497.00|8634.00|0.00|0.00|3539.50|7770.50|7770.50|11310.00|11310.00|3273.50| +2450832|27073|2450885|85188|1197707|5073|43192|85188|1197707|5073|43192|4|37|6|2|14257|249|891|41|3.49|5.30|0.74|186.96|30.34|143.09|217.30|1.05|12.74|10.66|17.60|18.65|28.26|29.31|-125.49| +2450832|27073|2450867|85188|1197707|5073|43192|85188|1197707|5073|43192|4|58|3|1|11306|215|891|27|98.62|271.20|78.64|5199.12|2123.28|2662.74|7322.40|1.69|2102.04|0.00|21.24|22.93|21.24|22.93|-2641.50| +2450832|27073|2450893|85188|1197707|5073|43192|85188|1197707|5073|43192|4|40|12|2|7984|199|891|91|95.42|164.12|100.11|5824.91|9110.01|8683.22|14934.92|637.70|0.00|7018.83|9110.01|9747.71|16128.84|16766.54|426.79| +2450832|27073|2450851|85188|1197707|5073|43192|85188|1197707|5073|43192|4|34|11|1|14120|281|891|15|45.30|84.71|69.46|228.75|1041.90|679.50|1270.65|10.41|0.00|444.60|1041.90|1052.31|1486.50|1496.91|362.40| +2450832|27073|2450877|85188|1197707|5073|43192|85188|1197707|5073|43192|4|32|19|2|2816|5|891|42|40.87|52.31|28.24|1010.94|1186.08|1716.54|2197.02|71.16|0.00|614.88|1186.08|1257.24|1800.96|1872.12|-530.46| +2450832|27073|2450840|85188|1197707|5073|43192|85188|1197707|5073|43192|4|97|3|4|6916|29|891|47|29.81|72.14|50.49|1017.55|2373.03|1401.07|3390.58|118.65|0.00|305.03|2373.03|2491.68|2678.06|2796.71|971.96| +2450832|27073|2450870|85188|1197707|5073|43192|85188|1197707|5073|43192|4|19|7|3|4681|152|891|74|6.91|8.01|7.28|54.02|538.72|511.34|592.74|4.52|474.07|53.28|64.65|69.17|117.93|122.45|-446.69| +2450832|27073|2450906|85188|1197707|5073|43192|85188|1197707|5073|43192|4|35|18|1|2384|127|891|87|36.55|84.79|2.54|7155.75|220.98|3179.85|7376.73|11.04|0.00|2212.41|220.98|232.02|2433.39|2444.43|-2958.87| +2450832|27073|2450875|85188|1197707|5073|43192|85188|1197707|5073|43192|4|58|4|2|16054|165|891|59|78.97|157.15|105.29|3059.74|6212.11|4659.23|9271.85|186.36|0.00|1947.00|6212.11|6398.47|8159.11|8345.47|1552.88| +2450832|27073|2450874|85188|1197707|5073|43192|85188|1197707|5073|43192|4|35|17|2|3734|290|891|48|2.61|2.89|1.96|44.64|94.08|125.28|138.72|8.46|0.00|20.64|94.08|102.54|114.72|123.18|-31.20| +2450832|27073|2450883|85188|1197707|5073|43192|85188|1197707|5073|43192|4|68|13|5|2032|267|891|79|86.61|147.23|17.66|10236.03|1395.14|6842.19|11631.17|97.65|0.00|5349.88|1395.14|1492.79|6745.02|6842.67|-5447.05| +2450832|27073|2450889|85188|1197707|5073|43192|85188|1197707|5073|43192|4|14|7|2|892|29|891|15|1.74|4.97|3.08|28.35|46.20|26.10|74.55|0.00|0.00|17.10|46.20|46.20|63.30|63.30|20.10| +2450832|45362|2450853|39445|691646|1000|7601|39445|691646|1000|7601|2|72|16|4|13190|137|892|79|40.29|87.83|0.87|6869.84|68.73|3182.91|6938.57|3.43|0.00|2497.19|68.73|72.16|2565.92|2569.35|-3114.18| +2450832|45362|2450873|39445|691646|1000|7601|39445|691646|1000|7601|2|15|18|4|5684|11|892|7|1.16|1.75|0.15|11.20|1.05|8.12|12.25|0.07|0.00|1.19|1.05|1.12|2.24|2.31|-7.07| +2450832|45362|2450920|39445|691646|1000|7601|39445|691646|1000|7601|2|53|5|2|17872|244|892|44|92.29|97.82|44.99|2324.52|1979.56|4060.76|4304.08|0.00|0.00|1118.92|1979.56|1979.56|3098.48|3098.48|-2081.20| +2450832|45362|2450896|39445|691646|1000|7601|39445|691646|1000|7601|2|18|3|2|5116|83|892|81|28.28|69.28|20.78|3928.50|1683.18|2290.68|5611.68|117.82|0.00|617.22|1683.18|1801.00|2300.40|2418.22|-607.50| +2450832|45362|2450894|39445|691646|1000|7601|39445|691646|1000|7601|2|42|16|5|10708|158|892|28|53.83|156.64|1.56|4342.24|43.68|1507.24|4385.92|3.93|0.00|43.68|43.68|47.61|87.36|91.29|-1463.56| +2450832|45362|2450871|39445|691646|1000|7601|39445|691646|1000|7601|2|50|8|4|16742|300|892|29|53.69|112.74|45.09|1961.85|1307.61|1557.01|3269.46|26.15|0.00|817.22|1307.61|1333.76|2124.83|2150.98|-249.40| +2450832|45362|2450870|39445|691646|1000|7601|39445|691646|1000|7601|2|105|6|1|14041|184|892|14|96.40|131.10|62.92|954.52|880.88|1349.60|1835.40|8.80|0.00|807.52|880.88|889.68|1688.40|1697.20|-468.72| +2450832|45362|2450859|39445|691646|1000|7601|39445|691646|1000|7601|2|66|1|1|1172|124|892|40|7.35|15.28|14.66|24.80|586.40|294.00|611.20|41.04|0.00|73.20|586.40|627.44|659.60|700.64|292.40| +2450832|45362|2450901|39445|691646|1000|7601|39445|691646|1000|7601|2|9|6|3|6382|109|892|17|3.00|6.93|4.29|44.88|72.93|51.00|117.81|0.72|0.00|23.46|72.93|73.65|96.39|97.11|21.93| +2450832|34365|2450843|81662|1805243|1977|32758|81662|1805243|1977|32758|2|60|2|3|2060|12|893|72|96.60|255.02|170.86|6059.52|12301.92|6955.20|18361.44|738.11|0.00|6793.20|12301.92|13040.03|19095.12|19833.23|5346.72| +2450832|34365|2450893|81662|1805243|1977|32758|81662|1805243|1977|32758|2|58|14|1|10339|167|893|67|66.06|124.85|22.47|6859.46|1505.49|4426.02|8364.95|75.27|0.00|3094.73|1505.49|1580.76|4600.22|4675.49|-2920.53| +2450832|34365|2450922|81662|1805243|1977|32758|81662|1805243|1977|32758|2|12|15|1|15886|259|893|20|9.81|19.32|16.80|50.40|336.00|196.20|386.40|6.72|0.00|170.00|336.00|342.72|506.00|512.72|139.80| +2450832|34365|2450846|81662|1805243|1977|32758|81662|1805243|1977|32758|2|20|16|3|11188|59|893|85|3.39|6.81|4.69|180.20|398.65|288.15|578.85|11.95|0.00|51.85|398.65|410.60|450.50|462.45|110.50| +2450832|34365|2450841|81662|1805243|1977|32758|81662|1805243|1977|32758|2|34|14|2|9829|134|893|46|99.16|124.94|29.98|4368.16|1379.08|4561.36|5747.24|68.95|0.00|172.04|1379.08|1448.03|1551.12|1620.07|-3182.28| +2450832|34365|2450862|81662|1805243|1977|32758|81662|1805243|1977|32758|2|54|4|4|10778|46|893|83|21.23|56.47|0.56|4640.53|46.48|1762.09|4687.01|1.85|0.00|0.00|46.48|48.33|46.48|48.33|-1715.61| +2450832|71921|2450871|9988|376508|5428|21995|9988|376508|5428|21995|4|103|2|3|17443|160|894|37|33.41|67.15|22.83|1639.84|844.71|1236.17|2484.55|16.89|0.00|471.75|844.71|861.60|1316.46|1333.35|-391.46| +2450832|71921|2450921|9988|376508|5428|21995|9988|376508|5428|21995|4|87|11|3|14566|86|894|73|23.97|66.87|2.00|4735.51|146.00|1749.81|4881.51|10.22|0.00|243.82|146.00|156.22|389.82|400.04|-1603.81| +2450832|71921|2450851|9988|376508|5428|21995|9988|376508|5428|21995|4|38|4|2|9259|183|894|6|64.39|76.62|71.25|32.22|427.50|386.34|459.72|6.84|342.00|55.14|85.50|92.34|140.64|147.48|-300.84| +2450832|71921|2450840|9988|376508|5428|21995|9988|376508|5428|21995|4|29|3|1|7478|157|894|14|49.11|86.43|8.64|1089.06|120.96|687.54|1210.02|7.25|0.00|556.50|120.96|128.21|677.46|684.71|-566.58| +2450832|71921|2450911|9988|376508|5428|21995|9988|376508|5428|21995|4|10|15|3|5965|277|894|82|27.00|64.26|8.35|4584.62|684.70|2214.00|5269.32|41.08|0.00|1633.44|684.70|725.78|2318.14|2359.22|-1529.30| +2450832|71921|2450884|9988|376508|5428|21995|9988|376508|5428|21995|4|66|19|4|8098|51|894|51|63.67|187.82|37.56|7663.26|1915.56|3247.17|9578.82|114.93|0.00|95.37|1915.56|2030.49|2010.93|2125.86|-1331.61| +2450832|71921|2450913|9988|376508|5428|21995|9988|376508|5428|21995|4|3|20|4|11665|219|894|1|62.20|167.94|6.71|161.23|6.71|62.20|167.94|0.13|0.00|45.34|6.71|6.84|52.05|52.18|-55.49| +2450832|71921|2450917|9988|376508|5428|21995|9988|376508|5428|21995|4|50|13|5|3907|133|894|66|28.98|50.42|41.34|599.28|2728.44|1912.68|3327.72|163.70|0.00|33.00|2728.44|2892.14|2761.44|2925.14|815.76| +2450832|71921|2450849|9988|376508|5428|21995|9988|376508|5428|21995|4|15|12|5|1891|56|894|55|63.43|139.54|108.84|1688.50|5986.20|3488.65|7674.70|419.03|0.00|153.45|5986.20|6405.23|6139.65|6558.68|2497.55| +2450832|1625|2450887|98698|61881|3464|47447|98698|61881|3464|47447|4|75|1|3|2539|147|895|22|93.23|109.07|66.53|935.88|1463.66|2051.06|2399.54|102.45|0.00|71.94|1463.66|1566.11|1535.60|1638.05|-587.40| +2450832|1625|2450914|98698|61881|3464|47447|98698|61881|3464|47447|4|4|16|2|8689|2|895|90|79.79|102.13|37.78|5791.50|3400.20|7181.10|9191.70|238.01|0.00|2389.50|3400.20|3638.21|5789.70|6027.71|-3780.90| +2450832|1625|2450911|98698|61881|3464|47447|98698|61881|3464|47447|4|99|19|1|17911|6|895|51|51.13|84.87|60.25|1255.62|3072.75|2607.63|4328.37|92.18|0.00|1255.11|3072.75|3164.93|4327.86|4420.04|465.12| +2450832|1625|2450835|98698|61881|3464|47447|98698|61881|3464|47447|4|105|19|5|1084|29|895|46|71.69|83.16|47.40|1644.96|2180.40|3297.74|3825.36|21.80|0.00|1032.70|2180.40|2202.20|3213.10|3234.90|-1117.34| +2450832|1625|2450888|98698|61881|3464|47447|98698|61881|3464|47447|4|28|19|1|10297|200|895|29|68.66|198.42|31.74|4833.72|920.46|1991.14|5754.18|14.72|736.36|2646.83|184.10|198.82|2830.93|2845.65|-1807.04| +2450832|1625|2450913|98698|61881|3464|47447|98698|61881|3464|47447|4|12|4|2|5594|269|895|4|56.93|116.13|13.93|408.80|55.72|227.72|464.52|2.22|0.00|222.96|55.72|57.94|278.68|280.90|-172.00| +2450832|1625|2450897|98698|61881|3464|47447|98698|61881|3464|47447|4|98|15|5|17954|40|895|91|32.49|54.58|26.74|2533.44|2433.34|2956.59|4966.78|24.33|0.00|297.57|2433.34|2457.67|2730.91|2755.24|-523.25| +2450832|1625|2450870|98698|61881|3464|47447|98698|61881|3464|47447|4|89|15|5|2143|53|895|100|67.19|188.13|167.43|2070.00|16743.00|6719.00|18813.00|669.72|0.00|752.00|16743.00|17412.72|17495.00|18164.72|10024.00| +2450832|1625|2450871|98698|61881|3464|47447|98698|61881|3464|47447|4|104|2|3|4208|137|895|88|85.96|238.96|160.10|6939.68|14088.80|7564.48|21028.48|690.35|4226.64|3995.20|9862.16|10552.51|13857.36|14547.71|2297.68| +2450832|1625|2450868|98698|61881|3464|47447|98698|61881|3464|47447|4|103|5|3|5852|252|895|89|9.89|11.07|4.87|551.80|433.43|880.21|985.23|26.00|0.00|97.90|433.43|459.43|531.33|557.33|-446.78| +2450832|1625|2450865|98698|61881|3464|47447|98698|61881|3464|47447|4|6|20|4|14749|72|895|77|51.24|69.17|58.10|852.39|4473.70|3945.48|5326.09|44.73|0.00|532.07|4473.70|4518.43|5005.77|5050.50|528.22| +2450832|1625|2450863|98698|61881|3464|47447|98698|61881|3464|47447|4|53|11|3|14977|137|895|96|37.48|87.32|28.81|5616.96|2765.76|3598.08|8382.72|193.60|0.00|2514.24|2765.76|2959.36|5280.00|5473.60|-832.32| +2450832|9536|2450892|14335|1299623|6064|10552|14335|1299623|6064|10552|4|92|6|4|16216|45|896|61|78.79|204.85|53.26|9246.99|3248.86|4806.19|12495.85|259.90|0.00|3623.40|3248.86|3508.76|6872.26|7132.16|-1557.33| +2450832|9536|2450881|14335|1299623|6064|10552|14335|1299623|6064|10552|4|75|15|3|2701|244|896|10|2.74|4.54|0.09|44.50|0.90|27.40|45.40|0.02|0.00|13.10|0.90|0.92|14.00|14.02|-26.50| +2450832|9536|2450839|14335|1299623|6064|10552|14335|1299623|6064|10552|4|17|13|2|10972|160|896|75|69.04|133.93|28.12|7935.75|2109.00|5178.00|10044.75|21.09|0.00|1305.75|2109.00|2130.09|3414.75|3435.84|-3069.00| +2450832|9536|2450865|14335|1299623|6064|10552|14335|1299623|6064|10552|4|90|2|1|3064|164|896|38|20.29|38.14|0.38|1434.88|14.44|771.02|1449.32|1.01|0.00|623.20|14.44|15.45|637.64|638.65|-756.58| +2450832|9536|2450855|14335|1299623|6064|10552|14335|1299623|6064|10552|4|52|17|3|3188|225|896|26|34.45|84.40|9.28|1953.12|241.28|895.70|2194.40|4.00|41.01|153.40|200.27|204.27|353.67|357.67|-695.43| +2450832|9536|2450886|14335|1299623|6064|10552|14335|1299623|6064|10552|4|25|16|4|6664|27|896|21|79.23|137.86|103.39|723.87|2171.19|1663.83|2895.06|43.42|0.00|1157.94|2171.19|2214.61|3329.13|3372.55|507.36| +2450832|9536|2450849|14335|1299623|6064|10552|14335|1299623|6064|10552|4|99|18|1|17288|141|896|90|41.67|53.75|9.67|3967.20|870.30|3750.30|4837.50|43.51|0.00|1161.00|870.30|913.81|2031.30|2074.81|-2880.00| +|44669|2450917|33172||6022|14047|33172|1857997||14047|4|||2|4195||897|9||77.34|72.69|41.85|654.21|446.22|696.06|24.86||0.00|||497.20|522.06|50.98| +2450832|44669|2450907|33172|1857997|6022|14047|33172|1857997|6022|14047|4|103|12|2|10160|161|897|64|55.40|113.01|47.46|4195.20|3037.44|3545.60|7232.64|242.99|0.00|216.96|3037.44|3280.43|3254.40|3497.39|-508.16| +2450832|44669|2450891|33172|1857997|6022|14047|33172|1857997|6022|14047|4|50|20|2|7288|39|897|68|38.93|52.94|4.76|3276.24|323.68|2647.24|3599.92|16.99|80.92|683.40|242.76|259.75|926.16|943.15|-2404.48| +2450832|44669|2450876|33172|1857997|6022|14047|33172|1857997|6022|14047|4|37|1|5|148|42|897|89|63.96|76.11|54.03|1965.12|4808.67|5692.44|6773.79|100.02|3558.41|3115.89|1250.26|1350.28|4366.15|4466.17|-4442.18| +2450832|37959|2450870|45509|130790|1671|27752|45509|130790|1671|27752|4|86|10|2|11911|164|898|96|66.11|143.45|114.76|2754.24|11016.96|6346.56|13771.20|991.52|0.00|412.80|11016.96|12008.48|11429.76|12421.28|4670.40| +2450832|37959|2450843|45509|130790|1671|27752|45509|130790|1671|27752|4|17|10|4|17338|275|898|15|94.68|160.00|115.20|672.00|1728.00|1420.20|2400.00|103.68|0.00|216.00|1728.00|1831.68|1944.00|2047.68|307.80| +2450832|37959|2450897|45509|130790|1671|27752|45509|130790|1671|27752|4|84|14|2|9962|253|898|40|76.64|145.61|2.91|5708.00|116.40|3065.60|5824.40|1.60|36.08|2038.40|80.32|81.92|2118.72|2120.32|-2985.28| +2450832|37959|2450843|45509|130790|1671|27752|45509|130790|1671|27752|4|28|17|4|3451|34|898|76|87.74|111.42|101.39|762.28|7705.64|6668.24|8467.92|539.39|0.00|2032.24|7705.64|8245.03|9737.88|10277.27|1037.40| +2450832|37959|2450882|45509|130790|1671|27752|45509|130790|1671|27752|4|77|7|2|3403|3|898|64|60.15|129.92|111.73|1164.16|7150.72|3849.60|8314.88|572.05|0.00|3408.64|7150.72|7722.77|10559.36|11131.41|3301.12| +2450832|37959|2450886|45509|130790|1671|27752|45509|130790|1671|27752|4|23|6|4|8629|3|898|12|12.33|27.61|11.87|188.88|142.44|147.96|331.32|0.00|0.00|62.88|142.44|142.44|205.32|205.32|-5.52| +2450832|37959|2450848|45509|130790|1671|27752|45509|130790|1671|27752|4|95|4|5|9446|30|898|18|95.65|271.64|239.04|586.80|4302.72|1721.70|4889.52|172.10|0.00|635.58|4302.72|4474.82|4938.30|5110.40|2581.02| +2450832|37959|2450863|45509|130790|1671|27752|45509|130790|1671|27752|4|12|18|3|14401|1|898|34|54.37|90.25|21.66|2332.06|736.44|1848.58|3068.50|15.31|353.49|91.80|382.95|398.26|474.75|490.06|-1465.63| +2450832|37959|2450850|45509|130790|1671|27752|45509|130790|1671|27752|4|107|18|2|4274|30|898|89|62.14|147.89|122.74|2238.35|10923.86|5530.46|13162.21|218.47|0.00|525.99|10923.86|11142.33|11449.85|11668.32|5393.40| +2450832|37959|2450888|45509|130790|1671|27752|45509|130790|1671|27752|4|47|9|4|3854|258|898|44|55.54|112.74|72.15|1785.96|3174.60|2443.76|4960.56|253.96|0.00|1735.80|3174.60|3428.56|4910.40|5164.36|730.84| +2450832|37959|2450857|45509|130790|1671|27752|45509|130790|1671|27752|4|7|18|4|10903|246|898|20|83.16|85.65|64.23|428.40|1284.60|1663.20|1713.00|64.23|0.00|531.00|1284.60|1348.83|1815.60|1879.83|-378.60| +2450832|37959|2450858|45509|130790|1671|27752|45509|130790|1671|27752|4|38|4|2|8666|185|898|33|95.51|113.65|84.10|975.15|2775.30|3151.83|3750.45|22.75|499.55|862.29|2275.75|2298.50|3138.04|3160.79|-876.08| +2450832|69562|2450862|85547|599956|6812|49221|85547|599956|6812|49221|1|102|8|1|6847|177|899|62|10.04|10.84|7.69|195.30|476.78|622.48|672.08|42.91|0.00|133.92|476.78|519.69|610.70|653.61|-145.70| +2450832|69562|2450886|85547|599956|6812|49221|85547|599956|6812|49221|1|73|7|5|10334|151|899|63|37.06|68.56|30.16|2419.20|1900.08|2334.78|4319.28|171.00|0.00|1468.53|1900.08|2071.08|3368.61|3539.61|-434.70| +2450832|69562|2450898|85547|599956|6812|49221|85547|599956|6812|49221|1|19|9|1|14750|58|899|47|81.18|212.69|42.53|7997.52|1998.91|3815.46|9996.43|39.97|0.00|1499.30|1998.91|2038.88|3498.21|3538.18|-1816.55| +2450832|69562|2450904|85547|599956|6812|49221|85547|599956|6812|49221|1|24|16|4|9304|79|899|96|60.54|115.63|83.25|3108.48|7992.00|5811.84|11100.48|99.10|5514.48|2997.12|2477.52|2576.62|5474.64|5573.74|-3334.32| +2450832|69562|2450872|85547|599956|6812|49221|85547|599956|6812|49221|1|85|12|3|872|23|899|4|11.59|13.56|1.08|49.92|4.32|46.36|54.24|0.18|2.24|21.68|2.08|2.26|23.76|23.94|-44.28| +2450832|69562|2450880|85547|599956|6812|49221|85547|599956|6812|49221|1|13|20|3|7646|74|899|7|30.08|65.27|62.65|18.34|438.55|210.56|456.89|0.08|429.77|91.35|8.78|8.86|100.13|100.21|-201.78| +2450832|69562|2450851|85547|599956|6812|49221|85547|599956|6812|49221|1|103|8|4|15638|228|899|13|79.71|219.20|105.21|1481.87|1367.73|1036.23|2849.60|0.00|0.00|854.88|1367.73|1367.73|2222.61|2222.61|331.50| +2450832|69562|2450855|85547|599956|6812|49221|85547|599956|6812|49221|1|108|3|4|5744|78|899|27|43.33|56.76|37.46|521.10|1011.42|1169.91|1532.52|20.22|0.00|536.22|1011.42|1031.64|1547.64|1567.86|-158.49| +2450832|69562|2450919|85547|599956|6812|49221|85547|599956|6812|49221|1|32|6|5|8444|225|899|62|42.12|108.24|36.80|4429.28|2281.60|2611.44|6710.88|182.52|0.00|2885.48|2281.60|2464.12|5167.08|5349.60|-329.84| +2450832|69562|2450909|85547|599956|6812|49221|85547|599956|6812|49221|1|45|1|3|16138|119|899|31|70.83|190.53|104.79|2657.94|3248.49|2195.73|5906.43|227.39|0.00|2539.52|3248.49|3475.88|5788.01|6015.40|1052.76| +2450832|69584|2450861|43127|191577|4474|18124|43127|191577|4474|18124|1|2|11|5|13783|125|900|51|6.50|17.94|3.40|741.54|173.40|331.50|914.94|13.87|0.00|36.21|173.40|187.27|209.61|223.48|-158.10| +2450832|69584|2450902|43127|191577|4474|18124|43127|191577|4474|18124|1|74|18|5|2522|22|900|59|4.45|11.12|0.00|656.08|0.00|262.55|656.08|0.00|0.00|97.94|0.00|0.00|97.94|97.94|-262.55| +2450832|69584|2450902|43127|191577|4474|18124|43127|191577|4474|18124|1|11|11|1|2221|114|900|12|18.58|48.12|23.57|294.60|282.84|222.96|577.44|14.14|0.00|207.84|282.84|296.98|490.68|504.82|59.88| +2450832|69584|2450838|43127|191577|4474|18124|43127|191577|4474|18124|1|70|9|2|6770|230|900|28|56.11|57.23|18.88|1073.80|528.64|1571.08|1602.44|0.00|0.00|608.72|528.64|528.64|1137.36|1137.36|-1042.44| +2450832|69584|2450906|43127|191577|4474|18124|43127|191577|4474|18124|1|43|2|5|8542|234|900|75|93.00|93.00|63.24|2232.00|4743.00|6975.00|6975.00|94.86|0.00|1325.25|4743.00|4837.86|6068.25|6163.11|-2232.00| +2450832|69584|2450869|43127|191577|4474|18124|43127|191577|4474|18124|1|107|16|1|9118|123|900|14|12.19|25.72|20.57|72.10|287.98|170.66|360.08|0.00|0.00|75.60|287.98|287.98|363.58|363.58|117.32| +2450832|69584|2450839|43127|191577|4474|18124|43127|191577|4474|18124|1|71|15|3|3806|119|900|30|41.92|101.86|68.24|1008.60|2047.20|1257.60|3055.80|184.24|0.00|1069.50|2047.20|2231.44|3116.70|3300.94|789.60| +2450832|69584|2450860|43127|191577|4474|18124|43127|191577|4474|18124|1|98|2|3|964|253|900|78|11.68|16.46|7.24|719.16|564.72|911.04|1283.88|39.53|0.00|384.54|564.72|604.25|949.26|988.79|-346.32| +2450832|69584|2450850|43127|191577|4474|18124|43127|191577|4474|18124|1|78|5|1|754|290|900|44|95.31|199.19|27.88|7537.64|1226.72|4193.64|8764.36|11.28|944.57|1577.40|282.15|293.43|1859.55|1870.83|-3911.49| +2450832|69584|2450900|43127|191577|4474|18124|43127|191577|4474|18124|1|27|3|2|10202|41|900|72|4.02|5.26|2.63|189.36|189.36|289.44|378.72|11.36|0.00|120.96|189.36|200.72|310.32|321.68|-100.08| +2450832|63420|2450914|14676|1704466|5103|26093|14676|1704466|5103|26093|1|60|20|4|6316|117|901|83|93.87|239.36|59.84|14900.16|4966.72|7791.21|19866.88|248.33|0.00|6952.91|4966.72|5215.05|11919.63|12167.96|-2824.49| +2450832|63420|2450840|14676|1704466|5103|26093|14676|1704466|5103|26093|1|36|14|2|17023|127|901|76|43.70|95.70|90.91|364.04|6909.16|3321.20|7273.20|207.27|0.00|508.44|6909.16|7116.43|7417.60|7624.87|3587.96| +2450832|63420|2450858|14676|1704466|5103|26093|14676|1704466|5103|26093|1|36|11|2|388|158|901|59|60.26|86.17|29.29|3355.92|1728.11|3555.34|5084.03|155.52|0.00|2440.24|1728.11|1883.63|4168.35|4323.87|-1827.23| +2450832|63420|2450880|14676|1704466|5103|26093|14676|1704466|5103|26093|1|64|10|4|5672|297|901|25|28.94|34.72|15.27|486.25|381.75|723.50|868.00|3.81|0.00|34.50|381.75|385.56|416.25|420.06|-341.75| +2450832|63420|2450896|14676|1704466|5103|26093|14676|1704466|5103|26093|1|13|16|3|5446|27|901|53|45.73|124.38|47.26|4087.36|2504.78|2423.69|6592.14|125.23|0.00|1318.11|2504.78|2630.01|3822.89|3948.12|81.09| +2450832|63420|2450875|14676|1704466|5103|26093|14676|1704466|5103|26093|1|67|12|3|7036|197|901|98|12.12|21.33|5.75|1526.84|563.50|1187.76|2090.34|33.81|0.00|877.10|563.50|597.31|1440.60|1474.41|-624.26| +2450832|63420|2450869|14676|1704466|5103|26093|14676|1704466|5103|26093|1|97|1|5|16358|5|901|18|8.07|9.03|2.25|122.04|40.50|145.26|162.54|3.64|0.00|14.58|40.50|44.14|55.08|58.72|-104.76| +2450832|63420|2450836|14676|1704466|5103|26093|14676|1704466|5103|26093|1|65|14|4|3824|80|901|57|65.15|89.25|66.93|1272.24|3815.01|3713.55|5087.25|305.20|0.00|610.47|3815.01|4120.21|4425.48|4730.68|101.46| +2450832|63420|2450850|14676|1704466|5103|26093|14676|1704466|5103|26093|1|41|19|2|6488|296|901|66|32.17|53.40|21.89|2079.66|1444.74|2123.22|3524.40|115.57|0.00|881.10|1444.74|1560.31|2325.84|2441.41|-678.48| +2450832|63420|2450920|14676|1704466|5103|26093|14676|1704466|5103|26093|1|2|7|3|2608|51|901|4|31.37|77.79|5.44|289.40|21.76|125.48|311.16|0.87|0.00|6.20|21.76|22.63|27.96|28.83|-103.72| +2450832|63420|2450866|14676|1704466|5103|26093|14676|1704466|5103|26093|1|52|17|2|187|287|901|51|71.58|127.41|31.85|4873.56|1624.35|3650.58|6497.91|0.00|211.16|1364.25|1413.19|1413.19|2777.44|2777.44|-2237.39| +2450832|63420|2450904|14676|1704466|5103|26093|14676|1704466|5103|26093|1|104|5|3|11131|169|901|96|19.26|29.66|2.37|2619.84|227.52|1848.96|2847.36|0.00|0.00|227.52|227.52|227.52|455.04|455.04|-1621.44| +2450832|63420|2450840|14676|1704466|5103|26093|14676|1704466|5103|26093|1|5|16|2|5455|152|901|27|81.72|201.84|74.68|3433.32|2016.36|2206.44|5449.68|80.65|0.00|1307.88|2016.36|2097.01|3324.24|3404.89|-190.08| +2450832|26494|2450905|62204|943693|5182|42271|62204|943693|5182|42271|1|107|12|1|196|236|902|3|62.26|125.14|30.03|285.33|90.09|186.78|375.42|0.00|0.00|157.65|90.09|90.09|247.74|247.74|-96.69| +2450832|26494|2450905|62204|943693|5182|42271|62204|943693|5182|42271|1|27|16|3|15901|193|902|65|14.02|27.33|19.40|515.45|1261.00|911.30|1776.45|12.61|0.00|763.75|1261.00|1273.61|2024.75|2037.36|349.70| +2450832|26494|2450841|62204|943693|5182|42271|62204|943693|5182|42271|1|35|16|5|2036|128|902|8|15.45|45.88|22.94|183.52|183.52|123.60|367.04|16.51|0.00|139.44|183.52|200.03|322.96|339.47|59.92| +2450832|26494|2450851|62204|943693|5182|42271|62204|943693|5182|42271|1|93|18|2|7082|18|902|34|35.79|104.14|7.28|3293.24|247.52|1216.86|3540.76|4.95|0.00|566.44|247.52|252.47|813.96|818.91|-969.34| +2450832|26494|2450917|62204|943693|5182|42271|62204|943693|5182|42271|1|78|20|1|7660|220|902|46|14.01|25.21|9.32|730.94|428.72|644.46|1159.66|18.00|68.59|0.00|360.13|378.13|360.13|378.13|-284.33| +2450832|26494|2450852|62204|943693|5182|42271|62204|943693|5182|42271|1|38|18|5|10820|168|902|36|41.74|111.86|108.50|120.96|3906.00|1502.64|4026.96|156.24|0.00|644.04|3906.00|4062.24|4550.04|4706.28|2403.36| +2450832|26494|2450835|62204|943693|5182|42271|62204|943693|5182|42271|1|12|18|1|1804|138|902|97|97.13|163.17|128.90|3324.19|12503.30|9421.61|15827.49|1000.26|0.00|5380.59|12503.30|13503.56|17883.89|18884.15|3081.69| +2450832|76960|2450868|95504|346596|888|47980|95504|346596|888|47980|4|16|6|4|13894|283|903|76|39.14|102.54|37.93|4910.36|2882.68|2974.64|7793.04|172.96|0.00|2649.36|2882.68|3055.64|5532.04|5705.00|-91.96| +2450832|76960|2450880|95504|346596|888|47980|95504|346596|888|47980|4|36|4|2|14434|40|903|53|47.34|116.92|28.06|4709.58|1487.18|2509.02|6196.76|89.23|0.00|2354.26|1487.18|1576.41|3841.44|3930.67|-1021.84| +2450832|76960|2450895|95504|346596|888|47980|95504|346596|888|47980|4|66|14|1|7177|123|903|15|16.29|34.69|12.48|333.15|187.20|244.35|520.35|1.96|147.88|182.10|39.32|41.28|221.42|223.38|-205.03| +2450832|76960|2450865|95504|346596|888|47980|95504|346596|888|47980|4|62|20|4|8612|257|903|22|68.57|202.28|50.57|3337.62|1112.54|1508.54|4450.16|77.87|0.00|1379.40|1112.54|1190.41|2491.94|2569.81|-396.00| +2450832|60515|2450919|66418|326907|6596|13485|66418|326907|6596|13485|2|11|5|1|15086|18|904|38|28.32|69.38|0.69|2610.22|26.22|1076.16|2636.44|1.04|0.00|448.02|26.22|27.26|474.24|475.28|-1049.94| +2450832|60515|2450839|66418|326907|6596|13485|66418|326907|6596|13485|2|40|13|3|9584|250|904|26|49.20|56.08|6.16|1297.92|160.16|1279.20|1458.08|0.00|0.00|0.00|160.16|160.16|160.16|160.16|-1119.04| +2450832|60515|2450838|66418|326907|6596|13485|66418|326907|6596|13485|2|72|14|4|10370|284|904|83|72.82|185.69|139.26|3853.69|11558.58|6044.06|15412.27|346.75|0.00|5239.79|11558.58|11905.33|16798.37|17145.12|5514.52| +2450832|60515|2450896|66418|326907|6596|13485|66418|326907|6596|13485|2|22|16|1|3574|226|904|27|18.43|37.04|32.96|110.16|889.92|497.61|1000.08|11.74|596.24|339.93|293.68|305.42|633.61|645.35|-203.93| +2450832|60515|2450911|66418|326907|6596|13485|66418|326907|6596|13485|2|7|11|4|919|190|904|35|50.39|140.08|88.25|1814.05|3088.75|1763.65|4902.80|191.81|957.51|931.35|2131.24|2323.05|3062.59|3254.40|367.59| +2450832|60515|2450877|66418|326907|6596|13485|66418|326907|6596|13485|2|66|13|2|9565|110|904|9|50.59|121.92|57.30|581.58|515.70|455.31|1097.28|20.62|0.00|416.88|515.70|536.32|932.58|953.20|60.39| +2450832|60515|2450871|66418|326907|6596|13485|66418|326907|6596|13485|2|75|6|1|14612|234|904|24|29.96|76.69|36.81|957.12|883.44|719.04|1840.56|17.66|0.00|239.04|883.44|901.10|1122.48|1140.14|164.40| +2450832|60515|2450835|66418|326907|6596|13485|66418|326907|6596|13485|2|93|18|2|12092|125|904|51|7.05|14.02|2.80|572.22|142.80|359.55|715.02|8.56|0.00|99.96|142.80|151.36|242.76|251.32|-216.75| +2450832|79809|2450877|31825|1563826|627|13475|31825|1563826|627|13475|2|66|7|1|7208|112|905|42|56.66|158.08|120.14|1593.48|5045.88|2379.72|6639.36|151.37|0.00|1726.20|5045.88|5197.25|6772.08|6923.45|2666.16| +2450832|79809|2450876|31825|1563826|627|13475|31825|1563826|627|13475|2|53|5|3|10735|277|905|58|43.99|121.41|63.13|3380.24|3661.54|2551.42|7041.78|329.53|0.00|1901.24|3661.54|3991.07|5562.78|5892.31|1110.12| +2450832|79809|2450853|31825|1563826|627|13475|31825|1563826|627|13475|2|66|10|4|12670|167|905|95|42.33|69.42|47.20|2110.90|4484.00|4021.35|6594.90|269.04|0.00|1054.50|4484.00|4753.04|5538.50|5807.54|462.65| +2450832|79809|2450866|31825|1563826|627|13475|31825|1563826|627|13475|2|70|17|4|2078|33|905|30|9.01|11.35|5.78|167.10|173.40|270.30|340.50|10.40|0.00|129.30|173.40|183.80|302.70|313.10|-96.90| +2450832|24124|2450865|71888|1244933|5178|42576|39950|1469466|4111|48356|2|86|5|2|14146|86|906|61|11.42|13.47|7.54|361.73|459.94|696.62|821.67|4.59|0.00|123.22|459.94|464.53|583.16|587.75|-236.68| +2450832|24124|2450914|71888|1244933|5178|42576|39950|1469466|4111|48356|2|102|18|2|13594|214|906|93|5.44|9.90|8.51|129.27|791.43|505.92|920.70|25.72|277.00|17.67|514.43|540.15|532.10|557.82|8.51| +2450832|24124|2450913|71888|1244933|5178|42576|39950|1469466|4111|48356|2|66|12|5|17377|269|906|29|49.81|105.59|68.63|1071.84|1990.27|1444.49|3062.11|139.31|0.00|796.05|1990.27|2129.58|2786.32|2925.63|545.78| +2450832|24124|2450917|71888|1244933|5178|42576|39950|1469466|4111|48356|2|21|8|3|10546|5|906|66|19.60|42.53|6.80|2358.18|448.80|1293.60|2806.98|40.39|0.00|757.68|448.80|489.19|1206.48|1246.87|-844.80| +2450832|24124|2450888|71888|1244933|5178|42576|39950|1469466|4111|48356|2|86|16|3|13802|35|906|73|50.61|94.64|35.96|4283.64|2625.08|3694.53|6908.72|78.75|0.00|137.97|2625.08|2703.83|2763.05|2841.80|-1069.45| +2450832|24124|2450912|71888|1244933|5178|42576|39950|1469466|4111|48356|2|18|9|5|16777|294|906|93|43.18|51.38|19.01|3010.41|1767.93|4015.74|4778.34|35.35|0.00|716.10|1767.93|1803.28|2484.03|2519.38|-2247.81| +2450832|36461|2450878|69625|884802|3179|21556|95351|571110|687|7464|1|23|12|1|1270|117|907|1|4.59|12.39|6.44|5.95|6.44|4.59|12.39|0.38|0.00|1.48|6.44|6.82|7.92|8.30|1.85| +2450832|36461|2450838|69625|884802|3179|21556|95351|571110|687|7464|1|38|6|3|25|154|907|30|42.72|77.32|62.62|441.00|1878.60|1281.60|2319.60|150.28|0.00|1043.70|1878.60|2028.88|2922.30|3072.58|597.00| +2450832|36461|2450883|69625|884802|3179|21556|95351|571110|687|7464|1|94|2|1|17428|114|907|81|15.67|45.28|39.39|477.09|3190.59|1269.27|3667.68|0.00|0.00|989.82|3190.59|3190.59|4180.41|4180.41|1921.32| +2450832|36461|2450877|69625|884802|3179|21556|95351|571110|687|7464|1|55|14|3|15463|68|907|95|11.94|15.88|10.16|543.40|965.20|1134.30|1508.60|19.30|0.00|29.45|965.20|984.50|994.65|1013.95|-169.10| +2450832|36461|2450857|69625|884802|3179|21556|95351|571110|687|7464|1|45|16|3|9667|83|907|80|2.86|4.74|4.07|53.60|325.60|228.80|379.20|0.00|0.00|128.80|325.60|325.60|454.40|454.40|96.80| +2450832|36461|2450844|69625|884802|3179|21556|95351|571110|687|7464|1|102|14|1|17932|291|907|51|72.09|198.96|121.36|3957.60|6189.36|3676.59|10146.96|237.67|2228.16|4261.56|3961.20|4198.87|8222.76|8460.43|284.61| +2450832|36461|2450881|69625|884802|3179|21556|95351|571110|687|7464|1|29|4|2|1489|228|907|91|57.12|96.53|67.57|2635.36|6148.87|5197.92|8784.23|491.90|0.00|3337.88|6148.87|6640.77|9486.75|9978.65|950.95| +2450832|36461|2450880|69625|884802|3179|21556|95351|571110|687|7464|1|5|3|5|511|67|907|98|69.19|186.81|141.97|4394.32|13913.06|6780.62|18307.38|13.91|13634.79|6223.98|278.27|292.18|6502.25|6516.16|-6502.35| +2450832|36461|2450912|69625|884802|3179|21556|95351|571110|687|7464|1|4|18|1|56|295|907|71|95.20|233.24|86.29|10433.45|6126.59|6759.20|16560.04|63.71|2940.76|7783.02|3185.83|3249.54|10968.85|11032.56|-3573.37| +2450832|36461|2450910|69625|884802|3179|21556|95351|571110|687|7464|1|26|16|2|9934|135|907|72|6.56|11.61|7.31|309.60|526.32|472.32|835.92|10.52|0.00|375.84|526.32|536.84|902.16|912.68|54.00| +2450832|36461|2450845|69625|884802|3179|21556|95351|571110|687|7464|1|41|4|1|1207|181|907|33|61.05|120.26|20.44|3294.06|674.52|2014.65|3968.58|6.74|0.00|833.25|674.52|681.26|1507.77|1514.51|-1340.13| +2450832|36461|2450899|69625|884802|3179|21556|95351|571110|687|7464|1|26|7|1|15490|70|907|77|62.49|109.35|73.26|2778.93|5641.02|4811.73|8419.95|282.05|0.00|4209.59|5641.02|5923.07|9850.61|10132.66|829.29| +2450832|28845|2450914|72143|1379954|6612|49623|72143|1379954|6612|49623|2|5|5|4|1946|272|908|67|99.68|138.55|123.30|1021.75|8261.10|6678.56|9282.85|0.00|0.00|1949.03|8261.10|8261.10|10210.13|10210.13|1582.54| +2450832|28845|2450878|72143|1379954|6612|49623|72143|1379954|6612|49623|2|108|12|5|7759|286|908|56|72.48|72.48|52.18|1136.80|2922.08|4058.88|4058.88|204.54|0.00|1217.44|2922.08|3126.62|4139.52|4344.06|-1136.80| +2450832|28845|2450921|72143|1379954|6612|49623|72143|1379954|6612|49623|2|23|2|5|17336|248|908|52|54.11|85.49|65.82|1022.84|3422.64|2813.72|4445.48|49.62|2430.07|1955.72|992.57|1042.19|2948.29|2997.91|-1821.15| +2450832|28845|2450865|72143|1379954|6612|49623|72143|1379954|6612|49623|2|102|12|1|11452|191|908|19|25.24|69.91|20.97|929.86|398.43|479.56|1328.29|7.96|0.00|331.93|398.43|406.39|730.36|738.32|-81.13| +2450832|28845|2450904|72143|1379954|6612|49623|72143|1379954|6612|49623|2|22|4|3|7784|161|908|99|8.17|11.27|1.57|960.30|155.43|808.83|1115.73|9.32|0.00|501.93|155.43|164.75|657.36|666.68|-653.40| +2450832|28845|2450884|72143|1379954|6612|49623|72143|1379954|6612|49623|2|87|9|3|11500|186|908|78|1.41|2.43|2.01|32.76|156.78|109.98|189.54|0.00|0.00|73.32|156.78|156.78|230.10|230.10|46.80| +2450832|28845|2450892|72143|1379954|6612|49623|72143|1379954|6612|49623|2|108|7|2|16100|194|908|12|53.21|53.21|30.86|268.20|370.32|638.52|638.52|0.00|0.00|223.44|370.32|370.32|593.76|593.76|-268.20| +2450832|6063|2450838|85499|555565|1631|6944|85499|555565|1631|6944|1|55|2|5|13441|226|909|58|74.45|147.41|33.90|6583.58|1966.20|4318.10|8549.78|0.00|963.43|1709.84|1002.77|1002.77|2712.61|2712.61|-3315.33| +2450832|6063|2450892|85499|555565|1631|6944|85499|555565|1631|6944|1|59|11|3|3235|7|909|18|37.31|72.00|67.68|77.76|1218.24|671.58|1296.00|36.54|0.00|609.12|1218.24|1254.78|1827.36|1863.90|546.66| +2450832|6063|2450887|85499|555565|1631|6944|85499|555565|1631|6944|1|81|17|1|110|39|909|80|23.60|29.26|15.80|1076.80|1264.00|1888.00|2340.80|113.76|0.00|1100.00|1264.00|1377.76|2364.00|2477.76|-624.00| +2450832|6063|2450878|85499|555565|1631|6944|85499|555565|1631|6944|1|66|15|5|16400|32|909|89|1.56|2.77|0.38|212.71|33.82|138.84|246.53|3.04|0.00|29.37|33.82|36.86|63.19|66.23|-105.02| +2450832|15848|2450854|7372|1220686|7153|8867|7372|1220686|7153|8867|1|104|7|3|10768|137|910|65|74.40|93.74|22.49|4631.25|1461.85|4836.00|6093.10|73.09|0.00|2559.05|1461.85|1534.94|4020.90|4093.99|-3374.15| +2450832|15848|2450901|7372|1220686|7153|8867|7372|1220686|7153|8867|1|33|11|3|12646|157|910|94|39.12|108.36|50.92|5399.36|4786.48|3677.28|10185.84|239.32|0.00|1018.02|4786.48|5025.80|5804.50|6043.82|1109.20| +2450832|15848|2450886|7372|1220686|7153|8867|7372|1220686|7153|8867|1|7|4|5|4136|265|910|38|80.34|237.80|221.15|632.70|8403.70|3052.92|9036.40|219.33|5966.62|2530.04|2437.08|2656.41|4967.12|5186.45|-615.84| +2450832|15848|2450869|7372|1220686|7153|8867|7372|1220686|7153|8867|1|10|14|5|7442|195|910|15|47.73|115.98|57.99|869.85|869.85|715.95|1739.70|78.28|0.00|173.85|869.85|948.13|1043.70|1121.98|153.90| +2450832|15848|2450843|7372|1220686|7153|8867|7372|1220686|7153|8867|1|54|12|3|16418|161|910|5|73.31|205.26|129.31|379.75|646.55|366.55|1026.30|51.72|0.00|482.35|646.55|698.27|1128.90|1180.62|280.00| +2450832|15848|2450866|7372|1220686|7153|8867|7372|1220686|7153|8867|1|40|16|5|11383|218|910|94|25.29|33.88|29.13|446.50|2738.22|2377.26|3184.72|246.43|0.00|222.78|2738.22|2984.65|2961.00|3207.43|360.96| +2450832|15848|2450880|7372|1220686|7153|8867|7372|1220686|7153|8867|1|41|3|2|17281|18|910|94|49.28|139.46|103.20|3408.44|9700.80|4632.32|13109.24|679.05|0.00|1179.70|9700.80|10379.85|10880.50|11559.55|5068.48| +2450832|3940|2450918|32960|1851088|5663|2238|74479|763341|4669|38844|2|1|17|3|304|232|911|52|5.08|12.95|5.18|404.04|269.36|264.16|673.40|6.54|51.17|100.88|218.19|224.73|319.07|325.61|-45.97| +2450832|3940|2450848|32960|1851088|5663|2238|74479|763341|4669|38844|2|34|7|5|1768|116|911|68|17.82|45.79|42.12|249.56|2864.16|1211.76|3113.72|85.92|0.00|124.44|2864.16|2950.08|2988.60|3074.52|1652.40| +2450832|3940|2450839|32960|1851088|5663|2238|74479|763341|4669|38844|2|91|15|2|4864|216|911|3|35.45|39.70|16.67|69.09|50.01|106.35|119.10|1.50|0.00|3.57|50.01|51.51|53.58|55.08|-56.34| +2450832|3940|2450893|32960|1851088|5663|2238|74479|763341|4669|38844|2|5|9|2|6584|188|911|94|98.24|167.00|5.01|15227.06|470.94|9234.56|15698.00|32.96|0.00|2511.68|470.94|503.90|2982.62|3015.58|-8763.62| +2450832|3940|2450866|32960|1851088|5663|2238|74479|763341|4669|38844|2|44|1|4|15400|88|911|47|87.21|248.54|181.43|3154.17|8527.21|4098.87|11681.38|170.54|0.00|2803.08|8527.21|8697.75|11330.29|11500.83|4428.34| +2450832|3940|2450878|32960|1851088|5663|2238|74479|763341|4669|38844|2|94|15|1|16|53|911|42|99.32|188.70|164.16|1030.68|6894.72|4171.44|7925.40|68.94|0.00|2615.34|6894.72|6963.66|9510.06|9579.00|2723.28| +2450832|3940|2450839|32960|1851088|5663|2238|74479|763341|4669|38844|2|17|6|2|17666|276|911|9|50.17|147.49|106.19|371.70|955.71|451.53|1327.41|47.78|0.00|185.76|955.71|1003.49|1141.47|1189.25|504.18| +2450832|45699|2450839|60572|578785|1445|30458|60572|578785|1445|30458|2|82|2|5|3824|121|912|1|81.85|244.73|124.81|119.92|124.81|81.85|244.73|5.99|49.92|0.00|74.89|80.88|74.89|80.88|-6.96| +2450832|45699|2450843|60572|578785|1445|30458|60572|578785|1445|30458|2|32|18|1|6488|80|912|76|98.19|108.99|58.85|3810.64|4472.60|7462.44|8283.24|402.53|0.00|744.80|4472.60|4875.13|5217.40|5619.93|-2989.84| +2450832|45699|2450886|60572|578785|1445|30458|60572|578785|1445|30458|2|41|16|4|2608|295|912|16|51.05|77.08|0.00|1233.28|0.00|816.80|1233.28|0.00|0.00|443.84|0.00|0.00|443.84|443.84|-816.80| +2450832|45699|2450849|60572|578785|1445|30458|60572|578785|1445|30458|2|75|11|3|187|175|912|40|72.63|201.91|34.32|6703.60|1372.80|2905.20|8076.40|82.36|0.00|3068.80|1372.80|1455.16|4441.60|4523.96|-1532.40| +2450832|35680|2450903|84815|1098986|710|41281|84815|1098986|710|41281|1|92|11|2|13927|282|913|65|55.08|126.68|30.40|6258.20|1976.00|3580.20|8234.20|118.56|0.00|247.00|1976.00|2094.56|2223.00|2341.56|-1604.20| +2450832|35680|2450865|84815|1098986|710|41281|84815|1098986|710|41281|1|52|2|2|12860|255|913|87|77.43|218.35|163.76|4749.33|14247.12|6736.41|18996.45|142.47|0.00|4558.80|14247.12|14389.59|18805.92|18948.39|7510.71| +2450832|35680|2450847|84815|1098986|710|41281|84815|1098986|710|41281|1|23|8|4|16976|226|913|84|57.57|112.83|108.31|379.68|9098.04|4835.88|9477.72|181.96|0.00|2368.80|9098.04|9280.00|11466.84|11648.80|4262.16| +2450832|35680|2450877|84815|1098986|710|41281|84815|1098986|710|41281|1|16|18|4|9823|164|913|69|5.29|12.00|8.52|240.12|587.88|365.01|828.00|35.27|0.00|91.08|587.88|623.15|678.96|714.23|222.87| +2450832|35680|2450911|84815|1098986|710|41281|84815|1098986|710|41281|1|54|13|5|3956|286|913|61|73.60|77.28|69.55|471.53|4242.55|4489.60|4714.08|169.70|0.00|518.50|4242.55|4412.25|4761.05|4930.75|-247.05| +2450832|45605|2450834|66151|1578428|5552|30229|66151|1578428|5552|30229|2|59|16|3|9530|44|914|33|19.95|59.05|47.24|389.73|1558.92|658.35|1948.65|62.35|0.00|642.84|1558.92|1621.27|2201.76|2264.11|900.57| +2450832|45605|2450911|66151|1578428|5552|30229|66151|1578428|5552|30229|2|37|16|4|14332|271|914|34|1.76|4.52|3.88|21.76|131.92|59.84|153.68|9.23|0.00|59.84|131.92|141.15|191.76|200.99|72.08| +2450832|45605|2450888|66151|1578428|5552|30229|66151|1578428|5552|30229|2|13|13|5|2659|203|914|91|40.58|79.13|15.03|5833.10|1367.73|3692.78|7200.83|123.09|0.00|1079.26|1367.73|1490.82|2446.99|2570.08|-2325.05| +2450832|45605|2450873|66151|1578428|5552|30229|66151|1578428|5552|30229|2|70|17|1|847|36|914|42|88.99|162.85|136.79|1094.52|5745.18|3737.58|6839.70|517.06|0.00|1709.82|5745.18|6262.24|7455.00|7972.06|2007.60| +2450832|45605|2450904|66151|1578428|5552|30229|66151|1578428|5552|30229|2|18|12|5|16897|172|914|8|99.67|106.64|24.52|656.96|196.16|797.36|853.12|13.73|0.00|213.28|196.16|209.89|409.44|423.17|-601.20| +2450832|45605|2450835|66151|1578428|5552|30229|66151|1578428|5552|30229|2|24|13|3|61|43|914|83|9.47|19.12|13.19|492.19|1094.77|786.01|1586.96|57.80|131.37|522.90|963.40|1021.20|1486.30|1544.10|177.39| +2450832|12018|2450866|40296|779212|5328|5844|40296|779212|5328|5844|4|6|16|5|4279|6|915|72|57.16|85.74|14.57|5124.24|1049.04|4115.52|6173.28|66.08|104.90|2098.80|944.14|1010.22|3042.94|3109.02|-3171.38| +2450832|12018|2450843|40296|779212|5328|5844|40296|779212|5328|5844|4|43|18|5|12772|227|915|88|81.41|148.16|109.63|3390.64|9647.44|7164.08|13038.08|509.38|3280.12|5605.60|6367.32|6876.70|11972.92|12482.30|-796.76| +2450832|12018|2450882|40296|779212|5328|5844|40296|779212|5328|5844|4|105|20|5|8995|192|915|97|98.98|145.50|100.39|4375.67|9737.83|9601.06|14113.50|779.02|0.00|987.46|9737.83|10516.85|10725.29|11504.31|136.77| +2450832|12018|2450840|40296|779212|5328|5844|40296|779212|5328|5844|4|85|2|5|8228|78|915|9|23.86|68.71|39.16|265.95|352.44|214.74|618.39|0.00|0.00|105.12|352.44|352.44|457.56|457.56|137.70| +2450832|12018|2450912|40296|779212|5328|5844|40296|779212|5328|5844|4|103|14|1|6908|280|915|73|99.98|201.95|177.71|1769.52|12972.83|7298.54|14742.35|129.72|0.00|3685.04|12972.83|13102.55|16657.87|16787.59|5674.29| +2450832|12018|2450840|40296|779212|5328|5844|40296|779212|5328|5844|4|57|7|3|8066|203|915|95|81.11|206.01|57.68|14091.35|5479.60|7705.45|19570.95|438.36|0.00|8806.50|5479.60|5917.96|14286.10|14724.46|-2225.85| +2450832|12018|2450918|40296|779212|5328|5844|40296|779212|5328|5844|4|24|4|1|170|227|915|5|75.26|78.27|54.78|117.45|273.90|376.30|391.35|8.21|0.00|46.95|273.90|282.11|320.85|329.06|-102.40| +2450832|12018|2450862|40296|779212|5328|5844|40296|779212|5328|5844|4|7|17|5|14378|241|915|54|96.84|242.10|89.57|8236.62|4836.78|5229.36|13073.40|338.57|0.00|6013.44|4836.78|5175.35|10850.22|11188.79|-392.58| +2450832|12018|2450881|40296|779212|5328|5844|40296|779212|5328|5844|4|15|19|1|602|202|915|21|22.92|33.00|16.50|346.50|346.50|481.32|693.00|10.39|0.00|339.57|346.50|356.89|686.07|696.46|-134.82| +2450832|12018|2450912|40296|779212|5328|5844|40296|779212|5328|5844|4|11|8|4|6350|19|915|93|4.42|9.37|0.00|871.41|0.00|411.06|871.41|0.00|0.00|261.33|0.00|0.00|261.33|261.33|-411.06| +2450832|12018|2450901|40296|779212|5328|5844|40296|779212|5328|5844|4|12|13|1|4726|232|915|13|13.45|27.84|6.96|271.44|90.48|174.85|361.92|4.52|0.00|86.84|90.48|95.00|177.32|181.84|-84.37| +2450832|12018|2450853|40296|779212|5328|5844|40296|779212|5328|5844|4|6|3|2|13771|184|915|63|81.88|203.88|118.25|5394.69|7449.75|5158.44|12844.44|74.49|0.00|1027.53|7449.75|7524.24|8477.28|8551.77|2291.31| +2450832|12018|2450870|40296|779212|5328|5844|40296|779212|5328|5844|4|43|5|1|5824|60|915|97|85.94|124.61|79.75|4351.42|7735.75|8336.18|12087.17|464.14|0.00|361.81|7735.75|8199.89|8097.56|8561.70|-600.43| +2450832|12018|2450901|40296|779212|5328|5844|40296|779212|5328|5844|4|9|8|2|14300|45|915|36|84.49|238.26|119.13|4288.68|4288.68|3041.64|8577.36|0.00|0.00|943.20|4288.68|4288.68|5231.88|5231.88|1247.04| +2450832|55340|2450882|44797|1578884|6924|8611|7169|45185|4872|4912|2|31|20|4|7807|270|916|20|96.29|112.65|36.04|1532.20|720.80|1925.80|2253.00|28.83|0.00|1036.20|720.80|749.63|1757.00|1785.83|-1205.00| +2450832|55340|2450897|44797|1578884|6924|8611|7169|45185|4872|4912|2|30|10|4|12736|79|916|54|88.45|173.36|60.67|6085.26|3276.18|4776.30|9361.44|98.28|0.00|655.02|3276.18|3374.46|3931.20|4029.48|-1500.12| +2450832|55340|2450921|44797|1578884|6924|8611|7169|45185|4872|4912|2|67|19|4|8678|199|916|80|98.44|137.81|107.49|2425.60|8599.20|7875.20|11024.80|171.98|0.00|5181.60|8599.20|8771.18|13780.80|13952.78|724.00| +2450832|55340|2450877|44797|1578884|6924|8611|7169|45185|4872|4912|2|91|4|3|16261|88|916|26|38.32|72.42|26.79|1186.38|696.54|996.32|1882.92|0.00|0.00|771.94|696.54|696.54|1468.48|1468.48|-299.78| +2450832|55340|2450915|44797|1578884|6924|8611|7169|45185|4872|4912|2|93|4|4|17594|284|916|94|83.85|219.68|15.37|19205.14|1444.78|7881.90|20649.92|86.68|0.00|8672.44|1444.78|1531.46|10117.22|10203.90|-6437.12| +2450832|55340|2450837|44797|1578884|6924|8611|7169|45185|4872|4912|2|63|10|5|1087|146|916|24|96.98|277.36|113.71|3927.60|2729.04|2327.52|6656.64|0.00|0.00|931.92|2729.04|2729.04|3660.96|3660.96|401.52| +2450832|55340|2450880|44797|1578884|6924|8611|7169|45185|4872|4912|2|44|3|5|8|116|916|84|57.48|58.05|46.44|975.24|3900.96|4828.32|4876.20|234.05|0.00|194.88|3900.96|4135.01|4095.84|4329.89|-927.36| +2450832|55340|2450836|44797|1578884|6924|8611|7169|45185|4872|4912|2|65|5|3|10051|240|916|70|1.78|3.80|0.38|239.40|26.60|124.60|266.00|2.39|0.00|133.00|26.60|28.99|159.60|161.99|-98.00| +2450832|55340|2450893|44797|1578884|6924|8611|7169|45185|4872|4912|2|77|2|4|17716|125|916|39|5.03|12.87|1.28|452.01|49.92|196.17|501.93|3.49|0.00|19.89|49.92|53.41|69.81|73.30|-146.25| +2450832|55340|2450848|44797|1578884|6924|8611|7169|45185|4872|4912|2|48|11|3|2827|20|916|78|51.59|111.95|111.95|0.00|8732.10|4024.02|8732.10|0.00|0.00|2968.68|8732.10|8732.10|11700.78|11700.78|4708.08| +2450832|72556||||6587|15641|||6587|15641|1||2||14810||917|||54.58||851.76||1163.37|2128.62||||||1404.39|1468.23|113.49| +2450832|72556|2450859|50526|318380|6587|15641|50526|318380|6587|15641|1|9|19|3|11209|133|917|54|77.95|203.44|122.06|4394.52|6591.24|4209.30|10985.76|395.47|0.00|3515.40|6591.24|6986.71|10106.64|10502.11|2381.94| +2450832|72556|2450894|50526|318380|6587|15641|50526|318380|6587|15641|1|66|5|3|6998|209|917|17|62.01|62.01|60.76|21.25|1032.92|1054.17|1054.17|92.96|0.00|263.50|1032.92|1125.88|1296.42|1389.38|-21.25| +2450832|72556|2450904|50526|318380|6587|15641|50526|318380|6587|15641|1|23|10|1|17816|118|917|11|19.32|30.33|12.43|196.90|136.73|212.52|333.63|9.57|0.00|59.95|136.73|146.30|196.68|206.25|-75.79| +2450832|72556|2450837|50526|318380|6587|15641|50526|318380|6587|15641|1|66|10|2|16082|58|917|2|27.32|50.54|29.31|42.46|58.62|54.64|101.08|1.75|0.00|2.02|58.62|60.37|60.64|62.39|3.98| +2450832|72556|2450894|50526|318380|6587|15641|50526|318380|6587|15641|1|84|8|5|368|9|917|99|39.75|106.53|8.52|9702.99|843.48|3935.25|10546.47|2.53|792.87|2847.24|50.61|53.14|2897.85|2900.38|-3884.64| +2450832|60300|2450897|8610|996035|6213|21191|91315|1507947|972|26750|1|64|14|1|7292|130|918|13|70.32|208.14|101.98|1380.08|1325.74|914.16|2705.82|0.00|0.00|919.88|1325.74|1325.74|2245.62|2245.62|411.58| +2450832|60300|2450899|8610|996035|6213|21191|91315|1507947|972|26750|1|35|2|4|3643|246|918|40|82.79|248.37|183.79|2583.20|7351.60|3311.60|9934.80|0.00|0.00|1291.20|7351.60|7351.60|8642.80|8642.80|4040.00| +2450832|60300|2450884|8610|996035|6213|21191|91315|1507947|972|26750|1|53|12|4|3716|140|918|67|9.70|16.78|8.05|584.91|539.35|649.90|1124.26|16.18|0.00|168.17|539.35|555.53|707.52|723.70|-110.55| +2450832|60300|2450848|8610|996035|6213|21191|91315|1507947|972|26750|1|100|14|3|15103|244|918|4|59.03|83.23|6.65|306.32|26.60|236.12|332.92|2.39|0.00|96.52|26.60|28.99|123.12|125.51|-209.52| +2450832|60300|2450899|8610|996035|6213|21191|91315|1507947|972|26750|1|102|16|5|6811|150|918|98|11.00|29.15|7.87|2085.44|771.26|1078.00|2856.70|7.71|0.00|599.76|771.26|778.97|1371.02|1378.73|-306.74| +2450832|60300|2450880|8610|996035|6213|21191|91315|1507947|972|26750|1|58|19|5|290|149|918|68|46.66|92.38|2.77|6093.48|188.36|3172.88|6281.84|13.18|0.00|376.72|188.36|201.54|565.08|578.26|-2984.52| +2450832|60300|2450858|8610|996035|6213|21191|91315|1507947|972|26750|1|67|5|4|4141|24|918|99|10.85|30.16|22.01|806.85|2178.99|1074.15|2985.84|65.36|0.00|686.07|2178.99|2244.35|2865.06|2930.42|1104.84| +2450832|60300|2450874|8610|996035|6213|21191|91315|1507947|972|26750|1|15|19|1|9184|85|918|55|6.23|14.82|3.70|611.60|203.50|342.65|815.10|14.24|0.00|187.00|203.50|217.74|390.50|404.74|-139.15| +2450832|60300|2450917|8610|996035|6213|21191|91315|1507947|972|26750|1|86|6|3|1633|60|918|74|41.21|74.17|26.70|3512.78|1975.80|3049.54|5488.58|59.27|0.00|1097.42|1975.80|2035.07|3073.22|3132.49|-1073.74| +2450832|60300|2450890|8610|996035|6213|21191|91315|1507947|972|26750|1|33|9|2|17995|18|918|17|92.24|123.60|0.00|2101.20|0.00|1568.08|2101.20|0.00|0.00|1029.52|0.00|0.00|1029.52|1029.52|-1568.08| +2450832|60300|2450836|8610|996035|6213|21191|91315|1507947|972|26750|1|39|4|3|1844|271|918|69|29.08|56.99|37.61|1337.22|2595.09|2006.52|3932.31|103.80|0.00|1572.51|2595.09|2698.89|4167.60|4271.40|588.57| +2450832|60300|2450841|8610|996035|6213|21191|91315|1507947|972|26750|1|46|11|5|8806|28|918|67|82.20|140.56|16.86|8287.90|1129.62|5507.40|9417.52|67.77|0.00|4237.75|1129.62|1197.39|5367.37|5435.14|-4377.78| +2450832|60300|2450868|8610|996035|6213|21191|91315|1507947|972|26750|1|10|3|5|16135|252|918|7|23.48|55.88|41.91|97.79|293.37|164.36|391.16|4.10|88.01|23.45|205.36|209.46|228.81|232.91|41.00| +2450832|60300|2450919|8610|996035|6213|21191|91315|1507947|972|26750|1|50|19|5|3152|48|918|100|65.08|155.54|41.99|11355.00|4199.00|6508.00|15554.00|251.94|0.00|6377.00|4199.00|4450.94|10576.00|10827.94|-2309.00| +2450832|61371|2450864|95990|1834161|3179|43350|95990|1834161|3179|43350|1|55|17|4|11060|98|919|81|78.49|179.74|176.14|291.60|14267.34|6357.69|14558.94|998.71|0.00|5240.70|14267.34|15266.05|19508.04|20506.75|7909.65| +2450832|61371|2450853|95990|1834161|3179|43350|95990|1834161|3179|43350|1|49|10|2|445|164|919|33|68.96|122.74|31.91|2997.39|1053.03|2275.68|4050.42|94.77|0.00|607.53|1053.03|1147.80|1660.56|1755.33|-1222.65| +2450832|61371|2450911|95990|1834161|3179|43350|95990|1834161|3179|43350|1|93|10|2|15997|298|919|46|7.11|14.43|3.46|504.62|159.16|327.06|663.78|14.32|0.00|205.62|159.16|173.48|364.78|379.10|-167.90| +2450832|61371|2450838|95990|1834161|3179|43350|95990|1834161|3179|43350|1|69|1|4|12376|105|919|35|64.18|121.94|20.72|3542.70|725.20|2246.30|4267.90|36.26|0.00|1578.85|725.20|761.46|2304.05|2340.31|-1521.10| +2450832|61371|2450878|95990|1834161|3179|43350|95990|1834161|3179|43350|1|64|6|4|4442|206|919|31|52.87|132.70|95.54|1151.96|2961.74|1638.97|4113.70|266.55|0.00|740.28|2961.74|3228.29|3702.02|3968.57|1322.77| +2450832|61371|2450863|95990|1834161|3179|43350|95990|1834161|3179|43350|1|38|15|5|4531|53|919|18|73.62|97.17|49.55|857.16|891.90|1325.16|1749.06|62.43|0.00|647.10|891.90|954.33|1539.00|1601.43|-433.26| +2450832|61371|2450889|95990|1834161|3179|43350|95990|1834161|3179|43350|1|25|18|1|4003|17|919|72|19.43|24.87|6.71|1307.52|483.12|1398.96|1790.64|38.64|0.00|554.40|483.12|521.76|1037.52|1076.16|-915.84| +2450832|61371|2450866|95990|1834161|3179|43350|95990|1834161|3179|43350|1|70|9|1|11194|205|919|65|77.09|116.40|77.98|2497.30|5068.70|5010.85|7566.00|152.06|0.00|3026.40|5068.70|5220.76|8095.10|8247.16|57.85| +2450832|61371|2450853|95990|1834161|3179|43350|95990|1834161|3179|43350|1|14|14|3|9694|52|919|69|18.42|41.62|12.48|2010.66|861.12|1270.98|2871.78|68.88|0.00|171.81|861.12|930.00|1032.93|1101.81|-409.86| +2450832|61371|2450922|95990|1834161|3179|43350|95990|1834161|3179|43350|1|91|15|2|5461|268|919|3|21.57|31.70|1.58|90.36|4.74|64.71|95.10|0.18|0.00|2.85|4.74|4.92|7.59|7.77|-59.97| +2450832|61371|2450868|95990|1834161|3179|43350|95990|1834161|3179|43350|1|91|11|5|10453|26|919|78|18.24|52.16|50.07|163.02|3905.46|1422.72|4068.48|273.38|0.00|854.10|3905.46|4178.84|4759.56|5032.94|2482.74| +2450832|61371|2450841|95990|1834161|3179|43350|95990|1834161|3179|43350|1|74|1|5|5719|130|919|99|46.07|96.74|84.16|1245.42|8331.84|4560.93|9577.26|749.86|0.00|3543.21|8331.84|9081.70|11875.05|12624.91|3770.91| +2450832|61371|2450903|95990|1834161|3179|43350|95990|1834161|3179|43350|1|92|5|5|16831|92|919|38|61.55|174.18|66.18|4104.00|2514.84|2338.90|6618.84|3.52|2338.80|2647.46|176.04|179.56|2823.50|2827.02|-2162.86| +2450832|61371|2450891|95990|1834161|3179|43350|95990|1834161|3179|43350|1|14|12|5|14632|224|919|94|30.04|84.71|20.33|6051.72|1911.02|2823.76|7962.74|114.66|0.00|0.00|1911.02|2025.68|1911.02|2025.68|-912.74| +2450833|32684|2450920|44188|1651223|5811|37646|44188|1651223|5811|37646|4|104|6|3|16202|6|920|68|82.19|83.01|13.28|4741.64|903.04|5588.92|5644.68|81.27|0.00|620.84|903.04|984.31|1523.88|1605.15|-4685.88| +2450833|32684|2450865|44188|1651223|5811|37646|44188|1651223|5811|37646|4|70|11|2|3820|169|920|46|73.15|143.37|130.46|593.86|6001.16|3364.90|6595.02|420.08|0.00|1714.42|6001.16|6421.24|7715.58|8135.66|2636.26| +2450833|32684|2450918|44188|1651223|5811|37646|44188|1651223|5811|37646|4|60|9|5|9211|144|920|99|10.02|12.32|6.89|537.57|682.11|991.98|1219.68|40.92|0.00|450.45|682.11|723.03|1132.56|1173.48|-309.87| +2450833|32684|2450888|44188|1651223|5811|37646|44188|1651223|5811|37646|4|79|17|4|17918|15|920|44|67.28|111.01|108.78|98.12|4786.32|2960.32|4884.44|95.72|0.00|2148.96|4786.32|4882.04|6935.28|7031.00|1826.00| +2450833|32684|2450840|44188|1651223|5811|37646|44188|1651223|5811|37646|4|100|6|3|11803|159|920|63|82.14|195.49|37.14|9976.05|2339.82|5174.82|12315.87|116.99|0.00|1970.01|2339.82|2456.81|4309.83|4426.82|-2835.00| +2450833|32684|2450917|44188|1651223|5811|37646|44188|1651223|5811|37646|4|8|13|2|6244|84|920|35|70.11|112.87|27.08|3002.65|947.80|2453.85|3950.45|18.95|0.00|631.75|947.80|966.75|1579.55|1598.50|-1506.05| +2450833|32684|2450852|44188|1651223|5811|37646|44188|1651223|5811|37646|4|17|6|2|7924|155|920|30|98.66|278.22|0.00|8346.60|0.00|2959.80|8346.60|0.00|0.00|750.90|0.00|0.00|750.90|750.90|-2959.80| +2450833|37927|2450838|67363|126060|1557|49454|67363|126060|1557|49454|1|8|18|4|12535|151|921|5|82.19|217.80|156.81|304.95|784.05|410.95|1089.00|70.56|0.00|98.00|784.05|854.61|882.05|952.61|373.10| +2450833|37927|2450855|67363|126060|1557|49454|67363|126060|1557|49454|1|73|11|2|13612|70|921|82|1.17|3.45|3.13|26.24|256.66|95.94|282.90|5.95|182.22|41.82|74.44|80.39|116.26|122.21|-21.50| +2450833|37927|2450837|67363|126060|1557|49454|67363|126060|1557|49454|1|103|6|4|5764|88|921|89|80.59|209.53|178.10|2797.27|15850.90|7172.51|18648.17|1426.58|0.00|6340.36|15850.90|17277.48|22191.26|23617.84|8678.39| +2450833|37927|2450841|67363|126060|1557|49454|67363|126060|1557|49454|1|42|12|5|13444|289|921|56|49.90|110.27|15.43|5311.04|864.08|2794.40|6175.12|51.84|0.00|246.96|864.08|915.92|1111.04|1162.88|-1930.32| +2450833|37927|2450916|67363|126060|1557|49454|67363|126060|1557|49454|1|104|18|3|5173|153|921|49|4.93|6.26|5.38|43.12|263.62|241.57|306.74|0.00|118.62|49.00|145.00|145.00|194.00|194.00|-96.57| +2450833|37927|2450886|67363|126060|1557||||||1||5|2|8698|71|921|62||149.76|131.78|||3917.78|9285.12|653.62||649.76||8823.98|8820.12|9473.74|4252.58| +2450833|37927|2450877|67363|126060|1557|49454|67363|126060|1557|49454|1|16|6|3|15100|147|921|71|20.08|24.29|8.98|1087.01|637.58|1425.68|1724.59|57.38|0.00|68.87|637.58|694.96|706.45|763.83|-788.10| +2450833|63018|2450866|10395|138483|2571|47412|10395|138483|2571|47412|1|72|5|2|3100|111|922|77|73.26|183.15|73.26|8461.53|5641.02|5641.02|14102.55|282.05|0.00|6909.98|5641.02|5923.07|12551.00|12833.05|0.00| +2450833|63018|2450854|10395|138483|2571|47412|10395|138483|2571|47412|1|67|19|1|14306|240|922|56|10.16|14.52|1.01|756.56|56.56|568.96|813.12|2.26|0.00|235.76|56.56|58.82|292.32|294.58|-512.40| +2450833|63018|2450859|10395|138483|2571|47412|10395|138483|2571|47412|1|26|9|2|15592|183|922|69|47.71|76.81|44.54|2226.63|3073.26|3291.99|5299.89|276.59|0.00|2649.60|3073.26|3349.85|5722.86|5999.45|-218.73| +2450833|63018|2450898|10395|138483|2571|47412|10395|138483|2571|47412|1|23|9|5|6338|130|922|85|6.53|19.52|10.93|730.15|929.05|555.05|1659.20|74.32|0.00|729.30|929.05|1003.37|1658.35|1732.67|374.00| +2450833|63018|2450893|10395|138483|2571|47412|10395|138483|2571|47412|1|2|17|3|7126|211|922|91|10.21|15.72|5.81|901.81|528.71|929.11|1430.52|26.43|0.00|300.30|528.71|555.14|829.01|855.44|-400.40| +2450833|63018|2450884|10395|138483|2571|47412|10395|138483|2571|47412|1|90|14|3|12208|14|922|63|97.90|128.24|10.25|7433.37|645.75|6167.70|8079.12|5.94|51.66|3473.82|594.09|600.03|4067.91|4073.85|-5573.61| +2450833|63018|2450919|10395|138483|2571|47412|10395|138483|2571|47412|1|71|20|4|13742|218|922|15|5.58|16.18|0.32|237.90|4.80|83.70|242.70|0.04|0.00|70.35|4.80|4.84|75.15|75.19|-78.90| +2450833|63018|2450919|10395|138483|2571|47412|10395|138483|2571|47412|1|31|14|4|8102|177|922|5|47.20|133.10|118.45|73.25|592.25|236.00|665.50|47.38|0.00|173.00|592.25|639.63|765.25|812.63|356.25| +2450833|63018|2450854|10395|138483|2571|47412|10395|138483|2571|47412|1|61|1|4|9784|143|922|44|96.93|170.59|25.58|6380.44|1125.52|4264.92|7505.96|101.29|0.00|3302.20|1125.52|1226.81|4427.72|4529.01|-3139.40| +2450833|63018|2450919|10395|138483|2571|47412|10395|138483|2571|47412|1|98|16|3|2647|179|922|8|86.54|124.61|108.41|129.60|867.28|692.32|996.88|60.70|0.00|348.88|867.28|927.98|1216.16|1276.86|174.96| +2450833|74846|2450869|20004|982448|2139|37030|20004|982448|2139|37030|2|77|5|1|8020|5|923|69|39.37|65.35|62.73|180.78|4328.37|2716.53|4509.15|0.00|0.00|315.33|4328.37|4328.37|4643.70|4643.70|1611.84| +2450833|74846|2450877|20004|982448|2139|37030|20004|982448|2139|37030|2|71|20|4|2480|219|923|32|95.11|211.14|86.56|3986.56|2769.92|3043.52|6756.48|110.79|0.00|2702.40|2769.92|2880.71|5472.32|5583.11|-273.60| +2450833|74846|2450837|20004|982448|2139|37030|20004|982448|2139|37030|2|16|8|2|1957|290|923|27|66.94|87.02|67.00|540.54|1809.00|1807.38|2349.54|108.54|0.00|869.13|1809.00|1917.54|2678.13|2786.67|1.62| +2450833|74846|2450847|20004|982448|2139|37030|20004|982448|2139|37030|2|101|15|3|15484|212|923|90|96.95|164.81|161.51|297.00|14535.90|8725.50|14832.90|1162.87|0.00|4745.70|14535.90|15698.77|19281.60|20444.47|5810.40| +2450833|74846|2450887|20004|982448|2139|37030|20004|982448|2139|37030|2|56|17|4|16816|210|923|66|34.78|70.95|16.31|3606.24|1076.46|2295.48|4682.70|43.05|0.00|608.52|1076.46|1119.51|1684.98|1728.03|-1219.02| +2450833|74846|2450860|20004|982448|2139|37030|20004|982448|2139|37030|2|108|16|5|10624|170|923|72|84.43|114.82|13.77|7275.60|991.44|6078.96|8267.04|39.65|0.00|82.08|991.44|1031.09|1073.52|1113.17|-5087.52| +2450833|74846|2450839|20004|982448|2139|37030|20004|982448|2139|37030|2|54|3|5|4198|194|923|31|69.22|96.90|38.76|1802.34|1201.56|2145.82|3003.90|36.04|0.00|1291.46|1201.56|1237.60|2493.02|2529.06|-944.26| +2450833|74846|2450855|20004|982448|2139|37030|20004|982448|2139|37030|2|6|2|3|206|13|923|28|17.77|36.60|13.54|645.68|379.12|497.56|1024.80|0.00|0.00|245.84|379.12|379.12|624.96|624.96|-118.44| +2450833|74846|2450893|20004|982448|2139|37030|20004|982448|2139|37030|2|1|6|5|17182|266|923|57|43.14|128.12|70.46|3286.62|4016.22|2458.98|7302.84|0.00|0.00|2774.76|4016.22|4016.22|6790.98|6790.98|1557.24| +2450833|74846|2450873|20004|982448|2139|37030|20004|982448|2139|37030|2|35|17|2|3175|190|923|98|46.66|112.91|102.74|996.66|10068.52|4572.68|11065.18|906.16|0.00|774.20|10068.52|10974.68|10842.72|11748.88|5495.84| +2450833|74846|2450844|20004|982448|2139|37030|20004|982448|2139|37030|2|46|18|5|15379|81|923|43|79.34|136.46|57.31|3403.45|2464.33|3411.62|5867.78|98.57|0.00|1877.38|2464.33|2562.90|4341.71|4440.28|-947.29| +2450833|74846|2450908|20004|982448|2139|37030|20004|982448|2139|37030|2|14|17|5|124|46|923|57|64.29|141.43|76.37|3708.42|4353.09|3664.53|8061.51|391.77|0.00|1611.96|4353.09|4744.86|5965.05|6356.82|688.56| +2450833|74846|2450891|20004|982448|2139|37030|20004|982448|2139|37030|2|15|17|1|4582|124|923|65|96.21|142.39|75.46|4350.45|4904.90|6253.65|9255.35|49.04|0.00|2498.60|4904.90|4953.94|7403.50|7452.54|-1348.75| +2450833|74846|2450838|20004|982448|2139|37030|20004|982448|2139|37030|2|73|14|1|10736|267|923|62|35.49|41.16|30.04|689.44|1862.48|2200.38|2551.92|55.87|0.00|663.40|1862.48|1918.35|2525.88|2581.75|-337.90| +2450833|35259|2450893|70212|894583|107|20431|88921|1654726|1850|26174|4|44|14|2|10142|78|924|4|81.59|179.49|113.07|265.68|452.28|326.36|717.96|36.18|0.00|107.68|452.28|488.46|559.96|596.14|125.92| +2450833|35259|2450852|70212|894583|107|20431|88921|1654726|1850|26174|4|5|6|2|1927|99|924|91|18.53|53.73|51.58|195.65|4693.78|1686.23|4889.43|233.75|797.94|1319.50|3895.84|4129.59|5215.34|5449.09|2209.61| +2450833|35259|2450896|70212|894583|107|20431|88921|1654726|1850|26174|4|38|16|1|15128|3|924|84|96.80|172.30|122.33|4197.48|10275.72|8131.20|14473.20|71.93|9248.14|2460.36|1027.58|1099.51|3487.94|3559.87|-7103.62| +2450833|35259|2450909|70212|894583|107|20431|88921|1654726|1850|26174|4|108|16|2|5977|227|924|51|65.50|161.78|124.57|1897.71|6353.07|3340.50|8250.78|111.17|4764.80|2062.44|1588.27|1699.44|3650.71|3761.88|-1752.23| +2450833|35259|2450890|70212|894583|107|20431|88921|1654726|1850|26174|4|20|14|1|6907|102|924|86|35.84|104.29|21.90|7085.54|1883.40|3082.24|8968.94|145.77|263.67|2062.28|1619.73|1765.50|3682.01|3827.78|-1462.51| +2450833|64763|2450891|37625|751981|7137|48693|37625|751981|7137|48693|1|103|10|5|8240|289|925|67|98.49|141.82|95.01|3136.27|6365.67|6598.83|9501.94|381.94|0.00|950.06|6365.67|6747.61|7315.73|7697.67|-233.16| +2450833|64763|2450874|37625|751981|7137|48693|37625|751981|7137|48693|1|22|10|1|13645|188|925|88|29.78|39.60|22.57|1498.64|1986.16|2620.64|3484.80|29.79|1489.62|208.56|496.54|526.33|705.10|734.89|-2124.10| +2450833|64763|2450903|37625|751981|7137|48693|37625|751981|7137|48693|1|98|3|4|12262|166|925|97|29.28|55.63|30.59|2428.88|2967.23|2840.16|5396.11|0.00|0.00|2158.25|2967.23|2967.23|5125.48|5125.48|127.07| +2450833|64763|2450906|37625|751981|7137|48693|37625|751981|7137|48693|1|69|13|5|17749|244|925|61|25.79|65.24|22.83|2587.01|1392.63|1573.19|3979.64|13.36|724.16|954.65|668.47|681.83|1623.12|1636.48|-904.72| +2450833|64763|2450858|37625|751981|7137|48693|37625|751981|7137|48693|1|61|11|5|10828|293|925|81|60.48|129.42|1.29|10378.53|104.49|4898.88|10483.02|8.35|0.00|1048.14|104.49|112.84|1152.63|1160.98|-4794.39| +2450833|64763|2450850|37625|751981|7137|48693|37625|751981|7137|48693|1|64|9|5|8071|129|925|65|87.12|139.39|73.87|4258.80|4801.55|5662.80|9060.35|192.06|0.00|4529.85|4801.55|4993.61|9331.40|9523.46|-861.25| +2450833|64763|2450849|37625|751981|7137|48693|37625|751981|7137|48693|1|25|11|4|4951|177|925|76|41.75|53.44|27.25|1990.44|2071.00|3173.00|4061.44|144.97|0.00|649.80|2071.00|2215.97|2720.80|2865.77|-1102.00| +2450833|64763|2450910|37625|751981|7137|48693|37625|751981|7137|48693|1|71|9|5|3998|20|925|61|35.08|90.15|5.40|5169.75|329.40|2139.88|5499.15|5.53|191.05|1924.55|138.35|143.88|2062.90|2068.43|-2001.53| +2450833|64763|2450920|37625|751981|7137|48693|37625|751981|7137|48693|1|24|8|2|7498|72|925|67|29.21|33.00|9.24|1591.92|619.08|1957.07|2211.00|0.00|92.86|1061.28|526.22|526.22|1587.50|1587.50|-1430.85| +2450833|64763|2450922|37625|751981|7137|48693|37625|751981|7137|48693|1|69|12|1|10922|210|925|92|43.95|77.79|38.89|3578.80|3577.88|4043.40|7156.68|250.45|0.00|0.00|3577.88|3828.33|3577.88|3828.33|-465.52| +2450833|64763|2450892|37625|751981|7137|48693|37625|751981|7137|48693|1|26|20|2|11714|26|925|94|39.98|43.57|33.11|983.24|3112.34|3758.12|4095.58|7.46|2738.85|1310.36|373.49|380.95|1683.85|1691.31|-3384.63| +2450833|64763|2450907|37625|751981|7137|48693|37625|751981|7137|48693|1|19|16|3|7436|122|925|96|30.75|37.51|33.38|396.48|3204.48|2952.00|3600.96|96.13|0.00|864.00|3204.48|3300.61|4068.48|4164.61|252.48| +2450833|64763|2450893|37625|751981|7137|48693|37625|751981|7137|48693|1|22|19|2|2713|126|925|90|48.13|77.97|71.73|561.60|6455.70|4331.70|7017.30|581.01|0.00|3438.00|6455.70|7036.71|9893.70|10474.71|2124.00| +2450833|64763|2450921|37625|751981|7137|48693|37625|751981|7137|48693|1|2|8|2|12506|293|925|61|22.95|26.85|13.42|819.23|818.62|1399.95|1637.85|73.67|0.00|802.15|818.62|892.29|1620.77|1694.44|-581.33| +2450833|33462|2450839|32771|1272970|7122|24121|32771|1272970|7122|24121|4|63|19|2|4874|80|926|81|4.10|8.15|5.29|231.66|428.49|332.10|660.15|25.70|0.00|237.33|428.49|454.19|665.82|691.52|96.39| +2450833|33462|2450873|32771|1272970|7122|24121|32771|1272970|7122|24121|4|16|1|5|6434|50|926|36|1.03|1.21|0.71|18.00|25.56|37.08|43.56|1.53|0.00|17.28|25.56|27.09|42.84|44.37|-11.52| +2450833|33462|2450850|32771|1272970|7122|24121|32771|1272970|7122|24121|4|38|15|4|12487|288|926|14|30.50|45.14|2.25|600.46|31.50|427.00|631.96|1.26|0.00|18.90|31.50|32.76|50.40|51.66|-395.50| +2450833|33462|2450891|32771|1272970|7122|24121|32771|1272970|7122|24121|4|52|13|3|12164|203|926|65|8.05|9.01|8.37|41.60|544.05|523.25|585.65|17.51|293.78|35.10|250.27|267.78|285.37|302.88|-272.98| +2450833|33462|2450885|32771|1272970|7122|24121|32771|1272970|7122|24121|4|30|13|4|13130|29|926|8|49.88|139.16|66.79|578.96|534.32|399.04|1113.28|37.40|0.00|122.40|534.32|571.72|656.72|694.12|135.28| +2450833|33462|2450867|32771|1272970|7122|24121|32771|1272970|7122|24121|4|47|2|3|2192|192|926|43|31.00|56.11|17.39|1664.96|747.77|1333.00|2412.73|67.29|0.00|1037.16|747.77|815.06|1784.93|1852.22|-585.23| +2450833|33462|2450837|32771|1272970|7122|24121|32771|1272970|7122|24121|4|8|12|3|14980|105|926|88|49.47|60.35|41.64|1646.48|3664.32|4353.36|5310.80|0.00|3664.32|1752.08|0.00|0.00|1752.08|1752.08|-4353.36| +2450833|33462|2450861|32771|1272970|7122|24121|32771|1272970|7122|24121|4|38|10|3|9296|221|926|50|24.83|50.40|45.36|252.00|2268.00|1241.50|2520.00|204.12|0.00|756.00|2268.00|2472.12|3024.00|3228.12|1026.50| +2450833|80927|2450835|75763|225590|3417|4844|75763|225590|3417|4844|4|60|2|4|13172|10|927|8|67.10|77.16|72.53|37.04|580.24|536.80|617.28|1.85|394.56|18.48|185.68|187.53|204.16|206.01|-351.12| +2450833|80927|2450864|75763|225590|3417|4844|75763|225590|3417|4844|4|103|6|4|6272|50|927|12|45.95|68.92|19.29|595.56|231.48|551.40|827.04|18.51|0.00|0.00|231.48|249.99|231.48|249.99|-319.92| +2450833|80927|2450852|75763|225590|3417|4844|75763|225590|3417|4844|4|12|12|1|10372|239|927|16|6.67|19.07|2.47|265.60|39.52|106.72|305.12|3.55|0.00|131.20|39.52|43.07|170.72|174.27|-67.20| +2450833|80927|2450920|75763|225590|3417|4844|75763|225590|3417|4844|4|98|17|1|10912|264|927|80|71.20|211.46|90.92|9643.20|7273.60|5696.00|16916.80|218.20|0.00|4905.60|7273.60|7491.80|12179.20|12397.40|1577.60| +2450833|80927|2450861|75763|225590|3417|4844|75763|225590|3417|4844|4|7|1|1|8156|236|927|31|94.56|259.09|256.49|80.60|7951.19|2931.36|8031.79|397.55|0.00|3051.95|7951.19|8348.74|11003.14|11400.69|5019.83| +2450833|80927|2450877|75763|225590|3417|4844|75763|225590|3417|4844|4|38|12|3|16336|252|927|77|73.12|114.06|10.26|7992.60|790.02|5630.24|8782.62|55.30|0.00|2897.51|790.02|845.32|3687.53|3742.83|-4840.22| +2450833|80927|2450835|75763|225590|3417|4844|75763|225590|3417|4844|4|16|18|4|14845|248|927|70|17.37|38.73|11.61|1898.40|812.70|1215.90|2711.10|24.38|0.00|26.60|812.70|837.08|839.30|863.68|-403.20| +2450833|80927|2450899|75763|225590|3417|4844|75763|225590|3417|4844|4|23|10|4|8768|120|927|12|77.70|217.56|23.93|2323.56|287.16|932.40|2610.72|6.89|57.43|574.32|229.73|236.62|804.05|810.94|-702.67| +2450833|80927|2450922|75763|225590|3417|4844|75763|225590|3417|4844|4|54|11|1|11888|19|927|65|11.60|11.94|4.77|466.05|310.05|754.00|776.10|27.90|0.00|217.10|310.05|337.95|527.15|555.05|-443.95| +2450833|80927|2450899|75763|225590|3417|4844|75763|225590|3417|4844|4|23|7|3|2632|260|927|38|55.94|120.83|100.28|780.90|3810.64|2125.72|4591.54|190.53|0.00|1423.10|3810.64|4001.17|5233.74|5424.27|1684.92| +2450833|80927|2450844|75763|225590|3417|4844|75763|225590|3417|4844|4|17|9|4|1699|199|927|36|22.84|48.19|17.34|1110.60|624.24|822.24|1734.84|18.72|0.00|433.44|624.24|642.96|1057.68|1076.40|-198.00| +2450833|59221|2450855|82845|1111015|3985|39355|82845|1111015|3985|39355|4|61|19|1|4130|208|928|15|12.00|25.68|1.79|358.35|26.85|180.00|385.20|1.61|0.00|100.05|26.85|28.46|126.90|128.51|-153.15| +2450833|59221|2450851|82845|1111015|3985|39355|82845|1111015|3985|39355|4|55|13|3|13735|223|928|18|11.15|22.85|15.30|135.90|275.40|200.70|411.30|19.27|0.00|24.66|275.40|294.67|300.06|319.33|74.70| +2450833|59221|2450845|82845|1111015|3985|39355|82845|1111015|3985|39355|4|55|5|1|15727|59|928|15|28.23|30.20|8.15|330.75|122.25|423.45|453.00|4.49|66.01|58.80|56.24|60.73|115.04|119.53|-367.21| +2450833|59221|2450900|82845|1111015|3985|39355|82845|1111015|3985|39355|4|25|13|4|10831|119|928|46|58.48|140.35|47.71|4261.44|2194.66|2690.08|6456.10|109.73|0.00|2453.18|2194.66|2304.39|4647.84|4757.57|-495.42| +2450833|59221|2450837|82845|1111015|3985|39355|82845|1111015|3985|39355|4|87|13|4|10868|151|928|82|89.28|174.98|173.23|143.50|14204.86|7320.96|14348.36|600.86|7528.57|1434.18|6676.29|7277.15|8110.47|8711.33|-644.67| +2450833|59221|2450892|82845|1111015|3985|39355|82845|1111015|3985|39355|4|43|3|1|6397|240|928|79|10.39|20.36|14.45|466.89|1141.55|820.81|1608.44|91.32|0.00|31.60|1141.55|1232.87|1173.15|1264.47|320.74| +2450833|59221|2450865|82845|1111015|3985|39355|82845|1111015|3985|39355|4|51|8|4|16765|234|928|88|73.48|199.86|49.96|13191.20|4396.48|6466.24|17587.68|131.89|0.00|0.00|4396.48|4528.37|4396.48|4528.37|-2069.76| +2450833|59221|2450835|82845|1111015|3985|39355|82845|1111015|3985|39355|4|81|3|4|451|29|928|98|57.48|59.77|57.37|235.20|5622.26|5633.04|5857.46|168.66|0.00|1464.12|5622.26|5790.92|7086.38|7255.04|-10.78| +2450833|59221|2450867|82845|1111015|3985|39355|82845|1111015|3985|39355|4|100|7|3|10874|116|928|13|21.04|54.07|43.25|140.66|562.25|273.52|702.91|3.54|522.89|168.61|39.36|42.90|207.97|211.51|-234.16| +2450833|59221|2450896|82845|1111015|3985|39355|82845|1111015|3985|39355|4|31|18|3|2986|36|928|66|60.50|98.61|45.36|3514.50|2993.76|3993.00|6508.26|269.43|0.00|2537.70|2993.76|3263.19|5531.46|5800.89|-999.24| +2450833|59221|2450892|82845|1111015|3985|39355|82845|1111015|3985|39355|4|75|10|4|2926|105|928|82|72.47|92.03|17.48|6113.10|1433.36|5942.54|7546.46|26.66|100.33|3772.82|1333.03|1359.69|5105.85|5132.51|-4609.51| +2450833|61641|2450859|43664|606314|7011|13661|43664|606314|7011|13661|2|13|6|5|13952|203|929|95|24.67|28.61|9.72|1794.55|923.40|2343.65|2717.95|18.46|0.00|244.15|923.40|941.86|1167.55|1186.01|-1420.25| +2450833|61641|2450843|43664|606314|7011|13661|43664|606314|7011|13661|2|80|20|3|10348|22|929|23|38.49|101.22|81.98|442.52|1885.54|885.27|2328.06|0.00|1885.54|1070.88|0.00|0.00|1070.88|1070.88|-885.27| +2450833|61641|2450857|43664|606314|7011|13661|43664|606314|7011|13661|2|16|13|1|13819|160|929|95|93.22|183.64|148.74|3315.50|14130.30|8855.90|17445.80|565.21|0.00|6802.95|14130.30|14695.51|20933.25|21498.46|5274.40| +2450833|61641|2450922|43664|606314|7011|13661|43664|606314|7011|13661|2|10|9|2|6049|193|929|69|63.12|169.16|155.62|934.26|10737.78|4355.28|11672.04|859.02|0.00|4084.80|10737.78|11596.80|14822.58|15681.60|6382.50| +2450833|61641|2450895|43664|606314|7011|13661|43664|606314|7011|13661|2|70|19|4|17026|118|929|32|99.14|204.22|79.64|3986.56|2548.48|3172.48|6535.04|76.45|0.00|1045.44|2548.48|2624.93|3593.92|3670.37|-624.00| +2450833|61641|2450920|43664|606314|7011|13661|43664|606314|7011|13661|2|6|12|1|13016|262|929|73|51.18|148.93|99.78|3587.95|7283.94|3736.14|10871.89|145.67|0.00|2826.56|7283.94|7429.61|10110.50|10256.17|3547.80| +2450833|61641|2450912|43664|606314|7011|13661|43664|606314|7011|13661|2|28|4|5|10699|22|929|76|72.51|72.51|23.20|3747.56|1763.20|5510.76|5510.76|158.68|0.00|2204.00|1763.20|1921.88|3967.20|4125.88|-3747.56| +2450833|61641|2450856|43664|606314|7011|13661|43664|606314|7011|13661|2|87|15|3|836|92|929|48|7.54|9.04|8.22|39.36|394.56|361.92|433.92|11.83|0.00|25.92|394.56|406.39|420.48|432.31|32.64| +2450833|61641|2450836|43664|606314|7011|13661|43664|606314|7011|13661|2|83|20|4|2629|180|929|41|90.27|194.08|194.08|0.00|7957.28|3701.07|7957.28|557.00|0.00|2625.64|7957.28|8514.28|10582.92|11139.92|4256.21| +2450833|61641|2450845|43664|606314|7011|13661|43664|606314|7011|13661|2|78|6|5|10226|116|929|64|13.45|39.13|20.73|1177.60|1326.72|860.80|2504.32|92.87|0.00|625.92|1326.72|1419.59|1952.64|2045.51|465.92| +2450833|61641|2450920|43664|606314|7011|13661|43664|606314|7011|13661|2|48|14|1|11644|137|929|33|22.97|65.69|59.77|195.36|1972.41|758.01|2167.77|35.50|1084.82|455.07|887.59|923.09|1342.66|1378.16|129.58| +2450833|61641|2450877|43664|606314|7011|13661|43664|606314|7011|13661|2|100|5|3|3596|142|929|56|80.51|186.78|102.72|4707.36|5752.32|4508.56|10459.68|460.18|0.00|1568.56|5752.32|6212.50|7320.88|7781.06|1243.76| +2450833|61641|2450851|43664|606314|7011|13661|43664|606314|7011|13661|2|34|3|5|3529|189|929|35|70.89|95.70|30.62|2277.80|1071.70|2481.15|3349.50|10.71|0.00|1440.25|1071.70|1082.41|2511.95|2522.66|-1409.45| +2450833|75674|2450896|91792|551651|1910|49816|45786|1709934|5440|49299|2|49|13|4|8335|92|930|41|48.37|123.34|22.20|4146.74|910.20|1983.17|5056.94|45.51|0.00|1112.33|910.20|955.71|2022.53|2068.04|-1072.97| +2450833|75674|2450896|91792|551651|1910|49816|45786|1709934|5440|49299|2|81|4|2|4534|98|930|79|49.89|97.28|32.10|5149.22|2535.90|3941.31|7685.12|172.18|76.07|3150.52|2459.83|2632.01|5610.35|5782.53|-1481.48| +2450833|75674|2450884|91792|551651|1910|49816|45786|1709934|5440|49299|2|95|9|1|3034|142|930|90|40.03|62.84|4.39|5260.50|395.10|3602.70|5655.60|35.55|0.00|678.60|395.10|430.65|1073.70|1109.25|-3207.60| +2450833|75674|2450850|91792|551651|1910|49816|45786|1709934|5440|49299|2|49|15|5|2818|202|930|36|76.92|157.68|36.26|4371.12|1305.36|2769.12|5676.48|0.00|0.00|2667.60|1305.36|1305.36|3972.96|3972.96|-1463.76| +2450833|75674|2450885|91792|551651|1910|49816|45786|1709934|5440|49299|2|61|16|1|5152|66|930|79|41.68|120.87|61.64|4679.17|4869.56|3292.72|9548.73|48.69|0.00|94.80|4869.56|4918.25|4964.36|5013.05|1576.84| +2450833|75674|2450877|91792|551651|1910|49816|45786|1709934|5440|49299|2|99|16|1|158|32|930|19|47.99|107.49|15.04|1756.55|285.76|911.81|2042.31|8.57|0.00|387.98|285.76|294.33|673.74|682.31|-626.05| +2450833|75674|2450837|91792|551651|1910|49816|45786|1709934|5440|49299|2|92|18|4|1976|146|930|35|56.56|144.79|57.91|3040.80|2026.85|1979.60|5067.65|182.41|0.00|1114.75|2026.85|2209.26|3141.60|3324.01|47.25| +2450833|75674|2450890|91792|551651|1910|49816|45786|1709934|5440|49299|2|39|8|1|12043|294|930|26|70.92|197.86|136.52|1594.84|3549.52|1843.92|5144.36|177.47|0.00|411.32|3549.52|3726.99|3960.84|4138.31|1705.60| +2450833|75674|2450907|91792|551651|1910|49816|45786|1709934|5440|49299|2|44|14|3|14461|69|930|64|88.04|255.31|30.63|14379.52|1960.32|5634.56|16339.84|0.00|0.00|1960.32|1960.32|1960.32|3920.64|3920.64|-3674.24| +2450833|75674|2450907|91792|551651|1910|49816|45786|1709934|5440|49299|2|37|4|2|8647|176|930|2|14.48|39.82|24.68|30.28|49.36|28.96|79.64|0.98|0.00|22.28|49.36|50.34|71.64|72.62|20.40| +2450833|75674|2450885|91792|551651|1910|49816|45786|1709934|5440|49299|2|2|8|2|11902|257|930|2|82.36|151.54|86.37|130.34|172.74|164.72|303.08|13.81|0.00|112.12|172.74|186.55|284.86|298.67|8.02| +2450833|75674|2450890|91792|551651|1910|49816|45786|1709934|5440|49299|2|21|2|2|3916|90|930|25|44.48|112.08|31.38|2017.50|784.50|1112.00|2802.00|47.07|0.00|588.25|784.50|831.57|1372.75|1419.82|-327.50| +2450833|75674|2450898|91792|551651|1910|49816|45786|1709934|5440|49299|2|88|18|1|11581|217|930|75|9.71|27.57|11.57|1200.00|867.75|728.25|2067.75|14.83|373.13|847.50|494.62|509.45|1342.12|1356.95|-233.63| +2450833|55976|2450871|37122|1012957|3192|11827|37122|1012957|3192|11827|4|71|19|1|13552|87|931|71|46.59|119.27|98.99|1439.88|7028.29|3307.89|8468.17|632.54|0.00|1016.01|7028.29|7660.83|8044.30|8676.84|3720.40| +2450833|55976|2450881|37122|1012957|3192|11827|37122|1012957|3192|11827|4|106|10|3|17518|67|931|32|8.97|18.02|14.59|109.76|466.88|287.04|576.64|32.68|0.00|17.28|466.88|499.56|484.16|516.84|179.84| +2450833|55976|2450922|37122|1012957|3192|11827|37122|1012957|3192|11827|4|9|2|3|14218|103|931|76|86.08|189.37|39.76|11370.36|3021.76|6542.08|14392.12|151.08|0.00|6907.64|3021.76|3172.84|9929.40|10080.48|-3520.32| +2450833|55976|2450839|37122|1012957|3192|11827|37122|1012957|3192|11827|4|97|5|5|1652|62|931|30|19.89|38.58|20.83|532.50|624.90|596.70|1157.40|49.99|0.00|358.50|624.90|674.89|983.40|1033.39|28.20| +2450833|55976|2450922|37122|1012957|3192|11827|37122|1012957|3192|11827|4|28|13|4|7075|271|931|63|36.64|52.76|26.38|1661.94|1661.94|2308.32|3323.88|149.57|0.00|1229.76|1661.94|1811.51|2891.70|3041.27|-646.38| +2450833|55976|2450914|37122|1012957|3192|11827|37122|1012957|3192|11827|4|70|2|4|17905|98|931|68|2.00|3.18|0.41|188.36|27.88|136.00|216.24|0.83|0.00|66.64|27.88|28.71|94.52|95.35|-108.12| +2450833|55976|2450920|37122|1012957|3192|11827|37122|1012957|3192|11827|4|89|19|5|15866|70|931|28|5.54|15.51|4.65|304.08|130.20|155.12|434.28|3.90|0.00|160.44|130.20|134.10|290.64|294.54|-24.92| +2450833|55976|2450867|37122|1012957|3192|11827|37122|1012957|3192|11827|4|80|17|5|9463|51|931|47|23.74|27.30|0.81|1245.03|38.07|1115.78|1283.10|0.35|2.28|153.69|35.79|36.14|189.48|189.83|-1079.99| +2450833|55976|2450897|37122|1012957|3192|11827|37122|1012957|3192|11827|4|78|19|3|13904|116|931|92|61.89|81.07|44.58|3357.08|4101.36|5693.88|7458.44|123.04|0.00|1491.32|4101.36|4224.40|5592.68|5715.72|-1592.52| +2450833|55976|2450889|37122|1012957|3192|11827|37122|1012957|3192|11827|4|106|4|3|11234|118|931|24|15.76|24.27|16.50|186.48|396.00|378.24|582.48|7.92|0.00|63.84|396.00|403.92|459.84|467.76|17.76| +2450833|57233|2450885|73566|750478|6237|5149|73566|750478|6237|5149|2|63|9|1|1828|47|932|15|79.97|127.95|104.91|345.60|1573.65|1199.55|1919.25|62.94|0.00|921.15|1573.65|1636.59|2494.80|2557.74|374.10| +2450833|57233|2450921|73566|750478|6237|5149|73566|750478|6237|5149|2|16|9|3|11359|3|932|61|7.22|12.85|11.05|109.80|674.05|440.42|783.85|27.50|330.28|70.15|343.77|371.27|413.92|441.42|-96.65| +2450833|57233|2450920|73566|750478|6237|5149|73566|750478|6237|5149|2|101|19|1|10124|72|932|3|91.78|167.03|158.67|25.08|476.01|275.34|501.09|0.00|114.24|5.01|361.77|361.77|366.78|366.78|86.43| +2450833|57233|2450873|73566|750478|6237|5149|73566|750478|6237|5149|2|70|10|1|7280|47|932|71|74.31|97.34|49.64|3386.70|3524.44|5276.01|6911.14|176.22|0.00|1036.60|3524.44|3700.66|4561.04|4737.26|-1751.57| +2450833|57233|2450867|73566|750478|6237|5149|73566|750478|6237|5149|2|79|1|2|655|187|932|78|11.43|21.60|14.47|556.14|1128.66|891.54|1684.80|7.11|1049.65|252.72|79.01|86.12|331.73|338.84|-812.53| +2450833|81347|2450918|62157|990948|2492|28483|62157|990948|2492|28483|2|83|6|1|7831|268|933|71|14.75|34.07|7.15|1911.32|507.65|1047.25|2418.97|10.15|0.00|1184.99|507.65|517.80|1692.64|1702.79|-539.60| +2450833|81347|2450860|62157|990948|2492|28483|62157|990948|2492|28483|2|45|8|5|4156|204|933|25|86.01|164.27|77.20|2176.75|1930.00|2150.25|4106.75|19.30|0.00|123.00|1930.00|1949.30|2053.00|2072.30|-220.25| +2450833|81347|2450874|62157|990948|2492|28483|62157|990948|2492|28483|2|101|20|2|2344|121|933|6|4.48|12.81|6.91|35.40|41.46|26.88|76.86|3.73|0.00|9.18|41.46|45.19|50.64|54.37|14.58| +2450833|81347|2450904|62157|990948|2492|28483|62157|990948|2492|28483|2|70|8|2|5212|97|933|34|75.88|175.28|127.95|1609.22|4350.30|2579.92|5959.52|0.00|3741.25|1608.88|609.05|609.05|2217.93|2217.93|-1970.87| +2450833|81347|2450848|62157|990948|2492|28483|62157|990948|2492|28483|2|31|19|3|5198|101|933|85|90.95|201.90|48.45|13043.25|4118.25|7730.75|17161.50|123.54|0.00|6349.50|4118.25|4241.79|10467.75|10591.29|-3612.50| +2450833|81347|2450856|62157|990948|2492|28483|62157|990948|2492|28483|2|53|15|4|10694|185|933|59|40.79|84.02|0.00|4957.18|0.00|2406.61|4957.18|0.00|0.00|1139.88|0.00|0.00|1139.88|1139.88|-2406.61| +2450833|81347|2450908|62157|990948|2492|28483|62157|990948|2492|28483|2|61|17|5|4369|164|933|15|18.07|24.03|21.86|32.55|327.90|271.05|360.45|13.11|0.00|108.00|327.90|341.01|435.90|449.01|56.85| +2450833|33983|2450838|39767|986531|142|6906|39767|986531|142|6906|4|30|12|4|14068|34|934|79|64.20|127.75|51.10|6055.35|4036.90|5071.80|10092.25|322.95|0.00|504.02|4036.90|4359.85|4540.92|4863.87|-1034.90| +2450833|33983|2450892|39767|986531|142|6906|39767|986531|142|6906|4|44|11|4|6112|15|934|61|99.03|205.98|76.21|7915.97|4648.81|6040.83|12564.78|92.97|0.00|250.71|4648.81|4741.78|4899.52|4992.49|-1392.02| +2450833|33983|2450884|39767|986531|142|6906|39767|986531|142|6906|4|35|8|4|1486|4|934|23|13.16|21.05|8.63|285.66|198.49|302.68|484.15|7.93|0.00|57.96|198.49|206.42|256.45|264.38|-104.19| +2450833|33983|2450842|39767|986531|142|6906|39767|986531|142|6906|4|3|20|1|12196|68|934|79|6.47|8.15|5.86|180.91|462.94|511.13|643.85|37.03|0.00|192.76|462.94|499.97|655.70|692.73|-48.19| +2450833|33983|2450894|39767|986531|142|6906|39767|986531|142|6906|4|86|15|2|6788|63|934|27|40.49|102.03|21.42|2176.47|578.34|1093.23|2754.81|23.13|0.00|743.58|578.34|601.47|1321.92|1345.05|-514.89| +2450833|33983|2450904|39767|986531|142|6906|39767|986531|142|6906|4|13|7|5|16522|187|934|11|44.72|94.35|4.71|986.04|51.81|491.92|1037.85|1.03|0.00|186.78|51.81|52.84|238.59|239.62|-440.11| +2450833|33983|2450915|39767|986531|142|6906|39767|986531|142|6906|4|48|7|2|14338|136|934|99|87.84|131.76|21.08|10957.32|2086.92|8696.16|13044.24|10.85|1815.62|912.78|271.30|282.15|1184.08|1194.93|-8424.86| +2450833|33983|2450865|39767|986531|142|6906|39767|986531|142|6906|4|92|9|2|10982|297|934|8|65.72|195.18|105.39|718.32|843.12|525.76|1561.44|4.04|792.53|187.36|50.59|54.63|237.95|241.99|-475.17| +2450833|33983|2450886|39767|986531|142|6906|39767|986531|142|6906|4|38|10|1|15685|300|934|78|39.09|64.10|3.84|4700.28|299.52|3049.02|4999.80|5.99|0.00|1399.32|299.52|305.51|1698.84|1704.83|-2749.50| +2450833|33983|2450847|39767|986531|142|6906|39767|986531|142|6906|4|2|5|1|9220|253|934|18|66.21|90.04|81.93|145.98|1474.74|1191.78|1620.72|117.97|0.00|129.60|1474.74|1592.71|1604.34|1722.31|282.96| +2450833|33219|2450890|54683|901748|366|30231|54683|901748|366|30231|2|35|6|3|5215|265|935|83|96.95|206.50|105.31|8398.77|8740.73|8046.85|17139.50|30.59|8303.69|5141.85|437.04|467.63|5578.89|5609.48|-7609.81| +2450833|33219|2450879|54683|901748|366|30231|54683|901748|366|30231|2|89|3|2|13562|61|935|99|34.91|67.72|32.50|3486.78|3217.50|3456.09|6704.28|193.05|0.00|0.00|3217.50|3410.55|3217.50|3410.55|-238.59| +2450833|33219|2450893|54683|901748|366|30231|54683|901748|366|30231|2|87|9|5|478|244|935|21|57.12|93.10|70.75|469.35|1485.75|1199.52|1955.10|14.85|0.00|879.69|1485.75|1500.60|2365.44|2380.29|286.23| +2450833|33219|2450865|54683|901748|366|30231|54683|901748|366|30231|2|60|3|5|277|31|935|82|98.58|156.74|31.34|10282.80|2569.88|8083.56|12852.68|205.59|0.00|2827.36|2569.88|2775.47|5397.24|5602.83|-5513.68| +2450833|33219|2450877|54683|901748|366|30231|54683|901748|366|30231|2|25|9|4|6748|246|935|80|85.71|178.27|8.91|13548.80|712.80|6856.80|14261.60|26.08|277.99|142.40|434.81|460.89|577.21|603.29|-6421.99| +2450833|33219|2450854|54683|901748|366|30231|54683|901748|366|30231|2|41|5|4|14986|227|935|94|13.44|35.61|9.25|2477.84|869.50|1263.36|3347.34|69.56|0.00|401.38|869.50|939.06|1270.88|1340.44|-393.86| +2450833|33219|2450846|54683|901748|366|30231|54683|901748|366|30231|2|87|6|5|7772|65|935|15|59.02|75.54|63.45|181.35|951.75|885.30|1133.10|19.03|0.00|373.80|951.75|970.78|1325.55|1344.58|66.45| +2450833|33219|2450848|54683|901748|366|30231|54683|901748|366|30231|2|36|11|1|2284|268|935|87|87.17|124.65|105.95|1626.90|9217.65|7583.79|10844.55|829.58|0.00|5096.46|9217.65|10047.23|14314.11|15143.69|1633.86| +2450833|33219|2450871|54683|901748|366|30231|54683|901748|366|30231|2|24|13|1|6332|19|935|37|7.02|14.18|9.21|183.89|340.77|259.74|524.66|6.81|0.00|256.78|340.77|347.58|597.55|604.36|81.03| +2450833|33219|2450880|54683|901748|366|30231|54683|901748|366|30231|2|71|14|3|8480|183|935|94|31.61|63.22|34.13|2734.46|3208.22|2971.34|5942.68|288.73|0.00|2436.48|3208.22|3496.95|5644.70|5933.43|236.88| +2450833|69662|2450900|92193|503106|3266|1826|92193|503106|3266|1826|4|65|2|4|841|61|936|14|43.89|65.39|41.19|338.80|576.66|614.46|915.46|40.36|0.00|164.78|576.66|617.02|741.44|781.80|-37.80| +2450833|69662|2450843|92193|503106|3266|1826|92193|503106|3266|1826|4|33|5|1|5539|247|936|94|43.13|74.61|53.71|1964.60|5048.74|4054.22|7013.34|100.97|0.00|1542.54|5048.74|5149.71|6591.28|6692.25|994.52| +2450833|69662|2450867|92193|503106|3266|1826|92193|503106|3266|1826|4|18|11|3|3025|214|936|43|65.51|100.88|70.61|1301.61|3036.23|2816.93|4337.84|273.26|0.00|867.31|3036.23|3309.49|3903.54|4176.80|219.30| +2450833|69662|2450838|92193|503106|3266|1826|92193|503106|3266|1826|4|62|6|2|1459|256|936|83|8.33|19.65|12.96|555.27|1075.68|691.39|1630.95|86.05|0.00|15.77|1075.68|1161.73|1091.45|1177.50|384.29| +2450833|69662|2450907|92193|503106|3266|1826|92193|503106|3266|1826|4|80|13|5|6328|200|936|2|97.86|145.81|11.66|268.30|23.32|195.72|291.62|0.39|17.72|122.48|5.60|5.99|128.08|128.47|-190.12| +2450833|69662|2450905|92193|503106|3266|1826|92193|503106|3266|1826|4|42|10|5|8962|65|936|45|47.07|88.49|42.47|2070.90|1911.15|2118.15|3982.05|57.33|0.00|1314.00|1911.15|1968.48|3225.15|3282.48|-207.00| +2450833|69662|2450879|92193|503106|3266|1826|92193|503106|3266|1826|4|2|6|4|10484|33|936|46|8.25|12.78|3.32|435.16|152.72|379.50|587.88|1.52|0.00|64.40|152.72|154.24|217.12|218.64|-226.78| +2450833|69662|2450891|92193|503106|3266|1826|92193|503106|3266|1826|4|32|19|1|17044|63|936|65|26.49|65.69|15.10|3288.35|981.50|1721.85|4269.85|39.26|0.00|512.20|981.50|1020.76|1493.70|1532.96|-740.35| +2450833|69662|2450892|92193|503106|3266|1826|92193|503106|3266|1826|4|27|6|5|5750|166|936|29|7.32|15.73|3.93|342.20|113.97|212.28|456.17|5.01|51.28|218.95|62.69|67.70|281.64|286.65|-149.59| +2450833|69662|2450857|92193|503106|3266|1826|92193|503106|3266|1826|4|17|15|2|2107|218|936|39|56.26|105.76|62.39|1691.43|2433.21|2194.14|4124.64|121.66|0.00|412.23|2433.21|2554.87|2845.44|2967.10|239.07| +2450833|69662|2450864|92193|503106|3266|1826|92193|503106|3266|1826|4|82|10|1|8875|211|936|46|99.30|186.68|41.06|6698.52|1888.76|4567.80|8587.28|75.55|0.00|1974.78|1888.76|1964.31|3863.54|3939.09|-2679.04| +2450833|69662|2450893|92193|503106|3266|1826|92193|503106|3266|1826|4|63|1|5|10543|128|936|60|27.84|82.96|9.95|4380.60|597.00|1670.40|4977.60|35.82|0.00|1095.00|597.00|632.82|1692.00|1727.82|-1073.40| +2450833|69662|2450876|92193|503106|3266|1826|92193|503106|3266|1826|4|82|16|3|9187|157|936|45|83.56|238.14|185.74|2358.00|8358.30|3760.20|10716.30|752.24|0.00|1071.45|8358.30|9110.54|9429.75|10181.99|4598.10| +2450833|74380|2450923|43946|476430|1714|49845|43946|476430|1714|49845|2|49|9|5|7042|107|937|74|2.44|4.90|4.90|0.00|362.60|180.56|362.60|18.13|0.00|148.00|362.60|380.73|510.60|528.73|182.04| +2450833|74380|2450898|43946|476430|1714|49845|43946|476430|1714|49845|2|60|12|1|3446|132|937|36|33.23|52.17|42.77|338.40|1539.72|1196.28|1878.12|0.00|0.00|431.64|1539.72|1539.72|1971.36|1971.36|343.44| +2450833|74380|2450876|43946|476430|1714|49845|43946|476430|1714|49845|2|32|16|1|14912|188|937|86|6.29|11.07|4.76|542.66|409.36|540.94|952.02|0.00|0.00|456.66|409.36|409.36|866.02|866.02|-131.58| +2450833|74380|2450873|43946|476430|1714|49845|43946|476430|1714|49845|2|10|11|3|17206|282|937|94|93.19|214.33|57.86|14708.18|5438.84|8759.86|20147.02|135.97|2719.42|1207.90|2719.42|2855.39|3927.32|4063.29|-6040.44| +2450833|74380|2450876|43946|476430|1714|49845|43946|476430|1714|49845|2|95|4|2|9914|215|937|1|5.66|9.84|5.60|4.24|5.60|5.66|9.84|0.16|0.00|3.14|5.60|5.76|8.74|8.90|-0.06| +2450833|74380|2450918|43946|476430|1714|49845|43946|476430|1714|49845|2|22|8|2|2516|135|937|91|37.04|44.81|31.81|1183.00|2894.71|3370.64|4077.71|0.00|0.00|325.78|2894.71|2894.71|3220.49|3220.49|-475.93| +2450833|74380|2450901|43946|476430|1714|49845|43946|476430|1714|49845|2|34|14|2|16075|293|937|85|47.03|47.03|11.28|3038.75|958.80|3997.55|3997.55|67.11|0.00|838.95|958.80|1025.91|1797.75|1864.86|-3038.75| +2450833|74380|2450887|43946|476430|1714|49845|43946|476430|1714|49845|2|32|11|2|13201|51|937|58|30.22|42.00|13.86|1632.12|803.88|1752.76|2436.00|72.34|0.00|267.96|803.88|876.22|1071.84|1144.18|-948.88| +2450833|74380|2450865|43946|476430|1714|49845|43946|476430|1714|49845|2|8|9|4|8414|87|937|6|29.86|61.21|41.01|121.20|246.06|179.16|367.26|19.68|0.00|10.98|246.06|265.74|257.04|276.72|66.90| +2450833|74380|2450874|43946|476430|1714|49845|43946|476430|1714|49845|2|105|15|2|440|222|937|29|47.61|67.13|33.56|973.53|973.24|1380.69|1946.77|68.12|0.00|252.88|973.24|1041.36|1226.12|1294.24|-407.45| +2450833|67863|2450898|4849|1230740|4631|47675|4849|1230740|4631|47675|1|9|6|5|11554|298|938|39|78.42|163.89|8.19|6072.30|319.41|3058.38|6391.71|15.97|0.00|1214.07|319.41|335.38|1533.48|1549.45|-2738.97| +2450833|67863|2450864|4849|1230740|4631|47675|4849|1230740|4631|47675|1|54|1|3|12290|235|938|65|73.33|87.26|31.41|3630.25|2041.65|4766.45|5671.90|142.91|0.00|1587.95|2041.65|2184.56|3629.60|3772.51|-2724.80| +2450833|67863|2450848|4849|1230740|4631|47675|4849|1230740|4631|47675|1|44|15|1|12512|83|938|87|73.08|188.54|37.70|13123.08|3279.90|6357.96|16402.98|32.79|0.00|3607.89|3279.90|3312.69|6887.79|6920.58|-3078.06| +2450833|67863|2450885|4849|1230740|4631|47675|4849|1230740|4631|47675|1|79|19|3|2492|111|938|20|67.22|189.56|147.85|834.20|2957.00|1344.40|3791.20|0.00|0.00|530.60|2957.00|2957.00|3487.60|3487.60|1612.60| +2450833|67863|2450857|4849|1230740|4631|47675|4849|1230740|4631|47675|1|57|1|5|6554|36|938|96|97.52|154.08|109.39|4290.24|10501.44|9361.92|14791.68|98.71|630.08|3549.12|9871.36|9970.07|13420.48|13519.19|509.44| +2450833|67863|2450845|4849|1230740|4631|47675|4849|1230740|4631|47675|1|89|14|5|12304|265|938|72|64.07|124.93|67.46|4137.84|4857.12|4613.04|8994.96|48.57|0.00|3687.84|4857.12|4905.69|8544.96|8593.53|244.08| +2450833|67863|2450914|4849|1230740|4631|47675|4849|1230740|4631|47675|1|48|4|2|8179|247|938|92|16.62|43.21|22.03|1948.56|2026.76|1529.04|3975.32|8.91|1134.98|1033.16|891.78|900.69|1924.94|1933.85|-637.26| +2450833|67863|2450862|4849|1230740|4631|47675|4849|1230740|4631|47675|1|72|15|2|2848|8|938|79|49.58|117.00|7.02|8688.42|554.58|3916.82|9243.00|11.09|0.00|3882.06|554.58|565.67|4436.64|4447.73|-3362.24| +2450833|58495|2450867|7993|362940|4640|49187|22180|334634|6734|31681|2|92|14|5|100|28|939|35|54.31|75.49|16.60|2061.15|581.00|1900.85|2642.15|5.81|0.00|105.35|581.00|586.81|686.35|692.16|-1319.85| +2450833|58495|2450897|7993|362940|4640|49187|22180|334634|6734|31681|2|2|17|5|12158|63|939|85|50.67|140.35|87.01|4533.90|7395.85|4306.95|11929.75|221.87|3697.92|5009.90|3697.93|3919.80|8707.83|8929.70|-609.02| +2450833|58495|2450847|7993|362940|4640|49187|22180|334634|6734|31681|2|54|14|1|8578|224|939|47|14.92|29.24|5.84|1099.80|274.48|701.24|1374.28|8.23|0.00|439.45|274.48|282.71|713.93|722.16|-426.76| +2450833|58495|2450868|7993|362940|4640|49187|22180|334634|6734|31681|2|11|8|5|8050|153|939|23|43.81|86.30|26.75|1369.65|615.25|1007.63|1984.90|18.45|0.00|654.81|615.25|633.70|1270.06|1288.51|-392.38| +2450833|58495|2450860|7993|362940|4640|49187|22180|334634|6734|31681|2|29|16|2|9562|16|939|14|51.74|78.64|60.55|253.26|847.70|724.36|1100.96|12.20|712.06|98.98|135.64|147.84|234.62|246.82|-588.72| +2450833|58495|2450848|7993|362940|4640|49187|22180|334634|6734|31681|2|13|14|3|6380|114|939|60|44.41|72.83|68.46|262.20|4107.60|2664.60|4369.80|41.07|0.00|961.20|4107.60|4148.67|5068.80|5109.87|1443.00| +2450833|58495|2450841|7993|362940|4640|49187|22180|334634|6734|31681|2|66|5|3|16285|123|939|14|52.35|82.18|33.69|678.86|471.66|732.90|1150.52|42.44|0.00|0.00|471.66|514.10|471.66|514.10|-261.24| +2450833|58495|2450898|7993|362940|4640|49187|22180|334634|6734|31681|2|92|14|4|12206|114|939|67|26.72|66.53|25.94|2719.53|1737.98|1790.24|4457.51|34.75|0.00|1158.43|1737.98|1772.73|2896.41|2931.16|-52.26| +2450833|58495|2450881|7993|362940|4640|49187|22180|334634|6734|31681|2|101|13|4|17834|179|939|52|54.40|155.04|40.31|5965.96|2096.12|2828.80|8062.08|83.84|0.00|241.80|2096.12|2179.96|2337.92|2421.76|-732.68| +2450833|58495|2450861|7993|362940|4640|49187|22180|334634|6734|31681|2|82|20|4|14725|169|939|68|13.70|25.48|0.50|1698.64|34.00|931.60|1732.64|2.72|0.00|623.56|34.00|36.72|657.56|660.28|-897.60| +2450833|58495|2450906|7993|362940|4640|49187|22180|334634|6734|31681|2|15|15|5|16888|137|939|42|65.12|138.70|131.76|291.48|5533.92|2735.04|5825.40|55.33|0.00|57.96|5533.92|5589.25|5591.88|5647.21|2798.88| +2450833|18407|2450879|39060|117457|3054|21098|39060|117457|3054|21098|4|96|9|1|9868|153|940|7|56.01|117.06|12.87|729.33|90.09|392.07|819.42|0.90|0.00|8.19|90.09|90.99|98.28|99.18|-301.98| +2450833|18407|2450866|39060|117457|3054|21098|39060|117457|3054|21098|4|7|19|3|10810|181|940|53|28.42|82.70|43.83|2060.11|2322.99|1506.26|4383.10|209.06|0.00|2191.55|2322.99|2532.05|4514.54|4723.60|816.73| +2450833|18407|2450915|39060|117457|3054|21098|39060|117457|3054|21098|4|40|19|5|15577|162|940|7|90.82|231.59|16.21|1507.66|113.47|635.74|1621.13|2.26|0.00|778.12|113.47|115.73|891.59|893.85|-522.27| +2450833|18407|2450905|39060|117457|3054|21098|39060|117457|3054|21098|4|26|10|4|2167|110|940|27|58.97|164.52|157.93|177.93|4264.11|1592.19|4442.04|341.12|0.00|399.60|4264.11|4605.23|4663.71|5004.83|2671.92| +2450833|18407|2450902|39060|117457|3054|21098|39060|117457|3054|21098|4|5|16|5|12115|169|940|6|49.24|98.97|88.08|65.34|528.48|295.44|593.82|4.65|470.34|35.58|58.14|62.79|93.72|98.37|-237.30| +2450833|18407|2450920|39060|117457|3054|21098|39060|117457|3054|21098|4|27|8|2|14440|103|940|46|57.60|171.07|135.14|1652.78|6216.44|2649.60|7869.22|186.49|0.00|1652.32|6216.44|6402.93|7868.76|8055.25|3566.84| +2450833|18407|2450839|39060|117457|3054|21098|39060|117457|3054|21098|4|59|14|1|12937|63|940|70|75.83|92.51|38.85|3756.20|2719.50|5308.10|6475.70|5.71|2637.91|129.50|81.59|87.30|211.09|216.80|-5226.51| +2450833|18407|2450837|39060|117457|3054|21098|39060|117457|3054|21098|4|69|9|5|394|263|940|28|89.98|174.56|76.80|2737.28|2150.40|2519.44|4887.68|21.28|1913.85|2248.12|236.55|257.83|2484.67|2505.95|-2282.89| +2450833|78747|2450878|88656|520316|6376|31236|88656|520316|6376|31236|2|17|18|2|988|126|941|35|69.97|87.46|70.84|581.70|2479.40|2448.95|3061.10|223.14|0.00|30.45|2479.40|2702.54|2509.85|2732.99|30.45| +2450833|78747|2450848|88656|520316|6376|31236|88656|520316|6376|31236|2|106|6|1|5038|264|941|22|83.64|122.95|6.14|2569.82|135.08|1840.08|2704.90|2.99|85.10|1352.34|49.98|52.97|1402.32|1405.31|-1790.10| +2450833|78747|2450885|88656|520316|6376|31236|88656|520316|6376|31236|2|74|14|2|12374|88|941|31|53.97|108.47|74.84|1042.53|2320.04|1673.07|3362.57|20.88|2088.03|1008.74|232.01|252.89|1240.75|1261.63|-1441.06| +2450833|78747|2450889|88656|520316|6376|31236|88656|520316|6376|31236|2|76|17|2|1814|112|941|26|70.39|209.76|130.05|2072.46|3381.30|1830.14|5453.76|304.31|0.00|1254.24|3381.30|3685.61|4635.54|4939.85|1551.16| +2450833|78747|2450909|88656|520316|6376|31236|88656|520316|6376|31236|2|91|7|4|9830|151|941|67|20.22|49.94|24.97|1672.99|1672.99|1354.74|3345.98|50.18|0.00|468.33|1672.99|1723.17|2141.32|2191.50|318.25| +2450833|78747|2450902|88656|520316|6376|31236|88656|520316|6376|31236|2|65|9|2|8929|253|941|87|87.79|218.59|45.90|15024.03|3993.30|7637.73|19017.33|119.79|0.00|569.85|3993.30|4113.09|4563.15|4682.94|-3644.43| +2450833|78747|2450919|88656|520316|6376|31236|88656|520316|6376|31236|2|48|15|3|5270|209|941|74|64.28|154.91|27.88|9400.22|2063.12|4756.72|11463.34|82.52|0.00|343.36|2063.12|2145.64|2406.48|2489.00|-2693.60| +2450833|63161|2450871|43706|1877625|344|48663|68881|1782901|5620|21939|2|61|3|1|8407|72|942|16|36.01|64.45|1.93|1000.32|30.88|576.16|1031.20|0.33|19.76|422.72|11.12|11.45|433.84|434.17|-565.04| +2450833|63161|2450861|43706|1877625|344|48663|68881|1782901|5620|21939|2|88|8|5|12074|198|942|44|69.84|118.02|67.27|2233.00|2959.88|3072.96|5192.88|207.19|0.00|2024.88|2959.88|3167.07|4984.76|5191.95|-113.08| +2450833|63161|2450865|43706|1877625|344|48663|68881|1782901|5620|21939|2|56|6|4|12880|182|942|21|59.01|100.90|14.12|1822.38|296.52|1239.21|2118.90|5.93|0.00|63.42|296.52|302.45|359.94|365.87|-942.69| +2450833|63161|2450893|43706|1877625|344|48663|68881|1782901|5620|21939|2|72|15|2|6070|128|942|82|69.81|164.05|73.82|7398.86|6053.24|5724.42|13452.10|0.00|0.00|1479.28|6053.24|6053.24|7532.52|7532.52|328.82| +2450833|63161|2450862|43706|1877625|344|48663|68881|1782901|5620|21939|2|60|12|2|10816|180|942|54|49.00|109.27|25.13|4543.56|1357.02|2646.00|5900.58|122.13|0.00|1710.72|1357.02|1479.15|3067.74|3189.87|-1288.98| +2450833|63161|2450837|43706|1877625|344|48663|68881|1782901|5620|21939|2|71|18|5|1162|293|942|41|86.63|188.85|66.09|5033.16|2709.69|3551.83|7742.85|216.77|0.00|2090.18|2709.69|2926.46|4799.87|5016.64|-842.14| +2450833|63161|2450911|43706|1877625|344|48663|68881|1782901|5620|21939|2|78|7|2|13588|130|942|58|59.43|108.16|1.08|6210.64|62.64|3446.94|6273.28|3.75|0.00|2383.80|62.64|66.39|2446.44|2450.19|-3384.30| +2450833|63161|2450900|43706|1877625|344|48663|68881|1782901|5620|21939|2|80|4|5|16744|72|942|28|34.34|36.40|1.45|978.60|40.60|961.52|1019.20|1.21|0.00|254.80|40.60|41.81|295.40|296.61|-920.92| +2450833|63161|2450843|43706|1877625|344|48663|68881|1782901|5620|21939|2|71|9|2|9574|22|942|17|13.49|17.13|4.79|209.78|81.43|229.33|291.21|7.32|0.00|104.72|81.43|88.75|186.15|193.47|-147.90| +2450833|63161|2450846|43706|1877625|344|48663|68881|1782901|5620|21939|2|107|17|3|4795|55|942|62|40.93|49.52|10.89|2395.06|675.18|2537.66|3070.24|20.93|256.56|429.66|418.62|439.55|848.28|869.21|-2119.04| +2450833|63161|2450853|43706|1877625|344|48663|68881|1782901|5620|21939|2|88|18|3|13606|141|942|97|95.06|131.18|101.00|2927.46|9797.00|9220.82|12724.46|357.59|2645.19|1144.60|7151.81|7509.40|8296.41|8654.00|-2069.01| +2450833|63161|2450912|43706|1877625|344|48663|68881|1782901|5620|21939|2|52|8|1|10561|95|942|91|20.47|28.24|7.62|1876.42|693.42|1862.77|2569.84|5.75|117.88|204.75|575.54|581.29|780.29|786.04|-1287.23| +2450833|63161|2450902|43706|1877625|344|48663|68881|1782901|5620|21939|2|12|19|5|11672|55|942|35|14.56|19.36|9.68|338.80|338.80|509.60|677.60|2.57|274.42|338.80|64.38|66.95|403.18|405.75|-445.22| +2450833|36192|2450917|9675|1364205|1488|30418|9675|1364205|1488|30418|4|78|13|5|14353|222|943|51|52.67|119.56|82.49|1890.57|4206.99|2686.17|6097.56|42.06|0.00|60.69|4206.99|4249.05|4267.68|4309.74|1520.82| +2450833|36192|2450868|9675|1364205|1488|30418|9675|1364205|1488|30418|4|61|2|3|12862|112|943|35|65.80|192.13|192.13|0.00|6724.55|2303.00|6724.55|336.22|0.00|1344.70|6724.55|7060.77|8069.25|8405.47|4421.55| +2450833|36192|2450894|9675|1364205|1488|30418|9675|1364205|1488|30418|4|36|13|4|13435|200|943|40|42.42|60.23|50.59|385.60|2023.60|1696.80|2409.20|0.00|0.00|1180.40|2023.60|2023.60|3204.00|3204.00|326.80| +2450833|36192|2450921|9675|1364205|1488|30418|9675|1364205|1488|30418|4|64|1|3|9745|205|943|90|76.62|159.36|116.33|3872.70|10469.70|6895.80|14342.40|942.27|0.00|6884.10|10469.70|11411.97|17353.80|18296.07|3573.90| +2450833|36192|2450917|9675|1364205|1488|30418|9675|1364205|1488|30418|4|104|11|1|15184|181|943|40|74.19|91.25|36.50|2190.00|1460.00|2967.60|3650.00|14.60|0.00|1277.20|1460.00|1474.60|2737.20|2751.80|-1507.60| +2450833|36192|2450909|9675|1364205|1488|30418|9675|1364205|1488|30418|4|55|15|2|11983|264|943|60|27.48|54.68|25.15|1771.80|1509.00|1648.80|3280.80|135.81|0.00|1180.80|1509.00|1644.81|2689.80|2825.61|-139.80| +2450833|36192|2450868|9675|1364205|1488|30418|9675|1364205|1488|30418|4|49|19|1|8450|150|943|41|85.22|238.61|112.14|5185.27|4597.74|3494.02|9783.01|167.35|413.79|1369.40|4183.95|4351.30|5553.35|5720.70|689.93| +2450833|36192|2450879|9675|1364205|1488|30418|9675|1364205|1488|30418|4|90|9|1|2329|33|943|8|18.98|40.61|0.00|324.88|0.00|151.84|324.88|0.00|0.00|6.48|0.00|0.00|6.48|6.48|-151.84| +2450833|36192|2450851|9675|1364205|1488|30418|9675|1364205|1488|30418|4|27|12|4|14053|24|943|48|83.16|166.32|111.43|2634.72|5348.64|3991.68|7983.36|160.45|0.00|1436.64|5348.64|5509.09|6785.28|6945.73|1356.96| +2450833|36192|2450896|9675|1364205|1488|30418|9675|1364205|1488|30418|4|20|14|5|14752|219|943|56|11.64|30.61|15.91|823.20|890.96|651.84|1714.16|53.45|0.00|719.60|890.96|944.41|1610.56|1664.01|239.12| +2450833|36192|2450858|9675|1364205|1488|30418|9675|1364205|1488|30418|4|44|7|1|16988|294|943|52|70.05|180.02|163.81|842.92|8518.12|3642.60|9361.04|766.63|0.00|2714.40|8518.12|9284.75|11232.52|11999.15|4875.52| +2450833|36192|2450913|9675|1364205|1488|30418|9675|1364205|1488|30418|4|27|2|3|10946|170|943|76|72.06|92.95|50.19|3249.76|3814.44|5476.56|7064.20|5.72|3700.00|3390.36|114.44|120.16|3504.80|3510.52|-5362.12| +2450833|68643|2450875|44831|1721233|835|45237|44831|1721233|835|45237|4|63|17|3|13892|254|944|97|57.23|151.65|50.04|9856.17|4853.88|5551.31|14710.05|97.07|0.00|1176.61|4853.88|4950.95|6030.49|6127.56|-697.43| +2450833|68643|2450901|44831|1721233|835|45237|44831|1721233|835|45237|4|42|19|3|2572|228|944|4|78.26|82.17|6.57|302.40|26.28|313.04|328.68|2.36|0.00|32.84|26.28|28.64|59.12|61.48|-286.76| +2450833|68643|2450920|44831|1721233|835|45237|44831|1721233|835|45237|4|92|15|4|6884|210|944|7|97.10|253.43|76.02|1241.87|532.14|679.70|1774.01|0.15|526.81|0.00|5.33|5.48|5.33|5.48|-674.37| +2450833|68643|2450896|44831|1721233|835|45237|44831|1721233|835|45237|4|8|8|5|574|297|944|33|97.53|252.60|237.44|500.28|7835.52|3218.49|8335.80|235.06|0.00|4167.90|7835.52|8070.58|12003.42|12238.48|4617.03| +2450833|68643|2450864|44831|1721233|835|45237|44831|1721233|835|45237|4|93|6|1|11428|2|944|98|93.34|218.41|128.86|8775.90|12628.28|9147.32|21404.18|0.00|0.00|6849.22|12628.28|12628.28|19477.50|19477.50|3480.96| +2450833|68643|2450893|44831|1721233|835|45237|44831|1721233|835|45237|4|28|7|1|379|136|944|82|97.09|120.39|48.15|5923.68|3948.30|7961.38|9871.98|236.89|0.00|3158.64|3948.30|4185.19|7106.94|7343.83|-4013.08| +2450833|68643|2450854|44831|1721233|835|45237|44831|1721233|835|45237|4|81|2|3|5902|238|944|43|47.13|64.09|38.45|1102.52|1653.35|2026.59|2755.87|82.66|0.00|578.35|1653.35|1736.01|2231.70|2314.36|-373.24| +2450833|68643|2450912|44831|1721233|835|45237|44831|1721233|835|45237|4|107|4|5|2671|234|944|94|62.48|175.56|66.71|10231.90|6270.74|5873.12|16502.64|158.02|4514.93|2804.96|1755.81|1913.83|4560.77|4718.79|-4117.31| +2450833|67663|2450883|10784|78591|6845|4252|10784|78591|6845|4252|2|68|14|4|8834|144|945|6|25.23|69.13|37.33|190.80|223.98|151.38|414.78|2.23|0.00|165.90|223.98|226.21|389.88|392.11|72.60| +2450833|67663|2450913|10784|78591|6845|4252|10784|78591|6845|4252|2|65|15|5|12896|32|945|97|8.20|20.17|19.96|20.37|1936.12|795.40|1956.49|58.08|0.00|410.31|1936.12|1994.20|2346.43|2404.51|1140.72| +2450833|67663|2450838|10784|78591|6845|4252|10784|78591|6845|4252|2|91|1|3|4792|109|945|40|12.55|17.57|11.24|253.20|449.60|502.00|702.80|40.46|0.00|112.40|449.60|490.06|562.00|602.46|-52.40| +2450833|67663|2450858|10784|78591|6845|4252|10784|78591|6845|4252|2|53|12|5|3512|21|945|81|68.20|203.23|67.06|11029.77|5431.86|5524.20|16461.63|434.54|0.00|4773.33|5431.86|5866.40|10205.19|10639.73|-92.34| +2450833|67663|2450919|10784|78591|6845|4252|10784|78591|6845|4252|2|66|13|5|9643|222|945|8|33.62|99.85|62.90|295.60|503.20|268.96|798.80|30.19|0.00|263.60|503.20|533.39|766.80|796.99|234.24| +2450833|26099|2450841|92919|1344232|5945|2458|92919|1344232|5945|2458|2|95|4|3|592|87|946|38|73.02|116.10|75.46|1544.32|2867.48|2774.76|4411.80|86.02|0.00|793.82|2867.48|2953.50|3661.30|3747.32|92.72| +2450833|26099|2450864|92919|1344232|5945|2458|92919|1344232|5945|2458|2|95|9|4|15724|36|946|16|64.41|74.71|20.17|872.64|322.72|1030.56|1195.36|12.90|0.00|549.76|322.72|335.62|872.48|885.38|-707.84| +2450833|26099|2450911|92919|1344232|5945|2458|92919|1344232|5945|2458|2|32|9|4|16580|44|946|65|18.55|20.77|6.64|918.45|431.60|1205.75|1350.05|17.26|86.32|417.95|345.28|362.54|763.23|780.49|-860.47| +2450833|26099|2450908|92919|1344232|5945|2458|92919|1344232|5945|2458|2|18|7|2|3086|34|946|52|83.06|240.87|12.04|11899.16|626.08|4319.12|12525.24|50.08|0.00|2003.56|626.08|676.16|2629.64|2679.72|-3693.04| +2450833|26099|2450902|92919|1344232|5945|2458|92919|1344232|5945|2458|2|96|2|4|1228|210|946|96|1.95|5.22|1.25|381.12|120.00|187.20|501.12|1.20|0.00|179.52|120.00|121.20|299.52|300.72|-67.20| +2450833|26099|2450868|92919|1344232|5945|2458|92919|1344232|5945|2458|2|59|18|5|8095|75|946|33|84.06|154.67|91.25|2092.86|3011.25|2773.98|5104.11|0.00|3011.25|612.48|0.00|0.00|612.48|612.48|-2773.98| +2450833|75947|2450861|67704|21597|3937|11830|67704|21597|3937|11830|1|55|1|1|16819|171|947|12|26.54|64.75|12.30|629.40|147.60|318.48|777.00|5.90|0.00|155.40|147.60|153.50|303.00|308.90|-170.88| +2450833|75947|2450874|67704|21597|3937|11830|67704|21597|3937|11830|1|87|3|1|15656|225|947|99|68.91|188.12|24.45|16203.33|2420.55|6822.09|18623.88|145.23|0.00|2048.31|2420.55|2565.78|4468.86|4614.09|-4401.54| +2450833|75947|2450920|67704|21597|3937|11830|67704|21597|3937|11830|1|46|8|4|244|168|947|37|46.28|99.96|88.96|407.00|3291.52|1712.36|3698.52|296.23|0.00|702.63|3291.52|3587.75|3994.15|4290.38|1579.16| +2450833|75947|2450838|67704|21597|3937|11830|67704|21597|3937|11830|1|100|7|5|5068|220|947|5|43.79|98.08|10.78|436.50|53.90|218.95|490.40|2.15|0.00|88.25|53.90|56.05|142.15|144.30|-165.05| +2450833|75947|2450850|67704|21597|3937|11830|67704|21597|3937|11830|1|84|9|1|17506|248|947|89|48.77|135.09|71.59|5651.50|6371.51|4340.53|12023.01|382.29|0.00|4928.82|6371.51|6753.80|11300.33|11682.62|2030.98| +2450833|75947|2450853|67704|21597|3937|11830|67704|21597|3937|11830|1|57|11|1|11542|88|947|67|28.39|43.72|36.28|498.48|2430.76|1902.13|2929.24|0.00|0.00|58.29|2430.76|2430.76|2489.05|2489.05|528.63| +2450833|75947|2450859|67704|21597|3937|11830|67704|21597|3937|11830|1|22|8|5|2008|129|947|78|44.18|81.29|3.25|6087.12|253.50|3446.04|6340.62|15.21|0.00|2409.42|253.50|268.71|2662.92|2678.13|-3192.54| +2450833|75947|2450856|67704|21597|3937|11830|67704|21597|3937|11830|1|28|20|3|8209|262|947|28|80.64|121.76|59.66|1738.80|1670.48|2257.92|3409.28|33.40|0.00|477.12|1670.48|1703.88|2147.60|2181.00|-587.44| +2450833|75947|2450840|67704|21597|3937|11830|67704|21597|3937|11830|1|51|15|2|17426|166|947|100|75.51|209.91|48.27|16164.00|4827.00|7551.00|20991.00|0.00|2316.96|9655.00|2510.04|2510.04|12165.04|12165.04|-5040.96| +2450833|75947|2450903|67704|21597|3937|11830|67704|21597|3937|11830|1|30|2|5|17509|211|947|18|41.76|96.46|21.22|1354.32|381.96|751.68|1736.28|0.00|53.47|729.18|328.49|328.49|1057.67|1057.67|-423.19| +2450833|75947|2450868|67704|21597|3937|11830|67704|21597|3937|11830|1|71|11|5|11269|255|947|3|89.39|257.44|244.56|38.64|733.68|268.17|772.32|36.68|0.00|301.20|733.68|770.36|1034.88|1071.56|465.51| +2450833|75947|2450900|67704|21597|3937|11830|67704|21597|3937|11830|1|68|7|2|15817|209|947|34|78.07|132.71|15.92|3970.86|541.28|2654.38|4512.14|0.00|0.00|1398.76|541.28|541.28|1940.04|1940.04|-2113.10| +2450833|75947|2450901|67704|21597|3937|11830|67704|21597|3937|11830|1|78|14|3|17332|144|947|55|41.66|57.90|38.79|1051.05|2133.45|2291.30|3184.50|0.00|2069.44|763.95|64.01|64.01|827.96|827.96|-2227.29| +2450833|58167|2450872|55568|754174|6733|21292|55568|754174|6733|21292|2|49|13|3|14314|137|948|71|33.87|87.04|10.44|5438.60|741.24|2404.77|6179.84|14.82|0.00|1792.04|741.24|756.06|2533.28|2548.10|-1663.53| +2450833|58167|2450907|55568|754174|6733|21292|55568|754174|6733|21292|2|37|14|1|13474|41|948|13|89.81|239.79|81.52|2057.51|1059.76|1167.53|3117.27|0.00|731.23|654.55|328.53|328.53|983.08|983.08|-839.00| +2450833|58167|2450909|55568|754174|6733|21292|55568|754174|6733|21292|2|87|7|4|4928|218|948|90|76.71|152.65|24.42|11540.70|2197.80|6903.90|13738.50|65.93|0.00|411.30|2197.80|2263.73|2609.10|2675.03|-4706.10| +2450833|58167|2450849|55568|754174|6733|21292|55568|754174|6733|21292|2|27|20|3|5155|300|948|20|24.53|54.94|20.87|681.40|417.40|490.60|1098.80|20.87|0.00|406.40|417.40|438.27|823.80|844.67|-73.20| +2450833|58167|2450882|55568|754174|6733|21292|55568|754174|6733|21292|2|7|18|3|16454|78|948|85|45.27|49.34|4.44|3816.50|377.40|3847.95|4193.90|22.64|0.00|670.65|377.40|400.04|1048.05|1070.69|-3470.55| +2450833|58167|2450912|55568|754174|6733|21292|55568|754174|6733|21292|2|19|16|1|500|31|948|41|98.54|126.13|47.92|3206.61|1964.72|4040.14|5171.33|39.29|0.00|517.01|1964.72|2004.01|2481.73|2521.02|-2075.42| +2450833|58167|2450916|55568|754174|6733|21292|55568|754174|6733|21292|2|59|11|5|4564|134|948|3|28.67|36.98|28.10|26.64|84.30|86.01|110.94|7.58|0.00|32.16|84.30|91.88|116.46|124.04|-1.71| +2450833|58167|2450884|55568|754174|6733|21292|55568|754174|6733|21292|2|7|18|1|16324|52|948|71|92.52|105.47|87.54|1273.03|6215.34|6568.92|7488.37|559.38|0.00|2994.78|6215.34|6774.72|9210.12|9769.50|-353.58| +2450833|58167|2450923|55568|754174|6733|21292|55568|754174|6733|21292|2|25|2|2|8143|243|948|20|46.63|76.93|38.46|769.40|769.20|932.60|1538.60|38.46|0.00|246.00|769.20|807.66|1015.20|1053.66|-163.40| +2450833|58167|2450839|55568|754174|6733|21292|55568|754174|6733|21292|2|7|12|4|2186|3|948|14|44.80|133.95|70.99|881.44|993.86|627.20|1875.30|19.87|0.00|656.32|993.86|1013.73|1650.18|1670.05|366.66| +2450833|58167|2450884|55568|754174|6733|21292|55568|754174|6733|21292|2|18|2|2|8780|281|948|95|80.27|119.60|50.23|6590.15|4771.85|7625.65|11362.00|190.87|0.00|2953.55|4771.85|4962.72|7725.40|7916.27|-2853.80| +2450833|58167|2450913|55568|754174|6733|21292|55568|754174|6733|21292|2|54|17|3|15181|61|948|49|62.20|90.81|88.08|133.77|4315.92|3047.80|4449.69|388.43|0.00|311.15|4315.92|4704.35|4627.07|5015.50|1268.12| +2450833|58167|2450910|55568|754174|6733|21292|55568|754174|6733|21292|2|37|10|1|15049|238|948|21|21.27|62.53|8.12|1142.61|170.52|446.67|1313.13|15.34|0.00|459.48|170.52|185.86|630.00|645.34|-276.15| +2450833|74775|2450863|43866|677911|3956|46401|43866|677911|3956|46401|2|71|18|3|16225|300|949|32|41.37|58.33|43.74|466.88|1399.68|1323.84|1866.56|97.97|0.00|653.12|1399.68|1497.65|2052.80|2150.77|75.84| +2450833|74775|2450835|43866|677911|3956|46401|43866|677911|3956|46401|2|74|11|4|8533|51|949|96|45.95|86.84|39.94|4502.40|3834.24|4411.20|8336.64|345.08|0.00|1416.96|3834.24|4179.32|5251.20|5596.28|-576.96| +2450833|74775|2450859|43866|677911|3956|46401|43866|677911|3956|46401|2|36|1|1|8522|226|949|92|81.45|115.65|25.44|8299.32|2340.48|7493.40|10639.80|140.42|0.00|4680.96|2340.48|2480.90|7021.44|7161.86|-5152.92| +2450833|74775|2450869|43866|677911|3956|46401|43866|677911|3956|46401|2|86|20|4|4904|1|949|75|55.79|75.87|32.62|3243.75|2446.50|4184.25|5690.25|122.32|0.00|625.50|2446.50|2568.82|3072.00|3194.32|-1737.75| +2450833|74775|2450888|43866|677911|3956|46401|43866|677911|3956|46401|2|24|1|4|12157|176|949|70|11.67|13.77|0.00|963.90|0.00|816.90|963.90|0.00|0.00|366.10|0.00|0.00|366.10|366.10|-816.90| +2450833|74775|2450902|43866|677911|3956|46401|43866|677911|3956|46401|2|54|18|4|13762|189|949|21|7.09|15.81|4.11|245.70|86.31|148.89|332.01|4.31|0.00|19.74|86.31|90.62|106.05|110.36|-62.58| +2450833|33437|2450864|1400|1249405|1057|43109|1400|1249405|1057|43109|2|77|14|2|3163|281|950|6|9.93|21.15|1.48|118.02|8.88|59.58|126.90|0.17|6.74|55.80|2.14|2.31|57.94|58.11|-57.44| +2450833|33437|2450853|1400|1249405|1057|43109|1400|1249405|1057|43109|2|98|13|2|4748|134|950|65|86.62|202.69|152.01|3294.20|9880.65|5630.30|13174.85|261.83|4643.90|921.70|5236.75|5498.58|6158.45|6420.28|-393.55| +2450833|33437|2450867|1400|1249405|1057|43109|1400|1249405|1057|43109|2|54|2|1|13022|276|950|1|35.64|75.20|36.09|39.11|36.09|35.64|75.20|0.50|23.45|18.80|12.64|13.14|31.44|31.94|-23.00| +2450833|33437|2450922|1400|1249405|1057|43109|1400|1249405|1057|43109|2|33|2|5|2956|116|950|45|7.80|22.69|19.74|132.75|888.30|351.00|1021.05|0.00|0.00|61.20|888.30|888.30|949.50|949.50|537.30| +2450833|33437|2450904|1400|1249405|1057|43109|1400|1249405|1057|43109|2|22|9|3|12356|228|950|100|6.77|9.74|0.09|965.00|9.00|677.00|974.00|0.18|0.00|107.00|9.00|9.18|116.00|116.18|-668.00| +2450833|33437|2450895|1400|1249405|1057|43109|1400|1249405|1057|43109|2|91|7|1|4544|281|950|66|42.89|59.18|36.69|1484.34|2421.54|2830.74|3905.88|77.00|1138.12|546.48|1283.42|1360.42|1829.90|1906.90|-1547.32| +2450833|3699|2450871|73256|1702010|4811|39212|73256|1702010|4811|39212|1|36|9|1|3508|147|951|70|98.97|233.56|100.43|9319.10|7030.10|6927.90|16349.20|344.47|2109.03|3596.60|4921.07|5265.54|8517.67|8862.14|-2006.83| +2450833|3699|2450866|73256|1702010|4811|39212|73256|1702010|4811|39212|1|64|12|1|1264|114|951|34|16.02|44.69|31.28|455.94|1063.52|544.68|1519.46|21.27|0.00|75.82|1063.52|1084.79|1139.34|1160.61|518.84| +2450833|3699|2450870|73256|1702010|4811|39212|73256|1702010|4811|39212|1|10|11|4|4225|92|951|38|50.71|78.60|69.95|328.70|2658.10|1926.98|2986.80|159.48|0.00|238.64|2658.10|2817.58|2896.74|3056.22|731.12| +2450833|3699|2450887|73256|1702010|4811|39212|73256|1702010|4811|39212|1|75|15|3|6814|234|951|45|46.09|53.00|17.49|1597.95|787.05|2074.05|2385.00|15.74|0.00|119.25|787.05|802.79|906.30|922.04|-1287.00| +2450833|3699|2450911|73256|1702010|4811|39212|73256|1702010|4811|39212|1|90|14|3|1630|138|951|61|4.68|13.57|9.09|273.28|554.49|285.48|827.77|27.72|0.00|372.10|554.49|582.21|926.59|954.31|269.01| +2450833|3699|2450902|73256|1702010|4811|39212|73256|1702010|4811|39212|1|10|12|1|14188|114|951|67|4.06|11.53|5.30|417.41|355.10|272.02|772.51|10.65|0.00|162.14|355.10|365.75|517.24|527.89|83.08| +2450833|3699|2450893|73256|1702010|4811|39212|73256|1702010|4811|39212|1|102|5|4|5482|25|951|90|18.95|29.18|29.18|0.00|2626.20|1705.50|2626.20|26.26|0.00|839.70|2626.20|2652.46|3465.90|3492.16|920.70| +2450833|22748|2450869|78612|1040835|1458|17234|78612|1040835|1458|17234|4|8|18|1|8179|209|952|31|11.16|16.29|6.67|298.22|206.77|345.96|504.99|13.17|18.60|40.30|188.17|201.34|228.47|241.64|-157.79| +2450833|22748|2450917|78612|1040835|1458|17234|78612|1040835|1458|17234|4|1|7|1|2848|90|952|30|18.84|20.91|17.56|100.50|526.80|565.20|627.30|0.00|0.00|244.50|526.80|526.80|771.30|771.30|-38.40| +2450833|22748|2450918|78612|1040835|1458|17234|78612|1040835|1458|17234|4|72|2|1|15872|229|952|8|31.86|88.57|22.14|531.44|177.12|254.88|708.56|2.65|88.56|42.48|88.56|91.21|131.04|133.69|-166.32| +2450833|22748|2450892|78612|1040835|1458|17234|78612|1040835|1458|17234|4|12|18|5|5521|172|952|52|6.19|6.43|5.65|40.56|293.80|321.88|334.36|0.00|0.00|113.36|293.80|293.80|407.16|407.16|-28.08| +2450833|22748|2450883|78612|1040835|1458|17234|78612|1040835|1458|17234|4|81|15|3|4696|52|952|91|1.10|1.69|0.60|99.19|54.60|100.10|153.79|0.52|48.04|42.77|6.56|7.08|49.33|49.85|-93.54| +2450833|75575|2450845|326|727569|1659|37749|326|727569|1659|37749|1|40|16|2|15916|299|953|84|49.95|83.41|14.17|5816.16|1190.28|4195.80|7006.44|71.41|0.00|489.72|1190.28|1261.69|1680.00|1751.41|-3005.52| +|75575||326||1659|||727569|1659|||||4|3487|273|953||48.84|96.21|||||6734.70|303.03|||||6127.80|6430.83|-51.80| +2450833|75575|2450901|326|727569|1659|37749|326|727569|1659|37749|1|80|5|2|3655|121|953|17|56.04|62.20|23.63|655.69|401.71|952.68|1057.40|25.58|36.15|200.77|365.56|391.14|566.33|591.91|-587.12| +2450833|75575|2450860|326|727569|1659|37749|326|727569|1659|37749|1|50|7|3|4561|115|953|57|5.74|13.25|11.79|83.22|672.03|327.18|755.25|20.16|0.00|279.30|672.03|692.19|951.33|971.49|344.85| +2450833|75575|2450876|326|727569|1659|37749|326|727569|1659|37749|1|46|19|5|3103|46|953|64|68.52|195.96|137.17|3762.56|8778.88|4385.28|12541.44|351.15|0.00|2257.28|8778.88|9130.03|11036.16|11387.31|4393.60| +2450833|75575|2450844|326|727569|1659|37749|326|727569|1659|37749|1|41|19|1|7780|57|953|80|47.27|127.62|74.01|4288.80|5920.80|3781.60|10209.60|118.41|0.00|408.00|5920.80|6039.21|6328.80|6447.21|2139.20| +2450833|75575|2450862|326|727569|1659|37749|326|727569|1659|37749|1|11|4|2|4669|298|953|19|48.12|66.40|23.90|807.50|454.10|914.28|1261.60|31.78|0.00|290.13|454.10|485.88|744.23|776.01|-460.18| +2450833|23176|2450914|4276|1879261|7046|22431|4276|1879261|7046|22431|2|77|16|2|16156|184|954|88|62.34|142.13|106.59|3127.52|9379.92|5485.92|12507.44|750.39|0.00|374.88|9379.92|10130.31|9754.80|10505.19|3894.00| +2450833|23176|2450878|4276|1879261|7046|22431|4276|1879261|7046|22431|2|102|1|3|3674|252|954|20|13.59|20.65|7.84|256.20|156.80|271.80|413.00|10.42|7.84|198.20|148.96|159.38|347.16|357.58|-122.84| +2450833|23176|2450864|4276|1879261|7046|22431|4276|1879261|7046|22431|2|55|6|2|1381|226|954|72|43.61|44.48|1.77|3075.12|127.44|3139.92|3202.56|6.37|0.00|672.48|127.44|133.81|799.92|806.29|-3012.48| +2450833|23176|2450858|4276|1879261|7046|22431|4276|1879261|7046|22431|2|105|18|4|7909|283|954|28|3.43|3.56|1.95|45.08|54.60|96.04|99.68|1.63|0.00|43.68|54.60|56.23|98.28|99.91|-41.44| +2450833|23176|2450874|4276|1879261|7046|22431|4276|1879261|7046|22431|2|48|10|3|13843|211|954|82|41.29|104.87|36.70|5589.94|3009.40|3385.78|8599.34|90.28|0.00|1117.66|3009.40|3099.68|4127.06|4217.34|-376.38| +2450833|23176|2450890|4276|1879261|7046|22431|4276|1879261|7046|22431|2|71|20|1|5911|188|954|48|50.44|52.96|35.48|839.04|1703.04|2421.12|2542.08|153.27|0.00|1194.72|1703.04|1856.31|2897.76|3051.03|-718.08| +2450833|23176|2450872|4276|1879261|7046|22431|4276|1879261|7046|22431|2|34|8|1|12544|37|954|49|42.00|76.86|4.61|3540.25|225.89|2058.00|3766.14|2.25|0.00|1581.72|225.89|228.14|1807.61|1809.86|-1832.11| +2450833|23176|2450843|4276|1879261|7046|22431|4276|1879261|7046|22431|2|40|2|3|16868|250|954|56|49.99|88.48|66.36|1238.72|3716.16|2799.44|4954.88|260.13|0.00|1486.24|3716.16|3976.29|5202.40|5462.53|916.72| +2450833|23176|2450917|4276|1879261|7046|22431|4276|1879261|7046|22431|2|8|9|3|13738|54|954|67|30.25|53.54|21.41|2152.71|1434.47|2026.75|3587.18|129.10|0.00|1542.34|1434.47|1563.57|2976.81|3105.91|-592.28| +2450833|67862|2450912|32749|1310917|4285|32500|32749|1310917|4285|32500|1|65|16|5|12385|241|955|73|64.59|118.84|68.92|3644.16|5031.16|4715.07|8675.32|100.62|0.00|3295.95|5031.16|5131.78|8327.11|8427.73|316.09| +2450833|67862|2450853|32749|1310917|4285|32500|32749|1310917|4285|32500|1|27|16|4|13339|282|955|13|89.72|105.86|4.23|1321.19|54.99|1166.36|1376.18|2.74|0.00|577.98|54.99|57.73|632.97|635.71|-1111.37| +2450833|67862|2450846|32749|1310917|4285|32500|32749|1310917|4285|32500|1|74|7|4|8020|15|955|54|2.08|5.82|3.37|132.30|181.98|112.32|314.28|9.09|0.00|128.52|181.98|191.07|310.50|319.59|69.66| +2450833|67862|2450881|32749|1310917|4285|32500|32749|1310917|4285|32500|1|4|10|5|2480|122|955|45|91.99|127.86|28.12|4488.30|1265.40|4139.55|5753.70|5.31|1088.24|1093.05|177.16|182.47|1270.21|1275.52|-3962.39| +2450833|67862|2450908|32749|1310917|4285|32500|32749|1310917|4285|32500|1|3|11|5|1957|3|955|76|91.32|241.08|103.66|10443.92|7878.16|6940.32|18322.08|472.68|0.00|2197.92|7878.16|8350.84|10076.08|10548.76|937.84| +2450833|67862|2450904|32749|1310917|4285|32500|32749|1310917|4285|32500|1|66|18|4|15484|2|955|69|84.41|241.41|43.45|13659.24|2998.05|5824.29|16657.29|0.00|0.00|4996.98|2998.05|2998.05|7995.03|7995.03|-2826.24| +2450833|67862|2450923|32749|1310917|4285|32500|32749|1310917|4285|32500|1|55|19|2|16816|131|955|87|15.24|40.08|6.01|2964.09|522.87|1325.88|3486.96|36.60|0.00|627.27|522.87|559.47|1150.14|1186.74|-803.01| +2450833|67862|2450861|32749|1310917|4285|32500|32749|1310917|4285|32500|1|39|6|4|10624|272|955|44|95.95|133.37|17.33|5105.76|762.52|4221.80|5868.28|15.25|0.00|704.00|762.52|777.77|1466.52|1481.77|-3459.28| +2450833|67862|2450851|32749|1310917|4285|32500|32749|1310917|4285|32500|1|53|15|3|4198|134|955|10|54.14|70.38|61.93|84.50|619.30|541.40|703.80|12.38|0.00|351.90|619.30|631.68|971.20|983.58|77.90| +2450833|67862|2450903|32749|1310917|4285|32500|32749|1310917|4285|32500|1|36|8|3|206|67|955|11|80.71|120.25|68.54|568.81|753.94|887.81|1322.75|15.07|0.00|158.73|753.94|769.01|912.67|927.74|-133.87| +2450833|28021|2450853|89644|796602|3560|17271|89644|796602|3560|17271|4|43|10|4|9532|163|956|50|80.33|106.83|41.66|3258.50|2083.00|4016.50|5341.50|187.47|0.00|2350.00|2083.00|2270.47|4433.00|4620.47|-1933.50| +2450833|28021|2450874|89644|796602|3560|17271|89644|796602|3560|17271|4|95|4|5|5228|75|956|47|22.21|26.20|16.76|443.68|787.72|1043.87|1231.40|61.12|23.63|184.71|764.09|825.21|948.80|1009.92|-279.78| +2450833|28021|2450874|89644|796602|3560|17271|89644|796602|3560|17271|4|4|20|3|14704|211|956|44|24.47|41.35|33.49|345.84|1473.56|1076.68|1819.40|29.47|0.00|218.24|1473.56|1503.03|1691.80|1721.27|396.88| +2450833|28021|2450869|89644|796602|3560|17271|89644|796602|3560|17271|4|103|3|3|10357|33|956|49|3.62|9.33|1.58|379.75|77.42|177.38|457.17|1.54|0.00|95.55|77.42|78.96|172.97|174.51|-99.96| +2450833|84898|2450858|26461|274540|6972|47404|26461|274540|6972|47404|4|79|13|3|1582|280|957|66|3.76|6.16|1.17|329.34|77.22|248.16|406.56|0.77|0.00|101.64|77.22|77.99|178.86|179.63|-170.94| +2450833|84898|2450841|26461|274540|6972|47404|26461|274540|6972|47404|4|50|16|4|1279|223|957|97|42.53|108.02|18.36|8697.02|1780.92|4125.41|10477.94|71.23|0.00|5238.97|1780.92|1852.15|7019.89|7091.12|-2344.49| +2450833|84898|2450920|26461|274540|6972|47404|26461|274540|6972|47404|4|20|18|3|8425|200|957|89|56.21|77.00|45.43|2809.73|4043.27|5002.69|6853.00|283.02|0.00|3083.85|4043.27|4326.29|7127.12|7410.14|-959.42| +2450833|84898|2450916|26461|274540|6972|47404|26461|274540|6972|47404|4|55|6|3|9410|90|957|20|63.24|187.82|116.44|1427.60|2328.80|1264.80|3756.40|209.59|0.00|0.00|2328.80|2538.39|2328.80|2538.39|1064.00| +2450833|84898|2450845|26461|274540|6972|47404|26461|274540|6972|47404|4|25|16|5|15878|116|957|68|89.35|125.09|0.00|8506.12|0.00|6075.80|8506.12|0.00|0.00|1020.68|0.00|0.00|1020.68|1020.68|-6075.80| +2450833|84898|2450839|26461|274540|6972|47404|26461|274540|6972|47404|4|106|2|5|8624|51|957|75|61.86|141.65|32.57|8181.00|2442.75|4639.50|10623.75|101.13|1319.08|2124.75|1123.67|1224.80|3248.42|3349.55|-3515.83| +2450833|84898|2450865|26461|274540|6972|47404|26461|274540|6972|47404|4|57|11|5|11128|257|957|99|93.35|171.76|168.32|340.56|16663.68|9241.65|17004.24|179.96|10664.75|0.00|5998.93|6178.89|5998.93|6178.89|-3242.72| +2450833|84898|2450866|26461|274540|6972|47404|26461|274540|6972|47404|4|95|8|2|9097|290|957|23|83.57|162.96|120.59|974.51|2773.57|1922.11|3748.08|26.34|138.67|1874.04|2634.90|2661.24|4508.94|4535.28|712.79| +2450833|60430|2450909|68103|1060335|214|39646|68103|1060335|214|39646|1|73|6|4|12877|218|958|89|51.94|121.53|7.29|10167.36|648.81|4622.66|10816.17|27.70|253.03|865.08|395.78|423.48|1260.86|1288.56|-4226.88| +2450833|60430|2450895|68103|1060335|214|39646|68103|1060335|214|39646|1|86|6|2|3220|218|958|61|94.12|189.18|58.64|7962.94|3577.04|5741.32|11539.98|250.39|0.00|4731.16|3577.04|3827.43|8308.20|8558.59|-2164.28| +2450833|60430|2450900|68103|1060335|214|39646|68103|1060335|214|39646|1|81|15|4|14090|222|958|46|79.78|106.90|88.72|836.28|4081.12|3669.88|4917.40|244.86|0.00|885.04|4081.12|4325.98|4966.16|5211.02|411.24| +2450833|60430|2450875|68103|1060335|214|39646|68103|1060335|214|39646|1|71|2|3|9080|35|958|10|65.34|158.12|55.34|1027.80|553.40|653.40|1581.20|5.53|0.00|332.00|553.40|558.93|885.40|890.93|-100.00| +2450833|71699|2450845|51528|1597198|4468|11291|51528|1597198|4468|11291|4|22|7|4|10639|231|959|55|19.44|28.57|12.28|895.95|675.40|1069.20|1571.35|47.27|0.00|628.10|675.40|722.67|1303.50|1350.77|-393.80| +2450833|71699|2450923|51528|1597198|4468|11291|51528|1597198|4468|11291|4|107|7|3|6562|153|959|89|49.04|91.70|79.77|1061.77|7099.53|4364.56|8161.30|70.28|6318.58|1632.26|780.95|851.23|2413.21|2483.49|-3583.61| +2450833|71699|2450917|51528|1597198|4468|11291|51528|1597198|4468|11291|4|84|15|3|7546|84|959|88|37.06|83.38|24.18|5209.60|2127.84|3261.28|7337.44|63.83|0.00|1613.92|2127.84|2191.67|3741.76|3805.59|-1133.44| +2450833|71699|2450891|51528|1597198|4468|11291|51528|1597198|4468|11291|4|24|12|3|17473|138|959|93|80.51|120.76|12.07|10108.17|1122.51|7487.43|11230.68|53.31|56.12|3144.33|1066.39|1119.70|4210.72|4264.03|-6421.04| +2450833|71699|2450885|51528|1597198|4468|11291|51528|1597198|4468|11291|4|79|9|1|1954|5|959|60|11.13|27.93|20.38|453.00|1222.80|667.80|1675.80|36.68|0.00|770.40|1222.80|1259.48|1993.20|2029.88|555.00| +2450833|71699|2450873|51528|1597198|4468|11291|51528|1597198|4468|11291|4|6|4|5|17362|94|959|57|48.10|53.87|28.01|1474.02|1596.57|2741.70|3070.59|15.96|0.00|368.22|1596.57|1612.53|1964.79|1980.75|-1145.13| +2450833|76890|2450910|35338|1248392|5446|30100|35338|1248392|5446|30100|2|36|16|4|12950|115|960|14|91.58|155.68|1.55|2157.82|21.70|1282.12|2179.52|1.95|0.00|413.98|21.70|23.65|435.68|437.63|-1260.42| +2450833|76890|2450848|35338|1248392|5446|30100|35338|1248392|5446|30100|2|35|9|2|7657|148|960|84|98.35|222.27|120.02|8589.00|10081.68|8261.40|18670.68|100.81|0.00|4667.04|10081.68|10182.49|14748.72|14849.53|1820.28| +2450833|76890|2450887|35338|1248392|5446|30100|35338|1248392|5446|30100|2|34|6|3|5770|109|960|2|88.90|122.68|78.51|88.34|157.02|177.80|245.36|1.47|83.22|105.50|73.80|75.27|179.30|180.77|-104.00| +2450833|76890|2450916|35338|1248392|5446|30100|35338|1248392|5446|30100|2|89|17|4|11510|120|960|69|27.57|63.68|24.83|2680.65|1713.27|1902.33|4393.92|85.66|0.00|1889.22|1713.27|1798.93|3602.49|3688.15|-189.06| +2450833|76890|2450866|35338|1248392|5446|30100|35338|1248392|5446|30100|2|27|17|1|6076|231|960|66|89.27|136.58|43.70|6130.08|2884.20|5891.82|9014.28|0.28|2855.35|2162.82|28.85|29.13|2191.67|2191.95|-5862.97| +2450833|52418|2450849|50487|1668416|5192|21121|50487|1668416|5192|21121|2|61|5|2|11564|19|961|65|35.78|86.94|58.24|1865.50|3785.60|2325.70|5651.10|37.85|0.00|904.15|3785.60|3823.45|4689.75|4727.60|1459.90| +2450833|52418|2450917|50487|1668416|5192|21121|50487|1668416|5192|21121|2|16|17|4|16507|193|961|5|92.59|123.14|18.47|523.35|92.35|462.95|615.70|3.69|0.00|270.90|92.35|96.04|363.25|366.94|-370.60| +2450833|52418|2450914|50487|1668416|5192|21121|50487|1668416|5192|21121|2|14|11|4|8000|47|961|91|63.72|181.60|0.00|16525.60|0.00|5798.52|16525.60|0.00|0.00|2147.60|0.00|0.00|2147.60|2147.60|-5798.52| +2450833|52418|2450909|50487|1668416|5192|21121|50487|1668416|5192|21121|2|79|19|2|1312|113|961|95|59.08|86.84|22.57|6105.65|2144.15|5612.60|8249.80|42.88|0.00|1897.15|2144.15|2187.03|4041.30|4084.18|-3468.45| +2450833|52418|2450915|50487|1668416|5192|21121|50487|1668416|5192|21121|2|32|3|4|3500|66|961|16|56.45|66.04|15.18|813.76|242.88|903.20|1056.64|14.57|0.00|327.52|242.88|257.45|570.40|584.97|-660.32| +2450833|52418|2450888|50487|1668416|5192|21121|50487|1668416|5192|21121|2|98|9|1|11566|242|961|52|86.14|150.74|34.67|6035.64|1802.84|4479.28|7838.48|18.02|0.00|2272.92|1802.84|1820.86|4075.76|4093.78|-2676.44| +2450833|52418|2450895|50487|1668416|5192|21121|50487|1668416|5192|21121|2|73|4|4|5395|93|961|25|94.70|147.73|130.00|443.25|3250.00|2367.50|3693.25|0.00|0.00|1772.75|3250.00|3250.00|5022.75|5022.75|882.50| +2450833|52418|2450873|50487|1668416|5192|21121|50487|1668416|5192|21121|2|102|11|3|11678|154|961|77|94.38|261.43|245.74|1208.13|18921.98|7267.26|20130.11|1513.75|0.00|1409.10|18921.98|20435.73|20331.08|21844.83|11654.72| +2450833|52418|2450914|50487|1668416|5192|21121|50487|1668416|5192|21121|2|28|16|2|8497|63|961|75|43.04|60.25|10.24|3750.75|768.00|3228.00|4518.75|23.04|0.00|1761.75|768.00|791.04|2529.75|2552.79|-2460.00| +2450833|42622|2450917|28009|1894820|4031|17195|28009|1894820|4031|17195|2|71|13|3|14755|223|962|17|81.36|96.81|47.43|839.46|806.31|1383.12|1645.77|32.25|0.00|444.21|806.31|838.56|1250.52|1282.77|-576.81| +2450833|42622|2450857|28009|1894820|4031|17195|28009|1894820|4031|17195|2|53|14|4|682|137|962|76|53.68|96.08|8.64|6645.44|656.64|4079.68|7302.08|13.13|0.00|2336.24|656.64|669.77|2992.88|3006.01|-3423.04| +2450833|42622|2450872|28009|1894820|4031|17195|28009|1894820|4031|17195|2|99|2|5|5572|156|962|87|79.25|107.78|47.42|5251.32|4125.54|6894.75|9376.86|206.27|0.00|281.01|4125.54|4331.81|4406.55|4612.82|-2769.21| +2450833|42622|2450856|28009|1894820|4031|17195|28009|1894820|4031|17195|2|75|12|2|10472|47|962|67|55.08|89.78|7.18|5534.20|481.06|3690.36|6015.26|19.24|0.00|1382.88|481.06|500.30|1863.94|1883.18|-3209.30| +2450833|42622|2450885|28009|1894820|4031|17195|28009|1894820|4031|17195|2|29|19|3|3904|229|962|56|50.40|66.02|21.12|2514.40|1182.72|2822.40|3697.12|59.13|0.00|295.68|1182.72|1241.85|1478.40|1537.53|-1639.68| +2450833|42622|2450869|28009|1894820|4031|17195|28009|1894820|4031|17195|2|90|19|4|979|141|962|1|82.61|245.35|93.23|152.12|93.23|82.61|245.35|0.93|0.00|58.88|93.23|94.16|152.11|153.04|10.62| +2450833|42622|2450876|28009|1894820|4031|17195|28009|1894820|4031|17195|2|48|8|3|16375|42|962|6|75.63|130.83|94.19|219.84|565.14|453.78|784.98|5.65|0.00|86.34|565.14|570.79|651.48|657.13|111.36| +2450833|42622|2450903|28009|1894820|4031|17195|28009|1894820|4031|17195|2|56|1|5|11540|209|962|85|30.86|39.19|32.91|533.80|2797.35|2623.10|3331.15|111.89|0.00|1498.55|2797.35|2909.24|4295.90|4407.79|174.25| +2450833|66609|2450895|21451|829087|1523|29959|21451|829087|1523|29959|4|1|10|2|9182|255|963|21|8.41|11.85|8.53|69.72|179.13|176.61|248.85|1.79|0.00|54.60|179.13|180.92|233.73|235.52|2.52| +2450833|66609|2450898|21451|829087|1523|29959|21451|829087|1523|29959|4|54|1|2|7510|153|963|36|83.66|151.42|121.13|1090.44|4360.68|3011.76|5451.12|305.24|0.00|1798.56|4360.68|4665.92|6159.24|6464.48|1348.92| +2450833|66609|2450845|21451|829087|1523|29959|21451|829087|1523|29959|4|73|15|4|13726|55|963|9|77.43|91.36|74.91|148.05|674.19|696.87|822.24|60.67|0.00|411.12|674.19|734.86|1085.31|1145.98|-22.68| +2450833|66609|2450865|21451|829087|1523|29959|21451|829087|1523|29959|4|25|8|5|16246|244|963|53|67.50|110.02|44.00|3499.06|2332.00|3577.50|5831.06|15.39|2075.48|466.40|256.52|271.91|722.92|738.31|-3320.98| +2450833|66609|2450864|21451|829087|1523|29959|21451|829087|1523|29959|4|21|18|1|10897|216|963|10|8.64|13.73|10.29|34.40|102.90|86.40|137.30|6.17|0.00|65.90|102.90|109.07|168.80|174.97|16.50| +2450833|66609|2450879|21451|829087|1523|29959|21451|829087|1523|29959|4|85|4|2|14030|26|963|82|64.70|144.28|70.69|6034.38|5796.58|5305.40|11830.96|0.00|0.00|2838.84|5796.58|5796.58|8635.42|8635.42|491.18| +2450833|37684|2450846|86824|1604453|3682|39711|86824|1604453|3682|39711|2|98|17|3|7219|175|964|6|53.35|124.83|92.37|194.76|554.22|320.10|748.98|11.08|0.00|157.26|554.22|565.30|711.48|722.56|234.12| +2450833|37684|2450907|86824|1604453|3682|39711|86824|1604453|3682|39711|2|20|19|3|14323|64|964|35|60.12|140.68|125.20|541.80|4382.00|2104.20|4923.80|3.94|4338.18|886.20|43.82|47.76|930.02|933.96|-2060.38| +2450833|37684|2450898|86824|1604453|3682|39711|86824|1604453|3682|39711|2|65|14|5|11332|215|964|14|27.13|57.51|4.60|740.74|64.40|379.82|805.14|2.42|34.13|394.38|30.27|32.69|424.65|427.07|-349.55| +2450833|37684|2450915|86824|1604453|3682|39711|86824|1604453|3682|39711|2|4|17|4|13546|178|964|78|84.89|158.74|141.27|1362.66|11019.06|6621.42|12381.72|330.57|0.00|2847.78|11019.06|11349.63|13866.84|14197.41|4397.64| +2450833|37684|2450886|86824|1604453|3682|39711|86824|1604453|3682|39711|2|69|7|4|2077|154|964|95|20.37|56.62|11.32|4303.50|1075.40|1935.15|5378.90|43.01|0.00|1129.55|1075.40|1118.41|2204.95|2247.96|-859.75| +2450833|37684|2450914|86824|1604453|3682|39711|86824|1604453|3682|39711|2|83|16|1|12139|210|964|71|24.20|58.32|35.57|1615.25|2525.47|1718.20|4140.72|75.76|0.00|1241.79|2525.47|2601.23|3767.26|3843.02|807.27| +2450833|37684|2450841|86824|1604453|3682|39711|86824|1604453|3682|39711|2|70|4|1|10960|221|964|85|66.51|72.49|29.72|3635.45|2526.20|5653.35|6161.65|50.52|0.00|492.15|2526.20|2576.72|3018.35|3068.87|-3127.15| +2450833|37684|2450903|86824|1604453|3682|39711|86824|1604453|3682|39711|2|75|8|1|14536|7|964|40|80.43|169.70|72.97|3869.20|2918.80|3217.20|6788.00|129.59|758.88|67.60|2159.92|2289.51|2227.52|2357.11|-1057.28| +2450833|25347|2450892|52574|1806862|4350|19791|52574|1806862|4350|19791|2|74|18|1|1354|240|965|18|71.18|153.74|76.87|1383.66|1383.66|1281.24|2767.32|96.85|0.00|1079.10|1383.66|1480.51|2462.76|2559.61|102.42| +2450833|25347|2450917|52574|1806862|4350|19791|52574|1806862|4350|19791|2|40|11|4|5210|180|965|69|54.61|132.15|97.79|2370.84|6747.51|3768.09|9118.35|134.95|0.00|4102.74|6747.51|6882.46|10850.25|10985.20|2979.42| +2450833|25347|2450913|52574|1806862|4350|19791|52574|1806862|4350|19791|2|91|2|4|16288|245|965|93|49.31|57.69|8.65|4560.72|804.45|4585.83|5365.17|16.08|0.00|267.84|804.45|820.53|1072.29|1088.37|-3781.38| +2450833|25347|2450901|52574|1806862|4350|19791|52574|1806862|4350|19791|2|97|4|4|13690|26|965|81|29.23|79.50|77.91|128.79|6310.71|2367.63|6439.50|441.74|0.00|1416.69|6310.71|6752.45|7727.40|8169.14|3943.08| +2450833|25347|2450839|52574|1806862|4350|19791|52574|1806862|4350|19791|2|38|8|3|17060|198|965|79|37.11|81.64|68.57|1032.53|5417.03|2931.69|6449.56|108.34|0.00|838.19|5417.03|5525.37|6255.22|6363.56|2485.34| +2450833|45727|2450874|96382|150391|6353|40362|96382|150391|6353|40362|2|78|8|5|2630|28|966|84|63.48|175.83|138.90|3102.12|11667.60|5332.32|14769.72|350.02|0.00|1476.72|11667.60|12017.62|13144.32|13494.34|6335.28| +2450833|45727|2450841|96382|150391|6353|40362|96382|150391|6353|40362|2|44|8|2|10231|95|966|77|2.50|7.27|6.25|78.54|481.25|192.50|559.79|38.50|0.00|217.91|481.25|519.75|699.16|737.66|288.75| +2450833|45727|2450836|96382|150391|6353|40362|96382|150391|6353|40362|2|45|19|5|2738|59|966|14|16.90|25.51|17.09|117.88|239.26|236.60|357.14|6.53|145.94|114.24|93.32|99.85|207.56|214.09|-143.28| +2450833|45727|2450900|96382|150391|6353|40362|96382|150391|6353|40362|2|75|1|1|8662|221|966|95|60.29|159.16|63.66|9072.50|6047.70|5727.55|15120.20|302.38|0.00|6500.85|6047.70|6350.08|12548.55|12850.93|320.15| +2450833|45727|2450916|96382|150391|6353|40362|96382|150391|6353|40362|2|46|7|4|4735|48|966|12|90.47|123.94|26.02|1175.04|312.24|1085.64|1487.28|7.68|56.20|743.64|256.04|263.72|999.68|1007.36|-829.60| +2450833|45727|2450909|96382|150391|6353|40362|96382|150391|6353|40362|2|50|18|2|13159|62|966|94|63.79|149.90|133.41|1550.06|12540.54|5996.26|14090.60|203.15|2382.70|5636.24|10157.84|10360.99|15794.08|15997.23|4161.58| +2450833|45727|2450908|96382|150391|6353|40362|96382|150391|6353|40362|2|41|18|5|5650|107|966|37|99.77|207.52|16.60|7064.04|614.20|3691.49|7678.24|6.14|0.00|1765.64|614.20|620.34|2379.84|2385.98|-3077.29| +2450833|65573|2450859|91478|179235|6639|29243|91478|179235|6639|29243|1|2|20|5|10094|63|967|16|66.06|110.98|102.10|142.08|1633.60|1056.96|1775.68|130.68|0.00|550.40|1633.60|1764.28|2184.00|2314.68|576.64| +2450833|65573|2450867|91478|179235|6639|29243|91478|179235|6639|29243|1|25|9|5|12232|256|967|8|6.58|13.09|1.17|95.36|9.36|52.64|104.72|0.65|0.00|35.60|9.36|10.01|44.96|45.61|-43.28| +2450833|65573|2450891|91478|179235|6639|29243|91478|179235|6639|29243|1|46|18|3|1430|200|967|18|29.60|77.84|12.45|1177.02|224.10|532.80|1401.12|11.20|0.00|266.04|224.10|235.30|490.14|501.34|-308.70| +2450833|65573|2450837|91478|179235|6639|29243|91478|179235|6639|29243|1|62|11|3|4766|40|967|48|28.70|47.06|30.58|791.04|1467.84|1377.60|2258.88|117.42|0.00|519.36|1467.84|1585.26|1987.20|2104.62|90.24| +2450833|65573|2450865|91478|179235|6639|29243|91478|179235|6639|29243|1|37|14|2|8485|261|967|32|18.00|43.92|22.83|674.88|730.56|576.00|1405.44|0.00|292.22|547.84|438.34|438.34|986.18|986.18|-137.66| +2450833|10618|2450882|21984|281933|7129|6678|21984|281933|7129|6678|1|50|9|2|3505|263|968|49|33.26|90.46|59.70|1507.24|2925.30|1629.74|4432.54|175.51|0.00|132.79|2925.30|3100.81|3058.09|3233.60|1295.56| +2450833|10618|2450844|21984|281933|7129|6678|21984|281933|7129|6678|1|8|4|1|12619|3|968|83|75.78|189.45|15.15|14466.90|1257.45|6289.74|15724.35|12.57|0.00|4087.75|1257.45|1270.02|5345.20|5357.77|-5032.29| +2450833|10618|2450874|21984|281933|7129|6678|21984|281933|7129|6678|1|81|15|2|15331|168|968|3|52.70|112.25|31.43|242.46|94.29|158.10|336.75|3.77|0.00|101.01|94.29|98.06|195.30|199.07|-63.81| +2450833|10618|2450859|21984|281933|7129|6678|21984|281933|7129|6678|1|22|9|5|2017|181|968|58|18.69|43.54|1.30|2449.92|75.40|1084.02|2525.32|0.72|51.27|858.40|24.13|24.85|882.53|883.25|-1059.89| +2450833|10618|2450878|21984|281933|7129|6678|21984|281933|7129|6678|1|35|13|4|5398|64|968|68|34.13|44.36|5.76|2624.80|391.68|2320.84|3016.48|35.25|0.00|663.00|391.68|426.93|1054.68|1089.93|-1929.16| +2450833|10618|2450900|21984|281933|7129|6678|21984|281933|7129|6678|1|99|11|5|17248|227|968|10|58.32|71.73|37.29|344.40|372.90|583.20|717.30|18.64|0.00|136.20|372.90|391.54|509.10|527.74|-210.30| +2450833|54279|2450876|20820|1094281|895|19156|20820|1094281|895|19156|1|27|3|5|8852|14|969|67|88.54|110.67|101.81|593.62|6821.27|5932.18|7414.89|0.00|0.00|3336.60|6821.27|6821.27|10157.87|10157.87|889.09| +2450833|54279|2450890|20820|1094281|895|19156|20820|1094281|895|19156|1|23|7|1|7468|140|969|76|78.89|186.18|44.68|10754.00|3395.68|5995.64|14149.68|33.95|0.00|706.80|3395.68|3429.63|4102.48|4136.43|-2599.96| +2450833|54279|2450895|20820|1094281|895|19156|20820|1094281|895|19156|1|33|6|1|10159|126|969|75|64.21|170.79|56.36|8582.25|4227.00|4815.75|12809.25|169.08|0.00|5123.25|4227.00|4396.08|9350.25|9519.33|-588.75| +2450833|54279|2450901|20820|1094281|895|19156|20820|1094281|895|19156|1|30|3|5|4627|206|969|65|89.50|145.88|36.47|7111.65|2370.55|5817.50|9482.20|142.23|0.00|2939.30|2370.55|2512.78|5309.85|5452.08|-3446.95| +2450833|54279|2450839|20820|1094281|895|19156|20820|1094281|895|19156|1|69|13|3|17906|133|969|91|58.43|139.64|6.98|12072.06|635.18|5317.13|12707.24|16.45|400.16|1524.25|235.02|251.47|1759.27|1775.72|-5082.11| +2450833|54279|2450889|20820|1094281|895|19156|20820|1094281|895|19156|1|33|3|2|3535|138|969|30|66.80|122.24|101.45|623.70|3043.50|2004.00|3667.20|60.87|0.00|293.10|3043.50|3104.37|3336.60|3397.47|1039.50| +2450833|54279|2450874|20820|1094281|895|19156|20820|1094281|895|19156|1|16|4|4|4732|136|969|62|25.71|76.61|13.02|3942.58|807.24|1594.02|4749.82|40.36|0.00|1471.88|807.24|847.60|2279.12|2319.48|-786.78| +2450833|75856|2450888|3179|1353628|877|20526|3179|1353628|877|20526|4|55|7|2|5866|129|970|35|42.60|55.38|54.82|19.60|1918.70|1491.00|1938.30|111.66|57.56|193.55|1861.14|1972.80|2054.69|2166.35|370.14| +2450833|75856|2450880|3179|1353628|877|20526|3179|1353628|877|20526|4|20|20|2|2341|34|970|39|48.06|139.37|16.72|4783.35|652.08|1874.34|5435.43|0.00|0.00|1141.14|652.08|652.08|1793.22|1793.22|-1222.26| +2450833|75856|2450903|3179|1353628|877|20526|3179|1353628|877|20526|4|4|1|2|7522|249|970|70|93.49|128.08|103.74|1703.80|7261.80|6544.30|8965.60|21.78|6898.71|3675.70|363.09|384.87|4038.79|4060.57|-6181.21| +2450833|75856|2450878|3179|1353628|877|20526|3179|1353628|877|20526|4|45|5|4|3254|295|970|67|60.79|145.28|97.33|3212.65|6521.11|4072.93|9733.76|65.21|0.00|389.27|6521.11|6586.32|6910.38|6975.59|2448.18| +2450833|75856|2450847|3179|1353628|877|20526|3179|1353628|877|20526|4|38|13|4|10327|278|970|3|91.46|208.52|198.09|31.29|594.27|274.38|625.56|11.88|0.00|256.47|594.27|606.15|850.74|862.62|319.89| +2450833|75856|2450890|3179|1353628|877|20526|3179|1353628|877|20526|4|2|3|5|17840|249|970|36|94.47|124.70|92.27|1167.48|3321.72|3400.92|4489.20|132.86|0.00|583.56|3321.72|3454.58|3905.28|4038.14|-79.20| +2450833|75856|2450865|3179|1353628|877|20526|3179|1353628|877|20526|4|57|16|5|11408|146|970|73|8.09|10.67|7.36|241.63|537.28|590.57|778.91|0.00|0.00|0.00|537.28|537.28|537.28|537.28|-53.29| +2450833|75856|2450888|3179|1353628|877|20526|3179|1353628|877|20526|4|4|8|5|1030|201|970|60|97.10|128.17|112.78|923.40|6766.80|5826.00|7690.20|205.71|1624.03|3691.20|5142.77|5348.48|8833.97|9039.68|-683.23| +2450833|75856|2450917|3179|1353628|877|20526|3179|1353628|877|20526|4|75|6|4|4340|115|970|8|41.11|96.19|40.39|446.40|323.12|328.88|769.52|3.23|0.00|184.64|323.12|326.35|507.76|510.99|-5.76| +2450834|70454|2450850|68237|1250390|4543|31849|68237|1250390|4543|31849|4|66|3|5|6004|2|971|83|50.56|144.60|127.24|1440.88|10560.92|4196.48|12001.80|739.26|0.00|1200.18|10560.92|11300.18|11761.10|12500.36|6364.44| +2450834|70454|2450889|68237|1250390|4543|31849|68237|1250390|4543|31849|4|74|16|4|14882|6|971|90|46.33|52.81|32.21|1854.00|2898.90|4169.70|4752.90|28.98|0.00|998.10|2898.90|2927.88|3897.00|3925.98|-1270.80| +2450834|70454|2450853|68237|1250390|4543|31849|68237|1250390|4543|31849|4|51|20|1|7465|64|971|14|34.85|55.41|44.88|147.42|628.32|487.90|775.74|25.13|0.00|162.82|628.32|653.45|791.14|816.27|140.42| +2450834|70454|2450846|68237|1250390|4543|31849|68237|1250390|4543|31849|4|100|3|3|14060|73|971|17|29.01|70.49|11.98|994.67|203.66|493.17|1198.33|14.25|0.00|563.21|203.66|217.91|766.87|781.12|-289.51| +2450834|70454|2450888|68237|1250390|4543|31849|68237|1250390|4543|31849|4|34|5|1|3848|195|971|100|10.74|16.64|0.16|1648.00|16.00|1074.00|1664.00|0.16|0.00|815.00|16.00|16.16|831.00|831.16|-1058.00| +2450834|70454|2450860|68237|1250390|4543|31849|68237|1250390|4543|31849|4|97|18|4|13171|197|971|72|67.62|119.68|16.75|7410.96|1206.00|4868.64|8616.96|108.54|0.00|2154.24|1206.00|1314.54|3360.24|3468.78|-3662.64| +2450834|70454|2450888|68237|1250390|4543|31849|68237|1250390|4543|31849|4|42|13|5|10214|192|971|17|67.43|156.43|62.57|1595.62|1063.69|1146.31|2659.31|45.10|499.93|0.00|563.76|608.86|563.76|608.86|-582.55| +2450834|70454|2450887|68237|1250390|4543|31849|68237|1250390|4543|31849|4|17|5|5|17462|180|971|90|57.77|131.13|124.57|590.40|11211.30|5199.30|11801.70|0.00|0.00|3776.40|11211.30|11211.30|14987.70|14987.70|6012.00| +2450834|70454|2450858|68237|1250390|4543|31849|68237|1250390|4543|31849|4|71|9|1|2152|257|971|22|88.21|142.01|103.66|843.70|2280.52|1940.62|3124.22|68.41|0.00|687.28|2280.52|2348.93|2967.80|3036.21|339.90| +2450834|70454|2450890|68237|1250390|4543|31849|68237|1250390|4543|31849|4|82|16|3|14476|129|971|82|64.34|149.26|113.43|2938.06|9301.26|5275.88|12239.32|0.00|0.00|1835.16|9301.26|9301.26|11136.42|11136.42|4025.38| +2450834|70454|2450882|68237|1250390|4543|31849|68237|1250390|4543|31849|4|65|11|2|7738|211|971|24|45.55|79.25|38.83|970.08|931.92|1093.20|1902.00|74.55|0.00|37.92|931.92|1006.47|969.84|1044.39|-161.28| +2450834|70454|2450917|68237|1250390|4543|31849|68237|1250390|4543|31849|4|25|5|2|9028|174|971|4|20.71|27.33|9.01|73.28|36.04|82.84|109.32|1.44|0.00|45.88|36.04|37.48|81.92|83.36|-46.80| +2450834|70454|2450851|68237|1250390|4543|31849|68237|1250390|4543|31849|4|104|8|2|10766|45|971|41|11.03|31.76|0.31|1289.45|12.71|452.23|1302.16|0.50|0.00|220.99|12.71|13.21|233.70|234.20|-439.52| +2450834|16887|2450870|45847|1333790|133|26076|45847|1333790|133|26076|4|41|3|5|9676|224|972|61|94.39|213.32|162.12|3123.20|9889.32|5757.79|13012.52|692.25|0.00|4033.32|9889.32|10581.57|13922.64|14614.89|4131.53| +2450834|16887|2450908|45847|1333790|133|26076|45847|1333790|133|26076|4|79|10|5|5608|169|972|31|62.18|155.45|17.09|4289.16|529.79|1927.58|4818.95|37.08|0.00|867.38|529.79|566.87|1397.17|1434.25|-1397.79| +2450834|16887|2450864|45847|1333790|133|26076|45847|1333790|133|26076|4|89|8|1|16021|239|972|96|73.79|129.13|95.55|3223.68|9172.80|7083.84|12396.48|458.64|0.00|2726.40|9172.80|9631.44|11899.20|12357.84|2088.96| +2450834|16887|2450876|45847|1333790|133|26076|45847|1333790|133|26076|4|99|5|5|6427|135|972|30|63.57|71.19|29.18|1260.30|875.40|1907.10|2135.70|44.12|323.89|491.10|551.51|595.63|1042.61|1086.73|-1355.59| +2450834|16887|2450873|45847|1333790|133|26076|45847|1333790|133|26076|4|98|5|2|10399|79|972|78|23.13|55.04|33.02|1717.56|2575.56|1804.14|4293.12|55.89|1777.13|1244.88|798.43|854.32|2043.31|2099.20|-1005.71| +2450834|16887|2450919|45847|1333790|133|26076|45847|1333790|133|26076|4|47|4|1|934|275|972|77|61.62|117.07|83.11|2614.92|6399.47|4744.74|9014.39|191.98|0.00|2794.33|6399.47|6591.45|9193.80|9385.78|1654.73| +2450834|16887|2450860|45847|1333790|133|26076|45847|1333790|133|26076|4|41|10|4|12854|37|972|11|58.79|153.44|87.46|725.78|962.06|646.69|1687.84|28.86|0.00|826.98|962.06|990.92|1789.04|1817.90|315.37| +2450834|16887|2450876|45847|1333790|133|26076|45847|1333790|133|26076|4|107|11|1|13406|236|972|30|55.57|88.35|53.01|1060.20|1590.30|1667.10|2650.50|79.51|0.00|980.40|1590.30|1669.81|2570.70|2650.21|-76.80| +2450834|16887|2450909|45847|1333790|133|26076|45847|1333790|133|26076|4|4|12|1|16282|224|972|80|32.80|84.29|75.86|674.40|6068.80|2624.00|6743.20|424.81|0.00|3236.00|6068.80|6493.61|9304.80|9729.61|3444.80| +2450834|16887|2450881|45847|1333790|133|26076|45847|1333790|133|26076|4|106|3|5|3506|224|972|7|81.95|138.49|16.61|853.16|116.27|573.65|969.43|5.81|0.00|184.17|116.27|122.08|300.44|306.25|-457.38| +2450834|16887|2450837|45847|1333790|133|26076|45847|1333790|133|26076|4|9|19|3|12632|21|972|40|76.98|205.53|2.05|8139.20|82.00|3079.20|8221.20|6.56|0.00|3699.20|82.00|88.56|3781.20|3787.76|-2997.20| +2450834|16887|2450880|45847|1333790|133|26076|45847|1333790|133|26076|4|76|17|1|10660|18|972|58|13.06|38.13|30.12|464.58|1746.96|757.48|2211.54|122.28|0.00|110.20|1746.96|1869.24|1857.16|1979.44|989.48| +2450834|16887|2450902|45847|1333790|133|26076|45847|1333790|133|26076|4|2|4|3|614|86|972|6|30.60|46.51|4.65|251.16|27.90|183.60|279.06|1.95|0.00|36.24|27.90|29.85|64.14|66.09|-155.70| +2450834|28631|2450886|6201|1886344|4887|17030|6201|1886344|4887|17030|2|97|5|2|13510|151|973|95|33.16|73.61|54.47|1818.30|5174.65|3150.20|6992.95|155.23|0.00|699.20|5174.65|5329.88|5873.85|6029.08|2024.45| +2450834|28631|2450910|6201|1886344|4887|17030|6201|1886344|4887|17030|2|92|16|4|4672|248|973|78|52.62|71.56|0.71|5526.30|55.38|4104.36|5581.68|2.21|0.00|1450.80|55.38|57.59|1506.18|1508.39|-4048.98| +2450834|28631|2450863|6201|1886344|4887|17030|6201|1886344|4887|17030|2|86|10|1|6062|78|973|83|93.23|250.78|35.10|17901.44|2913.30|7738.09|20814.74|233.06|0.00|1873.31|2913.30|3146.36|4786.61|5019.67|-4824.79| +2450834|28631|2450911|6201|1886344|4887|17030|6201|1886344|4887|17030|2|71|17|3|1922|65|973|16|69.73|78.79|36.24|680.80|579.84|1115.68|1260.64|2.55|452.27|352.96|127.57|130.12|480.53|483.08|-988.11| +2450834|28631|2450922|6201|1886344|4887|17030|6201|1886344|4887|17030|2|103|16|3|3667|72|973|42|50.43|53.45|44.89|359.52|1885.38|2118.06|2244.90|18.85|0.00|89.46|1885.38|1904.23|1974.84|1993.69|-232.68| +2450834|28631|2450921|6201|1886344|4887|17030|6201|1886344|4887|17030|2|38|11|4|3232|81|973|35|48.64|65.17|29.97|1232.00|1048.95|1702.40|2280.95|13.11|786.71|570.15|262.24|275.35|832.39|845.50|-1440.16| +2450834|28631|2450921|6201|1886344|4887|17030|6201|1886344|4887|17030|2|72|7|1|10052|221|973|79|69.12|181.09|88.73|7296.44|7009.67|5460.48|14306.11|280.38|0.00|6580.70|7009.67|7290.05|13590.37|13870.75|1549.19| +2450834|65781|2450911|19111|1342055|6952|36296|33132|129737|1034|36778|2|1|2|5|3517|189|974|58|24.04|34.61|26.64|462.26|1545.12|1394.32|2007.38|31.52|494.43|1003.40|1050.69|1082.21|2054.09|2085.61|-343.63| +2450834|65781|2450854|19111|1342055|6952|36296|33132|129737|1034|36778|2|17|12|5|8389|89|974|22|23.09|54.72|31.19|517.66|686.18|507.98|1203.84|20.58|0.00|505.56|686.18|706.76|1191.74|1212.32|178.20| +2450834|65781|2450854|19111|1342055|6952|36296|33132|129737|1034|36778|2|93|17|2|1627|275|974|57|8.59|16.49|9.56|395.01|544.92|489.63|939.93|10.89|0.00|74.67|544.92|555.81|619.59|630.48|55.29| +2450834|65781|2450860|19111|1342055|6952|36296|33132|129737|1034|36778|2|53|16|5|16466|77|974|57|27.82|69.82|58.64|637.26|3342.48|1585.74|3979.74|33.42|0.00|1830.27|3342.48|3375.90|5172.75|5206.17|1756.74| +|||19111|1342055|6952|||129737|1034|36778|2|108|3|5|14326||974|28|39.48|61.58|||34.44|1105.44|||8.61|775.88||26.08|801.71|801.96|-1079.61| +2450834|65781|2450911|19111|1342055|6952|36296|33132|129737|1034|36778|2|15|14|1|1940|73|974|95|41.43|103.57|100.46|295.45|9543.70|3935.85|9839.15|95.43|0.00|491.15|9543.70|9639.13|10034.85|10130.28|5607.85| +2450834|65781|2450842|19111|1342055|6952|36296|33132|129737|1034|36778|2|6|13|4|10411|13|974|1|66.07|130.15|122.34|7.81|122.34|66.07|130.15|8.56|0.00|0.00|122.34|130.90|122.34|130.90|56.27| +2450834|65781|2450897|19111|1342055|6952|36296|33132|129737|1034|36778|2|25|17|2|10753|250|974|1|14.74|35.96|6.47|29.49|6.47|14.74|35.96|0.04|5.43|8.27|1.04|1.08|9.31|9.35|-13.70| +2450834|65781|2450873|19111|1342055|6952|36296|33132|129737|1034|36778|2|68|10|1|9199|20|974|28|4.20|5.67|4.70|27.16|131.60|117.60|158.76|9.21|0.00|53.76|131.60|140.81|185.36|194.57|14.00| +2450834|65781|2450854|19111|1342055|6952|36296|33132|129737|1034|36778|2|26|17|4|14714|266|974|53|17.58|36.74|9.91|1421.99|525.23|931.74|1947.22|0.00|273.11|486.54|252.12|252.12|738.66|738.66|-679.62| +2450834|65781|2450874|19111|1342055|6952|36296|33132|129737|1034|36778|2|58|6|3|3088|32|974|30|73.19|143.45|110.45|990.00|3313.50|2195.70|4303.50|231.94|0.00|1936.50|3313.50|3545.44|5250.00|5481.94|1117.80| +2450834|44429|2450892|6510|1803602|5941|33833|6510|1803602|5941|33833|2|33|10|3|12962|234|975|32|23.10|34.41|19.61|473.60|627.52|739.20|1101.12|18.82|0.00|308.16|627.52|646.34|935.68|954.50|-111.68| +2450834|44429|2450874|6510|1803602|5941|33833|6510|1803602|5941|33833|2|69|15|3|7606|32|975|64|26.06|54.72|41.04|875.52|2626.56|1667.84|3502.08|157.59|0.00|1645.44|2626.56|2784.15|4272.00|4429.59|958.72| +2450834|44429|2450871|6510|1803602|5941|33833|6510|1803602|5941|33833|2|84|4|1|26|37|975|14|54.12|64.94|42.86|309.12|600.04|757.68|909.16|12.60|420.02|409.08|180.02|192.62|589.10|601.70|-577.66| +2450834|44429|2450854|6510|1803602|5941|33833|6510|1803602|5941|33833|2|74|5|1|17983|296|975|66|3.28|8.88|5.06|252.12|333.96|216.48|586.08|30.05|0.00|140.58|333.96|364.01|474.54|504.59|117.48| +2450834|44429|2450852|6510|1803602|5941|33833|6510|1803602|5941|33833|2|72|3|2|12494|87|975|33|22.65|51.41|41.64|322.41|1374.12|747.45|1696.53|0.00|302.30|339.24|1071.82|1071.82|1411.06|1411.06|324.37| +2450834|44429|2450919|6510|1803602|5941|33833|6510|1803602|5941|33833|2|99|12|1|14992|39|975|95|44.80|80.19|56.13|2285.70|5332.35|4256.00|7618.05|479.91|0.00|3579.60|5332.35|5812.26|8911.95|9391.86|1076.35| +2450834|44429|2450852|6510|1803602|5941|33833|6510|1803602|5941|33833|2|91|12|2|5713|110|975|56|42.01|122.24|79.45|2396.24|4449.20|2352.56|6845.44|88.98|0.00|1094.80|4449.20|4538.18|5544.00|5632.98|2096.64| +2450834|44429|2450886|6510|1803602|5941|33833|6510|1803602|5941|33833|2|33|16|3|11689|214|975|13|70.03|204.48|134.95|903.89|1754.35|910.39|2658.24|52.63|0.00|1010.10|1754.35|1806.98|2764.45|2817.08|843.96| +2450834|44429|2450856|6510|1803602|5941|33833|6510|1803602|5941|33833|2|70|13|2|11728|190|975|20|73.85|208.99|43.88|3302.20|877.60|1477.00|4179.80|18.60|412.47|710.40|465.13|483.73|1175.53|1194.13|-1011.87| +2450834|44429|2450917|6510|1803602|5941|33833|6510|1803602|5941|33833|2|103|20|2|5726|101|975|98|63.58|176.11|95.09|7939.96|9318.82|6230.84|17258.78|186.37|0.00|4486.44|9318.82|9505.19|13805.26|13991.63|3087.98| +2450834|44429|2450886|6510|1803602|5941|33833|6510|1803602|5941|33833|2|52|10|2|14216|79|975|57|28.31|50.67|32.93|1011.18|1877.01|1613.67|2888.19|28.53|1163.74|866.40|713.27|741.80|1579.67|1608.20|-900.40| +2450834|44429|2450905|6510|1803602|5941|33833|6510|1803602|5941|33833|2|59|2|2|9016|175|975|83|89.88|176.16|70.46|8773.10|5848.18|7460.04|14621.28|58.48|0.00|3801.40|5848.18|5906.66|9649.58|9708.06|-1611.86| +2450834|72098|2450838|6397|349204|6411|32065|68659|6126|2533|21740|4|6|7|5|1285|32|976|6|85.74|215.20|86.08|774.72|516.48|514.44|1291.20|36.15|0.00|38.70|516.48|552.63|555.18|591.33|2.04| +2450834|72098|2450852|6397|349204|6411|32065|68659|6126|2533|21740|4|72|9|5|6428|42|976|78|98.92|258.18|41.30|16916.64|3221.40|7715.76|20138.04|32.21|0.00|9263.28|3221.40|3253.61|12484.68|12516.89|-4494.36| +2450834|72098|2450877|6397|349204|6411|32065|68659|6126|2533|21740|4|93|1|1|10196|206|976|62|12.96|29.41|27.64|109.74|1713.68|803.52|1823.42|102.82|0.00|35.96|1713.68|1816.50|1749.64|1852.46|910.16| +2450834|72098|2450922|6397|349204|6411|32065|68659|6126|2533|21740|4|104|20|2|17065|169|976|72|70.38|183.69|3.67|12961.44|264.24|5067.36|13225.68|7.92|0.00|1321.92|264.24|272.16|1586.16|1594.08|-4803.12| +2450834|72098|2450844|6397|349204|6411|32065|68659|6126|2533|21740|4|102|16|1|9878|81|976|53|9.86|10.05|3.41|351.92|180.73|522.58|532.65|14.45|0.00|106.53|180.73|195.18|287.26|301.71|-341.85| +2450834|72098|2450888|6397|349204|6411|32065|68659|6126|2533|21740|4|102|20|5|7136|204|976|72|5.63|8.10|1.78|455.04|128.16|405.36|583.20|11.53|0.00|209.52|128.16|139.69|337.68|349.21|-277.20| +2450834|72098|2450917|6397|349204|6411|32065|68659|6126|2533|21740|4|1|4|1|12394|19|976|49|79.74|229.65|59.70|8327.55|2925.30|3907.26|11252.85|234.02|0.00|4950.96|2925.30|3159.32|7876.26|8110.28|-981.96| +2450834|72098|2450898|6397|349204|6411|32065|68659|6126|2533|21740|4|85|9|3|1904|243|976|100|59.28|83.58|52.65|3093.00|5265.00|5928.00|8358.00|157.95|0.00|1253.00|5265.00|5422.95|6518.00|6675.95|-663.00| +2450834|72098|2450882|6397|349204|6411|32065|68659|6126|2533|21740|4|30|11|4|1088|172|976|44|59.74|163.68|148.94|648.56|6553.36|2628.56|7201.92|458.73|0.00|2088.24|6553.36|7012.09|8641.60|9100.33|3924.80| +2450834|72098|2450879|6397|349204|6411|32065|68659|6126|2533|21740|4|13|18|5|9055|130|976|38|45.81|104.90|73.43|1195.86|2790.34|1740.78|3986.20|139.51|0.00|1833.50|2790.34|2929.85|4623.84|4763.35|1049.56| +2450834|72098|2450861|6397|349204|6411|32065|68659|6126|2533|21740|4|104|20|2|8338|11|976|87|53.66|131.46|26.29|9149.79|2287.23|4668.42|11437.02|0.00|0.00|3659.22|2287.23|2287.23|5946.45|5946.45|-2381.19| +2450834|72098|2450836|6397|349204|6411|32065|68659|6126|2533|21740|4|62|8|5|15424|269|976|10|13.30|32.31|13.24|190.70|132.40|133.00|323.10|10.59|0.00|87.20|132.40|142.99|219.60|230.19|-0.60| +2450834|28356|2450907|53917|1087231|6945|12659|53917|1087231|6945|12659|4|48|12|4|1442|92|977|42|16.28|48.84|28.81|841.26|1210.02|683.76|2051.28|60.50|0.00|656.04|1210.02|1270.52|1866.06|1926.56|526.26| +2450834|28356|2450899|53917|1087231|6945|12659|53917|1087231|6945|12659|4|72|14|2|12614|128|977|14|93.22|278.72|139.36|1951.04|1951.04|1305.08|3902.08|39.02|0.00|1365.70|1951.04|1990.06|3316.74|3355.76|645.96| +2450834|28356|2450903|53917|1087231|6945|12659|53917|1087231|6945|12659|4|14|19|4|13922|176|977|56|32.53|51.07|21.44|1659.28|1200.64|1821.68|2859.92|96.05|0.00|1000.72|1200.64|1296.69|2201.36|2297.41|-621.04| +2450834|28356|2450887|53917|1087231|6945|12659|53917|1087231|6945|12659|4|14|12|2|2734|100|977|71|88.36|216.48|101.74|8146.54|7223.54|6273.56|15370.08|0.00|0.00|0.00|7223.54|7223.54|7223.54|7223.54|949.98| +2450834|28356|2450873|53917|1087231|6945|12659|53917|1087231|6945|12659|4|83|3|1|6218|52|977|86|62.45|125.52|40.16|7340.96|3453.76|5370.70|10794.72|172.68|0.00|4101.34|3453.76|3626.44|7555.10|7727.78|-1916.94| +2450834|28356|2450867|53917|1087231|6945|12659|53917|1087231|6945|12659|4|25|9|5|10324|94|977|67|5.51|10.24|0.71|638.51|47.57|369.17|686.08|3.80|0.00|116.58|47.57|51.37|164.15|167.95|-321.60| +2450834|37038|2450844|10426|741065|5304|45943|10426|741065|5304|45943|4|30|7|1|10930|184|978|52|49.29|61.11|24.44|1906.84|1270.88|2563.08|3177.72|88.96|0.00|1207.44|1270.88|1359.84|2478.32|2567.28|-1292.20| +2450834|37038|2450845|10426|741065|5304|45943|10426|741065|5304|45943|4|7|3|1|15862|230|978|8|2.39|2.86|1.97|7.12|15.76|19.12|22.88|1.10|0.00|10.48|15.76|16.86|26.24|27.34|-3.36| +2450834|37038|2450854|10426|741065|5304|45943|10426|741065|5304|45943|4|30|13|3|12766|280|978|96|72.35|214.15|205.58|822.72|19735.68|6945.60|20558.40|986.78|0.00|4111.68|19735.68|20722.46|23847.36|24834.14|12790.08| +2450834|37038|2450910|10426|741065|5304|45943|10426|741065|5304|45943|4|26|15|1|16460|168|978|15|88.93|100.49|20.09|1206.00|301.35|1333.95|1507.35|27.12|0.00|271.20|301.35|328.47|572.55|599.67|-1032.60| +2450834|70795|2450913|69759|247101|2043|185|69759|247101|2043|185|1|28|8|1|15022|280|979|95|54.67|120.82|113.57|688.75|10789.15|5193.65|11477.90|431.56|0.00|3557.75|10789.15|11220.71|14346.90|14778.46|5595.50| +2450834|70795|2450869|69759|247101|2043|185|69759|247101|2043|185|1|80|18|1|4304|163|979|46|20.28|26.36|15.02|521.64|690.92|932.88|1212.56|7.46|504.37|230.00|186.55|194.01|416.55|424.01|-746.33| +2450834|70795|2450853|69759|247101|2043|185|69759|247101|2043|185|1|100|13|1|3793|34|979|51|33.11|56.28|19.69|1866.09|1004.19|1688.61|2870.28|26.51|120.50|1061.82|883.69|910.20|1945.51|1972.02|-804.92| +2450834|70795|2450855|69759|247101|2043|185|69759|247101|2043|185|1|81|1|5|14408|38|979|40|8.16|12.81|5.76|282.00|230.40|326.40|512.40|20.73|0.00|122.80|230.40|251.13|353.20|373.93|-96.00| +2450834|70795|2450860|69759|247101|2043|185|69759|247101|2043|185|1|45|11|2|15506|247|979|66|28.75|54.62|13.65|2704.02|900.90|1897.50|3604.92|36.03|0.00|1297.56|900.90|936.93|2198.46|2234.49|-996.60| +2450834|70795|2450915|69759|247101|2043|185|69759|247101|2043|185|1|33|3|1|12184|259|979|50|70.01|155.42|17.09|6916.50|854.50|3500.50|7771.00|51.27|0.00|3419.00|854.50|905.77|4273.50|4324.77|-2646.00| +2450834|70795|2450864|69759|247101|2043|185|69759|247101|2043|185|1|14|7|3|3217|272|979|100|93.72|98.40|56.08|4232.00|5608.00|9372.00|9840.00|11.21|5047.20|4526.00|560.80|572.01|5086.80|5098.01|-8811.20| +2450834|70795|2450860|69759|247101|2043|185|69759|247101|2043|185|1|26|13|2|4087|262|979|30|52.69|136.46|109.16|819.00|3274.80|1580.70|4093.80|130.99|0.00|204.60|3274.80|3405.79|3479.40|3610.39|1694.10| +2450834|70795|2450854|69759|247101|2043|185|69759|247101|2043|185|1|52|6|5|866|235|979|1|43.73|79.15|70.44|8.71|70.44|43.73|79.15|2.11|0.00|31.66|70.44|72.55|102.10|104.21|26.71| +2450834|70795|2450873|69759|247101|2043|185|69759|247101|2043|185|1|79|12|5|52|163|979|15|72.82|139.81|8.38|1971.45|125.70|1092.30|2097.15|1.73|67.87|293.55|57.83|59.56|351.38|353.11|-1034.47| +2450834|56340|2450853|64942|1168988|3434|25880|64942|1168988|3434|25880|4|73|1|1|7132|38|980|9|9.37|27.73|20.79|62.46|187.11|84.33|249.57|15.99|9.35|112.23|177.76|193.75|289.99|305.98|93.43| +2450834|56340|2450868|64942|1168988|3434|25880|64942|1168988|3434|25880|4|85|13|3|1160|192|980|9|55.61|137.35|46.69|815.94|420.21|500.49|1236.15|2.81|138.66|24.66|281.55|284.36|306.21|309.02|-218.94| +2450834|56340|2450845|64942|1168988|3434|25880|64942|1168988|3434|25880|4|60|3|1|14821|202|980|84|74.08|213.35|36.26|14875.56|3045.84|6222.72|17921.40|0.00|1005.12|7885.08|2040.72|2040.72|9925.80|9925.80|-4182.00| +2450834|56340|2450908|64942|1168988|3434|25880|64942|1168988|3434|25880|4|19|14|3|2968|28|980|25|84.56|169.12|18.60|3763.00|465.00|2114.00|4228.00|9.30|0.00|295.75|465.00|474.30|760.75|770.05|-1649.00| +2450834|56340|2450886|64942|1168988|3434|25880|64942|1168988|3434|25880|4|56|9|3|493|39|980|42|73.53|144.11|77.81|2784.60|3268.02|3088.26|6052.62|163.40|0.00|3026.10|3268.02|3431.42|6294.12|6457.52|179.76| +2450834|76071|2450885|27283|901789|2695|14801|27283|901789|2695|14801|4|9|8|5|17716|41|981|49|59.31|82.44|68.42|686.98|3352.58|2906.19|4039.56|100.57|0.00|1898.26|3352.58|3453.15|5250.84|5351.41|446.39| +2450834|76071|2450845|27283|901789|2695|14801|27283|901789|2695|14801|4|6|13|2|2827|50|981|5|48.43|61.99|25.41|182.90|127.05|242.15|309.95|5.08|0.00|142.55|127.05|132.13|269.60|274.68|-115.10| +2450834|76071|2450860|27283|901789|2695|14801|27283|901789|2695|14801|4|60|19|1|9680|83|981|41|28.58|76.02|3.80|2961.02|155.80|1171.78|3116.82|0.23|151.12|342.76|4.68|4.91|347.44|347.67|-1167.10| +2450834|76071|2450871|27283|901789|2695|14801|27283|901789|2695|14801|4|73|1|2|14797|4|981|78|44.80|52.86|37.53|1195.74|2927.34|3494.40|4123.08|12.88|2605.33|783.12|322.01|334.89|1105.13|1118.01|-3172.39| +2450834|76071|2450877|27283|901789|2695|14801|27283|901789|2695|14801|4|58|9|2|12541|48|981|90|7.74|22.67|20.17|225.00|1815.30|696.60|2040.30|114.36|544.59|774.90|1270.71|1385.07|2045.61|2159.97|574.11| +2450834|76581|2450858|12753|284271|1742|43260|41968|299496|4167|28485|1|101|5|1|16042|135|982|11|60.65|64.89|63.59|14.30|699.49|667.15|713.79|13.98|0.00|128.48|699.49|713.47|827.97|841.95|32.34| +2450834|76581|2450885|12753|284271|1742|43260|41968|299496|4167|28485|1|24|4|1|4324|155|982|26|51.43|55.03|28.06|701.22|729.56|1337.18|1430.78|50.55|167.79|600.86|561.77|612.32|1162.63|1213.18|-775.41| +2450834|76581|2450863|12753|284271|1742|43260|41968|299496|4167|28485|1|82|4|1|17302|28|982|17|79.75|191.40|179.91|195.33|3058.47|1355.75|3253.80|275.26|0.00|585.65|3058.47|3333.73|3644.12|3919.38|1702.72| +2450834|76581|2450896|12753|284271|1742|43260|41968|299496|4167|28485|1|61|12|3|16394|288|982|84|4.93|12.42|10.30|178.08|865.20|414.12|1043.28|19.20|224.95|510.72|640.25|659.45|1150.97|1170.17|226.13| +2450834|45496|2450870|4822|996826|2430|40834|4822|996826|2430|40834|1|29|13|5|3469|224|983|13|3.93|10.92|8.29|34.19|107.77|51.09|141.96|3.23|0.00|21.19|107.77|111.00|128.96|132.19|56.68| +2450834|45496|2450884|4822|996826|2430|40834|4822|996826|2430|40834|1|29|11|1|16411|96|983|16|24.81|43.16|36.68|103.68|586.88|396.96|690.56|0.00|234.75|89.76|352.13|352.13|441.89|441.89|-44.83| +2450834|45496|2450862|4822|996826|2430|40834|4822|996826|2430|40834|1|68|12|2|5872|147|983|75|55.24|109.92|83.53|1979.25|6264.75|4143.00|8244.00|187.94|0.00|0.00|6264.75|6452.69|6264.75|6452.69|2121.75| +2450834|45496|2450844|4822|996826|2430|40834|4822|996826|2430|40834|1|84|6|2|8402|112|983|53|69.00|73.83|28.79|2387.12|1525.87|3657.00|3912.99|16.78|1190.17|1291.08|335.70|352.48|1626.78|1643.56|-3321.30| +2450834|45496|2450896|4822|996826|2430|40834|4822|996826|2430|40834|1|70|3|2|13160|197|983|65|96.03|257.36|33.45|14554.15|2174.25|6241.95|16728.40|147.63|65.22|7527.65|2109.03|2256.66|9636.68|9784.31|-4132.92| +2450834|45496|2450868|4822|996826|2430|40834|4822|996826|2430|40834|1|86|11|4|5596|170|983|53|77.09|77.86|68.51|495.55|3631.03|4085.77|4126.58|217.86|0.00|536.36|3631.03|3848.89|4167.39|4385.25|-454.74| +2450834|45496|2450846|4822|996826|2430|40834|4822|996826|2430|40834|1|25|6|3|6496|35|983|94|68.45|115.68|49.74|6198.36|4675.56|6434.30|10873.92|24.78|2197.51|5436.96|2478.05|2502.83|7915.01|7939.79|-3956.25| +2450834|45496|2450895|4822|996826|2430|40834|4822|996826|2430|40834|1|42|12|5|10579|198|983|38|90.44|92.24|47.96|1682.64|1822.48|3436.72|3505.12|109.34|0.00|1016.12|1822.48|1931.82|2838.60|2947.94|-1614.24| +2450834|45496|2450850|4822|996826|2430|40834|4822|996826|2430|40834|1|29|9|4|4894|201|983|55|9.62|20.49|17.21|180.40|946.55|529.10|1126.95|6.81|265.03|179.85|681.52|688.33|861.37|868.18|152.42| +2450834|45496|2450893|4822|996826|2430|40834|4822|996826|2430|40834|1|36|10|1|12424|298|983|11|28.27|37.88|7.95|329.23|87.45|310.97|416.68|7.87|0.00|162.47|87.45|95.32|249.92|257.79|-223.52| +2450834|45496|2450855|4822|996826|2430|40834|4822|996826|2430|40834|1|91|18|4|13792|143|983|11|68.24|178.78|96.54|904.64|1061.94|750.64|1966.58|2.23|1030.08|589.93|31.86|34.09|621.79|624.02|-718.78| +2450834|45496|2450879|4822|996826|2430|40834|4822|996826|2430|40834|1|19|19|5|5666|175|983|60|60.26|159.68|129.34|1820.40|7760.40|3615.60|9580.80|481.92|2405.72|3257.40|5354.68|5836.60|8612.08|9094.00|1739.08| +2450834|45496|2450891|4822|996826|2430|40834|4822|996826|2430|40834|1|45|15|5|5947|107|983|95|82.83|104.36|84.53|1883.85|8030.35|7868.85|9914.20|513.13|2328.80|1685.30|5701.55|6214.68|7386.85|7899.98|-2167.30| +2450834|45496|2450836|4822|996826|2430|40834|4822|996826|2430|40834|1|97|14|4|2383|64|983|39|84.04|247.07|217.42|1156.35|8479.38|3277.56|9635.73|678.35|0.00|4336.02|8479.38|9157.73|12815.40|13493.75|5201.82| +2450834|47732|2450875|20363|318749|5488|12274|20363|318749|5488|12274|2|49|17|1|12866|162|984|79|65.04|74.79|24.68|3958.69|1949.72|5138.16|5908.41|0.00|0.00|413.17|1949.72|1949.72|2362.89|2362.89|-3188.44| +2450834|47732|2450905|20363|318749|5488|12274|20363|318749|5488|12274|2|33|11|5|14092|210|984|3|67.15|122.88|114.27|25.83|342.81|201.45|368.64|23.99|0.00|18.42|342.81|366.80|361.23|385.22|141.36| +2450834|47732|2450920|20363|318749|5488|12274|20363|318749|5488|12274|2|70|9|2|2914|283|984|87|22.85|32.67|27.44|455.01|2387.28|1987.95|2842.29|95.49|0.00|937.86|2387.28|2482.77|3325.14|3420.63|399.33| +2450834|47732|2450918|20363|318749|5488|12274|20363|318749|5488|12274|2|1|12|1|13837|186|984|76|91.55|271.90|87.00|14052.40|6612.00|6957.80|20664.40|330.60|0.00|0.00|6612.00|6942.60|6612.00|6942.60|-345.80| +2450834|47732|2450839|20363|318749|5488|12274|20363|318749|5488|12274|2|86|8|3|13220|230|984|12|46.77|101.95|41.79|721.92|501.48|561.24|1223.40|6.26|376.11|330.24|125.37|131.63|455.61|461.87|-435.87| +2450834|36465|2450913|64411|919956|1101|13046|64411|919956|1101|13046|2|65|1|5|958|86|985|98|90.32|153.54|59.88|9178.68|5868.24|8851.36|15046.92|12.32|5457.46|1956.08|410.78|423.10|2366.86|2379.18|-8440.58| +2450834|36465|2450879|64411|919956|1101|13046|64411|919956|1101|13046|2|64|14|5|16939|164|985|49|77.23|120.47|30.11|4427.64|1475.39|3784.27|5903.03|73.76|0.00|1593.48|1475.39|1549.15|3068.87|3142.63|-2308.88| +2450834|36465|2450870||919956|||64411|919956||||10|10|5|12608||985||43.86|97.80|40.09||||||66.54||||80.14|81.36|-74.08| +2450834|36465|2450877|64411|919956|1101|13046|64411|919956|1101|13046|2|51|5|3|17617|174|985|3|20.65|32.62|1.63|92.97|4.89|61.95|97.86|0.29|0.00|27.39|4.89|5.18|32.28|32.57|-57.06| +2450834|36465|2450843|64411|919956|1101|13046|64411|919956|1101|13046|2|56|7|3|16156|93|985|24|21.15|61.96|27.88|817.92|669.12|507.60|1487.04|60.22|0.00|356.88|669.12|729.34|1026.00|1086.22|161.52| +2450834|36465|2450887|64411|919956|1101|13046|64411|919956|1101|13046|2|35|12|4|3674|145|985|26|69.32|175.37|171.86|91.26|4468.36|1802.32|4559.62|0.00|0.00|820.56|4468.36|4468.36|5288.92|5288.92|2666.04| +2450834|46976|2450875|3886|1641723|1160|41797|3886|1641723|1160|41797|4|87|7|1|820|149|986|46|93.49|240.26|165.77|3426.54|7625.42|4300.54|11051.96|76.25|0.00|994.52|7625.42|7701.67|8619.94|8696.19|3324.88| +2450834|46976|2450857|3886|1641723|1160|41797|3886|1641723|1160|41797|4|88|8|4|5480|69|986|7|39.09|69.58|20.87|340.97|146.09|273.63|487.06|5.84|0.00|233.73|146.09|151.93|379.82|385.66|-127.54| +2450834|46976|2450889|3886|1641723|1160|41797|3886|1641723|1160|41797|4|75|19|2|15031|23|986|64|61.73|153.70|6.14|9443.84|392.96|3950.72|9836.80|12.37|145.39|4524.80|247.57|259.94|4772.37|4784.74|-3703.15| +2450834|46976|2450881|3886|1641723|1160|41797|3886|1641723|1160|41797|4|77|13|5|16658|262|986|93|56.79|158.44|66.54|8546.70|6188.22|5281.47|14734.92|556.93|0.00|3241.05|6188.22|6745.15|9429.27|9986.20|906.75| +2450834|46976|2450859|3886|1641723|1160|41797|3886|1641723|1160|41797|4|108|1|1|7105|297|986|88|80.35|110.07|68.24|3681.04|6005.12|7070.80|9686.16|60.05|0.00|4455.44|6005.12|6065.17|10460.56|10520.61|-1065.68| +2450834|46976|2450836|3886|1641723|1160|41797|3886|1641723|1160|41797|4|105|20|4|1114|294|986|91|6.02|9.45|1.13|757.12|102.83|547.82|859.95|8.22|0.00|42.77|102.83|111.05|145.60|153.82|-444.99| +2450834|65331|2450869|16675|857649|1023|18444|16675|857649|1023|18444|4|11|19|1|15116|140|987|36|16.33|30.37|19.43|393.84|699.48|587.88|1093.32|6.99|0.00|513.72|699.48|706.47|1213.20|1220.19|111.60| +2450834|65331|2450914|16675|857649|1023|18444|16675|857649|1023|18444|4|57|3|3|6337|71|987|55|94.21|260.96|75.67|10190.95|4161.85|5181.55|14352.80|124.85|0.00|3874.75|4161.85|4286.70|8036.60|8161.45|-1019.70| +2450834|65331|2450840|16675|857649|1023|18444|16675|857649|1023|18444|4|73|14|2|307|85|987|35|17.12|44.51|0.89|1526.70|31.15|599.20|1557.85|0.31|0.00|155.75|31.15|31.46|186.90|187.21|-568.05| +2450834|65331|2450883|16675|857649|1023|18444|16675|857649|1023|18444|4|40|3|3|17492|135|987|48|43.55|43.55|16.98|1275.36|815.04|2090.40|2090.40|48.90|0.00|292.32|815.04|863.94|1107.36|1156.26|-1275.36| +2450834|40886|2450893|14317|542556|4748|8187|14317|542556|4748|8187|1|81|12|5|9586|178|988|97|61.43|128.38|3.85|12079.41|373.45|5958.71|12452.86|14.93|0.00|5105.11|373.45|388.38|5478.56|5493.49|-5585.26| +2450834|40886|2450901|14317|542556|4748|8187|14317|542556|4748|8187|1|20|9|2|14528|142|988|98|27.45|37.60|25.94|1142.68|2542.12|2690.10|3684.80|152.52|0.00|662.48|2542.12|2694.64|3204.60|3357.12|-147.98| +2450834|40886|2450886|14317|542556|4748|8187|14317|542556|4748|8187|1|6|4|4|6988|164|988|50|72.65|135.85|91.01|2242.00|4550.50|3632.50|6792.50|182.02|0.00|2784.50|4550.50|4732.52|7335.00|7517.02|918.00| +2450834|40886|2450881|14317|542556|4748|8187|14317|542556|4748|8187|1|8|18|3|6010|286|988|64|24.24|47.51|23.75|1520.64|1520.00|1551.36|3040.64|136.80|0.00|1216.00|1520.00|1656.80|2736.00|2872.80|-31.36| +2450834|40886|2450922|14317|542556|4748|8187|14317|542556|4748|8187|1|99|11|2|2587|126|988|9|46.55|72.61|2.17|633.96|19.53|418.95|653.49|0.39|0.00|91.44|19.53|19.92|110.97|111.36|-399.42| +2450834|40886|2450843|14317|542556|4748|8187|14317|542556|4748|8187|1|48|6|4|4958|73|988|28|8.79|18.72|11.79|194.04|330.12|246.12|524.16|0.66|297.10|172.76|33.02|33.68|205.78|206.44|-213.10| +2450834|40886|2450880|14317|542556|4748|8187|14317|542556|4748|8187|1|51|6|3|2326|67|988|59|68.72|151.18|89.19|3657.41|5262.21|4054.48|8919.62|315.73|0.00|713.31|5262.21|5577.94|5975.52|6291.25|1207.73| +2450834|40886|2450874|14317|542556|4748|8187|14317|542556|4748|8187|1|106|9|5|6955|78|988|4|25.34|60.05|6.00|216.20|24.00|101.36|240.20|0.72|0.00|69.64|24.00|24.72|93.64|94.36|-77.36| +2450834|40886|2450914|14317|542556|4748|8187|14317|542556|4748|8187|1|3|17|2|9127|84|988|82|61.67|162.80|112.33|4138.54|9211.06|5056.94|13349.60|92.11|0.00|1868.78|9211.06|9303.17|11079.84|11171.95|4154.12| +2450834|40886|2450871|14317|542556|4748|8187|14317|542556|4748|8187|1|79|10|2|17828|177|988|56|69.39|160.29|36.86|6912.08|2064.16|3885.84|8976.24|41.28|0.00|3410.96|2064.16|2105.44|5475.12|5516.40|-1821.68| +2450834|40886|2450892|14317|542556|4748|8187|14317|542556|4748|8187|1|42|17|3|6829|78|988|14|8.74|24.29|5.58|261.94|78.12|122.36|340.06|0.00|0.00|23.80|78.12|78.12|101.92|101.92|-44.24| +2450834|36390|2450868|83215|1020606|4844|22635|83215|1020606|4844|22635|1|30|8|1|6421|199|989|64|73.49|112.43|92.19|1295.36|5900.16|4703.36|7195.52|295.00|0.00|2230.40|5900.16|6195.16|8130.56|8425.56|1196.80| +2450834|36390|2450861|83215|1020606|4844|22635|83215|1020606|4844|22635|1|72|13|1|10855|222|989|15|2.80|5.15|2.72|36.45|40.80|42.00|77.25|0.00|0.00|28.50|40.80|40.80|69.30|69.30|-1.20| +2450834|36390|2450889|83215|1020606|4844|22635|83215|1020606|4844|22635|1|49|8|3|13096|110|989|99|95.46|224.33|96.46|12659.13|9549.54|9450.54|22208.67|95.49|0.00|8438.76|9549.54|9645.03|17988.30|18083.79|99.00| +2450834|36390|2450870|83215|1020606|4844|22635|83215|1020606|4844|22635|1|97|3|3|80|299|989|10|41.58|108.10|67.02|410.80|670.20|415.80|1081.00|6.70|0.00|54.00|670.20|676.90|724.20|730.90|254.40| +2450834|36390|2450874|83215|1020606|4844|22635|83215|1020606|4844|22635|1|52|11|4|2521|232|989|70|90.75|156.09|10.92|10161.90|764.40|6352.50|10926.30|0.00|0.00|1420.30|764.40|764.40|2184.70|2184.70|-5588.10| +2450834|53107|2450866|92843|1330107|6571|40682|92843|1330107|6571|40682|4|71|4|4|508|280|990|84|25.74|75.16|56.37|1578.36|4735.08|2162.16|6313.44|378.80|0.00|1767.36|4735.08|5113.88|6502.44|6881.24|2572.92| +2450834|53107|2450911|92843|1330107|6571|40682|92843|1330107|6571|40682|4|95|14|2|17776|168|990|8|10.21|25.62|6.14|155.84|49.12|81.68|204.96|4.42|0.00|73.76|49.12|53.54|122.88|127.30|-32.56| +2450834|53107|2450898|92843|1330107|6571|40682|92843|1330107|6571|40682|4|53|14|5|5281|29|990|98|28.44|36.97|1.10|3515.26|107.80|2787.12|3623.06|2.15|0.00|1484.70|107.80|109.95|1592.50|1594.65|-2679.32| +2450834|53107|2450859|92843|1330107|6571|40682|92843|1330107|6571|40682|4|89|20|1|1064|253|990|56|17.32|33.25|10.97|1247.68|614.32|969.92|1862.00|3.74|239.58|204.40|374.74|378.48|579.14|582.88|-595.18| +2450834|53107|2450904|92843|1330107|6571|40682|92843|1330107|6571|40682|4|58|19|5|16232|287|990|49|87.89|95.80|80.47|751.17|3943.03|4306.61|4694.20|39.43|0.00|1314.18|3943.03|3982.46|5257.21|5296.64|-363.58| +2450834|53107|2450886|92843|1330107|6571|40682|92843|1330107|6571|40682|4|90|9|1|2299|102|990|8|27.35|40.20|6.83|266.96|54.64|218.80|321.60|4.37|0.00|35.36|54.64|59.01|90.00|94.37|-164.16| +2450834|53107|2450866|92843|1330107|6571|40682|92843|1330107|6571|40682|4|63|1|3|16634|124|990|45|20.95|49.23|32.49|753.30|1462.05|942.75|2215.35|11.25|1301.22|797.40|160.83|172.08|958.23|969.48|-781.92| +2450834|53107|2450880|92843|1330107|6571|40682|92843|1330107|6571|40682|4|87|18|2|1999|108|990|71|52.71|89.60|74.36|1082.04|5279.56|3742.41|6361.60|369.56|0.00|2607.83|5279.56|5649.12|7887.39|8256.95|1537.15| +2450834|53107|2450865|92843|1330107|6571|40682|92843|1330107|6571|40682|4|108|3|4|12889|49|990|43|45.59|81.60|26.11|2386.07|1122.73|1960.37|3508.80|22.45|0.00|1052.64|1122.73|1145.18|2175.37|2197.82|-837.64| +2450834|53107|2450886|92843|1330107|6571|40682|92843|1330107|6571|40682|4|38|20|5|1702|122|990|93|64.48|179.89|71.95|10038.42|6691.35|5996.64|16729.77|133.82|0.00|4683.48|6691.35|6825.17|11374.83|11508.65|694.71| +2450834|53107|2450874|92843|1330107|6571|40682|92843|1330107|6571|40682|4|77|8|2|15422|230|990|83|98.38|127.89|57.55|5838.22|4776.65|8165.54|10614.87|191.06|0.00|3290.12|4776.65|4967.71|8066.77|8257.83|-3388.89| +2450834|53107|2450922|92843|1330107|6571|40682|92843|1330107|6571|40682|4|42|3|2|3968|228|990|84|7.93|13.48|4.71|736.68|395.64|666.12|1132.32|23.73|0.00|486.36|395.64|419.37|882.00|905.73|-270.48| +2450834|53107|2450870|92843|1330107|6571|40682|92843|1330107|6571|40682|4|105|16|2|6385|75|990|71|3.58|5.15|2.42|193.83|171.82|254.18|365.65|12.02|0.00|32.66|171.82|183.84|204.48|216.50|-82.36| +2450834|53107|2450923|92843|1330107|6571|40682|92843|1330107|6571|40682|4|85|9|3|7640|109|990|56|29.81|49.48|18.30|1746.08|1024.80|1669.36|2770.88|0.00|0.00|0.00|1024.80|1024.80|1024.80|1024.80|-644.56| +2450834|73940|2450897|13718|604341|4211|24434|13718|604341|4211|24434|2|20|8|4|1964|129|991|47|88.53|145.18|53.71|4299.09|2524.37|4160.91|6823.46|0.00|0.00|2661.14|2524.37|2524.37|5185.51|5185.51|-1636.54| +2450834|73940|2450846|13718|604341|4211|24434|13718|604341|4211|24434|2|80|17|1|17150|74|991|67|1.84|4.01|0.28|249.91|18.76|123.28|268.67|0.93|0.00|2.68|18.76|19.69|21.44|22.37|-104.52| +2450834|73940|2450838|13718|604341|4211|24434|13718|604341|4211|24434|2|23|8|3|6430|95|991|24|52.13|124.59|88.45|867.36|2122.80|1251.12|2990.16|127.36|0.00|1046.40|2122.80|2250.16|3169.20|3296.56|871.68| +2450834|73940|2450905|13718|604341|4211|24434|13718|604341|4211|24434|2|61|2|4|15968|256|991|76|73.93|161.16|9.66|11514.00|734.16|5618.68|12248.16|14.68|0.00|4041.68|734.16|748.84|4775.84|4790.52|-4884.52| +2450834|73940|2450885|13718|604341|4211|24434|13718|604341|4211|24434|2|79|11|1|14842|219|991|38|2.08|5.17|4.65|19.76|176.70|79.04|196.46|15.90|0.00|76.38|176.70|192.60|253.08|268.98|97.66| +2450834|20368|2450907|90829|375387|2345|4903|90829|375387|2345|4903|2|102|5|2|17134|206|992|37|49.81|72.22|37.55|1282.79|1389.35|1842.97|2672.14|78.35|83.36|961.63|1305.99|1384.34|2267.62|2345.97|-536.98| +2450834|20368|2450851|90829|375387|2345|4903|90829|375387|2345|4903|2|48|5|3|17743|164|992|26|94.40|171.80|39.51|3439.54|1027.26|2454.40|4466.80|4.62|873.17|937.82|154.09|158.71|1091.91|1096.53|-2300.31| +2450834|20368|2450902|90829|375387|2345|4903|90829|375387|2345|4903|2|55|12|1|16621|171|992|39|20.39|28.95|22.58|248.43|880.62|795.21|1129.05|0.00|0.00|169.26|880.62|880.62|1049.88|1049.88|85.41| +2450834|20368|2450844|90829|375387|2345|4903|90829|375387|2345|4903|2|51|20|4|10928|254|992|29|44.67|60.30|38.59|629.59|1119.11|1295.43|1748.70|5.37|850.52|192.27|268.59|273.96|460.86|466.23|-1026.84| +2450834|20368|2450900|90829|375387|2345|4903|90829|375387|2345|4903|2|89|7|3|3631|78|992|78|32.79|67.87|65.15|212.16|5081.70|2557.62|5293.86|97.56|1829.41|475.80|3252.29|3349.85|3728.09|3825.65|694.67| +2450834|20368|2450901|90829|375387|2345|4903|90829|375387|2345|4903|2|6|7|3|11630|217|992|38|90.83|227.07|161.21|2502.68|6125.98|3451.54|8628.66|183.77|0.00|2502.30|6125.98|6309.75|8628.28|8812.05|2674.44| +2450834|20368|2450836|90829|375387|2345|4903|90829|375387|2345|4903|2|100|13|1|1274|219|992|69|37.04|107.78|53.89|3718.41|3718.41|2555.76|7436.82|185.92|0.00|594.78|3718.41|3904.33|4313.19|4499.11|1162.65| +2450834|20368|2450919|90829|375387|2345|4903|90829|375387|2345|4903|2|107|8|3|10177|256|992|3|41.42|96.92|27.13|209.37|81.39|124.26|290.76|1.62|0.00|46.50|81.39|83.01|127.89|129.51|-42.87| +2450834|20368|2450861|90829|375387|2345|4903|90829|375387|2345|4903|2|30|14|5|17131|234|992|63|2.97|7.66|7.43|14.49|468.09|187.11|482.58|1.12|454.04|212.31|14.05|15.17|226.36|227.48|-173.06| +2450834|20368|2450901|90829|375387|2345|4903|90829|375387|2345|4903|2|75|13|3|5203|155|992|76|7.11|10.52|9.88|48.64|750.88|540.36|799.52|0.00|0.00|199.88|750.88|750.88|950.76|950.76|210.52| +2450834|20368|2450881|90829|375387|2345|4903|90829|375387|2345|4903|2|66|3|3|7912|33|992|42|62.90|152.21|71.53|3388.56|3004.26|2641.80|6392.82|240.34|0.00|2556.96|3004.26|3244.60|5561.22|5801.56|362.46| +2450834|20368|2450894|90829|375387|2345|4903|90829|375387|2345|4903|2|38|16|4|4532|247|992|46|11.37|16.14|5.97|467.82|274.62|523.02|742.44|13.73|0.00|7.36|274.62|288.35|281.98|295.71|-248.40| +2450834|19568|2450883|99324|336044|3964|14284|99324|336044|3964|14284|1|84|4|5|10264|245|993|89|61.27|101.09|77.83|2070.14|6926.87|5453.03|8997.01|69.26|0.00|0.00|6926.87|6996.13|6926.87|6996.13|1473.84| +2450834|19568|2450912|99324|336044|3964|14284|99324|336044|3964|14284|1|94|19|2|13048|80|993|74|65.97|93.01|49.29|3235.28|3647.46|4881.78|6882.74|255.32|0.00|757.02|3647.46|3902.78|4404.48|4659.80|-1234.32| +2450834|19568|2450919|99324|336044|3964|14284|99324|336044|3964|14284|1|31|3|1|13292|100|993|69|73.73|211.60|207.36|292.56|14307.84|5087.37|14600.40|715.39|0.00|4525.71|14307.84|15023.23|18833.55|19548.94|9220.47| +2450834|19568|2450877|99324|336044|3964|14284|99324|336044|3964|14284|1|59|6|5|553|168|993|77|12.45|22.53|0.00|1734.81|0.00|958.65|1734.81|0.00|0.00|329.56|0.00|0.00|329.56|329.56|-958.65| +2450834|19568|2450907|99324|336044|3964|14284|99324|336044|3964|14284|1|82|17|5|14966|200|993|74|66.24|137.11|23.30|8421.94|1724.20|4901.76|10146.14|68.96|0.00|5072.70|1724.20|1793.16|6796.90|6865.86|-3177.56| +2450834|19568|2450899|99324|336044|3964|14284|99324|336044|3964|14284|1|39|5|2|1771|147|993|67|72.89|146.50|80.57|4417.31|5398.19|4883.63|9815.50|323.89|0.00|1864.61|5398.19|5722.08|7262.80|7586.69|514.56| +2450834|19568|2450921|99324|336044|3964|14284|99324|336044|3964|14284|1|22|1|5|9991|62|993|18|22.10|22.10|0.88|381.96|15.84|397.80|397.80|0.00|14.25|35.64|1.59|1.59|37.23|37.23|-396.21| +2450834|19568|2450843|99324|336044|3964|14284|99324|336044|3964|14284|1|56|1|4|3650|22|993|80|33.82|61.21|0.00|4896.80|0.00|2705.60|4896.80|0.00|0.00|1762.40|0.00|0.00|1762.40|1762.40|-2705.60| +2450834|19568|2450912|99324|336044|3964|14284|99324|336044|3964|14284|1|19|8|5|1108|251|993|9|20.30|28.21|13.25|134.64|119.25|182.70|253.89|3.99|39.35|116.73|79.90|83.89|196.63|200.62|-102.80| +2450834|19568|2450885|99324|336044|3964|14284|99324|336044|3964|14284|1|96|4|4|17236|279|993|91|3.87|9.90|6.73|288.47|612.43|352.17|900.90|29.58|189.85|323.96|422.58|452.16|746.54|776.12|70.41| +2450834|19568|2450851|99324|336044|3964|14284|99324|336044|3964|14284|1|76|3|1|3331|70|993|73|98.36|135.73|90.93|3270.40|6637.89|7180.28|9908.29|464.65|0.00|990.61|6637.89|7102.54|7628.50|8093.15|-542.39| +2450834|19568|2450899|99324|336044|3964|14284|99324|336044|3964|14284|1|106|20|2|4408|18|993|14|31.99|80.29|26.49|753.20|370.86|447.86|1124.06|17.80|74.17|460.74|296.69|314.49|757.43|775.23|-151.17| +2450834|19568|2450884|99324|336044|3964|14284|99324|336044|3964|14284|1|4|8|4|13831|65|993|40|46.30|79.17|13.45|2628.80|538.00|1852.00|3166.80|37.66|0.00|190.00|538.00|575.66|728.00|765.66|-1314.00| +2450834|12003|2450873|24802|96704|3006|14061|24802|96704|3006|14061|2|107|20|5|7628|35|994|31|44.38|119.38|35.81|2590.67|1110.11|1375.78|3700.78|3.10|799.27|480.81|310.84|313.94|791.65|794.75|-1064.94| +2450834|12003|2450915|24802|96704|3006|14061|24802|96704|3006|14061|2|33|4|3|3337|58|994|54|71.95|134.54|2.69|7119.90|145.26|3885.30|7265.16|11.62|0.00|2833.38|145.26|156.88|2978.64|2990.26|-3740.04| +2450834|12003|2450863|24802|96704|3006|14061|24802|96704|3006|14061|2|62|5|5|14432|221|994|47|49.34|91.77|25.69|3105.76|1207.43|2318.98|4313.19|24.14|0.00|1336.68|1207.43|1231.57|2544.11|2568.25|-1111.55| +2450834|12003|2450866|24802|96704|3006|14061|24802|96704|3006|14061|2|31|3|3|12553|198|994|78|98.09|206.96|2.06|15982.20|160.68|7651.02|16142.88|1.60|0.00|5972.46|160.68|162.28|6133.14|6134.74|-7490.34| +2450834|12003|2450854|24802|96704|3006|14061|24802|96704|3006|14061|2|38|7|4|2324|90|994|33|43.26|108.58|72.74|1182.72|2400.42|1427.58|3583.14|120.02|0.00|35.64|2400.42|2520.44|2436.06|2556.08|972.84| +2450834|12003|2450875|24802|96704|3006|14061|24802|96704|3006|14061|2|32|6|3|5695|279|994|66|84.59|186.94|157.02|1974.72|10363.32|5582.94|12338.04|725.43|0.00|739.86|10363.32|11088.75|11103.18|11828.61|4780.38| +2450834|12003|2450869|24802|96704|3006|14061|24802|96704|3006|14061|2|14|19|4|2893|19|994|20|37.28|49.58|22.31|545.40|446.20|745.60|991.60|4.46|0.00|267.60|446.20|450.66|713.80|718.26|-299.40| +2450834|12003|2450859|24802|96704|3006|14061|24802|96704|3006|14061|2|3|6|4|7030|243|994|78|70.75|171.21|136.96|2671.50|10682.88|5518.50|13354.38|411.29|2457.06|133.38|8225.82|8637.11|8359.20|8770.49|2707.32| +2450834|12003|2450904|24802|96704|3006|14061|24802|96704|3006|14061|2|43|20|5|2644|25|994|58|53.18|85.08|55.30|1727.24|3207.40|3084.44|4934.64|128.29|0.00|838.68|3207.40|3335.69|4046.08|4174.37|122.96| +2450834|12003|2450919|24802|96704|3006|14061|24802|96704|3006|14061|2|86|15|4|3430|57|994|73|80.52|110.31|15.44|6925.51|1127.12|5877.96|8052.63|101.44|0.00|3462.39|1127.12|1228.56|4589.51|4690.95|-4750.84| +2450834|12003|2450901|24802|96704|3006|14061|24802|96704|3006|14061|2|65|3|4|6979|136|994|62|29.50|53.98|46.42|468.72|2878.04|1829.00|3346.76|143.90|0.00|99.82|2878.04|3021.94|2977.86|3121.76|1049.04| +2450834|29296|2450843|22079|1396025|1971|317|22079|1396025|1971|317|1|37|20|5|10444|200|995|80|63.05|145.01|110.20|2784.80|8816.00|5044.00|11600.80|705.28|0.00|1276.00|8816.00|9521.28|10092.00|10797.28|3772.00| +2450834|29296|2450841|22079|1396025|1971|317|22079|1396025|1971|317|1|22|13|5|7124|7|995|8|15.62|33.27|28.27|40.00|226.16|124.96|266.16|0.00|0.00|63.84|226.16|226.16|290.00|290.00|101.20| +2450834|29296|2450863|22079|1396025|1971|317|22079|1396025|1971|317|1|60|6|1|4570|259|995|84|78.91|166.50|113.22|4475.52|9510.48|6628.44|13986.00|570.62|0.00|2936.64|9510.48|10081.10|12447.12|13017.74|2882.04| +2450834|29296|2450907|22079|1396025|1971|317|22079|1396025|1971|317|1|9|7|4|8476|263|995|45|53.64|57.93|34.75|1043.10|1563.75|2413.80|2606.85|7.03|860.06|1016.55|703.69|710.72|1720.24|1727.27|-1710.11| +2450834|29296|2450848|22079|1396025|1971|317|22079|1396025|1971|317|1|43|13|3|535|160|995|2|73.63|147.26|1.47|291.58|2.94|147.26|294.52|0.11|0.00|79.52|2.94|3.05|82.46|82.57|-144.32| +2450834|29296|2450839|22079|1396025|1971|317|22079|1396025|1971|317|1|101|16|5|3139|281|995|5|55.18|81.66|57.97|118.45|289.85|275.90|408.30|0.00|0.00|93.90|289.85|289.85|383.75|383.75|13.95| +2450834|29296|2450889|22079|1396025|1971|317|22079|1396025|1971|317|1|10|6|1|164|81|995|17|98.25|269.20|16.15|4301.85|274.55|1670.25|4576.40|24.70|0.00|91.46|274.55|299.25|366.01|390.71|-1395.70| +2450834|62492|2450903|53751|1649527|3350|41173|37178|1582772|1395|1971|4|63|3|4|16189|246|996|16|92.34|173.59|116.30|916.64|1860.80|1477.44|2777.44|148.86|0.00|111.04|1860.80|2009.66|1971.84|2120.70|383.36| +2450834|62492|2450907|53751|1649527|3350|41173|37178|1582772|1395|1971|4|14|11|1|11900|289|996|80|10.15|29.53|17.12|992.80|1369.60|812.00|2362.40|123.26|0.00|1062.40|1369.60|1492.86|2432.00|2555.26|557.60| +2450834|62492|2450910|53751|1649527|3350|41173|37178|1582772|1395|1971|4|73|14|2|12274|201|996|52|42.93|120.63|92.88|1443.00|4829.76|2232.36|6272.76|60.85|3960.40|3136.12|869.36|930.21|4005.48|4066.33|-1363.00| +2450834|62492|2450897|53751|1649527|3350|41173|37178|1582772|1395|1971|4|2|19|3|10954|264|996|67|84.36|103.76|49.80|3615.32|3336.60|5652.12|6951.92|70.06|1935.22|2502.45|1401.38|1471.44|3903.83|3973.89|-4250.74| +2450834|62492|2450846|53751|1649527|3350|41173|37178|1582772|1395|1971|4|56|8|2|6637|157|996|75|43.02|116.58|90.93|1923.75|6819.75|3226.50|8743.50|136.39|0.00|3409.50|6819.75|6956.14|10229.25|10365.64|3593.25| +2450834|62492|2450854|53751|1649527|3350|41173|37178|1582772|1395|1971|4|27|18|3|10069|123|996|88|98.13|200.18|134.12|5813.28|11802.56|8635.44|17615.84|590.12|0.00|1232.88|11802.56|12392.68|13035.44|13625.56|3167.12| +2450834|73740|2450879|61995|1525377|4312|17694|61995|1525377|4312|17694|1|102|17|3|3470|223|997|94|43.48|84.35|75.07|872.32|7056.58|4087.12|7928.90|564.52|0.00|3647.20|7056.58|7621.10|10703.78|11268.30|2969.46| +2450834|73740|2450922|61995|1525377|4312|17694|61995|1525377|4312|17694|1|99|11|1|5162|24|997|77|29.17|46.96|7.98|3001.46|614.46|2246.09|3615.92|24.57|0.00|324.94|614.46|639.03|939.40|963.97|-1631.63| +2450834|73740|2450837|61995|1525377|4312|17694|61995|1525377|4312|17694|1|25|5|3|16672|145|997|88|15.26|30.21|12.38|1569.04|1089.44|1342.88|2658.48|43.57|0.00|185.68|1089.44|1133.01|1275.12|1318.69|-253.44| +2450834|73740|2450850|61995|1525377|4312|17694|61995|1525377|4312|17694|1|64|14|2|14446|70|997|17|65.64|155.56|29.55|2142.17|502.35|1115.88|2644.52|0.00|0.00|952.00|502.35|502.35|1454.35|1454.35|-613.53| +2450834|73740|2450872|61995|1525377|4312|17694|61995|1525377|4312|17694|1|96|7|2|17542|283|997|14|12.98|12.98|7.52|76.44|105.28|181.72|181.72|1.05|0.00|50.82|105.28|106.33|156.10|157.15|-76.44| +2450834|73740|2450861|61995|1525377|4312|17694|61995|1525377|4312|17694|1|90|18|2|9434|6|997|4|39.42|51.24|43.55|30.76|174.20|157.68|204.96|12.19|0.00|10.24|174.20|186.39|184.44|196.63|16.52| +2450834|73740|2450837|61995|1525377|4312|17694|61995|1525377|4312|17694|1|12|14|1|2024|238|997|29|76.07|222.12|128.82|2705.70|3735.78|2206.03|6441.48|74.71|0.00|1223.80|3735.78|3810.49|4959.58|5034.29|1529.75| +2450834|73740|2450891|61995|1525377|4312|17694|61995|1525377|4312|17694|1|36|18|1|11845|184|997|74|20.89|21.93|17.54|324.86|1297.96|1545.86|1622.82|103.83|0.00|762.20|1297.96|1401.79|2060.16|2163.99|-247.90| +2450834|73740|2450916|61995|1525377|4312|17694|61995|1525377|4312|17694|1|1|2|1|9871|79|997|41|17.86|30.36|5.76|1008.60|236.16|732.26|1244.76|21.25|0.00|298.48|236.16|257.41|534.64|555.89|-496.10| +2450834|73740|2450866|61995|1525377|4312|17694|61995|1525377|4312|17694|1|91|9|5|15310|250|997|97|79.48|236.05|203.00|3205.85|19691.00|7709.56|22896.85|1575.28|0.00|686.76|19691.00|21266.28|20377.76|21953.04|11981.44| +2450834|73740|2450899|61995|1525377|4312|17694|61995|1525377|4312|17694|1|39|3|1|17677|284|997|20|32.59|55.40|23.82|631.60|476.40|651.80|1108.00|14.29|0.00|33.20|476.40|490.69|509.60|523.89|-175.40| +2450834|73740|2450871|61995|1525377|4312|17694|61995|1525377|4312|17694|1|77|13|4|6241|96|997|85|41.39|72.01|26.64|3856.45|2264.40|3518.15|6120.85|177.30|294.37|2019.60|1970.03|2147.33|3989.63|4166.93|-1548.12| +2450834|73740|2450852|61995|1525377|4312|17694|61995|1525377|4312|17694|1|71|19|1|6580|261|997|44|76.46|229.38|94.04|5954.96|4137.76|3364.24|10092.72|3.72|4096.38|2018.28|41.38|45.10|2059.66|2063.38|-3322.86| +2450834|73740|2450873|61995|1525377|4312|17694|61995|1525377|4312|17694|1|14|6|5|7684|25|997|68|52.81|155.26|86.94|4645.76|5911.92|3591.08|10557.68|0.00|0.00|4855.88|5911.92|5911.92|10767.80|10767.80|2320.84| +2450834|62639|2450919|54496|1877464|7185|46506|54496|1877464|7185|46506|2|102|17|3|6235|112|998|90|53.53|82.97|68.86|1269.90|6197.40|4817.70|7467.30|148.73|2478.96|2612.70|3718.44|3867.17|6331.14|6479.87|-1099.26| +2450834|62639|2450907|54496|1877464|7185|46506|54496|1877464|7185|46506|2|15|3|3|8707|148|998|37|21.72|44.30|4.87|1458.91|180.19|803.64|1639.10|1.80|0.00|540.57|180.19|181.99|720.76|722.56|-623.45| +2450834|62639|2450880|54496|1877464|7185|46506|54496|1877464|7185|46506|2|39|9|5|17368|222|998|65|72.45|72.45|38.39|2213.90|2495.35|4709.25|4709.25|7.48|1746.74|1977.30|748.61|756.09|2725.91|2733.39|-3960.64| +2450834|62639|2450856|54496|1877464|7185|46506|54496|1877464|7185|46506|2|66|10|1|6295|83|998|47|54.60|133.22|73.27|2817.65|3443.69|2566.20|6261.34|16.53|3030.44|2253.65|413.25|429.78|2666.90|2683.43|-2152.95| +2450834|62639|2450888|54496|1877464|7185|46506|54496|1877464|7185|46506|2|99|6|4|6230|295|998|79|63.74|131.30|69.58|4875.88|5496.82|5035.46|10372.70|329.80|0.00|2904.04|5496.82|5826.62|8400.86|8730.66|461.36| +2450834|62639|2450865|54496|1877464|7185|46506|54496|1877464|7185|46506|2|26|19|2|14458|51|998|2|9.16|13.00|12.74|0.52|25.48|18.32|26.00|0.25|0.00|1.30|25.48|25.73|26.78|27.03|7.16| +2450834|62639|2450839|54496|1877464|7185|46506|54496|1877464|7185|46506|2|5|5|1|17671|241|998|43|47.99|108.93|49.01|2576.56|2107.43|2063.57|4683.99|0.00|0.00|842.80|2107.43|2107.43|2950.23|2950.23|43.86| +2450834|62639|2450851|54496|1877464|7185|46506|54496|1877464|7185|46506|2|69|16|2|9106|114|998|85|3.08|6.49|0.38|519.35|32.30|261.80|551.65|0.32|25.84|231.20|6.46|6.78|237.66|237.98|-255.34| +2450834|62639|2450863|54496|1877464|7185|46506|54496|1877464|7185|46506|2|23|7|3|1042|155|998|61|3.54|7.25|6.96|17.69|424.56|215.94|442.25|8.49|0.00|101.26|424.56|433.05|525.82|534.31|208.62| +2450834|62639|2450846|54496|1877464|7185|46506|54496|1877464|7185|46506|2|31|11|3|11713|182|998|56|4.20|7.47|5.30|121.52|296.80|235.20|418.32|23.74|0.00|100.24|296.80|320.54|397.04|420.78|61.60| +2450834|62639|2450856|54496|1877464|7185|46506|54496|1877464|7185|46506|2|99|4|1|16501|185|998|71|7.83|11.51|6.56|351.45|465.76|555.93|817.21|13.97|0.00|220.10|465.76|479.73|685.86|699.83|-90.17| +2450834|62639|2450866|54496|1877464|7185|46506|54496|1877464|7185|46506|2|7|11|2|464|46|998|7|83.29|204.89|96.29|760.20|674.03|583.03|1434.23|15.09|296.57|444.57|377.46|392.55|822.03|837.12|-205.57| +2450834|62639|2450895|54496|1877464|7185|46506|54496|1877464|7185|46506|2|100|20|1|11683|133|998|20|46.58|77.32|64.94|247.60|1298.80|931.60|1546.40|12.98|0.00|170.00|1298.80|1311.78|1468.80|1481.78|367.20| +2450834|62639|2450845|54496|1877464|7185|46506|54496|1877464|7185|46506|2|17|11|5|16144|290|998|64|32.67|66.64|19.32|3028.48|1236.48|2090.88|4264.96|49.45|0.00|1364.48|1236.48|1285.93|2600.96|2650.41|-854.40| +2450834|50855|2450877|14173|1801438|5503|5568|14173|1801438|5503|5568|2|40|10|1|7825|295|999|47|37.89|44.33|39.89|208.68|1874.83|1780.83|2083.51|112.48|0.00|749.65|1874.83|1987.31|2624.48|2736.96|94.00| +2450834|50855|2450865|14173|1801438|5503|5568|14173|1801438|5503|5568|2|68|20|2|6058|243|999|36|83.62|249.18|209.31|1435.32|7535.16|3010.32|8970.48|226.05|0.00|986.40|7535.16|7761.21|8521.56|8747.61|4524.84| +2450834|50855|2450885|14173|1801438|5503|5568|14173|1801438|5503|5568|2|5|4|2|17554|138|999|90|8.16|14.68|8.22|581.40|739.80|734.40|1321.20|0.00|0.00|620.10|739.80|739.80|1359.90|1359.90|5.40| +2450834|50855|2450850|14173|1801438|5503|5568|14173|1801438|5503|5568|2|46|9|3|9511|181|999|27|82.97|117.81|2.35|3117.42|63.45|2240.19|3180.87|1.26|0.00|1176.66|63.45|64.71|1240.11|1241.37|-2176.74| +2450834|50855|2450846|14173|1801438|5503|5568|14173|1801438|5503|5568|2|49|15|2|15404|215|999|49|54.99|147.37|78.10|3394.23|3826.90|2694.51|7221.13|229.61|0.00|3249.19|3826.90|4056.51|7076.09|7305.70|1132.39| +2450834|7255|2450839|68957|1593114|6518|34275|68957|1593114|6518|34275|1|6|8|3|10906|49|1000|78|43.39|76.80|10.75|5151.90|838.50|3384.42|5990.40|25.15|0.00|478.92|838.50|863.65|1317.42|1342.57|-2545.92| +2450834|7255|2450883|68957|1593114|6518|34275|68957|1593114|6518|34275|1|23|9|3|5923|114|1000|19|33.44|51.49|26.77|469.68|508.63|635.36|978.31|0.00|381.47|459.80|127.16|127.16|586.96|586.96|-508.20| +|7255|2450885|||6518|34275||1593114|6518|34275|1||20|2|17299|160|1000|42|31.45|54.09||||1320.90|||0.00|1090.32|||1703.52|1758.70|-707.70| +2450834|7255|2450881|68957|1593114|6518|34275|68957|1593114|6518|34275|1|6|3|3|70|71|1000|37|29.20|65.99|28.37|1391.94|1049.69|1080.40|2441.63|94.47|0.00|146.15|1049.69|1144.16|1195.84|1290.31|-30.71| +2450834|7255|2450895|68957|1593114|6518|34275|68957|1593114|6518|34275|1|48|13|3|16291|225|1000|84|67.37|202.11|179.87|1868.16|15109.08|5659.08|16977.24|489.53|6950.17|7299.60|8158.91|8648.44|15458.51|15948.04|2499.83| +2450834|7255|2450912|68957|1593114|6518|34275|68957|1593114|6518|34275|1|23|13|4|1178|161|1000|48|40.47|110.88|110.88|0.00|5322.24|1942.56|5322.24|372.55|0.00|957.60|5322.24|5694.79|6279.84|6652.39|3379.68| +2450834|47682|2450880|46333|755203|3273|36143|46333|755203|3273|36143|1|104|20|2|2882|169|1001|25|60.29|123.59|25.95|2441.00|648.75|1507.25|3089.75|38.92|0.00|61.75|648.75|687.67|710.50|749.42|-858.50| +2450834|47682|2450850|46333|755203|3273|36143|46333|755203|3273|36143|1|36|6|4|9373|189|1001|36|69.23|110.07|56.13|1941.84|2020.68|2492.28|3962.52|40.41|0.00|475.20|2020.68|2061.09|2495.88|2536.29|-471.60| +2450834|47682|2450848|46333|755203|3273|36143|46333|755203|3273|36143|1|84|13|3|16111|36|1001|51|54.55|160.37|96.22|3271.65|4907.22|2782.05|8178.87|245.36|0.00|981.24|4907.22|5152.58|5888.46|6133.82|2125.17| +2450834|47682|2450865|46333|755203|3273|36143|46333|755203|3273|36143|1|36|6|4|4945|128|1001|82|18.34|51.53|18.55|2704.36|1521.10|1503.88|4225.46|45.63|0.00|802.78|1521.10|1566.73|2323.88|2369.51|17.22| +2450834|47682|2450895|46333|755203|3273|36143|46333|755203|3273|36143|1|90|15|4|5983|4|1001|52|2.01|4.78|0.19|238.68|9.88|104.52|248.56|0.29|0.00|86.84|9.88|10.17|96.72|97.01|-94.64| +2450834|47682|2450866|46333|755203|3273|36143|46333|755203|3273|36143|1|81|14|5|15320|67|1001|80|43.38|116.69|22.17|7561.60|1773.60|3470.40|9335.20|17.73|0.00|3733.60|1773.60|1791.33|5507.20|5524.93|-1696.80| +2450834|47682|2450872|46333|755203|3273|36143|46333|755203|3273|36143|1|68|12|1|17437|61|1001|79|65.95|104.20|17.71|6832.71|1399.09|5210.05|8231.80|55.96|0.00|1151.82|1399.09|1455.05|2550.91|2606.87|-3810.96| +2450834|47682|2450897|46333|755203|3273|36143|46333|755203|3273|36143|1|7|11|3|16708|177|1001|98|86.98|103.50|76.59|2637.18|7505.82|8524.04|10143.00|375.29|0.00|2129.54|7505.82|7881.11|9635.36|10010.65|-1018.22| +2450834|47682|2450903|46333|755203|3273|36143|46333|755203|3273|36143|1|10|16|4|3988|224|1001|14|33.12|96.71|79.30|243.74|1110.20|463.68|1353.94|77.71|0.00|162.40|1110.20|1187.91|1272.60|1350.31|646.52| +2450834|47682|2450840|46333|755203|3273|36143|46333|755203|3273|36143|1|50|9|3|15151|24|1001|32|84.13|135.44|51.46|2687.36|1646.72|2692.16|4334.08|20.41|1136.23|2167.04|510.49|530.90|2677.53|2697.94|-2181.67| +2450834|47682|2450856|46333|755203|3273|36143|46333|755203|3273|36143|1|38|15|5|4930|185|1001|79|12.80|33.79|30.07|293.88|2375.53|1011.20|2669.41|95.02|0.00|186.44|2375.53|2470.55|2561.97|2656.99|1364.33| +2450834|47682|2450908|46333|755203|3273|36143|46333|755203|3273|36143|1|34|3|4|16526|162|1001|27|53.10|153.45|33.75|3231.90|911.25|1433.70|4143.15|72.90|0.00|1615.68|911.25|984.15|2526.93|2599.83|-522.45| +2450834|47682|2450857|46333|755203|3273|36143|46333|755203|3273|36143|1|8|20|2|2002|18|1001|58|13.35|25.36|15.97|544.62|926.26|774.30|1470.88|74.10|0.00|735.44|926.26|1000.36|1661.70|1735.80|151.96| +2450834|47682|2450866|46333|755203|3273|36143|46333|755203|3273|36143|1|93|9|2|730|141|1001|23|62.19|172.26|63.73|2496.19|1465.79|1430.37|3961.98|14.65|0.00|1663.82|1465.79|1480.44|3129.61|3144.26|35.42| +2450834|70132|2450877|15695|1655190|1085|45380|15695|1655190|1085|45380|1|44|14|1|3002|226|1002|88|27.88|50.46|48.94|133.76|4306.72|2453.44|4440.48|129.20|0.00|799.04|4306.72|4435.92|5105.76|5234.96|1853.28| +2450834|70132|2450846|15695|1655190|1085|45380|15695|1655190|1085|45380|1|11|20|5|15488|39|1002|6|20.50|34.44|4.13|181.86|24.78|123.00|206.64|0.99|0.00|95.04|24.78|25.77|119.82|120.81|-98.22| +2450834|70132|2450869|15695|1655190|1085|45380|15695|1655190|1085|45380|1|67|8|3|2311|173|1002|3|69.33|129.64|44.07|256.71|132.21|207.99|388.92|3.96|0.00|112.77|132.21|136.17|244.98|248.94|-75.78| +2450834|70132|2450884|15695|1655190|1085|45380|15695|1655190|1085|45380|1|16|5|4|4664|85|1002|17|5.41|12.76|5.86|117.30|99.62|91.97|216.92|6.97|0.00|58.48|99.62|106.59|158.10|165.07|7.65| +2450834|70132|2450876|15695|1655190|1085|45380|15695|1655190|1085|45380|1|87|9|4|565|66|1002|62|69.96|155.31|136.67|1155.68|8473.54|4337.52|9629.22|593.14|0.00|4718.20|8473.54|9066.68|13191.74|13784.88|4136.02| +2450834|70132|2450904|15695|1655190|1085|45380|15695|1655190|1085|45380|1|20|12|5|7369|64|1002|78|74.36|148.72|55.02|7308.60|4291.56|5800.08|11600.16|214.57|0.00|2203.50|4291.56|4506.13|6495.06|6709.63|-1508.52| +2450834|70294|2450911|12291|1738409|6480|14598|12291|1738409|6480|14598|2|54|13|5|15529|220|1003|38|11.74|11.74|8.33|129.58|316.54|446.12|446.12|0.57|307.04|160.36|9.50|10.07|169.86|170.43|-436.62| +2450834|70294|2450849|12291|1738409|6480|14598|12291|1738409|6480|14598|2|85|15|2|2428|187|1003|21|13.11|22.68|21.99|14.49|461.79|275.31|476.28|18.28|157.00|0.00|304.79|323.07|304.79|323.07|29.48| +2450834|70294|2450898|12291|1738409|6480|14598|12291|1738409|6480|14598|2|25|14|3|1418|83|1003|37|29.07|81.39|39.06|1566.21|1445.22|1075.59|3011.43|17.19|1199.53|933.51|245.69|262.88|1179.20|1196.39|-829.90| +2450834|70294|2450877|12291|1738409|6480|14598|12291|1738409|6480|14598|2|62|1|3|2971|68|1003|69|84.38|86.91|73.00|959.79|5037.00|5822.22|5996.79|0.00|0.00|2817.96|5037.00|5037.00|7854.96|7854.96|-785.22| +2450834|70294|2450908|12291|1738409|6480|14598|12291|1738409|6480|14598|2|50|12|3|2588|18|1003|6|23.16|45.62|9.58|216.24|57.48|138.96|273.72|0.00|0.00|46.50|57.48|57.48|103.98|103.98|-81.48| +2450834|70294|2450905|12291|1738409|6480|14598|12291|1738409|6480|14598|2|62|10|3|1987|179|1003|36|72.87|212.05|44.53|6030.72|1603.08|2623.32|7633.80|32.06|0.00|305.28|1603.08|1635.14|1908.36|1940.42|-1020.24| +2450834|70294|2450893|12291|1738409|6480|14598|12291|1738409|6480|14598|2|105|7|1|2390|206|1003|98|2.16|2.85|2.42|42.14|237.16|211.68|279.30|2.37|0.00|27.44|237.16|239.53|264.60|266.97|25.48| +2450834|70294|2450838|12291|1738409|6480|14598|12291|1738409|6480|14598|2|47|18|4|12538|252|1003|14|82.80|239.29|31.10|2914.66|435.40|1159.20|3350.06|21.77|0.00|234.50|435.40|457.17|669.90|691.67|-723.80| +2450834|70294|2450839|12291|1738409|6480|14598|12291|1738409|6480|14598|2|8|7|2|1640|177|1003|40|13.51|23.23|8.82|576.40|352.80|540.40|929.20|24.69|0.00|278.40|352.80|377.49|631.20|655.89|-187.60| +2450834|1101|2450913|547|1895729|4864|19999|547|1895729|4864|19999|4|44|19|5|8164|222|1004|90|28.98|60.27|57.85|217.80|5206.50|2608.20|5424.30|416.52|0.00|1952.10|5206.50|5623.02|7158.60|7575.12|2598.30| +2450834|1101|2450911|547|1895729|4864|19999|547|1895729|4864|19999|4|73|3|2|16048|55|1004|97|54.38|96.25|79.88|1587.89|7748.36|5274.86|9336.25|619.86|0.00|1120.35|7748.36|8368.22|8868.71|9488.57|2473.50| +2450834|1101|2450916|547|1895729|4864|19999|547|1895729|4864|19999|4|17|1|3|10366|279|1004|66|52.72|78.02|60.07|1184.70|3964.62|3479.52|5149.32|198.23|0.00|1853.28|3964.62|4162.85|5817.90|6016.13|485.10| +2450834|1101|2450916|547|1895729|4864|19999|547|1895729|4864|19999|4|72|11|3|14203|19|1004|6|88.71|132.17|7.93|745.44|47.58|532.26|793.02|0.95|0.00|214.08|47.58|48.53|261.66|262.61|-484.68| +2450834|1101|2450883|547|1895729|4864|19999|547|1895729|4864|19999|4|82|5|2|13460|52|1004|66|79.83|154.87|21.68|8790.54|1430.88|5268.78|10221.42|128.77|0.00|4497.24|1430.88|1559.65|5928.12|6056.89|-3837.90| +2450834|1101|2450852|547|1895729|4864|19999|547|1895729|4864|19999|4|36|17|1|4598|248|1004|35|70.70|199.37|85.72|3977.75|3000.20|2474.50|6977.95|240.01|0.00|2930.55|3000.20|3240.21|5930.75|6170.76|525.70| +2450834|1101|2450875|547|1895729|4864|19999|547|1895729|4864|19999|4|71|13|1|1729|231|1004|60|40.95|49.14|38.82|619.20|2329.20|2457.00|2948.40|0.00|0.00|1356.00|2329.20|2329.20|3685.20|3685.20|-127.80| +2450834|1101|2450865|547|1895729|4864|19999|547|1895729|4864|19999|4|17|6|4|1453|192|1004|76|5.66|6.84|5.47|104.12|415.72|430.16|519.84|20.78|0.00|51.68|415.72|436.50|467.40|488.18|-14.44| +2450834|25477|2450919|40403|476548|1940|32885|40403|476548|1940|32885|1|108|9|4|15526|281|1005|92|47.49|62.21|36.08|2403.96|3319.36|4369.08|5723.32|66.38|0.00|1716.72|3319.36|3385.74|5036.08|5102.46|-1049.72| +2450834|25477|2450877|40403|476548|1940|32885|40403|476548|1940|32885|1|28|18|2|1154|266|1005|5|89.05|193.23|65.69|637.70|328.45|445.25|966.15|11.69|36.12|164.20|292.33|304.02|456.53|468.22|-152.92| +2450834|25477|2450844|40403|476548|1940|32885|40403|476548|1940|32885|1|43|18|1|3991|193|1005|55|82.83|173.94|19.13|8514.55|1052.15|4555.65|9566.70|42.08|0.00|1052.15|1052.15|1094.23|2104.30|2146.38|-3503.50| +2450834|25477|2450851|40403|476548|1940|32885|40403|476548|1940|32885|1|3|15|3|13772|75|1005|73|69.49|135.50|113.82|1582.64|8308.86|5072.77|9891.50|332.35|0.00|3165.28|8308.86|8641.21|11474.14|11806.49|3236.09| +2450834|25477|2450923|40403|476548|1940|32885|40403|476548|1940|32885|1|53|5|5|2692|215|1005|22|78.13|233.60|60.73|3803.14|1336.06|1718.86|5139.20|51.43|601.22|924.88|734.84|786.27|1659.72|1711.15|-984.02| +2450834|25477|2450864|40403|476548|1940|32885|40403|476548|1940|32885|1|1|12|3|7849|73|1005|71|21.05|31.15|0.00|2211.65|0.00|1494.55|2211.65|0.00|0.00|486.35|0.00|0.00|486.35|486.35|-1494.55| +2450834|79499|2450919|36734|1579970|2759|41195|36734|1579970|2759|41195|1|33|5|5|386|166|1006|62|24.20|59.29|53.95|331.08|3344.90|1500.40|3675.98|0.00|1572.10|587.76|1772.80|1772.80|2360.56|2360.56|272.40| +2450834|79499|2450849|36734|1579970|2759|41195|36734|1579970|2759|41195|1|20|2|3|1256|173|1006|91|30.49|89.03|0.00|8101.73|0.00|2774.59|8101.73|0.00|0.00|2105.74|0.00|0.00|2105.74|2105.74|-2774.59| +2450834|79499|2450842|36734|1579970|2759|41195|36734|1579970|2759|41195|1|101|16|4|12379|173|1006|22|94.34|241.51|91.77|3294.28|2018.94|2075.48|5313.22|99.33|363.40|1753.18|1655.54|1754.87|3408.72|3508.05|-419.94| +2450834|79499|2450855|36734|1579970|2759|41195|36734|1579970|2759|41195|1|17|3|2|7186|38|1006|97|90.67|243.90|114.63|12539.19|11119.11|8794.99|23658.30|810.58|2112.63|9463.32|9006.48|9817.06|18469.80|19280.38|211.49| +2450834|25896|2450911|61856|512917|5578|18686|61856|512917|5578|18686|4|12|9|1|13831|267|1007|50|1.12|1.59|0.38|60.50|19.00|56.00|79.50|0.95|0.00|38.00|19.00|19.95|57.00|57.95|-37.00| +2450834|25896|2450851|61856|512917|5578|18686|61856|512917|5578|18686|4|41|17|1|9248|127|1007|73|21.33|34.34|15.45|1378.97|1127.85|1557.09|2506.82|11.27|0.00|751.90|1127.85|1139.12|1879.75|1891.02|-429.24| +2450834|25896|2450886|61856|512917|5578|18686|61856|512917|5578|18686|4|49|2|4|15668|16|1007|16|33.32|40.31|12.49|445.12|199.84|533.12|644.96|5.99|0.00|167.68|199.84|205.83|367.52|373.51|-333.28| +2450834|25896|2450897|61856|512917|5578|18686|61856|512917|5578|18686|4|65|13|5|15992|229|1007|89|11.62|15.57|4.04|1026.17|359.56|1034.18|1385.73|17.43|10.78|0.00|348.78|366.21|348.78|366.21|-685.40| +2450834|25896|2450905|61856|512917|5578|18686|61856|512917|5578|18686|4|92|2|5|12248|184|1007|78|80.96|201.59|104.82|7548.06|8175.96|6314.88|15724.02|408.79|0.00|5346.12|8175.96|8584.75|13522.08|13930.87|1861.08| +2450834|25896|2450841|61856|512917|5578|18686|61856|512917|5578|18686|4|97|18|5|8770|294|1007|40|8.88|11.01|5.83|207.20|233.20|355.20|440.40|6.99|0.00|101.20|233.20|240.19|334.40|341.39|-122.00| +2450834|25896|2450847|61856|512917|5578|18686|61856|512917|5578|18686|4|64|3|5|13858|158|1007|2|59.74|81.24|43.86|74.76|87.72|119.48|162.48|4.21|3.50|24.36|84.22|88.43|108.58|112.79|-35.26| +2450834|65944|2450884|1115|866516|3917|23318|1115|866516|3917|23318|2|97|4|1|1576|293|1008|76|7.51|16.82|10.09|511.48|766.84|570.76|1278.32|38.34|0.00|38.00|766.84|805.18|804.84|843.18|196.08| +2450834|65944|2450869|1115|866516|3917|23318|1115|866516|3917|23318|2|33|8|4|17641|118|1008|34|51.11|122.66|69.91|1793.50|2376.94|1737.74|4170.44|23.76|0.00|1042.44|2376.94|2400.70|3419.38|3443.14|639.20| +2450834|65944|2450895|1115|866516|3917|23318|1115|866516|3917|23318|2|32|17|3|17710|45|1008|62|88.76|228.11|173.36|3394.50|10748.32|5503.12|14142.82|967.34|0.00|4525.38|10748.32|11715.66|15273.70|16241.04|5245.20| +2450834|65944|2450857|1115|866516|3917|23318|1115|866516|3917|23318|2|56|1|1|16909|11|1008|73|60.73|77.12|20.82|4109.90|1519.86|4433.29|5629.76|75.99|0.00|449.68|1519.86|1595.85|1969.54|2045.53|-2913.43| +2450834|51582|2450869|47003|1878148|2893|40038|47003|1878148|2893|40038|4|84|6|3|15268|245|1009|97|83.40|210.16|155.51|5301.05|15084.47|8089.80|20385.52|1055.91|0.00|1834.27|15084.47|16140.38|16918.74|17974.65|6994.67| +2450834|51582|2450914|47003|1878148|2893|40038|47003|1878148|2893|40038|4|92|1|5|10724|253|1009|83|61.95|164.16|0.00|13625.28|0.00|5141.85|13625.28|0.00|0.00|3269.37|0.00|0.00|3269.37|3269.37|-5141.85| +2450834|51582|2450852|47003|1878148|2893|40038|47003|1878148|2893|40038|4|65|12|2|6856|215|1009|46|62.45|157.37|143.20|651.82|6587.20|2872.70|7239.02|31.61|5796.73|2895.24|790.47|822.08|3685.71|3717.32|-2082.23| +2450834|51582|2450906|47003|1878148|2893|40038|47003|1878148|2893|40038|4|50|6|3|9841|203|1009|27|66.55|125.11|97.58|743.31|2634.66|1796.85|3377.97|210.77|0.00|1148.31|2634.66|2845.43|3782.97|3993.74|837.81| +2450834|51582|2450913|47003|1878148|2893|40038|47003|1878148|2893|40038|4|20|15|2|17203|188|1009|8|59.55|153.04|123.96|232.64|991.68|476.40|1224.32|69.41|0.00|24.48|991.68|1061.09|1016.16|1085.57|515.28| +2450834|51582|2450892|47003|1878148|2893|40038|47003|1878148|2893|40038|4|8|6|1|11068|26|1009|91|82.05|96.81|34.85|5638.36|3171.35|7466.55|8809.71|285.42|0.00|4052.23|3171.35|3456.77|7223.58|7509.00|-4295.20| +2450834|51582|2450904|47003|1878148|2893|40038|47003|1878148|2893|40038|4|57|3|4|17821|244|1009|46|75.79|201.60|179.42|1020.28|8253.32|3486.34|9273.60|247.59|0.00|649.06|8253.32|8500.91|8902.38|9149.97|4766.98| +2450834|51582|2450879|47003|1878148|2893|40038|47003|1878148|2893|40038|4|102|5|3|4684|299|1009|8|65.52|90.41|24.41|528.00|195.28|524.16|723.28|3.90|0.00|188.00|195.28|199.18|383.28|387.18|-328.88| +2450834|14471|2450887|84013|1884427|2441|29655|84013|1884427|2441|29655|4|43|7|5|10846|12|1010|88|24.68|33.56|5.70|2451.68|501.60|2171.84|2953.28|10.03|0.00|265.76|501.60|511.63|767.36|777.39|-1670.24| +2450834|14471|2450913|84013|1884427|2441|29655|84013|1884427|2441|29655|4|62|17|2|8665|119|1010|76|44.78|124.93|116.18|665.00|8829.68|3403.28|9494.68|174.82|88.29|3512.72|8741.39|8916.21|12254.11|12428.93|5338.11| +2450834|14471|2450905|84013|1884427|2441|29655|84013|1884427|2441|29655|4|8|6|3|2414|17|1010|3|41.43|106.88|87.64|57.72|262.92|124.29|320.64|18.22|60.47|60.90|202.45|220.67|263.35|281.57|78.16| +2450834|14471|2450862|84013|1884427|2441|29655|84013|1884427|2441|29655|4|85|12|5|5330|48|1010|11|63.26|129.05|32.26|1064.69|354.86|695.86|1419.55|21.29|0.00|596.20|354.86|376.15|951.06|972.35|-341.00| +2450834|14471|2450836|84013|1884427|2441|29655|84013|1884427|2441|29655|4|108|17|4|4298|226|1010|22|58.88|120.70|13.27|2363.46|291.94|1295.36|2655.40|14.88|43.79|743.38|248.15|263.03|991.53|1006.41|-1047.21| +2450834|14471|2450841|84013|1884427|2441|29655|84013|1884427|2441|29655|4|98|12|4|296|177|1010|84|67.91|79.45|15.89|5339.04|1334.76|5704.44|6673.80|120.12|0.00|1668.24|1334.76|1454.88|3003.00|3123.12|-4369.68| +2450834|14471|2450883|84013|1884427|2441|29655|84013|1884427|2441|29655|4|44|11|4|11647|103|1010|60|56.41|100.97|13.12|5271.00|787.20|3384.60|6058.20|32.66|133.82|847.80|653.38|686.04|1501.18|1533.84|-2731.22| +2450834|14471|2450916|84013|1884427|2441|29655|84013|1884427|2441|29655|4|16|10|1|6019|193|1010|92|95.18|96.13|73.05|2123.36|6720.60|8756.56|8843.96|201.61|0.00|2918.24|6720.60|6922.21|9638.84|9840.45|-2035.96| +2450834|20118|2450901|86267|453852|1884|39916|86267|453852|1884|39916|1|88|7|1|4976|25|1011|50|2.50|3.17|1.96|60.50|98.00|125.00|158.50|3.17|45.08|41.00|52.92|56.09|93.92|97.09|-72.08| +2450834|20118|2450896|86267|453852|1884|39916|86267|453852|1884|39916|1|73|16|4|12904|212|1011|74|27.73|74.59|20.13|4030.04|1489.62|2052.02|5519.66|134.06|0.00|220.52|1489.62|1623.68|1710.14|1844.20|-562.40| +2450834|20118|2450913|86267|453852|1884|39916|86267|453852|1884|39916|1|52|7|2|12706|102|1011|31|28.13|45.85|8.71|1151.34|270.01|872.03|1421.35|2.70|0.00|0.00|270.01|272.71|270.01|272.71|-602.02| +2450834|20118|2450876|86267|453852|1884|39916|86267|453852|1884|39916|1|42|1|4|17414|182|1011|70|5.39|13.25|10.20|213.50|714.00|377.30|927.50|7.14|0.00|296.80|714.00|721.14|1010.80|1017.94|336.70| +2450834|15347|2450889|67055|437675|3048|14468|67055|437675|3048|14468|1|54|9|1|8368|17|1012|82|80.24|186.95|39.25|12111.40|3218.50|6579.68|15329.90|32.18|0.00|305.86|3218.50|3250.68|3524.36|3556.54|-3361.18| +2450834|15347|2450859|67055|437675|3048|14468|67055|437675|3048|14468|1|103|7|3|3380|10|1012|74|56.47|97.12|26.22|5246.60|1940.28|4178.78|7186.88|77.61|0.00|2227.40|1940.28|2017.89|4167.68|4245.29|-2238.50| +2450834|15347|2450869|67055|437675|3048|14468|67055|437675|3048|14468|1|105|3|1|16864|239|1012|38|50.12|71.67|26.51|1716.08|1007.38|1904.56|2723.46|19.04|735.38|1089.08|272.00|291.04|1361.08|1380.12|-1632.56| +2450834|15347|2450836|67055|437675|3048|14468|67055|437675|3048|14468|1|106|5|3|7796|91|1012|20|81.39|95.22|43.80|1028.40|876.00|1627.80|1904.40|26.28|0.00|266.60|876.00|902.28|1142.60|1168.88|-751.80| +2450834|15347|2450879|67055|437675|3048|14468|67055|437675|3048|14468|1|86|7|4|16676|212|1012|87|50.67|117.55|72.88|3886.29|6340.56|4408.29|10226.85|91.30|1775.35|1431.15|4565.21|4656.51|5996.36|6087.66|156.92| +2450834|15347|2450849|67055|437675|3048|14468|67055|437675|3048|14468|1|93|9|3|3434|39|1012|91|7.60|20.21|4.85|1397.76|441.35|691.60|1839.11|0.00|211.84|91.91|229.51|229.51|321.42|321.42|-462.09| +2450834|15347|2450899|67055|437675|3048|14468|67055|437675|3048|14468|1|47|6|4|12514|110|1012|84|99.32|164.87|64.29|8448.72|5400.36|8342.88|13849.08|108.00|0.00|5816.16|5400.36|5508.36|11216.52|11324.52|-2942.52| +2450834|15347|2450907|67055|437675|3048|14468|67055|437675|3048|14468|1|97|5|2|4873|1|1012|100|63.60|83.95|1.67|8228.00|167.00|6360.00|8395.00|1.67|0.00|2938.00|167.00|168.67|3105.00|3106.67|-6193.00| +2450834|15347|2450862|67055|437675|3048|14468|67055|437675|3048|14468|1|43|16|2|10102|142|1012|30|83.82|207.87|193.31|436.80|5799.30|2514.60|6236.10|347.95|0.00|2182.50|5799.30|6147.25|7981.80|8329.75|3284.70| +2450834|46416|2450870|84861|67956|5098|17793|84861|67956|5098|17793|1|27|2|2|860|109|1013|26|30.75|64.88|55.79|236.34|1450.54|799.50|1686.88|58.02|0.00|742.04|1450.54|1508.56|2192.58|2250.60|651.04| +2450834|46416|2450924|84861|67956|5098|17793|84861|67956|5098|17793|1|33|19|4|6403|256|1013|74|73.74|104.71|63.87|3022.16|4726.38|5456.76|7748.54|125.72|2930.35|0.00|1796.03|1921.75|1796.03|1921.75|-3660.73| +2450834|46416|2450860|84861|67956|5098|17793|84861|67956|5098|17793|1|59|10|1|9112|50|1013|28|29.30|59.18|42.01|480.76|1176.28|820.40|1657.04|70.57|0.00|795.20|1176.28|1246.85|1971.48|2042.05|355.88| +2450834|46416|2450902|84861|67956|5098|17793|84861|67956|5098|17793|1|8|10|1|1352|190|1013|44|38.48|87.34|41.92|1998.48|1844.48|1693.12|3842.96|0.00|0.00|1613.92|1844.48|1844.48|3458.40|3458.40|151.36| +||||67956|||84861|67956||17793|1||||7273||1013|13|73.47|||111.67||955.11|1594.97|0.00||||1483.30|2280.72|2280.72|528.19| +2450834|46416|2450886|84861|67956|5098|17793|84861|67956|5098|17793|1|97|5|1|16417|219|1013|92|18.47|27.70|7.20|1886.00|662.40|1699.24|2548.40|6.62|0.00|101.20|662.40|669.02|763.60|770.22|-1036.84| +2450834|62004|2450868|45781|1912573|2565|49357|2549|1238588|580|37772|2|94|13|2|4424|147|1014|63|71.86|97.01|63.05|2139.48|3972.15|4527.18|6111.63|238.32|0.00|2811.06|3972.15|4210.47|6783.21|7021.53|-555.03| +2450834|62004|2450908|45781|1912573|2565|49357|2549|1238588|580|37772|2|105|15|2|12416|171|1014|36|74.19|144.67|0.00|5208.12|0.00|2670.84|5208.12|0.00|0.00|1926.72|0.00|0.00|1926.72|1926.72|-2670.84| +2450834|62004|2450911|45781|1912573|2565|49357|2549|1238588|580|37772|2|49|16|4|6490|248|1014|92|3.47|3.85|3.15|64.40|289.80|319.24|354.20|23.18|0.00|148.12|289.80|312.98|437.92|461.10|-29.44| +2450834|62004|2450892|45781|1912573|2565|49357|2549|1238588|580|37772|2|55|10|3|17452|197|1014|85|2.32|4.10|1.88|188.70|159.80|197.20|348.50|6.95|20.77|170.00|139.03|145.98|309.03|315.98|-58.17| +2450834|62004|2450909|45781|1912573|2565|49357|2549|1238588|580|37772|2|90|11|2|4882|189|1014|82|50.19|114.43|32.04|6755.98|2627.28|4115.58|9383.26|122.95|578.00|1688.38|2049.28|2172.23|3737.66|3860.61|-2066.30| +2450834|62004|2450857|45781|1912573|2565|49357|2549|1238588|580|37772|2|90|3|5|8275|216|1014|93|43.74|119.84|61.11|5461.89|5683.23|4067.82|11145.12|0.00|0.00|4234.29|5683.23|5683.23|9917.52|9917.52|1615.41| +2450834|62004|2450882|45781|1912573|2565|49357|2549|1238588|580|37772|2|94|17|5|12250|91|1014|67|64.02|153.00|130.05|1537.65|8713.35|4289.34|10251.00|261.40|0.00|4202.91|8713.35|8974.75|12916.26|13177.66|4424.01| +2450834|62004|2450876|45781|1912573|2565|49357|2549|1238588|580|37772|2|50|1|2|9733|244|1014|33|35.85|60.22|1.80|1927.86|59.40|1183.05|1987.26|3.56|0.00|238.26|59.40|62.96|297.66|301.22|-1123.65| +2450834|62004|2450880|45781|1912573|2565|49357|2549|1238588|580|37772|2|11|17|3|14620|93|1014|6|26.52|77.43|3.09|446.04|18.54|159.12|464.58|0.18|0.00|83.58|18.54|18.72|102.12|102.30|-140.58| +2450834|62004|2450894|45781|1912573|2565|49357|2549|1238588|580|37772|2|30|7|3|14894|142|1014|14|47.82|127.20|22.89|1460.34|320.46|669.48|1780.80|25.63|0.00|498.54|320.46|346.09|819.00|844.63|-349.02| +2450834|33265|2450840|19461|840029|6197|15965|19461|840029|6197|15965|4|14|2|3|11908|65|1015|89|45.98|52.41|36.16|1446.25|3218.24|4092.22|4664.49|160.91|0.00|1678.54|3218.24|3379.15|4896.78|5057.69|-873.98| +2450834|33265|2450840|19461|840029|6197|15965|19461|840029|6197|15965|4|13|3|4|9469|233|1015|64|47.38|91.44|63.09|1814.40|4037.76|3032.32|5852.16|242.26|0.00|292.48|4037.76|4280.02|4330.24|4572.50|1005.44| +2450834|33265|2450855|19461|840029|6197|15965|19461|840029|6197|15965|4|12|8|2|9104|210|1015|24|30.40|59.58|52.43|171.60|1258.32|729.60|1429.92|50.33|0.00|257.28|1258.32|1308.65|1515.60|1565.93|528.72| +2450834|33265|2450901|19461|840029|6197|15965|19461|840029|6197|15965|4|50|20|2|1778|44|1015|10|69.10|79.46|31.78|476.80|317.80|691.00|794.60|25.42|0.00|0.00|317.80|343.22|317.80|343.22|-373.20| +2450834|33265|2450880|19461|840029|6197|15965|19461|840029|6197|15965|4|95|9|4|10105|266|1015|33|31.13|81.24|16.24|2145.00|535.92|1027.29|2680.92|37.51|0.00|777.15|535.92|573.43|1313.07|1350.58|-491.37| +2450834|34553|2450840|40359|1736021|6530|42639|40359|1736021|6530|42639|2|70|19|1|10033|255|1016|76|12.85|37.00|12.95|1827.80|984.20|976.60|2812.00|22.73|659.41|140.60|324.79|347.52|465.39|488.12|-651.81| +2450834|34553|2450896|40359|1736021|6530|42639|40359|1736021|6530|42639|2|38|19|1|17524|255|1016|35|54.53|163.59|135.77|973.70|4751.95|1908.55|5725.65|47.51|0.00|57.05|4751.95|4799.46|4809.00|4856.51|2843.40| +2450834|34553|2450896|40359|1736021|6530|42639|40359|1736021|6530|42639|2|42|15|5|4405|195|1016|7|84.73|151.66|9.09|997.99|63.63|593.11|1061.62|0.89|19.08|148.61|44.55|45.44|193.16|194.05|-548.56| +2450834|34553|2450902|40359|1736021|6530|42639|40359|1736021|6530|42639|2|90|13|1|2728|218|1016|53|20.52|21.54|17.66|205.64|935.98|1087.56|1141.62|9.35|0.00|376.30|935.98|945.33|1312.28|1321.63|-151.58| +2450834|34553|2450878|40359|1736021|6530|42639|40359|1736021|6530|42639|2|14|8|5|3142|68|1016|55|9.61|14.99|7.34|420.75|403.70|528.55|824.45|20.18|0.00|321.20|403.70|423.88|724.90|745.08|-124.85| +2450834|34553|2450889|40359|1736021|6530|42639|40359|1736021|6530|42639|2|41|9|2|7832|223|1016|45|43.27|64.03|16.64|2132.55|748.80|1947.15|2881.35|67.39|0.00|864.00|748.80|816.19|1612.80|1680.19|-1198.35| +2450834|28322|2450913|70397|756583|5169|14155|70397|756583|5169|14155|2|101|17|5|12553|211|1017|69|96.42|138.84|15.27|8526.33|1053.63|6652.98|9579.96|84.29|0.00|1149.54|1053.63|1137.92|2203.17|2287.46|-5599.35| +2450834|28322|2450865|70397|756583|5169|14155|70397|756583|5169|14155|2|53|14|4|2324|56|1017|80|13.94|21.32|6.82|1160.00|545.60|1115.20|1705.60|32.73|0.00|426.40|545.60|578.33|972.00|1004.73|-569.60| +2450834|28322|2450883|70397|756583|5169|14155|70397|756583|5169|14155|2|68|8|5|5695|232|1017|37|95.19|274.14|205.60|2535.98|7607.20|3522.03|10143.18|76.07|0.00|4462.94|7607.20|7683.27|12070.14|12146.21|4085.17| +2450834|28322|2450903|70397|756583|5169|14155|70397|756583|5169|14155|2|58|9|4|2893|190|1017|3|98.14|114.82|56.26|175.68|168.78|294.42|344.46|1.68|0.00|130.89|168.78|170.46|299.67|301.35|-125.64| +2450834|28322|2450874|70397|756583|5169|14155|70397|756583|5169|14155|2|46|15|2|7030|254|1017|7|53.66|126.10|13.87|785.61|97.09|375.62|882.70|0.00|44.66|406.00|52.43|52.43|458.43|458.43|-323.19| +2450834|63696|2450839|40408|1012152|4696|46393|40408|1012152|4696|46393|4|22|13|4|5725|1|1018|77|57.50|70.15|22.44|3673.67|1727.88|4427.50|5401.55|155.50|0.00|2052.05|1727.88|1883.38|3779.93|3935.43|-2699.62| +2450834|63696|2450924|40408|1012152|4696|46393|40408|1012152|4696|46393|4|66|11|5|6158|5|1018|85|44.39|93.21|62.45|2614.60|5308.25|3773.15|7922.85|0.00|0.00|79.05|5308.25|5308.25|5387.30|5387.30|1535.10| +2450834|63696|2450907|40408|1012152|4696|46393|40408|1012152|4696|46393|4|102|11|4|6322|63|1018|13|43.05|111.06|85.51|332.15|1111.63|559.65|1443.78|33.34|0.00|303.16|1111.63|1144.97|1414.79|1448.13|551.98| +2450834|63696|2450862|40408|1012152|4696|46393|40408|1012152|4696|46393|4|3|2|1|11965|67|1018|45|85.50|244.53|200.51|1980.90|9022.95|3847.50|11003.85|721.83|0.00|3300.75|9022.95|9744.78|12323.70|13045.53|5175.45| +2450834|63696|2450884|40408|1012152|4696|46393|40408|1012152|4696|46393|4|94|1|1|17884|164|1018|40|54.53|62.16|4.97|2287.60|198.80|2181.20|2486.40|1.98|0.00|969.60|198.80|200.78|1168.40|1170.38|-1982.40| +2450834|63696|2450850|40408|1012152|4696|46393|40408|1012152|4696|46393|4|46|7|2|7345|115|1018|11|45.78|124.06|102.96|232.10|1132.56|503.58|1364.66|34.88|634.23|68.20|498.33|533.21|566.53|601.41|-5.25| +2450834|67793|2450873|52582|1785501|6067|46586|52582|1785501|6067|46586|4|82|12|5|451|59|1019|45|25.56|39.36|0.00|1771.20|0.00|1150.20|1771.20|0.00|0.00|265.50|0.00|0.00|265.50|265.50|-1150.20| +2450834|67793|2450874|52582|1785501|6067|46586|52582|1785501|6067|46586|4|65|14|1|10874|71|1019|91|22.75|36.17|16.63|1778.14|1513.33|2070.25|3291.47|105.93|0.00|65.52|1513.33|1619.26|1578.85|1684.78|-556.92| +2450834|67793|2450886|52582|1785501|6067|46586|52582|1785501|6067|46586|4|46|1|3|2986|157|1019|36|20.78|60.88|28.61|1161.72|1029.96|748.08|2191.68|10.29|0.00|218.88|1029.96|1040.25|1248.84|1259.13|281.88| +2450834|67793|2450878|52582|1785501|6067|46586|52582|1785501|6067|46586|4|63|1|4|2926|123|1019|39|86.65|258.21|131.68|4934.67|5135.52|3379.35|10070.19|51.35|0.00|3322.80|5135.52|5186.87|8458.32|8509.67|1756.17| +2450834|67793|2450903|52582|1785501|6067|46586|52582|1785501|6067|46586|4|24|1|1|16567|54|1019|75|91.68|204.44|188.08|1227.00|14106.00|6876.00|15333.00|564.24|0.00|1992.75|14106.00|14670.24|16098.75|16662.99|7230.00| +2450834|67793|2450892|52582|1785501|6067|46586|52582|1785501|6067|46586|4|16|9|5|982|257|1019|11|98.66|197.32|132.20|716.32|1454.20|1085.26|2170.52|43.62|0.00|759.66|1454.20|1497.82|2213.86|2257.48|368.94| +2450834|67793|2450921|52582|1785501|6067|46586|52582|1785501|6067|46586|4|44|4|2|10156|165|1019|10|76.62|86.58|27.70|588.80|277.00|766.20|865.80|2.77|0.00|173.10|277.00|279.77|450.10|452.87|-489.20| +2450834|61161|2450882|53761|1644698|1235|42619|53761|1644698|1235|42619|1|30|15|5|7111|233|1020|61|82.01|106.61|51.17|3381.84|3121.37|5002.61|6503.21|55.56|343.35|1755.58|2778.02|2833.58|4533.60|4589.16|-2224.59| +2450834|61161|2450896|53761|1644698|1235|42619|53761|1644698|1235|42619|1|78|1|4|10420|111|1020|2|38.50|68.91|43.41|51.00|86.82|77.00|137.82|7.81|0.00|59.26|86.82|94.63|146.08|153.89|9.82| +2450834|61161|2450921|53761|1644698|1235|42619|53761|1644698|1235|42619|1|82|9|3|3703|276|1020|64|20.27|53.51|37.99|993.28|2431.36|1297.28|3424.64|145.88|0.00|479.36|2431.36|2577.24|2910.72|3056.60|1134.08| +2450834|61161|2450842|53761|1644698|1235|42619|53761|1644698|1235|42619|1|35|4|2|1888|228|1020|1|6.89|14.26|6.13|8.13|6.13|6.89|14.26|0.18|0.00|7.13|6.13|6.31|13.26|13.44|-0.76| +2450834|61161|2450849|53761|1644698|1235|42619|53761|1644698|1235|42619|1|107|2|5|12010|93|1020|8|43.65|68.96|5.51|507.60|44.08|349.20|551.68|3.08|0.00|49.60|44.08|47.16|93.68|96.76|-305.12| +2450834|61161|2450896|53761|1644698|1235|42619|53761|1644698|1235|42619|1|77|8|1|4489|190|1020|70|92.91|240.63|178.06|4379.90|12464.20|6503.70|16844.10|872.49|0.00|0.00|12464.20|13336.69|12464.20|13336.69|5960.50| +2450834|61161|2450870|53761|1644698|1235|42619|53761|1644698|1235|42619|1|34|18|4|2336|41|1020|84|1.87|2.44|0.90|129.36|75.60|157.08|204.96|5.29|0.00|47.04|75.60|80.89|122.64|127.93|-81.48| +2450834|61161|2450903|53761|1644698|1235|42619|53761|1644698|1235|42619|1|34|13|4|13436|13|1020|39|77.56|215.61|131.52|3279.51|5129.28|3024.84|8408.79|46.67|4462.47|3615.69|666.81|713.48|4282.50|4329.17|-2358.03| +2450834|61161|2450865|53761|1644698|1235|42619|53761|1644698|1235|42619|1|17|12|3|6283|246|1020|97|27.92|50.81|4.57|4485.28|443.29|2708.24|4928.57|39.89|0.00|2266.89|443.29|483.18|2710.18|2750.07|-2264.95| +2450834|61161|2450914|53761|1644698|1235|42619|53761|1644698|1235|42619|1|79|3|4|5818|136|1020|20|51.81|147.65|75.30|1447.00|1506.00|1036.20|2953.00|0.00|0.00|1181.20|1506.00|1506.00|2687.20|2687.20|469.80| +2450834|61161|2450862|53761|1644698|1235|42619|53761|1644698|1235|42619|1|49|18|1|914|298|1020|42|90.41|249.53|0.00|10480.26|0.00|3797.22|10480.26|0.00|0.00|523.74|0.00|0.00|523.74|523.74|-3797.22| +2450834|61161|2450917|53761|1644698|1235|42619|53761|1644698|1235|42619|1|85|3|4|14515|113|1020|38|65.29|176.28|19.39|5961.82|736.82|2481.02|6698.64|15.62|346.30|1272.62|390.52|406.14|1663.14|1678.76|-2090.50| +2450834|69167|2450876|60666|31241|1369|6297|60666|31241|1369|6297|4|85|11|4|8920|50|1021|32|68.42|156.68|34.46|3911.04|1102.72|2189.44|5013.76|27.34|419.03|350.72|683.69|711.03|1034.41|1061.75|-1505.75| +2450834|69167|2450863|60666|31241|1369|6297|60666|31241|1369|6297|4|4|13|1|9670|130|1021|81|3.53|9.56|9.27|23.49|750.87|285.93|774.36|1.50|713.32|340.20|37.55|39.05|377.75|379.25|-248.38| +2450834|69167|2450882|60666|31241|1369|6297|60666|31241|1369|6297|4|32|20|2|11707|98|1021|88|45.73|68.13|53.14|1319.12|4676.32|4024.24|5995.44|187.05|0.00|1738.00|4676.32|4863.37|6414.32|6601.37|652.08| +2450834|69167|2450885|60666|31241|1369|6297|60666|31241|1369|6297|4|42|11|1|13981|241|1021|94|35.96|81.98|72.96|847.88|6858.24|3380.24|7706.12|205.74|0.00|769.86|6858.24|7063.98|7628.10|7833.84|3478.00| +2450834|69167|2450918|60666|31241|1369|6297|60666|31241|1369|6297|4|13|10|4|13700|4|1021|39|95.35|239.32|215.38|933.66|8399.82|3718.65|9333.48|25.19|7139.84|4666.74|1259.98|1285.17|5926.72|5951.91|-2458.67| +2450834|69167|2450856|60666|31241|1369|6297|60666|31241|1369|6297|4|72|10|3|13213|129|1021|47|61.78|106.87|40.61|3114.22|1908.67|2903.66|5022.89|2.86|1622.36|1255.37|286.31|289.17|1541.68|1544.54|-2617.35| +2450834|69167|2450912|60666|31241|1369|6297|60666|31241|1369|6297|4|68|19|2|4858|12|1021|100|61.46|104.48|13.58|9090.00|1358.00|6146.00|10448.00|122.22|0.00|3552.00|1358.00|1480.22|4910.00|5032.22|-4788.00| +2450834|69167|2450851|60666|31241|1369|6297|60666|31241|1369|6297|4|102|8|5|15848|64|1021|66|78.54|106.02|53.01|3498.66|3498.66|5183.64|6997.32|69.97|0.00|2098.80|3498.66|3568.63|5597.46|5667.43|-1684.98| +2450834|69167|2450867|60666|31241|1369|6297|60666|31241|1369|6297|4|11|20|5|10483|282|1021|59|68.72|82.46|37.93|2627.27|2237.87|4054.48|4865.14|44.75|0.00|388.81|2237.87|2282.62|2626.68|2671.43|-1816.61| +2450835|47636|2450895|46323|1569350|2777|20204|46323|1569350|2777|20204|4|105|14|2|13682|141|1022|14|64.81|93.97|26.31|947.24|368.34|907.34|1315.58|11.05|0.00|631.40|368.34|379.39|999.74|1010.79|-539.00| +2450835|47636|2450868|46323|1569350|2777|20204|46323|1569350|2777|20204|4|62|10|2|16942|166|1022|87|46.41|93.74|52.49|3588.75|4566.63|4037.67|8155.38|319.66|0.00|1059.66|4566.63|4886.29|5626.29|5945.95|528.96| +2450835|47636|2450877|46323|1569350|2777|20204|46323|1569350|2777|20204|4|16|10|2|8053|294|1022|99|33.83|35.85|2.50|3301.65|247.50|3349.17|3549.15|14.85|0.00|709.83|247.50|262.35|957.33|972.18|-3101.67| +2450835|47636|2450911|46323|1569350|2777|20204|46323|1569350|2777|20204|4|4|16|4|11018|265|1022|48|92.65|274.24|180.99|4476.00|8687.52|4447.20|13163.52|766.23|173.75|2895.84|8513.77|9280.00|11409.61|12175.84|4066.57| +2450835|47636|2450877|46323|1569350|2777|20204|46323|1569350|2777|20204|4|60|12|2|6832|52|1022|80|31.79|54.04|2.70|4107.20|216.00|2543.20|4323.20|10.80|0.00|1599.20|216.00|226.80|1815.20|1826.00|-2327.20| +2450835|47636|2450919|46323|1569350|2777|20204|46323|1569350|2777|20204|4|31|12|3|3577|184|1022|21|36.84|85.83|24.03|1297.80|504.63|773.64|1802.43|21.44|75.69|126.00|428.94|450.38|554.94|576.38|-344.70| +2450835|61752|2450869|56858|1798089|920|47798|56858|1798089|920|47798|1|89|7|4|9056|251|1023|100|88.66|218.10|170.11|4799.00|17011.00|8866.00|21810.00|1530.99|0.00|2181.00|17011.00|18541.99|19192.00|20722.99|8145.00| +2450835|61752|2450922|56858|1798089|920|47798|56858|1798089|920|47798|1|50|11|1|17540|241|1023|45|17.22|44.42|1.77|1919.25|79.65|774.90|1998.90|3.98|0.00|279.45|79.65|83.63|359.10|363.08|-695.25| +2450835|61752|2450837|56858|1798089|920|47798|56858|1798089|920|47798|1|37|1|5|15295|64|1023|78|3.31|6.91|2.34|356.46|182.52|258.18|538.98|0.87|94.91|113.10|87.61|88.48|200.71|201.58|-170.57| +2450835|61752|2450879|56858|1798089|920|47798|56858|1798089|920|47798|1|29|13|2|6946|7|1023|85|53.04|110.32|103.70|562.70|8814.50|4508.40|9377.20|162.18|705.16|2906.15|8109.34|8271.52|11015.49|11177.67|3600.94| +2450835|61752|2450854|56858|1798089|920|47798|56858|1798089|920|47798|1|25|7|4|9838|108|1023|69|4.71|5.69|1.13|314.64|77.97|324.99|392.61|0.23|54.57|113.85|23.40|23.63|137.25|137.48|-301.59| +2450835|61752|2450884|56858|1798089|920|47798|56858|1798089|920|47798|1|101|7|3|313|175|1023|85|39.09|102.41|96.26|522.75|8182.10|3322.65|8704.85|0.00|6382.03|3655.85|1800.07|1800.07|5455.92|5455.92|-1522.58| +2450835|61752|2450911|56858|1798089|920|47798|56858|1798089|920|47798|1|89|15|1|13034|283|1023|97|7.36|13.83|9.12|456.87|884.64|713.92|1341.51|4.24|672.32|66.93|212.32|216.56|279.25|283.49|-501.60| +2450835|61752|2450838|56858|1798089|920|47798|56858|1798089|920|47798|1|57|6|4|16993|285|1023|15|41.86|92.09|29.46|939.45|441.90|627.90|1381.35|13.25|0.00|82.80|441.90|455.15|524.70|537.95|-186.00| +2450835|61752|2450894|56858|1798089|920|47798|56858|1798089|920|47798|1|14|5|5|7834|157|1023|88|79.70|224.75|92.14|11669.68|8108.32|7013.60|19778.00|325.95|2675.74|4746.72|5432.58|5758.53|10179.30|10505.25|-1581.02| +2450835|61752|2450914|56858|1798089|920|47798|56858|1798089|920|47798|1|14|10|5|11320|278|1023|39|76.87|142.20|48.34|3660.54|1885.26|2997.93|5545.80|68.62|169.67|1053.39|1715.59|1784.21|2768.98|2837.60|-1282.34| +2450835|61752|2450919|56858|1798089|920|47798|56858|1798089|920|47798|1|62|13|1|6226|57|1023|47|38.86|102.97|51.48|2420.03|2419.56|1826.42|4839.59|96.78|0.00|1016.14|2419.56|2516.34|3435.70|3532.48|593.14| +2450835|61752|2450874|56858|1798089|920|47798|56858|1798089|920|47798|1|40|12|4|15010|156|1023|32|14.02|15.56|13.38|69.76|428.16|448.64|497.92|0.00|0.00|24.64|428.16|428.16|452.80|452.80|-20.48| +2450835|61752|2450882|56858|1798089|920|47798|56858|1798089|920|47798|1|31|2|1|5108|259|1023|47|59.81|168.06|33.61|6319.15|1579.67|2811.07|7898.82|14.53|1216.34|0.00|363.33|377.86|363.33|377.86|-2447.74| +2450835|51488|2450842|65503|581078|7053|10228|65503|581078|7053|10228|2|108|17|3|2458|180|1024|69|99.59|174.28|155.10|1323.42|10701.90|6871.71|12025.32|428.07|0.00|3607.32|10701.90|11129.97|14309.22|14737.29|3830.19| +2450835|51488|2450903|65503|581078|7053|10228|65503|581078|7053|10228|2|5|5|5|7418|260|1024|93|96.44|152.37|146.27|567.30|13603.11|8968.92|14170.41|816.18|0.00|3542.37|13603.11|14419.29|17145.48|17961.66|4634.19| +2450835|51488|2450921|65503|581078|7053|10228|65503|581078|7053|10228|2|6|19|2|15770|140|1024|78|15.83|18.36|11.19|559.26|872.82|1234.74|1432.08|17.45|0.00|716.04|872.82|890.27|1588.86|1606.31|-361.92| +2450835|51488|2450883|65503|581078|7053|10228|65503|581078|7053|10228|2|54|4|4|3344|14|1024|32|93.50|112.20|32.53|2549.44|1040.96|2992.00|3590.40|72.86|0.00|753.92|1040.96|1113.82|1794.88|1867.74|-1951.04| +2450835|51488|2450865|65503|581078|7053|10228|65503|581078|7053|10228|2|49|3|3|12916|63|1024|97|77.26|157.61|96.14|5962.59|9325.58|7494.22|15288.17|478.40|4009.99|7338.05|5315.59|5793.99|12653.64|13132.04|-2178.63| +2450835|51488|2450881|65503|581078|7053|10228|65503|581078|7053|10228|2|92|14|4|1166|176|1024|17|97.53|248.70|196.47|887.91|3339.99|1658.01|4227.90|66.79|0.00|760.92|3339.99|3406.78|4100.91|4167.70|1681.98| +2450835|51488|2450914|65503|581078|7053|10228|65503|581078|7053|10228|2|76|13|5|3313|52|1024|76|32.33|91.81|45.90|3489.16|3488.40|2457.08|6977.56|313.95|0.00|1744.20|3488.40|3802.35|5232.60|5546.55|1031.32| +2450835|51488|2450887|65503|581078|7053|10228|65503|581078|7053|10228|2|10|19|3|16334|284|1024|30|30.15|71.15|57.63|405.60|1728.90|904.50|2134.50|25.58|449.51|234.60|1279.39|1304.97|1513.99|1539.57|374.89| +2450835|47146|2450897|57415|223727|5592|25104|57415|223727|5592|25104|1|11|5|3|14635|91|1025|44|50.66|130.19|102.85|1202.96|4525.40|2229.04|5728.36|226.27|0.00|2577.52|4525.40|4751.67|7102.92|7329.19|2296.36| +2450835|47146|2450876|57415|223727|5592|25104|57415|223727|5592|25104|1|25|3|4|13099|268|1025|10|47.72|79.21|27.72|514.90|277.20|477.20|792.10|0.00|0.00|0.00|277.20|277.20|277.20|277.20|-200.00| +2450835|47146|2450923|57415|223727|5592|25104|57415|223727|5592|25104|1|63|8|5|11494|182|1025|38|21.54|36.83|15.46|812.06|587.48|818.52|1399.54|35.24|0.00|349.60|587.48|622.72|937.08|972.32|-231.04| +2450835|47146|2450870|57415|223727|5592|25104|57415|223727|5592|25104|1|59|20|2|15200|156|1025|34|97.92|213.46|57.63|5298.22|1959.42|3329.28|7257.64|41.14|1273.62|3048.10|685.80|726.94|3733.90|3775.04|-2643.48| +2450835|47146|2450901|57415|223727|5592|25104|57415|223727|5592|25104|1|50|6|5|4808|119|1025|71|87.23|211.09|88.65|8693.24|6294.15|6193.33|14987.39|566.47|0.00|6894.10|6294.15|6860.62|13188.25|13754.72|100.82| +2450835|47146|2450911|57415|223727|5592|25104|57415|223727|5592|25104|1|7|5|5|17173|63|1025|35|67.50|89.10|86.42|93.80|3024.70|2362.50|3118.50|241.97|0.00|561.05|3024.70|3266.67|3585.75|3827.72|662.20| +2450835|47146|2450853|57415|223727|5592|25104|57415|223727|5592|25104|1|87|18|3|3122|278|1025|33|98.70|170.75|105.86|2141.37|3493.38|3257.10|5634.75|102.70|1781.62|845.13|1711.76|1814.46|2556.89|2659.59|-1545.34| +2450835|47146|2450887|57415|223727|5592|25104|57415|223727|5592|25104|1|18|7|1|2821|24|1025|61|27.84|39.81|35.43|267.18|2161.23|1698.24|2428.41|86.44|0.00|533.75|2161.23|2247.67|2694.98|2781.42|462.99| +2450835|47146|2450896|57415|223727|5592|25104|57415|223727|5592|25104|1|11|9|2|10322|268|1025|58|9.30|23.25|10.92|715.14|633.36|539.40|1348.50|12.66|0.00|80.62|633.36|646.02|713.98|726.64|93.96| +2450835|74200|2450873|76242|354452|4215|32025|49741|17245|608|17618|2|76|15|4|13948|84|1026|98|30.28|56.01|14.56|4062.10|1426.88|2967.44|5488.98|28.53|0.00|548.80|1426.88|1455.41|1975.68|2004.21|-1540.56| +2450835|74200|2450878|76242|354452|4215|32025|49741|17245|608|17618|2|86|9|4|5662|140|1026|17|54.93|91.18|10.94|1364.08|185.98|933.81|1550.06|7.43|0.00|340.85|185.98|193.41|526.83|534.26|-747.83| +2450835|74200|2450847|76242|354452|4215|32025|49741|17245|608|17618|2|35|12|5|9922|254|1026|97|23.60|37.28|5.59|3073.93|542.23|2289.20|3616.16|37.95|0.00|795.40|542.23|580.18|1337.63|1375.58|-1746.97| +2450835|74200|2450913|76242|354452|4215|32025|49741|17245|608|17618|2|97|5|4|7880|109|1026|21|77.36|83.54|42.60|859.74|894.60|1624.56|1754.34|0.00|0.00|315.63|894.60|894.60|1210.23|1210.23|-729.96| +2450835|81680|2450837|88480|950000|4767|18334|88480|950000|4767|18334|2|95|5|4|11077|217|1027|55|50.19|58.22|55.89|128.15|3073.95|2760.45|3202.10|34.42|1352.53|1280.40|1721.42|1755.84|3001.82|3036.24|-1039.03| +2450835|81680|2450924|88480|950000|4767|18334|88480|950000|4767|18334|2|49|12|1|1897|24|1027|15|66.43|116.91|101.71|228.00|1525.65|996.45|1753.65|30.51|0.00|631.20|1525.65|1556.16|2156.85|2187.36|529.20| +2450835|81680|2450856|88480|950000|4767|18334|88480|950000|4767|18334|2|21|9|3|16372|186|1027|91|10.14|15.21|1.82|1218.49|165.62|922.74|1384.11|13.24|0.00|262.08|165.62|178.86|427.70|440.94|-757.12| +2450835|81680|2450874|88480|950000|4767|18334|88480|950000|4767|18334|2|63|7|2|15094|47|1027|55|43.96|58.90|36.51|1231.45|2008.05|2417.80|3239.50|160.64|0.00|615.45|2008.05|2168.69|2623.50|2784.14|-409.75| +2450835|81680|2450885|88480|950000|4767|18334|88480|950000|4767|18334|2|54|20|5|2558|72|1027|53|28.55|66.52|26.60|2115.76|1409.80|1513.15|3525.56|70.49|0.00|1409.80|1409.80|1480.29|2819.60|2890.09|-103.35| +2450835|81680|2450843|88480|950000|4767|18334|88480|950000|4767|18334|2|36|4|3|8419|79|1027|97|44.87|104.99|26.24|7638.75|2545.28|4352.39|10184.03|203.62|0.00|4174.88|2545.28|2748.90|6720.16|6923.78|-1807.11| +2450835|81680|2450841|88480|950000|4767|18334|88480|950000|4767|18334|2|72|13|2|350|160|1027|11|51.83|154.45|109.65|492.80|1206.15|570.13|1698.95|108.55|0.00|815.43|1206.15|1314.70|2021.58|2130.13|636.02| +2450835|81680|2450920|88480|950000|4767|18334|88480|950000|4767|18334|2|107|11|3|9301|85|1027|51|69.11|181.06|74.23|5448.33|3785.73|3524.61|9234.06|189.28|0.00|4431.90|3785.73|3975.01|8217.63|8406.91|261.12| +2450835|31183|2450917|45378|1231749|1320|4491|45378|1231749|1320|4491|1|105|6|5|2110|251|1028|44|81.77|154.54|6.18|6527.84|271.92|3597.88|6799.76|5.98|152.27|1019.92|119.65|125.63|1139.57|1145.55|-3478.23| +2450835|31183|2450889|45378|1231749|1320|4491|45378|1231749|1320|4491|1|107|2|4|12685|113|1028|69|90.90|137.25|5.49|9091.44|378.81|6272.10|9470.25|14.54|15.15|3503.82|363.66|378.20|3867.48|3882.02|-5908.44| +2450835|31183|2450896|45378|1231749|1320|4491|45378|1231749|1320|4491|1|54|1|5|9265|88|1028|40|67.29|76.71|72.87|153.60|2914.80|2691.60|3068.40|58.29|0.00|1043.20|2914.80|2973.09|3958.00|4016.29|223.20| +2450835|31183|2450897|45378|1231749|1320|4491|45378|1231749|1320|4491|1|8|19|1|9038|113|1028|54|2.78|3.75|3.48|14.58|187.92|150.12|202.50|5.63|0.00|84.78|187.92|193.55|272.70|278.33|37.80| +2450835|31183|2450878|45378|1231749|1320|4491|45378|1231749|1320|4491|1|70|16|4|11941|148|1028|67|54.62|102.13|82.72|1300.47|5542.24|3659.54|6842.71|498.80|0.00|615.73|5542.24|6041.04|6157.97|6656.77|1882.70| +2450835|31699|2450870|73862|1133402|2542|18082|79719|1125162|5502|27042|1|55|12|3|11888|235|1029|50|45.57|51.03|28.57|1123.00|1428.50|2278.50|2551.50|0.00|0.00|714.00|1428.50|1428.50|2142.50|2142.50|-850.00| +2450835|31699|2450887|73862|1133402|2542|18082|79719|1125162|5502|27042|1|95|13|5|2632|178|1029|44|32.24|35.78|22.18|598.40|975.92|1418.56|1574.32|68.31|0.00|235.84|975.92|1044.23|1211.76|1280.07|-442.64| +2450835|31699|2450922|73862|1133402|2542|18082|79719|1125162|5502|27042|1|80|19|5|1699|11|1029|43|61.43|127.16|27.97|4265.17|1202.71|2641.49|5467.88|12.02|0.00|1694.63|1202.71|1214.73|2897.34|2909.36|-1438.78| +2450835|31699|2450861|73862|1133402|2542|18082|79719|1125162|5502|27042|1|85|4|5|5890|43|1029|39|38.64|61.43|13.51|1868.88|526.89|1506.96|2395.77|47.42|0.00|95.55|526.89|574.31|622.44|669.86|-980.07| +2450835|31699|2450851|73862|1133402|2542|18082|79719|1125162|5502|27042|1|89|4|5|8140|220|1029|49|38.61|40.92|22.09|922.67|1082.41|1891.89|2005.08|86.59|0.00|200.41|1082.41|1169.00|1282.82|1369.41|-809.48| +2450835|31699|2450846|73862|1133402|2542|18082|79719|1125162|5502|27042|1|71|17|4|10384|26|1029|69|45.31|68.41|49.25|1322.04|3398.25|3126.39|4720.29|135.93|0.00|1934.76|3398.25|3534.18|5333.01|5468.94|271.86| +2450835|31699|2450925|73862|1133402|2542|18082|79719|1125162|5502|27042|1|64|5|2|6676|98|1029|52|52.63|122.10|57.38|3365.44|2983.76|2736.76|6349.20|238.70|0.00|380.64|2983.76|3222.46|3364.40|3603.10|247.00| +2450835|28703|2450896|44261|1913461|2308|39908|44261|1913461|2308|39908|1|88|19|5|11962|59|1030|74|86.12|242.85|228.27|1078.92|16891.98|6372.88|17970.90|506.75|0.00|3054.72|16891.98|17398.73|19946.70|20453.45|10519.10| +2450835|28703|2450914|44261|1913461|2308|39908|44261|1913461|2308|39908|1|11|17|1|6154|114|1030|74|43.92|84.32|26.98|4243.16|1996.52|3250.08|6239.68|179.68|0.00|2558.18|1996.52|2176.20|4554.70|4734.38|-1253.56| +2450835|28703|2450900|44261|1913461|2308|39908|44261|1913461|2308|39908|1|60|1|1|10880|166|1030|26|32.78|67.52|63.46|105.56|1649.96|852.28|1755.52|32.99|0.00|509.08|1649.96|1682.95|2159.04|2192.03|797.68| +2450835|28703|2450841|44261|1913461|2308|39908|44261|1913461|2308|39908|1|53|7|5|16208|180|1030|94|45.32|96.53|29.92|6261.34|2812.48|4260.08|9073.82|112.49|0.00|453.08|2812.48|2924.97|3265.56|3378.05|-1447.60| +2450835|28703|2450851|44261|1913461|2308|39908|44261|1913461|2308|39908|1|24|7|5|6529|93|1030|82|77.36|126.09|97.08|2378.82|7960.56|6343.52|10339.38|238.81|0.00|1447.30|7960.56|8199.37|9407.86|9646.67|1617.04| +2450835|28703|2450915|44261|1913461|2308|39908|44261|1913461|2308|39908|1|68|7|3|10292|292|1030|41|4.71|11.68|8.40|134.48|344.40|193.11|478.88|3.44|0.00|200.90|344.40|347.84|545.30|548.74|151.29| +2450835|28703|2450912|44261|1913461|2308|39908|44261|1913461|2308|39908|1|91|8|2|6211|211|1030|100|87.49|147.85|29.57|11828.00|2957.00|8749.00|14785.00|118.28|0.00|5618.00|2957.00|3075.28|8575.00|8693.28|-5792.00| +2450835|28703|2450889|44261|1913461|2308|39908|44261|1913461|2308|39908|1|60|9|5|7093|97|1030|7|5.63|8.50|4.08|30.94|28.56|39.41|59.50|2.57|0.00|24.99|28.56|31.13|53.55|56.12|-10.85| +2450835|28703|2450906|44261|1913461|2308|39908|44261|1913461|2308|39908|1|13|5|2|10870|46|1030|66|53.57|143.56|91.87|3411.54|6063.42|3535.62|9474.96|485.07|0.00|2463.12|6063.42|6548.49|8526.54|9011.61|2527.80| +2450835|28703|2450914|44261|1913461|2308|39908|44261|1913461|2308|39908|1|77|7|2|7576|178|1030|29|70.91|190.74|95.37|2765.73|2765.73|2056.39|5531.46|193.60|0.00|442.25|2765.73|2959.33|3207.98|3401.58|709.34| +2450835|28703|2450878|44261|1913461|2308|39908|44261|1913461|2308|39908|1|71|4|4|15566|82|1030|23|72.88|200.42|128.26|1659.68|2949.98|1676.24|4609.66|58.99|0.00|1982.14|2949.98|3008.97|4932.12|4991.11|1273.74| +2450835|28703|2450872|44261|1913461|2308|39908|44261|1913461|2308|39908|1|23|10|1|1876|10|1030|71|45.75|135.42|14.89|8557.63|1057.19|3248.25|9614.82|10.57|0.00|3557.10|1057.19|1067.76|4614.29|4624.86|-2191.06| +2450835|71323|2450848|12863|30717|4571|32425|12863|30717|4571|32425|4|64|18|4|5611|262|1031|99|69.10|93.28|11.19|8126.91|1107.81|6840.90|9234.72|49.63|487.43|830.61|620.38|670.01|1450.99|1500.62|-6220.52| +2450835|71323|2450849|12863|30717|4571|32425|12863|30717|4571|32425|4|95|15|5|12415|97|1031|80|97.95|267.40|96.26|13691.20|7700.80|7836.00|21392.00|462.04|0.00|7914.40|7700.80|8162.84|15615.20|16077.24|-135.20| +2450835|71323|2450913|12863|30717|4571|32425|12863|30717|4571|32425|4|9|18|5|2252|37|1031|14|17.52|38.54|28.51|140.42|399.14|245.28|539.56|11.97|0.00|264.32|399.14|411.11|663.46|675.43|153.86| +2450835|71323|2450879|12863|30717|4571|32425|12863|30717|4571|32425|4|8|4|4|8536|259|1031|10|60.19|121.58|64.43|571.50|644.30|601.90|1215.80|30.92|25.77|121.50|618.53|649.45|740.03|770.95|16.63| +2450835|71323|2450883|12863|30717|4571|32425|12863|30717|4571|32425|4|53|19|5|15146|197|1031|64|26.77|45.77|20.13|1640.96|1288.32|1713.28|2929.28|12.88|0.00|175.36|1288.32|1301.20|1463.68|1476.56|-424.96| +2450835|71323|2450901|12863|30717|4571|32425|12863|30717|4571|32425|4|100|17|2|17839|150|1031|83|94.17|221.29|35.40|15428.87|2938.20|7816.11|18367.07|58.76|0.00|2754.77|2938.20|2996.96|5692.97|5751.73|-4877.91| +2450835|71323|2450899|12863|30717|4571|32425|12863|30717|4571|32425|4|57|17|3|8872|7|1031|5|44.90|110.00|20.90|445.50|104.50|224.50|550.00|0.00|0.00|187.00|104.50|104.50|291.50|291.50|-120.00| +2450835|71323|2450903|12863|30717|4571|32425|12863|30717|4571|32425|4|75|16|2|11641|97|1031|31|25.12|35.92|7.90|868.62|244.90|778.72|1113.52|17.14|0.00|55.49|244.90|262.04|300.39|317.53|-533.82| +2450835|71323|2450915|12863|30717|4571|32425|12863|30717|4571|32425|4|5|19|3|38|197|1031|62|26.77|69.60|11.13|3625.14|690.06|1659.74|4315.20|62.10|0.00|603.88|690.06|752.16|1293.94|1356.04|-969.68| +2450835|71323|2450861|12863|30717|4571|32425|12863|30717|4571|32425|4|91|11|2|4196|27|1031|87|16.70|19.87|12.12|674.25|1054.44|1452.90|1728.69|20.45|31.63|362.79|1022.81|1043.26|1385.60|1406.05|-430.09| +2450835|71323|2450866|12863|30717|4571|32425|12863|30717|4571|32425|4|34|12|1|9370|207|1031|1|2.08|5.74|3.32|2.42|3.32|2.08|5.74|0.12|1.52|2.64|1.80|1.92|4.44|4.56|-0.28| +2450835|71323|2450864|12863|30717|4571|32425|12863|30717|4571|32425|4|36|7|5|14377|278|1031|75|49.87|69.31|11.78|4314.75|883.50|3740.25|5198.25|26.50|0.00|1767.00|883.50|910.00|2650.50|2677.00|-2856.75| +2450835|71323|2450876|12863|30717|4571|32425|12863|30717|4571|32425|4|38|1|3|12961|156|1031|46|48.33|129.04|73.55|2552.54|3383.30|2223.18|5935.84|33.83|0.00|2848.78|3383.30|3417.13|6232.08|6265.91|1160.12| +2450835|46703|2450853|97309|213057|6758|17659|97309|213057|6758|17659|1|84|17|3|13382|205|1032|43|14.98|39.24|27.07|523.31|1164.01|644.14|1687.32|34.92|0.00|185.33|1164.01|1198.93|1349.34|1384.26|519.87| +2450835|46703|2450877|97309|213057|6758|17659|97309|213057|6758|17659|1|14|20|2|16972|154|1032|12|34.85|77.71|73.04|56.04|876.48|418.20|932.52|78.88|0.00|261.00|876.48|955.36|1137.48|1216.36|458.28| +2450835|46703|2450917|97309|213057|6758|17659|97309|213057|6758|17659|1|81|11|3|2416|154|1032|80|85.70|149.11|35.78|9066.40|2862.40|6856.00|11928.80|114.49|0.00|715.20|2862.40|2976.89|3577.60|3692.09|-3993.60| +2450835|46703|2450905|97309|213057|6758|17659|97309|213057|6758|17659|1|25|9|4|4642|179|1032|95|24.83|45.43|4.08|3928.25|387.60|2358.85|4315.85|7.75|0.00|2157.45|387.60|395.35|2545.05|2552.80|-1971.25| +2450835|46703|2450925|97309|213057|6758|17659|97309|213057|6758|17659|1|91|19|4|9049|29|1032|90|91.52|95.18|43.78|4626.00|3940.20|8236.80|8566.20|157.60|0.00|4283.10|3940.20|4097.80|8223.30|8380.90|-4296.60| +2450835|46703|2450916|97309|213057|6758|17659|97309|213057|6758|17659|1|69|3|1|14668|247|1032|33|9.46|10.31|3.71|217.80|122.43|312.18|340.23|2.13|15.91|88.44|106.52|108.65|194.96|197.09|-205.66| +2450835|46703|2450872|97309|213057|6758|17659|97309|213057|6758|17659|1|105|9|1|6802|89|1032|18|60.05|164.53|110.23|977.40|1984.14|1080.90|2961.54|20.83|1289.69|88.74|694.45|715.28|783.19|804.02|-386.45| +2450835|25308|2450904|31318|1103600|3082|34517|31318|1103600|3082|34517|1|12|14|1|505|211|1033|42|5.68|5.90|3.89|84.42|163.38|238.56|247.80|4.90|0.00|106.26|163.38|168.28|269.64|274.54|-75.18| +2450835|25308|2450888|31318|1103600|3082|34517|31318|1103600|3082|34517|1|59|15|5|17251|137|1033|34|61.95|136.90|97.19|1350.14|3304.46|2106.30|4654.60|0.00|1487.00|698.02|1817.46|1817.46|2515.48|2515.48|-288.84| +2450835|25308|2450838|31318|1103600|3082|34517|31318|1103600|3082|34517|1|33|13|2|8212|145|1033|18|42.17|58.61|52.16|116.10|938.88|759.06|1054.98|65.72|0.00|10.44|938.88|1004.60|949.32|1015.04|179.82| +2450835|25308|2450892|31318|1103600|3082|34517|31318|1103600|3082|34517|1|97|16|1|12046|136|1033|7|20.89|28.41|4.26|169.05|29.82|146.23|198.87|0.29|0.00|13.86|29.82|30.11|43.68|43.97|-116.41| +2450835|25308|2450853|31318|1103600|3082|34517|31318|1103600|3082|34517|1|22|7|4|1873|220|1033|74|96.85|111.37|3.34|7994.22|247.16|7166.90|8241.38|12.35|0.00|164.28|247.16|259.51|411.44|423.79|-6919.74| +2450835|25308|2450915|31318|1103600|3082|34517|31318|1103600|3082|34517|1|42|17|1|241|51|1033|18|92.93|174.70|141.50|597.60|2547.00|1672.74|3144.60|203.76|0.00|1352.16|2547.00|2750.76|3899.16|4102.92|874.26| +2450835|25308|2450916|31318|1103600|3082|34517|31318|1103600|3082|34517|1|88|3|4|16006|106|1033|32|60.82|147.79|7.38|4493.12|236.16|1946.24|4729.28|11.80|0.00|189.12|236.16|247.96|425.28|437.08|-1710.08| +2450835|25308|2450893|31318|1103600|3082|34517|31318|1103600|3082|34517|1|45|3|2|5161|98|1033|30|83.99|125.98|118.42|226.80|3552.60|2519.70|3779.40|36.94|3090.76|1549.50|461.84|498.78|2011.34|2048.28|-2057.86| +2450835|25308|2450879|31318|1103600|3082|34517|31318|1103600|3082|34517|1|14|14|3|9590|76|1033|10|99.75|175.56|12.28|1632.80|122.80|997.50|1755.60|1.32|89.64|561.70|33.16|34.48|594.86|596.18|-964.34| +2450835|25308|2450892|31318|1103600|3082|34517|31318|1103600|3082|34517|1|50|6|3|3194|174|1033|20|64.17|88.55|12.39|1523.20|247.80|1283.40|1771.00|0.00|0.00|513.40|247.80|247.80|761.20|761.20|-1035.60| +2450835|25308|2450925|31318|1103600|3082|34517|31318|1103600|3082|34517|1|71|3|5|1832|204|1033|91|48.79|113.68|63.66|4551.82|5793.06|4439.89|10344.88|231.72|0.00|2896.53|5793.06|6024.78|8689.59|8921.31|1353.17| +2450835|25308|2450922|31318|1103600|3082|34517|31318|1103600|3082|34517|1|90|20|3|1570|166|1033|21|62.72|144.25|111.07|696.78|2332.47|1317.12|3029.25|34.28|1842.65|666.33|489.82|524.10|1156.15|1190.43|-827.30| +2450835|37203|2450919|237|916222|5807|33345|68643|1335715|4053|22038|4|46|11|4|17396|273|1034|46|68.56|155.63|121.39|1575.04|5583.94|3153.76|7158.98|446.71|0.00|1932.92|5583.94|6030.65|7516.86|7963.57|2430.18| +2450835|37203|2450896|237|916222|5807|33345|68643|1335715|4053|22038|4|37|2|3|11564|145|1034|17|68.10|185.91|33.46|2591.65|568.82|1157.70|3160.47|5.68|0.00|31.45|568.82|574.50|600.27|605.95|-588.88| +2450835|37203|2450874|237|916222|5807|33345|68643|1335715|4053|22038|4|63|18|5|16507|64|1034|80|2.25|3.78|3.40|30.40|272.00|180.00|302.40|8.16|0.00|151.20|272.00|280.16|423.20|431.36|92.00| +2450835|37203|2450848|237|916222|5807|33345|68643|1335715|4053|22038|4|16|11|1|8000|134|1034|32|46.34|48.19|8.19|1280.00|262.08|1482.88|1542.08|2.62|0.00|215.68|262.08|264.70|477.76|480.38|-1220.80| +2450835|37203|2450872|237|916222|5807|33345|68643|1335715|4053|22038|4|105|3|5|1312|254|1034|19|6.60|14.98|2.99|227.81|56.81|125.40|284.62|2.84|0.00|136.61|56.81|59.65|193.42|196.26|-68.59| +2450835|37203|2450894|237|916222|5807|33345|68643|1335715|4053|22038|4|24|14|5|3500|254|1034|7|45.61|82.55|4.95|543.20|34.65|319.27|577.85|3.11|0.00|34.65|34.65|37.76|69.30|72.41|-284.62| +2450835|37203|2450889|237|916222|5807|33345|68643|1335715|4053|22038|4|43|12|1|11566|131|1034|24|51.28|150.76|76.88|1773.12|1845.12|1230.72|3618.24|166.06|0.00|397.92|1845.12|2011.18|2243.04|2409.10|614.40| +2450835|37203|2450924|237|916222|5807|33345|68643|1335715|4053|22038|4|1|1|3|5395|168|1034|4|60.23|95.16|5.70|357.84|22.80|240.92|380.64|1.14|0.00|186.48|22.80|23.94|209.28|210.42|-218.12| +2450835|37203|2450906|237|916222|5807|33345|68643|1335715|4053|22038|4|40|20|4|11678|153|1034|50|84.14|111.06|48.86|3110.00|2443.00|4207.00|5553.00|21.98|2076.55|55.50|366.45|388.43|421.95|443.93|-3840.55| +2450835|40413|2450847|61384|1868737|1243|18149|61384|1868737|1243|18149|4|105|14|1|5977|95|1035|100|83.98|147.80|121.19|2661.00|12119.00|8398.00|14780.00|278.73|6544.26|147.00|5574.74|5853.47|5721.74|6000.47|-2823.26| +2450835|40413|2450890|61384|1868737|1243|18149|61384|1868737|1243|18149|4|90|2|2|6907|149|1035|71|69.36|112.36|59.55|3749.51|4228.05|4924.56|7977.56|380.52|0.00|3589.76|4228.05|4608.57|7817.81|8198.33|-696.51| +2450835|40413|2450855|61384|1868737|1243|18149|61384|1868737|1243|18149|4|44|15|4|7550|135|1035|55|95.90|133.30|106.64|1466.30|5865.20|5274.50|7331.50|117.30|0.00|1979.45|5865.20|5982.50|7844.65|7961.95|590.70| +2450835|40413|2450898|61384|1868737|1243|18149|61384|1868737|1243|18149|4|100|12|1|4651|94|1035|91|96.75|174.15|22.63|13788.32|2059.33|8804.25|15847.65|61.77|0.00|158.34|2059.33|2121.10|2217.67|2279.44|-6744.92| +2450835|40413|2450838|61384|1868737|1243|18149|61384|1868737|1243|18149|4|69|17|2|15907|40|1035|7|58.12|108.10|7.56|703.78|52.92|406.84|756.70|2.11|0.00|227.01|52.92|55.03|279.93|282.04|-353.92| +2450835|40413|2450887|61384|1868737|1243|18149|61384|1868737|1243|18149|4|32|19|2|8528|5|1035|81|56.15|57.27|35.50|1763.37|2875.50|4548.15|4638.87|172.53|0.00|1066.77|2875.50|3048.03|3942.27|4114.80|-1672.65| +2450835|40413|2450910|61384|1868737|1243|18149|61384|1868737|1243|18149|4|80|4|3|673|211|1035|84|47.44|54.55|47.45|596.40|3985.80|3984.96|4582.20|39.85|0.00|961.80|3985.80|4025.65|4947.60|4987.45|0.84| +2450835|40413|2450856|61384|1868737|1243|18149|61384|1868737|1243|18149|4|48|1|5|17626|120|1035|21|26.63|35.95|4.31|664.44|90.51|559.23|754.95|8.14|0.00|324.45|90.51|98.65|414.96|423.10|-468.72| +2450835|40413|2450846|61384|1868737|1243|18149|61384|1868737|1243|18149|4|43|6|5|16807|133|1035|76|79.29|185.53|77.92|8178.36|5921.92|6026.04|14100.28|296.09|0.00|3242.92|5921.92|6218.01|9164.84|9460.93|-104.12| +2450835|40413|2450919|61384|1868737|1243|18149|61384|1868737|1243|18149|4|42|9|1|3619|57|1035|48|81.68|155.19|45.00|5289.12|2160.00|3920.64|7449.12|151.20|0.00|74.40|2160.00|2311.20|2234.40|2385.60|-1760.64| +2450835|18256|2450878|3881|1748891|5901|18111|3881|1748891|5901|18111|4|86|4|1|2372|288|1036|71|36.24|60.52|56.28|301.04|3995.88|2573.04|4296.92|239.75|0.00|1074.23|3995.88|4235.63|5070.11|5309.86|1422.84| +2450835|18256|2450894|3881|1748891|5901|18111|3881|1748891|5901|18111|4|8|5|3|8354|156|1036|100|9.35|12.90|12.38|52.00|1238.00|935.00|1290.00|99.04|0.00|528.00|1238.00|1337.04|1766.00|1865.04|303.00| +2450835|18256|2450921|3881|1748891|5901|18111|3881|1748891|5901|18111|4|45|19|5|16532|269|1036|34|84.48|188.39|145.06|1473.22|4932.04|2872.32|6405.26|345.24|0.00|832.66|4932.04|5277.28|5764.70|6109.94|2059.72| +2450835|18256|2450850|3881|1748891|5901|18111|3881|1748891|5901|18111|4|98|4|5|5318|85|1036|59|93.26|102.58|81.03|1271.45|4780.77|5502.34|6052.22|382.46|0.00|1451.99|4780.77|5163.23|6232.76|6615.22|-721.57| +2450835|18256|2450915|3881|1748891|5901|18111|3881|1748891|5901|18111|4|32|20|2|3760|71|1036|97|40.30|79.39|63.51|1540.36|6160.47|3909.10|7700.83|246.41|0.00|3388.21|6160.47|6406.88|9548.68|9795.09|2251.37| +2450835|18256|2450917|3881|1748891|5901|18111|3881|1748891|5901|18111|4|20|2|4|12278|30|1036|26|18.68|21.66|15.37|163.54|399.62|485.68|563.16|0.00|0.00|118.04|399.62|399.62|517.66|517.66|-86.06| +2450835|18256|2450883|3881|1748891|5901|18111|3881|1748891|5901|18111|4|50|14|1|13657|160|1036|19|35.19|70.38|4.92|1243.74|93.48|668.61|1337.22|1.86|0.00|427.88|93.48|95.34|521.36|523.22|-575.13| +2450835|18256|2450860|3881|1748891|5901|18111|3881|1748891|5901|18111|4|9|9|3|3772|191|1036|92|3.64|4.58|2.97|148.12|273.24|334.88|421.36|2.73|0.00|150.88|273.24|275.97|424.12|426.85|-61.64| +2450835|18256|2450915|3881|1748891|5901|18111|3881|1748891|5901|18111|4|55|16|1|11668|85|1036|11|40.27|89.80|81.71|88.99|898.81|442.97|987.80|0.00|0.00|237.05|898.81|898.81|1135.86|1135.86|455.84| +2450835|18256|2450846|3881|1748891|5901|18111|3881|1748891|5901|18111|4|26|8|3|668|267|1036|1|44.06|46.26|24.51|21.75|24.51|44.06|46.26|0.73|0.00|0.46|24.51|25.24|24.97|25.70|-19.55| +2450835|18256|2450894|3881|1748891|5901|18111|3881|1748891|5901|18111|4|50|13|4|14887|165|1036|1|2.19|5.45|0.00|5.45|0.00|2.19|5.45|0.00|0.00|0.32|0.00|0.00|0.32|0.32|-2.19| +2450835|62788|2450899|7762|572326|890|17054|7762|572326|890|17054|1|75|5|4|15286|92|1037|49|99.11|266.60|130.63|6662.53|6400.87|4856.39|13063.40|320.04|0.00|2743.02|6400.87|6720.91|9143.89|9463.93|1544.48| +2450835|62788|2450852|7762|572326|890|17054|7762|572326|890|17054|1|77|11|4|9400|287|1037|90|45.00|84.15|0.00|7573.50|0.00|4050.00|7573.50|0.00|0.00|1514.70|0.00|0.00|1514.70|1514.70|-4050.00| +2450835|62788|2450886|7762|572326|890|17054|7762|572326|890|17054|1|60|18|4|10999|41|1037|10|59.04|157.04|28.26|1287.80|282.60|590.40|1570.40|0.00|0.00|455.40|282.60|282.60|738.00|738.00|-307.80| +2450835|62788|2450885|7762|572326|890|17054|7762|572326|890|17054|1|25|3|4|10562|173|1037|95|15.94|37.77|17.37|1938.00|1650.15|1514.30|3588.15|66.00|0.00|501.60|1650.15|1716.15|2151.75|2217.75|135.85| +2450835|62788|2450850|7762|572326|890|17054|7762|572326|890|17054|1|34|2|3|11863|295|1037|47|78.24|169.78|146.01|1117.19|6862.47|3677.28|7979.66|115.29|4940.97|3590.80|1921.50|2036.79|5512.30|5627.59|-1755.78| +2450835|62788|2450845|7762|572326|890|17054|7762|572326|890|17054|1|108|6|5|6844|243|1037|52|32.67|93.10|78.20|774.80|4066.40|1698.84|4841.20|284.64|0.00|1403.48|4066.40|4351.04|5469.88|5754.52|2367.56| +2450835|62788|2450911|7762|572326|890|17054|7762|572326|890|17054|1|87|14|4|7699|262|1037|48|42.57|108.55|73.81|1667.52|3542.88|2043.36|5210.40|12.04|2940.59|468.48|602.29|614.33|1070.77|1082.81|-1441.07| +2450835|62788|2450905|7762|572326|890|17054|7762|572326|890|17054|1|26|17|3|16682|147|1037|35|40.21|78.40|61.93|576.45|2167.55|1407.35|2744.00|21.67|0.00|356.65|2167.55|2189.22|2524.20|2545.87|760.20| +2450835|62788|2450872|7762|572326|890|17054|7762|572326|890|17054|1|50|20|3|758|121|1037|35|73.90|90.89|56.35|1208.90|1972.25|2586.50|3181.15|157.78|0.00|1145.20|1972.25|2130.03|3117.45|3275.23|-614.25| +2450835|44272|2450837|8354|1262564|1127|36137|8354|1262564|1127|36137|2|62|12|2|16372|147|1038|65|8.09|23.86|19.08|310.70|1240.20|525.85|1550.90|51.59|595.29|480.35|644.91|696.50|1125.26|1176.85|119.06| +2450835|44272|2450913|8354|1262564|1127|36137|8354|1262564|1127|36137|2|29|6|5|15094|38|1038|53|7.62|11.96|3.22|463.22|170.66|403.86|633.88|7.06|92.15|63.07|78.51|85.57|141.58|148.64|-325.35| +2450835|44272|2450875|8354|1262564|1127|36137|8354|1262564|1127|36137|2|90|12|1|2558|161|1038|92|17.12|42.97|36.95|553.84|3399.40|1575.04|3953.24|33.99|0.00|1620.12|3399.40|3433.39|5019.52|5053.51|1824.36| +2450835|44272|2450856|8354|1262564|1127|36137|8354|1262564|1127|36137|2|52|9|2|8419|50|1038|6|88.14|156.00|12.48|861.12|74.88|528.84|936.00|2.99|0.00|196.56|74.88|77.87|271.44|274.43|-453.96| +2450835|44272|2450852|8354|1262564|1127|36137|8354|1262564|1127|36137|2|55|11|4|350|232|1038|52|9.38|20.63|2.06|965.64|107.12|487.76|1072.76|0.00|0.00|182.00|107.12|107.12|289.12|289.12|-380.64| +2450835|44272|2450868|8354|1262564|1127|36137|8354|1262564|1127|36137|2|10|8|3|9301|123|1038|28|61.10|125.25|83.91|1157.52|2349.48|1710.80|3507.00|164.46|0.00|701.40|2349.48|2513.94|3050.88|3215.34|638.68| +2450835|44272|2450874|8354|1262564|1127|36137|8354|1262564|1127|36137|2|57|2|5|16441|138|1038|61|7.10|16.11|6.28|599.63|383.08|433.10|982.71|15.32|0.00|87.84|383.08|398.40|470.92|486.24|-50.02| +2450835|44272|2450870|8354|1262564|1127|36137|8354|1262564|1127|36137|2|107|6|5|11737|38|1038|51|95.36|238.40|157.34|4134.06|8024.34|4863.36|12158.40|561.70|0.00|5714.04|8024.34|8586.04|13738.38|14300.08|3160.98| +2450835|44272|2450899|8354|1262564|1127|36137|8354|1262564|1127|36137|2|5|15|5|14026|90|1038|4|27.26|73.32|13.93|237.56|55.72|109.04|293.28|5.01|0.00|102.64|55.72|60.73|158.36|163.37|-53.32| +2450835|44272|2450884|8354|1262564|1127|36137|8354|1262564|1127|36137|2|23|1|2|15586|32|1038|11|82.00|159.08|84.31|822.47|927.41|902.00|1749.88|9.27|0.00|52.47|927.41|936.68|979.88|989.15|25.41| +2450835|44272|2450858|8354|1262564|1127|36137|8354|1262564|1127|36137|2|21|15|4|9343|55|1038|11|55.64|86.24|73.30|142.34|806.30|612.04|948.64|0.16|798.23|28.38|8.07|8.23|36.45|36.61|-603.97| +2450835|44272|2450867|8354|1262564|1127|36137|8354|1262564|1127|36137|2|95|1|2|7804|34|1038|96|34.42|99.47|42.77|5443.20|4105.92|3304.32|9549.12|0.00|1478.13|2959.68|2627.79|2627.79|5587.47|5587.47|-676.53| +2450835|44272|2450911|8354|1262564|1127|36137|8354|1262564|1127|36137|2|35|7|4|122|113|1038|96|40.37|74.68|23.89|4875.84|2293.44|3875.52|7169.28|137.60|0.00|645.12|2293.44|2431.04|2938.56|3076.16|-1582.08| +2450835|44272|2450877|8354|1262564|1127|36137|8354|1262564|1127|36137|2|48|9|5|11462|124|1038|30|86.58|199.13|57.74|4241.70|1732.20|2597.40|5973.90|51.96|0.00|537.60|1732.20|1784.16|2269.80|2321.76|-865.20| +2450835|34148|2450869|17048|1671134|1573|15259|17048|1671134|1573|15259|1|85|10|2|6634|174|1039|48|78.90|198.03|104.95|4467.84|5037.60|3787.20|9505.44|251.88|0.00|1710.72|5037.60|5289.48|6748.32|7000.20|1250.40| +2450835|34148|2450848|17048|1671134|1573|15259|17048|1671134|1573|15259|1|59|10|3|6331|281|1039|90|43.82|62.66|34.46|2538.00|3101.40|3943.80|5639.40|62.02|0.00|2706.30|3101.40|3163.42|5807.70|5869.72|-842.40| +2450835|34148|2450895|17048|1671134|1573|15259|17048|1671134|1573|15259|1|81|13|5|3562|9|1039|53|78.39|195.97|180.29|831.04|9555.37|4154.67|10386.41|191.10|0.00|3427.51|9555.37|9746.47|12982.88|13173.98|5400.70| +2450835|34148|2450880|17048|1671134|1573|15259|17048|1671134|1573|15259|1|22|8|4|3448|81|1039|29|7.42|20.62|14.22|185.60|412.38|215.18|597.98|8.24|0.00|17.69|412.38|420.62|430.07|438.31|197.20| +2450835|34148|2450870|17048|1671134|1573|15259|17048|1671134|1573|15259|1|52|3|1|8266|151|1039|79|67.63|118.35|48.52|5516.57|3833.08|5342.77|9349.65|114.99|0.00|0.00|3833.08|3948.07|3833.08|3948.07|-1509.69| +2450835|34148|2450845|17048|1671134|1573|15259|17048|1671134|1573|15259|1|21|19|5|12628|35|1039|60|84.69|188.85|66.09|7365.60|3965.40|5081.40|11331.00|0.00|0.00|1473.00|3965.40|3965.40|5438.40|5438.40|-1116.00| +2450835|34148|2450848|17048|1671134|1573|15259|17048|1671134|1573|15259|1|73|19|2|1759|103|1039|53|88.12|217.65|117.53|5306.36|6229.09|4670.36|11535.45|498.32|0.00|3806.46|6229.09|6727.41|10035.55|10533.87|1558.73| +2450835|34148|2450904|17048|1671134|1573|15259|17048|1671134|1573|15259|1|24|4|1|3206|48|1039|33|34.43|35.46|28.01|245.85|924.33|1136.19|1170.18|46.21|0.00|479.49|924.33|970.54|1403.82|1450.03|-211.86| +2450835|34148|2450839|17048|1671134|1573|15259|17048|1671134|1573|15259|1|53|9|1|4160|162|1039|41|55.79|103.76|43.57|2467.79|1786.37|2287.39|4254.16|53.59|0.00|1786.37|1786.37|1839.96|3572.74|3626.33|-501.02| +2450835|34148|2450840|17048|1671134|1573|15259|17048|1671134|1573|15259|1|76|5|5|10996|45|1039|58|23.95|48.61|38.88|564.34|2255.04|1389.10|2819.38|18.04|1353.02|563.76|902.02|920.06|1465.78|1483.82|-487.08| +2450835|34148|2450874|17048|1671134|1573|15259|17048|1671134|1573|15259|1|51|7|2|9536|243|1039|18|62.54|96.93|78.51|331.56|1413.18|1125.72|1744.74|70.65|0.00|418.68|1413.18|1483.83|1831.86|1902.51|287.46| +2450835|34148|2450890|17048|1671134|1573|15259|17048|1671134|1573|15259|1|58|3|4|10328|71|1039|71|48.96|106.24|19.12|6185.52|1357.52|3476.16|7543.04|81.45|0.00|2111.54|1357.52|1438.97|3469.06|3550.51|-2118.64| +2450835|34148|2450905|17048|1671134|1573|15259|17048|1671134|1573|15259|1|99|1|5|15808|138|1039|48|76.68|189.39|62.49|6091.20|2999.52|3680.64|9090.72|179.97|0.00|3908.64|2999.52|3179.49|6908.16|7088.13|-681.12| +2450835|25565|2450861|28142|1858653|2764|36475|28142|1858653|2764|36475|2|63|18|3|17269|288|1040|6|7.82|22.91|17.41|33.00|104.46|46.92|137.46|0.00|0.00|67.32|104.46|104.46|171.78|171.78|57.54| +2450835|25565|2450871|28142|1858653|2764|36475|28142|1858653|2764|36475|2|86|7|2|15091|232|1040|31|7.89|11.36|3.74|236.22|115.94|244.59|352.16|3.47|0.00|165.23|115.94|119.41|281.17|284.64|-128.65| +2450835|25565|2450872|28142|1858653|2764|36475|28142|1858653|2764|36475|2|47|4|1|11330|100|1040|49|70.03|147.76|138.89|434.63|6805.61|3431.47|7240.24|61.93|612.50|1302.91|6193.11|6255.04|7496.02|7557.95|2761.64| +2450835|25565|2450871|28142|1858653|2764|36475|28142|1858653|2764|36475|2|75|16|2|14074|17|1040|27|7.34|14.60|12.26|63.18|331.02|198.18|394.20|6.62|0.00|55.08|331.02|337.64|386.10|392.72|132.84| +2450835|25565|2450892|28142|1858653|2764|36475|28142|1858653|2764|36475|2|46|19|4|13558|221|1040|89|35.27|47.61|44.27|297.26|3940.03|3139.03|4237.29|39.40|0.00|931.83|3940.03|3979.43|4871.86|4911.26|801.00| +|||28142|1858653||36475||1858653||36475|2|65|||7106|90|1040||78.09||144.30|1422.63|||6184.53|77.14|||||3887.54|3964.68|-1719.82| +2450835|25565|2450850|28142|1858653|2764|36475|28142|1858653|2764|36475|2|105|15|3|10748|157|1040|79|28.38|77.47|56.55|1652.68|4467.45|2242.02|6120.13|402.07|0.00|1713.51|4467.45|4869.52|6180.96|6583.03|2225.43| +2450835|25565|2450899|28142|1858653|2764|36475|28142|1858653|2764|36475|2|108|13|2|11416|42|1040|52|83.95|240.93|187.92|2756.52|9771.84|4365.40|12528.36|879.46|0.00|2505.36|9771.84|10651.30|12277.20|13156.66|5406.44| +2450835|25565|2450856|28142|1858653|2764|36475|28142|1858653|2764|36475|2|97|1|3|16592|155|1040|70|88.74|183.69|47.75|9515.80|3342.50|6211.80|12858.30|33.42|0.00|2956.80|3342.50|3375.92|6299.30|6332.72|-2869.30| +2450835|25565|2450853|28142|1858653|2764|36475|28142|1858653|2764|36475|2|73|15|2|2678|253|1040|36|9.02|9.56|4.39|186.12|158.04|324.72|344.16|0.00|0.00|44.64|158.04|158.04|202.68|202.68|-166.68| +2450835|25565|2450903|28142|1858653|2764|36475|28142|1858653|2764|36475|2|101|16|3|3499|293|1040|73|70.69|212.07|195.10|1238.81|14242.30|5160.37|15481.11|1281.80|0.00|7430.67|14242.30|15524.10|21672.97|22954.77|9081.93| +2450835|25565|2450881|28142|1858653|2764|36475|28142|1858653|2764|36475|2|16|2|1|7628|52|1040|58|56.26|115.89|28.97|5041.36|1680.26|3263.08|6721.62|67.21|0.00|2016.08|1680.26|1747.47|3696.34|3763.55|-1582.82| +2450835|25565|2450910|28142|1858653|2764|36475|28142|1858653|2764|36475|2|97|13|2|3337|216|1040|79|50.07|119.66|61.02|4632.56|4820.58|3955.53|9453.14|385.64|0.00|1133.65|4820.58|5206.22|5954.23|6339.87|865.05| +2450835|25565|2450866|28142|1858653|2764|36475|28142|1858653|2764|36475|2|96|1|2|14432|254|1040|78|65.74|186.04|33.48|11899.68|2611.44|5127.72|14511.12|0.00|0.00|6239.22|2611.44|2611.44|8850.66|8850.66|-2516.28| +2450835|69802|2450913|10228|1064878|1481|13731|95300|1238231|5681|29931|2|105|15|1|2725|85|1041|97|49.38|86.90|33.02|5226.36|3202.94|4789.86|8429.30|0.00|0.00|3118.55|3202.94|3202.94|6321.49|6321.49|-1586.92| +2450835|69802|2450844|10228|1064878|1481|13731|95300|1238231|5681|29931|2|65|9|2|4022|60|1041|96|86.47|232.60|155.84|7368.96|14960.64|8301.12|22329.60|149.60|0.00|222.72|14960.64|15110.24|15183.36|15332.96|6659.52| +2450835|69802|2450898|10228|1064878|1481|13731|95300|1238231|5681|29931|2|66|9|4|787|93|1041|5|14.88|43.15|31.93|56.10|159.65|74.40|215.75|14.36|0.00|75.50|159.65|174.01|235.15|249.51|85.25| +2450835|69802|2450922|10228|1064878|1481|13731|95300|1238231|5681|29931|2|19|20|5|9058|252|1041|23|59.27|145.80|10.20|3118.80|234.60|1363.21|3353.40|18.76|0.00|301.76|234.60|253.36|536.36|555.12|-1128.61| +2450835|69802|2450922|10228|1064878|1481|13731|95300|1238231|5681|29931|2|88|17|4|2800|166|1041|75|47.14|140.00|9.80|9765.00|735.00|3535.50|10500.00|28.22|382.20|1995.00|352.80|381.02|2347.80|2376.02|-3182.70| +2450835|69802|2450896|10228|1064878|1481|13731|95300|1238231|5681|29931|2|61|7|3|13142|93|1041|58|76.44|116.95|105.25|678.60|6104.50|4433.52|6783.10|305.22|0.00|2102.50|6104.50|6409.72|8207.00|8512.22|1670.98| +2450835|69802|2450925|10228|1064878|1481|13731|95300|1238231|5681|29931|2|36|16|1|40|26|1041|50|93.18|122.99|43.04|3997.50|2152.00|4659.00|6149.50|172.16|0.00|1598.50|2152.00|2324.16|3750.50|3922.66|-2507.00| +2450835|69802|2450880|10228|1064878|1481|13731|95300|1238231|5681|29931|2|97|7|4|2206|245|1041|19|16.01|19.69|5.71|265.62|108.49|304.19|374.11|6.50|0.00|3.61|108.49|114.99|112.10|118.60|-195.70| +2450835|69802|2450925|10228|1064878|1481|13731|95300|1238231|5681|29931|2|4|7|1|1942|294|1041|94|58.30|142.25|98.15|4145.40|9226.10|5480.20|13371.50|92.26|0.00|1871.54|9226.10|9318.36|11097.64|11189.90|3745.90| +2450835|69802|2450918|10228|1064878|1481|13731|95300|1238231|5681|29931|2|55|10|3|10498|155|1041|42|18.66|34.52|9.32|1058.40|391.44|783.72|1449.84|7.82|0.00|188.16|391.44|399.26|579.60|587.42|-392.28| +2450835|69802|2450888|10228|1064878|1481|13731|95300|1238231|5681|29931|2|22|14|1|8035|85|1041|67|19.47|35.24|21.14|944.70|1416.38|1304.49|2361.08|70.81|0.00|401.33|1416.38|1487.19|1817.71|1888.52|111.89| +2450835|69802|2450901|10228|1064878|1481|13731|95300|1238231|5681|29931|2|50|5|2|10630|291|1041|97|11.47|18.69|0.74|1741.15|71.78|1112.59|1812.93|6.46|0.00|797.34|71.78|78.24|869.12|875.58|-1040.81| +2450835|69802|2450886|10228|1064878|1481|13731|95300|1238231|5681|29931|2|35|4|1|10022|62|1041|65|97.70|261.83|39.27|14466.40|2552.55|6350.50|17018.95|178.67|0.00|510.25|2552.55|2731.22|3062.80|3241.47|-3797.95| +2450835|48019|2450923|80548|1039723|6451|33226|80548|1039723|6451|33226|4|88|9|4|17404|102|1042|41|57.50|78.77|15.75|2583.82|645.75|2357.50|3229.57|32.28|0.00|1291.50|645.75|678.03|1937.25|1969.53|-1711.75| +2450835|48019|2450840|80548|1039723|6451|33226|80548|1039723|6451|33226|4|33|7|3|5356|10|1042|94|13.66|16.39|5.73|1002.04|538.62|1284.04|1540.66|5.38|0.00|523.58|538.62|544.00|1062.20|1067.58|-745.42| +2450835|48019|2450861|80548|1039723|6451|33226|80548|1039723|6451|33226|4|89|10|1|3637|173|1042|42|21.38|54.09|31.91|931.56|1340.22|897.96|2271.78|40.20|0.00|681.24|1340.22|1380.42|2021.46|2061.66|442.26| +2450835|48019|2450915|80548|1039723|6451|33226|80548|1039723|6451|33226|4|80|9|3|12064|233|1042|21|30.88|57.12|15.99|863.73|335.79|648.48|1199.52|6.71|0.00|59.85|335.79|342.50|395.64|402.35|-312.69| +2450835|48019|2450846|80548|1039723|6451|33226|80548|1039723|6451|33226|4|8|3|5|3223|169|1042|12|23.78|59.45|31.50|335.40|378.00|285.36|713.40|18.90|0.00|57.00|378.00|396.90|435.00|453.90|92.64| +2450835|48019|2450841|80548|1039723|6451|33226|80548|1039723|6451|33226|4|100|9|5|14480|144|1042|42|19.37|43.96|34.72|388.08|1458.24|813.54|1846.32|1.45|1429.07|350.70|29.17|30.62|379.87|381.32|-784.37| +2450835|69086|2450898|76564|1326909|3301|30093|76564|1326909|3301|30093|2|65|5|2|7765|193|1043|60|30.77|32.00|3.20|1728.00|192.00|1846.20|1920.00|17.28|0.00|345.60|192.00|209.28|537.60|554.88|-1654.20| +2450835|69086|2450847|76564|1326909|3301|30093|76564|1326909|3301|30093|2|26|11|4|3944|62|1043|57|91.20|160.51|149.27|640.68|8508.39|5198.40|9149.07|595.58|0.00|731.88|8508.39|9103.97|9240.27|9835.85|3309.99| +2450835|69086|2450858|76564|1326909|3301|30093|76564|1326909|3301|30093|2|58|13|3|3814|241|1043|39|39.08|41.42|8.28|1292.46|322.92|1524.12|1615.38|8.52|109.79|145.08|213.13|221.65|358.21|366.73|-1310.99| +2450835|69086|2450856|76564|1326909|3301|30093|76564|1326909|3301|30093|2|3|14|3|6442|110|1043|90|4.77|9.96|1.19|789.30|107.10|429.30|896.40|2.14|0.00|259.20|107.10|109.24|366.30|368.44|-322.20| +2450835|69086|2450859|76564|1326909|3301|30093|76564|1326909|3301|30093|2|58|16|3|11308|65|1043|22|2.86|4.54|3.40|25.08|74.80|62.92|99.88|2.24|0.00|24.86|74.80|77.04|99.66|101.90|11.88| +2450835|69086|2450916|76564|1326909|3301|30093|76564|1326909|3301|30093|2|93|13|1|4039|262|1043|14|6.53|7.64|7.64|0.00|106.96|91.42|106.96|8.55|0.00|8.54|106.96|115.51|115.50|124.05|15.54| +2450835|69086|2450840|76564|1326909|3301|30093|76564|1326909|3301|30093|2|88|20|4|12163|87|1043|44|16.71|33.08|26.46|291.28|1164.24|735.24|1455.52|0.00|0.00|407.44|1164.24|1164.24|1571.68|1571.68|429.00| +2450835|69086|2450920|76564|1326909|3301|30093|76564|1326909|3301|30093|2|4|16|4|12776|106|1043|9|65.67|145.78|112.25|301.77|1010.25|591.03|1312.02|90.92|0.00|656.01|1010.25|1101.17|1666.26|1757.18|419.22| +2450835|69086|2450840|76564|1326909|3301|30093|76564|1326909|3301|30093|2|98|9|2|12451|256|1043|85|39.86|83.30|52.47|2620.55|4459.95|3388.10|7080.50|243.51|981.18|3185.80|3478.77|3722.28|6664.57|6908.08|90.67| +2450835|69086|2450867|76564|1326909|3301|30093|76564|1326909|3301|30093|2|27|20|4|9902|217|1043|20|22.50|31.05|9.00|441.00|180.00|450.00|621.00|3.60|0.00|229.60|180.00|183.60|409.60|413.20|-270.00| +2450835|69086|2450879|76564|1326909|3301|30093|76564|1326909|3301|30093|2|55|2|4|3722|206|1043|29|95.88|256.95|190.14|1937.49|5514.06|2780.52|7451.55|243.17|2812.17|3427.51|2701.89|2945.06|6129.40|6372.57|-78.63| +2450835|69086|2450888|76564|1326909|3301|30093|76564|1326909|3301|30093|2|80|3|1|16339|267|1043|76|79.57|229.16|48.12|13759.04|3657.12|6047.32|17416.16|329.14|0.00|7837.12|3657.12|3986.26|11494.24|11823.38|-2390.20| +2450835|69086|2450894|76564|1326909|3301|30093|76564|1326909|3301|30093|2|48|3|5|380|1|1043|45|66.21|121.82|40.20|3672.90|1809.00|2979.45|5481.90|143.27|217.08|548.10|1591.92|1735.19|2140.02|2283.29|-1387.53| +2450835|69086|2450901|76564|1326909|3301|30093|76564|1326909|3301|30093|2|2|10|1|4612|68|1043|90|1.82|4.93|2.76|195.30|248.40|163.80|443.70|0.00|156.49|30.60|91.91|91.91|122.51|122.51|-71.89| +2450835|43586|2450921|23830|794912|3241|17523|23830|794912|3241|17523|1|93|1|2|22|272|1044|25|29.72|34.17|15.03|478.50|375.75|743.00|854.25|3.75|0.00|162.25|375.75|379.50|538.00|541.75|-367.25| +2450835|43586|2450879|23830|794912|3241|17523|23830|794912|3241|17523|1|34|11|2|5245|113|1044|51|67.37|158.99|49.28|5595.21|2513.28|3435.87|8108.49|25.13|0.00|2188.92|2513.28|2538.41|4702.20|4727.33|-922.59| +2450835|43586|2450919|23830|794912|3241|17523|23830|794912|3241|17523|1|1|5|3|2750|104|1044|71|58.10|170.81|49.53|8610.88|3516.63|4125.10|12127.51|316.49|0.00|3759.45|3516.63|3833.12|7276.08|7592.57|-608.47| +2450835|43586|2450903|23830|794912|3241|17523|23830|794912|3241|17523|1|101|7|4|10808|36|1044|84|56.96|84.30|6.74|6515.04|566.16|4784.64|7081.20|25.98|277.41|353.64|288.75|314.73|642.39|668.37|-4495.89| +2450835|43586|2450862|23830|794912|3241|17523|23830|794912|3241|17523|1|42|14|4|11524|38|1044|18|45.64|114.55|61.85|948.60|1113.30|821.52|2061.90|44.53|0.00|453.60|1113.30|1157.83|1566.90|1611.43|291.78| +2450835|43586|2450868|23830|794912|3241|17523|23830|794912|3241|17523|1|42|12|4|9640|207|1044|74|76.06|143.75|103.50|2978.50|7659.00|5628.44|10637.50|689.31|0.00|2553.00|7659.00|8348.31|10212.00|10901.31|2030.56| +2450835|43586|2450901|23830|794912|3241|17523|23830|794912|3241|17523|1|58|7|3|10696|76|1044|26|74.27|219.09|87.63|3417.96|2278.38|1931.02|5696.34|0.00|205.05|512.46|2073.33|2073.33|2585.79|2585.79|142.31| +2450835|43586|2450921|23830|794912|3241|17523|23830|794912|3241|17523|1|33|4|2|5540|50|1044|9|49.71|140.67|74.55|595.08|670.95|447.39|1266.03|13.41|0.00|443.07|670.95|684.36|1114.02|1127.43|223.56| +2450835|43586|2450920|23830|794912|3241|17523|23830|794912|3241|17523|1|107|7|5|7444|172|1044|47|68.20|203.23|83.32|5635.77|3916.04|3205.40|9551.81|0.00|0.00|2578.89|3916.04|3916.04|6494.93|6494.93|710.64| +2450835|43586|2450892|23830|794912|3241|17523|23830|794912|3241|17523|1|49|3|4|6952|181|1044|20|44.05|106.60|12.79|1876.20|255.80|881.00|2132.00|17.90|0.00|874.00|255.80|273.70|1129.80|1147.70|-625.20| +2450835|70044|2450876|98695|1603663|3031|49552|96782|801488|56|28577|2|23|1|4|12961|38|1045|23|82.47|85.76|37.73|1104.69|867.79|1896.81|1972.48|34.71|0.00|473.34|867.79|902.50|1341.13|1375.84|-1029.02| +2450835|70044|2450910|98695|1603663|3031|49552|96782|801488|56|28577|2|100|10|3|7310|214|1045|62|11.87|21.12|8.65|773.14|536.30|735.94|1309.44|0.00|21.45|445.16|514.85|514.85|960.01|960.01|-221.09| +2450835|70044|2450842|98695|1603663|3031|49552|96782|801488|56|28577|2|48|11|3|6500|62|1045|4|79.41|159.61|158.01|6.40|632.04|317.64|638.44|6.32|0.00|261.76|632.04|638.36|893.80|900.12|314.40| +2450835|70044|2450906|98695|1603663|3031|49552|96782|801488|56|28577|2|96|6|1|8906|166|1045|95|67.13|96.66|32.86|6061.00|3121.70|6377.35|9182.70|62.43|0.00|4315.85|3121.70|3184.13|7437.55|7499.98|-3255.65| +2450835|70044|2450858|98695|1603663|3031|49552|96782|801488|56|28577|2|53|15|1|15824|285|1045|49|5.60|14.44|0.00|707.56|0.00|274.40|707.56|0.00|0.00|42.14|0.00|0.00|42.14|42.14|-274.40| +2450835|70044|2450868|98695|1603663|3031|49552|96782|801488|56|28577|2|24|12|4|16915|158|1045|36|97.35|116.82|2.33|4121.64|83.88|3504.60|4205.52|4.19|0.00|1387.80|83.88|88.07|1471.68|1475.87|-3420.72| +2450835|70044|2450855|98695|1603663|3031|49552|96782|801488|56|28577|2|71|8|1|5972|250|1045|79|70.30|139.19|93.25|3629.26|7366.75|5553.70|10996.01|589.34|0.00|5387.80|7366.75|7956.09|12754.55|13343.89|1813.05| +2450835|70044|2450882|98695|1603663|3031|49552|96782|801488|56|28577|2|71|16|2|9764|219|1045|100|70.60|141.90|93.65|4825.00|9365.00|7060.00|14190.00|842.85|0.00|6385.00|9365.00|10207.85|15750.00|16592.85|2305.00| +2450835|70044|2450892|98695|1603663|3031|49552|96782|801488|56|28577|2|77|17|1|6424|104|1045|56|88.99|133.48|1.33|7400.40|74.48|4983.44|7474.88|0.00|0.00|1719.20|74.48|74.48|1793.68|1793.68|-4908.96| +2450835|70044|2450915|98695|1603663|3031|49552|96782|801488|56|28577|2|16|1|4|3199|82|1045|34|19.97|46.33|20.84|866.66|708.56|678.98|1575.22|56.68|0.00|204.68|708.56|765.24|913.24|969.92|29.58| +2450835|32370|2450903|40849|103255|4414|45620|51153|1643185|344|29550|1|93|13|1|1330|229|1046|23|62.57|62.57|23.77|892.40|546.71|1439.11|1439.11|49.20|0.00|359.72|546.71|595.91|906.43|955.63|-892.40| +2450835|32370|2450914|40849|103255|4414|45620|51153|1643185|344|29550|1|101|15|5|907|283|1046|7|4.56|7.75|5.73|14.14|40.11|31.92|54.25|0.52|31.28|13.02|8.83|9.35|21.85|22.37|-23.09| +2450835|32370|2450910|40849|103255|4414|45620|51153|1643185|344|29550|1|33|13|4|11396|216|1046|77|65.12|187.54|146.28|3177.02|11263.56|5014.24|14440.58|0.00|0.00|6064.52|11263.56|11263.56|17328.08|17328.08|6249.32| +2450835|32370|2450863|40849|103255|4414|45620|51153|1643185|344|29550|1|86|4|1|10222|130|1046|43|52.56|97.76|82.11|672.95|3530.73|2260.08|4203.68|282.45|0.00|2101.84|3530.73|3813.18|5632.57|5915.02|1270.65| +2450835|32370|2450855|40849|103255|4414|45620|51153|1643185|344|29550|1|79|2|1|3044|269|1046|51|89.10|244.13|65.91|9089.22|3361.41|4544.10|12450.63|268.91|0.00|3485.85|3361.41|3630.32|6847.26|7116.17|-1182.69| +2450835|32370|2450840|40849|103255|4414|45620|51153|1643185|344|29550|1|16|20|5|10423|98|1046|33|3.80|4.25|0.42|126.39|13.86|125.40|140.25|0.35|1.94|30.69|11.92|12.27|42.61|42.96|-113.48| +2450835|32370|2450890|40849|103255|4414|45620|51153|1643185|344|29550|1|72|10|5|11576|60|1046|27|49.59|84.79|52.56|870.21|1419.12|1338.93|2289.33|99.33|0.00|434.97|1419.12|1518.45|1854.09|1953.42|80.19| +2450835|32370|2450841|40849|103255|4414|45620|51153|1643185|344|29550|1|96|7|1|2125|254|1046|64|84.89|242.78|218.50|1553.92|13984.00|5432.96|15537.92|978.88|0.00|6680.96|13984.00|14962.88|20664.96|21643.84|8551.04| +2450835|32370|2450854|40849|103255|4414|45620|51153|1643185|344|29550|1|12|19|2|17386|228|1046|96|94.03|212.50|193.37|1836.48|18563.52|9026.88|20400.00|556.90|0.00|1019.52|18563.52|19120.42|19583.04|20139.94|9536.64| +2450835|32370|2450900|40849|103255|4414|45620|51153|1643185|344|29550|1|37|1|1|14167|73|1046|29|34.81|104.08|57.24|1358.36|1659.96|1009.49|3018.32|33.19|0.00|935.54|1659.96|1693.15|2595.50|2628.69|650.47| +2450835|62142|2450860|17327|1775401|1922|28796|17327|1775401|1922|28796|2|107|9|1|1723|101|1047|50|24.41|47.11|40.98|306.50|2049.00|1220.50|2355.50|61.47|0.00|353.00|2049.00|2110.47|2402.00|2463.47|828.50| +2450835|62142|2450845|17327|1775401|1922|28796|17327|1775401|1922|28796|2|95|14|3|9802|256|1047|41|21.43|23.35|2.80|842.55|114.80|878.63|957.35|9.18|0.00|363.67|114.80|123.98|478.47|487.65|-763.83| +2450835|62142|2450903|17327|1775401|1922|28796|17327|1775401|1922|28796|2|89|17|2|12013|195|1047|77|76.56|169.19|47.37|9380.14|3647.49|5895.12|13027.63|328.27|0.00|2474.78|3647.49|3975.76|6122.27|6450.54|-2247.63| +2450835|62142|2450895|17327|1775401|1922|28796|17327|1775401|1922|28796|2|103|8|1|4706|219|1047|55|98.12|205.07|192.76|677.05|10601.80|5396.60|11278.85|572.49|1060.18|3947.35|9541.62|10114.11|13488.97|14061.46|4145.02| +2450835|62142|2450894|17327|1775401|1922|28796|17327|1775401|1922|28796|2|47|16|5|17602|213|1047|91|50.59|114.83|83.82|2821.91|7627.62|4603.69|10449.53|533.93|0.00|4074.98|7627.62|8161.55|11702.60|12236.53|3023.93| +2450835|62142|2450865|17327|1775401|1922|28796|17327|1775401|1922|28796|2|21|15|1|5887|186|1047|19|69.77|117.21|32.81|1603.60|623.39|1325.63|2226.99|24.93|0.00|178.03|623.39|648.32|801.42|826.35|-702.24| +2450835|62142|2450841|17327|1775401|1922|28796|17327|1775401|1922|28796|2|93|16|3|16207|97|1047|99|5.30|13.67|6.42|717.75|635.58|524.70|1353.33|57.20|0.00|134.64|635.58|692.78|770.22|827.42|110.88| +2450835|62142|2450866|17327|1775401|1922|28796|17327|1775401|1922|28796|2|101|20|4|7234|255|1047|72|32.03|88.08|59.01|2093.04|4248.72|2306.16|6341.76|382.38|0.00|253.44|4248.72|4631.10|4502.16|4884.54|1942.56| +2450835|34864|2450896|35888|1091991|2494|3750|35888|1091991|2494|3750|4|70|20|2|16564|171|1048|93|28.58|64.30|1.92|5801.34|178.56|2657.94|5979.90|3.57|0.00|418.50|178.56|182.13|597.06|600.63|-2479.38| +2450835|34864|2450909|35888|1091991|2494|3750|35888|1091991|2494|3750|4|32|5|3|14137|19|1048|57|35.74|78.62|31.44|2689.26|1792.08|2037.18|4481.34|17.92|0.00|313.50|1792.08|1810.00|2105.58|2123.50|-245.10| +2450835|34864|2450860|35888|1091991|2494|3750|35888|1091991|2494|3750|4|99|19|3|11104|141|1048|49|34.70|55.17|32.55|1108.38|1594.95|1700.30|2703.33|79.74|0.00|756.56|1594.95|1674.69|2351.51|2431.25|-105.35| +2450835|34864|2450888|35888|1091991|2494|3750|35888|1091991|2494|3750|4|10|15|1|15508|238|1048|31|15.02|22.53|18.02|139.81|558.62|465.62|698.43|44.68|0.00|293.26|558.62|603.30|851.88|896.56|93.00| +2450835|34864|2450894|35888|1091991|2494|3750|35888|1091991|2494|3750|4|58|20|3|2944|256|1048|48|31.91|39.88|7.57|1550.88|363.36|1531.68|1914.24|7.26|0.00|822.72|363.36|370.62|1186.08|1193.34|-1168.32| +2450835|34864|2450850|35888|1091991|2494|3750|35888|1091991|2494|3750|4|10|14|5|8818|190|1048|50|65.45|189.80|5.69|9205.50|284.50|3272.50|9490.00|8.53|0.00|3606.00|284.50|293.03|3890.50|3899.03|-2988.00| +2450835|34864|2450839|35888|1091991|2494|3750|35888|1091991|2494|3750|4|103|6|2|6175|279|1048|6|13.79|14.89|1.63|79.56|9.78|82.74|89.34|0.09|0.00|0.00|9.78|9.87|9.78|9.87|-72.96| +2450835|34864|2450837|35888|1091991|2494|3750|35888|1091991|2494|3750|4|11|9|3|15908|214|1048|81|73.95|76.16|21.32|4442.04|1726.92|5989.95|6168.96|86.34|0.00|1172.07|1726.92|1813.26|2898.99|2985.33|-4263.03| +2450835|34864|2450915|35888|1091991|2494|3750|35888|1091991|2494|3750|4|51|15|2|10144|244|1048|90|21.75|22.18|11.75|938.70|1057.50|1957.50|1996.20|0.00|0.00|698.40|1057.50|1057.50|1755.90|1755.90|-900.00| +2450835|34864|2450870|35888|1091991|2494|3750|35888|1091991|2494|3750|4|89|9|5|5683|162|1048|36|75.03|106.54|28.76|2800.08|1035.36|2701.08|3835.44|82.82|0.00|1227.24|1035.36|1118.18|2262.60|2345.42|-1665.72| +2450835|34864|2450889|35888|1091991|2494|3750|35888|1091991|2494|3750|4|102|8|3|7753|153|1048|84|12.18|16.80|4.53|1030.68|380.52|1023.12|1411.20|19.02|0.00|154.56|380.52|399.54|535.08|554.10|-642.60| +2450835|34864|2450842|35888|1091991|2494|3750|35888|1091991|2494|3750|4|60|14|1|182|206|1048|51|67.20|103.48|93.13|527.85|4749.63|3427.20|5277.48|142.48|0.00|474.81|4749.63|4892.11|5224.44|5366.92|1322.43| +2450835|69542|2450915|21462|105442|622|15802|21462|105442|622|15802|4|68|4|2|14200|220|1049|1|50.02|59.02|1.18|57.84|1.18|50.02|59.02|0.05|0.34|1.18|0.84|0.89|2.02|2.07|-49.18| +2450835|69542|2450915|21462|105442|622|15802|21462|105442|622|15802|4|87|17|2|3226|254|1049|53|42.15|48.05|0.48|2521.21|25.44|2233.95|2546.65|2.03|0.00|203.52|25.44|27.47|228.96|230.99|-2208.51| +2450835|69542|2450867|21462|105442|622|15802|21462|105442|622|15802|4|78|10|2|8755|97|1049|55|64.77|132.77|106.21|1460.80|5841.55|3562.35|7302.35|175.24|0.00|3358.85|5841.55|6016.79|9200.40|9375.64|2279.20| +2450835|69542|2450904|21462|105442|622|15802|21462|105442|622|15802|4|57|1|4|17935|237|1049|60|59.42|64.76|18.78|2758.80|1126.80|3565.20|3885.60|56.34|0.00|116.40|1126.80|1183.14|1243.20|1299.54|-2438.40| +2450835|69542|2450898|21462|105442|622|15802|21462|105442|622|15802|4|11|14|5|13490|34|1049|38|63.29|91.77|32.11|2267.08|1220.18|2405.02|3487.26|61.00|0.00|69.54|1220.18|1281.18|1289.72|1350.72|-1184.84| +2450835|69542|2450878|21462|105442|622|15802|21462|105442|622|15802|4|18|4|3|17029|49|1049|36|57.35|167.46|133.96|1206.00|4822.56|2064.60|6028.56|241.12|0.00|1989.36|4822.56|5063.68|6811.92|7053.04|2757.96| +2450835|69542|2450875|21462|105442|622|15802|21462|105442|622|15802|4|74|16|3|2342|275|1049|86|23.08|51.00|51.00|0.00|4386.00|1984.88|4386.00|175.44|0.00|1315.80|4386.00|4561.44|5701.80|5877.24|2401.12| +2450835|69542|2450918|21462|105442|622|15802|21462|105442|622|15802|4|73|3|4|3940|231|1049|57|71.81|182.39|16.41|9460.86|935.37|4093.17|10396.23|65.47|0.00|2286.84|935.37|1000.84|3222.21|3287.68|-3157.80| +2450835|58203|2450845|29373|1759329|132|28888|29373|1759329|132|28888|1|76|7|3|8668|68|1050|56|34.77|92.83|14.85|4366.88|831.60|1947.12|5198.48|41.58|0.00|1351.28|831.60|873.18|2182.88|2224.46|-1115.52| +2450835|58203|2450875|29373|1759329|132|28888|29373|1759329|132|28888|1|68|6|3|10760|47|1050|88|33.04|55.50|13.32|3711.84|1172.16|2907.52|4884.00|58.60|0.00|536.80|1172.16|1230.76|1708.96|1767.56|-1735.36| +2450835|58203|2450905|29373|1759329|132|28888|29373|1759329|132|28888|1|16|5|2|8341|221|1050|36|32.67|86.57|48.47|1371.60|1744.92|1176.12|3116.52|157.04|0.00|62.28|1744.92|1901.96|1807.20|1964.24|568.80| +2450835|58203|2450919|29373|1759329|132|28888|29373|1759329|132|28888|1|49|14|3|3302|160|1050|80|76.77|184.24|105.01|6338.40|8400.80|6141.60|14739.20|84.00|0.00|5158.40|8400.80|8484.80|13559.20|13643.20|2259.20| +2450835|58203|2450897|29373|1759329|132|28888|29373|1759329|132|28888|1|49|4|3|12229|123|1050|50|80.16|162.72|115.53|2359.50|5776.50|4008.00|8136.00|346.59|0.00|81.00|5776.50|6123.09|5857.50|6204.09|1768.50| +2450835|58203|2450895|29373|1759329|132|28888|29373|1759329|132|28888|1|22|20|3|9202|22|1050|61|59.48|166.54|124.90|2540.04|7618.90|3628.28|10158.94|457.13|0.00|710.65|7618.90|8076.03|8329.55|8786.68|3990.62| +2450835|58203|2450857|29373|1759329|132|28888|29373|1759329|132|28888|1|4|13|1|8810|54|1050|27|85.26|190.12|178.71|308.07|4825.17|2302.02|5133.24|180.94|1206.29|1539.81|3618.88|3799.82|5158.69|5339.63|1316.86| +2450835|58203|2450866|29373|1759329|132|28888|29373|1759329|132|28888|1|92|6|2|7501|68|1050|49|33.24|48.86|39.57|455.21|1938.93|1628.76|2394.14|116.33|0.00|71.54|1938.93|2055.26|2010.47|2126.80|310.17| +2450835|58203|2450886|29373|1759329|132|28888|29373|1759329|132|28888|1|50|8|4|2810|138|1050|69|83.46|193.62|58.08|9352.26|4007.52|5758.74|13359.78|160.30|0.00|801.09|4007.52|4167.82|4808.61|4968.91|-1751.22| +2450835|58203|2450901|29373|1759329|132|28888|29373|1759329|132|28888|1|29|15|2|15283|189|1050|68|48.19|48.19|37.10|754.12|2522.80|3276.92|3276.92|176.59|0.00|818.72|2522.80|2699.39|3341.52|3518.11|-754.12| +2450835|58203|2450861|29373|1759329|132|28888|29373|1759329|132|28888|1|90|17|3|17257|172|1050|12|90.46|180.92|175.49|65.16|2105.88|1085.52|2171.04|42.11|0.00|65.04|2105.88|2147.99|2170.92|2213.03|1020.36| +2450835|38376|2450871|21094|1143823|6552|27681|21094|1143823|6552|27681|2|15|16|1|8590|36|1051|68|19.45|42.20|29.54|860.88|2008.72|1322.60|2869.60|140.61|0.00|831.64|2008.72|2149.33|2840.36|2980.97|686.12| +2450835|38376|2450868|21094|1143823|6552|27681|21094|1143823|6552|27681|2|29|19|2|4964|36|1051|89|7.96|12.01|6.72|470.81|598.08|708.44|1068.89|53.82|0.00|224.28|598.08|651.90|822.36|876.18|-110.36| +2450835|38376|2450885|21094|1143823|6552|27681|21094|1143823|6552|27681|2|54|5|5|14926|252|1051|27|36.02|59.79|35.27|662.04|952.29|972.54|1614.33|7.61|761.83|322.65|190.46|198.07|513.11|520.72|-782.08| +2450835|38376|2450892|21094|1143823|6552|27681|21094|1143823|6552|27681|2|70|19|4|2774|116|1051|56|37.76|71.36|70.64|40.32|3955.84|2114.56|3996.16|237.35|0.00|1358.56|3955.84|4193.19|5314.40|5551.75|1841.28| +2450835|38376|2450880|21094|1143823|6552|27681|21094|1143823|6552|27681|2|107|9|5|5449|62|1051|70|64.01|68.49|42.46|1822.10|2972.20|4480.70|4794.30|89.16|0.00|910.70|2972.20|3061.36|3882.90|3972.06|-1508.50| +2450835|38376|2450871|21094|1143823|6552|27681|21094|1143823|6552|27681|2|107|4|3|12709|9|1051|19|66.36|94.89|69.26|486.97|1315.94|1260.84|1802.91|26.31|0.00|35.91|1315.94|1342.25|1351.85|1378.16|55.10| +2450835|38376|2450859|21094|1143823|6552|27681|21094|1143823|6552|27681|2|25|6|1|15127|226|1051|73|53.51|67.95|64.55|248.20|4712.15|3906.23|4960.35|424.09|0.00|197.83|4712.15|5136.24|4909.98|5334.07|805.92| +2450835|38376|2450841|21094|1143823|6552|27681|21094|1143823|6552|27681|2|36|17|4|1081|10|1051|23|11.90|28.20|11.28|389.16|259.44|273.70|648.60|20.75|0.00|32.43|259.44|280.19|291.87|312.62|-14.26| +2450835|38376|2450895|21094|1143823|6552|27681|21094|1143823|6552|27681|2|100|3|1|1412|140|1051|14|71.39|154.20|141.86|172.76|1986.04|999.46|2158.80|178.74|0.00|496.44|1986.04|2164.78|2482.48|2661.22|986.58| +2450835|38376|2450912|21094|1143823|6552|27681|21094|1143823|6552|27681|2|10|16|4|5953|97|1051|74|13.84|25.74|12.61|971.62|933.14|1024.16|1904.76|65.31|0.00|475.82|933.14|998.45|1408.96|1474.27|-91.02| +2450835|38376|2450867|21094|1143823|6552|27681|21094|1143823|6552|27681|2|79|7|5|10168|131|1051|97|1.48|1.58|1.04|52.38|100.88|143.56|153.26|6.05|0.00|69.84|100.88|106.93|170.72|176.77|-42.68| +2450835|38376|2450911|21094|1143823|6552|27681|21094|1143823|6552|27681|2|84|19|1|3286|232|1051|93|43.06|103.34|73.37|2787.21|6823.41|4004.58|9610.62|0.00|4639.91|3843.69|2183.50|2183.50|6027.19|6027.19|-1821.08| +2450835|38376|2450885|21094|1143823|6552|27681|21094|1143823|6552|27681|2|46|16|3|15691|220|1051|40|37.39|37.39|29.53|314.40|1181.20|1495.60|1495.60|100.99|59.06|702.80|1122.14|1223.13|1824.94|1925.93|-373.46| +2450835|38376|2450856|21094|1143823|6552|27681|21094|1143823|6552|27681|2|92|12|4|7603|42|1051|37|24.17|66.70|47.35|715.95|1751.95|894.29|2467.90|157.67|0.00|1233.95|1751.95|1909.62|2985.90|3143.57|857.66| +2450835|68348|2450879|31866|592312|2005|19298|31866|592312|2005|19298|2|96|17|3|320|5|1052|57|52.86|84.04|13.44|4024.20|766.08|3013.02|4790.28|30.64|0.00|2059.41|766.08|796.72|2825.49|2856.13|-2246.94| +2450835|68348|2450869|31866|592312|2005|19298|31866|592312|2005|19298|2|59|5|1|15806|46|1052|37|38.29|54.37|15.22|1448.55|563.14|1416.73|2011.69|5.23|388.56|60.31|174.58|179.81|234.89|240.12|-1242.15| +2450835|68348|2450864|31866|592312|2005|19298|31866|592312|2005|19298|2|70|17|5|13846|219|1052|24|79.12|168.52|148.29|485.52|3558.96|1898.88|4044.48|35.58|0.00|1698.48|3558.96|3594.54|5257.44|5293.02|1660.08| +2450835|68348|2450860|31866|592312|2005|19298|31866|592312|2005|19298|2|30|1|2|9946|154|1052|8|4.16|10.69|2.67|64.16|21.36|33.28|85.52|0.07|20.29|9.36|1.07|1.14|10.43|10.50|-32.21| +2450835|68348|2450878|31866|592312|2005|19298|31866|592312|2005|19298|2|105|3|5|17078|176|1052|16|40.34|87.13|25.26|989.92|404.16|645.44|1394.08|12.12|0.00|446.08|404.16|416.28|850.24|862.36|-241.28| +2450835|27321|2450911|9187|784529|1471|33260|9187|784529|1471|33260|1|61|7|5|3625|169|1053|19|14.72|18.84|16.95|35.91|322.05|279.68|357.96|12.88|0.00|0.00|322.05|334.93|322.05|334.93|42.37| +2450835|27321|2450902|9187|784529|1471|33260|9187|784529|1471|33260|1|26|7|5|16549|278|1053|30|40.30|56.42|35.54|626.40|1066.20|1209.00|1692.60|6.71|394.49|135.30|671.71|678.42|807.01|813.72|-537.29| +2450835|27321|2450898|9187|784529|1471|33260|9187|784529|1471|33260|1|78|6|3|11911|245|1053|58|8.23|19.17|16.29|167.04|944.82|477.34|1111.86|0.00|85.03|299.86|859.79|859.79|1159.65|1159.65|382.45| +2450835|27321|2450846|9187|784529|1471|33260|9187|784529|1471|33260|1|6|2|3|17338|118|1053|16|10.49|26.43|12.95|215.68|207.20|167.84|422.88|16.57|0.00|59.20|207.20|223.77|266.40|282.97|39.36| +2450835|27321|2450852|9187|784529|1471|33260|9187|784529|1471|33260|1|31|16|2|9962|251|1053|56|70.63|138.43|16.61|6821.92|930.16|3955.28|7752.08|27.90|0.00|2790.48|930.16|958.06|3720.64|3748.54|-3025.12| +2450835|29968|2450914|23194|847605|3569|17980|23194|847605|3569|17980|2|35|10|4|8234|189|1054|1|81.84|124.39|58.46|65.93|58.46|81.84|124.39|1.75|0.00|53.48|58.46|60.21|111.94|113.69|-23.38| +2450835|29968|2450844|23194|847605|3569|17980|23194|847605|3569|17980|2|72|11|3|10840|71|1054|61|22.32|49.77|16.92|2003.85|1032.12|1361.52|3035.97|92.89|0.00|1396.29|1032.12|1125.01|2428.41|2521.30|-329.40| +2450835|29968|2450856|23194|847605|3569|17980|23194|847605|3569|17980|2|1|7|5|9776|116|1054|90|79.96|238.28|114.37|11151.90|10293.30|7196.40|21445.20|823.46|0.00|8148.60|10293.30|11116.76|18441.90|19265.36|3096.90| +2450835|29968|2450892|23194|847605|3569|17980|23194|847605|3569|17980|2|78|9|1|9490|113|1054|90|38.82|101.32|54.71|4194.90|4923.90|3493.80|9118.80|443.15|0.00|2187.90|4923.90|5367.05|7111.80|7554.95|1430.10| +2450835|29968|2450857|23194|847605|3569|17980|23194|847605|3569|17980|2|100|16|5|12787|21|1054|55|96.93|96.93|90.14|373.45|4957.70|5331.15|5331.15|347.03|0.00|1492.70|4957.70|5304.73|6450.40|6797.43|-373.45| +2450835|29968|2450878|23194|847605|3569|17980|23194|847605|3569|17980|2|46|18|2|7096|253|1054|49|97.00|174.60|43.65|6416.55|2138.85|4753.00|8555.40|42.77|0.00|4191.95|2138.85|2181.62|6330.80|6373.57|-2614.15| +2450835|29968|2450873|23194|847605|3569|17980|23194|847605|3569|17980|2|3|3|4|9614|87|1054|43|74.01|172.44|101.73|3040.53|4374.39|3182.43|7414.92|43.74|0.00|2743.40|4374.39|4418.13|7117.79|7161.53|1191.96| +2450835|29968|2450839|23194|847605|3569|17980|23194|847605|3569|17980|2|87|14|4|5836|71|1054|42|77.40|144.73|62.23|3465.00|2613.66|3250.80|6078.66|235.22|0.00|1762.74|2613.66|2848.88|4376.40|4611.62|-637.14| +2450835|29968|2450840|23194|847605|3569|17980|23194|847605|3569|17980|2|54|10|1|14984|88|1054|70|95.06|104.56|56.46|3367.00|3952.20|6654.20|7319.20|39.52|0.00|511.70|3952.20|3991.72|4463.90|4503.42|-2702.00| +2450835|81262|2450837|12322|1614837|4905|45420|12322|1614837|4905|45420|1|54|10|2|7784|279|1055|80|38.30|67.40|55.26|971.20|4420.80|3064.00|5392.00|397.87|0.00|1401.60|4420.80|4818.67|5822.40|6220.27|1356.80| +2450835|81262|2450925|12322|1614837|4905|45420|12322|1614837|4905|45420|1|17|20|5|11500|265|1055|51|60.54|70.22|18.25|2650.47|930.75|3087.54|3581.22|65.15|0.00|71.40|930.75|995.90|1002.15|1067.30|-2156.79| +2450835|81262|2450882|12322|1614837|4905|45420|12322|1614837|4905|45420|1|32|11|3|16100|142|1055|1|64.27|106.04|61.50|44.54|61.50|64.27|106.04|0.61|0.00|11.66|61.50|62.11|73.16|73.77|-2.77| +2450835|81262|2450881|12322|1614837|4905|45420|12322|1614837|4905|45420|1|66|13|5|8968|167|1055|20|49.81|141.95|124.91|340.80|2498.20|996.20|2839.00|0.00|2198.41|141.80|299.79|299.79|441.59|441.59|-696.41| +||||||||1614837|4905||1|81|||17402||1055|41|61.29||78.82||3231.62|||24.23|||807.91||2615.60|2639.83|-1704.98| +2450835|81262|2450839|12322|1614837|4905|45420|12322|1614837|4905|45420|1|55|20|5|1552|188|1055|61|59.50|123.76|64.35|3624.01|3925.35|3629.50|7549.36|0.00|0.00|2339.96|3925.35|3925.35|6265.31|6265.31|295.85| +2450835|81262|2450899|12322|1614837|4905|45420|12322|1614837|4905|45420|1|96|8|4|16928|158|1055|32|8.24|24.14|1.68|718.72|53.76|263.68|772.48|0.86|41.39|339.84|12.37|13.23|352.21|353.07|-251.31| +2450835|81262|2450838|12322|1614837|4905|45420|12322|1614837|4905|45420|1|73|16|2|17260|202|1055|26|62.36|150.28|115.71|898.82|3008.46|1621.36|3907.28|240.67|0.00|898.56|3008.46|3249.13|3907.02|4147.69|1387.10| +2450835|81262|2450877|12322|1614837|4905|45420|12322|1614837|4905|45420|1|99|20|5|877|155|1055|68|54.74|105.10|53.60|3502.00|3644.80|3722.32|7146.80|291.58|0.00|2501.04|3644.80|3936.38|6145.84|6437.42|-77.52| +2450835|81262|2450905|12322|1614837|4905|45420|12322|1614837|4905|45420|1|24|9|5|11140|12|1055|30|54.83|83.34|27.50|1675.20|825.00|1644.90|2500.20|0.00|825.00|499.80|0.00|0.00|499.80|499.80|-1644.90| +2450835|81262|2450891|12322|1614837|4905|45420|12322|1614837|4905|45420|1|36|2|5|2624|83|1055|83|30.67|58.27|0.58|4788.27|48.14|2545.61|4836.41|0.48|0.00|0.00|48.14|48.62|48.14|48.62|-2497.47| +2450835|81262|2450890|12322|1614837|4905|45420|12322|1614837|4905|45420|1|61|17|3|3688|107|1055|86|98.37|242.97|41.30|17343.62|3551.80|8459.82|20895.42|281.30|426.21|8357.48|3125.59|3406.89|11483.07|11764.37|-5334.23| +2450835|81262|2450889|12322|1614837|4905|45420|12322|1614837|4905|45420|1|65|16|4|12175|80|1055|1|7.39|17.29|5.70|11.59|5.70|7.39|17.29|0.06|4.10|7.95|1.60|1.66|9.55|9.61|-5.79| +2450835|47060|2450915|11388|346626|1064|23502|11388|346626|1064|23502|4|101|8|2|3829|237|1056|1|86.20|133.61|46.76|86.85|46.76|86.20|133.61|2.80|0.00|36.07|46.76|49.56|82.83|85.63|-39.44| +2450835|47060|2450900|11388|346626|1064|23502|11388|346626|1064|23502|4|40|9|1|17194|253|1056|85|14.87|26.17|10.20|1357.45|867.00|1263.95|2224.45|46.73|199.41|600.10|667.59|714.32|1267.69|1314.42|-596.36| +2450835|47060|2450873|11388|346626|1064|23502|11388|346626|1064|23502|4|38|11|5|12907|65|1056|12|76.57|165.39|56.23|1309.92|674.76|918.84|1984.68|33.73|0.00|258.00|674.76|708.49|932.76|966.49|-244.08| +2450835|47060|2450869|11388|346626|1064|23502|11388|346626|1064|23502|4|19|10|2|7340|228|1056|21|71.69|83.16|72.34|227.22|1519.14|1505.49|1746.36|121.53|0.00|715.89|1519.14|1640.67|2235.03|2356.56|13.65| +2450835|47060|2450876|11388|346626|1064|23502|11388|346626|1064|23502|4|22|10|5|2131|3|1056|90|65.54|135.01|129.60|486.90|11664.00|5898.60|12150.90|1049.76|0.00|4860.00|11664.00|12713.76|16524.00|17573.76|5765.40| +2450835|47060|2450863|11388|346626|1064|23502|11388|346626|1064|23502|4|33|13|4|13663|261|1056|90|99.70|141.57|80.69|5479.20|7262.10|8973.00|12741.30|217.86|0.00|254.70|7262.10|7479.96|7516.80|7734.66|-1710.90| +2450835|47060|2450918|11388|346626|1064|23502|11388|346626|1064|23502|4|36|14|3|3470|299|1056|13|36.40|67.70|54.16|176.02|704.08|473.20|880.10|14.08|0.00|105.56|704.08|718.16|809.64|823.72|230.88| +2450835|47060|2450845|11388|346626|1064|23502|11388|346626|1064|23502|4|99|3|3|5162|85|1056|8|37.94|70.94|65.97|39.76|527.76|303.52|567.52|31.66|0.00|22.64|527.76|559.42|550.40|582.06|224.24| +2450835|47060|2450924|11388|346626|1064|23502|11388|346626|1064|23502|4|87|14|3|16672|66|1056|56|32.39|71.90|50.33|1207.92|2818.48|1813.84|4026.40|253.66|0.00|925.68|2818.48|3072.14|3744.16|3997.82|1004.64| +2450835|47060|2450854|11388|346626|1064|23502|11388|346626|1064|23502|4|1|20|1|14446|143|1056|17|14.05|22.19|11.09|188.70|188.53|238.85|377.23|16.96|0.00|33.83|188.53|205.49|222.36|239.32|-50.32| +2450835|45335|2450869|38012|1743571|2730|45136|38012|1743571|2730|45136|2|62|17|3|14864|130|1057|3|45.13|76.72|42.19|103.59|126.57|135.39|230.16|8.85|0.00|64.44|126.57|135.42|191.01|199.86|-8.82| +2450835|45335|2450922|38012|1743571|2730|45136|38012|1743571|2730|45136|2|12|12|2|12032|231|1057|70|25.44|70.21|42.82|1917.30|2997.40|1780.80|4914.70|209.81|0.00|786.10|2997.40|3207.21|3783.50|3993.31|1216.60| +2450835|45335|2450916|38012|1743571|2730|45136|38012|1743571|2730|45136|2|96|19|2|11084|80|1057|94|36.99|79.52|64.41|1420.34|6054.54|3477.06|7474.88|363.27|0.00|2540.82|6054.54|6417.81|8595.36|8958.63|2577.48| +2450835|45335|2450875|38012|1743571|2730|45136|38012|1743571|2730|45136|2|54|8|5|7567|219|1057|56|30.47|36.56|27.42|511.84|1535.52|1706.32|2047.36|138.19|0.00|347.76|1535.52|1673.71|1883.28|2021.47|-170.80| +2450835|45335|2450910|38012|1743571|2730|45136|38012|1743571|2730|45136|2|75|3|4|10106|18|1057|100|79.96|175.11|113.82|6129.00|11382.00|7996.00|17511.00|455.28|0.00|5428.00|11382.00|11837.28|16810.00|17265.28|3386.00| +2450835|45335|2450886|38012|1743571|2730|45136|38012|1743571|2730|45136|2|31|6|5|13756|55|1057|93|56.31|141.90|114.93|2508.21|10688.49|5236.83|13196.70|320.65|0.00|1450.80|10688.49|11009.14|12139.29|12459.94|5451.66| +2450835|45335|2450909|38012|1743571|2730|45136|38012|1743571|2730|45136|2|35|19|1|13405|60|1057|52|40.45|80.09|14.41|3415.36|749.32|2103.40|4164.68|59.94|0.00|1207.44|749.32|809.26|1956.76|2016.70|-1354.08| +2450835|45335|2450850|38012|1743571|2730|45136|38012|1743571|2730|45136|2|63|2|1|15991|27|1057|19|58.38|84.06|68.08|303.62|1293.52|1109.22|1597.14|22.63|840.78|702.62|452.74|475.37|1155.36|1177.99|-656.48| +2450835|45335|2450875|38012|1743571|2730|45136|38012|1743571|2730|45136|2|99|9|4|10345|7|1057|20|13.45|15.73|11.95|75.60|239.00|269.00|314.60|0.23|234.22|88.00|4.78|5.01|92.78|93.01|-264.22| +2450835|53458|2450853|45274|1477875|4876|35077|45274|1477875|4876|35077|2|19|5|5|6614|105|1058|54|22.82|26.47|3.70|1229.58|199.80|1232.28|1429.38|1.99|0.00|99.90|199.80|201.79|299.70|301.69|-1032.48| +2450835|53458|2450900|45274|1477875|4876|35077|45274|1477875|4876|35077|2|14|7|5|14878|118|1058|16|60.11|131.03|62.89|1090.24|1006.24|961.76|2096.48|30.18|0.00|964.32|1006.24|1036.42|1970.56|2000.74|44.48| +2450835|53458|2450887|45274|1477875|4876|35077|45274|1477875|4876|35077|2|90|11|5|7076|128|1058|70|28.99|77.98|9.35|4804.10|654.50|2029.30|5458.60|23.56|183.26|2729.30|471.24|494.80|3200.54|3224.10|-1558.06| +2450835|53458|2450847|45274|1477875|4876|35077|45274|1477875|4876|35077|2|63|3|3|7694|171|1058|21|48.40|106.96|58.82|1010.94|1235.22|1016.40|2246.16|111.16|0.00|673.68|1235.22|1346.38|1908.90|2020.06|218.82| +2450835|53458|2450868|45274|1477875|4876|35077|45274|1477875|4876|35077|2|91|11|3|5113|150|1058|15|40.40|88.88|15.99|1093.35|239.85|606.00|1333.20|21.58|0.00|93.30|239.85|261.43|333.15|354.73|-366.15| +2450835|53458|2450881|45274|1477875|4876|35077|45274|1477875|4876|35077|2|69|5|3|5785|98|1058|65|67.44|80.25|34.50|2973.75|2242.50|4383.60|5216.25|89.70|0.00|1407.90|2242.50|2332.20|3650.40|3740.10|-2141.10| +2450835|53458|2450849|45274|1477875|4876|35077|45274|1477875|4876|35077|2|12|1|5|5671|165|1058|92|32.25|88.36|53.89|3171.24|4957.88|2967.00|8129.12|347.05|0.00|2600.84|4957.88|5304.93|7558.72|7905.77|1990.88| +2450835|53458|2450916|45274|1477875|4876|35077|45274|1477875|4876|35077|2|53|3|2|4393|48|1058|30|6.84|8.48|5.59|86.70|167.70|205.20|254.40|13.41|0.00|2.40|167.70|181.11|170.10|183.51|-37.50| +2450835|53458|2450920|45274|1477875|4876|35077|45274|1477875|4876|35077|2|15|16|4|11305|47|1058|23|91.11|172.19|67.15|2415.92|1544.45|2095.53|3960.37|30.88|0.00|395.83|1544.45|1575.33|1940.28|1971.16|-551.08| +2450835|53458|2450857|45274|1477875|4876|35077|45274|1477875|4876|35077|2|88|20|5|14884|101|1058|99|67.08|180.44|126.30|5359.86|12503.70|6640.92|17863.56|750.22|0.00|892.98|12503.70|13253.92|13396.68|14146.90|5862.78| +2450835|75396|2450869|24379|478983|4787|28991|24379|478983|4787|28991|2|80|17|5|8780|276|1059|31|2.59|3.57|3.24|10.23|100.44|80.29|110.67|5.20|26.11|22.01|74.33|79.53|96.34|101.54|-5.96| +2450835|75396|2450840|24379|478983|4787|28991|24379|478983|4787|28991|2|97|19|1|15181|262|1059|54|36.04|67.39|35.04|1746.90|1892.16|1946.16|3639.06|132.45|0.00|72.36|1892.16|2024.61|1964.52|2096.97|-54.00| +2450835|75396|2450871|24379|478983|4787|28991|24379|478983|4787|28991|2|41|1|3|15049|222|1059|24|40.50|53.86|33.93|478.32|814.32|972.00|1292.64|8.14|0.00|491.04|814.32|822.46|1305.36|1313.50|-157.68| +2450835|75396|2450910|24379|478983|4787|28991|24379|478983|4787|28991|2|63|10|3|2134|194|1059|77|83.06|119.60|76.54|3315.62|5893.58|6395.62|9209.20|294.67|0.00|2025.87|5893.58|6188.25|7919.45|8214.12|-502.04| +2450835|75396|2450861|24379|478983|4787|28991|24379|478983|4787|28991|2|28|7|5|5248|169|1059|37|74.23|200.42|34.07|6154.95|1260.59|2746.51|7415.54|1.13|1247.98|0.00|12.61|13.74|12.61|13.74|-2733.90| +2450835|75396|2450878|24379|478983|4787|28991|24379|478983|4787|28991|2|56|3|2|2071|230|1059|37|51.64|54.22|35.78|682.28|1323.86|1910.68|2006.14|119.14|0.00|0.00|1323.86|1443.00|1323.86|1443.00|-586.82| +2450835|75396|2450840|24379|478983|4787|28991|24379|478983|4787|28991|2|36|3|2|5779|12|1059|57|9.76|23.22|0.92|1271.10|52.44|556.32|1323.54|3.14|0.00|118.56|52.44|55.58|171.00|174.14|-503.88| +2450835|22758|2450869|46871|1343895|4119|1143|46871|1343895|4119|1143|2|41|7|5|10420|189|1060|31|89.57|135.25|108.20|838.55|3354.20|2776.67|4192.75|100.62|0.00|628.68|3354.20|3454.82|3982.88|4083.50|577.53| +2450835|22758|2450896|46871|1343895|4119|1143|46871|1343895|4119|1143|2|25|3|2|3703|75|1060|58|4.06|8.72|2.70|349.16|156.60|235.48|505.76|12.52|0.00|156.60|156.60|169.12|313.20|325.72|-78.88| +2450835|22758|2450869|46871|1343895|4119|1143|46871|1343895|4119|1143|2|75|5|2|1888|211|1060|91|60.39|108.70|23.91|7715.89|2175.81|5495.49|9891.70|87.03|0.00|3560.83|2175.81|2262.84|5736.64|5823.67|-3319.68| +2450835|22758|2450885|46871|1343895|4119|1143|46871|1343895|4119|1143|2|14|5|5|12010|178|1060|59|75.74|218.88|170.72|2841.44|10072.48|4468.66|12913.92|100.72|0.00|257.83|10072.48|10173.20|10330.31|10431.03|5603.82| +2450835|22758|2450874|46871|1343895|4119|1143|46871|1343895|4119|1143|2|14|15|1|4489|16|1060|23|36.44|60.12|46.29|318.09|1064.67|838.12|1382.76|85.17|0.00|483.92|1064.67|1149.84|1548.59|1633.76|226.55| +2450835|22758|2450856|46871|1343895|4119|1143|46871|1343895|4119|1143|2|5|4|5|2336|39|1060|72|92.77|209.66|52.41|11322.00|3773.52|6679.44|15095.52|75.47|0.00|2566.08|3773.52|3848.99|6339.60|6415.07|-2905.92| +2450835|22758|2450885|46871|1343895|4119|1143|46871|1343895|4119|1143|2|90|9|2|13436|198|1060|60|5.57|9.58|8.23|81.00|493.80|334.20|574.80|29.62|0.00|206.40|493.80|523.42|700.20|729.82|159.60| +2450835|66014|2450840|59489|895759|29|11499|59489|895759|29|11499|2|37|20|4|13831|5|1061|3|75.08|102.10|14.29|263.43|42.87|225.24|306.30|0.42|0.00|110.25|42.87|43.29|153.12|153.54|-182.37| +2450835|66014|2450841|59489|895759|29|11499|59489|895759|29|11499|2|77|4|1|9248|102|1061|35|83.68|132.21|15.86|4072.25|555.10|2928.80|4627.35|16.65|0.00|1989.75|555.10|571.75|2544.85|2561.50|-2373.70| +2450835|66014|2450870|59489|895759|29|11499|59489|895759|29|11499|2|10|17|4|15668|205|1061|52|82.18|227.63|93.32|6984.12|4852.64|4273.36|11836.76|48.52|0.00|3787.68|4852.64|4901.16|8640.32|8688.84|579.28| +2450835|66014|2450861|59489|895759|29|11499|59489|895759|29|11499|2|8|19|2|15992|288|1061|31|7.59|21.85|8.74|406.41|270.94|235.29|677.35|5.41|0.00|135.47|270.94|276.35|406.41|411.82|35.65| +2450835|66014|2450847|59489|895759|29|11499|59489|895759|29|11499|2|35|4|1|12248|103|1061|23|27.71|49.32|30.08|442.52|691.84|637.33|1134.36|62.26|0.00|158.70|691.84|754.10|850.54|912.80|54.51| +2450835|66014|2450919|59489|895759|29|11499|59489|895759|29|11499|2|84|14|2|8770|65|1061|63|17.83|36.55|29.97|414.54|1888.11|1123.29|2302.65|113.28|0.00|1127.70|1888.11|2001.39|3015.81|3129.09|764.82| +2450835|66014|2450880|59489|895759|29|11499|59489|895759|29|11499|2|66|5|5|13858|197|1061|72|99.66|112.61|34.90|5595.12|2512.80|7175.52|8107.92|94.23|628.20|567.36|1884.60|1978.83|2451.96|2546.19|-5290.92| +2450835|66014|2450867|59489|895759|29|11499|59489|895759|29|11499|2|14|16|3|6235|89|1061|90|29.45|69.50|25.02|4003.20|2251.80|2650.50|6255.00|127.00|135.10|2251.80|2116.70|2243.70|4368.50|4495.50|-533.80| +2450835|66014|2450876|59489|895759|29|11499|59489|895759|29|11499|2|30|7|1|8707|2|1061|12|27.22|33.75|1.68|384.84|20.16|326.64|405.00|0.80|0.00|52.56|20.16|20.96|72.72|73.52|-306.48| +2450835|66014|2450850|59489|895759|29|11499|59489|895759|29|11499|2|80|4|4|17368|9|1061|58|59.53|145.84|125.42|1184.36|7274.36|3452.74|8458.72|218.23|0.00|845.64|7274.36|7492.59|8120.00|8338.23|3821.62| +2450835|66014|2450905|59489|895759|29|11499|59489|895759|29|11499|2|102|13|3|6295|241|1061|54|74.09|142.25|34.14|5837.94|1843.56|4000.86|7681.50|117.80|534.63|76.68|1308.93|1426.73|1385.61|1503.41|-2691.93| +2450835|48944|2450896|4933|1777706|6926|4099|4933|1777706|6926|4099|2|65|6|2|8773|126|1062|59|69.16|194.33|93.27|5962.54|5502.93|4080.44|11465.47|220.11|0.00|3210.19|5502.93|5723.04|8713.12|8933.23|1422.49| +2450835|48944|2450882|4933|1777706|6926|4099|4933|1777706|6926|4099|2|1|11|5|958|163|1062|28|73.26|184.61|5.53|5014.24|154.84|2051.28|5169.08|12.38|0.00|1188.88|154.84|167.22|1343.72|1356.10|-1896.44| +2450835|48944|2450854|4933|1777706|6926|4099|4933|1777706|6926|4099|2|76|1|5|16939|158|1062|74|29.98|66.55|27.95|2856.40|2068.30|2218.52|4924.70|62.04|0.00|1378.62|2068.30|2130.34|3446.92|3508.96|-150.22| +2450835|48944|2450880|4933|1777706|6926|4099|4933|1777706|6926|4099|2|99|18|3|12608|273|1062|94|40.51|88.71|41.69|4419.88|3918.86|3807.94|8338.74|117.56|0.00|4002.52|3918.86|4036.42|7921.38|8038.94|110.92| +2450835|77578|2450883|84639|596338|1637|26536|84639|596338|1637|26536|4|104|20|5|10639|38|1063|15|80.44|230.86|83.10|2216.40|1246.50|1206.60|3462.90|0.00|0.00|0.00|1246.50|1246.50|1246.50|1246.50|39.90| +2450835|77578|2450841|84639|596338|1637|26536|84639|596338|1637|26536|4|9|20|2|6562|46|1063|83|47.90|84.78|72.91|985.21|6051.53|3975.70|7036.74|60.51|0.00|914.66|6051.53|6112.04|6966.19|7026.70|2075.83| +2450835|77578|2450868|84639|596338|1637|26536|84639|596338|1637|26536|4|90|20|3|7546|284|1063|18|82.60|199.89|83.95|2086.92|1511.10|1486.80|3598.02|45.33|0.00|287.82|1511.10|1556.43|1798.92|1844.25|24.30| +2450835|77578|2450922|84639|596338|1637|26536|84639|596338|1637|26536|4|27|3|2|17473|63|1063|92|14.09|18.45|3.69|1357.92|339.48|1296.28|1697.40|3.39|0.00|152.72|339.48|342.87|492.20|495.59|-956.80| +2450835|77578|2450892|84639|596338|1637|26536|84639|596338|1637|26536|4|24|3|2|1954|141|1063|46|98.34|173.07|43.26|5971.26|1989.96|4523.64|7961.22|119.39|0.00|1592.06|1989.96|2109.35|3582.02|3701.41|-2533.68| +2450835|632|2450885|36995|1056055|362|669|36995|1056055|362|669|4|21|1|3|14908|134|1064|38|3.69|5.49|0.65|183.92|24.70|140.22|208.62|0.48|19.26|66.50|5.44|5.92|71.94|72.42|-134.78| +2450835|632|2450859|36995|1056055|362|669|36995|1056055|362|669|4|36|8|3|17870|243|1064|50|31.14|57.60|49.53|403.50|2476.50|1557.00|2880.00|49.53|0.00|1209.50|2476.50|2526.03|3686.00|3735.53|919.50| +2450835|632|2450914|36995|1056055|362|669|36995|1056055|362|669|4|36|16|1|9481|144|1064|67|6.74|14.22|6.25|533.99|418.75|451.58|952.74|4.18|0.00|66.33|418.75|422.93|485.08|489.26|-32.83| +2450835|632|2450860|36995|1056055|362|669|36995|1056055|362|669|4|8|14|1|13045|80|1064|49|58.25|58.83|37.65|1037.82|1844.85|2854.25|2882.67|11.80|1697.26|1008.91|147.59|159.39|1156.50|1168.30|-2706.66| +2450835|632|2450867|36995|1056055|362|669|36995|1056055|362|669|4|57|8|3|4057|198|1064|55|35.08|36.13|30.71|298.10|1689.05|1929.40|1987.15|135.12|0.00|575.85|1689.05|1824.17|2264.90|2400.02|-240.35| +2450835|632|2450904|36995|1056055|362|669|36995|1056055|362|669|4|19|3|3|14281|243|1064|28|19.92|22.90|6.64|455.28|185.92|557.76|641.20|1.11|74.36|121.80|111.56|112.67|233.36|234.47|-446.20| +2450835|632|2450885|36995|1056055|362|669|36995|1056055|362|669|4|107|13|1|14168|97|1064|7|89.28|221.41|28.78|1348.41|201.46|624.96|1549.87|16.11|0.00|154.98|201.46|217.57|356.44|372.55|-423.50| +2450835|632|2450853|36995|1056055|362|669|36995|1056055|362|669|4|46|5|2|10910|114|1064|89|29.58|30.17|5.43|2201.86|483.27|2632.62|2685.13|9.66|0.00|1207.73|483.27|492.93|1691.00|1700.66|-2149.35| +2450835|632|2450848|36995|1056055|362|669|36995|1056055|362|669|4|80|1|5|2455|158|1064|43|11.02|14.65|2.19|535.78|94.17|473.86|629.95|2.82|0.00|258.00|94.17|96.99|352.17|354.99|-379.69| +2450835|632|2450854|36995|1056055|362|669|36995|1056055|362|669|4|34|17|1|13987|217|1064|90|66.68|133.36|109.35|2160.90|9841.50|6001.20|12002.40|295.24|0.00|2160.00|9841.50|10136.74|12001.50|12296.74|3840.30| +2450835|632|2450901|36995|1056055|362|669|36995|1056055|362|669|4|54|15|5|16808|138|1064|77|29.79|50.64|46.58|312.62|3586.66|2293.83|3899.28|0.00|0.00|701.47|3586.66|3586.66|4288.13|4288.13|1292.83| +2450835|39806|2450855|82074|771849|3671|30646|82074|771849|3671|30646|1|11|5|3|64|44|1065|97|23.51|24.21|17.43|657.66|1690.71|2280.47|2348.37|0.00|0.00|938.96|1690.71|1690.71|2629.67|2629.67|-589.76| +2450835|39806|2450915|82074|771849|3671|30646|82074|771849|3671|30646|1|71|4|3|13132|237|1065|62|40.52|108.18|84.38|1475.60|5231.56|2512.24|6707.16|169.50|3348.19|804.76|1883.37|2052.87|2688.13|2857.63|-628.87| +2450835|39806|2450894|82074|771849|3671|30646|82074|771849|3671|30646|1|64|9|5|4012|142|1065|28|11.46|18.22|8.01|285.88|224.28|320.88|510.16|17.94|0.00|224.28|224.28|242.22|448.56|466.50|-96.60| +2450835||||771849||30646|82074|771849|||1|54||2|3589||1065||15.43|44.90||395.20|502.80||898.00||0.00|215.40||502.80|718.20|718.20|194.20| +2450835|39806|2450920|82074|771849|3671|30646|82074|771849|3671|30646|1|67|17|2|5366|290|1065|92|3.93|9.78|8.99|72.68|827.08|361.56|899.76|33.08|0.00|170.20|827.08|860.16|997.28|1030.36|465.52| +2450835|39806|2450893|82074|771849|3671|30646|82074|771849|3671|30646|1|79|19|5|9070|80|1065|46|94.96|156.68|148.84|360.64|6846.64|4368.16|7207.28|205.39|0.00|2378.20|6846.64|7052.03|9224.84|9430.23|2478.48| +2450835|39806|2450874|82074|771849|3671|30646|82074|771849|3671|30646|1|47|12|1|17092|239|1065|26|61.36|161.99|113.39|1263.60|2948.14|1595.36|4211.74|29.48|0.00|210.34|2948.14|2977.62|3158.48|3187.96|1352.78| +2450835|39806|2450837|82074|771849|3671|30646|82074|771849|3671|30646|1|37|4|1|11998|117|1065|50|49.20|117.58|1.17|5820.50|58.50|2460.00|5879.00|5.26|0.00|1998.50|58.50|63.76|2057.00|2062.26|-2401.50| +2450835|39806|2450921|82074|771849|3671|30646|82074|771849|3671|30646|1|93|16|5|15356|26|1065|84|93.24|158.50|136.31|1863.96|11450.04|7832.16|13314.00|458.00|0.00|931.56|11450.04|11908.04|12381.60|12839.60|3617.88| +2450835|39806|2450895|82074|771849|3671|30646|82074|771849|3671|30646|1|107|4|2|9154|138|1065|54|70.58|163.74|91.69|3890.70|4951.26|3811.32|8841.96|396.10|0.00|4332.42|4951.26|5347.36|9283.68|9679.78|1139.94| +2450835|39806|2450874|82074|771849|3671|30646|82074|771849|3671|30646|1|106|1|2|4705|219|1065|85|14.70|43.36|27.31|1364.25|2321.35|1249.50|3685.60|162.49|0.00|442.00|2321.35|2483.84|2763.35|2925.84|1071.85| +2450835|61914|2450903|47904|987188|5672|23756|47904|987188|5672|23756|1|98|10|2|8722|15|1066|77|12.59|27.94|13.69|1097.25|1054.13|969.43|2151.38|31.62|0.00|903.21|1054.13|1085.75|1957.34|1988.96|84.70| +2450835|61914|2450853|47904|987188|5672|23756|47904|987188|5672|23756|1|29|2|2|16136|148|1066|97|66.28|112.67|1.12|10820.35|108.64|6429.16|10928.99|6.51|0.00|873.97|108.64|115.15|982.61|989.12|-6320.52| +2450835|61914|2450884|47904|987188|5672|23756|47904|987188|5672|23756|1|86|9|5|1340|163|1066|58|43.45|117.74|76.53|2390.18|4438.74|2520.10|6828.92|355.09|0.00|682.66|4438.74|4793.83|5121.40|5476.49|1918.64| +2450835|61914|2450853|47904|987188|5672|23756|47904|987188|5672|23756|1|94|18|4|9973|231|1066|33|55.98|125.39|28.83|3186.48|951.39|1847.34|4137.87|9.98|808.68|1448.04|142.71|152.69|1590.75|1600.73|-1704.63| +2450835|61914|2450880|47904|987188|5672|23756|47904|987188|5672|23756|1|102|13|5|7450|192|1066|5|57.62|88.15|11.45|383.50|57.25|288.10|440.75|0.61|46.94|211.55|10.31|10.92|221.86|222.47|-277.79| +2450835|61914|2450901|47904|987188|5672|23756|47904|987188|5672|23756|1|92|3|3|11617|111|1066|94|64.68|174.63|10.47|15431.04|984.18|6079.92|16415.22|19.68|0.00|328.06|984.18|1003.86|1312.24|1331.92|-5095.74| +2450835|61914|2450901|47904|987188|5672|23756|47904|987188|5672|23756|1|17|1|3|13720|233|1066|83|80.73|83.15|53.21|2485.02|4416.43|6700.59|6901.45|132.49|0.00|3173.92|4416.43|4548.92|7590.35|7722.84|-2284.16| +2450835|61914|2450894|47904|987188|5672|23756|47904|987188|5672|23756|1|89|15|5|7006|229|1066|41|51.63|138.88|84.71|2220.97|3473.11|2116.83|5694.08|138.92|0.00|227.55|3473.11|3612.03|3700.66|3839.58|1356.28| +2450835|61914|2450850|47904|987188|5672|23756|47904|987188|5672|23756|1|81|11|1|14347|39|1066|82|56.38|115.57|33.51|6728.92|2747.82|4623.16|9476.74|192.34|0.00|1989.32|2747.82|2940.16|4737.14|4929.48|-1875.34| +2450835|61914|2450855|47904|987188|5672|23756|47904|987188|5672|23756|1|96|18|4|12592|229|1066|63|94.51|110.57|99.51|696.78|6269.13|5954.13|6965.91|125.38|0.00|2785.86|6269.13|6394.51|9054.99|9180.37|315.00| +2450835|61914|2450853|47904|987188|5672|23756|47904|987188|5672|23756|1|62|11|1|6757|113|1066|33|3.99|7.90|0.55|242.55|18.15|131.67|260.70|0.89|0.18|62.37|17.97|18.86|80.34|81.23|-113.70| +2450835|61914|2450913|47904|987188|5672|23756|47904|987188|5672|23756|1|19|5|2|11035|11|1066|12|6.82|15.61|7.02|103.08|84.24|81.84|187.32|2.56|52.22|48.60|32.02|34.58|80.62|83.18|-49.82| +2450835|68721|2450862|81733|56110|6487|32064|81733|56110|6487|32064|4|48|6|1|5252|71|1067|21|68.98|74.49|17.13|1204.56|359.73|1448.58|1564.29|3.59|0.00|281.40|359.73|363.32|641.13|644.72|-1088.85| +2450835|68721|2450912|81733|56110|6487|32064|81733|56110|6487|32064|4|85|16|5|8912|296|1067|17|4.43|6.60|6.33|4.59|107.61|75.31|112.20|9.68|0.00|4.42|107.61|117.29|112.03|121.71|32.30| +2450835|68721|2450899|81733|56110|6487|32064|81733|56110|6487|32064|4|75|3|2|16801|173|1067|23|96.31|209.95|178.45|724.50|4104.35|2215.13|4828.85|29.55|1149.21|96.37|2955.14|2984.69|3051.51|3081.06|740.01| +||||56110|6487|32064|81733|56110|6487|32064|4|69|||7618||1067||||70.27|4280.70||2289.90|||0.00|||2297.82|3258.00|3447.72|-181.80| +2450835|68721|2450903|81733|56110|6487|32064|81733|56110|6487|32064|4|73|7|5|1052|214|1067|79|66.29|119.98|44.39|5971.61|3506.81|5236.91|9478.42|70.13|0.00|568.01|3506.81|3576.94|4074.82|4144.95|-1730.10| +2450835|68721|2450884|81733|56110|6487|32064|81733|56110|6487|32064|4|48|2|4|7279|1|1067|43|11.45|14.88|6.54|358.62|281.22|492.35|639.84|14.06|0.00|287.67|281.22|295.28|568.89|582.95|-211.13| +2450835|68721|2450850|81733|56110|6487|32064|81733|56110|6487|32064|4|43|12|1|3700|245|1067|73|87.64|261.16|154.08|7816.84|11247.84|6397.72|19064.68|449.91|0.00|6481.67|11247.84|11697.75|17729.51|18179.42|4850.12| +2450835|68721|2450897|81733|56110|6487|32064|81733|56110|6487|32064|4|13|16|3|331|105|1067|69|25.92|55.20|50.78|304.98|3503.82|1788.48|3808.80|0.00|0.00|342.24|3503.82|3503.82|3846.06|3846.06|1715.34| +2450835|68721|2450888|81733|56110|6487|32064|81733|56110|6487|32064|4|101|16|3|8800|100|1067|83|11.96|21.64|7.35|1186.07|610.05|992.68|1796.12|6.10|0.00|89.64|610.05|616.15|699.69|705.79|-382.63| +2450835|68721|2450897|81733|56110|6487|32064|81733|56110|6487|32064|4|63|20|1|529|160|1067|9|49.85|95.21|54.26|368.55|488.34|448.65|856.89|43.95|0.00|197.01|488.34|532.29|685.35|729.30|39.69| +2450835|68721|2450920|81733|56110|6487|32064|81733|56110|6487|32064|4|49|9|1|8371|58|1067|25|49.30|139.02|136.23|69.75|3405.75|1232.50|3475.50|238.40|0.00|1216.25|3405.75|3644.15|4622.00|4860.40|2173.25| +2450835|68721|2450842|81733|56110|6487|32064|81733|56110|6487|32064|4|65|19|1|15613|175|1067|35|25.77|53.08|46.17|241.85|1615.95|901.95|1857.80|129.27|0.00|371.35|1615.95|1745.22|1987.30|2116.57|714.00| +2450835|68721|2450890|81733|56110|6487|32064|81733|56110|6487|32064|4|95|11|3|11809|77|1067|48|90.47|143.84|115.07|1380.96|5523.36|4342.56|6904.32|331.40|0.00|2830.56|5523.36|5854.76|8353.92|8685.32|1180.80| +2450835|68721|2450882|81733|56110|6487|32064|81733|56110|6487|32064|4|15|16|5|7435|274|1067|89|9.33|16.79|6.04|956.75|537.56|830.37|1494.31|43.00|0.00|522.43|537.56|580.56|1059.99|1102.99|-292.81| +||2450900|||1116|||||32636|4|61||5|7969|219|1068||94.45|263.51||||4628.05||||387.10|||7101.08|7705.33|2085.93| +2450835|70910|2450870|95676|605596|1116|32636|95676|605596|1116|32636|4|103|5|3|3469|212|1068|75|43.89|66.71|0.66|4953.75|49.50|3291.75|5003.25|0.80|36.13|1900.50|13.37|14.17|1913.87|1914.67|-3278.38| +2450835|70910|2450850|95676|605596|1116|32636|95676|605596|1116|32636|4|68|2|3|16411|252|1068|89|60.43|117.23|76.19|3652.56|6780.91|5378.27|10433.47|0.00|0.00|3859.93|6780.91|6780.91|10640.84|10640.84|1402.64| +2450835|70910|2450924|95676|605596|1116|32636|95676|605596|1116|32636|4|5|1|3|5872|105|1068|91|33.36|36.02|11.88|2196.74|1081.08|3035.76|3277.82|97.29|0.00|1375.92|1081.08|1178.37|2457.00|2554.29|-1954.68| +2450835|70910|2450896|95676|605596|1116|32636|95676|605596|1116|32636|4|5|10|5|8402|55|1068|65|26.92|74.29|31.94|2752.75|2076.10|1749.80|4828.85|14.94|1889.25|1303.25|186.85|201.79|1490.10|1505.04|-1562.95| +2450835|70910|2450862|95676|605596|1116|32636|95676|605596|1116|32636|4|66|8|1|13160|1|1068|33|37.01|40.71|7.73|1088.34|255.09|1221.33|1343.43|7.65|0.00|308.88|255.09|262.74|563.97|571.62|-966.24| +2450835|70910|2450872|95676|605596|1116|32636|95676|605596|1116|32636|4|26|2|2|5596|110|1068|93|36.87|64.15|11.54|4892.73|1073.22|3428.91|5965.95|75.12|0.00|835.14|1073.22|1148.34|1908.36|1983.48|-2355.69| +2450835|58141|2450842|25259|1782980|6654|23575|63275|261770|4291|49212|1|92|19|2|9398|110|1069|2|61.65|154.74|86.65|136.18|173.30|123.30|309.48|15.59|0.00|30.94|173.30|188.89|204.24|219.83|50.00| +2450835|58141|2450895|25259|1782980|6654|23575|63275|261770|4291|49212|1|21|7|4|10132|146|1069|59|28.93|34.71|6.24|1679.73|368.16|1706.87|2047.89|18.40|0.00|818.92|368.16|386.56|1187.08|1205.48|-1338.71| +2450835|58141|2450877|25259|1782980|6654|23575|63275|261770|4291|49212|1|85|15|1|9646|81|1069|89|86.33|113.09|3.39|9763.30|301.71|7683.37|10065.01|6.15|96.54|4830.92|205.17|211.32|5036.09|5042.24|-7478.20| +2450835|58141|2450880|25259|1782980|6654|23575|63275|261770|4291|49212|1|73|19|5|13448|97|1069|10|81.43|223.11|200.79|223.20|2007.90|814.30|2231.10|60.23|0.00|1093.20|2007.90|2068.13|3101.10|3161.33|1193.60| +2450835|58141|2450859|25259|1782980|6654|23575|63275|261770|4291|49212|1|3|13|4|13972|221|1069|96|42.27|81.58|62.00|1879.68|5952.00|4057.92|7831.68|408.30|119.04|704.64|5832.96|6241.26|6537.60|6945.90|1775.04| +2450835|58141|2450859|25259|1782980|6654|23575|63275|261770|4291|49212|1|99|15|4|1544|146|1069|83|32.45|65.87|61.91|328.68|5138.53|2693.35|5467.21|146.44|256.92|1913.15|4881.61|5028.05|6794.76|6941.20|2188.26| +2450835|58141|2450914|25259|1782980|6654|23575|63275|261770|4291|49212|1|25|14|5|1280|250|1069|16|25.45|28.24|21.74|104.00|347.84|407.20|451.84|31.30|0.00|54.08|347.84|379.14|401.92|433.22|-59.36| +2450835|58141|2450879|25259|1782980|6654|23575|63275|261770|4291|49212|1|37|6|2|8510|119|1069|41|19.57|25.63|12.55|536.28|514.55|802.37|1050.83|5.14|0.00|62.73|514.55|519.69|577.28|582.42|-287.82| +2450835|58141|2450862|25259|1782980|6654|23575|63275|261770|4291|49212|1|4|19|5|16598|274|1069|97|64.00|69.76|0.00|6766.72|0.00|6208.00|6766.72|0.00|0.00|1014.62|0.00|0.00|1014.62|1014.62|-6208.00| +2450835|43545|2450864|49220|217873|479|6685|49220|217873|479|6685|2|7|7|2|11347|270|1070|99|59.82|144.16|109.56|3425.40|10846.44|5922.18|14271.84|0.00|0.00|2568.06|10846.44|10846.44|13414.50|13414.50|4924.26| +2450835|43545|2450846|49220|217873|479|6685|49220|217873|479|6685|2|66|15|4|15265|202|1070|15|26.27|77.23|50.97|393.90|764.55|394.05|1158.45|61.16|0.00|370.65|764.55|825.71|1135.20|1196.36|370.50| +2450835|43545|2450880|49220|217873|479|6685|49220|217873|479|6685|2|33|16|1|4765|181|1070|84|60.61|150.91|57.34|7859.88|4816.56|5091.24|12676.44|192.66|0.00|4183.20|4816.56|5009.22|8999.76|9192.42|-274.68| +2450835|43545|2450855|49220|217873|479|6685|49220|217873|479|6685|2|13|7|3|5275|50|1070|93|83.36|111.70|25.69|7998.93|2389.17|7752.48|10388.10|97.47|1170.69|2908.11|1218.48|1315.95|4126.59|4224.06|-6534.00| +2450835|8640|2450848|6338|526703|6621|25386|44630|1917669|3187|44705|4|4|13|4|9808|60|1071|16|81.49|181.72|50.88|2093.44|814.08|1303.84|2907.52|73.26|0.00|1221.12|814.08|887.34|2035.20|2108.46|-489.76| +2450835|8640|2450847|6338|526703|6621|25386|44630|1917669|3187|44705|4|52|15|1|226|111|1071|29|66.10|127.57|95.67|925.10|2774.43|1916.90|3699.53|16.64|1109.77|554.77|1664.66|1681.30|2219.43|2236.07|-252.24| +2450835|8640|2450891|6338|526703|6621|25386|44630|1917669|3187|44705|4|46|3|2|3397|247|1071|19|84.11|92.52|66.61|492.29|1265.59|1598.09|1757.88|63.27|0.00|87.78|1265.59|1328.86|1353.37|1416.64|-332.50| +2450835|8640|2450846|6338|526703|6621|25386|44630|1917669|3187|44705|4|86|20|5|15152|180|1071|3|68.48|81.49|34.22|141.81|102.66|205.44|244.47|8.21|0.00|92.88|102.66|110.87|195.54|203.75|-102.78| +2450835|8640|2450874|6338|526703|6621|25386|44630|1917669|3187|44705|4|90|13|4|13310|153|1071|11|81.27|179.60|26.94|1679.26|296.34|893.97|1975.60|17.78|0.00|493.90|296.34|314.12|790.24|808.02|-597.63| +2450835|8640|2450853|6338|526703|6621|25386|44630|1917669|3187|44705|4|66|18|1|542|63|1071|77|86.03|217.65|156.70|4693.15|12065.90|6624.31|16759.05|965.27|0.00|8211.28|12065.90|13031.17|20277.18|21242.45|5441.59| +2450835|8640|2450837|6338|526703|6621|25386|44630|1917669|3187|44705|4|31|6|1|12106|296|1071|61|13.03|26.58|17.27|567.91|1053.47|794.83|1621.38|39.18|73.74|761.89|979.73|1018.91|1741.62|1780.80|184.90| +2450835|8640|2450866|6338|526703|6621|25386|44630|1917669|3187|44705|4|87|4|3|932|131|1071|6|96.16|140.39|56.15|505.44|336.90|576.96|842.34|8.69|192.03|252.66|144.87|153.56|397.53|406.22|-432.09| +2450835|8640|2450878|6338|526703|6621|25386|44630|1917669|3187|44705|4|96|13|1|5242|21|1071|56|28.81|65.11|28.64|2042.32|1603.84|1613.36|3646.16|80.19|0.00|947.52|1603.84|1684.03|2551.36|2631.55|-9.52| +2450835|44856|2450911|24830|572201|1172|30143|24830|572201|1172|30143|4|7|16|4|8152|225|1072|87|47.69|139.73|81.04|5106.03|7050.48|4149.03|12156.51|493.53|0.00|1215.39|7050.48|7544.01|8265.87|8759.40|2901.45| +2450835|44856|2450902|24830|572201|1172|30143|24830|572201|1172|30143|4|7|16|2|13724|256|1072|87|22.91|54.06|10.81|3762.75|940.47|1993.17|4703.22|56.42|0.00|1786.98|940.47|996.89|2727.45|2783.87|-1052.70| +2450835|44856|2450874|24830|572201|1172|30143|24830|572201|1172|30143|4|105|17|5|14611|203|1072|12|50.57|145.64|97.57|576.84|1170.84|606.84|1747.68|23.41|0.00|576.72|1170.84|1194.25|1747.56|1770.97|564.00| +2450835|44856|2450869|24830|572201|1172|30143|24830|572201|1172|30143|4|76|1|2|17074|267|1072|89|24.14|65.41|22.89|3784.28|2037.21|2148.46|5821.49|122.23|0.00|1571.74|2037.21|2159.44|3608.95|3731.18|-111.25| +2450835|44856|2450840|24830|572201|1172|30143|24830|572201|1172|30143|4|76|17|4|2674|186|1072|83|68.19|192.97|164.02|2402.85|13613.66|5659.77|16016.51|680.68|0.00|7046.70|13613.66|14294.34|20660.36|21341.04|7953.89| +2450835|44856|2450846|24830|572201|1172|30143|24830|572201|1172|30143|4|71|1|5|16576|13|1072|63|69.45|72.92|72.19|45.99|4547.97|4375.35|4593.96|24.10|2137.54|1010.52|2410.43|2434.53|3420.95|3445.05|-1964.92| +2450835|44856|2450862|24830|572201|1172|30143|24830|572201|1172|30143|4|15|19|5|434|82|1072|46|51.29|84.62|2.53|3776.14|116.38|2359.34|3892.52|8.14|0.00|1556.64|116.38|124.52|1673.02|1681.16|-2242.96| +2450835|44856|2450846|24830|572201|1172|30143|24830|572201|1172|30143|4|18|15|4|8728|160|1072|81|55.22|134.18|71.11|5108.67|5759.91|4472.82|10868.58|0.00|2246.36|1304.10|3513.55|3513.55|4817.65|4817.65|-959.27| +2450835|44856|2450910|24830|572201|1172|30143|24830|572201|1172|30143|4|20|5|1|12265|41|1072|58|36.60|105.77|70.86|2024.78|4109.88|2122.80|6134.66|74.79|3041.31|1533.52|1068.57|1143.36|2602.09|2676.88|-1054.23| +2450835|44856|2450856|24830|572201|1172|30143|24830|572201|1172|30143|4|96|10|4|14140|85|1072|92|24.02|48.04|14.89|3049.80|1369.88|2209.84|4419.68|41.09|0.00|1988.12|1369.88|1410.97|3358.00|3399.09|-839.96| +2450835|44856|2450866|24830|572201|1172|30143|24830|572201|1172|30143|4|12|5|3|9685|91|1072|79|76.61|92.69|17.61|5931.32|1391.19|6052.19|7322.51|125.20|0.00|1171.57|1391.19|1516.39|2562.76|2687.96|-4661.00| +2450835|44856|2450890|24830|572201|1172|30143|24830|572201|1172|30143|4|63|1|3|15626|67|1072|17|61.74|79.64|53.35|446.93|906.95|1049.58|1353.88|63.48|0.00|514.42|906.95|970.43|1421.37|1484.85|-142.63| +2450836|70315|2450846|63864|1624496|2281|9713|63864|1624496|2281|9713|1|4|13|1|4724|248|1073|77|53.22|109.63|46.04|4896.43|3545.08|4097.94|8441.51|283.60|0.00|168.63|3545.08|3828.68|3713.71|3997.31|-552.86| +2450836||2450908|63864|1624496|2281|||1624496|2281|9713||48||5|634|169|1073|48||127.38||||2982.72||9.56||2995.68|956.03||3951.71|3961.27|-2026.69| +2450836|70315|2450896|63864|1624496|2281|9713|63864|1624496|2281|9713|1|73|17|5|10228|144|1073|5|27.59|64.56|35.50|145.30|177.50|137.95|322.80|7.10|0.00|158.15|177.50|184.60|335.65|342.75|39.55| +2450836|70315|2450840|63864|1624496|2281|9713|63864|1624496|2281|9713|1|89|11|4|4069|155|1073|79|95.37|254.63|0.00|20115.77|0.00|7534.23|20115.77|0.00|0.00|8649.71|0.00|0.00|8649.71|8649.71|-7534.23| +2450836|70315|2450873|63864|1624496|2281|9713|63864|1624496|2281|9713|1|93|2|1|7144|107|1073|4|19.59|26.25|23.10|12.60|92.40|78.36|105.00|4.62|0.00|14.68|92.40|97.02|107.08|111.70|14.04| +2450836|70315|2450880|63864|1624496|2281|9713|63864|1624496|2281|9713|1|88|2|1|11686|231|1073|50|35.48|71.31|9.98|3066.50|499.00|1774.00|3565.50|6.58|334.33|784.00|164.67|171.25|948.67|955.25|-1609.33| +2450836|70315|2450865|63864|1624496|2281|9713|63864|1624496|2281|9713|1|17|17|3|9382|43|1073|37|76.07|201.58|104.82|3580.12|3878.34|2814.59|7458.46|349.05|0.00|1044.14|3878.34|4227.39|4922.48|5271.53|1063.75| +2450836|70315|2450858|63864|1624496|2281|9713|63864|1624496|2281|9713|1|28|20|5|10615|25|1073|55|63.52|144.19|25.95|6503.20|1427.25|3493.60|7930.45|14.27|0.00|2854.50|1427.25|1441.52|4281.75|4296.02|-2066.35| +2450836|70315|2450866|63864|1624496|2281|9713|63864|1624496|2281|9713|1|66|1|1|10267|48|1073|86|73.69|100.95|40.38|5209.02|3472.68|6337.34|8681.70|69.45|0.00|433.44|3472.68|3542.13|3906.12|3975.57|-2864.66| +2450836|70315|2450907|63864|1624496|2281|9713|63864|1624496|2281|9713|1|41|14|2|7850|45|1073|89|64.95|180.56|7.22|15427.26|642.58|5780.55|16069.84|7.71|385.54|4981.33|257.04|264.75|5238.37|5246.08|-5523.51| +2450836|70315|2450904|63864|1624496|2281|9713|63864|1624496|2281|9713|1|16|1|3|10346|27|1073|38|2.06|5.29|2.59|102.60|98.42|78.28|201.02|6.88|0.00|61.94|98.42|105.30|160.36|167.24|20.14| +2450836|70315|2450907|63864|1624496|2281|9713|63864|1624496|2281|9713|1|99|1|1|11894|296|1073|33|73.83|86.38|5.18|2679.60|170.94|2436.39|2850.54|2.76|124.78|341.88|46.16|48.92|388.04|390.80|-2390.23| +2450836|46713|2450842|99127|1869610|4540|33870|99127|1869610|4540|33870|2|37|19|1|3760|239|1074|44|59.22|170.55|136.44|1500.84|6003.36|2605.68|7504.20|0.00|0.00|1575.64|6003.36|6003.36|7579.00|7579.00|3397.68| +2450836|46713|2450870|99127|1869610|4540|33870|99127|1869610|4540|33870|2|34|17|2|12278|133|1074|24|49.16|113.55|82.89|735.84|1989.36|1179.84|2725.20|15.11|477.44|354.24|1511.92|1527.03|1866.16|1881.27|332.08| +2450836|46713|2450917|99127|1869610|4540|33870|99127|1869610|4540|33870|2|93|6|3|13657|109|1074|27|90.24|101.06|12.12|2401.38|327.24|2436.48|2728.62|22.90|0.00|1091.34|327.24|350.14|1418.58|1441.48|-2109.24| +2450836|46713|2450852|99127|1869610|4540|33870|99127|1869610|4540|33870|2|103|5|1|3772|276|1074|75|30.10|47.25|24.09|1737.00|1806.75|2257.50|3543.75|108.40|0.00|921.00|1806.75|1915.15|2727.75|2836.15|-450.75| +2450836|46713|2450879|99127|1869610|4540|33870|99127|1869610|4540|33870|2|63|14|2|11668|133|1074|63|42.98|83.81|41.06|2693.25|2586.78|2707.74|5280.03|206.94|0.00|2217.60|2586.78|2793.72|4804.38|5011.32|-120.96| +2450836|46713|2450908|99127|1869610|4540|33870|99127|1869610|4540|33870|2|24|8|4|668|3|1074|33|62.06|155.15|125.67|972.84|4147.11|2047.98|5119.95|331.76|0.00|1126.29|4147.11|4478.87|5273.40|5605.16|2099.13| +2450836|46713|2450846|99127|1869610|4540|33870|99127|1869610|4540|33870|2|62|17|4|14887|165|1074|29|45.08|93.31|0.00|2705.99|0.00|1307.32|2705.99|0.00|0.00|1217.42|0.00|0.00|1217.42|1217.42|-1307.32| +2450836|46713|2450844|99127|1869610|4540|33870|99127|1869610|4540|33870|2|101|3|1|16615|247|1074|59|81.27|121.09|30.27|5358.38|1785.93|4794.93|7144.31|97.51|160.73|2000.10|1625.20|1722.71|3625.30|3722.81|-3169.73| +2450836|46713|2450840|99127|1869610|4540|33870|99127|1869610|4540|33870|2|76|8|3|55|296|1074|48|97.93|201.73|80.69|5809.92|3873.12|4700.64|9683.04|0.00|0.00|96.48|3873.12|3873.12|3969.60|3969.60|-827.52| +2450836|46713|2450869|99127|1869610|4540|33870|99127|1869610|4540|33870|2|101|4|3|6050|211|1074|12|4.30|8.42|5.47|35.40|65.64|51.60|101.04|0.00|0.00|33.24|65.64|65.64|98.88|98.88|14.04| +2450836|46713|2450876|99127|1869610|4540|33870|99127|1869610|4540|33870|2|16|19|3|1334|191|1074|7|98.26|261.37|209.09|365.96|1463.63|687.82|1829.59|102.45|0.00|91.42|1463.63|1566.08|1555.05|1657.50|775.81| +2450836|56702|2450900|43604|727574|4344|25867|43604|727574|4344|25867|2|55|8|1|16570|259|1075|54|51.02|148.97|92.36|3056.94|4987.44|2755.08|8044.38|299.24|0.00|3056.40|4987.44|5286.68|8043.84|8343.08|2232.36| +2450836|56702|2450914|43604|727574|4344|25867|43604|727574|4344|25867|2|65|19|3|15326|256|1075|58|11.83|25.55|7.40|1052.70|429.20|686.14|1481.90|4.29|0.00|459.36|429.20|433.49|888.56|892.85|-256.94| +2450836|56702|2450891|43604|727574|4344|25867|43604|727574|4344|25867|2|54|17|5|5278|244|1075|87|16.72|26.91|15.60|983.97|1357.20|1454.64|2341.17|67.86|0.00|1170.15|1357.20|1425.06|2527.35|2595.21|-97.44| +2450836|56702|2450851|43604|727574|4344|25867|43604|727574|4344|25867|2|97|2|4|4496|222|1075|31|11.48|26.97|21.57|167.40|668.67|355.88|836.07|46.80|0.00|66.65|668.67|715.47|735.32|782.12|312.79| +2450836|56702|2450879|43604|727574|4344|25867|43604|727574|4344|25867|2|100|16|5|4555|108|1075|25|90.12|230.70|126.88|2595.50|3172.00|2253.00|5767.50|95.16|0.00|2133.75|3172.00|3267.16|5305.75|5400.91|919.00| +2450836|56702|2450873|43604|727574|4344|25867|43604|727574|4344|25867|2|84|7|1|5122|5|1075|21|7.91|12.33|10.97|28.56|230.37|166.11|258.93|6.91|0.00|100.80|230.37|237.28|331.17|338.08|64.26| +2450836|56702|2450855|43604|727574|4344|25867|43604|727574|4344|25867|2|48|15|4|16729|14|1075|34|83.59|101.14|35.39|2235.50|1203.26|2842.06|3438.76|38.02|252.68|928.20|950.58|988.60|1878.78|1916.80|-1891.48| +2450836|56702|2450896|43604|727574|4344|25867|43604|727574|4344|25867|2|3|17|3|10220|96|1075|10|89.64|113.84|12.52|1013.20|125.20|896.40|1138.40|8.76|0.00|34.10|125.20|133.96|159.30|168.06|-771.20| +2450836|58568|2450847|3462|1371601|6940|10279|3462|1371601|6940|10279|2|9|20|5|1891|26|1076|33|73.99|175.35|54.35|3993.00|1793.55|2441.67|5786.55|89.67|0.00|520.74|1793.55|1883.22|2314.29|2403.96|-648.12| +2450836|58568|2450855|3462|1371601|6940|10279|3462|1371601|6940|10279|2|66|3|4|15439|72|1076|63|95.90|111.24|71.19|2523.15|4484.97|6041.70|7008.12|17.93|2690.98|1471.68|1793.99|1811.92|3265.67|3283.60|-4247.71| +2450836|58568|2450841|3462|1371601|6940|10279|3462|1371601|6940|10279|2|88|5|5|17827|39|1076|67|37.82|38.57|25.84|852.91|1731.28|2533.94|2584.19|51.41|1159.95|852.24|571.33|622.74|1423.57|1474.98|-1962.61| +2450836|58568|2450890|3462|1371601|6940|10279|3462|1371601|6940|10279|2|46|1|1|7142|249|1076|89|84.73|105.91|90.02|1414.21|8011.78|7540.97|9425.99|80.11|0.00|1224.64|8011.78|8091.89|9236.42|9316.53|470.81| +2450836|58568|2450892|3462|1371601|6940|10279|3462|1371601|6940|10279|2|18|6|4|2887|42|1076|88|64.12|104.51|97.19|644.16|8552.72|5642.56|9196.88|171.05|0.00|1930.72|8552.72|8723.77|10483.44|10654.49|2910.16| +2450836|58568|2450906|3462|1371601|6940|10279|3462|1371601|6940|10279|2|6|3|4|8570|77|1076|53|14.50|38.57|11.57|1431.00|613.21|768.50|2044.21|0.00|0.00|224.72|613.21|613.21|837.93|837.93|-155.29| +2450836|25413|2450858|12210|420138|6913|841|12210|420138|6913|841|4|26|14|5|17527|192|1077|59|17.02|47.65|3.33|2614.88|196.47|1004.18|2811.35|9.82|0.00|252.52|196.47|206.29|448.99|458.81|-807.71| +2450836|25413|2450855|12210|420138|6913|841|12210|420138|6913|841|4|104|3|2|8623|215|1077|49|40.98|49.99|20.49|1445.50|1004.01|2008.02|2449.51|70.28|0.00|171.01|1004.01|1074.29|1175.02|1245.30|-1004.01| +2450836|25413|2450884|12210|420138|6913|841|12210|420138|6913|841|4|18|10|2|10750|261|1077|10|52.50|146.47|120.10|263.70|1201.00|525.00|1464.70|108.09|0.00|424.70|1201.00|1309.09|1625.70|1733.79|676.00| +2450836|25413|2450910|12210|420138|6913|841|12210|420138|6913|841|4|106|13|1|6901|253|1077|87|73.90|102.72|102.72|0.00|8936.64|6429.30|8936.64|714.93|0.00|2859.69|8936.64|9651.57|11796.33|12511.26|2507.34| +2450836|43530|2450866|97141|1409110|5079|17848|97141|1409110|5079|17848|4|96|14|5|8203|67|1078|3|15.26|23.19|16.92|18.81|50.76|45.78|69.57|1.01|0.00|21.54|50.76|51.77|72.30|73.31|4.98| +2450836|43530|2450867|97141|1409110|5079|17848|97141|1409110|5079|17848|4|22|14|4|6848|172|1078|58|77.33|102.84|27.76|4354.64|1610.08|4485.14|5964.72|112.70|0.00|1132.74|1610.08|1722.78|2742.82|2855.52|-2875.06| +2450836|43530|2450870|97141|1409110|5079|17848|97141|1409110|5079|17848|4|69|3|2|11830|180|1078|95|64.42|77.94|24.94|5035.00|2369.30|6119.90|7404.30|118.46|0.00|2812.95|2369.30|2487.76|5182.25|5300.71|-3750.60| +2450836|43530|2450868|97141|1409110|5079|17848|97141|1409110|5079|17848|4|53|19|3|17359|121|1078|11|58.70|157.31|26.74|1436.27|294.14|645.70|1730.41|0.00|250.01|744.04|44.13|44.13|788.17|788.17|-601.57| +2450836|75157|2450856|4101|1004026|5551|13476|23619|1152973|4898|9623|2|28|19|2|8588|235|1079|88|89.54|146.84|92.50|4781.92|8140.00|7879.52|12921.92|651.20|0.00|2067.12|8140.00|8791.20|10207.12|10858.32|260.48| +2450836|75157|2450875|4101|1004026|5551|13476|23619|1152973|4898|9623|2|31|11|3|22|30|1079|24|3.56|5.23|4.96|6.48|119.04|85.44|125.52|4.76|0.00|55.20|119.04|123.80|174.24|179.00|33.60| +2450836|75157|2450881|4101|1004026|5551|13476|23619|1152973|4898|9623|2|54|9|3|5245|101|1079|81|42.37|47.03|18.34|2323.89|1485.54|3431.97|3809.43|0.00|0.00|380.70|1485.54|1485.54|1866.24|1866.24|-1946.43| +2450836|75157|2450871|4101|1004026|5551|13476|23619|1152973|4898|9623|2|17|16|5|2750|101|1079|12|67.31|176.35|17.63|1904.64|211.56|807.72|2116.20|19.04|0.00|761.76|211.56|230.60|973.32|992.36|-596.16| +2450836|75157|2450859|4101|1004026|5551|13476|23619|1152973|4898|9623|2|14|7|5|10808|93|1079|77|97.82|281.72|92.96|14534.52|7157.92|7532.14|21692.44|0.00|0.00|6941.55|7157.92|7157.92|14099.47|14099.47|-374.22| +2450836|75157|2450906|4101|1004026|5551|13476|23619|1152973|4898|9623|2|101|18|3|11524|148|1079|57|90.32|229.41|142.23|4969.26|8107.11|5148.24|13076.37|133.76|5431.76|4707.06|2675.35|2809.11|7382.41|7516.17|-2472.89| +2450836|75157|2450884|4101|1004026|5551|13476|23619|1152973|4898|9623|2|22|12|4|9640|141|1079|48|53.99|98.80|24.70|3556.80|1185.60|2591.52|4742.40|106.70|0.00|1043.04|1185.60|1292.30|2228.64|2335.34|-1405.92| +2450836|64931|2450843|28125|455482|4886|48690|28125|455482|4886|48690|4|102|11|5|3601|103|1080|49|33.47|77.98|10.13|3324.65|496.37|1640.03|3821.02|9.92|0.00|458.15|496.37|506.29|954.52|964.44|-1143.66| +2450836|64931|2450908|28125|455482|4886|48690|28125|455482|4886|48690|4|107|15|1|3607|14|1080|51|38.54|53.57|18.21|1803.36|928.71|1965.54|2732.07|65.00|0.00|437.07|928.71|993.71|1365.78|1430.78|-1036.83| +2450836|64931|2450890|28125|455482|4886|48690|28125|455482|4886|48690|4|75|18|1|6433|154|1080|55|44.36|85.17|1.70|4590.85|93.50|2439.80|4684.35|5.55|0.93|562.10|92.57|98.12|654.67|660.22|-2347.23| +2450836|64931|2450878|28125|455482|4886|48690|28125|455482|4886|48690|4|20|13|4|17653|238|1080|84|37.58|86.05|46.46|3325.56|3902.64|3156.72|7228.20|351.23|0.00|2312.52|3902.64|4253.87|6215.16|6566.39|745.92| +2450836|64931|2450914|28125|455482|4886|48690|28125|455482|4886|48690|4|9|12|3|11324|51|1080|98|79.48|79.48|68.35|1090.74|6698.30|7789.04|7789.04|334.91|0.00|3816.12|6698.30|7033.21|10514.42|10849.33|-1090.74| +2450836|64931|2450902|28125|455482|4886|48690|28125|455482|4886|48690|4|92|16|4|10024|152|1080|46|34.46|85.46|7.69|3577.42|353.74|1585.16|3931.16|24.76|0.00|550.16|353.74|378.50|903.90|928.66|-1231.42| +2450836|64931|2450920|28125|455482|4886|48690|28125|455482|4886|48690|4|86|17|1|16256|31|1080|67|65.78|184.84|158.96|1733.96|10650.32|4407.26|12384.28|213.00|0.00|0.00|10650.32|10863.32|10650.32|10863.32|6243.06| +2450836|64931|2450864|28125|455482|4886|48690|28125|455482|4886|48690|4|9|17|3|3187|259|1080|14|28.74|52.30|39.22|183.12|549.08|402.36|732.20|11.53|318.46|43.82|230.62|242.15|274.44|285.97|-171.74| +2450836|64931|2450857|28125|455482|4886|48690|28125|455482|4886|48690|4|53|1|4|6422|157|1080|72|23.34|40.14|30.90|665.28|2224.80|1680.48|2890.08|88.99|0.00|635.76|2224.80|2313.79|2860.56|2949.55|544.32| +2450836|72265|2450865|38399|1380590|1181|15325|38399|1380590|1181|15325|4|60|5|5|3578|142|1081|60|29.56|78.33|24.28|3243.00|1456.80|1773.60|4699.80|116.54|0.00|0.00|1456.80|1573.34|1456.80|1573.34|-316.80| +2450836|72265|2450902|38399|1380590|1181|15325|38399|1380590|1181|15325|4|13|4|2|13705|85|1081|92|65.68|128.07|43.54|7776.76|4005.68|6042.56|11782.44|62.48|2443.46|4594.48|1562.22|1624.70|6156.70|6219.18|-4480.34| +2450836|72265|2450901|38399|1380590|1181|15325|38399|1380590|1181|15325|4|108|2|2|9392|53|1081|97|6.65|16.02|0.80|1476.34|77.60|645.05|1553.94|3.88|0.00|713.92|77.60|81.48|791.52|795.40|-567.45| +2450836|72265|2450914|38399|1380590|1181|15325|38399|1380590|1181|15325|4|103|12|4|12136|112|1081|36|79.87|150.15|85.58|2324.52|3080.88|2875.32|5405.40|56.38|1201.54|1999.80|1879.34|1935.72|3879.14|3935.52|-995.98| +2450836|72265|2450906|38399|1380590|1181|15325|38399|1380590|1181|15325|4|11|10|3|14990|117|1081|12|30.19|50.41|35.28|181.56|423.36|362.28|604.92|38.10|0.00|6.00|423.36|461.46|429.36|467.46|61.08| +2450836|72265|2450871|38399|1380590|1181|15325|38399|1380590|1181|15325|4|16|3|3|4090|223|1081|99|88.28|148.31|44.49|10278.18|4404.51|8739.72|14682.69|132.13|0.00|2201.76|4404.51|4536.64|6606.27|6738.40|-4335.21| +2450836|72265|2450914|38399|1380590|1181|15325|38399|1380590|1181|15325|4|44|2|5|10555|279|1081|24|15.10|25.67|6.16|468.24|147.84|362.40|616.08|11.82|0.00|277.20|147.84|159.66|425.04|436.86|-214.56| +2450836|72265|2450898|38399|1380590|1181|15325|38399|1380590|1181|15325|4|4|3|2|14824|288|1081|28|27.11|65.33|4.57|1701.28|127.96|759.08|1829.24|1.27|0.00|402.36|127.96|129.23|530.32|531.59|-631.12| +2450836|72265|2450909|38399|1380590|1181|15325|38399|1380590|1181|15325|4|67|13|4|2444|194|1081|82|24.15|49.99|33.99|1312.00|2787.18|1980.30|4099.18|167.23|696.79|1475.18|2090.39|2257.62|3565.57|3732.80|110.09| +2450836|64038|2450924|53701|1351094|840|8986|53701|1351094|840|8986|2|68|13|5|16670|29|1082|74|93.42|102.76|54.46|3574.20|4030.04|6913.08|7604.24|35.06|523.90|2052.76|3506.14|3541.20|5558.90|5593.96|-3406.94| +2450836|64038|2450901|53701|1351094|840|8986|53701|1351094|840|8986|2|6|1|4|14716|142|1082|55|78.58|146.94|96.98|2747.80|5333.90|4321.90|8081.70|266.69|0.00|1777.60|5333.90|5600.59|7111.50|7378.19|1012.00| +2450836|64038|2450900|53701|1351094|840|8986|53701|1351094|840|8986|2|20|8|4|16072|242|1082|56|15.92|21.01|7.56|753.20|423.36|891.52|1176.56|29.63|0.00|70.56|423.36|452.99|493.92|523.55|-468.16| +2450836|64038|2450916|53701|1351094|840|8986|53701|1351094|840|8986|2|46|14|3|12584|183|1082|57|25.79|71.43|48.57|1303.02|2768.49|1470.03|4071.51|138.42|0.00|1098.96|2768.49|2906.91|3867.45|4005.87|1298.46| +2450836|64038|2450917|53701|1351094|840|8986|53701|1351094|840|8986|2|67|6|5|274|176|1082|63|34.90|41.18|30.06|700.56|1893.78|2198.70|2594.34|75.75|0.00|0.00|1893.78|1969.53|1893.78|1969.53|-304.92| +2450836|64038|2450924|53701|1351094|840|8986|53701|1351094|840|8986|2|73|8|1|10772|259|1082|32|32.81|87.60|53.43|1093.44|1709.76|1049.92|2803.20|13.33|1487.49|196.16|222.27|235.60|418.43|431.76|-827.65| +2450836|81596|2450908|85391|1894708|5978|15295|85391|1894708|5978|15295|4|90|8|5|4495|26|1083|73|21.71|52.75|39.56|962.87|2887.88|1584.83|3850.75|173.27|0.00|1501.61|2887.88|3061.15|4389.49|4562.76|1303.05| +2450836|81596|2450865|85391|1894708|5978|15295|85391|1894708|5978|15295|4|18|18|5|12400|116|1083|81|78.93|169.69|61.08|8797.41|4947.48|6393.33|13744.89|49.47|0.00|6322.05|4947.48|4996.95|11269.53|11319.00|-1445.85| +2450836|81596|2450846|85391|1894708|5978|15295|85391|1894708|5978|15295|4|35|7|1|3482|209|1083|62|72.13|83.67|53.54|1868.06|3319.48|4472.06|5187.54|165.97|0.00|1037.26|3319.48|3485.45|4356.74|4522.71|-1152.58| +2450836|81596|2450896|85391|1894708|5978|15295|85391|1894708|5978|15295|4|52|20|3|4444|210|1083|65|47.65|51.93|23.88|1823.25|1552.20|3097.25|3375.45|93.13|0.00|607.10|1552.20|1645.33|2159.30|2252.43|-1545.05| +2450836|81596|2450922|85391|1894708|5978|15295|85391|1894708|5978|15295|4|78|6|1|1466|258|1083|6|41.55|122.98|103.30|118.08|619.80|249.30|737.88|18.59|0.00|213.96|619.80|638.39|833.76|852.35|370.50| +2450836|81596|2450840|85391|1894708|5978|15295|85391|1894708|5978|15295|4|78|9|2|6082|150|1083|47|65.30|161.29|90.32|3335.59|4245.04|3069.10|7580.63|42.45|0.00|1515.75|4245.04|4287.49|5760.79|5803.24|1175.94| +2450836|81596|2450907|85391|1894708|5978|15295|85391|1894708|5978|15295|4|9|18|3|15829|105|1083|4|55.14|82.15|40.25|167.60|161.00|220.56|328.60|3.91|30.59|105.12|130.41|134.32|235.53|239.44|-90.15| +2450836|81596|2450911|85391|1894708|5978|15295|85391|1894708|5978|15295|4|6|5|4|3440|12|1083|62|49.68|66.57|26.62|2476.90|1650.44|3080.16|4127.34|33.00|0.00|371.38|1650.44|1683.44|2021.82|2054.82|-1429.72| +2450836|81596|2450907|85391|1894708|5978|15295|85391|1894708|5978|15295|4|50|10|3|5527|47|1083|60|19.17|50.99|5.60|2723.40|336.00|1150.20|3059.40|26.88|0.00|458.40|336.00|362.88|794.40|821.28|-814.20| +2450836|81596|2450904|85391|1894708|5978|15295|85391|1894708|5978|15295|4|13|2|5|17443|225|1083|93|3.75|8.55|5.30|302.25|492.90|348.75|795.15|6.16|369.67|238.08|123.23|129.39|361.31|367.47|-225.52| +2450836|44060|2450873|31850|96838|4673|30508|31850|96838|4673|30508|1|41|4|3|2437|174|1084|98|4.07|4.96|1.09|379.26|106.82|398.86|486.08|0.00|0.00|13.72|106.82|106.82|120.54|120.54|-292.04| +2450836|44060|2450845|31850|96838|4673|30508|31850|96838|4673|30508|1|15|1|2|10838|218|1084|63|90.48|197.24|31.55|10438.47|1987.65|5700.24|12426.12|79.50|0.00|6213.06|1987.65|2067.15|8200.71|8280.21|-3712.59| +2450836|44060|2450843|31850|96838|4673|30508|31850|96838|4673|30508|1|12|14|2|2743|11|1084|87|43.35|56.78|19.30|3260.76|1679.10|3771.45|4939.86|67.16|0.00|98.31|1679.10|1746.26|1777.41|1844.57|-2092.35| +2450836|44060|2450850|31850|96838|4673|30508|31850|96838|4673|30508|1|78|10|4|3148|91|1084|5|82.48|89.07|3.56|427.55|17.80|412.40|445.35|0.53|0.00|13.35|17.80|18.33|31.15|31.68|-394.60| +2450836|44060|2450840|31850|96838|4673|30508|31850|96838|4673|30508|1|21|12|5|310|166|1084|2|24.17|52.69|48.47|8.44|96.94|48.34|105.38|0.21|86.27|16.86|10.67|10.88|27.53|27.74|-37.67| +2450836|44060|2450897|31850|96838|4673|30508|31850|96838|4673|30508|1|19|20|3|14416|150|1084|95|16.66|23.65|21.52|202.35|2044.40|1582.70|2246.75|40.88|0.00|606.10|2044.40|2085.28|2650.50|2691.38|461.70| +2450836|44060|2450883|31850|96838|4673|30508|31850|96838|4673|30508|1|1|2|4|14125|186|1084|20|26.92|28.26|0.28|559.60|5.60|538.40|565.20|0.39|0.61|180.80|4.99|5.38|185.79|186.18|-533.41| +2450836|44060|2450899|31850|96838|4673|30508|31850|96838|4673|30508|1|75|19|3|8260|11|1084|39|70.20|131.27|17.06|4454.19|665.34|2737.80|5119.53|0.00|665.34|2252.25|0.00|0.00|2252.25|2252.25|-2737.80| +2450836|67220|2450891|65861|1529258|5902|38899|63458|1092093|1890|8087|1|7|1|3|1552|217|1085|13|49.46|63.80|44.66|248.82|580.58|642.98|829.40|5.80|0.00|381.42|580.58|586.38|962.00|967.80|-62.40| +2450836|67220|2450871|65861|1529258|5902|38899|63458|1092093|1890|8087|1|96|7|3|16928|11|1085|74|29.28|74.95|32.97|3106.52|2439.78|2166.72|5546.30|219.58|0.00|2273.28|2439.78|2659.36|4713.06|4932.64|273.06| +2450836|67220|2450872|65861|1529258|5902|38899|63458|1092093|1890|8087|1|61|16|2|17260|174|1085|9|68.55|77.46|58.86|167.40|529.74|616.95|697.14|5.29|0.00|104.49|529.74|535.03|634.23|639.52|-87.21| +2450836|67220|2450917|65861|1529258|5902|38899|63458|1092093|1890|8087|1|4|12|4|877|21|1085|48|90.40|151.87|48.59|4957.44|2332.32|4339.20|7289.76|46.64|0.00|1603.68|2332.32|2378.96|3936.00|3982.64|-2006.88| +2450836|67220|2450908|65861|1529258|5902|38899|63458|1092093|1890|8087|1|76|5|3|11140|119|1085|66|12.53|29.44|11.48|1185.36|757.68|826.98|1943.04|0.00|0.00|718.74|757.68|757.68|1476.42|1476.42|-69.30| +2450836|67220|2450884|65861|1529258|5902|38899|63458|1092093|1890|8087|1|106|15|4|2624|24|1085|79|84.65|108.35|35.75|5735.40|2824.25|6687.35|8559.65|254.18|0.00|3851.25|2824.25|3078.43|6675.50|6929.68|-3863.10| +2450836|67220|2450877|65861|1529258|5902|38899|63458|1092093|1890|8087|1|87|18|4|3688|185|1085|7|39.18|101.47|45.66|390.67|319.62|274.26|710.29|25.56|0.00|269.85|319.62|345.18|589.47|615.03|45.36| +2450836|67220|2450860|65861|1529258|5902|38899|63458|1092093|1890|8087|1|34|4|3|12175|67|1085|27|46.62|137.52|94.88|1151.28|2561.76|1258.74|3713.04|102.47|0.00|148.50|2561.76|2664.23|2710.26|2812.73|1303.02| +2450836|67220|2450868|65861|1529258|5902|38899|63458|1092093|1890|8087|1|5|11|4|6355|105|1085|50|27.76|44.13|10.59|1677.00|529.50|1388.00|2206.50|26.47|0.00|772.00|529.50|555.97|1301.50|1327.97|-858.50| +2450836|67220|2450906|65861|1529258|5902|38899|63458|1092093|1890|8087|1|18|2|1|430|3|1085|69|73.24|109.12|49.10|4141.38|3387.90|5053.56|7529.28|169.39|0.00|2484.00|3387.90|3557.29|5871.90|6041.29|-1665.66| +2450836|44797|2450871|8559|726683|2054|45778|8559|726683|2054|45778|2|1|10|2|2194|159|1086|75|32.35|45.61|20.06|1916.25|1504.50|2426.25|3420.75|90.27|0.00|1128.75|1504.50|1594.77|2633.25|2723.52|-921.75| +2450836|44797|2450849|8559|726683|2054|45778|8559|726683|2054|45778|2|64|8|2|7634|157|1086|23|74.23|87.59|23.64|1470.85|543.72|1707.29|2014.57|16.31|0.00|40.25|543.72|560.03|583.97|600.28|-1163.57| +2450836|44797|2450889|8559|726683|2054|45778|8559|726683|2054|45778|2|95|2|5|3925|171|1086|37|79.18|186.07|122.80|2340.99|4543.60|2929.66|6884.59|136.30|0.00|2409.44|4543.60|4679.90|6953.04|7089.34|1613.94| +2450836|44797|2450849|8559|726683|2054|45778|8559|726683|2054|45778|2|3|6|3|4996|3|1086|54|82.74|169.61|169.61|0.00|9158.94|4467.96|9158.94|173.10|3388.80|366.12|5770.14|5943.24|6136.26|6309.36|1302.18| +2450836|44797|2450917|8559|726683|2054|45778|8559|726683|2054|45778|2|86|14|4|1210|219|1086|2|60.41|149.21|23.87|250.68|47.74|120.82|298.42|1.90|0.00|5.96|47.74|49.64|53.70|55.60|-73.08| +2450836|44797|2450880|8559|726683|2054|45778|8559|726683|2054|45778|2|92|13|1|16934|61|1086|46|57.73|157.60|110.32|2174.88|5074.72|2655.58|7249.60|152.24|0.00|2609.58|5074.72|5226.96|7684.30|7836.54|2419.14| +2450836|44797|2450853|8559|726683|2054|45778|8559|726683|2054|45778|2|35|8|2|2162|85|1086|43|49.75|117.90|44.80|3143.30|1926.40|2139.25|5069.70|77.05|0.00|405.49|1926.40|2003.45|2331.89|2408.94|-212.85| +2450836|44797|2450912|8559|726683|2054|45778|8559|726683|2054|45778|2|95|19|2|6296|36|1086|94|80.82|125.27|26.30|9303.18|2472.20|7597.08|11775.38|222.49|0.00|2354.70|2472.20|2694.69|4826.90|5049.39|-5124.88| +2450836|44797|2450920|8559|726683|2054|45778|8559|726683|2054|45778|2|50|3|5|7651|172|1086|76|40.08|46.89|39.38|570.76|2992.88|3046.08|3563.64|209.50|0.00|1211.44|2992.88|3202.38|4204.32|4413.82|-53.20| +2450836|44797|2450858|8559|726683|2054|45778|8559|726683|2054|45778|2|34|4|4|2575|67|1086|11|80.40|131.85|94.93|406.12|1044.23|884.40|1450.35|52.21|0.00|362.56|1044.23|1096.44|1406.79|1459.00|159.83| +2450836|44797|2450844|8559|726683|2054|45778|8559|726683|2054|45778|2|45|16|5|13063|204|1086|73|54.40|77.79|10.89|4883.70|794.97|3971.20|5678.67|15.89|0.00|1646.15|794.97|810.86|2441.12|2457.01|-3176.23| +2450836|44797|2450896|8559|726683|2054|45778|8559|726683|2054|45778|2|89|16|1|3373|39|1086|51|20.31|26.40|8.71|902.19|444.21|1035.81|1346.40|26.65|0.00|592.11|444.21|470.86|1036.32|1062.97|-591.60| +2450836|33358|2450879|3340|1377914|6088|7862|3340|1377914|6088|7862|1|82|4|2|3284|250|1087|3|54.46|130.15|119.73|31.26|359.19|163.38|390.45|8.33|150.85|160.08|208.34|216.67|368.42|376.75|44.96| +2450836|33358|2450907|3340|1377914|6088|7862|3340|1377914|6088|7862|1|78|3|4|6577|207|1087|72|93.21|166.84|125.13|3003.12|9009.36|6711.12|12012.48|0.00|0.00|3242.88|9009.36|9009.36|12252.24|12252.24|2298.24| +2450836|33358|2450917|3340|1377914|6088|7862|3340|1377914|6088|7862|1|14|10|1|17200|16|1087|95|57.21|157.32|72.36|8071.20|6874.20|5434.95|14945.40|68.74|0.00|5230.70|6874.20|6942.94|12104.90|12173.64|1439.25| +2450836|33358|2450896|3340|1377914|6088|7862|3340|1377914|6088|7862|1|4|9|4|1681|42|1087|71|92.84|225.60|196.27|2082.43|13935.17|6591.64|16017.60|367.88|9336.56|1441.30|4598.61|4966.49|6039.91|6407.79|-1993.03| +2450836|33358|2450841|3340|1377914|6088|7862|3340|1377914|6088|7862|1|21|4|1|11437|97|1087|33|40.01|104.42|99.19|172.59|3273.27|1320.33|3445.86|261.86|0.00|999.24|3273.27|3535.13|4272.51|4534.37|1952.94| +2450836|33358|2450842|3340|1377914|6088|7862|3340|1377914|6088|7862|1|101|19|4|8828|24|1087|100|50.97|133.03|106.42|2661.00|10642.00|5097.00|13303.00|957.78|0.00|2128.00|10642.00|11599.78|12770.00|13727.78|5545.00| +2450836|33358|2450875|3340|1377914|6088|7862|3340|1377914|6088|7862|1|37|13|3|13135|128|1087|17|96.82|139.42|136.63|47.43|2322.71|1645.94|2370.14|116.13|0.00|23.63|2322.71|2438.84|2346.34|2462.47|676.77| +2450836|33358|2450865|3340|1377914|6088|7862|3340|1377914|6088|7862|1|20|18|2|10478|215|1087|58|88.43|140.60|2.81|7991.82|162.98|5128.94|8154.80|11.40|0.00|488.94|162.98|174.38|651.92|663.32|-4965.96| +2450836|33358|2450838|3340|1377914|6088|7862|3340|1377914|6088|7862|1|84|5|4|10414|203|1087|43|97.84|222.09|104.38|5061.53|4488.34|4207.12|9549.87|89.76|0.00|1432.33|4488.34|4578.10|5920.67|6010.43|281.22| +2450836|33358|2450922|3340|1377914|6088|7862|3340|1377914|6088|7862|1|42|4|1|8599|115|1087|2|90.65|105.15|6.30|197.70|12.60|181.30|210.30|0.22|8.82|10.50|3.78|4.00|14.28|14.50|-177.52| +2450836|23834|2450922|36220|32920|2067|7614|36220|32920|2067|7614|1|55|15|5|2816|115|1088|35|8.96|25.44|21.87|124.95|765.45|313.60|890.40|9.41|451.61|44.45|313.84|323.25|358.29|367.70|0.24| +2450836|23834|2450877|36220|32920|2067|7614|36220|32920|2067|7614|1|6|17|5|6916|64|1088|84|11.70|21.64|6.92|1236.48|581.28|982.80|1817.76|0.00|0.00|472.08|581.28|581.28|1053.36|1053.36|-401.52| +2450836|23834|2450922|36220|32920|2067|7614|36220|32920|2067|7614|1|86|7|2|4681|264|1088|6|66.62|179.20|111.10|408.60|666.60|399.72|1075.20|53.32|0.00|279.54|666.60|719.92|946.14|999.46|266.88| +2450836|23834|2450873|36220|32920|2067|7614|36220|32920|2067|7614|1|68|4|4|2384|286|1088|58|6.09|13.45|10.76|156.02|624.08|353.22|780.10|37.44|0.00|312.04|624.08|661.52|936.12|973.56|270.86| +2450836|23834|2450878|36220|32920|2067|7614|36220|32920|2067|7614|1|18|7|1|16054|84|1088|61|78.33|115.92|91.57|1485.35|5585.77|4778.13|7071.12|223.43|0.00|1837.93|5585.77|5809.20|7423.70|7647.13|807.64| +2450836|23834|2450925|36220|32920|2067|7614|36220|32920|2067|7614|1|90|13|1|3734|276|1088|66|23.65|45.17|23.03|1461.24|1519.98|1560.90|2981.22|75.99|0.00|894.30|1519.98|1595.97|2414.28|2490.27|-40.92| +2450836|23834|2450893|36220|32920|2067|7614|36220|32920|2067|7614|1|6|3|4|2032|244|1088|13|49.87|142.12|22.73|1552.07|295.49|648.31|1847.56|14.77|0.00|757.38|295.49|310.26|1052.87|1067.64|-352.82| +2450836|23834|2450884|36220|32920|2067|7614|36220|32920|2067|7614|1|42|10|4|892|164|1088|82|17.49|19.23|0.19|1561.28|15.58|1434.18|1576.86|0.93|0.00|409.18|15.58|16.51|424.76|425.69|-1418.60| +2450836|23834|2450906|36220|32920|2067|7614|36220|32920|2067|7614|1|32|5|3|17803|231|1088|33|35.52|68.90|27.56|1364.22|909.48|1172.16|2273.70|72.75|0.00|431.97|909.48|982.23|1341.45|1414.20|-262.68| +2450836|23834|2450858|36220|32920|2067|7614|36220|32920|2067|7614|1|51|9|4|6871|102|1088|46|84.01|110.89|64.31|2142.68|2958.26|3864.46|5100.94|147.91|0.00|1275.12|2958.26|3106.17|4233.38|4381.29|-906.20| +2450836|23834|2450872|36220|32920|2067|7614|36220|32920|2067|7614|1|46|7|1|6386|184|1088|72|30.12|54.21|28.18|1874.16|2028.96|2168.64|3903.12|142.02|0.00|1404.72|2028.96|2170.98|3433.68|3575.70|-139.68| +2450836|23834|2450912|36220|32920|2067|7614|36220|32920|2067|7614|1|23|6|2|17365|194|1088|79|35.18|101.31|27.35|5842.84|2160.65|2779.22|8003.49|129.63|0.00|3840.98|2160.65|2290.28|6001.63|6131.26|-618.57| +2450836|30642|2450880|87119|502030|6894|25405|87119|502030|6894|25405|1|23|6|3|8827|43|1089|35|99.96|123.95|59.49|2256.10|2082.15|3498.60|4338.25|41.64|0.00|737.45|2082.15|2123.79|2819.60|2861.24|-1416.45| +2450836|30642|2450866|87119|502030|6894|25405|87119|502030|6894|25405|1|79|9|4|1376|281|1089|11|92.88|209.90|117.54|1015.96|1292.94|1021.68|2308.90|25.85|0.00|715.66|1292.94|1318.79|2008.60|2034.45|271.26| +2450836|30642|2450861|87119|502030|6894|25405|87119|502030|6894|25405|1|62|16|1|16510|269|1089|81|68.11|94.67|25.56|5597.91|2070.36|5516.91|7668.27|20.70|0.00|1456.38|2070.36|2091.06|3526.74|3547.44|-3446.55| +2450836|30642|2450893|87119|502030|6894|25405|87119|502030|6894|25405|1|15|11|1|16849|234|1089|56|88.85|216.79|23.84|10805.20|1335.04|4975.60|12140.24|93.45|0.00|242.48|1335.04|1428.49|1577.52|1670.97|-3640.56| +2450836|30642|2450852|87119|502030|6894|25405|87119|502030|6894|25405|1|7|17|2|4615|65|1089|28|18.62|30.16|11.46|523.60|320.88|521.36|844.48|6.41|0.00|59.08|320.88|327.29|379.96|386.37|-200.48| +2450836|30642|2450892|87119|502030|6894|25405|87119|502030|6894|25405|1|84|3|4|7861|49|1089|37|39.16|87.32|12.22|2778.70|452.14|1448.92|3230.84|7.95|54.25|710.77|397.89|405.84|1108.66|1116.61|-1051.03| +2450836|30642|2450882|87119|502030|6894|25405|87119|502030|6894|25405|1|51|5|2|6038|263|1089|49|12.72|30.14|29.83|15.19|1461.67|623.28|1476.86|102.31|0.00|679.14|1461.67|1563.98|2140.81|2243.12|838.39| +2450836|30642|2450901|87119|502030|6894|25405|87119|502030|6894|25405|1|57|15|1|988|212|1089|67|88.04|162.87|16.28|9821.53|1090.76|5898.68|10912.29|65.44|0.00|108.54|1090.76|1156.20|1199.30|1264.74|-4807.92| +2450836|30642|2450924|87119|502030|6894|25405|87119|502030|6894|25405|1|35|2|5|5038|288|1089|66|72.80|107.74|14.00|6186.84|924.00|4804.80|7110.84|18.48|0.00|2702.04|924.00|942.48|3626.04|3644.52|-3880.80| +2450836|30642|2450855|87119|502030|6894|25405|87119|502030|6894|25405|1|5|19|5|12374|168|1089|7|78.24|230.02|115.01|805.07|805.07|547.68|1610.14|64.40|0.00|193.20|805.07|869.47|998.27|1062.67|257.39| +2450836|30642|2450890|87119|502030|6894|25405|87119|502030|6894|25405|1|100|15|2|1814|224|1089|75|87.19|177.86|174.30|267.00|13072.50|6539.25|13339.50|261.45|0.00|5202.00|13072.50|13333.95|18274.50|18535.95|6533.25| +2450836|30642|2450849|87119|502030|6894|25405|87119|502030|6894|25405|1|34|1|5|9830|253|1089|85|78.53|124.86|21.22|8809.40|1803.70|6675.05|10613.10|36.07|0.00|3714.50|1803.70|1839.77|5518.20|5554.27|-4871.35| +2450836|30642|2450872|87119|502030|6894|25405|87119|502030|6894|25405|1|60|2|1|8929|126|1089|15|58.15|93.62|66.47|407.25|997.05|872.25|1404.30|59.82|0.00|42.00|997.05|1056.87|1039.05|1098.87|124.80| +2450836|10288|2450892|44811|427909|3454|42152|44811|427909|3454|42152|2|22|9|2|7772|248|1090|22|30.03|77.17|47.84|645.26|1052.48|660.66|1697.74|10.52|0.00|407.44|1052.48|1063.00|1459.92|1470.44|391.82| +2450836|10288|2450911|44811|427909|3454|42152|44811|427909|3454|42152|2|94|14|4|2284|172|1090|19|95.02|274.60|74.14|3808.74|1408.66|1805.38|5217.40|26.48|746.58|1252.10|662.08|688.56|1914.18|1940.66|-1143.30| +2450836|10288|2450908|44811|427909|3454|42152|44811|427909|3454|42152|2|68|13|1|6332|15|1090|43|42.45|78.10|39.83|1645.61|1712.69|1825.35|3358.30|68.50|0.00|1041.03|1712.69|1781.19|2753.72|2822.22|-112.66| +2450836|10288|2450926|44811|427909|3454|42152|44811|427909|3454|42152|2|69|13|3|8480|18|1090|34|15.24|37.03|12.96|818.38|440.64|518.16|1259.02|22.03|0.00|125.80|440.64|462.67|566.44|588.47|-77.52| +2450836|10288|2450898|44811|427909|3454|42152|44811|427909|3454|42152|2|5|6|3|8581|117|1090|87|32.94|61.92|36.53|2208.93|3178.11|2865.78|5387.04|286.02|0.00|1292.82|3178.11|3464.13|4470.93|4756.95|312.33| +2450836|10288|2450847|44811|427909|3454|42152|44811|427909|3454|42152|2|100|3|1|13496|240|1090|98|26.98|41.81|1.67|3933.72|163.66|2644.04|4097.38|3.27|81.83|614.46|81.83|85.10|696.29|699.56|-2562.21| +2450836|10288|2450887|44811|427909|3454|42152|44811|427909|3454|42152|2|93|9|1|16316|176|1090|30|34.74|91.01|69.16|655.50|2074.80|1042.20|2730.30|1.65|1908.81|1337.70|165.99|167.64|1503.69|1505.34|-876.21| +2450836|10288|2450842|44811|427909|3454|42152|44811|427909|3454|42152|2|73|7|1|5530|142|1090|13|79.42|150.89|72.42|1020.11|941.46|1032.46|1961.57|65.90|0.00|509.99|941.46|1007.36|1451.45|1517.35|-91.00| +2450836|10288|2450902|44811|427909|3454|42152|44811|427909|3454|42152|2|108|19|5|9331|267|1090|51|72.35|149.04|28.31|6157.23|1443.81|3689.85|7601.04|28.87|0.00|151.98|1443.81|1472.68|1595.79|1624.66|-2246.04| +2450836|10288|2450859|44811|427909|3454|42152|44811|427909|3454|42152|2|2|10|4|1045|223|1090|64|75.68|83.24|66.59|1065.60|4261.76|4843.52|5327.36|298.32|0.00|2183.68|4261.76|4560.08|6445.44|6743.76|-581.76| +2450836|10288|2450863|44811|427909|3454|42152|44811|427909|3454|42152|2|56|2|2|2014|174|1090|53|21.60|54.86|32.91|1163.35|1744.23|1144.80|2907.58|104.65|0.00|1104.52|1744.23|1848.88|2848.75|2953.40|599.43| +2450836|10288|2450868|44811|427909|3454|42152|44811|427909|3454|42152|2|79|13|1|6265|217|1090|28|7.95|19.39|8.72|298.76|244.16|222.60|542.92|2.44|0.00|130.20|244.16|246.60|374.36|376.80|21.56| +2450836|10288|2450863|44811|427909|3454|42152|44811|427909|3454|42152|2|19|8|3|12908|180|1090|91|46.09|47.93|34.98|1178.45|3183.18|4194.19|4361.63|159.15|0.00|1482.39|3183.18|3342.33|4665.57|4824.72|-1011.01| +2450836|71065|2450915|31862|543632|3755|2725|31862|543632|3755|2725|4|13|6|1|2018|105|1091|90|98.48|149.68|20.95|11585.70|1885.50|8863.20|13471.20|18.85|0.00|4176.00|1885.50|1904.35|6061.50|6080.35|-6977.70| +2450836|71065|2450848|31862|543632|3755|2725|31862|543632|3755|2725|4|61|1|5|8660|50|1091|14|3.19|8.93|7.76|16.38|108.64|44.66|125.02|4.34|0.00|57.40|108.64|112.98|166.04|170.38|63.98| +2450836|71065|2450842|31862|543632|3755|2725|31862|543632|3755|2725|4|100|17|5|13852|123|1091|26|42.19|76.78|42.22|898.56|1097.72|1096.94|1996.28|21.95|0.00|79.82|1097.72|1119.67|1177.54|1199.49|0.78| +2450836|71065|2450921|31862|543632|3755|2725|31862|543632|3755|2725|4|88|6|3|6115|178|1091|66|91.55|214.22|126.38|5797.44|8341.08|6042.30|14138.52|250.23|0.00|6503.64|8341.08|8591.31|14844.72|15094.95|2298.78| +2450836|71065|2450869|31862|543632|3755|2725|31862|543632|3755|2725|4|42|16|4|17137|229|1091|60|59.76|119.52|2.39|7027.80|143.40|3585.60|7171.20|8.60|0.00|2581.20|143.40|152.00|2724.60|2733.20|-3442.20| +2450836|71065|2450880|31862|543632|3755|2725|31862|543632|3755|2725|4|50|18|4|13078|135|1091|95|72.10|90.84|19.98|6731.70|1898.10|6849.50|8629.80|2.46|1651.34|1984.55|246.76|249.22|2231.31|2233.77|-6602.74| +2450836|71065|2450877|31862|543632|3755|2725|31862|543632|3755|2725|4|28|3|3|8630|273|1091|67|32.71|59.85|39.50|1363.45|2646.50|2191.57|4009.95|0.00|0.00|160.13|2646.50|2646.50|2806.63|2806.63|454.93| +2450836|71065|2450848|31862|543632|3755|2725|31862|543632|3755|2725|4|72|17|4|9283|15|1091|62|50.18|115.91|18.54|6036.94|1149.48|3111.16|7186.42|45.97|0.00|574.74|1149.48|1195.45|1724.22|1770.19|-1961.68| +2450836|71065|2450900|31862|543632|3755|2725|31862|543632|3755|2725|4|46|11|5|15034|106|1091|72|66.94|135.21|60.84|5354.64|4380.48|4819.68|9735.12|262.82|0.00|4283.28|4380.48|4643.30|8663.76|8926.58|-439.20| +2450836|71065|2450847|31862|543632|3755|2725|31862|543632|3755|2725|4|70|20|4|12661|266|1091|56|29.36|38.75|25.57|738.08|1431.92|1644.16|2170.00|14.31|0.00|846.16|1431.92|1446.23|2278.08|2292.39|-212.24| +2450836|71065|2450905|31862|543632|3755|2725|31862|543632|3755|2725|4|44|18|3|17356|60|1091|84|64.88|188.15|41.39|12327.84|3476.76|5449.92|15804.60|104.30|0.00|5056.80|3476.76|3581.06|8533.56|8637.86|-1973.16| +2450836|71065|2450882|31862|543632|3755|2725|31862|543632|3755|2725|4|11|13|2|10741|200|1091|10|91.34|232.00|146.16|858.40|1461.60|913.40|2320.00|102.31|0.00|1136.80|1461.60|1563.91|2598.40|2700.71|548.20| +2450836|71065|2450863|31862|543632|3755|2725|31862|543632|3755|2725|4|81|17|5|14236|206|1091|64|64.23|122.67|52.74|4475.52|3375.36|4110.72|7850.88|33.75|0.00|1726.72|3375.36|3409.11|5102.08|5135.83|-735.36| +2450836|71065|2450842|31862|543632|3755|2725|31862|543632|3755|2725|4|8|18|1|1300|76|1091|38|35.59|91.11|88.37|104.12|3358.06|1352.42|3462.18|167.90|0.00|346.18|3358.06|3525.96|3704.24|3872.14|2005.64| +2450836|311|2450892|73449|1024495|2741|45787|73449|1024495|2741|45787|2|106|11|1|11119|54|1092|4|98.92|236.41|231.68|18.92|926.72|395.68|945.64|37.06|0.00|18.88|926.72|963.78|945.60|982.66|531.04| +2450836|311|2450867|73449|1024495|2741|45787|73449|1024495|2741|45787|2|62|13|3|11818|253|1092|96|97.02|115.45|6.92|10418.88|664.32|9313.92|11083.20|39.85|0.00|2992.32|664.32|704.17|3656.64|3696.49|-8649.60| +2450836|311|2450908|73449|1024495|2741|45787|73449|1024495|2741|45787|2|67|4|4|9379|264|1092|40|81.79|230.64|184.51|1845.20|7380.40|3271.60|9225.60|516.62|0.00|276.40|7380.40|7897.02|7656.80|8173.42|4108.80| +2450836|311|2450879|73449|1024495|2741|45787|73449|1024495|2741|45787|2|9|8|2|2410|80|1092|70|17.30|17.47|11.70|403.90|819.00|1211.00|1222.90|24.57|0.00|280.70|819.00|843.57|1099.70|1124.27|-392.00| +2450836|311|2450914|73449|1024495|2741|45787|73449|1024495|2741|45787|2|50|17|3|1177|194|1092|58|16.81|37.31|11.93|1472.04|691.94|974.98|2163.98|4.98|629.66|908.86|62.28|67.26|971.14|976.12|-912.70| +2450836|311|2450841|73449|1024495|2741|45787|73449|1024495|2741|45787|2|106|5|4|1465|40|1092|32|55.94|139.85|68.52|2282.56|2192.64|1790.08|4475.20|89.46|701.64|1924.16|1491.00|1580.46|3415.16|3504.62|-299.08| +2450836|311|2450888|73449|1024495|2741|45787|73449|1024495|2741|45787|2|38|9|3|9032|144|1092|47|75.22|185.04|37.00|6957.88|1739.00|3535.34|8696.88|139.12|0.00|3391.52|1739.00|1878.12|5130.52|5269.64|-1796.34| +2450836|311|2450918|73449|1024495|2741|45787|73449|1024495|2741|45787|2|73|12|4|9406|1|1092|56|2.13|2.66|1.22|80.64|68.32|119.28|148.96|0.00|0.00|29.68|68.32|68.32|98.00|98.00|-50.96| +2450836|311|2450926|73449|1024495|2741|45787|73449|1024495|2741|45787|2|104|19|4|16714|247|1092|88|77.75|160.94|125.53|3116.08|11046.64|6842.00|14162.72|331.39|0.00|2406.80|11046.64|11378.03|13453.44|13784.83|4204.64| +2450836|54342|2450921|71742|1802546|903|18460|71742|1802546|903|18460|4|41|16|4|13964|143|1093|73|84.34|101.20|86.02|1108.14|6279.46|6156.82|7387.60|376.76|0.00|2585.66|6279.46|6656.22|8865.12|9241.88|122.64| +2450836|54342|2450900|71742|1802546|903|18460|71742|1802546|903|18460|4|69|16|5|8257|105|1093|64|96.33|223.48|80.45|9153.92|5148.80|6165.12|14302.72|463.39|0.00|2001.92|5148.80|5612.19|7150.72|7614.11|-1016.32| +2450836|54342|2450924|71742|1802546|903|18460|71742|1802546|903|18460|4|75|16|2|11276|159|1093|40|99.61|168.34|131.30|1481.60|5252.00|3984.40|6733.60|30.98|2153.32|1414.00|3098.68|3129.66|4512.68|4543.66|-885.72| +2450836|54342|2450895|71742|1802546|903|18460|71742|1802546|903|18460|4|92|8|2|7856|109|1093|64|30.32|56.39|24.81|2021.12|1587.84|1940.48|3608.96|47.63|0.00|1731.84|1587.84|1635.47|3319.68|3367.31|-352.64| +2450836|54342|2450861|71742|1802546|903|18460|71742|1802546|903|18460|4|98|5|4|14380|205|1093|82|40.65|105.69|48.61|4680.56|3986.02|3333.30|8666.58|358.74|0.00|259.94|3986.02|4344.76|4245.96|4604.70|652.72| +2450836|54342|2450863|71742|1802546|903|18460|71742|1802546|903|18460|4|22|17|1|11666|204|1093|79|80.02|187.24|93.62|7395.98|7395.98|6321.58|14791.96|517.71|0.00|6212.56|7395.98|7913.69|13608.54|14126.25|1074.40| +2450836|54342|2450874|71742|1802546|903|18460|71742|1802546|903|18460|4|9|10|1|13544|254|1093|100|6.27|13.54|7.85|569.00|785.00|627.00|1354.00|15.70|0.00|54.00|785.00|800.70|839.00|854.70|158.00| +2450836|54342|2450843|71742|1802546|903|18460|71742|1802546|903|18460|4|28|7|1|11263|10|1093|18|31.46|57.57|37.99|352.44|683.82|566.28|1036.26|22.97|300.88|20.70|382.94|405.91|403.64|426.61|-183.34| +2450836|54342|2450885|71742|1802546|903|18460|71742|1802546|903|18460|4|75|4|1|1060|95|1093|55|93.42|187.77|101.39|4750.90|5576.45|5138.10|10327.35|446.11|0.00|1239.15|5576.45|6022.56|6815.60|7261.71|438.35| +2450836|54342|2450854|71742|1802546|903|18460|71742|1802546|903|18460|4|57|16|2|15037|79|1093|96|89.87|129.41|51.76|7454.40|4968.96|8627.52|12423.36|149.06|0.00|1117.44|4968.96|5118.02|6086.40|6235.46|-3658.56| +2450836|54342|2450901|71742|1802546|903|18460|71742|1802546|903|18460|4|33|1|5|6392|281|1093|66|21.39|45.34|20.40|1646.04|1346.40|1411.74|2992.44|0.00|0.00|238.92|1346.40|1346.40|1585.32|1585.32|-65.34| +2450836|54342|2450868|71742|1802546|903|18460|71742|1802546|903|18460|4|73|4|1|11632|169|1093|88|24.77|40.37|18.16|1954.48|1598.08|2179.76|3552.56|0.00|0.00|1526.80|1598.08|1598.08|3124.88|3124.88|-581.68| +2450836|54342|2450866|71742|1802546|903|18460|71742|1802546|903|18460|4|39|4|5|2638|45|1093|82|61.80|177.98|101.44|6276.28|8318.08|5067.60|14594.36|0.00|0.00|5253.74|8318.08|8318.08|13571.82|13571.82|3250.48| +2450836|54342|2450901|71742|1802546|903|18460|71742|1802546|903|18460|4|2|1|2|5884|128|1093|34|23.52|27.51|16.23|383.52|551.82|799.68|935.34|0.11|540.78|9.18|11.04|11.15|20.22|20.33|-788.64| +2450836|40279|2450894|91603|720990|2824|13011|91603|720990|2824|13011|1|12|13|3|2534|40|1094|54|69.26|113.58|35.20|4232.52|1900.80|3740.04|6133.32|152.06|0.00|1103.76|1900.80|2052.86|3004.56|3156.62|-1839.24| +2450836|40279|2450862|91603|720990|2824|13011|91603|720990|2824|13011|1|21|4|3|2173|233|1094|93|67.24|88.75|21.30|6272.85|1980.90|6253.32|8253.75|79.23|0.00|3466.11|1980.90|2060.13|5447.01|5526.24|-4272.42| +2450836|40279|2450862|91603|720990|2824|13011|91603|720990|2824|13011|1|42|14|5|9520|244|1094|75|76.27|97.62|61.50|2709.00|4612.50|5720.25|7321.50|184.50|0.00|804.75|4612.50|4797.00|5417.25|5601.75|-1107.75| +2450836|40279|2450923|91603|720990|2824|13011|91603|720990|2824|13011|1|35|10|1|7873|252|1094|54|64.39|166.77|16.67|8105.40|900.18|3477.06|9005.58|63.01|0.00|720.36|900.18|963.19|1620.54|1683.55|-2576.88| +2450836|40279|2450889|91603|720990|2824|13011|91603|720990|2824|13011|1|92|6|3|14077|197|1094|60|77.47|113.10|0.00|6786.00|0.00|4648.20|6786.00|0.00|0.00|2510.40|0.00|0.00|2510.40|2510.40|-4648.20| +2450836|40279|2450900|91603|720990|2824|13011|91603|720990|2824|13011|1|107|10|3|16898|229|1094|25|96.56|117.80|51.83|1649.25|1295.75|2414.00|2945.00|25.91|0.00|1148.50|1295.75|1321.66|2444.25|2470.16|-1118.25| +2450836|40279|2450870|91603|720990|2824|13011|91603|720990|2824|13011|1|22|2|3|16502|157|1094|14|43.56|71.43|71.43|0.00|1000.02|609.84|1000.02|27.90|690.01|289.94|310.01|337.91|599.95|627.85|-299.83| +2450836|40279|2450876|91603|720990|2824|13011|91603|720990|2824|13011|1|33|10|5|13832|125|1094|1|1.87|5.51|2.75|2.76|2.75|1.87|5.51|0.22|0.00|1.65|2.75|2.97|4.40|4.62|0.88| +2450836|40279|2450893|91603|720990|2824|13011|91603|720990|2824|13011|1|46|14|2|10834|186|1094|92|33.08|44.32|26.14|1672.56|2404.88|3043.36|4077.44|72.14|0.00|1794.00|2404.88|2477.02|4198.88|4271.02|-638.48| +2450836|40279|2450908|91603|720990|2824|13011|91603|720990|2824|13011|1|59|6|4|17470|206|1094|82|23.70|44.08|33.50|867.56|2747.00|1943.40|3614.56|109.88|0.00|252.56|2747.00|2856.88|2999.56|3109.44|803.60| +2450836|40279|2450854|91603|720990|2824|13011|91603|720990|2824|13011|1|25|4|4|15520|89|1094|42|60.26|131.96|5.27|5320.98|221.34|2530.92|5542.32|15.49|0.00|775.74|221.34|236.83|997.08|1012.57|-2309.58| +2450836|40279|2450852|91603|720990|2824|13011|91603|720990|2824|13011|1|8|12|2|13810|157|1094|20|25.03|58.31|22.74|711.40|454.80|500.60|1166.20|22.74|0.00|244.80|454.80|477.54|699.60|722.34|-45.80| +2450836|40279|2450889|91603|720990|2824|13011|91603|720990|2824|13011|1|34|1|3|10544|265|1094|94|19.93|47.43|11.85|3344.52|1113.90|1873.42|4458.42|14.25|935.67|445.56|178.23|192.48|623.79|638.04|-1695.19| +2450836|70582|2450895|29556|102756|2709|7862|29556|102756|2709|7862|1|28|8|3|17131|213|1095|39|4.10|4.22|0.50|145.08|19.50|159.90|164.58|0.78|0.00|1.56|19.50|20.28|21.06|21.84|-140.40| +2450836|70582|2450853|29556|102756|2709|7862|29556|102756|2709|7862|1|53|5|2|5203|133|1095|69|4.84|13.69|9.71|274.62|669.99|333.96|944.61|33.49|0.00|349.14|669.99|703.48|1019.13|1052.62|336.03| +2450836|70582|2450924|29556|102756|2709|7862|29556|102756|2709|7862|1|22|17|4|7912|244|1095|47|20.82|39.76|37.77|93.53|1775.19|978.54|1868.72|159.76|0.00|859.16|1775.19|1934.95|2634.35|2794.11|796.65| +2450836|70582|2450903|29556|102756|2709|7862|29556|102756|2709|7862|1|75|19|3|4532|247|1095|43|94.52|102.08|64.31|1624.11|2765.33|4064.36|4389.44|27.65|0.00|1448.24|2765.33|2792.98|4213.57|4241.22|-1299.03| +2450836|70582|2450910|29556|102756|2709|7862|29556|102756|2709|7862|1|65|11|3|589|242|1095|89|22.53|60.83|4.86|4981.33|432.54|2005.17|5413.87|0.00|276.82|1190.82|155.72|155.72|1346.54|1346.54|-1849.45| +2450836|59631|2450883|90675|942585|4895|5605|90675|942585|4895|5605|4|27|2|5|9601|228|1096|70|10.51|18.07|5.24|898.10|366.80|735.70|1264.90|11.00|0.00|315.70|366.80|377.80|682.50|693.50|-368.90| +2450836|59631|2450838|90675|942585|4895|5605|90675|942585|4895|5605|4|60|15|5|11164|147|1096|73|62.97|108.93|47.92|4453.73|3498.16|4596.81|7951.89|209.88|0.00|2703.19|3498.16|3708.04|6201.35|6411.23|-1098.65| +2450836|59631|2450888|90675|942585|4895|5605|90675|942585|4895|5605|4|98|6|2|13933|82|1096|9|33.71|91.01|77.35|122.94|696.15|303.39|819.09|0.00|0.00|221.13|696.15|696.15|917.28|917.28|392.76| +2450836|59631|2450911|90675|942585|4895|5605|90675|942585|4895|5605|4|40|2|3|8863|203|1096|67|54.70|117.05|104.17|862.96|6979.39|3664.90|7842.35|139.58|0.00|2509.15|6979.39|7118.97|9488.54|9628.12|3314.49| +2450836|59631|2450894|90675|942585|4895|5605|90675|942585|4895|5605|4|103|3|4|14270|13|1096|58|73.43|169.62|32.22|7969.20|1868.76|4258.94|9837.96|0.00|0.00|196.62|1868.76|1868.76|2065.38|2065.38|-2390.18| +2450836|59631|2450873|90675|942585|4895|5605|90675|942585|4895|5605|4|97|8|3|14779|147|1096|97|38.04|51.73|51.73|0.00|5017.81|3689.88|5017.81|100.35|0.00|0.00|5017.81|5118.16|5017.81|5118.16|1327.93| +2450836|59631|2450839|90675|942585|4895|5605|90675|942585|4895|5605|4|44|17|2|5869|93|1096|58|43.17|116.99|37.43|4614.48|2170.94|2503.86|6785.42|108.54|0.00|2374.52|2170.94|2279.48|4545.46|4654.00|-332.92| +2450836|59631|2450920|90675|942585|4895|5605|90675|942585|4895|5605|4|69|2|2|12188|269|1096|82|4.35|10.83|4.98|479.70|408.36|356.70|888.06|0.65|392.02|186.14|16.34|16.99|202.48|203.13|-340.36| +2450836|59631|2450926|90675|942585|4895|5605|90675|942585|4895|5605|4|19|9|2|13958|63|1096|28|78.00|185.64|9.28|4938.08|259.84|2184.00|5197.92|13.72|88.34|2494.80|171.50|185.22|2666.30|2680.02|-2012.50| +2450836|61411|2450862|92607|229509|11|5666|92607|229509|11|5666|1|97|12|2|8995|205|1097|82|53.83|58.67|4.10|4474.74|336.20|4414.06|4810.94|20.17|0.00|336.20|336.20|356.37|672.40|692.57|-4077.86| +2450836|61411|2450857|92607|229509|11|5666|92607|229509|11|5666|1|1|17|1|8228|109|1097|20|3.87|8.55|1.88|133.40|37.60|77.40|171.00|0.88|22.93|23.80|14.67|15.55|38.47|39.35|-62.73| +2450836|61411|2450874|92607|229509|11|5666|92607|229509|11|5666|1|99|18|5|6908|143|1097|76|27.88|68.86|53.71|1151.40|4081.96|2118.88|5233.36|40.81|0.00|104.12|4081.96|4122.77|4186.08|4226.89|1963.08| +2450836|61411|2450858|92607|229509|11|5666|92607|229509|11|5666|1|30|15|4|8066|35|1097|11|61.34|85.26|16.19|759.77|178.09|674.74|937.86|16.02|0.00|206.25|178.09|194.11|384.34|400.36|-496.65| +2450836|61411|2450849|92607|229509|11|5666|92607|229509|11|5666|1|4|4|4|170|284|1097|79|57.48|117.25|0.00|9262.75|0.00|4540.92|9262.75|0.00|0.00|2964.08|0.00|0.00|2964.08|2964.08|-4540.92| +2450836|61411|2450872|92607|229509|11|5666|92607|229509|11|5666|1|17|20|3|14378|134|1097|79|77.22|168.33|141.39|2128.26|11169.81|6100.38|13298.07|0.00|0.00|5584.51|11169.81|11169.81|16754.32|16754.32|5069.43| +2450836|61411|2450905|92607|229509|11|5666|92607|229509|11|5666|1|54|15|1|602|209|1097|19|1.23|2.15|1.48|12.73|28.12|23.37|40.85|2.53|0.00|16.34|28.12|30.65|44.46|46.99|4.75| +2450836|61411|2450916|92607|229509|11|5666|92607|229509|11|5666|1|108|6|5|6350|75|1097|66|22.52|35.58|8.89|1761.54|586.74|1486.32|2348.28|29.33|0.00|116.82|586.74|616.07|703.56|732.89|-899.58| +2450836|61411|2450892|92607|229509|11|5666|92607|229509|11|5666|1|71|10|2|4726|46|1097|52|24.42|26.86|9.66|894.40|502.32|1269.84|1396.72|5.02|0.00|488.80|502.32|507.34|991.12|996.14|-767.52| +2450836|61411|2450866|92607|229509|11|5666|92607|229509|11|5666|1|51|12|4|13771|55|1097|22|31.66|32.60|5.86|588.28|128.92|696.52|717.20|9.02|0.00|315.48|128.92|137.94|444.40|453.42|-567.60| +2450836|61411|2450904|92607|229509|11|5666|92607|229509|11|5666|1|69|5|3|5824|16|1097|37|84.39|244.73|110.12|4980.57|4074.44|3122.43|9055.01|122.23|0.00|1357.90|4074.44|4196.67|5432.34|5554.57|952.01| +2450836|61411|2450902|92607|229509|11|5666|92607|229509|11|5666|1|48|15|5|14300|273|1097|98|50.77|100.01|77.00|2254.98|7546.00|4975.46|9800.98|377.30|0.00|1274.00|7546.00|7923.30|8820.00|9197.30|2570.54| +2450836|61411|2450846|92607|229509|11|5666|92607|229509|11|5666|1|31|9|3|1508|271|1097|76|55.95|149.94|22.49|9686.20|1709.24|4252.20|11395.44|153.83|0.00|569.24|1709.24|1863.07|2278.48|2432.31|-2542.96| +2450836|61411|2450878|92607|229509|11|5666|92607|229509|11|5666|1|21|19|5|8878|154|1097|55|51.10|123.15|44.33|4335.10|2438.15|2810.50|6773.25|48.76|0.00|1489.95|2438.15|2486.91|3928.10|3976.86|-372.35| +2450836|73369|2450902|74072|863650|1675|2500|74072|863650|1675|2500|2|31|11|3|397|198|1098|57|37.79|55.92|25.16|1753.32|1434.12|2154.03|3187.44|7.88|645.35|1019.73|788.77|796.65|1808.50|1816.38|-1365.26| +2450836|73369|2450838|74072|863650|1675|2500|74072|863650|1675|2500|2|22|10|1|16678|248|1098|25|24.88|45.53|43.70|45.75|1092.50|622.00|1138.25|87.40|0.00|432.50|1092.50|1179.90|1525.00|1612.40|470.50| +2450836|73369|2450890|74072|863650|1675|2500|74072|863650|1675|2500|2|5|9|2|5659|59|1098|24|70.52|154.43|151.34|74.16|3632.16|1692.48|3706.32|36.32|0.00|852.24|3632.16|3668.48|4484.40|4520.72|1939.68| +2450836|73369|2450914|74072|863650|1675|2500|74072|863650|1675|2500|2|66|13|5|17569|298|1098|86|24.12|68.50|26.71|3593.94|2297.06|2074.32|5891.00|206.73|0.00|2179.24|2297.06|2503.79|4476.30|4683.03|222.74| +2450836|76271|2450849||||29395|1214||4268||1|108|||1718|153|1099|78|17.10|19.15|6.51||507.78|1333.80|1493.70|0.00||104.52|507.78|507.78|612.30|612.30|-826.02| +2450836|76271|2450843|1214|832741|4268|29395|1214|832741|4268|29395|1|49|15|3|11930|264|1099|83|67.45|89.70|3.58|7147.96|297.14|5598.35|7445.10|2.97|0.00|967.78|297.14|300.11|1264.92|1267.89|-5301.21| +2450836|76271|2450893|1214|832741|4268|29395|1214|832741|4268|29395|1|95|16|5|13286|18|1099|89|58.27|121.20|94.53|2373.63|8413.17|5186.03|10786.80|336.52|0.00|2696.70|8413.17|8749.69|11109.87|11446.39|3227.14| +2450836|76271|2450891|1214|832741|4268|29395|1214|832741|4268|29395|1|107|17|5|12886|289|1099|1|38.26|54.32|51.60|2.72|51.60|38.26|54.32|1.87|24.76|10.86|26.84|28.71|37.70|39.57|-11.42| +2450836|76271|2450895|1214|832741|4268|29395|1214|832741|4268|29395|1|28|12|1|6013|236|1099|70|22.46|64.90|12.33|3679.90|863.10|1572.20|4543.00|17.26|0.00|1180.90|863.10|880.36|2044.00|2061.26|-709.10| +2450836|7976|2450881|49556|431484|6659|38987|49556|431484|6659|38987|4|6|12|2|8245|134|1100|9|10.85|26.69|23.48|28.89|211.32|97.65|240.21|8.45|0.00|55.17|211.32|219.77|266.49|274.94|113.67| +2450836|7976|2450853|49556|431484|6659|38987|49556|431484|6659|38987|4|85|15|2|14576|291|1100|76|83.21|148.94|4.46|10980.48|338.96|6323.96|11319.44|20.33|0.00|4414.08|338.96|359.29|4753.04|4773.37|-5985.00| +2450836|7976|2450857|49556|431484|6659|38987|49556|431484|6659|38987|4|88|14|1|15391|295|1100|49|63.41|106.52|79.89|1304.87|3914.61|3107.09|5219.48|78.29|0.00|1931.09|3914.61|3992.90|5845.70|5923.99|807.52| +2450836|7976|2450891|49556|431484|6659|38987|49556|431484|6659|38987|4|27|3|3|5845|280|1100|89|9.36|20.59|13.79|605.20|1227.31|833.04|1832.51|110.45|0.00|640.80|1227.31|1337.76|1868.11|1978.56|394.27| +2450836|7976|2450906|49556|431484|6659|38987|49556|431484|6659|38987|4|33|14|1|17521|275|1100|76|90.23|138.05|74.54|4826.76|5665.04|6857.48|10491.80|113.30|0.00|1153.68|5665.04|5778.34|6818.72|6932.02|-1192.44| +2450836|7976|2450876|49556|431484|6659|38987|49556|431484|6659|38987|4|91|10|1|10981|116|1100|2|67.39|149.60|131.64|35.92|263.28|134.78|299.20|7.89|0.00|8.96|263.28|271.17|272.24|280.13|128.50| +2450836|7976|2450890|49556|431484|6659|38987|49556|431484|6659|38987|4|53|2|1|15796|132|1100|44|64.55|180.74|84.94|4215.20|3737.36|2840.20|7952.56|261.61|0.00|2783.00|3737.36|3998.97|6520.36|6781.97|897.16| +2450836|7976|2450856|49556|431484|6659|38987|49556|431484|6659|38987|4|100|2|5|8752|174|1100|98|61.02|73.22|23.43|4879.42|2296.14|5979.96|7175.56|24.79|2020.60|1793.40|275.54|300.33|2068.94|2093.73|-5704.42| +2450836|7976|2450871|49556|431484|6659|38987|49556|431484|6659|38987|4|9|7|1|2443|184|1100|40|45.66|100.45|3.01|3897.60|120.40|1826.40|4018.00|10.83|0.00|321.20|120.40|131.23|441.60|452.43|-1706.00| +2450836|7976|2450855|49556|431484|6659|38987|49556|431484|6659|38987|4|8|14|5|15380|60|1100|63|9.93|26.01|7.02|1196.37|442.26|625.59|1638.63|16.71|256.51|229.32|185.75|202.46|415.07|431.78|-439.84| +2450836|14654|2450891|42525|1354969|3860|16304|42525|1354969|3860|16304|4|58|10|2|13420|66|1101|57|23.74|38.93|29.58|532.95|1686.06|1353.18|2219.01|101.16|0.00|221.73|1686.06|1787.22|1907.79|2008.95|332.88| +2450836|14654|2450925|42525|1354969|3860|16304|42525|1354969|3860|16304|4|94|5|2|463|84|1101|24|51.24|101.45|9.13|2215.68|219.12|1229.76|2434.80|6.57|0.00|365.04|219.12|225.69|584.16|590.73|-1010.64| +2450836|14654|2450924|42525|1354969|3860|16304|42525|1354969|3860|16304|4|8|4|4|10894|166|1101|56|65.77|134.17|68.42|3682.00|3831.52|3683.12|7513.52|344.83|0.00|2854.88|3831.52|4176.35|6686.40|7031.23|148.40| +2450836|14654|2450853|42525|1354969|3860|16304|42525|1354969|3860|16304|4|6|16|4|6334|87|1101|74|51.47|105.51|46.42|4372.66|3435.08|3808.78|7807.74|100.99|1751.89|233.84|1683.19|1784.18|1917.03|2018.02|-2125.59| +2450836|14654|2450897|42525|1354969|3860|16304|42525|1354969|3860|16304|4|84|15|4|2623|107|1101|22|76.92|156.14|148.33|171.82|3263.26|1692.24|3435.08|261.06|0.00|790.02|3263.26|3524.32|4053.28|4314.34|1571.02| +2450836|59174|2450860|34406|13468|883|49145|34406|13468|883|49145|2|107|18|3|16552|162|1102|38|84.46|150.33|55.62|3598.98|2113.56|3209.48|5712.54|169.08|0.00|1942.18|2113.56|2282.64|4055.74|4224.82|-1095.92| +2450836|59174|2450850|34406|13468|883|49145|34406|13468|883|49145|2|89|13|1|8440|104|1102|37|25.97|54.79|6.02|1804.49|222.74|960.89|2027.23|0.00|0.00|607.91|222.74|222.74|830.65|830.65|-738.15| +2450836|59174|2450839|34406|13468|883|49145|34406|13468|883|49145|2|64|15|5|5044|225|1102|45|90.76|193.31|81.19|5045.40|3653.55|4084.20|8698.95|255.74|0.00|3131.55|3653.55|3909.29|6785.10|7040.84|-430.65| +2450836|59174|2450902|34406|13468|883|49145|34406|13468|883|49145|2|83|7|3|8059|42|1102|12|3.01|8.03|7.62|4.92|91.44|36.12|96.36|0.87|62.17|24.00|29.27|30.14|53.27|54.14|-6.85| +2450836|69036|2450849|57059|1858554|49|30697|57059|1858554|49|30697|2|101|6|5|17770|83|1103|90|13.76|20.64|20.02|55.80|1801.80|1238.40|1857.60|36.03|0.00|166.50|1801.80|1837.83|1968.30|2004.33|563.40| +2450836|69036|2450863|57059|1858554|49|30697|57059|1858554|49|30697|2|71|18|1|7666|46|1103|47|97.31|253.97|53.33|9430.08|2506.51|4573.57|11936.59|42.10|401.04|2864.65|2105.47|2147.57|4970.12|5012.22|-2468.10| +2450836|69036|2450921|57059|1858554|49|30697|57059|1858554|49|30697|2|21|11|1|7897|262|1103|97|51.28|130.76|117.68|1268.76|11414.96|4974.16|12683.72|0.00|6620.67|1902.17|4794.29|4794.29|6696.46|6696.46|-179.87| +2450836|69036|2450902|57059|1858554|49|30697|57059|1858554|49|30697|2|29|18|4|6274|173|1103|26|40.76|101.90|20.38|2119.52|529.88|1059.76|2649.40|10.59|0.00|370.76|529.88|540.47|900.64|911.23|-529.88| +2450836|69036|2450872|57059|1858554|49|30697|57059|1858554|49|30697|2|21|7|2|4742|94|1103|54|40.15|103.98|12.47|4941.54|673.38|2168.10|5614.92|8.48|552.17|729.54|121.21|129.69|850.75|859.23|-2046.89| +2450836|69036|2450866|57059|1858554|49|30697|57059|1858554|49|30697|2|72|14|3|3523|18|1103|97|83.51|91.86|50.52|4009.98|4900.44|8100.47|8910.42|441.03|0.00|3653.02|4900.44|5341.47|8553.46|8994.49|-3200.03| +2450836|64741|2450891|89040|150902|2811|25473|89040|150902|2811|25473|4|45|15|5|7024|67|1104|48|24.12|49.20|12.79|1747.68|613.92|1157.76|2361.60|30.69|0.00|1133.28|613.92|644.61|1747.20|1777.89|-543.84| +2450836|64741|2450886|89040|150902|2811|25473|89040|150902|2811|25473|4|55|20|4|13898|59|1104|16|45.14|77.64|53.57|385.12|857.12|722.24|1242.24|48.59|162.85|546.56|694.27|742.86|1240.83|1289.42|-27.97| +2450836|64741|2450896|89040|150902|2811|25473|89040|150902|2811|25473|4|96|8|2|10015|119|1104|44|63.90|76.04|63.87|535.48|2810.28|2811.60|3345.76|60.70|2135.81|1037.08|674.47|735.17|1711.55|1772.25|-2137.13| +2450836|64741|2450888|89040|150902|2811|25473|89040|150902|2811|25473|4|88|20|5|10162|269|1104|32|22.42|46.63|46.16|15.04|1477.12|717.44|1492.16|14.77|0.00|552.00|1477.12|1491.89|2029.12|2043.89|759.68| +2450836|64741|2450906|89040|150902|2811|25473|89040|150902|2811|25473|4|85|8|2|16162|132|1104|93|57.86|115.72|56.70|5488.86|5273.10|5380.98|10761.96|369.11|0.00|106.95|5273.10|5642.21|5380.05|5749.16|-107.88| +2450836|64741|2450925|89040|150902|2811|25473|89040|150902|2811|25473|4|9|12|3|9686|83|1104|91|61.31|150.20|37.55|10251.15|3417.05|5579.21|13668.20|71.75|2391.93|5193.37|1025.12|1096.87|6218.49|6290.24|-4554.09| +2450836|64741|2450913|89040|150902|2811|25473|89040|150902|2811|25473|4|15|15|3|15634|296|1104|51|98.09|245.22|29.42|11005.80|1500.42|5002.59|12506.22|135.03|0.00|2501.04|1500.42|1635.45|4001.46|4136.49|-3502.17| +2450836|64741|||150902|2811|||||25473|4|11||1|868||1104|47|29.93|69.73|44.62||2097.14||3277.31||629.14|163.56|1468.00||1631.56|1763.68|61.29| +2450836|64741|2450860|89040|150902|2811|25473|89040|150902|2811|25473|4|83|9|2|4634|138|1104|52|6.13|13.66|3.27|540.28|170.04|318.76|710.32|8.50|0.00|170.04|170.04|178.54|340.08|348.58|-148.72| +2450836|64741|2450885|89040|150902|2811|25473|89040|150902|2811|25473|4|91|14|5|15043|287|1104|96|57.38|83.77|9.21|7157.76|884.16|5508.48|8041.92|70.73|0.00|1527.36|884.16|954.89|2411.52|2482.25|-4624.32| +2450836|35119|2450848|79228|1004777|726|47475|79228|1004777|726|47475|2|10|13|3|14764|91|1105|100|70.42|85.20|84.34|86.00|8434.00|7042.00|8520.00|168.68|0.00|3408.00|8434.00|8602.68|11842.00|12010.68|1392.00| +2450836|35119|2450897|79228|1004777|726|47475|79228|1004777|726|47475|2|106|11|2|16406|181|1105|4|24.95|33.68|27.95|22.92|111.80|99.80|134.72|5.59|0.00|0.00|111.80|117.39|111.80|117.39|12.00| +|||79228||726|||1004777|726|||||5|7568|262|1105|64|||27.97|842.88|1790.08||2632.96|0.00|1288.85|1184.64||501.23|1685.87|1685.87|-679.57| +2450836|35119|2450909|79228|1004777|726|47475|79228|1004777|726|47475|2|102|15|4|16891|162|1105|57|42.37|87.28|1.74|4875.78|99.18|2415.09|4974.96|1.31|80.33|347.70|18.85|20.16|366.55|367.86|-2396.24| +2450836|35119|2450923|79228|1004777|726|47475|79228|1004777|726|47475|2|34|2|2|854|92|1105|4|90.86|177.17|58.46|474.84|233.84|363.44|708.68|21.04|0.00|92.12|233.84|254.88|325.96|347.00|-129.60| +2450836|35119|2450853|79228|1004777|726|47475|79228|1004777|726|47475|2|103|18|4|16304|116|1105|13|74.54|137.15|123.43|178.36|1604.59|969.02|1782.95|128.36|0.00|837.98|1604.59|1732.95|2442.57|2570.93|635.57| +2450836|35119|2450882|79228|1004777|726|47475|79228|1004777|726|47475|2|52|10|4|8740|60|1105|81|88.00|102.96|35.00|5504.76|2835.00|7128.00|8339.76|79.09|1956.15|3836.16|878.85|957.94|4715.01|4794.10|-6249.15| +2450836|859|2450864|1673|1486618|5610|7355|1673|1486618|5610|7355|4|81|12|3|176|106|1106|13|52.77|70.18|28.07|547.43|364.91|686.01|912.34|0.00|0.00|200.59|364.91|364.91|565.50|565.50|-321.10| +2450836|859|2450905|1673|1486618|5610|7355|1673|1486618|5610|7355|4|52|13|5|13988|289|1106|33|47.28|88.41|66.30|729.63|2187.90|1560.24|2917.53|58.63|722.00|525.03|1465.90|1524.53|1990.93|2049.56|-94.34| +2450836|859|2450898|1673|1486618|5610|7355|1673|1486618|5610|7355|4|75|3|3|7789|295|1106|7|9.94|14.41|7.92|45.43|55.44|69.58|100.87|4.98|0.00|6.02|55.44|60.42|61.46|66.44|-14.14| +2450836|859|2450890|1673|1486618|5610|7355|1673|1486618|5610|7355|4|47|1|3|4939|264|1106|38|5.01|10.07|7.75|88.16|294.50|190.38|382.66|0.61|232.65|141.36|61.85|62.46|203.21|203.82|-128.53| +2450836|859|2450916|1673|1486618|5610|7355|1673|1486618|5610|7355|4|59|10|1|12458|216|1106|81|10.96|12.16|4.49|621.27|363.69|887.76|984.96|12.51|50.91|225.99|312.78|325.29|538.77|551.28|-574.98| +2450836|859|2450847|1673|1486618|5610|7355|1673|1486618|5610|7355|4|44|7|2|2203|32|1106|89|72.87|179.26|51.98|11327.92|4626.22|6485.43|15954.14|262.30|878.98|3509.27|3747.24|4009.54|7256.51|7518.81|-2738.19| +2450836|859|2450905|1673|1486618|5610|7355|1673|1486618|5610|7355|4|26|8|1|5570|170|1106|88|3.55|3.94|0.94|264.00|82.72|312.40|346.72|6.61|0.00|9.68|82.72|89.33|92.40|99.01|-229.68| +2450836|859|2450841|1673|1486618|5610|7355|1673|1486618|5610|7355|4|12|14|1|12829|142|1106|70|52.68|63.21|42.98|1416.10|3008.60|3687.60|4424.70|270.77|0.00|707.70|3008.60|3279.37|3716.30|3987.07|-679.00| +2450836|78356|2450877|5130|1247646|2609|19125|5130|1247646|2609|19125|1|91|16|2|1399|160|1107|100|1.93|3.24|2.94|30.00|294.00|193.00|324.00|8.82|0.00|61.00|294.00|302.82|355.00|363.82|101.00| +2450836|78356|2450923|5130|1247646|2609|19125|5130|1247646|2609|19125|1|63|6|1|5611|96|1107|88|60.96|140.81|68.99|6320.16|6071.12|5364.48|12391.28|303.55|0.00|742.72|6071.12|6374.67|6813.84|7117.39|706.64| +2450836|78356|2450840|5130|1247646|2609|19125|5130|1247646|2609|19125|1|104|18|1|12415|272|1107|30|68.39|168.92|76.01|2787.30|2280.30|2051.70|5067.60|91.21|0.00|1722.90|2280.30|2371.51|4003.20|4094.41|228.60| +2450836|78356|2450841|5130|1247646|2609|19125|5130|1247646|2609|19125|1|83|14|2|2252|288|1107|47|14.66|19.20|10.36|415.48|486.92|689.02|902.40|4.86|0.00|415.01|486.92|491.78|901.93|906.79|-202.10| +2450836|78356|2450867|5130|1247646|2609|19125|5130|1247646|2609|19125|1|34|5|5|8536|223|1107|44|43.71|45.89|0.00|2019.16|0.00|1923.24|2019.16|0.00|0.00|847.88|0.00|0.00|847.88|847.88|-1923.24| +2450836|78356|2450844|5130|1247646|2609|19125|5130|1247646|2609|19125|1|62|6|2|15146|162|1107|100|82.98|125.29|90.20|3509.00|9020.00|8298.00|12529.00|81.18|6314.00|2881.00|2706.00|2787.18|5587.00|5668.18|-5592.00| +2450836|78356|2450841|5130|1247646|2609|19125|5130|1247646|2609|19125|1|46|5|1|17839|95|1107|43|90.17|112.71|22.54|3877.31|969.22|3877.31|4846.53|38.76|0.00|1599.17|969.22|1007.98|2568.39|2607.15|-2908.09| +2450836|78356|2450888|5130|1247646|2609|19125|5130|1247646|2609|19125|1|65|7|5|8872|238|1107|53|36.49|75.16|12.77|3306.67|676.81|1933.97|3983.48|5.07|507.60|1314.40|169.21|174.28|1483.61|1488.68|-1764.76| +2450836|78356|2450907|5130|1247646|2609|19125|5130|1247646|2609|19125|1|92|4|1|11641|190|1107|69|8.93|21.87|16.83|347.76|1161.27|616.17|1509.03|23.22|0.00|286.35|1161.27|1184.49|1447.62|1470.84|545.10| +2450836|78356|2450849|5130|1247646|2609|19125|5130|1247646|2609|19125|1|97|20|3|38|17|1107|73|99.79|166.64|89.98|5596.18|6568.54|7284.67|12164.72|525.48|0.00|4622.36|6568.54|7094.02|11190.90|11716.38|-716.13| +2450836|78356|2450877|5130|1247646|2609|19125|5130|1247646|2609|19125|1|24|5|1|4196|220|1107|87|34.46|63.06|31.53|2743.11|2743.11|2998.02|5486.22|109.72|1371.55|1699.98|1371.56|1481.28|3071.54|3181.26|-1626.46| +2450836|78356|2450926|5130|1247646|2609|19125|5130|1247646|2609|19125|1|97|8|1|9370|78|1107|74|44.88|106.81|86.51|1502.20|6401.74|3321.12|7903.94|512.13|0.00|473.60|6401.74|6913.87|6875.34|7387.47|3080.62| +2450836|78356|2450893|5130|1247646|2609|19125|5130|1247646|2609|19125|1|101|8|1|14377|272|1107|51|40.86|95.20|31.41|3253.29|1601.91|2083.86|4855.20|84.58|544.64|1844.67|1057.27|1141.85|2901.94|2986.52|-1026.59| +2450836|78356|2450907|5130|1247646|2609|19125|5130|1247646|2609|19125|1|55|2|2|12961|140|1107|77|14.05|17.56|10.18|568.26|783.86|1081.85|1352.12|15.67|0.00|229.46|783.86|799.53|1013.32|1028.99|-297.99| +2450836|61678|2450847|5836|1363181|5326|20671|5836|1363181|5326|20671|2|33|11|5|6878|80|1108|76|69.95|183.26|164.93|1393.08|12534.68|5316.20|13927.76|100.27|11281.21|5152.80|1253.47|1353.74|6406.27|6506.54|-4062.73| +2450836|61678|2450861|5836|1363181|5326|20671|5836|1363181|5326|20671|2|69|19|2|12578|8|1108|66|12.07|14.36|1.14|872.52|75.24|796.62|947.76|1.50|0.00|302.94|75.24|76.74|378.18|379.68|-721.38| +2450836|61678|2450887|5836|1363181|5326|20671|5836|1363181|5326|20671|2|67|9|3|2371|7|1108|75|52.51|98.19|0.98|7290.75|73.50|3938.25|7364.25|0.73|0.00|294.00|73.50|74.23|367.50|368.23|-3864.75| +2450836|61678|2450896|5836|1363181|5326|20671|5836|1363181|5326|20671|2|23|11|5|7462|163|1108|64|70.91|102.81|17.47|5461.76|1118.08|4538.24|6579.84|33.54|0.00|2302.72|1118.08|1151.62|3420.80|3454.34|-3420.16| +2450836|61678|2450840|5836|1363181|5326|20671|5836|1363181|5326|20671|2|89|19|2|4351|268|1108|12|88.98|113.00|93.79|230.52|1125.48|1067.76|1356.00|63.81|416.42|244.08|709.06|772.87|953.14|1016.95|-358.70| +2450836|61678|2450918|5836|1363181|5326|20671|5836|1363181|5326|20671|2|68|10|4|5179|132|1108|87|64.17|109.73|48.28|5346.15|4200.36|5582.79|9546.51|336.02|0.00|4582.29|4200.36|4536.38|8782.65|9118.67|-1382.43| +2450836|31151|2450909|94170|1846664|3158|18899|94170|1846664|3158|18899|1|37|16|3|8491|29|1109|78|60.98|118.91|49.94|5379.66|3895.32|4756.44|9274.98|0.00|0.00|1390.74|3895.32|3895.32|5286.06|5286.06|-861.12| +2450836|31151|2450899|94170|1846664|3158|18899|94170|1846664|3158|18899|1|36|20|4|15112|143|1109|69|19.42|48.93|22.99|1789.86|1586.31|1339.98|3376.17|47.58|0.00|1484.88|1586.31|1633.89|3071.19|3118.77|246.33| +2450836|31151|2450849|94170|1846664|3158|18899|94170|1846664|3158|18899|1|96|7|3|3392|204|1109|65|19.76|22.72|5.45|1122.55|354.25|1284.40|1476.80|31.88|0.00|58.50|354.25|386.13|412.75|444.63|-930.15| +2450836|31151|2450866|94170|1846664|3158|18899|94170|1846664|3158|18899|1|1|4|3|4768|116|1109|27|27.18|45.39|42.21|85.86|1139.67|733.86|1225.53|79.77|0.00|183.60|1139.67|1219.44|1323.27|1403.04|405.81| +2450836|31151|2450879|94170|1846664|3158|18899|94170|1846664|3158|18899|1|33|20|2|8438|119|1109|96|3.09|8.93|3.39|531.84|325.44|296.64|857.28|0.00|0.00|127.68|325.44|325.44|453.12|453.12|28.80| +2450836|31151|2450908|94170|1846664|3158|18899|94170|1846664|3158|18899|1|102|3|2|4906|21|1109|29|49.08|114.35|114.35|0.00|3316.15|1423.32|3316.15|198.96|0.00|99.47|3316.15|3515.11|3415.62|3614.58|1892.83| +|||||||||||1||4||6793|137|1109|76|1.10|2.67|2.02|49.40|153.52||202.92||0.00||153.52||209.76|220.50|69.92| +2450836|31151|2450926|94170|1846664|3158|18899|94170|1846664|3158|18899|1|1|1|4|3622|85|1109|64|63.82|176.78|0.00|11313.92|0.00|4084.48|11313.92|0.00|0.00|339.20|0.00|0.00|339.20|339.20|-4084.48| +2450836|31151|2450922|94170|1846664|3158|18899|94170|1846664|3158|18899|1|33|2|2|9926|237|1109|76|24.87|63.66|56.02|580.64|4257.52|1890.12|4838.16|298.02|0.00|1257.80|4257.52|4555.54|5515.32|5813.34|2367.40| +2450836|52476|2450871|69892|1813816|5293|2370|69892|1813816|5293|2370|2|70|15|2|8966|189|1110|68|38.41|97.94|49.94|3264.00|3395.92|2611.88|6659.92|305.63|0.00|65.96|3395.92|3701.55|3461.88|3767.51|784.04| +2450836|52476|2450845|69892|1813816|5293|2370|69892|1813816|5293|2370|2|9|8|1|13357|263|1110|18|10.60|12.19|8.53|65.88|153.54|190.80|219.42|9.21|0.00|45.90|153.54|162.75|199.44|208.65|-37.26| +2450836|52476|2450888|69892|1813816|5293|2370|69892|1813816|5293|2370|2|103|3|1|14324|230|1110|77|14.94|14.94|11.35|276.43|873.95|1150.38|1150.38|78.65|0.00|217.91|873.95|952.60|1091.86|1170.51|-276.43| +2450836|52476|2450906|69892|1813816|5293|2370|69892|1813816|5293|2370|2|99|17|4|1636|3|1110|67|95.44|158.43|15.84|9553.53|1061.28|6394.48|10614.81|30.56|551.86|1061.28|509.42|539.98|1570.70|1601.26|-5885.06| +2450836|52476|2450854|69892|1813816|5293|2370|69892|1813816|5293|2370|2|16|14|3|10376|196|1110|61|8.64|10.28|7.19|188.49|438.59|527.04|627.08|8.77|0.00|218.99|438.59|447.36|657.58|666.35|-88.45| +2450836|52476|2450913|69892|1813816|5293|2370|69892|1813816|5293|2370|2|40|7|5|15518|114|1110|83|43.03|62.39|3.74|4867.95|310.42|3571.49|5178.37|27.93|0.00|2226.06|310.42|338.35|2536.48|2564.41|-3261.07| +2450836|52476|2450910|69892|1813816|5293|2370|69892|1813816|5293|2370|2|73|1|4|8737|137|1110|96|99.17|142.80|119.95|2193.60|11515.20|9520.32|13708.80|6.90|11400.04|1781.76|115.16|122.06|1896.92|1903.82|-9405.16| +2450836|52476|2450918|69892|1813816|5293|2370|69892|1813816|5293|2370|2|95|2|4|7357|157|1110|69|22.81|44.25|7.96|2504.01|549.24|1573.89|3053.25|16.47|0.00|1129.53|549.24|565.71|1678.77|1695.24|-1024.65| +2450836|52476|2450850|69892|1813816|5293|2370|69892|1813816|5293|2370|2|89|4|3|6121|168|1110|51|24.99|33.23|29.24|203.49|1491.24|1274.49|1694.73|89.47|0.00|762.45|1491.24|1580.71|2253.69|2343.16|216.75| +2450836|52476|2450848|69892|1813816|5293|2370|69892|1813816|5293|2370|2|35|10|5|15328|11|1110|19|26.51|71.04|34.80|688.56|661.20|503.69|1349.76|13.88|198.36|269.80|462.84|476.72|732.64|746.52|-40.85| +2450836|52476|2450879|69892|1813816|5293|2370|69892|1813816|5293|2370|2|62|3|4|15139|24|1110|65|20.84|48.76|28.28|1331.20|1838.20|1354.60|3169.40|94.85|257.34|982.15|1580.86|1675.71|2563.01|2657.86|226.26| +2450836|32394|2450841|5238|1441253|115|16130|5238|1441253|115|16130|1|41|5|4|12028|174|1111|23|53.72|74.67|20.90|1236.71|480.70|1235.56|1717.41|19.22|0.00|772.80|480.70|499.92|1253.50|1272.72|-754.86| +2450836|32394|2450878|5238|1441253|115|16130|5238|1441253|115|16130|1|26|4|5|14798|49|1111|50|98.37|290.19|203.13|4353.00|10156.50|4918.50|14509.50|507.82|0.00|1450.50|10156.50|10664.32|11607.00|12114.82|5238.00| +2450836|32394|2450839|5238|1441253|115|16130|5238|1441253|115|16130|1|11|8|2|13678|223|1111|74|71.11|152.17|6.08|10810.66|449.92|5262.14|11260.58|8.99|0.00|1463.72|449.92|458.91|1913.64|1922.63|-4812.22| +2450836|32394|2450901|5238|1441253|115|16130|5238|1441253|115|16130|1|41|1|5|9847|161|1111|67|98.06|110.80|48.75|4157.35|3266.25|6570.02|7423.60|130.65|0.00|1558.42|3266.25|3396.90|4824.67|4955.32|-3303.77| +2450836|32394|2450894|5238|1441253|115|16130|5238|1441253|115|16130|1|76|10|5|7978|150|1111|85|50.71|119.16|46.47|6178.65|3949.95|4310.35|10128.60|39.49|0.00|606.90|3949.95|3989.44|4556.85|4596.34|-360.40| +2450836|32394|2450865|5238|1441253|115|16130|5238|1441253|115|16130|1|69|12|4|2842|168|1111|18|99.43|254.54|251.99|45.90|4535.82|1789.74|4581.72|45.35|0.00|1328.58|4535.82|4581.17|5864.40|5909.75|2746.08| +2450836|32394|2450856|5238|1441253|115|16130|5238|1441253|115|16130|1|66|20|2|5282|277|1111|6|89.74|227.93|98.00|779.58|588.00|538.44|1367.58|23.52|0.00|109.38|588.00|611.52|697.38|720.90|49.56| +2450836|32394|2450914|5238|1441253|115|16130|5238|1441253|115|16130|1|62|3|2|17707|4|1111|96|46.40|117.39|102.12|1465.92|9803.52|4454.40|11269.44|686.24|0.00|901.44|9803.52|10489.76|10704.96|11391.20|5349.12| +2450836|32394|2450868|5238|1441253|115|16130|5238|1441253|115|16130|1|25|15|1|14482|73|1111|68|82.85|125.93|98.22|1884.28|6678.96|5633.80|8563.24|42.07|2471.21|2397.68|4207.75|4249.82|6605.43|6647.50|-1426.05| +2450836|32394|2450858|5238|1441253|115|16130|5238|1441253|115|16130|1|18|10|3|13256|118|1111|37|82.81|203.71|6.11|7311.20|226.07|3063.97|7537.27|15.82|0.00|828.80|226.07|241.89|1054.87|1070.69|-2837.90| +2450836|32394|2450896|5238|1441253|115|16130|5238|1441253|115|16130|1|23|18|5|3796|62|1111|90|92.80|95.58|45.87|4473.90|4128.30|8352.00|8602.20|123.84|0.00|1978.20|4128.30|4252.14|6106.50|6230.34|-4223.70| +2450836|32394|2450918|5238|1441253|115|16130|5238|1441253|115|16130|1|60|2|5|2450|165|1111|80|28.55|81.08|61.62|1556.80|4929.60|2284.00|6486.40|246.48|0.00|2204.80|4929.60|5176.08|7134.40|7380.88|2645.60| +2450836|75350|2450890|36556|1303125|5694|36710|36556|1303125|5694|36710|4|26|5|3|1108|116|1112|4|13.73|23.61|14.63|35.92|58.52|54.92|94.44|0.58|0.00|43.44|58.52|59.10|101.96|102.54|3.60| +2450836|75350|2450840|36556|1303125|5694|36710|36556|1303125|5694|36710|4|52|16|5|17236|63|1112|37|22.76|30.95|2.47|1053.76|91.39|842.12|1145.15|5.48|0.00|389.24|91.39|96.87|480.63|486.11|-750.73| +2450836|75350|2450889|36556|1303125|5694|36710|36556|1303125|5694|36710|4|30|16|2|3331|57|1112|54|85.83|120.16|112.95|389.34|6099.30|4634.82|6488.64|51.23|4391.49|2595.24|1707.81|1759.04|4303.05|4354.28|-2927.01| +2450836|75350|2450916|36556|1303125|5694|36710|36556|1303125|5694|36710|4|43|14|4|4408|224|1112|12|21.66|29.45|7.95|258.00|95.40|259.92|353.40|5.72|0.00|130.68|95.40|101.12|226.08|231.80|-164.52| +2450836|75350|2450911|36556|1303125|5694|36710|36556|1303125|5694|36710|4|55|12|1|13831|224|1112|19|53.43|110.60|44.24|1260.84|840.56|1015.17|2101.40|16.81|0.00|630.42|840.56|857.37|1470.98|1487.79|-174.61| +2450836|75350|2450875|36556|1303125|5694|36710|36556|1303125|5694|36710|4|3|13|1|9248|123|1112|55|75.52|181.24|154.05|1495.45|8472.75|4153.60|9968.20|423.63|0.00|2591.60|8472.75|8896.38|11064.35|11487.98|4319.15| +2450836|75350|2450878|36556|1303125|5694|36710|36556|1303125|5694|36710|4|27|8|4|15668|168|1112|66|54.15|132.66|115.41|1138.50|7617.06|3573.90|8755.56|351.90|2589.80|2539.02|5027.26|5379.16|7566.28|7918.18|1453.36| +2450836|55135|2450850|71630|1856196|1562|37328|71630|1856196|1562|37328|2|65|4|4|11362|231|1113|23|60.43|138.98|136.20|63.94|3132.60|1389.89|3196.54|62.65|0.00|191.59|3132.60|3195.25|3324.19|3386.84|1742.71| +2450836|55135|2450838|71630|1856196|1562|37328|71630|1856196|1562|37328|2|46|18|2|17503|227|1113|87|78.05|193.56|63.87|11283.03|5556.69|6790.35|16839.72|166.70|0.00|3367.77|5556.69|5723.39|8924.46|9091.16|-1233.66| +2450836|55135|2450898|71630|1856196|1562|37328|71630|1856196|1562|37328|2|108|10|1|8312|208|1113|65|3.43|4.18|3.00|76.70|195.00|222.95|271.70|15.60|0.00|2.60|195.00|210.60|197.60|213.20|-27.95| +2450836|55135|2450855|71630|1856196|1562|37328|71630|1856196|1562|37328|2|21|11|2|1180|267|1113|22|87.81|210.74|120.12|1993.64|2642.64|1931.82|4636.28|38.05|1374.17|185.24|1268.47|1306.52|1453.71|1491.76|-663.35| +2450836|55135|2450902|71630|1856196|1562|37328|71630|1856196|1562|37328|2|61|3|2|15454|149|1113|9|39.84|64.54|41.30|209.16|371.70|358.56|580.86|11.15|0.00|52.20|371.70|382.85|423.90|435.05|13.14| +2450836|55135|2450890|71630|1856196|1562|37328|71630|1856196|1562|37328|2|57|8|4|4213|91|1113|49|35.35|82.01|59.04|1125.53|2892.96|1732.15|4018.49|58.14|954.67|241.08|1938.29|1996.43|2179.37|2237.51|206.14| +2450836|55135|2450858|71630|1856196|1562|37328|71630|1856196|1562|37328|2|38|16|1|3934|183|1113|42|8.06|19.10|4.20|625.80|176.40|338.52|802.20|1.65|10.58|248.64|165.82|167.47|414.46|416.11|-172.70| +2450836|55135|2450841|71630|1856196|1562|37328|71630|1856196|1562|37328|2|28|16|5|16558|180|1113|58|40.67|77.67|67.57|585.80|3919.06|2358.86|4504.86|39.19|0.00|1035.88|3919.06|3958.25|4954.94|4994.13|1560.20| +2450836|55135|2450909|71630|1856196|1562|37328|71630|1856196|1562|37328|2|61|4|4|16651|177|1113|16|57.72|66.37|61.72|74.40|987.52|923.52|1061.92|19.75|0.00|127.36|987.52|1007.27|1114.88|1134.63|64.00| +2450836|25839|2450880|18929|968597|3512|32427|18929|968597|3512|32427|2|62|12|3|8126|25|1114|33|78.20|117.30|110.26|232.32|3638.58|2580.60|3870.90|291.08|0.00|115.83|3638.58|3929.66|3754.41|4045.49|1057.98| +2450836|25839|2450907|18929|968597|3512|32427|18929|968597|3512|32427|2|87|3|5|7712|106|1114|20|55.99|125.41|1.25|2483.20|25.00|1119.80|2508.20|0.26|19.75|75.20|5.25|5.51|80.45|80.71|-1114.55| +2450836|25839|2450860|18929|968597|3512|32427|18929|968597|3512|32427|2|11|1|5|5266|245|1114|30|27.72|45.73|22.40|699.90|672.00|831.60|1371.90|53.76|0.00|630.90|672.00|725.76|1302.90|1356.66|-159.60| +2450836|25839|2450904|18929|968597|3512|32427|18929|968597|3512|32427|2|72|14|1|13249|23|1114|96|35.32|86.88|19.11|6505.92|1834.56|3390.72|8340.48|36.69|0.00|500.16|1834.56|1871.25|2334.72|2371.41|-1556.16| +2450836|25839|2450862|18929|968597|3512|32427|18929|968597|3512|32427|2|29|18|2|15943|7|1114|52|18.17|40.33|13.30|1405.56|691.60|944.84|2097.16|48.41|0.00|377.00|691.60|740.01|1068.60|1117.01|-253.24| +2450836|25839|2450867|18929|968597|3512|32427|18929|968597|3512|32427|2|37|18|1|2485|207|1114|1|27.20|31.00|8.06|22.94|8.06|27.20|31.00|0.40|0.00|0.93|8.06|8.46|8.99|9.39|-19.14| +2450836|25839|2450870|18929|968597|3512|32427|18929|968597|3512|32427|2|65|16|1|6818|61|1114|4|70.88|113.40|32.88|322.08|131.52|283.52|453.60|10.52|0.00|204.12|131.52|142.04|335.64|346.16|-152.00| +2450836|25839|2450861|18929|968597|3512|32427|18929|968597|3512|32427|2|33|12|4|3368|62|1114|96|45.61|74.80|5.23|6678.72|502.08|4378.56|7180.80|30.72|160.66|646.08|341.42|372.14|987.50|1018.22|-4037.14| +||||1340857|792|24727|20922|1340857|||2|34|11||6001|24|1115||77.36|||7239.00|||||0.00||380.50||1523.00|1534.41|-3487.50| +2450836|42023|2450864|20922|1340857|792|24727|20922|1340857|792|24727|2|71|16|3|17863|47|1115|14|62.00|138.26|73.27|909.86|1025.78|868.00|1935.64|71.80|0.00|425.74|1025.78|1097.58|1451.52|1523.32|157.78| +2450836|42023|2450860|20922|1340857|792|24727|20922|1340857|792|24727|2|76|18|2|3115|67|1115|90|19.12|32.88|8.54|2190.60|768.60|1720.80|2959.20|7.68|0.00|0.00|768.60|776.28|768.60|776.28|-952.20| +2450836|42023|2450923|20922|1340857|792|24727|20922|1340857|792|24727|2|77|8|3|13712|30|1115|73|10.03|20.16|4.03|1177.49|294.19|732.19|1471.68|12.79|38.24|161.33|255.95|268.74|417.28|430.07|-476.24| +2450836|42023|2450877|20922|1340857|792|24727|20922|1340857|792|24727|2|47|16|3|190|288|1115|3|22.27|50.10|44.58|16.56|133.74|66.81|150.30|0.00|0.00|66.12|133.74|133.74|199.86|199.86|66.93| +2450836|42023|2450873|20922|1340857|792|24727|20922|1340857|792|24727|2|21|15|2|13466|73|1115|53|8.36|15.38|11.38|212.00|603.14|443.08|815.14|36.18|0.00|0.00|603.14|639.32|603.14|639.32|160.06| +2450836|42023|2450895|20922|1340857|792|24727|20922|1340857|792|24727|2|81|13|4|1852|53|1115|53|33.25|57.85|17.93|2115.76|950.29|1762.25|3066.05|0.00|0.00|122.43|950.29|950.29|1072.72|1072.72|-811.96| +2450836|53657|2450859|11358|309877|3149|21268|11358|309877|3149|21268|1|15|8|4|7411|104|1116|65|88.81|172.29|53.40|7727.85|3471.00|5772.65|11198.85|34.71|0.00|3695.25|3471.00|3505.71|7166.25|7200.96|-2301.65| +2450836|53657|2450874|11358|309877|3149|21268|11358|309877|3149|21268|1|98|3|3|9493|146|1116|25|91.93|181.10|90.55|2263.75|2263.75|2298.25|4527.50|33.50|1426.16|0.00|837.59|871.09|837.59|871.09|-1460.66| +2450836|53657|2450917|11358|309877|3149|21268|11358|309877|3149|21268|1|49|9|3|13156|101|1116|22|8.47|10.16|2.94|158.84|64.68|186.34|223.52|2.58|0.00|91.52|64.68|67.26|156.20|158.78|-121.66| +2450836|53657|2450894|11358|309877|3149|21268|11358|309877|3149|21268|1|67|2|2|16009|222|1116|20|32.90|79.94|20.78|1183.20|415.60|658.00|1598.80|24.93|0.00|735.40|415.60|440.53|1151.00|1175.93|-242.40| +2450836|22218|2450911|64168|1761588|3361|1581|64168|1761588|3361|1581|4|43|15|2|12619|92|1117|46|31.48|64.84|9.07|2565.42|417.22|1448.08|2982.64|29.20|0.00|447.12|417.22|446.42|864.34|893.54|-1030.86| +2450836|22218|2450911|64168|1761588|3361|1581|64168|1761588|3361|1581|4|48|12|2|15331|231|1117|35|82.55|145.28|97.33|1678.25|3406.55|2889.25|5084.80|136.26|0.00|1982.75|3406.55|3542.81|5389.30|5525.56|517.30| +2450836|22218|2450910|64168|1761588|3361|1581|64168|1761588|3361|1581|4|76|12|3|2017|160|1117|75|58.11|94.13|58.36|2682.75|4377.00|4358.25|7059.75|131.31|0.00|1411.50|4377.00|4508.31|5788.50|5919.81|18.75| +2450836|22218|2450883|64168|1761588|3361|1581|64168|1761588|3361|1581|4|34|5|5|5398|238|1117|50|6.16|16.01|2.72|664.50|136.00|308.00|800.50|10.88|0.00|256.00|136.00|146.88|392.00|402.88|-172.00| +2450836|22218|2450888|64168|1761588|3361|1581|64168|1761588|3361|1581|4|89|13|5|17248|104|1117|85|3.74|8.45|6.42|172.55|545.70|317.90|718.25|27.28|0.00|272.85|545.70|572.98|818.55|845.83|227.80| +2450836|22218|2450840|64168|1761588|3361|1581|64168|1761588|3361|1581|4|104|8|2|811|170|1117|9|27.65|59.44|49.92|85.68|449.28|248.85|534.96|12.13|314.49|171.18|134.79|146.92|305.97|318.10|-114.06| +2450836|22218|2450896|64168|1761588|3361|1581|64168|1761588|3361|1581|4|40|3|1|7072|37|1117|80|55.01|80.31|56.21|1928.00|4496.80|4400.80|6424.80|44.96|0.00|1412.80|4496.80|4541.76|5909.60|5954.56|96.00| +2450836|22218|2450850|64168|1761588|3361|1581|64168|1761588|3361|1581|4|63|3|3|12452|33|1117|22|90.28|175.14|162.88|269.72|3583.36|1986.16|3853.08|71.66|0.00|1464.10|3583.36|3655.02|5047.46|5119.12|1597.20| +2450836|28506|2450844|99314|1676072|4830|48341|99314|1676072|4830|48341|1|14|13|2|10162|148|1118|59|43.59|122.05|98.86|1368.21|5832.74|2571.81|7200.95|233.30|0.00|3312.26|5832.74|6066.04|9145.00|9378.30|3260.93| +2450836|28506|2450910|99314|1676072|4830|48341|99314|1676072|4830|48341|1|3|5|2|16162|136|1118|69|33.47|83.34|15.00|4715.46|1035.00|2309.43|5750.46|93.15|0.00|2012.04|1035.00|1128.15|3047.04|3140.19|-1274.43| +2450836|28506|2450867|99314|1676072|4830|48341|99314|1676072|4830|48341|1|24|7|2|9686|257|1118|84|21.00|24.57|2.45|1858.08|205.80|1764.00|2063.88|16.46|0.00|535.92|205.80|222.26|741.72|758.18|-1558.20| +2450836|28506|2450894|99314|1676072|4830|48341|99314|1676072|4830|48341|1|23|2|2|15634|278|1118|91|61.71|152.42|38.10|10403.12|3467.10|5615.61|13870.22|208.02|0.00|3744.65|3467.10|3675.12|7211.75|7419.77|-2148.51| +2450836|28506|2450868|99314|1676072|4830|48341|99314|1676072|4830|48341|1|31|2|2|868|278|1118|2|3.87|7.62|3.42|8.40|6.84|7.74|15.24|0.00|0.00|1.82|6.84|6.84|8.66|8.66|-0.90| +2450836|28506|2450842|99314|1676072|4830|48341|99314|1676072|4830|48341|1|10|10|3|4634|159|1118|78|9.66|14.97|6.43|666.12|501.54|753.48|1167.66|0.00|320.98|186.42|180.56|180.56|366.98|366.98|-572.92| +2450836|28506|2450885|99314|1676072|4830|48341|99314|1676072|4830|48341|1|88|14|1|15043|172|1118|12|77.26|220.19|94.68|1506.12|1136.16|927.12|2642.28|11.36|0.00|237.72|1136.16|1147.52|1373.88|1385.24|209.04| +2450836|28506|2450911|99314|1676072|4830|48341|99314|1676072|4830|48341|1|48|18|4|1654|160|1118|33|8.11|10.70|9.30|46.20|306.90|267.63|353.10|3.31|251.65|28.05|55.25|58.56|83.30|86.61|-212.38| +2450836|28506|2450857|99314|1676072|4830|48341|99314|1676072|4830|48341|1|89|15|5|337|215|1118|57|32.95|50.08|19.53|1741.35|1113.21|1878.15|2854.56|18.03|512.07|1170.21|601.14|619.17|1771.35|1789.38|-1277.01| +2450836|28506|2450920|99314|1676072|4830|48341|99314|1676072|4830|48341|1|30|2|1|14570|289|1118|1|38.62|67.19|53.75|13.44|53.75|38.62|67.19|3.74|6.98|29.56|46.77|50.51|76.33|80.07|8.15| +2450836|50387|2450912|12846|1666089|3685|40128|12846|1666089|3685|40128|1|69|19|2|1300|59|1119|97|99.24|276.87|163.35|11011.44|15844.95|9626.28|26856.39|475.34|0.00|7519.44|15844.95|16320.29|23364.39|23839.73|6218.67| +2450836|50387|2450889|12846|1666089|3685|40128|12846|1666089|3685|40128|1|22|5|4|9763|54|1119|22|90.89|270.85|108.34|3575.22|2383.48|1999.58|5958.70|119.17|0.00|357.50|2383.48|2502.65|2740.98|2860.15|383.90| +2450836|50387|2450917|12846|1666089|3685|40128|12846|1666089|3685|40128|1|31|4|2|14317|203|1119|7|32.83|54.16|18.95|246.47|132.65|229.81|379.12|11.93|0.00|151.62|132.65|144.58|284.27|296.20|-97.16| +2450836|50387|2450869|12846|1666089|3685|40128|12846|1666089|3685|40128|1|1|9|1|10492|227|1119|57|48.49|66.43|29.89|2082.78|1703.73|2763.93|3786.51|119.26|0.00|1476.30|1703.73|1822.99|3180.03|3299.29|-1060.20| +2450836|50387|2450854|12846|1666089|3685|40128|12846|1666089|3685|40128|1|16|16|1|727|171|1119|27|32.99|73.56|61.79|317.79|1668.33|890.73|1986.12|4.67|1601.59|973.08|66.74|71.41|1039.82|1044.49|-823.99| +2450836|50387|2450887|12846|1666089|3685|40128|12846|1666089|3685|40128|1|14|13|5|14860|206|1119|94|9.33|12.78|6.39|600.66|600.66|877.02|1201.32|6.00|0.00|108.10|600.66|606.66|708.76|714.76|-276.36| +2450836|50387|2450901|12846|1666089|3685|40128|12846|1666089|3685|40128|1|26|14|3|10939|194|1119|42|44.75|119.93|73.15|1964.76|3072.30|1879.50|5037.06|0.00|0.00|150.78|3072.30|3072.30|3223.08|3223.08|1192.80| +2450836|50387|2450850|12846|1666089|3685|40128|12846|1666089|3685|40128|1|71|13|3|5722|212|1119|95|95.17|98.97|83.13|1504.80|7897.35|9041.15|9402.15|391.70|3000.99|4700.60|4896.36|5288.06|9596.96|9988.66|-4144.79| +2450836|69575|2450841|91581|1783562|343|3971|4889|181533|4714|25942|4|73|20|4|12607|277|1120|52|55.12|125.12|62.56|3253.12|3253.12|2866.24|6506.24|0.00|0.00|975.52|3253.12|3253.12|4228.64|4228.64|386.88| +2450836|69575|2450903|91581|1783562|343|3971|4889|181533|4714|25942|4|48|14|2|10642|76|1120|83|46.86|52.01|22.36|2460.95|1855.88|3889.38|4316.83|74.23|0.00|1985.36|1855.88|1930.11|3841.24|3915.47|-2033.50| +2450836|69575|2450865|91581|1783562|343|3971|4889|181533|4714|25942|4|12|13|4|4576|55|1120|20|41.41|111.80|110.68|22.40|2213.60|828.20|2236.00|88.54|0.00|134.00|2213.60|2302.14|2347.60|2436.14|1385.40| +2450836|69575|2450856|91581|1783562|343|3971|4889|181533|4714|25942|4|103|14|5|4849|207|1120|42|56.30|108.09|75.66|1362.06|3177.72|2364.60|4539.78|84.84|349.54|2224.32|2828.18|2913.02|5052.50|5137.34|463.58| +2450836|69575|2450879|91581|1783562|343|3971|4889|181533|4714|25942|4|78|7|4|12140|290|1120|60|2.23|3.83|1.80|121.80|108.00|133.80|229.80|0.00|0.00|108.00|108.00|108.00|216.00|216.00|-25.80| +2450836|69575|2450864|91581|1783562|343|3971|4889|181533|4714|25942|4|97|12|1|9475|59|1120|61|7.11|9.81|1.66|497.15|101.26|433.71|598.41|6.07|0.00|137.25|101.26|107.33|238.51|244.58|-332.45| +2450836|69575|2450874|91581|1783562|343|3971|4889|181533|4714|25942|4|94|5|2|15553|278|1120|41|22.92|35.06|1.40|1380.06|57.40|939.72|1437.46|1.14|0.00|86.10|57.40|58.54|143.50|144.64|-882.32| +2450836|69575|2450857|91581|1783562|343|3971|4889|181533|4714|25942|4|19|17|1|6944|28|1120|2|18.73|50.75|40.09|21.32|80.18|37.46|101.50|6.41|0.00|47.70|80.18|86.59|127.88|134.29|42.72| +2450836|69575|2450876|91581|1783562|343|3971|4889|181533|4714|25942|4|5|7|5|9817|91|1120|59|17.02|35.91|22.62|784.11|1334.58|1004.18|2118.69|80.07|0.00|550.47|1334.58|1414.65|1885.05|1965.12|330.40| +2450836|69575|2450918|91581|1783562|343|3971|4889|181533|4714|25942|4|10|7|4|6172|16|1120|83|9.53|27.16|5.70|1781.18|473.10|790.99|2254.28|42.57|0.00|224.93|473.10|515.67|698.03|740.60|-317.89| +2450836|37140|2450856|68211|582492|4773|44654|68211|582492|4773|44654|2|10|13|4|7849|266|1121|82|22.16|25.70|7.96|1454.68|652.72|1817.12|2107.40|13.05|0.00|484.62|652.72|665.77|1137.34|1150.39|-1164.40| +2450836|37140|2450919|68211|582492|4773|44654|68211|582492|4773|44654|2|71|3|4|392|171|1121|77|38.71|106.45|65.99|3115.42|5081.23|2980.67|8196.65|203.24|0.00|3769.92|5081.23|5284.47|8851.15|9054.39|2100.56| +2450836|37140|2450897|68211|582492|4773|44654|68211|582492|4773|44654|2|5|11|4|14965|48|1121|33|80.72|226.01|194.36|1044.45|6413.88|2663.76|7458.33|0.00|0.00|2535.72|6413.88|6413.88|8949.60|8949.60|3750.12| +2450836|37140|2450924|68211|582492|4773|44654|68211|582492|4773|44654|2|60|14|5|4273|49|1121|80|25.93|40.19|12.05|2251.20|964.00|2074.40|3215.20|24.67|347.04|1543.20|616.96|641.63|2160.16|2184.83|-1457.44| +2450836|32458|2450878|17975|1430496|2664|44785|17975|1430496|2664|44785|2|75|2|3|3895|222|1122|57|78.59|79.37|23.81|3166.92|1357.17|4479.63|4524.09|54.28|0.00|1130.88|1357.17|1411.45|2488.05|2542.33|-3122.46| +2450836|32458|2450863|17975|1430496|2664|44785|17975|1430496|2664|44785|2|83|13|5|15580|42|1122|49|98.96|171.20|63.34|5285.14|3103.66|4849.04|8388.80|93.10|0.00|1174.04|3103.66|3196.76|4277.70|4370.80|-1745.38| +2450836|32458|2450862|17975|1430496|2664|44785|17975|1430496|2664|44785|2|92|16|2|1982|197|1122|32|32.69|61.13|56.85|136.96|1819.20|1046.08|1956.16|18.19|0.00|508.48|1819.20|1837.39|2327.68|2345.87|773.12| +2450836|32458|2450916|17975|1430496|2664|44785|17975|1430496|2664|44785|2|6|6|1|3841|264|1122|46|20.49|32.98|15.17|819.26|697.82|942.54|1517.08|6.97|0.00|0.00|697.82|704.79|697.82|704.79|-244.72| +2450836|32458|2450921|17975|1430496|2664|44785|17975|1430496|2664|44785|2|107|2|4|4810|95|1122|75|90.16|148.76|44.62|7810.50|3346.50|6762.00|11157.00|133.86|0.00|4239.00|3346.50|3480.36|7585.50|7719.36|-3415.50| +2450836|32458|2450915|17975|1430496|2664|44785|17975|1430496|2664|44785|2|22|9|4|7|276|1122|69|31.47|35.24|4.22|2140.38|291.18|2171.43|2431.56|3.26|250.41|753.48|40.77|44.03|794.25|797.51|-2130.66| +2450836|32458|2450893|17975|1430496|2664|44785|17975|1430496|2664|44785|2|34|7|2|2833|269|1122|83|3.41|6.95|5.21|144.42|432.43|283.03|576.85|34.59|0.00|235.72|432.43|467.02|668.15|702.74|149.40| +2450836|32458|2450877|17975|1430496|2664|44785|17975|1430496|2664|44785|2|3|9|2|4744|22|1122|22|32.41|47.64|40.49|157.30|890.78|713.02|1048.08|0.00|0.00|471.46|890.78|890.78|1362.24|1362.24|177.76| +2450836|32458|2450867|17975|1430496|2664|44785|17975|1430496|2664|44785|2|42|19|3|7927|83|1122|35|97.09|117.47|66.95|1768.20|2343.25|3398.15|4111.45|19.91|1944.89|246.40|398.36|418.27|644.76|664.67|-2999.79| +2450836|32458|2450862|17975|1430496|2664|44785|17975|1430496|2664|44785|2|25|9|3|13285|137|1122|55|91.20|262.65|5.25|14157.00|288.75|5016.00|14445.75|8.66|0.00|6067.05|288.75|297.41|6355.80|6364.46|-4727.25| +2450836|32458|2450906|17975|1430496|2664|44785|17975|1430496|2664|44785|2|73|8|4|14276|23|1122|26|9.59|12.37|4.32|209.30|112.32|249.34|321.62|4.49|0.00|96.46|112.32|116.81|208.78|213.27|-137.02| +2450836|32458|2450853|17975|1430496|2664|44785|17975|1430496|2664|44785|2|40|11|4|8695|181|1122|4|68.96|77.92|10.90|268.08|43.60|275.84|311.68|3.05|0.00|81.00|43.60|46.65|124.60|127.65|-232.24| +2450836|32458|2450893|17975|1430496|2664|44785|17975|1430496|2664|44785|2|63|5|1|10639|119|1122|18|61.44|164.65|85.61|1422.72|1540.98|1105.92|2963.70|15.40|0.00|681.48|1540.98|1556.38|2222.46|2237.86|435.06| +2450836|77730|2450886|45617|81271|1725|42132|45617|81271|1725|42132|2|90|6|1|6808|204|1123|15|70.74|200.90|76.34|1868.40|1145.10|1061.10|3013.50|91.60|0.00|451.95|1145.10|1236.70|1597.05|1688.65|84.00| +2450836|77730|2450857|45617|81271|1725|42132|45617|81271|1725|42132|2|91|6|1|12760|249|1123|90|2.93|3.07|1.93|102.60|173.70|263.70|276.30|12.15|0.00|96.30|173.70|185.85|270.00|282.15|-90.00| +2450836|77730|2450873|45617|81271|1725|42132|45617|81271|1725|42132|2|76|2|5|10520|208|1123|47|58.86|165.98|29.87|6397.17|1403.89|2766.42|7801.06|98.27|0.00|2496.17|1403.89|1502.16|3900.06|3998.33|-1362.53| +2450836|77730|2450856|45617|81271|1725|42132|45617|81271|1725|42132|2|39|3|5|7394|177|1123|67|30.09|45.13|4.51|2721.54|302.17|2016.03|3023.71|2.75|262.88|272.02|39.29|42.04|311.31|314.06|-1976.74| +2450837|41913|2450889|66169|1339116|5310|39104|66169|1339116|5310|39104|4|63|2|4|3326|215|1124|98|96.48|103.23|73.29|2934.12|7182.42|9455.04|10116.54|646.41|0.00|910.42|7182.42|7828.83|8092.84|8739.25|-2272.62| +2450837|41913|2450921|66169|1339116|5310|39104|66169|1339116|5310|39104|4|23|17|3|16333|240|1124|21|66.96|100.44|63.27|780.57|1328.67|1406.16|2109.24|93.00|0.00|716.94|1328.67|1421.67|2045.61|2138.61|-77.49| +2450837|41913|2450918|66169|1339116|5310|39104|66169|1339116|5310|39104|4|4|11|2|4730|206|1124|90|71.80|211.09|40.10|15389.10|3609.00|6462.00|18998.10|220.87|1154.88|5698.80|2454.12|2674.99|8152.92|8373.79|-4007.88| +2450837|41913|2450842|66169|1339116|5310|39104|66169|1339116|5310|39104|4|108|19|5|12973|300|1124|23|74.93|158.10|64.82|2145.44|1490.86|1723.39|3636.30|11.33|1207.59|545.33|283.27|294.60|828.60|839.93|-1440.12| +2450837|41913|2450878|66169|1339116|5310|39104|66169|1339116|5310|39104|4|56|20|5|3580|185|1124|10|50.59|92.57|72.20|203.70|722.00|505.90|925.70|7.22|0.00|148.10|722.00|729.22|870.10|877.32|216.10| +2450837|41913|2450925|66169|1339116|5310|39104|66169|1339116|5310|39104|4|46|13|1|7336|77|1124|51|39.56|51.82|18.13|1718.19|924.63|2017.56|2642.82|3.69|739.70|264.18|184.93|188.62|449.11|452.80|-1832.63| +2450837|41913|2450858|66169|1339116|5310|39104|66169|1339116|5310|39104|4|36|1|2|17959|11|1124|12|34.04|46.29|41.66|55.56|499.92|408.48|555.48|9.24|314.94|33.24|184.98|194.22|218.22|227.46|-223.50| +2450837|41913|2450868|66169|1339116|5310|39104|66169|1339116|5310|39104|4|80|2|2|4453|70|1124|76|85.20|143.13|105.91|2828.72|8049.16|6475.20|10877.88|0.00|0.00|5003.08|8049.16|8049.16|13052.24|13052.24|1573.96| +2450837|41913|2450921|66169|1339116|5310|39104|66169|1339116|5310|39104|4|99|4|5|13744|227|1124|45|41.30|114.40|46.90|3037.50|2110.50|1858.50|5148.00|168.84|0.00|977.85|2110.50|2279.34|3088.35|3257.19|252.00| +2450837|43388|2450850|71325|970401|677|14153|71325|970401|677|14153|1|70|20|3|12446|184|1125|54|48.45|120.64|1.20|6449.76|64.80|2616.30|6514.56|0.38|51.84|1693.44|12.96|13.34|1706.40|1706.78|-2603.34| +2450837|43388|2450901|71325|970401|677|14153|71325|970401|677|14153|1|68|20|4|8749|42|1125|69|19.84|47.02|31.97|1038.45|2205.93|1368.96|3244.38|110.29|0.00|583.74|2205.93|2316.22|2789.67|2899.96|836.97| +2450837|43388|2450885|71325|970401|677|14153|71325|970401|677|14153|1|13|6|3|11440|190|1125|45|25.59|63.97|30.06|1525.95|1352.70|1151.55|2878.65|54.10|0.00|1035.90|1352.70|1406.80|2388.60|2442.70|201.15| +2450837|43388|2450921|71325|970401|677|14153|71325|970401|677|14153|1|95|12|4|199|152|1125|80|20.85|38.15|8.39|2380.80|671.20|1668.00|3052.00|13.89|208.07|701.60|463.13|477.02|1164.73|1178.62|-1204.87| +2450837|43388|2450897|71325|970401|677|14153|71325|970401|677|14153|1|42|20|1|2498|175|1125|56|23.81|35.71|12.14|1319.92|679.84|1333.36|1999.76|47.58|0.00|299.60|679.84|727.42|979.44|1027.02|-653.52| +2450837|43388|2450882|71325|970401|677|14153|71325|970401|677|14153|1|72|16|3|4207|224|1125|79|79.26|171.20|63.34|8520.94|5003.86|6261.54|13524.80|150.11|0.00|1487.57|5003.86|5153.97|6491.43|6641.54|-1257.68| +2450837|43388|2450865|71325|970401|677|14153|71325|970401|677|14153|1|38|1|4|3289|194|1125|51|7.02|20.35|19.53|41.82|996.03|358.02|1037.85|49.80|0.00|41.31|996.03|1045.83|1037.34|1087.14|638.01| +2450837|43388|2450888|71325|970401|677|14153|71325|970401|677|14153|1|1|5|4|11048|245|1125|42|72.51|82.66|52.90|1249.92|2221.80|3045.42|3471.72|44.43|0.00|1701.00|2221.80|2266.23|3922.80|3967.23|-823.62| +2450837|43388|2450840|71325|970401|677|14153|71325|970401|677|14153|1|44|18|5|4942|241|1125|58|43.94|74.25|49.74|1421.58|2884.92|2548.52|4306.50|173.09|0.00|731.96|2884.92|3058.01|3616.88|3789.97|336.40| +2450837|43388|2450884|71325|970401|677|14153|71325|970401|677|14153|1|92|4|2|4952|120|1125|12|38.13|56.05|16.81|470.88|201.72|457.56|672.60|0.00|0.00|26.88|201.72|201.72|228.60|228.60|-255.84| +2450837|43388|2450901|71325|970401|677|14153|71325|970401|677|14153|1|15|13|1|1411|109|1125|2|72.91|217.27|104.28|225.98|208.56|145.82|434.54|6.25|0.00|26.06|208.56|214.81|234.62|240.87|62.74| +2450837|43388|2450843|71325|970401|677|14153|71325|970401|677|14153|1|67|7|5|17773|214|1125|6|28.29|30.83|15.41|92.52|92.46|169.74|184.98|7.39|0.00|49.92|92.46|99.85|142.38|149.77|-77.28| +2450837|12996|2450868|89884|288680|1876|19472|89884|288680|1876|19472|4|55|16|3|12847|100|1126|62|31.40|43.33|12.56|1907.74|778.72|1946.80|2686.46|5.45|700.84|536.92|77.88|83.33|614.80|620.25|-1868.92| +2450837|12996|2450853|89884|288680|1876|19472|89884|288680|1876|19472|4|104|20|5|6439|237|1126|11|11.27|14.31|0.42|152.79|4.62|123.97|157.41|0.18|0.09|73.92|4.53|4.71|78.45|78.63|-119.44| +2450837|12996|2450892|89884|288680|1876|19472|89884|288680|1876|19472|4|69|3|5|15562|98|1126|8|5.84|7.41|5.11|18.40|40.88|46.72|59.28|0.40|0.00|7.68|40.88|41.28|48.56|48.96|-5.84| +2450837|12996|2450871|89884|288680|1876|19472|89884|288680|1876|19472|4|16|13|5|13300|194|1126|82|34.66|70.35|7.03|5192.24|576.46|2842.12|5768.70|15.68|380.46|865.10|196.00|211.68|1061.10|1076.78|-2646.12| +2450837|12996|2450903|89884|288680|1876|19472|89884|288680|1876|19472|4|107|11|2|5023|127|1126|8|29.75|58.90|41.81|136.72|334.48|238.00|471.20|30.10|0.00|141.36|334.48|364.58|475.84|505.94|96.48| +2450837|12996|2450878|89884|288680|1876|19472|89884|288680|1876|19472|4|36|7|3|6991|140|1126|34|84.36|128.22|112.83|523.26|3836.22|2868.24|4359.48|115.08|0.00|1264.12|3836.22|3951.30|5100.34|5215.42|967.98| +2450837|12996|2450888|89884|288680|1876|19472|89884|288680|1876|19472|4|82|9|3|16405|268|1126|14|98.91|245.29|213.40|446.46|2987.60|1384.74|3434.06|89.62|0.00|1339.24|2987.60|3077.22|4326.84|4416.46|1602.86| +2450837|46569|2450857|97677|760773|1469|11365|97677|760773|1469|11365|4|94|12|4|10351|287|1127|33|82.53|103.98|57.18|1544.40|1886.94|2723.49|3431.34|37.73|0.00|548.79|1886.94|1924.67|2435.73|2473.46|-836.55| +2450837|46569|2450907|97677|760773|1469|11365|97677|760773|1469|11365|4|30|1|3|11533|14|1127|19|64.44|103.74|57.05|887.11|1083.95|1224.36|1971.06|75.87|0.00|98.42|1083.95|1159.82|1182.37|1258.24|-140.41| +2450837|46569|2450885|97677|760773|1469|11365|97677|760773|1469|11365|4|43|18|1|14089|42|1127|45|14.07|23.77|6.65|770.40|299.25|633.15|1069.65|14.96|0.00|352.80|299.25|314.21|652.05|667.01|-333.90| +2450837|46569|2450917|97677|760773|1469|11365|97677|760773|1469|11365|4|64|13|2|15871|137|1127|46|46.82|121.26|121.26|0.00|5577.96|2153.72|5577.96|50.20|3904.57|1729.14|1673.39|1723.59|3402.53|3452.73|-480.33| +2450837|46569|2450850|97677|760773|1469|11365|97677|760773|1469|11365|4|104|11|5|391|240|1127|73|75.10|201.26|66.41|9844.05|4847.93|5482.30|14691.98|193.91|0.00|5729.77|4847.93|5041.84|10577.70|10771.61|-634.37| +2450837|39185|2450901|42103|886754|6605|30503|42103|886754|6605|30503|1|36|13|3|16358|11|1128|89|9.49|9.96|8.96|89.00|797.44|844.61|886.44|47.84|0.00|425.42|797.44|845.28|1222.86|1270.70|-47.17| +2450837|39185|2450902|42103|886754|6605|30503|42103|886754|6605|30503|1|82|5|5|3824|115|1128|10|4.41|8.42|1.09|73.30|10.90|44.10|84.20|0.65|0.00|31.10|10.90|11.55|42.00|42.65|-33.20| +2450837|39185|2450898|42103|886754|6605|30503|42103|886754|6605|30503|1|86|6|2|6488|284|1128|58|34.99|76.97|41.56|2053.78|2410.48|2029.42|4464.26|96.41|0.00|1205.24|2410.48|2506.89|3615.72|3712.13|381.06| +2450837|39185|2450876|42103|886754|6605|30503|42103|886754|6605|30503|1|96|6|5|2608|244|1128|100|41.04|113.27|107.60|567.00|10760.00|4104.00|11327.00|258.24|2152.00|1245.00|8608.00|8866.24|9853.00|10111.24|4504.00| +2450837|39185|2450884|42103|886754|6605|30503|42103|886754|6605|30503|1|46|15|1|187|135|1128|16|92.52|175.78|0.00|2812.48|0.00|1480.32|2812.48|0.00|0.00|1349.92|0.00|0.00|1349.92|1349.92|-1480.32| +2450837|69873|2450880|31522|1790434|6499|39011|31522|1790434|6499|39011|2|71|12|3|9907|184|1129|37|69.01|204.95|120.92|3109.11|4474.04|2553.37|7583.15|79.63|492.14|1971.36|3981.90|4061.53|5953.26|6032.89|1428.53| +2450837|69873|2450923|31522|1790434|6499|39011|31522|1790434|6499|39011|2|87|5|1|10138|58|1129|99|17.00|21.93|0.65|2106.72|64.35|1683.00|2171.07|2.57|0.00|1085.04|64.35|66.92|1149.39|1151.96|-1618.65| +2450837|69873|2450873|31522|1790434|6499|39011|31522|1790434|6499|39011|2|29|13|4|9151|8|1129|62|96.08|101.84|92.67|568.54|5745.54|5956.96|6314.08|12.06|5573.17|2462.02|172.37|184.43|2634.39|2646.45|-5784.59| +2450837|69873|2450911|31522|1790434|6499|39011|31522|1790434|6499|39011|2|51|7|4|5968|160|1129|28|80.77|104.19|97.93|175.28|2742.04|2261.56|2917.32|0.00|0.00|262.36|2742.04|2742.04|3004.40|3004.40|480.48| +2450837|69873|2450910|31522|1790434|6499|39011|31522|1790434|6499|39011|2|103|12|4|15124|276|1129|43|48.12|117.41|73.96|1868.35|3180.28|2069.16|5048.63|190.81|0.00|1009.64|3180.28|3371.09|4189.92|4380.73|1111.12| +2450837|26234|2450844|85598|271578|2005|17232|85598|271578|2005|17232|4|51|9|1|6470|57|1130|98|74.50|84.93|3.39|7990.92|332.22|7301.00|8323.14|6.64|0.00|2995.86|332.22|338.86|3328.08|3334.72|-6968.78| +2450837|26234|2450853|85598|271578|2005|17232|85598|271578|2005|17232|4|61|16|2|5494|189|1130|61|4.97|8.15|5.21|179.34|317.81|303.17|497.15|7.78|162.08|69.54|155.73|163.51|225.27|233.05|-147.44| +2450837|26234|2450884|85598|271578|2005|17232|85598|271578|2005|17232|4|43|8|4|10942|194|1130|16|48.64|107.98|49.67|932.96|794.72|778.24|1727.68|39.73|0.00|431.84|794.72|834.45|1226.56|1266.29|16.48| +2450837|26234|2450907|85598|271578|2005|17232|85598|271578|2005|17232|4|52|19|5|8846|227|1130|33|46.00|63.48|52.05|377.19|1717.65|1518.00|2094.84|68.70|0.00|754.05|1717.65|1786.35|2471.70|2540.40|199.65| +2450837|44390|2450852|57452|1335134|463|18192|57452|1335134|463|18192|2|35|1|1|4970|198|1131|54|4.84|10.84|1.08|527.04|58.32|261.36|585.36|1.16|0.00|169.56|58.32|59.48|227.88|229.04|-203.04| +2450837|44390|2450845|57452|1335134|463|18192|57452|1335134|463|18192|2|81|15|4|9004|126|1131|45|88.06|159.38|89.25|3155.85|4016.25|3962.70|7172.10|321.30|0.00|2653.65|4016.25|4337.55|6669.90|6991.20|53.55| +2450837|44390|2450876|57452|1335134|463|18192|57452|1335134|463|18192|2|61|13|1|12884|179|1131|13|47.15|112.68|92.39|263.77|1201.07|612.95|1464.84|72.06|0.00|703.04|1201.07|1273.13|1904.11|1976.17|588.12| +2450837|44390|2450842|57452|1335134|463|18192|57452|1335134|463|18192|2|85|12|4|14035|51|1131|37|79.83|110.16|53.97|2079.03|1996.89|2953.71|4075.92|119.81|0.00|1752.32|1996.89|2116.70|3749.21|3869.02|-956.82| +2450837|14356|2450875|14650|439584|5318|11452|14650|439584|5318|11452|4|14|2|4|229|208|1132|57|38.03|50.57|42.47|461.70|2420.79|2167.71|2882.49|121.03|0.00|979.83|2420.79|2541.82|3400.62|3521.65|253.08| +2450837|14356|2450885|14650|439584|5318|11452|14650|439584|5318|11452|4|6|15|5|9608|272|1132|69|44.48|62.27|45.45|1160.58|3136.05|3069.12|4296.63|156.80|0.00|1245.45|3136.05|3292.85|4381.50|4538.30|66.93| +2450837|14356|2450910|14650|439584|5318|11452|14650|439584|5318|11452|4|26|19|5|13009|64|1132|40|19.01|36.49|33.93|102.40|1357.20|760.40|1459.60|81.43|0.00|58.00|1357.20|1438.63|1415.20|1496.63|596.80| +2450837|14356|2450907|14650|439584|5318|11452|14650|439584|5318|11452|4|83|11|5|12967|173|1132|70|16.23|23.53|16.23|511.00|1136.10|1136.10|1647.10|102.24|0.00|378.70|1136.10|1238.34|1514.80|1617.04|0.00| +2450837|14356|2450852|14650|439584|5318|11452|14650|439584|5318|11452|4|90|1|3|13826|144|1132|95|65.76|161.11|141.77|1837.30|13468.15|6247.20|15305.45|673.40|0.00|4438.40|13468.15|14141.55|17906.55|18579.95|7220.95| +2450837|68854|2450922|45620|1165605|7190|11928|45620|1165605|7190|11928|4|24|6|5|9475|41|1133|8|30.96|77.40|50.31|216.72|402.48|247.68|619.20|8.04|0.00|241.44|402.48|410.52|643.92|651.96|154.80| +2450837|68854|2450883|45620|1165605|7190|11928|45620|1165605|7190|11928|4|66|5|2|15553|111|1133|71|2.06|3.15|1.44|121.41|102.24|146.26|223.65|0.73|89.97|57.51|12.27|13.00|69.78|70.51|-133.99| +2450837|68854|2450863|45620|1165605|7190|11928|45620|1165605|7190|11928|4|59|18|4|6944|1|1133|75|45.85|130.67|53.57|5782.50|4017.75|3438.75|9800.25|361.59|0.00|2547.75|4017.75|4379.34|6565.50|6927.09|579.00| +2450837|68854|2450853|45620|1165605|7190|11928|45620|1165605|7190|11928|4|82|5|5|9817|63|1133|16|53.19|113.82|40.97|1165.60|655.52|851.04|1821.12|0.91|642.40|473.44|13.12|14.03|486.56|487.47|-837.92| +2450837|68854|2450861|45620|1165605|7190|11928|45620|1165605|7190|11928|4|17|13|2|6172|119|1133|18|89.71|145.33|13.07|2380.68|235.26|1614.78|2615.94|9.41|0.00|156.78|235.26|244.67|392.04|401.45|-1379.52| +2450837|68854|2450861|45620|1165605|7190|11928|45620|1165605|7190|11928|4|100|12|3|10430|248|1133|50|88.61|175.44|124.56|2544.00|6228.00|4430.50|8772.00|311.40|0.00|3421.00|6228.00|6539.40|9649.00|9960.40|1797.50| +2450837|48415|2450926|87294|641840|1186|42132|87294|641840|1186|42132|1|32|15|3|13286|154|1134|21|76.04|174.13|80.09|1974.84|1681.89|1596.84|3656.73|134.55|0.00|329.07|1681.89|1816.44|2010.96|2145.51|85.05| +2450837|48415|2450878|87294|641840|1186|42132|87294|641840|1186|42132|1|80|13|3|12886|26|1134|13|57.89|149.35|132.92|213.59|1727.96|752.57|1941.55|11.75|1434.20|795.99|293.76|305.51|1089.75|1101.50|-458.81| +2450837|48415|2450876|87294|641840|1186|42132|87294|641840|1186|42132|1|43|9|4|6013|286|1134|45|95.76|204.92|178.28|1198.80|8022.60|4309.20|9221.40|89.85|6738.98|922.05|1283.62|1373.47|2205.67|2295.52|-3025.58| +2450837|48415|2450862|87294|641840|1186|42132|87294|641840|1186|42132|1|51|17|4|14659|66|1134|51|52.74|85.43|13.66|3660.27|696.66|2689.74|4356.93|41.79|0.00|1786.02|696.66|738.45|2482.68|2524.47|-1993.08| +2450837|40753|2450923|80933|1701305|2030|18402|80933|1701305|2030|18402|2|106|7|5|11984|21|1135|75|71.60|105.96|41.32|4848.00|3099.00|5370.00|7947.00|169.20|681.78|1668.75|2417.22|2586.42|4085.97|4255.17|-2952.78| +2450837|40753|2450923|80933|1701305|2030|18402|80933|1701305|2030|18402|2|104|19|1|16754|45|1135|77|30.21|45.01|41.40|277.97|3187.80|2326.17|3465.77|255.02|0.00|1420.65|3187.80|3442.82|4608.45|4863.47|861.63| +2450837|40753|2450921|80933|1701305|2030|18402|80933|1701305|2030|18402|2|59|15|4|13456|266|1135|25|40.24|85.30|13.64|1791.50|341.00|1006.00|2132.50|13.64|0.00|469.00|341.00|354.64|810.00|823.64|-665.00| +2450837|40753|2450890|80933|1701305|2030|18402|80933|1701305|2030|18402|2|40|3|2|3080|33|1135|83|80.43|199.46|69.81|10760.95|5794.23|6675.69|16555.18|315.20|1854.15|0.00|3940.08|4255.28|3940.08|4255.28|-2735.61| +2450837|40753|2450859|80933|1701305|2030|18402|80933|1701305|2030|18402|2|37|4|1|4244|82|1135|40|89.24|257.01|239.01|720.00|9560.40|3569.60|10280.40|286.81|0.00|5140.00|9560.40|9847.21|14700.40|14987.21|5990.80| +2450837|40753|2450848|80933|1701305|2030|18402|80933|1701305|2030|18402|2|44|13|2|3116|104|1135|17|23.62|31.65|19.30|209.95|328.10|401.54|538.05|22.96|0.00|145.18|328.10|351.06|473.28|496.24|-73.44| +2450837|40753|2450850|80933|1701305|2030|18402|80933|1701305|2030|18402|2|76|9|3|9223|9|1135|87|33.05|35.03|16.81|1585.14|1462.47|2875.35|3047.61|116.99|0.00|426.30|1462.47|1579.46|1888.77|2005.76|-1412.88| +2450837|40753|2450850|80933|1701305|2030|18402|80933|1701305|2030|18402|2|48|7|3|3406|123|1135|93|71.97|150.41|100.77|4616.52|9371.61|6693.21|13988.13|843.44|0.00|5874.81|9371.61|10215.05|15246.42|16089.86|2678.40| +2450837|40753|2450925|80933|1701305|2030|18402|80933|1701305|2030|18402|2|90|1|4|11155|78|1135|35|80.46|214.02|107.01|3745.35|3745.35|2816.10|7490.70|108.98|112.36|374.50|3632.99|3741.97|4007.49|4116.47|816.89| +2450837|40753|2450882|80933|1701305|2030|18402|80933|1701305|2030|18402|2|79|8|3|8030|296|1135|5|92.74|163.22|83.24|399.90|416.20|463.70|816.10|29.13|0.00|48.95|416.20|445.33|465.15|494.28|-47.50| +|40753||||2030|||1701305||18402|||18||15964||1135||||59.54|||2556.35|4882.28||||||4345.18|4564.88|-115.21| +2450837|40753|2450926|80933|1701305|2030|18402|80933|1701305|2030|18402|2|38|20|3|6314|125|1135|4|95.11|186.41|167.76|74.60|671.04|380.44|745.64|33.55|0.00|22.36|671.04|704.59|693.40|726.95|290.60| +2450837|40753|2450839|80933|1701305|2030|18402|80933|1701305|2030|18402|2|100|18|4|7294|49|1135|66|65.65|116.85|12.85|6864.00|848.10|4332.90|7712.10|25.44|0.00|3393.06|848.10|873.54|4241.16|4266.60|-3484.80| +2450837|40753|2450863|80933|1701305|2030|18402|80933|1701305|2030|18402|2|19|6|4|17600|204|1135|66|46.31|74.55|30.56|2903.34|2016.96|3056.46|4920.30|181.52|0.00|1968.12|2016.96|2198.48|3985.08|4166.60|-1039.50| +2450837|18341|2450864|3003|1128295|5234|18173|92272|1214004|6143|32635|4|91|3|4|15325|191|1136|42|44.68|104.10|34.35|2929.50|1442.70|1876.56|4372.20|17.31|577.08|262.08|865.62|882.93|1127.70|1145.01|-1010.94| +2450837|18341|2450860|3003|1128295|5234|18173|92272|1214004|6143|32635|4|43|5|4|7150|275|1136|57|88.39|108.71|3.26|6010.65|185.82|5038.23|6196.47|6.46|78.04|866.97|107.78|114.24|974.75|981.21|-4930.45| +2450837|18341|2450903|3003|1128295|5234|18173|92272|1214004|6143|32635|4|87|2|3|916|100|1136|58|17.97|48.33|11.11|2158.76|644.38|1042.26|2803.14|12.88|0.00|196.04|644.38|657.26|840.42|853.30|-397.88| +2450837|18341|2450921|3003|1128295|5234|18173|92272|1214004|6143|32635|4|60|10|1|17704|155|1136|18|45.67|94.53|70.89|425.52|1276.02|822.06|1701.54|51.04|0.00|50.94|1276.02|1327.06|1326.96|1378.00|453.96| +2450837|18341|2450891|3003|1128295|5234|18173|92272|1214004|6143|32635|4|69|1|3|12490|119|1136|56|24.16|69.33|65.17|232.96|3649.52|1352.96|3882.48|36.49|0.00|931.28|3649.52|3686.01|4580.80|4617.29|2296.56| +2450837|18341|2450918|3003|1128295|5234|18173|92272|1214004|6143|32635|4|39|5|2|8911|193|1136|94|20.48|33.38|26.70|627.92|2509.80|1925.12|3137.72|19.57|1857.25|1505.88|652.55|672.12|2158.43|2178.00|-1272.57| +2450837|18341|2450864|3003|1128295|5234|18173|92272|1214004|6143|32635|4|44|3|1|1504|245|1136|96|91.88|134.14|2.68|12620.16|257.28|8820.48|12877.44|12.86|0.00|6309.12|257.28|270.14|6566.40|6579.26|-8563.20| +2450837|18341|2450909|3003|1128295|5234|18173|92272|1214004|6143|32635|4|26|10|3|2911|78|1136|88|27.38|62.70|48.90|1214.40|4303.20|2409.44|5517.60|122.64|215.16|1048.08|4088.04|4210.68|5136.12|5258.76|1678.60| +2450837|18341|2450860|3003|1128295|5234|18173|92272|1214004|6143|32635|4|91|4|5|7004|122|1136|68|49.65|65.04|11.70|3627.12|795.60|3376.20|4422.72|15.91|0.00|1945.48|795.60|811.51|2741.08|2756.99|-2580.60| +2450837|18341|2450898|3003|1128295|5234|18173|92272|1214004|6143|32635|4|9|10|1|16966|170|1136|83|28.74|43.11|21.55|1789.48|1788.65|2385.42|3578.13|125.20|0.00|643.25|1788.65|1913.85|2431.90|2557.10|-596.77| +2450837|18341|2450864|3003|1128295|5234|18173|92272|1214004|6143|32635|4|89|9|5|17215|34|1136|40|45.26|75.58|11.33|2570.00|453.20|1810.40|3023.20|4.53|0.00|453.20|453.20|457.73|906.40|910.93|-1357.20| +2450837|18341|2450888|3003|1128295|5234|18173|92272|1214004|6143|32635|4|70|20|3|15698|128|1136|83|26.25|64.31|32.79|2616.16|2721.57|2178.75|5337.73|217.72|0.00|2401.19|2721.57|2939.29|5122.76|5340.48|542.82| +2450837|18341|2450904|3003|1128295|5234|18173|92272|1214004|6143|32635|4|106|2|3|7228|110|1136|68|47.13|57.96|11.59|3153.16|788.12|3204.84|3941.28|5.67|504.39|945.88|283.73|289.40|1229.61|1235.28|-2921.11| +2450837|18341|2450899|3003|1128295|5234|18173|92272|1214004|6143|32635|4|21|7|3|10765|23|1136|95|8.46|9.39|3.94|517.75|374.30|803.70|892.05|22.45|0.00|427.50|374.30|396.75|801.80|824.25|-429.40| +2450837|84031|2450912|83037|1120886|4688|34137|83037|1120886|4688|34137|1|87|12|5|17794|216|1137|43|31.31|91.11|30.97|2586.02|1331.71|1346.33|3917.73|13.31|0.00|939.98|1331.71|1345.02|2271.69|2285.00|-14.62| +2450837|84031|2450884|83037|1120886|4688|34137|83037|1120886|4688|34137|1|59|3|5|2128|256|1137|73|62.65|76.43|62.67|1004.48|4574.91|4573.45|5579.39|182.99|0.00|1115.44|4574.91|4757.90|5690.35|5873.34|1.46| +2450837|84031|2450861|83037|1120886|4688|34137|83037|1120886|4688|34137|1|46|2|4|6103|115|1137|62|57.73|113.72|113.72|0.00|7050.64|3579.26|7050.64|493.54|0.00|1550.62|7050.64|7544.18|8601.26|9094.80|3471.38| +2450837|84031|2450866|83037|1120886|4688|34137|83037|1120886|4688|34137|1|2|4|3|12268|138|1137|58|30.23|64.38|45.06|1120.56|2613.48|1753.34|3734.04|156.80|0.00|37.12|2613.48|2770.28|2650.60|2807.40|860.14| +2450837|69779|2450856|12443|1226745|5047|3349|12443|1226745|5047|3349|1|48|6|2|13984|46|1138|100|46.82|126.88|59.63|6725.00|5963.00|4682.00|12688.00|536.67|0.00|2537.00|5963.00|6499.67|8500.00|9036.67|1281.00| +2450837|69779|2450866|12443|1226745|5047|3349|12443|1226745|5047|3349|1|10|8|1|6130|142|1138|70|10.51|13.87|0.83|912.80|58.10|735.70|970.90|4.06|0.00|475.30|58.10|62.16|533.40|537.46|-677.60| +2450837|69779|2450902|12443|1226745|5047|3349|12443|1226745|5047|3349|1|107|3|5|10813|190|1138|15|18.98|38.71|32.90|87.15|493.50|284.70|580.65|19.74|0.00|0.00|493.50|513.24|493.50|513.24|208.80| +2450837|69779|2450879|12443|1226745|5047|3349|12443|1226745|5047|3349|1|9|8|2|15472|105|1138|43|88.60|228.58|128.00|4324.94|5504.00|3809.80|9828.94|158.51|2862.08|196.51|2641.92|2800.43|2838.43|2996.94|-1167.88| +2450837|69779|2450861|12443|1226745|5047|3349|12443|1226745|5047|3349|1|80|3|1|11290|297|1138|66|22.24|65.16|49.52|1032.24|3268.32|1467.84|4300.56|39.22|2287.82|0.00|980.50|1019.72|980.50|1019.72|-487.34| +2450837|69779|2450900|12443|1226745|5047|3349|12443|1226745|5047|3349|1|103|14|5|12724|300|1138|8|76.29|153.34|144.13|73.68|1153.04|610.32|1226.72|2.07|1129.97|392.48|23.07|25.14|415.55|417.62|-587.25| +2450837|69779|2450926|12443|1226745|5047|3349|12443|1226745|5047|3349|1|16|16|5|1004|93|1138|2|25.42|36.60|9.15|54.90|18.30|50.84|73.20|0.00|0.00|32.94|18.30|18.30|51.24|51.24|-32.54| +2450837|69779|2450905|12443|1226745|5047|3349|12443|1226745|5047|3349|1|2|14|1|2404|180|1138|34|96.92|188.02|26.32|5497.80|894.88|3295.28|6392.68|28.63|178.97|319.60|715.91|744.54|1035.51|1064.14|-2579.37| +2450837|74983|2450907|79238|1601|5211|774|79238|1601|5211|774|2|24|10|4|2162|154|1139|87|95.98|239.95|151.16|7724.73|13150.92|8350.26|20875.65|1052.07|0.00|207.93|13150.92|14202.99|13358.85|14410.92|4800.66| +2450837|74983|2450869|79238|1601|5211|774|79238|1601|5211|774|2|55|1|3|6296|250|1139|97|95.45|185.17|162.94|2156.31|15805.18|9258.65|17961.49|94.83|6322.07|4310.68|9483.11|9577.94|13793.79|13888.62|224.46| +2450837|74983|2450878|79238|1601|5211|774|79238|1601|5211|774|2|59|13|2|7651|298|1139|28|63.89|68.36|17.77|1416.52|497.56|1788.92|1914.08|4.97|0.00|133.84|497.56|502.53|631.40|636.37|-1291.36| +2450837|74983|2450881|79238|1601|5211|774|79238|1601|5211|774|2|56|10|3|2575|224|1139|22|72.28|93.96|93.96|0.00|2067.12|1590.16|2067.12|103.35|0.00|289.30|2067.12|2170.47|2356.42|2459.77|476.96| +2450837|74983|2450864|79238|1601|5211|774|79238|1601|5211|774|2|92|2|4|13063|238|1139|82|94.71|185.63|122.51|5175.84|10045.82|7766.22|15221.66|803.66|0.00|455.92|10045.82|10849.48|10501.74|11305.40|2279.60| +2450837|74983|2450893|79238|1601|5211|774|79238|1601|5211|774|2|53|5|4|3373|167|1139|47|36.32|85.35|7.68|3650.49|360.96|1707.04|4011.45|32.48|0.00|1163.25|360.96|393.44|1524.21|1556.69|-1346.08| +2450837|74983|2450850|79238|1601|5211|774|79238|1601|5211|774|2|24|8|2|17762|293|1139|90|43.77|124.30|41.01|7496.10|3690.90|3939.30|11187.00|332.18|0.00|3243.60|3690.90|4023.08|6934.50|7266.68|-248.40| +2450837|74983|2450859|79238|1601|5211|774|79238|1601|5211|774|2|4|10|1|632|137|1139|11|79.31|123.72|81.65|462.77|898.15|872.41|1360.92|62.87|0.00|258.50|898.15|961.02|1156.65|1219.52|25.74| +2450837|74983|2450846|79238|1601|5211|774|79238|1601|5211|774|2|26|8|4|1831|86|1139|36|24.63|59.11|44.92|510.84|1617.12|886.68|2127.96|80.85|0.00|425.52|1617.12|1697.97|2042.64|2123.49|730.44| +2450837|74983|2450885|79238|1601|5211|774|79238|1601|5211|774|2|42|12|4|12344|19|1139|12|39.75|70.35|40.09|363.12|481.08|477.00|844.20|24.05|0.00|92.76|481.08|505.13|573.84|597.89|4.08| +2450837|74983|2450927|79238|1601|5211|774|79238|1601|5211|774|2|1|15|1|4549|300|1139|82|63.75|169.57|154.30|1252.14|12652.60|5227.50|13904.74|506.10|0.00|5005.28|12652.60|13158.70|17657.88|18163.98|7425.10| +2450837|74983|2450846|79238|1601|5211|774|79238|1601|5211|774|2|45|12|3|10363|97|1139|59|82.93|245.47|130.09|6807.42|7675.31|4892.87|14482.73|307.01|0.00|1882.69|7675.31|7982.32|9558.00|9865.01|2782.44| +2450837|66732|2450903|19672|753144|6616|26088|19672|753144|6616|26088|4|38|8|5|1504|19|1140|84|93.16|245.01|7.35|19963.44|617.40|7825.44|20580.84|6.17|0.00|10084.20|617.40|623.57|10701.60|10707.77|-7208.04| +2450837|66732|2450894|19672|753144|6616|26088|19672|753144|6616|26088|4|82|8|3|2911|176|1140|28|14.50|25.52|24.75|21.56|693.00|406.00|714.56|27.72|0.00|7.00|693.00|720.72|700.00|727.72|287.00| +2450837|66732|2450891|19672|753144|6616|26088|19672|753144|6616|26088|4|46|14|3|7004|206|1140|41|89.31|187.55|180.04|307.91|7381.64|3661.71|7689.55|0.00|0.00|999.58|7381.64|7381.64|8381.22|8381.22|3719.93| +2450837|66732|2450899|19672|753144|6616|26088|19672|753144|6616|26088|4|105|7|3|16966|104|1140|39|78.79|145.76|110.77|1364.61|4320.03|3072.81|5684.64|302.40|0.00|2330.64|4320.03|4622.43|6650.67|6953.07|1247.22| +2450837|66732|2450843|19672|753144|6616|26088|19672|753144|6616|26088|4|18|13|2|17215|25|1140|9|80.00|236.00|198.24|339.84|1784.16|720.00|2124.00|17.84|0.00|722.16|1784.16|1802.00|2506.32|2524.16|1064.16| +2450837|81386|2450895|58841|388062|4930|1751|58841|388062|4930|1751|1|95|9|5|2527|47|1141|81|94.81|116.61|45.47|5762.34|3683.07|7679.61|9445.41|257.81|0.00|2927.34|3683.07|3940.88|6610.41|6868.22|-3996.54| +2450837|81386|||388062|4930||58841||4930|1751|||15||6346||1141|85|||54.84||||||||4661.40||6992.10|7365.01|1521.50| +2450837|81386|2450853|58841|388062|4930|1751|58841|388062|4930|1751|1|11|18|5|3092|112|1141|95|20.03|35.65|16.04|1862.95|1523.80|1902.85|3386.75|15.23|0.00|1455.40|1523.80|1539.03|2979.20|2994.43|-379.05| +2450837|81386|2450896|58841|388062|4930|1751|58841|388062|4930|1751|1|39|1|2|6268|56|1141|32|65.24|69.15|69.15|0.00|2212.80|2087.68|2212.80|66.38|0.00|685.76|2212.80|2279.18|2898.56|2964.94|125.12| +2450837|81386|2450927|58841|388062|4930|1751|58841|388062|4930|1751|1|83|1|5|3382|235|1141|16|74.22|182.58|113.19|1110.24|1811.04|1187.52|2921.28|18.11|0.00|29.12|1811.04|1829.15|1840.16|1858.27|623.52| +2450837|81386|2450921|58841|388062|4930|1751|58841|388062|4930|1751|1|14|5|4|15931|173|1141|55|23.80|60.92|19.49|2278.65|1071.95|1309.00|3350.60|0.00|0.00|1172.60|1071.95|1071.95|2244.55|2244.55|-237.05| +2450837|81386|2450923|58841|388062|4930|1751|58841|388062|4930|1751|1|17|11|1|1066|54|1141|100|76.00|130.72|0.00|13072.00|0.00|7600.00|13072.00|0.00|0.00|3137.00|0.00|0.00|3137.00|3137.00|-7600.00| +2450837|81386|2450866|58841|388062|4930|1751|58841|388062|4930|1751|1|103|5|3|15088|134|1141|56|42.39|43.66|7.42|2029.44|415.52|2373.84|2444.96|0.00|186.98|953.12|228.54|228.54|1181.66|1181.66|-2145.30| +2450837|81386|2450857|58841|388062|4930|1751|58841|388062|4930|1751|1|102|3|4|11864|228|1141|40|48.86|88.43|55.71|1308.80|2228.40|1954.40|3537.20|0.00|1582.16|0.00|646.24|646.24|646.24|646.24|-1308.16| +2450837|81386|2450923|58841|388062|4930|1751|58841|388062|4930|1751|1|14|13|5|13639|76|1141|38|9.00|12.96|9.97|113.62|378.86|342.00|492.48|0.11|367.49|24.32|11.37|11.48|35.69|35.80|-330.63| +2450837|66305|2450898|62344|1682131|3518|42105|62344|1682131|3518|42105|2|29|16|2|9001|291|1142|60|63.64|122.82|18.42|6264.00|1105.20|3818.40|7369.20|0.00|751.53|1547.40|353.67|353.67|1901.07|1901.07|-3464.73| +2450837|66305|2450856|62344|1682131|3518|42105|62344|1682131|3518|42105|2|103|13|1|139|255|1142|29|61.79|111.83|12.30|2886.37|356.70|1791.91|3243.07|0.00|0.00|421.37|356.70|356.70|778.07|778.07|-1435.21| +2450837|66305|2450922|62344|1682131|3518|42105|62344|1682131|3518|42105|2|48|15|4|13393|10|1142|41|30.41|59.29|10.67|1993.42|437.47|1246.81|2430.89|13.12|0.00|899.13|437.47|450.59|1336.60|1349.72|-809.34| +2450837|66305|2450923|62344|1682131|3518|42105|62344|1682131|3518|42105|2|62|9|2|5578|132|1142|56|32.95|56.01|21.28|1944.88|1191.68|1845.20|3136.56|35.75|0.00|1411.20|1191.68|1227.43|2602.88|2638.63|-653.52| +2450837|66305|2450909|62344|1682131|3518|42105|62344|1682131|3518|42105|2|42|19|2|11062|114|1142|80|7.22|12.92|10.98|155.20|878.40|577.60|1033.60|26.35|0.00|144.00|878.40|904.75|1022.40|1048.75|300.80| +2450837|66305|2450856|62344|1682131|3518|42105|62344|1682131|3518|42105|2|104|4|3|15194|48|1142|38|41.54|110.91|82.07|1095.92|3118.66|1578.52|4214.58|280.67|0.00|1432.60|3118.66|3399.33|4551.26|4831.93|1540.14| +2450837|66305|2450844|62344|1682131|3518|42105|62344|1682131|3518|42105|2|75|14|4|16759|251|1142|68|30.75|66.11|3.30|4271.08|224.40|2091.00|4495.48|11.22|0.00|1483.08|224.40|235.62|1707.48|1718.70|-1866.60| +2450837|66305|2450845|62344|1682131|3518|42105|62344|1682131|3518|42105|2|83|14|5|15244|199|1142|15|94.51|129.47|24.59|1573.20|368.85|1417.65|1942.05|0.00|0.00|97.05|368.85|368.85|465.90|465.90|-1048.80| +2450837|68153|2450918|57821|884924|2874|21577|57821|884924|2874|21577|4|29|8|3|8938|273|1143|71|82.93|226.39|124.51|7233.48|8840.21|5888.03|16073.69|353.60|0.00|6429.05|8840.21|9193.81|15269.26|15622.86|2952.18| +2450837|68153|2450897|57821|884924|2874|21577|57821|884924|2874|21577|4|20|3|1|9602|208|1143|85|90.48|224.39|71.80|12970.15|6103.00|7690.80|19073.15|366.18|0.00|6103.00|6103.00|6469.18|12206.00|12572.18|-1587.80| +2450837|68153|2450918|57821|884924|2874|21577|57821|884924|2874|21577|4|83|4|3|5338|161|1143|84|95.96|114.19|15.98|8249.64|1342.32|8060.64|9591.96|120.80|0.00|1246.56|1342.32|1463.12|2588.88|2709.68|-6718.32| +2450837|68153|2450873|57821|884924|2874|21577|57821|884924|2874|21577|4|86|17|5|5870|96|1143|22|26.57|28.16|14.36|303.60|315.92|584.54|619.52|12.63|0.00|117.70|315.92|328.55|433.62|446.25|-268.62| +2450837|68153|2450898|57821|884924|2874|21577|57821|884924|2874|21577|4|60|6|3|9514|284|1143|50|67.94|105.98|20.13|4292.50|1006.50|3397.00|5299.00|20.13|0.00|1059.50|1006.50|1026.63|2066.00|2086.13|-2390.50| +2450837|68153|2450878|57821|884924|2874|21577|57821|884924|2874|21577|4|37|7|2|4963|81|1143|42|6.73|16.69|11.01|238.56|462.42|282.66|700.98|32.36|0.00|97.86|462.42|494.78|560.28|592.64|179.76| +2450837|68153|2450905|57821|884924|2874|21577|57821|884924|2874|21577|4|43|9|1|15805|146|1143|41|4.67|8.96|3.13|239.03|128.33|191.47|367.36|0.00|0.00|36.49|128.33|128.33|164.82|164.82|-63.14| +2450837|68153|2450916|57821|884924|2874|21577|57821|884924|2874|21577|4|102|19|5|14128|281|1143|27|43.01|51.18|43.50|207.36|1174.50|1161.27|1381.86|0.00|0.00|331.56|1174.50|1174.50|1506.06|1506.06|13.23| +2450837|68153|2450909|57821|884924|2874|21577|57821|884924|2874|21577|4|46|4|2|9925|118|1143|70|62.42|93.63|28.08|4588.50|1965.60|4369.40|6554.10|19.65|0.00|2097.20|1965.60|1985.25|4062.80|4082.45|-2403.80| +2450837|68153|2450904|57821|884924|2874|21577|57821|884924|2874|21577|4|108|18|3|691|133|1143|61|19.75|39.69|9.12|1864.77|556.32|1204.75|2421.09|27.81|0.00|847.29|556.32|584.13|1403.61|1431.42|-648.43| +2450837|68153|2450916|57821|884924|2874|21577|57821|884924|2874|21577|4|12|3|5|17786|294|1143|5|46.39|131.74|28.98|513.80|144.90|231.95|658.70|4.34|0.00|270.05|144.90|149.24|414.95|419.29|-87.05| +2450837|68153|2450841|57821|884924|2874|21577|57821|884924|2874|21577|4|62|20|2|16094|188|1143|6|88.33|184.60|14.76|1019.04|88.56|529.98|1107.60|0.88|0.00|199.32|88.56|89.44|287.88|288.76|-441.42| +2450837|68153|2450923|57821|884924|2874|21577|57821|884924|2874|21577|4|22|9|5|3706|137|1143|23|7.25|15.29|10.85|102.12|249.55|166.75|351.67|9.98|0.00|14.03|249.55|259.53|263.58|273.56|82.80| +2450837|68153|2450904|57821|884924|2874|21577|57821|884924|2874|21577|4|84|17|2|17149|87|1143|95|51.55|71.65|22.92|4629.35|2177.40|4897.25|6806.75|66.41|849.18|135.85|1328.22|1394.63|1464.07|1530.48|-3569.03| +2450837|39363|2450905|48997|1187760|4923|25733|48997|1187760|4923|25733|2|44|3|5|16804|52|1144|99|12.93|23.14|21.05|206.91|2083.95|1280.07|2290.86|104.19|0.00|778.14|2083.95|2188.14|2862.09|2966.28|803.88| +2450837|39363|2450904|48997|1187760|4923|25733|48997|1187760|4923|25733|2|85|10|2|14738|129|1144|88|65.85|190.96|114.57|6722.32|10082.16|5794.80|16804.48|705.75|0.00|7057.60|10082.16|10787.91|17139.76|17845.51|4287.36| +2450837|39363|2450842|48997|1187760|4923|25733|48997|1187760|4923|25733|2|29|17|2|5509|289|1144|31|46.15|68.76|30.25|1193.81|937.75|1430.65|2131.56|13.50|768.95|1001.61|168.80|182.30|1170.41|1183.91|-1261.85| +2450837|39363|2450927|48997|1187760|4923|25733|48997|1187760|4923|25733|2|45|15|2|5620|6|1144|13|29.50|49.26|23.64|333.06|307.32|383.50|640.38|15.36|0.00|313.69|307.32|322.68|621.01|636.37|-76.18| +2450837|39363|2450916|48997|1187760|4923|25733|48997|1187760|4923|25733|2|65|15|5|1105|109|1144|10|74.75|101.66|74.21|274.50|742.10|747.50|1016.60|59.36|0.00|111.80|742.10|801.46|853.90|913.26|-5.40| +2450837|39363|2450846|48997|1187760|4923|25733|48997|1187760|4923|25733|2|103|3|1|5900|122|1144|4|89.66|229.52|2.29|908.92|9.16|358.64|918.08|0.73|0.00|45.88|9.16|9.89|55.04|55.77|-349.48| +2450837|39363|2450875|48997|1187760|4923|25733|48997|1187760|4923|25733|2|55|16|3|9892|282|1144|79|58.37|130.74|112.43|1446.49|8881.97|4611.23|10328.46|266.45|0.00|102.70|8881.97|9148.42|8984.67|9251.12|4270.74| +2450837|39363|2450846|48997|1187760|4923|25733|48997|1187760|4923|25733|2|62|9|5|13825|45|1144|34|95.53|202.52|52.65|5095.58|1790.10|3248.02|6885.68|35.80|0.00|2685.32|1790.10|1825.90|4475.42|4511.22|-1457.92| +2450837|39363|2450889|48997|1187760|4923|25733|48997|1187760|4923|25733|2|7|11|2|7022|87|1144|26|32.00|92.80|85.37|193.18|2219.62|832.00|2412.80|88.78|0.00|892.58|2219.62|2308.40|3112.20|3200.98|1387.62| +2450837|78611|2450914|87638|812929|4267|5558|51244|364587|278|1569|4|5|1|1|11312|56|1145|62|39.68|101.58|26.41|4660.54|1637.42|2460.16|6297.96|16.37|0.00|1825.90|1637.42|1653.79|3463.32|3479.69|-822.74| +2450837|78611|2450927|87638|812929|4267|5558|51244|364587|278|1569|4|37|15|1|13766|211|1145|24|81.29|223.54|15.64|4989.60|375.36|1950.96|5364.96|30.02|0.00|2521.44|375.36|405.38|2896.80|2926.82|-1575.60| +2450837|78611|2450913|87638|812929|4267|5558|51244|364587|278|1569|4|99|6|1|2858|174|1145|1|91.15|250.66|105.27|145.39|105.27|91.15|250.66|2.55|20.00|27.57|85.27|87.82|112.84|115.39|-5.88| +2450837|78611|2450916|87638|812929|4267|5558|51244|364587|278|1569|4|32|19|1|9790|195|1145|88|75.71|180.94|36.18|12738.88|3183.84|6662.48|15922.72|254.70|0.00|1910.48|3183.84|3438.54|5094.32|5349.02|-3478.64| +2450837|78611|2450919|87638|812929|4267|5558|51244|364587|278|1569|4|76|9|4|5990|194|1145|88|32.85|78.51|51.03|2418.24|4490.64|2890.80|6908.88|179.62|0.00|1312.08|4490.64|4670.26|5802.72|5982.34|1599.84| +2450837|78611|2450907|87638|812929|4267|5558|51244|364587|278|1569|4|30|8|5|14047|162|1145|97|75.99|204.41|102.20|9914.37|9913.40|7371.03|19827.77|892.20|0.00|2577.29|9913.40|10805.60|12490.69|13382.89|2542.37| +2450837|78611|2450845|87638|812929|4267|5558|51244|364587|278|1569|4|59|16|5|5230|61|1145|37|94.14|250.41|27.54|8246.19|1018.98|3483.18|9265.17|30.56|0.00|1667.59|1018.98|1049.54|2686.57|2717.13|-2464.20| +2450837|78611|2450925|87638|812929|4267|5558|51244|364587|278|1569|4|53|2|4|15988|108|1145|22|73.26|78.38|26.64|1138.28|586.08|1611.72|1724.36|41.02|0.00|137.94|586.08|627.10|724.02|765.04|-1025.64| +2450837|78611|2450872|87638|812929|4267|5558|51244|364587|278|1569|4|62|11|3|5629|289|1145|94|57.10|165.01|69.30|8996.74|6514.20|5367.40|15510.94|65.14|0.00|3257.10|6514.20|6579.34|9771.30|9836.44|1146.80| +2450837|78611|2450866|87638|812929|4267|5558|51244|364587|278|1569|4|9|20|4|14840|172|1145|4|72.90|102.06|62.25|159.24|249.00|291.60|408.24|4.98|0.00|183.68|249.00|253.98|432.68|437.66|-42.60| +2450837|78611|2450842|87638|812929|4267|5558|51244|364587|278|1569|4|53|19|3|8563|214|1145|76|81.03|153.95|89.29|4914.16|6786.04|6158.28|11700.20|475.02|0.00|1286.68|6786.04|7261.06|8072.72|8547.74|627.76| +2450837|78611|2450846|87638|812929|4267|5558|51244|364587|278|1569|4|20|17|3|6680|101|1145|86|46.61|113.72|103.48|880.64|8899.28|4008.46|9779.92|0.00|0.00|2542.16|8899.28|8899.28|11441.44|11441.44|4890.82| +2450837|78611|2450848|87638|812929|4267|5558|51244|364587|278|1569|4|99|9|4|10306|199|1145|3|74.74|134.53|118.38|48.45|355.14|224.22|403.59|3.55|0.00|68.61|355.14|358.69|423.75|427.30|130.92| +2450837|78611|2450867|87638|812929|4267|5558|51244|364587|278|1569|4|5|12|2|17650|165|1145|36|16.07|45.96|19.30|959.76|694.80|578.52|1654.56|48.63|0.00|744.48|694.80|743.43|1439.28|1487.91|116.28| +2450837|58116|2450885|89085|74150|4815|39880|89085|74150|4815|39880|2|97|14|4|7558|99|1146|45|31.28|84.45|30.40|2432.25|1368.00|1407.60|3800.25|41.04|0.00|151.65|1368.00|1409.04|1519.65|1560.69|-39.60| +2450837|58116|2450921|89085|74150|4815|39880|89085|74150|4815|39880|2|47|3|2|1342|167|1146|50|40.45|117.70|65.91|2589.50|3295.50|2022.50|5885.00|32.95|0.00|647.00|3295.50|3328.45|3942.50|3975.45|1273.00| +2450837|58116|2450850|89085|74150|4815|39880|89085|74150|4815|39880|2|63|9|2|12952|155|1146|100|41.67|73.75|37.61|3614.00|3761.00|4167.00|7375.00|84.24|1654.84|811.00|2106.16|2190.40|2917.16|3001.40|-2060.84| +2450837|58116|2450908|89085|74150|4815|39880|89085|74150|4815|39880|2|69|7|4|4418|280|1146|59|90.01|234.02|2.34|13669.12|138.06|5310.59|13807.18|8.28|0.00|3313.44|138.06|146.34|3451.50|3459.78|-5172.53| +2450837|58116|2450873|89085|74150|4815|39880|89085|74150|4815|39880|2|38|6|1|15070|9|1146|22|77.25|116.64|53.65|1385.78|1180.30|1699.50|2566.08|62.32|141.63|102.52|1038.67|1100.99|1141.19|1203.51|-660.83| +2450837|58116|2450861|89085|74150|4815|39880|89085|74150|4815|39880|2|13|5|4|7690|237|1146|99|55.98|115.87|33.60|8144.73|3326.40|5542.02|11471.13|232.84|0.00|4472.82|3326.40|3559.24|7799.22|8032.06|-2215.62| +2450837|27360|2450872|13082|864539|6749|6206|13082|864539|6749|6206|1|49|4|5|4783|171|1147|100|30.04|62.48|6.24|5624.00|624.00|3004.00|6248.00|0.00|0.00|499.00|624.00|624.00|1123.00|1123.00|-2380.00| +2450837|27360|2450901|13082|864539|6749|6206|13082|864539|6749|6206|1|98|11|1|3502|53|1147|31|97.28|239.30|179.47|1854.73|5563.57|3015.68|7418.30|389.44|0.00|2299.58|5563.57|5953.01|7863.15|8252.59|2547.89| +2450837|27360|2450886|13082|864539|6749|6206|13082|864539|6749|6206|1|16|12|5|3836|36|1147|60|97.29|263.65|63.27|12022.80|3796.20|5837.40|15819.00|189.81|0.00|316.20|3796.20|3986.01|4112.40|4302.21|-2041.20| +2450837|27360|2450873|13082|864539|6749|6206|13082|864539|6749|6206|1|78|4|4|4334|67|1147|11|45.03|81.50|66.01|170.39|726.11|495.33|896.50|14.52|0.00|268.95|726.11|740.63|995.06|1009.58|230.78| +2450837|27360|2450893|13082|864539|6749|6206|13082|864539|6749|6206|1|47|7|2|15904|34|1147|8|61.51|62.12|42.24|159.04|337.92|492.08|496.96|4.49|273.71|154.00|64.21|68.70|218.21|222.70|-427.87| +2450837|27360|2450869|13082|864539|6749|6206|13082|864539|6749|6206|1|29|12|4|1514|148|1147|24|9.05|14.84|2.96|285.12|71.04|217.20|356.16|4.97|0.00|0.00|71.04|76.01|71.04|76.01|-146.16| +2450837|27360|2450890|13082|864539|6749|6206|13082|864539|6749|6206|1|56|17|5|6472|235|1147|96|56.00|58.80|53.50|508.80|5136.00|5376.00|5644.80|462.24|0.00|1636.80|5136.00|5598.24|6772.80|7235.04|-240.00| +2450837|27360|2450892|13082|864539|6749|6206|13082|864539|6749|6206|1|59|13|4|10034|71|1147|54|61.95|90.44|37.98|2832.84|2050.92|3345.30|4883.76|49.22|1066.47|1318.14|984.45|1033.67|2302.59|2351.81|-2360.85| +2450837|27360|2450897|13082|864539|6749|6206|13082|864539|6749|6206|1|21|17|2|15229|49|1147|12|61.19|148.69|11.89|1641.60|142.68|734.28|1784.28|8.56|0.00|892.08|142.68|151.24|1034.76|1043.32|-591.60| +2450837|27360|2450860|13082|864539|6749|6206|13082|864539|6749|6206|1|54|12|4|2182|52|1147|3|62.05|81.90|40.13|125.31|120.39|186.15|245.70|6.01|0.00|12.27|120.39|126.40|132.66|138.67|-65.76| +2450837|27360|2450900|13082|864539|6749|6206|13082|864539|6749|6206|1|12|18|2|16450|240|1147|80|82.57|120.55|49.42|5690.40|3953.60|6605.60|9644.00|197.68|0.00|1542.40|3953.60|4151.28|5496.00|5693.68|-2652.00| +2450837|27360|2450884|13082|864539|6749|6206|13082|864539|6749|6206|1|64|1|2|9808|212|1147|68|54.56|104.75|3.14|6909.48|213.52|3710.08|7123.00|12.81|0.00|284.92|213.52|226.33|498.44|511.25|-3496.56| +2450837|27360|2450862|13082|864539|6749|6206|13082|864539|6749|6206|1|62|18|2|226|30|1147|25|55.39|116.31|26.75|2239.00|668.75|1384.75|2907.75|14.44|307.62|494.25|361.13|375.57|855.38|869.82|-1023.62| +2450837|16705|2450883|13346|717670|394|19118|13346|717670|394|19118|1|28|17|4|10760|39|1148|41|92.19|215.72|138.06|3184.06|5660.46|3779.79|8844.52|56.60|0.00|2034.01|5660.46|5717.06|7694.47|7751.07|1880.67| +2450837|16705|2450867|13346|717670|394|19118|13346|717670|394|19118|1|102|3|1|8341|133|1148|96|66.71|170.77|59.76|10656.96|5736.96|6404.16|16393.92|458.95|0.00|3934.08|5736.96|6195.91|9671.04|10129.99|-667.20| +2450837|16705|2450848|13346|717670|394|19118|13346|717670|394|19118|1|67|10|5|3302|154|1148|51|60.07|106.92|60.94|2344.98|3107.94|3063.57|5452.92|124.31|0.00|545.19|3107.94|3232.25|3653.13|3777.44|44.37| +2450837|16705|2450851|13346|717670|394|19118|13346|717670|394|19118|1|15|4|5|12229|141|1148|85|11.44|20.47|2.66|1513.85|226.10|972.40|1739.95|20.34|0.00|852.55|226.10|246.44|1078.65|1098.99|-746.30| +2450837|16705|2450916|13346|717670|394|19118|13346|717670|394|19118|1|39|17|5|9202|81|1148|23|33.84|43.99|6.59|860.20|151.57|778.32|1011.77|1.57|112.16|323.61|39.41|40.98|363.02|364.59|-738.91| +2450837|16705|2450857|13346|717670|394|19118|13346|717670|394|19118|1|36|5|4|8810|235|1148|45|64.25|113.72|25.01|3991.95|1125.45|2891.25|5117.40|33.76|0.00|1176.75|1125.45|1159.21|2302.20|2335.96|-1765.80| +2450837|16705|2450897|13346|717670|394|19118|13346|717670|394|19118|1|21|15|3|7501|269|1148|55|85.48|123.09|1.23|6702.30|67.65|4701.40|6769.95|2.02|0.00|1624.70|67.65|69.67|1692.35|1694.37|-4633.75| +2450837|16705|2450896|13346|717670|394|19118|13346|717670|394|19118|1|17|10|4|2810|50|1148|44|94.82|118.52|90.07|1251.80|3963.08|4172.08|5214.88|237.78|0.00|364.76|3963.08|4200.86|4327.84|4565.62|-209.00| +2450837|35960|2450922|34680|395209|2585|14314|34680|395209|2585|14314|1|23|18|4|12115|186|1149|57|13.61|22.86|1.37|1224.93|78.09|775.77|1303.02|2.95|35.92|299.25|42.17|45.12|341.42|344.37|-733.60| +2450837|35960|2450882|34680|395209|2585|14314|34680|395209|2585|14314|1|77|4|3|14440|88|1149|58|13.82|34.13|0.00|1979.54|0.00|801.56|1979.54|0.00|0.00|494.74|0.00|0.00|494.74|494.74|-801.56| +2450837|35960|2450843|34680|395209|2585|14314|34680|395209|2585|14314|1|87|1|3|12937|260|1149|10|3.88|6.79|3.12|36.70|31.20|38.80|67.90|1.56|0.00|24.40|31.20|32.76|55.60|57.16|-7.60| +2450837|35960|2450906|34680|395209|2585|14314|34680|395209|2585|14314|1|15|15|1|394|134|1149|55|78.24|200.29|62.08|7601.55|3414.40|4303.20|11015.95|267.34|443.87|1101.10|2970.53|3237.87|4071.63|4338.97|-1332.67| +2450837|35960|2450911|34680|395209|2585|14314|34680|395209|2585|14314|1|36|13|4|2882|167|1149|32|5.58|16.40|5.74|341.12|183.68|178.56|524.80|0.00|42.24|136.32|141.44|141.44|277.76|277.76|-37.12| +2450837|35960|2450860|34680|395209|2585|14314|34680|395209|2585|14314|1|44|17|4|9373|280|1149|53|73.06|199.45|59.83|7399.86|3170.99|3872.18|10570.85|126.83|0.00|5285.16|3170.99|3297.82|8456.15|8582.98|-701.19| +2450837|35960|2450883|34680|395209|2585|14314|34680|395209|2585|14314|1|106|17|3|16111|90|1149|39|12.98|31.28|8.75|878.67|341.25|506.22|1219.92|7.50|191.10|597.48|150.15|157.65|747.63|755.13|-356.07| +2450837|35960|2450842|34680|395209|2585|14314|34680|395209|2585|14314|1|32|14|2|4945|295|1149|38|36.21|63.00|59.85|119.70|2274.30|1375.98|2394.00|90.97|0.00|215.46|2274.30|2365.27|2489.76|2580.73|898.32| +2450837|35960|2450868|34680|395209|2585|14314|34680|395209|2585|14314|1|101|5|5|5983|60|1149|42|38.15|44.25|17.70|1115.10|743.40|1602.30|1858.50|4.46|520.38|836.22|223.02|227.48|1059.24|1063.70|-1379.28| +2450837|35960|2450843|34680|395209|2585|14314|34680|395209|2585|14314|1|58|15|2|15320|8|1149|36|55.66|124.12|43.44|2904.48|1563.84|2003.76|4468.32|31.27|0.00|759.60|1563.84|1595.11|2323.44|2354.71|-439.92| +2450837|35960|2450922|34680|395209|2585|14314|34680|395209|2585|14314|1|66|7|3|17437|40|1149|88|51.32|93.91|54.46|3471.60|4792.48|4516.16|8264.08|143.77|0.00|1404.48|4792.48|4936.25|6196.96|6340.73|276.32| +2450837|35960|2450858|34680|395209|2585|14314|34680|395209|2585|14314|1|23|13|5|16708|179|1149|24|37.16|95.50|22.92|1741.92|550.08|891.84|2292.00|33.00|0.00|320.88|550.08|583.08|870.96|903.96|-341.76| +2450837|35960|2450901|34680|395209|2585|14314|34680|395209|2585|14314|1|92|1|3|3988|244|1149|79|11.69|24.43|7.08|1370.65|559.32|923.51|1929.97|5.59|0.00|405.27|559.32|564.91|964.59|970.18|-364.19| +2450837|44724|2450914|63624|1051619|3167|20715|63624|1051619|3167|20715|1|23|16|1|17101|298|1150|56|43.56|48.35|37.71|595.84|2111.76|2439.36|2707.60|42.23|0.00|785.12|2111.76|2153.99|2896.88|2939.11|-327.60| +2450837|44724|2450871|63624|1051619|3167|20715|63624|1051619|3167|20715|1|86|14|5|2065|87|1150|54|98.07|290.28|95.79|10502.46|5172.66|5295.78|15675.12|362.08|0.00|5015.52|5172.66|5534.74|10188.18|10550.26|-123.12| +2450837|44724|2450905|63624|1051619|3167|20715|63624|1051619|3167|20715|1|28|13|2|6937|11|1150|60|31.93|35.76|27.89|472.20|1673.40|1915.80|2145.60|33.46|0.00|1072.80|1673.40|1706.86|2746.20|2779.66|-242.40| +2450837|44724|2450893|63624|1051619|3167|20715|63624|1051619|3167|20715|1|98|1|5|7406|101|1150|90|94.35|137.75|26.17|10042.20|2355.30|8491.50|12397.50|70.65|0.00|4586.40|2355.30|2425.95|6941.70|7012.35|-6136.20| +2450837|44724|2450855|63624|1051619|3167|20715|63624|1051619|3167|20715|1|65|10|4|15193|40|1150|59|67.79|162.69|19.52|8447.03|1151.68|3999.61|9598.71|11.51|0.00|2591.28|1151.68|1163.19|3742.96|3754.47|-2847.93| +2450837|44724|2450858|63624|1051619|3167|20715|63624|1051619|3167|20715|1|104|7|2|12979|235|1150|45|43.41|117.64|75.28|1906.20|3387.60|1953.45|5293.80|0.00|609.76|1164.60|2777.84|2777.84|3942.44|3942.44|824.39| +2450837|44724|2450874|63624|1051619|3167|20715|63624|1051619|3167|20715|1|58|11|2|15284|256|1150|53|9.29|16.53|7.60|473.29|402.80|492.37|876.09|4.02|0.00|113.42|402.80|406.82|516.22|520.24|-89.57| +2450837|44724|2450920|63624|1051619|3167|20715|63624|1051619|3167|20715|1|47|17|3|4474|145|1150|56|22.51|32.41|16.52|889.84|925.12|1260.56|1814.96|27.75|0.00|235.76|925.12|952.87|1160.88|1188.63|-335.44| +2450837|44724|2450897|63624|1051619|3167|20715|63624|1051619|3167|20715|1|67|19|4|13484|34|1150|71|28.21|38.64|34.00|329.44|2414.00|2002.91|2743.44|193.12|0.00|274.06|2414.00|2607.12|2688.06|2881.18|411.09| +2450837|44724|2450924|63624|1051619|3167|20715|63624|1051619|3167|20715|1|44|9|1|1874|119|1150|52|17.51|24.16|2.17|1143.48|112.84|910.52|1256.32|5.64|0.00|388.96|112.84|118.48|501.80|507.44|-797.68| +2450837|41550|2450917|42074|865108|2095|40682|42074|865108|2095|40682|4|79|19|3|1306|138|1151|78|8.00|14.80|7.10|600.60|553.80|624.00|1154.40|0.00|0.00|80.34|553.80|553.80|634.14|634.14|-70.20| +2450837|41550|2450861|42074|865108|2095|40682|42074|865108|2095|40682|4|40|15|2|1282|56|1151|9|14.14|31.95|27.15|43.20|244.35|127.26|287.55|0.00|2.44|86.22|241.91|241.91|328.13|328.13|114.65| +2450837|41550|2450888|42074|865108|2095|40682|42074|865108|2095|40682|4|6|5|2|10987|279|1151|45|96.22|259.79|168.86|4091.85|7598.70|4329.90|11690.55|227.96|0.00|116.55|7598.70|7826.66|7715.25|7943.21|3268.80| +2450837|41550|2450901|42074|865108|2095|40682|42074|865108|2095|40682|4|57|12|5|11884|274|1151|70|64.14|157.78|41.02|8173.20|2871.40|4489.80|11044.60|143.57|0.00|3091.90|2871.40|3014.97|5963.30|6106.87|-1618.40| +2450837|41550|2450856|42074|865108|2095|40682|42074|865108|2095|40682|4|20|7|2|9664|252|1151|49|76.54|159.20|144.87|702.17|7098.63|3750.46|7800.80|354.93|0.00|1013.81|7098.63|7453.56|8112.44|8467.37|3348.17| +2450837|34554|2450856|26685|1138136|3187|22591|26685|1138136|3187|22591|2|10|7|5|14240|213|1152|58|77.37|93.61|30.89|3637.76|1791.62|4487.46|5429.38|0.53|1773.70|2714.40|17.92|18.45|2732.32|2732.85|-4469.54| +2450837|34554|2450882|26685|1138136|3187|22591|26685|1138136|3187|22591|2|46|5|5|15032|192|1152|12|60.65|141.31|1.41|1678.80|16.92|727.80|1695.72|0.50|0.00|610.44|16.92|17.42|627.36|627.86|-710.88| +2450837|34554|2450888|26685|1138136|3187|22591|26685|1138136|3187|22591|2|59|20|4|3260|230|1152|56|85.68|195.35|171.90|1313.20|9626.40|4798.08|10939.60|0.00|2695.39|1202.88|6931.01|6931.01|8133.89|8133.89|2132.93| +2450837|34554|2450854|26685|1138136|3187|22591|26685|1138136|3187|22591|2|66|19|2|638|200|1152|38|70.76|147.18|75.06|2740.56|2852.28|2688.88|5592.84|199.65|0.00|2292.92|2852.28|3051.93|5145.20|5344.85|163.40| +2450837|34554|2450893|26685|1138136|3187|22591|26685|1138136|3187|22591|2|60|13|3|2920|82|1152|40|15.95|20.89|11.48|376.40|459.20|638.00|835.60|28.65|101.02|342.40|358.18|386.83|700.58|729.23|-279.82| +2450837|34554|2450870|26685|1138136|3187|22591|26685|1138136|3187|22591|2|1|12|1|12938|212|1152|78|72.87|94.73|27.47|5246.28|2142.66|5683.86|7388.94|51.42|1499.86|738.66|642.80|694.22|1381.46|1432.88|-5041.06| +2450837|34554|2450865|26685|1138136|3187|22591|26685|1138136|3187|22591|2|81|8|5|14656|212|1152|58|69.07|155.40|59.05|5588.30|3424.90|4006.06|9013.20|80.14|753.47|3424.90|2671.43|2751.57|6096.33|6176.47|-1334.63| +2450837|34554|2450895|26685|1138136|3187|22591|26685|1138136|3187|22591|2|65|13|1|16978|112|1152|65|52.27|76.31|55.70|1339.65|3620.50|3397.55|4960.15|0.00|0.00|991.90|3620.50|3620.50|4612.40|4612.40|222.95| +2450837|34554|2450864|26685|1138136|3187|22591|26685|1138136|3187|22591|2|71|11|1|13555|166|1152|61|45.66|69.40|44.41|1524.39|2709.01|2785.26|4233.40|27.09|0.00|1438.99|2709.01|2736.10|4148.00|4175.09|-76.25| +2450837|34554|2450925|26685|1138136|3187|22591|26685|1138136|3187|22591|2|18|5|1|14689|251|1152|92|34.93|78.94|32.36|4285.36|2977.12|3213.56|7262.48|7.44|2232.84|2178.56|744.28|751.72|2922.84|2930.28|-2469.28| +2450837|7315|2450854|19720|237784|201|15018|19720|237784|201|15018|4|58|10|1|1549|119|1153|78|34.94|78.96|34.74|3449.16|2709.72|2725.32|6158.88|108.38|0.00|307.32|2709.72|2818.10|3017.04|3125.42|-15.60| +2450837|7315|2450861|19720|237784|201|15018|19720|237784|201|15018|4|76|9|1|4984|290|1153|82|44.98|87.26|79.40|644.52|6510.80|3688.36|7155.32|447.94|911.51|929.88|5599.29|6047.23|6529.17|6977.11|1910.93| +2450837|7315|2450869|19720|237784|201|15018|19720|237784|201|15018|4|7|12|2|9862|2|1153|71|69.83|141.05|139.63|100.82|9913.73|4957.93|10014.55|297.41|0.00|2503.46|9913.73|10211.14|12417.19|12714.60|4955.80| +2450837|7315|2450923|19720|237784|201|15018|19720|237784|201|15018|4|90|14|5|2509|154|1153|45|96.02|179.55|154.41|1131.30|6948.45|4320.90|8079.75|555.87|0.00|969.30|6948.45|7504.32|7917.75|8473.62|2627.55| +2450837|7315|2450904|19720|237784|201|15018|19720|237784|201|15018|4|31|9|2|11491|181|1153|82|22.68|66.45|38.54|2288.62|3160.28|1859.76|5448.90|31.60|0.00|1852.38|3160.28|3191.88|5012.66|5044.26|1300.52| +2450837|7315|2450922|19720|237784|201|15018|19720|237784|201|15018|4|4|12|4|8372|143|1153|42|1.34|3.88|1.08|117.60|45.36|56.28|162.96|1.36|0.00|81.48|45.36|46.72|126.84|128.20|-10.92| +|7315|||237784|||||201|||96|12||6592|78|1153|37||9.28|0.09|||||0.26||||3.59|41.07|41.33|-323.75| +2450837|7315|2450875|19720|237784|201|15018|19720|237784|201|15018|4|65|4|2|8305|135|1153|65|53.96|138.13|85.64|3411.85|5566.60|3507.40|8978.45|111.33|0.00|1526.20|5566.60|5677.93|7092.80|7204.13|2059.20| +2450837|7315|2450906|19720|237784|201|15018|19720|237784|201|15018|4|61|11|1|14863|66|1153|37|36.10|68.95|5.51|2347.28|203.87|1335.70|2551.15|16.30|0.00|535.39|203.87|220.17|739.26|755.56|-1131.83| +2450837|7315|2450876|19720|237784|201|15018|19720|237784|201|15018|4|103|16|2|7516|89|1153|10|73.15|156.54|61.05|954.90|610.50|731.50|1565.40|48.84|0.00|140.80|610.50|659.34|751.30|800.14|-121.00| +2450837|46668|2450871|35488|560485|329|20526|35488|560485|329|20526|4|46|4|1|16384|263|1154|45|40.93|116.65|88.65|1260.00|3989.25|1841.85|5249.25|37.89|3231.29|2309.40|757.96|795.85|3067.36|3105.25|-1083.89| +2450837|46668|2450855|35488|560485|329|20526|35488|560485|329|20526|4|41|2|1|4372|298|1154|15|61.01|115.30|27.67|1314.45|415.05|915.15|1729.50|8.30|0.00|311.25|415.05|423.35|726.30|734.60|-500.10| +2450837|46668|2450894|35488|560485|329|20526|35488|560485|329|20526|4|34|10|2|8632|243|1154|85|12.87|26.38|19.78|561.00|1681.30|1093.95|2242.30|84.06|0.00|650.25|1681.30|1765.36|2331.55|2415.61|587.35| +2450837|46668|2450894|35488|560485|329|20526|35488|560485|329|20526|4|5|3|3|2044|75|1154|29|5.41|14.39|10.50|112.81|304.50|156.89|417.31|18.27|0.00|0.00|304.50|322.77|304.50|322.77|147.61| +2450837|46668|2450895|35488|560485|329|20526|35488|560485|329|20526|4|49|11|4|2398|257|1154|23|90.57|157.59|64.61|2138.54|1486.03|2083.11|3624.57|71.32|297.20|362.25|1188.83|1260.15|1551.08|1622.40|-894.28| +2450837|46668|2450905|35488|560485|329|20526|35488|560485|329|20526|4|57|16|3|12016|77|1154|12|18.00|23.22|20.66|30.72|247.92|216.00|278.64|9.91|0.00|125.28|247.92|257.83|373.20|383.11|31.92| +2450837|46668|2450860|35488|560485|329|20526|35488|560485|329|20526|4|106|3|2|1906|260|1154|78|12.87|24.58|8.84|1227.72|689.52|1003.86|1917.24|55.16|0.00|785.46|689.52|744.68|1474.98|1530.14|-314.34| +2450837|46668|2450896|35488|560485|329|20526|35488|560485|329|20526|4|71|7|2|13951|117|1154|19|41.70|112.17|10.09|1939.52|191.71|792.30|2131.23|5.75|0.00|511.48|191.71|197.46|703.19|708.94|-600.59| +2450837|12981|2450854|81682|1461958|5081|14557|81682|1461958|5081|14557|2|16|8|4|12700|267|1155|9|10.10|22.22|19.55|24.03|175.95|90.90|199.98|5.27|0.00|35.91|175.95|181.22|211.86|217.13|85.05| +2450837|12981|2450895|81682|1461958|5081|14557|81682|1461958|5081|14557|2|60|14|2|5833|200|1155|91|84.66|182.86|111.54|6490.12|10150.14|7704.06|16640.26|487.20|406.00|831.74|9744.14|10231.34|10575.88|11063.08|2040.08| +2450837|12981|2450852|81682|1461958|5081|14557|81682|1461958|5081|14557|2|70|20|3|2875|217|1155|17|52.50|112.35|24.71|1489.88|420.07|892.50|1909.95|8.40|0.00|401.03|420.07|428.47|821.10|829.50|-472.43| +2450837|12981|2450871|81682|1461958|5081|14557|81682|1461958|5081|14557|2|17|19|1|16543|235|1155|39|67.67|75.11|6.75|2666.04|263.25|2639.13|2929.29|19.58|18.42|351.39|244.83|264.41|596.22|615.80|-2394.30| +2450837|12981|2450866|81682|1461958|5081|14557|81682|1461958|5081|14557|2|77|17|4|12739|95|1155|61|81.37|103.33|4.13|6051.20|251.93|4963.57|6303.13|12.59|0.00|377.59|251.93|264.52|629.52|642.11|-4711.64| +2450837|12981|2450852|81682|1461958|5081|14557|81682|1461958|5081|14557|2|74|13|5|1003|184|1155|59|65.84|115.22|102.54|748.12|6049.86|3884.56|6797.98|0.00|3206.42|2446.73|2843.44|2843.44|5290.17|5290.17|-1041.12| +2450837|12981|2450905|81682|1461958|5081|14557|81682|1461958|5081|14557|2|40|16|5|12154|216|1155|70|62.24|83.40|16.68|4670.40|1167.60|4356.80|5838.00|93.40|0.00|933.80|1167.60|1261.00|2101.40|2194.80|-3189.20| +2450837|12981|2450855|81682|1461958|5081|14557|81682|1461958|5081|14557|2|44|18|3|11300|138|1155|56|58.50|132.21|126.92|296.24|7107.52|3276.00|7403.76|355.37|0.00|444.08|7107.52|7462.89|7551.60|7906.97|3831.52| +2450837|12981|2450879|81682|1461958|5081|14557|81682|1461958|5081|14557|2|31|2|3|11834|36|1155|73|73.32|132.70|55.73|5618.81|4068.29|5352.36|9687.10|0.00|0.00|4261.74|4068.29|4068.29|8330.03|8330.03|-1284.07| +2450837|12981|2450845|81682|1461958|5081|14557|81682|1461958|5081|14557|2|52|6|1|14768|142|1155|84|74.37|80.31|76.29|337.68|6408.36|6247.08|6746.04|64.08|0.00|2023.56|6408.36|6472.44|8431.92|8496.00|161.28| +2450837|12981|2450851|81682|1461958|5081|14557|81682|1461958|5081|14557|2|78|10|5|12166|192|1155|82|16.43|19.55|15.44|337.02|1266.08|1347.26|1603.10|88.62|0.00|576.46|1266.08|1354.70|1842.54|1931.16|-81.18| +2450837|12981|2450873|81682|1461958|5081|14557|81682|1461958|5081|14557|2|67|9|2|11056|243|1155|74|5.17|15.09|10.86|313.02|803.64|382.58|1116.66|40.18|0.00|33.30|803.64|843.82|836.94|877.12|421.06| +2450837|12981|2450872|81682|1461958|5081|14557|81682|1461958|5081|14557|2|41|4|5|16591|195|1155|86|85.90|124.55|100.88|2035.62|8675.68|7387.40|10711.30|607.29|0.00|4177.02|8675.68|9282.97|12852.70|13459.99|1288.28| +2450837|12981|2450849|81682|1461958|5081|14557|81682|1461958|5081|14557|2|99|20|4|13538|15|1155|79|86.25|209.58|199.10|827.92|15728.90|6813.75|16556.82|737.68|5190.53|5794.65|10538.37|11276.05|16333.02|17070.70|3724.62| +2450837|60214|2450915|27316|1107831|7099|34414|27316|1107831|7099|34414|2|108|10|3|17485|256|1156|32|45.37|56.25|12.93|1386.24|413.76|1451.84|1800.00|0.00|297.90|665.92|115.86|115.86|781.78|781.78|-1335.98| +2450837|60214|2450848|27316|1107831|7099|34414|27316|1107831|7099|34414|2|60|16|1|163|273|1156|4|68.34|120.96|72.57|193.56|290.28|273.36|483.84|23.22|0.00|159.64|290.28|313.50|449.92|473.14|16.92| +2450837|60214|2450911|27316|1107831|7099|34414|27316|1107831|7099|34414|2|34|9|1|12235|60|1156|15|11.01|15.74|13.22|37.80|198.30|165.15|236.10|7.93|0.00|16.50|198.30|206.23|214.80|222.73|33.15| +2450837|60214|2450874|27316|1107831|7099|34414|27316|1107831|7099|34414|2|15|12|1|706|17|1156|88|24.28|26.22|21.23|439.12|1868.24|2136.64|2307.36|149.45|0.00|506.88|1868.24|2017.69|2375.12|2524.57|-268.40| +2450837|60214|2450875|27316|1107831|7099|34414|27316|1107831|7099|34414|2|18|11|5|6466|8|1156|89|29.03|72.86|5.10|6030.64|453.90|2583.67|6484.54|18.15|0.00|2658.43|453.90|472.05|3112.33|3130.48|-2129.77| +2450837|60214|2450889|27316|1107831|7099|34414|27316|1107831|7099|34414|2|107|6|2|17318|128|1156|97|85.68|182.49|94.89|8497.20|9204.33|8310.96|17701.53|0.00|0.00|2654.89|9204.33|9204.33|11859.22|11859.22|893.37| +2450837|60214|2450842|27316|1107831|7099|34414|27316|1107831|7099|34414|2|59|9|1|14636|71|1156|65|3.56|3.66|3.11|35.75|202.15|231.40|237.90|18.19|0.00|118.95|202.15|220.34|321.10|339.29|-29.25| +2450837|60214|2450920|27316|1107831|7099|34414|27316|1107831|7099|34414|2|73|17|3|7922|107|1156|30|45.13|123.20|9.85|3400.50|295.50|1353.90|3696.00|26.59|0.00|480.30|295.50|322.09|775.80|802.39|-1058.40| +2450837|60214|2450902|27316|1107831|7099|34414|27316|1107831|7099|34414|2|104|1|1|13190|26|1156|91|80.91|139.16|130.81|759.85|11903.71|7362.81|12663.56|114.27|9046.81|2659.02|2856.90|2971.17|5515.92|5630.19|-4505.91| +2450837|60214|2450845|27316|1107831|7099|34414|27316|1107831|7099|34414|2|42|16|3|5684|292|1156|86|1.73|3.52|2.21|112.66|190.06|148.78|302.72|1.90|0.00|96.32|190.06|191.96|286.38|288.28|41.28| +2450837|44561|2450869|97238|527357|3076|7995|97238|527357|3076|7995|4|14|15|3|11893|149|1157|53|8.38|10.72|0.96|517.28|50.88|444.14|568.16|3.05|0.00|221.54|50.88|53.93|272.42|275.47|-393.26| +2450837|44561|2450911|97238|527357|3076|7995|97238|527357|3076|7995|4|39|11|1|15416|74|1157|69|3.47|4.33|1.21|215.28|83.49|239.43|298.77|2.50|0.00|128.34|83.49|85.99|211.83|214.33|-155.94| +2450837|44561|2450844|97238|527357|3076|7995|97238|527357|3076|7995|4|101|4|3|12223|61|1157|74|79.48|147.03|10.29|10118.76|761.46|5881.52|10880.22|60.91|0.00|3263.40|761.46|822.37|4024.86|4085.77|-5120.06| +2450837|44561|2450920|97238|527357|3076|7995|97238|527357|3076|7995|4|18|4|5|9896|282|1157|40|40.32|108.05|94.00|562.00|3760.00|1612.80|4322.00|188.00|0.00|0.00|3760.00|3948.00|3760.00|3948.00|2147.20| +2450837|44561|2450885|97238|527357|3076|7995|97238|527357|3076|7995|4|13|3|2|12820|258|1157|91|16.56|40.57|4.46|3286.01|405.86|1506.96|3691.87|11.20|125.81|1698.06|280.05|291.25|1978.11|1989.31|-1226.91| +2450837|44561|2450919|97238|527357|3076|7995|97238|527357|3076|7995|4|34|17|5|3764|176|1157|12|63.17|74.54|72.30|26.88|867.60|758.04|894.48|78.08|0.00|17.88|867.60|945.68|885.48|963.56|109.56| +2450837|44561|2450840|97238|527357|3076|7995|97238|527357|3076|7995|4|56|2|2|3074|250|1157|50|39.48|60.79|57.14|182.50|2857.00|1974.00|3039.50|57.14|0.00|455.50|2857.00|2914.14|3312.50|3369.64|883.00| +2450837|72128|2450852|90759|200103|3047|34391|90759|200103|3047|34391|4|21|5|1|10682|76|1158|30|56.60|114.33|19.43|2847.00|582.90|1698.00|3429.90|40.80|0.00|1166.10|582.90|623.70|1749.00|1789.80|-1115.10| +2450837|72128|2450852|90759|200103|3047|34391|90759|200103|3047|34391|4|31|6|5|16177|188|1158|98|29.98|39.27|27.09|1193.64|2654.82|2938.04|3848.46|79.64|0.00|1616.02|2654.82|2734.46|4270.84|4350.48|-283.22| +2450837|72128|2450881|90759|200103|3047|34391|90759|200103|3047|34391|4|48|18|4|1322|161|1158|59|11.69|23.26|0.00|1372.34|0.00|689.71|1372.34|0.00|0.00|425.39|0.00|0.00|425.39|425.39|-689.71| +2450837|72128|2450856|90759|200103|3047|34391|90759|200103|3047|34391|4|74|3|3|7438|50|1158|4|46.92|107.44|79.50|111.76|318.00|187.68|429.76|25.44|0.00|201.96|318.00|343.44|519.96|545.40|130.32| +2450837|72128|2450887|90759|200103|3047|34391|90759|200103|3047|34391|4|68|18|4|2948|231|1158|100|16.31|30.49|26.83|366.00|2683.00|1631.00|3049.00|107.32|0.00|1036.00|2683.00|2790.32|3719.00|3826.32|1052.00| +2450837|72128|2450898|90759|200103|3047|34391|90759|200103|3047|34391|4|99|19|4|12004|44|1158|88|94.43|169.97|112.18|5085.52|9871.84|8309.84|14957.36|789.74|0.00|1495.12|9871.84|10661.58|11366.96|12156.70|1562.00| +2450837|10984|2450880|94573|271231|1489|42500|94573|271231|1489|42500|4|95|12|3|3208|273|1159|81|52.12|82.34|44.46|3068.28|3601.26|4221.72|6669.54|113.44|2340.81|2667.33|1260.45|1373.89|3927.78|4041.22|-2961.27| +2450837|10984|2450908|94573|271231|1489|42500|94573|271231|1489|42500|4|81|2|2|13282|110|1159|27|57.49|91.40|84.08|197.64|2270.16|1552.23|2467.80|136.20|0.00|666.09|2270.16|2406.36|2936.25|3072.45|717.93| +2450837|10984|2450910|94573|271231|1489|42500|94573|271231|1489|42500|4|72|20|4|5503|184|1159|55|68.09|68.09|29.27|2135.10|1609.85|3744.95|3744.95|32.19|0.00|973.50|1609.85|1642.04|2583.35|2615.54|-2135.10| +2450837|10984|2450913|94573|271231|1489|42500|94573|271231|1489|42500|4|107|3|3|775|238|1159|79|62.41|141.04|0.00|11142.16|0.00|4930.39|11142.16|0.00|0.00|3676.66|0.00|0.00|3676.66|3676.66|-4930.39| +2450837|10984|2450914|94573|271231|1489|42500|94573|271231|1489|42500|4|29|9|5|17989|267|1159|10|18.05|35.37|2.82|325.50|28.20|180.50|353.70|0.84|0.00|169.70|28.20|29.04|197.90|198.74|-152.30| +2450837|10984|2450864|94573|271231|1489|42500|94573|271231|1489|42500|4|75|17|5|889|65|1159|97|5.03|8.19|4.17|389.94|404.49|487.91|794.43|8.08|0.00|23.28|404.49|412.57|427.77|435.85|-83.42| +2450837|10984|2450919|94573|271231|1489|42500|94573|271231|1489|42500|4|88|14|4|17197|32|1159|28|43.19|122.65|11.03|3125.36|308.84|1209.32|3434.20|6.17|0.00|583.80|308.84|315.01|892.64|898.81|-900.48| +2450837|10984|2450903|94573|271231|1489|42500|94573|271231|1489|42500|4|84|6|3|14164|222|1159|9|26.33|44.49|36.92|68.13|332.28|236.97|400.41|9.96|0.00|63.99|332.28|342.24|396.27|406.23|95.31| +2450837|53398|2450901|32375|4093|4611|43129|32375|4093|4611|43129|2|25|12|4|12296|276|1160|80|69.06|203.72|28.52|14016.00|2281.60|5524.80|16297.60|159.71|0.00|6681.60|2281.60|2441.31|8963.20|9122.91|-3243.20| +2450837|53398|2450865|32375|4093|4611|43129|32375|4093|4611|43129|2|14|8|5|7945|66|1160|58|2.64|4.59|2.61|114.84|151.38|153.12|266.22|9.08|0.00|122.38|151.38|160.46|273.76|282.84|-1.74| +2450837|53398|2450900|32375|4093|4611|43129|32375|4093|4611|43129|2|86|15|3|3616|74|1160|17|72.03|160.62|6.42|2621.40|109.14|1224.51|2730.54|2.18|0.00|1310.53|109.14|111.32|1419.67|1421.85|-1115.37| +2450837|53398|2450880|32375|4093|4611|43129|32375|4093|4611|43129|2|11|19|5|8494|18|1160|60|75.20|221.84|159.72|3727.20|9583.20|4512.00|13310.40|766.65|0.00|5856.00|9583.20|10349.85|15439.20|16205.85|5071.20| +2450837|53398|2450874|32375|4093|4611|43129|32375|4093|4611|43129|2|86|9|3|1318|234|1160|37|8.85|16.54|14.22|85.84|526.14|327.45|611.98|47.35|0.00|232.36|526.14|573.49|758.50|805.85|198.69| +2450837|53398|2450854|32375|4093|4611|43129|32375|4093|4611|43129|2|77|20|1|13112|115|1160|24|1.85|3.20|2.75|10.80|66.00|44.40|76.80|3.96|0.00|9.12|66.00|69.96|75.12|79.08|21.60| +2450837|53398|2450863|32375|4093|4611|43129|32375|4093|4611|43129|2|5|11|3|11984|17|1160|100|33.72|81.93|58.98|2295.00|5898.00|3372.00|8193.00|530.82|0.00|245.00|5898.00|6428.82|6143.00|6673.82|2526.00| +2450837|53398|2450872|32375|4093|4611|43129|32375|4093|4611|43129|2|75|14|4|16754|191|1160|8|28.42|67.63|8.79|470.72|70.32|227.36|541.04|5.62|0.00|183.92|70.32|75.94|254.24|259.86|-157.04| +2450837|53398|2450903|32375|4093|4611|43129|32375|4093|4611|43129|2|90|10|3|13456|119|1160|35|35.06|37.51|12.75|866.60|446.25|1227.10|1312.85|4.46|0.00|432.95|446.25|450.71|879.20|883.66|-780.85| +2450837|53398|2450857|32375|4093|4611|43129|32375|4093|4611|43129|2|7|16|5|3080|81|1160|87|61.82|165.05|4.95|13928.70|430.65|5378.34|14359.35|0.00|116.27|7035.69|314.38|314.38|7350.07|7350.07|-5063.96| +2450837|73649|2450847|3099|572726|1906|37533|3099|572726|1906|37533|1|81|7|2|15688|227|1161|20|72.70|85.05|39.97|901.60|799.40|1454.00|1701.00|23.98|0.00|833.40|799.40|823.38|1632.80|1656.78|-654.60| +2450837|73649|2450922|3099|572726|1906|37533|3099|572726|1906|37533|1|11|20|2|16237|126|1161|72|74.16|215.06|43.01|12387.60|3096.72|5339.52|15484.32|247.73|0.00|774.00|3096.72|3344.45|3870.72|4118.45|-2242.80| +2450837|73649|2450927|3099|572726|1906|37533|3099|572726|1906|37533|1|103|3|1|15889|147|1161|41|28.44|45.50|20.47|1026.23|839.27|1166.04|1865.50|58.74|0.00|111.93|839.27|898.01|951.20|1009.94|-326.77| +2450837|73649|2450895|3099|572726|1906|37533|3099|572726|1906|37533|1|2|8|4|5092|281|1161|50|53.11|78.07|63.23|742.00|3161.50|2655.50|3903.50|31.61|0.00|1717.50|3161.50|3193.11|4879.00|4910.61|506.00| +2450837|73649|2450917|3099|572726|1906|37533|3099|572726|1906|37533|1|66|16|5|16298|174|1161|52|88.44|251.16|183.34|3526.64|9533.68|4598.88|13060.32|95.33|0.00|4179.24|9533.68|9629.01|13712.92|13808.25|4934.80| +2450837|73649|2450872|3099|572726|1906|37533|3099|572726|1906|37533|1|47|6|4|7040|250|1161|69|83.66|204.96|92.23|7778.37|6363.87|5772.54|14142.24|0.00|0.00|5939.52|6363.87|6363.87|12303.39|12303.39|591.33| +2450837|73649|2450878|3099|572726|1906|37533|3099|572726|1906|37533|1|82|16|4|14582|215|1161|56|4.70|8.50|6.63|104.72|371.28|263.20|476.00|1.11|352.71|76.16|18.57|19.68|94.73|95.84|-244.63| +2450837|73649|2450886|3099|572726|1906|37533|3099|572726|1906|37533|1|105|14|2|15892|233|1161|60|12.31|21.41|11.56|591.00|693.60|738.60|1284.60|34.68|0.00|64.20|693.60|728.28|757.80|792.48|-45.00| +2450837|73649|2450874|3099|572726|1906|37533|3099|572726|1906|37533|1|79|12|5|5605|75|1161|79|35.17|69.98|18.89|4036.11|1492.31|2778.43|5528.42|14.92|0.00|1658.21|1492.31|1507.23|3150.52|3165.44|-1286.12| +2450837|73649|2450889|3099|572726|1906|37533|3099|572726|1906|37533|1|17|8|1|1657|107|1161|52|96.23|128.94|109.59|1006.20|5698.68|5003.96|6704.88|398.90|0.00|133.64|5698.68|6097.58|5832.32|6231.22|694.72| +2450837|73649|2450902|3099|572726|1906|37533|3099|572726|1906|37533|1|67|17|1|12292|297|1161|43|16.31|41.91|32.27|414.52|1387.61|701.33|1802.13|27.75|0.00|162.11|1387.61|1415.36|1549.72|1577.47|686.28| +2450837|73649|2450898|3099|572726|1906|37533|3099|572726|1906|37533|1|28|14|1|5284|107|1161|54|10.33|26.34|0.52|1394.28|28.08|557.82|1422.36|2.52|0.00|568.62|28.08|30.60|596.70|599.22|-529.74| +2450837|73649|2450922|3099|572726|1906|37533|3099|572726|1906|37533|1|7|1|4|15166|125|1161|18|5.77|15.05|7.22|140.94|129.96|103.86|270.90|5.19|0.00|40.50|129.96|135.15|170.46|175.65|26.10| +2450837|73649|2450868|3099|572726|1906|37533|3099|572726|1906|37533|1|10|4|2|11758|103|1161|83|15.52|33.52|21.11|1030.03|1752.13|1288.16|2782.16|105.12|0.00|0.00|1752.13|1857.25|1752.13|1857.25|463.97| +2450837|58112|2450859|3837|1596225|4613|17401|3837|1596225|4613|17401|2|50|14|3|10700|206|1162|71|26.95|70.60|56.48|1002.52|4010.08|1913.45|5012.60|360.90|0.00|751.89|4010.08|4370.98|4761.97|5122.87|2096.63| +2450837|58112|2450905|3837|1596225|4613|17401|3837|1596225|4613|17401|2|73|6|4|17258|215|1162|3|12.57|31.29|10.63|61.98|31.89|37.71|93.87|0.71|17.53|29.07|14.36|15.07|43.43|44.14|-23.35| +2450837|58112|2450925|3837|1596225|4613|17401|3837|1596225|4613|17401|2|106|19|1|13165|28|1162|87|31.40|55.89|45.82|876.09|3986.34|2731.80|4862.43|264.69|677.67|1555.56|3308.67|3573.36|4864.23|5128.92|576.87| +2450837|58112|2450844|3837|1596225|4613|17401|3837|1596225|4613|17401|2|34|2|4|12596|248|1162|69|27.29|35.74|33.59|148.35|2317.71|1883.01|2466.06|185.41|0.00|418.83|2317.71|2503.12|2736.54|2921.95|434.70| +2450837|58112|2450907|3837|1596225|4613|17401|3837|1596225|4613|17401|2|21|15|1|8386|286|1162|34|47.92|129.86|12.98|3973.92|441.32|1629.28|4415.24|17.65|0.00|573.92|441.32|458.97|1015.24|1032.89|-1187.96| +2450837|58112|2450856|3837|1596225|4613|17401|3837|1596225|4613|17401|2|23|14|1|7178|35|1162|44|1.48|4.21|0.42|166.76|18.48|65.12|185.24|0.00|0.00|23.76|18.48|18.48|42.24|42.24|-46.64| +2450837|58112|2450873|3837|1596225|4613|17401|3837|1596225|4613|17401|2|43|14|1|7940|8|1162|30|50.94|120.21|44.47|2272.20|1334.10|1528.20|3606.30|53.36|0.00|612.90|1334.10|1387.46|1947.00|2000.36|-194.10| +2450837|58112|2450907|3837|1596225|4613|17401|3837|1596225|4613|17401|2|20|11|2|2276|212|1162|66|69.45|78.47|18.04|3988.38|1190.64|4583.70|5179.02|19.28|869.16|1708.74|321.48|340.76|2030.22|2049.50|-4262.22| +2450837|71475|2450867|69890|1551151|4412|5273|69890|1551151|4412|5273|1|13|16|4|12688|295|1163|27|20.76|26.98|10.25|451.71|276.75|560.52|728.46|11.07|0.00|335.07|276.75|287.82|611.82|622.89|-283.77| +2450837|71475|2450858|69890|1551151|4412|5273|69890|1551151|4412|5273|1|6|18|5|2437|175|1163|69|63.74|142.77|55.68|6009.21|3841.92|4398.06|9851.13|67.61|2151.47|2659.26|1690.45|1758.06|4349.71|4417.32|-2707.61| +2450837|71475|2450864|69890|1551151|4412|5273|69890|1551151|4412|5273|1|45|18|1|10838|39|1163|38|59.16|107.07|58.88|1831.22|2237.44|2248.08|4068.66|89.49|0.00|1708.48|2237.44|2326.93|3945.92|4035.41|-10.64| +2450837|71475|2450899|69890|1551151|4412|5273|69890|1551151|4412|5273|1|105|13|3|2743|96|1163|73|75.75|190.13|36.12|11242.73|2636.76|5529.75|13879.49|210.94|0.00|3885.79|2636.76|2847.70|6522.55|6733.49|-2892.99| +2450837|71475|2450906|69890|1551151|4412|5273|69890|1551151|4412|5273|1|19|13|2|3148|81|1163|25|48.18|64.07|26.26|945.25|656.50|1204.50|1601.75|6.49|584.28|448.25|72.22|78.71|520.47|526.96|-1132.28| +2450837|71475|2450868|69890|1551151|4412|5273|69890|1551151|4412|5273|1|97|6|5|310|126|1163|17|95.18|145.62|132.51|222.87|2252.67|1618.06|2475.54|0.00|0.00|222.70|2252.67|2252.67|2475.37|2475.37|634.61| +2450837||2450855|||4412|||||||13|||14416|23|1163|88|94.50|207.90|83.16|10977.12|7318.08|8316.00|18295.20|512.26|0.00|0.00|7318.08||7318.08|7830.34|-997.92| +2450837|71475|2450896|69890|1551151|4412|5273|69890|1551151|4412|5273|1|41|20|1|14125|143|1163|42|1.08|2.10|1.95|6.30|81.90|45.36|88.20|0.00|0.00|30.66|81.90|81.90|112.56|112.56|36.54| +2450837|71475|2450876|69890|1551151|4412|5273|69890|1551151|4412|5273|1|31|13|3|8260|78|1163|76|46.52|74.43|64.00|792.68|4864.00|3535.52|5656.68|243.20|0.00|621.68|4864.00|5107.20|5485.68|5728.88|1328.48| +2450837|71475|2450878|69890|1551151|4412|5273|69890|1551151|4412|5273|1|66|9|2|13993|66|1163|89|30.03|55.85|31.27|2187.62|2783.03|2672.67|4970.65|55.66|0.00|2037.21|2783.03|2838.69|4820.24|4875.90|110.36| +2450837|71475|2450895|69890|1551151|4412|5273|69890|1551151|4412|5273|1|70|4|5|14500|114|1163|89|78.87|138.81|36.09|9142.08|3212.01|7019.43|12354.09|160.60|0.00|4447.33|3212.01|3372.61|7659.34|7819.94|-3807.42| +2450837|71475|2450903|69890|1551151|4412|5273|69890|1551151|4412|5273|1|54|8|2|12748|50|1163|31|73.97|148.67|115.96|1014.01|3594.76|2293.07|4608.77|323.52|0.00|2073.90|3594.76|3918.28|5668.66|5992.18|1301.69| +2450837|71475|2450860|69890|1551151|4412|5273|69890|1551151|4412|5273|1|36|1|2|5515|157|1163|86|30.35|30.65|29.42|105.78|2530.12|2610.10|2635.90|177.10|0.00|1317.52|2530.12|2707.22|3847.64|4024.74|-79.98| +2450837|71475|2450859|69890|1551151|4412|5273|69890|1551151|4412|5273|1|60|18|2|8842|119|1163|2|1.08|2.00|0.36|3.28|0.72|2.16|4.00|0.05|0.00|0.80|0.72|0.77|1.52|1.57|-1.44| +2450837|45499|2450846|18878|1781900|1369|15191|18878|1781900|1369|15191|4|4|12|3|12712|242|1164|43|81.43|238.58|28.62|9028.28|1230.66|3501.49|10258.94|3.44|1144.51|923.21|86.15|89.59|1009.36|1012.80|-3415.34| +2450837|45499|2450895|18878|1781900|1369|15191|18878|1781900|1369|15191|4|96|8|1|9416|17|1164|100|97.06|118.41|113.67|474.00|11367.00|9706.00|11841.00|341.01|0.00|3552.00|11367.00|11708.01|14919.00|15260.01|1661.00| +2450837|45499|2450860|18878|1781900|1369|15191|18878|1781900|1369|15191|4|4|13|2|8174|8|1164|7|34.25|47.60|12.37|246.61|86.59|239.75|333.20|0.00|0.00|159.88|86.59|86.59|246.47|246.47|-153.16| +2450837|45499|2450901|18878|1781900|1369|15191|18878|1781900|1369|15191|4|69|13|2|12454|295|1164|64|56.94|59.21|58.02|76.16|3713.28|3644.16|3789.44|111.39|0.00|681.60|3713.28|3824.67|4394.88|4506.27|69.12| +2450837|73194|2450861|22523|1142853|3183|713|22523|1142853|3183|713|4|88|13|1|15286|114|1165|9|32.82|82.70|23.15|535.95|208.35|295.38|744.30|4.16|0.00|230.67|208.35|212.51|439.02|443.18|-87.03| +2450837|73194|2450925|22523|1142853|3183|713|22523|1142853|3183|713|4|6|12|4|9400|66|1165|42|74.72|189.78|5.69|7731.78|238.98|3138.24|7970.76|0.00|0.00|796.74|238.98|238.98|1035.72|1035.72|-2899.26| +2450837|73194|2450892|22523|1142853|3183|713|22523|1142853|3183|713|4|43|11|1|10999|253|1165|27|60.78|117.30|52.78|1742.04|1425.06|1641.06|3167.10|128.25|0.00|284.85|1425.06|1553.31|1709.91|1838.16|-216.00| +2450837|73194|2450863|22523|1142853|3183|713|22523|1142853|3183|713|4|38|2|1|10562|94|1165|37|77.32|228.86|219.70|338.92|8128.90|2860.84|8467.82|0.00|0.00|253.82|8128.90|8128.90|8382.72|8382.72|5268.06| +2450837|73194|2450917|22523|1142853|3183|713|22523|1142853|3183|713|4|25|3|4|11863|164|1165|23|86.27|245.86|100.80|3336.38|2318.40|1984.21|5654.78|0.00|0.00|1583.32|2318.40|2318.40|3901.72|3901.72|334.19| +2450837|67750|2450865|54026|1018806|5539|45519|54026|1018806|5539|45519|4|53|9|3|13094|90|1166|36|28.39|33.50|26.13|265.32|940.68|1022.04|1206.00|16.46|705.51|349.56|235.17|251.63|584.73|601.19|-786.87| +2450837|67750|2450841|54026|1018806|5539|45519|54026|1018806|5539|45519|4|53|10|2|6271|269|1166|14|93.92|105.19|34.71|986.72|485.94|1314.88|1472.66|34.01|0.00|250.32|485.94|519.95|736.26|770.27|-828.94| +2450837|67750|2450925|54026|1018806|5539|45519|54026|1018806|5539|45519|4|17|8|2|4468|67|1166|71|5.26|12.30|5.90|454.40|418.90|373.46|873.30|7.87|222.01|112.89|196.89|204.76|309.78|317.65|-176.57| +2450837|67750|2450917|54026|1018806|5539|45519|54026|1018806|5539|45519|4|102|8|4|10268|123|1166|27|24.60|31.48|29.90|42.66|807.30|664.20|849.96|24.21|0.00|195.48|807.30|831.51|1002.78|1026.99|143.10| +2450837|67750|2450871|54026|1018806|5539|45519|54026|1018806|5539|45519|4|65|19|1|1969|262|1166|71|49.84|131.07|117.96|930.81|8375.16|3538.64|9305.97|586.26|0.00|4187.58|8375.16|8961.42|12562.74|13149.00|4836.52| +2450837|67750|2450852|54026|1018806|5539|45519|54026|1018806|5539|45519|4|27|6|3|16303|4|1166|40|29.23|81.84|20.46|2455.20|818.40|1169.20|3273.60|16.36|0.00|720.00|818.40|834.76|1538.40|1554.76|-350.80| +2450837|67750|2450848|54026|1018806|5539|45519|54026|1018806|5539|45519|4|11|5|3|2936|159|1166|88|50.76|147.20|145.72|130.24|12823.36|4466.88|12953.60|641.16|0.00|2331.12|12823.36|13464.52|15154.48|15795.64|8356.48| +2450837|67750|2450925|54026|1018806|5539|45519|54026|1018806|5539|45519|4|34|5|5|7264|265|1166|69|30.82|71.81|4.30|4658.19|296.70|2126.58|4954.89|11.86|0.00|1733.97|296.70|308.56|2030.67|2042.53|-1829.88| +2450837|67750|2450901|54026|1018806|5539|45519|54026|1018806|5539|45519|4|2|6|5|17186|13|1166|59|20.46|29.66|17.20|735.14|1014.80|1207.14|1749.94|10.14|0.00|559.91|1014.80|1024.94|1574.71|1584.85|-192.34| +2450837|67750|2450907|54026|1018806|5539|45519|54026|1018806|5539|45519|4|1|9|4|14854|7|1166|5|50.23|130.09|0.00|650.45|0.00|251.15|650.45|0.00|0.00|130.05|0.00|0.00|130.05|130.05|-251.15| +2450837|74243|2450909|74537|363780|1470|5913|74537|363780|1470|5913|4|84|5|2|3550|18|1167|42|77.47|145.64|13.10|5566.68|550.20|3253.74|6116.88|22.00|0.00|2935.80|550.20|572.20|3486.00|3508.00|-2703.54| +2450837|74243|2450850|74537|363780|1470|5913|74537|363780|1470|5913|4|87|11|2|8642|99|1167|51|57.93|60.24|34.93|1290.81|1781.43|2954.43|3072.24|0.00|0.00|1443.81|1781.43|1781.43|3225.24|3225.24|-1173.00| +2450837|74243|2450899|74537|363780|1470|5913|74537|363780|1470|5913|4|104|17|1|9955|21|1167|82|7.04|20.62|0.20|1674.44|16.40|577.28|1690.84|0.00|0.00|710.12|16.40|16.40|726.52|726.52|-560.88| +2450837|74243|2450890|74537|363780|1470|5913|74537|363780|1470|5913|4|20|18|3|13472|243|1167|45|24.75|68.06|54.44|612.90|2449.80|1113.75|3062.70|195.98|0.00|336.60|2449.80|2645.78|2786.40|2982.38|1336.05| +2450837|74243|2450926|74537|363780|1470|5913|74537|363780|1470|5913|4|24|8|1|7232|292|1167|83|25.73|74.10|60.76|1107.22|5043.08|2135.59|6150.30|151.29|0.00|2767.22|5043.08|5194.37|7810.30|7961.59|2907.49| +2450837|74243|2450914|74537|363780|1470|5913|74537|363780|1470|5913|4|78|4|4|8344|87|1167|51|83.41|150.97|57.36|4774.11|2925.36|4253.91|7699.47|175.52|0.00|1385.67|2925.36|3100.88|4311.03|4486.55|-1328.55| +2450837|74243|2450876|74537|363780|1470|5913|74537|363780|1470|5913|4|97|9|2|4540|186|1167|2|88.09|191.15|108.95|164.40|217.90|176.18|382.30|2.39|169.96|149.08|47.94|50.33|197.02|199.41|-128.24| +2450837|74243|2450847|74537|363780|1470|5913|74537|363780|1470|5913|4|46|10|5|11323|168|1167|30|15.02|15.77|0.31|463.80|9.30|450.60|473.10|0.40|2.51|118.20|6.79|7.19|124.99|125.39|-443.81| +2450837|74243|2450927|74537|363780|1470|5913|74537|363780|1470|5913|4|90|10|3|580|112|1167|40|35.83|106.05|61.50|1782.00|2460.00|1433.20|4242.00|0.00|0.00|1739.20|2460.00|2460.00|4199.20|4199.20|1026.80| +2450837|74243|2450911|74537|363780|1470|5913|74537|363780|1470|5913|4|40|12|1|7970|269|1167|71|80.76|117.10|48.01|4905.39|3408.71|5733.96|8314.10|68.17|0.00|2244.31|3408.71|3476.88|5653.02|5721.19|-2325.25| +2450837|74243|2450844|74537|363780|1470|5913|74537|363780|1470|5913|4|103|13|3|4|271|1167|67|26.98|78.24|67.28|734.32|4507.76|1807.66|5242.08|4.05|4102.06|2096.43|405.70|409.75|2502.13|2506.18|-1401.96| +2450837|74243|2450849|74537|363780|1470|5913|74537|363780|1470|5913|4|41|2|5|289|266|1167|47|20.50|45.71|39.31|300.80|1847.57|963.50|2148.37|110.85|0.00|837.54|1847.57|1958.42|2685.11|2795.96|884.07| +2450837|74243|2450884|74537|363780|1470|5913|74537|363780|1470|5913|4|7|16|5|11986|18|1167|21|14.50|17.83|3.38|303.45|70.98|304.50|374.43|0.00|0.00|29.82|70.98|70.98|100.80|100.80|-233.52| +2450837|25704|2450922|10750|1652544|6338|26132|10750|1652544|6338|26132|1|31|6|2|7348|204|1168|53|35.65|96.25|27.91|3622.02|1479.23|1889.45|5101.25|54.43|798.78|560.74|680.45|734.88|1241.19|1295.62|-1209.00| +2450837|25704|2450927|10750|1652544|6338|26132|10750|1652544|6338|26132|1|19|18|4|16958|263|1168|84|50.85|109.83|68.09|3506.16|5719.56|4271.40|9225.72|114.39|0.00|3044.16|5719.56|5833.95|8763.72|8878.11|1448.16| +2450837|25704|2450878|10750|1652544|6338|26132|10750|1652544|6338|26132|1|24|7|4|13084|158|1168|55|78.95|96.31|54.89|2278.10|3018.95|4342.25|5297.05|181.13|0.00|0.00|3018.95|3200.08|3018.95|3200.08|-1323.30| +2450837|25704|2450882|10750|1652544|6338|26132|10750|1652544|6338|26132|1|7|4|5|12658|67|1168|25|47.02|71.47|30.01|1036.50|750.25|1175.50|1786.75|15.00|0.00|857.50|750.25|765.25|1607.75|1622.75|-425.25| +2450837|25704|2450869|10750|1652544|6338|26132|10750|1652544|6338|26132|1|60|11|4|11006|220|1168|79|85.07|85.07|51.04|2688.37|4032.16|6720.53|6720.53|201.60|0.00|1612.39|4032.16|4233.76|5644.55|5846.15|-2688.37| +2450837|25704|2450909|10750|1652544|6338|26132|10750|1652544|6338|26132|1|11|2|2|14684|107|1168|67|28.70|39.89|23.13|1122.92|1549.71|1922.90|2672.63|61.98|0.00|1255.58|1549.71|1611.69|2805.29|2867.27|-373.19| +2450837|25704|2450841|10750|1652544|6338|26132|10750|1652544|6338|26132|1|79|18|2|10900|165|1168|100|97.76|153.48|42.97|11051.00|4297.00|9776.00|15348.00|85.94|0.00|5678.00|4297.00|4382.94|9975.00|10060.94|-5479.00| +2450837|75740|2450910|18741|858324|5907|290|18741|858324|5907|290|2|82|4|2|6430|135|1169|43|62.12|66.46|42.53|1028.99|1828.79|2671.16|2857.78|36.57|0.00|171.14|1828.79|1865.36|1999.93|2036.50|-842.37| +2450837|75740|2450862|18741|858324|5907|290|18741|858324|5907|290|2|53|2|2|15968|204|1169|72|43.16|55.24|4.41|3659.76|317.52|3107.52|3977.28|0.00|0.00|715.68|317.52|317.52|1033.20|1033.20|-2790.00| +2450837|75740|2450889|18741|858324|5907|290|18741|858324|5907|290|2|68|16|4|14842|169|1169|34|66.51|100.43|9.03|3107.60|307.02|2261.34|3414.62|12.28|0.00|1502.12|307.02|319.30|1809.14|1821.42|-1954.32| +2450837|75740|2450889|18741|858324|5907|290|18741|858324|5907|290|2|35|9|1|16810|123|1169|34|21.62|40.86|4.08|1250.52|138.72|735.08|1389.24|6.93|0.00|583.44|138.72|145.65|722.16|729.09|-596.36| +2450837|75740|2450842|18741|858324|5907|290|18741|858324|5907|290|2|32|11|3|326|290|1169|99|35.04|45.90|5.96|3954.06|590.04|3468.96|4544.10|12.74|165.21|1407.78|424.83|437.57|1832.61|1845.35|-3044.13| +2450837|75740|2450857|18741|858324|5907|290|18741|858324|5907|290|2|60|11|2|1543|287|1169|45|85.22|204.52|34.76|7639.20|1564.20|3834.90|9203.40|0.00|0.00|3865.05|1564.20|1564.20|5429.25|5429.25|-2270.70| +2450837|75740|2450881|18741|858324|5907|290|18741|858324|5907|290|2|65|12|4|7754|67|1169|7|78.87|199.54|55.87|1005.69|391.09|552.09|1396.78|3.91|0.00|223.44|391.09|395.00|614.53|618.44|-161.00| +2450837|80579|2450874|60710|888796|6129|9015|60710|888796|6129|9015|4|74|15|1|14948|253|1170|73|57.53|115.63|1.15|8357.04|83.95|4199.69|8440.99|5.87|0.00|3966.82|83.95|89.82|4050.77|4056.64|-4115.74| +2450837|80579|2450852|60710|888796|6129|9015|60710|888796|6129|9015|4|53|15|5|9454|208|1170|9|47.54|58.47|27.48|278.91|247.32|427.86|526.23|2.47|0.00|263.07|247.32|249.79|510.39|512.86|-180.54| +2450837|80579|2450905|60710|888796|6129|9015|60710|888796|6129|9015|4|75|6|1|7639|32|1170|22|60.11|68.52|65.77|60.50|1446.94|1322.42|1507.44|28.93|0.00|45.10|1446.94|1475.87|1492.04|1520.97|124.52| +2450837|80579|2450919|60710|888796|6129|9015|60710|888796|6129|9015|4|15|15|4|6679|152|1170|74|84.97|192.03|161.30|2274.02|11936.20|6287.78|14210.22|477.44|0.00|6252.26|11936.20|12413.64|18188.46|18665.90|5648.42| +2450837|80579|2450889|60710|888796|6129|9015|60710|888796|6129|9015|4|39|18|1|5342|85|1170|43|58.44|94.67|70.05|1058.66|3012.15|2512.92|4070.81|46.98|662.67|1221.20|2349.48|2396.46|3570.68|3617.66|-163.44| +2450837|80579|2450855|60710|888796|6129|9015|60710|888796|6129|9015|4|33|9|3|7453|160|1170|25|76.30|97.66|20.50|1929.00|512.50|1907.50|2441.50|15.37|0.00|73.00|512.50|527.87|585.50|600.87|-1395.00| +2450837|59706|2450903|74793|1158288|4183|38925|74793|1158288|4183|38925|4|56|14|1|16093|142|1171|4|59.36|82.51|4.95|310.24|19.80|237.44|330.04|1.58|0.00|141.88|19.80|21.38|161.68|163.26|-217.64| +2450837|59706|2450843|74793|1158288|4183|38925|74793|1158288|4183|38925|4|90|4|2|3350|103|1171|59|94.23|195.99|48.99|8673.00|2890.41|5559.57|11563.41|0.00|0.00|1734.01|2890.41|2890.41|4624.42|4624.42|-2669.16| +2450837|59706|2450926|74793|1158288|4183|38925|74793|1158288|4183|38925|4|15|11|2|1580|226|1171|33|30.61|78.05|70.24|257.73|2317.92|1010.13|2575.65|0.00|1066.24|1081.74|1251.68|1251.68|2333.42|2333.42|241.55| +2450837|59706|2450856|74793|1158288|4183|38925|74793|1158288|4183|38925|4|38|10|5|15538|55|1171|96|2.78|8.22|0.57|734.40|54.72|266.88|789.12|3.28|0.00|62.40|54.72|58.00|117.12|120.40|-212.16| +2450837|61388|2450840|58256|946792|6881|43417|58256|946792|6881|43417|4|30|7|4|13843|123|1172|42|9.14|24.22|15.25|376.74|640.50|383.88|1017.24|38.43|0.00|20.16|640.50|678.93|660.66|699.09|256.62| +2450837|61388|2450905|58256|946792|6881|43417|58256|946792|6881|43417|4|39|15|4|5911|57|1172|60|65.20|164.30|27.93|8182.20|1675.80|3912.00|9858.00|50.27|0.00|1478.40|1675.80|1726.07|3154.20|3204.47|-2236.20| +2450837|61388|2450888|58256|946792|6881|43417|58256|946792|6881|43417|4|95|4|1|12544|50|1172|25|86.42|231.60|164.43|1679.25|4110.75|2160.50|5790.00|328.86|0.00|984.25|4110.75|4439.61|5095.00|5423.86|1950.25| +2450837|61388|2450854|58256|946792|6881|43417|58256|946792|6881|43417|4|6|14|3|16868|198|1172|54|5.30|12.29|7.74|245.70|417.96|286.20|663.66|1.50|342.72|311.58|75.24|76.74|386.82|388.32|-210.96| +2450837|61388|2450839|58256|946792|6881|43417|58256|946792|6881|43417|4|82|7|2|13738|190|1172|33|10.51|13.76|8.94|159.06|295.02|346.83|454.08|20.65|0.00|104.28|295.02|315.67|399.30|419.95|-51.81| +2450837|61388|2450904|58256|946792|6881|43417|58256|946792|6881|43417|4|47|10|5|2098|105|1172|37|53.87|117.43|44.62|2693.97|1650.94|1993.19|4344.91|132.07|0.00|1998.37|1650.94|1783.01|3649.31|3781.38|-342.25| +2450837|61388|2450889|58256|946792|6881|43417|58256|946792|6881|43417|4|14|15|1|3343|33|1172|17|13.82|37.03|33.32|63.07|566.44|234.94|629.51|28.32|0.00|25.16|566.44|594.76|591.60|619.92|331.50| +2450837|61388|2450887|58256|946792|6881|43417|58256|946792|6881|43417|4|93|18|4|3046|233|1172|29|58.31|114.28|90.28|696.00|2618.12|1690.99|3314.12|52.36|0.00|1557.59|2618.12|2670.48|4175.71|4228.07|927.13| +2450837|61388|2450851|58256|946792|6881|43417|58256|946792|6881|43417|4|80|3|3|4400|219|1172|56|7.08|20.60|5.97|819.28|334.32|396.48|1153.60|26.74|0.00|322.56|334.32|361.06|656.88|683.62|-62.16| +2450837|61388|2450852|58256|946792|6881|43417|58256|946792|6881|43417|4|10|17|3|88|115|1172|17|89.05|154.05|16.94|2330.87|287.98|1513.85|2618.85|14.39|0.00|785.57|287.98|302.37|1073.55|1087.94|-1225.87| +2450837|61388|2450918|58256|946792|6881|43417|58256|946792|6881|43417|4|47|20|1|10970|231|1172|69|17.86|31.43|16.65|1019.82|1148.85|1232.34|2168.67|11.48|0.00|1062.60|1148.85|1160.33|2211.45|2222.93|-83.49| +2450837|61388|2450842|58256|946792|6881|43417|58256|946792|6881|43417|4|81|1|2|12868|136|1172|44|23.66|70.50|45.82|1085.92|2016.08|1041.04|3102.00|141.12|0.00|1395.68|2016.08|2157.20|3411.76|3552.88|975.04| +2450837|61388|2450892|58256|946792|6881|43417|58256|946792|6881|43417|4|105|5|2|13982|64|1172|30|43.78|52.09|45.31|203.40|1359.30|1313.40|1562.70|39.96|693.24|484.20|666.06|706.02|1150.26|1190.22|-647.34| +2450837|61388|2450889|58256|946792|6881|43417|58256|946792|6881|43417|4|78|2|5|1694|3|1172|50|75.79|156.12|103.03|2654.50|5151.50|3789.50|7806.00|257.57|0.00|3668.50|5151.50|5409.07|8820.00|9077.57|1362.00| +2450837|58018|2450892|20797|671578|6806|41587|20797|671578|6806|41587|4|57|13|2|4238|225|1173|64|77.97|141.12|128.41|813.44|8218.24|4990.08|9031.68|575.27|0.00|3883.52|8218.24|8793.51|12101.76|12677.03|3228.16| +2450837|58018|2450877|20797|671578|6806|41587|20797|671578|6806|41587|4|83|19|1|3068|13|1173|44|41.59|60.72|41.28|855.36|1816.32|1829.96|2671.68|0.00|0.00|320.32|1816.32|1816.32|2136.64|2136.64|-13.64| +2450837|58018|2450906|20797|671578|6806|41587|20797|671578|6806|41587|4|32|14|2|14896|201|1173|47|34.56|44.58|37.44|335.58|1759.68|1624.32|2095.26|68.62|615.88|20.68|1143.80|1212.42|1164.48|1233.10|-480.52| +2450837|58018|2450894|20797|671578|6806|41587|20797|671578|6806|41587|4|97|9|3|9979|160|1173|94|78.56|128.83|34.78|8840.70|3269.32|7384.64|12110.02|196.15|0.00|3511.84|3269.32|3465.47|6781.16|6977.31|-4115.32| +2450837|58018|2450862|20797|671578|6806|41587|20797|671578|6806|41587|4|52|7|2|14395|102|1173|74|92.87|200.59|2.00|14695.66|148.00|6872.38|14843.66|11.84|0.00|4452.58|148.00|159.84|4600.58|4612.42|-6724.38| +2450837|58018|2450876|20797|671578|6806|41587|20797|671578|6806|41587|4|83|4|4|8594|197|1173|63|20.53|60.76|43.13|1110.69|2717.19|1293.39|3827.88|190.20|0.00|1530.90|2717.19|2907.39|4248.09|4438.29|1423.80| +2450837|58018|2450904|20797|671578|6806|41587|20797|671578|6806|41587|4|60|4|5|2066|58|1173|86|69.28|184.28|158.48|2218.80|13629.28|5958.08|15848.08|136.29|0.00|2535.28|13629.28|13765.57|16164.56|16300.85|7671.20| +2450837|58018|2450892|20797|671578|6806|41587|20797|671578|6806|41587|4|73|4|1|8278|217|1173|43|41.61|66.99|41.53|1094.78|1785.79|1789.23|2880.57|25.00|1160.76|1094.35|625.03|650.03|1719.38|1744.38|-1164.20| +2450837|58018|2450897|20797|671578||||||41587|4|91|12||11590||1173||43.42||80.96||8015.04|4298.58||320.60|0.00||8015.04||10908.81|11229.41|3716.46| +2450837|58018|2450842|20797|671578|6806|41587|20797|671578|6806|41587|4|17|14|4|7609|199|1173|51|95.81|129.34|28.45|5145.39|1450.95|4886.31|6596.34|29.01|0.00|1318.86|1450.95|1479.96|2769.81|2798.82|-3435.36| +2450837|70856|2450839|19485|826210|2182|7307|19485|826210|2182|7307|4|4|18|5|8906|162|1174|82|97.43|194.86|190.96|319.80|15658.72|7989.26|15978.52|782.93|0.00|1917.16|15658.72|16441.65|17575.88|18358.81|7669.46| +2450837|70856|2450916|19485|826210|2182|7307|19485|826210|2182|7307|4|40|2|1|15824|274|1174|41|30.34|65.83|3.29|2564.14|134.89|1243.94|2699.03|2.87|39.11|1133.24|95.78|98.65|1229.02|1231.89|-1148.16| +2450837|70856|2450915|19485|826210|2182|7307|19485|826210|2182|7307|4|22|17|3|16915|245|1174|54|41.63|100.32|17.05|4496.58|920.70|2248.02|5417.28|64.44|0.00|1245.78|920.70|985.14|2166.48|2230.92|-1327.32| +2450837|70856|2450897|19485|826210|2182|7307|19485|826210|2182|7307|4|4|4|2|5972|240|1174|71|84.44|96.26|7.70|6287.76|546.70|5995.24|6834.46|0.00|0.00|2733.50|546.70|546.70|3280.20|3280.20|-5448.54| +2450838|58848|2450869|46521|13834|3900|24766|46521|13834|3900|24766|2|16|5|4|17542|137|1175|21|47.91|92.46|0.92|1922.34|19.32|1006.11|1941.66|0.38|0.00|504.63|19.32|19.70|523.95|524.33|-986.79| +2450838|58848|2450896|46521|13834|3900|24766|46521|13834|3900|24766|2|77|12|3|9434|28|1175|11|77.96|208.93|190.12|206.91|2091.32|857.56|2298.23|41.82|0.00|22.88|2091.32|2133.14|2114.20|2156.02|1233.76| +2450838|58848|2450852|46521|13834|3900|24766|46521|13834|3900|24766|2|31|6|1|2024|109|1175|65|20.10|49.04|2.94|2996.50|191.10|1306.50|3187.60|7.18|11.46|892.45|179.64|186.82|1072.09|1079.27|-1126.86| +2450838|58848|2450892|46521|13834|3900|24766|46521|13834|3900|24766|2|20|9|2|11845|128|1175|82|46.65|102.16|9.19|7623.54|753.58|3825.30|8377.12|7.53|0.00|2764.22|753.58|761.11|3517.80|3525.33|-3071.72| +2450838|58848|2450925|46521|13834|3900|24766|46521|13834|3900|24766|2|82|18|4|9871|134|1175|43|86.28|240.72|60.18|7763.22|2587.74|3710.04|10350.96|155.26|0.00|4864.59|2587.74|2743.00|7452.33|7607.59|-1122.30| +2450838|58848|2450842|46521|13834|3900|24766|46521|13834|3900|24766|2|92|8|1|15310|3|1175|60|29.36|70.46|62.70|465.60|3762.00|1761.60|4227.60|338.58|0.00|1183.20|3762.00|4100.58|4945.20|5283.78|2000.40| +2450838|58848|2450869|46521|13834|3900|24766|46521|13834|3900|24766|2|4|10|5|17677|174|1175|36|59.01|115.65|74.01|1499.04|2664.36|2124.36|4163.40|159.86|0.00|1873.44|2664.36|2824.22|4537.80|4697.66|540.00| +2450838|58848|2450897|46521|13834|3900|24766|46521|13834|3900|24766|2|36|9|5|6241|224|1175|63|16.18|44.33|20.83|1480.50|1312.29|1019.34|2792.79|26.24|0.00|977.13|1312.29|1338.53|2289.42|2315.66|292.95| +2450838|58848|2450910|46521|13834|3900|24766|46521|13834|3900|24766|2|11|3|5|6580|152|1175|11|71.43|145.00|36.25|1196.25|398.75|785.73|1595.00|27.91|0.00|223.30|398.75|426.66|622.05|649.96|-386.98| +2450838|52088|2450924|53093|225027|2114|36848|53093|225027|2114|36848|1|83|12|1|2252|297|1176|73|9.26|15.09|7.54|551.15|550.42|675.98|1101.57|49.53|0.00|286.16|550.42|599.95|836.58|886.11|-125.56| +2450838|52088|2450854|53093|225027|2114|36848|53093|225027|2114|36848|1|27|2|1|8536|19|1176|3|74.75|162.20|19.46|428.22|58.38|224.25|486.60|4.08|0.00|180.03|58.38|62.46|238.41|242.49|-165.87| +2450838|52088|2450901|53093|225027|2114|36848|53093|225027|2114|36848|1|98|8|2|15146|265|1176|32|64.36|85.59|4.27|2602.24|136.64|2059.52|2738.88|10.93|0.00|218.88|136.64|147.57|355.52|366.45|-1922.88| +2450838|52088|2450905|53093|225027|2114|36848|53093|225027|2114|36848|1|32|15|3|17839|271|1176|64|35.27|36.68|19.80|1080.32|1267.20|2257.28|2347.52|101.37|0.00|1079.68|1267.20|1368.57|2346.88|2448.25|-990.08| +2450838|52088|2450926|53093|225027|2114|36848|53093|225027|2114|36848|1|25|13|1|8872|247|1176|10|69.71|189.61|164.96|246.50|1649.60|697.10|1896.10|82.48|0.00|208.50|1649.60|1732.08|1858.10|1940.58|952.50| +2450838|52088|2450926|53093|225027|2114|36848|53093|225027|2114|36848|1|46|4|3|11641|206|1176|91|84.83|136.57|124.27|1119.30|11308.57|7719.53|12427.87|339.25|0.00|2236.78|11308.57|11647.82|13545.35|13884.60|3589.04| +2450838|52088|2450854|53093|225027|2114|36848|53093|225027|2114|36848|1|64|7|3|38|184|1176|27|84.27|143.25|1.43|3829.14|38.61|2275.29|3867.75|2.31|0.00|386.64|38.61|40.92|425.25|427.56|-2236.68| +2450838|52088|2450905|53093|225027|2114|36848|53093|225027|2114|36848|1|4|11|4|4196|225|1176|51|95.85|274.13|46.60|11604.03|2376.60|4888.35|13980.63|43.72|1829.98|1677.39|546.62|590.34|2224.01|2267.73|-4341.73| +2450838|52088|2450847|53093|225027|2114|36848|53093|225027|2114|36848|1|6|19|1|9370|3|1176|43|47.15|49.03|28.43|885.80|1222.49|2027.45|2108.29|12.22|0.00|1011.79|1222.49|1234.71|2234.28|2246.50|-804.96| +2450838|52088|2450856|53093|225027|2114|36848|53093|225027|2114|36848|1|88|20|2|14377|153|1176|92|49.25|49.74|5.47|4072.84|503.24|4531.00|4576.08|10.06|0.00|228.16|503.24|513.30|731.40|741.46|-4027.76| +2450838|52088|2450928|53093|225027|2114|36848|53093|225027|2114|36848|1|91|12|1|12961|103|1176|43|85.58|134.36|99.42|1502.42|4275.06|3679.94|5777.48|42.75|0.00|2715.02|4275.06|4317.81|6990.08|7032.83|595.12| +2450838|52088|2450859|53093|225027|2114|36848|53093|225027|2114|36848|1|79|14|1|7310|170|1176|73|38.60|53.26|51.66|116.80|3771.18|2817.80|3887.98|113.13|0.00|1787.77|3771.18|3884.31|5558.95|5672.08|953.38| +2450838|52088|2450855|53093|225027|2114|36848|53093|225027|2114|36848|1|53|12|2|6500|72|1176|64|10.55|29.32|7.91|1370.24|506.24|675.20|1876.48|20.24|0.00|449.92|506.24|526.48|956.16|976.40|-168.96| +2450838|52088|2450890|53093|225027|2114|36848|53093|225027|2114|36848|1|70|2|1|8906|10|1176|11|2.07|3.97|0.67|36.30|7.37|22.77|43.67|0.14|0.00|2.09|7.37|7.51|9.46|9.60|-15.40| +2450838|75447|2450865|87221|204752|1483|47061|87221|204752|1483|47061|4|33|13|1|10075|237|1177|33|13.79|23.02|20.02|99.00|660.66|455.07|759.66|0.00|145.34|204.93|515.32|515.32|720.25|720.25|60.25| +2450838|75447|2450845|87221|204752|1483|47061|87221|204752|1483|47061|4|98|4|4|13394|142|1177|2|22.75|30.48|0.30|60.36|0.60|45.50|60.96|0.04|0.00|10.36|0.60|0.64|10.96|11.00|-44.90| +2450838|75447|2450865|87221|204752|1483|47061|87221|204752|1483|47061|4|25|2|3|9566|130|1177|43|18.64|49.58|19.33|1300.75|831.19|801.52|2131.94|0.00|0.00|340.99|831.19|831.19|1172.18|1172.18|29.67| +2450838|75447|2450847|87221|204752|1483|47061|87221|204752|1483|47061|4|41|19|1|9019|238|1177|35|61.96|77.45|0.00|2710.75|0.00|2168.60|2710.75|0.00|0.00|704.55|0.00|0.00|704.55|704.55|-2168.60| +2450838|75447|2450843|87221|204752|1483|47061|87221|204752|1483|47061|4|23|16|4|13354|5|1177|80|12.96|20.34|10.17|813.60|813.60|1036.80|1627.20|40.68|0.00|162.40|813.60|854.28|976.00|1016.68|-223.20| +2450838|75447|2450902|87221|204752|1483|47061|87221|204752|1483|47061|4|28|2|5|9856|169|1177|65|49.65|119.65|47.86|4666.35|3110.90|3227.25|7777.25|0.00|0.00|3888.30|3110.90|3110.90|6999.20|6999.20|-116.35| +2450838|37604|2450892|56889|234668|4447|45826|56889|234668|4447|45826|1|80|1|2|17521|27|1178|68|63.35|112.12|72.87|2669.00|4955.16|4307.80|7624.16|297.30|0.00|2744.48|4955.16|5252.46|7699.64|7996.94|647.36| +2450838|37604|2450866|56889|234668|4447|45826|56889|234668|4447|45826|1|42|6|1|10981|258|1178|58|21.67|36.62|8.78|1614.72|509.24|1256.86|2123.96|10.18|0.00|828.24|509.24|519.42|1337.48|1347.66|-747.62| +2450838|37604|2450842|56889|234668|4447|45826|56889|234668|4447|45826|1|31|18|1|15796|197|1178|15|78.42|151.35|72.64|1180.65|1089.60|1176.30|2270.25|21.79|0.00|635.55|1089.60|1111.39|1725.15|1746.94|-86.70| +2450838|37604|2450921|56889|234668|4447|45826|56889|234668|4447|45826|1|35|6|1|8752|228|1178|4|79.04|139.11|50.07|356.16|200.28|316.16|556.44|1.76|156.21|127.96|44.07|45.83|172.03|173.79|-272.09| +2450838|37604|2450914|56889|234668|4447|45826|56889|234668|4447|45826|1|28|17|4|2443|110|1178|99|86.12|243.71|129.16|11340.45|12786.84|8525.88|24127.29|895.07|0.00|11097.90|12786.84|13681.91|23884.74|24779.81|4260.96| +2450838|37604|2450840|56889|234668|4447|45826|56889|234668|4447|45826|1|53|4|4|15380|271|1178|71|97.43|236.75|160.99|5378.96|11430.29|6917.53|16809.25|457.21|0.00|3193.58|11430.29|11887.50|14623.87|15081.08|4512.76| +2450838|37604|2450904|56889|234668|4447|45826|56889|234668|4447|45826|1|52|7|1|8944|4|1178|49|71.83|107.02|99.52|367.50|4876.48|3519.67|5243.98|341.35|0.00|891.31|4876.48|5217.83|5767.79|6109.14|1356.81| +2450838|37604|2450857|56889|234668|4447|45826|56889|234668|4447|45826|1|40|18|3|7213|230|1178|20|81.57|110.11|27.52|1651.80|550.40|1631.40|2202.20|16.51|0.00|924.80|550.40|566.91|1475.20|1491.71|-1081.00| +2450838|37604|2450860|56889|234668|4447|45826|56889|234668|4447|45826|1|57|6|2|13094|16|1178|93|84.06|189.13|77.54|10377.87|7211.22|7817.58|17589.09|576.89|0.00|2989.95|7211.22|7788.11|10201.17|10778.06|-606.36| +2450838|37604|2450872|56889|234668|4447|45826|56889|234668|4447|45826|1|22|6|2|6271|1|1178|37|43.94|100.62|55.34|1675.36|2047.58|1625.78|3722.94|20.47|0.00|818.81|2047.58|2068.05|2866.39|2886.86|421.80| +2450838|37604|2450860|56889|234668|4447|45826|56889|234668|4447|45826|1|101|16|5|4468|128|1178|73|15.15|37.26|11.55|1876.83|843.15|1105.95|2719.98|8.43|0.00|271.56|843.15|851.58|1114.71|1123.14|-262.80| +2450838|20045|2450892|97004|566411|6218|39672|97004|566411|6218|39672|1|41|10|5|17536|274|1179|47|67.44|153.76|0.00|7226.72|0.00|3169.68|7226.72|0.00|0.00|2890.50|0.00|0.00|2890.50|2890.50|-3169.68| +2450838|20045|2450868|97004|566411|6218|39672|97004|566411|6218|39672|1|90|20|1|12211|256|1179|42|76.81|112.14|58.31|2260.86|2449.02|3226.02|4709.88|167.51|587.76|988.68|1861.26|2028.77|2849.94|3017.45|-1364.76| +2450838|20045|2450894|97004|566411|6218|39672|97004|566411|6218|39672|1|100|7|2|6340|197|1179|4|99.75|226.43|97.36|516.28|389.44|399.00|905.72|15.57|0.00|407.56|389.44|405.01|797.00|812.57|-9.56| +2450838|20045|2450864|97004|566411|6218|39672|97004|566411|6218|39672|1|106|4|4|8894|112|1179|26|74.45|182.40|158.68|616.72|4125.68|1935.70|4742.40|123.77|0.00|663.78|4125.68|4249.45|4789.46|4913.23|2189.98| +2450838|26514|2450916|32539|42584|4629|27521|86481|606411|4456|42948|2|92|17|5|619|111|1180|6|94.11|106.34|80.81|153.18|484.86|564.66|638.04|33.94|0.00|197.76|484.86|518.80|682.62|716.56|-79.80| +2450838|26514|2450868|32539|42584|4629|27521|86481|606411|4456|42948|2|29|4|1|4070|9|1180|34|54.65|142.63|68.46|2521.78|2327.64|1858.10|4849.42|209.48|0.00|678.64|2327.64|2537.12|3006.28|3215.76|469.54| +2450838|26514|2450875|32539|42584|4629|27521|86481|606411|4456|42948|2|73|1|1|2938|103|1180|5|27.67|60.59|31.50|145.45|157.50|138.35|302.95|6.30|0.00|124.20|157.50|163.80|281.70|288.00|19.15| +2450838|26514|2450871|32539|42584|4629|27521|86481|606411|4456|42948|2|30|1|1|11966|208|1180|41|79.97|98.36|96.39|80.77|3951.99|3278.77|4032.76|0.00|0.00|1411.22|3951.99|3951.99|5363.21|5363.21|673.22| +2450838|26514|2450882|32539|42584|4629|27521|86481|606411|4456|42948|2|40|1|1|15934|280|1180|16|9.82|18.36|9.73|138.08|155.68|157.12|293.76|14.01|0.00|46.88|155.68|169.69|202.56|216.57|-1.44| +2450838|26514|2450874|32539|42584|4629|27521|86481|606411|4456|42948|2|83|12|5|5863|168|1180|56|66.92|123.80|27.23|5407.92|1524.88|3747.52|6932.80|137.23|0.00|554.40|1524.88|1662.11|2079.28|2216.51|-2222.64| +2450838|26514|2450861|32539|42584|4629|27521|86481|606411|4456|42948|2|98|14|5|7261|229|1180|100|36.48|102.50|44.07|5843.00|4407.00|3648.00|10250.00|88.14|0.00|1742.00|4407.00|4495.14|6149.00|6237.14|759.00| +2450838|26514|2450844|32539|42584|4629|27521|86481|606411|4456|42948|2|72|16|1|4162|23|1180|51|61.12|105.12|55.71|2519.91|2841.21|3117.12|5361.12|198.88|0.00|964.92|2841.21|3040.09|3806.13|4005.01|-275.91| +2450838|26514|2450906|32539|42584|4629|27521|86481|606411|4456|42948|2|41|14|5|5924|10|1180|7|68.19|185.47|72.33|791.98|506.31|477.33|1298.29|40.50|0.00|649.11|506.31|546.81|1155.42|1195.92|28.98| +2450838|26514|2450867|32539|42584|4629|27521|86481|606411|4456|42948|2|81|20|4|15208|5|1180|45|13.64|32.87|6.24|1198.35|280.80|613.80|1479.15|0.00|0.00|428.85|280.80|280.80|709.65|709.65|-333.00| +2450838|26514|2450919|32539|42584|4629|27521|86481|606411|4456|42948|2|34|13|3|16675|261|1180|11|18.90|56.70|19.84|405.46|218.24|207.90|623.70|0.00|0.00|0.00|218.24|218.24|218.24|218.24|10.34| +2450838|26514|2450844|32539|42584|4629|27521|86481|606411|4456|42948|2|29|8|1|10072|289|1180|84|34.08|66.79|40.74|2188.20|3422.16|2862.72|5610.36|208.40|444.88|1177.68|2977.28|3185.68|4154.96|4363.36|114.56| +2450838|26514|2450870|32539|42584|4629|27521|86481|606411|4456|42948|2|83|17|4|11438|131|1180|55|27.66|33.19|0.00|1825.45|0.00|1521.30|1825.45|0.00|0.00|474.10|0.00|0.00|474.10|474.10|-1521.30| +2450838|26514|2450908|32539|42584|4629|27521|86481|606411|4456|42948|2|80|18|2|16858|170|1180|79|21.58|44.23|37.59|524.56|2969.61|1704.82|3494.17|84.63|148.48|1012.78|2821.13|2905.76|3833.91|3918.54|1116.31| +2450838|70198|2450907|4099|1420721|6943|33965|4099|1420721|6943|33965|1|106|18|4|9526|83|1181|24|5.69|8.25|7.92|7.92|190.08|136.56|198.00|1.90|0.00|17.76|190.08|191.98|207.84|209.74|53.52| +2450838|70198|2450843|4099|1420721|6943|33965|4099|1420721|6943|33965|1|83|1|5|3484|282|1181|84|76.01|227.26|220.44|572.88|18516.96|6384.84|19089.84|1296.18|0.00|6872.04|18516.96|19813.14|25389.00|26685.18|12132.12| +2450838|70198|2450863|4099|1420721|6943|33965|4099|1420721|6943|33965|1|11|13|4|16096|111|1181|95|50.77|132.50|75.52|5413.10|7174.40|4823.15|12587.50|430.46|0.00|2391.15|7174.40|7604.86|9565.55|9996.01|2351.25| +2450838|70198|2450895|4099|1420721|6943|33965|4099|1420721|6943|33965|1|38|19|3|12217|210|1181|52|74.84|216.28|157.88|3036.80|8209.76|3891.68|11246.56|101.80|5664.73|5285.80|2545.03|2646.83|7830.83|7932.63|-1346.65| +2450838|70198|2450923|4099|1420721|6943|33965|4099|1420721|6943|33965|1|44|6|5|14131|65|1181|61|58.93|166.77|111.73|3357.44|6815.53|3594.73|10172.97|229.00|1090.48|3865.57|5725.05|5954.05|9590.62|9819.62|2130.32| +2450838|70198|2450861|4099|1420721|6943|33965|4099|1420721|6943|33965|1|99|12|3|8029|68|1181|88|92.35|268.73|257.98|946.00|22702.24|8126.80|23648.24|2043.20|0.00|10405.12|22702.24|24745.44|33107.36|35150.56|14575.44| +2450838|70198|2450887|4099|1420721|6943|33965|4099|1420721|6943|33965|1|4|13|4|8731|287|1181|78|95.85|233.87|112.25|9486.36|8755.50|7476.30|18241.86|437.77|0.00|1823.64|8755.50|9193.27|10579.14|11016.91|1279.20| +2450838|70198|2450883|4099|1420721|6943|33965|4099|1420721|6943|33965|1|99|19|4|1310|240|1181|24|24.15|46.12|35.97|243.60|863.28|579.60|1106.88|51.79|0.00|99.60|863.28|915.07|962.88|1014.67|283.68| +2450838|70198|2450880|4099|1420721|6943|33965|4099|1420721|6943|33965|1|11|15|1|2654|228|1181|30|7.15|9.58|8.52|31.80|255.60|214.50|287.40|17.89|0.00|111.90|255.60|273.49|367.50|385.39|41.10| +2450838|70198|2450926|4099|1420721|6943|33965|4099|1420721|6943|33965|1|74|7|5|745|288|1181|61|81.95|92.60|36.11|3445.89|2202.71|4998.95|5648.60|176.21|0.00|2710.84|2202.71|2378.92|4913.55|5089.76|-2796.24| +2450838|77875|2450853|90204|1312517|4077|37590|56777|1796263|1423|6536|4|82|8|4|13591|109|1182|9|86.30|192.44|136.63|502.29|1229.67|776.70|1731.96|49.18|0.00|484.92|1229.67|1278.85|1714.59|1763.77|452.97| +2450838|77875|2450845|90204|1312517|4077|37590|56777|1796263|1423|6536|4|106|17|1|4051|276|1182|34|90.38|169.91|23.78|4968.42|808.52|3072.92|5776.94|32.34|0.00|1270.92|808.52|840.86|2079.44|2111.78|-2264.40| +2450838|77875|2450911|90204|1312517|4077|37590|56777|1796263|1423|6536|4|78|4|1|14908|290|1182|19|67.21|192.89|135.02|1099.53|2565.38|1276.99|3664.91|15.39|2257.53|806.17|307.85|323.24|1114.02|1129.41|-969.14| +2450838|77875|2450878|90204|1312517|4077|37590|56777|1796263|1423|6536|4|5|5|4|17870|194|1182|20|92.59|201.84|193.76|161.60|3875.20|1851.80|4036.80|155.00|0.00|1412.80|3875.20|4030.20|5288.00|5443.00|2023.40| +2450838|77875|2450928|90204|1312517|4077|37590|56777|1796263|1423|6536|4|18|17|5|9481|71|1182|28|49.43|102.32|10.23|2578.52|286.44|1384.04|2864.96|14.32|0.00|544.32|286.44|300.76|830.76|845.08|-1097.60| +2450838|77875|2450928|90204|1312517|4077|37590|56777|1796263|1423|6536|4|38|12|3|13045|263|1182|84|38.30|56.68|13.60|3618.72|1142.40|3217.20|4761.12|0.00|0.00|237.72|1142.40|1142.40|1380.12|1380.12|-2074.80| +2450838|77875|2450865|90204|1312517|4077|37590|56777|1796263|1423|6536|4|36|5|4|4057|74|1182|59|86.35|165.79|26.52|8216.93|1564.68|5094.65|9781.61|78.23|0.00|4010.23|1564.68|1642.91|5574.91|5653.14|-3529.97| +2450838|77875|2450906|90204|1312517|4077|37590|56777|1796263|1423|6536|4|66|9|1|14281|19|1182|100|77.06|214.22|19.27|19495.00|1927.00|7706.00|21422.00|19.27|0.00|6212.00|1927.00|1946.27|8139.00|8158.27|-5779.00| +2450838|77875|2450902|90204|1312517|4077|37590|56777|1796263|1423|6536|4|64|16|5|14168|238|1182|76|67.86|179.15|161.23|1361.92|12253.48|5157.36|13615.40|0.00|0.00|5718.24|12253.48|12253.48|17971.72|17971.72|7096.12| +2450838|77875|2450870|90204|1312517|4077|37590|56777|1796263|1423|6536|4|47|18|3|10910|263|1182|57|55.88|127.40|84.08|2469.24|4792.56|3185.16|7261.80|191.70|0.00|1379.40|4792.56|4984.26|6171.96|6363.66|1607.40| +2450838|72880|2450917|1395|397997|5799|17601|2074|1430005|1007|39209|4|57|20|4|10358|127|1183|42|7.93|21.09|19.82|53.34|832.44|333.06|885.78|0.00|466.16|26.46|366.28|366.28|392.74|392.74|33.22| +2450838|72880|2450898|1395|397997|5799|17601|2074|1430005|1007|39209|4|75|1|3|16052|259|1183|68|39.32|61.73|48.14|924.12|3273.52|2673.76|4197.64|294.61|0.00|1049.24|3273.52|3568.13|4322.76|4617.37|599.76| +2450838|72880|2450861|1395|397997|5799|17601|2074|1430005|1007|39209|4|75|7|5|3524|208|1183|100|63.58|85.83|62.65|2318.00|6265.00|6358.00|8583.00|563.85|0.00|686.00|6265.00|6828.85|6951.00|7514.85|-93.00| +2450838|72880|2450908|1395|397997|5799|17601|2074|1430005|1007|39209|4|40|1|3|8857|74|1183|11|33.46|91.68|41.25|554.73|453.75|368.06|1008.48|27.22|0.00|10.01|453.75|480.97|463.76|490.98|85.69| +2450838|60311|2450881|50565|1831824|283|45942|50565|1831824|283|45942|2|75|14|1|10177|190|1184|91|77.53|155.83|67.00|8083.53|6097.00|7055.23|14180.53|60.97|0.00|1700.79|6097.00|6157.97|7797.79|7858.76|-958.23| +2450838|60311|2450860|50565|1831824|283|45942|50565|1831824|283|45942|2|29|1|3|17131|292|1184|15|66.69|110.03|86.92|346.65|1303.80|1000.35|1650.45|0.00|0.00|379.50|1303.80|1303.80|1683.30|1683.30|303.45| +2450838|60311|2450849|50565|1831824|283|45942|50565|1831824|283|45942|2|28|16|2|5203|143|1184|36|12.95|21.10|12.87|296.28|463.32|466.20|759.60|13.89|0.00|60.48|463.32|477.21|523.80|537.69|-2.88| +2450838|60311|2450876|50565|1831824|283|45942|50565|1831824|283|45942|2|80|3|3|7912|163|1184|62|98.03|292.12|37.97|15757.30|2354.14|6077.86|18111.44|188.33|0.00|6700.96|2354.14|2542.47|9055.10|9243.43|-3723.72| +2450838|60311|2450926|50565|1831824|283|45942|50565|1831824|283|45942|2|81|11|2|4532|119|1184|75|72.32|163.44|29.41|10052.25|2205.75|5424.00|12258.00|44.11|0.00|5515.50|2205.75|2249.86|7721.25|7765.36|-3218.25| +2450838|60311|2450862|50565|1831824|283|45942|50565|1831824|283|45942|2|56|20|2|589|186|1184|96|37.55|107.39|53.69|5155.20|5154.24|3604.80|10309.44|309.25|0.00|4020.48|5154.24|5463.49|9174.72|9483.97|1549.44| +2450838|60311|2450882|50565|1831824|283|45942|50565|1831824|283|45942|2|9|19|1|15046|160|1184|53|3.03|7.30|7.22|4.24|382.66|160.59|386.90|0.07|378.83|42.40|3.83|3.90|46.23|46.30|-156.76| +2450838|60311|2450902|50565|1831824|283|45942|50565|1831824|283|45942|2|98|14|2|14356|267|1184|50|93.58|128.20|78.20|2500.00|3910.00|4679.00|6410.00|234.60|0.00|1474.00|3910.00|4144.60|5384.00|5618.60|-769.00| +2450838|69097|2450897|72866|116845|4024|34850|72866|116845|4024|34850|2|49|6|1|6266|91|1185|46|87.86|158.14|101.20|2619.24|4655.20|4041.56|7274.44|139.65|0.00|3491.40|4655.20|4794.85|8146.60|8286.25|613.64| +2450838|69097|2450895|72866|116845|4024|34850|72866|116845|4024|34850|2|102|4|4|5462|156|1185|85|82.77|207.75|201.51|530.40|17128.35|7035.45|17658.75|1370.26|0.00|1235.90|17128.35|18498.61|18364.25|19734.51|10092.90| +2450838|69097|2450922|72866|116845|4024|34850|72866|116845|4024|34850|2|80|17|2|874|35|1185|84|61.71|156.74|72.10|7109.76|6056.40|5183.64|13166.16|484.51|0.00|4344.48|6056.40|6540.91|10400.88|10885.39|872.76| +2450838|69097|2450840|72866|116845|4024|34850|72866|116845|4024|34850|2|10|8|1|3488|294|1185|56|38.22|114.27|113.12|64.40|6334.72|2140.32|6399.12|190.04|0.00|2175.60|6334.72|6524.76|8510.32|8700.36|4194.40| +2450838|69097|2450885|72866|116845|4024|34850|72866|116845|4024|34850|2|100|13|2|6742|190|1185|24|58.81|153.49|92.09|1473.60|2210.16|1411.44|3683.76|22.10|0.00|331.44|2210.16|2232.26|2541.60|2563.70|798.72| +2450838|69097|2450903|72866|116845|4024|34850|72866|116845|4024|34850|2|102|4|1|1676|246|1185|57|20.20|39.79|26.26|771.21|1496.82|1151.40|2268.03|29.93|0.00|793.44|1496.82|1526.75|2290.26|2320.19|345.42| +2450838|69097|2450874|72866|116845|4024|34850|72866|116845|4024|34850|2|101|7|1|9556|258|1185|11|94.54|274.16|172.72|1115.84|1899.92|1039.94|3015.76|170.99|0.00|1145.98|1899.92|2070.91|3045.90|3216.89|859.98| +2450838|69097|2450905|72866|116845|4024|34850|72866|116845|4024|34850|2|15|13|4|2785|179|1185|58|58.06|101.60|27.43|4301.86|1590.94|3367.48|5892.80|63.63|0.00|1060.24|1590.94|1654.57|2651.18|2714.81|-1776.54| +2450838|69097|2450904|72866|116845|4024|34850|72866|116845|4024|34850|2|24|8|5|5714|173|1185|68|10.75|29.99|8.69|1448.40|590.92|731.00|2039.32|9.45|401.82|611.32|189.10|198.55|800.42|809.87|-541.90| +2450838|69097|2450915|72866|116845|4024|34850|72866|116845|4024|34850|2|107|6|3|8852|170|1185|65|67.30|136.61|91.52|2930.85|5948.80|4374.50|8879.65|356.92|0.00|2930.20|5948.80|6305.72|8879.00|9235.92|1574.30| +2450838|69097|2450923|72866|116845|4024|34850|72866|116845|4024|34850|2|67|8|1|7468|237|1185|57|84.63|157.41|73.98|4755.51|4216.86|4823.91|8972.37|379.51|0.00|2332.44|4216.86|4596.37|6549.30|6928.81|-607.05| +2450838|69097|2450896|72866|116845|4024|34850|72866|116845|4024|34850|2|28|15|3|10159|234|1185|59|89.88|230.09|23.00|12218.31|1357.00|5302.92|13575.31|54.28|0.00|6651.66|1357.00|1411.28|8008.66|8062.94|-3945.92| +2450838|69097|2450924|72866|116845|4024|34850|72866|116845|4024|34850|2|43|16|1|4627|276|1185|14|61.82|155.16|125.67|412.86|1759.38|865.48|2172.24|35.18|0.00|412.72|1759.38|1794.56|2172.10|2207.28|893.90| +2450838|69097|2450878|72866|116845|4024|34850|72866|116845|4024|34850|2|8|8|1|17906|284|1185|55|22.78|26.42|2.90|1293.60|159.50|1252.90|1453.10|9.57|0.00|580.80|159.50|169.07|740.30|749.87|-1093.40| +2450838|35177|2450850|42510|1002122|4934|22946|42510|1002122|4934|22946|1|89|11|1|8479|24|1186|44|76.75|127.40|19.11|4764.76|840.84|3377.00|5605.60|42.04|0.00|1008.92|840.84|882.88|1849.76|1891.80|-2536.16| +2450838|35177|2450868|42510|1002122|4934|22946|42510|1002122|4934|22946|1|22|17|4|6595|243|1186|33|58.98|156.29|129.72|876.81|4280.76|1946.34|5157.57|385.26|0.00|618.75|4280.76|4666.02|4899.51|5284.77|2334.42| +2450838|35177|2450899|42510|1002122|4934|22946|42510|1002122|4934|22946|1|74|18|5|16024|102|1186|38|96.95|171.60|82.36|3391.12|3129.68|3684.10|6520.80|114.23|845.01|1369.14|2284.67|2398.90|3653.81|3768.04|-1399.43| +2450838|35177|2450846|42510|1002122|4934|22946|42510|1002122|4934|22946|1|17|2|5|15982|193|1186|46|61.69|154.22|128.00|1206.12|5888.00|2837.74|7094.12|58.88|0.00|2270.10|5888.00|5946.88|8158.10|8216.98|3050.26| +2450838|35177|2450928|42510|1002122|4934|22946|42510|1002122|4934|22946|1|9|17|4|16855|221|1186|14|6.63|19.69|5.31|201.32|74.34|92.82|275.66|3.71|0.00|115.64|74.34|78.05|189.98|193.69|-18.48| +2450838|35177|2450861|42510|1002122|4934|22946|42510|1002122|4934|22946|1|78|14|2|5128|115|1186|34|49.48|146.95|55.84|3097.74|1898.56|1682.32|4996.30|132.89|0.00|1648.66|1898.56|2031.45|3547.22|3680.11|216.24| +2450838|35177|2450894|42510|1002122|4934|22946|42510|1002122|4934|22946|1|101|10|2|17764|176|1186|82|18.61|33.68|8.75|2044.26|717.50|1526.02|2761.76|64.57|0.00|1159.48|717.50|782.07|1876.98|1941.55|-808.52| +2450838|35177|2450863|42510|1002122|4934|22946|42510|1002122|4934|22946|1|40|20|2|5263|241|1186|26|88.52|169.07|3.38|4307.94|87.88|2301.52|4395.82|3.51|0.00|791.18|87.88|91.39|879.06|882.57|-2213.64| +2450838|35177|2450923|42510|1002122|4934|22946|42510|1002122|4934|22946|1|6|8|5|3826|65|1186|14|51.90|148.95|25.32|1730.82|354.48|726.60|2085.30|18.71|42.53|792.40|311.95|330.66|1104.35|1123.06|-414.65| +2450838|35177|2450879|42510|1002122|4934|22946|42510|1002122|4934|22946|1|77|18|4|8827|132|1186|45|7.47|15.53|3.26|552.15|146.70|336.15|698.85|4.40|0.00|244.35|146.70|151.10|391.05|395.45|-189.45| +2450838|35177|2450875|42510|1002122|4934|22946|42510|1002122|4934|22946|1|63|18|5|1376|251|1186|7|22.99|47.81|8.12|277.83|56.84|160.93|334.67|1.70|0.00|110.39|56.84|58.54|167.23|168.93|-104.09| +2450838|35177|2450925|42510|1002122|4934|22946|42510|1002122|4934|22946|1|16|14|3|16510|289|1186|34|51.86|53.93|3.77|1705.44|128.18|1763.24|1833.62|5.12|0.00|329.80|128.18|133.30|457.98|463.10|-1635.06| +2450838|35177|2450907|42510|1002122|4934|22946|42510|1002122|4934|22946|1|19|12|1|16849|106|1186|64|93.66|102.08|38.79|4050.56|2482.56|5994.24|6533.12|74.47|0.00|456.96|2482.56|2557.03|2939.52|3013.99|-3511.68| +2450838|48434|2450888|28035|1700773|2311|44696|28035|1700773|2311|44696|4|32|5|5|8894|29|1187|88|56.65|66.28|7.29|5191.12|641.52|4985.20|5832.64|25.66|0.00|2099.68|641.52|667.18|2741.20|2766.86|-4343.68| +2450838|48434|2450897|28035|1700773|2311|44696|28035|1700773|2311|44696|4|70|8|1|7460|52|1187|87|33.23|45.52|9.55|3129.39|830.85|2891.01|3960.24|66.46|0.00|910.02|830.85|897.31|1740.87|1807.33|-2060.16| +2450838|48434|2450910|28035|1700773|2311|44696|28035|1700773|2311|44696|4|22|7|2|15560|190|1187|50|66.93|90.35|87.63|136.00|4381.50|3346.50|4517.50|350.52|0.00|1897.00|4381.50|4732.02|6278.50|6629.02|1035.00| +2450838|48434|2450888|28035|1700773|2311|44696|28035|1700773|2311|44696|4|101|6|5|472|176|1187|34|37.14|37.14|16.34|707.20|555.56|1262.76|1262.76|50.00|0.00|568.14|555.56|605.56|1123.70|1173.70|-707.20| +2450838|48434|2450848|28035|1700773|2311|44696|28035|1700773|2311|44696|4|43|15|4|14456|266|1187|42|4.69|5.48|3.28|92.40|137.76|196.98|230.16|0.33|129.49|39.06|8.27|8.60|47.33|47.66|-188.71| +2450838|48434|2450867|28035|1700773|2311|44696|28035|1700773|2311|44696|4|100|18|4|10534|291|1187|47|35.04|92.85|81.70|524.05|3839.90|1646.88|4363.95|115.19|0.00|654.24|3839.90|3955.09|4494.14|4609.33|2193.02| +2450838|48434|2450865|28035|1700773|2311|44696|28035|1700773|2311|44696|4|32|14|1|15667|77|1187|14|72.01|175.70|112.44|885.64|1574.16|1008.14|2459.80|0.00|0.00|172.06|1574.16|1574.16|1746.22|1746.22|566.02| +2450838|58783|2450927|86651|1455044|3884|677|86651|1455044|3884|677|2|34|19|4|15787|264|1188|41|46.83|94.59|0.00|3878.19|0.00|1920.03|3878.19|0.00|0.00|1318.56|0.00|0.00|1318.56|1318.56|-1920.03| +2450838|58783|2450880|86651|1455044|3884|677|86651|1455044|3884|677|2|36|12|2|11857|167|1188|42|47.96|64.26|8.99|2321.34|377.58|2014.32|2698.92|26.43|0.00|674.52|377.58|404.01|1052.10|1078.53|-1636.74| +2450838|58783|2450858|86651|1455044|3884|677|86651|1455044|3884|677|2|81|17|3|10994|84|1188|16|83.48|202.85|52.74|2401.76|843.84|1335.68|3245.60|50.63|0.00|1363.04|843.84|894.47|2206.88|2257.51|-491.84| +2450838|58783|2450848|86651|1455044|3884|677|86651|1455044|3884|677|2|85|9|2|3553|83|1188|95|17.00|43.69|17.91|2449.10|1701.45|1615.00|4150.55|85.07|0.00|871.15|1701.45|1786.52|2572.60|2657.67|86.45| +2450838|58783|2450864|86651|1455044|3884|677|86651|1455044|3884|677|2|25|1|1|3038|254|1188|13|39.03|66.74|44.04|295.10|572.52|507.39|867.62|22.90|0.00|364.39|572.52|595.42|936.91|959.81|65.13| +2450838|58783|2450919|86651|1455044|3884|677|86651|1455044|3884|677|2|64|10|1|15895|181|1188|63|71.96|176.30|68.75|6775.65|4331.25|4533.48|11106.90|389.81|0.00|0.00|4331.25|4721.06|4331.25|4721.06|-202.23| +2450838|58783|2450883|86651|1455044|3884|677|86651|1455044|3884|677|2|89|17|2|505|255|1188|76|99.62|266.98|37.37|17450.36|2840.12|7571.12|20290.48|28.40|0.00|3043.04|2840.12|2868.52|5883.16|5911.56|-4731.00| +2450838|58783|2450849|86651|1455044|3884|677|86651|1455044|3884|677|2|79|6|2|17251|294|1188|86|23.10|25.64|14.35|970.94|1234.10|1986.60|2205.04|12.34|0.00|21.50|1234.10|1246.44|1255.60|1267.94|-752.50| +2450838|58783|2450891|86651|1455044|3884|677|86651|1455044|3884|677|2|22|11|4|8212|142|1188|20|56.71|161.62|71.11|1810.20|1422.20|1134.20|3232.40|14.22|0.00|646.40|1422.20|1436.42|2068.60|2082.82|288.00| +2450838|58783|2450844|86651|1455044|3884|677|86651|1455044|3884|677|2|3|1|4|12046|185|1188|30|88.02|220.05|220.05|0.00|6601.50|2640.60|6601.50|462.10|0.00|1518.30|6601.50|7063.60|8119.80|8581.90|3960.90| +2450838|58783|2450923|86651|1455044|3884|677|86651|1455044|3884|677|2|35|15|4|1873|286|1188|44|14.96|19.29|5.40|611.16|237.60|658.24|848.76|11.88|0.00|322.52|237.60|249.48|560.12|572.00|-420.64| +2450838|58783|2450916|86651|1455044|3884|677|86651|1455044|3884|677|2|87|19|5|241|295|1188|29|6.59|8.76|6.21|73.95|180.09|191.11|254.04|1.80|0.00|12.47|180.09|181.89|192.56|194.36|-11.02| +2450838|58783|2450853|86651|1455044|3884|677|86651|1455044|3884|677|2|53|8|2|16006|246|1188|64|34.95|66.40|39.17|1742.72|2506.88|2236.80|4249.60|25.06|0.00|1869.44|2506.88|2531.94|4376.32|4401.38|270.08| +2450838|44213|2450911|10067|585882|4097|39112|10067|585882|4097|39112|4|97|16|5|2462|293|1189|70|52.36|69.63|26.45|3022.60|1851.50|3665.20|4874.10|26.84|1314.56|97.30|536.94|563.78|634.24|661.08|-3128.26| +2450838|44213|2450880|10067|585882|4097|39112|10067|585882|4097|39112|4|7|6|5|14581|253|1189|3|83.48|219.55|212.96|19.77|638.88|250.44|658.65|6.38|0.00|243.69|638.88|645.26|882.57|888.95|388.44| +2450838|44213|2450903|10067|585882|4097|39112|10067|585882|4097|39112|4|86|6|5|11608|276|1189|22|19.11|55.80|17.29|847.22|380.38|420.42|1227.60|7.60|0.00|306.90|380.38|387.98|687.28|694.88|-40.04| +2450838|44213|2450844|10067|585882|4097|39112|10067|585882|4097|39112|4|26|1|3|412|102|1189|37|92.56|273.97|104.10|6285.19|3851.70|3424.72|10136.89|231.10|0.00|101.01|3851.70|4082.80|3952.71|4183.81|426.98| +2450838|44213|2450872|10067|585882|4097|39112|10067|585882|4097|39112|4|34|20|3|3274|102|1189|8|75.01|93.01|86.49|52.16|691.92|600.08|744.08|35.70|96.86|126.48|595.06|630.76|721.54|757.24|-5.02| +2450838|44213|2450882|10067|585882|4097|39112|10067|585882|4097|39112|4|39|19|5|9037|287|1189|87|34.00|89.76|26.03|5544.51|2264.61|2958.00|7809.12|142.67|679.38|1795.68|1585.23|1727.90|3380.91|3523.58|-1372.77| +2450838|44213|2450882|10067|585882|4097|39112|10067|585882|4097|39112|4|66|5|5|12562|297|1189|25|63.44|188.41|175.22|329.75|4380.50|1586.00|4710.25|262.83|0.00|2166.50|4380.50|4643.33|6547.00|6809.83|2794.50| +2450838|44213|2450888|10067|585882|4097|39112|10067|585882|4097|39112|4|38|12|2|4064|2|1189|100|8.08|22.46|6.06|1640.00|606.00|808.00|2246.00|9.81|496.92|0.00|109.08|118.89|109.08|118.89|-698.92| +2450838|44213|2450903|10067|585882|4097|39112|10067|585882|4097|39112|4|75|10|1|3238|68|1189|98|80.77|235.84|63.67|16872.66|6239.66|7915.46|23112.32|23.08|3930.98|1617.00|2308.68|2331.76|3925.68|3948.76|-5606.78| +2450838|44213|2450914|10067|585882|4097|39112|10067|585882|4097|39112|4|44|4|4|15218|104|1189|98|99.65|267.06|112.16|15180.20|10991.68|9765.70|26171.88|549.58|0.00|6019.16|10991.68|11541.26|17010.84|17560.42|1225.98| +2450838|44213|2450870|10067|585882|4097|39112|10067|585882|4097|39112|4|16|8|3|4558|22|1189|53|74.25|147.01|136.71|545.90|7245.63|3935.25|7791.53|268.08|1883.86|3817.59|5361.77|5629.85|9179.36|9447.44|1426.52| +2450838|68451|2450858|46172|1171808|1444|37927|46172|1171808|1444|37927|4|96|6|1|10988|182|1190|69|50.66|146.40|23.42|8485.62|1615.98|3495.54|10101.60|16.15|0.00|403.65|1615.98|1632.13|2019.63|2035.78|-1879.56| +2450838|68451|2450908|46172|1171808|1444|37927|46172|1171808|1444|37927|4|9|3|1|11216|199|1190|85|84.77|228.87|226.58|194.65|19259.30|7205.45|19453.95|1232.59|3851.86|1361.70|15407.44|16640.03|16769.14|18001.73|8201.99| +2450838|68451|2450911|46172|1171808|1444|37927|46172|1171808|1444|37927|4|93|6|1|13630|198|1190|20|37.02|51.82|31.61|404.20|632.20|740.40|1036.40|18.96|0.00|10.20|632.20|651.16|642.40|661.36|-108.20| +2450838|68451|2450856|46172|1171808|1444|37927|46172|1171808|1444|37927|4|7|12|4|17539|185|1190|26|82.54|212.12|76.36|3529.76|1985.36|2146.04|5515.12|39.70|0.00|2426.58|1985.36|2025.06|4411.94|4451.64|-160.68| +2450838|68451|2450857|46172|1171808|1444|37927|46172|1171808|1444|37927|4|96|2|4|11920|158|1190|35|87.81|165.08|156.82|289.10|5488.70|3073.35|5777.80|329.32|0.00|404.25|5488.70|5818.02|5892.95|6222.27|2415.35| +2450838|68451|2450863|46172|1171808|1444|37927|46172|1171808|1444|37927|4|52|11|1|1966|261|1190|72|16.25|41.43|27.34|1014.48|1968.48|1170.00|2982.96|98.42|0.00|626.40|1968.48|2066.90|2594.88|2693.30|798.48| +2450838|68451|2450878|46172|1171808|1444|37927|46172|1171808|1444|37927|4|103|18|5|9506|117|1190|45|4.47|7.46|3.35|184.95|150.75|201.15|335.70|12.06|0.00|137.25|150.75|162.81|288.00|300.06|-50.40| +2450838|68451|2450883|46172|1171808|1444|37927|46172|1171808|1444|37927|4|7|8|1|1930|235|1190|42|46.61|85.29|67.37|752.64|2829.54|1957.62|3582.18|254.65|0.00|1217.58|2829.54|3084.19|4047.12|4301.77|871.92| +2450838|68451|2450893|46172|1171808|1444|37927|46172|1171808|1444|37927|4|85|17|2|976|229|1190|23|84.83|218.01|180.94|852.61|4161.62|1951.09|5014.23|124.84|0.00|1805.04|4161.62|4286.46|5966.66|6091.50|2210.53| +2450838|68451|2450900|46172|1171808|1444|37927|46172|1171808|1444|37927|4|95|19|2|13534|178|1190|56|4.16|4.49|3.41|60.48|190.96|232.96|251.44|11.45|0.00|19.60|190.96|202.41|210.56|222.01|-42.00| +2450838|68451|2450889|46172|1171808|1444|37927|46172|1171808|1444|37927|4|8|5|3|11971|139|1190|41|56.19|71.36|19.26|2136.10|789.66|2303.79|2925.76|7.81|702.79|789.66|86.87|94.68|876.53|884.34|-2216.92| +2450838|68451|2450903|46172|1171808|1444|37927|46172|1171808|1444|37927|4|6|20|3|4147|24|1190|46|97.35|188.85|139.74|2259.06|6428.04|4478.10|8687.10|37.28|2699.77|2432.02|3728.27|3765.55|6160.29|6197.57|-749.83| +2450838|68451|2450913|46172|1171808|1444|37927|46172|1171808|1444|37927|4|82|17|3|3517|185|1190|93|67.91|138.53|84.50|5024.79|7858.50|6315.63|12883.29|314.34|0.00|1931.61|7858.50|8172.84|9790.11|10104.45|1542.87| +2450838|68451|2450925|46172|1171808|1444|37927|46172|1171808|1444|37927|4|103|9|3|8389|116|1190|36|85.38|105.87|13.76|3315.96|495.36|3073.68|3811.32|4.95|0.00|1105.20|495.36|500.31|1600.56|1605.51|-2578.32| +2450838|42278|2450918|19432|600466|3900|48134|19432|600466|3900|48134|2|23|6|5|2342|171|1191|25|59.33|135.86|63.85|1800.25|1596.25|1483.25|3396.50|0.00|0.00|509.25|1596.25|1596.25|2105.50|2105.50|113.00| +2450838|42278|2450868|19432|600466|3900|48134|19432|600466|3900|48134|2|10|20|3|3940|26|1191|5|54.11|118.50|67.54|254.80|337.70|270.55|592.50|13.77|165.47|53.30|172.23|186.00|225.53|239.30|-98.32| +2450838|42278|2450904|19432|600466|3900|48134|19432|600466|3900|48134|2|55|1|3|15784|42|1191|48|20.70|50.92|21.38|1417.92|1026.24|993.60|2444.16|30.78|0.00|635.04|1026.24|1057.02|1661.28|1692.06|32.64| +2450838|42278|2450915|19432|600466|3900|48134|19432|600466|3900|48134|2|51|16|3|9620|197|1191|66|4.35|4.35|4.30|3.30|283.80|287.10|287.10|22.70|0.00|88.44|283.80|306.50|372.24|394.94|-3.30| +2450838|42278|2450851|19432|600466|3900|48134|19432|600466|3900|48134|2|6|19|2|7996|190|1191|91|29.35|37.27|9.69|2509.78|881.79|2670.85|3391.57|70.54|0.00|982.80|881.79|952.33|1864.59|1935.13|-1789.06| +2450838|42278|2450847|19432|600466|3900|48134|19432|600466|3900|48134|2|77|11|5|17818|270|1191|8|48.85|93.79|9.37|675.36|74.96|390.80|750.32|0.36|65.96|247.60|9.00|9.36|256.60|256.96|-381.80| +2450838|42278|2450888|19432|600466|3900|48134|19432|600466|3900|48134|2|43|18|2|12586|34|1191|48|29.27|60.00|15.00|2160.00|720.00|1404.96|2880.00|50.40|0.00|720.00|720.00|770.40|1440.00|1490.40|-684.96| +2450838|42278|2450922|19432|600466|3900|48134|19432|600466|3900|48134|2|77|17|5|1100|159|1191|73|31.42|48.07|34.12|1018.35|2490.76|2293.66|3509.11|199.26|0.00|596.41|2490.76|2690.02|3087.17|3286.43|197.10| +2450838|42278|2450919|19432|600466|3900|48134|19432|600466|3900|48134|2|53|10|5|7447|55|1191|48|35.01|102.57|48.20|2609.76|2313.60|1680.48|4923.36|56.45|902.30|2215.20|1411.30|1467.75|3626.50|3682.95|-269.18| +2450838|35934|2450916|29421|1901348|2428|755|29421|1901348|2428|755|2|55|12|4|4330|105|1192|20|21.38|23.51|18.33|103.60|366.60|427.60|470.20|0.00|0.00|84.60|366.60|366.60|451.20|451.20|-61.00| +2450838|35934|2450900|29421|1901348|2428|755|29421|1901348|2428|755|2|105|5|1|1546|26|1192|79|28.43|68.51|35.62|2598.31|2813.98|2245.97|5412.29|168.83|0.00|324.69|2813.98|2982.81|3138.67|3307.50|568.01| +2450838|35934|2450874|29421|1901348|2428|755|29421|1901348|2428|755|2|2|10|4|10285|174|1192|24|52.44|144.73|5.78|3334.80|138.72|1258.56|3473.52|0.00|0.00|1250.40|138.72|138.72|1389.12|1389.12|-1119.84| +2450838|35934|2450904|29421|1901348|2428|755|29421|1901348|2428|755|2|82|13|3|12118|256|1192|88|41.52|66.84|34.75|2823.92|3058.00|3653.76|5881.92|244.64|0.00|1352.56|3058.00|3302.64|4410.56|4655.20|-595.76| +2450838|35934|2450860|29421|1901348|2428|755|29421|1901348|2428|755|2|13|9|4|1531|62|1192|65|41.26|46.62|28.90|1151.80|1878.50|2681.90|3030.30|169.06|0.00|1302.60|1878.50|2047.56|3181.10|3350.16|-803.40| +2450838|35934|2450897|29421|1901348|2428|755|29421|1901348|2428|755|2|16|19|4|13058|112|1192|75|3.99|8.65|5.01|273.00|375.75|299.25|648.75|0.48|326.90|233.25|48.85|49.33|282.10|282.58|-250.40| +2450838|35934|2450866|29421|1901348|2428|755|29421|1901348|2428|755|2|97|6|5|8038|70|1192|51|45.37|124.31|87.01|1902.30|4437.51|2313.87|6339.81|0.00|0.00|506.94|4437.51|4437.51|4944.45|4944.45|2123.64| +2450838|15629|2450917|7836|1816693|3597|15701|7836|1816693|3597|15701|1|50|13|3|12568|34|1193|90|99.01|269.30|51.16|19632.60|4604.40|8910.90|24237.00|0.00|414.39|6786.00|4190.01|4190.01|10976.01|10976.01|-4720.89| +2450838|15629|2450847|7836|1816693|3597|15701|7836|1816693|3597|15701|1|68|17|4|16000|92|1193|27|10.79|20.06|2.60|471.42|70.20|291.33|541.62|6.31|0.00|86.40|70.20|76.51|156.60|162.91|-221.13| +2450838|15629|2450900|7836|1816693|3597|15701|7836|1816693|3597|15701|1|44|3|3|8120|105|1193|100|94.18|180.82|142.84|3798.00|14284.00|9418.00|18082.00|1142.72|0.00|3073.00|14284.00|15426.72|17357.00|18499.72|4866.00| +2450838|15629|2450928|7836|1816693|3597|15701|7836|1816693|3597|15701|1|64|16|1|4948|275|1193|31|51.90|153.62|149.01|142.91|4619.31|1608.90|4762.22|184.77|0.00|904.58|4619.31|4804.08|5523.89|5708.66|3010.41| +2450838|15629|2450914|7836|1816693|3597|15701|7836|1816693|3597|15701|1|84|8|3|1621|203|1193|30|5.44|5.71|3.48|66.90|104.40|163.20|171.30|3.13|0.00|59.70|104.40|107.53|164.10|167.23|-58.80| +2450838|15629|2450889|7836|1816693|3597|15701|7836|1816693|3597|15701|1|22|19|3|17323|96|1193|55|1.03|2.16|1.44|39.60|79.20|56.65|118.80|5.54|0.00|57.75|79.20|84.74|136.95|142.49|22.55| +2450838|15629|2450920|7836|1816693|3597|15701|7836|1816693|3597|15701|1|27|11|3|1393|295|1193|58|86.62|207.02|207.02|0.00|12007.16|5023.96|12007.16|480.28|0.00|240.12|12007.16|12487.44|12247.28|12727.56|6983.20| +2450838|15629|2450903|7836|1816693|3597|15701|7836|1816693|3597|15701|1|10|16|1|11143|35|1193|10|70.87|117.64|17.64|1000.00|176.40|708.70|1176.40|3.52|0.00|23.50|176.40|179.92|199.90|203.42|-532.30| +2450838|15629|2450884|7836|1816693|3597|15701|7836|1816693|3597|15701|1|73|18|2|4420|223|1193|52|98.68|170.71|139.98|1597.96|7278.96|5131.36|8876.92|655.10|0.00|1420.12|7278.96|7934.06|8699.08|9354.18|2147.60| +2450838|15629|2450864|7836|1816693|3597|15701|7836|1816693|3597|15701|1|88|6|1|2884|224|1193|71|71.55|107.32|55.80|3657.92|3961.80|5080.05|7619.72|145.00|1545.10|3352.62|2416.70|2561.70|5769.32|5914.32|-2663.35| +2450838|15629|2450872|7836|1816693|3597|15701|7836|1816693|3597|15701|1|59|11|4|2086|27|1193|46|5.42|14.20|9.65|209.30|443.90|249.32|653.20|8.87|0.00|274.16|443.90|452.77|718.06|726.93|194.58| +2450838|15629|2450907|7836|1816693|3597|15701|7836|1816693|3597|15701|1|5|20|5|16376|128|1193|51|69.71|147.08|39.71|5475.87|2025.21|3555.21|7501.08|20.25|0.00|1499.91|2025.21|2045.46|3525.12|3545.37|-1530.00| +2450838|50810|2450850|80413|837823|2880|25937|80413|837823|2880|25937|1|96|6|5|8138|263|1194|10|41.78|73.11|29.24|438.70|292.40|417.80|731.10|8.77|0.00|29.20|292.40|301.17|321.60|330.37|-125.40| +2450838|50810|2450921|80413|837823|2880|25937|80413|837823|2880|25937|1|29|18|2|9286|122|1194|65|94.64|234.70|192.45|2746.25|12509.25|6151.60|15255.50|500.37|0.00|5796.70|12509.25|13009.62|18305.95|18806.32|6357.65| +2450838|50810|2450879|80413|837823|2880|25937|80413|837823|2880|25937|1|87|9|3|4210|121|1194|17|47.52|55.59|1.11|926.16|18.87|807.84|945.03|0.41|11.88|349.52|6.99|7.40|356.51|356.92|-800.85| +2450838|50810|2450890|80413|837823|2880|25937|80413|837823|2880|25937|1|61|7|2|17636|32|1194|90|60.67|153.49|138.14|1381.50|12432.60|5460.30|13814.10|1118.93|0.00|4005.90|12432.60|13551.53|16438.50|17557.43|6972.30| +2450838|50810|2450911|80413|837823|2880|25937|80413|837823|2880|25937|1|24|16|3|10940|232|1194|93|16.74|17.40|0.17|1602.39|15.81|1556.82|1618.20|1.10|0.00|485.46|15.81|16.91|501.27|502.37|-1541.01| +2450838|50810|2450868|80413|837823|2880|25937|80413|837823|2880|25937|1|93|3|2|16076|244|1194|91|50.20|119.47|53.76|5979.61|4892.16|4568.20|10871.77|97.84|0.00|4022.20|4892.16|4990.00|8914.36|9012.20|323.96| +2450838|50810|2450880|80413|837823|2880|25937|80413|837823|2880|25937|1|7|7|2|2318|195|1194|33|14.04|16.42|5.74|352.44|189.42|463.32|541.86|13.25|0.00|232.98|189.42|202.67|422.40|435.65|-273.90| +2450838|50810|2450900|80413|837823|2880|25937|80413|837823|2880|25937|1|65|8|5|7708|108|1194|43|9.64|10.89|9.47|61.06|407.21|414.52|468.27|20.36|0.00|51.17|407.21|427.57|458.38|478.74|-7.31| +2450838|45137|2450841|34829|1872554|2279|20404|34829|1872554|2279|20404|1|44|18|3|17830|216|1195|89|70.56|184.86|72.09|10036.53|6416.01|6279.84|16452.54|51.97|5838.56|5922.06|577.45|629.42|6499.51|6551.48|-5702.39| +2450838|45137|2450905|34829|1872554|2279|20404|34829|1872554|2279|20404|1|35|13|3|3079|221|1195|90|75.65|77.91|39.73|3436.20|3575.70|6808.50|7011.90|178.78|0.00|771.30|3575.70|3754.48|4347.00|4525.78|-3232.80| +2450838|45137|2450845|34829|1872554|2279|20404|34829|1872554|2279|20404|1|65|20|2|2792|30|1195|28|13.64|29.05|22.94|171.08|642.32|381.92|813.40|38.53|0.00|341.60|642.32|680.85|983.92|1022.45|260.40| +2450838|45137|2450908|34829|1872554|2279|20404|34829|1872554|2279|20404|1|1|14|2|16124|96|1195|51|85.64|92.49|47.16|2311.83|2405.16|4367.64|4716.99|0.00|0.00|1414.74|2405.16|2405.16|3819.90|3819.90|-1962.48| +2450838|45137|2450899|34829|1872554|2279|20404|34829|1872554|2279|20404|1|26|15|5|10418|93|1195|20|50.56|141.06|126.95|282.20|2539.00|1011.20|2821.20|5.83|1955.03|1213.00|583.97|589.80|1796.97|1802.80|-427.23| +2450838|45137|2450855|34829|1872554|2279|20404|34829|1872554|2279|20404|1|106|9|5|1435|92|1195|2|47.64|135.77|73.31|124.92|146.62|95.28|271.54|1.46|0.00|95.02|146.62|148.08|241.64|243.10|51.34| +2450838|45137|2450847|34829|1872554|2279|20404|34829|1872554|2279|20404|1|76|4|3|2366|89|1195|21|62.25|186.12|173.09|273.63|3634.89|1307.25|3908.52|36.34|0.00|625.17|3634.89|3671.23|4260.06|4296.40|2327.64| +2450838|45137|2450876|34829|1872554|2279|20404|34829|1872554|2279|20404|1|10|19|3|4606|186|1195|28|53.12|95.61|82.22|374.92|2302.16|1487.36|2677.08|0.00|0.00|1311.52|2302.16|2302.16|3613.68|3613.68|814.80| +2450838|45137|2450860|34829|1872554|2279|20404|34829|1872554|2279|20404|1|35|15|1|841|189|1195|8|90.38|162.68|87.84|598.72|702.72|723.04|1301.44|8.36|583.25|208.16|119.47|127.83|327.63|335.99|-603.57| +2450838|45137|2450848|34829|1872554|2279|20404|34829|1872554|2279|20404|1|38|19|3|5539|238|1195|28|12.75|37.35|5.22|899.64|146.16|357.00|1045.80|11.69|0.00|209.16|146.16|157.85|355.32|367.01|-210.84| +2450838|45137|2450892|34829|1872554|2279|20404|34829|1872554|2279|20404|1|51|13|4|3025|34|1195|97|16.32|16.80|3.36|1303.68|325.92|1583.04|1629.60|6.51|0.00|211.46|325.92|332.43|537.38|543.89|-1257.12| +2450838|45137|2450840|34829|1872554|2279|20404|34829|1872554|2279|20404|1|17|4|3|1459|128|1195|36|24.81|37.21|31.62|201.24|1138.32|893.16|1339.56|0.00|0.00|214.20|1138.32|1138.32|1352.52|1352.52|245.16| +2450838|45137|2450887|34829|1872554|2279|20404|34829|1872554|2279|20404|1|20|8|2|6328|150|1195|34|4.64|8.07|5.48|88.06|186.32|157.76|274.38|16.76|0.00|21.76|186.32|203.08|208.08|224.84|28.56| +2450838|78142|2450862|75391|1908512|2880|13601|75391|1908512|2880|13601|4|85|11|4|277|180|1196|70|73.05|105.92|14.82|6377.00|1037.40|5113.50|7414.40|62.24|0.00|2891.00|1037.40|1099.64|3928.40|3990.64|-4076.10| +2450838|78142|2450883|75391|1908512|2880|13601|75391|1908512|2880|13601|4|46|14|5|6748|164|1196|56|11.02|12.23|10.02|123.76|561.12|617.12|684.88|33.66|0.00|191.52|561.12|594.78|752.64|786.30|-56.00| +2450838|78142|2450888|75391|1908512|2880|13601|75391|1908512|2880|13601|4|6|6|4|14986|201|1196|71|23.92|26.79|2.67|1712.52|189.57|1698.32|1902.09|11.37|0.00|171.11|189.57|200.94|360.68|372.05|-1508.75| +2450838|78142|2450913|75391|1908512|2880|13601|75391|1908512|2880|13601|4|59|16|2|7772|54|1196|80|59.39|95.02|82.66|988.80|6612.80|4751.20|7601.60|462.89|0.00|2356.00|6612.80|7075.69|8968.80|9431.69|1861.60| +2450838|78142|2450907|75391|1908512|2880|13601|75391|1908512|2880|13601|4|62|5|5|2284|2|1196|46|48.13|59.68|0.00|2745.28|0.00|2213.98|2745.28|0.00|0.00|356.50|0.00|0.00|356.50|356.50|-2213.98| +2450838|78142|2450896|75391|1908512|2880|13601|75391|1908512|2880|13601|4|65|13|1|6332|284|1196|54|65.92|96.90|22.28|4029.48|1203.12|3559.68|5232.60|74.11|276.71|1988.28|926.41|1000.52|2914.69|2988.80|-2633.27| +2450838|78142|2450892|75391|1908512|2880|13601|75391|1908512|2880|13601|4|7|20|2|8480|204|1196|23|5.31|10.62|10.51|2.53|241.73|122.13|244.26|16.92|0.00|63.48|241.73|258.65|305.21|322.13|119.60| +2450838|78142|2450889|75391|1908512|2880|13601|75391|1908512|2880|13601|4|35|16|5|8581|148|1196|25|3.95|6.63|3.64|74.75|91.00|98.75|165.75|1.80|30.94|46.25|60.06|61.86|106.31|108.11|-38.69| +2450838|78142|2450909|75391|1908512|2880|13601|75391|1908512|2880|13601|4|90|17|3|13496|233|1196|12|21.51|23.01|12.88|121.56|154.56|258.12|276.12|4.54|78.82|24.84|75.74|80.28|100.58|105.12|-182.38| +2450838|78142|2450928|75391|1908512|2880|13601|75391|1908512|2880|13601|4|75|6|5|16316|106|1196|65|55.57|92.80|6.49|5610.15|421.85|3612.05|6032.00|4.21|0.00|2050.75|421.85|426.06|2472.60|2476.81|-3190.20| +2450838|78142|2450914|75391|1908512|2880|13601|75391|1908512|2880|13601|4|6|8|3|5530|88|1196|24|41.49|51.03|44.39|159.36|1065.36|995.76|1224.72|53.26|0.00|110.16|1065.36|1118.62|1175.52|1228.78|69.60| +2450838|78142|2450852|75391|1908512|2880|13601|75391|1908512|2880|13601|4|61|15|2|9331|161|1196|55|51.53|130.37|125.15|287.10|6883.25|2834.15|7170.35|412.99|0.00|3513.40|6883.25|7296.24|10396.65|10809.64|4049.10| +2450838|78142|2450924|75391|1908512|2880|13601|75391|1908512|2880|13601|4|77|12|5|1045|181|1196|38|40.04|118.51|105.47|495.52|4007.86|1521.52|4503.38|160.31|0.00|2251.50|4007.86|4168.17|6259.36|6419.67|2486.34| +2450838|78142|2450926|75391|1908512|2880|13601|75391|1908512|2880|13601|4|97|3|5|2014|224|1196|54|89.66|168.56|32.02|7373.16|1729.08|4841.64|9102.24|155.61|0.00|3822.66|1729.08|1884.69|5551.74|5707.35|-3112.56| +2450838|6027|2450882|76998|1784201|204|27360|3095|1453719|122|22078|4|22|16|2|302|215|1197|52|97.92|211.50|135.36|3959.28|7038.72|5091.84|10998.00|281.54|0.00|4619.16|7038.72|7320.26|11657.88|11939.42|1946.88| +2450838|6027|2450891|76998|1784201|204|27360|3095|1453719|122|22078|4|14|2|4|7448|280|1197|9|92.56|242.50|230.37|109.17|2073.33|833.04|2182.50|103.66|0.00|523.80|2073.33|2176.99|2597.13|2700.79|1240.29| +2450838|6027|2450887|76998|1784201|204|27360|3095|1453719|122|22078|4|25|6|3|718|99|1197|48|95.51|130.84|39.25|4396.32|1884.00|4584.48|6280.32|75.36|0.00|376.80|1884.00|1959.36|2260.80|2336.16|-2700.48| +2450838|6027|2450841|76998|1784201|204|27360|3095|1453719|122|22078|4|4|2|4|6358|258|1197|71|64.49|85.12|3.40|5802.12|241.40|4578.79|6043.52|2.41|0.00|905.96|241.40|243.81|1147.36|1149.77|-4337.39| +2450838|6027|2450892|76998|1784201|204|27360|3095|1453719|122|22078|4|47|4|1|12964|196|1197|41|38.58|101.07|82.87|746.20|3397.67|1581.78|4143.87|33.97|0.00|787.20|3397.67|3431.64|4184.87|4218.84|1815.89| +2450838|6027|2450909|76998|1784201|204|27360|3095|1453719|122|22078|4|32|2|5|2254|140|1197|51|60.00|151.20|146.66|231.54|7479.66|3060.00|7711.20|448.77|0.00|1850.28|7479.66|7928.43|9329.94|9778.71|4419.66| +2450838|6027|2450900|76998|1784201|204|27360|3095|1453719|122|22078|4|80|16|3|12931|202|1197|55|33.11|78.80|0.00|4334.00|0.00|1821.05|4334.00|0.00|0.00|1603.25|0.00|0.00|1603.25|1603.25|-1821.05| +2450838|6027|2450882|76998|1784201|204|27360|3095|1453719|122|22078|4|74|19|3|5791|222|1197|53|22.37|23.93|16.27|405.98|862.31|1185.61|1268.29|51.73|0.00|253.34|862.31|914.04|1115.65|1167.38|-323.30| +2450838|6027|2450878|76998|1784201|204|27360|3095|1453719|122|22078|4|56|1|5|650|156|1197|64|30.30|86.96|57.39|1892.48|3672.96|1939.20|5565.44|293.83|0.00|2170.24|3672.96|3966.79|5843.20|6137.03|1733.76| +2450838|67517|2450888|89292|1663986|6680|10221|89292|1663986|6680|10221|1|91|14|3|11851|281|1198|52|52.53|52.53|26.79|1338.48|1393.08|2731.56|2731.56|83.58|0.00|1310.92|1393.08|1476.66|2704.00|2787.58|-1338.48| +2450838|67517|2450867|89292|1663986|6680|10221|89292|1663986|6680|10221|1|73|2|3|4052|45|1198|39|53.65|81.01|26.73|2116.92|1042.47|2092.35|3159.39|83.39|0.00|663.39|1042.47|1125.86|1705.86|1789.25|-1049.88| +2450838|67517|2450866|89292|1663986|6680|10221|89292|1663986|6680|10221|1|18|10|5|14185|177|1198|17|27.52|40.72|39.49|20.91|671.33|467.84|692.24|13.42|0.00|304.47|671.33|684.75|975.80|989.22|203.49| +2450838|67517|2450927|89292|1663986|6680|10221|89292|1663986|6680|10221|1|64|18|5|5311|166|1198|91|69.08|183.75|143.32|3679.13|13042.12|6286.28|16721.25|130.42|0.00|1337.70|13042.12|13172.54|14379.82|14510.24|6755.84| +2450838|67517|2450921|89292|1663986|6680|10221|89292|1663986|6680|10221|1|74|6|3|12001|243|1198|92|69.58|84.19|71.56|1161.96|6583.52|6401.36|7745.48|100.72|3225.92|3174.92|3357.60|3458.32|6532.52|6633.24|-3043.76| +2450838|67517|2450842|89292|1663986|6680|10221|89292|1663986|6680|10221|1|25|15|3|7346|219|1198|4|25.07|44.62|32.12|50.00|128.48|100.28|178.48|1.28|0.00|48.16|128.48|129.76|176.64|177.92|28.20| +2450838|67517|2450864|89292|1663986|6680|10221|89292|1663986|6680|10221|1|98|19|1|15224|286|1198|93|6.62|17.60|1.93|1457.31|179.49|615.66|1636.80|14.35|0.00|588.69|179.49|193.84|768.18|782.53|-436.17| +2450838|67517|2450851|89292|1663986|6680|10221|89292|1663986|6680|10221|1|60|7|5|16172|111|1198|93|91.90|118.55|15.41|9592.02|1433.13|8546.70|11025.15|114.65|0.00|1102.05|1433.13|1547.78|2535.18|2649.83|-7113.57| +2450838|67517|2450897|89292|1663986|6680|10221|89292|1663986|6680|10221|1|7|8|5|5011|264|1198|65|73.44|86.65|14.73|4674.80|957.45|4773.60|5632.25|6.89|880.85|55.90|76.60|83.49|132.50|139.39|-4697.00| +2450838|58243|2450844|58994|1589368|3389|19763|58994|1589368|3389|19763|1|98|1|1|15472|194|1199|47|67.88|192.10|132.54|2799.32|6229.38|3190.36|9028.70|0.00|0.00|1624.79|6229.38|6229.38|7854.17|7854.17|3039.02| +2450838|58243|2450859|58994|1589368|3389|19763|58994|1589368|3389|19763|1|17|7|4|11290|195|1199|26|50.14|102.78|50.36|1362.92|1309.36|1303.64|2672.28|52.37|0.00|1148.94|1309.36|1361.73|2458.30|2510.67|5.72| +2450838|58243|2450886|58994|1589368|3389|19763|58994|1589368|3389|19763|1|12|15|4|12724|212|1199|39|81.40|143.26|85.95|2235.09|3352.05|3174.60|5587.14|234.64|0.00|223.47|3352.05|3586.69|3575.52|3810.16|177.45| +2450838|58243|2450886|58994|1589368|3389|19763|58994|1589368|3389|19763|1|27|16|5|1004|3|1199|77|82.51|120.46|108.41|927.85|8347.57|6353.27|9275.42|250.42|0.00|1761.76|8347.57|8597.99|10109.33|10359.75|1994.30| +2450838|58243|2450900|58994|1589368|3389|19763|58994|1589368|3389|19763|1|14|19|4|2404|76|1199|46|89.14|136.38|107.74|1317.44|4956.04|4100.44|6273.48|446.04|0.00|2823.02|4956.04|5402.08|7779.06|8225.10|855.60| +2450838|75383|2450921|79472|223358|6444|6330|79472|223358|6444|6330|4|59|15|1|14239|283|1200|23|78.13|192.19|147.98|1016.83|3403.54|1796.99|4420.37|102.10|0.00|1944.88|3403.54|3505.64|5348.42|5450.52|1606.55| +2450838|75383|2450844|79472|223358|6444|6330|79472|223358|6444|6330|4|39|8|2|13502|141|1200|30|20.67|38.85|1.55|1119.00|46.50|620.10|1165.50|4.18|0.00|81.30|46.50|50.68|127.80|131.98|-573.60| +2450838|75383|2450925|79472|223358|6444|6330|79472|223358|6444|6330|4|29|20|5|1861|267|1200|60|1.99|3.76|1.01|165.00|60.60|119.40|225.60|1.81|0.00|78.60|60.60|62.41|139.20|141.01|-58.80| +2450838|75383|2450884|79472|223358|6444|6330|79472|223358|6444|6330|4|43|8|5|944|207|1200|81|14.94|41.83|28.86|1050.57|2337.66|1210.14|3388.23|0.00|0.00|236.52|2337.66|2337.66|2574.18|2574.18|1127.52| +2450838|28577|2450920|93460|1449206|952|24298|93460|1449206|952|24298|1|22|16|4|10574|240|1201|55|49.71|60.14|14.43|2514.05|793.65|2734.05|3307.70|47.61|0.00|1653.85|793.65|841.26|2447.50|2495.11|-1940.40| +2450838|28577|2450845|93460|1449206|952|24298|93460|1449206|952|24298|1|34|15|5|5104|173|1201|36|79.46|161.30|90.32|2555.28|3251.52|2860.56|5806.80|162.57|0.00|987.12|3251.52|3414.09|4238.64|4401.21|390.96| +2450838|28577|2450902|93460|1449206|952|24298|93460|1449206|952|24298|1|6|16|3|9727|247|1201|65|81.25|134.87|124.08|701.35|8065.20|5281.25|8766.55|725.86|0.00|3857.10|8065.20|8791.06|11922.30|12648.16|2783.95| +2450838|28577|2450890|93460|1449206|952|24298|93460|1449206|952|24298|1|80|20|3|4772|276|1201|14|6.86|7.06|2.61|62.30|36.54|96.04|98.84|1.71|2.19|0.00|34.35|36.06|34.35|36.06|-61.69| +2450838|20137|2450845|52436|857622|2008|41641|52436|857622|2008|41641|4|32|7|4|16024|136|1202|26|25.89|69.38|35.38|884.00|919.88|673.14|1803.88|5.79|340.35|306.54|579.53|585.32|886.07|891.86|-93.61| +2450838|20137|2450847|52436|857622|2008|41641|52436|857622|2008|41641|4|73|3|1|15982|296|1202|67|49.87|92.25|82.10|680.05|5500.70|3341.29|6180.75|385.04|0.00|1359.43|5500.70|5885.74|6860.13|7245.17|2159.41| +2450838|20137|2450891|52436|857622|2008|41641|52436|857622|2008|41641|4|93|9|4|16855|205|1202|80|62.89|123.89|4.95|9515.20|396.00|5031.20|9911.20|3.96|0.00|1387.20|396.00|399.96|1783.20|1787.16|-4635.20| +2450838|20137|2450842|52436|857622|2008|41641|52436|857622|2008|41641|4|92|5|2|5128|84|1202|96|15.08|41.47|13.68|2667.84|1313.28|1447.68|3981.12|7.61|551.57|437.76|761.71|769.32|1199.47|1207.08|-685.97| +2450838|20137|2450882|52436|857622|2008|41641|52436|857622|2008|41641|4|72|12|1|17764|249|1202|20|42.06|80.75|62.98|355.40|1259.60|841.20|1615.00|37.78|0.00|80.60|1259.60|1297.38|1340.20|1377.98|418.40| +2450838|20137|2450842|52436|857622|2008|41641|52436|857622|2008|41641|4|32|4|3|5263|51|1202|90|18.21|27.31|25.12|197.10|2260.80|1638.90|2457.90|45.21|0.00|958.50|2260.80|2306.01|3219.30|3264.51|621.90| +2450838|20137|2450898|52436|857622|2008|41641|52436|857622|2008|41641|4|33|14|3|3826|261|1202|48|43.76|43.76|34.13|462.24|1638.24|2100.48|2100.48|122.37|278.50|840.00|1359.74|1482.11|2199.74|2322.11|-740.74| +2450838|76335|2450847|8833|691839|1245|45451|8833|691839|1245|45451|4|42|11|3|16534|62|1203|5|6.23|10.15|5.27|24.40|26.35|31.15|50.75|0.78|16.60|11.65|9.75|10.53|21.40|22.18|-21.40| +2450838|76335|2450871|8833|691839|1245|45451|8833|691839|1245|45451|4|106|18|5|16882|189|1203|81|33.37|35.37|24.75|860.22|2004.75|2702.97|2864.97|40.09|0.00|56.70|2004.75|2044.84|2061.45|2101.54|-698.22| +2450838|76335|2450874|8833|691839|1245|45451|8833|691839|1245|45451|4|83|10|1|3040|187|1203|66|74.45|93.80|28.14|4333.56|1857.24|4913.70|6190.80|74.28|0.00|557.04|1857.24|1931.52|2414.28|2488.56|-3056.46| +2450838|76335|2450897|8833|691839|1245|45451|8833|691839|1245|45451|4|39|1|1|16153|77|1203|84|80.86|184.36|49.77|11305.56|4180.68|6792.24|15486.24|334.45|0.00|929.04|4180.68|4515.13|5109.72|5444.17|-2611.56| +2450838|76335|2450920|8833|691839|1245|45451|8833|691839|1245|45451|4|22|19|5|2464|100|1203|3|48.37|99.64|63.76|107.64|191.28|145.11|298.92|5.73|0.00|83.67|191.28|197.01|274.95|280.68|46.17| +2450838|76335|2450911|8833|691839|1245|45451|8833|691839|1245|45451|4|60|17|3|15952|165|1203|10|87.30|124.83|16.22|1086.10|162.20|873.00|1248.30|12.97|0.00|524.20|162.20|175.17|686.40|699.37|-710.80| +2450838|76335|2450897|8833|691839|1245|45451|8833|691839|1245|45451|4|23|9|5|16363|209|1203|17|95.01|136.81|46.51|1535.10|790.67|1615.17|2325.77|23.72|0.00|930.24|790.67|814.39|1720.91|1744.63|-824.50| +2450838|76335|2450881|8833|691839|1245|45451|8833|691839|1245|45451|4|57|14|1|15349|120|1203|4|76.43|105.47|11.60|375.48|46.40|305.72|421.88|3.15|1.39|12.64|45.01|48.16|57.65|60.80|-260.71| +2450838|76335|2450907|8833|691839|1245|45451|8833|691839|1245|45451|4|63|9|1|2114|283|1203|80|16.73|37.14|23.39|1100.00|1871.20|1338.40|2971.20|130.98|0.00|1009.60|1871.20|2002.18|2880.80|3011.78|532.80| +2450838|76335|2450852|8833|691839|1245|45451|8833|691839|1245|45451|4|61|2|2|11698|104|1203|41|53.94|83.60|49.32|1405.48|2022.12|2211.54|3427.60|181.99|0.00|68.47|2022.12|2204.11|2090.59|2272.58|-189.42| +2450838|76335|2450872|8833|691839|1245|45451|8833|691839|1245|45451|4|15|8|3|4862|259|1203|40|55.37|160.01|142.40|704.40|5696.00|2214.80|6400.40|170.88|0.00|512.00|5696.00|5866.88|6208.00|6378.88|3481.20| +|||8833||1245|45451|8833||1245|||60||1|14623|28|1203||||77.91|1136.77|2415.21||3551.98|48.30|0.00|674.87|2415.21|2463.51|3090.08|3138.38|495.07| +2450838|37506|2450922|93564|1734798|32|24866|93564|1734798|32|24866|2|25|1|2|8474|194|1204|5|7.00|9.94|5.16|23.90|25.80|35.00|49.70|0.25|0.00|17.35|25.80|26.05|43.15|43.40|-9.20| +2450838|37506|2450883|93564|1734798|32|24866|93564|1734798|32|24866|2|49|18|5|13514|244|1204|85|36.24|75.01|30.00|3825.85|2550.00|3080.40|6375.85|178.50|0.00|382.50|2550.00|2728.50|2932.50|3111.00|-530.40| +2450838|37506|2450899|93564|1734798|32|24866|93564|1734798|32|24866|2|19|13|1|4736|182|1204|58|30.84|87.89|14.94|4231.10|866.52|1788.72|5097.62|8.66|0.00|1325.30|866.52|875.18|2191.82|2200.48|-922.20| +2450838|37506|2450876|93564|1734798|32|24866|93564|1734798|32|24866|2|33|3|1|2560|126|1204|84|50.71|65.92|13.84|4374.72|1162.56|4259.64|5537.28|0.00|639.40|2048.76|523.16|523.16|2571.92|2571.92|-3736.48| +2450838|37506|2450874|93564|1734798|32|24866|93564|1734798|32|24866|2|104|8|1|17270|182|1204|69|46.52|86.06|55.93|2078.97|3859.17|3209.88|5938.14|0.00|0.00|1009.47|3859.17|3859.17|4868.64|4868.64|649.29| +2450838|37506|2450878|93564|1734798|32|24866|93564|1734798|32|24866|2|49|2|5|9656|151|1204|73|98.30|254.59|165.48|6505.03|12080.04|7175.90|18585.07|845.60|0.00|928.56|12080.04|12925.64|13008.60|13854.20|4904.14| +2450838|37506|2450899|93564|1734798|32|24866|93564|1734798|32|24866|2|74|16|4|6298|250|1204|77|57.49|62.08|39.11|1768.69|3011.47|4426.73|4780.16|271.03|0.00|1386.00|3011.47|3282.50|4397.47|4668.50|-1415.26| +2450838|37506|2450914|93564|1734798|32|24866|93564|1734798|32|24866|2|1|19|4|9718|235|1204|14|19.28|44.72|5.81|544.74|81.34|269.92|626.08|3.25|0.00|137.62|81.34|84.59|218.96|222.21|-188.58| +2450838|37506|2450892|93564|1734798|32|24866|93564|1734798|32|24866|2|22|1|1|11782|193|1204|48|93.21|151.93|13.67|6636.48|656.16|4474.08|7292.64|13.12|0.00|510.24|656.16|669.28|1166.40|1179.52|-3817.92| +2450838|37506|2450882|93564|1734798|32|24866|93564|1734798|32|24866|2|64|13|4|14368|46|1204|86|37.26|62.96|50.36|1083.60|4330.96|3204.36|5414.56|86.61|0.00|1353.64|4330.96|4417.57|5684.60|5771.21|1126.60| +2450838|64805|2450851|42730|750452|6527|15402|42730|750452|6527|15402|2|43|16|4|13930|140|1205|5|54.87|91.08|81.06|50.10|405.30|274.35|455.40|36.47|0.00|191.25|405.30|441.77|596.55|633.02|130.95| +2450838|64805|2450865|42730|750452|6527|15402|42730|750452|6527|15402|2|99|4|4|586|48|1205|68|46.83|117.54|105.78|799.68|7193.04|3184.44|7992.72|161.12|3164.93|3436.72|4028.11|4189.23|7464.83|7625.95|843.67| +2450838|64805|2450920|42730|750452|6527|15402|42730|750452|6527|15402|2|29|14|2|17317|197|1205|68|43.90|57.07|25.68|2134.52|1746.24|2985.20|3880.76|0.00|471.48|1785.00|1274.76|1274.76|3059.76|3059.76|-1710.44| +2450838|64805|2450847|42730|750452|6527|15402|42730|750452|6527|15402|2|19|20|5|1837|168|1205|79|11.99|13.30|11.83|116.13|934.57|947.21|1050.70|0.00|719.61|293.88|214.96|214.96|508.84|508.84|-732.25| +2450838|64805|2450893|42730|750452|6527|15402|42730|750452|6527|15402|2|37|16|5|1762|153|1205|2|80.36|195.27|41.00|308.54|82.00|160.72|390.54|2.46|0.00|132.78|82.00|84.46|214.78|217.24|-78.72| +2450838|64805|2450887|42730|750452|6527|15402|42730|750452|6527|15402|2|3|19|5|11860|262|1205|91|34.91|53.06|35.55|1593.41|3235.05|3176.81|4828.46|32.35|0.00|626.99|3235.05|3267.40|3862.04|3894.39|58.24| +2450838|64805|2450854|42730|750452|6527|15402|42730|750452|6527|15402|2|72|16|4|16462|300|1205|57|66.88|145.12|63.85|4632.39|3639.45|3812.16|8271.84|0.00|0.00|2977.68|3639.45|3639.45|6617.13|6617.13|-172.71| +2450838|64805|2450889|42730|750452|6527|15402|42730|750452|6527|15402|2|32|14|1|4148|299|1205|32|33.07|64.48|14.83|1588.80|474.56|1058.24|2063.36|0.00|0.00|268.16|474.56|474.56|742.72|742.72|-583.68| +2450838|64805|2450840|42730|750452|6527|15402|42730|750452|6527|15402|2|71|17|3|5899|235|1205|60|36.88|54.58|36.56|1081.20|2193.60|2212.80|3274.80|21.93|0.00|556.20|2193.60|2215.53|2749.80|2771.73|-19.20| +2450838|64805|2450865|42730|750452|6527|15402|42730|750452|6527|15402|2|71|8|4|14720|90|1205|81|49.23|49.72|8.94|3303.18|724.14|3987.63|4027.32|21.72|0.00|241.38|724.14|745.86|965.52|987.24|-3263.49| +2450838|64805|2450913|42730|750452|6527|15402|42730|750452|6527|15402|2|3|20|3|592|111|1205|3|56.38|127.98|122.86|15.36|368.58|169.14|383.94|0.00|0.00|142.05|368.58|368.58|510.63|510.63|199.44| +2450838|64805|2450841|42730|750452|6527|15402|42730|750452|6527|15402|2|16|14|1|15724|258|1205|56|22.43|34.99|11.89|1293.60|665.84|1256.08|1959.44|33.29|0.00|665.84|665.84|699.13|1331.68|1364.97|-590.24| +2450838|43605|2450882|48817|589534|5098|40615|48817|589534|5098|40615|2|18|11|2|3560|142|1206|97|35.40|96.28|87.61|840.99|8498.17|3433.80|9339.16|169.96|0.00|186.24|8498.17|8668.13|8684.41|8854.37|5064.37| +2450838|43605|2450871|48817|589534|5098|40615|48817|589534|5098|40615|2|56|5|5|12176|283|1206|27|65.81|115.82|83.39|875.61|2251.53|1776.87|3127.14|157.60|0.00|0.00|2251.53|2409.13|2251.53|2409.13|474.66| +2450838|43605|2450895|48817|589534|5098|40615|48817|589534|5098|40615|2|40|14|4|12511|192|1206|58|72.10|194.67|190.77|226.20|11064.66|4181.80|11290.86|221.29|0.00|4628.98|11064.66|11285.95|15693.64|15914.93|6882.86| +2450838|43605|2450845|48817|589534|5098|40615|48817|589534|5098|40615|2|89|3|3|7555|271|1206|24|9.38|13.31|5.45|188.64|130.80|225.12|319.44|10.46|0.00|150.00|130.80|141.26|280.80|291.26|-94.32| +2450838|43605|2450928|48817|589534|5098|40615|48817|589534|5098|40615|2|65|3|4|17047|78|1206|44|42.55|69.78|11.16|2579.28|491.04|1872.20|3070.32|39.28|0.00|1381.60|491.04|530.32|1872.64|1911.92|-1381.16| +2450838|43605|2450902|48817|589534|5098|40615|48817|589534|5098|40615|2|22|16|5|607|28|1206|3|36.46|67.45|28.32|117.39|84.96|109.38|202.35|6.79|0.00|32.37|84.96|91.75|117.33|124.12|-24.42| +2450838|43605|2450915|48817|589534|5098|40615|48817|589534|5098|40615|2|108|7|4|9820|260|1206|27|37.31|90.66|49.86|1101.60|1346.22|1007.37|2447.82|80.77|0.00|416.07|1346.22|1426.99|1762.29|1843.06|338.85| +2450838|43605|2450892|48817|589534|5098|40615|48817|589534|5098|40615|2|81|14|5|2461|155|1206|17|18.95|53.24|2.66|859.86|45.22|322.15|905.08|4.06|0.00|153.85|45.22|49.28|199.07|203.13|-276.93| +2450838|43605|2450911|48817|589534|5098|40615|48817|589534|5098|40615|2|34|12|5|13543|83|1206|43|84.19|144.80|47.78|4171.86|2054.54|3620.17|6226.40|3.28|1890.17|2303.51|164.37|167.65|2467.88|2471.16|-3455.80| +2450838|43605|2450928|48817|589534|5098|40615|48817|589534|5098|40615|2|26|4|1|17371|229|1206|81|93.66|98.34|79.65|1513.89|6451.65|7586.46|7965.54|516.13|0.00|2628.45|6451.65|6967.78|9080.10|9596.23|-1134.81| +2450838|66046|2450888|18426|153783|6740|4050|18426|153783|6740|4050|2|29|19|3|16946|219|1207|6|31.36|47.04|1.41|273.78|8.46|188.16|282.24|0.33|0.00|121.32|8.46|8.79|129.78|130.11|-179.70| +2450838|66046|2450882|18426|153783|6740|4050|18426|153783|6740|4050|2|11|10|3|1129|7|1207|60|93.43|245.72|68.80|10615.20|4128.00|5605.80|14743.20|247.68|0.00|2506.20|4128.00|4375.68|6634.20|6881.88|-1477.80| +2450838|66046|2450890|18426|153783|6740|4050|18426|153783|6740|4050|2|43|17|4|11239|250|1207|83|65.97|65.97|13.85|4325.96|1149.55|5475.51|5475.51|21.84|712.72|1204.33|436.83|458.67|1641.16|1663.00|-5038.68| +2450838|66046|2450883|18426|153783|6740|4050|18426|153783|6740|4050|2|36|5|1|5372|53|1207|49|69.94|80.43|57.10|1143.17|2797.90|3427.06|3941.07|251.81|0.00|354.27|2797.90|3049.71|3152.17|3403.98|-629.16| +2450838|66046|2450886|18426|153783|6740|4050|18426|153783|6740|4050|2|56|20|1|17714|116|1207|44|32.98|68.26|19.11|2162.60|840.84|1451.12|3003.44|33.63|0.00|1051.16|840.84|874.47|1892.00|1925.63|-610.28| +2450838|66046|2450924|18426|153783|6740|4050|18426|153783|6740|4050|2|42|12|3|457|91|1207|49|84.97|98.56|56.17|2077.11|2752.33|4163.53|4829.44|165.13|0.00|386.12|2752.33|2917.46|3138.45|3303.58|-1411.20| +2450838|66046|2450890|18426|153783|6740|4050|18426|153783|6740|4050|2|12|2|3|5306|262|1207|41|63.83|75.95|35.69|1650.66|1463.29|2617.03|3113.95|102.43|0.00|965.14|1463.29|1565.72|2428.43|2530.86|-1153.74| +2450838|66046|2450848|18426|153783|6740|4050|18426|153783|6740|4050|2|102|9|4|1555|276|1207|84|31.54|76.95|46.17|2585.52|3878.28|2649.36|6463.80|100.83|1357.39|645.96|2520.89|2621.72|3166.85|3267.68|-128.47| +2450838|66046|2450881|18426|153783|6740|4050|18426|153783|6740|4050|2|46|9|4|8756|206|1207|89|79.08|209.56|182.31|2425.25|16225.59|7038.12|18650.84|1067.64|973.53|5781.44|15252.06|16319.70|21033.50|22101.14|8213.94| +2450838|17409|2450906|10099|1409711|6123|49735|10099|1409711|6123|49735|1|18|8|2|7508|243|1208|11|43.11|48.28|45.86|26.62|504.46|474.21|531.08|5.04|0.00|90.20|504.46|509.50|594.66|599.70|30.25| +2450838|17409|2450887|10099|1409711|6123|49735|10099|1409711|6123|49735|1|2|8|4|5692|148|1208|68|42.50|71.82|44.52|1856.40|3027.36|2890.00|4883.76|60.54|0.00|1513.68|3027.36|3087.90|4541.04|4601.58|137.36| +2450838|17409|2450905|10099|1409711|6123|49735|10099|1409711|6123|49735|1|45|6|5|10198|140|1208|32|93.43|203.67|42.77|5148.80|1368.64|2989.76|6517.44|41.05|0.00|2346.24|1368.64|1409.69|3714.88|3755.93|-1621.12| +2450838|17409|2450859|10099|1409711|6123|49735|10099|1409711|6123|49735|1|58|8|1|1564|122|1208|17|32.76|75.67|56.75|321.64|964.75|556.92|1286.39|17.36|675.32|12.75|289.43|306.79|302.18|319.54|-267.49| +2450838|17409|2450884|10099|1409711|6123|49735|10099|1409711|6123|49735|1|81|7|3|2989|272|1208|36|49.91|95.32|89.60|205.92|3225.60|1796.76|3431.52|129.02|0.00|1612.80|3225.60|3354.62|4838.40|4967.42|1428.84| +2450838|17409|2450842|10099|1409711|6123|49735|10099|1409711|6123|49735|1|8|17|4|12718|263|1208|88|82.89|201.42|151.06|4431.68|13293.28|7294.32|17724.96|521.09|265.86|5316.96|13027.42|13548.51|18344.38|18865.47|5733.10| +2450838|17409|2450898|10099|1409711|6123|49735|10099|1409711|6123|49735|1|27|8|2|10873|108|1208|67|67.53|106.02|55.13|3409.63|3693.71|4524.51|7103.34|184.68|0.00|1562.44|3693.71|3878.39|5256.15|5440.83|-830.80| +2450838|17409|2450917|10099|1409711|6123|49735|10099|1409711|6123|49735|1|44|18|2|6188|43|1208|80|18.64|31.68|26.61|405.60|2128.80|1491.20|2534.40|149.01|0.00|1114.40|2128.80|2277.81|3243.20|3392.21|637.60| +2450838|17409|2450911|10099|1409711|6123|49735|10099|1409711|6123|49735|1|60|17|2|14560|15|1208|55|13.08|24.32|14.59|535.15|802.45|719.40|1337.60|24.07|0.00|53.35|802.45|826.52|855.80|879.87|83.05| +2450838|17409|2450846|10099|1409711|6123|49735|10099|1409711|6123|49735|1|26|19|5|4291|85|1208|68|70.02|116.23|112.74|237.32|7666.32|4761.36|7903.64|383.31|0.00|947.92|7666.32|8049.63|8614.24|8997.55|2904.96| +2450838|17409|2450844|10099|1409711|6123|49735|10099|1409711|6123|49735|1|84|5|1|8924|113|1208|44|39.82|72.47|33.33|1722.16|1466.52|1752.08|3188.68|73.32|0.00|988.24|1466.52|1539.84|2454.76|2528.08|-285.56| +|||12049|1738714||23244|12049|1738714|2179|23244|4||9|4|5704||1209|32|79.63|96.35|||30.72|2548.16|3083.20||||4.00||713.12|713.40|-2544.16| +2450838|14642|2450888|12049|1738714|2179|23244|12049|1738714|2179|23244|4|58|4|3|11908|211|1209|33|7.59|20.94|16.33|152.13|538.89|250.47|691.02|5.38|0.00|193.38|538.89|544.27|732.27|737.65|288.42| +2450838|14642|2450869|12049|1738714|2179|23244|12049|1738714|2179|23244|4|60|11|5|9469|281|1209|17|80.70|134.76|51.20|1420.52|870.40|1371.90|2290.92|11.14|591.87|778.77|278.53|289.67|1057.30|1068.44|-1093.37| +2450838|14642|2450882|12049|1738714|2179|23244|12049|1738714|2179|23244|4|22|3|4|9104|186|1209|10|42.33|109.63|92.08|175.50|920.80|423.30|1096.30|27.62|0.00|416.50|920.80|948.42|1337.30|1364.92|497.50| +2450838|14642|2450907|12049|1738714|2179|23244|12049|1738714|2179|23244|4|9|10|5|1778|153|1209|85|51.59|87.70|56.12|2684.30|4770.20|4385.15|7454.50|143.10|0.00|2608.65|4770.20|4913.30|7378.85|7521.95|385.05| +2450838|14642|2450916|12049|1738714|2179|23244|12049|1738714|2179|23244|4|42|12|2|10105|222|1209|60|72.12|80.77|8.88|4313.40|532.80|4327.20|4846.20|47.95|0.00|630.00|532.80|580.75|1162.80|1210.75|-3794.40| +2450838|14642|2450886|12049|1738714|2179|23244|12049|1738714|2179|23244|4|67|5|5|12182|221|1209|91|56.69|60.65|12.73|4360.72|1158.43|5158.79|5519.15|23.63|370.69|2427.88|787.74|811.37|3215.62|3239.25|-4371.05| +2450838|68810|2450896|34985|527286|2434|15182|34985|527286|2434|15182|1|80|12|2|373|265|1210|61|56.76|166.30|58.20|6594.10|3550.20|3462.36|10144.30|142.00|0.00|1825.73|3550.20|3692.20|5375.93|5517.93|87.84| +2450838|68810|2450922|34985|527286|2434|15182|34985|527286|2434|15182|1|100|7|3|12752|40|1210|30|32.81|95.80|31.61|1925.70|948.30|984.30|2874.00|85.34|0.00|28.50|948.30|1033.64|976.80|1062.14|-36.00| +2450838|68810|2450868|34985|527286|2434|15182|34985|527286|2434|15182|1|58|16|1|8960|191|1210|8|60.80|62.62|33.81|230.48|270.48|486.40|500.96|8.11|0.00|95.12|270.48|278.59|365.60|373.71|-215.92| +2450838|68810|2450926|34985|527286|2434|15182|34985|527286|2434|15182|1|89|3|2|5476|215|1210|47|83.70|247.75|141.21|5007.38|6636.87|3933.90|11644.25|331.84|0.00|2561.50|6636.87|6968.71|9198.37|9530.21|2702.97| +2450838|68810|2450871|34985|527286|2434|15182|34985|527286|2434|15182|1|14|8|1|116|37|1210|85|43.24|121.93|97.54|2073.15|8290.90|3675.40|10364.05|331.63|0.00|2901.90|8290.90|8622.53|11192.80|11524.43|4615.50| +2450838|68810|2450919|34985|527286|2434|15182|34985|527286|2434|15182|1|84|2|1|1015|289|1210|8|77.58|80.68|56.47|193.68|451.76|620.64|645.44|8.67|234.91|258.16|216.85|225.52|475.01|483.68|-403.79| +2450838|68810|2450863|34985|527286|2434|15182|34985|527286|2434|15182|1|44|3|4|15304|160|1210|14|79.26|110.17|85.93|339.36|1203.02|1109.64|1542.38|24.06|0.00|740.32|1203.02|1227.08|1943.34|1967.40|93.38| +2450838|68810|2450853|34985|527286|2434|15182|34985|527286|2434|15182|1|63|10|5|4114|220|1210|41|5.36|13.29|9.96|136.53|408.36|219.76|544.89|32.66|0.00|184.91|408.36|441.02|593.27|625.93|188.60| +2450838|68810|2450906|34985|527286|2434|15182|34985|527286|2434|15182|1|9|8|1|3224|117|1210|6|92.76|189.23|77.58|669.90|465.48|556.56|1135.38|23.27|0.00|68.10|465.48|488.75|533.58|556.85|-91.08| +2450838|68810|2450884|34985|527286|2434|15182|34985|527286|2434|15182|1|48|11|1|6734|14|1210|61|49.34|137.65|79.83|3527.02|4869.63|3009.74|8396.65|0.00|0.00|1510.97|4869.63|4869.63|6380.60|6380.60|1859.89| +2450838|68810|2450840|34985|527286|2434|15182|34985|527286|2434|15182|1|81|14|1|15385|52|1210|67|10.60|24.06|10.82|887.08|724.94|710.20|1612.02|57.99|0.00|144.72|724.94|782.93|869.66|927.65|14.74| +2450838|68810|2450903|34985|527286|2434|15182|34985|527286|2434|15182|1|74|6|2|10732|62|1210|12|84.59|139.57|124.21|184.32|1490.52|1015.08|1674.84|44.71|0.00|251.16|1490.52|1535.23|1741.68|1786.39|475.44| +2450838|68810|2450868|34985|527286|2434|15182|34985|527286|2434|15182|1|95|2|2|9787|197|1210|96|12.68|27.26|5.72|2067.84|549.12|1217.28|2616.96|9.06|247.10|1229.76|302.02|311.08|1531.78|1540.84|-915.26| +2450838|33222|2450922|93138|493433|2081|3576|93138|493433|2081|3576|2|38|11|2|9463|122|1211|89|50.08|143.22|81.63|5481.51|7265.07|4457.12|12746.58|653.85|0.00|1146.32|7265.07|7918.92|8411.39|9065.24|2807.95| +2450838|33222|2450901|93138|493433|2081|3576|93138|493433|2081|3576|2|68|12|1|13904|196|1211|26|76.80|177.40|72.73|2721.42|1890.98|1996.80|4612.40|113.45|0.00|783.90|1890.98|2004.43|2674.88|2788.33|-105.82| +2450838|33222|2450872|93138|493433|2081|3576|93138|493433|2081|3576|2|6|20|5|11234|78|1211|34|52.50|64.57|28.41|1229.44|965.94|1785.00|2195.38|38.63|0.00|592.62|965.94|1004.57|1558.56|1597.19|-819.06| +2450838|33222|2450895|93138|493433|2081|3576|93138|493433|2081|3576|2|3|13|5|14710|42|1211|83|93.76|245.65|68.78|14680.21|5708.74|7782.08|20388.95|228.34|0.00|4485.32|5708.74|5937.08|10194.06|10422.40|-2073.34| +2450838|33222|2450894|93138|493433|2081|3576|93138|493433|2081|3576|2|47|10|2|16636|210|1211|18|99.20|139.87|36.36|1863.18|654.48|1785.60|2517.66|52.35|0.00|679.68|654.48|706.83|1334.16|1386.51|-1131.12| +2450838|33222|2450857|93138|493433|2081|3576|93138|493433|2081|3576|2|28|2|5|541|4|1211|86|53.19|139.35|8.36|11265.14|718.96|4574.34|11984.10|3.95|639.87|2875.84|79.09|83.04|2954.93|2958.88|-4495.25| +2450838|33222|2450840|93138|493433|2081|3576|93138|493433|2081|3576|2|99|17|2|17804|56|1211|21|83.20|133.12|90.52|894.60|1900.92|1747.20|2795.52|57.02|0.00|83.79|1900.92|1957.94|1984.71|2041.73|153.72| +2450838|33222|2450854|93138|493433|2081|3576|93138|493433|2081|3576|2|77|7|5|17623|13|1211|32|31.32|72.97|38.67|1097.60|1237.44|1002.24|2335.04|74.24|0.00|420.16|1237.44|1311.68|1657.60|1731.84|235.20| +2450838|33222|2450879|93138|493433|2081|3576|93138|493433|2081|3576|2|37|1|5|7540|193|1211|34|25.81|43.36|4.76|1312.40|161.84|877.54|1474.24|12.94|0.00|29.24|161.84|174.78|191.08|204.02|-715.70| +2450838|33222|2450897|93138|493433|2081|3576|93138|493433|2081|3576|2|61|10|5|6787|27|1211|60|97.34|117.78|69.49|2897.40|4169.40|5840.40|7066.80|250.16|0.00|1130.40|4169.40|4419.56|5299.80|5549.96|-1671.00| +2450838|33222|2450853|93138|493433|2081|3576|93138|493433|2081|3576|2|12|4|5|272|98|1211|60|71.77|124.87|101.14|1423.80|6068.40|4306.20|7492.20|546.15|0.00|2921.40|6068.40|6614.55|8989.80|9535.95|1762.20| +2450838|33222|2450854|93138|493433|2081|3576|93138|493433|2081|3576|2|10|8|3|17228|146|1211|17|79.06|178.67|171.52|121.55|2915.84|1344.02|3037.39|262.42|0.00|1336.37|2915.84|3178.26|4252.21|4514.63|1571.82| +2450838|33222|2450868|93138|493433|2081|3576|93138|493433|2081|3576|2|28|18|2|14248|283|1211|15|51.44|141.46|12.73|1930.95|190.95|771.60|2121.90|17.18|0.00|594.00|190.95|208.13|784.95|802.13|-580.65| +2450838|33222|2450897|93138|493433|2081|3576|93138|493433|2081|3576|2|25|12|1|15620|179|1211|65|52.36|126.18|26.49|6479.85|1721.85|3403.40|8201.70|38.57|1170.85|3608.15|551.00|589.57|4159.15|4197.72|-2852.40| +2450838|28440|2450852|21108|1642238|5321|15251|21108|1642238|5321|15251|2|66|17|5|17449|110|1212|79|48.01|62.89|28.30|2732.61|2235.70|3792.79|4968.31|178.85|0.00|248.06|2235.70|2414.55|2483.76|2662.61|-1557.09| +2450838|28440|2450874|21108|1642238|5321|15251|21108|1642238|5321|15251|2|11|14|3|11722|121|1212|95|36.87|81.11|50.28|2928.85|4776.60|3502.65|7705.45|26.74|4442.23|924.35|334.37|361.11|1258.72|1285.46|-3168.28| +2450838|28440|2450849|21108|1642238|5321|15251|21108|1642238|5321|15251|2|90|8|4|17533|118|1212|55|40.54|84.72|56.76|1537.80|3121.80|2229.70|4659.60|24.97|624.36|792.00|2497.44|2522.41|3289.44|3314.41|267.74| +2450838|28440|2450924|21108|1642238|5321|15251|21108|1642238|5321|15251|2|14|17|5|11755|249|1212|16|70.92|82.26|68.27|223.84|1092.32|1134.72|1316.16|45.87|174.77|473.76|917.55|963.42|1391.31|1437.18|-217.17| +2450838|28440|2450851|21108|1642238|5321|15251|21108|1642238|5321|15251|2|24|13|4|3668|279|1212|43|57.73|160.48|78.63|3519.55|3381.09|2482.39|6900.64|236.67|0.00|2690.94|3381.09|3617.76|6072.03|6308.70|898.70| +2450838|67966|2450924|1833|129413|3393|46973|1833|129413|3393|46973|4|78|1|3|4936|129|1213|4|66.27|90.12|67.59|90.12|270.36|265.08|360.48|2.70|0.00|32.44|270.36|273.06|302.80|305.50|5.28| +2450838|67966|2450850|1833|129413|3393|46973|1833|129413|3393|46973|4|56|15|2|4382|148|1213|56|41.42|84.08|36.15|2684.08|2024.40|2319.52|4708.48|76.52|749.02|423.36|1275.38|1351.90|1698.74|1775.26|-1044.14| +2450838|67966|2450846|1833|129413|3393|46973|1833|129413|3393|46973|4|5|16|3|17491|175|1213|94|24.17|41.33|9.50|2992.02|893.00|2271.98|3885.02|8.93|0.00|426.76|893.00|901.93|1319.76|1328.69|-1378.98| +2450838|67966|2450883|1833|129413|3393|46973|1833|129413|3393|46973|4|68|3|5|17275|117|1213|43|14.28|32.98|5.93|1163.15|254.99|614.04|1418.14|7.64|0.00|113.09|254.99|262.63|368.08|375.72|-359.05| +2450838|31571|2450854|60959|436229|4652|39875|60959|436229|4652|39875|1|71|19|2|1573|81|1214|60|82.92|140.13|93.88|2775.00|5632.80|4975.20|8407.80|112.65|0.00|1429.20|5632.80|5745.45|7062.00|7174.65|657.60| +2450838|31571|2450909|60959|436229|4652|39875|60959|436229|4652|39875|1|107|20|1|4234|1|1214|86|25.00|25.75|13.90|1019.10|1195.40|2150.00|2214.50|23.90|0.00|198.66|1195.40|1219.30|1394.06|1417.96|-954.60| +2450838|31571|2450908|60959|436229|4652|39875|60959|436229|4652|39875|1|10|2|5|8251|147|1214|36|66.33|171.79|142.58|1051.56|5132.88|2387.88|6184.44|153.98|0.00|2473.56|5132.88|5286.86|7606.44|7760.42|2745.00| +2450838|31571|2450861|60959|436229|4652|39875|60959|436229|4652|39875|1|22|20|1|9188|215|1214|98|29.65|32.91|1.31|3096.80|128.38|2905.70|3225.18|1.28|0.00|483.14|128.38|129.66|611.52|612.80|-2777.32| +2450838|31571|2450899|60959|436229|4652|39875|60959|436229|4652|39875|1|23|16|5|4802|251|1214|24|2.26|2.28|1.64|15.36|39.36|54.24|54.72|1.29|17.71|16.32|21.65|22.94|37.97|39.26|-32.59| +2450838|31571|2450891|60959|436229|4652|39875|60959|436229|4652|39875|1|77|7|2|14098|205|1214|64|40.18|57.45|52.27|331.52|3345.28|2571.52|3676.80|0.00|1304.65|1764.48|2040.63|2040.63|3805.11|3805.11|-530.89| +2450838|31571|2450890|60959|436229|4652|39875|60959|436229|4652|39875|1|3|9|5|1369|49|1214|96|83.24|233.07|37.29|18794.88|3579.84|7991.04|22374.72|250.58|0.00|5592.96|3579.84|3830.42|9172.80|9423.38|-4411.20| +|||60959|436229|||60959|436229|||1||||6505||1214|99||73.19||||4645.08||159.39|||||6375.60|6534.99|-1457.28| +2450838|31571|2450910|60959|436229|4652|39875|60959|436229|4652|39875|1|84|3|3|2851|269|1214|52|3.78|4.64|0.60|210.08|31.20|196.56|241.28|2.49|0.00|21.32|31.20|33.69|52.52|55.01|-165.36| +2450838|31571|2450885|60959|436229|4652|39875|60959|436229|4652|39875|1|31|11|1|5630|1|1214|19|86.88|111.20|67.83|824.03|1288.77|1650.72|2112.80|51.55|0.00|760.57|1288.77|1340.32|2049.34|2100.89|-361.95| +2450838|31571|2450847|60959|436229|4652|39875|60959|436229|4652|39875|1|106|15|5|3422|148|1214|57|46.29|98.59|7.88|5170.47|449.16|2638.53|5619.63|31.44|0.00|2753.10|449.16|480.60|3202.26|3233.70|-2189.37| +2450838|28478|2450893|37048|1014828|161|7941|37048|1014828|161|7941|1|85|10|3|16070|224|1215|72|28.44|56.31|6.19|3608.64|445.68|2047.68|4054.32|40.11|0.00|242.64|445.68|485.79|688.32|728.43|-1602.00| +2450838|28478|2450915|37048|1014828|161|7941|37048|1014828|161|7941|1|52|2|3|4888|9|1215|10|77.03|168.69|50.60|1180.90|506.00|770.30|1686.90|35.42|0.00|843.40|506.00|541.42|1349.40|1384.82|-264.30| +2450838|28478|2450913|37048|1014828|161|7941|37048|1014828|161|7941|1|3|13|2|9746|209|1215|7|16.30|20.21|19.60|4.27|137.20|114.10|141.47|4.11|0.00|18.34|137.20|141.31|155.54|159.65|23.10| +2450838|28478|2450856|37048|1014828|161|7941|37048|1014828|161|7941|1|35|14|3|15646|121|1215|27|23.38|29.69|13.65|433.08|368.55|631.26|801.63|29.48|0.00|208.17|368.55|398.03|576.72|606.20|-262.71| +2450838|28478|2450887|37048|1014828|161|7941|37048|1014828|161|7941|1|69|2|3|14198|259|1215|34|49.48|137.05|106.89|1025.44|3634.26|1682.32|4659.70|145.37|0.00|1211.42|3634.26|3779.63|4845.68|4991.05|1951.94| +2450838|28478|2450912|37048|1014828|161|7941|37048|1014828|161|7941|1|94|13|4|4624|216|1215|20|63.59|169.14|43.97|2503.40|879.40|1271.80|3382.80|43.97|0.00|1420.60|879.40|923.37|2300.00|2343.97|-392.40| +2450838|28478|2450877|37048|1014828|161|7941|37048|1014828|161|7941|1|8|15|3|13609|272|1215|14|92.04|221.81|212.93|124.32|2981.02|1288.56|3105.34|45.90|685.63|310.52|2295.39|2341.29|2605.91|2651.81|1006.83| +2450838|28478|2450908|37048|1014828|161|7941|37048|1014828|161|7941|1|44|6|5|11011|193|1215|78|20.81|47.03|7.99|3045.12|623.22|1623.18|3668.34|16.57|386.39|696.54|236.83|253.40|933.37|949.94|-1386.35| +2450838|28478|2450897|37048|1014828|161|7941|37048|1014828|161|7941|1|4|7|5|13822|18|1215|52|62.87|93.67|59.94|1753.96|3116.88|3269.24|4870.84|20.57|1059.73|1022.84|2057.15|2077.72|3079.99|3100.56|-1212.09| +2450838|28478|2450882|37048|1014828|161|7941|37048|1014828|161|7941|1|40|14|5|11852|194|1215|78|83.22|122.33|47.70|5821.14|3720.60|6491.16|9541.74|148.82|0.00|953.94|3720.60|3869.42|4674.54|4823.36|-2770.56| +2450838|28478|2450840|37048|1014828|161|7941|37048|1014828|161|7941|1|60|4|1|1124|132|1215|28|37.00|73.63|26.50|1319.64|742.00|1036.00|2061.64|0.00|0.00|535.92|742.00|742.00|1277.92|1277.92|-294.00| +2450838|77912|2450882|62831|1241586|6294|35802|62831|1241586|6294|35802|4|25|1|3|11822|107|1216|85|79.17|92.62|47.23|3858.15|4014.55|6729.45|7872.70|0.00|0.00|2912.10|4014.55|4014.55|6926.65|6926.65|-2714.90| +2450838|77912|2450879|62831|1241586|6294|35802|62831|1241586|6294|35802|4|45|9|4|5224|164|1216|65|47.66|133.44|60.04|4771.00|3902.60|3097.90|8673.60|195.13|0.00|1474.20|3902.60|4097.73|5376.80|5571.93|804.70| +2450838|77912|2450863|62831|1241586|6294|35802|62831|1241586|6294|35802|4|45|5|4|17656|263|1216|22|70.29|154.63|105.14|1088.78|2313.08|1546.38|3401.86|69.39|0.00|67.98|2313.08|2382.47|2381.06|2450.45|766.70| +2450838|77912|2450842|62831|1241586|6294|35802|62831|1241586|6294|35802|4|50|19|3|3910|243|1216|42|98.05|222.57|95.70|5328.54|4019.40|4118.10|9347.94|0.00|0.00|4113.06|4019.40|4019.40|8132.46|8132.46|-98.70| +2450838|77912|2450896|62831|1241586|6294|35802|62831|1241586|6294|35802|4|91|6|2|16364|247|1216|22|35.17|69.98|69.98|0.00|1539.56|773.74|1539.56|36.94|1077.69|107.58|461.87|498.81|569.45|606.39|-311.87| +2450838|77912|2450869|62831|1241586|6294|35802|62831|1241586|6294|35802|4|46|3|4|17722|22|1216|33|26.52|28.37|1.70|880.11|56.10|875.16|936.21|0.86|43.75|0.00|12.35|13.21|12.35|13.21|-862.81| +2450838|77912|2450851|62831|1241586|6294|35802|62831|1241586|6294|35802|4|74|10|2|17647|294|1216|5|46.40|135.95|53.02|414.65|265.10|232.00|679.75|2.65|0.00|6.75|265.10|267.75|271.85|274.50|33.10| +2450838|77912|2450924|62831|1241586|6294|35802|62831|1241586|6294|35802|4|106|5|4|352|24|1216|4|4.61|13.36|1.60|47.04|6.40|18.44|53.44|0.57|0.00|9.60|6.40|6.97|16.00|16.57|-12.04| +2450838|77912|2450888|62831|1241586|6294|35802|62831|1241586|6294|35802|4|66|1|1|782|118|1216|33|87.17|195.26|115.20|2641.98|3801.60|2876.61|6443.58|152.06|0.00|901.89|3801.60|3953.66|4703.49|4855.55|924.99| +2450838|77912|2450879|62831|1241586|6294|35802|62831|1241586|6294|35802|4|91|8|2|11486|293|1216|96|31.03|82.53|59.42|2218.56|5704.32|2978.88|7922.88|228.17|0.00|3010.56|5704.32|5932.49|8714.88|8943.05|2725.44| +2450838|77912|2450864|62831|1241586|6294|35802|62831|1241586|6294|35802|4|101|1|5|7123|48|1216|86|17.90|29.35|17.31|1035.44|1488.66|1539.40|2524.10|59.54|0.00|24.94|1488.66|1548.20|1513.60|1573.14|-50.74| +2450838|69046|2450904|98409|716056|1703|31350|98409|716056|1703|31350|2|58|7|1|17224|171|1217|47|86.91|234.65|30.50|9595.05|1433.50|4084.77|11028.55|0.00|358.37|4521.40|1075.13|1075.13|5596.53|5596.53|-3009.64| +2450838|69046|2450879|98409|716056|1703|31350|98409|716056|1703|31350|2|98|16|3|3454|54|1217|71|96.17|137.52|55.00|5858.92|3905.00|6828.07|9763.92|351.45|0.00|2831.48|3905.00|4256.45|6736.48|7087.93|-2923.07| +2450838|69046|2450878|98409|716056|1703|31350|98409|716056|1703|31350|2|72|10|4|2720|284|1217|78|27.46|47.23|25.97|1658.28|2025.66|2141.88|3683.94|82.03|384.87|1804.92|1640.79|1722.82|3445.71|3527.74|-501.09| +2450838|69046|2450899|98409|716056|1703|31350|98409|716056|1703|31350|2|88|6|4|4112|298|1217|21|93.42|177.49|126.01|1081.08|2646.21|1961.82|3727.29|40.22|635.09|260.82|2011.12|2051.34|2271.94|2312.16|49.30| +2450838|69046|2450905|98409|716056|1703|31350|98409|716056|1703|31350|2|16|2|3|14186|194|1217|40|41.85|100.85|84.71|645.60|3388.40|1674.00|4034.00|33.88|0.00|1452.00|3388.40|3422.28|4840.40|4874.28|1714.40| +2450838|69046|2450914|98409|716056|1703|31350|98409|716056|1703|31350|2|102|16|2|14905|204|1217|99|46.41|80.28|51.37|2862.09|5085.63|4594.59|7947.72|355.99|0.00|3337.29|5085.63|5441.62|8422.92|8778.91|491.04| +2450838|69046|2450852|98409|716056|1703|31350|98409|716056|1703|31350|2|52|18|4|9523|184|1217|99|66.25|170.92|107.67|6261.75|10659.33|6558.75|16921.08|532.96|0.00|3214.53|10659.33|11192.29|13873.86|14406.82|4100.58| +2450838|69046|2450862|98409|716056|1703|31350|98409|716056|1703|31350|2|45|16|3|14954|179|1217|11|71.03|166.92|86.79|881.43|954.69|781.33|1836.12|85.92|0.00|18.26|954.69|1040.61|972.95|1058.87|173.36| +2450838|69046|2450897|98409|716056|1703|31350|98409|716056|1703|31350|2|93|9|1|517|238|1217|83|69.02|108.36|6.50|8454.38|539.50|5728.66|8993.88|43.16|0.00|1888.25|539.50|582.66|2427.75|2470.91|-5189.16| +2450838|27144|2450865|28016|764596|5176|26203|28016|764596|5176|26203|4|29|11|5|2653|251|1218|96|82.97|82.97|2.48|7727.04|238.08|7965.12|7965.12|4.76|0.00|3583.68|238.08|242.84|3821.76|3826.52|-7727.04| +2450838|27144|2450926|28016|764596|5176|26203|28016|764596|5176|26203|4|108|20|1|1606|36|1218|81|56.50|148.03|111.02|2997.81|8992.62|4576.50|11990.43|720.30|989.18|4915.89|8003.44|8723.74|12919.33|13639.63|3426.94| +2450838|27144|2450920|28016|764596|5176|26203|28016|764596|5176|26203|4|86|13|4|5912|76|1218|50|63.10|157.11|150.82|314.50|7541.00|3155.00|7855.50|452.46|0.00|3692.00|7541.00|7993.46|11233.00|11685.46|4386.00| +2450838|27144|2450866|28016|764596|5176|26203|28016|764596|5176|26203|4|15|14|2|11504|288|1218|78|86.95|172.16|58.53|8863.14|4565.34|6782.10|13428.48|228.26|0.00|2685.54|4565.34|4793.60|7250.88|7479.14|-2216.76| +2450838|27144|2450927|28016|764596|5176|26203|28016|764596|5176|26203|4|104|20|2|9778|50|1218|97|3.40|8.60|6.19|233.77|600.43|329.80|834.20|35.30|12.00|299.73|588.43|623.73|888.16|923.46|258.63| +2450838|27144|2450865|28016|764596|5176|26203|28016|764596|5176|26203|4|33|11|3|1291|125|1218|63|83.82|123.21|59.14|4036.41|3725.82|5280.66|7762.23|74.51|0.00|0.00|3725.82|3800.33|3725.82|3800.33|-1554.84| +2450838|27144|2450893|28016|764596|5176|26203|28016|764596|5176|26203|4|20|9|2|6722|300|1218|80|36.61|38.44|18.83|1568.80|1506.40|2928.80|3075.20|0.00|0.00|676.00|1506.40|1506.40|2182.40|2182.40|-1422.40| +2450838|27144|2450874|28016|764596|5176|26203|28016|764596|5176|26203|4|48|9|5|11029|46|1218|81|85.42|224.65|74.13|12192.12|6004.53|6919.02|18196.65|60.04|0.00|5276.34|6004.53|6064.57|11280.87|11340.91|-914.49| +2450838|27144|2450923|28016|764596|5176|26203|28016|764596|5176|26203|4|80|10|5|7381|246|1218|55|97.25|222.70|42.31|9921.45|2327.05|5348.75|12248.50|0.00|0.00|4899.40|2327.05|2327.05|7226.45|7226.45|-3021.70| +2450838|27144|2450911|28016|764596|5176|26203|28016|764596|5176|26203|4|41|3|1|14540|211|1218|33|58.52|101.23|64.78|1202.85|2137.74|1931.16|3340.59|149.64|0.00|1001.88|2137.74|2287.38|3139.62|3289.26|206.58| +2450838|74276|2450875|4962|1279766|6123|25727|4962|1279766|6123|25727|1|56|6|2|7622|36|1219|40|37.02|68.48|10.27|2328.40|410.80|1480.80|2739.20|0.00|398.47|1287.20|12.33|12.33|1299.53|1299.53|-1468.47| +2450838|74276|2450916|4962|1279766|6123|25727|4962|1279766|6123|25727|1|101|8|3|698|285|1219|64|11.77|18.36|14.13|270.72|904.32|753.28|1175.04|72.34|0.00|270.08|904.32|976.66|1174.40|1246.74|151.04| +2450838|74276|2450849|4962|1279766|6123|25727|4962|1279766|6123|25727|1|23|13|3|5881|2|1219|19|93.18|177.04|100.91|1446.47|1917.29|1770.42|3363.76|115.03|0.00|67.26|1917.29|2032.32|1984.55|2099.58|146.87| +2450838|74276|2450903|4962|1279766|6123|25727|4962|1279766|6123|25727|1|90|3|3|5761|145|1219|49|14.66|15.68|3.29|607.11|161.21|718.34|768.32|14.50|0.00|384.16|161.21|175.71|545.37|559.87|-557.13| +2450838|74276|2450852|4962|1279766|6123|25727|4962|1279766|6123|25727|1|94|11|2|16184|29|1219|88|48.46|133.74|111.00|2001.12|9768.00|4264.48|11769.12|586.08|0.00|5413.76|9768.00|10354.08|15181.76|15767.84|5503.52| +2450838|74276|2450873|4962|1279766|6123|25727|4962|1279766|6123|25727|1|50|14|4|7405|176|1219|85|15.20|40.43|35.57|413.10|3023.45|1292.00|3436.55|211.64|0.00|377.40|3023.45|3235.09|3400.85|3612.49|1731.45| +2450838|74276|2450864|4962|1279766|6123|25727|4962|1279766|6123|25727|1|84|1|3|13388|122|1219|98|12.97|17.63|4.93|1244.60|483.14|1271.06|1727.74|28.98|0.00|725.20|483.14|512.12|1208.34|1237.32|-787.92| +2450838|15974|2450923|42388|657479|4217|42108|67184|221937|7075|14166|2|52|10|2|11785|266|1220|56|11.95|13.62|5.85|435.12|327.60|669.20|762.72|16.38|0.00|121.52|327.60|343.98|449.12|465.50|-341.60| +2450838|15974|2450897|42388|657479|4217|42108|67184|221937|7075|14166|2|5|6|1|824|226|1220|7|3.37|6.26|2.19|28.49|15.33|23.59|43.82|1.22|0.00|20.58|15.33|16.55|35.91|37.13|-8.26| +2450838|15974|2450864|42388|657479|4217|42108|67184|221937|7075|14166|2|12|2|2|6646|65|1220|82|83.95|213.23|117.27|7868.72|9616.14|6883.90|17484.86|480.80|0.00|4545.26|9616.14|10096.94|14161.40|14642.20|2732.24| +2450838|15974|2450927|42388|657479|4217|42108|67184|221937|7075|14166|2|20|1|1|7282|286|1220|66|5.81|10.45|10.24|13.86|675.84|383.46|689.70|29.80|344.67|20.46|331.17|360.97|351.63|381.43|-52.29| +2450838|15974|2450844|42388|657479|4217|42108|67184|221937|7075|14166|2|61|19|2|8869|173|1220|30|90.68|254.81|114.66|4204.50|3439.80|2720.40|7644.30|33.02|3027.02|2063.70|412.78|445.80|2476.48|2509.50|-2307.62| +2450838|15974|2450842|42388|657479|4217|42108|67184|221937|7075|14166|2|58|15|5|7387|285|1220|58|34.34|95.12|45.65|2869.26|2647.70|1991.72|5516.96|2.11|2621.22|1213.36|26.48|28.59|1239.84|1241.95|-1965.24| +2450838|15974|2450851|42388|657479|4217|42108|67184|221937|7075|14166|2|17|10|1|15242|86|1220|38|25.17|40.77|34.65|232.56|1316.70|956.46|1549.26|26.33|0.00|123.88|1316.70|1343.03|1440.58|1466.91|360.24| +2450838|15974|2450907|42388|657479|4217|42108|67184|221937|7075|14166|2|87|12|2|13879|172|1220|5|34.54|102.23|4.08|490.75|20.40|172.70|511.15|1.83|0.00|235.10|20.40|22.23|255.50|257.33|-152.30| +2450838|15974|2450915|42388|657479|4217|42108|67184|221937|7075|14166|2|78|12|1|10675|261|1220|38|43.05|126.56|78.46|1827.80|2981.48|1635.90|4809.28|16.69|2146.66|817.38|834.82|851.51|1652.20|1668.89|-801.08| +2450838|15974|2450920|42388|657479|4217|42108|67184|221937|7075|14166|2|54|19|3|2095|104|1220|36|73.77|177.04|17.70|5736.24|637.20|2655.72|6373.44|44.60|0.00|1210.68|637.20|681.80|1847.88|1892.48|-2018.52| +2450838|15974|2450921|42388|657479|4217|42108|67184|221937|7075|14166|2|75|10|1|10300|253|1220|54|11.85|33.41|28.73|252.72|1551.42|639.90|1804.14|59.57|558.51|577.26|992.91|1052.48|1570.17|1629.74|353.01| +2450838|21658|2450885|28968|52629|3843|7504|28968|52629|3843|7504|4|23|19|2|4729|261|1221|22|29.89|62.76|38.91|524.70|856.02|657.58|1380.72|25.68|0.00|690.36|856.02|881.70|1546.38|1572.06|198.44| +2450838|21658|2450910|28968|52629|3843|7504|28968|52629|3843|7504|4|76|20|4|4492|114|1221|74|70.86|206.91|60.00|10871.34|4440.00|5243.64|15311.34|44.40|0.00|3062.12|4440.00|4484.40|7502.12|7546.52|-803.64| +2450838|21658|2450884|28968|52629|3843|7504|28968|52629|3843|7504|4|53|15|2|10076|153|1221|21|25.75|63.08|8.83|1139.25|185.43|540.75|1324.68|12.98|0.00|13.23|185.43|198.41|198.66|211.64|-355.32| +2450838|21658|2450850|28968|52629|3843|7504|28968|52629|3843|7504|4|64|7|2|16990|114|1221|90|78.82|114.28|61.71|4731.30|5553.90|7093.80|10285.20|388.77|0.00|0.00|5553.90|5942.67|5553.90|5942.67|-1539.90| +2450838|21658|2450891|28968|52629|3843|7504|28968|52629|3843|7504|4|69|19|4|15019|69|1221|77|57.90|165.59|9.93|11985.82|764.61|4458.30|12750.43|68.81|0.00|381.92|764.61|833.42|1146.53|1215.34|-3693.69| +2450838|21658|2450897|28968|52629|3843|7504|28968|52629|3843|7504|4|7|5|2|11978|14|1221|65|12.13|18.19|3.27|969.80|212.55|788.45|1182.35|0.00|0.00|413.40|212.55|212.55|625.95|625.95|-575.90| +2450838|21658|2450845|28968|52629|3843|7504|28968|52629|3843|7504|4|9|7|1|10364|57|1221|76|63.02|114.69|51.61|4794.08|3922.36|4789.52|8716.44|39.22|0.00|2091.52|3922.36|3961.58|6013.88|6053.10|-867.16| +2450838|21658|2450921|28968|52629|3843|7504|28968|52629|3843|7504|4|89|11|3|16297|281|1221|11|79.91|128.65|34.73|1033.12|382.03|879.01|1415.15|18.72|114.60|141.46|267.43|286.15|408.89|427.61|-611.58| +2450838|21658|2450853|28968|52629|3843|7504|28968|52629|3843|7504|4|10|7|1|5008|241|1221|80|61.81|173.06|147.10|2076.80|11768.00|4944.80|13844.80|92.96|2471.28|4568.00|9296.72|9389.68|13864.72|13957.68|4351.92| +2450838|41539|2450856|34684|1319938|6792|47999|51500|1689307|6561|10615|2|99|10|3|13753|71|1222|30|36.37|40.37|22.60|533.10|678.00|1091.10|1211.10|40.68|0.00|544.80|678.00|718.68|1222.80|1263.48|-413.10| +2450838|41539|2450919|34684|1319938|6792|47999|51500|1689307|6561|10615|2|96|4|5|17857|75|1222|52|19.32|35.74|16.44|1003.60|854.88|1004.64|1858.48|9.74|367.59|743.08|487.29|497.03|1230.37|1240.11|-517.35| +2450838|41539|2450886|34684|1319938|6792|47999|51500|1689307|6561|10615|2|95|3|3|16843|236|1222|28|92.83|233.93|212.87|589.68|5960.36|2599.24|6550.04|0.00|0.00|1113.28|5960.36|5960.36|7073.64|7073.64|3361.12| +2450838|41539|2450877|34684|1319938|6792|47999|51500|1689307|6561|10615|2|12|8|3|5504|33|1222|34|14.97|19.46|2.91|562.70|98.94|508.98|661.64|0.98|0.00|138.72|98.94|99.92|237.66|238.64|-410.04| +2450838|41539|2450925|34684|1319938|6792|47999|51500|1689307|6561|10615|2|96|14|3|17900|169|1222|99|47.70|103.98|62.38|4118.40|6175.62|4722.30|10294.02|370.53|0.00|719.73|6175.62|6546.15|6895.35|7265.88|1453.32| +2450838|41539|2450862|34684|1319938|6792|47999|51500|1689307|6561|10615|2|70|14|4|4084|222|1222|2|99.31|161.87|155.39|12.96|310.78|198.62|323.74|2.51|59.04|123.02|251.74|254.25|374.76|377.27|53.12| +2450838|23577|2450902|11774|484758|589|35220|11774|484758|589|35220|1|21|3|1|7160|265|1223|91|3.07|8.71|3.74|452.27|340.34|279.37|792.61|17.01|0.00|308.49|340.34|357.35|648.83|665.84|60.97| +2450838|23577|2450841|11774|484758|589|35220|11774|484758|589|35220|1|72|19|3|16342|154|1223|46|59.83|69.40|44.41|1149.54|2042.86|2752.18|3192.40|61.28|0.00|1021.20|2042.86|2104.14|3064.06|3125.34|-709.32| +2450838|23577|2450921|11774|484758|589|35220|11774|484758|589|35220|1|17|14|1|3361|203|1223|55|85.14|219.66|129.59|4953.85|7127.45|4682.70|12081.30|71.27|0.00|4711.30|7127.45|7198.72|11838.75|11910.02|2444.75| +2450838|23577|2450921|11774|484758|589|35220|11774|484758|589|35220|1|14|2|1|13642|222|1223|49|16.70|46.09|32.72|655.13|1603.28|818.30|2258.41|0.00|0.00|247.94|1603.28|1603.28|1851.22|1851.22|784.98| +2450838|23577|2450874|11774|484758|589|35220|11774|484758|589|35220|1|13|10|5|1288|142|1223|92|45.44|63.61|59.15|410.32|5441.80|4180.48|5852.12|108.83|0.00|1228.20|5441.80|5550.63|6670.00|6778.83|1261.32| +2450838|23577|2450919|11774|484758|589|35220|11774|484758|589|35220|1|26|20|2|11342|169|1223|29|60.48|74.39|11.15|1833.96|323.35|1753.92|2157.31|2.13|216.64|409.77|106.71|108.84|516.48|518.61|-1647.21| +2450838|23577|2450840|11774|484758|589|35220|11774|484758|589|35220|1|88|3|2|15788|43|1223|33|92.30|260.28|130.14|4294.62|4294.62|3045.90|8589.24|214.73|0.00|4208.49|4294.62|4509.35|8503.11|8717.84|1248.72| +2450838|23577|2450928|11774|484758|589|35220|11774|484758|589|35220|1|40|6|5|6607|45|1223|69|12.87|18.27|2.55|1084.68|175.95|888.03|1260.63|1.75|0.00|390.54|175.95|177.70|566.49|568.24|-712.08| +2450838|23577|2450912|11774|484758|589|35220|11774|484758|589|35220|1|58|17|3|11588|50|1223|51|85.15|150.71|116.04|1768.17|5918.04|4342.65|7686.21|3.55|5740.49|691.56|177.55|181.10|869.11|872.66|-4165.10| +2450838|23577|2450900|11774|484758|589|35220|11774|484758|589|35220|1|42|1|1|11278|116|1223|29|13.53|16.37|5.40|318.13|156.60|392.37|474.73|1.56|0.00|189.66|156.60|158.16|346.26|347.82|-235.77| +2450838|23577|2450877|11774|484758|589|35220|11774|484758|589|35220|1|70|6|1|2677|141|1223|39|76.16|77.68|21.75|2181.27|848.25|2970.24|3029.52|76.34|0.00|969.15|848.25|924.59|1817.40|1893.74|-2121.99| +2450838|23577|2450872|11774|484758|589|35220|11774|484758|589|35220|1|16|10|5|14524|142|1223|24|14.58|15.74|8.34|177.60|200.16|349.92|377.76|2.00|0.00|143.52|200.16|202.16|343.68|345.68|-149.76| +2450838|42722|2450907|33246|513084|2969|6007|33246|513084|2969|6007|2|91|12|1|17665|109|1224|22|64.49|139.29|87.75|1133.88|1930.50|1418.78|3064.38|154.44|0.00|827.20|1930.50|2084.94|2757.70|2912.14|511.72| +2450838|42722|2450919|33246|513084|2969|6007|33246|513084|2969|6007|2|2|11|1|11474|224|1224|99|28.21|78.70|25.18|5298.48|2492.82|2792.79|7791.30|49.85|0.00|2337.39|2492.82|2542.67|4830.21|4880.06|-299.97| +2450838|42722|2450924|33246|513084|2969|6007|33246|513084|2969|6007|2|97|5|1|2581|193|1224|31|15.04|22.56|5.64|524.52|174.84|466.24|699.36|6.99|0.00|321.47|174.84|181.83|496.31|503.30|-291.40| +2450838|42722|2450870|33246|513084|2969|6007|33246|513084|2969|6007|2|97|16|1|3763|231|1224|90|21.51|27.96|25.72|201.60|2314.80|1935.90|2516.40|69.44|0.00|553.50|2314.80|2384.24|2868.30|2937.74|378.90| +2450838|65376|2450877|58430|963774|7165|25356|15095|586938|2329|42473|4|9|18|2|8348|250|1225|95|8.39|21.89|20.79|104.50|1975.05|797.05|2079.55|59.25|0.00|269.80|1975.05|2034.30|2244.85|2304.10|1178.00| +2450838|65376|2450903|58430|963774|7165|25356|15095|586938|2329|42473|4|57|1|4|4670|142|1225|68|28.18|43.67|41.04|178.84|2790.72|1916.24|2969.56|27.90|0.00|1454.52|2790.72|2818.62|4245.24|4273.14|874.48| +2450838|65376|2450914|58430|963774|7165|25356|15095|586938|2329|42473|4|56|18|1|5533|84|1225|49|94.19|270.32|243.28|1324.96|11920.72|4615.31|13245.68|715.24|0.00|5562.97|11920.72|12635.96|17483.69|18198.93|7305.41| +2450838|65376|2450899|58430|963774|7165|25356|15095|586938|2329|42473|4|104|13|2|7219|187|1225|31|91.70|264.09|256.16|245.83|7940.96|2842.70|8186.79|79.40|0.00|1964.78|7940.96|8020.36|9905.74|9985.14|5098.26| +2450838|65376|2450882|58430|963774|7165|25356|15095|586938|2329|42473|4|92|10|3|14323|226|1225|4|20.63|27.02|13.23|55.16|52.92|82.52|108.08|0.19|49.74|12.96|3.18|3.37|16.14|16.33|-79.34| +2450838|65376|2450876|58430|963774|7165|25356|15095|586938|2329|42473|4|108|3|5|11332|87|1225|89|30.08|68.58|30.17|3418.49|2685.13|2677.12|6103.62|0.00|0.00|609.65|2685.13|2685.13|3294.78|3294.78|8.01| +2450838|65376|2450925|58430|963774|7165|25356|15095|586938|2329|42473|4|85|6|2|13546|74|1225|3|32.74|44.19|2.65|124.62|7.95|98.22|132.57|0.10|4.53|5.28|3.42|3.52|8.70|8.80|-94.80| +2450838|65376|2450905|58430|963774|7165|25356|15095|586938|2329|42473|4|99|20|3|2077|226|1225|19|76.53|76.53|26.78|945.25|508.82|1454.07|1454.07|20.35|0.00|14.44|508.82|529.17|523.26|543.61|-945.25| +2450838|65376|2450870|58430|963774|7165|25356|15095|586938|2329|42473|4|64|1|3|12139|94|1225|36|1.30|3.54|2.23|47.16|80.28|46.80|127.44|0.91|49.77|43.20|30.51|31.42|73.71|74.62|-16.29| +2450838|65376|2450904|58430|963774|7165|25356|15095|586938|2329|42473|4|5|14|2|10960|194|1225|40|49.91|59.89|17.96|1677.20|718.40|1996.40|2395.60|28.73|0.00|766.40|718.40|747.13|1484.80|1513.53|-1278.00| +2450839|7700|2450843|10719|718228|4490|48752|10719|718228|4490|48752|4|76|3|1|11078|60|1226|6|42.13|86.78|48.59|229.14|291.54|252.78|520.68|23.32|0.00|5.16|291.54|314.86|296.70|320.02|38.76| +2450839|7700|2450928|10719|718228|4490|48752|10719|718228|4490|48752|4|20|12|1|8420|194|1226|23|29.63|81.18|48.70|747.04|1120.10|681.49|1867.14|89.60|0.00|615.94|1120.10|1209.70|1736.04|1825.64|438.61| +2450839|7700|2450913|10719|718228|4490|48752|10719|718228|4490|48752|4|81|19|3|13015|33|1226|61|62.90|89.31|61.62|1689.09|3758.82|3836.90|5447.91|112.76|0.00|708.21|3758.82|3871.58|4467.03|4579.79|-78.08| +2450839|7700|2450925|10719|718228|4490|48752|10719|718228|4490|48752|4|64|15|2|15256|113|1226|75|89.50|239.86|16.79|16730.25|1259.25|6712.50|17989.50|75.55|0.00|6835.50|1259.25|1334.80|8094.75|8170.30|-5453.25| +2450839|7700|2450886|10719|718228|4490|48752|10719|718228|4490|48752|4|14|1|3|12566|154|1226|68|73.75|205.76|146.08|4058.24|9933.44|5015.00|13991.68|125.16|5761.39|3777.40|4172.05|4297.21|7949.45|8074.61|-842.95| +2450839|7700|2450866|10719|718228|4490|48752|10719|718228|4490|48752|4|92|16|1|2132|128|1226|28|23.76|40.86|40.45|11.48|1132.60|665.28|1144.08|56.63|0.00|80.08|1132.60|1189.23|1212.68|1269.31|467.32| +2450839|7700|2450885|10719|718228|4490|48752|10719|718228|4490|48752|4|105|14|5|17726|11|1226|84|3.74|5.38|4.68|58.80|393.12|314.16|451.92|3.93|0.00|184.80|393.12|397.05|577.92|581.85|78.96| +2450839|7700|2450921|10719|718228|4490|48752|10719|718228|4490|48752|4|33|15|2|7117|219|1226|25|52.69|61.12|20.16|1024.00|504.00|1317.25|1528.00|20.16|0.00|152.75|504.00|524.16|656.75|676.91|-813.25| +2450839|7700|2450907|10719|718228|4490|48752|10719|718228|4490|48752|4|20|13|2|10898|272|1226|2|75.49|187.21|119.81|134.80|239.62|150.98|374.42|4.79|0.00|116.06|239.62|244.41|355.68|360.47|88.64| +2450839|7700|2450889|10719|718228|4490|48752|10719|718228|4490|48752|4|65|8|4|1117|66|1226|71|76.59|172.32|24.12|10522.20|1712.52|5437.89|12234.72|102.75|0.00|5872.41|1712.52|1815.27|7584.93|7687.68|-3725.37| +2450839|7700|2450918|10719|718228|4490|48752|10719|718228|4490|48752|4|32|2|5|17209|101|1226|91|99.37|255.38|40.86|19521.32|3718.26|9042.67|23239.58|111.54|0.00|6506.50|3718.26|3829.80|10224.76|10336.30|-5324.41| +2450839|54490|2450914|70868|1431975|5529|19036|70868|1431975|5529|19036|2|70|5|2|13906|93|1227|1|29.14|44.00|34.32|9.68|34.32|29.14|44.00|0.00|24.02|14.52|10.30|10.30|24.82|24.82|-18.84| +2450839|54490|2450888|70868|1431975|5529|19036|70868|1431975|5529|19036|2|80|5|1|10747|193|1227|42|47.92|127.94|29.42|4137.84|1235.64|2012.64|5373.48|0.00|0.00|1396.92|1235.64|1235.64|2632.56|2632.56|-777.00| +2450839|54490|2450904|70868|1431975|5529|19036|70868|1431975|5529|19036|2|56|5|3|12686|37|1227|33|89.04|123.76|99.00|817.08|3267.00|2938.32|4084.08|261.36|0.00|1919.28|3267.00|3528.36|5186.28|5447.64|328.68| +2450839|54490|2450879|70868|1431975|5529|19036|70868|1431975|5529|19036|2|104|9|5|10340|51|1227|18|80.36|115.71|111.08|83.34|1999.44|1446.48|2082.78|91.97|159.95|708.12|1839.49|1931.46|2547.61|2639.58|393.01| +2450839|54490|2450890|70868|1431975|5529|19036|70868|1431975|5529|19036|2|96|11|5|17410|26|1227|93|71.42|89.27|79.45|913.26|7388.85|6642.06|8302.11|443.33|0.00|2490.54|7388.85|7832.18|9879.39|10322.72|746.79| +2450839|54490|2450856|70868|1431975|5529|19036|70868|1431975|5529|19036|2|10|9|4|7135|16|1227|100|55.96|155.56|37.33|11823.00|3733.00|5596.00|15556.00|186.65|0.00|7311.00|3733.00|3919.65|11044.00|11230.65|-1863.00| +2450839|54490|2450866|70868|1431975|5529|19036|70868|1431975|5529|19036|2|96|5|5|9206|80|1227|82|44.45|123.12|98.49|2019.66|8076.18|3644.90|10095.84|242.28|0.00|1614.58|8076.18|8318.46|9690.76|9933.04|4431.28| +2450839|54490|2450865|70868|1431975|5529|19036|70868|1431975|5529|19036|2|99|20|1|11192|160|1227|29|21.35|56.15|16.28|1156.23|472.12|619.15|1628.35|0.00|472.12|341.91|0.00|0.00|341.91|341.91|-619.15| +2450839|54490|2450913|70868|1431975|5529|19036|70868|1431975|5529|19036|2|25|2|4|10681|183|1227|85|33.18|40.14|28.09|1024.25|2387.65|2820.30|3411.90|47.75|0.00|1637.10|2387.65|2435.40|4024.75|4072.50|-432.65| +2450839|54490|2450873|70868|1431975|5529|19036|70868|1431975|5529|19036|2|30|14|2|16784|106|1227|20|58.92|119.60|98.07|430.60|1961.40|1178.40|2392.00|176.52|0.00|502.20|1961.40|2137.92|2463.60|2640.12|783.00| +2450839|54490|2450857|70868|1431975|5529|19036|70868|1431975|5529|19036|2|39|14|4|14383|167|1227|53|9.71|26.70|9.34|920.08|495.02|514.63|1415.10|34.65|0.00|56.18|495.02|529.67|551.20|585.85|-19.61| +2450839|54490|2450905|70868|1431975|5529|19036|70868|1431975|5529|19036|2|29|10|4|4796|237|1227|75|66.37|100.21|41.08|4434.75|3081.00|4977.75|7515.75|277.29|0.00|901.50|3081.00|3358.29|3982.50|4259.79|-1896.75| +2450839|54490|2450893|70868|1431975|5529|19036|70868|1431975|5529|19036|2|77|6|5|13051|276|1227|41|72.80|149.96|122.96|1107.00|5041.36|2984.80|6148.36|43.85|655.37|860.59|4385.99|4429.84|5246.58|5290.43|1401.19| +2450839|7000|2450879|23437|356393|1349|41105|23437|356393|1349|41105|4|75|10|4|4156|146|1228|68|49.75|140.29|16.83|8395.28|1144.44|3383.00|9539.72|11.44|0.00|95.20|1144.44|1155.88|1239.64|1251.08|-2238.56| +2450839|7000|2450845|23437|356393|1349|41105|23437|356393|1349|41105|4|90|13|3|2344|104|1228|9|64.71|95.12|10.46|761.94|94.14|582.39|856.08|0.94|0.00|0.00|94.14|95.08|94.14|95.08|-488.25| +2450839||||356393||41105|23437|356393|||||14|2|5212||1228|81|71.04|||11359.44||5754.24|||0.00|2271.24|4867.29|5061.98|7138.53|7333.22|-886.95| +2450839|7000|2450897|23437|356393|1349|41105|23437|356393|1349|41105|4|88|19|2|5198|88|1228|64|34.11|84.93|13.58|4566.40|869.12|2183.04|5435.52|78.22|0.00|488.96|869.12|947.34|1358.08|1436.30|-1313.92| +2450839|7000|2450884|23437|356393|1349|41105|23437|356393|1349|41105|4|16|18|1|10694|107|1228|52|28.07|55.85|29.60|1365.00|1539.20|1459.64|2904.20|46.17|0.00|493.48|1539.20|1585.37|2032.68|2078.85|79.56| +2450839|7000|2450844|23437|356393|1349|41105|23437|356393|1349|41105|4|95|1|4|4369|166|1228|18|23.06|61.80|61.80|0.00|1112.40|415.08|1112.40|22.24|0.00|222.48|1112.40|1134.64|1334.88|1357.12|697.32| +2450839|7000|2450917|23437|356393|1349|41105|23437|356393|1349|41105|4|17|1|3|16996|185|1228|19|3.25|9.13|6.20|55.67|117.80|61.75|173.47|5.89|0.00|67.64|117.80|123.69|185.44|191.33|56.05| +2450839|7000|2450871|23437|356393|1349|41105|23437|356393|1349|41105|4|48|12|4|260|153|1228|62|35.38|55.54|39.98|964.72|2478.76|2193.56|3443.48|198.30|0.00|1549.38|2478.76|2677.06|4028.14|4226.44|285.20| +2450839|7000|2450925|23437|356393|1349|41105|23437|356393|1349|41105|4|87|5|3|1141|79|1228|59|76.56|149.29|70.16|4668.67|4139.44|4517.04|8808.11|206.97|0.00|3787.21|4139.44|4346.41|7926.65|8133.62|-377.60| +2450839|7000|2450920|23437|356393|1349|41105|23437|356393|1349|41105|4|40|14|5|1120|198|1228|45|77.27|105.08|24.16|3641.40|1087.20|3477.15|4728.60|97.84|0.00|141.75|1087.20|1185.04|1228.95|1326.79|-2389.95| +2450839|7000|2450884|23437|356393|1349|41105|23437|356393|1349|41105|4|5|11|2|6232|100|1228|10|71.78|111.97|19.03|929.40|190.30|717.80|1119.70|0.72|154.14|156.70|36.16|36.88|192.86|193.58|-681.64| +2450839|7000|2450915|23437|356393|1349|41105|23437|356393|1349|41105|4|99|20|1|1216|299|1228|53|11.22|13.35|1.86|608.97|98.58|594.66|707.55|6.90|0.00|141.51|98.58|105.48|240.09|246.99|-496.08| +2450839|7000|2450899|23437|356393|1349|41105|23437|356393|1349|41105|4|39|7|1|11023|147|1228|42|57.51|92.59|61.10|1322.58|2566.20|2415.42|3888.78|25.66|0.00|1166.34|2566.20|2591.86|3732.54|3758.20|150.78| +2450839|47452|2450868|58283|1867374|4112|47250|70169|1042326|1464|43755|1|73|4|4|5587|136|1229|87|42.95|109.09|105.81|285.36|9205.47|3736.65|9490.83|552.32|0.00|1233.66|9205.47|9757.79|10439.13|10991.45|5468.82| +2450839|47452|2450899|58283|1867374|4112|47250|70169|1042326|1464|43755|1|72|10|2|9532|218|1229|4|72.23|126.40|25.28|404.48|101.12|288.92|505.60|4.04|0.00|166.84|101.12|105.16|267.96|272.00|-187.80| +2450839|47452|2450915|58283|1867374|4112|47250|70169|1042326|1464|43755|1|30|6|2|5228|72|1229|58|61.25|124.95|92.46|1884.42|5362.68|3552.50|7247.10|429.01|0.00|1593.84|5362.68|5791.69|6956.52|7385.53|1810.18| +2450839|47452|2450841|58283|1867374|4112|47250|70169|1042326|1464|43755|1|83|12|4|14704|210|1229|69|20.88|35.70|9.28|1822.98|640.32|1440.72|2463.30|25.61|0.00|1108.14|640.32|665.93|1748.46|1774.07|-800.40| +2450839|47452|2450913|58283|1867374|4112|47250|70169|1042326|1464|43755|1|36|1|4|10357|138|1229|99|97.10|249.54|112.29|13587.75|11116.71|9612.90|24704.46|222.33|0.00|6423.12|11116.71|11339.04|17539.83|17762.16|1503.81| +2450839|47452|2450903|58283|1867374|4112|47250|70169|1042326|1464|43755|1|49|19|3|1304|3|1229|59|76.07|167.35|58.57|6418.02|3455.63|4488.13|9873.65|172.78|0.00|4640.35|3455.63|3628.41|8095.98|8268.76|-1032.50| +2450839|47452|2450892|58283|1867374|4112|47250|70169|1042326|1464|43755|1|43|7|1|16003|243|1229|99|72.61|124.88|103.65|2101.77|10261.35|7188.39|12363.12|205.22|0.00|3337.29|10261.35|10466.57|13598.64|13803.86|3072.96| +2450839|47452|2450897|58283|1867374|4112|47250|70169|1042326|1464|43755|1|94|20|5|15190|78|1229|13|99.67|252.16|226.94|327.86|2950.22|1295.71|3278.08|236.01|0.00|360.49|2950.22|3186.23|3310.71|3546.72|1654.51| +2450839|40165|2450868|19161|921780|2799|34665|19161|921780|2799|34665|4|86|20|2|16324|207|1230|19|66.29|162.41|142.92|370.31|2715.48|1259.51|3085.79|108.61|0.00|678.87|2715.48|2824.09|3394.35|3502.96|1455.97| +2450839|40165|2450871|19161|921780|2799|34665|19161|921780|2799|34665|4|23|15|4|8143|249|1230|69|72.85|136.22|27.24|7519.62|1879.56|5026.65|9399.18|69.16|1014.96|4417.38|864.60|933.76|5281.98|5351.14|-4162.05| +2450839|40165|2450889|19161|921780|2799|34665|19161|921780|2799|34665|4|45|19|3|2186|81|1230|87|53.65|156.65|75.19|7087.02|6541.53|4667.55|13628.55|26.16|6214.45|3406.92|327.08|353.24|3734.00|3760.16|-4340.47| +2450839|40165|2450929|19161|921780|2799|34665|19161|921780|2799|34665|4|96|13|1|8780|79|1230|19|88.82|136.78|53.34|1585.36|1013.46|1687.58|2598.82|60.80|0.00|51.87|1013.46|1074.26|1065.33|1126.13|-674.12| +2450839|40165|2450852|19161|921780|2799|34665|19161|921780|2799|34665|4|33|15|5|15181|173|1230|76|21.24|21.45|12.22|701.48|928.72|1614.24|1630.20|0.00|260.04|326.04|668.68|668.68|994.72|994.72|-945.56| +2450839|68347|2450910|14196|1409023|6603|4630|14196|1409023|6603|4630|1|84|10|3|2005|27|1231|24|75.20|188.00|50.76|3293.76|1218.24|1804.80|4512.00|109.64|0.00|496.32|1218.24|1327.88|1714.56|1824.20|-586.56| +2450839|68347|2450843|14196|1409023|6603|4630|14196|1409023|6603|4630|1|49|5|1|8506|103|1231|75|1.69|4.91|3.68|92.25|276.00|126.75|368.25|19.32|0.00|147.00|276.00|295.32|423.00|442.32|149.25| +2450839|68347|2450903|14196|1409023|6603|4630|14196|1409023|6603|4630|1|75|10|3|14392|41|1231|7|41.10|53.84|16.69|260.05|116.83|287.70|376.88|10.51|0.00|154.49|116.83|127.34|271.32|281.83|-170.87| +2450839|68347|2450885|14196|1409023|6603|4630|14196|1409023|6603|4630|1|64|16|5|14330|68|1231|91|97.65|172.84|82.96|8179.08|7549.36|8886.15|15728.44|150.98|0.00|1415.05|7549.36|7700.34|8964.41|9115.39|-1336.79| +2450839|68347|2450887|14196|1409023|6603|4630|14196|1409023|6603|4630|1|60|4|4|2260|241|1231|55|73.32|200.16|64.05|7486.05|3522.75|4032.60|11008.80|211.36|0.00|4843.85|3522.75|3734.11|8366.60|8577.96|-509.85| +2450839|68347|2450863|14196|1409023|6603|4630|14196|1409023|6603|4630|1|31|8|3|3583|45|1231|73|96.96|240.46|57.71|13340.75|4212.83|7078.08|17553.58|379.15|0.00|5441.42|4212.83|4591.98|9654.25|10033.40|-2865.25| +2450839|68347|2450844|14196|1409023|6603|4630|14196|1409023|6603|4630|1|30|6|4|1232|197|1231|72|16.25|39.97|7.59|2331.36|546.48|1170.00|2877.84|32.78|0.00|949.68|546.48|579.26|1496.16|1528.94|-623.52| +2450839|68347|2450867|14196|1409023|6603|4630|14196|1409023|6603|4630|1|18|4|1|1582|42|1231|91|97.00|160.05|144.04|1456.91|13107.64|8827.00|14564.55|655.38|0.00|6990.62|13107.64|13763.02|20098.26|20753.64|4280.64| +2450839|68347|2450898|14196|1409023|6603|4630|14196|1409023|6603|4630|1|57|13|1|1279|50|1231|76|51.20|105.98|47.69|4430.04|3624.44|3891.20|8054.48|326.19|0.00|3463.32|3624.44|3950.63|7087.76|7413.95|-266.76| +2450839|68347|2450857|14196|1409023|6603|4630|14196|1409023|6603|4630|1|52|7|4|8425|231|1231|66|56.77|151.00|55.87|6278.58|3687.42|3746.82|9966.00|184.37|0.00|4684.02|3687.42|3871.79|8371.44|8555.81|-59.40| +2450839|68347|2450893|14196|1409023|6603|4630|14196|1409023|6603|4630|1|49|15|4|9410|139|1231|39|65.55|156.00|87.36|2676.96|3407.04|2556.45|6084.00|102.21|0.00|121.68|3407.04|3509.25|3528.72|3630.93|850.59| +2450839|25049|2450900|76016|147026|2836|26246|76016|147026|2836|26246|4|106|19|3|8632|201|1232|9|88.11|135.68|111.25|219.87|1001.25|792.99|1221.12|9.41|530.66|219.78|470.59|480.00|690.37|699.78|-322.40| +2450839|25049|2450864|76016|147026|2836|26246|76016|147026|2836|26246|4|57|20|5|2044|231|1232|55|41.39|56.29|1.68|3003.55|92.40|2276.45|3095.95|4.62|0.00|30.80|92.40|97.02|123.20|127.82|-2184.05| +2450839|25049|2450880|76016|147026|2836|26246|76016|147026|2836|26246|4|18|18|4|2398|181|1232|6|18.88|51.73|46.03|34.20|276.18|113.28|310.38|22.09|0.00|55.86|276.18|298.27|332.04|354.13|162.90| +2450839|25049|2450901|76016|147026|2836|26246|76016|147026|2836|26246|4|24|7|3|12016|30|1232|4|50.28|56.81|32.94|95.48|131.76|201.12|227.24|3.39|75.10|63.60|56.66|60.05|120.26|123.65|-144.46| +2450839|25049|2450925|76016|147026|2836|26246|76016|147026|2836|26246|4|21|18|5|1906|119|1232|6|85.03|167.50|13.40|924.60|80.40|510.18|1005.00|4.82|0.00|80.40|80.40|85.22|160.80|165.62|-429.78| +2450839|25049|2450909|76016|147026|2836|26246|76016|147026|2836|26246|4|94|10|4|13951|24|1232|5|73.79|152.74|29.02|618.60|145.10|368.95|763.70|7.25|0.00|351.30|145.10|152.35|496.40|503.65|-223.85| +2450839|25049|2450852|76016|147026|2836|26246|76016|147026|2836|26246|4|83|17|4|4088|298|1232|69|42.49|85.40|29.03|3889.53|2003.07|2931.81|5892.60|0.00|2003.07|176.64|0.00|0.00|176.64|176.64|-2931.81| +2450839|25049|2450910|76016|147026|2836|26246|76016|147026|2836|26246|4|84|1|4|17374|155|1232|12|90.75|168.79|77.64|1093.80|931.68|1089.00|2025.48|27.95|0.00|465.84|931.68|959.63|1397.52|1425.47|-157.32| +2450839|25049|2450858|76016|147026|2836|26246|76016|147026|2836|26246|4|27|5|1|5644|278|1232|75|6.28|17.64|9.70|595.50|727.50|471.00|1323.00|0.00|0.00|555.00|727.50|727.50|1282.50|1282.50|256.50| +2450839|25049|2450849|76016|147026|2836|26246|76016|147026|2836|26246|4|102|19|5|15703|217|1232|23|95.26|110.50|65.19|1042.13|1499.37|2190.98|2541.50|0.00|0.00|1194.39|1499.37|1499.37|2693.76|2693.76|-691.61| +2450839|25049|2450863|76016|147026|2836|26246|76016|147026|2836|26246|4|50|7|2|188|284|1232|51|7.01|12.26|2.94|475.32|149.94|357.51|625.26|8.99|0.00|199.92|149.94|158.93|349.86|358.85|-207.57| +2450839|70492|2450907|64266|1799275|4684|8046|64266|1799275|4684|8046|2|43|17|1|13690|83|1233|51|58.27|118.87|41.60|3940.77|2121.60|2971.77|6062.37|84.86|0.00|1151.58|2121.60|2206.46|3273.18|3358.04|-850.17| +2450839|70492|2450892|64266|1799275|4684|8046|64266|1799275|4684|8046|2|77|4|5|17060|46|1233|30|93.36|275.41|148.72|3800.70|4461.60|2800.80|8262.30|312.31|0.00|3800.40|4461.60|4773.91|8262.00|8574.31|1660.80| +2450839|70492|2450894|64266|1799275|4684|8046|64266|1799275|4684|8046|2|28|1|4|2984|138|1233|11|51.29|153.35|119.61|371.14|1315.71|564.19|1686.85|41.83|618.38|708.40|697.33|739.16|1405.73|1447.56|133.14| +2450839|70492|2450863|64266|1799275|4684|8046|64266|1799275|4684|8046|2|21|15|4|7054|233|1233|81|96.02|183.39|154.04|2377.35|12477.24|7777.62|14854.59|0.00|0.00|5198.58|12477.24|12477.24|17675.82|17675.82|4699.62| +2450839|70492|2450929|64266|1799275|4684|8046|64266|1799275|4684|8046|2|31|19|2|9626|170|1233|85|67.48|167.35|153.96|1138.15|13086.60|5735.80|14224.75|654.33|0.00|710.60|13086.60|13740.93|13797.20|14451.53|7350.80| +2450839|70492|2450919|64266|1799275|4684|8046|64266|1799275|4684|8046|2|41|12|2|9908|237|1233|68|94.82|164.03|54.12|7473.88|3680.16|6447.76|11154.04|35.69|110.40|3457.12|3569.76|3605.45|7026.88|7062.57|-2878.00| +|70492||||||||4684||2|6|||6250|223|1233|||151.61|||309.28|324.32||6.18|0.00|54.56|309.28|315.46|363.84|370.02|-15.04| +2450839|70492|2450905|64266|1799275|4684|8046|64266|1799275|4684|8046|2|88|9|1|7645|177|1233|79|36.40|85.90|59.27|2103.77|4682.33|2875.60|6786.10|421.40|0.00|3257.17|4682.33|5103.73|7939.50|8360.90|1806.73| +2450839|70492|2450844|64266|1799275|4684|8046|64266|1799275|4684|8046|2|106|12|1|1111|212|1233|52|57.57|60.44|1.20|3080.48|62.40|2993.64|3142.88|5.61|0.00|1162.72|62.40|68.01|1225.12|1230.73|-2931.24| +2450839|36160|2450841|73876|822319|2762|17958|25851|349244|1687|21283|4|23|20|4|6170|85|1234|15|17.27|30.56|28.72|27.60|430.80|259.05|458.40|0.00|0.00|68.70|430.80|430.80|499.50|499.50|171.75| +2450839|36160|2450892|73876|822319|2762|17958|25851|349244|1687|21283|4|97|4|3|8917|59|1234|68|42.93|44.21|24.31|1353.20|1653.08|2919.24|3006.28|83.31|727.35|811.24|925.73|1009.04|1736.97|1820.28|-1993.51| +2450839|36160|2450860|73876|822319|2762|17958|25851|349244|1687|21283|4|88|18|3|6524|112|1234|23|6.09|8.40|1.34|162.38|30.82|140.07|193.20|1.54|0.00|28.98|30.82|32.36|59.80|61.34|-109.25| +2450839|36160|2450907|73876|822319|2762|17958|25851|349244|1687|21283|4|49|5|1|11734|118|1234|75|92.45|197.84|41.54|11722.50|3115.50|6933.75|14838.00|62.31|0.00|1780.50|3115.50|3177.81|4896.00|4958.31|-3818.25| +2450839|36160|2450879|73876|822319|2762|17958|25851|349244|1687|21283|4|86|2|3|14194|130|1234|26|84.90|229.23|217.76|298.22|5661.76|2207.40|5959.98|264.97|2717.64|2622.36|2944.12|3209.09|5566.48|5831.45|736.72| +2450839|36160|2450929|73876|822319|2762|17958|25851|349244|1687|21283|4|44|5|3|1201|84|1234|46|56.45|154.67|12.37|6545.80|569.02|2596.70|7114.82|9.21|415.38|569.02|153.64|162.85|722.66|731.87|-2443.06| +2450839|36160|2450844|73876|822319|2762|17958|25851|349244|1687|21283|4|43|4|3|5240|238|1234|32|10.89|12.08|5.43|212.80|173.76|348.48|386.56|12.16|0.00|57.92|173.76|185.92|231.68|243.84|-174.72| +2450839|71053|2450895|66054|135044|34|10624|66054|135044|34|10624|4|83|9|4|15253|94|1235|78|80.89|167.44|123.90|3396.12|9664.20|6309.42|13060.32|483.21|0.00|3917.94|9664.20|10147.41|13582.14|14065.35|3354.78| +2450839|71053|2450855|66054|135044|34|10624|66054|135044|34|10624|4|8|19|4|1645|29|1235|90|16.86|48.72|13.15|3201.30|1183.50|1517.40|4384.80|59.17|0.00|1534.50|1183.50|1242.67|2718.00|2777.17|-333.90| +2450839|71053|2450896|66054|135044|34|10624|66054|135044|34|10624|4|2|3|1|17986|84|1235|4|78.50|215.87|116.56|397.24|466.24|314.00|863.48|37.29|0.00|0.00|466.24|503.53|466.24|503.53|152.24| +2450839|71053|2450888|66054|135044|34|10624|66054|135044|34|10624|4|93|20|2|9655|11|1235|6|5.78|5.89|2.65|19.44|15.90|34.68|35.34|0.95|0.00|0.66|15.90|16.85|16.56|17.51|-18.78| +2450839|71053|2450921|66054|135044|34|10624|66054|135044|34|10624|4|73|12|4|2908|118|1235|14|92.53|258.15|175.54|1156.54|2457.56|1295.42|3614.10|24.57|0.00|1373.26|2457.56|2482.13|3830.82|3855.39|1162.14| +2450839|71053|2450866|66054|135044|34|10624|66054|135044|34|10624|4|2|7|5|266|36|1235|11|65.02|159.29|30.26|1419.33|332.86|715.22|1752.19|9.98|0.00|420.42|332.86|342.84|753.28|763.26|-382.36| +2450839|71053|2450883|66054|135044|34|10624|66054|135044|34|10624|4|49|14|1|16078|288|1235|3|99.16|166.58|101.61|194.91|304.83|297.48|499.74|3.04|0.00|149.91|304.83|307.87|454.74|457.78|7.35| +2450839|71053|2450847|66054|135044|34|10624|66054|135044|34|10624|4|92|19|2|16652|30|1235|58|35.77|106.95|66.30|2357.70|3845.40|2074.66|6203.10|192.27|0.00|1178.56|3845.40|4037.67|5023.96|5216.23|1770.74| +2450839|71053|2450928|66054|135044|34|10624|66054|135044|34|10624|4|63|10|2|15286|13|1235|47|7.87|16.76|10.89|275.89|511.83|369.89|787.72|0.00|470.88|39.01|40.95|40.95|79.96|79.96|-328.94| +2450839|71053|2450886|66054|135044|34|10624|66054|135044|34|10624|4|38|10|2|9400|294|1235|79|52.39|73.86|53.17|1634.51|4200.43|4138.81|5834.94|294.03|0.00|2450.58|4200.43|4494.46|6651.01|6945.04|61.62| +2450839|71053|2450896|66054|135044|34|10624|66054|135044|34|10624|4|14|4|3|10999|130|1235|55|38.83|70.67|30.38|2215.95|1670.90|2135.65|3886.85|50.12|0.00|543.95|1670.90|1721.02|2214.85|2264.97|-464.75| +2450839|29605|2450925|94976|1511195|4567|45186|94976|1511195|4567|45186|2|53|9|3|2344|52|1236|99|66.82|123.61|92.70|3060.09|9177.30|6615.18|12237.39|825.95|0.00|2447.28|9177.30|10003.25|11624.58|12450.53|2562.12| +2450839|29605|2450897|94976|1511195|4567|45186|94976|1511195|4567|45186|2|30|14|4|5212|131|1236|89|94.75|206.55|37.17|15074.82|3308.13|8432.75|18382.95|132.32|0.00|1102.71|3308.13|3440.45|4410.84|4543.16|-5124.62| +2450839|29605|2450923|94976|1511195|4567|45186|94976|1511195|4567|45186|2|55|14|3|5198|148|1236|85|15.29|32.87|1.97|2626.50|167.45|1299.65|2793.95|6.69|0.00|1116.90|167.45|174.14|1284.35|1291.04|-1132.20| +2450839|29605|2450895|94976|1511195|4567|45186|94976|1511195|4567|45186|2|24|2|2|10694|184|1236|26|32.39|33.68|33.34|8.84|866.84|842.14|875.68|78.01|0.00|210.08|866.84|944.85|1076.92|1154.93|24.70| +2450839|29605|2450849|94976|1511195|4567|45186|94976|1511195|4567|45186|2|28|11|4|4369|198|1236|45|23.06|32.05|13.46|836.55|605.70|1037.70|1442.25|10.53|78.74|620.10|526.96|537.49|1147.06|1157.59|-510.74| +2450839|29605|2450880|94976|1511195|4567|45186|94976|1511195|4567|45186|2|22|13|1|16996|129|1236|93|25.48|66.75|5.34|5711.13|496.62|2369.64|6207.75|4.96|0.00|1675.86|496.62|501.58|2172.48|2177.44|-1873.02| +2450839|29605|2450875|94976|1511195|4567|45186|94976|1511195|4567|45186|2|27|13|2|260|87|1236|49|95.60|195.02|138.46|2771.44|6784.54|4684.40|9555.98|15.60|5224.09|2006.55|1560.45|1576.05|3567.00|3582.60|-3123.95| +2450839|29605|2450861|94976|1511195|4567|45186|94976|1511195|4567|45186|2|81|16|3|1141|149|1236|43|25.81|70.46|53.54|727.56|2302.22|1109.83|3029.78|184.17|0.00|757.23|2302.22|2486.39|3059.45|3243.62|1192.39| +2450839|17477|2450873|42840|852368|2343|39471|86261|799727|2055|6783|4|6|6|5|16033|69|1237|95|54.85|139.86|2.79|13021.65|265.05|5210.75|13286.70|0.00|0.00|2258.15|265.05|265.05|2523.20|2523.20|-4945.70| +2450839|17477|2450893|42840|852368|2343|39471|86261|799727|2055|6783|4|55|20|4|16472|227|1237|21|62.47|76.83|23.81|1113.42|500.01|1311.87|1613.43|15.00|0.00|48.30|500.01|515.01|548.31|563.31|-811.86| +2450839|17477|2450846|42840|852368|2343|39471|86261|799727|2055|6783|4|18|11|3|4690|290|1237|56|44.39|88.78|0.00|4971.68|0.00|2485.84|4971.68|0.00|0.00|1690.08|0.00|0.00|1690.08|1690.08|-2485.84| +2450839|17477|2450897|42840|852368|2343|39471|86261|799727|2055|6783|4|56|20|4|2330|296|1237|24|76.04|161.20|140.24|503.04|3365.76|1824.96|3868.80|33.65|0.00|464.16|3365.76|3399.41|3829.92|3863.57|1540.80| +2450839|17477|2450853|42840|852368|2343|39471|86261|799727|2055|6783|4|68|11|5|1663|283|1237|96|12.26|12.99|4.80|786.24|460.80|1176.96|1247.04|29.03|138.24|523.20|322.56|351.59|845.76|874.79|-854.40| +2450839|17477|2450855|42840|852368|2343|39471|86261|799727|2055|6783|4|57|7|1|12346|246|1237|34|95.57|153.86|87.70|2249.44|2981.80|3249.38|5231.24|238.54|0.00|784.38|2981.80|3220.34|3766.18|4004.72|-267.58| +2450839|17477|2450875|42840|852368|2343|39471|86261|799727|2055|6783|4|55|4|4|4652|186|1237|43|73.54|108.10|15.13|3997.71|650.59|3162.22|4648.30|13.01|0.00|697.03|650.59|663.60|1347.62|1360.63|-2511.63| +2450839|51651|2450877|68314|794873|985|24744|68314|794873|985|24744|2|24|17|5|10712|188|1238|60|23.73|70.71|22.62|2885.40|1357.20|1423.80|4242.60|0.00|0.00|805.80|1357.20|1357.20|2163.00|2163.00|-66.60| +2450839|51651|2450877|68314|794873|985|24744|68314|794873|985|24744|2|50|18|5|10504|169|1238|37|57.87|85.64|83.07|95.09|3073.59|2141.19|3168.68|276.62|0.00|94.72|3073.59|3350.21|3168.31|3444.93|932.40| +2450839|51651|2450862|68314|794873|985|24744|68314|794873|985|24744|2|6|16|3|10651|137|1238|80|79.16|119.53|38.24|6503.20|3059.20|6332.80|9562.40|134.91|1560.19|573.60|1499.01|1633.92|2072.61|2207.52|-4833.79| +2450839|51651|2450857|68314|794873|985|24744|68314|794873|985|24744|2|96|3|3|10382|115|1238|33|74.33|146.43|106.89|1304.82|3527.37|2452.89|4832.19|105.82|0.00|772.86|3527.37|3633.19|4300.23|4406.05|1074.48| +2450839|51651|2450895|68314|794873|985|24744|68314|794873|985|24744|2|68|17|4|5335|266|1238|56|67.74|81.96|16.39|3671.92|917.84|3793.44|4589.76|13.30|651.66|917.84|266.18|279.48|1184.02|1197.32|-3527.26| +2450839|51651|2450920|68314|794873|985|24744|68314|794873|985|24744|2|19|3|5|8492|164|1238|96|55.42|129.12|58.10|6817.92|5577.60|5320.32|12395.52|111.55|0.00|4214.40|5577.60|5689.15|9792.00|9903.55|257.28| +2450839|51651|2450901|68314|794873|985|24744|68314|794873|985|24744|2|52|11|5|11959|191|1238|59|4.27|10.88|0.76|597.08|44.84|251.93|641.92|0.00|0.00|307.98|44.84|44.84|352.82|352.82|-207.09| +2450839|51651|2450905|68314|794873|985|24744|68314|794873|985|24744|2|24|15|4|13138|127|1238|92|79.43|146.94|52.89|8652.60|4865.88|7307.56|13518.48|291.95|0.00|5271.60|4865.88|5157.83|10137.48|10429.43|-2441.68| +2450839|51651|2450909|68314|794873|985|24744|68314|794873|985|24744|2|97|1|5|9157|261|1238|79|28.78|47.19|36.33|857.94|2870.07|2273.62|3728.01|148.67|746.21|1789.35|2123.86|2272.53|3913.21|4061.88|-149.76| +2450839|51651|2450928|68314|794873|985|24744|68314|794873|985|24744|2|38|9|4|9442|127|1238|52|80.60|232.12|81.24|7845.76|4224.48|4191.20|12070.24|295.71|0.00|2051.92|4224.48|4520.19|6276.40|6572.11|33.28| +2450839|51651|2450844|68314|794873|985|24744|68314|794873|985|24744|2|24|6|5|10549|26|1238|27|95.43|172.72|91.54|2191.86|2471.58|2576.61|4663.44|123.57|0.00|1072.44|2471.58|2595.15|3544.02|3667.59|-105.03| +2450839|51651|2450870|68314|794873|985|24744|68314|794873|985|24744|2|17|18|3|14722|280|1238|52|21.75|45.24|33.93|588.12|1764.36|1131.00|2352.48|3.88|1570.28|611.52|194.08|197.96|805.60|809.48|-936.92| +2450839|51651|2450863|68314|794873|985|24744|68314|794873|985|24744|2|55|17|1|14971|278|1238|48|54.86|74.60|70.87|179.04|3401.76|2633.28|3580.80|81.64|1360.70|1217.28|2041.06|2122.70|3258.34|3339.98|-592.22| +|51651||||985||68314||985||2|42||5|11528||1238|||||327.60|4341.48|||130.24||2053.74|4341.48||6395.22|6525.46|17.94| +2450839|66966|2450872|44901|1831180|2612|30491|54259|1021002|4881|5513|4|22|13|5|3847|116|1239|52|77.30|93.53|33.67|3112.72|1750.84|4019.60|4863.56|105.05|0.00|1313.00|1750.84|1855.89|3063.84|3168.89|-2268.76| +2450839|66966|2450889|44901|1831180|2612|30491|54259|1021002|4881|5513|4|85|19|3|9512|253|1239|97|54.55|119.46|40.61|7648.45|3939.17|5291.35|11587.62|44.90|1693.84|4055.57|2245.33|2290.23|6300.90|6345.80|-3046.02| +2450839|66966|2450910|44901|1831180|2612|30491|54259|1021002|4881|5513|4|45|15|2|4342|152|1239|18|99.24|293.75|111.62|3278.34|2009.16|1786.32|5287.50|40.18|0.00|1216.08|2009.16|2049.34|3225.24|3265.42|222.84| +2450839|66966|2450868|44901|1831180|2612|30491|54259|1021002|4881|5513|4|67|8|2|10250|32|1239|61|7.83|16.05|3.69|753.96|225.09|477.63|979.05|0.00|155.31|303.17|69.78|69.78|372.95|372.95|-407.85| +2450839|66966|2450862|44901|1831180|2612|30491|54259|1021002|4881|5513|4|11|2|5|7558|188|1239|69|65.82|69.11|20.04|3385.83|1382.76|4541.58|4768.59|13.82|0.00|428.49|1382.76|1396.58|1811.25|1825.07|-3158.82| +2450839|66966|2450912|44901|1831180|2612|30491|54259|1021002|4881|5513|4|63|15|2|1342|153|1239|75|63.59|180.59|81.26|7449.75|6094.50|4769.25|13544.25|0.00|609.45|6230.25|5485.05|5485.05|11715.30|11715.30|715.80| +2450839|66966|2450890|44901|1831180|2612|30491|54259|1021002|4881|5513|4|77|17|1|12952|277|1239|46|80.10|132.16|35.68|4438.08|1641.28|3684.60|6079.36|15.10|1263.78|2917.78|377.50|392.60|3295.28|3310.38|-3307.10| +2450839|66966|2450903|44901|1831180|2612|30491|54259|1021002|4881|5513|4|71|14|1|4418|197|1239|5|15.27|27.48|2.74|123.70|13.70|76.35|137.40|1.23|0.00|49.45|13.70|14.93|63.15|64.38|-62.65| +2450839|66966|2450860|44901|1831180|2612|30491|54259|1021002|4881|5513|4|80|17|5|15070|242|1239|73|7.55|17.96|6.28|852.64|458.44|551.15|1311.08|25.03|41.25|288.35|417.19|442.22|705.54|730.57|-133.96| +2450839|66966|2450877|44901|1831180|2612|30491|54259|1021002|4881|5513|4|54|11|5|7690|186|1239|98|3.55|9.54|6.29|318.50|616.42|347.90|934.92|49.31|0.00|457.66|616.42|665.73|1074.08|1123.39|268.52| +2450839|66966|2450915|44901|1831180|2612|30491|54259|1021002|4881|5513|4|92|7|1|662|156|1239|54|89.89|118.65|33.22|4613.22|1793.88|4854.06|6407.10|107.63|0.00|3203.28|1793.88|1901.51|4997.16|5104.79|-3060.18| +2450839|66966|2450858|44901|1831180|2612|30491|54259|1021002|4881|5513|4|59|16|2|12280|30|1239|16|91.26|115.90|33.61|1316.64|537.76|1460.16|1854.40|4.84|295.76|407.84|242.00|246.84|649.84|654.68|-1218.16| +2450839|66966|2450865|44901|1831180|2612|30491|54259|1021002|4881|5513|4|66|6|2|14149|142|1239|26|98.13|172.70|75.98|2514.72|1975.48|2551.38|4490.20|138.28|0.00|314.08|1975.48|2113.76|2289.56|2427.84|-575.90| +2450839|4876|2450926|33000|246466|285|16504|85477|1139789|1608|274|2|98|20|4|14914|100|1240|45|67.92|201.72|143.22|2632.50|6444.90|3056.40|9077.40|0.00|6251.55|1089.00|193.35|193.35|1282.35|1282.35|-2863.05| +2450839|4876|2450927|33000|246466|285|16504|85477|1139789|1608|274|2|26|14|2|4622|30|1240|91|82.27|163.71|139.15|2234.96|12662.65|7486.57|14897.61|759.75|0.00|2532.53|12662.65|13422.40|15195.18|15954.93|5176.08| +2450839|4876|2450874|33000|246466|285|16504|85477|1139789|1608|274|2|106|17|3|3715|15|1240|19|93.03|240.94|33.73|3936.99|640.87|1767.57|4577.86|51.26|0.00|549.29|640.87|692.13|1190.16|1241.42|-1126.70| +2450839|4876|2450890|33000|246466|285|16504|85477|1139789|1608|274|2|16|12|5|17138|198|1240|90|57.14|62.28|15.57|4203.90|1401.30|5142.60|5605.20|28.02|0.00|1064.70|1401.30|1429.32|2466.00|2494.02|-3741.30| +2450839|4876|2450914|33000|246466|285|16504|85477|1139789|1608|274|2|17|4|2|1594|75|1240|45|69.83|97.76|87.98|440.10|3959.10|3142.35|4399.20|316.72|0.00|1891.35|3959.10|4275.82|5850.45|6167.17|816.75| +2450839|42067|2450849|28207|122032|2704|23729|28207|122032|2704|23729|4|56|13|2|9410|238|1241|87|10.62|23.36|0.46|1992.30|40.02|923.94|2032.32|2.40|0.00|771.69|40.02|42.42|811.71|814.11|-883.92| +2450839|42067|2450907|28207|122032|2704|23729|28207|122032|2704|23729|4|7|7|1|15878|274|1241|36|88.68|235.00|150.40|3045.60|5414.40|3192.48|8460.00|270.72|0.00|1522.80|5414.40|5685.12|6937.20|7207.92|2221.92| +2450839|42067|2450856|28207|122032|2704|23729|28207|122032|2704|23729|4|98|14|4|8624|99|1241|44|76.03|155.86|95.07|2674.76|4183.08|3345.32|6857.84|199.11|1338.58|3085.72|2844.50|3043.61|5930.22|6129.33|-500.82| +2450839|42067|2450898|28207|122032|2704|23729|28207|122032|2704|23729|4|44|18|3|11128|110|1241|84|32.31|39.09|13.68|2134.44|1149.12|2714.04|3283.56|0.00|0.00|1214.64|1149.12|1149.12|2363.76|2363.76|-1564.92| +2450839|42067|2450885|28207|122032|2704|23729|28207|122032|2704|23729|4|83|12|4|9097|8|1241|95|91.12|236.00|167.56|6501.80|15918.20|8656.40|22420.00|477.54|0.00|6277.60|15918.20|16395.74|22195.80|22673.34|7261.80| +2450839|42067|2450912|28207|122032|2704|23729|28207|122032|2704|23729|4|107|17|4|4918|105|1241|24|63.76|151.74|72.83|1893.84|1747.92|1530.24|3641.76|0.00|0.00|1784.40|1747.92|1747.92|3532.32|3532.32|217.68| +2450839|42067|2450864|28207|122032|2704|23729|28207|122032|2704|23729|4|29|18|1|9014|175|1241|98|74.31|184.28|70.02|11197.48|6861.96|7282.38|18059.44|98.81|5626.80|4514.86|1235.16|1333.97|5750.02|5848.83|-6047.22| +2450839|42067|2450917|28207|122032|2704|23729|28207|122032|2704|23729|4|97|9|4|14419|95|1241|60|19.13|23.52|15.05|508.20|903.00|1147.80|1411.20|45.15|0.00|13.80|903.00|948.15|916.80|961.95|-244.80| +2450839|47720|2450876|83332|553796|3013|16277|34768|607558|4422|4439|2|57|1|2|4075|297|1242|91|85.11|182.98|131.74|4662.84|11988.34|7745.01|16651.18|959.06|0.00|5661.11|11988.34|12947.40|17649.45|18608.51|4243.33| +2450839|47720|2450855|83332|553796|3013|16277|34768|607558|4422|4439|2|36|17|1|3505|284|1242|39|34.63|80.34|77.92|94.38|3038.88|1350.57|3133.26|30.38|0.00|751.92|3038.88|3069.26|3790.80|3821.18|1688.31| +2450839|47720|2450906|83332|553796|3013|16277|34768|607558|4422|4439|2|51|19|3|12619|117|1242|41|49.34|120.88|105.16|644.52|4311.56|2022.94|4956.08|129.34|0.00|1288.22|4311.56|4440.90|5599.78|5729.12|2288.62| +2450839|47720|2450881|83332|553796|3013|16277|34768|607558|4422|4439|2|29|15|4|15331|192|1242|99|29.52|62.87|34.57|2801.70|3422.43|2922.48|6224.13|206.03|479.14|559.35|2943.29|3149.32|3502.64|3708.67|20.81| +2450839|47720|2450889|83332|553796|3013|16277|34768|607558|4422|4439|2|55|4|5|2017|114|1242|88|99.07|151.57|60.62|8003.60|5334.56|8718.16|13338.16|106.69|0.00|6001.60|5334.56|5441.25|11336.16|11442.85|-3383.60| +2450839|47720|2450842|83332|553796|3013|16277|34768|607558|4422|4439|2|89|14|3|5398|241|1242|7|62.16|128.04|58.89|484.05|412.23|435.12|896.28|16.48|0.00|134.40|412.23|428.71|546.63|563.11|-22.89| +2450839|47720|2450897|83332|553796|3013|16277|34768|607558|4422|4439|2|35|16|1|17248|291|1242|41|31.23|49.03|9.80|1608.43|401.80|1280.43|2010.23|4.01|0.00|542.43|401.80|405.81|944.23|948.24|-878.63| +2450839|47720|2450907|83332|553796|3013|16277|34768|607558|4422|4439|2|21|13|1|811|264|1242|10|88.85|159.04|106.55|524.90|1065.50|888.50|1590.40|0.42|1054.84|524.80|10.66|11.08|535.46|535.88|-877.84| +2450839|47720|2450885|83332|553796|3013|16277|34768|607558|4422|4439|2|89|14|5|7072|110|1242|7|58.15|127.34|25.46|713.16|178.22|407.05|891.38|10.69|0.00|445.69|178.22|188.91|623.91|634.60|-228.83| +2450839|47720|2450879|83332|553796|3013|16277|34768|607558|4422|4439|2|77|13|2|12452|226|1242|78|50.00|128.00|23.04|8186.88|1797.12|3900.00|9984.00|89.85|0.00|1896.96|1797.12|1886.97|3694.08|3783.93|-2102.88| +2450839|67763|2450871|6020|930720|6267|39814|6020|930720|6267|39814|1|68|10|5|8188|53|1243|55|4.87|5.30|2.12|174.90|116.60|267.85|291.50|2.33|0.00|52.25|116.60|118.93|168.85|171.18|-151.25| +2450839|67763|2450901|6020|930720|6267|39814|6020|930720|6267|39814|1|97|15|2|13708|281|1243|88|45.67|46.12|4.61|3652.88|405.68|4018.96|4058.56|28.39|0.00|1947.44|405.68|434.07|2353.12|2381.51|-3613.28| +2450839|67763|2450877|6020|930720|6267|39814|6020|930720|6267|39814|1|31|3|5|2320|60|1243|93|32.87|34.84|33.44|130.20|3109.92|3056.91|3240.12|155.49|0.00|810.03|3109.92|3265.41|3919.95|4075.44|53.01| +2450839|67763|2450847|6020|930720|6267|39814|6020|930720|6267|39814|1|16|13|4|17929|232|1243|81|64.06|108.90|89.29|1588.41|7232.49|5188.86|8820.90|578.59|0.00|1851.66|7232.49|7811.08|9084.15|9662.74|2043.63| +2450839|67763|2450872|6020|930720|6267|39814|6020|930720|6267|39814|1|89|20|2|8593|261|1243|85|37.66|94.15|31.06|5362.65|2640.10|3201.10|8002.75|5.28|2508.09|560.15|132.01|137.29|692.16|697.44|-3069.09| +2450839|60042|2450877|9692|1316718|897|47123|9692|1316718|897|47123|2|99|15|2|13870|245|1244|49|42.80|89.88|46.73|2114.35|2289.77|2097.20|4404.12|137.38|0.00|1541.05|2289.77|2427.15|3830.82|3968.20|192.57| +2450839|60042|2450929|9692|1316718|897|47123|9692|1316718|897|47123|2|88|15|5|13168|14|1244|20|78.47|197.74|164.12|672.40|3282.40|1569.40|3954.80|49.89|787.77|355.80|2494.63|2544.52|2850.43|2900.32|925.23| +2450839|60042|2450845|9692|1316718|897|47123|9692|1316718|897|47123|2|17|15|3|10|4|1244|1|93.98|234.01|217.62|16.39|217.62|93.98|234.01|0.91|202.38|14.04|15.24|16.15|29.28|30.19|-78.74| +2450839|60042|2450887|9692|1316718|897|47123|9692|1316718|897|47123|2|81|7|4|12550|18|1244|28|74.93|86.16|31.01|1544.20|868.28|2098.04|2412.48|8.68|0.00|1109.64|868.28|876.96|1977.92|1986.60|-1229.76| +2450839|60042|2450848|9692|1316718|897|47123|9692|1316718|897|47123|2|96|20|5|3026|126|1244|1|79.20|171.07|141.98|29.09|141.98|79.20|171.07|7.09|0.00|30.79|141.98|149.07|172.77|179.86|62.78| +2450839|60042|2450873|9692|1316718|897|47123|9692|1316718|897|47123|2|62|6|3|8191|166|1244|48|46.25|94.81|35.07|2867.52|1683.36|2220.00|4550.88|16.32|50.50|455.04|1632.86|1649.18|2087.90|2104.22|-587.14| +2450839|60042|2450903|9692|1316718|897|47123|9692|1316718|897|47123|2|76|4|4|16844|221|1244|47|37.82|86.60|70.14|773.62|3296.58|1777.54|4070.20|0.00|1813.11|1342.79|1483.47|1483.47|2826.26|2826.26|-294.07| +2450839|60042|2450926|9692|1316718|897|47123|9692|1316718|897|47123|2|102|12|3|13994|260|1244|33|77.54|136.47|35.48|3332.67|1170.84|2558.82|4503.51|0.00|0.00|1576.08|1170.84|1170.84|2746.92|2746.92|-1387.98| +2450839|60042|2450854|9692|1316718|897|47123|9692|1316718|897|47123|2|51|11|5|1136|200|1244|59|29.17|58.92|36.53|1321.01|2155.27|1721.03|3476.28|36.20|1551.79|417.13|603.48|639.68|1020.61|1056.81|-1117.55| +2450839|60042|2450919|9692|1316718|897|47123|9692|1316718|897|47123|2|70|16|5|8870|42|1244|58|79.19|114.03|2.28|6481.50|132.24|4593.02|6613.74|9.25|0.00|3174.34|132.24|141.49|3306.58|3315.83|-4460.78| +2450839|60042|2450904|9692|1316718|897|47123|9692|1316718|897|47123|2|83|5|3|6088|219|1244|43|11.02|23.58|17.21|273.91|740.03|473.86|1013.94|59.20|0.00|456.23|740.03|799.23|1196.26|1255.46|266.17| +2450839|60042|2450908|9692|1316718|897|47123|9692|1316718|897|47123|2|73|16|3|4654|132|1244|35|8.95|14.05|12.64|49.35|442.40|313.25|491.75|5.04|274.28|221.20|168.12|173.16|389.32|394.36|-145.13| +2450839|69620|2450893|66300|426399|545|33444|66300|426399|545|33444|1|69|15|2|13562|138|1245|48|19.69|51.98|1.03|2445.60|49.44|945.12|2495.04|0.98|0.00|1122.72|49.44|50.42|1172.16|1173.14|-895.68| +2450839|69620|2450923|66300|426399|545|33444|66300|426399|545|33444|1|34|11|2|478|209|1245|71|65.58|123.94|86.75|2640.49|6159.25|4656.18|8799.74|307.96|0.00|1055.77|6159.25|6467.21|7215.02|7522.98|1503.07| +2450839|69620|2450855|66300|426399|545|33444|66300|426399|545|33444|1|83|10|5|277|157|1245|33|28.32|43.89|34.23|318.78|1129.59|934.56|1448.37|11.29|0.00|274.89|1129.59|1140.88|1404.48|1415.77|195.03| +2450839|69620|2450926|66300|426399|545|33444|66300|426399|545|33444|1|63|20|5|6748|22|1245|91|36.85|36.85|16.95|1810.90|1542.45|3353.35|3353.35|61.69|0.00|502.32|1542.45|1604.14|2044.77|2106.46|-1810.90| +2450839|69620|2450866|66300|426399|545|33444|66300|426399|545|33444|1|10|7|1|14986|195|1245|86|85.03|197.26|47.34|12893.12|4071.24|7312.58|16964.36|68.39|1791.34|6276.28|2279.90|2348.29|8556.18|8624.57|-5032.68| +2450839|69620|2450892|66300|426399|545|33444|66300|426399|545|33444|1|49|3|2|7772|162|1245|35|71.38|202.71|133.78|2412.55|4682.30|2498.30|7094.85|88.02|2481.61|1986.25|2200.69|2288.71|4186.94|4274.96|-297.61| +2450839|69620|2450880|66300|426399|545|33444|66300|426399|545|33444|1|83|8|3|2284|297|1245|30|17.36|32.63|7.83|744.00|234.90|520.80|978.90|9.39|0.00|479.40|234.90|244.29|714.30|723.69|-285.90| +2450839|69620|2450881|66300|426399|545|33444|66300|426399|545|33444|1|72|20|5|6332|78|1245|62|87.04|112.28|89.82|1392.52|5568.84|5396.48|6961.36|278.44|0.00|1879.22|5568.84|5847.28|7448.06|7726.50|172.36| +2450839|69620|2450849|66300|426399|545|33444|66300|426399|545|33444|1|108|15|5|8480|298|1245|65|87.27|191.99|80.63|7238.40|5240.95|5672.55|12479.35|471.68|0.00|3493.75|5240.95|5712.63|8734.70|9206.38|-431.60| +2450839|69620|2450842|66300|426399|545|33444|66300|426399|545|33444|1|60|10|3|8581|111|1245|65|87.52|152.28|38.07|7423.65|2474.55|5688.80|9898.20|148.47|0.00|2177.50|2474.55|2623.02|4652.05|4800.52|-3214.25| +2450839|69620|2450928|66300|426399|545|33444|66300|426399|545|33444|1|9|18|3|13496|255|1245|17|5.87|11.32|4.86|109.82|82.62|99.79|192.44|5.78|0.00|7.65|82.62|88.40|90.27|96.05|-17.17| +2450839|69620|2450872|66300|426399|545|33444|66300|426399|545|33444|1|39|4|5|16316|82|1245|71|56.63|83.81|41.06|3035.25|2915.26|4020.73|5950.51|87.45|0.00|1070.68|2915.26|3002.71|3985.94|4073.39|-1105.47| +2450839|69620|2450877|66300|426399|545|33444|66300|426399|545|33444|1|14|5|3|5530|207|1245|25|4.15|6.55|0.39|154.00|9.75|103.75|163.75|0.56|0.39|49.00|9.36|9.92|58.36|58.92|-94.39| +2450839|69620|2450864|66300|426399|545|33444|66300|426399|545|33444|1|103|14|2|9331|75|1245|8|58.09|137.67|61.95|605.76|495.60|464.72|1101.36|0.00|0.00|363.44|495.60|495.60|859.04|859.04|30.88| +2450839|71894|2450848|15539|76227|5715|28403|82020|1616579|20|11908|2|14|17|1|5570|297|1246|80|95.39|205.08|94.33|8860.00|7546.40|7631.20|16406.40|603.71|0.00|6562.40|7546.40|8150.11|14108.80|14712.51|-84.80| +2450839|71894|2450923|15539|76227|5715|28403|82020|1616579|20|11908|2|1|19|4|12829|33|1246|56|16.95|24.74|22.01|152.88|1232.56|949.20|1385.44|24.65|0.00|221.20|1232.56|1257.21|1453.76|1478.41|283.36| +2450839|71894|2450872|15539|76227|5715|28403|82020|1616579|20|11908|2|58|5|1|3868|160|1246|61|3.62|7.16|7.16|0.00|436.76|220.82|436.76|26.20|0.00|196.42|436.76|462.96|633.18|659.38|215.94| +2450839|71894|2450927|15539|76227|5715|28403|82020|1616579|20|11908|2|89|20|3|4360|182|1246|87|71.02|145.59|135.39|887.40|11778.93|6178.74|12666.33|24.73|10954.40|2785.74|824.53|849.26|3610.27|3635.00|-5354.21| +2450839|71894|2450898|15539|76227|5715|28403|82020|1616579|20|11908|2|71|16|5|9524|183|1246|54|97.77|183.80|141.52|2283.12|7642.08|5279.58|9925.20|73.36|305.68|793.80|7336.40|7409.76|8130.20|8203.56|2056.82| +2450839|71894|2450891|15539|76227|5715|28403|82020|1616579|20|11908|2|16|3|2|14353|270|1246|39|96.12|219.15|43.83|6837.48|1709.37|3748.68|8546.85|75.38|871.77|3162.12|837.60|912.98|3999.72|4075.10|-2911.08| +2450839|54214|2450908|83452|359355|3258|8761|83452|359355|3258|8761|2|7|2|2|16090|97|1247|78|75.10|166.72|145.04|1691.04|11313.12|5857.80|13004.16|905.04|0.00|4290.78|11313.12|12218.16|15603.90|16508.94|5455.32| +2450839|54214|2450852|83452|359355|3258|8761|83452|359355|3258|8761|2|29|11|5|6464|63|1247|76|83.25|169.83|164.73|387.60|12519.48|6327.00|12907.08|0.00|0.00|3743.00|12519.48|12519.48|16262.48|16262.48|6192.48| +2450839|54214|2450905|83452|359355|3258|8761|83452|359355|3258|8761|2|14|5|5|8486|123|1247|56|39.32|53.47|16.04|2096.08|898.24|2201.92|2994.32|71.85|0.00|1467.20|898.24|970.09|2365.44|2437.29|-1303.68| +2450839|54214|2450853|83452|359355|3258|8761|83452|359355|3258|8761|2|33|1|1|7846|134|1247|93|15.67|28.67|7.74|1946.49|719.82|1457.31|2666.31|64.78|0.00|585.90|719.82|784.60|1305.72|1370.50|-737.49| +2450839|54214|2450888|83452|359355|3258|8761|83452|359355|3258|8761|2|3|4|4|12338|184|1247|26|74.67|197.87|75.19|3189.68|1954.94|1941.42|5144.62|136.84|0.00|360.10|1954.94|2091.78|2315.04|2451.88|13.52| +2450839|54214|2450886|83452|359355|3258|8761|83452|359355|3258|8761|2|19|8|4|931|32|1247|40|70.27|191.13|139.52|2064.40|5580.80|2810.80|7645.20|214.86|1283.58|1070.00|4297.22|4512.08|5367.22|5582.08|1486.42| +2450839|54214|2450923|83452|359355|3258|8761|83452|359355|3258|8761|2|41|7|3|9997|75|1247|3|42.81|98.03|19.60|235.29|58.80|128.43|294.09|2.61|6.46|120.57|52.34|54.95|172.91|175.52|-76.09| +2450839|54214|2450867|83452|359355|3258|8761|83452|359355|3258|8761|2|51|18|2|3014|15|1247|35|5.21|10.99|8.68|80.85|303.80|182.35|384.65|21.26|0.00|180.60|303.80|325.06|484.40|505.66|121.45| +2450839|22056|2450850|89492|1150992|3245|10745|89492|1150992|3245|10745|2|6|10|3|10648|149|1248|53|23.08|26.54|6.63|1055.23|351.39|1223.24|1406.62|7.02|0.00|168.54|351.39|358.41|519.93|526.95|-871.85| +2450839|22056|2450882|89492|1150992|3245|10745|89492|1150992|3245|10745|2|68|1|1|938|113|1248|4|23.81|68.81|15.13|214.72|60.52|95.24|275.24|0.60|0.00|96.32|60.52|61.12|156.84|157.44|-34.72| +2450839|22056|2450875|89492|1150992|3245|10745|89492|1150992|3245|10745|2|22|3|3|4850|58|1248|16|17.20|35.77|13.23|360.64|211.68|275.20|572.32|4.23|0.00|268.96|211.68|215.91|480.64|484.87|-63.52| +2450839|22056|2450905|89492|1150992|3245|10745|89492|1150992|3245|10745|2|77|4|1|7612|65|1248|39|17.08|40.47|36.01|173.94|1404.39|666.12|1578.33|56.17|0.00|441.87|1404.39|1460.56|1846.26|1902.43|738.27| +2450839|22056|2450929|89492|1150992|3245|10745|89492|1150992|3245|10745|2|64|19|2|3889|254|1248|27|14.91|31.31|25.67|152.28|693.09|402.57|845.37|48.51|0.00|219.78|693.09|741.60|912.87|961.38|290.52| +2450839|22056|2450896|89492|1150992|3245|10745|89492|1150992|3245|10745|2|72|10|5|5101|57|1248|55|28.73|49.70|43.73|328.35|2405.15|1580.15|2733.50|72.15|0.00|54.45|2405.15|2477.30|2459.60|2531.75|825.00| +2450839|22056|2450893|89492|1150992|3245|10745|89492|1150992|3245|10745|2|88|10|4|15067|196|1248|21|10.66|16.94|2.37|305.97|49.77|223.86|355.74|0.49|0.00|3.36|49.77|50.26|53.13|53.62|-174.09| +2450839|22056|2450915|89492|1150992|3245|10745|89492|1150992|3245|10745|2|11|12|2|6094|4|1248|53|11.26|19.25|18.09|61.48|958.77|596.78|1020.25|28.76|0.00|509.86|958.77|987.53|1468.63|1497.39|361.99| +2450839|22056|2450896|89492|1150992|3245|10745|89492|1150992|3245|10745|2|17|17|5|7225|210|1248|5|36.99|53.26|41.54|58.60|207.70|184.95|266.30|0.00|180.69|106.50|27.01|27.01|133.51|133.51|-157.94| +2450839|22056|2450855|89492|1150992|3245|10745|89492|1150992|3245|10745|2|79|16|3|1054|132|1248|29|30.95|37.14|11.14|754.00|323.06|897.55|1077.06|9.69|0.00|473.86|323.06|332.75|796.92|806.61|-574.49| +2450839|22056|2450856|89492|1150992|3245|10745|89492|1150992|3245|10745|2|106|6|4|4711|28|1248|66|71.81|84.73|58.46|1733.82|3858.36|4739.46|5592.18|0.00|3781.19|1733.16|77.17|77.17|1810.33|1810.33|-4662.29| +2450839|22056|2450861|89492|1150992|3245|10745|89492|1150992|3245|10745|2|51|2|4|6292|141|1248|24|37.04|44.44|32.88|277.44|789.12|888.96|1066.56|71.02|0.00|490.56|789.12|860.14|1279.68|1350.70|-99.84| +2450839|22056|2450897|89492|1150992|3245|10745|89492|1150992|3245|10745|2|19|14|1|3208|191|1248|25|25.30|31.37|27.29|102.00|682.25|632.50|784.25|40.93|0.00|196.00|682.25|723.18|878.25|919.18|49.75| +2450839|22056|2450917|89492|1150992|3245|10745|89492|1150992|3245|10745|2|104|15|4|13282|1|1248|69|39.25|71.04|37.65|2303.91|2597.85|2708.25|4901.76|103.91|0.00|1568.37|2597.85|2701.76|4166.22|4270.13|-110.40| +2450839|40838|2450885|60990|429800|210|21420|60990|429800|210|21420|1|80|15|3|5188|227|1249|95|90.11|198.24|47.57|14313.65|4519.15|8560.45|18832.80|225.95|0.00|9227.35|4519.15|4745.10|13746.50|13972.45|-4041.30| +2450839|40838|2450859|60990|429800|210|21420|60990|429800|210|21420|1|3|1|3|1786|257|1249|94|72.67|172.22|101.60|6638.28|9550.40|6830.98|16188.68|668.52|0.00|3399.04|9550.40|10218.92|12949.44|13617.96|2719.42| +2450839|40838|2450843|60990|429800|210|21420|60990|429800|210|21420|1|102|13|1|17254|232|1249|24|92.16|265.42|191.10|1783.68|4586.40|2211.84|6370.08|366.91|0.00|0.00|4586.40|4953.31|4586.40|4953.31|2374.56| +2450839|40838|2450863|60990|429800|210|21420|60990|429800|210|21420|1|30|7|3|1774|245|1249|48|95.15|126.54|56.94|3340.80|2733.12|4567.20|6073.92|245.98|0.00|2611.68|2733.12|2979.10|5344.80|5590.78|-1834.08| +2450839|40838|2450917|60990|429800|210|21420|60990|429800|210|21420|1|86|7|1|2491|90|1249|22|17.30|25.95|24.39|34.32|536.58|380.60|570.90|11.80|300.48|74.14|236.10|247.90|310.24|322.04|-144.50| +2450839|40838|2450864|60990|429800|210|21420|60990|429800|210|21420|1|28|14|3|6134|146|1249|73|1.50|1.51|0.10|102.93|7.30|109.50|110.23|0.03|6.86|43.80|0.44|0.47|44.24|44.27|-109.06| +2450839|40838|2450909|60990|429800|210|21420|60990|429800|210|21420|1|60|6|1|7430|193|1249|91|45.97|95.61|43.02|4785.69|3914.82|4183.27|8700.51|156.59|0.00|3653.65|3914.82|4071.41|7568.47|7725.06|-268.45| +2450839|40838|2450906|60990|429800|210|21420|60990|429800|210|21420|1|69|11|4|1670|10|1249|83|46.03|104.48|73.13|2602.05|6069.79|3820.49|8671.84|58.27|4127.45|1560.40|1942.34|2000.61|3502.74|3561.01|-1878.15| +2450839|40838|2450882|60990|429800|210|21420|60990|429800|210|21420|1|47|2|2|6280|189|1249|62|18.14|37.73|17.35|1263.56|1075.70|1124.68|2339.26|43.02|0.00|70.06|1075.70|1118.72|1145.76|1188.78|-48.98| +2450839|40838|2450917|60990|429800|210|21420|60990|429800|210|21420|1|87|18|4|2278|275|1249|7|82.20|89.59|41.21|338.66|288.47|575.40|627.13|23.07|0.00|131.67|288.47|311.54|420.14|443.21|-286.93| +2450839|40838|2450890|60990|429800|210|21420|60990|429800|210|21420|1|79|13|5|16483|209|1249|16|50.42|101.84|6.11|1531.68|97.76|806.72|1629.44|5.86|0.00|228.00|97.76|103.62|325.76|331.62|-708.96| +2450839|40838|2450888|60990|429800|210|21420|60990|429800|210|21420|1|12|6|4|5348|86|1249|25|65.39|131.43|124.85|164.50|3121.25|1634.75|3285.75|124.85|0.00|65.50|3121.25|3246.10|3186.75|3311.60|1486.50| +2450839|40838|2450900|60990|429800|210|21420|60990|429800|210|21420|1|45|4|5|10771|257|1249|89|6.81|14.64|1.90|1133.86|169.10|606.09|1302.96|5.07|0.00|273.23|169.10|174.17|442.33|447.40|-436.99| +2450839|62508|2450853|67448|28961|3721|30434|67448|28961|3721|30434|1|53|15|2|10939|126|1250|12|53.17|83.47|74.28|110.28|891.36|638.04|1001.64|8.91|0.00|9.96|891.36|900.27|901.32|910.23|253.32| +2450839|62508|2450923|67448|28961|3721|30434|67448|28961|3721|30434|1|107|14|1|5722|172|1250|22|40.60|117.74|116.56|25.96|2564.32|893.20|2590.28|76.92|0.00|1061.94|2564.32|2641.24|3626.26|3703.18|1671.12| +2450839|62508|2450896|67448|28961|3721|30434|67448|28961|3721|30434|1|96|7|5|12340|3|1250|94|97.05|259.12|142.51|10961.34|13395.94|9122.70|24357.28|267.91|0.00|9985.62|13395.94|13663.85|23381.56|23649.47|4273.24| +2450839|62508|2450847|67448|28961|3721|30434|67448|28961|3721|30434|1|74|8|4|17204|153|1250|13|32.78|73.42|51.39|286.39|668.07|426.14|954.46|60.12|0.00|400.79|668.07|728.19|1068.86|1128.98|241.93| +2450839|62508|2450863|67448|28961|3721|30434|67448|28961|3721|30434|1|84|2|1|10114|214|1250|87|38.61|103.08|69.06|2959.74|6008.22|3359.07|8967.96|420.57|0.00|2600.43|6008.22|6428.79|8608.65|9029.22|2649.15| +2450839|62508|2450880|67448|28961|3721|30434|67448|28961|3721|30434|1|84|12|1|1910|265|1250|52|52.12|143.85|112.20|1645.80|5834.40|2710.24|7480.20|116.68|0.00|3216.20|5834.40|5951.08|9050.60|9167.28|3124.16| +2450839|62508|2450890|67448|28961|3721|30434|67448|28961|3721|30434|1|66|5|4|5414|220|1250|14|33.60|98.78|43.46|774.48|608.44|470.40|1382.92|6.08|0.00|262.64|608.44|614.52|871.08|877.16|138.04| +2450839|62508|2450924|67448|28961|3721|30434|67448|28961|3721|30434|1|56|17|2|15271|119|1250|5|49.38|138.75|108.22|152.65|541.10|246.90|693.75|9.73|432.88|333.00|108.22|117.95|441.22|450.95|-138.68| +2450839|62508|2450887|67448|28961|3721|30434|67448|28961|3721|30434|1|97|10|5|14918|128|1250|39|75.10|78.85|41.00|1476.15|1599.00|2928.90|3075.15|127.92|0.00|676.26|1599.00|1726.92|2275.26|2403.18|-1329.90| +2450839|62508|2450853|67448|28961|3721|30434|67448|28961|3721|30434|1|80|12|4|15440|35|1250|59|9.81|23.05|0.00|1359.95|0.00|578.79|1359.95|0.00|0.00|543.98|0.00|0.00|543.98|543.98|-578.79| +2450839|62508|2450883|67448|28961|3721|30434|67448|28961|3721|30434|1|35|7|5|9850|3|1250|38|92.92|105.92|55.07|1932.30|2092.66|3530.96|4024.96|167.41|0.00|684.00|2092.66|2260.07|2776.66|2944.07|-1438.30| +2450839|62508|2450861|67448|28961|3721|30434|67448|28961|3721|30434|1|10|12|4|14467|165|1250|61|15.09|35.15|2.46|1994.09|150.06|920.49|2144.15|4.50|0.00|535.58|150.06|154.56|685.64|690.14|-770.43| +2450839|62508|2450908|67448|28961|3721|30434|67448|28961|3721|30434|1|75|13|4|16240|11|1250|62|42.87|93.88|74.16|1222.64|4597.92|2657.94|5820.56|183.91|0.00|0.00|4597.92|4781.83|4597.92|4781.83|1939.98| +2450839|62508|2450867|67448|28961|3721|30434|67448|28961|3721|30434|1|8|12|3|14222|115|1250|47|15.90|28.30|11.88|771.74|558.36|747.30|1330.10|24.84|61.41|585.15|496.95|521.79|1082.10|1106.94|-250.35| +2450839|29411|2450910|77900|1270228|7003|16978|77900|1270228|7003|16978|2|37|16|5|2072|253|1251|70|80.86|206.19|30.92|12268.90|2164.40|5660.20|14433.30|86.57|0.00|1442.70|2164.40|2250.97|3607.10|3693.67|-3495.80| +2450839|29411|2450895|77900|1270228|7003|16978|77900|1270228|7003|16978|2|13|10|5|17476|213|1251|70|91.50|274.50|161.95|7878.50|11336.50|6405.00|19215.00|23.80|10542.94|384.30|793.56|817.36|1177.86|1201.66|-5611.44| +2450839|29411|2450844|77900|1270228|7003|16978|77900|1270228|7003|16978|2|103|13|3|8672|218|1251|17|91.11|145.77|104.95|693.94|1784.15|1548.87|2478.09|0.00|0.00|1164.67|1784.15|1784.15|2948.82|2948.82|235.28| +2450839|29411|2450877|77900|1270228|7003|16978|77900|1270228|7003|16978|2|92|5|2|11174|59|1251|71|16.20|29.80|0.29|2095.21|20.59|1150.20|2115.80|0.82|0.00|401.86|20.59|21.41|422.45|423.27|-1129.61| +2450839|29411|2450883|77900|1270228|7003|16978|77900|1270228|7003|16978|2|14|7|1|1093|244|1251|82|62.19|139.30|0.00|11422.60|0.00|5099.58|11422.60|0.00|0.00|2055.74|0.00|0.00|2055.74|2055.74|-5099.58| +2450839|29411|2450863|77900|1270228|7003|16978|77900|1270228|7003|16978|2|17|10|1|5389|28|1251|83|38.75|49.21|7.38|3471.89|612.54|3216.25|4084.43|55.12|0.00|122.01|612.54|667.66|734.55|789.67|-2603.71| +2450839|82254|2450921|42755|424587|2772|19404|42755|424587|2772|19404|1|89|15|1|12986|75|1252|83|50.94|133.46|24.02|9083.52|1993.66|4228.02|11077.18|0.00|937.02|3765.71|1056.64|1056.64|4822.35|4822.35|-3171.38| +2450839|82254|2450923|42755|424587|2772|19404|42755|424587|2772|19404|1|60|6|1|4819|167|1252|34|32.67|65.99|58.07|269.28|1974.38|1110.78|2243.66|78.97|0.00|179.18|1974.38|2053.35|2153.56|2232.53|863.60| +2450839|82254|2450917|42755|424587|2772|19404|42755|424587|2772|19404|1|97|1|2|9418|79|1252|67|88.18|171.95|156.47|1037.16|10483.49|5908.06|11520.65|629.00|0.00|3340.62|10483.49|11112.49|13824.11|14453.11|4575.43| +2450839|82254|2450879|42755|424587|2772|19404|42755|424587|2772|19404|1|106|3|1|16870|214|1252|55|16.72|29.76|5.65|1326.05|310.75|919.60|1636.80|3.10|0.00|179.85|310.75|313.85|490.60|493.70|-608.85| +2450839|82254|2450873|42755|424587|2772|19404|42755|424587|2772|19404|1|6|10|3|1255|154|1252|59|67.76|198.53|87.35|6559.62|5153.65|3997.84|11713.27|257.68|0.00|3747.68|5153.65|5411.33|8901.33|9159.01|1155.81| +2450839|36071|2450910|31729|865673|6817|7561|31729|865673|6817|7561|4|65|3|4|328|286|1253|49|50.24|67.32|2.01|3200.19|98.49|2461.76|3298.68|5.90|0.00|461.58|98.49|104.39|560.07|565.97|-2363.27| +2450839|36071|2450914|31729|865673|6817|7561|31729|865673|6817|7561|4|51|14|1|5989|247|1253|14|59.61|148.42|89.05|831.18|1246.70|834.54|2077.88|49.86|0.00|436.24|1246.70|1296.56|1682.94|1732.80|412.16| +2450839|36071|2450905|31729|865673|6817|7561|31729|865673|6817|7561|4|32|16|3|4723|4|1253|82|86.00|174.58|90.78|6871.60|7443.96|7052.00|14315.56|595.51|0.00|3292.30|7443.96|8039.47|10736.26|11331.77|391.96| +2450839|36071|2450921|31729|865673|6817|7561|31729|865673|6817|7561|4|25|20|1|8461|206|1253|25|91.90|247.21|96.41|3770.00|2410.25|2297.50|6180.25|192.82|0.00|2842.75|2410.25|2603.07|5253.00|5445.82|112.75| +2450839|36071|2450865|31729|865673|6817|7561|31729|865673|6817|7561|4|10|19|5|17788|289|1253|69|43.67|98.25|1.96|6644.01|135.24|3013.23|6779.25|5.40|0.00|1084.68|135.24|140.64|1219.92|1225.32|-2877.99| +2450839|36071|2450913|31729|865673|6817|7561|31729|865673|6817|7561|4|81|20|2|4603|128|1253|68|20.15|56.42|29.90|1803.36|2033.20|1370.20|3836.56|20.33|0.00|191.76|2033.20|2053.53|2224.96|2245.29|663.00| +2450839|36071|2450914|31729|865673|6817|7561|31729|865673|6817|7561|4|14|10|4|6799|137|1253|76|49.80|97.60|95.64|148.96|7268.64|3784.80|7417.60|436.11|0.00|148.20|7268.64|7704.75|7416.84|7852.95|3483.84| +2450839|36071|2450911|31729|865673|6817|7561|31729|865673|6817|7561|4|50|10|5|6836|12|1253|22|51.69|103.89|72.72|685.74|1599.84|1137.18|2285.58|47.99|0.00|45.54|1599.84|1647.83|1645.38|1693.37|462.66| +2450839|36071|2450903|31729|865673|6817|7561|31729|865673|6817|7561|4|93|2|3|8941|78|1253|63|80.78|161.56|19.38|8957.34|1220.94|5089.14|10178.28|48.83|0.00|1220.94|1220.94|1269.77|2441.88|2490.71|-3868.20| +2450839|36071|2450889|31729|865673|6817|7561|31729|865673|6817|7561|4|83|19|5|9554|110|1253|29|16.29|36.16|29.28|199.52|849.12|472.41|1048.64|76.42|0.00|20.88|849.12|925.54|870.00|946.42|376.71| +2450839|36071|2450860|31729|865673|6817|7561|31729|865673|6817|7561|4|45|4|3|7957|119|1253|84|5.59|7.65|5.96|141.96|500.64|469.56|642.60|20.02|0.00|166.32|500.64|520.66|666.96|686.98|31.08| +2450839|36071|2450896|31729|865673|6817|7561|31729|865673|6817|7561|4|66|6|1|4801|144|1253|27|68.79|77.73|17.10|1637.01|461.70|1857.33|2098.71|41.55|0.00|1007.37|461.70|503.25|1469.07|1510.62|-1395.63| +2450839|41897|2450892|13402|1531474|2346|18818|13402|1531474|2346|18818|4|87|9|4|7918|292|1254|47|34.08|56.23|46.67|449.32|2193.49|1601.76|2642.81|12.94|899.33|422.53|1294.16|1307.10|1716.69|1729.63|-307.60| +2450839|41897|2450878|13402|1531474|2346|18818|13402|1531474|2346|18818|4|55|8|3|8680|66|1254|64|84.80|210.30|170.34|2557.44|10901.76|5427.20|13459.20|327.05|0.00|5921.92|10901.76|11228.81|16823.68|17150.73|5474.56| +2450839|41897|2450869|13402|1531474|2346|18818|13402|1531474|2346|18818|4|73|14|4|16948|85|1254|48|28.30|33.39|23.37|480.96|1121.76|1358.40|1602.72|11.21|0.00|144.00|1121.76|1132.97|1265.76|1276.97|-236.64| +2450839|41897|2450847|13402|1531474|2346|18818|13402|1531474|2346|18818|4|68|18|5|11072|269|1254|27|84.03|134.44|106.20|762.48|2867.40|2268.81|3629.88|28.67|0.00|254.07|2867.40|2896.07|3121.47|3150.14|598.59| +2450839|41897|2450920|13402|1531474|2346|18818|13402|1531474|2346|18818|4|99|11|1|17830|122|1254|17|23.68|32.67|30.38|38.93|516.46|402.56|555.39|30.98|0.00|5.44|516.46|547.44|521.90|552.88|113.90| +2450839|41897|2450902|13402|1531474|2346|18818|13402|1531474|2346|18818|4|19|8|2|3079|295|1254|95|70.83|169.99|81.59|8398.00|7751.05|6728.85|16149.05|390.65|3410.46|968.05|4340.59|4731.24|5308.64|5699.29|-2388.26| +2450839|41897|2450895|13402|1531474|2346|18818|13402|1531474|2346|18818|4|87|10|3|2792|264|1254|61|11.84|33.15|18.89|869.86|1152.29|722.24|2022.15|23.04|0.00|363.56|1152.29|1175.33|1515.85|1538.89|430.05| +2450839|41897|2450928|13402|1531474|2346|18818|13402|1531474|2346|18818|4|28|11|5|16124|16|1254|67|55.32|108.42|27.10|5448.44|1815.70|3706.44|7264.14|0.00|0.00|3413.65|1815.70|1815.70|5229.35|5229.35|-1890.74| +2450839|41897|2450867|13402|1531474|2346|18818|13402|1531474|2346|18818|4|42|7|1|10418|148|1254|11|37.86|91.24|49.26|461.78|541.86|416.46|1003.64|32.51|0.00|270.93|541.86|574.37|812.79|845.30|125.40| +2450839|41897|2450905|13402|1531474|2346|18818|13402|1531474|2346|18818|4|33|13|3|1435|200|1254|35|33.99|56.76|17.59|1370.95|615.65|1189.65|1986.60|49.25|0.00|297.85|615.65|664.90|913.50|962.75|-574.00| +2450839|74771|2450850|9998|1717443|3735|49580|9998|1717443|3735|49580|1|21|2|4|13544|38|1255|18|81.35|104.94|78.70|472.32|1416.60|1464.30|1888.92|8.92|1317.43|585.54|99.17|108.09|684.71|693.63|-1365.13| +2450839|74771|2450916|9998|1717443|3735|49580|9998|1717443|3735|49580|1|94|5|3|11263|181|1255|96|4.87|14.61|13.73|84.48|1318.08|467.52|1402.56|92.26|0.00|41.28|1318.08|1410.34|1359.36|1451.62|850.56| +2450839|74771|2450906|9998|1717443|3735|49580|9998|1717443|3735|49580|1|91|4|5|1060|262|1255|15|4.59|6.10|6.10|0.00|91.50|68.85|91.50|8.23|0.00|15.45|91.50|99.73|106.95|115.18|22.65| +2450839|74771|2450914|9998|1717443|3735|49580|9998|1717443|3735|49580|1|97|3|4|15037|75|1255|14|42.24|96.72|40.62|785.40|568.68|591.36|1354.08|5.68|0.00|555.10|568.68|574.36|1123.78|1129.46|-22.68| +2450839|74771|2450893|9998|1717443|3735|49580|9998|1717443|3735|49580|1|61|7|1|6392|157|1255|29|79.54|209.19|121.33|2547.94|3518.57|2306.66|6066.51|281.48|0.00|2729.77|3518.57|3800.05|6248.34|6529.82|1211.91| +2450839|74771|2450914|9998|1717443|3735|49580|9998|1717443|3735|49580|1|39|14|3|11632|286|1255|62|38.12|44.60|17.39|1687.02|1078.18|2363.44|2765.20|10.78|0.00|1106.08|1078.18|1088.96|2184.26|2195.04|-1285.26| +2450839|74771|2450842|9998|1717443|3735|49580|9998|1717443|3735|49580|1|27|15|2|2638|143|1255|74|38.65|82.71|58.72|1775.26|4345.28|2860.10|6120.54|43.45|0.00|1774.52|4345.28|4388.73|6119.80|6163.25|1485.18| +2450839|44672|2450920|18733|1702429|4878|21727|18733|1702429|4878|21727|1|94|6|2|5455|6|1256|60|90.65|108.78|100.07|522.60|6004.20|5439.00|6526.80|395.07|360.25|1631.40|5643.95|6039.02|7275.35|7670.42|204.95| +2450839|44672|2450927|18733|1702429|4878|21727|18733|1702429|4878|21727|1|17|11|4|17366|206|1256|6|39.42|87.11|63.59|141.12|381.54|236.52|522.66|30.52|0.00|219.48|381.54|412.06|601.02|631.54|145.02| +2450839|44672|2450914|18733|1702429|4878|21727|18733|1702429|4878|21727|1|31|4|4|1651|212|1256|65|26.24|27.55|22.04|358.15|1432.60|1705.60|1790.75|0.00|0.00|393.90|1432.60|1432.60|1826.50|1826.50|-273.00| +2450839|44672|2450928|18733|1702429|4878|21727|18733|1702429|4878|21727|1|86|16|1|4519|97|1256|70|33.93|38.68|22.43|1137.50|1570.10|2375.10|2707.60|43.96|1020.56|216.30|549.54|593.50|765.84|809.80|-1825.56| +2450839|44672|2450842|18733|1702429|4878|21727|18733|1702429|4878|21727|1|70|16|5|10009|132|1256|52|89.09|204.90|127.03|4049.24|6605.56|4632.68|10654.80|0.00|0.00|3835.52|6605.56|6605.56|10441.08|10441.08|1972.88| +2450839|44672|2450897|18733|1702429|4878|21727|18733|1702429|4878|21727|1|94|18|5|13984|112|1256|41|39.93|69.47|63.91|227.96|2620.31|1637.13|2848.27|52.40|0.00|882.73|2620.31|2672.71|3503.04|3555.44|983.18| +2450839|44672|2450901|18733|1702429|4878|21727|18733|1702429|4878|21727|1|106|18|4|6130|91|1256|92|90.71|180.51|111.91|6311.20|10295.72|8345.32|16606.92|574.50|3912.37|3985.44|6383.35|6957.85|10368.79|10943.29|-1961.97| +2450839|25465|2450890|25596|923164|379|21994|25596|923164|379|21994|4|67|9|4|4237|179|1257|76|77.29|197.86|168.18|2255.68|12781.68|5874.04|15037.36|0.00|0.00|2856.84|12781.68|12781.68|15638.52|15638.52|6907.64| +2450839|25465|2450845|25596|923164|379|21994|25596|923164|379|21994|4|103|14|5|7621|22|1257|59|44.29|121.35|93.43|1647.28|5512.37|2613.11|7159.65|440.98|0.00|2362.36|5512.37|5953.35|7874.73|8315.71|2899.26| +2450839|25465|2450852|25596|923164|379|21994|25596|923164|379|21994|4|40|11|3|7051|97|1257|39|27.53|64.69|56.92|303.03|2219.88|1073.67|2522.91|110.99|0.00|151.32|2219.88|2330.87|2371.20|2482.19|1146.21| +2450839|25465|2450929|25596|923164|379|21994|25596|923164|379|21994|4|47|7|4|3160|85|1257|2|95.86|193.63|36.78|313.70|73.56|191.72|387.26|3.67|0.00|27.10|73.56|77.23|100.66|104.33|-118.16| +2450839|25465|2450923|25596|923164|379|21994|25596|923164|379|21994|4|11|10|3|1072|256|1257|90|65.80|102.64|75.95|2402.10|6835.50|5922.00|9237.60|478.48|0.00|2216.70|6835.50|7313.98|9052.20|9530.68|913.50| +2450839|25465|2450929|25596|923164|379|21994|25596|923164|379|21994|4|61|20|5|16276|143|1257|27|69.74|131.80|31.63|2704.59|854.01|1882.98|3558.60|17.08|0.00|1708.02|854.01|871.09|2562.03|2579.11|-1028.97| +2450839|25465|2450895|25596|923164|379|21994|25596|923164|379|21994|4|24|16|2|10864|264|1257|81|18.07|39.03|17.17|1770.66|1390.77|1463.67|3161.43|27.81|0.00|1042.47|1390.77|1418.58|2433.24|2461.05|-72.90| +2450839|25465|2450907|25596|923164|379|21994|25596|923164|379|21994|4|9|17|4|13471|238|1257|24|20.94|44.60|43.26|32.16|1038.24|502.56|1070.40|93.44|0.00|192.48|1038.24|1131.68|1230.72|1324.16|535.68| +2450839|25465|2450872|25596|923164|379|21994|25596|923164|379|21994|4|101|2|3|788|75|1257|84|39.09|99.28|12.90|7255.92|1083.60|3283.56|8339.52|54.18|0.00|3085.32|1083.60|1137.78|4168.92|4223.10|-2199.96| +2450839|25465|2450905|25596|923164|379|21994|25596|923164|379|21994|4|68|1|5|1540|294|1257|39|33.31|81.27|46.32|1363.05|1806.48|1299.09|3169.53|36.12|0.00|982.41|1806.48|1842.60|2788.89|2825.01|507.39| +2450839|25465|2450887|25596|923164|379|21994|25596|923164|379|21994|4|77|3|4|8150|221|1257|44|74.46|106.47|5.32|4450.60|234.08|3276.24|4684.68|21.06|0.00|1358.28|234.08|255.14|1592.36|1613.42|-3042.16| +2450839|11040|2450906|10808|110058|508|42958|10808|110058|508|42958|2|6|5|2|12142|57|1258|57|57.37|118.75|84.31|1963.08|4805.67|3270.09|6768.75|192.22|0.00|1895.25|4805.67|4997.89|6700.92|6893.14|1535.58| +2450839|11040|2450847|10808|110058|508|42958|10808|110058|508|42958|2|18|4|5|13237|127|1258|10|96.51|103.26|103.26|0.00|1032.60|965.10|1032.60|53.90|433.69|361.40|598.91|652.81|960.31|1014.21|-366.19| +2450839|11040|2450918|10808|110058|508|42958|10808|110058|508|42958|2|103|6|1|13501|146|1258|58|79.09|186.65|50.39|7903.08|2922.62|4587.22|10825.70|233.80|0.00|3138.96|2922.62|3156.42|6061.58|6295.38|-1664.60| +2450839|11040|2450885|10808|110058|508|42958|10808|110058|508|42958|2|47|12|5|8608|184|1258|21|13.10|38.25|21.03|361.62|441.63|275.10|803.25|39.74|0.00|160.65|441.63|481.37|602.28|642.02|166.53| +2450839|11040|2450897|10808|110058|508|42958|10808|110058|508|42958|2|16|14|2|7324|297|1258|10|31.11|34.22|8.89|253.30|88.90|311.10|342.20|0.88|0.00|92.30|88.90|89.78|181.20|182.08|-222.20| +2450839|11040|2450928|10808|110058|508|42958|10808|110058|508|42958|2|97|9|3|8887|276|1258|94|6.13|13.42|8.32|479.40|782.08|576.22|1261.48|10.79|242.44|175.78|539.64|550.43|715.42|726.21|-36.58| +2450839|11040|2450908|10808|110058|508|42958|10808|110058|508|42958|2|70|20|4|14512|162|1258|20|98.02|250.93|160.59|1806.80|3211.80|1960.40|5018.60|64.23|0.00|2258.20|3211.80|3276.03|5470.00|5534.23|1251.40| +2450839|11040|2450854|10808|110058|508|42958|10808|110058|508|42958|2|44|14|2|15352|216|1258|96|74.73|150.95|134.34|1594.56|12896.64|7174.08|14491.20|773.79|0.00|5940.48|12896.64|13670.43|18837.12|19610.91|5722.56| +2450839|5811|2450873|31356|1604486|4016|1946|31356|1604486|4016|1946|1|4|19|3|15260|293|1259|94|1.88|1.97|0.61|127.84|57.34|176.72|185.18|0.46|41.85|53.58|15.49|15.95|69.07|69.53|-161.23| +2450839|5811|2450899|31356|1604486|4016|1946|31356|1604486|4016|1946|1|66|17|3|14587|113|1259|66|74.85|74.85|65.86|593.34|4346.76|4940.10|4940.10|304.27|0.00|147.84|4346.76|4651.03|4494.60|4798.87|-593.34| +2450839|5811|2450925|31356|1604486|4016|1946|31356|1604486|4016|1946|1|74|12|4|15763|250|1259|34|95.76|243.23|150.80|3142.62|5127.20|3255.84|8269.82|119.97|3127.59|3721.30|1999.61|2119.58|5720.91|5840.88|-1256.23| +2450839|5811|2450875|31356|1604486|4016|1946|31356|1604486|4016|1946|1|102|15|5|7198|290|1259|73|51.70|140.62|64.68|5543.62|4721.64|3774.10|10265.26|330.51|0.00|513.19|4721.64|5052.15|5234.83|5565.34|947.54| +2450839|5811|2450913|31356|1604486|4016|1946|31356|1604486|4016|1946|1|61|19|5|8918|255|1259|11|30.59|65.46|47.13|201.63|518.43|336.49|720.06|3.88|440.66|323.95|77.77|81.65|401.72|405.60|-258.72| +2450839|5811|2450919|31356|1604486|4016|1946|31356|1604486|4016|1946|1|9|6|2|16594|281|1259|89|71.40|182.78|177.29|488.61|15778.81|6354.60|16267.42|1262.30|0.00|4879.87|15778.81|17041.11|20658.68|21920.98|9424.21| +2450839|5811|2450865|31356|1604486|4016|1946|31356|1604486|4016|1946|1|107|6|5|16042|148|1259|35|98.42|164.36|46.02|4141.90|1610.70|3444.70|5752.60|0.00|0.00|1092.70|1610.70|1610.70|2703.40|2703.40|-1834.00| +2450839|5811|2450856|31356|1604486|4016|1946|31356|1604486|4016|1946|1|83|7|3|4324|170|1259|27|76.27|80.84|12.93|1833.57|349.11|2059.29|2182.68|3.49|0.00|1025.73|349.11|352.60|1374.84|1378.33|-1710.18| +2450839|5811|2450917|31356|1604486|4016|1946|31356|1604486|4016|1946|1|10|6|2|17302|89|1259|57|57.35|84.87|24.61|3434.82|1402.77|3268.95|4837.59|112.22|0.00|96.33|1402.77|1514.99|1499.10|1611.32|-1866.18| +2450839|5811|2450845|31356|1604486|4016|1946|31356|1604486|4016|1946|1|52|3|4|16394|226|1259|53|17.75|49.70|43.73|316.41|2317.69|940.75|2634.10|139.06|0.00|869.20|2317.69|2456.75|3186.89|3325.95|1376.94| +2450839|5811|2450884|31356|1604486|4016|1946|31356|1604486|4016|1946|1|31|7|2|10609|110|1259|47|84.69|95.69|92.81|135.36|4362.07|3980.43|4497.43|348.96|0.00|2068.47|4362.07|4711.03|6430.54|6779.50|381.64| +2450839|5811|2450853|31356|1604486|4016|1946|31356|1604486|4016|1946|1|89|5|4|14170|35|1259|25|26.10|75.42|21.87|1338.75|546.75|652.50|1885.50|21.87|0.00|433.50|546.75|568.62|980.25|1002.12|-105.75| +2450839|58971|2450882|35262|690840|6771|37949|35262|690840|6771|37949|4|16|18|1|3721|249|1260|82|45.49|74.60|51.47|1896.66|4220.54|3730.18|6117.20|337.64|0.00|1284.12|4220.54|4558.18|5504.66|5842.30|490.36| +2450839|58971|2450875|35262|690840|6771|37949|35262|690840|6771|37949|4|81|4|2|9098|3|1260|6|16.25|40.78|33.43|44.10|200.58|97.50|244.68|4.01|0.00|102.72|200.58|204.59|303.30|307.31|103.08| +2450839|58971|2450907|35262|690840|6771|37949|35262|690840|6771|37949|4|106|7|1|16736|239|1260|10|19.57|42.27|17.33|249.40|173.30|195.70|422.70|10.39|0.00|194.40|173.30|183.69|367.70|378.09|-22.40| +2450839|58971|2450855|35262|690840|6771|37949|35262|690840|6771|37949|4|104|20|5|13777|211|1260|15|8.31|13.87|6.65|108.30|99.75|124.65|208.05|1.46|50.87|47.85|48.88|50.34|96.73|98.19|-75.77| +2450839|58971|2450849|35262|690840|6771|37949|35262|690840|6771|37949|4|107|10|1|928|96|1260|13|36.57|91.05|60.09|402.48|781.17|475.41|1183.65|54.68|0.00|153.79|781.17|835.85|934.96|989.64|305.76| +2450839|58971|2450879|35262|690840|6771|37949|35262|690840|6771|37949|4|21|5|3|17072|264|1260|47|97.69|262.78|115.62|6916.52|5434.14|4591.43|12350.66|108.68|0.00|4692.95|5434.14|5542.82|10127.09|10235.77|842.71| +2450839|58971|2450915|35262|690840|6771|37949|35262|690840|6771|37949|4|26|4|4|14221|299|1260|13|56.38|157.30|31.46|1635.92|408.98|732.94|2044.90|8.17|0.00|102.18|408.98|417.15|511.16|519.33|-323.96| +2450839|58971|2450879|35262|690840|6771|37949|35262|690840|6771|37949|4|49|20|2|16069|11|1260|69|48.08|132.22|33.05|6842.73|2280.45|3317.52|9123.18|114.02|0.00|638.25|2280.45|2394.47|2918.70|3032.72|-1037.07| +2450839|58971|2450866|35262|690840|6771|37949|35262|690840|6771|37949|4|97|16|5|5473|236|1260|9|73.02|193.50|46.44|1323.54|417.96|657.18|1741.50|33.43|0.00|278.64|417.96|451.39|696.60|730.03|-239.22| +2450839|58971|2450858|35262|690840|6771|37949|35262|690840|6771|37949|4|9|5|1|16657|281|1260|61|7.84|23.20|12.06|679.54|735.66|478.24|1415.20|44.13|0.00|311.10|735.66|779.79|1046.76|1090.89|257.42| +2450839|58971|2450845|35262|690840|6771|37949|35262|690840|6771|37949|4|34|2|5|17020|274|1260|70|3.50|5.53|2.26|228.90|158.20|245.00|387.10|1.01|132.88|154.70|25.32|26.33|180.02|181.03|-219.68| +2450839|64109|2450926|24735|669510|685|1460|66083|940610|5385|3760|4|77|17|4|8041|236|1261|66|48.37|82.71|72.78|655.38|4803.48|3192.42|5458.86|30.26|4371.16|0.00|432.32|462.58|432.32|462.58|-2760.10| +2450839|64109|2450901|24735|669510|685|1460|66083|940610|5385|3760|4|89|9|3|5647|48|1261|26|99.21|105.16|58.88|1203.28|1530.88|2579.46|2734.16|0.00|0.00|437.32|1530.88|1530.88|1968.20|1968.20|-1048.58| +2450839|64109|2450929|24735|669510|685|1460|66083|940610|5385|3760|4|105|3|3|970|58|1261|33|80.19|100.23|80.18|661.65|2645.94|2646.27|3307.59|26.45|0.00|1421.97|2645.94|2672.39|4067.91|4094.36|-0.33| +2450839|64109|2450882|24735|669510|685|1460|66083|940610|5385|3760|4|7|11|1|1346|143|1261|66|41.22|110.46|48.60|4082.76|3207.60|2720.52|7290.36|256.60|0.00|2040.72|3207.60|3464.20|5248.32|5504.92|487.08| +2450839|64109|2450889|24735|669510|685|1460|66083|940610|5385|3760|4|49|8|2|818|200|1261|100|70.14|196.39|51.06|14533.00|5106.00|7014.00|19639.00|51.06|0.00|4909.00|5106.00|5157.06|10015.00|10066.06|-1908.00| +2450839|64109|2450927|24735|669510|685|1460|66083|940610|5385|3760|4|74|10|2|11092|232|1261|9|78.96|98.70|4.93|843.93|44.37|710.64|888.30|0.18|38.15|195.39|6.22|6.40|201.61|201.79|-704.42| +2450839|64109|2450858|24735|669510|685|1460|66083|940610|5385|3760|4|68|14|1|9487|49|1261|65|70.99|123.52|25.93|6343.35|1685.45|4614.35|8028.80|9.43|741.59|561.60|943.86|953.29|1505.46|1514.89|-3670.49| +2450839|64109|2450903|24735|669510|685|1460|66083|940610|5385|3760|4|90|2|5|13561|126|1261|74|43.28|49.77|22.39|2026.12|1656.86|3202.72|3682.98|60.47|447.35|589.04|1209.51|1269.98|1798.55|1859.02|-1993.21| +2450839|64109|2450878|24735|669510|685|1460|66083|940610|5385|3760|4|34|19|5|17659|211|1261|60|65.83|193.54|11.61|10915.80|696.60|3949.80|11612.40|34.83|0.00|4180.20|696.60|731.43|4876.80|4911.63|-3253.20| +2450839|62183|2450915|14322|116330|1538|12346|14322|116330|1538|12346|4|29|1|5|10|228|1262|49|60.15|70.97|25.54|2226.07|1251.46|2947.35|3477.53|87.60|0.00|556.15|1251.46|1339.06|1807.61|1895.21|-1695.89| +2450839|62183|2450907|14322|116330|1538|12346|14322|116330|1538|12346|4|64|4|3|12550|243|1262|35|79.78|169.93|125.74|1546.65|4400.90|2792.30|5947.55|88.01|0.00|2378.95|4400.90|4488.91|6779.85|6867.86|1608.60| +2450839|62183|2450851|14322|116330|1538|12346|14322|116330|1538|12346|4|27|20|1|3026|13|1262|6|38.43|91.84|26.63|391.26|159.78|230.58|551.04|4.79|0.00|5.46|159.78|164.57|165.24|170.03|-70.80| +2450839|62183|2450896|14322|116330|1538|12346|14322|116330|1538|12346|4|95|17|1|8191|68|1262|30|91.87|180.98|173.74|217.20|5212.20|2756.10|5429.40|208.48|0.00|597.00|5212.20|5420.68|5809.20|6017.68|2456.10| +2450839|62183|2450864|14322|116330|1538|12346|14322|116330|1538|12346|4|38|17|1|16844|291|1262|79|98.13|269.85|116.03|12151.78|9166.37|7752.27|21318.15|733.30|0.00|639.11|9166.37|9899.67|9805.48|10538.78|1414.10| +2450839|4283|2450856|41614|1401624|5987|15197|41614|1401624|5987|15197|1|90|18|5|5641|104|1263|79|52.21|81.96|77.86|323.90|6150.94|4124.59|6474.84|123.01|0.00|841.35|6150.94|6273.95|6992.29|7115.30|2026.35| +2450839|4283|2450898|41614|1401624|5987|15197|41614|1401624|5987|15197|1|5|14|5|8086|159|1263|89|29.58|81.04|43.76|3317.92|3894.64|2632.62|7212.56|350.51|0.00|3461.21|3894.64|4245.15|7355.85|7706.36|1262.02| +2450839|4283|2450887|41614|1401624|5987|15197|41614|1401624|5987|15197|1|55|17|5|7876|39|1263|37|41.01|80.78|13.73|2480.85|508.01|1517.37|2988.86|25.40|0.00|209.05|508.01|533.41|717.06|742.46|-1009.36| +2450839|4283|2450854|41614|1401624|5987|15197|41614|1401624|5987|15197|1|89|7|2|2630|82|1263|43|19.02|22.25|21.80|19.35|937.40|817.86|956.75|74.99|0.00|9.46|937.40|1012.39|946.86|1021.85|119.54| +2450839|4283|2450910|41614|1401624|5987|15197|41614|1401624|5987|15197|1|36|2|5|10231|46|1263|72|91.01|102.84|76.10|1925.28|5479.20|6552.72|7404.48|219.16|0.00|961.92|5479.20|5698.36|6441.12|6660.28|-1073.52| +2450839|4283|2450879|41614|1401624|5987|15197|41614|1401624|5987|15197|1|12|14|1|2738|237|1263|4|21.71|50.80|10.16|162.56|40.64|86.84|203.20|0.00|0.00|18.28|40.64|40.64|58.92|58.92|-46.20| +2450839|4283|2450864|41614|1401624|5987|15197|41614|1401624|5987|15197|1|7|6|1|8662|101|1263|32|44.95|119.11|27.39|2935.04|876.48|1438.40|3811.52|43.82|0.00|495.36|876.48|920.30|1371.84|1415.66|-561.92| +2450839|4283|2450852|41614|1401624|5987|15197|41614|1401624|5987|15197|1|41|5|5|4735|178|1263|40|49.59|93.72|7.49|3449.20|299.60|1983.60|3748.80|0.00|0.00|1424.40|299.60|299.60|1724.00|1724.00|-1684.00| +2450839|4283|2450860|41614|1401624|5987|15197|41614|1401624|5987|15197|1|73|5|1|13159|213|1263|2|89.20|224.78|143.85|161.86|287.70|178.40|449.56|6.32|161.11|85.40|126.59|132.91|211.99|218.31|-51.81| +2450839|4283|2450846|41614|1401624|5987|15197|41614|1401624|5987|15197|1|8|17|1|5650|44|1263|14|38.73|72.03|14.40|806.82|201.60|542.22|1008.42|1.18|82.65|504.14|118.95|120.13|623.09|624.27|-423.27| +2450839|4283|2450903|41614|1401624|5987|15197|41614|1401624|5987|15197|1|83|20|5|2452|18|1263|36|84.76|138.15|138.15|0.00|4973.40|3051.36|4973.40|348.13|0.00|1839.96|4973.40|5321.53|6813.36|7161.49|1922.04| +2450839|4283|2450867|41614|1401624|5987|15197|41614|1401624|5987|15197|1|87|20|5|13231|215|1263|59|59.58|109.62|67.96|2457.94|4009.64|3515.22|6467.58|40.09|0.00|1422.49|4009.64|4049.73|5432.13|5472.22|494.42| +2450839|78326|2450841|85612|334651|3337|1176|85612|334651|3337|1176|1|15|14|3|13327|98|1264|8|69.89|152.36|68.56|670.40|548.48|559.12|1218.88|38.39|0.00|450.96|548.48|586.87|999.44|1037.83|-10.64| +2450839|78326|2450850|85612|334651|3337|1176|85612|334651|3337|1176|1|9|7|4|14413|125|1264|72|22.95|55.76|50.18|401.76|3612.96|1652.40|4014.72|108.38|0.00|1043.28|3612.96|3721.34|4656.24|4764.62|1960.56| +2450839|78326|2450909|85612|334651|3337|1176|85612|334651|3337|1176|1|29|17|1|15223|203|1264|66|59.55|88.13|36.13|3432.00|2384.58|3930.30|5816.58|95.38|0.00|2384.58|2384.58|2479.96|4769.16|4864.54|-1545.72| +2450839|78326|2450894|85612|334651|3337|1176|85612|334651|3337|1176|1|36|19|5|3481|132|1264|42|72.56|156.72|45.44|4673.76|1908.48|3047.52|6582.24|0.00|0.00|2829.96|1908.48|1908.48|4738.44|4738.44|-1139.04| +2450839|75806|2450924|49526|700262|3798|20134|49526|700262|3798|20134|4|34|12|5|4333|249|1265|3|58.88|163.09|22.83|420.78|68.49|176.64|489.27|0.68|0.00|132.09|68.49|69.17|200.58|201.26|-108.15| +2450839|75806|2450873|49526|700262|3798|20134|49526|700262|3798|20134|4|20|18|5|13786|2|1265|38|85.69|178.23|156.84|812.82|5959.92|3256.22|6772.74|417.19|0.00|1489.98|5959.92|6377.11|7449.90|7867.09|2703.70| +2450839|75806|2450899|49526|700262|3798|20134|49526|700262|3798|20134|4|68|18|3|1303|178|1265|5|68.83|135.59|66.43|345.80|332.15|344.15|677.95|29.89|0.00|20.30|332.15|362.04|352.45|382.34|-12.00| +2450839|75806|2450875|49526|700262|3798|20134|49526|700262|3798|20134|4|37|12|4|9124|85|1265|34|20.69|51.93|19.73|1094.80|670.82|703.46|1765.62|20.12|0.00|459.00|670.82|690.94|1129.82|1149.94|-32.64| +2450839|75806|2450887|49526|700262|3798|20134|49526|700262|3798|20134|4|65|4|2|12472|101|1265|62|25.52|59.46|32.70|1659.12|2027.40|1582.24|3686.52|141.91|0.00|257.92|2027.40|2169.31|2285.32|2427.23|445.16| +2450839|75806|2450919|49526|700262|3798|20134|49526|700262|3798|20134|4|21|17|1|8180|282|1265|28|62.03|132.12|19.81|3144.68|554.68|1736.84|3699.36|44.37|0.00|1294.72|554.68|599.05|1849.40|1893.77|-1182.16| +2450839|75806|2450876|49526|700262|3798|20134|49526|700262|3798|20134|4|35|4|3|5534|46|1265|85|89.95|100.74|90.66|856.80|7706.10|7645.75|8562.90|231.18|0.00|1026.80|7706.10|7937.28|8732.90|8964.08|60.35| +2450839|75806|2450897|49526|700262|3798|20134|49526|700262|3798|20134|4|27|6|2|14672|101|1265|35|13.53|24.08|10.35|480.55|362.25|473.55|842.80|10.86|0.00|75.60|362.25|373.11|437.85|448.71|-111.30| +2450839|75806|2450869|49526|700262|3798|20134|49526|700262|3798|20134|4|80|13|3|14818|115|1265|46|62.19|116.91|79.49|1721.32|3656.54|2860.74|5377.86|146.26|0.00|1183.12|3656.54|3802.80|4839.66|4985.92|795.80| +2450839|75806|2450855|49526|700262|3798|20134|49526|700262|3798|20134|4|81|5|5|13426|54|1265|44|59.03|118.06|4.72|4986.96|207.68|2597.32|5194.64|5.23|149.52|259.60|58.16|63.39|317.76|322.99|-2539.16| +2450839|75806|2450927|49526|700262|3798|20134|49526|700262|3798|20134|4|61|15|2|6061|139|1265|8|59.10|130.02|109.21|166.48|873.68|472.80|1040.16|78.63|0.00|249.60|873.68|952.31|1123.28|1201.91|400.88| +2450839|75806|2450880|49526|700262|3798|20134|49526|700262|3798|20134|4|18|8|1|17917|295|1265|58|41.87|123.51|87.69|2077.56|5086.02|2428.46|7163.58|152.58|0.00|3294.98|5086.02|5238.60|8381.00|8533.58|2657.56| +2450839|39092|2450900|73684|67357|5477|15150|73684|67357|5477|15150|1|47|20|1|1021|146|1266|97|85.27|162.86|9.77|14849.73|947.69|8271.19|15797.42|9.47|0.00|7108.16|947.69|957.16|8055.85|8065.32|-7323.50| +2450839|39092|2450879|73684|67357|5477|15150|73684|67357|5477|15150|1|66|15|2|7586|171|1266|41|49.35|89.32|73.24|659.28|3002.84|2023.35|3662.12|90.08|0.00|72.98|3002.84|3092.92|3075.82|3165.90|979.49| +2450839|39092|2450865|73684|67357|5477|15150|73684|67357|5477|15150|1|17|17|1|6940|16|1266|63|53.92|100.29|82.23|1137.78|5180.49|3396.96|6318.27|207.21|0.00|2463.93|5180.49|5387.70|7644.42|7851.63|1783.53| +2450839|39092|2450897|73684|67357|5477|15150|73684|67357|5477|15150|1|38|1|3|14497|106|1266|38|59.69|159.96|116.77|1641.22|4437.26|2268.22|6078.48|88.74|0.00|790.02|4437.26|4526.00|5227.28|5316.02|2169.04| +2450839|39092|2450912|73684|67357|5477|15150|73684|67357|5477|15150|1|61|8|4|508|9|1266|82|65.03|143.71|81.91|5067.60|6716.62|5332.46|11784.22|537.32|0.00|2356.68|6716.62|7253.94|9073.30|9610.62|1384.16| +2450839|39092|2450898|73684|67357|5477|15150|73684|67357|5477|15150|1|17|3|3|17776|249|1266|97|87.01|233.18|4.66|22166.44|452.02|8439.97|22618.46|40.68|0.00|9047.19|452.02|492.70|9499.21|9539.89|-7987.95| +2450839|39092|2450895|73684|67357|5477|15150|73684|67357|5477|15150|1|103|13|4|5281|41|1266|60|16.07|46.92|30.96|957.60|1857.60|964.20|2815.20|26.74|1560.38|112.20|297.22|323.96|409.42|436.16|-666.98| +2450839|39092|2450891|73684|67357|5477|15150|73684|67357|5477|15150|1|85|19|1|1064|10|1266|84|99.93|141.90|29.79|9417.24|2502.36|8394.12|11919.60|225.21|0.00|2383.92|2502.36|2727.57|4886.28|5111.49|-5891.76| +2450839|39092|2450893|73684|67357|5477|15150|73684|67357|5477|15150|1|104|7|1|16232|45|1266|21|66.53|144.37|49.08|2001.09|1030.68|1397.13|3031.77|92.76|0.00|1242.99|1030.68|1123.44|2273.67|2366.43|-366.45| +2450839|39092|2450877|73684|67357|5477|15150|73684|67357|5477|15150|1|79|19|3|2299|43|1266|29|62.80|178.35|103.44|2172.39|2999.76|1821.20|5172.15|209.98|0.00|1396.35|2999.76|3209.74|4396.11|4606.09|1178.56| +|39092|||67357||15150|73684|67357|5477|15150|1|90|9||16634|3|1266|15||136.59||2048.85|0.00|816.30|2048.85||0.00||0.00||163.80|163.80|-816.30| +2450839|39092|2450856|73684|67357|5477|15150|73684|67357|5477|15150|1|36|8|5|1999|127|1266|71|28.38|81.73|13.07|4874.86|927.97|2014.98|5802.83|46.39|0.00|1276.58|927.97|974.36|2204.55|2250.94|-1087.01| +2450839|39092|2450842|73684|67357|5477|15150|73684|67357|5477|15150|1|99|14|2|12889|153|1266|91|73.48|132.26|2.64|11795.42|240.24|6686.68|12035.66|7.20|0.00|5415.41|240.24|247.44|5655.65|5662.85|-6446.44| +2450839|39092|2450851|73684|67357|5477|15150|73684|67357|5477|15150|1|50|18|4|1702|261|1266|24|70.82|80.73|25.02|1337.04|600.48|1699.68|1937.52|23.29|18.01|522.96|582.47|605.76|1105.43|1128.72|-1117.21| +2450839|38843|2450887|21175|489957|4339|9921|21175|489957|4339|9921|1|73|2|1|1880|116|1267|34|5.60|7.67|2.30|182.58|78.20|190.40|260.78|1.45|53.95|44.20|24.25|25.70|68.45|69.90|-166.15| +2450839|38843|2450906|21175|489957|4339|9921|21175|489957|4339|9921|1|30|9|2|16360|285|1267|80|23.97|69.99|24.49|3640.00|1959.20|1917.60|5599.20|0.00|0.00|2407.20|1959.20|1959.20|4366.40|4366.40|41.60| +2450839|38843|2450859|21175|489957|4339|9921|21175|489957|4339|9921|1|78|16|5|15016|50|1267|100|18.04|45.10|40.59|451.00|4059.00|1804.00|4510.00|40.59|0.00|1804.00|4059.00|4099.59|5863.00|5903.59|2255.00| +2450839|38843|2450928|21175|489957|4339|9921|21175|489957|4339|9921|1|59|5|2|154|226|1267|90|95.48|217.69|21.76|17633.70|1958.40|8593.20|19592.10|156.67|0.00|6269.40|1958.40|2115.07|8227.80|8384.47|-6634.80| +2450839|38843|2450874|21175|489957|4339|9921|21175|489957|4339|9921|1|91|2|3|5029|95|1267|74|83.14|136.34|38.17|7264.58|2824.58|6152.36|10089.16|56.49|0.00|201.28|2824.58|2881.07|3025.86|3082.35|-3327.78| +2450839|38843|2450913|21175|489957|4339|9921|21175|489957|4339|9921|1|107|5|2|9085|68|1267|35|17.18|34.18|23.92|359.10|837.20|601.30|1196.30|66.97|0.00|406.70|837.20|904.17|1243.90|1310.87|235.90| +2450839|38843|2450915|21175|489957|4339|9921|21175|489957|4339|9921|1|57|19|4|13234|3|1267|56|23.22|68.96|28.27|2278.64|1583.12|1300.32|3861.76|22.79|1329.82|1467.20|253.30|276.09|1720.50|1743.29|-1047.02| +2450839|38843|2450911|21175|489957|4339|9921|21175|489957|4339|9921|1|53|20|2|367|219|1267|95|92.84|106.76|27.75|7505.95|2636.25|8819.80|10142.20|69.86|1239.03|5071.10|1397.22|1467.08|6468.32|6538.18|-7422.58| +2450839|38843|2450906|21175|489957|4339|9921|21175|489957|4339|9921|1|45|15|4|7190|155|1267|58|14.43|41.84|0.00|2426.72|0.00|836.94|2426.72|0.00|0.00|314.94|0.00|0.00|314.94|314.94|-836.94| +2450839|38843|2450880|21175|489957|4339|9921|21175|489957|4339|9921|1|83|3|1|3931|170|1267|57|94.67|107.92|103.60|246.24|5905.20|5396.19|6151.44|318.88|2362.08|922.26|3543.12|3862.00|4465.38|4784.26|-1853.07| +2450839|38843|2450880|21175|489957|4339|9921|21175|489957|4339|9921|1|107|14|5|13261|167|1267|16|24.38|48.76|5.36|694.40|85.76|390.08|780.16|6.00|0.00|132.48|85.76|91.76|218.24|224.24|-304.32| +2450839|38843|2450919|21175|489957|4339|9921|21175|489957|4339|9921|1|69|2|3|2863|247|1267|55|26.40|27.72|27.72|0.00|1524.60|1452.00|1524.60|37.81|1051.97|121.55|472.63|510.44|594.18|631.99|-979.37| +2450839|38843|2450912|21175|489957|4339|9921|21175|489957|4339|9921|1|23|19|2|10321|120|1267|35|77.30|194.79|52.59|4977.00|1840.65|2705.50|6817.65|92.03|0.00|340.55|1840.65|1932.68|2181.20|2273.23|-864.85| +2450839|38843|2450864|21175|489957|4339|9921|21175|489957|4339|9921|1|55|17|2|12256|224|1267|82|74.37|214.92|176.23|3172.58|14450.86|6098.34|17623.44|1156.06|0.00|3524.36|14450.86|15606.92|17975.22|19131.28|8352.52| +2450839|68755|2450868|55928|1498390|4052|30067|55928|1498390|4052|30067|4|72|8|2|7102|110|1268|4|2.86|4.20|3.90|1.20|15.60|11.44|16.80|1.09|0.00|5.88|15.60|16.69|21.48|22.57|4.16| +2450839|68755|2450868|55928|1498390|4052|30067|55928|1498390|4052|30067|4|23|16|4|9164|158|1268|78|38.47|55.39|18.83|2851.68|1468.74|3000.66|4320.42|88.12|0.00|2159.82|1468.74|1556.86|3628.56|3716.68|-1531.92| +2450839|68755|2450908|55928|1498390|4052|30067|55928|1498390|4052|30067|4|78|1|2|7526|126|1268|31|2.24|3.04|0.12|90.52|3.72|69.44|94.24|0.33|0.00|22.32|3.72|4.05|26.04|26.37|-65.72| +2450839|68755|2450914|55928|1498390|4052|30067|55928|1498390|4052|30067|4|80|19|5|3098|127|1268|11|4.93|10.00|7.10|31.90|78.10|54.23|110.00|5.46|0.00|22.00|78.10|83.56|100.10|105.56|23.87| +2450839|68755|2450917|55928|1498390|4052|30067|55928|1498390|4052|30067|4|90|15|2|1765|123|1268|96|19.84|45.83|28.41|1672.32|2727.36|1904.64|4399.68|163.64|0.00|1847.04|2727.36|2891.00|4574.40|4738.04|822.72| +2450839|68755|2450910|55928|1498390|4052|30067|55928|1498390|4052|30067|4|48|10|3|1321|232|1268|62|1.38|3.04|0.06|184.76|3.72|85.56|188.48|0.29|0.00|82.46|3.72|4.01|86.18|86.47|-81.84| +2450839|68755|2450903|55928|1498390|4052|30067|55928|1498390|4052|30067|4|64|19|3|11872|260|1268|52|72.41|123.09|44.31|4096.56|2304.12|3765.32|6400.68|69.12|0.00|2240.16|2304.12|2373.24|4544.28|4613.40|-1461.20| +2450839|68755|2450898|55928|1498390|4052|30067|55928|1498390|4052|30067|4|61|6|3|16322|292|1268|87|18.36|34.51|18.98|1351.11|1651.26|1597.32|3002.37|0.00|0.00|689.91|1651.26|1651.26|2341.17|2341.17|53.94| +2450839|68755|2450857|55928|1498390|4052|30067|55928|1498390|4052|30067|4|107|12|1|4099|157|1268|32|3.01|7.31|1.38|189.76|44.16|96.32|233.92|0.64|31.35|32.64|12.81|13.45|45.45|46.09|-83.51| +2450839|68755|2450919|55928|1498390|4052|30067|55928|1498390|4052|30067|4|62|4|2|15832|51|1268|61|7.33|8.13|3.25|297.68|198.25|447.13|495.93|9.91|0.00|198.25|198.25|208.16|396.50|406.41|-248.88| +2450839|68755|2450847|55928|1498390|4052|30067|55928|1498390|4052|30067|4|14|17|5|7894|9|1268|27|68.36|185.93|48.34|3714.93|1305.18|1845.72|5020.11|78.31|0.00|853.20|1305.18|1383.49|2158.38|2236.69|-540.54| +2450839|43890|2450846|57862|810046|5430|49167|57862|810046|5430|49167|1|99|18|2|17918|262|1269|73|33.24|56.84|48.88|581.08|3568.24|2426.52|4149.32|159.85|1284.56|40.88|2283.68|2443.53|2324.56|2484.41|-142.84| +2450839|43890|2450877|57862|810046|5430|49167|57862|810046|5430|49167|1|94|2|5|11803|101|1269|10|8.57|10.54|3.47|70.70|34.70|85.70|105.40|0.34|0.00|30.50|34.70|35.04|65.20|65.54|-51.00| +2450839|43890|2450903|57862|810046|5430|49167|57862|810046|5430|49167|1|64|9|4|6244|214|1269|18|19.98|54.14|30.31|428.94|545.58|359.64|974.52|0.00|0.00|68.04|545.58|545.58|613.62|613.62|185.94| +2450839|43890|2450874|57862|810046|5430|49167|57862|810046|5430|49167|1|68|3|1|7924|272|1269|33|94.60|208.12|62.43|4807.77|2060.19|3121.80|6867.96|41.20|0.00|3296.37|2060.19|2101.39|5356.56|5397.76|-1061.61| +2450839|43890|2450862|57862|810046|5430|49167|57862|810046|5430|49167|1|26|17|3|16486|264|1269|24|34.29|70.98|17.74|1277.76|425.76|822.96|1703.52|29.80|0.00|800.64|425.76|455.56|1226.40|1256.20|-397.20| +2450839|43890|2450881|57862|810046|5430|49167|57862|810046|5430|49167|1|45|7|1|17480|69|1269|41|63.10|113.58|80.64|1350.54|3306.24|2587.10|4656.78|176.55|363.68|1722.82|2942.56|3119.11|4665.38|4841.93|355.46| +2450839|43890|2450896|57862|810046|5430|49167|57862|810046|5430|49167|1|25|12|1|2876|144|1269|35|99.25|207.43|114.08|3267.25|3992.80|3473.75|7260.05|79.85|0.00|3557.40|3992.80|4072.65|7550.20|7630.05|519.05| +2450839|43890|2450911|57862|810046|5430|49167|57862|810046|5430|49167|1|102|12|5|8299|192|1269|56|56.87|143.88|7.19|7654.64|402.64|3184.72|8057.28|2.97|104.68|1772.40|297.96|300.93|2070.36|2073.33|-2886.76| +2450839|43890|2450922|57862|810046|5430|49167|57862|810046|5430|49167|1|74|4|3|6418|51|1269|75|32.41|51.20|17.92|2496.00|1344.00|2430.75|3840.00|26.88|0.00|1190.25|1344.00|1370.88|2534.25|2561.13|-1086.75| +2450839|43890|2450873|57862|810046|5430|49167|57862|810046|5430|49167|1|26|16|1|13927|266|1269|70|90.67|246.62|229.35|1208.90|16054.50|6346.90|17263.40|0.00|0.00|172.20|16054.50|16054.50|16226.70|16226.70|9707.60| +2450839|43890|2450882|57862|810046|5430|49167|57862|810046|5430|49167|1|76|1|5|12860|187|1269|13|56.48|114.65|0.00|1490.45|0.00|734.24|1490.45|0.00|0.00|566.28|0.00|0.00|566.28|566.28|-734.24| +2450839|43890|2450892|57862|810046|5430|49167|57862|810046|5430|49167|1|18|18|5|16976|117|1269|27|55.60|102.86|72.00|833.22|1944.00|1501.20|2777.22|38.88|0.00|194.40|1944.00|1982.88|2138.40|2177.28|442.80| +2450839|43890|2450858|57862|810046|5430|49167|57862|810046|5430|49167|1|17|13|5|9823|139|1269|100|10.30|23.38|16.13|725.00|1613.00|1030.00|2338.00|70.00|612.94|46.00|1000.06|1070.06|1046.06|1116.06|-29.94| +2450839|73606|2450899|63362|1209798|681|3917|63362|1209798|681|3917|2|60|3|3|3656|168|1270|48|70.55|134.04|48.25|4117.92|2316.00|3386.40|6433.92|115.80|0.00|192.96|2316.00|2431.80|2508.96|2624.76|-1070.40| +|||||681|||||3917|2|33|||17242|49|1270|50|17.41|44.04|0.88|2158.00|44.00||2202.00|3.96|0.00|22.00|||66.00|69.96|-826.50| +2450839|73606|2450880|63362|1209798|681|3917|63362|1209798|681|3917|2|70|6|1|12572|45|1270|67|55.20|64.58|29.70|2336.96|1989.90|3698.40|4326.86|4.17|1930.20|1687.06|59.70|63.87|1746.76|1750.93|-3638.70| +2450839|73606|2450907|63362|1209798|681|3917|63362|1209798|681|3917|2|92|11|1|9589|244|1270|51|52.09|92.72|0.00|4728.72|0.00|2656.59|4728.72|0.00|0.00|1701.87|0.00|0.00|1701.87|1701.87|-2656.59| +2450839|73606|2450928|63362|1209798|681|3917|63362|1209798|681|3917|2|103|17|5|13114|226|1270|11|65.00|111.15|100.03|122.32|1100.33|715.00|1222.65|33.00|0.00|489.06|1100.33|1133.33|1589.39|1622.39|385.33| +2450839|73606|2450882|63362|1209798|681|3917|63362|1209798|681|3917|2|35|14|2|9466|5|1270|68|6.13|10.66|6.92|254.32|470.56|416.84|724.88|18.82|0.00|14.28|470.56|489.38|484.84|503.66|53.72| +2450839|73606|2450906|63362|1209798|681|3917|63362|1209798|681|3917|2|3|4|3|10591|34|1270|100|9.18|23.50|23.03|47.00|2303.00|918.00|2350.00|132.65|92.12|634.00|2210.88|2343.53|2844.88|2977.53|1292.88| +2450839|73606|2450877|63362|1209798|681|3917|63362|1209798|681|3917|2|92|7|4|14794|111|1270|89|45.70|116.07|6.96|9710.79|619.44|4067.30|10330.23|18.58|0.00|2891.61|619.44|638.02|3511.05|3529.63|-3447.86| +2450839|73606|2450905|63362|1209798|681|3917|63362|1209798|681|3917|2|31|11|3|8374|65|1270|52|26.75|66.34|28.52|1966.64|1483.04|1391.00|3449.68|103.81|0.00|517.40|1483.04|1586.85|2000.44|2104.25|92.04| +2450839|73606|2450924|63362|1209798|681|3917|63362|1209798|681|3917|2|99|2|5|13813|240|1270|21|84.58|212.29|99.77|2362.92|2095.17|1776.18|4458.09|104.75|0.00|1292.76|2095.17|2199.92|3387.93|3492.68|318.99| +2450839|73606|2450889|63362|1209798|681|3917|63362|1209798|681|3917|2|76|14|2|2594|72|1270|33|44.21|109.64|105.25|144.87|3473.25|1458.93|3618.12|173.66|0.00|361.68|3473.25|3646.91|3834.93|4008.59|2014.32| +2450839|73606|2450908|63362|1209798|681|3917|63362|1209798|681|3917|2|42|19|3|58|174|1270|27|93.23|121.19|110.28|294.57|2977.56|2517.21|3272.13|0.00|0.00|1603.26|2977.56|2977.56|4580.82|4580.82|460.35| +2450839|61644|2450871|23911|1257893|5828|7763|23911|1257893|5828|7763|1|99|9|2|8000|115|1271|6|29.62|47.39|8.53|233.16|51.18|177.72|284.34|2.04|0.00|76.74|51.18|53.22|127.92|129.96|-126.54| +2450839|61644|2450897|23911|1257893|5828|7763|23911|1257893|5828|7763|1|84|17|3|1312|269|1271|67|10.78|22.63|0.00|1516.21|0.00|722.26|1516.21|0.00|0.00|682.06|0.00|0.00|682.06|682.06|-722.26| +2450839|61644|2450862|23911|1257893|5828|7763|23911|1257893|5828|7763|1|85|6|4|3500|172|1271|20|79.94|83.93|52.87|621.20|1057.40|1598.80|1678.60|10.57|0.00|151.00|1057.40|1067.97|1208.40|1218.97|-541.40| +2450839|61644|2450889|23911|1257893|5828|7763|23911|1257893|5828|7763|1|65|20|1|11566|11|1271|3|85.57|181.40|112.46|206.82|337.38|256.71|544.20|30.36|0.00|255.75|337.38|367.74|593.13|623.49|80.67| +2450839|61644|2450848|23911|1257893|5828|7763|23911|1257893|5828|7763|1|71|15|2|5395|121|1271|59|70.43|184.52|42.43|8383.31|2503.37|4155.37|10886.68|125.16|0.00|1741.68|2503.37|2628.53|4245.05|4370.21|-1652.00| +2450839|61644|2450906|23911|1257893|5828|7763|23911|1257893|5828|7763|1|78|20|3|11678|85|1271|47|38.73|89.07|58.78|1423.63|2762.66|1820.31|4186.29|38.95|1464.20|962.56|1298.46|1337.41|2261.02|2299.97|-521.85| +2450839|61644|2450885|23911|1257893|5828|7763|23911|1257893|5828|7763|1|33|3|1|8497|15|1271|4|73.73|153.35|13.80|558.20|55.20|294.92|613.40|1.65|0.00|110.40|55.20|56.85|165.60|167.25|-239.72| +2450839|61644|2450883|23911|1257893|5828|7763|23911|1257893|5828|7763|1|62|2|1|2356|14|1271|90|34.98|53.86|11.84|3781.80|1065.60|3148.20|4847.40|0.00|0.00|2375.10|1065.60|1065.60|3440.70|3440.70|-2082.60| +2450839|61644|2450870|23911|1257893|5828|7763|23911|1257893|5828|7763|1|7|17|4|6674|181|1271|62|5.85|12.87|5.92|430.90|367.04|362.70|797.94|11.01|0.00|23.56|367.04|378.05|390.60|401.61|4.34| +2450839|61644|2450859|23911|1257893|5828|7763|23911|1257893|5828|7763|1|34|8|1|10993|107|1271|58|15.89|32.25|9.03|1346.76|523.74|921.62|1870.50|41.89|0.00|635.68|523.74|565.63|1159.42|1201.31|-397.88| +2450839|61644|2450855|23911|1257893|5828|7763|23911|1257893|5828|7763|1|67|18|4|13711|183|1271|50|50.04|128.10|17.93|5508.50|896.50|2502.00|6405.00|8.96|0.00|1985.50|896.50|905.46|2882.00|2890.96|-1605.50| +2450839|48033|2450917|6843|1327831|3572|28513|6843|1327831|3572|28513|1|81|11|5|14702|211|1272|22|63.06|84.50|72.67|260.26|1598.74|1387.32|1859.00|31.97|0.00|0.00|1598.74|1630.71|1598.74|1630.71|211.42| +2450839|48033|2450895|6843|1327831|3572|28513|6843|1327831|3572|28513|1|84|14|4|3632|141|1272|31|90.81|176.17|61.65|3550.12|1911.15|2815.11|5461.27|133.78|0.00|1255.81|1911.15|2044.93|3166.96|3300.74|-903.96| +2450839|48033|2450852|6843|1327831|3572|28513|6843|1327831|3572|28513|1|96|4|4|8788|289|1272|7|45.68|120.13|6.00|798.91|42.00|319.76|840.91|3.78|0.00|159.74|42.00|45.78|201.74|205.52|-277.76| +2450839|48033|2450855|6843|1327831|3572|28513|6843|1327831|3572|28513|1|65|8|4|12418|110|1272|1|1.49|1.66|0.00|1.66|0.00|1.49|1.66|0.00|0.00|0.34|0.00|0.00|0.34|0.34|-1.49| +2450839|48033|2450887|6843|1327831|3572|28513|6843|1327831|3572|28513|1|92|19|1|7454|138|1272|61|76.20|112.77|111.64|68.93|6810.04|4648.20|6878.97|563.87|544.80|1444.48|6265.24|6829.11|7709.72|8273.59|1617.04| +2450839|48033|2450851|6843|1327831|3572|28513|6843|1327831|3572|28513|1|21|3|3|9440|300|1272|53|41.32|43.79|38.53|278.78|2042.09|2189.96|2320.87|122.52|0.00|928.03|2042.09|2164.61|2970.12|3092.64|-147.87| +2450839|48033|2450917|6843|1327831|3572|28513|6843|1327831|3572|28513|1|104|8|3|12301|126|1272|15|36.39|72.78|62.59|152.85|938.85|545.85|1091.70|46.94|0.00|109.05|938.85|985.79|1047.90|1094.84|393.00| +2450839|74182|2450908|98729|257727|4372|2170|98729|257727|4372|2170|4|70|9|5|2012|112|1273|6|25.31|61.25|9.80|308.70|58.80|151.86|367.50|0.58|0.00|180.06|58.80|59.38|238.86|239.44|-93.06| +2450839|74182|2450866|98729|257727|4372|2170|98729|257727|4372|2170|4|96|2|4|17809|28|1273|48|84.21|182.73|78.57|4999.68|3771.36|4042.08|8771.04|113.14|0.00|175.20|3771.36|3884.50|3946.56|4059.70|-270.72| +2450839|74182|2450915|98729|257727|4372|2170|98729|257727|4372|2170|4|53|1|1|2233|8|1273|86|39.91|75.42|61.84|1167.88|5318.24|3432.26|6486.12|159.54|0.00|1945.32|5318.24|5477.78|7263.56|7423.10|1885.98| +2450839|74182|2450893|98729|257727|4372|2170|98729|257727|4372|2170|4|105|9|1|5350|153|1273|3|25.77|61.33|26.98|103.05|80.94|77.31|183.99|3.34|33.18|12.87|47.76|51.10|60.63|63.97|-29.55| +2450839|74182|2450863|98729|257727|4372|2170|98729|257727|4372|2170|4|94|2|1|3572|107|1273|40|27.87|64.93|42.85|883.20|1714.00|1114.80|2597.20|102.84|0.00|25.60|1714.00|1816.84|1739.60|1842.44|599.20| +2450839|74182|2450851|98729|257727|4372|2170|98729|257727|4372|2170|4|92|14|3|2419|136|1273|23|59.54|111.93|99.61|283.36|2291.03|1369.42|2574.39|114.55|0.00|205.85|2291.03|2405.58|2496.88|2611.43|921.61| +2450839|67403|2450919|92370|1146735|5900|3818|92370|1146735|5900|3818|4|87|16|3|15518|4|1274|68|26.67|28.00|23.80|285.60|1618.40|1813.56|1904.00|32.36|0.00|399.84|1618.40|1650.76|2018.24|2050.60|-195.16| +2450839|67403|2450865|92370|1146735|5900|3818|92370|1146735|5900|3818|4|68|19|5|8737|212|1274|46|93.99|230.27|179.61|2330.36|8262.06|4323.54|10592.42|495.72|0.00|1694.64|8262.06|8757.78|9956.70|10452.42|3938.52| +2450839|67403|2450853|92370|1146735|5900|3818|92370|1146735|5900|3818|4|104|9|3|7357|251|1274|60|23.07|61.82|13.60|2893.20|816.00|1384.20|3709.20|16.32|0.00|1594.80|816.00|832.32|2410.80|2427.12|-568.20| +2450839|67403|2450905|92370|1146735|5900|3818|92370|1146735|5900|3818|4|37|11|3|6121|277|1274|62|20.61|38.33|13.03|1568.60|807.86|1277.82|2376.46|16.15|0.00|854.98|807.86|824.01|1662.84|1678.99|-469.96| +2450839|41015|2450869|76862|694508|601|37847|76862|694508|601|37847|2|30|10|5|4447|150|1275|47|14.24|35.03|5.25|1399.66|246.75|669.28|1646.41|0.00|0.00|65.80|246.75|246.75|312.55|312.55|-422.53| +2450839|41015|2450917|76862|694508|601|37847|76862|694508|601|37847|2|36|13|3|10234|142|1275|29|48.17|105.01|5.25|2893.04|152.25|1396.93|3045.29|7.61|0.00|1157.10|152.25|159.86|1309.35|1316.96|-1244.68| +2450839|41015|2450889|76862|694508|601|37847|76862|694508|601|37847|2|67|13|1|7441|52|1275|15|88.38|156.43|26.59|1947.60|398.85|1325.70|2346.45|23.93|0.00|422.25|398.85|422.78|821.10|845.03|-926.85| +2450839|41015|2450902|76862|694508|601|37847|76862|694508|601|37847|2|33|20|2|11102|11|1275|70|3.38|7.30|5.98|92.40|418.60|236.60|511.00|8.37|0.00|153.30|418.60|426.97|571.90|580.27|182.00| +2450839|41015|2450898|76862|694508|601|37847|76862|694508|601|37847|2|108|19|5|8413|147|1275|14|21.81|61.28|3.67|806.54|51.38|305.34|857.92|1.02|0.00|111.44|51.38|52.40|162.82|163.84|-253.96| +2450839|41015|2450873|76862|694508|601|37847|76862|694508|601|37847|2|45|10|5|10606|80|1275|82|65.61|140.40|70.20|5756.40|5756.40|5380.02|11512.80|466.26|575.64|5640.78|5180.76|5647.02|10821.54|11287.80|-199.26| +2450839|41015|2450882|76862|694508|601|37847|76862|694508|601|37847|2|18|16|4|961|127|1275|57|63.00|126.00|60.48|3734.64|3447.36|3591.00|7182.00|172.36|0.00|1077.30|3447.36|3619.72|4524.66|4697.02|-143.64| +2450839|41015|2450923|76862|694508|601|37847|76862|694508|601|37847|2|27|14|5|5305|286|1275|38|76.54|172.21|17.22|5889.62|654.36|2908.52|6543.98|19.63|0.00|1831.98|654.36|673.99|2486.34|2505.97|-2254.16| +2450839|41015|2450914|76862|694508|601|37847|76862|694508|601|37847|2|27|15|4|12637|29|1275|66|40.71|78.57|22.78|3682.14|1503.48|2686.86|5185.62|135.31|0.00|2229.48|1503.48|1638.79|3732.96|3868.27|-1183.38| +2450839|41015|2450897|76862|694508|601|37847|76862|694508|601|37847|2|49|15|2|5707|211|1275|87|59.73|140.36|26.66|9891.90|2319.42|5196.51|12211.32|208.74|0.00|366.27|2319.42|2528.16|2685.69|2894.43|-2877.09| +2450839|41015|2450846|76862|694508|601|37847|76862|694508|601|37847|2|71|16|3|9595|287|1275|40|29.29|39.83|38.63|48.00|1545.20|1171.60|1593.20|28.58|973.47|15.60|571.73|600.31|587.33|615.91|-599.87| +2450839|41015|2450902|76862|694508|601|37847|76862|694508|601|37847|2|33|16|2|12488|144|1275|92|51.75|153.69|13.83|12867.12|1272.36|4761.00|14139.48|50.89|0.00|282.44|1272.36|1323.25|1554.80|1605.69|-3488.64| +2450839|45775|2450852|21948|1688772|5549|9526|21948|1688772|5549|9526|1|83|5|3|17845|26|1276|8|83.68|117.98|15.33|821.20|122.64|669.44|943.84|11.03|0.00|160.40|122.64|133.67|283.04|294.07|-546.80| +2450839|45775|2450854|21948|1688772|5549|9526|21948|1688772|5549|9526|1|79|15|4|16051|194|1276|21|42.88|122.63|116.49|128.94|2446.29|900.48|2575.23|11.74|2250.58|412.02|195.71|207.45|607.73|619.47|-704.77| +2450839|45775|2450902|21948|1688772|5549|9526|21948|1688772|5549|9526|1|15|10|1|17638|98|1276|51|22.18|33.27|15.96|882.81|813.96|1131.18|1696.77|56.97|0.00|271.32|813.96|870.93|1085.28|1142.25|-317.22| +2450839|45775|2450915|21948|1688772|5549|9526|21948|1688772|5549|9526|1|25|4|2|127|17|1276|11|44.59|119.05|55.95|694.10|615.45|490.49|1309.55|12.30|0.00|117.81|615.45|627.75|733.26|745.56|124.96| +2450839|45775|2450876|21948|1688772|5549|9526|21948|1688772|5549|9526|1|16|3|3|68|256|1276|31|62.05|168.15|45.40|3805.25|1407.40|1923.55|5212.65|56.29|0.00|2137.14|1407.40|1463.69|3544.54|3600.83|-516.15| +2450839|45775|2450925|21948|1688772|5549|9526|21948|1688772|5549|9526|1|18|3|1|16279|36|1276|3|61.22|97.33|35.03|186.90|105.09|183.66|291.99|1.54|83.02|2.91|22.07|23.61|24.98|26.52|-161.59| +2450839|45775|2450905|21948|1688772|5549|9526|21948|1688772|5549|9526|1|100|6|4|16760|137|1276|44|53.42|146.90|44.07|4524.52|1939.08|2350.48|6463.60|0.00|0.00|64.24|1939.08|1939.08|2003.32|2003.32|-411.40| +2450839|45775|2450841|21948|1688772|5549|9526|21948|1688772|5549|9526|1|88|5|1|8272|143|1276|89|37.68|113.04|73.47|3521.73|6538.83|3353.52|10060.56|194.20|65.38|1106.27|6473.45|6667.65|7579.72|7773.92|3119.93| +2450839|45775|2450929|21948|1688772|5549|9526|21948|1688772|5549|9526|1|48|5|4|17662|162|1276|14|9.94|24.65|13.80|151.90|193.20|139.16|345.10|13.52|0.00|120.68|193.20|206.72|313.88|327.40|54.04| +2450839|45775|2450866|21948|1688772|5549|9526|21948|1688772|5549|9526|1|4|10|4|2413|8|1276|35|10.69|25.65|21.03|161.70|736.05|374.15|897.75|36.80|0.00|53.55|736.05|772.85|789.60|826.40|361.90| +2450839|45775|2450856|21948|1688772|5549|9526|21948|1688772|5549|9526|1|4|3|5|10693|89|1276|30|46.30|111.58|108.23|100.50|3246.90|1389.00|3347.40|129.87|0.00|401.40|3246.90|3376.77|3648.30|3778.17|1857.90| +2450840|75546|2450903|43113|651663|213|3610|43113|651663|213|3610|2|67|19|4|7844|2|1277|26|3.91|4.30|4.21|2.34|109.46|101.66|111.80|5.47|0.00|15.60|109.46|114.93|125.06|130.53|7.80| +2450840|75546|2450913|43113|651663|213|3610|43113|651663|213|3610|2|100|16|5|12343|22|1277|47|58.06|145.73|72.86|3424.89|3424.42|2728.82|6849.31|308.19|0.00|616.17|3424.42|3732.61|4040.59|4348.78|695.60| +2450840|75546|2450910|43113|651663|213|3610|43113|651663|213|3610|2|6|19|4|6728|298|1277|27|36.78|71.35|64.92|173.61|1752.84|993.06|1926.45|122.69|0.00|308.07|1752.84|1875.53|2060.91|2183.60|759.78| +2450840|75546|2450864|43113|651663|213|3610|43113|651663|213|3610|2|65|7|3|428|242|1277|69|73.35|85.81|12.87|5032.86|888.03|5061.15|5920.89|71.04|0.00|1065.36|888.03|959.07|1953.39|2024.43|-4173.12| +2450840|75546|2450923|43113|651663|213|3610|43113|651663|213|3610|2|101|18|5|14575|74|1277|52|29.99|58.18|23.27|1815.32|1210.04|1559.48|3025.36|72.60|0.00|1119.04|1210.04|1282.64|2329.08|2401.68|-349.44| +2450840|75546|2450869|43113|651663|213|3610|43113|651663|213|3610|2|19|19|4|7528|185|1277|79|48.18|125.26|52.60|5740.14|4155.40|3806.22|9895.54|249.32|0.00|2671.78|4155.40|4404.72|6827.18|7076.50|349.18| +2450840|75546|2450893|43113|651663|213|3610|43113|651663|213|3610|2|75|16|1|11732|44|1277|72|47.83|132.48|121.88|763.20|8775.36|3443.76|9538.56|702.02|0.00|285.84|8775.36|9477.38|9061.20|9763.22|5331.60| +2450840|75546|2450925|43113|651663|213|3610|43113|651663|213|3610|2|64|3|1|3374|257|1277|55|42.26|111.14|71.12|2201.10|3911.60|2324.30|6112.70|312.92|0.00|1222.10|3911.60|4224.52|5133.70|5446.62|1587.30| +2450840|75546|2450844|43113|651663|213|3610|43113|651663|213|3610|2|31|18|4|16886|181|1277|45|84.27|199.71|103.84|4314.15|4672.80|3792.15|8986.95|186.91|0.00|1437.75|4672.80|4859.71|6110.55|6297.46|880.65| +2450840|72886|2450888|45933|1257095|7104|33866|45933|1257095|7104|33866|1|97|13|5|15898|50|1278|90|85.58|87.29|37.53|4478.40|3377.70|7702.20|7856.10|303.99|0.00|784.80|3377.70|3681.69|4162.50|4466.49|-4324.50| +2450840|72886|2450899|45933|1257095|7104|33866|45933|1257095|7104|33866|1|14|14|3|9652|12|1278|95|70.37|83.03|82.19|79.80|7808.05|6685.15|7887.85|312.32|0.00|1813.55|7808.05|8120.37|9621.60|9933.92|1122.90| +2450840|72886|2450905|45933|1257095|7104|33866|45933|1257095|7104|33866|1|36|15|1|12289|75|1278|4|39.06|106.24|59.49|187.00|237.96|156.24|424.96|7.13|95.18|165.72|142.78|149.91|308.50|315.63|-13.46| +2450840|72886|2450919|45933|1257095|7104|33866|45933|1257095|7104|33866|1|11|9|3|12440|116|1278|3|78.57|187.78|105.15|247.89|315.45|235.71|563.34|0.00|0.00|276.03|315.45|315.45|591.48|591.48|79.74| +2450840|72886|2450905|45933|1257095|7104|33866|45933|1257095|7104|33866|1|60|3|4|7622|14|1278|38|24.92|46.60|44.27|88.54|1682.26|946.96|1770.80|67.29|0.00|531.24|1682.26|1749.55|2213.50|2280.79|735.30| +2450840|72886|2450843|45933|1257095|7104|33866|45933|1257095|7104|33866|1|20|20|2|698|44|1278|31|93.33|243.59|21.92|6871.77|679.52|2893.23|7551.29|47.56|0.00|754.85|679.52|727.08|1434.37|1481.93|-2213.71| +2450840|72886|2450861|45933|1257095|7104|33866|45933|1257095|7104|33866|1|15|1|5|5881|180|1278|96|37.83|103.65|80.84|2189.76|7760.64|3631.68|9950.40|344.57|2017.76|1293.12|5742.88|6087.45|7036.00|7380.57|2111.20| +2450840|72886|2450842|45933|1257095|7104|33866|45933|1257095|7104|33866|1|74|2|1|5761|76|1278|73|78.24|127.53|1.27|9216.98|92.71|5711.52|9309.69|5.56|0.00|4096.03|92.71|98.27|4188.74|4194.30|-5618.81| +2450840|72886|2450871|45933|1257095|7104|33866|45933|1257095|7104|33866|1|3|10|2|16184|256|1278|93|13.30|21.28|4.68|1543.80|435.24|1236.90|1979.04|4.35|0.00|949.53|435.24|439.59|1384.77|1389.12|-801.66| +2450840|72886|2450861|45933|1257095|7104|33866|45933|1257095|7104|33866|1|64|16|3|7405|197|1278|70|60.12|169.53|100.02|4865.70|7001.40|4208.40|11867.10|70.01|0.00|5339.60|7001.40|7071.41|12341.00|12411.01|2793.00| +2450840|71982|2450903|54955|1135706|4457|22808|54955|1135706|4457|22808|1|51|5|4|956|79|1279|30|93.25|255.50|189.07|1992.90|5672.10|2797.50|7665.00|397.04|0.00|2146.20|5672.10|6069.14|7818.30|8215.34|2874.60| +2450840|71982|2450881|54955|1135706|4457|22808|54955|1135706|4457|22808|1|91|17|1|14341|294|1279|76|61.34|64.40|52.16|930.24|3964.16|4661.84|4894.40|118.92|0.00|244.72|3964.16|4083.08|4208.88|4327.80|-697.68| +2450840|71982|2450881|54955|1135706|4457|22808|54955|1135706|4457|22808|1|3|11|5|1579|263|1279|11|11.00|18.59|3.16|169.73|34.76|121.00|204.49|0.69|0.00|8.14|34.76|35.45|42.90|43.59|-86.24| +2450840|71982|2450853|54955|1135706|4457|22808|54955|1135706|4457|22808|1|32|3|1|446|72|1279|69|78.32|79.10|59.32|1364.82|4093.08|5404.08|5457.90|163.72|0.00|2073.45|4093.08|4256.80|6166.53|6330.25|-1311.00| +2450840|71982|2450881|54955|1135706|4457|22808|54955|1135706|4457|22808|1|17|12|3|14839|55|1279|48|92.35|248.42|191.28|2742.72|9181.44|4432.80|11924.16|82.63|6427.00|3934.56|2754.44|2837.07|6689.00|6771.63|-1678.36| +2450840|71982|2450854|54955|1135706|4457|22808|54955|1135706|4457|22808|1|34|20|1|14858|203|1279|79|75.17|139.81|92.27|3755.66|7289.33|5938.43|11044.99|291.57|3644.66|2761.05|3644.67|3936.24|6405.72|6697.29|-2293.76| +2450840|71982|2450923|54955|1135706|4457|22808|54955|1135706|4457|22808|1|73|11|3|13717|110|1279|7|46.76|50.96|13.24|264.04|92.68|327.32|356.72|0.89|2.78|32.06|89.90|90.79|121.96|122.85|-237.42| +2450840|17498|2450846|26600|403291|4782|41473|63672|1827302|7193|28284|1|8|13|4|6692|203|1280|34|5.22|8.09|6.39|57.80|217.26|177.48|275.06|6.51|0.00|21.76|217.26|223.77|239.02|245.53|39.78| +2450840|17498|2450855|26600|403291|4782|41473|63672|1827302|7193|28284|1|81|15|4|4015|289|1280|53|71.36|147.00|27.93|6310.71|1480.29|3782.08|7791.00|103.62|0.00|3350.13|1480.29|1583.91|4830.42|4934.04|-2301.79| +2450840|17498|2450911|26600|403291|4782|41473|63672|1827302|7193|28284|1|36|11|4|3211|7|1280|66|44.35|93.13|12.10|5347.98|798.60|2927.10|6146.58|15.97|0.00|553.08|798.60|814.57|1351.68|1367.65|-2128.50| +2450840|17498|2450906|26600|403291|4782|41473|63672|1827302|7193|28284|1|35|14|1|4375|288|1280|19|51.06|107.22|25.73|1548.31|488.87|970.14|2037.18|1.27|361.76|529.53|127.11|128.38|656.64|657.91|-843.03| +2450840|17498|2450861|26600|403291|4782|41473|63672|1827302|7193|28284|1|2|7|1|7855|33|1280|68|86.34|92.38|33.25|4020.84|2261.00|5871.12|6281.84|135.66|0.00|1130.16|2261.00|2396.66|3391.16|3526.82|-3610.12| +2450840|17498|2450861|26600|403291|4782|41473|63672|1827302|7193|28284|1|32|15|3|16243|274|1280|99|41.27|73.46|63.91|945.45|6327.09|4085.73|7272.54|120.21|3922.79|0.00|2404.30|2524.51|2404.30|2524.51|-1681.43| +2450840|17498|2450891|26600|403291|4782|41473|63672|1827302|7193|28284|1|89|10|1|14624|86|1280|70|11.17|11.28|2.14|639.80|149.80|781.90|789.60|0.67|82.39|323.40|67.41|68.08|390.81|391.48|-714.49| +2450840|17498|2450874|26600|403291|4782|41473|63672|1827302|7193|28284|1|77|10|4|9637|137|1280|45|9.19|14.97|6.43|384.30|289.35|413.55|673.65|0.00|0.00|154.80|289.35|289.35|444.15|444.15|-124.20| +2450840|17498|2450862|26600|403291|4782|41473|63672|1827302|7193|28284|1|22|5|5|766|185|1280|81|27.17|63.84|46.60|1396.44|3774.60|2200.77|5171.04|163.06|1736.31|2326.32|2038.29|2201.35|4364.61|4527.67|-162.48| +2450840|17498|2450896|26600|403291|4782|41473|63672|1827302|7193|28284|1|7|9|5|7760|17|1280|6|11.26|29.50|11.50|108.00|69.00|67.56|177.00|5.52|0.00|22.98|69.00|74.52|91.98|97.50|1.44| +2450840|17498|2450899|26600|403291|4782|41473|63672|1827302|7193|28284|1|29|8|5|15637|243|1280|76|80.44|98.13|30.42|5145.96|2311.92|6113.44|7457.88|115.59|0.00|2236.68|2311.92|2427.51|4548.60|4664.19|-3801.52| +2450840|17498|2450895|26600|403291|4782|41473|63672|1827302|7193|28284|1|43|20|5|13592|170|1280|44|20.61|53.17|20.73|1427.36|912.12|906.84|2339.48|45.60|0.00|701.80|912.12|957.72|1613.92|1659.52|5.28| +2450840|17498|2450915|26600|403291|4782|41473|63672|1827302|7193|28284|1|85|9|2|16183|166|1280|35|45.90|132.19|100.46|1110.55|3516.10|1606.50|4626.65|259.48|632.89|647.50|2883.21|3142.69|3530.71|3790.19|1276.71| +2450840|17498|2450889|26600|403291|4782|41473|63672|1827302|7193|28284|1|68|1|2|15811|246|1280|52|54.48|146.55|126.03|1067.04|6553.56|2832.96|7620.60|458.74|0.00|2362.36|6553.56|7012.30|8915.92|9374.66|3720.60| +2450840|8478|2450881|2290|1823495|5947|44752|2290|1823495|5947|44752|4|52|6|3|1148|291|1281|7|39.87|91.70|24.75|468.65|173.25|279.09|641.90|10.39|0.00|295.26|173.25|183.64|468.51|478.90|-105.84| +2450840|8478|2450906|2290|1823495|5947|44752|2290|1823495|5947|44752|4|28|3|1|6085|164|1281|3|58.40|167.60|53.63|341.91|160.89|175.20|502.80|9.65|0.00|75.42|160.89|170.54|236.31|245.96|-14.31| +2450840|8478|2450896|2290|1823495|5947|44752|2290|1823495|5947|44752|4|5|2|1|4370|271|1281|44|31.26|80.33|55.42|1096.04|2438.48|1375.44|3534.52|195.07|0.00|989.56|2438.48|2633.55|3428.04|3623.11|1063.04| +2450840|8478|2450895|2290|1823495|5947|44752|2290|1823495|5947|44752|4|22|20|5|12412|157|1281|30|59.95|173.85|159.94|417.30|4798.20|1798.50|5215.50|191.92|0.00|364.80|4798.20|4990.12|5163.00|5354.92|2999.70| +2450840|8478|2450899|2290|1823495|5947|44752|2290|1823495|5947|44752|4|49|15|5|6079|268|1281|11|45.06|88.76|31.95|624.91|351.45|495.66|976.36|21.08|0.00|165.88|351.45|372.53|517.33|538.41|-144.21| +2450840|8478|2450855|2290|1823495|5947|44752|2290|1823495|5947|44752|4|48|8|1|12662|152|1281|42|86.21|164.66|79.03|3596.46|3319.26|3620.82|6915.72|66.38|0.00|68.88|3319.26|3385.64|3388.14|3454.52|-301.56| +2450840|8478|2450891|2290|1823495|5947|44752|2290|1823495|5947|44752|4|74|5|1|3266|160|1281|10|59.98|74.97|16.49|584.80|164.90|599.80|749.70|6.59|0.00|134.90|164.90|171.49|299.80|306.39|-434.90| +2450840|43642|2450870|25717|1480001|6638|38875|25717|1480001|6638|38875|1|29|6|4|4208|258|1282|80|98.12|100.08|93.07|560.80|7445.60|7849.60|8006.40|446.73|0.00|1280.80|7445.60|7892.33|8726.40|9173.13|-404.00| +2450840|43642|2450901|25717|1480001|6638|38875|25717|1480001|6638|38875|1|26|2|5|5852|236|1282|70|82.99|180.08|124.25|3908.10|8697.50|5809.30|12605.60|782.77|0.00|1386.00|8697.50|9480.27|10083.50|10866.27|2888.20| +2450840|43642|2450866|25717|1480001|6638|38875|25717|1480001|6638|38875|1|44|17|4|14749|88|1282|24|89.13|148.84|47.62|2429.28|1142.88|2139.12|3572.16|68.57|0.00|1500.24|1142.88|1211.45|2643.12|2711.69|-996.24| +2450840|43642|2450892|25717|1480001|6638|38875|25717|1480001|6638|38875|1|71|19|1|14977|105|1282|88|17.24|48.61|3.40|3978.48|299.20|1517.12|4277.68|8.97|0.00|1026.08|299.20|308.17|1325.28|1334.25|-1217.92| +2450840|43642|2450871|25717|1480001|6638|38875|25717|1480001|6638|38875|1|74|11|4|301|162|1282|23|90.38|109.35|8.74|2314.03|201.02|2078.74|2515.05|6.03|0.00|603.52|201.02|207.05|804.54|810.57|-1877.72| +2450840|43642|2450898|25717|1480001|6638|38875|25717|1480001|6638|38875|1|61|4|4|12523|79|1282|72|57.37|128.50|25.70|7401.60|1850.40|4130.64|9252.00|55.51|0.00|1202.40|1850.40|1905.91|3052.80|3108.31|-2280.24| +2450840|43642|2450871|25717|1480001|6638|38875|25717|1480001|6638|38875|1|48|18|3|17575|145|1282|41|19.60|47.82|47.82|0.00|1960.62|803.60|1960.62|44.11|1470.46|921.27|490.16|534.27|1411.43|1455.54|-313.44| +2450840|27016|2450869|66616|2269|2275|13528|28133|362387|6393|3646|1|92|17|3|6848|30|1283|87|11.01|24.33|2.43|1905.30|211.41|957.87|2116.71|7.04|133.18|465.45|78.23|85.27|543.68|550.72|-879.64| +2450840|27016|2450892|66616|2269|2275|13528|28133|362387|6393|3646|1|16|5|2|11830|123|1283|16|16.02|39.24|29.03|163.36|464.48|256.32|627.84|35.11|74.31|37.60|390.17|425.28|427.77|462.88|133.85| +2450840|27016|2450883|66616|2269|2275|13528|28133|362387|6393|3646|1|67|17|3|17359|233|1283|95|2.74|6.30|4.28|191.90|406.60|260.30|598.50|4.06|0.00|250.80|406.60|410.66|657.40|661.46|146.30| +2450840|27016|2450863|66616|2269|2275|13528|28133|362387|6393|3646|1|55|5|3|15748|276|1283|85|35.40|68.67|30.21|3269.10|2567.85|3009.00|5836.95|25.67|0.00|2334.10|2567.85|2593.52|4901.95|4927.62|-441.15| +2450840|27016|2450898|66616|2269|2275|13528|28133|362387|6393|3646|1|69|5|1|1892|68|1283|21|16.08|29.74|17.84|249.90|374.64|337.68|624.54|14.98|0.00|237.30|374.64|389.62|611.94|626.92|36.96| +2450840|27016|2450850|66616|2269|2275|13528|28133|362387|6393|3646|1|43|20|5|14557|102|1283|10|8.29|23.79|3.56|202.30|35.60|82.90|237.90|0.85|18.51|68.90|17.09|17.94|85.99|86.84|-65.81| +2450840|27016|2450901|66616|2269|2275|13528|28133|362387|6393|3646|1|77|15|1|17965|190|1283|92|50.80|107.18|33.22|6804.32|3056.24|4673.60|9860.56|0.00|0.00|295.32|3056.24|3056.24|3351.56|3351.56|-1617.36| +2450840|27016|2450910|66616|2269|2275|13528|28133|362387|6393|3646|1|16|13|3|12922|128|1283|58|18.93|50.92|37.17|797.50|2155.86|1097.94|2953.36|21.55|0.00|1092.72|2155.86|2177.41|3248.58|3270.13|1057.92| +2450840|27016|2450926|66616|2269|2275|13528|28133|362387|6393|3646|1|27|7|5|5773|278|1283|92|20.21|36.37|24.36|1104.92|2241.12|1859.32|3346.04|156.87|0.00|1572.28|2241.12|2397.99|3813.40|3970.27|381.80| +2450840|27016|2450859|66616|2269|2275|13528|28133|362387|6393|3646|1|27|18|2|7724|146|1283|18|25.30|66.79|56.77|180.36|1021.86|455.40|1202.22|0.00|1001.42|492.84|20.44|20.44|513.28|513.28|-434.96| +2450840|27016|2450874|66616|2269|2275|13528|28133|362387|6393|3646|1|44|19|2|13360|184|1283|15|92.54|242.45|82.43|2400.30|1236.45|1388.10|3636.75|24.72|0.00|1054.65|1236.45|1261.17|2291.10|2315.82|-151.65| +2450840|27016|2450920|66616|2269|2275|13528|28133|362387|6393|3646|1|12|12|1|10570|191|1283|17|29.09|52.94|40.76|207.06|692.92|494.53|899.98|0.00|0.00|26.86|692.92|692.92|719.78|719.78|198.39| +2450840|46598|2450892|73655|1436656|4457|9058|73655|1436656|4457|9058|4|25|1|5|2356|13|1284|15|45.32|48.03|9.60|576.45|144.00|679.80|720.45|0.21|122.40|324.15|21.60|21.81|345.75|345.96|-658.20| +2450840|46598|2450908|73655|1436656|4457|9058|73655|1436656|4457|9058|4|21|4|5|6674|57|1284|38|67.67|107.59|75.31|1226.64|2861.78|2571.46|4088.42|64.39|715.44|981.16|2146.34|2210.73|3127.50|3191.89|-425.12| +2450840|46598|2450846|73655|1436656|4457|9058|73655|1436656|4457|9058|4|91|10|3|10993|224|1284|74|51.76|61.59|29.56|2370.22|2187.44|3830.24|4557.66|0.00|0.00|592.00|2187.44|2187.44|2779.44|2779.44|-1642.80| +2450840|46598|2450842|73655|1436656|4457|9058|73655|1436656|4457|9058|4|5|8|3|13711|161|1284|42|41.18|59.71|41.79|752.64|1755.18|1729.56|2507.82|122.86|0.00|651.84|1755.18|1878.04|2407.02|2529.88|25.62| +2450840|46598|2450918|73655|1436656|4457|9058|73655|1436656|4457|9058|4|44|7|1|17305|289|1284|88|1.25|1.63|0.03|140.80|2.64|110.00|143.44|0.05|0.00|44.00|2.64|2.69|46.64|46.69|-107.36| +2450840|46598|2450893|73655|1436656|4457|9058|73655|1436656|4457|9058|4|27|14|2|8428|10|1284|28|49.91|137.25|67.25|1960.00|1883.00|1397.48|3843.00|169.47|0.00|1037.40|1883.00|2052.47|2920.40|3089.87|485.52| +2450840|46598|2450915|73655|1436656|4457|9058|73655|1436656|4457|9058|4|94|1|1|13390|256|1284|2|35.92|68.60|43.21|50.78|86.42|71.84|137.20|0.86|0.00|17.82|86.42|87.28|104.24|105.10|14.58| +2450840|46598|2450928|73655|1436656|4457|9058|73655|1436656|4457|9058|4|78|19|3|15086|27|1284|81|93.30|181.00|48.87|10702.53|3958.47|7557.30|14661.00|79.16|0.00|6744.06|3958.47|4037.63|10702.53|10781.69|-3598.83| +2450840|46598|2450869|73655|1436656|4457|9058|73655|1436656|4457|9058|4|19|2|3|9584|178|1284|67|44.69|52.73|42.18|706.85|2826.06|2994.23|3532.91|197.82|0.00|1342.01|2826.06|3023.88|4168.07|4365.89|-168.17| +2450840|46598|2450909|73655|1436656|4457|9058|73655|1436656|4457|9058|4|85|2|3|10370|199|1284|43|54.70|96.27|6.73|3850.22|289.39|2352.10|4139.61|8.68|0.00|1283.12|289.39|298.07|1572.51|1581.19|-2062.71| +2450840|46598|2450901|73655|1436656|4457|9058|73655|1436656|4457|9058|4|67|5|5|3574|11|1284|12|39.20|94.08|92.19|22.68|1106.28|470.40|1128.96|66.37|0.00|191.88|1106.28|1172.65|1298.16|1364.53|635.88| +2450840|46598|2450902|73655|1436656|4457|9058|73655|1436656|4457|9058|4|54|7|2|919|27|1284|51|86.94|133.01|133.01|0.00|6783.51|4433.94|6783.51|271.34|0.00|1153.11|6783.51|7054.85|7936.62|8207.96|2349.57| +2450840|21522|2450908|74191|950103|4768|37033|31600|1138810|1122|26587|4|88|8|5|1627|276|1285|37|80.54|240.81|130.03|4098.86|4811.11|2979.98|8909.97|432.99|0.00|1158.10|4811.11|5244.10|5969.21|6402.20|1831.13| +2450840|21522|2450844|74191|950103|4768|37033|31600|1138810|1122|26587|4|62|20|5|16466|212|1285|40|82.69|120.72|32.59|3525.20|1303.60|3307.60|4828.80|52.14|0.00|1641.60|1303.60|1355.74|2945.20|2997.34|-2004.00| +2450840|21522|2450847|74191|950103|4768|37033|31600|1138810|1122|26587|4|72|5|2|14326|200|1285|60|36.44|100.57|50.28|3017.40|3016.80|2186.40|6034.20|38.01|1116.21|241.20|1900.59|1938.60|2141.79|2179.80|-285.81| +2450840|21522|2450924|74191|950103|4768|37033|31600|1138810|1122|26587|4|84|7|2|1940|296|1285|61|82.78|238.40|219.32|1163.88|13378.52|5049.58|14542.40|936.49|0.00|1890.39|13378.52|14315.01|15268.91|16205.40|8328.94| +2450840|21522|2450901|74191|950103|4768|37033|31600|1138810|1122|26587|4|67|13|2|10411|296|1285|64|62.39|111.05|57.74|3411.84|3695.36|3992.96|7107.20|36.95|0.00|3553.28|3695.36|3732.31|7248.64|7285.59|-297.60| +2450840|21522|2450925|74191|950103|4768|37033|31600|1138810|1122|26587|4|70|3|5|10753|47|1285|86|55.79|93.16|85.70|641.56|7370.20|4797.94|8011.76|294.80|0.00|79.98|7370.20|7665.00|7450.18|7744.98|2572.26| +2450840|21522|2450926|74191|950103|4768|37033|31600|1138810|1122|26587|4|2|18|3|9199|256|1285|55|70.90|90.75|47.19|2395.80|2595.45|3899.50|4991.25|155.72|0.00|149.60|2595.45|2751.17|2745.05|2900.77|-1304.05| +2450840|21522|2450884|74191|950103|4768|37033|31600|1138810|1122|26587|4|59|10|5|14714|80|1285|22|99.27|246.18|108.31|3033.14|2382.82|2183.94|5415.96|214.45|0.00|812.24|2382.82|2597.27|3195.06|3409.51|198.88| +2450840|21522|2450898|74191|950103|4768|37033|31600|1138810|1122|26587|4|25|15|2|3088|76|1285|63|21.93|28.72|26.70|127.26|1682.10|1381.59|1809.36|2.01|1581.17|162.54|100.93|102.94|263.47|265.48|-1280.66| +2450840|21522|2450891|74191|950103|4768|37033|31600|1138810|1122|26587|4|89|2|4|7960|198|1285|69|40.33|58.88|40.03|1300.65|2762.07|2782.77|4062.72|193.34|0.00|1990.65|2762.07|2955.41|4752.72|4946.06|-20.70| +2450840|21522|2450902|74191|950103|4768|37033|31600|1138810|1122|26587|4|83|12|5|2354|78|1285|90|23.83|37.65|13.93|2134.80|1253.70|2144.70|3388.50|100.29|0.00|236.70|1253.70|1353.99|1490.40|1590.69|-891.00| +2450840|76840|2450864|8633|948315|5597|34032|8633|948315|5597|34032|2|22|7|4|13844|126|1286|88|23.69|23.92|4.30|1726.56|378.40|2084.72|2104.96|15.13|0.00|968.00|378.40|393.53|1346.40|1361.53|-1706.32| +2450840|76840|2450863|8633|948315|5597|34032|8633|948315|5597|34032|2|84|19|4|3184|92|1286|89|72.20|158.84|25.41|11875.27|2261.49|6425.80|14136.76|45.22|0.00|5089.02|2261.49|2306.71|7350.51|7395.73|-4164.31| +2450840|76840|2450866|8633|948315|5597|34032|8633|948315|5597|34032|2|76|5|5|6800|47|1286|27|89.67|184.72|121.91|1695.87|3291.57|2421.09|4987.44|197.49|0.00|1246.86|3291.57|3489.06|4538.43|4735.92|870.48| +2450840|76840|2450892|8633|948315|5597|34032|8633|948315|5597|34032|2|105|5|1|12878|140|1286|4|73.58|164.08|137.82|105.04|551.28|294.32|656.32|49.61|0.00|183.76|551.28|600.89|735.04|784.65|256.96| +2450840|76840|2450918|8633|948315|5597|34032|8633|948315|5597|34032|2|2|4|4|11968|21|1286|51|75.84|186.56|46.64|7135.92|2378.64|3867.84|9514.56|47.57|0.00|4471.68|2378.64|2426.21|6850.32|6897.89|-1489.20| +2450840|76840|2450876|8633|948315|5597|34032|8633|948315|5597|34032|2|1|2|2|13192|237|1286|97|62.37|176.50|10.59|16093.27|1027.23|6049.89|17120.50|24.03|760.15|7190.61|267.08|291.11|7457.69|7481.72|-5782.81| +2450840|55658|2450849|3838|1745443|5095|3547|3838|1745443|5095|3547|4|71|16|3|5095|292|1287|97|73.59|197.95|98.97|9601.06|9600.09|7138.23|19201.15|65.28|3072.02|191.09|6528.07|6593.35|6719.16|6784.44|-610.16| +2450840|55658|2450877|3838|1745443|5095|3547|3838|1745443|5095|3547|4|29|18|3|1171|20|1287|96|8.46|19.71|4.13|1495.68|396.48|812.16|1892.16|27.75|0.00|226.56|396.48|424.23|623.04|650.79|-415.68| +2450840|55658|2450851|3838|1745443|5095|3547|3838|1745443|5095|3547|4|20|10|1|11701|39|1287|32|84.04|93.28|85.81|239.04|2745.92|2689.28|2984.96|247.13|0.00|1402.88|2745.92|2993.05|4148.80|4395.93|56.64| +2450840|55658|2450887|3838|1745443|5095|3547|3838|1745443|5095|3547|4|90|18|4|7840|207|1287|34|34.33|82.39|4.11|2661.52|139.74|1167.22|2801.26|0.00|6.98|1288.26|132.76|132.76|1421.02|1421.02|-1034.46| +2450840|55658|2450930|3838|1745443|5095|3547|3838|1745443|5095|3547|4|81|7|3|14402|194|1287|63|72.56|117.54|109.31|518.49|6886.53|4571.28|7405.02|275.46|0.00|1036.35|6886.53|7161.99|7922.88|8198.34|2315.25| +2450840|55658|2450871|3838|1745443|5095|3547|3838|1745443|5095|3547|4|85|1|3|3776|2|1287|51|98.63|234.73|75.11|8140.62|3830.61|5030.13|11971.23|76.61|0.00|3231.87|3830.61|3907.22|7062.48|7139.09|-1199.52| +2450840|33988|2450909|46228|379645|2662|16079|46228|379645|2662|16079|1|9|14|4|9769|151|1288|7|78.07|90.56|85.12|38.08|595.84|546.49|633.92|29.49|5.95|120.40|589.89|619.38|710.29|739.78|43.40| +2450840|33988|2450847|46228|379645|2662|16079|46228|379645|2662|16079|1|41|15|2|8218|167|1288|46|2.88|3.13|1.06|95.22|48.76|132.48|143.98|0.48|0.00|40.02|48.76|49.24|88.78|89.26|-83.72| +2450840|33988|2450895|46228|379645|2662|16079|46228|379645|2662|16079|1|29|17|5|3062|149|1288|66|85.80|120.12|69.66|3330.36|4597.56|5662.80|7927.92|367.80|0.00|1188.66|4597.56|4965.36|5786.22|6154.02|-1065.24| +2450840|33988|2450846|46228|379645|2662|16079|46228|379645|2662|16079|1|67|4|2|6673|268|1288|14|52.50|71.40|3.57|949.62|49.98|735.00|999.60|1.71|25.48|29.96|24.50|26.21|54.46|56.17|-710.50| +2450840|33988|2450914|46228|379645|2662|16079|46228|379645|2662|16079|1|13|20|5|15446|13|1288|30|30.05|30.95|29.09|55.80|872.70|901.50|928.50|0.87|785.43|380.40|87.27|88.14|467.67|468.54|-814.23| +2450840|33988|2450882|46228|379645|2662|16079|46228|379645|2662|16079|1|105|7|3|1903|132|1288|24|46.67|96.14|22.11|1776.72|530.64|1120.08|2307.36|5.30|0.00|553.68|530.64|535.94|1084.32|1089.62|-589.44| +2450840|33988|2450924|46228|379645|2662|16079|46228|379645|2662|16079|1|104|6|5|10804|131|1288|46|73.95|117.58|12.93|4813.90|594.78|3401.70|5408.68|5.94|0.00|53.82|594.78|600.72|648.60|654.54|-2806.92| +2450840|33988|2450915|46228|379645|2662|16079|46228|379645|2662|16079|1|81|19|3|3322|84|1288|1|3.66|10.72|10.39|0.33|10.39|3.66|10.72|0.10|8.93|1.07|1.46|1.56|2.53|2.63|-2.20| +2450840|33988|2450844|46228|379645|2662|16079|46228|379645|2662|16079|1|97|10|2|962|291|1288|7|78.87|139.59|80.96|410.41|566.72|552.09|977.13|0.00|0.00|214.90|566.72|566.72|781.62|781.62|14.63| +2450840|33988|2450912|46228|379645|2662|16079|46228|379645|2662|16079|1|28|16|3|11575|98|1288|1|61.66|116.53|4.66|111.87|4.66|61.66|116.53|0.04|0.00|54.76|4.66|4.70|59.42|59.46|-57.00| +2450840|33988|2450868|46228|379645|2662|16079|46228|379645|2662|16079|1|92|6|5|91|15|1288|34|7.63|8.77|3.15|191.08|107.10|259.42|298.18|5.05|43.91|122.06|63.19|68.24|185.25|190.30|-196.23| +2450840|33988|2450870|46228|379645|2662|16079|46228|379645|2662|16079|1|48|14|3|12277|181|1288|21|92.39|149.67|115.24|723.03|2420.04|1940.19|3143.07|217.80|0.00|0.00|2420.04|2637.84|2420.04|2637.84|479.85| +2450840|33988|2450889|46228|379645|2662|16079|46228|379645|2662|16079|1|75|12|5|8186|9|1288|8|17.02|17.36|14.92|19.52|119.36|136.16|138.88|8.35|0.00|40.24|119.36|127.71|159.60|167.95|-16.80| +2450840|133|2450866|39730|517825|2673|48321|39730|517825|2673|48321|1|12|9|3|1792|115|1289|30|6.36|6.99|2.16|144.90|64.80|190.80|209.70|3.88|0.00|81.60|64.80|68.68|146.40|150.28|-126.00| +2450840|133|2450851|39730|517825|2673|48321|39730|517825|2673|48321|1|27|20|3|2372|257|1289|26|66.36|78.30|47.76|794.04|1241.76|1725.36|2035.80|62.08|0.00|529.10|1241.76|1303.84|1770.86|1832.94|-483.60| +2450840|133|2450912|39730|517825|2673|48321|39730|517825|2673|48321|1|83|5|3|8354|14|1289|35|94.35|144.35|72.17|2526.30|2525.95|3302.25|5052.25|101.03|0.00|2273.25|2525.95|2626.98|4799.20|4900.23|-776.30| +2450840|133|2450856|39730|517825|2673|48321|39730|517825|2673|48321|1|26|12|5|16532|198|1289|58|45.34|60.75|49.20|669.90|2853.60|2629.72|3523.50|166.93|998.76|246.50|1854.84|2021.77|2101.34|2268.27|-774.88| +2450840|133|2450922|39730|517825|2673|48321|39730|517825|2673|48321|1|51|3|2|5318|177|1289|71|99.77|170.60|95.53|5329.97|6782.63|7083.67|12112.60|542.61|0.00|6056.30|6782.63|7325.24|12838.93|13381.54|-301.04| +2450840|133|2450928|39730|517825|2673|48321|39730|517825|2673|48321|1|97|5|5|3760|228|1289|68|75.42|217.96|89.36|8744.80|6076.48|5128.56|14821.28|182.29|0.00|6372.96|6076.48|6258.77|12449.44|12631.73|947.92| +2450840|133|2450925|39730|517825|2673|48321|39730|517825|2673|48321|1|58|18|1|12278|144|1289|22|52.26|69.50|19.46|1100.88|428.12|1149.72|1529.00|0.00|0.00|275.22|428.12|428.12|703.34|703.34|-721.60| +2450840|133|2450849|39730|517825|2673|48321|39730|517825|2673|48321|1|70|1|2|13657|186|1289|6|7.50|12.75|5.22|45.18|31.32|45.00|76.50|2.81|0.00|11.46|31.32|34.13|42.78|45.59|-13.68| +2450840|133|2450905|39730|517825|2673|48321|39730|517825|2673|48321|1|54|16|5|3772|166|1289|39|80.62|91.90|79.95|466.05|3118.05|3144.18|3584.10|218.26|0.00|788.19|3118.05|3336.31|3906.24|4124.50|-26.13| +2450840|63213|2450901|72223|1505516|6009|43302|72223|1505516|6009|43302|1|106|1|2|6634|14|1290|97|17.46|22.69|14.29|814.80|1386.13|1693.62|2200.93|13.86|0.00|307.49|1386.13|1399.99|1693.62|1707.48|-307.49| +2450840|63213|2450928|72223|1505516|6009|43302|72223|1505516|6009|43302|1|18|8|1|6331|9|1290|51|69.12|103.68|8.29|4864.89|422.79|3525.12|5287.68|33.82|0.00|105.57|422.79|456.61|528.36|562.18|-3102.33| +2450840|63213|2450851|72223|1505516|6009|43302|72223|1505516|6009|43302|1|21|14|5|3562|188|1290|63|93.60|175.96|47.50|8092.98|2992.50|5896.80|11085.48|89.77|0.00|5320.98|2992.50|3082.27|8313.48|8403.25|-2904.30| +2450840|63213|2450905|72223|1505516|6009|43302|72223|1505516|6009|43302|1|55|10|4|3448|140|1290|25|70.19|86.33|63.02|582.75|1575.50|1754.75|2158.25|141.79|0.00|129.25|1575.50|1717.29|1704.75|1846.54|-179.25| +2450840|63213|2450844|72223|1505516|6009|43302|72223|1505516|6009|43302|1|11|11|5|8266|64|1290|72|16.00|35.52|22.02|972.00|1585.44|1152.00|2557.44|110.98|0.00|331.92|1585.44|1696.42|1917.36|2028.34|433.44| +2450840|63213|2450908|72223|1505516|6009|43302|72223|1505516|6009|43302|1|96|17|2|12628|230|1290|17|85.52|193.27|90.83|1741.48|1544.11|1453.84|3285.59|30.88|0.00|624.24|1544.11|1574.99|2168.35|2199.23|90.27| +2450840|63213|2450927|72223|1505516|6009|43302|72223|1505516|6009|43302|1|21|20|4|1759|37|1290|17|35.42|46.40|33.40|221.00|567.80|602.14|788.80|6.47|352.03|228.65|215.77|222.24|444.42|450.89|-386.37| +2450840|63213|2450912|72223|1505516|6009|43302|72223|1505516|6009|43302|1|102|6|1|3206|68|1290|54|30.04|34.24|8.90|1368.36|480.60|1622.16|1848.96|2.54|225.88|18.36|254.72|257.26|273.08|275.62|-1367.44| +2450840|63213|2450881|72223|1505516|6009|43302|72223|1505516|6009|43302|1|11|1|1|4160|283|1290|53|85.24|213.10|115.07|5195.59|6098.71|4517.72|11294.30|175.03|3598.23|5082.17|2500.48|2675.51|7582.65|7757.68|-2017.24| +2450840|77402|2450924|52535|1375327|3719|17711|52535|1375327|3719|17711|2|43|18|5|15044|144|1291|72|57.65|95.12|8.56|6232.32|616.32|4150.80|6848.64|0.00|0.00|2807.28|616.32|616.32|3423.60|3423.60|-3534.48| +2450840|77402|2450928|52535|1375327|3719|17711|52535|1375327|3719|17711|2|33|3|2|4042|255|1291|41|48.50|96.51|34.74|2532.57|1424.34|1988.50|3956.91|113.94|0.00|1305.44|1424.34|1538.28|2729.78|2843.72|-564.16| +2450840||2450888||1375327||17711|||||2||7|1|12220||1291||||135.73||3664.71|||0.00|0.00|999.27|||4663.98|4663.98|1506.87| +2450840|77402|2450905|52535|1375327|3719|17711|52535|1375327|3719|17711|2|44|4|5|1405|249|1291|22|81.15|127.40|72.61|1205.38|1597.42|1785.30|2802.80|15.97|0.00|84.04|1597.42|1613.39|1681.46|1697.43|-187.88| +2450840|77402|2450876|52535|1375327|3719|17711|52535|1375327|3719|17711|2|65|7|4|17611|60|1291|26|98.27|144.45|56.33|2291.12|1464.58|2555.02|3755.70|39.54|805.51|826.02|659.07|698.61|1485.09|1524.63|-1895.95| +2450840|34257|2450908|53117|1820681|245|1809|53117|1820681|245|1809|4|71|11|1|16513|48|1292|48|5.13|11.43|4.11|351.36|197.28|246.24|548.64|13.80|0.00|87.36|197.28|211.08|284.64|298.44|-48.96| +2450840|34257|2450906|53117|1820681|245|1809|53117|1820681|245|1809|4|7|5|1|9355|115|1292|55|3.99|7.26|7.11|8.25|391.05|219.45|399.30|19.55|0.00|79.75|391.05|410.60|470.80|490.35|171.60| +2450840|34257|2450922|53117|1820681|245|1809|53117|1820681|245|1809|4|4|16|1|16892|281|1292|44|83.01|175.15|0.00|7706.60|0.00|3652.44|7706.60|0.00|0.00|1078.88|0.00|0.00|1078.88|1078.88|-3652.44| +2450840|34257|2450904|53117|1820681|245|1809|53117|1820681|245|1809|4|23|9|5|14479|264|1292|85|79.72|143.49|2.86|11953.55|243.10|6776.20|12196.65|14.58|0.00|243.10|243.10|257.68|486.20|500.78|-6533.10| +2450840|34257|2450882|53117|1820681|245|1809|53117|1820681|245|1809|4|48|12|4|12730|58|1292|52|58.57|79.65|78.85|41.60|4100.20|3045.64|4141.80|246.01|0.00|372.32|4100.20|4346.21|4472.52|4718.53|1054.56| +2450840|34257|2450847|53117|1820681|245|1809|53117|1820681|245|1809|4|12|17|5|15913|261|1292|84|93.11|103.35|60.97|3559.92|5121.48|7821.24|8681.40|0.00|0.00|260.40|5121.48|5121.48|5381.88|5381.88|-2699.76| +2450840|34257|2450916|53117|1820681|245|1809|53117|1820681|245|1809|4|39|11|1|13060|56|1292|20|17.12|38.34|8.43|598.20|168.60|342.40|766.80|15.17|0.00|161.00|168.60|183.77|329.60|344.77|-173.80| +2450840|34257|2450897|53117|1820681|245|1809|53117|1820681|245|1809|4|61|20|1|15292|139|1292|89|73.37|190.76|162.14|2547.18|14430.46|6529.93|16977.64|316.02|3896.22|4583.50|10534.24|10850.26|15117.74|15433.76|4004.31| +2450840|34257|2450858|53117|1820681|245|1809|53117|1820681|245|1809|4|30|12|4|13906|16|1292|94|83.93|186.32|44.71|13311.34|4202.74|7889.42|17514.08|294.19|0.00|3327.60|4202.74|4496.93|7530.34|7824.53|-3686.68| +2450840|34257|2450876|53117|1820681|245|1809|53117|1820681|245|1809|4|49|2|5|10747|235|1292|85|76.37|196.27|9.81|15849.10|833.85|6491.45|16682.95|41.69|0.00|833.85|833.85|875.54|1667.70|1709.39|-5657.60| +2450840|34257|2450852|53117|1820681|245|1809|53117|1820681|245|1809|4|33|5|2|12686|116|1292|20|91.32|105.01|77.70|546.20|1554.00|1826.40|2100.20|31.08|0.00|504.00|1554.00|1585.08|2058.00|2089.08|-272.40| +2450840|30062|2450906|51579|246557|6782|40207|51579|246557|6782|40207|1|85|10|3|8156|6|1293|22|14.55|36.81|15.46|469.70|340.12|320.10|809.82|26.93|3.40|364.32|336.72|363.65|701.04|727.97|16.62| +2450840|30062|2450872|51579|246557|6782|40207|51579|246557|6782|40207|1|54|12|3|16336|106|1293|39|49.26|69.94|46.16|927.42|1800.24|1921.14|2727.66|36.00|0.00|627.12|1800.24|1836.24|2427.36|2463.36|-120.90| +2450840|30062|2450881|51579|246557|6782|40207|51579|246557|6782|40207|1|54|10|3|14845|269|1293|41|48.53|76.67|30.66|1886.41|1257.06|1989.73|3143.47|12.57|0.00|1194.33|1257.06|1269.63|2451.39|2463.96|-732.67| +2450840|30062|2450925|51579|246557|6782|40207|51579|246557|6782|40207|1|53|15|3|8768|239|1293|66|67.43|173.96|137.42|2411.64|9069.72|4450.38|11481.36|244.88|906.97|4132.92|8162.75|8407.63|12295.67|12540.55|3712.37| +2450840|30062|2450859|51579|246557|6782|40207|51579|246557|6782|40207|1|66|9|2|11888|67|1293|15|55.39|139.58|118.64|314.10|1779.60|830.85|2093.70|88.98|0.00|460.50|1779.60|1868.58|2240.10|2329.08|948.75| +2450840|30062|2450875|51579|246557|6782|40207|51579|246557|6782|40207|1|37|18|2|2632|140|1293|95|85.07|189.70|34.14|14778.20|3243.30|8081.65|18021.50|291.89|0.00|3062.80|3243.30|3535.19|6306.10|6597.99|-4838.35| +2450840|30062|2450849|51579|246557|6782|40207|51579|246557|6782|40207|1|16|18|1|1699|54|1293|4|36.45|78.36|67.38|43.92|269.52|145.80|313.44|5.39|0.00|131.64|269.52|274.91|401.16|406.55|123.72| +2450840|30062|2450881|51579|246557|6782|40207|51579|246557|6782|40207|1|78|1|4|5890|60|1293|8|74.35|156.13|15.61|1124.16|124.88|594.80|1249.04|8.74|0.00|274.72|124.88|133.62|399.60|408.34|-469.92| +2450840|30062|2450898|51579|246557|6782|40207|51579|246557|6782|40207|1|70|20|5|8140|49|1293|89|72.19|93.84|48.79|4009.45|4342.31|6424.91|8351.76|260.53|0.00|834.82|4342.31|4602.84|5177.13|5437.66|-2082.60| +2450840|30062|2450852|51579|246557|6782|40207|51579|246557|6782|40207|1|69|6|4|10384|159|1293|47|94.86|240.94|142.15|4643.13|6681.05|4458.42|11324.18|334.05|0.00|226.07|6681.05|7015.10|6907.12|7241.17|2222.63| +2450840|30062|2450877|51579|246557|6782|40207|51579|246557|6782|40207|1|58|13|1|6676|294|1293|24|36.56|36.92|22.15|354.48|531.60|877.44|886.08|15.94|0.00|318.96|531.60|547.54|850.56|866.50|-345.84| +2450840|37538|2450870|48112|1579244|4776|21508|48112|1579244|4776|21508|4|17|16|3|7096|14|1294|12|64.52|148.39|111.29|445.20|1335.48|774.24|1780.68|80.12|0.00|498.48|1335.48|1415.60|1833.96|1914.08|561.24| +2450840|37538|2450874|48112|1579244|4776|21508|48112|1579244|4776|21508|4|94|9|2|9614|75|1294|73|42.68|85.36|53.77|2306.07|3925.21|3115.64|6231.28|39.25|0.00|2242.56|3925.21|3964.46|6167.77|6207.02|809.57| +2450840|37538|2450903|48112|1579244|4776|21508|48112|1579244|4776|21508|4|99|17|4|5836|70|1294|76|98.05|261.79|151.83|8356.96|11539.08|7451.80|19896.04|807.73|0.00|9351.04|11539.08|12346.81|20890.12|21697.85|4087.28| +2450840|37538|2450879|48112|1579244|4776|21508|48112|1579244|4776|21508|4|62|18|2|14984|88|1294|15|32.24|87.69|7.89|1197.00|118.35|483.60|1315.35|10.65|0.00|433.95|118.35|129.00|552.30|562.95|-365.25| +2450840|37538|2450857|48112|1579244|4776|21508|48112|1579244|4776|21508|4|53|17|5|13036|178|1294|37|74.02|190.23|36.14|5701.33|1337.18|2738.74|7038.51|26.74|0.00|1125.91|1337.18|1363.92|2463.09|2489.83|-1401.56| +2450840|37538|2450898|48112|1579244|4776|21508|48112|1579244|4776|21508|4|79|2|2|11449|172|1294|81|33.94|73.31|36.65|2969.46|2968.65|2749.14|5938.11|29.68|0.00|1840.32|2968.65|2998.33|4808.97|4838.65|219.51| +2450840|37538|2450863|48112|1579244|4776|21508|48112|1579244|4776|21508|4|15|9|1|1348|46|1294|90|64.22|100.18|27.04|6582.60|2433.60|5779.80|9016.20|219.02|0.00|3606.30|2433.60|2652.62|6039.90|6258.92|-3346.20| +2450840|37538|2450845|48112|1579244|4776|21508|48112|1579244|4776|21508|4|40|3|5|12464|44|1294|70|26.58|61.93|16.72|3164.70|1170.40|1860.60|4335.10|31.13|725.64|779.80|444.76|475.89|1224.56|1255.69|-1415.84| +2450840|37538|2450882|48112|1579244|4776|21508|48112|1579244|4776|21508|4|78|11|5|16633|39|1294|75|7.97|9.96|1.19|657.75|89.25|597.75|747.00|0.89|0.00|306.00|89.25|90.14|395.25|396.14|-508.50| +2450840|37538|2450907|48112|1579244|4776|21508|48112|1579244|4776|21508|4|93|1|3|9580|32|1294|18|26.43|48.63|12.64|647.82|227.52|475.74|875.34|20.47|0.00|0.00|227.52|247.99|227.52|247.99|-248.22| +2450840|37538|2450844|48112|1579244|4776|21508|48112|1579244|4776|21508|4|72|7|3|2726|237|1294|31|93.28|100.74|58.42|1311.92|1811.02|2891.68|3122.94|126.77|0.00|1280.30|1811.02|1937.79|3091.32|3218.09|-1080.66| +2450840|37538|2450899|48112|1579244|4776|21508|48112|1579244|4776|21508|4|10|17|5|8690|189|1294|50|82.41|112.07|65.00|2353.50|3250.00|4120.50|5603.50|292.50|0.00|112.00|3250.00|3542.50|3362.00|3654.50|-870.50| +2450840|37538|2450887|48112|1579244|4776|21508|48112|1579244|4776|21508|4|36|14|2|8270|3|1294|9|83.06|221.77|48.78|1556.91|439.02|747.54|1995.93|0.00|0.00|219.51|439.02|439.02|658.53|658.53|-308.52| +2450840|73710|2450918|98395|1530896|3844|49145|98395|1530896|3844|49145|2|57|2|3|8534|290|1295|56|61.84|165.73|125.95|2227.68|7053.20|3463.04|9280.88|423.19|0.00|742.00|7053.20|7476.39|7795.20|8218.39|3590.16| +2450840|73710|2450894|98395|1530896|3844|49145|98395|1530896|3844|49145|2|70|10|2|14762|73|1295|80|60.06|99.09|6.93|7372.80|554.40|4804.80|7927.20|49.89|0.00|2456.80|554.40|604.29|3011.20|3061.09|-4250.40| +2450840|73710|2450864|98395|1530896|3844|49145|98395|1530896|3844|49145|2|108|9|1|1639|245|1295|2|45.87|112.38|106.76|11.24|213.52|91.74|224.76|8.32|46.97|20.22|166.55|174.87|186.77|195.09|74.81| +2450840|73710|2450864|98395|1530896|3844|49145|98395|1530896|3844|49145|2|3|18|4|11144|70|1295|94|62.82|150.13|39.03|10443.40|3668.82|5905.08|14112.22|330.19|0.00|2115.94|3668.82|3999.01|5784.76|6114.95|-2236.26| +2450840|73710|2450849|98395|1530896|3844|49145|98395|1530896|3844|49145|2|37|7|4|15874|204|1295|29|82.41|143.39|84.60|1704.91|2453.40|2389.89|4158.31|49.06|0.00|82.94|2453.40|2502.46|2536.34|2585.40|63.51| +2450840|73710|2450845|98395|1530896|3844|49145|98395|1530896|3844|49145|2|15|13|2|15130|65|1295|21|62.46|166.76|148.41|385.35|3116.61|1311.66|3501.96|280.49|0.00|665.28|3116.61|3397.10|3781.89|4062.38|1804.95| +2450840|73710|2450921|98395|1530896|3844|49145|98395|1530896|3844|49145|2|4|8|1|14899|190|1295|27|2.67|4.96|1.48|93.96|39.96|72.09|133.92|1.88|16.38|10.53|23.58|25.46|34.11|35.99|-48.51| +2450840|73710|2450851|98395|1530896|3844|49145|98395|1530896|3844|49145|2|34|12|1|10489|36|1295|63|99.23|220.29|30.84|11935.35|1942.92|6251.49|13878.27|77.71|0.00|4301.64|1942.92|2020.63|6244.56|6322.27|-4308.57| +2450840|73710|2450878|98395|1530896|3844|49145|98395|1530896|3844|49145|2|41|19|1|15673|292|1295|10|65.39|72.58|66.04|65.40|660.40|653.90|725.80|0.00|0.00|159.60|660.40|660.40|820.00|820.00|6.50| +2450840|73710|2450868|98395|1530896|3844|49145|98395|1530896|3844|49145|2|1|16|4|1948|240|1295|40|12.07|16.89|16.21|27.20|648.40|482.80|675.60|32.42|0.00|263.20|648.40|680.82|911.60|944.02|165.60| +2450840|73710|2450911|98395|1530896|3844|49145|98395|1530896|3844|49145|2|44|13|2|15625|138|1295|23|76.44|147.52|88.51|1357.23|2035.73|1758.12|3392.96|142.50|0.00|0.00|2035.73|2178.23|2035.73|2178.23|277.61| +2450840|73710|2450927|98395|1530896|3844|49145|98395|1530896|3844|49145|2|55|2|4|17944|176|1295|74|54.20|92.68|26.87|4869.94|1988.38|4010.80|6858.32|99.41|0.00|1645.76|1988.38|2087.79|3634.14|3733.55|-2022.42| +2450840|31322|2450904|96255|1536385|5668|23398|96255|1536385|5668|23398|2|25|11|4|10438|200|1296|80|16.19|46.62|22.84|1902.40|1827.20|1295.20|3729.60|36.54|0.00|671.20|1827.20|1863.74|2498.40|2534.94|532.00| +2450840|31322|2450869|96255|1536385|5668|23398|96255|1536385|5668|23398|2|3|3|4|8407|191|1296|41|81.44|167.76|100.65|2751.51|4126.65|3339.04|6878.16|371.39|0.00|3232.44|4126.65|4498.04|7359.09|7730.48|787.61| +2450840|31322|2450861|96255|1536385|5668|23398|96255|1536385|5668|23398|2|3|10|1|12074|155|1296|16|60.00|100.80|3.02|1564.48|48.32|960.00|1612.80|2.89|0.00|209.60|48.32|51.21|257.92|260.81|-911.68| +2450840|31322|2450901|96255|1536385|5668|23398|96255|1536385|5668|23398|2|70|14|2|12880|173|1296|41|51.94|131.40|91.98|1616.22|3771.18|2129.54|5387.40|301.69|0.00|1077.48|3771.18|4072.87|4848.66|5150.35|1641.64| +2450840|31322|2450900|96255|1536385|5668|23398|96255|1536385|5668|23398|2|88|16|4|6070|53|1296|49|33.14|80.19|71.36|432.67|3496.64|1623.86|3929.31|244.76|0.00|156.80|3496.64|3741.40|3653.44|3898.20|1872.78| +2450840|31322|2450899|96255|1536385|5668|23398|96255|1536385|5668|23398|2|88|19|1|10816|137|1296|54|11.57|32.16|1.60|1650.24|86.40|624.78|1736.64|4.32|0.00|260.28|86.40|90.72|346.68|351.00|-538.38| +2450840||||1536385||23398|||5668||2|12|17||1162||1296|3||||19.14|2.61|13.95|21.75|0.23|0.00|1.08|||3.69|3.92|-11.34| +2450840|31322|2450848|96255|1536385|5668|23398|96255|1536385|5668|23398|2|44|19|5|13588|279|1296|97|10.90|28.44|14.22|1379.34|1379.34|1057.30|2758.68|0.00|0.00|799.28|1379.34|1379.34|2178.62|2178.62|322.04| +2450840|31322|2450872|96255|1536385|5668|23398|96255|1536385|5668|23398|2|47|1|5|16744|281|1296|16|50.88|110.91|43.25|1082.56|692.00|814.08|1774.56|0.00|0.00|319.36|692.00|692.00|1011.36|1011.36|-122.08| +2450840|31322|2450905|96255|1536385|5668|23398|96255|1536385|5668|23398|2|72|8|5|9574|85|1296|49|32.04|80.10|66.48|667.38|3257.52|1569.96|3924.90|260.60|0.00|352.80|3257.52|3518.12|3610.32|3870.92|1687.56| +2450840|18654|2450842|66965|1041626|3840|24901|66965|1041626|3840|24901|1|75|1|2|10004|209|1297|63|34.63|57.48|56.33|72.45|3548.79|2181.69|3621.24|141.95|0.00|1266.93|3548.79|3690.74|4815.72|4957.67|1367.10| +2450840|18654|2450849|66965|1041626|3840|24901|66965|1041626|3840|24901|1|65|14|4|13870|277|1297|86|3.58|4.54|0.09|382.70|7.74|307.88|390.44|0.07|0.00|26.66|7.74|7.81|34.40|34.47|-300.14| +2450840|18654|2450926|66965|1041626|3840|24901|66965|1041626|3840|24901|1|81|20|1|13168|85|1297|36|15.75|42.36|17.36|900.00|624.96|567.00|1524.96|49.99|0.00|518.40|624.96|674.95|1143.36|1193.35|57.96| +2450840|18654|2450879|66965|1041626|3840|24901|66965|1041626|3840|24901|1|26|10|2|10|164|1297|2|62.72|146.13|138.82|14.62|277.64|125.44|292.26|0.00|0.00|61.36|277.64|277.64|339.00|339.00|152.20| +2450840|18654|2450909|66965|1041626|3840|24901|66965|1041626|3840|24901|1|84|12|3|12550|137|1297|88|94.59|204.31|61.29|12585.76|5393.52|8323.92|17979.28|269.67|0.00|3056.24|5393.52|5663.19|8449.76|8719.43|-2930.40| +2450840|18654|2450916|66965|1041626|3840|24901|66965|1041626|3840|24901|1|82|3|1|3026|1|1297|90|38.07|89.46|39.36|4509.00|3542.40|3426.30|8051.40|212.54|0.00|2575.80|3542.40|3754.94|6118.20|6330.74|116.10| +2450840|18654|2450844|66965|1041626|3840|24901|66965|1041626|3840|24901|1|24|20|1|8191|288|1297|91|52.15|148.62|49.04|9061.78|4462.64|4745.65|13524.42|178.50|0.00|4732.91|4462.64|4641.14|9195.55|9374.05|-283.01| +2450840|18654|2450929|66965|1041626|3840|24901|66965|1041626|3840|24901|1|69|13|2|16844|199|1297|65|20.66|52.06|21.86|1963.00|1420.90|1342.90|3383.90|127.88|0.00|778.05|1420.90|1548.78|2198.95|2326.83|78.00| +2450840|18654|2450874|66965|1041626|3840|24901|66965|1041626|3840|24901|1|10|20|2|13994|136|1297|85|70.99|153.33|127.26|2215.95|10817.10|6034.15|13033.05|268.26|7463.79|5604.05|3353.31|3621.57|8957.36|9225.62|-2680.84| +2450840|18654|2450903|66965|1041626|3840|24901|66965|1041626|3840|24901|1|39|9|4|1136|216|1297|54|63.93|83.10|45.70|2019.60|2467.80|3452.22|4487.40|49.35|0.00|807.30|2467.80|2517.15|3275.10|3324.45|-984.42| +2450840|47340|2450866|22587|1682898|2409|33967|22587|1682898|2409|33967|2|85|7|2|1802|177|1298|4|81.30|180.48|137.16|173.28|548.64|325.20|721.92|27.43|0.00|259.88|548.64|576.07|808.52|835.95|223.44| +2450840|47340|2450905|22587|1682898|2409|33967|22587|1682898|2409|33967|2|1|5|1|739|145|1298|40|11.99|32.85|16.42|657.20|656.80|479.60|1314.00|59.11|0.00|643.60|656.80|715.91|1300.40|1359.51|177.20| +2450840|47340|2450851|22587|1682898|2409|33967|22587|1682898|2409|33967|2|76|5|5|16903|225|1298|31|87.92|202.21|129.41|2256.80|4011.71|2725.52|6268.51|200.58|0.00|2507.28|4011.71|4212.29|6518.99|6719.57|1286.19| +2450840|47340|2450876|22587|1682898|2409|33967|22587|1682898|2409|33967|2|76|9|3|4310|119|1298|99|80.12|193.08|5.79|18541.71|573.21|7931.88|19114.92|17.19|0.00|1719.63|573.21|590.40|2292.84|2310.03|-7358.67| +2450840|47340|2450894|22587|1682898|2409|33967|22587|1682898|2409|33967|2|4|15|2|2161|39|1298|35|59.71|94.93|48.41|1628.20|1694.35|2089.85|3322.55|0.00|0.00|1162.70|1694.35|1694.35|2857.05|2857.05|-395.50| +2450840|47340|2450880|22587|1682898|2409|33967|22587|1682898|2409|33967|2|66|7|3|14138|258|1298|30|87.52|102.39|44.02|1751.10|1320.60|2625.60|3071.70|92.44|0.00|1013.40|1320.60|1413.04|2334.00|2426.44|-1305.00| +2450840|47340|2450853|22587|1682898|2409|33967|22587|1682898|2409|33967|2|39|9|1|17860|17|1298|42|46.88|65.63|6.56|2480.94|275.52|1968.96|2756.46|0.00|0.00|165.06|275.52|275.52|440.58|440.58|-1693.44| +2450840|47340|2450868|22587|1682898|2409|33967|22587|1682898|2409|33967|2|104|14|4|5800|200|1298|16|15.14|19.98|5.79|227.04|92.64|242.24|319.68|4.63|0.00|70.24|92.64|97.27|162.88|167.51|-149.60| +2450840|47340|2450873|22587|1682898|2409|33967|22587|1682898|2409|33967|2|61|18|4|12769|153|1298|96|55.15|78.31|30.54|4585.92|2931.84|5294.40|7517.76|234.54|0.00|224.64|2931.84|3166.38|3156.48|3391.02|-2362.56| +2450840|47340|2450882|22587|1682898|2409|33967|22587|1682898|2409|33967|2|5|6|4|3691|198|1298|31|98.94|282.96|274.47|263.19|8508.57|3067.14|8771.76|12.76|8083.14|3508.58|425.43|438.19|3934.01|3946.77|-2641.71| +2450840|47340|2450887|22587|1682898|2409|33967|22587|1682898|2409|33967|2|46|8|4|9958|121|1298|96|48.97|123.89|50.79|7017.60|4875.84|4701.12|11893.44|83.37|3949.43|2735.04|926.41|1009.78|3661.45|3744.82|-3774.71| +2450840|47340|2450902|22587|1682898|2409|33967|22587|1682898|2409|33967|2|25|19|2|986|203|1298|14|89.29|100.00|99.00|14.00|1386.00|1250.06|1400.00|20.51|873.18|84.00|512.82|533.33|596.82|617.33|-737.24| +2450840|47340|2450856|22587|1682898|2409|33967|22587|1682898|2409|33967|2|86|13|1|196|145|1298|16|34.17|56.72|2.26|871.36|36.16|546.72|907.52|0.16|34.35|417.44|1.81|1.97|419.25|419.41|-544.91| +2450840|12475|2450844|41447|1911723|5199|31428|41447|1911723|5199|31428|1|49|15|3|9560|234|1299|46|5.09|13.99|9.51|206.08|437.46|234.14|643.54|21.65|4.37|63.94|433.09|454.74|497.03|518.68|198.95| +2450840|12475|2450923|41447|1911723|5199|31428|41447|1911723|5199|31428|1|55|13|3|13681|78|1299|11|71.25|153.90|138.51|169.29|1523.61|783.75|1692.90|15.23|0.00|389.29|1523.61|1538.84|1912.90|1928.13|739.86| +2450840|12475|2450868|41447|1911723|5199|31428|41447|1911723|5199|31428|1|91|3|4|7862|121|1299|71|92.27|176.23|75.77|7132.66|5379.67|6551.17|12512.33|27.43|2636.03|3878.73|2743.64|2771.07|6622.37|6649.80|-3807.53| +2450840|12475|2450874|41447|1911723|5199|31428|41447|1911723|5199|31428|1|101|17|5|8936|136|1299|100|93.97|138.13|66.30|7183.00|6630.00|9397.00|13813.00|198.90|0.00|5387.00|6630.00|6828.90|12017.00|12215.90|-2767.00| +2450840|12475|2450870|41447|1911723|5199|31428|41447|1911723|5199|31428|1|29|16|2|11026|29|1299|32|10.37|13.68|4.78|284.80|152.96|331.84|437.76|10.70|0.00|13.12|152.96|163.66|166.08|176.78|-178.88| +2450840|12475|2450855|41447|1911723|5199|31428|41447|1911723|5199|31428|1|33|6|1|5995|278|1299|65|13.71|26.73|26.19|35.10|1702.35|891.15|1737.45|136.18|0.00|520.65|1702.35|1838.53|2223.00|2359.18|811.20| +2450840|51273|2450928|79155|1866300|2245|10143|79155|1866300|2245|10143|1|78|4|5|4924|270|1300|93|93.75|97.50|52.65|4171.05|4896.45|8718.75|9067.50|293.78|0.00|1088.10|4896.45|5190.23|5984.55|6278.33|-3822.30| +2450840|51273|2450884|79155|1866300|2245|10143|79155|1866300|2245|10143|1|20|10|2|194|181|1300|66|53.70|131.56|57.88|4862.88|3820.08|3544.20|8682.96|343.80|0.00|694.32|3820.08|4163.88|4514.40|4858.20|275.88| +2450840|51273|2450894|79155|1866300|2245|10143|79155|1866300|2245|10143|1|6|15|4|6506|97|1300|13|32.70|33.35|9.33|312.26|121.29|425.10|433.55|0.00|0.00|160.29|121.29|121.29|281.58|281.58|-303.81| +2450840|51273|2450872|79155|1866300|2245|10143|79155|1866300|2245|10143|1|27|4|5|4330|224|1300|98|46.53|54.44|46.27|800.66|4534.46|4559.94|5335.12|269.34|45.34|1919.82|4489.12|4758.46|6408.94|6678.28|-70.82| +2450840|8476|2450903|90134|1507591|3801|13611|90134|1507591|3801|13611|1|56|6|4|12062|183|1301|43|63.48|121.24|4.84|5005.20|208.12|2729.64|5213.32|4.16|0.00|1146.81|208.12|212.28|1354.93|1359.09|-2521.52| +2450840|8476|2450926|90134|1507591|3801|13611|90134|1507591|3801|13611|1|80|7|2|5858|74|1301|72|1.21|2.67|1.46|87.12|105.12|87.12|192.24|8.40|0.00|46.08|105.12|113.52|151.20|159.60|18.00| +2450840|8476|2450916|90134|1507591|3801|13611|90134|1507591|3801|13611|1|89|14|1|14464|234|1301|6|81.99|236.95|132.69|625.56|796.14|491.94|1421.70|39.80|0.00|497.58|796.14|835.94|1293.72|1333.52|304.20| +2450840|8476|2450868|90134|1507591|3801|13611|90134|1507591|3801|13611|1|58|18|1|740|114|1301|8|38.86|114.63|67.63|376.00|541.04|310.88|917.04|27.05|0.00|110.00|541.04|568.09|651.04|678.09|230.16| +2450840|8476|2450899|90134|1507591|3801|13611|90134|1507591|3801|13611|1|32|11|4|1772|42|1301|15|42.21|106.79|92.90|208.35|1393.50|633.15|1601.85|0.00|0.00|192.15|1393.50|1393.50|1585.65|1585.65|760.35| +2450840|8476|2450855|90134|1507591|3801|13611|90134|1507591|3801|13611|1|75|12|5|15517|113|1301|81|23.65|52.50|14.17|3104.73|1147.77|1915.65|4252.50|103.29|0.00|680.40|1147.77|1251.06|1828.17|1931.46|-767.88| +2450840|8476|2450867|90134|1507591|3801|13611|90134|1507591|3801|13611|1|13|19|4|14197|79|1301|33|1.75|5.05|1.26|125.07|41.58|57.75|166.65|2.91|0.00|23.10|41.58|44.49|64.68|67.59|-16.17| +2450840|8476|2450872|90134|1507591|3801|13611|90134|1507591|3801|13611|1|20|9|2|8467|270|1301|78|18.23|31.72|2.85|2251.86|222.30|1421.94|2474.16|2.22|0.00|889.98|222.30|224.52|1112.28|1114.50|-1199.64| +2450840|69126|2450926|69373|1518495|6817|47939|69373|1518495|6817|47939|2|12|12|3|11527|208|1302|83|92.59|166.66|139.99|2213.61|11619.17|7684.97|13832.78|580.95|0.00|3596.39|11619.17|12200.12|15215.56|15796.51|3934.20| +2450840|69126|2450913|69373|1518495|6817|47939|69373|1518495|6817|47939|2|74|4|1|13480|27|1302|64|45.57|108.45|70.49|2429.44|4511.36|2916.48|6940.80|360.90|0.00|3331.20|4511.36|4872.26|7842.56|8203.46|1594.88| +2450840|69126|2450849|69373|1518495|6817|47939|69373|1518495|6817|47939|2|40|11|2|6436|191|1302|14|80.69|185.58|128.05|805.42|1792.70|1129.66|2598.12|125.48|0.00|961.24|1792.70|1918.18|2753.94|2879.42|663.04| +2450840|69126|2450920|69373|1518495|6817|47939|69373|1518495|6817|47939|2|52|8|2|4472|17|1302|16|48.25|88.29|23.83|1031.36|381.28|772.00|1412.64|11.43|0.00|127.04|381.28|392.71|508.32|519.75|-390.72| +2450840|69126|2450881|69373|1518495|6817|47939|69373|1518495|6817|47939|2|5|17|5|9412|30|1302|43|94.02|251.97|239.37|541.80|10292.91|4042.86|10834.71|0.00|0.00|2600.21|10292.91|10292.91|12893.12|12893.12|6250.05| +2450840|69126|2450900|69373|1518495|6817|47939|69373|1518495|6817|47939|2|73|1|4|1496|155|1302|60|97.42|132.49|125.86|397.80|7551.60|5845.20|7949.40|75.51|0.00|635.40|7551.60|7627.11|8187.00|8262.51|1706.40| +2450840|69126|2450885|69373|1518495|6817|47939|69373|1518495|6817|47939|2|15|16|2|8560|104|1302|81|15.48|33.74|10.12|1913.22|819.72|1253.88|2732.94|57.38|0.00|873.99|819.72|877.10|1693.71|1751.09|-434.16| +2450840|69126|2450883|69373|1518495|6817|47939|69373|1518495|6817|47939|2|68|20|2|3620|148|1302|68|83.39|176.78|76.01|6852.36|5168.68|5670.52|12021.04|103.37|0.00|5769.80|5168.68|5272.05|10938.48|11041.85|-501.84| +2450840|69126|2450845|69373|1518495|6817|47939|69373|1518495|6817|47939|2|70|2|2|11224|40|1302|23|45.50|116.93|45.60|1640.59|1048.80|1046.50|2689.39|40.58|597.81|295.78|450.99|491.57|746.77|787.35|-595.51| +2450840|69126|2450847|69373|1518495|6817|47939|69373|1518495|6817|47939|2|100|7|5|10723|132|1302|30|3.08|7.23|7.01|6.60|210.30|92.40|216.90|10.51|0.00|0.00|210.30|220.81|210.30|220.81|117.90| +2450840|69126|2450920|69373|1518495|6817|47939|69373|1518495|6817|47939|2|52|20|4|12595|112|1302|85|51.57|68.58|43.89|2098.65|3730.65|4383.45|5829.30|0.00|0.00|524.45|3730.65|3730.65|4255.10|4255.10|-652.80| +2450840|58099|2450913|91399|1494734|5269|40251|91399|1494734|5269|40251|4|73|19|3|17365|267|1303|8|39.13|115.43|111.96|27.76|895.68|313.04|923.44|26.87|0.00|101.52|895.68|922.55|997.20|1024.07|582.64| +2450840|58099|2450890|91399|1494734|5269|40251|91399|1494734|5269|40251|4|9|20|5|676|33|1303|54|97.48|290.49|49.38|13019.94|2666.52|5263.92|15686.46|159.99|0.00|1725.30|2666.52|2826.51|4391.82|4551.81|-2597.40| +2450840|58099|2450902|91399|1494734|5269|40251|91399|1494734|5269|40251|4|28|16|4|4387|145|1303|2|72.64|159.08|34.99|248.18|69.98|145.28|318.16|1.99|3.49|73.16|66.49|68.48|139.65|141.64|-78.79| +2450840|58099|2450856|91399|1494734|5269|40251|91399|1494734|5269|40251|4|32|17|5|2770|254|1303|88|49.31|69.52|10.42|5200.80|916.96|4339.28|6117.76|73.35|0.00|2568.72|916.96|990.31|3485.68|3559.03|-3422.32| +|58099|||1494734|5269|40251|||5269||4|91||4|7483|239|1303|95|68.74|94.86|1.89||179.55|6530.30|9011.70|0.70|156.20|3604.30|||3627.65|3628.35|-6506.95| +2450840|58099|2450915|91399|1494734|5269|40251|91399|1494734|5269|40251|4|95|16|4|6556|174|1303|66|40.50|121.09|121.09|0.00|7991.94|2673.00|7991.94|319.67|0.00|479.16|7991.94|8311.61|8471.10|8790.77|5318.94| +2450840|58099|2450845|91399|1494734|5269|40251|91399|1494734|5269|40251|4|44|6|3|6362|290|1303|36|88.54|120.41|60.20|2167.56|2167.20|3187.44|4334.76|173.37|0.00|823.32|2167.20|2340.57|2990.52|3163.89|-1020.24| +2450840|58099|2450845|91399|1494734|5269|40251|91399|1494734|5269|40251|4|58|1|1|14011|141|1303|34|64.64|108.59|69.49|1329.40|2362.66|2197.76|3692.06|47.25|0.00|1107.38|2362.66|2409.91|3470.04|3517.29|164.90| +2450840|58099|2450901|91399|1494734|5269|40251|91399|1494734|5269|40251|4|31|8|1|17119|22|1303|61|60.73|143.93|15.83|7814.10|965.63|3704.53|8779.73|57.93|0.00|2282.62|965.63|1023.56|3248.25|3306.18|-2738.90| +2450840|58099|2450908|91399|1494734|5269|40251|91399|1494734|5269|40251|4|89|5|4|3010|254|1303|99|4.98|10.50|4.41|602.91|436.59|493.02|1039.50|4.36|0.00|176.22|436.59|440.95|612.81|617.17|-56.43| +2450840|58099|2450874|91399|1494734|5269|40251|91399|1494734|5269|40251|4|51|1|1|13352|131|1303|54|54.66|129.54|53.11|4127.22|2867.94|2951.64|6995.16|258.11|0.00|2238.30|2867.94|3126.05|5106.24|5364.35|-83.70| +2450840|58099|2450848|91399|1494734|5269|40251|91399|1494734|5269|40251|4|84|3|5|13676|137|1303|26|16.54|28.11|14.61|351.00|379.86|430.04|730.86|0.00|0.00|358.02|379.86|379.86|737.88|737.88|-50.18| +2450840|64517|2450870|57457|376470|2683|32449|57457|376470|2683|32449|1|79|20|3|304|86|1304|67|45.69|123.36|69.08|3636.76|4628.36|3061.23|8265.12|138.85|0.00|1404.99|4628.36|4767.21|6033.35|6172.20|1567.13| +2450840|64517|2450886|57457|376470|2683|32449|57457|376470|2683|32449|1|9|9|5|1768|12|1304|11|10.10|24.34|19.47|53.57|214.17|111.10|267.74|2.14|0.00|5.28|214.17|216.31|219.45|221.59|103.07| +2450840|64517|2450856|57457|376470|2683|32449|57457|376470|2683|32449|1|92|20|5|4864|235|1304|67|72.49|126.13|60.54|4394.53|4056.18|4856.83|8450.71|0.00|0.00|4056.18|4056.18|4056.18|8112.36|8112.36|-800.65| +2450840|64517|2450845|57457|376470|2683|32449|57457|376470|2683|32449|1|47|14|5|6584|235|1304|9|52.50|115.50|41.58|665.28|374.22|472.50|1039.50|18.71|0.00|457.38|374.22|392.93|831.60|850.31|-98.28| +2450840|64517|2450910|57457|376470|2683|32449|57457|376470|2683|32449|1|104|18|4|15400|168|1304|56|24.26|61.37|50.93|584.64|2852.08|1358.56|3436.72|199.64|0.00|652.96|2852.08|3051.72|3505.04|3704.68|1493.52| +2450840|64517|2450847|57457|376470|2683|32449|57457|376470|2683|32449|1|107|1|5|16|290|1304|8|64.09|112.15|103.17|71.84|825.36|512.72|897.20|57.77|0.00|197.36|825.36|883.13|1022.72|1080.49|312.64| +2450840|64517|2450849|57457|376470|2683|32449|57457|376470|2683|32449|1|34|15|1|17666|193|1304|24|62.46|139.28|36.21|2473.68|869.04|1499.04|3342.72|52.14|0.00|100.08|869.04|921.18|969.12|1021.26|-630.00| +2450840|64517|2450849|57457|376470|2683|32449|57457|376470|2683|32449|1|58|1|3|17401|240|1304|84|13.70|17.94|8.43|798.84|708.12|1150.80|1506.96|0.00|0.00|59.64|708.12|708.12|767.76|767.76|-442.68| +2450840|64517|2450863|57457|376470|2683|32449|57457|376470|2683|32449|1|47|9|2|16267|255|1304|46|27.73|37.43|14.22|1067.66|654.12|1275.58|1721.78|0.78|575.62|86.02|78.50|79.28|164.52|165.30|-1197.08| +2450840|64517|2450899|57457|376470|2683|32449|57457|376470|2683|32449|1|39|18|5|11059|9|1304|58|89.97|120.55|107.28|769.66|6222.24|5218.26|6991.90|248.88|0.00|2097.28|6222.24|6471.12|8319.52|8568.40|1003.98| +2450840|71687|2450887|47937|548418|6499|40180|47937|548418|6499|40180|1|25|20|2|6926|21|1305|75|14.96|44.43|39.54|366.75|2965.50|1122.00|3332.25|26.68|1631.02|933.00|1334.48|1361.16|2267.48|2294.16|212.48| +2450840|71687|2450915|47937|548418|6499|40180|47937|548418|6499|40180|1|89|6|4|6835|215|1305|80|24.41|40.76|22.41|1468.00|1792.80|1952.80|3260.80|161.35|0.00|945.60|1792.80|1954.15|2738.40|2899.75|-160.00| +2450840|71687|2450927|47937|548418|6499|40180|47937|548418|6499|40180|1|6|3|5|12178|15|1305|88|17.35|49.10|25.53|2074.16|2246.64|1526.80|4320.80|202.19|0.00|604.56|2246.64|2448.83|2851.20|3053.39|719.84| +2450840|71687|2450878|47937|548418|6499|40180|47937|548418|6499|40180|1|101|18|4|9751|46|1305|64|13.85|41.55|1.24|2579.84|79.36|886.40|2659.20|1.33|46.02|1169.92|33.34|34.67|1203.26|1204.59|-853.06| +2450840|71687|2450926|47937|548418|6499|40180|47937|548418|6499|40180|1|95|11|3|2359|297|1305|89|54.55|69.82|49.57|1802.25|4411.73|4854.95|6213.98|2.64|4367.61|621.22|44.12|46.76|665.34|667.98|-4810.83| +2450840|71687|2450913|47937|548418|6499|40180|47937|548418|6499|40180|1|74|5|4|5126|237|1305|15|83.04|205.10|69.73|2030.55|1045.95|1245.60|3076.50|62.75|0.00|1353.60|1045.95|1108.70|2399.55|2462.30|-199.65| +2450840|71687|2450906|47937|548418|6499|40180|47937|548418|6499|40180|1|81|17|2|11546|65|1305|22|57.34|116.40|102.43|307.34|2253.46|1261.48|2560.80|22.53|0.00|384.12|2253.46|2275.99|2637.58|2660.11|991.98| +2450840|71687|2450891|47937|548418|6499|40180|47937|548418|6499|40180|1|57|1|5|9526|199|1305|65|25.10|56.22|51.72|292.50|3361.80|1631.50|3654.30|268.94|0.00|949.65|3361.80|3630.74|4311.45|4580.39|1730.30| +2450840|71687|2450860|47937|548418|6499|40180|47937|548418|6499|40180|1|8|12|1|3484|206|1305|100|89.01|186.03|74.41|11162.00|7441.00|8901.00|18603.00|148.82|0.00|7813.00|7441.00|7589.82|15254.00|15402.82|-1460.00| +2450840|71687|2450865|47937|548418|6499|40180|47937|548418|6499|40180|1|71|10|4|16096|59|1305|1|31.85|39.81|21.89|17.92|21.89|31.85|39.81|1.53|0.00|17.11|21.89|23.42|39.00|40.53|-9.96| +2450840|71687|2450871|47937|548418|6499|40180|47937|548418|6499|40180|1|6|7|4|12217|128|1305|62|84.16|171.68|29.18|8835.00|1809.16|5217.92|10644.16|54.27|0.00|3724.96|1809.16|1863.43|5534.12|5588.39|-3408.76| +2450840|71687|2450889|47937|548418|6499|40180|47937|548418|6499|40180|1|3|18|2|14131|12|1305|86|49.11|101.65|98.60|262.30|8479.60|4223.46|8741.90|0.00|0.00|1835.24|8479.60|8479.60|10314.84|10314.84|4256.14| +2450840|71687|2450889|47937|548418|6499|40180|47937|548418|6499|40180|1|57|5|5|8029|188|1305|10|26.40|44.88|36.80|80.80|368.00|264.00|448.80|1.87|180.32|134.60|187.68|189.55|322.28|324.15|-76.32| +2450840|41171|2450849|13668|462293|1765|16037|33301|1190581|2853|47078|4|89|18|1|12788|49|1306|81|47.00|74.73|23.91|4116.42|1936.71|3807.00|6053.13|30.21|1433.16|2541.78|503.55|533.76|3045.33|3075.54|-3303.45| +2450840|41171|2450844|13668|462293|1765|16037|33301|1190581|2853|47078|4|27|8|1|16666|222|1306|83|96.29|236.87|182.38|4522.67|15137.54|7992.07|19660.21|756.87|0.00|8453.55|15137.54|15894.41|23591.09|24347.96|7145.47| +2450840|41171|2450920|13668|462293|1765|16037|33301|1190581|2853|47078|4|37|4|2|16330|292|1306|97|36.84|50.83|33.03|1726.60|3203.91|3573.48|4930.51|0.00|0.00|788.61|3203.91|3203.91|3992.52|3992.52|-369.57| +2450840|41171|2450903|13668|462293|1765|16037|33301|1190581|2853|47078|4|70|18|5|9092|74|1306|9|83.51|156.99|1.56|1398.87|14.04|751.59|1412.91|0.28|0.00|381.42|14.04|14.32|395.46|395.74|-737.55| +2450840|41171|2450913|13668|462293|1765|16037|33301|1190581|2853|47078|4|33|20|3|17038|149|1306|31|90.90|96.35|48.17|1493.58|1493.27|2817.90|2986.85|0.00|1463.40|1134.91|29.87|29.87|1164.78|1164.78|-2788.03| +2450840|41171|2450920|13668|462293|1765|16037|33301|1190581|2853|47078|4|73|5|5|14228|181|1306|94|47.81|125.74|100.59|2364.10|9455.46|4494.14|11819.56|567.32|0.00|2363.16|9455.46|10022.78|11818.62|12385.94|4961.32| +2450840|53661|2450844|9240|540776|3512|9848|9240|540776|3512|9848|2|59|2|2|4195|6|1307|56|35.18|66.49|33.90|1825.04|1898.40|1970.08|3723.44|56.95|0.00|446.32|1898.40|1955.35|2344.72|2401.67|-71.68| +2450840|53661|2450853|9240|540776|3512|9848|9240|540776|3512|9848|2|74|9|4|10160|223|1307|27|63.53|149.93|128.93|567.00|3481.11|1715.31|4048.11|0.00|0.00|242.73|3481.11|3481.11|3723.84|3723.84|1765.80| +2450840|53661|2450898|9240|540776|3512|9848|9240|540776|3512|9848|2|80|19|1|7288|251|1307|28|83.12|179.53|17.95|4524.24|502.60|2327.36|5026.84|0.00|0.00|1156.12|502.60|502.60|1658.72|1658.72|-1824.76| +2450840|53661|2450862|9240|540776|3512|9848|9240|540776|3512|9848|2|70|1|5|148|133|1307|66|8.90|11.30|8.13|209.22|536.58|587.40|745.80|26.82|0.00|66.66|536.58|563.40|603.24|630.06|-50.82| +2450840|53661|2450910|9240|540776|3512|9848|9240|540776|3512|9848|2|27|8|2|1696|52|1307|19|61.76|85.22|65.61|372.59|1246.59|1173.44|1619.18|74.79|0.00|242.82|1246.59|1321.38|1489.41|1564.20|73.15| +2450840|53661|2450926|9240|540776|3512|9848|9240|540776|3512|9848|2|46|11|5|8044|228|1307|10|83.46|121.01|29.04|919.70|290.40|834.60|1210.10|17.42|0.00|423.50|290.40|307.82|713.90|731.32|-544.20| +2450840|53661|2450886|9240|540776|3512|9848|9240|540776|3512|9848|2|69|20|3|10172|276|1307|65|94.54|243.91|17.07|14744.60|1109.55|6145.10|15854.15|22.19|0.00|6975.80|1109.55|1131.74|8085.35|8107.54|-5035.55| +2450840|53661|2450913|9240|540776|3512|9848|9240|540776|3512|9848|2|54|8|2|9002|17|1307|84|12.63|27.65|5.25|1881.60|441.00|1060.92|2322.60|8.82|0.00|951.72|441.00|449.82|1392.72|1401.54|-619.92| +2450840|63502|2450898|46048|448606|3964|41275|46048|448606|3964|41275|1|26|4|5|15742|262|1308|47|60.60|75.14|72.13|141.47|3390.11|2848.20|3531.58|0.00|0.00|529.69|3390.11|3390.11|3919.80|3919.80|541.91| +2450840|63502|2450844|46048|448606|3964|41275|46048|448606|3964|41275|1|81|15|3|8002|164|1308|64|13.50|23.49|10.33|842.24|661.12|864.00|1503.36|6.47|568.56|44.80|92.56|99.03|137.36|143.83|-771.44| +2450840|63502|2450916|46048|448606|3964|41275|46048|448606|3964|41275|1|22|13|1|7312|164|1308|45|19.49|36.25|21.75|652.50|978.75|877.05|1631.25|48.93|0.00|587.25|978.75|1027.68|1566.00|1614.93|101.70| +2450840|63502|2450916|46048|448606|3964|41275|46048|448606|3964|41275|1|74|11|5|15494|95|1308|99|51.74|88.47|66.35|2189.88|6568.65|5122.26|8758.53|197.05|0.00|3941.19|6568.65|6765.70|10509.84|10706.89|1446.39| +2450840|63502|2450918|46048|448606|3964|41275|46048|448606|3964|41275|1|9|17|2|8923|299|1308|59|88.22|157.91|66.32|5403.81|3912.88|5204.98|9316.69|12.13|2699.88|4099.32|1213.00|1225.13|5312.32|5324.45|-3991.98| +2450840|63502|2450917|46048|448606|3964|41275|46048|448606|3964|41275|1|4|14|2|7652|81|1308|2|60.99|150.64|40.67|219.94|81.34|121.98|301.28|3.25|0.00|33.14|81.34|84.59|114.48|117.73|-40.64| +2450840|69335|2450867|73832|185159|933|43460|86393|403361|2688|8415|2|8|2|1|5323|259|1309|1|33.50|93.46|66.35|27.11|66.35|33.50|93.46|3.98|0.00|42.05|66.35|70.33|108.40|112.38|32.85| +2450840|69335|2450887|73832|185159|933|43460|86393|403361|2688|8415|2|49|12|4|8635|256|1309|76|40.87|87.46|37.60|3789.36|2857.60|3106.12|6646.96|114.30|0.00|1129.36|2857.60|2971.90|3986.96|4101.26|-248.52| +2450840|69335|2450846|73832|185159|933|43460|86393|403361|2688|8415|2|88|8|5|5143|279|1309|85|41.41|75.36|55.01|1729.75|4675.85|3519.85|6405.60|140.27|0.00|1857.25|4675.85|4816.12|6533.10|6673.37|1156.00| +2450840|69335|2450878|73832|185159|933|43460|86393|403361|2688|8415|2|76|2|4|17008|215|1309|20|64.07|174.91|17.49|3148.40|349.80|1281.40|3498.20|17.49|0.00|1189.20|349.80|367.29|1539.00|1556.49|-931.60| +2450840|69335|2450883|73832|185159|933|43460|86393|403361|2688|8415|2|101|13|2|152|201|1309|6|19.73|51.88|5.70|277.08|34.20|118.38|311.28|2.39|0.00|93.36|34.20|36.59|127.56|129.95|-84.18| +2450840|69335|2450882|73832|185159|933|43460|86393|403361|2688|8415|2|92|13|3|10526|189|1309|97|59.19|114.23|17.13|9418.70|1661.61|5741.43|11080.31|99.69|0.00|3102.06|1661.61|1761.30|4763.67|4863.36|-4079.82| +2450840|69335|2450862|73832|185159|933|43460|86393|403361|2688|8415|2|21|12|1|997|150|1309|86|51.45|88.49|7.07|7002.12|608.02|4424.70|7610.14|6.08|0.00|836.78|608.02|614.10|1444.80|1450.88|-3816.68| +2450840|69335|2450850|73832|185159|933|43460|86393|403361|2688|8415|2|77|18|4|15790|193|1309|15|41.89|111.42|80.22|468.00|1203.30|628.35|1671.30|12.03|0.00|350.85|1203.30|1215.33|1554.15|1566.18|574.95| +2450840|69335|2450916|73832|185159|933|43460|86393|403361|2688|8415|2|65|19|2|6925|101|1309|5|53.37|154.23|10.79|717.20|53.95|266.85|771.15|0.37|16.72|331.55|37.23|37.60|368.78|369.15|-229.62| +2450840|69335|2450866|73832|185159|933|43460|86393|403361|2688|8415|2|44|1|3|7801|90|1309|92|81.08|161.34|25.81|12468.76|2374.52|7459.36|14843.28|0.00|0.00|5639.60|2374.52|2374.52|8014.12|8014.12|-5084.84| +2450840|69335|2450890|73832|185159|933|43460|86393|403361|2688|8415|2|49|13|4|5954|204|1309|23|45.72|98.75|9.87|2044.24|227.01|1051.56|2271.25|9.08|0.00|704.03|227.01|236.09|931.04|940.12|-824.55| +2450840|69335|2450895|73832|185159|933|43460|86393|403361|2688|8415|2|57|6|4|3790|84|1309|51|24.80|67.45|58.00|481.95|2958.00|1264.80|3439.95|207.06|0.00|1444.32|2958.00|3165.06|4402.32|4609.38|1693.20| +2450840|69335|2450877|73832|185159|933|43460|86393|403361|2688|8415|2|58|9|1|11710|123|1309|47|31.09|49.43|41.02|395.27|1927.94|1461.23|2323.21|173.51|0.00|905.69|1927.94|2101.45|2833.63|3007.14|466.71| +2450840|25789|2450909|30864|891534|6251|32521|30864|891534|6251|32521|2|22|4|4|8668|300|1310|53|39.72|80.23|60.97|1020.78|3231.41|2105.16|4252.19|129.25|0.00|2125.83|3231.41|3360.66|5357.24|5486.49|1126.25| +2450840|25789|2450860|30864|891534|6251|32521|30864|891534|6251|32521|2|70|4|2|10760|263|1310|58|99.55|131.40|77.52|3125.04|4496.16|5773.90|7621.20|26.97|3147.31|1905.30|1348.85|1375.82|3254.15|3281.12|-4425.05| +2450840|25789|2450922|30864|891534|6251|32521|30864|891534|6251|32521|2|93|2|3|8341|15|1310|66|64.47|173.42|168.21|343.86|11101.86|4255.02|11445.72|333.05|0.00|1258.62|11101.86|11434.91|12360.48|12693.53|6846.84| +2450840|25789|2450925|30864|891534|6251|32521|30864|891534|6251|32521|2|85|10|2|3302|48|1310|71|83.29|105.77|19.03|6158.54|1351.13|5913.59|7509.67|32.42|702.58|976.25|648.55|680.97|1624.80|1657.22|-5265.04| +2450840|25789|2450891|30864|891534|6251|32521|30864|891534|6251|32521|2|35|2|4|12229|186|1310|15|57.12|105.10|5.25|1497.75|78.75|856.80|1576.50|3.93|0.00|189.15|78.75|82.68|267.90|271.83|-778.05| +2450840|25789|2450891|30864|891534|6251|32521|30864|891534|6251|32521|2|31|16|2|9202|279|1310|79|28.62|36.06|5.76|2393.70|455.04|2260.98|2848.74|0.00|0.00|996.98|455.04|455.04|1452.02|1452.02|-1805.94| +2450840|25789|2450895|30864|891534|6251|32521|30864|891534|6251|32521|2|93|18|2|8810|75|1310|40|87.72|145.61|62.61|3320.00|2504.40|3508.80|5824.40|50.08|0.00|640.40|2504.40|2554.48|3144.80|3194.88|-1004.40| +2450840|47498|2450869|47852|732228|5404|23380|47852|732228|5404|23380|2|65|12|3|2804|115|1311|43|6.63|17.03|11.92|219.73|512.56|285.09|732.29|10.25|0.00|175.44|512.56|522.81|688.00|698.25|227.47| +2450840|47498|2450858|47852|732228|5404|23380|47852|732228|5404|23380|2|8|10|2|2348|281|1311|34|15.80|29.70|2.97|908.82|100.98|537.20|1009.80|4.03|0.00|323.00|100.98|105.01|423.98|428.01|-436.22| +2450840|47498|2450902|47852|732228|5404|23380|47852|732228|5404|23380|2|59|17|1|6056|158|1311|21|63.16|123.16|71.43|1086.33|1500.03|1326.36|2586.36|60.00|0.00|1189.65|1500.03|1560.03|2689.68|2749.68|173.67| +2450840|47498|2450925|47852|732228|5404|23380|47852|732228|5404|23380|2|12|1|4|5024|250|1311|53|21.38|46.39|22.73|1253.98|1204.69|1133.14|2458.67|60.23|0.00|48.76|1204.69|1264.92|1253.45|1313.68|71.55| +2450840|47498|2450891|47852|732228|5404|23380|47852|732228|5404|23380|2|43|12|1|11671|50|1311|88|2.61|3.57|2.74|73.04|241.12|229.68|314.16|2.96|142.26|134.64|98.86|101.82|233.50|236.46|-130.82| +2450840|47498|2450863|47852|732228|5404|23380|47852|732228|5404|23380|2|60|8|5|11788|166|1311|69|32.91|56.93|19.92|2553.69|1374.48|2270.79|3928.17|13.74|0.00|1492.47|1374.48|1388.22|2866.95|2880.69|-896.31| +2450840|47498|2450863|47852|732228|5404|23380|47852|732228|5404|23380|2|24|20|5|16832|269|1311|93|45.14|82.60|38.82|4071.54|3610.26|4198.02|7681.80|144.41|0.00|1381.98|3610.26|3754.67|4992.24|5136.65|-587.76| +2450840|81723|2450858|34367|958879|3838|13150|34367|958879|3838|13150|1|9|1|1|14128|244|1312|52|82.41|93.12|43.76|2566.72|2275.52|4285.32|4842.24|204.79|0.00|241.80|2275.52|2480.31|2517.32|2722.11|-2009.80| +2450840|81723|2450901|34367|958879|3838|13150|34367|958879|3838|13150|1|81|12|3|9925|217|1312|45|69.09|163.74|26.19|6189.75|1178.55|3109.05|7368.30|3.65|813.19|2578.50|365.36|369.01|2943.86|2947.51|-2743.69| +2450840|81723|2450923|34367|958879|3838|13150|34367|958879|3838|13150|1|77|18|4|691|216|1312|17|47.17|82.07|13.95|1158.04|237.15|801.89|1395.19|11.85|0.00|97.58|237.15|249.00|334.73|346.58|-564.74| +2450840|81723|2450889|34367|958879|3838|13150|34367|958879|3838|13150|1|104|8|1|17786|220|1312|43|31.69|52.92|27.51|1092.63|1182.93|1362.67|2275.56|23.65|0.00|295.41|1182.93|1206.58|1478.34|1501.99|-179.74| +2450840|81723|2450842|34367|958879|3838|13150|34367|958879|3838|13150|1|88|3|1|16094|42|1312|13|25.09|70.50|2.11|889.07|27.43|326.17|916.50|0.00|13.16|137.41|14.27|14.27|151.68|151.68|-311.90| +2450840|81723|2450883|34367|958879|3838|13150|34367|958879|3838|13150|1|64|1|5|3706|53|1312|14|95.78|173.36|109.21|898.10|1528.94|1340.92|2427.04|91.73|0.00|509.60|1528.94|1620.67|2038.54|2130.27|188.02| +2450840|81723|2450842|34367|958879|3838|13150|34367|958879|3838|13150|1|92|8|1|17149|210|1312|62|8.84|15.82|0.79|931.86|48.98|548.08|980.84|3.42|0.00|205.84|48.98|52.40|254.82|258.24|-499.10| +2450840|61523|2450847|1340|567001|7062|45076|1340|567001|7062|45076|1|3|9|4|146|236|1313|78|37.93|40.58|19.07|1677.78|1487.46|2958.54|3165.24|74.37|0.00|854.10|1487.46|1561.83|2341.56|2415.93|-1471.08| +|61523|2450920||567001||45076|1340|||45076||45|||9145|259|1313||67.76||19.63|5725.44||4336.64||75.37|0.00|209.28||1331.69|1465.60|1540.97|-3080.32| +2450840|61523|2450869|1340|567001|7062|45076|1340|567001|7062|45076|1|93|6|2|12112|11|1313|25|67.20|69.21|4.15|1626.50|103.75|1680.00|1730.25|1.03|0.00|709.25|103.75|104.78|813.00|814.03|-1576.25| +2450840|61523|2450849|1340|567001|7062|45076|1340|567001|7062|45076|1|2|5|1|1408|83|1313|58|40.23|71.20|3.56|3923.12|206.48|2333.34|4129.60|12.38|0.00|1527.72|206.48|218.86|1734.20|1746.58|-2126.86| +2450840|61523|2450930|1340|567001|7062|45076|1340|567001|7062|45076|1|45|18|4|15650|288|1313|58|34.76|77.86|46.71|1806.70|2709.18|2016.08|4515.88|108.36|0.00|1444.78|2709.18|2817.54|4153.96|4262.32|693.10| +2450840|61523|2450862|1340|567001|7062|45076|1340|567001|7062|45076|1|19|12|4|4501|98|1313|98|98.39|206.61|68.18|13566.14|6681.64|9642.22|20247.78|0.00|0.00|2226.56|6681.64|6681.64|8908.20|8908.20|-2960.58| +2450840|61523|||567001|7062|45076|1340|567001|||1|||2|4327||1313|32|44.25|128.32||3490.56|615.68||4106.24|24.62|0.00|||640.30|615.68|640.30|-800.32| +2450840|61523|2450900|1340|567001|7062|45076|1340|567001|7062|45076|1|59|14|5|13868|87|1313|25|65.33|177.04|171.72|133.00|4293.00|1633.25|4426.00|214.65|0.00|973.50|4293.00|4507.65|5266.50|5481.15|2659.75| +2450840|61523|2450882|1340|567001|7062|45076|1340|567001|7062|45076|1|74|18|3|17648|117|1313|91|77.46|197.52|193.56|360.36|17613.96|7048.86|17974.32|176.13|0.00|7369.18|17613.96|17790.09|24983.14|25159.27|10565.10| +2450840|61523|2450866|1340|567001|7062|45076|1340|567001|7062|45076|1|20|17|1|7993|143|1313|49|10.29|26.44|9.25|842.31|453.25|504.21|1295.56|13.59|0.00|362.60|453.25|466.84|815.85|829.44|-50.96| +2450840|61523|2450891|1340|567001|7062|45076|1340|567001|7062|45076|1|49|7|4|10417|156|1313|42|85.90|92.77|44.52|2026.50|1869.84|3607.80|3896.34|112.19|0.00|545.16|1869.84|1982.03|2415.00|2527.19|-1737.96| +2450840|61523|2450845|1340|567001|7062|45076|1340|567001|7062|45076|1|81|6|1|4297|253|1313|4|26.63|60.98|25.00|143.92|100.00|106.52|243.92|3.00|0.00|87.80|100.00|103.00|187.80|190.80|-6.52| +2450840|61523|2450926|1340|567001|7062|45076|1340|567001|7062|45076|1|59|6|1|16166|131|1313|42|92.61|232.45|216.17|683.76|9079.14|3889.62|9762.90|272.37|0.00|4881.24|9079.14|9351.51|13960.38|14232.75|5189.52| +2450840|59348|2450889|18352|1679852|6599|37299|18352|1679852|6599|37299|1|20|2|2|3886|282|1314|52|39.86|98.05|83.34|764.92|4333.68|2072.72|5098.60|303.35|0.00|509.60|4333.68|4637.03|4843.28|5146.63|2260.96| +2450840|59348|2450885|18352|1679852|6599|37299|18352|1679852|6599|37299|1|53|4|4|2786|4|1314|45|42.99|72.22|40.44|1430.10|1819.80|1934.55|3249.90|18.19|0.00|1559.70|1819.80|1837.99|3379.50|3397.69|-114.75| +2450840|59348|2450853|18352|1679852|6599|37299|18352|1679852|6599|37299|1|14|14|4|1468|135|1314|78|10.42|16.04|1.60|1126.32|124.80|812.76|1251.12|9.98|0.00|587.34|124.80|134.78|712.14|722.12|-687.96| +2450840|59348|2450906|18352|1679852|6599|37299|18352|1679852|6599|37299|1|48|14|2|13910|243|1314|47|73.73|115.75|30.09|4026.02|1414.23|3465.31|5440.25|4.52|961.67|1740.88|452.56|457.08|2193.44|2197.96|-3012.75| +2450840|59348|2450884|18352|1679852|6599|37299|18352|1679852|6599|37299|1|96|15|3|4034|32|1314|70|45.02|83.28|41.64|2914.80|2914.80|3151.40|5829.60|233.18|0.00|2856.00|2914.80|3147.98|5770.80|6003.98|-236.60| +2450840|59348|2450920|18352|1679852|6599|37299|18352|1679852|6599|37299|1|33|2|4|14257|180|1314|90|82.88|91.99|0.91|8197.20|81.90|7459.20|8279.10|6.55|0.00|3311.10|81.90|88.45|3393.00|3399.55|-7377.30| +2450840|59348|2450869|18352|1679852|6599|37299|18352|1679852|6599|37299|1|21|1|2|11306|212|1314|55|35.34|44.17|0.00|2429.35|0.00|1943.70|2429.35|0.00|0.00|242.55|0.00|0.00|242.55|242.55|-1943.70| +2450840|59348|2450871|18352|1679852|6599|37299|18352|1679852|6599|37299|1|49|20|1|7984|146|1314|29|2.61|3.49|2.75|21.46|79.75|75.69|101.21|0.00|0.00|19.14|79.75|79.75|98.89|98.89|4.06| +2450840|59348|2450906|18352|1679852|6599|37299|18352|1679852|6599|37299|1|68|6|5|14120|75|1314|65|46.57|48.43|32.44|1039.35|2108.60|3027.05|3147.95|189.77|0.00|1290.25|2108.60|2298.37|3398.85|3588.62|-918.45| +2450840|59348|2450890|18352|1679852|6599|37299|18352|1679852|6599|37299|1|81|17|4|2816|13|1314|84|84.50|181.67|30.88|12666.36|2593.92|7098.00|15260.28|103.75|0.00|304.92|2593.92|2697.67|2898.84|3002.59|-4504.08| +2450840|59348|2450868|18352|1679852|6599|37299|18352|1679852|6599|37299|1|70|13|4|6916|74|1314|39|77.12|179.68|1.79|6937.71|69.81|3007.68|7007.52|0.69|0.00|1471.47|69.81|70.50|1541.28|1541.97|-2937.87| +2450840|59348|2450871|18352|1679852|6599|37299|18352|1679852|6599|37299|1|27|15|1|4681|173|1314|11|7.05|13.39|0.66|140.03|7.26|77.55|147.29|0.00|0.00|48.51|7.26|7.26|55.77|55.77|-70.29| +2450840|70195|2450846|27332|948648|2764|38127|76333|1686359|3675|40991|2|37|18|2|8761|112|1315|73|4.08|7.87|2.20|413.91|160.60|297.84|574.51|3.46|73.87|286.89|86.73|90.19|373.62|377.08|-211.11| +2450840|70195|2450849|27332|948648|2764|38127|76333|1686359|3675|40991|2|35|6|2|10358|31|1315|30|3.26|5.02|3.86|34.80|115.80|97.80|150.60|1.15|0.00|10.50|115.80|116.95|126.30|127.45|18.00| +2450840|70195|2450855|27332|948648|2764|38127|76333|1686359|3675|40991|2|69|17|1|16052|199|1315|1|17.08|17.59|11.25|6.34|11.25|17.08|17.59|0.78|0.00|1.93|11.25|12.03|13.18|13.96|-5.83| +2450840|70195|2450905|27332|948648|2764|38127|76333|1686359|3675|40991|2|96|5|1|3524|249|1315|34|46.61|111.39|65.72|1552.78|2234.48|1584.74|3787.26|134.06|0.00|189.04|2234.48|2368.54|2423.52|2557.58|649.74| +2450840|70195|2450854|27332|948648|2764|38127|76333|1686359|3675|40991|2|6|10|1|8857|168|1315|92|4.86|13.94|6.13|718.52|563.96|447.12|1282.48|39.47|0.00|63.48|563.96|603.43|627.44|666.91|116.84| +2450840|70195|2450928|27332|948648|2764|38127|76333|1686359|3675|40991|2|63|7|5|13513|266|1315|14|43.34|74.54|15.65|824.46|219.10|606.76|1043.56|10.95|0.00|260.82|219.10|230.05|479.92|490.87|-387.66| +2450840|70195|2450903|27332|948648|2764|38127|76333|1686359|3675|40991|2|78|6|5|16786|8|1315|22|68.95|152.37|48.75|2279.64|1072.50|1516.90|3352.14|38.61|300.30|201.08|772.20|810.81|973.28|1011.89|-744.70| +2450840|70195|2450864|27332|948648|2764|38127|76333|1686359|3675|40991|2|56|3|1|1150|119|1315|30|1.77|5.00|2.50|75.00|75.00|53.10|150.00|6.00|0.00|22.50|75.00|81.00|97.50|103.50|21.90| +2450840|70195|2450864|27332|948648|2764|38127|76333|1686359|3675|40991|2|106|16|1|11150|131|1315|13|97.47|173.49|29.49|1872.00|383.37|1267.11|2255.37|11.50|0.00|1060.02|383.37|394.87|1443.39|1454.89|-883.74| +2450840|70195|2450930|27332|948648|2764|38127|76333|1686359|3675|40991|2|4|11|1|17083|116|1315|57|70.68|138.53|1.38|7817.55|78.66|4028.76|7896.21|3.93|0.00|2447.58|78.66|82.59|2526.24|2530.17|-3950.10| +2450840|70195|2450903|27332|948648|2764|38127|76333|1686359|3675|40991|2|65|12|1|10846|111|1315|100|83.71|100.45|85.38|1507.00|8538.00|8371.00|10045.00|85.38|0.00|200.00|8538.00|8623.38|8738.00|8823.38|167.00| +2450840|69224|2450909|37666|1790910|4154|29751|37666|1790910|4154|29751|4|78|5|5|14002|108|1316|98|8.16|23.33|13.99|915.32|1371.02|799.68|2286.34|13.71|0.00|1074.08|1371.02|1384.73|2445.10|2458.81|571.34| +2450840|69224|2450915|37666|1790910|4154|29751|37666|1790910|4154|29751|4|68|15|3|7934|2|1316|60|97.95|289.93|11.59|16700.40|695.40|5877.00|17395.80|0.00|0.00|8175.60|695.40|695.40|8871.00|8871.00|-5181.60| +2450840|69224|2450913|37666|1790910|4154|29751|37666|1790910|4154|29751|4|33|3|4|10129|245|1316|100|25.08|65.70|21.02|4468.00|2102.00|2508.00|6570.00|105.10|0.00|1182.00|2102.00|2207.10|3284.00|3389.10|-406.00| +2450840|69224|2450863|37666|1790910|4154|29751|37666|1790910|4154|29751|4|59|3|1|9649|252|1316|77|88.17|241.58|234.33|558.25|18043.41|6789.09|18601.66|0.00|0.00|5765.76|18043.41|18043.41|23809.17|23809.17|11254.32| +2450840|69224|2450857|37666|1790910|4154|29751|37666|1790910|4154|29751|4|19|13|4|2564|48|1316|90|12.66|17.09|13.33|338.40|1199.70|1139.40|1538.10|47.98|0.00|660.60|1199.70|1247.68|1860.30|1908.28|60.30| +2450840|69224|2450930|37666|1790910|4154|29751|37666|1790910|4154|29751|4|14|2|1|17593|229|1316|71|18.39|47.44|6.64|2896.80|471.44|1305.69|3368.24|14.14|0.00|369.91|471.44|485.58|841.35|855.49|-834.25| +2450840|69224|2450847|37666|1790910|4154|29751|37666|1790910|4154|29751|4|75|4|4|3757|159|1316|77|22.02|22.90|8.47|1111.11|652.19|1695.54|1763.30|38.34|13.04|616.77|639.15|677.49|1255.92|1294.26|-1056.39| +|69224|||1790910|4154||37666||4154|||94|11||2030|50|1316||21.21||||1208.05||1373.07|24.16|0.00|150.96|||1359.01|1383.17|423.28| +2450840|74458|2450904|40690|1551176|862|12548|40690|1551176|862|12548|4|28|15|5|17431|220|1317|34|98.22|200.36|160.28|1362.72|5449.52|3339.48|6812.24|0.00|0.00|2792.76|5449.52|5449.52|8242.28|8242.28|2110.04| +2450840|74458|2450899|40690|1551176|862|12548|40690|1551176|862|12548|4|49|7|3|15104|191|1317|4|74.14|186.09|137.70|193.56|550.80|296.56|744.36|22.03|0.00|230.72|550.80|572.83|781.52|803.55|254.24| +2450840|74458|2450874|40690|1551176|862|12548|40690|1551176|862|12548|4|84|17|1|11779|151|1317|61|39.64|97.51|30.22|4104.69|1843.42|2418.04|5948.11|129.03|0.00|416.02|1843.42|1972.45|2259.44|2388.47|-574.62| +2450840|74458|2450895|40690|1551176|862|12548|40690|1551176|862|12548|4|3|7|4|17446|298|1317|9|76.06|175.69|84.33|822.24|758.97|684.54|1581.21|68.30|0.00|126.45|758.97|827.27|885.42|953.72|74.43| +2450840|74458|2450861|40690|1551176|862|12548|40690|1551176|862|12548|4|67|10|4|9130|265|1317|17|52.33|74.83|40.40|585.31|686.80|889.61|1272.11|61.81|0.00|330.65|686.80|748.61|1017.45|1079.26|-202.81| +2450840|74458|2450848|40690|1551176|862|12548|40690|1551176|862|12548|4|82|3|5|98|64|1317|86|42.16|80.94|22.66|5012.08|1948.76|3625.76|6960.84|77.95|0.00|208.12|1948.76|2026.71|2156.88|2234.83|-1677.00| +2450840|74458|2450919|40690|1551176|862|12548|40690|1551176|862|12548|4|23|5|2|7756|86|1317|53|21.39|35.07|33.31|93.28|1765.43|1133.67|1858.71|105.92|0.00|631.76|1765.43|1871.35|2397.19|2503.11|631.76| +2450840|74458|2450844|40690|1551176|862|12548|40690|1551176|862|12548|4|85|17|3|5216|238|1317|53|27.18|62.24|26.14|1913.30|1385.42|1440.54|3298.72|83.12|0.00|1517.39|1385.42|1468.54|2902.81|2985.93|-55.12| +2450840|74458|2450913|40690|1551176|862|12548|40690|1551176|862|12548|4|57|19|1|6212|163|1317|22|38.24|58.88|57.11|38.94|1256.42|841.28|1295.36|87.94|0.00|168.30|1256.42|1344.36|1424.72|1512.66|415.14| +2450840|74458|2450925|40690|1551176|862|12548|40690|1551176|862|12548|4|91|3|1|7396|284|1317|38|87.13|176.87|28.29|5646.04|1075.02|3310.94|6721.06|6.23|763.26|3158.56|311.76|317.99|3470.32|3476.55|-2999.18| +2450840|74458|2450920|40690|1551176|862|12548|40690|1551176|862|12548|4|73|15|2|344|18|1317|90|35.04|65.52|38.65|2418.30|3478.50|3153.60|5896.80|102.26|1774.03|765.90|1704.47|1806.73|2470.37|2572.63|-1449.13| +2450840|74458|2450899|40690|1551176|862|12548|40690|1551176|862|12548|4|80|14|1|8947|67|1317|46|94.04|161.74|113.21|2232.38|5207.66|4325.84|7440.04|416.61|0.00|2529.54|5207.66|5624.27|7737.20|8153.81|881.82| +2450840|22445|2450910|14702|779215|6128|12641|36668|1643667|5780|9893|2|26|9|3|1174|291|1318|2|94.68|254.68|241.94|25.48|483.88|189.36|509.36|0.00|0.00|254.68|483.88|483.88|738.56|738.56|294.52| +2450840|22445|2450908|14702|779215|6128|12641|36668|1643667|5780|9893|2|58|8|1|128|232|1318|17|56.00|140.00|35.00|1785.00|595.00|952.00|2380.00|0.00|0.00|690.20|595.00|595.00|1285.20|1285.20|-357.00| +2450840|22445|2450869|14702|779215|6128|12641|36668|1643667|5780|9893|2|54|19|4|4106|36|1318|22|49.76|64.19|55.20|197.78|1214.40|1094.72|1412.18|47.36|425.04|494.12|789.36|836.72|1283.48|1330.84|-305.36| +2450840|22445|2450895|14702|779215|6128|12641|36668|1643667|5780|9893|2|40|1|1|2497|68|1318|62|2.65|6.30|3.78|156.24|234.36|164.30|390.60|1.68|213.26|78.12|21.10|22.78|99.22|100.90|-143.20| +2450840|22445|2450899|14702|779215|6128|12641|36668|1643667|5780|9893|2|97|14|3|4663|216|1318|85|17.60|41.71|20.85|1773.10|1772.25|1496.00|3545.35|0.00|0.00|1063.35|1772.25|1772.25|2835.60|2835.60|276.25| +2450840|22445|2450855|14702|779215|6128|12641|36668|1643667|5780|9893|2|71|4|5|17066|265|1318|97|12.91|23.75|1.42|2166.01|137.74|1252.27|2303.75|11.01|0.00|1151.39|137.74|148.75|1289.13|1300.14|-1114.53| +2450840|22445|2450847|14702|779215|6128|12641|36668|1643667|5780|9893|2|2|1|2|16328|4|1318|60|28.65|36.67|30.80|352.20|1848.00|1719.00|2200.20|0.00|425.04|1099.80|1422.96|1422.96|2522.76|2522.76|-296.04| +2450840|22445|2450876|14702|779215|6128|12641|36668|1643667|5780|9893|2|92|15|4|6199|114|1318|45|98.89|281.83|28.18|11414.25|1268.10|4450.05|12682.35|63.40|0.00|6087.15|1268.10|1331.50|7355.25|7418.65|-3181.95| +2450840|22445|2450904|14702|779215|6128|12641|36668|1643667|5780|9893|2|83|17|1|6097|224|1318|70|66.81|148.98|26.81|8551.90|1876.70|4676.70|10428.60|131.36|0.00|2293.90|1876.70|2008.06|4170.60|4301.96|-2800.00| +2450840|22445|2450917|14702|779215|6128|12641|36668|1643667|5780|9893|2|36|17|4|2122|90|1318|60|5.63|16.77|0.50|976.20|30.00|337.80|1006.20|0.62|23.10|171.00|6.90|7.52|177.90|178.52|-330.90| +2450840|22445|2450875|14702|779215|6128|12641|36668|1643667|5780|9893|2|24|4|3|8254|210|1318|44|18.55|52.86|19.55|1465.64|860.20|816.20|2325.84|3.61|498.91|534.60|361.29|364.90|895.89|899.50|-454.91| +2450840|64274|2450844|19222|82919|6263|4407|19222|82919|6263|4407|4|81|7|1|8594|242|1319|55|33.69|75.80|25.77|2751.65|1417.35|1852.95|4169.00|37.41|481.89|1500.40|935.46|972.87|2435.86|2473.27|-917.49| +2450840|64274|2450897|19222|82919|6263|4407|19222|82919|6263|4407|4|74|18|4|2066|214|1319|74|63.12|143.91|77.71|4898.80|5750.54|4670.88|10649.34|345.03|0.00|1490.36|5750.54|6095.57|7240.90|7585.93|1079.66| +2450840|64274|2450869|19222|82919|6263|4407|19222|82919|6263|4407|4|63|17|4|8278|156|1319|20|24.01|34.33|32.27|41.20|645.40|480.20|686.60|32.27|0.00|151.00|645.40|677.67|796.40|828.67|165.20| +2450840|64274|2450861|19222|82919|6263|4407|19222|82919|6263|4407|4|95|3|1|11590|238|1319|74|59.75|93.80|80.66|972.36|5968.84|4421.50|6941.20|205.32|3402.23|1388.24|2566.61|2771.93|3954.85|4160.17|-1854.89| +2450840|64274|2450913|19222|82919|6263|4407|19222|82919|6263|4407|4|38|20|5|7609|250|1319|73|9.41|25.68|24.65|75.19|1799.45|686.93|1874.64|0.00|1799.45|805.92|0.00|0.00|805.92|805.92|-686.93| +2450840|64274|2450913|19222|82919|6263|4407|19222|82919|6263|4407|4|107|8|2|13964|276|1319|53|57.53|105.27|26.31|4184.88|1394.43|3049.09|5579.31|55.77|0.00|669.39|1394.43|1450.20|2063.82|2119.59|-1654.66| +2450840|64274|2450865|19222|82919|6263|4407|19222|82919|6263|4407|4|26|10|1|8257|215|1319|79|59.73|136.18|87.15|3873.37|6884.85|4718.67|10758.22|344.24|0.00|4087.46|6884.85|7229.09|10972.31|11316.55|2166.18| +2450840|33486|2450913|77344|811691|4885|32832|77344|811691|4885|32832|4|102|17|1|6403|92|1320|48|77.34|88.94|74.70|683.52|3585.60|3712.32|4269.12|12.55|3334.60|1109.76|251.00|263.55|1360.76|1373.31|-3461.32| +2450840|33486|2450860|77344|811691|4885|32832|77344|811691|4885|32832|4|76|8|3|9112|71|1320|9|57.28|153.51|84.43|621.72|759.87|515.52|1381.59|53.19|0.00|690.75|759.87|813.06|1450.62|1503.81|244.35| +2450840|33486|2450925|77344|811691|4885|32832|77344|811691|4885|32832|4|106|16|5|1352|300|1320|10|45.30|90.60|5.43|851.70|54.30|453.00|906.00|1.08|0.00|190.20|54.30|55.38|244.50|245.58|-398.70| +2450840|33486|2450899|77344|811691|4885|32832|77344|811691|4885|32832|4|63|20|3|7273|202|1320|34|58.88|70.65|29.67|1393.32|1008.78|2001.92|2402.10|30.26|0.00|1104.66|1008.78|1039.04|2113.44|2143.70|-993.14| +2450840|33486|2450912|77344|811691|4885|32832|77344|811691|4885|32832|4|22|1|5|16417|135|1320|55|95.32|203.98|157.06|2580.60|8638.30|5242.60|11218.90|691.06|0.00|1233.65|8638.30|9329.36|9871.95|10563.01|3395.70| +2450840|33486|2450867|77344|811691|4885|32832|77344|811691|4885|32832|4|48|16|2|1009|256|1320|67|66.49|67.81|12.88|3680.31|862.96|4454.83|4543.27|43.14|0.00|681.39|862.96|906.10|1544.35|1587.49|-3591.87| +2450840|33486|2450906|77344|811691|4885|32832|77344|811691|4885|32832|4|51|10|1|15403|298|1320|78|38.95|98.15|52.01|3598.92|4056.78|3038.10|7655.70|243.40|0.00|3368.04|4056.78|4300.18|7424.82|7668.22|1018.68| +2450840|33486|2450844|77344|811691|4885|32832|77344|811691|4885|32832|4|77|9|4|1366|109|1320|62|61.30|172.25|111.96|3737.98|6941.52|3800.60|10679.50|208.24|0.00|2135.90|6941.52|7149.76|9077.42|9285.66|3140.92| +2450840|33486|2450903|77344|811691|4885|32832|77344|811691|4885|32832|4|6|5|4|568|145|1320|13|57.20|84.08|4.20|1038.44|54.60|743.60|1093.04|1.09|0.00|153.01|54.60|55.69|207.61|208.70|-689.00| +2450840|33486|2450872|77344|811691|4885|32832|77344|811691|4885|32832|4|80|1|2|15062|176|1320|42|33.18|85.93|61.86|1010.94|2598.12|1393.56|3609.06|128.60|1169.15|324.66|1428.97|1557.57|1753.63|1882.23|35.41| +2450840|33486|2450844|77344|811691|4885|32832|77344|811691|4885|32832|4|100|14|4|11371|66|1320|98|7.92|17.97|8.08|969.22|791.84|776.16|1761.06|63.34|0.00|756.56|791.84|855.18|1548.40|1611.74|15.68| +2450840|33486|2450923|77344|811691|4885|32832|77344|811691|4885|32832|4|25|12|5|3379|39|1320|71|73.56|81.65|1.63|5681.42|115.73|5222.76|5797.15|6.94|0.00|695.09|115.73|122.67|810.82|817.76|-5107.03| +2450840|67191|2450844|61624|1505919|1414|909|61624|1505919|1414|909|2|41|18|3|8612|196|1321|23|40.60|118.95|109.43|218.96|2516.89|933.80|2735.85|0.00|0.00|109.25|2516.89|2516.89|2626.14|2626.14|1583.09| +2450840|67191|2450927|61624|1505919|1414|909|61624|1505919|1414|909|2|66|9|5|17887|295|1321|48|74.49|94.60|74.73|953.76|3587.04|3575.52|4540.80|179.35|0.00|1089.60|3587.04|3766.39|4676.64|4855.99|11.52| +2450840|67191|2450887|61624|1505919|1414|909|61624|1505919|1414|909|2|26|12|1|5197|154|1321|28|58.35|167.46|135.64|890.96|3797.92|1633.80|4688.88|341.81|0.00|2297.40|3797.92|4139.73|6095.32|6437.13|2164.12| +2450840|67191|2450884|61624|1505919|1414|909|61624|1505919|1414|909|2|64|12|2|10501|45|1321|93|5.14|9.35|0.65|809.10|60.45|478.02|869.55|0.60|0.00|190.65|60.45|61.05|251.10|251.70|-417.57| +2450840|67191|2450856|61624|1505919|1414|909|61624|1505919|1414|909|2|45|15|1|12145|208|1321|52|55.57|137.81|37.20|5231.72|1934.40|2889.64|7166.12|38.68|0.00|1003.08|1934.40|1973.08|2937.48|2976.16|-955.24| +2450840|67191|2450849|61624|1505919|1414|909|61624|1505919|1414|909|2|101|13|2|16720|80|1321|38|99.63|245.08|0.00|9313.04|0.00|3785.94|9313.04|0.00|0.00|4283.74|0.00|0.00|4283.74|4283.74|-3785.94| +2450840|67191|2450871|61624|1505919|1414|909|61624|1505919|1414|909|2|42|17|4|4138|275|1321|95|90.31|141.78|34.02|10237.20|3231.90|8579.45|13469.10|6.46|2908.71|672.60|323.19|329.65|995.79|1002.25|-8256.26| +2450840|67191|2450887|61624|1505919|1414|909|61624|1505919|1414|909|2|23|9|2|13808|196|1321|47|6.23|17.25|1.89|721.92|88.83|292.81|810.75|4.44|0.00|250.98|88.83|93.27|339.81|344.25|-203.98| +2450840|67191|2450883|61624|1505919|1414|909|61624|1505919|1414|909|2|14|13|4|230|213|1321|84|53.84|75.37|28.64|3925.32|2405.76|4522.56|6331.08|216.51|0.00|1582.56|2405.76|2622.27|3988.32|4204.83|-2116.80| +2450840|25180|2450881|61788|1122599|210|17821|61788|1122599|210|17821|1|4|11|5|7724|253|1322|51|90.93|177.31|147.16|1537.65|7505.16|4637.43|9042.81|225.15|0.00|2893.23|7505.16|7730.31|10398.39|10623.54|2867.73| +2450840|25180|2450923|61788|1122599|210|17821|61788|1122599|210|17821|1|94|2|4|13360|23|1322|31|20.33|50.62|41.50|282.72|1286.50|630.23|1569.22|0.00|0.00|125.24|1286.50|1286.50|1411.74|1411.74|656.27| +2450840|25180|2450870|61788|1122599|210|17821|61788|1122599|210|17821|1|65|13|4|10570|174|1322|70|11.61|15.67|8.30|515.90|581.00|812.70|1096.90|38.11|104.58|449.40|476.42|514.53|925.82|963.93|-336.28| +2450840|25180|2450877|61788|1122599|210|17821|61788|1122599|210|17821|1|99|18|3|15932|287|1322|48|68.30|163.23|27.74|6503.52|1331.52|3278.40|7835.04|79.89|0.00|1880.16|1331.52|1411.41|3211.68|3291.57|-1946.88| +2450840|25180|2450882|61788|1122599|210|17821|61788|1122599|210|17821|1|9|11|4|3799|213|1322|67|65.48|119.82|106.63|883.73|7144.21|4387.16|8027.94|115.73|5215.27|79.73|1928.94|2044.67|2008.67|2124.40|-2458.22| +2450840|25180|2450909|61788|1122599|210|17821|61788|1122599|210|17821|1|97|12|4|17272|92|1322|8|70.29|76.61|10.72|527.12|85.76|562.32|612.88|0.46|39.44|67.36|46.32|46.78|113.68|114.14|-516.00| +2450840|25180|2450922|61788|1122599|210|17821|61788|1122599|210|17821|1|98|5|3|17326|157|1322|69|33.97|82.54|18.98|4385.64|1309.62|2343.93|5695.26|26.19|0.00|1480.74|1309.62|1335.81|2790.36|2816.55|-1034.31| +2450840|25180|2450888|61788|1122599|210|17821|61788|1122599|210|17821|1|26|4|4|16219|198|1322|41|24.47|34.99|8.39|1090.60|343.99|1003.27|1434.59|25.04|30.95|329.64|313.04|338.08|642.68|667.72|-690.23| +2450840|25180|2450930|61788|1122599|210|17821|61788|1122599|210|17821|1|43|5|5|10045|132|1322|7|38.43|39.96|36.36|25.20|254.52|269.01|279.72|17.81|0.00|19.53|254.52|272.33|274.05|291.86|-14.49| +2450840|25180|2450901|61788|1122599|210|17821|61788|1122599|210|17821|1|101|5|3|7132|112|1322|23|44.82|126.39|109.95|378.12|2528.85|1030.86|2906.97|30.34|2149.52|1104.46|379.33|409.67|1483.79|1514.13|-651.53| +2450840|25180|2450847|61788|1122599|210|17821|61788|1122599|210|17821|1|55|17|1|1160|9|1322|58|75.61|143.65|25.85|6832.40|1499.30|4385.38|8331.70|29.98|0.00|2582.74|1499.30|1529.28|4082.04|4112.02|-2886.08| +2450840|25180|2450918|61788|1122599|210|17821|61788|1122599|210|17821|1|58|11|4|14821|142|1322|50|21.92|57.43|23.54|1694.50|1177.00|1096.00|2871.50|77.44|70.62|229.50|1106.38|1183.82|1335.88|1413.32|10.38| +2450840|25180|2450855|61788|1122599|210|17821|61788|1122599|210|17821|1|68|3|5|2968|159|1322|89|48.43|128.82|104.34|2178.72|9286.26|4310.27|11464.98|464.31|0.00|2750.99|9286.26|9750.57|12037.25|12501.56|4975.99| +2450840|25180|2450872|61788|1122599|210|17821|61788|1122599|210|17821|1|80|20|2|493|111|1322|94|24.84|64.33|63.04|121.26|5925.76|2334.96|6047.02|533.31|0.00|785.84|5925.76|6459.07|6711.60|7244.91|3590.80| +2450840|9783|2450890|46759|1087755|751|4947|46759|1087755|751|4947|1|54|14|4|11102|189|1323|74|91.63|130.11|101.48|2118.62|7509.52|6780.62|9628.14|450.57|0.00|481.00|7509.52|7960.09|7990.52|8441.09|728.90| +2450840|9783|2450897|46759|1087755|751|4947|46759|1087755|751|4947|1|33|2|1|8413|34|1323|46|57.13|129.68|50.57|3639.06|2326.22|2627.98|5965.28|69.78|0.00|59.34|2326.22|2396.00|2385.56|2455.34|-301.76| +2450840|9783|2450851|46759|1087755|751|4947|46759|1087755|751|4947|1|62|20|5|10606|151|1323|12|44.02|91.12|30.06|732.72|360.72|528.24|1093.44|26.29|68.53|98.40|292.19|318.48|390.59|416.88|-236.05| +2450840|9783|2450919|46759|1087755|751|4947|46759|1087755|751|4947|1|30|7|1|961|2|1323|65|46.33|118.14|66.15|3379.35|4299.75|3011.45|7679.10|257.98|0.00|2072.85|4299.75|4557.73|6372.60|6630.58|1288.30| +2450840|9783|2450903|46759|1087755|751|4947|46759|1087755|751|4947|1|38|20|3|5305|294|1323|92|75.10|189.25|24.60|15147.80|2263.20|6909.20|17411.00|90.52|0.00|173.88|2263.20|2353.72|2437.08|2527.60|-4646.00| +2450840|9783|2450877|46759|1087755|751|4947|46759|1087755|751|4947|1|108|4|3|12637|121|1323|42|4.72|7.69|3.99|155.40|167.58|198.24|322.98|13.40|0.00|48.30|167.58|180.98|215.88|229.28|-30.66| +2450840|9783|2450874|46759|1087755|751|4947|46759|1087755|751|4947|1|94|12|4|5707|293|1323|28|67.39|81.54|59.52|616.56|1666.56|1886.92|2283.12|83.32|0.00|798.84|1666.56|1749.88|2465.40|2548.72|-220.36| +2450840|9783|2450866|46759|1087755|751|4947|46759|1087755|751|4947|1|68|2|5|9595|81|1323|13|61.02|121.42|104.42|221.00|1357.46|793.26|1578.46|49.54|366.51|173.55|990.95|1040.49|1164.50|1214.04|197.69| +2450840|9783|2450910|46759|1087755|751|4947|46759|1087755|751|4947|1|18|9|4|12488|2|1323|95|1.39|3.89|2.56|126.35|243.20|132.05|369.55|12.16|0.00|18.05|243.20|255.36|261.25|273.41|111.15| +2450840|79289|2450930|53559|777276|546|36263|53559|777276|546|36263|4|19|18|4|6644|125|1324|74|96.06|126.79|39.30|6474.26|2908.20|7108.44|9382.46|87.24|0.00|4315.68|2908.20|2995.44|7223.88|7311.12|-4200.24| +2450840|79289|2450845|53559|777276|546|36263|53559|777276|546|36263|4|4|8|5|3949|72|1324|3|84.00|246.96|246.96|0.00|740.88|252.00|740.88|59.27|0.00|22.20|740.88|800.15|763.08|822.35|488.88| +2450840|79289|2450852|53559|777276|546|36263|53559|777276|546|36263|4|53|10|5|3673|168|1324|30|92.42|132.16|11.89|3608.10|356.70|2772.60|3964.80|24.96|0.00|277.50|356.70|381.66|634.20|659.16|-2415.90| +2450840|79289|2450854|53559|777276|546|36263|53559|777276|546|36263|4|62|9|4|1196|141|1324|31|11.95|32.62|12.72|616.90|394.32|370.45|1011.22|19.87|173.50|424.70|220.82|240.69|645.52|665.39|-149.63| +2450840|79289|2450877|53559|777276|546|36263|53559|777276|546|36263|4|73|14|3|5336|59|1324|63|37.97|62.65|23.18|2486.61|1460.34|2392.11|3946.95|102.22|0.00|1736.28|1460.34|1562.56|3196.62|3298.84|-931.77| +2450840|79289|2450877|53559|777276|546|36263|53559|777276|546|36263|4|66|10|4|7696|193|1324|47|55.63|77.88|28.03|2342.95|1317.41|2614.61|3660.36|104.33|13.17|475.64|1304.24|1408.57|1779.88|1884.21|-1310.37| +2450840|79289|2450856|53559|777276|546|36263|53559|777276|546|36263|4|25|14|5|3626|65|1324|56|4.72|11.09|9.75|75.04|546.00|264.32|621.04|0.00|0.00|260.40|546.00|546.00|806.40|806.40|281.68| +2450840|79289|2450889|53559|777276|546|36263|53559|777276|546|36263|4|31|12|3|4636|210|1324|100|10.58|14.17|7.36|681.00|736.00|1058.00|1417.00|14.72|0.00|226.00|736.00|750.72|962.00|976.72|-322.00| +2450840|79289|2450875|53559|777276|546|36263|53559|777276|546|36263|4|61|20|1|2551|166|1324|51|54.69|68.90|59.25|492.15|3021.75|2789.19|3513.90|211.52|0.00|281.01|3021.75|3233.27|3302.76|3514.28|232.56| +2450840|79289|2450858|53559|777276|546|36263|53559|777276|546|36263|4|88|5|3|10888|37|1324|71|28.12|64.39|1.93|4434.66|137.03|1996.52|4571.69|1.04|32.88|548.12|104.15|105.19|652.27|653.31|-1892.37| +2450840|29695|2450927|50369|1532119|1069|12772|50369|1532119|1069|12772|2|42|2|2|9631|155|1325|99|13.64|34.50|26.56|786.06|2629.44|1350.36|3415.50|78.88|0.00|1092.96|2629.44|2708.32|3722.40|3801.28|1279.08| +2450840|29695|2450891|50369|1532119|1069|12772|50369|1532119|1069|12772|2|6|6|1|6790|61|1325|18|52.56|157.15|72.28|1527.66|1301.04|946.08|2828.70|39.03|0.00|1103.04|1301.04|1340.07|2404.08|2443.11|354.96| +2450840|29695|2450927|50369|1532119|1069|12772|50369|1532119|1069|12772|2|20|11|2|9436|291|1325|60|94.43|283.29|266.29|1020.00|15977.40|5665.80|16997.40|798.87|0.00|5608.80|15977.40|16776.27|21586.20|22385.07|10311.60| +2450840|29695|2450910|50369|1532119|1069|12772|50369|1532119|1069|12772|2|63|7|3|1706|51|1325|72|16.58|42.27|28.74|974.16|2069.28|1193.76|3043.44|186.23|0.00|30.24|2069.28|2255.51|2099.52|2285.75|875.52| +2450840|29695|2450866|50369|1532119|1069|12772|50369|1532119|1069|12772|2|78|10|3|17468|215|1325|53|8.41|14.96|3.88|587.24|205.64|445.73|792.88|0.00|0.00|316.94|205.64|205.64|522.58|522.58|-240.09| +2450840|29695|2450930|50369|1532119|1069|12772|50369|1532119|1069|12772|2|104|14|5|11858|45|1325|38|79.86|103.81|96.54|276.26|3668.52|3034.68|3944.78|293.48|0.00|354.92|3668.52|3962.00|4023.44|4316.92|633.84| +2450840|29695|2450896|50369|1532119|1069|12772|50369|1532119|1069|12772|2|39|15|4|17990|56|1325|80|25.30|48.32|46.38|155.20|3710.40|2024.00|3865.60|74.20|0.00|1197.60|3710.40|3784.60|4908.00|4982.20|1686.40| +2450840|29695|2450879|50369|1532119|1069|12772|50369|1532119|1069|12772|2|40|16|1|3028|28|1325|38|35.82|44.41|42.18|84.74|1602.84|1361.16|1687.58|0.00|0.00|151.62|1602.84|1602.84|1754.46|1754.46|241.68| +2450840|29695|2450912|50369|1532119|1069|12772|50369|1532119|1069|12772|2|76|12|1|8648|199|1325|44|67.36|120.57|113.33|318.56|4986.52|2963.84|5305.08|0.00|0.00|1750.32|4986.52|4986.52|6736.84|6736.84|2022.68| +2450840|29695|2450910|50369|1532119|1069|12772|50369|1532119|1069|12772|2|48|18|5|13124|234|1325|27|32.48|42.22|29.55|342.09|797.85|876.96|1139.94|7.97|0.00|501.39|797.85|805.82|1299.24|1307.21|-79.11| +2450840|9975|2450848|57915|451051|187|30523|57915|451051|187|30523|4|16|8|1|3976|260|1326|96|27.90|38.22|7.26|2972.16|696.96|2678.40|3669.12|8.36|529.68|256.32|167.28|175.64|423.60|431.96|-2511.12| +2450840|9975|2450866|57915|451051|187|30523|57915|451051|187|30523|4|72|13|2|13144|29|1326|86|79.34|88.86|72.86|1376.00|6265.96|6823.24|7641.96|501.27|0.00|1680.44|6265.96|6767.23|7946.40|8447.67|-557.28| +2450840|9975|2450905|57915|451051|187|30523|57915|451051|187|30523|4|60|19|3|7508|114|1326|95|19.31|51.94|21.29|2911.75|2022.55|1834.45|4934.30|121.35|0.00|1381.30|2022.55|2143.90|3403.85|3525.20|188.10| +2450840|9975|2450899|57915|451051|187|30523|57915|451051|187|30523|4|99|12|1|5692|2|1326|59|55.83|140.69|5.62|7969.13|331.58|3293.97|8300.71|0.00|0.00|3154.14|331.58|331.58|3485.72|3485.72|-2962.39| +2450840|9975|2450843|57915|451051|187|30523|57915|451051|187|30523|4|29|1|5|10198|175|1326|60|87.40|139.84|81.10|3524.40|4866.00|5244.00|8390.40|291.96|0.00|2852.40|4866.00|5157.96|7718.40|8010.36|-378.00| +2450840|9975|2450899|57915|451051|187|30523|57915|451051|187|30523|4|81|3|3|1564|185|1326|33|31.30|40.06|34.45|185.13|1136.85|1032.90|1321.98|40.92|625.26|317.13|511.59|552.51|828.72|869.64|-521.31| +2450840|9975|2450918|57915|451051|187|30523|57915|451051|187|30523|4|39|9|3|2989|279|1326|9|20.45|30.06|25.25|43.29|227.25|184.05|270.54|0.00|0.00|105.48|227.25|227.25|332.73|332.73|43.20| +2450840|9975|2450919|57915|451051|187|30523|57915|451051|187|30523|4|48|11|3|12718|50|1326|30|57.61|74.31|34.18|1203.90|1025.40|1728.30|2229.30|30.76|0.00|312.00|1025.40|1056.16|1337.40|1368.16|-702.90| +2450840|19066|2450874|88040|902463|2039|12062|88040|902463|2039|12062|2|71|17|4|6733|223|1327|1|33.30|39.62|8.71|30.91|8.71|33.30|39.62|0.25|3.65|11.88|5.06|5.31|16.94|17.19|-28.24| +2450840|19066|2450858|88040|902463|2039|12062|88040|902463|2039|12062|2|21|15|1|7538|118|1327|98|10.71|16.60|5.81|1057.42|569.38|1049.58|1626.80|12.24|324.54|244.02|244.84|257.08|488.86|501.10|-804.74| +2450840|19066|2450868|88040|902463|2039|12062|88040|902463|2039|12062|2|6|4|1|2149|103|1327|63|91.03|124.71|23.69|6364.26|1492.47|5734.89|7856.73|119.39|0.00|2906.82|1492.47|1611.86|4399.29|4518.68|-4242.42| +2450840|19066|2450892|88040|902463|2039|12062|88040|902463|2039|12062|2|83|4|3|13342|189|1327|15|31.28|49.42|30.14|289.20|452.10|469.20|741.30|13.56|0.00|118.50|452.10|465.66|570.60|584.16|-17.10| +2450840|19066|2450873|88040|902463|2039|12062|88040|902463|2039|12062|2|17|3|2|17869|265|1327|66|21.96|52.92|48.15|314.82|3177.90|1449.36|3492.72|31.77|0.00|1396.56|3177.90|3209.67|4574.46|4606.23|1728.54| +2450840|19066|2450881|88040|902463|2039|12062|88040|902463|2039|12062|2|43|12|3|2041|90|1327|7|95.15|195.05|39.01|1092.28|273.07|666.05|1365.35|19.11|0.00|204.75|273.07|292.18|477.82|496.93|-392.98| +2450840|19066|2450877|88040|902463|2039|12062|88040|902463|2039|12062|2|37|12|5|79|77|1327|96|73.14|116.29|47.67|6587.52|4576.32|7021.44|11163.84|137.28|0.00|2678.40|4576.32|4713.60|7254.72|7392.00|-2445.12| +2450841|69722|2450847|16838|1711797|1956|40115|76263|1724080|2239|4798|1|43|3|2|2788|58|1328|40|66.44|184.70|108.97|3029.20|4358.80|2657.60|7388.00|167.37|2266.57|590.80|2092.23|2259.60|2683.03|2850.40|-565.37| +2450841|69722|2450851|16838|1711797|1956|40115|76263|1724080|2239|4798|1|22|5|1|4328|1|1328|72|2.73|3.19|2.80|28.08|201.60|196.56|229.68|6.53|38.30|6.48|163.30|169.83|169.78|176.31|-33.26| +2450841|69722|2450877|16838|1711797|1956|40115|76263|1724080|2239|4798|1|29|5|2|12704|47|1328|9|13.40|30.95|14.23|150.48|128.07|120.60|278.55|2.56|0.00|83.52|128.07|130.63|211.59|214.15|7.47| +2450841|69722|2450928|16838|1711797|1956|40115|76263|1724080|2239|4798|1|6|4|2|10580|93|1328|6|79.88|134.19|134.19|0.00|805.14|479.28|805.14|8.05|0.00|201.24|805.14|813.19|1006.38|1014.43|325.86| +2450841|69722|2450888|16838|1711797|1956|40115|76263|1724080|2239|4798|1|102|19|4|13262|136|1328|22|89.84|246.16|113.23|2924.46|2491.06|1976.48|5415.52|99.64|0.00|2328.48|2491.06|2590.70|4819.54|4919.18|514.58| +2450841|69722|2450907|16838|1711797|1956|40115|76263|1724080|2239|4798|1|62|19|4|14293|259|1328|31|80.68|121.02|58.08|1951.14|1800.48|2501.08|3751.62|162.04|0.00|300.08|1800.48|1962.52|2100.56|2262.60|-700.60| +2450841|69722|2450850|16838|1711797|1956|40115|76263|1724080|2239|4798|1|42|19|2|3277|300|1328|42|93.29|117.54|76.40|1727.88|3208.80|3918.18|4936.68|224.61|0.00|2221.38|3208.80|3433.41|5430.18|5654.79|-709.38| +2450841|69722|2450861|16838|1711797|1956|40115|76263|1724080|2239|4798|1|79|8|1|16378|62|1328|45|97.58|267.36|53.47|9625.05|2406.15|4391.10|12031.20|216.55|0.00|2646.45|2406.15|2622.70|5052.60|5269.15|-1984.95| +2450841|21492|2450911|14117|462968|6219|3423|14117|462968|6219|3423|1|73|14|4|3536|152|1329|53|83.27|158.21|88.59|3689.86|4695.27|4413.31|8385.13|129.12|2112.87|83.74|2582.40|2711.52|2666.14|2795.26|-1830.91| +2450841|21492|2450914|14117|462968|6219|3423|14117|462968|6219|3423|1|9|9|2|9356|4|1329|55|52.91|85.71|2.57|4572.70|141.35|2910.05|4714.05|4.24|0.00|1602.70|141.35|145.59|1744.05|1748.29|-2768.70| +2450841|21492|2450923|14117|462968|6219|3423|14117|462968|6219|3423|1|44|17|3|6961|160|1329|46|97.70|239.36|148.40|4184.16|6826.40|4494.20|11010.56|409.58|0.00|1321.12|6826.40|7235.98|8147.52|8557.10|2332.20| +2450841|21492|2450870|14117|462968|6219|3423|14117|462968|6219|3423|1|49|7|1|4399|284|1329|64|34.95|44.03|30.38|873.60|1944.32|2236.80|2817.92|155.54|0.00|872.96|1944.32|2099.86|2817.28|2972.82|-292.48| +2450841|21492|2450868|14117|462968|6219|3423|14117|462968|6219|3423|1|17|20|1|10006|118|1329|53|21.70|29.94|18.56|603.14|983.68|1150.10|1586.82|59.02|0.00|539.01|983.68|1042.70|1522.69|1581.71|-166.42| +2450841|21492|2450905|14117|462968|6219|3423|14117|462968|6219|3423|1|78|11|4|12313|84|1329|22|48.77|74.13|41.51|717.64|913.22|1072.94|1630.86|36.52|0.00|619.52|913.22|949.74|1532.74|1569.26|-159.72| +2450841|21492|2450914|14117|462968|6219|3423|14117|462968|6219|3423|1|44|7|2|14230|281|1329|54|28.37|51.63|42.33|502.20|2285.82|1531.98|2788.02|91.43|0.00|529.20|2285.82|2377.25|2815.02|2906.45|753.84| +2450841|21492|2450873|14117|462968|6219|3423|14117|462968|6219|3423|1|16|6|4|4597|272|1329|82|88.05|162.89|115.65|3873.68|9483.30|7220.10|13356.98|284.49|0.00|1869.60|9483.30|9767.79|11352.90|11637.39|2263.20| +2450841|21492|2450895|14117|462968|6219|3423|14117|462968|6219|3423|1|11|1|4|16645|300|1329|97|63.13|70.70|18.38|5075.04|1782.86|6123.61|6857.90|35.65|0.00|3222.34|1782.86|1818.51|5005.20|5040.85|-4340.75| +2450841|21492|2450873|14117|462968|6219|3423|14117|462968|6219|3423|1|20|20|5|16201|83|1329|60|44.07|90.34|7.22|4987.20|433.20|2644.20|5420.40|12.99|0.00|1626.00|433.20|446.19|2059.20|2072.19|-2211.00| +2450841|21492|2450885|14117|462968|6219|3423|14117|462968|6219|3423|1|96|18|3|8404|74|1329|60|95.57|147.17|16.18|7859.40|970.80|5734.20|8830.20|9.70|0.00|3531.60|970.80|980.50|4502.40|4512.10|-4763.40| +2450841|21492|2450894|14117|462968|6219|3423|14117|462968|6219|3423|1|6|3|4|1462|43|1329|66|95.05|98.85|62.27|2414.28|4109.82|6273.30|6524.10|0.00|0.00|2739.66|4109.82|4109.82|6849.48|6849.48|-2163.48| +2450841|14963|2450844|77698|1288847|3996|30745|77698|1288847|3996|30745|4|16|8|2|10465|23|1330|25|59.64|157.44|29.91|3188.25|747.75|1491.00|3936.00|37.38|0.00|1574.25|747.75|785.13|2322.00|2359.38|-743.25| +2450841|14963|2450876|77698|1288847|3996|30745|77698|1288847|3996|30745|4|96|1|1|5437|203|1330|21|10.56|17.10|9.74|154.56|204.54|221.76|359.10|0.00|0.00|143.64|204.54|204.54|348.18|348.18|-17.22| +2450841|14963|2450854|77698|1288847|3996|30745|77698|1288847|3996|30745|4|50|9|2|14455|108|1330|24|9.38|22.60|20.79|43.44|498.96|225.12|542.40|29.93|0.00|0.00|498.96|528.89|498.96|528.89|273.84| +2450841|14963|2450863|77698|1288847|3996|30745|77698|1288847|3996|30745|4|72|1|1|16588|281|1330|66|18.74|26.42|21.13|349.14|1394.58|1236.84|1743.72|0.00|0.00|243.54|1394.58|1394.58|1638.12|1638.12|157.74| +2450841|14963|2450848|77698|1288847|3996|30745|77698|1288847|3996|30745|4|45|5|5|12676|225|1330|57|49.81|85.17|64.72|1165.65|3689.04|2839.17|4854.69|110.67|0.00|2038.89|3689.04|3799.71|5727.93|5838.60|849.87| +2450841|14963|2450906|77698|1288847|3996|30745|77698|1288847|3996|30745|4|70|13|3|8773|108|1330|58|31.28|93.21|0.93|5352.24|53.94|1814.24|5406.18|4.85|0.00|2702.80|53.94|58.79|2756.74|2761.59|-1760.30| +2450841|14963|2450844|77698|1288847|3996|30745|77698|1288847|3996|30745|4|61|6|4|958|153|1330|83|26.36|36.37|9.09|2264.24|754.47|2187.88|3018.71|60.35|0.00|1418.47|754.47|814.82|2172.94|2233.29|-1433.41| +2450841|33689|2450863|91643|1601324|2205|23886|93335|1489201|2082|16563|2|16|11|1|7222|198|1331|35|68.47|143.78|104.95|1359.05|3673.25|2396.45|5032.30|121.95|624.45|1408.75|3048.80|3170.75|4457.55|4579.50|652.35| +2450841|33689|2450915|91643|1601324|2205|23886|93335|1489201|2082|16563|2|58|18|4|4822|241|1331|37|85.04|107.15|19.28|3251.19|713.36|3146.48|3964.55|19.47|435.14|1863.32|278.22|297.69|2141.54|2161.01|-2868.26| +2450841|33689|2450862|91643|1601324|2205|23886|93335|1489201|2082|16563|2|27|10|2|17833|246|1331|24|88.20|261.07|159.25|2443.68|3822.00|2116.80|6265.68|111.22|114.66|1253.04|3707.34|3818.56|4960.38|5071.60|1590.54| +2450841|33689|2450910|91643|1601324|2205|23886|93335|1489201|2082|16563|2|96|3|4|2180|86|1331|29|38.56|104.11|53.09|1479.58|1539.61|1118.24|3019.19|69.28|384.90|513.01|1154.71|1223.99|1667.72|1737.00|36.47| +2450841|33689|2450923|91643|1601324|2205|23886|93335|1489201|2082|16563|2|52|6|5|13040|285|1331|93|89.19|232.78|123.37|10175.13|11473.41|8294.67|21648.54|123.91|7342.98|2380.80|4130.43|4254.34|6511.23|6635.14|-4164.24| +2450841|33689|2450905|91643|1601324|2205|23886|93335|1489201|2082|16563|2|13|3|4|5731|169|1331|12|73.23|90.07|29.72|724.20|356.64|878.76|1080.84|17.83|0.00|399.84|356.64|374.47|756.48|774.31|-522.12| +2450841|33689|2450861|91643|1601324|2205|23886|93335|1489201|2082|16563|2|17|7|1|6938|29|1331|35|21.42|23.56|12.72|379.40|445.20|749.70|824.60|0.00|84.58|40.95|360.62|360.62|401.57|401.57|-389.08| +2450841|33689|2450856|91643|1601324|2205|23886|93335|1489201|2082|16563|2|77|18|3|9313|41|1331|12|1.64|4.19|1.25|35.28|15.00|19.68|50.28|0.60|0.00|16.56|15.00|15.60|31.56|32.16|-4.68| +2450841|33689|2450848|91643|1601324|2205|23886|93335|1489201|2082|16563|2|60|13|1|8702|25|1331|92|19.44|43.54|27.43|1482.12|2523.56|1788.48|4005.68|176.64|0.00|1601.72|2523.56|2700.20|4125.28|4301.92|735.08| +2450841|71135|2450917|86526|528853|5133|17813|72131|740373|3500|14980|1|7|11|1|7405|166|1332|97|16.49|27.53|17.34|988.43|1681.98|1599.53|2670.41|33.63|0.00|132.89|1681.98|1715.61|1814.87|1848.50|82.45| +2450841|71135|2450844|86526|528853|5133|17813|72131|740373|3500|14980|1|99|8|3|13388|281|1332|45|52.63|114.73|58.51|2529.90|2632.95|2368.35|5162.85|0.00|0.00|257.85|2632.95|2632.95|2890.80|2890.80|264.60| +2450841|71135|2450850|86526|528853|5133|17813|72131|740373|3500|14980|1|67|9|4|3901|149|1332|9|14.38|35.23|12.33|206.10|110.97|129.42|317.07|0.00|11.09|126.81|99.88|99.88|226.69|226.69|-29.54| +2450841|71135|2450846|86526|528853|5133|17813|72131|740373|3500|14980|1|82|12|5|235|122|1332|89|51.85|128.06|120.37|684.41|10712.93|4614.65|11397.34|107.12|0.00|341.76|10712.93|10820.05|11054.69|11161.81|6098.28| +2450841|71135|2450881|86526|528853|5133|17813|72131|740373|3500|14980|1|23|11|4|5767|258|1332|93|26.32|60.79|55.92|452.91|5200.56|2447.76|5653.47|52.00|0.00|2374.29|5200.56|5252.56|7574.85|7626.85|2752.80| +2450841|71135|2450854|86526|528853|5133|17813|72131|740373|3500|14980|1|80|18|4|4252|206|1332|2|53.39|134.54|44.39|180.30|88.78|106.78|269.08|1.77|0.00|110.32|88.78|90.55|199.10|200.87|-18.00| +2450841|75475|2450893|85338|1371209|7127|10198|85338|1371209|7127|10198|2|38|1|1|12724|186|1333|78|73.63|88.35|9.71|6133.92|757.38|5743.14|6891.30|68.16|0.00|0.00|757.38|825.54|757.38|825.54|-4985.76| +2450841|75475|2450892|85338|1371209|7127|10198|85338|1371209|7127|10198|2|39|12|2|1004|63|1333|65|44.66|75.02|7.50|4388.80|487.50|2902.90|4876.30|16.18|82.87|1608.75|404.63|420.81|2013.38|2029.56|-2498.27| +2450841|75475|2450857|85338|1371209|7127|10198|85338|1371209|7127|10198|2|66|9|1|2404|272|1333|50|64.46|176.62|0.00|8831.00|0.00|3223.00|8831.00|0.00|0.00|2649.00|0.00|0.00|2649.00|2649.00|-3223.00| +2450841|75475|2450875|85338|1371209|7127|10198|85338|1371209|7127|10198|2|16|6|2|2593|222|1333|82|71.61|97.38|87.64|798.68|7186.48|5872.02|7985.16|431.18|0.00|3352.98|7186.48|7617.66|10539.46|10970.64|1314.46| +2450841|75475|2450871|85338|1371209|7127|10198|85338|1371209|7127|10198|2|49|7|2|9169|137|1333|57|78.32|200.49|188.46|685.71|10742.22|4464.24|11427.93|106.34|9560.57|4113.69|1181.65|1287.99|5295.34|5401.68|-3282.59| +2450841|75475|2450926|85338|1371209|7127|10198|85338|1371209|7127|10198|2|86|12|5|12332|33|1333|97|41.61|46.18|6.92|3808.22|671.24|4036.17|4479.46|33.56|0.00|1701.38|671.24|704.80|2372.62|2406.18|-3364.93| +2450841|75475|2450914|85338|1371209|7127|10198|85338|1371209|7127|10198|2|51|15|5|3685|160|1333|57|28.91|43.36|13.87|1680.93|790.59|1647.87|2471.52|55.34|0.00|246.81|790.59|845.93|1037.40|1092.74|-857.28| +2450841|75475|2450880|85338|1371209|7127|10198|85338|1371209|7127|10198|2|6|5|3|15280|262|1333|7|7.42|17.43|13.24|29.33|92.68|51.94|122.01|2.78|0.00|4.83|92.68|95.46|97.51|100.29|40.74| +2450841|75475|2450901|85338|1371209|7127|10198|85338|1371209|7127|10198|2|13|9|4|8521|273|1333|1|68.91|130.23|92.46|37.77|92.46|68.91|130.23|7.39|0.00|7.81|92.46|99.85|100.27|107.66|23.55| +2450841|75475|2450865|85338|1371209|7127|10198|85338|1371209|7127|10198|2|13|16|3|2408|215|1333|16|67.58|78.39|61.92|263.52|990.72|1081.28|1254.24|45.07|89.16|175.52|901.56|946.63|1077.08|1122.15|-179.72| +2450841|75475|2450884|85338|1371209|7127|10198|85338|1371209|7127|10198|2|76|15|4|5371|77|1333|76|93.87|169.90|32.28|10459.12|2453.28|7134.12|12912.40|0.00|0.00|1548.88|2453.28|2453.28|4002.16|4002.16|-4680.84| +2450841|75475|2450885|85338|1371209|7127|10198|85338|1371209|7127|10198|2|2|9|3|3958|136|1333|42|41.04|108.34|92.08|682.92|3867.36|1723.68|4550.28|270.71|0.00|0.00|3867.36|4138.07|3867.36|4138.07|2143.68| +2450841|75475|2450863|85338|1371209|7127|10198|85338|1371209|7127|10198|2|19|17|4|15268|228|1333|88|70.71|198.69|162.92|3147.76|14336.96|6222.48|17484.72|430.10|0.00|349.36|14336.96|14767.06|14686.32|15116.42|8114.48| +2450841|75475|2450908|85338|1371209|7127|10198|85338|1371209|7127|10198|2|72|10|4|10724|115|1333|33|84.77|211.07|208.95|69.96|6895.35|2797.41|6965.31|275.81|0.00|3343.23|6895.35|7171.16|10238.58|10514.39|4097.94| +2450841|85737|2450922|18918|709285|2499|9666|18918|709285|2499|9666|1|92|2|3|15428|252|1334|4|84.87|86.56|48.47|152.36|193.88|339.48|346.24|1.93|0.00|38.08|193.88|195.81|231.96|233.89|-145.60| +||2450917||709285|||18918|||||79|11|1|1682|74|1334|26|19.35|||446.42||503.10||12.86|0.00|||441.60|796.12|808.98|-74.36| +2450841|85737|2450906|18918|709285|2499|9666|18918|709285|2499|9666|1|88|13|3|2020|200|1334|4|31.54|33.43|9.36|96.28|37.44|126.16|133.72|2.99|0.00|46.80|37.44|40.43|84.24|87.23|-88.72| +2450841|85737|2450919|18918|709285|2499|9666|18918|709285|2499|9666|1|40|19|4|12526|286|1334|41|37.95|54.26|8.13|1891.33|333.33|1555.95|2224.66|9.99|0.00|311.19|333.33|343.32|644.52|654.51|-1222.62| +2450841|85737|2450908|18918|709285|2499|9666|18918|709285|2499|9666|1|2|9|3|2176|106|1334|12|16.43|49.29|3.94|544.20|47.28|197.16|591.48|1.41|0.00|272.04|47.28|48.69|319.32|320.73|-149.88| +2450841|85737|2450929|18918|709285|2499|9666|18918|709285|2499|9666|1|102|10|3|10886|76|1334|87|54.35|61.41|58.33|267.96|5074.71|4728.45|5342.67|152.24|0.00|0.00|5074.71|5226.95|5074.71|5226.95|346.26| +2450841|85737|2450916|18918|709285|2499|9666|18918|709285|2499|9666|1|101|13|1|2797|164|1334|7|17.67|25.44|16.53|62.37|115.71|123.69|178.08|0.50|102.98|58.73|12.73|13.23|71.46|71.96|-110.96| +2450841|85737|2450931|18918|709285|2499|9666|18918|709285|2499|9666|1|22|2|5|5392|15|1334|10|4.67|12.00|9.00|30.00|90.00|46.70|120.00|0.00|0.00|56.40|90.00|90.00|146.40|146.40|43.30| +2450841|85737|2450843|18918|709285|2499|9666|18918|709285|2499|9666|1|23|10|1|8042|120|1334|31|94.45|211.56|209.44|65.72|6492.64|2927.95|6558.36|519.41|0.00|2426.37|6492.64|7012.05|8919.01|9438.42|3564.69| +2450841|85737|2450882|18918|709285|2499|9666|18918|709285|2499|9666|1|63|5|3|3775|161|1334|89|9.12|23.80|17.85|529.55|1588.65|811.68|2118.20|11.91|1350.35|380.92|238.30|250.21|619.22|631.13|-573.38| +2450841|85737|2450844|18918|709285|2499|9666|18918|709285|2499|9666|1|1|5|3|3782|250|1334|100|65.30|116.23|53.46|6277.00|5346.00|6530.00|11623.00|37.42|1603.80|3951.00|3742.20|3779.62|7693.20|7730.62|-2787.80| +2450841|85737|2450886|18918|709285|2499|9666|18918|709285|2499|9666|1|82|17|3|12835|62|1334|19|24.85|36.03|14.77|403.94|280.63|472.15|684.57|5.61|0.00|171.00|280.63|286.24|451.63|457.24|-191.52| +2450841|18996|2450867|80309|205819|576|48086|80309|205819|576|48086|2|23|8|2|4826|72|1335|57|67.22|169.39|113.49|3186.30|6468.93|3831.54|9655.23|129.37|0.00|3475.86|6468.93|6598.30|9944.79|10074.16|2637.39| +2450841|18996|2450884|80309|205819|576|48086|80309|205819|576|48086|2|34|10|5|10262|20|1335|77|20.15|55.00|17.60|2879.80|1355.20|1551.55|4235.00|0.00|460.76|1312.85|894.44|894.44|2207.29|2207.29|-657.11| +2450841|18996|2450892|80309|205819|576|48086|80309|205819|576|48086|2|47|10|2|16702|179|1335|46|79.39|90.50|17.19|3372.26|790.74|3651.94|4163.00|63.25|0.00|1998.24|790.74|853.99|2788.98|2852.23|-2861.20| +2450841|18996|2450888|80309|205819|576|48086|80309|205819|576|48086|2|105|7|2|14320|238|1335|25|72.30|173.52|50.32|3080.00|1258.00|1807.50|4338.00|37.74|0.00|1605.00|1258.00|1295.74|2863.00|2900.74|-549.50| +||2450920|80309|||48086||205819|576|48086||40|||7316||1335||29.66|||||2610.08|5115.44||1391.38||||5053.74|5112.87|346.62| +2450841|18996|2450852|80309|205819|576|48086|80309|205819|576|48086|2|46|15|5|6859|141|1335|35|77.14|106.45|22.35|2943.50|782.25|2699.90|3725.75|7.82|0.00|1601.95|782.25|790.07|2384.20|2392.02|-1917.65| +2450841|18996|2450859|80309|205819|576|48086|80309|205819|576|48086|2|34|13|1|7963|74|1335|93|85.33|249.16|181.88|6257.04|16914.84|7935.69|23171.88|0.00|0.00|4865.76|16914.84|16914.84|21780.60|21780.60|8979.15| +2450841|18996|2450881|80309|205819|576|48086|80309|205819|576|48086|2|77|18|5|4216|67|1335|23|14.72|28.70|6.60|508.30|151.80|338.56|660.10|13.66|0.00|230.92|151.80|165.46|382.72|396.38|-186.76| +2450841|18996|2450898|80309|205819|576|48086|80309|205819|576|48086|2|66|15|3|7220|279|1335|22|12.63|17.80|14.41|74.58|317.02|277.86|391.60|19.02|0.00|86.02|317.02|336.04|403.04|422.06|39.16| +2450841|18996|2450928|80309|205819|576|48086|80309|205819|576|48086|2|106|11|3|956|55|1335|22|9.76|9.85|9.16|15.18|201.52|214.72|216.70|0.00|0.00|64.90|201.52|201.52|266.42|266.42|-13.20| +2450841|18996|2450904|80309|205819|576|48086|80309|205819|576|48086|2|93|20|4|14341|202|1335|44|78.47|123.98|63.22|2673.44|2781.68|3452.68|5455.12|222.53|0.00|545.16|2781.68|3004.21|3326.84|3549.37|-671.00| +2450841|18996|2450859|80309|205819|576|48086|80309|205819|576|48086|2|43|4|5|1579|147|1335|42|22.24|39.58|10.68|1213.80|448.56|934.08|1662.36|19.73|201.85|448.56|246.71|266.44|695.27|715.00|-687.37| +2450841|18996|2450892|80309|205819|576|48086|80309|205819|576|48086|2|69|5|3|446|9|1335|77|43.66|117.88|75.44|3267.88|5808.88|3361.82|9076.76|464.71|0.00|180.95|5808.88|6273.59|5989.83|6454.54|2447.06| +2450841|13274|2450872|75917|114928|2386|1797|75917|114928|2386|1797|4|4|11|1|17794|96|1336|79|88.97|225.98|47.45|14103.87|3748.55|7028.63|17852.42|56.22|937.13|3927.09|2811.42|2867.64|6738.51|6794.73|-4217.21| +2450841|13274|2450846|75917|114928|2386|1797|75917|114928|2386|1797|4|61|10|2|2128|256|1336|66|32.81|68.57|15.08|3530.34|995.28|2165.46|4525.62|58.52|159.24|949.74|836.04|894.56|1785.78|1844.30|-1329.42| +2450841|13274|2450879|75917|114928|2386|1797|75917|114928|2386|1797|4|4|5|1|6103|1|1336|6|71.26|203.09|85.29|706.80|511.74|427.56|1218.54|25.58|0.00|377.70|511.74|537.32|889.44|915.02|84.18| +2450841|13274|2450922|75917|114928|2386|1797|75917|114928|2386|1797|4|107|8|3|12268|133|1336|57|70.85|72.97|11.67|3494.10|665.19|4038.45|4159.29|46.56|0.00|1247.73|665.19|711.75|1912.92|1959.48|-3373.26| +2450841|13274|2450923|75917|114928|2386|1797|75917|114928|2386|1797|4|88|6|1|848|89|1336|88|5.78|7.28|4.00|288.64|352.00|508.64|640.64|31.68|0.00|88.88|352.00|383.68|440.88|472.56|-156.64| +2450841|13274|2450898|75917|114928|2386|1797|75917|114928|2386|1797|4|4|11|1|11071|19|1336|8|40.91|92.45|85.05|59.20|680.40|327.28|739.60|6.80|0.00|14.72|680.40|687.20|695.12|701.92|353.12| +2450841|13274|2450878|75917|114928|2386|1797|75917|114928|2386|1797|4|64|12|3|17056|94|1336|73|65.61|192.23|142.25|3648.54|10384.25|4789.53|14032.79|311.52|0.00|3227.33|10384.25|10695.77|13611.58|13923.10|5594.72| +2450841|13274|2450882|75917|114928|2386|1797|75917|114928|2386|1797|4|103|11|1|9586|123|1336|27|85.66|144.76|59.35|2306.07|1602.45|2312.82|3908.52|16.02|0.00|312.66|1602.45|1618.47|1915.11|1931.13|-710.37| +2450841|13274|2450914|75917|114928|2386|1797|75917|114928|2386|1797|4|95|14|2|14528|165|1336|54|10.10|25.35|23.82|82.62|1286.28|545.40|1368.90|65.98|553.10|218.70|733.18|799.16|951.88|1017.86|187.78| +2450841|13274|2450854|75917|114928|2386|1797|75917|114928|2386|1797|4|41|2|4|6988|161|1336|49|68.62|75.48|54.34|1035.86|2662.66|3362.38|3698.52|106.50|0.00|1220.10|2662.66|2769.16|3882.76|3989.26|-699.72| +2450841|33262|2450881|81137|424914|3891|31268|81137|424914|3891|31268|4|108|17|1|4366|154|1337|35|76.53|122.44|42.85|2785.65|1499.75|2678.55|4285.40|104.98|0.00|899.85|1499.75|1604.73|2399.60|2504.58|-1178.80| +2450841|33262|2450922|81137|424914|3891|31268|81137|424914|3891|31268|4|8|12|3|3646|180|1337|47|47.15|121.64|85.14|1715.50|4001.58|2216.05|5717.08|200.07|0.00|971.49|4001.58|4201.65|4973.07|5173.14|1785.53| +2450841|33262|2450850|81137|424914|3891|31268|81137|424914|3891|31268|4|80|9|3|14260|166|1337|97|60.59|116.33|96.55|1918.66|9365.35|5877.23|11284.01|352.13|4963.63|5190.47|4401.72|4753.85|9592.19|9944.32|-1475.51| +2450841|33262|2450886|81137|424914|3891|31268|81137|424914|3891|31268|4|37|16|1|15740|233|1337|20|67.66|126.52|126.52|0.00|2530.40|1353.20|2530.40|151.82|0.00|835.00|2530.40|2682.22|3365.40|3517.22|1177.20| +2450841|33262|2450843|81137|424914|3891|31268|81137|424914|3891|31268|4|97|14|1|17936|240|1337|69|10.90|15.36|14.59|53.13|1006.71|752.10|1059.84|70.46|0.00|105.57|1006.71|1077.17|1112.28|1182.74|254.61| +2450841|46726|2450912|20224|271734|2444|37907|20224|271734|2444|37907|2|52|13|5|17623|37|1338|100|75.64|185.31|111.18|7413.00|11118.00|7564.00|18531.00|0.00|0.00|555.00|11118.00|11118.00|11673.00|11673.00|3554.00| +2450841|46726|2450901|20224|271734|2444|37907|20224|271734|2444|37907|2|58|4|3|7540|29|1338|67|64.30|167.82|120.83|3148.33|8095.61|4308.10|11243.94|647.64|0.00|5621.97|8095.61|8743.25|13717.58|14365.22|3787.51| +2450841|46726|2450913|20224|271734|2444|37907|20224|271734|2444|37907|2|16|16|4|6787|60|1338|42|17.15|44.41|26.20|764.82|1100.40|720.30|1865.22|33.01|0.00|671.16|1100.40|1133.41|1771.56|1804.57|380.10| +2450841|46726|2450870|20224|271734|2444|37907|20224|271734|2444|37907|2|2|14|4|272|28|1338|46|53.12|68.52|28.77|1828.50|1323.42|2443.52|3151.92|105.87|0.00|977.04|1323.42|1429.29|2300.46|2406.33|-1120.10| +2450841|69227|2450914|45361|1665779|2550|15768|45361|1665779|2550|15768|4|76|18|3|14444|113|1339|22|15.74|38.40|2.30|794.20|50.60|346.28|844.80|1.08|28.84|42.24|21.76|22.84|64.00|65.08|-324.52| +2450841|69227|2450924|45361|1665779|2550|15768|45361|1665779|2550|15768|4|19|3|5|518|109|1339|28|42.49|117.27|52.77|1806.00|1477.56|1189.72|3283.56|33.24|1108.17|360.92|369.39|402.63|730.31|763.55|-820.33| +2450841|69227|2450908|45361|1665779|2550|15768|45361|1665779|2550|15768|4|42|10|1|5551|191|1339|69|77.72|216.06|125.31|6261.75|8646.39|5362.68|14908.14|345.85|4323.19|4919.01|4323.20|4669.05|9242.21|9588.06|-1039.48| +2450841|69227|2450857|45361|1665779|2550|15768|45361|1665779|2550|15768|4|28|11|5|10405|291|1339|80|89.64|172.10|144.56|2203.20|11564.80|7171.20|13768.00|166.53|6013.69|1927.20|5551.11|5717.64|7478.31|7644.84|-1620.09| +2450841|69227|2450916|45361|1665779|2550|15768|45361|1665779|2550|15768|4|93|3|3|7027|86|1339|79|42.18|72.97|18.24|4323.67|1440.96|3332.22|5764.63|14.40|0.00|2074.54|1440.96|1455.36|3515.50|3529.90|-1891.26| +2450841|69227|2450908|45361|1665779|2550|15768|45361|1665779|2550|15768|4|58|20|2|3109|39|1339|63|20.01|28.41|6.81|1360.80|429.03|1260.63|1789.83|21.45|0.00|35.28|429.03|450.48|464.31|485.76|-831.60| +2450841|69227|2450864|45361|1665779|2550|15768|45361|1665779|2550|15768|4|102|5|3|6410|22|1339|14|83.60|183.08|146.46|512.68|2050.44|1170.40|2563.12|164.03|0.00|307.44|2050.44|2214.47|2357.88|2521.91|880.04| +2450841|69227|2450862|45361|1665779|2550|15768|45361|1665779|2550|15768|4|69|19|5|7172|244|1339|88|3.20|8.09|5.33|242.88|469.04|281.60|711.92|18.76|0.00|198.88|469.04|487.80|667.92|686.68|187.44| +2450841|69227|2450906|45361|1665779|2550|15768|45361|1665779|2550|15768|4|79|11|2|14050|201|1339|54|34.09|90.67|45.33|2448.36|2447.82|1840.86|4896.18|57.27|1811.38|1468.80|636.44|693.71|2105.24|2162.51|-1204.42| +2450841|74999|2450885|88232|1597817|6155|17293|88232|1597817|6155|17293|4|42|3|5|9230|205|1340|71|70.35|166.02|116.21|3536.51|8250.91|4994.85|11787.42|742.58|0.00|2828.64|8250.91|8993.49|11079.55|11822.13|3256.06| +2450841|74999|2450878|88232|1597817|6155|17293|88232|1597817|6155|17293|4|40|3|4|12148|224|1340|21|69.40|106.18|55.21|1070.37|1159.41|1457.40|2229.78|11.59|0.00|156.03|1159.41|1171.00|1315.44|1327.03|-297.99| +2450841|74999|2450877|88232|1597817|6155|17293|88232|1597817|6155|17293|4|61|18|5|13166|26|1340|100|53.91|154.18|29.29|12489.00|2929.00|5391.00|15418.00|29.29|0.00|2158.00|2929.00|2958.29|5087.00|5116.29|-2462.00| +2450841|74999|2450917|88232|1597817|6155|17293|88232|1597817|6155|17293|4|18|18|4|7549|268|1340|46|36.88|85.19|32.37|2429.72|1489.02|1696.48|3918.74|104.23|0.00|1214.40|1489.02|1593.25|2703.42|2807.65|-207.46| +2450841|74999|2450922|88232|1597817|6155|17293|88232|1597817|6155|17293|4|90|16|2|5485|16|1340|32|13.92|36.88|17.33|625.60|554.56|445.44|1180.16|49.91|0.00|224.00|554.56|604.47|778.56|828.47|109.12| +2450841|74999|2450871|88232|1597817|6155|17293|88232|1597817|6155|17293|4|103|1|4|8062|230|1340|30|25.42|41.18|25.53|469.50|765.90|762.60|1235.40|0.00|0.00|98.70|765.90|765.90|864.60|864.60|3.30| +2450841|74999|2450894|88232|1597817|6155|17293|88232|1597817|6155|17293|4|36|17|3|4237|145|1340|62|71.09|100.94|29.27|4443.54|1814.74|4407.58|6258.28|36.29|0.00|2377.70|1814.74|1851.03|4192.44|4228.73|-2592.84| +2450841|74999|2450881|88232|1597817|6155|17293|88232|1597817|6155|17293|4|23|2|3|7621|92|1340|44|89.47|263.93|97.65|7316.32|4296.60|3936.68|11612.92|343.72|0.00|3948.12|4296.60|4640.32|8244.72|8588.44|359.92| +2450841|74999|2450889|88232|1597817|6155|17293|88232|1597817|6155|17293|4|20|9|3|7051|296|1340|61|2.70|6.99|6.36|38.43|387.96|164.70|426.39|27.15|0.00|131.76|387.96|415.11|519.72|546.87|223.26| +2450841|74999|2450914|88232|1597817|6155|17293|88232|1597817|6155|17293|4|101|16|2|3160|262|1340|53|27.63|54.98|44.53|553.85|2360.09|1464.39|2913.94|94.40|0.00|57.77|2360.09|2454.49|2417.86|2512.26|895.70| +2450841|74999|2450896|88232|1597817|6155|17293|88232|1597817|6155|17293|4|60|20|3|1072|6|1340|87|75.56|91.42|75.87|1352.85|6600.69|6573.72|7953.54|0.00|264.02|476.76|6336.67|6336.67|6813.43|6813.43|-237.05| +2450841|74999|2450855|88232|1597817|6155|17293|88232|1597817|6155|17293|4|40|14|3|16276|247|1340|24|32.90|46.71|27.55|459.84|661.20|789.60|1121.04|59.50|0.00|190.56|661.20|720.70|851.76|911.26|-128.40| +2450841|74999|2450857|88232|1597817|6155|17293|88232|1597817|6155|17293|4|100|18|4|10864|63|1340|44|88.86|230.14|202.52|1215.28|8910.88|3909.84|10126.16|356.43|0.00|911.24|8910.88|9267.31|9822.12|10178.55|5001.04| +2450841|75874|2450851|81349|1682534|4872|10|81349|1682534|4872|10|1|2|18|4|5594|286|1341|45|33.92|94.29|35.83|2630.70|1612.35|1526.40|4243.05|14.51|1322.12|1102.95|290.23|304.74|1393.18|1407.69|-1236.17| +2450841|75874|2450882|81349|1682534|4872|10|81349|1682534|4872|10|1|10|14|5|17954|252|1341|80|29.17|81.96|55.73|2098.40|4458.40|2333.60|6556.80|267.50|0.00|64.80|4458.40|4725.90|4523.20|4790.70|2124.80| +2450841|75874|2450921|81349|1682534|4872|10|81349|1682534|4872|10|1|96|14|5|2143|280|1341|86|59.43|162.24|11.35|12976.54|976.10|5110.98|13952.64|0.00|0.00|3627.48|976.10|976.10|4603.58|4603.58|-4134.88| +2450841|75874|2450895|81349|1682534|4872|10|81349|1682534|4872|10|1|21|10|3|4208|4|1341|95|24.20|55.90|50.31|531.05|4779.45|2299.00|5310.50|191.17|0.00|1751.80|4779.45|4970.62|6531.25|6722.42|2480.45| +2450841|75874|2450869|81349|1682534|4872|10|81349|1682534|4872|10|1|21|10|4|5852|87|1341|86|6.39|13.73|2.60|957.18|223.60|549.54|1180.78|13.41|0.00|424.84|223.60|237.01|648.44|661.85|-325.94| +2450841|75874|2450855|81349|1682534|4872|10|81349|1682534|4872|10|1|40|1|3|14749|231|1341|42|57.42|84.40|0.84|3509.52|35.28|2411.64|3544.80|3.17|0.00|1169.70|35.28|38.45|1204.98|1208.15|-2376.36| +2450841|75874|2450870|81349|1682534|4872|10|81349|1682534|4872|10|1|93|8|2|14977|68|1341|19|12.00|17.04|8.00|171.76|152.00|228.00|323.76|1.52|0.00|139.08|152.00|153.52|291.08|292.60|-76.00| +2450841|75874|2450849|81349|1682534|4872|10|81349|1682534|4872|10|1|98|8|2|301|277|1341|58|5.26|11.15|7.47|213.44|433.26|305.08|646.70|38.99|0.00|187.34|433.26|472.25|620.60|659.59|128.18| +2450841|75874|2450880|81349|1682534|4872|10|81349|1682534|4872|10|1|54|7|3|12523|102|1341|69|91.55|147.39|135.59|814.20|9355.71|6316.95|10169.91|280.67|0.00|4067.55|9355.71|9636.38|13423.26|13703.93|3038.76| +2450841|75874|2450844|81349|1682534|4872|10|81349|1682534|4872|10|1|46|12|1|17575|141|1341|41|82.90|104.45|100.27|171.38|4111.07|3398.90|4282.45|14.38|2672.19|1883.95|1438.88|1453.26|3322.83|3337.21|-1960.02| +2450841|75874|2450890|81349|1682534|4872|10|81349|1682534|4872|10|1|55|14|3|17392|10|1341|70|20.77|29.70|4.45|1767.50|311.50|1453.90|2079.00|9.84|65.41|394.80|246.09|255.93|640.89|650.73|-1207.81| +2450841|85723|2450856|86310|25681|1852|25976|86310|25681|1852|25976|2|18|3|2|4789|257|1342|14|97.18|226.42|31.69|2726.22|443.66|1360.52|3169.88|13.30|0.00|1236.20|443.66|456.96|1679.86|1693.16|-916.86| +2450841|85723|2450859|86310|25681|1852|25976|86310|25681|1852|25976|2|34|10|3|11204|246|1342|83|6.94|9.09|0.81|687.24|67.23|576.02|754.47|6.05|0.00|74.70|67.23|73.28|141.93|147.98|-508.79| +2450841|85723|2450843|86310|25681|1852|25976|86310|25681|1852|25976|2|16|18|5|15979|161|1342|97|23.73|36.30|9.80|2570.50|950.60|2301.81|3521.10|0.00|0.00|1443.36|950.60|950.60|2393.96|2393.96|-1351.21| +2450841|85723|2450870|86310|25681|1852|25976|86310|25681|1852|25976|2|90|17|3|17467|228|1342|52|12.93|17.84|8.56|482.56|445.12|672.36|927.68|40.06|0.00|315.12|445.12|485.18|760.24|800.30|-227.24| +2450841|85723|2450875|86310|25681|1852|25976|86310|25681|1852|25976|2|8|19|5|9770|82|1342|34|72.80|88.81|68.38|694.62|2324.92|2475.20|3019.54|46.49|0.00|1237.94|2324.92|2371.41|3562.86|3609.35|-150.28| +2450841|85723|2450851|86310|25681|1852|25976|86310|25681|1852|25976|2|9|3|5|2305|24|1342|6|16.98|36.67|8.43|169.44|50.58|101.88|220.02|0.16|46.53|4.38|4.05|4.21|8.43|8.59|-97.83| +2450841|85723|2450884|86310|25681|1852|25976|86310|25681|1852|25976|2|95|16|3|14414|222|1342|71|33.06|67.77|23.04|3175.83|1635.84|2347.26|4811.67|130.86|0.00|1106.18|1635.84|1766.70|2742.02|2872.88|-711.42| +2450841|85723|2450849|86310|25681|1852|25976|86310|25681|1852|25976|2|62|5|3|12130|124|1342|19|81.52|106.79|95.04|223.25|1805.76|1548.88|2029.01|36.11|0.00|568.10|1805.76|1841.87|2373.86|2409.97|256.88| +2450841|85723|2450911|86310|25681|1852|25976|86310|25681|1852|25976|2|30|3|2|17390|84|1342|41|12.62|24.10|2.65|879.45|108.65|517.42|988.10|5.43|0.00|207.46|108.65|114.08|316.11|321.54|-408.77| +2450841|85723|2450922|86310|25681|1852|25976|86310|25681|1852|25976|2|47|19|4|4646|79|1342|9|79.23|87.94|22.86|585.72|205.74|713.07|791.46|8.22|0.00|7.83|205.74|213.96|213.57|221.79|-507.33| +2450841|70797|2450902|19341|1783790|7086|12382|19341|1783790|7086|12382|1|71|14|5|13195|85|1343|64|54.83|86.63|51.97|2218.24|3326.08|3509.12|5544.32|0.00|2926.95|2328.32|399.13|399.13|2727.45|2727.45|-3109.99| +2450841|70797|2450870|19341|1783790|7086|12382|19341|1783790|7086|12382|1|83|5|2|6236|263|1343|61|99.56|123.45|17.28|6476.37|1054.08|6073.16|7530.45|21.08|0.00|1430.45|1054.08|1075.16|2484.53|2505.61|-5019.08| +2450841|70797|2450856|19341|1783790|7086|12382|19341|1783790|7086|12382|1|48|2|2|3718|82|1343|32|31.81|41.35|23.98|555.84|767.36|1017.92|1323.20|23.02|0.00|423.36|767.36|790.38|1190.72|1213.74|-250.56| +2450841|70797|2450927|19341|1783790|7086|12382|19341|1783790|7086|12382|1|17|3|2|17017|157|1343|26|29.76|37.79|35.14|68.90|913.64|773.76|982.54|54.81|0.00|39.26|913.64|968.45|952.90|1007.71|139.88| +2450841|60091|2450867|49147|1280351|5955|21559|49147|1280351|5955|21559|2|52|2|3|5018|153|1344|79|13.32|37.02|11.84|1989.22|935.36|1052.28|2924.58|84.18|0.00|993.82|935.36|1019.54|1929.18|2013.36|-116.92| +2450841|60091|2450859|49147|1280351|5955|21559|49147|1280351|5955|21559|2|76|6|1|8998|129|1344|88|1.19|2.21|0.90|115.28|79.20|104.72|194.48|2.37|0.00|79.20|79.20|81.57|158.40|160.77|-25.52| +2450841|60091|2450913|49147|1280351|5955|21559|49147|1280351|5955|21559|2|69|8|4|17090|179|1344|94|1.05|1.85|0.51|125.96|47.94|98.70|173.90|1.43|0.00|65.80|47.94|49.37|113.74|115.17|-50.76| +2450841|60091|2450880|49147|1280351||21559|||5955|21559|2|23||2|6931|39|1344||35.44|||7855.14|||8925.81|0.00|0.00|802.78||1070.67|1873.45|1873.45|-2083.49| +2450841|54902|2450867|99355|635427|299|4818|99355|635427|299|4818|2|41|2|2|15782|167|1345|48|82.02|127.95|43.50|4053.60|2088.00|3936.96|6141.60|41.76|0.00|2640.48|2088.00|2129.76|4728.48|4770.24|-1848.96| +2450841|54902|2450885|99355|635427|299|4818|99355|635427|299|4818|2|51|9|4|8611|161|1345|90|43.61|93.76|3.75|8100.90|337.50|3924.90|8438.40|3.37|0.00|3797.10|337.50|340.87|4134.60|4137.97|-3587.40| +2450841|54902|2450875|99355|635427|299|4818|99355|635427|299|4818|2|94|10|3|3832|295|1345|34|51.27|85.10|34.04|1736.04|1157.36|1743.18|2893.40|92.58|0.00|1012.52|1157.36|1249.94|2169.88|2262.46|-585.82| +2450841|54902|2450848|99355|635427|299|4818|99355|635427|299|4818|2|40|18|4|16700|224|1345|29|12.25|18.13|3.26|431.23|94.54|355.25|525.77|7.56|0.00|225.91|94.54|102.10|320.45|328.01|-260.71| +2450841|54902|2450870|99355|635427|299|4818|99355|635427|299|4818|2|45|16|3|8524|73|1345|46|80.12|128.19|78.19|2300.00|3596.74|3685.52|5896.74|0.00|3344.96|1414.96|251.78|251.78|1666.74|1666.74|-3433.74| +2450841|54902|2450858|99355|635427|299|4818|99355|635427|299|4818|2|64|11|5|10003|253|1345|63|74.29|101.03|80.82|1273.23|5091.66|4680.27|6364.89|203.66|0.00|0.00|5091.66|5295.32|5091.66|5295.32|411.39| +2450841|18616|2450907|47978|693538|2298|3458|32122|1852578|3738|44098|4|77|4|3|11611|239|1346|67|75.83|84.17|70.70|902.49|4736.90|5080.61|5639.39|47.36|0.00|168.84|4736.90|4784.26|4905.74|4953.10|-343.71| +2450841|18616|2450866|47978|693538|2298|3458|32122|1852578|3738|44098|4|94|15|2|12667|113|1346|68|56.38|156.17|29.67|8602.00|2017.56|3833.84|10619.56|100.87|0.00|2229.72|2017.56|2118.43|4247.28|4348.15|-1816.28| +2450841|18616|2450861|47978|693538|2298|3458|32122|1852578|3738|44098|4|52|17|5|11101|205|1346|56|62.51|134.39|67.19|3763.20|3762.64|3500.56|7525.84|60.20|752.52|1806.00|3010.12|3070.32|4816.12|4876.32|-490.44| +2450841|18616|2450904|47978|693538|2298|3458|32122|1852578|3738|44098|4|37|17|4|4006|11|1346|32|29.20|82.63|74.36|264.64|2379.52|934.40|2644.16|0.00|1832.23|978.24|547.29|547.29|1525.53|1525.53|-387.11| +2450841|18616|2450870|47978|693538|2298|3458|32122|1852578|3738|44098|4|99|3|4|17293|161|1346|94|5.59|7.37|6.33|97.76|595.02|525.46|692.78|17.85|0.00|193.64|595.02|612.87|788.66|806.51|69.56| +2450841|18616|2450890|47978|693538|2298|3458|32122|1852578|3738|44098|4|6|5|5|2650|251|1346|92|65.00|135.85|8.15|11748.40|749.80|5980.00|12498.20|7.49|0.00|3749.00|749.80|757.29|4498.80|4506.29|-5230.20| +2450841|18616|2450896|47978|693538|2298|3458|32122|1852578|3738|44098|4|10|14|5|16129|245|1346|19|13.41|25.88|18.11|147.63|344.09|254.79|491.72|20.64|0.00|127.68|344.09|364.73|471.77|492.41|89.30| +2450841|18616|2450920|47978|693538|2298|3458|32122|1852578|3738|44098|4|72|11|1|4520|265|1346|49|98.04|137.25|91.95|2219.70|4505.55|4803.96|6725.25|135.16|0.00|1546.44|4505.55|4640.71|6051.99|6187.15|-298.41| +2450841|18616|2450911|47978|693538|2298|3458|32122|1852578|3738|44098|4|23|17|5|10646|281|1346|42|9.33|10.07|4.12|249.90|173.04|391.86|422.94|13.84|0.00|109.62|173.04|186.88|282.66|296.50|-218.82| +2450841|18616|2450887|47978|693538|2298|3458|32122|1852578|3738|44098|4|38|3|4|8176|254|1346|48|8.06|19.42|1.35|867.36|64.80|386.88|932.16|3.88|0.00|251.52|64.80|68.68|316.32|320.20|-322.08| +2450841|18616|2450904|47978|693538|2298|3458|32122|1852578|3738|44098|4|27|13|2|11005|243|1346|27|96.04|216.09|196.64|525.15|5309.28|2593.08|5834.43|53.09|0.00|2625.48|5309.28|5362.37|7934.76|7987.85|2716.20| +2450841|18616|2450911|47978|693538|2298|3458|32122|1852578|3738|44098|4|89|3|2|10219|107|1346|94|41.07|52.56|35.21|1630.90|3309.74|3860.58|4940.64|165.48|0.00|2024.76|3309.74|3475.22|5334.50|5499.98|-550.84| +2450841|18616|2450852|47978|693538|2298|3458|32122|1852578|3738|44098|4|82|14|5|16820|7|1346|99|40.89|66.24|43.71|2230.47|4327.29|4048.11|6557.76|302.91|0.00|2622.51|4327.29|4630.20|6949.80|7252.71|279.18| +2450841|18616|2450846|47978|693538|2298|3458|32122|1852578|3738|44098|4|96|3|2|3385|67|1346|41|3.36|3.83|2.68|47.15|109.88|137.76|157.03|7.69|0.00|27.88|109.88|117.57|137.76|145.45|-27.88| +2450841|71168|2450928|47504|1096808|4512|5816|47504|1096808|4512|5816|4|34|9|3|2752|229|1347|1|5.14|12.18|11.08|1.10|11.08|5.14|12.18|0.22|0.00|0.60|11.08|11.30|11.68|11.90|5.94| +2450841|71168|2450882|47504|1096808|4512|5816|47504|1096808|4512|5816|4|53|11|2|1780|154|1347|18|88.94|217.01|30.38|3359.34|546.84|1600.92|3906.18|27.34|0.00|1601.46|546.84|574.18|2148.30|2175.64|-1054.08| +2450841|71168|2450879|47504|1096808|4512|5816|47504|1096808|4512|5816|4|75|14|2|604|172|1347|27|84.28|127.26|83.99|1168.29|2267.73|2275.56|3436.02|90.70|0.00|412.29|2267.73|2358.43|2680.02|2770.72|-7.83| +2450841|71168|2450885|47504|1096808|4512|5816|47504|1096808|4512|5816|4|4|8|2|2722|121|1347|91|59.49|65.43|56.26|834.47|5119.66|5413.59|5954.13|153.58|0.00|1785.42|5119.66|5273.24|6905.08|7058.66|-293.93| +2450841|71168|2450906|47504|1096808|4512|5816|47504|1096808|4512|5816|4|29|9|5|16660|185|1347|3|23.89|61.87|43.92|53.85|131.76|71.67|185.61|4.42|21.08|59.37|110.68|115.10|170.05|174.47|39.01| +2450841|71168|2450869|47504|1096808|4512|5816|47504|1096808|4512|5816|4|73|1|5|3218|19|1347|97|46.35|77.40|47.21|2928.43|4579.37|4495.95|7507.80|366.34|0.00|2627.73|4579.37|4945.71|7207.10|7573.44|83.42| +2450841|71168|2450876|47504|1096808|4512|5816|47504|1096808|4512|5816|4|36|10|1|14776|68|1347|89|23.25|69.28|37.41|2836.43|3329.49|2069.25|6165.92|0.00|0.00|61.41|3329.49|3329.49|3390.90|3390.90|1260.24| +2450841|71168|2450915|47504|1096808|4512|5816|47504|1096808|4512|5816|4|16|5|2|5986|167|1347|39|28.64|54.41|17.41|1443.00|678.99|1116.96|2121.99|61.10|0.00|127.14|678.99|740.09|806.13|867.23|-437.97| +2450841|71168|2450913|47504|1096808|4512|5816|47504|1096808|4512|5816|4|46|3|4|10933|98|1347|35|75.13|141.99|107.91|1192.80|3776.85|2629.55|4969.65|37.76|0.00|1242.15|3776.85|3814.61|5019.00|5056.76|1147.30| +2450841|71168|2450914|47504|1096808|4512|5816|47504|1096808|4512|5816|4|92|14|1|13136|162|1347|49|4.14|12.33|1.10|550.27|53.90|202.86|604.17|4.31|0.00|96.53|53.90|58.21|150.43|154.74|-148.96| +2450841|71168|2450921|47504|1096808|4512|5816|47504|1096808|4512|5816|4|28|17|3|3314|62|1347|36|39.01|94.01|15.04|2842.92|541.44|1404.36|3384.36|10.82|0.00|1150.56|541.44|552.26|1692.00|1702.82|-862.92| +2450841|71168|2450931|47504|1096808|4512|5816|47504|1096808|4512|5816|4|99|8|3|12172|226|1347|90|51.72|141.19|104.48|3303.90|9403.20|4654.80|12707.10|50.77|8556.91|889.20|846.29|897.06|1735.49|1786.26|-3808.51| +2450841|31008|2450898|82987|144457|1927|36997|82987|144457|1927|36997|2|84|17|1|17053|252|1348|92|4.67|6.91|2.76|381.80|253.92|429.64|635.72|20.11|30.47|69.92|223.45|243.56|293.37|313.48|-206.19| +2450841|31008|2450880|82987|144457|1927|36997|82987|144457|1927|36997|2|33|14|5|670|16|1348|18|63.10|176.68|174.91|31.86|3148.38|1135.80|3180.24|157.41|0.00|222.48|3148.38|3305.79|3370.86|3528.27|2012.58| +2450841|31008|2450874|82987|144457|1927|36997|82987|144457|1927|36997|2|53|15|2|5569|284|1348|12|44.23|46.44|24.61|261.96|295.32|530.76|557.28|0.00|268.74|144.84|26.58|26.58|171.42|171.42|-504.18| +2450841|31008|2450854|82987|144457|1927|36997|82987|144457|1927|36997|2|9|18|5|9086|61|1348|92|44.87|105.44|39.01|6111.56|3588.92|4128.04|9700.48|271.32|574.22|0.00|3014.70|3286.02|3014.70|3286.02|-1113.34| +2450841|69419|2450927|61505|647591|5677|18163|61505|647591|5677|18163|2|51|17|3|397|115|1349|30|54.16|138.10|111.86|787.20|3355.80|1624.80|4143.00|67.11|0.00|786.90|3355.80|3422.91|4142.70|4209.81|1731.00| +2450841|69419|2450856|61505|647591|5677|18163|61505|647591|5677|18163|2|9|1|4|16678|257|1349|27|80.86|141.50|82.07|1604.61|2215.89|2183.22|3820.50|88.63|0.00|190.89|2215.89|2304.52|2406.78|2495.41|32.67| +2450841|69419|2450904|61505|647591|5677|18163|61505|647591|5677|18163|2|82|18|3|5659|208|1349|23|97.77|233.67|16.35|4998.36|376.05|2248.71|5374.41|15.04|0.00|0.00|376.05|391.09|376.05|391.09|-1872.66| +2450841|69419|2450869|61505|647591|5677|18163|61505|647591|5677|18163|2|78|18|3|17569|33|1349|87|62.91|112.60|63.05|4310.85|5485.35|5473.17|9796.20|54.85|0.00|1959.24|5485.35|5540.20|7444.59|7499.44|12.18| +2450841|69419|2450906|61505|647591|5677|18163|61505|647591|5677|18163|2|14|1|2|3334|7|1349|46|5.89|11.42|5.82|257.60|267.72|270.94|525.32|0.00|0.00|262.66|267.72|267.72|530.38|530.38|-3.22| +2450841|69419|2450926|61505|647591|5677|18163|61505|647591|5677|18163|2|63|15|1|15142|67|1349|80|52.99|158.44|7.92|12041.60|633.60|4239.20|12675.20|50.68|0.00|6084.00|633.60|684.28|6717.60|6768.28|-3605.60| +2450841|69419|2450924|61505|647591|5677|18163|61505|647591|5677|18163|2|41|18|4|15652|243|1349|74|26.41|55.98|4.47|3811.74|330.78|1954.34|4142.52|0.00|0.00|1532.54|330.78|330.78|1863.32|1863.32|-1623.56| +2450841|69419|2450888|61505|647591|5677|18163|61505|647591|5677|18163|2|58|19|3|15632|126|1349|68|87.03|103.56|3.10|6831.28|210.80|5918.04|7042.08|2.82|69.56|915.28|141.24|144.06|1056.52|1059.34|-5776.80| +2450841|69419|2450929|61505|647591|5677|18163|61505|647591|5677|18163|2|54|18|5|11806|70|1349|14|29.81|56.63|47.00|134.82|658.00|417.34|792.82|4.47|210.56|174.30|447.44|451.91|621.74|626.21|30.10| +2450841|69419|2450926|61505|647591|5677|18163|61505|647591|5677|18163|2|27|3|1|7495|50|1349|94|10.26|15.90|2.38|1270.88|223.72|964.44|1494.60|2.23|0.00|567.76|223.72|225.95|791.48|793.71|-740.72| +2450841|69419|2450856|61505|647591|5677|18163|61505|647591|5677|18163|2|51|2|1|8248|162|1349|3|53.67|93.38|20.54|218.52|61.62|161.01|280.14|2.46|0.00|56.01|61.62|64.08|117.63|120.09|-99.39| +2450841|69419|2450864|61505|647591|5677|18163|61505|647591|5677|18163|2|73|1|2|11401|172|1349|90|66.72|110.08|37.42|6539.40|3367.80|6004.80|9907.20|269.42|0.00|3070.80|3367.80|3637.22|6438.60|6708.02|-2637.00| +2450841|10981|2450907|3005|6891|3224|31510|3005|6891|3224|31510|1|40|3|2|13078|107|1350|36|70.06|173.74|128.56|1626.48|4628.16|2522.16|6254.64|92.56|0.00|2939.40|4628.16|4720.72|7567.56|7660.12|2106.00| +2450841|10981|2450869|3005|6891|3224|31510|3005|6891|3224|31510|1|67|6|4|8630|42|1350|18|46.89|120.97|108.87|217.80|1959.66|844.02|2177.46|0.00|666.28|457.20|1293.38|1293.38|1750.58|1750.58|449.36| +2450841|10981|2450915|3005|6891|3224|31510|3005|6891|3224|31510|1|54|20|5|9283|227|1350|36|92.40|228.22|193.98|1232.64|6983.28|3326.40|8215.92|0.00|0.00|3614.76|6983.28|6983.28|10598.04|10598.04|3656.88| +2450841|10981|2450880|3005|6891|3224|31510|3005|6891|3224|31510|1|48|4|2|15034|13|1350|45|81.41|195.38|146.53|2198.25|6593.85|3663.45|8792.10|0.00|0.00|3604.50|6593.85|6593.85|10198.35|10198.35|2930.40| +2450841|10981|2450862|3005|6891|3224|31510|3005|6891|3224|31510|1|66|2|2|12661|110|1350|22|21.76|25.67|4.62|463.10|101.64|478.72|564.74|3.04|0.00|90.20|101.64|104.68|191.84|194.88|-377.08| +2450841|10981|2450865|3005|6891|3224|31510|3005|6891|3224|31510|1|41|12|2|17356|20|1350|96|89.76|135.53|16.26|11449.92|1560.96|8616.96|13010.88|12.33|327.80|2211.84|1233.16|1245.49|3445.00|3457.33|-7383.80| +2450841|10981|2450917|3005|6891|3224|31510|3005|6891|3224|31510|1|10|2|2|10741|275|1350|1|58.25|108.34|96.42|11.92|96.42|58.25|108.34|5.78|0.00|1.08|96.42|102.20|97.50|103.28|38.17| +2450841|10981|2450931|3005|6891|3224|31510|3005|6891|3224|31510|1|81|8|5|14236|79|1350|2|14.29|16.86|2.69|28.34|5.38|28.58|33.72|0.29|1.18|16.86|4.20|4.49|21.06|21.35|-24.38| +2450841|10981|2450850|3005|6891|3224|31510|3005|6891|3224|31510|1|90|9|5|1300|19|1350|73|21.94|41.24|34.22|512.46|2498.06|1601.62|3010.52|49.96|0.00|270.83|2498.06|2548.02|2768.89|2818.85|896.44| +2450841|10981|2450852|3005|6891|3224|31510|3005|6891|3224|31510|1|33|11|1|9763|299|1350|47|16.30|17.76|15.27|117.03|717.69|766.10|834.72|7.17|0.00|408.90|717.69|724.86|1126.59|1133.76|-48.41| +2450841|68827|2450898|22347|442043|139|20588|22347|442043|139|20588|2|100|16|4|583|238|1351|77|24.12|27.73|9.42|1409.87|725.34|1857.24|2135.21|39.16|290.13|255.64|435.21|474.37|690.85|730.01|-1422.03| +2450841|68827|2450897|22347|442043|139|20588|22347|442043|139|20588|2|10|10|1|15728|286|1351|85|24.67|31.08|1.55|2510.05|131.75|2096.95|2641.80|2.63|0.00|818.55|131.75|134.38|950.30|952.93|-1965.20| +2450841|68827|2450880|22347|442043|139|20588|22347|442043|139|20588|2|40|1|5|14545|38|1351|23|73.03|182.57|144.23|881.82|3317.29|1679.69|4199.11|0.00|132.69|755.78|3184.60|3184.60|3940.38|3940.38|1504.91| +2450841|68827|2450876|22347|442043|139|20588|22347|442043|139|20588|2|38|9|2|1370|253|1351|28|31.86|78.69|25.18|1498.28|705.04|892.08|2203.32|21.15|0.00|550.76|705.04|726.19|1255.80|1276.95|-187.04| +2450841|68827|2450912|22347|442043|139|20588|22347|442043|139|20588|2|82|7|4|17587|230|1351|39|27.01|39.70|26.99|495.69|1052.61|1053.39|1548.30|23.68|263.15|386.88|789.46|813.14|1176.34|1200.02|-263.93| +2450841|68827|2450848|22347|442043|139|20588|22347|442043|139|20588|2|102|5|5|7165|103|1351|77|71.16|78.98|69.50|729.96|5351.50|5479.32|6081.46|0.00|0.00|120.89|5351.50|5351.50|5472.39|5472.39|-127.82| +2450841|68827|2450851|22347|442043|139|20588|22347|442043|139|20588|2|59|17|2|3394|260|1351|10|42.62|104.41|2.08|1023.30|20.80|426.20|1044.10|1.66|0.00|271.40|20.80|22.46|292.20|293.86|-405.40| +2450841|68827|2450850|22347|442043|139|20588|22347|442043|139|20588|2|55|12|5|4246|158|1351|20|81.32|85.38|63.18|444.00|1263.60|1626.40|1707.60|63.18|0.00|85.20|1263.60|1326.78|1348.80|1411.98|-362.80| +2450841|68827|2450865|22347|442043|139|20588|22347|442043|139|20588|2|72|1|1|5032|116|1351|93|3.39|4.57|3.65|85.56|339.45|315.27|425.01|16.97|0.00|207.39|339.45|356.42|546.84|563.81|24.18| +2450841|68827|2450914|22347|442043|139|20588|22347|442043|139|20588|2|8|11|4|5497|298|1351|67|82.23|135.67|67.83|4545.28|4544.61|5509.41|9089.89|363.56|0.00|3181.16|4544.61|4908.17|7725.77|8089.33|-964.80| +2450841|73974|2450896|45792|227919|1246|34573|45792|227919|1246|34573|1|55|20|4|3968|37|1352|59|32.99|84.12|5.04|4665.72|297.36|1946.41|4963.08|0.00|92.18|2134.03|205.18|205.18|2339.21|2339.21|-1741.23| +2450841|73974|2450894|45792|227919|1246|34573|45792|227919|1246|34573|1|30|17|1|6385|292|1352|67|80.34|161.48|132.41|1947.69|8871.47|5382.78|10819.16|589.06|1508.14|1189.92|7363.33|7952.39|8553.25|9142.31|1980.55| +2450841|73974|2450926|45792|227919|1246|34573|45792|227919|1246|34573|1|72|2|1|7640|243|1352|59|87.78|100.94|97.91|178.77|5776.69|5179.02|5955.46|288.83|0.00|1428.98|5776.69|6065.52|7205.67|7494.50|597.67| +2450841|73974|2450910|45792|227919|1246|34573|45792|227919|1246|34573|1|24|9|2|10333|102|1352|88|41.58|49.48|20.78|2525.60|1828.64|3659.04|4354.24|12.43|585.16|565.84|1243.48|1255.91|1809.32|1821.75|-2415.56| +2450841|73974|2450915|45792|227919|1246|34573|45792|227919|1246|34573|1|69|10|1|1712|166|1352|12|80.04|130.46|74.36|673.20|892.32|960.48|1565.52|71.38|0.00|532.20|892.32|963.70|1424.52|1495.90|-68.16| +2450841|73974|2450887|45792|227919|1246|34573|45792|227919|1246|34573|1|46|3|4|13916|293|1352|91|94.56|148.45|132.12|1486.03|12022.92|8604.96|13508.95|961.83|0.00|1350.44|12022.92|12984.75|13373.36|14335.19|3417.96| +2450841|73974|2450851|45792|227919|1246|34573|45792|227919|1246|34573|1|51|15|1|14764|265|1352|75|78.34|125.34|32.58|6957.00|2443.50|5875.50|9400.50|171.04|0.00|2820.00|2443.50|2614.54|5263.50|5434.54|-3432.00| +2450841|73974|2450863|45792|227919|1246|34573|45792|227919|1246|34573|1|85|18|5|16406|54|1352|77|29.42|87.37|61.15|2018.94|4708.55|2265.34|6727.49|193.05|847.53|3363.36|3861.02|4054.07|7224.38|7417.43|1595.68| +2450841|73974|2450926|45792|227919|1246|34573|45792|227919|1246|34573|1|54|3|1|7568|294|1352|65|22.02|35.45|13.11|1452.10|852.15|1431.30|2304.25|12.78|213.03|898.30|639.12|651.90|1537.42|1550.20|-792.18| +2450841|73974|2450870|45792|227919|1246|34573|45792|227919|1246|34573|1|86|2|2|16891|238|1352|4|68.78|142.37|4.27|552.40|17.08|275.12|569.48|1.36|0.00|39.84|17.08|18.44|56.92|58.28|-258.04| +2450841|78324|2450878|90375|1103421|6514|2407|90375|1103421|6514|2407|1|58|11|1|5654|163|1353|17|28.07|69.33|31.89|636.48|542.13|477.19|1178.61|10.84|0.00|153.17|542.13|552.97|695.30|706.14|64.94| +2450841|78324|2450852|90375|1103421|6514|2407|90375|1103421|6514|2407|1|24|7|3|2618|153|1353|76|73.35|88.75|38.16|3844.84|2900.16|5574.60|6745.00|0.00|0.00|2292.92|2900.16|2900.16|5193.08|5193.08|-2674.44| +2450841|78324|2450889|90375|1103421|6514|2407|90375|1103421|6514|2407|1|90|12|1|6319|104|1353|68|37.76|102.70|7.18|6495.36|488.24|2567.68|6983.60|16.11|219.70|2373.88|268.54|284.65|2642.42|2658.53|-2299.14| +2450841|78324|2450863|90375|1103421|6514|2407|90375|1103421|6514|2407|1|82|19|1|2996|275|1353|59|23.79|30.92|18.86|711.54|1112.74|1403.61|1824.28|11.12|0.00|729.24|1112.74|1123.86|1841.98|1853.10|-290.87| +2450841|78324|2450881|90375|1103421|6514|2407|90375|1103421|6514|2407|1|11|13|2|13153|259|1353|64|90.38|96.70|43.51|3404.16|2784.64|5784.32|6188.80|23.39|2394.79|618.88|389.85|413.24|1008.73|1032.12|-5394.47| +2450841|78324|2450922|90375|1103421|6514|2407|90375|1103421|6514|2407|1|76|2|5|3230|166|1353|6|60.43|71.30|69.16|12.84|414.96|362.58|427.80|33.19|0.00|175.38|414.96|448.15|590.34|623.53|52.38| +2450841|78324|2450898|90375|1103421|6514|2407|90375|1103421|6514|2407|1|92|20|2|12370|203|1353|36|55.12|136.69|23.23|4084.56|836.28|1984.32|4920.84|50.17|0.00|1180.80|836.28|886.45|2017.08|2067.25|-1148.04| +2450841|78324|2450925|90375|1103421|6514|2407|90375|1103421|6514|2407|1|54|10|3|2767|279|1353|11|49.31|71.99|15.83|617.76|174.13|542.41|791.89|15.67|0.00|332.53|174.13|189.80|506.66|522.33|-368.28| +2450841|78324|2450851|90375|1103421|6514|2407|90375|1103421|6514|2407|1|58|19|4|11419|269|1353|96|21.18|48.50|48.01|47.04|4608.96|2033.28|4656.00|138.26|0.00|1442.88|4608.96|4747.22|6051.84|6190.10|2575.68| +2450841|77352|2450922|9018|25626|987|40348|93284|608317|4478|4239|1|58|16|5|14960|108|1354|52|69.70|135.21|135.21|0.00|7030.92|3624.40|7030.92|562.47|0.00|1194.96|7030.92|7593.39|8225.88|8788.35|3406.52| +2450841|77352|2450905|9018|25626|987|40348|93284|608317|4478|4239|1|84|17|4|14530|175|1354|33|72.31|159.80|62.32|3216.84|2056.56|2386.23|5273.40|0.00|0.00|474.54|2056.56|2056.56|2531.10|2531.10|-329.67| +2450841|77352|2450866|9018|25626|987|40348|93284|608317|4478|4239|1|7|19|3|4982|2|1354|72|63.18|102.98|31.92|5116.32|2298.24|4548.96|7414.56|0.00|0.00|592.56|2298.24|2298.24|2890.80|2890.80|-2250.72| +2450841|77352|2450890|9018|25626|987|40348|93284|608317|4478|4239|1|9|11|3|6913|111|1354|97|87.13|219.56|10.97|20233.23|1064.09|8451.61|21297.32|21.28|0.00|851.66|1064.09|1085.37|1915.75|1937.03|-7387.52| +2450841|77352|2450845|9018|25626|987|40348|93284|608317|4478|4239|1|20|2|3|7726|218|1354|28|81.28|146.30|93.63|1474.76|2621.64|2275.84|4096.40|157.29|0.00|614.32|2621.64|2778.93|3235.96|3393.25|345.80| +2450841|||||987|40348|||||||6|5|2758||1354||1.65|||78.08||52.80|||0.00|60.80||68.77|124.48|129.57|10.88| +2450841|19879|2450872|25828|1111165|4502|2639|25828|1111165|4502|2639|1|39|8|2|16364|240|1355|54|53.22|80.89|26.69|2926.80|1441.26|2873.88|4368.06|14.41|0.00|2008.80|1441.26|1455.67|3450.06|3464.47|-1432.62| +2450841|19879|2450869|25828|1111165|4502|2639|25828|1111165|4502|2639|1|103|16|3|17722|57|1355|47|5.65|14.52|9.43|239.23|443.21|265.55|682.44|26.59|0.00|313.49|443.21|469.80|756.70|783.29|177.66| +2450841|19879|2450895|25828|1111165|4502|2639|25828|1111165|4502|2639|1|32|6|3|17647|268|1355|84|24.97|63.92|21.09|3597.72|1771.56|2097.48|5369.28|88.57|0.00|2147.04|1771.56|1860.13|3918.60|4007.17|-325.92| +2450841|19879|2450879|25828|1111165|4502|2639|25828|1111165|4502|2639|1|80|13|5|352|41|1355|7|22.97|44.79|7.61|260.26|53.27|160.79|313.53|2.66|0.00|31.29|53.27|55.93|84.56|87.22|-107.52| +2450841|19879|2450919|25828|1111165|4502|2639|25828|1111165|4502|2639|1|56|5|5|782|95|1355|10|41.21|52.74|36.39|163.50|363.90|412.10|527.40|3.16|47.30|105.40|316.60|319.76|422.00|425.16|-95.50| +2450841|19879|2450887|25828|1111165|4502|2639|25828|1111165|4502|2639|1|3|2|2|11486|159|1355|97|54.02|145.31|139.49|564.54|13530.53|5239.94|14095.07|676.52|0.00|5214.72|13530.53|14207.05|18745.25|19421.77|8290.59| +2450841|19879|2450929|25828|1111165|4502|2639|25828|1111165|4502|2639|1|80|19|4|7123|109|1355|86|37.84|46.54|26.52|1721.72|2280.72|3254.24|4002.44|0.00|0.00|1920.38|2280.72|2280.72|4201.10|4201.10|-973.52| +2450841|19879|2450889|25828|1111165|4502|2639|25828|1111165|4502|2639|1|17|18|1|8888|122|1355|45|71.57|95.18|64.72|1370.70|2912.40|3220.65|4283.10|82.12|1543.57|1927.35|1368.83|1450.95|3296.18|3378.30|-1851.82| +2450841|73416|2450929|79674|956417|2760|1503|79674|956417|2760|1503|4|90|19|2|7094|126|1356|38|39.38|47.64|13.81|1285.54|524.78|1496.44|1810.32|41.98|0.00|543.02|524.78|566.76|1067.80|1109.78|-971.66| +2450841|73416|2450868|79674|956417|2760|1503|79674|956417|2760|1503|4|10|1|4|11980|135|1356|44|74.13|95.62|1.91|4123.24|84.04|3261.72|4207.28|3.96|34.45|84.04|49.59|53.55|133.63|137.59|-3212.13| +2450841|73416|2450876|79674|956417|2760|1503|79674|956417|2760|1503|4|62|8|3|13714|15|1356|43|2.13|5.49|0.00|236.07|0.00|91.59|236.07|0.00|0.00|46.87|0.00|0.00|46.87|46.87|-91.59| +2450841|73416|2450924|79674|956417|2760|1503|79674|956417|2760|1503|4|3|5|3|3973|216|1356|29|60.30|83.21|69.89|386.28|2026.81|1748.70|2413.09|28.78|587.77|361.92|1439.04|1467.82|1800.96|1829.74|-309.66| +2450841|||79674||2760|1503|||2760|1503|4|70|13||13420|142|1356|40||||3194.00|3068.40|3278.80|6262.40|214.78||2880.40|3068.40||5948.80|6163.58|-210.40| +2450841|73416|2450887|79674|956417|2760|1503|79674|956417|2760|1503|4|106|12|2|463|189|1356|4|72.30|192.31|175.00|69.24|700.00|289.20|769.24|28.00|0.00|276.92|700.00|728.00|976.92|1004.92|410.80| +2450841|73416|2450906|79674|956417|2760|1503|79674|956417|2760|1503|4|99|6|1|10894|258|1356|26|62.94|153.57|32.24|3154.58|838.24|1636.44|3992.82|0.00|0.00|1636.96|838.24|838.24|2475.20|2475.20|-798.20| +2450841|73416|2450890|79674|956417|2760|1503|79674|956417|2760|1503|4|4|20|4|6334|107|1356|61|51.54|151.52|145.45|370.27|8872.45|3143.94|9242.72|709.79|0.00|92.11|8872.45|9582.24|8964.56|9674.35|5728.51| +2450841|73416|2450898|79674|956417|2760|1503|79674|956417|2760|1503|4|73|13|1|2623|17|1356|71|97.47|269.01|16.14|17953.77|1145.94|6920.37|19099.71|41.94|446.91|4774.75|699.03|740.97|5473.78|5515.72|-6221.34| +2450841|73416|2450929|79674|956417|2760|1503|79674|956417|2760|1503|4|85|5|3|7826|32|1356|46|11.57|24.64|19.95|215.74|917.70|532.22|1133.44|36.70|0.00|11.04|917.70|954.40|928.74|965.44|385.48| +2450841|73416|2450848|79674|956417|2760|1503|79674|956417|2760|1503|4|48|2|3|3253|2|1356|17|34.35|95.49|19.09|1298.80|324.53|583.95|1623.33|25.96|0.00|210.97|324.53|350.49|535.50|561.46|-259.42| +2450841|73416|2450878|79674|956417|2760|1503|79674|956417|2760|1503|4|50|1|5|974|196|1356|75|14.61|35.64|28.51|534.75|2138.25|1095.75|2673.00|42.76|0.00|400.50|2138.25|2181.01|2538.75|2581.51|1042.50| +2450841|54731|2450883|98488|1652371|5807|24090|98488|1652371|5807|24090|1|104|20|4|7798|132|1357|32|37.40|71.06|10.65|1933.12|340.80|1196.80|2273.92|0.95|245.37|249.92|95.43|96.38|345.35|346.30|-1101.37| +2450841|54731|2450919|98488|1652371|5807|24090|98488|1652371|5807|24090|1|76|8|3|13268|274|1357|62|23.31|27.73|10.81|1049.04|670.22|1445.22|1719.26|33.51|0.00|721.68|670.22|703.73|1391.90|1425.41|-775.00| +2450841|54731|2450905|98488|1652371|5807|24090|98488|1652371|5807|24090|1|51|15|4|13586|8|1357|71|3.77|10.97|6.14|342.93|435.94|267.67|778.87|21.79|0.00|194.54|435.94|457.73|630.48|652.27|168.27| +2450841|54731|2450886|98488|1652371|5807|24090|98488|1652371|5807|24090|1|85|19|4|10369|24|1357|60|51.11|110.39|85.00|1523.40|5100.00|3066.60|6623.40|0.00|0.00|2119.20|5100.00|5100.00|7219.20|7219.20|2033.40| +2450841|54731|2450903|98488|1652371|5807|24090|98488|1652371|5807|24090|1|2|6|3|5518|144|1357|78|78.20|142.32|140.89|111.54|10989.42|6099.60|11100.96|329.68|0.00|4439.76|10989.42|11319.10|15429.18|15758.86|4889.82| +2450841|54731|2450907|98488|1652371|5807|24090|98488|1652371|5807|24090|1|27|15|5|3439|5|1357|27|82.99|169.29|164.21|137.16|4433.67|2240.73|4570.83|399.03|0.00|1325.43|4433.67|4832.70|5759.10|6158.13|2192.94| +2450841|31239|2450859|39066|1689535|804|22073|39066|1689535|804|22073|1|90|8|2|3463|200|1358|91|18.44|24.34|18.74|509.60|1705.34|1678.04|2214.94|102.32|0.00|242.97|1705.34|1807.66|1948.31|2050.63|27.30| +2450841|31239|2450926|39066|1689535|804|22073|39066|1689535|804|22073|1|57|2|2|1900|155|1358|31|24.20|54.93|18.67|1124.06|578.77|750.20|1702.83|0.00|0.00|442.68|578.77|578.77|1021.45|1021.45|-171.43| +2450841|31239|2450868|39066|1689535|804|22073|39066|1689535|804|22073|1|37|11|2|8078|72|1358|32|21.33|40.74|28.92|378.24|925.44|682.56|1303.68|46.27|0.00|443.20|925.44|971.71|1368.64|1414.91|242.88| +2450841|31239|2450912|39066|1689535|804|22073|39066|1689535|804|22073|1|90|12|4|4594|4|1358|32|32.47|84.09|68.95|484.48|2206.40|1039.04|2690.88|0.00|0.00|726.40|2206.40|2206.40|2932.80|2932.80|1167.36| +2450841|31239|2450897|39066|1689535|804|22073|39066|1689535|804|22073|1|55|9|4|17344|293|1358|61|7.11|20.97|16.77|256.20|1022.97|433.71|1279.17|40.91|0.00|229.97|1022.97|1063.88|1252.94|1293.85|589.26| +2450841|31239|2450909|39066|1689535|804|22073|39066|1689535|804|22073|1|75|19|3|13699|183|1358|36|40.92|122.35|13.45|3920.40|484.20|1473.12|4404.60|19.36|0.00|1277.28|484.20|503.56|1761.48|1780.84|-988.92| +2450841|31239|2450872|39066|1689535|804|22073|39066|1689535|804|22073|1|31|11|3|2306|175|1358|97|59.87|167.63|147.51|1951.64|14308.47|5807.39|16260.11|858.50|0.00|6340.89|14308.47|15166.97|20649.36|21507.86|8501.08| +2450841|31239|2450915|39066|1689535|804|22073|39066|1689535|804|22073|1|47|10|2|17971|21|1358|20|99.97|282.91|217.84|1301.40|4356.80|1999.40|5658.20|130.70|0.00|1357.80|4356.80|4487.50|5714.60|5845.30|2357.40| +2450841|31239|2450931|39066|1689535|804|22073|39066|1689535|804|22073|1|44|18|5|1789|281|1358|27|63.88|106.67|24.53|2217.78|662.31|1724.76|2880.09|13.24|0.00|748.71|662.31|675.55|1411.02|1424.26|-1062.45| +2450841|31239|2450927|39066|1689535|804|22073|39066|1689535|804|22073|1|36|12|4|7120|120|1358|27|11.04|28.92|0.00|780.84|0.00|298.08|780.84|0.00|0.00|273.24|0.00|0.00|273.24|273.24|-298.08| +2450841|47211|2450850|38261|1703900|7139|825|38261|1703900|7139|825|1|4|10|1|17122|140|1359|14|45.73|96.49|76.22|283.78|1067.08|640.22|1350.86|16.96|501.52|607.88|565.56|582.52|1173.44|1190.40|-74.66| +2450841|47211|2450850|38261|1703900|7139|825|38261|1703900|7139|825|1|41|20|1|9634|70|1359|14|46.97|53.07|39.80|185.78|557.20|657.58|742.98|27.86|0.00|103.88|557.20|585.06|661.08|688.94|-100.38| +2450841|47211|2450852|38261|1703900|7139|825|38261|1703900|7139|825|1|28|12|3|14335|96|1359|4|38.66|96.26|5.77|361.96|23.08|154.64|385.04|0.92|0.00|73.12|23.08|24.00|96.20|97.12|-131.56| +2450841|47211|2450873|38261|1703900|7139|825|38261|1703900|7139|825|1|92|6|3|13886|184|1359|51|38.47|86.55|9.52|3928.53|485.52|1961.97|4414.05|5.53|301.02|1633.02|184.50|190.03|1817.52|1823.05|-1777.47| +2450841|47211|2450861|38261|1703900|7139|825|38261|1703900|7139|825|1|14|17|1|1846|217|1359|84|72.35|133.12|37.27|8051.40|3130.68|6077.40|11182.08|52.59|1377.49|335.16|1753.19|1805.78|2088.35|2140.94|-4324.21| +2450841|47211|2450894|38261|1703900|7139|825|38261|1703900|7139|825|1|41|11|5|14600|298|1359|56|6.58|19.67|4.32|859.60|241.92|368.48|1101.52|19.35|0.00|109.76|241.92|261.27|351.68|371.03|-126.56| +2450841|47211|2450910|38261|1703900|7139|825|38261|1703900|7139|825|1|32|16|1|8902|238|1359|33|57.12|160.50|16.05|4766.85|529.65|1884.96|5296.50|5.29|0.00|529.65|529.65|534.94|1059.30|1064.59|-1355.31| +2450841|47211|2450843|38261|1703900|7139|825|38261|1703900|7139|825|1|72|5|3|16030|80|1359|34|76.60|193.79|46.50|5007.86|1581.00|2604.40|6588.86|31.62|0.00|592.96|1581.00|1612.62|2173.96|2205.58|-1023.40| +2450841|5794|2450860|24|1697762|1100|2438|24|1697762|1100|2438|1|61|1|4|17614|207|1360|6|10.46|19.24|14.04|31.20|84.24|62.76|115.44|1.46|10.95|25.38|73.29|74.75|98.67|100.13|10.53| +2450841|5794|2450914|24|1697762|1100|2438|24|1697762|1100|2438|1|74|4|1|7417|263|1360|64|3.96|8.31|3.65|298.24|233.60|253.44|531.84|16.81|46.72|244.48|186.88|203.69|431.36|448.17|-66.56| +2450841|5794|2450885|24|1697762|1100|2438|24|1697762|1100|2438|1|26|16|1|9001|9|1360|40|87.36|217.52|150.08|2697.60|6003.20|3494.40|8700.80|264.74|2221.18|4002.00|3782.02|4046.76|7784.02|8048.76|287.62| +2450841|5794|2450844|24|1697762|1100|2438|24|1697762|1100|2438|1|40|18|3|139|11|1360|51|93.95|171.92|68.76|5261.16|3506.76|4791.45|8767.92|140.27|0.00|438.09|3506.76|3647.03|3944.85|4085.12|-1284.69| +2450841|5794|2450861|24|1697762|1100|2438|24|1697762|1100|2438|1|70|15|4|13393|244|1360|93|13.40|30.28|20.28|930.00|1886.04|1246.20|2816.04|9.05|1659.71|929.07|226.33|235.38|1155.40|1164.45|-1019.87| +2450841|5794|2450859|24|1697762|1100|2438|24|1697762|1100|2438|1|40|14|1|5578|187|1360|63|39.51|106.28|91.40|937.44|5758.20|2489.13|6695.64|460.65|0.00|2410.38|5758.20|6218.85|8168.58|8629.23|3269.07| +2450841|5794|2450907|24|1697762|1100|2438|24|1697762|1100|2438|1|98|3|5|11062|45|1360|30|22.88|59.48|11.30|1445.40|339.00|686.40|1784.40|20.34|0.00|392.40|339.00|359.34|731.40|751.74|-347.40| +2450841|5794|2450884|24|1697762|1100|2438|24|1697762|1100|2438|1|40|10|2|15194|84|1360|97|74.66|158.27|128.19|2917.76|12434.43|7242.02|15352.19|746.06|0.00|4451.33|12434.43|13180.49|16885.76|17631.82|5192.41| +2450841|5794|2450881|24|1697762|1100|2438|24|1697762|1100|2438|1|5|6|2|16759|157|1360|22|68.57|155.65|121.40|753.50|2670.80|1508.54|3424.30|160.24|0.00|547.80|2670.80|2831.04|3218.60|3378.84|1162.26| +2450841|5794|2450875|24|1697762|1100|2438|24|1697762|1100|2438|1|9|6|5|15244|84|1360|87|64.60|105.29|53.69|4489.20|4671.03|5620.20|9160.23|277.45|46.71|3663.57|4624.32|4901.77|8287.89|8565.34|-995.88| +2450841|45484|2450916|87978|1163640|1255|18870|87978|1163640|1255|18870|2|54|5|4|13000|131|1361|84|27.22|67.50|66.82|57.12|5612.88|2286.48|5670.00|336.77|0.00|736.68|5612.88|5949.65|6349.56|6686.33|3326.40| +2450841|45484|2450919|87978|1163640|1255|18870|87978|1163640|1255|18870|2|30|11|1|3386|221|1361|49|2.49|6.29|3.83|120.54|187.67|122.01|308.21|5.10|123.86|135.24|63.81|68.91|199.05|204.15|-58.20| +2450841|45484|2450898|87978|1163640|1255|18870|87978|1163640|1255|18870|2|73|12|1|2980|159|1361|43|7.87|18.80|11.46|315.62|492.78|338.41|808.40|29.56|0.00|24.08|492.78|522.34|516.86|546.42|154.37| +2450841|45484|2450865|87978|1163640|1255|18870|87978|1163640|1255|18870|2|106|17|1|14617|294|1361|72|3.78|10.01|1.20|634.32|86.40|272.16|720.72|0.00|0.00|158.40|86.40|86.40|244.80|244.80|-185.76| +2450841|45484|2450872|87978|1163640|1255|18870|87978|1163640|1255|18870|2|66|4|5|3613|281|1361|70|69.27|187.02|108.47|5498.50|7592.90|4848.90|13091.40|455.57|0.00|3272.50|7592.90|8048.47|10865.40|11320.97|2744.00| +2450841|45484|2450924|87978|1163640|1255|18870|87978|1163640|1255|18870|2|18|17|3|13760|32|1361|14|80.47|81.27|75.58|79.66|1058.12|1126.58|1137.78|21.16|0.00|56.84|1058.12|1079.28|1114.96|1136.12|-68.46| +2450841|45484|2450884|87978|1163640|1255|18870|87978|1163640|1255|18870|2|5|15|1|15259|83|1361|62|84.18|213.81|207.39|398.04|12858.18|5219.16|13256.22|1028.65|0.00|5169.56|12858.18|13886.83|18027.74|19056.39|7639.02| +2450841|45484|2450891|87978|1163640|1255|18870|87978|1163640|1255|18870|2|70|12|3|10381|150|1361|11|17.68|50.38|22.67|304.81|249.37|194.48|554.18|17.45|0.00|177.32|249.37|266.82|426.69|444.14|54.89| +2450841|45484|2450922|87978|1163640|1255|18870|87978|1163640|1255|18870|2|98|16|5|17404|91|1361|19|49.24|143.28|118.92|462.84|2259.48|935.56|2722.32|180.75|0.00|1034.36|2259.48|2440.23|3293.84|3474.59|1323.92| +2450841|45484|2450906|87978|1163640|1255|18870|87978|1163640|1255|18870|2|62|2|3|5356|238|1361|46|93.60|141.33|56.53|3900.80|2600.38|4305.60|6501.18|0.00|936.13|1430.14|1664.25|1664.25|3094.39|3094.39|-2641.35| +2450841|45484|2450855|87978|1163640|1255|18870|87978|1163640|1255|18870|2|101|12|3|3637|228|1361|11|31.44|40.87|13.48|301.29|148.28|345.84|449.57|13.34|0.00|76.34|148.28|161.62|224.62|237.96|-197.56| +2450841|45484|2450923|87978|1163640|1255|18870|87978|1163640|1255|18870|2|70|8|2|12064|92|1361|95|37.07|65.61|59.04|624.15|5608.80|3521.65|6232.95|56.08|0.00|2741.70|5608.80|5664.88|8350.50|8406.58|2087.15| +2450841|45484|2450858|87978|1163640|1255|18870|87978|1163640|1255|18870|2|36|15|1|3223|136|1361|54|77.44|116.16|63.88|2823.12|3449.52|4181.76|6272.64|68.99|0.00|2446.20|3449.52|3518.51|5895.72|5964.71|-732.24| +2450841|17116|2450860|93909|212063|5064|4418|93909|212063|5064|4418|4|107|17|3|13735|217|1362|64|51.36|137.64|74.32|4052.48|4756.48|3287.04|8808.96|380.51|0.00|3875.84|4756.48|5136.99|8632.32|9012.83|1469.44| +2450841|17116|2450888|93909|212063|5064|4418|93909|212063|5064|4418|4|90|17|4|15727|145|1362|20|37.02|105.50|18.99|1730.20|379.80|740.40|2110.00|7.59|0.00|801.80|379.80|387.39|1181.60|1189.19|-360.60| +2450841|17116|2450912|93909|212063|5064|4418|93909|212063|5064|4418|4|82|20|3|10831|17|1362|31|42.20|120.27|19.24|3131.93|596.44|1308.20|3728.37|11.92|0.00|745.55|596.44|608.36|1341.99|1353.91|-711.76| +2450841|17116|2450908|93909|212063|5064|4418|93909|212063|5064|4418|4|83|13|4|10868|80|1362|51|87.13|241.35|96.54|7385.31|4923.54|4443.63|12308.85|0.00|0.00|245.82|4923.54|4923.54|5169.36|5169.36|479.91| +2450841|17116|2450929|93909|212063|5064|4418|93909|212063|5064|4418|4|62|9|1|6397|142|1362|35|96.38|185.04|37.00|5181.40|1295.00|3373.30|6476.40|99.06|194.25|3043.60|1100.75|1199.81|4144.35|4243.41|-2272.55| +|17116||||5064||93909|212063|5064||4|86||5|16765|93|1362|10|||||746.20||1203.60|22.38|0.00||746.20||902.60|924.98|62.30| +2450841|17116|2450892|93909|212063|5064|4418|93909|212063|5064|4418|4|91|14|3|451|79|1362|87|70.23|198.04|124.76|6375.36|10854.12|6110.01|17229.48|542.70|0.00|3445.20|10854.12|11396.82|14299.32|14842.02|4744.11| +2450841|17116|2450844|93909|212063|5064|4418|93909|212063|5064|4418|4|53|11|4|10874|193|1362|58|25.70|55.51|31.08|1416.94|1802.64|1490.60|3219.58|18.02|0.00|1545.12|1802.64|1820.66|3347.76|3365.78|312.04| +2450841|29363|2450857|52818|677385|352|44119|52818|677385|352|44119|4|27|18|1|3127|68|1363|16|8.83|22.51|0.00|360.16|0.00|141.28|360.16|0.00|0.00|104.32|0.00|0.00|104.32|104.32|-141.28| +2450841|29363|2450885|52818|677385|352|44119|52818|677385|352|44119|4|64|17|1|850|295|1363|94|21.02|45.61|41.96|343.10|3944.24|1975.88|4287.34|118.32|0.00|770.80|3944.24|4062.56|4715.04|4833.36|1968.36| +2450841|29363|2450885|52818|677385|352|44119|52818|677385|352|44119|4|45|6|2|9494|62|1363|26|65.78|190.76|22.89|4364.62|595.14|1710.28|4959.76|0.89|565.38|644.54|29.76|30.65|674.30|675.19|-1680.52| +2450841|29363|2450922|52818|677385|352|44119|52818|677385|352|44119|4|28|18|4|5125|163|1363|25|85.43|158.89|117.57|1033.00|2939.25|2135.75|3972.25|0.00|0.00|357.50|2939.25|2939.25|3296.75|3296.75|803.50| +2450841|29363|2450888|52818|677385|352|44119|52818|677385|352|44119|4|3|2|4|12602|66|1363|64|33.03|76.95|23.85|3398.40|1526.40|2113.92|4924.80|76.32|0.00|1772.80|1526.40|1602.72|3299.20|3375.52|-587.52| +2450841|29363|||677385|||52818|677385||||14|7||8620||1363||19.23|25.96||||807.66|1090.32|33.56|0.00|||872.72|1024.38|1057.94|31.50| +2450841|29363|2450852|52818|677385|352|44119|52818|677385|352|44119|4|61|11|1|15055|238|1363|90|23.22|61.76|45.08|1501.20|4057.20|2089.80|5558.40|365.14|0.00|1333.80|4057.20|4422.34|5391.00|5756.14|1967.40| +2450841|29363|2450872|52818|677385|352|44119|52818|677385|352|44119|4|40|3|2|1795|137|1363|97|51.95|128.83|38.64|8748.43|3748.08|5039.15|12496.51|262.36|0.00|4623.02|3748.08|4010.44|8371.10|8633.46|-1291.07| +2450841|29363|2450855|52818|677385|352|44119|52818|677385|352|44119|4|99|6|1|16945|13|1363|91|82.25|197.40|80.93|10598.77|7364.63|7484.75|17963.40|662.81|0.00|2873.78|7364.63|8027.44|10238.41|10901.22|-120.12| +2450841|29363|2450887|52818|677385|352|44119|52818|677385|352|44119|4|95|19|5|11551|70|1363|64|1.71|4.97|4.52|28.80|289.28|109.44|318.08|23.14|0.00|0.00|289.28|312.42|289.28|312.42|179.84| +2450841|29363|2450856|52818|677385|352|44119|52818|677385|352|44119|4|90|1|5|15958|291|1363|81|82.40|188.69|32.07|12686.22|2597.67|6674.40|15283.89|207.81|0.00|6877.71|2597.67|2805.48|9475.38|9683.19|-4076.73| +2450841|29363|2450926|52818|677385|352|44119|52818|677385|352|44119|4|83|16|3|15847|257|1363|28|65.24|192.45|17.32|4903.64|484.96|1826.72|5388.60|33.94|0.00|2317.00|484.96|518.90|2801.96|2835.90|-1341.76| +2450841|29363|2450918|52818|677385|352|44119|52818|677385|352|44119|4|59|17|1|712|80|1363|67|22.30|52.40|37.20|1018.40|2492.40|1494.10|3510.80|0.00|0.00|1228.78|2492.40|2492.40|3721.18|3721.18|998.30| +2450841|64980|2450890|70003|813665|1327|14636|70003|813665|1327|14636|4|64|12|5|103|61|1364|91|36.67|103.04|13.39|8158.15|1218.49|3336.97|9376.64|12.18|0.00|468.65|1218.49|1230.67|1687.14|1699.32|-2118.48| +2450841|64980|2450892|70003|813665|1327|14636|70003|813665|1327|14636|4|76|17|1|15212|104|1364|32|73.91|113.82|92.19|692.16|2950.08|2365.12|3642.24|206.50|0.00|801.28|2950.08|3156.58|3751.36|3957.86|584.96| +2450841|64980|2450872|70003|813665|1327|14636|70003|813665|1327|14636|4|103|3|5|13100|242|1364|36|77.07|146.43|57.10|3215.88|2055.60|2774.52|5271.48|102.78|0.00|2635.56|2055.60|2158.38|4691.16|4793.94|-718.92| +2450841|64980|2450883|70003|813665|1327|14636|70003|813665|1327|14636|4|4|19|4|11882|199|1364|86|53.60|77.72|24.09|4612.18|2071.74|4609.60|6683.92|0.00|0.00|2940.34|2071.74|2071.74|5012.08|5012.08|-2537.86| +2450841|64980|2450908|70003|813665|1327|14636|70003|813665|1327|14636|4|92|18|1|5626|119|1364|79|25.05|46.84|10.77|2849.53|850.83|1978.95|3700.36|8.50|0.00|776.57|850.83|859.33|1627.40|1635.90|-1128.12| +2450841|64980|2450882|70003|813665|1327|14636|70003|813665|1327|14636|4|34|3|4|15025|212|1364|60|90.64|219.34|0.00|13160.40|0.00|5438.40|13160.40|0.00|0.00|6053.40|0.00|0.00|6053.40|6053.40|-5438.40| +2450841|64980|2450925|70003|813665|1327|14636|70003|813665|1327|14636|4|33|6|2|17176|209|1364|67|59.91|59.91|4.79|3693.04|320.93|4013.97|4013.97|12.83|0.00|1725.92|320.93|333.76|2046.85|2059.68|-3693.04| +2450841|64980|2450865|70003|813665|1327|14636|70003|813665|1327|14636|4|35|5|1|3919|182|1364|5|37.37|64.27|39.20|125.35|196.00|186.85|321.35|6.17|107.80|86.75|88.20|94.37|174.95|181.12|-98.65| +2450841|64980|2450889|70003|813665|1327|14636|70003|813665|1327|14636|4|48|4|2|1189|29|1364|95|44.65|87.06|35.69|4880.15|3390.55|4241.75|8270.70|305.14|0.00|3638.50|3390.55|3695.69|7029.05|7334.19|-851.20| +2450841|64980|2450861|70003|813665|1327|14636|70003|813665|1327|14636|4|69|18|2|11659|60|1364|76|93.69|203.30|197.20|463.60|14987.20|7120.44|15450.80|314.73|10491.04|1545.08|4496.16|4810.89|6041.24|6355.97|-2624.28| +2450841|64980|2450894|70003|813665|1327|14636|70003|813665|1327|14636|4|94|4|3|8914|249|1364|2|3.87|4.87|3.11|3.52|6.22|7.74|9.74|0.12|0.00|1.16|6.22|6.34|7.38|7.50|-1.52| +2450841|64980|2450876|70003|813665|1327|14636|70003|813665|1327|14636|4|55|8|4|2239|49|1364|65|57.50|143.75|57.50|5606.25|3737.50|3737.50|9343.75|112.12|0.00|3737.50|3737.50|3849.62|7475.00|7587.12|0.00| +2450841|64980|2450843|70003|813665|1327|14636|70003|813665|1327|14636|4|26|7|1|14305|197|1364|95|70.48|119.11|8.33|10524.10|791.35|6695.60|11315.45|39.56|0.00|1470.60|791.35|830.91|2261.95|2301.51|-5904.25| +2450841|54640|2450895|82811|886612|2060|42708|82811|886612|2060|42708|1|106|17|3|16321|208|1365|64|12.92|28.16|19.99|522.88|1279.36|826.88|1802.24|63.96|0.00|504.32|1279.36|1343.32|1783.68|1847.64|452.48| +2450841|54640|2450907|82811|886612|2060|42708|82811|886612|2060|42708|1|20|3|3|9277|51|1365|75|18.14|45.89|2.29|3270.00|171.75|1360.50|3441.75|7.72|85.87|412.50|85.88|93.60|498.38|506.10|-1274.62| +2450841|54640|2450879|82811|886612|2060|42708|82811|886612|2060|42708|1|15|4|5|3992|185|1365|46|74.52|169.16|71.04|4513.52|3267.84|3427.92|7781.36|98.03|0.00|1633.92|3267.84|3365.87|4901.76|4999.79|-160.08| +2450841|54640|2450927|82811|886612|2060|42708|82811|886612|2060|42708|1|5|11|4|11917|80|1365|33|76.12|93.62|74.89|618.09|2471.37|2511.96|3089.46|148.28|0.00|833.91|2471.37|2619.65|3305.28|3453.56|-40.59| +2450841|54640|2450919|82811|886612|2060|42708|82811|886612|2060|42708|1|65|13|4|6316|283|1365|26|63.23|99.90|57.94|1090.96|1506.44|1643.98|2597.40|30.12|0.00|77.74|1506.44|1536.56|1584.18|1614.30|-137.54| +2450841|54640|2450909|82811|886612|2060|42708|82811|886612|2060|42708|1|89|5|5|17023|117|1365|100|14.64|29.57|17.44|1213.00|1744.00|1464.00|2957.00|33.48|906.88|413.00|837.12|870.60|1250.12|1283.60|-626.88| +2450841|54640|||886612|2060|42708|||||1|16||2|388|26|1365||3.16||2.89|||123.24|171.21||0.00||112.71||165.75|171.38|-10.53| +2450841|65675|2450857|84934|1363127|1767|23207|84934|1363127|1767|23207|1|106|13|4|16184|169|1366|43|56.88|85.32|68.25|734.01|2934.75|2445.84|3668.76|117.39|0.00|733.58|2934.75|3052.14|3668.33|3785.72|488.91| +2450841|65675|2450892|84934|1363127|1767|23207|84934|1363127|1767|23207|1|96|17|4|7405|218|1366|26|29.64|58.68|5.86|1373.32|152.36|770.64|1525.68|0.00|0.00|472.94|152.36|152.36|625.30|625.30|-618.28| +2450841|65675|2450846|84934|1363127|1767|23207|84934|1363127|1767|23207|1|4|9|1|13388|57|1366|14|89.61|100.36|50.18|702.52|702.52|1254.54|1405.04|14.05|0.00|505.68|702.52|716.57|1208.20|1222.25|-552.02| +2450841|65675|2450900|84934|1363127|1767|23207|84934|1363127|1767|23207|1|95|8|4|3901|108|1366|25|60.40|81.54|12.23|1732.75|305.75|1510.00|2038.50|3.05|0.00|468.75|305.75|308.80|774.50|777.55|-1204.25| +2450841|65675|2450916|84934|1363127|1767|23207|84934|1363127|1767|23207|1|53|10|2|235|253|1366|12|42.49|112.59|72.05|486.48|864.60|509.88|1351.08|60.52|0.00|310.68|864.60|925.12|1175.28|1235.80|354.72| +2450841|65675|2450851|84934|1363127|1767|23207|84934|1363127|1767|23207|1|3|14|2|5767|188|1366|55|55.88|63.14|1.26|3403.40|69.30|3073.40|3472.70|0.00|0.00|937.20|69.30|69.30|1006.50|1006.50|-3004.10| +2450841|65675|2450848|84934|1363127|1767|23207|84934|1363127|1767|23207|1|94|8|5|4252|240|1366|61|88.51|155.77|14.01|8647.36|854.61|5399.11|9501.97|68.36|0.00|1804.99|854.61|922.97|2659.60|2727.96|-4544.50| +2450841|65675|2450862|84934|1363127|1767|23207|84934|1363127|1767|23207|1|51|14|5|494|194|1366|15|19.33|32.66|27.43|78.45|411.45|289.95|489.90|16.45|0.00|230.25|411.45|427.90|641.70|658.15|121.50| +2450841|65675|2450869|84934|1363127|1767|23207|84934|1363127|1767|23207|1|102|19|4|14264|246|1366|15|44.90|93.84|31.90|929.10|478.50|673.50|1407.60|14.35|0.00|239.25|478.50|492.85|717.75|732.10|-195.00| +2450841|65675|2450872|84934|1363127|1767|23207|84934|1363127|1767|23207|1|77|9|4|6781|267|1366|44|91.04|136.56|53.25|3665.64|2343.00|4005.76|6008.64|76.14|820.05|2883.76|1522.95|1599.09|4406.71|4482.85|-2482.81| +2450841|65675|2450931|84934|1363127|1767|23207|84934|1363127|1767|23207|1|91|2|5|10495|114|1366|87|55.70|86.33|26.76|5182.59|2328.12|4845.90|7510.71|93.12|0.00|2252.43|2328.12|2421.24|4580.55|4673.67|-2517.78| +2450841|65675|2450857|84934|1363127|1767|23207|84934|1363127|1767|23207|1|31|9|2|7168|263|1366|58|41.12|110.61|58.62|3015.42|3399.96|2384.96|6415.38|305.99|0.00|641.48|3399.96|3705.95|4041.44|4347.43|1015.00| +2450841|46094|2450855|23267|119298|5729|4207|23267|119298|5729|4207|4|31|18|3|14929|235|1367|97|34.58|40.11|0.80|3813.07|77.60|3354.26|3890.67|0.77|0.00|38.80|77.60|78.37|116.40|117.17|-3276.66| +2450841|46094|2450865|23267|119298|5729|4207|23267|119298|5729|4207|4|56|6|4|17132|205|1367|58|85.61|116.42|114.09|135.14|6617.22|4965.38|6752.36|66.17|0.00|67.28|6617.22|6683.39|6684.50|6750.67|1651.84| +2450841|46094|2450852|23267|119298|5729|4207|23267|119298|5729|4207|4|56|12|5|8509|9|1367|64|26.30|29.19|11.09|1158.40|709.76|1683.20|1868.16|42.58|0.00|803.20|709.76|752.34|1512.96|1555.54|-973.44| +2450841|46094|2450895|23267|119298|5729|4207|23267|119298|5729|4207|4|71|4|2|11530|69|1367|47|60.98|145.74|32.06|5342.96|1506.82|2866.06|6849.78|15.06|0.00|753.41|1506.82|1521.88|2260.23|2275.29|-1359.24| +2450841|46094|2450851|23267|119298|5729|4207|23267|119298|5729|4207|4|19|4|3|4436|121|1367|59|24.65|25.88|5.17|1221.89|305.03|1454.35|1526.92|8.96|6.10|564.63|298.93|307.89|863.56|872.52|-1155.42| +2450841|46094|2450865|23267|119298|5729|4207|23267|119298|5729|4207|4|1|4|2|17461|78|1367|78|59.58|159.07|92.26|5211.18|7196.28|4647.24|12407.46|0.00|1007.47|1985.10|6188.81|6188.81|8173.91|8173.91|1541.57| +2450841|46094|2450914|23267|119298|5729|4207|23267|119298|5729|4207|4|39|12|2|12946|275|1367|63|93.19|150.03|75.01|4726.26|4725.63|5870.97|9451.89|378.05|0.00|2740.50|4725.63|5103.68|7466.13|7844.18|-1145.34| +2450841|46094|2450923|23267|119298|5729|4207|23267|119298|5729|4207|4|75|18|5|1988|214|1367|72|29.26|33.64|29.93|267.12|2154.96|2106.72|2422.08|129.29|0.00|847.44|2154.96|2284.25|3002.40|3131.69|48.24| +2450841|46094|2450849|23267|119298|5729|4207|23267|119298|5729|4207|4|100|2|2|16280|199|1367|4|20.94|43.55|39.19|17.44|156.76|83.76|174.20|6.27|0.00|48.76|156.76|163.03|205.52|211.79|73.00| +2450841|46094|2450930|23267|119298|5729|4207|23267|119298|5729|4207|4|65|13|3|14803|18|1367|25|42.39|74.18|72.69|37.25|1817.25|1059.75|1854.50|19.62|1163.04|445.00|654.21|673.83|1099.21|1118.83|-405.54| +2450841|46094|2450878|23267|119298|5729|4207|23267|119298|5729|4207|4|46|19|2|3970|195|1367|80|53.68|72.46|59.41|1044.00|4752.80|4294.40|5796.80|142.58|0.00|2028.80|4752.80|4895.38|6781.60|6924.18|458.40| +2450841|46094|2450876|23267|119298|5729|4207|23267|119298|5729|4207|4|48|3|3|6178|135|1367|9|15.86|28.23|12.98|137.25|116.82|142.74|254.07|1.16|0.00|76.14|116.82|117.98|192.96|194.12|-25.92| +2450841|46094|2450897|23267|119298|5729|4207|23267|119298|5729|4207|4|105|15|2|17690|151|1367|99|61.75|120.41|55.38|6437.97|5482.62|6113.25|11920.59|383.78|0.00|2860.11|5482.62|5866.40|8342.73|8726.51|-630.63| +2450841|67412|2450845|36847|1288208|816|22577|36847|1288208|816|22577|4|57|2|5|14588|299|1368|42|66.13|192.43|46.18|6142.50|1939.56|2777.46|8082.06|58.18|0.00|2505.30|1939.56|1997.74|4444.86|4503.04|-837.90| +2450841|67412|2450918|36847|1288208|816|22577|36847|1288208|816|22577|4|94|16|1|7214|284|1368|71|53.58|116.80|102.78|995.42|7297.38|3804.18|8292.80|72.97|0.00|2238.63|7297.38|7370.35|9536.01|9608.98|3493.20| +2450841|67412|2450903|36847|1288208|816|22577|36847|1288208|816|22577|4|99|13|4|17779|299|1368|38|45.54|113.85|104.74|346.18|3980.12|1730.52|4326.30|119.40|0.00|1427.66|3980.12|4099.52|5407.78|5527.18|2249.60| +2450841|67412|2450899|36847|1288208|816|22577|36847|1288208|816|22577|4|17|7|2|12715|194|1368|53|13.54|18.41|9.94|448.91|526.82|717.62|975.73|4.74|368.77|399.62|158.05|162.79|557.67|562.41|-559.57| +2450841|67412|2450920|36847|1288208|816|22577|36847|1288208|816|22577|4|6|8|5|17227|94|1368|15|45.52|114.71|30.97|1256.10|464.55|682.80|1720.65|27.87|0.00|533.40|464.55|492.42|997.95|1025.82|-218.25| +2450841|67412|2450859|36847|1288208|816|22577|36847|1288208|816|22577|4|76|8|4|7024|86|1368|60|83.45|100.14|80.11|1201.80|4806.60|5007.00|6008.40|0.00|1970.70|901.20|2835.90|2835.90|3737.10|3737.10|-2171.10| +2450841|67412|2450868|36847|1288208|816|22577|36847|1288208|816|22577|4|26|12|2|13898|257|1368|52|76.66|122.65|84.62|1977.56|4400.24|3986.32|6377.80|352.01|0.00|2741.96|4400.24|4752.25|7142.20|7494.21|413.92| +2450841|67412|2450862|36847|1288208|816|22577|36847|1288208|816|22577|4|5|17|5|10015|91|1368|22|98.18|131.56|35.52|2112.88|781.44|2159.96|2894.32|31.25|0.00|289.30|781.44|812.69|1070.74|1101.99|-1378.52| +2450841|67412|2450921|36847|1288208|816|22577|36847|1288208|816|22577|4|85|7|4|10162|191|1368|76|31.49|82.50|5.77|5831.48|438.52|2393.24|6270.00|0.00|0.00|1567.12|438.52|438.52|2005.64|2005.64|-1954.72| +2450841|68408|2450866|34014|922967|5873|9626|34014|922967|5873|9626|1|42|9|1|17728|45|1369|46|92.68|126.04|118.47|348.22|5449.62|4263.28|5797.84|435.96|0.00|927.36|5449.62|5885.58|6376.98|6812.94|1186.34| +2450841|68408|2450873|34014|922967|5873|9626|34014|922967|5873|9626|1|28|15|5|3100|82|1369|39|54.05|109.72|24.13|3338.01|941.07|2107.95|4279.08|28.79|461.12|1454.70|479.95|508.74|1934.65|1963.44|-1628.00| +2450841|68408|2450868|34014|922967|5873|9626|34014|922967|5873|9626|1|107|3|4|14306|204|1369|7|13.59|40.22|3.21|259.07|22.47|95.13|281.54|1.79|0.00|84.42|22.47|24.26|106.89|108.68|-72.66| +2450841|68408|2450851|34014|922967|5873|9626|34014|922967|5873|9626|1|104|14|5|15592|118|1369|86|87.69|103.47|72.42|2670.30|6228.12|7541.34|8898.42|344.41|1307.90|2401.98|4920.22|5264.63|7322.20|7666.61|-2621.12| +2450841|68408|2450899|34014|922967|5873|9626|34014|922967|5873|9626|1|72|12|2|6338|148|1369|31|88.75|197.91|77.18|3742.63|2392.58|2751.25|6135.21|0.00|0.00|1349.74|2392.58|2392.58|3742.32|3742.32|-358.67| +2450841|68408|2450895|34014|922967|5873|9626|34014|922967|5873|9626|1|101|15|5|7126|283|1369|74|61.18|68.52|9.59|4360.82|709.66|4527.32|5070.48|28.38|0.00|2128.98|709.66|738.04|2838.64|2867.02|-3817.66| +2450841|68408|2450909|34014|922967|5873|9626|34014|922967|5873|9626|1|9|10|2|12208|246|1369|3|32.00|36.80|3.68|99.36|11.04|96.00|110.40|0.33|0.00|52.98|11.04|11.37|64.02|64.35|-84.96| +2450841|28925|2450891|74030|323322|614|28306|74030|323322|614|28306|1|103|5|1|6877|103|1370|59|83.12|198.65|176.79|1289.74|10430.61|4904.08|11720.35|0.00|9909.07|468.46|521.54|521.54|990.00|990.00|-4382.54| +2450841|28925|2450853|74030|323322|614|28306|74030|323322|614|28306|1|66|13|2|16670|155|1370|72|72.92|153.86|64.62|6425.28|4652.64|5250.24|11077.92|100.96|3210.32|2104.56|1442.32|1543.28|3546.88|3647.84|-3807.92| +2450841|28925|2450845|74030|323322|614|28306|74030|323322|614|28306|1|80|11|4|14716|205|1370|23|20.28|32.04|11.85|464.37|272.55|466.44|736.92|10.90|0.00|221.03|272.55|283.45|493.58|504.48|-193.89| +2450841|28925|2450908|74030|323322|614|28306|74030|323322|614|28306|1|18|4|2|16072|291|1370|32|44.25|63.72|0.63|2018.88|20.16|1416.00|2039.04|0.40|0.00|489.28|20.16|20.56|509.44|509.84|-1395.84| +2450841|28925|2450899|74030|323322|614|28306|74030|323322|614|28306|1|71|4|3|12584|36|1370|81|90.61|125.04|112.53|1013.31|9114.93|7339.41|10128.24|546.89|0.00|3342.06|9114.93|9661.82|12456.99|13003.88|1775.52| +2450841|44923|2450904|48124|1197603|5835|15714|48124|1197603|5835|15714|4|90|18|3|7396|272|1371|35|3.37|7.34|2.93|154.35|102.55|117.95|256.90|3.07|0.00|20.30|102.55|105.62|122.85|125.92|-15.40| +2450841|44923|2450870|48124|1197603|5835|15714|48124|1197603|5835|15714|4|90|8|5|344|72|1371|30|27.04|28.66|0.28|851.40|8.40|811.20|859.80|0.25|0.00|197.70|8.40|8.65|206.10|206.35|-802.80| +2450841|44923|2450871|48124|||||||15714|4|29|6||8947||1371|30||181.58||4303.50|1143.90||5447.40||0.00|2178.90|1143.90||3322.80|3322.80|-1708.20| +2450841|44923|2450858|48124|1197603|5835|15714|48124|1197603|5835|15714|4|97|12|1|9553|151|1371|92|12.64|14.15|12.31|169.28|1132.52|1162.88|1301.80|8.26|305.78|572.24|826.74|835.00|1398.98|1407.24|-336.14| +2450841|44923|2450864|48124|1197603|5835|15714|48124|1197603|5835|15714|4|105|7|5|16036|290|1371|100|97.14|199.13|5.97|19316.00|597.00|9714.00|19913.00|53.73|0.00|9359.00|597.00|650.73|9956.00|10009.73|-9117.00| +2450841|44923|2450923|48124|1197603|5835|15714|48124|1197603|5835|15714|4|76|6|2|14539|81|1371|56|85.15|160.93|51.49|6128.64|2883.44|4768.40|9012.08|144.17|0.00|2252.88|2883.44|3027.61|5136.32|5280.49|-1884.96| +2450841|44923|2450864|48124|1197603|5835|15714|48124|1197603|5835|15714|4|70|12|4|5575|256|1371|53|24.04|42.31|2.53|2108.34|134.09|1274.12|2242.43|8.04|0.00|941.81|134.09|142.13|1075.90|1083.94|-1140.03| +2450841|43450|2450856|4270|1499715|4760|31358|4270|1499715|4760|31358|4|31|7|4|17960|200|1372|8|83.03|204.25|163.40|326.80|1307.20|664.24|1634.00|26.14|0.00|81.68|1307.20|1333.34|1388.88|1415.02|642.96| +2450841|43450|2450887|4270|1499715|4760|31358|4270|1499715|4760|31358|4|37|11|3|1102|197|1372|12|1.43|3.04|0.54|30.00|6.48|17.16|36.48|0.51|0.00|13.80|6.48|6.99|20.28|20.79|-10.68| +2450841|43450|2450917|4270|1499715|4760|31358|4270|1499715|4760|31358|4|13|3|2|2170|85|1372|29|37.96|105.90|69.89|1044.29|2026.81|1100.84|3071.10|182.41|0.00|859.85|2026.81|2209.22|2886.66|3069.07|925.97| +2450841|43450|2450855|4270|1499715|4760|31358|4270|1499715|4760|31358|4|14|1|3|112|15|1372|70|46.59|134.17|100.62|2348.50|7043.40|3261.30|9391.90|202.85|281.73|1690.50|6761.67|6964.52|8452.17|8655.02|3500.37| +2450841|43450|2450850|4270|1499715|4760|31358|4270|1499715|4760|31358|4|10|8|3|4220|165|1372|52|69.54|112.65|16.89|4979.52|878.28|3616.08|5857.80|35.13|0.00|2811.64|878.28|913.41|3689.92|3725.05|-2737.80| +2450841|43450|2450863|4270|1499715|4760|31358|4270|1499715|4760|31358|4|32|11|1|11036|264|1372|79|51.89|154.63|85.04|5497.61|6718.16|4099.31|12215.77|403.08|0.00|3175.80|6718.16|7121.24|9893.96|10297.04|2618.85| +2450841|43450|2450884|4270|1499715|4760|31358|4270|1499715|4760|31358|4|80|3|3|1429|53|1372|100|61.10|133.80|128.44|536.00|12844.00|6110.00|13380.00|642.20|0.00|3478.00|12844.00|13486.20|16322.00|16964.20|6734.00| +2450841|43450|2450910|4270|1499715|4760|31358|4270|1499715|4760|31358|4|48|17|1|8890|249|1372|41|60.66|126.17|34.06|3776.51|1396.46|2487.06|5172.97|0.00|0.00|51.66|1396.46|1396.46|1448.12|1448.12|-1090.60| +2450841|43450|2450846|4270|1499715|4760|31358|4270|1499715|4760|31358|4|25|19|4|5689|42|1372|68|10.61|19.41|13.58|396.44|923.44|721.48|1319.88|0.00|0.00|435.20|923.44|923.44|1358.64|1358.64|201.96| +2450841|43450|2450922|4270|1499715|4760|31358|4270|1499715|4760|31358|4|51|19|5|6896|23|1372|37|89.76|165.15|132.12|1222.11|4888.44|3321.12|6110.55|391.07|0.00|916.49|4888.44|5279.51|5804.93|6196.00|1567.32| +2450841|43450|2450883|4270|1499715|4760|31358|4270|1499715|4760|31358|4|76|3|3|13441|88|1372|79|52.06|106.72|72.56|2698.64|5732.24|4112.74|8430.88|57.32|0.00|4130.91|5732.24|5789.56|9863.15|9920.47|1619.50| +2450841|43450|2450843|4270|1499715|4760|31358|4270|1499715|4760|31358|4|45|14|5|3235|257|1372|14|70.92|99.99|21.99|1092.00|307.86|992.88|1399.86|9.23|0.00|237.86|307.86|317.09|545.72|554.95|-685.02| +2450841|43450|2450873|4270|1499715|4760|31358|4270|1499715|4760|31358|4|52|16|3|110|2|1372|68|91.78|202.83|174.43|1931.20|11861.24|6241.04|13792.44|355.83|0.00|5654.88|11861.24|12217.07|17516.12|17871.95|5620.20| +2450841|55781|2450880|35298|964753|2208|31263|35298|964753|2208|31263|1|97|11|1|16520|161|1373|16|14.71|27.50|24.20|52.80|387.20|235.36|440.00|0.00|0.00|8.80|387.20|387.20|396.00|396.00|151.84| +2450841|55781|2450890|35298|964753|2208|31263|35298|964753|2208|31263|1|61|17|2|1784|114|1373|28|6.42|15.08|14.02|29.68|392.56|179.76|422.24|3.92|0.00|105.56|392.56|396.48|498.12|502.04|212.80| +2450841|55781|2450863|35298|964753|2208|31263|35298|964753|2208|31263|1|22|2|5|15451|3|1373|83|67.95|199.77|25.97|14425.40|2155.51|5639.85|16580.91|58.19|215.55|4145.02|1939.96|1998.15|6084.98|6143.17|-3699.89| +2450841|55781|2450844|35298|964753|2208|31263|35298|964753|2208|31263|1|76|11|5|8300|150|1373|11|38.79|64.00|38.40|281.60|422.40|426.69|704.00|11.65|33.79|49.28|388.61|400.26|437.89|449.54|-38.08| +2450841|55781|2450903|35298|964753|2208|31263|35298|964753|2208|31263|1|104|7|2|1688|141|1373|80|10.52|15.56|13.22|187.20|1057.60|841.60|1244.80|84.60|0.00|86.40|1057.60|1142.20|1144.00|1228.60|216.00| +2450841|55781|2450921|35298|964753|2208|31263|35298|964753|2208|31263|1|4|4|5|8546|160|1373|43|34.91|85.52|41.90|1875.66|1801.70|1501.13|3677.36|54.05|0.00|147.06|1801.70|1855.75|1948.76|2002.81|300.57| +2450841|55781|2450924|35298|964753|2208|31263|35298|964753|2208|31263|1|64|15|2|1660|224|1373|50|53.04|59.93|55.73|210.00|2786.50|2652.00|2996.50|6.68|2619.31|149.50|167.19|173.87|316.69|323.37|-2484.81| +2450841|55781|2450926|35298|964753|2208|31263|35298|964753|2208|31263|1|3|17|2|8125|215|1373|52|42.41|56.82|31.81|1300.52|1654.12|2205.32|2954.64|115.78|0.00|265.72|1654.12|1769.90|1919.84|2035.62|-551.20| +2450841|55781|2450891|35298|964753|2208|31263|35298|964753|2208|31263|1|17|2|3|4897|252|1373|11|3.95|9.79|2.44|80.85|26.84|43.45|107.69|1.07|0.00|32.23|26.84|27.91|59.07|60.14|-16.61| +2450841|55781|2450862|35298|964753|2208|31263|35298|964753|2208|31263|1|64|3|5|3628|194|1373|45|71.76|147.82|130.08|798.30|5853.60|3229.20|6651.90|351.21|0.00|2061.90|5853.60|6204.81|7915.50|8266.71|2624.40| +2450841|42893|2450913|49242|806558|2274|40790|49242|806558|2274|40790|1|78|18|3|16444|287|1374|11|23.81|25.23|4.03|233.20|44.33|261.91|277.53|0.19|25.26|88.77|19.07|19.26|107.84|108.03|-242.84| +2450841|42893|2450916|49242|806558|2274|40790|49242|806558|2274|40790|1|47|17|4|9884|105|1374|62|26.90|50.03|16.50|2078.86|1023.00|1667.80|3101.86|11.66|439.89|403.00|583.11|594.77|986.11|997.77|-1084.69| +2450841|42893|2450902|49242|806558|2274|40790|49242|806558|2274|40790|1|6|8|4|10754|264|1374|22|61.32|181.50|179.68|40.04|3952.96|1349.04|3993.00|237.17|0.00|1118.04|3952.96|4190.13|5071.00|5308.17|2603.92| +2450841|42893|2450930|49242|806558|2274|40790|49242|806558|2274|40790|1|86|12|3|16390|271|1374|38|3.12|7.33|5.71|61.56|216.98|118.56|278.54|17.35|0.00|38.76|216.98|234.33|255.74|273.09|98.42| +2450841|42893|2450854|49242|806558|2274|40790|49242|806558|2274|40790|1|59|14|1|15170|86|1374|10|64.56|163.98|59.03|1049.50|590.30|645.60|1639.80|11.80|0.00|606.70|590.30|602.10|1197.00|1208.80|-55.30| +2450841|42893|2450853|49242|806558|2274|40790|49242|806558|2274|40790|1|96|17|5|17498|125|1374|57|90.22|234.57|206.42|1604.55|11765.94|5142.54|13370.49|470.63|0.00|5882.97|11765.94|12236.57|17648.91|18119.54|6623.40| +2450841|42893|2450873|49242|806558|2274|40790|49242|806558|2274|40790|1|35|15|5|2695|282|1374|26|14.63|39.50|19.35|523.90|503.10|380.38|1027.00|25.15|0.00|318.24|503.10|528.25|821.34|846.49|122.72| +2450841|42893|2450891|49242|806558|2274|40790|49242|806558|2274|40790|1|39|3|1|2473|57|1374|26|9.71|22.23|6.89|398.84|179.14|252.46|577.98|10.74|0.00|74.88|179.14|189.88|254.02|264.76|-73.32| +2450841|42893|2450914|49242|806558|2274|40790|49242|806558|2274|40790|1|23|19|4|4486|254|1374|68|75.67|106.69|75.74|2104.60|5150.32|5145.56|7254.92|360.52|0.00|2393.60|5150.32|5510.84|7543.92|7904.44|4.76| +2450841|33969|2450850|16545|915002|2259|2316|16545|915002|2259|2316|1|105|12|2|8720|253|1375|93|57.60|161.85|66.35|8881.50|6170.55|5356.80|15052.05|48.13|4566.20|4515.15|1604.35|1652.48|6119.50|6167.63|-3752.45| +2450841|33969|2450844|16545|915002|2259|2316|16545|915002|2259|2316|1|101|4|2|1810|170|1375|15|30.90|36.77|34.93|27.60|523.95|463.50|551.55|10.47|0.00|88.20|523.95|534.42|612.15|622.62|60.45| +2450841|33969|2450889|16545|915002|2259|2316|16545|915002|2259|2316|1|60|13|1|12754|243|1375|34|98.16|179.63|159.87|671.84|5435.58|3337.44|6107.42|489.20|0.00|1587.80|5435.58|5924.78|7023.38|7512.58|2098.14| +2450841|33969|2450916|16545|915002|2259|2316|16545|915002|2259|2316|1|72|6|1|2365|206|1375|2|46.64|76.02|35.72|80.60|71.44|93.28|152.04|4.28|0.00|50.16|71.44|75.72|121.60|125.88|-21.84| +|33969|2450921|16545|915002|||16545|915002|2259||1||||166||1375|35|69.07|||4248.65|223.30|||||134.05|223.30|229.99|357.35|364.04|-2194.15| +2450841|33969|2450867|16545|915002|2259|2316|16545|915002|2259|2316|1|20|1|5|8269|8|1375|35|12.48|33.19|11.94|743.75|417.90|436.80|1161.65|12.53|0.00|92.75|417.90|430.43|510.65|523.18|-18.90| +2450841|33969|2450902|16545|915002|2259|2316|16545|915002|2259|2316|1|31|11|4|571|90|1375|28|65.29|90.10|23.42|1867.04|655.76|1828.12|2522.80|59.01|0.00|908.04|655.76|714.77|1563.80|1622.81|-1172.36| +2450841|33969|2450872|16545|915002|2259|2316|16545|915002|2259|2316|1|49|13|4|6220|26|1375|58|50.99|116.25|47.66|3978.22|2764.28|2957.42|6742.50|138.21|0.00|1955.18|2764.28|2902.49|4719.46|4857.67|-193.14| +2450841|55843|2450918|48793|1418006|2419|49985|48793|1418006|2419|49985|4|58|14|5|8263|264|1376|23|18.44|25.63|15.37|235.98|353.51|424.12|589.49|17.67|0.00|29.44|353.51|371.18|382.95|400.62|-70.61| +2450841|55843|2450912|48793|1418006|2419|49985|48793|1418006|2419|49985|4|63|19|1|8074|75|1376|58|40.33|89.53|13.42|4414.38|778.36|2339.14|5192.74|23.35|0.00|1038.20|778.36|801.71|1816.56|1839.91|-1560.78| +2450841|55843|2450871|48793|1418006|2419|49985|48793|1418006|2419|49985|4|11|7|5|5077|68|1376|1|16.45|25.99|22.09|3.90|22.09|16.45|25.99|0.66|0.00|11.95|22.09|22.75|34.04|34.70|5.64| +2450841|55843|2450876|48793|1418006|2419|49985|48793|1418006|2419|49985|4|100|7|1|8047|286|1376|44|29.40|53.80|13.45|1775.40|591.80|1293.60|2367.20|5.91|0.00|757.24|591.80|597.71|1349.04|1354.95|-701.80| +2450841|69782|2450882|25492|244438|5784|2730|25492|244438|5784|2730|4|34|17|1|1315|299|1377|84|58.44|73.63|58.16|1299.48|4885.44|4908.96|6184.92|293.12|0.00|2720.76|4885.44|5178.56|7606.20|7899.32|-23.52| +2450841|69782|2450927|25492|244438|5784|2730|25492|244438|5784|2730|4|45|19|2|17012|168|1377|38|25.41|43.95|6.15|1436.40|233.70|965.58|1670.10|15.05|18.69|283.86|215.01|230.06|498.87|513.92|-750.57| +2450841|69782|2450867|25492|244438|5784|2730|25492|244438|5784|2730|4|81|5|1|1384|160|1377|11|94.65|246.09|95.97|1651.32|1055.67|1041.15|2706.99|21.11|0.00|920.37|1055.67|1076.78|1976.04|1997.15|14.52| +2450841|69782|2450917|25492|244438|5784|2730|25492|244438|5784|2730|4|31|9|5|15028|4|1377|53|80.71|163.03|14.67|7863.08|777.51|4277.63|8640.59|31.10|0.00|1641.41|777.51|808.61|2418.92|2450.02|-3500.12| +2450841|69782|2450889|25492|244438|5784|2730|25492|244438|5784|2730|4|104|9|2|17432|263|1377|31|96.93|174.47|101.19|2271.68|3136.89|3004.83|5408.57|282.32|0.00|2704.13|3136.89|3419.21|5841.02|6123.34|132.06| +2450841|69782|2450852|25492|244438|5784|2730|25492|244438|5784|2730|4|97|3|3|814|298|1377|81|45.68|90.44|34.36|4542.48|2783.16|3700.08|7325.64|111.32|0.00|2710.26|2783.16|2894.48|5493.42|5604.74|-916.92| +2450841|69782|2450927|25492|244438|5784|2730|25492|244438|5784|2730|4|60|3|5|11282|228|1377|56|93.89|200.92|194.89|337.68|10913.84|5257.84|11251.52|0.00|0.00|2812.88|10913.84|10913.84|13726.72|13726.72|5656.00| +2450841|69782|2450844|25492|244438|5784|2730|25492|244438|5784|2730|4|69|20|4|7243|92|1377|47|85.44|170.02|52.70|5514.04|2476.90|4015.68|7990.94|24.76|0.00|2716.60|2476.90|2501.66|5193.50|5218.26|-1538.78| +2450841|69782|2450916|25492|244438|5784|2730|25492|244438|5784|2730|4|16|9|2|4288|148|1377|93|80.29|222.40|40.03|16960.41|3722.79|7466.97|20683.20|63.66|3015.45|4136.64|707.34|771.00|4843.98|4907.64|-6759.63| +2450841|78967|2450907|61645|1270176|307|27238|61645|1270176|307|27238|4|22|17|3|14224|174|1378|1|92.69|155.71|60.72|94.99|60.72|92.69|155.71|5.46|0.00|51.38|60.72|66.18|112.10|117.56|-31.97| +2450841|78967|2450921|61645|1270176|307|27238|61645|1270176|307|27238|4|5|16|2|13556|275|1378|10|28.74|45.12|2.25|428.70|22.50|287.40|451.20|1.35|0.00|153.40|22.50|23.85|175.90|177.25|-264.90| +2450841|78967|2450921|61645|1270176|307|27238|61645|1270176|307|27238|4|26|10|5|12703|69|1378|11|2.03|4.58|4.25|3.63|46.75|22.33|50.38|0.46|0.00|4.95|46.75|47.21|51.70|52.16|24.42| +2450841|78967|2450869|61645|1270176|307|27238|61645|1270176|307|27238|4|70|19|4|15337|6|1378|60|50.48|121.65|74.20|2847.00|4452.00|3028.80|7299.00|267.12|0.00|2262.60|4452.00|4719.12|6714.60|6981.72|1423.20| +2450841|78967|2450894|61645|1270176|307|27238|61645|1270176|307|27238|4|104|11|1|11074|233|1378|94|53.34|84.81|50.88|3189.42|4782.72|5013.96|7972.14|382.61|0.00|1354.54|4782.72|5165.33|6137.26|6519.87|-231.24| +2450841|78967|2450915|61645|1270176|307|27238|61645|1270176|307|27238|4|76|12|4|6560|181|1378|18|79.87|237.21|37.95|3586.68|683.10|1437.66|4269.78|47.81|0.00|853.92|683.10|730.91|1537.02|1584.83|-754.56| +2450841|78967|2450904|61645|1270176|307|27238|61645|1270176|307|27238|4|76|8|2|12841|196|1378|55|56.11|163.84|65.53|5407.05|3604.15|3086.05|9011.20|180.20|0.00|3604.15|3604.15|3784.35|7208.30|7388.50|518.10| +2450841|78967|2450924|61645|1270176|307|27238|61645|1270176|307|27238|4|53|2|5|13255|91|1378|60|42.28|45.23|2.71|2551.20|162.60|2536.80|2713.80|0.00|0.00|1057.80|162.60|162.60|1220.40|1220.40|-2374.20| +2450841|78967|2450872|61645|1270176|307|27238|61645|1270176|307|27238|4|99|3|5|6986|287|1378|16|65.85|88.89|65.77|369.92|1052.32|1053.60|1422.24|17.99|852.37|682.56|199.95|217.94|882.51|900.50|-853.65| +2450841|78967|2450844|61645|1270176|307|27238|61645|1270176|307|27238|4|20|4|4|6710|90|1378|88|73.89|150.73|1.50|13132.24|132.00|6502.32|13264.24|0.63|100.32|6366.80|31.68|32.31|6398.48|6399.11|-6470.64| +2450841|78967|2450870|61645|1270176|307|27238|61645|1270176|307|27238|4|20|10|3|5467|19|1378|38|91.29|217.27|56.49|6109.64|2146.62|3469.02|8256.26|96.59|1073.31|2724.22|1073.31|1169.90|3797.53|3894.12|-2395.71| +2450841|78967|2450902|61645|1270176|307|27238|61645|1270176|307|27238|4|65|18|3|5710|170|1378|99|3.70|6.21|4.71|148.50|466.29|366.30|614.79|9.65|359.04|36.63|107.25|116.90|143.88|153.53|-259.05| +2450841|78967|2450849|61645|1270176|307|27238|61645|1270176|307|27238|4|107|14|3|14906|151|1378|76|2.74|5.23|0.57|354.16|43.32|208.24|397.48|3.46|0.00|198.36|43.32|46.78|241.68|245.14|-164.92| +2450842|71620|2450923|72904|1059263|2775|14287|72904|1059263|2775|14287|2|43|5|3|12139|76|1379|56|98.03|178.41|3.56|9791.60|199.36|5489.68|9990.96|9.96|0.00|2198.00|199.36|209.32|2397.36|2407.32|-5290.32| +2450842|71620|2450856|72904|1059263|2775|14287|72904|1059263|2775|14287|2|70|18|3|10960|85|1379|77|69.42|137.45|50.85|6668.20|3915.45|5345.34|10583.65|0.00|0.00|1586.97|3915.45|3915.45|5502.42|5502.42|-1429.89| +2450842|71620|2450916|72904|1059263|2775|14287|72904|1059263|2775|14287|2|47|15|1|14536|260|1379|79|83.01|219.14|149.01|5540.27|11771.79|6557.79|17312.06|1059.46|0.00|8136.21|11771.79|12831.25|19908.00|20967.46|5214.00| +2450842|71620|2450918|72904|1059263|2775|14287|72904|1059263|2775|14287|2|33|12|1|11197|66|1379|94|2.85|6.55|5.30|117.50|498.20|267.90|615.70|34.87|0.00|288.58|498.20|533.07|786.78|821.65|230.30| +2450842|71620|2450911|72904|1059263|2775|14287|72904|1059263|2775|14287|2|108|2|5|7390|210|1379|40|90.42|113.92|93.41|820.40|3736.40|3616.80|4556.80|224.18|0.00|956.80|3736.40|3960.58|4693.20|4917.38|119.60| +2450842|71620|2450846|72904|1059263|2775|14287|72904|1059263|2775|14287|2|80|15|3|6590|174|1379|55|70.57|100.91|64.58|1998.15|3551.90|3881.35|5550.05|142.07|0.00|443.85|3551.90|3693.97|3995.75|4137.82|-329.45| +2450842|71620|2450862|72904|1059263|2775|14287|72904|1059263|2775|14287|2|52|10|5|10120|53|1379|30|18.44|24.52|8.82|471.00|264.60|553.20|735.60|5.29|0.00|154.20|264.60|269.89|418.80|424.09|-288.60| +2450842|71620|2450925|72904|1059263|2775|14287|72904|1059263|2775|14287|2|80|16|1|5386|182|1379|63|82.03|205.89|88.53|7393.68|5577.39|5167.89|12971.07|105.97|278.86|3372.39|5298.53|5404.50|8670.92|8776.89|130.64| +2450842|71620|2450918|72904|1059263|2775|14287|72904|1059263|2775|14287|2|3|1|5|17068|96|1379|92|32.08|69.93|11.18|5405.00|1028.56|2951.36|6433.56|52.76|442.28|256.68|586.28|639.04|842.96|895.72|-2365.08| +2450842|71620|2450903|72904|1059263|2775|14287|72904|1059263|2775|14287|2|73|17|5|10514|4|1379|75|35.31|54.73|54.18|41.25|4063.50|2648.25|4104.75|0.00|0.00|1190.25|4063.50|4063.50|5253.75|5253.75|1415.25| +2450842|71620|2450921|72904|1059263|2775|14287|72904|1059263|2775|14287|2|88|12|5|6254|279|1379|4|33.66|82.80|56.30|106.00|225.20|134.64|331.20|11.03|4.50|109.28|220.70|231.73|329.98|341.01|86.06| +2450842|71620|2450869|72904|1059263|2775|14287|72904|1059263|2775|14287|2|48|10|3|10876|166|1379|32|68.83|172.07|122.16|1597.12|3909.12|2202.56|5506.24|117.27|0.00|550.40|3909.12|4026.39|4459.52|4576.79|1706.56| +2450842|71620|2450929|72904|1059263|2775|14287|72904|1059263|2775|14287|2|7|7|5|14767|284|1379|49|94.08|103.48|98.30|253.82|4816.70|4609.92|5070.52|127.16|2697.35|2129.54|2119.35|2246.51|4248.89|4376.05|-2490.57| +2450842|40547|2450849|43082|1591714|7193|225|43082|1591714|7193|225|4|9|5|1|8533|136|1380|20|24.98|58.70|27.00|634.00|540.00|499.60|1174.00|0.00|0.00|270.00|540.00|540.00|810.00|810.00|40.40| +2450842|||||7193|||1591714||225|4|75|||8522||1380||95.41||90.98|||||||895.50|1119.06|1208.58|2014.56|2104.08|-312.09| +2450842|40547|2450873|43082|1591714|7193|225|43082|1591714|7193|225|4|39|3|1|4904|297|1380|16|9.92|28.96|4.34|393.92|69.44|158.72|463.36|5.55|0.00|176.00|69.44|74.99|245.44|250.99|-89.28| +2450842|40547|2450913|43082|1591714|7193|225|43082|1591714|7193|225|4|101|18|5|12157|136|1380|31|32.86|58.16|20.93|1154.13|648.83|1018.66|1802.96|45.41|0.00|450.74|648.83|694.24|1099.57|1144.98|-369.83| +2450842|40547|2450865|43082|1591714|7193|225|43082|1591714|7193|225|4|50|5|2|13762|248|1380|67|94.67|246.14|135.37|7421.59|9069.79|6342.89|16491.38|816.28|0.00|3462.56|9069.79|9886.07|12532.35|13348.63|2726.90| +2450842|40547|2450869|43082|1591714|7193|225|43082|1591714|7193|225|4|57|6|1|8785|272|1380|59|49.44|87.01|52.20|2053.79|3079.80|2916.96|5133.59|61.59|0.00|821.28|3079.80|3141.39|3901.08|3962.67|162.84| +2450842|40547|2450900|43082|1591714|7193|225|43082|1591714|7193|225|4|12|4|2|8854|67|1380|93|41.50|66.40|5.97|5619.99|555.21|3859.50|6175.20|33.31|0.00|431.52|555.21|588.52|986.73|1020.04|-3304.29| +2450842|40547|2450873|43082|1591714|7193|225|43082|1591714|7193|225|4|20|7|5|2449|221|1380|61|44.60|65.56|43.92|1320.04|2679.12|2720.60|3999.16|26.79|0.00|1679.33|2679.12|2705.91|4358.45|4385.24|-41.48| +2450842|40547|2450919|43082|1591714|7193|225|43082|1591714|7193|225|4|21|4|3|15058|283|1380|8|5.73|6.13|1.83|34.40|14.64|45.84|49.04|0.58|0.00|10.72|14.64|15.22|25.36|25.94|-31.20| +2450842|40547|2450932|43082|1591714|7193|225|43082|1591714|7193|225|4|61|15|3|16264|229|1380|59|50.50|140.89|49.31|5403.22|2909.29|2979.50|8312.51|116.37|0.00|2493.34|2909.29|3025.66|5402.63|5519.00|-70.21| +2450842|40547|2450852|43082|1591714|7193|225|43082|1591714|7193|225|4|33|10|5|16387|157|1380|92|39.15|81.43|6.51|6892.64|598.92|3601.80|7491.56|11.97|0.00|1197.84|598.92|610.89|1796.76|1808.73|-3002.88| +2450842|40547|2450869|43082|1591714|7193|225|43082|1591714|7193|225|4|43|2|3|15898|268|1380|24|49.35|115.47|114.31|27.84|2743.44|1184.40|2771.28|18.38|905.33|803.52|1838.11|1856.49|2641.63|2660.01|653.71| +2450842|40547|2450888|43082|1591714|7193|225|43082|1591714|7193|225|4|82|10|1|9652|173|1380|79|8.19|23.01|10.58|981.97|835.82|647.01|1817.79|26.41|175.52|345.23|660.30|686.71|1005.53|1031.94|13.29| +2450842|40547|2450896|43082|1591714|7193|225|43082|1591714|7193|225|4|61|8|3|12289|266|1380|12|38.85|50.50|4.04|557.52|48.48|466.20|606.00|3.09|14.05|193.92|34.43|37.52|228.35|231.44|-431.77| +2450842|15783|2450920|57149|1246692|235|8578|57149|1246692|235|8578|4|90|20|2|4339|126|1381|8|73.13|116.27|61.62|437.20|492.96|585.04|930.16|19.71|0.00|381.36|492.96|512.67|874.32|894.03|-92.08| +2450842|15783|2450880|57149|1246692|235|8578|57149|1246692|235|8578|4|44|8|3|15596|25|1381|99|41.20|97.23|54.44|4236.21|5389.56|4078.80|9625.77|0.00|3718.79|3657.06|1670.77|1670.77|5327.83|5327.83|-2408.03| +2450842|15783|2450883|57149|1246692|235|8578|57149|1246692|235|8578|4|74|11|4|2440|182|1381|1|6.16|7.76|1.39|6.37|1.39|6.16|7.76|0.11|0.00|1.39|1.39|1.50|2.78|2.89|-4.77| +2450842|15783|2450922|57149|1246692|235|8578|57149|1246692|235|8578|4|83|20|3|6307|220|1381|8|54.12|114.19|93.63|164.48|749.04|432.96|913.52|59.92|0.00|182.64|749.04|808.96|931.68|991.60|316.08| +2450842|15783|2450847|57149|1246692|235|8578|57149|1246692|235|8578|4|82|1|5|4082|38|1381|10|90.53|100.48|6.02|944.60|60.20|905.30|1004.80|2.38|20.46|422.00|39.74|42.12|461.74|464.12|-865.56| +2450842|15783|2450918|57149|1246692|235|8578|57149|1246692|235|8578|4|34|3|3|9826|216|1381|90|51.70|85.30|70.79|1305.90|6371.10|4653.00|7677.00|0.00|0.00|3454.20|6371.10|6371.10|9825.30|9825.30|1718.10| +2450842|15783|2450917|57149|1246692|235|8578|57149|1246692|235|8578|4|39|16|4|9175|14|1381|11|41.69|88.79|68.36|224.73|751.96|458.59|976.69|30.07|0.00|371.14|751.96|782.03|1123.10|1153.17|293.37| +2450842|15783|2450915|57149|1246692|235|8578|57149|1246692|235|8578|4|37|8|4|386|113|1381|62|30.25|62.61|15.02|2950.58|931.24|1875.50|3881.82|55.87|0.00|0.00|931.24|987.11|931.24|987.11|-944.26| +2450842|15783|2450858|57149|1246692|235|8578|57149|1246692|235|8578|4|66|20|1|1256|265|1381|68|5.23|9.04|4.33|320.28|294.44|355.64|614.72|1.88|247.32|239.36|47.12|49.00|286.48|288.36|-308.52| +2450842|15783|2450906|57149|1246692|235|8578|57149|1246692|235|8578|4|87|6|2|12379|259|1381|27|3.95|9.91|5.05|131.22|136.35|106.65|267.57|0.00|87.26|69.39|49.09|49.09|118.48|118.48|-57.56| +2450842|15783|2450855|57149|1246692|235|8578|57149|1246692|235|8578|4|107|8|2|7186|33|1381|68|28.80|29.08|9.88|1305.60|671.84|1958.40|1977.44|53.74|0.00|592.96|671.84|725.58|1264.80|1318.54|-1286.56| +2450842|41582|2450892|17911|1884108|3458|40808|17911|1884108|3458|40808|1|24|19|3|3577|296|1382|42|42.12|70.34|8.44|2599.80|354.48|1769.04|2954.28|7.08|0.00|1299.48|354.48|361.56|1653.96|1661.04|-1414.56| +2450842|41582|2450863|17911|1884108|3458|40808|17911|1884108|3458|40808|1|105|8|3|14546|168|1382|98|68.02|102.71|53.40|4832.38|5233.20|6665.96|10065.58|119.31|1255.96|1509.20|3977.24|4096.55|5486.44|5605.75|-2688.72| +2450842|41582|2450925|17911|1884108|3458|40808|17911|1884108|3458|40808|1|75|20|1|721|255|1382|6|9.11|12.48|0.74|70.44|4.44|54.66|74.88|0.26|0.00|28.44|4.44|4.70|32.88|33.14|-50.22| +2450842|41582|2450911|17911|1884108|3458|40808|17911|1884108|3458|40808|1|41|8|5|17785|9|1382|99|21.15|56.68|37.40|1908.72|3702.60|2093.85|5611.32|259.18|0.00|2749.23|3702.60|3961.78|6451.83|6711.01|1608.75| +2450842|28854|2450922|53574|1138180|2433|9039|53574|1138180|2433|9039|4|12|1|5|4226|12|1383|4|43.45|118.18|49.63|274.20|198.52|173.80|472.72|15.88|0.00|118.16|198.52|214.40|316.68|332.56|24.72| +2450842|28854|2450904|53574|1138180|2433|9039|53574|1138180|2433|9039|4|99|19|2|7561|107|1383|60|34.39|71.87|40.96|1854.60|2457.60|2063.40|4312.20|0.00|0.00|85.80|2457.60|2457.60|2543.40|2543.40|394.20| +2450842|28854|2450873|53574|1138180|2433|9039|53574|1138180|2433|9039|4|58|17|2|2158|259|1383|35|64.71|185.71|35.28|5265.05|1234.80|2264.85|6499.85|98.78|0.00|324.80|1234.80|1333.58|1559.60|1658.38|-1030.05| +2450842|28854|2450886|53574|1138180|2433|9039|53574|1138180|2433|9039|4|102|7|4|1226|57|1383|89|13.65|27.84|11.13|1487.19|990.57|1214.85|2477.76|39.62|0.00|346.21|990.57|1030.19|1336.78|1376.40|-224.28| +2450842|28854|2450868|53574|1138180|2433|9039|53574|1138180|2433|9039|4|48|6|4|12302|45|1383|99|85.23|105.68|39.10|6591.42|3870.90|8437.77|10462.32|348.38|0.00|1359.27|3870.90|4219.28|5230.17|5578.55|-4566.87| +2450842|28854|2450926|53574|1138180|2433|9039|53574|1138180|2433|9039|4|78|19|1|7100|215|1383|18|20.53|51.73|31.55|363.24|567.90|369.54|931.14|45.43|0.00|307.26|567.90|613.33|875.16|920.59|198.36| +2450842|28854|2450900|53574|1138180|2433|9039|53574|1138180|2433|9039|4|105|1|2|15940|154|1383|98|43.74|101.47|94.36|696.78|9247.28|4286.52|9944.06|554.83|0.00|3777.90|9247.28|9802.11|13025.18|13580.01|4960.76| +2450842|28854|2450860|53574|1138180|2433|9039|53574|1138180|2433|9039|4|46|10|3|6854|186|1383|73|10.00|10.60|10.17|31.39|742.41|730.00|773.80|37.12|0.00|146.73|742.41|779.53|889.14|926.26|12.41| +2450842|28854|2450929|53574|1138180|2433|9039|53574|1138180|2433|9039|4|82|14|4|6074|72|1383|56|19.82|49.35|33.55|884.80|1878.80|1109.92|2763.60|131.51|0.00|1271.20|1878.80|2010.31|3150.00|3281.51|768.88| +2450842|||||2433|9039|||||4||||5443|96|1383|||108.22|60.60|714.30|||1623.30||||||1493.25|1538.70|325.05| +2450842|28854|2450923|53574|1138180|2433|9039|53574|1138180|2433|9039|4|96|15|4|13250|3|1383|94|40.83|53.48|6.95|4373.82|653.30|3838.02|5027.12|26.13|0.00|2513.56|653.30|679.43|3166.86|3192.99|-3184.72| +2450842|28854|2450890|53574|1138180|2433|9039|53574|1138180|2433|9039|4|50|15|4|14510|85|1383|14|45.63|99.47|76.59|320.32|1072.26|638.82|1392.58|10.72|0.00|194.88|1072.26|1082.98|1267.14|1277.86|433.44| +2450842|28854|2450918|53574|1138180|2433|9039|53574|1138180|2433|9039|4|63|3|1|14923|218|1383|49|29.72|58.84|10.00|2393.16|490.00|1456.28|2883.16|44.10|0.00|576.24|490.00|534.10|1066.24|1110.34|-966.28| +2450842|26126|2450931|20978|152448|1871|20941|20978|152448|1871|20941|4|55|2|3|10726|134|1384|63|57.40|96.43|22.17|4678.38|1396.71|3616.20|6075.09|27.93|0.00|2065.14|1396.71|1424.64|3461.85|3489.78|-2219.49| +2450842|26126|2450857|20978|152448|1871|20941|20978|152448|1871|20941|4|93|14|3|5347|147|1384|99|84.34|231.93|211.05|2067.12|20893.95|8349.66|22961.07|1165.88|7939.70|6429.06|12954.25|14120.13|19383.31|20549.19|4604.59| +2450842|26126|2450869|20978|152448|1871|20941|20978|152448|1871|20941|4|11|7|5|10033|202|1384|70|54.01|84.79|22.89|4333.00|1602.30|3780.70|5935.30|144.20|0.00|2076.90|1602.30|1746.50|3679.20|3823.40|-2178.40| +2450842|26126|2450908|20978|152448|1871|20941|20978|152448|1871|20941|4|105|5|3|17524|194|1384|29|68.41|146.39|16.10|3778.41|466.90|1983.89|4245.31|9.33|0.00|976.14|466.90|476.23|1443.04|1452.37|-1516.99| +2450842|26126|2450898|20978|152448|1871|20941|20978|152448|1871|20941|4|2|15|4|4405|11|1384|87|61.16|178.58|87.50|7923.96|7612.50|5320.92|15536.46|380.62|0.00|1552.95|7612.50|7993.12|9165.45|9546.07|2291.58| +2450842|26126|2450870|20978|152448|1871|20941|20978|152448|1871|20941|4|44|17|4|2728|66|1384|61|50.39|119.42|76.42|2623.00|4661.62|3073.79|7284.62|419.54|0.00|145.18|4661.62|5081.16|4806.80|5226.34|1587.83| +2450842|26126|2450847|20978|152448|1871|20941|20978|152448|1871|20941|4|8|11|4|3142|47|1384|17|78.69|129.05|104.53|416.84|1777.01|1337.73|2193.85|106.62|0.00|570.35|1777.01|1883.63|2347.36|2453.98|439.28| +2450842|26126|2450918|20978|152448|1871|20941|20978|152448|1871|20941|4|99|3|1|7832|275|1384|91|90.36|115.66|15.03|9157.33|1367.73|8222.76|10525.06|0.00|0.00|1157.52|1367.73|1367.73|2525.25|2525.25|-6855.03| +2450842|26126|2450867|20978|152448|1871|20941|20978|152448|1871|20941|4|76|18|5|4028|238|1384|42|84.63|160.79|115.76|1891.26|4861.92|3554.46|6753.18|388.95|0.00|3106.32|4861.92|5250.87|7968.24|8357.19|1307.46| +2450842|26126|2450861|20978|152448|1871|20941|20978|152448|1871|20941|4|20|18|4|14269|263|1384|47|39.78|114.56|12.60|4792.12|592.20|1869.66|5384.32|0.00|0.00|1991.86|592.20|592.20|2584.06|2584.06|-1277.46| +2450842|26126|2450888|20978|152448|1871|20941|20978|152448|1871|20941|4|59|4|5|1798|197|1384|99|15.81|32.25|2.58|2937.33|255.42|1565.19|3192.75|22.98|0.00|1372.14|255.42|278.40|1627.56|1650.54|-1309.77| +2450842|26126|2450859|20978|152448|1871|20941|20978|152448|1871|20941|4|56|17|1|2287|66|1384|77|55.32|87.95|37.81|3860.78|2911.37|4259.64|6772.15|203.79|0.00|609.07|2911.37|3115.16|3520.44|3724.23|-1348.27| +2450842|26126|2450851|20978|152448|1871|20941|20978|152448|1871|20941|4|10|12|5|6704|14|1384|53|47.53|119.30|107.37|632.29|5690.61|2519.09|6322.90|56.90|0.00|2086.08|5690.61|5747.51|7776.69|7833.59|3171.52| +2450842|34670|2450883|82031|1394703|5775|32494|82031|1394703|5775|32494|1|95|19|2|184|96|1385|89|14.79|34.75|25.36|835.71|2257.04|1316.31|3092.75|67.71|0.00|1267.36|2257.04|2324.75|3524.40|3592.11|940.73| +2450842|34670|2450932|82031|1394703|5775|32494|82031|1394703|5775|32494|1|22|15|4|16213|54|1385|40|80.00|165.60|105.98|2384.80|4239.20|3200.00|6624.00|42.39|0.00|2318.40|4239.20|4281.59|6557.60|6599.99|1039.20| +2450842|34670|2450902|82031|1394703|5775|32494|82031|1394703|5775|32494|1|29|5|5|4130|34|1385|27|42.61|48.57|11.65|996.84|314.55|1150.47|1311.39|9.43|0.00|65.34|314.55|323.98|379.89|389.32|-835.92| +2450842|34670|2450848|82031|1394703|5775|32494|82031|1394703|5775|32494|1|72|4|3|13735|283|1385|40|80.31|113.23|97.37|634.40|3894.80|3212.40|4529.20|2.33|3816.90|135.60|77.90|80.23|213.50|215.83|-3134.50| +2450842|34670|2450901|82031|1394703|5775|32494|82031|1394703|5775|32494|1|52|17|3|15727|192|1385|29|49.35|115.97|55.66|1748.99|1614.14|1431.15|3363.13|129.13|0.00|1345.02|1614.14|1743.27|2959.16|3088.29|182.99| +2450842|34670|2450884|82031|1394703|5775|32494|82031|1394703|5775|32494|1|77|7|5|10831|134|1385|62|73.67|77.35|18.56|3644.98|1150.72|4567.54|4795.70|34.52|0.00|1966.02|1150.72|1185.24|3116.74|3151.26|-3416.82| +2450842|74128|2450915|26069|1850656|4396|33336|26069|1850656|4396|33336|1|50|9|4|5113|176|1386|95|50.08|100.16|98.15|190.95|9324.25|4757.60|9515.20|93.24|0.00|1616.90|9324.25|9417.49|10941.15|11034.39|4566.65| +2450842|74128|2450870|26069|1850656|4396|33336|26069|1850656|4396|33336|1|5|17|1|5785|95|1386|7|67.69|107.62|37.66|489.72|263.62|473.83|753.34|18.45|0.00|30.10|263.62|282.07|293.72|312.17|-210.21| +2450842|74128|2450880|26069|1850656|4396|33336|26069|1850656|4396|33336|1|106|4|1|5671|67|1386|45|91.02|265.77|132.88|5980.05|5979.60|4095.90|11959.65|179.38|0.00|238.95|5979.60|6158.98|6218.55|6397.93|1883.70| +2450842|74128|2450901|26069|1850656|4396|33336|26069|1850656|4396|33336|1|8|12|1|4393|216|1386|31|95.54|229.29|100.88|3980.71|3127.28|2961.74|7107.99|62.54|0.00|1421.35|3127.28|3189.82|4548.63|4611.17|165.54| +2450842|67375|2450897|45548|1816617|933|17602|45548|1816617|933|17602|2|60|5|3|16705|234|1387|100|85.69|133.67|20.05|11362.00|2005.00|8569.00|13367.00|40.10|0.00|4544.00|2005.00|2045.10|6549.00|6589.10|-6564.00| +2450842||||1816617|||||||2|107|||4970|200|1387||81.20|||5454.64||6658.40|8522.26|||4004.88|3067.62|3282.35|7072.50|7287.23|-3590.78| +2450842|67375|2450853|45548|1816617|933|17602|45548|1816617|933|17602|2|83|16|5|9004|249|1387|1|78.85|106.44|105.37|1.07|105.37|78.85|106.44|6.32|0.00|5.32|105.37|111.69|110.69|117.01|26.52| +2450842|67375|2450849|45548|1816617|933|17602|45548|1816617|933|17602|2|79|5|1|12884|270|1387|97|96.06|234.38|185.16|4774.34|17960.52|9317.82|22734.86|1077.63|0.00|2272.71|17960.52|19038.15|20233.23|21310.86|8642.70| +2450842|67375|2450871|45548|1816617|933|17602|45548|1816617|933|17602|2|5|5|5|14035|184|1387|79|43.26|86.52|51.04|2802.92|4032.16|3417.54|6835.08|362.89|0.00|67.94|4032.16|4395.05|4100.10|4462.99|614.62| +2450842|67375|2450920|45548|1816617|933|17602|45548|1816617|933|17602|2|6|6|5|17792|50|1387|93|20.11|37.40|16.83|1913.01|1565.19|1870.23|3478.20|3.75|1439.97|590.55|125.22|128.97|715.77|719.52|-1745.01| +2450842|67375|2450860|45548|1816617|933|17602|45548|1816617|933|17602|2|59|9|2|14758|102|1387|56|14.47|25.46|13.49|670.32|755.44|810.32|1425.76|30.21|0.00|85.12|755.44|785.65|840.56|870.77|-54.88| +2450842|67375|2450879|45548|1816617|933|17602|45548|1816617|933|17602|2|70|2|4|17389|21|1387|98|85.86|165.70|23.19|13965.98|2272.62|8414.28|16238.60|204.53|0.00|5683.02|2272.62|2477.15|7955.64|8160.17|-6141.66| +2450842|67375|2450882|45548|1816617|933|17602|45548|1816617|933|17602|2|70|18|2|1490|143|1387|90|20.18|41.36|10.34|2791.80|930.60|1816.20|3722.40|27.91|0.00|1674.90|930.60|958.51|2605.50|2633.41|-885.60| +2450842|67375|2450881|45548|1816617|933|17602|45548|1816617|933|17602|2|84|11|1|3410|163|1387|88|47.73|52.02|1.56|4440.48|137.28|4200.24|4577.76|12.35|0.00|2013.44|137.28|149.63|2150.72|2163.07|-4062.96| +2450842|67375|2450908|45548|1816617|933|17602|45548|1816617|933|17602|2|55|8|2|3520|80|1387|17|64.24|116.91|39.74|1311.89|675.58|1092.08|1987.47|40.53|0.00|655.86|675.58|716.11|1331.44|1371.97|-416.50| +2450842|67375|2450888|45548|1816617|933|17602|45548|1816617|933|17602|2|97|8|4|2527|56|1387|6|16.96|19.67|8.85|64.92|53.10|101.76|118.02|0.53|0.00|49.56|53.10|53.63|102.66|103.19|-48.66| +2450842|67375|2450895|45548|1816617|933|17602|45548|1816617|933|17602|2|84|20|1|6346|50|1387|25|14.86|33.43|22.06|284.25|551.50|371.50|835.75|27.57|0.00|91.75|551.50|579.07|643.25|670.82|180.00| +2450842|31243|2450918|25691|44352|4260|10921|25691|44352|4260|10921|4|66|20|2|15259|247|1388|96|11.32|23.88|2.38|2064.00|228.48|1086.72|2292.48|15.99|0.00|984.96|228.48|244.47|1213.44|1229.43|-858.24| +2450842|31243|2450914|25691|44352|4260|10921|25691|44352|4260|10921|4|2|4|3|10381|159|1388|84|24.38|25.11|4.01|1772.40|336.84|2047.92|2109.24|13.47|0.00|168.00|336.84|350.31|504.84|518.31|-1711.08| +2450842|31243|2450900|25691|44352|4260|10921|25691|44352|4260|10921|4|1|3|2|17404|5|1388|62|65.76|145.32|77.01|4235.22|4774.62|4077.12|9009.84|381.96|0.00|990.76|4774.62|5156.58|5765.38|6147.34|697.50| +2450842|31243|2450896|25691|44352|4260|10921|25691|44352|4260|10921|4|99|12|5|5356|108|1388|62|13.48|28.57|8.57|1240.00|531.34|835.76|1771.34|31.88|0.00|602.02|531.34|563.22|1133.36|1165.24|-304.42| +2450842|31243|2450889|25691|44352|4260|10921|25691|44352|4260|10921|4|80|13|4|3637|281|1388|17|56.97|137.29|90.61|793.56|1540.37|968.49|2333.93|61.61|0.00|1026.80|1540.37|1601.98|2567.17|2628.78|571.88| +2450842|31243|2450929|25691|44352|4260|10921|25691|44352|4260|10921|4|92|12|2|12064|169|1388|64|68.19|144.56|37.58|6846.72|2405.12|4364.16|9251.84|192.40|0.00|832.64|2405.12|2597.52|3237.76|3430.16|-1959.04| +2450842|31243|2450853|25691|44352|4260|10921|25691|44352|4260|10921|4|6|3|1|3223|34|1388|92|18.74|46.85|8.90|3491.40|818.80|1724.08|4310.20|5.32|286.58|1982.60|532.22|537.54|2514.82|2520.14|-1191.86| +2450842|31243|2450898|25691|44352|4260|10921|25691|44352|4260|10921|4|98|13|4|14480|99|1388|69|4.25|6.07|1.88|289.11|129.72|293.25|418.83|1.29|0.00|188.37|129.72|131.01|318.09|319.38|-163.53| +2450842|14843|2450898|24107|1479518|6576|47187|24107|1479518|6576|47187|1|97|20|3|3212|213|1389|21|87.78|100.06|45.02|1155.84|945.42|1843.38|2101.26|85.08|0.00|210.00|945.42|1030.50|1155.42|1240.50|-897.96| +2450842|14843|2450904|24107|1479518|6576|47187|24107|1479518|6576|47187|1|74|17|3|6470|202|1389|44|18.47|22.34|21.66|29.92|953.04|812.68|982.96|9.53|0.00|294.80|953.04|962.57|1247.84|1257.37|140.36| +2450842|14843|2450886|24107|1479518|6576|47187|24107|1479518|6576|47187|1|9|2|2|5494|137|1389|39|69.07|180.96|150.19|1200.03|5857.41|2693.73|7057.44|57.40|2987.27|2964.00|2870.14|2927.54|5834.14|5891.54|176.41| +2450842|14843|2450932|24107|1479518|6576|47187|24107|1479518|6576|47187|1|77|6|1|10942|50|1389|59|99.96|166.93|98.48|4038.55|5810.32|5897.64|9848.87|58.10|0.00|2067.95|5810.32|5868.42|7878.27|7936.37|-87.32| +2450842|14843|2450907|24107|1479518|6576|47187|24107|1479518|6576|47187|1|69|12|5|8846|194|1389|40|70.94|143.29|114.63|1146.40|4585.20|2837.60|5731.60|91.70|0.00|974.00|4585.20|4676.90|5559.20|5650.90|1747.60| +2450842|14843|2450889|24107|1479518|6576|47187|24107|1479518|6576|47187|1|52|1|5|3853|250|1389|41|26.47|69.35|51.31|739.64|2103.71|1085.27|2843.35|5.46|1830.22|938.08|273.49|278.95|1211.57|1217.03|-811.78| +2450842|14843|2450891|24107|1479518|6576|47187|24107|1479518|6576|47187|1|20|16|1|8746|188|1389|43|7.34|12.55|4.14|361.63|178.02|315.62|539.65|14.24|0.00|102.34|178.02|192.26|280.36|294.60|-137.60| +2450842|14843|2450902|24107|1479518|6576|47187|24107|1479518|6576|47187|1|105|17|4|1096|258|1389|99|59.24|88.26|52.07|3582.81|5154.93|5864.76|8737.74|257.74|0.00|3931.29|5154.93|5412.67|9086.22|9343.96|-709.83| +2450842|14843|2450872|24107|1479518|6576|47187|24107|1479518|6576|47187|1|11|8|4|10315|256|1389|47|9.22|12.35|10.74|75.67|504.78|433.34|580.45|10.09|0.00|0.00|504.78|514.87|504.78|514.87|71.44| +2450842||2450866|||6576|47187|24107|1479518|6576|47187|1||19||2191||1389|||||2023.68||||0.00|0.00|1556.52|3164.72||4721.24|4721.24|462.40| +2450842|14843|2450905|24107|1479518|6576|47187|24107|1479518|6576|47187|1|91|10|4|14809|227|1389|61|55.68|100.78|52.40|2951.18|3196.40|3396.48|6147.58|223.74|0.00|1536.59|3196.40|3420.14|4732.99|4956.73|-200.08| +2450842|14843|2450888|24107|1479518|6576|47187|24107|1479518|6576|47187|1|17|14|1|5839|101|1389|64|24.28|58.02|0.58|3676.16|37.12|1553.92|3713.28|2.22|0.00|1559.04|37.12|39.34|1596.16|1598.38|-1516.80| +2450842|56005|2450856|34941|921766|591|13816|34941|921766|591|13816|1|36|8|3|2473|103|1390|18|12.27|24.66|18.74|106.56|337.32|220.86|443.88|26.98|0.00|168.66|337.32|364.30|505.98|532.96|116.46| +2450842|56005|2450856|34941|921766|591|13816|34941|921766|591|13816|1|107|11|2|4486|207|1390|62|32.21|74.08|16.29|3582.98|1009.98|1997.02|4592.96|70.69|0.00|1148.24|1009.98|1080.67|2158.22|2228.91|-987.04| +2450842|56005|2450866|34941|921766|591|13816|34941|921766|591|13816|1|86|5|4|10432|293|1390|40|16.93|17.26|5.17|483.60|206.80|677.20|690.40|18.61|0.00|255.20|206.80|225.41|462.00|480.61|-470.40| +2450842|56005|2450857|34941|921766|591|13816|34941|921766|591|13816|1|48|20|4|7537|42|1390|79|93.43|272.81|122.76|11853.95|9698.04|7380.97|21551.99|387.92|0.00|3878.90|9698.04|10085.96|13576.94|13964.86|2317.07| +2450842|56005|2450931|34941|921766|591|13816|34941|921766|591|13816|1|56|1|1|3019|278|1390|48|94.96|241.19|21.70|10535.52|1041.60|4558.08|11577.12|62.49|0.00|347.04|1041.60|1104.09|1388.64|1451.13|-3516.48| +2450842|56005|2450893|34941|921766|591|13816|34941|921766|591|13816|1|37|19|2|7616|263|1390|7|15.88|38.74|20.53|127.47|143.71|111.16|271.18|4.31|0.00|48.79|143.71|148.02|192.50|196.81|32.55| +2450842|56005|2450892|34941|921766|591|13816|34941|921766|591|13816|1|13|14|5|11554|112|1390|95|29.15|48.68|44.29|417.05|4207.55|2769.25|4624.60|42.07|0.00|508.25|4207.55|4249.62|4715.80|4757.87|1438.30| +2450842|56005|2450851|34941|921766|591|13816|34941|921766|591|13816|1|77|14|3|12290|151|1390|29|4.53|10.32|9.49|24.07|275.21|131.37|299.28|11.00|0.00|20.88|275.21|286.21|296.09|307.09|143.84| +2450842|56005|2450898|34941|921766|591|13816|34941|921766|591|13816|1|46|14|1|12512|15|1390|46|92.95|176.60|137.74|1787.56|6336.04|4275.70|8123.60|253.44|0.00|3005.64|6336.04|6589.48|9341.68|9595.12|2060.34| +2450842|57663|2450859|17757|1194690|5223|18156|17757|1194690|5223|18156|4|46|5|5|7801|13|1391|58|50.58|72.83|17.47|3210.88|1013.26|2933.64|4224.14|10.13|0.00|84.10|1013.26|1023.39|1097.36|1107.49|-1920.38| +2450842|57663|2450882|17757|1194690|5223|18156|17757|1194690|5223|18156|4|59|15|4|5954|149|1391|74|21.80|49.26|19.70|2187.44|1457.80|1613.20|3645.24|72.89|0.00|984.20|1457.80|1530.69|2442.00|2514.89|-155.40| +2450842|57663|2450918|17757|1194690|5223|18156|17757|1194690|5223|18156|4|91|4|1|3790|75|1391|39|41.88|68.68|20.60|1875.12|803.40|1633.32|2678.52|48.20|0.00|937.17|803.40|851.60|1740.57|1788.77|-829.92| +2450842|57663|2450910|17757|1194690|5223|18156|17757|1194690|5223|18156|4|77|10|2|11710|44|1391|55|80.84|234.43|65.64|9283.45|3610.20|4446.20|12893.65|216.61|0.00|3352.25|3610.20|3826.81|6962.45|7179.06|-836.00| +2450842|57663|2450889|17757|1194690|5223|18156|17757|1194690|5223|18156|4|43|2|2|12697|184|1391|100|85.80|180.18|16.21|16397.00|1621.00|8580.00|18018.00|97.26|0.00|9009.00|1621.00|1718.26|10630.00|10727.26|-6959.00| +2450842|57663|2450921|17757|1194690|5223|18156|17757|1194690|5223|18156|4|99|1|1|7948|111|1391|51|79.34|101.55|78.19|1191.36|3987.69|4046.34|5179.05|279.13|0.00|1294.38|3987.69|4266.82|5282.07|5561.20|-58.65| +2450842|57663|2450871|17757|1194690|5223|18156|17757|1194690|5223|18156|4|79|17|4|5878|258|1391|82|57.40|129.72|47.99|6701.86|3935.18|4706.80|10637.04|157.40|0.00|1063.54|3935.18|4092.58|4998.72|5156.12|-771.62| +2450842|57663|2450903|17757|1194690|5223|18156|17757|1194690|5223|18156|4|64|4|1|5828|105|1391|89|82.18|147.10|47.07|8902.67|4189.23|7314.02|13091.90|209.46|0.00|261.66|4189.23|4398.69|4450.89|4660.35|-3124.79| +2450842|57663|2450913|17757|1194690|5223|18156|17757|1194690|5223|18156|4|74|10|5|1156|140|1391|3|79.99|211.17|143.59|202.74|430.77|239.97|633.51|34.46|0.00|234.39|430.77|465.23|665.16|699.62|190.80| +2450842|57663|2450919|17757|1194690|5223|18156|17757|1194690|5223|18156|4|71|2|5|10814|52|1391|5|27.79|71.42|56.42|75.00|282.10|138.95|357.10|15.12|93.09|107.10|189.01|204.13|296.11|311.23|50.06| +2450842|43607|2450913|49474|1124985|6896|37207|49474|1124985|6896|37207|4|31|7|5|5390|188|1392|91|84.83|131.48|97.29|3111.29|8853.39|7719.53|11964.68|88.53|0.00|4785.69|8853.39|8941.92|13639.08|13727.61|1133.86| +2450842|43607|2450863|49474|1124985|6896|37207|49474|1124985|6896|37207|4|10|1|3|11236|119|1392|17|40.35|81.91|72.08|167.11|1225.36|685.95|1392.47|73.52|0.00|682.21|1225.36|1298.88|1907.57|1981.09|539.41| +2450842|43607|2450906|49474|1124985|6896|37207|49474|1124985|6896|37207|4|60|14|5|3472|262|1392|35|42.19|62.44|19.35|1508.15|677.25|1476.65|2185.40|27.09|0.00|109.20|677.25|704.34|786.45|813.54|-799.40| +2450842|43607|2450849|49474|1124985|6896|37207|49474|1124985|6896|37207|4|60|15|1|15614|170|1392|76|12.40|26.90|13.98|981.92|1062.48|942.40|2044.40|63.74|0.00|797.24|1062.48|1126.22|1859.72|1923.46|120.08| +2450842|43607|2450924|49474|1124985|6896|37207|49474|1124985|6896|37207|4|75|11|4|2438|175|1392|49|52.00|81.12|8.92|3537.80|437.08|2548.00|3974.88|34.96|0.00|993.72|437.08|472.04|1430.80|1465.76|-2110.92| +2450842|66014|2450871|68896|866536|4976|36216|68896|866536|4976|36216|1|93|2|1|14294|211|1393|10|12.14|32.04|2.24|298.00|22.40|121.40|320.40|0.00|0.00|137.70|22.40|22.40|160.10|160.10|-99.00| +2450842|66014|2450922|68896|866536|4976|36216|68896|866536|4976|36216|1|2|16|2|16756|77|1393|92|86.06|241.82|212.80|2669.84|19577.60|7917.52|22247.44|1566.20|0.00|3559.48|19577.60|21143.80|23137.08|24703.28|11660.08| +2450842|66014|2450872|68896|866536|4976|36216|68896|866536|4976|36216|1|70|1|2|7564|247|1393|63|12.75|22.31|10.03|773.64|631.89|803.25|1405.53|28.75|221.16|505.89|410.73|439.48|916.62|945.37|-392.52| +2450842|66014|2450910|68896|866536|4976|36216|68896|866536|4976|36216|1|1|15|3|6364|12|1393|83|35.98|68.72|17.18|4277.82|1425.94|2986.34|5703.76|114.07|0.00|2623.63|1425.94|1540.01|4049.57|4163.64|-1560.40| +2450842|66014|2450920|68896|866536|4976|36216|68896|866536|4976|36216|1|69|14|3|14998|179|1393|40|10.61|29.60|18.64|438.40|745.60|424.40|1184.00|44.73|0.00|142.00|745.60|790.33|887.60|932.33|321.20| +2450842|66014|2450927|68896|866536|4976|36216|68896|866536|4976|36216|1|82|2|5|15172|228|1393|50|65.75|71.01|55.38|781.50|2769.00|3287.50|3550.50|221.52|0.00|1278.00|2769.00|2990.52|4047.00|4268.52|-518.50| +2450842|66014|2450880|68896|866536|4976|36216|68896|866536|4976|36216|1|40|17|2|17968|185|1393|55|98.23|267.18|173.66|5143.60|9551.30|5402.65|14694.90|765.05|1050.64|7053.20|8500.66|9265.71|15553.86|16318.91|3098.01| +2450842|77973|2450888|52842|826996|6750|44457|52842|826996|6750|44457|1|40|14|2|9463|280|1394|18|10.64|29.26|20.18|163.44|363.24|191.52|526.68|21.79|0.00|73.62|363.24|385.03|436.86|458.65|171.72| +2450842|77973|2450927|52842|826996|6750|44457|52842|826996|6750|44457|1|22|7|4|13904|243|1394|90|95.10|197.80|83.07|10325.70|7476.30|8559.00|17802.00|0.00|0.00|6942.60|7476.30|7476.30|14418.90|14418.90|-1082.70| +2450842|77973|2450879|52842|826996|6750|44457|52842|826996|6750|44457|1|87|20|4|11234|194|1394|32|39.47|49.33|11.83|1200.00|378.56|1263.04|1578.56|18.92|0.00|15.68|378.56|397.48|394.24|413.16|-884.48| +2450842|77973|2450906|52842|826996|6750|44457|52842|826996|6750|44457|1|100|7|3|14710|197|1394|24|66.01|163.04|132.06|743.52|3169.44|1584.24|3912.96|285.24|0.00|273.84|3169.44|3454.68|3443.28|3728.52|1585.20| +2450842|77973|2450920|52842|826996|6750|44457|52842|826996|6750|44457|1|49|11|2|16636|212|1394|88|40.47|110.48|60.76|4375.36|5346.88|3561.36|9722.24|205.85|1229.78|1166.00|4117.10|4322.95|5283.10|5488.95|555.74| +2450842|77973|2450850|52842|826996|6750|44457|52842|826996|6750|44457|1|25|8|5|541|62|1394|49|46.50|53.01|9.54|2130.03|467.46|2278.50|2597.49|18.69|0.00|129.85|467.46|486.15|597.31|616.00|-1811.04| +2450842|77973|2450906|52842|826996|6750|44457|52842|826996|6750|44457|1|55|12|3|17804|3|1394|10|91.22|141.39|104.62|367.70|1046.20|912.20|1413.90|52.31|0.00|650.30|1046.20|1098.51|1696.50|1748.81|134.00| +2450842|77973|2450914|52842|826996|6750|44457|52842|826996|6750|44457|1|97|9|4|17623|154|1394|72|81.95|227.82|220.98|492.48|15910.56|5900.40|16403.04|318.21|0.00|7053.12|15910.56|16228.77|22963.68|23281.89|10010.16| +2450842|77973|2450904|52842|826996|6750|44457|52842|826996|6750|44457|1|85|2|5|7540|193|1394|23|50.81|74.18|70.47|85.33|1620.81|1168.63|1706.14|64.83|0.00|648.14|1620.81|1685.64|2268.95|2333.78|452.18| +2450842|77973|2450908|52842|826996|6750|44457|52842|826996|6750|44457|1|69|11|1|6787|253|1394|14|59.78|125.53|23.85|1423.52|333.90|836.92|1757.42|0.00|0.00|650.16|333.90|333.90|984.06|984.06|-503.02| +2450842|77973|2450875|52842|826996|6750|44457|52842|826996|6750|44457|1|72|15|1|272|264|1394|80|96.67|269.70|78.21|15319.20|6256.80|7733.60|21576.00|375.40|0.00|4746.40|6256.80|6632.20|11003.20|11378.60|-1476.80| +2450842|77973|2450908|52842|826996|6750|44457|52842|826996|6750|44457|1|95|11|2|17228|4|1394|88|38.95|101.65|45.74|4920.08|4025.12|3427.60|8945.20|241.50|0.00|983.84|4025.12|4266.62|5008.96|5250.46|597.52| +2450842|73064|2450900|89447|453833|3466|41852|89447|453833|3466|41852|4|7|10|3|901|291|1395|34|68.18|96.81|62.92|1152.26|2139.28|2318.12|3291.54|171.14|0.00|1415.08|2139.28|2310.42|3554.36|3725.50|-178.84| +2450842|73064|2450889|89447|453833|3466|41852|89447|453833|3466|41852|4|73|12|3|16894|2|1395|18|43.86|107.01|62.06|809.10|1117.08|789.48|1926.18|33.51|0.00|577.80|1117.08|1150.59|1694.88|1728.39|327.60| +2450842|73064|2450897|89447|453833|3466|41852|89447|453833|3466|41852|4|19|8|5|1402|154|1395|60|15.88|19.85|3.77|964.80|226.20|952.80|1191.00|1.49|76.90|523.80|149.30|150.79|673.10|674.59|-803.50| +2450842|73064|2450857|89447|453833|3466|41852|89447|453833|3466|41852|4|44|20|3|10354|120|1395|55|61.85|150.29|66.12|4629.35|3636.60|3401.75|8265.95|181.83|0.00|826.10|3636.60|3818.43|4462.70|4644.53|234.85| +2450842|73064|2450901|89447|453833|3466|41852|89447|453833|3466|41852|4|10|16|4|2716|70|1395|83|52.61|104.16|91.66|1037.50|7607.78|4366.63|8645.28|304.31|0.00|1296.46|7607.78|7912.09|8904.24|9208.55|3241.15| +2450842|73064|2450914|89447|453833|3466|41852|89447|453833|3466|41852|4|1|13|4|6724|131|1395|96|96.02|113.30|6.79|10224.96|651.84|9217.92|10876.80|0.00|0.00|3588.48|651.84|651.84|4240.32|4240.32|-8566.08| +2450842|73064|2450875|89447|453833|3466|41852|89447|453833|3466|41852|4|37|20|4|9044|259|1395|33|74.84|197.57|160.03|1238.82|5280.99|2469.72|6519.81|87.13|2376.44|1238.49|2904.55|2991.68|4143.04|4230.17|434.83| +2450842|73064|2450853|89447|453833|3466|41852|89447|453833|3466|41852|4|100|1|5|1324|33|1395|34|96.10|173.94|24.35|5086.06|827.90|3267.40|5913.96|66.23|0.00|0.00|827.90|894.13|827.90|894.13|-2439.50| +2450842|73064|2450917|89447|453833|3466|41852|89447|453833|3466|41852|4|13|8|1|5110|33|1395|72|9.49|22.87|5.71|1235.52|411.12|683.28|1646.64|28.77|0.00|394.56|411.12|439.89|805.68|834.45|-272.16| +2450842|73064|2450881|89447|453833|3466|41852|89447|453833|3466|41852|4|67|16|3|10351|235|1395|86|43.72|92.24|92.24|0.00|7932.64|3759.92|7932.64|41.24|3807.66|475.58|4124.98|4166.22|4600.56|4641.80|365.06| +2450842|73064|2450855|89447|453833|3466|41852|89447|453833|3466|41852|4|2|16|4|11533|230|1395|93|55.97|119.77|107.79|1114.14|10024.47|5205.21|11138.61|240.58|2004.89|3897.63|8019.58|8260.16|11917.21|12157.79|2814.37| +2450842|73064|2450845|89447|453833|3466|41852|89447|453833|3466|41852|4|31|11|1|14089|1|1395|20|21.67|29.25|15.50|275.00|310.00|433.40|585.00|27.90|0.00|117.00|310.00|337.90|427.00|454.90|-123.40| +2450842|73064|2450889|89447|453833|3466|41852|89447|453833|3466|41852|4|48|7|5|15871|168|1395|70|59.11|124.72|4.98|8381.80|348.60|4137.70|8730.40|5.64|66.23|3317.30|282.37|288.01|3599.67|3605.31|-3855.33| +2450842|73064|2450907|89447|453833|3466|41852|89447|453833|3466|41852|4|97|20|2|391|248|1395|11|60.19|107.13|46.06|671.77|506.66|662.09|1178.43|10.13|0.00|553.85|506.66|516.79|1060.51|1070.64|-155.43| +2450842|24504|2450897|91724|1004775|226|32830|91724|1004775|226|32830|2|36|18|3|410|63|1396|49|62.43|113.62|31.81|4008.69|1558.69|3059.07|5567.38|46.76|0.00|55.37|1558.69|1605.45|1614.06|1660.82|-1500.38| +2450842|24504|2450888|91724|1004775|226|32830|91724|1004775|226|32830|2|44|16|5|9142|50|1396|51|41.65|45.81|21.53|1238.28|1098.03|2124.15|2336.31|32.94|0.00|303.45|1098.03|1130.97|1401.48|1434.42|-1026.12| +2450842|24504|2450844|91724|1004775|226|32830|91724|1004775|226|32830|2|17|13|5|3007|201|1396|25|25.86|58.18|15.70|1062.00|392.50|646.50|1454.50|23.55|0.00|43.50|392.50|416.05|436.00|459.55|-254.00| +2450842|24504|2450914|91724|1004775|226|32830|91724|1004775|226|32830|2|40|15|3|3748|289|1396|39|40.16|100.40|63.25|1448.85|2466.75|1566.24|3915.60|197.34|0.00|1213.68|2466.75|2664.09|3680.43|3877.77|900.51| +2450842|24504|2450919|91724|1004775|226|32830|91724|1004775|226|32830|2|78|10|2|238|263|1396|91|50.31|84.01|5.88|7109.83|535.08|4578.21|7644.91|37.45|0.00|3210.48|535.08|572.53|3745.56|3783.01|-4043.13| +2450842|24504|2450871|91724|1004775|226|32830|91724|1004775|226|32830|2|93|6|2|15853|73|1396|35|44.43|121.73|52.34|2428.65|1831.90|1555.05|4260.55|0.00|1557.11|1959.65|274.79|274.79|2234.44|2234.44|-1280.26| +2450842|24504|2450928|91724|1004775|226|32830|91724|1004775|226|32830|2|63|19|1|8426|245|1396|60|10.95|22.77|6.60|970.20|396.00|657.00|1366.20|23.76|0.00|0.00|396.00|419.76|396.00|419.76|-261.00| +2450842|24504|2450915|91724|1004775|226|32830|91724|1004775|226|32830|2|17|17|5|10945|81|1396|91|32.45|66.52|49.89|1513.33|4539.99|2952.95|6053.32|0.00|0.00|1876.42|4539.99|4539.99|6416.41|6416.41|1587.04| +2450842|24504|2450918|91724|1004775|226|32830|91724|1004775|226|32830|2|100|17|4|12007|26|1396|81|29.91|36.19|8.32|2257.47|673.92|2422.71|2931.39|0.00|0.00|468.99|673.92|673.92|1142.91|1142.91|-1748.79| +2450842|24504|2450860|91724|1004775|226|32830|91724|1004775|226|32830|2|93|5|3|16412|291|1396|40|50.85|79.32|19.03|2411.60|761.20|2034.00|3172.80|19.41|372.98|1332.40|388.22|407.63|1720.62|1740.03|-1645.78| +2450842|24504|2450920|91724|1004775|226|32830|91724|1004775|226|32830|2|81|9|2|2210|223|1396|97|99.10|215.04|38.70|17104.98|3753.90|9612.70|20858.88|37.53|0.00|3753.90|3753.90|3791.43|7507.80|7545.33|-5858.80| +2450842|24504|2450883|91724|1004775|226|32830|91724|1004775|226|32830|2|2|16|1|1880|146|1396|47|39.21|58.42|53.74|219.96|2525.78|1842.87|2745.74|101.03|0.00|411.72|2525.78|2626.81|2937.50|3038.53|682.91| +2450842|37054|2450894|2367|1215323|5526|32482|2367|1215323|5526|32482|2|70|2|3|6967|185|1397|17|20.01|38.41|2.30|613.87|39.10|340.17|652.97|0.34|21.89|254.49|17.21|17.55|271.70|272.04|-322.96| +2450842|37054|2450920|2367|1215323|5526|32482|2367|1215323|5526|32482|2|95|11|1|17294|286|1397|40|84.11|153.92|106.20|1908.80|4248.00|3364.40|6156.80|53.52|2463.84|184.40|1784.16|1837.68|1968.56|2022.08|-1580.24| +2450842|37054|2450918|2367|1215323|5526|32482|2367|1215323|5526|32482|2|46|14|2|3445|243|1397|57|91.79|120.24|32.46|5003.46|1850.22|5232.03|6853.68|18.50|0.00|821.94|1850.22|1868.72|2672.16|2690.66|-3381.81| +2450842|37054|2450904|2367|1215323|5526|32482|2367|1215323|5526|32482|2|37|4|2|10525|120|1397|100|52.39|59.72|12.54|4718.00|1254.00|5239.00|5972.00|37.62|0.00|1672.00|1254.00|1291.62|2926.00|2963.62|-3985.00| +2450842|37054|2450907|2367|1215323|5526|32482|2367|1215323|5526|32482|2|29|1|4|1048|223|1397|78|53.01|69.97|55.97|1092.00|4365.66|4134.78|5457.66|392.90|0.00|53.82|4365.66|4758.56|4419.48|4812.38|230.88| +2450842|37054|2450900|2367|1215323|5526|32482|2367|1215323|5526|32482|2|71|12|3|12691|252|1397|46|10.35|18.94|10.98|366.16|505.08|476.10|871.24|35.35|0.00|217.58|505.08|540.43|722.66|758.01|28.98| +2450842|37054|2450874|2367|1215323|5526|32482|2367|1215323|5526|32482|2|20|4|4|9415|137|1397|92|32.12|48.18|33.72|1330.32|3102.24|2955.04|4432.56|279.20|0.00|176.64|3102.24|3381.44|3278.88|3558.08|147.20| +2450842|65579|2450918|5004|188280|6403|37279|5004|188280|6403|37279|4|92|3|2|428|249|1398|38|57.25|156.86|114.50|1609.68|4351.00|2175.50|5960.68|130.53|0.00|1549.64|4351.00|4481.53|5900.64|6031.17|2175.50| +2450842|65579|2450896|5004|188280|6403|37279|5004|188280|6403|37279|4|52|12|4|14575|83|1398|89|80.89|211.12|147.78|5637.26|13152.42|7199.21|18789.68|263.04|0.00|7327.37|13152.42|13415.46|20479.79|20742.83|5953.21| +2450842|65579|2450893|5004|188280|6403|37279|5004|188280|6403|37279|4|49|5|3|7528|280|1398|18|38.67|84.68|53.34|564.12|960.12|696.06|1524.24|54.72|48.00|716.22|912.12|966.84|1628.34|1683.06|216.06| +2450842|65579|2450925|5004|188280|6403|37279|5004|188280|6403|37279|4|84|13|2|11732|24|1398|36|65.67|160.89|112.62|1737.72|4054.32|2364.12|5792.04|0.00|0.00|1100.16|4054.32|4054.32|5154.48|5154.48|1690.20| +2450842|65579|2450892|5004|188280|6403|37279|5004|188280|6403|37279|4|98|19|4|3374|131|1398|73|2.35|3.14|2.63|37.23|191.99|171.55|229.22|13.43|0.00|43.07|191.99|205.42|235.06|248.49|20.44| +2450842|65579|2450869|5004|188280|6403|37279|5004|188280|6403|37279|4|35|20|3|16886|284|1398|54|15.19|21.26|14.45|367.74|780.30|820.26|1148.04|62.42|0.00|539.46|780.30|842.72|1319.76|1382.18|-39.96| +2450842|65579|2450857|5004|188280|6403|37279|5004|188280|6403|37279|4|9|17|2|16088|256|1398|27|95.02|236.59|80.44|4216.05|2171.88|2565.54|6387.93|21.71|0.00|2171.88|2171.88|2193.59|4343.76|4365.47|-393.66| +2450842|65579|2450850|5004|188280|6403|37279|5004|188280|6403|37279|4|9|20|4|16669|210|1398|30|26.35|32.93|5.92|810.30|177.60|790.50|987.90|15.98|0.00|296.10|177.60|193.58|473.70|489.68|-612.90| +2450842|65579|2450909|5004|188280|6403|37279|5004|188280|6403|37279|4|64|6|4|12638|95|1398|39|12.05|27.71|21.89|226.98|853.71|469.95|1080.69|8.53|0.00|97.11|853.71|862.24|950.82|959.35|383.76| +2450842|77290|2450916|14964|862374|5431|43241|14964|862374|5431|43241|1|45|7|3|304|187|1399|57|9.97|23.52|1.41|1260.27|80.37|568.29|1340.64|2.41|0.00|442.32|80.37|82.78|522.69|525.10|-487.92| +2450842|77290|2450892|14964|862374|5431|43241|14964|862374|5431|43241|1|21|16|2|1768|2|1399|21|73.64|180.41|10.82|3561.39|227.22|1546.44|3788.61|9.08|0.00|1174.32|227.22|236.30|1401.54|1410.62|-1319.22| +2450842|77290|2450864|14964|862374|5431|43241|14964|862374|5431|43241|1|46|3|1|4864|94|1399|98|47.80|59.75|43.61|1581.72|4273.78|4684.40|5855.50|76.07|470.11|819.28|3803.67|3879.74|4622.95|4699.02|-880.73| +2450842|77290|2450854|14964|862374|5431|43241|14964|862374|5431|43241|1|86|11|2|6584|147|1399|9|90.17|111.81|101.74|90.63|915.66|811.53|1006.29|54.93|0.00|412.56|915.66|970.59|1328.22|1383.15|104.13| +2450842|77290|2450932|14964|862374|5431|43241|14964|862374|5431|43241|1|88|13|2|15400|98|1399|42|25.97|35.83|9.67|1098.72|406.14|1090.74|1504.86|20.22|69.04|120.12|337.10|357.32|457.22|477.44|-753.64| +2450842|77290|2450856|14964|862374|5431|43241|14964|862374|5431|43241|1|98|16|3|16|192|1399|27|8.60|15.22|14.91|8.37|402.57|232.20|410.94|20.12|0.00|164.16|402.57|422.69|566.73|586.85|170.37| +2450842|77290|2450904|14964|862374|5431|43241|14964|862374|5431|43241|1|48|4|5|17666|79|1399|74|49.18|83.11|55.68|2029.82|4120.32|3639.32|6150.14|247.21|0.00|2705.44|4120.32|4367.53|6825.76|7072.97|481.00| +2450842|77290|2450925|14964|862374|5431|43241|14964|862374|5431|43241|1|95|20|4|17401|50|1399|70|39.47|50.52|18.69|2228.10|1308.30|2762.90|3536.40|65.93|209.32|1272.60|1098.98|1164.91|2371.58|2437.51|-1663.92| +2450842|77290|2450851|14964|862374|5431|43241|14964|862374|5431|43241|1|101|12|2|16267|142|1399|36|33.87|55.54|22.21|1199.88|799.56|1219.32|1999.44|55.96|0.00|579.60|799.56|855.52|1379.16|1435.12|-419.76| +2450842|77290|2450858|14964|862374|5431|43241|14964|862374|5431|43241|1|30|1|2|11059|254|1399|4|82.36|120.24|18.03|408.84|72.12|329.44|480.96|2.88|0.00|96.16|72.12|75.00|168.28|171.16|-257.32| +2450842|77290|2450907|14964|862374|5431|43241|14964|862374|5431|43241|1|79|20|5|5894|39|1399|82|84.55|116.67|51.33|5357.88|4209.06|6933.10|9566.94|210.45|0.00|2583.00|4209.06|4419.51|6792.06|7002.51|-2724.04| +2450842|77290|2450862|14964|862374|5431|43241|14964|862374|5431|43241|1|108|20|1|3808|15|1399|62|43.55|114.10|101.54|778.72|6295.48|2700.10|7074.20|0.00|0.00|3537.10|6295.48|6295.48|9832.58|9832.58|3595.38| +||2450862|1373||6548|41712|1373|||41712|||16|5|3949|240|1400||||42.42|||80.78|163.17|||||44.54|110.95|113.07|-38.36| +2450842|47018|2450851|1373|1043737|6548|41712|1373|1043737|6548|41712|1|79|11|4|3673|129|1400|51|49.06|132.46|100.66|1621.80|5133.66|2502.06|6755.46|102.67|0.00|2837.13|5133.66|5236.33|7970.79|8073.46|2631.60| +2450842|47018|2450859|1373|1043737|6548|41712|1373|1043737|6548|41712|1|74|13|4|1196|61|1400|92|61.76|95.72|92.84|264.96|8541.28|5681.92|8806.24|228.05|939.54|439.76|7601.74|7829.79|8041.50|8269.55|1919.82| +2450842|47018|2450849|1373|1043737|6548|41712|1373|1043737|6548|41712|1|105|18|1|5336|149|1400|51|5.35|12.84|0.51|628.83|26.01|272.85|654.84|1.30|0.00|32.64|26.01|27.31|58.65|59.95|-246.84| +2450842|8790|2450871|24178|949588|1751|43561|24178|949588|1751|43561|2|9|8|2|17779|80|1401|46|93.79|264.48|163.97|4623.46|7542.62|4314.34|12166.08|527.98|0.00|729.56|7542.62|8070.60|8272.18|8800.16|3228.28| +2450842|8790|2450885|24178|949588|1751|43561|24178|949588|1751|43561|2|21|12|1|12715|251|1401|59|49.80|104.08|80.14|1412.46|4728.26|2938.20|6140.72|0.00|1938.58|306.80|2789.68|2789.68|3096.48|3096.48|-148.52| +2450842|8790|2450881|24178|949588|1751|43561|24178|949588|1751|43561|2|69|18|3|17227|275|1401|63|89.76|265.68|252.39|837.27|15900.57|5654.88|16737.84|1431.05|0.00|669.06|15900.57|17331.62|16569.63|18000.68|10245.69| +2450842|8790|2450860|24178|949588|1751|43561|24178|949588|1751|43561|2|1|18|3|7024|276|1401|9|80.05|236.14|129.87|956.43|1168.83|720.45|2125.26|105.19|0.00|21.24|1168.83|1274.02|1190.07|1295.26|448.38| +2450842|8790|2450852|24178|949588|1751|43561|24178|949588|1751|43561|2|93|3|4|13898|138|1401|84|40.01|52.01|46.28|481.32|3887.52|3360.84|4368.84|0.00|0.00|1616.16|3887.52|3887.52|5503.68|5503.68|526.68| +2450842|8790|2450906|24178|949588|1751|43561|24178|949588|1751|43561|2|108|2|1|10015|23|1401|43|69.73|203.61|91.62|4815.57|3939.66|2998.39|8755.23|42.54|1812.24|3939.66|2127.42|2169.96|6067.08|6109.62|-870.97| +2450842|8790|2450879|24178|949588|1751|43561|24178|949588|1751|43561|2|37|8|4|10162|4|1401|74|32.95|52.39|51.34|77.70|3799.16|2438.30|3876.86|0.00|0.00|930.18|3799.16|3799.16|4729.34|4729.34|1360.86| +2450842|62177|2450886|80068|1185280|4092|14104|80068|1185280|4092|14104|1|13|14|2|14974|191|1402|39|81.79|160.30|89.76|2751.06|3500.64|3189.81|6251.70|280.05|0.00|62.40|3500.64|3780.69|3563.04|3843.09|310.83| +2450842|62177|2450868|80068|1185280|4092|14104|80068|1185280|4092|14104|1|106|4|1|8167|217|1402|50|58.86|165.98|46.47|5975.50|2323.50|2943.00|8299.00|69.70|0.00|1659.50|2323.50|2393.20|3983.00|4052.70|-619.50| +2450842|62177|2450845|80068|1185280|4092|14104|80068|1185280|4092|14104|1|2|7|5|8036|76|1402|88|65.84|110.61|97.33|1168.64|8565.04|5793.92|9733.68|428.25|0.00|3990.80|8565.04|8993.29|12555.84|12984.09|2771.12| +2450842|62177|2450900|80068|1185280|4092|14104|80068|1185280|4092|14104|1|41|10|5|15158|78|1402|98|7.28|21.54|10.55|1077.02|1033.90|713.44|2110.92|57.69|72.37|231.28|961.53|1019.22|1192.81|1250.50|248.09| +2450842|62177|2450930|80068|1185280|4092|14104|80068|1185280|4092|14104|1|13|10|1|15967|153|1402|74|75.14|172.07|5.16|12351.34|381.84|5560.36|12733.18|6.14|294.01|6238.94|87.83|93.97|6326.77|6332.91|-5472.53| +2450842|62177|2450919|80068|1185280|4092|14104|80068|1185280|4092|14104|1|104|14|5|14462|152|1402|93|78.25|80.59|78.17|225.06|7269.81|7277.25|7494.87|26.17|6979.01|524.52|290.80|316.97|815.32|841.49|-6986.45| +2450842|62177|2450900|80068|1185280|4092|14104|80068|1185280|4092|14104|1|5|18|1|15376|37|1402|16|45.30|72.93|56.88|256.80|910.08|724.80|1166.88|81.90|0.00|256.64|910.08|991.98|1166.72|1248.62|185.28| +2450842|62177|2450854|80068|1185280|4092|14104|80068|1185280|4092|14104|1|18|6|1|4891|171|1402|40|33.93|69.55|2.08|2698.80|83.20|1357.20|2782.00|2.49|0.00|1390.80|83.20|85.69|1474.00|1476.49|-1274.00| +2450842|62177|2450915|80068|1185280|4092|14104|80068|1185280|4092|14104|1|30|1|4|12100|185|1402|44|49.49|123.72|49.48|3266.56|2177.12|2177.56|5443.68|65.31|0.00|870.76|2177.12|2242.43|3047.88|3113.19|-0.44| +2450842|32440|2450900|19449|1693495|7188|46081|19449|1693495|7188|46081|4|101|4|5|15970|37|1403|80|75.61|123.24|96.12|2169.60|7689.60|6048.80|9859.20|461.37|0.00|196.80|7689.60|8150.97|7886.40|8347.77|1640.80| +2450842|32440|2450916|19449|1693495|7188|46081|19449|1693495|7188|46081|4|9|2|5|13579|11|1403|73|22.28|57.48|51.15|462.09|3733.95|1626.44|4196.04|261.37|0.00|1678.27|3733.95|3995.32|5412.22|5673.59|2107.51| +2450842|32440|2450853|19449|1693495|7188|46081|19449|1693495|7188|46081|4|82|18|4|8498|13|1403|54|27.41|51.80|19.68|1734.48|1062.72|1480.14|2797.20|95.64|0.00|475.20|1062.72|1158.36|1537.92|1633.56|-417.42| +2450842|32440|2450895|19449|1693495|7188|46081|19449|1693495|7188|46081|4|66|7|2|12790|259|1403|10|39.70|71.06|51.16|199.00|511.60|397.00|710.60|19.18|127.90|14.20|383.70|402.88|397.90|417.08|-13.30| +2450842|32440|2450920|19449|1693495|7188|46081|19449|1693495|7188|46081|4|28|17|5|15464|207|1403|15|55.52|112.70|77.76|524.10|1166.40|832.80|1690.50|58.32|0.00|760.65|1166.40|1224.72|1927.05|1985.37|333.60| +2450842|32440|2450915|19449|1693495|7188|46081|19449|1693495|7188|46081|4|96|19|1|6412|187|1403|38|90.30|255.54|245.31|388.74|9321.78|3431.40|9710.52|0.00|2982.96|291.08|6338.82|6338.82|6629.90|6629.90|2907.42| +2450842|32440|2450883|19449|1693495|7188|46081|19449|1693495|7188|46081|4|88|19|2|4870|216|1403|15|6.48|10.56|0.42|152.10|6.30|97.20|158.40|0.24|1.38|23.70|4.92|5.16|28.62|28.86|-92.28| +2450842|32440|2450911|19449|1693495|7188|46081|19449|1693495|7188|46081|4|20|20|1|6142|121|1403|66|54.21|75.89|65.26|701.58|4307.16|3577.86|5008.74|258.42|0.00|250.14|4307.16|4565.58|4557.30|4815.72|729.30| +2450842|32440|2450868|19449|1693495|7188|46081|19449|1693495|7188|46081|4|28|6|4|5056|60|1403|21|90.48|165.57|145.70|417.27|3059.70|1900.08|3476.97|0.00|0.00|1494.99|3059.70|3059.70|4554.69|4554.69|1159.62| +2450842|32440|2450879|19449|1693495|7188|46081|19449|1693495|7188|46081|4|18|1|1|4750|115|1403|45|7.19|8.69|5.47|144.90|246.15|323.55|391.05|2.46|0.00|117.00|246.15|248.61|363.15|365.61|-77.40| +2450842|38994|2450844|7154|363733|4312|46138|7154|363733|4312|46138|2|46|17|1|8539|64|1404|14|5.81|8.36|3.51|67.90|49.14|81.34|117.04|0.00|0.00|54.88|49.14|49.14|104.02|104.02|-32.20| +2450842|38994|2450887|7154|363733|4312|46138|7154|363733|4312|46138|2|64|18|3|7610|168|1404|98|48.56|91.77|78.00|1349.46|7644.00|4758.88|8993.46|0.00|6191.64|359.66|1452.36|1452.36|1812.02|1812.02|-3306.52| +2450842|38994|2450865|7154|363733|4312|46138|7154|363733|4312|46138|2|80|6|4|11708|227|1404|74|77.13|131.89|89.68|3123.54|6636.32|5707.62|9759.86|13.27|6304.50|1463.72|331.82|345.09|1795.54|1808.81|-5375.80| +2450842|38994|2450918|7154|363733|4312|46138|7154|363733|4312|46138|2|99|19|2|2737|13|1404|7|44.00|121.00|6.05|804.65|42.35|308.00|847.00|0.00|0.00|110.11|42.35|42.35|152.46|152.46|-265.65| +2450842|48085|2450929|62172|841650|1128|10606|54392|1725162|5887|42165|1|62|4|1|14833|25|1405|61|83.91|183.76|60.64|7510.32|3699.04|5118.51|11209.36|221.94|0.00|2129.51|3699.04|3920.98|5828.55|6050.49|-1419.47| +2450842|48085|2450885|62172|841650|1128|10606|54392|1725162|5887|42165|1|95|9|4|16724|164|1405|13|99.36|143.07|120.17|297.70|1562.21|1291.68|1859.91|62.48|0.00|650.91|1562.21|1624.69|2213.12|2275.60|270.53| +2450842|48085|2450870|62172|841650|1128|10606|54392|1725162|5887|42165|1|35|8|2|15799|146|1405|28|87.35|180.81|90.40|2531.48|2531.20|2445.80|5062.68|202.49|0.00|1771.84|2531.20|2733.69|4303.04|4505.53|85.40| +2450842|48085|2450883|62172|841650|1128|10606|54392|1725162|5887|42165|1|59|18|4|2294|298|1405|2|14.40|16.27|2.44|27.66|4.88|28.80|32.54|0.00|0.00|5.20|4.88|4.88|10.08|10.08|-23.92| +2450842|48085|2450863|62172|841650|1128|10606|54392|1725162|5887|42165|1|99|8|1|3955|14|1405|97|50.23|97.94|6.85|8835.73|664.45|4872.31|9500.18|59.80|0.00|1044.69|664.45|724.25|1709.14|1768.94|-4207.86| +2450842|48085|2450932|62172|841650|1128|10606|54392|1725162|5887|42165|1|1|8|1|12974|171|1405|15|58.00|98.60|25.63|1094.55|384.45|870.00|1479.00|3.84|0.00|44.25|384.45|388.29|428.70|432.54|-485.55| +2450842|48085|2450854|62172|841650|1128|10606|54392|1725162|5887|42165|1|8|19|2|7520|26|1405|20|31.86|92.07|42.35|994.40|847.00|637.20|1841.40|4.48|398.09|349.80|448.91|453.39|798.71|803.19|-188.29| +2450842|48085|2450886|62172|841650|1128|10606|54392|1725162|5887|42165|1|11|3|3|7064|191|1405|28|58.49|161.43|71.02|2531.48|1988.56|1637.72|4520.04|19.88|0.00|1265.60|1988.56|2008.44|3254.16|3274.04|350.84| +2450842|48085|2450845|62172|841650|1128|10606|54392|1725162|5887|42165|1|104|8|4|4306|88|1405|97|62.46|68.70|21.29|4598.77|2065.13|6058.62|6663.90|98.50|970.61|932.17|1094.52|1193.02|2026.69|2125.19|-4964.10| +2450842|48085|2450903|62172|841650|1128|10606|54392|1725162|5887|42165|1|89|17|2|2578|30|1405|21|99.35|232.47|227.82|97.65|4784.22|2086.35|4881.87|430.57|0.00|1317.96|4784.22|5214.79|6102.18|6532.75|2697.87| +2450842|48085|2450893|62172|841650|1128|10606|54392|1725162|5887|42165|1|52|20|4|2072|193|1405|11|13.69|21.90|21.68|2.42|238.48|150.59|240.90|2.38|0.00|40.92|238.48|240.86|279.40|281.78|87.89| +2450842|59163|2450925|28149|620952|3355|44205|28149|620952|3355|44205|2|89|14|5|10118|266|1406|67|13.78|18.87|11.88|468.33|795.96|923.26|1264.29|7.48|421.85|176.88|374.11|381.59|550.99|558.47|-549.15| +2450842|59163|2450884|28149|620952|3355|44205|28149|620952|3355|44205|2|52|11|4|6850|70|1406|96|15.36|28.41|26.42|191.04|2536.32|1474.56|2727.36|50.72|0.00|136.32|2536.32|2587.04|2672.64|2723.36|1061.76| +2450842|59163|2450874|28149|620952|3355|44205|28149|620952|3355|44205|2|52|18|4|11240|151|1406|60|94.47|177.60|136.75|2451.00|8205.00|5668.20|10656.00|328.20|0.00|2450.40|8205.00|8533.20|10655.40|10983.60|2536.80| +2450842|59163|2450896|28149|620952|3355|44205|28149|620952|3355|44205|2|90|10|5|5548|170|1406|10|14.51|27.71|15.24|124.70|152.40|145.10|277.10|6.09|0.00|19.30|152.40|158.49|171.70|177.79|7.30| +2450842|59163|2450924|28149|620952|3355|44205|28149|620952|3355|44205|2|69|4|3|14594|296|1406|74|57.47|67.23|28.23|2886.00|2089.02|4252.78|4975.02|188.01|0.00|1044.14|2089.02|2277.03|3133.16|3321.17|-2163.76| +2450842|59163|2450893|28149|620952|3355|44205|28149|620952|3355|44205|2|74|6|1|44|23|1406|99|34.18|93.31|73.71|1940.40|7297.29|3383.82|9237.69|364.86|0.00|2863.08|7297.29|7662.15|10160.37|10525.23|3913.47| +2450842|59163|2450909|28149|620952|3355|44205|28149|620952|3355|44205|2|42|7|4|5353|113|1406|39|3.11|6.46|2.51|154.05|97.89|121.29|251.94|3.17|45.02|52.65|52.87|56.04|105.52|108.69|-68.42| +2450842|60135|2450880|98131|365691|4451|27611|98131|365691|4451|27611|2|96|19|4|11348|164|1407|80|30.09|89.66|77.10|1004.80|6168.00|2407.20|7172.80|123.36|0.00|1505.60|6168.00|6291.36|7673.60|7796.96|3760.80| +2450842|60135|2450872|98131|365691|4451|27611|98131|365691|4451|27611|2|73|11|1|4892|193|1407|86|57.70|58.27|7.57|4360.20|651.02|4962.20|5011.22|58.59|0.00|1452.54|651.02|709.61|2103.56|2162.15|-4311.18| +2450842|60135|2450931|98131|365691|4451|27611|98131|365691|4451|27611|2|43|17|4|14062|28|1407|92|28.11|72.80|17.47|5090.36|1607.24|2586.12|6697.60|48.21|0.00|2410.40|1607.24|1655.45|4017.64|4065.85|-978.88| +2450842|60135|2450902|98131|365691|4451|27611|98131|365691|4451|27611|2|86|17|3|7424|182|1407|47|8.80|25.43|7.12|860.57|334.64|413.60|1195.21|2.30|103.73|71.44|230.91|233.21|302.35|304.65|-182.69| +2450842|2765|2450921|90150|1112589|5075|44361|90150|1112589|5075|44361|4|101|8|3|13420|126|1408|33|12.49|34.22|23.61|350.13|779.13|412.17|1129.26|0.00|0.00|44.88|779.13|779.13|824.01|824.01|366.96| +2450842|2765|2450848|90150|1112589|5075|44361|90150|1112589|5075|44361|4|90|16|5|463|208|1408|39|93.53|198.28|95.17|4021.29|3711.63|3647.67|7732.92|0.00|0.00|386.49|3711.63|3711.63|4098.12|4098.12|63.96| +2450842|2765|2450899|90150|1112589|5075|44361|90150|1112589|5075|44361|4|26|12|5|10894|68|1408|68|92.87|270.25|237.82|2205.24|16171.76|6315.16|18377.00|161.71|0.00|4594.08|16171.76|16333.47|20765.84|20927.55|9856.60| +2450842|2765|2450864|90150|1112589|5075|44361|90150|1112589|5075|44361|4|63|18|4|6334|275|1408|57|97.82|147.70|67.94|4546.32|3872.58|5575.74|8418.90|38.72|0.00|3030.69|3872.58|3911.30|6903.27|6941.99|-1703.16| +2450842|2765|2450894|90150|1112589|5075|44361|90150|1112589|5075|44361|4|41|20|3|2623|244|1408|12|56.76|132.25|35.70|1158.60|428.40|681.12|1587.00|21.42|0.00|174.48|428.40|449.82|602.88|624.30|-252.72| +2450842|2765|2450908|90150|1112589|5075|44361|90150|1112589|5075|44361|4|86|14|1|7826|52|1408|84|9.89|19.48|7.98|966.00|670.32|830.76|1636.32|13.40|0.00|162.96|670.32|683.72|833.28|846.68|-160.44| +2450842|2765|2450893|90150|1112589|5075|44361|90150|1112589|5075|44361|4|63|3|2|3253|250|1408|33|53.16|134.49|92.79|1376.10|3062.07|1754.28|4438.17|275.58|0.00|1153.68|3062.07|3337.65|4215.75|4491.33|1307.79| +2450842|2765|2450912|90150|1112589|5075|44361|90150|1112589|5075|44361|4|23|7|5|974|38|1408|64|26.65|76.75|66.00|688.00|4224.00|1705.60|4912.00|42.24|0.00|48.64|4224.00|4266.24|4272.64|4314.88|2518.40| +2450842|2765|2450905|90150|1112589|5075|44361|90150|1112589|5075|44361|4|61|14|2|5992|91|1408|36|93.18|113.67|40.92|2619.00|1473.12|3354.48|4092.12|73.65|0.00|1554.84|1473.12|1546.77|3027.96|3101.61|-1881.36| +2450842|2765|2450923|90150|1112589|5075|44361|90150|1112589|5075|44361|4|33|3|5|7268|252|1408|80|29.76|43.44|17.81|2050.40|1424.80|2380.80|3475.20|0.00|484.43|833.60|940.37|940.37|1773.97|1773.97|-1440.43| +2450842|2765|2450915|90150|1112589|5075|44361|90150|1112589|5075|44361|4|78|4|1|10294|63|1408|96|55.68|78.50|4.71|7083.84|452.16|5345.28|7536.00|0.00|0.00|3165.12|452.16|452.16|3617.28|3617.28|-4893.12| +2450842|52749|2450896|56408|1266377|4793|15962|56408|1266377|4793|15962|2|102|9|1|5752|140|1409|49|37.10|37.47|20.60|826.63|1009.40|1817.90|1836.03|40.37|0.00|568.89|1009.40|1049.77|1578.29|1618.66|-808.50| +2450842|52749|2450877|56408|1266377|4793|15962|56408|1266377|4793|15962|2|106|7|4|9910|66|1409|9|79.12|192.26|132.65|536.49|1193.85|712.08|1730.34|23.87|0.00|190.26|1193.85|1217.72|1384.11|1407.98|481.77| +2450842|52749|2450860|56408|1266377|4793|15962|56408|1266377|4793|15962|2|88|19|3|4096|90|1409|11|26.02|61.92|13.00|538.12|143.00|286.22|681.12|5.72|0.00|238.37|143.00|148.72|381.37|387.09|-143.22| +2450842|52749|2450876|56408|1266377|4793|15962|56408|1266377|4793|15962|2|41|20|1|11974|285|1409|72|88.41|170.63|112.61|4177.44|8107.92|6365.52|12285.36|81.07|0.00|736.56|8107.92|8188.99|8844.48|8925.55|1742.40| +2450842|52749|2450905|56408|1266377|4793|15962|56408|1266377|4793|15962|2|80|8|2|14654|157|1409|42|96.63|288.92|80.89|8737.26|3397.38|4058.46|12134.64|135.89|0.00|1456.14|3397.38|3533.27|4853.52|4989.41|-661.08| +2450842|52749|2450862|56408|1266377|4793|15962|56408|1266377|4793|15962|2|98|10|4|7942|225|1409|13|75.03|75.78|66.68|118.30|866.84|975.39|985.14|52.01|0.00|147.68|866.84|918.85|1014.52|1066.53|-108.55| +2450842|52749|2450874|56408|1266377|4793|15962|56408|1266377|4793|15962|2|50|5|5|12751|279|1409|51|31.44|49.98|29.98|1020.00|1528.98|1603.44|2548.98|91.73|0.00|75.99|1528.98|1620.71|1604.97|1696.70|-74.46| +2450842|52749|2450871|56408|1266377|4793|15962|56408|1266377|4793|15962|2|87|9|5|9622|70|1409|8|63.15|169.87|54.35|924.16|434.80|505.20|1358.96|17.39|0.00|597.92|434.80|452.19|1032.72|1050.11|-70.40| +2450842|3024|2450866|14961|85464|5567|38870|14961|85464|5567|38870|4|16|17|4|8773|230|1410|52|53.87|144.91|23.18|6329.96|1205.36|2801.24|7535.32|108.48|0.00|2109.64|1205.36|1313.84|3315.00|3423.48|-1595.88| +2450842|3024|2450894|14961|85464|5567|38870|14961|85464|5567|38870|4|19|11|4|958|40|1410|24|49.93|79.38|0.79|1886.16|18.96|1198.32|1905.12|1.70|0.00|780.96|18.96|20.66|799.92|801.62|-1179.36| +2450842|3024|2450883|14961|85464|5567|38870|14961|85464|5567|38870|4|86|13|5|16939|99|1410|53|86.60|95.26|60.96|1817.90|3230.88|4589.80|5048.78|129.23|0.00|100.70|3230.88|3360.11|3331.58|3460.81|-1358.92| +2450842|3024|2450918|14961|85464|5567|38870|14961|85464|5567|38870|4|70|9|3|12608|145|1410|76|63.30|185.46|76.03|8316.68|5778.28|4810.80|14094.96|0.00|0.00|1409.04|5778.28|5778.28|7187.32|7187.32|967.48| +2450842|75019|2450886|45067|1718012|3757|22702|45067|1718012|3757|22702|2|56|13|3|2980|243|1411|45|44.18|69.80|10.47|2669.85|471.15|1988.10|3141.00|13.56|320.38|659.25|150.77|164.33|810.02|823.58|-1837.33| +2450842|75019|2450868|45067|1718012|3757|22702|45067|1718012|3757|22702|2|3|13|5|14617|19|1411|10|80.55|154.65|109.80|448.50|1098.00|805.50|1546.50|43.92|0.00|587.60|1098.00|1141.92|1685.60|1729.52|292.50| +2450842|75019|2450890|45067|1718012|3757|22702|45067|1718012|3757|22702|2|65|19|5|3613|214|1411|52|39.34|75.92|69.08|355.68|3592.16|2045.68|3947.84|323.29|0.00|1578.72|3592.16|3915.45|5170.88|5494.17|1546.48| +2450842|75019|2450882|45067|1718012|3757|22702|45067|1718012|3757|22702|2|65|20|2|13760|248|1411|80|58.65|58.65|31.08|2205.60|2486.40|4692.00|4692.00|174.04|0.00|280.80|2486.40|2660.44|2767.20|2941.24|-2205.60| +2450842|75019|2450929|45067|1718012|3757|22702|45067|1718012|3757|22702|2|72|14|5|15259|25|1411|81|47.46|80.20|10.42|5652.18|844.02|3844.26|6496.20|42.20|0.00|2532.87|844.02|886.22|3376.89|3419.09|-3000.24| +2450842|75019|2450928|45067|1718012|3757|22702|45067|1718012|3757|22702|2|3|16|2|10381|127|1411|82|24.45|64.05|48.03|1313.64|3938.46|2004.90|5252.10|0.00|0.00|0.00|3938.46|3938.46|3938.46|3938.46|1933.56| +2450842|75019|2450866|45067|1718012|3757|22702|45067|1718012|3757|22702|2|43|7|3|17404|195|1411|52|81.26|197.46|11.84|9652.24|615.68|4225.52|10267.92|0.00|584.89|3388.32|30.79|30.79|3419.11|3419.11|-4194.73| +2450842|15283|2450929|25729|193596|3416|25636|25729|193596|3416|25636|1|105|8|2|3700|99|1412|7|4.77|12.02|11.53|3.43|80.71|33.39|84.14|1.61|0.00|9.24|80.71|82.32|89.95|91.56|47.32| +2450842|15283|2450850|25729|193596|3416|25636|25729|193596|3416|25636|1|72|7|1|331|19|1412|74|98.75|270.57|219.16|3804.34|16217.84|7307.50|20022.18|324.35|0.00|1201.02|16217.84|16542.19|17418.86|17743.21|8910.34| +2450842|15283|2450848|25729|193596|3416|25636|25729|193596|3416|25636|1|81|2|1|8800|176|1412|34|7.41|10.22|6.74|118.32|229.16|251.94|347.48|11.45|0.00|58.82|229.16|240.61|287.98|299.43|-22.78| +2450842|15283|2450918|25729|193596|3416|25636|25729|193596|3416|25636|1|9|18|2|529|178|1412|22|11.09|21.29|7.02|313.94|154.44|243.98|468.38|13.89|0.00|168.52|154.44|168.33|322.96|336.85|-89.54| +2450842|15283|2450928|25729|193596|3416|25636|25729|193596|3416|25636|1|14|9|3|8371|279|1412|23|26.06|43.25|18.16|577.07|417.68|599.38|994.75|37.59|0.00|149.04|417.68|455.27|566.72|604.31|-181.70| +2450842|15283|2450851|25729|193596|3416|25636|25729|193596|3416|25636|1|34|18|5|15613|179|1412|84|95.51|114.61|43.55|5969.04|3658.20|8022.84|9627.24|182.91|0.00|3080.28|3658.20|3841.11|6738.48|6921.39|-4364.64| +2450842|15283|2450911|25729|193596|3416|25636|25729|193596|3416|25636|1|104|15|1|11809|230|1412|34|45.60|124.94|111.19|467.50|3780.46|1550.40|4247.96|28.35|3213.39|1316.82|567.07|595.42|1883.89|1912.24|-983.33| +2450842|15283|2450923|25729|193596|3416|25636|25729|193596|3416|25636|1|70|7|1|7435|136|1412|73|90.38|252.16|88.25|11965.43|6442.25|6597.74|18407.68|257.69|0.00|919.80|6442.25|6699.94|7362.05|7619.74|-155.49| +2450842|15283|2450910|25729|193596|3416|25636|25729|193596|3416|25636|1|76|2|3|11450|28|1412|84|96.67|131.47|80.19|4307.52|6735.96|8120.28|11043.48|606.23|0.00|1103.76|6735.96|7342.19|7839.72|8445.95|-1384.32| +2450842|15283|2450930|25729|193596|3416|25636|25729|193596|3416|25636|1|57|12|2|7153|135|1412|51|61.13|158.93|141.44|891.99|7213.44|3117.63|8105.43|504.94|0.00|3809.19|7213.44|7718.38|11022.63|11527.57|4095.81| +2450842|15283|2450897|25729|193596|3416|25636|25729|193596|3416|25636|1|75|16|3|7084|65|1412|88|3.71|6.45|2.83|318.56|249.04|326.48|567.60|0.00|0.00|232.32|249.04|249.04|481.36|481.36|-77.44| +2450842|15283|2450932|25729|193596|3416|25636|25729|193596|3416|25636|1|72|20|1|7496|104|1412|39|93.89|106.09|9.54|3765.45|372.06|3661.71|4137.51|18.75|163.70|2068.56|208.36|227.11|2276.92|2295.67|-3453.35| +2450842|15283|2450894|25729|193596|3416|25636|25729|193596|3416|25636|1|99|7|4|11635|283|1412|85|74.73|222.69|160.33|5300.60|13628.05|6352.05|18928.65|1090.24|0.00|6435.35|13628.05|14718.29|20063.40|21153.64|7276.00| +2450842|28875|2450854|96553|1812010|2314|9371|96553|1812010|2314|9371|2|7|18|4|1741|236|1413|99|14.61|32.87|17.74|1497.87|1756.26|1446.39|3254.13|140.50|0.00|422.73|1756.26|1896.76|2178.99|2319.49|309.87| +2450842|28875|2450929|96553|1812010|2314|9371|96553|1812010|2314|9371|2|29|8|4|16513|213|1413|88|9.53|21.82|19.41|212.08|1708.08|838.64|1920.16|17.08|0.00|172.48|1708.08|1725.16|1880.56|1897.64|869.44| +2450842|28875|2450913|96553|1812010|2314|9371|96553|1812010|2314|9371|2|1|13|2|9355|180|1413|11|5.66|8.71|3.65|55.66|40.15|62.26|95.81|2.81|0.00|8.58|40.15|42.96|48.73|51.54|-22.11| +|||96553|1812010|2314|9371|96553|1812010|2314|9371||91|||16892||1413|25|47.72||66.53|1307.25||1193.00|2970.50|0.00|||1663.25|1663.25|3059.25|3059.25|470.25| +2450842|28875|2450914|96553|1812010|2314|9371|96553|1812010|2314|9371|2|40|8|2|14479|222|1413|79|37.64|56.83|39.21|1391.98|3097.59|2973.56|4489.57|154.87|0.00|313.63|3097.59|3252.46|3411.22|3566.09|124.03| +2450842|60801|2450902|7826|970311|6749|30434|7826|970311|6749|30434|4|36|11|3|16855|135|1414|16|47.44|91.55|9.15|1318.40|146.40|759.04|1464.80|13.17|0.00|278.24|146.40|159.57|424.64|437.81|-612.64| +2450842|60801|2450860|7826|970311|6749|30434|7826|970311|6749|30434|4|23|11|4|5128|65|1414|19|14.69|20.27|10.13|192.66|192.47|279.11|385.13|0.00|0.00|92.34|192.47|192.47|284.81|284.81|-86.64| +2450842|60801|2450871|7826|970311|6749|30434|7826|970311|6749|30434|4|78|20|1|17764|228|1414|26|17.63|27.85|10.58|449.02|275.08|458.38|724.10|19.25|0.00|318.50|275.08|294.33|593.58|612.83|-183.30| +2450842|60801|2450922|7826|970311|6749|30434|7826|970311|6749|30434|4|33|19|3|5263|42|1414|36|90.61|173.06|44.99|4610.52|1619.64|3261.96|6230.16|113.37|0.00|249.12|1619.64|1733.01|1868.76|1982.13|-1642.32| +2450842|60801|2450882|7826|970311|6749|30434|7826|970311|6749|30434|4|46|1|2|3826|219|1414|69|75.73|107.53|55.91|3561.78|3857.79|5225.37|7419.57|0.00|655.82|3709.44|3201.97|3201.97|6911.41|6911.41|-2023.40| +2450842|60801|2450853|7826|970311|6749|30434|7826|970311|6749|30434|4|42|17|1|8827|114|1414|60|36.94|44.32|43.87|27.00|2632.20|2216.40|2659.20|105.28|0.00|159.00|2632.20|2737.48|2791.20|2896.48|415.80| +2450842|60801|2450919|7826|970311|6749|30434|7826|970311|6749|30434|4|66|19|4|1376|211|1414|87|88.02|133.79|46.82|7566.39|4073.34|7657.74|11639.73|172.30|2158.87|3607.89|1914.47|2086.77|5522.36|5694.66|-5743.27| +2450842|60801|2450875|7826|970311|6749|30434|7826|970311|6749|30434|4|92|9|5|16510|100|1414|92|77.71|116.56|81.59|3217.24|7506.28|7149.32|10723.52|0.00|0.00|1714.88|7506.28|7506.28|9221.16|9221.16|356.96| +2450842|60801|2450896|7826|970311|6749|30434|7826|970311|6749|30434|4|55|18|5|16849|273|1414|56|78.05|141.27|70.63|3955.84|3955.28|4370.80|7911.12|53.79|1265.68|869.68|2689.60|2743.39|3559.28|3613.07|-1681.20| +2450842|60801|2450884|7826|970311|6749|30434|7826|970311|6749|30434|4|64|18|5|4615|196|1414|73|55.26|82.89|24.03|4296.78|1754.19|4033.98|6050.97|140.33|0.00|3025.12|1754.19|1894.52|4779.31|4919.64|-2279.79| +2450842|60801|2450926|7826|970311|6749|30434|7826|970311|6749|30434|4|87|14|2|7861|184|1414|41|41.28|91.64|73.31|751.53|3005.71|1692.48|3757.24|180.34|0.00|1239.84|3005.71|3186.05|4245.55|4425.89|1313.23| +2450842|80515|2450881|77536|1202200|3311|39175|52215|1447350|461|27125|2|82|19|2|15847|205|1415|99|82.64|190.07|5.70|18252.63|564.30|8181.36|18816.93|28.21|0.00|9219.87|564.30|592.51|9784.17|9812.38|-7617.06| +2450842|80515|2450919|77536|1202200|3311|39175|52215|1447350|461|27125|2|1|16|5|712|138|1415|70|17.37|45.33|37.17|571.20|2601.90|1215.90|3173.10|182.13|0.00|1141.70|2601.90|2784.03|3743.60|3925.73|1386.00| +2450842|80515|2450924|77536|1202200|3311|39175|52215|1447350|461|27125|2|19|6|4|7981|20|1415|23|65.48|86.43|47.53|894.70|1093.19|1506.04|1987.89|98.38|0.00|0.00|1093.19|1191.57|1093.19|1191.57|-412.85| +2450842|80515|2450845|77536|1202200|3311|39175|52215|1447350|461|27125|2|35|5|4|5740|255|1415|94|42.91|96.54|39.58|5354.24|3720.52|4033.54|9074.76|0.00|0.00|1088.52|3720.52|3720.52|4809.04|4809.04|-313.02| +2450842|80515|2450929|77536|1202200|3311|39175|52215|1447350|461|27125|2|106|9|4|16921|81|1415|79|77.34|170.92|85.46|6751.34|6751.34|6109.86|13502.68|270.05|0.00|269.39|6751.34|7021.39|7020.73|7290.78|641.48| +2450842|73953|2450894|66361|1561966|3171|5576|66361|1561966|3171|5576|2|48|14|2|1030|115|1416|15|81.28|142.24|38.40|1557.60|576.00|1219.20|2133.60|10.59|46.08|725.40|529.92|540.51|1255.32|1265.91|-689.28| +2450842|73953|2450857|66361|1561966|3171|5576|66361|1561966|3171|5576|2|74|18|4|4340|41|1416|82|68.85|105.34|102.17|259.94|8377.94|5645.70|8637.88|251.33|0.00|3886.80|8377.94|8629.27|12264.74|12516.07|2732.24| +2450842|73953|2450917|66361|1561966|3171|5576|66361|1561966|3171|5576|2|22|20|2|6535|279|1416|7|14.18|15.88|0.63|106.75|4.41|99.26|111.16|0.13|0.00|22.19|4.41|4.54|26.60|26.73|-94.85| +2450842|73953|2450886|66361|1561966|3171|5576|66361|1561966|3171|5576|2|55|13|1|15466|245|1416|67|52.08|69.26|66.48|186.26|4454.16|3489.36|4640.42|267.24|0.00|509.87|4454.16|4721.40|4964.03|5231.27|964.80| +2450842|73953|2450863|66361|1561966|3171|5576|66361|1561966|3171|5576|2|71|20|5|12457|252|1416|69|27.56|42.71|28.18|1002.57|1944.42|1901.64|2946.99|155.55|0.00|883.89|1944.42|2099.97|2828.31|2983.86|42.78| +2450842|73953|2450879|66361|1561966|3171|5576|66361|1561966|3171|5576|2|71|15|2|2005|56|1416|15|36.95|86.83|34.73|781.50|520.95|554.25|1302.45|0.00|0.00|625.05|520.95|520.95|1146.00|1146.00|-33.30| +2450842|73953|2450896|66361|1561966|3171|5576|66361|1561966|3171|5576|2|76|3|4|8506|300|1416|90|71.29|90.53|66.99|2118.60|6029.10|6416.10|8147.70|256.23|904.36|3421.80|5124.74|5380.97|8546.54|8802.77|-1291.36| +2450842|73953|2450846|66361|1561966|3171|5576|66361|1561966|3171|5576|2|4|10|4|14392|61|1416|15|75.89|121.42|8.49|1693.95|127.35|1138.35|1821.30|7.64|0.00|582.75|127.35|134.99|710.10|717.74|-1011.00| +2450842|73953|2450915|66361|1561966|3171|5576|66361|1561966|3171|5576|2|20|1|4|14330|16|1416|76|17.19|24.40|4.88|1483.52|370.88|1306.44|1854.40|0.00|0.00|649.04|370.88|370.88|1019.92|1019.92|-935.56| +2450842|73953|2450869|66361|1561966|3171|5576|66361|1561966|3171|5576|2|69|14|5|2260|251|1416|39|7.55|20.23|14.97|205.14|583.83|294.45|788.97|2.80|490.41|149.76|93.42|96.22|243.18|245.98|-201.03| +2450842|73953|2450922|66361|1561966|3171|5576|66361|1561966|3171|5576|2|98|6|5|3583|300|1416|40|81.77|167.62|95.54|2883.20|3821.60|3270.80|6704.80|51.59|2789.76|2547.60|1031.84|1083.43|3579.44|3631.03|-2238.96| +2450842|73953|2450889|66361|1561966|3171|5576|66361|1561966|3171|5576|2|54|2|4|1232|86|1416|49|51.02|96.42|26.99|3402.07|1322.51|2499.98|4724.58|52.90|0.00|94.08|1322.51|1375.41|1416.59|1469.49|-1177.47| +2450842|73953|2450920|66361|1561966|3171|5576|66361|1561966|3171|5576|2|21|3|1|1582|182|1416|35|89.50|91.29|34.69|1981.00|1214.15|3132.50|3195.15|60.70|0.00|542.85|1214.15|1274.85|1757.00|1817.70|-1918.35| +2450842|73953|2450896|66361|1561966|3171|5576|66361|1561966|3171|5576|2|90|13|2|1279|278|1416|33|4.46|11.59|8.69|95.70|286.77|147.18|382.47|25.80|0.00|152.79|286.77|312.57|439.56|465.36|139.59| +|9459|||874194||18542||874194|||2||9|2|1885|73|1417||||19.56|||4961.55|11162.76|113.83|0.00||1897.32||3571.54|3685.37|-3064.23| +2450842|9459|2450882|62973|874194|3368|18542|62973|874194|3368|18542|2|69|9|1|15742|288|1417|85|79.20|116.42|82.65|2870.45|7025.25|6732.00|9895.70|421.51|0.00|3859.00|7025.25|7446.76|10884.25|11305.76|293.25| +2450842|9459|2450860|62973|874194|3368|18542|62973|874194|3368|18542|2|52|9|2|8002|263|1417|45|24.98|70.94|36.17|1564.65|1627.65|1124.10|3192.30|81.38|0.00|1468.35|1627.65|1709.03|3096.00|3177.38|503.55| +2450842|9459|2450911|62973|874194|3368|18542|62973|874194|3368|18542|2|54|8|2|7312|57|1417|4|18.93|25.36|6.08|77.12|24.32|75.72|101.44|0.00|0.00|16.20|24.32|24.32|40.52|40.52|-51.40| +2450842|37998|2450898|52551|902991|1078|18318|52551|902991|1078|18318|2|92|19|4|11132|281|1418|12|71.32|72.74|1.45|855.48|17.40|855.84|872.88|0.23|5.74|418.92|11.66|11.89|430.58|430.81|-844.18| +2450842|37998|2450845|52551|902991|1078|18318|52551|902991|1078|18318|2|59|3|2|12476|231|1418|50|5.40|14.04|3.93|505.50|196.50|270.00|702.00|5.89|0.00|28.00|196.50|202.39|224.50|230.39|-73.50| +2450842|37998|2450896|52551|902991|1078|18318|52551|902991|1078|18318|2|34|1|1|13646|141|1418|89|41.52|77.22|0.77|6804.05|68.53|3695.28|6872.58|5.48|0.00|962.09|68.53|74.01|1030.62|1036.10|-3626.75| +2450842|37998|2450909|52551|902991|1078|18318|52551|902991|1078|18318|2|53|20|1|3724|288|1418|45|13.21|14.26|7.84|288.90|352.80|594.45|641.70|21.16|0.00|102.60|352.80|373.96|455.40|476.56|-241.65| +2450842|37998|2450907|52551|902991|1078|18318|52551|902991|1078|18318|2|1|16|1|6202|286|1418|97|11.78|30.62|25.41|505.37|2464.77|1142.66|2970.14|98.59|0.00|88.27|2464.77|2563.36|2553.04|2651.63|1322.11| +2450842|37998|2450923|52551|902991|1078|18318|52551|902991|1078|18318|2|69|11|5|2380|53|1418|85|7.98|22.58|16.70|499.80|1419.50|678.30|1919.30|28.39|0.00|76.50|1419.50|1447.89|1496.00|1524.39|741.20| +2450842|37998|2450846|52551|902991|1078|18318|52551|902991|1078|18318|2|37|13|4|3694|164|1418|28|86.39|106.25|15.93|2528.96|446.04|2418.92|2975.00|31.22|0.00|565.04|446.04|477.26|1011.08|1042.30|-1972.88| +2450842|37998|2450885|52551|902991|1078|18318|52551|902991|1078|18318|2|60|8|2|6004|135|1418|19|10.79|25.03|4.25|394.82|80.75|205.01|475.57|3.23|0.00|171.19|80.75|83.98|251.94|255.17|-124.26| +2450842|49331|2450888|26559|963847|1935|4470|26559|963847|1935|4470|4|13|19|4|16460|169|1419|52|43.33|43.76|17.50|1365.52|910.00|2253.16|2275.52|45.50|0.00|864.24|910.00|955.50|1774.24|1819.74|-1343.16| +2450842|49331|2450927|26559|963847|1935|4470|26559|963847|1935|4470|4|50|15|1|17174|78|1419|71|11.23|11.34|6.91|314.53|490.61|797.33|805.14|24.53|0.00|80.23|490.61|515.14|570.84|595.37|-306.72| +2450842|49331|2450887|26559|963847|1935|4470|26559|963847|1935|4470|4|59|14|5|13198|77|1419|76|37.62|72.23|34.67|2854.56|2634.92|2859.12|5489.48|237.14|0.00|1591.44|2634.92|2872.06|4226.36|4463.50|-224.20| +2450842|49331|2450904|26559|963847|1935|4470|26559|963847|1935|4470|4|64|18|2|3250|165|1419|8|37.94|77.01|15.40|492.88|123.20|303.52|616.08|10.86|2.46|129.36|120.74|131.60|250.10|260.96|-182.78| +2450842|49331|2450856|26559|963847|1935|4470|26559|963847|1935|4470|4|88|11|1|3004|24|1419|94|54.40|128.38|37.23|8568.10|3499.62|5113.60|12067.72|0.00|3184.65|2171.40|314.97|314.97|2486.37|2486.37|-4798.63| +2450842|49331|2450858|26559|963847|1935|4470|26559|963847|1935|4470|4|95|1|4|8263|289|1419|69|16.09|47.14|22.62|1691.88|1560.78|1110.21|3252.66|62.43|0.00|747.96|1560.78|1623.21|2308.74|2371.17|450.57| +2450842|49331|2450886|26559|963847|1935|4470|26559|963847|1935|4470|4|17|2|5|8074|181|1419|11|81.80|179.14|164.80|157.74|1812.80|899.80|1970.54|126.89|0.00|867.02|1812.80|1939.69|2679.82|2806.71|913.00| +2450842|49331|2450848|26559|963847|1935|4470|26559|963847|1935|4470|4|89|12|1|5077|101|1419|81|58.17|163.45|122.58|3310.47|9928.98|4711.77|13239.45|0.00|0.00|1058.67|9928.98|9928.98|10987.65|10987.65|5217.21| +2450842|49331|2450876|26559|963847|1935|4470|26559|963847|1935|4470|4|65|19|4|8047|61|1419|47|55.43|161.85|12.94|6998.77|608.18|2605.21|7606.95|4.07|200.69|3650.96|407.49|411.56|4058.45|4062.52|-2197.72| +2450842|49331|2450917|26559|963847|1935|4470|26559|963847|1935|4470|4|12|13|1|2347|238|1419|62|73.16|132.41|13.24|7388.54|820.88|4535.92|8209.42|5.33|287.30|3940.10|533.58|538.91|4473.68|4479.01|-4002.34| +2450842|49331|2450860|26559|963847|1935|4470|26559|963847|1935|4470|4|101|11|5|13945|65|1419|75|38.18|50.77|42.13|648.00|3159.75|2863.50|3807.75|94.79|0.00|456.75|3159.75|3254.54|3616.50|3711.29|296.25| +2450842|49331|2450904|26559|963847|1935|4470|26559|963847|1935|4470|4|93|14|4|14516|247|1419|46|84.90|170.64|46.07|5730.22|2119.22|3905.40|7849.44|18.01|317.88|2119.22|1801.34|1819.35|3920.56|3938.57|-2104.06| +2450842|49331|2450855|26559|963847|1935|4470|26559|963847|1935|4470|4|108|9|3|8126|196|1419|59|92.24|163.26|128.97|2023.11|7609.23|5442.16|9632.34|152.18|0.00|96.17|7609.23|7761.41|7705.40|7857.58|2167.07| +2450842|||||2072||83714|816493|||4||2||2458||1420|||96.71|59.96|||871.71||32.10|616.98|263.97|||906.15|938.25|-229.53| +2450842|67821|2450918|83714|816493|2072|7651|83714|816493|2072|7651|4|101|12|4|7418|76|1420|35|53.23|122.96|70.08|1850.80|2452.80|1863.05|4303.60|49.05|0.00|1549.10|2452.80|2501.85|4001.90|4050.95|589.75| +2450842|67821|2450894|83714|816493|2072|7651|83714|816493|2072|7651|4|74|2|5|15770|223|1420|19|14.76|35.71|19.64|305.33|373.16|280.44|678.49|11.19|0.00|135.66|373.16|384.35|508.82|520.01|92.72| +2450842|67821|2450867|83714|816493|2072|7651|83714|816493|2072|7651|4|81|20|1|3344|33|1420|34|93.91|170.91|167.49|116.28|5694.66|3192.94|5810.94|113.89|0.00|987.70|5694.66|5808.55|6682.36|6796.25|2501.72| +2450842|67821|2450887|83714|816493|2072|7651|83714|816493|2072|7651|4|88|12|3|12916|15|1420|34|89.70|134.55|45.74|3019.54|1555.16|3049.80|4574.70|15.55|0.00|1418.14|1555.16|1570.71|2973.30|2988.85|-1494.64| +2450842|67821|2450867|83714|816493|2072|7651|83714|816493|2072|7651|4|27|8|2|1166|85|1420|53|76.51|100.99|14.13|4603.58|748.89|4055.03|5352.47|67.40|0.00|1926.55|748.89|816.29|2675.44|2742.84|-3306.14| +2450842|67821|2450857|83714|816493|2072|7651|83714|816493|2072|7651|4|76|11|5|3313|260|1420|84|62.78|97.93|90.09|658.56|7567.56|5273.52|8226.12|454.05|0.00|1151.64|7567.56|8021.61|8719.20|9173.25|2294.04| +2450842|67821|2450890|83714|816493|2072|7651|83714|816493|2072|7651|4|18|15|1|16334|75|1420|49|92.83|164.30|115.01|2415.21|5635.49|4548.67|8050.70|507.19|0.00|1610.14|5635.49|6142.68|7245.63|7752.82|1086.82| +2450842|42593|2450872|62267|1768715|5116|30717|62267|1768715|5116|30717|4|93|18|2|7784|153|1421|4|10.92|25.55|16.60|35.80|66.40|43.68|102.20|4.64|0.00|15.32|66.40|71.04|81.72|86.36|22.72| +2450842|42593|2450918|62267|1768715|5116|30717|62267|1768715|5116|30717|4|47|10|2|11500|79|1421|72|74.85|201.34|167.11|2464.56|12031.92|5389.20|14496.48|0.00|0.00|6523.20|12031.92|12031.92|18555.12|18555.12|6642.72| +2450842|42593|2450921|62267|1768715|5116|30717|62267|1768715|5116|30717|4|75|2|4|16100|144|1421|3|33.56|84.57|37.21|142.08|111.63|100.68|253.71|6.69|0.00|22.83|111.63|118.32|134.46|141.15|10.95| +2450842|42593|2450890|62267|1768715|5116|30717|62267|1768715|5116|30717|4|54|8|1|8968|212|1421|53|41.44|56.77|21.00|1895.81|1113.00|2196.32|3008.81|0.00|0.00|541.13|1113.00|1113.00|1654.13|1654.13|-1083.32| +2450842|76167|2450906|75844|139779|4514|4849|75844|139779|4514|4849|1|46|12|4|16406|193|1422|75|68.78|140.99|38.06|7719.75|2854.50|5158.50|10574.25|142.72|0.00|3912.00|2854.50|2997.22|6766.50|6909.22|-2304.00| +2450842|76167|2450892|75844|139779|4514|4849|75844|139779|4514|4849|1|9|3|2|7568|80|1422|34|66.02|134.68|51.17|2839.34|1739.78|2244.68|4579.12|121.78|0.00|45.56|1739.78|1861.56|1785.34|1907.12|-504.90| +2450842|76167|2450924|75844|139779|4514|4849|75844|139779|4514|4849|1|51|9|2|16891|53|1422|40|39.86|117.18|76.16|1640.80|3046.40|1594.40|4687.20|30.46|0.00|1124.80|3046.40|3076.86|4171.20|4201.66|1452.00| +2450842|76167|2450866|75844|139779|4514|4849|75844|139779|4514|4849|1|52|5|4|854|37|1422|88|73.95|162.69|8.13|13601.28|715.44|6507.60|14316.72|36.70|307.63|2003.76|407.81|444.51|2411.57|2448.27|-6099.79| +2450842|76167|2450896|75844|139779|4514|4849|75844|139779|4514|4849|1|64|20|3|16304|299|1422|24|14.23|25.18|12.08|314.40|289.92|341.52|604.32|8.69|0.00|144.96|289.92|298.61|434.88|443.57|-51.60| +2450842|76167|2450878|75844|139779|4514|4849|75844|139779|4514|4849|1|26|12|3|8740|172|1422|18|14.03|19.36|6.58|230.04|118.44|252.54|348.48|6.82|33.16|62.64|85.28|92.10|147.92|154.74|-167.26| +2450842|76167|2450891|75844|139779|4514|4849|75844|139779|4514|4849|1|38|6|4|10330|103|1422|48|49.99|58.98|15.33|2095.20|735.84|2399.52|2831.04|36.79|0.00|792.48|735.84|772.63|1528.32|1565.11|-1663.68| +2450842|76167|2450905|75844|139779|4514|4849|75844|139779|4514|4849|1|104|3|5|12655|199|1422|58|2.51|6.65|6.11|31.32|354.38|145.58|385.70|10.63|0.00|3.48|354.38|365.01|357.86|368.49|208.80| +2450842|76167|2450928|75844|139779|4514|4849|75844|139779|4514|4849|1|25|12|2|7480|7|1422|24|18.65|52.40|46.11|150.96|1106.64|447.60|1257.60|99.59|0.00|402.24|1106.64|1206.23|1508.88|1608.47|659.04| +2450842|76167|2450887|75844|139779|4514|4849|75844|139779|4514|4849|1|59|14|5|5218|94|1422|18|55.60|104.52|97.20|131.76|1749.60|1000.80|1881.36|69.98|0.00|696.06|1749.60|1819.58|2445.66|2515.64|748.80| +2450842|76167|2450915|75844|139779|4514|4849|75844|139779|4514|4849|1|73|5|1|11774|73|1422|56|69.82|111.01|108.78|124.88|6091.68|3909.92|6216.56|60.91|0.00|1491.84|6091.68|6152.59|7583.52|7644.43|2181.76| +2450842|57370|2450882|95307|1757433|7112|26838|95307|1757433|7112|26838|4|72|6|2|15206|56|1423|28|82.32|147.35|30.94|3259.48|866.32|2304.96|4125.80|25.98|0.00|1443.96|866.32|892.30|2310.28|2336.26|-1438.64| +2450842|57370|2450917|95307|1757433|7112|26838|95307|1757433|7112|26838|4|11|7|4|3710|270|1423|70|55.54|137.73|60.60|5399.10|4242.00|3887.80|9641.10|127.26|0.00|1156.40|4242.00|4369.26|5398.40|5525.66|354.20| +2450842|57370|2450880|95307|1757433|7112|26838|95307|1757433|7112|26838|4|27|8|2|14950|95|1423|20|45.87|99.07|56.46|852.20|1129.20|917.40|1981.40|67.75|0.00|772.60|1129.20|1196.95|1901.80|1969.55|211.80| +2450842|57370|2450931|95307|1757433|7112|26838|95307|1757433|7112|26838|4|27|12|3|11404|236|1423|79|73.92|92.40|85.00|584.60|6715.00|5839.68|7299.60|402.90|0.00|291.51|6715.00|7117.90|7006.51|7409.41|875.32| +2450842|57370|2450876|95307|1757433|7112|26838|95307|1757433|7112|26838|4|44|10|4|2290|85|1423|97|9.80|14.40|1.29|1271.67|125.13|950.60|1396.80|11.26|0.00|292.94|125.13|136.39|418.07|429.33|-825.47| +2450842|57370|2450926|95307|1757433|7112|26838|95307|1757433|7112|26838|4|49|11|2|12260|197|1423|60|36.11|46.22|2.31|2634.60|138.60|2166.60|2773.20|4.15|0.00|0.00|138.60|142.75|138.60|142.75|-2028.00| +2450842|57370|2450877|95307|1757433|7112|26838|95307|1757433|7112|26838|4|93|5|5|6919|32|1423|10|8.34|24.60|0.24|243.60|2.40|83.40|246.00|0.16|0.00|105.70|2.40|2.56|108.10|108.26|-81.00| +2450842|57370|2450900|95307|1757433|7112|26838|95307|1757433|7112|26838|4|75|7|5|7580|255|1423|85|86.13|176.56|100.63|6454.05|8553.55|7321.05|15007.60|0.00|855.35|2100.35|7698.20|7698.20|9798.55|9798.55|377.15| +2450842|57370|2450873|95307|1757433|7112|26838|95307|1757433|7112|26838|4|6|1|3|2198|282|1423|8|21.27|49.98|39.98|80.00|319.84|170.16|399.84|2.33|86.35|19.92|233.49|235.82|253.41|255.74|63.33| +2450842|57370|2450855|95307|1757433|7112|26838|95307|1757433|7112|26838|4|76|1|5|8162|131|1423|30|36.43|37.52|28.14|281.40|844.20|1092.90|1125.60|67.53|0.00|360.00|844.20|911.73|1204.20|1271.73|-248.70| +2450842|57370|2450871|95307|1757433|7112|26838|95307|1757433|7112|26838|4|97|8|3|12116|33|1423|69|72.11|101.67|53.88|3297.51|3717.72|4975.59|7015.23|297.41|0.00|2174.19|3717.72|4015.13|5891.91|6189.32|-1257.87| +2450842|57370|2450848|95307|1757433|7112|26838|95307|1757433|7112|26838|4|39|16|3|9022|134|1423|30|7.92|9.34|4.20|154.20|126.00|237.60|280.20|11.34|0.00|36.30|126.00|137.34|162.30|173.64|-111.60| +2450842|57370|2450858|95307|1757433|7112|26838|95307|1757433|7112|26838|4|68|6|3|17930|290|1423|93|78.32|142.54|88.37|5037.81|8218.41|7283.76|13256.22|164.36|0.00|4638.84|8218.41|8382.77|12857.25|13021.61|934.65| +2450842|57370|2450849|95307|1757433|7112|26838|95307|1757433|7112|26838|4|94|16|5|17578|5|1423|6|96.60|190.30|34.25|936.30|205.50|579.60|1141.80|4.11|0.00|102.72|205.50|209.61|308.22|312.33|-374.10| +2450842|40634|2450881|81999|668326|2300|39074|81999|668326|2300|39074|4|51|7|1|11162|187|1424|98|4.69|13.31|12.51|78.40|1225.98|459.62|1304.38|31.38|441.35|403.76|784.63|816.01|1188.39|1219.77|325.01| +2450842|40634|2450878|81999|668326|2300|39074|81999|668326|2300|39074|4|46|8|1|5354|277|1424|78|8.67|12.65|8.09|355.68|631.02|676.26|986.70|25.24|0.00|482.82|631.02|656.26|1113.84|1139.08|-45.24| +2450842|40634|2450889|81999|668326|2300|39074|81999|668326|2300|39074|4|63|5|1|2866|278|1424|98|97.83|221.09|92.85|12567.52|9099.30|9587.34|21666.82|545.95|0.00|8016.40|9099.30|9645.25|17115.70|17661.65|-488.04| +2450842|40634|2450913|81999|668326|2300|39074|81999|668326|2300|39074|4|42|6|1|2533|248|1424|100|48.05|127.81|60.07|6774.00|6007.00|4805.00|12781.00|230.66|3123.64|894.00|2883.36|3114.02|3777.36|4008.02|-1921.64| +2450842|40634|2450867|81999|668326|2300|39074|81999|668326|2300|39074|4|57|3|1|2272|95|1424|26|80.20|92.23|40.58|1342.90|1055.08|2085.20|2397.98|94.95|0.00|1198.86|1055.08|1150.03|2253.94|2348.89|-1030.12| +2450842|40634|2450930|81999|668326|2300|39074|81999|668326|2300|39074|4|35|20|2|10112|236|1424|36|28.27|60.21|33.11|975.60|1191.96|1017.72|2167.56|35.75|0.00|64.80|1191.96|1227.71|1256.76|1292.51|174.24| +2450842|40634|2450907|81999|668326|2300|39074|81999|668326|2300|39074|4|87|15|4|12848|275|1424|77|18.02|41.80|10.45|2413.95|804.65|1387.54|3218.60|56.32|0.00|1448.37|804.65|860.97|2253.02|2309.34|-582.89| +2450842|29826|2450869|13465|1582799|2886|39420|13465|1582799|2886|39420|4|69|19|1|2968|188|1425|71|59.05|80.89|31.54|3503.85|2239.34|4192.55|5743.19|89.57|0.00|1378.11|2239.34|2328.91|3617.45|3707.02|-1953.21| +2450842|29826|2450868|13465|1582799|2886|39420|13465|1582799|2886|39420|4|96|12|4|493|296|1425|97|59.13|164.38|50.95|11002.71|4942.15|5735.61|15944.86|88.95|3459.50|4623.99|1482.65|1571.60|6106.64|6195.59|-4252.96| +2450842|29826|2450900|13465|1582799|2886|39420|13465|1582799|2886|39420|4|51|4|1|5536|282|1425|7|30.84|80.49|72.44|56.35|507.08|215.88|563.43|30.42|0.00|185.92|507.08|537.50|693.00|723.42|291.20| +2450842|29826|2450862|13465|1582799|2886|39420|13465|1582799|2886|39420|4|91|18|5|6866|194|1425|54|70.52|188.99|151.19|2041.20|8164.26|3808.08|10205.46|734.78|0.00|1122.12|8164.26|8899.04|9286.38|10021.16|4356.18| +2450842|29826|2450916|13465|1582799|2886|39420|13465|1582799|2886|39420|4|70|10|2|17551|256|1425|13|65.18|168.81|106.35|811.98|1382.55|847.34|2194.53|110.60|0.00|65.78|1382.55|1493.15|1448.33|1558.93|535.21| +2450842|29826|2450845|13465|1582799|2886|39420|13465|1582799|2886|39420|4|26|14|2|8288|50|1425|77|38.61|99.61|74.70|1918.07|5751.90|2972.97|7669.97|317.50|1783.08|3297.91|3968.82|4286.32|7266.73|7584.23|995.85| +2450842|12273|2450913|91712|1731538|1764|39590|91712|1731538|1764|39590|2|45|18|5|2686|216|1426|29|32.97|36.59|28.54|233.45|827.66|956.13|1061.11|16.55|0.00|63.51|827.66|844.21|891.17|907.72|-128.47| +2450842|12273|2450916|91712|1731538|1764|39590|91712|1731538|1764|39590|2|20|9|3|146|262|1426|16|21.60|27.00|14.85|194.40|237.60|345.60|432.00|0.71|201.96|194.40|35.64|36.35|230.04|230.75|-309.96| +2450842|12273|2450905|91712|1731538|1764|39590|91712|1731538|1764|39590|2|58|20|2|9145|34|1426|54|4.95|7.27|1.52|310.50|82.08|267.30|392.58|1.31|65.66|38.88|16.42|17.73|55.30|56.61|-250.88| +2450842|12273|2450867|91712|1731538|1764|39590|91712|1731538|1764|39590|2|4|11|4|12112|235|1426|87|26.38|57.50|33.92|2051.46|2951.04|2295.06|5002.50|265.59|0.00|300.15|2951.04|3216.63|3251.19|3516.78|655.98| +2450842|12273|2450895|91712|1731538|1764|39590|91712|1731538|1764|39590|2|46|19|3|1408|241|1426|79|52.04|87.42|52.45|2762.63|4143.55|4111.16|6906.18|0.00|0.00|2623.59|4143.55|4143.55|6767.14|6767.14|32.39| +2450842|12273|2450929|91712|1731538|1764|39590|91712|1731538|1764|39590|2|69|12|4|15650|18|1426|62|69.39|103.39|29.98|4551.42|1858.76|4302.18|6410.18|0.00|0.00|2627.56|1858.76|1858.76|4486.32|4486.32|-2443.42| +2450842|12273|2450916|91712|1731538|1764|39590|91712|1731538|1764|39590|2|90|16|4|4501|114|1426|75|46.02|71.33|47.79|1765.50|3584.25|3451.50|5349.75|71.68|0.00|1497.75|3584.25|3655.93|5082.00|5153.68|132.75| +2450842|12273|2450885|91712|1731538|1764|39590|91712|1731538|1764|39590|2|54|10|4|4327|290|1426|56|39.06|78.12|28.12|2800.00|1574.72|2187.36|4374.72|47.24|0.00|1224.72|1574.72|1621.96|2799.44|2846.68|-612.64| +2450842|12273|2450857|91712|1731538|1764|39590|91712|1731538|1764|39590|2|7|5|4|13868|35|1426|13|28.19|28.75|14.95|179.40|194.35|366.47|373.75|5.83|0.00|14.95|194.35|200.18|209.30|215.13|-172.12| +2450842|12273|2450884|91712|1731538|1764|39590|91712|1731538|1764|39590|2|57|3|1|17648|102|1426|17|22.56|30.68|6.74|406.98|114.58|383.52|521.56|3.43|0.00|146.03|114.58|118.01|260.61|264.04|-268.94| +2450842|60600|2450846|54322|171714|6706|33298|54322|171714|6706|33298|1|6|5|3|925|292|1427|90|89.84|183.27|131.95|4618.80|11875.50|8085.60|16494.30|0.00|0.00|7751.70|11875.50|11875.50|19627.20|19627.20|3789.90| +2450842|60600|2450846|54322|171714|6706|33298|54322|171714|6706|33298|1|74|17|5|4867|296|1427|59|67.32|201.28|187.19|831.31|11044.21|3971.88|11875.52|331.32|0.00|2137.57|11044.21|11375.53|13181.78|13513.10|7072.33| +2450842|60600|2450852|54322|171714|6706|33298|54322|171714|6706|33298|1|1|12|1|13232|111|1427|94|99.88|282.66|226.12|5314.76|21255.28|9388.72|26570.04|425.10|0.00|9564.50|21255.28|21680.38|30819.78|31244.88|11866.56| +2450842|60600|2450878|54322|171714|6706|33298|54322|171714|6706|33298|1|57|2|1|2798|66|1427|71|28.88|76.82|37.64|2781.78|2672.44|2050.48|5454.22|187.07|0.00|926.55|2672.44|2859.51|3598.99|3786.06|621.96| +2450842|60600|2450868|54322|171714|6706|33298|54322|171714|6706|33298|1|76|16|1|8054|41|1427|47|21.92|56.11|2.24|2531.89|105.28|1030.24|2637.17|0.54|50.53|0.00|54.75|55.29|54.75|55.29|-975.49| +2450842|60600|2450871|54322|171714|6706|33298|54322|171714|6706|33298|1|95|15|1|12355|268|1427|23|7.17|20.29|12.98|168.13|298.54|164.91|466.67|20.89|0.00|200.56|298.54|319.43|499.10|519.99|133.63| +2450842|60600|2450844|54322|171714|6706|33298|54322|171714|6706|33298|1|83|16|5|11090|220|1427|5|67.68|93.39|2.80|452.95|14.00|338.40|466.95|0.84|0.00|0.00|14.00|14.84|14.00|14.84|-324.40| +2450842|60600|2450872|54322|171714|6706|33298|54322|171714|6706|33298|1|64|1|2|12002|274|1427|1|34.97|59.44|47.55|11.89|47.55|34.97|59.44|3.80|0.00|7.13|47.55|51.35|54.68|58.48|12.58| +2450842|60600|2450928|54322|171714|6706|33298|54322|171714|6706|33298|1|108|2|2|15361|188|1427|18|88.59|251.59|244.04|135.90|4392.72|1594.62|4528.62|175.70|0.00|769.86|4392.72|4568.42|5162.58|5338.28|2798.10| +2450842|60600|2450852|54322|171714|6706|33298|54322|171714|6706|33298|1|63|14|3|6370|263|1427|85|39.92|100.59|2.01|8379.30|170.85|3393.20|8550.15|6.83|0.00|3077.85|170.85|177.68|3248.70|3255.53|-3222.35| +2450842|60600|2450902|54322|171714|6706|33298|54322|171714|6706|33298|1|58|2|4|15926|176|1427|19|54.18|115.40|54.23|1162.23|1030.37|1029.42|2192.60|1.54|999.45|657.78|30.92|32.46|688.70|690.24|-998.50| +2450842|25194|2450896|48262|1345331|800|30269|48262|1345331|800|30269|4|103|16|3|10546|53|1428|49|18.13|37.71|13.57|1182.86|664.93|888.37|1847.79|26.59|0.00|572.81|664.93|691.52|1237.74|1264.33|-223.44| +2450842|25194|2450919|48262|1345331|800|30269|48262|1345331|800|30269|4|36|2|4|13802|254|1428|11|78.99|208.53|125.11|917.62|1376.21|868.89|2293.83|0.00|0.00|688.05|1376.21|1376.21|2064.26|2064.26|507.32| +2450842|25194|2450868|48262|1345331|800|30269|48262|1345331|800|30269|4|71|17|1|16777|102|1428|52|40.14|70.64|45.20|1322.88|2350.40|2087.28|3673.28|23.50|0.00|1248.52|2350.40|2373.90|3598.92|3622.42|263.12| +2450842|25194|2450879|48262|1345331|800|30269|48262|1345331|800|30269|4|102|10|1|11341|98|1428|77|40.29|60.43|26.58|2606.45|2046.66|3102.33|4653.11|6.54|1882.92|139.37|163.74|170.28|303.11|309.65|-2938.59| +2450842|25194|2450879|48262|1345331|800|30269|48262|1345331|800|30269|4|10|5|2|15622|249|1428|66|19.07|26.31|6.05|1337.16|399.30|1258.62|1736.46|23.95|0.00|364.32|399.30|423.25|763.62|787.57|-859.32| +2450842|25194|2450845|48262|1345331|800|30269|48262|1345331|800|30269|4|78|18|3|4879|18|1428|10|3.05|7.68|7.52|1.60|75.20|30.50|76.80|4.51|0.00|21.50|75.20|79.71|96.70|101.21|44.70| +2450842|25194|2450900|48262|1345331|800|30269|48262|1345331|800|30269|4|2|7|5|5048|155|1428|13|22.23|23.78|10.46|173.16|135.98|288.99|309.14|0.40|95.18|58.63|40.80|41.20|99.43|99.83|-248.19| +2450842|25194|2450899|48262|1345331|800|30269|48262|1345331|800|30269|4|29|14|1|17185|198|1428|79|30.16|76.90|3.07|5832.57|242.53|2382.64|6075.10|14.55|0.00|425.02|242.53|257.08|667.55|682.10|-2140.11| +2450842|25194|2450889|48262|1345331|800|30269|48262|1345331|800|30269|4|9|17|5|6709|108|1428|3|40.74|95.73|70.84|74.67|212.52|122.22|287.19|6.37|0.00|0.00|212.52|218.89|212.52|218.89|90.30| +2450842|25194|2450885|48262|1345331|800|30269|48262|1345331|800|30269|4|26|18|2|17672|40|1428|99|64.85|92.73|39.87|5233.14|3947.13|6420.15|9180.27|157.88|0.00|3212.55|3947.13|4105.01|7159.68|7317.56|-2473.02| +2450842|25194|2450853|48262|1345331|800|30269|48262|1345331|800|30269|4|72|1|2|14017|86|1428|98|97.42|252.31|204.37|4698.12|20028.26|9547.16|24726.38|1802.54|0.00|11125.94|20028.26|21830.80|31154.20|32956.74|10481.10| +2450842|25194|2450859|48262|1345331|800|30269|48262|1345331|800|30269|4|26|18|4|8959|298|1428|86|38.75|61.61|35.73|2225.68|3072.78|3332.50|5298.46|61.45|0.00|1165.30|3072.78|3134.23|4238.08|4299.53|-259.72| +2450842|25194|2450854|48262|1345331|800|30269|48262|1345331|800|30269|4|14|9|2|12734|90|1428|28|7.15|11.86|0.35|322.28|9.80|200.20|332.08|0.78|0.00|23.24|9.80|10.58|33.04|33.82|-190.40| +2450842|59276|2450848|89602|689828|2054|11527|89602|689828|2054|11527|4|23|3|1|10466|119|1429|69|78.57|113.92|21.64|6367.32|1493.16|5421.33|7860.48|119.45|0.00|1021.20|1493.16|1612.61|2514.36|2633.81|-3928.17| +2450842|59276|2450878|89602|689828|2054|11527|89602|689828|2054|11527|4|58|13|2|9682|52|1429|98|64.25|145.84|42.29|10147.90|4144.42|6296.50|14292.32|248.66|0.00|2572.50|4144.42|4393.08|6716.92|6965.58|-2152.08| +2450842|59276|2450865|89602|689828|2054|11527|89602|689828|2054|11527|4|14|10|2|34|21|1429|96|65.64|185.10|5.55|17236.80|532.80|6301.44|17769.60|37.29|0.00|177.60|532.80|570.09|710.40|747.69|-5768.64| +2450842|59276|2450892|89602|689828|2054|11527|89602|689828|2054|11527|4|79|14|1|16171|218|1429|38|79.34|238.02|130.91|4070.18|4974.58|3014.92|9044.76|99.49|0.00|4160.24|4974.58|5074.07|9134.82|9234.31|1959.66| +2450842|59276|2450871|89602|689828|2054|11527|89602|689828|2054|11527|4|15|16|4|16142|105|1429|91|49.84|128.08|6.40|11072.88|582.40|4535.44|11655.28|0.00|0.00|3962.14|582.40|582.40|4544.54|4544.54|-3953.04| +2450843|4779|2450898|48801|1014147|4293|37686|48801|1014147|4293|37686|4|18|11|5|649|241|1430|72|23.58|35.84|10.03|1858.32|722.16|1697.76|2580.48|36.10|0.00|154.80|722.16|758.26|876.96|913.06|-975.60| +2450843|4779|2450906|48801|1014147|4293|37686|48801|1014147|4293|37686|4|11|10|5|13790|237|1430|54|67.07|123.40|55.53|3664.98|2998.62|3621.78|6663.60|29.98|0.00|1199.34|2998.62|3028.60|4197.96|4227.94|-623.16| +2450843|4779|2450915|48801|1014147|4293|37686|48801|1014147|4293|37686|4|93|11|1|17866|211|1430|80|10.89|15.79|10.10|455.20|808.00|871.20|1263.20|72.72|0.00|24.80|808.00|880.72|832.80|905.52|-63.20| +2450843|4779|2450887|48801|1014147|4293|37686|48801|1014147|4293|37686|4|17|20|3|14938|207|1430|74|15.90|32.91|29.28|268.62|2166.72|1176.60|2435.34|0.00|0.00|364.82|2166.72|2166.72|2531.54|2531.54|990.12| +2450843|4779|2450851|48801|1014147|4293|37686|48801|1014147|4293|37686|4|94|16|5|13489|233|1430|76|63.20|71.41|27.13|3365.28|2061.88|4803.20|5427.16|144.33|0.00|379.24|2061.88|2206.21|2441.12|2585.45|-2741.32| +2450843|4779|2450925|48801|1014147|4293|37686|48801|1014147|4293|37686|4|40|14|1|265|105|1430|87|85.15|153.27|18.39|11734.56|1599.93|7408.05|13334.49|79.99|0.00|1733.04|1599.93|1679.92|3332.97|3412.96|-5808.12| +2450843|4779|2450882|48801|1014147|4293|37686|48801|1014147|4293|37686|4|33|3|2|2240|55|1430|35|75.86|160.82|46.63|3996.65|1632.05|2655.10|5628.70|32.64|0.00|2588.95|1632.05|1664.69|4221.00|4253.64|-1023.05| +2450843|4779|2450894|48801|1014147|4293|37686|48801|1014147|4293|37686|4|75|12|5|14503|95|1430|55|29.54|65.87|9.22|3115.75|507.10|1624.70|3622.85|5.07|0.00|0.00|507.10|512.17|507.10|512.17|-1117.60| +2450843|4779|2450853|48801|1014147|4293|37686|48801|1014147|4293|37686|4|60|5|1|14428|121|1430|26|21.42|48.83|46.87|50.96|1218.62|556.92|1269.58|0.00|353.39|203.06|865.23|865.23|1068.29|1068.29|308.31| +2450843|4779|2450922|48801|1014147|4293|37686|48801|1014147|4293|37686|4|41|18|3|14342|264|1430|64|10.85|19.63|1.96|1130.88|125.44|694.40|1256.32|5.01|0.00|339.20|125.44|130.45|464.64|469.65|-568.96| +2450843|4779|2450893|48801|1014147|4293|37686|48801|1014147|4293|37686|4|99|9|4|13498|267|1430|23|99.18|129.92|103.93|597.77|2390.39|2281.14|2988.16|119.51|0.00|1195.08|2390.39|2509.90|3585.47|3704.98|109.25| +2450843|4779|2450865|48801|1014147|4293|37686|48801|1014147|4293|37686|4|22|16|2|67|231|1430|4|11.64|33.98|27.86|24.48|111.44|46.56|135.92|4.45|0.00|58.44|111.44|115.89|169.88|174.33|64.88| +2450843|34368|2450874|76932|1270083|834|45538|76932|1270083|834|45538|4|8|3|3|11710|140|1431|73|98.48|153.62|110.60|3140.46|8073.80|7189.04|11214.26|0.00|0.00|2915.62|8073.80|8073.80|10989.42|10989.42|884.76| +2450843|34368|2450848|76932|1270083|834|45538|76932|1270083|834|45538|4|49|4|2|12697|221|1431|100|37.86|97.30|8.75|8855.00|875.00|3786.00|9730.00|2.10|805.00|486.00|70.00|72.10|556.00|558.10|-3716.00| +2450843|34368|2450866|76932|1270083|834|45538|76932|1270083|834|45538|4|23|18|2|7948|190|1431|8|52.53|136.05|95.23|326.56|761.84|420.24|1088.40|1.14|647.56|97.92|114.28|115.42|212.20|213.34|-305.96| +2450843|34368|2450893|76932|1270083|834|45538|76932|1270083|834|45538|4|21|1|3|5878|239|1431|34|48.31|87.44|72.57|505.58|2467.38|1642.54|2972.96|74.02|0.00|445.74|2467.38|2541.40|2913.12|2987.14|824.84| +2450843|34368|2450923|76932|1270083|834|45538|76932|1270083|834|45538|4|66|12|2|5828|47|1431|4|99.72|106.70|55.48|204.88|221.92|398.88|426.80|4.43|0.00|102.40|221.92|226.35|324.32|328.75|-176.96| +2450843|34368|2450904|76932|1270083|834|45538|76932|1270083|834|45538|4|18|12|3|1156|280|1431|16|33.84|84.26|24.43|957.28|390.88|541.44|1348.16|7.81|0.00|309.92|390.88|398.69|700.80|708.61|-150.56| +2450843|34368|2450892|76932|1270083|834|45538|76932|1270083|834|45538|4|97|9|3|10814|191|1431|75|17.39|47.82|0.00|3586.50|0.00|1304.25|3586.50|0.00|0.00|1649.25|0.00|0.00|1649.25|1649.25|-1304.25| +2450843|34368|2450882|76932|1270083|834|45538|76932|1270083|834|45538|4|75|13|4|15844|17|1431|57|53.86|143.26|68.76|4246.50|3919.32|3070.02|8165.82|39.19|0.00|1224.36|3919.32|3958.51|5143.68|5182.87|849.30| +2450843|34368|2450869|76932|1270083|834|45538|76932|1270083|834|45538|4|6|18|1|16684|39|1431|31|1.63|2.18|0.89|39.99|27.59|50.53|67.58|0.00|0.00|16.12|27.59|27.59|43.71|43.71|-22.94| +2450843|47150|2450849|6224|1120096|3215|31887|6224|1120096|3215|31887|4|107|9|3|10052|23|1432|93|10.27|13.96|6.56|688.20|610.08|955.11|1298.28|17.57|170.82|129.27|439.26|456.83|568.53|586.10|-515.85| +2450843|47150|2450933|6224|1120096|3215|31887|6224|1120096|3215|31887|4|80|16|1|15884|17|1432|84|43.19|52.69|27.92|2080.68|2345.28|3627.96|4425.96|164.16|0.00|1504.44|2345.28|2509.44|3849.72|4013.88|-1282.68| +2450843|47150|2450868|6224|1120096|3215|31887|6224|1120096|3215|31887|4|51|12|4|3800|51|1432|80|41.31|78.07|21.85|4497.60|1748.00|3304.80|6245.60|157.32|0.00|1873.60|1748.00|1905.32|3621.60|3778.92|-1556.80| +2450843|47150|2450896|6224|1120096|3215|31887|6224|1120096|3215|31887|4|58|19|1|7366|270|1432|94|25.18|38.27|34.06|395.74|3201.64|2366.92|3597.38|224.11|0.00|359.08|3201.64|3425.75|3560.72|3784.83|834.72| +2450843|47150|2450866|6224|1120096|3215|31887|6224|1120096|3215|31887|4|62|1|3|2950|135|1432|88|12.10|29.52|14.46|1325.28|1272.48|1064.80|2597.76|50.89|0.00|129.36|1272.48|1323.37|1401.84|1452.73|207.68| +2450843|47150|2450933|6224|1120096|3215|31887|6224|1120096|3215|31887|4|28|1|2|2779|267|1432|19|8.56|17.12|11.98|97.66|227.62|162.64|325.28|20.48|0.00|123.50|227.62|248.10|351.12|371.60|64.98| +2450843|47150|2450933|6224|1120096|3215|31887|6224|1120096|3215|31887|4|75|9|3|15338|56|1432|19|46.95|94.36|46.23|914.47|878.37|892.05|1792.84|17.56|0.00|717.06|878.37|895.93|1595.43|1612.99|-13.68| +2450843|47150|2450881|6224|1120096|3215|31887|6224|1120096|3215|31887|4|8|19|1|7141|57|1432|19|26.16|41.07|25.87|288.80|491.53|497.04|780.33|9.83|0.00|210.52|491.53|501.36|702.05|711.88|-5.51| +2450843|47150|2450882|6224|1120096|3215|31887|6224|1120096|3215|31887|4|90|17|2|14365|235|1432|13|73.10|149.12|38.77|1434.55|504.01|950.30|1938.56|5.04|0.00|232.57|504.01|509.05|736.58|741.62|-446.29| +2450843|54329|2450930|66381|23997|3566|47182|66381|23997|3566|47182|2|62|17|5|13214|216|1433|91|87.75|250.96|105.40|13245.96|9591.40|7985.25|22837.36|140.99|7577.20|10961.86|2014.20|2155.19|12976.06|13117.05|-5971.05| +2450843|54329|2450877|66381|23997|3566|47182|66381|23997|3566|47182|2|31|19|1|14359|17|1433|8|97.87|288.71|124.14|1316.56|993.12|782.96|2309.68|0.00|506.49|161.60|486.63|486.63|648.23|648.23|-296.33| +2450843|54329|2450909|66381|23997|3566|47182|66381|23997|3566|47182|2|8|5|5|7154|131|1433|91|19.01|27.37|1.91|2316.86|173.81|1729.91|2490.67|10.42|0.00|273.91|173.81|184.23|447.72|458.14|-1556.10| +2450843|54329|2450889|66381|23997|3566|47182|66381|23997|3566|47182|2|71|17|2|5317|196|1433|16|16.75|33.66|3.70|479.36|59.20|268.00|538.56|5.32|0.00|231.52|59.20|64.52|290.72|296.04|-208.80| +2450843|54329|2450930|66381|23997|3566|47182|66381|23997|3566|47182|2|40|7|2|16840|215|1433|65|35.02|52.88|22.73|1959.75|1477.45|2276.30|3437.20|0.00|0.00|996.45|1477.45|1477.45|2473.90|2473.90|-798.85| +2450843|54329|2450927|66381|23997|3566|47182|66381|23997|3566|47182|2|51|18|1|658|112|1433|71|5.47|15.42|14.95|33.37|1061.45|388.37|1094.82|0.00|902.23|218.68|159.22|159.22|377.90|377.90|-229.15| +2450843|54329|2450854|66381|23997|3566|47182|66381|23997|3566|47182|2|15|1|2|6376|14|1433|76|53.12|109.42|62.36|3576.56|4739.36|4037.12|8315.92|379.14|0.00|2494.32|4739.36|5118.50|7233.68|7612.82|702.24| +2450843|54329|2450901|66381|23997|3566|47182|66381|23997|3566|47182|2|43|16|3|9116|162|1433|31|19.14|53.20|9.57|1352.53|296.67|593.34|1649.20|23.73|0.00|0.00|296.67|320.40|296.67|320.40|-296.67| +2450843|76698|2450915|63790|135934|6478|48622|63790|135934|6478|48622|1|80|16|3|2569|277|1434|99|32.96|39.88|27.51|1224.63|2723.49|3263.04|3948.12|54.46|0.00|670.23|2723.49|2777.95|3393.72|3448.18|-539.55| +2450843|76698|2450872|63790|135934|6478|48622|63790|135934|6478|48622|1|72|19|2|4762|78|1434|8|11.15|17.61|16.02|12.72|128.16|89.20|140.88|3.03|26.91|9.84|101.25|104.28|111.09|114.12|12.05| +2450843|76698|2450910|63790|135934|6478|48622|63790|135934|6478|48622|1|60|16|1|15406|135|1434|13|14.69|40.39|12.11|367.64|157.43|190.97|525.07|6.61|25.18|231.01|132.25|138.86|363.26|369.87|-58.72| +2450843|76698|2450868|63790|135934|6478|48622|63790|135934|6478|48622|1|100|18|2|6206|12|1434|90|89.11|237.92|237.92|0.00|21412.80|8019.90|21412.80|1070.64|0.00|8564.40|21412.80|22483.44|29977.20|31047.84|13392.90| +2450843|76698|2450890|63790|135934|6478|48622|63790|135934|6478|48622|1|91|11|5|14815|181|1434|60|77.86|140.92|38.04|6172.80|2282.40|4671.60|8455.20|205.41|0.00|2790.00|2282.40|2487.81|5072.40|5277.81|-2389.20| +2450843|76698|2450878|63790|135934|6478|48622|63790|135934|6478|48622|1|81|20|3|16628|246|1434|27|33.24|46.53|1.86|1206.09|50.22|897.48|1256.31|1.00|0.00|125.55|50.22|51.22|175.77|176.77|-847.26| +2450843|76698|2450921|63790|135934|6478|48622|63790|135934|6478|48622|1|65|1|4|12314|14|1434|89|51.25|111.72|52.50|5270.58|4672.50|4561.25|9943.08|0.00|0.00|1888.58|4672.50|4672.50|6561.08|6561.08|111.25| +2450843|76698|2450855|63790|135934|6478|48622|63790|135934|6478|48622|1|4|16|4|3754|218|1434|11|27.76|37.75|3.02|382.03|33.22|305.36|415.25|2.32|0.00|120.34|33.22|35.54|153.56|155.88|-272.14| +2450843|76698|2450848|63790|135934|6478|48622|63790|135934|6478|48622|1|57|19|2|14785|281|1434|98|14.31|38.92|12.06|2632.28|1181.88|1402.38|3814.16|2.24|957.32|533.12|224.56|226.80|757.68|759.92|-1177.82| +2450843|13217|2450910|93218|463296|3327|47758|93218|463296|3327|47758|1|35|15|1|14690|244|1435|70|40.63|49.97|17.48|2274.30|1223.60|2844.10|3497.90|0.00|1223.60|1748.60|0.00|0.00|1748.60|1748.60|-2844.10| +2450843|13217|2450930|93218|463296|3327|47758|93218|463296|3327|47758|1|67|4|5|3598|240|1435|82|97.93|146.89|80.78|5421.02|6623.96|8030.26|12044.98|0.00|2914.54|3252.12|3709.42|3709.42|6961.54|6961.54|-4320.84| +2450843|13217|2450912|93218|463296|3327|47758|93218|463296|3327|47758|1|12|15|3|7936|276|1435|18|80.12|233.95|161.42|1305.54|2905.56|1442.16|4211.10|232.44|0.00|757.98|2905.56|3138.00|3663.54|3895.98|1463.40| +2450843|13217|2450892|93218|463296|3327|47758|93218|463296|3327|47758|1|105|18|2|14995|73|1435|25|86.29|155.32|147.55|194.25|3688.75|2157.25|3883.00|184.43|0.00|1281.25|3688.75|3873.18|4970.00|5154.43|1531.50| +2450843|5409|2450893|99925|847142|2359|17129|99925|847142|2359|17129|4|59|3|5|9001|260|1436|96|54.46|148.13|102.20|4409.28|9811.20|5228.16|14220.48|883.00|0.00|1421.76|9811.20|10694.20|11232.96|12115.96|4583.04| +2450843|5409|2450932|99925|847142|2359|17129|99925|847142|2359|17129|4|81|19|4|139|74|1436|73|4.22|4.93|4.68|18.25|341.64|308.06|359.89|6.76|228.89|53.29|112.75|119.51|166.04|172.80|-195.31| +2450843|5409|2450902|99925|847142|2359|17129|99925|847142|2359|17129|4|27|5|1|13393|216|1436|96|25.78|27.32|17.48|944.64|1678.08|2474.88|2622.72|83.90|0.00|655.68|1678.08|1761.98|2333.76|2417.66|-796.80| +2450843|5409|2450895|99925|847142|2359|17129|99925|847142|2359|17129|4|107|16|5|5578|94|1436|20|16.68|47.70|32.91|295.80|658.20|333.60|954.00|39.49|0.00|295.60|658.20|697.69|953.80|993.29|324.60| +2450843|5409|2450925|99925|847142|2359|17129|99925|847142|2359|17129|4|9|2|4|11062|128|1436|43|34.48|48.61|39.86|376.25|1713.98|1482.64|2090.23|137.11|0.00|375.82|1713.98|1851.09|2089.80|2226.91|231.34| +2450843|5409|2450911|99925|847142|2359|17129|99925|847142|2359|17129|4|64|15|5|15194|71|1436|92|30.30|78.17|10.94|6185.16|1006.48|2787.60|7191.64|40.25|0.00|1437.96|1006.48|1046.73|2444.44|2484.69|-1781.12| +2450843|5409|2450855|99925|847142|2359|17129|99925|847142|2359|17129|4|2|5|5|16759|128|1436|71|58.69|72.77|54.57|1292.20|3874.47|4166.99|5166.67|271.21|0.00|878.27|3874.47|4145.68|4752.74|5023.95|-292.52| +2450843|5409|2450870|99925|847142|2359|17129|99925|847142|2359|17129|4|44|16|2|15244|122|1436|91|17.12|50.50|28.28|2022.02|2573.48|1557.92|4595.50|231.61|0.00|229.32|2573.48|2805.09|2802.80|3034.41|1015.56| +2450843|57722|2450913|75788|310469|2750|3384|75788|310469|2750|3384|4|86|17|4|847|203|1437|27|86.05|235.77|122.60|3055.59|3310.20|2323.35|6365.79|165.51|0.00|2291.49|3310.20|3475.71|5601.69|5767.20|986.85| +2450843|57722|2450865|75788|310469|2750|3384|75788|310469|2750|3384|4|40|20|2|16897|70|1437|29|56.24|145.66|93.22|1520.76|2703.38|1630.96|4224.14|27.03|0.00|929.16|2703.38|2730.41|3632.54|3659.57|1072.42| +2450843|57722|2450889|75788|310469|2750|3384|75788|310469|2750|3384|4|30|20|3|61|121|1437|73|24.66|51.78|41.94|718.32|3061.62|1800.18|3779.94|214.31|0.00|1662.94|3061.62|3275.93|4724.56|4938.87|1261.44| +2450843|57722|2450855|75788|310469|2750|3384|75788|310469|2750|3384|4|36|19|3|9788|165|1437|57|75.33|119.77|71.86|2730.87|4096.02|4293.81|6826.89|163.84|0.00|1569.78|4096.02|4259.86|5665.80|5829.64|-197.79| +2450843|57722|2450873|75788|310469|2750|3384|75788|310469|2750|3384|4|18|4|2|5140|83|1437|98|67.44|118.02|21.24|9484.44|2081.52|6609.12|11565.96|8.74|1956.62|1965.88|124.90|133.64|2090.78|2099.52|-6484.22| +2450843|57722|2450890|75788|310469|2750|3384|75788|310469|2750|3384|4|98|7|4|16630|172|1437|21|18.96|25.59|1.02|515.97|21.42|398.16|537.39|0.93|2.78|112.77|18.64|19.57|131.41|132.34|-379.52| +2450843|57722|2450885|75788|310469|2750|3384|75788|310469|2750|3384|4|52|2|5|1741|207|1437|17|99.41|116.30|4.65|1898.05|79.05|1689.97|1977.10|2.56|27.66|138.38|51.39|53.95|189.77|192.33|-1638.58| +2450843|57722|2450861|75788|310469|2750|3384|75788|310469|2750|3384|4|79|4|1|16513|23|1437|65|21.96|63.90|31.31|2118.35|2035.15|1427.40|4153.50|162.81|0.00|913.25|2035.15|2197.96|2948.40|3111.21|607.75| +2450843|57722|2450860|75788|310469|2750|3384|75788|310469|2750|3384|4|28|10|4|9355|49|1437|74|38.24|102.10|90.86|831.76|6723.64|2829.76|7555.40|268.94|0.00|1057.46|6723.64|6992.58|7781.10|8050.04|3893.88| +2450843|57722|2450903|75788|310469|2750|3384|75788|310469|2750|3384|4|62|7|4|16892|243|1437|32|88.05|238.61|202.81|1145.60|6489.92|2817.60|7635.52|389.39|0.00|2290.56|6489.92|6879.31|8780.48|9169.87|3672.32| +2450843|57722|2450855|75788|310469|2750|3384|75788|310469|2750|3384|4|17|18|4|14479|238|1437|28|9.38|15.94|4.14|330.40|115.92|262.64|446.32|10.43|0.00|40.04|115.92|126.35|155.96|166.39|-146.72| +2450843|70320|2450861|77542|1147585|1628|39718|77542|1147585|1628|39718|2|17|16|1|5380|277|1438|54|39.71|79.42|77.03|129.06|4159.62|2144.34|4288.68|0.00|0.00|1458.00|4159.62|4159.62|5617.62|5617.62|2015.28| +2450843|70320|2450884|77542|1147585|1628|39718|77542|1147585|1628|39718|2|89|20|5|12368|275|1438|28|67.73|190.32|0.00|5328.96|0.00|1896.44|5328.96|0.00|0.00|639.24|0.00|0.00|639.24|639.24|-1896.44| +2450843|70320|2450869|77542|1147585|1628|39718|77542|1147585|1628|39718|2|35|13|2|8588|200|1438|15|48.50|131.43|11.82|1794.15|177.30|727.50|1971.45|14.18|0.00|768.75|177.30|191.48|946.05|960.23|-550.20| +2450843|70320|2450933|77542|1147585|1628|39718|77542|1147585|1628|39718|2|5|20|4|22|160|1438|11|36.15|61.45|35.64|283.91|392.04|397.65|675.95|23.52|0.00|114.84|392.04|415.56|506.88|530.40|-5.61| +2450843|70320|2450887|77542|1147585|1628|39718|77542|1147585|1628|39718|2|74|13|3|5245|233|1438|40|40.49|76.12|35.01|1644.40|1400.40|1619.60|3044.80|18.48|784.22|212.80|616.18|634.66|828.98|847.46|-1003.42| +2450843|70320|2450908|77542|1147585|1628|39718|77542|1147585|1628|39718|2|80|8|1|2750|234|1438|17|94.78|143.11|44.36|1678.75|754.12|1611.26|2432.87|30.16|0.00|170.17|754.12|784.28|924.29|954.45|-857.14| +2450843|70320|2450865|77542|1147585|1628|39718|77542|1147585|1628|39718|2|27|4|2|10808|201|1438|73|41.20|50.67|0.50|3662.41|36.50|3007.60|3698.91|1.09|0.00|591.30|36.50|37.59|627.80|628.89|-2971.10| +2450843|70320|2450924|77542|1147585|1628|39718|77542|1147585|1628|39718|2|5|2|5|11524|280|1438|64|32.95|78.42|43.91|2208.64|2810.24|2108.80|5018.88|196.71|0.00|2308.48|2810.24|3006.95|5118.72|5315.43|701.44| +2450843|70320|2450862|77542|1147585|1628|39718|77542|1147585|1628|39718|2|53|1|1|9640|23|1438|33|16.17|33.63|28.24|177.87|931.92|533.61|1109.79|1.49|857.36|232.98|74.56|76.05|307.54|309.03|-459.05| +2450843|70320|2450870|77542|1147585|1628|39718|77542|1147585|1628|39718|2|94|1|3|10696|62|1438|33|57.14|119.42|11.94|3546.84|394.02|1885.62|3940.86|3.94|0.00|788.04|394.02|397.96|1182.06|1186.00|-1491.60| +2450843|70320|2450853|77542|1147585|1628|39718|77542|1147585|1628|39718|2|75|4|2|5540|75|1438|23|36.83|69.24|0.00|1592.52|0.00|847.09|1592.52|0.00|0.00|716.45|0.00|0.00|716.45|716.45|-847.09| +2450843|70320|2450933|77542|1147585|1628|39718|77542|1147585|1628|39718|2|97|1|4|7444|127|1438|8|4.49|9.47|2.46|56.08|19.68|35.92|75.76|0.39|0.00|6.80|19.68|20.07|26.48|26.87|-16.24| +2450843|73664|2450863|55707|1298790|3473|5756|55707|1298790|3473|5756|1|32|16|5|224|111|1439|86|94.41|243.57|224.08|1676.14|19270.88|8119.26|20947.02|1156.25|0.00|208.98|19270.88|20427.13|19479.86|20636.11|11151.62| +2450843|73664|2450893|55707|1298790|3473|5756|55707|1298790|3473|5756|1|49|2|4|10282|120|1439|70|39.52|67.97|7.47|4235.00|522.90|2766.40|4757.90|2.56|266.67|856.10|256.23|258.79|1112.33|1114.89|-2510.17| +2450843|73664|2450900|55707|1298790|3473|5756|55707|1298790|3473|5756|1|22|4|2|13784|183|1439|79|31.86|76.46|12.23|5074.17|966.17|2516.94|6040.34|38.64|0.00|361.82|966.17|1004.81|1327.99|1366.63|-1550.77| +2450843|73664|2450846|55707|1298790|3473|5756|55707|1298790|3473|5756|1|94|20|4|8056|91|1439|53|25.98|58.45|22.21|1920.72|1177.13|1376.94|3097.85|47.08|0.00|61.48|1177.13|1224.21|1238.61|1285.69|-199.81| +2450843|73664|2450851|55707|1298790|3473|5756|55707|1298790|3473|5756|1|106|19|5|11420|279|1439|18|11.31|24.99|5.24|355.50|94.32|203.58|449.82|6.60|0.00|4.32|94.32|100.92|98.64|105.24|-109.26| +2450843|73664|2450847|55707|1298790|3473|5756|55707|1298790|3473|5756|1|1|10|4|7747|244|1439|53|68.59|190.68|131.56|3133.36|6972.68|3635.27|10106.04|69.72|0.00|2222.82|6972.68|7042.40|9195.50|9265.22|3337.41| +2450843|50760|2450865|42799|1105785|6818|29004|35393|793376|1962|15748|2|96|13|3|17348|258|1440|60|18.32|32.97|2.63|1820.40|157.80|1099.20|1978.20|7.89|0.00|850.20|157.80|165.69|1008.00|1015.89|-941.40| +2450843|50760|2450898|42799|1105785|6818|29004|35393|793376|1962|15748|2|102|2|4|4498|283|1440|77|2.54|5.71|2.56|242.55|197.12|195.58|439.67|11.82|0.00|113.96|197.12|208.94|311.08|322.90|1.54| +2450843|50760|2450888|42799|1105785|6818|29004|35393|793376|1962|15748|2|86|15|1|15343|113|1440|47|12.28|13.38|11.50|88.36|540.50|577.16|628.86|21.62|270.25|138.18|270.25|291.87|408.43|430.05|-306.91| +2450843|50760|2450875|42799|1105785|6818|29004|35393|793376|1962|15748|2|78|11|5|17924|6|1440|4|35.84|70.24|44.25|103.96|177.00|143.36|280.96|0.00|0.00|98.32|177.00|177.00|275.32|275.32|33.64| +2450843|50760|2450927|42799|1105785|6818|29004|35393|793376|1962|15748|2|6|7|5|13954|14|1440|63|55.97|153.91|113.89|2521.26|7175.07|3526.11|9696.33|502.25|0.00|3393.18|7175.07|7677.32|10568.25|11070.50|3648.96| +2450843|50760|2450915|42799|1105785|6818|29004|35393|793376|1962|15748|2|71|15|2|17036|21|1440|19|33.86|34.87|13.94|397.67|264.86|643.34|662.53|23.83|0.00|331.17|264.86|288.69|596.03|619.86|-378.48| +2450843|50258|2450857|72692|749155|3218|44481|72692|749155|3218|44481|4|80|1|4|2131|189|1441|43|20.28|24.33|23.35|42.14|1004.05|872.04|1046.19|10.04|0.00|125.13|1004.05|1014.09|1129.18|1139.22|132.01| +2450843|50258|2450891|72692|749155|3218|44481|72692|749155|3218|44481|4|95|13|2|13663|62|1441|47|24.70|33.34|5.00|1331.98|235.00|1160.90|1566.98|0.14|230.30|172.02|4.70|4.84|176.72|176.86|-1156.20| +2450843|50258|2450887|72692|749155|3218|44481|72692|749155|3218|44481|4|82|14|2|3470|116|1441|9|88.91|140.47|8.42|1188.45|75.78|800.19|1264.23|3.03|0.00|594.18|75.78|78.81|669.96|672.99|-724.41| +2450843|50258|2450926|72692|749155|3218|44481|72692|749155|3218|44481|4|18|15|4|5162|164|1441|87|28.05|67.88|8.82|5138.22|767.34|2440.35|5905.56|13.81|306.93|2656.98|460.41|474.22|3117.39|3131.20|-1979.94| +2450843|50258|2450897|72692|749155|3218|44481|72692|749155|3218|44481|4|84|12|3|16672|83|1441|73|87.57|211.91|190.71|1547.60|13921.83|6392.61|15469.43|1252.96|0.00|5413.68|13921.83|15174.79|19335.51|20588.47|7529.22| +2450843|29625|2450875|31566|1509584|6355|29173|31566|1509584|6355|29173|2|99|7|5|10450|180|1442|10|33.01|78.56|54.99|235.70|549.90|330.10|785.60|5.05|423.42|259.20|126.48|131.53|385.68|390.73|-203.62| +2450843|29625|2450890|31566|1509584|6355|29173|31566|1509584|6355|29173|2|47|8|2|2137|162|1442|4|63.67|171.90|139.23|130.68|556.92|254.68|687.60|16.70|0.00|130.64|556.92|573.62|687.56|704.26|302.24| +2450843|29625|2450881|31566|1509584|6355|29173|31566|1509584|6355|29173|2|78|5|1|15442|73|1442|90|97.39|280.48|2.80|24991.20|252.00|8765.10|25243.20|2.52|0.00|756.90|252.00|254.52|1008.90|1011.42|-8513.10| +2450843|29625|2450920|31566|1509584|6355|29173|31566|1509584|6355|29173|2|51|15|2|7810|73|1442|73|98.16|116.81|88.77|2046.92|6480.21|7165.68|8527.13|518.41|0.00|1790.69|6480.21|6998.62|8270.90|8789.31|-685.47| +2450843|29625|2450849|31566|1509584|6355|29173|31566|1509584|6355|29173|2|43|10|2|11218|15|1442|61|79.34|92.82|38.05|3340.97|2321.05|4839.74|5662.02|116.05|0.00|2207.59|2321.05|2437.10|4528.64|4644.69|-2518.69| +2450843|29625|2450903|31566|1509584|6355|29173|31566|1509584|6355|29173|2|30|5|1|15800|215|1442|92|59.03|100.94|22.20|7244.08|2042.40|5430.76|9286.48|122.54|0.00|4550.32|2042.40|2164.94|6592.72|6715.26|-3388.36| +2450843|29625|2450872|31566|1509584|6355|29173|31566|1509584|6355|29173|2|31|12|5|5768|255|1442|59|6.85|8.83|8.47|21.24|499.73|404.15|520.97|44.97|0.00|119.77|499.73|544.70|619.50|664.47|95.58| +2450843|29625|2450854|31566|1509584|6355|29173|31566|1509584|6355|29173|2|95|14|3|14827|190|1442|79|45.81|51.30|25.65|2026.35|2026.35|3618.99|4052.70|162.10|0.00|80.58|2026.35|2188.45|2106.93|2269.03|-1592.64| +2450843|29625|2450889|31566|1509584|6355|29173|31566|1509584|6355|29173|2|37|7|4|12994|80|1442|93|65.93|143.72|41.67|9490.65|3875.31|6131.49|13365.96|9.68|3681.54|1068.57|193.77|203.45|1262.34|1272.02|-5937.72| +2450843|29625|2450850|31566|1509584|6355|29173|31566|1509584|6355|29173|2|104|3|2|2983|220|1442|49|2.00|2.24|0.15|102.41|7.35|98.00|109.76|0.18|1.32|40.18|6.03|6.21|46.21|46.39|-91.97| +2450843|29625|2450912|31566|1509584|6355|29173|31566|1509584|6355|29173|2|83|5|5|12337|53|1442|29|42.41|47.49|17.57|867.68|509.53|1229.89|1377.21|20.38|0.00|0.00|509.53|529.91|509.53|529.91|-720.36| +2450843|29625|2450862|31566|1509584|6355|29173|31566|1509584|6355|29173|2|79|19|2|3332|91|1442|33|61.75|143.26|97.41|1513.05|3214.53|2037.75|4727.58|160.72|0.00|1654.62|3214.53|3375.25|4869.15|5029.87|1176.78| +2450843|37259|2450902|70759|1142853|547|17041|70759|1142853|547|17041|1|45|6|2|15896|192|1443|26|90.79|158.88|9.53|3883.10|247.78|2360.54|4130.88|4.95|0.00|991.38|247.78|252.73|1239.16|1244.11|-2112.76| +2450843|37259|2450920|70759|1142853|547|17041|70759|1142853|547|17041|1|3|7|1|3190|211|1443|23|62.65|167.90|60.44|2471.58|1390.12|1440.95|3861.70|27.80|0.00|1853.57|1390.12|1417.92|3243.69|3271.49|-50.83| +2450843|37259|2450873|70759|1142853|547|17041|70759|1142853|547|17041|1|20|15|4|11479|106|1443|34|53.06|101.34|57.76|1481.72|1963.84|1804.04|3445.56|38.09|58.91|34.34|1904.93|1943.02|1939.27|1977.36|100.89| +2450843|37259|2450887|70759|1142853|547|17041|70759|1142853|547|17041|1|49|4|1|4120|273|1443|1|87.18|202.25|34.38|167.87|34.38|87.18|202.25|0.00|0.00|24.27|34.38|34.38|58.65|58.65|-52.80| +2450843|37259|2450877|70759|1142853|547|17041|70759|1142853|547|17041|1|65|17|5|9212|87|1443|78|99.77|128.70|9.00|9336.60|702.00|7782.06|10038.60|56.16|0.00|1003.86|702.00|758.16|1705.86|1762.02|-7080.06| +2450843|37259|2450881|70759|1142853|547|17041|70759|1142853|547|17041|1|91|18|3|12224|100|1443|77|3.82|10.69|9.83|66.22|756.91|294.14|823.13|22.70|0.00|123.20|756.91|779.61|880.11|902.81|462.77| +2450843|37259|2450915|70759|1142853|547|17041|70759|1142853|547|17041|1|82|19|3|4000|141|1443|16|82.97|155.98|138.82|274.56|2221.12|1327.52|2495.68|111.05|0.00|1222.88|2221.12|2332.17|3444.00|3555.05|893.60| +2450843|37259|2450924|70759|1142853|547|17041|70759|1142853|547|17041|1|68|10|4|12205|113|1443|95|66.78|198.33|146.76|4899.15|13942.20|6344.10|18841.35|0.00|0.00|564.30|13942.20|13942.20|14506.50|14506.50|7598.10| +2450843|37259|2450914|70759|1142853|547|17041|70759|1142853|547|17041|1|60|9|3|5362|214|1443|7|35.29|53.28|22.91|212.59|160.37|247.03|372.96|9.62|0.00|63.35|160.37|169.99|223.72|233.34|-86.66| +2450843|37259|2450857|70759|1142853|547|17041|70759|1142853|547|17041|1|25|2|1|1138|215|1443|72|6.01|8.17|6.78|100.08|488.16|432.72|588.24|24.40|0.00|228.96|488.16|512.56|717.12|741.52|55.44| +2450843|37259|2450925|70759|1142853|547|17041|70759|1142853|547|17041|1|88|16|1|6718|271|1443|65|60.51|168.82|10.12|10315.50|657.80|3933.15|10973.30|39.46|0.00|3730.35|657.80|697.26|4388.15|4427.61|-3275.35| +2450843|78767|2450863|80513|1586834|3108|32629|80513|1586834|3108|32629|2|15|10|3|3001|246|1444|99|87.56|175.12|145.34|2948.22|14388.66|8668.44|17336.88|319.42|3741.05|5373.72|10647.61|10967.03|16021.33|16340.75|1979.17| +2450843|78767|2450853|80513|1586834|3108|32629|80513|1586834|3108|32629|2|23|14|1|13693|193|1444|66|44.32|70.46|54.25|1069.86|3580.50|2925.12|4650.36|286.44|0.00|418.44|3580.50|3866.94|3998.94|4285.38|655.38| +2450843|78767|2450933|80513|1586834|3108|32629|80513|1586834|3108|32629|2|28|8|2|10621|20|1444|7|88.13|225.61|81.21|1010.80|568.47|616.91|1579.27|0.00|0.00|663.25|568.47|568.47|1231.72|1231.72|-48.44| +2450843|78767|2450855|80513|1586834|3108|32629|80513|1586834|3108|32629|2|100|11|5|613|276|1444|3|92.95|212.85|119.19|280.98|357.57|278.85|638.55|32.18|0.00|140.46|357.57|389.75|498.03|530.21|78.72| +2450843|78767|2450866|80513|1586834|3108|32629|80513|1586834|3108|32629|2|20|19|5|4610|2|1444|13|19.27|44.70|34.86|127.92|453.18|250.51|581.10|36.25|0.00|278.85|453.18|489.43|732.03|768.28|202.67| +2450843|78767|2450863|80513|1586834|3108|32629|80513|1586834|3108|32629|2|9|9|5|5377|67|1444|82|26.80|64.58|51.01|1112.74|4182.82|2197.60|5295.56|0.00|0.00|2170.54|4182.82|4182.82|6353.36|6353.36|1985.22| +2450843|78767|2450863|80513|1586834|3108|32629|80513|1586834|3108|32629|2|107|13|2|1879|61|1444|62|11.95|27.48|24.45|187.86|1515.90|740.90|1703.76|106.11|0.00|476.78|1515.90|1622.01|1992.68|2098.79|775.00| +2450843|57844|2450882|15075|323839|2203|20120|15075|323839|2203|20120|4|1|13|4|12307|291|1445|15|83.08|215.17|178.59|548.70|2678.85|1246.20|3227.55|133.94|0.00|935.85|2678.85|2812.79|3614.70|3748.64|1432.65| +2450843|57844|2450916|15075|323839|2203|20120|15075|323839|2203|20120|4|25|8|1|16435|217|1445|10|11.44|32.71|4.57|281.40|45.70|114.40|327.10|0.91|0.00|121.00|45.70|46.61|166.70|167.61|-68.70| +2450843|57844|2450932|15075|323839|2203|20120|15075|323839|2203|20120|4|44|13|5|14602|189|1445|58|96.56|169.94|141.05|1675.62|8180.90|5600.48|9856.52|163.61|0.00|886.82|8180.90|8344.51|9067.72|9231.33|2580.42| +2450843|57844|2450895|15075|323839|2203|20120|15075|323839|2203|20120|4|86|2|4|14056|283|1445|30|37.74|53.96|21.04|987.60|631.20|1132.20|1618.80|4.79|511.27|48.30|119.93|124.72|168.23|173.02|-1012.27| +2450843|57844|2450922|15075|323839|2203|20120|15075|323839|2203|20120|4|44|5|2|2828|169|1445|76|12.76|12.88|2.44|793.44|185.44|969.76|978.88|3.89|120.53|440.04|64.91|68.80|504.95|508.84|-904.85| +2450843|46383|2450890|23021|66630|5086|36295|23021|66630|5086|36295|4|85|5|4|4105|290|1446|81|66.90|99.68|13.95|6944.13|1129.95|5418.90|8074.08|101.69|0.00|3956.04|1129.95|1231.64|5085.99|5187.68|-4288.95| +2450843|46383|2450859|23021|66630|5086|36295|23021|66630|5086|36295|4|82|13|2|5524|183|1446|65|47.73|56.32|11.26|2928.90|731.90|3102.45|3660.80|7.31|0.00|988.00|731.90|739.21|1719.90|1727.21|-2370.55| +2450843|46383|2450893|23021|66630|5086|36295|23021|66630|5086|36295|4|103|4|2|16696|288|1446|54|43.82|124.88|42.45|4451.22|2292.30|2366.28|6743.52|0.00|0.00|876.42|2292.30|2292.30|3168.72|3168.72|-73.98| +2450843|46383|2450868|23021|66630|5086|36295|23021|66630|5086|36295|4|3|9|2|11914|242|1446|70|76.65|182.42|151.40|2171.40|10598.00|5365.50|12769.40|741.86|0.00|5235.30|10598.00|11339.86|15833.30|16575.16|5232.50| +2450843|46383|2450854|23021|66630|5086|36295|23021|66630|5086|36295|4|80|2|3|11206|177|1446|6|29.70|89.10|7.12|491.88|42.72|178.20|534.60|0.32|10.68|203.10|32.04|32.36|235.14|235.46|-146.16| +2450843|46383|2450903|23021|66630|5086|36295|23021|66630|5086|36295|4|52|19|1|4898|165|1446|30|27.44|81.77|56.42|760.50|1692.60|823.20|2453.10|135.40|0.00|564.00|1692.60|1828.00|2256.60|2392.00|869.40| +2450843|46383|2450922|23021|66630|5086|36295|23021|66630|5086|36295|4|86|12|5|8636|154|1446|60|10.05|13.26|13.12|8.40|787.20|603.00|795.60|0.00|0.00|182.40|787.20|787.20|969.60|969.60|184.20| +2450843|46383|2450913|23021|66630|5086|36295|23021|66630|5086|36295|4|85|1|5|16490|51|1446|59|58.57|132.36|52.94|4685.78|3123.46|3455.63|7809.24|156.17|0.00|3279.81|3123.46|3279.63|6403.27|6559.44|-332.17| +2450843|46383|2450861|23021|66630|5086|36295|23021|66630|5086|36295|4|31|3|4|17134|202|1446|26|36.65|59.73|51.96|202.02|1350.96|952.90|1552.98|81.05|0.00|543.40|1350.96|1432.01|1894.36|1975.41|398.06| +2450843|55861|2450920|91356|799358|4607|42256|91356|799358|4607|42256|4|23|3|5|5998|202|1447|52|18.46|21.78|17.20|238.16|894.40|959.92|1132.56|62.60|0.00|226.20|894.40|957.00|1120.60|1183.20|-65.52| +2450843|55861|2450893|91356|799358|4607|42256|91356|799358|4607|42256|4|23|6|5|1807|140|1447|22|43.34|96.64|79.24|382.80|1743.28|953.48|2126.08|69.73|0.00|1063.04|1743.28|1813.01|2806.32|2876.05|789.80| +2450843|55861|2450893|91356|799358|4607|42256|91356|799358|4607|42256|4|30|14|5|583|235|1447|62|24.11|32.78|32.45|20.46|2011.90|1494.82|2032.36|20.11|0.00|832.66|2011.90|2032.01|2844.56|2864.67|517.08| +2450843|55861|2450897|91356|799358|4607|42256|91356|799358|4607|42256|4|11|18|5|15728|254|1447|56|1.34|1.95|0.64|73.36|35.84|75.04|109.20|3.22|0.00|35.84|35.84|39.06|71.68|74.90|-39.20| +2450843|55861|2450855|91356|799358|4607|42256|91356|799358|4607|42256|4|105|4|2|14545|136|1447|61|84.12|97.57|22.44|4582.93|1368.84|5131.32|5951.77|47.90|684.42|2618.73|684.42|732.32|3303.15|3351.05|-4446.90| +2450843|55861|2450883|91356|799358|4607|42256|91356|799358|4607|42256|4|48|2|4|1370|223|1447|84|72.71|140.33|37.88|8605.80|3181.92|6107.64|11787.72|190.91|0.00|1060.08|3181.92|3372.83|4242.00|4432.91|-2925.72| +2450843|55861|2450909|91356|799358|4607|42256|91356|799358|4607|42256|4|91|20|2|17587|181|1447|54|71.31|71.31|59.90|616.14|3234.60|3850.74|3850.74|161.73|0.00|1116.18|3234.60|3396.33|4350.78|4512.51|-616.14| +2450843|34704|2450850|7996|811700|4651|41955|7996|811700|4651|41955|4|108|15|2|11248|286|1448|13|65.22|90.65|82.49|106.08|1072.37|847.86|1178.45|53.61|0.00|223.86|1072.37|1125.98|1296.23|1349.84|224.51| +2450843|34704|2450883|7996|811700|4651|41955|7996|811700|4651|41955|4|52|9|4|9136|236|1448|12|25.32|41.27|0.00|495.24|0.00|303.84|495.24|0.00|0.00|128.76|0.00|0.00|128.76|128.76|-303.84| +2450843|34704|2450875|7996|811700|4651|41955|7996|811700|4651|41955|4|17|20|2|14422|261|1448|25|58.25|111.84|78.28|839.00|1957.00|1456.25|2796.00|176.13|0.00|1398.00|1957.00|2133.13|3355.00|3531.13|500.75| +2450843|34704|2450889|7996|811700|4651|41955|7996|811700|4651|41955|4|8|1|1|16447|34|1448|9|99.23|287.76|149.63|1243.17|1346.67|893.07|2589.84|42.55|282.80|129.42|1063.87|1106.42|1193.29|1235.84|170.80| +2450843|34704|2450850|7996|811700|4651|41955|7996|811700|4651|41955|4|18|2|1|8293|104|1448|86|81.60|114.24|111.95|196.94|9627.70|7017.60|9824.64|673.93|0.00|392.16|9627.70|10301.63|10019.86|10693.79|2610.10| +2450843|34704|2450873|7996|811700|4651|41955|7996|811700|4651|41955|4|67|8|5|11242|239|1448|21|75.11|194.53|143.95|1062.18|3022.95|1577.31|4085.13|272.06|0.00|694.47|3022.95|3295.01|3717.42|3989.48|1445.64| +2450843|34704|2450848|7996|811700|4651|41955|7996|811700|4651|41955|4|53|12|2|15523|189|1448|21|15.60|26.83|8.58|383.25|180.18|327.60|563.43|3.60|0.00|174.51|180.18|183.78|354.69|358.29|-147.42| +2450843|34704|2450930|7996|811700|4651|41955|7996|811700|4651|41955|4|4|17|2|5665|42|1448|85|7.97|18.57|14.85|316.20|1262.25|677.45|1578.45|113.60|0.00|741.20|1262.25|1375.85|2003.45|2117.05|584.80| +2450843|34704|2450866|7996|811700|4651|41955|7996|811700|4651|41955|4|42|2|4|3476|205|1448|95|76.89|223.74|35.79|17855.25|3400.05|7304.55|21255.30|28.90|510.00|637.45|2890.05|2918.95|3527.50|3556.40|-4414.50| +2450843|34704|2450861|7996|811700|4651|41955|7996|811700|4651|41955|4|40|4|1|15074|243|1448|59|33.09|59.89|29.94|1767.05|1766.46|1952.31|3533.51|105.98|0.00|247.21|1766.46|1872.44|2013.67|2119.65|-185.85| +2450843|34704|2450904|7996|811700|4651|41955|7996|811700|4651|41955|4|53|3|4|7838|223|1448|98|25.62|74.81|68.82|587.02|6744.36|2510.76|7331.38|472.10|0.00|3372.18|6744.36|7216.46|10116.54|10588.64|4233.60| +2450843|82339|2450885|41661|1374345|3029|21760|41661|1374345|3029|21760|4|42|5|3|9920|214|1449|63|68.05|119.08|80.97|2400.93|5101.11|4287.15|7502.04|306.06|0.00|975.24|5101.11|5407.17|6076.35|6382.41|813.96| +2450843|82339|2450913|41661|1374345|3029|21760|41661|1374345|3029|21760|4|85|6|3|13064|191|1449|80|91.73|161.44|14.52|11753.60|1161.60|7338.40|12915.20|36.24|708.57|5036.80|453.03|489.27|5489.83|5526.07|-6885.37| +2450843|82339|2450897|41661|1374345|3029|21760|41661|1374345|3029|21760|4|7|12|4|4466|186|1449|91|29.42|37.95|34.15|345.80|3107.65|2677.22|3453.45|31.07|0.00|966.42|3107.65|3138.72|4074.07|4105.14|430.43| +2450843|82339|2450919|41661|1374345|3029|21760|41661|1374345|3029|21760|4|79|17|2|15448|99|1449|36|13.60|21.21|4.87|588.24|175.32|489.60|763.56|2.36|149.02|15.12|26.30|28.66|41.42|43.78|-463.30| +2450843|82339|2450897|41661|1374345|3029|21760|41661|1374345|3029|21760|4|3|1|4|1705|170|1449|16|29.95|49.11|24.55|392.96|392.80|479.20|785.76|7.85|0.00|15.68|392.80|400.65|408.48|416.33|-86.40| +2450843|82339|2450916|41661|1374345|3029|21760|41661|1374345|3029|21760|4|98|13|2|13316|135|1449|67|18.21|19.30|16.21|207.03|1086.07|1220.07|1293.10|86.88|0.00|180.90|1086.07|1172.95|1266.97|1353.85|-134.00| +2450843|82339|2450880|41661|1374345|3029|21760|41661|1374345|3029|21760|4|61|19|5|17114|116|1449|100|63.94|163.68|36.00|12768.00|3600.00|6394.00|16368.00|59.04|648.00|7365.00|2952.00|3011.04|10317.00|10376.04|-3442.00| +2450843|82339|2450875|41661|1374345|3029|21760|41661|1374345|3029|21760|4|97|6|4|12247|235|1449|71|16.18|38.02|12.92|1782.10|917.32|1148.78|2699.42|64.21|0.00|26.98|917.32|981.53|944.30|1008.51|-231.46| +2450843|82339|2450928|41661|1374345|3029|21760|41661|1374345|3029|21760|4|24|1|4|2402|214|1449|9|37.87|58.69|34.04|221.85|306.36|340.83|528.21|0.00|0.00|179.55|306.36|306.36|485.91|485.91|-34.47| +2450843|82339|2450900|41661|1374345|3029|21760|41661|1374345|3029|21760|4|42|6|4|8474|107|1449|87|62.85|165.29|123.96|3595.71|10784.52|5467.95|14380.23|539.22|0.00|5751.57|10784.52|11323.74|16536.09|17075.31|5316.57| +2450843|82339|2450900|41661|1374345|3029|21760|41661|1374345|3029|21760|4|104|9|4|13514|247|1449|89|66.88|154.49|54.07|8937.38|4812.23|5952.32|13749.61|48.12|0.00|6462.29|4812.23|4860.35|11274.52|11322.64|-1140.09| +2450843|82339|2450912|41661|1374345|3029|21760|41661|1374345|3029|21760|4|43|13|1|4736|86|1449|20|25.86|45.77|5.49|805.60|109.80|517.20|915.40|0.00|0.00|375.20|109.80|109.80|485.00|485.00|-407.40| +2450843|82339|2450877|41661|1374345|3029|21760|41661|1374345|3029|21760|4|15|7|3|2560|294|1449|88|65.06|111.25|85.66|2251.92|7538.08|5725.28|9790.00|376.90|0.00|880.88|7538.08|7914.98|8418.96|8795.86|1812.80| +2450843|58952|2450921|95408|897186|6964|44342|95408|897186|6964|44342|2|32|1|3|3278|87|1450|81|69.99|175.67|137.02|3130.65|11098.62|5669.19|14229.27|221.97|0.00|4837.32|11098.62|11320.59|15935.94|16157.91|5429.43| +2450843|58952|2450892|95408|897186|6964|44342|95408|897186|6964|44342|2|107|17|3|10288|30|1450|47|4.67|13.44|2.68|505.72|125.96|219.49|631.68|10.07|0.00|145.23|125.96|136.03|271.19|281.26|-93.53| +2450843|58952|2450909|95408|897186|6964|44342|95408|897186|6964|44342|2|92|4|3|6133|268|1450|3|38.47|50.01|15.00|105.03|45.00|115.41|150.03|3.60|0.00|51.00|45.00|48.60|96.00|99.60|-70.41| +2450843|58952|2450919|95408|897186|6964|44342|95408|897186|6964|44342|2|14|19|3|3355|293|1450|45|69.03|140.13|21.01|5360.40|945.45|3106.35|6305.85|37.81|0.00|2332.80|945.45|983.26|3278.25|3316.06|-2160.90| +2450843|58952|2450870|95408|897186|6964|44342|95408|897186|6964|44342|2|3|3|3|8215|193|1450|24|61.54|114.46|9.15|2527.44|219.60|1476.96|2747.04|10.54|8.78|961.44|210.82|221.36|1172.26|1182.80|-1266.14| +2450843|58952|2450893|95408|897186|6964|44342|95408|897186|6964|44342|2|75|5|2|8005|94|1450|78|31.37|62.74|31.37|2446.86|2446.86|2446.86|4893.72|48.93|0.00|195.00|2446.86|2495.79|2641.86|2690.79|0.00| +2450843|58952|2450896|95408|897186|6964|44342|95408|897186|6964|44342|2|94|6|4|6416|73|1450|99|62.90|101.26|86.07|1503.81|8520.93|6227.10|10024.74|426.04|0.00|500.94|8520.93|8946.97|9021.87|9447.91|2293.83| +2450843|58952|2450892|95408|897186|6964|44342|95408|897186|6964|44342|2|35|1|2|7207|121|1450|54|40.96|60.62|11.51|2651.94|621.54|2211.84|3273.48|12.43|0.00|556.20|621.54|633.97|1177.74|1190.17|-1590.30| +2450843|58952|2450892|95408|897186|6964|44342|95408|897186|6964|44342|2|52|17|1|302|129|1450|12|42.33|119.37|97.88|257.88|1174.56|507.96|1432.44|35.23|0.00|229.08|1174.56|1209.79|1403.64|1438.87|666.60| +2450843|58952|2450889|95408|897186|6964|44342|95408|897186|6964|44342|2|86|16|3|7448|157|1450|82|7.88|10.08|7.86|182.04|644.52|646.16|826.56|13.92|296.47|223.04|348.05|361.97|571.09|585.01|-298.11| +2450843|58952|2450876|95408|897186|6964|44342|95408|897186|6964|44342|2|61|3|1|718|237|1450|78|2.09|6.16|1.29|379.86|100.62|163.02|480.48|3.01|0.00|100.62|100.62|103.63|201.24|204.25|-62.40| +2450843|58952|2450891|95408|897186|6964|44342|95408|897186|6964|44342|2|88|8|5|6358|141|1450|88|98.64|123.30|106.03|1519.76|9330.64|8680.32|10850.40|186.61|0.00|4122.80|9330.64|9517.25|13453.44|13640.05|650.32| +2450843|58952|2450906|95408|897186|6964|44342|95408|897186|6964|44342|2|4|15|3|12964|151|1450|19|45.68|56.64|9.06|904.02|172.14|867.92|1076.16|12.04|0.00|32.11|172.14|184.18|204.25|216.29|-695.78| +2450843|58952|2450913|95408|897186|6964|44342|95408|897186|6964|44342|2|93|16|2|2254|261|1450|33|77.72|223.83|53.71|5613.96|1772.43|2564.76|7386.39|159.51|0.00|442.86|1772.43|1931.94|2215.29|2374.80|-792.33| +2450843|24321|2450878|94631|330701|2771|10088|94631|330701|2771|10088|2|36|18|4|14779|42|1451|4|89.88|208.52|83.40|500.48|333.60|359.52|834.08|20.01|0.00|400.32|333.60|353.61|733.92|753.93|-25.92| +2450843|24321|2450925|94631|330701|2771|10088|94631|330701|2771|10088|2|98|1|1|5869|78|1451|57|64.01|186.90|143.91|2450.43|8202.87|3648.57|10653.30|0.00|0.00|4261.32|8202.87|8202.87|12464.19|12464.19|4554.30| +2450843|24321|2450899|94631|330701|2771|10088|94631|330701|2771|10088|2|67|15|3|12188|86|1451|15|75.91|163.96|78.70|1278.90|1180.50|1138.65|2459.40|70.83|0.00|1057.50|1180.50|1251.33|2238.00|2308.83|41.85| +2450843|24321|2450869|94631|330701|2771|10088|94631|330701|2771|10088|2|23|1|1|13958|285|1451|43|47.17|137.26|64.51|3128.25|2773.93|2028.31|5902.18|0.00|0.00|2124.63|2773.93|2773.93|4898.56|4898.56|745.62| +2450843|24321|2450869|94631|330701|2771|10088|94631|330701|2771|10088|2|56|19|5|14563|75|1451|39|97.52|103.37|83.72|766.35|3265.08|3803.28|4031.43|195.90|0.00|120.90|3265.08|3460.98|3385.98|3581.88|-538.20| +2450843|24321|2450923|94631|330701|2771|10088|94631|330701|2771|10088|2|62|1|5|7946|196|1451|71|18.62|54.92|34.59|1443.43|2455.89|1322.02|3899.32|221.03|0.00|1013.17|2455.89|2676.92|3469.06|3690.09|1133.87| +2450843|24321|2450888|94631|330701|2771|10088|94631|330701|2771|10088|2|57|18|2|7147|160|1451|73|45.11|91.57|5.49|6283.84|400.77|3293.03|6684.61|8.01|0.00|2539.67|400.77|408.78|2940.44|2948.45|-2892.26| +2450843|24321|2450865|94631|330701|2771|10088|94631|330701|2771|10088|2|7|16|3|181|1|1451|1|1.81|3.00|1.92|1.08|1.92|1.81|3.00|0.07|0.00|1.41|1.92|1.99|3.33|3.40|0.11| +2450843|24321|2450856|94631|330701|2771|10088|94631|330701|2771|10088|2|46|19|1|5456|82|1451|73|65.63|187.70|152.03|2603.91|11098.19|4790.99|13702.10|110.98|0.00|5754.59|11098.19|11209.17|16852.78|16963.76|6307.20| +2450843|24321|2450915|94631|330701|2771|10088|94631|330701|2771|10088|2|13|8|1|14944|192|1451|98|73.34|217.08|121.56|9360.96|11912.88|7187.32|21273.84|1072.15|0.00|3828.86|11912.88|12985.03|15741.74|16813.89|4725.56| +2450843|24321|2450900|94631|330701|2771|10088|94631|330701|2771|10088|2|105|14|2|3398|274|1451|89|66.49|196.14|47.07|13267.23|4189.23|5917.61|17456.46|87.97|1256.76|7506.26|2932.47|3020.44|10438.73|10526.70|-2985.14| +2450843|24321|2450904|94631|330701|2771|10088|94631|330701|2771|10088|2|7|4|5|15458|252|1451|90|9.47|12.59|5.53|635.40|497.70|852.30|1133.10|24.63|223.96|271.80|273.74|298.37|545.54|570.17|-578.56| +2450843|11653|2450868|14909|632125|3741|48957|27924|905789|2205|11021|4|73|19|5|2848|115|1452|27|24.52|60.56|10.90|1340.82|294.30|662.04|1635.12|20.60|0.00|130.68|294.30|314.90|424.98|445.58|-367.74| +2450843|11653|2450903|14909|632125|3741|48957|27924|905789|2205|11021|4|8|14|1|15872|108|1452|3|64.50|139.96|5.59|403.11|16.77|193.50|419.88|0.83|0.00|159.54|16.77|17.60|176.31|177.14|-176.73| +2450843|11653|2450900|14909|632125|3741|48957|27924|905789|2205|11021|4|37|19|3|5521|94|1452|85|24.16|26.57|16.20|881.45|1377.00|2053.60|2258.45|68.85|0.00|293.25|1377.00|1445.85|1670.25|1739.10|-676.60| +2450843|11653|2450907|14909|632125|3741|48957|27924|905789|2205|11021|4|44|13|1|4696|110|1452|25|76.38|214.62|27.90|4668.00|697.50|1909.50|5365.50|48.82|0.00|107.25|697.50|746.32|804.75|853.57|-1212.00| +2450843|11653|2450907|14909|632125|3741|48957|27924|905789|2205|11021|4|21|11|3|17146|114|1452|80|12.74|19.11|0.00|1528.80|0.00|1019.20|1528.80|0.00|0.00|626.40|0.00|0.00|626.40|626.40|-1019.20| +2450843|11653|2450869|14909|632125|3741|48957|27924|905789|2205|11021|4|3|6|2|7570|26|1452|71|91.53|170.24|107.25|4472.29|7614.75|6498.63|12087.04|152.29|0.00|5438.60|7614.75|7767.04|13053.35|13205.64|1116.12| +2450843|73902|2450852|40918|533020|3169|39318|61603|1151569|118|15067|2|13|9|4|6667|115|1453|19|38.94|115.26|38.03|1467.37|722.57|739.86|2189.94|43.35|0.00|109.44|722.57|765.92|832.01|875.36|-17.29| +2450843|73902|2450913|40918|533020|3169|39318|61603|1151569|118|15067|2|69|14|1|15529|125|1453|78|39.88|87.73|27.19|4722.12|2120.82|3110.64|6842.94|23.75|933.16|2120.82|1187.66|1211.41|3308.48|3332.23|-1922.98| +2450843|73902|2450845|40918|533020|3169|39318|61603|1151569|118|15067|2|41|17|2|2428|189|1453|86|31.09|46.94|25.81|1817.18|2219.66|2673.74|4036.84|177.57|0.00|1412.12|2219.66|2397.23|3631.78|3809.35|-454.08| +2450843|73902|2450861|40918|533020|3169|39318|61603|1151569|118|15067|2|21|16|5|1418|13|1453|47|62.11|131.67|63.20|3218.09|2970.40|2919.17|6188.49|0.00|0.00|1361.12|2970.40|2970.40|4331.52|4331.52|51.23| +2450843|73902|2450872|40918|533020|3169|39318|61603|1151569|118|15067|2|107|16|2|2971|184|1453|46|59.30|72.34|32.55|1830.34|1497.30|2727.80|3327.64|47.61|703.73|1131.14|793.57|841.18|1924.71|1972.32|-1934.23| +2450843|6682|2450909|55611|1205854|1250|19093|55611|1205854|1250|19093|2|105|12|1|1532|289|1454|1|62.82|179.66|156.30|23.36|156.30|62.82|179.66|3.12|0.00|0.00|156.30|159.42|156.30|159.42|93.48| +2450843|6682|2450925|55611|1205854|1250|19093|55611|1205854|1250|19093|2|61|7|1|12031|244|1454|24|65.60|192.20|107.63|2029.68|2583.12|1574.40|4612.80|0.00|0.00|2214.00|2583.12|2583.12|4797.12|4797.12|1008.72| +2450843|6682|2450883|55611|1205854|1250|19093|55611|1205854|1250|19093|2|39|14|2|7868|110|1454|73|57.22|156.21|145.27|798.62|10604.71|4177.06|11403.33|460.24|4029.78|1025.65|6574.93|7035.17|7600.58|8060.82|2397.87| +2450843|6682|2450901|55611|1205854|1250|19093|55611|1205854|1250|19093|2|95|9|4|10819|256|1454|36|80.42|152.79|70.28|2970.36|2530.08|2895.12|5500.44|0.00|0.00|1265.04|2530.08|2530.08|3795.12|3795.12|-365.04| +2450843|6682|2450898|55611|1205854|1250|19093|55611|1205854|1250|19093|2|106|3|3|17608|253|1454|100|40.64|48.36|41.58|678.00|4158.00|4064.00|4836.00|249.48|0.00|193.00|4158.00|4407.48|4351.00|4600.48|94.00| +2450843|6682|2450865|55611|1205854|1250|19093|55611|1205854|1250|19093|2|92|10|4|1484|287|1454|68|69.58|127.33|106.95|1385.84|7272.60|4731.44|8658.44|0.00|0.00|1385.16|7272.60|7272.60|8657.76|8657.76|2541.16| +2450843|6682|2450882|55611|1205854|1250|19093|55611|1205854|1250|19093|2|55|17|3|6176|133|1454|23|58.61|74.43|11.16|1455.21|256.68|1348.03|1711.89|9.70|148.87|547.63|107.81|117.51|655.44|665.14|-1240.22| +2450843|6682|2450883|55611|1205854|1250|19093|55611|1205854|1250|19093|2|6|12|4|9226|281|1454|81|95.99|251.49|191.13|4889.16|15481.53|7775.19|20370.69|928.89|0.00|2240.46|15481.53|16410.42|17721.99|18650.88|7706.34| +2450843|6682|2450851|55611|1205854|1250|19093|55611|1205854|1250|19093|2|49|6|1|13834|196|1454|45|61.96|116.48|45.42|3197.70|2043.90|2788.20|5241.60|61.31|0.00|681.30|2043.90|2105.21|2725.20|2786.51|-744.30| +2450843|6682|2450886|55611|1205854|1250|19093|55611|1205854|1250|19093|2|90|5|5|13418|128|1454|54|49.67|93.87|49.75|2382.48|2686.50|2682.18|5068.98|241.78|0.00|2027.16|2686.50|2928.28|4713.66|4955.44|4.32| +2450843|6682|2450895|55611|1205854|1250|19093|55611|1205854|1250|19093|2|50|14|1|1687|45|1454|98|87.69|221.85|219.63|217.56|21523.74|8593.62|21741.30|430.47|0.00|2173.64|21523.74|21954.21|23697.38|24127.85|12930.12| +2450843|6682|2450912|55611|1205854|1250|19093|55611|1205854|1250|19093|2|85|4|3|1273|55|1454|39|25.03|58.57|26.94|1233.57|1050.66|976.17|2284.23|0.00|0.00|616.59|1050.66|1050.66|1667.25|1667.25|74.49| +2450843|6682|2450918|55611|1205854|1250|19093|55611|1205854|1250|19093|2|31|13|5|15073|65|1454|11|59.03|69.65|57.80|130.35|635.80|649.33|766.15|19.07|0.00|291.06|635.80|654.87|926.86|945.93|-13.53| +2450843|6682|2450883|55611|1205854|1250|19093|55611|1205854|1250|19093|2|5|11|1|2804|180|1454|27|64.90|123.31|43.15|2164.32|1165.05|1752.30|3329.37|34.95|0.00|266.22|1165.05|1200.00|1431.27|1466.22|-587.25| +2450843|32097|2450903|67683|1056884|4485|11733|29779|363735|6536|8937|2|81|9|4|4879|269|1455|84|70.45|192.32|90.39|8562.12|7592.76|5917.80|16154.88|151.85|0.00|7430.64|7592.76|7744.61|15023.40|15175.25|1674.96| +2450843|32097|2450877|67683|1056884|4485|11733|29779|363735|6536|8937|2|75|5|1|5048|121|1455|31|12.58|14.97|3.74|348.13|115.94|389.98|464.07|0.00|0.00|171.43|115.94|115.94|287.37|287.37|-274.04| +2450843|32097|2450875|67683|1056884|4485|11733|29779|363735|6536|8937|2|14|18|3|17185|41|1455|96|18.39|54.25|44.48|937.92|4270.08|1765.44|5208.00|128.10|0.00|468.48|4270.08|4398.18|4738.56|4866.66|2504.64| +2450843|32097|2450867|67683|1056884|4485|11733|29779|363735|6536|8937|2|7|15|1|6709|131|1455|19|47.17|57.07|5.70|976.03|108.30|896.23|1084.33|7.58|0.00|466.26|108.30|115.88|574.56|582.14|-787.93| +2450843|32097|2450883|67683|1056884|4485|11733|29779|363735|6536|8937|2|27|14|4|17672|85|1455|6|94.74|144.95|114.51|182.64|687.06|568.44|869.70|20.61|0.00|130.44|687.06|707.67|817.50|838.11|118.62| +2450843|3789|2450880|52219|572013|160|19154|52219|572013|160|19154|2|105|10|2|7012|161|1456|23|34.84|63.75|22.31|953.12|513.13|801.32|1466.25|35.91|0.00|469.20|513.13|549.04|982.33|1018.24|-288.19| +2450843|3789|2450870|52219|572013|160|19154|52219|572013|160|19154|2|11|2|3|2620|58|1456|5|48.86|82.08|33.65|242.15|168.25|244.30|410.40|3.36|0.00|28.70|168.25|171.61|196.95|200.31|-76.05| +2450843|3789|2450930|52219|572013|160|19154|52219|572013|160|19154|2|23|7|3|15014|117|1456|55|83.63|135.48|10.83|6855.75|595.65|4599.65|7451.40|0.00|0.00|670.45|595.65|595.65|1266.10|1266.10|-4004.00| +2450843|3789|2450917|52219|572013|160|19154|52219|572013|160|19154|2|53|1|4|10204|272|1456|54|67.20|110.20|56.20|2916.00|3034.80|3628.80|5950.80|60.69|0.00|2915.46|3034.80|3095.49|5950.26|6010.95|-594.00| +2450843|3789|2450852|52219|572013|160|19154|52219|572013|160|19154|2|107|6|2|17116|205|1456|62|19.11|49.30|44.37|305.66|2750.94|1184.82|3056.60|0.00|0.00|641.70|2750.94|2750.94|3392.64|3392.64|1566.12| +2450843|3789|2450907|52219|572013|160|19154|52219|572013|160|19154|2|94|7|1|322|294|1456|95|59.65|116.91|1.16|10996.25|110.20|5666.75|11106.45|6.61|0.00|2998.20|110.20|116.81|3108.40|3115.01|-5556.55| +2450843|3789|2450859|52219|572013|160|19154|52219|572013|160|19154|2|35|13|4|11312|153|1456|93|48.84|136.26|1.36|12545.70|126.48|4542.12|12672.18|0.00|0.00|6208.68|126.48|126.48|6335.16|6335.16|-4415.64| +2450843|3789|2450909|52219|572013|160|19154|52219|572013|160|19154|2|90|6|1|13766|143|1456|47|95.83|204.11|187.78|767.51|8825.66|4504.01|9593.17|617.79|0.00|4604.59|8825.66|9443.45|13430.25|14048.04|4321.65| +2450843|3789|2450852|52219|572013|160|19154|52219|572013|160|19154|2|19|13|3|2858|91|1456|41|40.83|46.54|35.83|439.11|1469.03|1674.03|1908.14|0.00|0.00|362.44|1469.03|1469.03|1831.47|1831.47|-205.00| +2450843|34579|2450910|1417|1920765|1713|17861|1417|1920765|1713|17861|4|68|3|2|5326|77|1457|1|8.78|13.17|12.51|0.66|12.51|8.78|13.17|0.25|0.00|3.29|12.51|12.76|15.80|16.05|3.73| +2450843|34579|2450905|1417|1920765|1713|17861|1417|1920765|1713|17861|4|22|5|1|16790|192|1457|68|69.39|159.59|78.19|5535.20|5316.92|4718.52|10852.12|106.33|0.00|1302.20|5316.92|5423.25|6619.12|6725.45|598.40| +2450843|34579|2450880|1417|1920765|1713|17861|1417|1920765|1713|17861|4|18|6|5|12644|1|1457|91|57.87|84.49|46.46|3460.73|4227.86|5266.17|7688.59|81.17|3213.17|384.02|1014.69|1095.86|1398.71|1479.88|-4251.48| +2450843|34579|2450920|1417|1920765|1713|17861|1417|1920765|1713|17861|4|107|13|5|5635|176|1457|44|5.29|6.87|3.16|163.24|139.04|232.76|302.28|5.56|0.00|66.44|139.04|144.60|205.48|211.04|-93.72| +2450843|34579|2450904|1417|1920765|1713|17861|1417|1920765|1713|17861|4|10|13|5|15110|252|1457|61|85.63|95.90|48.90|2867.00|2982.90|5223.43|5849.90|238.63|0.00|233.63|2982.90|3221.53|3216.53|3455.16|-2240.53| +2450843|34579|2450867|1417|1920765|1713|17861|1417|1920765|1713|17861|4|86|15|5|5170|239|1457|38|91.86|242.51|50.92|7280.42|1934.96|3490.68|9215.38|38.69|0.00|1658.70|1934.96|1973.65|3593.66|3632.35|-1555.72| +2450843|34579|2450909|1417|1920765|1713|17861|1417|1920765|1713|17861|4|65|5|3|4508|124|1457|80|33.51|70.03|49.02|1680.80|3921.60|2680.80|5602.40|196.08|0.00|112.00|3921.60|4117.68|4033.60|4229.68|1240.80| +2450843|34579|2450908|1417|1920765|1713|17861|1417|1920765|1713|17861|4|12|12|5|17797|65|1457|40|18.43|24.32|11.91|496.40|476.40|737.20|972.80|14.29|0.00|77.60|476.40|490.69|554.00|568.29|-260.80| +2450843|34579|2450914|1417|1920765|1713|17861|1417|1920765|1713|17861|4|4|3|3|2506|86|1457|43|63.04|74.38|7.43|2878.85|319.49|2710.72|3198.34|19.16|0.00|1471.03|319.49|338.65|1790.52|1809.68|-2391.23| +2450843|34579|2450883|1417|1920765|1713|17861|1417|1920765|1713|17861|4|98|9|3|11690|133|1457|89|72.01|196.58|178.88|1575.30|15920.32|6408.89|17495.62|318.40|0.00|349.77|15920.32|16238.72|16270.09|16588.49|9511.43| +2450843|38021|2450914|4228|1644266|6704|31942|4228|1644266|6704|31942|1|106|20|3|436|123|1458|91|65.56|118.66|10.67|9827.09|970.97|5965.96|10798.06|48.54|0.00|3455.27|970.97|1019.51|4426.24|4474.78|-4994.99| +2450843|38021|2450929|4228|1644266|6704|31942|4228|1644266|6704|31942|1|59|10|2|12127|132|1458|56|96.69|137.29|30.20|5997.04|1691.20|5414.64|7688.24|33.82|0.00|1998.64|1691.20|1725.02|3689.84|3723.66|-3723.44| +2450843|38021|2450924|4228|1644266|6704|31942|4228|1644266|6704|31942|1|65|18|2|11326|56|1458|68|46.88|110.63|5.53|7146.80|376.04|3187.84|7522.84|3.76|0.00|676.60|376.04|379.80|1052.64|1056.40|-2811.80| +2450843|38021|2450918|4228|1644266|6704|31942|4228|1644266|6704|31942|1|15|8|3|13520|120|1458|40|62.31|157.02|69.08|3517.60|2763.20|2492.40|6280.80|248.68|0.00|188.40|2763.20|3011.88|2951.60|3200.28|270.80| +2450843|38021|2450889|4228|1644266|6704|31942|4228|1644266|6704|31942|1|14|19|1|6649|145|1458|38|30.02|32.42|8.75|899.46|332.50|1140.76|1231.96|9.97|0.00|615.98|332.50|342.47|948.48|958.45|-808.26| +2450843|38021|2450900|4228|1644266|6704|31942|4228|1644266|6704|31942|1|59|14|3|1952|49|1458|52|81.70|103.75|1.03|5341.44|53.56|4248.40|5395.00|0.00|0.00|53.56|53.56|53.56|107.12|107.12|-4194.84| +2450843|38021|2450906|4228|1644266|6704|31942|4228|1644266|6704|31942|1|65|11|2|14360|256|1458|72|75.12|101.41|36.50|4673.52|2628.00|5408.64|7301.52|0.00|0.00|1751.76|2628.00|2628.00|4379.76|4379.76|-2780.64| +2450843|38021|2450917|4228|1644266|6704|31942|4228|1644266|6704|31942|1|57|15|1|11311|113|1458|28|67.58|68.25|41.63|745.36|1165.64|1892.24|1911.00|23.31|0.00|515.76|1165.64|1188.95|1681.40|1704.71|-726.60| +2450843|38021|2450920|4228|1644266|6704|31942|4228|1644266|6704|31942|1|4|13|1|12950|180|1458|13|9.04|19.25|7.12|157.69|92.56|117.52|250.25|0.00|52.75|47.45|39.81|39.81|87.26|87.26|-77.71| +2450843|38021|2450857|4228|1644266|6704|31942|4228|1644266|6704|31942|1|59|5|2|7657|53|1458|4|63.49|131.42|9.19|488.92|36.76|253.96|525.68|1.47|0.00|36.76|36.76|38.23|73.52|74.99|-217.20| +2450843|42622|2450920|57595|1625441|934|24427|57595|1625441|934|24427|4|11|5|1|3170|5|1459|84|28.21|81.80|12.27|5840.52|1030.68|2369.64|6871.20|10.30|0.00|687.12|1030.68|1040.98|1717.80|1728.10|-1338.96| +2450843|42622|2450873|57595|1625441|934|24427|57595|1625441|934|24427|4|15|3|3|12049|98|1459|32|12.33|24.53|17.41|227.84|557.12|394.56|784.96|50.14|0.00|125.44|557.12|607.26|682.56|732.70|162.56| +2450843|42622|2450908|57595|1625441|934|24427|57595|1625441|934|24427|4|93|5|5|11222|190|1459|54|52.84|154.82|38.70|6270.48|2089.80|2853.36|8360.28|20.89|0.00|3845.34|2089.80|2110.69|5935.14|5956.03|-763.56| +2450843|42622|2450885|57595|1625441|934|24427|57595|1625441|934|24427|4|81|4|2|16369|162|1459|93|87.73|244.76|93.00|14113.68|8649.00|8158.89|22762.68|778.41|0.00|8194.23|8649.00|9427.41|16843.23|17621.64|490.11| +2450843|42622|2450852|57595|1625441|934|24427|57595|1625441|934|24427|4|92|5|2|13327|194|1459|74|11.44|28.02|26.05|145.78|1927.70|846.56|2073.48|77.10|0.00|145.04|1927.70|2004.80|2072.74|2149.84|1081.14| +2450843|42622|2450916|57595|1625441|934|24427|57595|1625441|934|24427|4|64|19|5|14413|135|1459|88|38.70|108.36|30.34|6865.76|2669.92|3405.60|9535.68|106.79|0.00|3050.96|2669.92|2776.71|5720.88|5827.67|-735.68| +2450843|42622|2450857|57595|1625441|934|24427|57595|1625441|934|24427|4|77|16|5|15223|154|1459|68|42.49|60.76|38.27|1529.32|2602.36|2889.32|4131.68|208.18|0.00|660.96|2602.36|2810.54|3263.32|3471.50|-286.96| +2450843|59923|2450918|97961|482023|4485|42194|97961|482023|4485|42194|1|49|19|1|1396|100|1460|32|10.25|22.55|21.42|36.16|685.44|328.00|721.60|0.00|452.39|288.64|233.05|233.05|521.69|521.69|-94.95| +2450843|59923|2450927|97961|482023|4485|42194|97961|482023|4485|42194|1|97|9|1|2462|212|1460|82|84.20|237.44|189.95|3894.18|15575.90|6904.40|19470.08|934.55|0.00|6424.70|15575.90|16510.45|22000.60|22935.15|8671.50| +2450843|59923|2450918|97961|482023|4485|42194|97961|482023|4485|42194|1|19|1|1|14581|124|1460|45|13.98|35.64|33.14|112.50|1491.30|629.10|1603.80|74.56|0.00|15.75|1491.30|1565.86|1507.05|1581.61|862.20| +2450843|59923|2450850|97961|482023|4485|42194|97961|482023|4485|42194|1|65|7|5|11608|39|1460|21|6.00|13.68|7.25|135.03|152.25|126.00|287.28|1.52|0.00|97.65|152.25|153.77|249.90|251.42|26.25| +2450843|59923|2450877|97961|482023|4485|42194|97961|482023|4485|42194|1|31|10|3|412|168|1460|22|37.12|65.70|0.00|1445.40|0.00|816.64|1445.40|0.00|0.00|534.60|0.00|0.00|534.60|534.60|-816.64| +2450843|59923|2450851|97961|482023|4485|42194|97961|482023|4485|42194|1|100|18|1|3274|37|1460|51|33.51|78.07|50.74|1393.83|2587.74|1709.01|3981.57|0.00|0.00|1712.07|2587.74|2587.74|4299.81|4299.81|878.73| +2450843|59923|2450902|97961|482023|4485|42194|97961|482023|4485|42194|1|60|11|1|9037|185|1460|52|10.69|29.18|3.20|1350.96|166.40|555.88|1517.36|3.32|0.00|364.00|166.40|169.72|530.40|533.72|-389.48| +2450843|59923|2450858|97961|482023|4485|42194|97961|482023|4485|42194|1|3|8|3|12562|195|1460|53|26.64|76.45|36.69|2107.28|1944.57|1411.92|4051.85|0.00|0.00|1053.11|1944.57|1944.57|2997.68|2997.68|532.65| +2450843|59923|2450846|97961|482023|4485|42194|97961|482023|4485|42194|1|100|8|1|4064|22|1460|51|66.97|81.70|4.08|3958.62|208.08|3415.47|4166.70|4.16|0.00|1124.55|208.08|212.24|1332.63|1336.79|-3207.39| +2450843|73334|2450923|88720|484215|6557|16499|88720|484215|6557|16499|4|31|1|5|3244|221|1461|40|46.84|111.94|5.59|4254.00|223.60|1873.60|4477.60|20.12|0.00|2149.20|223.60|243.72|2372.80|2392.92|-1650.00| +2450843|73334|2450888|88720|484215|6557|16499|88720|484215|6557|16499|4|44|15|3|10249|227|1461|94|80.93|137.58|134.82|259.44|12673.08|7607.42|12932.52|1013.84|0.00|2585.94|12673.08|13686.92|15259.02|16272.86|5065.66| +2450843|73334|2450879|88720|484215|6557|16499|88720|484215|6557|16499|4|42|16|2|6922|119|1461|66|37.30|42.14|40.45|111.54|2669.70|2461.80|2781.24|133.48|0.00|917.40|2669.70|2803.18|3587.10|3720.58|207.90| +2450843|73334|2450914|88720|484215|6557|16499|88720|484215|6557|16499|4|27|20|4|10988|74|1461|59|17.06|34.29|30.17|243.08|1780.03|1006.54|2023.11|89.00|0.00|201.78|1780.03|1869.03|1981.81|2070.81|773.49| +2450843|73334|2450933|88720|484215|6557|16499|88720|484215|6557|16499|4|78|17|2|11216|145|1461|86|87.93|189.04|149.34|3414.20|12843.24|7561.98|16257.44|642.16|0.00|1300.32|12843.24|13485.40|14143.56|14785.72|5281.26| +2450843|34416|2450886|984|1761548|1759|40723|984|1761548|1759|40723|4|74|5|5|11341|37|1462|50|78.59|161.89|6.47|7771.00|323.50|3929.50|8094.50|25.88|0.00|3075.50|323.50|349.38|3399.00|3424.88|-3606.00| +2450843|34416|2450870|984|1761548|1759|40723|984|1761548|1759|40723|4|53|17|2|15622|108|1462|66|33.45|91.31|66.65|1627.56|4398.90|2207.70|6026.46|263.93|0.00|120.12|4398.90|4662.83|4519.02|4782.95|2191.20| +2450843|34416|2450857|984|1761548|1759|40723|984|1761548|1759|40723|4|55|12|4|4879|197|1462|42|30.41|34.36|21.30|548.52|894.60|1277.22|1443.12|62.62|0.00|245.28|894.60|957.22|1139.88|1202.50|-382.62| +2450843|34416|2450884|984|1761548|1759|40723|984|1761548|1759|40723|4|47|16|4|5048|140|1462|46|8.55|20.86|18.14|125.12|834.44|393.30|959.56|41.72|0.00|345.00|834.44|876.16|1179.44|1221.16|441.14| +2450843|1210|2450915|53672|1405316|3807|44251|53672|1405316|3807|44251|2|90|18|1|5716|164|1463|81|43.42|99.86|97.86|162.00|7926.66|3517.02|8088.66|158.53|0.00|1051.38|7926.66|8085.19|8978.04|9136.57|4409.64| +2450843|1210|2450868|53672|1405316|3807|44251|53672|1405316|3807|44251|2|58|14|5|14948|25|1463|5|88.33|171.36|56.54|574.10|282.70|441.65|856.80|14.13|0.00|368.40|282.70|296.83|651.10|665.23|-158.95| +2450843|1210|2450907|53672|1405316|3807|44251|53672|1405316|3807|44251|2|87|4|5|9454|277|1463|94|52.99|94.32|21.69|6827.22|2038.86|4981.06|8866.08|40.77|0.00|1950.50|2038.86|2079.63|3989.36|4030.13|-2942.20| +2450843|1210|2450880|53672|1405316|3807|44251|53672|1405316|3807|44251|2|86|8|3|7639|156|1463|32|33.21|92.32|35.08|1831.68|1122.56|1062.72|2954.24|55.00|22.45|856.64|1100.11|1155.11|1956.75|2011.75|37.39| +2450843|1210|2450873|53672|1405316|3807|44251|53672|1405316|3807|44251|2|47|1|5|6679|10|1463|38|13.57|40.43|14.95|968.24|568.10|515.66|1536.34|17.04|0.00|15.20|568.10|585.14|583.30|600.34|52.44| +2450843|1210|2450888|53672|1405316|3807|44251|53672|1405316|3807|44251|2|48|3|2|5342|78|1463|58|85.60|124.97|16.24|6306.34|941.92|4964.80|7248.26|9.41|0.00|1666.92|941.92|951.33|2608.84|2618.25|-4022.88| +2450843|1210|2450883|53672|1405316|3807|44251|53672|1405316|3807|44251|2|55|19|2|7453|130|1463|46|48.40|48.40|9.68|1781.12|445.28|2226.40|2226.40|40.07|0.00|155.48|445.28|485.35|600.76|640.83|-1781.12| +2450843|1210|2450910|53672|1405316|3807|44251|53672|1405316|3807|44251|2|38|16|4|17926|188|1463|83|69.71|86.44|5.18|6744.58|429.94|5785.93|7174.52|17.19|0.00|1434.24|429.94|447.13|1864.18|1881.37|-5355.99| +2450843|1210|2450902|53672|1405316|3807|44251|53672|1405316|3807|44251|2|58|6|2|4432|211|1463|42|50.55|89.47|87.68|75.18|3682.56|2123.10|3757.74|73.65|0.00|1841.28|3682.56|3756.21|5523.84|5597.49|1559.46| +2450843|38661|2450900|36586|1346210|6234|48705|36586|1346210|6234|48705|4|38|10|5|5455|295|1464|33|92.61|115.76|108.81|229.35|3590.73|3056.13|3820.08|287.25|0.00|1642.41|3590.73|3877.98|5233.14|5520.39|534.60| +2450843|38661|2450891|36586|1346210|6234|48705|36586|1346210|6234|48705|4|4|8|3|17366|238|1464|22|30.90|48.20|21.20|594.00|466.40|679.80|1060.40|18.65|0.00|530.20|466.40|485.05|996.60|1015.25|-213.40| +2450843|38661|2450933|36586|1346210|6234|48705|36586|1346210|6234|48705|4|71|6|4|1651|285|1464|85|74.55|99.89|80.91|1613.30|6877.35|6336.75|8490.65|206.32|0.00|3310.75|6877.35|7083.67|10188.10|10394.42|540.60| +2450843|38661|2450858|36586|1346210|6234|48705|36586|1346210|6234|48705|4|105|16|3|4519|257|1464|91|8.31|11.96|3.82|740.74|347.62|756.21|1088.36|14.11|146.00|141.05|201.62|215.73|342.67|356.78|-554.59| +2450843|38661|2450910|36586|1346210|6234|48705|36586|1346210|6234|48705|4|58|16|3|10009|219|1464|80|11.98|15.09|0.15|1195.20|12.00|958.40|1207.20|0.60|0.00|422.40|12.00|12.60|434.40|435.00|-946.40| +2450843|38661|2450892|36586|1346210|6234|48705|36586|1346210|6234|48705|4|103|14|1|13984|242|1464|35|87.96|182.95|34.76|5186.65|1216.60|3078.60|6403.25|36.49|0.00|960.40|1216.60|1253.09|2177.00|2213.49|-1862.00| +2450843|51154|2450857|63963|374562|1608|13054|81948|275099|4545|22744|1|79|7|5|6674|191|1465|20|9.84|14.07|7.87|124.00|157.40|196.80|281.40|4.40|94.44|19.60|62.96|67.36|82.56|86.96|-133.84| +2450843|51154|2450895|63963|374562|1608|13054|81948|275099|4545|22744|1|75|11|5|10993|278|1465|15|67.34|76.76|2.30|1116.90|34.50|1010.10|1151.40|0.07|26.91|103.50|7.59|7.66|111.09|111.16|-1002.51| +2450843|51154|2450848|63963|374562|1608|13054|81948|275099|4545|22744|1|65|2|4|13711|123|1465|23|67.31|73.36|20.54|1214.86|472.42|1548.13|1687.28|14.17|0.00|590.41|472.42|486.59|1062.83|1077.00|-1075.71| +2450843|51154|2450871|63963|374562|1608|13054|81948|275099|4545|22744|1|85|15|5|17305|88|1465|69|97.86|276.94|69.23|14331.99|4776.87|6752.34|19108.86|382.14|0.00|6687.48|4776.87|5159.01|11464.35|11846.49|-1975.47| +2450843|51154|2450860|63963|374562|1608|13054|81948|275099|4545|22744|1|14|9|1|8428|43|1465|29|1.18|3.44|0.51|84.97|14.79|34.22|99.76|0.83|2.81|20.88|11.98|12.81|32.86|33.69|-22.24| +2450843|52861|2450916|94340|272761|5582|12403|97663|1546445|1666|36487|2|42|5|1|14077|88|1466|88|95.98|145.88|78.77|5905.68|6931.76|8446.24|12837.44|277.27|0.00|5519.36|6931.76|7209.03|12451.12|12728.39|-1514.48| +2450843|52861|2450850|94340|272761|5582|12403|97663|1546445|1666|36487|2|37|9|5|16898|110|1466|62|54.90|71.91|54.65|1070.12|3388.30|3403.80|4458.42|203.29|0.00|401.14|3388.30|3591.59|3789.44|3992.73|-15.50| +2450843|52861|2450922|94340|272761|5582|12403|97663|1546445|1666|36487|2|30|4|1|16502|102|1466|25|96.32|206.12|45.34|4019.50|1133.50|2408.00|5153.00|90.68|0.00|1906.50|1133.50|1224.18|3040.00|3130.68|-1274.50| +2450843|52861|2450896|94340|272761|5582|12403|97663|1546445|1666|36487|2|55|11|5|13832|168|1466|71|3.68|8.39|0.58|554.51|41.18|261.28|595.69|0.41|0.00|41.18|41.18|41.59|82.36|82.77|-220.10| +2450843|52861|2450928|94340|272761|5582|12403|97663|1546445|1666|36487|2|25|19|1|10834|129|1466|47|95.90|213.85|149.69|3015.52|7035.43|4507.30|10050.95|0.00|0.00|2512.62|7035.43|7035.43|9548.05|9548.05|2528.13| +2450843|52861|2450867|94340|272761|5582|12403|97663|1546445|1666|36487|2|62|18|3|17470|85|1466|36|99.09|161.51|6.46|5581.80|232.56|3567.24|5814.36|4.65|0.00|2732.40|232.56|237.21|2964.96|2969.61|-3334.68| +2450843|52861|2450933|94340|272761|5582|12403|97663|1546445|1666|36487|2|1|20|1|15520|16|1466|28|93.28|98.87|86.01|360.08|2408.28|2611.84|2768.36|24.08|0.00|1245.72|2408.28|2432.36|3654.00|3678.08|-203.56| +2450843|52861|2450899|94340|272761|5582|12403|97663|1546445|1666|36487|2|101|1|4|13810|3|1466|99|5.62|13.37|5.08|820.71|502.92|556.38|1323.63|2.01|462.68|515.79|40.24|42.25|556.03|558.04|-516.14| +2450843|52861|2450921|94340|272761|5582|12403|97663|1546445|1666|36487|2|18|8|3|10544|100|1466|100|34.13|71.33|53.49|1784.00|5349.00|3413.00|7133.00|320.94|0.00|2068.00|5349.00|5669.94|7417.00|7737.94|1936.00| +2450843|52861|2450873|94340|272761|5582|12403|97663|1546445|1666|36487|2|97|17|3|4396|248|1466|56|73.92|79.09|51.40|1550.64|2878.40|4139.52|4429.04|6.90|2792.04|1594.32|86.36|93.26|1680.68|1687.58|-4053.16| +2450843|52861|2450847|94340|272761|5582|12403|97663|1546445|1666|36487|2|60|17|2|12460|127|1466|10|6.01|11.11|0.22|108.90|2.20|60.10|111.10|0.13|0.00|14.40|2.20|2.33|16.60|16.73|-57.90| +2450843|41765|2450930|36677|497732|2340|10195|36677|497732|2340|10195|4|29|10|4|8851|189|1467|87|62.60|78.25|67.29|953.52|5854.23|5446.20|6807.75|356.52|761.04|408.03|5093.19|5449.71|5501.22|5857.74|-353.01| +2450843|41765|2450855|36677|497732|2340|10195|36677|497732|2340|10195|4|1|1|4|1615|2|1467|81|24.87|41.53|28.24|1076.49|2287.44|2014.47|3363.93|205.86|0.00|1412.64|2287.44|2493.30|3700.08|3905.94|272.97| +2450843|41765|2450866|36677|497732|2340|10195|36677|497732|2340|10195|4|60|15|1|8276|282|1467|59|11.08|26.59|14.09|737.50|831.31|653.72|1568.81|6.98|482.15|736.91|349.16|356.14|1086.07|1093.05|-304.56| +2450843|41765|2450897|36677|497732|2340|10195|36677|497732|2340|10195|4|29|7|3|5606|213|1467|74|55.10|88.16|20.27|5023.86|1499.98|4077.40|6523.84|9.60|1019.98|195.36|480.00|489.60|675.36|684.96|-3597.40| +2450843|41765|2450919|36677|497732|2340|10195|36677|497732|2340|10195|4|34|16|2|8119|42|1467|92|5.81|13.59|11.41|200.56|1049.72|534.52|1250.28|52.48|0.00|512.44|1049.72|1102.20|1562.16|1614.64|515.20| +2450843|41765|2450915|36677|497732|2340|10195|36677|497732|2340|10195|4|14|17|1|11122|54|1467|83|70.70|164.73|80.71|6973.66|6698.93|5868.10|13672.59|200.96|0.00|3144.04|6698.93|6899.89|9842.97|10043.93|830.83| +2450843|41765|2450902|36677|497732|2340|10195|36677|497732|2340|10195|4|8|4|2|15085|191|1467|83|25.20|44.10|11.90|2672.60|987.70|2091.60|3660.30|69.13|0.00|1317.21|987.70|1056.83|2304.91|2374.04|-1103.90| +2450843|41765|2450881|36677|497732|2340|10195|36677|497732|2340|10195|4|63|10|1|17684|227|1467|37|90.95|190.99|40.10|5582.93|1483.70|3365.15|7066.63|133.53|0.00|3391.79|1483.70|1617.23|4875.49|5009.02|-1881.45| +2450843|41765|2450927|36677|497732|2340|10195|36677|497732|2340|10195|4|64|1|5|12913|108|1467|75|25.02|57.54|1.72|4186.50|129.00|1876.50|4315.50|10.32|0.00|258.75|129.00|139.32|387.75|398.07|-1747.50| +2450843|41765|2450869|36677|497732|2340|10195|36677|497732|2340|10195|4|77|6|5|10148|71|1467|81|65.59|141.01|115.62|2056.59|9365.22|5312.79|11421.81|134.85|7866.78|3312.09|1498.44|1633.29|4810.53|4945.38|-3814.35| +2450843|41765|2450913|36677|497732|2340|10195|36677|497732|2340|10195|4|37|9|5|640|112|1467|14|84.43|253.29|30.39|3120.60|425.46|1182.02|3546.06|34.03|0.00|212.66|425.46|459.49|638.12|672.15|-756.56| +2450843|34943|2450882|33111|984242|6220|7351|33111|984242|6220|7351|1|63|15|1|17834|31|1468|81|73.70|93.59|29.01|5230.98|2349.81|5969.70|7580.79|211.48|0.00|605.88|2349.81|2561.29|2955.69|3167.17|-3619.89| +2450843|34943|2450856|33111|984242|6220|7351|33111|984242|6220|7351|1|73|7|2|14725|225|1468|13|40.58|88.46|23.88|839.54|310.44|527.54|1149.98|11.17|124.17|333.45|186.27|197.44|519.72|530.89|-341.27| +2450843|34943|2450868|33111|984242|6220|7351|33111|984242|6220|7351|1|26|20|2|16888|11|1468|37|46.13|93.18|21.43|2654.75|792.91|1706.81|3447.66|63.43|0.00|1240.98|792.91|856.34|2033.89|2097.32|-913.90| +2450843|34943|2450914|33111|984242|6220|7351|33111|984242|6220|7351|1|98|18|4|5792|226|1468|28|3.18|5.78|0.80|139.44|22.40|89.04|161.84|0.00|0.00|25.76|22.40|22.40|48.16|48.16|-66.64| +2450843|34943|2450858|33111|984242|6220|7351|33111|984242|6220|7351|1|74|18|1|5227|137|1468|36|2.76|8.00|4.24|135.36|152.64|99.36|288.00|10.68|0.00|106.56|152.64|163.32|259.20|269.88|53.28| +2450843|34943|2450909|33111|984242|6220|7351|33111|984242|6220|7351|1|28|12|2|11461|204|1468|70|26.59|72.32|2.16|4911.20|151.20|1861.30|5062.40|12.09|0.00|0.00|151.20|163.29|151.20|163.29|-1710.10| +2450843|34943|2450902|33111|984242|6220|7351|33111|984242|6220|7351|1|77|18|1|3586|287|1468|4|68.16|162.90|136.83|104.28|547.32|272.64|651.60|49.25|0.00|71.64|547.32|596.57|618.96|668.21|274.68| +2450843|34943|2450928|33111|984242|6220|7351|33111|984242|6220|7351|1|97|17|1|16927|9|1468|93|53.15|100.45|21.09|7380.48|1961.37|4942.95|9341.85|78.45|0.00|372.93|1961.37|2039.82|2334.30|2412.75|-2981.58| +2450843|34943|2450930|33111|984242|6220|7351|33111|984242|6220|7351|1|91|16|3|7148|220|1468|33|57.80|116.75|115.58|38.61|3814.14|1907.40|3852.75|190.70|0.00|1117.05|3814.14|4004.84|4931.19|5121.89|1906.74| +2450843|34943|2450894|33111|984242|6220|7351|33111|984242|6220|7351|1|61|15|3|1808|220|1468|96|48.98|88.16|21.15|6432.96|2030.40|4702.08|8463.36|182.73|0.00|2792.64|2030.40|2213.13|4823.04|5005.77|-2671.68| +2450843|34943|2450889|33111|984242|6220|7351|33111|984242|6220|7351|1|20|6|1|16471|156|1468|52|71.88|84.09|41.20|2230.28|2142.40|3737.76|4372.68|21.42|0.00|305.76|2142.40|2163.82|2448.16|2469.58|-1595.36| +2450843|34943|2450860|33111|984242|6220|7351|33111|984242|6220|7351|1|59|14|2|17635|111|1468|25|3.49|7.29|7.07|5.50|176.75|87.25|182.25|10.60|0.00|87.25|176.75|187.35|264.00|274.60|89.50| +2450843|34943|2450912|33111|984242|6220|7351|33111|984242|6220|7351|1|82|5|1|14278|182|1468|76|8.17|12.09|3.38|661.96|256.88|620.92|918.84|5.13|0.00|100.32|256.88|262.01|357.20|362.33|-364.04| +2450843|66042|2450903|7797|608844|3844|37691|7797|608844|3844|37691|2|105|3|5|9358|175|1469|65|31.99|55.34|24.34|2015.00|1582.10|2079.35|3597.10|94.92|0.00|431.60|1582.10|1677.02|2013.70|2108.62|-497.25| +2450843|66042|2450882|7797|608844|3844|37691|7797|608844|3844|37691|2|10|15|2|16802|226|1469|92|39.17|82.64|18.18|5930.32|1672.56|3603.64|7602.88|0.00|1254.42|2052.52|418.14|418.14|2470.66|2470.66|-3185.50| +2450843|66042|2450859|7797|608844|3844|37691|7797|608844|3844|37691|2|11|9|4|11053|153|1469|34|74.52|190.77|45.78|4929.66|1556.52|2533.68|6486.18|0.00|0.00|3048.44|1556.52|1556.52|4604.96|4604.96|-977.16| +2450843|66042|2450919|7797|608844|3844|37691|7797|608844|3844|37691|2|47|11|4|1292|180|1469|58|98.04|121.56|81.44|2326.96|4723.52|5686.32|7050.48|0.00|0.00|634.52|4723.52|4723.52|5358.04|5358.04|-962.80| +2450843|15483|2450916|27111|412624|6937|32450|27111|412624|6937|32450|1|74|3|5|12619|85|1470|2|5.92|15.03|4.35|21.36|8.70|11.84|30.06|0.17|0.00|2.70|8.70|8.87|11.40|11.57|-3.14| +2450843|15483|2450886|27111|412624|6937|32450|27111|412624|6937|32450|1|5|14|3|15331|153|1470|31|54.31|148.80|14.88|4151.52|461.28|1683.61|4612.80|8.48|249.09|368.90|212.19|220.67|581.09|589.57|-1471.42| +2450843|15483|2450887|27111|412624|6937|32450|27111|412624|6937|32450|1|2|3|4|2017|28|1470|1|15.35|30.08|15.04|15.04|15.04|15.35|30.08|0.90|0.00|14.43|15.04|15.94|29.47|30.37|-0.31| +2450843|15483|2450909|27111|412624|6937|32450|27111|412624|6937|32450|1|58|12|2|5398|71|1470|27|37.97|108.59|72.75|967.68|1964.25|1025.19|2931.93|137.49|0.00|410.40|1964.25|2101.74|2374.65|2512.14|939.06| +2450843|15483|2450923|27111|412624|6937|32450|27111|412624|6937|32450|1|21|14|5|17248|164|1470|41|62.85|184.15|103.12|3322.23|4227.92|2576.85|7550.15|5.49|3678.29|75.44|549.63|555.12|625.07|630.56|-2027.22| +2450843|15483|2450845|27111|412624|6937|32450|27111|412624|6937|32450|1|4|18|1|811|90|1470|83|68.46|134.86|89.00|3806.38|7387.00|5682.18|11193.38|221.61|0.00|3693.50|7387.00|7608.61|11080.50|11302.11|1704.82| +2450843|15483|2450914|27111|412624|6937|32450|27111|412624|6937|32450|1|3|17|2|7072|234|1470|69|13.81|33.55|29.52|278.07|2036.88|952.89|2314.95|0.00|1548.02|115.23|488.86|488.86|604.09|604.09|-464.03| +2450843|15483|2450927|27111|412624|6937|32450|27111|412624|6937|32450|1|99|19|3|12452|229|1470|3|67.12|67.79|13.55|162.72|40.65|201.36|203.37|1.21|0.00|81.33|40.65|41.86|121.98|123.19|-160.71| +2450843|15483|2450878|27111|412624|6937|32450|27111|412624|6937|32450|1|42|4|5|1610|105|1470|38|18.17|36.52|9.49|1027.14|360.62|690.46|1387.76|3.60|0.00|221.92|360.62|364.22|582.54|586.14|-329.84| +2450843|15483|2450914|27111|412624|6937|32450|27111|412624|6937|32450|1|27|8|2|9547|70|1470|37|75.35|89.66|83.38|232.36|3085.06|2787.95|3317.42|92.55|0.00|497.28|3085.06|3177.61|3582.34|3674.89|297.11| +2450843|15483|2450873|27111|412624|6937|32450|27111|412624|6937|32450|1|53|13|1|13537|212|1470|79|21.18|39.39|9.84|2334.45|777.36|1673.22|3111.81|54.41|0.00|248.85|777.36|831.77|1026.21|1080.62|-895.86| +2450843|15483|2450856|27111|412624|6937|32450|27111|412624|6937|32450|1|22|12|4|3319|206|1470|14|36.02|102.29|101.26|14.42|1417.64|504.28|1432.06|56.70|0.00|200.48|1417.64|1474.34|1618.12|1674.82|913.36| +2450843|15483|2450848|27111|412624|6937|32450|27111|412624|6937|32450|1|76|17|5|3625|199|1470|67|94.22|109.29|95.08|952.07|6370.36|6312.74|7322.43|0.00|3376.29|2635.78|2994.07|2994.07|5629.85|5629.85|-3318.67| +2450843|44346|2450912|6485|852335|2185|24435|6485|852335|2185|24435|4|25|10|1|8587|66|1471|93|55.23|60.75|26.12|3220.59|2429.16|5136.39|5649.75|97.16|0.00|281.79|2429.16|2526.32|2710.95|2808.11|-2707.23| +2450843|44346|2450908|6485|852335|2185|24435|6485|852335|2185|24435|4|82|9|5|1183|11|1471|62|31.98|51.16|48.60|158.72|3013.20|1982.76|3171.92|0.00|0.00|475.54|3013.20|3013.20|3488.74|3488.74|1030.44| +2450843|44346|2450909|6485|852335|2185|24435|6485|852335|2185|24435|4|45|14|4|16384|293|1471|25|16.71|18.54|17.42|28.00|435.50|417.75|463.50|21.77|0.00|134.25|435.50|457.27|569.75|591.52|17.75| +2450843|44346|2450848|6485|852335|2185|24435|6485|852335|2185|24435|4|24|20|3|4372|258|1471|84|8.48|19.84|11.70|683.76|982.80|712.32|1666.56|58.96|0.00|782.88|982.80|1041.76|1765.68|1824.64|270.48| +2450843|44346|2450885|6485|852335|2185|24435|6485|852335|2185|24435|4|61|12|4|8632|238|1471|46|52.13|146.48|10.25|6266.58|471.50|2397.98|6738.08|0.00|0.00|2897.08|471.50|471.50|3368.58|3368.58|-1926.48| +2450843|44346|2450918|6485|852335|2185|24435|6485|852335|2185|24435|4|63|7|2|2044|109|1471|31|42.06|71.08|20.61|1564.57|638.91|1303.86|2203.48|0.00|0.00|1013.39|638.91|638.91|1652.30|1652.30|-664.95| +2450843|44346|2450916|6485|852335|2185|24435|6485|852335|2185|24435|4|21|12|2|2398|271|1471|58|69.93|98.60|28.59|4060.58|1658.22|4055.94|5718.80|149.23|0.00|742.98|1658.22|1807.45|2401.20|2550.43|-2397.72| +2450843|44346|2450915|6485|852335|2185|24435|6485|852335|2185|24435|4|12|15|3|12016|117|1471|10|25.56|72.07|16.57|555.00|165.70|255.60|720.70|9.94|0.00|317.10|165.70|175.64|482.80|492.74|-89.90| +2450843|44346|2450857|6485|852335|2185|24435|6485|852335|2185|24435|4|58|12|3|1906|85|1471|31|70.49|97.98|38.21|1852.87|1184.51|2185.19|3037.38|59.22|0.00|485.77|1184.51|1243.73|1670.28|1729.50|-1000.68| +2450843|44346|2450888|6485|852335|2185|24435|6485|852335|2185|24435|4|8|17|5|13951|221|1471|65|29.21|39.43|2.76|2383.55|179.40|1898.65|2562.95|8.97|0.00|409.50|179.40|188.37|588.90|597.87|-1719.25| +2450843|44346|2450852|6485|852335|2185|24435|6485|852335|2185|24435|4|107|5|4|4088|66|1471|97|75.84|128.92|103.13|2501.63|10003.61|7356.48|12505.24|200.07|0.00|4751.06|10003.61|10203.68|14754.67|14954.74|2647.13| +2450843|34090|2450875|12435|567670|3258|19293|12435|567670|3258|19293|1|22|8|3|16052|260|1472|14|36.63|80.95|46.14|487.34|645.96|512.82|1133.30|58.13|0.00|22.54|645.96|704.09|668.50|726.63|133.14| +2450843|34090|2450845|12435|567670|3258|19293|12435|567670|3258|19293|1|58|19|5|3524|125|1472|48|18.15|24.13|21.23|139.20|1019.04|871.20|1158.24|9.78|692.94|439.68|326.10|335.88|765.78|775.56|-545.10| +2450843|34090|2450882|12435|567670|3258|19293|12435|567670|3258|19293|1|94|16|2|8857|181|1472|90|25.69|66.28|64.29|179.10|5786.10|2312.10|5965.20|520.74|0.00|1491.30|5786.10|6306.84|7277.40|7798.14|3474.00| +2450843|34090|2450920|12435|567670|3258|19293|12435|567670|3258|19293|1|80|5|1|13513|300|1472|39|11.31|29.74|22.30|290.16|869.70|441.09|1159.86|50.53|147.84|533.52|721.86|772.39|1255.38|1305.91|280.77| +2450843|34090|2450862|12435|567670|3258|19293|12435|567670|3258|19293|1|54|1|3|16786|122|1472|3|20.61|34.83|0.69|102.42|2.07|61.83|104.49|0.02|0.00|39.69|2.07|2.09|41.76|41.78|-59.76| +2450843|34090|2450928|12435|567670|3258|19293|12435|567670|3258|19293|1|73|12|1|1150|184|1472|67|12.94|34.16|11.95|1488.07|800.65|866.98|2288.72|4.40|712.57|343.04|88.08|92.48|431.12|435.52|-778.90| +2450843|34090|2450872|12435|567670|3258|19293|12435|567670|3258|19293|1|7|15|1|11150|73|1472|2|98.39|260.73|80.82|359.82|161.64|196.78|521.46|4.84|0.00|203.36|161.64|166.48|365.00|369.84|-35.14| +2450843|34090|2450850|12435|567670|3258|19293|12435|567670|3258|19293|1|76|15|4|17083|295|1472|8|32.02|82.29|41.14|329.20|329.12|256.16|658.32|23.03|0.00|85.52|329.12|352.15|414.64|437.67|72.96| +2450843|34090|2450859|12435|567670|3258|19293|12435|567670|3258|19293|1|57|19|3|10846|145|1472|99|41.21|105.49|81.22|2402.73|8040.78|4079.79|10443.51|723.67|0.00|5117.31|8040.78|8764.45|13158.09|13881.76|3960.99| +2450843|34090|2450846|12435|567670|3258|19293|12435|567670|3258|19293|1|80|6|1|8665|31|1472|94|55.39|76.43|40.50|3377.42|3807.00|5206.66|7184.42|76.14|0.00|3088.84|3807.00|3883.14|6895.84|6971.98|-1399.66| +2450843|34090|2450921|12435|567670|3258|19293|12435|567670|3258|19293|1|56|19|3|2414|168|1472|4|50.10|112.22|8.97|413.00|35.88|200.40|448.88|0.00|0.00|98.72|35.88|35.88|134.60|134.60|-164.52| +2450843|34090|2450860|12435|567670|3258|19293|12435|567670|3258|19293|1|38|13|4|5330|145|1472|98|67.34|169.02|64.22|10270.40|6293.56|6599.32|16563.96|0.00|0.00|828.10|6293.56|6293.56|7121.66|7121.66|-305.76| +2450843|2418|2450875|83065|570648|278|9861|83065|570648|278|9861|2|67|20|3|15889|187|1473|78|17.02|20.59|5.35|1188.72|417.30|1327.56|1606.02|4.17|0.00|481.26|417.30|421.47|898.56|902.73|-910.26| +|2418||83065|570648|||83065||||2||10||5092|89|1473|32||178.96||||||111.09|0.00||||6241.92|6353.01|3600.32| +2450843|2418|2450851|83065|570648|278|9861|83065|570648|278|9861|2|97|18|2|16298|195|1473|17|90.74|246.81|123.40|2097.97|2097.80|1542.58|4195.77|0.00|230.75|1678.24|1867.05|1867.05|3545.29|3545.29|324.47| +2450843|2418|2450850|83065|570648|278|9861|83065|570648|278|9861|2|10|12|3|7040|289|1473|32|98.20|227.82|13.66|6853.12|437.12|3142.40|7290.24|8.74|0.00|1603.84|437.12|445.86|2040.96|2049.70|-2705.28| +2450843|2418|2450911|83065|570648|278|9861|83065|570648|278|9861|2|29|4|4|14582|287|1473|14|71.74|210.91|175.05|502.04|2450.70|1004.36|2952.74|171.54|0.00|620.06|2450.70|2622.24|3070.76|3242.30|1446.34| +2450843|2418|2450864|83065|570648|278|9861|83065|570648|278|9861|2|33|14|4|15892|169|1473|74|45.35|74.37|33.46|3027.34|2476.04|3355.90|5503.38|0.00|0.00|2311.02|2476.04|2476.04|4787.06|4787.06|-879.86| +2450843|31974|2450846|94300|1185920|3587|8854|94300|1185920|3587|8854|1|10|19|5|8737|205|1474|7|3.23|8.78|1.05|54.11|7.35|22.61|61.46|0.44|0.00|23.31|7.35|7.79|30.66|31.10|-15.26| +2450843|31974|2450904|94300|1185920|3587|8854|94300|1185920|3587|8854|1|78|15|4|7357|174|1474|10|46.11|53.48|26.20|272.80|262.00|461.10|534.80|15.72|0.00|240.60|262.00|277.72|502.60|518.32|-199.10| +2450843|31974|2450883|94300|1185920|3587|8854|94300|1185920|3587|8854|1|76|4|4|6121|101|1474|75|96.54|140.94|46.51|7082.25|3488.25|7240.50|10570.50|244.17|0.00|0.00|3488.25|3732.42|3488.25|3732.42|-3752.25| +2450843|31974|2450895|94300|1185920|3587|8854|94300|1185920|3587|8854|1|50|10|3|15328|214|1474|87|55.29|98.96|41.56|4993.80|3615.72|4810.23|8609.52|36.15|0.00|2152.38|3615.72|3651.87|5768.10|5804.25|-1194.51| +2450843|31974|2450877|94300|1185920|3587|8854|94300|1185920|3587|8854|1|34|16|4|15139|163|1474|53|81.97|206.56|198.29|438.31|10509.37|4344.41|10947.68|105.09|0.00|1313.34|10509.37|10614.46|11822.71|11927.80|6164.96| +2450843|31974|2450847|94300|1185920|3587|8854|94300|1185920|3587|8854|1|24|11|3|8714|240|1474|83|83.09|155.37|59.04|7995.39|4900.32|6896.47|12895.71|294.01|0.00|2836.94|4900.32|5194.33|7737.26|8031.27|-1996.15| +2450843|31974|2450903|94300|1185920|3587|8854|94300|1185920|3587|8854|1|11|20|3|14374|187|1474|96|86.97|254.82|168.18|8317.44|16145.28|8349.12|24462.72|322.90|0.00|0.00|16145.28|16468.18|16145.28|16468.18|7796.16| +2450843|31974|2450888|94300|1185920|3587|8854|94300|1185920|3587|8854|1|3|20|1|9616|131|1474|96|20.14|51.15|30.17|2014.08|2896.32|1933.44|4910.40|57.92|0.00|883.20|2896.32|2954.24|3779.52|3837.44|962.88| +2450843|31974|2450923|94300|1185920|3587|8854|94300|1185920|3587|8854|1|79|16|1|5474|86|1474|56|35.12|81.82|65.45|916.72|3665.20|1966.72|4581.92|256.56|0.00|504.00|3665.20|3921.76|4169.20|4425.76|1698.48| +2450843|31974|2450870|94300|1185920|3587|8854|94300|1185920|3587|8854|1|81|1|5|11623|95|1474|50|15.69|44.71|8.49|1811.00|424.50|784.50|2235.50|25.47|0.00|335.00|424.50|449.97|759.50|784.97|-360.00| +2450843|31974|2450905|94300|1185920|3587|8854|94300|1185920|3587|8854|1|45|17|1|2536|179|1474|11|40.14|48.97|1.95|517.22|21.45|441.54|538.67|0.85|0.00|96.91|21.45|22.30|118.36|119.21|-420.09| +2450843|31974|2450865|94300|1185920|3587|8854|94300|1185920|3587|8854|1|27|20|4|4699|180|1474|66|63.95|170.10|61.23|7185.42|4041.18|4220.70|11226.60|40.41|0.00|3030.72|4041.18|4081.59|7071.90|7112.31|-179.52| +2450843|38509|2450922|60875|1345834|40|34199|60875|1345834|40|34199|4|106|18|1|5198|12|1475|45|9.26|25.55|4.85|931.50|218.25|416.70|1149.75|2.22|144.04|264.15|74.21|76.43|338.36|340.58|-342.49| +2450843|38509|2450854|60875|1345834|40|34199|60875|1345834|40|34199|4|63|11|4|10694|253|1475|26|94.76|213.21|29.84|4767.62|775.84|2463.76|5543.46|46.55|0.00|2161.90|775.84|822.39|2937.74|2984.29|-1687.92| +2450843|38509|2450912|60875|1345834|40|34199|60875|1345834|40|34199|4|55|10|2|4369|11|1475|45|26.54|33.70|4.04|1334.70|181.80|1194.30|1516.50|0.00|0.00|136.35|181.80|181.80|318.15|318.15|-1012.50| +2450843|38509|2450845|60875|1345834|40|34199|60875|1345834|40|34199|4|106|13|2|16996|65|1475|73|66.90|78.27|7.04|5199.79|513.92|4883.70|5713.71|25.69|0.00|2113.35|513.92|539.61|2627.27|2652.96|-4369.78| +2450843|38509|2450916|60875|1345834|40|34199|60875|1345834|40|34199|4|74|13|1|260|212|1475|18|16.54|26.13|13.06|235.26|235.08|297.72|470.34|9.21|4.70|23.40|230.38|239.59|253.78|262.99|-67.34| +2450843|38509|2450915|60875|1345834|40|34199|60875|1345834|40|34199|4|32|17|5|1141|7|1475|72|31.72|66.92|13.38|3854.88|963.36|2283.84|4818.24|38.53|0.00|2167.92|963.36|1001.89|3131.28|3169.81|-1320.48| +2450843|38509|2450876|60875|1345834|40|34199|60875|1345834|40|34199|4|35|14|3|1120|74|1475|41|15.38|43.21|22.90|832.71|938.90|630.58|1771.61|51.82|291.05|194.75|647.85|699.67|842.60|894.42|17.27| +2450843|38509|2450857|60875|1345834|40|34199|60875|1345834|40|34199|4|70|10|1|6232|169|1475|35|76.84|99.89|77.91|769.30|2726.85|2689.40|3496.15|27.26|0.00|104.65|2726.85|2754.11|2831.50|2858.76|37.45| +2450843|38509|2450845|60875|1345834|40|34199|60875|1345834|40|34199|4|46|4|5|1216|252|1475|6|68.97|111.04|13.32|586.32|79.92|413.82|666.24|0.00|3.19|79.92|76.73|76.73|156.65|156.65|-337.09| +2450843|38509|2450873|60875|1345834|40|34199|60875|1345834|40|34199|4|57|7|2|11023|190|1475|95|45.69|131.58|78.94|5000.80|7499.30|4340.55|12500.10|299.97|0.00|124.45|7499.30|7799.27|7623.75|7923.72|3158.75| +2450843|38509|2450876|60875|1345834|40|34199|60875|1345834|40|34199|4|25|11|4|13309|177|1475|89|32.42|61.59|31.41|2686.02|2795.49|2885.38|5481.51|223.63|0.00|328.41|2795.49|3019.12|3123.90|3347.53|-89.89| +2450843|38509|2450864|60875|1345834|40|34199|60875|1345834|40|34199|4|35|18|3|8222|108|1475|94|34.01|54.41|44.61|921.20|4193.34|3196.94|5114.54|125.80|0.00|2096.20|4193.34|4319.14|6289.54|6415.34|996.40| +2450843|38509|2450864|60875|1345834|40|34199|60875|1345834|40|34199|4|38|7|4|5732|95|1475|10|23.91|35.38|0.00|353.80|0.00|239.10|353.80|0.00|0.00|14.10|0.00|0.00|14.10|14.10|-239.10| +2450843|43552|2450914|50130|180567|7004|19441|50130|180567|7004|19441|4|25|13|3|9496|125|1476|86|52.97|58.26|24.46|2906.80|2103.56|4555.42|5010.36|168.28|0.00|250.26|2103.56|2271.84|2353.82|2522.10|-2451.86| +2450843|43552|2450927|50130|180567|7004|19441|50130|180567|7004|19441|4|89|5|5|6100|122|1476|54|13.63|31.07|8.38|1225.26|452.52|736.02|1677.78|0.90|362.01|100.44|90.51|91.41|190.95|191.85|-645.51| +2450843|43552|2450920|50130|180567|7004|19441|50130|180567|7004|19441|4|88|7|5|10687|66|1476|14|65.30|190.02|49.40|1968.68|691.60|914.20|2660.28|20.74|0.00|798.00|691.60|712.34|1489.60|1510.34|-222.60| +2450843|43552|2450906|50130|180567|7004|19441|50130|180567|7004|19441|4|19|19|4|10858|102|1476|7|25.93|37.33|35.46|13.09|248.22|181.51|261.31|17.37|0.00|31.29|248.22|265.59|279.51|296.88|66.71| +2450843|43552|2450886|50130|180567|7004|19441|50130|180567|7004|19441|4|48|2|1|13396|47|1476|67|80.42|188.98|119.05|4685.31|7976.35|5388.14|12661.66|110.07|6141.78|2152.04|1834.57|1944.64|3986.61|4096.68|-3553.57| +2450843|43552|2450849|50130|180567|7004|19441|50130|180567|7004|19441|4|100|17|2|3872|87|1476|87|10.59|21.92|9.20|1106.64|800.40|921.33|1907.04|48.02|0.00|609.87|800.40|848.42|1410.27|1458.29|-120.93| +2450843|43552|2450919|50130|180567|7004|19441|50130|180567|7004|19441|4|34|5|1|17053|220|1476|31|68.46|202.64|166.16|1130.88|5150.96|2122.26|6281.84|2.57|4893.41|1067.64|257.55|260.12|1325.19|1327.76|-1864.71| +2450843|43552|2450848|50130|180567|7004|19441|50130|180567|7004|19441|4|15|15|3|670|137|1476|28|16.14|20.82|16.03|134.12|448.84|451.92|582.96|22.44|0.00|128.24|448.84|471.28|577.08|599.52|-3.08| +2450843|43552|2450923|50130|180567|7004|19441|50130|180567|7004|19441|4|107|11|2|5569|217|1476|64|44.96|110.60|97.32|849.92|6228.48|2877.44|7078.40|59.79|4733.64|495.36|1494.84|1554.63|1990.20|2049.99|-1382.60| +2450843|43552|2450929|50130|180567|7004|19441|50130|180567|7004|19441|4|100|6|1|9086|133|1476|58|33.63|60.19|52.36|454.14|3036.88|1950.54|3491.02|182.21|0.00|1396.06|3036.88|3219.09|4432.94|4615.15|1086.34| +2450843|43552|2450867|50130|180567|7004|19441|50130|180567|7004|19441|4|79|20|2|1634|108|1476|30|61.95|114.60|60.73|1616.10|1821.90|1858.50|3438.00|127.53|0.00|378.00|1821.90|1949.43|2199.90|2327.43|-36.60| +2450843|43552|2450868|50130|180567|7004|19441|50130|180567|7004|19441|4|72|3|4|7582|261|1476|15|37.38|52.70|17.39|529.65|260.85|560.70|790.50|23.47|0.00|197.55|260.85|284.32|458.40|481.87|-299.85| +2450843|39804|2450896|5045|1443948|5604|19660|19424|1701419|6722|11578|4|56|15|5|1165|285|1477|35|25.64|25.64|9.23|574.35|323.05|897.40|897.40|25.84|0.00|134.40|323.05|348.89|457.45|483.29|-574.35| +2450843|39804|2450866|5045|1443948|5604|19660|19424|1701419|6722|11578|4|46|6|4|7039|128|1477|80|19.37|25.76|9.53|1298.40|762.40|1549.60|2060.80|45.74|0.00|391.20|762.40|808.14|1153.60|1199.34|-787.20| +2450843|39804|2450921|5045|1443948|5604|19660|19424|1701419|6722|11578|4|88|1|2|9688|279|1477|59|17.78|40.00|31.20|519.20|1840.80|1049.02|2360.00|27.61|1380.60|188.80|460.20|487.81|649.00|676.61|-588.82| +2450843|39804|2450928|5045|1443948|5604|19660|19424|1701419|6722|11578|4|37|18|3|6620|108|1477|39|62.56|185.17|85.17|3900.00|3321.63|2439.84|7221.63|66.43|0.00|1227.33|3321.63|3388.06|4548.96|4615.39|881.79| +2450843|39804|2450911|5045|1443948|5604|19660|19424|1701419|6722|11578|4|12|2|5|5528|217|1477|53|58.46|168.36|60.60|5711.28|3211.80|3098.38|8923.08|96.35|0.00|4461.54|3211.80|3308.15|7673.34|7769.69|113.42| +2450843|39804|2450847|5045|1443948|5604|19660|19424|1701419|6722|11578|4|66|13|4|15452|86|1477|12|61.83|85.94|39.53|556.92|474.36|741.96|1031.28|37.94|0.00|154.68|474.36|512.30|629.04|666.98|-267.60| +2450843|39804|2450884|5045|1443948|5604|19660|19424|1701419|6722|11578|4|98|18|1|6668|137|1477|90|68.26|98.29|42.26|5042.70|3803.40|6143.40|8846.10|190.17|0.00|2387.70|3803.40|3993.57|6191.10|6381.27|-2340.00| +2450843|39804|2450910|5045|1443948|5604|19660|19424|1701419|6722|11578|4|88|9|5|11744|117|1477|7|85.39|174.19|90.57|585.34|633.99|597.73|1219.33|38.03|0.00|170.66|633.99|672.02|804.65|842.68|36.26| +2450843|39804|2450855|5045|1443948|5604|19660|19424|1701419|6722|11578|4|81|7|3|10286|123|1477|80|34.74|75.03|56.27|1500.80|4501.60|2779.20|6002.40|59.42|3016.07|2460.80|1485.53|1544.95|3946.33|4005.75|-1293.67| +2450843|8338|2450869|87879|78131|1983|49835|87879|78131|1983|49835|2|77|20|5|16358|30|1478|27|5.34|10.57|4.22|171.45|113.94|144.18|285.39|3.00|76.33|125.55|37.61|40.61|163.16|166.16|-106.57| +2450843|8338|2450895|87879|78131|1983|49835|87879|78131|1983|49835|2|79|3|1|3824|270|1478|33|71.76|157.15|14.14|4719.33|466.62|2368.08|5185.95|26.03|177.31|0.00|289.31|315.34|289.31|315.34|-2078.77| +2450843|8338|2450880|87879|78131|1983|49835|87879|78131|1983|49835|2|40|2|5|6488|293|1478|17|56.97|160.08|129.66|517.14|2204.22|968.49|2721.36|19.17|1564.99|598.57|639.23|658.40|1237.80|1256.97|-329.26| +2450843|8338|2450872|87879|78131|1983|49835|87879|78131|1983|49835|2|56|12|1|2608|229|1478|2|17.41|43.69|36.69|14.00|73.38|34.82|87.38|2.20|0.00|7.86|73.38|75.58|81.24|83.44|38.56| +2450843|8338|2450851|87879|78131|1983|49835|87879|78131|1983|49835|2|6|2|5|187|114|1478|32|36.96|102.00|81.60|652.80|2611.20|1182.72|3264.00|156.67|0.00|848.64|2611.20|2767.87|3459.84|3616.51|1428.48| +2450843|8338|2450891|87879|78131|1983|49835|87879|78131|1983|49835|2|102|17|2|11131|8|1478|73|72.39|194.72|163.56|2274.68|11939.88|5284.47|14214.56|0.00|0.00|7107.28|11939.88|11939.88|19047.16|19047.16|6655.41| +2450843|8338|2450899|87879|78131|1983|49835|87879|78131|1983|49835|2|65|4|5|5455|113|1478|18|16.41|48.90|37.65|202.50|677.70|295.38|880.20|6.77|0.00|88.02|677.70|684.47|765.72|772.49|382.32| +2450843|8338|2450911|87879|78131|1983|49835|87879|78131|1983|49835|2|52|2|1|17366|291|1478|97|50.53|113.18|28.29|8234.33|2744.13|4901.41|10978.46|137.20|0.00|3732.56|2744.13|2881.33|6476.69|6613.89|-2157.28| +2450843|8338|2450903|87879|78131|1983|49835|87879|78131|1983|49835|2|52|16|1|1651|31|1478|93|19.17|45.04|34.23|1005.33|3183.39|1782.81|4188.72|127.33|0.00|1967.88|3183.39|3310.72|5151.27|5278.60|1400.58| +2450843|8338|2450885|87879|78131|1983|49835|87879|78131|1983|49835|2|96|20|5|4519|78|1478|93|92.02|214.40|85.76|11963.52|7975.68|8557.86|19939.20|478.54|0.00|7177.74|7975.68|8454.22|15153.42|15631.96|-582.18| +2450843|71174|2450892|87846|1270191|6489|48787|87846|1270191|6489|48787|2|41|1|5|16957|190|1479|62|65.74|151.20|54.43|5999.74|3374.66|4075.88|9374.40|202.47|0.00|4124.24|3374.66|3577.13|7498.90|7701.37|-701.22| +2450843|71174|2450933|87846|1270191|6489|48787|87846|1270191|6489|48787|2|97|19|5|9337|250|1479|44|1.58|3.79|0.98|123.64|43.12|69.52|166.76|0.00|0.00|29.92|43.12|43.12|73.04|73.04|-26.40| +2450843|71174|2450852|87846|1270191|6489|48787|87846|1270191|6489|48787|2|91|11|4|6658|167|1479|37|10.14|30.11|0.90|1080.77|33.30|375.18|1114.07|0.00|23.31|445.48|9.99|9.99|455.47|455.47|-365.19| +2450843|71174|2450870|87846|1270191|6489|48787|87846|1270191|6489|48787|2|51|9|5|1184|174|1479|29|11.46|34.38|27.16|209.38|787.64|332.34|997.02|63.01|0.00|39.73|787.64|850.65|827.37|890.38|455.30| +2450843|71174|2450930|87846|1270191|6489|48787|87846|1270191|6489|48787|2|47|7|4|15160|281|1479|98|16.60|35.35|17.67|1732.64|1731.66|1626.80|3464.30|34.63|0.00|1489.60|1731.66|1766.29|3221.26|3255.89|104.86| +2450843|71174|2450872|87846|1270191|6489|48787|87846|1270191|6489|48787|2|71|16|3|3154|179|1479|85|6.35|10.03|8.42|136.85|715.70|539.75|852.55|21.47|0.00|400.35|715.70|737.17|1116.05|1137.52|175.95| +2450843|71174|2450916|87846|1270191|6489|48787|87846|1270191|6489|48787|2|77|7|2|5560|138|1479|61|61.01|81.14|78.70|148.84|4800.70|3721.61|4949.54|336.04|0.00|2474.77|4800.70|5136.74|7275.47|7611.51|1079.09| +2450843|71174|2450896|87846|1270191|6489|48787|87846|1270191|6489|48787|2|99|12|1|11764|195|1479|42|88.44|240.55|72.16|7072.38|3030.72|3714.48|10103.10|212.15|0.00|3940.02|3030.72|3242.87|6970.74|7182.89|-683.76| +2450843|71174|2450903|87846|1270191|6489|48787|87846|1270191|6489|48787|2|7|5|1|13414|61|1479|28|8.39|16.69|3.83|360.08|107.24|234.92|467.32|2.14|0.00|158.76|107.24|109.38|266.00|268.14|-127.68| +2450843|38166|2450889|65517|608961|1205|19849|65517|608961|1205|19849|2|34|13|3|6730|180|1480|32|94.40|106.67|33.06|2355.52|1057.92|3020.80|3413.44|58.39|327.95|409.60|729.97|788.36|1139.57|1197.96|-2290.83| +2450843|38166|2450905|65517|608961|1205|19849|65517|608961|1205|19849|2|34|20|1|3442|195|1480|37|81.40|126.98|72.37|2020.57|2677.69|3011.80|4698.26|187.43|0.00|1785.25|2677.69|2865.12|4462.94|4650.37|-334.11| +2450843|38166|2450848|65517|608961|1205|19849|65517|608961|1205|19849|2|59|14|3|8356|206|1480|59|82.55|88.32|15.01|4325.29|885.59|4870.45|5210.88|8.85|0.00|1302.72|885.59|894.44|2188.31|2197.16|-3984.86| +2450843|38166|2450887|65517|608961|1205|19849|65517|608961|1205|19849|2|64|8|3|2707|135|1480|63|9.43|21.59|1.72|1251.81|108.36|594.09|1360.17|7.58|0.00|203.49|108.36|115.94|311.85|319.43|-485.73| +2450843|38166|2450922|65517|608961|1205|19849|65517|608961|1205|19849|2|18|4|3|12733|127|1480|9|63.37|152.72|126.75|233.73|1140.75|570.33|1374.48|91.26|0.00|357.30|1140.75|1232.01|1498.05|1589.31|570.42| +2450843|38166|2450911|65517|608961|1205|19849|65517|608961|1205|19849|2|38|13|3|16718|23|1480|17|22.76|44.60|14.71|508.13|250.07|386.92|758.20|0.00|0.00|250.07|250.07|250.07|500.14|500.14|-136.85| +2450843|38166|2450852|65517|608961|1205|19849|65517|608961|1205|19849|2|80|18|1|5545|201|1480|76|20.65|38.40|0.76|2860.64|57.76|1569.40|2918.40|4.04|0.00|87.40|57.76|61.80|145.16|149.20|-1511.64| +2450843|38166|2450928|65517|608961|1205|19849|65517|608961|1205|19849|2|51|15|1|17815|235|1480|61|41.03|98.88|70.20|1749.48|4282.20|2502.83|6031.68|385.39|0.00|2834.67|4282.20|4667.59|7116.87|7502.26|1779.37| +2450843|38166|2450906|65517|608961|1205|19849|65517|608961|1205|19849|2|99|4|1|10975|47|1480|39|4.09|4.13|3.75|14.82|146.25|159.51|161.07|4.38|0.00|67.47|146.25|150.63|213.72|218.10|-13.26| +2450843|38166|2450866|65517|608961|1205|19849|65517|608961|1205|19849|2|2|5|4|15148|148|1480|92|20.62|28.45|23.32|471.96|2145.44|1897.04|2617.40|0.00|0.00|627.44|2145.44|2145.44|2772.88|2772.88|248.40| +2450843|38166|2450854|65517|608961|1205|19849|65517|608961|1205|19849|2|14|20|5|4972|187|1480|94|12.59|23.16|6.94|1524.68|652.36|1183.46|2177.04|32.61|0.00|304.56|652.36|684.97|956.92|989.53|-531.10| +2450844|83002|2450852|10536|1471645|4307|12616|10536|1471645|4307|12616|2|94|20|4|2509|172|1481|80|11.52|13.70|11.09|208.80|887.20|921.60|1096.00|8.16|683.14|175.20|204.06|212.22|379.26|387.42|-717.54| +2450844|83002|2450892|10536|1471645|4307|12616|10536|1471645|4307|12616|2|103|1|1|11491|16|1481|56|34.21|84.49|36.33|2696.96|2034.48|1915.76|4731.44|0.00|0.00|1892.24|2034.48|2034.48|3926.72|3926.72|118.72| +2450844|83002|2450869|10536|1471645|4307|12616|10536|1471645|4307|12616|2|59|3|2|8372|61|1481|73|98.68|236.83|220.25|1210.34|16078.25|7203.64|17288.59|160.78|0.00|7260.58|16078.25|16239.03|23338.83|23499.61|8874.61| +2450844|83002|2450899|10536|1471645|4307|12616|10536|1471645|4307|12616|2|92|5|2|6592|46|1481|100|85.11|108.08|108.08|0.00|10808.00|8511.00|10808.00|972.72|0.00|0.00|10808.00|11780.72|10808.00|11780.72|2297.00| +2450844|83002|2450869|10536|1471645|4307|12616|10536|1471645|4307|12616|2|14|11|1|8305|143|1481|22|24.13|50.43|18.15|710.16|399.30|530.86|1109.46|2.27|171.69|155.32|227.61|229.88|382.93|385.20|-303.25| +2450844|69758|2450920|35825|604608|4591|10574|35825|604608|4591|10574|4|91|16|4|7808|259|1482|90|43.30|80.10|43.25|3316.50|3892.50|3897.00|7209.00|38.92|0.00|2522.70|3892.50|3931.42|6415.20|6454.12|-4.50| +2450844|69758|2450906|35825|604608|4591|10574|35825|604608|4591|10574|4|108|4|2|14348|176|1482|87|71.63|88.82|82.60|541.14|7186.20|6231.81|7727.34|287.44|0.00|308.85|7186.20|7473.64|7495.05|7782.49|954.39| +2450844|69758|2450847|35825|604608|4591|10574|35825|604608|4591|10574|4|65|18|2|14102|226|1482|77|56.89|114.91|66.64|3716.79|5131.28|4380.53|8848.07|153.93|0.00|1769.46|5131.28|5285.21|6900.74|7054.67|750.75| +2450844|69758|2450903|35825|604608|4591|10574|35825|604608|4591|10574|4|84|12|4|6122|103|1482|50|50.74|117.71|12.94|5238.50|647.00|2537.00|5885.50|0.00|381.73|1883.00|265.27|265.27|2148.27|2148.27|-2271.73| +2450844|69758|2450930|35825|604608|4591|10574|35825|604608|4591|10574|4|5|1|3|2570|8|1482|65|77.55|177.58|5.32|11196.90|345.80|5040.75|11542.70|24.20|0.00|5194.15|345.80|370.00|5539.95|5564.15|-4694.95| +2450844|69758|2450873|35825|604608|4591|10574|35825|604608|4591|10574|4|10|5|5|15949|102|1482|11|80.23|158.85|128.66|332.09|1415.26|882.53|1747.35|14.15|0.00|856.13|1415.26|1429.41|2271.39|2285.54|532.73| +2450844|69758|2450891|35825|604608|4591|10574|35825|604608|4591|10574|4|81|1|3|8137|155|1482|37|58.74|158.59|42.81|4283.86|1583.97|2173.38|5867.83|142.55|0.00|645.28|1583.97|1726.52|2229.25|2371.80|-589.41| +2450844|69758|2450874|35825|604608|4591|10574|35825|604608|4591|10574|4|8|4|2|8596|285|1482|71|92.77|245.84|100.79|10298.55|7156.09|6586.67|17454.64|31.48|5581.75|2268.45|1574.34|1605.82|3842.79|3874.27|-5012.33| +2450844|69758|2450850|35825|604608|4591|10574|35825|604608|4591|10574|4|4|19|3|8992|127|1482|43|28.89|71.35|29.25|1810.30|1257.75|1242.27|3068.05|0.00|0.00|490.63|1257.75|1257.75|1748.38|1748.38|15.48| +2450844|69758|2450866|35825|604608|4591|10574|35825|604608|4591|10574|4|63|3|3|4774|130|1482|6|58.15|74.43|57.31|102.72|343.86|348.90|446.58|24.07|0.00|214.32|343.86|367.93|558.18|582.25|-5.04| +2450844|18464|2450891|71827|149426|7025|15824|71827|149426|7025|15824|2|28|17|4|3556|91|1483|65|58.95|62.48|42.48|1300.00|2761.20|3831.75|4061.20|0.00|0.00|202.80|2761.20|2761.20|2964.00|2964.00|-1070.55| +2450844|18464|2450853|71827|149426|7025|15824|71827|149426|7025|15824|2|13|20|3|13004|12|1483|3|5.52|8.72|0.87|23.55|2.61|16.56|26.16|0.18|0.00|2.61|2.61|2.79|5.22|5.40|-13.95| +2450844|18464|2450902|71827|149426|7025|15824|71827|149426|7025|15824|2|74|19|2|14314|98|1483|93|42.91|48.48|39.26|857.46|3651.18|3990.63|4508.64|36.51|0.00|810.96|3651.18|3687.69|4462.14|4498.65|-339.45| +2450844|18464|2450905|71827|149426|7025|15824|71827|149426|7025|15824|2|49|5|3|13474|241|1483|4|75.32|151.39|84.77|266.48|339.08|301.28|605.56|30.51|0.00|102.92|339.08|369.59|442.00|472.51|37.80| +2450844|18464|2450882|71827|149426|7025|15824|71827|149426|7025|15824|2|75|9|2|4928|63|1483|31|19.75|39.69|26.98|394.01|836.38|612.25|1230.39|66.91|0.00|221.34|836.38|903.29|1057.72|1124.63|224.13| +2450844|18464|2450858|71827|149426|7025|15824|71827|149426|7025|15824|2|32|1|1|5155|110|1483|20|74.41|183.04|100.67|1647.40|2013.40|1488.20|3660.80|0.00|0.00|768.60|2013.40|2013.40|2782.00|2782.00|525.20| +2450844|18464|2450901|71827|149426|7025|15824|71827|149426|7025|15824|2|77|3|1|16454|205|1483|78|78.41|210.92|10.54|15629.64|822.12|6115.98|16451.76|57.54|0.00|657.54|822.12|879.66|1479.66|1537.20|-5293.86| +2450844|18464|2450917|71827|149426|7025|15824|71827|149426|7025|15824|2|53|9|3|500|233|1483|47|78.29|105.69|26.42|3725.69|1241.74|3679.63|4967.43|49.66|0.00|49.35|1241.74|1291.40|1291.09|1340.75|-2437.89| +2450844|18464|2450877|71827|149426|7025|15824|71827|149426|7025|15824|2|32|8|5|4564|273|1483|48|37.69|58.04|4.64|2563.20|222.72|1809.12|2785.92|11.13|0.00|724.32|222.72|233.85|947.04|958.17|-1586.40| +2450844|18464|2450892|71827|149426|7025|15824|71827|149426|7025|15824|2|75|10|1|16324|163|1483|87|3.31|5.52|3.03|216.63|263.61|287.97|480.24|18.45|0.00|4.35|263.61|282.06|267.96|286.41|-24.36| +2450844|63272|2450874|54778|51865|5399|12486|54778|51865|5399|12486|4|63|16|1|7165|293|1484|58|90.90|145.44|114.89|1771.90|6663.62|5272.20|8435.52|266.54|0.00|2192.98|6663.62|6930.16|8856.60|9123.14|1391.42| +2450844|63272|2450866|54778|51865|5399|12486|54778|51865|5399|12486|4|49|2|5|3394|161|1484|28|10.13|14.08|11.12|82.88|311.36|283.64|394.24|0.00|211.72|153.72|99.64|99.64|253.36|253.36|-184.00| +2450844|63272|2450912|54778|51865|5399|12486|54778|51865|5399|12486|4|86|17|1|4246|86|1484|99|77.42|213.67|70.51|14172.84|6980.49|7664.58|21153.33|0.00|0.00|9941.58|6980.49|6980.49|16922.07|16922.07|-684.09| +2450844|63272|2450922|54778|51865|5399|12486|54778|51865|5399|12486|4|83|13|3|5032|300|1484|3|76.01|90.45|80.50|29.85|241.50|228.03|271.35|12.07|0.00|130.23|241.50|253.57|371.73|383.80|13.47| +2450844|63272|2450868|54778|51865|5399|12486|54778|51865|5399|12486|4|82|9|2|5497|236|1484|45|18.07|47.16|32.54|657.90|1464.30|813.15|2122.20|6.58|1391.08|551.70|73.22|79.80|624.92|631.50|-739.93| +2450844|63272|2450884|54778|51865|5399|12486|54778|51865|5399|12486|4|54|10|1|9832|73|1484|61|27.76|47.46|16.13|1911.13|983.93|1693.36|2895.06|36.01|383.73|636.84|600.20|636.21|1237.04|1273.05|-1093.16| +2450844|43669|2450880|5804|115467|4330|28393|5804|115467|4330|28393|1|65|8|5|17365|29|1485|55|21.23|49.04|33.83|836.55|1860.65|1167.65|2697.20|130.24|0.00|782.10|1860.65|1990.89|2642.75|2772.99|693.00| +2450844|43669|2450903|5804|115467|4330|28393|5804|115467|4330|28393|1|4|8|2|676|82|1485|65|84.18|244.96|63.68|11783.20|4139.20|5471.70|15922.40|331.13|0.00|6846.45|4139.20|4470.33|10985.65|11316.78|-1332.50| +2450844|43669|2450908|5804|115467|4330|28393|5804|115467|4330|28393|1|82|2|5|4387|146|1485|60|4.76|8.99|7.64|81.00|458.40|285.60|539.40|0.00|0.00|118.20|458.40|458.40|576.60|576.60|172.80| +2450844|43669|2450872|5804|115467|4330|28393|5804|115467|4330|28393|1|67|5|4|2770|252|1485|84|72.18|122.70|11.04|9379.44|927.36|6063.12|10306.80|64.91|0.00|4843.44|927.36|992.27|5770.80|5835.71|-5135.76| +2450844|43669|2450933|5804|115467|4330|28393|5804|115467|4330|28393|1|43|9|2|7483|101|1485|89|15.94|21.67|1.30|1812.93|115.70|1418.66|1928.63|6.94|0.00|597.19|115.70|122.64|712.89|719.83|-1302.96| +|43669|2450917|5804|115467|4330||||4330|||40|||6556|5|1485|||133.12|||2265.14||9850.88|||1280.20||2355.74|3545.34|3635.94|-3364.04| +2450844|43669|2450905|5804|115467|4330|28393|5804|115467|4330|28393|1|77|8|4|6362|177|1485|51|35.68|84.56|4.22|4097.34|215.22|1819.68|4312.56|4.30|0.00|1422.90|215.22|219.52|1638.12|1642.42|-1604.46| +2450844|43669|2450848|5804|115467|4330|28393|5804|115467|4330|28393|1|97|6|4|14011|45|1485|50|71.40|75.68|5.29|3519.50|264.50|3570.00|3784.00|12.85|50.25|529.50|214.25|227.10|743.75|756.60|-3355.75| +2450844|43669|2450876|5804|115467|4330|28393|5804|115467|4330|28393|1|32|5|5|17119|108|1485|74|59.08|67.35|53.20|1047.10|3936.80|4371.92|4983.90|354.31|0.00|2342.10|3936.80|4291.11|6278.90|6633.21|-435.12| +2450844|43669|2450889|5804|115467|4330|28393|5804|115467|4330|28393|1|94|9|3|3010|29|1485|84|68.16|122.00|120.78|102.48|10145.52|5725.44|10248.00|710.18|0.00|4509.12|10145.52|10855.70|14654.64|15364.82|4420.08| +2450844|67445|2450913|5497|1498498|6252|453|5497|1498498|6252|453|4|59|14|5|704|285|1486|31|17.42|20.38|10.19|315.89|315.89|540.02|631.78|12.63|0.00|233.74|315.89|328.52|549.63|562.26|-224.13| +2450844|67445|2450867|5497|1498498|6252|453|5497|1498498|6252|453|4|97|10|5|4574|1|1486|56|2.18|2.63|1.63|56.00|91.28|122.08|147.28|8.21|0.00|47.04|91.28|99.49|138.32|146.53|-30.80| +2450844|67445|2450915|5497|1498498|6252|453|5497|1498498|6252|453|4|54|3|4|3301|70|1486|15|80.37|133.41|129.40|60.15|1941.00|1205.55|2001.15|174.69|0.00|620.25|1941.00|2115.69|2561.25|2735.94|735.45| +2450844|67445|2450903|5497|1498498|6252|453|5497|1498498|6252|453|4|40|19|1|7382|210|1486|40|45.15|108.81|75.07|1349.60|3002.80|1806.00|4352.40|90.08|0.00|2132.40|3002.80|3092.88|5135.20|5225.28|1196.80| +2450844|67445|2450879|5497|1498498|6252|453|5497|1498498|6252|453|4|67|17|3|7556|132|1486|11|57.79|160.07|134.45|281.82|1478.95|635.69|1760.77|0.00|0.00|457.71|1478.95|1478.95|1936.66|1936.66|843.26| +2450844|67445|2450910|5497|1498498|6252|453|5497|1498498|6252|453|4|54|11|5|10352|125|1486|51|5.72|10.06|5.23|246.33|266.73|291.72|513.06|18.67|0.00|153.51|266.73|285.40|420.24|438.91|-24.99| +2450844|67445|2450870|5497|1498498|6252|453|5497|1498498|6252|453|4|65|20|4|6559|69|1486|10|99.36|173.88|73.02|1008.60|730.20|993.60|1738.80|38.84|175.24|730.20|554.96|593.80|1285.16|1324.00|-438.64| +2450844|33893|2450860|71088|1667592|449|5753|71088|1667592|449|5753|2|19|14|3|2918|154|1487|95|77.79|151.69|66.74|8070.25|6340.30|7390.05|14410.55|443.82|0.00|5619.25|6340.30|6784.12|11959.55|12403.37|-1049.75| +2450844|33893|2450918|71088|1667592|449|5753|71088|1667592|449|5753|2|52|4|3|11336|60|1487|75|21.25|37.40|8.60|2160.00|645.00|1593.75|2805.00|22.63|393.45|140.25|251.55|274.18|391.80|414.43|-1342.20| +2450844|33893|2450913|71088|1667592|449|5753|71088|1667592|449|5753|2|34|8|2|12331|73|1487|26|80.84|123.68|87.81|932.62|2283.06|2101.84|3215.68|0.00|0.00|514.28|2283.06|2283.06|2797.34|2797.34|181.22| +2450844|33893|2450864|71088|1667592|449|5753|71088|1667592|449|5753|2|17|8|5|5576|50|1487|41|95.23|163.79|32.75|5372.64|1342.75|3904.43|6715.39|13.42|0.00|1947.09|1342.75|1356.17|3289.84|3303.26|-2561.68| +2450844|33893|2450851|71088|1667592|449|5753|71088|1667592|449|5753|2|99|1|5|17528|91|1487|21|73.96|124.99|14.99|2310.00|314.79|1553.16|2624.79|15.73|0.00|577.29|314.79|330.52|892.08|907.81|-1238.37| +2450844|33893|2450933|71088|1667592|449|5753|71088|1667592|449|5753|2|25|19|5|17864|108|1487|37|2.54|5.71|4.73|36.26|175.01|93.98|211.27|2.53|124.25|78.07|50.76|53.29|128.83|131.36|-43.22| +2450844|33893|2450861|71088|1667592|449|5753|71088|1667592|449|5753|2|21|12|2|5932|292|1487|87|55.22|57.98|23.77|2976.27|2067.99|4804.14|5044.26|124.07|0.00|251.43|2067.99|2192.06|2319.42|2443.49|-2736.15| +2450844|33893|2450928|71088|1667592|449|5753|71088|1667592|449|5753|2|14|20|2|15842|129|1487|39|83.61|164.71|146.59|706.68|5717.01|3260.79|6423.69|228.68|0.00|385.32|5717.01|5945.69|6102.33|6331.01|2456.22| +2450844|33893|2450859|71088|1667592|449|5753|71088|1667592|449|5753|2|54|6|5|4412|286|1487|9|69.13|192.18|7.68|1660.50|69.12|622.17|1729.62|0.00|0.00|726.39|69.12|69.12|795.51|795.51|-553.05| +2450844|42762|2450907|83368|1236773|578|28783|27168|1474393|2198|6003|2|70|18|2|148|36|1488|46|93.97|125.91|105.76|926.90|4864.96|4322.62|5791.86|194.59|0.00|2779.78|4864.96|5059.55|7644.74|7839.33|542.34| +2450844|42762|2450872|83368|1236773|578|28783|27168|1474393|2198|6003|2|61|4|1|1696|12|1488|41|42.35|65.21|40.43|1015.98|1657.63|1736.35|2673.61|99.45|0.00|614.59|1657.63|1757.08|2272.22|2371.67|-78.72| +2450844|42762|2450905|83368|1236773|578|28783|27168|1474393|2198|6003|2|89|4|3|8044|108|1488|24|71.63|122.48|93.08|705.60|2233.92|1719.12|2939.52|44.67|0.00|323.28|2233.92|2278.59|2557.20|2601.87|514.80| +2450844|42762|2450915|83368|1236773|578|28783|27168|1474393|2198|6003|2|9|7|4|10172|279|1488|97|84.68|217.62|45.70|16676.24|4432.90|8213.96|21109.14|44.32|0.00|3166.08|4432.90|4477.22|7598.98|7643.30|-3781.06| +2450844|42762|2450895|83368|1236773|578|28783|27168|1474393|2198|6003|2|72|15|1|9002|61|1488|100|4.96|12.74|8.02|472.00|802.00|496.00|1274.00|32.08|0.00|76.00|802.00|834.08|878.00|910.08|306.00| +2450844|42762|2450859|83368|1236773|578|28783|27168|1474393|2198|6003|2|38|13|5|2188|287|1488|43|21.46|30.68|19.94|461.82|857.42|922.78|1319.24|25.72|0.00|224.03|857.42|883.14|1081.45|1107.17|-65.36| +2450844|31379|2450877|80671|570355|2792|17676|80671|570355|2792|17676|2|89|5|5|314|201|1489|41|92.87|207.10|35.20|7047.90|1443.20|3807.67|8491.10|72.16|0.00|3820.79|1443.20|1515.36|5263.99|5336.15|-2364.47| +2450844|31379|2450872|80671|570355|2792|17676|80671|570355|2792|17676|2|2|7|1|8173|194|1489|45|66.38|69.69|62.72|313.65|2822.40|2987.10|3136.05|254.01|0.00|188.10|2822.40|3076.41|3010.50|3264.51|-164.70| +2450844|31379|2450908|80671|570355|2792|17676|80671|570355|2792|17676|2|28|9|2|5876|168|1489|6|43.87|128.97|119.94|54.18|719.64|263.22|773.82|28.78|0.00|61.86|719.64|748.42|781.50|810.28|456.42| +2450844|31379|2450866|80671|570355|2792|17676|80671|570355|2792|17676|2|33|4|2|12385|116|1489|83|45.43|54.51|18.53|2986.34|1537.99|3770.69|4524.33|26.45|215.31|1220.93|1322.68|1349.13|2543.61|2570.06|-2448.01| +2450844|31379|2450918|80671|570355|2792|17676|80671|570355|2792|17676|2|66|17|3|13339|117|1489|19|32.12|94.43|1.88|1758.45|35.72|610.28|1794.17|0.35|0.00|17.86|35.72|36.07|53.58|53.93|-574.56| +2450844|31379|2450881|80671|570355|2792|17676|80671|570355|2792|17676|2|74|10|5|8020|60|1489|36|30.75|35.67|23.18|449.64|834.48|1107.00|1284.12|0.00|834.48|166.68|0.00|0.00|166.68|166.68|-1107.00| +2450844|31379|2450924|80671|570355|2792|17676|80671|570355|2792|17676|2|17|16|1|2480|17|1489|10|99.88|265.68|146.12|1195.60|1461.20|998.80|2656.80|131.50|0.00|292.20|1461.20|1592.70|1753.40|1884.90|462.40| +2450844|31379|2450848|80671|570355|2792|17676|80671|570355|2792|17676|2|81|15|2|1957|230|1489|33|98.09|111.82|26.83|2804.67|885.39|3236.97|3690.06|53.12|0.00|1217.70|885.39|938.51|2103.09|2156.21|-2351.58| +2450844|31379|2450870|80671|570355|2792|17676|80671|570355|2792|17676|2|16|3|2|15484|197|1489|44|95.57|129.01|61.92|2951.96|2724.48|4205.08|5676.44|245.20|0.00|2554.20|2724.48|2969.68|5278.68|5523.88|-1480.60| +2450844|20058|2450895|42732|1337154|3129|16304|42732|1337154|3129|16304|2|49|20|4|12289|290|1490|77|71.51|199.51|181.55|1382.92|13979.35|5506.27|15362.27|279.58|0.00|6759.06|13979.35|14258.93|20738.41|21017.99|8473.08| +2450844|20058|2450933|42732|1337154|3129|16304|42732|1337154|3129|16304|2|26|9|4|12440|54|1490|91|86.19|195.65|78.26|10682.49|7121.66|7843.29|17804.15|0.00|0.00|5874.96|7121.66|7121.66|12996.62|12996.62|-721.63| +2450844|20058|2450892|42732|1337154|3129|16304|42732|1337154|3129|16304|2|92|14|4|7622|34|1490|62|87.61|181.35|108.81|4497.48|6746.22|5431.82|11243.70|0.00|0.00|3597.86|6746.22|6746.22|10344.08|10344.08|1314.40| +2450844|20058|2450880|42732|1337154|3129|16304|42732|1337154|3129|16304|2|80|8|5|698|171|1490|96|75.33|210.92|179.28|3037.44|17210.88|7231.68|20248.32|1204.76|0.00|1822.08|17210.88|18415.64|19032.96|20237.72|9979.20| +2450844|20058|2450915|42732|1337154|3129|16304|42732|1337154|3129|16304|2|64|1|2|5881|26|1490|41|91.34|162.58|81.29|3332.89|3332.89|3744.94|6665.78|0.00|0.00|1066.41|3332.89|3332.89|4399.30|4399.30|-412.05| +2450844|20058|2450927|42732|1337154|3129|16304|42732|1337154|3129|16304|2|24|3|2|5761|135|1490|19|96.50|99.39|86.46|245.67|1642.74|1833.50|1888.41|98.56|0.00|207.67|1642.74|1741.30|1850.41|1948.97|-190.76| +2450844|20058|2450874|42732|1337154|3129|16304|42732|1337154|3129|16304|2|20|8|5|16184|279|1490|99|77.23|121.25|16.97|10323.72|1680.03|7645.77|12003.75|134.40|0.00|3240.27|1680.03|1814.43|4920.30|5054.70|-5965.74| +2450844|20058|2450882|42732|1337154|3129|16304|42732|1337154|3129|16304|2|108|11|2|7405|243|1490|83|14.66|27.56|22.32|434.92|1852.56|1216.78|2287.48|53.90|55.57|571.87|1796.99|1850.89|2368.86|2422.76|580.21| +2450844|20058|2450862|42732|1337154|3129|16304|42732|1337154|3129|16304|2|78|5|1|13388|95|1490|12|71.96|73.39|31.55|502.08|378.60|863.52|880.68|3.78|0.00|123.24|378.60|382.38|501.84|505.62|-484.92| +2450844|20058|2450897|42732|1337154|3129|16304|42732|1337154|3129|16304|2|33|2|1|3901|112|1490|44|14.31|30.48|0.60|1314.72|26.40|629.64|1341.12|0.26|0.00|106.92|26.40|26.66|133.32|133.58|-603.24| +2450844|20058|2450859|42732|1337154|3129|16304|42732|1337154|3129|16304|2|73|6|4|235|200|1490|87|19.10|37.24|24.57|1102.29|2137.59|1661.70|3239.88|81.22|513.02|1554.69|1624.57|1705.79|3179.26|3260.48|-37.13| +2450844|21870|2450857|69384|1594859|2287|20515|69384|1594859|2287|20515|4|5|20|2|11962|1|1491|51|71.63|104.57|77.38|1386.69|3946.38|3653.13|5333.07|78.92|0.00|906.27|3946.38|4025.30|4852.65|4931.57|293.25| +2450844|21870|2450913|69384|1594859|2287|20515|69384|1594859|2287|20515|4|96|7|5|6154|211|1491|42|69.28|158.65|41.24|4931.22|1732.08|2909.76|6663.30|34.64|0.00|799.26|1732.08|1766.72|2531.34|2565.98|-1177.68| +2450844|21870|2450859|69384|1594859|2287|20515|69384|1594859|2287|20515|4|43|1|1|10880|214|1491|57|61.23|180.01|99.00|4617.57|5643.00|3490.11|10260.57|446.92|677.16|307.80|4965.84|5412.76|5273.64|5720.56|1475.73| +2450844|21870|2450873|69384|1594859|2287|20515|69384|1594859|2287|20515|4|81|3|3|16208|103|1491|45|55.21|78.95|75.00|177.75|3375.00|2484.45|3552.75|303.75|0.00|0.00|3375.00|3678.75|3375.00|3678.75|890.55| +2450844|21870|2450897|69384|1594859|2287|20515|69384|1594859|2287|20515|4|73|14|2|6529|259|1491|78|92.44|180.25|72.10|8435.70|5623.80|7210.32|14059.50|168.71|0.00|3374.28|5623.80|5792.51|8998.08|9166.79|-1586.52| +2450844|21870|2450924|69384|1594859|2287|20515|69384|1594859|2287|20515|4|44|8|2|10292|165|1491|98|87.67|257.74|231.96|2526.44|22732.08|8591.66|25258.52|1591.24|0.00|2272.62|22732.08|24323.32|25004.70|26595.94|14140.42| +2450844|21870|2450923|69384|1594859|2287|20515|69384|1594859|2287|20515|4|57|2|2|6211|254|1491|27|28.45|33.00|25.74|196.02|694.98|768.15|891.00|62.54|0.00|151.47|694.98|757.52|846.45|908.99|-73.17| +2450844|36481|2450926|21475|1167551|3674|14197|21475|1167551|3674|14197|4|70|1|5|8440|133|1492|88|52.16|127.27|7.63|10528.32|671.44|4590.08|11199.76|0.00|0.00|3583.36|671.44|671.44|4254.80|4254.80|-3918.64| +2450844|36481|2450926|21475|1167551|3674|14197|21475|1167551|3674|14197|4|79|10|1|5044|48|1492|6|83.75|161.63|40.40|727.38|242.40|502.50|969.78|7.27|0.00|96.96|242.40|249.67|339.36|346.63|-260.10| +2450844|36481|2450867|21475|1167551|3674|14197|21475|1167551|3674|14197|4|28|14|4|8059|21|1492|97|90.74|272.22|24.49|24029.81|2375.53|8801.78|26405.34|19.95|1876.66|8184.86|498.87|518.82|8683.73|8703.68|-8302.91| +2450844|36481|2450904|21475|1167551|3674|14197|21475|1167551|3674|14197|4|57|11|4|15607|16|1492|8|65.09|70.94|5.67|522.16|45.36|520.72|567.52|0.00|0.00|170.24|45.36|45.36|215.60|215.60|-475.36| +2450844|36481|2450870|21475|1167551|3674|14197|21475|1167551|3674|14197|4|98|3|2|710|70|1492|64|60.76|123.95|8.67|7377.92|554.88|3888.64|7932.80|27.74|0.00|3807.36|554.88|582.62|4362.24|4389.98|-3333.76| +2450844|36481|2450882|21475|1167551|3674|14197|21475|1167551|3674|14197|4|9|1|5|9133|163|1492|73|59.23|134.45|33.61|7361.32|2453.53|4323.79|9814.85|49.07|0.00|4220.13|2453.53|2502.60|6673.66|6722.73|-1870.26| +2450844|36481|2450904|21475|1167551|3674|14197|21475|1167551|3674|14197|4|66|7|1|13102|77|1492|84|59.21|158.68|150.74|666.96|12662.16|4973.64|13329.12|1139.59|0.00|5997.60|12662.16|13801.75|18659.76|19799.35|7688.52| +2450844|36481|2450857|21475|1167551|3674|14197|21475|1167551|3674|14197|4|67|20|5|15544|265|1492|96|20.70|49.47|3.95|4369.92|379.20|1987.20|4749.12|26.54|0.00|47.04|379.20|405.74|426.24|452.78|-1608.00| +2450844|36481|2450863|21475|1167551|3674|14197|21475|1167551|3674|14197|4|42|17|2|3166|279|1492|89|48.70|111.52|110.40|99.68|9825.60|4334.30|9925.28|309.50|6386.64|2183.17|3438.96|3748.46|5622.13|5931.63|-895.34| +2450844|61277|2450932|16686|461513|2787|42061|16686|461513|2787|42061|1|97|2|1|7352|110|1493|57|77.78|136.11|107.52|1629.63|6128.64|4433.46|7758.27|364.65|919.29|1395.93|5209.35|5574.00|6605.28|6969.93|775.89| +2450844|61277|2450870|16686|461513|2787|42061|16686|461513|2787|42061|1|92|3|2|2902|36|1493|26|56.82|113.07|110.80|59.02|2880.80|1477.32|2939.82|144.04|0.00|940.68|2880.80|3024.84|3821.48|3965.52|1403.48| +2450844|61277|2450898|16686|461513|2787|42061|16686|461513|2787|42061|1|30|6|3|17482|87|1493|34|73.88|152.19|91.31|2069.92|3104.54|2511.92|5174.46|248.36|0.00|1345.04|3104.54|3352.90|4449.58|4697.94|592.62| +2450844|61277|2450892|16686|461513|2787|42061|16686|461513|2787|42061|1|47|11|3|14642|267|1493|56|81.85|243.91|14.63|12839.68|819.28|4583.60|13658.96|24.57|0.00|5873.28|819.28|843.85|6692.56|6717.13|-3764.32| +2450844|61277|2450861|16686|461513|2787|42061|16686|461513|2787|42061|1|39|16|5|11077|49|1493|64|62.44|73.05|48.94|1543.04|3132.16|3996.16|4675.20|219.25|0.00|1542.40|3132.16|3351.41|4674.56|4893.81|-864.00| +2450844|44022|2450868|33097|1441501|305|21556|33097|1441501|305|21556|1|83|4|5|11656|164|1494|24|2.55|3.92|3.21|17.04|77.04|61.20|94.08|5.39|0.00|42.24|77.04|82.43|119.28|124.67|15.84| +2450844|44022|2450857|33097|1441501|305|21556|33097|1441501|305|21556|1|13|12|2|13264|262|1494|64|87.14|202.16|113.20|5693.44|7244.80|5576.96|12938.24|217.34|0.00|387.84|7244.80|7462.14|7632.64|7849.98|1667.84| +2450844|44022|2450885|33097|1441501|305|21556|33097|1441501|305|21556|1|52|16|2|13081|82|1494|56|50.51|124.25|38.51|4801.44|2156.56|2828.56|6958.00|43.13|0.00|278.32|2156.56|2199.69|2434.88|2478.01|-672.00| +2450844|44022|2450902|33097|1441501|305|21556|33097|1441501|305|21556|1|103|7|5|5725|81|1494|98|9.02|17.76|5.15|1235.78|504.70|883.96|1740.48|5.04|0.00|208.74|504.70|509.74|713.44|718.48|-379.26| +2450844|44022|2450931|33097|1441501|305|21556|33097|1441501|305|21556|1|82|13|3|6158|185|1494|86|9.79|12.53|9.89|227.04|850.54|841.94|1077.58|51.03|0.00|258.00|850.54|901.57|1108.54|1159.57|8.60| +2450844|44022|2450883|33097|1441501|305|21556|33097|1441501|305|21556|1|73|12|5|6322|156|1494|64|73.83|90.81|36.32|3487.36|2324.48|4725.12|5811.84|185.95|0.00|348.16|2324.48|2510.43|2672.64|2858.59|-2400.64| +2450844|44022|2450913|33097|1441501|305|21556|33097|1441501|305|21556|1|58|12|2|11965|293|1494|95|14.09|36.35|35.25|104.50|3348.75|1338.55|3453.25|133.95|0.00|966.15|3348.75|3482.70|4314.90|4448.85|2010.20| +2450844|60094|2450892|52424|1243522|6267|22965|56428|415561|2659|22157|1|36|9|1|9184|145|1495|42|96.04|126.77|96.34|1278.06|4046.28|4033.68|5324.34|80.92|0.00|1330.98|4046.28|4127.20|5377.26|5458.18|12.60| +2450844|60094|2450885|52424|1243522|6267|22965|56428|415561|2659|22157|1|15|2|2|1633|109|1495|36|66.98|73.00|28.47|1603.08|1024.92|2411.28|2628.00|92.24|0.00|289.08|1024.92|1117.16|1314.00|1406.24|-1386.36| +2450844|60094|2450868|52424|1243522|6267|22965|56428|415561|2659|22157|1|67|17|3|17995|55|1495|33|13.81|36.59|25.24|374.55|832.92|455.73|1207.47|16.65|0.00|434.61|832.92|849.57|1267.53|1284.18|377.19| +2450844|60094|2450882|52424|1243522|6267|22965|56428|415561|2659|22157|1|57|12|1|1844|237|1495|56|70.57|186.30|67.06|6677.44|3755.36|3951.92|10432.80|187.76|0.00|1773.52|3755.36|3943.12|5528.88|5716.64|-196.56| +2450844|60094|2450929|52424|1243522|6267|22965|56428|415561|2659|22157|1|101|14|3|8806|164|1495|100|83.23|240.53|122.67|11786.00|12267.00|8323.00|24053.00|1104.03|0.00|3848.00|12267.00|13371.03|16115.00|17219.03|3944.00| +2450844|52262|2450925|85997|1101667|2137|4207|85997|1101667|2137|4207|2|93|7|3|7510|235|1496|97|88.99|169.08|91.30|7544.66|8856.10|8632.03|16400.76|265.68|0.00|6067.35|8856.10|9121.78|14923.45|15189.13|224.07| +2450844|52262|2450889|85997|1101667|2137|4207|85997|1101667|2137|4207|2|75|14|4|13726|166|1496|20|96.47|96.47|9.64|1736.60|192.80|1929.40|1929.40|7.71|0.00|424.40|192.80|200.51|617.20|624.91|-1736.60| +2450844|52262|2450874|85997|1101667|2137|4207|85997|1101667|2137|4207|2|43|15|2|16246|131|1496|49|36.41|73.18|35.12|1864.94|1720.88|1784.09|3585.82|22.02|619.51|967.75|1101.37|1123.39|2069.12|2091.14|-682.72| +2450844|52262|2450871|85997|1101667|2137|4207|85997|1101667|2137|4207|2|65|17|2|10897|109|1496|57|75.76|81.06|50.25|1756.17|2864.25|4318.32|4620.42|229.14|0.00|1755.60|2864.25|3093.39|4619.85|4848.99|-1454.07| +2450844|52262|2450931|85997|1101667|2137|4207|85997|1101667|2137|4207|2|50|8|4|14030|51|1496|98|64.75|192.95|177.51|1513.12|17395.98|6345.50|18909.10|347.91|0.00|0.00|17395.98|17743.89|17395.98|17743.89|11050.48| +2450844|52262|2450908|85997|1101667|2137|4207|85997|1101667|2137|4207|2|44|17|1|9361|158|1496|87|51.18|66.02|48.19|1551.21|4192.53|4452.66|5743.74|150.93|419.25|229.68|3773.28|3924.21|4002.96|4153.89|-679.38| +2450844|52262|2450846|85997|1101667|2137|4207|85997|1101667|2137|4207|2|50|13|4|6697|85|1496|34|5.89|11.54|11.54|0.00|392.36|200.26|392.36|31.38|0.00|129.20|392.36|423.74|521.56|552.94|192.10| +2450844|65260|2450890|80922|583951|6180|29609|56573|378114|1143|28172|1|81|15|4|11564|54|1497|73|6.75|16.20|10.36|426.32|756.28|492.75|1182.60|68.06|0.00|153.30|756.28|824.34|909.58|977.64|263.53| +2450844|65260|2450919|80922|583951|6180|29609|56573|378114|1143|28172|1|69|8|5|16507|221|1497|72|90.32|209.54|10.47|14333.04|753.84|6503.04|15086.88|0.00|361.84|5732.64|392.00|392.00|6124.64|6124.64|-6111.04| +2450844|||||6180||||1143|28172||67|18|2|8000||1497|25|||27.30|||1006.00||47.77|||||1129.75|1177.52|-323.50| +2450844|65260|2450909|80922|583951|6180|29609|56573|378114|1143|28172|1|16|15|1|1312|33|1497|2|83.05|102.15|101.12|2.06|202.24|166.10|204.30|3.03|101.12|96.02|101.12|104.15|197.14|200.17|-64.98| +2450844|65260|2450885|80922|583951|6180|29609|56573|378114|1143|28172|1|69|14|2|3500|98|1497|10|43.26|68.78|32.32|364.60|323.20|432.60|687.80|9.69|0.00|41.20|323.20|332.89|364.40|374.09|-109.40| +2450844|62047|2450895|93056|895729|2|2275|93056|895729|2|2275|1|10|8|3|7406|174|1498|43|79.23|208.37|87.51|5196.98|3762.93|3406.89|8959.91|37.62|0.00|3673.49|3762.93|3800.55|7436.42|7474.04|356.04| +2450844|62047|2450852|93056|895729|2|2275|93056|895729|2|2275|1|60|15|5|15193|214|1498|17|92.97|265.89|109.01|2666.96|1853.17|1580.49|4520.13|37.06|0.00|135.49|1853.17|1890.23|1988.66|2025.72|272.68| +2450844|62047|2450928|93056|895729|2|2275|93056|895729|2|2275|1|38|19|1|12979|244|1498|94|23.95|52.21|50.12|196.46|4711.28|2251.30|4907.74|376.90|0.00|1962.72|4711.28|5088.18|6674.00|7050.90|2459.98| +2450844|62047|2450907|93056|895729|2|2275|93056|895729|2|2275|1|38|12|2|15284|151|1498|53|57.81|95.96|53.73|2238.19|2847.69|3063.93|5085.88|199.33|0.00|1576.22|2847.69|3047.02|4423.91|4623.24|-216.24| +2450844|8590|2450867|6922|92849|4729|49856|6922|92849|4729|49856|4|8|12|2|13810|25|1499|73|93.18|137.90|125.48|906.66|9160.04|6802.14|10066.70|261.06|458.00|3321.50|8702.04|8963.10|12023.54|12284.60|1899.90| +2450844|8590|2450893|6922|92849|4729|49856|6922|92849|4729|49856|4|48|12|2|10544|258|1499|62|14.45|31.35|9.09|1380.12|563.58|895.90|1943.70|5.63|0.00|641.08|563.58|569.21|1204.66|1210.29|-332.32| +2450844|8590|2450897|6922|92849|4729|49856|6922|92849|4729|49856|4|20|11|4|4396|230|1499|16|19.05|56.00|18.48|600.32|295.68|304.80|896.00|17.74|0.00|35.84|295.68|313.42|331.52|349.26|-9.12| +2450844|8590||6922||4729||6922|92849|||4|||5|12460|34|1499|24||145.74|59.75|2063.76|1434.00|1547.76||63.23|||702.66||807.54|870.77|-845.10| +2450844|8590|2450910|6922|92849|4729|49856|6922|92849|4729|49856|4|57|12|3|14240|54|1499|98|12.76|25.90|21.49|432.18|2106.02|1250.48|2538.20|63.18|0.00|1039.78|2106.02|2169.20|3145.80|3208.98|855.54| +2450844|8590|2450908|6922|92849|4729|49856|6922|92849|4729|49856|4|89|1|1|15032|281|1499|97|90.79|184.30|44.23|13586.79|4290.31|8806.63|17877.10|386.12|0.00|8759.10|4290.31|4676.43|13049.41|13435.53|-4516.32| +2450844|8590|2450908|6922|92849|4729|49856|6922|92849|4729|49856|4|47|20|5|3260|235|1499|61|34.11|98.57|85.75|782.02|5230.75|2080.71|6012.77|209.23|0.00|1863.55|5230.75|5439.98|7094.30|7303.53|3150.04| +2450844|8590|2450883|6922|92849|4729|49856|6922|92849|4729|49856|4|18|5|1|638|141|1499|7|1.74|4.35|0.39|27.72|2.73|12.18|30.45|0.19|0.00|1.47|2.73|2.92|4.20|4.39|-9.45| +2450844|8590|2450915|6922|92849|4729|49856|6922|92849|4729|49856|4|101|14|1|2920|102|1499|11|28.18|83.69|16.73|736.56|184.03|309.98|920.59|0.00|0.00|128.81|184.03|184.03|312.84|312.84|-125.95| +2450844|39010|2450852|85886|1805495|304|33913|85886|1805495|304|33913|1|95|17|5|5677|196|1500|86|89.56|173.74|34.74|11954.00|2987.64|7702.16|14941.64|29.87|0.00|2389.94|2987.64|3017.51|5377.58|5407.45|-4714.52| +2450844|39010|2450865|85886|1805495|304|33913|85886|1805495|304|33913|1|23|12|1|10672|274|1500|59|53.76|97.84|53.81|2597.77|3174.79|3171.84|5772.56|190.48|0.00|634.84|3174.79|3365.27|3809.63|4000.11|2.95| +2450844|39010|2450921|85886|1805495|304|33913|85886|1805495|304|33913|1|75|8|1|2468|46|1500|61|55.38|112.42|61.83|3085.99|3771.63|3378.18|6857.62|301.73|0.00|342.82|3771.63|4073.36|4114.45|4416.18|393.45| +2450844|39010|2450892|85886|1805495|304|33913|85886|1805495|304|33913|1|72|6|2|16004|224|1500|84|62.96|163.69|11.45|12788.16|961.80|5288.64|13749.96|19.23|0.00|4399.92|961.80|981.03|5361.72|5380.95|-4326.84| +2450844|39010|2450883|85886|1805495|304|33913|85886|1805495|304|33913|1|43|9|4|11221|287|1500|27|60.03|73.23|24.16|1324.89|652.32|1620.81|1977.21|13.04|0.00|949.05|652.32|665.36|1601.37|1614.41|-968.49| +2450844|39010|2450856|85886|1805495|304|33913|85886|1805495|304|33913|1|20|18|3|15976|243|1500|78|86.71|225.44|211.91|1055.34|16528.98|6763.38|17584.32|1322.31|0.00|3340.74|16528.98|17851.29|19869.72|21192.03|9765.60| +2450844|39010|2450880|85886|1805495|304|33913|85886|1805495|304|33913|1|88|9|2|532|93|1500|90|13.39|22.09|18.11|358.20|1629.90|1205.10|1988.10|81.49|0.00|516.60|1629.90|1711.39|2146.50|2227.99|424.80| +2450844|39010|2450874|85886|1805495|304|33913|85886|1805495|304|33913|1|12|5|3|17950|244|1500|70|58.93|98.41|28.53|4891.60|1997.10|4125.10|6888.70|159.76|0.00|1377.60|1997.10|2156.86|3374.70|3534.46|-2128.00| +2450844|39010|2450862|85886|1805495|304|33913|85886|1805495|304|33913|1|45|18|1|8132|205|1500|67|55.87|63.69|34.39|1963.10|2304.13|3743.29|4267.23|23.50|1520.72|2048.19|783.41|806.91|2831.60|2855.10|-2959.88| +2450844|39010|2450907|85886|1805495|304|33913|85886|1805495|304|33913|1|90|14|5|7592|254|1500|74|90.27|92.97|68.79|1789.32|5090.46|6679.98|6879.78|356.33|0.00|2270.32|5090.46|5446.79|7360.78|7717.11|-1589.52| +2450844|39010|2450919|85886|1805495|304|33913|85886|1805495|304|33913|1|90|17|2|5941|244|1500|53|25.11|69.05|1.38|3586.51|73.14|1330.83|3659.65|5.85|0.00|731.93|73.14|78.99|805.07|810.92|-1257.69| +2450844|39010|2450928|85886|1805495|304|33913|85886|1805495|304|33913|1|104|1|2|7975|272|1500|4|17.56|42.49|15.29|108.80|61.16|70.24|169.96|4.89|0.00|0.00|61.16|66.05|61.16|66.05|-9.08| +2450844|57405|2450854|71220|1324221|4069|8317|71220|1324221|4069|8317|4|19|6|3|1888|124|1501|60|45.69|100.06|30.01|4203.00|1800.60|2741.40|6003.60|36.01|0.00|2581.20|1800.60|1836.61|4381.80|4417.81|-940.80| +2450844|57405|2450885|71220|1324221|4069|8317|71220|1324221|4069|8317|4|64|19|1|12010|280|1501|23|46.91|119.15|35.74|1918.43|822.02|1078.93|2740.45|8.22|0.00|1260.40|822.02|830.24|2082.42|2090.64|-256.91| +2450844|57405|2450873|71220|1324221|4069|8317|71220|1324221|4069|8317|4|82|1|2|4489|18|1501|45|71.66|199.21|29.88|7619.85|1344.60|3224.70|8964.45|26.89|0.00|1792.80|1344.60|1371.49|3137.40|3164.29|-1880.10| +2450844|57405|2450872|71220|1324221|4069|8317|71220|1324221|4069|8317|4|65|2|1|2336|213|1501|57|28.02|29.14|19.81|531.81|1129.17|1597.14|1660.98|45.16|0.00|580.83|1129.17|1174.33|1710.00|1755.16|-467.97| +2450844|57405|2450859|71220|1324221|4069|8317|71220|1324221|4069|8317|4|67|14|1|13436|67|1501|34|8.81|22.72|9.08|463.76|308.72|299.54|772.48|27.78|0.00|169.66|308.72|336.50|478.38|506.16|9.18| +2450844|57405|2450875|71220|1324221|4069|8317|71220|1324221|4069|8317|4|105|6|5|6283|270|1501|79|90.42|179.93|37.78|11229.85|2984.62|7143.18|14214.47|179.07|0.00|1990.01|2984.62|3163.69|4974.63|5153.70|-4158.56| +2450844|80960|2450862|96919|507526|180|17234|96919|507526|180|17234|4|103|11|3|1442|97|1502|58|38.21|65.33|60.75|265.64|3523.50|2216.18|3789.14|176.17|0.00|492.42|3523.50|3699.67|4015.92|4192.09|1307.32| +2450844|80960|2450895|96919|507526|180|17234|96919|507526|180|17234|4|57|6|2|12614|89|1502|82|68.74|191.09|24.84|13632.50|2036.88|5636.68|15669.38|183.31|0.00|5170.10|2036.88|2220.19|7206.98|7390.29|-3599.80| +2450844|80960|2450864|96919|507526|180|17234|96919|507526|180|17234|4|108|19|2|13922|45|1502|89|83.65|91.17|12.76|6978.49|1135.64|7444.85|8114.13|0.00|942.58|2028.31|193.06|193.06|2221.37|2221.37|-7251.79| +2450844|80960|2450871|96919|507526|180|17234|96919|507526|180|17234|4|108|15|3|2734|131|1502|33|3.19|5.39|0.00|177.87|0.00|105.27|177.87|0.00|0.00|40.59|0.00|0.00|40.59|40.59|-105.27| +2450844|80960|2450929|96919|507526|180|17234|96919|507526|180|17234|4|105|10|4|6218|142|1502|71|72.80|199.47|53.85|10339.02|3823.35|5168.80|14162.37|192.69|1070.53|2124.32|2752.82|2945.51|4877.14|5069.83|-2415.98| +2450844|80960|2450865|96919|507526|180|17234|96919|507526|180|17234|4|80|8|3|10324|253|1502|95|46.63|137.55|5.50|12544.75|522.50|4429.85|13067.25|20.90|0.00|522.50|522.50|543.40|1045.00|1065.90|-3907.35| +2450844|80960|2450876|96919|507526|180|17234|96919|507526|180|17234|4|92|17|5|4268|180|1502|70|68.59|119.34|36.99|5764.50|2589.30|4801.30|8353.80|50.49|1579.47|83.30|1009.83|1060.32|1093.13|1143.62|-3791.47| +2450844|80960|2450891|96919|507526|180|17234|96919|507526|180|17234|4|45|16|2|10918|127|1502|28|14.18|32.04|14.73|484.68|412.44|397.04|897.12|20.62|0.00|430.36|412.44|433.06|842.80|863.42|15.40| +2450844|80960|2450856|96919|507526|180|17234|96919|507526|180|17234|4|85|17|4|5380|256|1502|68|78.54|127.23|122.14|346.12|8305.52|5340.72|8651.64|302.32|747.49|3633.24|7558.03|7860.35|11191.27|11493.59|2217.31| +2450844|80960|2450909|96919|507526|180|17234|96919|507526|180|17234|4|7|12|1|12368|197|1502|53|76.40|124.53|2.49|6468.12|131.97|4049.20|6600.09|8.59|9.23|3167.81|122.74|131.33|3290.55|3299.14|-3926.46| +2450844|80960|2450862|96919|507526|180|17234|96919|507526|180|17234|4|108|8|4|8588|84|1502|98|73.07|156.36|148.54|766.36|14556.92|7160.86|15323.28|1164.55|0.00|765.38|14556.92|15721.47|15322.30|16486.85|7396.06| +2450844|80960|2450895|96919|507526|180|17234|96919|507526|180|17234|4|44|6|2|22|48|1502|22|24.75|68.31|61.47|150.48|1352.34|544.50|1502.82|67.61|0.00|450.78|1352.34|1419.95|1803.12|1870.73|807.84| +2450844|80960|2450871|96919|507526|180|17234|96919|507526|180|17234|4|86|14|4|5245|55|1502|26|76.84|107.57|17.21|2349.36|447.46|1997.84|2796.82|3.22|125.28|531.18|322.18|325.40|853.36|856.58|-1675.66| +2450844|80960|2450887|96919|507526|180|17234|96919|507526|180|17234|4|81|6|2|2750|43|1502|71|65.58|128.53|123.38|365.65|8759.98|4656.18|9125.63|87.59|0.00|3740.99|8759.98|8847.57|12500.97|12588.56|4103.80| +2450844|61176|2450873|5395|170051|5907|24563|5395|170051|5907|24563|4|4|10|5|13898|43|1503|77|26.01|53.84|13.46|3109.26|1036.42|2002.77|4145.68|51.82|0.00|1616.23|1036.42|1088.24|2652.65|2704.47|-966.35| +2450844|61176|2450848|5395|170051|5907|24563|5395|170051|5907|24563|4|55|10|3|10015|254|1503|71|87.09|210.75|187.56|1646.49|13316.76|6183.39|14963.25|266.33|0.00|897.44|13316.76|13583.09|14214.20|14480.53|7133.37| +2450844|61176|2450853|5395|170051|5907|24563|5395|170051|5907|24563|4|40|20|3|10162|113|1503|77|20.18|37.53|5.25|2485.56|404.25|1553.86|2889.81|32.34|0.00|1068.76|404.25|436.59|1473.01|1505.35|-1149.61| +2450844|61176|2450916|5395|170051|5907|24563|5395|170051|5907|24563|4|101|9|3|16162|265|1503|13|79.59|180.66|21.67|2066.87|281.71|1034.67|2348.58|19.71|0.00|657.54|281.71|301.42|939.25|958.96|-752.96| +2450844|61176|2450872|5395|170051|5907|24563|5395|170051|5907|24563|4|35|2|4|9686|236|1503|2|45.37|100.72|19.13|163.18|38.26|90.74|201.44|0.29|8.79|92.66|29.47|29.76|122.13|122.42|-61.27| +2450844|61176|2450849|5395|170051|5907|24563|5395|170051|5907|24563|4|88|3|1|15634|182|1503|92|58.25|133.39|32.01|9326.96|2944.92|5359.00|12271.88|117.79|0.00|5153.84|2944.92|3062.71|8098.76|8216.55|-2414.08| +2450844|61176|2450864|5395|170051|5907|24563|5395|170051|5907|24563|4|94|14|4|868|255|1503|65|98.76|150.11|100.57|3220.10|6537.05|6419.40|9757.15|261.48|0.00|3414.45|6537.05|6798.53|9951.50|10212.98|117.65| +2450844|61176|2450886|5395|170051|5907|24563|5395|170051|5907|24563|4|57|15|5|4634|235|1503|78|57.49|146.59|120.20|2058.42|9375.60|4484.22|11434.02|281.26|0.00|4229.94|9375.60|9656.86|13605.54|13886.80|4891.38| +2450844|61176|2450874|5395|170051|5907|24563|5395|170051|5907|24563|4|89|19|1|15043|228|1503|21|60.50|175.45|100.00|1584.45|2100.00|1270.50|3684.45|0.00|294.00|220.92|1806.00|1806.00|2026.92|2026.92|535.50| +2450844|57772|2450911|48284|303488|4292|21896|48284|303488|4292|21896|4|59|17|5|1886|43|1504|42|21.86|45.25|41.63|152.04|1748.46|918.12|1900.50|139.87|0.00|627.06|1748.46|1888.33|2375.52|2515.39|830.34| +2450844|57772|2450903|48284|303488|4292|21896|48284|303488|4292|21896|4|88|10|5|17108|72|1504|51|83.47|165.27|118.99|2360.28|6068.49|4256.97|8428.77|0.00|0.00|2781.03|6068.49|6068.49|8849.52|8849.52|1811.52| +2450844|57772|2450847|48284|303488|4292|21896|48284|303488|4292|21896|4|1|11|4|15715|230|1504|78|3.60|4.28|2.05|173.94|159.90|280.80|333.84|10.36|44.77|86.58|115.13|125.49|201.71|212.07|-165.67| +2450844|57772|2450883|48284|303488|4292|21896|48284|303488|4292|21896|4|2|11|2|7790|206|1504|29|52.47|61.38|34.98|765.60|1014.42|1521.63|1780.02|91.29|0.00|355.83|1014.42|1105.71|1370.25|1461.54|-507.21| +2450844|57772|2450855|48284|303488|4292|21896|48284|303488|4292|21896|4|99|3|5|16508|144|1504|17|23.16|50.95|6.11|762.28|103.87|393.72|866.15|9.34|0.00|155.89|103.87|113.21|259.76|269.10|-289.85| +2450844|57772|2450919|48284|303488|4292|21896|48284|303488|4292|21896|4|104|16|3|14437|241|1504|6|7.89|11.59|11.24|2.10|67.44|47.34|69.54|0.00|57.99|14.58|9.45|9.45|24.03|24.03|-37.89| +2450844|57772|2450893|48284|303488|4292|21896|48284|303488|4292|21896|4|100|6|2|1558|246|1504|97|2.31|3.69|1.80|183.33|174.60|224.07|357.93|0.00|0.00|34.92|174.60|174.60|209.52|209.52|-49.47| +2450844|57772|2450902|48284|303488|4292|21896|48284|303488|4292|21896|4|1|1|1|17449|1|1504|12|95.32|190.64|120.10|846.48|1441.20|1143.84|2287.68|14.41|0.00|228.72|1441.20|1455.61|1669.92|1684.33|297.36| +2450844|57772|2450874|48284|303488|4292|21896|48284|303488|4292|21896|4|17|4|1|11722|256|1504|27|73.67|206.27|88.69|3174.66|2394.63|1989.09|5569.29|191.57|0.00|501.12|2394.63|2586.20|2895.75|3087.32|405.54| +2450844|57772|2450922|48284|303488|4292|21896|48284|303488|4292|21896|4|44|16|4|17533|248|1504|37|78.10|143.70|67.53|2818.29|2498.61|2889.70|5316.90|99.94|0.00|2498.61|2498.61|2598.55|4997.22|5097.16|-391.09| +2450844|57772|2450921|48284|303488|4292|21896|48284|303488|4292|21896|4|97|8|5|11755|242|1504|83|7.88|8.43|2.36|503.81|195.88|654.04|699.69|5.87|0.00|76.36|195.88|201.75|272.24|278.11|-458.16| +2450844|57772|2450911|48284|303488|4292|21896|48284|303488|4292|21896|4|19|10|1|3668|139|1504|57|17.39|40.86|31.05|559.17|1769.85|991.23|2329.02|88.49|0.00|163.02|1769.85|1858.34|1932.87|2021.36|778.62| +2450844|55922|2450851|68516|1402845|821|9150|68516|1402845|821|9150|1|8|2|3|13508|79|1505|48|97.97|191.04|19.10|8253.12|916.80|4702.56|9169.92|14.85|669.26|4401.12|247.54|262.39|4648.66|4663.51|-4455.02| +2450844|55922|2450870|68516|1402845|821|9150|68516|1402845|821|9150|1|16|19|4|3976|132|1505|50|42.41|44.53|32.06|623.50|1603.00|2120.50|2226.50|0.00|0.00|289.00|1603.00|1603.00|1892.00|1892.00|-517.50| +2450844|55922|2450897|68516|1402845|821|9150|68516|1402845|821|9150|1|55|12|2|13144|26|1505|39|88.77|90.54|75.14|600.60|2930.46|3462.03|3531.06|29.30|0.00|847.08|2930.46|2959.76|3777.54|3806.84|-531.57| +2450844|55922|2450912|68516|1402845|821|9150|68516|1402845|821|9150|1|22|16|3|7508|174|1505|32|48.60|145.31|138.04|232.64|4417.28|1555.20|4649.92|176.69|0.00|1162.24|4417.28|4593.97|5579.52|5756.21|2862.08| +2450844|55922|2450861|68516|1402845|821|9150|68516|1402845|821|9150|1|8|19|3|5692|216|1505|28|77.86|193.09|185.36|216.44|5190.08|2180.08|5406.52|363.30|0.00|1838.20|5190.08|5553.38|7028.28|7391.58|3010.00| +2450844|55922|2450911|68516|1402845|821|9150|68516|1402845|821|9150|1|5|20|3|10198|166|1505|62|39.78|40.97|2.45|2388.24|151.90|2466.36|2540.14|7.59|0.00|990.14|151.90|159.49|1142.04|1149.63|-2314.46| +2450844|55922|2450873|68516|1402845|821|9150|68516|1402845|821|9150|1|74|7|1|1564|112|1505|44|27.89|68.33|30.06|1683.88|1322.64|1227.16|3006.52|6.61|1190.37|841.72|132.27|138.88|973.99|980.60|-1094.89| +2450844|||68516|1402845|821|9150||1402845||9150|1|42|11|1|2989|9|1505|15|69.24||20.16|||1038.60||||||317.52|1372.50|1387.62|-736.20| +2450844|55922|2450858|68516|1402845|821|9150|68516|1402845|821|9150|1|15|19|4|12718|119|1505|8|94.01|257.58|25.75|1854.64|206.00|752.08|2060.64|10.30|0.00|473.92|206.00|216.30|679.92|690.22|-546.08| +2450844|55922|2450879|68516|1402845|821|9150|68516|1402845|821|9150|1|42|5|1|10873|155|1505|52|38.74|57.33|28.09|1520.48|1460.68|2014.48|2981.16|12.41|1212.36|745.16|248.32|260.73|993.48|1005.89|-1766.16| +2450844|55922|2450899|68516|1402845|821|9150|68516|1402845|821|9150|1|30|6|5|6188|183|1505|91|82.35|98.82|50.39|4407.13|4585.49|7493.85|8992.62|275.12|0.00|2157.61|4585.49|4860.61|6743.10|7018.22|-2908.36| +2450844|55922|2450867|68516|1402845|821|9150|68516|1402845|821|9150|1|63|2|4|14560|28|1505|98|57.13|129.68|7.78|11946.20|762.44|5598.74|12708.64|22.87|0.00|5845.70|762.44|785.31|6608.14|6631.01|-4836.30| +2450844|63668|2450932|85052|100347|5595|39643|18984|946939|6234|17882|1|23|3|1|8596|277|1506|3|53.61|150.10|100.56|148.62|301.68|160.83|450.30|6.03|0.00|49.53|301.68|307.71|351.21|357.24|140.85| +2450844|63668|2450931|85052|100347|5595|39643|18984|946939|6234|17882|1|105|7|1|8992|114|1506|68|25.53|39.57|4.35|2394.96|295.80|1736.04|2690.76|0.00|165.64|1317.84|130.16|130.16|1448.00|1448.00|-1605.88| +2450844|63668|2450906|85052|100347|5595|39643|18984|946939|6234|17882|1|102|20|2|4774|291|1506|93|73.71|161.42|22.59|12911.19|2100.87|6855.03|15012.06|63.02|0.00|3302.43|2100.87|2163.89|5403.30|5466.32|-4754.16| +2450844|63668|2450924|85052|100347|5595|39643|18984|946939|6234|17882|1|4|17|1|12499|21|1506|68|39.69|68.66|19.22|3361.92|1306.96|2698.92|4668.88|91.48|0.00|326.40|1306.96|1398.44|1633.36|1724.84|-1391.96| +2450844|63668|2450881|85052|100347|5595|39643|18984|946939|6234|17882|1|107|13|4|4861|17|1506|4|32.68|69.28|29.09|160.76|116.36|130.72|277.12|0.62|105.88|77.56|10.48|11.10|88.04|88.66|-120.24| +2450844|63668|2450895|85052|100347|5595|39643|18984|946939|6234|17882|1|58|4|1|16711|230|1506|3|32.43|86.26|5.17|243.27|15.51|97.29|258.78|0.46|0.00|28.44|15.51|15.97|43.95|44.41|-81.78| +2450844|63668|2450909|85052|100347|5595|39643|18984|946939|6234|17882|1|58|5|5|692|176|1506|35|2.07|2.50|2.47|1.05|86.45|72.45|87.50|4.32|0.00|35.70|86.45|90.77|122.15|126.47|14.00| +2450844|43196|2450890|58716|1539409|902|37195|58716|1539409|902|37195|4|1|5|2|14572|231|1507|85|53.32|74.64|26.87|4060.45|2283.95|4532.20|6344.40|137.03|0.00|1586.10|2283.95|2420.98|3870.05|4007.08|-2248.25| +2450844|43196|2450909|58716|1539409|902|37195|58716|1539409|902|37195|4|12|4|4|6460|249|1507|65|41.50|50.63|39.49|724.10|2566.85|2697.50|3290.95|25.66|0.00|0.00|2566.85|2592.51|2566.85|2592.51|-130.65| +2450844|43196|2450881|58716|1539409|902|37195|58716|1539409|902|37195|4|63|1|3|3896|110|1507|21|96.00|170.88|138.41|681.87|2906.61|2016.00|3588.48|261.59|0.00|681.66|2906.61|3168.20|3588.27|3849.86|890.61| +2450844|43196|2450920|58716|1539409|902|37195|58716|1539409|902|37195|4|4|17|1|12679|117|1507|52|99.69|163.49|0.00|8501.48|0.00|5183.88|8501.48|0.00|0.00|3060.20|0.00|0.00|3060.20|3060.20|-5183.88| +2450844|43196|2450921|58716|1539409|902|37195|58716|1539409|902|37195|4|93|13|1|17218|273|1507|20|16.80|44.35|30.60|275.00|612.00|336.00|887.00|9.36|507.96|44.20|104.04|113.40|148.24|157.60|-231.96| +2450844||2450916|||902|37195||1539409|902|37195||28||3|10576|33|1507||38.35|61.74||2160.90|2160.90||||324.13||1836.77|2002.07|2571.07|2736.37|-847.73| +2450844|69934|2450861|26354|1311486|3371|6456|26354|1311486|3371|6456|1|76|7|2|1456|175|1508|47|42.47|61.15|26.90|1609.75|1264.30|1996.09|2874.05|0.00|0.00|1321.64|1264.30|1264.30|2585.94|2585.94|-731.79| +2450844|69934|2450924|26354|1311486|3371|6456|26354|1311486|3371|6456|1|21|15|5|17620|8|1508|88|49.60|85.80|54.91|2718.32|4832.08|4364.80|7550.40|0.00|0.00|226.16|4832.08|4832.08|5058.24|5058.24|467.28| +2450844|69934|2450882|26354|1311486|3371|6456|26354|1311486|3371|6456|1|7|9|3|5794|234|1508|100|34.55|82.22|0.00|8222.00|0.00|3455.00|8222.00|0.00|0.00|3535.00|0.00|0.00|3535.00|3535.00|-3455.00| +2450844|69934|2450915|26354|1311486|3371|6456|26354|1311486|3371|6456|1|100|18|1|16238|137|1508|31|86.22|167.26|93.66|2281.60|2903.46|2672.82|5185.06|174.20|0.00|1451.73|2903.46|3077.66|4355.19|4529.39|230.64| +2450844|69934|2450874|26354|1311486|3371|6456|26354|1311486|3371|6456|1|93|4|5|11942|175|1508|3|64.33|93.27|83.94|27.99|251.82|192.99|279.81|12.59|0.00|19.56|251.82|264.41|271.38|283.97|58.83| +2450844|69934|2450855|26354|1311486|3371|6456|26354|1311486|3371|6456|1|58|6|5|14588|107|1508|13|46.91|106.01|53.00|689.13|689.00|609.83|1378.13|48.23|0.00|151.58|689.00|737.23|840.58|888.81|79.17| +2450844|70544|2450915|97599|1441194|3989|31937|97599|1441194|3989|31937|1|86|12|4|2377|65|1509|35|2.16|2.63|0.47|75.60|16.45|75.60|92.05|0.65|0.00|24.85|16.45|17.10|41.30|41.95|-59.15| +2450844|70544|2450915|97599|1441194|3989|31937|97599|1441194|3989|31937|1|73|8|1|3823|220|1509|17|96.68|147.92|34.02|1936.30|578.34|1643.56|2514.64|17.35|0.00|125.63|578.34|595.69|703.97|721.32|-1065.22| +2450844|70544|2450920|97599|1441194|3989|31937|97599|1441194|3989|31937|1|95|9|4|11692|129|1509|4|98.45|258.92|134.63|497.16|538.52|393.80|1035.68|26.92|0.00|486.76|538.52|565.44|1025.28|1052.20|144.72| +2450844|70544|2450905|97599|1441194|3989|31937|97599|1441194|3989|31937|1|42|5|1|3493|171|1509|1|32.37|58.26|30.87|27.39|30.87|32.37|58.26|0.13|26.23|15.73|4.64|4.77|20.37|20.50|-27.73| +2450844|70544|2450876|97599|1441194|3989|31937|97599|1441194|3989|31937|1|104|11|4|2846|134|1509|2|89.38|154.62|23.19|262.86|46.38|178.76|309.24|2.78|0.00|142.24|46.38|49.16|188.62|191.40|-132.38| +2450844|70544|2450867|97599|1441194|3989|31937|97599|1441194|3989|31937|1|50|13|3|1972|40|1509|39|91.51|126.28|101.02|985.14|3939.78|3568.89|4924.92|236.38|0.00|886.47|3939.78|4176.16|4826.25|5062.63|370.89| +2450844|70544|2450857|97599|1441194|3989|31937|97599|1441194|3989|31937|1|108|17|3|1198|28|1509|15|25.82|27.62|25.41|33.15|381.15|387.30|414.30|7.62|0.00|82.80|381.15|388.77|463.95|471.57|-6.15| +2450844|70544|2450861|97599|1441194|3989|31937|97599|1441194|3989|31937|1|60|1|1|4825|22|1509|14|34.26|79.82|34.32|637.00|480.48|479.64|1117.48|1.05|427.62|491.68|52.86|53.91|544.54|545.59|-426.78| +2450844|28606|2450866|75840|1743395|6597|32152|75840|1743395|6597|32152|2|3|8|4|2473|201|1510|50|48.04|125.38|6.26|5956.00|313.00|2402.00|6269.00|6.26|0.00|1880.50|313.00|319.26|2193.50|2199.76|-2089.00| +2450844|28606|2450884|75840|1743395|6597|32152|75840|1743395|6597|32152|2|30|1|4|4486|200|1510|76|48.04|97.04|65.01|2434.28|4940.76|3651.04|7375.04|345.85|0.00|1105.80|4940.76|5286.61|6046.56|6392.41|1289.72| +2450844|28606|2450909|75840|1743395|6597|32152|75840|1743395|6597|32152|2|39|19|3|10432|271|1510|1|51.86|91.27|48.37|42.90|48.37|51.86|91.27|2.52|6.28|31.03|42.09|44.61|73.12|75.64|-9.77| +2450844|28606|2450872|75840|1743395|6597|32152|75840|1743395|6597|32152|2|28|15|3|7537|272|1510|99|98.61|253.42|139.38|11289.96|13798.62|9762.39|25088.58|0.00|0.00|4264.92|13798.62|13798.62|18063.54|18063.54|4036.23| +2450844|28606|2450887|75840|1743395|6597|32152|75840|1743395|6597|32152|2|106|5|1|3019|165|1510|91|48.76|78.01|67.08|994.63|6104.28|4437.16|7098.91|61.04|0.00|3052.14|6104.28|6165.32|9156.42|9217.46|1667.12| +2450844|28606|2450881|75840|1743395|6597|32152|75840|1743395|6597|32152|2|107|20|3|7616|235|1510|1|73.42|211.44|97.26|114.18|97.26|73.42|211.44|3.89|0.00|33.83|97.26|101.15|131.09|134.98|23.84| +2450844|28606|2450925|75840|1743395|6597|32152|75840|1743395|6597|32152|2|15|19|2|11554|235|1510|68|90.32|181.54|30.86|10246.24|2098.48|6141.76|12344.72|62.95|0.00|5554.92|2098.48|2161.43|7653.40|7716.35|-4043.28| +2450844|28606|2450866|75840|1743395|6597|32152|75840|1743395|6597|32152|2|68|9|5|12290|299|1510|70|24.71|53.86|12.92|2865.80|904.40|1729.70|3770.20|45.22|0.00|74.90|904.40|949.62|979.30|1024.52|-825.30| +2450844|28606|2450857|75840|1743395|6597|32152|75840|1743395|6597|32152|2|107|13|1|12512|300|1510|41|83.85|208.78|127.35|3338.63|5221.35|3437.85|8559.98|156.64|0.00|3680.57|5221.35|5377.99|8901.92|9058.56|1783.50| +2450844|28606|2450877|75840|1743395|6597|32152|75840|1743395|6597|32152|2|72|17|5|2492|184|1510|43|72.85|131.85|27.68|4479.31|1190.24|3132.55|5669.55|59.51|0.00|1077.15|1190.24|1249.75|2267.39|2326.90|-1942.31| +2450844|28606|2450887|75840|1743395|6597|32152|75840|1743395|6597|32152|2|72|3|2|6554|7|1510|76|25.86|33.61|19.82|1048.04|1506.32|1965.36|2554.36|90.37|0.00|25.08|1506.32|1596.69|1531.40|1621.77|-459.04| +2450844|28606|2450847|75840|1743395|6597|32152|75840|1743395|6597|32152|2|70|17|4|12304|206|1510|37|87.91|120.43|40.94|2941.13|1514.78|3252.67|4455.91|121.18|0.00|1915.86|1514.78|1635.96|3430.64|3551.82|-1737.89| +2450844|28606|2450930|75840|1743395|6597|32152|75840|1743395|6597|32152|2|56|1|4|8179|13|1510|45|38.19|53.08|7.43|2054.25|334.35|1718.55|2388.60|7.22|254.10|119.25|80.25|87.47|199.50|206.72|-1638.30| +2450844|66037|2450855|92598|94604|5057|37463|92598|94604|5057|37463|4|8|3|4|7826|151|1511|27|74.43|115.36|109.59|155.79|2958.93|2009.61|3114.72|266.30|0.00|93.42|2958.93|3225.23|3052.35|3318.65|949.32| +2450844|66037|2450859|92598|94604|5057|37463|92598|94604|5057|37463|4|47|2|1|3253|244|1511|42|10.02|14.92|11.19|156.66|469.98|420.84|626.64|28.19|0.00|244.02|469.98|498.17|714.00|742.19|49.14| +2450844|66037|2450898|92598|94604|5057|37463|92598|94604|5057|37463|4|43|11|4|974|280|1511|9|10.49|25.28|12.13|118.35|109.17|94.41|227.52|0.77|31.65|11.34|77.52|78.29|88.86|89.63|-16.89| +2450844|66037|2450850|92598|94604|5057|37463|92598|94604|5057|37463|4|105|10|1|5992|60|1511|85|3.88|5.78|2.13|310.25|181.05|329.80|491.30|12.67|0.00|191.25|181.05|193.72|372.30|384.97|-148.75| +2450844|66037|2450886|92598|94604|5057|37463|92598|94604|5057|37463|4|92|11|3|7268|169|1511|99|29.22|30.68|30.37|30.69|3006.63|2892.78|3037.32|60.13|0.00|515.79|3006.63|3066.76|3522.42|3582.55|113.85| +2450844|66037|2450878|92598|94604|5057|37463|92598|94604|5057|37463|4|19|19|1|10294|206|1511|1|94.15|140.28|19.63|120.65|19.63|94.15|140.28|1.57|0.00|40.68|19.63|21.20|60.31|61.88|-74.52| +2450844|66037|2450885|92598|94604|5057|37463|92598|94604|5057|37463|4|31|4|4|12958|8|1511|7|81.63|205.70|121.36|590.38|849.52|571.41|1439.90|59.46|0.00|57.54|849.52|908.98|907.06|966.52|278.11| +2450844|66037|2450882|92598|94604|5057|37463|92598|94604|5057|37463|4|6|15|1|10060|180|1511|28|47.28|137.11|83.63|1497.44|2341.64|1323.84|3839.08|23.41|0.00|1151.64|2341.64|2365.05|3493.28|3516.69|1017.80| +2450844|66037|2450880|92598|94604|5057|37463|92598|94604|5057|37463|4|99|4|4|6026|179|1511|69|15.11|29.61|3.84|1778.13|264.96|1042.59|2043.09|23.84|0.00|102.12|264.96|288.80|367.08|390.92|-777.63| +2450844|66037|2450852|92598|94604|5057|37463|92598|94604|5057|37463|4|87|9|5|1550|182|1511|54|53.17|60.08|27.63|1752.30|1492.02|2871.18|3244.32|0.00|0.00|1199.88|1492.02|1492.02|2691.90|2691.90|-1379.16| +2450844|66037|2450933|92598|94604|5057|37463|92598|94604|5057|37463|4|48|1|1|3602|266|1511|67|49.70|88.46|53.96|2311.50|3615.32|3329.90|5926.82|216.91|0.00|2963.41|3615.32|3832.23|6578.73|6795.64|285.42| +2450844|66037|2450894|92598|94604|5057|37463|92598|94604|5057|37463|4|1|20|5|5293|274|1511|95|5.10|13.66|10.65|285.95|1011.75|484.50|1297.70|0.00|0.00|310.65|1011.75|1011.75|1322.40|1322.40|527.25| +2450844|66037|2450908|92598|94604|5057|37463|92598|94604|5057|37463|4|1|11|1|6448|6|1511|20|86.51|129.76|48.01|1635.00|960.20|1730.20|2595.20|48.01|0.00|181.60|960.20|1008.21|1141.80|1189.81|-770.00| +2450844|57788|2450929|93405|1606790|4975|17960|93405|1606790|4975|17960|1|32|6|5|4058|126|1512|61|87.29|164.97|44.54|7346.23|2716.94|5324.69|10063.17|15.48|2200.72|3119.54|516.22|531.70|3635.76|3651.24|-4808.47| +2450844|57788|2450914|93405|1606790|4975|17960|93405|1606790|4975|17960|1|9|20|3|17581|291|1512|33|72.92|84.58|17.76|2205.06|586.08|2406.36|2791.14|21.09|351.64|1144.11|234.44|255.53|1378.55|1399.64|-2171.92| +2450844|57788|2450864|93405|1606790|4975|17960|93405|1606790|4975|17960|1|24|4|5|7852|210|1512|35|47.71|51.52|21.12|1064.00|739.20|1669.85|1803.20|0.00|510.04|108.15|229.16|229.16|337.31|337.31|-1440.69| +2450844|57788|2450910|93405|1606790|4975|17960|93405|1606790|4975|17960|1|32|13|4|9631|14|1512|71|32.72|34.35|7.55|1902.80|536.05|2323.12|2438.85|5.36|0.00|1170.08|536.05|541.41|1706.13|1711.49|-1787.07| +2450844|57788|2450889|93405|1606790|4975|17960|93405|1606790|4975|17960|1|52|10|1|6790|246|1512|56|26.66|29.85|17.91|668.64|1002.96|1492.96|1671.60|30.08|0.00|618.24|1002.96|1033.04|1621.20|1651.28|-490.00| +2450844|57788|2450927|93405|1606790|4975|17960|93405|1606790|4975|17960|1|58|10|5|9436|282|1512|95|35.37|93.02|16.74|7246.60|1590.30|3360.15|8836.90|95.41|0.00|2385.45|1590.30|1685.71|3975.75|4071.16|-1769.85| +2450844|57788|2450914|93405|1606790|4975|17960|93405|1606790|4975|17960|1|30|5|1|1706|276|1512|100|62.97|115.23|40.33|7490.00|4033.00|6297.00|11523.00|116.15|2742.44|921.00|1290.56|1406.71|2211.56|2327.71|-5006.44| +2450844|57788|2450891|93405|1606790|4975|17960|93405|1606790|4975|17960|1|5|2|2|17468|124|1512|26|5.09|9.06|3.44|146.12|89.44|132.34|235.56|8.04|0.00|65.78|89.44|97.48|155.22|163.26|-42.90| +2450844|57788|2450926|93405|1606790|4975|17960|93405|1606790|4975|17960|1|83|18|4|11858|7|1512|5|54.10|81.69|60.45|106.20|302.25|270.50|408.45|10.27|45.33|61.25|256.92|267.19|318.17|328.44|-13.58| +2450844|58663|2450879|71314|564519|6877|4835|71314|564519|6877|4835|4|41|18|1|3058|275|1513|18|2.25|5.60|5.60|0.00|100.80|40.50|100.80|5.04|0.00|3.96|100.80|105.84|104.76|109.80|60.30| +2450844|58663|2450881|71314|564519|6877|4835|71314|564519|6877|4835|4|89|12|5|2752|243|1513|19|67.69|84.61|73.61|209.00|1398.59|1286.11|1607.59|72.16|195.80|417.81|1202.79|1274.95|1620.60|1692.76|-83.32| +2450844|58663|2450867|71314|564519|6877|4835|71314|564519|6877|4835|4|32|12|3|1780|253|1513|98|34.80|36.88|18.44|1807.12|1807.12|3410.40|3614.24|36.14|0.00|541.94|1807.12|1843.26|2349.06|2385.20|-1603.28| +2450844|58663|2450848|71314|564519|6877|4835|71314|564519|6877|4835|4|92|4|1|604|5|1513|63|31.21|61.79|2.47|3737.16|155.61|1966.23|3892.77|10.89|0.00|233.10|155.61|166.50|388.71|399.60|-1810.62| +2450844|58663|2450884|71314|564519|6877|4835|71314|564519|6877|4835|4|24|11|5|2722|123|1513|85|72.85|137.68|53.69|7139.15|4563.65|6192.25|11702.80|410.72|0.00|4212.60|4563.65|4974.37|8776.25|9186.97|-1628.60| +2450844|58663|2450928|71314|564519|6877|4835|71314|564519|6877|4835|4|4|9|2|16660|129|1513|60|7.22|20.86|13.35|450.60|801.00|433.20|1251.60|64.08|0.00|350.40|801.00|865.08|1151.40|1215.48|367.80| +2450844|58663|2450870|71314|564519|6877|4835|71314|564519|6877|4835|4|42|9|1|3218|101|1513|56|94.70|203.60|109.94|5244.96|6156.64|5303.20|11401.60|0.00|0.00|911.68|6156.64|6156.64|7068.32|7068.32|853.44| +2450844|58663|2450926|71314|564519|6877|4835|71314|564519|6877|4835|4|65|11|3|14776|250|1513|40|45.73|135.81|40.74|3802.80|1629.60|1829.20|5432.40|130.36|0.00|1194.80|1629.60|1759.96|2824.40|2954.76|-199.60| +2450844|58663|2450865|71314|564519|6877|4835|71314|564519|6877|4835|4|99|18|2|5986|232|1513|25|40.35|59.31|2.96|1408.75|74.00|1008.75|1482.75|2.22|0.00|88.75|74.00|76.22|162.75|164.97|-934.75| +2450844|58663|2450887|71314|564519|6877|4835|71314|564519|6877|4835|4|72|20|1|10933|83|1513|73|41.79|48.05|11.05|2701.00|806.65|3050.67|3507.65|72.59|0.00|1332.25|806.65|879.24|2138.90|2211.49|-2244.02| +2450844|58663|2450900|71314|564519|6877|4835|71314|564519|6877|4835|4|78|14|1|13136|52|1513|84|75.64|87.74|35.09|4422.60|2947.56|6353.76|7370.16|265.28|0.00|3095.40|2947.56|3212.84|6042.96|6308.24|-3406.20| +2450844|58663|2450892|71314|564519|6877|4835|71314|564519|6877|4835|4|82|19|1|3314|188|1513|26|34.35|86.21|3.44|2152.02|89.44|893.10|2241.46|1.78|0.00|470.60|89.44|91.22|560.04|561.82|-803.66| +2450844|58663|2450923|71314|564519|6877|4835|71314|564519|6877|4835|4|64|20|4|12172|95|1513|80|69.77|98.37|29.51|5508.80|2360.80|5581.60|7869.60|165.25|0.00|1573.60|2360.80|2526.05|3934.40|4099.65|-3220.80| +2450844|72243|2450908|54078|1686051|6377|41550|54078|1686051|6377|41550|2|85|20|5|17701|80|1514|88|94.18|168.58|87.66|7120.96|7714.08|8287.84|14835.04|539.98|0.00|6081.68|7714.08|8254.06|13795.76|14335.74|-573.76| +2450844|72243|2450861|54078|1686051|6377|41550|54078|1686051|6377|41550|2|60|6|4|10951|45|1514|84|82.00|100.86|57.49|3643.08|4829.16|6888.00|8472.24|386.33|0.00|168.84|4829.16|5215.49|4998.00|5384.33|-2058.84| +2450844|72243|2450853|54078|1686051|6377|41550|54078|1686051|6377|41550|2|56|4|5|11125|99|1514|63|71.35|103.45|24.82|4953.69|1563.66|4495.05|6517.35|15.63|0.00|1368.36|1563.66|1579.29|2932.02|2947.65|-2931.39| +2450844|72243|2450888|54078|1686051|6377|41550|54078|1686051|6377|41550|2|57|16|1|6025|149|1514|24|37.64|40.27|4.42|860.40|106.08|903.36|966.48|8.48|0.00|212.40|106.08|114.56|318.48|326.96|-797.28| +2450844|72243|2450907|54078|1686051|6377|41550|54078|1686051|6377|41550|2|45|9|4|550|11|1514|54|2.14|3.53|2.78|40.50|150.12|115.56|190.62|1.50|0.00|81.54|150.12|151.62|231.66|233.16|34.56| +2450844|72243|2450879|54078|1686051|6377|41550|54078|1686051|6377|41550|2|61|1|5|12806|156|1514|54|39.75|106.53|31.95|4027.32|1725.30|2146.50|5752.62|0.00|0.00|1323.00|1725.30|1725.30|3048.30|3048.30|-421.20| +2450844|72243|2450912|54078|1686051|6377|41550|54078|1686051|6377|41550|2|93|17|5|4376|249|1514|62|44.72|44.72|38.01|416.02|2356.62|2772.64|2772.64|94.26|0.00|748.34|2356.62|2450.88|3104.96|3199.22|-416.02| +2450844|72243|2450895|54078|1686051|6377|41550|54078|1686051|6377|41550|2|104|1|1|15512|96|1514|25|82.95|83.77|22.61|1529.00|565.25|2073.75|2094.25|34.42|73.48|251.25|491.77|526.19|743.02|777.44|-1581.98| +2450844|72243|2450930|54078|1686051|6377|41550|54078|1686051|6377|41550|2|28|18|5|1142|58|1514|6|27.00|40.50|3.24|223.56|19.44|162.00|243.00|0.97|0.00|65.58|19.44|20.41|85.02|85.99|-142.56| +2450844|72243|2450860|54078|1686051|6377|41550|54078|1686051|6377|41550|2|35|13|3|11116|155|1514|84|33.93|47.50|29.45|1516.20|2473.80|2850.12|3990.00|123.69|0.00|877.80|2473.80|2597.49|3351.60|3475.29|-376.32| +2450844|72243|2450883|54078|1686051|6377|41550|54078|1686051|6377|41550|2|4|8|1|308|181|1514|89|45.73|71.33|27.10|3936.47|2411.90|4069.97|6348.37|105.64|651.21|507.30|1760.69|1866.33|2267.99|2373.63|-2309.28| +2450844|72243|2450888|54078|1686051|6377|41550|54078|1686051|6377|41550|2|108|1|4|12104|160|1514|68|41.15|119.33|48.92|4787.88|3326.56|2798.20|8114.44|266.12|0.00|2109.36|3326.56|3592.68|5435.92|5702.04|528.36| +2450844|72243|2450847|54078|1686051|6377|41550|54078|1686051|6377|41550|2|13|19|4|10744|104|1514|66|7.66|10.57|10.35|14.52|683.10|505.56|697.62|0.00|0.00|90.42|683.10|683.10|773.52|773.52|177.54| +2450844|72243|2450900|54078|1686051|6377|41550|54078|1686051|6377|41550|2|56|1|3|1933|22|1514|69|23.31|25.87|19.91|411.24|1373.79|1608.39|1785.03|68.68|0.00|267.72|1373.79|1442.47|1641.51|1710.19|-234.60| +2450844|61831|2450886|3803|275985|967|31149|3803|275985|967|31149|4|6|11|2|5762|31|1515|33|92.54|140.66|63.29|2553.21|2088.57|3053.82|4641.78|41.77|0.00|1113.75|2088.57|2130.34|3202.32|3244.09|-965.25| +2450844|61831|2450888|3803|275985|967|31149|3803|275985|967|31149|4|79|11|3|14968|280|1515|15|62.62|79.52|73.95|83.55|1109.25|939.30|1192.80|55.46|0.00|453.15|1109.25|1164.71|1562.40|1617.86|169.95| +2450844|61831|2450874|3803|275985|967|31149|3803|275985|967|31149|4|27|20|1|9182|101|1515|58|88.99|169.08|155.55|784.74|9021.90|5161.42|9806.64|90.21|0.00|0.00|9021.90|9112.11|9021.90|9112.11|3860.48| +2450844|61831|2450852|3803|275985|967|31149|3803|275985|967|31149|4|108|11|1|7510|191|1515|78|93.73|224.95|159.71|5088.72|12457.38|7310.94|17546.10|622.86|0.00|3158.22|12457.38|13080.24|15615.60|16238.46|5146.44| +2450844|61831|2450869|3803|275985|967|31149|3803|275985|967|31149|4|91|19|2|13726|195|1515|16|52.78|139.86|138.46|22.40|2215.36|844.48|2237.76|44.30|0.00|895.04|2215.36|2259.66|3110.40|3154.70|1370.88| +2450844|61831|2450848|3803|275985|967|31149|3803|275985|967|31149|4|13|12|3|16246|292|1515|72|13.00|20.67|10.74|714.96|773.28|936.00|1488.24|0.00|0.00|133.92|773.28|773.28|907.20|907.20|-162.72| +2450844|61831|2450848|3803|275985|967|31149|3803|275985|967|31149|4|45|12|1|10897|119|1515|97|46.28|75.89|50.08|2503.57|4857.76|4489.16|7361.33|291.46|0.00|1692.65|4857.76|5149.22|6550.41|6841.87|368.60| +2450844|61831|2450926|3803|275985|967|31149|3803|275985|967|31149|4|30|9|4|14030|92|1515|59|15.46|23.19|5.56|1040.17|328.04|912.14|1368.21|0.00|0.00|464.92|328.04|328.04|792.96|792.96|-584.10| +2450844|61831|2450879|3803|275985|967|31149|3803|275985|967|31149|4|38|13|2|9361|161|1515|59|96.81|194.58|159.55|2066.77|9413.45|5711.79|11480.22|564.80|0.00|459.02|9413.45|9978.25|9872.47|10437.27|3701.66| +2450844|68025|2450866|54496|515485|6074|44263|54496|515485|6074|44263|2|51|15|3|6646|159|1516|71|25.42|76.26|0.00|5414.46|0.00|1804.82|5414.46|0.00|0.00|2165.50|0.00|0.00|2165.50|2165.50|-1804.82| +2450844|68025|2450926|54496|515485|6074|44263|54496|515485|6074|44263|2|8|10|5|7282|165|1516|42|55.51|160.42|136.35|1010.94|5726.70|2331.42|6737.64|458.13|0.00|2627.52|5726.70|6184.83|8354.22|8812.35|3395.28| +2450844|68025|2450852|54496|515485|6074|44263|54496|515485|6074|44263|2|15|14|1|8869|202|1516|73|22.57|64.77|34.97|2175.40|2552.81|1647.61|4728.21|153.16|0.00|1985.60|2552.81|2705.97|4538.41|4691.57|905.20| +2450844|68025|2450857|54496|515485|6074|44263|54496|515485|6074|44263|2|52|9|5|7387|5|1516|71|71.07|137.16|80.92|3993.04|5745.32|5045.97|9738.36|114.90|0.00|3310.73|5745.32|5860.22|9056.05|9170.95|699.35| +2450844|68025|2450901|54496|515485|6074|44263|54496|515485|6074|44263|2|101|9|1|15242|7|1516|77|79.58|142.44|116.80|1974.28|8993.60|6127.66|10967.88|449.68|0.00|5154.38|8993.60|9443.28|14147.98|14597.66|2865.94| +2450844|68025|2450851|54496|515485|6074|44263|54496|515485|6074|44263|2|85|2|2|13879|135|1516|20|90.16|238.92|83.62|3106.00|1672.40|1803.20|4778.40|0.00|0.00|955.60|1672.40|1672.40|2628.00|2628.00|-130.80| +2450844|68025|2450881|54496|515485|6074|44263|54496|515485|6074|44263|2|17|19|4|10675|191|1516|78|88.08|115.38|23.07|7200.18|1799.46|6870.24|8999.64|42.10|1097.67|1799.46|701.79|743.89|2501.25|2543.35|-6168.45| +2450844|38972|2450908|46467|1043783|7146|32462|46467|1043783|7146|32462|1|86|8|3|15496|95|1517|78|25.92|52.09|30.73|1666.08|2396.94|2021.76|4063.02|47.93|0.00|446.16|2396.94|2444.87|2843.10|2891.03|375.18| +2450844|38972|2450863|46467|1043783|7146|32462|46467|1043783|7146|32462|1|8|14|5|17686|105|1517|31|43.02|49.90|14.47|1098.33|448.57|1333.62|1546.90|26.91|0.00|340.07|448.57|475.48|788.64|815.55|-885.05| +2450844|38972|2450901|46467|1043783|7146|32462|46467|1043783|7146|32462|1|13|5|2|13921|56|1517|73|89.77|189.41|37.88|11061.69|2765.24|6553.21|13826.93|114.75|470.09|552.61|2295.15|2409.90|2847.76|2962.51|-4258.06| +2450844|38972|2450898|46467|1043783|7146|32462|46467|1043783|7146|32462|1|57|11|1|7777|118|1517|18|20.47|40.12|4.41|642.78|79.38|368.46|722.16|2.38|0.00|151.56|79.38|81.76|230.94|233.32|-289.08| +2450844|38972|2450887|46467|1043783|7146|32462|46467|1043783|7146|32462|1|75|6|5|2168|95|1517|52|93.14|134.12|130.09|209.56|6764.68|4843.28|6974.24|202.94|0.00|69.68|6764.68|6967.62|6834.36|7037.30|1921.40| +2450844|38972|2450915|46467|1043783|7146|32462|46467|1043783|7146|32462|1|21|19|5|5582|28|1517|38|63.92|159.80|3.19|5951.18|121.22|2428.96|6072.40|1.21|0.00|182.02|121.22|122.43|303.24|304.45|-2307.74| +2450844|38972|2450859|46467|1043783|7146|32462|46467|1043783|7146|32462|1|59|8|3|16627|25|1517|60|93.56|195.54|29.33|9972.60|1759.80|5613.60|11732.40|52.79|0.00|3402.00|1759.80|1812.59|5161.80|5214.59|-3853.80| +2450844|38972|2450886|46467|1043783|7146|32462|46467|1043783|7146|32462|1|86|17|4|475|230|1517|31|81.94|174.53|134.38|1244.65|4165.78|2540.14|5410.43|124.97|0.00|919.77|4165.78|4290.75|5085.55|5210.52|1625.64| +2450844|38972|2450910|46467|1043783|7146|32462|46467|1043783|7146|32462|1|67|6|3|13148|172|1517|48|14.30|28.45|16.78|560.16|805.44|686.40|1365.60|32.21|0.00|422.88|805.44|837.65|1228.32|1260.53|119.04| +2450844|38972|2450905|46467|1043783|7146|32462|46467|1043783|7146|32462|1|4|16|2|13942|145|1517|92|96.71|184.71|162.54|2039.64|14953.68|8897.32|16993.32|897.22|0.00|4247.64|14953.68|15850.90|19201.32|20098.54|6056.36| +2450844|38972|2450899|46467|1043783|7146|32462|46467|1043783|7146|32462|1|26|2|4|15700|221|1517|12|67.15|180.63|93.92|1040.52|1127.04|805.80|2167.56|33.81|0.00|585.24|1127.04|1160.85|1712.28|1746.09|321.24| +2450844|38972|2450910|46467|1043783|7146|32462|46467|1043783|7146|32462|1|74|18|4|9848|51|1517|56|75.24|213.68|196.58|957.60|11008.48|4213.44|11966.08|220.16|0.00|3110.80|11008.48|11228.64|14119.28|14339.44|6795.04| +2450844|38972|2450932|46467|1043783|7146|32462|46467|1043783|7146|32462|1|53|5|1|6007|22|1517|16|24.64|50.51|21.71|460.80|347.36|394.24|808.16|13.89|0.00|129.28|347.36|361.25|476.64|490.53|-46.88| +2450844|54415|2450895|92253|1801821|2589|49357|92253|1801821|2589|49357|4|20|19|1|5920|162|1518|23|83.29|92.45|30.50|1424.85|701.50|1915.67|2126.35|14.94|203.43|488.98|498.07|513.01|987.05|1001.99|-1417.60| +2450844|54415|2450859|92253|1801821|2589|49357|92253|1801821|2589|49357|4|38|11|1|1258|193|1518|3|65.69|89.99|56.69|99.90|170.07|197.07|269.97|10.20|0.00|99.87|170.07|180.27|269.94|280.14|-27.00| +2450844|54415|2450875|92253|1801821|2589|49357|92253|1801821|2589|49357|4|57|12|3|11810|178|1518|34|94.77|163.95|142.63|724.88|4849.42|3222.18|5574.30|18.91|4218.99|1449.08|630.43|649.34|2079.51|2098.42|-2591.75| +2450844|54415|2450867|92253|1801821|2589|49357|92253|1801821|2589|49357|4|16|17|4|9572|243|1518|79|98.76|124.43|107.00|1376.97|8453.00|7802.04|9829.97|82.83|7269.58|491.38|1183.42|1266.25|1674.80|1757.63|-6618.62| +2450844|54415||92253|||49357||1801821||||49|18||13789||1518|49|71.41|||2388.26|||6822.76|||1227.94|4434.50|4656.22|5662.44|5884.16|935.41| +2450844|54415|2450866|92253|1801821|2589|49357|92253|1801821|2589|49357|4|13|15|1|16141|293|1518|26|35.93|61.08|20.76|1048.32|539.76|934.18|1588.08|21.59|0.00|762.06|539.76|561.35|1301.82|1323.41|-394.42| +2450844|54415|2450858|92253|1801821|2589|49357|92253|1801821|2589|49357|4|97|20|3|2120|37|1518|28|15.39|21.54|14.21|205.24|397.88|430.92|603.12|31.83|0.00|229.04|397.88|429.71|626.92|658.75|-33.04| +2450844|54415|2450852|92253|1801821|2589|49357|92253|1801821|2589|49357|4|58|16|3|4153|170|1518|96|3.57|5.96|3.57|229.44|342.72|342.72|572.16|9.87|95.96|85.44|246.76|256.63|332.20|342.07|-95.96| +2450844|35461|2450867|24660|1694187|6561|18888|24660|1694187|6561|18888|4|49|4|3|4810|171|1519|78|57.32|146.73|101.24|3548.22|7896.72|4470.96|11444.94|85.28|5053.90|2746.38|2842.82|2928.10|5589.20|5674.48|-1628.14| +2450844|35461|2450870|24660|1694187|6561|18888|24660|1694187|6561|18888|4|37|14|2|7|294|1519|84|27.11|33.88|26.08|655.20|2190.72|2277.24|2845.92|30.23|679.12|568.68|1511.60|1541.83|2080.28|2110.51|-765.64| +2450844|35461|2450910|24660|1694187|6561|18888|24660|1694187|6561|18888|4|96|4|2|2833|51|1519|15|1.18|2.18|0.63|23.25|9.45|17.70|32.70|0.18|0.00|14.25|9.45|9.63|23.70|23.88|-8.25| +2450844|35461|2450914|24660|1694187|6561|18888|24660|1694187|6561|18888|4|47|13|3|4744|91|1519|68|27.56|33.34|12.33|1428.68|838.44|1874.08|2267.12|15.59|318.60|1110.44|519.84|535.43|1630.28|1645.87|-1354.24| +2450844|70125|2450900|72517|1732926|4088|40497|72517|1732926|4088|40497|1|28|14|2|16390|231|1520|8|20.86|57.15|36.57|164.64|292.56|166.88|457.20|5.85|0.00|160.00|292.56|298.41|452.56|458.41|125.68| +2450844|70125|2450868|72517|1732926|4088|40497|72517|1732926|4088|40497|1|40|19|3|15170|246|1520|55|5.79|16.61|0.00|913.55|0.00|318.45|913.55|0.00|0.00|420.20|0.00|0.00|420.20|420.20|-318.45| +2450844|70125|2450910|72517|1732926|4088|40497|72517|1732926|4088|40497|1|96|14|3|17498|55|1520|83|28.69|59.67|25.06|2872.63|2079.98|2381.27|4952.61|62.39|0.00|1188.56|2079.98|2142.37|3268.54|3330.93|-301.29| +2450844|70125|2450921|72517|1732926|4088|40497|72517|1732926|4088|40497|1|69|12|5|2695|291|1520|7|44.62|51.31|41.04|71.89|287.28|312.34|359.17|20.10|0.00|157.99|287.28|307.38|445.27|465.37|-25.06| +2450844|70125|2450934|72517|1732926|4088|40497|72517|1732926|4088|40497|1|18|12|5|2473|69|1520|39|2.81|6.96|2.99|154.83|116.61|109.59|271.44|0.99|96.78|108.42|19.83|20.82|128.25|129.24|-89.76| +2450844|70125|2450867|72517|1732926|4088|40497|72517|1732926|4088|40497|1|26|15|5|4486|118|1520|25|55.49|134.84|129.44|135.00|3236.00|1387.25|3371.00|4.53|3009.48|674.00|226.52|231.05|900.52|905.05|-1160.73| +2450844|70125|2450868|72517|1732926|4088|40497|72517|1732926|4088|40497|1|98|8|2|10432|187|1520|62|11.81|31.29|14.08|1067.02|872.96|732.22|1939.98|26.18|0.00|872.96|872.96|899.14|1745.92|1772.10|140.74| +2450844|70125|2450917|72517|1732926|4088|40497|72517|1732926|4088|40497|1|68|11|2|7537|153|1520|35|31.10|74.64|65.68|313.60|2298.80|1088.50|2612.40|45.97|0.00|940.45|2298.80|2344.77|3239.25|3285.22|1210.30| +2450844|70125|2450857|72517|1732926|4088|40497|72517|1732926|4088|40497|1|106|6|4|3019|118|1520|20|40.93|74.08|24.44|992.80|488.80|818.60|1481.60|9.77|0.00|548.00|488.80|498.57|1036.80|1046.57|-329.80| +2450844|70125|2450898|72517|1732926|4088|40497|72517|1732926|4088|40497|1|57|5|2|7616|193|1520|76|73.83|83.42|9.17|5643.00|696.92|5611.08|6339.92|27.87|0.00|2028.44|696.92|724.79|2725.36|2753.23|-4914.16| +2450844|70125|2450932|72517|1732926|4088|40497|72517|1732926|4088|40497|1|7|14|2|11554|173|1520|13|84.29|89.34|59.85|383.37|778.05|1095.77|1161.42|6.30|567.97|209.04|210.08|216.38|419.12|425.42|-885.69| +2450844|36156|2450867|21065|1336663|2214|26212|21065|1336663|2214|26212|4|82|14|4|17254|134|1521|26|53.50|83.46|61.76|564.20|1605.76|1391.00|2169.96|48.17|0.00|716.04|1605.76|1653.93|2321.80|2369.97|214.76| +2450844|36156|2450873|21065|1336663|2214|26212|21065|1336663|2214|26212|4|76|1|3|1774|94|1521|99|17.41|25.24|8.07|1699.83|798.93|1723.59|2498.76|31.95|0.00|124.74|798.93|830.88|923.67|955.62|-924.66| +2450844|36156|2450930|21065|1336663|2214|26212|21065|1336663|2214|26212|4|46|8|3|2491|43|1521|92|61.11|111.22|43.37|6242.20|3990.04|5622.12|10232.24|119.70|0.00|5013.08|3990.04|4109.74|9003.12|9122.82|-1632.08| +2450844|36156|2450902|21065|1336663|2214|26212|21065|1336663|2214|26212|4|55|5|1|6134|11|1521|53|40.51|84.66|34.71|2647.35|1839.63|2147.03|4486.98|73.58|0.00|1345.67|1839.63|1913.21|3185.30|3258.88|-307.40| +2450844|36156|2450865|21065|1336663|2214|26212|21065|1336663|2214|26212|4|24|10|2|7430|189|1521|96|74.56|93.20|0.93|8857.92|89.28|7157.76|8947.20|0.00|24.99|3399.36|64.29|64.29|3463.65|3463.65|-7093.47| +2450844|36156|2450910|21065|1336663|2214|26212|21065|1336663|2214|26212|4|108|2|3|1670|48|1521|30|85.14|137.92|31.72|3186.00|951.60|2554.20|4137.60|66.61|0.00|703.20|951.60|1018.21|1654.80|1721.41|-1602.60| +2450844|36156|2450877|21065|1336663|2214|26212|21065|1336663|2214|26212|4|71|9|4|6280|174|1521|7|79.12|159.82|75.11|592.97|525.77|553.84|1118.74|15.77|0.00|22.33|525.77|541.54|548.10|563.87|-28.07| +2450844|36156|2450910|21065|1336663|2214|26212|21065|1336663|2214|26212|4|47|12|1|2278|219|1521|46|38.13|106.38|80.84|1174.84|3718.64|1753.98|4893.48|37.18|0.00|1419.10|3718.64|3755.82|5137.74|5174.92|1964.66| +2450844|57170|2450857|26155|1266946|2137|26128|26155|1266946|2137|26128|4|2|14|3|15517|141|1522|65|95.92|163.06|83.16|5193.50|5405.40|6234.80|10598.90|108.10|0.00|3391.05|5405.40|5513.50|8796.45|8904.55|-829.40| +2450844|57170|2450867|26155|1266946|2137|26128|26155|1266946|2137|26128|4|13|2|4|14197|116|1522|10|55.00|102.30|37.85|644.50|378.50|550.00|1023.00|15.14|0.00|388.70|378.50|393.64|767.20|782.34|-171.50| +2450844|57170|2450893|26155|1266946|2137|26128|26155|1266946|2137|26128|4|71|15|5|8467|143|1522|32|88.47|200.82|78.31|3920.32|2505.92|2831.04|6426.24|25.05|0.00|835.20|2505.92|2530.97|3341.12|3366.17|-325.12| +2450844|57170|2450927|26155|1266946|2137|26128|26155|1266946|2137|26128|4|25|11|5|12152|60|1522|87|42.88|63.89|17.25|4057.68|1500.75|3730.56|5558.43|45.02|0.00|1500.75|1500.75|1545.77|3001.50|3046.52|-2229.81| +2450844|57170|2450899|26155|1266946|2137|26128|26155|1266946|2137|26128|4|58|17|3|15022|63|1522|75|53.08|107.22|60.04|3538.50|4503.00|3981.00|8041.50|180.12|0.00|80.25|4503.00|4683.12|4583.25|4763.37|522.00| +2450844|57170|2450883|26155|1266946|2137|26128|26155|1266946|2137|26128|4|42|3|1|4304|52|1522|73|98.63|165.69|109.35|4112.82|7982.55|7199.99|12095.37|0.00|558.77|2539.67|7423.78|7423.78|9963.45|9963.45|223.79| +2450844|33815|2450861|82679|569132|7138|39591|82679|569132|7138|39591|4|99|14|3|10267|91|1523|25|45.77|48.05|46.12|48.25|1153.00|1144.25|1201.25|103.77|0.00|96.00|1153.00|1256.77|1249.00|1352.77|8.75| +2450844|33815|2450930|82679|569132|7138|39591|82679|569132|7138|39591|4|6|16|3|7850|160|1523|56|87.66|169.18|62.59|5969.04|3505.04|4908.96|9474.08|70.10|0.00|1326.08|3505.04|3575.14|4831.12|4901.22|-1403.92| +2450844|33815|2450909|82679|569132|7138|39591|82679|569132|7138|39591|4|26|12|4|10346|219|1523|2|71.84|207.61|137.02|141.18|274.04|143.68|415.22|10.96|0.00|132.86|274.04|285.00|406.90|417.86|130.36| +2450844|33815|2450913|82679|569132|7138|39591|82679|569132|7138|39591|4|54|19|1|11894|207|1523|50|22.37|54.58|13.64|2047.00|682.00|1118.50|2729.00|54.56|0.00|1282.50|682.00|736.56|1964.50|2019.06|-436.50| +2450844|33815|2450871|82679|569132|7138|39591|82679|569132|7138|39591|4|32|5|2|4537|85|1523|64|40.84|114.76|110.16|294.40|7050.24|2613.76|7344.64|282.00|0.00|3451.52|7050.24|7332.24|10501.76|10783.76|4436.48| +2450844|33815|2450901|82679|569132|7138|39591|82679|569132|7138|39591|4|1|12|3|6526|52|1523|85|74.82|151.13|9.06|12075.95|770.10|6359.70|12846.05|46.20|0.00|3211.30|770.10|816.30|3981.40|4027.60|-5589.60| +2450844|33815|2450886|82679|569132|7138|39591|82679|569132|7138|39591|4|21|11|3|790|269|1523|23|50.07|96.63|88.89|178.02|2044.47|1151.61|2222.49|122.66|0.00|555.45|2044.47|2167.13|2599.92|2722.58|892.86| +2450844|33815|2450924|82679|569132|7138|39591|82679|569132|7138|39591|4|30|8|5|17744|167|1523|45|48.44|128.36|127.07|58.05|5718.15|2179.80|5776.20|57.18|0.00|2367.90|5718.15|5775.33|8086.05|8143.23|3538.35| +2450844|33815|2450924|82679|569132|7138|39591|82679|569132|7138|39591|4|6|2|1|9218|239|1523|62|24.41|71.03|37.64|2070.18|2333.68|1513.42|4403.86|116.68|0.00|440.20|2333.68|2450.36|2773.88|2890.56|820.26| +2450844|33815|2450900|82679|569132|7138|39591|82679|569132|7138|39591|4|5|7|3|17674|178|1523|11|14.89|37.67|25.23|136.84|277.53|163.79|414.37|0.00|0.00|99.44|277.53|277.53|376.97|376.97|113.74| +2450844|33815|2450933|82679|569132|7138|39591|82679|569132|7138|39591|4|61|9|3|2912|131|1523|21|86.09|120.52|40.97|1670.55|860.37|1807.89|2530.92|17.20|0.00|328.86|860.37|877.57|1189.23|1206.43|-947.52| +2450844|71833|2450927|86558|243783|7055|20280|86558|243783|7055|20280|2|24|5|3|14845|1|1524|34|81.80|244.58|34.24|7151.56|1164.16|2781.20|8315.72|0.00|0.00|2577.54|1164.16|1164.16|3741.70|3741.70|-1617.04| +2450844|71833|2450890|86558|243783|7055|20280|86558|243783|7055|20280|2|104|19|3|8768|189|1524|96|54.20|142.54|19.95|11768.64|1915.20|5203.20|13683.84|38.30|0.00|3283.20|1915.20|1953.50|5198.40|5236.70|-3288.00| +2450844|71833|2450895|86558|243783|7055|20280|86558|243783|7055|20280|2|3|20|5|11888|120|1524|35|96.09|195.06|19.50|6144.60|682.50|3363.15|6827.10|34.12|0.00|2730.70|682.50|716.62|3413.20|3447.32|-2680.65| +2450844|71833|2450913|86558|243783|7055|20280|86558|243783|7055|20280|2|64|19|3|2632|128|1524|70|2.33|2.53|1.74|55.30|121.80|163.10|177.10|7.30|0.00|14.00|121.80|129.10|135.80|143.10|-41.30| +2450844|71833|2450917|86558|243783|7055|20280|86558|243783|7055|20280|2|56|10|2|1699|56|1524|31|54.89|134.48|60.51|2293.07|1875.81|1701.59|4168.88|93.79|0.00|1500.71|1875.81|1969.60|3376.52|3470.31|174.22| +2450844|71833|2450870|86558|243783|7055|20280|86558|243783|7055|20280|2|77|19|4|5890|8|1524|23|16.63|21.12|19.43|38.87|446.89|382.49|485.76|0.00|218.97|242.88|227.92|227.92|470.80|470.80|-154.57| +2450844|71833|2450849|86558|243783|7055|20280|86558|243783|7055|20280|2|98|13|1|8140|262|1524|30|24.73|38.33|21.84|494.70|655.20|741.90|1149.90|40.81|72.07|241.20|583.13|623.94|824.33|865.14|-158.77| +2450844|71833|2450881|86558|243783|7055|20280|86558|243783|7055|20280|2|105|4|1|10384|22|1524|33|40.75|46.45|20.43|858.66|674.19|1344.75|1532.85|0.00|202.25|704.88|471.94|471.94|1176.82|1176.82|-872.81| +2450844|71833|2450898|86558|243783|7055|20280|86558|243783|7055|20280|2|106|13|4|6676|51|1524|52|92.36|185.64|16.70|8784.88|868.40|4802.72|9653.28|43.42|0.00|2413.32|868.40|911.82|3281.72|3325.14|-3934.32| +2450844|71833|2450892|86558|243783|7055|20280|86558|243783|7055|20280|2|43|19|2|4004|283|1524|55|27.39|32.86|23.33|524.15|1283.15|1506.45|1807.30|25.66|0.00|433.40|1283.15|1308.81|1716.55|1742.21|-223.30| +2450844|71833|2450907|86558|243783|7055|20280|86558|243783|7055|20280|2|47|9|1|15232|228|1524|60|61.08|147.81|82.77|3902.40|4966.20|3664.80|8868.60|347.63|0.00|1152.60|4966.20|5313.83|6118.80|6466.43|1301.40| +2450844|71833|2450925|86558|243783|7055|20280|86558|243783|7055|20280|2|48|2|1|2083|149|1524|29|81.61|149.34|103.04|1342.70|2988.16|2366.69|4330.86|268.93|0.00|2035.22|2988.16|3257.09|5023.38|5292.31|621.47| +2450844|46049|2450857|57661|21459|5704|27549|57661|21459|5704|27549|2|36|9|5|8305|17|1525|3|57.08|122.72|8.59|342.39|25.77|171.24|368.16|2.06|0.00|51.54|25.77|27.83|77.31|79.37|-145.47| +2450844|46049|2450903|57661|21459|5704|27549|57661|21459|5704|27549|2|69|5|3|14863|143|1525|82|41.68|122.95|71.31|4234.48|5847.42|3417.76|10081.90|116.94|0.00|2217.28|5847.42|5964.36|8064.70|8181.64|2429.66| +2450844|46049|2450867|57661|21459|5704|27549|57661|21459|5704|27549|2|68|4|5|7516|190|1525|69|74.16|129.03|74.83|3739.80|5163.27|5117.04|8903.07|361.42|0.00|2848.32|5163.27|5524.69|8011.59|8373.01|46.23| +2450844|46049|2450860|57661|21459|5704|27549|57661|21459|5704|27549|2|72|3|3|3295|248|1525|22|91.22|194.29|101.03|2051.72|2222.66|2006.84|4274.38|0.00|0.00|470.14|2222.66|2222.66|2692.80|2692.80|215.82| +2450844|46049|2450851|57661|21459|5704|27549|57661|21459|5704|27549|2|57|14|2|8606|165|1525|12|69.07|163.69|116.21|569.76|1394.52|828.84|1964.28|27.89|0.00|707.04|1394.52|1422.41|2101.56|2129.45|565.68| +2450844|46049|2450861|57661|21459|5704|27549|57661|21459|5704|27549|2|69|16|2|8557|209|1525|26|46.36|50.53|32.84|459.94|853.84|1205.36|1313.78|0.00|0.00|78.78|853.84|853.84|932.62|932.62|-351.52| +2450844|46049|2450889|57661|21459|5704|27549|57661|21459|5704|27549|2|76|16|1|9901|234|1525|35|63.04|187.85|43.20|5062.75|1512.00|2206.40|6574.75|75.60|0.00|2564.10|1512.00|1587.60|4076.10|4151.70|-694.40| +2450844|46049|2450852|57661|21459|5704|27549|57661|21459|5704|27549|2|16|15|4|12742|73|1525|39|99.24|279.85|131.52|5784.87|5129.28|3870.36|10914.15|0.00|0.00|3383.25|5129.28|5129.28|8512.53|8512.53|1258.92| +2450844|46049|2450860|57661|21459|5704|27549|57661|21459|5704|27549|2|53|2|2|11288|169|1525|92|84.15|160.72|3.21|14490.92|295.32|7741.80|14786.24|11.81|0.00|4731.56|295.32|307.13|5026.88|5038.69|-7446.48| +2450844|46049|2450909|57661|21459|5704|27549|57661|21459|5704|27549|2|26|12|2|7286|72|1525|33|18.06|42.26|35.07|237.27|1157.31|595.98|1394.58|0.00|0.00|390.39|1157.31|1157.31|1547.70|1547.70|561.33| +2450844|46049|2450900|57661|21459|5704|27549|57661|21459|5704|27549|2|61|10|4|3778|15|1525|75|78.98|216.40|125.51|6816.75|9413.25|5923.50|16230.00|282.39|0.00|1785.00|9413.25|9695.64|11198.25|11480.64|3489.75| +2450844|61606|2450907|22814|341529|1044|24959|22814|341529|1044|24959|2|28|20|1|12610|198|1526|69|62.00|65.10|38.40|1842.30|2649.60|4278.00|4491.90|0.00|105.98|1257.18|2543.62|2543.62|3800.80|3800.80|-1734.38| +2450844|61606|2450917|22814|341529|1044|24959|22814|341529|1044|24959|2|27|3|4|8534|291|1526|81|94.76|208.47|93.81|9287.46|7598.61|7675.56|16886.07|75.98|0.00|336.96|7598.61|7674.59|7935.57|8011.55|-76.95| +2450844|61606|2450884|22814|341529|1044|24959|22814|341529|1044|24959|2|52|20|3|14762|187|1526|45|9.27|21.69|2.81|849.60|126.45|417.15|976.05|6.32|0.00|351.00|126.45|132.77|477.45|483.77|-290.70| +2450844|61606|2450928|22814|341529|1044|24959|22814|341529|1044|24959|2|75|1|3|1639|227|1526|80|29.83|64.73|54.37|828.80|4349.60|2386.40|5178.40|304.47|0.00|310.40|4349.60|4654.07|4660.00|4964.47|1963.20| +2450844|61606|2450895|22814|341529|1044|24959|22814|341529|1044|24959|2|18|7|2|11144|162|1526|33|84.10|163.15|154.99|269.28|5114.67|2775.30|5383.95|255.73|0.00|538.23|5114.67|5370.40|5652.90|5908.63|2339.37| +2450844|61606|2450901|22814|341529|1044|24959|22814|341529|1044|24959|2|27|8|5|15874|87|1526|73|82.20|129.87|59.74|5119.49|4361.02|6000.60|9480.51|305.27|0.00|3696.72|4361.02|4666.29|8057.74|8363.01|-1639.58| +2450844|41306|2450888|41734|979631|2328|44625|41734|979631|2328|44625|4|102|10|3|11383|40|1527|79|31.78|38.45|29.22|729.17|2308.38|2510.62|3037.55|0.00|0.00|1275.06|2308.38|2308.38|3583.44|3583.44|-202.24| +2450844|41306|2450915|41734|979631|2328|44625|41734|979631|2328|44625|4|62|8|3|17281|150|1527|91|78.41|161.52|46.84|10435.88|4262.44|7135.31|14698.32|340.99|0.00|5438.16|4262.44|4603.43|9700.60|10041.59|-2872.87| +2450844|41306|2450860|41734|979631|2328|44625|41734|979631|2328|44625|4|27|15|1|5956|206|1527|71|28.11|36.54|34.71|129.93|2464.41|1995.81|2594.34|96.60|49.28|337.25|2415.13|2511.73|2752.38|2848.98|419.32| +2450844|41306|2450864|41734|979631|2328|44625|41734|979631|2328|44625|4|37|4|2|16480|230|1527|8|43.40|103.72|63.26|323.68|506.08|347.20|829.76|5.06|0.00|306.96|506.08|511.14|813.04|818.10|158.88| +2450844|41306|2450927|41734|979631|2328|44625|41734|979631|2328|44625|4|93|7|4|8294|85|1527|79|47.60|62.35|28.68|2659.93|2265.72|3760.40|4925.65|45.31|0.00|2413.45|2265.72|2311.03|4679.17|4724.48|-1494.68| +2450844|41306|2450870|41734|979631|2328|44625|41734|979631|2328|44625|4|5|18|5|2978|251|1527|81|51.90|154.66|35.57|9646.29|2881.17|4203.90|12527.46|172.87|0.00|250.29|2881.17|3054.04|3131.46|3304.33|-1322.73| +2450844|41306|2450856|41734|979631|2328|44625|41734|979631|2328|44625|4|51|3|1|7045|23|1527|53|3.78|9.52|7.23|121.37|383.19|200.34|504.56|34.48|0.00|186.56|383.19|417.67|569.75|604.23|182.85| +2450844|41306|2450925|41734|979631|2328|44625|41734|979631|2328|44625|4|47|2|5|7063|23|1527|54|92.23|262.85|252.33|568.08|13625.82|4980.42|14193.90|953.80|0.00|1135.08|13625.82|14579.62|14760.90|15714.70|8645.40| +2450844|41306|2450864|41734|979631|2328|44625|41734|979631|2328|44625|4|54|11|5|3478|231|1527|18|69.41|106.19|74.33|573.48|1337.94|1249.38|1911.42|25.95|40.13|76.32|1297.81|1323.76|1374.13|1400.08|48.43| +2450844|41306|2450932|41734|979631|2328|44625|41734|979631|2328|44625|4|77|12|3|2612|292|1527|38|43.26|84.78|34.75|1901.14|1320.50|1643.88|3221.64|26.41|0.00|128.82|1320.50|1346.91|1449.32|1475.73|-323.38| +2450844|41306|2450861|41734|979631|2328|44625|41734|979631|2328|44625|4|4|1|5|8302|68|1527|35|40.62|42.65|2.55|1403.50|89.25|1421.70|1492.75|3.07|27.66|14.70|61.59|64.66|76.29|79.36|-1360.11| +2450844|43502|2450882|55148|1681589|4830|22871|55148|1681589|4830|22871|4|65|7|1|15194|289|1528|50|49.96|92.92|30.66|3113.00|1533.00|2498.00|4646.00|61.32|0.00|511.00|1533.00|1594.32|2044.00|2105.32|-965.00| +2450844|43502|2450915|55148|1681589|4830|22871|55148|1681589|4830|22871|4|38|13|4|16759|287|1528|1|5.99|13.47|7.67|5.80|7.67|5.99|13.47|0.53|0.00|2.82|7.67|8.20|10.49|11.02|1.68| +2450844|43502|2450860|55148|1681589|4830|22871|55148|1681589|4830|22871|4|32|20|5|15244|200|1528|91|52.89|153.38|124.23|2652.65|11304.93|4812.99|13957.58|339.14|0.00|4326.14|11304.93|11644.07|15631.07|15970.21|6491.94| +2450844|43502|2450912|55148|1681589|4830|22871|55148|1681589|4830|22871|4|12|9|2|13675|180|1528|71|44.20|61.88|37.12|1757.96|2635.52|3138.20|4393.48|105.42|0.00|351.45|2635.52|2740.94|2986.97|3092.39|-502.68| +2450844|43502|2450870|55148|1681589|4830|22871|55148|1681589|4830|22871|4|39|17|2|6445|279|1528|36|4.97|5.26|2.99|81.72|107.64|178.92|189.36|4.30|0.00|18.72|107.64|111.94|126.36|130.66|-71.28| +2450844|43502|2450891|55148|1681589|4830|22871|55148|1681589|4830|22871|4|48|12|1|5809|22|1528|14|31.69|48.16|2.88|633.92|40.32|443.66|674.24|0.80|0.00|175.28|40.32|41.12|215.60|216.40|-403.34| +2450844|77260|2450846|34177|919860|3143|6826|6267|454592|6322|4005|4|51|20|5|1126|67|1529|43|63.62|171.77|49.81|5244.28|2141.83|2735.66|7386.11|21.41|0.00|590.82|2141.83|2163.24|2732.65|2754.06|-593.83| +2450844|77260|2450894|34177|919860|3143|6826|6267|454592|6322|4005|4|15|18|2|13202|252|1529|74|13.07|37.77|30.59|531.32|2263.66|967.18|2794.98|181.09|0.00|1257.26|2263.66|2444.75|3520.92|3702.01|1296.48| +2450844|77260|2450920|34177|919860|3143|6826|6267|454592|6322|4005|4|81|17|5|13174|71|1529|38|65.48|172.86|100.25|2759.18|3809.50|2488.24|6568.68|190.47|0.00|656.64|3809.50|3999.97|4466.14|4656.61|1321.26| +2450844|77260|2450912|34177|919860|3143|6826|6267|454592|6322|4005|4|39|7|1|8932|194|1529|5|1.49|2.08|0.58|7.50|2.90|7.45|10.40|0.05|0.00|3.40|2.90|2.95|6.30|6.35|-4.55| +2450844|77260|2450932|34177|919860|3143|6826|6267|454592|6322|4005|4|103|1|2|3452|209|1529|30|82.98|188.36|169.52|565.20|5085.60|2489.40|5650.80|101.71|0.00|2203.80|5085.60|5187.31|7289.40|7391.11|2596.20| +2450844|15803|2450928|12805|176709|966|13670|12805|176709|966|13670|4|102|20|3|11686|139|1530|70|84.97|229.41|100.94|8992.90|7065.80|5947.90|16058.70|211.97|0.00|2729.30|7065.80|7277.77|9795.10|10007.07|1117.90| +2450844|15803|2450908|12805|176709|966|13670|12805|176709|966|13670|4|76|18|4|9382|34|1530|56|42.62|43.47|1.73|2337.44|96.88|2386.72|2434.32|7.75|0.00|1095.36|96.88|104.63|1192.24|1199.99|-2289.84| +2450844|15803|2450856|12805|176709|966|13670|12805|176709|966|13670|4|25|20|5|10615|135|1530|31|77.15|126.52|12.65|3529.97|392.15|2391.65|3922.12|11.76|0.00|1843.26|392.15|403.91|2235.41|2247.17|-1999.50| +2450844|15803|2450866|12805|176709|966|13670|12805|176709|966|13670|4|63|15|1|10267|288|1530|52|51.72|126.71|102.63|1252.16|5336.76|2689.44|6588.92|266.83|0.00|2832.96|5336.76|5603.59|8169.72|8436.55|2647.32| +2450844|15803|2450909|12805|176709|966|13670|12805|176709|966|13670|4|14|8|5|7850|89|1530|60|15.69|25.73|21.87|231.60|1312.20|941.40|1543.80|65.61|0.00|524.40|1312.20|1377.81|1836.60|1902.21|370.80| +2450844|45687|2450882|16661|798618|5375|3332|16661|798618|5375|3332|1|38|8|1|11348|154|1531|47|9.70|21.82|19.41|113.27|912.27|455.90|1025.54|0.00|0.00|492.09|912.27|912.27|1404.36|1404.36|456.37| +2450844|45687|2450877|16661|798618|5375|3332|16661|798618|5375|3332|1|27|5|2|4892|157|1531|78|26.93|30.96|23.52|580.32|1834.56|2100.54|2414.88|36.69|0.00|651.30|1834.56|1871.25|2485.86|2522.55|-265.98| +2450844|45687|2450862|16661|798618|5375|3332|16661|798618|5375|3332|1|1|18|3|14062|235|1531|18|63.25|130.92|112.59|329.94|2026.62|1138.50|2356.56|20.26|0.00|376.92|2026.62|2046.88|2403.54|2423.80|888.12| +2450844|45687|2450893|16661|798618|5375|3332|16661|798618|5375|3332|1|35|10|3|7424|162|1531|7|17.12|49.47|18.30|218.19|128.10|119.84|346.29|3.84|0.00|90.02|128.10|131.94|218.12|221.96|8.26| +2450844|45687|2450873|16661|798618|5375|3332|16661|798618|5375|3332|1|106|16|4|7351|195|1531|5|25.62|59.69|5.96|268.65|29.80|128.10|298.45|0.00|0.00|131.30|29.80|29.80|161.10|161.10|-98.30| +2450844|45687|2450895|16661|798618|5375|3332|16661|798618|5375|3332|1|25|13|3|12392|229|1531|5|25.10|66.51|65.84|3.35|329.20|125.50|332.55|15.63|16.46|9.95|312.74|328.37|322.69|338.32|187.24| +2450844|45687|2450921|16661|798618|5375|3332|16661|798618|5375|3332|1|19|9|2|8954|270|1531|42|11.54|31.15|29.59|65.52|1242.78|484.68|1308.30|14.91|745.66|222.18|497.12|512.03|719.30|734.21|12.44| +2450844|45687|2450873|16661|798618|5375|3332|16661|798618|5375|3332|1|107|14|3|15457|141|1531|80|90.44|197.15|128.14|5520.80|10251.20|7235.20|15772.00|717.58|0.00|7885.60|10251.20|10968.78|18136.80|18854.38|3016.00| +2450844|45687|2450918|16661|798618|5375|3332|16661|798618|5375|3332|1|40|1|5|1622|121|1531|34|37.72|79.96|64.76|516.80|2201.84|1282.48|2718.64|19.37|1717.43|1304.92|484.41|503.78|1789.33|1808.70|-798.07| +2450844|45687|2450861|16661|798618|5375|3332|16661|798618|5375|3332|1|52|9|1|7939|115|1531|55|65.64|140.46|39.32|5562.70|2162.60|3610.20|7725.30|149.86|497.39|926.75|1665.21|1815.07|2591.96|2741.82|-1944.99| +2450845|72275|2450933|39813|589785|4910|482|39813|589785|4910|482|4|19|7|2|1321|202|1532|35|78.00|190.32|36.16|5395.60|1265.60|2730.00|6661.20|101.24|0.00|1198.75|1265.60|1366.84|2464.35|2565.59|-1464.40| +2450845|72275|2450874|39813|589785|4910|482|39813|589785|4910|482|4|83|9|1|11872|253|1532|93|3.28|9.64|8.00|152.52|744.00|305.04|896.52|7.44|0.00|421.29|744.00|751.44|1165.29|1172.73|438.96| +2450845|72275|2450877|39813|589785|4910|482|39813|589785|4910|482|4|38|1|3|16322|227|1532|40|46.50|100.44|22.09|3134.00|883.60|1860.00|4017.60|0.00|0.00|160.40|883.60|883.60|1044.00|1044.00|-976.40| +2450845|72275|2450902|39813|589785|4910|482|39813|589785|4910|482|4|25|5|4|4099|75|1532|57|69.87|193.53|160.62|1875.87|9155.34|3982.59|11031.21|732.42|0.00|3529.44|9155.34|9887.76|12684.78|13417.20|5172.75| +2450845|72275|2450876|39813|589785|4910|482|39813|589785|4910|482|4|60|19|1|15832|243|1532|74|65.58|195.42|91.84|7664.92|6796.16|4852.92|14461.08|0.00|0.00|722.98|6796.16|6796.16|7519.14|7519.14|1943.24| +2450845|67279|2450910|58402|96002|2|7202|58402|96002|2|7202|2|69|10|1|11704|186|1533|84|70.98|181.70|168.98|1068.48|14194.32|5962.32|15262.80|425.82|0.00|4425.96|14194.32|14620.14|18620.28|19046.10|8232.00| +2450845|67279|2450919|58402|96002|2|7202|58402|96002|2|7202|2|17|7|4|7573|295|1533|52|63.23|132.78|30.53|5317.00|1587.56|3287.96|6904.56|1.11|1476.43|897.52|111.13|112.24|1008.65|1009.76|-3176.83| +2450845|67279|2450912|58402|96002|2|7202|58402|96002|2|7202|2|74|11|5|6782|83|1533|8|31.30|36.62|27.46|73.28|219.68|250.40|292.96|0.00|76.88|2.88|142.80|142.80|145.68|145.68|-107.60| +2450845|67279|2450910|58402|96002|2|7202|58402|96002|2|7202|2|77|6|1|16537|206|1533|29|67.26|188.32|9.41|5188.39|272.89|1950.54|5461.28|8.18|0.00|873.77|272.89|281.07|1146.66|1154.84|-1677.65| +2450845|67279|2450875|58402|96002|2|7202|58402|96002|2|7202|2|18|11|3|8182|152|1533|79|78.16|175.07|169.81|415.54|13414.99|6174.64|13830.53|1207.34|0.00|5116.83|13414.99|14622.33|18531.82|19739.16|7240.35| +2450845|67279|2450889|58402|96002|2|7202|58402|96002|2|7202|2|2|10|4|16789|140|1533|44|40.95|50.36|14.60|1573.44|642.40|1801.80|2215.84|24.15|38.54|376.64|603.86|628.01|980.50|1004.65|-1197.94| +2450845|67279|2450923|58402|96002|2|7202|58402|96002|2|7202|2|67|20|1|2881|45|1533|58|21.08|47.21|34.93|712.24|2025.94|1222.64|2738.18|78.60|60.77|218.66|1965.17|2043.77|2183.83|2262.43|742.53| +2450845|67279|2450898|58402|96002|2|7202|58402|96002|2|7202|2|49|6|3|9352|177|1533|34|15.65|23.78|15.69|275.06|533.46|532.10|808.52|32.00|0.00|258.40|533.46|565.46|791.86|823.86|1.36| +2450845|67279|2450856|58402|96002|2|7202|58402|96002|2|7202|2|61|1|2|5078|291|1533|62|28.44|53.18|21.80|1945.56|1351.60|1763.28|3297.16|97.31|270.32|724.78|1081.28|1178.59|1806.06|1903.37|-682.00| +2450845|67279|2450920|58402|96002|2|7202|58402|96002|2|7202|2|46|10|2|17422|114|1533|7|87.55|179.47|69.99|766.36|489.93|612.85|1256.29|0.00|0.00|364.28|489.93|489.93|854.21|854.21|-122.92| +2450845|32268|2450905|2245|1838273|701|33129|2245|1838273|701|33129|2|28|17|1|13363|111|1534|35|23.13|55.74|31.21|858.55|1092.35|809.55|1950.90|87.38|0.00|546.00|1092.35|1179.73|1638.35|1725.73|282.80| +2450845|32268|2450880|2245|1838273|701|33129|2245|1838273|701|33129|2|24|2|3|5780|159|1534|83|1.50|2.29|1.67|51.46|138.61|124.50|190.07|12.47|0.00|92.96|138.61|151.08|231.57|244.04|14.11| +2450845|32268|2450911|2245|1838273|701|33129|2245|1838273|701|33129|2|107|7|1|10670|219|1534|59|21.95|31.38|5.64|1518.66|332.76|1295.05|1851.42|4.65|216.29|610.65|116.47|121.12|727.12|731.77|-1178.58| +2450845|32268|2450851|2245|1838273|701|33129|2245|1838273|701|33129|2|96|18|2|8719|242|1534|45|90.49|113.11|81.43|1425.60|3664.35|4072.05|5089.95|146.57|0.00|508.95|3664.35|3810.92|4173.30|4319.87|-407.70| +2450845|32268|2450884|2245|1838273|701|33129|2245|1838273|701|33129|2|70|8|4|11287|102|1534|86|61.43|90.91|46.36|3831.30|3986.96|5282.98|7818.26|159.47|0.00|3517.40|3986.96|4146.43|7504.36|7663.83|-1296.02| +2450845|32268|2450864|2245|1838273|701|33129|2245|1838273|701|33129|2|75|19|3|2554|169|1534|69|73.95|117.58|104.64|892.86|7220.16|5102.55|8113.02|288.80|0.00|2595.78|7220.16|7508.96|9815.94|10104.74|2117.61| +2450845|32268|2450860|2245|1838273|701|33129|2245|1838273|701|33129|2|72|20|2|5158|256|1534|54|37.11|70.13|60.31|530.28|3256.74|2003.94|3787.02|162.83|0.00|113.40|3256.74|3419.57|3370.14|3532.97|1252.80| +2450845|32268|2450855|2245|1838273|701|33129|2245|1838273|701|33129|2|9|11|3|3002|221|1534|99|81.75|232.17|58.04|17238.87|5745.96|8093.25|22984.83|459.67|0.00|10342.53|5745.96|6205.63|16088.49|16548.16|-2347.29| +2450845|32268|2450852|2245|1838273|701|33129|2245|1838273|701|33129|2|67|19|5|15488|108|1534|14|51.23|148.05|54.77|1305.92|766.78|717.22|2072.70|23.00|0.00|310.80|766.78|789.78|1077.58|1100.58|49.56| +2450845|32268|2450849|2245|1838273|701|33129|2245|1838273|701|33129|2|59|19|3|2311|143|1534|81|30.66|47.21|3.77|3518.64|305.37|2483.46|3824.01|12.21|0.00|1835.46|305.37|317.58|2140.83|2153.04|-2178.09| +2450845|32268|2450900|2245|1838273|701|33129|2245|1838273|701|33129|2|53|8|1|4664|72|1534|52|73.45|141.75|51.03|4717.44|2653.56|3819.40|7371.00|132.67|0.00|2432.04|2653.56|2786.23|5085.60|5218.27|-1165.84| +2450845|32268|2450910|2245|1838273|701|33129|2245|1838273|701|33129|2|30|4|1|565|69|1534|65|77.11|80.96|18.62|4052.10|1210.30|5012.15|5262.40|0.00|0.00|2578.55|1210.30|1210.30|3788.85|3788.85|-3801.85| +2450845|32268|2450882|2245|1838273|701|33129|2245|1838273|701|33129|2|82|2|5|7369|80|1534|42|97.60|271.32|35.27|9914.10|1481.34|4099.20|11395.44|37.33|859.17|911.40|622.17|659.50|1533.57|1570.90|-3477.03| +2450845|32268|2450879|2245|1838273|701|33129|2245|1838273|701|33129|2|87|18|5|3169|16|1534|13|92.14|171.38|85.69|1113.97|1113.97|1197.82|2227.94|0.00|0.00|0.00|1113.97|1113.97|1113.97|1113.97|-83.85| +2450845|2416|2450875|93999|1527551|3503|28702|93999|1527551|3503|28702|1|5|7|1|5726|212|1535|71|13.39|25.84|23.25|183.89|1650.75|950.69|1834.64|0.00|0.00|54.67|1650.75|1650.75|1705.42|1705.42|700.06| +2450845|2416|2450872|93999|1527551|3503|28702|93999|1527551|3503|28702|1|66|19|3|14216|68|1535|89|47.36|113.19|10.18|9167.89|906.02|4215.04|10073.91|72.48|0.00|3827.89|906.02|978.50|4733.91|4806.39|-3309.02| +2450845|2416|2450891|93999|1527551|3503|28702|93999|1527551|3503|28702|1|83|2|5|9016|59|1535|85|52.16|110.05|110.05|0.00|9354.25|4433.60|9354.25|0.00|0.00|2338.35|9354.25|9354.25|11692.60|11692.60|4920.65| +2450845|2416|2450885|93999|1527551|3503|28702|93999|1527551|3503|28702|1|17|3|1|15068|207|1535|21|16.07|41.29|4.95|763.14|103.95|337.47|867.09|5.19|0.00|156.03|103.95|109.14|259.98|265.17|-233.52| +2450845|2416|2450879|93999|1527551|3503|28702|93999|1527551|3503|28702|1|39|20|1|13345|160|1535|32|13.72|22.50|5.40|547.20|172.80|439.04|720.00|3.45|0.00|122.24|172.80|176.25|295.04|298.49|-266.24| +2450845|2416|2450898|93999|1527551|3503|28702|93999|1527551|3503|28702|1|21|3|2|15247|20|1535|51|75.01|219.02|94.17|6367.35|4802.67|3825.51|11170.02|288.16|0.00|4579.29|4802.67|5090.83|9381.96|9670.12|977.16| +2450845|2416|2450921|93999|1527551|3503|28702|93999|1527551|3503|28702|1|6|7|2|11215|169|1535|2|13.92|32.71|28.13|9.16|56.26|27.84|65.42|4.50|0.00|3.26|56.26|60.76|59.52|64.02|28.42| +2450845|2416|2450907|93999|1527551|3503|28702|93999|1527551|3503|28702|1|32|4|5|8350|162|1535|58|93.17|95.96|92.12|222.72|5342.96|5403.86|5565.68|267.14|0.00|2671.48|5342.96|5610.10|8014.44|8281.58|-60.90| +2450845|2416|2450885|93999|1527551|3503|28702|93999|1527551|3503|28702|1|57|14|4|11413|247|1535|70|79.45|91.36|78.56|896.00|5499.20|5561.50|6395.20|43.99|3299.52|3069.50|2199.68|2243.67|5269.18|5313.17|-3361.82| +2450845|67535|2450882|53317|674655|4440|20978|53317|674655|4440|20978|2|88|6|2|7886|22|1536|28|58.39|142.47|86.90|1555.96|2433.20|1634.92|3989.16|28.95|2019.55|0.00|413.65|442.60|413.65|442.60|-1221.27| +2450845|67535|2450863|53317|674655|4440|20978|53317|674655|4440|20978|2|26|5|5|16984|37|1536|58|42.34|66.89|16.05|2948.72|930.90|2455.72|3879.62|55.85|0.00|1861.80|930.90|986.75|2792.70|2848.55|-1524.82| +2450845|67535|2450883|53317|674655|4440|20978|53317|674655|4440|20978|2|25|17|2|4267|6|1536|81|26.16|74.29|0.74|5957.55|59.94|2118.96|6017.49|0.00|0.00|179.82|59.94|59.94|239.76|239.76|-2059.02| +2450845|67535|2450880|53317|674655|4440|20978|53317|674655|4440|20978|2|81|11|1|15266|81|1536|48|15.23|35.94|28.75|345.12|1380.00|731.04|1725.12|0.00|0.00|16.80|1380.00|1380.00|1396.80|1396.80|648.96| +2450845|67535|2450866|53317|674655|4440|20978|53317|674655|4440|20978|2|80|2|3|968|220|1536|39|3.21|6.42|0.57|228.15|22.23|125.19|250.38|0.66|0.00|87.36|22.23|22.89|109.59|110.25|-102.96| +2450845|67535|2450922|53317|674655|4440|20978|53317|674655|4440|20978|2|32|15|4|10012|140|1536|21|12.20|12.20|10.00|46.20|210.00|256.20|256.20|12.60|0.00|71.61|210.00|222.60|281.61|294.21|-46.20| +2450845|67535|2450908|53317|674655|4440|20978|53317|674655|4440|20978|2|65|15|4|3847|235|1536|100|65.30|125.37|25.07|10030.00|2507.00|6530.00|12537.00|0.00|2055.74|1504.00|451.26|451.26|1955.26|1955.26|-6078.74| +2450845|67535|2450852|53317|674655|4440|20978|53317|674655|4440|20978|2|63|9|1|9512|141|1536|28|39.23|95.32|68.63|747.32|1921.64|1098.44|2668.96|134.51|0.00|293.44|1921.64|2056.15|2215.08|2349.59|823.20| +2450845|67535|2450910|53317|674655|4440|20978|53317|674655|4440|20978|2|104|3|1|4342|254|1536|98|47.82|104.24|23.97|7866.46|2349.06|4686.36|10215.52|140.94|0.00|1531.74|2349.06|2490.00|3880.80|4021.74|-2337.30| +2450845|67535|2450884|53317|674655|4440|20978|53317|674655|4440|20978|2|37|9|2|10250|218|1536|87|48.56|58.75|54.05|408.90|4702.35|4224.72|5111.25|28.21|3996.99|459.36|705.36|733.57|1164.72|1192.93|-3519.36| +2450845|67535|2450894|53317|674655|4440|20978|53317|674655|4440|20978|2|34|4|3|7558|116|1536|62|34.75|49.69|41.24|523.90|2556.88|2154.50|3080.78|178.98|0.00|985.80|2556.88|2735.86|3542.68|3721.66|402.38| +2450845|67535|2450869|53317|674655|4440|20978|53317|674655|4440|20978|2|29|6|5|1342|65|1536|6|47.31|49.20|31.98|103.32|191.88|283.86|295.20|3.83|0.00|35.40|191.88|195.71|227.28|231.11|-91.98| +2450845|67535|2450930|53317|674655|4440|20978|53317|674655|4440|20978|2|6|8|5|12952|119|1536|15|8.07|22.75|20.02|40.95|300.30|121.05|341.25|15.01|0.00|6.75|300.30|315.31|307.05|322.06|179.25| +2450845|67535|2450930|53317|674655|4440|20978|53317|674655|4440|20978|2|84|1|5|4418|170|1536|88|4.37|7.86|3.30|401.28|290.40|384.56|691.68|3.94|191.66|269.28|98.74|102.68|368.02|371.96|-285.82| +2450845|70902|2450873|50261|912374|5334|3847|50261|912374|5334|3847|4|64|15|1|16138|37|1537|78|57.58|137.04|64.40|5665.92|5023.20|4491.24|10689.12|251.16|0.00|3740.88|5023.20|5274.36|8764.08|9015.24|531.96| +2450845|70902|2450920|50261|912374|5334|3847|50261|912374|5334|3847|4|25|12|4|1936|260|1537|76|29.92|62.23|26.13|2743.60|1985.88|2273.92|4729.48|0.00|0.00|47.12|1985.88|1985.88|2033.00|2033.00|-288.04| +2450845|70902|2450920|50261|912374|5334|3847|50261|912374|5334|3847|4|67|8|3|1816|261|1537|26|73.48|124.18|38.49|2227.94|1000.74|1910.48|3228.68|10.00|0.00|1485.12|1000.74|1010.74|2485.86|2495.86|-909.74| +2450845|70902|2450935|50261|912374|5334|3847|50261|912374|5334|3847|4|95|19|3|4484|281|1537|18|60.01|167.42|3.34|2953.44|60.12|1080.18|3013.56|0.75|51.70|632.70|8.42|9.17|641.12|641.87|-1071.76| +2450845|70902|2450871|50261|912374|5334|3847|50261|912374|5334|3847|4|13|3|2|17329|74|1537|7|26.07|37.54|7.88|207.62|55.16|182.49|262.78|1.65|0.00|118.23|55.16|56.81|173.39|175.04|-127.33| +2450845|70902|2450876|50261|912374|5334|3847|50261|912374|5334|3847|4|59|9|2|13600|245|1537|46|14.39|30.65|4.59|1198.76|211.14|661.94|1409.90|12.66|0.00|13.80|211.14|223.80|224.94|237.60|-450.80| +2450845|70902|2450871|50261|912374|5334|3847|50261|912374|5334|3847|4|107|20|5|5384|265|1537|63|43.54|90.12|45.06|2838.78|2838.78|2743.02|5677.56|170.32|0.00|2441.25|2838.78|3009.10|5280.03|5450.35|95.76| +2450845|70902|2450920|50261|912374|5334|3847|50261|912374|5334|3847|4|24|1|1|16432|266|1537|94|78.05|177.17|19.48|14822.86|1831.12|7336.70|16653.98|109.86|0.00|2663.96|1831.12|1940.98|4495.08|4604.94|-5505.58| +2450845|70902|2450854|50261|912374|5334|3847|50261|912374|5334|3847|4|10|6|4|11560|190|1537|91|62.68|167.98|90.70|7032.48|8253.70|5703.88|15286.18|660.29|0.00|6267.17|8253.70|8913.99|14520.87|15181.16|2549.82| +2450845|70902|2450851|50261|912374|5334|3847|50261|912374|5334|3847|4|73|2|2|9874|164|1537|51|67.17|189.41|73.86|5893.05|3766.86|3425.67|9659.91|150.67|0.00|96.39|3766.86|3917.53|3863.25|4013.92|341.19| +2450845|70902|2450907|50261|912374|5334|3847|50261|912374|5334|3847|4|22|9|4|9700|86|1537|52|36.97|105.36|30.55|3890.12|1588.60|1922.44|5478.72|63.54|0.00|273.52|1588.60|1652.14|1862.12|1925.66|-333.84| +2450845|70902|2450913|50261|912374|5334|3847|50261|912374|5334|3847|4|37|19|2|16562|121|1537|91|32.20|67.62|67.62|0.00|6153.42|2930.20|6153.42|254.13|2522.90|122.85|3630.52|3884.65|3753.37|4007.50|700.32| +2450845|70902|2450922|50261|912374|5334|3847|50261|912374|5334|3847|4|42|15|5|15841|162|1537|55|91.39|226.64|160.91|3615.15|8850.05|5026.45|12465.20|796.50|0.00|2866.60|8850.05|9646.55|11716.65|12513.15|3823.60| +2450845|70902|2450899|50261|912374|5334|3847|50261|912374|5334|3847|4|5|4|2|7046|126|1537|100|81.83|102.28|0.00|10228.00|0.00|8183.00|10228.00|0.00|0.00|204.00|0.00|0.00|204.00|204.00|-8183.00| +2450845|56281|2450928|5823|1198428|7033|36438|5823|1198428|7033|36438|1|70|6|1|6220|124|1538|37|62.64|155.97|3.11|5655.82|115.07|2317.68|5770.89|6.90|0.00|1904.39|115.07|121.97|2019.46|2026.36|-2202.61| +2450845|56281|2450911|5823|1198428|7033|36438|5823|1198428|7033|36438|1|74|4|4|17596|97|1538|19|27.48|56.60|19.24|709.84|365.56|522.12|1075.40|6.90|288.79|376.39|76.77|83.67|453.16|460.06|-445.35| +2450845|56281|2450927|5823|1198428|7033|36438|5823|1198428|7033|36438|1|10|6|5|11384|148|1538|24|22.02|40.07|20.43|471.36|490.32|528.48|961.68|14.26|14.70|211.44|475.62|489.88|687.06|701.32|-52.86| +2450845|56281|2450851|5823|1198428|7033|36438|5823|1198428|7033|36438|1|1|9|1|17545|13|1538|44|11.75|32.90|11.18|955.68|491.92|517.00|1447.60|19.67|0.00|14.08|491.92|511.59|506.00|525.67|-25.08| +2450845|56281|2450887|5823|1198428|7033|36438|5823|1198428|7033|36438|1|83|20|1|11354|61|1538|52|92.64|138.96|15.28|6431.36|794.56|4817.28|7225.92|15.89|0.00|2962.44|794.56|810.45|3757.00|3772.89|-4022.72| +2450845|56281|2450883|5823|1198428|7033|36438|5823|1198428|7033|36438|1|61|15|5|1664|210|1538|26|83.80|130.72|24.83|2753.14|645.58|2178.80|3398.72|51.64|0.00|1631.24|645.58|697.22|2276.82|2328.46|-1533.22| +2450845|56281|2450848|5823|1198428|7033|36438|5823|1198428|7033|36438|1|4|3|3|13532|210|1538|62|69.26|136.44|20.46|7190.76|1268.52|4294.12|8459.28|55.81|570.83|592.10|697.69|753.50|1289.79|1345.60|-3596.43| +2450845|56281|2450861|5823|1198428|7033|36438|5823|1198428|7033|36438|1|105|7|2|14782|70|1538|4|9.01|26.48|4.23|89.00|16.92|36.04|105.92|1.52|0.00|42.36|16.92|18.44|59.28|60.80|-19.12| +2450845|56281|2450924|5823|1198428|7033|36438|5823|1198428|7033|36438|1|37|20|2|16477|149|1538|28|55.04|85.86|23.18|1755.04|649.04|1541.12|2404.08|25.96|0.00|600.88|649.04|675.00|1249.92|1275.88|-892.08| +2450845|56281|2450848|5823|1198428|7033|36438|5823|1198428|7033|36438|1|47|18|5|7298|115|1538|12|23.02|40.05|36.84|38.52|442.08|276.24|480.60|0.00|0.00|144.12|442.08|442.08|586.20|586.20|165.84| +2450845|56281|2450893|5823|1198428|7033|36438|5823|1198428|7033|36438|1|40|4|2|8083|103|1538|24|76.55|96.45|21.21|1805.76|509.04|1837.20|2314.80|32.06|50.90|624.96|458.14|490.20|1083.10|1115.16|-1379.06| +2450845|56281|2450871|5823|1198428|7033|36438|5823|1198428|7033|36438|1|59|7|1|2338|251|1538|86|65.50|187.98|172.94|1293.44|14872.84|5633.00|16166.28|0.00|4908.03|969.22|9964.81|9964.81|10934.03|10934.03|4331.81| +2450845|56281|2450883|5823|1198428|7033|36438|5823|1198428|7033|36438|1|71|20|2|13207|90|1538|97|23.72|61.19|42.22|1840.09|4095.34|2300.84|5935.43|0.00|0.00|2135.94|4095.34|4095.34|6231.28|6231.28|1794.50| +2450845|56281|2450865|5823|1198428|7033|36438|5823|1198428|7033|36438|1|7|20|1|8092|150|1538|37|87.15|149.89|140.89|333.00|5212.93|3224.55|5545.93|156.38|0.00|110.63|5212.93|5369.31|5323.56|5479.94|1988.38| +2450845|38009|2450861|35518|848342|4813|6037|35518|848342|4813|6037|2|76|15|2|3892|289|1539|80|14.20|16.33|13.22|248.80|1057.60|1136.00|1306.40|10.57|0.00|600.80|1057.60|1068.17|1658.40|1668.97|-78.40| +2450845|38009|2450863|35518|848342|4813|6037|35518|848342|4813|6037|2|97|4|4|121|184|1539|83|23.40|62.47|56.22|518.75|4666.26|1942.20|5185.01|319.17|1119.90|1036.67|3546.36|3865.53|4583.03|4902.20|1604.16| +2450845|38009|2450923|35518|848342|4813|6037|35518|848342|4813|6037|2|100|3|2|11422|13|1539|78|18.51|48.31|24.63|1847.04|1921.14|1443.78|3768.18|153.69|0.00|1167.66|1921.14|2074.83|3088.80|3242.49|477.36| +2450845|38009|2450918|35518|848342|4813|6037|35518|848342|4813|6037|2|98|11|4|16159|38|1539|88|37.88|102.27|29.65|6390.56|2609.20|3333.44|8999.76|52.18|0.00|2969.12|2609.20|2661.38|5578.32|5630.50|-724.24| +2450845|38009|2450927|35518|848342|4813|6037|35518|848342|4813|6037|2|40|4|5|11509|157|1539|78|27.34|76.82|4.60|5633.16|358.80|2132.52|5991.96|0.00|0.00|59.28|358.80|358.80|418.08|418.08|-1773.72| +2450845|38009|2450915|35518|848342|4813|6037|35518|848342|4813|6037|2|50|12|3|17599|80|1539|40|31.90|34.77|9.04|1029.20|361.60|1276.00|1390.80|0.00|0.00|292.00|361.60|361.60|653.60|653.60|-914.40| +2450845|38009|2450912|35518|848342|4813|6037|35518|848342|4813|6037|2|82|16|3|3112|192|1539|85|37.80|57.45|44.81|1074.40|3808.85|3213.00|4883.25|228.53|0.00|781.15|3808.85|4037.38|4590.00|4818.53|595.85| +2450845|38009|2450896|35518|848342|4813|6037|35518|848342|4813|6037|2|108|4|2|757|291|1539|66|71.53|138.76|22.20|7692.96|1465.20|4720.98|9158.16|0.00|0.00|732.60|1465.20|1465.20|2197.80|2197.80|-3255.78| +2450845|38009|2450864|35518|848342|4813|6037|35518|848342|4813|6037|2|99|16|1|11618|179|1539|12|12.63|17.05|9.71|88.08|116.52|151.56|204.60|5.82|0.00|75.60|116.52|122.34|192.12|197.94|-35.04| +2450845|38009|2450869|35518|848342|4813|6037|35518|848342|4813|6037|2|91|15|2|5998|267|1539|84|51.03|61.74|9.87|4357.08|829.08|4286.52|5186.16|16.58|0.00|2073.96|829.08|845.66|2903.04|2919.62|-3457.44| +2450845|38009|2450916|35518|848342|4813|6037|35518|848342|4813|6037|2|72|14|4|1807|206|1539|61|67.03|113.28|16.99|5873.69|1036.39|4088.83|6910.08|20.72|0.00|1589.05|1036.39|1057.11|2625.44|2646.16|-3052.44| +2450845|38009|2450857|35518|848342|4813|6037|35518|848342|4813|6037|2|95|7|2|583|166|1539|46|59.82|116.64|37.32|3648.72|1716.72|2751.72|5365.44|34.33|0.00|1287.54|1716.72|1751.05|3004.26|3038.59|-1035.00| +2450845|38009|2450858|35518|848342|4813|6037|35518|848342|4813|6037|2|83|17|3|15728|256|1539|39|90.18|214.62|193.15|837.33|7532.85|3517.02|8370.18|376.64|0.00|669.24|7532.85|7909.49|8202.09|8578.73|4015.83| +2450845|38009|2450919|35518|848342|4813|6037|35518|848342|4813|6037|2|44|18|4|14545|190|1539|71|31.00|31.62|23.08|606.34|1638.68|2201.00|2245.02|15.73|65.54|516.17|1573.14|1588.87|2089.31|2105.04|-627.86| +2450845|46284|2450921|73661|427979|3097|14262|73661|427979|3097|14262|2|20|8|3|2402|278|1540|70|96.61|159.40|17.53|9930.90|1227.10|6762.70|11158.00|85.89|0.00|4574.50|1227.10|1312.99|5801.60|5887.49|-5535.60| +2450845|46284|2450905|73661|427979|3097|14262|73661|427979|3097|14262|2|50|18|3|8474|146|1540|83|65.40|109.87|60.42|4104.35|5014.86|5428.20|9119.21|250.74|0.00|2370.48|5014.86|5265.60|7385.34|7636.08|-413.34| +2450845|46284|2450852|73661|427979|3097|14262|73661|427979|3097|14262|2|105|9|4|13514|289|1540|74|39.51|39.90|35.11|354.46|2598.14|2923.74|2952.60|0.00|0.00|973.84|2598.14|2598.14|3571.98|3571.98|-325.60| +2450845|46284|2450873|73661|427979|3097|14262|73661|427979|3097|14262|2|70|10|1|4736|296|1540|63|96.33|148.34|2.96|9158.94|186.48|6068.79|9345.42|16.78|0.00|2336.04|186.48|203.26|2522.52|2539.30|-5882.31| +2450845|46284|2450901|73661|427979|3097|14262|73661|427979|3097|14262|2|32|12|3|2560|163|1540|4|88.32|104.21|62.52|166.76|250.08|353.28|416.84|20.00|0.00|204.24|250.08|270.08|454.32|474.32|-103.20| +2450845|46284|2450864|73661|427979|3097|14262|73661|427979|3097|14262|2|104|14|2|17270|235|1540|6|11.02|11.13|0.55|63.48|3.30|66.12|66.78|0.26|0.00|32.70|3.30|3.56|36.00|36.26|-62.82| +2450845|46284|2450862|73661|427979|3097|14262|73661|427979|3097|14262|2|34|1|2|9656|141|1540|97|80.78|239.10|186.49|5103.17|18089.53|7835.66|23192.70|180.89|0.00|9044.28|18089.53|18270.42|27133.81|27314.70|10253.87| +2450845|46284|2450891|73661|427979|3097|14262|73661|427979|3097|14262|2|30|19|2|6298|170|1540|87|6.35|6.47|5.43|90.48|472.41|552.45|562.89|33.06|0.00|134.85|472.41|505.47|607.26|640.32|-80.04| +2450845|46284|2450886|73661|427979|3097|14262|73661|427979|3097|14262|2|61|7|5|9718|14|1540|50|32.99|88.41|71.61|840.00|3580.50|1649.50|4420.50|250.63|0.00|751.00|3580.50|3831.13|4331.50|4582.13|1931.00| +2450845|46284|2450886|73661|427979|3097|14262|73661|427979|3097|14262|2|14|12|5|11782|61|1540|84|55.07|100.77|8.06|7787.64|677.04|4625.88|8464.68|33.85|0.00|3893.40|677.04|710.89|4570.44|4604.29|-3948.84| +2450845|46284|2450863|73661|427979|3097|14262|73661|427979|3097|14262|2|41|5|5|14368|20|1540|2|43.27|54.95|20.33|69.24|40.66|86.54|109.90|2.84|0.00|37.36|40.66|43.50|78.02|80.86|-45.88| +2450845|46284|2450919|73661|427979|3097|14262|73661|427979|3097|14262|2|66|6|5|2062|153|1540|1|24.56|45.19|40.21|4.98|40.21|24.56|45.19|1.60|0.00|20.33|40.21|41.81|60.54|62.14|15.65| +2450845|46284|2450882|73661|427979|3097|14262|73661|427979|3097|14262|2|106|16|1|7532|300|1540|33|23.43|49.43|31.14|603.57|1027.62|773.19|1631.19|20.55|0.00|505.56|1027.62|1048.17|1533.18|1553.73|254.43| +2450845|46284|2450848|73661|427979|3097|14262|73661|427979|3097|14262|2|23|15|2|15206|8|1540|27|86.07|178.16|42.75|3656.07|1154.25|2323.89|4810.32|11.54|0.00|2260.71|1154.25|1165.79|3414.96|3426.50|-1169.64| +2450845|42884|2450900|20986|800999|3059|8672|20986|800999|3059|8672|1|46|13|4|7634|168|1541|64|87.29|182.43|118.57|4087.04|7588.48|5586.56|11675.52|379.42|0.00|5020.16|7588.48|7967.90|12608.64|12988.06|2001.92| +2450845|42884|2450914|20986|800999|3059|8672|20986|800999|3059|8672|1|83|3|3|3925|242|1541|100|50.48|119.13|40.50|7863.00|4050.00|5048.00|11913.00|243.00|0.00|4050.00|4050.00|4293.00|8100.00|8343.00|-998.00| +2450845|42884|2450911|20986|800999|3059|8672|20986|800999|3059|8672|1|48|17|3|4996|53|1541|57|32.43|58.04|42.36|893.76|2414.52|1848.51|3308.28|120.72|0.00|628.14|2414.52|2535.24|3042.66|3163.38|566.01| +2450845|42884|2450911|20986|800999|3059|8672|20986|800999|3059|8672|1|31|17|3|1210|288|1541|87|93.68|133.02|101.09|2777.91|8794.83|8150.16|11572.74|703.58|0.00|2661.33|8794.83|9498.41|11456.16|12159.74|644.67| +2450845|42884|2450935|20986|800999|3059|8672|20986|800999|3059|8672|1|101|14|1|16934|32|1541|78|19.85|49.82|32.38|1360.32|2525.64|1548.30|3885.96|73.49|75.76|659.88|2449.88|2523.37|3109.76|3183.25|901.58| +2450845|42884|2450893|20986|800999|3059|8672|20986|800999|3059|8672|1|101|17|5|2162|181|1541|14|16.22|42.82|27.83|209.86|389.62|227.08|599.48|23.37|0.00|155.82|389.62|412.99|545.44|568.81|162.54| +2450845|42884|2450926|20986|800999|3059|8672|20986|800999|3059|8672|1|2|9|3|6296|223|1541|33|50.26|104.03|47.85|1853.94|1579.05|1658.58|3432.99|31.58|0.00|1579.05|1579.05|1610.63|3158.10|3189.68|-79.53| +2450845|42884|2450893|20986|800999|3059|8672|20986|800999|3059|8672|1|39|8|5|7651|200|1541|47|28.56|41.98|28.96|611.94|1361.12|1342.32|1973.06|27.22|0.00|493.03|1361.12|1388.34|1854.15|1881.37|18.80| +2450845|42884|2450904|20986|800999|3059|8672|20986|800999|3059|8672|1|70|19|5|2575|297|1541|25|13.00|17.94|8.07|246.75|201.75|325.00|448.50|8.07|0.00|4.25|201.75|209.82|206.00|214.07|-123.25| +2450845|42884|2450851|20986|800999|3059|8672|20986|800999|3059|8672|1|50|20|4|13063|8|1541|6|38.95|89.58|39.41|301.02|236.46|233.70|537.48|7.09|0.00|209.58|236.46|243.55|446.04|453.13|2.76| +2450845|42884|2450935|20986|800999|3059|8672|20986|800999|3059|8672|1|24|8|1|3373|37|1541|61|23.34|26.37|24.78|96.99|1511.58|1423.74|1608.57|105.81|0.00|0.00|1511.58|1617.39|1511.58|1617.39|87.84| +2450845|42884|2450856|20986|800999|3059|8672|20986|800999|3059|8672|1|39|11|3|17762|86|1541|32|40.03|118.48|48.57|2237.12|1554.24|1280.96|3791.36|62.16|0.00|682.24|1554.24|1616.40|2236.48|2298.64|273.28| +2450845|42884|2450882|20986|800999|3059|8672|20986|800999|3059|8672|1|108|11|3|632|212|1541|50|33.46|58.22|24.45|1688.50|1222.50|1673.00|2911.00|110.02|0.00|291.00|1222.50|1332.52|1513.50|1623.52|-450.50| +2450845|42884|2450935|20986|800999|3059|8672|20986|800999|3059|8672|1|60|11|1|1831|137|1541|25|79.37|79.37|15.87|1587.50|396.75|1984.25|1984.25|15.87|0.00|277.75|396.75|412.62|674.50|690.37|-1587.50| +2450845|82180|2450847|78604|1200428|1899|11770|78604|1200428|1899|11770|2|14|1|4|15883|221|1542|30|5.23|15.32|5.51|294.30|165.30|156.90|459.60|8.26|0.00|32.10|165.30|173.56|197.40|205.66|8.40| +2450845|82180|2450909|78604|1200428|1899|11770|78604|1200428|1899|11770|2|11|16|2|9766|126|1542|66|70.35|209.64|148.84|4012.80|9823.44|4643.10|13836.24|27.50|8448.15|2628.78|1375.29|1402.79|4004.07|4031.57|-3267.81| +2450845|82180|2450860|78604|1200428|1899|11770|78604|1200428|1899|11770|2|55|2|3|2102|42|1542|100|20.92|28.45|0.28|2817.00|28.00|2092.00|2845.00|1.68|0.00|483.00|28.00|29.68|511.00|512.68|-2064.00| +2450845|82180|2450873|78604|1200428|1899|11770|78604|1200428|1899|11770|2|33|16|1|16352|231|1542|68|49.24|132.94|19.94|7684.00|1355.92|3348.32|9039.92|40.67|0.00|2530.96|1355.92|1396.59|3886.88|3927.55|-1992.40| +2450845|82180|2450902|78604|1200428|1899|11770|78604|1200428|1899|11770|2|17|13|3|16762|213|1542|86|64.71|123.59|119.88|319.06|10309.68|5565.06|10628.74|927.87|0.00|743.90|10309.68|11237.55|11053.58|11981.45|4744.62| +2450845|82180|2450902|78604|1200428|1899|11770|78604|1200428|1899|11770|2|27|2|2|16622|201|1542|63|74.69|80.66|61.30|1219.68|3861.90|4705.47|5081.58|193.09|0.00|914.13|3861.90|4054.99|4776.03|4969.12|-843.57| +2450845|82180|2450865|78604|1200428|1899|11770|78604|1200428|1899|11770|2|50|20|1|5782|237|1542|69|57.92|68.92|64.78|285.66|4469.82|3996.48|4755.48|312.88|0.00|1569.06|4469.82|4782.70|6038.88|6351.76|473.34| +2450845|79141|2450886|80402|560361|3520|3479|80402|560361|3520|3479|2|15|16|1|10366|198|1543|25|71.85|139.38|27.87|2787.75|696.75|1796.25|3484.50|0.00|62.70|1533.00|634.05|634.05|2167.05|2167.05|-1162.20| +2450845|79141|2450897|80402|560361|3520|3479|80402|560361|3520|3479|2|73|14|3|14203|296|1543|66|25.72|48.35|14.98|2202.42|988.68|1697.52|3191.10|39.54|0.00|988.68|988.68|1028.22|1977.36|2016.90|-708.84| +2450845|79141|2450847|80402|560361|3520|3479|80402|560361|3520|3479|2|102|17|1|13460|223|1543|36|87.16|203.08|123.87|2851.56|4459.32|3137.76|7310.88|267.55|0.00|1827.72|4459.32|4726.87|6287.04|6554.59|1321.56| +2450845|79141|2450931|80402|560361|3520|3479|80402|560361|3520|3479|2|18|17|1|4598|276|1543|17|49.35|85.37|54.63|522.58|928.71|838.95|1451.29|9.28|0.00|174.08|928.71|937.99|1102.79|1112.07|89.76| +2450845|79141|2450913|80402|560361|3520|3479|80402|560361|3520|3479|2|80|7|5|1729|124|1543|56|11.78|17.31|10.90|358.96|610.40|659.68|969.36|6.10|0.00|271.04|610.40|616.50|881.44|887.54|-49.28| +2450845|79141|2450866|80402|560361|3520|3479|80402|560361|3520|3479|2|82|8|2|1453|245|1543|41|86.23|194.01|190.12|159.49|7794.92|3535.43|7954.41|40.53|3741.56|238.62|4053.36|4093.89|4291.98|4332.51|517.93| +2450845|79141|2450883|80402|560361|3520|3479|80402|560361|3520|3479|2|8|6|5|4030|40|1543|1|94.17|192.10|44.18|147.92|44.18|94.17|192.10|1.76|0.00|80.68|44.18|45.94|124.86|126.62|-49.99| +2450845|47436|2450884|65339|1527872|2052|5432|65339|1527872|2052|5432|1|21|10|1|4804|30|1544|20|14.73|40.65|13.41|544.80|268.20|294.60|813.00|2.68|0.00|24.20|268.20|270.88|292.40|295.08|-26.40| +2450845|47436|2450847|65339|1527872|2052|5432|65339|1527872|2052|5432|1|3|17|3|13328|78|1544|36|92.57|225.87|103.90|4390.92|3740.40|3332.52|8131.32|336.63|0.00|4065.48|3740.40|4077.03|7805.88|8142.51|407.88| +2450845|47436|2450926|65339|1527872|2052|5432|65339|1527872|2052|5432|1|51|9|5|11654|176|1544|30|70.28|119.47|94.38|752.70|2831.40|2108.40|3584.10|35.67|2321.74|1505.10|509.66|545.33|2014.76|2050.43|-1598.74| +2450845|47436|2450878|65339|1527872|2052|5432|65339|1527872|2052|5432|1|33|17|4|14182|291|1544|55|11.64|25.84|25.84|0.00|1421.20|640.20|1421.20|127.90|0.00|13.75|1421.20|1549.10|1434.95|1562.85|781.00| +2450845|47436|2450922|65339|1527872|2052|5432|65339|1527872|2052|5432|1|63|12|4|17144|96|1544|1|39.96|117.48|23.49|93.99|23.49|39.96|117.48|0.46|0.00|36.41|23.49|23.95|59.90|60.36|-16.47| +2450845|67960|2450917|16550|411693|6389|17485|16550|411693|6389|17485|1|48|10|5|15103|282|1545|66|89.03|186.07|132.10|3562.02|8718.60|5875.98|12280.62|0.00|0.00|2578.62|8718.60|8718.60|11297.22|11297.22|2842.62| +2450845|67960|2450849|16550|411693|6389|17485|16550|411693|6389|17485|1|77|10|1|6811|276|1545|40|85.86|111.61|82.59|1160.80|3303.60|3434.40|4464.40|297.32|0.00|356.80|3303.60|3600.92|3660.40|3957.72|-130.80| +2450845|67960|2450932|16550|411693|6389|17485|16550|411693|6389|17485|1|78|15|2|290|48|1545|1|35.77|98.72|12.83|85.89|12.83|35.77|98.72|0.00|0.00|37.51|12.83|12.83|50.34|50.34|-22.94| +2450845|67960|2450880|16550|411693|6389|17485|16550|411693|6389|17485|1|1|2|1|4141|132|1545|77|25.25|62.87|12.57|3873.10|967.89|1944.25|4840.99|48.39|0.00|677.60|967.89|1016.28|1645.49|1693.88|-976.36| +2450845|67960|2450849|16550|411693|6389|17485|16550|411693|6389|17485|1|58|13|3|9184|41|1545|82|44.50|76.09|14.45|5054.48|1184.90|3649.00|6239.38|29.85|758.33|1621.96|426.57|456.42|2048.53|2078.38|-3222.43| +2450845|67960|2450877|16550|411693|6389|17485|16550|411693|6389|17485|1|56|20|2|1633|128|1545|83|1.35|3.56|2.49|88.81|206.67|112.05|295.48|6.20|0.00|0.00|206.67|212.87|206.67|212.87|94.62| +2450845|67960|2450897|16550|411693|6389|17485|16550|411693|6389|17485|1|41|18|4|17995|36|1545|63|20.15|39.29|30.64|544.95|1930.32|1269.45|2475.27|115.81|0.00|49.14|1930.32|2046.13|1979.46|2095.27|660.87| +2450845|67960|2450861|16550|411693|6389|17485|16550|411693|6389|17485|1|70|10|4|1844|227|1545|99|13.07|35.02|20.66|1421.64|2045.34|1293.93|3466.98|0.00|1758.99|1593.90|286.35|286.35|1880.25|1880.25|-1007.58| +2450845|40148|2450924|20220|1171571|522|4626|20220|1171571|522|4626|1|83|12|3|12836|42|1546|31|43.70|98.76|29.62|2143.34|918.22|1354.70|3061.56|73.45|0.00|673.32|918.22|991.67|1591.54|1664.99|-436.48| +2450845|40148|2450849|20220|1171571|522|4626|20220|1171571|522|4626|1|13|20|2|6730|58|1546|37|55.90|57.01|5.13|1919.56|189.81|2068.30|2109.37|17.08|0.00|84.36|189.81|206.89|274.17|291.25|-1878.49| +2450845|40148|2450906|20220|1171571|522|4626|20220|1171571|522|4626|1|52|8|4|3442|27|1546|66|88.26|233.00|228.34|307.56|15070.44|5825.16|15378.00|452.11|0.00|7689.00|15070.44|15522.55|22759.44|23211.55|9245.28| +2450845|40148|2450897|20220|1171571|522|4626|20220|1171571|522|4626|1|36|6|2|8356|267|1546|6|55.47|154.76|30.95|742.86|185.70|332.82|928.56|11.14|0.00|157.80|185.70|196.84|343.50|354.64|-147.12| +2450845|40148|2450921|20220|1171571|522|4626|20220|1171571|522|4626|1|57|14|2|2707|101|1546|61|44.31|101.91|4.07|5968.24|248.27|2702.91|6216.51|4.96|0.00|3045.73|248.27|253.23|3294.00|3298.96|-2454.64| +2450845|40148|2450853|20220|1171571|522|4626|20220|1171571|522|4626|1|64|18|1|12733|96|1546|61|85.90|139.15|16.69|7470.06|1018.09|5239.90|8488.15|32.98|193.43|2970.70|824.66|857.64|3795.36|3828.34|-4415.24| +2450845|40148|2450858|20220|1171571|522|4626|20220|1171571|522|4626|1|99|16|4|16718|211|1546|16|70.93|187.25|176.01|179.84|2816.16|1134.88|2996.00|140.80|0.00|688.96|2816.16|2956.96|3505.12|3645.92|1681.28| +2450845|40148|2450890|20220|1171571|522|4626|20220|1171571|522|4626|1|30|2|4|5545|48|1546|46|79.98|119.97|67.18|2428.34|3090.28|3679.08|5518.62|0.00|0.00|606.74|3090.28|3090.28|3697.02|3697.02|-588.80| +2450845|40148|2450922|20220|1171571|522|4626|20220|1171571|522|4626|1|80|20|1|17815|103|1546|72|76.43|132.98|14.62|8521.92|1052.64|5502.96|9574.56|10.52|0.00|4500.00|1052.64|1063.16|5552.64|5563.16|-4450.32| +2450845|40148|2450911|20220|1171571|522|4626|20220|1171571|522|4626|1|13|18|3|10975|188|1546|43|26.95|57.40|8.61|2097.97|370.23|1158.85|2468.20|29.61|0.00|518.15|370.23|399.84|888.38|917.99|-788.62| +2450845|40148|2450860|20220|1171571|522|4626|20220|1171571|522|4626|1|34|8|2|15148|175|1546|73|88.70|94.02|76.15|1304.51|5558.95|6475.10|6863.46|0.00|0.00|3019.28|5558.95|5558.95|8578.23|8578.23|-916.15| +2450845|40148|2450918|20220|1171571|522|4626|20220|1171571|522|4626|1|4|18|4|4972|141|1546|10|54.89|55.98|41.42|145.60|414.20|548.90|559.80|12.42|0.00|223.90|414.20|426.62|638.10|650.52|-134.70| +2450845|40148|2450856|20220|1171571|522|4626|20220|1171571|522|4626|1|32|8|1|12367|87|1546|31|14.73|40.80|28.56|379.44|885.36|456.63|1264.80|26.56|0.00|606.98|885.36|911.92|1492.34|1518.90|428.73| +2450845|40148|2450848|20220|1171571|522|4626|20220|1171571|522|4626|1|93|3|4|7180|68|1546|26|14.44|24.40|7.56|437.84|196.56|375.44|634.40|7.86|0.00|317.20|196.56|204.42|513.76|521.62|-178.88| +2450845|50262|2450921|87208|1658065|922|32626|87208|1658065|922|32626|4|56|17|2|13114|214|1547|56|48.31|49.75|13.93|2005.92|780.08|2705.36|2786.00|31.20|0.00|1002.96|780.08|811.28|1783.04|1814.24|-1925.28| +2450845|50262|2450861|87208|1658065|922|32626|87208|1658065|922|32626|4|76|6|4|9466|148|1547|67|32.97|83.08|45.69|2505.13|3061.23|2208.99|5566.36|122.44|0.00|0.00|3061.23|3183.67|3061.23|3183.67|852.24| +2450845|50262|2450871|87208|1658065|922|32626|87208|1658065|922|32626|4|54|15|4|10591|14|1547|67|29.48|59.84|20.94|2606.30|1402.98|1975.16|4009.28|0.00|0.00|1683.71|1402.98|1402.98|3086.69|3086.69|-572.18| +2450845|50262|2450902|87208|1658065|922|32626|87208|1658065|922|32626|4|48|10|4|14794|77|1547|96|20.78|50.91|16.29|3323.52|1563.84|1994.88|4887.36|0.00|0.00|1416.96|1563.84|1563.84|2980.80|2980.80|-431.04| +2450845|50262|2450886|87208|1658065|922|32626|87208|1658065|922|32626|4|70|6|2|8374|257|1547|68|65.14|89.24|24.09|4430.20|1638.12|4429.52|6068.32|76.17|114.66|1820.36|1523.46|1599.63|3343.82|3419.99|-2906.06| +2450845|50262|2450903|87208|1658065|922|32626|87208|1658065|922|32626|4|80|9|3|13813|36|1547|74|59.88|105.98|84.78|1568.80|6273.72|4431.12|7842.52|439.16|0.00|1724.94|6273.72|6712.88|7998.66|8437.82|1842.60| +2450845|50262|2450912|87208|1658065|922|32626|87208|1658065|922|32626|4|99|5|4|2594|57|1547|67|15.03|38.47|6.15|2165.44|412.05|1007.01|2577.49|3.70|226.62|360.46|185.43|189.13|545.89|549.59|-821.58| +2450845|50262|2450914|87208|1658065|922|32626|87208|1658065|922|32626|4|49|18|4|58|10|1547|17|96.85|149.14|85.00|1090.38|1445.00|1646.45|2535.38|86.70|0.00|709.75|1445.00|1531.70|2154.75|2241.45|-201.45| +2450845|50262|2450930|87208|1658065|922|32626|87208|1658065|922|32626|4|108|19|2|15658|175|1547|4|41.62|53.27|21.30|127.88|85.20|166.48|213.08|7.66|0.00|12.76|85.20|92.86|97.96|105.62|-81.28| +2450845|50262|2450874|87208|1658065|922|32626|87208|1658065|922|32626|4|16|3|2|16982|33|1547|68|69.58|132.89|87.70|3072.92|5963.60|4731.44|9036.52|59.63|0.00|3253.12|5963.60|6023.23|9216.72|9276.35|1232.16| +2450845|50262|2450857|87208|1658065|922|32626|87208|1658065|922|32626|4|46|2|4|16654|45|1547|37|24.37|40.21|31.36|327.45|1160.32|901.69|1487.77|30.05|731.00|490.62|429.32|459.37|919.94|949.99|-472.37| +2450845|6794|2450881|35065|390459|4253|49647|35065|390459|4253|49647|4|7|17|5|7921|106|1548|31|66.64|126.61|20.25|3297.16|627.75|2065.84|3924.91|18.83|0.00|981.15|627.75|646.58|1608.90|1627.73|-1438.09| +2450845|6794|2450871|35065|390459|4253|49647|35065|390459|4253|49647|4|36|16|3|15511|124|1548|93|87.09|181.14|85.13|8928.93|7917.09|8099.37|16846.02|237.51|0.00|4042.71|7917.09|8154.60|11959.80|12197.31|-182.28| +2450845|6794|2450901|35065|390459|4253|49647|35065|390459|4253|49647|4|72|11|1|1921|233|1548|74|32.52|43.57|15.68|2063.86|1160.32|2406.48|3224.18|92.82|0.00|96.20|1160.32|1253.14|1256.52|1349.34|-1246.16| +2450845|6794|2450888|35065|390459|4253|49647|35065|390459|4253|49647|4|108|6|3|15962|124|1548|39|57.75|68.14|19.76|1886.82|770.64|2252.25|2657.46|23.11|0.00|584.61|770.64|793.75|1355.25|1378.36|-1481.61| +2450845|6794|2450885|35065|390459|4253|49647|35065|390459|4253|49647|4|70|4|1|10256|160|1548|19|42.80|71.47|15.00|1072.93|285.00|813.20|1357.93|22.80|0.00|407.36|285.00|307.80|692.36|715.16|-528.20| +2450845|6794|2450872|35065|390459|4253|49647|35065|390459|4253|49647|4|48|17|1|1717|256|1548|49|54.94|131.85|5.27|6202.42|258.23|2692.06|6460.65|7.74|0.00|2260.86|258.23|265.97|2519.09|2526.83|-2433.83| +2450845|6794|2450932|35065|390459|4253|49647|35065|390459|4253|49647|4|61|12|5|3050|298|1548|88|42.42|107.32|75.12|2833.60|6610.56|3732.96|9444.16|40.98|4561.28|1510.96|2049.28|2090.26|3560.24|3601.22|-1683.68| +2450845|6794|2450862|35065|390459|4253|49647|35065|390459|4253|49647|4|73|19|5|12325|54|1548|96|6.06|12.72|4.19|818.88|402.24|581.76|1221.12|4.02|0.00|414.72|402.24|406.26|816.96|820.98|-179.52| +2450845|6794|2450889|35065|390459|4253|49647|35065|390459|4253|49647|4|97|6|5|4033|23|1548|81|76.51|174.44|144.78|2402.46|11727.18|6197.31|14129.64|586.35|0.00|5085.99|11727.18|12313.53|16813.17|17399.52|5529.87| +2450845|25682|2450904|1009|1420574|1386|2951|1009|1420574|1386|2951|2|5|7|2|6601|137|1549|17|86.33|191.65|99.65|1564.00|1694.05|1467.61|3258.05|0.00|0.00|846.94|1694.05|1694.05|2540.99|2540.99|226.44| +2450845|25682|2450893|1009|1420574|1386|2951|1009|1420574|1386|2951|2|78|2|1|12794|226|1549|28|53.84|143.21|64.44|2205.56|1804.32|1507.52|4009.88|108.25|0.00|1042.44|1804.32|1912.57|2846.76|2955.01|296.80| +2450845|25682|2450921|1009|1420574|1386|2951|1009|1420574|1386|2951|2|28|16|4|16766|138|1549|98|33.97|50.95|36.68|1398.46|3594.64|3329.06|4993.10|0.00|0.00|998.62|3594.64|3594.64|4593.26|4593.26|265.58| +2450845|25682|2450864|1009|1420574|1386|2951|1009|1420574|1386|2951|2|103|12|4|16222|221|1549|38|72.18|117.65|54.11|2414.52|2056.18|2742.84|4470.70|164.49|0.00|1206.88|2056.18|2220.67|3263.06|3427.55|-686.66| +2450845|25682|2450910|1009|1420574|1386|2951|1009|1420574|1386|2951|2|67|12|5|499|83|1549|72|18.84|18.84|9.04|705.60|650.88|1356.48|1356.48|19.52|0.00|420.48|650.88|670.40|1071.36|1090.88|-705.60| +2450845|25682|2450879|1009|1420574|1386|2951|1009|1420574|1386|2951|2|46|11|2|9061|192|1549|33|35.64|78.05|47.61|1004.52|1571.13|1176.12|2575.65|0.00|0.00|746.79|1571.13|1571.13|2317.92|2317.92|395.01| +2450845|25682|2450933|1009|1420574|1386|2951|1009|1420574|1386|2951|2|20|14|3|10195|238|1549|16|40.18|102.45|20.49|1311.36|327.84|642.88|1639.20|22.94|0.00|147.52|327.84|350.78|475.36|498.30|-315.04| +2450845|25682|2450922|1009|1420574|1386|2951|1009|1420574|1386|2951|2|2|17|4|481|212|1549|96|53.54|69.60|16.00|5145.60|1536.00|5139.84|6681.60|138.24|0.00|3072.96|1536.00|1674.24|4608.96|4747.20|-3603.84| +2450845|25682|2450882|1009|1420574|1386|2951|1009|1420574|1386|2951|2|21|2|4|17720|140|1549|45|37.41|55.36|7.75|2142.45|348.75|1683.45|2491.20|6.97|0.00|224.10|348.75|355.72|572.85|579.82|-1334.70| +2450845|25682|2450925|1009|1420574|1386|2951|1009|1420574|1386|2951|2|102|7|1|1934|4|1549|9|37.81|40.45|12.53|251.28|112.77|340.29|364.05|1.12|0.00|156.51|112.77|113.89|269.28|270.40|-227.52| +2450845|25682|2450924|1009|1420574|1386|2951|1009|1420574|1386|2951|2|32|15|1|8518|20|1549|94|10.09|15.94|10.20|539.56|958.80|948.46|1498.36|67.11|0.00|329.00|958.80|1025.91|1287.80|1354.91|10.34| +2450845|60656|2450876|31123|1146690|5104|41071|31123|1146690|5104|41071|4|48|1|5|15422|48|1550|32|96.96|152.22|92.85|1899.84|2971.20|3102.72|4871.04|118.84|0.00|292.16|2971.20|3090.04|3263.36|3382.20|-131.52| +2450845|60656|2450928|31123|1146690|5104|41071|31123|1146690|5104|41071|4|7|1|4|3968|295|1550|70|73.48|177.08|42.49|9421.30|2974.30|5143.60|12395.60|237.94|0.00|371.70|2974.30|3212.24|3346.00|3583.94|-2169.30| +2450845|60656|2450918|31123|1146690|5104|41071|31123|1146690|5104|41071|4|74|3|2|6385|255|1550|34|25.65|25.65|16.41|314.16|557.94|872.10|872.10|5.57|0.00|209.10|557.94|563.51|767.04|772.61|-314.16| +2450845|60656|2450879|31123|1146690|5104|41071|31123|1146690|5104|41071|4|75|14|5|7640|232|1550|5|69.19|138.38|120.39|89.95|601.95|345.95|691.90|9.75|276.89|325.15|325.06|334.81|650.21|659.96|-20.89| +2450845|60194|2450862|34841|1358699|4957|46368|34841|1358699|4957|46368|4|71|13|4|8617|201|1551|40|3.05|8.99|1.61|295.20|64.40|122.00|359.60|0.00|0.00|68.00|64.40|64.40|132.40|132.40|-57.60| +2450845|60194|2450869|34841|1358699|4957|46368|34841|1358699|4957|46368|4|2|13|1|646|147|1551|74|46.52|86.06|73.15|955.34|5413.10|3442.48|6368.44|0.00|4330.48|954.60|1082.62|1082.62|2037.22|2037.22|-2359.86| +2450845|60194|2450934|34841|1358699|4957|46368|34841|1358699|4957|46368|4|15|5|1|2377|249|1551|97|53.95|107.90|88.47|1884.71|8581.59|5233.15|10466.30|0.00|0.00|627.59|8581.59|8581.59|9209.18|9209.18|3348.44| +2450845|60194|2450933|34841|1358699|4957|46368|34841|1358699|4957|46368|4|74|20|3|3823|91|1551|98|20.56|24.46|11.49|1271.06|1126.02|2014.88|2397.08|101.34|0.00|933.94|1126.02|1227.36|2059.96|2161.30|-888.86| +2450845|60194|2450890|34841|1358699|4957|46368|34841|1358699|4957|46368|4|43|12|2|11692|37|1551|43|73.59|82.42|1.64|3473.54|70.52|3164.37|3544.06|4.93|0.00|992.01|70.52|75.45|1062.53|1067.46|-3093.85| +2450845|39171|2450902|95318|1112622|5479|15983|95318|1112622|5479|15983|1|49|15|1|727|254|1552|41|60.63|154.00|15.40|5682.60|631.40|2485.83|6314.00|0.00|107.33|2462.46|524.07|524.07|2986.53|2986.53|-1961.76| +2450845|39171|2450921|95318|1112622|5479|15983|95318|1112622|5479|15983|1|29|5|2|14860|170|1552|64|97.76|229.73|119.45|7057.92|7644.80|6256.64|14702.72|152.89|0.00|4557.44|7644.80|7797.69|12202.24|12355.13|1388.16| +2450845|39171|2450925|95318|1112622|5479|15983|95318|1112622|5479|15983|1|98|16|4|10939|205|1552|96|19.69|55.72|8.35|4547.52|801.60|1890.24|5349.12|40.08|0.00|1283.52|801.60|841.68|2085.12|2125.20|-1088.64| +2450845|39171|2450912|95318|1112622|5479|15983|95318|1112622|5479|15983|1|14|13|2|5722|148|1552|81|74.70|163.59|122.69|3312.90|9937.89|6050.70|13250.79|596.27|0.00|6095.25|9937.89|10534.16|16033.14|16629.41|3887.19| +2450845|39171|2450896|95318|1112622|5479|15983|95318|1112622|5479|15983|1|33|2|4|12340|52|1552|41|6.34|16.61|9.30|299.71|381.30|259.94|681.01|22.87|0.00|217.71|381.30|404.17|599.01|621.88|121.36| +2450845|39171|2450899|95318|1112622|5479|15983|95318|1112622|5479|15983|1|7|9|4|17204|206|1552|53|14.23|33.01|24.42|455.27|1294.26|754.19|1749.53|25.88|0.00|472.23|1294.26|1320.14|1766.49|1792.37|540.07| +2450845|39171|2450908|95318|1112622|5479|15983|95318|1112622|5479|15983|1|1|6|5|10114|210|1552|32|73.23|87.14|53.15|1087.68|1700.80|2343.36|2788.48|0.34|1683.79|557.44|17.01|17.35|574.45|574.79|-2326.35| +2450845|39171|2450918|95318|1112622|5479|15983|95318|1112622|5479|15983|1|82|9|5|1910|145|1552|21|41.54|111.32|51.20|1262.52|1075.20|872.34|2337.72|86.01|0.00|1098.72|1075.20|1161.21|2173.92|2259.93|202.86| +2450845|39171|2450855|95318|1112622|5479|15983|95318|1112622|5479|15983|1|14|16|3|5414|66|1552|13|5.44|15.55|1.24|186.03|16.12|70.72|202.15|1.12|0.00|54.47|16.12|17.24|70.59|71.71|-54.60| +2450845|39171|2450903|95318|1112622|5479|15983|95318|1112622|5479|15983|1|107|15|1|15271|169|1552|48|56.87|141.60|28.32|5437.44|1359.36|2729.76|6796.80|13.59|0.00|203.52|1359.36|1372.95|1562.88|1576.47|-1370.40| +2450845|49627|2450919|40501|998301|901|23501|40501|998301|901|23501|4|9|12|3|3188|73|1553|23|85.90|246.53|54.23|4422.90|1247.29|1975.70|5670.19|99.78|0.00|2778.17|1247.29|1347.07|4025.46|4125.24|-728.41| +2450845|49627|2450867|40501|998301|901|23501|40501|998301|901|23501|4|43|3|2|6664|285|1553|52|45.03|63.49|19.04|2311.40|990.08|2341.56|3301.48|39.60|0.00|395.72|990.08|1029.68|1385.80|1425.40|-1351.48| +2450845|49627|2450851|40501|998301|901|23501|40501|998301|901|23501|4|45|8|4|17288|252|1553|57|25.99|46.78|7.48|2240.10|426.36|1481.43|2666.46|0.00|0.00|399.57|426.36|426.36|825.93|825.93|-1055.07| +2450845|49627|2450915|40501|998301|901|23501|40501|998301|901|23501|4|97|4|3|15812|64|1553|78|70.95|148.28|14.82|10409.88|1155.96|5534.10|11565.84|92.47|0.00|4509.96|1155.96|1248.43|5665.92|5758.39|-4378.14| +2450845|49627|2450863|40501|998301|901|23501|40501|998301|901|23501|4|75|4|2|284|108|1553|22|84.43|188.27|43.30|3189.34|952.60|1857.46|4141.94|9.52|0.00|165.66|952.60|962.12|1118.26|1127.78|-904.86| +2450845|49627|2450894|40501|998301|901|23501|40501|998301|901|23501|4|15|18|2|4838|16|1553|21|26.29|74.13|0.74|1541.19|15.54|552.09|1556.73|0.89|0.62|404.67|14.92|15.81|419.59|420.48|-537.17| +2450845|49627|2450858|40501|998301|901|23501|40501|998301|901|23501|4|65|20|1|5774|186|1553|41|91.35|139.76|39.13|4125.83|1604.33|3745.35|5730.16|2.56|1347.63|2291.90|256.70|259.26|2548.60|2551.16|-3488.65| +2450845|49627|2450926|40501|998301|901|23501|40501|998301|901|23501|4|29|8|5|17395|3|1553|100|56.98|149.28|128.38|2090.00|12838.00|5698.00|14928.00|249.05|385.14|5971.00|12452.86|12701.91|18423.86|18672.91|6754.86| +2450845|49627|2450857|40501|998301|901|23501|40501|998301|901|23501|4|105|4|2|9853|160|1553|85|81.98|95.09|81.77|1132.20|6950.45|6968.30|8082.65|625.54|0.00|1292.85|6950.45|7575.99|8243.30|8868.84|-17.85| +2450845|49627|2450874|40501|998301|901|23501|40501|998301|901|23501|4|15|3|1|4976|249|1553|30|91.96|153.57|87.53|1981.20|2625.90|2758.80|4607.10|52.51|0.00|875.10|2625.90|2678.41|3501.00|3553.51|-132.90| +2450845|49627|2450912|40501|998301|901|23501|40501|998301|901|23501|4|3|13|1|12904|174|1553|17|58.59|72.65|35.59|630.02|605.03|996.03|1235.05|36.30|0.00|12.24|605.03|641.33|617.27|653.57|-391.00| +2450845|49627|2450875|40501|998301|901|23501|40501|998301|901|23501|4|85|10|5|12706|6|1553|57|91.74|216.50|201.34|864.12|11476.38|5229.18|12340.50|0.00|0.00|3455.34|11476.38|11476.38|14931.72|14931.72|6247.20| +2450845|43496|2450934|35042|1346955|6868|34781|35042|1346955|6868|34781|4|93|6|2|2581|59|1554|71|46.43|119.32|35.79|5930.63|2541.09|3296.53|8471.72|0.00|0.00|2286.91|2541.09|2541.09|4828.00|4828.00|-755.44| +2450845|43496|2450916|35042|1346955|6868|34781|35042|1346955|6868|34781|4|59|7|4|3763|146|1554|42|54.26|54.26|43.40|456.12|1822.80|2278.92|2278.92|72.91|0.00|1116.36|1822.80|1895.71|2939.16|3012.07|-456.12| +2450845|43496|2450872|35042|1346955|6868|34781|35042|1346955|6868|34781|4|104|16|4|13640|257|1554|36|67.69|122.51|88.20|1235.16|3175.20|2436.84|4410.36|285.76|0.00|1146.60|3175.20|3460.96|4321.80|4607.56|738.36| +2450845|43496|2450913|35042|1346955|6868|34781|35042|1346955|6868|34781|4|71|20|3|14071|49|1554|84|7.26|14.88|5.20|813.12|436.80|609.84|1249.92|30.57|0.00|336.84|436.80|467.37|773.64|804.21|-173.04| +2450845|43496|2450904|35042|1346955|6868|34781|35042|1346955|6868|34781|4|79|14|1|973|25|1554|42|99.53|206.02|18.54|7874.16|778.68|4180.26|8652.84|31.14|0.00|3028.20|778.68|809.82|3806.88|3838.02|-3401.58| +2450845|43496|2450868|35042|1346955|6868|34781|35042|1346955|6868|34781|4|74|15|3|13382|149|1554|75|67.64|137.30|35.69|7620.75|2676.75|5073.00|10297.50|160.60|0.00|3192.00|2676.75|2837.35|5868.75|6029.35|-2396.25| +2450845|23959|2450861|87084|1304332|780|21228|87084|1304332|780|21228|4|16|13|2|11366|118|1555|2|12.95|29.00|9.57|38.86|19.14|25.90|58.00|0.38|0.00|2.32|19.14|19.52|21.46|21.84|-6.76| +2450845|||87084|1304332||21228|||780|21228|4||||6196|156|1555|88|12.23|||733.04|||1786.40|31.60||249.92||1084.96|1303.28|1334.88|-22.88| +2450845|23959|2450852|87084|1304332|780|21228|87084|1304332|780|21228|4|91|16|3|15332|94|1555|27|45.70|79.51|44.52|944.73|1202.04|1233.90|2146.77|60.10|0.00|42.93|1202.04|1262.14|1244.97|1305.07|-31.86| +2450845|23959|2450909|87084|1304332|780|21228|87084|1304332|780|21228|4|25|6|2|6034|82|1555|60|8.18|22.24|14.23|480.60|853.80|490.80|1334.40|59.76|0.00|386.40|853.80|913.56|1240.20|1299.96|363.00| +2450845|23959|2450875|87084|1304332|780|21228|87084|1304332|780|21228|4|27|3|3|5264|9|1555|62|89.26|256.17|176.75|4924.04|10958.50|5534.12|15882.54|208.21|6794.27|4923.42|4164.23|4372.44|9087.65|9295.86|-1369.89| +2450845|39094|2450870|70429|1196947|5065|25583|70429|1196947|5065|25583|1|44|8|5|8905|34|1556|85|22.99|44.60|13.82|2616.30|1174.70|1954.15|3791.00|35.24|0.00|909.50|1174.70|1209.94|2084.20|2119.44|-779.45| +2450845|39094|2450868|70429|1196947|5065|25583|70429|1196947|5065|25583|1|108|10|3|9970|199|1556|28|62.79|148.81|72.91|2125.20|2041.48|1758.12|4166.68|61.24|816.59|583.24|1224.89|1286.13|1808.13|1869.37|-533.23| +2450845|39094|2450914|70429|1196947|5065|25583|70429|1196947|5065|25583|1|90|9|5|13424|269|1556|33|11.20|17.58|5.80|388.74|191.40|369.60|580.14|2.46|109.09|144.87|82.31|84.77|227.18|229.64|-287.29| +2450845|39094|2450873|70429|1196947|5065|25583|70429|1196947|5065|25583|1|4|18|1|14630|80|1556|19|40.07|92.16|16.58|1436.02|315.02|761.33|1751.04|0.00|189.01|682.86|126.01|126.01|808.87|808.87|-635.32| +2450845|39094|2450929|70429|1196947|5065|25583|70429|1196947|5065|25583|1|9|17|3|16388|160|1556|96|72.85|121.65|119.21|234.24|11444.16|6993.60|11678.40|343.32|0.00|3152.64|11444.16|11787.48|14596.80|14940.12|4450.56| +2450845|39094|2450919|70429|1196947|5065|25583|70429|1196947|5065|25583|1|41|12|2|11503|297|1556|97|69.00|100.74|68.50|3127.28|6644.50|6693.00|9771.78|504.98|332.22|2637.43|6312.28|6817.26|8949.71|9454.69|-380.72| +2450845|39094|2450876|70429|1196947|5065|25583|70429|1196947|5065|25583|1|102|8|5|16910|4|1556|67|80.60|176.51|26.47|10052.68|1773.49|5400.20|11826.17|124.14|0.00|1773.49|1773.49|1897.63|3546.98|3671.12|-3626.71| +2450845|39094|2450916|70429|1196947|5065|25583|70429|1196947|5065|25583|1|29|18|4|16315|199|1556|1|53.25|123.00|55.35|67.65|55.35|53.25|123.00|3.87|0.00|27.06|55.35|59.22|82.41|86.28|2.10| +2450845|39094|2450929|70429|1196947|5065|25583|70429|1196947|5065|25583|1|102|14|2|16348|58|1556|45|55.16|146.17|87.70|2631.15|3946.50|2482.20|6577.65|0.00|0.00|1907.10|3946.50|3946.50|5853.60|5853.60|1464.30| +2450845|39094|2450901|70429|1196947|5065|25583|70429|1196947|5065|25583|1|100|17|5|6548|257|1556|60|59.52|137.49|81.11|3382.80|4866.60|3571.20|8249.40|97.33|0.00|3794.40|4866.60|4963.93|8661.00|8758.33|1295.40| +2450845|39094|2450880|70429|1196947|5065|25583|70429|1196947|5065|25583|1|103|15|1|12142|285|1556|39|60.14|81.18|77.93|126.75|3039.27|2345.46|3166.02|212.74|0.00|1551.03|3039.27|3252.01|4590.30|4803.04|693.81| +2450845|39094|2450909|70429|1196947|5065|25583|70429|1196947|5065|25583|1|92|4|5|13237|65|1556|38|55.05|94.68|75.74|719.72|2878.12|2091.90|3597.84|0.00|2072.24|1043.10|805.88|805.88|1848.98|1848.98|-1286.02| +2450845|73154|2450894|83697|1304691|4885|44232|83697|1304691|4885|44232|2|48|5|1|13342|28|1557|43|83.64|212.44|114.71|4202.39|4932.53|3596.52|9134.92|177.57|1973.01|1461.57|2959.52|3137.09|4421.09|4598.66|-637.00| +2450845|73154|2450898|83697|1304691|4885|44232|83697|1304691|4885|44232|2|25|16|2|17869|283|1557|76|37.31|98.87|35.59|4809.28|2704.84|2835.56|7514.12|162.29|0.00|0.00|2704.84|2867.13|2704.84|2867.13|-130.72| +2450845|73154|2450884|83697|1304691|4885|44232|83697|1304691|4885|44232|2|31|5|1|2041|173|1557|32|87.07|250.76|180.54|2247.04|5777.28|2786.24|8024.32|519.95|0.00|240.64|5777.28|6297.23|6017.92|6537.87|2991.04| +2450845|73154|2450896|83697|1304691|4885|44232|83697|1304691|4885|44232|2|97|10|2|79|254|1557|46|24.06|25.98|7.01|872.62|322.46|1106.76|1195.08|9.67|0.00|202.86|322.46|332.13|525.32|534.99|-784.30| +2450845|73154|2450852|83697|1304691|4885|44232|83697|1304691|4885|44232|2|37|5|3|13288|282|1557|77|97.30|279.25|61.43|16772.14|4730.11|7492.10|21502.25|26.96|2033.94|7740.81|2696.17|2723.13|10436.98|10463.94|-4795.93| +2450845|73154|2450927|83697|1304691|4885|44232|83697|1304691|4885|44232|2|13|10|5|242|178|1557|2|46.38|70.96|19.86|102.20|39.72|92.76|141.92|0.79|0.00|43.98|39.72|40.51|83.70|84.49|-53.04| +2450845|73154|2450908|83697|1304691|4885|44232|83697|1304691|4885|44232|2|82|2|4|3124|237|1557|29|20.05|24.26|20.37|112.81|590.73|581.45|703.54|41.35|0.00|316.39|590.73|632.08|907.12|948.47|9.28| +2450845|73154|2450891|83697|1304691|4885|44232|83697|1304691|4885|44232|2|21|16|4|3325|84|1557|75|88.02|125.86|10.06|8685.00|754.50|6601.50|9439.50|0.00|0.00|3964.50|754.50|754.50|4719.00|4719.00|-5847.00| +2450845|73154|2450849|83697|1304691|4885|44232|83697|1304691|4885|44232|2|72|17|4|946|1|1557|3|5.49|6.20|5.14|3.18|15.42|16.47|18.60|0.61|0.00|3.72|15.42|16.03|19.14|19.75|-1.05| +2450845|73154|2450888|83697|1304691|4885|44232|83697|1304691|4885|44232|2|44|12|5|2012|41|1557|21|43.50|127.45|61.17|1391.88|1284.57|913.50|2676.45|51.38|0.00|428.19|1284.57|1335.95|1712.76|1764.14|371.07| +2450845|73154|2450886|83697|1304691|4885|44232|83697|1304691|4885|44232|2|41|18|5|17809|196|1557|47|67.92|167.08|83.54|3926.38|3926.38|3192.24|7852.76|274.84|0.00|3297.99|3926.38|4201.22|7224.37|7499.21|734.14| +2450845|73154|2450899|83697|1304691|4885|44232|83697|1304691|4885|44232|2|34|4|1|2233|20|1557|64|83.84|152.58|6.10|9374.72|390.40|5365.76|9765.12|7.80|0.00|4882.56|390.40|398.20|5272.96|5280.76|-4975.36| +2450845|73154|2450900|83697|1304691|4885|44232|83697|1304691|4885|44232|2|102|12|4|5350|176|1557|80|64.04|188.27|96.01|7380.80|7680.80|5123.20|15061.60|614.46|0.00|5421.60|7680.80|8295.26|13102.40|13716.86|2557.60| +2450845|73154|2450935|83697|1304691|4885|44232|83697|1304691|4885|44232|2|103|10|1|3572|298|1557|60|15.70|30.30|15.75|873.00|945.00|942.00|1818.00|37.80|0.00|708.60|945.00|982.80|1653.60|1691.40|3.00| +2450845|47143|2450855|20531|45400|669|38738|20531|45400|669|38738|4|54|11|5|3457|258|1558|7|60.92|124.88|1.24|865.48|8.68|426.44|874.16|0.43|0.00|96.11|8.68|9.11|104.79|105.22|-417.76| +2450845|47143|2450906|20531|45400|669|38738|20531|45400|669|38738|4|94|11|3|9208|66|1558|91|91.70|242.08|38.73|18504.85|3524.43|8344.70|22029.28|0.00|0.00|1541.54|3524.43|3524.43|5065.97|5065.97|-4820.27| +2450845|47143|2450921|20531|45400|669|38738|20531|45400|669|38738|4|11|20|2|5491|146|1558|6|10.93|24.70|2.96|130.44|17.76|65.58|148.20|0.71|0.00|57.78|17.76|18.47|75.54|76.25|-47.82| +2450845|47143|2450921|20531|45400|669|38738|20531|45400|669|38738|4|10|9|5|4856|207|1558|3|7.47|7.76|6.28|4.44|18.84|22.41|23.28|0.18|0.00|5.34|18.84|19.02|24.18|24.36|-3.57| +2450845|47143|2450925|20531|45400|669|38738|20531|45400|669|38738|4|104|15|3|9812|166|1558|18|53.23|93.15|43.78|888.66|788.04|958.14|1676.70|70.92|0.00|804.78|788.04|858.96|1592.82|1663.74|-170.10| +2450845|47143|2450885|20531|45400|669|38738|20531|45400|669|38738|4|47|20|4|16834|119|1558|69|56.31|109.24|33.86|5201.22|2336.34|3885.39|7537.56|186.90|0.00|3165.72|2336.34|2523.24|5502.06|5688.96|-1549.05| +2450845|47143|2450862|20531|45400|669|38738|20531|45400|669|38738|4|26|1|5|12037|298|1558|8|37.19|107.85|37.74|560.88|301.92|297.52|862.80|9.05|0.00|86.24|301.92|310.97|388.16|397.21|4.40| +2450845|47143|2450896|20531|45400|669|38738|20531|45400|669|38738|4|78|8|4|14396|229|1558|8|99.20|195.42|60.58|1078.72|484.64|793.60|1563.36|19.38|0.00|484.64|484.64|504.02|969.28|988.66|-308.96| +2450845|47143|2450923|20531|45400|669|38738|20531|45400|669|38738|4|3|3|4|1018|191|1558|30|76.60|117.96|36.56|2442.00|1096.80|2298.00|3538.80|32.90|0.00|1273.80|1096.80|1129.70|2370.60|2403.50|-1201.20| +2450845|47143|2450884|20531|45400|669|38738|20531|45400|669|38738|4|45|1|3|16045|14|1558|15|12.05|16.02|8.65|110.55|129.75|180.75|240.30|1.58|50.60|36.00|79.15|80.73|115.15|116.73|-101.60| +2450845|47143|2450872|20531|45400|669||20531||669||4|84|7||3862||1558||19.64|33.97||217.60||1256.96|||0.00|521.60|||2478.08|2615.03|699.52| +2450845|47143|2450915|20531|45400|669|38738|20531|45400|669|38738|4|94|11|5|11264|19|1558|11|12.65|36.93|24.74|134.09|272.14|139.15|406.23|0.00|0.00|93.39|272.14|272.14|365.53|365.53|132.99| +2450845|47143|2450890|20531|45400|669|38738|20531|45400|669|38738|4|11|11|5|9289|210|1558|99|34.81|45.94|43.18|273.24|4274.82|3446.19|4548.06|213.74|0.00|1136.52|4274.82|4488.56|5411.34|5625.08|828.63| +2450845|72399|2450913|21224|1757455|1333|8011|21224|1757455|1333|8011|1|2|12|5|10732|265|1559|40|72.64|74.81|17.95|2274.40|718.00|2905.60|2992.40|28.72|0.00|89.60|718.00|746.72|807.60|836.32|-2187.60| +2450845|72399|2450856|21224|1757455|1333|8011|21224|1757455|1333|8011|1|33|12|3|9787|65|1559|97|79.37|196.04|148.99|4563.85|14452.03|7698.89|19015.88|1300.68|0.00|7795.89|14452.03|15752.71|22247.92|23548.60|6753.14| +2450845|72399|2450852|21224|1757455|1333|8011|21224|1757455|1333|8011|1|32|17|4|3508|151|1559|53|83.89|104.02|95.69|441.49|5071.57|4446.17|5513.06|456.44|0.00|1598.48|5071.57|5528.01|6670.05|7126.49|625.40| +2450845|72399|2450884|21224|1757455|1333|8011|21224|1757455|1333|8011|1|95|16|5|1264|218|1559|10|60.38|93.58|20.58|730.00|205.80|603.80|935.80|8.23|0.00|46.70|205.80|214.03|252.50|260.73|-398.00| +2450845|72399|2450877|21224|1757455|1333|8011|21224|1757455|1333|8011|1|94|6|1|4225|191|1559|96|61.29|64.35|12.22|5004.48|1173.12|5883.84|6177.60|35.19|0.00|1606.08|1173.12|1208.31|2779.20|2814.39|-4710.72| +2450845|72399|2450918|21224|1757455|1333|8011|21224|1757455|1333|8011|1|38|1|5|6814|60|1559|84|96.42|214.05|145.55|5754.00|12226.20|8099.28|17980.20|0.00|0.00|2516.64|12226.20|12226.20|14742.84|14742.84|4126.92| +2450845|72399|2450872|21224|1757455|1333|8011|21224|1757455|1333|8011|1|84|5|4|1630|60|1559|38|52.43|102.23|42.93|2253.40|1631.34|1992.34|3884.74|130.50|0.00|1475.92|1631.34|1761.84|3107.26|3237.76|-361.00| +2450845|72399|2450861|21224|1757455|1333|8011|21224|1757455|1333|8011|1|80|1|5|14188|66|1559|72|1.18|2.17|2.01|11.52|144.72|84.96|156.24|11.57|0.00|0.00|144.72|156.29|144.72|156.29|59.76| +2450845|72399|2450933|21224|1757455|1333|8011|21224|1757455|1333|8011|1|15|15|1|5482|283|1559|66|55.39|110.22|11.02|6547.20|727.32|3655.74|7274.52|7.27|0.00|3127.74|727.32|734.59|3855.06|3862.33|-2928.42| +2450845|72399|2450853|21224|1757455|1333|8011|21224|1757455|1333|8011|1|87|17|4|17527|50|1559|44|8.61|12.31|11.94|16.28|525.36|378.84|541.64|21.01|0.00|118.80|525.36|546.37|644.16|665.17|146.52| +2450845|57689|2450909|98256|30823|4990|19604|98256|30823|4990|19604|1|90|5|3|9325|190|1560|16|70.06|138.71|55.48|1331.68|887.68|1120.96|2219.36|53.26|0.00|932.00|887.68|940.94|1819.68|1872.94|-233.28| +2450845|57689|2450909|98256|30823|4990|19604|98256|30823|4990|19604|1|1|5|2|4060|166|1560|84|42.64|83.57|78.55|421.68|6598.20|3581.76|7019.88|527.85|0.00|771.96|6598.20|7126.05|7370.16|7898.01|3016.44| +2450845|57689|2450914|98256|30823|4990|19604|98256|30823|4990|19604|1|32|1|4|1000|50|1560|35|38.21|74.89|8.98|2306.85|314.30|1337.35|2621.15|11.66|147.72|261.80|166.58|178.24|428.38|440.04|-1170.77| +2450845|57689|2450930|98256|30823|4990|19604|98256|30823|4990|19604|1|34|5|3|4648|106|1560|12|98.68|265.44|225.62|477.84|2707.44|1184.16|3185.28|151.07|189.52|382.20|2517.92|2668.99|2900.12|3051.19|1333.76| +2450845|57689|2450916|98256|30823|4990|19604|98256|30823|4990|19604|1|54|18|2|17030|217|1560|67|48.95|67.06|52.30|988.92|3504.10|3279.65|4493.02|140.16|700.82|1347.37|2803.28|2943.44|4150.65|4290.81|-476.37| +2450845|57689|2450931|98256|30823|4990|19604|98256|30823|4990|19604|1|39|12|2|6148|45|1560|13|40.75|67.23|36.97|393.38|480.61|529.75|873.99|43.25|0.00|78.65|480.61|523.86|559.26|602.51|-49.14| +2450845|57689|2450920|98256|30823|4990|19604|98256|30823|4990|19604|1|83|14|3|3433|191|1560|39|19.54|41.22|21.02|787.80|819.78|762.06|1607.58|8.85|721.40|80.34|98.38|107.23|178.72|187.57|-663.68| +2450845|4912|2450903|64550|1873775|1753|24578|64550|1873775|1753|24578|4|5|13|2|12169|115|1561|28|12.01|26.30|9.73|463.96|272.44|336.28|736.40|3.67|198.88|147.28|73.56|77.23|220.84|224.51|-262.72| +2450845|4912|2450927|64550|1873775|1753|24578|64550|1873775|1753|24578|4|32|18|2|13900|213|1561|99|4.38|11.56|0.23|1121.67|22.77|433.62|1144.44|0.22|0.00|411.84|22.77|22.99|434.61|434.83|-410.85| +2450845|4912|2450899|64550|1873775|1753|24578|64550|1873775|1753|24578|4|67|19|2|398|163|1561|96|98.93|145.42|31.99|10889.28|3071.04|9497.28|13960.32|122.84|0.00|4048.32|3071.04|3193.88|7119.36|7242.20|-6426.24| +2450845|4912|2450894|64550|1873775|1753|24578|64550|1873775|1753|24578|4|68|14|2|4303|240|1561|67|46.16|77.08|2.31|5009.59|154.77|3092.72|5164.36|12.38|0.00|309.54|154.77|167.15|464.31|476.69|-2937.95| +2450845|4912|2450879|64550|1873775|1753|24578|64550|1873775|1753|24578|4|96|18|4|15026|101|1561|98|20.17|40.94|36.43|441.98|3570.14|1976.66|4012.12|214.20|0.00|1403.36|3570.14|3784.34|4973.50|5187.70|1593.48| +2450845|48280|2450847|77195|804732|3469|35853|77195|804732|3469|35853|4|84|11|2|16658|291|1562|73|91.21|269.06|172.19|7071.51|12569.87|6658.33|19641.38|125.69|0.00|196.37|12569.87|12695.56|12766.24|12891.93|5911.54| +2450845|48280|2450899|77195|804732|3469|35853|77195|804732|3469|35853|4|58|2|5|7105|207|1562|60|5.01|9.66|7.92|104.40|475.20|300.60|579.60|28.51|0.00|202.80|475.20|503.71|678.00|706.51|174.60| +2450845|48280|2450870|77195|804732|3469|35853|77195|804732|3469|35853|4|26|13|2|1114|285|1562|43|66.12|128.27|66.70|2647.51|2868.10|2843.16|5515.61|258.12|0.00|1378.58|2868.10|3126.22|4246.68|4504.80|24.94| +2450845|48280|2450907|77195|804732|3469|35853|77195|804732|3469|35853|4|20|1|4|10726|251|1562|83|97.76|256.13|12.80|20196.39|1062.40|8114.08|21258.79|0.00|0.00|2338.11|1062.40|1062.40|3400.51|3400.51|-7051.68| +2450845|39204|2450894|56338|133376|6414|10252|56338|133376|6414|10252|1|87|7|3|15284|20|1563|92|51.69|145.24|108.93|3340.52|10021.56|4755.48|13362.08|200.43|0.00|0.00|10021.56|10221.99|10021.56|10221.99|5266.08| +2450845|39204|2450917|56338|133376|6414|10252|56338|133376|6414|10252|1|2|5|2|4474|116|1563|33|24.72|50.18|34.62|513.48|1142.46|815.76|1655.94|0.00|0.00|66.00|1142.46|1142.46|1208.46|1208.46|326.70| +2450845|39204|2450858|56338|133376|6414|10252|56338|133376|6414|10252|1|100|15|3|13484|62|1563|69|16.08|22.99|17.01|412.62|1173.69|1109.52|1586.31|0.00|0.00|316.71|1173.69|1173.69|1490.40|1490.40|64.17| +2450845|39204|2450893|56338|133376|6414|10252|56338|133376|6414|10252|1|6|10|3|1874|107|1563|47|34.43|60.25|14.46|2152.13|679.62|1618.21|2831.75|2.71|645.63|254.74|33.99|36.70|288.73|291.44|-1584.22| +2450845|39204|2450928|56338|133376|6414|10252|56338|133376|6414|10252|1|52|12|3|16484|177|1563|95|59.58|178.74|84.00|9000.30|7980.00|5660.10|16980.30|0.00|0.00|1867.70|7980.00|7980.00|9847.70|9847.70|2319.90| +2450845|39204|2450905|56338|133376|6414|10252|56338|133376|6414|10252|1|46|8|5|3835|87|1563|19|77.77|97.21|97.21|0.00|1846.99|1477.63|1846.99|129.28|0.00|646.38|1846.99|1976.27|2493.37|2622.65|369.36| +2450845|39204|2450923|56338|133376|6414|10252|56338|133376|6414|10252|1|67|17|4|10010|220|1563|51|4.73|9.46|6.62|144.84|337.62|241.23|482.46|3.37|0.00|23.97|337.62|340.99|361.59|364.96|96.39| +2450845|39204|2450852|56338|133376|6414|10252|56338|133376|6414|10252|1|55|2|1|9325|191|1563|60|16.36|38.44|29.59|531.00|1775.40|981.60|2306.40|35.50|0.00|91.80|1775.40|1810.90|1867.20|1902.70|793.80| +2450845|39204|2450855|56338|133376|6414|10252|56338|133376|6414|10252|1|81|1|3|4060|217|1563|23|73.84|113.71|86.41|627.90|1987.43|1698.32|2615.33|19.87|0.00|339.94|1987.43|2007.30|2327.37|2347.24|289.11| +2450845|39204|2450930|56338|133376|6414|10252|56338|133376|6414|10252|1|93|7|1|1000|224|1563|21|9.99|29.07|11.04|378.63|231.84|209.79|610.47|6.95|0.00|73.08|231.84|238.79|304.92|311.87|22.05| +2450845|39204|2450862|56338|133376|6414|10252|56338|133376|6414|10252|1|56|3|4|4648|279|1563|93|5.47|14.33|13.90|39.99|1292.70|508.71|1332.69|51.70|0.00|492.90|1292.70|1344.40|1785.60|1837.30|783.99| +2450845|39204|2450858|56338|133376|6414|10252|56338|133376|6414|10252|1|103|4|1|17030|180|1563|79|28.98|44.91|0.00|3547.89|0.00|2289.42|3547.89|0.00|0.00|1028.58|0.00|0.00|1028.58|1028.58|-2289.42| +2450845|39204|2450934|56338|133376|6414|10252|56338|133376|6414|10252|1|18|20|4|6148|277|1563|36|27.07|54.41|42.98|411.48|1547.28|974.52|1958.76|40.22|541.54|489.60|1005.74|1045.96|1495.34|1535.56|31.22| +2450845|39204|2450923|56338|133376|6414|10252|56338|133376|6414|10252|1|70|19|5|3433|77|1563|72|72.72|106.89|52.37|3925.44|3770.64|5235.84|7696.08|301.65|0.00|538.56|3770.64|4072.29|4309.20|4610.85|-1465.20| +2450845|65668|2450888|4965|731356|5300|27291|4965|731356|5300|27291|1|36|8|5|496|47|1564|66|90.74|215.05|126.87|5819.88|8373.42|5988.84|14193.30|418.67|0.00|3548.16|8373.42|8792.09|11921.58|12340.25|2384.58| +2450845|65668|2450926|4965|731356|5300|27291|4965|731356|5300|27291|1|54|18|2|229|163|1564|55|73.24|146.48|99.60|2578.40|5478.00|4028.20|8056.40|54.78|0.00|80.30|5478.00|5532.78|5558.30|5613.08|1449.80| +2450845|65668|2450872|4965|731356|5300|27291|4965|731356|5300|27291|1|103|12|4|9608|231|1564|6|84.32|252.96|202.36|303.60|1214.16|505.92|1517.76|72.84|0.00|364.26|1214.16|1287.00|1578.42|1651.26|708.24| +2450845|65668|2450886|4965|731356|5300|27291|4965|731356|5300|27291|1|58|10|3|13009|218|1564|66|95.04|233.79|156.63|5092.56|10337.58|6272.64|15430.14|930.38|0.00|770.88|10337.58|11267.96|11108.46|12038.84|4064.94| +2450845|9959|2450908|39832|944590|2595|35353|39832|944590|2595|35353|4|95|20|1|10411|33|1565|44|6.82|16.64|5.49|490.60|241.56|300.08|732.16|2.41|0.00|358.60|241.56|243.97|600.16|602.57|-58.52| +2450845|9959|2450894|39832|944590|2595|35353|39832|944590|2595|35353|4|16|3|5|10753|53|1565|4|33.02|72.64|57.38|61.04|229.52|132.08|290.56|9.18|0.00|122.00|229.52|238.70|351.52|360.70|97.44| +2450845|9959|2450934|39832|944590|2595|35353|39832|944590|2595|35353|4|68|6|2|9199|8|1565|58|14.78|43.60|34.88|505.76|2023.04|857.24|2528.80|53.81|1254.28|1036.46|768.76|822.57|1805.22|1859.03|-88.48| +2450845|9959|2450875|39832|944590|2595|35353|39832|944590|2595|35353|4|58|11|5|14714|205|1565|86|35.52|59.67|51.91|667.36|4464.26|3054.72|5131.62|357.14|0.00|769.70|4464.26|4821.40|5233.96|5591.10|1409.54| +2450845|9959|2450886|39832|944590|2595|35353|39832|944590|2595|35353|4|1|12|2|3088|47|1565|9|44.77|58.20|24.44|303.84|219.96|402.93|523.80|13.19|0.00|240.93|219.96|233.15|460.89|474.08|-182.97| +2450845|9959|2450853|39832|944590|2595|35353|39832|944590|2595|35353|4|65|5|4|7960|223|1565|11|35.86|93.23|41.02|574.31|451.22|394.46|1025.53|36.09|0.00|122.98|451.22|487.31|574.20|610.29|56.76| +2450845|9959|2450850|39832|944590|2595|35353|39832|944590|2595|35353|4|76|7|2|2354|257|1565|96|41.06|114.55|95.07|1870.08|9126.72|3941.76|10996.80|547.60|0.00|1978.56|9126.72|9674.32|11105.28|11652.88|5184.96| +2450845|9959|2450919|39832|944590|2595|35353|39832|944590|2595|35353|4|18|12|1|13816|6|1565|69|13.81|34.93|13.62|1470.39|939.78|952.89|2410.17|56.38|0.00|433.32|939.78|996.16|1373.10|1429.48|-13.11| +2450845|9959|2450889|39832|944590|2595|35353|39832|944590|2595|35353|4|63|2|5|16774|204|1565|52|72.40|172.31|170.58|89.96|8870.16|3764.80|8960.12|500.27|532.20|3315.00|8337.96|8838.23|11652.96|12153.23|4573.16| +2450845|9959|2450877|39832|944590|2595|35353|39832|944590|2595|35353|4|32|15|3|16442|148|1565|64|38.00|101.08|40.43|3881.60|2587.52|2432.00|6469.12|155.25|0.00|1034.88|2587.52|2742.77|3622.40|3777.65|155.52| +2450845|9959|2450898|39832|944590|2595|35353|39832|944590|2595|35353|4|20|11|4|12320|206|1565|89|25.23|45.16|23.93|1889.47|2129.77|2245.47|4019.24|111.81|532.44|682.63|1597.33|1709.14|2279.96|2391.77|-648.14| +2450845|9959|2450918|39832|944590|2595|35353|39832|944590|2595|35353|4|107|9|4|8708|132|1565|41|99.66|161.44|12.91|6089.73|529.31|4086.06|6619.04|47.63|0.00|661.74|529.31|576.94|1191.05|1238.68|-3556.75| +2450845|9959|2450920|39832|944590|2595|35353|39832|944590|2595|35353|4|90|8|1|9535|22|1565|78|72.09|136.25|119.90|1275.30|9352.20|5623.02|10627.50|93.52|0.00|1806.48|9352.20|9445.72|11158.68|11252.20|3729.18| +2450845|22586|2450848|10260|1057228|1443|2011|10260|1057228|1443|2011|2|3|11|1|15739|278|1566|55|88.53|175.28|45.57|7134.05|2506.35|4869.15|9640.40|225.57|0.00|3856.05|2506.35|2731.92|6362.40|6587.97|-2362.80| +2450845|22586|2450896|10260|1057228|1443|2011|10260|1057228|1443|2011|2|88|14|3|7678|61|1566|88|64.73|114.57|58.43|4940.32|5141.84|5696.24|10082.16|51.41|0.00|4737.92|5141.84|5193.25|9879.76|9931.17|-554.40| +2450845|22586|2450884|10260|1057228|1443|2011|10260|1057228|1443|2011|2|48|8|4|13069|148|1566|100|57.99|86.98|77.41|957.00|7741.00|5799.00|8698.00|387.05|0.00|3131.00|7741.00|8128.05|10872.00|11259.05|1942.00| +2450845|22586|2450934|10260|1057228|1443|2011|10260|1057228|1443|2011|2|16|16|1|5150|268|1566|38|4.65|7.06|6.35|26.98|241.30|176.70|268.28|21.71|0.00|66.88|241.30|263.01|308.18|329.89|64.60| +2450845|22586|2450917|10260|1057228|1443|2011|10260|1057228|1443|2011|2|88|14|3|16738|44|1566|22|85.89|121.10|48.44|1598.52|1065.68|1889.58|2664.20|74.59|0.00|479.38|1065.68|1140.27|1545.06|1619.65|-823.90| +2450845|22586|2450882|10260|1057228|1443|2011|10260|1057228|1443|2011|2|18|3|3|14176|73|1566|92|88.54|168.22|26.91|13000.52|2475.72|8145.68|15476.24|123.78|0.00|4333.20|2475.72|2599.50|6808.92|6932.70|-5669.96| +2450845|22586|2450849|10260|1057228|1443|2011|10260|1057228|1443|2011|2|65|20|2|9146|146|1566|71|19.72|47.91|42.63|374.88|3026.73|1400.12|3401.61|0.00|0.00|1632.29|3026.73|3026.73|4659.02|4659.02|1626.61| +2450845|22586|2450882|10260|1057228|1443|2011|10260|1057228|1443|2011|2|65|11|5|4381|93|1566|69|5.47|12.79|8.56|291.87|590.64|377.43|882.51|0.00|0.00|87.63|590.64|590.64|678.27|678.27|213.21| +2450845|22586|2450888|10260|1057228|1443|2011|10260|1057228|1443|2011|2|83|14|5|14816|125|1566|97|59.83|127.43|109.58|1731.45|10629.26|5803.51|12360.71|212.58|0.00|1111.62|10629.26|10841.84|11740.88|11953.46|4825.75| +2450845|22586|2450860|10260|1057228|1443|2011|10260|1057228|1443|2011|2|27|9|3|12547|74|1566|26|35.95|64.71|42.70|572.26|1110.20|934.70|1682.46|99.91|0.00|336.44|1110.20|1210.11|1446.64|1546.55|175.50| +2450845|22586|2450881|10260|1057228|1443|2011|10260|1057228|1443|2011|2|47|3|3|16568|41|1566|96|72.39|154.19|106.39|4588.80|10213.44|6949.44|14802.24|0.00|0.00|1923.84|10213.44|10213.44|12137.28|12137.28|3264.00| +2450845|65877|2450932|59907|304417|1727|24637|59907|304417|1727|24637|2|14|12|5|12134|42|1567|65|87.83|89.58|84.20|349.70|5473.00|5708.95|5822.70|164.19|0.00|1047.80|5473.00|5637.19|6520.80|6684.99|-235.95| +2450845|65877|2450918|59907|304417|1727|24637|59907|304417|1727|24637|2|67|18|4|17431|31|1567|51|7.13|19.75|14.61|262.14|745.11|363.63|1007.25|22.35|0.00|452.88|745.11|767.46|1197.99|1220.34|381.48| +2450845|65877|2450854|59907|304417|1727|24637|59907|304417|1727|24637|2|86|7|1|15104|137|1567|75|7.26|11.39|10.82|42.75|811.50|544.50|854.25|32.46|0.00|102.00|811.50|843.96|913.50|945.96|267.00| +2450845|65877|2450896|59907|304417|1727|24637|59907|304417|1727|24637|2|105|2|4|11779|214|1567|65|98.60|150.85|85.98|4216.55|5588.70|6409.00|9805.25|111.77|0.00|3431.35|5588.70|5700.47|9020.05|9131.82|-820.30| +2450845|65877|2450870|59907|304417|1727|24637|59907|304417|1727|24637|2|43|10|4|17446|75|1567|36|34.47|93.75|39.37|1957.68|1417.32|1240.92|3375.00|113.38|0.00|1653.48|1417.32|1530.70|3070.80|3184.18|176.40| +2450845|65877|2450895|59907|304417|1727|24637|59907|304417|1727|24637|2|87|6|2|9130|76|1567|49|46.19|68.36|62.20|301.84|3047.80|2263.31|3349.64|30.17|2042.02|1272.53|1005.78|1035.95|2278.31|2308.48|-1257.53| +2450845|65877|2450872|59907|304417|1727|24637|59907|304417|1727|24637|2|78|6|1|98|131|1567|29|51.68|86.82|63.37|680.05|1837.73|1498.72|2517.78|110.26|0.00|478.21|1837.73|1947.99|2315.94|2426.20|339.01| +2450845|65877|2450892|59907|304417|1727|24637|59907|304417|1727|24637|2|34|18|5|7756|265|1567|33|25.89|47.11|18.84|932.91|621.72|854.37|1554.63|31.08|0.00|264.00|621.72|652.80|885.72|916.80|-232.65| +2450845|65877|2450868|59907|304417|1727|24637|59907|304417|1727|24637|2|33|20|2|5216|281|1567|25|92.20|218.51|203.21|382.50|5080.25|2305.00|5462.75|304.81|0.00|874.00|5080.25|5385.06|5954.25|6259.06|2775.25| +2450845|65877|2450870|59907|304417|1727|24637|59907|304417|1727|24637|2|10|5|5|6212|137|1567|20|65.74|99.26|67.49|635.40|1349.80|1314.80|1985.20|0.00|755.88|595.40|593.92|593.92|1189.32|1189.32|-720.88| +2450845|65877|2450870|59907|304417|1727|24637|59907|304417|1727|24637|2|98|10|3|7396|151|1567|23|71.17|172.94|88.19|1949.25|2028.37|1636.91|3977.62|182.55|0.00|1909.23|2028.37|2210.92|3937.60|4120.15|391.46| +2450845|65877|2450877|59907|304417|1727|24637|59907|304417|1727|24637|2|15|12|2|344|62|1567|77|37.64|89.95|30.58|4571.49|2354.66|2898.28|6926.15|59.33|1506.98|1038.73|847.68|907.01|1886.41|1945.74|-2050.60| +2450845|65877|2450895|59907|304417|1727|24637|59907|304417|1727|24637|2|2|14|3|8947|226|1567|72|48.74|101.86|47.87|3887.28|3446.64|3509.28|7333.92|34.46|0.00|1026.72|3446.64|3481.10|4473.36|4507.82|-62.64| +2450845|27923|2450854|74638|558159|2480|49201|74638|558159|2480|49201|4|16|5|4|8707|210|1568|13|51.46|112.18|5.60|1385.54|72.80|668.98|1458.34|0.00|0.00|218.66|72.80|72.80|291.46|291.46|-596.18| +2450845|27923|2450857|74638|558159|2480|49201|74638|558159|2480|49201|4|74|12|5|17368|216|1568|100|93.02|203.71|59.07|14464.00|5907.00|9302.00|20371.00|118.14|0.00|1222.00|5907.00|6025.14|7129.00|7247.14|-3395.00| +2450845|27923|2450907|74638|558159|2480|49201|74638|558159|2480|49201|4|29|17|4|6295|43|1568|60|70.07|90.39|25.30|3905.40|1518.00|4204.20|5423.40|29.75|1092.96|487.80|425.04|454.79|912.84|942.59|-3779.16| +2450845|27923|2450928|74638|558159|2480|49201|74638|558159|2480|49201|4|10|5|4|6230|55|1568|60|1.66|2.98|0.14|170.40|8.40|99.60|178.80|0.75|0.00|84.00|8.40|9.15|92.40|93.15|-91.20| +2450845|27923|2450917|74638|558159|2480|49201|74638|558159|2480|49201|4|34|2|2|14458|159|1568|14|3.33|6.72|2.75|55.58|38.50|46.62|94.08|2.69|0.00|41.30|38.50|41.19|79.80|82.49|-8.12| +2450845|27923|2450895|74638|558159|2480|49201|74638|558159|2480|49201|4|66|12|3|17671|258|1568|63|54.66|94.56|45.38|3098.34|2858.94|3443.58|5957.28|28.58|0.00|2858.94|2858.94|2887.52|5717.88|5746.46|-584.64| +2450845|52228|2450935|60903|1326883|1310|47737|60903|1326883|1310|47737|1|52|18|3|5359|63|1569|57|85.98|187.43|183.68|213.75|10469.76|4900.86|10683.51|0.00|0.00|3632.04|10469.76|10469.76|14101.80|14101.80|5568.90| +2450845|52228|2450875|60903|1326883|1310|47737|60903|1326883|1310|47737|1|1|12|1|15710|267|1569|65|47.09|68.75|55.00|893.75|3575.00|3060.85|4468.75|286.00|0.00|2055.30|3575.00|3861.00|5630.30|5916.30|514.15| +2450845|52228|2450865|60903|1326883|1310|47737|60903|1326883|1310|47737|1|103|8|3|15679|65|1569|7|56.62|76.43|74.13|16.10|518.91|396.34|535.01|36.32|0.00|117.67|518.91|555.23|636.58|672.90|122.57| +2450845|52228|2450888|60903|1326883|1310|47737|60903|1326883|1310|47737|1|17|3|4|5593|15|1569|65|34.90|101.55|32.49|4488.90|2111.85|2268.50|6600.75|21.11|1689.48|858.00|422.37|443.48|1280.37|1301.48|-1846.13| +2450845|67723|2450908|70543|546488|4033|33531|70543|546488|4033|33531|1|74|19|4|2420|97|1570|58|76.74|163.45|13.07|8722.04|758.06|4450.92|9480.10|37.90|0.00|4739.76|758.06|795.96|5497.82|5535.72|-3692.86| +2450845|67723|2450904|70543|546488|4033|33531|70543|546488|4033|33531|1|74|18|3|9190|273|1570|46|89.51|102.04|52.04|2300.00|2393.84|4117.46|4693.84|47.87|0.00|1079.16|2393.84|2441.71|3473.00|3520.87|-1723.62| +2450845|67723|2450900|70543|546488|4033|33531|70543|546488|4033|33531|1|84|8|2|14902|265|1570|74|19.55|32.45|17.52|1104.82|1296.48|1446.70|2401.30|0.00|0.00|1152.18|1296.48|1296.48|2448.66|2448.66|-150.22| +2450845|67723|2450931|70543|546488|4033|33531|70543|546488|4033|33531|1|48|15|2|6170|104|1570|32|6.76|9.05|1.99|225.92|63.68|216.32|289.60|0.84|49.67|104.00|14.01|14.85|118.01|118.85|-202.31| +2450845|73689|2450862|60635|128966|2897|34381|60635|128966|2897|34381|1|54|7|5|1694|142|1571|73|20.75|39.01|28.08|797.89|2049.84|1514.75|2847.73|143.48|0.00|1195.74|2049.84|2193.32|3245.58|3389.06|535.09| +2450845|73689|2450891|60635|128966|2897|34381|60635|128966|2897|34381|1|15|13|4|15334|226|1571|32|46.49|118.54|17.78|3224.32|568.96|1487.68|3793.28|7.51|318.61|1479.36|250.35|257.86|1729.71|1737.22|-1237.33| +2450845|73689|2450920|60635|128966|2897|34381|60635|128966|2897|34381|1|15|15|4|8317|25|1571|5|76.81|164.37|93.69|353.40|468.45|384.05|821.85|3.37|426.28|41.05|42.17|45.54|83.22|86.59|-341.88| +2450845|73689|2450882|60635|128966|2897|34381|60635|128966|2897|34381|1|30|6|2|7663|72|1571|19|78.34|165.29|44.62|2292.73|847.78|1488.46|3140.51|76.30|0.00|219.83|847.78|924.08|1067.61|1143.91|-640.68| +2450845|73689|2450847|60635|128966|2897|34381|60635|128966|2897|34381|1|55|8|1|3838|178|1571|22|9.81|20.99|6.50|318.78|143.00|215.82|461.78|10.01|0.00|78.32|143.00|153.01|221.32|231.33|-72.82| +2450845|73689|2450900|60635|128966|2897|34381|60635|128966|2897|34381|1|103|18|5|15502|152|1571|45|3.51|9.12|2.28|307.80|102.60|157.95|410.40|4.10|0.00|77.85|102.60|106.70|180.45|184.55|-55.35| +2450845|73689|2450908|60635|128966|2897|34381|60635|128966|2897|34381|1|3|1|1|14162|160|1571|53|94.80|271.12|122.00|7903.36|6466.00|5024.40|14369.36|452.62|0.00|1293.20|6466.00|6918.62|7759.20|8211.82|1441.60| +2450845|73689|2450895|60635|128966|2897|34381|60635|128966|2897|34381|1|44|6|3|12589|54|1571|40|16.65|31.13|4.35|1071.20|174.00|666.00|1245.20|12.18|0.00|410.80|174.00|186.18|584.80|596.98|-492.00| +2450845|73689|2450911|60635|128966|2897|34381|60635|128966|2897|34381|1|101|2|3|2467|48|1571|30|82.85|117.64|16.46|3035.40|493.80|2485.50|3529.20|12.44|182.70|70.50|311.10|323.54|381.60|394.04|-2174.40| +2450845|73689|2450862|60635|128966|2897|34381|60635|128966|2897|34381|1|48|11|2|8473|181|1571|70|68.75|191.81|174.54|1208.90|12217.80|4812.50|13426.70|287.11|6475.43|3759.00|5742.37|6029.48|9501.37|9788.48|929.87| +2450845|63769|2450917|9988|1251485|4535|39232|9988|1251485|4535|39232|4|78|12|3|7183|260|1572|100|28.77|69.62|45.94|2368.00|4594.00|2877.00|6962.00|229.70|0.00|2436.00|4594.00|4823.70|7030.00|7259.70|1717.00| +2450845|63769|2450907|9988|1251485|4535|39232|9988|1251485|4535|39232|4|10|2|4|403|208|1572|45|67.67|171.20|44.51|5701.05|2002.95|3045.15|7704.00|80.11|0.00|3312.45|2002.95|2083.06|5315.40|5395.51|-1042.20| +2450845|63769|2450899|9988|1251485|4535|39232|9988|1251485|4535|39232|4|96|5|1|4724|96|1572|22|77.75|94.07|27.28|1469.38|600.16|1710.50|2069.54|12.00|0.00|62.04|600.16|612.16|662.20|674.20|-1110.34| +2450845|63769|2450889|9988|1251485|4535|39232|9988|1251485|4535|39232|4|38|2|2|634|11|1572|10|23.13|49.03|6.37|426.60|63.70|231.30|490.30|3.82|0.00|0.00|63.70|67.52|63.70|67.52|-167.60| +2450845|63769|2450903|9988|1251485|4535|39232|9988|1251485|4535|39232|4|100|1|2|10228|294|1572|90|67.94|148.78|59.51|8034.30|5355.90|6114.60|13390.20|267.79|0.00|1472.40|5355.90|5623.69|6828.30|7096.09|-758.70| +2450845|63769|2450894|9988|1251485|4535|39232|9988|1251485|4535|39232|4|106|8|3|4069|261|1572|47|60.03|64.83|55.10|457.31|2589.70|2821.41|3047.01|51.79|0.00|822.50|2589.70|2641.49|3412.20|3463.99|-231.71| +2450845|63769|2450860|9988|1251485|4535|39232|9988|1251485|4535|39232|4|19|15|5|7144|277|1572|85|17.71|20.54|2.67|1518.95|226.95|1505.35|1745.90|6.80|0.00|628.15|226.95|233.75|855.10|861.90|-1278.40| +2450845|63769|2450847|9988|1251485|4535|39232|9988|1251485|4535|39232|4|45|8|3|11686|200|1572|90|75.59|114.14|15.97|8835.30|1437.30|6803.10|10272.60|71.86|0.00|1746.00|1437.30|1509.16|3183.30|3255.16|-5365.80| +2450845|63769|2450858|9988|1251485|4535|39232|9988|1251485|4535|39232|4|91|9|5|9382|68|1572|71|63.82|137.85|122.68|1077.07|8710.28|4531.22|9787.35|696.82|0.00|2544.64|8710.28|9407.10|11254.92|11951.74|4179.06| +2450845|63769|2450869|9988|1251485|4535|39232|9988|1251485|4535|39232|4|105|17|5|10615|59|1572|63|12.94|25.62|7.94|1113.84|500.22|815.22|1614.06|25.01|0.00|677.88|500.22|525.23|1178.10|1203.11|-315.00| +2450845|63769|2450878|9988|1251485|4535|39232|9988|1251485|4535|39232|4|80|5|1|10267|281|1572|85|95.32|108.66|58.67|4249.15|4986.95|8102.20|9236.10|7.48|4737.60|2585.70|249.35|256.83|2835.05|2842.53|-7852.85| +2450845|63769|2450876|9988|1251485|4535|39232|9988|1251485|4535|39232|4|67|13|5|7850|294|1572|61|80.97|106.88|69.47|2282.01|4237.67|4939.17|6519.68|127.13|0.00|2607.75|4237.67|4364.80|6845.42|6972.55|-701.50| +2450845|63769|2450890|9988|1251485|4535|39232|9988|1251485|4535|39232|4|90|19|2|10346|168|1572|14|61.06|182.56|173.43|127.82|2428.02|854.84|2555.84|72.84|0.00|1124.48|2428.02|2500.86|3552.50|3625.34|1573.18| +2450845|63769|2450883|9988|1251485|4535|39232|9988|1251485|4535|39232|4|65|1|3|11894|215|1572|69|67.99|76.82|26.88|3445.86|1854.72|4691.31|5300.58|37.09|0.00|2384.64|1854.72|1891.81|4239.36|4276.45|-2836.59| +2450845|64844|2450915|98675|1716247|1819|20591|98675|1716247|1819|20591|4|95|3|1|9428|300|1573|100|35.19|76.71|17.64|5907.00|1764.00|3519.00|7671.00|69.14|899.64|1457.00|864.36|933.50|2321.36|2390.50|-2654.64| +2450845|64844|2450858|98675|1716247|1819|20591|98675|1716247|1819|20591|4|32|2|3|9268|112|1573|92|39.18|70.91|39.70|2871.32|3652.40|3604.56|6523.72|0.00|1241.81|2934.80|2410.59|2410.59|5345.39|5345.39|-1193.97| +2450845|64844|2450925|98675|1716247|1819|20591|98675|1716247|1819|20591|4|19|3|3|2666|85|1573|12|69.09|137.48|137.48|0.00|1649.76|829.08|1649.76|0.00|0.00|412.44|1649.76|1649.76|2062.20|2062.20|820.68| +2450845|64844|2450930|98675|1716247|1819|20591|98675|1716247|1819|20591|4|70|13|5|2116|95|1573|50|66.03|174.97|8.74|8311.50|437.00|3301.50|8748.50|26.22|0.00|2712.00|437.00|463.22|3149.00|3175.22|-2864.50| +2450845|64844|2450910|98675|1716247|1819|20591|98675|1716247|1819|20591|4|67|20|1|1882|65|1573|1|6.33|15.25|7.47|7.78|7.47|6.33|15.25|0.52|0.00|1.37|7.47|7.99|8.84|9.36|1.14| +2450845|64844|2450873|98675|1716247|1819|20591|98675|1716247|1819|20591|4|72|1|3|3686|100|1573|96|46.46|69.22|57.45|1129.92|5515.20|4460.16|6645.12|0.00|0.00|2192.64|5515.20|5515.20|7707.84|7707.84|1055.04| +2450845|64844|2450876|98675|1716247|1819|20591|98675|1716247|1819|20591|4|69|13|1|8792|9|1573|45|93.66|183.57|71.59|5039.10|3221.55|4214.70|8260.65|161.07|0.00|2147.40|3221.55|3382.62|5368.95|5530.02|-993.15| +2450845|64844|2450888|98675|1716247|1819|20591|98675|1716247|1819|20591|4|12|4|5|15134|257|1573|38|90.07|140.50|74.46|2509.52|2829.48|3422.66|5339.00|84.88|0.00|961.02|2829.48|2914.36|3790.50|3875.38|-593.18| +2450845|64844|2450932|98675|1716247|1819|20591|98675|1716247|1819|20591|4|99|19|4|17128|299|1573|48|6.33|6.58|6.38|9.60|306.24|303.84|315.84|18.37|0.00|34.56|306.24|324.61|340.80|359.17|2.40| +2450845|64844|2450929|98675|1716247|1819|20591|98675|1716247|1819|20591|4|74|11|4|11612|267|1573|64|4.01|11.54|2.76|561.92|176.64|256.64|738.56|10.59|0.00|309.76|176.64|187.23|486.40|496.99|-80.00| +2450845|64844|2450882|98675|1716247|1819|20591|98675|1716247|1819|20591|4|83|18|4|6841|261|1573|78|56.68|158.70|60.30|7675.20|4703.40|4421.04|12378.60|154.27|846.61|1485.12|3856.79|4011.06|5341.91|5496.18|-564.25| +2450845|64844|2450892|98675|1716247|1819|20591|98675|1716247|1819|20591|4|76|5|4|16726|75|1573|61|75.08|144.15|38.92|6419.03|2374.12|4579.88|8793.15|166.18|0.00|1318.82|2374.12|2540.30|3692.94|3859.12|-2205.76| +2450845|64844|2450921|98675|1716247|1819|20591|98675|1716247|1819|20591|4|75|19|3|16555|38|1573|65|68.08|114.37|60.61|3494.40|3939.65|4425.20|7434.05|44.12|1733.44|1858.35|2206.21|2250.33|4064.56|4108.68|-2218.99| +2450845|64844|2450881|98675|1716247|1819|20591|98675|1716247|1819|20591|4|54|2|2|13750|120|1573|86|82.69|109.97|32.99|6620.28|2837.14|7111.34|9457.42|0.00|0.00|2080.34|2837.14|2837.14|4917.48|4917.48|-4274.20| +2450845|70675|2450905|78053|451276|852|38875|78053|451276|852|38875|4|57|2|3|5797|237|1574|62|78.52|153.89|63.09|5629.60|3911.58|4868.24|9541.18|250.34|782.31|2766.44|3129.27|3379.61|5895.71|6146.05|-1738.97| +2450845|70675|2450912|78053|451276|852|38875|78053|451276|852|38875|4|3|4|1|4525|283|1574|90|90.24|221.08|176.86|3979.80|15917.40|8121.60|19897.20|955.04|0.00|7958.70|15917.40|16872.44|23876.10|24831.14|7795.80| +2450845|70675|2450866|78053|451276|852|38875|78053|451276|852|38875|4|80|19|2|16904|152|1574|10|26.80|80.40|3.21|771.90|32.10|268.00|804.00|2.24|0.00|241.20|32.10|34.34|273.30|275.54|-235.90| +2450845|70675|2450916|78053|451276|852|38875|78053|451276|852|38875|4|64|19|1|6769|272|1574|100|56.07|132.32|67.48|6484.00|6748.00|5607.00|13232.00|134.96|0.00|793.00|6748.00|6882.96|7541.00|7675.96|1141.00| +2450845|70675|2450879|78053|451276|852|38875|78053|451276|852|38875|4|37|6|3|16867|240|1574|2|53.79|125.86|47.82|156.08|95.64|107.58|251.72|0.95|0.00|45.30|95.64|96.59|140.94|141.89|-11.94| +2450845|70675|2450853|78053|451276|852|38875|78053|451276|852|38875|4|59|12|4|11113|166|1574|91|52.24|111.79|7.82|9461.27|711.62|4753.84|10172.89|2.49|462.55|2440.62|249.07|251.56|2689.69|2692.18|-4504.77| +2450845|70675|2450853|78053|451276|852|38875|78053|451276|852|38875|4|76|13|4|15994|159|1574|93|77.74|116.61|32.65|7808.28|3036.45|7229.82|10844.73|60.72|0.00|4012.02|3036.45|3097.17|7048.47|7109.19|-4193.37| +2450845|70675|2450900|78053|451276|852|38875|78053|451276|852|38875|4|98|2|1|4915|129|1574|89|28.86|31.16|10.59|1830.73|942.51|2568.54|2773.24|9.42|0.00|637.24|942.51|951.93|1579.75|1589.17|-1626.03| +2450845|70675|2450908|78053|451276|852|38875|78053|451276|852|38875|4|60|17|1|7334|105|1574|80|11.03|30.11|16.86|1060.00|1348.80|882.40|2408.80|80.92|0.00|24.00|1348.80|1429.72|1372.80|1453.72|466.40| +2450845|70675|2450869|78053|451276|852|38875|78053|451276|852|38875|4|30|11|3|7513|140|1574|67|66.97|108.49|19.52|5960.99|1307.84|4486.99|7268.83|26.15|0.00|1235.48|1307.84|1333.99|2543.32|2569.47|-3179.15| +2450845|59875|2450856|5794|1661137|1280|29823|5794|1661137|1280|29823|1|105|3|5|16327|186|1575|95|71.37|139.88|95.11|4253.15|9035.45|6780.15|13288.60|451.77|0.00|4916.25|9035.45|9487.22|13951.70|14403.47|2255.30| +2450845|59875|2450868|5794|1661137|1280|29823|5794|1661137|1280|29823|1|81|12|2|10040|129|1575|73|4.81|5.48|1.64|280.32|119.72|351.13|400.04|9.57|0.00|35.77|119.72|129.29|155.49|165.06|-231.41| +2450845|59875|2450879|5794|1661137|1280|29823|5794|1661137|1280|29823|1|11|13|5|3298|26|1575|93|16.90|48.33|30.44|1663.77|2830.92|1571.70|4494.69|113.23|0.00|403.62|2830.92|2944.15|3234.54|3347.77|1259.22| +2450845|59875|2450866|5794|1661137|1280|29823|5794|1661137|1280|29823|1|104|19|1|1364|75|1575|66|35.83|49.08|6.87|2785.86|453.42|2364.78|3239.28|27.20|0.00|906.84|453.42|480.62|1360.26|1387.46|-1911.36| +2450845|59875|2450861|5794|1661137|1280|29823|5794|1661137|1280|29823|1|24|14|3|10496|106|1575|28|38.20|92.06|77.33|412.44|2165.24|1069.60|2577.68|21.65|0.00|953.68|2165.24|2186.89|3118.92|3140.57|1095.64| +2450845|59875|2450909|5794|1661137|1280|29823|5794|1661137|1280|29823|1|75|2|1|10468|197|1575|64|33.56|95.31|15.24|5124.48|975.36|2147.84|6099.84|0.00|302.36|2195.84|673.00|673.00|2868.84|2868.84|-1474.84| +2450845|75552|2450847|87414|1185831|7001|33018|90136|91852|5822|5039|1|4|5|4|10820|18|1576|33|67.69|82.58|42.11|1335.51|1389.63|2233.77|2725.14|83.37|0.00|1144.44|1389.63|1473.00|2534.07|2617.44|-844.14| +2450845|75552|2450883|87414|1185831|7001|33018|90136|91852|5822|5039|1|34|13|4|1804|241|1576|100|93.62|93.62|46.81|4681.00|4681.00|9362.00|9362.00|71.15|2902.22|842.00|1778.78|1849.93|2620.78|2691.93|-7583.22| +2450845|75552|2450921|87414|1185831|7001|33018|90136|91852|5822|5039|1|102|20|2|11248|54|1576|37|70.68|171.75|66.98|3876.49|2478.26|2615.16|6354.75|198.26|0.00|1715.69|2478.26|2676.52|4193.95|4392.21|-136.90| +2450845|75552|2450926|87414|1185831|7001|33018|90136|91852|5822|5039|1|72|18|2|9136|225|1576|93|67.75|170.05|127.53|3954.36|11860.29|6300.75|15814.65|830.22|0.00|2529.60|11860.29|12690.51|14389.89|15220.11|5559.54| +2450845|75552|2450912|87414|1185831|7001|33018|90136|91852|5822|5039|1|29|3|3|14422|48|1576|40|34.90|78.87|11.83|2681.60|473.20|1396.00|3154.80|0.00|0.00|1324.80|473.20|473.20|1798.00|1798.00|-922.80| +2450845|75552|2450868|87414|1185831|7001|33018|90136|91852|5822|5039|1|61|3|4|16447|284|1576|11|62.36|83.56|17.54|726.22|192.94|685.96|919.16|1.92|0.00|248.16|192.94|194.86|441.10|443.02|-493.02| +2450845|75552|2450902|87414|1185831|7001|33018|90136|91852|5822|5039|1|56|18|4|8293|149|1576|23|81.57|186.79|119.54|1546.75|2749.42|1876.11|4296.17|61.58|1979.58|2104.96|769.84|831.42|2874.80|2936.38|-1106.27| +2450845|75552|2450902|87414|1185831|7001|33018|90136|91852|5822|5039|1|42|17|2|11242|62|1576|70|71.04|83.11|4.15|5527.20|290.50|4972.80|5817.70|2.90|0.00|1279.60|290.50|293.40|1570.10|1573.00|-4682.30| +2450845|75552|2450877|87414|1185831|7001|33018|90136|91852|5822|5039|1|29|4|5|15523|129|1576|35|60.06|75.67|58.26|609.35|2039.10|2102.10|2648.45|163.12|0.00|1112.30|2039.10|2202.22|3151.40|3314.52|-63.00| +2450845|75552|2450895|87414|1185831|7001|33018|90136|91852|5822|5039|1|67|15|3|5665|231|1576|98|83.70|137.26|26.07|10896.62|2554.86|8202.60|13451.48|0.00|0.00|1882.58|2554.86|2554.86|4437.44|4437.44|-5647.74| +2450845|75552|2450927|87414|1185831|7001|33018|90136|91852|5822|5039|1|100|12|5|3476|274|1576|7|82.64|134.70|53.88|565.74|377.16|578.48|942.90|9.99|177.26|462.00|199.90|209.89|661.90|671.89|-378.58| +2450845|75552|2450868|87414|1185831|7001|33018|90136|91852|5822|5039|1|43|15|5|15074|111|1576|31|50.18|97.34|36.01|1901.23|1116.31|1555.58|3017.54|89.30|0.00|1116.31|1116.31|1205.61|2232.62|2321.92|-439.27| +2450845|59423|2450881|11524|1109028|6179|1683|11524|1109028|6179|1683|1|66|11|2|12616|60|1577|63|5.79|10.76|9.46|81.90|595.98|364.77|677.88|11.91|0.00|128.52|595.98|607.89|724.50|736.41|231.21| +2450845|59423|2450854|11524|1109028|6179|1683|11524|1109028|6179|1683|1|17|9|1|10738|160|1577|84|51.77|117.51|21.15|8094.24|1776.60|4348.68|9870.84|137.50|248.72|1480.08|1527.88|1665.38|3007.96|3145.46|-2820.80| +2450845|59423|2450928|11524|1109028|6179|1683|11524|1109028|6179|1683|1|27|13|5|5422|294|1577|32|84.92|211.45|42.29|5413.12|1353.28|2717.44|6766.40|21.92|622.50|2841.60|730.78|752.70|3572.38|3594.30|-1986.66| +2450845|59423|2450868|11524|1109028|6179|1683|11524|1109028|6179|1683|1|23|17|1|16862|121|1577|23|63.23|134.67|45.78|2044.47|1052.94|1454.29|3097.41|35.80|157.94|1083.99|895.00|930.80|1978.99|2014.79|-559.29| +2450845|59423|2450868|11524|1109028|6179|1683|11524|1109028|6179|1683|1|24|7|5|5516|73|1577|85|52.32|81.09|80.27|69.70|6822.95|4447.20|6892.65|614.06|0.00|2756.55|6822.95|7437.01|9579.50|10193.56|2375.75| +2450845|59423|2450866|11524|1109028|6179|1683|11524|1109028|6179|1683|1|101|5|4|13273|129|1577|74|79.84|83.03|34.04|3625.26|2518.96|5908.16|6144.22|0.00|0.00|2948.90|2518.96|2518.96|5467.86|5467.86|-3389.20| +2450845|59423|2450860|11524|1109028|6179|1683|11524|1109028|6179|1683|1|16|16|2|13369|53|1577|44|50.73|87.76|55.28|1429.12|2432.32|2232.12|3861.44|145.93|0.00|1042.36|2432.32|2578.25|3474.68|3620.61|200.20| +2450845|25550|2450880|60184|133163|2142|45921|67168|985879|5258|10237|4|45|12|2|16504|295|1578|25|52.77|64.37|46.34|450.75|1158.50|1319.25|1609.25|0.00|0.00|193.00|1158.50|1158.50|1351.50|1351.50|-160.75| +2450845|25550|2450900|60184|133163|2142|45921|67168|985879|5258|10237|4|108|12|3|15319|72|1578|29|16.16|27.63|7.18|593.05|208.22|468.64|801.27|2.08|0.00|224.17|208.22|210.30|432.39|434.47|-260.42| +2450845|25550|2450885|60184|133163|2142|45921|67168|985879|5258|10237|4|90|19|3|7058|137|1578|23|38.62|77.24|10.04|1545.60|230.92|888.26|1776.52|11.54|0.00|35.42|230.92|242.46|266.34|277.88|-657.34| +2450845|25550|2450912|60184|133163|2142|45921|67168|985879|5258|10237|4|59|1|5|4657|120|1578|50|37.84|40.11|4.01|1805.00|200.50|1892.00|2005.50|2.00|0.00|581.50|200.50|202.50|782.00|784.00|-1691.50| +2450845|49849|2450895|56433|1049598|2410|7622|56433|1049598|2410|7622|4|97|20|1|10274|13|1579|72|90.48|209.00|190.19|1354.32|13693.68|6514.56|15048.00|0.00|3423.42|150.48|10270.26|10270.26|10420.74|10420.74|3755.70| +2450845|49849|2450858|56433|1049598|2410|7622|56433|1049598|2410|7622|4|40|3|4|7102|174|1579|34|92.80|170.75|80.25|3077.00|2728.50|3155.20|5805.50|81.85|0.00|115.94|2728.50|2810.35|2844.44|2926.29|-426.70| +2450845|49849|2450877|56433|1049598|2410|7622|56433|1049598|2410|7622|4|100|5|1|9164|293|1579|17|71.30|75.57|16.62|1002.15|282.54|1212.10|1284.69|25.42|0.00|243.95|282.54|307.96|526.49|551.91|-929.56| +2450845|49849|2450860|56433|1049598|2410|7622|56433|1049598|2410|7622|4|79|8|3|7526|136|1579|29|80.32|179.11|157.61|623.50|4570.69|2329.28|5194.19|0.00|0.00|571.30|4570.69|4570.69|5141.99|5141.99|2241.41| +2450845|49849|2450877|56433|1049598|2410|7622|56433|1049598|2410|7622|4|55|9|3|3098|117|1579|72|3.47|7.25|7.17|5.76|516.24|249.84|522.00|5.16|0.00|109.44|516.24|521.40|625.68|630.84|266.40| +2450845|49849|2450856|56433|1049598|2410|7622|56433|1049598|2410|7622|4|102|17|5|1765|188|1579|1|4.16|7.19|3.23|3.96|3.23|4.16|7.19|0.03|0.22|0.35|3.01|3.04|3.36|3.39|-1.15| +2450845|49849|2450869|56433|1049598|2410|7622|56433|1049598|2410|7622|4|22|3|2|1321|285|1579|43|65.42|138.69|84.60|2325.87|3637.80|2813.06|5963.67|72.75|0.00|2981.62|3637.80|3710.55|6619.42|6692.17|824.74| +2450845|49849|2450861|56433|1049598|2410|7622|56433|1049598|2410|7622|4|60|10|1|11872|151|1579|22|98.36|204.58|104.33|2205.50|2295.26|2163.92|4500.76|68.85|0.00|2070.20|2295.26|2364.11|4365.46|4434.31|131.34| +2450845|49849|2450886|56433|1049598|2410|7622|56433|1049598|2410|7622|4|75|19|5|16322|177|1579|39|8.53|20.98|6.29|572.91|245.31|332.67|818.22|10.30|73.59|253.50|171.72|182.02|425.22|435.52|-160.95| +2450845|49849|2450927|56433|1049598|2410|7622|56433|1049598|2410|7622|4|87|20|1|4099|68|1579|63|58.06|154.43|74.12|5059.53|4669.56|3657.78|9729.09|186.78|0.00|2724.12|4669.56|4856.34|7393.68|7580.46|1011.78| +2450845|57326|2450933|36841|617205|3522|21086|36841|617205|3522|21086|4|14|14|1|15499|152|1580|10|49.86|88.75|35.50|532.50|355.00|498.60|887.50|28.40|0.00|390.50|355.00|383.40|745.50|773.90|-143.60| +2450845|57326|2450893|36841|617205|3522|21086|36841|617205|3522|21086|4|107|10|1|10474|35|1580|74|46.12|71.48|55.03|1217.30|4072.22|3412.88|5289.52|366.49|0.00|1850.74|4072.22|4438.71|5922.96|6289.45|659.34| +2450845|57326|2450927|36841|617205|3522|21086|36841|617205|3522|21086|4|49|11|4|4180|12|1580|32|65.47|178.07|44.51|4273.92|1424.32|2095.04|5698.24|14.24|0.00|2108.16|1424.32|1438.56|3532.48|3546.72|-670.72| +2450845|57326|2450875|36841|617205|3522|21086|36841|617205|3522|21086|4|43|4|1|1454|294|1580|65|89.66|232.21|97.52|8754.85|6338.80|5827.90|15093.65|0.00|0.00|5734.95|6338.80|6338.80|12073.75|12073.75|510.90| +2450845|57326|2450932|36841|617205|3522|21086|36841|617205|3522|21086|4|89|3|2|13654|280|1580|55|11.60|21.22|1.06|1108.80|58.30|638.00|1167.10|2.91|0.00|583.55|58.30|61.21|641.85|644.76|-579.70| +2450845|57326|2450877|36841|617205|3522|21086|36841|617205|3522|21086|4|3|9|1|2998|78|1580|71|78.05|216.97|8.67|14789.30|615.57|5541.55|15404.87|2.70|480.14|2926.62|135.43|138.13|3062.05|3064.75|-5406.12| +2450845|57326|2450884|36841|617205|3522|21086|36841|617205|3522|21086|4|39|5|5|11356|133|1580|84|57.36|73.99|11.83|5221.44|993.72|4818.24|6215.16|25.43|357.73|1926.12|635.99|661.42|2562.11|2587.54|-4182.25| +2450845|57326|2450933|36841|617205|3522|21086|36841|617205|3522|21086|4|87|11|2|17725|212|1580|33|19.75|25.87|8.79|563.64|290.07|651.75|853.71|18.10|63.81|110.88|226.26|244.36|337.14|355.24|-425.49| +2450845|57326|2450911|36841|617205|3522|21086|36841|617205|3522|21086|4|100|20|5|7276|100|1580|89|12.24|28.02|15.41|1122.29|1371.49|1089.36|2493.78|4.93|877.75|1071.56|493.74|498.67|1565.30|1570.23|-595.62| +2450845|57326|2450910|36841|617205|3522|21086|36841|617205|3522|21086|4|82|19|3|13706|74|1580|65|21.59|61.74|44.45|1123.85|2889.25|1403.35|4013.10|202.24|0.00|401.05|2889.25|3091.49|3290.30|3492.54|1485.90| +2450845|57326|2450902|36841|617205|3522|21086|36841|617205|3522|21086|4|16|2|4|6109|298|1580|58|84.44|146.08|90.56|3220.16|5252.48|4897.52|8472.64|210.09|0.00|1524.82|5252.48|5462.57|6777.30|6987.39|354.96| +2450845|57326|2450924|36841|617205|3522|21086|36841|617205|3522|21086|4|40|10|4|1261|189|1580|13|1.73|4.61|1.29|43.16|16.77|22.49|59.93|0.33|0.00|25.74|16.77|17.10|42.51|42.84|-5.72| +2450845|43355|2450934|20148|1317180|4259|1738|20148|1317180|4259|1738|2|82|10|3|1568|170|1581|56|21.09|36.27|2.17|1909.60|121.52|1181.04|2031.12|7.29|0.00|406.00|121.52|128.81|527.52|534.81|-1059.52| +2450845|43355|2450913|20148|1317180|4259|1738|20148|1317180|4259|1738|2|34|6|1|10885|246|1581|74|67.46|180.79|74.12|7893.58|5484.88|4992.04|13378.46|219.39|0.00|2809.04|5484.88|5704.27|8293.92|8513.31|492.84| +2450845|43355|2450872|20148|1317180|4259|1738|20148|1317180|4259|1738|2|37|4|4|17896|294|1581|79|94.75|248.24|126.60|9609.56|10001.40|7485.25|19610.96|97.01|300.04|7452.07|9701.36|9798.37|17153.43|17250.44|2216.11| +2450845|43355|2450912|20148|1317180|4259|1738|20148|1317180|4259|1738|2|66|6|4|11335|43|1581|17|24.63|66.50|29.92|621.86|508.64|418.71|1130.50|0.00|0.00|56.44|508.64|508.64|565.08|565.08|89.93| +2450845|43355|2450909|20148|1317180|4259|1738|20148|1317180|4259|1738|2|16|19|1|16648|65|1581|61|32.73|58.25|27.37|1883.68|1669.57|1996.53|3553.25|133.56|0.00|1279.17|1669.57|1803.13|2948.74|3082.30|-326.96| +2450845|43355|2450904|20148|1317180|4259|1738|20148|1317180|4259|1738|2|39|2|5|4987|244|1581|59|31.11|72.17|69.28|170.51|4087.52|1835.49|4258.03|161.86|2289.01|1362.31|1798.51|1960.37|3160.82|3322.68|-36.98| +2450845|43355|2450880|20148|1317180|4259|1738|20148|1317180|4259|1738|2|16|18|4|2048|238|1581|69|3.44|9.70|7.27|167.67|501.63|237.36|669.30|20.06|0.00|93.15|501.63|521.69|594.78|614.84|264.27| +2450845|37761|2450928|18731|820334|3537|34740|18731|820334|3537|34740|1|24|7|3|17788|88|1582|24|60.11|146.66|95.32|1232.16|2287.68|1442.64|3519.84|114.38|0.00|387.12|2287.68|2402.06|2674.80|2789.18|845.04| +2450845|37761|2450859|18731|820334|3537|34740|18731|820334|3537|34740|1|36|5|2|4603|122|1582|84|11.98|14.97|3.29|981.12|276.36|1006.32|1257.48|24.87|0.00|238.56|276.36|301.23|514.92|539.79|-729.96| +2450845|37761|2450916|18731|820334|3537|34740|18731|820334|3537|34740|1|88|8|4|6799|90|1582|23|87.81|224.79|4.49|5066.90|103.27|2019.63|5170.17|1.30|38.20|2584.97|65.07|66.37|2650.04|2651.34|-1954.56| +2450845|37761|2450899|18731|820334|3537|34740|18731|820334|3537|34740|1|41|18|5|6836|99|1582|11|65.91|168.72|45.55|1354.87|501.05|725.01|1855.92|35.07|0.00|37.07|501.05|536.12|538.12|573.19|-223.96| +2450845|37761|2450927|18731|820334|3537|34740|18731|820334|3537|34740|1|14|17|3|8941|78|1582|28|8.87|21.73|18.03|103.60|504.84|248.36|608.44|45.43|0.00|152.04|504.84|550.27|656.88|702.31|256.48| +2450845|37761|2450853|18731|820334|3537|34740|18731|820334|3537|34740|1|98|15|1|9554|28|1582|76|56.09|84.13|54.68|2238.20|4155.68|4262.84|6393.88|207.78|0.00|766.84|4155.68|4363.46|4922.52|5130.30|-107.16| +2450845|37761|2450899|18731|820334|3537|34740|18731|820334|3537|34740|1|99|19|3|7957|142|1582|33|61.57|181.63|98.08|2757.15|3236.64|2031.81|5993.79|258.93|0.00|2996.73|3236.64|3495.57|6233.37|6492.30|1204.83| +2450845|37761|2450881|18731|820334|3537|34740|18731|820334|3537|34740|1|33|9|2|4801|186|1582|51|22.20|55.27|16.02|2001.75|817.02|1132.20|2818.77|9.47|343.14|225.42|473.88|483.35|699.30|708.77|-658.32| +2450845|37761|2450892|18731|820334|3537|34740|18731|820334|3537|34740|1|11|2|2|7898|142|1582|70|84.59|110.81|24.37|6050.80|1705.90|5921.30|7756.70|0.00|0.00|232.40|1705.90|1705.90|1938.30|1938.30|-4215.40| +2450845|37761|2450853|18731|820334|3537|34740|18731|820334|3537|34740|1|39|9|5|8858|109|1582|29|99.56|101.55|77.17|707.02|2237.93|2887.24|2944.95|22.37|0.00|971.79|2237.93|2260.30|3209.72|3232.09|-649.31| +2450845|37761|2450854|18731|820334|3537|34740|18731|820334|3537|34740|1|26|5|1|1747|250|1582|39|75.58|97.49|29.24|2661.75|1140.36|2947.62|3802.11|91.22|0.00|1292.46|1140.36|1231.58|2432.82|2524.04|-1807.26| +2450845|37761|2450901|18731|820334|3537|34740|18731|820334|3537|34740|1|11|11|4|11798|35|1582|91|72.82|115.05|86.28|2618.07|7851.48|6626.62|10469.55|0.00|0.00|4815.72|7851.48|7851.48|12667.20|12667.20|1224.86| +2450845|37761|2450878|18731|820334|3537|34740|18731|820334|3537|34740|1|33|17|3|697|193|1582|32|12.53|31.70|30.43|40.64|973.76|400.96|1014.40|29.21|0.00|496.96|973.76|1002.97|1470.72|1499.93|572.80| +2450846|56284|2450876|32420|1638885|550|10968|32420|1638885|550|10968|4|30|3|2|1861|262|1583|38|6.37|15.41|1.38|533.14|52.44|242.06|585.58|1.57|0.00|34.96|52.44|54.01|87.40|88.97|-189.62| +2450846|56284|2450906|32420|1638885|550|10968|32420|1638885|550|10968|4|42|4|1|944|103|1583|20|3.26|7.04|5.35|33.80|107.00|65.20|140.80|5.35|0.00|50.60|107.00|112.35|157.60|162.95|41.80| +2450846|56284|2450874|32420|1638885|550|10968|32420|1638885|550|10968|4|22|6|5|5581|97|1583|83|56.10|118.37|71.02|3930.05|5894.66|4656.30|9824.71|294.73|0.00|294.65|5894.66|6189.39|6189.31|6484.04|1238.36| +2450846|56284|2450906|32420|1638885|550|10968|32420|1638885|550|10968|4|54|6|3|14252|241|1583|84|38.81|48.51|46.56|163.80|3911.04|3260.04|4074.84|0.00|2698.61|488.88|1212.43|1212.43|1701.31|1701.31|-2047.61| +2450846|56284|2450899|32420|1638885|550|10968|32420|1638885|550|10968|4|84|17|4|13276|170|1583|41|16.09|39.42|14.97|1002.45|613.77|659.69|1616.22|42.96|0.00|371.46|613.77|656.73|985.23|1028.19|-45.92| +2450846|56284|2450873|32420|1638885|550|10968|32420|1638885|550|10968|4|70|18|4|14410|39|1583|27|92.62|164.86|141.77|623.43|3827.79|2500.74|4451.22|267.94|0.00|2047.41|3827.79|4095.73|5875.20|6143.14|1327.05| +2450846|56284|2450852|32420|1638885|550|10968|32420|1638885|550|10968|4|56|6|5|14756|81|1583|98|35.50|37.27|13.78|2302.02|1350.44|3479.00|3652.46|67.52|0.00|108.78|1350.44|1417.96|1459.22|1526.74|-2128.56| +2450846|85145|2450897|15435|553733|4384|4235|15435|553733|4384|4235|1|51|4|3|13432|266|1584|25|56.09|143.59|103.38|1005.25|2584.50|1402.25|3589.75|1.03|2558.65|287.00|25.85|26.88|312.85|313.88|-1376.40| +2450846|85145|2450862|15435|553733|4384|4235|15435|553733|4384|4235|1|53|20|4|5602|220|1584|92|50.08|78.62|3.14|6944.16|288.88|4607.36|7233.04|20.22|0.00|1301.80|288.88|309.10|1590.68|1610.90|-4318.48| +2450846|85145|2450900|15435|553733|4384|4235|15435|553733|4384|4235|1|84|5|1|16186|265|1584|80|83.64|157.24|18.86|11070.40|1508.80|6691.20|12579.20|90.52|0.00|1257.60|1508.80|1599.32|2766.40|2856.92|-5182.40| +2450846|85145|2450932|15435|553733|4384|4235|15435|553733|4384|4235|1|37|9|1|11302|178|1584|37|44.63|116.93|86.52|1125.17|3201.24|1651.31|4326.41|256.09|0.00|778.48|3201.24|3457.33|3979.72|4235.81|1549.93| +2450846|85145|2450935|15435|553733|4384|4235|15435|553733|4384|4235|1|82|15|5|10087|110|1584|82|82.88|186.48|160.37|2141.02|13150.34|6796.16|15291.36|46.02|12492.82|3822.84|657.52|703.54|4480.36|4526.38|-6138.64| +2450846|85145|2450887|15435|553733|4384|4235|15435|553733|4384|4235|1|30|20|4|14698|85|1584|84|90.33|266.47|103.92|13654.20|8729.28|7587.72|22383.48|0.00|0.00|10072.44|8729.28|8729.28|18801.72|18801.72|1141.56| +2450846|85145|2450875|15435|553733|4384|4235|15435|553733|4384|4235|1|19|9|3|11185|298|1584|32|94.14|239.11|129.11|3520.00|4131.52|3012.48|7651.52|82.63|0.00|994.56|4131.52|4214.15|5126.08|5208.71|1119.04| +2450846|85145|2450848|15435|553733|4384|4235|15435|553733|4384|4235|1|71|12|2|14096|139|1584|86|74.80|174.28|137.68|3147.60|11840.48|6432.80|14988.08|355.21|0.00|4496.08|11840.48|12195.69|16336.56|16691.77|5407.68| +2450846|85145|2450921|15435|553733|4384|4235|15435|553733|4384|4235|1|36|6|2|11780|266|1584|84|14.55|26.91|11.30|1311.24|949.20|1222.20|2260.44|37.96|0.00|1061.76|949.20|987.16|2010.96|2048.92|-273.00| +2450846|85145|2450901|15435|553733|4384|4235|15435|553733|4384|4235|1|66|8|4|584|278|1584|59|69.52|131.39|119.56|697.97|7054.04|4101.68|7752.01|564.32|0.00|1395.35|7054.04|7618.36|8449.39|9013.71|2952.36| +2450846|85145|2450880|15435|553733|4384|4235|15435|553733|4384|4235|1|33|10|3|12740|292|1584|64|25.22|39.59|2.37|2382.08|151.68|1614.08|2533.76|9.10|0.00|455.68|151.68|160.78|607.36|616.46|-1462.40| +2450846|65985|2450895|95983|96005|4474|143|95983|96005|4474|143|4|8|19|3|15646|206|1585|73|3.93|10.33|1.54|641.67|112.42|286.89|754.09|3.37|0.00|29.93|112.42|115.79|142.35|145.72|-174.47| +2450846|65985|2450874|95983|96005|4474|143|95983|96005|4474|143|4|18|9|2|14198|171|1585|46|7.25|7.39|0.44|319.70|20.24|333.50|339.94|0.22|17.40|37.26|2.84|3.06|40.10|40.32|-330.66| +2450846|65985|2450856|95983|96005|4474|143|95983|96005|4474|143|4|91|3|3|4624|58|1585|58|41.88|107.63|4.30|5993.14|249.40|2429.04|6242.54|12.47|0.00|186.76|249.40|261.87|436.16|448.63|-2179.64| +2450846|65985|2450885|95983|96005|4474|143|95983|96005|4474|143|4|82|11|2|13609|94|1585|82|45.66|89.95|16.19|6048.32|1327.58|3744.12|7375.90|13.27|0.00|2802.76|1327.58|1340.85|4130.34|4143.61|-2416.54| +2450846|65985|2450936|95983|96005|4474|143|95983|96005|4474|143|4|53|5|4|11011|24|1585|29|83.88|124.98|104.98|580.00|3044.42|2432.52|3624.42|273.99|0.00|1594.71|3044.42|3318.41|4639.13|4913.12|611.90| +2450846|65985|2450888|95983|96005|4474|143|95983|96005|4474|143|4|13|6|2|13822|209|1585|44|67.82|105.12|56.76|2127.84|2497.44|2984.08|4625.28|199.79|0.00|2266.00|2497.44|2697.23|4763.44|4963.23|-486.64| +2450846|79169|2450883|83803|1556734|4512|47090|83803|1556734|4512|47090|1|63|10|2|15520|247|1586|61|88.18|208.98|167.18|2549.80|10197.98|5378.98|12747.78|203.95|0.00|2039.23|10197.98|10401.93|12237.21|12441.16|4819.00| +2450846|79169|2450921|83803|1556734|4512|47090|83803|1556734|4512|47090|1|21|17|1|13810|184|1586|16|43.83|106.94|69.51|598.88|1112.16|701.28|1711.04|20.90|66.72|376.32|1045.44|1066.34|1421.76|1442.66|344.16| +2450846|79169|2450913|83803|1556734|4512|47090|83803|1556734|4512|47090|1|43|4|3|10544|258|1586|90|33.89|61.67|8.63|4773.60|776.70|3050.10|5550.30|30.44|15.53|2718.90|761.17|791.61|3480.07|3510.51|-2288.93| +2450846|79169|2450865|83803|1556734|4512|47090|83803|1556734|4512|47090|1|89|9|3|4396|290|1586|54|83.98|251.94|17.63|12652.74|952.02|4534.92|13604.76|19.04|0.00|2856.60|952.02|971.06|3808.62|3827.66|-3582.90| +2450846|79169|2450851|83803|1556734|4512|47090|83803|1556734|4512|47090|1|83|9|5|12460|140|1586|70|42.04|77.77|76.99|54.60|5389.30|2942.80|5443.90|0.00|0.00|0.00|5389.30|5389.30|5389.30|5389.30|2446.50| +2450846|79169|2450904|83803|1556734|4512|47090|83803|1556734|4512|47090|1|43|1|1|14240|291|1586|95|9.15|22.60|17.40|494.00|1653.00|869.25|2147.00|16.53|0.00|751.45|1653.00|1669.53|2404.45|2420.98|783.75| +2450846|79169|2450916|83803|1556734|4512|47090|83803|1556734|4512|47090|1|87|10|2|15032|256|1586|89|39.70|52.00|25.48|2360.28|2267.72|3533.30|4628.00|68.03|0.00|46.28|2267.72|2335.75|2314.00|2382.03|-1265.58| +2450846|61260|2450885|38762|1268612|2462|5865|38762|1268612|2462|5865|2|91|1|4|3331|168|1587|55|98.31|103.22|102.18|57.20|5619.90|5407.05|5677.10|393.39|0.00|397.10|5619.90|6013.29|6017.00|6410.39|212.85| +2450846|61260|2450909|38762|1268612|2462|5865|38762|1268612|2462|5865|2|38|14|2|4408|33|1587|13|55.34|65.30|35.26|390.52|458.38|719.42|848.90|0.73|440.04|178.23|18.34|19.07|196.57|197.30|-701.08| +2450846|61260|2450860||||5865||||5865|2|16|||13831|261|1587|68|95.31||||3576.80|6481.08|7452.80|||2831.52|3576.80||6408.32|6408.32|-2904.28| +2450846|61260|2450857|38762|1268612|2462|5865|38762|1268612|2462|5865|2|5|2|2|9248|265|1587|36|5.66|7.30|4.74|92.16|170.64|203.76|262.80|0.00|0.00|75.96|170.64|170.64|246.60|246.60|-33.12| +2450846|61260|2450878|38762|1268612|2462|5865|38762|1268612|2462|5865|2|62|15|5|15668|87|1587|79|30.10|64.11|57.05|557.74|4506.95|2377.90|5064.69|84.73|2388.68|1670.85|2118.27|2203.00|3789.12|3873.85|-259.63| +2450846|61260|2450898|38762|1268612|2462|5865|38762|1268612|2462|5865|2|12|1|1|15992|29|1587|80|39.65|77.31|29.37|3835.20|2349.60|3172.00|6184.80|72.36|540.40|2412.00|1809.20|1881.56|4221.20|4293.56|-1362.80| +2450846|61260|2450886|38762|1268612|2462|5865|38762|1268612|2462|5865|2|5|3|4|12248|66|1587|37|74.42|77.39|39.46|1403.41|1460.02|2753.54|2863.43|45.26|554.80|56.98|905.22|950.48|962.20|1007.46|-1848.32| +2450846|61260|2450878|38762|1268612|2462|5865|38762|1268612|2462|5865|2|10|14|2|8770|259|1587|14|54.02|160.43|113.90|651.42|1594.60|756.28|2246.02|79.73|0.00|875.84|1594.60|1674.33|2470.44|2550.17|838.32| +2450846|61260|2450868|38762|1268612|2462|5865|38762|1268612|2462|5865|2|39|15|4|13858|295|1587|15|25.84|60.72|41.28|291.60|619.20|387.60|910.80|0.00|0.00|163.80|619.20|619.20|783.00|783.00|231.60| +2450846|61260|2450933|38762|1268612|2462|5865|38762|1268612|2462|5865|2|108|8|5|6235|42|1587|83|77.28|139.87|18.18|10100.27|1508.94|6414.24|11609.21|75.44|0.00|3017.88|1508.94|1584.38|4526.82|4602.26|-4905.30| +2450846|61260|2450890|38762|1268612|2462|5865|38762|1268612|2462|5865|2|7|17|2|8707|217|1587|88|46.89|103.15|83.55|1724.80|7352.40|4126.32|9077.20|588.19|0.00|2359.28|7352.40|7940.59|9711.68|10299.87|3226.08| +2450846|37423|2450906|99995|330678|5714|6397|24098|1800963|2846|43129|1|92|6|2|3674|34|1588|33|65.18|102.33|91.07|371.58|3005.31|2150.94|3376.89|7.81|2223.92|100.98|781.39|789.20|882.37|890.18|-1369.55| +2450846|37423|2450927|99995|330678|5714|6397|24098|1800963|2846|43129|1|33|14|5|1381|273|1588|32|1.67|4.60|0.73|123.84|23.36|53.44|147.20|0.70|0.00|42.56|23.36|24.06|65.92|66.62|-30.08| +2450846|37423|2450861|99995|330678|5714|6397|24098|1800963|2846|43129|1|74|5|5|7909|173|1588|4|9.05|18.19|17.82|1.48|71.28|36.20|72.76|5.70|0.00|21.08|71.28|76.98|92.36|98.06|35.08| +2450846|37423|2450856|99995|330678|5714|6397|24098|1800963|2846|43129|1|36|1|3|13843|135|1588|91|49.20|73.30|43.98|2668.12|4002.18|4477.20|6670.30|280.15|0.00|1867.32|4002.18|4282.33|5869.50|6149.65|-475.02| +2450846|37423|2450880|99995|330678|5714|6397|24098|1800963|2846|43129|1|93|7|2|5911|285|1588|29|25.97|59.47|23.78|1035.01|689.62|753.13|1724.63|0.00|0.00|344.81|689.62|689.62|1034.43|1034.43|-63.51| +2450846|37423|2450851|99995|330678|5714|6397|24098|1800963|2846|43129|1|78|5|4|12544|170|1588|12|17.58|36.39|1.09|423.60|13.08|210.96|436.68|0.52|0.00|170.28|13.08|13.60|183.36|183.88|-197.88| +2450846|37423|2450884|99995|330678|5714|6397|24098|1800963|2846|43129|1|104|12|4|16868|173|1588|21|97.09|157.28|108.52|1023.96|2278.92|2038.89|3302.88|45.57|0.00|693.42|2278.92|2324.49|2972.34|3017.91|240.03| +2450846|55005|2450852|84599|1749148|544|17940|84599|1749148|544|17940|1|42|7|2|3980|128|1589|99|1.30|3.47|0.55|289.08|54.45|128.70|343.53|3.81|0.00|122.76|54.45|58.26|177.21|181.02|-74.25| +2450846|55005|2450876|84599|1749148|544|17940|84599|1749148|544|17940|1|18|17|3|12349|173|1589|78|45.04|105.39|36.88|5343.78|2876.64|3513.12|8220.42|27.04|1524.61|2301.00|1352.03|1379.07|3653.03|3680.07|-2161.09| +2450846|55005|2450877|84599|1749148|544|17940|84599|1749148|544|17940|1|39|15|5|4172|27|1589|24|78.65|152.58|143.42|219.84|3442.08|1887.60|3661.92|34.42|0.00|842.16|3442.08|3476.50|4284.24|4318.66|1554.48| +2450846|55005|2450933|84599|1749148|544|17940|84599|1749148|544|17940|1|8|10|1|2854|128|1589|19|44.39|124.29|116.83|141.74|2219.77|843.41|2361.51|44.39|0.00|755.63|2219.77|2264.16|2975.40|3019.79|1376.36| +2450846|55005|2450927|84599|1749148|544|17940|84599|1749148|544|17940|1|30|1|4|2566|82|1589|82|45.85|113.24|19.25|7707.18|1578.50|3759.70|9285.68|1.10|1468.00|3157.00|110.50|111.60|3267.50|3268.60|-3649.20| +2450846|55005|2450861|84599|1749148|544|17940|84599|1749148|544|17940|1|41|14|5|3610|137|1589|85|32.22|56.38|19.16|3163.70|1628.60|2738.70|4792.30|16.28|0.00|2396.15|1628.60|1644.88|4024.75|4041.03|-1110.10| +2450846|55005|2450877|84599|1749148|544|17940|84599|1749148|544|17940|1|98|13|1|2500|271|1589|64|21.16|35.97|24.09|760.32|1541.76|1354.24|2302.08|107.92|0.00|437.12|1541.76|1649.68|1978.88|2086.80|187.52| +2450846|55005|2450931|84599|1749148|544|17940|84599|1749148|544|17940|1|6|17|1|6650|59|1589|16|38.18|53.07|3.18|798.24|50.88|610.88|849.12|0.00|0.00|195.20|50.88|50.88|246.08|246.08|-560.00| +2450846|55005|2450919|84599|1749148|544|17940|84599|1749148|544|17940|1|92|18|4|5374|298|1589|28|18.48|52.29|8.36|1230.04|234.08|517.44|1464.12|9.83|93.63|585.48|140.45|150.28|725.93|735.76|-376.99| +2450846|55005|2450852|84599|1749148|544|17940|84599|1749148|544|17940|1|72|16|1|16270|51|1589|57|46.38|47.77|43.94|218.31|2504.58|2643.66|2722.89|125.22|0.00|54.15|2504.58|2629.80|2558.73|2683.95|-139.08| +2450846|55005|2450858|84599|1749148|544|17940|84599|1749148|544|17940|1|65|14|4|15556|42|1589|65|9.96|28.88|18.19|694.85|1182.35|647.40|1877.20|47.29|0.00|431.60|1182.35|1229.64|1613.95|1661.24|534.95| +2450846|55005|2450900|84599|1749148|544|17940|84599|1749148|544|17940|1|81|2|4|16214|60|1589|23|53.24|68.14|36.79|721.05|846.17|1224.52|1567.22|59.23|0.00|658.03|846.17|905.40|1504.20|1563.43|-378.35| +2450846|45378|2450874|40584|526729|6474|38219|40584|526729|6474|38219|2|59|8|1|13930|12|1590|3|36.71|59.83|0.00|179.49|0.00|110.13|179.49|0.00|0.00|3.57|0.00|0.00|3.57|3.57|-110.13| +2450846|45378|2450923|40584|526729|6474|38219|40584|526729|6474|38219|2|33|16|4|586|295|1590|69|76.61|225.99|160.45|4522.26|11071.05|5286.09|15593.31|664.26|0.00|6236.91|11071.05|11735.31|17307.96|17972.22|5784.96| +2450846|45378|2450873|40584|526729|6474|38219|40584|526729|6474|38219|2|23|17|2|17317|112|1590|77|37.00|61.05|30.52|2350.81|2350.04|2849.00|4700.85|47.00|0.00|516.67|2350.04|2397.04|2866.71|2913.71|-498.96| +2450846|45378|2450929|40584|526729|6474|38219|40584|526729|6474|38219|2|68|7|3|1837|237|1590|2|53.29|53.29|3.73|99.12|7.46|106.58|106.58|0.22|0.00|18.10|7.46|7.68|25.56|25.78|-99.12| +2450846|45378|2450859|40584|526729|6474|38219|40584|526729|6474|38219|2|41|15|4|1762|299|1590|77|13.14|34.29|5.14|2244.55|395.78|1011.78|2640.33|19.78|0.00|924.00|395.78|415.56|1319.78|1339.56|-616.00| +2450846|45378|2450857|40584|526729|6474|38219|40584|526729|6474|38219|2|73|1|5|11860|256|1590|43|38.92|48.26|34.74|581.36|1493.82|1673.56|2075.18|119.50|0.00|456.23|1493.82|1613.32|1950.05|2069.55|-179.74| +2450846|45378|2450869|40584|526729|6474|38219|40584|526729|6474|38219|2|34|5|5|16462|209|1590|64|89.62|95.89|85.34|675.20|5461.76|5735.68|6136.96|491.55|0.00|613.12|5461.76|5953.31|6074.88|6566.43|-273.92| +2450846|37612|2450933|53669|55016|6410|40557|53669|55016|6410|40557|2|7|18|4|5305|240|1591|51|9.89|27.09|2.16|1271.43|110.16|504.39|1381.59|9.91|0.00|207.06|110.16|120.07|317.22|327.13|-394.23| +2450846|37612|2450893|53669|55016|6410|40557|53669|55016|6410|40557|2|73|11|2|12637|225|1591|84|71.41|107.11|40.70|5578.44|3418.80|5998.44|8997.24|136.75|0.00|899.64|3418.80|3555.55|4318.44|4455.19|-2579.64| +2450846|37612|2450877|53669|55016|6410|40557|53669|55016|6410|40557|2|84|7|2|5707|128|1591|74|81.41|132.69|26.53|7855.84|1963.22|6024.34|9819.06|19.63|0.00|1668.70|1963.22|1982.85|3631.92|3651.55|-4061.12| +2450846|37612|2450901|53669|55016|6410|40557|53669|55016|6410|40557|2|24|5|2|9595|115|1591|53|94.96|194.66|3.89|10110.81|206.17|5032.88|10316.98|14.43|0.00|3610.89|206.17|220.60|3817.06|3831.49|-4826.71| +2450846|37612|2450863|53669|55016|6410|40557|53669|55016|6410|40557|2|7|6|3|12488|162|1591|82|74.08|108.15|48.66|4878.18|3990.12|6074.56|8868.30|119.70|0.00|177.12|3990.12|4109.82|4167.24|4286.94|-2084.44| +2450846|3810|2450893|34839|639640|5594|13195|34839|639640|5594|13195|1|56|7|1|16327|74|1592|59|84.38|156.10|3.12|9025.82|184.08|4978.42|9209.90|14.72|0.00|2026.06|184.08|198.80|2210.14|2224.86|-4794.34| +2450846|3810|2450854|34839|639640|5594|13195|34839|639640|5594|13195|1|32|18|5|10040|108|1592|21|24.16|42.76|23.09|413.07|484.89|507.36|897.96|3.39|436.40|296.31|48.49|51.88|344.80|348.19|-458.87| +2450846|3810|2450900|34839|639640|5594|13195|34839|639640|5594|13195|1|87|19|1|3298|199|1592|62|95.50|233.02|130.49|6356.86|8090.38|5921.00|14447.24|78.47|242.71|6645.16|7847.67|7926.14|14492.83|14571.30|1926.67| +2450846|3810|2450917|34839|639640|5594|13195|34839|639640|5594|13195|1|50|12|2|1364|95|1592|24|55.92|114.63|73.36|990.48|1760.64|1342.08|2751.12|158.45|0.00|1045.20|1760.64|1919.09|2805.84|2964.29|418.56| +2450846|3810|2450886|34839|639640|5594|13195|34839|639640|5594|13195|1|39|8|4|10496|134|1592|33|36.93|77.92|7.01|2340.03|231.33|1218.69|2571.36|13.87|0.00|1131.24|231.33|245.20|1362.57|1376.44|-987.36| +2450846|3810|2450848|34839|639640|5594|13195|34839|639640|5594|13195|1|28|13|5|10468|37|1592|87|67.28|94.19|90.42|327.99|7866.54|5853.36|8194.53|278.47|3225.28|1719.99|4641.26|4919.73|6361.25|6639.72|-1212.10| +2450846|3810|2450880|34839|639640|5594|13195|34839|639640|5594|13195|1|45|6|4|12398|247|1592|49|60.83|147.81|51.73|4707.92|2534.77|2980.67|7242.69|35.48|760.43|289.59|1774.34|1809.82|2063.93|2099.41|-1206.33| +2450846|3810|2450924|34839|639640|5594|13195|34839|639640|5594|13195|1|101|14|2|14005|23|1592|97|97.47|104.29|45.88|5665.77|4450.36|9454.59|10116.13|264.35|44.50|3641.38|4405.86|4670.21|8047.24|8311.59|-5048.73| +2450846|3810|2450876|34839|639640|5594|13195|34839|639640|5594|13195|1|68|5|4|15469|269|1592|8|23.90|54.25|33.09|169.28|264.72|191.20|434.00|10.58|0.00|117.12|264.72|275.30|381.84|392.42|73.52| +2450846|3810|2450926|34839|639640|5594|13195|34839|639640|5594|13195|1|50|1|2|7300|21|1592|41|56.09|103.20|16.51|3554.29|676.91|2299.69|4231.20|9.20|446.76|1015.16|230.15|239.35|1245.31|1254.51|-2069.54| +2450846|3810|2450856|34839|639640|5594|13195|34839|639640|5594|13195|1|104|10|2|6583|99|1592|1|67.58|148.00|103.60|44.40|103.60|67.58|148.00|5.18|0.00|44.40|103.60|108.78|148.00|153.18|36.02| +2450846|3810|2450857|34839|639640|5594|13195|34839|639640|5594|13195|1|44|9|5|1063|95|1592|54|12.64|23.63|23.63|0.00|1276.02|682.56|1276.02|114.84|0.00|331.56|1276.02|1390.86|1607.58|1722.42|593.46| +2450846|3810|2450923|34839|639640|5594|13195|34839|639640|5594|13195|1|43|10|2|15868|160|1592|58|90.32|252.89|108.74|8360.70|6306.92|5238.56|14667.62|252.27|0.00|5426.48|6306.92|6559.19|11733.40|11985.67|1068.36| +2450846|74555|2450870|86651|737768|3638|4755|86651|737768|3638|4755|4|25|5|2|8329|38|1593|12|45.40|113.04|57.65|664.68|691.80|544.80|1356.48|6.91|0.00|610.32|691.80|698.71|1302.12|1309.03|147.00| +2450846|74555|2450931|86651|737768|3638|4755|86651|737768|3638|4755|4|69|1|3|13903|63|1593|99|82.56|216.30|160.06|5567.76|15845.94|8173.44|21413.70|158.45|0.00|4067.91|15845.94|16004.39|19913.85|20072.30|7672.50| +2450846|74555|2450874|86651|737768|3638|4755|86651|737768|3638|4755|4|50|14|4|1285|101|1593|69|18.25|52.74|24.26|1965.12|1673.94|1259.25|3639.06|83.69|0.00|254.61|1673.94|1757.63|1928.55|2012.24|414.69| +2450846|74555|2450879|86651|737768|3638|4755|86651|737768|3638|4755|4|24|7|3|6428|49|1593|95|32.84|34.15|8.87|2401.60|842.65|3119.80|3244.25|42.13|0.00|1459.20|842.65|884.78|2301.85|2343.98|-2277.15| +2450846|74555|2450914|86651|737768|3638|4755|86651|737768|3638|4755|4|37|18|5|10196|28|1593|21|1.99|4.43|4.25|3.78|89.25|41.79|93.03|5.35|0.00|10.08|89.25|94.60|99.33|104.68|47.46| +2450846|58074|2450906|37558|100530|1455|25980|37558|100530|1455|25980|1|31|17|1|3121|197|1594|30|71.22|172.35|151.66|620.70|4549.80|2136.60|5170.50|136.49|0.00|982.20|4549.80|4686.29|5532.00|5668.49|2413.20| +2450846|58074|2450919|37558|100530|1455|25980|37558|100530|1455|25980|1|29|7|3|16906|20|1594|1|60.09|126.18|98.42|27.76|98.42|60.09|126.18|1.39|28.54|61.82|69.88|71.27|131.70|133.09|9.79| +2450846|58074|2450890|37558|100530|1455|25980|37558|100530|1455|25980|1|11|1|4|16354|118|1594|63|95.95|232.19|143.95|5559.12|9068.85|6044.85|14627.97|634.81|0.00|2193.66|9068.85|9703.66|11262.51|11897.32|3024.00| +2450846|58074|2450901|37558|100530|1455|25980|37558|100530|1455|25980|1|13|8|4|5948|1|1594|7|6.75|8.03|1.92|42.77|13.44|47.25|56.21|0.94|0.00|17.36|13.44|14.38|30.80|31.74|-33.81| +2450846|58074|2450869|37558|100530|1455|25980|37558|100530|1455|25980|1|30|5|2|7976|156|1594|44|85.15|104.73|60.74|1935.56|2672.56|3746.60|4608.12|26.72|0.00|1705.00|2672.56|2699.28|4377.56|4404.28|-1074.04| +2450846|58074|2450916|37558|100530|1455|25980|37558|100530|1455|25980|1|88|19|3|15205|202|1594|90|3.91|4.06|1.46|234.00|131.40|351.90|365.40|3.94|0.00|65.70|131.40|135.34|197.10|201.04|-220.50| +2450846|58074|2450884|37558|100530|1455|25980|37558|100530|1455|25980|1|13|1|2|8965|28|1594|71|41.72|108.47|3.25|7470.62|230.75|2962.12|7701.37|13.84|0.00|769.64|230.75|244.59|1000.39|1014.23|-2731.37| +2450846|58074|2450864|37558|100530|1455|25980|37558|100530|1455|25980|1|51|15|1|10216|149|1594|73|69.37|156.08|76.47|5811.53|5582.31|5064.01|11393.84|390.76|0.00|2050.57|5582.31|5973.07|7632.88|8023.64|518.30| +2450846|24274|2450860|2034|1834402|1170|19938|2034|1834402|1170|19938|2|2|20|5|3638|168|1595|67|20.38|41.98|12.59|1969.13|843.53|1365.46|2812.66|0.00|843.53|899.81|0.00|0.00|899.81|899.81|-1365.46| +2450846|24274|2450934|2034|1834402|1170|19938|2034|1834402|1170|19938|2|61|16|3|5188|115|1595|87|24.97|70.41|41.54|2511.69|3613.98|2172.39|6125.67|180.69|0.00|1469.43|3613.98|3794.67|5083.41|5264.10|1441.59| +2450846|24274|2450884|2034|1834402|1170|19938|2034|1834402|1170|19938|2|28|7|5|1786|148|1595|89|6.01|9.07|8.25|72.98|734.25|534.89|807.23|2.49|484.60|387.15|249.65|252.14|636.80|639.29|-285.24| +2450846|24274|2450910|2034|1834402|1170|19938|2034|1834402|1170|19938|2|98|9|3|17254|117|1595|97|68.66|72.09|67.76|420.01|6572.72|6660.02|6992.73|131.45|0.00|69.84|6572.72|6704.17|6642.56|6774.01|-87.30| +2450846|24274|2450893|2034|1834402|1170|19938|2034|1834402|1170|19938|2|4|8|2|1774|2|1595|31|22.41|31.37|25.40|185.07|787.40|694.71|972.47|55.11|0.00|447.33|787.40|842.51|1234.73|1289.84|92.69| +2450846|68620|2450929|41060|654707|4001|41295|41060|654707|4001|41295|1|95|6|4|13897|287|1596|46|81.66|96.35|51.06|2083.34|2348.76|3756.36|4432.10|117.43|0.00|177.10|2348.76|2466.19|2525.86|2643.29|-1407.60| +2450846|68620|2450856|41060|654707|4001|41295|41060|654707|4001|41295|1|57|8|5|770|125|1596|1|98.64|251.53|67.91|183.62|67.91|98.64|251.53|2.81|11.54|72.94|56.37|59.18|129.31|132.12|-42.27| +2450846|68620|2450875|41060|654707|4001|41295|41060|654707|4001|41295|1|32|7|1|12568|224|1596|44|60.80|181.79|181.79|0.00|7998.76|2675.20|7998.76|399.93|0.00|2079.44|7998.76|8398.69|10078.20|10478.13|5323.56| +2450846|68620|2450859|41060|654707|4001|41295|41060|654707|4001|41295|1|63|19|2|16000|241|1596|69|8.61|14.63|11.70|202.17|807.30|594.09|1009.47|0.00|0.00|413.31|807.30|807.30|1220.61|1220.61|213.21| +2450846|68620|2450895|41060|654707|4001|41295|41060|654707|4001|41295|1|67|2|2|8120|253|1596|53|12.99|31.17|15.27|842.70|809.31|688.47|1652.01|8.09|0.00|65.72|809.31|817.40|875.03|883.12|120.84| +2450846|68620|2450908|41060|654707|4001|41295|41060|654707|4001|41295|1|15|9|1|4948|60|1596|63|63.32|133.60|20.04|7154.28|1262.52|3989.16|8416.80|8.58|404.00|3955.77|858.52|867.10|4814.29|4822.87|-3130.64| +2450846|68620|2450931|41060|654707|4001|41295|41060|654707|4001|41295|1|5|19|3|1621|159|1596|39|35.39|87.41|13.98|2863.77|545.22|1380.21|3408.99|27.26|0.00|579.15|545.22|572.48|1124.37|1151.63|-834.99| +2450846|68620|2450876|41060|654707|4001|41295|41060|654707|4001|41295|1|77|8|3|17323|69|1596|23|46.43|102.61|76.95|590.18|1769.85|1067.89|2360.03|35.39|0.00|873.08|1769.85|1805.24|2642.93|2678.32|701.96| +2450846|68620|2450862|41060|654707|4001|41295|41060|654707|4001|41295|1|85|15|4|1393|137|1596|55|31.68|49.42|27.67|1196.25|1521.85|1742.40|2718.10|0.00|1506.63|1141.25|15.22|15.22|1156.47|1156.47|-1727.18| +2450846|68620|2450879|41060|654707|4001|41295|41060|654707|4001|41295|1|41|8|5|11143|298|1596|66|15.86|40.44|27.49|854.70|1814.34|1046.76|2669.04|0.00|0.00|933.90|1814.34|1814.34|2748.24|2748.24|767.58| +2450846|68620|2450850|41060|654707|4001|41295|41060|654707|4001|41295|1|62|20|3|4420|29|1596|48|80.52|199.68|99.84|4792.32|4792.32|3864.96|9584.64|335.46|0.00|2587.68|4792.32|5127.78|7380.00|7715.46|927.36| +2450846|68620|2450905|41060|654707|4001|41295|41060|654707|4001|41295|1|104|11|5|2884|238|1596|56|24.85|48.20|41.45|378.00|2321.20|1391.60|2699.20|48.74|1346.29|1025.36|974.91|1023.65|2000.27|2049.01|-416.69| +2450846|68620|2450873|41060|654707|4001|41295|41060|654707|4001|41295|1|70|18|4|2086|235|1596|73|76.63|205.36|156.07|3598.17|11393.11|5593.99|14991.28|79.75|7405.52|1648.34|3987.59|4067.34|5635.93|5715.68|-1606.40| +2450846|55848|2450917|29285|574614|6696|47225|29285|574614|6696|47225|1|42|9|1|7742|145|1597|16|14.61|34.77|11.47|372.80|183.52|233.76|556.32|1.83|0.00|122.24|183.52|185.35|305.76|307.59|-50.24| +2450846|55848|2450878|29285|574614|6696|47225|29285|574614|6696|47225|1|13|9|3|15985|59|1597|59|62.83|118.74|35.62|4904.08|2101.58|3706.97|7005.66|189.14|0.00|2311.62|2101.58|2290.72|4413.20|4602.34|-1605.39| +2450846|55848|2450870|29285|574614|6696|47225|29285|574614|6696|47225|1|44|11|2|9967|118|1597|82|53.79|145.23|59.54|7026.58|4882.28|4410.78|11908.86|439.40|0.00|5835.12|4882.28|5321.68|10717.40|11156.80|471.50| +2450846|55848|2450890|29285|574614|6696|47225|29285|574614|6696|47225|1|58|16|1|4124|153|1597|83|34.22|70.49|21.85|4037.12|1813.55|2840.26|5850.67|108.81|0.00|1988.68|1813.55|1922.36|3802.23|3911.04|-1026.71| +2450846|55848|2450870|29285|574614|6696|47225|29285|574614|6696|47225|1|71|15|4|1225|272|1597|94|70.04|107.86|56.08|4867.32|5271.52|6583.76|10138.84|0.00|0.00|202.10|5271.52|5271.52|5473.62|5473.62|-1312.24| +2450846|55848|2450874|29285|574614|6696|47225|29285|574614|6696|47225|1|108|9|3|8848|189|1597|47|13.56|22.64|5.20|819.68|244.40|637.32|1064.08|19.55|0.00|234.06|244.40|263.95|478.46|498.01|-392.92| +|||29285|574614|6696||29285||6696||1||3||12968|98|1597|66||137.37||272.58|8793.84||9066.42||0.00|452.76|||9246.60|9334.53|2363.46| +2450846|55848|2450875|29285|574614|6696|47225|29285|574614|6696|47225|1|99|16|2|8600|251|1597|85|27.53|45.69|6.85|3301.40|582.25|2340.05|3883.65|0.00|0.00|1203.60|582.25|582.25|1785.85|1785.85|-1757.80| +2450846|55848|2450879|29285|574614|6696|47225|29285|574614|6696|47225|1|101|14|4|6166|4|1597|56|97.60|146.40|133.22|738.08|7460.32|5465.60|8198.40|298.41|0.00|2787.12|7460.32|7758.73|10247.44|10545.85|1994.72| +2450846|55848|2450867|29285|574614|6696|47225|29285|574614|6696|47225|1|104|5|4|17152|119|1597|35|10.06|25.25|1.76|822.15|61.60|352.10|883.75|0.00|10.47|185.50|51.13|51.13|236.63|236.63|-300.97| +2450846|55848|2450915|29285|574614|6696|47225|29285|574614|6696|47225|1|57|4|4|13894|17|1597|36|77.79|154.80|47.98|3845.52|1727.28|2800.44|5572.80|34.54|0.00|1560.24|1727.28|1761.82|3287.52|3322.06|-1073.16| +2450846|55848|2450879|29285|574614|6696|47225|29285|574614|6696|47225|1|17|17|4|14434|100|1597|60|82.27|116.82|61.91|3294.60|3714.60|4936.20|7009.20|133.72|2043.03|2452.80|1671.57|1805.29|4124.37|4258.09|-3264.63| +2450846|36681|2450876|82262|1170568|2521|27227|20668|633839|5792|9298|2|106|6|4|12854|233|1598|47|46.27|119.83|46.73|3435.70|2196.31|2174.69|5632.01|0.00|0.00|731.79|2196.31|2196.31|2928.10|2928.10|21.62| +2450846|36681|2450860|82262|1170568|2521|27227|20668|633839|5792|9298|2|46|18|3|13406|44|1598|68|84.85|151.88|41.00|7539.84|2788.00|5769.80|10327.84|195.16|0.00|3408.16|2788.00|2983.16|6196.16|6391.32|-2981.80| +2450846|36681|2450886|82262|1170568|2521|27227|20668|633839|5792|9298|2|53|15|2|16282|66|1598|75|72.42|144.11|95.11|3675.00|7133.25|5431.50|10808.25|570.66|0.00|324.00|7133.25|7703.91|7457.25|8027.91|1701.75| +2450846|36681|2450927|82262|1170568|2521|27227|20668|633839|5792|9298|2|86|2|2|3506|172|1598|32|60.60|138.77|115.17|755.20|3685.44|1939.20|4440.64|23.21|1363.61|1465.28|2321.83|2345.04|3787.11|3810.32|382.63| +2450846|36681|2450858|82262|1170568|2521|27227|20668|633839|5792|9298|2|50|20|2|12632|182|1598|34|38.15|106.82|97.20|327.08|3304.80|1297.10|3631.88|39.65|2511.64|1052.98|793.16|832.81|1846.14|1885.79|-503.94| +2450846|36681|2450870|82262|1170568|2521|27227|20668|633839|5792|9298|2|81|11|4|10660|168|1598|17|74.09|171.88|106.56|1110.44|1811.52|1259.53|2921.96|126.80|0.00|409.02|1811.52|1938.32|2220.54|2347.34|551.99| +2450846|36681|2450858|82262|1170568|2521|27227|20668|633839|5792|9298|2|64|16|4|614|208|1598|2|63.81|75.29|69.26|12.06|138.52|127.62|150.58|12.46|0.00|18.06|138.52|150.98|156.58|169.04|10.90| +2450846|36681|2450934|82262|1170568|2521|27227|20668|633839|5792|9298|2|43|18|2|15721|115|1598|14|19.01|53.22|37.78|216.16|528.92|266.14|745.08|31.73|0.00|305.48|528.92|560.65|834.40|866.13|262.78| +2450846|36681|2450922|82262|1170568|2521|27227|20668|633839|5792|9298|2|67|9|5|14312|133|1598|2|92.72|251.27|7.53|487.48|15.06|185.44|502.54|0.90|0.00|70.34|15.06|15.96|85.40|86.30|-170.38| +2450846|36681|2450913|82262|1170568|2521|27227|20668|633839|5792|9298|2|70|10|1|2660|126|1598|60|18.42|20.07|10.63|566.40|637.80|1105.20|1204.20|12.75|0.00|469.20|637.80|650.55|1107.00|1119.75|-467.40| +2450846|31332|2450862|51641|1662694|3347|46400|52958|1548506|1159|23059|4|90|15|5|2180|126|1599|20|95.39|273.76|104.02|3394.80|2080.40|1907.80|5475.20|166.43|0.00|273.60|2080.40|2246.83|2354.00|2520.43|172.60| +2450846|31332|2450921|51641|1662694|3347|46400|52958|1548506|1159|23059|4|63|13|4|13040|4|1599|48|47.19|58.98|7.66|2463.36|367.68|2265.12|2831.04|33.09|0.00|594.24|367.68|400.77|961.92|995.01|-1897.44| +2450846|31332|2450868|51641|1662694|3347|46400|52958|1548506|1159|23059|4|46|11|2|5731|252|1599|28|74.62|191.77|40.27|4242.00|1127.56|2089.36|5369.56|36.30|608.88|1771.84|518.68|554.98|2290.52|2326.82|-1570.68| +2450846|31332|2450909|51641|1662694|3347|46400|52958|1548506|1159|23059|4|13|6|3|6938|182|1599|14|99.65|254.10|185.49|960.54|2596.86|1395.10|3557.40|103.87|0.00|1173.90|2596.86|2700.73|3770.76|3874.63|1201.76| +2450846|31332|2450916|51641|1662694|3347|46400|52958|1548506|1159|23059|4|12|10|5|9313|43|1599|88|23.65|30.03|16.81|1163.36|1479.28|2081.20|2642.64|103.54|0.00|158.40|1479.28|1582.82|1637.68|1741.22|-601.92| +2450846|31332|2450849|51641|1662694|3347|46400|52958|1548506|1159|23059|4|8|19|5|8702|237|1599|62|26.28|37.84|1.51|2252.46|93.62|1629.36|2346.08|0.00|0.00|163.68|93.62|93.62|257.30|257.30|-1535.74| +2450846|31332|2450875|51641|1662694|3347|46400|52958|1548506|1159|23059|4|17|7|5|15182|184|1599|15|2.84|4.57|1.96|39.15|29.40|42.60|68.55|1.76|0.00|1.95|29.40|31.16|31.35|33.11|-13.20| +2450846|31332|2450896|51641|1662694|3347|46400|52958|1548506|1159|23059|4|6|8|5|12710|194|1599|15|62.78|113.63|101.13|187.50|1516.95|941.70|1704.45|56.27|712.96|85.20|803.99|860.26|889.19|945.46|-137.71| +2450846|31332|2450861|51641|1662694|3347|46400|52958|1548506|1159|23059|4|30|10|2|1328|88|1599|22|70.90|113.44|105.49|174.90|2320.78|1559.80|2495.68|185.66|0.00|49.72|2320.78|2506.44|2370.50|2556.16|760.98| +2450846|31332|2450882|51641|1662694|3347|46400|52958|1548506|1159|23059|4|100|17|4|11716|190|1599|98|3.21|8.28|4.38|382.20|429.24|314.58|811.44|0.00|0.00|72.52|429.24|429.24|501.76|501.76|114.66| +2450846|76688|2450934|20616|494731|6799|114|20616|494731|6799|114|2|7|2|4|3238|273|1600|5|56.80|116.44|97.80|93.20|489.00|284.00|582.20|0.00|39.12|197.90|449.88|449.88|647.78|647.78|165.88| +2450846|76688|2450849|20616|494731|6799|114|20616|494731|6799|114|2|74|20|2|15218|239|1600|22|48.39|85.65|55.67|659.56|1224.74|1064.58|1884.30|55.11|306.18|75.24|918.56|973.67|993.80|1048.91|-146.02| +2450846|76688|2450893|20616|494731|6799|114|20616|494731|6799|114|2|106|18|2|4558|155|1600|5|74.61|135.79|33.94|509.25|169.70|373.05|678.95|8.48|0.00|196.85|169.70|178.18|366.55|375.03|-203.35| +2450846|76688|2450886|20616|494731|6799|114|20616|494731|6799|114|2|93|18|2|15682|143|1600|6|69.70|204.91|129.09|454.92|774.54|418.20|1229.46|15.49|0.00|295.02|774.54|790.03|1069.56|1085.05|356.34| +2450846|48679|2450886|77545|950131|5117|36056|77545|950131|5117|36056|4|76|3|3|3106|203|1601|23|88.78|123.40|49.36|1702.92|1135.28|2041.94|2838.20|34.05|0.00|1163.57|1135.28|1169.33|2298.85|2332.90|-906.66| +2450846|48679|2450924|77545|950131|5117|36056|77545|950131|5117|36056|4|7|9|3|8024|131|1601|57|77.17|140.44|133.41|400.71|7604.37|4398.69|8005.08|76.04|0.00|1920.90|7604.37|7680.41|9525.27|9601.31|3205.68| +2450846|48679|2450905|77545|950131|5117|36056|77545|950131|5117|36056|4|32|5|4|14065|81|1601|82|37.35|42.95|12.02|2536.26|985.64|3062.70|3521.90|9.85|0.00|879.86|985.64|995.49|1865.50|1875.35|-2077.06| +2450846|48679|2450900|77545|950131|5117|36056|77545|950131|5117|36056|4|105|18|4|17854|99|1601|71|51.10|148.19|142.26|421.03|10100.46|3628.10|10521.49|606.02|0.00|2524.76|10100.46|10706.48|12625.22|13231.24|6472.36| +2450846|48679|2450864|77545|950131|5117|36056|77545|950131|5117|36056|4|38|13|5|10969|112|1601|46|67.18|77.92|23.37|2509.30|1075.02|3090.28|3584.32|96.75|0.00|967.38|1075.02|1171.77|2042.40|2139.15|-2015.26| +2450846|48679|2450891|77545|950131|5117|36056|77545|950131|5117|36056|4|14|12|2|11146|72|1601|37|56.28|118.75|20.18|3647.09|746.66|2082.36|4393.75|14.93|0.00|1317.94|746.66|761.59|2064.60|2079.53|-1335.70| +2450846|48679|2450902|77545|950131|5117|36056|77545|950131|5117|36056|4|32|13|1|8221|277|1601|67|6.18|14.95|8.52|430.81|570.84|414.06|1001.65|39.95|0.00|89.78|570.84|610.79|660.62|700.57|156.78| +2450846|48679|2450856|77545|950131|5117|36056|77545|950131|5117|36056|4|63|5|3|4957|9|1601|16|69.00|200.10|56.02|2305.28|896.32|1104.00|3201.60|0.00|0.00|768.32|896.32|896.32|1664.64|1664.64|-207.68| +2450846|48679|2450895|77545|950131|5117|36056|77545|950131|5117|36056|4|45|2|5|16351|167|1601|80|28.58|84.88|79.78|408.00|6382.40|2286.40|6790.40|191.47|0.00|135.20|6382.40|6573.87|6517.60|6709.07|4096.00| +2450846|48679|2450904|77545|950131|5117|36056|77545|950131|5117|36056|4|19|14|3|6874|233|1601|56|84.62|124.39|106.97|975.52|5990.32|4738.72|6965.84|419.32|0.00|2716.56|5990.32|6409.64|8706.88|9126.20|1251.60| +2450846|48679|2450864|77545|950131|5117|36056|77545|950131|5117|36056|4|93|6|5|14740|228|1601|29|10.10|18.28|14.98|95.70|434.42|292.90|530.12|21.72|0.00|58.29|434.42|456.14|492.71|514.43|141.52| +2450846|48679|2450866|77545|950131|5117|36056|77545|950131|5117|36056|4|44|5|5|15313|253|1601|69|38.76|53.87|31.24|1561.47|2155.56|2674.44|3717.03|43.11|0.00|1040.52|2155.56|2198.67|3196.08|3239.19|-518.88| +2450846|48679|2450910|77545|950131|5117|36056|77545|950131|5117|36056|4|11|8|3|11191|124|1601|1|13.12|30.70|16.57|14.13|16.57|13.12|30.70|0.49|0.00|12.89|16.57|17.06|29.46|29.95|3.45| +2450846|48679|2450876|77545|950131|5117|36056|77545|950131|5117|36056|4|105|13|1|488|143|1601|25|33.22|84.04|26.05|1449.75|651.25|830.50|2101.00|19.53|0.00|735.25|651.25|670.78|1386.50|1406.03|-179.25| +2450846|51654|2450897|92581|926027|2673|9319|92581|926027|2673|9319|1|80|17|4|6967|178|1602|9|92.68|168.67|139.99|258.12|1259.91|834.12|1518.03|16.63|705.54|683.10|554.37|571.00|1237.47|1254.10|-279.75| +2450846|51654|2450927|92581|926027|2673|9319|92581|926027|2673|9319|1|24|11|2|17294|97|1602|64|18.36|18.54|15.57|190.08|996.48|1175.04|1186.56|29.89|0.00|248.96|996.48|1026.37|1245.44|1275.33|-178.56| +2450846|51654|2450903|92581|926027|2673|9319|92581|926027|2673|9319|1|51|1|5|3445|183|1602|79|30.29|73.60|18.40|4360.80|1453.60|2392.91|5814.40|14.53|0.00|522.98|1453.60|1468.13|1976.58|1991.11|-939.31| +2450846|51654|2450914|92581|926027|2673|9319|92581|926027|2673|9319|1|60|17|1|10525|202|1602|40|97.04|162.05|100.47|2463.20|4018.80|3881.60|6482.00|40.18|0.00|2333.20|4018.80|4058.98|6352.00|6392.18|137.20| +2450846|51654|2450876|92581|926027|2673|9319|92581|926027|2673|9319|1|24|20|4|1048|157|1602|56|79.13|83.87|35.22|2724.40|1972.32|4431.28|4696.72|39.44|0.00|281.68|1972.32|2011.76|2254.00|2293.44|-2458.96| +2450846|51654|2450913|92581|926027|2673|9319|92581|926027|2673|9319|1|25|15|5|12691|298|1602|7|58.37|97.47|57.50|279.79|402.50|408.59|682.29|28.17|0.00|20.44|402.50|430.67|422.94|451.11|-6.09| +2450846|51654|2450877|92581|926027|2673|9319|92581|926027|2673|9319|1|62|20|3|9415|155|1602|90|97.61|101.51|52.78|4385.70|4750.20|8784.90|9135.90|332.51|0.00|913.50|4750.20|5082.71|5663.70|5996.21|-4034.70| +2450846|51654|2450903|92581|926027|2673|9319|92581|926027|2673|9319|1|78|3|1|6014|282|1602|17|48.31|60.38|39.24|359.38|667.08|821.27|1026.46|13.34|0.00|123.08|667.08|680.42|790.16|803.50|-154.19| +2450846|51654|2450893|92581|926027|2673|9319|92581|926027|2673|9319|1|93|13|1|8452|22|1602|99|91.93|223.38|102.75|11942.37|10172.25|9101.07|22114.62|712.05|0.00|441.54|10172.25|10884.30|10613.79|11325.84|1071.18| +2450846|51654|2450871|92581|926027|2673|9319|92581|926027|2673|9319|1|12|5|1|4186|196|1602|62|32.15|79.41|39.70|2462.02|2461.40|1993.30|4923.42|49.22|0.00|2166.28|2461.40|2510.62|4627.68|4676.90|468.10| +2450846|51654|2450888|92581|926027|2673|9319|92581|926027|2673|9319|1|73|6|2|11512|252|1602|18|60.69|142.62|84.14|1052.64|1514.52|1092.42|2567.16|90.87|0.00|615.96|1514.52|1605.39|2130.48|2221.35|422.10| +2450846|51654|2450862|92581|926027|2673|9319|92581|926027|2673|9319|1|70|2|5|13442|292|1602|92|92.82|256.18|187.01|6363.64|17204.92|8539.44|23568.56|688.19|0.00|9898.28|17204.92|17893.11|27103.20|27791.39|8665.48| +2450846|51654|2450923|92581|926027|2673|9319|92581|926027|2673|9319|1|18|18|1|1501|97|1602|86|61.55|141.56|76.44|5600.32|6573.84|5293.30|12174.16|0.00|0.00|2921.42|6573.84|6573.84|9495.26|9495.26|1280.54| +2450846|51654|2450887|92581|926027|2673|9319|92581|926027|2673|9319|1|101|8|3|16498|45|1602|53|94.89|159.41|20.72|7350.57|1098.16|5029.17|8448.73|10.98|0.00|1435.77|1098.16|1109.14|2533.93|2544.91|-3931.01| +2450846|46562|2450907|66036|1367657|6078|99|66036|1367657|6078|99|2|49|16|2|7453|238|1603|29|60.03|147.67|124.04|685.27|3597.16|1740.87|4282.43|107.91|0.00|128.47|3597.16|3705.07|3725.63|3833.54|1856.29| +2450846|46562|2450898|66036|1367657|6078|99|66036|1367657|6078|99|2|23|2|3|17926|203|1603|66|88.61|110.76|93.03|1170.18|6139.98|5848.26|7310.16|122.79|0.00|1754.28|6139.98|6262.77|7894.26|8017.05|291.72| +2450846|46562|2450891|66036|1367657|6078|99|66036|1367657|6078|99|2|98|14|2|4432|299|1603|39|33.11|46.35|38.93|289.38|1518.27|1291.29|1807.65|0.00|0.00|596.31|1518.27|1518.27|2114.58|2114.58|226.98| +2450846|46562|2450857|66036|1367657|6078|99|66036|1367657|6078|99|2|88|9|3|12026|226|1603|27|83.48|100.17|5.00|2569.59|135.00|2253.96|2704.59|9.45|0.00|270.27|135.00|144.45|405.27|414.72|-2118.96| +2450846|46562|2450922|66036|1367657|6078|99|66036|1367657|6078|99|2|58|14|5|12571|72|1603|30|42.14|42.98|0.00|1289.40|0.00|1264.20|1289.40|0.00|0.00|38.40|0.00|0.00|38.40|38.40|-1264.20| +2450846|46562|2450927|66036|1367657|6078|99|66036|1367657|6078|99|2|56|10|4|12481|22|1603|12|24.67|70.30|23.90|556.80|286.80|296.04|843.60|5.73|0.00|227.76|286.80|292.53|514.56|520.29|-9.24| +2450846|46562|2450871|66036|1367657|6078|99|66036|1367657|6078|99|2|99|13|3|8971|250|1603|30|38.77|79.09|43.49|1068.00|1304.70|1163.10|2372.70|65.23|0.00|213.30|1304.70|1369.93|1518.00|1583.23|141.60| +2450846|46562|2450920|66036|1367657|6078|99|66036|1367657|6078|99|2|107|14|4|2197|61|1603|24|72.60|140.84|8.45|3177.36|202.80|1742.40|3380.16|2.61|115.59|1318.08|87.21|89.82|1405.29|1407.90|-1655.19| +2450846|46562|2450898|66036|1367657|6078|99|66036|1367657|6078|99|2|80|18|2|733|90|1603|31|11.60|22.27|19.59|83.08|607.29|359.60|690.37|54.65|0.00|331.08|607.29|661.94|938.37|993.02|247.69| +2450846|46562|2450876|66036|1367657|6078|99|66036|1367657|6078|99|2|73|2|5|10336|163|1603|31|57.88|99.55|13.93|2654.22|431.83|1794.28|3086.05|25.90|0.00|678.90|431.83|457.73|1110.73|1136.63|-1362.45| +2450846|46562|2450883|66036|1367657|6078|99|66036|1367657|6078|99|2|25|12|2|751|32|1603|71|5.00|13.15|10.25|205.90|727.75|355.00|933.65|0.00|0.00|158.33|727.75|727.75|886.08|886.08|372.75| +2450846|46562|2450849|66036|1367657|6078|99|66036|1367657|6078|99|2|72|12|3|8899|281|1603|85|39.41|117.04|73.73|3681.35|6267.05|3349.85|9948.40|313.35|0.00|1989.00|6267.05|6580.40|8256.05|8569.40|2917.20| +2450846|46562|2450866|66036|1367657|6078|99|66036|1367657|6078|99|2|31|14|5|9427|170|1603|30|85.97|153.02|67.32|2571.00|2019.60|2579.10|4590.60|40.39|0.00|1790.10|2019.60|2059.99|3809.70|3850.09|-559.50| +2450846|59785|2450896|36918|1179441|2317|20040|36918|1179441|2317|20040|1|62|2|5|3817|172|1604|75|54.88|109.21|25.11|6307.50|1883.25|4116.00|8190.75|131.82|0.00|491.25|1883.25|2015.07|2374.50|2506.32|-2232.75| +2450846|59785|2450904|36918|1179441|2317|20040|36918|1179441|2317|20040|1|12|1|4|416|151|1604|78|31.71|87.20|68.01|1496.82|5304.78|2473.38|6801.60|0.00|0.00|2720.64|5304.78|5304.78|8025.42|8025.42|2831.40| +2450846|59785|2450904|36918|1179441|2317|20040|36918|1179441|2317|20040|1|58|1|4|3751|122|1604|22|84.60|250.41|92.65|3470.72|2038.30|1861.20|5509.02|24.86|794.93|2313.74|1243.37|1268.23|3557.11|3581.97|-617.83| +2450846|59785|2450933|36918|1179441|2317|20040|36918|1179441|2317|20040|1|67|12|4|559|81|1604|67|16.68|44.86|29.15|1052.57|1953.05|1117.56|3005.62|136.71|0.00|89.78|1953.05|2089.76|2042.83|2179.54|835.49| +2450846|59785|2450905|36918|1179441|2317|20040|36918|1179441|2317|20040|1|6|13|5|12236|267|1604|21|65.32|89.48|44.74|939.54|939.54|1371.72|1879.08|0.00|0.00|0.00|939.54|939.54|939.54|939.54|-432.18| +2450846|59785|2450911|36918|1179441|2317|20040|36918|1179441|2317|20040|1|3|1|3|4105|5|1604|77|41.87|98.81|26.67|5554.78|2053.59|3223.99|7608.37|41.07|0.00|1521.52|2053.59|2094.66|3575.11|3616.18|-1170.40| +2450846|65093|2450907|11093|350565|84|12756|11093|350565|84|12756|2|24|16|5|4549|183|1605|62|23.92|54.53|35.98|1150.10|2230.76|1483.04|3380.86|44.61|0.00|236.22|2230.76|2275.37|2466.98|2511.59|747.72| +2450846|65093|2450872|11093|350565|84|12756|11093|350565|84|12756|2|49|16|3|10363|12|1605|50|24.48|29.37|0.88|1424.50|44.00|1224.00|1468.50|0.44|0.00|234.50|44.00|44.44|278.50|278.94|-1180.00| +2450846|65093|2450870|11093|350565|84|12756|11093|350565|84|12756|2|53|19|5|16081|147|1605|16|51.31|104.67|88.96|251.36|1423.36|820.96|1674.72|71.16|0.00|200.96|1423.36|1494.52|1624.32|1695.48|602.40| +2450846|65093|2450886|11093|350565|84|12756|11093|350565|84|12756|2|29|1|2|415|224|1605|28|74.95|172.38|77.57|2654.68|2171.96|2098.60|4826.64|140.74|412.67|1109.92|1759.29|1900.03|2869.21|3009.95|-339.31| +2450846|65093|2450856|11093|350565|84|12756|11093|350565|84|12756|2|104|17|1|13039|122|1605|64|48.79|104.41|20.88|5345.92|1336.32|3122.56|6682.24|106.90|0.00|668.16|1336.32|1443.22|2004.48|2111.38|-1786.24| +2450846|65093|2450890|11093|350565|84|12756|11093|350565|84|12756|2|63|5|5|2782|264|1605|90|66.22|80.78|37.96|3853.80|3416.40|5959.80|7270.20|307.47|0.00|1890.00|3416.40|3723.87|5306.40|5613.87|-2543.40| +2450846|65093|2450912|11093|350565|84|12756|11093|350565|84|12756|2|105|9|4|13033|127|1605|16|39.41|53.59|0.00|857.44|0.00|630.56|857.44|0.00|0.00|77.12|0.00|0.00|77.12|77.12|-630.56| +2450846|65093|2450859|11093|350565|84|12756|11093|350565|84|12756|2|70|20|4|1819|87|1605|71|31.76|63.52|56.53|496.29|4013.63|2254.96|4509.92|160.54|0.00|631.19|4013.63|4174.17|4644.82|4805.36|1758.67| +2450846|65093|2450888|11093|350565|84|12756|11093|350565|84|12756|2|38|12|4|760|173|1605|49|82.48|148.46|81.65|3273.69|4000.85|4041.52|7274.54|280.05|0.00|1527.33|4000.85|4280.90|5528.18|5808.23|-40.67| +2450846|65093|2450886|11093|350565|84|12756|11093|350565|84|12756|2|2|17|3|8824|90|1605|20|43.75|75.25|11.28|1279.40|225.60|875.00|1505.00|7.10|83.47|752.40|142.13|149.23|894.53|901.63|-732.87| +2450846|74728|2450871|94468|1237249|6077|45658|94468|1237249|6077|45658|4|86|11|5|7186|10|1606|12|71.19|126.71|5.06|1459.80|60.72|854.28|1520.52|2.42|0.00|608.16|60.72|63.14|668.88|671.30|-793.56| +2450846|74728|2450863|94468|1237249|6077|45658|94468|1237249|6077|45658|4|59|7|4|10906|283|1606|63|3.74|9.87|7.40|155.61|466.20|235.62|621.81|37.29|0.00|98.91|466.20|503.49|565.11|602.40|230.58| +2450846|74728|2450888|94468|1237249|6077|45658|94468|1237249|6077|45658|4|102|6|2|5923|111|1606|88|50.41|50.91|10.69|3539.36|940.72|4436.08|4480.08|47.03|0.00|985.60|940.72|987.75|1926.32|1973.35|-3495.36| +2450846|74728|2450916|94468|1237249|6077|45658|94468|1237249|6077|45658|4|35|3|5|17299|27|1606|16|15.25|19.97|13.57|102.40|217.12|244.00|319.52|19.54|0.00|92.64|217.12|236.66|309.76|329.30|-26.88| +2450846|74728|2450907|94468|1237249|6077|45658|94468|1237249|6077|45658|4|98|17|1|70|211|1606|47|54.43|82.18|12.32|3283.42|579.04|2558.21|3862.46|17.37|0.00|888.30|579.04|596.41|1467.34|1484.71|-1979.17| +2450846|74728|2450885|94468|1237249|6077|45658|94468|1237249|6077|45658|4|86|13|1|16291|210|1606|95|81.44|191.38|17.22|16545.20|1635.90|7736.80|18181.10|98.15|0.00|4363.35|1635.90|1734.05|5999.25|6097.40|-6100.90| +2450846|74728|2450887|94468|1237249|6077|45658|94468|1237249|6077|45658|4|29|1|1|1178|9|1606|70|49.60|131.93|124.01|554.40|8680.70|3472.00|9235.10|434.03|0.00|3601.50|8680.70|9114.73|12282.20|12716.23|5208.70| +2450846|74728|2450876|94468|1237249|6077|45658|94468|1237249|6077|45658|4|64|6|3|140|169|1606|58|95.94|279.18|228.92|2915.08|13277.36|5564.52|16192.44|531.09|0.00|8096.22|13277.36|13808.45|21373.58|21904.67|7712.84| +2450846|74728|2450848|94468|1237249|6077|45658|94468|1237249|6077|45658|4|78|19|5|3106|97|1606|6|79.58|108.22|24.89|499.98|149.34|477.48|649.32|7.46|0.00|116.82|149.34|156.80|266.16|273.62|-328.14| +2450846|40087|2450899|13016|854073|5177|18634|13016|854073|5177|18634|2|94|19|1|6577|27|1607|47|65.54|175.64|47.42|6026.34|2228.74|3080.38|8255.08|200.58|0.00|3219.03|2228.74|2429.32|5447.77|5648.35|-851.64| +2450846|40087|2450851|13016|854073|5177|18634|13016|854073|5177|18634|2|87|6|2|17200|277|1607|27|4.03|9.83|9.63|5.40|260.01|108.81|265.41|23.40|0.00|36.99|260.01|283.41|297.00|320.40|151.20| +2450846|40087|2450918|13016|854073|5177|18634|13016|854073|5177|18634|2|30|20|2|1681|196|1607|81|93.70|260.48|127.63|10760.85|10338.03|7589.70|21098.88|3.10|10234.64|3375.27|103.39|106.49|3478.66|3481.76|-7486.31| +2450846|40087|2450860|13016|854073|5177|18634|13016|854073|5177|18634|2|4|10|4|11437|57|1607|50|97.70|228.61|226.32|114.50|11316.00|4885.00|11430.50|1018.44|0.00|571.50|11316.00|12334.44|11887.50|12905.94|6431.00| +2450846|44006|2450910|24524|866377|1030|130|24524|866377|1030|130|1|57|9|5|5896|262|1608|60|22.97|43.87|16.23|1658.40|973.80|1378.20|2632.20|58.42|0.00|1184.40|973.80|1032.22|2158.20|2216.62|-404.40| +2450846|44006|2450852|24524|866377|1030|130|24524|866377|1030|130|1|52|17|1|1372|13|1608|97|39.13|70.82|43.90|2611.24|4258.30|3795.61|6869.54|383.24|0.00|2403.66|4258.30|4641.54|6661.96|7045.20|462.69| +2450846|44006|2450905|24524|866377|1030|130|24524|866377|1030|130|1|17|12|4|3560|40|1608|47|91.43|228.57|109.71|5586.42|5156.37|4297.21|10742.79|360.94|0.00|1288.74|5156.37|5517.31|6445.11|6806.05|859.16| +2450846|44006|2450922|24524|866377|1030|130|24524|866377|1030|130|1|73|3|3|12176|265|1608|30|61.01|100.66|64.42|1087.20|1932.60|1830.30|3019.80|173.93|0.00|633.90|1932.60|2106.53|2566.50|2740.43|102.30| +2450846|44006|2450923|24524|866377|1030|130|24524|866377|1030|130|1|6|14|1|12511|222|1608|3|32.98|52.10|20.31|95.37|60.93|98.94|156.30|0.00|0.00|78.15|60.93|60.93|139.08|139.08|-38.01| +2450846|44006|2450859|24524|866377|1030|130|24524|866377|1030|130|1|104|8|2|7555|62|1608|91|99.08|193.20|119.78|6681.22|10899.98|9016.28|17581.20|435.99|0.00|3339.70|10899.98|11335.97|14239.68|14675.67|1883.70| +2450846|28182|2450881|16823|344241|3659|18677|19737|1018713|1331|21149|2|101|17|5|8776|281|1609|96|53.34|61.87|17.94|4217.28|1722.24|5120.64|5939.52|17.22|0.00|1781.76|1722.24|1739.46|3504.00|3521.22|-3398.40| +2450846|28182|2450897|16823|344241|3659|18677|19737|1018713|1331|21149|2|7|12|4|14521|231|1609|78|14.22|28.58|13.71|1159.86|1069.38|1109.16|2229.24|96.24|0.00|668.46|1069.38|1165.62|1737.84|1834.08|-39.78| +2450846|28182|2450867|16823|344241|3659|18677|19737|1018713|1331|21149|2|101|14|3|11228|68|1609|63|63.08|173.47|163.06|655.83|10272.78|3974.04|10928.61|0.00|0.00|764.82|10272.78|10272.78|11037.60|11037.60|6298.74| +2450846|28182|2450868|16823|344241|3659|18677|19737|1018713|1331|21149|2|60|6|1|5875|109|1609|77|73.97|179.74|179.74|0.00|13839.98|5695.69|13839.98|276.79|0.00|3183.18|13839.98|14116.77|17023.16|17299.95|8144.29| +2450846|28182|2450928|16823|344241|3659|18677|19737|1018713|1331|21149|2|94|13|2|6847|265|1609|48|36.75|52.55|3.15|2371.20|151.20|1764.00|2522.40|13.60|0.00|655.68|151.20|164.80|806.88|820.48|-1612.80| +2450846|28182|2450925|16823|344241|3659|18677|19737|1018713|1331|21149|2|24|5|5|10334|162|1609|99|15.52|25.45|15.27|1007.82|1511.73|1536.48|2519.55|45.35|0.00|780.12|1511.73|1557.08|2291.85|2337.20|-24.75| +2450846|28182|2450933|16823|344241|3659|18677|19737|1018713|1331|21149|2|50|16|4|14750|133|1609|9|96.13|181.68|12.71|1520.73|114.39|865.17|1635.12|2.12|43.46|670.32|70.93|73.05|741.25|743.37|-794.24| +2450846|28182|2450905|16823|344241|3659|18677|19737|1018713|1331|21149|2|101|17|5|9304|81|1609|71|67.23|142.52|138.24|303.88|9815.04|4773.33|10118.92|785.20|0.00|5059.46|9815.04|10600.24|14874.50|15659.70|5041.71| +2450846|28182|2450934|16823|344241|3659|18677|19737|1018713|1331|21149|2|14|14|5|872|262|1609|31|15.25|31.87|21.35|326.12|661.85|472.75|987.97|6.61|0.00|207.39|661.85|668.46|869.24|875.85|189.10| +2450846|28182|2450916|16823|344241|3659|18677|19737|1018713|1331|21149|2|71|14|4|7646|61|1609|56|75.47|196.97|59.09|7721.28|3309.04|4226.32|11030.32|132.36|0.00|3419.36|3309.04|3441.40|6728.40|6860.76|-917.28| +2450846|28182|2450872|16823|344241|3659|18677|19737|1018713|1331|21149|2|44|14|5|15638|17|1609|16|48.75|97.98|31.35|1066.08|501.60|780.00|1567.68|10.03|0.00|658.40|501.60|511.63|1160.00|1170.03|-278.40| +2450846|28182|2450889|16823|344241|3659|18677|19737|1018713|1331|21149|2|34|16|2|5744|64|1609|8|92.61|261.16|73.12|1504.32|584.96|740.88|2089.28|17.54|0.00|877.44|584.96|602.50|1462.40|1479.94|-155.92| +2450846|28182|2450858|16823|344241|3659|18677|19737|1018713|1331|21149|2|54|13|5|8444|96|1609|29|28.33|38.24|26.00|354.96|754.00|821.57|1108.96|45.24|0.00|410.06|754.00|799.24|1164.06|1209.30|-67.57| +2450846|77223|2450931|84307|1548472|2237|11602|84307|1548472|2237|11602|2|75|9|3|8431|86|1610|2|19.40|51.60|25.80|51.60|51.60|38.80|103.20|0.00|0.00|15.48|51.60|51.60|67.08|67.08|12.80| +2450846|77223|2450883|84307|1548472|2237|11602|84307|1548472|2237|11602|2|99|2|4|3746|123|1610|19|14.52|14.66|6.01|164.35|114.19|275.88|278.54|1.14|0.00|30.59|114.19|115.33|144.78|145.92|-161.69| +2450846|77223|2450876|84307|1548472|2237|11602|84307|1548472|2237|11602|2|108|19|4|2228|116|1610|31|80.70|140.41|29.48|3438.83|913.88|2501.70|4352.71|45.69|0.00|304.42|913.88|959.57|1218.30|1263.99|-1587.82| +2450846|77223|2450928|84307|1548472|2237|11602|84307|1548472|2237|11602|2|16|16|2|3229|137|1610|88|13.64|22.64|2.26|1793.44|198.88|1200.32|1992.32|5.96|0.00|258.72|198.88|204.84|457.60|463.56|-1001.44| +2450846|77223|2450889|84307|1548472|2237|11602|84307|1548472|2237|11602|2|19|8|3|12649|220|1610|40|32.96|65.92|24.39|1661.20|975.60|1318.40|2636.80|36.29|68.29|790.80|907.31|943.60|1698.11|1734.40|-411.09| +2450846|77223|2450893|84307|1548472|2237|11602|84307|1548472|2237|11602|2|88|7|4|12068|8|1610|42|2.77|6.78|3.32|145.32|139.44|116.34|284.76|12.54|0.00|96.60|139.44|151.98|236.04|248.58|23.10| +2450846|77223|2450904|84307|1548472|2237|11602|84307|1548472|2237|11602|2|70|16|2|2300|76|1610|31|79.94|116.71|25.67|2822.24|795.77|2478.14|3618.01|15.91|0.00|1193.81|795.77|811.68|1989.58|2005.49|-1682.37| +2450846|77223|2450897|84307|1548472|2237|11602|84307|1548472|2237|11602|2|94|5|5|12022|130|1610|13|82.49|182.30|91.15|1184.95|1184.95|1072.37|2369.90|46.80|248.83|995.28|936.12|982.92|1931.40|1978.20|-136.25| +2450846|77223|2450914|84307|1548472|2237|11602|84307|1548472|2237|11602|2|52|10|3|6962|107|1610|78|17.26|27.61|26.78|64.74|2088.84|1346.28|2153.58|0.00|0.00|516.36|2088.84|2088.84|2605.20|2605.20|742.56| +2450846|77223|2450849|84307|1548472|2237|11602|84307|1548472|2237|11602|2|32|9|1|16544|120|1610|71|38.49|98.14|89.30|627.64|6340.30|2732.79|6967.94|443.82|0.00|2159.82|6340.30|6784.12|8500.12|8943.94|3607.51| +2450846|77223|2450871|84307|1548472|2237|11602|84307|1548472|2237|11602|2|18|3|3|9241|93|1610|79|23.50|40.42|8.48|2523.26|669.92|1856.50|3193.18|60.29|0.00|542.73|669.92|730.21|1212.65|1272.94|-1186.58| +2450846|77223|2450855|84307|1548472|2237|11602|84307|1548472|2237|11602|2|82|15|3|7705|9|1610|24|6.32|11.94|10.62|31.68|254.88|151.68|286.56|22.93|0.00|48.48|254.88|277.81|303.36|326.29|103.20| +2450846|56797|2450919|51691|1776713|2182|22404|51691|1776713|2182|22404|1|69|13|3|8266|151|1611|83|32.17|89.11|32.97|4659.62|2736.51|2670.11|7396.13|0.00|0.00|2958.12|2736.51|2736.51|5694.63|5694.63|66.40| diff --git a/spark-connector/spark-common/src/test/resources/data/tpcds/customer/customer.dat b/spark-connector/spark-common/src/test/resources/data/tpcds/customer/customer.dat new file mode 100644 index 00000000000..a4a15d31bae --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/data/tpcds/customer/customer.dat @@ -0,0 +1,10000 @@ +1|AAAAAAAABAAAAAAA|980124|7135|32946|2452238|2452208|Mr.|Javier|Lewis|Y|9|12|1936|CHILE||Javier.Lewis@VFAxlnZEvOx.org|2452508| +2|AAAAAAAACAAAAAAA|819667|1461|31655|2452318|2452288|Dr.|Amy|Moses|Y|9|4|1966|TOGO||Amy.Moses@Ovk9KjHH.com|2452318| +3|AAAAAAAADAAAAAAA|1473522|6247|48572|2449130|2449100|Miss|Latisha|Hamilton|N|18|9|1979|NIUE||Latisha.Hamilton@V.com|2452313| +4|AAAAAAAAEAAAAAAA|1703214|3986|39558|2450030|2450000|Dr.|Michael|White|N|7|6|1983|MEXICO||Michael.White@i.org|2452361| +5|AAAAAAAAFAAAAAAA|953372|4470|36368|2449438|2449408|Sir|Robert|Moran|N|8|5|1956|FIJI||Robert.Moran@Hh.edu|2452469| +6|AAAAAAAAGAAAAAAA|213219|6374|27082|2451883|2451853|Ms.|Brunilda|Sharp|N|4|12|1925|SURINAME||Brunilda.Sharp@T3pylZEUQjm.org|2452430| +7|AAAAAAAAHAAAAAAA|68377|3219|44814|2451438|2451408|Ms.|Fonda|Wiles|Y|24|4|1985|GAMBIA||Fonda.Wiles@S9KnyEtz9hv.org|2452360| +8|AAAAAAAAIAAAAAAA|1215897|2471|16598|2449406|2449376|Sir|Ollie|Shipman|N|26|12|1938|KOREA, REPUBLIC OF||Ollie.Shipman@be.org|2452334| +9|AAAAAAAAJAAAAAAA|1168667|1404|49388|2452275|2452245|Sir|Karl|Gilbert|N|26|10|1966|MONTSERRAT||Karl.Gilbert@Crg5KyP2IxX9C4d6.edu|2452454| +10|AAAAAAAAKAAAAAAA|1207553|5143|19580|2451353|2451323|Ms.|Albert|Brunson|N|15|10|1973|JORDAN||Albert.Brunson@62.com|2452641| +11|AAAAAAAALAAAAAAA|1114415|6807|47999|2452288|2452258|Ms.|Betty|Williams|N|18|12|1963|BURKINA FASO||Betty.Williams@xRtDqM1eLBVQNoYAJ.com|2452398| +12|AAAAAAAAMAAAAAAA|502141|6577|47366|2451039|2451009|Ms.|Margaret|Farias|N|2|6|1956|TURKMENISTAN||Margaret.Farias@cb.edu|2452634| +13|AAAAAAAANAAAAAAA|1128748|2777|14006|2449658|2449628|Mrs.|Rosalinda|Grimes|N|1|3|1970|UKRAINE||Rosalinda.Grimes@tC8pcU7Lt.edu|2452616| +14|AAAAAAAAOAAAAAAA|929344|892|6440|2450318|2450288|Mr.|Jack|Wilcox|N|30|3|1937|SLOVENIA||Jack.Wilcox@Y3Etqyv3.org|2452641| +15|AAAAAAAAPAAAAAAA||134|30469||2449010|Ms.|Tonya||||12|1969||||2452376| +16|AAAAAAAAABAAAAAA|1196373|3014|29302|2451346|2451316|Dr.|Margie|Browning|N|24|12|1933|PHILIPPINES||Margie.Browning@LM674NrE2.org|2452573| +17|AAAAAAAABBAAAAAA|707524|3876|2228|2451068|2451038|Dr.|Lee|Stovall|N|23|12|1972|PHILIPPINES||Lee.Stovall@fqKC83UU0f.org|2452454| +18|AAAAAAAACBAAAAAA|1361151|6580|18456|2450041|2450011|Sir|Brad|Lynch|Y|1|9|1950|URUGUAY||Brad.Lynch@nAbai.edu|2452549| +19|AAAAAAAADBAAAAAA|1161742|4238|45581|2449580|2449550|Dr.|Andre|Moore|N|20|10|1978|NICARAGUA||Andre.Moore@cTZLGYi1ZJi.org|2452576| +20|AAAAAAAAEBAAAAAA|1185612|89|38966|2450965|2450935|Mr.|Stanton|Dallas|Y|17|5|1976|SWITZERLAND||Stanton.Dallas@DBXgl18FGo.edu|2452334| +21|AAAAAAAAFBAAAAAA|442697|6538|25979|2451827|2451797|Miss|Naomi|Barnett|N|18|5|1956|BAHAMAS||Naomi.Barnett@2T3V3OZOy4KBNAHsT.edu|2452433| +22|AAAAAAAAGBAAAAAA|490494|4504|40467|2451465|2451435|Sir|Victor|Martinez|N|21|12|1991|CAYMAN ISLANDS||Victor.Martinez@fC.edu|2452529| +23|AAAAAAAAHBAAAAAA||2107|686|2451012|2450982|Sir||Garrison|N||5||||Earl.Garrison@G3sM4P.com|| +24|AAAAAAAAIBAAAAAA|17113|1102|14291|2450098|2450068|Dr.|Paul|Morris|Y|6|6|1951|GUINEA-BISSAU||Paul.Morris@FMGalegqc3.com|2452499| +25|AAAAAAAAJBAAAAAA|476176|3278|25933|2449246|2449216|Miss|Nancy|Mccormick|N|6|10|1973|BAHAMAS||Nancy.Mccormick@DA26I9ZArLF9rxJ6Z.edu|2452425| +26|AAAAAAAAKBAAAAAA|230278|1242|28206|2449816|2449786|Ms.|Monique|Baker|N|14|6|1991|MOROCCO||Monique.Baker@9uEucNczY.org|2452356| +27|AAAAAAAALBAAAAAA|111621|835|16096|2450611|2450581|Dr.|Shawn|Prather|N|7|3|1972|PUERTO RICO||Shawn.Prather@8BusRYegn6.org|2452353| +28|AAAAAAAAMBAAAAAA|889961|897|33480|2452615|2452585|Miss|Edith|Hernandez|Y|17|8|1959|C�TE D'IVOIRE||Edith.Hernandez@BNHL0k.com|2452631| +29|AAAAAAAANBAAAAAA|1634269|1204|44139|2452216|2452186|Ms.|Margaret|Collins|Y|25|8|1973|NETHERLANDS ANTILLES||Margaret.Collins@9obPr3UV.org|2452496| +30|AAAAAAAAOBAAAAAA|84232|5869|45459|2449916|2449886|Dr.|Pamela|Luna|Y|4|7|1956|GUAM||Pamela.Luna@QBGuhL36lnA.edu|2452410| +31|AAAAAAAAPBAAAAAA|1275120|5724|7128|2452054|2452024|Dr.|William|Craig|N|13|6|1964|SPAIN||William.Craig@prVDE1E8AHc.org|2452336| +32|AAAAAAAAACAAAAAA|1369589|6163|48290|2449471|2449441|Dr.|Kenneth|Wood|N|2|8|1927|MOLDOVA, REPUBLIC OF||Kenneth.Wood@RIA.edu|2452510| +33|AAAAAAAABCAAAAAA|1680761|5823|32438|2451572|2451542|Mrs.|Donna|Betts|Y|25|3|1958|FRANCE||Donna.Betts@YJ14k.edu|2452455| +34|AAAAAAAACCAAAAAA|||37501|2452192|2452162|||Woods|||||||Brandon.Woods@hjKbf.edu|2452408| +35|AAAAAAAADCAAAAAA|339036|2057|32231|2451937|2451907|Dr.|Marie|Peterson|N|19|6|1953|KIRIBATI||Marie.Peterson@1zg9tydFHafA5.com|2452581| +36|AAAAAAAAECAAAAAA|655414|2223|26585|2449222|2449192|Dr.|Anthony|Fisher|N|29|11|1968|KYRGYZSTAN||Anthony.Fisher@jJrZfeDcz8P.com|2452470| +37|AAAAAAAAFCAAAAAA|827972|2281|12943|2449848|2449818|Dr.|Dwight|Schneider|Y|9|1|1949|RUSSIAN FEDERATION||Dwight.Schneider@koxO7zAysvOd.com|2452293| +38|AAAAAAAAGCAAAAAA|766645|3879|36360|2450323|2450293|Mrs.|Ha|Carpenter|Y|30|3|1939|PARAGUAY||Ha.Carpenter@XgcUt4svNz.com|2452351| +39|AAAAAAAAHCAAAAAA|1617182|4013|22844|2452131|2452101|Sir|Neil|Cox|N|14|12|1951|ECUADOR||Neil.Cox@FRuR2bFK.com|2452586| +40|AAAAAAAAICAAAAAA|1795301||21045|2451159|||Jacqueline|Hatfield||26||1942||||| +41|AAAAAAAAJCAAAAAA|941420|5068|18316|2449989|2449959|Miss|Maxine|Carlson|N|26|6|1939|ISLE OF MAN||Maxine.Carlson@StyP5lAokmQ29QHYMLa.edu|2452464| +42|AAAAAAAAKCAAAAAA|1867377|3327|11277|2449869|2449839|Dr.|Deborah|Burton|N|17|1|1950|COMOROS||Deborah.Burton@xt.edu|2452498| +43|AAAAAAAALCAAAAAA|385562|3676|43743|2451705|2451675|Mr.|William|Warner|Y|5|12|1965|GUAM||William.Warner@zegnrzurU.org|2452537| +44|AAAAAAAAMCAAAAAA|497758|5427|32296|2449750|2449720|Miss|Mabel|Richmond|Y|7|12|1932|CAMBODIA||Mabel.Richmond@Tkla.edu|2452333| +45|AAAAAAAANCAAAAAA|1036174|2399|19777|2451118|2451088|||Hammonds||5|||NIGER|||| +46|AAAAAAAAOCAAAAAA|843672|4729|21386|2451857|2451827|Mrs.|Jane|Stephenson|Y|13|4|1990|TURKEY||Jane.Stephenson@lq8ZQLAUMZhR.edu|2452301| +47|AAAAAAAAPCAAAAAA|1634314|5368|38469|2449165|2449135|Mr.|Luis|Young|N|3|12|1982|BRUNEI DARUSSALAM||Luis.Young@0DmV.edu|2452586| +48|AAAAAAAAADAAAAAA|8817|311|38252|2449531|2449501|Ms.|Jill|Jackson|N|15|9|1961|LUXEMBOURG||Jill.Jackson@n6I7SF.org|2452575| +49|AAAAAAAABDAAAAAA|936800|1514|20581|2449364|2449334|Mr.|Steven|Venable|N|13|6|1961|NETHERLANDS ANTILLES||Steven.Venable@0hA90vhfK7k9F4h.com|2452350| +50|AAAAAAAACDAAAAAA|759177|72|32567|2449384|2449354|Mr.|Christopher|Eller|N|24|11|1965|AUSTRALIA||Christopher.Eller@gV5Ua7HOmt.com|2452520| +51|AAAAAAAADDAAAAAA|532799|6920|10688|2452390|2452360|Dr.|David|Nieves|N|5|10|1926|CHRISTMAS ISLAND||David.Nieves@LcDkQ.edu|2452570| +52|AAAAAAAAEDAAAAAA|534808|4768|27128|2451421|2451391|Mrs.|Wendy|Colley|N|1|4|1982|FRENCH GUIANA||Wendy.Colley@qLBjqbAQQGj.edu|2452553| +53|AAAAAAAAFDAAAAAA|1597348|7017|43439|2451820|2451790|Mr.|Paul|Higgins|Y|23|9|1950|GABON||Paul.Higgins@qG9NrSTLz9HaNHX.edu|2452447| +54|AAAAAAAAGDAAAAAA|380102|2381|40660|2449980|2449950|Ms.|Donna|King|Y|3|2|1940|TUNISIA||Donna.King@TEftU.com|2452446| +55|AAAAAAAAHDAAAAAA|783093|1151|23256|2451539|2451509|Dr.|Beryl|Thomason|N|12|8|1933|UNITED KINGDOM||Beryl.Thomason@OeqefhtCmZTAj.com|2452380| +56|AAAAAAAAIDAAAAAA|344460|3092|2524|2451424|2451394|Miss|Pamela|Delgado|N|29|1|1960|GUERNSEY||Pamela.Delgado@8OpV0Ldj8vq2K9ZK.org|2452334| +57|AAAAAAAAJDAAAAAA|752932|128|571|2449741|2449711|Mr.|Travis|Melendez|Y|10|3|1961|AFGHANISTAN||Travis.Melendez@344rCMk.edu|2452606| +58|AAAAAAAAKDAAAAAA|528756|6879|14155|2451386|2451356|Mr.|Cecil|Peterman|Y|29|2|1988|ICELAND||Cecil.Peterman@tbeqEuUvS4ZM4Px9N.com|2452447| +59|AAAAAAAALDAAAAAA|1895444|4787|45683|2451157|2451127|Miss|Melanie|Morrison|N|6|1|1947|HUNGARY||Melanie.Morrison@F2foqn.edu|2452530| +60|AAAAAAAAMDAAAAAA|1179671|3200|44282|2449714|2449684|Ms.|Dollie|Thao|N|15|7|1990|SWITZERLAND||Dollie.Thao@Xead5vagsekdHDLUkv.edu|2452407| +61|AAAAAAAANDAAAAAA|1620078|6683|11688|2451136|2451106|Sir|Joseph|White|Y|6|5|1978|SENEGAL||Joseph.White@c0EJ7pimuu.com|2452462| +62|AAAAAAAAODAAAAAA|821787|4700|28413|2451854|2451824|Mr.|Matthew|Brown|Y|1|6|1973|KOREA, REPUBLIC OF||Matthew.Brown@F.edu|2452319| +63|AAAAAAAAPDAAAAAA|1790374|2445|9716|2450179|2450149|Dr.|Frank|Morton|N|26|4|1943|EGYPT||Frank.Morton@Hd7jNaA3s.com|2452425| +64|AAAAAAAAAEAAAAAA|1260191|6284|25930|2452379|2452349|Dr.|Michael|Frye|N|27|6|1941|CAPE VERDE||Michael.Frye@aM1HsbOs0smgpLo.org|2452426| +65|AAAAAAAABEAAAAAA|1778884|2234|37584|2451898|2451868|Dr.|Rodney|Taft|N|18|9|1991|VIRGIN ISLANDS, U.S.||Rodney.Taft@qe.com|2452353| +66|AAAAAAAACEAAAAAA|747190|6036|42882|2449294|2449264|Ms.|Julie|Chester|N|8|9|1950|HONG KONG||Julie.Chester@Kv.com|2452375| +67|AAAAAAAADEAAAAAA|1118294|2032|24970|2452033|2452003|Dr.|Gerald|Thomas|N|27|10|1948|NORWAY||Gerald.Thomas@zSuIGSgb6iyu.org|2452414| +68|AAAAAAAAEEAAAAAA|779965|6069|16126|2451936|2451906|Dr.|Arthur|Troy|Y|20|8|1928|LUXEMBOURG||Arthur.Troy@3VY5bV30AifrO.com|2452542| +69|AAAAAAAAFEAAAAAA|971368|4167|9366|2450154|2450124|Sir|Steven|Mcclellan|Y|15|1|1950|NEPAL||Steven.Mcclellan@UviyOLnu2m1POo.edu|2452340| +70|AAAAAAAAGEAAAAAA|1460929|5492|48102|2451944|2451914|Sir|James|Smith|N|7|12|1963|KUWAIT||James.Smith@ifJngGlNG.edu|2452342| +71|AAAAAAAAHEAAAAAA|1292064|2257|42450|2450248|2450218|Mr.|Roderick|Rogers|N|4|11|1967|OMAN||Roderick.Rogers@pJdioQ.com|2452462| +72|AAAAAAAAIEAAAAAA|1608738|6364|26390|2451700|2451670|Dr.|Bradley|Barry|Y|21|3|1937|TAJIKISTAN||Bradley.Barry@Kq2ONpEXU9YSno31.edu|2452601| +73|AAAAAAAAJEAAAAAA|425740|431|48369|2449133|2449103|Mr.|David|Vasquez|N|17|4|1956|ISRAEL||David.Vasquez@j.org|2452489| +74|AAAAAAAAKEAAAAAA|1434225|347|15269|2452641|2452611|Sir|Eric|Woods|Y|19|2|1953|SAINT LUCIA||Eric.Woods@CfPzy1AUqxd2.com|2452584| +75|AAAAAAAALEAAAAAA|1888603|143|37730|2451755|2451725|Dr.|Annie|Grant|Y|25|5|1925|TUVALU||Annie.Grant@tccug5KC1oT2nL.com|2452401| +76|AAAAAAAAMEAAAAAA|526064|2054|9691|2451760|2451730|Mr.|Craig|Lowry|Y|7|3|1946|WALLIS AND FUTUNA||Craig.Lowry@92zokgx8duX.org|2452430| +77|AAAAAAAANEAAAAAA|915180|2167|37154|2451632|2451602|Dr.|Darrin|Smith|N|30|5|1931|UNITED STATES||Darrin.Smith@Mti.edu|2452355| +78|AAAAAAAAOEAAAAAA|1092537|3677|7264|2449388|2449358|Ms.|Wanda|Davis|N|11|5|1940|NETHERLANDS ANTILLES||Wanda.Davis@I6s7DD86i6.edu|2452339| +79|AAAAAAAAPEAAAAAA|389494|3493|9539|2450863|2450833|Miss|Sandi|Tran|Y|10|6|1937|FRENCH GUIANA||Sandi.Tran@myikqStif1Q.edu|2452644| +80|AAAAAAAAAFAAAAAA|1499808|3891|44727|2450558|2450528|Mrs.|Eleanor|Evans|N|21|12|1948|C�TE D'IVOIRE||Eleanor.Evans@zxvr5rl.org|2452413| +81|AAAAAAAABFAAAAAA|728917|388|48306|2452562|2452532|Ms.|Jessica|Levesque|Y|17|7|1940|ALBANIA||Jessica.Levesque@06mGqI9mHG.org|2452414| +82|AAAAAAAACFAAAAAA|75627|5081|30088|2450357|2450327|Sir|Max|Mueller|N|26|2|1947|IRAQ||Max.Mueller@xqCZRBSrTGD6CBvXh.com|2452303| +83|AAAAAAAADFAAAAAA|976724|5574|40824|2451032|2451002|Miss|Daisy|Flynn|N|24|1|1943|NEW ZEALAND||Daisy.Flynn@288e6Z0csxJ.com|2452338| +84|AAAAAAAAEFAAAAAA|1250744|2821|40898|2449496|2449466|Mrs.|Ami|Montgomery|Y|2|5|1964|JAMAICA||Ami.Montgomery@VBSKqhL36j55.edu|2452526| +85|AAAAAAAAFFAAAAAA|1293499|37|10575|2449868|2449838|Miss|Michele|Baldwin|Y|30|10|1978|GEORGIA||Michele.Baldwin@sIVO1J4U.org|2452491| +86|AAAAAAAAGFAAAAAA|1428237|6963|38442|2451560|2451530|Dr.|Marvin|Matlock|Y|21|10|1983|BRUNEI DARUSSALAM||Marvin.Matlock@0FXEZp.org|2452633| +87|AAAAAAAAHFAAAAAA|1452824|1427|22030|2449404|2449374|Dr.|Kevin|White|N|2|7|1934|BENIN||Kevin.White@x9oTPjEI6AdDQ7n4l.edu|2452492| +88|AAAAAAAAIFAAAAAA|495575|131|42687|2450991|2450961|Dr.|Phyllis|Horner|N|20|8|1965|UZBEKISTAN||Phyllis.Horner@uQy.edu|2452403| +89|AAAAAAAAJFAAAAAA|694848|5383|26318|2451425|2451395|Mrs.|Helen|Macdonald|Y|15|3|1981|DOMINICA||Helen.Macdonald@3d4.com|2452626| +90|AAAAAAAAKFAAAAAA|417827|5083|45139|2451494|2451464|Mr.|Nathan|Pond|Y|7|5|1985|GUYANA||Nathan.Pond@nPh7drM687MhI.org|2452637| +91|AAAAAAAALFAAAAAA|827176|2441|14906|2450406|2450376|Miss|Heather|White|Y|28|3|1962|MARTINIQUE||Heather.White@3JitjmxYQnXAtCNAl.com|2452295| +92|AAAAAAAAMFAAAAAA|953084|5771|35211|2452582|2452552|Miss|Crystal|Ryan|Y|8|2|1973|ECUADOR||Crystal.Ryan@Ju2rO6u.com|2452294| +93|AAAAAAAANFAAAAAA|647375|6229|35836|2449341|2449311|Sir|Clyde|Williams|N|6|10|1927|FRENCH POLYNESIA||Clyde.Williams@en.com|2452510| +94|AAAAAAAAOFAAAAAA|451893|1990|33287|2449553|2449523|Mr.|Craig|Byrd|Y|14|5|1982|FRENCH POLYNESIA||Craig.Byrd@Dc0OEMXkvvuJ.com|2452445| +95|AAAAAAAAPFAAAAAA|796503|1663|16023|2452468|2452438|Ms.|Elizabeth|Hollingsworth|N|17|4|1943|GREECE||Elizabeth.Hollingsworth@lVpeDS5Rcs.com|2452584| +96|AAAAAAAAAGAAAAAA|1148074|6019|35611|2451505|2451475|Sir|Shaun|Lewis|N|22|10|1955|NIGERIA||Shaun.Lewis@MTRUPYFTXf9.com|2452395| +97|AAAAAAAABGAAAAAA|418763|102|49041|2452467|2452437|Mr.|Stewart|Ruffin|Y|19|5|1971|MYANMAR||Stewart.Ruffin@R7Mrx.edu|2452528| +98|AAAAAAAACGAAAAAA|574977|1615|43853|2450894|2450864|Dr.|David|Lewis|N|23|6|1965|KIRIBATI||David.Lewis@5mhvq.org|2452558| +99|AAAAAAAADGAAAAAA|622676|2152|17228|2451687|2451657|Sir|Austin|Tran|Y|9|12|1961|NAMIBIA||Austin.Tran@ect7cnjLsucbd.edu|2452437| +100|AAAAAAAAEGAAAAAA|1254468|6370|6672|2449148|2449118|Ms.|Jeannette|Johnson|Y|13|7|1958|BANGLADESH||Jeannette.Johnson@8BvSqgp.com|2452635| +101|AAAAAAAAFGAAAAAA|1730669|4190|464|2450159|2450129|Dr.|Jeffrey|Bruce|N|7|5|1951|MAURITANIA||Jeffrey.Bruce@SPZG.com|2452590| +102|AAAAAAAAGGAAAAAA|1250712|1207|4149|2450198|2450168|Ms.|Jodi|Silva|N|9|8|1937|CANADA||Jodi.Silva@lntBSGFbpEOSVs.com|2452443| +103|AAAAAAAAHGAAAAAA|1659630|5909|33035|2451586|2451556|Dr.|James|Porter|N|3|5|1947|AFGHANISTAN||James.Porter@3C1oBhj.com|2452359| +104|AAAAAAAAIGAAAAAA|1090695|3116|25490|2450355|2450325|Dr.|Benjamin|Johnson|Y|29|11|1936|BAHRAIN||Benjamin.Johnson@HL2ugJBTO.com|2452499| +105|AAAAAAAAJGAAAAAA|949850|383|46916|2452463|2452433|Dr.|Frank|Strain|Y|14|1|1945|VIRGIN ISLANDS, U.S.||Frank.Strain@MbOHByB.edu|2452378| +106|AAAAAAAAKGAAAAAA|341252|5949|18246|2451785|2451755|Miss|Anne|Massey|Y|3|6|1974|PORTUGAL||Anne.Massey@7XXGTvh.edu|2452602| +107|AAAAAAAALGAAAAAA|546775|1414|16806|2451094|2451064|Dr.|Charles|Robinson|N|11|10|1970|VANUATU||Charles.Robinson@SKBvOYuE06xlJ6.org|2452312| +108|AAAAAAAAMGAAAAAA|847273|1898|4123|2449106|2449076|Mr.|Alfred|Bryant|Y|18|4|1964|BURUNDI||Alfred.Bryant@TRiZbgq.com|2452283| +109|AAAAAAAANGAAAAAA|1487931|5544|39604|2452506|2452476|Sir|Alfred|Watson|Y|13|1|1989|PORTUGAL||Alfred.Watson@Q.org|2452644| +110|AAAAAAAAOGAAAAAA|578986|5158|17330|2449769|2449739|Dr.|David|Thornton|N|30|7|1974|EGYPT||David.Thornton@eIz6xPr53uo8.com|2452426| +111|AAAAAAAAPGAAAAAA|291167|4554|27188|2450460|2450430|Dr.|Allen|Merritt|Y|19|4|1937|SOMALIA||Allen.Merritt@3fsiZ.edu|2452481| +112|AAAAAAAAAHAAAAAA|1032747|4822|14144|2451513|2451483|Sir|Floyd|Kelly|N|6|3|1984|PORTUGAL||Floyd.Kelly@VYyRIjGro.org|2452299| +113|AAAAAAAABHAAAAAA|204333|5202|4871|2451982|2451952|Mrs.|Karen|Roman|N|14|4|1985|BOTSWANA||Karen.Roman@IxcYF9rKJKomt.com|2452507| +114|AAAAAAAACHAAAAAA|1185641|5735|7076|2449631|2449601|Dr.|Richard|Chang|Y|2|5|1988|MEXICO||Richard.Chang@VKy9d4gdkatVugH.edu|2452359| +115|AAAAAAAADHAAAAAA|1498128|6766|20604|2449264|2449234|Sir|Freddie|Rojas|N|23|10|1971|MALDIVES||Freddie.Rojas@EfcO6EJZhAFac2.com|2452377| +116|AAAAAAAAEHAAAAAA|1824586|1944|18502|2449121|2449091|Miss|Carolyn|Callaway|Y|15|8|1932|ARUBA||Carolyn.Callaway@PjDvifaFqz0EZ.com|2452353| +117|AAAAAAAAFHAAAAAA|1759097|4215|32980|2449143|2449113|Miss|Renee|Jones|Y|29|8|1967|ARUBA||Renee.Jones@CKGxUpmCMLnxLg3.edu|2452624| +118|AAAAAAAAGHAAAAAA|648018|1399|43533|2451091|2451061|Dr.|Polly|Marlow|Y|7|10|1954|BAHRAIN||Polly.Marlow@OkB7beq1dpZn5InXZ8.org|2452476| +119|AAAAAAAAHHAAAAAA|1111698|4858|22818|2449829|2449799|Mr.|Jack|Spriggs|Y|4|8|1935|SAUDI ARABIA||Jack.Spriggs@jtQVaqKJXZnPPp3.org|2452475| +120|AAAAAAAAIHAAAAAA|609294|6365|42236|2450799|2450769|Dr.|Dan|Layne|N|25|11|1966|NORFOLK ISLAND||Dan.Layne@8btahREthm2.com|2452633| +121|AAAAAAAAJHAAAAAA||3852|44960||2449480|Mrs.|||N|4||||||2452542| +122|AAAAAAAAKHAAAAAA|802359|2399|26839|2450519|2450489|Sir|Christopher|Magee|N|1|10|1927|BERMUDA||Christopher.Magee@YJGYG5Y00r.com|2452431| +123|AAAAAAAALHAAAAAA|1715811|1013|17015|2450151|2450121|Mr.|Issac|Vega|Y|26|9|1953|SENEGAL||Issac.Vega@BB3eo23hUchr.edu|2452578| +124|AAAAAAAAMHAAAAAA|1813357|5275|27993|2449506|2449476|Dr.|Katharine|Hope|Y|3|11|1972|ARGENTINA||Katharine.Hope@UAtskHDZYaf2GLKqd.edu|2452312| +125|AAAAAAAANHAAAAAA|184237|3375|24513|2451019|2450989|Mr.|Stephen|Bruton|Y|15|9|1976|NIGER||Stephen.Bruton@KyRk407i9d.com|2452606| +126|AAAAAAAAOHAAAAAA|1297001|7129|22057|2449697|2449667|Sir|Daniel|Underwood|Y|20|4|1929|CAMEROON||Daniel.Underwood@UFBLooOoyKm.org|2452596| +127|AAAAAAAAPHAAAAAA|1172846|1728|1410|2450366|2450336|Miss|Tina|Clay|Y|23|5|1988|SAINT LUCIA||Tina.Clay@9XXnkEUtJlAsiKH.com|2452326| +128|AAAAAAAAAIAAAAAA|1035624|727|30983|2451336|2451306|Dr.|Michele|Benitez|Y|21|10|1961|PARAGUAY||Michele.Benitez@TSo2.org|2452308| +129|AAAAAAAABIAAAAAA|675894|2919|18791|2449881|2449851|Dr.|Brandi|Daniels|N|14|7|1954|SAUDI ARABIA||Brandi.Daniels@45lz1nXM8yJ.org|2452425| +130|AAAAAAAACIAAAAAA|784063|1299|23382|2450940|2450910|Mr.|Julius|Locke|N|24|3|1940|CAPE VERDE||Julius.Locke@bQ.org|2452507| +131|AAAAAAAADIAAAAAA|21268|6537|8606|2452497|2452467|Mrs.|Christine|Walton|Y|6|12|1959|ISRAEL||Christine.Walton@v.com|2452514| +132|AAAAAAAAEIAAAAAA|73004|6105|15959|2452287|2452257|Miss|Delia|Holland|N|21|7|1950|MYANMAR||Delia.Holland@4p5.org|2452607| +133|AAAAAAAAFIAAAAAA|1746749|571|42040|2449794|2449764|Dr.|David|Mcnabb|Y|10|7|1981|KENYA||David.Mcnabb@LT4hqKbQOBdZlv9T.org|2452472| +134|AAAAAAAAGIAAAAAA|1466626|3546|38066|2451995|2451965|Dr.|Timothy|Mendoza|N|1|10|1957|BOTSWANA||Timothy.Mendoza@sTDpAuVIO0.com|2452647| +135|AAAAAAAAHIAAAAAA|770531|2849|16767|2450899|2450869|Ms.|Vivian|Elliott|N|19|7|1938|ROMANIA||Vivian.Elliott@2InpKDq1ZB.org|2452631| +136|AAAAAAAAIIAAAAAA|1585327|6456|15232|2449583|2449553|Ms.|Elizabeth|Horn|Y|5|2|1954|BELIZE||Elizabeth.Horn@07DoPCi67U8.edu|2452507| +137|AAAAAAAAJIAAAAAA|743641|4304|16167|2451243|2451213|Sir|Roger|Burns|Y|3|1|1933|BAHRAIN||Roger.Burns@ZfvcJQpboR.com|2452283| +138|AAAAAAAAKIAAAAAA|1584692|5491|18337|2449468|2449438|Miss|Marjorie|Smith|N|3|4|1978|WALLIS AND FUTUNA||Marjorie.Smith@pT4LYd9jYKp9gZPSS.org|2452404| +139|AAAAAAAALIAAAAAA|1676720|5931|43895|2450260|2450230|Dr.|Catherine|Newman|N|29|3|1938|BERMUDA||Catherine.Newman@XYU8uAboQoTY35lq.org|2452497| +140|AAAAAAAAMIAAAAAA|1023202|3414|47226|2451434|2451404|Dr.|Debra|Ferguson|N|15|5|1936|UNITED ARAB EMIRATES||Debra.Ferguson@n.org|2452570| +141|AAAAAAAANIAAAAAA|1482167|4206|17798|2450487|2450457|Sir|Roberto|Johnson|N|24|6|1934|KIRIBATI||Roberto.Johnson@Ix1LO6c.org|2452323| +142|AAAAAAAAOIAAAAAA|1656828|6260|14892|2449129|2449099|Dr.|Marie|Clark|N|13|6|1959|SAINT HELENA||Marie.Clark@vB.edu|2452574| +143|AAAAAAAAPIAAAAAA|1659500|5465|27383|2450934|2450904|Mr.|Gilberto|Bennett|N|24|7|1950|NEW CALEDONIA||Gilberto.Bennett@M9cDJqGnuMVN.edu|2452292| +144|AAAAAAAAAJAAAAAA|1390646|5861|5476|2451219|2451189|Dr.|Cecil|Patterson|Y|12|1|1963|INDONESIA||Cecil.Patterson@PG48D3r9RkzS5cHF.org|2452619| +145|AAAAAAAABJAAAAAA|632398|2601|23204|2449213|2449183|Mr.|Lucas|Dewey|N|15|8|1938|JERSEY||Lucas.Dewey@ZHJkTXkTSxf7VtfaK.org|2452305| +146|AAAAAAAACJAAAAAA|99608|812|22816|2451504|2451474|Miss|Sue|Garner|N|25|8|1984|TOGO||Sue.Garner@nj7MaJfQVn2XnU.com|2452598| +147|AAAAAAAADJAAAAAA|584601|124|22494|2451306|2451276|Mr.|David|Bell|Y|1|4|1945|TIMOR-LESTE||David.Bell@mczTttGdMUhP.edu|2452613| +148|AAAAAAAAEJAAAAAA|13731|4895|45659|2451936|2451906|Dr.|Timothy|Randall|Y|22|9|1974|GUADELOUPE||Timothy.Randall@Iha.org|2452540| +149|AAAAAAAAFJAAAAAA|1092922|6291|4413|2450192|2450162|Miss|Anthony|Bell|N|30|11|1992|GRENADA||Anthony.Bell@EK1UOvs.com|2452607| +150|AAAAAAAAGJAAAAAA|170732|5380|37675|2452516|2452486|Sir|Carlos|Rivers|N|17|4|1951|GIBRALTAR||Carlos.Rivers@kXExbrcHQr7.com|2452463| +151|AAAAAAAAHJAAAAAA|1268169|1470|18724|2452131|2452101|Ms.|Inez|Britt|N|1|7|1981|INDONESIA||Inez.Britt@iuP18200P.com|2452441| +152|AAAAAAAAIJAAAAAA|11202|2418|33587|2451121|2451091|Dr.|Charles|Valdez|Y|31|5|1957|SOLOMON ISLANDS||Charles.Valdez@BZ5SNzn.edu|2452535| +153|AAAAAAAAJJAAAAAA|1553082|2967|39299|2451026|2450996|Sir|Oscar|Rodriguez|N|1|3|1956|AMERICAN SAMOA||Oscar.Rodriguez@cl3gRATjot.edu|2452357| +154|AAAAAAAAKJAAAAAA|615984|2520|5456|2451372|2451342|Mr.|Oscar|Jolley|Y|4|10|1947|SWAZILAND||Oscar.Jolley@fXYAyXeJCf.edu|2452283| +155|AAAAAAAALJAAAAAA|116723|2035|32147|2452341|2452311|Mr.|Robert|Bailey|Y|26|4|1952|LATVIA||Robert.Bailey@Kg2HymHYnhiFQ.com|2452308| +156|AAAAAAAAMJAAAAAA|1239996|3753|33463|2449910|2449880|Mrs.|Ebony|Herrera|N|24|7|1945|INDIA||Ebony.Herrera@FxPQDpja2A.edu|2452622| +157|AAAAAAAANJAAAAAA|140571|2515|26859|2452340|2452310|Dr.|Timothy|Dickey|N|23|11|1970|NIGERIA||Timothy.Dickey@4pz4mFsRXHDHn9Bsei.com|2452460| +158|AAAAAAAAOJAAAAAA|1766379|3371|15163|2451514|2451484|Mr.|James|Smith|N|8|5|1942|JAPAN||James.Smith@TdTbs.edu|2452422| +159|AAAAAAAAPJAAAAAA|1592543|2088|23280|2450169|2450139|Mrs.|Lydia|Parker|N|29|9|1941|TURKMENISTAN||Lydia.Parker@gaCKhtjHBv.com|2452539| +160|AAAAAAAAAKAAAAAA|100623|330|38084|2449929|2449899|Ms.|Jeanne|Fisher|Y|25|12|1966|QATAR||Jeanne.Fisher@o89qHfPiE4GRTPHU.org|2452439| +161|AAAAAAAABKAAAAAA|555176|3982|9188|2449566|2449536|Ms.|Wendy|Chiu|Y|25|11|1957|TIMOR-LESTE||Wendy.Chiu@ETOB393IPN.org|2452555| +162|AAAAAAAACKAAAAAA|265628|2401|26021|2452594|2452564|Dr.|Pamela|Perez|Y|24|3|1954|INDONESIA||Pamela.Perez@Xanx2Z6eDZY2K68.com|2452373| +163|AAAAAAAADKAAAAAA|1349987|3914|36241|2449332|2449302|Ms.|Carly|Chavez|Y|18|12|1941|TIMOR-LESTE||Carly.Chavez@oIeMa8xK7R.org|2452552| +164|AAAAAAAAEKAAAAAA|1561803|6070|39537|2449656|2449626|Miss|Kathleen|Keegan|N|24|10|1938|TOKELAU||Kathleen.Keegan@CVIZzgGq.org|2452627| +165|AAAAAAAAFKAAAAAA|171221|4745|28622|2449501|2449471|Miss|Eleanor|Muller|N|4|9|1966|VIRGIN ISLANDS, U.S.||Eleanor.Muller@pv3YT9DSjUHYPhA.edu|2452585| +166|AAAAAAAAGKAAAAAA|277430|4819|11855|2451862|2451832|Mr.|Richard|Taylor|Y|13|10|1992|NAURU||Richard.Taylor@IPGGsdy9uatPzD.edu|2452506| +167|AAAAAAAAHKAAAAAA|995414|653|12692|2452617|2452587|Dr.|Ricky|Lee|Y|8|1|1948|BAHRAIN||Ricky.Lee@6tjVJREkz3m.org|2452478| +168|AAAAAAAAIKAAAAAA|823657|609|30361|2449510|2449480|Dr.|Walter|Clarke|Y|15|1|1924|GUYANA||Walter.Clarke@RKoQ39fyLG.edu|2452478| +169|AAAAAAAAJKAAAAAA|360308|3101|43494|2450702|2450672|Ms.|Samantha|Long|Y|21|5|1971|EL SALVADOR||Samantha.Long@DRyuV0NvXuboR.org|2452370| +170|AAAAAAAAKKAAAAAA|1514859|5547|11835|2450613|2450583|Sir|Randy|Wilson|N|19|2|1934|ARUBA||Randy.Wilson@Hz.org|2452393| +171|AAAAAAAALKAAAAAA|1147675|6180|9485|2450715|2450685|Dr.|Rosa|Nixon|N|7|6|1926|SOMALIA||Rosa.Nixon@ghkTsItbO5o8hKtVkdI.com|2452380| +172|AAAAAAAAMKAAAAAA||5588|1514|2450803||Miss|||||3|1924||||| +173|AAAAAAAANKAAAAAA|1028237|2827|29017|2450947|2450917|Ms.|Heather|Joseph|Y|17|7|1964|MONTSERRAT||Heather.Joseph@xS5p.edu|2452530| +174|AAAAAAAAOKAAAAAA|1883378|5789|32200|2450141|2450111|Miss|Linda|Murdock|Y|9|7|1981|EQUATORIAL GUINEA||Linda.Murdock@pJR.edu|2452323| +175|AAAAAAAAPKAAAAAA|1593693|4956|5866|2449185|2449155|Miss|Kathryn|Boyd|Y|6|5|1937|SAUDI ARABIA||Kathryn.Boyd@1HbD5gQXQFg.org|2452297| +176|AAAAAAAAALAAAAAA|1267506|1797|25688|2451536|2451506|Mrs.|Bonnie|Cunningham|N|12|2|1978|BERMUDA||Bonnie.Cunningham@aeB7sFe1xodAK.com|2452556| +177|AAAAAAAABLAAAAAA|1109447|932|19170|2449752|2449722|Sir|Joshua|Parr|Y|1|8|1949|MYANMAR||Joshua.Parr@8Nr76TFzm.org|2452472| +178|AAAAAAAACLAAAAAA|297805|986|14520|2451731|2451701|Ms.|Gladys|Clancy|Y|15|1|1926|BANGLADESH||Gladys.Clancy@9raQR8eDl.edu|2452489| +179|AAAAAAAADLAAAAAA|1500071|6369|24267|2450741|2450711|Dr.|Robert|Rand|N|21|9|1946|JAMAICA||Robert.Rand@gIsgOsXLveqH.org|2452457| +180|AAAAAAAAELAAAAAA|916887|6100|37666|2449694|2449664|Mr.|Robert|Larsen|Y|24|2|1983|JAPAN||Robert.Larsen@rT.com|2452475| +181|AAAAAAAAFLAAAAAA|999550|653|6956|2450511|2450481|Sir|Joseph|Riley|N|19|10|1972|BELIZE||Joseph.Riley@znxF.com|2452416| +182|AAAAAAAAGLAAAAAA|234442|6642|9289|2452326|2452296|Sir|Michael|Brown|N|27|12|1976|NIUE||Michael.Brown@scVqD1Ayq3.edu|2452425| +183|AAAAAAAAHLAAAAAA|365062|4506|7550|2450561|2450531|Sir|Peter|Collins|Y|4|2|1965|PERU||Peter.Collins@HyeA4GFuSt.org|2452466| +184|AAAAAAAAILAAAAAA|1029065|2586|3354|2452572|2452542|Mr.|Clifford|Flynn|N|8|1|1964|AZERBAIJAN||Clifford.Flynn@xj7u.org|2452630| +185|AAAAAAAAJLAAAAAA|874016|6988|25913|2449177|2449147|Miss|Yvette|Eastman|Y|12|1|1925|TONGA||Yvette.Eastman@C5F68ATco7.org|2452616| +186|AAAAAAAAKLAAAAAA|1642233|6554|36122|2451474|2451444|Miss|Marilyn|Chapa|Y|13|4|1968|SRI LANKA||Marilyn.Chapa@nGPdfzUIEoCqX.org|2452601| +187|AAAAAAAALLAAAAAA|1447273|2767|35061|2452415|2452385|Miss|Florence|Foster|Y|27|4|1974|SPAIN||Florence.Foster@9UuxIfghgPUq.com|2452306| +188|AAAAAAAAMLAAAAAA|1194851|1207|24363|2450899|2450869|Mr.|Vincent|Martinez|Y|4|12|1981|DENMARK||Vincent.Martinez@v.edu|2452284| +189|AAAAAAAANLAAAAAA|1373685|3805|40725|2452633|2452603|Sir|Bruce|Betz|Y|18|10|1961|MALDIVES||Bruce.Betz@T8cs6MI3.edu|2452354| +190|AAAAAAAAOLAAAAAA|1317424|2089|11554|2450929|2450899|Miss|Romona|Woodruff|Y|1|9|1964|KUWAIT||Romona.Woodruff@tst458XZmsz3k.com|2452302| +191|AAAAAAAAPLAAAAAA|553851|5174|2944|2452079|2452049|Sir|Jack|Marino|N|14|8|1977|BOUVET ISLAND||Jack.Marino@s4VlvxMMCr.com|2452479| +192|AAAAAAAAAMAAAAAA|624770|4667|30211|2451481|2451451|Sir|Corey|Quinones|Y|23|12|1962|ROMANIA||Corey.Quinones@iR983lsrbEXs3FGJC.org|2452442| +193|AAAAAAAABMAAAAAA|220795|4096|7844|2449375|2449345|Dr.|David|Neeley|N|2|11|1931|MALAWI||David.Neeley@RvD3OryEP.com|2452441| +194|AAAAAAAACMAAAAAA|555321|4747|19020|2451226|2451196|Sir|William|Faison|Y|22|4|1957|WESTERN SAHARA||William.Faison@Q.org|2452433| +195|AAAAAAAADMAAAAAA|923632|805|14378|2449196|2449166|Mrs.|Deborah|Terrell|N|24|5|1924|CAMBODIA||Deborah.Terrell@Ufs60H6gcr2gjy.edu|2452384| +196|AAAAAAAAEMAAAAAA|126554|2276|1551|2452628|2452598|Mr.|Shawn|Cowan|Y|13|7|1933|BOUVET ISLAND||Shawn.Cowan@Ryflqzn5RFr.org|2452586| +197|AAAAAAAAFMAAAAAA|1752208|3408|32208|2451503|2451473|Miss|Debra|Carter|Y|4|3|1957|GERMANY||Debra.Carter@AT1tanFkGTJ.edu|2452509| +198|AAAAAAAAGMAAAAAA|1827315|4982|23496|2452672|2452642|Dr.|Dusty|Bryan|Y|20|2|1988|NEW CALEDONIA||Dusty.Bryan@4SUMJMY50Fc.org|2452419| +199|AAAAAAAAHMAAAAAA|1279496|5584|42025|2452285|2452255|Mr.|Edwardo|Porter|Y|14|7|1950|BURKINA FASO||Edwardo.Porter@Juh2suS.org|2452474| +200|AAAAAAAAIMAAAAAA|78046|5077|26155|2451444|2451414|Dr.|Ted|Ingalls|N|17|6|1990|CHRISTMAS ISLAND||Ted.Ingalls@VoeeTfC04pGNt.com|2452471| +201|AAAAAAAAJMAAAAAA|32508|6523|42491|2450742|2450712|Dr.|James|Scott|Y|29|3|1988|CANADA||James.Scott@Cqo77afYrMqPEL.org|2452622| +202|AAAAAAAAKMAAAAAA|1372547|6820|33493|2450940|2450910|Dr.|Trena|Grant|N|21|9|1989|SAN MARINO||Trena.Grant@tvQJEJNtmx341m.org|2452383| +203|AAAAAAAALMAAAAAA|1358958|255|22752|2451001|2450971|Mr.|Clifford|Markham|N|16|4|1939|CHILE||Clifford.Markham@bUdzHp.edu|2452312| +204|AAAAAAAAMMAAAAAA|1454612|5591|5017|2449076|2449046|Mrs.|Christine|Hebert|Y|4|8|1960|HONDURAS||Christine.Hebert@LK7DsUezOy.com|2452299| +205|AAAAAAAANMAAAAAA|1657734|3679|44824|2452639|2452609|Dr.|Heriberto|Putnam|Y|8|10|1941|MALAYSIA||Heriberto.Putnam@8qZ5ecY.org|2452353| +206|AAAAAAAAOMAAAAAA|800878|6588|32745|2450083|2450053|Sir|Brian|Serna|Y|15|2|1989|SLOVENIA||Brian.Serna@jMrH3hytNT.com|2452431| +207|AAAAAAAAPMAAAAAA|37305|4470|33235|2451926|2451896|Sir|Floyd|Scroggins|Y|17|1|1974|FRENCH POLYNESIA||Floyd.Scroggins@zpx4V5VQ7jMTU.edu|2452444| +208|AAAAAAAAANAAAAAA|1181602|5473|39344|2449891|2449861|Miss|Karla|Rincon|Y|28|6|1961|MONTENEGRO||Karla.Rincon@R0.org|2452388| +209|AAAAAAAABNAAAAAA|251525|5744|13963|2449213|2449183|Mr.|Curtis|Williams|Y|28|7|1987|ANDORRA||Curtis.Williams@pNqB.org|2452433| +210|AAAAAAAACNAAAAAA|893704|6632|41960|2450462|2450432|Ms.|Diane|Blount|N|17|1|1972|MAYOTTE||Diane.Blount@x8.edu|2452502| +211|AAAAAAAADNAAAAAA|1075746|5285|40377|2451509|2451479|Miss|Barbara|Hernandez|Y|11|4|1961|NICARAGUA||Barbara.Hernandez@S.org|2452437| +212|AAAAAAAAENAAAAAA|938838|3570|28702|2450458||Dr.||Monk|||9|||||2452603| +213|AAAAAAAAFNAAAAAA|1596601|4645|49089|2451950|2451920|Miss|Christene|Mccullough|Y|8|10|1945|LUXEMBOURG||Christene.Mccullough@kalRTzJl8KjD5G7H.org|2452299| +214|AAAAAAAAGNAAAAAA|325205|3154|33103|2450585|2450555|Mr.|Archie|Mayes|Y|6|11|1941|NEW ZEALAND||Archie.Mayes@ktKG1s9VpMGY.edu|2452334| +215|AAAAAAAAHNAAAAAA|1040836|2700|39717|2450769|2450739|Miss|Lillian|Morgan|Y|22|10|1982|KUWAIT||Lillian.Morgan@53sRO4941QRyx.com|2452355| +216|AAAAAAAAINAAAAAA|359850|2530|2410|2451359|2451329|Mrs.|Jacqueline|Russo|Y|25|12|1943|KYRGYZSTAN||Jacqueline.Russo@VcjQALO78c.com|2452585| +217|AAAAAAAAJNAAAAAA|1397675|2358|34194|2449674|2449644|Dr.|James|Lackey|Y|24|6|1933|MALI||James.Lackey@tqHuV9.edu|2452362| +218|AAAAAAAAKNAAAAAA|247753|6176|2552|2449706|2449676|Sir|Cody|Fowler|Y|19|1|1961|GHANA||Cody.Fowler@C4PEjdXI1cvZVkupF.edu|2452546| +219|AAAAAAAALNAAAAAA|1095236|2503|9017|2449220|2449190|Sir|Roy|Aguilar|Y|23|10|1988|SWITZERLAND||Roy.Aguilar@haRtA0p.org|2452521| +220|AAAAAAAAMNAAAAAA|1845912|5357|36002|2452461|2452431|Sir|William|Chu|Y|18|5|1977|GUAM||William.Chu@jrDhML4b.org|2452522| +221|AAAAAAAANNAAAAAA|228357|6441|6925|2450142|2450112|Dr.|Kenny|Johnson|N|14|10|1963|BELIZE||Kenny.Johnson@pcbnkVHQtUA7.org|2452602| +222|AAAAAAAAONAAAAAA|1499262||26462||||Omar||Y|||1944|ITALY||Omar.Garcia@9lh519BY66jzAf.com|| +223|AAAAAAAAPNAAAAAA|497955|6872|18189|2452497|2452467|Dr.|Rudolph|Hutchins|Y|11|9|1958|LIBERIA||Rudolph.Hutchins@n2sVh5a3ykauteVNas.com|2452418| +224|AAAAAAAAAOAAAAAA|538767|4950|17980|2452470|2452440|Ms.|Adela|Rutherford|Y|14|12|1953|ALGERIA||Adela.Rutherford@X.com|2452333| +225|AAAAAAAABOAAAAAA|1057243|3356|43869|2450927|2450897|Dr.|Roscoe|Balderas|N|8|12|1935|CAMBODIA||Roscoe.Balderas@lcAXRX7pSO18v.edu|2452591| +226|AAAAAAAACOAAAAAA|994138||25117|2449452||||Lee||22|12|1925|NIUE||James.Lee@xFk9m.edu|| +227|AAAAAAAADOAAAAAA|1146454||12659||2451472|Miss|Angelia|Vaughn|N|15||1972|NIGERIA|||2452560| +228|AAAAAAAAEOAAAAAA|652264|6607|12150|2449832|2449802|Sir|Daniel|Croteau|N|18|2|1988|FRENCH POLYNESIA||Daniel.Croteau@KLNHYpbyM2i7xEQK.com|2452313| +229|AAAAAAAAFOAAAAAA|125162||27806|||||Skinner|N|7|||ETHIOPIA|||| +230|AAAAAAAAGOAAAAAA|779201|2365|35129|2449971|2449941|Miss|Shirlene|Guerra|N|8|3|1962|SOUTH AFRICA||Shirlene.Guerra@LvAxQ5TvOY85Vdc.edu|2452617| +231|AAAAAAAAHOAAAAAA|1200393|5286|45379|2451931|2451901|Ms.|Janet|Welch|Y|9|7|1991|PERU||Janet.Welch@K1dYk2cCcLC.org|2452375| +232|AAAAAAAAIOAAAAAA|1213087|4275|13463|2450582|2450552|Sir|David|Hudspeth|Y|30|9|1982|IRELAND||David.Hudspeth@vujJsnUszvZzR9zEG.org|2452532| +233|AAAAAAAAJOAAAAAA||2278|46475||2451382|Dr.|Harry||Y|29|6||||Harry.Dalton@DFOVaqZkzTp.edu|| +234|AAAAAAAAKOAAAAAA|1569035|959|10883|2451923|2451893|Mr.|Dustin|Perry||10|4|||||2452362| +235|AAAAAAAALOAAAAAA|1425007|42|35877|2452188|2452158|Mr.|Salvador|Ramirez|Y|20|1|1973|UZBEKISTAN||Salvador.Ramirez@cGDOfYJ.edu|2452298| +236|AAAAAAAAMOAAAAAA|272752|4670|27835|2449715|2449685|Dr.|Becky|Downey|N|30|7|1962|GIBRALTAR||Becky.Downey@LfVPF.org|2452510| +237|AAAAAAAANOAAAAAA|904909|7171|3433||||Sean||Y|1||1959||||2452493| +238|AAAAAAAAOOAAAAAA|1592906|1415|29477|2449229|2449199|Sir|Sean|Sanderson|N|26|2|1944|SWITZERLAND||Sean.Sanderson@0rAG1YgxruusVGnvNA.org|2452363| +239|AAAAAAAAPOAAAAAA|414449|2345|24241|2450501|2450471|Dr.|Elizabeth|Smith|N|10|3|1933|INDONESIA||Elizabeth.Smith@z.com|2452591| +240|AAAAAAAAAPAAAAAA|1548896|4636|34423|||Mrs.|Carrie||N|||1941|BERMUDA|||| +241|AAAAAAAABPAAAAAA|288616|5879|5542|2450397|2450367|Mr.|Scott|Ward|Y|25|2|1930|ANDORRA||Scott.Ward@CzpTDaCmmYVV.org|2452634| +242|AAAAAAAACPAAAAAA|611134|4572|7163|2452532|2452502|Dr.|Lance|Redman|N|13|2|1969|GUADELOUPE||Lance.Redman@EM69kK.edu|2452302| +243|AAAAAAAADPAAAAAA|756855|5748|44594|2450306|2450276|Sir|James|Catron|Y|4|12|1945|EQUATORIAL GUINEA||James.Catron@yYjRH2ryUMi70yXYk.org|2452616| +244|AAAAAAAAEPAAAAAA|845511|727|15896|2450938|2450908|Sir|Stephen|Morgan|Y|3|2|1942|BAHAMAS||Stephen.Morgan@cb0.org|2452475| +245|AAAAAAAAFPAAAAAA|196556|2258|33913|2452486|2452456|Dr.|Russell|Donnelly|Y|31|10|1982|KYRGYZSTAN||Russell.Donnelly@IjVh06eeAG8ixu9i.org|2452316| +246|AAAAAAAAGPAAAAAA|747052|286|46673|2449824|2449794|Mrs.|Rosalie|Low|Y|22|1|1930|ARGENTINA||Rosalie.Low@GKe6czSvZh.org|2452477| +247|AAAAAAAAHPAAAAAA|1350958|1489|29573|2450579|2450549|Dr.|Earl|Holden|N|16|9|1928|MAURITANIA||Earl.Holden@5USlfiaaA.com|2452357| +248|AAAAAAAAIPAAAAAA|1001304|301|12498|2449499|2449469|Mr.|Leonard|Munoz|Y|20|9|1952|LIBERIA||Leonard.Munoz@Dvd7KYB7s9.edu|2452582| +249|AAAAAAAAJPAAAAAA|87712|2767|9069|2452535|2452505|Ms.|Cindy|Jackson|N|8|8|1989|TUVALU||Cindy.Jackson@pKLhquF6mljh4uVx.com|2452549| +250|AAAAAAAAKPAAAAAA|493542|345|16301|2449052|2449022|Mr.|Derick|Stewart|N|3|2|1953|HONG KONG||Derick.Stewart@KOdZQdYU.org|2452355| +251|AAAAAAAALPAAAAAA|384131|1194|15410|2450010|2449980|Dr.|Richard|Craig|N|22|2|1979|SUDAN||Richard.Craig@Lm6xST9.com|2452476| +252|AAAAAAAAMPAAAAAA|281285|4556|274|2450642|2450612|Dr.|Troy|Brewer|N|5|7|1939|VIRGIN ISLANDS, U.S.||Troy.Brewer@ZDt1Qk5q2.com|2452607| +253|AAAAAAAANPAAAAAA|1585267|4376|27085|2449284|2449254|Sir|Robert|Walker|N|9|10|1931|BOTSWANA||Robert.Walker@EazZfkopxvr9j.com|2452615| +254|AAAAAAAAOPAAAAAA|1665044|3106|41521|2449402|2449372|Ms.|Tiffanie|Holliday|Y|7|3|1991|KIRIBATI||Tiffanie.Holliday@hZ.com|2452536| +255|AAAAAAAAPPAAAAAA|1583680|3768|11456|2451471|2451441|Mr.|Michael|Thompson|N|6|2|1949|BOUVET ISLAND||Michael.Thompson@D7P7H7a9Tfy9hu1M.org|2452457| +256|AAAAAAAAAABAAAAA|321240|4092|28544|2451932|2451902|Sir|Kevin|Garrison|N|5|6|1941|KENYA||Kevin.Garrison@n8j.com|2452442| +257|AAAAAAAABABAAAAA|1251965|3504|29043|2451051|2451021|Ms.|Petra|Brown|N|8|6|1961|LUXEMBOURG||Petra.Brown@MSq6yPx51zt3IioI.com|2452508| +258|AAAAAAAACABAAAAA|||45278|2451082||Sir|Clyde|Bernard||14||1974|PHILIPPINES||Clyde.Bernard@IlM4cxVZghXay.edu|2452640| +259|AAAAAAAADABAAAAA|945819|4358|40497|2450055|2450025|Mr.|Carl|Motley|N|26|9|1948|MALAYSIA||Carl.Motley@7XEsdRS2Qg.com|2452419| +260|AAAAAAAAEABAAAAA|1354896|5413|20283|2451576|2451546|Dr.|Steven|Deleon|Y|13|11|1957|SAUDI ARABIA||Steven.Deleon@AcFdH75siBXhu2.com|2452519| +261|AAAAAAAAFABAAAAA|692189|2930|22424|2450805|2450775|Miss|Honey|Raymond|N|28|11|1991|GRENADA||Honey.Raymond@DvgDPolBRk0bSs.org|2452418| +262|AAAAAAAAGABAAAAA|601411|2451|29091|2452213|2452183|Ms.|Angela|Mauro|Y|3|3|1937|TUVALU||Angela.Mauro@yjzVkHtZvKOTCmxrx.edu|2452371| +263|AAAAAAAAHABAAAAA|178882|5021|26807|2452449|2452419|Mrs.|Rebecca|Hermann|N|3|11|1972|VENEZUELA||Rebecca.Hermann@sYcaFy.com|2452409| +264|AAAAAAAAIABAAAAA|||18063|2450258|2450228||Claude|Stewart|N|27|3||FRENCH GUIANA||Claude.Stewart@cutXggd.edu|| +265|AAAAAAAAJABAAAAA|973825|1062|14346|2449728|2449698|Miss|Laura|Pham|N|7|3|1979|UNITED STATES||Laura.Pham@mxCbEghi5Y.com|2452289| +266|AAAAAAAAKABAAAAA|1134414|2313|43812|2451156|2451126|Sir|Walter|Martinez|Y|5|7|1968|GUYANA||Walter.Martinez@vQnVJXQ4seh9pLBDJ.com|2452376| +267|AAAAAAAALABAAAAA|1250943|4768|21746|2451369|2451339|Ms.|Dessie|Simms|Y|12|9|1931|BERMUDA||Dessie.Simms@t8fo.edu|2452373| +268|AAAAAAAAMABAAAAA|1462242|1291|29987|2449422|2449392|Ms.|Ann|Siegel|Y|3|10|1943|ZIMBABWE||Ann.Siegel@f3V7nPs.com|2452485| +269|AAAAAAAANABAAAAA|1408713|888|12710|2449663|2449633|Mr.|William|Burns|N|23|5|1943|MAYOTTE||William.Burns@ojrPxK.com|2452575| +270|AAAAAAAAOABAAAAA|1747702|2270|12085|2449066|2449036|Dr.|Diane|Molina|Y|16|1|1981|TOKELAU||Diane.Molina@Lu64EONTqV.edu|2452303| +271|AAAAAAAAPABAAAAA|974572|1459|33946|2449485|2449455|Ms.|Ashley|Wallace|N|27|10|1989|PANAMA||Ashley.Wallace@jGtH0oX.com|2452643| +272|AAAAAAAAABBAAAAA|58528|468|4408|2451613|2451583|Miss|Claudia|Butler|Y|3|8|1929|NETHERLANDS ANTILLES||Claudia.Butler@8p2VT6M.org|2452454| +273|AAAAAAAABBBAAAAA|916621|1316|47611|2450274|2450244|Sir|Everett|Jenkins|N|18|11|1941|MOLDOVA, REPUBLIC OF||Everett.Jenkins@8IGK3viZTO.edu|2452415| +274|AAAAAAAACBBAAAAA|1207798|5098|31598|2449045|2449015|Ms.|Helen|Gray|N|8|4|1983|SAINT LUCIA||Helen.Gray@qRV1qZc.com|2452311| +275|AAAAAAAADBBAAAAA|792731|883|38635|2451901|2451871|Mrs.|Janice|Kirby|Y|16|2|1949|CUBA||Janice.Kirby@qx.edu|2452526| +276|AAAAAAAAEBBAAAAA|1351960|2030|11253|2450868|2450838|Dr.|David|Moreau|N|31|5|1930|TRINIDAD AND TOBAGO||David.Moreau@lVcIFYe6e9i.com|2452528| +277|AAAAAAAAFBBAAAAA|530527|2728|31729|2450807|2450777|Mr.|Bernard|Bryant|Y|18|6|1988|GUATEMALA||Bernard.Bryant@Iz8KAURMPPB.com|2452550| +278|AAAAAAAAGBBAAAAA|999018|6992|21413|2451200|2451170|Miss|Rachel|Hernandez|N|18|2|1934|SLOVAKIA||Rachel.Hernandez@3ndYYBHSLvT.org|2452342| +279|AAAAAAAAHBBAAAAA|408201|1504|2807|2449574|2449544|Dr.|William|Kaye|N|12|2|1961|AZERBAIJAN||William.Kaye@Mx1n74bCJjkc08.edu|2452510| +280|AAAAAAAAIBBAAAAA|472007|6069|6084|2449053|2449023|Dr.|Christopher|Randolph|Y|12|6|1967|NEPAL||Christopher.Randolph@HJhyNrdvkKzHVG.org|2452534| +281|AAAAAAAAJBBAAAAA|363108|7109|27110|2450273|2450243|Dr.|Tommy|Whitney|Y|30|4|1929|BOLIVIA||Tommy.Whitney@PoZcVZzeLj.org|2452359| +282|AAAAAAAAKBBAAAAA|817596||169|||Sir||Neal|N||||MALDIVES|||| +283|AAAAAAAALBBAAAAA|1531961|4221|8128|2452331|2452301|Dr.|Joseph|Hilton|Y|2|12|1975|BERMUDA||Joseph.Hilton@io2VIr.org|2452283| +284|AAAAAAAAMBBAAAAA|553655|4881|42460|2451545|2451515|Mrs.|Adam|Chambers|Y|13|6|1961|MONTSERRAT||Adam.Chambers@1fmL5RoSch74.edu|2452298| +285|AAAAAAAANBBAAAAA|311786|6886|24786|2449844|2449814|Sir|Stephen|Burton|N|25|12|1954|BELIZE||Stephen.Burton@vUZLt1J.edu|2452424| +286|AAAAAAAAOBBAAAAA|428772|3923|28274|2450267|2450237|Dr.|Ima|Amos|N|16|9|1982|KUWAIT||Ima.Amos@hEZu94B6kdfOrsAPA.edu|2452481| +287|AAAAAAAAPBBAAAAA|463191|366|25094|2452654|2452624|Dr.|Carolyn|Wong|N|15|2|1940|MALDIVES||Carolyn.Wong@qDMA4gY.edu|2452290| +288|AAAAAAAAACBAAAAA|338763|6861|11359|2451174|2451144|Dr.|Ruth|Duarte|Y|27|1|1954|ANGOLA||Ruth.Duarte@aYJinh7bzQiMP.org|2452576| +289|AAAAAAAABCBAAAAA|401587|2758|25529|2450500|2450470|Dr.|Nancy|Jones|N|1|6|1935|LITHUANIA||Nancy.Jones@Q6kuq.org|2452568| +290|AAAAAAAACCBAAAAA|238104|2602|31100|2449668|2449638|Mrs.|Florence|Caro|N|23|2|1983|MOZAMBIQUE||Florence.Caro@mACMoIsUY.org|2452362| +291|AAAAAAAADCBAAAAA|244059|6378|13544|2449302|2449272|Mr.|Ralph|Johnson|Y|2|11|1968|NICARAGUA||Ralph.Johnson@uuCPEDT5B8oV0ll.edu|2452312| +292|AAAAAAAAECBAAAAA|126296|4363|23677|2450549|2450519|Mr.|Billy|Coffin|N|15|5|1949|SWITZERLAND||Billy.Coffin@DrhdyQf.com|2452331| +293|AAAAAAAAFCBAAAAA|1521828|4146|23264|2449716|2449686|Ms.|Heather|Brown|N|10|6|1955|WALLIS AND FUTUNA||Heather.Brown@g.org|2452488| +294|AAAAAAAAGCBAAAAA||1222|19272|2449373|2449343||||||||NETHERLANDS||Thomas.Holder@bQnouG.com|| +295|AAAAAAAAHCBAAAAA|903118|5620|46969|2451540|2451510|Dr.|Michael|Mcguire|Y|2|6|1928|IRELAND||Michael.Mcguire@iTFQNbciZ3CdSv8.org|2452599| +296|AAAAAAAAICBAAAAA|654664|3131|13645|2449671|2449641|Miss|Christy|Hudson|Y|1|10|1940|TIMOR-LESTE||Christy.Hudson@8HMH0C2ydsabR9.edu|2452487| +297|AAAAAAAAJCBAAAAA|394444|2855|40466|2449499|2449469|Mr.|Lee|Farias|Y|24|6|1954|SWEDEN||Lee.Farias@BJN5bHOoiKDGUgN.org|2452390| +298|AAAAAAAAKCBAAAAA|1336062|2437|20012|2450213|2450183|Ms.|Iris|Roberts|N|25|5|1937|BENIN||Iris.Roberts@golv5Jj9oga.com|2452559| +299|AAAAAAAALCBAAAAA|273612|539|10266|2449936|2449906|Dr.|Gerald|Stone|N|24|6|1990|YEMEN||Gerald.Stone@1qgrObT0Cm.com|2452495| +300|AAAAAAAAMCBAAAAA|948309|5015|46921|2452539|2452509|Dr.|Gina|Batson|N|14|8|1927|SURINAME||Gina.Batson@vTFlyNS.com|2452515| +301|AAAAAAAANCBAAAAA|1174819|2514|34962|2452559|2452529|Sir|Richard|Whitaker|N|8|9|1940|GUERNSEY||Richard.Whitaker@NLpYdlQZJ9vxK71zQA.com|2452498| +302|AAAAAAAAOCBAAAAA|377497|3697|26250|2450610|2450580|Mrs.|Concepcion|Pugh|Y|23|11|1989|MOLDOVA, REPUBLIC OF||Concepcion.Pugh@v1Xq.edu|2452564| +303|AAAAAAAAPCBAAAAA|1670529|1189|13049|2450726|2450696|Miss|Luz|Ransom|N|17|9|1934|ANDORRA||Luz.Ransom@fXu5MmHo2tfK.edu|2452559| +304|AAAAAAAAADBAAAAA|1753564|6022|32833|2450818|2450788|Mr.|James|Sherrod|Y|27|11|1945|ALBANIA||James.Sherrod@7AC.edu|2452620| +305|AAAAAAAABDBAAAAA|55400|4041|28329|2449215|2449185|Sir|Matt|Santana|Y|17|10|1938|TAJIKISTAN||Matt.Santana@KcUsJpvRUM.org|2452398| +306|AAAAAAAACDBAAAAA|188927|6098|31269|2449556|2449526|Dr.|Peter|Smith|Y|4|12|1956|SINGAPORE||Peter.Smith@pM.org|2452383| +307|AAAAAAAADDBAAAAA|800479|4455|31231|2450833|2450803|Dr.|Kathy|Stone|Y|16|3|1984|CUBA||Kathy.Stone@6tAl8V5eeCO.org|2452287| +308|AAAAAAAAEDBAAAAA|771775|5417|49106|2450368|2450338|Dr.|Ashley|Thompson|N|12|1|1973|KENYA||Ashley.Thompson@uH34bg8ike.com|2452406| +309|AAAAAAAAFDBAAAAA|850062|6201|38340|2450519|2450489|Dr.|John|Blair|Y|29|5|1957|FRENCH POLYNESIA||John.Blair@Vofq58ICABQD.org|2452448| +310|AAAAAAAAGDBAAAAA|1537945|578|11611|2449965|2449935|Sir|John|Nunes|N|8|3|1982|NAURU||John.Nunes@gYhAO49St7Fv.org|2452627| +311|AAAAAAAAHDBAAAAA|713772|358|29744|2450387|2450357|Mr.|Darrell|Yates|Y|8|10|1924|GEORGIA||Darrell.Yates@igIz.edu|2452552| +312|AAAAAAAAIDBAAAAA||1930|10249|2452424|2452394|Sir||Lee|Y|||1951|||John.Lee@tceMrGJDjXzf.edu|2452346| +313|AAAAAAAAJDBAAAAA|1058645|7058|21424|||Ms.||Foster|N|||1966||||| +314|AAAAAAAAKDBAAAAA|1217705|5507|46109|2452326|2452296|Dr.|Anthony|White|Y|9|9|1951|UNITED STATES||Anthony.White@ufGQQoY5T.org|2452456| +315|AAAAAAAALDBAAAAA|441840|2629|37018|2452478|2452448|Sir|Ross|Burgess|N|5|11|1938|SINGAPORE||Ross.Burgess@n9.org|2452348| +316|AAAAAAAAMDBAAAAA|1461794|6320|42399|2451094|2451064|Sir|Benjamin|Smith|Y|11|8|1983|UZBEKISTAN||Benjamin.Smith@TQIZcUOhJvjl6.org|2452329| +317|AAAAAAAANDBAAAAA|338928|2171|24614|2452630|2452600|Miss|Anthony|Silva|Y|16|4|1986|ISLE OF MAN||Anthony.Silva@O.org|2452401| +318|AAAAAAAAODBAAAAA|427268|3885|4949|2452302|2452272|Ms.|Rosemary|Anderson|N|2|6|1955|ARGENTINA||Rosemary.Anderson@Zlu1Omiy.com|2452329| +319|AAAAAAAAPDBAAAAA|318924|4311|12451|2450847|2450817|Ms.|Evelyn|Ritchey|N|1|3|1950|NEW ZEALAND||Evelyn.Ritchey@8Kz4hM4xDYMDE.edu|2452433| +320|AAAAAAAAAEBAAAAA|959669|1738|43054||2451927||Brian||Y||10||NORFOLK ISLAND||Brian.Hall@agYPsNGu4G.org|2452541| +321|AAAAAAAABEBAAAAA|1187642|4386|17977|2451443|2451413|Ms.|Jerrie|Vickers|N|24|9|1942|ZAMBIA||Jerrie.Vickers@0Ez.org|2452319| +322|AAAAAAAACEBAAAAA|367590|256|43369|2449888|2449858|Miss|Sunshine|Solomon|Y|15|2|1928|R�UNION||Sunshine.Solomon@KqIOuo0jiM9KN.com|2452392| +323|AAAAAAAADEBAAAAA|981780|4178|42223|2449070|2449040|Dr.|Benton|Hite|Y|28|10|1961|NEPAL||Benton.Hite@Q1cdHxR5.edu|2452337| +324|AAAAAAAAEEBAAAAA|1312042|3777|45512|2450595|2450565|Miss|Monica|Holloway|Y|25|9|1958|ZAMBIA||Monica.Holloway@r5Ad8H3ak8Rx.org|2452418| +325|AAAAAAAAFEBAAAAA|1114717|6052|48634|2449153|2449123|Mr.|Fred|Shipley|N|16|5|1955|GEORGIA||Fred.Shipley@Kd5YTcBtnxJq.com|2452438| +326|AAAAAAAAGEBAAAAA|184072|2151|29477|2452616|2452586|Mr.|Kenneth|Cox|Y|13|10|1979|QATAR||Kenneth.Cox@3T0.org|2452397| +327|AAAAAAAAHEBAAAAA|1112009|3205|41648|2450252|2450222|Sir|Jeffrey|Wiles|Y|29|7|1950|CHILE||Jeffrey.Wiles@02H7IkvCIK0RLm9j.com|2452421| +328|AAAAAAAAIEBAAAAA|1516390|4936|43882|2452136|2452106|Dr.|Jose|Larson|N|23|4|1989|SRI LANKA||Jose.Larson@C6hK4P2Z3l8pLU.com|2452644| +329|AAAAAAAAJEBAAAAA|1226826|5940|17807|2450444|2450414|Sir|Christopher|Rodriguez|N|3|1|1928|BELIZE||Christopher.Rodriguez@d2spQlKT0UinRu.edu|2452291| +330|AAAAAAAAKEBAAAAA|1515958|6778|39245|2451792|2451762|Mr.|Johnny|Young|Y|12|6|1955|TUVALU||Johnny.Young@6Q2evME.com|2452517| +331|AAAAAAAALEBAAAAA|1665094|5328|33609|2449089|2449059|Mrs.|Agnes|Hinton|Y|16|11|1981|CAPE VERDE||Agnes.Hinton@sf.edu|2452408| +332|AAAAAAAAMEBAAAAA|1210128|2936|34897|2450678|2450648|Sir|John|Mcdonald|Y|5|2|1980|SERBIA||John.Mcdonald@gTHUZ4GQNrMTn.com|2452391| +333|AAAAAAAANEBAAAAA|1234328|113|47098|2450844|2450814|Ms.|Veronica|Miller|N|14|2|1948|MONGOLIA||Veronica.Miller@5LvN0V2jKYIb.org|2452340| +334|AAAAAAAAOEBAAAAA|842262|3262|21909|2449254|2449224|Miss|Donna|Hernandez|N|8|12|1926|EGYPT||Donna.Hernandez@2.org|2452441| +335|AAAAAAAAPEBAAAAA|632595||14557|2449899|2449869|||Williams|Y||||SLOVENIA|||2452538| +336|AAAAAAAAAFBAAAAA|1387009|7149|18136||2450112|Mrs.|Belle|Day|Y||6||VIRGIN ISLANDS, U.S.|||| +337|AAAAAAAABFBAAAAA|1239132|3915|6251|2452588|2452558|Dr.|Matthew|Lee|Y|9|12|1988|ISLE OF MAN||Matthew.Lee@UeRGgyKn6aOSfnqt.com|2452345| +338|AAAAAAAACFBAAAAA|1880655|3292|49129|2452592|2452562|Dr.|Kymberly|Conway|Y|13|8|1969|CROATIA||Kymberly.Conway@xuTkro.edu|2452446| +339|AAAAAAAADFBAAAAA|1711240|4430|37573|2450253|2450223|Mrs.|Barbara|Knutson|Y|31|5|1933|MADAGASCAR||Barbara.Knutson@JfJvyNh6QNHgB.edu|2452311| +340|AAAAAAAAEFBAAAAA|637211|3479|36100|2451521|2451491|Dr.|Richard|Mills|N|8|1|1970|MADAGASCAR||Richard.Mills@cvRPPv4s18.edu|2452358| +341|AAAAAAAAFFBAAAAA|||17801|||||Hart|Y|7||1963||||| +342|AAAAAAAAGFBAAAAA|1737114||4040||2449681|||Tolbert|N||11|1988|BELARUS|||2452607| +343|AAAAAAAAHFBAAAAA|671777|2089|40401|2450132|2450102|Ms.|Catherine|Andrews|Y|27|5|1975|GEORGIA||Catherine.Andrews@0TPXPO.edu|2452355| +344|AAAAAAAAIFBAAAAA|1306472|4639|22806|2449871|2449841|Sir|Stuart|Spencer|N|17|12|1939|LESOTHO||Stuart.Spencer@IQ2m5GJzdbn.com|2452340| +345|AAAAAAAAJFBAAAAA|118221|1173|3772|2452634|2452604|Miss|Jill|Wilkins|N|13|10|1981|BOUVET ISLAND||Jill.Wilkins@dTt9y7r24FJi.com|2452399| +346|AAAAAAAAKFBAAAAA|968766|676|4233|2451316|2451286|Ms.|Fiona|Jones|Y|9|2|1961|BHUTAN||Fiona.Jones@VJixuza2F.com|2452299| +347|AAAAAAAALFBAAAAA|567733|99|43618|2451368|2451338|Dr.|Wesley|Wallace|N|11|3|1951|R�UNION||Wesley.Wallace@PmaAJX4Ckt7bJ7EM.edu|2452523| +348|AAAAAAAAMFBAAAAA|922591|325|2906|2449371|2449341|Dr.|Pauline|Cyr|Y|20|8|1967|BARBADOS||Pauline.Cyr@DKkYSeYipVsYI.org|2452529| +349|AAAAAAAANFBAAAAA|1632911|6634|44310|2450034|2450004|Miss|Jeanette|Hewitt|N|4|5|1942|VENEZUELA||Jeanette.Hewitt@JmE0XR7uSroaPE.edu|2452456| +350|AAAAAAAAOFBAAAAA|1080591|7084|43930|2449912|2449882|Mrs.|Anne|Ellison|Y|25|5|1945|GUINEA-BISSAU||Anne.Ellison@9ZYFOvhRimgoxks81.org|2452455| +351|AAAAAAAAPFBAAAAA|1189803|1552|33701|2452263|2452233|Sir|Francisco|Hutchinson|N|24|11|1948|C�TE D'IVOIRE||Francisco.Hutchinson@9Ye3E2iGbtz.org|2452316| +352|AAAAAAAAAGBAAAAA|271428|5926|44777|2450464|2450434|Sir|Steven|Moore|N|5|10|1983|BULGARIA||Steven.Moore@XquVULA.com|2452346| +353|AAAAAAAABGBAAAAA|166856|1094|34132|2449938|2449908|Mr.|Michael|Littleton|Y|11|8|1947|SENEGAL||Michael.Littleton@AmSlxMG1ejNx.edu|2452311| +354|AAAAAAAACGBAAAAA|733562|3013|29311|2450947|2450917|Dr.|Jeanette|Huff|N|16|10|1947|MOZAMBIQUE||Jeanette.Huff@aInjAx6XZD.edu|2452303| +355|AAAAAAAADGBAAAAA|796680|3097|47114|2450717|2450687|Sir|Bruno|Butcher|Y|18|4|1930|ARMENIA||Bruno.Butcher@nkCHXxZO4Pu9T.org|2452539| +356|AAAAAAAAEGBAAAAA||6849|9949||2452586|Sir|Paul||N||||RUSSIAN FEDERATION||Paul.Prescott@ZVC.edu|2452523| +357|AAAAAAAAFGBAAAAA|1654661|3208|44155|2449523|2449493|Mrs.|Angie|Washington|N|17|2|1932|CAMEROON||Angie.Washington@7psIUGaS7cd.edu|2452550| +358|AAAAAAAAGGBAAAAA|41789|2178|42967|2451351|2451321|Dr.|Raymond|Gilmore|Y|12|11|1962|DENMARK||Raymond.Gilmore@0ONGxknGfrk.org|2452586| +359|AAAAAAAAHGBAAAAA|405574|4866|28558|2451427|2451397|Mr.|Daniel|Campbell|Y|20|9|1985|DOMINICA||Daniel.Campbell@aQsnd0O4vKmXz.org|2452320| +360|AAAAAAAAIGBAAAAA|395116|1600|4931|2449186|2449156|Sir|James|Qualls|Y|27|8|1985|KAZAKHSTAN||James.Qualls@I.org|2452368| +361|AAAAAAAAJGBAAAAA|214077|6680|17730|2451746|2451716|Miss|Bonnie|Harrison|Y|7|11|1973|EL SALVADOR||Bonnie.Harrison@AKYSOPlHEYkd.com|2452611| +362|AAAAAAAAKGBAAAAA|230588|6015|6642|2451460|2451430|Dr.|Herbert|Snowden|N|6|6|1960|ICELAND||Herbert.Snowden@DhFD.org|2452603| +363|AAAAAAAALGBAAAAA|739514|4393|6072|2452237|2452207|Sir|Bobby|Oneill|N|12|6|1977|ALAND ISLANDS||Bobby.Oneill@At6gpC6nCBNYZv.edu|2452578| +364|AAAAAAAAMGBAAAAA|591315|5896|49630|2452264|2452234|Mrs.|Hilda|Hall|N|29|7|1952|EL SALVADOR||Hilda.Hall@27N.edu|2452323| +365|AAAAAAAANGBAAAAA|801301|94|487|2451317|2451287|Dr.|Donald|May|Y|11|4|1943|TUNISIA||Donald.May@RSOlmpRUf6I9xlq.org|2452318| +366|AAAAAAAAOGBAAAAA|321741|6715|489|2452577|2452547|Mrs.|Damaris|Lane|Y|14|4|1991|TURKEY||Damaris.Lane@N6Ump4t6JS7IL.com|2452551| +367|AAAAAAAAPGBAAAAA|139973|4271|2569|2451539|2451509|Ms.|Aaron|Browder|Y|20|8|1992|ECUADOR||Aaron.Browder@iUpddkHI9z8.org|2452444| +368|AAAAAAAAAHBAAAAA|360046|907|22521|2451522|2451492|Dr.|Stanford|Glover|N|11|3|1948|SUDAN||Stanford.Glover@zb2YLXAm8vFIYj.com|2452601| +369|AAAAAAAABHBAAAAA|231483|5357|16831|2449464|2449434|Mr.|Arnold|Lewis|N|16|2|1944|AUSTRIA||Arnold.Lewis@1rzfdKmXg0N.edu|2452307| +370|AAAAAAAACHBAAAAA|654149|5536|32170|2451454|2451424|Miss|Freda|Becker|N|5|6|1934|UGANDA||Freda.Becker@KpPUmktJGY.com|2452501| +371|AAAAAAAADHBAAAAA|855011|4847|5930|2451327|2451297|Ms.|Beth|Gonzalez|Y|31|12|1954|SLOVAKIA||Beth.Gonzalez@a1URiAybjs7OlJA.edu|2452298| +372|AAAAAAAAEHBAAAAA|831818|2250|21082|2452435|2452405|Ms.|Maxine|Houston|N|3|1|1969|BELGIUM||Maxine.Houston@A77i7oKx.com|2452349| +373|AAAAAAAAFHBAAAAA|1688531|4258|14385|2450418|2450388|Dr.|Michael|Perkins|N|10|5|1937|SLOVAKIA||Michael.Perkins@VfiH2HUz1.edu|2452296| +374|AAAAAAAAGHBAAAAA|958238|2043|1048|2449655|2449625|Miss|Carla|Johnson|N|19|10|1949|HONG KONG||Carla.Johnson@7kKRNRYaDtt.org|2452474| +375|AAAAAAAAHHBAAAAA|674253|7056|9059|2450338|2450308|Mr.|David|Saunders|N|6|10|1977|SAUDI ARABIA||David.Saunders@7hKzvM3EF.edu|2452374| +376|AAAAAAAAIHBAAAAA|276865|1878|22891|2450565|2450535|Dr.|Ronald|Wilcox|Y|4|12|1934|TONGA||Ronald.Wilcox@i5bSOcz5otB.edu|2452458| +377|AAAAAAAAJHBAAAAA|1291493|6532|46018|2449716|2449686|Miss|Gloria|Davis|N|17|7|1927|PUERTO RICO||Gloria.Davis@x7p.com|2452543| +378|AAAAAAAAKHBAAAAA|1182816|5963|21463|2450021|2449991|Ms.|Gloria|Carlton|Y|29|12|1934|SURINAME||Gloria.Carlton@xNR1eEs.edu|2452433| +379|AAAAAAAALHBAAAAA|1009042|1624|4253|2450506|2450476|Dr.|Mark|Fields|N|16|4|1963|SAMOA||Mark.Fields@4Tar.org|2452585| +380|AAAAAAAAMHBAAAAA|787814|2108|20402|2450164|2450134|Mr.|Carl|Hoover|N|29|6|1957|EGYPT||Carl.Hoover@hNJoouFPa8znE.edu|2452548| +381|AAAAAAAANHBAAAAA|381945|1488|7784|2451072|2451042|Dr.|Colby|Robinson|N|12|6|1973|PAPUA NEW GUINEA||Colby.Robinson@51vtQ.edu|2452357| +382|AAAAAAAAOHBAAAAA|27884|6637|40237|2450325|2450295|Miss|Rose|Sturgill|Y|15|6|1958|TOGO||Rose.Sturgill@43K5L4ncfX.com|2452411| +383|AAAAAAAAPHBAAAAA|1657160|7162|18411|2452440|2452410|Miss|Linda|Isaacson|N|31|1|1985|MADAGASCAR||Linda.Isaacson@0koZn.org|2452604| +384|AAAAAAAAAIBAAAAA|850850|3481|8465|2449422|2449392|Dr.|Kimberly|Allen|Y|20|8|1969|BRUNEI DARUSSALAM||Kimberly.Allen@IKhh.edu|2452394| +385|AAAAAAAABIBAAAAA|427250|2590|46330|2451899|2451869|Miss|Susan|Hansen|Y|30|6|1949|TONGA||Susan.Hansen@AHnvfv05tS.org|2452340| +386|AAAAAAAACIBAAAAA|418446|2229|2812|2450472|2450442|Sir|Chris|King|N|3|12|1978|UNITED ARAB EMIRATES||Chris.King@6f3M0y1aZU6ca9m.com|2452520| +387|AAAAAAAADIBAAAAA|1435797|2345|30093|2452671|2452641|Mr.|Thomas|Brookins|Y|6|4|1967|MONGOLIA||Thomas.Brookins@iQ9ADlddsAV1.org|2452635| +388|AAAAAAAAEIBAAAAA|1421974|6998|14069|2450146|2450116|Sir|Kelly|Lee|Y|5|1|1939|MONACO||Kelly.Lee@6sQsIKA.org|2452394| +389|AAAAAAAAFIBAAAAA|1628503|5051|23646|2450310|2450280|Dr.|Scott|Kerns|N|27|7|1970|BARBADOS||Scott.Kerns@uDLBY.org|2452574| +390|AAAAAAAAGIBAAAAA|814546|521|6496|2449748|2449718|Mr.|Daniel|Madera|Y|5|5|1976|SOLOMON ISLANDS||Daniel.Madera@KbR57SpnEbSG.edu|2452494| +391|AAAAAAAAHIBAAAAA|136267|4256|24245|2449291|2449261|Miss|Eleanor|Bates|Y|29|1|1927|LATVIA||Eleanor.Bates@EA8dbU.edu|2452324| +392|AAAAAAAAIIBAAAAA|928857|249|20841|2451358|2451328|Ms.|Amanda|Cox|N|1|10|1986|MARTINIQUE||Amanda.Cox@vaEq.com|2452453| +393|AAAAAAAAJIBAAAAA|885446|3799|21705|2450058|2450028|Mr.|Bruce|Olvera|Y|2|2|1947|BELIZE||Bruce.Olvera@x4L7yzc2I65OB.org|2452329| +394|AAAAAAAAKIBAAAAA|1612064|413|38762|2450401|2450371|Ms.|Brenda|Miner|N|8|5|1979|PARAGUAY||Brenda.Miner@8L.edu|2452440| +395|AAAAAAAALIBAAAAA|313764|6637|44757|2450517|2450487|Miss|Rosemary|Alba|Y|2|12|1940|LEBANON||Rosemary.Alba@qHDd.org|2452575| +396|AAAAAAAAMIBAAAAA|1361490|6432|35774|2450147|2450117|Mrs.|Jennifer|Espinoza|Y|26|2|1943|LESOTHO||Jennifer.Espinoza@i.edu|2452486| +397|AAAAAAAANIBAAAAA|1876603|1292|36781|2450553|2450523|Sir|Colby|King|N|17|6|1949|ANTARCTICA||Colby.King@mm6GFyx7Tc8.org|2452633| +398|AAAAAAAAOIBAAAAA|1260072|2712|11399|2450592|2450562|Mrs.|Albert|Keller|N|27|5|1926|BELIZE||Albert.Keller@6lOVZ3e1hBQux0d.edu|2452368| +399|AAAAAAAAPIBAAAAA|244101|5107|46466|2450439|2450409|Sir|Raymond|Mccain|Y|5|11|1940|ECUADOR||Raymond.Mccain@bVusRLT6Ns09D.edu|2452595| +400|AAAAAAAAAJBAAAAA|1879023|5593|11716|2451982|2451952|Sir|Lance|Mason|N|20|5|1954|BANGLADESH||Lance.Mason@ImfD2XTJC.org|2452551| +401|AAAAAAAABJBAAAAA|843069|3069|37469|2452304|2452274|Dr.|Sarah|Clemons|Y|21|1|1927|ICELAND||Sarah.Clemons@IRzSDCIYrNOo.org|2452491| +402|AAAAAAAACJBAAAAA|212204|64|46724|2450423|2450393|Sir|William|Flores|Y|14|3|1961|ARGENTINA||William.Flores@txRxKKjKhlnCPZrgBd7.org|2452637| +403|AAAAAAAADJBAAAAA|857614|3891|15768|2451286|2451256|Dr.|Matthew|Nugent|Y|25|3|1961|ERITREA||Matthew.Nugent@vvFbE7sr.com|2452379| +404|AAAAAAAAEJBAAAAA|1501438|4276|21067|2450330|2450300|Dr.|Ethel|Frazier|Y|3|2|1956|BAHAMAS||Ethel.Frazier@1647IEdy6G.com|2452367| +405|AAAAAAAAFJBAAAAA|507618|6662|553|2449586|2449556|Ms.|Penelope|Wooten|Y|7|11|1990|HAITI||Penelope.Wooten@QlblIrPvGkkPU3j.org|2452631| +406|AAAAAAAAGJBAAAAA|835966|933|24347|2451303|2451273|Miss|Catherine|Foster|N|24|4|1948|CZECH REPUBLIC||Catherine.Foster@6di.com|2452565| +407|AAAAAAAAHJBAAAAA|386933|7171|9409|2451262|2451232|Dr.|Aaron|Taylor|Y|27|5|1981|CAYMAN ISLANDS||Aaron.Taylor@PUQRHMMUkx.com|2452611| +408|AAAAAAAAIJBAAAAA|467977|3568|10806|2452305|2452275|Dr.|Paul|Harry|Y|3|3|1987|SRI LANKA||Paul.Harry@B83pz32xhYGhXErBY.org|2452432| +409|AAAAAAAAJJBAAAAA|1282847|1692|37337|2450635|2450605|Dr.|Jason|Woodson|Y|3|4|1958|TAJIKISTAN||Jason.Woodson@FA.edu|2452638| +410|AAAAAAAAKJBAAAAA|704126|433|35893|2451144|2451114|Sir|Clayton|Rogers|Y|29|9|1963|CHRISTMAS ISLAND||Clayton.Rogers@gIe8oIdLxYeVd.edu|2452589| +411|AAAAAAAALJBAAAAA|198732|51|42970|2449431|2449401|Sir|Cleo|Spangler|N|17|7|1926|GUADELOUPE||Cleo.Spangler@dfn.org|2452285| +412|AAAAAAAAMJBAAAAA|738104|6479|30807|2449543|2449513|Mrs.|Dorothy|Billings|N|28|12|1987|NEW ZEALAND||Dorothy.Billings@Ov2FddK.edu|2452535| +413|AAAAAAAANJBAAAAA|373251|2333|6262|2451964|2451934|Mr.|Marc|Anderson|Y|8|12|1932|NORWAY||Marc.Anderson@bLxRfhp.com|2452312| +414|AAAAAAAAOJBAAAAA|970503|3587|9871|2452472|2452442|Dr.|Joseph|Moore|N|20|9|1976|ETHIOPIA||Joseph.Moore@glOxm7C5.com|2452345| +415|AAAAAAAAPJBAAAAA|1777255|6813|33524|2452364|2452334|Dr.|Teri|Anderson|Y|15|4|1941|HAITI||Teri.Anderson@fU5NINb11Ev5.com|2452639| +416|AAAAAAAAAKBAAAAA|1536891|2799|26754|2451257|2451227|Dr.|Chester|Velazquez|N|25|2|1982|CYPRUS||Chester.Velazquez@X8rPXKUpfpdvuF5.edu|2452427| +417|AAAAAAAABKBAAAAA|1709254|5912|14970|2451863|2451833|Ms.|Kayleen|Edwards|N|23|6|1942|FRANCE||Kayleen.Edwards@30KnGgPvZ.org|2452477| +418|AAAAAAAACKBAAAAA|163869|6073|8230|2449599|2449569|Dr.|Maurice|Morales|Y|28|1|1988|GREECE||Maurice.Morales@oypP.edu|2452382| +419|AAAAAAAADKBAAAAA|1277562|7136|23110|2451399|2451369|Dr.|Mattie|Kelly|N|25|8|1942|ERITREA||Mattie.Kelly@d3rtraUKLL.com|2452327| +420|AAAAAAAAEKBAAAAA||5634|17953|2449244||Mrs.||Dickson||17|4||||Jane.Dickson@UYQlvlLEHYZ17To8.edu|2452585| +421|AAAAAAAAFKBAAAAA|675771|3752|14533|2450708|2450678|Miss|Allison|Cosgrove|Y|2|9|1980|ARGENTINA||Allison.Cosgrove@MhRr7O.org|2452472| +422|AAAAAAAAGKBAAAAA|359297|1415|27180|2449355|2449325|Mr.|Matthew|Leake|Y|22|6|1953|GREENLAND||Matthew.Leake@PFavB6Sij.edu|2452446| +423|AAAAAAAAHKBAAAAA|157575|3479|40936|2449492|2449462|Sir|Craig|Sylvester|Y|26|3|1979|GRENADA||Craig.Sylvester@xuOcg3jA3.com|2452339| +424|AAAAAAAAIKBAAAAA|219181|415|14049|2449363|2449333|Miss|Diane|Gallo|N|7|8|1955|ANTIGUA AND BARBUDA||Diane.Gallo@rS1VzEoPTk.com|2452524| +425|AAAAAAAAJKBAAAAA|1688704|308|2712|2452658|2452628|Mr.|Danny|Donald|N|25|10|1982|ECUADOR||Danny.Donald@CAek6q.com|2452352| +426|AAAAAAAAKKBAAAAA|214637|3959|30881|2449961|2449931|Dr.|Bessie|Peek|N|26|4|1935|SEYCHELLES||Bessie.Peek@kC2QjuxH3eMQov.com|2452519| +427|AAAAAAAALKBAAAAA|1768613|5828|38243|2449460|2449430|Mrs.|Gladys|Hernandez|Y|2|12|1960|ERITREA||Gladys.Hernandez@P4Jptcuh9X.com|2452455| +428|AAAAAAAAMKBAAAAA|1783769|6483|23644|2450722|2450692|Mr.|Issac|Marcum|Y|21|11|1933|UKRAINE||Issac.Marcum@PuHv4R6.com|2452387| +429|AAAAAAAANKBAAAAA|241430|5512|17594|2451436|2451406|Dr.|Cynthia|Sutton|N|19|5|1960|JAMAICA||Cynthia.Sutton@QlKZtCLQvNHvg1QyjIc.org|2452429| +430|AAAAAAAAOKBAAAAA|856544|1774|38157|2451627|2451597|Dr.|Darrell|Davidson|N|17|4|1947|LEBANON||Darrell.Davidson@rnjAql2vC3iXThN.org|2452513| +431|AAAAAAAAPKBAAAAA|1893694|883|41672|2450638|2450608|Mr.|Ronald|Dunbar|Y|25|10|1926|KOREA, REPUBLIC OF||Ronald.Dunbar@zVI3Dbj.edu|2452439| +432|AAAAAAAAALBAAAAA|1767302|6528|25801|2450120|2450090|Sir|Cedric|Walton|Y|22|2|1968|WALLIS AND FUTUNA||Cedric.Walton@LbpmO5nOfp9xa.edu|2452460| +433|AAAAAAAABLBAAAAA|395255|2606|39557|2452010|2451980|Sir|Robert|Leatherman|N|1|11|1982|MONGOLIA||Robert.Leatherman@ch.com|2452504| +434|AAAAAAAACLBAAAAA|500127|827|33574|2449188|2449158|Dr.|Sammie|Clark|N|28|7|1947|MOLDOVA, REPUBLIC OF||Sammie.Clark@VgYQvHRdbs0oy4zCRI.com|2452620| +435|AAAAAAAADLBAAAAA|901820|966|1359|2450540|2450510|Mrs.|Doris|Ward|Y|15|11|1942|AFGHANISTAN||Doris.Ward@JL3sAguFtP.edu|2452617| +436|AAAAAAAAELBAAAAA|1887689|3587|13438|2449840|2449810|Dr.|Anthony|Mathis|Y|20|5|1976|UNITED ARAB EMIRATES||Anthony.Mathis@tAltfQzqNeXl7B08YdG.edu|2452575| +437|AAAAAAAAFLBAAAAA|1191560|3899|36285|2452522|2452492|Miss|Marie|Brown|N|20|5|1989|NEPAL||Marie.Brown@L23q5.edu|2452519| +438|AAAAAAAAGLBAAAAA|751122|4351|12380|2451623|2451593|Sir|Dennis|Thomas|Y|30|3|1965|BOTSWANA||Dennis.Thomas@9sh6o8pEo6p.org|2452283| +439|AAAAAAAAHLBAAAAA|1266601|1525|26049|2451942|2451912|Dr.|Helen|Culver|Y|4|2|1932|COSTA RICA||Helen.Culver@JIAGdZMQyMNRp9b.com|2452482| +440|AAAAAAAAILBAAAAA|1620574|3756|20985|2452037|2452007|Mr.|Julio|Barber|Y|31|8|1970|KOREA, REPUBLIC OF||Julio.Barber@KnfrPH00gZhpJtCK.edu|2452382| +441|AAAAAAAAJLBAAAAA|137255|6987|3072|2449734|2449704|Miss|Lan|Thompson|Y|14|7|1955|BOLIVIA||Lan.Thompson@qb2S22pXC.com|2452435| +442|AAAAAAAAKLBAAAAA|1223718|1010|17502|2449636|2449606|Miss|Shelley|Giordano|N|19|10|1962|ALBANIA||Shelley.Giordano@R9hV.edu|2452548| +443|AAAAAAAALLBAAAAA|308583|1065|27500|2450144|2450114|Sir|Jay|Johns|Y|11|2|1924|MACAO||Jay.Johns@ivjefkrUAVkq.org|2452317| +444|AAAAAAAAMLBAAAAA|388463|6797|22378|2450599|2450569|Ms.|Ashley|Douglas|N|19|3|1968|MALTA||Ashley.Douglas@KAkSIjvgiPr.com|2452377| +445|AAAAAAAANLBAAAAA|1357302|3464|32873|2450287|2450257|Dr.|Joseph|Norris|N|12|7|1948|AFGHANISTAN||Joseph.Norris@Epky6h9InLVhSKS.org|2452496| +446|AAAAAAAAOLBAAAAA|1205715|6654|31546|2451278|2451248|Dr.|Ines|Taylor|N|7|6|1944|SWAZILAND||Ines.Taylor@H5.org|2452369| +447|AAAAAAAAPLBAAAAA|950369|4511|29100|2449706|2449676|Ms.|Janet|Schramm|Y|5|4|1991|NAMIBIA||Janet.Schramm@USTIzry4drM.org|2452293| +448|AAAAAAAAAMBAAAAA|306418|5944|26317|2450044|2450014|Miss|Lynda|Lee|Y|30|10|1949|LEBANON||Lynda.Lee@IZ4mo.com|2452411| +449|AAAAAAAABMBAAAAA|1231040|2470|49053|2451391|2451361|Ms.|Kathryn|Mcknight|N|9|4|1958|SOLOMON ISLANDS||Kathryn.Mcknight@6vzNDlZe.edu|2452417| +450|AAAAAAAACMBAAAAA|308390|1126|33062|2449263|2449233|Mr.|Jesse|Benjamin|Y|24|9|1959|MADAGASCAR||Jesse.Benjamin@Q0qNUbMXd3nR8tnReo.edu|2452552| +451|AAAAAAAADMBAAAAA|46224|3042|11060|2449247|2449217|Mr.|Richard|Mcbride|Y|16|2|1978|BURUNDI||Richard.Mcbride@v2TcLI.com|2452536| +452|AAAAAAAAEMBAAAAA||2782|10522|||Mr.|Joel|Ferrell|Y||8||||Joel.Ferrell@nImcUhoobGaHyi.org|2452444| +453|AAAAAAAAFMBAAAAA|167918|1713|27108|2450766|2450736|Sir|Bobby|White|N|15|3|1934|LITHUANIA||Bobby.White@jU.org|2452536| +454|AAAAAAAAGMBAAAAA|743277|4747|3864|2451873|2451843|Dr.|Carrie|Hernandez|Y|29|8|1957|POLAND||Carrie.Hernandez@EXb5hvbyPELtI.edu|2452573| +455|AAAAAAAAHMBAAAAA|1135065|2843|11821|2450723|2450693|Miss|Eunice|Shaw|N|25|2|1989|CAPE VERDE||Eunice.Shaw@e2zrneFyn.com|2452321| +456|AAAAAAAAIMBAAAAA|805322|1413|10657|2451773|2451743|Mrs.|Kristin|Hargrove|N|8|11|1960|MONACO||Kristin.Hargrove@3AVTc6.com|2452508| +457|AAAAAAAAJMBAAAAA|1288961|6737|11313|2452015|2451985|Mrs.|Mariela|Hunter|N|2|1|1945|MALTA||Mariela.Hunter@E0bZpALLIN.org|2452376| +458|AAAAAAAAKMBAAAAA|343609|4865|26121|2450553|2450523|Mr.|Warren|Hendrickson|Y|23|6|1973|MAURITANIA||Warren.Hendrickson@kPI.org|2452618| +459|AAAAAAAALMBAAAAA|176999|5343|48040|2450433|2450403|Dr.|Howard|Chan|N|7|2|1925|LESOTHO||Howard.Chan@rf.org|2452423| +460|AAAAAAAAMMBAAAAA|1395786|1943|38853|2449180|2449150|Dr.|Veronica|Werner|Y|2|11|1991|CYPRUS||Veronica.Werner@uqmEal6Vur.org|2452490| +461|AAAAAAAANMBAAAAA|1402553|5671|21142|2451291|2451261|Mrs.|Hollie|Leonard|Y|17|6|1981|HUNGARY||Hollie.Leonard@0KDuOriqRByaKS.com|2452534| +462|AAAAAAAAOMBAAAAA|1647398|2233|25915|2450964|2450934|Mr.|Lawrence|Guevara|N|14|3|1934|CAMEROON||Lawrence.Guevara@p1xmOP76PO.com|2452440| +463|AAAAAAAAPMBAAAAA|456020|594|17615|2449343|2449313|Sir|Charlie|Deleon|N|9|1|1939|BELGIUM||Charlie.Deleon@HMd2qQxcgL0u.edu|2452473| +464|AAAAAAAAANBAAAAA|513341|4607|21520|2451979|2451949|Mr.|Derek|Murphy|N|19|9|1979|GRENADA||Derek.Murphy@me5OhGGeFU9iIH.com|2452421| +465|AAAAAAAABNBAAAAA|1569066|5280|1941|2451336|2451306|Dr.|Jeanette|Garcia|N|13|11|1932|FINLAND||Jeanette.Garcia@GVC.com|2452394| +466|AAAAAAAACNBAAAAA|1752934|2403|31919|2449967|2449937|Mrs.|Dottie|Rodriguez|N|30|8|1931|ISLE OF MAN||Dottie.Rodriguez@tA0kRC4m7FhZ.edu|2452499| +467|AAAAAAAADNBAAAAA|387105|1807|36909|2451710|2451680|Sir|Russell|Gentry|N|8|10|1968|CHILE||Russell.Gentry@P3DzsF2l1G2GZnc.com|2452535| +468|AAAAAAAAENBAAAAA||5506|49523|2451170|||Haywood||N|18||1983|MEXICO|||| +469|AAAAAAAAFNBAAAAA|1783561|387|49213|2452439|2452409|Dr.|Ambrose|Callahan|N|27|3|1930|NEW CALEDONIA||Ambrose.Callahan@J.com|2452630| +470|AAAAAAAAGNBAAAAA|378740|3922|41104|2450271|2450241|Miss|Barbara|Lee|N|4|10|1941|SYRIAN ARAB REPUBLIC||Barbara.Lee@k8LOS1b2nAET9PJ1k.org|2452450| +471|AAAAAAAAHNBAAAAA|1321553|1258|32516|2449845|2449815|Ms.|Brigida|Doolittle|N|21|1|1971|MYANMAR||Brigida.Doolittle@eoJpnkSDI3EPFq0Tt.edu|2452474| +472|AAAAAAAAINBAAAAA|1497981|855|9329|2451862|2451832|Sir|Brian|Lewis|Y|3|7|1980|LATVIA||Brian.Lewis@MTeOx7iVgOxH7.edu|2452362| +473|AAAAAAAAJNBAAAAA|1344845|4584|43876|2452286|2452256|Dr.|Myrtle|Bennett|Y|25|8|1948|THAILAND||Myrtle.Bennett@OSVQ5PZc2sBv.edu|2452327| +474|AAAAAAAAKNBAAAAA|389417|5462|9154|2450897|2450867|Sir|Michael|Espinoza|N|16|12|1979|SEYCHELLES||Michael.Espinoza@ApPrES1deF0.edu|2452484| +475|AAAAAAAALNBAAAAA|111390|3118|1246|2452231|2452201|Mr.|Fletcher|Palmer|Y|17|10|1984|GERMANY||Fletcher.Palmer@XJMuVd85x.com|2452570| +476|AAAAAAAAMNBAAAAA|1778596|7070|33144|2451255|2451225|Dr.|Brian|Miller|Y|7|11|1980|LIECHTENSTEIN||Brian.Miller@0Kp9A5HsjSqh.org|2452375| +477|AAAAAAAANNBAAAAA|447338|5230|3522|2450108|2450078|Dr.|Sonia|Tyler|Y|11|5|1940|NEW ZEALAND||Sonia.Tyler@9CP3yASBLLjVikfIV.edu|2452558| +478|AAAAAAAAONBAAAAA|275408||34943||2449891||Leonel||N|7|||MALDIVES||Leonel.Chan@ZyOdGO8Uh.com|2452517| +479|AAAAAAAAPNBAAAAA|1482234|3592|48550|2449973|2449943|Sir|Jose|Coulter|Y|8|1|1980|PUERTO RICO||Jose.Coulter@31x6T2LmqSZI0Ipc8niP.com|2452365| +480|AAAAAAAAAOBAAAAA|176688|4794|853|2451894|2451864|Dr.|Jean|Fuller|Y|24|1|1971|LIECHTENSTEIN||Jean.Fuller@xozpIeBkSRLb6qE.org|2452511| +481|AAAAAAAABOBAAAAA|1489061|513|19565|2452129|2452099|Dr.|Shawn|Rowe|N|12|8|1990|UZBEKISTAN||Shawn.Rowe@AjX70QMy348YMA.com|2452376| +482|AAAAAAAACOBAAAAA|1706645|829|31013|2450083|2450053|Dr.|Brandi|Littlefield|N|20|5|1926|LIECHTENSTEIN||Brandi.Littlefield@z.com|2452534| +483|AAAAAAAADOBAAAAA|1111570|3143|9869|2452574|2452544|Miss|Nancy|Berry|Y|2|8|1991|INDIA||Nancy.Berry@fT4XP72HZbX.edu|2452322| +484|AAAAAAAAEOBAAAAA|929828|1732|47636|2451509|2451479|Miss|Amber|Hoffman|Y|24|11|1965|URUGUAY||Amber.Hoffman@023Gqdi76DfPzdhEOnN.com|2452447| +485|AAAAAAAAFOBAAAAA|1547872|4031|43837|2452023|2451993|Dr.|James|Jones|N|11|3|1964|HONG KONG||James.Jones@ESAV.edu|2452625| +486|AAAAAAAAGOBAAAAA|1592340|617|28047|2452192|2452162|Sir|Michael|Harvey|N|31|3|1985|AZERBAIJAN||Michael.Harvey@2aVFIg1.edu|2452422| +487|AAAAAAAAHOBAAAAA|274675|5270|33865|2451097|2451067|Ms.|Dolores|Williams|Y|28|12|1976|TOGO||Dolores.Williams@gVvAymI5F6JMIt.edu|2452607| +488|AAAAAAAAIOBAAAAA|1406126|848|19970|2450850|2450820|Ms.|Christina|Hoyt|Y|23|10|1943|SAUDI ARABIA||Christina.Hoyt@FKMFs6LITVe9f.com|2452328| +489|AAAAAAAAJOBAAAAA|324569|5099|22076|2452361|2452331|Dr.|Kristin|Mack|Y|11|10|1967|MADAGASCAR||Kristin.Mack@SX4IrT5eXlL.edu|2452508| +490|AAAAAAAAKOBAAAAA|1125378|6945|2512|2451233|2451203|Miss|Cheri|Alvarez|N|12|7|1935|DOMINICA||Cheri.Alvarez@nznGG7VuAZlTY.com|2452556| +491|AAAAAAAALOBAAAAA|1906868|1|2734|2451431|2451401|Ms.|Janice|Hopkins|N|22|8|1946|ITALY||Janice.Hopkins@yluYDIjKP0dm5.edu|2452293| +492|AAAAAAAAMOBAAAAA|50950|260|48370|2451924|2451894|Mr.|Dale|Young|N|4|6|1987|CYPRUS||Dale.Young@jNG.org|2452287| +493|AAAAAAAANOBAAAAA|769736|3668|49553|2450780|2450750|Miss|Machelle|Hopson|N|18|12|1926|LIECHTENSTEIN||Machelle.Hopson@o2QxL0F.org|2452306| +494|AAAAAAAAOOBAAAAA|1647873|6916|20359|2449797|2449767|Sir|Jason|Martin|Y|8|10|1987|GAMBIA||Jason.Martin@Rl4IggVJgCqV6.edu|2452347| +495|AAAAAAAAPOBAAAAA|1620000|5813|2379|2450937|2450907|Dr.|Marina|Foster|Y|18|4|1967|BAHRAIN||Marina.Foster@0Rjef6pgKOlPGp.com|2452383| +496|AAAAAAAAAPBAAAAA|53413|5789|6965|2451217|2451187|Mr.|James|Talley|N|30|3|1972|CAMBODIA||James.Talley@GBONatuo4.edu|2452502| +497|AAAAAAAABPBAAAAA|790767|3231|4142|2450947|2450917|Dr.|Joseph|Bradford|Y|27|2|1935|GUAM||Joseph.Bradford@6gamxNNilkt.org|2452628| +498|AAAAAAAACPBAAAAA|327419|6452|27132|2450963|2450933|Dr.|Jeffrey|Thompson|Y|17|9|1984|SERBIA||Jeffrey.Thompson@qIxmtJPA.com|2452555| +499|AAAAAAAADPBAAAAA|767647|4261|6122|2449996|2449966|Mr.|Jan|Hardy|Y|8|5|1957|SEYCHELLES||Jan.Hardy@cGQ.com|2452528| +500|AAAAAAAAEPBAAAAA|799625|4672|39272|2450476|2450446|Mr.|Charles|Cromwell|N|2|3|1966|LITHUANIA||Charles.Cromwell@dfbk.com|2452405| +501|AAAAAAAAFPBAAAAA|1031854|5724|3994|2451897|2451867|Miss|Felica|Brunner|Y|27|11|1944|SUDAN||Felica.Brunner@0XaJ26cuYhkiVEyekg.com|2452551| +502|AAAAAAAAGPBAAAAA|846749|3655|5808|2449605|2449575|Sir|Charles|Witte|N|17|2|1953|MALI||Charles.Witte@a.org|2452590| +503|AAAAAAAAHPBAAAAA|1788212|4692|10419|2449080|2449050|Dr.|Lisa|Zamora|Y|24|1|1927|SRI LANKA||Lisa.Zamora@2NDdfLk.edu|2452398| +504|AAAAAAAAIPBAAAAA|1854443|6825|3607|2449225|2449195|Dr.|Doris|Moreno|Y|21|8|1960|YEMEN||Doris.Moreno@87kHG5taAr.org|2452312| +505|AAAAAAAAJPBAAAAA|1317773|689|38358|2452435|2452405|Ms.|Jennifer|Reyes|Y|1|11|1965|EL SALVADOR||Jennifer.Reyes@1C97j1qs.org|2452591| +506|AAAAAAAAKPBAAAAA|432968|660|13552|2449334|2449304|Dr.|Lena|Dominguez|Y|3|1|1964|EGYPT||Lena.Dominguez@d0ZV4KAQ99S.org|2452451| +507|AAAAAAAALPBAAAAA|267382|1556|36977|2452343|2452313|Sir|Kermit|Thompson|Y|6|12|1936|UZBEKISTAN||Kermit.Thompson@G0Qx8CIpiQ.com|2452549| +508|AAAAAAAAMPBAAAAA|1860508|1037|33166|2451432|2451402|Mr.|Carlos|Garza|N|8|12|1948|BARBADOS||Carlos.Garza@r.edu|2452391| +509|AAAAAAAANPBAAAAA|1194101|2748|28595|2452363|2452333|Ms.|Helen|Wright|Y|3|8|1951|UNITED ARAB EMIRATES||Helen.Wright@eyR35TJ9ReGLd.edu|2452603| +510|AAAAAAAAOPBAAAAA|41475|3732|7989|2452230|2452200|Dr.|Tonya|Martinez|Y|12|12|1972|SWAZILAND||Tonya.Martinez@3.edu|2452643| +511|AAAAAAAAPPBAAAAA|282448|708|27368|2452668|2452638|Mr.|Emanuel|Davis|N|7|4|1954|ICELAND||Emanuel.Davis@XNChruRJk.com|2452331| +512|AAAAAAAAAACAAAAA|1000718|3199|43680|2451674|2451644|Mr.|Kevin|Burns|Y|9|5|1957|PITCAIRN||Kevin.Burns@2X.edu|2452510| +513|AAAAAAAABACAAAAA|521026|6418|9363|2452617|2452587|Mrs.|Claudia|Sterling|Y|23|9|1969|VENEZUELA||Claudia.Sterling@uG7OQPm8rnuf.com|2452475| +514|AAAAAAAACACAAAAA|1180816|3845|45274|2451921|2451891|Mrs.|Evelyn|Burt|Y|11|6|1928|GREECE||Evelyn.Burt@u2LNJ0QecTgdVKqeM.org|2452334| +515|AAAAAAAADACAAAAA|765128|2596|45264|2449575|2449545|Dr.|Pauline|Hammond|Y|31|5|1979|SWITZERLAND||Pauline.Hammond@Ry70gseLpx.org|2452507| +516|AAAAAAAAEACAAAAA|1047296|4668|41240|2449553|2449523|Mr.|James|Burch|Y|22|3|1985|UGANDA||James.Burch@VLTdUSFmSxSAi.com|2452362| +517|AAAAAAAAFACAAAAA|980301|4210|20119|2450573|2450543|Dr.|Laurence|Beyer|N|1|11|1940|MALI||Laurence.Beyer@OGZm1oP.edu|2452480| +518|AAAAAAAAGACAAAAA|1574960|1827|36741|2449242|2449212|Miss|Patti|Woods|N|12|11|1926|TOKELAU||Patti.Woods@JtiE.org|2452396| +519|AAAAAAAAHACAAAAA|730768|1756|41097|2450121|2450091|Mr.|Jose|Larson|Y|23|10|1958|ALBANIA||Jose.Larson@QeaR3d.com|2452356| +520|AAAAAAAAIACAAAAA|1437828|1254|21527|2452096|2452066|Miss|Minnie|Schwartz|Y|8|12|1925|CZECH REPUBLIC||Minnie.Schwartz@e20.com|2452344| +521|AAAAAAAAJACAAAAA|1510519|6195|22271|2452117|2452087|Dr.|Kim|Evans|Y|2|4|1987|LATVIA||Kim.Evans@xNfjL9xUEsqsTbOp.com|2452425| +522|AAAAAAAAKACAAAAA|62558|374|3837|2452157|2452127|Dr.|Gary|Lancaster|Y|15|11|1926|MACAO||Gary.Lancaster@36gSVHxpa3A.edu|2452393| +523|AAAAAAAALACAAAAA|1873615|448|26881|2452001|2451971|Sir|Robert|Ruiz|Y|1|10|1987|SRI LANKA||Robert.Ruiz@KsioJ0hQCn3.edu|2452622| +524|AAAAAAAAMACAAAAA|287648|6499|41350|2449664|2449634|Dr.|Sheldon|Hewitt|N|14|8|1986|SLOVAKIA||Sheldon.Hewitt@FunpALv2O4cj.com|2452453| +525|AAAAAAAANACAAAAA|1124332|6998|4464|2449300|2449270|Miss|Judith|Winkler|Y|28|7|1947|SAMOA||Judith.Winkler@hAehG8rv9Up.org|2452373| +526|AAAAAAAAOACAAAAA|||7429|2451470|||||N||1|||||| +527|AAAAAAAAPACAAAAA|434962|5613|36264|2449939|2449909|Mr.|Kenneth|Mitchell|Y|25|9|1984|UNITED ARAB EMIRATES||Kenneth.Mitchell@qh427iUFxQQHbfoD8.edu|2452323| +528|AAAAAAAAABCAAAAA|1654868|1759|20650|2451414|2451384|Mrs.|Tammara|Somerville|N|1|11|1971|SURINAME||Tammara.Somerville@XgC31.org|2452601| +529|AAAAAAAABBCAAAAA|1879432|1086|12340|2449900|2449870|Sir|Michael|Herring|Y|20|9|1932|AMERICAN SAMOA||Michael.Herring@jl5PTAS.com|2452300| +530|AAAAAAAACBCAAAAA|1544185|2063|31941|2451363|2451333|Sir|Herbert|Beach|Y|27|7|1938|ANGOLA||Herbert.Beach@HsiGbC3T.edu|2452566| +531|AAAAAAAADBCAAAAA|1113109|2798|11734|2452525|2452495|Mrs.|Flor|Braun|N|7|8|1939|ICELAND||Flor.Braun@hF4QuLRT6z.edu|2452380| +532|AAAAAAAAEBCAAAAA|1543653||35786||2449078|||||24||1980|||Perry.Adkins@ymlg.org|| +533|AAAAAAAAFBCAAAAA|1913607|5097|9340|2450684|2450654|Mr.|Douglas|Scott|Y|5|5|1962|LESOTHO||Douglas.Scott@IJY9YdRq6.edu|2452504| +534|AAAAAAAAGBCAAAAA|1180890|3483|16876|2452297|2452267|Dr.|Jerry|Preston|N|23|9|1990|GUINEA||Jerry.Preston@5zmmzgY.com|2452639| +535|AAAAAAAAHBCAAAAA|755409|6528|2094|2452273|2452243|Dr.|Vera|Prewitt|Y|17|6|1967|PANAMA||Vera.Prewitt@3hH2HVCSUd0.edu|2452323| +536|AAAAAAAAIBCAAAAA|1711094|5513|20332|2449039|2449009|Dr.|Jerald|Delarosa|N|25|9|1956|LIECHTENSTEIN||Jerald.Delarosa@G.edu|2452540| +537|AAAAAAAAJBCAAAAA|1646043|6056|4869|2452023|2451993|Miss|Kristin|Dominguez|Y|4|12|1967|CHINA||Kristin.Dominguez@KXzUrRIrYMU.org|2452621| +538|AAAAAAAAKBCAAAAA|929716|5850|36384|2452270|2452240|Miss|Margaret|Black|Y|17|1|1973|CANADA||Margaret.Black@C.org|2452545| +539|AAAAAAAALBCAAAAA|606100|108|516|2452052|2452022|Mr.|John|Jackson|N|4|3|1984|NETHERLANDS ANTILLES||John.Jackson@R7EVeyKho8.com|2452628| +540|AAAAAAAAMBCAAAAA|1325005|5801|40244|2449171|2449141|Mr.|Christopher|Henderson|Y|1|10|1968|GUATEMALA||Christopher.Henderson@77o6D15L.org|2452325| +541|AAAAAAAANBCAAAAA|1556342|3673|6204|2451761|2451731|Dr.|Isela|Austin|Y|29|5|1954|ROMANIA||Isela.Austin@Q0XrXdc7aZRK5V9.edu|2452634| +542|AAAAAAAAOBCAAAAA|1105499|3950|11648|2450014|2449984|Mr.|Everett|Meyers|N|9|2|1942|NIUE||Everett.Meyers@Y8NHbyvR3oDqb.com|2452560| +543|AAAAAAAAPBCAAAAA|1708720||5411|||Mr.||Ritchie||||1962|CYPRUS||Dennis.Ritchie@HHRGELMSDiy6.edu|2452451| +544|AAAAAAAAACCAAAAA|523649|4032|1168|2450012|2449982|Miss|Melissa|Brooks|N|4|4|1978|KAZAKHSTAN||Melissa.Brooks@sRbyt.edu|2452365| +545|AAAAAAAABCCAAAAA|1800220|1871|39169|2451410|2451380|Ms.|Tara|Alexander|N|30|9|1986|PUERTO RICO||Tara.Alexander@41Vp5RH2u.org|2452498| +546|AAAAAAAACCCAAAAA|||9764|2450442||Miss|||N||2|1925||||2452345| +547|AAAAAAAADCCAAAAA|1736192|5073|17554|2449032|2449002|Miss|Viola|Hammond|N|8|12|1968|BOTSWANA||Viola.Hammond@qQEKK4Q09yr36sNgNu.com|2452604| +548|AAAAAAAAECCAAAAA|118021|790|28006|2449368|2449338|Dr.|Martha|Motley|N|18|7|1936|NIGERIA||Martha.Motley@c9bdzMkXGGoK7XUQ.edu|2452424| +549|AAAAAAAAFCCAAAAA|1417889|1467|43092|2450202|2450172|Dr.|Darlene|Braxton|N|1|1|1955|IRAQ||Darlene.Braxton@kAAYZpV1gIF85E.edu|2452437| +550|AAAAAAAAGCCAAAAA|1722627|6847|16667|2452563|2452533|Mr.|Leonard|Simmons|N|6|4|1975|BAHAMAS||Leonard.Simmons@tQSIIoRarBC.org|2452572| +551|AAAAAAAAHCCAAAAA|1019091|6937|37630|2450367|2450337|Mrs.|Joni|Contreras|N|11|8|1942|KUWAIT||Joni.Contreras@klUZEh9nmsZGp.org|2452409| +552|AAAAAAAAICCAAAAA|1129762|515|1315|2451206|2451176|Dr.|Patrick|Mcdonough|N|23|4|1952|BOLIVIA||Patrick.Mcdonough@56UgP5Dqxjs4.com|2452589| +553|AAAAAAAAJCCAAAAA|1483265|2028|10791|2452112|2452082|Mrs.|Jennifer|Greene|N|10|5|1975|SPAIN||Jennifer.Greene@QF9TTt2H.edu|2452367| +554|AAAAAAAAKCCAAAAA|324774|4704|34634|2450968|2450938|Miss|Karen|Hernandez|Y|7|5|1955|TOGO||Karen.Hernandez@vCqKXT2DDVNh.edu|2452447| +555|AAAAAAAALCCAAAAA|1414098|3273|95|2449062|2449032|Miss|Valerie|Bass|Y|25|11|1955|MONGOLIA||Valerie.Bass@lz.com|2452495| +556|AAAAAAAAMCCAAAAA|184320|2125|30330|2451112|2451082|Ms.|Karen|Douglas|Y|8|10|1970|TUNISIA||Karen.Douglas@3PAHg14cV36I1TIem.org|2452425| +557|AAAAAAAANCCAAAAA|832247|5887|22727|2451445|2451415|Dr.|Dionne|Rhea|N|13|8|1987|VANUATU||Dionne.Rhea@1pLul5rJzsg.com|2452407| +558|AAAAAAAAOCCAAAAA|1873874|5219|49411||2452385|||White|Y|||1925|SAINT HELENA||Bill.White@PeqbqmCsN.edu|2452383| +559|AAAAAAAAPCCAAAAA|614470|5895|43673|2450966|2450936|Dr.|Samuel|Thompson|N|17|5|1927|CHILE||Samuel.Thompson@PLQ8KFgTD7vZgZ.edu|2452503| +560|AAAAAAAAADCAAAAA|1168576|6858|3587|2451172|2451142|Dr.|Robert|Clawson|Y|18|11|1970|DOMINICA||Robert.Clawson@3tlLY73s7Fz.com|2452575| +561|AAAAAAAABDCAAAAA|180023|5146|41869|2449311|2449281|Mrs.|Noemi|Tiller|Y|22|10|1981|CHINA||Noemi.Tiller@AXD9sB.edu|2452603| +562|AAAAAAAACDCAAAAA|1138242|753|20464|2452217|2452187|Mr.|Will|Little|N|26|3|1991|BERMUDA||Will.Little@kLBN9MpVYKKro.edu|2452365| +563|AAAAAAAADDCAAAAA|1633187|1407|16427|2449635|2449605|Mr.|Roger|Moye|Y|13|9|1964|SOLOMON ISLANDS||Roger.Moye@VTOK.org|2452286| +564|AAAAAAAAEDCAAAAA|942691|7084|21030|2452146|2452116|Sir|John|Albert|Y|28|11|1981|LIBERIA||John.Albert@4zs.com|2452348| +565|AAAAAAAAFDCAAAAA|1301975|2658|141|2450494|2450464|Mr.|Steven|Young|N|28|5|1939|MARTINIQUE||Steven.Young@IsLXX9P.edu|2452410| +566|AAAAAAAAGDCAAAAA|1695522|6526|42730|2449349|2449319|Mr.|Eric|Mcgee|N|28|2|1960|MONTSERRAT||Eric.Mcgee@pyi9r.edu|2452329| +567|AAAAAAAAHDCAAAAA|329041|1658|23075|2452420|2452390|Ms.|Carrie|Segura|Y|10|6|1959|OMAN||Carrie.Segura@qvJG0xogBfxG.edu|2452325| +568|AAAAAAAAIDCAAAAA|108090|6258|43673|2451174|2451144|Dr.|Michelle|Bruner|Y|2|1|1950|AUSTRIA||Michelle.Bruner@C.com|2452370| +569|AAAAAAAAJDCAAAAA|668061|1593|47125|||Dr.|Robert|||25|||LIBERIA|||2452413| +570|AAAAAAAAKDCAAAAA|1077813|31|2202|2450798|2450768|Sir|James|Jones|Y|27|11|1971|BELGIUM||James.Jones@0DzSxPXCFv1.org|2452504| +571|AAAAAAAALDCAAAAA|136261|4756|38051|2450528|2450498|Dr.|John|Floyd|Y|26|1|1936|PAKISTAN||John.Floyd@FOr1h86s.edu|2452610| +572|AAAAAAAAMDCAAAAA|1122794|536|27478|2452504|2452474|Dr.|Louann|Hamel|Y|26|1|1933|TIMOR-LESTE||Louann.Hamel@cryopHM6lqau9u.com|2452376| +573|AAAAAAAANDCAAAAA|1048005|2258|12958|2449763|2449733|Mrs.|Betty|Combs|N|17|9|1944|BULGARIA||Betty.Combs@pjQc9M96.com|2452585| +574|AAAAAAAAODCAAAAA|913824|3871|6765|2451117|2451087|Mr.|Fred|Rowland|N|24|4|1933|HUNGARY||Fred.Rowland@s5bgJk1L5DMxspOdy.org|2452619| +575|AAAAAAAAPDCAAAAA|1853228|5980|7532|2449242|2449212|Sir|Barry|Hunt|Y|23|2|1945|HONDURAS||Barry.Hunt@u4Z4R1yTIGaahGN5K.edu|2452587| +576|AAAAAAAAAECAAAAA|315315|1404|33940|2450930|2450900|Dr.|Gary|Amundson|N|18|12|1970|PITCAIRN||Gary.Amundson@GXkUYxXfIbPe8MgHQzup.org|2452427| +577|AAAAAAAABECAAAAA|845716|3632|9148|2449477|2449447|Sir|Dewey|Miranda|N|4|1|1971|PARAGUAY||Dewey.Miranda@4Y.org|2452552| +578|AAAAAAAACECAAAAA|576112|783|24701|2451579|2451549|Sir|Michael|Hernandez|N|18|9|1937|ISLE OF MAN||Michael.Hernandez@Y3HoyrQiUdF7T5l1.org|2452468| +579|AAAAAAAADECAAAAA|102652|4282|35512|2450364|2450334|Mrs.|Betty|Day|Y|22|7|1937|TIMOR-LESTE||Betty.Day@SnMdgfNf51z.org|2452352| +580|AAAAAAAAEECAAAAA|475728|2218|3108|2452313|2452283|Ms.|Victoria|Prieto|Y|18|11|1949|BULGARIA||Victoria.Prieto@RQD40KpN8fFx.edu|2452615| +581|AAAAAAAAFECAAAAA|410487|3203|25519|2452004|2451974|Ms.|Katrina|Delgadillo|N|11|9|1969|HONDURAS||Katrina.Delgadillo@vOVbu1fqSRtto1ss2.org|2452388| +582|AAAAAAAAGECAAAAA|118828|6754|5527||2451754|||||||1966|DENMARK||Christopher.Kerr@Uct0dtYMrr7CINj.org|2452512| +583|AAAAAAAAHECAAAAA|1645726|7092|43658|2449933|2449903|Dr.|Grace|Mcdaniel|N|21|1|1930|CHINA||Grace.Mcdaniel@IifT8IYzXIbHH.com|2452421| +584|AAAAAAAAIECAAAAA|1556298|2803|4461|2450982|2450952|Dr.|Shawn|Martin|Y|23|9|1971|UNITED ARAB EMIRATES||Shawn.Martin@S3Cyz71z3m2tTPaE.org|2452438| +585|AAAAAAAAJECAAAAA|1211800|963|3773|2449793|2449763|Dr.|George|Willard|Y|1|8|1970|PALAU||George.Willard@0NTJaJHd2caR.com|2452582| +586|AAAAAAAAKECAAAAA|1520253|1537|8374|2451215|2451185|Sir|Douglas|Lloyd|N|3|1|1949|ROMANIA||Douglas.Lloyd@KrlHARH7P0.edu|2452402| +587|AAAAAAAALECAAAAA|629964|5514|8264|2450937|2450907|Ms.|Ethelyn|Gomez|Y|9|11|1938|C�TE D'IVOIRE||Ethelyn.Gomez@65lqQxNUBv8.com|2452468| +588|AAAAAAAAMECAAAAA|650147|4116|25285|2449080|2449050|Miss|Amy|Quillen|N|15|10|1973|POLAND||Amy.Quillen@44XlKaPl2IU3GbS.edu|2452334| +589|AAAAAAAANECAAAAA|716482|5826|19970|2452518|2452488|Dr.|Julia|Williams|N|12|11|1991|SERBIA||Julia.Williams@6G91v2eKq.org|2452574| +590|AAAAAAAAOECAAAAA|209577|1415|40053|2451515|2451485|Sir|William|Newell|Y|10|11|1979|ERITREA||William.Newell@G2nPbIfk.com|2452491| +591|AAAAAAAAPECAAAAA|448763|3939|22315|2452238|2452208|Dr.|Eldon|Arroyo|Y|17|4|1989|TIMOR-LESTE||Eldon.Arroyo@Mm1D.org|2452435| +592|AAAAAAAAAFCAAAAA|769768|3892|3169|2449285|2449255|Miss|Virginia|Sullivan|N|30|1|1924|SAINT LUCIA||Virginia.Sullivan@CFY.edu|2452382| +593|AAAAAAAABFCAAAAA|854414|654|23694|2449473|2449443|Dr.|Jamar|Kramer|N|29|11|1930|TONGA||Jamar.Kramer@d2YMu5.edu|2452489| +594|AAAAAAAACFCAAAAA|298213|201|46236|2450820|2450790|Mrs.|Carrie|Kramer|N|6|8|1953|CUBA||Carrie.Kramer@RV1Y.edu|2452590| +595|AAAAAAAADFCAAAAA|1845751|497|47243|2450312|2450282|Dr.|Jennifer|Pierson|N|8|1|1966|CAMEROON||Jennifer.Pierson@6vcj19chOM.edu|2452632| +596|AAAAAAAAEFCAAAAA|1318350|1677|5004|2450548|2450518|Mr.|Richard|Allen|N|31|8|1937|LATVIA||Richard.Allen@bZ2noRABhh9.edu|2452575| +597|AAAAAAAAFFCAAAAA|1761998|5554|18757|2449627|2449597|Dr.|Sally|Weis|N|26|2|1969|PALAU||Sally.Weis@C.org|2452292| +598|AAAAAAAAGFCAAAAA|485137|920|6703|2449742|2449712|Mr.|Paul|Bravo|Y|9|3|1983|LESOTHO||Paul.Bravo@KFunFF.edu|2452410| +599|AAAAAAAAHFCAAAAA|348042|5282|12169|2450519|2450489|Dr.|Leroy|Roe|N|16|10|1924|INDONESIA||Leroy.Roe@K4Jyta0.edu|2452569| +600|AAAAAAAAIFCAAAAA|3855|5988|34121|2452213|2452183|Dr.|Maribel|Singleton|N|30|8|1987|GUYANA||Maribel.Singleton@zY0hg.org|2452336| +601|AAAAAAAAJFCAAAAA|1321658|5055|36852|2451206|2451176|Mrs.|Bernice|Mendez|N|25|5|1969|AFGHANISTAN||Bernice.Mendez@irbVjn.org|2452617| +602|AAAAAAAAKFCAAAAA|1521975|316|7504|2449913|2449883|Mr.|Anderson|Katz|Y|17|9|1970|SAINT HELENA||Anderson.Katz@S2K8YK8UZiapN.edu|2452488| +603|AAAAAAAALFCAAAAA|464839|2692|23745|2451452|2451422|Ms.|Sue|Ellis|Y|3|2|1939|RUSSIAN FEDERATION||Sue.Ellis@dD1EI5mu6diM.edu|2452424| +604|AAAAAAAAMFCAAAAA|1791897|5859|41468|2450193|2450163|Mrs.|Julia|Phillips|Y|8|6|1952|BELIZE||Julia.Phillips@1QTz.edu|2452308| +605|AAAAAAAANFCAAAAA|910631|3285|6739|2450271|2450241|Ms.|Arthur|Marroquin|N|2|2|1978|HONDURAS||Arthur.Marroquin@Y4D9iCG91taMQH.org|2452351| +606|AAAAAAAAOFCAAAAA|187899|462|36272|2449104|2449074|Sir|Timothy|Grogan|Y|23|7|1965|SPAIN||Timothy.Grogan@pn3NH2BDng.org|2452531| +607|AAAAAAAAPFCAAAAA|611413|4436|7906|2449789|2449759|Sir|Joseph|Porter|Y|18|2|1958|AZERBAIJAN||Joseph.Porter@Z7lJqLqDuP.edu|2452370| +608|AAAAAAAAAGCAAAAA|969735|5548|49808|2451408|2451378|Ms.|Charlotte|Scott|Y|21|2|1925|MALDIVES||Charlotte.Scott@XM8p5iA38lGi.org|2452525| +609|AAAAAAAABGCAAAAA|660118|5839|12007|2449344|2449314|Ms.|Alexander|Marshall|N|8|1|1959|NEW CALEDONIA||Alexander.Marshall@jUTAHo093oLt7BR.com|2452488| +610|AAAAAAAACGCAAAAA|107602|5159|41516|2449059|2449029|Dr.|Charles|Lane|N|13|11|1957|PANAMA||Charles.Lane@n2TJpVM1UyIh.com|2452471| +611|AAAAAAAADGCAAAAA|1353759|3828|48650|2449926|2449896|Sir|Randall|Smith|N|21|9|1950|ALBANIA||Randall.Smith@QAImMJ3org5mPi.edu|2452628| +612|AAAAAAAAEGCAAAAA|58235|4395|13755|2451724|2451694|Dr.|Dana|Nelson|N|14|1|1975|PUERTO RICO||Dana.Nelson@KiomOfY85E.org|2452337| +613|AAAAAAAAFGCAAAAA|193888|4419|6197|2451048|2451018|Mr.|Charles|Boone|N|13|11|1940|ISLE OF MAN||Charles.Boone@e90x.com|2452525| +614|AAAAAAAAGGCAAAAA|1745441|2245|16296|2452083|2452053|Sir|David|Peterson|N|4|8|1986|LIECHTENSTEIN||David.Peterson@CLGDZq91O.edu|2452324| +615|AAAAAAAAHGCAAAAA|377632|4512|42992|2451577|2451547|Dr.|Pauline|Hudson|N|9|5|1958|DOMINICA||Pauline.Hudson@448tITbUBB.com|2452625| +616|AAAAAAAAIGCAAAAA|1544925|3083|31241|2451204|2451174|Dr.|Yolonda|Muller|Y|15|1|1950|MONTSERRAT||Yolonda.Muller@MKj5fvqQxdSO02.edu|2452391| +617|AAAAAAAAJGCAAAAA|1600339|6458|24577|2450270|2450240|Mr.|Jorge|Carroll|N|11|11|1935|PUERTO RICO||Jorge.Carroll@DuK54a.edu|2452462| +618|AAAAAAAAKGCAAAAA|249369|1300|34831|2450352|2450322|Sir|James|Mueller|Y|16|8|1985|TUVALU||James.Mueller@c1.org|2452614| +619|AAAAAAAALGCAAAAA|378904|812|1920|2451639|2451609|Mrs.|Nancy|Crawford|Y|14|9|1979|GUAM||Nancy.Crawford@Ap.com|2452396| +620|AAAAAAAAMGCAAAAA|179768|3634|48300|2450523|2450493|Miss|Yolanda|Haley|Y|14|8|1970|HONG KONG||Yolanda.Haley@mvOzCx19OEvIf72A7i.org|2452455| +621|AAAAAAAANGCAAAAA|106894|5423|19505|2450428|2450398|Mrs.|Denise|Hutchings|Y|25|4|1968|BAHRAIN||Denise.Hutchings@7.org|2452307| +622|AAAAAAAAOGCAAAAA|1908655|4810|165|||Miss||Johnson|||9|1953|||Laurie.Johnson@0P2bV5aT.org|2452430| +623|AAAAAAAAPGCAAAAA|269245|2431|35970|2452620|2452590|Mrs.|Mildred|Smith|Y|14|8|1957|AZERBAIJAN||Mildred.Smith@J.org|2452430| +624|AAAAAAAAAHCAAAAA|872363|2415|41714|2450336|2450306|Dr.|Andrew|Richards|Y|25|8|1973|NEW ZEALAND||Andrew.Richards@UC.edu|2452506| +625|AAAAAAAABHCAAAAA|625582|1937|35892|2452531|2452501|Ms.|Nellie|Utley|Y|13|10|1970|KYRGYZSTAN||Nellie.Utley@v9JYi4Ae91k.org|2452638| +626|AAAAAAAACHCAAAAA|508259|6599|24539|2451300|2451270|Dr.|Eula|Blair|N|25|11|1962|HAITI||Eula.Blair@7CvxB6x.edu|2452643| +627|AAAAAAAADHCAAAAA|1592569|3843|26764|2449775|2449745|Sir|Lee|Cooper|N|29|1|1986|RUSSIAN FEDERATION||Lee.Cooper@MZUYFZ31RyB6hxB.edu|2452420| +628|AAAAAAAAEHCAAAAA|1042431|6409|4787|2451209|2451179|Mrs.|Laureen|Gulley|Y|27|2|1981|IRELAND||Laureen.Gulley@DT5E02Bnk9gvGzy4.edu|2452419| +629|AAAAAAAAFHCAAAAA|903078|1356|21234|2452319|2452289|Dr.|Melva|Booth|N|20|11|1982|DJIBOUTI||Melva.Booth@HRF7ANICiaeDXLmY.org|2452520| +630|AAAAAAAAGHCAAAAA|147275|6407|33986|2450227|2450197|Miss|Stephanie|Kilgore|N|1|10|1927|CYPRUS||Stephanie.Kilgore@hApgoPV6QSK5s.org|2452589| +631|AAAAAAAAHHCAAAAA|1574590|638|37533|2451077|2451047|Sir|Robert|Patterson|N|1|9|1968|FRANCE||Robert.Patterson@d5P.edu|2452305| +632|AAAAAAAAIHCAAAAA|408957|1688|25666|2449709|2449679|Sir|John|Hall|N|30|8|1987|NAURU||John.Hall@0XCnrmVogJ5EhN.org|2452591| +633|AAAAAAAAJHCAAAAA|1573099|6232|18965|2450596|2450566|Dr.|Bobby|Gill|N|16|4|1930|EQUATORIAL GUINEA||Bobby.Gill@tFfCnzqgpF11E5E.com|2452314| +634|AAAAAAAAKHCAAAAA|1462845|1204|35563|2452404|2452374|Dr.|Manuel|Haynes|Y|10|8|1966|ALGERIA||Manuel.Haynes@S5F2.org|2452304| +635|AAAAAAAALHCAAAAA|582030|5544|27411|2449775|2449745|Ms.|Etta|Marquez|N|21|10|1945|GERMANY||Etta.Marquez@BL85T1.org|2452384| +636|AAAAAAAAMHCAAAAA|1133759|4525|9644|2450439|2450409|Mrs.|Wanda|Munoz|Y|26|7|1924|BAHRAIN||Wanda.Munoz@J.edu|2452330| +637|AAAAAAAANHCAAAAA|670678|3843|32608|2451073|2451043|Sir|John|Smith|N|11|3|1991|AZERBAIJAN||John.Smith@MacFJSiblI.edu|2452456| +638|AAAAAAAAOHCAAAAA|46022|322|21822|2452169|2452139|Mr.|Thomas|Warren|Y|30|7|1928|SAINT HELENA||Thomas.Warren@e3.com|2452460| +639|AAAAAAAAPHCAAAAA|113001|1396|1744|2449242|2449212|Dr.|Ethel|Black|N|31|3|1990|BOUVET ISLAND||Ethel.Black@aKhzXAFbD34VnjX6.edu|2452295| +640|AAAAAAAAAICAAAAA|1328800|2207|36814|2451783|2451753|Dr.|Linda|Leeper|Y|9|2|1988|MARSHALL ISLANDS||Linda.Leeper@6Jh.org|2452367| +641|AAAAAAAABICAAAAA|108988|722|23656|2449180||Miss||Caldwell||4|2||||Nicole.Caldwell@fxUZDOJmYXVZ.edu|2452304| +642|AAAAAAAACICAAAAA|768761|5026|23244|2451558|2451528|Dr.|James|William|N|9|4|1975|GABON||James.William@7MS0.org|2452630| +643|AAAAAAAADICAAAAA|1642680|5249|23018|2450253|2450223|Mr.|John|Miller|N|11|6|1926|KENYA||John.Miller@xPCIGdBa.com|2452515| +644|AAAAAAAAEICAAAAA|||16296|2449515||||Glaze||21||1971|SLOVAKIA||Michael.Glaze@YKZ8zYBxhCQ97uLah.com|2452505| +645|AAAAAAAAFICAAAAA|220625|1014|3803|2452410|2452380|Sir|Charles|Taylor|Y|30|8|1969|RWANDA||Charles.Taylor@0RSuCHpryZ8Q.com|2452432| +646|AAAAAAAAGICAAAAA|||39492|2451295||Miss|Anita|||2|||GABON|||| +647|AAAAAAAAHICAAAAA|1086668|3210|43305|2451498|2451468|Mrs.|Jessica|Martinez|N|7|10|1980|PERU||Jessica.Martinez@Aq6olcFjGu0pTTFb3Dj.edu|2452460| +648|AAAAAAAAIICAAAAA|10370|2836|44455|2452417|2452387|Ms.|Sherri|Tyler|Y|19|3|1958|URUGUAY||Sherri.Tyler@HySjUktGPnSz3.com|2452556| +649|AAAAAAAAJICAAAAA|1384399|239|39726|2449139|2449109|Ms.|Sheila|Campbell|N|21|8|1925|ICELAND||Sheila.Campbell@S9qLSe9fr8VXl.com|2452582| +650|AAAAAAAAKICAAAAA|423705|3433|40314|2451855|2451825|Ms.|Alexis|Marshall|Y|1|5|1949|RWANDA||Alexis.Marshall@3S4poU9lRo.org|2452637| +651|AAAAAAAALICAAAAA|1635344|1780|33416|2450120|2450090|Mrs.|Janis|Prado|N|1|4|1972|AZERBAIJAN||Janis.Prado@1.com|2452337| +652|AAAAAAAAMICAAAAA|1246729|6863|39397|2450168|2450138|Dr.|Colleen|Davis|Y|20|5|1941|NIGERIA||Colleen.Davis@OauAUIYcTCoX.edu|2452392| +653|AAAAAAAANICAAAAA|29605|320|37788|2451449|2451419|Dr.|David|Lanier|N|1|10|1955|HONG KONG||David.Lanier@YLt5q3dC.org|2452402| +654|AAAAAAAAOICAAAAA|1343768|3068|44321|2451868|2451838|Ms.|Ruth|Robbins|N|21|1|1926|BOTSWANA||Ruth.Robbins@0pdQ7.edu|2452497| +655|AAAAAAAAPICAAAAA|393653|2689|16478|2450825|2450795|Mrs.|Patsy|Ross|Y|9|10|1983|MOLDOVA, REPUBLIC OF||Patsy.Ross@SmnL21JPNTrqolGc.edu|2452372| +656|AAAAAAAAAJCAAAAA|1043040|4101|26715|2450196|2450166|Dr.|Sterling|Smith|N|14|4|1978|DENMARK||Sterling.Smith@l7uM9.org|2452309| +657|AAAAAAAABJCAAAAA|1851333|4369|21005|2450687|2450657|Miss|Augusta|Field|Y|21|12|1978|GUADELOUPE||Augusta.Field@tIQfJBsIL.com|2452467| +658|AAAAAAAACJCAAAAA|274406|5869|20579|2451421|2451391|Sir|Gavin|Ayala|N|2|3|1944|BENIN||Gavin.Ayala@oHKcIYHFd3R.com|2452375| +659|AAAAAAAADJCAAAAA|443806|3414|38669|2449133|2449103|Sir|Nathan|Hall|N|25|4|1926|FIJI||Nathan.Hall@A.org|2452539| +660|AAAAAAAAEJCAAAAA|1099226|140|5854|2452233|2452203|Dr.|Robert|Hamilton|N|15|4|1989|CAMEROON||Robert.Hamilton@YzH3jTyOQhp.com|2452489| +661|AAAAAAAAFJCAAAAA|65900|6895|30243|2452525|2452495|Miss|Megan|Hamilton|N|7|3|1988|URUGUAY||Megan.Hamilton@sIyef8cCqpZ5c.edu|2452475| +662|AAAAAAAAGJCAAAAA|||17885||2450792||James|Davis|Y||||GUATEMALA||James.Davis@K2e7b.org|| +663|AAAAAAAAHJCAAAAA|636714|510|18753|2450640|2450610|Dr.|Mark|Smith|N|13|5|1927|BELIZE||Mark.Smith@jicL.edu|2452366| +664|AAAAAAAAIJCAAAAA|748143|1750|25310|2449395|2449365|Dr.|Eleanor|Nelson|Y|17|10|1949|BAHRAIN||Eleanor.Nelson@fGPz5yIH28.com|2452577| +665|AAAAAAAAJJCAAAAA|542073|6563|47806|2450338|2450308|Dr.|Daniel|Null|Y|21|2|1926|BOLIVIA||Daniel.Null@1FC.org|2452413| +666|AAAAAAAAKJCAAAAA|739015|887|45606|2450654|2450624|Miss|Theresa|Brown|Y|6|8|1926|MAURITANIA||Theresa.Brown@KAFsveGBPfGIm1G.com|2452643| +667|AAAAAAAALJCAAAAA|1438699|933|25567|2451880|2451850|Miss|Susan|Valdez|Y|3|10|1987|SWAZILAND||Susan.Valdez@nA1Orr3I9CGXL7Mv.org|2452386| +668|AAAAAAAAMJCAAAAA|592104|5669|27681|2452007|2451977|Miss|Tonia|Dudley|Y|20|6|1972|KYRGYZSTAN||Tonia.Dudley@3k.org|2452365| +669|AAAAAAAANJCAAAAA|401845|1601|4604|2449127|2449097|Mrs.|Ella|Barnette|Y|6|10|1931|KIRIBATI||Ella.Barnette@s4DmJBuqy8vqI.edu|2452401| +670|AAAAAAAAOJCAAAAA|1341362|3089|40769|2451739|2451709|Sir|Russell|Greer|N|10|10|1933|GUYANA||Russell.Greer@atEUGmuvNe.edu|2452493| +671|AAAAAAAAPJCAAAAA|1154312|7123|11981|2451091|2451061|Dr.|Francis|Madison|N|27|8|1924|SRI LANKA||Francis.Madison@bJ4S3S88.org|2452310| +672|AAAAAAAAAKCAAAAA|1806911|1616|16721|2449082|2449052|Sir|Walter|Lee|N|11|2|1941|AZERBAIJAN||Walter.Lee@tyVdrg0mhgr24.com|2452389| +673|AAAAAAAABKCAAAAA|1877365|2538|19664|2450439|2450409|Dr.|Terrence|Albright|Y|14|5|1989|PANAMA||Terrence.Albright@spK.com|2452603| +674|AAAAAAAACKCAAAAA|1208569|6086|37203|2449617|2449587|Mr.|Robert|Paquette|N|14|6|1963|MAURITANIA||Robert.Paquette@gOKTdxvdC4u.com|2452344| +675|AAAAAAAADKCAAAAA|33127|2168|48409|2451423|2451393|Ms.|Tina|Rubio|Y|6|5|1975|MONACO||Tina.Rubio@LY.com|2452493| +676|AAAAAAAAEKCAAAAA|139479|4833|24587|2452214|2452184|Miss|Linda|Middleton|Y|24|7|1938|HAITI||Linda.Middleton@DQKxmeDMCi.edu|2452287| +677|AAAAAAAAFKCAAAAA|941983|4001|3219|2450960|2450930|Miss|Martha|Weldon|N|6|12|1931|MAURITIUS||Martha.Weldon@hNiYla9bf9M9Sa.edu|2452369| +678|AAAAAAAAGKCAAAAA|1904164|5456|25501|2452358|2452328|Mrs.|Eva|Ferrell|Y|6|10|1930|NIGERIA||Eva.Ferrell@Bj3yHSfhIoDDL.edu|2452363| +679|AAAAAAAAHKCAAAAA|217283|2306|12529|2451480|2451450|Mr.|Kermit|Cohn|Y|9|12|1988|OMAN||Kermit.Cohn@3L8kSVj1.edu|2452579| +680|AAAAAAAAIKCAAAAA|227039|376|23266|2449926|2449896|Miss|Joanne|Derr|Y|21|3|1927|GUYANA||Joanne.Derr@p7p9lfKP.com|2452303| +681|AAAAAAAAJKCAAAAA|439779|88|1597|2450145|2450115|Ms.|Rochel|Smith|N|17|8|1951|MALAYSIA||Rochel.Smith@DAIJH.org|2452489| +682|AAAAAAAAKKCAAAAA|762294|856|38265|2451580|2451550|Dr.|Rebecca|Butler|Y|3|1|1945|HAITI||Rebecca.Butler@4JJGxZisoH.org|2452291| +683|AAAAAAAALKCAAAAA|854151|5891|40831|2452483|2452453|Ms.|Jeanne|Langford|N|24|6|1982|SAN MARINO||Jeanne.Langford@ykYpHHfq5P2r.edu|2452317| +684|AAAAAAAAMKCAAAAA|1850943|4419|27542|2450429|2450399|Dr.|Glenn|Thompson|Y|12|3|1979|BAHAMAS||Glenn.Thompson@Fl.org|2452557| +685|AAAAAAAANKCAAAAA|1335142|6872|2249|2451188|2451158|Sir|Robert|Spencer|Y|21|4|1950|INDONESIA||Robert.Spencer@dpmY8A4d2P.edu|2452380| +686|AAAAAAAAOKCAAAAA|1519708|5336|31717|2450135|2450105|Dr.|Charles|Martin|Y|16|10|1984|ETHIOPIA||Charles.Martin@pkM9.edu|2452501| +687|AAAAAAAAPKCAAAAA|1317410|3440|29470|2449063|2449033|Sir|Eduardo|Sandlin|Y|19|2|1980|PITCAIRN||Eduardo.Sandlin@8EhalDQDiy.edu|2452429| +688|AAAAAAAAALCAAAAA|1328729|5863|2597|2452071|2452041|Mr.|Jeff|Thompson|N|18|4|1928|MALTA||Jeff.Thompson@dp0kqg.edu|2452467| +689|AAAAAAAABLCAAAAA|1689186|6001|14416|2450968|2450938|Ms.|Jolene|Hawkins|N|13|12|1960|KUWAIT||Jolene.Hawkins@EsR.com|2452574| +690|AAAAAAAACLCAAAAA|731746|6650|22307|2450786|2450756|Ms.|Jo|Lang|Y|13|10|1938|TUVALU||Jo.Lang@o974bZbMbud83fHL.org|2452493| +691|AAAAAAAADLCAAAAA|932300|6144|33588|2449862|2449832|Mrs.|Natasha|Carl|N|8|9|1951|UNITED KINGDOM||Natasha.Carl@L9Dg1FIjuR9.edu|2452392| +692|AAAAAAAAELCAAAAA|1804333|5900|21020|2450294|2450264|Mrs.|Viola|Burgess|Y|6|9|1970|ANDORRA||Viola.Burgess@14.org|2452420| +693|AAAAAAAAFLCAAAAA|659674|2228|17629|2451179|2451149|Dr.|Loren|Harrell|N|20|9|1928|KIRIBATI||Loren.Harrell@6fPrjlrUpEelq.com|2452423| +694|AAAAAAAAGLCAAAAA|10613|7129|14398|2449366|2449336|Miss|Julie|Johnson|Y|21|4|1942|GERMANY||Julie.Johnson@UMo.org|2452610| +695|AAAAAAAAHLCAAAAA|1419207|869|23684|2450817|2450787|Sir|David|Merritt|N|3|2|1944|WALLIS AND FUTUNA||David.Merritt@6.com|2452569| +696|AAAAAAAAILCAAAAA|147743|746|41796|2450548|2450518|Ms.|Joanne|Eaton|N|1|11|1940|HAITI||Joanne.Eaton@2k0.org|2452518| +697|AAAAAAAAJLCAAAAA|1907878|1098|15871|2451539|2451509|Mrs.|Hazel|Chase|Y|24|5|1968|UKRAINE||Hazel.Chase@sIORj6U1TP7qPS9mOr.org|2452334| +698|AAAAAAAAKLCAAAAA|494882|1712|47789|2451379|2451349|Sir|Herbert|Ortega|N|19|2|1961|SURINAME||Herbert.Ortega@vOxxv.edu|2452576| +699|AAAAAAAALLCAAAAA|1668353|637|28474|2450855|2450825|Miss|Helen|Wise|Y|11|10|1990|SWITZERLAND||Helen.Wise@nZEfl3P.edu|2452431| +700|AAAAAAAAMLCAAAAA|1896891|4101|44911|2449953|2449923|Sir|John|Mcgee|N|18|10|1943|GUINEA-BISSAU||John.Mcgee@KvjqVNSNqe4.org|2452462| +701|AAAAAAAANLCAAAAA|1007805|1997|10189|2450007|2449977|Dr.|Eddie|Moore|Y|7|10|1987|BOTSWANA||Eddie.Moore@dZRmUc9IfkDMk.com|2452379| +702|AAAAAAAAOLCAAAAA|1470257|3941|1225|2449140|2449110|Dr.|Booker|Jones|Y|24|3|1976|EL SALVADOR||Booker.Jones@BZZ4.com|2452408| +703|AAAAAAAAPLCAAAAA|2631|4526|28421|2450570|2450540|Mr.|George|Smith|Y|27|4|1945|FINLAND||George.Smith@1ODgx1etaZDLnJDLZ.com|2452332| +704|AAAAAAAAAMCAAAAA|1864611|1403|25348|2450026|2449996|Dr.|Shawn|Travers|N|28|7|1936|CAMBODIA||Shawn.Travers@2sDEK2rh.edu|2452579| +705|AAAAAAAABMCAAAAA|1170402|587|31619|2450758|2450728|Miss|Heather|Cady|N|21|8|1971|CROATIA||Heather.Cady@s2c.edu|2452536| +706|AAAAAAAACMCAAAAA|1473885|3853|2668|2452194|2452164|Dr.|Joseph|Caldwell|N|1|1|1939|OMAN||Joseph.Caldwell@zLfy.edu|2452334| +707|AAAAAAAADMCAAAAA|708118|3488|19705|2450968|2450938|Ms.|Doris|Wise|Y|5|2|1950|LIECHTENSTEIN||Doris.Wise@TCV9vpm.edu|2452370| +708|AAAAAAAAEMCAAAAA|1449738|825|31112|2450014|2449984|Ms.|Rose|Santana|N|11|9|1928|LITHUANIA||Rose.Santana@Ltt.edu|2452445| +709|AAAAAAAAFMCAAAAA|508295|5373|37604|2452129|2452099|Dr.|Christopher|Smith|N|19|3|1976|BAHAMAS||Christopher.Smith@zL5cupx5oJo.org|2452630| +710|AAAAAAAAGMCAAAAA|996750|4843|14489|2449925|2449895|Dr.|Darrell|Matthews|Y|25|9|1940|MALDIVES||Darrell.Matthews@SbQAmMn6kJd.org|2452600| +711|AAAAAAAAHMCAAAAA|1239998|1343|17888|2450100|2450070|Mrs.|Kimberly|Anderson|N|9|3|1946|UZBEKISTAN||Kimberly.Anderson@G1JFfzfpR7.org|2452355| +712|AAAAAAAAIMCAAAAA|994497|1046|47548|2452457|2452427|Sir|Leonard|Copley|Y|28|6|1929|VENEZUELA||Leonard.Copley@iicJQ737mFu3.edu|2452508| +713|AAAAAAAAJMCAAAAA|378344|5783|44069|2451001|2450971|Dr.|James|Hunt|Y|11|1|1964|MACAO||James.Hunt@rxTPO.edu|2452318| +714|AAAAAAAAKMCAAAAA|415391|6947|20903|2452203|2452173|Ms.|Andrea|Nelson|Y|19|8|1933|TUVALU||Andrea.Nelson@u5QKP1YKdFanFEH.org|2452534| +715|AAAAAAAALMCAAAAA|620729|4845|19314|2450210|2450180|Mr.|Richard|Chastain|Y|18|5|1927|KYRGYZSTAN||Richard.Chastain@JVZK.org|2452640| +716|AAAAAAAAMMCAAAAA|298406|5409|36012|2452208|2452178|Dr.|Ruby|Cole|N|2|6|1953|SUDAN||Ruby.Cole@d.org|2452369| +717|AAAAAAAANMCAAAAA|1593941|4593|27645|2449289|2449259|Sir|Drew|Fuller|Y|8|8|1983|BERMUDA||Drew.Fuller@C7Qs5CRS.edu|2452495| +718|AAAAAAAAOMCAAAAA||6628|21174||2452482||Lisa|Brown|Y||||CZECH REPUBLIC||Lisa.Brown@KhKd4D96KLv3vuO.edu|2452289| +719|AAAAAAAAPMCAAAAA|682804|728|21852|2449743|2449713|Ms.|Anne|Bush|N|3|6|1988|CROATIA||Anne.Bush@EmVECM9s4Sm.com|2452340| +720|AAAAAAAAANCAAAAA|1810255|6680|23552|2451185|2451155|Sir|Lawrence|Rodriguez|N|1|8|1932|BRAZIL||Lawrence.Rodriguez@H0KsKMadbQ.org|2452287| +721|AAAAAAAABNCAAAAA|1417550|68|48939|2449924|2449894|Mrs.|Harriet|Peterson|Y|7|12|1929|GUYANA||Harriet.Peterson@l.org|2452512| +722|AAAAAAAACNCAAAAA|1110544|1219|46741|2449839|2449809|Mr.|Malcolm|Hudson|Y|30|9|1932|KUWAIT||Malcolm.Hudson@vQaeGnQfPR9.org|2452426| +723|AAAAAAAADNCAAAAA|1577731|5007|36683|2450184|2450154|Dr.|John|Hightower|Y|20|12|1943|CAYMAN ISLANDS||John.Hightower@kdVeRDGmmpiy7DcR.com|2452374| +724|AAAAAAAAENCAAAAA|1444365|569|48820|2450878|2450848|Dr.|Victor|Chappell|Y|20|7|1976|CYPRUS||Victor.Chappell@GNS3VoT.edu|2452305| +725|AAAAAAAAFNCAAAAA|1103567|674|19334|2452358|2452328|Mrs.|Jennifer|Fitzpatrick|N|3|5|1925|UNITED ARAB EMIRATES||Jennifer.Fitzpatrick@9AK6ZnfkDN7T.edu|2452364| +726|AAAAAAAAGNCAAAAA|1596982|6201|36573|2451852|2451822|Miss|Nellie|Clay|N|26|12|1932|TURKMENISTAN||Nellie.Clay@m9IsRVO.edu|2452413| +727|AAAAAAAAHNCAAAAA|1809162|7193|44824|2450722|2450692|Mrs.|Sandra|Brown|Y|9|1|1954|EL SALVADOR||Sandra.Brown@A30L8DU5vn5Arj.com|2452405| +728|AAAAAAAAINCAAAAA|751915|831|1300|2450895|2450865|Mrs.|Toni|Orourke|N|28|4|1979|JAPAN||Toni.Orourke@8aBCdmDdpLTEq2.com|2452633| +729|AAAAAAAAJNCAAAAA|892660||31682|||||Lyles||2|12|1985||||| +730|AAAAAAAAKNCAAAAA|1389522|5607|21739|2451741|2451711|Mr.|James|Grider|Y|28|1|1951|SWAZILAND||James.Grider@q4RedOFk8.org|2452343| +731|AAAAAAAALNCAAAAA|1384592|4512|13679|2449585|2449555|Ms.|Kathryn|Ponce|N|14|7|1932|LIBERIA||Kathryn.Ponce@pfIGvNRLYJK5q.com|2452570| +732|AAAAAAAAMNCAAAAA|395497|4903|37109|2449310|2449280|Mrs.|Viola|Brown|Y|29|3|1944|GRENADA||Viola.Brown@UGtFa3fm.edu|2452348| +733|AAAAAAAANNCAAAAA|75980|1414|6495|2452207|2452177|Mr.|Richard|Wells|N|29|2|1968|SOMALIA||Richard.Wells@zOTbD1Hs.com|2452456| +734|AAAAAAAAONCAAAAA|1598366|2758|9503|2450415|2450385|Miss|Helen|Madsen|Y|23|11|1944|ARGENTINA||Helen.Madsen@OKJgbvCv1K.edu|2452318| +735|AAAAAAAAPNCAAAAA|1040827|4135|19843|2450984|2450954|Mr.|Steve|Carroll|Y|15|3|1986|MALAYSIA||Steve.Carroll@TIjD1.org|2452613| +736|AAAAAAAAAOCAAAAA|876474|7148|1775|2449922|2449892|Dr.|Lisa|Andrade|Y|25|3|1946|BHUTAN||Lisa.Andrade@qi.org|2452312| +737|AAAAAAAABOCAAAAA|488546|1584|10175|2449664|2449634|Dr.|Sean|Williams|N|5|11|1982|BERMUDA||Sean.Williams@aYJAX45gBza.com|2452467| +738|AAAAAAAACOCAAAAA|1598922|2403|41084|2450088|2450058|Dr.|Ashlee|Davis|N|20|9|1942|GREENLAND||Ashlee.Davis@3.org|2452630| +739|AAAAAAAADOCAAAAA|212402|3938|20674|2451827|2451797|Dr.|Joy|Baker|Y|5|11|1967|PANAMA||Joy.Baker@St8l0a0YtrZKD.edu|2452615| +740|AAAAAAAAEOCAAAAA|1439756|2117|78|2450450|2450420|Dr.|Pattie|Mcmanus|Y|15|4|1935|INDONESIA||Pattie.Mcmanus@GZt6Dki.org|2452630| +741|AAAAAAAAFOCAAAAA|660662|874|22639|2452057|2452027|Mr.|Otis|Gulley|N|3|9|1948|SINGAPORE||Otis.Gulley@HKYm0Qi29ZiT.com|2452365| +742|AAAAAAAAGOCAAAAA|1001212|4043|14074|2452557|2452527|Dr.|Chris|Adams|N|30|11|1954|UNITED KINGDOM||Chris.Adams@IoKRVYudS67Q1oIfM1.org|2452647| +743|AAAAAAAAHOCAAAAA|1007194|67|540|2449534|2449504|Sir|Eugene|Aldridge|Y|18|7|1948|VANUATU||Eugene.Aldridge@f9om.com|2452340| +744|AAAAAAAAIOCAAAAA|1088541|2085|26429|2450470|2450440|Dr.|Julio|Mcdaniel|Y|18|4|1952|UNITED STATES||Julio.Mcdaniel@2Hc.edu|2452455| +745|AAAAAAAAJOCAAAAA|1078929|1244|45912|2450516|2450486|Mrs.|Angela|Johnson|Y|3|4|1935|RWANDA||Angela.Johnson@jfkr.edu|2452551| +746|AAAAAAAAKOCAAAAA|1490429|6884|4939|2449967|2449937|Ms.|Katie|Ross|N|4|4|1942|MONTENEGRO||Katie.Ross@l3.com|2452623| +747|AAAAAAAALOCAAAAA|913400|1989|6531|2450026|2449996|Mr.|Paul|May|Y|12|12|1952|MARTINIQUE||Paul.May@Jo5PbHtaO5XZMfP.edu|2452528| +748|AAAAAAAAMOCAAAAA|424645|7120|24748|2449698|2449668|Mr.|Blair|Allen|N|19|5|1935|MOLDOVA, REPUBLIC OF||Blair.Allen@4XZqy5ahUq.edu|2452467| +749|AAAAAAAANOCAAAAA|1822433|5712|36991|2449583|2449553|Dr.|Arthur|Waters|Y|18|11|1989|ISLE OF MAN||Arthur.Waters@Bi522QfCVRt.org|2452396| +750|AAAAAAAAOOCAAAAA|264883|6170|27857|2449639|2449609|Mrs.|Edna|Wilson|Y|8|12|1935|SINGAPORE||Edna.Wilson@aS8sffDtdnt.edu|2452335| +751|AAAAAAAAPOCAAAAA|1477686|2439|9992|2452342|2452312|Sir|Rick|Wade|N|4|8|1927|MYANMAR||Rick.Wade@qhlYnHUHvpEogf.edu|2452626| +752|AAAAAAAAAPCAAAAA|1572129|5591|24253|2450262|2450232|Dr.|Ida|Rojas|Y|16|4|1958|SAUDI ARABIA||Ida.Rojas@7cAJ0lVjQbB.com|2452563| +753|AAAAAAAABPCAAAAA|493241|5957|10273|2449571|2449541|Mr.|Charles|Phillips|N|11|2|1961|TUNISIA||Charles.Phillips@C.org|2452571| +754|AAAAAAAACPCAAAAA|765776|4853|8683|2449751|2449721|Mr.|Robert|Sanchez|Y|18|1|1944|JERSEY||Robert.Sanchez@vYcoLJJVIfrqs.edu|2452470| +755|AAAAAAAADPCAAAAA|430954|4012|15423|2450020|2449990|Dr.|Junior|Peeler|N|12|7|1959|LITHUANIA||Junior.Peeler@u9OIEl.org|2452517| +756|AAAAAAAAEPCAAAAA|1726492|6158|15024|2449839|2449809|Miss|Rebecca|Owens|Y|19|8|1964|NIGER||Rebecca.Owens@JTABErekYV681.org|2452298| +757|AAAAAAAAFPCAAAAA|517432|5397|39362|2452231|2452201|Ms.|Dawn|Wells|N|30|5|1957|MOZAMBIQUE||Dawn.Wells@tsdvySqq5k.org|2452620| +758|AAAAAAAAGPCAAAAA|1548150|2646|20742|2449613|2449583|Sir|Miles|Stephens|Y|28|6|1942|GUYANA||Miles.Stephens@9zS0V.com|2452383| +759|AAAAAAAAHPCAAAAA|1809378|3539|13747|2449390|2449360|Dr.|Ralph|Allison|N|25|5|1949|R�UNION||Ralph.Allison@oOlBHrpykcRD.com|2452560| +760|AAAAAAAAIPCAAAAA|617384|4391|10645|2449554|2449524|Mrs.|Glenda|Jones|N|20|10|1990|KUWAIT||Glenda.Jones@pmmg3I89OasUg0afyPl.org|2452351| +761|AAAAAAAAJPCAAAAA|288065|5205|3192|2450436|2450406|Dr.|Alvin|Buchanan|Y|14|5|1934|ARGENTINA||Alvin.Buchanan@8eD4L.com|2452460| +762|AAAAAAAAKPCAAAAA|945978|2789|6000|2449527|2449497|Mr.|Jeffrey|Orr|N|6|1|1933|GUERNSEY||Jeffrey.Orr@zIdBG.org|2452390| +763|AAAAAAAALPCAAAAA|579158|2988|5171|2451694|2451664|Dr.|James|Byrd|N|29|10|1932|SWAZILAND||James.Byrd@Cv.com|2452571| +764|AAAAAAAAMPCAAAAA|1236980|5836|26692|2449442|2449412|Dr.|Joel|Miller|Y|15|12|1973|SENEGAL||Joel.Miller@0vq.org|2452468| +765|AAAAAAAANPCAAAAA|1602506|1346|24139|2451978|2451948|Dr.|Juan|Smith|N|26|3|1978|NEW CALEDONIA||Juan.Smith@EVCCmzbTIRJSF.edu|2452548| +766|AAAAAAAAOPCAAAAA|1261871|4268|29065|2451648|2451618|Mr.|Sean|Saylor|N|18|12|1926|UKRAINE||Sean.Saylor@IzsrYqml.edu|2452429| +767|AAAAAAAAPPCAAAAA|1660484|2424|28764|2452124|2452094|Mr.|Leroy|Levi|N|4|7|1990|NIUE||Leroy.Levi@rJ2aKXlcz6.edu|2452450| +768|AAAAAAAAAADAAAAA|1516025|6122|2219|2449398|2449368|Miss|Evelyn|Wright|N|22|6|1925|WALLIS AND FUTUNA||Evelyn.Wright@QfMrfa28b5Z.com|2452363| +769|AAAAAAAABADAAAAA|1088909|2076|20443|2450728|2450698|Miss|Marilyn|Johnson|Y|26|2|1990|ARMENIA||Marilyn.Johnson@vEo04P4gmrN2r.com|2452626| +770|AAAAAAAACADAAAAA|479080||18431|2449746|2449716|Miss||Porter|N||6|||||2452523| +771|AAAAAAAADADAAAAA|161361|2848|21488|2451957|2451927|Mr.|Charles|Loveless|N|29|9|1950|FIJI||Charles.Loveless@V49sFSYqAJUv.org|2452454| +772|AAAAAAAAEADAAAAA|518262|4792|10969|2451315|2451285|Dr.|Robert|Brown|Y|4|4|1950|SINGAPORE||Robert.Brown@fdjRfB.com|2452500| +773|AAAAAAAAFADAAAAA|1855303|6837|19171|2450239|2450209|Sir|Paul|Martinez|Y|24|2|1949|ANGUILLA||Paul.Martinez@tcNq.org|2452304| +774|AAAAAAAAGADAAAAA|1730007|1676|39698|2450599|2450569|Ms.|Rosie|Stephens|N|18|7|1985|MAURITANIA||Rosie.Stephens@viORkZEQMsmj.org|2452603| +775|AAAAAAAAHADAAAAA|1674210|6771|21779|2452292|2452262|Miss|Althea|Young|N|14|10|1946|NORWAY||Althea.Young@C0gT997poTRVx5P.edu|2452388| +776|AAAAAAAAIADAAAAA|803372|4797|7469|2450298|2450268|Dr.|Mark|Hinton|Y|28|4|1989|EL SALVADOR||Mark.Hinton@p5v3XG3HEn.edu|2452506| +777|AAAAAAAAJADAAAAA|1638128|4973|19018|2449791|2449761|Dr.|Jimmy|Thompson|Y|23|5|1925|PARAGUAY||Jimmy.Thompson@ULcY3.edu|2452298| +778|AAAAAAAAKADAAAAA|1372543|558|23773|2452494|2452464|Ms.|Pamela|Johnson|N|27|4|1924|PAKISTAN||Pamela.Johnson@R0qS7KpREi.org|2452309| +779|AAAAAAAALADAAAAA|1259605|6974|19590|2451555|2451525|Mr.|James|Edwards|Y|1|9|1969|LITHUANIA||James.Edwards@7xV6jpIj6OUQokaqkAS.edu|2452479| +780|AAAAAAAAMADAAAAA|1426148|98|16448|2449775|2449745|Mr.|Kenneth|Martinez|Y|4|11|1979|THAILAND||Kenneth.Martinez@Tgo8.edu|2452317| +781|AAAAAAAANADAAAAA|436983|76|13122|2450415|2450385|Dr.|Jennifer|Helms|N|30|3|1955|BARBADOS||Jennifer.Helms@pZdtNr4O3V.com|2452405| +782|AAAAAAAAOADAAAAA|1006791|2652|18513|2451430|2451400|Ms.|Amanda|Williams|Y|20|4|1971|VIET NAM||Amanda.Williams@ibcQT2pp.edu|2452442| +783|AAAAAAAAPADAAAAA|939962|4104|8999|2449823|2449793|Sir|William|Pierce|N|3|5|1979|NAURU||William.Pierce@czdlVZsBqV1B0lz.org|2452486| +784|AAAAAAAAABDAAAAA|983842|2279|18316|2450871|2450841|Sir|Joseph|Wilson|Y|9|6|1943|GUERNSEY||Joseph.Wilson@M4YRQ7dFgl5lL.com|2452617| +785|AAAAAAAABBDAAAAA|248073|2592|33911|2450906|2450876|Dr.|Danielle|Robinson|Y|21|2|1938|WALLIS AND FUTUNA||Danielle.Robinson@mAVr.com|2452410| +786|AAAAAAAACBDAAAAA|303972|4037|6549|2450771|2450741|Sir|Wilson|Rodriquez|N|25|1|1925|MADAGASCAR||Wilson.Rodriquez@m9gXjLTY.edu|2452528| +787|AAAAAAAADBDAAAAA|940690|2105|17920|2451539|2451509|Mr.|Harold|Martin|Y|21|2|1944|MOROCCO||Harold.Martin@lbBttbGyHqdqA.com|2452404| +788|AAAAAAAAEBDAAAAA|1677016|607|14951|2451279|2451249|Miss|Dorothy|Pearson|N|10|6|1985|UGANDA||Dorothy.Pearson@cKDk7mK.edu|2452286| +789|AAAAAAAAFBDAAAAA|508399|1498|33844|2449293|2449263|Miss|Tara|Rodgers|Y|26|6|1955|KAZAKHSTAN||Tara.Rodgers@0.org|2452322| +790|AAAAAAAAGBDAAAAA|1416621|917|44860|2451535|2451505|Dr.|Irene|Torres|N|8|10|1930|PAKISTAN||Irene.Torres@A6.edu|2452409| +791|AAAAAAAAHBDAAAAA|||34695|2449397|2449367||Ernestina|||||1926||||2452294| +792|AAAAAAAAIBDAAAAA|1441495|3687|3479|2451311|2451281|Miss|Christine|Miller|Y|11|4|1950|NETHERLANDS||Christine.Miller@PQKe4TQvCm.edu|2452434| +793|AAAAAAAAJBDAAAAA|1036459|5011|19516|2449308|2449278|Sir|Travis|Finch|Y|17|3|1930|CHINA||Travis.Finch@Tvas7nqd4TpV.org|2452525| +794|AAAAAAAAKBDAAAAA|1053003|4028|37453|2449740|2449710|Sir|Howard|Albert|Y|31|8|1960|ANDORRA||Howard.Albert@YOQ40tgHblhVfkvGcMa.com|2452622| +795|AAAAAAAALBDAAAAA|1908126|5361|49349|2449908|2449878|Mrs.|Erma|Shelton|N|6|2|1958|TUVALU||Erma.Shelton@Dy.org|2452555| +796|AAAAAAAAMBDAAAAA|830441|2459|29677|2451474|2451444|Mr.|Donald|Adams|Y|27|5|1990|LIECHTENSTEIN||Donald.Adams@l4rV4PDAVcI.com|2452596| +797|AAAAAAAANBDAAAAA|238153|3620|29887|2451337|2451307|Dr.|Benjamin|Crawford|Y|1|10|1969|ECUADOR||Benjamin.Crawford@TSXuTUbUHq7.org|2452561| +798|AAAAAAAAOBDAAAAA|838944|5565|2139|2452496|2452466|Dr.|Theresa|Harris|N|12|3|1960|HUNGARY||Theresa.Harris@NAbKjmFp.edu|2452552| +799|AAAAAAAAPBDAAAAA|161452|598|27744|2450419|2450389|||Chavis|N||||||Edward.Chavis@klh9.com|| +800|AAAAAAAAACDAAAAA|1792496|6452|8761|2452034|2452004|Miss|Aaron|Fuller|Y|16|7|1944|KUWAIT||Aaron.Fuller@Y4M99U.com|2452613| +801|AAAAAAAABCDAAAAA|1796296|1153|40457|2451269|2451239|Mrs.|Rita|Davis|Y|24|5|1952|ECUADOR||Rita.Davis@BLTEy0koi8bD.edu|2452423| +802|AAAAAAAACCDAAAAA|674101|2587|19473|2450655|2450625|Ms.|Martha|Lynch|N|20|5|1956|SAINT HELENA||Martha.Lynch@9hNx1VvZ7MNGN3Ez.com|2452572| +803|AAAAAAAADCDAAAAA|459346|6825|8704|2450857|2450827|Mr.|Scot|Harper|N|18|1|1953|AFGHANISTAN||Scot.Harper@mb4PUXULaUtOlZP5Mn.edu|2452498| +804|AAAAAAAAECDAAAAA|760959|4401|19890|2450904|2450874|Miss|Betty|Gagnon|N|28|6|1941|GEORGIA||Betty.Gagnon@dCm.com|2452636| +805|AAAAAAAAFCDAAAAA|573828|5331|34434|2452098|2452068|Mrs.|Beverly|Lane|N|26|8|1943|QATAR||Beverly.Lane@5vk1FlmNQ8OX.edu|2452402| +806|AAAAAAAAGCDAAAAA|693912|5359|14806|2450221|2450191|Sir|Bryan|Carroll|Y|18|2|1957|JORDAN||Bryan.Carroll@p5QdEH8sr99cFVY.edu|2452561| +807|AAAAAAAAHCDAAAAA|1297289||45070|||Dr.||Randolph|||6|1925|||Martin.Randolph@h8LjMTKC5i.com|2452356| +808|AAAAAAAAICDAAAAA|904146|381|39016|2452046|2452016|Dr.|Kevin|Simpson|N|20|12|1983|MOROCCO||Kevin.Simpson@lAXnyn.org|2452433| +809|AAAAAAAAJCDAAAAA|1731428|6213|48198||2449143|||Waldron|N||||JAPAN|||| +810|AAAAAAAAKCDAAAAA|1573414|934|4854|2449923|2449893|Dr.|Omar|Haggerty|Y|3|5|1962|EQUATORIAL GUINEA||Omar.Haggerty@YY7jGAtmEuy.com|2452345| +811|AAAAAAAALCDAAAAA|1911071|5343|10015|2449332|2449302|Sir|Howard|Kay|Y|7|3|1988|AZERBAIJAN||Howard.Kay@I75vsZardHC.com|2452612| +812|AAAAAAAAMCDAAAAA|1107798|4963|13375|2450929|2450899|Mr.|Steve|Price|N|17|11|1976|GREECE||Steve.Price@EdtJxuafOn.org|2452429| +813|AAAAAAAANCDAAAAA|652555|4409|7910|2449055|2449025|Mr.|David|Foley|Y|2|7|1937|NAURU||David.Foley@X1O1PpmGmYsq4b.com|2452531| +814|AAAAAAAAOCDAAAAA|10185||38581|2451953|2451923|Mr.|Timothy||||9|1948|INDIA|||2452579| +815|AAAAAAAAPCDAAAAA|727275|5046|5617|2450992|2450962|Mrs.|Ella|Martinez|Y|25|5|1953|ALGERIA||Ella.Martinez@aN9.edu|2452514| +816|AAAAAAAAADDAAAAA|863770|1442|39914|2449586|2449556|Mr.|David|Harbin|Y|4|4|1951|ESTONIA||David.Harbin@08jvyn7J6uQ.org|2452512| +817|AAAAAAAABDDAAAAA|1766091|2703|15715|2452126|2452096|Miss|Allen|Lacey|Y|5|1|1965|FAROE ISLANDS||Allen.Lacey@uJ5NigROGGyf3p2Q.edu|2452612| +818|AAAAAAAACDDAAAAA|761537|1046|22508|2449572|2449542|Ms.|Rebecca|Johnson|N|9|10|1956|INDIA||Rebecca.Johnson@JFJJ.com|2452319| +819|AAAAAAAADDDAAAAA|374368|1151|26687|2451304|2451274|Sir|Jose|Sullivan|N|21|5|1938|GUINEA||Jose.Sullivan@Nqo.com|2452605| +820|AAAAAAAAEDDAAAAA|566999|5290|42781|2449892|2449862|Mr.|Ryan|Jackson|Y|28|4|1942|SEYCHELLES||Ryan.Jackson@2gX8X4qk4u.com|2452362| +821|AAAAAAAAFDDAAAAA|1273654|5124|33794|2450778|2450748|Dr.|Amy|Joyce|N|13|10|1944|R�UNION||Amy.Joyce@k.org|2452493| +822|AAAAAAAAGDDAAAAA|1648361|2243|16572|2452093|2452063|Miss|Nadia|Shay|Y|17|1|1970|SPAIN||Nadia.Shay@Jq.com|2452393| +823|AAAAAAAAHDDAAAAA|53449|5206|20963|2450900|2450870|Dr.|Danny|Lopez|N|15|3|1934|MONGOLIA||Danny.Lopez@pk8IPEPjRHjN.edu|2452355| +824|AAAAAAAAIDDAAAAA|100669|1456|8196|2451297|2451267|Sir|Ernesto|Higgins|Y|16|7|1963|MALAYSIA||Ernesto.Higgins@goUsmxIvr98ZC9.edu|2452418| +825|AAAAAAAAJDDAAAAA|737168|5221|35321|2450350|2450320|Mr.|Darnell|Smith|N|17|10|1986|EQUATORIAL GUINEA||Darnell.Smith@k3iUm0.edu|2452623| +826|AAAAAAAAKDDAAAAA|1312472|5614|15603|2451994|2451964|Mrs.|Jennifer|Alger|Y|18|12|1934|GRENADA||Jennifer.Alger@ZhyL8Z.com|2452439| +827|AAAAAAAALDDAAAAA|1020999|5142|31732|2449980|2449950|Ms.|Claudia|Foster|N|30|6|1946|GHANA||Claudia.Foster@P.com|2452357| +828|AAAAAAAAMDDAAAAA|1671475|3526|27530|2452315|2452285|Sir|Garland|Robinson|N|15|6|1967|UKRAINE||Garland.Robinson@C.org|2452620| +829|AAAAAAAANDDAAAAA|||7724|||Miss|||Y|||1942|SAMOA||Virginia.Whitfield@FbQcPAEF37IgstDC.edu|| +830|AAAAAAAAODDAAAAA|308996|2175|954|2452302|2452272|Mr.|Alfred|Sheppard|Y|18|12|1926|MEXICO||Alfred.Sheppard@FZFJRY7VTo.com|2452412| +831|AAAAAAAAPDDAAAAA|1763322|2220|3118|2450079|2450049|Mr.|Reginald|Garcia|N|25|8|1957|HONDURAS||Reginald.Garcia@8NKIJxQZ.org|2452463| +832|AAAAAAAAAEDAAAAA|1370600|936|14872|2450206|2450176|Sir|Craig|Eaves|Y|13|11|1971|JAPAN||Craig.Eaves@El4pR.org|2452387| +833|AAAAAAAABEDAAAAA|163602|244|20439|2449126|2449096|Sir|Paul|Villanueva|Y|25|6|1963|NEW ZEALAND||Paul.Villanueva@JiPPT4FRUlMmmVU.edu|2452643| +834|AAAAAAAACEDAAAAA|909474|6168|32062|2449581|2449551|Ms.|Catherine|Adams|N|30|8|1929|MOROCCO||Catherine.Adams@7sROi8QPSRM3.org|2452311| +835|AAAAAAAADEDAAAAA|1073225|2963|45854|2451618|2451588|Ms.|Annie|Burke|Y|16|4|1934|C�TE D'IVOIRE||Annie.Burke@XP7.com|2452350| +836|AAAAAAAAEEDAAAAA|490064|6396|43528|2452122|2452092|Sir|John|Young|N|19|2|1932|URUGUAY||John.Young@rxtHIXvzkTcFaruTZTQ.edu|2452514| +837|AAAAAAAAFEDAAAAA|1374850|4745|32640|2450501|2450471|Mr.|Guy|Bray|Y|8|2|1991|NORFOLK ISLAND||Guy.Bray@J3UmcAmPs94kQFOrI.edu|2452565| +838|AAAAAAAAGEDAAAAA|175359|7016|31920|2450960|2450930|Sir|Michael|Numbers|N|12|9|1924|AZERBAIJAN||Michael.Numbers@9DnUVVV.org|2452524| +839|AAAAAAAAHEDAAAAA|1218003|1225|23200|2452099|2452069|Miss|Cherise|Welsh|N|29|9|1935|JERSEY||Cherise.Welsh@ug9mx9kfueuIbc.com|2452612| +840|AAAAAAAAIEDAAAAA|390857|448|49286|2451933|2451903|Sir|Matt|Salazar|Y|28|6|1946|AZERBAIJAN||Matt.Salazar@rDN6k49nQGt.edu|2452468| +841|AAAAAAAAJEDAAAAA|927943|662|4981|2450811|2450781|Mr.|Jose|Jones|N|6|8|1926|MACAO||Jose.Jones@11sYUUc58Xa.com|2452456| +842|AAAAAAAAKEDAAAAA|938847|1416|27185|2450116|2450086|Ms.|Brandi|Miller|Y|24|1|1955|KUWAIT||Brandi.Miller@JH5mldviNI7xQKVQj.com|2452289| +843|AAAAAAAALEDAAAAA|454327|1922|27517|2450407|2450377|Miss|Peggy|Moore|Y|2|1|1976|TAJIKISTAN||Peggy.Moore@uNI7.org|2452605| +844|AAAAAAAAMEDAAAAA|1072114|5643|47525|2450806|2450776|Ms.|Michele|Michael|Y|9|9|1937|GUYANA||Michele.Michael@hyhonYcMd3xo08.org|2452445| +845|AAAAAAAANEDAAAAA|309001|4153|505|2451063|2451033|Mrs.|Annie|Kuhn|N|30|5|1938|GABON||Annie.Kuhn@BnGqUvHOg8MY5Ar4.edu|2452576| +846|AAAAAAAAOEDAAAAA|527799|4403|7807|2449908|2449878|Ms.|Theresa|Buchanan|N|5|3|1981|SWITZERLAND||Theresa.Buchanan@FeV9K7cC.com|2452532| +847|AAAAAAAAPEDAAAAA|815597|6808|28066|2450125|2450095|Miss|Aaron|Labbe|N|19|12|1958|JAPAN||Aaron.Labbe@0znR.com|2452292| +848|AAAAAAAAAFDAAAAA|1391158|2495|12679|2449343|2449313|Ms.|Teresa|Riggs|Y|31|12|1985|BANGLADESH||Teresa.Riggs@aBnvzgT9KnQcx.org|2452566| +849|AAAAAAAABFDAAAAA|60345|2612|11279|2450064|2450034|Dr.|Marcela|Arias|N|3|2|1966|EL SALVADOR||Marcela.Arias@u7.com|2452516| +850|AAAAAAAACFDAAAAA|1309827|4019|25858|2449426|2449396|Dr.|Joseph|Conyers|N|20|12|1957|SAN MARINO||Joseph.Conyers@4oUqp.com|2452291| +851|AAAAAAAADFDAAAAA|257056|241|41073|2450863|2450833|Sir|Jeffrey|Baker|N|22|5|1953|OMAN||Jeffrey.Baker@yHPmPH.edu|2452614| +852|AAAAAAAAEFDAAAAA|784821|7100|6532|2449404|2449374|Miss|Heather|Dupont|N|14|10|1972|ARMENIA||Heather.Dupont@RgC3X.org|2452342| +853|AAAAAAAAFFDAAAAA|1642754|5377|42800|2450099|2450069|Miss|Janet|Boston|Y|31|12|1953|FIJI||Janet.Boston@i8ZGHb0HA1D4r9SyD8.edu|2452485| +854|AAAAAAAAGFDAAAAA|503304|2472|40440|2452553|2452523|Dr.|Terry|Ortiz|N|27|8|1961|KUWAIT||Terry.Ortiz@gyF.edu|2452498| +855|AAAAAAAAHFDAAAAA||4020|40039||||Stephen|Livingston|||7|||||2452549| +856|AAAAAAAAIFDAAAAA||5188|41066|2449711||Sir|||N||||SAINT HELENA|||2452608| +857|AAAAAAAAJFDAAAAA|1264546|5147|45795|2450207|2450177|Sir|John|Brewer|Y|30|11|1977|JAMAICA||John.Brewer@aM.edu|2452539| +858|AAAAAAAAKFDAAAAA|876901|2037|24020|2449403|2449373|Mr.|Darrick|Hankins|N|13|8|1952|MOROCCO||Darrick.Hankins@28.org|2452446| +859|AAAAAAAALFDAAAAA|708533|2468|6850|2451246|2451216|Dr.|Kizzie|Larson|N|26|3|1960|KAZAKHSTAN||Kizzie.Larson@5Ip0HUOPfDO2.edu|2452537| +860|AAAAAAAAMFDAAAAA|1897158|1524|25890|2452555|2452525|Mr.|Stephen|Grimes|Y|20|12|1931|MALTA||Stephen.Grimes@8U.com|2452555| +861|AAAAAAAANFDAAAAA|1302362|1586|13610|2451619|2451589|Mrs.|Angel|Casanova|Y|10|10|1971|HAITI||Angel.Casanova@3e3h3V1HoC9q4.com|2452609| +862|AAAAAAAAOFDAAAAA|1373600|5919|35485|2450973|2450943|Sir|Gregory|Butler|Y|5|11|1985|GIBRALTAR||Gregory.Butler@99bsABVlu6BqN1HHh.edu|2452519| +863|AAAAAAAAPFDAAAAA|72331|3259|31434|2450872|2450842|Dr.|Amanda|Simpson|Y|23|2|1985|LIBERIA||Amanda.Simpson@2MX45uHJ5.edu|2452537| +864|AAAAAAAAAGDAAAAA|1731780|5538|40096|2450074|2450044|Sir|Kermit|Wiggins|N|19|7|1986|SOMALIA||Kermit.Wiggins@Vo2DZs5.edu|2452441| +865|AAAAAAAABGDAAAAA|11269|1791|1913|2449071|2449041|Dr.|Betty|Kelly|Y|20|8|1956|NICARAGUA||Betty.Kelly@QYzdtVRqlL.org|2452403| +866|AAAAAAAACGDAAAAA|1834955|4608|49061|2449430|2449400|Miss|Jerrie|Carter|N|17|12|1950|UNITED KINGDOM||Jerrie.Carter@sQycAzxsvhXMKIgIum.org|2452439| +867|AAAAAAAADGDAAAAA|1086179|4029|28679|2449870|2449840|Mr.|Alberto|Mccreary|Y|28|5|1990|MAURITIUS||Alberto.Mccreary@MIm7Vf.org|2452333| +868|AAAAAAAAEGDAAAAA|1296412|4004|5196|2451086|2451056|Dr.|Andy|Orourke|N|21|6|1974|JERSEY||Andy.Orourke@Ylr51dj23quK.edu|2452454| +869|AAAAAAAAFGDAAAAA|1426416|6323|41877|2451478|2451448|Mr.|Marcelino|Thompson|Y|23|1|1938|BOLIVIA||Marcelino.Thompson@xs.edu|2452317| +870|AAAAAAAAGGDAAAAA|1610065|6304|49743|2451239|2451209|Ms.|Ella|Dominguez|N|8|8|1936|NEW CALEDONIA||Ella.Dominguez@ojiMJmjm.edu|2452622| +871|AAAAAAAAHGDAAAAA|479682|1647|39165|2451589|2451559|Dr.|Carla|Beckwith|N|7|12|1955|YEMEN||Carla.Beckwith@B.edu|2452451| +872|AAAAAAAAIGDAAAAA|1262338|3006|5227|2451387|2451357|Dr.|Matilde|Adams|Y|6|1|1985|ANGOLA||Matilde.Adams@csq4.edu|2452329| +873|AAAAAAAAJGDAAAAA|1305635|5984|8686|2449607|2449577|Mrs.|Stephanie|Horn|Y|14|3|1962|CAMBODIA||Stephanie.Horn@dcSY47tBN.edu|2452577| +874|AAAAAAAAKGDAAAAA|733366|2469|27619|2451133|2451103|Sir|Steven|Dawkins|N|29|4|1991|LIECHTENSTEIN||Steven.Dawkins@x7.com|2452296| +875|AAAAAAAALGDAAAAA|1202319|6217|38562|2450203|2450173|Mr.|Marcus|White|Y|29|5|1954|UNITED ARAB EMIRATES||Marcus.White@9Q.edu|2452293| +876|AAAAAAAAMGDAAAAA|692612|4413|38614|2451160|2451130|Ms.|Gladys|Howell|Y|13|9|1937|CANADA||Gladys.Howell@PSogHbX3hO.com|2452414| +877|AAAAAAAANGDAAAAA|1248319|5387|14408|2451331|2451301|Dr.|Ruth|Taylor|N|21|6|1937|HONDURAS||Ruth.Taylor@GcRqISd9.org|2452423| +878|AAAAAAAAOGDAAAAA|1614218|1883|46301|2450526|2450496|Mr.|Robert|Lyons|Y|25|7|1959|ARUBA||Robert.Lyons@Nvilmm2d4DbXUS.com|2452451| +879|AAAAAAAAPGDAAAAA|700147|1583|24619|2449112|2449082|Dr.|Angela|Long|Y|23|8|1926|SLOVENIA||Angela.Long@5K.edu|2452643| +880|AAAAAAAAAHDAAAAA|126529|4700|27224|2451670|2451640|Mr.|Donald|Stout|N|25|5|1942|CAMBODIA||Donald.Stout@Glh2ieMS.org|2452423| +881|AAAAAAAABHDAAAAA|215329|3163|30550|2449779|2449749|Mrs.|Rose|Estes|Y|3|11|1926|CUBA||Rose.Estes@Ofq4LS.com|2452462| +882|AAAAAAAACHDAAAAA|173616|1191|44366|2450185|2450155|Mr.|Patrick|Hughes|Y|5|10|1968|NEW CALEDONIA||Patrick.Hughes@a6f67C8ivzl5pjqU.edu|2452546| +883|AAAAAAAADHDAAAAA|670587|3189|28591|2452525|2452495|Dr.|Minnie|Roberson|Y|11|5|1980|NETHERLANDS ANTILLES||Minnie.Roberson@k.edu|2452384| +884|AAAAAAAAEHDAAAAA|1601075|2339|13603|2451873|2451843|Dr.|Richard|Todd|N|24|8|1956|SAINT LUCIA||Richard.Todd@P6.edu|2452627| +885|AAAAAAAAFHDAAAAA|1589235|6566|37497|2451595|2451565|Ms.|Jane|Morrison|N|23|12|1973|MAYOTTE||Jane.Morrison@K5P9tdHM2rhvhdxrnBUr.com|2452537| +886|AAAAAAAAGHDAAAAA|434401|6886|8218|2450816|2450786|Dr.|Joshua|Pierce|N|19|10|1949|ESTONIA||Joshua.Pierce@xhAndKm4c2pR.org|2452324| +887|AAAAAAAAHHDAAAAA|147022|4031|39840||2451282||Eddie|Peck|||4|1930|||Eddie.Peck@Pe19EkDTfYIkEcxbHi.org|| +888|AAAAAAAAIHDAAAAA|303042|933|32377|2451248|2451218|Sir|Mark|Bartley|Y|17|10|1967|AUSTRALIA||Mark.Bartley@PeUD8Gdp8QIQ.edu|2452326| +889|AAAAAAAAJHDAAAAA|726268|5683|7898|2451797|2451767|Dr.|Timothy|Farley|Y|18|1|1956|NEPAL||Timothy.Farley@EJgb99k.com|2452350| +890|AAAAAAAAKHDAAAAA|13508|6487|49513|2450199|2450169|Ms.|Dorothy|Fabian|N|11|12|1983|SOUTH AFRICA||Dorothy.Fabian@bJqnYqo.org|2452606| +891|AAAAAAAALHDAAAAA|847191|957|17476|2450900|2450870|Mrs.|Denise|Angel|Y|20|7|1933|SWAZILAND||Denise.Angel@kYgcDj.edu|2452493| +892|AAAAAAAAMHDAAAAA|1506072|2273|42427|2450597|2450567|Sir|Terrance|Saunders|N|2|10|1950|CUBA||Terrance.Saunders@fZGvCAVnyaon.org|2452572| +893|AAAAAAAANHDAAAAA|1773270|4851|8432|2451360|2451330|Mr.|Tim|Pope|N|26|6|1947|MEXICO||Tim.Pope@yLRSZ8aCzi.com|2452611| +894|AAAAAAAAOHDAAAAA|1215912|4832|8552|2449825|2449795|Sir|Jay|Butler|Y|1|8|1939|CAYMAN ISLANDS||Jay.Butler@PaL.com|2452489| +895|AAAAAAAAPHDAAAAA|1366593|176|32959|2449338|2449308|Mr.|Riley|Sanchez|Y|8|10|1939|GERMANY||Riley.Sanchez@fKR6lOipNF0PQt.com|2452477| +896|AAAAAAAAAIDAAAAA|1421303|4191|43032|2451390|2451360|Ms.|Penelope|Malloy|N|11|10|1982|LESOTHO||Penelope.Malloy@P2XbGORa4fc.com|2452516| +897|AAAAAAAABIDAAAAA|402734|2759|43584|2450279|2450249|Ms.|Naida|Newton|Y|10|9|1992|R�UNION||Naida.Newton@pJ6A6.com|2452427| +898|AAAAAAAACIDAAAAA|589481|1962|26043|2450404|2450374|Sir|Timothy|Williams|Y|10|8|1949|CHINA||Timothy.Williams@RHH.com|2452309| +899|AAAAAAAADIDAAAAA|122900|2109|6965|2451182|2451152|Mr.|Scott|Mcdonnell|N|19|2|1929|KENYA||Scott.Mcdonnell@Rlpi.edu|2452634| +900|AAAAAAAAEIDAAAAA|1468014|4911|1361|2449679|2449649|Dr.|Ronald|Marsh|N|2|11|1946|NEW CALEDONIA||Ronald.Marsh@RfoILr.com|2452291| +901|AAAAAAAAFIDAAAAA|71778|1568|20158|2452243|2452213|Sir|David|Brill|Y|9|4|1961|BAHRAIN||David.Brill@N5djBEgHj8.com|2452604| +902|AAAAAAAAGIDAAAAA|1522080|6769|47058|2451863|2451833|Mr.|Raymond|Fenton|N|5|12|1967|MOROCCO||Raymond.Fenton@4opCBpsgdqd2N.edu|2452466| +903|AAAAAAAAHIDAAAAA|1603400|92|45184|2452353|2452323|Ms.|Wanda|Robinson|Y|5|9|1988|ROMANIA||Wanda.Robinson@HLs15VC8is.edu|2452354| +904|AAAAAAAAIIDAAAAA|1861541|3630|44919|2449372|2449342|Miss|Elois|Hunter|Y|14|4|1936|MONACO||Elois.Hunter@irJ.com|2452591| +905|AAAAAAAAJIDAAAAA|610192|2993|41041|2450644|2450614|Mr.|Issac|Jones|Y|17|2|1929|CUBA||Issac.Jones@Fn7Ugmjfbh3dOur1.com|2452358| +906|AAAAAAAAKIDAAAAA|1554610|6055|45100|2450114|2450084|Sir|William|Ahrens|Y|13|5|1981|INDIA||William.Ahrens@gtox6y5NK3aYr.com|2452586| +907|AAAAAAAALIDAAAAA|1707763|3740|44117|2452596|2452566|Mrs.|Hellen|Child|N|20|5|1991|NEW ZEALAND||Hellen.Child@7V4q8x9ty.com|2452506| +908|AAAAAAAAMIDAAAAA|1073266|6324|44935|2451131|2451101|Ms.|Daisy|Smith|Y|4|9|1970|NIGER||Daisy.Smith@xfiluZYGel.com|2452293| +909|AAAAAAAANIDAAAAA|1773724|2637|9903|2449199|2449169|Miss|Andra|Hailey|Y|4|10|1929|BOLIVIA||Andra.Hailey@LHrnR8hoU8.org|2452315| +910|AAAAAAAAOIDAAAAA|1297469|383|24050|2451155|2451125|Dr.|Marian|Schuler|N|16|3|1966|TUVALU||Marian.Schuler@6d8rLRfLgPbm1zl.org|2452367| +911|AAAAAAAAPIDAAAAA|1166764|4092|32667||2452530|Mr.|Gino|Ortiz|||3||||Gino.Ortiz@3H55U.org|2452512| +912|AAAAAAAAAJDAAAAA|953253|713|973|2450111|2450081|Sir|Zachary|Parsons|Y|2|6|1960|INDONESIA||Zachary.Parsons@hHmnLrbKsfY.com|2452595| +913|AAAAAAAABJDAAAAA|1262632|5956|12127|2450422|2450392|Mr.|Rob|Blanco|N|9|2|1977|UKRAINE||Rob.Blanco@6GhI1MtoAQPsIc.edu|2452526| +914|AAAAAAAACJDAAAAA|286912|5589|7113|2450204|2450174|Miss|Chrystal|Nielson|N|28|3|1932|COSTA RICA||Chrystal.Nielson@fu2hEcMZTizK.edu|2452335| +915|AAAAAAAADJDAAAAA|919998|6114|48230|2450839|2450809|Ms.|Emelina|Fultz|Y|31|8|1950|JERSEY||Emelina.Fultz@yA1h0a7SrJ.com|2452398| +916|AAAAAAAAEJDAAAAA|423374|1814|25101|2450793|2450763|Mrs.|Victoria|Soto|N|8|8|1977|ARUBA||Victoria.Soto@uSkdoJe9CPnn2x.org|2452384| +917|AAAAAAAAFJDAAAAA|1151355|4809|14216|2449905|2449875|Ms.|Alecia|Bearden|N|11|9|1990|ANTIGUA AND BARBUDA||Alecia.Bearden@MKPXVrr.org|2452305| +918|AAAAAAAAGJDAAAAA|1614150|775|30553|2451740|2451710|Mrs.|Kerrie|Rodgers|Y|12|12|1934|CZECH REPUBLIC||Kerrie.Rodgers@t.edu|2452379| +919|AAAAAAAAHJDAAAAA|74346|1317|49888|2451443|2451413|Mrs.|Deborah|Sanders|N|25|7|1964|MOROCCO||Deborah.Sanders@N7JkcVyjlxDGPk.org|2452422| +920|AAAAAAAAIJDAAAAA|1521425|4813|32648|2451961|2451931|Sir|Wesley|Harris|N|13|12|1990|ARGENTINA||Wesley.Harris@ty4FeR.org|2452412| +921|AAAAAAAAJJDAAAAA|705778|475|30772|2452406|2452376|Sir|James|Hudson|N|21|8|1985|BAHRAIN||James.Hudson@CfyK.org|2452633| +922|AAAAAAAAKJDAAAAA|1326069|1710|15798|2452191|2452161|Ms.|Cristy|Tillery|Y|10|1|1943|VENEZUELA||Cristy.Tillery@vhS46uiYkm.edu|2452486| +923|AAAAAAAALJDAAAAA|243980|3671|43762|2450070|2450040|Dr.|Elizabeth|Harness|N|14|9|1987|CHINA||Elizabeth.Harness@cHQhj04pauPl6crMhB.edu|2452383| +924|AAAAAAAAMJDAAAAA|1166547|1671|43995|2452259|2452229|Ms.|Brenda|Delgado|N|10|9|1934|CHRISTMAS ISLAND||Brenda.Delgado@Y.com|2452628| +925|AAAAAAAANJDAAAAA|1219435|638|37394|2449593|2449563|Miss|Felicia|Chew|Y|28|2|1937|KUWAIT||Felicia.Chew@XBU5JpAEEbpdG.org|2452402| +926|AAAAAAAAOJDAAAAA|1109087|2439|43438|2452239|2452209|Miss|Jennifer|Rasmussen|Y|29|3|1976|UNITED KINGDOM||Jennifer.Rasmussen@ucyl0JCVQxgb.org|2452308| +927|AAAAAAAAPJDAAAAA|466436|3463|30490|2449310|2449280|Ms.|Shea|Hardy|N|7|4|1943|GAMBIA||Shea.Hardy@A8BTPubPlj.com|2452416| +928|AAAAAAAAAKDAAAAA|1231070|3593|17763|2449905|2449875|Dr.|Debra|Casillas|Y|1|2|1971|ANTARCTICA||Debra.Casillas@cIQrlu.org|2452533| +929|AAAAAAAABKDAAAAA|449563|7060|17310|2450235|2450205|Dr.|Donna|Hunter|N|15|9|1986|AFGHANISTAN||Donna.Hunter@qVxlANlv5r.com|2452338| +930|AAAAAAAACKDAAAAA|1771445|3751|21210|2449815|2449785|Dr.|Roberta|Mcdonald|Y|26|2|1948|SYRIAN ARAB REPUBLIC||Roberta.Mcdonald@jo.com|2452378| +931|AAAAAAAADKDAAAAA|1512005|640|32875|2450222|2450192|Dr.|Richard|Akers|Y|3|1|1946|ANTARCTICA||Richard.Akers@TnhkCcZyZcGn.org|2452608| +932|AAAAAAAAEKDAAAAA|471021|1533|1692|2449697|2449667|Dr.|Dana|Dye|N|24|7|1959|MACAO||Dana.Dye@U8QmUrY.com|2452373| +933|AAAAAAAAFKDAAAAA|186028|3619|21563|2449593|2449563|Mr.|Steven|Ball|N|20|6|1939|SURINAME||Steven.Ball@irz3tIyZ2jbT3.org|2452457| +934|AAAAAAAAGKDAAAAA|958704|4315|24679|2449605|2449575|Mr.|Larry|Bond|N|14|6|1943|ITALY||Larry.Bond@mI66aDk1Ua8QL.edu|2452469| +935|AAAAAAAAHKDAAAAA|738424|1842|8107|2450299|2450269|Dr.|Steven|Williams|Y|14|3|1949|QATAR||Steven.Williams@uxeKdX2slbhOxguR.edu|2452588| +936|AAAAAAAAIKDAAAAA|1910415|5775|47888|2451738|2451708|Dr.|Phillip|Farmer|Y|23|3|1981|LESOTHO||Phillip.Farmer@s.com|2452332| +937|AAAAAAAAJKDAAAAA|1110879|5763|8247|2452210|2452180|Dr.|Jason|Haley|N|30|10|1992|GAMBIA||Jason.Haley@Aldc2ni2IbRYDbxOqcpq.org|2452296| +938|AAAAAAAAKKDAAAAA|528166|6057|23548|2451115|2451085|Mr.|Emerson|Martin|N|30|11|1991|KYRGYZSTAN||Emerson.Martin@rh9uuoru.com|2452628| +939|AAAAAAAALKDAAAAA|625180|3680|227|2450108|2450078|Sir|Bernard|Martinez|N|9|6|1930|GABON||Bernard.Martinez@msKqlgVeMqH2l9.com|2452642| +940|AAAAAAAAMKDAAAAA|314898|2542|22939|2449311|2449281|Mr.|Cornelius|Moore|Y|24|9|1970|GABON||Cornelius.Moore@DQfD2Yv.edu|2452626| +941|AAAAAAAANKDAAAAA|566230|6690|40350|2451358|2451328|Ms.|Karina|Irwin|N|19|4|1973|PORTUGAL||Karina.Irwin@OdndnrJqGit.com|2452619| +942|AAAAAAAAOKDAAAAA|1030675|1394|10113|2450514|2450484|Miss|Meghan|Murray|Y|7|6|1959|SYRIAN ARAB REPUBLIC||Meghan.Murray@jaLTHI3EdbJX.com|2452624| +943|AAAAAAAAPKDAAAAA|1503369|1452|45582|2451612|2451582|Dr.|Douglas|Johnson|N|22|4|1932|MALTA||Douglas.Johnson@Gd43TbSM.org|2452467| +944|AAAAAAAAALDAAAAA|831731|133|47735|2449330|2449300|Sir|Max|Stephens|Y|21|1|1928|ARMENIA||Max.Stephens@0X.com|2452380| +945|AAAAAAAABLDAAAAA|1332390||31651|2449886|2449856|Dr.|Ellamae|Carter|||8||CHRISTMAS ISLAND|||2452646| +946|AAAAAAAACLDAAAAA|752860|2573|37086|2450013|2449983|Miss|Barbara|Fountain|N|9|1|1952|ICELAND||Barbara.Fountain@YPC5HibGjoD1JxAPM.com|2452411| +947|AAAAAAAADLDAAAAA|863425|2118|47658|2450295|2450265|Dr.|Eduardo|Boles|Y|31|1|1951|KIRIBATI||Eduardo.Boles@EBFS.org|2452536| +948|AAAAAAAAELDAAAAA|1646433|5392|2751|2452077|2452047|Dr.|Joanne|Wynne|Y|3|6|1947|BHUTAN||Joanne.Wynne@YDBGegN6D.com|2452321| +949|AAAAAAAAFLDAAAAA|792070|5315|44960|2449068|2449038|Ms.|Margaret|Roberts|Y|17|1|1954|LUXEMBOURG||Margaret.Roberts@JXBlZh.edu|2452553| +950|AAAAAAAAGLDAAAAA|460765|2817|2069|2451678|2451648|Ms.|Etta|Phillips|Y|13|12|1973|VIET NAM||Etta.Phillips@VYG6v.com|2452327| +951|AAAAAAAAHLDAAAAA|985271|72|48473|2450070|2450040|Miss|Margie|Bailey|Y|4|6|1966|SYRIAN ARAB REPUBLIC||Margie.Bailey@3TMyCZpne.org|2452369| +952|AAAAAAAAILDAAAAA|813628||20827||2449311||Raymond|||||1970||||2452421| +953|AAAAAAAAJLDAAAAA|964241|5531|9574|2451092|2451062|Dr.|Nadia|Albert|N|24|6|1941|SWAZILAND||Nadia.Albert@liFoQZkQu.edu|2452331| +954|AAAAAAAAKLDAAAAA|489259|1844|46029|2451645|2451615|Miss|Ruth|White|Y|14|6|1927|GUINEA||Ruth.White@PfcakoB.edu|2452460| +955|AAAAAAAALLDAAAAA|1699532|4900|3115|2451406|2451376|Miss|Rosemary|Bryant|N|10|3|1984|PALAU||Rosemary.Bryant@KX8AurRb4I.edu|2452463| +956|AAAAAAAAMLDAAAAA|1121672|3030|20788|2451151|2451121|Miss|Lydia|Flores|N|13|7|1969|FRANCE||Lydia.Flores@Zomck7zbynQV.edu|2452484| +957|AAAAAAAANLDAAAAA|81912|7144|41976|2449610|2449580|Dr.|Kimberly|Simpson|N|29|3|1928|CAPE VERDE||Kimberly.Simpson@R9ApuIYFK.com|2452437| +958|AAAAAAAAOLDAAAAA|1234001|2233|25265|2449388|2449358|Sir|David|Banks|Y|24|8|1986|GUINEA||David.Banks@7AYfQLf.edu|2452533| +959|AAAAAAAAPLDAAAAA|223953|1976|28446|2452026|2451996|Ms.|Rose|King|Y|2|2|1972|SOUTH AFRICA||Rose.King@uaqhf6V.com|2452428| +960|AAAAAAAAAMDAAAAA|15931|2585|16439|2451612|2451582|Mrs.|Manuela|Thayer|Y|26|6|1932|PHILIPPINES||Manuela.Thayer@GeEKIU.com|2452374| +961|AAAAAAAABMDAAAAA|664635|1411|32940|2452386|2452356|Sir|Jose|Guzman|N|22|6|1947|MARSHALL ISLANDS||Jose.Guzman@stJp.edu|2452642| +962|AAAAAAAACMDAAAAA|1345722|5686|44097|2451508|2451478|Mrs.|Amy|Warren|N|10|5|1953|MOZAMBIQUE||Amy.Warren@EQ9.org|2452629| +963|AAAAAAAADMDAAAAA|1020160|596|35832|2449621|2449591|Dr.|Cameron|Godfrey|Y|1|9|1982|BURKINA FASO||Cameron.Godfrey@qpL7LNcq0v0toOD.org|2452454| +964|AAAAAAAAEMDAAAAA|957856|3485|21867|2450036|2450006|Dr.|Aline|Ramos|N|31|3|1981|EQUATORIAL GUINEA||Aline.Ramos@Gr5.edu|2452413| +965|AAAAAAAAFMDAAAAA|713730|2418|29506|2451112|2451082|Mr.|Joseph|Aponte|N|29|3|1971|COSTA RICA||Joseph.Aponte@sJnBiFHsBOd.org|2452481| +966|AAAAAAAAGMDAAAAA|695642|5651|12860|2450523|2450493|Mrs.|Sherry|Bowles|Y|16|2|1964|GIBRALTAR||Sherry.Bowles@1v8CGxTQQcOj1Bd.com|2452475| +967|AAAAAAAAHMDAAAAA|1795113|6870|39874|2450353|2450323|Sir|Earl|Sheehan|N|6|8|1925|GUINEA||Earl.Sheehan@8BElet.org|2452366| +968|AAAAAAAAIMDAAAAA|1824328|1877|21473|2452315|2452285|Dr.|Andrea|Hill|N|10|7|1973|SWEDEN||Andrea.Hill@1.com|2452522| +969|AAAAAAAAJMDAAAAA|926951|5212|7026|2449299|2449269|Mrs.|Laurie|Burrows|Y|2|5|1980|MARSHALL ISLANDS||Laurie.Burrows@F.edu|2452648| +970|AAAAAAAAKMDAAAAA|1008053|2502|16551|2450781|2450751|Ms.|Alexander|White|N|30|6|1936|FRANCE||Alexander.White@7v2XsptkFrc50K.org|2452412| +971|AAAAAAAALMDAAAAA|1175110|6863|47416|2449467|2449437|Sir|Lemuel|Waggoner|N|14|1|1927|VIRGIN ISLANDS, U.S.||Lemuel.Waggoner@Jp7.com|2452510| +972|AAAAAAAAMMDAAAAA|116199|1742|13685|2451662|2451632|Miss|Felicia|Duke|N|25|12|1965|MARSHALL ISLANDS||Felicia.Duke@foxX7cvz.org|2452547| +973|AAAAAAAANMDAAAAA|138906|6249|48392|2449291|2449261|Dr.|Brant|Johnson|N|15|5|1968|THAILAND||Brant.Johnson@gIdIlcbeMhkL.org|2452621| +974|AAAAAAAAOMDAAAAA|256184|4728|41225|2449845|2449815|Mrs.|Natalia|Rickard|N|15|12|1948|PALAU||Natalia.Rickard@3HshANcuvPs7iV.edu|2452372| +975|AAAAAAAAPMDAAAAA|825276|7168|8660|2451300|2451270|Miss|Ingrid|Cook|Y|24|1|1976|SOLOMON ISLANDS||Ingrid.Cook@D9Qp.com|2452535| +976|AAAAAAAAANDAAAAA|885721|1775|21382|2451016|2450986|Mrs.|Marjorie|Beatty|N|15|3|1955|GUERNSEY||Marjorie.Beatty@yeTdy0G9bQZz.com|2452458| +977|AAAAAAAABNDAAAAA|894602|3916|36830|2452391|2452361|Ms.|Camille|Slocum|N|30|5|1966|FINLAND||Camille.Slocum@1KQq4OgX7rGUROr.com|2452392| +978|AAAAAAAACNDAAAAA|191767|1606|33045|2451329|2451299|Mrs.|Norma|Roden|Y|1|12|1992|SPAIN||Norma.Roden@dNzAM.com|2452437| +979|AAAAAAAADNDAAAAA|1415790|6732|43674|2450383|2450353|Ms.|Arlene|Moore|Y|19|6|1984|HUNGARY||Arlene.Moore@QFY.com|2452478| +980|AAAAAAAAENDAAAAA|543772|4618|12664|2451258|2451228|Sir|Richard|Bell|N|1|12|1934|PORTUGAL||Richard.Bell@n6jT0yOrk5.com|2452563| +981|AAAAAAAAFNDAAAAA|597876|6782|9288|2450801|2450771|Dr.|Hazel|Benson|N|12|4|1972|SYRIAN ARAB REPUBLIC||Hazel.Benson@P5OvqmiMPsU24vD.com|2452516| +982|AAAAAAAAGNDAAAAA|1392814|2732|48650|2449346|2449316|Dr.|Terry|Mcdowell|N|7|10|1981|TONGA||Terry.Mcdowell@q0MspRrmB74klnp2zlo.com|2452375| +983|AAAAAAAAHNDAAAAA|24268|4272|2276|2450152|2450122|Dr.|Kevin|Palmer|Y|5|3|1967|MOLDOVA, REPUBLIC OF||Kevin.Palmer@Dm9gEh.edu|2452416| +984|AAAAAAAAINDAAAAA|990820|4406|40392|2451761|2451731|Miss|Sara|Loomis|Y|13|9|1955|MAURITIUS||Sara.Loomis@d4.org|2452432| +985|AAAAAAAAJNDAAAAA|120838|1183|49975|2450337|2450307|Mr.|Francis|Crowley|N|23|1|1977|MONACO||Francis.Crowley@6sMzNQXIcRLF.edu|2452345| +986|AAAAAAAAKNDAAAAA|295480|1317|43954|2452297|2452267|Miss|Jennifer|Larson|Y|2|11|1961|HONG KONG||Jennifer.Larson@8hTpXTCTCKdOUkZq.org|2452562| +987|AAAAAAAALNDAAAAA|312392|6166|46340|2450124|2450094|Mr.|Todd|Norris|Y|12|5|1941|TAJIKISTAN||Todd.Norris@N.org|2452617| +988|AAAAAAAAMNDAAAAA|1875282|321|27760|2451150|2451120|Miss|Lori|Wall|Y|31|12|1977|OMAN||Lori.Wall@bfdzzQhN6as.edu|2452321| +989|AAAAAAAANNDAAAAA|1027578|5416|15207|2451507|2451477|Mr.|Jack|Hoyt|N|1|9|1928|FIJI||Jack.Hoyt@v3Sa8geGZ.com|2452367| +990|AAAAAAAAONDAAAAA|497345|7056|35520|2451175|2451145|Sir|John|Thames|Y|10|2|1965|MONTSERRAT||John.Thames@fpgzn.org|2452609| +991|AAAAAAAAPNDAAAAA|431980|5803|9226|2451020|2450990|Sir|William|Aleman|Y|7|1|1963|LUXEMBOURG||William.Aleman@cGd246jZI7Dt2.com|2452366| +992|AAAAAAAAAODAAAAA|1583939|3697|27855|2449154|2449124|Mrs.|Elnora|Dabney|N|20|10|1984|ITALY||Elnora.Dabney@NUVhk1lGbprY4s.com|2452344| +993|AAAAAAAABODAAAAA|134778|298|49418|2450228|2450198|Mr.|Joseph|Fountain|Y|7|4|1942|PERU||Joseph.Fountain@9iBf4kl8dSfsfr.com|2452511| +994|AAAAAAAACODAAAAA||4792|12294||||Geneva|Archie||15|5|1948|||Geneva.Archie@eXycmPJO0e.edu|2452637| +995|AAAAAAAADODAAAAA|330521|5267|32860|2452591|2452561|Sir|Richie|Smith|Y|14|10|1960|BERMUDA||Richie.Smith@Z9FqyaNxVVr.com|2452612| +996|AAAAAAAAEODAAAAA|951614|990|13613|2449734|2449704|Dr.|Chris|Davis|Y|24|6|1947|ALBANIA||Chris.Davis@k6S3Q.com|2452570| +997|AAAAAAAAFODAAAAA|626818|5753|30335|||Sir|Harvey||N|29|8|1963|||Harvey.Stanford@sl59JiHqrp8X.org|2452632| +998|AAAAAAAAGODAAAAA|1773352|3520|4041|2449992|2449962|Ms.|Evelyn|Joyner|N|3|5|1980|TUVALU||Evelyn.Joyner@ialYx1zLN.edu|2452439| +999|AAAAAAAAHODAAAAA|780031|5696|17361|2449771|2449741|Mrs.|Lisa|Clark|Y|20|2|1957|ITALY||Lisa.Clark@goPYS4tMB0.org|2452646| +1000|AAAAAAAAIODAAAAA|647134|6138|12295|2451536|2451506|Sir|Mark|Bailey|N|26|8|1947|MOROCCO||Mark.Bailey@rg9qCNVJ0s7qeY.com|2452443| +1001|AAAAAAAAJODAAAAA|590875|1345|12662|2451585|2451555|Sir|Henry|Cunningham|Y|8|8|1932|PERU||Henry.Cunningham@5C.edu|2452354| +1002|AAAAAAAAKODAAAAA|||25966|2450288|2450258||Alexander|Nicholson|||5|1979|||Alexander.Nicholson@SJ6vcIBH.com|| +1003|AAAAAAAALODAAAAA|1661811|2329|21247|2451781|2451751|Dr.|Pamela|Reis|Y|22|8|1928|BRAZIL||Pamela.Reis@iUyNsOXoha3gyNlv1.com|2452531| +1004|AAAAAAAAMODAAAAA|875535|6717|32652|2450555|2450525|Miss|Marlene|Burton|N|27|1|1989|JORDAN||Marlene.Burton@ErIl4DGxs.org|2452602| +1005|AAAAAAAANODAAAAA|1640524|2269|29261|2452221|2452191|Dr.|Travis|Thomson|Y|4|12|1989|ISLE OF MAN||Travis.Thomson@LJcBz8yqys3.edu|2452514| +1006|AAAAAAAAOODAAAAA|844032|6468|41704|2452260|2452230|Sir|Peter|Macon|Y|3|1|1929|CROATIA||Peter.Macon@VJodnEjH6AzE0uBnbu.org|2452455| +1007|AAAAAAAAPODAAAAA|779185|1893|25354|2451389|2451359|Miss|Leanne|Mercado|Y|4|5|1971|MAYOTTE||Leanne.Mercado@eXHguBpgCJda3hNh.edu|2452578| +1008|AAAAAAAAAPDAAAAA|774000|5001|30802|2451496|2451466|Sir|Edward|Ingle|Y|24|5|1944|CAPE VERDE||Edward.Ingle@s4mkqIkeNX.com|2452412| +1009|AAAAAAAABPDAAAAA|918183|2928|7273|2450834|2450804|Mr.|James|Akers|Y|29|9|1966|MONGOLIA||James.Akers@vzRLIjJHQrsVr.edu|2452641| +1010|AAAAAAAACPDAAAAA|1281086|3100|27161|2450739|2450709|Sir|Emmett|Guerra|N|15|7|1974|WESTERN SAHARA||Emmett.Guerra@uUM6.edu|2452358| +1011|AAAAAAAADPDAAAAA|1113766|998|29783|2451290|2451260|Dr.|Megan|Gomez|N|23|9|1943|BARBADOS||Megan.Gomez@dF1MIyZljB.org|2452610| +1012|AAAAAAAAEPDAAAAA|949251|2271|44091|2451510|2451480|Mr.|Vaughn|Gunn|Y|14|10|1990|UGANDA||Vaughn.Gunn@o8.org|2452638| +1013|AAAAAAAAFPDAAAAA|87906|601|26896|2450501|2450471|Miss|Elisabeth|Downey|N|26|7|1953|PUERTO RICO||Elisabeth.Downey@a.com|2452463| +1014|AAAAAAAAGPDAAAAA|1913500|1092|29931|2449889|2449859|Miss|Charlene|Gibson|Y|11|1|1925|SYRIAN ARAB REPUBLIC||Charlene.Gibson@jtzm7O9rfC4dk.edu|2452442| +1015|AAAAAAAAHPDAAAAA|1751069|6037|37005|2450347|2450317|Miss|Rachel|Murrell|N|5|11|1927|WALLIS AND FUTUNA||Rachel.Murrell@EpIAIUcZDZVrYy.edu|2452492| +1016|AAAAAAAAIPDAAAAA|1304455|1364|13426|2451334|2451304|Miss|Georgette|Dorman|Y|9|12|1932|JORDAN||Georgette.Dorman@8P.edu|2452341| +1017|AAAAAAAAJPDAAAAA|640477|3463|19249|2450666|2450636|Sir|Jimmy|Winston|Y|20|7|1936|SIERRA LEONE||Jimmy.Winston@4REX0s3YHhQx9RReb.org|2452339| +1018|AAAAAAAAKPDAAAAA|1740186|5530|34874|2449780|2449750|Ms.|Gaye|Waters|Y|16|12|1974|GRENADA||Gaye.Waters@yNAG37MZXtuvtA8k.com|2452343| +1019|AAAAAAAALPDAAAAA|988868|3387|11106|2450603|2450573|Miss|Marsha|Anderson|Y|3|4|1958|LATVIA||Marsha.Anderson@3q28BjMKfB5n.com|2452619| +1020|AAAAAAAAMPDAAAAA|824143|3080|42417|2451267|2451237|Dr.|Robert|Miller|N|27|3|1969|FRENCH POLYNESIA||Robert.Miller@4th5duxUShO.com|2452619| +1021|AAAAAAAANPDAAAAA|827788|763|8491|2452326|2452296|Sir|Mark|Larsen|Y|15|1|1961|BERMUDA||Mark.Larsen@IRnFyj1.com|2452428| +1022|AAAAAAAAOPDAAAAA|1101984|211|33285|2449200|2449170|Dr.|Michael|Hopper|Y|3|1|1961|WALLIS AND FUTUNA||Michael.Hopper@98gOj6hCGi.com|2452460| +1023|AAAAAAAAPPDAAAAA|1201259|1690|40568|2449620|2449590|Miss|Emelina|Young|N|19|8|1956|ZIMBABWE||Emelina.Young@vcXlhQ8MiSOhu.org|2452390| +1024|AAAAAAAAAAEAAAAA||5540|43915|||||Estrada|N|15||1986|||Dean.Estrada@h5.com|| +1025|AAAAAAAABAEAAAAA|78919|4784|19896|2449955|2449925|Miss|Brandi|Pierce|Y|26|8|1979|GHANA||Brandi.Pierce@nRaR01dYFQ3Fy8Iqk.edu|2452458| +1026|AAAAAAAACAEAAAAA|1054952|4362|48525|2450157|2450127|Dr.|Allen|Sandoval|Y|9|8|1972|LATVIA||Allen.Sandoval@Ypa.com|2452554| +1027|AAAAAAAADAEAAAAA|828544|1711|33278|2452280|2452250|Sir|Armando|Pickens|Y|19|6|1927|LITHUANIA||Armando.Pickens@Ue9FQbTfyqEZ.edu|2452354| +1028|AAAAAAAAEAEAAAAA|1872205|2576|35747|2451187|2451157|Mr.|James||N||10|1935|PALAU||James.Smith@uUAFrcLNfTrlR.com|| +1029|AAAAAAAAFAEAAAAA|1057182|4722|13462|2452447|2452417|Mrs.|Ehtel|Woods|Y|26|4|1970|KYRGYZSTAN||Ehtel.Woods@S.org|2452286| +1030|AAAAAAAAGAEAAAAA|1854143|2543|6143|2450625|2450595|Sir|John|Rowland|Y|22|9|1925|CYPRUS||John.Rowland@BYtXlhHPKdf.org|2452593| +1031|AAAAAAAAHAEAAAAA||4557|41611|2450965|||April|Logan|N|27|7|1937|||April.Logan@D12kahyYlfx.edu|2452292| +1032|AAAAAAAAIAEAAAAA|685005|5906|8054|2452651|2452621|Mrs.|Dora|Guerrero|Y|24|1|1972|BAHRAIN||Dora.Guerrero@k.edu|2452389| +1033|AAAAAAAAJAEAAAAA|690788|3423|33305|2451420|2451390|Sir|Darryl|Cruz|N|11|12|1965|RWANDA||Darryl.Cruz@qR7B6x8Is.edu|2452435| +1034|AAAAAAAAKAEAAAAA|452797|3982|37167|2452252|2452222|Sir|Charles|Childress|Y|4|5|1939|C�TE D'IVOIRE||Charles.Childress@sK6.org|2452426| +1035|AAAAAAAALAEAAAAA|1390329|2035|39741|2450166|2450136|Sir|Bruce|Kelly|Y|19|6|1930|SRI LANKA||Bruce.Kelly@O8.edu|2452597| +1036|AAAAAAAAMAEAAAAA|235249|1059|44869|2449990|2449960|Dr.|Phillip|Gore|N|8|6|1950|PAPUA NEW GUINEA||Phillip.Gore@Ns7fgfm.edu|2452482| +1037|AAAAAAAANAEAAAAA|804559|1123|30087|2449828|2449798|Dr.|Paul|Vera|N|6|11|1992|PANAMA||Paul.Vera@zKMmq5.edu|2452475| +1038|AAAAAAAAOAEAAAAA|826241|6051|37814|2452529|2452499|Dr.|Jill|Harmon|N|7|4|1973|WESTERN SAHARA||Jill.Harmon@DA2vMe2dhoFXNON8.org|2452520| +1039|AAAAAAAAPAEAAAAA|695775|2661|7547|2452663|2452633|Sir|David|Davidson|N|11|9|1986|GEORGIA||David.Davidson@qJ.com|2452436| +1040|AAAAAAAAABEAAAAA|935842|5553|8111|2449682|2449652|Dr.|Joan|Parrott|N|1|11|1944|MONGOLIA||Joan.Parrott@qhg4BdeBJaju.edu|2452574| +1041|AAAAAAAABBEAAAAA|871893|943|20346|2449444|2449414|Mr.|Victor|Arrington|N|14|8|1963|SRI LANKA||Victor.Arrington@CkkIOtkYTkCS2.edu|2452629| +1042|AAAAAAAACBEAAAAA|1551128|7103|11431|2449148|2449118|Dr.|Yuri|Stephens|N|5|3|1956|SIERRA LEONE||Yuri.Stephens@s078nL5C3d7rYZOK9Jp.edu|2452609| +1043|AAAAAAAADBEAAAAA|1101649|3194|1186|2450756|2450726|Dr.|Sarah|Mcintosh|N|15|6|1937|MYANMAR||Sarah.Mcintosh@ljd4BL8NtuAyh.org|2452446| +1044|AAAAAAAAEBEAAAAA|895133|6662|31344|2451310|2451280|Sir|Patrick|Cook|N|4|9|1931|GUADELOUPE||Patrick.Cook@9kugAyME2j1xV.org|2452642| +1045|AAAAAAAAFBEAAAAA|1335362|1930|5298|2450608|2450578|Mr.|Lee|Manzo|Y|30|5|1977|ANGUILLA||Lee.Manzo@IfYDcpV4gQsCHrv.edu|2452640| +1046|AAAAAAAAGBEAAAAA|1877031|1390|13796|2451694|2451664|Sir|John|Stark|N|26|5|1932|UKRAINE||John.Stark@xip45rmGTQIq.edu|2452514| +1047|AAAAAAAAHBEAAAAA|48726|1647|28521|2449856|2449826|Dr.|Virginia|Garner|N|28|7|1963|NIGERIA||Virginia.Garner@ZD0.com|2452631| +1048|AAAAAAAAIBEAAAAA|1434908|611|38714|2451243|2451213|Miss|Adelaide|Bickford|N|17|10|1973|ANTARCTICA||Adelaide.Bickford@VtpA4IGjHl66mg2.edu|2452449| +1049|AAAAAAAAJBEAAAAA|1283455|1035|29015|2449801|2449771|Sir|Patrick|Lott|Y|24|12|1964|RWANDA||Patrick.Lott@MxONVIuuHFN.edu|2452626| +1050|AAAAAAAAKBEAAAAA|409771|6927|27730|2451887|2451857|Dr.|Kenneth|Taylor|Y|1|8|1934|CZECH REPUBLIC||Kenneth.Taylor@bt7.com|2452589| +1051|AAAAAAAALBEAAAAA|586561|5643|40372|2449306|2449276|Dr.|Marilyn|Gardner|N|10|12|1961|BOTSWANA||Marilyn.Gardner@TaNU47ZG0U.com|2452547| +1052|AAAAAAAAMBEAAAAA|1176342|5588|24881|2449067|2449037|Dr.|Bree|Barrett|N|18|12|1935|GAMBIA||Bree.Barrett@gfsZL56J.edu|2452351| +1053|AAAAAAAANBEAAAAA|1224841|4983|19572|2450516|2450486|Mr.|Robert|Rogers|Y|12|1|1992|PANAMA||Robert.Rogers@RradqLOdsEVNIM.org|2452369| +1054|AAAAAAAAOBEAAAAA|1232813|3624|29988|2450558|2450528|Ms.|Cindy|Guzman|N|23|11|1965|GUATEMALA||Cindy.Guzman@dScR3gEuhig.com|2452526| +1055|AAAAAAAAPBEAAAAA|1501281|2996|47864|2452502|2452472|Mr.|Keith|Kozlowski|Y|3|1|1945|MEXICO||Keith.Kozlowski@NXqjnTbv.edu|2452403| +1056|AAAAAAAAACEAAAAA|1592573|1858|7496|2449286|2449256|Sir|Wilbert|English|Y|18|5|1961|BHUTAN||Wilbert.English@jBSrHTANjj.edu|2452511| +1057|AAAAAAAABCEAAAAA|880477|5683|20842|2452448|2452418|Sir|Clarence|Terrell|Y|12|11|1951|GIBRALTAR||Clarence.Terrell@ODN5gy7.edu|2452328| +1058|AAAAAAAACCEAAAAA|1394888|3489|28490|2451722|2451692|Mr.|Maurice|Manley|Y|4|9|1972|FRANCE||Maurice.Manley@ZHigNlup54V.com|2452384| +1059|AAAAAAAADCEAAAAA|1414178|2382|36186|2451508|2451478|Dr.|Lori|Cervantes|Y|31|12|1979|ANGUILLA||Lori.Cervantes@n4GzBl7Zom3M.org|2452492| +1060|AAAAAAAAECEAAAAA|1072653|877|40301|2452248|2452218|Sir|August|Peterson|N|22|12|1956|ZAMBIA||August.Peterson@LxPpXYeaa0P.org|2452329| +1061|AAAAAAAAFCEAAAAA|190737|7007|20877|2452059|2452029|Mr.|Edgar|Hayes|Y|10|1|1977|DOMINICA||Edgar.Hayes@RjLssgEA.org|2452286| +1062|AAAAAAAAGCEAAAAA|985200|4373|34793|2451866|2451836|Dr.|Debra|Sallee|Y|2|5|1979|JAPAN||Debra.Sallee@ALyX33KVKy2.org|2452438| +1063|AAAAAAAAHCEAAAAA|1790189|2335|44834|2450417|2450387|Mr.|Michael|Campbell|N|23|12|1962|INDIA||Michael.Campbell@oVIuDrqBLO8Zhrs.com|2452638| +1064|AAAAAAAAICEAAAAA|1814050|3510|30217|2451804|2451774|Ms.|Angie|Thompson|N|23|12|1946|ESTONIA||Angie.Thompson@XoPJsRuLTQk.org|2452510| +1065|AAAAAAAAJCEAAAAA|806982|6357|35779|2451495|2451465|Dr.|Pearl|Murray|Y|5|10|1976|UNITED KINGDOM||Pearl.Murray@k5M97LIHF.org|2452597| +1066|AAAAAAAAKCEAAAAA|712508|6545|41029|2451054|2451024|Mrs.|Sarah|Sallee|N|26|8|1930|CHINA||Sarah.Sallee@q2uZD.com|2452602| +1067|AAAAAAAALCEAAAAA|535282|1659|39283|2451203|2451173|Ms.|Candis|Gable|N|19|9|1938|UZBEKISTAN||Candis.Gable@MzgkYMKI7RiaaZa73.com|2452377| +1068|AAAAAAAAMCEAAAAA|8989|2109|4829|2452285|2452255|Miss|Wanda|Mcclellan|Y|18|7|1988|GUYANA||Wanda.Mcclellan@CgdLlhDd0DRnmq.org|2452359| +1069|AAAAAAAANCEAAAAA|1827693|4967|30688|2450855|2450825|Dr.|John|Jacobs|N|11|7|1980|KUWAIT||John.Jacobs@qy18jgejrzTyZB.org|2452581| +1070|AAAAAAAAOCEAAAAA|79189|995|34401|2451504|2451474|Ms.|Lillian|Ward|N|14|11|1966|BOUVET ISLAND||Lillian.Ward@R.edu|2452440| +1071|AAAAAAAAPCEAAAAA|805574|5139|35057|2451134|2451104|Sir|Tom|Bennett|N|11|1|1982|QATAR||Tom.Bennett@rozAjdt78rv.org|2452630| +1072|AAAAAAAAADEAAAAA|1597469|840|46611|2450830|2450800|Mr.|Paul|Maurer|N|12|9|1942|PARAGUAY||Paul.Maurer@T8uo64RDo6.edu|2452639| +1073|AAAAAAAABDEAAAAA|1487740|3125|557|2450357|2450327|Dr.|Rosetta|Lebron|Y|10|6|1966|FINLAND||Rosetta.Lebron@I5OTqt4dfcz.com|2452616| +1074|AAAAAAAACDEAAAAA|1622609|3139|39622|2449324|2449294|Mrs.|Samantha|West|N|1|11|1964|ALAND ISLANDS||Samantha.West@ff8.org|2452580| +1075|AAAAAAAADDEAAAAA|695076|3046|37816|2449382|2449352|Mr.|Michael|Kelley|Y|31|10|1925|LESOTHO||Michael.Kelley@3BX4Z518S.com|2452361| +1076|AAAAAAAAEDEAAAAA|713638|5777|28316|2451669|2451639|Ms.|Ruth|Burgess|Y|14|1|1939|TURKMENISTAN||Ruth.Burgess@XjNzdttRUUA.com|2452339| +1077|AAAAAAAAFDEAAAAA|1367638|2766|49094|2449466|2449436|Mrs.|Jessica|Crocker|Y|25|12|1931|LEBANON||Jessica.Crocker@33uf0i3LrMSzKSJtp.com|2452406| +1078|AAAAAAAAGDEAAAAA|1723963|6332|39901|2452521|2452491|Mrs.|Melonie|Sasser|Y|19|10|1970|GUERNSEY||Melonie.Sasser@cZJErlEgRaE.org|2452522| +1079|AAAAAAAAHDEAAAAA|1297086|5811|47336|2451289|2451259|Dr.|Sam|Alves|N|9|4|1975|JERSEY||Sam.Alves@L.com|2452382| +1080|AAAAAAAAIDEAAAAA|913266|3823|13227|2450746|2450716|Sir|Basil|Black|N|20|6|1938|PANAMA||Basil.Black@K2Uler.com|2452357| +1081|AAAAAAAAJDEAAAAA|1719328|3119|10510|2452247|2452217|Dr.|Virginia|White|Y|27|4|1985|FRENCH GUIANA||Virginia.White@ujnqgC6QqE.edu|2452305| +1082|AAAAAAAAKDEAAAAA|1081860|5653|46599|2450049|2450019|Sir|Roy|Jones|N|23|7|1960|KYRGYZSTAN||Roy.Jones@gyU0syqPS.com|2452570| +1083|AAAAAAAALDEAAAAA|277438|6755|10425|2450707|2450677|Sir|Felix|Avila|N|26|1|1959|COMOROS||Felix.Avila@llAcn.edu|2452534| +1084|AAAAAAAAMDEAAAAA|117557|3792|26427|2451774|2451744|Mr.|Richard|Lynch|N|15|2|1972|VIET NAM||Richard.Lynch@a.com|2452435| +1085|AAAAAAAANDEAAAAA|1202782|4579|25976|2452239|2452209|Ms.|Arthur|Smith|N|23|1|1971|GUINEA-BISSAU||Arthur.Smith@OqFCe6.org|2452427| +1086|AAAAAAAAODEAAAAA|589505|5229|13400|2451414|2451384|Sir|Larry|Ayala|N|22|3|1952|BELIZE||Larry.Ayala@vzFAMAtQRYkL.com|2452343| +1087|AAAAAAAAPDEAAAAA||2832|13006|2449516||Mrs.||||24|||||Jennie.Quick@LSKDGIIH.org|2452590| +1088|AAAAAAAAAEEAAAAA|781995||49042|2449352||Sir|||N|||1936|PALAU||David.Wilson@8pAsLy45L3BfU.org|2452324| +1089|AAAAAAAABEEAAAAA|296632|1496|41207|2452382|2452352|Mr.|Jessie|Salazar|N|8|4|1928|AUSTRIA||Jessie.Salazar@pMXuJ9AfIsvyMIavyUh.edu|2452427| +1090|AAAAAAAACEEAAAAA|729758|4856|13554|2449542|2449512|Mr.|John|Calhoun|N|15|6|1936|CANADA||John.Calhoun@rKMjQbHK8I.com|2452345| +1091|AAAAAAAADEEAAAAA|219264|3387|3957|2450751|2450721|Sir|William|Smart|N|11|6|1969|NICARAGUA||William.Smart@oFhK4fzo1UHniyK.com|2452580| +1092|AAAAAAAAEEEAAAAA|932397|4243|21289|2449180|2449150|Sir|Rufus|Williams|N|31|12|1954|CAMEROON||Rufus.Williams@cT.com|2452298| +1093|AAAAAAAAFEEAAAAA|1136696|2253|14257|2451743|2451713|Dr.|Peter|Wright|Y|1|1|1984|MALTA||Peter.Wright@Bp5.edu|2452371| +1094|AAAAAAAAGEEAAAAA|1044591|5178|30612|2451053|2451023|Ms.|Donna|Rivera|N|8|1|1953|CROATIA||Donna.Rivera@B.com|2452525| +1095|AAAAAAAAHEEAAAAA|602640|5793|11793|2451041|2451011|Dr.|Alfredo|Dick|Y|2|8|1959|NIGER||Alfredo.Dick@BPm5QfMtic.edu|2452583| +1096|AAAAAAAAIEEAAAAA|957185|3495|36605|2451783|2451753|Mrs.|Louella|Turner|N|2|1|1972|CAPE VERDE||Louella.Turner@3QuiTQf.org|2452291| +1097|AAAAAAAAJEEAAAAA|505895|2350|30452|2450138|2450108|Mr.|Tyrone|Diaz|Y|16|8|1961|INDIA||Tyrone.Diaz@1uQFZCrqpm.org|2452491| +1098|AAAAAAAAKEEAAAAA|399258|836|47661|2450912|2450882|Dr.|Carey|Cromer|Y|14|11|1966|COSTA RICA||Carey.Cromer@J.com|2452546| +1099|AAAAAAAALEEAAAAA|1318511|438|24765|2450908|2450878|Dr.|John|Howell|N|21|8|1971|GUAM||John.Howell@XC2ARF9IVG255rvC63.edu|2452506| +1100|AAAAAAAAMEEAAAAA|1117596|5924|29499|2449882|2449852|Mrs.|Sherry|Sampson|N|7|10|1968|NIUE||Sherry.Sampson@8yO.org|2452375| +1101|AAAAAAAANEEAAAAA|1683514|7158|14402|2452133|2452103|Mr.|James|Hayes|Y|27|9|1938|MALTA||James.Hayes@suMnY3U.edu|2452562| +1102|AAAAAAAAOEEAAAAA|1246998|13|25475|2450854|2450824|Dr.|Tyler|Rush|N|6|10|1926|BANGLADESH||Tyler.Rush@IyIt6QEVoY.org|2452285| +1103|AAAAAAAAPEEAAAAA|1918458|4306|49801|2451655|2451625|Dr.|Eva|Sanford|Y|22|2|1952|CANADA||Eva.Sanford@JMVYNTXBDl6d0.edu|2452381| +1104|AAAAAAAAAFEAAAAA|238595|2504|15166|2449312|2449282|Mrs.|Josephine|Jordan|Y|20|9|1937|TOGO||Josephine.Jordan@FpixVRaMxb.com|2452302| +1105|AAAAAAAABFEAAAAA|643149|6298|38247|2452448|2452418|Mr.|William|Epstein|Y|24|6|1968|TIMOR-LESTE||William.Epstein@qgPJhH7IGISXLm3.com|2452617| +1106|AAAAAAAACFEAAAAA|95417|4809|3001|2450025|2449995|Sir|Stephen|Taylor|N|7|1|1986|CHINA||Stephen.Taylor@Es8L.com|2452399| +1107|AAAAAAAADFEAAAAA|750290|1253|31769|2450215|2450185|Sir|Mose|Preston|Y|15|3|1971|ANDORRA||Mose.Preston@XlNkumrZYDv.edu|2452315| +1108|AAAAAAAAEFEAAAAA|1856334|6479|38224|2452206|2452176|Dr.|David|Woods|N|26|9|1943|NEW CALEDONIA||David.Woods@DrduU6bh.com|2452318| +1109|AAAAAAAAFFEAAAAA|60071|212|11106|2451573|2451543|Sir|Byron|Barnes|Y|20|10|1929|GRENADA||Byron.Barnes@l4tSe.org|2452450| +1110|AAAAAAAAGFEAAAAA|311547|271|40148|2450363|2450333|Mr.|Richard|Wright|Y|16|6|1932|EQUATORIAL GUINEA||Richard.Wright@bF5oJ3Cx9zt.com|2452631| +1111|AAAAAAAAHFEAAAAA|1347101|4716|24778|2449611|2449581|Mr.|Greg|Gagnon|N|1|6|1973|MALDIVES||Greg.Gagnon@tYspqpLxyzc.com|2452641| +1112|AAAAAAAAIFEAAAAA|476083|4699|11668|2449425|2449395|Miss|Lisa|Flannery|Y|27|10|1971|THAILAND||Lisa.Flannery@4Co4zj00.com|2452402| +1113|AAAAAAAAJFEAAAAA|1537138|6504|14270|2451925|2451895|Dr.|Len|Hill|N|10|7|1985|UZBEKISTAN||Len.Hill@TNCVRsFeyadx1.org|2452434| +1114|AAAAAAAAKFEAAAAA|945605|4520|30235|2452220|2452190|Ms.|Aaron|Quarles|N|29|9|1940|NETHERLANDS ANTILLES||Aaron.Quarles@0XM3Uq.com|2452405| +1115|AAAAAAAALFEAAAAA|1370|2905|36440|2452535|2452505|Mr.|Jeremy|Atkinson|Y|28|4|1924|SAN MARINO||Jeremy.Atkinson@l7R.edu|2452391| +1116|AAAAAAAAMFEAAAAA|1703151|2823|22942|2452315|2452285|Ms.|Felisa|Singh|N|3|1|1992|PORTUGAL||Felisa.Singh@salJ2IFxlfVjLDbs.edu|2452455| +1117|AAAAAAAANFEAAAAA|679799|6019|25792|2449141|2449111|Ms.|Angela|Hooper|N|22|9|1924|ANTARCTICA||Angela.Hooper@1cp9GK0KX0z74t.com|2452592| +1118|AAAAAAAAOFEAAAAA|1043907|6265|14223|2451706|2451676|Sir|Harry|Alexander|Y|12|4|1938|EL SALVADOR||Harry.Alexander@aqko7q3h.edu|2452390| +1119|AAAAAAAAPFEAAAAA|||35626|2450925|2450895|Mr.||Ellis|N|5|||ESTONIA||George.Ellis@e077563dcQR.com|2452422| +1120|AAAAAAAAAGEAAAAA|1872929|910|44938|2451603|2451573|Mrs.|Latonya|Clayton|Y|3|2|1985|MONTSERRAT||Latonya.Clayton@jYHjkFkuFQBAQ.com|2452402| +1121|AAAAAAAABGEAAAAA|466348|3829|36510|2450320|2450290|Miss|Katherine|Ladner|N|8|2|1946|LEBANON||Katherine.Ladner@jBq8BU5doI.com|2452584| +1122|AAAAAAAACGEAAAAA|1162972|4293|48814|2452550|2452520|Mr.|Kenneth|Rivera|Y|7|2|1928|GREENLAND||Kenneth.Rivera@ebxIlfCd.edu|2452463| +1123|AAAAAAAADGEAAAAA|509787|1441|45895|2449985|2449955|Ms.|Yolanda|Hopkins|N|26|6|1986|BHUTAN||Yolanda.Hopkins@kAzDfsZbmsn.com|2452469| +1124|AAAAAAAAEGEAAAAA|1493589|936|31930|2452437|2452407|Mr.|Daniel|Hill|Y|13|3|1970|NEW ZEALAND||Daniel.Hill@Vz.org|2452536| +1125|AAAAAAAAFGEAAAAA|457836|5712|45188|2452145|2452115|Dr.|Charles|Johnson|Y|18|6|1961|MAYOTTE||Charles.Johnson@FtjN6b3sgkkNyFYDt2.com|2452318| +1126|AAAAAAAAGGEAAAAA||3893|15842|2451635||Mr.|Michael|Malone||||||||2452441| +1127|AAAAAAAAHGEAAAAA|676219|915|30811|2449036|2449006|Miss|Anastasia|Baker|Y|3|11|1939|PAPUA NEW GUINEA||Anastasia.Baker@PCZhR6ir.com|2452620| +1128|AAAAAAAAIGEAAAAA|188111|3962|8613|2451808|2451778|Ms.|Thelma|Bowers|Y|2|1|1988|WALLIS AND FUTUNA||Thelma.Bowers@U0MaLlj.org|2452610| +1129|AAAAAAAAJGEAAAAA|1211920|6385|25297|2449915|2449885|Miss|Kenya|Park|N|8|7|1938|INDONESIA||Kenya.Park@nrB6hY95M7dAIHoCOh.org|2452504| +1130|AAAAAAAAKGEAAAAA|18954|5381|36208|2452001|2451971|Mrs.|Leann|Withers|Y|30|6|1984|ZAMBIA||Leann.Withers@YE5fp88ZQ8za2le.org|2452429| +1131|AAAAAAAALGEAAAAA|600896|6578|27907|2452175|2452145|Mr.|Ralph|Stephenson|Y|15|8|1961|MOLDOVA, REPUBLIC OF||Ralph.Stephenson@lxxbt.com|2452375| +1132|AAAAAAAAMGEAAAAA|1018919|3853|35540|2451862|2451832|Miss|Elaine|Knudson|N|24|4|1981|RUSSIAN FEDERATION||Elaine.Knudson@Mht8.org|2452643| +1133|AAAAAAAANGEAAAAA|427427|2612|42197|2449052|2449022|Dr.|David|Springer|N|16|12|1960|BERMUDA||David.Springer@ihmN.org|2452647| +1134|AAAAAAAAOGEAAAAA|1709642|3388|35134|2452388|2452358|Ms.|Angela|Rose|Y|12|6|1937|NICARAGUA||Angela.Rose@O9LQadF.edu|2452424| +1135|AAAAAAAAPGEAAAAA|1065987|3159|43931|2452155|2452125|Dr.|Laura|Miller|Y|21|9|1970|GABON||Laura.Miller@ViLNxb.com|2452541| +1136|AAAAAAAAAHEAAAAA|370827|566|15858|2452193|2452163|Dr.|Jason|Jenkins|N|7|5|1969|MALAYSIA||Jason.Jenkins@7ULQclNl5y0.com|2452329| +1137|AAAAAAAABHEAAAAA|567135|4184|9233|2450603|2450573|Ms.|Samantha|Ball|Y|21|9|1940|HONDURAS||Samantha.Ball@R.com|2452326| +1138|AAAAAAAACHEAAAAA|600272|5774|40876|2450835|2450805|Miss|Claudia|Adkins|Y|2|5|1932|TOGO||Claudia.Adkins@l.edu|2452534| +1139|AAAAAAAADHEAAAAA|362396|1206|5169|2449658|2449628|Dr.|Norman|Winters|Y|24|12|1979|NIGER||Norman.Winters@ydxEDhMeo75.edu|2452484| +1140|AAAAAAAAEHEAAAAA|689726|4798|36670|2449822|2449792|Dr.|Virginia|Evans|Y|19|5|1988|INDONESIA||Virginia.Evans@UAMMg.com|2452337| +1141|AAAAAAAAFHEAAAAA|1598273|5894|39562|2450426|2450396|Mr.|Jason|Welsh|N|19|6|1959|URUGUAY||Jason.Welsh@L2Y5hmrfKf6GlGYQCg.edu|2452636| +1142|AAAAAAAAGHEAAAAA|483944|3378|26765|2450851|2450821|Mrs.|Lolita|Rogers|N|5|2|1958|UKRAINE||Lolita.Rogers@tzPMIRpk51ZFJD.com|2452530| +1143|AAAAAAAAHHEAAAAA|810770|2320|27823|2450531|2450501|Sir|Ryan|Jones|Y|21|4|1929|ESTONIA||Ryan.Jones@frcRbk.com|2452285| +1144|AAAAAAAAIHEAAAAA|484061|3624|9987|2449349|2449319|Mrs.|Albert|Williams|Y|8|8|1942|MONTENEGRO||Albert.Williams@XdSr1e9QPOmi.edu|2452638| +1145|AAAAAAAAJHEAAAAA|346507|245|32336|2449060|2449030|Ms.|Stephanie|Kelly|N|3|9|1930|TOGO||Stephanie.Kelly@MKO.org|2452454| +1146|AAAAAAAAKHEAAAAA|823061|2526|32391|2449171|2449141|Miss|Barbara|Bowen|Y|1|7|1928|CAMBODIA||Barbara.Bowen@uL7tLesoL.com|2452579| +1147|AAAAAAAALHEAAAAA|418731|5388|49598|2452245|2452215|Sir|Christopher|Swain|N|17|12|1940|GUADELOUPE||Christopher.Swain@hQuzNXPcPu5.org|2452443| +1148|AAAAAAAAMHEAAAAA|1706717|3841|39365|2451818|2451788|Mr.|Tony|Thompson|N|27|2|1941|OMAN||Tony.Thompson@fqU3oj3Kx.org|2452312| +1149|AAAAAAAANHEAAAAA|853134|1663|19839|2449791|2449761|Dr.|Henry|Weed|N|24|9|1934|GRENADA||Henry.Weed@KH6jF5LS8uGFAmY8j.org|2452347| +1150|AAAAAAAAOHEAAAAA|764353|6748|47896|2452509|2452479|Miss|Monique|Vincent|Y|31|5|1988|UKRAINE||Monique.Vincent@LN5xUI7XRxIis.edu|2452367| +1151|AAAAAAAAPHEAAAAA|341744|3978|28965|2452591|2452561|Ms.|Jenny|Davis|Y|3|9|1965|ARUBA||Jenny.Davis@X.com|2452434| +1152|AAAAAAAAAIEAAAAA|1251306|357|608|2452440|2452410|Dr.|Lorrie|Bell|N|28|2|1959|ERITREA||Lorrie.Bell@s.edu|2452529| +1153|AAAAAAAABIEAAAAA|269565|2382|14046|2451990|2451960|Miss|Anthony|Morgan|N|4|2|1976|NORWAY||Anthony.Morgan@jjhYS.com|2452491| +1154|AAAAAAAACIEAAAAA|1267891|3288|7485|2449898|2449868|Mrs.|Suzanne|Becker|N|28|7|1929|PORTUGAL||Suzanne.Becker@bEkoJ.com|2452616| +1155|AAAAAAAADIEAAAAA|444005|5081|22957|2451382|2451352|Dr.|Beatrice|Sanchez|N|7|1|1956|MOROCCO||Beatrice.Sanchez@K3hAmBNsc2TFEa.edu|2452413| +1156|AAAAAAAAEIEAAAAA|1157781|1849|15564|2450800|2450770|Sir|Steve|Morrow|Y|20|10|1930|UNITED STATES||Steve.Morrow@opSl868APEbJr.edu|2452632| +1157|AAAAAAAAFIEAAAAA|1748419|4491|39410|2452501|2452471|Mrs.|Hien|Means|N|19|11|1991|WALLIS AND FUTUNA||Hien.Means@ep.edu|2452350| +1158|AAAAAAAAGIEAAAAA|857219|4963|7107|2451275|2451245|Ms.|Michelle|Roberts|N|28|4|1982|BURKINA FASO||Michelle.Roberts@TiG.org|2452316| +1159|AAAAAAAAHIEAAAAA|841789|1647|14305|2450475|2450445|Dr.|Catherine|Johnson|N|21|3|1981|ANGOLA||Catherine.Johnson@Bfha3U.com|2452643| +1160|AAAAAAAAIIEAAAAA|1834624|742|7213|2452531|2452501|Dr.|James|Johnson|N|8|12|1990|TURKEY||James.Johnson@73.edu|2452350| +1161|AAAAAAAAJIEAAAAA|74479|5706|38533|2452125|2452095|Sir|Kyle|Castillo|Y|23|4|1962|INDIA||Kyle.Castillo@DA.org|2452325| +1162|AAAAAAAAKIEAAAAA|1548628|5016|35004|2451139|2451109|Sir|Steve|Sullivan|N|28|6|1972|SWITZERLAND||Steve.Sullivan@zUX0RFM23Kr8U.org|2452504| +1163|AAAAAAAALIEAAAAA|92279|7140|22801|2451645|2451615|Mr.|Frank|Jones|Y|4|10|1928|MONGOLIA||Frank.Jones@Bxb.edu|2452385| +1164|AAAAAAAAMIEAAAAA|1453478|500|41522|2449870|2449840|Dr.|Lieselotte|Duke|Y|11|10|1939|LATVIA||Lieselotte.Duke@QDYi6Y73k.org|2452418| +1165|AAAAAAAANIEAAAAA|119510|6240|15370|2449378|2449348|Sir|Kenneth|Sellers|N|31|8|1964|CROATIA||Kenneth.Sellers@qJnToQKSg.com|2452622| +1166|AAAAAAAAOIEAAAAA|415643|5223|2756|2451555|2451525|Mr.|Russell|Taylor|Y|16|4|1926|HUNGARY||Russell.Taylor@B.com|2452481| +1167|AAAAAAAAPIEAAAAA|1204664|1907|17197|2449075|2449045|Mr.|Donald|Sanders|Y|18|4|1948|CHRISTMAS ISLAND||Donald.Sanders@PjlLVm2PubkNbo56.org|2452544| +1168|AAAAAAAAAJEAAAAA|1191087|6481|14275|2451578|2451548|Ms.|Jennifer|Mitchell|Y|24|1|1974|JAPAN||Jennifer.Mitchell@uB9LuLDUbdZ.com|2452385| +1169|AAAAAAAABJEAAAAA|1178519|6102|36885|2450671|2450641|Miss|Sandra|Trevino|N|1|7|1968|ETHIOPIA||Sandra.Trevino@S.com|2452379| +1170|AAAAAAAACJEAAAAA|274566|5052|40738|2449369|2449339|Mr.|Eugene|Flores|Y|9|5|1991|FINLAND||Eugene.Flores@cfppFLCRsC4.com|2452529| +1171|AAAAAAAADJEAAAAA|1332426|3921|17416|2450330|2450300|Sir|Robert|Estrada|Y|14|3|1930|HAITI||Robert.Estrada@cCSDFk30.edu|2452509| +1172|AAAAAAAAEJEAAAAA|749260|4549|38285|2450560|2450530|Mrs.|Holly|Mendoza|N|4|8|1937|MALDIVES||Holly.Mendoza@i.org|2452523| +1173|AAAAAAAAFJEAAAAA|776992|6173|36154|2452204|2452174|Sir|Robert|Ryan|N|10|12|1977|TURKMENISTAN||Robert.Ryan@YUGIc7cTJ.org|2452325| +1174|AAAAAAAAGJEAAAAA|756399|4371|45896|2452064|2452034|Sir|Carlos|Belcher|Y|9|5|1953|MALI||Carlos.Belcher@tLHoQ6r.org|2452569| +1175|AAAAAAAAHJEAAAAA|1611193|4106|28572|2449870|2449840|Sir|Robert|Cooks|Y|22|9|1955|NEW ZEALAND||Robert.Cooks@1V8vK0YZtROAv7KtxGx.com|2452580| +1176|AAAAAAAAIJEAAAAA|1162656|2590|49277|2451649|2451619|Dr.|George|Baker|Y|5|7|1941|ICELAND||George.Baker@2xFaEyQ5vzD3.edu|2452539| +1177|AAAAAAAAJJEAAAAA|200622|6153|39284|2450285|2450255|Mrs.|Anthony|Wynn|Y|26|9|1984|NIGER||Anthony.Wynn@aNnGq6u1JXpsRz8.org|2452358| +1178|AAAAAAAAKJEAAAAA|1686891|3070|26049|2449559|2449529|Sir|George|Zielinski|N|13|1|1992|PERU||George.Zielinski@eoC.edu|2452592| +1179|AAAAAAAALJEAAAAA|1175834|4888|4695|2451609|2451579|Mr.|Raul|Banks|Y|18|2|1935|BRAZIL||Raul.Banks@8COrd.edu|2452508| +1180|AAAAAAAAMJEAAAAA|477919|6364|36809|2451783|2451753|Dr.|Della|Prince|Y|3|7|1926|NAMIBIA||Della.Prince@oDm.edu|2452398| +1181|AAAAAAAANJEAAAAA|1898582|3204|2626|2450008|2449978|Dr.|Trent|High|Y|22|9|1966|UZBEKISTAN||Trent.High@8Q91o3MibR6.edu|2452381| +1182|AAAAAAAAOJEAAAAA|39391|6951|29664|2451761|2451731|Mr.|David|Rudolph|N|30|12|1975|JERSEY||David.Rudolph@Kmcie8dnozNgzpVz.com|2452626| +1183|AAAAAAAAPJEAAAAA|820271|6473|17475|2449269|2449239|Mr.|Michael|Lester|N|8|11|1933|INDIA||Michael.Lester@6c48V6IkvQtsohk.com|2452592| +1184|AAAAAAAAAKEAAAAA|207984|1243|2455|2450785|2450755|Dr.|Marilyn|Taylor|N|5|1|1953|SURINAME||Marilyn.Taylor@C.edu|2452381| +1185|AAAAAAAABKEAAAAA|231009|2988|34967|2451512|2451482|Dr.|Sue|Williamson|N|7|6|1949|NEW ZEALAND||Sue.Williamson@zB2eA0Sls.edu|2452473| +1186|AAAAAAAACKEAAAAA|701178|5590|30402|2450531|2450501|Dr.|Juana|Hobson|N|24|9|1941|PALAU||Juana.Hobson@rfrVtGG46z22tIi.com|2452308| +1187|AAAAAAAADKEAAAAA|1712643|2181|38966|2450675|2450645|Miss|Lisa|Cole|N|4|7|1942|AUSTRALIA||Lisa.Cole@Hljl3eogAaXiE.edu|2452420| +1188|AAAAAAAAEKEAAAAA|856620|1013|26606|2450997|2450967|Miss|Edith|Mcconnell|Y|19|10|1941|INDIA||Edith.Mcconnell@XlIGMKUYAoGml.com|2452382| +1189|AAAAAAAAFKEAAAAA|1149061|6029|27844|2450940|2450910|Dr.|Fred|Wolfe|N|8|6|1964|SOLOMON ISLANDS||Fred.Wolfe@asr540zjTtupY8alc.com|2452572| +1190|AAAAAAAAGKEAAAAA|1654951|6987|8670|2451611|2451581|Miss|Melinda|Kimmel|N|12|11|1968|BAHAMAS||Melinda.Kimmel@QJTDdY9g.com|2452646| +1191|AAAAAAAAHKEAAAAA|841594|6628|38909|2452593|2452563|Ms.|Betty|Reed|Y|4|6|1982|WALLIS AND FUTUNA||Betty.Reed@62t6nLtAxsYodXDs.org|2452518| +1192|AAAAAAAAIKEAAAAA|149542|3069|7796|2451192|2451162|Mrs.|Brenda|Martinez|Y|22|3|1982|LIECHTENSTEIN||Brenda.Martinez@Q.org|2452624| +1193|AAAAAAAAJKEAAAAA|990777|5681|6938|2451437|2451407|Mrs.|Jennifer|Pope|N|2|2|1933|NEW CALEDONIA||Jennifer.Pope@AJxHjpUjfjq9D9.org|2452466| +1194|AAAAAAAAKKEAAAAA|821521|978|31635|2452185|2452155|Dr.|Tiffany|Ingram|N|13|5|1948|MALAYSIA||Tiffany.Ingram@SckIash67VcG.com|2452429| +1195|AAAAAAAALKEAAAAA|1404049|5774|45946|2449405|2449375|Sir|Peter|Schroeder|N|11|1|1965|DENMARK||Peter.Schroeder@lfSvKIN.com|2452640| +1196|AAAAAAAAMKEAAAAA|840082|3250|9171|2449854|2449824|Miss|Alice|Perkins|N|25|10|1988|HAITI||Alice.Perkins@aYsvt4uJzkgk88.org|2452371| +1197|AAAAAAAANKEAAAAA|721160|716|25119|2451734|2451704|Dr.|Catherine|Payne|N|13|4|1969|LIECHTENSTEIN||Catherine.Payne@NmA4tTzYDSGN2N.org|2452370| +1198|AAAAAAAAOKEAAAAA|1669287||39522|2450392|||Jerry||Y|28|4||||Jerry.Walton@NrxYJnRsx.com|2452363| +1199|AAAAAAAAPKEAAAAA|366664|2638|25012|2451873|2451843|Miss|Raquel|North|N|5|4|1963|BARBADOS||Raquel.North@K6SNSqSm.edu|2452467| +1200|AAAAAAAAALEAAAAA|1437093|179|37265|2451909|2451879|Mrs.|Rebekah|Dennis|Y|15|1|1974|CAPE VERDE||Rebekah.Dennis@J.edu|2452479| +1201|AAAAAAAABLEAAAAA|870208|2754|21300|2452474|2452444|Mr.|Vincent|French|Y|23|2|1967|CAMEROON||Vincent.French@iyPRAevQc4z.edu|2452400| +1202|AAAAAAAACLEAAAAA|302068|2454|13640|2452064|2452034|Mrs.|Amanda|Elmore|Y|5|10|1935|MEXICO||Amanda.Elmore@fioX0ytyVL1BokmU.com|2452464| +1203|AAAAAAAADLEAAAAA|1841555|4561|14367|2449949|2449919|Miss|Adele|Floyd|N|4|2|1940|BURKINA FASO||Adele.Floyd@yJ86.com|2452425| +1204|AAAAAAAAELEAAAAA|1397609|1243|27277|2449733|2449703|Sir|Bryan|Stewart|Y|27|10|1944|TUVALU||Bryan.Stewart@q9BORc9hKD.edu|2452420| +1205|AAAAAAAAFLEAAAAA|839132|3654|1729|2450000|2449970|Mrs.|Adam|Simpson|Y|28|11|1955|ROMANIA||Adam.Simpson@bVHlTtLs2gR5qG.com|2452594| +1206|AAAAAAAAGLEAAAAA|438791|2708|48672|2451020|2450990|Dr.|Martha|Dowdy|N|8|2|1938|NEW ZEALAND||Martha.Dowdy@GyZ5TZCuYb9.org|2452375| +1207|AAAAAAAAHLEAAAAA|458934|1491|4001|2449245|2449215|Dr.|William|Good|Y|5|4|1935|RUSSIAN FEDERATION||William.Good@l.com|2452550| +1208|AAAAAAAAILEAAAAA|556485|4097|18462|2452457|2452427|Dr.|Robert|Wright|Y|9|3|1985|SWITZERLAND||Robert.Wright@K5b3pveGhs.edu|2452439| +1209|AAAAAAAAJLEAAAAA|555084|4949|26414|2450087|2450057|Mrs.|Agnes|Bermudez|N|10|6|1982|MACAO||Agnes.Bermudez@sCa.com|2452453| +1210|AAAAAAAAKLEAAAAA|1213268|2016|1964|2451925|2451895|Dr.|Jayson|Ott|Y|31|3|1948|SAINT LUCIA||Jayson.Ott@mVg5.edu|2452583| +1211|AAAAAAAALLEAAAAA|656114|3962|23410|2451187|2451157|Mrs.|Luisa|Bisson|N|26|5|1925|HONG KONG||Luisa.Bisson@6NvgGkPIjF9n.com|2452464| +1212|AAAAAAAAMLEAAAAA|208708|2638|1721|2450943|2450913|Miss|Julia|Edmonds|Y|3|9|1949|SLOVENIA||Julia.Edmonds@it8NJaji.edu|2452423| +1213|AAAAAAAANLEAAAAA|1537873|2253|37033|2449186|2449156|Ms.|Helen|Briggs|N|30|1|1932|TOKELAU||Helen.Briggs@XGpnot6t8fUZcyOe.com|2452303| +1214|AAAAAAAAOLEAAAAA|1513691|2114|2584|2449663|2449633|Ms.|Andrew|Mcintyre|Y|25|7|1931|NEW ZEALAND||Andrew.Mcintyre@uMcvo84lVeuURLka.edu|2452497| +1215|AAAAAAAAPLEAAAAA|17112|1292|45072|2449217|||||N|18|7|1927|||Verna.Parker@OObXf.com|2452613| +1216|AAAAAAAAAMEAAAAA|831429|4137|4445|2449516|2449486|Mrs.|Christina|Hoffman|N|23|2|1971|MOLDOVA, REPUBLIC OF||Christina.Hoffman@7YGKUqyIZNnGBkvXCy.com|2452530| +1217|AAAAAAAABMEAAAAA|911395||4289|2451226|2451196||||Y|30|12|1956||||2452303| +1218|AAAAAAAACMEAAAAA|1278432|1059|8039|2449591|2449561|Dr.|Darren|Hayes|Y|26|7|1986|MONTSERRAT||Darren.Hayes@3tbN9kcN.com|2452484| +1219|AAAAAAAADMEAAAAA|81703|2107|26064|2450188|2450158|Mr.|Jonathan|Arnold|Y|23|5|1992|ANGUILLA||Jonathan.Arnold@EIuMbnqKq9.org|2452610| +1220|AAAAAAAAEMEAAAAA|994386|2277|46615|2450439|2450409|Ms.|Janet|Johnson|N|2|11|1963|ARGENTINA||Janet.Johnson@HzF59tQqaY.edu|2452442| +1221|AAAAAAAAFMEAAAAA|1248299|3960|1174|2450810|2450780|Dr.|James|Carney|N|9|1|1978|SLOVENIA||James.Carney@pocF9gsaPzBduVVnum.com|2452464| +1222|AAAAAAAAGMEAAAAA|1136289|7143|27197|2450989|2450959|Sir|John|Hopkins|Y|10|4|1979|AZERBAIJAN||John.Hopkins@uVJIsak4vx5Oj5U.org|2452383| +1223|AAAAAAAAHMEAAAAA|795353|3984|17368|2449040|2449010|Ms.|Christina|Fannin|N|2|12|1978|LUXEMBOURG||Christina.Fannin@UAnD73fL96v7Mv.edu|2452504| +1224|AAAAAAAAIMEAAAAA|1570031|1869|5707|2452411|2452381|Miss|Cheryl|Mullen|Y|12|9|1991|PALAU||Cheryl.Mullen@l.edu|2452500| +1225|AAAAAAAAJMEAAAAA|1173406|3197|30541|2449763|2449733|Mr.|Saul|Walling|N|2|5|1940|JAMAICA||Saul.Walling@PD5shqxUZZ.org|2452615| +1226|AAAAAAAAKMEAAAAA|1505851|2960|22022|2451270|2451240|Sir|Wesley|Guest|Y|21|3|1981|JAPAN||Wesley.Guest@VPdzdRnFqZHOVhk.edu|2452439| +1227|AAAAAAAALMEAAAAA|727286|5993|3147|2449889|2449859|Sir|Richard|Martinez|N|8|12|1960|CAMBODIA||Richard.Martinez@57EV8CqU.edu|2452387| +1228|AAAAAAAAMMEAAAAA|1399742|5851|39207|2451092|2451062|Miss|Diane|Anderson|N|9|11|1944|SWITZERLAND||Diane.Anderson@akhtHO4rkErYlC.edu|2452292| +1229|AAAAAAAANMEAAAAA|1292445|1536|24274|2449570|2449540|Dr.|Stevie|Toliver|Y|5|6|1976|TONGA||Stevie.Toliver@kZbbPcVBzyc9n2M0t.com|2452642| +1230|AAAAAAAAOMEAAAAA|1879882|4901|49567|2450330|2450300|Mr.|Ray|Swenson|N|23|9|1955|TRINIDAD AND TOBAGO||Ray.Swenson@aXTt.edu|2452474| +1231|AAAAAAAAPMEAAAAA|473411|5838|47865|2449704|2449674|Ms.|Norma|Hanson|Y|14|9|1939|VIET NAM||Norma.Hanson@OhT1X2eC.org|2452529| +1232|AAAAAAAAANEAAAAA|1013314|677|26887|2451684|2451654|Dr.|Mattie|Nelson|N|24|1|1970|NETHERLANDS||Mattie.Nelson@Hk3vPVz.org|2452481| +1233|AAAAAAAABNEAAAAA|1822044|5053|9215|2450392|2450362|Dr.|Madge|Clark|Y|8|5|1982|TOKELAU||Madge.Clark@uM.edu|2452560| +1234|AAAAAAAACNEAAAAA|746240|283|7126|2450074|2450044|Mrs.|Darcy|Davenport|Y|13|4|1988|GREENLAND||Darcy.Davenport@l163e6OIr.org|2452435| +1235|AAAAAAAADNEAAAAA|187697|3087|46477|2452112|2452082|Ms.|Shae|Lima|N|22|11|1967|FAROE ISLANDS||Shae.Lima@fcPOXKpXamTa.org|2452293| +1236|AAAAAAAAENEAAAAA|1098099|1518|24090|2449863|2449833|Mr.|Stephen|Clement|Y|23|11|1990|SAMOA||Stephen.Clement@ritGzs085oD1uZ.edu|2452600| +1237|AAAAAAAAFNEAAAAA|1230786|4761|1489|2449054|2449024|Mr.|Francis|Mcwilliams|Y|24|2|1980|JORDAN||Francis.Mcwilliams@0.edu|2452527| +1238|AAAAAAAAGNEAAAAA|677698|2657|47969|2449064|2449034|Dr.|Stephen|Delatorre|Y|13|11|1927|GUINEA||Stephen.Delatorre@pXyNB6z8.com|2452447| +1239|AAAAAAAAHNEAAAAA|1583904|1736|12415|2449967|2449937|Sir|John|Dumas|Y|3|3|1963|MOZAMBIQUE||John.Dumas@a.com|2452432| +1240|AAAAAAAAINEAAAAA|1387133|4952|35171|2450758|2450728|Mr.|Ronald|Wu|Y|18|8|1951|LIBERIA||Ronald.Wu@VpjHUOqozvtoA.edu|2452485| +1241|AAAAAAAAJNEAAAAA|1447556|1028|8853|2452612|2452582|Ms.|Robbin|Menard|N|27|12|1958|NEW ZEALAND||Robbin.Menard@P.org|2452450| +1242|AAAAAAAAKNEAAAAA|1065718|535|48599|2452615|2452585|Sir|Daniel|Bishop|Y|27|10|1937|BURUNDI||Daniel.Bishop@8Pdy8BiQAa.edu|2452432| +1243|AAAAAAAALNEAAAAA|1058195|6095|842|2449943|2449913|Dr.|Betty|Conner|N|19|7|1988|HONDURAS||Betty.Conner@Did.edu|2452385| +1244|AAAAAAAAMNEAAAAA|100751|4577|5203|2452647|2452617|Mrs.|Jill|Perkins|N|9|11|1992|ALGERIA||Jill.Perkins@eh0uJlGVhraCSBaMr.com|2452373| +1245|AAAAAAAANNEAAAAA|1638367|3713|34612|2450305|2450275|Mrs.|Ma|Cannon|N|19|12|1969|MEXICO||Ma.Cannon@30R0YzsavEaM3.com|2452485| +1246|AAAAAAAAONEAAAAA|1722295|6983|2471|2451114|2451084|Ms.|Jodi|Walton|N|14|4|1970|MONTENEGRO||Jodi.Walton@ggtzDoThJopl.org|2452442| +1247|AAAAAAAAPNEAAAAA|1770684|540|2043|2450470|2450440|Dr.|Alyssa|Zepeda|N|19|11|1982|UNITED ARAB EMIRATES||Alyssa.Zepeda@zohy7gqhde.com|2452468| +1248|AAAAAAAAAOEAAAAA|1072046|5099|32199|2449832|2449802|Dr.|Edwin|Martinez|N|2|3|1983|HONG KONG||Edwin.Martinez@KxRUsuXhhY7My2.edu|2452581| +1249|AAAAAAAABOEAAAAA|813111|2368|17978|2449394|2449364|Dr.|Tyler|Tucker|N|2|3|1977|SOLOMON ISLANDS||Tyler.Tucker@Zji8bY3Zsc2IlXc.org|2452473| +1250|AAAAAAAACOEAAAAA|280635|6948|14908|2450928|2450898|Mr.|Ray|Collett|N|19|12|1965|GREENLAND||Ray.Collett@YtERDCSBC8DVVXgu.org|2452545| +1251|AAAAAAAADOEAAAAA|532386|5161|27136|2450146|2450116|Sir|Grant|Pollard|Y|14|11|1937|BELIZE||Grant.Pollard@HLAv42114ihr5.org|2452325| +1252|AAAAAAAAEOEAAAAA|||42624|||Dr.|Larry|Harris|||||NEW ZEALAND|||| +1253|AAAAAAAAFOEAAAAA|1346566|4510|41254|2451625||||Gabriel|||9||NORWAY|||2452360| +1254|AAAAAAAAGOEAAAAA|327785|1057|35929|2449178|2449148|Miss|Andrew|Brown|Y|2|3|1965|BOLIVIA||Andrew.Brown@c2MOXVIvHA4OugkMRi.org|2452403| +1255|AAAAAAAAHOEAAAAA|1783872|2517|9562|2450543|2450513|Miss|Donna|Lewis|Y|17|12|1963|QATAR||Donna.Lewis@NmYy.edu|2452399| +1256|AAAAAAAAIOEAAAAA|1835818|3067|29516|2452421|2452391|Mr.|Donald|King|N|21|8|1979|BELGIUM||Donald.King@c3IlHcjg1sK.org|2452434| +1257|AAAAAAAAJOEAAAAA|1723837|5052|7020|2450469|2450439|Miss|Charlene|Ingram|Y|13|2|1984|SWITZERLAND||Charlene.Ingram@AH.edu|2452541| +1258|AAAAAAAAKOEAAAAA|1376896|2546|7396|2449987|2449957|Mr.|Eric|Walker|N|27|5|1985|TUNISIA||Eric.Walker@Hj.edu|2452448| +1259|AAAAAAAALOEAAAAA|1675167|297|37027|2451822|2451792|Mr.|Robert|Wright|Y|17|6|1950|TOKELAU||Robert.Wright@Fs4YyyJ8mNGm.com|2452478| +1260|AAAAAAAAMOEAAAAA|1637039|7017|46148|2449586|2449556|Miss|Virginia|Allen|N|22|12|1985|ANTARCTICA||Virginia.Allen@hE0g.edu|2452371| +1261|AAAAAAAANOEAAAAA|||30441|2452553|||Maggie|Chun|||4||||Maggie.Chun@ps5UDsNtk.com|2452362| +1262|AAAAAAAAOOEAAAAA|951708|916|40171|2451075|2451045|Dr.|Barbara|Robinson|N|27|3|1954|MAURITANIA||Barbara.Robinson@5NSkDY.edu|2452287| +1263|AAAAAAAAPOEAAAAA|7571|3134|1544|2451720|2451690|Mr.|Daniel|Villalobos|Y|17|7|1949|IRELAND||Daniel.Villalobos@ipyHK6DUvZnz.com|2452629| +1264|AAAAAAAAAPEAAAAA|1620929||13054|2450576|||Nancy||N||7|1985|SERBIA|||| +1265|AAAAAAAABPEAAAAA|1693588|1924|8260|2452517|2452487|Miss|Dora|Myers|Y|20|9|1985|RUSSIAN FEDERATION||Dora.Myers@MHmaUSlHHKKC.com|2452344| +1266|AAAAAAAACPEAAAAA|1130622|1895|32015|2450476|2450446|Mrs.|Beverly|Murray|Y|12|10|1992|HAITI||Beverly.Murray@TmYGta5.org|2452546| +1267|AAAAAAAADPEAAAAA|371691|1273|4455|2450951|2450921|Mrs.|Misty|James|N|28|8|1948|VENEZUELA||Misty.James@9m91r8h1m1.com|2452547| +1268|AAAAAAAAEPEAAAAA|44920|4182|38997|2450340|2450310|Mr.|Edward|Morse|N|9|8|1977|MOLDOVA, REPUBLIC OF||Edward.Morse@eKuH.edu|2452489| +1269|AAAAAAAAFPEAAAAA|504384|2168|20752|2449295|2449265|Ms.|Willa|Carlson|Y|23|11|1957|IRELAND||Willa.Carlson@GyVXP.edu|2452407| +1270|AAAAAAAAGPEAAAAA|1637741|1424|9860|2451130|2451100|Ms.|Hilda|Beals|Y|3|8|1967|TUNISIA||Hilda.Beals@51V.edu|2452620| +1271|AAAAAAAAHPEAAAAA|1661558|5493|13428|2452259|2452229|Dr.|Pamela|Gregg|Y|7|2|1941|FRENCH POLYNESIA||Pamela.Gregg@jSKl6xGb8n.org|2452643| +1272|AAAAAAAAIPEAAAAA|1286365|6106|34247|2452380|2452350|Sir|Timothy|Register|Y|29|10|1984|MONACO||Timothy.Register@fza.edu|2452404| +1273|AAAAAAAAJPEAAAAA|1347264|1462|33707|2449518|2449488|Sir|Ronnie|Nielsen|Y|6|9|1936|BERMUDA||Ronnie.Nielsen@6393s3k3ecj.com|2452339| +1274|AAAAAAAAKPEAAAAA|722829|747|5912|2451576|2451546|Sir|Michael|Riley|Y|25|6|1947|PERU||Michael.Riley@niNTZv9y5.org|2452352| +1275|AAAAAAAALPEAAAAA|84939|985|10231|2452350|2452320|Ms.|Gloria|Howard|N|8|1|1943|NIGERIA||Gloria.Howard@k4xEA33Nyc.org|2452304| +1276|AAAAAAAAMPEAAAAA|1279944|5168|11145|2451270|2451240|Mrs.|June|Whitt|Y|16|7|1977|MEXICO||June.Whitt@zyocu3d2d6oUMzjk.org|2452639| +1277|AAAAAAAANPEAAAAA|1817024|1221|22171|2449310|2449280|Miss|Barbara|Tomlinson|N|19|3|1930|POLAND||Barbara.Tomlinson@lUUvcZ6OMD5oN4fxVe.org|2452424| +1278|AAAAAAAAOPEAAAAA|470206|4568|6530|2452264|2452234|Sir|Kenneth|Boyd|N|31|10|1974|GREECE||Kenneth.Boyd@t7KKp.edu|2452332| +1279|AAAAAAAAPPEAAAAA|503564|5115|7466|2450940|2450910|Dr.|Keith|Marrero|Y|4|4|1939|GUAM||Keith.Marrero@AoZBFMdmixaGKe.com|2452304| +1280|AAAAAAAAAAFAAAAA|1838552|5643|9181|2451848|2451818|Sir|Gary|Hudson|N|8|8|1958|SUDAN||Gary.Hudson@4g8kg4RZ.org|2452474| +1281|AAAAAAAABAFAAAAA|1322691|4696|7548|2449599|2449569|Dr.|Ronald|Sullivan|Y|1|3|1931|BENIN||Ronald.Sullivan@Pvivt7Ud8ZF.com|2452429| +1282|AAAAAAAACAFAAAAA|964087|3971|6608|2451690|2451660|Mrs.|Angelia|Anderson|Y|1|2|1954|JAMAICA||Angelia.Anderson@79kBVOvD1hdNCXT.edu|2452579| +1283|AAAAAAAADAFAAAAA|917737|2990|17796|2450812|2450782|Dr.|David|Morton|Y|21|2|1942|EGYPT||David.Morton@x2zIyU5RypSy5C.edu|2452471| +1284|AAAAAAAAEAFAAAAA|1128470|3471|9225|2452059|2452029|Miss|Jacqueline|Pope|N|25|4|1930|BANGLADESH||Jacqueline.Pope@ZiIZzmb.edu|2452637| +1285|AAAAAAAAFAFAAAAA|1844919|3231|1496|2450648|2450618|Dr.|Mark|Blankenship|N|7|11|1951|KENYA||Mark.Blankenship@xoXVUguBsy.org|2452458| +1286|AAAAAAAAGAFAAAAA|1511091|1173|41208|2451831|2451801|Mr.|Christopher|Fenton|N|19|11|1963|LEBANON||Christopher.Fenton@M6qyVMuRljjd24.com|2452484| +1287|AAAAAAAAHAFAAAAA|1751838|5890|29095|2451055|2451025|Ms.|Natasha|Gandy|Y|25|10|1957|AUSTRALIA||Natasha.Gandy@nSBG.com|2452621| +1288|AAAAAAAAIAFAAAAA|184333|5750|9967|2451744|2451714|Sir|Jerry|Love|N|4|9|1978|LEBANON||Jerry.Love@bg.com|2452285| +1289|AAAAAAAAJAFAAAAA|716424|3672|42120|2450792|2450762|Dr.|Nora|Lam|N|20|3|1952|MARTINIQUE||Nora.Lam@plf0AiLIjXKeDj.edu|2452615| +1290|AAAAAAAAKAFAAAAA|1470533|626|39119|2449117|2449087|Sir|Douglas|Carrion|N|11|8|1985|BRUNEI DARUSSALAM||Douglas.Carrion@2ocTQS5PLyKjz.edu|2452623| +1291|AAAAAAAALAFAAAAA|434372|4364|39156|2449935|2449905|Mr.|Dennis|Miles|N|4|7|1961|ALAND ISLANDS||Dennis.Miles@bG4uDnv3FDeEnqa.org|2452489| +1292|AAAAAAAAMAFAAAAA|1521184|748|3912|2452095|2452065|Sir|Robert|Bauer|N|9|1|1942|MAURITANIA||Robert.Bauer@0ay.org|2452306| +1293|AAAAAAAANAFAAAAA|1063306|1931|45756|2449089|2449059|Miss|Wendy|Erwin|N|9|4|1940|MEXICO||Wendy.Erwin@FyDFCGSSuEB.org|2452564| +1294|AAAAAAAAOAFAAAAA|1146234|4566|26098|2450533|2450503|Ms.|Marianne|Cantwell|Y|16|7|1924|R�UNION||Marianne.Cantwell@1Anlzm.edu|2452495| +1295|AAAAAAAAPAFAAAAA|930703|6404|12905|2451868|2451838|Dr.|Lester|Walton|N|27|9|1985|RWANDA||Lester.Walton@2mx1zoOCUT1Y8MAk44K.org|2452535| +1296|AAAAAAAAABFAAAAA|35643|1373|35103|2449346|2449316|Sir|Douglas|Miller|Y|29|11|1975|EGYPT||Douglas.Miller@z4oSTYnYYIM.com|2452601| +1297|AAAAAAAABBFAAAAA|736319|5733|33994|2449035|2449005|Mrs.|Maxine|Kearney|Y|29|12|1992|ANGUILLA||Maxine.Kearney@LxrGTP.edu|2452545| +1298|AAAAAAAACBFAAAAA|997722|2480|12791|2450128|2450098|Ms.|Michelle|Sonnier|Y|28|6|1985|SENEGAL||Michelle.Sonnier@UFxKCExi4F.edu|2452304| +1299|AAAAAAAADBFAAAAA|596971|847|19076|2450152|2450122|Ms.|Bernice|Reyes|Y|9|3|1976|ALBANIA||Bernice.Reyes@uQJPbLqHKEiXpAsE.com|2452616| +1300|AAAAAAAAEBFAAAAA|1026007|6305|36250|2449214|2449184|Mr.|Martin|Morgan|Y|6|1|1943|HUNGARY||Martin.Morgan@PEdSvfm8Gd.com|2452590| +1301|AAAAAAAAFBFAAAAA|1226396|3053|40063|2450064|2450034|Sir|Terrence|Roberts|N|4|1|1936|NEW CALEDONIA||Terrence.Roberts@aAEtaS3.edu|2452293| +1302|AAAAAAAAGBFAAAAA|14691|3013|35782|2452302|2452272|Mr.|Barry|Weeks|N|10|6|1979|ETHIOPIA||Barry.Weeks@3Ea3.edu|2452478| +1303|AAAAAAAAHBFAAAAA|1177446|6381|30916|2451050|2451020|Mrs.|Elizabeth|Cole|Y|14|6|1982|VIRGIN ISLANDS, U.S.||Elizabeth.Cole@zCkIMXQFhN.com|2452469| +1304|AAAAAAAAIBFAAAAA|1283199|1412|33978|2450893|2450863|Mrs.|Zelda|Williams|Y|18|7|1963|SEYCHELLES||Zelda.Williams@HsomeikFR7j.com|2452594| +1305|AAAAAAAAJBFAAAAA|1100003|2884|45765|2452334|2452304|Sir|Joseph|Robinson|N|1|7|1975|SAINT HELENA||Joseph.Robinson@vZ8l.org|2452591| +1306|AAAAAAAAKBFAAAAA|780875|6747|25019|2449770|2449740|Miss|Gertrude|Mcnamee|Y|6|12|1943|IRAQ||Gertrude.Mcnamee@aec72.org|2452550| +1307|AAAAAAAALBFAAAAA|1329020|2156|16092|2452467|2452437|Ms.|Stella|Manning|Y|13|6|1975|BRUNEI DARUSSALAM||Stella.Manning@yhUOkVK.org|2452596| +1308|AAAAAAAAMBFAAAAA|398053|164|8722|2451047|2451017|Dr.|Dennis|Noble|Y|1|2|1950|TURKMENISTAN||Dennis.Noble@4INBeAUqrHvn2sAp.com|2452388| +1309|AAAAAAAANBFAAAAA|602069|596|7970|2450935|2450905|Sir|Charles|Francis|Y|22|3|1953|ROMANIA||Charles.Francis@f5tIHu.com|2452461| +1310|AAAAAAAAOBFAAAAA|142770|3440|43310|2451529|2451499|Ms.|Tracie|Jacobs|Y|18|11|1980|TONGA||Tracie.Jacobs@cUm3F1RlTi4.org|2452574| +1311|AAAAAAAAPBFAAAAA|1534775|1998|6774|2451649|2451619|Mr.|William|Stevens|Y|29|5|1949|GUATEMALA||William.Stevens@8hG.com|2452372| +1312|AAAAAAAAACFAAAAA|1767964|4476|3788|2451646|2451616|Dr.|William|Kinney|N|4|7|1960|EL SALVADOR||William.Kinney@OOJ9TcfB2p.edu|2452481| +1313|AAAAAAAABCFAAAAA|1085235|49|13310|2451710|2451680|Dr.|Herman|Hoffman|Y|9|2|1930|LIECHTENSTEIN||Herman.Hoffman@8L.com|2452416| +1314|AAAAAAAACCFAAAAA||2663|9763|2450926|2450896|Miss|Wendi||N|23|3||||Wendi.Smith@67Dza5zCTVuSeACV.edu|| +1315|AAAAAAAADCFAAAAA|474422|2138|7854|2451268|2451238|Mrs.|Joyce|Huey|Y|15|7|1950|TUNISIA||Joyce.Huey@ibEXVRK6FCZ2m.com|2452606| +1316|AAAAAAAAECFAAAAA|269174|4018|10015|2449225|2449195|Sir|Edward|Lewis|Y|4|2|1945|ICELAND||Edward.Lewis@BJdANQ0jd8HM.edu|2452495| +1317|AAAAAAAAFCFAAAAA|1026782|1268|12954|2449424|2449394|Mrs.|Arlene|Leonard|Y|13|6|1928|GABON||Arlene.Leonard@b7.com|2452575| +1318|AAAAAAAAGCFAAAAA|73302|6615|19928|2449288|2449258|Sir|Rodney|Richard|Y|2|2|1982|CAMEROON||Rodney.Richard@LKfMUTda.edu|2452554| +1319|AAAAAAAAHCFAAAAA|1865189|6133|32277|2449201|2449171|Sir|Luis|Drew|N|2|6|1957|ETHIOPIA||Luis.Drew@sknx.com|2452292| +1320|AAAAAAAAICFAAAAA||735|39929||2449629||||N|||1939||||| +1321|AAAAAAAAJCFAAAAA|1146269|1764|3659|2451142|2451112|Dr.|Mark|Valenzuela|Y|20|11|1927|FRENCH POLYNESIA||Mark.Valenzuela@KSpXa5.edu|2452397| +1322|AAAAAAAAKCFAAAAA|337736|6100|22111|2451802|2451772|Dr.|Philip|Etheridge|N|21|8|1925|SURINAME||Philip.Etheridge@gLQ.com|2452600| +1323|AAAAAAAALCFAAAAA|1793760|3556|38952|2451992|2451962|Ms.|Josephine|Cain|N|2|9|1979|LITHUANIA||Josephine.Cain@2BP8km.edu|2452342| +1324|AAAAAAAAMCFAAAAA|557159|76|32593|2451451|2451421|Sir|Douglas|Ames|N|4|2|1983|RWANDA||Douglas.Ames@T0vTo.edu|2452513| +1325|AAAAAAAANCFAAAAA|813905|1255|26958|2452351|2452321|Sir|Jeffrey|Brown|N|24|8|1962|ERITREA||Jeffrey.Brown@pvvHnAV554sgH.com|2452308| +1326|AAAAAAAAOCFAAAAA|1043648|6384|41120|2449833|2449803|Mrs.|Vera|Jorgenson|Y|21|2|1941|AZERBAIJAN||Vera.Jorgenson@Rqn8F.com|2452492| +1327|AAAAAAAAPCFAAAAA|1799715|538|1314|2449293|||||||6|1949||||2452297| +1328|AAAAAAAAADFAAAAA|1671868|2827|11786|2452217|2452187|Sir|Steven|Smith|Y|2|7|1990|OMAN||Steven.Smith@q8tabh.org|2452530| +1329|AAAAAAAABDFAAAAA|||49099||2451540||Sarah|Vandiver|N|14|||ETHIOPIA|||| +1330|AAAAAAAACDFAAAAA|1039407|2556|35305|2452648|2452618|Mr.|Justin|Shelton|Y|16|5|1945|PUERTO RICO||Justin.Shelton@R4Xn4ic.org|2452617| +1331|AAAAAAAADDFAAAAA|659239|6520|3001|2450149|2450119|Dr.|Janie|Browning|Y|3|12|1987|MONACO||Janie.Browning@gkPUdmkPP.edu|2452310| +1332|AAAAAAAAEDFAAAAA|1249542|669|24549|2451418|2451388|Sir|Jim|Mccray|Y|17|1|1935|SYRIAN ARAB REPUBLIC||Jim.Mccray@sy8xHv268.com|2452361| +1333|AAAAAAAAFDFAAAAA|1574976|3247|24765|2450302|2450272|Miss|Diana|Cannon|N|14|12|1956|BELIZE||Diana.Cannon@k150eazeIr1d.edu|2452641| +1334|AAAAAAAAGDFAAAAA||1560|49127|2451094|||Leonora|Arsenault|||1||VENEZUELA||Leonora.Arsenault@q00HS80jKYAD.com|2452326| +1335|AAAAAAAAHDFAAAAA|1544394|5904|23414|2449073|2449043|Sir|Ricky|Arrington|N|18|5|1990|NETHERLANDS||Ricky.Arrington@QcaSjTvL.edu|2452355| +1336|AAAAAAAAIDFAAAAA|1519793|404|43415|2450821|2450791|Sir|Harold|Cowart|N|27|12|1940|PARAGUAY||Harold.Cowart@YT9yXLOdzUDAgnx9.org|2452307| +1337|AAAAAAAAJDFAAAAA|1124475|252|31629|2449189|2449159|Dr.|Delores|Baker|N|30|10|1943|CAPE VERDE||Delores.Baker@Dcc9DNoQJ.org|2452499| +1338|AAAAAAAAKDFAAAAA|||33566||||Joseph|Freeman|N|12|7|1970|||Joseph.Freeman@R0hgyak1JacnfsjQ.org|| +1339|AAAAAAAALDFAAAAA|1754964|5335|8553|2450001|2449971|Dr.|Alexandra|Stewart|N|1|1|1947|SEYCHELLES||Alexandra.Stewart@KGTxr2gKYe.org|2452353| +1340|AAAAAAAAMDFAAAAA|1724746|2331|13869|2452210|2452180|Mr.|Dan|Lucas|N|8|3|1927|DOMINICA||Dan.Lucas@gZPnPcyX4pPr6.org|2452318| +1341|AAAAAAAANDFAAAAA|154922|6060|398|2451404|2451374|Sir|Shane|Hitchcock|N|25|6|1953|VANUATU||Shane.Hitchcock@rBY9a.edu|2452537| +1342|AAAAAAAAODFAAAAA|46291|862|27033|2449444|2449414|Ms.|Tasha|Harley|Y|17|5|1976|MONGOLIA||Tasha.Harley@Db6.org|2452382| +1343|AAAAAAAAPDFAAAAA|241978|2874|37770|2450688|2450658|Ms.|Angela|West|Y|13|5|1958|NIGER||Angela.West@pi9TvYBRqMfA.edu|2452337| +1344|AAAAAAAAAEFAAAAA|1245911|2715|46719|2452579|2452549|Ms.|Selina|Thurman|Y|7|6|1981|UGANDA||Selina.Thurman@b0YC6ghJcapxXYx9Vb.com|2452646| +1345|AAAAAAAABEFAAAAA|1427196|6468|28587|2451459|2451429|Mrs.|April|Breeden|Y|25|8|1925|INDONESIA||April.Breeden@bRkcVkpH39mYr69SB.org|2452626| +1346|AAAAAAAACEFAAAAA|1359705|2065|38025|2450663|2450633|Miss|Donna|Oneil|Y|6|9|1976|ISRAEL||Donna.Oneil@1gGafIiBdOEh4nUo.org|2452295| +1347|AAAAAAAADEFAAAAA|1739997|6101|42605|2451060|2451030|Mr.|Jason|Hart|N|11|7|1951|MALI||Jason.Hart@n5aQSiECBmd.com|2452622| +1348|AAAAAAAAEEFAAAAA|346329|6999|21669|2452516|2452486|Dr.|Charles|Fink|N|17|1|1944|CANADA||Charles.Fink@5cPLD.com|2452417| +1349|AAAAAAAAFEFAAAAA|810208|2694|35180|2449435|2449405|Dr.|Julia|Mcmillian|Y|24|2|1949|CROATIA||Julia.Mcmillian@l656F.com|2452302| +1350|AAAAAAAAGEFAAAAA|508672|5358|40091|2449191|2449161|Dr.|Vida|Moffitt|N|21|4|1951|VIET NAM||Vida.Moffitt@HXikC8LJ.edu|2452511| +1351|AAAAAAAAHEFAAAAA|1911572|868|11364|2450131|2450101|Sir|Franklin|Templeton|Y|24|9|1982|FAROE ISLANDS||Franklin.Templeton@6vXf6OB9DPI.org|2452367| +1352|AAAAAAAAIEFAAAAA|1626359|3686|13813|2449542|2449512|Dr.|Cristina|Hopkins|N|21|6|1953|LUXEMBOURG||Cristina.Hopkins@ZMU3OFcARpx7Sk65Hk2.org|2452353| +1353|AAAAAAAAJEFAAAAA|707510|3403|10449|2452563|2452533|Mrs.|Christina|Gray|N|22|2|1981|MAYOTTE||Christina.Gray@lYDrRR03uJ3.edu|2452338| +1354|AAAAAAAAKEFAAAAA|126006|4214|31575|2450409|2450379|Dr.|Billy|Mitchell|Y|1|11|1979|PALAU||Billy.Mitchell@J9s2NE.edu|2452589| +1355|AAAAAAAALEFAAAAA|761599|6936|15073|2449164|2449134|Sir|Randall|Smith|Y|5|3|1980|QATAR||Randall.Smith@2JrmlbrtuKoP.edu|2452364| +1356|AAAAAAAAMEFAAAAA|1516261|5804|3747|2449194|2449164|Ms.|Jacqueline|Lopez|Y|17|5|1973|CZECH REPUBLIC||Jacqueline.Lopez@64eMyOKL.edu|2452296| +1357|AAAAAAAANEFAAAAA|133714|6797|30633|2451937|2451907|Dr.|William|Stephens|Y|3|5|1952|JERSEY||William.Stephens@jeUoZq9IxeM.com|2452318| +1358|AAAAAAAAOEFAAAAA|1231829|1498|46767|2450432|2450402|Ms.|Sandra|Maynard|Y|28|10|1946|MALI||Sandra.Maynard@kKFv9OSHkFUc1Xbj.edu|2452331| +1359|AAAAAAAAPEFAAAAA|293460|1946|27585|2450442|2450412|Mr.|Bradley|Mullins|N|6|2|1929|KUWAIT||Bradley.Mullins@a5xNmn0.org|2452546| +1360|AAAAAAAAAFFAAAAA|1665080|6818|30956|2451884|2451854|Mrs.|Alberta|Green|N|30|1|1952|EGYPT||Alberta.Green@XmtUtui1.com|2452500| +1361|AAAAAAAABFFAAAAA|1374524|5739|3754|2449912|2449882|Sir|Jesus|Gray|N|21|11|1982|SURINAME||Jesus.Gray@gRqkodCj6d4FfFrb0.com|2452631| +1362|AAAAAAAACFFAAAAA||6761|41362||||Jewel|Ha||7|7||ANDORRA|||2452474| +1363|AAAAAAAADFFAAAAA|100670|5237|36204|2449125|2449095|Miss|Eleanor|Thomas|Y|13|12|1949|CANADA||Eleanor.Thomas@BQNt6caer.com|2452634| +1364|AAAAAAAAEFFAAAAA|215935|4397|35659|2452315|2452285|Ms.|Judith|Maddox|N|3|12|1943|SRI LANKA||Judith.Maddox@JsnCsU.edu|2452504| +1365|AAAAAAAAFFFAAAAA|1256565|6413|10261|2449325|2449295|Dr.|Agatha|Odom|N|27|3|1984|PAKISTAN||Agatha.Odom@eTujaU82odBFbM0.edu|2452619| +1366|AAAAAAAAGFFAAAAA|1281490|5730|23123|2451735|2451705|Dr.|Lorraine|Mcdonald|Y|23|7|1928|FIJI||Lorraine.Mcdonald@ZKteJKNveD.org|2452287| +1367|AAAAAAAAHFFAAAAA|1617433|5464|5095|2449071|2449041|Sir|Isaac|Stephenson|Y|26|8|1989|ANTARCTICA||Isaac.Stephenson@S.com|2452506| +1368|AAAAAAAAIFFAAAAA|12164|1572|36933|2450966|2450936|Dr.|Theodore|Snell|Y|5|10|1967|MALAWI||Theodore.Snell@OsBSUkej3jLa7Gnf.org|2452440| +1369|AAAAAAAAJFFAAAAA|285758|4311|37664|2452597|2452567|Mr.|Dana|Cannon|Y|19|1|1990|GUADELOUPE||Dana.Cannon@45aGa3z7e4SrmSh.edu|2452362| +1370|AAAAAAAAKFFAAAAA|1264091|2183|29475|2450595|2450565|Mr.|Lewis|Perdue|Y|27|11|1959|NICARAGUA||Lewis.Perdue@Sq3hBcg4.edu|2452391| +1371|AAAAAAAALFFAAAAA|61527|1406|25638|2452319|2452289|Sir|Chad|Hubbard|Y|3|10|1959|SLOVAKIA||Chad.Hubbard@Nq.com|2452530| +1372|AAAAAAAAMFFAAAAA|1313369|4814|43012|2449504|2449474|Ms.|Karen|Phifer|N|31|7|1966|ZAMBIA||Karen.Phifer@QY.edu|2452299| +1373|AAAAAAAANFFAAAAA|1239271|2326|11381|2450212|2450182|Mr.|Robert|Concepcion|N|19|4|1971|ERITREA||Robert.Concepcion@P.edu|2452318| +1374|AAAAAAAAOFFAAAAA|201838|5554|4470|2451860|2451830|Ms.|Odessa|Wynn|Y|15|2|1949|MALI||Odessa.Wynn@jddA8bYXxr3Xq.com|2452575| +1375|AAAAAAAAPFFAAAAA|1348852|909|27766|2451967|2451937|Ms.|Gladys|Muhammad|Y|10|1|1944|SOLOMON ISLANDS||Gladys.Muhammad@jjk.org|2452430| +1376|AAAAAAAAAGFAAAAA|783758|171|27784|2450662|2450632|Dr.|Billy|Randall|N|10|1|1987|NEPAL||Billy.Randall@Gqi3Ylkgi74yJQ.edu|2452445| +1377|AAAAAAAABGFAAAAA|639487|33|38979|2450830|2450800|Sir|Henry|Driscoll|Y|14|3|1985|CROATIA||Henry.Driscoll@L6Q8r8qazxv.edu|2452457| +1378|AAAAAAAACGFAAAAA|1652835|1366|18297|2451269|2451239|Sir|Kenneth|Crittenden|Y|7|2|1985|KENYA||Kenneth.Crittenden@R.org|2452599| +1379|AAAAAAAADGFAAAAA|864171|1283|30442|2451916|2451886|Sir|Larry|Perez|N|4|12|1947|AUSTRALIA||Larry.Perez@sFCsf60rG.org|2452284| +1380|AAAAAAAAEGFAAAAA|1701521|2953|9185|2452362|2452332|Mrs.|Betty|Warren|Y|3|8|1991|TRINIDAD AND TOBAGO||Betty.Warren@GDbY2m6KpT9.edu|2452292| +1381|AAAAAAAAFGFAAAAA|215689|3632|25575|2449851|2449821|Dr.|Dorothy|Scott|N|26|2|1951|ALAND ISLANDS||Dorothy.Scott@FHhCjlaZaFi.edu|2452607| +1382|AAAAAAAAGGFAAAAA|177069|6819|28969|2451452|2451422|Sir|George|Williams|N|10|12|1959|BOLIVIA||George.Williams@xfq.com|2452470| +1383|AAAAAAAAHGFAAAAA|903236|3489|1695|2451662|2451632|Mr.|Garland|Davis|N|12|7|1987|DJIBOUTI||Garland.Davis@HJCqIYUvg.org|2452573| +1384|AAAAAAAAIGFAAAAA|1871192|6662|46485|2449093|2449063|Mr.|Ian|Hogan|N|3|4|1978|SOMALIA||Ian.Hogan@aYn.com|2452353| +1385|AAAAAAAAJGFAAAAA|456649|521|24793|2449290|2449260|Mrs.|Johanna|Ross|Y|23|1|1966|GUYANA||Johanna.Ross@EiP23U7Cb5aeav.org|2452341| +1386|AAAAAAAAKGFAAAAA|224588|1528|13315|2452236|2452206|Miss|Sheila|Kirkpatrick|Y|8|3|1990|BAHRAIN||Sheila.Kirkpatrick@gTO.edu|2452536| +1387|AAAAAAAALGFAAAAA|1563181|3050|11719|2449111|2449081|Ms.|Doris|Barfield|Y|25|3|1929|SENEGAL||Doris.Barfield@u.org|2452647| +1388|AAAAAAAAMGFAAAAA|311655|1163|19424|2451142|2451112|Dr.|James|Manuel|N|6|4|1928|SLOVENIA||James.Manuel@ygYoYoT498Jn.org|2452454| +1389|AAAAAAAANGFAAAAA|1276501|712|19370|2451552|2451522|Mrs.|Dorothy|Williams|Y|9|8|1967|NEPAL||Dorothy.Williams@fCxG5GfZA.edu|2452517| +1390|AAAAAAAAOGFAAAAA|1249232|4057|20882|2451554|2451524|Sir|Jose|Boyd|N|27|1|1988|TUVALU||Jose.Boyd@paf.com|2452334| +1391|AAAAAAAAPGFAAAAA|1575611|2144|23077|2449194|2449164|Sir|Maurice|Hanks|Y|14|2|1925|SEYCHELLES||Maurice.Hanks@tcVSaANBpSIjqv.org|2452324| +1392|AAAAAAAAAHFAAAAA|1081833|6144|36855|2450015|2449985|Mr.|Louis|Ramirez|Y|26|9|1936|GREENLAND||Louis.Ramirez@5gf6PH0Ux.com|2452298| +1393|AAAAAAAABHFAAAAA|957875|3868|31908|2449730|2449700|Miss|Elisabeth|Butler|Y|6|9|1939|PANAMA||Elisabeth.Butler@abVg2BhxieXZq.org|2452639| +1394|AAAAAAAACHFAAAAA|1606671|3225|20179|2450205|2450175|Sir|Lamar|Tyner|Y|29|8|1973|MEXICO||Lamar.Tyner@ae.edu|2452330| +1395|AAAAAAAADHFAAAAA|1313573|1959|43945|2451524|2451494|Miss|Sandra|Denny|N|12|6|1963|ERITREA||Sandra.Denny@uJ8MmrS64xC5tBtu7y.com|2452310| +1396|AAAAAAAAEHFAAAAA|634298|1749|18000|2449995|2449965|Sir|Robert|Russell|N|8|11|1942|NIUE||Robert.Russell@m3PvFXlPb6Q1.com|2452417| +1397|AAAAAAAAFHFAAAAA|156859|2615|26018|2450761|2450731|Sir|Steven|Antonio|Y|9|11|1955|NIUE||Steven.Antonio@xIu.com|2452532| +1398|AAAAAAAAGHFAAAAA|1828550|4626|24749|2450748|2450718|Dr.|William|Jansen|N|28|11|1973|NORWAY||William.Jansen@vrFF.com|2452344| +1399|AAAAAAAAHHFAAAAA|1512026|5036|45246|2450985|2450955|Dr.|Judith|Gough|N|4|6|1982|FIJI||Judith.Gough@4CkvhSeCnEmenP3e.com|2452458| +1400|AAAAAAAAIHFAAAAA|1826169|180|11344|2450440|2450410|Mrs.|Irma|Medina|Y|6|7|1937|PHILIPPINES||Irma.Medina@uemqjtMlDrT2P6.com|2452450| +1401|AAAAAAAAJHFAAAAA|751312|1522|25285|2450001|2449971|Sir|Evan|Mccormick|N|19|9|1967|SEYCHELLES||Evan.Mccormick@MCY.com|2452443| +1402|AAAAAAAAKHFAAAAA|555653|3265|36077|2449391|2449361|Dr.|Rebecca|Muller|N|24|12|1956|ANTIGUA AND BARBUDA||Rebecca.Muller@Gp6cR7VI9blQ.com|2452399| +1403|AAAAAAAALHFAAAAA|1696936|6629|39522|2449737|2449707|Mrs.|Dyan|Maddox|N|17|10|1969|UZBEKISTAN||Dyan.Maddox@JykUCLL.edu|2452470| +1404|AAAAAAAAMHFAAAAA|1102685|3264|13843|2449849|2449819|Miss|Rosa|Rodriguez|N|19|12|1947|ALGERIA||Rosa.Rodriguez@rPhOAR8GEuqbq6.org|2452436| +1405|AAAAAAAANHFAAAAA|65818|943|26821|2449205|2449175|Ms.|Margie|Fernandez|N|5|3|1988|SIERRA LEONE||Margie.Fernandez@g.edu|2452477| +1406|AAAAAAAAOHFAAAAA|38145|548|12151|2450956|2450926|Mrs.|Linda|Elmore|N|1|9|1962|BOTSWANA||Linda.Elmore@o3ol.com|2452456| +1407|AAAAAAAAPHFAAAAA|1523011|4971|24531|2451502|2451472|Mr.|James|Kelly|Y|14|3|1932|SRI LANKA||James.Kelly@SMJri7R.org|2452527| +1408|AAAAAAAAAIFAAAAA|496501|987|7826|2451366|2451336|Dr.|Jose|Brice|N|20|2|1931|NETHERLANDS ANTILLES||Jose.Brice@y19di8XrEY1attt.org|2452482| +1409|AAAAAAAABIFAAAAA|189323|5692|8108|2450611|2450581|Mr.|Jeremy|Lester|N|2|2|1936|KIRIBATI||Jeremy.Lester@UjgTsfsRuig.com|2452340| +1410|AAAAAAAACIFAAAAA|243157|5660|18963|2449434|2449404|Dr.|Ruth|Travis|N|24|8|1948|AUSTRALIA||Ruth.Travis@uJbN3lJO6gNtO00L.org|2452316| +1411|AAAAAAAADIFAAAAA|1174059|3004|48749|2451468|2451438|Sir|Lloyd|Sanders|Y|15|10|1925|PHILIPPINES||Lloyd.Sanders@igP1E3kOG.com|2452343| +1412|AAAAAAAAEIFAAAAA|1628441|7061|14528|2451728|2451698|Dr.|Robert|Horton|Y|19|11|1971|WESTERN SAHARA||Robert.Horton@Qgoct5PH0ZPFb.com|2452377| +1413|AAAAAAAAFIFAAAAA|273895|1352|21656|2449594|2449564|Dr.|Bernard|Johnson|N|14|11|1947|BERMUDA||Bernard.Johnson@hqFdINRVB.edu|2452449| +1414|AAAAAAAAGIFAAAAA|1347104|3542|19227|2451860|2451830|Dr.|Gretchen|King|Y|27|10|1964|UKRAINE||Gretchen.King@aZ82RcA.org|2452304| +1415|AAAAAAAAHIFAAAAA|603261|7125|7036|2449815|2449785|Dr.|Helen|Hughes|Y|15|7|1942|HONDURAS||Helen.Hughes@bBb.org|2452570| +1416|AAAAAAAAIIFAAAAA|1662622|4180|3385|2451977|2451947|Dr.|Peg|Rice|Y|22|12|1924|BURUNDI||Peg.Rice@UHbULDL.com|2452314| +1417|AAAAAAAAJIFAAAAA|1814079|1653|39227|2449777|2449747|Miss|Fanny|Willoughby|Y|19|2|1955|MALAYSIA||Fanny.Willoughby@9p8.edu|2452333| +1418|AAAAAAAAKIFAAAAA|789035|5075|17915|2452063|2452033|Mr.|Thomas|Underwood|N|7|12|1952|BENIN||Thomas.Underwood@9DYDP7f.org|2452560| +1419|AAAAAAAALIFAAAAA|651902|390|5725|2449239|2449209|Mr.|James|Moore|N|1|12|1983|ANTARCTICA||James.Moore@h6JfczPqht.com|2452621| +1420|AAAAAAAAMIFAAAAA|1012798|777|4756|2449936|2449906|Sir|Gordon|Rutledge|Y|12|1|1937|GAMBIA||Gordon.Rutledge@o9DoHUKJpURp.com|2452415| +1421|AAAAAAAANIFAAAAA|855410|7011|14659|2450201|2450171|Dr.|Ivan|Boucher|N|27|9|1969|MAURITIUS||Ivan.Boucher@bguysFBo5.org|2452477| +1422|AAAAAAAAOIFAAAAA|1773798|2533|10068|2450576|2450546|Miss|Tamara|Nail|N|21|3|1989|PERU||Tamara.Nail@g8Smt.org|2452426| +1423|AAAAAAAAPIFAAAAA|1426096|6175|8701|2449870|2449840|Sir|Max|Mitchell|N|28|11|1980|VIRGIN ISLANDS, U.S.||Max.Mitchell@HRNzAj.com|2452453| +1424|AAAAAAAAAJFAAAAA|760447|6774|37548|2450610|2450580|Mr.|David|Mackey|N|28|8|1941|NETHERLANDS ANTILLES||David.Mackey@5zgf.edu|2452328| +1425|AAAAAAAABJFAAAAA|275664|6151|1885|2451814|2451784|Dr.|June|Preston|Y|7|6|1938|NORFOLK ISLAND||June.Preston@0rvdf.com|2452598| +1426|AAAAAAAACJFAAAAA|1048818|7044|44870|2452206|2452176|Sir|Alan|Patton|N|8|12|1950|BARBADOS||Alan.Patton@2Mm2i14.edu|2452340| +1427|AAAAAAAADJFAAAAA|347571|4163|40355|2451591|2451561|Ms.|Catherine|Hanks|N|16|8|1957|ZIMBABWE||Catherine.Hanks@XUrAjfK51bNkcBtpP.com|2452615| +1428|AAAAAAAAEJFAAAAA|114860|7129|27845|2452164|2452134|Mr.|Virgil|Kim|Y|31|12|1980|SAINT HELENA||Virgil.Kim@Zylj8tHgtlOB66Dbh.org|2452449| +1429|AAAAAAAAFJFAAAAA|1700778|6604|1230|2452442|2452412|Mr.|Jim|Smith|N|24|5|1963|GUINEA||Jim.Smith@lB5Cs2F7.com|2452379| +1430|AAAAAAAAGJFAAAAA|175454|5600|23346|2452090|2452060|Sir|William|Crumpton|Y|17|11|1925|NORWAY||William.Crumpton@PBls9RKMF9Dg.com|2452629| +1431|AAAAAAAAHJFAAAAA|888812|2316|2267|2451939|2451909|Dr.|Gary|Wilkins|N|21|2|1962|NEW ZEALAND||Gary.Wilkins@KhnF0y6SAkq.edu|2452637| +1432|AAAAAAAAIJFAAAAA|1397736|2008|2280|2449436|2449406|Mr.|Edward|Johnson|Y|7|2|1942|PERU||Edward.Johnson@9umly6u8uC.org|2452525| +1433|AAAAAAAAJJFAAAAA|1250402|2418|18787|2449193|2449163|Dr.|Derek|Evans|N|21|7|1949|ISLE OF MAN||Derek.Evans@pMDyhi5XdqsoiJx.edu|2452370| +1434|AAAAAAAAKJFAAAAA|124556||28844|2451194|2451164|Dr.||Cox|Y||5||RWANDA|||2452439| +1435|AAAAAAAALJFAAAAA|860466|5297|24928|2452615|2452585|Miss|Brenda|Daugherty|N|22|4|1952|ARUBA||Brenda.Daugherty@IUMR.org|2452613| +1436|AAAAAAAAMJFAAAAA|1112289|2706|9076|2451357|2451327|Mrs.|Mildred|Pitts|Y|28|1|1983|KAZAKHSTAN||Mildred.Pitts@Y4H5NO3Bi5Bj.com|2452511| +1437|AAAAAAAANJFAAAAA|1768655|3689|37970|2451344|2451314|Mrs.|Tina|Jordan|Y|24|11|1966|KYRGYZSTAN||Tina.Jordan@Dz3Ml2RKBQ.org|2452502| +1438|AAAAAAAAOJFAAAAA|1389175||13261|||Dr.||Turner||18|||LIBERIA||Calvin.Turner@3IrP.org|| +1439|AAAAAAAAPJFAAAAA|1521567|2603|22086|2449347|2449317|Mr.|James|Miller|Y|10|2|1969|JORDAN||James.Miller@f67VVJC.edu|2452461| +1440|AAAAAAAAAKFAAAAA|972902|335|12121|2450823|2450793|Mr.|Stephen|Stanley|N|22|5|1984|MALTA||Stephen.Stanley@8Qir2ggTKCh3.org|2452386| +1441|AAAAAAAABKFAAAAA|184074|3384|31447|2452639|2452609|Dr.|Debbie|Palmer|Y|3|12|1955|JORDAN||Debbie.Palmer@oQ1edumSA7kp.com|2452533| +1442|AAAAAAAACKFAAAAA|243498|3116|26287|2450075|2450045|Ms.|Monique|Whitcomb|N|11|2|1933|SLOVENIA||Monique.Whitcomb@Geae8D7VIr4kr.edu|2452382| +1443|AAAAAAAADKFAAAAA|1437625|919|21813|2450255|2450225|Mr.|Kenneth|Chaney|N|20|6|1952|GUERNSEY||Kenneth.Chaney@LfPAnVDgCl6nYib2X.com|2452574| +1444|AAAAAAAAEKFAAAAA|933041|162|20883|2451594|2451564|Dr.|Angel|Galindo|Y|15|8|1938|JORDAN||Angel.Galindo@ZLISZJN2lUcuB.edu|2452488| +1445|AAAAAAAAFKFAAAAA|1621872|236|29400|2451678|2451648|Ms.|Joyce|Young|N|20|11|1956|DENMARK||Joyce.Young@BY3hyMsrEej.org|2452336| +1446|AAAAAAAAGKFAAAAA|1870774|7183|22439|2450031|2450001|Mr.|Carl|Sawyer|N|27|2|1972|NIUE||Carl.Sawyer@I1ERL.org|2452468| +1447|AAAAAAAAHKFAAAAA|1347148|2350|15199|2450954|2450924|Mr.|Alfred|Frick|N|20|1|1940|MONTSERRAT||Alfred.Frick@krAc8FRH9D.org|2452355| +1448|AAAAAAAAIKFAAAAA||4851|17802||2452478|Mrs.|Cynthia|Blanchette||19|2||||Cynthia.Blanchette@kZqDLgPDPy.com|| +1449|AAAAAAAAJKFAAAAA|669888|1419|12903|2452057|2452027|Dr.|Robert|Wilson|Y|9|7|1930|FAROE ISLANDS||Robert.Wilson@k.edu|2452511| +1450|AAAAAAAAKKFAAAAA|349560|138|49116|2451514|2451484|Mr.|Francisco|Drew|N|22|2|1980|SERBIA||Francisco.Drew@qTPUhJfAmk.com|2452291| +1451|AAAAAAAALKFAAAAA|1782396|3266|23783|2451206|2451176|Mr.|John|Girard|N|14|12|1925|LIBERIA||John.Girard@rvUJ6n8Te2PXB.com|2452404| +1452|AAAAAAAAMKFAAAAA|1271964|1180|19196|2451296|2451266|Mrs.|Ora|Oh|N|9|5|1987|MARTINIQUE||Ora.Oh@cKXa8nDHF9S1KZSR.edu|2452311| +1453|AAAAAAAANKFAAAAA|950463|4965|31114|2451862|2451832|Dr.|Ashley|Hoag|N|26|10|1992|PITCAIRN||Ashley.Hoag@tqSN7Dv.com|2452547| +1454|AAAAAAAAOKFAAAAA|1645152|2948|10744|2451958|2451928|Miss|Bonnie|Valdez|Y|23|3|1978|SENEGAL||Bonnie.Valdez@Vj.com|2452291| +1455|AAAAAAAAPKFAAAAA|851297|5298|42546|2450357|2450327|Dr.|Robert|Harris|N|1|11|1970|JERSEY||Robert.Harris@s750UUIHOHPK4Ny.org|2452485| +1456|AAAAAAAAALFAAAAA|1679639|5033|37980|2451052|2451022|Dr.|Michael|Hoffman|Y|23|7|1941|SAUDI ARABIA||Michael.Hoffman@0GVn6zTFz.edu|2452523| +1457|AAAAAAAABLFAAAAA|1412994|3542|3690|2450481|2450451|Sir|Justin|Evans|Y|16|6|1957|SWEDEN||Justin.Evans@p9.com|2452547| +1458|AAAAAAAACLFAAAAA|1422942|1827|43865|2451427|2451397|Mr.|Frank|Williams|N|2|7|1941|QATAR||Frank.Williams@P3XMVae6zrJ9A8Phhb7.org|2452395| +1459|AAAAAAAADLFAAAAA|1907459|2152|10845|2451220|2451190|Dr.|John|Morris|Y|14|3|1984|MONTENEGRO||John.Morris@FP.edu|2452427| +1460|AAAAAAAAELFAAAAA|491164|2426|6088|2449068|2449038|Dr.|Christian|Gifford|Y|28|3|1976|TOGO||Christian.Gifford@pD.edu|2452557| +1461|AAAAAAAAFLFAAAAA|377858|1800|19695|2452605|2452575|Dr.|Virgil|Austin|N|2|7|1947|SEYCHELLES||Virgil.Austin@jj4bi7f2OyuJE.org|2452611| +1462|AAAAAAAAGLFAAAAA|745121|1332|3896|2450390|2450360|Sir|William|Alexander|Y|27|10|1940|MALDIVES||William.Alexander@h94.org|2452351| +1463|AAAAAAAAHLFAAAAA|943202|4446|490|2451971|2451941|Mr.|Man|Browning|N|30|10|1984|SAN MARINO||Man.Browning@Vz.com|2452429| +1464|AAAAAAAAILFAAAAA|1362956|2180|20604|2452270|2452240|Mr.|Scott|Martin|N|29|4|1945|ZAMBIA||Scott.Martin@vuA9Dyjjor63gIlCSoU.org|2452481| +1465|AAAAAAAAJLFAAAAA|814700|5746|46392|2450980|2450950|Mrs.|Karissa|Ross|N|10|7|1959|NIGER||Karissa.Ross@1oN.com|2452415| +1466|AAAAAAAAKLFAAAAA|412550|3771|592|2450419|2450389|Mr.|Darrell|Slone|N|21|9|1961|MAYOTTE||Darrell.Slone@kTP.com|2452576| +1467|AAAAAAAALLFAAAAA|776274|6929|25937|2450932|2450902|Mr.|Beau|Tabor|N|24|2|1953|TURKEY||Beau.Tabor@C1MdqQVFR6CbzT.org|2452511| +1468|AAAAAAAAMLFAAAAA|961336|5267|7998|2450241|2450211|Dr.|Richard|Helms|Y|10|7|1949|CHRISTMAS ISLAND||Richard.Helms@S2V5KrCmvFeg.edu|2452416| +1469|AAAAAAAANLFAAAAA|956972|3172|619|2449321|2449291|Dr.|Essie|Blank|Y|4|11|1948|MAYOTTE||Essie.Blank@czqveCAoYHN.com|2452529| +1470|AAAAAAAAOLFAAAAA|929052|165|15278|2451736|2451706|Mr.|George|Alarcon|Y|29|8|1940|TAJIKISTAN||George.Alarcon@tnNdcbcksFzkf.org|2452581| +1471|AAAAAAAAPLFAAAAA|447091|3341|45191|2449200|2449170|Sir|Mauricio|Nelson|Y|17|7|1979|PERU||Mauricio.Nelson@21KHI0b6fgAugC63O.edu|2452592| +1472|AAAAAAAAAMFAAAAA|1793813|3448|13836|2451147|2451117|Dr.|John|Harris|N|19|11|1949|MONTENEGRO||John.Harris@L.com|2452474| +1473|AAAAAAAABMFAAAAA|75806|5789|48619|2450975|2450945|Ms.|Jennifer|Berg|Y|11|1|1936|MALDIVES||Jennifer.Berg@5Bty2X.org|2452573| +1474|AAAAAAAACMFAAAAA|1872376|6043|16863|2452471|2452441|Miss|Joyce|Mcleod|Y|29|9|1935|WALLIS AND FUTUNA||Joyce.Mcleod@jhtK.org|2452643| +1475|AAAAAAAADMFAAAAA|273081|3287|10646|2452186|2452156|Mrs.|Ramona|Stewart|N|13|11|1971|MAYOTTE||Ramona.Stewart@sDTBcBfiOUjao.org|2452629| +1476|AAAAAAAAEMFAAAAA|1452265|4302|43139|2450503|2450473|Ms.|Lizzie|Large|Y|30|9|1987|KAZAKHSTAN||Lizzie.Large@l0dApTFrmvX3.org|2452440| +1477|AAAAAAAAFMFAAAAA|1621972|5675|6931|2449114|2449084|Mr.|Robert|York|N|10|10|1964|ANGUILLA||Robert.York@j0.org|2452329| +1478|AAAAAAAAGMFAAAAA|1008467|2719|29371|2451347|2451317|Dr.|Laura|Armstrong|Y|29|7|1975|MONGOLIA||Laura.Armstrong@Z8EdjXLgYyV42H.edu|2452345| +1479|AAAAAAAAHMFAAAAA|1675287|832|5530|2449521|2449491|Dr.|Lorraine|Newman|Y|24|4|1972|GHANA||Lorraine.Newman@DylbP.edu|2452612| +1480|AAAAAAAAIMFAAAAA|578977|821|7865|2452122|2452092|Mr.|Denis|Jones|N|10|12|1930|BAHAMAS||Denis.Jones@dgUvNdil1n.edu|2452340| +1481|AAAAAAAAJMFAAAAA|1818491|3600|46662|2450348|2450318|Ms.|Laurie|Delaney|N|10|1|1936|SWITZERLAND||Laurie.Delaney@kyu7QEdehvH6n.edu|2452520| +1482|AAAAAAAAKMFAAAAA|1072096|1326|2909|2449121|2449091|Miss|Amy|Goldstein|Y|19|10|1990|UGANDA||Amy.Goldstein@Y53iKMGnoUX0.org|2452644| +1483|AAAAAAAALMFAAAAA|1501036|5835|25434|2452550|2452520|Dr.|Richard|Dillard|Y|23|7|1991|ICELAND||Richard.Dillard@kUo50i9Gv5zK6S.com|2452376| +1484|AAAAAAAAMMFAAAAA|18954|841|28728|2449197|2449167|Dr.|Barbara|Todd|Y|20|2|1971|MONGOLIA||Barbara.Todd@0GtZUrbxPDM.org|2452285| +1485|AAAAAAAANMFAAAAA|704198|6661|40324|2451008|2450978|Sir|Bill|Allen|N|2|7|1932|CYPRUS||Bill.Allen@EOeTM.com|2452595| +1486|AAAAAAAAOMFAAAAA|1839042|1643|30597|2449048|2449018|Dr.|Chuck|Kohler|Y|12|7|1935|GUYANA||Chuck.Kohler@YQGN1.org|2452344| +1487|AAAAAAAAPMFAAAAA|558806|463|41367|2449105|2449075|Mr.|Brenton|Field|Y|12|4|1929|JERSEY||Brenton.Field@JJuQ.org|2452419| +1488|AAAAAAAAANFAAAAA|524311|5716|21921|2451941|2451911|Sir|Philip|Banks|Y|9|2|1933|SAINT LUCIA||Philip.Banks@VO0bANr.com|2452532| +1489|AAAAAAAABNFAAAAA|1199017|6654|16738|2449296|2449266|Miss|Wilma|Domingo|Y|18|3|1973|TURKEY||Wilma.Domingo@zKqoCGxduP49.edu|2452506| +1490|AAAAAAAACNFAAAAA|||12790||||Lupe||Y||5||||Lupe.White@pFaRcFn2yZP.edu|| +1491|AAAAAAAADNFAAAAA|1878706|6581|7209|2450431|2450401|Sir|Christopher|Crabtree|Y|13|10|1978|VIET NAM||Christopher.Crabtree@iTHzYIuzeHCMN.com|2452362| +1492|AAAAAAAAENFAAAAA|||2631||2452365|Dr.|||Y||||TIMOR-LESTE||Jenny.Schmidt@OeCXovIu7y.org|| +1493|AAAAAAAAFNFAAAAA||4914|32588||2449878|Mr.|Tyrone|Wells||8|4|1955|ALAND ISLANDS|||2452334| +1494|AAAAAAAAGNFAAAAA|943672|1323|33727|2452528|2452498|Miss|Dolly|Simmons|Y|22|1|1955|SWEDEN||Dolly.Simmons@Y4VzK3YsKOYa.org|2452318| +1495|AAAAAAAAHNFAAAAA|722077|6775|13520|2451423|2451393|Dr.|David|Vasquez|N|12|7|1924|NIUE||David.Vasquez@PB1lb8V.edu|2452378| +1496|AAAAAAAAINFAAAAA|630892|1409|30679|2450850|2450820|Mr.|Oscar|Cain|Y|17|2|1978|BOLIVIA||Oscar.Cain@syk.com|2452567| +1497|AAAAAAAAJNFAAAAA|898165|1994|31023|2449459|2449429|Dr.|Marian|Thomas|N|27|9|1992|DJIBOUTI||Marian.Thomas@Mir.org|2452476| +1498|AAAAAAAAKNFAAAAA|697901|6527|46800|2451705|2451675|Ms.|April|Jackson|Y|18|7|1967|GERMANY||April.Jackson@A1RGkA.edu|2452505| +1499|AAAAAAAALNFAAAAA|523419|4099|8826|2450936|2450906|Mrs.|Thelma|Mcfarland|Y|17|3|1987|COMOROS||Thelma.Mcfarland@6l5arlXkZg.edu|2452325| +1500|AAAAAAAAMNFAAAAA|965969|2378|27187|2451765|2451735|Ms.|Sandra|Branch|Y|14|6|1944|GUINEA-BISSAU||Sandra.Branch@gqd6.org|2452329| +1501|AAAAAAAANNFAAAAA|101515|2163|45611|2452012|2451982|||Donohue|Y|20||1935||||2452497| +1502|AAAAAAAAONFAAAAA|1830525|5579|27433|2450984|2450954|Mr.|Richard|Foote|N|24|2|1978|HAITI||Richard.Foote@MS2rSjcrjjf.org|2452301| +1503|AAAAAAAAPNFAAAAA|234229|1285|26694|2450983|2450953|Miss|Pauline|Brooks|Y|7|6|1987|QATAR||Pauline.Brooks@BOYVmC9mmc0Xfil.org|2452342| +1504|AAAAAAAAAOFAAAAA|586671|1875|14679|2449324|2449294|Ms.|Dyan|Arnold|N|16|10|1932|AUSTRALIA||Dyan.Arnold@g8BCzjT1Es.com|2452308| +1505|AAAAAAAABOFAAAAA|1543017|5346|17322|2450481|2450451|Dr.|Conrad|Perez|N|10|10|1933|R�UNION||Conrad.Perez@h7IHPQqiF4Np1.org|2452307| +1506|AAAAAAAACOFAAAAA|452852|6900|33795|2451833|2451803|Ms.|Tammy|Searcy|N|6|5|1989|SURINAME||Tammy.Searcy@6ZFFGtIuCqT.com|2452295| +1507|AAAAAAAADOFAAAAA||2284|39377|||||Harris||1||1969|HAITI|||| +1508|AAAAAAAAEOFAAAAA||5021|34106||||Violet|Lopez||||1987|KYRGYZSTAN|||| +1509|AAAAAAAAFOFAAAAA|||1870||2450862|Dr.|||||8||POLAND||Henry.Garner@e62EQZTh6O.edu|2452610| +1510|AAAAAAAAGOFAAAAA|1849188|7143|32745|2450750|2450720|Mr.|Donald|Jones|Y|13|1|1960|SAN MARINO||Donald.Jones@mBJBqg35hpsiA7pqM.org|2452648| +1511|AAAAAAAAHOFAAAAA|355554|2007|13|2449814|2449784|Sir|Stephen|Rickard|N|9|10|1967|SAMOA||Stephen.Rickard@GUt6gkL2yezXxnMGIp5.org|2452488| +1512|AAAAAAAAIOFAAAAA|509494||19864|||Sir||Valdez|N||7||ARMENIA||Bobby.Valdez@MRXhzkoEmCIlDS.edu|2452324| +1513|AAAAAAAAJOFAAAAA|1565929|2687|7045|2452492|2452462|Mrs.|Deanne|Melvin|Y|15|12|1973|ARGENTINA||Deanne.Melvin@ks.edu|2452563| +1514|AAAAAAAAKOFAAAAA|1133712|1238|12411|2449268|2449238|Dr.|Robert|Guy|Y|6|9|1958|LIECHTENSTEIN||Robert.Guy@BiisOpPr1z.org|2452479| +1515|AAAAAAAALOFAAAAA|1679636|5065|38847|2451435|2451405|Sir|Nicholas|Campos|N|24|11|1966|LIBERIA||Nicholas.Campos@RhyEetMZXtpFTx.org|2452479| +1516|AAAAAAAAMOFAAAAA|250925|4714|1703|2450636|2450606|Miss|Coleen|Swain|Y|13|3|1963|LEBANON||Coleen.Swain@L8X88uUbbCtV0I9x.org|2452625| +1517|AAAAAAAANOFAAAAA|593039|7155|14871|2449970|2449940|Dr.|Monte|Smith|Y|24|4|1979|BELARUS||Monte.Smith@4dax5pYOq6Msb.edu|2452531| +1518|AAAAAAAAOOFAAAAA|430496|4864|36032|2452349|2452319|Ms.|Trudie|Richardson|N|30|1|1936|C�TE D'IVOIRE||Trudie.Richardson@3y2jzVCN.com|2452335| +1519|AAAAAAAAPOFAAAAA|1290379|7106|38680|2451674|2451644|Sir|Tomas|Horner|Y|21|5|1932|NIGER||Tomas.Horner@tfuIQsq.edu|2452625| +1520|AAAAAAAAAPFAAAAA|1828742|5951|7113|2452460|2452430|Ms.|Robyn|Eldridge|N|14|6|1936|FINLAND||Robyn.Eldridge@xddRqcth.edu|2452627| +1521|AAAAAAAABPFAAAAA|121649|2047|48398|2449872|2449842|Dr.|Walter|Hall|N|25|6|1959|SOLOMON ISLANDS||Walter.Hall@O.edu|2452327| +1522|AAAAAAAACPFAAAAA|1557178|1169|7560|2452005|2451975|Dr.|James|Wilson|Y|14|11|1971|FRANCE||James.Wilson@5Dkv1kCsbAk0l.com|2452388| +1523|AAAAAAAADPFAAAAA|502270|5076|20329|2449315|2449285|Mrs.|Sally|Parker|N|11|5|1937|JORDAN||Sally.Parker@dJ4bDCQeV.com|2452439| +1524|AAAAAAAAEPFAAAAA|1170495|2567|27792|2452133|2452103|Mrs.|Teresa|Sears|N|19|12|1971|GUINEA-BISSAU||Teresa.Sears@8sraLYjM9c9CM0.org|2452397| +1525|AAAAAAAAFPFAAAAA|897066|1664|33109|2449829|2449799|Sir|Cody|Kramer|Y|4|6|1954|NIUE||Cody.Kramer@g7eHjIOf3yF9vSaY.edu|2452515| +1526|AAAAAAAAGPFAAAAA|224549|6464|33179|2449548|2449518|Mrs.|Eugenia|Sams|N|25|6|1943|VIET NAM||Eugenia.Sams@qUrfnaVP.edu|2452357| +1527|AAAAAAAAHPFAAAAA|1918973|2470|29967|2452078|2452048|Sir|Armando|Moore|N|18|4|1940|SLOVAKIA||Armando.Moore@gtRT3CTivdN3.edu|2452547| +1528|AAAAAAAAIPFAAAAA|1761506|3147|24788|2452024|2451994|Dr.|Joseph|Earnest|Y|22|10|1942|C�TE D'IVOIRE||Joseph.Earnest@fxpQkNjdJq.com|2452304| +1529|AAAAAAAAJPFAAAAA|1564366|2849|37732|2450385|2450355|Miss|Barbara|Michels|N|2|1|1936|FRANCE||Barbara.Michels@XqSybNbuL74p.org|2452633| +1530|AAAAAAAAKPFAAAAA|963417|6874|16378|2450712|2450682|Dr.|Jeremy|Freeman|N|14|5|1936|COSTA RICA||Jeremy.Freeman@Rbr.edu|2452399| +1531|AAAAAAAALPFAAAAA|837945|1502|37059|2451915|2451885|Mrs.|Joy|Vernon|Y|24|5|1967|ISRAEL||Joy.Vernon@6IVjHmfqFJINjo.com|2452624| +1532|AAAAAAAAMPFAAAAA|1513291|63|9635|2450103|2450073|Ms.|Andrew|Lowery|N|3|11|1957|KAZAKHSTAN||Andrew.Lowery@T7T4.com|2452296| +1533|AAAAAAAANPFAAAAA|1560708|6661|41461|2450615|2450585|Dr.|Jason|Fritz|Y|2|3|1928|NETHERLANDS ANTILLES||Jason.Fritz@Mx69qqcO.com|2452592| +1534|AAAAAAAAOPFAAAAA|75568|2796|46424|2452357|2452327|Dr.|Jeffrey|Williams|N|28|4|1985|BANGLADESH||Jeffrey.Williams@CUxcz.edu|2452511| +1535|AAAAAAAAPPFAAAAA|1893445|6997|44549|2450817|2450787|Ms.|Krystle|Paine|Y|23|3|1968|ANDORRA||Krystle.Paine@fdANmpR.com|2452294| +1536|AAAAAAAAAAGAAAAA|505360|5545|12483|2450055|2450025|Miss|Myra|Mccann|N|21|6|1935|BOLIVIA||Myra.Mccann@gH.com|2452539| +1537|AAAAAAAABAGAAAAA|951754|267|38427|2450409|2450379|Sir|Joshua|Wright|Y|17|9|1969|CAMEROON||Joshua.Wright@2peFv.edu|2452440| +1538|AAAAAAAACAGAAAAA|24107|3112|31317|2449715|2449685|Ms.|Christina|Groves|Y|7|11|1943|BOLIVIA||Christina.Groves@V.org|2452510| +1539|AAAAAAAADAGAAAAA|1617700|924|46948|2452067|2452037|Dr.|Vance|Atkinson|Y|16|12|1983|ZIMBABWE||Vance.Atkinson@59KuyHNFGvpN.org|2452632| +1540|AAAAAAAAEAGAAAAA|857227|5945|18263|2450061|2450031|Dr.|Bruce|Ball|Y|24|4|1946|SWEDEN||Bruce.Ball@Ecb.edu|2452345| +1541|AAAAAAAAFAGAAAAA|133879|2339|47152|2451862|2451832|Dr.|Michael|Fair|N|25|8|1952|BAHRAIN||Michael.Fair@RnXjGJxxVosOx.org|2452493| +1542|AAAAAAAAGAGAAAAA|485232|1903|49327|2449255|2449225|Mr.|Rafael|Vargas|N|19|9|1973|PANAMA||Rafael.Vargas@8Uby8YrVUjs.edu|2452365| +1543|AAAAAAAAHAGAAAAA|36373||23938|2451755||||Key||29|6|1960|C�TE D'IVOIRE|||2452311| +1544|AAAAAAAAIAGAAAAA|996768|3748|27481|2449558|2449528|Ms.|Deborah|Hoover|Y|22|8|1957|BAHRAIN||Deborah.Hoover@EtB0t0b.com|2452607| +1545|AAAAAAAAJAGAAAAA|906762|3858|34154|2449688|2449658|Mrs.|Margaret|Meyer|N|27|7|1930|UKRAINE||Margaret.Meyer@HejmLtg1bTu7iTY.org|2452518| +1546|AAAAAAAAKAGAAAAA|541041|5192|40496|2452418|2452388|Miss|Gertrude|Bowser|Y|21|3|1938|ISLE OF MAN||Gertrude.Bowser@nLpTFRXXv.com|2452479| +1547|AAAAAAAALAGAAAAA|1872150|2207|46664|2451298|2451268|Mr.|Jerry|Mccants|N|30|7|1964|MADAGASCAR||Jerry.Mccants@136ouG0f2Vf8gEfQA.com|2452516| +1548|AAAAAAAAMAGAAAAA|791661|3055|43649|2450158|2450128|Mr.|Gary|Anderson|Y|30|1|1971|TOGO||Gary.Anderson@3NH.edu|2452612| +1549|AAAAAAAANAGAAAAA|438737|3902|32667|2451772|2451742|Sir|Rickie|Davis|N|11|3|1925|SIERRA LEONE||Rickie.Davis@iIiC.org|2452393| +1550|AAAAAAAAOAGAAAAA|31774|2541|36508|2449850|2449820|Dr.|Willie|Ward|Y|20|4|1939|INDONESIA||Willie.Ward@MSQhC4.com|2452404| +1551|AAAAAAAAPAGAAAAA|112716|2974|1632|2452234|2452204|Sir|Dario|Brown|N|9|11|1953|BHUTAN||Dario.Brown@dFpD7Xtbp4StFal.com|2452525| +1552|AAAAAAAAABGAAAAA|22424|881|30185|2450489|2450459|Sir|Justin|Appleton|Y|24|8|1967|GEORGIA||Justin.Appleton@9Sxxll5ri71u.org|2452450| +1553|AAAAAAAABBGAAAAA|763113|1713|45913|2449960|2449930|Dr.|Samantha|Cornelius|N|13|7|1960|HAITI||Samantha.Cornelius@cr1MTN7ZMrK56.edu|2452488| +1554|AAAAAAAACBGAAAAA|912701|614|13327|2452289|2452259|Ms.|Judy|Farrell|Y|25|3|1975|SOLOMON ISLANDS||Judy.Farrell@yqO4Uc5I312QeVCyC.org|2452288| +1555|AAAAAAAADBGAAAAA|1607214|3662|41710|2450388|2450358|Dr.|Marvin|Roush|Y|18|10|1967|PUERTO RICO||Marvin.Roush@RNuQ8Gvdpd.com|2452478| +1556|AAAAAAAAEBGAAAAA|1272866|984|5502|2451730|2451700|Ms.|Donna|Pape|N|26|3|1978|SLOVAKIA||Donna.Pape@Z9xT2AFpx6H2S.com|2452336| +1557|AAAAAAAAFBGAAAAA|1562152|3146|19340|2450102|2450072|Dr.|Annette|Ashcraft|Y|14|5|1936|NIGERIA||Annette.Ashcraft@9Ftqfz.org|2452550| +1558|AAAAAAAAGBGAAAAA|1116306|3575|29644|2451840|2451810|Mr.|Jamar|Parish|Y|18|9|1960|TOGO||Jamar.Parish@ol8.org|2452598| +1559|AAAAAAAAHBGAAAAA|1699836|6914|38345|2449861|2449831|Dr.|Michael|Anderson|N|18|8|1987|ALGERIA||Michael.Anderson@ci6pkJNa.org|2452466| +1560|AAAAAAAAIBGAAAAA|550413|3780|21994|2449496|2449466|Miss|Sheila|Ingram|N|29|8|1967|NETHERLANDS||Sheila.Ingram@aKM09EOMzBqjPx.edu|2452367| +1561|AAAAAAAAJBGAAAAA|90277|3502|27080|2452177|2452147|Mr.|Clifford|Benge|N|13|7|1951|IRAQ||Clifford.Benge@BJzRndyVC.com|2452322| +1562|AAAAAAAAKBGAAAAA|1081671|2008|47145|2449032|2449002|Mrs.|Judith|Moore|N|9|5|1928|PAKISTAN||Judith.Moore@YCBQcdofjldgQ7.org|2452486| +1563|AAAAAAAALBGAAAAA|355876|2514|9152|2450960|2450930|Dr.|Giovanna|Stpierre|Y|22|10|1934|GUAM||Giovanna.Stpierre@faLtsH6sVqMRMr7.org|2452459| +1564|AAAAAAAAMBGAAAAA|427706|3697|14488|2452021|2451991|Ms.|Rebecca|Anderson|N|12|8|1981|NAMIBIA||Rebecca.Anderson@rgVZKnVUF8h.com|2452513| +1565|AAAAAAAANBGAAAAA|||21092|2449843|||Ada|||8||1981|SUDAN|||2452311| +1566|AAAAAAAAOBGAAAAA|1547702|6670|39685|2451996|2451966|Mr.|Walter|Button|N|14|3|1973|FAROE ISLANDS||Walter.Button@2vJ4uKSLt.edu|2452405| +1567|AAAAAAAAPBGAAAAA|138134|4244|39954|2450895|2450865|Mrs.|Ellen|Morales|N|5|1|1951|NICARAGUA||Ellen.Morales@E7K7Yo5pv.edu|2452468| +1568|AAAAAAAAACGAAAAA|840718|239|37360|2449749|2449719|Dr.|Robert|Reid|N|11|11|1937|LUXEMBOURG||Robert.Reid@4vc.edu|2452337| +1569|AAAAAAAABCGAAAAA|856212|4239|13866|2450214|2450184|Mr.|John|Carnes|Y|9|2|1979|BAHAMAS||John.Carnes@CSFA44r.com|2452327| +1570|AAAAAAAACCGAAAAA|1808501|2399|20344|2450240|2450210|Dr.|Ernest|Grace|N|11|10|1957|UNITED STATES||Ernest.Grace@9BygMa5Y.edu|2452449| +1571|AAAAAAAADCGAAAAA|1770884|2415|40346|2449075|2449045|Dr.|Justin|Woods|Y|3|4|1965|CAPE VERDE||Justin.Woods@3VJv.com|2452587| +1572|AAAAAAAAECGAAAAA|730757|5407|28104|2451328|2451298|Miss|Geraldine|Dixon|Y|5|1|1954|ARGENTINA||Geraldine.Dixon@yX7981.edu|2452305| +1573|AAAAAAAAFCGAAAAA|535376|6548|39320|2450895|2450865|Dr.|Betty|Carlin|N|30|10|1939|SURINAME||Betty.Carlin@7pnJc4.com|2452438| +1574|AAAAAAAAGCGAAAAA|1616481|457|31680|2450476|2450446|Ms.|Jennifer|Turnbull|Y|27|5|1953|ZAMBIA||Jennifer.Turnbull@mH0Ic.edu|2452405| +1575|AAAAAAAAHCGAAAAA|1426593|4336|7632|2450524|2450494|Mr.|Bruce|Hernandez|Y|29|10|1966|SURINAME||Bruce.Hernandez@cr.edu|2452405| +1576|AAAAAAAAICGAAAAA|139143|7160|23977|2449148|2449118|Miss|Tammy|Buchanan|N|16|2|1929|JERSEY||Tammy.Buchanan@OErhNU34y.edu|2452594| +1577|AAAAAAAAJCGAAAAA|201875|4979|37330|2451500|2451470|Mrs.|Lily|Ricks|Y|17|12|1924|LITHUANIA||Lily.Ricks@ITG96sbIJgyf.com|2452304| +1578|AAAAAAAAKCGAAAAA|1778186|3965|3344|2449371|2449341|Mrs.|Anthony|Maurer|Y|17|8|1990|LITHUANIA||Anthony.Maurer@NfM6GBx2KVRfzZd.org|2452357| +1579|AAAAAAAALCGAAAAA|755807|6219|19031|2451607|2451577|Mr.|Brian|Dean|N|24|9|1939|BELARUS||Brian.Dean@0j.edu|2452435| +1580|AAAAAAAAMCGAAAAA|1066038|3202|23519|2450848|2450818|Ms.|Rachel|Johnson|Y|8|3|1925|SAN MARINO||Rachel.Johnson@cDMqF5.edu|2452510| +1581|AAAAAAAANCGAAAAA|1231951|3830|1309|2451311||||Mendez|N|||||||2452568| +1582|AAAAAAAAOCGAAAAA|1476092|4095|32498|2450758|2450728|Sir|Glenn|Ambrose|N|11|4|1958|KYRGYZSTAN||Glenn.Ambrose@Ff4rah28Jk.com|2452469| +1583|AAAAAAAAPCGAAAAA|84655|1520|26785|2452178|2452148|Ms.|Kimberli|Hall|Y|5|4|1954|HONG KONG||Kimberli.Hall@L6D35uiSkPlrOQS.org|2452612| +1584|AAAAAAAAADGAAAAA|456714|4165|18416|2450153|2450123|Mrs.|Martha|Howell|Y|2|1|1982|QATAR||Martha.Howell@LvIBjqxVZ.edu|2452556| +1585|AAAAAAAABDGAAAAA|1283829|1898|47567|2451264|2451234|Mrs.|Stephanie|Zamora|N|14|10|1953|MAURITANIA||Stephanie.Zamora@8dQ.com|2452291| +1586|AAAAAAAACDGAAAAA|417203|181|39159|2451010|2450980|Mrs.|Margaret|Wisniewski|N|26|1|1933|UZBEKISTAN||Margaret.Wisniewski@Nn49YetAUZPc9.com|2452453| +1587|AAAAAAAADDGAAAAA|736851|3501|27704|2450448|2450418|Mrs.|Brooke|Johnson|N|24|10|1948|IRELAND||Brooke.Johnson@LrTVk.org|2452464| +1588|AAAAAAAAEDGAAAAA|70820|4703|31786|2451243|2451213|Miss|Marie|Kinney|Y|21|7|1961|FIJI||Marie.Kinney@rt.org|2452465| +1589|AAAAAAAAFDGAAAAA|1643295|6244|22440|2450635|2450605|Miss|Judith|Valentine|Y|5|3|1962|CAYMAN ISLANDS||Judith.Valentine@ieBbv.com|2452462| +1590|AAAAAAAAGDGAAAAA|1546211|4803|46148|2449051|2449021|Miss|April|Tuttle|N|13|7|1937|DJIBOUTI||April.Tuttle@VIkyYMA9d3yyBlPC.edu|2452283| +1591|AAAAAAAAHDGAAAAA|1076995|6789|20136|2452502|2452472|Mr.|Charlie|Fernandez|Y|19|5|1968|AFGHANISTAN||Charlie.Fernandez@zfaCooi94s.com|2452410| +1592|AAAAAAAAIDGAAAAA|936360|5161|41115|2449627|2449597|Dr.|Mercedes|Miller|Y|12|7|1973|HUNGARY||Mercedes.Miller@xdb.edu|2452429| +1593|AAAAAAAAJDGAAAAA|1608439|2309|38626|2451947|2451917|Dr.|Steven|Hull|Y|19|12|1975|JAPAN||Steven.Hull@l.edu|2452568| +1594|AAAAAAAAKDGAAAAA|965790|4315|20087|2449761|2449731|Dr.|Salvatore|Powell|N|29|8|1941|VENEZUELA||Salvatore.Powell@TQDzulMtAo.com|2452620| +1595|AAAAAAAALDGAAAAA|1580536|5257|5225|2451302|2451272|Mrs.|Sarah|Hall|Y|16|12|1946|SUDAN||Sarah.Hall@zc0.org|2452345| +1596|AAAAAAAAMDGAAAAA|1702750|2444|7291|2449097|2449067|Ms.|Rebecca|Prado|N|2|7|1991|IRAQ||Rebecca.Prado@N2mxQeyFi79V.com|2452297| +1597|AAAAAAAANDGAAAAA|466588|7117|44799|2451901|2451871|Sir|Shawn|Benner|N|1|2|1986|FRENCH GUIANA||Shawn.Benner@HmNla8LBEf1M.com|2452425| +1598|AAAAAAAAODGAAAAA|578807|5513|20266|2451085|2451055|Mr.|Scott|Miller|N|26|3|1984|QATAR||Scott.Miller@x5YTNTRJ1pl5U4l.edu|2452513| +1599|AAAAAAAAPDGAAAAA|1683843|4496|16349|2451315|2451285|Dr.|William|Rosado|Y|8|2|1927|TURKEY||William.Rosado@qvu0Eke.com|2452313| +1600|AAAAAAAAAEGAAAAA|860587|2702|23570|2450461|2450431|Miss|Heidi|Rodriguez|N|28|9|1935|KAZAKHSTAN||Heidi.Rodriguez@HXNZbLnC.com|2452438| +1601|AAAAAAAABEGAAAAA|1377037|3223|20162|2450152|2450122|Dr.|Joyce|Garris|N|13|3|1968|JERSEY||Joyce.Garris@yePrQ.org|2452420| +1602|AAAAAAAACEGAAAAA|159811|5257|34303|2450459|2450429|Mrs.|Rebecca|Granados|Y|13|3|1955|GAMBIA||Rebecca.Granados@GO6cRtP.edu|2452329| +1603|AAAAAAAADEGAAAAA|1456858|6479|17700|2451878|2451848|Dr.|Jennifer|Freed|N|25|9|1936|NIUE||Jennifer.Freed@5gnASj1iCX.edu|2452472| +1604|AAAAAAAAEEGAAAAA|893099|1575|37251|2450741|2450711|Ms.|Rosemarie|Hernandez|N|15|7|1980|AUSTRALIA||Rosemarie.Hernandez@ebP.com|2452524| +1605|AAAAAAAAFEGAAAAA|||39972||||||Y||1|1932|||Ricardo.Glaze@U9V1OCGTjb4DN.org|| +1606|AAAAAAAAGEGAAAAA|1525863|5091|15472|2451472|2451442|Sir|William|White|Y|26|10|1988|SAINT LUCIA||William.White@SKCLCGMzzDtRKD.edu|2452413| +1607|AAAAAAAAHEGAAAAA|266663|520|46824|2451730|2451700|Dr.|Julio|Shackelford|Y|4|9|1948|KIRIBATI||Julio.Shackelford@feV.com|2452549| +1608|AAAAAAAAIEGAAAAA|996749|2249|21702|2449420|2449390|Mr.|Matthew|Rice|N|12|1|1976|BAHAMAS||Matthew.Rice@ErUNtXL5.org|2452554| +1609|AAAAAAAAJEGAAAAA|1781107|3725|41543|2450136|2450106|Dr.|Lisa|Simmons|Y|14|3|1956|TUVALU||Lisa.Simmons@Rz3OGMGm9OtBdq.edu|2452546| +1610|AAAAAAAAKEGAAAAA|378036|6201|13966|2452093|2452063|Mr.|Edward|Velez|N|26|7|1924|POLAND||Edward.Velez@qjLDx1iGKF.edu|2452604| +1611|AAAAAAAALEGAAAAA|485773|6442|1864|2449968|2449938|Miss|Gloria|Hines|Y|9|10|1954|MALDIVES||Gloria.Hines@nv7.com|2452401| +1612|AAAAAAAAMEGAAAAA|214901|3689|32877|2450332|2450302|Mr.|Robert|Shaffer|N|4|3|1952|POLAND||Robert.Shaffer@6FIigmlXyN.com|2452605| +1613|AAAAAAAANEGAAAAA|1237028|4107|13586|2451222|2451192|Ms.|Kimberly|Cross|N|26|6|1943|ALAND ISLANDS||Kimberly.Cross@pxugX5JTSh.org|2452545| +1614|AAAAAAAAOEGAAAAA|336541|6828|4668|2450003|2449973|Dr.|Wesley|Hale|Y|6|6|1939|CAYMAN ISLANDS||Wesley.Hale@YMxbc0m0Pt.org|2452448| +1615|AAAAAAAAPEGAAAAA|981162|6737|27959|2451050|2451020|Mr.|Matthew|Richardson|Y|30|4|1978|SAN MARINO||Matthew.Richardson@6TUGFZJeND2SK.com|2452494| +1616|AAAAAAAAAFGAAAAA|749475|4441|39360|2450209|2450179|Mr.|Luis|Green|N|4|11|1969|SERBIA||Luis.Green@mPoPQEG1OGs4nz.org|2452600| +1617|AAAAAAAABFGAAAAA|734229|4062|39495|2450025|2449995|Mr.|Alfonso|Chapman|Y|13|9|1983|PERU||Alfonso.Chapman@SX7.edu|2452455| +1618|AAAAAAAACFGAAAAA|668570|461|49905|2450269|2450239|Sir|Justin|Erickson|N|7|7|1977|SYRIAN ARAB REPUBLIC||Justin.Erickson@UgACaYkl7TifQM34.org|2452523| +1619|AAAAAAAADFGAAAAA|293367|6826|5932|2451169|2451139|Dr.|Dorothy|Abrams|N|6|2|1952|INDONESIA||Dorothy.Abrams@u.edu|2452530| +1620|AAAAAAAAEFGAAAAA|1518003|6092|29381|2452612|2452582|Sir|Leonardo|Williamson|N|12|3|1971|CAMEROON||Leonardo.Williamson@z87ZDFf4X0gq.edu|2452324| +1621|AAAAAAAAFFGAAAAA|1061315|2523|8531|2450307|2450277|Miss|Rosemarie|Knapp|Y|4|11|1977|SWITZERLAND||Rosemarie.Knapp@oA4.com|2452558| +1622|AAAAAAAAGFGAAAAA|274419|3690|42514|2450427|2450397|Dr.|Jaime|Mason|N|27|9|1944|AFGHANISTAN||Jaime.Mason@4CDui99lj.org|2452526| +1623|AAAAAAAAHFGAAAAA|562438|70|19702|2451620|2451590|Dr.|Amanda|Sanchez|N|14|8|1960|ANTARCTICA||Amanda.Sanchez@pD.org|2452596| +1624|AAAAAAAAIFGAAAAA|459716|1759|39755|2450037|2450007|Dr.|Bruce|Foster|N|18|5|1950|BELGIUM||Bruce.Foster@bxr.edu|2452470| +1625|AAAAAAAAJFGAAAAA|1099768|6124|26880|2452391|2452361|Miss|Yvonne|Guthrie|Y|12|9|1974|NORWAY||Yvonne.Guthrie@49.org|2452541| +1626|AAAAAAAAKFGAAAAA|237568|6656|3791|2451784|2451754|Ms.|Stephanie|Brill|N|1|11|1979|KIRIBATI||Stephanie.Brill@0jd8IJJfn6aAlRrv.org|2452337| +1627|AAAAAAAALFGAAAAA|47353|969|20185|2451916|2451886|Sir|James|Martinez|Y|18|3|1988|PUERTO RICO||James.Martinez@AbG.org|2452409| +1628|AAAAAAAAMFGAAAAA|884536|5785|45834|2449955|2449925|Ms.|Dorothy|Jackson|N|2|6|1964|TAJIKISTAN||Dorothy.Jackson@7AtXjz.com|2452483| +1629|AAAAAAAANFGAAAAA|||31323|2452630|2452600||||N||||CHILE||Kevin.Clough@vRyos.org|| +1630|AAAAAAAAOFGAAAAA|1547423|4035|27800|2451043|2451013|Ms.|Jennifer|Stricklin|N|7|9|1951|JAPAN||Jennifer.Stricklin@5FhUD0ssy2Sf1508.com|2452351| +1631|AAAAAAAAPFGAAAAA|328391|2860|6082|2450269|2450239|Mrs.|Margaret|Romero|N|29|1|1964|ANTIGUA AND BARBUDA||Margaret.Romero@ojixqXeInr6BI7e9E.com|2452625| +1632|AAAAAAAAAGGAAAAA|979452|421|543|2452266|2452236|Mrs.|Albert|Horton|N|14|11|1947|ARUBA||Albert.Horton@9TFkBtrgftxO1.edu|2452382| +1633|AAAAAAAABGGAAAAA|942894|1972|8250|2450096|2450066|Mrs.|Brandi|Gallagher|Y|20|10|1958|SAINT HELENA||Brandi.Gallagher@p.com|2452509| +1634|AAAAAAAACGGAAAAA|152267|6761|49655|2452487|2452457|Mrs.|Diann|Mchugh|Y|9|5|1983|MONACO||Diann.Mchugh@cZr1rRYFqpvQ1ZUqQv.edu|2452587| +1635|AAAAAAAADGGAAAAA|1818209|2029|49896|2451825|2451795|Mr.|Richard|Beltran|Y|23|9|1965|COMOROS||Richard.Beltran@1Vj7yovbv.edu|2452619| +1636|AAAAAAAAEGGAAAAA|712162|3048|44687|2450973|2450943|Miss|Pamela|Hood|N|21|11|1972|COSTA RICA||Pamela.Hood@y.org|2452370| +1637|AAAAAAAAFGGAAAAA|312513|7046|12779|2451517|2451487|Miss|Doris|Dawson|N|30|12|1958|FRANCE||Doris.Dawson@FZmPJpkEXCz.edu|2452457| +1638|AAAAAAAAGGGAAAAA|792314|4007|12947|2452651|2452621|Miss|Beatrice|Glover|Y|26|11|1939|LIECHTENSTEIN||Beatrice.Glover@DxllCDufPG0I.edu|2452314| +1639|AAAAAAAAHGGAAAAA|1514405|5022|33686|2451148|2451118|Mr.|Donald|Jenkins|Y|1|11|1985|NEPAL||Donald.Jenkins@VBUTRf50QzllL.com|2452559| +1640|AAAAAAAAIGGAAAAA|295453|1849|23798|2450003|2449973|Mrs.|Rosanna|Davis|N|14|11|1956|HONDURAS||Rosanna.Davis@KHIVad.com|2452299| +1641|AAAAAAAAJGGAAAAA|989079|5232|13338|2452322|2452292|Mr.|George|Gold|N|25|11|1985|BRUNEI DARUSSALAM||George.Gold@JNkl.edu|2452624| +1642|AAAAAAAAKGGAAAAA|196336|3385|21234|2451621|2451591|Sir|Stephen|Orr|N|5|7|1939|UNITED KINGDOM||Stephen.Orr@8yENbrKnoy8tR.edu|2452618| +1643|AAAAAAAALGGAAAAA|984302|3654|3006|2449267|2449237|Miss|Juanita|Lynch|N|17|5|1939|BAHAMAS||Juanita.Lynch@1evVFBvJR7AJpZN8UM4.org|2452510| +1644|AAAAAAAAMGGAAAAA|1706487|557|26627|2451886|2451856|Mr.|Kyle|Hernandez|Y|1|2|1939|MEXICO||Kyle.Hernandez@395EaRonnnAXg.edu|2452436| +1645|AAAAAAAANGGAAAAA|1048194|4595|42549|2452492|2452462|Mr.|Christopher|Holcomb|N|22|2|1932|NIUE||Christopher.Holcomb@kHq.com|2452645| +1646|AAAAAAAAOGGAAAAA|1022855|4330|9852|2450839|2450809|Dr.|Steven|Bronson|Y|18|5|1951|ALAND ISLANDS||Steven.Bronson@XtQVf.org|2452304| +1647|AAAAAAAAPGGAAAAA|1721396|6666|6336|2450134|2450104|Ms.|Susan|Watkins|N|16|12|1951|MONACO||Susan.Watkins@Ri2LyU8sOuu.org|2452423| +1648|AAAAAAAAAHGAAAAA|759510|841|4172|2450387|2450357|Mrs.|Susan|Bradford|Y|27|4|1985|CZECH REPUBLIC||Susan.Bradford@9POuBI6r8ef3F9Inyd.com|2452543| +1649|AAAAAAAABHGAAAAA|35170|5136|5502|2451532|2451502|Dr.|Leola|Fender|Y|2|6|1987|GHANA||Leola.Fender@17MGKIPYEfPRql.org|2452478| +1650|AAAAAAAACHGAAAAA|1347218|3361|28704|2450895|2450865|Dr.|Larry|Harrison|Y|18|3|1950|BAHAMAS||Larry.Harrison@aNRZ.com|2452372| +1651|AAAAAAAADHGAAAAA|1287311|808|13458|2450350|2450320|Sir|Jason|Mobley|Y|17|4|1935|GRENADA||Jason.Mobley@ck8hyl0ZHeCXG.edu|2452438| +1652|AAAAAAAAEHGAAAAA|35719|1694|49316|2451461|2451431|Dr.|James|Shepard|Y|30|8|1983|PUERTO RICO||James.Shepard@eu.edu|2452544| +1653|AAAAAAAAFHGAAAAA|383622|3911|49000|2451793|2451763||Edward|Fischer||10||1942||||| +1654|AAAAAAAAGHGAAAAA|1070263|5977|18891|2450126|2450096|Dr.|Evan|Peeples|Y|4|3|1953|VIRGIN ISLANDS, U.S.||Evan.Peeples@N.edu|2452339| +1655|AAAAAAAAHHGAAAAA|231314|6225|35089|2450753|2450723|Mrs.|Nicole|Daniel|N|4|7|1934|SAINT LUCIA||Nicole.Daniel@PtM3BBr.edu|2452598| +1656|AAAAAAAAIHGAAAAA|1135937|3389|46441|2452090|2452060|Dr.|Jack|Partin|N|30|4|1972|IRELAND||Jack.Partin@lla4lvrTzRduChK7Dj.com|2452447| +1657|AAAAAAAAJHGAAAAA|33561|264|48967|2449153|2449123|Mrs.|Myrna|Martinez|N|3|5|1992|JORDAN||Myrna.Martinez@SGf29gKeYpebjp.com|2452380| +1658|AAAAAAAAKHGAAAAA|1692806||7367|||Mr.|||N|23|10|1964|||Peter.Harris@TyOyY6r3KL.com|| +1659|AAAAAAAALHGAAAAA|1226544|744|46897|2452673|2452643|Mr.|Roderick|Skelton|N|30|1|1972|UGANDA||Roderick.Skelton@eBvKiO.com|2452374| +1660|AAAAAAAAMHGAAAAA|1911521|7108|43448|2450541|2450511|Mrs.|Edna|Mahoney|Y|4|11|1929|NEPAL||Edna.Mahoney@ctAqifrerLJrm.org|2452593| +1661|AAAAAAAANHGAAAAA|585700|2133|39766|2450120|2450090|Dr.|Susana|Turner|Y|15|10|1924|EGYPT||Susana.Turner@7qARXg.com|2452512| +1662|AAAAAAAAOHGAAAAA|1313716|81|34799|2451608|2451578|Sir|Elmer|Ferguson|N|17|8|1954|SAINT LUCIA||Elmer.Ferguson@R.edu|2452417| +1663|AAAAAAAAPHGAAAAA|212420|5802|44337|2449091|2449061|Mr.|Gregory|Morris|N|6|8|1955|MALAWI||Gregory.Morris@rGlHEDcz.com|2452418| +1664|AAAAAAAAAIGAAAAA|1402912|4081|2097|2451714|2451684|Ms.|Jacqueline|Belcher|Y|19|12|1943|SENEGAL||Jacqueline.Belcher@rp8M.com|2452578| +1665|AAAAAAAABIGAAAAA|630754|1078|19802|2449808|2449778|Ms.|Eileen|Weber|N|15|5|1944|TUNISIA||Eileen.Weber@7gBgkA8.edu|2452443| +1666|AAAAAAAACIGAAAAA|1099752|15|35478|2451372|2451342|Dr.|Walter|Smith|N|10|11|1924|SEYCHELLES||Walter.Smith@ZosbNB1GrOO1Uf.edu|2452373| +1667|AAAAAAAADIGAAAAA|1764665|6949|29680|2451579|2451549|Dr.|Joseph|Laughlin|N|24|8|1961|URUGUAY||Joseph.Laughlin@rXmb2JhlAFnVqyl4m4.edu|2452551| +1668|AAAAAAAAEIGAAAAA|382919|3868|24017|2450813|2450783|Ms.|Dorothy|Greene|Y|16|12|1988|YEMEN||Dorothy.Greene@xaN8tVjjz.com|2452518| +1669|AAAAAAAAFIGAAAAA|1733909|2642|44175|2452511|2452481|Dr.|Jason|Kelly|Y|22|4|1979|SYRIAN ARAB REPUBLIC||Jason.Kelly@k1vTDp32frPKOl.org|2452556| +1670|AAAAAAAAGIGAAAAA|1032027|6155|38330|2452397|2452367|Sir|Robert|Harden|Y|28|7|1956|TOKELAU||Robert.Harden@GFXqIDxXtmQpnH.com|2452597| +1671|AAAAAAAAHIGAAAAA|187622|5316|39057||||Craig|Brown|Y|15|||TRINIDAD AND TOBAGO||Craig.Brown@T71kNgX1chul.edu|2452428| +1672|AAAAAAAAIIGAAAAA|936756|5588|4373|2451470|2451440|Dr.|Timothy|Hutton|N|2|10|1963|C�TE D'IVOIRE||Timothy.Hutton@xCSFqfb7hMf.edu|2452531| +1673|AAAAAAAAJIGAAAAA|1117762|5431|16700|2450294|2450264|Miss|Sally|Beasley|N|6|9|1948|ISRAEL||Sally.Beasley@nf7DKJbTDPU0.edu|2452636| +1674|AAAAAAAAKIGAAAAA|740234||21172||2449334|Sir|||Y|21|4||ZIMBABWE||Damon.Davis@YkVNt.com|2452315| +1675|AAAAAAAALIGAAAAA|39433|7164|14095|2451376|2451346|Mr.|Herbert|Ross|N|18|8|1978|BRAZIL||Herbert.Ross@ugDGpyFuPbShdX.org|2452515| +1676|AAAAAAAAMIGAAAAA|17525|5090|38208|2450383|2450353|Ms.|Estelle|Duong|N|29|9|1926|COSTA RICA||Estelle.Duong@3M504oRr5igZdZoH.com|2452524| +1677|AAAAAAAANIGAAAAA|255500|3009|16918|2451988|2451958|Mrs.|Keri|Marshall|N|27|9|1946|SAN MARINO||Keri.Marshall@44d3.org|2452642| +1678|AAAAAAAAOIGAAAAA|1167437|7196|45755|2451325|||Shanda|||12|12|1928|WALLIS AND FUTUNA|||| +1679|AAAAAAAAPIGAAAAA|1106568|2053|47091|2449499|2449469|Dr.|John|Knutson|N|20|1|1982|TOGO||John.Knutson@urZHEtfifQvh7HX.com|2452530| +1680|AAAAAAAAAJGAAAAA|||24069||||||N||1|||||| +1681|AAAAAAAABJGAAAAA|1691510|222|7334|2451926|2451896|Dr.|Deborah|Dodson|N|30|5|1934|ALGERIA||Deborah.Dodson@9EfBFjtx1FB2BZV.org|2452389| +1682|AAAAAAAACJGAAAAA|1850576|5606|34236||2451235||Sophia|Phillips|N|5|3||BELGIUM|||| +1683|AAAAAAAADJGAAAAA|55930|2336|27189|2452363|2452333|Dr.|Chester|Carper|N|29|2|1936|KENYA||Chester.Carper@DKbtJffi.org|2452346| +1684|AAAAAAAAEJGAAAAA|588069|445|35221|2452145|2452115|Sir|Don|Russell|N|28|5|1949|SUDAN||Don.Russell@Q2crO.com|2452462| +1685|AAAAAAAAFJGAAAAA|1020833|3267|16901|2451072|2451042|Dr.|Diane|Hudspeth|N|18|1|1987|BOUVET ISLAND||Diane.Hudspeth@0U1nzk8gOK.edu|2452647| +1686|AAAAAAAAGJGAAAAA|461330|4746|42915|2451965|2451935|Mr.|John|Hand|N|14|12|1933|NIGER||John.Hand@D22A.com|2452573| +1687|AAAAAAAAHJGAAAAA|1151564|3560|38403|2452412|2452382|Dr.|John|Battle|N|9|11|1955|TURKEY||John.Battle@7YBR.com|2452344| +1688|AAAAAAAAIJGAAAAA|194313|3909|45905|2451530|2451500||||Y|13|8|1931|RUSSIAN FEDERATION|||2452330| +1689|AAAAAAAAJJGAAAAA|1334539|2991|643|2449071|2449041|Dr.|Josefa|Hofmann|Y|12|11|1927|LUXEMBOURG||Josefa.Hofmann@MVubR70YOvp.org|2452556| +1690|AAAAAAAAKJGAAAAA|509758|5709|40060|2449208|2449178|Dr.|Brenda|Thompson|N|16|2|1968|NEW ZEALAND||Brenda.Thompson@5atz3R.edu|2452448| +1691|AAAAAAAALJGAAAAA|1260300|3986|40072|2449686|2449656|Dr.|Stephan|Chavez|Y|9|9|1960|GUINEA-BISSAU||Stephan.Chavez@GoU6jKNEJfKMQeu3hqV.org|2452615| +1692|AAAAAAAAMJGAAAAA|525833|6020|3760|2449448|2449418|Sir|David|Knight|Y|31|12|1945|ISRAEL||David.Knight@ci1cGpFr93le8gMBNz.edu|2452516| +1693|AAAAAAAANJGAAAAA|1143121|6764|49207|2451802|2451772|Miss|Priscilla|Williams|N|6|9|1934|NICARAGUA||Priscilla.Williams@GCVEzjat.org|2452573| +1694|AAAAAAAAOJGAAAAA|1662871|5371|26318|2451721|2451691|Miss|Renate|Duran|Y|17|5|1966|SOMALIA||Renate.Duran@4BRQh1M3ilHF.org|2452430| +1695|AAAAAAAAPJGAAAAA|783161|5656|31351|2451344|2451314|Dr.|Ronald|Evans|Y|18|6|1942|NIGERIA||Ronald.Evans@r5lVMIBCDq.com|2452475| +1696|AAAAAAAAAKGAAAAA|1325323|5883|6890|2452124|2452094|Sir|William|Smith|Y|12|6|1974|UNITED KINGDOM||William.Smith@3zrBnCai8UaVUcHFx.org|2452560| +1697|AAAAAAAABKGAAAAA|387038|5196|24154|2451110|2451080|Sir|Michael|Van|Y|17|9|1932|BELGIUM||Michael.Van@pMjDJsvOPYE.edu|2452535| +1698|AAAAAAAACKGAAAAA|1531918|1904|29090|2451681|2451651|Mrs.|Lisa|Lambert|N|8|11|1973|BOUVET ISLAND||Lisa.Lambert@IOuoUN4Nq00tpt.com|2452504| +1699|AAAAAAAADKGAAAAA|1322072|7031|29990|2450807|2450777|Ms.|Sherry|Johnson|Y|14|2|1943|GUINEA-BISSAU||Sherry.Johnson@E4HbiBfcgVXL1R0A8DvI.com|2452368| +1700|AAAAAAAAEKGAAAAA|160587|5236|17532|2452320|2452290|Mrs.|Helen|Mason|Y|29|10|1960|SAINT HELENA||Helen.Mason@HLYEKlhqanq2tUZ65.com|2452564| +1701|AAAAAAAAFKGAAAAA|714120|1089|42105|2451732|2451702|Dr.|Juan|Freeman|N|9|6|1955|MONTENEGRO||Juan.Freeman@mLFA3tpFKg4.org|2452389| +1702|AAAAAAAAGKGAAAAA|651186|3006|39579|2449190|2449160|Sir|Matthew|Metzler|N|25|6|1975|MONTSERRAT||Matthew.Metzler@qZy4zI.edu|2452435| +1703|AAAAAAAAHKGAAAAA|148724|4052|39733|2451614|2451584|Mr.|Brian|Smith|N|22|9|1968|BELIZE||Brian.Smith@D4YaaO1E.com|2452477| +1704|AAAAAAAAIKGAAAAA|933342|5403|28217|2450441|2450411|Dr.|Keith|Bailey|N|8|12|1925|MALTA||Keith.Bailey@eX3q5XrnnE3s.edu|2452489| +1705|AAAAAAAAJKGAAAAA|322193|3029|43018|2450203|2450173|Dr.|Helen|Garcia|N|11|8|1940|MADAGASCAR||Helen.Garcia@ABQ4u4IgHrl6UGrgu.org|2452646| +1706|AAAAAAAAKKGAAAAA|976103|5072|35594|2451583|2451553|Ms.|Joyce|Moore|Y|23|1|1971|MOZAMBIQUE||Joyce.Moore@xhLy7Jb.edu|2452537| +1707|AAAAAAAALKGAAAAA|693765|1742|28919|2451279|2451249|Mrs.|Dora|Gresham|N|19|7|1978|SAN MARINO||Dora.Gresham@ENxYX6VYRkSi.org|2452550| +1708|AAAAAAAAMKGAAAAA|1236377|3135|40693|2451646|2451616|Mrs.|Lori|Leal|N|20|4|1964|MALDIVES||Lori.Leal@S4NUVkYLjh.org|2452433| +1709|AAAAAAAANKGAAAAA|348513|2112|1358|2451531|2451501|Mrs.|Mia|Yoder|Y|30|3|1962|HONDURAS||Mia.Yoder@SBRIqIcr9ZiHZxNn.org|2452571| +1710|AAAAAAAAOKGAAAAA|1198329|6782|39588|2451666|2451636|Mrs.|Doris|Tilley|N|8|4|1970|BENIN||Doris.Tilley@X.org|2452305| +1711|AAAAAAAAPKGAAAAA|809422|806|26590|2451461|2451431|Mrs.|Leona|Looney|Y|8|11|1932|MARSHALL ISLANDS||Leona.Looney@7qYf1AVgtqay8S95.org|2452464| +1712|AAAAAAAAALGAAAAA|1393287|5170|26653|2449103|2449073|Mr.|Frank|Brooks|Y|5|7|1929|TUNISIA||Frank.Brooks@j8FJfl8A4j2k.org|2452469| +1713|AAAAAAAABLGAAAAA|569084|4547|25657|2449670|2449640|Mr.|Daniel|Thomas|Y|9|11|1981|NEPAL||Daniel.Thomas@zNf7TY2xveKy.org|2452405| +1714|AAAAAAAACLGAAAAA|378748|4264|31380|2451929|2451899|Mrs.|Albert|Wilkerson|N|20|3|1957|TUNISIA||Albert.Wilkerson@kv7c2.edu|2452322| +1715|AAAAAAAADLGAAAAA|655356|5342|34575|2452189|2452159|Dr.|Timothy|Moreno|Y|9|11|1978|PHILIPPINES||Timothy.Moreno@DIk1BKdD3ZfxOB.edu|2452581| +1716|AAAAAAAAELGAAAAA|1869858|5134|35257|2450521|2450491|Mr.|Mack|Garvey|N|22|4|1971|MALAYSIA||Mack.Garvey@MBgxAsU5DtFq.edu|2452576| +1717|AAAAAAAAFLGAAAAA|1173845|1159|29273|2449756|2449726|Mrs.|Dawn|Jones|N|6|3|1963|MALTA||Dawn.Jones@SHKhbVStNFL.com|2452313| +1718|AAAAAAAAGLGAAAAA|347181|2418|25255|2451650|2451620|Sir|Matt|Silva|Y|27|7|1942|BERMUDA||Matt.Silva@26rGISUX.org|2452359| +1719|AAAAAAAAHLGAAAAA|1915184|5523|12262|2451029|2450999|Sir|Joesph|Reynolds|Y|20|2|1983|SOLOMON ISLANDS||Joesph.Reynolds@hv.edu|2452370| +1720|AAAAAAAAILGAAAAA|534070|6709|2901|2449320|2449290|Sir|Joshua|Howard|N|29|9|1940|MONACO||Joshua.Howard@YqcjlVhZVnsm.edu|2452647| +1721|AAAAAAAAJLGAAAAA|545192|1342|12292|2450672|2450642|Mr.|Paul|Voss|N|23|4|1926|BELARUS||Paul.Voss@yP.org|2452386| +1722|AAAAAAAAKLGAAAAA|84546|2880|414|2450509|2450479|Mr.|Jeffrey|Tharp|Y|11|3|1979|JAPAN||Jeffrey.Tharp@0vj.org|2452557| +1723|AAAAAAAALLGAAAAA|65133|4336|33092|2449661|2449631|Sir|Hank|Hastings|N|16|12|1963|BELARUS||Hank.Hastings@1vFlpslatk.edu|2452499| +1724|AAAAAAAAMLGAAAAA|452198|3300|35602|2451793||Mr.|Brad|Carlos||10|11||||Brad.Carlos@Evu.com|2452524| +1725|AAAAAAAANLGAAAAA|1012192|351|5310|2451753|2451723|Sir|Ron|Brown|Y|2|6|1990|VIRGIN ISLANDS, U.S.||Ron.Brown@9N.com|2452392| +1726|AAAAAAAAOLGAAAAA|1840630|957|21684|2451719|2451689|Dr.|Joann|Jameson|N|11|7|1956|MALI||Joann.Jameson@Te5GA0GIke.org|2452629| +1727|AAAAAAAAPLGAAAAA|1154332|701|31470|2451526|2451496|Mr.|John|Broome|Y|27|3|1935|GUYANA||John.Broome@Mmr1eFGdD5pNv4yoUrV.com|2452610| +1728|AAAAAAAAAMGAAAAA|592698|3403|32061|2449230|2449200|Mrs.|Lucinda|Jones|Y|1|5|1935|NEPAL||Lucinda.Jones@4AMrieaQJaTkmn5Y.com|2452497| +1729|AAAAAAAABMGAAAAA|1544675|2302|6729|2451929|2451899|Sir|Wilson|Ivey|N|29|5|1990|ISLE OF MAN||Wilson.Ivey@Z6sMI5KXqLEG.com|2452421| +1730|AAAAAAAACMGAAAAA|591437|3936|42882|2449573|2449543|Dr.|Marcy|Singleton|Y|11|6|1926|ARGENTINA||Marcy.Singleton@FKTLXLiUcFJj.org|2452373| +1731|AAAAAAAADMGAAAAA|34934|710|37686|2452275|2452245|Dr.|Louis|Burnette|N|7|8|1930|BURKINA FASO||Louis.Burnette@C.com|2452377| +1732|AAAAAAAAEMGAAAAA|1065409|58|27107|2450312|2450282|Dr.|Eric|Hernandez|Y|14|2|1930|NEW ZEALAND||Eric.Hernandez@bkx2xbKmjKtn.edu|2452433| +1733|AAAAAAAAFMGAAAAA|1115213|5600|29587|2449710|2449680|Mr.|Miguel|Carter|N|26|4|1985|IRELAND||Miguel.Carter@dKimu.org|2452398| +1734|AAAAAAAAGMGAAAAA|236991|1230|42716|2451021|2450991|Dr.|Thomas|Alexander|N|3|3|1971|MONTSERRAT||Thomas.Alexander@VNrSuOHPtdLquDS.org|2452346| +1735|AAAAAAAAHMGAAAAA|422931|6820|4309|2452232|2452202|Mr.|Brad|Larsen|N|2|5|1968|SAINT LUCIA||Brad.Larsen@Ivblc9xIv.org|2452535| +1736|AAAAAAAAIMGAAAAA|675487|174|30108|2451116|2451086|Mrs.|Lois|Rockwell|N|13|9|1973|SAMOA||Lois.Rockwell@AcKFKU5B4zeY0B.com|2452410| +1737|AAAAAAAAJMGAAAAA|394342|4846|18150|2452643|2452613|Ms.|Madie|Moreno|Y|24|5|1947|GREECE||Madie.Moreno@vM2Bm8AdLOp7.edu|2452624| +1738|AAAAAAAAKMGAAAAA|395856||6630|2450997|2450967||Katherine|Pennington|Y||8|||||2452594| +1739|AAAAAAAALMGAAAAA|1591707|3030|30353|2451171|2451141|Mr.|James|Townsend|Y|24|4|1928|MOROCCO||James.Townsend@Syv53gLoLMJKtxhC.edu|2452645| +1740|AAAAAAAAMMGAAAAA|1127954|1310|40266|2451036|2451006|Ms.|Rita|Pollard|Y|30|1|1965|KENYA||Rita.Pollard@XrzFrtllxpy.org|2452404| +1741|AAAAAAAANMGAAAAA|1692493|6091|12842|2449386|2449356|Dr.|Martin|Peterson|N|26|6|1934|MONTENEGRO||Martin.Peterson@JroNmLTFRbY7x.edu|2452515| +1742|AAAAAAAAOMGAAAAA|326458|1238|13571|2452561|2452531|Dr.|Robert|Richards|N|6|11|1988|JORDAN||Robert.Richards@IfCGn3e.org|2452361| +1743|AAAAAAAAPMGAAAAA|1464091|6278|9312|2451719|2451689|Dr.|Tyrone|Blake|N|17|5|1934|LEBANON||Tyrone.Blake@MCBmTXdzDZ6UHfHEe.com|2452630| +1744|AAAAAAAAANGAAAAA|551709|1063|25264|2450224|2450194|Ms.|Robyn|Lowe|Y|10|2|1956|TIMOR-LESTE||Robyn.Lowe@Z3.com|2452383| +1745|AAAAAAAABNGAAAAA|993496|6466|37883|2451183|2451153|Sir|Zack|Ness|N|10|10|1981|ARMENIA||Zack.Ness@9T3yJYF11tSB.com|2452352| +1746|AAAAAAAACNGAAAAA|1276475|506|27290|2449441|2449411|Mr.|James|Holley|N|23|6|1936|ARGENTINA||James.Holley@7KR.org|2452512| +1747|AAAAAAAADNGAAAAA|1081532|136|31187|2451580|2451550|Ms.|Tammie|Velasquez|Y|7|8|1977|URUGUAY||Tammie.Velasquez@0zZOoTIZBXLyR6.org|2452493| +1748|AAAAAAAAENGAAAAA|720191|5456|43168|2451298|2451268|Mr.|Christopher|Gardner|Y|2|12|1971|BENIN||Christopher.Gardner@O5sXNsghaRd6.org|2452534| +1749|AAAAAAAAFNGAAAAA|1811705|3846|30034|2450871|2450841|Mr.|Tyrone|Williams|Y|13|6|1990|CYPRUS||Tyrone.Williams@yH.com|2452586| +1750|AAAAAAAAGNGAAAAA|527053|6390|45727|2451073|2451043|Ms.|Alicia|Carter|Y|17|10|1944|RUSSIAN FEDERATION||Alicia.Carter@v1R.com|2452648| +1751|AAAAAAAAHNGAAAAA|115200|231|18909|2452236|2452206|Mrs.|Laura|Clements|Y|26|11|1945|LUXEMBOURG||Laura.Clements@C9hPqYyhR.org|2452429| +1752|AAAAAAAAINGAAAAA|127120|4913|28353|2450231|2450201|Ms.|April|Camp|Y|17|2|1932|COMOROS||April.Camp@Tj8.org|2452315| +1753|AAAAAAAAJNGAAAAA|1659881|58|35882|2452023|2451993|Mr.|Ralph|Hill|N|6|7|1939|NEW CALEDONIA||Ralph.Hill@1oHrpL36q17z1A.com|2452633| +1754|AAAAAAAAKNGAAAAA|1547852|2359|14466|2450080|2450050|Sir|Ian|Madsen|N|19|11|1984|BAHRAIN||Ian.Madsen@Q8DcYs2ZT.edu|2452302| +1755|AAAAAAAALNGAAAAA|1134852|910|7321|2450046|2450016|Mrs.|Linda|Day|N|12|2|1926|PALAU||Linda.Day@4CRZbtxf1BhECur6YE.org|2452555| +1756|AAAAAAAAMNGAAAAA|799362|3820|3431|2451507|2451477|Dr.|Daryl|Gaddis|Y|20|7|1948|ARGENTINA||Daryl.Gaddis@NsD6cGsGulC.com|2452373| +1757|AAAAAAAANNGAAAAA|||34779|2451457||Mrs.|Donna|||19||1946||||| +1758|AAAAAAAAONGAAAAA|1143968|6040|34512|2451731|2451701|Mrs.|Janna|Gunderson|Y|29|9|1970|LEBANON||Janna.Gunderson@AhzrzV.org|2452345| +1759|AAAAAAAAPNGAAAAA|1633012|2870|8281|2450317|2450287|Mrs.|Clara|Anderson|Y|3|7|1963|GRENADA||Clara.Anderson@YxVgo0T1JFNCD.com|2452494| +1760|AAAAAAAAAOGAAAAA|1005756|321|13733|2450532|2450502|Sir|Ramon|Lee|Y|21|1|1945|MONTENEGRO||Ramon.Lee@3FaSmA.edu|2452450| +1761|AAAAAAAABOGAAAAA|497603|2911|48572|2449870|2449840|Sir|Bradley|Brooks|Y|3|6|1961|TAJIKISTAN||Bradley.Brooks@3ruarYuJ4KEAdp.com|2452499| +1762|AAAAAAAACOGAAAAA|410035|1948|10214|2449305|2449275|Ms.|Ruby|Gonzalez|N|30|6|1966|SUDAN||Ruby.Gonzalez@nLm7dQ.com|2452482| +1763|AAAAAAAADOGAAAAA|898648|7118|32388|2452029|2451999|Dr.|Jeffrey|Turner|N|6|5|1937|PORTUGAL||Jeffrey.Turner@pLXq.com|2452558| +1764|AAAAAAAAEOGAAAAA|958003|1522|8641|2450654|2450624|Mr.|Kenneth|Hare|Y|30|11|1976|INDONESIA||Kenneth.Hare@My.edu|2452402| +1765|AAAAAAAAFOGAAAAA|1502853|1323|16593|2451164|2451134|Dr.|George|Shull|N|17|4|1931|JERSEY||George.Shull@cEB10AdNjI.edu|2452617| +1766|AAAAAAAAGOGAAAAA|555562|5124|48286|2451844|2451814|Dr.|Edward|Villasenor|N|23|6|1981|MARSHALL ISLANDS||Edward.Villasenor@VsRc3Bf38.edu|2452382| +1767|AAAAAAAAHOGAAAAA|789706|3923|32140|2450991|2450961|Sir|Richard|Mclaughlin|Y|3|12|1978|PAKISTAN||Richard.Mclaughlin@g9jqy8LK2rqM.org|2452426| +1768|AAAAAAAAIOGAAAAA|1774893|1303|48160|2450758|2450728|Mr.|Richard|Collins|N|24|10|1932|GUERNSEY||Richard.Collins@5zr8pG.edu|2452616| +1769|AAAAAAAAJOGAAAAA|604763|5843|13723|2449836|2449806|Dr.|Joseph|Gould|Y|20|5|1943|MYANMAR||Joseph.Gould@kSHsmS.edu|2452601| +1770|AAAAAAAAKOGAAAAA|694276|3335|13947|2451983|2451953|Dr.|Vanessa|Smith|Y|21|12|1958|PARAGUAY||Vanessa.Smith@pAc.edu|2452414| +1771|AAAAAAAALOGAAAAA|225810|5336|16109|2449535|2449505|Miss|Sylvia|Medina|Y|26|11|1946|NAMIBIA||Sylvia.Medina@rtclQ.org|2452444| +1772|AAAAAAAAMOGAAAAA|1069168|5947|37879|2451117|2451087|Mr.|Rick|Estes|N|14|5|1934|CAMBODIA||Rick.Estes@VHcRQKjfOyS3FIeb.com|2452355| +1773|AAAAAAAANOGAAAAA|231029|4736|18443|2449120|2449090|Mr.|William|Brown|Y|10|5|1982|SENEGAL||William.Brown@BGlTFJQpoo.edu|2452620| +1774|AAAAAAAAOOGAAAAA|1878595|6177|23359|2451806|2451776|Dr.|Lori|Watts|Y|28|8|1963|FRENCH POLYNESIA||Lori.Watts@b4rGT.com|2452532| +1775|AAAAAAAAPOGAAAAA|1108747|2528|33909|2451252|2451222|Mr.|Harold|Moore|Y|9|11|1925|NIGERIA||Harold.Moore@HPlbEtb73E.com|2452547| +1776|AAAAAAAAAPGAAAAA|1702337|3447|12957|2451505|2451475|Dr.|Nathaniel|Bunnell|N|11|12|1933|BAHAMAS||Nathaniel.Bunnell@QYVCUN8ux.com|2452286| +1777|AAAAAAAABPGAAAAA|1024995|5133|33271|2451684|2451654|Dr.|Alice|Huber|Y|10|3|1932|LIECHTENSTEIN||Alice.Huber@TR8qIEp6R1.org|2452596| +1778|AAAAAAAACPGAAAAA|336731|3959|38434|2449450|2449420|Dr.|Thomas|Hoag|Y|30|11|1968|UNITED KINGDOM||Thomas.Hoag@7QCCNG2ZSc9K0B.edu|2452580| +1779|AAAAAAAADPGAAAAA|672262|6455|43801|2451307|2451277|Dr.|Eric|Pennington|Y|26|5|1924|YEMEN||Eric.Pennington@UpdgSCJj7t.com|2452405| +1780|AAAAAAAAEPGAAAAA|1411106|6469|27879|2449269|2449239|Sir|Steven|Clarke|Y|20|12|1942|MAYOTTE||Steven.Clarke@HgjpDFCF3TgE.org|2452558| +1781|AAAAAAAAFPGAAAAA|1570294|4691|288|2451027|2450997|Dr.|Geraldine|Williams|N|9|4|1980|TONGA||Geraldine.Williams@Iq0xUy6LAv8L.org|2452514| +1782|AAAAAAAAGPGAAAAA|568698|3478|44258|2450590|2450560|Sir|Benny|Welch|N|13|10|1942|GHANA||Benny.Welch@VxcRns.com|2452443| +1783|AAAAAAAAHPGAAAAA|627165|3005|45598|2451241|2451211|Dr.|Joseph|Davis|Y|24|3|1992|ANTARCTICA||Joseph.Davis@SR63HGbF6pp.edu|2452393| +1784|AAAAAAAAIPGAAAAA|167837|2650|24616|2449056|2449026|Mrs.|Stephanie|Scott|Y|6|1|1935|CAYMAN ISLANDS||Stephanie.Scott@LgcN.org|2452445| +1785|AAAAAAAAJPGAAAAA|1433646||12719|2449799|2449769|Dr.|Roy||||2||||Roy.Jones@F.edu|2452451| +1786|AAAAAAAAKPGAAAAA|875441|1602|22163|2449975|2449945|Mrs.|Twila|Mitchell|N|2|2|1951|DJIBOUTI||Twila.Mitchell@k07MkOG1dkI.com|2452475| +1787|AAAAAAAALPGAAAAA|670931|371|33058|2450853|2450823|Sir|Craig|Jackson|N|9|3|1925|SURINAME||Craig.Jackson@iz9AI8LRL3By3LT.edu|2452295| +1788|AAAAAAAAMPGAAAAA|1608341|775|41209|2450543|2450513|Sir|Jay|Rodriguez|N|20|2|1963|SAUDI ARABIA||Jay.Rodriguez@Hr5x4ouEdmX2u.edu|2452394| +1789|AAAAAAAANPGAAAAA|823377|2982|1434|2449130|2449100|Mr.|William|Loera|N|15|3|1955|MOROCCO||William.Loera@r966A2xKOz9.org|2452517| +1790|AAAAAAAAOPGAAAAA|65065|5701|20784|2449294|2449264|Sir|Douglas|Farmer|Y|5|7|1964|FRENCH GUIANA||Douglas.Farmer@ugZt2BudJQ.org|2452579| +1791|AAAAAAAAPPGAAAAA|1607762|3489|34816|2451180|2451150|Miss|Janet|Lively|N|13|3|1957|HONG KONG||Janet.Lively@CVs6fvJg8q1.com|2452577| +1792|AAAAAAAAAAHAAAAA|267531|4135|11986|2450266|2450236|Sir|Tim|Colon|Y|3|4|1982|AZERBAIJAN||Tim.Colon@toOgHd.org|2452555| +1793|AAAAAAAABAHAAAAA|||21073|||Mr.|Dennis|Price|N||2|1981||||| +1794|AAAAAAAACAHAAAAA|82816|3782|23548|2451222|2451192|Mrs.|Judy|Johnson|N|3|3|1939|THAILAND||Judy.Johnson@bbKq.org|2452462| +1795|AAAAAAAADAHAAAAA|1888892|592|34692|2449723|2449693|Sir|Brian|Sloan|Y|3|4|1929|CZECH REPUBLIC||Brian.Sloan@gseKqhmAZQVVfC.com|2452566| +1796|AAAAAAAAEAHAAAAA|143759|4544|25376|2451671|2451641|Mr.|Joseph|Oconnor|N|31|1|1979|UNITED ARAB EMIRATES||Joseph.Oconnor@s.org|2452431| +1797|AAAAAAAAFAHAAAAA|650628|6680|15485|2449417|2449387|Miss|Jonie|Smith|Y|28|4|1946|SLOVAKIA||Jonie.Smith@NxM.org|2452324| +1798|AAAAAAAAGAHAAAAA|1444353|158|16716|2450493|2450463|Sir|Nicholas|Mccabe|N|10|1|1991|VIRGIN ISLANDS, U.S.||Nicholas.Mccabe@uZBB4cRu8FMsH8.edu|2452369| +1799|AAAAAAAAHAHAAAAA|1570859|5720|39828|2451948|2451918|Sir|Brian|Hunt|N|8|12|1944|WESTERN SAHARA||Brian.Hunt@1QqkXk9qE.edu|2452543| +1800|AAAAAAAAIAHAAAAA|1653025|4961|27297|2451015|2450985|Miss|Anthony|Harris|Y|1|6|1946|BANGLADESH||Anthony.Harris@Kxk.org|2452355| +1801|AAAAAAAAJAHAAAAA||6612|42372|2452500||Mrs.|Rachel|Boyd|Y|30|||LUXEMBOURG||Rachel.Boyd@tor.com|2452439| +1802|AAAAAAAAKAHAAAAA|1050293|6136|14026|2451763|2451733|Dr.|Joey|Calderon|N|16|11|1950|RUSSIAN FEDERATION||Joey.Calderon@L3g8FqJp.com|2452426| +1803|AAAAAAAALAHAAAAA|700480|449|2371|2449046|2449016|Dr.|Rita|Holcomb|N|15|1|1954|PAPUA NEW GUINEA||Rita.Holcomb@KSxGVayc2U2.com|2452515| +1804|AAAAAAAAMAHAAAAA|822361|3870|27732|2452596|2452566|Ms.|Junie|Gonzalez|Y|26|8|1939|MAURITANIA||Junie.Gonzalez@M3N7tXavZGO.edu|2452641| +1805|AAAAAAAANAHAAAAA|1521291|4470|7649|2451540|2451510|Mr.|Daryl|Robinson|Y|24|7|1972|SAMOA||Daryl.Robinson@9bo7MGGP6rtE.com|2452436| +1806|AAAAAAAAOAHAAAAA|435505|1968|14031|2452020|2451990|Dr.|Glenn|Harlan|N|25|6|1970|ECUADOR||Glenn.Harlan@037qSalzN7OHL.edu|2452627| +1807|AAAAAAAAPAHAAAAA|365780|6448|37116|2450651|2450621|Dr.|Katherine|Mcfall|Y|11|6|1963|AFGHANISTAN||Katherine.Mcfall@caeoLUeaXZ5.org|2452436| +1808|AAAAAAAAABHAAAAA|1005493|4168|13643|2451332|2451302|Sir|Robert|Thornton|Y|24|5|1924|HONG KONG||Robert.Thornton@jas.edu|2452431| +1809|AAAAAAAABBHAAAAA||5143|3390|2449909|2449879|Sir||||12||1939||||| +1810|AAAAAAAACBHAAAAA||523|23256|||||Davis|||4|1940|TURKEY||Theodore.Davis@TtmQiM4EXO.com|2452543| +1811|AAAAAAAADBHAAAAA|650646|5570|13741|2452162|2452132|Dr.|Howard|Wilson|Y|2|11|1930|BAHAMAS||Howard.Wilson@s.edu|2452304| +1812|AAAAAAAAEBHAAAAA|1461688|6153|39865|2452240|2452210|Mr.|Thomas|Harding|Y|13|10|1939|JAMAICA||Thomas.Harding@Mm.org|2452465| +1813|AAAAAAAAFBHAAAAA|1083395|2207|20972|2451875|2451845|Mrs.|Molly|Burke|Y|2|7|1988|YEMEN||Molly.Burke@GiJu9FPh.org|2452537| +1814|AAAAAAAAGBHAAAAA|1485777|6431|15038|2449161|2449131|Ms.|Irene|Dow|Y|17|1|1949|AZERBAIJAN||Irene.Dow@uO.com|2452447| +1815|AAAAAAAAHBHAAAAA|608771|2742|18760|2452174|2452144|Mrs.|Donna|Buck|N|11|7|1971|GERMANY||Donna.Buck@1vB3tQVp98zgm4BEH.com|2452435| +1816|AAAAAAAAIBHAAAAA|431740|2140|10940|2451659|2451629|Ms.|Erica|Horner|N|10|10|1948|LIBERIA||Erica.Horner@4EFFlQi.com|2452314| +1817|AAAAAAAAJBHAAAAA|720529|6141|42200|2449377|2449347|Mrs.|Barbara|Gore|Y|2|3|1958|KUWAIT||Barbara.Gore@FTBuRGoE.com|2452576| +1818|AAAAAAAAKBHAAAAA|866740|6184|16075|2450620|2450590|Sir|Nathan|Hyatt|Y|14|11|1957|LIECHTENSTEIN||Nathan.Hyatt@n8yKdzs7Kjle2bf.org|2452358| +1819|AAAAAAAALBHAAAAA|1524059|3495|22978|2450912|2450882|Ms.|Julie|Neville|N|2|4|1929|CAYMAN ISLANDS||Julie.Neville@FXOIOUIICbf.edu|2452520| +1820|AAAAAAAAMBHAAAAA|1566068|687|14506|2451688|2451658|Mr.|Benito|Jones|Y|7|10|1975|GUYANA||Benito.Jones@cOJk.edu|2452404| +1821|AAAAAAAANBHAAAAA|697696|5553|24563|2450702|2450672|Sir|William|Tarver|Y|13|7|1977|NORWAY||William.Tarver@zg.edu|2452539| +1822|AAAAAAAAOBHAAAAA|737912|5233|15307|2452524|2452494|Dr.|Tommie|Dean|N|30|10|1935|UNITED ARAB EMIRATES||Tommie.Dean@0EDB.org|2452642| +1823|AAAAAAAAPBHAAAAA|480944|1313|10882|2451932|2451902|Dr.|Derek|Johnson|N|9|12|1927|COMOROS||Derek.Johnson@idgklsKhAM.org|2452426| +1824|AAAAAAAAACHAAAAA|1172936|4001|7066|2452631|2452601|Ms.|Naomi|Lee|N|6|3|1974|ETHIOPIA||Naomi.Lee@N.edu|2452515| +1825|AAAAAAAABCHAAAAA|1303316|6634|22799|2450214|2450184|Dr.|Moshe|Serrano|N|18|9|1981|RUSSIAN FEDERATION||Moshe.Serrano@n.edu|2452338| +1826|AAAAAAAACCHAAAAA|167264|2860|41609|2451650|2451620|Miss|Sherrie|Smith|Y|29|7|1965|CAMBODIA||Sherrie.Smith@sj5v15zjL8tU.com|2452401| +1827|AAAAAAAADCHAAAAA|645069|3707|13098|2452129|2452099|Mrs.|Cassandra|Oshea|N|26|12|1970|SAUDI ARABIA||Cassandra.Oshea@rh1.edu|2452302| +1828|AAAAAAAAECHAAAAA|1055006|2621|33836|2449473|2449443|Dr.|Terry|Acosta|Y|4|3|1944|JERSEY||Terry.Acosta@UkEG1LU8.edu|2452523| +1829|AAAAAAAAFCHAAAAA|1223362|5664|7566|2452105|2452075|Dr.|Yolanda|Vasquez|N|29|11|1988|EQUATORIAL GUINEA||Yolanda.Vasquez@d2RD3MOINE.org|2452296| +1830|AAAAAAAAGCHAAAAA|1237826|1827|24981|2450551|2450521|Mrs.|Soo|Ruiz|Y|7|12|1951|PAKISTAN||Soo.Ruiz@cq3xs9tlo2XkBX.com|2452339| +1831|AAAAAAAAHCHAAAAA|1354816||10513||2450065|||Romero|N|18|2||JORDAN||Bridgett.Romero@L9Z68QCpXkoL.com|2452517| +1832|AAAAAAAAICHAAAAA|128406|1495|15384|2451720|2451690|Mr.|William|Wright|Y|29|3|1964|WALLIS AND FUTUNA||William.Wright@5Zxtm.com|2452352| +1833|AAAAAAAAJCHAAAAA|377597|2897|12197|2449826|2449796|Mrs.|Leah|Fox|N|15|10|1950|DENMARK||Leah.Fox@aDZ8YQCcm6f.org|2452536| +1834|AAAAAAAAKCHAAAAA|1495244|6380|41916|2452560|2452530|Dr.|Kyle|Parham|N|24|5|1943|MADAGASCAR||Kyle.Parham@xrr7O4bo.edu|2452460| +1835|AAAAAAAALCHAAAAA|1801791|2440|10689|2449649|2449619|Mr.|Charles|Pritchard|Y|9|10|1936|SAN MARINO||Charles.Pritchard@KbgeeVaMx5MOF0lxSuD.org|2452300| +1836|AAAAAAAAMCHAAAAA|1742096|4065|33234|2450281|2450251|Dr.|Robert|Massey|N|14|7|1959|DENMARK||Robert.Massey@BBq.org|2452617| +1837|AAAAAAAANCHAAAAA|1661917|4746|13175|2449892|2449862|Miss|Crystal|Cohen|Y|16|11|1960|SLOVENIA||Crystal.Cohen@a5rXoEhxFVvcVftYOQ.edu|2452556| +1838|AAAAAAAAOCHAAAAA|576966|1450|42781|2450260|2450230|Dr.|Sue|Neal|Y|31|10|1941|SWAZILAND||Sue.Neal@iK5Z1MPH5lSAJo2.com|2452434| +1839|AAAAAAAAPCHAAAAA|709871|4023|9422|2452273|2452243|Ms.|Linda|Garcia|N|27|9|1930|RUSSIAN FEDERATION||Linda.Garcia@q8drHsLqIt1.com|2452298| +1840|AAAAAAAAADHAAAAA|362022|2676|49730|2450460|2450430|Mr.|Joseph|Evans|N|12|2|1973|MONTENEGRO||Joseph.Evans@3F2RxDhS4S.com|2452453| +1841|AAAAAAAABDHAAAAA|220487|5605|42723|2452256|2452226|Dr.|Troy|Akers|N|13|8|1969|ARMENIA||Troy.Akers@Rn9n.com|2452553| +1842|AAAAAAAACDHAAAAA|859941|819|34497|2450099|2450069|Miss|Trang|Taylor|Y|18|5|1944|VENEZUELA||Trang.Taylor@enJVGeGa4.org|2452609| +1843|AAAAAAAADDHAAAAA|711402|3640|21525|2451730|2451700|Ms.|Sharon|Barton|N|8|9|1971|GEORGIA||Sharon.Barton@VQ2kUyHSD73OQ.com|2452352| +1844|AAAAAAAAEDHAAAAA|1243963|1166|27969|2450760|2450730|Ms.|Josephine|Arnold|Y|18|5|1955|MADAGASCAR||Josephine.Arnold@FHXz3KpTGS1MPs2.org|2452624| +1845|AAAAAAAAFDHAAAAA|475816|133|13203|2450936|2450906|Sir|Damian|Glover|Y|19|7|1953|SIERRA LEONE||Damian.Glover@cHahR.com|2452291| +1846|AAAAAAAAGDHAAAAA|17302|5221|3093|2449543|2449513|Sir|Ruben|Olsen|N|24|6|1971|AZERBAIJAN||Ruben.Olsen@1SEEEltuv9rIAvDJXM.edu|2452365| +1847|AAAAAAAAHDHAAAAA|1607632|2253|74|2452548|2452518|Dr.|Courtney|Pollard|Y|31|10|1941|LATVIA||Courtney.Pollard@8JaY7NkKDmcg.org|2452339| +1848|AAAAAAAAIDHAAAAA|1031643|5759|49428|2450178|2450148|Sir|Donald|Galvan|N|23|12|1943|YEMEN||Donald.Galvan@nVeyaZrfMB8N5.edu|2452584| +1849|AAAAAAAAJDHAAAAA|1267952|4878|2157|2451848|2451818|Mr.|Willie|Anderson|Y|4|12|1934|ECUADOR||Willie.Anderson@aS2.com|2452392| +1850|AAAAAAAAKDHAAAAA|1853758|1774|42143|2450202|2450172|Mr.|Keith|Jones|N|22|8|1976|NAURU||Keith.Jones@3GrjtSAl2ZJODvld.edu|2452307| +1851|AAAAAAAALDHAAAAA|1016429|3577|28278|2450347|2450317|Sir|Charles|Uribe|Y|1|12|1976|UNITED STATES||Charles.Uribe@YcrR.com|2452551| +1852|AAAAAAAAMDHAAAAA|1920700|328|29156|2450457|2450427|Mrs.|Neva|Fitzsimmons|Y|30|12|1929|GUAM||Neva.Fitzsimmons@Or.edu|2452611| +1853|AAAAAAAANDHAAAAA|143618|4833|2001|2450378|2450348|Dr.|Benjamin|Murphy|N|11|11|1961|KAZAKHSTAN||Benjamin.Murphy@TvxcILRlQ.com|2452285| +1854|AAAAAAAAODHAAAAA|1727279|5576|6673|2452536|2452506|Mrs.|Hazel|Edge|N|16|8|1951|MACAO||Hazel.Edge@MfS5i563.com|2452414| +1855|AAAAAAAAPDHAAAAA|1453608|4211|31567|2449852|2449822|Dr.|Daniel|Valencia|Y|8|12|1960|CHRISTMAS ISLAND||Daniel.Valencia@zFXJP6eVzS.org|2452330| +1856|AAAAAAAAAEHAAAAA|92849|5855|16508|2449991|2449961|Ms.|Monique|Carlson|Y|14|11|1937|BOTSWANA||Monique.Carlson@pez.org|2452426| +1857|AAAAAAAABEHAAAAA|739527|1037|26547|2451759|2451729|Mr.|Randall|Kent|N|17|12|1938|HONG KONG||Randall.Kent@jAut1pS4UIblG.org|2452614| +1858|AAAAAAAACEHAAAAA||1619|49611||2451280||Ada|Carlton|||8||ANGUILLA|||| +1859|AAAAAAAADEHAAAAA|700687|5039|36238|2452495|2452465|Miss|Anthony|Jennings|N|15|4|1969|SAINT LUCIA||Anthony.Jennings@hkfe.org|2452584| +1860|AAAAAAAAEEHAAAAA|1542211|6555|44146|2451791|2451761|Ms.|Ella|Jones|N|16|12|1924|ANDORRA||Ella.Jones@v5MHDg2gexULmO1.org|2452646| +1861|AAAAAAAAFEHAAAAA|620768|7179|10743|2451245|2451215|Mrs.|Bernice|Walker|Y|20|6|1958|LITHUANIA||Bernice.Walker@SesSuhH2.com|2452484| +1862|AAAAAAAAGEHAAAAA|796922|1188|3501|2449181|2449151|Miss|Carolyn|Cotton|Y|19|12|1960|SENEGAL||Carolyn.Cotton@fvEyFp7u.edu|2452485| +1863|AAAAAAAAHEHAAAAA|68826|306|45339|2449614|2449584|Ms.|Edith|Petty|N|3|5|1945|KIRIBATI||Edith.Petty@SlpEAlu6R3y15Oe.com|2452599| +1864|AAAAAAAAIEHAAAAA|130453|6277|37654|2449786|2449756|Dr.|Leroy|Andersen|Y|22|1|1985|PUERTO RICO||Leroy.Andersen@ToaDq7kiFBV.com|2452465| +1865|AAAAAAAAJEHAAAAA|693341|379|32217|2451959|2451929|Ms.|Cynthia|Walsh|N|24|6|1955|LIBERIA||Cynthia.Walsh@8YcXpUmLS5nodO.edu|2452637| +1866|AAAAAAAAKEHAAAAA|61167|5859|31798|2449207|2449177|Mr.|Patrick|Turner|Y|8|10|1942|MOZAMBIQUE||Patrick.Turner@s.edu|2452354| +1867|AAAAAAAALEHAAAAA|1590358|2647|34889|2451700|2451670|Dr.|Cheryl|Batten|Y|15|10|1964|SERBIA||Cheryl.Batten@c79OgKETHda.org|2452522| +1868|AAAAAAAAMEHAAAAA|113652|2069|6486|2451771|2451741|Mr.|Gary|Douglass|Y|23|3|1985|FINLAND||Gary.Douglass@ZfuCJSZgNrB4.org|2452592| +1869|AAAAAAAANEHAAAAA|1152985|3110|435|2449106|2449076|Mrs.|Margaret|Shackelford|N|17|11|1980|MONACO||Margaret.Shackelford@ss5pPJ5omfC.com|2452438| +1870|AAAAAAAAOEHAAAAA|288396|6856|36516|2449917|2449887|Miss|Dolores|Simmons|N|27|5|1935|WESTERN SAHARA||Dolores.Simmons@BMRJi0vg0ENgIgv.org|2452569| +1871|AAAAAAAAPEHAAAAA|285412|6964|33669|2451578|2451548|Miss|Kimberly|York|N|9|5|1979|NEW CALEDONIA||Kimberly.York@SlfYucKkZJJE.org|2452644| +1872|AAAAAAAAAFHAAAAA|1396384|2895|16253|2450289|2450259|Sir|Larry|Miller|Y|22|6|1963|GUATEMALA||Larry.Miller@Qjq0lYIFh5zNdF.edu|2452351| +1873|AAAAAAAABFHAAAAA|1873854|1285|12763|2450431|2450401|Miss|Ericka|Barrow|N|10|3|1931|MALDIVES||Ericka.Barrow@k3pNpa.edu|2452403| +1874|AAAAAAAACFHAAAAA|543645|4406|27520|2450762|2450732|Mrs.|Gertrude|Ferguson|N|17|3|1979|BHUTAN||Gertrude.Ferguson@yPsyMM.com|2452505| +1875|AAAAAAAADFHAAAAA|143063|5943|22870|2450367|2450337|Dr.|Brenda|Rivera|N|2|7|1963|ISRAEL||Brenda.Rivera@itxUElBOuX60.edu|2452562| +1876|AAAAAAAAEFHAAAAA|831205|965|37525|2451505|2451475|Mr.|Ryan|Jones|Y|16|4|1925|SAUDI ARABIA||Ryan.Jones@ZJP71gzb.com|2452484| +1877|AAAAAAAAFFHAAAAA|528033|1639|13198|2450089|2450059|Dr.|Roger|Parker|Y|22|12|1964|TUNISIA||Roger.Parker@azSEDTvtV.edu|2452337| +1878|AAAAAAAAGFHAAAAA|1689966|3031|41296|2450514|2450484|Ms.|Bettie|Mullins|Y|17|3|1925|AUSTRIA||Bettie.Mullins@OMURnScbm1.org|2452320| +1879|AAAAAAAAHFHAAAAA|1677201|1811|41221|2452599|2452569|Dr.|Derek|Scott|N|17|2|1974|UGANDA||Derek.Scott@TayDVpbCHiPNK7br.org|2452590| +1880|AAAAAAAAIFHAAAAA|106965|1466|49614|2452396|2452366|Dr.|Barbara|Howell|N|6|11|1988|UNITED ARAB EMIRATES||Barbara.Howell@5ESUIT4X3xBkGU.edu|2452414| +1881|AAAAAAAAJFHAAAAA|644725|538|2798|2451699|2451669|Mrs.|Colleen|Moore|Y|14|4|1944|AUSTRIA||Colleen.Moore@p.edu|2452383| +1882|AAAAAAAAKFHAAAAA|86687|2819|18551|2450106|2450076|Miss|Maxine|Oneil|N|3|6|1956|MOLDOVA, REPUBLIC OF||Maxine.Oneil@Izz5MYxu2N9vz.edu|2452584| +1883|AAAAAAAALFHAAAAA|584644|5068|49139|2451924|2451894|Ms.|Harriett|Cave|N|12|8|1959|HONDURAS||Harriett.Cave@lGPnQZk4a1D7.edu|2452536| +1884|AAAAAAAAMFHAAAAA|65626|4587|27101|2450727|2450697|Dr.|Dale|Lehman|N|18|3|1951|C�TE D'IVOIRE||Dale.Lehman@BjthGJUFhdy.edu|2452572| +1885|AAAAAAAANFHAAAAA|1570835|232|18429|2452288|2452258|Sir|Ernest|Fields|N|12|4|1951|NORWAY||Ernest.Fields@xRA0BP8IG.edu|2452495| +1886|AAAAAAAAOFHAAAAA|1539380|7162|37697|2450496|2450466|Sir|Marlon|Faison|Y|17|5|1958|SAMOA||Marlon.Faison@XACa65TzNt0HFNO.edu|2452295| +1887|AAAAAAAAPFHAAAAA|591458|1794|27777|2450001|2449971|Dr.|Helen|Courtney|Y|17|1|1958|GUINEA-BISSAU||Helen.Courtney@yZkY5Urdskv.org|2452422| +1888|AAAAAAAAAGHAAAAA|1378705|3603|27301|2449729|2449699|Dr.|Larry|Olivas|Y|9|2|1975|WESTERN SAHARA||Larry.Olivas@7r7DYaK8.org|2452510| +1889|AAAAAAAABGHAAAAA|1198296|3735|36327|2450042|2450012|Mr.|Randy|Penn|N|18|6|1971|GUINEA-BISSAU||Randy.Penn@QLUMA4gtLD9c0I.edu|2452524| +1890|AAAAAAAACGHAAAAA|913655|5283|9311|2450875|2450845|Sir|Jonathan|Arredondo|Y|3|10|1927|MALAWI||Jonathan.Arredondo@xtAieU.edu|2452357| +1891|AAAAAAAADGHAAAAA|528371|966|46761|2449722|2449692|Ms.|Paige|Sawyer|Y|21|6|1945|ANTARCTICA||Paige.Sawyer@V03l2tUtaMzeh9xkq9f.edu|2452388| +1892|AAAAAAAAEGHAAAAA|559362|6140|25671|2452529|2452499|Dr.|Alexander|Walker|N|8|10|1983|TURKMENISTAN||Alexander.Walker@YvOE.com|2452403| +1893|AAAAAAAAFGHAAAAA||3782|45099||||Leonida||||6||||Leonida.Turner@eecx.org|| +1894|AAAAAAAAGGHAAAAA|1790088|2055|14375|2451683|2451653|Dr.|Lawrence|Rodriquez|Y|13|10|1925|GERMANY||Lawrence.Rodriquez@nrjrizgnYB.com|2452646| +1895|AAAAAAAAHGHAAAAA|1253123|2775|11227|2451240|2451210|Sir|Leroy|Thao|N|9|4|1970|UNITED KINGDOM||Leroy.Thao@oS9k1.com|2452405| +1896|AAAAAAAAIGHAAAAA|907338|6149|14560|2449289|2449259|Miss|Alice|Snyder|N|29|5|1933|QATAR||Alice.Snyder@uPPcVSmog11.edu|2452484| +1897|AAAAAAAAJGHAAAAA|510604|5207|24257|2452020|2451990|Sir|Kevin|Davis|Y|10|8|1940|IRAQ||Kevin.Davis@01ZkyG77JBXa8Gli.com|2452315| +1898|AAAAAAAAKGHAAAAA|441139|654|9016|2452345|2452315|Ms.|Donna|Pratt|N|24|12|1950|NICARAGUA||Donna.Pratt@9V75ZFU.edu|2452640| +1899|AAAAAAAALGHAAAAA|722639|6751|48110|2451907|2451877|Ms.|Stephanie|Watkins|Y|8|5|1938|OMAN||Stephanie.Watkins@2ao.com|2452571| +1900|AAAAAAAAMGHAAAAA|213780|3215|31003|2449511|2449481|Sir|Luther|Lebron|N|2|12|1933|SINGAPORE||Luther.Lebron@XI4Fpn9Fejkge.org|2452598| +1901|AAAAAAAANGHAAAAA|1917206|938|46670|2451711|2451681|Sir|Micheal|Wilber|Y|13|8|1992|ICELAND||Micheal.Wilber@65NAK3dyg8fd.org|2452353| +1902|AAAAAAAAOGHAAAAA|565631|1451|45224|2450215|2450185|Mr.|Timothy|Smith|N|18|7|1934|ARMENIA||Timothy.Smith@YTA7XrV.org|2452503| +1903|AAAAAAAAPGHAAAAA|1579071|6262|39100|2452547|2452517|Mr.|Jesse|Garcia|N|13|4|1929|FAROE ISLANDS||Jesse.Garcia@qaQSIMLTTeA.com|2452368| +1904|AAAAAAAAAHHAAAAA|886681|274|27020|2452351|2452321|Sir|Jimmy|Driscoll|N|27|5|1934|SERBIA||Jimmy.Driscoll@4YxUzsZZEUq.edu|2452400| +1905|AAAAAAAABHHAAAAA|371522|3270|48618|2451986|2451956|Dr.|Clayton|Kingsbury|Y|21|10|1976|BOLIVIA||Clayton.Kingsbury@U2Ckjvr7BTgoT6F.org|2452438| +1906|AAAAAAAACHHAAAAA|1373565|7050|33688|2449458|2449428|Miss|Allen|Turner|Y|28|7|1958|POLAND||Allen.Turner@4.com|2452575| +1907|AAAAAAAADHHAAAAA|1086481|6548|1062|2451437|2451407|Miss|Vivian|Beardsley|N|27|11|1956|OMAN||Vivian.Beardsley@J9CZD8Ko.org|2452323| +1908|AAAAAAAAEHHAAAAA|158555|6393|25831|2449586|2449556|Dr.|Hester|Lincoln|Y|2|12|1986|NEW ZEALAND||Hester.Lincoln@Re8gUG6Dxzovael.org|2452424| +1909|AAAAAAAAFHHAAAAA|1269986|1604|19975|2452281|2452251|Ms.|Ella|Bell|Y|7|4|1925|FRENCH POLYNESIA||Ella.Bell@3mM.org|2452287| +1910|AAAAAAAAGHHAAAAA|529240|675|4510|2449773|2449743|Mrs.|Danette|Paris|N|20|6|1968|UNITED STATES||Danette.Paris@d7fzqeSId1.org|2452288| +1911|AAAAAAAAHHHAAAAA|1249263|2635|9960|2452032|2452002|Miss|Freda|Caraway|N|22|7|1953|ARUBA||Freda.Caraway@mOjL638d5o.com|2452329| +1912|AAAAAAAAIHHAAAAA|1316940|2230|43520|2451401|2451371|Sir|Jerry|Rogers|N|7|6|1981|BULGARIA||Jerry.Rogers@lcQ3CDrPGTVXS.com|2452539| +1913|AAAAAAAAJHHAAAAA|876047|6237|42827|2452108|2452078|Dr.|Barbara|Jackson|Y|22|7|1966|ITALY||Barbara.Jackson@E.com|2452355| +1914|AAAAAAAAKHHAAAAA|533874|1466|15458|2451431|2451401|Mr.|William|Berry|Y|26|2|1950|CYPRUS||William.Berry@edrHttULeL.org|2452627| +1915|AAAAAAAALHHAAAAA|648859|1083|13354|2451636|2451606|Mr.|Todd|Schneider|N|12|2|1974|INDIA||Todd.Schneider@gSNFirsNHK.edu|2452336| +1916|AAAAAAAAMHHAAAAA|449626|2560|11094|2450006|2449976|Miss|Rose|Manuel|N|3|7|1964|R�UNION||Rose.Manuel@B7vBPPYN6hge0L43s.org|2452624| +1917|AAAAAAAANHHAAAAA|1730887|4547|19854|2450616|2450586|Dr.|Darrell|Moore|Y|30|3|1943|KUWAIT||Darrell.Moore@xGRTdV73gC6Cmh.org|2452356| +1918|AAAAAAAAOHHAAAAA|701502|6270|17838|2450689|2450659|Sir|Edward|Deal|Y|8|12|1960|TURKEY||Edward.Deal@sH0Ra3PnHiS1Tp9U8.edu|2452464| +1919|AAAAAAAAPHHAAAAA|876806|5606|23559|2449813|2449783|Miss|Vera|Mcclendon|Y|22|11|1975|NAURU||Vera.Mcclendon@LqXCh243svq20byvAdi.com|2452462| +1920|AAAAAAAAAIHAAAAA|618895|4038|7534|2451512|2451482|Mr.|Jason|Horne|N|9|3|1948|GERMANY||Jason.Horne@AhlHH6DdgdDQEOet.edu|2452284| +1921|AAAAAAAABIHAAAAA|76232|6680|36728|2450019|2449989|Dr.|Ashley|Oliver|Y|18|5|1970|SYRIAN ARAB REPUBLIC||Ashley.Oliver@7X4thGIQ764Oa.edu|2452461| +1922|AAAAAAAACIHAAAAA|976374|5427|32480|2451884|2451854|Mr.|Johnnie|Cole|N|6|11|1992|ARUBA||Johnnie.Cole@hKKmy7Y0RycMhybMHCz.com|2452387| +1923|AAAAAAAADIHAAAAA|1639487|6575|22416|2449783|2449753|Dr.|James|Casey|Y|29|3|1958|BELGIUM||James.Casey@Hc5l6NJk.com|2452632| +1924|AAAAAAAAEIHAAAAA|22020|6122|28671|2451853|2451823|Miss|Alice|Lee|Y|22|12|1971|KAZAKHSTAN||Alice.Lee@SxD6ZUGV.org|2452472| +1925|AAAAAAAAFIHAAAAA|31877|2285|6246|2449499|2449469|Miss|Dorothy|Burrell|Y|12|10|1971|MALI||Dorothy.Burrell@4rtkfAdTPlDvOL.org|2452615| +1926|AAAAAAAAGIHAAAAA|1455344|840|46336|2452672|2452642|Dr.|Christian|Eaton|N|14|9|1960|ITALY||Christian.Eaton@RQ2qH40JGP88Ec.org|2452504| +1927|AAAAAAAAHIHAAAAA|680838|3968|15098|2452422|2452392|Mrs.|Simone|Walker|N|10|6|1970|PERU||Simone.Walker@yQ.edu|2452627| +1928|AAAAAAAAIIHAAAAA|834130|6137|12944|2450091|2450061|Sir|Leonard|Bolt|Y|14|2|1980|MARTINIQUE||Leonard.Bolt@FV.edu|2452407| +1929|AAAAAAAAJIHAAAAA|1006201|3901|39248|2450194|2450164|Mrs.|Ruth|Lentz|Y|2|11|1982|MARSHALL ISLANDS||Ruth.Lentz@7bYcEaEsujy4.com|2452395| +1930|AAAAAAAAKIHAAAAA|615366|1850|49981|2450299|2450269|Sir|Joseph|White|Y|17|12|1963|GIBRALTAR||Joseph.White@23Us2UstOluKA8d.com|2452645| +1931|AAAAAAAALIHAAAAA|1308976|2734|38492|2452233|2452203|Sir|Steven|Lopez|N|10|12|1986|GUINEA-BISSAU||Steven.Lopez@E.edu|2452360| +1932|AAAAAAAAMIHAAAAA|1543212|3162|45065|2449523|2449493|Miss|Mina|Marsh|N|16|9|1988|SAMOA||Mina.Marsh@td6gtbj2iRi.edu|2452400| +1933|AAAAAAAANIHAAAAA|160818|7143|14268|2452422|2452392|Mr.|Michael|Helton|Y|27|11|1927|DENMARK||Michael.Helton@UAo4hRcC3BH9.com|2452522| +1934|AAAAAAAAOIHAAAAA|||49596|2450411|2450381|Ms.|||Y|1|9||CYPRUS||Joyce.Nicholson@jUM2.edu|2452521| +1935|AAAAAAAAPIHAAAAA|354688|4376|24111|2452483|2452453|Miss|Anita|Coleman|Y|16|10|1970|INDIA||Anita.Coleman@RFQ2Jy4X4UcisF.com|2452400| +1936|AAAAAAAAAJHAAAAA|1075941|1587|21233|2452675|2452645|Ms.|Valerie|Read|N|3|6|1963|BERMUDA||Valerie.Read@PBQFiYqCeQ5FLRM3.org|2452585| +1937|AAAAAAAABJHAAAAA|567128|5029|42530|2451040|2451010|Sir|Daniel|Lofton|N|11|6|1932|SURINAME||Daniel.Lofton@g.com|2452539| +1938|AAAAAAAACJHAAAAA|1150510|2634|25605|2449505|2449475|Ms.|Francine|Wing|Y|8|1|1935|MAYOTTE||Francine.Wing@EnC.org|2452332| +1939|AAAAAAAADJHAAAAA|1638612|3353|3294|2449908|2449878|Dr.|Sharon|Kennedy|N|23|10|1948|GAMBIA||Sharon.Kennedy@MEvpRGVKJ9q.com|2452457| +1940|AAAAAAAAEJHAAAAA|1130547|6936|25325|2451099|2451069|Mrs.|Linda|Slack|N|15|2|1982|EL SALVADOR||Linda.Slack@F9XeI7MjM8.com|2452551| +1941|AAAAAAAAFJHAAAAA|1753508|7186|8911|2449912|2449882|Dr.|Joy|Vaughn|Y|28|2|1924|TOGO||Joy.Vaughn@c9Lg.org|2452314| +1942|AAAAAAAAGJHAAAAA|622848|3189|11930|2450740|2450710|Miss|Catherine|Williams|Y|6|3|1948|BELARUS||Catherine.Williams@3M7zoQ.org|2452576| +1943|AAAAAAAAHJHAAAAA|473619|2732|11798|2451878|2451848|Mr.|Michael|Martin|Y|1|10|1937|NIGERIA||Michael.Martin@x.edu|2452323| +1944|AAAAAAAAIJHAAAAA|357099|376|406|2452320|2452290|Sir|Melvin|Dunbar|N|26|12|1951|SWAZILAND||Melvin.Dunbar@ih62B.com|2452408| +1945|AAAAAAAAJJHAAAAA|753855|4873|10338|2451867|2451837|Dr.|Queen|Shelton|Y|2|8|1937|BURUNDI||Queen.Shelton@lOKLs6Q.com|2452641| +1946|AAAAAAAAKJHAAAAA|1174987|6277|38320|2449157|2449127|Mrs.|Wilma|Turner|N|14|3|1926|SPAIN||Wilma.Turner@9zjYJY7GtMgju.edu|2452528| +1947|AAAAAAAALJHAAAAA|385341|1022|36703|2449361|2449331|Sir|Joe|Singer|N|9|1|1968|MARTINIQUE||Joe.Singer@R26XcAkIUF5.org|2452382| +1948|AAAAAAAAMJHAAAAA|1256241|6866|38291|2449185|2449155|Sir|Willis|Pope|N|13|12|1947|MOZAMBIQUE||Willis.Pope@ok82.com|2452485| +1949|AAAAAAAANJHAAAAA|1418363|5115|5820|2449486|2449456|Mr.|John|Hopper|N|12|6|1933|GIBRALTAR||John.Hopper@dPDX.com|2452639| +1950|AAAAAAAAOJHAAAAA|630071|5682|10046|2451401|2451371|Mr.|Nathan|Murillo|Y|22|8|1987|BHUTAN||Nathan.Murillo@JU06UuP5.edu|2452434| +1951|AAAAAAAAPJHAAAAA|702699|4993|28040|2452407|2452377|Dr.|Elliot|Rogers|N|2|7|1991|BHUTAN||Elliot.Rogers@Mc4aNeJqIuauXPt0PR.org|2452580| +1952|AAAAAAAAAKHAAAAA|1777762|6573|31784|2452343|2452313|Miss|Claudia|Campbell|N|4|4|1967|MAURITIUS||Claudia.Campbell@9e94Zg9zyNZ.org|2452408| +1953|AAAAAAAABKHAAAAA|1335417|6225|22633|2449446|2449416|Mrs.|Rose|Hornsby|N|16|9|1941|NETHERLANDS||Rose.Hornsby@EEhntEeDYCnaV.edu|2452342| +1954|AAAAAAAACKHAAAAA|1041540|689|11438|2451715|2451685|Mr.|William|Kasper|N|20|7|1942|SAINT LUCIA||William.Kasper@OK.org|2452588| +1955|AAAAAAAADKHAAAAA|1603823|2682|1541|2450194|2450164|Mrs.|Alice|Wood|N|24|4|1957|SAN MARINO||Alice.Wood@5kdov5o.org|2452550| +1956|AAAAAAAAEKHAAAAA|957355|5100|26492|2450034|2450004|Miss|Brenda|Hough|N|6|4|1992|JERSEY||Brenda.Hough@a0UAmh87KG41.com|2452308| +1957|AAAAAAAAFKHAAAAA|1611143||1980|||||Bennett||27|8|1943|EGYPT|||2452527| +1958|AAAAAAAAGKHAAAAA|1343118|6809|2500|2449599|2449569|Ms.|Elizabeth|Handley|Y|6|2|1926|GUINEA||Elizabeth.Handley@6n5x3itX.edu|2452433| +1959|AAAAAAAAHKHAAAAA|1663403|1911|28866|2450769|2450739|Mr.|Phillip|Rich|N|16|4|1945|MACAO||Phillip.Rich@FcuE9y9qum.org|2452295| +1960|AAAAAAAAIKHAAAAA|1673192|1615|47191|2450934|2450904|Sir|Kevin|Howell|Y|16|8|1924|KAZAKHSTAN||Kevin.Howell@dxpotLDXoB57SDEbL.edu|2452537| +1961|AAAAAAAAJKHAAAAA|420428|6142|15809|2451906|2451876|Ms.|Judy|Pickett|N|12|4|1924|BOTSWANA||Judy.Pickett@qHUKeKdkbSE43aGY.edu|2452460| +1962|AAAAAAAAKKHAAAAA|664142|4342|31389|2451750|2451720|Miss|Virginia|Reed|Y|10|6|1941|AUSTRALIA||Virginia.Reed@ERBhPb61BkKCdUYeR.org|2452562| +1963|AAAAAAAALKHAAAAA||1085|49470|2451759|2451729|Miss|Nancy|Willis||27|||||Nancy.Willis@LqRdLvjsR.org|| +1964|AAAAAAAAMKHAAAAA|1089720|6402|7884|2452274|2452244|Mr.|Jason|Howard|N|14|4|1992|LATVIA||Jason.Howard@KVu1Mba8G1.com|2452449| +1965|AAAAAAAANKHAAAAA|263522|6554|13586|2449493|2449463|Dr.|Janice|Broadway|N|6|8|1944|UKRAINE||Janice.Broadway@374K9PEMvl.com|2452453| +1966|AAAAAAAAOKHAAAAA|1137497|3504|48664|2450671|2450641|Dr.|Robert|Taylor|Y|12|1|1981|CAPE VERDE||Robert.Taylor@MPuJhsHBG72PJa9.com|2452366| +1967|AAAAAAAAPKHAAAAA|407590|3618|25999|2451203|2451173|Miss|Jacqueline|Jeffries|N|5|12|1937|MONTSERRAT||Jacqueline.Jeffries@rEEENdMXBSkpV2.org|2452300| +1968|AAAAAAAAALHAAAAA|696806|64|14122|2450327|2450297|Dr.|Frank|Mcfall|Y|24|9|1957|SLOVAKIA||Frank.Mcfall@rQTZ1i3v89Q2Qol.com|2452450| +1969|AAAAAAAABLHAAAAA|81203|2676|949|2449210|2449180|Mrs.|Alicia|Southerland|Y|3|2|1950|BOTSWANA||Alicia.Southerland@Q.edu|2452624| +1970|AAAAAAAACLHAAAAA|506711|3903|47448|2449263|2449233|Miss||Carson|N||12|1951|||Sophie.Carson@HopNCX1ZD.org|2452455| +1971|AAAAAAAADLHAAAAA|666557|604|15051|2450323|2450293|Mr.|James|Richard|N|23|1|1987|COMOROS||James.Richard@mDL5QAJrJn.edu|2452622| +1972|AAAAAAAAELHAAAAA|516371|550|25576|2449720|2449690|Dr.|Kenneth|Price|N|25|1|1947|ALGERIA||Kenneth.Price@CKoF.com|2452486| +1973|AAAAAAAAFLHAAAAA|413041|2683|22278|2451313|2451283|Mr.|Gene|Walton|N|18|7|1939|OMAN||Gene.Walton@P344VsD.com|2452480| +1974|AAAAAAAAGLHAAAAA|194087|1097|8060|2450457|||David|Brown|N|||1978|BANGLADESH||David.Brown@A4M9UPv7qDus.edu|2452599| +1975|AAAAAAAAHLHAAAAA|616680|405|49330|2452318|2452288|Mr.|Alfred|Kern|N|12|3|1961|ECUADOR||Alfred.Kern@vsz9Scj.org|2452296| +1976|AAAAAAAAILHAAAAA|1149483|5815|39794|2450387|2450357|Mrs.|Nancy|Link|Y|3|12|1977|GIBRALTAR||Nancy.Link@QcKE.com|2452539| +1977|AAAAAAAAJLHAAAAA|115194|3740|6286|2449568|2449538|Mr.|Ramon|Chambers|Y|20|9|1955|MONACO||Ramon.Chambers@71OiSmgAjZIC.com|2452353| +1978|AAAAAAAAKLHAAAAA|961310|4543|24223|2452456|2452426|Dr.|Douglas|Richards|Y|1|6|1943|BHUTAN||Douglas.Richards@ZRspAhBsC.com|2452545| +1979|AAAAAAAALLHAAAAA|1527247|1947|40647|2452150|2452120|Sir|Fred|Kessler|Y|19|7|1958|NIUE||Fred.Kessler@OfG1OA3hPM.edu|2452464| +1980|AAAAAAAAMLHAAAAA|196727|5624|30168|2450454|2450424|Dr.|Socorro|Scribner|N|25|7|1988|BAHAMAS||Socorro.Scribner@vqnJrL7.edu|2452359| +1981|AAAAAAAANLHAAAAA|1244546|728|20557|2451083|2451053|Dr.|Ronald|Johnson|Y|19|10|1962|HUNGARY||Ronald.Johnson@XCc9hM.com|2452283| +1982|AAAAAAAAOLHAAAAA|1647566|5659|199|2451127|2451097|Mr.|Tyler|Martin|Y|24|2|1943|CHILE||Tyler.Martin@VDMqk6Mqom5gCX63nqY.org|2452515| +1983|AAAAAAAAPLHAAAAA|1578955|4515|8828|2449571|2449541|Miss|Charlotte|Mueller|Y|10|8|1955|IRAQ||Charlotte.Mueller@ooXlA.com|2452404| +1984|AAAAAAAAAMHAAAAA||7044|43915|2450150||||||||1957|MOROCCO||Carolyn.Pinto@n.edu|2452476| +1985|AAAAAAAABMHAAAAA|1408624|159|24447|2449199|2449169|Mrs.|Anthony|Chapin|Y|3|10|1964|VIET NAM||Anthony.Chapin@yA5lnCrQF9HM.edu|2452490| +1986|AAAAAAAACMHAAAAA|730160|4863|23966|2450262|2450232|Dr.|Kenneth|Tucker|Y|13|4|1948|FINLAND||Kenneth.Tucker@Sy9kyCUvLN.edu|2452532| +1987|AAAAAAAADMHAAAAA|629558|450|34495|2449139|2449109|Dr.|Stephanie|Gonzales|N|4|1|1953|TRINIDAD AND TOBAGO||Stephanie.Gonzales@ft.com|2452363| +1988|AAAAAAAAEMHAAAAA|1707286|6961|44236|2450814|2450784|Miss|Jodie|Medeiros|N|17|6|1926|OMAN||Jodie.Medeiros@yROHlF0XmP.com|2452483| +1989|AAAAAAAAFMHAAAAA|1237467|1028|26589|2449584|2449554|Mr.|David|Johnson|N|25|1|1988|RWANDA||David.Johnson@IlXmtQ8SKX1V.com|2452625| +1990|AAAAAAAAGMHAAAAA|1237541|2403|26865|2450372|2450342|Ms.|Anna|Stevens|N|7|7|1975|PAKISTAN||Anna.Stevens@Bt.com|2452581| +1991|AAAAAAAAHMHAAAAA|1285968|5201|20081|2450453|2450423|Dr.|Katherin|Hansen|Y|23|4|1975|KYRGYZSTAN||Katherin.Hansen@iFgFQI.edu|2452549| +1992|AAAAAAAAIMHAAAAA|447397|3672|394|2452153|2452123|Dr.|Ashley|Britt|N|8|4|1945|GUADELOUPE||Ashley.Britt@f.edu|2452345| +1993|AAAAAAAAJMHAAAAA|145044|6279|33961|2450630|2450600|Dr.|Elia|Wills|Y|31|8|1981|LUXEMBOURG||Elia.Wills@aINkrTagEo7cS.com|2452560| +1994|AAAAAAAAKMHAAAAA|289|6898|42307|2450477|2450447|Sir|James|Watson|N|15|12|1933|VANUATU||James.Watson@tpyRKVE1Q5S.edu|2452631| +1995|AAAAAAAALMHAAAAA|1824185|4006|21827|2451511|2451481|Mr.|Patrick|Greco|Y|24|12|1927|UKRAINE||Patrick.Greco@74.org|2452607| +1996|AAAAAAAAMMHAAAAA|659453|258|33816|2450774|2450744|Miss|Delores|Ross|Y|27|6|1935|MALTA||Delores.Ross@O.edu|2452509| +1997|AAAAAAAANMHAAAAA|700314|3299|32237|2450482|2450452|Sir|Billy|Gardner|N|18|10|1988|BARBADOS||Billy.Gardner@X1f2yZtTsvMXIc.edu|2452640| +1998|AAAAAAAAOMHAAAAA|766318|3851|38184|2450027|2449997|Mr.|Marcos|Loyd|Y|19|1|1934|KENYA||Marcos.Loyd@DcbHT.com|2452520| +1999|AAAAAAAAPMHAAAAA|262728|1467|33453|2452384|2452354|Mrs.|Angela|Villegas|N|11|7|1987|PAPUA NEW GUINEA||Angela.Villegas@KKrRqoCBNSd20XkSV.com|2452593| +2000|AAAAAAAAANHAAAAA|210117|3245|7973|2451990|2451960|Sir|Jacob|Harry|Y|14|5|1973|SPAIN||Jacob.Harry@pQkrGRRhYZzg2cbL.org|2452420| +2001|AAAAAAAABNHAAAAA|1299860|5676|19092|2450059|2450029|Ms.|Louise|Butler|Y|10|10|1959|BARBADOS||Louise.Butler@USG0IMTm5yjhSkp2.org|2452586| +2002|AAAAAAAACNHAAAAA|1874865|5504|27790|2450165|2450135|Mrs.|Betty|Meek|Y|12|4|1944|ANTIGUA AND BARBUDA||Betty.Meek@P8cl77NvGJ.com|2452536| +2003|AAAAAAAADNHAAAAA|454402|5852|32855|2449589|2449559|Dr.|Charles|Shapiro|Y|4|10|1990|LITHUANIA||Charles.Shapiro@8N.edu|2452611| +2004|AAAAAAAAENHAAAAA|1401574|4890|36206|2449188|2449158|Mr.|Andres|Walker|N|24|5|1933|CAMEROON||Andres.Walker@DrzvAkgjmGP.edu|2452486| +2005|AAAAAAAAFNHAAAAA|||34654||2451707|Dr.||||27|8|1944|||Carlos.Shull@gRX61qz.edu|2452298| +2006|AAAAAAAAGNHAAAAA|999610|1377|31944|2449673|2449643|Mr.|Richard|Pearson|N|13|6|1975|PAKISTAN||Richard.Pearson@CpXL1rAIulL6fKh85.edu|2452295| +2007|AAAAAAAAHNHAAAAA|538842|1038|49187|2451870|2451840|Mrs.|Julia|Tennant|Y|6|12|1927|MADAGASCAR||Julia.Tennant@T5nfGm55tl.com|2452570| +2008|AAAAAAAAINHAAAAA|1908527|862|17550|2451441|2451411|Miss|Arthur|Norfleet|Y|12|1|1981|GAMBIA||Arthur.Norfleet@SAOlkAymk.edu|2452311| +2009|AAAAAAAAJNHAAAAA|1270697|2066|33882|2451073|2451043|Miss|Nicole|Dixon|Y|19|12|1929|LUXEMBOURG||Nicole.Dixon@RPfVrtg.edu|2452306| +2010|AAAAAAAAKNHAAAAA|910674|3301|40775|2452420|2452390|Miss|Lucille|Puckett|N|6|5|1990|KOREA, REPUBLIC OF||Lucille.Puckett@9IO6pBu5qp6ZO.com|2452477| +2011|AAAAAAAALNHAAAAA||7008|6082|||||||8||1962||||2452478| +2012|AAAAAAAAMNHAAAAA|614501||1484||2450023|||Holt|Y|16|9||SERBIA|||| +2013|AAAAAAAANNHAAAAA|323266|5451|48836|2449828|2449798|Dr.|Debra|Peterson|N|12|2|1980|INDONESIA||Debra.Peterson@lPnp320YYQt6MS.edu|2452511| +2014|AAAAAAAAONHAAAAA|630825|697|18922|2451337|2451307|Ms.|Margo|Oconnell|Y|22|2|1956|KIRIBATI||Margo.Oconnell@8tiDpJ95LIUHvzaXvs.edu|2452515| +2015|AAAAAAAAPNHAAAAA|1048111|6988|1865|2452490|2452460|Ms.|Caroline|Culp|N|13|1|1930|UGANDA||Caroline.Culp@Pk0sQmdZMOcAMEzjx.com|2452581| +2016|AAAAAAAAAOHAAAAA|1029245|2964|17130|2452572|2452542|Dr.|Joshua|Stewart|N|22|9|1970|C�TE D'IVOIRE||Joshua.Stewart@hU.org|2452393| +2017|AAAAAAAABOHAAAAA|1092043|2957|34271|2449239|2449209|Ms.|Deanne|Vasquez|Y|23|2|1934|WESTERN SAHARA||Deanne.Vasquez@HC6I9Cr6J.com|2452381| +2018|AAAAAAAACOHAAAAA|1505721|5485|10802|2451696|2451666|Sir|Richard|Finch|Y|9|12|1937|ITALY||Richard.Finch@jIK.org|2452348| +2019|AAAAAAAADOHAAAAA|148738|3306|11827|2450989|2450959|Mr.|Bradford|Harder|N|17|5|1941|KAZAKHSTAN||Bradford.Harder@eIVP16.edu|2452567| +2020|AAAAAAAAEOHAAAAA|1343119|3454|48989|2451581|2451551|Mrs.|Martha|Tuck|Y|5|9|1965|NAURU||Martha.Tuck@mOpJVsqVGOmj.org|2452521| +2021|AAAAAAAAFOHAAAAA|210057|558|43859|2449834|2449804|Mr.|Blake|Larkin|Y|6|9|1939|ANGUILLA||Blake.Larkin@s1.edu|2452333| +2022|AAAAAAAAGOHAAAAA|1545966|6111|23456|2451649|2451619|Sir|Danny|Bailey|N|17|5|1934|TUVALU||Danny.Bailey@zo.org|2452607| +2023|AAAAAAAAHOHAAAAA|246931|5198|43018|2450368|2450338|Ms.|Alexis|Lee|Y|26|2|1982|BELARUS||Alexis.Lee@kCZijosr2KspoV8CT.edu|2452345| +2024|AAAAAAAAIOHAAAAA|1285011|2912|2813|2451403|2451373|Mrs.|Beatrice|Gomez|Y|16|8|1961|FAROE ISLANDS||Beatrice.Gomez@LDgJ5AA.com|2452463| +2025|AAAAAAAAJOHAAAAA|1501840|2568|21249|2452635|2452605|Mr.|James|Johnson|N|7|11|1956|UNITED KINGDOM||James.Johnson@0g81KG18m1dl.org|2452419| +2026|AAAAAAAAKOHAAAAA|1366164|4140|28869|2449223|2449193|Mr.|Jesus|Gordon|N|23|1|1988|MALAYSIA||Jesus.Gordon@7XbjYl.org|2452371| +2027|AAAAAAAALOHAAAAA|1109556|3305|41901|2452318|2452288|Mr.|Ronald|Shelly|N|14|3|1924|BRAZIL||Ronald.Shelly@3.edu|2452348| +2028|AAAAAAAAMOHAAAAA|1400816|3091|1766|2450946|2450916|Dr.|Carin|Cortez|N|11|5|1979|SAUDI ARABIA||Carin.Cortez@BfkCt95nRMziz.org|2452284| +2029|AAAAAAAANOHAAAAA|1377964|4842|34167|2450996|2450966|Mrs.|Jeanne|Russ|N|17|4|1924|SAINT LUCIA||Jeanne.Russ@xhG.com|2452352| +2030|AAAAAAAAOOHAAAAA|600545|1854|21516|2449922|2449892|Mr.|Daniel|Serrano|N|20|8|1949|NIGERIA||Daniel.Serrano@vCrRpOg8RA3.com|2452507| +2031|AAAAAAAAPOHAAAAA|1512768|719|35023|2449861|2449831|Dr.|Abraham|Reynolds|N|14|7|1978|TIMOR-LESTE||Abraham.Reynolds@q3IkD.edu|2452477| +2032|AAAAAAAAAPHAAAAA|1191822|2389|48956|2451149|2451119|Sir|Karl|Newton|Y|21|3|1973|NEPAL||Karl.Newton@fPr.org|2452489| +2033|AAAAAAAABPHAAAAA|231079|7158|12037|2449120|2449090|Dr.|Kathy|Norton|N|3|9|1943|LEBANON||Kathy.Norton@Q.edu|2452370| +2034|AAAAAAAACPHAAAAA|15014|6313|28765|2451543|2451513|Sir|Barry|Blue|Y|7|12|1973|PERU||Barry.Blue@T.edu|2452517| +2035|AAAAAAAADPHAAAAA|499281|607|31580|2450131|2450101|Mr.|Timothy|Donovan|Y|19|9|1972|SEYCHELLES||Timothy.Donovan@BoqFfyececgLqn.org|2452372| +2036|AAAAAAAAEPHAAAAA|1177144|1129|8314|2450783|2450753|Miss|Martha|Garcia|Y|22|12|1943|TOGO||Martha.Garcia@JtfiFBYgLMIK.com|2452633| +2037|AAAAAAAAFPHAAAAA||3124|46942||||Gregory||Y|3||1952|||Gregory.Albert@T9rHhOcELKMX.com|2452521| +2038|AAAAAAAAGPHAAAAA|469676|5163|37003|2452138|2452108|Mrs.|Genia|Shaw|Y|24|1|1949|NAURU||Genia.Shaw@3SrSxNd0bz8X4.edu|2452385| +2039|AAAAAAAAHPHAAAAA|1202773|1127|3081|2451725|2451695|Mrs.|Heather|Gavin|Y|18|7|1927|MALAWI||Heather.Gavin@6fdH6YS.edu|2452531| +2040|AAAAAAAAIPHAAAAA|1520119|7007|46648|2449770|2449740|Mr.|Kevin|Lopez|Y|14|2|1970|MALAYSIA||Kevin.Lopez@YxFmvB.com|2452619| +2041|AAAAAAAAJPHAAAAA|1829844|1728|35565|2449714|2449684|Sir|Howard|Graves|N|11|6|1945|KENYA||Howard.Graves@Y.org|2452580| +2042|AAAAAAAAKPHAAAAA|1884780|3595|6410|2452300|2452270|Dr.|Howard|Woods|N|9|9|1978|NICARAGUA||Howard.Woods@UNV.edu|2452331| +2043|AAAAAAAALPHAAAAA|1225565|1646|4527|2451983|2451953|Mrs.|Anthony|Wagner|Y|22|3|1980|MARTINIQUE||Anthony.Wagner@JcjRA4VsF16LmO.org|2452561| +2044|AAAAAAAAMPHAAAAA|1033915|1269|48223|2450887|2450857|Mrs.|Karen|Lee|N|7|5|1991|NETHERLANDS ANTILLES||Karen.Lee@ZX.edu|2452559| +2045|AAAAAAAANPHAAAAA|965755|584|13766|2449283|2449253|Mr.|Mike|Jones|Y|8|7|1951|SAUDI ARABIA||Mike.Jones@U4GS5CY56.com|2452488| +2046|AAAAAAAAOPHAAAAA|1541708|2851|44794|2451975|2451945|Miss|Mildred|Jacobs|Y|7|11|1931|LITHUANIA||Mildred.Jacobs@0UTEKqu5sL.org|2452415| +2047|AAAAAAAAPPHAAAAA|199529|5508|14334||||Leonard|Saenz||29|8|1944||||| +2048|AAAAAAAAAAIAAAAA|1771697|2721|43698|2452596|2452566|Mr.|Dean|Baum|N|21|4|1965|NIGER||Dean.Baum@b3JgA5uHYVb1fpL.org|2452505| +2049|AAAAAAAABAIAAAAA|881514|4705|1896|2451285|2451255|Dr.|Vernon|Neal|Y|5|4|1960|EGYPT||Vernon.Neal@Nl6ZuMEhVy9y4K.com|2452424| +2050|AAAAAAAACAIAAAAA|224399|6645|37691|2452207|2452177|Sir|James|Gonzalez|Y|2|7|1948|MONACO||James.Gonzalez@GK680xHegKMLO.com|2452502| +2051|AAAAAAAADAIAAAAA|||47300|||Sir|Terrence|||28|4|||||2452325| +2052|AAAAAAAAEAIAAAAA|1394452|6940|8581|2449947|2449917|Sir|Douglas|Labelle|N|8|12|1937|PORTUGAL||Douglas.Labelle@7UmGPoZQZl.org|2452500| +2053|AAAAAAAAFAIAAAAA|1467030|6327|824|2451415|2451385|Mr.|James|Novak|N|30|3|1946|UGANDA||James.Novak@BgAVghsj4V.com|2452433| +2054|AAAAAAAAGAIAAAAA|373162|1022|40529|2449715|2449685|Mrs.|Ann|Marshall|N|30|4|1963|MAYOTTE||Ann.Marshall@K2uUX.com|2452364| +2055|AAAAAAAAHAIAAAAA|1652296|4958|36820|2450112|2450082|Mrs.|Rosa|Gilliam|||11|1926|MAYOTTE|||2452559| +2056|AAAAAAAAIAIAAAAA|1655902||30912|||Ms.||||9|2|1945||||2452519| +2057|AAAAAAAAJAIAAAAA|1795667|3309|46598|2449170|2449140|Mr.|Robert|Smith|N|3|6|1965|NORWAY||Robert.Smith@FeMRaR62BiZJj.edu|2452395| +2058|AAAAAAAAKAIAAAAA|1639764|3175|38986|2452258|2452228|Ms.|Catherine|Sykes|Y|26|6|1973|BOLIVIA||Catherine.Sykes@utHANxEtOhCThs.edu|2452338| +2059|AAAAAAAALAIAAAAA|1647248|3674|39700|2450493|2450463|Dr.|Charles|Brown|N|6|6|1945|GHANA||Charles.Brown@uXR9kP8sYcpnq.com|2452379| +2060|AAAAAAAAMAIAAAAA|70873|6699|46572|2450312|2450282|Sir|Gary|Rivera|N|25|11|1982|PITCAIRN||Gary.Rivera@EYd5cSgiE.edu|2452572| +2061|AAAAAAAANAIAAAAA|1361353|4604|34702|2452464|2452434|Dr.|Alvaro|Greene|N|16|3|1966|MALAYSIA||Alvaro.Greene@sfXBot.com|2452446| +2062|AAAAAAAAOAIAAAAA|446398|4556|15867|2450852|2450822|Dr.|Patrick|Brown|N|12|2|1932|SAINT HELENA||Patrick.Brown@Dv3nx.edu|2452422| +2063|AAAAAAAAPAIAAAAA|1132798|5911|7471|2449944|2449914|Mrs.|Brenda|Franklin|N|20|1|1937|LUXEMBOURG||Brenda.Franklin@cQZlUOF0s9m.edu|2452528| +2064|AAAAAAAAABIAAAAA|1519547|301|22208|2451673|2451643|Dr.|Bruce|Nelson|N|30|5|1943|GREECE||Bruce.Nelson@z1sEIVM5dtXB.org|2452609| +2065|AAAAAAAABBIAAAAA|4355|4904|30300|2449721|2449691|Mr.|Pablo|Young|Y|2|2|1986|ECUADOR||Pablo.Young@J8Qn5e4tgUBH.org|2452531| +2066|AAAAAAAACBIAAAAA|573942|3989|17636|2450997|2450967|Dr.|Michael|Maynard|Y|30|5|1936|NICARAGUA||Michael.Maynard@dT3OFSzt.com|2452613| +2067|AAAAAAAADBIAAAAA|318663|66|34622|2451402|2451372|Dr.|Alfred|Bartlett|Y|6|7|1933|BANGLADESH||Alfred.Bartlett@PS7G6rFay2.com|2452354| +2068|AAAAAAAAEBIAAAAA|1502099|3429|40712|2449797|2449767|Ms.|Linda|Morris|Y|14|5|1985|GAMBIA||Linda.Morris@Y8F8YXjsIGHeTKcR.org|2452504| +2069|AAAAAAAAFBIAAAAA|||15103|2449578|2449548||Jackson|Phillips|N|23|6|1970|GEORGIA|||| +2070|AAAAAAAAGBIAAAAA|386126|2957|39341|2450794|2450764|Mr.|Gerald|Valdez|N|8|4|1954|UNITED KINGDOM||Gerald.Valdez@ROhZza.edu|2452348| +2071|AAAAAAAAHBIAAAAA|486320|5349|39178|2452624|2452594|Dr.|Harvey|Mitchell|Y|5|7|1934|DJIBOUTI||Harvey.Mitchell@Ra2XvPqH0PHR.com|2452455| +2072|AAAAAAAAIBIAAAAA|1197726|4101|19723|2451519|2451489|Miss|Eunice|Rose|Y|21|6|1989|TOKELAU||Eunice.Rose@HDHl83F7BmYCcM.edu|2452299| +2073|AAAAAAAAJBIAAAAA|571185|2920|37408|2452515|2452485|Dr.|Jessenia|Stevens|N|10|2|1987|EGYPT||Jessenia.Stevens@C65XrxiZC9.org|2452384| +2074|AAAAAAAAKBIAAAAA|572235|797|6512|2451685|2451655|Miss|Louise|Clarke|Y|2|8|1933|LITHUANIA||Louise.Clarke@q1Z4XDqgdq.edu|2452475| +2075|AAAAAAAALBIAAAAA|1009142|6080|39018|2451190|2451160|Dr.|Michiko|Spain|Y|8|7|1971|UKRAINE||Michiko.Spain@Porbm2lhs2s.com|2452557| +2076|AAAAAAAAMBIAAAAA|||7165||||||Y|20|7|1953|||Arthur.Meyer@Vix8HvAnm09Z.com|| +2077|AAAAAAAANBIAAAAA|1651171|5848|21278|2449812|2449782|Ms.|Ruth|Sloan|N|1|5|1962|VENEZUELA||Ruth.Sloan@xjZ0zT7Y.com|2452396| +2078|AAAAAAAAOBIAAAAA|643355|2271|47140|2449268|2449238|Mrs.|Tiffany|Howard|Y|28|11|1951|YEMEN||Tiffany.Howard@4JffbxjMzumI6hUcKmK.org|2452389| +2079|AAAAAAAAPBIAAAAA||856|24122|2449046|2449016||||N||6|1949|WALLIS AND FUTUNA|||| +2080|AAAAAAAAACIAAAAA|644457|1344|49431|2451527|2451497|Dr.|Timothy|Patterson|Y|6|4|1956|WESTERN SAHARA||Timothy.Patterson@siLbx3.edu|2452540| +2081|AAAAAAAABCIAAAAA|217778|1300|38469|2450680|2450650|Sir|Charles|Shea|N|23|2|1983|ITALY||Charles.Shea@04JBRPjg5s.edu|2452596| +2082|AAAAAAAACCIAAAAA|||9035|2449921|2449891|Dr.||Griffin|N|10|2||||Linda.Griffin@LIZ1Hgud4.org|| +2083|AAAAAAAADCIAAAAA|947369|126|28883|2449795|2449765|Dr.|Elanor|Mckenney|Y|20|4|1988|CHRISTMAS ISLAND||Elanor.Mckenney@SyDoublL4beN4.edu|2452363| +2084|AAAAAAAAECIAAAAA|1289178|504|30630|2449603|2449573|Mr.|Joseph|Martin|N|15|4|1989|GREENLAND||Joseph.Martin@507eCZ.org|2452416| +2085|AAAAAAAAFCIAAAAA|76903|730|14910||2450457|||Rosario|Y||8|||||2452377| +2086|AAAAAAAAGCIAAAAA|143521|116|43158|2452568|2452538|Dr.|Helen|Means|Y|22|2|1992|CHINA||Helen.Means@yN.com|2452332| +2087|AAAAAAAAHCIAAAAA||394|651|2451956|2451926|Mr.|Lucas|Ellis||9|11|1946|||Lucas.Ellis@LdASS.com|| +2088|AAAAAAAAICIAAAAA|1394090|1152|44614|2451478|2451448|Mrs.|Glenda|Sager|Y|9|10|1946|PAPUA NEW GUINEA||Glenda.Sager@0Xp1icZMxzx.org|2452435| +2089|AAAAAAAAJCIAAAAA|708728|3151|49621|2449255|2449225|Miss|Martha|Sullivan|Y|21|2|1992|AZERBAIJAN||Martha.Sullivan@ASfaNjN70vNlTk0PH1.com|2452344| +2090|AAAAAAAAKCIAAAAA|1046119|3885|24498|2449647|2449617|Sir|Milton|Cardwell|Y|18|8|1985|JERSEY||Milton.Cardwell@kd9f1m.edu|2452363| +2091|AAAAAAAALCIAAAAA|||32027|2450147|2450117|Mr.|Marcus||||6||AZERBAIJAN||Marcus.Sherman@eCZnTo5t85tJzd.com|| +2092|AAAAAAAAMCIAAAAA|596251|1248|23845|2449600|2449570|Dr.|Kristy|Hendrix|N|9|12|1992|PHILIPPINES||Kristy.Hendrix@ES.org|2452350| +2093|AAAAAAAANCIAAAAA|1151085|6104|38723|2450123|2450093|Mrs.|Fern|Williams|N|9|9|1945|AMERICAN SAMOA||Fern.Williams@au0O10aKUd5.com|2452351| +2094|AAAAAAAAOCIAAAAA|537866|4542|23171|2451069|2451039|Dr.|Katherine|Jones|N|15|2|1969|PITCAIRN||Katherine.Jones@rqnTMjfXlznITY.edu|2452594| +2095|AAAAAAAAPCIAAAAA|1414915|4058|32048|2451206|2451176|Mr.|Walter|Gibson|N|30|1|1956|POLAND||Walter.Gibson@8.org|2452538| +2096|AAAAAAAAADIAAAAA|249880|3113|37499|2452406|2452376|Dr.|Joseph|Wright|Y|11|9|1933|EQUATORIAL GUINEA||Joseph.Wright@KL6sLBm2.edu|2452510| +2097|AAAAAAAABDIAAAAA|785617|3635|9253|2451822|2451792|Mr.|Robert|Cash|N|30|10|1942|RUSSIAN FEDERATION||Robert.Cash@OpObo16H9m7h.edu|2452355| +2098|AAAAAAAACDIAAAAA|541159|7087|568|2452640|2452610|Dr.|Raymond|Perez|N|7|11|1980|CROATIA||Raymond.Perez@e44x0LRMF.org|2452296| +2099|AAAAAAAADDIAAAAA|285977|4452|17327|2449446|2449416|Sir|Jonathan|Ruiz|N|30|5|1971|ECUADOR||Jonathan.Ruiz@6hZue2IITHSkfxF.com|2452558| +2100|AAAAAAAAEDIAAAAA|1399051|2111|9971|2449863|2449833|Dr.|Thelma|Jones|N|4|4|1934|NIUE||Thelma.Jones@XS890hOafVTGVja.org|2452426| +2101|AAAAAAAAFDIAAAAA|1250594|1743|29292|2450443|2450413|Sir|Kevin|Creamer|N|9|3|1977|BOUVET ISLAND||Kevin.Creamer@Rgn.com|2452617| +2102|AAAAAAAAGDIAAAAA|1584214|737|1260|2450098|2450068|Dr.|Sonja|Brown|Y|17|4|1991|CAPE VERDE||Sonja.Brown@zJ.com|2452335| +2103|AAAAAAAAHDIAAAAA|1134173|2938|45256|2450058|2450028|Mr.|James|Matlock|Y|16|5|1953|SERBIA||James.Matlock@UEM4QV1281UO.com|2452290| +2104|AAAAAAAAIDIAAAAA|205835|1611|37787|2449616|2449586|Dr.|Clark|Weller|Y|12|3|1978|GERMANY||Clark.Weller@osA5l.edu|2452507| +2105|AAAAAAAAJDIAAAAA|178284|1078|20625|2451639|2451609|Dr.|Peggy|Martin|Y|19|11|1963|OMAN||Peggy.Martin@2pt3E2H3uDeBzvTC.edu|2452405| +2106|AAAAAAAAKDIAAAAA|982604|5059|49514|2451257|2451227|Dr.|Kenneth|Sparks|Y|6|4|1926|BOLIVIA||Kenneth.Sparks@RA1ScfPs8.com|2452483| +2107|AAAAAAAALDIAAAAA|498630|6259|6288|2450279|2450249|Mr.|Stuart|Miles|Y|31|12|1991|NIUE||Stuart.Miles@4SiMBlPg8CQ1yUCFV.org|2452591| +2108|AAAAAAAAMDIAAAAA|240222|3723|21577|2452325|2452295|Mr.|Robert|Kohler|N|3|8|1939|KAZAKHSTAN||Robert.Kohler@CQf7yLRPGe.edu|2452438| +2109|AAAAAAAANDIAAAAA|1462329|2784|23992|2449139|2449109|Miss|Rachel|Gunn|N|8|8|1944|EQUATORIAL GUINEA||Rachel.Gunn@VeAp99kjneFYU.com|2452305| +2110|AAAAAAAAODIAAAAA|1883541|2745|42749|2449894|2449864|Mrs.|Loretta|Floyd|N|26|5|1972|ESTONIA||Loretta.Floyd@T.edu|2452329| +2111|AAAAAAAAPDIAAAAA|1096319|7044|7722|2452383|2452353|Sir|Richard|Vigil|N|22|12|1954|MALTA||Richard.Vigil@c3av8aHkknlzyj721z.com|2452573| +2112|AAAAAAAAAEIAAAAA|109328|6557|47339|2450575|2450545|Ms.|Helen|Crow|Y|22|3|1984|BELARUS||Helen.Crow@ie4oIc7pSqq0F5.edu|2452298| +2113|AAAAAAAABEIAAAAA|1912065|6868|32424|2449947|2449917|Miss|Margarete|Katz|Y|15|11|1950|GUINEA-BISSAU||Margarete.Katz@7sH1er64bLOVbcvPVj.com|2452430| +2114|AAAAAAAACEIAAAAA|741859|4225|23744|2451136|2451106|Mrs.|Selina|Youngblood|Y|28|11|1986|FAROE ISLANDS||Selina.Youngblood@2sSvBVa9.org|2452489| +2115|AAAAAAAADEIAAAAA|1060326|5670|31814|2452344|2452314|Dr.|Nicholas|Waters|N|4|3|1954|LIECHTENSTEIN||Nicholas.Waters@6CcHnvxPRX19.edu|2452388| +2116|AAAAAAAAEEIAAAAA|1132021|5788|6002|2452230|2452200|Mr.|Gary|Albritton|N|2|12|1962|BURKINA FASO||Gary.Albritton@IYln2zZ14Fjbo.com|2452536| +2117|AAAAAAAAFEIAAAAA|1759263|6563|26263|2449641|2449611|Mrs.|Barbara|Fuentes|N|19|6|1964|BAHRAIN||Barbara.Fuentes@Ol298N2aLZo.org|2452295| +2118|AAAAAAAAGEIAAAAA|155909|5033|47404|2451437|2451407|Sir|Kenneth|Fleming|N|30|8|1992|JAMAICA||Kenneth.Fleming@srOtBRDcgR.edu|2452545| +2119|AAAAAAAAHEIAAAAA|1796328|4412|8849|2450635|2450605|Sir|Alvin|Marra|N|19|10|1936|DOMINICA||Alvin.Marra@M.edu|2452360| +2120|AAAAAAAAIEIAAAAA|896963|3854|30792|2451136|2451106|Miss|Margaret|Scott|Y|15|8|1955|MONGOLIA||Margaret.Scott@1ZbhJTEbZK.com|2452490| +2121|AAAAAAAAJEIAAAAA|1373451|3368|37638|2451211|2451181|Mr.|Eddie|Gillum|N|26|3|1950|MALAYSIA||Eddie.Gillum@3hnYDogOVg.com|2452348| +2122|AAAAAAAAKEIAAAAA|114538|6514|12843|2451589|2451559|Ms.|Sarah|Otoole|Y|19|3|1968|LIECHTENSTEIN||Sarah.Otoole@iSHkLG1qsJqH4BL3.edu|2452432| +2123|AAAAAAAALEIAAAAA|286392|2949|25906|2452252|2452222|Ms.|Diana|Buxton|N|14|9|1929|UNITED ARAB EMIRATES||Diana.Buxton@ZuERvSH.com|2452584| +2124|AAAAAAAAMEIAAAAA|852769|84|42552|2451891|2451861|Ms.|Judy|Goff|Y|13|6|1939|ARGENTINA||Judy.Goff@uA3TSLRQna.org|2452605| +2125|AAAAAAAANEIAAAAA|895194|4228|9309|2450764|2450734|Ms.|Renee|Hicks|N|10|1|1987|CHRISTMAS ISLAND||Renee.Hicks@KVsov9glvyDZz.com|2452377| +2126|AAAAAAAAOEIAAAAA||2426|25249|2451248|||Andre|Ivey|||||LITHUANIA||Andre.Ivey@z8aJGneXIHMqcUP.org|2452562| +2127|AAAAAAAAPEIAAAAA|386325|135|23145|2449206|2449176|Sir|Russ|Williams|N|17|6|1977|ITALY||Russ.Williams@PKo3SDvpIXf.edu|2452392| +2128|AAAAAAAAAFIAAAAA|1448830|5272|16114|2451759|2451729|Mrs.|Sherly|Tran|Y|8|9|1941|SAINT HELENA||Sherly.Tran@G2v2gGkP3.org|2452388| +2129|AAAAAAAABFIAAAAA|398836|4139|36595|2451325|2451295|Mr.|Floyd|Stanley|N|19|8|1947|JORDAN||Floyd.Stanley@88.edu|2452588| +2130|AAAAAAAACFIAAAAA|781076|1703|25483|2451566|2451536||Victoria||||12||||Victoria.Woodson@CARzTU0539N.edu|2452378| +2131|AAAAAAAADFIAAAAA|1894938||20812||2450190||Kenneth|||18|8||ANGUILLA||Kenneth.Messer@MU8E.edu|2452428| +2132|AAAAAAAAEFIAAAAA|538064|1213|715|2451010|2450980|Dr.|Daniel|Hills|Y|29|8|1975|TURKMENISTAN||Daniel.Hills@u3MOAM0riFO8yFMoyuI.com|2452372| +2133|AAAAAAAAFFIAAAAA|158070|1667|30817|2452601|2452571|Dr.|Leon|Battles|Y|17|1|1958|ARGENTINA||Leon.Battles@mnSak1ynUyeNo.org|2452556| +2134|AAAAAAAAGFIAAAAA|1317253|2711|43816|2451439|2451409|Mr.|James|Bravo|N|19|5|1987|CAPE VERDE||James.Bravo@XByPHVTRdD.com|2452418| +2135|AAAAAAAAHFIAAAAA|987946|3804|23262|2451442|2451412|Dr.|Donald|Mack|Y|17|7|1928|CZECH REPUBLIC||Donald.Mack@RNG1lORrcMpV3B0u.org|2452448| +2136|AAAAAAAAIFIAAAAA|241705|5209|20760|2449807|2449777|Dr.|Debbie|Peterson|N|25|3|1985|PERU||Debbie.Peterson@acU.org|2452567| +2137|AAAAAAAAJFIAAAAA|120254|2186|44518|2450564|2450534|Miss|Wanda|Williams|Y|10|3|1957|HONDURAS||Wanda.Williams@gtknSgGQCcIRt.org|2452404| +2138|AAAAAAAAKFIAAAAA|897311|2939|33320|2451924|2451894|Miss|Andrew|Hall|N|15|2|1940|BAHAMAS||Andrew.Hall@cRnvo.org|2452527| +2139|AAAAAAAALFIAAAAA|1396685|5731|13224|2451478|2451448|Mrs.|Angelica|Latimer|Y|19|5|1929|MAURITIUS||Angelica.Latimer@Dv7.org|2452343| +2140|AAAAAAAAMFIAAAAA|691032|5238|6244|2452022|2451992|Dr.|Lucille|Harris|Y|5|6|1961|MONACO||Lucille.Harris@O6zXMOe.com|2452530| +2141|AAAAAAAANFIAAAAA|1558438|4895|42505|2450998|2450968|Sir|Christopher|Johnson|Y|11|2|1956|YEMEN||Christopher.Johnson@ofx0uJz6Vx.org|2452421| +2142|AAAAAAAAOFIAAAAA|1506811|2217|20423|2451192|2451162|Sir|Raymond|Johnson|N|28|1|1969|SINGAPORE||Raymond.Johnson@i7lJBDNTlbzP9Dv.org|2452558| +2143|AAAAAAAAPFIAAAAA|1899267|4621|728|2450476|2450446|Miss|Nikki|Fisher|N|5|2|1970|ISRAEL||Nikki.Fisher@q.com|2452377| +2144|AAAAAAAAAGIAAAAA|1312424|4632|12840|2450137|2450107|Sir|David|Bryant|N|12|12|1971|UZBEKISTAN||David.Bryant@OB3mlhsuDdJ.org|2452317| +2145|AAAAAAAABGIAAAAA|1383009|1446|330|2449693|2449663|Dr.|Scott|Quinn|N|21|10|1956|ANTIGUA AND BARBUDA||Scott.Quinn@pL9ALstgOzMUI.edu|2452301| +2146|AAAAAAAACGIAAAAA|591993|3062|20931|2449432|2449402|Mr.|Robert|Turner|Y|18|8|1961|CAMBODIA||Robert.Turner@qJCcAKSH1.org|2452507| +2147|AAAAAAAADGIAAAAA|903183|6129|14628|2450668|2450638|Mrs.|Nakisha|Zeller|Y|3|1|1924|ARMENIA||Nakisha.Zeller@EgY.org|2452463| +2148|AAAAAAAAEGIAAAAA|1893964|3247|11283|2449324|2449294|Mr.|Edward|Huskey|N|20|5|1967|NEPAL||Edward.Huskey@YmZLtIQ.edu|2452403| +2149|AAAAAAAAFGIAAAAA|1791173|494|43862|2450203|2450173|Dr.|Julio|Brooks|N|12|2|1970|ANGUILLA||Julio.Brooks@L2DqdaAdBbts.com|2452283| +2150|AAAAAAAAGGIAAAAA|226539|6669|10799|2451000|2450970|Mr.|John|Hobbs|Y|16|1|1957|MONACO||John.Hobbs@kLA67f2LH9JtT.com|2452431| +2151|AAAAAAAAHGIAAAAA|1900538|2572|29406|2452674|2452644|Dr.|Edward|Davis|Y|17|11|1963|DJIBOUTI||Edward.Davis@K2AVs8VuBAGizo4.org|2452612| +2152|AAAAAAAAIGIAAAAA|596210|5911|21965|2451831|2451801|Sir|David|Rodriguez|N|13|6|1935|ARGENTINA||David.Rodriguez@tK60IHGrEeo1QZ6.edu|2452559| +2153|AAAAAAAAJGIAAAAA|150262|3788|39714|2452183|2452153|Ms.|Luisa|Hayes|Y|26|1|1971|TONGA||Luisa.Hayes@nj8ALh4xelH5.org|2452497| +2154|AAAAAAAAKGIAAAAA|1899501|152|29203|2450358|2450328|Dr.|Thomas|Peterson|Y|14|1|1989|SLOVAKIA||Thomas.Peterson@6SYUdfASD1.org|2452512| +2155|AAAAAAAALGIAAAAA|984092|4746|31800|2451328|2451298|Mrs.|Kayla|Stoner|Y|22|3|1991|SWITZERLAND||Kayla.Stoner@SiRihcKQJCecMLzm.edu|2452415| +2156|AAAAAAAAMGIAAAAA|514468|1402|13536|2450902|2450872|Ms.|Diane|Hudson|N|12|9|1970|FIJI||Diane.Hudson@RzB.org|2452349| +2157|AAAAAAAANGIAAAAA|338276|6495|42267|2450375|2450345|Dr.|Scott|Massey|Y|19|4|1978|NIUE||Scott.Massey@1kRRzN.com|2452527| +2158|AAAAAAAAOGIAAAAA|777188|3657|24573|2449391|2449361|Mr.|Michal|Webb|N|29|1|1932|MAYOTTE||Michal.Webb@tJ6Hx.org|2452505| +2159|AAAAAAAAPGIAAAAA|859901|6530|10406|2451458|2451428|Dr.|Joe|Smith|Y|1|7|1976|VANUATU||Joe.Smith@ULYY6JDA7f.org|2452331| +2160|AAAAAAAAAHIAAAAA|175855|4601|23747||2451881|||Williams||10|||||Lucille.Williams@EDjKdZKMbbD6BaCX.edu|| +2161|AAAAAAAABHIAAAAA|1792181|5088|14348|2451638|2451608|Miss|Karen|Trujillo|N|3|7|1953|AFGHANISTAN||Karen.Trujillo@BMEzdbRaAA.edu|2452361| +2162|AAAAAAAACHIAAAAA|1596604|2866|37175|2451345|2451315|Mrs.|Chandra|Evans|Y|22|1|1949|GABON||Chandra.Evans@b11R8ekbrhLIs.org|2452477| +2163|AAAAAAAADHIAAAAA|1188528|4087|8046|2451800|2451770|Dr.|Blake|Oconnor|N|22|12|1953|NIUE||Blake.Oconnor@cDfJ3Dh.com|2452482| +2164|AAAAAAAAEHIAAAAA|214040|1513|27433|2451703|2451673|Mr.|Charles|Dehart|Y|20|3|1925|JAPAN||Charles.Dehart@7iuixUhXeY.com|2452416| +2165|AAAAAAAAFHIAAAAA|176757|517|42724|2451090|2451060|Mr.|Randall|Johnson|N|11|2|1988|FRENCH GUIANA||Randall.Johnson@KjtBKjtxMpkk7U.org|2452632| +2166|AAAAAAAAGHIAAAAA|334395|6875|37002|2451477|2451447|Dr.|Jack|Hardy|Y|9|4|1932|CUBA||Jack.Hardy@4m9bMc2mJEc2gJHkqay.com|2452411| +2167|AAAAAAAAHHIAAAAA|217216|6071|32926|2449426|2449396|Sir|Derek|Larsen|N|8|3|1941|IRAQ||Derek.Larsen@1akt9pQ3SKQZM.org|2452405| +2168|AAAAAAAAIHIAAAAA|||2012|2450360||Dr.|Brian|Paul|Y|26|||NETHERLANDS||Brian.Paul@feH65Ue0P.edu|2452418| +2169|AAAAAAAAJHIAAAAA|1310375|5024|31320|2451885|2451855|Mrs.|Teresa|Livingston|N|29|11|1936|DOMINICA||Teresa.Livingston@mRox.org|2452567| +2170|AAAAAAAAKHIAAAAA|1408589|1405|15868|2450791|2450761|Mrs.|Kali|Cox|N|9|12|1933|NEPAL||Kali.Cox@Nmc6RMGUZ6.edu|2452572| +2171|AAAAAAAALHIAAAAA|226606|4597|15341|2450889|2450859|Ms.|Doris|Cook|Y|18|3|1969|PANAMA||Doris.Cook@Eg.com|2452448| +2172|AAAAAAAAMHIAAAAA|44895|3548|37354|2449409|2449379|Dr.|Ileana|Cameron|Y|15|1|1933|HONG KONG||Ileana.Cameron@YXrRMA4DEZ.org|2452352| +2173|AAAAAAAANHIAAAAA|888088|6829|41970|2449190|2449160|Sir|Ronald|Mayfield|N|19|12|1929|FRANCE||Ronald.Mayfield@QuHPAIum4.com|2452454| +2174|AAAAAAAAOHIAAAAA|413809|2612|13768|2450216|2450186|Dr.|Timothy|Maes|Y|29|10|1936|TIMOR-LESTE||Timothy.Maes@m7pTYUZvpU3.com|2452518| +2175|AAAAAAAAPHIAAAAA|1909023|6977|31331|2452677|2452647|Dr.|Cassie|Gardner|N|26|2|1944|BERMUDA||Cassie.Gardner@ausAZLVMpttk.org|2452585| +2176|AAAAAAAAAIIAAAAA|272251|1792|23733|2451489|2451459|Mrs.|Leona|Gregg|N|27|6|1947|CANADA||Leona.Gregg@BgERZl1.org|2452429| +2177|AAAAAAAABIIAAAAA|1825989|2408|37627|2452551|2452521|Mrs.|Ruby|Ponder|Y|30|8|1947|MALAYSIA||Ruby.Ponder@i6mP.org|2452645| +2178|AAAAAAAACIIAAAAA|1809300|747|38994|2452656|2452626|Mrs.|Sallie|Wilder|N|13|12|1936|IRAQ||Sallie.Wilder@g2oUsI.org|2452488| +2179|AAAAAAAADIIAAAAA|930322|6613|14904|2450539|2450509|Sir|Leroy|Kilpatrick|N|24|5|1933|VENEZUELA||Leroy.Kilpatrick@dDA764G8xDk.org|2452398| +2180|AAAAAAAAEIIAAAAA|397342|4013|45437|2449057|2449027|Mrs.|Michele|Taylor|N|5|10|1990|GERMANY||Michele.Taylor@CoZj.edu|2452363| +2181|AAAAAAAAFIIAAAAA|807798|920|4395|2452160|2452130|Mrs.|Debra|Lawrence|Y|30|6|1957|ICELAND||Debra.Lawrence@HM8.com|2452378| +2182|AAAAAAAAGIIAAAAA|264934|706|49678|2451512|2451482|Dr.|Susan|Sandoval|Y|24|12|1960|BHUTAN||Susan.Sandoval@xCxq26Dx4Cxs.com|2452513| +2183|AAAAAAAAHIIAAAAA|||48490|2450586||||Price||1|8|1925|||Elizabeth.Price@a9bMy1GfO.org|2452595| +2184|AAAAAAAAIIIAAAAA|1664266|4156|13599|2451210|2451180|Miss|Lois|Wells|N|25|8|1955|SOUTH AFRICA||Lois.Wells@9.com|2452313| +2185|AAAAAAAAJIIAAAAA|887466|3829|14192|2451809|2451779|Mrs.|Lillia|Mendez|Y|23|9|1937|NEW CALEDONIA||Lillia.Mendez@94cg8U9LiiVt.edu|2452344| +2186|AAAAAAAAKIIAAAAA|271318|6143|32215|2449277|2449247|Sir|Gary|Clark|N|3|2|1925|RUSSIAN FEDERATION||Gary.Clark@Ki.org|2452380| +2187|AAAAAAAALIIAAAAA|427968|40|45359|2450340|2450310|Sir|Tracy|Ingram|N|23|5|1978|TUNISIA||Tracy.Ingram@PY0.edu|2452646| +2188|AAAAAAAAMIIAAAAA|1409889|4161|28433|2450074|2450044|Ms.|Linda|Martinez|Y|14|11|1963|SEYCHELLES||Linda.Martinez@CH8dk3yHh3.com|2452485| +2189|AAAAAAAANIIAAAAA|1135464|1495|2726|2452496|2452466|Dr.|Jonathan|Moore|N|26|10|1927|SUDAN||Jonathan.Moore@LbTZnd9aPJGEFk04.edu|2452291| +2190|AAAAAAAAOIIAAAAA|1552947|4322|544|2451436|2451406|Miss|Melanie|Scott|Y|27|11|1971|CAPE VERDE||Melanie.Scott@8ZPK46AxcLJT5bVAhN0.com|2452612| +2191|AAAAAAAAPIIAAAAA|||48720||2451314|Ms.||||5|10||PANAMA|||| +2192|AAAAAAAAAJIAAAAA|606540|4428|11963|2451377|2451347|Dr.|John|Rodriguez|N|1|2|1981|MAURITANIA||John.Rodriguez@Dmkiq730K.com|2452594| +2193|AAAAAAAABJIAAAAA|92306|3630|9307|2450166|2450136|Sir|Eddie|Hale|Y|21|9|1942|ESTONIA||Eddie.Hale@zAAe9.com|2452313| +2194|AAAAAAAACJIAAAAA|1812988|568|40148|2450478|2450448|Ms.|Krista|Carter|N|24|8|1955|SAN MARINO||Krista.Carter@laL.edu|2452625| +2195|AAAAAAAADJIAAAAA|1499674|6838|34402|2450259|2450229|Miss|Amanda|Alexander|Y|21|5|1989|SUDAN||Amanda.Alexander@S7OuMnQEcmAi.org|2452601| +2196|AAAAAAAAEJIAAAAA|279633|2296|34112|2452193|2452163|Mr.|James|Penn|Y|5|5|1990|BURKINA FASO||James.Penn@8tThF2U.org|2452383| +2197|AAAAAAAAFJIAAAAA|817524|3505|6239|2451104|2451074|Dr.|Jeffrey|Bertram|Y|6|4|1924|SLOVAKIA||Jeffrey.Bertram@6xd908anxSU.edu|2452302| +2198|AAAAAAAAGJIAAAAA|1265912|2633|40554|2452131|2452101|Dr.|James|Bruce|N|25|3|1943|NEW ZEALAND||James.Bruce@V6R8xPrcM2iIiGa.com|2452553| +2199|AAAAAAAAHJIAAAAA|1664162|6237|23912|2449244|2449214|Dr.|Graham|Reis|Y|3|10|1948|MADAGASCAR||Graham.Reis@zBq3nInHaJ.org|2452462| +2200|AAAAAAAAIJIAAAAA|277731|4979|7474|2449205|2449175|Sir|Chester|Cordova|Y|13|6|1957|GRENADA||Chester.Cordova@yoC.com|2452509| +2201|AAAAAAAAJJIAAAAA|1523869|2566|49310|2449594|2449564|Ms.|Ruth|Harper|N|21|8|1986|AZERBAIJAN||Ruth.Harper@f.edu|2452538| +2202|AAAAAAAAKJIAAAAA|1646310|1122|18781|2450820|2450790|Mr.|Myron|Burr|Y|6|8|1978|MONTENEGRO||Myron.Burr@l5Got.org|2452569| +2203|AAAAAAAALJIAAAAA|273549|4113|6230|2451918|2451888|Dr.|Bonny|Holman|Y|2|2|1951|BAHRAIN||Bonny.Holman@bxPqFdH8Yymk.edu|2452411| +2204|AAAAAAAAMJIAAAAA|1025524|5138|36705|||Dr.||Wilson|N|||1964|||Joe.Wilson@IiYX4iuLJFHbLx5Ns.org|| +2205|AAAAAAAANJIAAAAA|1570979|6920|27261|2450775|2450745|Sir|Robert|Tyler|N|2|2|1962|UZBEKISTAN||Robert.Tyler@SSAPcCV5KRfDIHG7v.com|2452469| +2206|AAAAAAAAOJIAAAAA|472526|875|37224|2449725|2449695|Sir|Troy|Joyner|N|15|3|1967|COMOROS||Troy.Joyner@pAhB5rJqmqTpJ.com|2452613| +2207|AAAAAAAAPJIAAAAA|502753|5673|26640|2452311|2452281|Mrs.|Ellie|Mcafee|Y|5|12|1983|GUERNSEY||Ellie.Mcafee@vaKmTIINNCv.edu|2452551| +2208|AAAAAAAAAKIAAAAA|1721933|4491|28249|2450625|2450595|Mr.|Robert|Walker|Y|13|11|1931|GAMBIA||Robert.Walker@VP8aeydDIoTrx.com|2452494| +2209|AAAAAAAABKIAAAAA|135189|1864|4892|2451897|2451867|Sir|Charles|Scott|N|16|10|1982|BRAZIL||Charles.Scott@hHeqzHv.org|2452417| +2210|AAAAAAAACKIAAAAA|1002631|60|5936|2451493|2451463|Dr.|Gus|Daniels|Y|10|1|1959|MYANMAR||Gus.Daniels@tnsylH6LVcoNigpT.edu|2452442| +2211|AAAAAAAADKIAAAAA|||15181|||||Oneal|Y|17|12||||William.Oneal@x9YJrLR.edu|2452415| +2212|AAAAAAAAEKIAAAAA|1199630|1191|37552|2452002|2451972|Miss|Minnie|Arnold|Y|2|9|1952|TAJIKISTAN||Minnie.Arnold@jq.com|2452512| +2213|AAAAAAAAFKIAAAAA|1614787|3734|2681|2451651|2451621|Mr.|John|Jacoby|Y|26|4|1938|MALAYSIA||John.Jacoby@dxDjqSn.edu|2452469| +2214|AAAAAAAAGKIAAAAA|449807|5535|14063|2450280|2450250|Dr.|Cindy|Gonzales|Y|9|11|1967|IRAQ||Cindy.Gonzales@aEvAZBt253QD.com|2452508| +2215|AAAAAAAAHKIAAAAA|||22804|2450105|2450075||Janet|||8||1983|IRELAND||Janet.Johnson@V299vIqIS8AnS.edu|| +2216|AAAAAAAAIKIAAAAA|1194102|2552|38649|2452012|2451982|Dr.|Rose|Chalmers|N|2|6|1943|UNITED STATES||Rose.Chalmers@ve.com|2452330| +2217|AAAAAAAAJKIAAAAA|1741401|7145|42042|2449120|2449090|Mr.|Leon|Frederick|Y|20|5|1962|PANAMA||Leon.Frederick@29R2h38Fy.org|2452438| +2218|AAAAAAAAKKIAAAAA|1496829|6108|9834|2449784|2449754|Dr.|Barbara|Gilliam|N|31|1|1941|BRUNEI DARUSSALAM||Barbara.Gilliam@6G84JCH12p.edu|2452320| +2219|AAAAAAAALKIAAAAA|1084436|1351|19066|2451600|2451570|Sir|John|Robbins|Y|10|6|1954|BAHAMAS||John.Robbins@MUMocVq5CilvMFtbihI.com|2452533| +2220|AAAAAAAAMKIAAAAA|329426|2125|38424|2450959|2450929|Sir|John|Cave|N|7|10|1987|NORFOLK ISLAND||John.Cave@yOpN2iiQ49v9l9.edu|2452631| +2221|AAAAAAAANKIAAAAA|144531|1212|29493|2450216|2450186|Miss|Doris|Howell|N|9|11|1930|AZERBAIJAN||Doris.Howell@UUmuKeuocD84.org|2452517| +2222|AAAAAAAAOKIAAAAA|1246524|3609|15094|2449079|2449049|Dr.|Ophelia|Dickerson|N|16|6|1935|SAINT HELENA||Ophelia.Dickerson@K90.org|2452292| +2223|AAAAAAAAPKIAAAAA|1033898|841|3831|2450110|2450080|Miss|Barbara|Thomas|Y|14|10|1975|BENIN||Barbara.Thomas@QUy2jVp3.com|2452288| +2224|AAAAAAAAALIAAAAA|1211931|5079|3380|2451743|2451713|Ms.|Stephanie|Murphy|Y|15|7|1957|GERMANY||Stephanie.Murphy@hBXdkcjPiB4.org|2452382| +2225|AAAAAAAABLIAAAAA|190841|4|27120|2449486|2449456|Mrs.|Merissa|Lanier|Y|13|1|1982|IRAQ||Merissa.Lanier@h.org|2452374| +2226|AAAAAAAACLIAAAAA|1878296||17314|2452234||Sir|Joseph|||8|7|1924|ANGUILLA||Joseph.Lopez@2cLGDA13E.com|2452529| +2227|AAAAAAAADLIAAAAA|88115|4210|48752|2449742|2449712|Ms.|Lynda|Brown|Y|12|6|1936|POLAND||Lynda.Brown@YSKchHztDLhzvtR.org|2452634| +2228|AAAAAAAAELIAAAAA|1142415|6620|38472|2451164|2451134|Miss|Kristen|Ervin|Y|12|5|1951|CANADA||Kristen.Ervin@BPrTBh93Eo9.org|2452587| +2229|AAAAAAAAFLIAAAAA|1699045|1269|1493|2450557|2450527|Mr.|Tim|Young|Y|26|6|1927|GUERNSEY||Tim.Young@QqTH0qjQ.org|2452457| +2230|AAAAAAAAGLIAAAAA|209316|2353|36990|2452034|2452004|Miss|Catherine|Mcclellan|N|20|7|1930|JORDAN||Catherine.Mcclellan@bzLrVg8Om.org|2452422| +2231|AAAAAAAAHLIAAAAA|822859|19|41179|2449751|2449721|Mr.|David|Smith|Y|4|7|1938|NIGER||David.Smith@NrH9En9plJ.edu|2452520| +2232|AAAAAAAAILIAAAAA|1903651|6212|33973|2450730|2450700|Dr.|David|Tanner|Y|30|8|1965|NETHERLANDS ANTILLES||David.Tanner@fSVlGJVf2D.com|2452362| +2233|AAAAAAAAJLIAAAAA|1224224|7167|3663|2452186|2452156|Dr.|Anthony|Kurtz|N|28|7|1982|CHRISTMAS ISLAND||Anthony.Kurtz@A.edu|2452596| +2234|AAAAAAAAKLIAAAAA|978389|138|5134|2451594|2451564|Dr.|Harold|Osborn|Y|3|11|1980|CZECH REPUBLIC||Harold.Osborn@JAMuGsaF5iSBbiGcXC.com|2452582| +2235|AAAAAAAALLIAAAAA|1447178|3219|38813|2449736|2449706|Mrs.|Janice|Wolff|Y|8|1|1951|UGANDA||Janice.Wolff@6z2d8UvoIk.com|2452513| +2236|AAAAAAAAMLIAAAAA|918833|2550|21467|2449121|2449091|Sir|Matthew|Sampson|Y|4|3|1978|NEPAL||Matthew.Sampson@JVuqaumAd7ypitRT.com|2452368| +2237|AAAAAAAANLIAAAAA|663314|109|37657|2450478|2450448|Miss|Sally|Smith|Y|3|9|1930|NIGER||Sally.Smith@n1urXNo95q1CKOf0Qh.org|2452293| +2238|AAAAAAAAOLIAAAAA|173140|6267|49747|2451395|2451365|Dr.|Luis|Sousa|N|5|4|1938|CAMEROON||Luis.Sousa@O8YcxPa8H.edu|2452600| +2239|AAAAAAAAPLIAAAAA|514579|3258|47137|2451543|2451513|Sir|Richard|Hughes|Y|21|12|1949|SLOVAKIA||Richard.Hughes@AvX8CgCmVkOuUZ.edu|2452536| +2240|AAAAAAAAAMIAAAAA|355843|1105|35214|2449363|2449333|Mrs.|Carolyn|Graham|N|3|10|1953|SOMALIA||Carolyn.Graham@AapEP8R.edu|2452517| +2241|AAAAAAAABMIAAAAA|827766|6948|8577|2452113|2452083|Mrs.|Rita|Johnson|N|17|2|1983|UNITED ARAB EMIRATES||Rita.Johnson@iHPQaocdqtjTU.org|2452312| +2242|AAAAAAAACMIAAAAA|160759|7193|46027|2452039|2452009|Ms.|Cynthia|Walker|N|6|4|1971|TAJIKISTAN||Cynthia.Walker@08nR3dMtu0JU.org|2452342| +2243|AAAAAAAADMIAAAAA|30993|804|21368|2452535|2452505|Miss|Petra|Roberts|Y|14|11|1958|NEW CALEDONIA||Petra.Roberts@3MZJuhjBbPTPVBt1G.edu|2452403| +2244|AAAAAAAAEMIAAAAA|1645951|2356|20808|2451194|2451164|Dr.|Danny|Ness|Y|25|10|1955|IRAQ||Danny.Ness@CC8vAp.edu|2452487| +2245|AAAAAAAAFMIAAAAA|430829|396|46763|2451001|2450971|Miss|Carla|Williams|N|17|7|1986|C�TE D'IVOIRE||Carla.Williams@p9gl8QQAZFAhtE.edu|2452566| +2246|AAAAAAAAGMIAAAAA|242244|538|39232|2449682|2449652|Dr.|Evelyn|Johnson|N|1|7|1929|BELIZE||Evelyn.Johnson@nVmFgBHfD0mC3g.com|2452608| +2247|AAAAAAAAHMIAAAAA|775574|5741|17108|2450851|2450821|Ms.|Linda|Wilson|Y|8|6|1951|INDONESIA||Linda.Wilson@yefTlNaje00.com|2452426| +2248|AAAAAAAAIMIAAAAA|660171|6498|1578|2451738|2451708|Mr.|Peter|Jones||22|1||ANGOLA|||2452424| +2249|AAAAAAAAJMIAAAAA|1105118|6491|16311|2452141|2452111|Dr.|Cindy|Smith|Y|4|11|1963|HONDURAS||Cindy.Smith@V.edu|2452567| +2250|AAAAAAAAKMIAAAAA|1181067|2906|15545|2451479|2451449|Ms.|Barbara|Neff|Y|16|2|1977|LUXEMBOURG||Barbara.Neff@ia4Gfn.org|2452397| +2251|AAAAAAAALMIAAAAA|698324|4675|1426|2452246|2452216|Mr.|Lionel|Watkins|N|29|9|1948|MAYOTTE||Lionel.Watkins@OR.org|2452548| +2252|AAAAAAAAMMIAAAAA|414306|4217|40575|2450879|2450849|Dr.|Joshua|Justice|Y|15|12|1978|MYANMAR||Joshua.Justice@KVQvy0gKpjs.edu|2452325| +2253|AAAAAAAANMIAAAAA|||20651||2451963|Ms.||Williams|N|13||1952|MONTSERRAT|||| +2254|AAAAAAAAOMIAAAAA|967308|460|21659|2451621|2451591|Mr.|Paul|Watkins|N|31|5|1963|ETHIOPIA||Paul.Watkins@e.org|2452283| +2255|AAAAAAAAPMIAAAAA|1167906|6496|29439|2450421|2450391|Dr.|Julie|Harris|N|15|10|1935|GRENADA||Julie.Harris@HijnckpaGLnq2P.edu|2452357| +2256|AAAAAAAAANIAAAAA|169271|3239|26007|2449485|2449455|Dr.|Jose|Watson|Y|13|2|1948|FIJI||Jose.Watson@aoA.com|2452595| +2257|AAAAAAAABNIAAAAA|1122400|2351|17902|2449968|2449938|Sir|Joseph|Martinez|Y|24|9|1947|GUINEA||Joseph.Martinez@Q9jeJKxQIlmbRa.com|2452378| +2258|AAAAAAAACNIAAAAA|289612|6216|44373|2450378|2450348|Ms.|Kathryn|Brown|Y|24|2|1939|ESTONIA||Kathryn.Brown@N6dZ.edu|2452571| +2259|AAAAAAAADNIAAAAA|1308120|5701|21235|2449847|2449817|Dr.|Tasha|Brown|Y|14|2|1982|ANGOLA||Tasha.Brown@VcfjbKyibLkn.org|2452534| +2260|AAAAAAAAENIAAAAA|1731755|653|16751|2450615|2450585|Mr.|Dean|Zuniga|N|17|10|1972|GREECE||Dean.Zuniga@3eV8.com|2452492| +2261|AAAAAAAAFNIAAAAA|392271|2425|932|2449395|2449365|Dr.|Johnny|Zeller|N|13|8|1982|UGANDA||Johnny.Zeller@7rANX7DMkAvh3.org|2452613| +2262|AAAAAAAAGNIAAAAA|1629362|868|17127|2451648|2451618||Margaret||N|1|9|1986||||2452509| +2263|AAAAAAAAHNIAAAAA|1151827|6521|28768|2450344|2450314|Mrs.|Diane|Hogan|N|3|3|1968|URUGUAY||Diane.Hogan@LpghlBzLkRoV.edu|2452564| +2264|AAAAAAAAINIAAAAA|870676|6304|45179|2451862|2451832|Miss|Clara|Manns|N|2|1|1949|FAROE ISLANDS||Clara.Manns@d3G.edu|2452526| +2265|AAAAAAAAJNIAAAAA|967316|5597|34278|2450013|2449983|Mrs.|Debra|Richards|Y|18|6|1957|SLOVAKIA||Debra.Richards@hJHDsVRbtE.edu|2452512| +2266|AAAAAAAAKNIAAAAA|618566|1787|11361|2450486|2450456|Miss|Isabel|Carney|N|7|2|1990|SLOVENIA||Isabel.Carney@c65X.com|2452522| +2267|AAAAAAAALNIAAAAA|529812|165|20118|2450479|2450449|Ms.|Allen|Homan|Y|3|12|1985|EL SALVADOR||Allen.Homan@6ykGXP.edu|2452290| +2268|AAAAAAAAMNIAAAAA|1047731|255|37132|2451589|2451559|Mrs.|Jennifer|Beardsley|N|3|3|1961|NAURU||Jennifer.Beardsley@cG.com|2452290| +2269|AAAAAAAANNIAAAAA|1334961|4097|786|2449270|2449240|Sir|Walter|Austin|Y|29|7|1986|BOLIVIA||Walter.Austin@DVuxqOZGOYDv8J.com|2452499| +2270|AAAAAAAAONIAAAAA|434574|3206|27085|2449219|||||Y|18||1955|BAHRAIN|||| +2271|AAAAAAAAPNIAAAAA|1223056|6463|5070|2451358|2451328|Mrs.|Joanne|Chance|Y|26|10|1968|PALAU||Joanne.Chance@4mJep4k.edu|2452365| +2272|AAAAAAAAAOIAAAAA|348644|2248|20252|2450188|2450158|Ms.|Rose|Barnes|Y|2|5|1986|UNITED KINGDOM||Rose.Barnes@kkqe1b.edu|2452539| +2273|AAAAAAAABOIAAAAA|221842|4422|40155|2449050|2449020|Mrs.|Katherine|Lee|N|3|5|1957|CUBA||Katherine.Lee@N.com|2452565| +2274|AAAAAAAACOIAAAAA|1365344|5945|10899|2449849|2449819|Mr.|Jon|Aguilar|N|2|3|1986|VENEZUELA||Jon.Aguilar@4kctG1PtVOpk.org|2452494| +2275|AAAAAAAADOIAAAAA|304723|2863|34603|2451887|2451857|Miss|Lora|Towle|Y|2|9|1960|MOROCCO||Lora.Towle@UgQA3gzyGRdijsLF.org|2452448| +2276|AAAAAAAAEOIAAAAA|328283|2545|39654|2450532|2450502|Dr.|Colleen|Moore|N|5|9|1946|SAINT HELENA||Colleen.Moore@11X19hp2qPf.com|2452452| +2277|AAAAAAAAFOIAAAAA|476444|6999|40707|2452643|2452613|Mrs.|Felicia|Nicholas|N|30|12|1965|BOTSWANA||Felicia.Nicholas@y5nok.edu|2452604| +2278|AAAAAAAAGOIAAAAA|1728651|5785|19672|2451393|2451363|Miss|Adam|Elliott|Y|9|1|1977|MACAO||Adam.Elliott@1fNLK7Ff7Orx.edu|2452510| +2279|AAAAAAAAHOIAAAAA|426491|714|8537|2450774|2450744|Ms.|Mattie|Stevens|N|25|3|1942|ARUBA||Mattie.Stevens@8umX9UhrPUBgk36UzRb.org|2452578| +2280|AAAAAAAAIOIAAAAA|740529|108|13687|2451681|2451651|Ms.|Carolyn|Pantoja|N|5|1|1944|GUINEA||Carolyn.Pantoja@M3LuLYgr9uaXS.edu|2452442| +2281|AAAAAAAAJOIAAAAA|887705|5079|49606|2449904|2449874|Dr.|Kurt|Cantu|Y|7|7|1948|NAMIBIA||Kurt.Cantu@k9uQOQyub.org|2452511| +2282|AAAAAAAAKOIAAAAA|132153|28|9856|2451678|2451648|Sir|Bret|Laughlin|N|10|1|1980|UZBEKISTAN||Bret.Laughlin@tqq4Kp0L.com|2452310| +2283|AAAAAAAALOIAAAAA|122370|3875|13827|2449031|2449001|Miss|Etta|Paquette|Y|27|10|1984|ANTARCTICA||Etta.Paquette@OQRK3KgUkxjtO.com|2452483| +2284|AAAAAAAAMOIAAAAA|189325|524|26523|2452029|2451999|Dr.|John|Moody|Y|11|5|1982|PHILIPPINES||John.Moody@epEraETOQ.com|2452627| +2285|AAAAAAAANOIAAAAA|1659007|6700|37193|2451000|2450970|Dr.|Bea|Ryan|Y|3|1|1980|SWAZILAND||Bea.Ryan@UFv0IFq.edu|2452410| +2286|AAAAAAAAOOIAAAAA|1048778|5943|5908||2449200|||||20|8||POLAND||Amanda.Nabors@0QMnNH3GeDjg.edu|2452572| +2287|AAAAAAAAPOIAAAAA|1476880|5553|16626|2452030|2452000|Mrs.|Michelle|Earls|Y|13|9|1928|AMERICAN SAMOA||Michelle.Earls@o70uckuOHP2tu.org|2452603| +2288|AAAAAAAAAPIAAAAA|66473|96|46072|2452323|2452293|Mr.|Jerry|Mcclain|N|17|12|1945|SWEDEN||Jerry.Mcclain@v.com|2452529| +2289|AAAAAAAABPIAAAAA|1126462|6314|25366|2452464|2452434|Mr.|Raphael|Runyan|Y|5|9|1953|SAUDI ARABIA||Raphael.Runyan@hpqnR.org|2452549| +2290|AAAAAAAACPIAAAAA|1405484|2307|49930|2451734|2451704|Mr.|Frankie|Potter|Y|15|2|1947|RWANDA||Frankie.Potter@bRiNlotAv.org|2452317| +2291|AAAAAAAADPIAAAAA|1917550|1674|19445|2450714|2450684|Ms.|Pamela|Russ|Y|14|3|1988|MADAGASCAR||Pamela.Russ@R0V8TQXdbDlf3CD5C8g.edu|2452303| +2292|AAAAAAAAEPIAAAAA|1813344|1800|4703|2452140|2452110|Dr.|Mildred|Cline|N|14|7|1953|VANUATU||Mildred.Cline@qzHGmrhm7L4.org|2452453| +2293|AAAAAAAAFPIAAAAA|892279|3813|18594|2450375|2450345|Mr.|Michael|Weber|N|6|1|1974|INDIA||Michael.Weber@TqDTGtBkhr.org|2452455| +2294|AAAAAAAAGPIAAAAA|1416342|6443|7744|2452362|2452332|Mr.|Larry|Brunner|N|2|2|1963|PAPUA NEW GUINEA||Larry.Brunner@RAF.com|2452334| +2295|AAAAAAAAHPIAAAAA|400738|2548|22758|2451246|2451216|Dr.|Nellie|Wilson|N|1|6|1976|NEW ZEALAND||Nellie.Wilson@trnZkKoi533aq.org|2452475| +2296|AAAAAAAAIPIAAAAA|1300317|6252|4187|2450441|2450411|Sir|Terry|Cook|N|23|1|1956|CUBA||Terry.Cook@QZKnkK.edu|2452634| +2297|AAAAAAAAJPIAAAAA|1595207|6798|28742|2450642|2450612|Mr.|Matthew|Brewer|Y|30|8|1929|SOLOMON ISLANDS||Matthew.Brewer@TOz4MQA.com|2452447| +2298|AAAAAAAAKPIAAAAA|1671203|4857|14911|2450381|2450351|Mr.|Isiah|Langley|Y|8|11|1952|BELARUS||Isiah.Langley@o3MVb4.org|2452317| +2299|AAAAAAAALPIAAAAA|980155|782|33009|2450827|2450797|Sir|Richard|Hendon|N|19|7|1974|JAPAN||Richard.Hendon@d84K1t2l8N.com|2452381| +2300|AAAAAAAAMPIAAAAA|1031302|3337|38572|2449723|2449693|Miss|Elizabeth|Montoya|N|18|3|1933|ARMENIA||Elizabeth.Montoya@aRFuAJaEmz.org|2452340| +2301|AAAAAAAANPIAAAAA|780260|932|5204|2452305|2452275|Dr.|Timothy|Jones|N|3|11|1951|UZBEKISTAN||Timothy.Jones@sVq.org|2452611| +2302|AAAAAAAAOPIAAAAA|893631|2493|38955|2452471|2452441|Miss|Kristina|Moss|N|10|4|1957|SRI LANKA||Kristina.Moss@PtbkO3642X6d2rYurC.com|2452580| +2303|AAAAAAAAPPIAAAAA|||38698|2450014||Sir|Joseph||Y||11||BOTSWANA|||| +2304|AAAAAAAAAAJAAAAA|63488|3239|21037|2449796|2449766|Mr.|Jason|Bullard|N|6|5|1965|SAMOA||Jason.Bullard@kh6SfenKv.edu|2452390| +2305|AAAAAAAABAJAAAAA|82546|2031|39069|2450280|2450250|Sir|Roger|Cox|N|6|1|1942|JORDAN||Roger.Cox@PRxJp6ZR2GaM.com|2452486| +2306|AAAAAAAACAJAAAAA|485139||41409|2449049||Dr.|Paul|Milner||15||1992||||| +2307|AAAAAAAADAJAAAAA|14652|3017|39829|2450610|2450580|Dr.|Wayne|Green|N|25|8|1953|NORFOLK ISLAND||Wayne.Green@N1BQhMJ.com|2452446| +2308|AAAAAAAAEAJAAAAA|1546752|4907|662|2450534|2450504|Mr.|Fernando|Smith|Y|12|11|1950|TRINIDAD AND TOBAGO||Fernando.Smith@O.edu|2452637| +2309|AAAAAAAAFAJAAAAA|986765|5155|3945|2449457|2449427|Sir|William|Ramirez|Y|5|6|1986|COMOROS||William.Ramirez@41FS41vS.edu|2452538| +2310|AAAAAAAAGAJAAAAA|1197977|6226|26475|2452545|2452515|Sir|Matthew|Stevenson|N|21|10|1953|VIET NAM||Matthew.Stevenson@Zb2U37zZoHGL.edu|2452370| +2311|AAAAAAAAHAJAAAAA|428713|5774|24482|2449105|2449075|Dr.|Kimberly|Burley|N|24|5|1935|NETHERLANDS||Kimberly.Burley@9hvZtHM4.edu|2452315| +2312|AAAAAAAAIAJAAAAA|1455580|6520|14260|2451218|2451188|Dr.|Doris|Banda|N|4|7|1954|SWEDEN||Doris.Banda@18YS3Nk.edu|2452481| +2313|AAAAAAAAJAJAAAAA|27091|3411|24531|2451832|2451802|Sir|William|Fritz|Y|9|1|1967|JAMAICA||William.Fritz@End.com|2452421| +2314|AAAAAAAAKAJAAAAA|275192|796|18400|2451819|2451789|Dr.|Paul|Braxton|Y|28|10|1942|MACAO||Paul.Braxton@ZTqNKLjq.edu|2452578| +2315|AAAAAAAALAJAAAAA|1059373||14011|2449720|||Chad|Price||20|6||||Chad.Price@szIFYDqyI5.edu|| +2316|AAAAAAAAMAJAAAAA|1467118|5218|34071|2452357|2452327|Dr.|Stephen|Huff|N|19|2|1967|BANGLADESH||Stephen.Huff@yo5StmEnMF0FRM.org|2452481| +2317|AAAAAAAANAJAAAAA|316428|2015|33155|2451221|2451191|Miss|Elane|Coleman|Y|4|3|1974|GREENLAND||Elane.Coleman@j1E4caAxRk.org|2452432| +2318|AAAAAAAAOAJAAAAA|945149|3299|47049|2449480|2449450|Mr.|Charles|Bender|N|28|8|1931|GIBRALTAR||Charles.Bender@BorHR57f18V3QlCf.org|2452393| +2319|AAAAAAAAPAJAAAAA|1224892|235|16378|2451053|2451023|Mr.|William|Mccoy|Y|19|1|1982|PUERTO RICO||William.Mccoy@zSD0.edu|2452287| +2320|AAAAAAAAABJAAAAA|460506|1829|38799|2452319|2452289|Dr.|Jack|Ervin|N|26|4|1959|MEXICO||Jack.Ervin@JUJLvBcj1Vqm9LS.com|2452595| +2321|AAAAAAAABBJAAAAA|1742987|2846|1552|2452502|2452472|Sir|Clarence|Lopez|Y|6|6|1926|BOTSWANA||Clarence.Lopez@cQqNYacPhVuxh.com|2452579| +2322|AAAAAAAACBJAAAAA|1046952|391|12583|2451148|2451118|Ms.|Carolyn|Butler|Y|14|3|1985|GEORGIA||Carolyn.Butler@yuNmqD.edu|2452422| +2323|AAAAAAAADBJAAAAA|600975|4347|21319|2449754|2449724|Mrs.|Lisa|Parham|N|31|12|1957|CHINA||Lisa.Parham@FJ4p140yEEafGg.org|2452327| +2324|AAAAAAAAEBJAAAAA|1512805|1127|10649|2451861|2451831|Ms.|Judith|Andrews|N|11|1|1954|LEBANON||Judith.Andrews@ha4hB6p.com|2452352| +2325|AAAAAAAAFBJAAAAA|359220|5585|28350|2452347|2452317|Sir|George|Summers|Y|6|3|1958|MOLDOVA, REPUBLIC OF||George.Summers@9DCPC.org|2452512| +2326|AAAAAAAAGBJAAAAA|1182841|2881|30121|2449892|2449862|Sir|Jordan|Wilder|Y|3|12|1976|IRELAND||Jordan.Wilder@p5NSo6GKKV9o.com|2452578| +2327|AAAAAAAAHBJAAAAA|1011477|5371|46465|2450275|2450245|Sir|James|Deal|N|9|4|1953|C�TE D'IVOIRE||James.Deal@gmCKEfhZujH.org|2452464| +2328|AAAAAAAAIBJAAAAA|274523|4358|8993|2451858|2451828|Miss|Caroline|Harris|N|24|6|1937|DJIBOUTI||Caroline.Harris@PUOSm.com|2452437| +2329|AAAAAAAAJBJAAAAA|797194|5572|23197|2450472|2450442|Ms.|Gladys|Hinojosa|N|11|9|1983|YEMEN||Gladys.Hinojosa@cRUiSNOt2L.edu|2452418| +2330|AAAAAAAAKBJAAAAA|1777305|1103|24054|2449234|2449204|Mr.|John|Richardson|Y|18|12|1946|AMERICAN SAMOA||John.Richardson@6vQjF.org|2452316| +2331|AAAAAAAALBJAAAAA|1735899|926|33153|2450892|2450862|Dr.|Lucy|Guzman|Y|6|9|1973|BELARUS||Lucy.Guzman@p686vH8i4sjLzCLvCu6.com|2452493| +2332|AAAAAAAAMBJAAAAA|660143|2874|14805|2451602|2451572|Ms.|Lila|Barrera|Y|13|1|1978|EL SALVADOR||Lila.Barrera@RfxxUbRGKg.org|2452306| +2333|AAAAAAAANBJAAAAA|1419266|1688|3757|2452359|2452329|Dr.|William|Jennings|Y|13|11|1928|KOREA, REPUBLIC OF||William.Jennings@iqDjCAtk5QNF.org|2452530| +2334|AAAAAAAAOBJAAAAA|33812|2500|24341|2449377|2449347|Dr.|Clarissa|Negrete|Y|13|7|1933|NIGER||Clarissa.Negrete@6iedq1gnUFUzF1.com|2452456| +2335|AAAAAAAAPBJAAAAA|1043895|1665|10682|2452322|2452292|Dr.|Jenny|Crosby|N|6|12|1949|CAPE VERDE||Jenny.Crosby@DuByLVXU7YKQ.org|2452626| +2336|AAAAAAAAACJAAAAA|1307178|6136|294|2450183|2450153|Sir|Roger|Gonzalez|Y|8|1|1980|TUVALU||Roger.Gonzalez@Qbki8Z3ti6ttqLr.com|2452380| +2337|AAAAAAAABCJAAAAA|829610|1507|27804|2449330|2449300|Mrs.|Felicia|Richards|N|19|4|1933|MACAO||Felicia.Richards@4R9.com|2452473| +2338|AAAAAAAACCJAAAAA|1879060|2393|16573|2450657|2450627|Sir|Larry|Mueller|N|1|3|1941|RUSSIAN FEDERATION||Larry.Mueller@rPfh8L28Pz.edu|2452605| +2339|AAAAAAAADCJAAAAA|1863533|6663|10993|2450480|2450450|Mr.|Jae|Freund|N|24|8|1970|BULGARIA||Jae.Freund@dFZTCE.org|2452287| +2340|AAAAAAAAECJAAAAA|532181|6341|24054|2451172|2451142|Mrs.|Phyllis|Carr|Y|19|10|1991|SAMOA||Phyllis.Carr@da.edu|2452417| +2341|AAAAAAAAFCJAAAAA|||32468|2452197||Mr.|Justin|Baker||17|||MALDIVES||Justin.Baker@snllIPnNu.org|2452297| +2342|AAAAAAAAGCJAAAAA|1855363|1335|31354|2452454|2452424|Mr.|Archie|Williams|N|14|12|1982|TONGA||Archie.Williams@xZ.com|2452336| +2343|AAAAAAAAHCJAAAAA|988663|5373|27730|2451371|2451341|Miss|Divina|Solomon|N|27|5|1990|SOMALIA||Divina.Solomon@TBJ0XG.edu|2452511| +2344|AAAAAAAAICJAAAAA|376329|3496|4616|2452482|2452452|Miss|Nancy|Mack|Y|26|9|1987|TURKMENISTAN||Nancy.Mack@zAYivRICB7bNZha.edu|2452297| +2345|AAAAAAAAJCJAAAAA|653380|1330|5327|2450370|2450340|Ms.|Amelia|Castillo|N|26|3|1985|SAN MARINO||Amelia.Castillo@2g4a2FckHNChpzSl3G5S.edu|2452548| +2346|AAAAAAAAKCJAAAAA|371125|2925|43925|2449570|2449540|Sir|Steve|Winchester|N|11|10|1948|GREECE||Steve.Winchester@Po.com|2452645| +2347|AAAAAAAALCJAAAAA|723321|1150|3426|2450416|2450386|Mrs.|Rita|Jenkins|N|27|4|1950|GUYANA||Rita.Jenkins@EPh.com|2452524| +2348|AAAAAAAAMCJAAAAA|916963|4738|9313|2450396|2450366|Dr.|Hildegard|Miller|Y|10|7|1966|JAPAN||Hildegard.Miller@gTqibY.org|2452549| +2349|AAAAAAAANCJAAAAA|1336794|407|9573|2451739|2451709|Sir|Robert|Delgado|N|16|3|1977|UGANDA||Robert.Delgado@EUu8PcXN.org|2452417| +2350|AAAAAAAAOCJAAAAA|441271|2518|5718|2451534|2451504|Ms.|Nicole|Conn|Y|14|3|1986|FRANCE||Nicole.Conn@r2fYjNOMZ26I.org|2452547| +2351|AAAAAAAAPCJAAAAA|1717714|6298|41235|2450686|2450656|Ms.|Michele|Mouton|N|27|3|1926|TUNISIA||Michele.Mouton@VBLm.org|2452358| +2352|AAAAAAAAADJAAAAA|1491347|5499|42804|2449844|2449814|Mrs.|Esther|Wells|Y|7|6|1983|NEW ZEALAND||Esther.Wells@JBJ.edu|2452494| +2353|AAAAAAAABDJAAAAA|721712|5222|45179|2450447|2450417|Mrs.|Allison|Purcell|N|1|7|1926|UGANDA||Allison.Purcell@5Ri.org|2452583| +2354|AAAAAAAACDJAAAAA|1643110|5918|29479|2451752|2451722|Mr.|Jared|Hardy|N|4|5|1959|ZAMBIA||Jared.Hardy@UZiBtDm.edu|2452333| +2355|AAAAAAAADDJAAAAA|1028522|1955|21741|2450197|2450167|Sir|Patrick|Boyle|Y|14|11|1956|ROMANIA||Patrick.Boyle@JZy18plDrgNf1Gjv0.org|2452307| +2356|AAAAAAAAEDJAAAAA|1735817|1127|34837|2452011|2451981|Sir|Victor|Roberts|Y|8|1|1958|OMAN||Victor.Roberts@IBv8Yl7jcyTnvk48Gl.edu|2452477| +2357|AAAAAAAAFDJAAAAA|1713175|1511|24247|2452536|2452506|Mr.|Matthew|Webb|Y|20|6|1931|CROATIA||Matthew.Webb@p2hniSKjoM.com|2452354| +2358|AAAAAAAAGDJAAAAA|1397169|1130|1091|2451704|2451674|Mrs.|Florence|Gomez|Y|7|9|1961|ALBANIA||Florence.Gomez@l7BOv7tEK9Ms9U5dN.com|2452458| +2359|AAAAAAAAHDJAAAAA|1567046|5686|21526|2452013|2451983|Mr.|Robert|Booker|N|18|7|1969|LIECHTENSTEIN||Robert.Booker@li9XvJPNT35v9E0X.edu|2452512| +2360|AAAAAAAAIDJAAAAA|1740908|4569|15430|2451664|2451634|Mr.|Bernard|Lemus|N|6|10|1989|ARGENTINA||Bernard.Lemus@22QCnqNrM4KMns.edu|2452567| +2361|AAAAAAAAJDJAAAAA|203713|3407|1901|2449161|2449131|Dr.|Bobby|Acosta|N|25|8|1927|LATVIA||Bobby.Acosta@tNuR2.org|2452573| +2362|AAAAAAAAKDJAAAAA|337176|5811|40846|2449701|2449671|Miss|Judith|Mitchell|N|12|11|1943|ROMANIA||Judith.Mitchell@XrmSeQ52buLJ.org|2452376| +2363|AAAAAAAALDJAAAAA|||48153|2451311|2451281|Dr.||Young||20|8||MAYOTTE||Helen.Young@M83L2ugyvy.edu|2452366| +2364|AAAAAAAAMDJAAAAA|1900322|3382|25289|2450500|2450470|Sir|Lawrence|Smith|N|2|7|1951|NEW ZEALAND||Lawrence.Smith@ZFK32nobhI4.edu|2452481| +2365|AAAAAAAANDJAAAAA|332136|504|13625|2450594|2450564|Sir|Gary|Trahan|Y|18|7|1925|ETHIOPIA||Gary.Trahan@OKjrVgErHo.com|2452417| +2366|AAAAAAAAODJAAAAA|800058|2198|41891|2451692|2451662|Mr.|Gale|Motley|Y|17|5|1927|OMAN||Gale.Motley@zt6JNXnhNusLMt.org|2452376| +2367|AAAAAAAAPDJAAAAA|1754731|4606|45281|2449306|2449276|Ms.|Marjorie|Pleasant|N|17|8|1945|GIBRALTAR||Marjorie.Pleasant@r3Is8b5pbb.org|2452462| +2368|AAAAAAAAAEJAAAAA|486101|741|43381|2452518|2452488|Ms.|Page|Kirkland|N|15|2|1956|PANAMA||Page.Kirkland@hgsOPMGrxHx5.com|2452596| +2369|AAAAAAAABEJAAAAA|17763|72|21981|2452444|2452414|Miss|Sarah|Washington|N|14|9|1976|GABON||Sarah.Washington@t0QbGYgGY4.com|2452367| +2370|AAAAAAAACEJAAAAA|139628|5023|14418|2449870|2449840|Dr.|Carolyn|Ford|N|27|11|1939|TOGO||Carolyn.Ford@DkTZPzqMNge7tninD.com|2452292| +2371|AAAAAAAADEJAAAAA|960317|4476|49082|2452559|2452529|Mr.|George|Hogan|Y|25|5|1962|FRENCH GUIANA||George.Hogan@8rooiIvTi.com|2452301| +2372|AAAAAAAAEEJAAAAA|153560|4076|3792|2451641|2451611|Ms.|Marguerite|Scott|Y|24|4|1965|UNITED KINGDOM||Marguerite.Scott@v.com|2452559| +2373|AAAAAAAAFEJAAAAA|1588606|659|25065|2449292|2449262|Dr.|Melvin|Stine|Y|16|9|1932|SWAZILAND||Melvin.Stine@PdHxx0CJp09.org|2452486| +2374|AAAAAAAAGEJAAAAA|47816|2427|45838|2449925|2449895|Dr.|Alex|Lewis|N|26|8|1981|TUVALU||Alex.Lewis@yE.com|2452532| +2375|AAAAAAAAHEJAAAAA|1452759|180|10001|2449615|2449585|Sir|Michael|Mullis|N|1|9|1953|BERMUDA||Michael.Mullis@Jh992sPIFg2YEHKqQTv.com|2452399| +2376|AAAAAAAAIEJAAAAA|1884014|493|40572|2450883|2450853|Sir|Roosevelt|Eng|Y|10|3|1987|SAINT LUCIA||Roosevelt.Eng@2fkJMSe7UMyLAG.edu|2452644| +2377|AAAAAAAAJEJAAAAA|1152218|4732|30046|2452031|2452001|Miss|Anthony|Shoemaker|Y|18|10|1946|TOGO||Anthony.Shoemaker@tHnSjU8uI6X.org|2452484| +2378|AAAAAAAAKEJAAAAA|601379|5037|24342|2452213|2452183|Dr.|Edwin|Lopez|Y|15|2|1947|BRUNEI DARUSSALAM||Edwin.Lopez@rpaqjqAqXXc.com|2452643| +2379|AAAAAAAALEJAAAAA|1609368|2389|33010|2452003|2451973|Mr.|James|Leblanc|Y|29|8|1930|BRAZIL||James.Leblanc@cLbEvog3Q.edu|2452303| +2380|AAAAAAAAMEJAAAAA|542852|3755|28305|2452536|2452506|Dr.|Carolyn|Mitchell|N|22|10|1925|BANGLADESH||Carolyn.Mitchell@pJ90phuxC.edu|2452313| +2381|AAAAAAAANEJAAAAA|252213|741|40069|2452229|2452199|Ms.|Lisa|Rodriguez|N|19|1|1944|GUATEMALA||Lisa.Rodriguez@udG4qf7uG4c2l.org|2452432| +2382|AAAAAAAAOEJAAAAA|9554|5610|3266|2449442|2449412|Mrs.|Ann|Latham|N|23|11|1958|ISLE OF MAN||Ann.Latham@is.com|2452594| +2383|AAAAAAAAPEJAAAAA|1781237|2625|16766|2450385|2450355|Mrs.|Anthony|Ouellette|N|3|5|1952|NAURU||Anthony.Ouellette@6.org|2452584| +2384|AAAAAAAAAFJAAAAA|128297|2699|23501|2450898|2450868|Mr.|Jeffrey|Busby|N|8|7|1968|NAMIBIA||Jeffrey.Busby@fBAJP36.com|2452503| +2385|AAAAAAAABFJAAAAA||4670|25964||2449718||Bobby||||4|1952|COSTA RICA|||| +2386|AAAAAAAACFJAAAAA|922311|1908|29458|2452248|2452218|Mr.|Louis|Luna|Y|1|1|1929|MALAYSIA||Louis.Luna@LrZBOvDJ842PiFOfe.edu|2452410| +2387|AAAAAAAADFJAAAAA|787162|4450|27338|2451143|2451113|Mrs.|Janet|Davis|N|2|12|1940|UNITED ARAB EMIRATES||Janet.Davis@NZoJHe.edu|2452499| +2388|AAAAAAAAEFJAAAAA|1257963|6223|41683|2451090|2451060|Dr.|Joey|Solomon|N|25|8|1983|ARUBA||Joey.Solomon@TdNHffqkCD.com|2452619| +2389|AAAAAAAAFFJAAAAA|1128971|2967|19363|2451499|2451469|Dr.|Julie|Nelson|N|24|7|1924|NAURU||Julie.Nelson@o4tSOCpfSEY9uy.org|2452356| +2390|AAAAAAAAGFJAAAAA|276700|4742|30784|2451496|2451466|Mr.|Calvin|Miller|Y|15|12|1951|PORTUGAL||Calvin.Miller@MU0NUzPjj6FjM4Yp.com|2452618| +2391|AAAAAAAAHFJAAAAA|299703|4319|48135|2452386|2452356|Ms.|Sarah|King|Y|6|5|1953|SEYCHELLES||Sarah.King@C4KyaeJJnNXia7c62ze.com|2452293| +2392|AAAAAAAAIFJAAAAA|1403408|6992|5423|2450246|2450216|Dr.|Arnold|Miles|N|20|5|1985|CANADA||Arnold.Miles@ZJhJSzmJqy.com|2452326| +2393|AAAAAAAAJFJAAAAA|1306404|3588|39572|2451547|2451517|Mrs.|Deborah|Wood|N|2|6|1991|BURKINA FASO||Deborah.Wood@qGov0z.com|2452491| +2394|AAAAAAAAKFJAAAAA|158694|6413|4932|2452238|2452208|Dr.|Donald|Unger|N|1|9|1989|OMAN||Donald.Unger@tvFXia.edu|2452580| +2395|AAAAAAAALFJAAAAA|978985|4947|30109|2451811|2451781|Sir|John|Clayton|Y|15|7|1988|UKRAINE||John.Clayton@5Kdg0qeuYi.org|2452333| +2396|AAAAAAAAMFJAAAAA|||9841|2451940|2451910|||Polk||20|8|||||| +2397|AAAAAAAANFJAAAAA|1477767|7034|34654|2449264|2449234|Sir|Kelvin|Vaughan|Y|16|10|1934|VIRGIN ISLANDS, U.S.||Kelvin.Vaughan@ppZmSlthMtzgL.com|2452509| +2398|AAAAAAAAOFJAAAAA|1250377|4075|46126|2450604|2450574|Dr.|Ronald|Smith|N|6|8|1986|MALDIVES||Ronald.Smith@3Y0XKBmv3Aun5qsmx.org|2452403| +2399|AAAAAAAAPFJAAAAA|950408|36|2064|2451520|2451490|Miss|Beverly|Paredes|N|15|1|1974|R�UNION||Beverly.Paredes@T0Voinnxf2.edu|2452481| +2400|AAAAAAAAAGJAAAAA|437930|16|36102|2452346|2452316|Dr.|Randee|Rogers|N|18|10|1985|NEW CALEDONIA||Randee.Rogers@8fnogVdUH8v.org|2452333| +2401|AAAAAAAABGJAAAAA|1013524|4134|11144|2449777|2449747|Ms.|Karen|Miller|N|29|9|1963|MOZAMBIQUE||Karen.Miller@H2I3iVAkeNntqbNs.edu|2452588| +2402|AAAAAAAACGJAAAAA|773769|4917|32865|2449501|2449471|Dr.|Leonard|Reed|Y|17|2|1965|AMERICAN SAMOA||Leonard.Reed@XIIK8TEKOUX.edu|2452635| +2403|AAAAAAAADGJAAAAA|615494|4299|32704|2449914|2449884|Dr.|Otto|James|Y|13|8|1942|MALI||Otto.James@AXxDg0i2FzuZirmQKt6.org|2452399| +2404|AAAAAAAAEGJAAAAA|1292768|1992|48416|2452607|2452577|Ms.|Mildred|Leonard|Y|1|12|1953|DENMARK||Mildred.Leonard@xhz5x94jpZyZif.org|2452339| +2405|AAAAAAAAFGJAAAAA|803112|1727|17589|2452178|2452148|Dr.|Victor|Davis|Y|28|7|1947|YEMEN||Victor.Davis@cSJlM7Ihcuv1.edu|2452319| +2406|AAAAAAAAGGJAAAAA|412557||12241||2450237||Joshua|Miller|N||||||Joshua.Miller@cFBqKjZ7Ca8.org|2452332| +2407|AAAAAAAAHGJAAAAA|899503|2553|12050|2450241|2450211|Ms.|Elena|Coleman|Y|8|4|1986|AZERBAIJAN||Elena.Coleman@MXsFKkXlnpEh.com|2452533| +2408|AAAAAAAAIGJAAAAA|265914|693|24272|2452408|2452378|Mrs.|Betty|Hughes|Y|6|9|1952|FIJI||Betty.Hughes@0IL.org|2452577| +2409|AAAAAAAAJGJAAAAA|994018|3304|42143|2449049|2449019|Dr.|Robert|Schuler|N|8|4|1944|CHRISTMAS ISLAND||Robert.Schuler@yl9E1jNUZIIv8.org|2452508| +2410|AAAAAAAAKGJAAAAA|746088|3943|24198|2452058|2452028|Mr.|Craig|Guthrie|N|9|8|1959|GERMANY||Craig.Guthrie@b69bqeh.com|2452631| +2411|AAAAAAAALGJAAAAA|251708|532|45756|2451086|2451056|Miss|Edna|Hall|N|2|7|1926|PERU||Edna.Hall@Rmeuif2aSUClQur.org|2452626| +2412|AAAAAAAAMGJAAAAA|1023806|751|16896|2449454|2449424|Dr.|Manuel|Fitzpatrick|N|30|3|1939|PERU||Manuel.Fitzpatrick@btGlOAbI29Vonl6.org|2452474| +2413|AAAAAAAANGJAAAAA|470743|4537|47484|2451600|2451570|Sir|Evan|Peterson|Y|10|11|1978|GERMANY||Evan.Peterson@hHUXR4LyPhAuX.org|2452406| +2414|AAAAAAAAOGJAAAAA|1626653|807|48608|2451947|2451917|Dr.|Joanne|Kyle|Y|18|6|1960|KIRIBATI||Joanne.Kyle@MDEA.com|2452349| +2415|AAAAAAAAPGJAAAAA|33213|3577|3094|2449910|2449880|Mr.|Ramon|Kim|N|13|9|1977|EQUATORIAL GUINEA||Ramon.Kim@AdUUeLCgi9d.com|2452372| +2416|AAAAAAAAAHJAAAAA|724993|3732|10871|2450917|2450887|Mr.|Eugene|Morris|N|22|9|1954|GREECE||Eugene.Morris@M.com|2452395| +2417|AAAAAAAABHJAAAAA||2599|1289||2451702||Robert||||||BANGLADESH||Robert.Chapman@er.edu|| +2418|AAAAAAAACHJAAAAA|309134|5328|12675|2452619|2452589|Mrs.|Nancy|Humphrey|Y|30|3|1968|PHILIPPINES||Nancy.Humphrey@PPXlKkpI.com|2452345| +2419|AAAAAAAADHJAAAAA|67698|3914|15777|2451119|2451089|Dr.|Joey|Hoffman|Y|8|11|1952|LIBERIA||Joey.Hoffman@Zs6ECm5VzZUIGR.edu|2452436| +2420|AAAAAAAAEHJAAAAA|890563|5378|18993|2451790|2451760|Sir|Rocky|Allen|N|30|3|1988|HONDURAS||Rocky.Allen@aKXheFfgb7YXKDAEvx.org|2452389| +2421|AAAAAAAAFHJAAAAA|765323|2557|20238|2449645|2449615|Dr.|Hilda|Curley|Y|12|12|1939|SRI LANKA||Hilda.Curley@Izp9VpqT.com|2452299| +2422|AAAAAAAAGHJAAAAA|1543973|3860|15747|2451241|2451211|Sir|Young|Calvin|N|6|2|1924|MOROCCO||Young.Calvin@QGYLI.org|2452461| +2423|AAAAAAAAHHJAAAAA|1706063|1803|16390|2450532|2450502|Sir|Paul|Byrne|Y|24|10|1964|PHILIPPINES||Paul.Byrne@9KO8K0.com|2452556| +2424|AAAAAAAAIHJAAAAA|1067186|4824|7615|2452496|2452466|Sir|Joseph|Walker|Y|18|11|1961|ANTARCTICA||Joseph.Walker@nxZZIXTG1aIsi1jUpG.com|2452609| +2425|AAAAAAAAJHJAAAAA|623955|6897|9439|2449464|2449434|Ms.|Ruth|Reynolds|N|22|9|1939|LIECHTENSTEIN||Ruth.Reynolds@XmCkvV.org|2452341| +2426|AAAAAAAAKHJAAAAA|309149|1535|39074|2449790|2449760|Dr.|William|Sammons|Y|16|6|1970|SUDAN||William.Sammons@uXOpanvGSnl.com|2452296| +2427|AAAAAAAALHJAAAAA|1456383|6813|32843|2450395|2450365|Ms.|Ramona|Thompson|N|24|4|1985|TOKELAU||Ramona.Thompson@oHjuuQ.com|2452566| +2428|AAAAAAAAMHJAAAAA|753898|4965|48479|2451638|2451608|Sir|Walter|Peck|N|14|4|1966|AUSTRIA||Walter.Peck@o.com|2452591| +2429|AAAAAAAANHJAAAAA|443751|6503|14008|2449303|2449273|Ms.|Rachael|Harlan|Y|18|5|1992|CAYMAN ISLANDS||Rachael.Harlan@lCD.org|2452466| +2430|AAAAAAAAOHJAAAAA|309220|2605|44790|2450612|2450582|Sir|Jeff|Doyle|Y|5|7|1969|ANTIGUA AND BARBUDA||Jeff.Doyle@zlV7fcRsbar76.com|2452378| +2431|AAAAAAAAPHJAAAAA|1487079|2979|22126|2452371|2452341|Dr.|Amy|Cruz|Y|20|6|1928|ISRAEL||Amy.Cruz@v.com|2452484| +2432|AAAAAAAAAIJAAAAA|192850|5524|30598|2451593|2451563|Dr.|Domingo|Shah|N|18|6|1938|CHRISTMAS ISLAND||Domingo.Shah@iSavgsYlePb.com|2452527| +2433|AAAAAAAABIJAAAAA|789241|408|5928|2451755|2451725|Mr.|William|Smith|Y|31|10|1987|UKRAINE||William.Smith@lGazTK1Kq.org|2452541| +2434|AAAAAAAACIJAAAAA|1257788|1523|46058|2451174|2451144|Dr.|Lenard|Marshall|Y|24|11|1951|TAJIKISTAN||Lenard.Marshall@m58SaQYeJm2NIpHT.edu|2452630| +2435|AAAAAAAADIJAAAAA|648940|3986|6232|2449532|2449502|Mrs.|Florence|Herbert|N|11|12|1961|TURKMENISTAN||Florence.Herbert@kDNtKABllJy.edu|2452598| +2436|AAAAAAAAEIJAAAAA|1004760|1828|31249|2450166|2450136|Mrs.|Lillian|Bradley|N|28|6|1956|DENMARK||Lillian.Bradley@IuN.org|2452514| +2437|AAAAAAAAFIJAAAAA|363589|5|31621|2451258|2451228|Mr.|Jess|Lawrence|N|28|2|1942|GAMBIA||Jess.Lawrence@H4BDduxrPGAmxOJbVD.org|2452324| +2438|AAAAAAAAGIJAAAAA|1130429|1764|47099|2450057|2450027|Dr.|Laureen|Isaac|Y|15|6|1932|AUSTRIA||Laureen.Isaac@P7VBtlTf2.org|2452584| +2439|AAAAAAAAHIJAAAAA|370435|4709|14183|2449770|2449740|Mrs.|Leandra|Harris|Y|25|2|1956|MOLDOVA, REPUBLIC OF||Leandra.Harris@OodvQjSnGtd9.com|2452439| +2440|AAAAAAAAIIJAAAAA|113251|2327|32603|2449826|2449796|Sir|William|Carter|N|23|4|1970|SINGAPORE||William.Carter@04haasILT2.com|2452507| +2441|AAAAAAAAJIJAAAAA|979874|5705|48336|2451183|2451153|Dr.|Patrick|Palmer|Y|10|8|1927|WALLIS AND FUTUNA||Patrick.Palmer@JRVze.edu|2452513| +2442|AAAAAAAAKIJAAAAA|10176|3233|29843|2450275|2450245|Dr.|John|Koch|N|11|10|1970|HONDURAS||John.Koch@n14tzxNCm.org|2452519| +2443|AAAAAAAALIJAAAAA|1050850|5069|14888|2449083|2449053|Sir|William|Campbell|N|2|1|1978|GIBRALTAR||William.Campbell@ffk6H.com|2452487| +2444|AAAAAAAAMIJAAAAA|1295289|2698|48907|2451094|2451064|Mrs.|Xiao|Pickett|Y|22|2|1946|TOGO||Xiao.Pickett@9dg0Sg2e.org|2452504| +2445|AAAAAAAANIJAAAAA|927901|5012|6923|2449461|2449431|Mr.|Robert|Forbes|Y|19|8|1971|BELGIUM||Robert.Forbes@xrZYc92xP.com|2452449| +2446|AAAAAAAAOIJAAAAA|680681|1928|46822|2449702|2449672|Dr.|Paul|Williams|N|17|8|1964|ITALY||Paul.Williams@JCv4KarUEt0x3ZXEN.edu|2452508| +2447|AAAAAAAAPIJAAAAA|733382|6879|33529|2449711|2449681|Dr.|Victoria|Trevino|N|27|11|1964|MARTINIQUE||Victoria.Trevino@PeAbGruz8AXCbZlR.com|2452497| +2448|AAAAAAAAAJJAAAAA|1615939|3947|19908|2451129|2451099|Dr.|Monica|Mclendon|N|7|9|1983|SOMALIA||Monica.Mclendon@RU2xrh8YyRRRp5.org|2452328| +2449|AAAAAAAABJJAAAAA|599504|3319|4381|2451352|2451322|Mr.|Charles|Bradshaw|Y|31|7|1959|PANAMA||Charles.Bradshaw@aQ.edu|2452508| +2450|AAAAAAAACJJAAAAA|950956|5909|4862|2452217|2452187|Dr.|Demetra|Cordero|N|29|1|1985|PALAU||Demetra.Cordero@Xqy3vvI03TU.org|2452488| +2451|AAAAAAAADJJAAAAA|558015|2603|44744|2449398|2449368|Dr.|Shaun|Ward|N|26|7|1934|RUSSIAN FEDERATION||Shaun.Ward@lbNliS4mm.org|2452598| +2452|AAAAAAAAEJJAAAAA|381360|3375|7390|2450196|2450166|Miss|Anita|Taylor|N|7|3|1954|LITHUANIA||Anita.Taylor@3JbGlsP5HSm56.org|2452469| +2453|AAAAAAAAFJJAAAAA|1720589|1721|12900|2452420|2452390|Mr.|Larry|Proffitt|Y|16|6|1927|MEXICO||Larry.Proffitt@1kHT8KDvCqSH.com|2452393| +2454|AAAAAAAAGJJAAAAA|391714|1414|19114|2451042|2451012|Dr.|Terrell|Moss|Y|10|8|1970|UNITED ARAB EMIRATES||Terrell.Moss@qPzpgVo4Vscl.com|2452399| +2455|AAAAAAAAHJJAAAAA|1716519|1354|29036|2451636|2451606|Dr.|Frank|Brown|N|27|3|1963|HAITI||Frank.Brown@rg2L.edu|2452352| +2456|AAAAAAAAIJJAAAAA|522316|3033|30550|2450491|2450461|Miss|Elsie|Brown|N|13|6|1928|ANTARCTICA||Elsie.Brown@uSHsY6X2R9t4.edu|2452289| +2457|AAAAAAAAJJJAAAAA|896260|4202|3697|2449241|2449211|Mrs.|Latisha|Horne|N|30|12|1953|SERBIA||Latisha.Horne@tQYD.org|2452332| +2458|AAAAAAAAKJJAAAAA|1670093|1079|38865|2449929|2449899|Dr.|Donald|Henderson|N|13|10|1935|NETHERLANDS ANTILLES||Donald.Henderson@YtCPqchlouF81.edu|2452371| +2459|AAAAAAAALJJAAAAA|302050|2442|10481|2451294|2451264|Mr.|Thomas|Dixon|Y|6|1|1965|JERSEY||Thomas.Dixon@dO.org|2452498| +2460|AAAAAAAAMJJAAAAA|1834332|6265|1845|2452468|2452438|Mr.|George|Borden|N|13|1|1969|SAUDI ARABIA||George.Borden@vGSlc8Pp.edu|2452597| +2461|AAAAAAAANJJAAAAA|393897|2567|9237|2452659|2452629|Ms.|Wendy|Jarrell|Y|18|4|1977|LUXEMBOURG||Wendy.Jarrell@SB2c6s.org|2452322| +2462|AAAAAAAAOJJAAAAA|1659764|2427|14643|2449072|2449042|Sir|Tyler|Kirk|N|10|1|1935|FRENCH GUIANA||Tyler.Kirk@BxznNrRS.com|2452592| +2463|AAAAAAAAPJJAAAAA|1180385|420|37702|2449532|2449502|Dr.|John|Mitchell|Y|20|12|1978|ANGOLA||John.Mitchell@0SLnoRVlq.com|2452642| +2464|AAAAAAAAAKJAAAAA|1101209|793|42424|2450154|2450124|Miss|Ann|Allison|Y|28|10|1941|SWEDEN||Ann.Allison@jQI0J853qGm3V.org|2452639| +2465|AAAAAAAABKJAAAAA|1099086|7083|34680|2450924|2450894|Dr.|Merlin|Simpson|Y|4|9|1935|COSTA RICA||Merlin.Simpson@KtOEbhRA.org|2452432| +2466|AAAAAAAACKJAAAAA|1848734|60|10986|2449650|2449620|Mr.|Roland|Knight|Y|20|9|1972|ICELAND||Roland.Knight@o7ltUmHuH.org|2452618| +2467|AAAAAAAADKJAAAAA|367752|574|27043|2449657|2449627|Dr.|Toni|Young|N|9|7|1925|AMERICAN SAMOA||Toni.Young@bxlx9sYF.edu|2452325| +2468|AAAAAAAAEKJAAAAA|310118|1584|17850|2449379|2449349|Ms.|Lana|Mckenzie|Y|12|5|1983|JAPAN||Lana.Mckenzie@KFpH.edu|2452330| +2469|AAAAAAAAFKJAAAAA|1565201|827|18406|2452099|2452069|Sir|James|Lewis|Y|9|7|1968|BARBADOS||James.Lewis@YEpAC06nRAF7.com|2452301| +2470|AAAAAAAAGKJAAAAA|556871|4307|4496|2451605|2451575|Mrs.|Brittany|Johnson|N|12|12|1946|PANAMA||Brittany.Johnson@Hsl90j8PAVuT.org|2452559| +2471|AAAAAAAAHKJAAAAA|1220951|4462|17173|2452033|2452003|Ms.|Marjorie|Rhodes|N|22|9|1924|EQUATORIAL GUINEA||Marjorie.Rhodes@xgP6NKFkZ65a.org|2452346| +2472|AAAAAAAAIKJAAAAA|1169282|5488|49694|2450560|2450530|Dr.|Lawrence|Reed|Y|18|10|1927|BAHRAIN||Lawrence.Reed@uQz9uY.org|2452410| +2473|AAAAAAAAJKJAAAAA|1450417|6115|47766|2449504|2449474|Ms.|Katherine|Kelley|Y|26|7|1961|JAPAN||Katherine.Kelley@RI2iF9Zg4jJnbcvyz1.com|2452364| +2474|AAAAAAAAKKJAAAAA|947043|3048|24606|2452581|2452551|Miss|Gloria|Hooper|N|14|7|1944|FIJI||Gloria.Hooper@bfnO6MXVv.com|2452456| +2475|AAAAAAAALKJAAAAA|758288|2044|1800|2450900|2450870|Sir|Mike|Quinn|N|8|9|1960|BULGARIA||Mike.Quinn@960hRqaEPuA.com|2452477| +2476|AAAAAAAAMKJAAAAA|1688341|720|37899|2451134|2451104|Mrs.|Rachael|Cleary|Y|12|2|1983|NORFOLK ISLAND||Rachael.Cleary@6y6IdB9DRN.org|2452522| +2477|AAAAAAAANKJAAAAA||2220|15009|2451952|||Vernon|Gustafson|||5|1970|PANAMA|||| +2478|AAAAAAAAOKJAAAAA|1861604|3584|8317|2449447|2449417|Dr.|Edward|Ward|Y|7|5|1928|KIRIBATI||Edward.Ward@bvtLj.org|2452526| +2479|AAAAAAAAPKJAAAAA|862800|6648|28449|2451272|2451242|Ms.|Gloria|Schultz|Y|9|9|1959|NAMIBIA||Gloria.Schultz@R.com|2452457| +2480|AAAAAAAAALJAAAAA|78171|1271|7571|2451804|2451774|Mr.|Brian|Jackson|N|21|9|1940|SWEDEN||Brian.Jackson@urjsXdGH3dH.org|2452412| +2481|AAAAAAAABLJAAAAA|579587|4694|35801|2451474|2451444|Mrs.|Monica|Herman|N|15|11|1986|BENIN||Monica.Herman@jq1k1nDCX.com|2452518| +2482|AAAAAAAACLJAAAAA|828274|7138|14449|2452452|2452422|Dr.|James|Lane|Y|28|8|1952|FRANCE||James.Lane@DbJ.org|2452336| +2483|AAAAAAAADLJAAAAA|496610|5020|44230|2452582|2452552|Ms.|Lola|Mitchell|N|28|10|1964|SAN MARINO||Lola.Mitchell@G1bDPjHCGbLks.com|2452364| +2484|AAAAAAAAELJAAAAA|1560003|1503|38603|2450129|2450099|Miss|Gwendolyn|Johnson|N|11|12|1960|CHILE||Gwendolyn.Johnson@81arO.edu|2452479| +2485|AAAAAAAAFLJAAAAA|731621|1735|23002|2449135|2449105|Ms.|Amy|Needham|Y|7|1|1985|PARAGUAY||Amy.Needham@KLoluj.com|2452386| +2486|AAAAAAAAGLJAAAAA|1369376|1981|32586|2451983|2451953|Sir|Robert|Ramirez|Y|17|8|1964|ALAND ISLANDS||Robert.Ramirez@2RcFUvE7orILO.org|2452404| +2487|AAAAAAAAHLJAAAAA|299451|3341|5984|2450307|2450277|Sir|Carlton|Rodriguez|N|15|7|1967|TOKELAU||Carlton.Rodriguez@ZzYoY.org|2452294| +2488|AAAAAAAAILJAAAAA|1649373|5308|29643|2449465|2449435|Mrs.|Mona|Hoover|N|27|1|1953|UZBEKISTAN||Mona.Hoover@C1qALMshID8.com|2452599| +2489|AAAAAAAAJLJAAAAA|924544|5474|23957|2449213|2449183|Ms.|Amanda|Coleman|Y|4|11|1933|KAZAKHSTAN||Amanda.Coleman@aHtKPkF6.org|2452581| +2490|AAAAAAAAKLJAAAAA|974571|3173|29022|2451358|2451328|Mr.|Wayne|Olivo|N|17|5|1932|SAUDI ARABIA||Wayne.Olivo@8ARyEbX9l.org|2452391| +2491|AAAAAAAALLJAAAAA|1268613|5183|26153|2450450|2450420|Sir|Brady|Falk|N|28|6|1956|GUADELOUPE||Brady.Falk@DRnmxTO.com|2452518| +2492|AAAAAAAAMLJAAAAA|1738744|1157|3970|2450896|2450866|Ms.|Irma|Mcintire|Y|16|5|1987|ANGOLA||Irma.Mcintire@lzpOB.com|2452384| +2493|AAAAAAAANLJAAAAA|||39561|||Dr.|Kathleen||N|||1926||||| +2494|AAAAAAAAOLJAAAAA|1385880|7154|44075|2451595|2451565|Miss|Dorothy|Kennedy|Y|7|9|1948|GREENLAND||Dorothy.Kennedy@AHyYbSvUopD.com|2452590| +2495|AAAAAAAAPLJAAAAA|1029357|2676|36395|2450144|2450114|Mr.|David|Gonzalez|Y|8|3|1943|CHILE||David.Gonzalez@JbngvS4h4J8Hvtqirs.com|2452492| +2496|AAAAAAAAAMJAAAAA|1088671|2284|32297|2449700|2449670|Ms.|Lena|Wheaton|Y|18|3|1929|CAPE VERDE||Lena.Wheaton@1KQ7tEMYJn8Yy8SV.org|2452399| +2497|AAAAAAAABMJAAAAA|828666|3485|14304|2450694|2450664|Dr.|Marion|Mclain|N|16|2|1931|ANGUILLA||Marion.Mclain@PKik2BMH.com|2452382| +2498|AAAAAAAACMJAAAAA|345809|815|21821|2450048|2450018|Dr.|Josephine|Neal|Y|16|12|1936|MEXICO||Josephine.Neal@x.edu|2452601| +2499|AAAAAAAADMJAAAAA|1663932|3733|21934|2450173|2450143|Mr.|Joseph|Coffey|Y|29|12|1932|MALAYSIA||Joseph.Coffey@isv.org|2452564| +2500|AAAAAAAAEMJAAAAA|248141|423|15152|2450829|2450799|Mrs.|Sandra|King|Y|21|7|1987|ICELAND||Sandra.King@dH7YN81v0ryl.edu|2452457| +2501|AAAAAAAAFMJAAAAA|1831374|4071|23568|2450360|2450330|Mr.|Thomas|Floyd|N|20|12|1925|BURKINA FASO||Thomas.Floyd@sfcTRdjhvS.org|2452550| +2502|AAAAAAAAGMJAAAAA|1622881|186|33138|2451996|2451966|Dr.|Johnny|Stubbs|Y|25|1|1987|MONTENEGRO||Johnny.Stubbs@ZGd0TdPnfba4tm7t.com|2452285| +2503|AAAAAAAAHMJAAAAA|642373|1356|47892|2450279|2450249|Sir|George|Marks|Y|29|6|1968|PARAGUAY||George.Marks@zbHS7R.org|2452529| +2504|AAAAAAAAIMJAAAAA|561463|2032|6601|2449141|2449111|Mr.|Claude|Grant|N|1|1|1983|SOMALIA||Claude.Grant@AsnpUQcbn1Fvm8U2OhE.edu|2452308| +2505|AAAAAAAAJMJAAAAA|1240880|595|11510|2450987|2450957|Dr.|Mark|Garza|N|12|5|1963|MONTSERRAT||Mark.Garza@7GKD3xaRVK.com|2452637| +2506|AAAAAAAAKMJAAAAA|290947|3743|6139|2452032|2452002|Mrs.|Magdalene|Hall|Y|14|7|1941|PANAMA||Magdalene.Hall@8CMALSPVVvYV.com|2452451| +2507|AAAAAAAALMJAAAAA|1739779|5770|32161|2449484|2449454|Ms.|Teri|Meza|N|18|1|1925|UZBEKISTAN||Teri.Meza@qN25.edu|2452312| +2508|AAAAAAAAMMJAAAAA|1776916|1765|29014|2451021|2450991|Mrs.|Helene|Pereira|Y|23|4|1972|NEW ZEALAND||Helene.Pereira@5.com|2452518| +2509|AAAAAAAANMJAAAAA|831577|4510|5443|2451595|2451565|Dr.|Jon|Robertson|Y|5|5|1934|EQUATORIAL GUINEA||Jon.Robertson@xg3F47x.org|2452393| +2510|AAAAAAAAOMJAAAAA|991816|6458|39900|2451574|2451544|Sir|Sergio|Lewis|Y|18|5|1949|TURKMENISTAN||Sergio.Lewis@y9Hroch.edu|2452344| +2511|AAAAAAAAPMJAAAAA|904128|3493|1258|2449061|2449031|Dr.|Betty|Hannah|Y|21|11|1967|UKRAINE||Betty.Hannah@1iTk.edu|2452416| +2512|AAAAAAAAANJAAAAA|341445|2109|44373|2449912|2449882|Mrs.|Helen|Ward|N|20|3|1934|PHILIPPINES||Helen.Ward@o8HVN5.org|2452600| +2513|AAAAAAAABNJAAAAA|714011|5143|11875|2451997|2451967|Miss|Betty|Vanover|Y|9|10|1939|EGYPT||Betty.Vanover@YYyD.edu|2452309| +2514|AAAAAAAACNJAAAAA|95238|5107|39776|2450879|2450849|Sir|Kenneth|Smith|N|9|10|1956|VENEZUELA||Kenneth.Smith@jA5Z2PBg.org|2452504| +2515|AAAAAAAADNJAAAAA|1476692|2889|28686|2450146|2450116|Dr.|Arthur|Pruitt|N|17|1|1991|GUADELOUPE||Arthur.Pruitt@7CrUqjd5.edu|2452602| +2516|AAAAAAAAENJAAAAA|1118804|201|27198|2452644|2452614|Miss|Rochelle|Walker|Y|4|6|1985|SPAIN||Rochelle.Walker@4JHGfJmGHAf6DR.com|2452412| +2517|AAAAAAAAFNJAAAAA|162049|3846|10396|2449841|2449811|Ms.|Andrew|Smith|N|24|11|1985|NEPAL||Andrew.Smith@If2Xk0VGtMZX5T.org|2452456| +2518|AAAAAAAAGNJAAAAA|1774547|803|23412|2451610|2451580|Dr.|James|Parsons|Y|4|4|1938|SLOVENIA||James.Parsons@MYJQ.org|2452561| +2519|AAAAAAAAHNJAAAAA|1261758|4717|35229|2449200|2449170|Ms.|Dora|Martin|N|4|4|1933|EL SALVADOR||Dora.Martin@ydvBpFHeEvmEdU83qT.edu|2452297| +2520|AAAAAAAAINJAAAAA|1161051|3510|36322|2449823|2449793|Mr.|Adolfo|Malcolm|Y|19|2|1969|MARSHALL ISLANDS||Adolfo.Malcolm@RBI991OLoH.com|2452548| +2521|AAAAAAAAJNJAAAAA|1731448|4703|35111|2450686|2450656|Miss|Andrew|Jones|N|11|5|1982|MACAO||Andrew.Jones@9yPmYa1ZamNAnPE.org|2452483| +2522|AAAAAAAAKNJAAAAA|1756040|1110|37380|2451060|2451030|Sir|Thomas|Stanley|N|1|2|1984|MALAWI||Thomas.Stanley@MA9.edu|2452450| +2523|AAAAAAAALNJAAAAA|789636|5823|2363|2451177|2451147|Dr.|Eleanor|James|Y|15|11|1932|SAMOA||Eleanor.James@0sACfadPheJ6V.edu|2452582| +2524|AAAAAAAAMNJAAAAA|1157408|4528|34401|2450626|2450596|Dr.|John|Naquin|Y|8|8|1940|MARTINIQUE||John.Naquin@aL7pb3KQnCfD9cEj.edu|2452487| +2525|AAAAAAAANNJAAAAA|1105816|1174|14132|2451923|2451893|Sir|Ricardo|Wirth|N|12|9|1985|PAPUA NEW GUINEA||Ricardo.Wirth@DikMDiokBNTT.org|2452543| +2526|AAAAAAAAONJAAAAA|1105805|806|49054|2451377|2451347|Mr.|Donald|Schwartz|N|21|8|1942|FINLAND||Donald.Schwartz@gx9cVG1tF8BdE.org|2452298| +2527|AAAAAAAAPNJAAAAA|1407667|1351|17788|2449320|2449290|Mr.|Ian|Rose|Y|1|7|1985|GABON||Ian.Rose@Oz3.org|2452578| +2528|AAAAAAAAAOJAAAAA|302508|5409|22710|2450945|2450915|Mr.|Luther|Smith|N|29|8|1981|R�UNION||Luther.Smith@xAdokKRF29Xqq.org|2452424| +2529|AAAAAAAABOJAAAAA|100804|302|26647|2449732|2449702|Dr.|Michael|Allen|Y|26|11|1937|AUSTRIA||Michael.Allen@Fa33fRvaGUtciqDF6.com|2452478| +2530|AAAAAAAACOJAAAAA|1267202|1771|10340|2449122|2449092|Ms.|Bernice|Vargas|Y|26|6|1983|THAILAND||Bernice.Vargas@d0chgMdfMzt4ba8.org|2452521| +2531|AAAAAAAADOJAAAAA|1078723|900|7477|2449361|2449331|Ms.|Christine|Thompson|N|5|2|1931|BELARUS||Christine.Thompson@Hs26HInjietjT.org|2452603| +2532|AAAAAAAAEOJAAAAA|1646185|4517|6049|2452032|2452002|Dr.|Dennis|Bishop|Y|11|8|1941|ANTIGUA AND BARBUDA||Dennis.Bishop@5O.com|2452621| +2533|AAAAAAAAFOJAAAAA|969271|3548|3378|2449701|2449671|Mr.|Don|Castillo|Y|19|10|1991|ROMANIA||Don.Castillo@egdlkdkmDCPDSI.com|2452321| +2534|AAAAAAAAGOJAAAAA|1019708|4062|25663|2452339|2452309|Dr.|Bruce|Goodwin|Y|9|7|1959|QATAR||Bruce.Goodwin@3mtLAZrvaCRJ9p.org|2452327| +2535|AAAAAAAAHOJAAAAA|595999|2062|6925|2451349|2451319|Dr.|Morris|Murray|Y|14|2|1985|VENEZUELA||Morris.Murray@Z8xfzAkoCYY.com|2452372| +2536|AAAAAAAAIOJAAAAA|739275|1965|41008|2452475|2452445|Dr.|John|Johnson|N|12|2|1977|CYPRUS||John.Johnson@t76izk9x4G.com|2452475| +2537|AAAAAAAAJOJAAAAA|465165|4045|3678|2452122|2452092|Sir|David|Ahmed|N|11|12|1952|GUINEA||David.Ahmed@aOqGG9Hc.com|2452529| +2538|AAAAAAAAKOJAAAAA|1758726|1211|41296|2450568|2450538|Dr.|Linda|Ryan|Y|11|1|1948|TIMOR-LESTE||Linda.Ryan@FcgfB.org|2452609| +2539|AAAAAAAALOJAAAAA|215121|2435|1749|2452651|2452621|Ms.|Anna|Burton|Y|16|3|1972|EL SALVADOR||Anna.Burton@dYskzn8iVbJjn2.edu|2452531| +2540|AAAAAAAAMOJAAAAA|1370749|34|29272|2452609|2452579|Mr.|Gerald|Wilkerson|N|6|10|1982|NETHERLANDS||Gerald.Wilkerson@igMznb12eJm0K.com|2452585| +2541|AAAAAAAANOJAAAAA|1440953|5905|46810|2450673|2450643|Ms.|Ming|Vargas|N|24|11|1973|BENIN||Ming.Vargas@JaZJBPUHsTTz0.org|2452370| +2542|AAAAAAAAOOJAAAAA|1548063|764|13418||||||N|30||1938|AMERICAN SAMOA||Desiree.Pettit@ItO15a.edu|2452473| +2543|AAAAAAAAPOJAAAAA|628393|1420|22400|2452644|2452614|Dr.|Amanda|Reed|Y|25|8|1924|MAURITIUS||Amanda.Reed@aNjPsVct3u.edu|2452318| +2544|AAAAAAAAAPJAAAAA|86750|2245|40587|2452477|2452447|Dr.|Calvin|Cooper|Y|16|2|1959|ZIMBABWE||Calvin.Cooper@q4LUZjl.com|2452313| +2545|AAAAAAAABPJAAAAA|1334627|3496|41965|2450457|2450427|Ms.|Susan|Smith|N|13|10|1933|DOMINICA||Susan.Smith@lYtFTBR9ooFxd.org|2452548| +2546|AAAAAAAACPJAAAAA|1286367|964|11608|2449028|2448998|Sir|David|Derr|Y|7|5|1959|SAMOA||David.Derr@9NQKA.edu|2452388| +2547|AAAAAAAADPJAAAAA|1752938|6778|6618|2450080|2450050|Mr.|Michael|Irving|N|20|6|1982|LIECHTENSTEIN||Michael.Irving@VUe882I1l.org|2452430| +2548|AAAAAAAAEPJAAAAA|1756893|752|40211|2451682|2451652|Dr.|Louise|Smith|Y|24|9|1964|SAN MARINO||Louise.Smith@bPCectuqqcqacU.com|2452284| +2549|AAAAAAAAFPJAAAAA|||27829|2452541|||||||||TOKELAU|||2452514| +2550|AAAAAAAAGPJAAAAA|1083013|1187|48914|2450485|2450455|Dr.|William|Farmer|Y|19|4|1954|SYRIAN ARAB REPUBLIC||William.Farmer@AlnCSfifUDz.edu|2452485| +2551|AAAAAAAAHPJAAAAA|975837|1894|23104|2452077|2452047|Sir|Mike|Harris|Y|27|1|1950|BELIZE||Mike.Harris@U9VYi.edu|2452460| +2552|AAAAAAAAIPJAAAAA|450842|6651|6460|2452306|2452276|Mrs.|Michelle|Krause|Y|6|9|1962|MADAGASCAR||Michelle.Krause@PQAA.org|2452620| +2553|AAAAAAAAJPJAAAAA|1852518|4973|6275|2449243|2449213|Dr.|Lauri|Thompson|Y|5|2|1987|FRENCH GUIANA||Lauri.Thompson@Of0K6Ui7SeR.com|2452638| +2554|AAAAAAAAKPJAAAAA|1126103|5684|4912|2451923|2451893|Miss|Dorothy|Little|N|11|6|1982|PALAU||Dorothy.Little@YDZFF2By.org|2452312| +2555|AAAAAAAALPJAAAAA|1875589|713|43390|2450613|2450583|Sir|Travis|Greer|Y|2|4|1982|COMOROS||Travis.Greer@eAQlSjnQbUJ0lZa.org|2452518| +2556|AAAAAAAAMPJAAAAA|254980|2040|9947|2451291|2451261|Ms.|Katherine|Davis|N|18|2|1968|CANADA||Katherine.Davis@yupy6RJGmNL9.edu|2452451| +2557|AAAAAAAANPJAAAAA|679025|5381|44890|2451878|2451848|Dr.|Ron|Small|N|30|3|1974|PAKISTAN||Ron.Small@D2SVx43nrDHN.com|2452503| +2558|AAAAAAAAOPJAAAAA|1422743|4182|24421|2452657|2452627|Dr.|Robert|Nelson|Y|7|5|1987|GERMANY||Robert.Nelson@nAVtJUdMv.edu|2452308| +2559|AAAAAAAAPPJAAAAA|592024|3056|18441|2451278|2451248|Mrs.|Michelle|Smith|Y|23|12|1953|NEPAL||Michelle.Smith@nJDYek6YkVS1QZaR.org|2452331| +2560|AAAAAAAAAAKAAAAA|32417|1987|13957|2452171|2452141||||N|||1982||||| +2561|AAAAAAAABAKAAAAA|665252|5418|47137|2451432|2451402|Mr.|Ralph|Simmons|N|10|3|1928|PARAGUAY||Ralph.Simmons@bAjdKMiP5.com|2452331| +2562|AAAAAAAACAKAAAAA|59905|2003|44101|2451333|2451303|Mr.|Brian|Casey|N|15|6|1948|ERITREA||Brian.Casey@LYrB4xjhGk30.edu|2452364| +2563|AAAAAAAADAKAAAAA|826758|6432|7259|2450216|2450186|Dr.|Heather|Paradis|Y|14|9|1956|NIUE||Heather.Paradis@IGJhGiX68u.com|2452423| +2564|AAAAAAAAEAKAAAAA|1079587|435|9236|2449641|2449611|Ms.|Missy|Carlson|Y|2|12|1952|NAURU||Missy.Carlson@TSfiSix4FzbgcvGaBY4T.org|2452312| +2565|AAAAAAAAFAKAAAAA|724276|4983|4937|2451260|2451230|Dr.|Paul|Lane|Y|22|12|1977|SLOVENIA||Paul.Lane@Kx22cg.org|2452288| +2566|AAAAAAAAGAKAAAAA|1579759|1647|33982|2451228|2451198|Mr.|Victor|Stroup|N|6|5|1975|NETHERLANDS ANTILLES||Victor.Stroup@N0htF.com|2452386| +2567|AAAAAAAAHAKAAAAA|1608893|6881|46022|2451589|2451559|Dr.|Max|Battle|N|29|5|1940|ANGOLA||Max.Battle@oicFh.com|2452431| +2568|AAAAAAAAIAKAAAAA|451161|4055|40502|2451837|2451807|Mrs.|Andrew|Schmitz|Y|13|7|1966|PHILIPPINES||Andrew.Schmitz@cat00l7.edu|2452437| +2569|AAAAAAAAJAKAAAAA|788720|5909|28651|2449511|2449481|Sir|Rudy|Mcintyre|Y|22|12|1992|BURKINA FASO||Rudy.Mcintyre@cg6h.com|2452459| +2570|AAAAAAAAKAKAAAAA|||49568||2451937||William|Bailey||||1970|||William.Bailey@M.org|2452283| +2571|AAAAAAAALAKAAAAA|1257191|798|28358|2452563|2452533|Mr.|Henry|Crum|N|1|12|1984|AZERBAIJAN||Henry.Crum@oX6kCu.edu|2452455| +2572|AAAAAAAAMAKAAAAA|395253|1188|16723|2450796|2450766|Mrs.|Allen|Miller|Y|5|11|1956|PALAU||Allen.Miller@c6gkjkrdLNI.org|2452552| +2573|AAAAAAAANAKAAAAA|52740|6519|25098|2449363|2449333|Sir|Nick|Joseph|Y|28|1|1929|BRUNEI DARUSSALAM||Nick.Joseph@AZGClIANJZe9HG44em5F.edu|2452638| +2574|AAAAAAAAOAKAAAAA|67287|5043|49599|2450456|2450426|Sir|Zack|Bennett|N|10|10|1954|NORFOLK ISLAND||Zack.Bennett@kUZTH3gYZxZ8tXEo.org|2452361| +2575|AAAAAAAAPAKAAAAA|269247|324|45848|2449852|2449822|Ms.|Margaret|Rollins|Y|16|1|1962|JERSEY||Margaret.Rollins@3bzpmDteeP.edu|2452365| +2576|AAAAAAAAABKAAAAA|878867|2456|41245|2450973|2450943|Ms.|Katherine|Horner|Y|28|2|1966|DOMINICA||Katherine.Horner@b6h.org|2452294| +2577|AAAAAAAABBKAAAAA|602998|5191|11784|2449795|2449765|Mr.|Gilbert|Jeffrey|Y|6|11|1948|MONACO||Gilbert.Jeffrey@UFCH5b.com|2452580| +2578|AAAAAAAACBKAAAAA|1588807|4495|38983|2451038|2451008|Mr.|James|Reid|N|17|10|1982|LATVIA||James.Reid@dGuuF8vriPdZfez.org|2452474| +2579|AAAAAAAADBKAAAAA|5606|3753|44700|2451439|2451409|Sir|Justin|Cox|N|14|10|1936|HUNGARY||Justin.Cox@q6ry5KoB3OsOiQnojML.com|2452480| +2580|AAAAAAAAEBKAAAAA|1445831|1897|32763|2451325|2451295|Dr.|Mildred|Taylor|Y|23|10|1929|SWEDEN||Mildred.Taylor@ktUHBUuRsEit.edu|2452322| +2581|AAAAAAAAFBKAAAAA|1292747|3491|46235|2449674|2449644|Miss|Alvina|Owens|Y|2|9|1973|MALTA||Alvina.Owens@7TEiAJA.org|2452299| +2582|AAAAAAAAGBKAAAAA|1474420|6056|45292|2450873|2450843|Dr.|Robert|Duncan|Y|6|7|1935|SAINT HELENA||Robert.Duncan@z76OPlLV8lUxMI.org|2452451| +2583|AAAAAAAAHBKAAAAA|38241|1440|24286|2451735|2451705|Mrs.|Ruth|Roberson|N|1|9|1990|MARSHALL ISLANDS||Ruth.Roberson@T4FUVm1O8J.com|2452350| +2584|AAAAAAAAIBKAAAAA|647655|1207|49159|2449173|2449143|Miss|Helen|Foster|N|27|10|1943|BRAZIL||Helen.Foster@5hs5RuljP8qUinhKm.org|2452285| +2585|AAAAAAAAJBKAAAAA|1875782|2616|21497|2449632|2449602|Dr.|Daniel|Kim|Y|27|2|1928|PHILIPPINES||Daniel.Kim@C4LTymN1VvIuL.org|2452312| +2586|AAAAAAAAKBKAAAAA|1008814|5291|46968|2451417|2451387|Dr.|Brent|King|Y|3|4|1940|VIRGIN ISLANDS, U.S.||Brent.King@VRadk.org|2452309| +2587|AAAAAAAALBKAAAAA|276376|2881|939|2451596|2451566|Dr.|Dorothy|Freeman|N|4|2|1943|EGYPT||Dorothy.Freeman@S1c6zmMvriJ06nQ.org|2452326| +2588|AAAAAAAAMBKAAAAA|952038|2264|9690|2450436|2450406|Sir|Mark|Cordova|Y|3|3|1934|GREECE||Mark.Cordova@tQ8elDidN3.com|2452645| +2589|AAAAAAAANBKAAAAA|359509|1966|4870|2450587|2450557|Dr.|Brandon|Silva|N|13|6|1950|CAMEROON||Brandon.Silva@cSfQR1IZZo9uk.edu|2452337| +2590|AAAAAAAAOBKAAAAA|702290|5008|30526|2451146|2451116|Mr.|Eddie|Jordan|Y|15|7|1937|TUNISIA||Eddie.Jordan@vSI.edu|2452435| +2591|AAAAAAAAPBKAAAAA|||36291||2450445|Mrs.|||Y|||1948|SYRIAN ARAB REPUBLIC|||| +2592|AAAAAAAAACKAAAAA|1811299|39|2779|2451782|2451752|Mr.|Mike|Edwards|Y|12|9|1941|AFGHANISTAN||Mike.Edwards@r.edu|2452316| +2593|AAAAAAAABCKAAAAA|566823|3239|24408|||Mr.|Paul|Graff|N|21||1984|CYPRUS||Paul.Graff@GOx3.org|2452644| +2594|AAAAAAAACCKAAAAA|1809353|6284|9662|2451127|2451097|Mr.|Richard|Holbrook|N|26|9|1962|PITCAIRN||Richard.Holbrook@DdvUsDnc.org|2452585| +2595|AAAAAAAADCKAAAAA|1700793|40|16487|2449568||Miss|Estella|Gomez|Y|10|||||Estella.Gomez@gv19U7PbMGG.edu|2452344| +2596|AAAAAAAAECKAAAAA|46476|6915|45401|2450148|2450118|Ms.|Vida|Somerville|Y|14|8|1950|PUERTO RICO||Vida.Somerville@EavathRTYJfI.edu|2452334| +2597|AAAAAAAAFCKAAAAA|705488|4444|7400|2451016|2450986|Sir|Robert|Young|N|12|2|1967|ARMENIA||Robert.Young@6zvP0DAvuCT7ZCMs.org|2452531| +2598|AAAAAAAAGCKAAAAA|1044645|2141|48037|2449200|2449170|Dr.|Zachary|Pearson|N|4|2|1981|GREECE||Zachary.Pearson@NxTMsx.edu|2452297| +2599|AAAAAAAAHCKAAAAA|916169|4620|29118|2450913|2450883|Mrs.|Kelli|Smith|N|13|3|1972|SWAZILAND||Kelli.Smith@QSF809S2YY7.com|2452335| +2600|AAAAAAAAICKAAAAA|551727|6796|28312|2451583|2451553|Mrs.|Vicki|Smith|Y|12|11|1929|NIUE||Vicki.Smith@9n07OzzIT8spFH.org|2452379| +2601|AAAAAAAAJCKAAAAA|1080919|556|8193|2450949||Mr.|Darryl|Curley|Y||9||||Darryl.Curley@K0kx5QgZpPvZ75v.edu|| +2602|AAAAAAAAKCKAAAAA|791641||19898|||Ms.||||3||1978||||| +2603|AAAAAAAALCKAAAAA|1777217|1441|19265|2450126|2450096|Mr.|Michael|Lawrence|N|5|2|1940|LEBANON||Michael.Lawrence@b2AivTyz1EgFfXhB.org|2452283| +2604|AAAAAAAAMCKAAAAA|1830338|1013|44888|2449378|2449348|Dr.|Janet|Nelson|Y|3|6|1984|AUSTRIA||Janet.Nelson@VVQA.edu|2452501| +2605|AAAAAAAANCKAAAAA|1326993|6010|12627|2450315|2450285|Dr.|Thomas|Butler|Y|30|3|1946|GERMANY||Thomas.Butler@UX85.com|2452435| +2606|AAAAAAAAOCKAAAAA|344046|967|13488|2451017|2450987|Ms.|Thelma|Williamson|N|18|12|1979|BELIZE||Thelma.Williamson@EBOO.edu|2452563| +2607|AAAAAAAAPCKAAAAA|58093|6869|12045|2449269|2449239|Mr.|William|Bonds|N|10|4|1925|INDIA||William.Bonds@14tGqTV.org|2452494| +2608|AAAAAAAAADKAAAAA|1822284|6594|6457|2451488|2451458|Dr.|William|Howard|Y|16|12|1981|SOMALIA||William.Howard@01U20nG9b0.edu|2452410| +2609|AAAAAAAABDKAAAAA|1784129|3608|17887|2449979|2449949|Mr.|Dale|Cohen|N|10|4|1924|SAN MARINO||Dale.Cohen@y8fG4CojGkbY.org|2452345| +2610|AAAAAAAACDKAAAAA|1596607|330|25653|2449268|2449238|Sir|Daryl|Jones|Y|25|9|1957|HONG KONG||Daryl.Jones@Me0EfpooczjUo.edu|2452334| +2611|AAAAAAAADDKAAAAA|755088|3983|14878|2452398|2452368|Mrs.|Susan|Gould|N|12|4|1984|HONG KONG||Susan.Gould@oFIXcU.com|2452582| +2612|AAAAAAAAEDKAAAAA|1885922|4269|28216|2451470|2451440|Miss|Angel|Grant|N|28|10|1941|HONG KONG||Angel.Grant@J3uM.edu|2452621| +2613|AAAAAAAAFDKAAAAA|1211159|5314|34222|2451114|2451084|Dr.|Sue|Hairston|N|7|10|1977|SOUTH AFRICA||Sue.Hairston@z.org|2452451| +2614|AAAAAAAAGDKAAAAA|387623|6812|3154|2449780|2449750|Dr.|Lance|Kim|N|22|1|1964|NEPAL||Lance.Kim@kUHnMtFrOq.edu|2452398| +2615|AAAAAAAAHDKAAAAA|249760|3402|49844|2451106|2451076|Mrs.|Ann|Hamrick|N|27|9|1959|MOROCCO||Ann.Hamrick@lJSgl2el.com|2452403| +2616|AAAAAAAAIDKAAAAA|205716|3756|48549|2450117|2450087|Sir|James|Durham|N|11|9|1983|SAINT LUCIA||James.Durham@bvfmAvje8.org|2452333| +2617|AAAAAAAAJDKAAAAA|1685013|4801|31789|2449443|2449413|Dr.|Barbara|Gruber|N|4|4|1979|FRANCE||Barbara.Gruber@LJU.org|2452349| +2618|AAAAAAAAKDKAAAAA|726171|1806|21441|2451428|2451398|Mr.|Darryl|Walker|Y|28|10|1979|EQUATORIAL GUINEA||Darryl.Walker@J.org|2452432| +2619|AAAAAAAALDKAAAAA|1826224|1634|22597|2449192|2449162|Ms.|Suzanne|Piper|N|2|12|1953|FINLAND||Suzanne.Piper@9rdVfbo9y8Jmf6a.org|2452501| +2620|AAAAAAAAMDKAAAAA|1385450|4038|9473|2452515|2452485|Dr.|Rodney|Jones|Y|14|9|1988|PITCAIRN||Rodney.Jones@ulEIKgGaB.org|2452567| +2621|AAAAAAAANDKAAAAA|218555|1944|11686|2449445|2449415|Mrs.|Anne|Lee|Y|17|5|1932|ROMANIA||Anne.Lee@QqQNVznxXP6.com|2452538| +2622|AAAAAAAAODKAAAAA|951581|2766|27151|2450570|2450540|Sir|Jonathan|Quinn|N|22|2|1974|BELARUS||Jonathan.Quinn@NcbahUeAKODFtoQXDmH.edu|2452374| +2623|AAAAAAAAPDKAAAAA|685371|4803|12235|2452202|2452172|Sir|Melvin|Rodgers|Y|8|5|1951|GABON||Melvin.Rodgers@aP5SRhpbOkp17X.com|2452630| +2624|AAAAAAAAAEKAAAAA|319721|4808|23095|2450332|2450302|Sir|Kenneth|Clark|Y|13|11|1936|ANGOLA||Kenneth.Clark@O1YPSUqrS0.com|2452332| +2625|AAAAAAAABEKAAAAA|1634757|2077|35844|2449792|2449762|Ms.|Sheree|Green|Y|31|1|1966|AZERBAIJAN||Sheree.Green@g77746.org|2452570| +2626|AAAAAAAACEKAAAAA|36433|84|24582|2452416|2452386|Miss|Arthur|Lees|Y|21|1|1965|LESOTHO||Arthur.Lees@Jp76qe4lSXy1YPF.edu|2452388| +2627|AAAAAAAADEKAAAAA|766354|4989|22024|2450774|2450744|Miss|Theresa|Davis|N|29|7|1944|MACAO||Theresa.Davis@YguNudlHAJH2Lg4.edu|2452507| +2628|AAAAAAAAEEKAAAAA|1445355|6554|45800|2451638|2451608|Mrs.|Constance|Jackson|N|28|11|1937|MEXICO||Constance.Jackson@FfrDLEFL43.com|2452563| +2629|AAAAAAAAFEKAAAAA|1666280|4654|35875|2451851|2451821|Ms.|Stephanie|Pearson|N|3|8|1968|CAYMAN ISLANDS||Stephanie.Pearson@M6Dd2ycy9FzsbX18C.org|2452473| +2630|AAAAAAAAGEKAAAAA|1419768|5932|8096|2449265|2449235|Dr.|Glen|Walden|N|22|2|1958|GIBRALTAR||Glen.Walden@PSUKh.com|2452513| +2631|AAAAAAAAHEKAAAAA|1804047|2527|11054|2452627|2452597|Miss|Stephanie|Carson|N|30|9|1986|SAN MARINO||Stephanie.Carson@ymn3f.org|2452459| +2632|AAAAAAAAIEKAAAAA|893730|6405|34727|2451608|2451578|Dr.|Michael|Cohn|Y|20|2|1944|SWAZILAND||Michael.Cohn@GdeuTJ.com|2452333| +2633|AAAAAAAAJEKAAAAA|622852|1927|5002|2450373|2450343|Dr.|Diana|Otto|N|1|3|1977|CAMEROON||Diana.Otto@D28E15FFSs.org|2452503| +2634|AAAAAAAAKEKAAAAA|1876097|5008|5566|2450123|2450093|Dr.|Margaret|Salmon|Y|23|6|1925|LIBERIA||Margaret.Salmon@DGn5eCZtYK.edu|2452374| +2635|AAAAAAAALEKAAAAA|881602|3368|38734|2450379|2450349||Stephen|Bowman||18|5|1926|ECUADOR|||2452303| +2636|AAAAAAAAMEKAAAAA|1285675|4461|44047|2449491|2449461|Dr.|Walter|Paige|Y|28|6|1941|PARAGUAY||Walter.Paige@PBrN4ie6.com|2452435| +2637|AAAAAAAANEKAAAAA|1555934|4557|24780|2451363|2451333|Dr.|Jo|Hawes|N|7|6|1940|UGANDA||Jo.Hawes@ZzD3caHGM9R.com|2452586| +2638|AAAAAAAAOEKAAAAA|492891|2714|13690|2450799|2450769|Miss|Edith|Wilburn|N|10|2|1965|UZBEKISTAN||Edith.Wilburn@k9hMpDAFnt.edu|2452631| +2639|AAAAAAAAPEKAAAAA|483189|628|820|2450632|2450602|Dr.|William|Mccarty|Y|24|3|1963|CHILE||William.Mccarty@a3hOCVT.edu|2452348| +2640|AAAAAAAAAFKAAAAA|300093|2904|22868|2450243||Ms.|||||4|1970|||Cora.Johnson@4vJ1uTFEaR.com|| +2641|AAAAAAAABFKAAAAA|1784667|5595|33323|2449267|2449237|Mr.|Christopher|Thomas|Y|31|1|1934|EL SALVADOR||Christopher.Thomas@r8jGfNEmzH53gjnD.com|2452511| +2642|AAAAAAAACFKAAAAA|1764249|2546|45643|2449523|2449493|Mrs.|Albert|Walton|N|11|3|1933|BARBADOS||Albert.Walton@XklpBNJJnm4Xj68HtZ.edu|2452639| +2643|AAAAAAAADFKAAAAA|1706726|4671|30663|2451136|2451106|Sir|Mark|Sherman|Y|25|9|1926|OMAN||Mark.Sherman@ZEMrrzKUcKDIKV.com|2452608| +2644|AAAAAAAAEFKAAAAA|431837|2734|36431|2451933|2451903|Mrs.|Brenda|Harrison|N|29|11|1954|CAPE VERDE||Brenda.Harrison@AsDZvo7609a8i1.com|2452289| +2645|AAAAAAAAFFKAAAAA|1835346|5081|23969|2451158|2451128|Mrs.|Lydia|Roberts|Y|29|10|1944|SENEGAL||Lydia.Roberts@ptB5HQtpsxg46.com|2452382| +2646|AAAAAAAAGFKAAAAA|430219|4544|3269|2451930|2451900|Mrs.|Katherine|Hall|N|24|5|1978|LIBERIA||Katherine.Hall@Tz3hKvB5sO.com|2452456| +2647|AAAAAAAAHFKAAAAA|1567864|3014|20564|2450640|2450610|Sir|Stephen|Brenner|N|10|8|1926|LUXEMBOURG||Stephen.Brenner@LYqyz51pcqtGYXI.org|2452631| +2648|AAAAAAAAIFKAAAAA|672616|3495|44327|2451941|2451911|Dr.|Peter|Beltran|Y|8|9|1932|BAHRAIN||Peter.Beltran@KGgHRi7C3UHnc.edu|2452336| +2649|AAAAAAAAJFKAAAAA|841274|6935|39396|2452530|2452500|Mr.|John|Palmer|Y|18|4|1929|UGANDA||John.Palmer@ZlJPfh.com|2452504| +2650|AAAAAAAAKFKAAAAA|1842407|1303|2552|2451298|2451268|Dr.|John|Campbell|Y|15|12|1983|ANGOLA||John.Campbell@BG8r.org|2452418| +2651|AAAAAAAALFKAAAAA|72992|7127|45662|2451150|2451120|Miss|Tara|Crawford|Y|21|10|1988|ESTONIA||Tara.Crawford@qotsEYZNVjq.org|2452329| +2652|AAAAAAAAMFKAAAAA|720221|4873|23125|2450981|2450951|Sir|John|Smith|Y|22|5|1931|WALLIS AND FUTUNA||John.Smith@cA29UBZQ0H8DUXVcP8.com|2452302| +2653|AAAAAAAANFKAAAAA|1679629|2191|49106|2451043|2451013|Miss|Sarah|Brandon|Y|16|1|1924|TOKELAU||Sarah.Brandon@evx28fnqEAVJ.com|2452375| +2654|AAAAAAAAOFKAAAAA|1239868|4918|14321|2451201|2451171|Mr.|Stephen|Grant|Y|5|11|1955|ARGENTINA||Stephen.Grant@k.org|2452482| +2655|AAAAAAAAPFKAAAAA|90134|1340|26593|2452220|2452190|Dr.|Stephen|Seidel|N|21|6|1958|TONGA||Stephen.Seidel@doRoYALykiUspnuO.org|2452442| +2656|AAAAAAAAAGKAAAAA|1242766|5405|24844|2450093|2450063|Dr.|Rebecca|Herman|N|9|10|1964|AUSTRIA||Rebecca.Herman@3Ue0ZOM8dk.edu|2452504| +2657|AAAAAAAABGKAAAAA|1710239|6432|8978|2451502|2451472|Ms.|Lillian|Whitcomb|N|27|7|1958|BAHRAIN||Lillian.Whitcomb@kCTohJegdYQKCFIrjFJ.edu|2452363| +2658|AAAAAAAACGKAAAAA|1657779|6045|21111|2451830|2451800|Mrs.|Opal|Mayberry|N|28|10|1972|MONGOLIA||Opal.Mayberry@L0NNsuGzy1UN9tF1HPF.org|2452629| +2659|AAAAAAAADGKAAAAA|1517617|5004|10038|2451027|2450997|Mr.|James|Mattos|N|29|5|1986|MACAO||James.Mattos@1pTy.org|2452520| +2660|AAAAAAAAEGKAAAAA|1285324|1443|42762|2449720|2449690|||||8|1|1951|MACAO|||| +2661|AAAAAAAAFGKAAAAA|431479|1812|5792|2452119|2452089|Dr.|Barbara|Wolfe|N|13|1|1962|DOMINICA||Barbara.Wolfe@1LFaVUfhIXOqO.org|2452292| +2662|AAAAAAAAGGKAAAAA|348892|4672|12005|2449685|2449655|Mr.|Peter|Ramirez|N|7|5|1936|ESTONIA||Peter.Ramirez@enByrmz3UmpXBb.com|2452606| +2663|AAAAAAAAHGKAAAAA|789502|1147|42392|2449723||Mr.|John|Jernigan|Y||6|||||| +2664|AAAAAAAAIGKAAAAA|328130|5386|35795|2450394|2450364|Dr.|Joann|Gates|Y|17|5|1934|SWEDEN||Joann.Gates@JoO5.edu|2452428| +2665|AAAAAAAAJGKAAAAA|1276267|2581|43648|2451054|2451024|Dr.|Alvaro|Wood|N|13|1|1932|MONACO||Alvaro.Wood@kyv9salFZyV.com|2452583| +2666|AAAAAAAAKGKAAAAA|167170|30|17290|2450486|2450456|Dr.|Fern|Sims|Y|20|10|1971|UZBEKISTAN||Fern.Sims@fZQX2IXcdL6l2.org|2452285| +2667|AAAAAAAALGKAAAAA|252003|5556|31462|2450046|2450016|Sir|Hans|Garrett|Y|17|10|1943|FIJI||Hans.Garrett@XzsiEMGBXpemyF2.org|2452344| +2668|AAAAAAAAMGKAAAAA|674609|3659|15509|2452272|2452242|Mr.|Michael|Fowlkes|N|30|9|1957|MYANMAR||Michael.Fowlkes@r.com|2452461| +2669|AAAAAAAANGKAAAAA|251053|5073|35140|2450405|2450375|Dr.|Maurice|Holt|N|10|9|1924|SWEDEN||Maurice.Holt@Pu5VSzTehc.edu|2452524| +2670|AAAAAAAAOGKAAAAA|1498308|539|8370|2450351|2450321|Dr.|Ronald|Goldstein|N|2|4|1952|OMAN||Ronald.Goldstein@2aHxH2TBFUM1Q.com|2452383| +2671|AAAAAAAAPGKAAAAA|1065868|2071|6627|2451025|2450995|Sir|Joe|Johnson|Y|8|10|1961|AMERICAN SAMOA||Joe.Johnson@inGo9nNDjHmC.org|2452516| +2672|AAAAAAAAAHKAAAAA|1307350|1023|2649|2451868|2451838|Mrs.|Isabelle|Green|Y|8|11|1944|LESOTHO||Isabelle.Green@KvI1rLDdUu.com|2452496| +2673|AAAAAAAABHKAAAAA|1643746|2544|3295|2450723|2450693|Mrs.|Nadine|Hedrick|N|27|5|1966|PUERTO RICO||Nadine.Hedrick@pi3V4Zh3uYQqun0B.edu|2452522| +2674|AAAAAAAACHKAAAAA|||3861|||||Schmidt||22||1953||||| +2675|AAAAAAAADHKAAAAA|1852359|3592|35672|2449227|2449197|Dr.|Mario|Watts|Y|23|4|1963|MALDIVES||Mario.Watts@CxHF8qjMITHHcT.com|2452498| +2676|AAAAAAAAEHKAAAAA|1418504|2252|7247|2449044|2449014|Sir|Minh|Kauffman|N|17|6|1968|PANAMA||Minh.Kauffman@02rMGLPoIo.com|2452567| +2677|AAAAAAAAFHKAAAAA|945203|5455|5260|2451052|2451022|Sir|David|Schofield|Y|9|12|1967|MAURITIUS||David.Schofield@DI.org|2452390| +2678|AAAAAAAAGHKAAAAA|1581440|6880|11920|2451845|2451815|Sir|James|Little|Y|21|9|1961|AUSTRIA||James.Little@9jVx.org|2452515| +2679|AAAAAAAAHHKAAAAA|714622|2208|20594|2450087|2450057|Mrs.|Rebecca|Perry|Y|6|3|1984|HONG KONG||Rebecca.Perry@YpZ1oR7TqK.com|2452326| +2680|AAAAAAAAIHKAAAAA|1455836|651|32619|2451888|2451858|Mr.|Lance|Clark|Y|29|9|1952|UNITED KINGDOM||Lance.Clark@X813aeaLUU.edu|2452508| +2681|AAAAAAAAJHKAAAAA|486994|5874|36307|2452003|2451973|Mrs.|Rose|Hernandez|Y|31|12|1945|SOMALIA||Rose.Hernandez@Qbk1oKe8PAR9L6.com|2452582| +2682|AAAAAAAAKHKAAAAA|1218866|499|28932|2450488|2450458|Dr.|Thaddeus|Bradley|N|23|12|1942|NEW CALEDONIA||Thaddeus.Bradley@ehvLi.org|2452400| +2683|AAAAAAAALHKAAAAA|1498297|6251|22958|2451060|2451030|Mr.|Mike|Johnson|N|4|8|1972|CAPE VERDE||Mike.Johnson@Ff33Ljxev7.com|2452290| +2684|AAAAAAAAMHKAAAAA|333928|1146|17917|2449138|2449108|Mr.|Stanley|Marshall|N|3|1|1946|SYRIAN ARAB REPUBLIC||Stanley.Marshall@kFP6pypMnyHmxOs.org|2452382| +2685|AAAAAAAANHKAAAAA|757397|7024|40651|2451388|2451358|Mrs.|Ivana|Outlaw|N|17|1|1934|BULGARIA||Ivana.Outlaw@jje9fbx2K.org|2452296| +2686|AAAAAAAAOHKAAAAA|960646|764|9729|2449109|2449079|Mr.|Roger|Douglas|Y|2|3|1972|MARSHALL ISLANDS||Roger.Douglas@j98x8hVcmLGTo.com|2452441| +2687|AAAAAAAAPHKAAAAA|151458|2090|31522|2452585|2452555|Mr.|Johnson|Mcneil|N|29|5|1989|FRANCE||Johnson.Mcneil@ddLzQP.edu|2452351| +2688|AAAAAAAAAIKAAAAA|155387|310|8080|2449186|2449156|Miss|Tina|Abraham|N|4|1|1937|PUERTO RICO||Tina.Abraham@U.org|2452443| +2689|AAAAAAAABIKAAAAA|1433396|1964|46485|2452037|2452007|Dr.|Ted|Griffin|N|13|8|1974|UNITED KINGDOM||Ted.Griffin@gLKPKl8cyF0l42ylb3.com|2452421| +2690|AAAAAAAACIKAAAAA|953996|371|45193|2452658|2452628|Sir|Robert|Page|Y|15|2|1924|CHRISTMAS ISLAND||Robert.Page@uvxCxk6XJDem.edu|2452514| +2691|AAAAAAAADIKAAAAA|1177511|6977|13243|2451750|2451720|Dr.|John|Aiken|Y|4|3|1991|PARAGUAY||John.Aiken@mvGKahON689F7s.edu|2452318| +2692|AAAAAAAAEIKAAAAA||5717|15411|2450830|2450800||||Y||12|1968||||2452431| +2693|AAAAAAAAFIKAAAAA|1651307|5980|12653|2452421|2452391|Mr.|Tony|Turner|N|16|6|1957|OMAN||Tony.Turner@zoxumx4APJr.edu|2452353| +2694|AAAAAAAAGIKAAAAA|848996|7057|33965|2451367|2451337|Dr.|Helen|Peterson|N|20|5|1932|HUNGARY||Helen.Peterson@4.org|2452438| +2695|AAAAAAAAHIKAAAAA|1249866|1531|36443|2449610|2449580|Ms.|Johanna|Allison|Y|26|12|1934|SLOVENIA||Johanna.Allison@aPxASh.org|2452499| +2696|AAAAAAAAIIKAAAAA|157556|3205|31301|||Ms.||Rogers||11|1|1948|||Jacqueline.Rogers@7FSRFaP0yp.edu|| +2697|AAAAAAAAJIKAAAAA|1813762|325|41335|2449186|2449156|Dr.|Jeanie|Snow|N|2|5|1957|SWITZERLAND||Jeanie.Snow@8xkbGPUvc.org|2452339| +2698|AAAAAAAAKIKAAAAA|171727|2290|47300|2450941|2450911|Dr.|Mike|Mcdaniel|Y|9|5|1947|GUADELOUPE||Mike.Mcdaniel@5OH.org|2452334| +2699|AAAAAAAALIKAAAAA|548066|3064|16109|2449736|2449706|Mrs.|Ursula|Gardner|N|25|1|1959|TURKMENISTAN||Ursula.Gardner@XgKlO1oQ8mHLU.com|2452343| +2700|AAAAAAAAMIKAAAAA|1026728|324|20720|2450869|2450839|Sir|Salvador|Brown|Y|23|8|1960|LEBANON||Salvador.Brown@0teOI.edu|2452383| +2701|AAAAAAAANIKAAAAA|1605953|4277|49448|2449585|2449555|Ms.|Barbie|Brown|N|28|5|1938|TOGO||Barbie.Brown@QtrRCzvrasce63.edu|2452578| +2702|AAAAAAAAOIKAAAAA|348675|3200|46125|2450394|2450364|Ms.|Viola|Gonzales|Y|28|5|1963|IRAQ||Viola.Gonzales@4d20Z.com|2452383| +2703|AAAAAAAAPIKAAAAA|61417|4871|25878|2449651|2449621|Mr.|Jimmy|Snell|Y|7|12|1947|ANTIGUA AND BARBUDA||Jimmy.Snell@Xe0cSlINYS9V73.com|2452567| +2704|AAAAAAAAAJKAAAAA|1764869|6299|21729|2451484|2451454|Sir|Lloyd|Schwarz|N|6|9|1966|ERITREA||Lloyd.Schwarz@MnP8AUitkdH2X1igF.edu|2452362| +2705|AAAAAAAABJKAAAAA|645809|6767|172|2449424|2449394|Mr.|Jose|Gillespie|N|4|8|1948|PAPUA NEW GUINEA||Jose.Gillespie@JF0Oak2aZr6v.org|2452346| +2706|AAAAAAAACJKAAAAA|782207|6657|11907|2452423|2452393|Mr.|Rodney|Marshall|N|6|2|1986|IRELAND||Rodney.Marshall@dXQNqQBh.com|2452372| +2707|AAAAAAAADJKAAAAA|1683713|3489|4818|2449689|2449659|Sir|Gerald|Perry|Y|22|9|1948|SAUDI ARABIA||Gerald.Perry@jVAZYltyYXpxxT6.com|2452591| +2708|AAAAAAAAEJKAAAAA|963481|1293|32258|2451582|2451552|Mr.|Jason|Williams|N|8|6|1969|CAMEROON||Jason.Williams@n89oniqg5IkhB.org|2452507| +2709|AAAAAAAAFJKAAAAA|609420|1383|37346|2449911|2449881|Mr.|John|Whiting|Y|13|6|1978|BURUNDI||John.Whiting@u5O.org|2452605| +2710|AAAAAAAAGJKAAAAA|226319|3160|20401|2450103|2450073|Ms.|Jo|Smith|N|23|11|1955|SERBIA||Jo.Smith@KoflGAFCoCit.org|2452370| +2711|AAAAAAAAHJKAAAAA|505405|1042|8679|2450057|2450027|Mr.|Kurt|Mercado|Y|11|11|1942|BELIZE||Kurt.Mercado@zSgcSItXgex.org|2452611| +2712|AAAAAAAAIJKAAAAA|687781|806|13878|2452381|2452351|Dr.|Cynthia|Taylor|N|9|3|1947|LESOTHO||Cynthia.Taylor@iZKaSHIBrDtQggaz32.com|2452557| +2713|AAAAAAAAJJKAAAAA|110151|6572|18593|2449974|2449944|Ms.|Mae|Beaulieu|N|14|10|1977|ANTARCTICA||Mae.Beaulieu@azXs6T.org|2452422| +2714|AAAAAAAAKJKAAAAA|1914100|2606|47559|2452049|2452019|Dr.|Ricardo|Mahoney|Y|16|6|1983|TOKELAU||Ricardo.Mahoney@MeipS.edu|2452613| +2715|AAAAAAAALJKAAAAA|1610675|3724|27620|2451990|2451960|Mr.|James|Williams|Y|11|12|1940|ANDORRA||James.Williams@mxFt5qi1gFb.org|2452415| +2716|AAAAAAAAMJKAAAAA|516381|2746|17111|2449412|2449382|Sir|Charles|Banda|Y|28|8|1978|PERU||Charles.Banda@xHokNEeRKp0EYilhDL.com|2452443| +2717|AAAAAAAANJKAAAAA|733012|6218|43424|2452361|2452331|Dr.|Pearl|Cannon|N|6|7|1932|LIECHTENSTEIN||Pearl.Cannon@S1fq9.edu|2452577| +2718|AAAAAAAAOJKAAAAA|1331309|3502|17695|2450125|2450095|Miss|Phyllis|Harris|Y|19|3|1954|MARTINIQUE||Phyllis.Harris@J.org|2452571| +2719|AAAAAAAAPJKAAAAA|507455|1030|2652|2451885|2451855|Sir|Ron|Simon|Y|7|4|1971|LATVIA||Ron.Simon@P5LoaS.com|2452614| +2720|AAAAAAAAAKKAAAAA|1360974|1397|32173|2449853|2449823|Mrs.|Shelly|Yang|Y|8|9|1986|SAMOA||Shelly.Yang@LQxk.edu|2452399| +2721|AAAAAAAABKKAAAAA|39969|3622|41228|2449310|2449280|Mrs.|Charlotte|Harwood|N|12|9|1956|KUWAIT||Charlotte.Harwood@rC2r1L8C.org|2452497| +2722|AAAAAAAACKKAAAAA|1531435|2122|14409|2449428|2449398|Dr.|Nicole|Hendrix|Y|6|1|1943|NETHERLANDS ANTILLES||Nicole.Hendrix@1RS6eZ9.org|2452426| +2723|AAAAAAAADKKAAAAA|1269663|5440|24864|2452537|2452507|Mr.|Steven|Patton|N|12|10|1983|ECUADOR||Steven.Patton@6KpI.org|2452617| +2724|AAAAAAAAEKKAAAAA|218718|6978|39191|2451635|2451605|Mrs.|Tonya|Hoskins|N|9|12|1940|CHILE||Tonya.Hoskins@htUV0fJvrC3.edu|2452594| +2725|AAAAAAAAFKKAAAAA|1455884|1974|4017|2452409|2452379|Mr.|Clifton|Phelps|N|24|6|1975|SURINAME||Clifton.Phelps@OtFAeFsDUMg3nLkel.com|2452641| +2726|AAAAAAAAGKKAAAAA|1334919|3522|6525|||Ms.||Morgan||||1960|||Vivian.Morgan@ErexcVTi.org|| +2727|AAAAAAAAHKKAAAAA|512335|1236|3337|2452451|2452421|Mrs.|Louise|Black|Y|3|2|1958|LIBERIA||Louise.Black@2o3q9CBosJS.com|2452505| +2728|AAAAAAAAIKKAAAAA|1039937|5518|19546|2449567|2449537|Miss|Lisa|Barron|N|1|11|1984|COMOROS||Lisa.Barron@TnG.com|2452562| +2729|AAAAAAAAJKKAAAAA|1118664|332|41153|2450651|2450621|Miss|Phyllis|Webber|N|23|5|1952|LITHUANIA||Phyllis.Webber@H.edu|2452499| +2730|AAAAAAAAKKKAAAAA|1659692|4640|31541|2450875|2450845|Ms.|Adam|Tong|N|6|10|1957|BANGLADESH||Adam.Tong@MvycKZdiHbn.org|2452413| +2731|AAAAAAAALKKAAAAA|337285|5275|14418|2451420|2451390|Dr.|James|Mcdonald|N|31|12|1934|BELARUS||James.Mcdonald@7bgVPpDqe6ooiSmP.edu|2452583| +2732|AAAAAAAAMKKAAAAA|1190105|5521|34490|2449246|2449216|Ms.|Valeria|Holley|Y|9|11|1946|TOGO||Valeria.Holley@nLg0vkyZkfmgyePgu.org|2452370| +2733|AAAAAAAANKKAAAAA|145360|428|34696|2449914|2449884|Dr.|Luis|Rodriguez|Y|11|8|1957|ARGENTINA||Luis.Rodriguez@04l82aAxGc.org|2452334| +2734|AAAAAAAAOKKAAAAA|326976|5298|47973|2451573|2451543|Mr.|Robert|Thompson|N|2|1|1945|ALGERIA||Robert.Thompson@AhjQm.org|2452584| +2735|AAAAAAAAPKKAAAAA||2519|18074||2450550||Kevin|Denson|Y|5|7|1949||||| +2736|AAAAAAAAALKAAAAA|1124900|2576|41452|2449716|2449686|Mr.|Jamie|Bess|Y|12|1|1969|NAURU||Jamie.Bess@HK189LM0G.edu|2452306| +2737|AAAAAAAABLKAAAAA|||26250|2450067|||Margrett||Y||11||||Margrett.Vargas@2sAAz4lvJmie.com|2452524| +2738|AAAAAAAACLKAAAAA|545127|4902|41877|2450344|2450314|Mrs.|Beulah|Keller|N|7|5|1956|IRAQ||Beulah.Keller@qad.org|2452352| +2739|AAAAAAAADLKAAAAA|1291837|5811|19385|2449888|2449858|Miss|Kelli|Osborne|Y|16|10|1972|GAMBIA||Kelli.Osborne@U2xXn.org|2452449| +2740|AAAAAAAAELKAAAAA|561771|4238|4258||2451869|||Bishop||25|||PALAU||Brenda.Bishop@tNGnel.com|2452448| +2741|AAAAAAAAFLKAAAAA|122576||47627|2449855||Ms.||Bridges|N|||1928|ICELAND||Aaron.Bridges@dJSLdFZ8.org|2452514| +2742|AAAAAAAAGLKAAAAA|1010838|4637|36789|2452357|2452327|Mr.|Robert|May|N|17|1|1942|NEPAL||Robert.May@oMDiSzKYj.edu|2452424| +2743|AAAAAAAAHLKAAAAA|213841|1608|2575|2452172|2452142|Mr.|Steven|Murphy|Y|11|4|1968|TOGO||Steven.Murphy@IXgkn.com|2452319| +2744|AAAAAAAAILKAAAAA|95624|423|3175|2450951|2450921|Sir|Jimmy|Priest|Y|13|7|1963|GUINEA-BISSAU||Jimmy.Priest@C.org|2452330| +2745|AAAAAAAAJLKAAAAA|342583|5597|14564|2450364|2450334|Mr.|John|Harris|Y|8|7|1982|PERU||John.Harris@KltH1.com|2452557| +2746|AAAAAAAAKLKAAAAA|18669||10351|2450264|2450234|||||22|7||SPAIN||James.Parker@vBTfJGOcCu.com|2452520| +2747|AAAAAAAALLKAAAAA|350376|897|17418|2450005|2449975|Miss|Melinda|Caldwell|Y|9|10|1965|ANTIGUA AND BARBUDA||Melinda.Caldwell@46zxMgYL.edu|2452597| +2748|AAAAAAAAMLKAAAAA|1160908|2286|48064|2452534|2452504|Dr.|Antonio|Parker|N|11|12|1931|HUNGARY||Antonio.Parker@UFu.org|2452418| +2749|AAAAAAAANLKAAAAA|1616375|1390|31958|2451038|2451008|Miss|Darlene|Rowley|Y|15|9|1981|PAPUA NEW GUINEA||Darlene.Rowley@mo8idOJ.com|2452520| +2750|AAAAAAAAOLKAAAAA|1654550|5651|9199|2450522|2450492|Miss|Nilda|Fulmer|Y|13|12|1965|ARUBA||Nilda.Fulmer@LKmCL.com|2452530| +2751|AAAAAAAAPLKAAAAA|880143|4662|25134|2450555|2450525|Dr.|Amelia|Weeks|N|7|1|1950|MEXICO||Amelia.Weeks@Pv5tRbBCjCiaLT.com|2452377| +2752|AAAAAAAAAMKAAAAA|1784874|2508|47789|2451131|2451101|Mr.|Randall|Christian|Y|17|5|1961|GUADELOUPE||Randall.Christian@vbBtrPDfyz.edu|2452598| +2753|AAAAAAAABMKAAAAA|||3648|2451045|2451015||Robert|||8||1933|||Robert.Chen@cLqjmfm.org|| +2754|AAAAAAAACMKAAAAA|1573591|3205|45572|2451189|2451159|Mrs.|Barbara|Wise|N|21|9|1942|MAYOTTE||Barbara.Wise@Uk3vT4n3UA8cvpj1yy.org|2452501| +2755|AAAAAAAADMKAAAAA|736017|4191|11565|2452247|2452217|Ms.|Andrea|Self|N|4|2|1925|ANTARCTICA||Andrea.Self@FZBrszboiltpnAPs0pM5.edu|2452301| +2756|AAAAAAAAEMKAAAAA|1037269|4503|8537|2452582|2452552|Dr.|Hector|Haney|N|17|1|1930|SLOVENIA||Hector.Haney@pRj.edu|2452571| +2757|AAAAAAAAFMKAAAAA|1737059|2935|26364|2449396|2449366|Dr.|Paul|Martin|Y|22|9|1984|TONGA||Paul.Martin@ao7Orrq5FkL9h.org|2452559| +2758|AAAAAAAAGMKAAAAA|114920|3446|31972|2451951|2451921|Miss|Veronica|Ellison|N|19|8|1956|EL SALVADOR||Veronica.Ellison@L.com|2452481| +2759|AAAAAAAAHMKAAAAA|1603130|3022|35314|2449178|2449148|Miss|Maryellen|Baird|N|23|2|1924|SWEDEN||Maryellen.Baird@dqv.com|2452405| +2760|AAAAAAAAIMKAAAAA|1133650|5554|47011|2449653|2449623|Dr.|Gerald|Mccarthy|Y|27|10|1967|MONGOLIA||Gerald.Mccarthy@eIyYQ8GM7.com|2452479| +2761|AAAAAAAAJMKAAAAA|1259809|4644|15479|2451247|2451217|Sir|Jack|Cuevas|Y|12|10|1955|IRAQ||Jack.Cuevas@tQJiG.org|2452371| +2762|AAAAAAAAKMKAAAAA|376366|2437|32108|2450233|2450203|Mr.|Brian|Rosenberg|N|10|9|1966|ALAND ISLANDS||Brian.Rosenberg@BCbyxm1kHuo.com|2452520| +2763|AAAAAAAALMKAAAAA|1553020|6176|33732|2450274|2450244|Dr.|Rhona|Flaherty|N|21|8|1979|TRINIDAD AND TOBAGO||Rhona.Flaherty@2.edu|2452523| +2764|AAAAAAAAMMKAAAAA|1776498|812|19479|2451597|2451567|Dr.|Robert|Galvan|Y|17|10|1981|GUINEA-BISSAU||Robert.Galvan@rzsEJpNczz3KYE.com|2452635| +2765|AAAAAAAANMKAAAAA|705796|5228|37460|2451174|2451144|Dr.|Gloria|Fagan|Y|5|11|1959|NORWAY||Gloria.Fagan@0RECqUZ.org|2452442| +2766|AAAAAAAAOMKAAAAA||724|17833||||||N|23|9|1962||||| +2767|AAAAAAAAPMKAAAAA|712884|2236|38435|2451706|2451676|Dr.|Joseph|Bush|N|31|1|1925|CAPE VERDE||Joseph.Bush@mVH.edu|2452373| +2768|AAAAAAAAANKAAAAA|1612832|7052|32072|2449834|2449804|Ms.|Fannie|Young|N|11|5|1942|LIBERIA||Fannie.Young@qQAhduaM3D2.org|2452447| +2769|AAAAAAAABNKAAAAA|712585|1426|17067|2450703|2450673|Dr.|Odette|Foster|N|24|7|1941|FRENCH POLYNESIA||Odette.Foster@l1zFypqmEM8a.com|2452384| +2770|AAAAAAAACNKAAAAA|504283|5658|3033|2450620|2450590|Dr.|Evelyn|Williams|N|15|2|1962|MONTENEGRO||Evelyn.Williams@iXxSZpQ.edu|2452314| +2771|AAAAAAAADNKAAAAA|1468009|2676|17790|2450406|2450376|Dr.|Ethel|Green|N|12|9|1957|ISRAEL||Ethel.Green@Oh0Aaj.org|2452633| +2772|AAAAAAAAENKAAAAA|239503|1805|24907|2450697|2450667|Ms.|Helen|Bishop|N|1|7|1988|KYRGYZSTAN||Helen.Bishop@qiJzPiIboFZEOVS.org|2452573| +2773|AAAAAAAAFNKAAAAA|1142714|1881|648|2451773|2451743|Sir|George|Hendricks|N|14|10|1961|AUSTRALIA||George.Hendricks@P7kzDLXZQX.org|2452589| +2774|AAAAAAAAGNKAAAAA|1341408|5530|18099|2449465|2449435|Dr.|Miriam|Lowe|N|24|10|1925|ISRAEL||Miriam.Lowe@G9IE9Lyfv1Ce.org|2452476| +2775|AAAAAAAAHNKAAAAA|55757|505|43700|2451260|2451230|Mr.|Larry|Lockwood|Y|5|7|1949|SLOVENIA||Larry.Lockwood@CvNvEvaHjY.edu|2452319| +2776|AAAAAAAAINKAAAAA|637762|3812|19815|2450861|2450831|Dr.|Duane|Singleton|Y|4|2|1975|YEMEN||Duane.Singleton@8Eq2P9BRe.edu|2452521| +2777|AAAAAAAAJNKAAAAA|836998|3573|30148|2450745|2450715|Dr.|Rodney|Amador|N|13|10|1942|SOUTH AFRICA||Rodney.Amador@JgyloAU.org|2452286| +2778|AAAAAAAAKNKAAAAA|1809271|1264|42010|2449243|2449213|Sir|Clarence|Campbell|Y|18|7|1955|ANGOLA||Clarence.Campbell@EzlFc.edu|2452294| +2779|AAAAAAAALNKAAAAA|388740|3050|26113|2449804|2449774|Miss|Sue|Pearson|N|26|11|1949|PAKISTAN||Sue.Pearson@dVRrjRBIi6K.org|2452314| +2780|AAAAAAAAMNKAAAAA|1372418|5293|15368|2450150|2450120|Sir|Matthew|Dunn|N|2|8|1929|NETHERLANDS ANTILLES||Matthew.Dunn@zDApF.org|2452366| +2781|AAAAAAAANNKAAAAA|594056|1301|39746|2449505|2449475|Dr.|Barbara|Lopez|N|10|9|1948|FIJI||Barbara.Lopez@dZzvF40.edu|2452510| +2782|AAAAAAAAONKAAAAA|||20779|2450317|2450287|Sir|Bobbie|Riley|||||GIBRALTAR|||| +2783|AAAAAAAAPNKAAAAA|1119029|2919|41962|2451260|2451230|Sir|Earl|Perez|Y|5|10|1988|BARBADOS||Earl.Perez@gjydvsGY3UvA30.org|2452629| +2784|AAAAAAAAAOKAAAAA|215438||41366|2450157|2450127|Miss|||Y|11||1940||||| +2785|AAAAAAAABOKAAAAA|1755956|2394|46832|2449239|2449209|Ms.|Barbara|Mccartney|Y|26|6|1945|MAYOTTE||Barbara.Mccartney@o2VX3nAYks.org|2452344| +2786|AAAAAAAACOKAAAAA|1151803|6773|12943|2451807|2451777|Sir|Steven|Cathey|Y|5|7|1960|PITCAIRN||Steven.Cathey@vuaxapFckSk.org|2452478| +2787|AAAAAAAADOKAAAAA|1425442|2963|29284|2449146|2449116|Sir|Randy|Houser|Y|4|3|1960|ERITREA||Randy.Houser@Ub1.org|2452460| +2788|AAAAAAAAEOKAAAAA|1113410|5270|43883|2450330|2450300|Ms.|Mamie|Fisher|N|26|6|1949|MOZAMBIQUE||Mamie.Fisher@SMeRLD2T1r.edu|2452373| +2789|AAAAAAAAFOKAAAAA|137753|5532|21311|2450848|2450818|Mrs.|Josefina|Nelson|Y|18|8|1943|GUAM||Josefina.Nelson@9IVEA3dzuJlb.org|2452455| +2790|AAAAAAAAGOKAAAAA|||24634|2450496|2450466|Mr.|||Y||||||John.Wright@0QhhU4daUuLH.com|| +2791|AAAAAAAAHOKAAAAA|1099932|5464|28596|2451109|2451079|Mr.|Leon|Gurley|Y|3|11|1939|GIBRALTAR||Leon.Gurley@xxsm1i16QtQ19fcJXAH.org|2452526| +2792|AAAAAAAAIOKAAAAA|1834758|322|40733|2450248|2450218|Dr.|Theodore|Dawson|Y|24|6|1945|GREECE||Theodore.Dawson@XY4HA3DDUz3p.com|2452337| +2793|AAAAAAAAJOKAAAAA|1132415|4695|33375|2449054|2449024|Miss|Susan|Griffin|Y|19|8|1950|FRENCH GUIANA||Susan.Griffin@HsINtKVJa3.com|2452549| +2794|AAAAAAAAKOKAAAAA|1140693|3021|18949|2451732|2451702|Mrs.|Sabrina|Cable|N|20|6|1941|SERBIA||Sabrina.Cable@AfUOygeIl.org|2452509| +2795|AAAAAAAALOKAAAAA|1828037|2935|5080|2449582|2449552|Sir|Jack|Siler|Y|20|9|1943|GHANA||Jack.Siler@3l7D44t5qHpZuiHd.org|2452504| +2796|AAAAAAAAMOKAAAAA|1648226|2071|763|2449980|2449950|Mr.|Johnny|Adams|Y|26|3|1950|TURKMENISTAN||Johnny.Adams@2m5q6spcLx.com|2452323| +2797|AAAAAAAANOKAAAAA|84774|537|19100|2452579|2452549|Miss|Albert|Esposito|Y|5|6|1955|INDIA||Albert.Esposito@Mx.org|2452499| +2798|AAAAAAAAOOKAAAAA|495323|4855|28834|2452341|2452311|Sir|Lance|Brandon|N|19|6|1973|WESTERN SAHARA||Lance.Brandon@l5.org|2452467| +2799|AAAAAAAAPOKAAAAA|967358|6872|11986|2449379|2449349|Dr.|Ladonna|Evans|Y|21|12|1970|ANTARCTICA||Ladonna.Evans@SgRkLBgTUO1siByop.com|2452571| +2800|AAAAAAAAAPKAAAAA|1112800|2041|43682|2452292|2452262|Mrs.|Lacey|Bell|N|28|3|1925|UKRAINE||Lacey.Bell@cx4.com|2452408| +2801|AAAAAAAABPKAAAAA|1490483|4656|4829|2449582|2449552|Miss|Joyce|Hall|N|30|8|1948|CZECH REPUBLIC||Joyce.Hall@ixD7YD1jqNogVRO.org|2452395| +2802|AAAAAAAACPKAAAAA|1831090|5421|31752|2449237|2449207|Dr.|Brian|Dixon|N|8|12|1952|FRENCH GUIANA||Brian.Dixon@ocXPHzm.org|2452526| +2803|AAAAAAAADPKAAAAA|1240186|3963|2093|2449407|2449377|Ms.|Tricia|Hunt|N|3|2|1992|UZBEKISTAN||Tricia.Hunt@LqloKE3NFp0T.org|2452548| +2804|AAAAAAAAEPKAAAAA|1355479|6728|377|2449965|2449935|Dr.|Michael|Rodgers|N|18|9|1968|ARMENIA||Michael.Rodgers@yvINeyJRO.org|2452569| +2805|AAAAAAAAFPKAAAAA|136753|6526|40252|2450509|2450479|Mr.|Henry|Hammond|Y|30|12|1944|GREENLAND||Henry.Hammond@teYjs2h2OgUrzefc.edu|2452641| +2806|AAAAAAAAGPKAAAAA|3465|4310|33908|2452544|2452514|Mrs.|Amanda|Archer|N|18|11|1956|VANUATU||Amanda.Archer@yERjjRg1Sn.edu|2452643| +2807|AAAAAAAAHPKAAAAA|216503|784|21465|2452242|2452212|Dr.|Marilyn|Richards|Y|20|11|1980|SLOVENIA||Marilyn.Richards@5ymO60xzCt.com|2452437| +2808|AAAAAAAAIPKAAAAA|267352|5565|3331|2452606|2452576|Dr.|Betty|Andrews|N|9|4|1948|FRENCH GUIANA||Betty.Andrews@LLe8ibe0Giq.org|2452620| +2809|AAAAAAAAJPKAAAAA|721101|5761|48021|2452049|2452019|Mrs.|Bernadette|Dulaney|Y|7|11|1961|EQUATORIAL GUINEA||Bernadette.Dulaney@BPeR5xS.com|2452550| +2810|AAAAAAAAKPKAAAAA|212424|5126|39475|2449073|2449043|Sir|James|Ellis|Y|29|4|1963|DOMINICA||James.Ellis@BseBI.com|2452354| +2811|AAAAAAAALPKAAAAA|94790|6970|36303|2451192|2451162|Miss|Brandy|Reyna|N|14|6|1966|ANGOLA||Brandy.Reyna@N49mjHhRX3yUD.org|2452328| +2812|AAAAAAAAMPKAAAAA|802069|994|32766|2451810|2451780|Sir|Ronald|Petty|N|9|10|1939|GAMBIA||Ronald.Petty@uz8A.org|2452550| +2813|AAAAAAAANPKAAAAA|1014252|4126|36353|2449260|2449230|Mrs.|Debora|Clinton|N|18|4|1969|LUXEMBOURG||Debora.Clinton@iN.org|2452308| +2814|AAAAAAAAOPKAAAAA|783147|1827|20907|2449283|2449253|Dr.|Michelle|Scott|N|14|1|1927|MOROCCO||Michelle.Scott@zJ1zohTAqMOFc.edu|2452551| +2815|AAAAAAAAPPKAAAAA|324416|94|4525|2451906|2451876|Dr.|Ann|Graf|N|18|7|1969|ISLE OF MAN||Ann.Graf@jSxvBJI07O4t.org|2452418| +2816|AAAAAAAAAALAAAAA|151246|238|37230|2449718|2449688|Mrs.|Marcia|Pepper|N|1|10|1924|PHILIPPINES||Marcia.Pepper@KG1XqMX.edu|2452326| +2817|AAAAAAAABALAAAAA|91307|611|42715|2449028|2448998|Ms.|Ana|Jensen|N|19|8|1933|BERMUDA||Ana.Jensen@m.org|2452328| +2818|AAAAAAAACALAAAAA|1023344|1780|38616|2449636|2449606|Miss|Jacqueline|Levy|Y|16|6|1942|MALI||Jacqueline.Levy@QMBQLQN.org|2452448| +2819|AAAAAAAADALAAAAA|213582|5113|46644|2451314|2451284|Dr.|Richard|Mcgowan|Y|18|2|1937|HONDURAS||Richard.Mcgowan@nvYh2kNEN.edu|2452598| +2820|AAAAAAAAEALAAAAA|295117|1084|42251|2449265|2449235|Sir|Michael|Nesbitt|Y|29|1|1953|DJIBOUTI||Michael.Nesbitt@xOBT.com|2452397| +2821|AAAAAAAAFALAAAAA|487176|375|47221|2449176|2449146|Dr.|Donald|Hunter|N|15|9|1984|PANAMA||Donald.Hunter@okLGzQHRj3Tsym.org|2452506| +2822|AAAAAAAAGALAAAAA|1081449|4475|26254|2451347|2451317|Mrs.|Edna|Hall|N|12|8|1940|KENYA||Edna.Hall@tjAExX2dO3GMVKS.org|2452442| +2823|AAAAAAAAHALAAAAA|999306|269|40832|2449859|2449829|Mr.|Michael|Harris|Y|28|7|1967|TIMOR-LESTE||Michael.Harris@dfiximImI.com|2452302| +2824|AAAAAAAAIALAAAAA|86935|5221|42307|2452674|2452644|Ms.|Arthur|Smith|N|13|10|1928|PUERTO RICO||Arthur.Smith@DOnG.com|2452609| +2825|AAAAAAAAJALAAAAA|1180462|1355|35848|2451278|2451248|Dr.|Sharon|Tolbert|Y|22|8|1960|CHILE||Sharon.Tolbert@lHLcvYn.edu|2452469| +2826|AAAAAAAAKALAAAAA|1581140|5089|12685|2450015|2449985|Dr.|Twana|Jung|Y|25|6|1984|AUSTRIA||Twana.Jung@o3FG6czieJtRULCkh.org|2452443| +2827|AAAAAAAALALAAAAA|552980|3668|42803|2450542|2450512|Dr.|Jennifer|Breen|N|23|5|1977|NIUE||Jennifer.Breen@g1.org|2452476| +2828|AAAAAAAAMALAAAAA|1535775|4274|2726|2452504|2452474|Sir|Elwood|Romero|Y|25|2|1946|EL SALVADOR||Elwood.Romero@FCdfMQHP7c.org|2452562| +2829|AAAAAAAANALAAAAA|325098|33|29815|2451765|2451735|Miss|Christine|Donovan|Y|13|6|1991|QATAR||Christine.Donovan@IGkmSxn.org|2452365| +2830|AAAAAAAAOALAAAAA|586760|5302|38644|2452257|2452227|Dr.|Rose|Whitcomb|Y|25|12|1953|GREECE||Rose.Whitcomb@Ej9edsDdfhvZ.org|2452424| +2831|AAAAAAAAPALAAAAA|821124|2478|27832|2449873|2449843|Mrs.|Laura|Stone|N|3|6|1928|GEORGIA||Laura.Stone@UfmMtyylbZvt.com|2452583| +2832|AAAAAAAAABLAAAAA|387325|6824|49123|2450668|2450638|Dr.|Tyler|Bell|Y|6|2|1927|PHILIPPINES||Tyler.Bell@cMICsl.com|2452463| +2833|AAAAAAAABBLAAAAA|273633|1735|19037|2450224|2450194|Mrs.|Daniell|Phelps|N|2|3|1944|TIMOR-LESTE||Daniell.Phelps@zxj6aMJ.org|2452491| +2834|AAAAAAAACBLAAAAA|510932|2547|39715|2449361|2449331|Miss|Paula|Webster|N|21|4|1961|MONTSERRAT||Paula.Webster@ZoEdAgmG.edu|2452405| +2835|AAAAAAAADBLAAAAA|1125943|4741|9892|2450800|2450770|Dr.|Robert|Jackson|N|15|2|1981|LUXEMBOURG||Robert.Jackson@srOLCdGqZpQY.edu|2452644| +2836|AAAAAAAAEBLAAAAA|1886433|6575|25470|2449112|2449082|Dr.|James|White|Y|13|4|1924|WALLIS AND FUTUNA||James.White@n0o5GrggTlYE7r5ji.com|2452507| +2837|AAAAAAAAFBLAAAAA|960621|7139|44410|2451100|2451070|Dr.|John|Reyes|Y|15|9|1983|ESTONIA||John.Reyes@YmC.edu|2452454| +2838|AAAAAAAAGBLAAAAA|1274443|4979|25115|2451332|2451302||Larry|Frank|||5||PARAGUAY||Larry.Frank@V1etBHt3ZhLFFfJC.edu|| +2839|AAAAAAAAHBLAAAAA|1834455|4805|30355|2452135|2452105|Ms.|Mercedes|Barclay|N|7|9|1971|FAROE ISLANDS||Mercedes.Barclay@ozaZyzbN7dFcIr2SPq.com|2452567| +2840|AAAAAAAAIBLAAAAA|1019163|7183|17203|2452523|2452493|Miss|Linda|Gibson|N|28|3|1948|CYPRUS||Linda.Gibson@fI0G8LTjpo99K.org|2452620| +2841|AAAAAAAAJBLAAAAA|683292|6589|1439|2451364|2451334|Mrs.|Teresa|Martinez|Y|22|10|1984|DJIBOUTI||Teresa.Martinez@3SrU6cYNRTkLK.org|2452393| +2842|AAAAAAAAKBLAAAAA|8377|1858|42092|2452109|2452079|Sir|Jessie|Wilson|N|19|6|1931|MOROCCO||Jessie.Wilson@Af8OBg.edu|2452619| +2843|AAAAAAAALBLAAAAA|11583|5|17274|2449723|2449693|Miss|Sadie|Rogers|N|17|8|1946|SIERRA LEONE||Sadie.Rogers@sLSN3nBjv5Esv.com|2452362| +2844|AAAAAAAAMBLAAAAA|780312|6735|10758|2451168|2451138|Mr.|Charles|Cruz|N|9|5|1938|SAN MARINO||Charles.Cruz@LPMszoydd9Cf.com|2452646| +2845|AAAAAAAANBLAAAAA|818484|2106|24928|2450899|2450869|Dr.|Krista|West|Y|28|2|1986|MYANMAR||Krista.West@kdBVM.com|2452622| +2846|AAAAAAAAOBLAAAAA|423338|2696|29254|2449851|2449821|Mrs.|Blanche|Mcdougal|Y|7|1|1968|GEORGIA||Blanche.Mcdougal@HNM33SgrvU.com|2452587| +2847|AAAAAAAAPBLAAAAA|84496|3197|33498|2450441|2450411|Dr.|Stephanie|Crockett|Y|23|9|1946|DJIBOUTI||Stephanie.Crockett@9jZ7.edu|2452321| +2848|AAAAAAAAACLAAAAA|778579|3027|47875|2449306|2449276|Sir|Edwin|Carter|N|14|8|1965|BURKINA FASO||Edwin.Carter@IfrPLl0t04o7pTB0.com|2452351| +2849|AAAAAAAABCLAAAAA|1844476|1661|10046|2451824|2451794|Miss|Barbara|Hill|N|9|11|1950|VANUATU||Barbara.Hill@zhUIinVs0R0oPG7IPb.edu|2452633| +2850|AAAAAAAACCLAAAAA|916536|5589|9795|2449692|2449662|Miss|Kathleen|Rider|N|11|8|1990|MADAGASCAR||Kathleen.Rider@dkvodvKcTAH.edu|2452610| +2851|AAAAAAAADCLAAAAA|1359544|1904|35017|2449977|2449947|Mrs.|Mildred|Mancuso|Y|12|4|1962|LIBERIA||Mildred.Mancuso@5Y.com|2452418| +2852|AAAAAAAAECLAAAAA|1181915|4716|14717|2451945|2451915|Mr.|James|Garcia|N|17|9|1970|TIMOR-LESTE||James.Garcia@cJN5meCekDXRG3.com|2452640| +2853|AAAAAAAAFCLAAAAA|1405201|6375|48302|2450156|2450126|Mr.|Joel|Hamilton|N|20|10|1924|CAMBODIA||Joel.Hamilton@rvXYVHI9itP.org|2452518| +2854|AAAAAAAAGCLAAAAA|675341|1463|23538|2451184|2451154|Dr.|Robert|Payne|N|11|9|1930|ARGENTINA||Robert.Payne@RNk8hP5ovjTBA.org|2452310| +2855|AAAAAAAAHCLAAAAA|1670319|6872|5378|2451170|2451140|Sir|James|Williams|Y|5|10|1937|HUNGARY||James.Williams@0QMqG.com|2452306| +2856|AAAAAAAAICLAAAAA|1288143|2162|45781|2451042|2451012|Mr.|Jeffery|German|Y|28|1|1958|IRAQ||Jeffery.German@5.org|2452615| +2857|AAAAAAAAJCLAAAAA|986227|3909|45638|2452293|2452263|Mr.|Nicholas|Alvarado|Y|22|12|1961|SAMOA||Nicholas.Alvarado@CeG2d8NrdKCgez.com|2452560| +2858|AAAAAAAAKCLAAAAA|241583|5843|46503|2450514|2450484|Sir|Chad|Boswell|Y|31|1|1934|MALAWI||Chad.Boswell@uq.edu|2452507| +2859|AAAAAAAALCLAAAAA|1477227|651|30475|2450057|2450027|Dr.|Emily|Parsons|N|12|10|1987|VANUATU||Emily.Parsons@bzUe.edu|2452387| +2860|AAAAAAAAMCLAAAAA||1081|35567|2452600|2452570|Sir|||||5|1938|||Frank.Schmitt@3iOYovA.com|2452617| +2861|AAAAAAAANCLAAAAA|1306068|981|3541|2450474|2450444|Dr.|Lisa|Johnson|N|28|10|1947|TOGO||Lisa.Johnson@igiXpZUlGSEn.org|2452601| +2862|AAAAAAAAOCLAAAAA|1862258|4117|39976|2452164|2452134|Dr.|Lori|Moore|Y|30|3|1990|DENMARK||Lori.Moore@vFD8h6EATBdHMM7n1p.com|2452455| +2863|AAAAAAAAPCLAAAAA|352880|5495|29710|2449169|2449139|Mr.|Louis|Townsend|Y|23|12|1950|BURKINA FASO||Louis.Townsend@8vn5jvBN0Q.com|2452603| +2864|AAAAAAAAADLAAAAA|11291|2161|32678|2449456|2449426|Dr.|Walter|Gage|N|12|6|1973|GHANA||Walter.Gage@9u.org|2452640| +2865|AAAAAAAABDLAAAAA|72773|6817|21708|2451918|2451888|Mrs.|Jennifer|Mahan|Y|22|6|1940|MALAYSIA||Jennifer.Mahan@XLhaEydVNzx.edu|2452290| +2866|AAAAAAAACDLAAAAA|1889434|6901|49168|2451154|2451124|Sir|Jimmie|Mcmillan|Y|13|2|1965|RUSSIAN FEDERATION||Jimmie.Mcmillan@8mJSsfAz6YS.org|2452496| +2867|AAAAAAAADDLAAAAA|1389894|5194|9647|2449118|2449088|Sir|Dale|Munoz|Y|29|9|1937|ISRAEL||Dale.Munoz@JCTFKqUf8SQ3tM.com|2452462| +2868|AAAAAAAAEDLAAAAA|252342|3436|4083|2451502|2451472|Dr.|John|Keith|Y|27|11|1973|EGYPT||John.Keith@bN0.com|2452446| +2869|AAAAAAAAFDLAAAAA|235368|4705|45289|2450119|2450089|Miss|Sherri|Bryant|N|2|5|1932|SOUTH AFRICA||Sherri.Bryant@VJTSNs.org|2452351| +2870|AAAAAAAAGDLAAAAA|261585|2793|28801|2452555||Dr.||Decker|N|||1941||||2452299| +2871|AAAAAAAAHDLAAAAA|1410417|6765|13513|2450178|2450148|Miss|Rosalina|Blake|Y|27|6|1938|JAMAICA||Rosalina.Blake@SbF4dUjZa.org|2452304| +2872|AAAAAAAAIDLAAAAA|1062287|4703|27072|2449432|2449402|Miss|Ethel|Dahl|N|25|1|1967|VENEZUELA||Ethel.Dahl@Qs2x7SkyU.com|2452481| +2873|AAAAAAAAJDLAAAAA|1396324|975|473|2449415|2449385|Miss|Annie|Thomas|Y|14|12|1975|SWEDEN||Annie.Thomas@z.edu|2452309| +2874|AAAAAAAAKDLAAAAA|1678029|4539|42249|2451928|2451898|Sir|Nicholas|Newsome|N|30|10|1958|NIGERIA||Nicholas.Newsome@0bL7RfaJjGrnbn.edu|2452335| +2875|AAAAAAAALDLAAAAA|1526148|1143|19738|2452398|2452368|Dr.|Bettie|Longoria|Y|20|3|1964|MOLDOVA, REPUBLIC OF||Bettie.Longoria@OzE1E6fTZm.org|2452355| +2876|AAAAAAAAMDLAAAAA|463528|3627|27326|2450350|2450320|Miss|Marcia|Gray|Y|11|7|1983|BURKINA FASO||Marcia.Gray@YDaC0HvP0k.edu|2452613| +2877|AAAAAAAANDLAAAAA|1499339|1877|32415|2449922|2449892|Miss|Alvina|Patton|N|13|7|1948|MONGOLIA||Alvina.Patton@Tn4YNnN9tEi.com|2452565| +2878|AAAAAAAAODLAAAAA|814825|3124|16623|2450588|2450558|Dr.|Cary|Brown|N|21|7|1968|IRELAND||Cary.Brown@vVyT7q9Bf8p.org|2452326| +2879|AAAAAAAAPDLAAAAA|1090565|2883|49648|2451732|2451702|Dr.|Robert|Terrell|N|28|3|1987|SURINAME||Robert.Terrell@ueeQ.com|2452436| +2880|AAAAAAAAAELAAAAA|1497029|1801|21773|2450631|2450601|Mr.|Edward|Lee|Y|14|7|1952|GIBRALTAR||Edward.Lee@MoyJiBbcyXP.edu|2452641| +2881|AAAAAAAABELAAAAA|146396|5173|29150|2449193|2449163|Mrs.|Marjorie|Vigil|N|8|7|1969|AUSTRIA||Marjorie.Vigil@qmZAZ2yC.edu|2452558| +2882|AAAAAAAACELAAAAA|307902|5646|1390|2451234|2451204|Ms.|Lydia|Mena|Y|28|11|1956|ARMENIA||Lydia.Mena@xFHxgJgST22.org|2452430| +2883|AAAAAAAADELAAAAA|1102369|362|27108|2451118|2451088|Dr.|Beth|Jones|N|17|1|1971|BANGLADESH||Beth.Jones@En5x.com|2452515| +2884|AAAAAAAAEELAAAAA|1897088|994|21347|2449126|2449096|Mr.|Thomas|Arnold|N|1|10|1955|LESOTHO||Thomas.Arnold@TJ88q1.edu|2452576| +2885|AAAAAAAAFELAAAAA|882458|5975|45939|2449857|2449827|Dr.|Elenore|Hunt|N|22|11|1982|SOMALIA||Elenore.Hunt@1hbkZItAYE.org|2452483| +2886|AAAAAAAAGELAAAAA|591041|6852|35863|2449750|2449720|Ms.|Peggy|Pham|N|30|6|1982|LATVIA||Peggy.Pham@Ka5v.com|2452396| +2887|AAAAAAAAHELAAAAA|132586|4118|48897|2452110|2452080|Sir|Justin|Ho|Y|19|4|1943|MONTENEGRO||Justin.Ho@bXNB8SxVoXEKZR.com|2452511| +2888|AAAAAAAAIELAAAAA|1767248|5540|29832|2450682|2450652|Miss|Betty|Jorgenson|Y|2|10|1938|LEBANON||Betty.Jorgenson@UH.edu|2452460| +2889|AAAAAAAAJELAAAAA|1420419|3302|19385|2450399|2450369|Miss|Katherine|Blackburn|Y|1|11|1948|SAINT HELENA||Katherine.Blackburn@aXtToVs7DgsgF0.com|2452472| +2890|AAAAAAAAKELAAAAA|1619209|3369|45929|2449223|2449193|Mr.|James|Seal|Y|30|11|1934|HONG KONG||James.Seal@cqisT6YAv8pb.edu|2452575| +2891|AAAAAAAALELAAAAA|11586|3714|23795|2449055|2449025|Mrs.|Carolyn|Petrie|N|7|5|1965|URUGUAY||Carolyn.Petrie@zNIPnf.org|2452399| +2892|AAAAAAAAMELAAAAA|1475518|7104|33090|2450437|2450407|Sir|Herman|Peralta|Y|29|9|1944|VANUATU||Herman.Peralta@O2CfpA9Pk1nLEP.edu|2452456| +2893|AAAAAAAANELAAAAA|874780|1985|29637|2449919|2449889|Miss|Joyce|Desjardins|Y|18|12|1990|MALAYSIA||Joyce.Desjardins@eINbZEoYXHn.edu|2452370| +2894|AAAAAAAAOELAAAAA|1626698|5063|22228|2449517|2449487|Dr.|Myrtis|Smith|Y|9|6|1926|FRENCH GUIANA||Myrtis.Smith@lLfkn9a.edu|2452637| +2895|AAAAAAAAPELAAAAA|813195|5303|46211|2449090|2449060|Miss|Andre|Hernandez|Y|7|4|1929|FRENCH POLYNESIA||Andre.Hernandez@o8MXgIt.com|2452375| +2896|AAAAAAAAAFLAAAAA|1571034|1664|20341|2452648|2452618|Dr.|Myles|Brothers|N|9|10|1943|GAMBIA||Myles.Brothers@VhsRLdgS1QjJJgTaKM.com|2452551| +2897|AAAAAAAABFLAAAAA|501979|1776|22373|2450622|2450592|Dr.|Barbara|Martinez|N|26|9|1952|POLAND||Barbara.Martinez@OsDvMtNe8vI.org|2452423| +2898|AAAAAAAACFLAAAAA|678433|6513|9393|2449106|2449076|Mrs.|Edith|Hood|Y|22|6|1958|LITHUANIA||Edith.Hood@v1Vm4KD44UkLEIECr.edu|2452513| +2899|AAAAAAAADFLAAAAA|170125|5790|32702|2450118|2450088|Dr.|Anabel|Anderson|Y|19|7|1929|ERITREA||Anabel.Anderson@KaUDH66J.com|2452504| +2900|AAAAAAAAEFLAAAAA|||6131|2451888|2451858|Dr.|||||6|1942||||2452300| +2901|AAAAAAAAFFLAAAAA|326933|2021|22356|2449369|2449339|Miss|Luz|Montgomery|N|5|10|1945|LATVIA||Luz.Montgomery@s01kVmjz.com|2452647| +2902|AAAAAAAAGFLAAAAA|798107|6494|37281|2450926|2450896|Dr.|Jose|Gonzales|Y|27|2|1966|JERSEY||Jose.Gonzales@08moCKq4buDucGX.com|2452503| +2903|AAAAAAAAHFLAAAAA|1140682|4117|5599|2452015|2451985|Dr.|Kevin|Castillo|Y|3|1|1932|JAPAN||Kevin.Castillo@alhZzDJZs8lXQ.com|2452448| +2904|AAAAAAAAIFLAAAAA|479470|3966|39262|2451050|2451020|Dr.|Caroline|Moore|Y|16|1|1936|MOLDOVA, REPUBLIC OF||Caroline.Moore@9cRbAgzLQFm.com|2452599| +2905|AAAAAAAAJFLAAAAA|76805|1459|19313|2449613|2449583|Sir|Martin|Martinez|Y|23|9|1938|ISLE OF MAN||Martin.Martinez@vI.com|2452538| +2906|AAAAAAAAKFLAAAAA|605793|4845|23544|2449903|2449873|Mr.|John|Marin|Y|19|2|1972|MYANMAR||John.Marin@E93xZ8.org|2452418| +2907|AAAAAAAALFLAAAAA|1181182|3124|9066|2449556|2449526|Sir|Steven|Adams|N|1|2|1928|FRANCE||Steven.Adams@ritchr0.edu|2452560| +2908|AAAAAAAAMFLAAAAA|214002|4235|5315|2451250|2451220|Miss|Adrian|Riggs|N|21|2|1988|SYRIAN ARAB REPUBLIC||Adrian.Riggs@U2hi.edu|2452609| +2909|AAAAAAAANFLAAAAA|1607349|2255|13961|2451939|2451909|Dr.|Adam|Mayfield|Y|3|3|1931|HONG KONG||Adam.Mayfield@KQuCE8Klyf.com|2452544| +2910|AAAAAAAAOFLAAAAA|1102630|3118|38806|2449385|2449355|Dr.|Charles|Pearce|N|2|7|1944|PHILIPPINES||Charles.Pearce@NcMM2.org|2452409| +2911|AAAAAAAAPFLAAAAA|1165011|515|40819|2450630|2450600|Dr.|Tony|Spencer|N|22|7|1941|PORTUGAL||Tony.Spencer@rVT4TMopmGuIVHJ.com|2452377| +2912|AAAAAAAAAGLAAAAA|196926|5725|33324|2449763|2449733|Dr.|Bobby|Levin|Y|22|2|1939|MAURITIUS||Bobby.Levin@4SGM.org|2452403| +2913|AAAAAAAABGLAAAAA|1352224|1242|37907|2451651|2451621|Dr.|Earl|Zamora|Y|15|4|1972|PITCAIRN||Earl.Zamora@T.com|2452623| +2914|AAAAAAAACGLAAAAA|766164|6456|36795|2451721|2451691|Mr.|John|Smith|N|15|3|1991|UNITED ARAB EMIRATES||John.Smith@DrLRMuLpR2ZH1.org|2452541| +2915|AAAAAAAADGLAAAAA|1595685|2284|30883|2450746|2450716|Sir|Robert|Duggan|Y|14|12|1977|MAURITIUS||Robert.Duggan@MzMqeM0ZMIF.org|2452337| +2916|AAAAAAAAEGLAAAAA|1215516|6861|29006|2450215|2450185|Sir|Jesse|Babcock|Y|13|10|1945|JAPAN||Jesse.Babcock@NbMq4.edu|2452344| +2917|AAAAAAAAFGLAAAAA|620266|5672|13478|2451617|2451587|Dr.|Frank|Snider|N|23|2|1978|PALAU||Frank.Snider@iLfsBvnMgu.com|2452528| +2918|AAAAAAAAGGLAAAAA|306710|361|36412|2451833|2451803|Sir|Roger|Skinner|Y|18|7|1964|AUSTRIA||Roger.Skinner@ZBjJssBpQfY75Jx.com|2452473| +2919|AAAAAAAAHGLAAAAA|327161|2239|564|2451901|2451871|Sir|Martin|Adair|N|22|1|1964|BOTSWANA||Martin.Adair@Vozi.edu|2452392| +2920|AAAAAAAAIGLAAAAA|1540033|4251|20422|2452083|2452053|Miss|Sarah|Brown|N|1|6|1941|GUADELOUPE||Sarah.Brown@zUc35icQ4x.edu|2452570| +2921|AAAAAAAAJGLAAAAA|1379233|6516|30646|2451758|2451728|Mr.|Daniel|Washington|Y|1|10|1953|YEMEN||Daniel.Washington@eQncTDzbsqeTs4F.edu|2452324| +2922|AAAAAAAAKGLAAAAA|172565|4181|26950|2452032|2452002|Miss|Thomasina|Hernandez|N|23|12|1956|SOLOMON ISLANDS||Thomasina.Hernandez@M2.com|2452554| +2923|AAAAAAAALGLAAAAA|226985|6731|47677|2449799|2449769|Sir|Brian|Gagnon|Y|28|10|1935|TAJIKISTAN||Brian.Gagnon@AjuOAgLJXUYj.com|2452289| +2924|AAAAAAAAMGLAAAAA|1734749|5042|6135|2449104|2449074|Dr.|Donald|Diaz|Y|30|3|1950|ERITREA||Donald.Diaz@BRA0eCqv3SlQd.org|2452393| +2925|AAAAAAAANGLAAAAA|342512|1333|48507|2449759|2449729|Mrs.|Dorothy|Jackson|Y|31|7|1943|MALAWI||Dorothy.Jackson@oekzsq41pD4mSxtpn.edu|2452620| +2926|AAAAAAAAOGLAAAAA|545672|6600|43928|2450128|2450098|Dr.|Susan|Barton|Y|13|12|1978|JORDAN||Susan.Barton@JmMi.org|2452436| +2927|AAAAAAAAPGLAAAAA|813132|6956|37180|2449695|2449665|Mr.|David|Jackson|Y|10|7|1971|HONDURAS||David.Jackson@LPqT0SOaRfs8ASvLf.edu|2452645| +2928|AAAAAAAAAHLAAAAA|||37161|2451500|||Romona|Vickers|N|30|12|1940|MACAO|||2452523| +2929|AAAAAAAABHLAAAAA|1345032|6990|48548|2452048|2452018|Dr.|Lonnie|White|Y|29|8|1972|ARGENTINA||Lonnie.White@9VF9xtsT7BABsDM.com|2452607| +2930|AAAAAAAACHLAAAAA|||38353||2449908|||||30||1939|||Estella.Conway@PT0cNaq3.org|| +2931|AAAAAAAADHLAAAAA|1804751|3277|43803|2452391|2452361|Miss|Diane|Taylor|N|5|5|1927|BOTSWANA||Diane.Taylor@mqi0ddyLva.com|2452498| +2932|AAAAAAAAEHLAAAAA|959410|3069|35928|2449450|2449420|Miss|Kayla|Elliott|N|28|2|1966|TURKEY||Kayla.Elliott@Xl2TkcINy031G8OP.com|2452448| +2933|AAAAAAAAFHLAAAAA|304169|48|8727|2449206|2449176|Dr.|Ruby|Lloyd|N|28|9|1968|PANAMA||Ruby.Lloyd@NPICCRUyxEaE1.com|2452571| +2934|AAAAAAAAGHLAAAAA|795684|6819|23154|2451420|2451390|Dr.|Gracie|Larson|Y|20|9|1969|CAMEROON||Gracie.Larson@LpaboP5mA4.com|2452583| +2935|AAAAAAAAHHLAAAAA|1040401|2538|5475|2451392|2451362|Ms.|Albert|Norwood|N|29|11|1926|BAHAMAS||Albert.Norwood@MP2CyVC3YRntI.edu|2452593| +2936|AAAAAAAAIHLAAAAA|1305613|2105|40150|2451890|2451860|Mr.|Gerald|Villarreal|N|5|6|1938|SWAZILAND||Gerald.Villarreal@mupjMZyZICVctsFe.com|2452456| +2937|AAAAAAAAJHLAAAAA|269333|750|7767|2449630|2449600|Sir|Robert|Dunn|Y|27|9|1964|KUWAIT||Robert.Dunn@D34voUB1Fm.com|2452464| +2938|AAAAAAAAKHLAAAAA|834764|1215|23713|2449533|2449503|Dr.|Felicia|Furr|N|17|10|1937|UNITED ARAB EMIRATES||Felicia.Furr@NfzV0.com|2452431| +2939|AAAAAAAALHLAAAAA|1380364|6914|16664|2449046|2449016|Dr.|Larry|Hoffman|N|12|7|1968|ANDORRA||Larry.Hoffman@GLtpu7OIQg1LMz.org|2452326| +2940|AAAAAAAAMHLAAAAA|1344665|4031|42997|2450628|2450598|Dr.|Shawn|Stallings|N|2|2|1949|ZAMBIA||Shawn.Stallings@XgiLAZ1LXt0gi.com|2452562| +2941|AAAAAAAANHLAAAAA|1609942|1321|3547|2451087|2451057|Ms.|Selma|Williams|N|29|9|1965|KIRIBATI||Selma.Williams@Ikn6bnfZz6K.edu|2452482| +2942|AAAAAAAAOHLAAAAA|1121932|821|15663|2452342|2452312|Mr.|Nicholas|Larson|N|21|8|1929|MAYOTTE||Nicholas.Larson@csZMzi8asU.org|2452442| +2943|AAAAAAAAPHLAAAAA|1660932|6548|21717|2452219|2452189|Mr.|Todd|Wheatley|N|14|1|1930|BENIN||Todd.Wheatley@tE.org|2452634| +2944|AAAAAAAAAILAAAAA|1230901|2183|19465|2452082|2452052|Dr.|Charles|Clark|N|6|3|1926|SYRIAN ARAB REPUBLIC||Charles.Clark@VTdMaA3Quq.edu|2452462| +2945|AAAAAAAABILAAAAA|677237|2331|39425|2449456|2449426|Mr.|Steven|West|Y|12|7|1953|C�TE D'IVOIRE||Steven.West@IRc4q8l.org|2452635| +2946|AAAAAAAACILAAAAA|1428765||9587|2451650|2451620|Sir||Healey|||6|||||| +2947|AAAAAAAADILAAAAA|1855533|5908|37483|2451294|2451264|Sir|Lawrence|Armstrong|Y|24|5|1974|CROATIA||Lawrence.Armstrong@pYSpyIMq0sdm9cT.edu|2452508| +2948|AAAAAAAAEILAAAAA|282558|402|7046||||Anthony|Shockley||3|3|1950|UGANDA||Anthony.Shockley@3xolokHzSH.com|| +2949|AAAAAAAAFILAAAAA|1752443|2033|34023|2449476|2449446|Mr.|David|Leblanc|N|18|4|1952|MALI||David.Leblanc@3iey7F.org|2452570| +2950|AAAAAAAAGILAAAAA|564542|2881|26773|2449806|2449776|Ms.|Lucille|Cardona|N|3|7|1969|IRELAND||Lucille.Cardona@A2ynrULBZf.edu|2452366| +2951|AAAAAAAAHILAAAAA|857662|1838|36014|2450865|2450835|Ms.|Dolores|Ralph|N|6|5|1963|KENYA||Dolores.Ralph@ZLtpV3dS6H.org|2452309| +2952|AAAAAAAAIILAAAAA|1538598|1600|42202|2452499|2452469|Mrs.|Violet|Betts|N|24|5|1935|SAN MARINO||Violet.Betts@XQtGP6L.com|2452360| +2953|AAAAAAAAJILAAAAA|722948|3877|49206|2449703|2449673|Dr.|Christopher|Brown|N|9|11|1966|PORTUGAL||Christopher.Brown@dxT1Z6s6P95XO3.com|2452368| +2954|AAAAAAAAKILAAAAA|1186073|6135|20597|2452090|2452060|Miss|Jayne|Barlow|Y|10|1|1945|AZERBAIJAN||Jayne.Barlow@TzFXSttfE4uKHp.com|2452320| +2955|AAAAAAAALILAAAAA|221244|6655|35266|2451791|2451761|Dr.|Maurice|Fuller|N|11|11|1929|BURUNDI||Maurice.Fuller@euzJ.edu|2452531| +2956|AAAAAAAAMILAAAAA|1116339|3279|43019|2451181|2451151|Dr.|Joseph|Browder|N|28|1|1945|RUSSIAN FEDERATION||Joseph.Browder@xDJPRurFo2NICQ.org|2452379| +2957|AAAAAAAANILAAAAA||3107|16339||2452498|||||21|||MOROCCO||Joseph.Haight@V.edu|2452371| +2958|AAAAAAAAOILAAAAA|939046||14429||||Paul|||||1934|ALBANIA||Paul.Leblanc@37UrYsgtcrXSdnP1.edu|| +2959|AAAAAAAAPILAAAAA|1306649|2300|28398|2450249|2450219|Dr.|Willie|Sims|Y|8|8|1991|LESOTHO||Willie.Sims@oEkezL0sBylh5LdB.org|2452321| +2960|AAAAAAAAAJLAAAAA|83714|4905|15649|2452485|2452455|Ms.|Theresa|Hutcheson|N|19|2|1977|BURUNDI||Theresa.Hutcheson@3plbUJ.com|2452542| +2961|AAAAAAAABJLAAAAA|1793624|309|21394|2452599|2452569|Sir|Larry|Deaton|Y|19|3|1961|BARBADOS||Larry.Deaton@KopxUNotD6.com|2452554| +2962|AAAAAAAACJLAAAAA|1576580|1890|26400|2451741|2451711|Mrs.|Angela|Rogers|Y|21|1|1966|MAURITANIA||Angela.Rogers@RQ37rv9UteaHk0Y.org|2452397| +2963|AAAAAAAADJLAAAAA|117273|5848|43576|2451199|2451169|Mr.|Robert|Garcia|N|11|11|1986|SWITZERLAND||Robert.Garcia@G1N.com|2452543| +2964|AAAAAAAAEJLAAAAA|1286950|2368|24586|2451920|2451890|Sir|Robert|Brown|Y|15|12|1963|FRENCH GUIANA||Robert.Brown@U9NB2FyFa5H9.com|2452436| +2965|AAAAAAAAFJLAAAAA|155418|947|6523|2449886|2449856|Dr.|Constance|Healy|N|2|1|1947|MARTINIQUE||Constance.Healy@LAuUCe.edu|2452624| +2966|AAAAAAAAGJLAAAAA|439104|2247|12143|2452046|2452016|Mr.|Willie|Tilley|N|3|8|1968|SENEGAL||Willie.Tilley@J0BOSLrAA56HtS.edu|2452483| +2967|AAAAAAAAHJLAAAAA|1887513|3624|26135|2451611|2451581|Dr.|Audrey|Beltran|Y|25|10|1953|SOMALIA||Audrey.Beltran@ZIx7Kk5A4D0vl3O5uH.edu|2452333| +2968|AAAAAAAAIJLAAAAA|1820449|196|30296|2451089|2451059|Dr.|Cynthia|Russell|N|8|9|1968|BULGARIA||Cynthia.Russell@p.org|2452322| +2969|AAAAAAAAJJLAAAAA|||44248|2452427|2452397||||Y||8|1981|KIRIBATI|||| +2970|AAAAAAAAKJLAAAAA|679129|5289|32649|2450374|2450344|Mrs.|Claudine|Pike|N|30|1|1935|BULGARIA||Claudine.Pike@tLKSGVY0r4.edu|2452384| +2971|AAAAAAAALJLAAAAA|471796|830|7911|2451429|2451399|Mr.|Scott|Goodman|Y|5|9|1955|MALAYSIA||Scott.Goodman@EN4g6ehG4eh1.com|2452340| +2972|AAAAAAAAMJLAAAAA|554040|4896|36905|2450265|2450235|Sir|Stephen|Odom|N|11|1|1985|GIBRALTAR||Stephen.Odom@TU.com|2452588| +2973|AAAAAAAANJLAAAAA|1565531|1940|5549|2449571|2449541|Mrs.|Andrea|Rice|Y|1|6|1948|CAPE VERDE||Andrea.Rice@RYIE0Truoe.com|2452475| +2974|AAAAAAAAOJLAAAAA|1733653|5997|49141|2452031|2452001|Miss|Gregoria|Spencer|Y|14|11|1968|MYANMAR||Gregoria.Spencer@UgvkT.edu|2452493| +2975|AAAAAAAAPJLAAAAA|627568|2978|40741|2450038|2450008|Dr.|Hilda|Douglass|Y|3|6|1942|EGYPT||Hilda.Douglass@xfI7uxVurRFnA.edu|2452626| +2976|AAAAAAAAAKLAAAAA|680718|2758|27551|2450411|2450381|Mrs.|Catherine|Witcher|Y|19|6|1933|PAKISTAN||Catherine.Witcher@ejPQ.edu|2452521| +2977|AAAAAAAABKLAAAAA|1239473|5151|26429|2450864|2450834|Miss|Alma|Logan|N|24|6|1928|THAILAND||Alma.Logan@IlbVQrlJUO5Gb7Grpy.edu|2452606| +2978|AAAAAAAACKLAAAAA|1389772|5513|42054|2451736|2451706|Dr.|William|Gomez|Y|19|12|1924|ARUBA||William.Gomez@pM.com|2452579| +2979|AAAAAAAADKLAAAAA|1310004|3331|5905|2449410|2449380|Ms.|Alice|Tyner|N|4|5|1930|INDONESIA||Alice.Tyner@zZ5JleyFVNeI.com|2452410| +2980|AAAAAAAAEKLAAAAA|706410|4087|40517|2452102|2452072|Miss|Betty|Mcbride|Y|3|11|1966|SYRIAN ARAB REPUBLIC||Betty.Mcbride@o04qbbL65dQb.org|2452518| +2981|AAAAAAAAFKLAAAAA|319991|6135|47479|2449616|2449586|Dr.|Mark|Rinehart|N|26|12|1938|LITHUANIA||Mark.Rinehart@K.org|2452493| +2982|AAAAAAAAGKLAAAAA|280924|7108|9739|2451922|2451892|Dr.|Randy|Giordano|N|29|10|1962|GEORGIA||Randy.Giordano@6SHoXNMFSzo3SIyI.edu|2452619| +2983|AAAAAAAAHKLAAAAA|256085|114|43790|2450155|2450125|Ms.|Anita|Ray|N|30|9|1956|PAKISTAN||Anita.Ray@nN81lXJE.com|2452633| +2984|AAAAAAAAIKLAAAAA|817392|6470|35501|2451693|2451663|Mrs.|Caroline|Miles|N|27|5|1978|TRINIDAD AND TOBAGO||Caroline.Miles@lLla2.edu|2452621| +2985|AAAAAAAAJKLAAAAA|90778|5034|34043|2449076|2449046|Dr.|Tessa|Washington|N|23|2|1924|CAMBODIA||Tessa.Washington@R8.com|2452605| +2986|AAAAAAAAKKLAAAAA|1656241|6198|36155|2450106|2450076|Ms.|Karen|Harris|N|13|6|1956|NORFOLK ISLAND||Karen.Harris@1j108Zgf.edu|2452604| +2987|AAAAAAAALKLAAAAA|1057897|1554|42811|2452376|2452346|Sir|Jeffrey|Jones|N|21|6|1955|UZBEKISTAN||Jeffrey.Jones@ciuIB.com|2452417| +2988|AAAAAAAAMKLAAAAA|1455383|57|8331|2452312|2452282|Sir|Robert|Woodward|N|24|5|1982|MALAYSIA||Robert.Woodward@S.org|2452539| +2989|AAAAAAAANKLAAAAA|963727|3967|29007|2451002|2450972|Mrs.|Alex|Hartman|N|14|6|1954|CHINA||Alex.Hartman@V5Q.com|2452464| +2990|AAAAAAAAOKLAAAAA|419570|2951|16732|2450797|2450767|Mr.|Christian|Foley|N|1|7|1978|BELGIUM||Christian.Foley@qvH.org|2452419| +2991|AAAAAAAAPKLAAAAA|107076|6640|25051|2451320|2451290|Dr.|Leota|Hughes|N|21|12|1972|KAZAKHSTAN||Leota.Hughes@U6r7EUGRAKPU.org|2452475| +2992|AAAAAAAAALLAAAAA|320744|3376|33961|2451475|2451445|Sir|James|Andrews|Y|22|12|1930|MALAWI||James.Andrews@3VuSY3.edu|2452520| +2993|AAAAAAAABLLAAAAA|1203578|2688|38151|2452501|2452471|Ms.|Liliana|Mcmichael|Y|28|11|1963|VIET NAM||Liliana.Mcmichael@jdlo.com|2452442| +2994|AAAAAAAACLLAAAAA|181294|4646|15598|2450961|2450931|Ms.|Holli|Hill|Y|17|10|1988|SYRIAN ARAB REPUBLIC||Holli.Hill@ZU810hHiN.com|2452348| +2995|AAAAAAAADLLAAAAA|1196136|4190|40244|2449872|2449842|Mrs.|Julie|Ferguson|N|9|6|1958|MALAYSIA||Julie.Ferguson@RNEbmOFcla.org|2452621| +2996|AAAAAAAAELLAAAAA|1813722|4435|4301|2449680|2449650|Mr.|Gregory|Miller|Y|29|9|1981|GREENLAND||Gregory.Miller@q3U1bttmv2H0.com|2452416| +2997|AAAAAAAAFLLAAAAA|940918|1435|46799|2449867|2449837|Mr.|Mark|Miller|Y|18|5|1930|SUDAN||Mark.Miller@PNdy.com|2452491| +2998|AAAAAAAAGLLAAAAA|1176665|521|32424|2450591|2450561|Mr.|Christian|Sexton|N|8|6|1969|MOROCCO||Christian.Sexton@EA7f6rP8q0IJz.com|2452615| +2999|AAAAAAAAHLLAAAAA|1513309|1752|46195|2449360|2449330|Mr.|Robert|Cardona|Y|26|7|1984|MALI||Robert.Cardona@clxOy7d.com|2452463| +3000|AAAAAAAAILLAAAAA|1012369|4900|12584|2450437|2450407|Ms.|Susan|Albright|N|2|12|1968|SAMOA||Susan.Albright@CPsHHOc.edu|2452303| +3001|AAAAAAAAJLLAAAAA|919690||49037|2450441|2450411||Karrie|Gray|||||||Karrie.Gray@Fcb1XJeB2EaYa6xff.com|| +3002|AAAAAAAAKLLAAAAA|605033|5739|31292|2449170|2449140|Ms.|Yolanda|Cherry|N|31|3|1955|THAILAND||Yolanda.Cherry@m2.org|2452415| +3003|AAAAAAAALLLAAAAA|303109|1674|15839|2449411|2449381|Sir|Seth|Hanson|Y|22|2|1987|BENIN||Seth.Hanson@mFXIO1nyBpN97E.org|2452525| +3004|AAAAAAAAMLLAAAAA|1405133|5582|43631|2450876|2450846|Ms.|Brinda|Barton|Y|9|6|1973|YEMEN||Brinda.Barton@yRlxi.edu|2452600| +3005|AAAAAAAANLLAAAAA|854160|4116|2519|2450456|2450426|Dr.|Ruby|Pederson|N|28|3|1965|SLOVENIA||Ruby.Pederson@V.org|2452395| +3006|AAAAAAAAOLLAAAAA|610423|1333|40690|2450229|2450199|Dr.|William|Spears|N|19|2|1937|VANUATU||William.Spears@LL37x.org|2452469| +3007|AAAAAAAAPLLAAAAA|1447493|206|44519|2451867|2451837|Sir|David|Engle|Y|12|6|1926|GEORGIA||David.Engle@KzCZrkthiKu.edu|2452555| +3008|AAAAAAAAAMLAAAAA|52033|3236|10486|2450942|2450912|Miss|Jana|Brown|N|13|12|1962|MALAYSIA||Jana.Brown@87fEU3AXv.com|2452571| +3009|AAAAAAAABMLAAAAA|73623|3506|5789|2452497|2452467|Dr.|Judith|Rodriguez|Y|13|5|1978|ESTONIA||Judith.Rodriguez@q4lZmn7Y6kCk.com|2452462| +3010|AAAAAAAACMLAAAAA|625482|4471|49507|2452264|2452234|Mrs.|Cecelia|Delacruz|Y|10|12|1934|TOKELAU||Cecelia.Delacruz@C.edu|2452483| +3011|AAAAAAAADMLAAAAA|823503|6857|30164|2451037|2451007|Mr.|Daniel|Verdin|N|15|11|1924|AMERICAN SAMOA||Daniel.Verdin@Tk0s.edu|2452536| +3012|AAAAAAAAEMLAAAAA|29322|851|21227|2452421|2452391|Dr.|Ronald|Ross|N|4|10|1985|JORDAN||Ronald.Ross@Rz5bB3LYH.com|2452634| +3013|AAAAAAAAFMLAAAAA|1879481|5487|31893|2451121|2451091|Dr.|Sonja|Coughlin|Y|17|3|1954|LIBERIA||Sonja.Coughlin@oLB.com|2452351| +3014|AAAAAAAAGMLAAAAA|1322201|2739|42030|2451389|2451359|Sir|Clayton|Green|Y|14|7|1963|BURUNDI||Clayton.Green@hTNC8dDGjHui6fBaU.edu|2452399| +3015|AAAAAAAAHMLAAAAA|1481268|3788|19061|2450119|2450089|Mrs.|Janet|Arsenault|N|1|7|1985|GABON||Janet.Arsenault@38d3.com|2452514| +3016|AAAAAAAAIMLAAAAA|1392693|4089|25085|2451559|2451529|Sir|Kenneth|Mangum|N|9|8|1932|ROMANIA||Kenneth.Mangum@k.org|2452557| +3017|AAAAAAAAJMLAAAAA|801297|6819|45094|2449315|2449285|Mrs.|Jennifer|Miller|N|15|2|1989|UNITED KINGDOM||Jennifer.Miller@hRYD44RoUKP.edu|2452374| +3018|AAAAAAAAKMLAAAAA|781917|1126|20335|2451341|2451311|Mrs.|Gretchen|Reddick|N|23|10|1991|POLAND||Gretchen.Reddick@YD9Gofhg7fsMb.org|2452473| +3019|AAAAAAAALMLAAAAA|1147086|3653|29373|2451266|2451236|Miss|Elsie|Sanders|N|17|10|1950|MAURITIUS||Elsie.Sanders@VL9v1O9.com|2452440| +3020|AAAAAAAAMMLAAAAA|796321|5005|23642|2449569|2449539|Dr.|Sean|Osorio|N|25|2|1972|SAMOA||Sean.Osorio@JFGxi9YU41uRGOZ.org|2452608| +3021|AAAAAAAANMLAAAAA|1300094|3281|21668|2450439|2450409|Sir|Greg|Goldberg|Y|13|10|1954|SLOVENIA||Greg.Goldberg@7sXpRGp7EdDp8bcO.edu|2452568| +3022|AAAAAAAAOMLAAAAA|489588|2335|35082|2450913|2450883|Mr.|Lawrence|Blair|N|5|11|1936|MAYOTTE||Lawrence.Blair@T0dnOFJKkPk1.edu|2452548| +3023|AAAAAAAAPMLAAAAA|759172|2227|16482|2449649|2449619|Mrs.|Barbara|Velasquez|Y|9|11|1937|ROMANIA||Barbara.Velasquez@Eo0t7ZhdGLUCZ3.org|2452514| +3024|AAAAAAAAANLAAAAA|5003|1651|39546|2449920|2449890|Dr.|Grady|Smith|N|10|1|1984|SRI LANKA||Grady.Smith@JB0u7Qb4Oaq50.com|2452628| +3025|AAAAAAAABNLAAAAA|1510872|2922|26619|2450864|2450834|Mr.|Brandon|Woodruff|Y|23|1|1960|SOUTH AFRICA||Brandon.Woodruff@DZIjalpy.edu|2452631| +3026|AAAAAAAACNLAAAAA|264630|3668|353|2450987|2450957|Dr.|Rose|Scott|Y|19|3|1929|UZBEKISTAN||Rose.Scott@GHCiB47pquC.com|2452296| +3027|AAAAAAAADNLAAAAA|1063148|4713|29125|2450330|2450300|Miss|Constance|Waller|Y|2|4|1992|MAURITIUS||Constance.Waller@tbUXvq5zI.edu|2452500| +3028|AAAAAAAAENLAAAAA|766590|5011|49032|2450425|2450395|Mrs.|Eva|Griffin|N|29|6|1974|NORWAY||Eva.Griffin@g5qVpVUDcYb93QGJe.com|2452470| +3029|AAAAAAAAFNLAAAAA|1243303|6990|43477|2450833|2450803|Mr.|Dwight|Ngo|N|25|11|1939|KAZAKHSTAN||Dwight.Ngo@KklXNcvm6tqftiT.edu|2452375| +3030|AAAAAAAAGNLAAAAA|1236513|3213|45960|2449911|2449881|Mr.|Curt|Busby|Y|20|6|1952|SURINAME||Curt.Busby@rGmU1xk3k.org|2452367| +3031|AAAAAAAAHNLAAAAA|1697620|3703|14186|2450212|2450182|Dr.|Charles|Wynn|Y|5|11|1944|EGYPT||Charles.Wynn@C.edu|2452576| +3032|AAAAAAAAINLAAAAA|1206639|299|26712|2450806|2450776|Mr.|Leon|Dickerson|N|24|6|1979|CYPRUS||Leon.Dickerson@L.edu|2452332| +3033|AAAAAAAAJNLAAAAA|1493107|2739|41971|2451405|2451375|Ms.|Traci|Wilkerson|Y|27|4|1944|ESTONIA||Traci.Wilkerson@zq.edu|2452536| +3034|AAAAAAAAKNLAAAAA|186383|1770|41557|2449092|2449062|Mr.|William|Lee|N|27|4|1970|AFGHANISTAN||William.Lee@nezDxemE7n.edu|2452408| +3035|AAAAAAAALNLAAAAA|1473784|4845|17106|2451100|2451070|Mrs.|Michelle|Coleman|Y|20|6|1977|VANUATU||Michelle.Coleman@zp8izk5sLBCvRzDfO.edu|2452454| +3036|AAAAAAAAMNLAAAAA|758545|3230|19915|2452143|2452113|Mrs.|Kaye|Morrison|N|9|8|1957|ISLE OF MAN||Kaye.Morrison@gtknPq3Se.edu|2452420| +3037|AAAAAAAANNLAAAAA|836470|4727|5831|2449930|2449900|Mr.|Raymond|Watkins|Y|20|2|1940|NAURU||Raymond.Watkins@x.com|2452289| +3038|AAAAAAAAONLAAAAA|592354||37008|2452075|2452045||Francis|Hardin||30|7|1957|KIRIBATI||Francis.Hardin@61n5XHmy7J1.org|2452388| +3039|AAAAAAAAPNLAAAAA|922395|4964|25980|2449258|2449228|Sir|Matthew|Vargas|Y|4|5|1941|BOLIVIA||Matthew.Vargas@ObDgTrV.org|2452532| +3040|AAAAAAAAAOLAAAAA|1840969|6896|39670|2451150|2451120|Sir|Jacob|Allred|Y|14|2|1963|MONTENEGRO||Jacob.Allred@h4Mxtr0MnXP.edu|2452332| +3041|AAAAAAAABOLAAAAA|1663113|3493|22626|2451766|2451736|Mr.|Carl|Kelly|N|12|11|1937|GREECE||Carl.Kelly@mSljNzhUj2m.edu|2452411| +3042|AAAAAAAACOLAAAAA|1105488|343|37951|2452227|2452197|Dr.|Garrett|Marrero|N|5|12|1947|TUVALU||Garrett.Marrero@IUluBp5IXaYrf8.edu|2452465| +3043|AAAAAAAADOLAAAAA|1098693|4397|43701|2450510|2450480|Mrs.|Shawna|Pham|Y|4|11|1928|ZAMBIA||Shawna.Pham@rAfgiEBR3U.com|2452326| +3044|AAAAAAAAEOLAAAAA|206419|2962|47552|2449774|2449744|Miss|Tammy|Cook|N|18|10|1980|SPAIN||Tammy.Cook@9.org|2452634| +3045|AAAAAAAAFOLAAAAA|550496|2918|12940|2452556|2452526|Dr.|Linda|Maldonado|Y|21|6|1950|MACAO||Linda.Maldonado@ErLToYIdn6ZZ5.edu|2452446| +3046|AAAAAAAAGOLAAAAA|1261132|2689|1799|2452489|2452459|Sir|Daryl|Williams|N|2|3|1928|MONGOLIA||Daryl.Williams@gcEDvd31d.com|2452438| +3047|AAAAAAAAHOLAAAAA|1156847|1744|26994|2452056|2452026|Dr.|Charles|Gibson|Y|15|9|1938|FRENCH POLYNESIA||Charles.Gibson@oBtAgYXYjP.com|2452520| +3048|AAAAAAAAIOLAAAAA|189665|212|22006|2451897|2451867|Sir|Elias|Mccollum|N|24|12|1926|BOLIVIA||Elias.Mccollum@ZRPfTHx5E08jI17MCHZs.edu|2452638| +3049|AAAAAAAAJOLAAAAA|979192|5477|4562|2450699|2450669|Mr.|William|Cowley|Y|16|9|1967|URUGUAY||William.Cowley@uBc.edu|2452359| +3050|AAAAAAAAKOLAAAAA|1601466|5032|29398|2451485|2451455|Ms.|April|Reid|Y|9|9|1964|HONG KONG||April.Reid@4sm24NoXlF6bg9.org|2452362| +3051|AAAAAAAALOLAAAAA|93570|5920|10670|2451551|2451521|Sir|Michael|Clevenger|Y|19|12|1939|LITHUANIA||Michael.Clevenger@b8.com|2452618| +3052|AAAAAAAAMOLAAAAA|323704|1697|38090|2451753|2451723|Dr.|Thomas|Seeley|Y|8|2|1955|SYRIAN ARAB REPUBLIC||Thomas.Seeley@rek76rC6.com|2452338| +3053|AAAAAAAANOLAAAAA|1044188|4104|4467|2452351|2452321|Mr.|David|Jones|N|20|5|1948|BARBADOS||David.Jones@0UjibfcEQdlffT.edu|2452617| +3054|AAAAAAAAOOLAAAAA|1535721|2879|37|2450982|2450952|Dr.|Mellie|Ferguson|Y|21|6|1950|INDIA||Mellie.Ferguson@rRd6nIiF1MeeT2cuBMvn.org|2452576| +3055|AAAAAAAAPOLAAAAA|||5607|2451691|||||||6|1945|NIUE|||2452314| +3056|AAAAAAAAAPLAAAAA|1026226|2460|8741|2450797|2450767|Ms.|Marjorie|Lopez|N|15|6|1929|FRENCH POLYNESIA||Marjorie.Lopez@8oaUbnG8.edu|2452429| +3057|AAAAAAAABPLAAAAA|1877907|6426|14945|2449256|2449226|Dr.|Loren|Kovach|N|24|5|1957|AUSTRIA||Loren.Kovach@PI8zvkqyLij3.org|2452516| +3058|AAAAAAAACPLAAAAA|515293|1492|23244|2451651|2451621|Sir|Al|Melton|Y|20|11|1947|CAMEROON||Al.Melton@uNlTrVVB.org|2452586| +3059|AAAAAAAADPLAAAAA|280735|1032|7329|2452409|2452379|Dr.|Lauren|Dale|Y|17|10|1943|GAMBIA||Lauren.Dale@pnD.edu|2452338| +3060|AAAAAAAAEPLAAAAA|||34993||2450797||Jose||N||5||||Jose.Holder@lrzMHSNhGzQZ6.com|| +3061|AAAAAAAAFPLAAAAA|1877660|4709|30958|2452194|2452164|Miss|Allen|Boyd|Y|27|6|1957|SINGAPORE||Allen.Boyd@mxjnldtY2vskHFF7.org|2452352| +3062|AAAAAAAAGPLAAAAA|676959|849|47492|2449248|2449218|Mrs.|Andrea|Castro|Y|4|4|1929|EL SALVADOR||Andrea.Castro@dCACEh7HSLj.org|2452306| +3063|AAAAAAAAHPLAAAAA|1770732|6842|5752|2450647|2450617|Mrs.|Elsie|Dalton|Y|26|5|1977|AZERBAIJAN||Elsie.Dalton@6vOuMLEgOgv0X.edu|2452587| +3064|AAAAAAAAIPLAAAAA|382084|2918|2505|2452166|2452136|Dr.|Leslie|Sullivan|Y|30|8|1965|SAMOA||Leslie.Sullivan@c1.com|2452308| +3065|AAAAAAAAJPLAAAAA|1150720|71|41822|2451919|2451889|Mrs.|Antonio|Smith|Y|25|4|1973|COMOROS||Antonio.Smith@IJYhtds2LCnzdoF.org|2452594| +3066|AAAAAAAAKPLAAAAA|346294|857|45820|2449497|2449467|Miss|Andrew|Martinez|Y|8|4|1932|CHRISTMAS ISLAND||Andrew.Martinez@1829pqdjCrFsRv.com|2452483| +3067|AAAAAAAALPLAAAAA|1022616|5720|824|2450334|2450304|Miss|Tari|Bethea|Y|16|3|1930|MONTSERRAT||Tari.Bethea@tAjyVt2lGI.org|2452330| +3068|AAAAAAAAMPLAAAAA|918860|1563|23866|2452104|2452074|Mr.|Christopher|Bradley|Y|31|8|1935|KAZAKHSTAN||Christopher.Bradley@qCECC7uDEo3ODDZK9u.edu|2452450| +3069|AAAAAAAANPLAAAAA|751188|1683|11425|2450232|2450202|Dr.|Gordon|Oakes|N|28|2|1935|NORWAY||Gordon.Oakes@GIB6uOR0B7vnOiQxAz.com|2452402| +3070|AAAAAAAAOPLAAAAA|95993|3473|7400|2449141|2449111|Sir|Brad|Rhodes|N|9|10|1990|PHILIPPINES||Brad.Rhodes@i.edu|2452336| +3071|AAAAAAAAPPLAAAAA|598464|6292|29673|2450547|2450517|Miss|Donna|Hughes|N|22|5|1970|PHILIPPINES||Donna.Hughes@YHx7cSqv5T.org|2452576| +3072|AAAAAAAAAAMAAAAA|408870|6341|49412|2449059|2449029|Miss|Catherine|Kimball|N|14|8|1988|PORTUGAL||Catherine.Kimball@RTIbPsfpYzpYDJb.org|2452648| +3073|AAAAAAAABAMAAAAA||4756|29408|2450374|2450344|||||24|11|1935|NAURU|||2452390| +3074|AAAAAAAACAMAAAAA|1322710|3484|47458|2451617|2451587|Sir|Ronald|Brown|Y|14|4|1988|UNITED KINGDOM||Ronald.Brown@iRzdcUcec1Zigrdj0.org|2452312| +3075|AAAAAAAADAMAAAAA|1077340|1443|48346|2450674|2450644|Ms.|Deborah|Robinson|N|17|10|1931|GEORGIA||Deborah.Robinson@XxFkRLVQ2qcMtU.org|2452395| +3076|AAAAAAAAEAMAAAAA|1612104|3986|36621|2449613|||William|||3||1969||||| +3077|AAAAAAAAFAMAAAAA|32966|5869|5172|2449183|2449153|Dr.|Elaine|Warner|N|7|11|1969|BARBADOS||Elaine.Warner@xXZP.com|2452553| +3078|AAAAAAAAGAMAAAAA|1333298|4362|9873|2451774|2451744|Dr.|Carrie|Hatcher|N|27|7|1968|INDONESIA||Carrie.Hatcher@7Jrr8KCHPNSo2jqQ.com|2452423| +3079|AAAAAAAAHAMAAAAA|320312|4827|14142|2452046|2452016|Mr.|Alton|Richardson|N|4|8|1982|BOUVET ISLAND||Alton.Richardson@B3g84t9FR.org|2452415| +3080|AAAAAAAAIAMAAAAA|88050|3783|44363|2449468|2449438|Sir|Lawrence|Gibson|Y|12|7|1957|SOUTH AFRICA||Lawrence.Gibson@x0ZPEg5TGTfp.edu|2452628| +3081|AAAAAAAAJAMAAAAA|488665|359|500|2449829|2449799|Mr.|Herbert|Gaines|Y|16|9|1969|FRENCH POLYNESIA||Herbert.Gaines@k94G25PYc5.com|2452521| +3082|AAAAAAAAKAMAAAAA|219162|4786|20410|2452314|2452284|Mr.|Patrick|Gann|Y|24|7|1951|VIET NAM||Patrick.Gann@uSqmyAYYQL.org|2452646| +3083|AAAAAAAALAMAAAAA|887123|3155|30034|2449805|2449775|Mr.|Patrick|Roberts|N|25|2|1931|BANGLADESH||Patrick.Roberts@10cFT9gHQD.edu|2452534| +3084|AAAAAAAAMAMAAAAA|891124|3867|34610|2451212|2451182|Mr.|Jerry|Aguilar|Y|10|1|1954|TURKEY||Jerry.Aguilar@N.edu|2452370| +3085|AAAAAAAANAMAAAAA|1229779|2526|24120|2449886|2449856|Mrs.|Susanne|Agee|Y|5|6|1961|ERITREA||Susanne.Agee@02Ilerjzo1dUY.edu|2452552| +3086|AAAAAAAAOAMAAAAA|||39257|2451936||||Vieira|N|8|11|1946|SWEDEN||Carl.Vieira@qtUsS4zU89.org|2452311| +3087|AAAAAAAAPAMAAAAA|1316831|4942|7100|2452291|2452261|Dr.|Yolanda|Garcia|N|21|11|1942|MADAGASCAR||Yolanda.Garcia@L0ghK.com|2452441| +3088|AAAAAAAAABMAAAAA|804484|2709|21734|2450161|2450131|Mrs.|Mollie|Bowling|Y|7|1|1945|MEXICO||Mollie.Bowling@5idDDuBkuNxJd.org|2452619| +3089|AAAAAAAABBMAAAAA|729930|1249|1768|2450285|2450255|Dr.|Patrick|Smith|N|28|7|1978|NETHERLANDS ANTILLES||Patrick.Smith@yNXQVb7Eoqy.edu|2452475| +3090|AAAAAAAACBMAAAAA|341322||44023|2449493|2449463|||||7|||||Veronica.Gibson@hOb8.com|2452288| +3091|AAAAAAAADBMAAAAA|544972|2536|14500|2450079|2450049|Mr.|Mark|Lee|N|4|2|1968|ALBANIA||Mark.Lee@zva1hZeF.edu|2452519| +3092|AAAAAAAAEBMAAAAA|1257917|830|23296|2450429|2450399|Sir|Curtis|Gutierrez|Y|7|9|1960|DENMARK||Curtis.Gutierrez@acokRf6MiZosk.com|2452614| +3093|AAAAAAAAFBMAAAAA|332747|4101|24302|2451438|2451408|Sir|Melvin|Kelley|N|17|9|1942|ISLE OF MAN||Melvin.Kelley@7iQIqAiSJMDDYf.org|2452317| +3094|AAAAAAAAGBMAAAAA|385612|521|34630|2449499|2449469|Miss|Mellisa|Davis|Y|11|8|1981|RWANDA||Mellisa.Davis@2x6gKQAPyU4nj1alJ.org|2452465| +3095|AAAAAAAAHBMAAAAA|1172128|617|22306|2450367|2450337|Sir|Paul|Schmidt|Y|12|1|1946|HUNGARY||Paul.Schmidt@9JqMFEE6sSQ8.org|2452398| +3096|AAAAAAAAIBMAAAAA|1635199|1435|10471|2451540|2451510|Sir|Jose|Freeman|N|23|9|1953|CZECH REPUBLIC||Jose.Freeman@aZSy.edu|2452459| +3097|AAAAAAAAJBMAAAAA|1021285|7198|49911|2450315|2450285|Mrs.|Mabel|Gray|N|27|4|1954|MALTA||Mabel.Gray@u5cEHJzHDMDaB.edu|2452542| +3098|AAAAAAAAKBMAAAAA|1841080|6422|6564|2450989|2450959|Dr.|John|Ball|N|24|5|1992|JERSEY||John.Ball@Gyq3RhSijEb4qZb.org|2452353| +3099|AAAAAAAALBMAAAAA|139874|1906|49985|2450617|2450587|Dr.|Gregory|Ervin|Y|8|5|1969|GREECE||Gregory.Ervin@QYvRjILnEC2SYDQ.com|2452554| +3100|AAAAAAAAMBMAAAAA|1756759|651|18590|2449263|2449233|Dr.|Vivian|Crouch|Y|3|2|1966|SENEGAL||Vivian.Crouch@Lb1BC616oYdV.com|2452472| +3101|AAAAAAAANBMAAAAA|1627374|2895|10816|2451387|2451357|Mr.|Paul|Jones|N|12|10|1931|NAURU||Paul.Jones@Cd.edu|2452324| +3102|AAAAAAAAOBMAAAAA|1630639|5576|14513|2449420|2449390|Mr.|William|Miller|N|12|7|1978|NEPAL||William.Miller@YA1RPuZV0Q.com|2452440| +3103|AAAAAAAAPBMAAAAA|907950|4125|23500|2450122|2450092|Miss|Maryann|Miller|N|25|4|1955|BELARUS||Maryann.Miller@2kTEU3OQByhy.com|2452553| +3104|AAAAAAAAACMAAAAA|40740|5451|47409|2449506|2449476|Sir|Ivan|Thomas|Y|10|9|1981|ALGERIA||Ivan.Thomas@XckXyQaQp.com|2452386| +3105|AAAAAAAABCMAAAAA|202190|3569|40548|2450125|2450095|Ms.|Crystal|Roberts|Y|22|7|1989|SINGAPORE||Crystal.Roberts@YY.com|2452495| +3106|AAAAAAAACCMAAAAA|512781|51|9721|2450308|2450278|Ms.|Geraldine|Neil|N|24|12|1976|FRANCE||Geraldine.Neil@KDMZHN.org|2452502| +3107|AAAAAAAADCMAAAAA|211317|3060|10850|2452605|2452575|Mrs.|Susanna|Fortier|N|4|3|1961|BELGIUM||Susanna.Fortier@08tAb.com|2452368| +3108|AAAAAAAAECMAAAAA|930066|289|7265|2450408|2450378|Dr.|Andrea|Gonzalez|N|25|5|1929|MALAYSIA||Andrea.Gonzalez@gL7gxdO2qYFc.org|2452515| +3109|AAAAAAAAFCMAAAAA|295944|4653|3762|2450453|2450423|Ms.|Anna|Faber|N|2|11|1986|MALTA||Anna.Faber@19Mv6vBr.org|2452384| +3110|AAAAAAAAGCMAAAAA|1523596|691|16539|2449882|2449852|Dr.|Corey|Mcinnis|Y|4|5|1962|VENEZUELA||Corey.Mcinnis@VaM7qhad0b8.com|2452358| +3111|AAAAAAAAHCMAAAAA|572695|1272|39896|2450715|2450685|Dr.|David|Evans|Y|6|1|1966|AFGHANISTAN||David.Evans@e0tO8sG5T2V.edu|2452392| +3112|AAAAAAAAICMAAAAA|1464790|2040|24843|2451947|2451917|Miss|Darcy|Fairley|Y|31|10|1954|NEPAL||Darcy.Fairley@yb9RPl0iclS.edu|2452520| +3113|AAAAAAAAJCMAAAAA|755867|717|31214|2450171|2450141|Mr.|Russell|Mason|N|13|7|1973|GREENLAND||Russell.Mason@IeI.org|2452342| +3114|AAAAAAAAKCMAAAAA|295447|6899|11551|2449926|2449896|Sir|Clarence|Hughes|Y|15|1|1930|ANTARCTICA||Clarence.Hughes@CVSTNZmvveOKIUMbOqTJ.com|2452418| +3115|AAAAAAAALCMAAAAA|1854870|3308|34146|2451501|2451471|Miss|Eileen|Hensley|N|9|11|1986|ALBANIA||Eileen.Hensley@MOh0DrXLElt.com|2452477| +3116|AAAAAAAAMCMAAAAA|1326100|3381|46709|2450830|2450800|Mrs.|Evelyn|Contreras|Y|11|5|1991|ALAND ISLANDS||Evelyn.Contreras@Bk.com|2452511| +3117|AAAAAAAANCMAAAAA|738452|6672|1692|2450263|2450233|Sir|William|Barba|Y|24|8|1977|ANDORRA||William.Barba@Fv3.com|2452352| +3118|AAAAAAAAOCMAAAAA|1699045|4757|48069|2451726|2451696|Mrs.|Catherine|Douglas|Y|4|3|1976|SLOVENIA||Catherine.Douglas@knBifGYt.org|2452646| +3119|AAAAAAAAPCMAAAAA|970434|6862|37290|2450105|2450075|Ms.|Sandra|Todd|N|22|8|1950|BERMUDA||Sandra.Todd@9KkX.edu|2452361| +3120|AAAAAAAAADMAAAAA|327849|3243|32284|2449323|2449293|Dr.|Ester|White|Y|8|6|1930|HAITI||Ester.White@ECSruxLcIuU.edu|2452553| +3121|AAAAAAAABDMAAAAA|741139|5943|20300|2451567|2451537|Mr.|Wendell|Jackson|N|9|10|1979|ALBANIA||Wendell.Jackson@5Fn5VOqNYFCLTu.org|2452365| +3122|AAAAAAAACDMAAAAA|810990|1149|4508|2452193|2452163|Mr.|Odis|Howell|N|29|8|1953|CYPRUS||Odis.Howell@iSmPijrFx3.com|2452381| +3123|AAAAAAAADDMAAAAA|273880|439|6245|2450656|2450626|Ms.|Sabrina|Leary|Y|9|4|1966|MONACO||Sabrina.Leary@Cnbi.com|2452470| +3124|AAAAAAAAEDMAAAAA|1284081|546|23011|2450584|2450554|Mrs.|Latonia|Perez|N|5|8|1989|SEYCHELLES||Latonia.Perez@UZAyBS50.org|2452490| +3125|AAAAAAAAFDMAAAAA|580479|5668|15704|2449697|2449667|Dr.|Herbert|Pollock|N|30|7|1989|ALGERIA||Herbert.Pollock@j7UURy9ma0rM.com|2452574| +3126|AAAAAAAAGDMAAAAA|1690157|5832|6307|2450747|2450717|Miss|Janet|Russell|N|30|4|1936|NAMIBIA||Janet.Russell@MIhU482L4f6iJo.edu|2452574| +3127|AAAAAAAAHDMAAAAA|453519|1331|5543|2452419|2452389|Ms.|Tamara|Hardy|N|5|11|1989|MACAO||Tamara.Hardy@udogVQgC7uE7xcGD.com|2452516| +3128|AAAAAAAAIDMAAAAA|1001949|2466|29383|2449324|2449294|Ms.|Dorothy|Cohn|N|8|7|1988|SYRIAN ARAB REPUBLIC||Dorothy.Cohn@lV68bZUF94cZG.com|2452322| +3129|AAAAAAAAJDMAAAAA|548506|4889|22472|2450062|2450032|Sir|Earl|Leal|N|1|5|1990|ALBANIA||Earl.Leal@DLu.com|2452481| +3130|AAAAAAAAKDMAAAAA|1853175|2766|17604|2449575|2449545|Dr.|Renee|Campbell|Y|10|4|1961|NAMIBIA||Renee.Campbell@iIQHbuJtmD.edu|2452641| +3131|AAAAAAAALDMAAAAA|763237|3009|9181|2450517|2450487|Mr.|Mark|Conover|Y|4|8|1980|NIGERIA||Mark.Conover@Qaq8Rof9T9UM.org|2452332| +3132|AAAAAAAAMDMAAAAA|1291072|2480|18688|2452586|2452556|Sir|Robert|Roman|N|9|8|1961|MONGOLIA||Robert.Roman@S99kTcv40y.org|2452286| +3133|AAAAAAAANDMAAAAA|984560|5296|37679|2450014|2449984|Sir|Kenneth|Silva|Y|5|10|1940|KIRIBATI||Kenneth.Silva@x.org|2452530| +3134|AAAAAAAAODMAAAAA|352518|5994|34270|2450338|2450308|Dr.|Harold|Kane|N|17|9|1981|ALBANIA||Harold.Kane@rn8MNUqlUpf6pXL2.edu|2452548| +3135|AAAAAAAAPDMAAAAA|1369233|1244|46102|2450744|2450714|Sir|Richard|Ortega|N|6|8|1961|CAYMAN ISLANDS||Richard.Ortega@MpPDbBI5uqV.org|2452471| +3136|AAAAAAAAAEMAAAAA|1582306|3302|18251|2449579|2449549|Mr.|Scott|Mcconnell|Y|5|5|1959|BAHRAIN||Scott.Mcconnell@xoPNmcLgc.com|2452516| +3137|AAAAAAAABEMAAAAA|127210|5057|29704|2452233|2452203|Mr.|Byron|Todd|Y|12|5|1963|BELIZE||Byron.Todd@SeXm4eVJfs.com|2452287| +3138|AAAAAAAACEMAAAAA|1779639|616|30393|2451202|2451172|Dr.|William|Smith|N|20|2|1934|ZIMBABWE||William.Smith@xLRkE97.org|2452559| +3139|AAAAAAAADEMAAAAA|4430|4196|5609|2450131|2450101|Miss|Marylou|Vernon|Y|9|10|1950|SERBIA||Marylou.Vernon@Q4OSq229fHKGM3uO.com|2452562| +3140|AAAAAAAAEEMAAAAA|1701177|5911|45045|2451882|2451852|Sir|Rueben|Roller|N|25|12|1951|MONTENEGRO||Rueben.Roller@6RSYUt2mrzZ.com|2452408| +3141|AAAAAAAAFEMAAAAA|530795|4390|20785|2452288|2452258|Dr.|Edward|Clark|Y|9|3|1936|POLAND||Edward.Clark@NBm5.org|2452418| +3142|AAAAAAAAGEMAAAAA|1312212|4138|17664|2451324|2451294|Mrs.|Rachel|Chandler|N|19|3|1950|MOZAMBIQUE||Rachel.Chandler@aVxj50h1MjxcY7Ed4YzC.com|2452408| +3143|AAAAAAAAHEMAAAAA|1504568|976|36631|2450920|2450890|Miss|Holly|Wilson|Y|17|4|1931|SWEDEN||Holly.Wilson@fxleme4EPNthimM.org|2452607| +3144|AAAAAAAAIEMAAAAA|1205683|5212|19141|2452175|2452145|Dr.|Jeremy|Jones|N|15|10|1984|EQUATORIAL GUINEA||Jeremy.Jones@Xv8p.org|2452454| +3145|AAAAAAAAJEMAAAAA|1880561|3538|46915|2451446|2451416|Sir|Edward|Ward|Y|9|10|1992|GREECE||Edward.Ward@i4uIC.org|2452531| +3146|AAAAAAAAKEMAAAAA|973748|5710|11719|2451420|2451390|Mrs.|Irene|Walker|Y|15|9|1948|SAINT LUCIA||Irene.Walker@6CYsFHxJrK9gglKD.com|2452318| +3147|AAAAAAAALEMAAAAA|1823753|4197|9088|2451561|2451531|Dr.|Ora|Spain|Y|19|3|1936|ZAMBIA||Ora.Spain@CT.edu|2452467| +3148|AAAAAAAAMEMAAAAA|||12051||||Joe|Gentry||||||||| +3149|AAAAAAAANEMAAAAA|1742666|2808|4103|2450439|2450409|Mrs.|Anthony|Ernst|N|9|5|1978|KOREA, REPUBLIC OF||Anthony.Ernst@e5bKg3bNluGU.edu|2452493| +3150|AAAAAAAAOEMAAAAA|814676|5896|17516|2452289|2452259|Dr.|Erik|Jackson|N|2|3|1938|NIUE||Erik.Jackson@PuuYcdnig.org|2452558| +3151|AAAAAAAAPEMAAAAA|1286810|2474|7785|2449578|2449548|Sir|Peter|Sloan|Y|30|4|1990|POLAND||Peter.Sloan@m1z5NiTEZE.com|2452522| +3152|AAAAAAAAAFMAAAAA|1733785|3439|10246|2449875|2449845|Mr.|Dominic|Fox|N|7|9|1944|TOKELAU||Dominic.Fox@VebbarM.com|2452526| +3153|AAAAAAAABFMAAAAA|1571166|827|1288|2451556|2451526|Dr.|Blake|Dotson|N|29|6|1973|MONTENEGRO||Blake.Dotson@7.edu|2452593| +3154|AAAAAAAACFMAAAAA|1368255|3734|7566|2451044|2451014|Mrs.|Carla|Taylor|Y|25|10|1974|HAITI||Carla.Taylor@hdqAA.edu|2452329| +3155|AAAAAAAADFMAAAAA|462337|176|26415|2452170|2452140|Dr.|Lance|Sparks|Y|18|2|1961|KIRIBATI||Lance.Sparks@TKkkVEs5ofqqdx.edu|2452547| +3156|AAAAAAAAEFMAAAAA|859625|6918|12211|2450589|2450559|Dr.|Elma|Smith|Y|17|12|1982|MALTA||Elma.Smith@y.edu|2452336| +3157|AAAAAAAAFFMAAAAA|1639458|6475|17092|2451742|2451712|Ms.|Lue|Doherty|Y|9|6|1956|FRANCE||Lue.Doherty@NsmEH8xq.org|2452394| +3158|AAAAAAAAGFMAAAAA|485341|5163|44585|2451608|2451578|Ms.|Cora|Gomez|N|11|7|1934|ALBANIA||Cora.Gomez@R7k.edu|2452409| +3159|AAAAAAAAHFMAAAAA|33316||41876|2452177|||Walter||||2|1988||||2452555| +3160|AAAAAAAAIFMAAAAA|799540|4333|30373|2451697|2451667|Dr.|Kristen|Alexander|Y|27|1|1962|OMAN||Kristen.Alexander@fR9F7nfHxmND.edu|2452572| +3161|AAAAAAAAJFMAAAAA|941952|587|822|2452634|2452604|Sir|Jonathan|Morrow|N|19|5|1926|PITCAIRN||Jonathan.Morrow@MAFVHcZj6DRXy.com|2452409| +3162|AAAAAAAAKFMAAAAA|1485925|661|27397|2451942|2451912|Mr.|James|Ortiz|N|20|9|1947|SPAIN||James.Ortiz@hK.edu|2452508| +3163|AAAAAAAALFMAAAAA||394|15783||2451946|Dr.|Jamie|Byrd||19|||SAINT HELENA||Jamie.Byrd@AK1o.edu|| +3164|AAAAAAAAMFMAAAAA|832358|1720|25529|2451293|2451263|Miss|Harriet|Cormier|Y|25|7|1979|AUSTRALIA||Harriet.Cormier@bd1jF5aM.edu|2452424| +3165|AAAAAAAANFMAAAAA|1670717|6805|25693|2452157|2452127|Dr.|Michael|Perry|N|14|12|1966|KENYA||Michael.Perry@q8b4JnB9m4.com|2452582| +3166|AAAAAAAAOFMAAAAA|675212|6954|48696|2450389|2450359|Mrs.|Anna|Williams|N|10|6|1932|EL SALVADOR||Anna.Williams@AQi.com|2452642| +3167|AAAAAAAAPFMAAAAA|130003|1334|2712|2452056|2452026|Dr.|Kayla|Forster|N|9|10|1943|KENYA||Kayla.Forster@t2zM2enxKqvqGO.org|2452387| +3168|AAAAAAAAAGMAAAAA|1879359|3069|23132|2449436|2449406|Ms.|Sheila|Stephens|Y|9|9|1950|ALGERIA||Sheila.Stephens@yTPG30bLS7eCPBDOPHgD.org|2452601| +3169|AAAAAAAABGMAAAAA|594531|4591|38298|2449733|2449703|Ms.|Diana|Davis|Y|15|9|1940|VIET NAM||Diana.Davis@pn4oAAz7mMagbKihC.org|2452576| +3170|AAAAAAAACGMAAAAA|1324327|3662|45797|2450191|2450161|Dr.|Robert|Havens|N|19|12|1950|RUSSIAN FEDERATION||Robert.Havens@l9OFJnVfnIre8nICp.edu|2452439| +3171|AAAAAAAADGMAAAAA|878674|4842|29410|2450133|2450103|Dr.|Donald|Short|Y|22|2|1987|CAMEROON||Donald.Short@r4aIJJy.edu|2452327| +3172|AAAAAAAAEGMAAAAA|435238|5|9219|2452222|2452192|Dr.|Ruth|Mullen|Y|12|10|1931|NORFOLK ISLAND||Ruth.Mullen@U6lHVn.org|2452515| +3173|AAAAAAAAFGMAAAAA|203772|6942|46112|2451523|2451493|Mr.|Billy|Adair|N|27|6|1987|HAITI||Billy.Adair@aqo44X7EuqxsRdFC8.org|2452551| +3174|AAAAAAAAGGMAAAAA|30720|7180|24440|2450336|2450306|Ms.|Jeanne|Clark|Y|28|10|1969|GHANA||Jeanne.Clark@v1rrdNtTL7dd0hh.org|2452387| +3175|AAAAAAAAHGMAAAAA|1179375||49547|2449660||Sir|Willie||Y||7||CAMEROON||Willie.Rollins@sRkk5mJV7ZTDcA.edu|| +3176|AAAAAAAAIGMAAAAA|509384|3163|28942|2449720|2449690|Dr.|Victor|Ingram|Y|11|12|1954|WALLIS AND FUTUNA||Victor.Ingram@rB81.com|2452548| +3177|AAAAAAAAJGMAAAAA|22009|5108|16254|2450990|2450960|Dr.|Bertha|Livingston|N|27|2|1980|DJIBOUTI||Bertha.Livingston@zrDtEzv2vdf.edu|2452643| +3178|AAAAAAAAKGMAAAAA|509831|3294|40404|2449473|2449443|Dr.|Clarice|Baldwin|Y|28|10|1968|MEXICO||Clarice.Baldwin@a7KCpnjNL.org|2452320| +3179|AAAAAAAALGMAAAAA|96105|4063|43621|2449150|2449120|Dr.|Michael|Rivera|Y|30|1|1949|LIBERIA||Michael.Rivera@Ot9hhMB.org|2452610| +3180|AAAAAAAAMGMAAAAA|1626016|7186|2709|2449649|2449619|Mr.|Jimmy|Parks|N|26|2|1949|BOTSWANA||Jimmy.Parks@XFUGbUED.com|2452469| +3181|AAAAAAAANGMAAAAA|1510294|5500|25553|2451381|2451351|Ms.|Laurie|Johnson|Y|18|12|1984|ALAND ISLANDS||Laurie.Johnson@X8xd34Vi67aExr.com|2452357| +3182|AAAAAAAAOGMAAAAA|72371|7157|22743|2451357|2451327|Sir|John|Dillon|Y|3|10|1956|KAZAKHSTAN||John.Dillon@KBt0EgcmLQqrIlT.com|2452639| +3183|AAAAAAAAPGMAAAAA|604649|3965|44881|2450579|2450549|Miss|Socorro|Palmer|N|27|6|1982|SAUDI ARABIA||Socorro.Palmer@nP1sQmB.edu|2452588| +3184|AAAAAAAAAHMAAAAA|745847|2255|12310|2451689|2451659|Miss|Daisy|Clancy|N|19|5|1991|NETHERLANDS||Daisy.Clancy@o0fFjdqxNHj.org|2452333| +3185|AAAAAAAABHMAAAAA|684163|4662|11561|2451234|2451204|Ms.|Sadie|Goode|Y|6|5|1957|LIECHTENSTEIN||Sadie.Goode@z4.com|2452452| +3186|AAAAAAAACHMAAAAA|1461933|868|44603|2451463|2451433|Sir|Dudley|Graham|N|9|3|1980|BOUVET ISLAND||Dudley.Graham@tYfroDzmnKe8.edu|2452509| +3187|AAAAAAAADHMAAAAA|293018|835|15452|2452585|2452555|Dr.|Lawrence|Miller|Y|11|11|1969|MALDIVES||Lawrence.Miller@P8lL1jeSjJSfmkBf.org|2452632| +3188|AAAAAAAAEHMAAAAA|486614|6081|41148|2450821|2450791|Miss|Magdalen|Shea|Y|2|7|1967|MONACO||Magdalen.Shea@PUvB9UAYyn.edu|2452472| +3189|AAAAAAAAFHMAAAAA|509350|1741|36485|2449657|2449627|Ms.|Delia|Shaw|N|27|6|1968|CZECH REPUBLIC||Delia.Shaw@0J4Xrferf2jgeXKm2.org|2452552| +3190|AAAAAAAAGHMAAAAA|1785361|3284|36360|2451826|2451796|Mr.|Gregory|Mitchell|N|20|3|1967|MALAWI||Gregory.Mitchell@0su.edu|2452592| +3191|AAAAAAAAHHMAAAAA|667068|5401|10534|2450080|2450050|Mrs.|Beverly|Jackson|N|11|3|1958|MALI||Beverly.Jackson@SpvQ.edu|2452521| +3192|AAAAAAAAIHMAAAAA|1644061|7167|21073|2451631|2451601|Sir|Ramon|Foster|N|21|2|1969|TOGO||Ramon.Foster@6TETABkFDo2.edu|2452572| +3193|AAAAAAAAJHMAAAAA|1127763|1980|19844|2451858|2451828|Miss|Irene|Burrell|Y|7|3|1942|ECUADOR||Irene.Burrell@DxTJS.org|2452350| +3194|AAAAAAAAKHMAAAAA|1193480|1992|35751|2452171|2452141|Miss|Brandi|Webb|Y|5|3|1970|SYRIAN ARAB REPUBLIC||Brandi.Webb@9BAzuaI.org|2452410| +3195|AAAAAAAALHMAAAAA|255685|2937|7342|2452022|2451992|Mrs.|Judith|Self|Y|5|3|1978|TOKELAU||Judith.Self@qOp.edu|2452572| +3196|AAAAAAAAMHMAAAAA|1195161|1790|3172|2451361|2451331|Mrs.|Brittany|Ross|N|6|8|1928|LITHUANIA||Brittany.Ross@oooFGFitt0Vs9u.com|2452516| +3197|AAAAAAAANHMAAAAA|1177823|4943|37616|2452325|2452295|Mrs.|Sarah|Mcvay|N|20|12|1979|UNITED ARAB EMIRATES||Sarah.Mcvay@tP4zJPUj3chzu.edu|2452541| +3198|AAAAAAAAOHMAAAAA|61594|1657|10920|2449055|2449025|Miss|Kelli|Davis|Y|29|8|1976|AZERBAIJAN||Kelli.Davis@BN.com|2452332| +3199|AAAAAAAAPHMAAAAA|676119|6328|27784|2452157|2452127|Mr.|Michael|Proctor|N|3|6|1950|TONGA||Michael.Proctor@TiUAK86CqHdLu.edu|2452497| +3200|AAAAAAAAAIMAAAAA|1596392|4994|23596|2450077|2450047|Mrs.|Dorothy|Thompson|Y|9|4|1985|ISLE OF MAN||Dorothy.Thompson@y3d.org|2452409| +3201|AAAAAAAABIMAAAAA|771523|768|10860|2452529|2452499|Dr.|Susan|Duncan|N|20|5|1959|CYPRUS||Susan.Duncan@TksXz.edu|2452445| +3202|AAAAAAAACIMAAAAA|496398|928|10658|2450254|2450224|Mr.|Thomas|Roberts|N|1|8|1931|KIRIBATI||Thomas.Roberts@Ne.com|2452358| +3203|AAAAAAAADIMAAAAA|1883675|4907|43339|2449581|2449551|Dr.|Marjorie|Purcell|N|14|8|1932|CAPE VERDE||Marjorie.Purcell@RIqc970.edu|2452546| +3204|AAAAAAAAEIMAAAAA|89488||4162|2450427|2450397|Ms.||Lewis|Y||1||||Marilyn.Lewis@ZrQF.org|2452456| +3205|AAAAAAAAFIMAAAAA|1404894|5048|33202|2449488|2449458|Mr.|Douglas|Lively|N|18|10|1964|BHUTAN||Douglas.Lively@vmrfqD.org|2452289| +3206|AAAAAAAAGIMAAAAA|1671541|4966|11544|2449878|2449848|Mrs.|Deanna|Hunter|Y|25|7|1937|SIERRA LEONE||Deanna.Hunter@rrNKG04poPJ1UgUX6Q.org|2452565| +3207|AAAAAAAAHIMAAAAA|1036343|3230|19270|2452547|2452517|Dr.|Javier|Smith|N|9|10|1983|PERU||Javier.Smith@AR0DuQC7qso7JlPCqE.org|2452326| +3208|AAAAAAAAIIMAAAAA|1728211|4504|47597|2449536|2449506|Mrs.|Lizzie|Sams|N|30|12|1986|SWEDEN||Lizzie.Sams@upJqh4fcYYJZcFa.com|2452551| +3209|AAAAAAAAJIMAAAAA|1664246|2605|2964|2451912|2451882|Sir|Paul|Vogel|N|8|1|1973|BAHRAIN||Paul.Vogel@ydab.com|2452541| +3210|AAAAAAAAKIMAAAAA|698695|3495|2648|2450871|2450841|Mrs.|Linda|Potter|Y|14|2|1978|MADAGASCAR||Linda.Potter@GnBkds8uqam.edu|2452341| +3211|AAAAAAAALIMAAAAA|1809869|3689|10356|2450225|2450195|Sir|James|Greene|N|11|12|1992|CROATIA||James.Greene@7pQ5sytGyDjj16rsn.com|2452457| +3212|AAAAAAAAMIMAAAAA|884973|7142|38864|2451121|2451091|Dr.|Hannah|Battaglia|N|16|7|1940|GABON||Hannah.Battaglia@Q.com|2452572| +3213|AAAAAAAANIMAAAAA|1761156|6502|34601|2450167|2450137|Dr.|Barbara|Faulkner|N|17|6|1953|BERMUDA||Barbara.Faulkner@G6mo2f.com|2452472| +3214|AAAAAAAAOIMAAAAA|15311|4948|11338|2451554|2451524|Dr.|Kendra|Terry|N|7|11|1936|GUATEMALA||Kendra.Terry@mLL7G59vtEJQUKih.edu|2452599| +3215|AAAAAAAAPIMAAAAA|1813727|3903|17679|2450860|2450830|Mr.|Lawrence|Wright|Y|23|12|1957|VIET NAM||Lawrence.Wright@FUE7404Jh9uxhH.edu|2452600| +3216|AAAAAAAAAJMAAAAA|165065|6205|2545|2449566|2449536|Ms.|Paula|Curry|N|7|12|1944|ANGOLA||Paula.Curry@LFklks11B00aJqZu.org|2452638| +3217|AAAAAAAABJMAAAAA|1409092|3324|25956|2450734|2450704|Ms.|Margie|Sells|Y|27|4|1963|LITHUANIA||Margie.Sells@3ZyMn1ZQ1f2.com|2452431| +3218|AAAAAAAACJMAAAAA|1628033|5441|22449|2449373|2449343|Miss|Ivonne|Montgomery|Y|3|12|1991|VENEZUELA||Ivonne.Montgomery@EHS2NGF.edu|2452508| +3219|AAAAAAAADJMAAAAA|15825|1184|12190|2449438|2449408|Miss|Sylvia|Marks|Y|1|2|1975|BRUNEI DARUSSALAM||Sylvia.Marks@IPH562aZS.com|2452332| +3220|AAAAAAAAEJMAAAAA|934305|3365|20072|2451647|2451617|Mr.|Howard|Tapia|N|19|2|1951|SLOVAKIA||Howard.Tapia@r1TZe.edu|2452417| +3221|AAAAAAAAFJMAAAAA|1857944|6306|29068|2451199|2451169|Sir|Stan|Spicer|N|3|2|1945|AFGHANISTAN||Stan.Spicer@mhjc0aqgl.com|2452370| +3222|AAAAAAAAGJMAAAAA|578712|874|46283|2450743|2450713|Dr.|Kenneth|Littlejohn|N|1|4|1929|SWITZERLAND||Kenneth.Littlejohn@Ok8mP7.org|2452382| +3223|AAAAAAAAHJMAAAAA|1045312|3177|45795|2450393|2450363|Dr.|Ruben|Stratton|N|7|7|1933|ALAND ISLANDS||Ruben.Stratton@ldpdSlBoTo.edu|2452641| +3224|AAAAAAAAIJMAAAAA|600025|5091|39357|2449755|2449725|Mr.|Stephen|Floyd|Y|7|10|1970|KAZAKHSTAN||Stephen.Floyd@sRdnchsSEv.org|2452580| +3225|AAAAAAAAJJMAAAAA|62779|6034|21689|2449133|2449103|Ms.|Rita|Ward|Y|27|4|1957|BHUTAN||Rita.Ward@K1Dt1tte4.edu|2452352| +3226|AAAAAAAAKJMAAAAA|1837198|488|24578|2450268|2450238|Mrs.|Jeanette|Jones|N|9|8|1927|CAMBODIA||Jeanette.Jones@UBlMHtKyUOVosGLfy.com|2452544| +3227|AAAAAAAALJMAAAAA|1518415|5766|39917|2450981|2450951|Mr.|Richard|Caldwell|Y|7|11|1932|TUVALU||Richard.Caldwell@ieQd7SMQ1ZnT9.edu|2452521| +3228|AAAAAAAAMJMAAAAA|335289|1440|47638|2449714|2449684|Sir|Carlos|Jacobs|Y|30|7|1952|CANADA||Carlos.Jacobs@p.com|2452436| +3229|AAAAAAAANJMAAAAA|1697739|1052|12318|2449476|2449446|Dr.|Chad|Gil|Y|14|3|1991|ARUBA||Chad.Gil@cxKefbq.com|2452621| +3230|AAAAAAAAOJMAAAAA|1613865|1838|17858|2449568|2449538|Dr.|Mark|Watkins|Y|9|7|1955|MARTINIQUE||Mark.Watkins@3k7MrV5hSmn.com|2452452| +3231|AAAAAAAAPJMAAAAA||551|8523|2450721||||Grice|N|14||1936|ANGOLA||Deann.Grice@BrMMijPgMR.org|2452639| +3232|AAAAAAAAAKMAAAAA|390496|2198|12253|2452049|2452019|Sir|Matthew|Smith|Y|25|5|1931|ISRAEL||Matthew.Smith@2.edu|2452620| +3233|AAAAAAAABKMAAAAA|1237813|4607|36201|2452170||Dr.||Miller||30|6|1962||||2452349| +3234|AAAAAAAACKMAAAAA|125060|5271|13035|2450333|2450303|Sir|James|Peterson|N|19|3|1931|MOZAMBIQUE||James.Peterson@66ka1Y4fs4L.edu|2452351| +3235|AAAAAAAADKMAAAAA|448736|1706|17923|2452548|2452518|Ms.|Cris|Hooper|N|30|8|1925|BOUVET ISLAND||Cris.Hooper@F31vHbs6TOL5CrbS.edu|2452310| +3236|AAAAAAAAEKMAAAAA|328969|5898|31227|2452375|2452345|Sir|Alan|Curtis|Y|20|7|1965|GABON||Alan.Curtis@XpFyR8Hx5Jy.edu|2452351| +3237|AAAAAAAAFKMAAAAA||1353|40782|||Ms.|Debbie||N|2|10||SAINT HELENA||Debbie.Anderson@2VaE.org|2452543| +3238|AAAAAAAAGKMAAAAA|711613|4015|29217|2449694|2449664|Sir|Kenneth|Martinez|N|5|9|1933|ZIMBABWE||Kenneth.Martinez@11MA.com|2452505| +3239|AAAAAAAAHKMAAAAA|509411|2504|37550|2450094|2450064|Ms.|Josefa|Bosley|Y|31|10|1925|TOKELAU||Josefa.Bosley@7udYAomiLLk3PERy.edu|2452467| +3240|AAAAAAAAIKMAAAAA|1102537|988|2639|2452046|2452016|Dr.|Josephine|Ashford|Y|29|9|1992|ARUBA||Josephine.Ashford@q9ulXlkZ0i4cBy78V.org|2452547| +3241|AAAAAAAAJKMAAAAA|1178858|4481|13257|2452598|2452568|Dr.|Lashonda|Marquis|Y|23|5|1932|HONG KONG||Lashonda.Marquis@Zlde5K9z0.com|2452505| +3242|AAAAAAAAKKMAAAAA|935248|828|10808|2451816|2451786|Mr.|John|Kelley|N|6|12|1952|CHRISTMAS ISLAND||John.Kelley@S4cCQybL7t.edu|2452438| +3243|AAAAAAAALKMAAAAA|1835195|308|22221|2449738|2449708|Mrs.|Shelia|Huddleston|Y|12|5|1970|BELIZE||Shelia.Huddleston@RKDheMLST5v4mJV5K.org|2452627| +3244|AAAAAAAAMKMAAAAA|816452|4820|42435|2451798|2451768|Dr.|Joseph|Tobias|Y|18|5|1958|VIET NAM||Joseph.Tobias@a3Gl.org|2452444| +3245|AAAAAAAANKMAAAAA|1146273|3614|10155|2449747|2449717|Miss|Catherine|Carter|N|4|4|1948|NAMIBIA||Catherine.Carter@38susZdqpHZr2NF.com|2452321| +3246|AAAAAAAAOKMAAAAA|596965|1159|33353|2451846|2451816|Miss|Angel|Foote|Y|1|11|1977|NEW CALEDONIA||Angel.Foote@fDoDz2HSOaKG.com|2452360| +3247|AAAAAAAAPKMAAAAA|549662|1527|19439|2451859|2451829|Mrs.|Latasha|Scully|N|4|8|1973|NICARAGUA||Latasha.Scully@tjmi4mYK.org|2452296| +3248|AAAAAAAAALMAAAAA|1012066||17807|2449099|||Rebecca|||25|||GHANA||Rebecca.Saunders@ChpIMeysJ.org|| +3249|AAAAAAAABLMAAAAA|743236|3536|32989|2452437|2452407|Sir|Corey|Dickey|Y|12|6|1977|AUSTRALIA||Corey.Dickey@l43p2RUb9bxk6a3Bovf.com|2452426| +3250|AAAAAAAACLMAAAAA|1435189|4199|6009|2450638|2450608|Dr.|George|Frank|Y|15|10|1964|DJIBOUTI||George.Frank@MNo.edu|2452425| +3251|AAAAAAAADLMAAAAA|114499|5538|4177|2452651|2452621|Mr.|William|Avila|N|15|9|1974|BELGIUM||William.Avila@QeVBcIqb7S9.edu|2452579| +3252|AAAAAAAAELMAAAAA|1894315|6399|43730|2449362|2449332|Dr.|Helen|Christensen|Y|8|1|1969|SLOVAKIA||Helen.Christensen@E.edu|2452648| +3253|AAAAAAAAFLMAAAAA|1480826|6705|14584|2451365|2451335|Miss|Beth|Fulmer|N|9|11|1926|CHILE||Beth.Fulmer@gTAdCvIEzksrTkd.edu|2452421| +3254|AAAAAAAAGLMAAAAA|743768|1693|40018|2451698|2451668|Dr.|Verna|Bellamy|N|1|5|1973|CAMBODIA||Verna.Bellamy@2.org|2452310| +3255|AAAAAAAAHLMAAAAA|1185197||36312||||Rocio|Dunaway|N|||1968|CROATIA|||2452467| +3256|AAAAAAAAILMAAAAA|392793|5554|48315|2452502|2452472|Dr.|Blossom|Doyle|N|8|6|1949|BELARUS||Blossom.Doyle@0rNnDV6MGYN.org|2452559| +3257|AAAAAAAAJLMAAAAA|312401|1508|17815|2451739|2451709|Dr.|Christopher|Rodriguez|N|20|9|1969|ARUBA||Christopher.Rodriguez@2sCR.org|2452357| +3258|AAAAAAAAKLMAAAAA|1058299|5866|11033|2450360|2450330|Ms.|Bertha|Ferguson|Y|13|8|1980|SAUDI ARABIA||Bertha.Ferguson@rypGlAv408A.edu|2452439| +3259|AAAAAAAALLMAAAAA|666419|4342|1418|2451732|2451702|Ms.|Evelyn|Berry|N|5|12|1978|GEORGIA||Evelyn.Berry@8TNBBHf65BgLl9.edu|2452569| +3260|AAAAAAAAMLMAAAAA|1272996|1618|46640|2452622|2452592|Sir|Juan|Smith|Y|3|6|1968|MAYOTTE||Juan.Smith@SuzYNhzdTfH.edu|2452432| +3261|AAAAAAAANLMAAAAA|1452372|646|29720|2450478|2450448|Miss|Anna|Todd|N|23|2|1988|BAHRAIN||Anna.Todd@aubOUOyCrdNeV.edu|2452368| +3262|AAAAAAAAOLMAAAAA|564489|2429|42416|2451140|2451110|Mrs.|Kathleen|Negron|Y|17|10|1948|AMERICAN SAMOA||Kathleen.Negron@5BetqFs4.com|2452288| +3263|AAAAAAAAPLMAAAAA|1118639|6988|584|2449104|2449074|Dr.|Joyce|Scherer|N|31|3|1958|ERITREA||Joyce.Scherer@yrX1hZ8r5iYT.com|2452547| +3264|AAAAAAAAAMMAAAAA|71860|4154|38448|2450517|2450487|Ms.|Barbara|Hall|N|10|9|1949|LEBANON||Barbara.Hall@JkIrkDercv0.org|2452614| +3265|AAAAAAAABMMAAAAA|201379|4473|44367|2452583|2452553|Sir|Kevin|Pierce|Y|14|2|1942|NEPAL||Kevin.Pierce@L02.org|2452565| +3266|AAAAAAAACMMAAAAA|985679|295|20064|2451868|2451838|Dr.|Aubrey|Brink|N|24|1|1966|FAROE ISLANDS||Aubrey.Brink@a6TxzhTCo71h.com|2452617| +3267|AAAAAAAADMMAAAAA|1737448|4421|31394|2450100||||Fleming|N||2||FIJI|||2452506| +3268|AAAAAAAAEMMAAAAA|1755387|1122|39257|2450116|2450086|Miss|Gladys|Bailey|N|19|5|1968|CROATIA||Gladys.Bailey@tBIL6JiQ4V.com|2452561| +3269|AAAAAAAAFMMAAAAA|319177|575|25173|2450088|2450058|Ms.|Grace|Dawson|N|5|10|1955|GRENADA||Grace.Dawson@4dIjvNN.edu|2452307| +3270|AAAAAAAAGMMAAAAA|567974|4603|47185|2451474|2451444|Miss|Emily|Roberts|N|17|7|1958|JAMAICA||Emily.Roberts@AsOE.com|2452471| +3271|AAAAAAAAHMMAAAAA|1051278|4629|16780|2451293|2451263|Dr.|Brian|Sommers|Y|8|2|1957|BANGLADESH||Brian.Sommers@R5u6BhuS.org|2452473| +3272|AAAAAAAAIMMAAAAA|1756209|2981|28506|2449703|2449673|Dr.|Edward|Sanchez|N|26|11|1927|PERU||Edward.Sanchez@MsTu.com|2452405| +3273|AAAAAAAAJMMAAAAA|1078513|6797|8281|2449923|2449893|Dr.|Adolph|Kerns|N|30|11|1975|GUAM||Adolph.Kerns@UioI362F01E.org|2452473| +3274|AAAAAAAAKMMAAAAA|811623|5551|13479|2451710|2451680|Dr.|William|Shea|N|15|2|1990|BHUTAN||William.Shea@mbtHZcjqu.edu|2452339| +3275|AAAAAAAALMMAAAAA|1544779|1834|11336|2451879|2451849|Sir|Tim|Combs|Y|14|4|1968|AUSTRIA||Tim.Combs@OGtkZOKM4Vl.com|2452476| +3276|AAAAAAAAMMMAAAAA|1577723|3258|43546|2451810|2451780|Dr.|Mildred|Webb|Y|26|12|1970|MARTINIQUE||Mildred.Webb@FoTBdOUYcEmI1.org|2452541| +3277|AAAAAAAANMMAAAAA|644159|4258|7204|2451541|2451511|Miss|Kimberly|Bennett|N|2|4|1989|THAILAND||Kimberly.Bennett@PRMjtFGq.org|2452419| +3278|AAAAAAAAOMMAAAAA|||33877|2452244||||||29||1990|GUADELOUPE|||2452433| +3279|AAAAAAAAPMMAAAAA|224432|4551|16670|2452096|2452066|Miss|Emma|Chavez|Y|24|8|1954|HONG KONG||Emma.Chavez@mFebUd7KoKdJrs.com|2452469| +3280|AAAAAAAAANMAAAAA|1793154|2474|9794|2451725|2451695|Miss|Ernestine|Gonzalez|N|2|2|1969|AFGHANISTAN||Ernestine.Gonzalez@SMnoAHB0ljzibGava.org|2452645| +3281|AAAAAAAABNMAAAAA|1477871|7104|30737|2449839|2449809|Miss|Martha|Moore|N|5|7|1985|BOUVET ISLAND||Martha.Moore@s.org|2452584| +3282|AAAAAAAACNMAAAAA|593644|4802|34313|2451802|2451772|Sir|Larry|Lockett|Y|17|5|1980|CANADA||Larry.Lockett@oV.org|2452326| +3283|AAAAAAAADNMAAAAA|649781|4503|28825|2449171|2449141|Dr.|Arthur|Santos|N|10|5|1992|HONDURAS||Arthur.Santos@po4VItpMKspGNV9Gqrms.org|2452353| +3284|AAAAAAAAENMAAAAA|||9867||||Jayne||||9|1924|UZBEKISTAN||Jayne.Johnson@aciaB5EARQC.edu|| +3285|AAAAAAAAFNMAAAAA|828095|341|27034|2449578|2449548|Dr.|Ryan|Herbert|Y|21|3|1937|TRINIDAD AND TOBAGO||Ryan.Herbert@q8A0pzp1YsI.edu|2452287| +3286|AAAAAAAAGNMAAAAA|850952|5229|19953|2450364|2450334|Dr.|Annette|Dodge|Y|28|4|1939|SAINT HELENA||Annette.Dodge@QfYDdZceoU7SxB1GLY.edu|2452506| +3287|AAAAAAAAHNMAAAAA|23707|5341|15422|2450721|2450691|Sir|Ezekiel|Davila|Y|7|11|1973|HONG KONG||Ezekiel.Davila@4jcgmQ.com|2452509| +3288|AAAAAAAAINMAAAAA|92715|813|22111|2449801|2449771|Dr.|Vernon|Thompson|N|12|5|1954|C�TE D'IVOIRE||Vernon.Thompson@NQ.com|2452437| +3289|AAAAAAAAJNMAAAAA|225776|3598|19420|2450894|2450864|Mrs.|Holly|Marin|N|1|5|1979|ZAMBIA||Holly.Marin@Q3vRZE6yebkGY6.edu|2452563| +3290|AAAAAAAAKNMAAAAA|1600355|1273|36591|2450723|2450693|Dr.|Kimberly|Talbot|N|10|11|1976|GREECE||Kimberly.Talbot@7glTr8GaUo1qK.com|2452343| +3291|AAAAAAAALNMAAAAA|1555002|815|22628|2449253|||||||||BOUVET ISLAND|||| +3292|AAAAAAAAMNMAAAAA|170235|6092|48796|2451119|2451089|Sir|Michael|Hacker|N|6|9|1982|SRI LANKA||Michael.Hacker@dkNgEh3Udg.org|2452648| +3293|AAAAAAAANNMAAAAA|504011|5649|2440|2450285|2450255|Dr.|Howard|Rowley|N|8|2|1937|NETHERLANDS ANTILLES||Howard.Rowley@FTHDR.org|2452613| +3294|AAAAAAAAONMAAAAA|1306133|4300|20067|2450064|2450034|Sir|David|Diaz|Y|24|4|1982|AFGHANISTAN||David.Diaz@xn9P.org|2452447| +3295|AAAAAAAAPNMAAAAA|1876279|5292|13505|2449090|2449060|Dr.|Paul|Laird|Y|2|11|1986|QATAR||Paul.Laird@am.com|2452523| +3296|AAAAAAAAAOMAAAAA|394179|92|3558|2450077|2450047|Mrs.|Aaron|Moore|Y|30|8|1982|EGYPT||Aaron.Moore@HG0KcLn8xjVnqm.org|2452416| +3297|AAAAAAAABOMAAAAA|1741|780|39466|2450145|2450115|Miss|Carrie|Nolan|Y|28|9|1951|CZECH REPUBLIC||Carrie.Nolan@zoc.edu|2452419| +3298|AAAAAAAACOMAAAAA|1131669|640|1164|2450586|2450556|Sir|Oscar|Toney|N|8|10|1975|GUADELOUPE||Oscar.Toney@T.edu|2452447| +3299|AAAAAAAADOMAAAAA|1581408|4756|10857|2451777|2451747|Ms.|Carolyn|Colvin|Y|11|11|1940|WESTERN SAHARA||Carolyn.Colvin@1jUSYqm3Apk.org|2452613| +3300|AAAAAAAAEOMAAAAA|||8494||||||N|23|11|1986|FINLAND|||| +3301|AAAAAAAAFOMAAAAA|1392273|4283|9540|2451378|2451348|Miss|Helen|Jenkins|N|6|12|1945|ALBANIA||Helen.Jenkins@PuJRAobZF0z.org|2452546| +3302|AAAAAAAAGOMAAAAA|1507894|3717|45187|2452262|2452232|Dr.|Joseph|Mclane|Y|23|9|1947|SURINAME||Joseph.Mclane@hzV7.edu|2452590| +3303|AAAAAAAAHOMAAAAA|1201541|5317|293|2452247|2452217|Miss|Louella|Daniels|N|30|4|1971|TOKELAU||Louella.Daniels@IoYijloyh.com|2452643| +3304|AAAAAAAAIOMAAAAA|734938|3398|45458|2449691|2449661|Dr.|Phillip|Davis|Y|10|7|1969|SLOVAKIA||Phillip.Davis@DqTNr2ApBon.edu|2452322| +3305|AAAAAAAAJOMAAAAA|502541|6980|12266|2450762|2450732|Miss|Joye|Julian|N|8|3|1935|MALDIVES||Joye.Julian@9p95A6Y0O.edu|2452290| +3306|AAAAAAAAKOMAAAAA|998772|2358|31944|2451943|2451913|Mr.|David|Nelson|N|13|5|1955|INDIA||David.Nelson@FMtcrDQR0gnTrXzp.com|2452618| +3307|AAAAAAAALOMAAAAA|1095991|2584|19577|2451242|2451212|Mr.|Raymond|Hunter|N|15|11|1964|NETHERLANDS ANTILLES||Raymond.Hunter@zEkOrG2O7OxU.com|2452488| +3308|AAAAAAAAMOMAAAAA|519432|1145|39211|2451613|2451583|Sir|Harry|Bowman|N|29|1|1943|SLOVENIA||Harry.Bowman@6Le3Ik.com|2452539| +3309|AAAAAAAANOMAAAAA|19722|5727|18179|2452521|2452491|Mr.|Horace|Cabrera|N|21|7|1962|MOROCCO||Horace.Cabrera@v2uKmAaSlkd.org|2452633| +3310|AAAAAAAAOOMAAAAA|909405|3380|31802|2449724|2449694|Mr.|Pete|Coleman|N|25|6|1932|ANTARCTICA||Pete.Coleman@eIKlhf7LGkKH9RV.edu|2452530| +3311|AAAAAAAAPOMAAAAA|1220634|5504|15127|2451175|2451145|Dr.|Ray|Golden|Y|1|12|1987|CANADA||Ray.Golden@Efhyampf41y.com|2452305| +3312|AAAAAAAAAPMAAAAA|1661485|922|5206|2450418|2450388|Dr.|Pablo|Mckenzie|Y|30|10|1976|LEBANON||Pablo.Mckenzie@1urQLV1bnVjlz.org|2452403| +3313|AAAAAAAABPMAAAAA|160619|3723|8827|2452370|2452340|Sir|Jason|Oliver|N|12|10|1963|BOTSWANA||Jason.Oliver@fA5IJuubXJ7I3.org|2452623| +3314|AAAAAAAACPMAAAAA|1783315|3161|9054|2450374|2450344|Dr.|Lourie|Miller|Y|1|11|1955|JERSEY||Lourie.Miller@feRTr4UPfaY.edu|2452379| +3315|AAAAAAAADPMAAAAA|831768|4215|32215|2450510|2450480|Miss|Marta|Ross|N|4|7|1931|AZERBAIJAN||Marta.Ross@2XSND61k4o9eFe.edu|2452417| +3316|AAAAAAAAEPMAAAAA|1017604|2446|14841|2450031|2450001|Dr.|Gaynell|Cartwright|N|12|3|1953|COSTA RICA||Gaynell.Cartwright@3IhRrEPp.org|2452533| +3317|AAAAAAAAFPMAAAAA|1018930|53|20376|2451529|2451499|Dr.|Linda|Jackson|N|1|3|1985|TUNISIA||Linda.Jackson@MQlKHk6E.com|2452514| +3318|AAAAAAAAGPMAAAAA|1729008|3375|48189|2449764|2449734|Ms.|Dorothy|Williams|Y|25|7|1924|KIRIBATI||Dorothy.Williams@cVXE2mj2LNPF.com|2452438| +3319|AAAAAAAAHPMAAAAA|1774749|5579|9256|2451098|2451068|Sir|Robert|Foret|N|15|3|1963|JAMAICA||Robert.Foret@rVVnJd4RluGf38zjFe5G.com|2452498| +3320|AAAAAAAAIPMAAAAA|247144|1236|37328|2449564|2449534|Mrs.|Beth|Hope|N|11|3|1957|HUNGARY||Beth.Hope@Rj1osPSM8Sxmt.org|2452301| +3321|AAAAAAAAJPMAAAAA|1156068|1976|6331|2452231|2452201|Ms.|Rita|Martin|Y|16|3|1976|SURINAME||Rita.Martin@17FKg0yTNek.com|2452630| +3322|AAAAAAAAKPMAAAAA|||15663|2451317||||Benjamin|Y|12|6|1931|GUINEA-BISSAU|||| +3323|AAAAAAAALPMAAAAA|1249951|5361|21971|2451284|2451254|Mrs.|Carla|Harrington|N|16|9|1950|MALTA||Carla.Harrington@UnBkbU0.org|2452355| +3324|AAAAAAAAMPMAAAAA|926763|3515|36667|2450427|2450397|Mr.|William|Healy|N|27|11|1926|NEW CALEDONIA||William.Healy@3IKCSKzoS.org|2452294| +3325|AAAAAAAANPMAAAAA|692227|790|39753|2449115|2449085|Sir|Travis|Ruiz|N|30|7|1962|LESOTHO||Travis.Ruiz@TVoc5yDR6zVodMSH.edu|2452544| +3326|AAAAAAAAOPMAAAAA|1847640|3531|11422|2452233|2452203|Sir|Jay|Kerr|Y|31|8|1933|LEBANON||Jay.Kerr@ax4vM3LAJb89Tj.com|2452511| +3327|AAAAAAAAPPMAAAAA|||1033||||||||12|1938|||Rosa.Coe@kNih4l.edu|2452618| +3328|AAAAAAAAAANAAAAA|836606|1642|24325|2451950|2451920|Mr.|Olen|Johnson|N|30|3|1967|TIMOR-LESTE||Olen.Johnson@o64e.edu|2452644| +3329|AAAAAAAABANAAAAA|363257|6558|43906|2449179|2449149|Ms.|Tina|Johnson|N|7|12|1987|ERITREA||Tina.Johnson@CIg0.edu|2452507| +3330|AAAAAAAACANAAAAA|238438|4991|8297|2449817|2449787|Dr.|Ophelia|Henson|Y|13|12|1967|MEXICO||Ophelia.Henson@AqboGe2Z.edu|2452519| +3331|AAAAAAAADANAAAAA|251065|2864|20216|2450568|2450538|Dr.|Kathryn|Marcum|N|25|9|1933|ANGUILLA||Kathryn.Marcum@EQyOeL0q3JM.edu|2452507| +3332|AAAAAAAAEANAAAAA|554913|4632|35998|2449120|2449090|Ms.|Darla|Johnson|N|31|5|1934|DOMINICA||Darla.Johnson@cAYkf7ZIyF8OqfEZ0O.edu|2452356| +3333|AAAAAAAAFANAAAAA|1495986|4672|18158|2449402|2449372|Mrs.|Nancy|Walker|N|19|4|1988|MALAYSIA||Nancy.Walker@ujXpv4.edu|2452637| +3334|AAAAAAAAGANAAAAA|470093|2154|22262|2450844|2450814|Mr.|Joseph|Jones|Y|1|10|1954|MOROCCO||Joseph.Jones@V3LN1f8vi5.org|2452507| +3335|AAAAAAAAHANAAAAA|1501918|4533|2395|2451507|2451477|Dr.|Billy|Swan|Y|18|6|1958|ANGOLA||Billy.Swan@syEg189TjG.org|2452630| +3336|AAAAAAAAIANAAAAA|852202|176|21703|2451332|2451302|Dr.|John|Roberts|N|16|5|1981|MARTINIQUE||John.Roberts@IbBp8UYxBHI7U9.org|2452385| +3337|AAAAAAAAJANAAAAA|1653421|3543|39665|2451060|2451030|Miss|Ida|Grady|Y|22|9|1965|PERU||Ida.Grady@9YXyOPZsGTPjf.org|2452510| +3338|AAAAAAAAKANAAAAA|657902|31|19713|2450542|2450512|Sir|Stanton|Griswold|Y|21|9|1968|FINLAND||Stanton.Griswold@ilp.org|2452633| +3339|AAAAAAAALANAAAAA|1192421|5582|47143|2449955|2449925|Ms.|Anna|Nash|Y|14|12|1981|ARMENIA||Anna.Nash@ctXbnvPNQR.edu|2452352| +3340|AAAAAAAAMANAAAAA|577091|1739|30387|2449995|2449965|Dr.|Carlos|Owens|N|23|12|1983|SERBIA||Carlos.Owens@dI3UZfS3qOZJ9.com|2452518| +3341|AAAAAAAANANAAAAA|772565|2317|24022|2451033|2451003|Mr.|Charles|Johnson|N|11|1|1964|MADAGASCAR||Charles.Johnson@Zog5DL5EBgb80tUSPA.org|2452365| +3342|AAAAAAAAOANAAAAA|949661|2048|16035|2451650|2451620|Miss|Colleen|Johnston|N|16|2|1967|UNITED STATES||Colleen.Johnston@Dhsyt02jpOp0Ttl.com|2452317| +3343|AAAAAAAAPANAAAAA|1771938|4002|48866|2450881|2450851|Ms.|Maribel|Stevens|Y|29|5|1935|WESTERN SAHARA||Maribel.Stevens@3.org|2452451| +3344|AAAAAAAAABNAAAAA|709842|4772|15702|2452674|2452644|Sir|William|Allen|Y|4|4|1949|MONTENEGRO||William.Allen@X7Va.com|2452579| +3345|AAAAAAAABBNAAAAA|998209|1144|34079|2452091|2452061|Mrs.|Alma|Parson|N|8|9|1964|UNITED KINGDOM||Alma.Parson@lPVKxXXjr5bG4.edu|2452517| +3346|AAAAAAAACBNAAAAA|857656|4456|27656|2451526|2451496|Sir|Robert|Gardner|Y|25|10|1972|AUSTRALIA||Robert.Gardner@sqGBtlu7.com|2452616| +3347|AAAAAAAADBNAAAAA|1183077|2770|20510|2451459|2451429|Miss|Maryanne|Grant|N|29|4|1983|NICARAGUA||Maryanne.Grant@uYs73DOKgHm6xxR.org|2452494| +3348|AAAAAAAAEBNAAAAA|1344022|2174|12373|2451111|2451081|Dr.|Jimmie|Mcneal|Y|1|5|1985|KYRGYZSTAN||Jimmie.Mcneal@dycaNtuIkM0kMtr.org|2452615| +3349|AAAAAAAAFBNAAAAA|1279395|4309|470|2452378|2452348|Sir|Douglas|Larsen|N|29|7|1987|MONTSERRAT||Douglas.Larsen@Jxxx390DAq4a.com|2452634| +3350|AAAAAAAAGBNAAAAA|1047813|5546|28479|2452359|2452329|Dr.|Karen|Fernandez|N|11|1|1980|TUVALU||Karen.Fernandez@B2Ms.edu|2452311| +3351|AAAAAAAAHBNAAAAA|1052594|4287|11533|2449412|2449382|Sir|Michael|Parks|Y|30|1|1976|ECUADOR||Michael.Parks@yumNRc7mH2KM9Gg.edu|2452288| +3352|AAAAAAAAIBNAAAAA|1180989|204|18219|2452435|2452405|Miss|Heidi|Lopez|Y|3|10|1940|BOUVET ISLAND||Heidi.Lopez@9toeCQqyad.org|2452385| +3353|AAAAAAAAJBNAAAAA|989773|1750|7727|2451284|2451254|Dr.|Arnold|Rice|N|13|3|1980|C�TE D'IVOIRE||Arnold.Rice@V.edu|2452508| +3354|AAAAAAAAKBNAAAAA|1611609|1035|8861|2451577|2451547|Ms.|Rhea|Long|N|26|5|1972|AUSTRIA||Rhea.Long@viy7VteEU1uf.edu|2452312| +3355|AAAAAAAALBNAAAAA|1515082|723|20364|2451222|2451192|Sir|Clifton|Rice|N|3|7|1924|SRI LANKA||Clifton.Rice@Mh5jZmv24.org|2452332| +3356|AAAAAAAAMBNAAAAA|1309239|6485|39731|2452516|2452486|Dr.|Richard|Driggers|N|20|3|1968|CHRISTMAS ISLAND||Richard.Driggers@H9H5Fd5Usk.org|2452372| +3357|AAAAAAAANBNAAAAA|878528|3141|38601|2449032|2449002|Ms.|Sheila|Haynes|Y|23|8|1931|YEMEN||Sheila.Haynes@7jip8pR.edu|2452396| +3358|AAAAAAAAOBNAAAAA|1659173|1352|23131|||||Clawson|N||4|1942|||Margaret.Clawson@oyZ.edu|2452498| +3359|AAAAAAAAPBNAAAAA|1295733|2758|21383|2450047|2450017|Ms.|Ella|Collins|Y|18|8|1957|MONTSERRAT||Ella.Collins@fcM1x3D.com|2452364| +3360|AAAAAAAAACNAAAAA|1652677|1746|4062|2450970|2450940|Sir|Clarence|Snyder|N|15|10|1946|ANDORRA||Clarence.Snyder@aIsh.com|2452360| +3361|AAAAAAAABCNAAAAA|519571|752|35133|2450949|2450919|Mr.|Christopher|Mcallister|Y|25|6|1947|MAYOTTE||Christopher.Mcallister@otSxcYYonqFVe1tEBF.edu|2452309| +3362|AAAAAAAACCNAAAAA|94636|3139|20489|2450748|2450718|Mrs.|Elaine|Lewis|Y|25|6|1963|GREENLAND||Elaine.Lewis@8.com|2452560| +3363|AAAAAAAADCNAAAAA|1804155|6766|3424|2450616|2450586|Sir|James|Craig|Y|23|12|1973|ERITREA||James.Craig@Ublr7r722B98BNS.edu|2452588| +3364|AAAAAAAAECNAAAAA|805084|2279|21827|2451832|2451802|Sir|Willis|Smallwood|Y|12|12|1949|BENIN||Willis.Smallwood@jF8AjbrbV1Eg.edu|2452424| +3365|AAAAAAAAFCNAAAAA|1164450|788|279|2451267|2451237|Dr.|Ricky|Dowdy|N|18|1|1979|SURINAME||Ricky.Dowdy@B2lROXztG390y1Nx8.edu|2452644| +3366|AAAAAAAAGCNAAAAA|774247|4373|33346|2451131|2451101|Dr.|Edward|Murdock|N|28|11|1961|TAJIKISTAN||Edward.Murdock@vzslHfTt8t.edu|2452383| +3367|AAAAAAAAHCNAAAAA|1487082|901|40673|2449651|2449621|Mrs.|Susan|Murray|N|10|12|1973|JORDAN||Susan.Murray@gcH5ihxOXMVBDh2LhAS.org|2452516| +3368|AAAAAAAAICNAAAAA|328527|1048|29922|2452389|2452359|Dr.|Vincent|Mcmurray|Y|15|7|1937|TOGO||Vincent.Mcmurray@SMV.edu|2452582| +3369|AAAAAAAAJCNAAAAA|1789254|7059|45264|2450053|2450023|Sir|Scott|Gilman|N|21|7|1947|NEPAL||Scott.Gilman@uoK00G9yU1bX.org|2452311| +3370|AAAAAAAAKCNAAAAA|804122|6142|6209|2450569|2450539|Mrs.|Sharon|Tavares|Y|1|1|1937|KUWAIT||Sharon.Tavares@mldB2ZU.org|2452359| +3371|AAAAAAAALCNAAAAA|544188|4232|49876|2452204|2452174|Sir|Matthew|Gerald|Y|1|1|1985|HAITI||Matthew.Gerald@V.com|2452643| +3372|AAAAAAAAMCNAAAAA|972880|3300|42920|2450252|2450222|Miss|Sally|Charles|Y|5|5|1969|JERSEY||Sally.Charles@jxR6DIvbYO0zm.edu|2452468| +3373|AAAAAAAANCNAAAAA|810724|6746|46015|2451020|2450990|Mrs.|Sandra|Deane|N|17|2|1943|TURKEY||Sandra.Deane@DsHNaQp5.edu|2452493| +3374|AAAAAAAAOCNAAAAA|1686442|135|31828|2450291|2450261|Dr.|Anthony|Robinson|N|14|9|1936|TIMOR-LESTE||Anthony.Robinson@pG.org|2452310| +3375|AAAAAAAAPCNAAAAA|660496|4583|47870|2451468|2451438|Ms.|Dorothy|Buckley|N|29|9|1991|FAROE ISLANDS||Dorothy.Buckley@cy.com|2452457| +3376|AAAAAAAAADNAAAAA|30885||2255||||||N|||||||| +3377|AAAAAAAABDNAAAAA|250889|731|38573|2449268|2449238|Ms.|Anh|Stock|Y|19|10|1958|BENIN||Anh.Stock@D4lPzU4yCQEMl.edu|2452440| +3378|AAAAAAAACDNAAAAA|475815|1984|2200|2449194|2449164|Dr.|Ana|Castro|Y|21|5|1976|UZBEKISTAN||Ana.Castro@poBMiUkrR6.edu|2452598| +3379|AAAAAAAADDNAAAAA|1631572|6819|12113|2451718|2451688|Sir|Kerry|Myers|Y|23|10|1948|PALAU||Kerry.Myers@b1KSOvrQbc.org|2452631| +3380|AAAAAAAAEDNAAAAA|865589|6029|12869|2451534|2451504|Dr.|Kim|Rome|N|1|12|1950|EL SALVADOR||Kim.Rome@970bbFkHOmAsn.edu|2452377| +3381|AAAAAAAAFDNAAAAA|411546|1640|5381|2452372|2452342|Miss|Janet|Stansberry|N|13|2|1973|ANTIGUA AND BARBUDA||Janet.Stansberry@O2.org|2452410| +3382|AAAAAAAAGDNAAAAA|235603|2639|19228|2449777|2449747|Mrs.|Olivia|Antonio|N|30|1|1972|CHRISTMAS ISLAND||Olivia.Antonio@bQxabzBguMXDPsuhV.com|2452557| +3383|AAAAAAAAHDNAAAAA|131425|1506|29187|2452562|2452532|Dr.|Leena|Jones|Y|15|2|1971|ANDORRA||Leena.Jones@rqbhyMKnlucU.edu|2452567| +3384|AAAAAAAAIDNAAAAA|248984|426|45021|2450006|2449976|Ms.|Minnie|Bruce|Y|24|9|1953|BELIZE||Minnie.Bruce@ZDiifZS7nEdeoddTA.com|2452357| +3385|AAAAAAAAJDNAAAAA|68690|4012|9781|2449519|2449489|Mrs.|Nancy|Hunter|N|24|5|1926|MALDIVES||Nancy.Hunter@o8CDjg.com|2452309| +3386|AAAAAAAAKDNAAAAA|293429|5093|15204|2451563|2451533|Sir|Myron|Phillips|N|2|5|1957|ANGUILLA||Myron.Phillips@Qsmuib2HnCId8.org|2452500| +3387|AAAAAAAALDNAAAAA|329855|6942|22029|2450639|2450609|Ms.|Joyce|Nichols|Y|26|12|1981|MAURITANIA||Joyce.Nichols@gb.edu|2452353| +3388|AAAAAAAAMDNAAAAA|56584|2566|49748|2452430|2452400|Miss|Sandra|Quiroz|N|31|1|1957|HONG KONG||Sandra.Quiroz@X5VfievJ.edu|2452314| +3389|AAAAAAAANDNAAAAA|925571|4982|7640|2451479|2451449|Mrs.|Marlena|Reddy|Y|12|10|1975|BAHRAIN||Marlena.Reddy@mI.com|2452502| +3390|AAAAAAAAODNAAAAA|1509302|3315|11375|2450947|2450917|Dr.|Arlene|Taylor|Y|17|3|1957|MAYOTTE||Arlene.Taylor@L1hs9mgNnK.edu|2452338| +3391|AAAAAAAAPDNAAAAA|1538416|6632|12001|2451317|2451287|Mrs.|Tonya|Foster|N|21|9|1965|YEMEN||Tonya.Foster@fA2pzNJi.org|2452634| +3392|AAAAAAAAAENAAAAA|507440|728|36416|2451688|2451658|Mr.|Trevor|Lynch|N|2|1|1978|KIRIBATI||Trevor.Lynch@2pXO1xSkiqTMS5.edu|2452645| +3393|AAAAAAAABENAAAAA|427427|2822|43817||||||N|27|5|1951||||2452464| +3394|AAAAAAAACENAAAAA|994769|4981|26393|2449784|2449754|Mr.|Robert|Alexander|Y|1|6|1979|VIRGIN ISLANDS, U.S.||Robert.Alexander@AzLUvcZiK3re.edu|2452384| +3395|AAAAAAAADENAAAAA|1360008|1791|35974|2449075|2449045|Mrs.|Betty|Clay|N|30|4|1950|VENEZUELA||Betty.Clay@S0K.org|2452413| +3396|AAAAAAAAEENAAAAA|809859|5854|31849|2449687|2449657|Miss|Amy|Morgan|Y|3|9|1965|GUAM||Amy.Morgan@lqjBgblO.org|2452544| +3397|AAAAAAAAFENAAAAA|1116335|1907|32726|2449582|2449552|Dr.|Christopher|Moten|Y|13|9|1926|AMERICAN SAMOA||Christopher.Moten@A.com|2452293| +3398|AAAAAAAAGENAAAAA|1393472|577|14835|2450795|2450765|Mrs.|Ruth|Barnes|N|16|12|1956|GRENADA||Ruth.Barnes@j2Lr.com|2452463| +3399|AAAAAAAAHENAAAAA|753765|1798|41831|2452089|2452059|Mr.|Paul|Belton|Y|23|5|1982|NIGERIA||Paul.Belton@a9fJ43i6QJp7A6U.org|2452467| +3400|AAAAAAAAIENAAAAA|625547|6883|27372|2450767|2450737|Dr.|Kevin|Lawrence|Y|22|7|1991|SIERRA LEONE||Kevin.Lawrence@MlpAL6xEnqeF.edu|2452373| +3401|AAAAAAAAJENAAAAA|927836|1572|5755|2452557|2452527|Mr.|Robert|Shaw|Y|27|11|1945|TIMOR-LESTE||Robert.Shaw@K0MNe4QjaU5RB1.org|2452383| +3402|AAAAAAAAKENAAAAA|932499|223|32747|2450838|2450808|Mrs.|Laura|Lucas|Y|22|10|1960|NAMIBIA||Laura.Lucas@jQMhaSelKyL1.edu|2452613| +3403|AAAAAAAALENAAAAA|227279|7180|40328|2452254|2452224|Dr.|Marguerite|Prado|Y|13|11|1924|IRAQ||Marguerite.Prado@iHaMof4d8XKcad8L.edu|2452499| +3404|AAAAAAAAMENAAAAA|1017998|826|11301|2450180|2450150|Dr.|Samuel|Gonzalez|N|3|8|1965|THAILAND||Samuel.Gonzalez@mZld6avMLb7YEfM.com|2452562| +3405|AAAAAAAANENAAAAA|1354527|1168|3456|2450206|2450176|Dr.|Jennifer|Clark|N|25|1|1928|C�TE D'IVOIRE||Jennifer.Clark@fXYt.edu|2452296| +3406|AAAAAAAAOENAAAAA|1565478|6918|13958|2449614|2449584|Dr.|Lisa|Harrison|Y|14|3|1959|ANGUILLA||Lisa.Harrison@TG3mY7Ph928Z.edu|2452418| +3407|AAAAAAAAPENAAAAA|703447|4087|17527|2452370|2452340|Mrs.||Lara|||||NORWAY||Charlotte.Lara@Krm5h1oeBjbQO7NNfqOX.com|2452461| +3408|AAAAAAAAAFNAAAAA|169790|2407|47471|2449840|2449810|Mrs.|Janet|Sherman|N|4|2|1931|NIGERIA||Janet.Sherman@3tCSg.com|2452491| +3409|AAAAAAAABFNAAAAA|758308|3690|5872|2450311|2450281|Sir|Stephen|Santana|N|1|12|1979|MACAO||Stephen.Santana@8hS8tG.com|2452612| +3410|AAAAAAAACFNAAAAA|1264452|598|37591|2452327|2452297|Dr.|Robert|Barrett|Y|25|8|1990|JERSEY||Robert.Barrett@krEuoofSnzFIjkdJPvQ.edu|2452633| +3411|AAAAAAAADFNAAAAA|22552|6604|34656|2449198|2449168|Mr.|John|Shores|N|25|5|1990|KUWAIT||John.Shores@l.edu|2452395| +3412|AAAAAAAAEFNAAAAA|1877164|586|28008|2452602|2452572|Mr.|Joseph|Mcnair|N|6|8|1985|MYANMAR||Joseph.Mcnair@ke530D1pKDxNtlcm3.edu|2452433| +3413|AAAAAAAAFFNAAAAA|32910|3985|36307|2451765|2451735|Mr.|James|Novak|Y|20|1|1978|GREENLAND||James.Novak@5o3ax0SjN.org|2452613| +3414|AAAAAAAAGFNAAAAA|191082|245|7008|2450557|2450527|Dr.|Shawana|Blanchard|Y|30|9|1950|MOZAMBIQUE||Shawana.Blanchard@csYo7glsmmd.com|2452442| +3415|AAAAAAAAHFNAAAAA|381486|226|1413|2450599|2450569|Dr.|Leo|Saenz|Y|13|5|1973|R�UNION||Leo.Saenz@xC00NsZfv2.com|2452311| +3416|AAAAAAAAIFNAAAAA|1237642|2123|47851|2449595|2449565|Sir|Jeromy|Landry|N|15|1|1925|JORDAN||Jeromy.Landry@lsEh.com|2452627| +3417|AAAAAAAAJFNAAAAA|1568025|6846|19667|2450737|2450707|Mrs.|Jennifer|Hunter|Y|7|11|1968|HUNGARY||Jennifer.Hunter@2cMKovtirDA.edu|2452345| +3418|AAAAAAAAKFNAAAAA|1830327|1269|41411|2452497|2452467|Mr.|William|Berg|Y|8|1|1976|WALLIS AND FUTUNA||William.Berg@LGXsxfESOXCi.org|2452315| +3419|AAAAAAAALFNAAAAA|975612|5116|15420|2451246|2451216|Sir|David|Gonzalez|N|12|9|1977|GABON||David.Gonzalez@4Zq.org|2452574| +3420|AAAAAAAAMFNAAAAA|1496635|5827|13866|2452280|2452250|Miss|Anthony|Hernandez|N|12|1|1981|MYANMAR||Anthony.Hernandez@RtBip.edu|2452442| +3421|AAAAAAAANFNAAAAA|685290|7178|19819|2450746|2450716|Mr.|John|Winters|Y|5|7|1988|BELIZE||John.Winters@UjnHGYNlpSB.edu|2452535| +3422|AAAAAAAAOFNAAAAA|340590|5077|26364|2450830|2450800|Sir|Mark|Carter|N|8|3|1966|HAITI||Mark.Carter@cL.com|2452635| +3423|AAAAAAAAPFNAAAAA|1670326|6300|49027|2450316|2450286|Dr.|Thomas|King|Y|12|10|1943|SAINT LUCIA||Thomas.King@DcracULKB.org|2452417| +3424|AAAAAAAAAGNAAAAA|554927|3682|15637|2451838|2451808|Dr.|William|Anderson|Y|3|9|1948|JORDAN||William.Anderson@DpBo.edu|2452422| +3425|AAAAAAAABGNAAAAA|565457|4058|1459|2450641|2450611|Miss|Kimberly|Crowe|N|3|4|1930|GHANA||Kimberly.Crowe@OVrF5EQGYeXXA.org|2452443| +3426|AAAAAAAACGNAAAAA|59991|2251|38864|2449684|2449654|Mrs.|Tanya|Wilcox|N|6|1|1990|PARAGUAY||Tanya.Wilcox@le4FC2XJr1GuBcHj.com|2452627| +3427|AAAAAAAADGNAAAAA|352973|6355|6937|2449565|2449535|Dr.|Thomas|Gibson|N|24|2|1951|NORWAY||Thomas.Gibson@yJhx5PPlMz4.org|2452456| +3428|AAAAAAAAEGNAAAAA|826779|4545|24311|2450182|2450152|Dr.|Wayne|Ortiz|Y|16|5|1985|PERU||Wayne.Ortiz@R8TU17DiS95t7.org|2452444| +3429|AAAAAAAAFGNAAAAA|693470|124|41131|2450688|2450658|Mrs.|Jennifer|Sanchez|N|28|10|1945|KENYA||Jennifer.Sanchez@s1MfiSfgjRK.edu|2452607| +3430|AAAAAAAAGGNAAAAA|1695197|6297|27397|2450774|2450744|Miss|Traci|Boyette|Y|12|5|1983|BARBADOS||Traci.Boyette@7dVhBqBDScj.com|2452293| +3431|AAAAAAAAHGNAAAAA|1576664|3263|46709|2450989|2450959|Mr.|Joseph|Cummings|Y|10|3|1991|FIJI||Joseph.Cummings@5O30KAD.edu|2452284| +3432|AAAAAAAAIGNAAAAA|1861021|2670|43519|2449676|2449646|Dr.|Audrey|Simone|Y|6|1|1958|PORTUGAL||Audrey.Simone@Tv2.com|2452634| +3433|AAAAAAAAJGNAAAAA|1855635|1970|45858|2451806|2451776|Ms.|Esther|Oconnell|Y|13|9|1959|ICELAND||Esther.Oconnell@Ihx8.org|2452440| +3434|AAAAAAAAKGNAAAAA|1879491|1937|26430|2451832|2451802|Mrs.|Sarah|Gabriel|Y|10|3|1973|FINLAND||Sarah.Gabriel@Ccz67u3LOg.com|2452643| +3435|AAAAAAAALGNAAAAA|1642696|531|4319|2451064|2451034|Sir|Charles|Medeiros|Y|30|7|1935|MONTSERRAT||Charles.Medeiros@C9nzT9dut.com|2452564| +3436|AAAAAAAAMGNAAAAA|884713|3290|2267|2451180|2451150|Mr.|Gerald|Miller|N|28|5|1926|BAHRAIN||Gerald.Miller@qvMoQ6yR2Kp.com|2452429| +3437|AAAAAAAANGNAAAAA|1240346|3942|25938|2450938|2450908|Sir|Roland|Morris|N|17|6|1939|SOMALIA||Roland.Morris@cDg.edu|2452403| +3438|AAAAAAAAOGNAAAAA|739426|547|44468|2451485|2451455|Dr.|Carlota|Coleman|N|29|4|1980|TAJIKISTAN||Carlota.Coleman@1lnYbuOP61.org|2452564| +3439|AAAAAAAAPGNAAAAA|1619173|6491|39362|2449699|2449669|Miss|Helen|Garrison|Y|1|12|1962|HAITI||Helen.Garrison@rCpScxdPQgzRJ2puC.org|2452393| +3440|AAAAAAAAAHNAAAAA|507022|6199|39329|2449981|2449951|Sir|Robert|Smith|N|2|7|1983|MALAWI||Robert.Smith@aaC2N5K9pDR9.com|2452599| +3441|AAAAAAAABHNAAAAA|901275|7129|11383|2449655|2449625|Miss|Ida|Alvarez|Y|3|6|1970|SLOVENIA||Ida.Alvarez@fZy3MQqZt9h.edu|2452510| +3442|AAAAAAAACHNAAAAA|1898225|4004|40630|2451451|2451421|Mrs.|Susan|Cooper|Y|3|12|1952|MOLDOVA, REPUBLIC OF||Susan.Cooper@R1B.org|2452289| +3443|AAAAAAAADHNAAAAA|688125|1074|22823|2452494|2452464|Miss|Heather|Patrick|N|24|2|1955|NEPAL||Heather.Patrick@l8Pqtd0XVy.edu|2452643| +3444|AAAAAAAAEHNAAAAA|13591|715|42639|2451184|2451154|Mrs.|Isabel|Harrison|N|24|2|1928|SERBIA||Isabel.Harrison@7CS36bp1EgK.edu|2452633| +3445|AAAAAAAAFHNAAAAA|111849|4135|9221|2449048|2449018|Dr.|Bob|Schultz|Y|19|8|1938|MYANMAR||Bob.Schultz@ZG3sHo9FCZ8.org|2452458| +3446|AAAAAAAAGHNAAAAA|1153082|1013|26191|2450603|2450573|Dr.|David|Miranda|N|21|7|1990|LESOTHO||David.Miranda@yVT7.org|2452397| +3447|AAAAAAAAHHNAAAAA|1515014|36|18305|2450937|2450907|Sir|Austin|Griffin|N|25|10|1963|LEBANON||Austin.Griffin@4DmJ9B5IX2m3pyK4EX6.edu|2452607| +3448|AAAAAAAAIHNAAAAA|1572790|442|32494|2451971|2451941|Dr.|William|Dancy|Y|9|3|1937|BULGARIA||William.Dancy@rEg4gB02n.com|2452333| +3449|AAAAAAAAJHNAAAAA|249428|4998|14168|2451908|2451878|Mrs.|Delores|Moon|Y|13|1|1937|BOTSWANA||Delores.Moon@Y9vS6fMRl85eu.edu|2452401| +3450|AAAAAAAAKHNAAAAA|823454|1744|25187|||Sir||Regan|N|||1947|LEBANON|||| +3451|AAAAAAAALHNAAAAA|1839622|3235|27695|2450936|2450906|Dr.|James|Courtney|Y|29|7|1991|LEBANON||James.Courtney@RkgCRDyiIMkU6.edu|2452348| +3452|AAAAAAAAMHNAAAAA|712413|245|42477|2451573|2451543|Sir|Jose|Phifer|N|12|2|1935|SWEDEN||Jose.Phifer@xK6UzL5HYv.com|2452403| +3453|AAAAAAAANHNAAAAA|558534|6801|38221|2450333|2450303|Mrs.|Elizabeth|Self|Y|5|3|1965|GEORGIA||Elizabeth.Self@EDnCTSnHKi64.edu|2452303| +3454|AAAAAAAAOHNAAAAA|1200834|5769|257|2450352|2450322|Sir|Casey|Johnson|Y|13|1|1980|SWEDEN||Casey.Johnson@DT.edu|2452410| +3455|AAAAAAAAPHNAAAAA|408790|474|18205|2450273|2450243|Miss|Bernadette|Riggins|N|23|1|1992|VANUATU||Bernadette.Riggins@J.edu|2452452| +3456|AAAAAAAAAINAAAAA|59867|2428|189|2450957|2450927|Mr.|Chris|Jennings|N|21|11|1934|SWEDEN||Chris.Jennings@CEP9vu.org|2452594| +3457|AAAAAAAABINAAAAA|1920334|5767|6353|2449751|2449721|Dr.|Daniel|Nash|N|5|9|1936|AFGHANISTAN||Daniel.Nash@k6jHuCFicG10XOoa.org|2452606| +3458|AAAAAAAACINAAAAA|1021124|1947|45970|2449081|2449051|Mr.|Charles|Perry|N|10|4|1991|TIMOR-LESTE||Charles.Perry@xxnoZzSu320.com|2452308| +3459|AAAAAAAADINAAAAA||612|28700||2451381||Edward|Mitchell|N||||||Edward.Mitchell@pBkAd7DGA.edu|2452636| +3460|AAAAAAAAEINAAAAA|986564|4647|47883|2449455|2449425|Sir|Johnnie|Morrow|N|24|3|1950|SAMOA||Johnnie.Morrow@N6Cmq8ZA6E1.com|2452523| +3461|AAAAAAAAFINAAAAA|1168486||23781|2449152|2449122|Sir|Charles|Jung|Y|16|6||||Charles.Jung@XNhe8FjsSgF.edu|2452432| +3462|AAAAAAAAGINAAAAA|877916|1621|27373|2450985|2450955|Ms.|Julie|Keith|Y|10|3|1974|GABON||Julie.Keith@vh7Ho8NOXfg2mF.org|2452628| +3463|AAAAAAAAHINAAAAA|1129190|977|8764|2449345|2449315|Ms.|Natalie|Tijerina|Y|17|4|1965|PALAU||Natalie.Tijerina@SlhZGPmdkJ6K4R5h1.com|2452357| +3464|AAAAAAAAIINAAAAA|773084|6894|30704|2450283|2450253|Mrs.|Dorthea|Clark|Y|19|1|1983|GUERNSEY||Dorthea.Clark@1K1bVP3MGgGQe5.org|2452462| +3465|AAAAAAAAJINAAAAA|1670889|4079|27269|2451455|2451425|Miss|Mollie|Austin|Y|10|9|1989|COMOROS||Mollie.Austin@aBiusy9oPM.com|2452355| +3466|AAAAAAAAKINAAAAA|1746381|5247|15666|2449192|2449162|Sir|Johnnie|Bain|N|24|4|1980|SAINT LUCIA||Johnnie.Bain@YRfLBzXn2bb.org|2452333| +3467|AAAAAAAALINAAAAA|1562955|2233|31111|2449451|2449421|Miss|Annette|Finn|Y|4|2|1936|CHILE||Annette.Finn@y.com|2452498| +3468|AAAAAAAAMINAAAAA|1740335|6503|32318|2451440|2451410|Dr.|Amy|Mcdowell|N|31|12|1930|BENIN||Amy.Mcdowell@76kaDJi54GxI5.org|2452491| +3469|AAAAAAAANINAAAAA|1235295|281|12467|2449858|2449828|Dr.|Cynthia|Perez|N|18|10|1947|PORTUGAL||Cynthia.Perez@9TiD3cFXv.com|2452596| +3470|AAAAAAAAOINAAAAA|1575806|6470|34781|2450410|2450380|Ms.|Florence|Moody|Y|28|12|1937|NICARAGUA||Florence.Moody@25B.com|2452499| +3471|AAAAAAAAPINAAAAA|992960|1528|1696|2451428|2451398|Sir|Edward|Simmons|Y|20|6|1931|RWANDA||Edward.Simmons@mqQV1jnUfLtJP8Q.org|2452547| +3472|AAAAAAAAAJNAAAAA|1737544|305|5866|2450879|2450849|Dr.|Bryan|Samples|Y|11|2|1961|PARAGUAY||Bryan.Samples@KzGEe.com|2452392| +3473|AAAAAAAABJNAAAAA|1559860|3941|7222|2451250|2451220|Dr.|Maggie|Hsu|N|4|4|1951|AFGHANISTAN||Maggie.Hsu@9RIVB05qP.edu|2452559| +3474|AAAAAAAACJNAAAAA|1432369|1219|16069|2452013|2451983|Miss|Amanda|Jenkins|Y|11|8|1992|NEW CALEDONIA||Amanda.Jenkins@RYY0.edu|2452328| +3475|AAAAAAAADJNAAAAA|531590|373|36709|2450935|2450905|Ms.|Kathryne|Aguilar|N|13|12|1951|SUDAN||Kathryne.Aguilar@sLP.org|2452501| +3476|AAAAAAAAEJNAAAAA|897830|2190|12597|2451622|2451592|Dr.|Danny|Hill|Y|3|4|1959|EQUATORIAL GUINEA||Danny.Hill@01EFJR.com|2452593| +3477|AAAAAAAAFJNAAAAA|659444|758|14472|2451741|2451711|Miss|Ashley|Rafferty|N|16|2|1983|PANAMA||Ashley.Rafferty@xYBRPqBekdOUZ.com|2452369| +3478|AAAAAAAAGJNAAAAA|1440540|844|14348|2449217|2449187|Mr.|Earl|Norris|N|17|5|1968|ROMANIA||Earl.Norris@cTcDMq1B.org|2452340| +3479|AAAAAAAAHJNAAAAA|1354940|2207|37427|2450140|2450110|Miss|Anna|Franco|N|19|3|1964|ARGENTINA||Anna.Franco@c5nDH.edu|2452539| +3480|AAAAAAAAIJNAAAAA|1123928|2862|38996|2449246|2449216|Dr.|Josephine|Chan|N|9|6|1953|MEXICO||Josephine.Chan@Y1SgRZFte.edu|2452430| +3481|AAAAAAAAJJNAAAAA|934187|5488|17542|2449038|2449008|Dr.|Joseph|Ryder|N|14|6|1964|SENEGAL||Joseph.Ryder@4VJ.edu|2452482| +3482|AAAAAAAAKJNAAAAA|1277702|1777|3852|2452042|2452012|Miss|Vanessa|Arndt|Y|19|11|1969|CANADA||Vanessa.Arndt@CQaYob.edu|2452566| +3483|AAAAAAAALJNAAAAA|1295374|3601|41875|2449642|2449612|Sir|Rod|Reyna|N|30|4|1951|UNITED STATES||Rod.Reyna@BvtovImLSQ.org|2452304| +3484|AAAAAAAAMJNAAAAA|256161|1430|18652|2452026|2451996|Dr.|George|Blankenship|N|27|10|1953|MARTINIQUE||George.Blankenship@jpT9.org|2452343| +3485|AAAAAAAANJNAAAAA|349492|154|43263|2450051|2450021|Miss|Lisa|Mcpherson|Y|9|2|1938|BOLIVIA||Lisa.Mcpherson@apgrXOHXC5VN72Dj.org|2452373| +3486|AAAAAAAAOJNAAAAA|644576|571|27013|2449817|2449787|Ms.|Theresa|Curran|Y|15|11|1947|SLOVENIA||Theresa.Curran@zi6d8sv.org|2452610| +3487|AAAAAAAAPJNAAAAA|1433067|3583|6499|2452221|2452191|Dr.|Rodney|Rodriguez|N|29|7|1947|ZIMBABWE||Rodney.Rodriguez@D8r5.edu|2452467| +3488|AAAAAAAAAKNAAAAA|38588|3066|15497|2452258|2452228|Dr.|Ricky|Hunt|Y|26|11|1933|WALLIS AND FUTUNA||Ricky.Hunt@I9N.edu|2452487| +3489|AAAAAAAABKNAAAAA|161198|4521|49397|2450528|2450498|Mr.|William|Perez|Y|17|10|1976|ZAMBIA||William.Perez@Hue.com|2452408| +3490|AAAAAAAACKNAAAAA|671210|3829|47601|2451367|2451337|Ms.|Dawn|Davison|N|11|2|1975|HONG KONG||Dawn.Davison@locmDmpJE395RqglhM.edu|2452314| +3491|AAAAAAAADKNAAAAA|709573|6204|38835|2452092|2452062|Dr.|Derek|Hook|N|31|8|1964|VIET NAM||Derek.Hook@kuJJ3Npuhd.org|2452492| +3492|AAAAAAAAEKNAAAAA|131781|5627|43120|2450120|2450090|Dr.|Rodney|Ochoa|N|21|6|1955|VIET NAM||Rodney.Ochoa@MNIdlgCiHVxYk.com|2452401| +3493|AAAAAAAAFKNAAAAA|975226|1651|29229|2451858|2451828|Dr.|Sam|Ross|Y|19|3|1981|SYRIAN ARAB REPUBLIC||Sam.Ross@On4RV9jNVM.org|2452471| +3494|AAAAAAAAGKNAAAAA|1674964|3767|30923|2449922|2449892|Mr.|Walter|Johnston|Y|31|10|1978|WALLIS AND FUTUNA||Walter.Johnston@Mjx.edu|2452538| +3495|AAAAAAAAHKNAAAAA|195141|3592|32043|2450465|2450435|Dr.|Irene|Young|N|16|3|1978|GERMANY||Irene.Young@HyIpjcGJz.edu|2452400| +3496|AAAAAAAAIKNAAAAA|991492|4467|19842|2449708|2449678|Mr.|Donald|Edwards|Y|12|2|1986|VIET NAM||Donald.Edwards@juMR8gJDXs.org|2452571| +3497|AAAAAAAAJKNAAAAA|88447|116|16585|2450172|2450142|Dr.|Erick|Varner|Y|19|3|1956|FRENCH POLYNESIA||Erick.Varner@qt67uYJauqXyqJH.edu|2452608| +3498|AAAAAAAAKKNAAAAA|744836|2675|37761|2449799|2449769|Mr.|Tim|Lawson|N|23|3|1936|SRI LANKA||Tim.Lawson@jnxyjsPBzj60AOqVR.org|2452638| +3499|AAAAAAAALKNAAAAA|432423|3225|18827|2452236|2452206||Sue|Webb|||3||||Sue.Webb@nDfnX0zgs7pH.com|| +3500|AAAAAAAAMKNAAAAA|450784|3094|32604|2450485|2450455|Sir|Claude|Guidry|N|30|8|1979|SAN MARINO||Claude.Guidry@31aN7BVKZd.com|2452515| +3501|AAAAAAAANKNAAAAA|72003|5305|2654|2449953|2449923|Dr.|Rosalie|Ford|Y|22|7|1970|FAROE ISLANDS||Rosalie.Ford@osO.edu|2452513| +3502|AAAAAAAAOKNAAAAA|1435510|323|27183|2452565|2452535|Miss|Dorothy|Lash|Y|26|1|1952|ANTIGUA AND BARBUDA||Dorothy.Lash@G25zNaAedBTsDny7J.edu|2452404| +3503|AAAAAAAAPKNAAAAA||1057|46285|||Mrs.|Marina|Mueller||11||1936|||Marina.Mueller@euUh8VSjt8ggy.com|| +3504|AAAAAAAAALNAAAAA|1119393|6584|40622|2449307|2449277|Mrs.|Tara|Johnson|Y|1|10|1969|MOZAMBIQUE||Tara.Johnson@b7ONHDaHAskL.org|2452496| +3505|AAAAAAAABLNAAAAA|903270|6862|47654|2452518|2452488|Dr.|Dolores|Aviles|Y|28|7|1935|AFGHANISTAN||Dolores.Aviles@uI1hNP89sTdYVC.edu|2452584| +3506|AAAAAAAACLNAAAAA|1717645|5046|7247|2452396|2452366|Sir|Edward|Smith|N|11|2|1933|CAMBODIA||Edward.Smith@U.edu|2452404| +3507|AAAAAAAADLNAAAAA|333277|6105|46533|2449997|2449967|Sir|Billy|Hunt|N|12|4|1960|LESOTHO||Billy.Hunt@sa06V.edu|2452608| +3508|AAAAAAAAELNAAAAA|105449|4142|36388|2452597|2452567|Dr.|Jack|Hayes|Y|26|9|1944|UKRAINE||Jack.Hayes@xa05m8su.com|2452497| +3509|AAAAAAAAFLNAAAAA|1904088|6405|4322|2450297|2450267|Mr.|Donald|Kline|Y|16|1|1949|HONDURAS||Donald.Kline@negpEfxx9KK.edu|2452635| +3510|AAAAAAAAGLNAAAAA|245954|3821|18441|2450906|2450876|Dr.|Heather|Smith|Y|13|5|1959|GUAM||Heather.Smith@1zTq.com|2452459| +3511|AAAAAAAAHLNAAAAA|1265339|6810|9481|2451137|2451107|Miss|Marie|Rice|Y|2|3|1991|TONGA||Marie.Rice@ZtLN.org|2452292| +3512|AAAAAAAAILNAAAAA|334626|5072|49918|2451751|2451721|Miss|Emma|Gustafson|N|10|2|1934|WESTERN SAHARA||Emma.Gustafson@UEMSB8VMOlUmI.org|2452536| +3513|AAAAAAAAJLNAAAAA|1761093|411|41282|2450609|2450579|Sir|Gary|Cassidy|N|27|8|1991|SAINT LUCIA||Gary.Cassidy@RP7FjVE8iD.org|2452563| +3514|AAAAAAAAKLNAAAAA|1545722|2861|36847|2450562|2450532|Dr.|Mark|Guerrero|N|29|8|1971|IRAQ||Mark.Guerrero@DjFqbcPq.org|2452475| +3515|AAAAAAAALLNAAAAA|1620530|4245|45160||2450440|Ms.|||N||8|||||2452524| +3516|AAAAAAAAMLNAAAAA|856740|1466|31839|2451536|2451506|Dr.|Brandi|Henderson|N|27|2|1984|BENIN||Brandi.Henderson@bL94O78.org|2452533| +3517|AAAAAAAANLNAAAAA|1397002|3225|29895|2449550|2449520|Ms.|Beth|Thigpen|Y|4|6|1933|MONGOLIA||Beth.Thigpen@OlQpgYKE9QaCB.org|2452325| +3518|AAAAAAAAOLNAAAAA|1849258|4871|18931|2450596|2450566|Ms.|Sheila|Hopkins|Y|6|12|1962|MARSHALL ISLANDS||Sheila.Hopkins@GvP.edu|2452356| +3519|AAAAAAAAPLNAAAAA|1393698|4757|17816|2452081|2452051|Mr.|George|Wheaton|N|26|7|1961|CZECH REPUBLIC||George.Wheaton@1MdI1g89.org|2452400| +3520|AAAAAAAAAMNAAAAA|1897863|2642|35468|2451474|2451444|Dr.|Brian|Lusk|Y|26|6|1934|GUINEA||Brian.Lusk@Pv4g4jELqmsi80mdj.edu|2452529| +3521|AAAAAAAABMNAAAAA|385526|2897|7421|2451347|2451317|Mr.|Donnell|Ray|N|27|3|1952|ALAND ISLANDS||Donnell.Ray@AdlijgybY.edu|2452387| +3522|AAAAAAAACMNAAAAA|627238|4672|39706|2451302|2451272|Mr.|Sean|Mayo|N|26|6|1969|CHRISTMAS ISLAND||Sean.Mayo@ZyQcEyNp.edu|2452600| +3523|AAAAAAAADMNAAAAA|281270|4263|23256|2449489|2449459|Mr.|Sean|Bell|Y|2|8|1946|GEORGIA||Sean.Bell@miBp.org|2452631| +3524|AAAAAAAAEMNAAAAA|464775|6593|44411|2450632|2450602|Dr.|David|Mark|Y|7|12|1987|ZIMBABWE||David.Mark@SsfayJQzk0VmzS.edu|2452647| +3525|AAAAAAAAFMNAAAAA|707750|5418|30686|2452087|2452057|Mrs.|Monica|Parsons|Y|6|8|1968|INDIA||Monica.Parsons@BV7ZyoQ.edu|2452577| +3526|AAAAAAAAGMNAAAAA|452989||15204|2450218||Mr.|Christopher||N|19|||GAMBIA|||2452338| +3527|AAAAAAAAHMNAAAAA|1361223|4154|19885|2450580|2450550|Mrs.|Anna|Taylor|N|16|3|1969|ESTONIA||Anna.Taylor@ZgqKb8v4T.edu|2452345| +3528|AAAAAAAAIMNAAAAA|880728|1620|42065|2452527|2452497|Sir|Spencer|Mcclendon|Y|18|11|1976|JAPAN||Spencer.Mcclendon@Pzox2mdQSHnO7.com|2452567| +3529|AAAAAAAAJMNAAAAA|790044|4176|42308|2452522|2452492|Miss|Heather|Sellars|Y|3|1|1952|WESTERN SAHARA||Heather.Sellars@UBq5.edu|2452641| +3530|AAAAAAAAKMNAAAAA|607632|6246|10460|2451992|2451962|Dr.|David|Smith|N|8|10|1959|GIBRALTAR||David.Smith@UdAmnbNb67d3Z.com|2452393| +3531|AAAAAAAALMNAAAAA|992370|4978|23986|2451193|2451163|Mrs.|Gladys|Berger|Y|12|3|1974|BARBADOS||Gladys.Berger@cQGvMB4c5ZLnKf8y0X.edu|2452360| +3532|AAAAAAAAMMNAAAAA|1876585|2673|29514|2452039|2452009|Dr.|George|Mitchell|Y|10|6|1939|GUINEA-BISSAU||George.Mitchell@NMI7JtyfypOtP.com|2452633| +3533|AAAAAAAANMNAAAAA|1569117|2670|8623|2451180|2451150|Dr.|Carl|Beeson|N|15|12|1983|NORWAY||Carl.Beeson@rxALv9QUAqg.com|2452522| +3534|AAAAAAAAOMNAAAAA|912369|1243|10070|2450079|2450049|Dr.|Donald|Sebastian|N|20|6|1967|COMOROS||Donald.Sebastian@4OKi7TipUDFIA.org|2452639| +3535|AAAAAAAAPMNAAAAA|1701556|3467|42625|2450294|2450264|Sir|Billy|Wright|N|15|9|1944|DENMARK||Billy.Wright@ZrNVZQX.org|2452581| +3536|AAAAAAAAANNAAAAA|1711332|4729|6079|2451485|2451455|Mrs.|Janet|White|Y|23|4|1984|GREECE||Janet.White@muzfzVLU0NO9BzK.com|2452607| +3537|AAAAAAAABNNAAAAA|406057|5860|9216|2449395|2449365|Ms.|Theresa|Sawyer|N|12|1|1947|BURUNDI||Theresa.Sawyer@i.org|2452444| +3538|AAAAAAAACNNAAAAA|445171|4745|1472|2452196|2452166|Dr.|Beth|Harris|Y|27|2|1987|NETHERLANDS ANTILLES||Beth.Harris@8hUyd1xs.com|2452371| +3539|AAAAAAAADNNAAAAA|785101|6831|25361|2449698|2449668|Mr.|Matthew|Perry|N|3|10|1992|CAPE VERDE||Matthew.Perry@GJXgIzPLT6gyUDS1Q.com|2452519| +3540|AAAAAAAAENNAAAAA|275102|2624|11746|2450161|2450131|Sir|Gale|Bell|N|20|12|1942|SLOVENIA||Gale.Bell@fNHXLqnIjasHLa5.edu|2452566| +3541|AAAAAAAAFNNAAAAA|172917|3942|30920|2451710|2451680|Miss|Shawna|Medina|Y|29|11|1991|VANUATU||Shawna.Medina@2AU.com|2452484| +3542|AAAAAAAAGNNAAAAA|1182915|1084|21700|2449783|2449753|Mr.|Thomas|Turner|N|20|12|1933|CAMEROON||Thomas.Turner@jyIT8U6.com|2452480| +3543|AAAAAAAAHNNAAAAA|1589445|6510|10775|2451821|2451791|Mr.|Michael|Bailey|Y|17|7|1946|SURINAME||Michael.Bailey@qyHgBsNT.org|2452454| +3544|AAAAAAAAINNAAAAA|626532|3939|14146|2450809|2450779|Dr.|Ryan|White|N|22|4|1938|MYANMAR||Ryan.White@sv7ISaQY2.edu|2452405| +3545|AAAAAAAAJNNAAAAA|1130563|1684|22452|2451280|2451250|Miss|Jacqueline|Jones|Y|5|11|1969|ALGERIA||Jacqueline.Jones@auEb7rhUy.org|2452360| +3546|AAAAAAAAKNNAAAAA|231467|6613|46512|2450293|2450263|Miss|Mattie|Dean|Y|10|2|1947|VANUATU||Mattie.Dean@0M3B3MePBXP1EH3lQ.com|2452292| +3547|AAAAAAAALNNAAAAA|472847|572|14650|2451444|2451414|Sir|Joseph|Garcia|Y|12|10|1982|EQUATORIAL GUINEA||Joseph.Garcia@PFm9jqM8Qmmh.edu|2452357| +3548|AAAAAAAAMNNAAAAA|940585|173|30608|2449625|2449595|Miss|Marisa|Hansen|Y|28|9|1982|ITALY||Marisa.Hansen@Nhv.com|2452603| +3549|AAAAAAAANNNAAAAA|410745|3303|1108|2449689|2449659|Sir|Elmer|Henderson|N|21|2|1972|ISRAEL||Elmer.Henderson@Qm0K4Jk.edu|2452422| +3550|AAAAAAAAONNAAAAA|204717|3998|41279|2450451|2450421|Dr.|William|Withers|N|19|5|1972|BOTSWANA||William.Withers@YopkVn2EKjqmvmE5q.edu|2452365| +3551|AAAAAAAAPNNAAAAA|1808347|4288|26229|2451853|2451823|Dr.|Doyle|Johnson|Y|2|5|1938|CZECH REPUBLIC||Doyle.Johnson@z0AoANDIxLji.org|2452524| +3552|AAAAAAAAAONAAAAA|1891069|4148|15180|2451952|2451922|Mr.|Thomas|White|N|16|12|1936|TUNISIA||Thomas.White@e1s.com|2452435| +3553|AAAAAAAABONAAAAA|1037527|4950|45573|2452120|2452090|Dr.|Rodney|Hodges|N|6|6|1992|TRINIDAD AND TOBAGO||Rodney.Hodges@NqGxJPHM.org|2452284| +3554|AAAAAAAACONAAAAA|977704|4259|22014|2449527|2449497|Dr.|Stanley|Schultz|Y|16|11|1953|AUSTRIA||Stanley.Schultz@D0Qab7L.edu|2452412| +3555|AAAAAAAADONAAAAA|408383|4868|40153|2452468|2452438|Miss|Barbara|Gordon|Y|2|1|1990|SAUDI ARABIA||Barbara.Gordon@10Uz7o9.org|2452640| +3556|AAAAAAAAEONAAAAA|549693|4368|15843|2451372|2451342|Mr.|Bruce|Ratliff|Y|18|9|1977|FRANCE||Bruce.Ratliff@FDMnSp6ETps.org|2452296| +3557|AAAAAAAAFONAAAAA|458092|5682|1672|2450949|2450919|Dr.|Albert|Rush|N|2|11|1953|VENEZUELA||Albert.Rush@nLEN4HD7.org|2452520| +3558|AAAAAAAAGONAAAAA|1402774|4816|24058|2452652|2452622|Miss|Marilyn|Youngblood|Y|26|3|1935|MARTINIQUE||Marilyn.Youngblood@2U.org|2452314| +3559|AAAAAAAAHONAAAAA|612747|1284|39068|2449126|2449096|Sir|Carlos|Redmon|Y|12|12|1982|LUXEMBOURG||Carlos.Redmon@cA4UX.edu|2452307| +3560|AAAAAAAAIONAAAAA|538778|195|37212|2451589|2451559|Miss|Katelyn|Moody|Y|8|11|1967|CZECH REPUBLIC||Katelyn.Moody@o1tU7bFdoC9cyjr.org|2452384| +3561|AAAAAAAAJONAAAAA|1537423|4592|15361|2452004|2451974|Miss|Evelyn|Walsh|N|17|6|1976|VIRGIN ISLANDS, U.S.||Evelyn.Walsh@8AYJ.com|2452333| +3562|AAAAAAAAKONAAAAA|1364459|1412|3165|2451066|2451036|Mr.|Devin|Stonge|N|4|12|1939|ANTIGUA AND BARBUDA||Devin.Stonge@z6kFpv2.org|2452427| +3563|AAAAAAAALONAAAAA|496569|5627|23885|2452204|2452174|Dr.|Darrell|Sullivan|Y|9|11|1959|MONACO||Darrell.Sullivan@jdtIjPt7s.com|2452426| +3564|AAAAAAAAMONAAAAA|162929|4680|49184|||Miss|||Y|||1977|ESTONIA||Julie.Cooper@pFAm18J.org|| +3565|AAAAAAAANONAAAAA|1480055|543|39784|2451830|2451800|Sir|James|Marshall|Y|2|11|1947|DENMARK||James.Marshall@C6enzIHB1rJDZ.com|2452371| +3566|AAAAAAAAOONAAAAA|1484761|6811|32014|2449461|2449431|Miss|Vicenta|Valdez|Y|20|9|1935|BOTSWANA||Vicenta.Valdez@HoZL.com|2452554| +3567|AAAAAAAAPONAAAAA|1122772|5989|26006|2449211|2449181|Ms.|Donna|Smalls|N|20|2|1970|NEPAL||Donna.Smalls@xPp2OcDB7R7pSeu.edu|2452306| +3568|AAAAAAAAAPNAAAAA|650093|999|8658|2449379|2449349|Dr.|Ryan|Bowman|N|2|7|1984|EL SALVADOR||Ryan.Bowman@J6MByCAygFfcr.com|2452358| +3569|AAAAAAAABPNAAAAA|1622959|3926|2493|2451208|2451178|Mrs.|Amanda|Levy|Y|1|8|1932|TIMOR-LESTE||Amanda.Levy@OZHf.com|2452563| +3570|AAAAAAAACPNAAAAA|264620|5853|20733|2449832|2449802|Dr.|Angela|Haskins|N|21|3|1959|INDIA||Angela.Haskins@h0.org|2452364| +3571|AAAAAAAADPNAAAAA|263848|6527|26406|2449443|2449413|Sir|Ricardo|Yarbrough|N|17|1|1975|BOTSWANA||Ricardo.Yarbrough@AExFNcU6QmsxN6.org|2452400| +3572|AAAAAAAAEPNAAAAA|976092|4695|14945|2450036|2450006|Dr.|Linda|Brady|N|1|11|1964|SPAIN||Linda.Brady@7YQAnB4nAhzB.com|2452299| +3573|AAAAAAAAFPNAAAAA|1211819|1495|41971|2450570|2450540|Dr.|Jeannette|Nowak|N|17|12|1929|UNITED KINGDOM||Jeannette.Nowak@IRGvhUC629R.com|2452497| +3574|AAAAAAAAGPNAAAAA|93265|3487|25262|2449103|2449073|Miss|Ethel|Norfleet|N|6|12|1973|BULGARIA||Ethel.Norfleet@jNo4.edu|2452366| +3575|AAAAAAAAHPNAAAAA|1076826|922|5818|2451218|2451188|Sir|David|Williams|Y|8|4|1946|DJIBOUTI||David.Williams@kcHcNUNXcyaYpk.edu|2452490| +3576|AAAAAAAAIPNAAAAA|894187|4156|22572|2451252|2451222|Dr.|Ellen|Thomas|Y|16|10|1932|MOROCCO||Ellen.Thomas@gtA7xLHr6KMEQflLi.org|2452372| +3577|AAAAAAAAJPNAAAAA|1919044|1195|36946|2451822|2451792|Dr.|George|Davis|Y|15|12|1956|AUSTRALIA||George.Davis@8q5.com|2452311| +3578|AAAAAAAAKPNAAAAA|572720|3843|21719|2450414|2450384|Mrs.|Ella|Helms|Y|14|5|1975|BANGLADESH||Ella.Helms@u0o2HDa9Jh9.edu|2452549| +3579|AAAAAAAALPNAAAAA|1570003||4682|2451922|2451892|Sir|Ernest|||14||1936||||| +3580|AAAAAAAAMPNAAAAA|875676|3660|7644|2451309|2451279|Miss|Verna|Everett|Y|27|1|1942|ESTONIA||Verna.Everett@1lQ9lz5xyFeBGD.org|2452355| +3581|AAAAAAAANPNAAAAA|885197|6853|11309|2451261|2451231|Sir|Dave|Neal|N|29|10|1980|C�TE D'IVOIRE||Dave.Neal@1xKHTkG5UHq75VDI.edu|2452598| +3582|AAAAAAAAOPNAAAAA|830016|4709|31402|2449216|2449186|Ms.|Carolyn|Clark|N|6|12|1924|BULGARIA||Carolyn.Clark@NmcOld0QBa5x9DBK3.com|2452401| +3583|AAAAAAAAPPNAAAAA|625388|6804|33820|2449501|2449471|Dr.|Bruce|Keith|Y|25|4|1943|SERBIA||Bruce.Keith@giNzLkH5VKLR.org|2452589| +3584|AAAAAAAAAAOAAAAA|1137900|1747|18841|2452471|2452441|Dr.|Christopher|Ayers|Y|8|10|1956|BARBADOS||Christopher.Ayers@sfhVCt4O2SG.edu|2452437| +3585|AAAAAAAABAOAAAAA|1719294|421|33595|2452315|2452285|Mr.|William|Roberts|N|25|5|1983|LIECHTENSTEIN||William.Roberts@r7M.org|2452632| +3586|AAAAAAAACAOAAAAA|1917373|2804|10635|2450739|2450709|Mr.|Roy|Thompson|Y|13|3|1937|CAPE VERDE||Roy.Thompson@Ul.edu|2452351| +3587|AAAAAAAADAOAAAAA|1797639|6025|36011|2452620|2452590|Ms.|Allen|Gunter|Y|10|12|1988|PARAGUAY||Allen.Gunter@T.com|2452532| +3588|AAAAAAAAEAOAAAAA|1313223|3734|1045|2452562|2452532|Mr.|David|Mcmillan|N|16|8|1945|FRANCE||David.Mcmillan@PG3lp8D.edu|2452352| +3589|AAAAAAAAFAOAAAAA|1862529|4370|7458|2450354|2450324|Miss|Eleanor|Smith|N|1|3|1961|FRENCH GUIANA||Eleanor.Smith@e.com|2452534| +3590|AAAAAAAAGAOAAAAA|1588849|5172|12648|2451242|2451212|Dr.|Nicolas|White|N|13|6|1953|NETHERLANDS ANTILLES||Nicolas.White@NREiUJjPekgT7NLAKs.com|2452572| +3591|AAAAAAAAHAOAAAAA|1558398|6432|11713|2451940|2451910|Dr.|Caterina|Garza|Y|18|8|1980|SLOVAKIA||Caterina.Garza@doGIz4x0nQTOaJ.com|2452467| +3592|AAAAAAAAIAOAAAAA|817533||43583|2450839|2450809|Ms.|Deborah|Jackson|N|10|2|1946||||2452578| +3593|AAAAAAAAJAOAAAAA|1382131|3639|8347|2450215|2450185|Miss|Evelyn|Hays|N|4|5|1974|BERMUDA||Evelyn.Hays@k0J61javS6.org|2452336| +3594|AAAAAAAAKAOAAAAA|771641|5911|17828|2450993|2450963|Dr.|William|Bryant|N|21|4|1985|BELARUS||William.Bryant@E9ePA1Ok.com|2452596| +3595|AAAAAAAALAOAAAAA|1585216|4555|1094|2449250|2449220|Dr.|Rosemary|Brooks|Y|14|3|1968|TURKMENISTAN||Rosemary.Brooks@roFxy3tOX54hR.edu|2452290| +3596|AAAAAAAAMAOAAAAA|1741089|2844|22199|2451403|2451373|Dr.|Harriett|Pugh|Y|14|2|1982|SURINAME||Harriett.Pugh@PoDMQY.edu|2452457| +3597|AAAAAAAANAOAAAAA|990101|3195|13136|2449490|2449460|Ms.|Gabrielle|Roper|Y|4|8|1974|GREECE||Gabrielle.Roper@0tMHR2mjEmp6Q.org|2452555| +3598|AAAAAAAAOAOAAAAA|666384|4176|23968|2452228|2452198|Dr.|Micaela|Allan|N|25|3|1967|PHILIPPINES||Micaela.Allan@d2JvvChEIO.edu|2452637| +3599|AAAAAAAAPAOAAAAA|1785763|1681|45999|2450403|2450373|Mr.|Robert|Mitchell|N|13|10|1981|MOZAMBIQUE||Robert.Mitchell@9m5gEs6VuP.org|2452615| +3600|AAAAAAAAABOAAAAA|1182801|1452|1303|2449525|2449495|Sir|Stanley|Martin|Y|19|11|1941|VIRGIN ISLANDS, U.S.||Stanley.Martin@nYeprXzQhon1gccb.org|2452607| +3601|AAAAAAAABBOAAAAA|1118885|5255|26472|2451106|2451076|Miss|Margaret|Branch|N|1|4|1985|SLOVENIA||Margaret.Branch@gEoV0oetRI.org|2452291| +3602|AAAAAAAACBOAAAAA|||12020|2450289|2450259|Dr.|Brenda||Y||5||||Brenda.Sargent@1nLRZjZi1acG.edu|2452434| +3603|AAAAAAAADBOAAAAA|1388420|4351|12682|2452177|2452147|Mr.|Chester|Waller|Y|16|3|1985|MOZAMBIQUE||Chester.Waller@JOR09KNKaqn.org|2452559| +3604|AAAAAAAAEBOAAAAA|959630|5477|45324|2452136|2452106|Sir|James|Hinojosa|N|13|11|1981|INDONESIA||James.Hinojosa@nkGTkBI0AZvfK.com|2452343| +3605|AAAAAAAAFBOAAAAA|1224464|5972|20633|2449468|2449438|Dr.|Sean|Banks|Y|4|2|1979|BANGLADESH||Sean.Banks@7tU9k0jBpboOY9C.com|2452641| +3606|AAAAAAAAGBOAAAAA|1163800|2500|47600|2451843|2451813|Miss|Allene|Rodriguez|Y|25|10|1977|ANDORRA||Allene.Rodriguez@jXkphf8k6a9.edu|2452397| +3607|AAAAAAAAHBOAAAAA|165127|3775|8423|2452370|2452340|Mr.|Pablo|Adams|N|26|9|1962|ERITREA||Pablo.Adams@lfqPXs5oO3.org|2452469| +3608|AAAAAAAAIBOAAAAA|1487936|6507|11925|2452445|2452415|Mr.|Robert|Jenkins|N|8|9|1939|MOLDOVA, REPUBLIC OF||Robert.Jenkins@BzyNeOdorhA.org|2452440| +3609|AAAAAAAAJBOAAAAA|1779201|1691|43381|2449576|2449546|Sir|Eric|Quinones|Y|18|12|1925|CROATIA||Eric.Quinones@XX7N4dAjfSk.com|2452543| +3610|AAAAAAAAKBOAAAAA|1384157|885|27660|2450385|2450355|Sir|Greg|Jackson|N|24|8|1957|ARUBA||Greg.Jackson@5r1P0QB611vB1.com|2452401| +3611|AAAAAAAALBOAAAAA|1008776|3091|41406|2450076|2450046|Dr.|Minnie|Smith|N|19|6|1969|TIMOR-LESTE||Minnie.Smith@cLYU2.org|2452373| +3612|AAAAAAAAMBOAAAAA|69529|2754|47979|2451903|2451873|Mrs.|Tawana|Fowler|N|22|1|1989|NETHERLANDS||Tawana.Fowler@PP1k.com|2452619| +3613|AAAAAAAANBOAAAAA|1270750|1252|32507|2451637|2451607|Mrs.|Janice|Johnson|N|31|5|1981|KIRIBATI||Janice.Johnson@cL5BKg9sxY.com|2452431| +3614|AAAAAAAAOBOAAAAA|1854011|602|10793|2449674|2449644|Mr.|Brandon|Martin|Y|30|5|1955|AZERBAIJAN||Brandon.Martin@NDb9iH5c9Xm8IQ.com|2452550| +3615|AAAAAAAAPBOAAAAA|883904|7050|5349|2451455|2451425|Ms.|Judy|Gooding|N|24|6|1978|UNITED ARAB EMIRATES||Judy.Gooding@eqbxBnEb.org|2452572| +3616|AAAAAAAAACOAAAAA|1520378|3598|22818|2451244|2451214|Dr.|Marguerite|Oakley|N|25|9|1945|TOGO||Marguerite.Oakley@Zdso.org|2452318| +3617|AAAAAAAABCOAAAAA|1474483|6048|35966|2452293|2452263|Miss|Erika|Green|Y|7|2|1957|ICELAND||Erika.Green@ojXSSodChB.edu|2452526| +3618|AAAAAAAACCOAAAAA|932715|6333|23951|2451431|2451401|Dr.|Roy|Goodwin|N|9|5|1929|SWEDEN||Roy.Goodwin@9JS.edu|2452405| +3619|AAAAAAAADCOAAAAA|351918|5842|38966|2452504|2452474|Ms.|Dolores|Waggoner|N|11|3|1943|FINLAND||Dolores.Waggoner@NpIBL20G7QZ6T.com|2452549| +3620|AAAAAAAAECOAAAAA|149445|3334|44023|2452544|2452514|Mrs.|Bridgette|Camacho|Y|26|7|1952|TUVALU||Bridgette.Camacho@npSJ7VbBDZ.com|2452491| +3621|AAAAAAAAFCOAAAAA|823982|1287|36545|2450092|2450062|Mr.|Gary|Thomas|N|22|6|1974|BAHRAIN||Gary.Thomas@eNuxd6PxHH1RnLFpFi.org|2452330| +3622|AAAAAAAAGCOAAAAA|1789547|6285|23023|2449765|2449735|Mrs.|Alexis|Schmitt|N|19|2|1992|SAINT HELENA||Alexis.Schmitt@P.com|2452640| +3623|AAAAAAAAHCOAAAAA|972906|2124|12895|2450388|2450358|Mr.|Samuel|Levy|N|16|9|1951|PAPUA NEW GUINEA||Samuel.Levy@TY11XP.com|2452513| +3624|AAAAAAAAICOAAAAA|1322850|6789|29928|2451395|2451365|Sir|Gregory|Bartlett|Y|13|2|1980|MADAGASCAR||Gregory.Bartlett@dNDd88CoxbI.edu|2452369| +3625|AAAAAAAAJCOAAAAA|1004170|4343|4069|2452640|2452610|Mr.|Rickey|Ford|Y|16|4|1937|ESTONIA||Rickey.Ford@GgzV0SjdfDG.org|2452536| +3626|AAAAAAAAKCOAAAAA|755479|6947|38015|2452590|2452560|Sir|Robert|Thigpen|N|26|10|1966|NAMIBIA||Robert.Thigpen@CXCEh5GlIOS6.com|2452538| +3627|AAAAAAAALCOAAAAA|716637|2376|21762|2450110|2450080|Dr.|Lucille|Sanchez|Y|13|12|1945|RWANDA||Lucille.Sanchez@AgsCEPNpnUdg6lU36yzh.org|2452586| +3628|AAAAAAAAMCOAAAAA|1310513|2701|8842|2451618|2451588|Mr.|Robert|Billingsley|Y|4|6|1952|HAITI||Robert.Billingsley@YfSKk.edu|2452503| +3629|AAAAAAAANCOAAAAA|969321|3476|36431|2449234|2449204|Miss|Helen|Henry|Y|12|12|1937|PANAMA||Helen.Henry@XB.com|2452555| +3630|AAAAAAAAOCOAAAAA|256574|3138|28855|2449695|2449665|Miss|Karla|Obryan|N|12|11|1935|KAZAKHSTAN||Karla.Obryan@TrFuUTs0j9q9.com|2452399| +3631|AAAAAAAAPCOAAAAA|695040|352|39311|2450836|2450806|Dr.|Steven|Regalado|N|1|9|1988|NIGERIA||Steven.Regalado@HCC3.org|2452358| +3632|AAAAAAAAADOAAAAA|355290|120|16550|2449325|2449295|Dr.|Christian|Gallegos|Y|26|8|1986|MALAYSIA||Christian.Gallegos@qc.com|2452350| +3633|AAAAAAAABDOAAAAA|1084929|6638|11547|2452521|2452491|Mrs.|Cathy|Schmitt|N|18|5|1952|TRINIDAD AND TOBAGO||Cathy.Schmitt@NCVfgoQrd.org|2452449| +3634|AAAAAAAACDOAAAAA|605697|3040|13583|2450456|2450426|Sir|Steven|Mills|N|11|11|1928|BAHAMAS||Steven.Mills@48DTMK9OuF.com|2452534| +3635|AAAAAAAADDOAAAAA|331182|1947|49065|2451682|2451652|Dr.|Timothy|Price|N|21|9|1927|PAPUA NEW GUINEA||Timothy.Price@Z7IC.org|2452356| +3636|AAAAAAAAEDOAAAAA|1592097|6181|37865|2452670|2452640|Dr.||Finley|Y||||SUDAN|||| +3637|AAAAAAAAFDOAAAAA|366488|6859|10430|2452177|2452147|Dr.|Danny|Ireland|N|30|4|1957|BOTSWANA||Danny.Ireland@5UAr.com|2452607| +3638|AAAAAAAAGDOAAAAA|598007|4325|34643|2450563|2450533|Mr.|Martin|Taylor|Y|15|8|1967|VIET NAM||Martin.Taylor@s.org|2452299| +3639|AAAAAAAAHDOAAAAA|1754455|3667|5632|2451295|2451265|Miss|Lana|Robertson|N|9|10|1929|GABON||Lana.Robertson@42N28x4VkI9LI.com|2452601| +3640|AAAAAAAAIDOAAAAA|191965|3459|8553|2452162|2452132|Dr.|Gloria|Serrano|N|1|8|1984|MARSHALL ISLANDS||Gloria.Serrano@5qNIcdcA6tY.edu|2452610| +3641|AAAAAAAAJDOAAAAA|1425372|5999|45026|2449119|2449089|Miss|Ann|Nowak|N|26|12|1928|URUGUAY||Ann.Nowak@ieFgEsd.edu|2452597| +3642|AAAAAAAAKDOAAAAA|1665195|275|32955|2452158|2452128|Mr.|Kyle|Scully|N|21|8|1967|NAURU||Kyle.Scully@VSBv09zp.edu|2452630| +3643|AAAAAAAALDOAAAAA|1376417|3859|20148|2449215|2449185|Sir|Richard|Roberts|N|21|3|1929|NETHERLANDS||Richard.Roberts@l6B6yhotcFaEr.com|2452442| +3644|AAAAAAAAMDOAAAAA|75866|6755|9244|2452067|2452037|Sir|Matthew|Caron|Y|22|10|1984|GIBRALTAR||Matthew.Caron@Qfbu.org|2452410| +3645|AAAAAAAANDOAAAAA|1674345|1499|37853|2451977|2451947|Miss|Alexis|Graham|Y|19|2|1970|JERSEY||Alexis.Graham@ohL9odAYTZe.edu|2452317| +3646|AAAAAAAAODOAAAAA|854601|953|42058|2451509|2451479|Mr.|Justin|Quinn|Y|5|9|1972|EQUATORIAL GUINEA||Justin.Quinn@ddrRmbYuZd.org|2452469| +3647|AAAAAAAAPDOAAAAA|1017326|3892|43658|2451481|2451451|Miss|Wanda|Hatcher|N|13|12|1958|HAITI||Wanda.Hatcher@mgKMpNZDalK2.org|2452572| +3648|AAAAAAAAAEOAAAAA|1843145|1491|47437|2450011|2449981|Mr.|Martin|Graves|Y|7|3|1948|GREENLAND||Martin.Graves@ZNXORf2.org|2452300| +3649|AAAAAAAABEOAAAAA|1273082|998|40467|2451997|2451967|Mrs.|Carolyn|Wilson|N|3|7|1969|CAYMAN ISLANDS||Carolyn.Wilson@K.com|2452494| +3650|AAAAAAAACEOAAAAA|188263|2478|28693|2450476|2450446|Sir|David|Moultrie|N|10|4|1969|BELIZE||David.Moultrie@c9mD3OU3t9QxLPTs.com|2452398| +3651|AAAAAAAADEOAAAAA|1692270|3613|24909|2452274|2452244|Dr.|Travis|Forbes|N|3|6|1930|ITALY||Travis.Forbes@Q1MGzhfJj.com|2452439| +3652|AAAAAAAAEEOAAAAA|484388|2977|22766|2452635|2452605|Ms.|Deanna|Chappell|N|2|9|1932|EGYPT||Deanna.Chappell@cgHro2UF61MtsVP9kO.com|2452622| +3653|AAAAAAAAFEOAAAAA|1048242|2809|46929|2452116|2452086|Mr.|Sam|Morris|N|8|8|1963|ICELAND||Sam.Morris@r0bn8xT.org|2452374| +3654|AAAAAAAAGEOAAAAA|1011416|1553|14490|2451328|2451298|Miss|Paula|Sanchez|N|10|6|1974|SAUDI ARABIA||Paula.Sanchez@MGgjGKE1rEzGhL4Sy.com|2452534| +3655|AAAAAAAAHEOAAAAA|875317|2224|1131|2451026|2450996|Dr.|Douglas|Harmon|N|12|6|1955|LITHUANIA||Douglas.Harmon@gYK3mg5skbkdR.org|2452443| +3656|AAAAAAAAIEOAAAAA|698575|1271|48767|2452221|2452191|Ms.|Shanna|Martin|Y|2|11|1968|GAMBIA||Shanna.Martin@Ha0EdHrHn9Rfr.org|2452420| +3657|AAAAAAAAJEOAAAAA|1771451|1333|48815|2450621|2450591|Mr.|Thomas|Jones|N|1|10|1983|BERMUDA||Thomas.Jones@nTTKVpo0.com|2452405| +3658|AAAAAAAAKEOAAAAA|912053|4787|5168|2451062|2451032|Miss|Denise|Cloud|Y|28|8|1969|KUWAIT||Denise.Cloud@O.org|2452562| +3659|AAAAAAAALEOAAAAA|797253|3635|6864|2452245|2452215|Dr.|Donald|Porter|Y|11|5|1968|INDONESIA||Donald.Porter@GSPATUycx2q5Lda4L.edu|2452420| +3660|AAAAAAAAMEOAAAAA|1745187|546|4705|2451988|2451958|Miss|Nicole|Jones|N|25|5|1930|KIRIBATI||Nicole.Jones@EX0CuP.org|2452628| +3661|AAAAAAAANEOAAAAA|859265|1740|15415|2450938|2450908|Dr.|Lionel|Leonard|N|26|4|1947|MADAGASCAR||Lionel.Leonard@LPari4lp1U41z.org|2452570| +3662|AAAAAAAAOEOAAAAA|1122680|5639|17798|2450706|2450676|Dr.|Marylyn|Fitzpatrick|Y|25|5|1981|SWITZERLAND||Marylyn.Fitzpatrick@1XqELtdsl.org|2452312| +3663|AAAAAAAAPEOAAAAA|80594|3240|30733|2451464|2451434|Dr.|Dania|Boyce|Y|16|9|1981|TRINIDAD AND TOBAGO||Dania.Boyce@RrJspIrZPH.com|2452336| +3664|AAAAAAAAAFOAAAAA|1069450|3612|6174|2450034|2450004|Sir|Manuel|Williams|N|14|10|1945|IRAQ||Manuel.Williams@yszhNVD0.edu|2452601| +3665|AAAAAAAABFOAAAAA|816178|6730|16082|2449343|2449313|Mr.|James|Elmore|Y|5|8|1990|AUSTRIA||James.Elmore@qoYESQT7b.com|2452391| +3666|AAAAAAAACFOAAAAA|307611|132|39453|2450267|2450237|Miss|Edith|Gray|Y|13|10|1963|BULGARIA||Edith.Gray@cLZZfucJEs.com|2452621| +3667|AAAAAAAADFOAAAAA|1850189||49844|2452305||Miss||||16|5||ITALY|||2452444| +3668|AAAAAAAAEFOAAAAA|147364|1660|20756|2451801|2451771|Ms.|Lillian|Covington|Y|12|6|1981|CUBA||Lillian.Covington@dvZp4.com|2452311| +3669|AAAAAAAAFFOAAAAA|6888|5622|33556|2452629|2452599|Mr.|Mark|Miller|Y|25|3|1941|KENYA||Mark.Miller@vOZ0HJi1GcKosX.org|2452577| +3670|AAAAAAAAGFOAAAAA|1263976|1243|18157|2450928|2450898|Dr.|Ethel|Riddle|Y|29|7|1937|SWEDEN||Ethel.Riddle@SksoGQj5FKXz5xN.edu|2452454| +3671|AAAAAAAAHFOAAAAA|2012|1430|47648|2452303|2452273|Miss|Betty|Stewart|Y|31|7|1943|FRENCH GUIANA||Betty.Stewart@jeKP.com|2452635| +3672|AAAAAAAAIFOAAAAA|1895502||31127||||Lori|||||1950|||Lori.Harris@gvJKCAEEfgOUQvtbJ.org|| +3673|AAAAAAAAJFOAAAAA|619816|1778|25740|2452102|2452072|Dr.|Essie|Durden|N|2|8|1938|TURKMENISTAN||Essie.Durden@bPK1gIPAIbNEEUzq.edu|2452571| +3674|AAAAAAAAKFOAAAAA|710121|595|12716|2449292|2449262|Mr.|Alan|Best|N|10|5|1991|AZERBAIJAN||Alan.Best@tYM5rctBevth858Ivj.com|2452286| +3675|AAAAAAAALFOAAAAA|493514|2957|8353|2449324|2449294|Sir|John|Chavez|Y|12|10|1970|UNITED ARAB EMIRATES||John.Chavez@HYpqkZ3M8gLdQ.edu|2452424| +3676|AAAAAAAAMFOAAAAA|93070|3272|12674|2449984|2449954|Mrs.|Barbara|Ramirez|N|23|8|1929|BRUNEI DARUSSALAM||Barbara.Ramirez@eFv6y.edu|2452423| +3677|AAAAAAAANFOAAAAA|1351338|2014|42643|2451190|2451160|Mr.|William|Wolford|N|27|5|1929|EL SALVADOR||William.Wolford@Pv.com|2452296| +3678|AAAAAAAAOFOAAAAA|1138260|100|38340|2449902|2449872|Dr.|Lisa|Lee|N|4|9|1966|PARAGUAY||Lisa.Lee@0.edu|2452414| +3679|AAAAAAAAPFOAAAAA|1733600|346|47092|2450942|2450912|Mrs.|Sandra|Bivins|N|10|12|1928|BERMUDA||Sandra.Bivins@jXdajP79SlLnLj.edu|2452557| +3680|AAAAAAAAAGOAAAAA|445349|2986|21023|2450855|2450825|Sir|Lynn|Hanson|N|13|8|1965|GAMBIA||Lynn.Hanson@rSVLnZt83uJc33JP1.org|2452367| +3681|AAAAAAAABGOAAAAA|1850891|5940|19389|2452300|2452270|Mr.|William|Khan|N|9|11|1946|SLOVAKIA||William.Khan@5l.com|2452335| +3682|AAAAAAAACGOAAAAA|1091884|2900|25069|2450510|2450480|Miss|Ethel|Keller|Y|24|7|1932|SAINT LUCIA||Ethel.Keller@VifNCivIlx7iq.org|2452639| +3683|AAAAAAAADGOAAAAA|1716874|3345|39416|2449278|2449248|Sir|Lazaro|Johnson|Y|24|11|1950|GHANA||Lazaro.Johnson@6IbtVznGC5F6.edu|2452349| +3684|AAAAAAAAEGOAAAAA|332852|5196|1987|2450072|2450042|Miss|Stephanie|Hayes|N|24|9|1979|HAITI||Stephanie.Hayes@9c.org|2452408| +3685|AAAAAAAAFGOAAAAA|1210259|2560|21261|2449285|2449255|Dr.|Milton|Hill|Y|5|6|1960|RWANDA||Milton.Hill@LBb3DciCp2hk.org|2452527| +3686|AAAAAAAAGGOAAAAA|861437|6130|14823|2451914|2451884|Miss|Rachel|Burns|N|19|7|1968|CHILE||Rachel.Burns@6EKVzynqu.org|2452601| +3687|AAAAAAAAHGOAAAAA|1546581|4548|26115|2451338|2451308|Mr.|Mark|Walsh|N|27|2|1928|SEYCHELLES||Mark.Walsh@81vA20gXJTY7h.edu|2452585| +3688|AAAAAAAAIGOAAAAA|252638|6025|21012|2449459|2449429|Sir|Michael|Green|N|24|4|1991|PHILIPPINES||Michael.Green@8SLNTJ7.org|2452623| +3689|AAAAAAAAJGOAAAAA|480594|2413|33032|2450075|2450045|Mrs.|Ruth|Morales|N|12|8|1937|PHILIPPINES||Ruth.Morales@a4b6gFMQagG.com|2452461| +3690|AAAAAAAAKGOAAAAA|332957|5483|24009|2451918|2451888|Ms.|Tina|Watson|Y|15|6|1983|JAPAN||Tina.Watson@hemc.edu|2452596| +3691|AAAAAAAALGOAAAAA|1887537|5431|34972|2450370|2450340|Ms.|Jacqueline|Simons|Y|25|12|1936|QATAR||Jacqueline.Simons@1biOIzlBS.org|2452605| +3692|AAAAAAAAMGOAAAAA|559309|3734|41406|2451607|2451577|Mrs.|Ema|Wright|N|10|5|1932|NIGER||Ema.Wright@3v.com|2452476| +3693|AAAAAAAANGOAAAAA||108|26907|2450927||Dr.|Donald|||||1952||||2452547| +3694|AAAAAAAAOGOAAAAA|276834|7118|39849|2451301|2451271|Miss|Margaret|Carson|N|6|8|1948|BRAZIL||Margaret.Carson@LpQI5bY3D0CnOD.org|2452467| +3695|AAAAAAAAPGOAAAAA|873778|3194|39057|2451578|2451548|Dr.|Albert|Parent|N|28|7|1992|MADAGASCAR||Albert.Parent@EOEyCBqfPp.edu|2452390| +3696|AAAAAAAAAHOAAAAA|393608|5595|31935|2451910|2451880|Dr.|Yvonne|Lewis|N|7|4|1988|MOZAMBIQUE||Yvonne.Lewis@Bt6pSbvj0E.org|2452310| +3697|AAAAAAAABHOAAAAA|1571239|3890|37694|2449991|2449961|Dr.|Lillian|Sanderson|Y|1|6|1943|FINLAND||Lillian.Sanderson@4k.edu|2452475| +3698|AAAAAAAACHOAAAAA|1716143|4218|1493|2449272|2449242|Dr.|Joseph|Castillo|Y|30|11|1949|NORFOLK ISLAND||Joseph.Castillo@GCFhFj6.edu|2452364| +3699|AAAAAAAADHOAAAAA|1178097|1422|2300|2452148|2452118|Miss|Belinda|Powell|Y|3|1|1931|GREENLAND||Belinda.Powell@xi5pLLYgUXxdpQV.edu|2452609| +3700|AAAAAAAAEHOAAAAA|14347|4294|38241|2451641|2451611|Mrs.|Laura|Chen|Y|7|11|1952|ALGERIA||Laura.Chen@BJlKF1nOC134gOCcXM.com|2452518| +3701|AAAAAAAAFHOAAAAA|127314|4443|17525|2449452|2449422|Sir|Jonathan|Earle|Y|16|10|1966|UKRAINE||Jonathan.Earle@17xc.edu|2452397| +3702|AAAAAAAAGHOAAAAA|657132|5016|4453|2451449|2451419|Ms.|Jocelyn|Brennan|N|22|12|1973|QATAR||Jocelyn.Brennan@PiJ072U4dmmCz7i.edu|2452317| +3703|AAAAAAAAHHOAAAAA|219068|2856|31044|2452210|2452180|Mrs.|Linda|Rangel|Y|14|2|1953|WESTERN SAHARA||Linda.Rangel@VtLLBXvb2cLUUM.com|2452381| +3704|AAAAAAAAIHOAAAAA|1496823|4256|5289|2449639|2449609|Mr.|Sam|Cunningham|N|4|8|1990|TAJIKISTAN||Sam.Cunningham@vF78H.edu|2452639| +3705|AAAAAAAAJHOAAAAA|1686525|5003|28234|2449333|2449303|Miss|Lisa|Sanchez|Y|13|11|1991|CAPE VERDE||Lisa.Sanchez@dZsshBG37pdY5Y4n.com|2452537| +3706|AAAAAAAAKHOAAAAA|1695359|356|19706|2449066|2449036|Mrs.|Darlene|Gilbertson|Y|25|7|1968|CHILE||Darlene.Gilbertson@8.org|2452292| +3707|AAAAAAAALHOAAAAA|547021|6223|27025|2452009|2451979|Miss|Rebecca|Stinson|N|29|9|1947|BENIN||Rebecca.Stinson@U9RlbrA5k1H.com|2452404| +3708|AAAAAAAAMHOAAAAA|641731|4543|34602|2452531|2452501|Sir|Ronald|Stephenson|N|21|8|1965|SOUTH AFRICA||Ronald.Stephenson@1ZptBnCfFCmB7ektJv.com|2452301| +3709|AAAAAAAANHOAAAAA|507982|3881|17380|2451570|2451540|Dr.|Dorothy|Shapiro|Y|24|11|1960|PANAMA||Dorothy.Shapiro@y66.edu|2452529| +3710|AAAAAAAAOHOAAAAA|388524|6523|10922|2449940|2449910|Mr.|Kristopher|Henry|N|12|4|1924|BURUNDI||Kristopher.Henry@bvEDMGrVX9fZzl.edu|2452364| +3711|AAAAAAAAPHOAAAAA|1159080|5033|21339|2450860|2450830|Miss|Latasha|Walker|N|30|12|1936|GIBRALTAR||Latasha.Walker@vqSbFvycHby6qi.com|2452311| +3712|AAAAAAAAAIOAAAAA|185073|1701|15882|2450588|2450558|Dr.|Linda|Soliz|N|12|12|1967|LITHUANIA||Linda.Soliz@5Oame.com|2452305| +3713|AAAAAAAABIOAAAAA|812983|5108|19186|2452669|2452639|Sir|Joseph|Lang|N|26|9|1960|NICARAGUA||Joseph.Lang@ei1h21AbAp8xa2V.org|2452522| +3714|AAAAAAAACIOAAAAA|1885683|5007|46331|2449416|2449386|Ms.|Susana|Coleman|Y|3|9|1971|TOGO||Susana.Coleman@nQG6pqX.org|2452297| +3715|AAAAAAAADIOAAAAA|689847|2194|46094|2452275|2452245|Mrs.|Althea|Ayala|Y|24|6|1971|JERSEY||Althea.Ayala@fcN4aKJkXAHSL.org|2452535| +3716|AAAAAAAAEIOAAAAA|129166|5619|11319|2452263|2452233|Mrs.|Millie|Bennett|Y|25|4|1959|BAHRAIN||Millie.Bennett@8aq.org|2452578| +3717|AAAAAAAAFIOAAAAA|787194|6746|19098|2451406|2451376|Mr.|Frank|Nguyen|Y|12|11|1959|GAMBIA||Frank.Nguyen@uE3.edu|2452569| +3718|AAAAAAAAGIOAAAAA|1866964|1498|16432|2451235|2451205|Miss|Mariana|Dinkins|Y|8|12|1944|MEXICO||Mariana.Dinkins@s5E31RLkjF.org|2452463| +3719|AAAAAAAAHIOAAAAA|276606|5701|40796|2450939|2450909|Mrs.|Barbara|Hunter|Y|22|2|1977|LIECHTENSTEIN||Barbara.Hunter@MMf54bxO40y1.edu|2452359| +3720|AAAAAAAAIIOAAAAA|322240|2470|7053|2451999|2451969|Mrs.|Jessica|Castillo|Y|21|10|1954|HUNGARY||Jessica.Castillo@Az2IQFpnf6I2O58q.com|2452292| +3721|AAAAAAAAJIOAAAAA|776821|6489|46557|2452252|2452222|Mr.|Stephen|Olson|N|27|9|1927|GERMANY||Stephen.Olson@8aC1Rfuj7yInc.org|2452405| +3722|AAAAAAAAKIOAAAAA|198929|6380|35478|2449487|2449457|Mr.|Jose|Smith|N|6|2|1930|BRAZIL||Jose.Smith@P1XUj1R0BFRT5v5.edu|2452324| +3723|AAAAAAAALIOAAAAA|1770826|4053|43233|2451376|2451346|Ms.|Carolyn|Stubbs|N|7|9|1944|HAITI||Carolyn.Stubbs@afKx0dcslXsnb.com|2452319| +3724|AAAAAAAAMIOAAAAA|404703|5984|19618|2452046|2452016|Miss|Andrew|Kirkland|N|4|7|1960|KENYA||Andrew.Kirkland@GGIXJTbsYmLAcCHZX54.org|2452502| +3725|AAAAAAAANIOAAAAA|258199|5185|44524|2450493|2450463|Dr.|David|Smith|Y|22|7|1949|HONDURAS||David.Smith@nxcdr2hzMHNdBhB8e.org|2452598| +3726|AAAAAAAAOIOAAAAA|603196|2355|33914|2452626|2452596|Dr.|Sam|Helm|Y|8|10|1989|ANDORRA||Sam.Helm@BqkOeSbcQtya.com|2452348| +3727|AAAAAAAAPIOAAAAA|1566264|3859|40607|2452236|2452206|Miss|Judith|Jordan|Y|27|7|1958|SWAZILAND||Judith.Jordan@IJhLC6cgJqscTlI.edu|2452411| +3728|AAAAAAAAAJOAAAAA|1569161|2512|37063|2449907|2449877|Dr.|Mattie|Stanley|Y|9|7|1928|MAURITIUS||Mattie.Stanley@0tPym2HaVug.org|2452542| +3729|AAAAAAAABJOAAAAA|1132972|4440|13108|2451262|2451232|Miss|Bonnie|Fuentes|Y|16|3|1956|SRI LANKA||Bonnie.Fuentes@hHtLJD.org|2452444| +3730|AAAAAAAACJOAAAAA|337050|539|41628|2449194|2449164|Mrs.|Teresa|Kiser|Y|15|1|1962|GRENADA||Teresa.Kiser@nfI01AQ27r.com|2452370| +3731|AAAAAAAADJOAAAAA|1246427|1819|48811|2452438|2452408|Dr.|Edwin|Thomas|N|12|6|1944|CANADA||Edwin.Thomas@PyavBO.com|2452520| +3732|AAAAAAAAEJOAAAAA|56903|3998|41446|2449370|2449340|Ms.|Heather|King|Y|14|7|1930|HUNGARY||Heather.King@eFkLGf.org|2452318| +3733|AAAAAAAAFJOAAAAA|362067|7012|18404|2451255||Dr.||||||||||2452345| +3734|AAAAAAAAGJOAAAAA|679886|402|30871|2451893|2451863|Ms.|Jaclyn|Stein|Y|27|10|1938|ECUADOR||Jaclyn.Stein@V8.org|2452356| +3735|AAAAAAAAHJOAAAAA|1526493|222|29951|2449990|2449960|Mr.|David|Mcintosh|Y|23|2|1987|BELIZE||David.Mcintosh@qi1TIDifMZ.edu|2452298| +3736|AAAAAAAAIJOAAAAA|1138183|4767|7304|2450711|2450681|Dr.|Johnny|Sullivan|N|29|7|1945|LATVIA||Johnny.Sullivan@BjOnbj.org|2452410| +3737|AAAAAAAAJJOAAAAA|555794|6498|26002|2449552|2449522|Sir|Daniel|Cummings|N|26|8|1926|COMOROS||Daniel.Cummings@kRgtQoSnBDmD.edu|2452362| +3738|AAAAAAAAKJOAAAAA|1084487|6889|21738|2451688|2451658|Dr.|Kerry|Jolley|Y|16|1|1944|TIMOR-LESTE||Kerry.Jolley@Emg.com|2452341| +3739|AAAAAAAALJOAAAAA|1908649|2080|12311|2450879|2450849|Mr.|John|Searcy|Y|9|5|1948|COMOROS||John.Searcy@losaNZ963hK25.org|2452574| +3740|AAAAAAAAMJOAAAAA|1790457|4516|3775|2451712|2451682|Miss|Isabelle|Seaman|Y|3|3|1974|UZBEKISTAN||Isabelle.Seaman@cS0Bc2MVjKl7s.org|2452318| +3741|AAAAAAAANJOAAAAA|430570|6687|34004|2449033|2449003|Dr.|Opal|Moore|N|17|4|1951|PHILIPPINES||Opal.Moore@HhCCMetTTS.edu|2452492| +3742|AAAAAAAAOJOAAAAA|1638076|6503|83|2452663|2452633|Dr.|Steve|Lorenzo|N|25|7|1930|SINGAPORE||Steve.Lorenzo@iFoCJjYnv6.org|2452310| +3743|AAAAAAAAPJOAAAAA|942340|131|35922|2452425|2452395|Sir|Dallas|Bolling|Y|9|1|1929|LUXEMBOURG||Dallas.Bolling@rBoYi2e.edu|2452294| +3744|AAAAAAAAAKOAAAAA|1465477|3183|47642|2449258|2449228|Sir|William|Anthony|N|24|6|1959|HONG KONG||William.Anthony@VcxAnLesJacV3dhQp.com|2452481| +3745|AAAAAAAABKOAAAAA|1432373|5137|21148|2451841|2451811|Mrs.|Miriam|Lindsey|N|28|8|1951|HUNGARY||Miriam.Lindsey@tzgKyF6jtAxL.edu|2452431| +3746|AAAAAAAACKOAAAAA|1818300|6374|32495|2451084|2451054|Sir|John|Lin|N|3|4|1964|MOZAMBIQUE||John.Lin@QEMyK9VqnHmdFdQx.edu|2452316| +3747|AAAAAAAADKOAAAAA||4274|22054|2450560|2450530|Sir|||Y||8|1939|HAITI||David.Larson@58.com|| +3748|AAAAAAAAEKOAAAAA|1192779|6946|32666|2449778|2449748|Dr.|Michael|Haskell|N|14|5|1933|NICARAGUA||Michael.Haskell@3RQYrSOPHadRCYOM1.edu|2452391| +3749|AAAAAAAAFKOAAAAA|576394|5772|40397|2449536|2449506|Ms.|Veronica|Carlisle|N|24|4|1972|NAURU||Veronica.Carlisle@CtZ.org|2452402| +3750|AAAAAAAAGKOAAAAA|340149|4670|41944|2451988|2451958|Sir|Al|Howard|N|9|2|1925|NORWAY||Al.Howard@THCQDHO6n6gMa.edu|2452549| +3751|AAAAAAAAHKOAAAAA|1273752|3781|33810|2449754|2449724|Mr.|Mark|Allen|N|6|4|1966|SURINAME||Mark.Allen@BqHeVG3tpTF7.edu|2452488| +3752|AAAAAAAAIKOAAAAA|188379|5182|42785|2451323|2451293|Dr.|Joseph|Lawton|Y|10|3|1977|NORFOLK ISLAND||Joseph.Lawton@nmpkOuOZjF.com|2452561| +3753|AAAAAAAAJKOAAAAA|1091465|7107|7149|2451971|2451941|Mrs.|Consuelo|Hudson|Y|25|5|1969|TONGA||Consuelo.Hudson@5noJq9C109.org|2452337| +3754|AAAAAAAAKKOAAAAA|1159207|5897|12987|2449254|2449224|Sir|Clarence|Robinson|Y|10|9|1979|ISRAEL||Clarence.Robinson@bSS5LQJt.org|2452353| +3755|AAAAAAAALKOAAAAA|358662|3310|7958|2452094|2452064|Miss|Adele|Bullock|Y|22|11|1934|MOLDOVA, REPUBLIC OF||Adele.Bullock@zTrfd5QP0.edu|2452568| +3756|AAAAAAAAMKOAAAAA|1624727|16|45752|2449829|2449799|Mr.|John|Mcgee|N|17|8|1934|SAINT LUCIA||John.Mcgee@6ed7U7uAHIUX.com|2452583| +3757|AAAAAAAANKOAAAAA|876060|2633|759|2450439|2450409|Sir|Shawn|Willis|Y|5|2|1977|NEPAL||Shawn.Willis@XTeTfeSSxD6kh.com|2452559| +3758|AAAAAAAAOKOAAAAA|1510040|1558|28276|2452056|2452026|Mr.|Robert|Locke|Y|27|1|1960|SAUDI ARABIA||Robert.Locke@Jgj.org|2452605| +3759|AAAAAAAAPKOAAAAA|994322|2424|13373|2450865|2450835|Sir|Jonathan|Crawford|Y|6|7|1980|TURKMENISTAN||Jonathan.Crawford@be0tAdGEoz.org|2452292| +3760|AAAAAAAAALOAAAAA|1829225|1795|41565|2451343|2451313|Mr.|Christopher|Jordan|N|8|12|1941|TURKEY||Christopher.Jordan@7bQmQ.com|2452337| +3761|AAAAAAAABLOAAAAA|1573075|150|34472|2452032|2452002|Dr.|Wanda|Evans|N|16|3|1945|NETHERLANDS ANTILLES||Wanda.Evans@Hk.org|2452451| +3762|AAAAAAAACLOAAAAA|1367246|774|3455|2451908|2451878|Dr.|Donna|Rome|N|11|12|1966|TUVALU||Donna.Rome@iPx.edu|2452395| +3763|AAAAAAAADLOAAAAA|593693|1223|26753|2450205|2450175|Mr.|Dong|Gordon|N|20|7|1946|UGANDA||Dong.Gordon@A7pXxnLloy.org|2452524| +3764|AAAAAAAAELOAAAAA|681694|1020|37593|2450274|2450244|Ms.|Judith|Williams|Y|19|10|1952|HUNGARY||Judith.Williams@4GBNnj4rXT8U.com|2452536| +3765|AAAAAAAAFLOAAAAA|728489|1267|4892|2452549|2452519|Dr.|Ashely|Christian|Y|13|11|1988|SWAZILAND||Ashely.Christian@N.com|2452395| +3766|AAAAAAAAGLOAAAAA|130169|3358|29347|2450090|2450060|Sir|Federico|Shelton|N|29|12|1935|BANGLADESH||Federico.Shelton@QgSmRRDy3.org|2452643| +3767|AAAAAAAAHLOAAAAA|769575|2992|17609|2450275|2450245|Ms.|Eleanor|Williams|Y|11|11|1974|UGANDA||Eleanor.Williams@yyo.org|2452397| +3768|AAAAAAAAILOAAAAA|1775200|1644|24088|2449297|2449267|Ms.|Anthony|Sandoval|Y|21|3|1930|UGANDA||Anthony.Sandoval@DcTyDH27AVm.edu|2452499| +3769|AAAAAAAAJLOAAAAA|923065|4336|20007||2451506|Ms.||||23|1|1957|SAN MARINO|||2452508| +3770|AAAAAAAAKLOAAAAA|335888|5902|44316|2450042|2450012|Dr.|John|Lugo|N|8|9|1930|LESOTHO||John.Lugo@aMQYU3clPQK4.org|2452568| +3771|AAAAAAAALLOAAAAA|786871|5446|47974|2452627|2452597|Sir|James|Turner|N|14|6|1937|CAMEROON||James.Turner@BdAxxs8XISa1.edu|2452325| +3772|AAAAAAAAMLOAAAAA|1645141|6646|46504|2450443|2450413|Mr.|Stanley|Davis|Y|4|9|1960|TAJIKISTAN||Stanley.Davis@bvzTzb292q.org|2452408| +3773|AAAAAAAANLOAAAAA|1790010|3826|9642|2451520|2451490|Dr.|Keith|Yoder|N|1|11|1992|EQUATORIAL GUINEA||Keith.Yoder@0Xa23JSe.com|2452581| +3774|AAAAAAAAOLOAAAAA|534396|6879|38562|2449274|2449244|Miss|Maxine|Bennett|Y|26|4|1984|CAPE VERDE||Maxine.Bennett@y9y8jbOFpqsPf3fTI2b.edu|2452362| +3775|AAAAAAAAPLOAAAAA|1372433|33|43633|2450374|2450344|Sir|Robert|Fort|N|6|6|1972|ECUADOR||Robert.Fort@FqQQ78JTGAXmn.com|2452540| +3776|AAAAAAAAAMOAAAAA|1823430|5850|39470|2451138|2451108|Dr.|Francisco|Reed|Y|4|6|1953|CZECH REPUBLIC||Francisco.Reed@BbTVlBPoeYPti.com|2452592| +3777|AAAAAAAABMOAAAAA|793561|1201|24088|2451420|2451390|Mrs.|Rita|Helton|N|13|4|1966|LIBERIA||Rita.Helton@IJDV9.com|2452370| +3778|AAAAAAAACMOAAAAA|1704269|2991|2066|2450987|2450957|Ms.|Ella|Sandoval|Y|17|3|1965|SURINAME||Ella.Sandoval@YyIngVV8.org|2452642| +3779|AAAAAAAADMOAAAAA|882095|1029|1516|2452596|2452566|Dr.|Kathryn|Johnson|N|18|5|1980|TUNISIA||Kathryn.Johnson@a.org|2452551| +3780|AAAAAAAAEMOAAAAA|1234518|3709|32900|2451356|2451326|Sir|Tommy|Swartz|N|22|6|1992|SOMALIA||Tommy.Swartz@Zl8Vs.org|2452318| +3781|AAAAAAAAFMOAAAAA|1187735|4075|35168|2450476|2450446|Dr.|Juan|Pennington|N|15|6|1930|SURINAME||Juan.Pennington@AaquH.edu|2452376| +3782|AAAAAAAAGMOAAAAA|1240176|5792|5455|2449049|2449019|Mr.|Marshall|Miller|N|26|10|1961|MYANMAR||Marshall.Miller@Iyu1VAmOe.com|2452578| +3783|AAAAAAAAHMOAAAAA|1617615|3670|32078|2450656|2450626|Mrs.|Jeanne|Sexton|Y|17|12|1959|SWITZERLAND||Jeanne.Sexton@IMHkYV.org|2452627| +3784|AAAAAAAAIMOAAAAA|206707||26555|2451006|||||Y||7|1955|||Teresa.Clark@gmdfARx9OAvDrmTgIM.org|2452626| +3785|AAAAAAAAJMOAAAAA|398653|2731|14056|2451957|2451927|Mrs.|Dorothy|Lawson|N|23|5|1955|BERMUDA||Dorothy.Lawson@XDX.edu|2452370| +3786|AAAAAAAAKMOAAAAA|24800|822|37197|2450967|2450937|Mr.|Robert|Stricklin|Y|18|4|1992|EL SALVADOR||Robert.Stricklin@91PRlvCAYIiQbf.com|2452371| +3787|AAAAAAAALMOAAAAA|229898|6594|46890|2452598|2452568|Miss|Luz|Overby|Y|3|6|1945|GUERNSEY||Luz.Overby@pNzn.com|2452607| +3788|AAAAAAAAMMOAAAAA|483458|2428|48951|2452079|2452049|Dr.|Leroy|Coleman|Y|27|5|1985|CANADA||Leroy.Coleman@nzi83.edu|2452430| +3789|AAAAAAAANMOAAAAA|1279935|6203|20518|2449705|2449675|Mrs.|Martha|Adams|N|28|1|1928|SERBIA||Martha.Adams@tEqSQs.org|2452560| +3790|AAAAAAAAOMOAAAAA||6754|39911|2450931|2450901|Dr.|Joe|Mcnair||||1987|NORWAY||Joe.Mcnair@iECxFdLI0VjNgg.edu|| +3791|AAAAAAAAPMOAAAAA|136325|6843|17761|2451982|2451952|Mr.|Stephen|Wright|N|3|10|1956|NETHERLANDS ANTILLES||Stephen.Wright@1omSTuIXT6aLa5V.edu|2452582| +3792|AAAAAAAAANOAAAAA|317742|3870|46845|2451215|2451185|Dr.|Paula|Ruth|N|2|10|1943|BRUNEI DARUSSALAM||Paula.Ruth@o3n5N62.com|2452462| +3793|AAAAAAAABNOAAAAA|847245|5824|41556|2452549|2452519|Mr.|Jason|Wilcox|N|24|2|1926|TIMOR-LESTE||Jason.Wilcox@2eNv.org|2452294| +3794|AAAAAAAACNOAAAAA|845775|7132|38842|2451957|2451927|Sir|Daniel|Hopson|N|4|4|1964|FRENCH GUIANA||Daniel.Hopson@0a8UKP03ORj.com|2452349| +3795|AAAAAAAADNOAAAAA|1496024|5978|47932|2452609|2452579|Mrs.|Andrea|Liles|N|21|10|1926|UKRAINE||Andrea.Liles@pDm.org|2452390| +3796|AAAAAAAAENOAAAAA|1489842|3204|5766|2451460|2451430|Ms.|Antonia|Link|Y|4|5|1973|MONACO||Antonia.Link@pmnRq7c08klyfs.org|2452617| +3797|AAAAAAAAFNOAAAAA|3463|6647|25431|2450850|2450820|Miss|Kathleen|Norman|Y|5|12|1939|ANTIGUA AND BARBUDA||Kathleen.Norman@C.edu|2452313| +3798|AAAAAAAAGNOAAAAA|481635|6437|46792|2452459|2452429|Sir|Alvin|Grimes|Y|3|10|1930|BENIN||Alvin.Grimes@TTvSih.edu|2452506| +3799|AAAAAAAAHNOAAAAA|1718000|1256|22465|2449218|2449188|Ms.|Jill|Ramirez|Y|3|10|1947|TOKELAU||Jill.Ramirez@ElH1TdD9G3.edu|2452555| +3800|AAAAAAAAINOAAAAA|1370705|1904|48750|2450294|2450264|Mrs.|Betty|Vogel|N|6|10|1953|HONDURAS||Betty.Vogel@19ye47GJ.com|2452340| +3801|AAAAAAAAJNOAAAAA|1388467|6025|46383|2449742|2449712|Ms.|Marta|Stephens|Y|20|9|1953|GABON||Marta.Stephens@fjMiGDt.com|2452412| +3802|AAAAAAAAKNOAAAAA|79430|2626|13775|2451411|2451381|Mr.|Alfredo|Berry|Y|12|3|1965|GREECE||Alfredo.Berry@AIhphsZ0SO.org|2452515| +3803|AAAAAAAALNOAAAAA|1627645|7150|27008|2451756|2451726|Miss|Georgia|Huffman|Y|22|3|1981|MAURITIUS||Georgia.Huffman@iVR1B.org|2452402| +3804|AAAAAAAAMNOAAAAA|564725|3765|14805|2452005|2451975|Dr.|Betty|Rees|N|16|9|1954|GUATEMALA||Betty.Rees@p9js32O.org|2452559| +3805|AAAAAAAANNOAAAAA|594739|2487|235|2451427|2451397|Sir|Jose|Crockett|N|15|10|1947|ANGUILLA||Jose.Crockett@HSC.com|2452358| +3806|AAAAAAAAONOAAAAA|1409878|3322|3119|2450869|2450839|Mr.|Paul|Dean|Y|28|1|1933|GERMANY||Paul.Dean@ihshUAx.org|2452417| +3807|AAAAAAAAPNOAAAAA|204113|723|34133|2450261|2450231|Dr.|Leslie|Maxwell|Y|17|12|1929|BHUTAN||Leslie.Maxwell@c42LS2CRkm.com|2452599| +3808|AAAAAAAAAOOAAAAA|1240478|5405|25085|2449538|2449508|Sir|Arturo|Dang|N|12|10|1976|HONG KONG||Arturo.Dang@9r2QmHcG0vKZo.edu|2452608| +3809|AAAAAAAABOOAAAAA|1860905|2772|20992|2450990|2450960|Dr.|Mack|Sherry|N|28|5|1969|LIBERIA||Mack.Sherry@aOa9gab62O.com|2452432| +3810|AAAAAAAACOOAAAAA|321626|4591|16709|2450011|2449981|Dr.|Thelma|Benoit|Y|10|3|1979|HUNGARY||Thelma.Benoit@LvOHJ.edu|2452403| +3811|AAAAAAAADOOAAAAA|1870322|3354|18833|2450942|2450912|Ms.|Mabel|Salinas|N|30|7|1949|ALGERIA||Mabel.Salinas@jL3nUsvtgO.org|2452585| +3812|AAAAAAAAEOOAAAAA|1597951|5830|12109|2452012|2451982|Dr.|Mark|Marsh|Y|2|1|1965|NEW CALEDONIA||Mark.Marsh@SCLN7cXoSnpfy.edu|2452447| +3813|AAAAAAAAFOOAAAAA|1456173|2468|31516|2452026|2451996|Ms.|Wanda|Byrd|Y|16|2|1954|ITALY||Wanda.Byrd@2LRVJ37j8Ozkq.org|2452470| +3814|AAAAAAAAGOOAAAAA|1887646|5662|6878|2449720|2449690|Dr.|John|Williams|Y|10|12|1968|SAINT LUCIA||John.Williams@30cILMMCxBkmL.com|2452491| +3815|AAAAAAAAHOOAAAAA||4754|36512||2451113||Mui|Joyner||||1957|||Mui.Joyner@r9aDmKjvVHs9HjXB.org|| +3816|AAAAAAAAIOOAAAAA|1180431|5844|29657|2450080|2450050|Dr.|Bob|Mendoza|N|1|10|1942|ALGERIA||Bob.Mendoza@bB8u5XIbgVF9.edu|2452441| +3817|AAAAAAAAJOOAAAAA|1682330|889|8695|2450449|2450419|Miss|Molly|Hampton|Y|19|3|1948|CROATIA||Molly.Hampton@Pd7RxokRBc.edu|2452467| +3818|AAAAAAAAKOOAAAAA|366952|6749|7793|2449087|2449057|Dr.|Angie|Emerson|Y|28|6|1945|ANTIGUA AND BARBUDA||Angie.Emerson@Vf7zAlxQPJN4xO.org|2452436| +3819|AAAAAAAALOOAAAAA|||37316|2449725|||||||9||ERITREA||Patrick.Powers@dcqUr.edu|2452436| +3820|AAAAAAAAMOOAAAAA|1478240|3399|18958|2451458|2451428|Mrs.|Rebecca|Martin|N|25|3|1934|WESTERN SAHARA||Rebecca.Martin@ZYYr5mP0EqOVSv.org|2452591| +3821|AAAAAAAANOOAAAAA|1230608|6985|49362|2451473||||||||1957|||Colin.Dubois@9.com|2452320| +3822|AAAAAAAAOOOAAAAA|1008747|4690|37480|2451102|2451072|Sir|Bryan|Holguin|Y|17|2|1964|JAPAN||Bryan.Holguin@TEQop.com|2452503| +3823|AAAAAAAAPOOAAAAA|1869930|2176|4375|2451667|2451637|Mr.|Fred|Johnson|N|25|3|1986|ECUADOR||Fred.Johnson@zG9Q.com|2452386| +3824|AAAAAAAAAPOAAAAA|480671|5570|25669|2451237|2451207|Mrs.|Johanna|Winslow|N|15|5|1924|TAJIKISTAN||Johanna.Winslow@ONbhSyXoz.org|2452283| +3825|AAAAAAAABPOAAAAA|857520|2339|29158|2450688|2450658|Miss|Pamela|Bartley|N|2|3|1937|PERU||Pamela.Bartley@tUCmOo2v.com|2452284| +3826|AAAAAAAACPOAAAAA|1585347|5083|46066|2449175|2449145|Ms.|Danielle|Crawford|Y|20|6|1963|ECUADOR||Danielle.Crawford@7S.edu|2452298| +3827|AAAAAAAADPOAAAAA|1330452|5810|35968|2452067|2452037|Dr.|Debra|Bedford|N|7|1|1937|EQUATORIAL GUINEA||Debra.Bedford@pTZqAQOg.edu|2452590| +3828|AAAAAAAAEPOAAAAA|819891|7153|768|2449677|2449647|Sir|Jared|Nelson|Y|17|5|1987|HAITI||Jared.Nelson@8G28gF7tusmPAvz2N5.com|2452443| +3829|AAAAAAAAFPOAAAAA|1637814|139|12911|2449950|2449920|Mrs.|Ona|Guzman|N|3|2|1951|TUNISIA||Ona.Guzman@fE9.edu|2452314| +3830|AAAAAAAAGPOAAAAA|1801012|6069|48726|2450843|2450813|Mr.|Emery|Johnson|N|4|7|1958|CUBA||Emery.Johnson@cBYudlOBQ.com|2452558| +3831|AAAAAAAAHPOAAAAA|1178057|1868|6526|2451038|2451008|Ms.|Laurie|Andrews|N|15|4|1947|KENYA||Laurie.Andrews@X6FeEGhu.com|2452338| +3832|AAAAAAAAIPOAAAAA|1040878|7058|43238|2451874|2451844|Sir|Herbert|Morse|Y|15|12|1963|AMERICAN SAMOA||Herbert.Morse@u9oo1r0iNoAi.edu|2452595| +3833|AAAAAAAAJPOAAAAA|1366627|6670|31560|2451861|2451831|Sir|Shelby|Harris|N|14|5|1980|PANAMA||Shelby.Harris@u.org|2452353| +3834|AAAAAAAAKPOAAAAA|1858362|6103|10597|2449536|2449506|Ms.|Terri|Martinez|Y|21|7|1955|ERITREA||Terri.Martinez@SvLC.edu|2452368| +3835|AAAAAAAALPOAAAAA|440267|7028|27836|2449397|2449367|Mrs.|Fannie|Bratton|N|21|2|1961|ARUBA||Fannie.Bratton@ldPr.edu|2452541| +3836|AAAAAAAAMPOAAAAA|1384212|1155|98|2449889|2449859|Sir|Stephen|Tyner|N|15|4|1984|TURKMENISTAN||Stephen.Tyner@DNotnZFKEtjlpK.com|2452482| +3837|AAAAAAAANPOAAAAA|1511978|1913|19848|2450052|2450022|Sir|Steven|Phillips|N|26|5|1926|KIRIBATI||Steven.Phillips@xax8TTb.edu|2452533| +3838|AAAAAAAAOPOAAAAA|388203|2432|25061|2452076|2452046|Mr.|Shannon|Briggs|Y|3|1|1928|ARMENIA||Shannon.Briggs@cCuIZ9pvrTc.edu|2452581| +3839|AAAAAAAAPPOAAAAA|477987|2248|4109|2451448|2451418|Mrs.|Marjorie|Turner|N|23|6|1959|BELGIUM||Marjorie.Turner@rLKBg09QJS.org|2452611| +3840|AAAAAAAAAAPAAAAA|270358|2779|25647|2449210|2449180|Mrs.|Donna|Williams|N|29|11|1982|FRENCH GUIANA||Donna.Williams@vIlpraq2Nmlp.org|2452416| +3841|AAAAAAAABAPAAAAA|9130|1574|1618|2451390|2451360|Sir|George|Binkley|Y|13|11|1971|BHUTAN||George.Binkley@YShkh4pAC92.org|2452285| +3842|AAAAAAAACAPAAAAA|314691|5620|2902|2452054|2452024|Mrs.|Regina|Powers|Y|7|2|1943|DENMARK||Regina.Powers@6gBG0zOIROEkAsp.com|2452546| +3843|AAAAAAAADAPAAAAA|1161406|3011|26663|2451184|2451154|Mr.|Gregory|Lyon|N|13|8|1987|SINGAPORE||Gregory.Lyon@gjojNo0bFoskozYcf4.edu|2452478| +3844|AAAAAAAAEAPAAAAA|1125857|6354|41204|2449944|2449914|Dr.|Carlos|Noland|N|28|11|1956|MALAWI||Carlos.Noland@0MfhlA6VNN7VGtJh.edu|2452577| +3845|AAAAAAAAFAPAAAAA|208406|3912|4618|2452198|2452168|Dr.|David|Belanger|Y|26|11|1989|SOUTH AFRICA||David.Belanger@xbVZntEtSyoTl8.com|2452561| +3846|AAAAAAAAGAPAAAAA|159268|1063|28811|2451753|2451723|Miss|Kristin|Ebert|Y|15|12|1965|GHANA||Kristin.Ebert@zkhOD2vT0Gs.com|2452340| +3847|AAAAAAAAHAPAAAAA|1294404|5113|21775|2450848|2450818|Ms.|Claudia|Keenan|N|11|8|1966|BARBADOS||Claudia.Keenan@cN2LE2IUUDaZ1rEG.org|2452549| +3848|AAAAAAAAIAPAAAAA|847022|1644|39466|2449299|2449269|Sir|John|Pierce|N|5|7|1925|LITHUANIA||John.Pierce@ITrDVjzQTRyHibLt.org|2452382| +3849|AAAAAAAAJAPAAAAA|1610744|6179|22014|2452296|2452266|Dr.|Stella|Mcdaniel|N|17|5|1970|SOMALIA||Stella.Mcdaniel@ckICPAqoE62kzEi4A.org|2452547| +3850|AAAAAAAAKAPAAAAA|1859042|5655|49115|2451696|2451666|Dr.|Maurice|Sorensen|Y|10|9|1989|JAMAICA||Maurice.Sorensen@L.com|2452373| +3851|AAAAAAAALAPAAAAA|48924|1038|11152|2452569|2452539|Mrs.|Wanda|Easter|N|4|10|1926|HUNGARY||Wanda.Easter@zH.com|2452572| +3852|AAAAAAAAMAPAAAAA|895073|7179|27685|2450164|2450134|Mrs.|Janice|Quick|Y|12|11|1925|VIET NAM||Janice.Quick@kimVdEiUgi.org|2452590| +3853|AAAAAAAANAPAAAAA|1828172|6781|29743|2450875|2450845|Dr.|Jose|Vega|N|27|6|1991|NICARAGUA||Jose.Vega@a6mXQ5N2L.com|2452370| +3854|AAAAAAAAOAPAAAAA|213917|3746|6422|2449205|2449175|Mr.|George|Torres|Y|15|10|1931|BURUNDI||George.Torres@hH6htKD2HBh0R.edu|2452409| +3855|AAAAAAAAPAPAAAAA|925228|1648|47315|2452279|2452249|Miss|Susan|Pruett|Y|24|7|1982|MALAYSIA||Susan.Pruett@kSE.edu|2452470| +3856|AAAAAAAAABPAAAAA|105006|5403|35000|2450398|2450368|Dr.|Sandra|Burks|Y|5|4|1946|PARAGUAY||Sandra.Burks@Uq1ByklHiyLe.com|2452349| +3857|AAAAAAAABBPAAAAA|1520104||29352||||||||7|1925|GREECE|||2452473| +3858|AAAAAAAACBPAAAAA|478340|1205|31717|2451803|2451773|Dr.|Kristin|Wood|Y|1|9|1992|CHRISTMAS ISLAND||Kristin.Wood@TDPrvOizM.edu|2452458| +3859|AAAAAAAADBPAAAAA|1794125|3941|11404|2450834|2450804|Sir|Brett|Walton|N|23|10|1970|BHUTAN||Brett.Walton@TCAGH.edu|2452631| +3860|AAAAAAAAEBPAAAAA|84029|2626|24776|2451368|2451338|Sir|James|Leslie|N|15|1|1950|HONDURAS||James.Leslie@mY.com|2452426| +3861|AAAAAAAAFBPAAAAA|1231150|2820|42090|2451853|2451823|Dr.|William|Walker|N|19|2|1949|BOTSWANA||William.Walker@FuqoTNUNBsnELcRYM.org|2452550| +3862|AAAAAAAAGBPAAAAA|1153870|5725|15933|2449932|2449902|Mr.|Harold|Roberge|Y|16|11|1970|UZBEKISTAN||Harold.Roberge@Snoj1TVUy3.org|2452568| +3863|AAAAAAAAHBPAAAAA|1582088|1326|26164|2451586|2451556|Sir|Michael|Ventura|N|7|1|1938|SAINT HELENA||Michael.Ventura@gkfU3cSv.edu|2452593| +3864|AAAAAAAAIBPAAAAA|939557|2637|36917|2452190|2452160|Sir|Donald|Nutter|N|17|9|1986|JORDAN||Donald.Nutter@rgQGstU4iGelO07.edu|2452602| +3865|AAAAAAAAJBPAAAAA|1253692|6983|34674|2450562|2450532|Miss|Carla|Sharkey|Y|12|9|1963|CHILE||Carla.Sharkey@Fl.com|2452568| +3866|AAAAAAAAKBPAAAAA||4227|3932|2452421||||||2||1980|||Andrew.Ward@gFjNS.com|| +3867|AAAAAAAALBPAAAAA|1135492|5100|17908|2449864|2449834|Ms.|Aaron|Rosado|N|22|6|1979|ETHIOPIA||Aaron.Rosado@E.org|2452442| +3868|AAAAAAAAMBPAAAAA|1214391|2104|49464|2451319|2451289|Dr.|Shawn|Young|N|16|1|1945|MYANMAR||Shawn.Young@8Yeoov8IGieB.org|2452569| +3869|AAAAAAAANBPAAAAA|1320609|5238|45867|2450969|2450939|Mrs.|Effie|Harris|N|27|4|1937|GIBRALTAR||Effie.Harris@MlxgEKZJKp.org|2452587| +3870|AAAAAAAAOBPAAAAA|483635|1168|33901|2450088|2450058|Dr.|Norman|Oneal|N|1|1|1944|ARMENIA||Norman.Oneal@SRZzUJ1tO7.edu|2452535| +3871|AAAAAAAAPBPAAAAA|1748424|656|9335|2452286|2452256|Dr.|Ralph|Mcdonald|N|31|12|1931|THAILAND||Ralph.Mcdonald@cfOK6P.org|2452562| +3872|AAAAAAAAACPAAAAA|72343|5225|17754|2451941|2451911|Ms.|Ella|Johnson|Y|8|2|1984|TUVALU||Ella.Johnson@s2f4McqcQFF3sO.org|2452508| +3873|AAAAAAAABCPAAAAA|12664|5578|34045|2450979|2450949|Ms.|Velma|Mcdonald|Y|1|11|1983|BRAZIL||Velma.Mcdonald@fr1DFcZTg3kn.com|2452501| +3874|AAAAAAAACCPAAAAA|1364563|470|34777|2450894|2450864|Mr.|William|Byrd|N|15|10|1961|SAUDI ARABIA||William.Byrd@fjO43JehScBhUq.edu|2452459| +3875|AAAAAAAADCPAAAAA|272961|5768|35328|2450886|2450856|Mr.|Randy|Schmidt|N|5|10|1959|MALAWI||Randy.Schmidt@hMJXC0A8GXNvuM.edu|2452514| +3876|AAAAAAAAECPAAAAA|1552193|3806|9419|2452524|2452494|Dr.|Kareem|Arellano|N|14|1|1968|MEXICO||Kareem.Arellano@CXRILtJ7Sd6y2CjS9.org|2452533| +3877|AAAAAAAAFCPAAAAA|549690|5157|49024|2451270|2451240|Ms.|Barbra|Shepherd|N|3|11|1965|GREECE||Barbra.Shepherd@ld0lA.edu|2452448| +3878|AAAAAAAAGCPAAAAA|592786|691|34149|2451721|2451691|Miss|Sibyl|Bray|Y|24|3|1973|TOKELAU||Sibyl.Bray@Ay41cd8E9F66.edu|2452521| +3879|AAAAAAAAHCPAAAAA|1915793|695|31997|2452362|2452332|Sir|Brian|Sacco|N|5|11|1967|UNITED ARAB EMIRATES||Brian.Sacco@xBV3CbVDL2n.org|2452306| +3880|AAAAAAAAICPAAAAA|1730642|1633|29424|2449962|2449932|Mr.|Dion|Nelson|N|21|4|1989|INDONESIA||Dion.Nelson@VlKcVrdJK7.org|2452318| +3881|AAAAAAAAJCPAAAAA|192238|6702|22813|2452087|2452057|Sir|Emil|Good|Y|4|1|1960|EQUATORIAL GUINEA||Emil.Good@HSNUkgP.edu|2452630| +3882|AAAAAAAAKCPAAAAA|1004426|1354|16235|2451856|2451826|Mr.|James|Davis|Y|27|10|1958|GUERNSEY||James.Davis@A9c5sgBAa.org|2452576| +3883|AAAAAAAALCPAAAAA|844411|5668|44125|2451931|2451901|Mr.|Daniel|Collier|N|16|5|1934|VENEZUELA||Daniel.Collier@XF1rFOlXb31kD.com|2452626| +3884|AAAAAAAAMCPAAAAA|200370|6687|9451|2450453|2450423|Dr.|Gerald|West|Y|5|4|1970|MARSHALL ISLANDS||Gerald.West@OdjmnHEJ1uG3.com|2452407| +3885|AAAAAAAANCPAAAAA|1135646|2139|18632|2450924|2450894|Dr.|Denise|Gooden|N|17|11|1962|ALGERIA||Denise.Gooden@57gOqTQd.org|2452624| +3886|AAAAAAAAOCPAAAAA|913623|862|36454|2450884|2450854|Sir|Rodney|Wilson|Y|28|3|1931|MYANMAR||Rodney.Wilson@0OedrfgRUQ1.org|2452617| +3887|AAAAAAAAPCPAAAAA|175662|3643|45224|2449979|2449949|Mrs.|Shirleen|Hawkins|N|11|4|1953|SENEGAL||Shirleen.Hawkins@aiyI.com|2452453| +3888|AAAAAAAAADPAAAAA|1589901|2335|21169|2449900|2449870|Sir|Joseph|Mills|Y|26|11|1965|BOTSWANA||Joseph.Mills@RjoLVg2XeC.edu|2452439| +3889|AAAAAAAABDPAAAAA|397277|2810|37143|2450151|2450121|Mr.|Benjamin|Brown|Y|6|9|1970|GUYANA||Benjamin.Brown@dc7ua4Z.org|2452319| +3890|AAAAAAAACDPAAAAA|753677|5448|1972|2450309|2450279|Sir|Roland|Barrett|Y|12|5|1969|SAINT LUCIA||Roland.Barrett@9LhY8Y.edu|2452434| +3891|AAAAAAAADDPAAAAA|7508|1780|4899|2449625|2449595|Ms.|Gloria|Duncan|Y|17|1|1977|KUWAIT||Gloria.Duncan@oiS1KB.com|2452382| +3892|AAAAAAAAEDPAAAAA|195084|1745|41206|2449131|2449101|Miss|Phyllis|Medina|Y|24|12|1941|GEORGIA||Phyllis.Medina@H6bjT.com|2452614| +3893|AAAAAAAAFDPAAAAA|1585676|5122|30121|2450595|2450565|Ms.|Edyth|Collins|Y|2|7|1925|GRENADA||Edyth.Collins@o.com|2452405| +3894|AAAAAAAAGDPAAAAA|1102087|1641|41995|2450031|2450001|Sir|James|Larue|Y|21|10|1980|ISRAEL||James.Larue@gmuFGdYsg8yQQ.com|2452624| +3895|AAAAAAAAHDPAAAAA|1772133|3044|49555|2451051|2451021|Mrs.|Alex|Morrell|Y|14|8|1986|MALTA||Alex.Morrell@D1l3bQaCmYhuvrm.edu|2452585| +3896|AAAAAAAAIDPAAAAA|1536342|3295|1048|2451319|2451289|Sir|Mark|Mcgehee|Y|18|3|1955|UZBEKISTAN||Mark.Mcgehee@L6oVPzBZA5yg.com|2452614| +3897|AAAAAAAAJDPAAAAA|643692|2806|35120|2450262|2450232|Miss|Pauline|Bradford|N|6|8|1937|MALAWI||Pauline.Bradford@N3upzmxqaHivgjl.edu|2452390| +3898|AAAAAAAAKDPAAAAA|1266782|5585|1188|2451079|2451049|Mrs.|Melina|Shelby|Y|18|8|1980|FIJI||Melina.Shelby@EcF7rijOlAKMh.org|2452524| +3899|AAAAAAAALDPAAAAA|1555049|3256|4663|2451751|2451721|Mr.|Steven|Mcghee|N|10|8|1963|BOTSWANA||Steven.Mcghee@Tplr77.org|2452639| +3900|AAAAAAAAMDPAAAAA|798151|3307|26016|2450301|2450271|Mr.|Neil|Garcia|Y|17|1|1950|ARUBA||Neil.Garcia@fcQOxjCvbtIZMPpBsb.edu|2452365| +3901|AAAAAAAANDPAAAAA|1596829|1117|2158|2452312|2452282|Ms.|Melissa|Hill|Y|16|5|1955|MALAYSIA||Melissa.Hill@tI.edu|2452540| +3902|AAAAAAAAODPAAAAA|196914|1742|21770|2450352|2450322|Ms.|Shari|Mclaughlin|N|12|5|1939|NORFOLK ISLAND||Shari.Mclaughlin@nE3ZVOg.org|2452317| +3903|AAAAAAAAPDPAAAAA|952959|2791|47870|2450419|2450389|Ms.|Nora|Pulido|Y|3|9|1971|MALI||Nora.Pulido@RnSu8Ayq4s.org|2452560| +3904|AAAAAAAAAEPAAAAA|696482|6622|37609|2451514|2451484|Dr.|Susan|Ralston|Y|21|1|1968|GUADELOUPE||Susan.Ralston@nH8h.edu|2452309| +3905|AAAAAAAABEPAAAAA|1558906|4936|28613|2451097|2451067|Dr.|Mildred|Williams|N|3|11|1962|KENYA||Mildred.Williams@trTI6l7rGuM.com|2452538| +3906|AAAAAAAACEPAAAAA|1895891|5850|10209|2450389|2450359|Mrs.|Stacie|Vogel|N|16|9|1953|SRI LANKA||Stacie.Vogel@5Eaj39y9x5e.edu|2452625| +3907|AAAAAAAADEPAAAAA|1111978|6917|37009|2451627|2451597|Ms.|Donna|Perez|N|14|1|1941|FAROE ISLANDS||Donna.Perez@Z8NQKoxt1xxtb.edu|2452409| +3908|AAAAAAAAEEPAAAAA|1255108|45|38182|2452409|2452379|Dr.|Frank|Morris|Y|23|2|1938|LIECHTENSTEIN||Frank.Morris@r9fJalSshK.org|2452357| +3909|AAAAAAAAFEPAAAAA|484633|1004|10175|2450603|2450573|Mr.|Daniel|Shaw|N|13|9|1977|ANGUILLA||Daniel.Shaw@CugQQIQlaTH7MYXVhxG.com|2452631| +3910|AAAAAAAAGEPAAAAA|79943|2290|25837|2450774|2450744|Sir|Ray|Davis|N|7|3|1980|DENMARK||Ray.Davis@E11gsrb1.org|2452391| +3911|AAAAAAAAHEPAAAAA|1053264|1028|34745|2451187|2451157|Miss|Hannah|Rose|N|10|7|1933|GABON||Hannah.Rose@q.org|2452305| +3912|AAAAAAAAIEPAAAAA|430664|4020|17023|2451216|2451186|Dr.|David|Gonzalez|Y|16|2|1961|SOLOMON ISLANDS||David.Gonzalez@lZB3MuO2nfFYgfzl.org|2452390| +3913|AAAAAAAAJEPAAAAA|465404|7179|11354|2450475|2450445|Dr.|Betty|Mccoy|N|16|1|1941|TRINIDAD AND TOBAGO||Betty.Mccoy@dJyb0q88xH.org|2452639| +3914|AAAAAAAAKEPAAAAA|401692|4274|16903|2451286|2451256|Sir|John|Thomas|Y|12|7|1935|UZBEKISTAN||John.Thomas@4r.edu|2452515| +3915|AAAAAAAALEPAAAAA|31303|6064|1578|2450229|2450199|Mr.|Tommie|Madrid|Y|5|1|1962|SAMOA||Tommie.Madrid@nE44.edu|2452481| +3916|AAAAAAAAMEPAAAAA|1750972|1988|9851|2449783|2449753|Dr.|James|Byrd|Y|15|7|1953|PANAMA||James.Byrd@fFGLSZtg.com|2452406| +3917|AAAAAAAANEPAAAAA|632310|3356|41155|2450305|2450275|Dr.|Ryan|Campbell|N|29|11|1938|TIMOR-LESTE||Ryan.Campbell@UsBgHKK8cxD.edu|2452473| +3918|AAAAAAAAOEPAAAAA|144617|2931|29245|2451394|2451364|Miss|Misty|Brenner|N|9|2|1983|BELGIUM||Misty.Brenner@r.edu|2452613| +3919|AAAAAAAAPEPAAAAA|1307029|2430|18984|2449823|2449793|Dr.|Jerry|Larkin|Y|2|1|1952|CHILE||Jerry.Larkin@8.com|2452386| +3920|AAAAAAAAAFPAAAAA|1598939|1514|10042|2449882|2449852|Mr.|John|Clark|Y|31|12|1944|DENMARK||John.Clark@hLgJjDq96Ae.edu|2452393| +3921|AAAAAAAABFPAAAAA|340640|6516|43545|2452121|2452091|Mr.|Kevin|Bacon|Y|7|3|1933|JAPAN||Kevin.Bacon@N8zkxYj.org|2452489| +3922|AAAAAAAACFPAAAAA|1068681|1849|617|2449910|2449880|Dr.|Shawn|Burton|Y|12|7|1978|TURKMENISTAN||Shawn.Burton@xS7pKB6UJ9eHlZ.com|2452306| +3923|AAAAAAAADFPAAAAA|857301|4886|2824|2449196|2449166|Mr.|David|Joyner|Y|28|12|1983|LIECHTENSTEIN||David.Joyner@cTsfrk9ZfoJG.com|2452636| +3924|AAAAAAAAEFPAAAAA|1391152|1404|47303|2450942|2450912|Miss|Charlene|Gonzales|N|10|4|1975|MONACO||Charlene.Gonzales@bQPF.edu|2452530| +3925|AAAAAAAAFFPAAAAA|384129|6155|8134|2452166|2452136|Ms.|Bonnie|Sweeney|N|17|5|1974|MYANMAR||Bonnie.Sweeney@8lAobXf0mP.edu|2452331| +3926|AAAAAAAAGFPAAAAA|1448542|6987|42679|2449449|2449419|Sir|Zachary|Jones|N|10|4|1976|PAKISTAN||Zachary.Jones@q18yxtSfekIh5kq.org|2452420| +3927|AAAAAAAAHFPAAAAA|1429178|7005|40432|2451688|2451658|Dr.|Jacqueline|Williams|N|23|5|1988|TONGA||Jacqueline.Williams@I.com|2452363| +3928|AAAAAAAAIFPAAAAA|||6591|||Miss|||Y|20||||||2452509| +3929|AAAAAAAAJFPAAAAA|1741831|3866|35901|2449133|2449103|Dr.|Nicole|Bailey|N|19|9|1992|GUINEA-BISSAU||Nicole.Bailey@BUMC4T.edu|2452646| +3930|AAAAAAAAKFPAAAAA|616013|5391|5569|2449458|2449428|Ms.|Bridget|Briseno|N|23|1|1982|SAMOA||Bridget.Briseno@FEvNZXy19d.edu|2452507| +3931|AAAAAAAALFPAAAAA|1200312|4578|11244|2450153|2450123|Sir|Kenneth|Pratt|N|27|8|1965|NICARAGUA||Kenneth.Pratt@y9qi.com|2452430| +3932|AAAAAAAAMFPAAAAA|1285216|6701|22186|2449960|2449930|Mrs.|Debra|Hughes|N|13|6|1939|ECUADOR||Debra.Hughes@G.edu|2452383| +3933|AAAAAAAANFPAAAAA|827894|5588|40835|2449558|2449528|Dr.|Curtis|Crane|N|19|12|1982|BOUVET ISLAND||Curtis.Crane@vYHaQC1s.org|2452421| +3934|AAAAAAAAOFPAAAAA|604745|3295|31798|2452604|2452574|Sir|Thomas|Walls|N|6|5|1977|JAPAN||Thomas.Walls@gBRoRUi6KPdQ.org|2452592| +3935|AAAAAAAAPFPAAAAA|294593|597|31448|2449768|2449738|Mrs.|Angela|Brito|Y|7|5|1927|SOUTH AFRICA||Angela.Brito@EIpmGa.com|2452301| +3936|AAAAAAAAAGPAAAAA|770983|6525|1267|2452561|2452531|Ms.|Tara|Nguyen|N|1|8|1940|JERSEY||Tara.Nguyen@BbyQe.com|2452554| +3937|AAAAAAAABGPAAAAA|824118|4136|2954|2452053|2452023|Ms.|Tammy|Carr|N|21|2|1926|LIBERIA||Tammy.Carr@IykrxL4p4U9.org|2452491| +3938|AAAAAAAACGPAAAAA|814717|5013|14509|2451621|2451591|Mr.|Hugh|Sims|N|16|6|1982|VANUATU||Hugh.Sims@kuBZn.edu|2452338| +3939|AAAAAAAADGPAAAAA|998628|300|24372|2450299|2450269|Miss|Sandra|Francis|N|26|2|1971|BARBADOS||Sandra.Francis@EPnMhdPp7Z.edu|2452607| +3940|AAAAAAAAEGPAAAAA|117254|888|26969|2451233|2451203|Dr.|Richard|Orton|Y|8|11|1967|VANUATU||Richard.Orton@27xs0vYt9Msv5ic.org|2452470| +3941|AAAAAAAAFGPAAAAA|846793|239|37685|2450280|2450250|Mr.|Christopher|Campbell|Y|13|1|1969|GUINEA-BISSAU||Christopher.Campbell@Cd4AsSNiRsU3uXY.com|2452427| +3942|AAAAAAAAGGPAAAAA|1014024|278|35732|2450125|2450095|Sir|Kenneth|Burdette|Y|18|8|1984|BRAZIL||Kenneth.Burdette@Jr5lsBI51R.edu|2452575| +3943|AAAAAAAAHGPAAAAA|1893198|4030|15|2451087|2451057|Dr.|David|Randall|N|25|4|1939|COMOROS||David.Randall@OYzTzP18JdICD2.edu|2452551| +3944|AAAAAAAAIGPAAAAA|56761|2110|49412|2450883|2450853|Mr.|Jeffery|Wells|N|21|7|1966|NEW CALEDONIA||Jeffery.Wells@3mCDYNs.edu|2452292| +3945|AAAAAAAAJGPAAAAA|631894|3659|13871|2449739|2449709|Ms.|Theresa|Wallace|N|16|4|1964|ERITREA||Theresa.Wallace@fSa.com|2452554| +3946|AAAAAAAAKGPAAAAA|677318|7121|17324|2449336|2449306|Ms.|April|Turner|N|14|12|1955|BENIN||April.Turner@udt949EntrsA.org|2452549| +3947|AAAAAAAALGPAAAAA|713464|2733|29602|2451547|2451517|Sir|Patrick|Coburn|Y|15|2|1985|JAPAN||Patrick.Coburn@dC50FD68C7VAJ110.org|2452405| +3948|AAAAAAAAMGPAAAAA|622666|913|9513|2449866|2449836|Sir|Joe|Williams|N|3|8|1964|ECUADOR||Joe.Williams@kNdYpy78V8M1.edu|2452438| +3949|AAAAAAAANGPAAAAA|682737|3146|18308|2450941|2450911|Sir|Gary|Barksdale|N|29|4|1990|TUVALU||Gary.Barksdale@rvQapCLj.org|2452512| +3950|AAAAAAAAOGPAAAAA|||40526|2451625|||Nancy|Shah||29|||BERMUDA||Nancy.Shah@TBQJ2Tix38A5.edu|2452401| +3951|AAAAAAAAPGPAAAAA|26952|3343|2134|2451413|2451383|Dr.|Thomas|King|N|16|9|1925|NORFOLK ISLAND||Thomas.King@2XaGy0MUetv9S.com|2452642| +3952|AAAAAAAAAHPAAAAA|1633774|7139|21704|2450017|2449987|Dr.|Stewart|Sanders|N|13|1|1932|GUYANA||Stewart.Sanders@PFCh6u1xeP.edu|2452463| +3953|AAAAAAAABHPAAAAA|823841|4211|24934|2451254|2451224|Sir|Jack|Johns|Y|16|2|1966|GREENLAND||Jack.Johns@vSMRI.org|2452530| +3954|AAAAAAAACHPAAAAA|1129424|2515|9206|2450672|2450642|Dr.|Nicole|Jones|N|20|4|1962|ZAMBIA||Nicole.Jones@outY.edu|2452529| +3955|AAAAAAAADHPAAAAA|530008|5275|4589|2450284|2450254|Mrs.|Susan|Ballard|Y|30|12|1952|PANAMA||Susan.Ballard@nlQU04G.com|2452629| +3956|AAAAAAAAEHPAAAAA|1478661|4081|4301|2452266|2452236|Ms.|Deborah|Perry|Y|8|8|1980|GERMANY||Deborah.Perry@HlcLQ0.com|2452313| +3957|AAAAAAAAFHPAAAAA|483720|5809|21498|2451970|2451940|Sir|James|Pepper|Y|20|4|1961|BRAZIL||James.Pepper@pkLkX4d.edu|2452369| +3958|AAAAAAAAGHPAAAAA|691634|5198|39515|2451161|2451131|Dr.|Charles|Walsh|N|25|3|1954|PERU||Charles.Walsh@VFZ.org|2452477| +3959|AAAAAAAAHHPAAAAA|846449|6152|44655|2449116|2449086|Sir|David|Johnson|N|15|2|1955|OMAN||David.Johnson@ipHbcZc8fNpVmTxk.com|2452283| +3960|AAAAAAAAIHPAAAAA|1250543|7137|46178|2452180|2452150|Miss|Louise|Neumann|N|3|12|1935|TUNISIA||Louise.Neumann@RKrsy.org|2452283| +3961|AAAAAAAAJHPAAAAA|288876||44319|2451914|2451884||Emma||||||||Emma.Holcomb@pO.com|2452600| +3962|AAAAAAAAKHPAAAAA|1117300|6676|32852|2449597|2449567|Dr.|Barry|Nolan|Y|26|9|1979|MOZAMBIQUE||Barry.Nolan@4zXCLBQp6db.com|2452586| +3963|AAAAAAAALHPAAAAA|1525374|6400|15379|2449314|2449284|Mrs.|Theresa|Chau|N|25|11|1931|INDONESIA||Theresa.Chau@FvnK1vokzACG17yYl0M2.com|2452442| +3964|AAAAAAAAMHPAAAAA|||24879|2450739|||||N||6|1929||||2452499| +3965|AAAAAAAANHPAAAAA|1551544|3294|30997|2452151|2452121|Dr.|Cassandra|Jeffries|Y|11|10|1955|DJIBOUTI||Cassandra.Jeffries@Fj2euL1dzTams.edu|2452353| +3966|AAAAAAAAOHPAAAAA|210967|4558|34149|2449874|2449844|Mr.|Shawn|Dalton|N|20|10|1936|MYANMAR||Shawn.Dalton@7QNcM4QxNkKB.edu|2452396| +3967|AAAAAAAAPHPAAAAA|45496|3982|38915|2449243|2449213|Dr.|Brandon|Walker|N|20|3|1928|PARAGUAY||Brandon.Walker@1E3xKElX5VaAr.com|2452407| +3968|AAAAAAAAAIPAAAAA|1521828|3434|37840|2449893|2449863|Dr.|Raul|Lopez|Y|12|6|1967|UNITED ARAB EMIRATES||Raul.Lopez@yBNTZmQr9Vz0FMCh1aBv.com|2452322| +3969|AAAAAAAABIPAAAAA|927888|7035|35829|2451565|2451535|Ms.|Eva|Hitchcock|N|28|5|1950|SIERRA LEONE||Eva.Hitchcock@211XHzMymy07.com|2452388| +3970|AAAAAAAACIPAAAAA|104209|4121|14880|2451280|2451250|Dr.|Ronald|Wilson|Y|22|1|1987|FRENCH GUIANA||Ronald.Wilson@xxYD9QiSi.org|2452294| +3971|AAAAAAAADIPAAAAA|1631590|6586|49135|2451066|2451036|Sir|Benjamin|Moore|Y|11|7|1931|PANAMA||Benjamin.Moore@YmvXT5yxhbF.com|2452474| +3972|AAAAAAAAEIPAAAAA|893253|2111|7416|2450320|2450290|Sir|John|Keys|Y|20|12|1943|DENMARK||John.Keys@3sK2ashBVaZop6N5rFK.org|2452647| +3973|AAAAAAAAFIPAAAAA|1844739|3665|16591|2449928|2449898|Dr.|James|Donnelly|Y|6|12|1928|FINLAND||James.Donnelly@4xDMmkEiLsBxtk.edu|2452288| +3974|AAAAAAAAGIPAAAAA|1844659|473|26287|2450571|2450541|Mrs.|Irene|Thompson|Y|29|5|1951|HONG KONG||Irene.Thompson@zTZ4KtoRDyejfyiY.org|2452466| +3975|AAAAAAAAHIPAAAAA|1171953|254|2602|2451479|2451449|Dr.|Lina|Hadley|N|27|11|1956|SAMOA||Lina.Hadley@xSqPn57.edu|2452591| +3976|AAAAAAAAIIPAAAAA|732155|5233|18358|2452441|2452411|Mrs.|Albert|Curtin|N|8|11|1930|FRENCH POLYNESIA||Albert.Curtin@AaKbnQRe5ZOLIxRgPz8F.edu|2452438| +3977|AAAAAAAAJIPAAAAA|489078|2989|13653|2451095|2451065|Sir|Justin|Griffin|N|28|7|1956|CYPRUS||Justin.Griffin@3f32H.edu|2452595| +3978|AAAAAAAAKIPAAAAA|1539318|4674|38877|2450990|2450960|Miss|Gloria|Embry|Y|3|11|1952|BARBADOS||Gloria.Embry@S.org|2452641| +3979|AAAAAAAALIPAAAAA|1141707|4494|20834|2449806|2449776|Miss|Christine|Smith|Y|16|10|1972|UNITED STATES||Christine.Smith@CcH4PtdrO6qI3OQyf.edu|2452441| +3980|AAAAAAAAMIPAAAAA|128674|7131|25988|2451611|2451581|Mr.|Don|Brandon|N|10|8|1943|BOTSWANA||Don.Brandon@s.org|2452384| +3981|AAAAAAAANIPAAAAA|442998|5142|22839|2449954|2449924|Mrs.|Deborah|Hughes|N|19|4|1925|UNITED STATES||Deborah.Hughes@mBfxRfpPa.com|2452580| +3982|AAAAAAAAOIPAAAAA|1308672|4251|25383|2451690|2451660|Dr.|Derek|Haney|N|13|2|1968|BELGIUM||Derek.Haney@scYK7.edu|2452533| +3983|AAAAAAAAPIPAAAAA|500175|6807|40239|2449555|2449525|Mr.|Willie|Arnold|N|29|9|1960|UKRAINE||Willie.Arnold@1bfneHrQi8UNm.org|2452570| +3984|AAAAAAAAAJPAAAAA|1005625|207|12836|2452455|2452425|Sir|Norris|Howard|Y|14|6|1930|CZECH REPUBLIC||Norris.Howard@ClN1V5zIKfXO9J.com|2452283| +3985|AAAAAAAABJPAAAAA|1475179|2956|29886|2452111|2452081|Sir|Douglas|Smith|Y|8|2|1971|JAPAN||Douglas.Smith@hF9KfOktaH5qz3.edu|2452552| +3986|AAAAAAAACJPAAAAA|1172232|5701|15217|2452174|2452144|Dr.|Norma|Greene|N|13|12|1928|KOREA, REPUBLIC OF||Norma.Greene@30AkD6xnb.com|2452640| +3987|AAAAAAAADJPAAAAA|164258|5707|9109|2450158|2450128|Ms.|Jane|Rogers|N|8|6|1989|ICELAND||Jane.Rogers@AUha7Z0eSv.com|2452608| +3988|AAAAAAAAEJPAAAAA|1041296|3897|40898|2451389|2451359|Sir|Mark|Gillespie|N|9|8|1948|MONTSERRAT||Mark.Gillespie@tMQR.org|2452287| +3989|AAAAAAAAFJPAAAAA|1388582|1659|12336|2449477|2449447|Mrs.|Nancy|Childs|N|27|9|1967|AMERICAN SAMOA||Nancy.Childs@X7HKlzcHRisn.org|2452382| +3990|AAAAAAAAGJPAAAAA|1111723|856|30389|2449034|2449004|Sir|David|Lang|Y|29|4|1953|BELIZE||David.Lang@ovAkLHaI8HG.edu|2452374| +3991|AAAAAAAAHJPAAAAA|543254|6759|10311|2450538|2450508|Sir|Travis|Ferrell|N|3|11|1969|NETHERLANDS ANTILLES||Travis.Ferrell@oHhEuiD1SU.org|2452311| +3992|AAAAAAAAIJPAAAAA|1571518|5691|18617|2451361|2451331|Mrs.|Nanci|Mercer|N|20|3|1966|TUNISIA||Nanci.Mercer@buoT4.edu|2452563| +3993|AAAAAAAAJJPAAAAA|467893|6605|24117|2450244|2450214|Sir|Darren|Gentry|N|1|3|1989|NIGERIA||Darren.Gentry@ciRDkimcu397a.com|2452351| +3994|AAAAAAAAKJPAAAAA|1809381|3465|28749|2449084|2449054|Mr.|Sean|Rosario|N|20|7|1986|COMOROS||Sean.Rosario@skH1B.org|2452408| +3995|AAAAAAAALJPAAAAA|762608|1433|13505|2451661|2451631|Miss|Nell|Seward|Y|22|9|1981|MONGOLIA||Nell.Seward@cD9MhyOpasJmQ.com|2452304| +3996|AAAAAAAAMJPAAAAA||5072|39891||2451693||Timothy|||||1989|||Timothy.White@1vsEKCncUuJ.org|| +3997|AAAAAAAANJPAAAAA|1063222|644|29666|2450647|2450617|Sir|David|Harris|N|29|5|1951|PUERTO RICO||David.Harris@xy5Yz9UAni.org|2452440| +3998|AAAAAAAAOJPAAAAA|926818|6496|24974|2449745|2449715|Ms.|Barbara|Dockery|N|14|1|1947|NORWAY||Barbara.Dockery@Dqy.com|2452545| +3999|AAAAAAAAPJPAAAAA|178584|5026|35421|2451616|2451586|Miss|Kristian|Westfall|Y|5|3|1985|JERSEY||Kristian.Westfall@1n.org|2452494| +4000|AAAAAAAAAKPAAAAA|1664199|1204|46209|2452236|2452206|Mr.|Michael|Cross|Y|26|11|1927|SOUTH AFRICA||Michael.Cross@KBVHuEySEJzliraf.org|2452521| +4001|AAAAAAAABKPAAAAA|1395917|4726|5135|2449105|2449075|Sir|Michael|Tucker|Y|1|4|1956|UNITED STATES||Michael.Tucker@Rkb8k3bMjhF.edu|2452631| +4002|AAAAAAAACKPAAAAA|341334|2054|41574|2452143|2452113|Mrs.|Josefina|Black|Y|6|2|1965|GERMANY||Josefina.Black@RGZ53BxlJ07e.org|2452552| +4003|AAAAAAAADKPAAAAA|1772524|4423|36322|2449449|2449419|Dr.|Mohammad|Woodcock|Y|29|6|1981|ZAMBIA||Mohammad.Woodcock@pCa3SuohMzUrEr.edu|2452503| +4004|AAAAAAAAEKPAAAAA|391772|386|22247|2452061|2452031|Ms.|Amanda|Bean|N|23|6|1965|ALAND ISLANDS||Amanda.Bean@Y6q.com|2452330| +4005|AAAAAAAAFKPAAAAA|744504|1455|10406|2452558|2452528|Mr.|David|Murphy|Y|27|7|1980|VIET NAM||David.Murphy@08v9vLXU.com|2452620| +4006|AAAAAAAAGKPAAAAA|489019|7179|32139|2451736|2451706|Dr.|Herbert|Poulin|Y|5|6|1960|IRAQ||Herbert.Poulin@9cxLTMDy1e.org|2452588| +4007|AAAAAAAAHKPAAAAA|43968|5064|32960|2451994|2451964|Ms.|Ann|Stewart|N|19|4|1928|SINGAPORE||Ann.Stewart@TbqVQ5rrKt2.edu|2452405| +4008|AAAAAAAAIKPAAAAA|238048|533|41371|2449102|2449072|Dr.|Felix|Hawkins|N|1|5|1973|NORWAY||Felix.Hawkins@4Yt8TDmvM80.com|2452606| +4009|AAAAAAAAJKPAAAAA|1656792|3417|36842|2451734|2451704|Mrs.|Dianna|Young|Y|24|1|1984|NIGER||Dianna.Young@4ZZLU.org|2452426| +4010|AAAAAAAAKKPAAAAA|1165725|7022|37119|2452482|2452452|Mrs.|Jessica|Hanes|N|17|5|1952|NORFOLK ISLAND||Jessica.Hanes@1RlXYjNu8jdRHl.edu|2452633| +4011|AAAAAAAALKPAAAAA|729946|645|28591|2451005|2450975|Ms.|Elizabeth|Carrillo|N|26|7|1924|KIRIBATI||Elizabeth.Carrillo@L87ZzNjvR.edu|2452349| +4012|AAAAAAAAMKPAAAAA|||17201|2452269||Sir|Jorge|Palmer||28|||NICARAGUA|||2452500| +4013|AAAAAAAANKPAAAAA|680286|4798|22557|2449143|2449113|Dr.|Willie|Bryant|N|7|10|1926|HAITI||Willie.Bryant@QhVQ0.edu|2452625| +4014|AAAAAAAAOKPAAAAA|1202379|4723|42267|2452178|2452148|Dr.|David|Foley|Y|13|12|1928|VIET NAM||David.Foley@FQbJ33bd3FE.org|2452358| +4015|AAAAAAAAPKPAAAAA|1356606|2350|14141|2449939|2449909|Ms.|Rosanna|Hale|Y|16|10|1934|UNITED KINGDOM||Rosanna.Hale@9T5o3b2AZylIF3H.edu|2452447| +4016|AAAAAAAAALPAAAAA|6975|2909|36396|2451816|2451786|Ms.|Marquita|Priest|Y|5|5|1942|PERU||Marquita.Priest@XXBlx48.edu|2452374| +4017|AAAAAAAABLPAAAAA|808828|3170|10665|2452440|2452410|Miss|Beverly|Goss|Y|3|2|1964|ANGOLA||Beverly.Goss@dDLya1Ijg6z8Yj9KG.com|2452323| +4018|AAAAAAAACLPAAAAA|942228|3034|12287|2452357|2452327|Dr.|Barbara|Parker|N|2|8|1936|SWITZERLAND||Barbara.Parker@mes15vmzCnhvY2b.org|2452387| +4019|AAAAAAAADLPAAAAA|637119|7042|27365|2450429|2450399|Mr.|Gabriel|Maynard|N|14|1|1928|SAINT HELENA||Gabriel.Maynard@dFGYyPR94L2i.edu|2452606| +4020|AAAAAAAAELPAAAAA|493985|5922|41506|2450816|2450786|Dr.|Julie|Teague|Y|27|1|1960|MALAWI||Julie.Teague@9.com|2452547| +4021|AAAAAAAAFLPAAAAA|43130|2871|4659|2452670|2452640|Sir|Don|Ragsdale|Y|29|11|1928|BERMUDA||Don.Ragsdale@L2UXpi44lshpoJS7I.com|2452330| +4022|AAAAAAAAGLPAAAAA|1834096|5240|15937|2449157|2449127|Mrs.|Hue|Berryman|Y|9|8|1979|MAURITIUS||Hue.Berryman@DR993H.org|2452334| +4023|AAAAAAAAHLPAAAAA|842129|2337|40498|2449863|2449833|Dr.|Donald|Johnson|Y|15|11|1934|SINGAPORE||Donald.Johnson@qGoScIi9AzCu6R.org|2452642| +4024|AAAAAAAAILPAAAAA|1867328|2233|8385|2450578|2450548|Dr.|Robert|Herring|Y|30|10|1978|BOLIVIA||Robert.Herring@ChYE3yKqa.org|2452350| +4025|AAAAAAAAJLPAAAAA|991548|3609|49270|2451076|2451046|Sir|Robert|Manning|N|10|10|1942|GUYANA||Robert.Manning@nV7ZL7VSzRn.com|2452306| +4026|AAAAAAAAKLPAAAAA|541811|4789|47367|2452544|2452514|Sir|Nicholas|Shockley|N|1|11|1952|MOLDOVA, REPUBLIC OF||Nicholas.Shockley@GAXy8.edu|2452644| +4027|AAAAAAAALLPAAAAA|1838246|3883|49473|2449642|2449612|Dr.|Charles|Mast|N|10|9|1988|SOMALIA||Charles.Mast@OK2Sk2s8V7ccPtFTB0Q.edu|2452352| +4028|AAAAAAAAMLPAAAAA|1503968|4753|32338|2451804|2451774|Ms.|Adam|Wagner|Y|16|10|1986|NEW CALEDONIA||Adam.Wagner@Xl4yG1tLgb7.com|2452617| +4029|AAAAAAAANLPAAAAA|753117|5551|30432|2449476|2449446|Mr.|Richard|Carson|N|6|10|1958|SPAIN||Richard.Carson@DIS.edu|2452375| +4030|AAAAAAAAOLPAAAAA|1274579|917|43655|2452224|2452194|Sir|David|Roberts|Y|5|1|1940|SUDAN||David.Roberts@d437.com|2452520| +4031|AAAAAAAAPLPAAAAA|812115|3870|20872|2451936|2451906|Miss|Edwina|Welch|Y|10|11|1938|SURINAME||Edwina.Welch@FF8HCb2kRm6.com|2452324| +4032|AAAAAAAAAMPAAAAA|1272348|2641|14134|2451078|2451048|Ms.|Gladys|Bullock|Y|7|12|1941|SUDAN||Gladys.Bullock@sJ.edu|2452328| +4033|AAAAAAAABMPAAAAA|1114547|3845|15143|2451468|2451438|Dr.|Eileen|Reid|N|9|12|1978|LUXEMBOURG||Eileen.Reid@J.org|2452645| +4034|AAAAAAAACMPAAAAA|1624709|2040|39418|2451863|2451833|Mr.|Adolph|Hodges|N|4|6|1979|SUDAN||Adolph.Hodges@NcUtBP3F0xqBIAB.edu|2452449| +4035|AAAAAAAADMPAAAAA|1120073|2569|14312|2449533|2449503|Sir|Claude|Rodriquez|Y|16|5|1970|WESTERN SAHARA||Claude.Rodriquez@bdegaMj0ApzA.org|2452323| +4036|AAAAAAAAEMPAAAAA|469690|5976|20615|2450498|2450468|Miss|Ashley|Escamilla|Y|29|9|1954|FINLAND||Ashley.Escamilla@tOgaqKvLzdquO5CIP.org|2452290| +4037|AAAAAAAAFMPAAAAA|1270261|6408|43802|2451037|2451007|Miss|Muriel|Houston|Y|4|11|1932|TOGO||Muriel.Houston@ZAr7phA5Dgd.org|2452611| +4038|AAAAAAAAGMPAAAAA|796240||8905|2451211|||Isaac||N||3||GUATEMALA||Isaac.Hodge@L58QyZ5Fg3hYbFGrz.edu|2452551| +4039|AAAAAAAAHMPAAAAA|1644948|4639|20777|2450296|2450266|Ms.|Mildred|Blackburn|Y|14|8|1988|SOMALIA||Mildred.Blackburn@dBzcZosZGjDPnO.org|2452589| +4040|AAAAAAAAIMPAAAAA|449537|6282|47427|2449800|2449770|Ms.|Lois|Washington|N|21|8|1973|TONGA||Lois.Washington@uvUAMB9DuxcK.edu|2452319| +4041|AAAAAAAAJMPAAAAA|1622880|2084|20888|2451183|2451153|Dr.|Michael|Harper|Y|26|12|1959|SEYCHELLES||Michael.Harper@ZjqH.com|2452394| +4042|AAAAAAAAKMPAAAAA|226961|3388|6215|2449473|2449443|Sir|James|Brunson|N|7|10|1985|BOUVET ISLAND||James.Brunson@soo4xet.edu|2452431| +4043|AAAAAAAALMPAAAAA|518262|3439|44216|2451641|2451611|Miss|Marta|Hodge|Y|9|7|1962|SYRIAN ARAB REPUBLIC||Marta.Hodge@jUJjaqvjbXy0MVNyN.org|2452321| +4044|AAAAAAAAMMPAAAAA|1107740|652|19964|2451056|2451026|Sir|Frank|Dees|Y|9|5|1930|PITCAIRN||Frank.Dees@C.edu|2452328| +4045|AAAAAAAANMPAAAAA|1393558|6015|21719|2450568|2450538|Dr.|Michael|Wallace|Y|26|5|1948|MACAO||Michael.Wallace@smYZkIAZp.org|2452350| +4046|AAAAAAAAOMPAAAAA|910121|3318|36468|2450208|2450178|Ms.|Helen|Milligan|N|9|4|1963|CANADA||Helen.Milligan@E5ZRE5UKi3.edu|2452539| +4047|AAAAAAAAPMPAAAAA|1568722|2212|26502|2452528|2452498|Sir|Carl|Larson|Y|19|5|1969|NEW ZEALAND||Carl.Larson@P3FROHXhQDCIjJHho.org|2452300| +4048|AAAAAAAAANPAAAAA|1423820|3477|6287|2450774|2450744|Dr.|Joseph|Estrada|N|21|9|1981|FRENCH GUIANA||Joseph.Estrada@Conlpo.org|2452365| +4049|AAAAAAAABNPAAAAA|970701|1553|11205|2449165|2449135|Dr.|Don|Southard|N|19|1|1987|GAMBIA||Don.Southard@Zz3nfG.org|2452339| +4050|AAAAAAAACNPAAAAA|144907|2417|4280|2451380|2451350|Mrs.|Sara|Ashe|Y|20|10|1969|BHUTAN||Sara.Ashe@z.com|2452335| +4051|AAAAAAAADNPAAAAA|1145781|1067|43600|2452062|2452032|Dr.|Christy|Chavez|N|17|2|1953|BAHAMAS||Christy.Chavez@cv.org|2452512| +4052|AAAAAAAAENPAAAAA|1859131|2737|27496|2450652|2450622|Ms.|Pearl|Daniels|Y|5|6|1962|NIGER||Pearl.Daniels@HtG8QG0fq3kV.org|2452310| +4053|AAAAAAAAFNPAAAAA|1315565|525|43885|2451332|2451302|Ms.|Margaret|Hammer|Y|11|10|1931|SEYCHELLES||Margaret.Hammer@1ej0zuK0VEyk.com|2452343| +4054|AAAAAAAAGNPAAAAA|1461331|3749|17414|2450716|2450686|Dr.|Antonio|Hardin|Y|11|4|1935|ANTARCTICA||Antonio.Hardin@zjPPNi4nOCONIgx.com|2452579| +4055|AAAAAAAAHNPAAAAA|1485425|6314|10403|2450280|2450250|Miss|Karen|Dickson|N|5|1|1960|RWANDA||Karen.Dickson@Gv1EIvn2ogv2yZ.com|2452479| +4056|AAAAAAAAINPAAAAA|714277|3517|29157|2450117|2450087|Dr.|Charlotte|Turner|Y|31|12|1949|TONGA||Charlotte.Turner@qEX33BL8sS9KU.org|2452527| +4057|AAAAAAAAJNPAAAAA|437496|2693|43043|2449828|2449798|Miss|Brittany|Conte|N|6|11|1940|YEMEN||Brittany.Conte@EuKOyPqu5H.org|2452409| +4058|AAAAAAAAKNPAAAAA|833934|251|24568|2449697|2449667|Dr.|Catarina|Meek|Y|20|11|1971|JORDAN||Catarina.Meek@Rj3YRee4qyRYrhq.edu|2452396| +4059|AAAAAAAALNPAAAAA|1204147|3664|23981|2452652|2452622|Ms.|Edie|Waddell|N|2|12|1956|BARBADOS||Edie.Waddell@xtaRNotkZAV7IfahGNO.edu|2452610| +4060|AAAAAAAAMNPAAAAA|768264|4895|31926|2449300|2449270|Sir|Carl|Brown|Y|23|5|1949|BAHAMAS||Carl.Brown@dg3CbOnHkEcplU4.org|2452455| +4061|AAAAAAAANNPAAAAA|||25734||2451298|Sir|David|Chapman|||1||||David.Chapman@9Gu.edu|| +4062|AAAAAAAAONPAAAAA|1802211|591|44924|2449689|2449659|Dr.|Rick|Melvin|Y|26|7|1988|ISRAEL||Rick.Melvin@Fz1t9p.edu|2452483| +4063|AAAAAAAAPNPAAAAA|176659|6505|42351|2451190|2451160|Mrs.|Alice|Myers|Y|17|8|1956|SLOVAKIA||Alice.Myers@gZVBO3MGlrdyp.com|2452627| +4064|AAAAAAAAAOPAAAAA|1319015|4120|10825|2451265|2451235|Sir|Vance|Warren|N|24|7|1972|HUNGARY||Vance.Warren@TKY3.org|2452610| +4065|AAAAAAAABOPAAAAA|1372664|2096|5881|2451323|2451293|Mrs.|Katherine|Perkins|N|29|2|1960|BOLIVIA||Katherine.Perkins@kVbKOb7FMv.org|2452292| +4066|AAAAAAAACOPAAAAA|1652587|4405|14223|2449403|2449373|Dr.|Joshua|Hunt|Y|3|12|1971|NAMIBIA||Joshua.Hunt@6Egh3OPVB.edu|2452515| +4067|AAAAAAAADOPAAAAA|238365|5692|25466|2451903|2451873|Mr.|George|Gardner|N|23|3|1970|DJIBOUTI||George.Gardner@v0KUduBGjkg.com|2452618| +4068|AAAAAAAAEOPAAAAA|399504|5886|33068|2451297|2451267|Mrs.|Luella|Thomas|N|6|2|1944|GREECE||Luella.Thomas@xC4V.org|2452615| +4069|AAAAAAAAFOPAAAAA|182993|3153|1360|2450006|2449976|Dr.|Jeremy|Rose|N|26|6|1970|LIBERIA||Jeremy.Rose@sKjFg5tssToA.edu|2452639| +4070|AAAAAAAAGOPAAAAA|1758493|5538|8983|2449334|2449304|Sir|Walter|Bell|Y|16|7|1991|BURUNDI||Walter.Bell@VgutBt.edu|2452349| +4071|AAAAAAAAHOPAAAAA|1070666|6176|24486|2449811|2449781|Sir|George|Neal|N|28|3|1940|BOLIVIA||George.Neal@ltZ67F5uv8.edu|2452638| +4072|AAAAAAAAIOPAAAAA|164790|5436|23729|2451634|2451604|Mrs.|Melissa|Adkins|Y|31|8|1926|DOMINICA||Melissa.Adkins@A8ZT4PO966EFnm.org|2452481| +4073|AAAAAAAAJOPAAAAA|578555|552|17349|2451552|2451522|Ms.|Edna|Sanchez|N|12|6|1966|FRENCH GUIANA||Edna.Sanchez@PKLA47Ylc.com|2452598| +4074|AAAAAAAAKOPAAAAA|443746|4121|20496|2451878|2451848|Ms.|Andrea|Durham|N|3|12|1982|UGANDA||Andrea.Durham@UOHf8GUPF6Q.com|2452516| +4075|AAAAAAAALOPAAAAA|1752960|5236|37959|2450089|2450059|Mrs.|Harriet|Pulido|Y|1|2|1975|MALAYSIA||Harriet.Pulido@4JkN0ZTar78t.com|2452523| +4076|AAAAAAAAMOPAAAAA|1375117|3123|45576|2451694|2451664|Mr.|Charles|Brown|N|25|3|1969|SENEGAL||Charles.Brown@ZJCEMy1lLBkD.com|2452353| +4077|AAAAAAAANOPAAAAA|1232823|6132|36194|2450250|2450220|Mrs.|Lisa|Parker|N|16|10|1969|EL SALVADOR||Lisa.Parker@nSDfn6Q.com|2452312| +4078|AAAAAAAAOOPAAAAA|786885|1633|3534|2451028|2450998|Ms.|Gloria|Sturgeon|N|3|5|1925|MARSHALL ISLANDS||Gloria.Sturgeon@CcEDKnJcEgzgzn.com|2452573| +4079|AAAAAAAAPOPAAAAA|291599|1611|34470|2452593|2452563|Sir|Scott|Cramer|N|19|7|1948|JERSEY||Scott.Cramer@8POMCYRlO.edu|2452442| +4080|AAAAAAAAAPPAAAAA|148888|5381|39521|2450513|2450483|Mr.|Ricky|Macklin|N|25|10|1956|TURKMENISTAN||Ricky.Macklin@iNVMFof7qexTeAu.com|2452617| +4081|AAAAAAAABPPAAAAA|940603|4067|41931|2450418|2450388|Mrs.|Rebecca|Davis|N|20|8|1948|URUGUAY||Rebecca.Davis@IUAfoiQg9dcvdilhb.org|2452445| +4082|AAAAAAAACPPAAAAA|730269|5018|41767|2450296|2450266|Miss|Latisha|Mitchell|N|19|10|1963|LEBANON||Latisha.Mitchell@SuVV254vZzzZs6x.org|2452594| +4083|AAAAAAAADPPAAAAA|142116|3343|23723|2451240|2451210|Dr.|Shirlene|Schrader|Y|23|6|1983|TOKELAU||Shirlene.Schrader@zs6FROi23dRk1KmV.edu|2452412| +4084|AAAAAAAAEPPAAAAA|721170|3847|32477|2450674|2450644|Miss|Linda|Graves|Y|17|10|1924|BERMUDA||Linda.Graves@SYxqJhzqUdg0.edu|2452359| +4085|AAAAAAAAFPPAAAAA|1765656|5789|44275|2451268|2451238|Sir|Francisco|Schultz|Y|18|2|1924|LIBERIA||Francisco.Schultz@NONDQB4HMKDx7.org|2452487| +4086|AAAAAAAAGPPAAAAA|6135|3160|11385|2451418|2451388|Sir|Troy|Simmons|Y|16|9|1957|MALTA||Troy.Simmons@v61jSVtlJBAd0sHU.com|2452525| +4087|AAAAAAAAHPPAAAAA|1817837|83|39929|2451345|2451315|Sir|Alan|Smith|N|10|9|1962|MARSHALL ISLANDS||Alan.Smith@ek1.edu|2452312| +4088|AAAAAAAAIPPAAAAA|1395595|281|32120|2452154|2452124|Ms.|Michaela|Stacey|N|10|11|1963|LATVIA||Michaela.Stacey@kGy6q.edu|2452292| +4089|AAAAAAAAJPPAAAAA|1792892|2995|27898|2449506|2449476|Sir|Roy|Estrada|N|15|5|1985|JAPAN||Roy.Estrada@J2XfdUaEfmJ.org|2452370| +4090|AAAAAAAAKPPAAAAA|1502676|5676|23476|2451316|2451286|Dr.|Teresa|Pulliam|N|16|7|1986|SWEDEN||Teresa.Pulliam@DFiCPSoPIo.edu|2452301| +4091|AAAAAAAALPPAAAAA|840216|6386|13309|2450900|2450870|Ms.|Tara|Weber|N|11|5|1948|MAURITIUS||Tara.Weber@D7He4houfA.org|2452375| +4092|AAAAAAAAMPPAAAAA|560138|1245|2399|2449312|2449282|Ms.|Katrina|Cunningham|N|1|4|1925|VIRGIN ISLANDS, U.S.||Katrina.Cunningham@GiTeYmN5cM.com|2452623| +4093|AAAAAAAANPPAAAAA|123507|5688|31869|2450572|2450542|Dr.|Charlie|Stevenson|N|29|11|1945|WALLIS AND FUTUNA||Charlie.Stevenson@9oum29KDenZmr.edu|2452418| +4094|AAAAAAAAOPPAAAAA|1655316|4989|21462|2449724|2449694|Mr.|Phillip|Peters|N|31|1|1948|WALLIS AND FUTUNA||Phillip.Peters@Iaa7Nl.edu|2452602| +4095|AAAAAAAAPPPAAAAA|457163|823|26483|2449856|2449826|Miss|Chelsey|George|N|23|12|1928|NIGER||Chelsey.George@Fy.edu|2452436| +4096|AAAAAAAAAAABAAAA|1873570|6173|30423|2452592|2452562|Mrs.|Sandra|Williams|Y|25|6|1962|PHILIPPINES||Sandra.Williams@Dd3uUafQpbxkt.org|2452535| +4097|AAAAAAAABAABAAAA|1295176|939|33902|2449298|2449268|Dr.|Kent|Spencer|N|27|2|1944|PHILIPPINES||Kent.Spencer@ySp6x.org|2452497| +4098|AAAAAAAACAABAAAA|1563218|311|9404|2452257|2452227|Mrs.|Julie|Vasquez|Y|4|2|1990|KENYA||Julie.Vasquez@QzTpumCJg.com|2452344| +4099|AAAAAAAADAABAAAA|1848670|1002|20981|2449242|2449212|Dr.|Evelyn|West|N|18|9|1981|GUADELOUPE||Evelyn.West@5BTSgnTK.org|2452313| +4100|AAAAAAAAEAABAAAA|908013|684|14155|2449875|2449845|Mrs.|Carolyn|Mccrary|N|20|3|1949|GABON||Carolyn.Mccrary@CBmtcBCIycCqI.org|2452443| +4101|AAAAAAAAFAABAAAA|1349728|1761|5747|2449579|2449549|Dr.|Estelle|Brim|N|8|12|1946|PUERTO RICO||Estelle.Brim@EnBejXnnfbUi.org|2452451| +4102|AAAAAAAAGAABAAAA|292191|524|16857|2451406|2451376|Miss|Pamela|Lewis|Y|8|10|1933|TUNISIA||Pamela.Lewis@g6g2SPFzbQiOdM.edu|2452587| +4103|AAAAAAAAHAABAAAA|444985|3423|15908|2450253|2450223|Miss|Linda|Cooper|N|16|1|1990|ARGENTINA||Linda.Cooper@amd.com|2452498| +4104|AAAAAAAAIAABAAAA|1535643|364|25438|2450715|2450685|Sir|Charlie|Short|N|29|9|1981|WALLIS AND FUTUNA||Charlie.Short@i4zuMRD.com|2452518| +4105|AAAAAAAAJAABAAAA|224431|1629|34027|2450549|2450519|Sir|Carter|Carrillo|N|18|7|1976|CZECH REPUBLIC||Carter.Carrillo@qmlpTqIiR3AD.org|2452345| +4106|AAAAAAAAKAABAAAA|1341609|597|37185|2450459|2450429|Dr.|Janet|Cherry|N|31|10|1933|CAMEROON||Janet.Cherry@LH.com|2452309| +4107|AAAAAAAALAABAAAA|115236|2772|14661|2449704|2449674|Mr.|James|Scott|Y|12|7|1931|ALGERIA||James.Scott@Pg0.org|2452597| +4108|AAAAAAAAMAABAAAA|1895862|83|41104|2450758|2450728|Dr.|Leroy|Maxfield|N|7|7|1950|AMERICAN SAMOA||Leroy.Maxfield@HUggCYS6xLlJgGnDmB.org|2452380| +4109|AAAAAAAANAABAAAA|322386|1712|35485|2450558|2450528|Dr.|Jeff|Spring|Y|22|7|1955|LIECHTENSTEIN||Jeff.Spring@ClD.edu|2452397| +4110|AAAAAAAAOAABAAAA|1377012|1950|24441|2450902|2450872|Sir|Marcus|Freeman|N|21|2|1989|MONACO||Marcus.Freeman@HgRv.edu|2452473| +4111|AAAAAAAAPAABAAAA|983526|1594|26062|2450255|2450225|Dr.|William|Drummond|Y|1|10|1948|RWANDA||William.Drummond@7c.org|2452288| +4112|AAAAAAAAABABAAAA|1262957|865|25126|2450954|2450924|Dr.|Gilbert|Sherrill|N|16|8|1980|SWAZILAND||Gilbert.Sherrill@rtJt89bL8I.org|2452359| +4113|AAAAAAAABBABAAAA|753485|1065|42292|2450318|2450288|Dr.|Nicholas|Carey|Y|24|7|1990|MARSHALL ISLANDS||Nicholas.Carey@POJdDhiTz6oN0.com|2452543| +4114|AAAAAAAACBABAAAA|1339174|2475|24623|2452357|2452327|Miss|Ola|Isaacson|N|23|5|1933|MALAYSIA||Ola.Isaacson@o.com|2452607| +4115|AAAAAAAADBABAAAA|893842|5735|28028|2451096|2451066|Dr.|Dennis|Mclean|N|26|1|1928|BELGIUM||Dennis.Mclean@s1aTXoQrouVs.com|2452572| +4116|AAAAAAAAEBABAAAA|799539|3753|16414|2449765|2449735|Miss|Candace|Smith|Y|3|4|1966|NAURU||Candace.Smith@gytlZ16uTSyl3MGXHBj.edu|2452316| +4117|AAAAAAAAFBABAAAA|1762455|763|18671|2451577|2451547|Miss|Margaret|Morgan|Y|18|6|1930|BENIN||Margaret.Morgan@DVgCN71n.com|2452370| +4118|AAAAAAAAGBABAAAA|1494001|18|31235|2451182|2451152|Mr.|Michael|Hawks|Y|25|3|1958|SWAZILAND||Michael.Hawks@AzKCiDiE9H9m.edu|2452565| +4119|AAAAAAAAHBABAAAA|597495|2782|40069|2451130|2451100|Mr.|Raymond|Burger|Y|1|2|1989|CROATIA||Raymond.Burger@47MvquLF.org|2452522| +4120|AAAAAAAAIBABAAAA|877103|3131|19559|2452569|2452539|Dr.|John|Wilmoth|N|10|11|1984|MYANMAR||John.Wilmoth@QmuRYpepDtK.org|2452510| +4121|AAAAAAAAJBABAAAA|1703942|3736|29130|2449277|2449247|Dr.|Jennifer|Ritter|N|6|9|1974|ETHIOPIA||Jennifer.Ritter@LbgdSTAzq.com|2452614| +4122|AAAAAAAAKBABAAAA|1435934|3073|10612|2449258|2449228|Miss|Dorothy|Marrero|Y|21|8|1935|TAJIKISTAN||Dorothy.Marrero@kuOOovOg4zNcD.org|2452490| +4123|AAAAAAAALBABAAAA|913333|3612|27491||2451751||||||||EQUATORIAL GUINEA|||| +4124|AAAAAAAAMBABAAAA|1524226|1355|13637|2449723|2449693|Mr.|Jeffrey|Bowen|N|20|9|1982|UNITED ARAB EMIRATES||Jeffrey.Bowen@TAyRltCvc.com|2452498| +4125|AAAAAAAANBABAAAA|319577|4605|16080|2452253|2452223|Dr.|William|Bailey|Y|12|10|1969|EL SALVADOR||William.Bailey@iVhu8cPv5k4tL.org|2452358| +4126|AAAAAAAAOBABAAAA|530140|6360|980|2449378|2449348|Sir|Thomas|Pierce|N|12|2|1984|CUBA||Thomas.Pierce@BOKbAH2uF5e4KF.edu|2452539| +4127|AAAAAAAAPBABAAAA|1234378|4906|31081|2449109|2449079|Sir|Raymond|Williams|Y|31|1|1958|UZBEKISTAN||Raymond.Williams@8TnCPH.org|2452596| +4128|AAAAAAAAACABAAAA|1079540|5897|7054|2451573|2451543|Mr.|Delbert|Wilson|N|29|3|1931|UGANDA||Delbert.Wilson@62iCHpnztKGmpM.org|2452587| +4129|AAAAAAAABCABAAAA|615789|1163|38184|2451234|2451204|Mr.|Vincent|Miller|N|21|5|1957|VANUATU||Vincent.Miller@4dB4UfC.org|2452325| +4130|AAAAAAAACCABAAAA|1277144|3972|46000|2449207|2449177|Sir|James|Brandt|N|27|9|1943|ANTARCTICA||James.Brandt@x4.org|2452485| +4131|AAAAAAAADCABAAAA||7148|26646||2450776|Dr.||||15|7||OMAN|||| +4132|AAAAAAAAECABAAAA|1296685|5958|4831|2451840|2451810|Mrs.|Debra|Peachey|N|28|4|1946|SRI LANKA||Debra.Peachey@SJPV8.com|2452292| +4133|AAAAAAAAFCABAAAA|1246594|2164|28534|2452132|2452102|Dr.|Lillian|Cote|N|13|7|1954|SAN MARINO||Lillian.Cote@tvHutjvI048psMD.com|2452477| +4134|AAAAAAAAGCABAAAA|352266|7027|24541|2450977|2450947|Ms.|Ruby|Stewart|Y|3|9|1964|ARUBA||Ruby.Stewart@t2EhT1xz7jZsOGXZ.org|2452380| +4135|AAAAAAAAHCABAAAA|26358|2243|28975|2449407|2449377|Mrs.|Beth|Jones|Y|8|6|1988|SERBIA||Beth.Jones@N7zyEO3oLG.org|2452522| +4136|AAAAAAAAICABAAAA|1812332|5143|11154|2449763|2449733|Sir|Paul|Moreno|N|21|6|1946|CROATIA||Paul.Moreno@71Ggi9DZ.com|2452352| +4137|AAAAAAAAJCABAAAA||3581|37126|2450146|2450116||||N||9|1955|||Joe.Nichols@CUob2.com|| +4138|AAAAAAAAKCABAAAA|1843820|1756|2492|2450230|2450200|Mr.|Lindsey|Yu|Y|4|5|1967|UNITED ARAB EMIRATES||Lindsey.Yu@Omk.org|2452523| +4139|AAAAAAAALCABAAAA|113908||2721||2450726|Mr.||Taylor|N|6|1||||Rich.Taylor@5sKimAdE0ie.com|| +4140|AAAAAAAAMCABAAAA|480275|3065|31455|2450055|2450025|Dr.|Norman|Wright|N|5|2|1981|NEW ZEALAND||Norman.Wright@Ckx0gTobIgx8.org|2452644| +4141|AAAAAAAANCABAAAA|80116|6399|25882|2450989|2450959|Sir|Joseph|Wagner|Y|13|12|1964|RUSSIAN FEDERATION||Joseph.Wagner@7EVZYtxJgM5Nhfs.com|2452414| +4142|AAAAAAAAOCABAAAA|1316179|243|25954|2450235|2450205|Dr.|Crystal|Clarke|Y|31|1|1947|JORDAN||Crystal.Clarke@K5iRU3BgqG.com|2452627| +4143|AAAAAAAAPCABAAAA|135729|6459|24789|2451185|2451155|Dr.|Diane|Weaver|Y|30|7|1939|BRUNEI DARUSSALAM||Diane.Weaver@jXMt3CQN.edu|2452539| +4144|AAAAAAAAADABAAAA|1584970|453|43936|2451343|2451313|Mr.|Justin|Cloutier|Y|16|1|1924|QATAR||Justin.Cloutier@5Bj6MN4P95b2h.com|2452552| +4145|AAAAAAAABDABAAAA|890992|6848|39057|2449192|2449162|Ms.|Genevieve|Hopper|N|5|8|1985|TOGO||Genevieve.Hopper@gpod.org|2452331| +4146|AAAAAAAACDABAAAA|160866|6884|12055|2450918|2450888|Miss|Tonya|Moore|N|1|2|1963|ISLE OF MAN||Tonya.Moore@RVhb02gpFEPH.com|2452451| +4147|AAAAAAAADDABAAAA|876264|4636|33008|2450887|2450857|Dr.|John|Davies|N|7|9|1991|ANGOLA||John.Davies@rrO4Ojk.edu|2452577| +4148|AAAAAAAAEDABAAAA|1553819|6128|28084|2449673|2449643|Miss|Ida|Crespo|Y|16|4|1961|VENEZUELA||Ida.Crespo@Jh5m4mGpkqYFPy.com|2452532| +4149|AAAAAAAAFDABAAAA|1528679|243|48654|2450391|2450361|Dr.|Nancy|Baker|N|3|12|1959|TRINIDAD AND TOBAGO||Nancy.Baker@2I.org|2452446| +4150|AAAAAAAAGDABAAAA|304651|2852|37406|2451620|2451590|Mr.|Andy|Snodgrass|N|2|4|1992|MALTA||Andy.Snodgrass@SgnnmhrAMKFn3.com|2452421| +4151|AAAAAAAAHDABAAAA|1870709|5260|14564|2452466|2452436|Dr.|Matthew|Hammer|N|2|5|1930|KOREA, REPUBLIC OF||Matthew.Hammer@0xkZgNC7myYEt.org|2452422| +4152|AAAAAAAAIDABAAAA|1596829|4493|36957|2451560|2451530|Mr.|Mike|Lindsey|Y|28|12|1975|ECUADOR||Mike.Lindsey@N.org|2452377| +4153|AAAAAAAAJDABAAAA|1007679|2066|20853|2449051|2449021|Dr.|Thomas|Strickland|N|2|9|1974|LESOTHO||Thomas.Strickland@HF.com|2452396| +4154|AAAAAAAAKDABAAAA|1851|3560|23622|2450308|2450278|Miss|Alice|Hayes|N|12|10|1941|NORFOLK ISLAND||Alice.Hayes@K3PhH2XTZyK.com|2452321| +4155|AAAAAAAALDABAAAA|1808815|2381|37947|2449234|2449204|Sir|Joseph|Moore|Y|2|8|1983|BULGARIA||Joseph.Moore@li7LzomefPbXOS.com|2452460| +4156|AAAAAAAAMDABAAAA|588531|1885|25992|2452081|2452051|Ms.|Tammy|Moore|Y|6|10|1951|BOUVET ISLAND||Tammy.Moore@Mu9Rk0uygAG.org|2452369| +4157|AAAAAAAANDABAAAA|229024|3755|18886|||Mr.|Fernando|Miley|Y|||1987|||Fernando.Miley@oICM7oE.com|| +4158|AAAAAAAAODABAAAA|1703636|5566|12696|2450351|2450321|Dr.|Giselle|Thibodeau|N|25|12|1960|BANGLADESH||Giselle.Thibodeau@3f00syzKgPnGlaK66Jl.com|2452327| +4159|AAAAAAAAPDABAAAA|167282|1148|12614|2451310|2451280|Dr.|David|Jackson|Y|29|9|1956|FIJI||David.Jackson@fsfEVX.edu|2452364| +4160|AAAAAAAAAEABAAAA|1287489|1626|17716|2450513|2450483|Dr.|Margarita|Martin|Y|29|3|1933|COMOROS||Margarita.Martin@AUCNnJLKnDscjpOR.edu|2452486| +4161|AAAAAAAABEABAAAA|1548713|7198|42483|2451019|2450989|Dr.|James|Fleming|N|4|6|1961|IRAQ||James.Fleming@bkVcVGYjQb.org|2452306| +4162|AAAAAAAACEABAAAA|325059|5200|42494|2450431|2450401|Mr.|Cory|Hollingsworth|N|19|8|1930|ECUADOR||Cory.Hollingsworth@De1y3r9o.org|2452555| +4163|AAAAAAAADEABAAAA|1663529|1316|45056|2451607|2451577|Dr.|Curtis|Rosenthal|Y|29|10|1986|JERSEY||Curtis.Rosenthal@p2I.org|2452326| +4164|AAAAAAAAEEABAAAA|674259|4753|28494|2450877|2450847|Miss|Victoria|Ford|N|15|5|1935|CYPRUS||Victoria.Ford@9nUfMqeUOyLstoco.org|2452462| +4165|AAAAAAAAFEABAAAA|||14648|2452197|2452167|||Smith|Y||8|||||2452505| +4166|AAAAAAAAGEABAAAA|136427|2583|29092|2449033|2449003|Ms.|Susan|Reeves|Y|21|11|1928|ANTARCTICA||Susan.Reeves@qpaHmaapC.edu|2452345| +4167|AAAAAAAAHEABAAAA|787642|4074|23753|2451582|2451552|Ms.|Cecilia|Jones|Y|8|11|1971|QATAR||Cecilia.Jones@1scJ2HcakjLYZymlUi.edu|2452379| +4168|AAAAAAAAIEABAAAA|16626|1541|4456|2450232|2450202|Sir|Jimmy|Hayes|Y|2|1|1977|AZERBAIJAN||Jimmy.Hayes@11g7.edu|2452510| +4169|AAAAAAAAJEABAAAA|1080959|5481|46003|2449069|2449039|Mrs.|Christina|Miller|N|5|6|1963|URUGUAY||Christina.Miller@V0uNbkeuIN.com|2452595| +4170|AAAAAAAAKEABAAAA|801614|1679|44544|2451029|2450999|Ms.|Rose|Davis|N|13|12|1971|OMAN||Rose.Davis@SoNMZs6feCj3.edu|2452563| +4171|AAAAAAAALEABAAAA|826665|4952|41639|2452035|2452005|Mr.|Jorge|Levesque|Y|9|11|1967|ESTONIA||Jorge.Levesque@uVfFSXadug1jUt1lq.com|2452442| +4172|AAAAAAAAMEABAAAA|1292321|2791|20414|2451636|2451606|Dr.|Bradford|Smith|Y|10|8|1978|SWEDEN||Bradford.Smith@PYv2585SuEG.org|2452421| +4173|AAAAAAAANEABAAAA|1038091|5154|5017|2449687|2449657|Mr.|Wayne|Hurt|N|31|3|1973|MAYOTTE||Wayne.Hurt@xTtsobL8ja.com|2452425| +4174|AAAAAAAAOEABAAAA|982175|3704|21280|2449937|2449907|Dr.|Matthew|Thomas|Y|3|10|1951|SOLOMON ISLANDS||Matthew.Thomas@ejy64HtRhPoktR.edu|2452361| +4175|AAAAAAAAPEABAAAA|971627|3981|33935|2449798|2449768|Mrs.|Jessica|Roberson|N|25|11|1962|GUAM||Jessica.Roberson@Vs.com|2452407| +4176|AAAAAAAAAFABAAAA|1376680|6849|15865|2449448|2449418|Sir|Marshall|Salisbury|N|23|9|1970|AUSTRIA||Marshall.Salisbury@sBgFMHOtPGrjaoY6CHZ.org|2452443| +4177|AAAAAAAABFABAAAA|1610287|6843|47799|2450858|2450828|Mr.|John|Dixon|Y|19|11|1956|FIJI||John.Dixon@A8Bs.com|2452470| +4178|AAAAAAAACFABAAAA|558318|5371|44824|2450392|2450362|Ms.|Gloria|Leonard|Y|18|3|1984|AFGHANISTAN||Gloria.Leonard@peyqQgLFgVG.org|2452316| +4179|AAAAAAAADFABAAAA|1295265|3465|12018|2452376|2452346|Dr.|Jerome|Booker|N|10|5|1982|BAHRAIN||Jerome.Booker@PFBJpZsFq63.edu|2452450| +4180|AAAAAAAAEFABAAAA|797258|5558|9411|2451781|2451751|Ms.|Terri|Savage|Y|12|5|1953|ISLE OF MAN||Terri.Savage@Z7F.edu|2452374| +4181|AAAAAAAAFFABAAAA|691029|2454|15432|2450711|2450681|Mr.|Pedro|Sloan|Y|22|8|1947|TURKMENISTAN||Pedro.Sloan@1av051R.com|2452345| +4182|AAAAAAAAGFABAAAA|1785274|2080|44486|2451040|2451010|Ms.|Elizabeth|Roth|N|14|7|1941|MONTENEGRO||Elizabeth.Roth@ReCt6JEXjF.edu|2452488| +4183|AAAAAAAAHFABAAAA|86236|3881|45879|2449751|2449721|Sir|Brian|Schulz|Y|17|3|1943|CYPRUS||Brian.Schulz@ofq.org|2452585| +4184|AAAAAAAAIFABAAAA|602107|4122|10890|2450609|2450579|Mrs.|Helen|Wilkins|N|17|6|1969|SUDAN||Helen.Wilkins@H2eBfE3U6ch.edu|2452407| +4185|AAAAAAAAJFABAAAA|1679811|39|23114|2450771|2450741|Mrs.|Lina|Haynes|N|2|1|1986|PANAMA||Lina.Haynes@0B5zgqp0iAZ49s931f.edu|2452480| +4186|AAAAAAAAKFABAAAA|1084405|4325|21445|2452037|2452007|Dr.|Deborah|George|N|7|2|1965|NEW ZEALAND||Deborah.George@fcN.edu|2452373| +4187|AAAAAAAALFABAAAA|||30530||2449754|Mrs.|Catherine|Mcgowan|Y|2|4|1976|SWEDEN||Catherine.Mcgowan@LXQCdBV9E0CK.edu|2452422| +4188|AAAAAAAAMFABAAAA|1776250|6005|18207|2451760|2451730|Dr.|Beverly|Hargett|Y|20|9|1980|KAZAKHSTAN||Beverly.Hargett@G6cmCJzlylJ01Om.edu|2452609| +4189|AAAAAAAANFABAAAA|652509|2178|15847|2449218|2449188|Mrs.|So|Erickson|Y|26|9|1967|ALBANIA||So.Erickson@EDNmO.edu|2452365| +4190|AAAAAAAAOFABAAAA|364620|5994|48121|2451137|2451107|Mr.|Jeffrey|Davison|Y|30|10|1965|ROMANIA||Jeffrey.Davison@4c6ObS2TV5hph.edu|2452530| +4191|AAAAAAAAPFABAAAA|269645|1482|44566|2449784|2449754|Mr.|Joseph|Parker|N|8|11|1930|RUSSIAN FEDERATION||Joseph.Parker@bebEQLN.edu|2452495| +4192|AAAAAAAAAGABAAAA|422369|5695|11774|2450425|2450395|Mr.|John|Mackey|Y|3|5|1962|GREENLAND||John.Mackey@GBMpl05G69m.org|2452553| +4193|AAAAAAAABGABAAAA|1685342|5248|754|2451479|2451449|Dr.|Willia|Hernandez|N|24|11|1958|JAPAN||Willia.Hernandez@yG71Hrmgnc9JLS49.edu|2452504| +4194|AAAAAAAACGABAAAA|1228371|2167|18763|2449457|2449427|Dr.|Suzanne|Sweeney|Y|3|1|1975|SAINT LUCIA||Suzanne.Sweeney@EX7AZ.org|2452427| +4195|AAAAAAAADGABAAAA|1424351|1283|6215|2450454|2450424|Dr.|Scott|Cote|N|9|10|1949|LIBERIA||Scott.Cote@IKeJqN586XMfA.edu|2452384| +4196|AAAAAAAAEGABAAAA|850747|2875|30603|2450335|2450305|Ms.|Lois|Bonilla|Y|26|11|1965|KIRIBATI||Lois.Bonilla@eamFEqaKiX.org|2452382| +4197|AAAAAAAAFGABAAAA|109966|2361|4709|2449605|2449575|Mrs.|Anna|Kaplan|Y|7|7|1942|ALBANIA||Anna.Kaplan@1u2ZM.org|2452458| +4198|AAAAAAAAGGABAAAA|1136676|2922|43121|2450606|2450576|Mrs.|Imogene|Lewis|N|7|12|1972|LIBERIA||Imogene.Lewis@813MelxMLZ9gy.com|2452426| +4199|AAAAAAAAHGABAAAA|||31721||2450405||Tommie|Rodriguez||||||||| +4200|AAAAAAAAIGABAAAA|1239229|3552|47922|2449383|2449353|Mr.|John|Heath|N|31|8|1992|HONG KONG||John.Heath@M.org|2452612| +4201|AAAAAAAAJGABAAAA|1326432|1921|31763|2449575|2449545|Ms.|Marie|Coleman|Y|7|7|1931|ECUADOR||Marie.Coleman@DsHpy9.edu|2452519| +4202|AAAAAAAAKGABAAAA|1483598|5734|8717|2452123|2452093|Sir|John|Brown|N|20|12|1987|BERMUDA||John.Brown@1P0nDAEZvZfpeEQ84H.com|2452626| +4203|AAAAAAAALGABAAAA|122448|4684|49673|2452161|2452131|Ms.|Joanne|Souza|N|16|5|1937|BHUTAN||Joanne.Souza@9Oqd7yZkuFT1zK.org|2452559| +4204|AAAAAAAAMGABAAAA|1331937|3432|16574|2452371|2452341|Miss|Kathleen|Elmore|Y|24|1|1964|VIET NAM||Kathleen.Elmore@59hUlMeFO.edu|2452483| +4205|AAAAAAAANGABAAAA|301971|3629|840|2450898|2450868|Dr.|Nora|Owens|N|25|6|1979|CHINA||Nora.Owens@V0jdZqSgZp.com|2452522| +4206|AAAAAAAAOGABAAAA|997326|5648|41523|2450074|2450044|Mr.|Robert|Barton|N|30|8|1937|LATVIA||Robert.Barton@bgQVGyLlcY8xq.com|2452340| +4207|AAAAAAAAPGABAAAA|967348|3187|20673|2452160|2452130|Dr.|Frank|Shinn|N|7|1|1928|ANDORRA||Frank.Shinn@ZnIlpHd0VozA.com|2452554| +4208|AAAAAAAAAHABAAAA|632429|2471|23313|2452282|2452252|Sir|Marvin|Carlisle|N|15|8|1939|WALLIS AND FUTUNA||Marvin.Carlisle@jH2kFiT7qcO.edu|2452316| +4209|AAAAAAAABHABAAAA|1438800|1029|29705|2451606|2451576|Dr.|Cathy|Davis|N|1|8|1925|SOMALIA||Cathy.Davis@J49.edu|2452624| +4210|AAAAAAAACHABAAAA|828481|1887|10093||||Doris|||21|10||||Doris.Ellis@eD3YlQ5svda.com|2452632| +4211|AAAAAAAADHABAAAA|1166813|4690|44567|2451149|2451119|Dr.|Andrea|Griffin|Y|1|2|1972|JORDAN||Andrea.Griffin@JC711G.com|2452345| +4212|AAAAAAAAEHABAAAA|1115578|3914|40697|2451211|2451181|Dr.|Dawne|Post|N|2|9|1968|MACAO||Dawne.Post@bS3bqPA.org|2452308| +4213|AAAAAAAAFHABAAAA|738993|4927|48461|2449413|2449383|Mr.|Henry|Herron|N|31|1|1974|ITALY||Henry.Herron@xvFkFXmV1zP7REGgPA94.org|2452578| +4214|AAAAAAAAGHABAAAA|1600316|4519|27922|2449969|2449939|Mrs.|Lina|Abbott|N|8|9|1928|SPAIN||Lina.Abbott@lkjk.edu|2452646| +4215|AAAAAAAAHHABAAAA|357522|3399|6076|2451126|2451096|Ms.|Katherine|Richardson|N|26|8|1979|CHILE||Katherine.Richardson@b9VinaDXp4xy9K1.edu|2452287| +4216|AAAAAAAAIHABAAAA|470263|6422|16981|2449988|2449958|Ms.|Cindy|Lassiter|N|26|4|1956|KUWAIT||Cindy.Lassiter@MLLPXiOvXx2V76Sa.edu|2452374| +4217|AAAAAAAAJHABAAAA|202546|5625|35951|2449880|2449850|Miss|Melissa|Meyer|Y|2|3|1929|SAINT LUCIA||Melissa.Meyer@FEJMJyN5YZofcaJ.org|2452350| +4218|AAAAAAAAKHABAAAA|188223|5351|11279|2449561|2449531|Dr.|Sean|Clark|N|12|8|1984|TURKEY||Sean.Clark@bMI3.com|2452303| +4219|AAAAAAAALHABAAAA|652986|5655|31924|2449066|2449036|Dr.|Norman|Garza|Y|7|2|1966|GUINEA||Norman.Garza@ZbEs.edu|2452405| +4220|AAAAAAAAMHABAAAA|1073862|4296|24283|2450315|2450285|Mr.|Willard|Riddle|N|10|5|1989|ROMANIA||Willard.Riddle@StePJazcuXx99uhd.org|2452553| +4221|AAAAAAAANHABAAAA|330187|6049|3911|2450616|2450586|Miss|Cecile|Greene|N|6|5|1992|PITCAIRN||Cecile.Greene@08vRabVfdr6us5M.com|2452528| +4222|AAAAAAAAOHABAAAA|1879075|5173|17224|2451973|2451943|Dr.|Brandon|Huang|N|29|2|1972|ANGOLA||Brandon.Huang@7Q1XKGMU64EH.edu|2452573| +4223|AAAAAAAAPHABAAAA|1561165|7094|23470|2451686|2451656|Miss|Jill|Perkins|Y|13|2|1950|SERBIA||Jill.Perkins@qhfjfIc.org|2452529| +4224|AAAAAAAAAIABAAAA|1446904|2585|22666|2450307|2450277|Miss|Fatima|Grady|Y|2|2|1938|FRENCH POLYNESIA||Fatima.Grady@3IX.org|2452611| +4225|AAAAAAAABIABAAAA|807510|5314|28718|2451508|2451478|Dr.|Floyd|Barber|Y|5|2|1988|ESTONIA||Floyd.Barber@kLdDpY.org|2452599| +4226|AAAAAAAACIABAAAA|1422986|6229|45919|2450607|2450577|Ms.|Emily|Stevenson|N|15|6|1965|ARUBA||Emily.Stevenson@3y0b9Za2Xgaxtsm9A43.com|2452588| +4227|AAAAAAAADIABAAAA|1148963|6614|35065|2449726|2449696|Mr.|Neil|Lee|N|20|4|1934|SAUDI ARABIA||Neil.Lee@E.com|2452400| +4228|AAAAAAAAEIABAAAA|1390625|2021|29370|2451389|2451359|Dr.|Winifred|Tavares|Y|16|2|1931|SIERRA LEONE||Winifred.Tavares@sgCko.com|2452409| +4229|AAAAAAAAFIABAAAA|1072694|4190|46086|2452300|2452270|Miss|Janet|Hamilton|Y|17|8|1986|GERMANY||Janet.Hamilton@Y5t3f.org|2452283| +4230|AAAAAAAAGIABAAAA|591931|5734|690|2452209|2452179|Sir|Phillip|Villarreal|Y|17|2|1958|BOUVET ISLAND||Phillip.Villarreal@VEtjbfo3yij6mbQ57.org|2452509| +4231|AAAAAAAAHIABAAAA|321908|4006|26104|2452570|2452540|Dr.|Daniel|Nichols|Y|2|2|1962|WESTERN SAHARA||Daniel.Nichols@SCXk0u7l0eAsHNZ.org|2452349| +4232|AAAAAAAAIIABAAAA|780851|1107|45410|2451215|2451185|Miss|Rosa|Phillips|Y|19|3|1949|SURINAME||Rosa.Phillips@ECXsk.com|2452488| +4233|AAAAAAAAJIABAAAA|1696317|5350|48383|2451733|2451703|Dr.|Stanley|Guerrero|Y|24|10|1992|WESTERN SAHARA||Stanley.Guerrero@xcAqVVVvAjS.org|2452429| +4234|AAAAAAAAKIABAAAA|365830|6446|15109|2450225|2450195|Sir|William|Simmons|N|25|12|1962|SWITZERLAND||William.Simmons@05TnX07xbBF9yIhPE.org|2452335| +4235|AAAAAAAALIABAAAA|||18925||2452249|||Russell|Y||3|1951|||Patty.Russell@1YiHo2lYmsxKX2.edu|| +4236|AAAAAAAAMIABAAAA|1112226|1130|8834|2452560|2452530|Mr.|Mark|Sullivan|N|10|11|1987|THAILAND||Mark.Sullivan@nukeHMMIFIVM2f.org|2452620| +4237|AAAAAAAANIABAAAA|1326441|464|17287|2452431|2452401|Mr.|Robert|Richardson|N|21|8|1987|HONG KONG||Robert.Richardson@do.org|2452585| +4238|AAAAAAAAOIABAAAA|965885|678|43824|2452241|2452211|Sir|James|Thompson|Y|18|10|1951|SEYCHELLES||James.Thompson@1amSMaHkBBn7.org|2452586| +4239|AAAAAAAAPIABAAAA|1663505|6519|34380|2452608|2452578|Dr.|Deborah|Kline|N|29|3|1941|BOTSWANA||Deborah.Kline@RrGYJ.org|2452632| +4240|AAAAAAAAAJABAAAA||3723|43371||2449089|Mr.|||N||12|||||| +4241|AAAAAAAABJABAAAA|1342313|1821|42929|2450345|2450315|Ms.|Vivian|Yazzie|Y|7|2|1963|ANGUILLA||Vivian.Yazzie@e4T1ZmFpAsYe7CSXf2.edu|2452371| +4242|AAAAAAAACJABAAAA|1390810|1370|38730|2449144|2449114|Sir|Ralph|Moore|N|12|7|1934|EL SALVADOR||Ralph.Moore@EgJBq5tKVfB.com|2452551| +4243|AAAAAAAADJABAAAA|797140|2626|20465|2450200|2450170|Mrs.|Charity|Arroyo|N|12|4|1941|VIET NAM||Charity.Arroyo@i.com|2452629| +4244|AAAAAAAAEJABAAAA|870251|6827|603|2452653|2452623|Ms.|Mamie|Lopez|Y|30|9|1972|HAITI||Mamie.Lopez@qgbTnxDIl.org|2452529| +4245|AAAAAAAAFJABAAAA|1566252|6191|26130|2452041|2452011|Miss|Jane|Fitzpatrick|N|1|9|1958|NAURU||Jane.Fitzpatrick@QogF2.com|2452501| +4246|AAAAAAAAGJABAAAA|1766606|1329|16899|2450207|2450177|Miss|Elvira|Young|Y|25|9|1977|MYANMAR||Elvira.Young@vaVcVE.edu|2452608| +4247|AAAAAAAAHJABAAAA|1842300|2214|13775|2450159|2450129|Dr.|Edythe|Williams|Y|20|4|1948|MYANMAR||Edythe.Williams@ov6heuBUJ9ukupJ.org|2452326| +4248|AAAAAAAAIJABAAAA|494189|1418|36200|2452510|2452480|Mr.|Joe|Huang|Y|29|6|1942|MARSHALL ISLANDS||Joe.Huang@L057KlMlunq1AOQFqX.org|2452433| +4249|AAAAAAAAJJABAAAA|1751982|211|21993|2450883|2450853|Dr.|Hubert|Harmon|Y|21|1|1984|BANGLADESH||Hubert.Harmon@39MQU2SJ67HxcbX9.edu|2452593| +4250|AAAAAAAAKJABAAAA|1342081|4278|33675|2449779|2449749|Dr.|James|Rankin|N|27|5|1974|COMOROS||James.Rankin@qYt98FDRTp.edu|2452328| +4251|AAAAAAAALJABAAAA|607677|6445|38813|2449575|2449545|Sir|Tim|Wagner|Y|24|5|1932|ANGOLA||Tim.Wagner@alPe6gmZ.org|2452312| +4252|AAAAAAAAMJABAAAA||2313|41717|2449719||Mr.||Wolfe|N|26|||SLOVENIA||Robert.Wolfe@zSfyUSh430u.com|| +4253|AAAAAAAANJABAAAA||3770|30030||||Carl|Crawford|||6|1937|CYPRUS|||| +4254|AAAAAAAAOJABAAAA|172427|3532|42237|2451715|2451685|Miss|Misty|Horsley|Y|10|11|1987|LEBANON||Misty.Horsley@JR5dN1KVHcZMX9e.org|2452442| +4255|AAAAAAAAPJABAAAA|1597318|6722|46526|2449700|2449670|Ms.|Kay|Davis|N|27|5|1936|NEPAL||Kay.Davis@vl7.com|2452504| +4256|AAAAAAAAAKABAAAA|1431301|6916|46578|2451167|2451137|Mrs.|Tara|Blevins|N|15|8|1985|AMERICAN SAMOA||Tara.Blevins@ePszuo3t6tT.org|2452541| +4257|AAAAAAAABKABAAAA|183352|1464|32023|2451738|2451708|Dr.|Louis|Green|N|29|5|1988|MOLDOVA, REPUBLIC OF||Louis.Green@kkB2BS91VGtP.org|2452505| +4258|AAAAAAAACKABAAAA|950283|4291|11499|2449516|2449486|Mr.|Johnathon|White|N|29|5|1959|ESTONIA||Johnathon.White@MnffcxxFQgVNEIi7h.com|2452597| +4259|AAAAAAAADKABAAAA|646913|6634|45955|2451203|2451173|Dr.|Alice|Hernandez|Y|28|6|1933|COMOROS||Alice.Hernandez@YVMZeDfZ.org|2452300| +4260|AAAAAAAAEKABAAAA|847975|4005|48388|2451887|2451857|Sir|Brandon|Masters|N|21|3|1989|AMERICAN SAMOA||Brandon.Masters@7eh.edu|2452412| +4261|AAAAAAAAFKABAAAA|1107313|5585|47857|2450148|2450118|Dr.|Daisy|Elmore|N|30|10|1962|NEW CALEDONIA||Daisy.Elmore@zG6zo1PlgOU.org|2452599| +4262|AAAAAAAAGKABAAAA|743889|6295|42254|2451732|2451702|Miss|Gina|King|Y|27|10|1924|BENIN||Gina.King@917iH5xGeRlmFyx2.org|2452301| +4263|AAAAAAAAHKABAAAA|1502511|6584|11857|2450988|2450958|Sir|Leonard|Mccracken|Y|15|8|1949|ZIMBABWE||Leonard.Mccracken@CteynuzhImsgmBARA7.com|2452509| +4264|AAAAAAAAIKABAAAA|721345|2922|15746|2450880|2450850|Dr.|James|Taylor|Y|2|1|1967|LEBANON||James.Taylor@eVoZuvYcqACN.org|2452537| +4265|AAAAAAAAJKABAAAA|59572|6757|39095|2451843|2451813|Dr.|Carla|Collins|N|19|1|1942|MADAGASCAR||Carla.Collins@q.edu|2452506| +4266|AAAAAAAAKKABAAAA|1879949||3844|2450690||Miss|Cynthia|Price|Y||5||SAMOA||Cynthia.Price@D9jyInPxPQ.edu|2452632| +4267|AAAAAAAALKABAAAA|426767|3680|39440|2450851|2450821|Sir|Kurt|Turner|Y|15|4|1973|TUVALU||Kurt.Turner@F9DGAB2X3F.org|2452403| +4268|AAAAAAAAMKABAAAA|1502093|6477|35261|2451654|2451624|Sir|Brandon|Fahey|Y|6|2|1990|GUINEA-BISSAU||Brandon.Fahey@IooEO.com|2452419| +4269|AAAAAAAANKABAAAA|1355552|4046|32940|2451879|2451849|Miss|Marjorie|Kelley|Y|25|11|1971|JAPAN||Marjorie.Kelley@ppNpvKLxTfjH4KVo.com|2452642| +4270|AAAAAAAAOKABAAAA|941957|6704|10298|2449696|2449666|Sir|Jack|Wells|Y|15|4|1953|HONG KONG||Jack.Wells@E04h7M0qcq5444TBDq.com|2452524| +4271|AAAAAAAAPKABAAAA|273504|2686|36715|2451970|2451940|Dr.|Terri|Jones|Y|22|7|1932|BAHAMAS||Terri.Jones@mgv.com|2452313| +4272|AAAAAAAAALABAAAA|994718|4292|49466|2450774|2450744|Dr.|David|Smith|N|12|3|1938|GABON||David.Smith@oUyO9Qn3h3HFo3OH1y.com|2452465| +4273|AAAAAAAABLABAAAA|645729|3438|32300|2450031|2450001|Miss|Caroline|Swanson|N|15|12|1927|QATAR||Caroline.Swanson@lHj2BNgdVbZNVU.com|2452331| +4274|AAAAAAAACLABAAAA|1626116|5108|947|2451950|2451920|Mrs.|Diana|Burch|Y|3|5|1940|TOKELAU||Diana.Burch@JGd032LZBIPf.com|2452295| +4275|AAAAAAAADLABAAAA|858034|86|14138|2450330|2450300|Mrs.|Sharla|Buchanan|N|1|4|1927|MOROCCO||Sharla.Buchanan@spuxVTb8PUSp.org|2452369| +4276|AAAAAAAAELABAAAA|838167|2712|13210|2452200|2452170|Dr.|Judy|Watts|N|14|3|1983|TRINIDAD AND TOBAGO||Judy.Watts@0NVHM0Bfm.com|2452298| +4277|AAAAAAAAFLABAAAA|1364358|1625|17292|2451571|2451541|Dr.|Israel|Bishop|Y|12|12|1968|COMOROS||Israel.Bishop@u1SQA6TUUHBi7iq.org|2452550| +4278|AAAAAAAAGLABAAAA|1199191|6647|7750|2451443|2451413|Mrs.|Diane|Hicks|N|16|12|1956|FRENCH GUIANA||Diane.Hicks@Hf2bEykspOPLSPq.edu|2452606| +4279|AAAAAAAAHLABAAAA|759467|6386|47658|2451096|2451066|Mrs.|Ali|Miller|N|11|6|1978|GHANA||Ali.Miller@b3KFv8jJl.org|2452597| +4280|AAAAAAAAILABAAAA|762963|1579|39348|2451156|2451126|Miss|Anthony|Guenther|N|22|6|1929|GUADELOUPE||Anthony.Guenther@bZ4.com|2452479| +4281|AAAAAAAAJLABAAAA|763622|4732|10595|2450999|2450969|Miss|Karen|Cope|N|22|12|1931|LITHUANIA||Karen.Cope@kSMANnUhLhuzz8.com|2452617| +4282|AAAAAAAAKLABAAAA|1774152|5907|45615|2451237|2451207|Dr.|Kristy|Gustafson|Y|6|2|1943|COMOROS||Kristy.Gustafson@9nM3J87HhhSnVo2tEFt.com|2452500| +4283|AAAAAAAALLABAAAA|1477382|3730|20078|2450911|2450881|Dr.|Robert|Hubbard|N|23|11|1940|AMERICAN SAMOA||Robert.Hubbard@b.edu|2452608| +4284|AAAAAAAAMLABAAAA||5686|75||2452432||||Y|||1936|||Horace.Merrell@r.edu|| +4285|AAAAAAAANLABAAAA|1672010|2650|35290|2449250|2449220|Mrs.|Terri|Slaughter|Y|3|2|1984|SRI LANKA||Terri.Slaughter@sy3.com|2452577| +4286|AAAAAAAAOLABAAAA|309617|4923|4629|2452444|2452414|Sir|Ryan|Cole|Y|11|10|1936|NETHERLANDS ANTILLES||Ryan.Cole@r6tNFg54tYmH.edu|2452594| +4287|AAAAAAAAPLABAAAA|||48784|2449569||Ms.|Sarah|Clark|N|26||1981|BARBADOS|||2452357| +4288|AAAAAAAAAMABAAAA|1685105|2317|37176|2449860|2449830|Sir|Christopher|Lamm|Y|13|3|1928|MARSHALL ISLANDS||Christopher.Lamm@H.edu|2452341| +4289|AAAAAAAABMABAAAA|1166912|1194|24676|2449706|2449676|Mrs.|Alex|Tyler|N|31|7|1954|EQUATORIAL GUINEA||Alex.Tyler@lkS3DujLepZjc.org|2452384| +4290|AAAAAAAACMABAAAA|1771649|3578|16307|2451305|2451275|Mr.|Larry|Coburn|Y|22|2|1941|MALDIVES||Larry.Coburn@VIJkjIqLapUCl.com|2452412| +4291|AAAAAAAADMABAAAA|1011423|584|33792|2450157|2450127|Ms.|Krista|Kendrick|Y|17|7|1959|BULGARIA||Krista.Kendrick@9fhhe0GsxThtF3KAC.edu|2452633| +4292|AAAAAAAAEMABAAAA|643484|2523|7609|2450955|2450925|Ms.|Andrew|Mcdonough|Y|11|6|1973|GABON||Andrew.Mcdonough@jZT2E2rhDiFeAJJ.edu|2452585| +4293|AAAAAAAAFMABAAAA|894951|1469|36034|2451034|2451004|Dr.|Micheal|Hall|N|2|10|1983|IRAQ||Micheal.Hall@Er6r.org|2452297| +4294|AAAAAAAAGMABAAAA|373380|4681|8829|2449384|2449354|Dr.|Robert|Stapleton|Y|9|11|1971|BRUNEI DARUSSALAM||Robert.Stapleton@6KEzQD5dopp0GHdyn.edu|2452522| +4295|AAAAAAAAHMABAAAA|150008|5802|41996|2451685|2451655|Ms.|Alberta|Murray|N|26|5|1986|NETHERLANDS||Alberta.Murray@lppq5iG0I4.org|2452593| +4296|AAAAAAAAIMABAAAA|156008|2755|36302|2452286|2452256|Miss|Catherine|Powers|Y|9|11|1974|VENEZUELA||Catherine.Powers@PT.edu|2452423| +4297|AAAAAAAAJMABAAAA|394866|4111|46956|2449313|2449283|Dr.|Kevin|Young|N|13|8|1964|GRENADA||Kevin.Young@eF.edu|2452334| +4298|AAAAAAAAKMABAAAA|1823850|5357|36864|2452578|2452548|Sir|Larry|Wallis|N|25|1|1924|ETHIOPIA||Larry.Wallis@Z5AjGaK14.com|2452422| +4299|AAAAAAAALMABAAAA|1843921|490|34306|2449281|2449251|Dr.|John|Dortch|N|9|6|1991|CUBA||John.Dortch@IUHxy.edu|2452348| +4300|AAAAAAAAMMABAAAA|1739714|532|46597|2449477|2449447|Dr.|Nathaniel|Harmon|Y|9|2|1990|NORFOLK ISLAND||Nathaniel.Harmon@li27R4S1.org|2452307| +4301|AAAAAAAANMABAAAA|||33086|2449835|2449805||||||7|1960|GUYANA|||| +4302|AAAAAAAAOMABAAAA|1911813|3241|24669|2449277|2449247|Dr.|Michelle|Golden|Y|13|4|1943|MALAWI||Michelle.Golden@tLj52dlCy8.edu|2452611| +4303|AAAAAAAAPMABAAAA|195670|5767|47111|2452677|2452647|Dr.|Todd|Hicks|Y|8|7|1962|BOUVET ISLAND||Todd.Hicks@NS5zId7V.org|2452472| +4304|AAAAAAAAANABAAAA|1012927|1926|20125|2449526|2449496|Mrs.|Adam|Cortez|Y|17|10|1953|SENEGAL||Adam.Cortez@n3g.edu|2452628| +4305|AAAAAAAABNABAAAA|225303|1812|47921|2450992|2450962|Mrs.|Carrie|Fleming|Y|26|9|1948|BANGLADESH||Carrie.Fleming@Dei44vYDkjha.edu|2452453| +4306|AAAAAAAACNABAAAA|774315||18868||2452247||||||3|1952||||| +4307|AAAAAAAADNABAAAA|525656|2992|30728|2452055|2452025|Ms.|Harriet|Bell|Y|2|3|1939|CAYMAN ISLANDS||Harriet.Bell@ORDxQNtmTj.org|2452405| +4308|AAAAAAAAENABAAAA|1050659|4126|29546|2450227|2450197|Ms.|Esther|Miller|Y|14|4|1952|CAMBODIA||Esther.Miller@T3OA6rIf.edu|2452595| +4309|AAAAAAAAFNABAAAA|390478|2574|27870|2450595|2450565|Miss|Marie|Sandlin|Y|27|5|1949|GHANA||Marie.Sandlin@XcGEZEDJ5Mi.edu|2452432| +4310|AAAAAAAAGNABAAAA|916152|6740|14081|2451296|2451266|Dr.|Anna|Patterson|Y|6|10|1978|BENIN||Anna.Patterson@eVzbVLCMsCZ61.org|2452585| +4311|AAAAAAAAHNABAAAA|445543|5593|26843|2450583|2450553|Dr.|Christopher|Randle|N|9|7|1930|JAPAN||Christopher.Randle@XHcVXP.com|2452596| +4312|AAAAAAAAINABAAAA|1495064|209|31001|2451233|2451203|Mr.|John|Hart|N|21|5|1931|VIRGIN ISLANDS, U.S.||John.Hart@2j.org|2452292| +4313|AAAAAAAAJNABAAAA|1158614|1451|25535|2451925|2451895|Mr.|Johnny|Clark|N|30|4|1944|CANADA||Johnny.Clark@XUZKm.edu|2452314| +4314|AAAAAAAAKNABAAAA|846976|753|1730|2451514|2451484|Mr.|Charles|Mcwilliams|Y|1|3|1966|ICELAND||Charles.Mcwilliams@FPO4nIQ7jy.org|2452587| +4315|AAAAAAAALNABAAAA|633155|4835|4562|2451647|2451617|Dr.|Francisco|Russell|N|20|8|1957|ESTONIA||Francisco.Russell@F1uIAEAUYsGACL.com|2452619| +4316|AAAAAAAAMNABAAAA|486942|6966|3437|2451210|2451180|Mrs.|Regina|Garrett|N|25|3|1943|HUNGARY||Regina.Garrett@Urs3XMsguhO03xc.com|2452332| +4317|AAAAAAAANNABAAAA|1264549|5792|15835|2450028|2449998|Sir|Christopher|Luster|N|26|2|1980|KAZAKHSTAN||Christopher.Luster@OJqQd27no1TddTSqH.org|2452568| +4318|AAAAAAAAONABAAAA|1647775|4384|15440||2451072|||Alexander||23|1|1980|PORTUGAL|||| +4319|AAAAAAAAPNABAAAA|1169294|620|36793|2450365|2450335|Mr.|Irving|Gilmore|Y|6|11|1937|GREENLAND||Irving.Gilmore@YSmUOjL6rLrQn.com|2452358| +4320|AAAAAAAAAOABAAAA|1365821|4452|33883|2451042|2451012|Mrs.|Esther|Shively|N|1|2|1944|BAHAMAS||Esther.Shively@kEIJn.edu|2452541| +4321|AAAAAAAABOABAAAA|414860|6233|6|2450208|2450178|Ms.|Theresa|Overstreet|Y|3|10|1969|ZIMBABWE||Theresa.Overstreet@T20uZnkNSpJ8ib0.org|2452301| +4322|AAAAAAAACOABAAAA|1274191|6652|19913|2451056|2451026|Ms.|Julie|Morris|Y|3|11|1981|ANTARCTICA||Julie.Morris@ksv9KmgX17Idgjx.edu|2452382| +4323|AAAAAAAADOABAAAA|449867|5165|27663|2449578|2449548|Dr.|Eileen|Morrill|N|24|12|1924|CZECH REPUBLIC||Eileen.Morrill@ZV.com|2452361| +4324|AAAAAAAAEOABAAAA|1477967|5906|26292|2450772|2450742|Sir|Christopher|Mcmanus|Y|8|5|1927|GHANA||Christopher.Mcmanus@Q.com|2452623| +4325|AAAAAAAAFOABAAAA|669012|2364|42516|2449480|2449450|Ms.|Jamila|Byrd|Y|16|12|1928|SLOVENIA||Jamila.Byrd@P3cZ0AjQJtM3hUFGc.com|2452448| +4326|AAAAAAAAGOABAAAA|1512981|4774|40567|2452454|2452424|Dr.|Jeffrey|Kelly|N|15|3|1987|JAPAN||Jeffrey.Kelly@B7x.com|2452570| +4327|AAAAAAAAHOABAAAA|1023303|1413|14723|2449593|2449563|Mrs.|Carrie|Sosa|Y|3|5|1975|PALAU||Carrie.Sosa@2ef4r0XcJ2.com|2452482| +4328|AAAAAAAAIOABAAAA|1812299|1742|42738|2450173|2450143|Ms.|Ida|Bruno|N|6|1|1937|PAKISTAN||Ida.Bruno@bUrRAx0.com|2452546| +4329|AAAAAAAAJOABAAAA|72544|3970|2549|2452650|2452620|Ms.|Rose|Workman|N|21|1|1928|BOTSWANA||Rose.Workman@Dta9gh1Kq3zLN8b.com|2452318| +4330|AAAAAAAAKOABAAAA|994185||29319|2452385||||||1|11|1925|R�UNION|||2452290| +4331|AAAAAAAALOABAAAA|1287631|2731|36231|2450064|2450034|Ms.|Albert|Her|N|21|12|1947|SAINT LUCIA||Albert.Her@LckA1XaiI6.org|2452334| +4332|AAAAAAAAMOABAAAA|1585349|5775|10154|2450498|2450468|Mr.|Rocky|Penn|Y|27|6|1967|YEMEN||Rocky.Penn@EB.com|2452580| +4333|AAAAAAAANOABAAAA|295220|2005|31190|2449815|2449785|Mr.|Tom|Burgess|Y|25|3|1929|URUGUAY||Tom.Burgess@OJLV.org|2452563| +4334|AAAAAAAAOOABAAAA|1609080|770|8907|2450368|2450338|Ms.|Trudy|Potter|N|20|2|1951|MARTINIQUE||Trudy.Potter@4tZlr.org|2452569| +4335|AAAAAAAAPOABAAAA|1750027|5330|32186|2451938|2451908|Sir|Eric|Saunders|Y|5|6|1966|MAURITIUS||Eric.Saunders@2ZaSAxLiY4d.com|2452585| +4336|AAAAAAAAAPABAAAA|730892|5706|37720|2450293|2450263|Mrs.|Shakira|Lopez|Y|12|2|1928|WESTERN SAHARA||Shakira.Lopez@BFsAJMHQbx.com|2452388| +4337|AAAAAAAABPABAAAA|103833|275|1870|2450080|2450050|Sir|Ivan|Hardy|N|5|1|1939|GUINEA-BISSAU||Ivan.Hardy@n7a90caKOGYKE.org|2452462| +4338|AAAAAAAACPABAAAA|1711933|1666|21846|2449149|2449119|Ms.|Eva|Coleman|N|19|8|1967|CUBA||Eva.Coleman@vyNby94bRBOVIBpnad7.edu|2452480| +4339|AAAAAAAADPABAAAA|524118|1611|15457|2449461|2449431|Ms.|Lisa|Phillips|Y|22|12|1941|FAROE ISLANDS||Lisa.Phillips@b6BOXQjHOv8.com|2452531| +4340|AAAAAAAAEPABAAAA|1416307|4991|12075|2451762|2451732|Dr.|Mark|Chambers|Y|9|4|1953|ALAND ISLANDS||Mark.Chambers@ii.edu|2452420| +4341|AAAAAAAAFPABAAAA|1102063||23106|2452015|||Ashley|||2||||||2452581| +4342|AAAAAAAAGPABAAAA|577135|2141|1947|2451884|2451854|Sir|Richard|Turner|Y|14|1|1979|AFGHANISTAN||Richard.Turner@llHZs12MSq.edu|2452359| +4343|AAAAAAAAHPABAAAA|1242791|1851|46464|2452496|2452466|Miss|Karla|Fraser|N|6|12|1935|MOLDOVA, REPUBLIC OF||Karla.Fraser@pYY.edu|2452604| +4344|AAAAAAAAIPABAAAA|22669||45240|||||||22||1959|ALGERIA|||2452490| +4345|AAAAAAAAJPABAAAA|678369|3593|43264|2452209|2452179|Dr.|Elizabeth|Land|Y|22|9|1926|INDONESIA||Elizabeth.Land@0T.com|2452396| +4346|AAAAAAAAKPABAAAA|879285|539|6640|2451661|2451631|Mr.|Marvin|Cox|N|5|8|1979|KUWAIT||Marvin.Cox@OBoexAs98Li0.com|2452379| +4347|AAAAAAAALPABAAAA|1369597|44|26091|2452293|2452263|Mr.|William|Lopez|Y|27|2|1974|CROATIA||William.Lopez@pdpb.org|2452351| +4348|AAAAAAAAMPABAAAA|764417|2160|40303|2449664|2449634|Sir|Wilbert|Powell|Y|16|10|1934|VIRGIN ISLANDS, U.S.||Wilbert.Powell@nfNk25l.com|2452551| +4349|AAAAAAAANPABAAAA|1547239|4624|26009|2451247|2451217|Dr.|Lisa|Chiu|Y|10|12|1945|MOZAMBIQUE||Lisa.Chiu@GRcXhPIbZ.org|2452295| +4350|AAAAAAAAOPABAAAA|839822|4103|1137|2449220|2449190|Ms.|Felicia|Escobedo|Y|11|1|1940|PERU||Felicia.Escobedo@ScEakR5KLaU.edu|2452525| +4351|AAAAAAAAPPABAAAA|367943|4463|10630|2452173|2452143|Ms.|Misty|Perkins|N|7|10|1944|ITALY||Misty.Perkins@IQsM.edu|2452571| +4352|AAAAAAAAAABBAAAA|1294040|7017|48441|2452538|2452508|Ms.|Jayme|Mcfarland|N|30|9|1955|VENEZUELA||Jayme.Mcfarland@PGDzUfgpy0.edu|2452332| +4353|AAAAAAAABABBAAAA|720332|4371|12857|2452659|2452629|Mr.|Michael|Mckenna|N|14|12|1950|LIECHTENSTEIN||Michael.Mckenna@ySK7e43jGSkl1t.org|2452488| +4354|AAAAAAAACABBAAAA|289680|4438|7149|2451080|2451050|Mrs.|Christi|Costello|N|3|6|1977|CAPE VERDE||Christi.Costello@SgD4n4Al5G.com|2452471| +4355|AAAAAAAADABBAAAA|576965|6781|14197|2449157|2449127|Dr.|Miguel|Miller|N|10|6|1942|ECUADOR||Miguel.Miller@0j0R6sgY.com|2452362| +4356|AAAAAAAAEABBAAAA|1771498|1932|42719|2451090|2451060|Sir|Paul|Ferguson|Y|14|4|1953|GUYANA||Paul.Ferguson@eG9j7d1iXi6.org|2452440| +4357|AAAAAAAAFABBAAAA|1853883|116|17196|2451359|2451329|Mr.|Nicholas|Brown|N|15|8|1970|PALAU||Nicholas.Brown@5pCgr.edu|2452638| +4358|AAAAAAAAGABBAAAA|349606|5761|1163|2449947|2449917|Miss|Ella|Davis|N|15|10|1979|ALAND ISLANDS||Ella.Davis@Q3XYKh3O.edu|2452427| +4359|AAAAAAAAHABBAAAA|424147|1783|25066|2449200|2449170|Dr.|Lamar|Leonard|N|21|11|1964|AMERICAN SAMOA||Lamar.Leonard@bThEq.edu|2452379| +4360|AAAAAAAAIABBAAAA|150503|6005|21154|2450482|2450452|Sir|William|Larkin|N|20|12|1979|GUAM||William.Larkin@ogOoiDcrrbN.com|2452581| +4361|AAAAAAAAJABBAAAA||1082|11592||2450226|Mr.|||Y|27||1953|||Johnny.Ramirez@8kH30SKR.edu|| +4362|AAAAAAAAKABBAAAA|1548207|4595|40983|2449144|2449114|Mrs.|Debra|Winters|Y|29|5|1992|BULGARIA||Debra.Winters@bOn3IVvmVUmE.edu|2452422| +4363|AAAAAAAALABBAAAA|895700|840|46733|2450279|2450249|Sir|Tyler|Dodson|Y|28|5|1955|SLOVENIA||Tyler.Dodson@jZaAZVM.org|2452473| +4364|AAAAAAAAMABBAAAA|683880|6029|44178|2451069|2451039|Dr.|John|Butler|Y|11|7|1929|CANADA||John.Butler@Vz4UYojOBGdFEE.edu|2452297| +4365|AAAAAAAANABBAAAA|90925|3068|24411|2451882|2451852|Dr.|Laura|Maddox|Y|15|11|1956|SWEDEN||Laura.Maddox@vrbXQMM6HV1.com|2452635| +4366|AAAAAAAAOABBAAAA|577130|39|21348|2452389|2452359|Dr.|Frank|Lewis|N|29|3|1976|BELARUS||Frank.Lewis@xCVeV2B6kbxJBi.com|2452507| +4367|AAAAAAAAPABBAAAA|905166|4027|49335|2449309|2449279|Ms.|Flossie|Adams|N|1|2|1933|WALLIS AND FUTUNA||Flossie.Adams@bmmMg9Gu4POV0p.edu|2452600| +4368|AAAAAAAAABBBAAAA|129464|2651|17438|2451556|2451526|Sir|Vincent|Albright|N|14|5|1988|BURUNDI||Vincent.Albright@rN3ozCdXMj1KiC.com|2452310| +4369|AAAAAAAABBBBAAAA|414363|4075|33340|2452032|2452002|Miss|Ann|Fleming|N|22|1|1955|KENYA||Ann.Fleming@h3I0uSujB.com|2452322| +4370|AAAAAAAACBBBAAAA||6402|33600|2449593|2449563|Dr.|||||4|1988|UGANDA|||2452374| +4371|AAAAAAAADBBBAAAA|630654|672|10690|2450764|2450734|Miss|Grace|Walker|Y|24|3|1984|R�UNION||Grace.Walker@g1VixYKc9zb.org|2452459| +4372|AAAAAAAAEBBBAAAA|1201506|645|18984|2451289|2451259|Mrs.|Melanie|Duke|Y|15|3|1956|TOKELAU||Melanie.Duke@zuIgs0FLp0SV.edu|2452561| +4373|AAAAAAAAFBBBAAAA|1083338|2638|30338|2449653|2449623|Ms.|Diane|Swenson|N|24|5|1950|SWAZILAND||Diane.Swenson@lOfQvC4Gajk162bv.com|2452506| +4374|AAAAAAAAGBBBAAAA|471769|5221|1120|2451463|2451433|Mrs.|Mia|Johnson|N|6|7|1959|BOLIVIA||Mia.Johnson@92ss.org|2452296| +4375|AAAAAAAAHBBBAAAA|1143641|7012|15183|2449718|2449688|Ms.|Ida|Simmons|Y|16|2|1962|SWAZILAND||Ida.Simmons@dsRROjCzZzA9JpH.com|2452364| +4376|AAAAAAAAIBBBAAAA|84944|6701|38058|2449100|2449070|Mr.|Maurice|Rodriguez|N|||1966||||| +4377|AAAAAAAAJBBBAAAA|1580171|3828|17932|2451060|2451030|Dr.|Joseph|Gregg|N|8|2|1972|NIGER||Joseph.Gregg@8QRVd2yTRZ8177y.edu|2452363| +4378|AAAAAAAAKBBBAAAA|475763|5347|32178|2451400|2451370|Mr.|Casey|Hall|N|9|1|1952|SIERRA LEONE||Casey.Hall@yO9Yc.edu|2452364| +4379|AAAAAAAALBBBAAAA|1074400|286|33772|2451619|2451589|Miss|Albert|Chamberlain|Y|22|6|1966|NIUE||Albert.Chamberlain@7icOhTVMcE.org|2452531| +4380|AAAAAAAAMBBBAAAA|653173|4580|25987|2451351|2451321|Mrs.|Donna|Newton|N|9|9|1955|MOZAMBIQUE||Donna.Newton@NHRYmCvj0cQ3qJ44HS.com|2452511| +4381|AAAAAAAANBBBAAAA|1530362|6801|25593|2451112|2451082|Mrs.|Sarah|Duncan|Y|6|2|1927|SOUTH AFRICA||Sarah.Duncan@gqGkAjqJqrh.com|2452508| +4382|AAAAAAAAOBBBAAAA|1852622|6970|16518|2452427|2452397|Mrs.|Linda|Harry|Y|24|5|1956|GUYANA||Linda.Harry@4F.com|2452518| +4383|AAAAAAAAPBBBAAAA|688440|2769|3498|2451744|2451714|Sir|Thomas|Bailey|Y|11|9|1959|ANGUILLA||Thomas.Bailey@ThPC7Pod969.edu|2452325| +4384|AAAAAAAAACBBAAAA|1153371|1125|44079|2452136|2452106|Dr.|Jason|Sharkey|N|4|9|1980|GUAM||Jason.Sharkey@nRSs9.com|2452607| +4385|AAAAAAAABCBBAAAA|327000|1922|32044|2452325|2452295|Dr.|Brenda|Sherry|Y|4|3|1966|SERBIA||Brenda.Sherry@8xLrrrH0ka.org|2452480| +4386|AAAAAAAACCBBAAAA|553358|4468|41625|2451425|2451395|Mrs.|Deborah|Shirley|Y|5|7|1985|TUVALU||Deborah.Shirley@rT4.com|2452431| +4387|AAAAAAAADCBBAAAA|805250|18|6786|2450251|2450221|Mrs.|Judy|Mcdonald|N|18|1|1963|ISRAEL||Judy.Mcdonald@0slSOsbLEXQ.edu|2452508| +4388|AAAAAAAAECBBAAAA|378127|4790|15806|2450881|2450851|Miss|Evangeline|Ocasio|Y|1|7|1991|SERBIA||Evangeline.Ocasio@ONRaEe7hFIkEtfyx.com|2452456| +4389|AAAAAAAAFCBBAAAA|1721520|1300|27433|2451316|2451286|Ms.|Elizabeth|Thomas|Y|9|11|1975|HONDURAS||Elizabeth.Thomas@pAVMN.org|2452287| +4390|AAAAAAAAGCBBAAAA|1415662|1359|47056|||Dr.|Edwin|Simons|N|6|10||BHUTAN|||2452537| +4391|AAAAAAAAHCBBAAAA|452711|4936|42361|2451500|2451470|Dr.|Terrie|Morse|Y|18|1|1989|MONTENEGRO||Terrie.Morse@XT0Ra8E33y.com|2452360| +4392|AAAAAAAAICBBAAAA|1591913|190|24467|2452616|2452586|Sir|Richard|Elam|N|15|8|1970|ERITREA||Richard.Elam@uLEY8YU.org|2452462| +4393|AAAAAAAAJCBBAAAA|1770497|3362|2674|2452058|2452028|Dr.|Steven|Michel|N|1|9|1964|SEYCHELLES||Steven.Michel@Kx5qXBd1k.org|2452353| +4394|AAAAAAAAKCBBAAAA|914292|4936|41580|2451316|2451286|Mr.|Carroll|Hamilton|Y|22|9|1926|NIGER||Carroll.Hamilton@hUzPaCV8JT7H1Y.com|2452400| +4395|AAAAAAAALCBBAAAA|976622|59|46649|2451757|2451727|Miss|Naomi|Viera|N|25|1|1932|CYPRUS||Naomi.Viera@OaLB0aC.org|2452411| +4396|AAAAAAAAMCBBAAAA|1210526|1453|46780|2450297|2450267|Mrs.|Michelle|Hill|Y|4|6|1982|TURKEY||Michelle.Hill@jdAuvl4R5u0FRSUM.org|2452472| +4397|AAAAAAAANCBBAAAA|1233492|4686|37480|2452589|2452559|Sir|Luis|Lopez|N|16|5|1952|BELARUS||Luis.Lopez@kcFIBjIFQRC.com|2452621| +4398|AAAAAAAAOCBBAAAA|979717|5809|15454|2452653|2452623|Mrs.|Sharon|Walker|N|29|11|1945|MEXICO||Sharon.Walker@AZdjRTfeE.com|2452522| +4399|AAAAAAAAPCBBAAAA|1656838|5844|7203|2451086|2451056|Mr.|Justin|Haney|Y|21|6|1947|MONGOLIA||Justin.Haney@MiJ8a0ySBFELv.org|2452414| +4400|AAAAAAAAADBBAAAA|837835|6431|39027|2450076|2450046|Ms.|Betty|Dunne|Y|3|3|1989|PORTUGAL||Betty.Dunne@znGyrKjMhy9tAq.edu|2452479| +4401|AAAAAAAABDBBAAAA|314362|6481|3800|2451694|2451664|Mr.|Mike|Brown|N|28|5|1976|GUAM||Mike.Brown@k3XiSBOEgm9nVo.org|2452415| +4402|AAAAAAAACDBBAAAA|500984|6432|27527|2451486|2451456|Dr.|Anthony|Ramsey|Y|17|8|1929|MALAYSIA||Anthony.Ramsey@afXGpa8lGi7Ozn.edu|2452382| +4403|AAAAAAAADDBBAAAA|1349413|6114|4368|2449500|2449470|Dr.|Thomas|Garner|N|6|12|1981|KENYA||Thomas.Garner@K.com|2452414| +4404|AAAAAAAAEDBBAAAA|1718520|1930|18140|2451709|2451679|Sir|Jose|Walker|Y|21|2|1988|LATVIA||Jose.Walker@thTKKt5yb3KYo.edu|2452505| +4405|AAAAAAAAFDBBAAAA|120206|4901|46796|2452607|2452577|Dr.|James|Wills|N|6|4|1950|NICARAGUA||James.Wills@a2X7M1kN2srbx.org|2452307| +4406|AAAAAAAAGDBBAAAA|848774|5414|8501|2450117|2450087|Dr.|Kenneth|Phillips|N|18|5|1958|ALBANIA||Kenneth.Phillips@vfjpX8.com|2452590| +4407|AAAAAAAAHDBBAAAA|748958|4337|20069|2450683|2450653|Dr.|Brenda|White|N|14|2|1961|MYANMAR||Brenda.White@0OnMovxoROrs.org|2452470| +4408|AAAAAAAAIDBBAAAA|571665|5216|11967|2452614|2452584|Mr.|Frank|Wright|Y|16|10|1930|KUWAIT||Frank.Wright@jXOiJg8s7.org|2452438| +4409|AAAAAAAAJDBBAAAA||360|39278|2449732|2449702|Dr.|||Y|12|5|1961|DENMARK||Gillian.Gulley@rfXe3j9pvf72nCyKl.com|2452420| +4410|AAAAAAAAKDBBAAAA|1831857|4337|43617|2452545|2452515|Miss|Sarah|Martin|Y|11|1|1959|MACAO||Sarah.Martin@4DCiDZEXJET.com|2452500| +4411|AAAAAAAALDBBAAAA|1204984|7187|1790|2451417|2451387|Sir|Alan|Newell|N|9|8|1929|TURKEY||Alan.Newell@rJD5k7H5f.org|2452595| +4412|AAAAAAAAMDBBAAAA|1071718|5022|41926|2451839|2451809|Sir|Eliseo|Mcguire|N|25|11|1939|KUWAIT||Eliseo.Mcguire@BaKBcEo2e.org|2452499| +4413|AAAAAAAANDBBAAAA|234528|238|11548|2449418|2449388|Dr.|Thomas|Patterson|N|6|10|1967|MOROCCO||Thomas.Patterson@UvPoHMG7A.edu|2452316| +4414|AAAAAAAAODBBAAAA|||35409|||Mr.|Jeffrey|Calvert||2|||JORDAN|||2452608| +4415|AAAAAAAAPDBBAAAA|1163167|5848|15729|2450070|2450040|Mr.|Dannie|Wooten|Y|21|8|1945|KIRIBATI||Dannie.Wooten@D5RVr.com|2452411| +4416|AAAAAAAAAEBBAAAA|824051|2441|32078|2450180|||Christine|Engel||30||1973|COSTA RICA|||2452293| +4417|AAAAAAAABEBBAAAA|335046|2667|31535|2449164|2449134|Mrs.|Gloria|Boynton|Y|24|5|1981|HAITI||Gloria.Boynton@QTDlQmZzxdq.org|2452588| +4418|AAAAAAAACEBBAAAA|350259|7062|17865|2449645|2449615|Miss|Kathy|Booker|N|16|10|1962|TONGA||Kathy.Booker@qjX7AFKY641.org|2452518| +4419|AAAAAAAADEBBAAAA|115966|6610|19254|2450822|2450792|Miss|Nora|Baker|N|12|8|1981|SINGAPORE||Nora.Baker@v2d6pyfzr.edu|2452507| +4420|AAAAAAAAEEBBAAAA|1219666|6506|26899|2450220|2450190|Dr.|Justin|Mcguire|Y|25|9|1926|PAKISTAN||Justin.Mcguire@hC.org|2452350| +4421|AAAAAAAAFEBBAAAA|73824|5296|39121|2449949|2449919|Mrs.|Lena|Williams|Y|4|6|1929|NEW ZEALAND||Lena.Williams@uHQqYG5Sk4b.com|2452313| +4422|AAAAAAAAGEBBAAAA|1477680|2493|45706|2449384|2449354|Dr.|Jason|Holmes|N|23|1|1932|BAHRAIN||Jason.Holmes@U.org|2452564| +4423|AAAAAAAAHEBBAAAA|957438|6153|13268|2451282|2451252|Sir|John|Young|N|17|3|1962|SOLOMON ISLANDS||John.Young@CTQVbm1Y5azmLnp.edu|2452365| +4424|AAAAAAAAIEBBAAAA|||12228|2451736|||Preston|Collins|N|25|3||SRI LANKA||Preston.Collins@VOgkPLE3NcK.edu|2452533| +4425|AAAAAAAAJEBBAAAA|946099|4537|23375|2451590|2451560|Mrs.|Hanna|Hurd|Y|20|4|1959|DJIBOUTI||Hanna.Hurd@b6eEzXXnrNqStk.com|2452556| +4426|AAAAAAAAKEBBAAAA|286514|4765|41063|2452546|2452516|Mr.|Earl|Butler|Y|18|1|1977|FAROE ISLANDS||Earl.Butler@UB.org|2452442| +4427|AAAAAAAALEBBAAAA|343218|2862|27753|2450184|2450154|Ms.|Rachel|Nava|N|11|12|1943|CHILE||Rachel.Nava@j709ftj4u77A.edu|2452641| +4428|AAAAAAAAMEBBAAAA|83259|5244|26429|2452311|2452281|Sir|Shawn|Rodriguez|Y|1|6|1963|BELARUS||Shawn.Rodriguez@tut.edu|2452607| +4429|AAAAAAAANEBBAAAA|987273|6939|1758|2450897|2450867|Miss|Lisa|Forsyth|Y|15|12|1943|PERU||Lisa.Forsyth@usCpT.edu|2452593| +4430|AAAAAAAAOEBBAAAA|1322168|5861|2707|2451595|2451565|Ms.|Doris|Stafford|N|2|7|1926|ETHIOPIA||Doris.Stafford@QMjv49.edu|2452608| +4431|AAAAAAAAPEBBAAAA|73418|3411|45051|2450769|2450739|Dr.|Christopher|Stanley|Y|8|7|1956|BOLIVIA||Christopher.Stanley@TcJD2Sutba.edu|2452367| +4432|AAAAAAAAAFBBAAAA|941447|7005|34610|2452401|2452371|Mr.|Ervin|Ferguson|N|7|11|1981|GERMANY||Ervin.Ferguson@sgbeR4HbSP.org|2452367| +4433|AAAAAAAABFBBAAAA|141720|2572|45424|2452658|2452628|Ms.|Helena|Whaley|N|3|4|1965|BENIN||Helena.Whaley@HyrodRpz9.org|2452465| +4434|AAAAAAAACFBBAAAA|83725|1709|13693|2450589|2450559|Sir|Daniel|Barnett|N|27|3|1992|NORWAY||Daniel.Barnett@976tUJySiD8.com|2452408| +4435|AAAAAAAADFBBAAAA|1902268|1575|12835|2451848|2451818|Miss|Mildred|Rice|Y|19|4|1967|CAYMAN ISLANDS||Mildred.Rice@68UOrFLhXB6c1VU0Ug.edu|2452344| +4436|AAAAAAAAEFBBAAAA|1335158|2911|15864|2451514|2451484|Sir|James|Mclean|N|20|1|1954|EL SALVADOR||James.Mclean@c4nIL2Yl9Nmqje0dvuQ.org|2452384| +4437|AAAAAAAAFFBBAAAA|633429|614|35799|2450460|2450430|Dr.|Amy|Peeler|N|16|12|1952|SRI LANKA||Amy.Peeler@6Rmpxvlc3pDCAI.edu|2452348| +4438|AAAAAAAAGFBBAAAA|1759253|4119|38538|2452409|2452379|Mrs.|Alva|Reynolds|Y|25|9|1944|SINGAPORE||Alva.Reynolds@sHrRZJY.edu|2452506| +4439|AAAAAAAAHFBBAAAA|1858992|5867|6389|2450011|2449981|Ms.|Phyllis|Evans|Y|5|8|1980|ALGERIA||Phyllis.Evans@McjiFc.edu|2452291| +4440|AAAAAAAAIFBBAAAA|1105183|1156|47129|2452669|2452639|Dr.|David|Smith|Y|9|2|1979|INDONESIA||David.Smith@JAVG2KBhCHuO.com|2452444| +4441|AAAAAAAAJFBBAAAA|1194564||14284|2451735|||||N|||||||2452522| +4442|AAAAAAAAKFBBAAAA|552600|4928|1703|2451761|2451731|Mr.|David|Brown|N|7|8|1977|ZAMBIA||David.Brown@EaQZ4iYIihjlp.org|2452543| +4443|AAAAAAAALFBBAAAA|418631|3608|16185|2451756|2451726|Dr.|Mae|Winston|N|20|8|1928|OMAN||Mae.Winston@nzK.com|2452335| +4444|AAAAAAAAMFBBAAAA|1516611|3285|26759|2449914|2449884|Ms.|Sara|Deleon|N|23|11|1973|GABON||Sara.Deleon@pAC3j9rT2FEvfNq.com|2452598| +4445|AAAAAAAANFBBAAAA|1696848|4479|32910|2451934|2451904|Sir|Reginald|White|N|13|3|1937|BHUTAN||Reginald.White@zzu2GXhU.com|2452624| +4446|AAAAAAAAOFBBAAAA|1628636|6376|42163|2451481|2451451|Miss|Lourdes|Mccann|Y|30|8|1983|GUYANA||Lourdes.Mccann@A6o8da.org|2452344| +4447|AAAAAAAAPFBBAAAA|946664|5455|44293|2451356|2451326|Dr.|Jeff|Douglas|N|8|12|1978|CHRISTMAS ISLAND||Jeff.Douglas@ugnln5aTZk47.org|2452580| +4448|AAAAAAAAAGBBAAAA|1848033|4747|7861|2452280|2452250|Dr.|Mark|Jones|N|6|1|1981|PORTUGAL||Mark.Jones@aGYaOf9IRJZG.org|2452399| +4449|AAAAAAAABGBBAAAA|1271616|6800|45231|2451509|2451479|Dr.|Markus|Wilson|Y|9|1|1931|NEPAL||Markus.Wilson@7.edu|2452446| +4450|AAAAAAAACGBBAAAA|1065575|4462|33349|2451263|2451233|Sir|Calvin|Smith|N|30|4|1971|FINLAND||Calvin.Smith@IVGqteE.org|2452331| +4451|AAAAAAAADGBBAAAA|1890096|1614|43132|2450089|2450059|Mrs.|Christine|Pierce|N|15|10|1988|JORDAN||Christine.Pierce@heMq.org|2452322| +4452|AAAAAAAAEGBBAAAA|127473|4191|8462|2449907|2449877|Mrs.|Alicia|Gillespie|Y|7|3|1927|HUNGARY||Alicia.Gillespie@Cz.org|2452510| +4453|AAAAAAAAFGBBAAAA|1427780|2718|43209|2452464|2452434|Sir|Paul|Bynum|Y|30|9|1945|MACAO||Paul.Bynum@nUMRQcpB2xD.com|2452601| +4454|AAAAAAAAGGBBAAAA|594248|3371|36094|2450225|2450195|Dr.|Christopher|Myers|Y|30|7|1942|HUNGARY||Christopher.Myers@mpTcSNFdJ.com|2452430| +4455|AAAAAAAAHGBBAAAA|651759|3289|17172|2449486|2449456|Mr.|Gregory|Long|Y|23|4|1979|SWITZERLAND||Gregory.Long@52oGESrxlJpuJkB.com|2452344| +4456|AAAAAAAAIGBBAAAA|250803|3706|42209|2451044|2451014|Sir|Ernest|Crawford|Y|27|6|1954|ANTIGUA AND BARBUDA||Ernest.Crawford@GnPbX9pXoYrt.com|2452545| +4457|AAAAAAAAJGBBAAAA|1249959|6298|36237|2450378|2450348|Sir|Robert|Morrison|Y|28|1|1977|MONGOLIA||Robert.Morrison@sZO4zFajTc77g.org|2452354| +4458|AAAAAAAAKGBBAAAA|1204621|4557|18493|2452656|2452626|Sir|Thomas|Hicks|Y|27|9|1954|NAMIBIA||Thomas.Hicks@6Q.edu|2452433| +4459|AAAAAAAALGBBAAAA|719622|2946|10670|2450277|2450247|Dr.|Thomas|Washington|Y|1|2|1929|ALAND ISLANDS||Thomas.Washington@OB.com|2452517| +4460|AAAAAAAAMGBBAAAA|129303|712|46168|2449067|2449037|Dr.|Brian|Heard|N|10|12|1960|CHRISTMAS ISLAND||Brian.Heard@kxyTSGFI7I.edu|2452407| +4461|AAAAAAAANGBBAAAA|||23551|||Ms.||Rose||28|||FRENCH GUIANA|||| +4462|AAAAAAAAOGBBAAAA|1146999|4521|25643|2450281|2450251|Mrs.|Gwendolyn|Mcclain|Y|9|12|1946|RUSSIAN FEDERATION||Gwendolyn.Mcclain@YVsH.edu|2452330| +4463|AAAAAAAAPGBBAAAA|1617949|575|14048|2449709|2449679|Ms.|Bessie|Joseph|Y|25|4|1975|BURUNDI||Bessie.Joseph@Z93NU8cPTa.com|2452374| +4464|AAAAAAAAAHBBAAAA|756809|4135|39061|2452625|2452595|Dr.|Lois|Fortune|Y|17|3|1984|BOUVET ISLAND||Lois.Fortune@aA7R3AYI7.edu|2452440| +4465|AAAAAAAABHBBAAAA|1215638|6167|24343|2451234|2451204|Miss|Beverly|Myers|Y|30|11|1957|PAKISTAN||Beverly.Myers@fY.org|2452588| +4466|AAAAAAAACHBBAAAA|599632|1084|16089|2449780|2449750|Mrs.|Julia|Sawyer|N|2|7|1936|UGANDA||Julia.Sawyer@x.com|2452642| +4467|AAAAAAAADHBBAAAA|311452|5508|17517|2451910|2451880|Dr.|Yvette|Bender|Y|20|7|1979|TAJIKISTAN||Yvette.Bender@bzItBvM8xl21A.edu|2452489| +4468|AAAAAAAAEHBBAAAA|918093|5437|44734|2449652|2449622|Dr.|Barbara|Finley|N|13|6|1971|PALAU||Barbara.Finley@Dds9JZ.com|2452304| +4469|AAAAAAAAFHBBAAAA|436908|2617|20373|2449339|2449309|Dr.|Daniel|Pierson|Y|12|3|1929|TURKEY||Daniel.Pierson@Zc3c0FxpEy.com|2452369| +4470|AAAAAAAAGHBBAAAA|1702634|4577|16967|2452472|2452442|Mr.|Jesus|Martin|Y|8|7|1931|CANADA||Jesus.Martin@NVk.org|2452321| +4471|AAAAAAAAHHBBAAAA|309542|6569|25196|2449094|2449064|Dr.|Ruben|Morgan|N|18|12|1927|KAZAKHSTAN||Ruben.Morgan@Frh0i4Hu.org|2452554| +4472|AAAAAAAAIHBBAAAA|424014|4445|40523|2451626|2451596|Sir|Louis|Williams|N|4|12|1928|URUGUAY||Louis.Williams@BuAza.edu|2452456| +4473|AAAAAAAAJHBBAAAA|1908673|3063|6653|2451545|2451515|Dr.|Jerome|Martin|N|17|8|1952|GIBRALTAR||Jerome.Martin@otXC7c3I.edu|2452453| +4474|AAAAAAAAKHBBAAAA|246480|6929|30225|2451399|2451369|Dr.|Margaret|Aleman|N|2|11|1983|SEYCHELLES||Margaret.Aleman@yUKPC5.edu|2452525| +4475|AAAAAAAALHBBAAAA|168772|1773|35574|2449284|2449254|Mrs.|Stephanie|Forrester|Y|16|10|1958|PORTUGAL||Stephanie.Forrester@mMmAE6QReTo1.edu|2452307| +4476|AAAAAAAAMHBBAAAA|675199|3368|23137|2451868|2451838|Sir|Brian|Ryan|N|8|2|1968|ANTARCTICA||Brian.Ryan@MgzHACxdTtnu.com|2452287| +4477|AAAAAAAANHBBAAAA|1344484|4718|30552|2450721|2450691|Dr.|Pedro|Rowell|N|30|3|1973|TUNISIA||Pedro.Rowell@OjasiNgZ.org|2452647| +4478|AAAAAAAAOHBBAAAA|880611|2714|13170|2450473|2450443|Dr.|Stephen|Levine|Y|17|8|1965|TRINIDAD AND TOBAGO||Stephen.Levine@Qo.org|2452339| +4479|AAAAAAAAPHBBAAAA|1843083|1143|10803|2452466|2452436|Mr.|Duane|Carder|Y|14|10|1989|KIRIBATI||Duane.Carder@e2Hi0HhmOv.org|2452468| +4480|AAAAAAAAAIBBAAAA|1274962|5019|20723|2451198|2451168|Miss|Anne|Carroll|Y|13|12|1949|YEMEN||Anne.Carroll@cbgvROKOjD.edu|2452430| +4481|AAAAAAAABIBBAAAA|685619|336|5900|2451930|2451900|Mr.|Larry|Crum|Y|24|7|1942|HAITI||Larry.Crum@iqfLh.org|2452451| +4482|AAAAAAAACIBBAAAA|1826380|5971|7962|2450714|2450684|Sir|Samuel|Thomason|Y|3|2|1987|JERSEY||Samuel.Thomason@GY8.edu|2452418| +4483|AAAAAAAADIBBAAAA|67496|6313|49177|2449588|2449558|Mrs.|Bessie|Rogers|Y|26|4|1987|NAURU||Bessie.Rogers@d9HeV7C.com|2452373| +4484|AAAAAAAAEIBBAAAA|65101|6110|33119|2450381|2450351|Dr.|Joseph|White|Y|18|4|1958|QATAR||Joseph.White@7axp.com|2452571| +4485|AAAAAAAAFIBBAAAA|1389583|952|48368|2450588|2450558|Mrs.|Eleanor|Snider|Y|19|5|1946|GABON||Eleanor.Snider@HylY0slvtjISn96mS.edu|2452305| +4486|AAAAAAAAGIBBAAAA|1665699|732|17365|2451691|2451661|Miss|Claudette|Harris|Y|29|3|1945|AUSTRIA||Claudette.Harris@GTIzpSHhSeuxD.org|2452388| +4487|AAAAAAAAHIBBAAAA|1840484|614|45897|2449341||Dr.|||N|||1933|||Pat.Gardner@3XV.com|2452483| +4488|AAAAAAAAIIBBAAAA|79303|2434|28412|2450521|2450491|Ms.|Ashley|Clark|N|4|5|1969|RUSSIAN FEDERATION||Ashley.Clark@08S5UA1sd1Ud.org|2452486| +4489|AAAAAAAAJIBBAAAA|267788|3212|7836|2449172|2449142|Miss|Rhonda|Ocasio|N|7|9|1973|PAPUA NEW GUINEA||Rhonda.Ocasio@8s4dZ32zdimSF.edu|2452345| +4490|AAAAAAAAKIBBAAAA|1166406|6758|6663|2451120|2451090|Dr.|Joseph|Bolton|Y|3|1|1928|NORFOLK ISLAND||Joseph.Bolton@d21NL.edu|2452307| +4491|AAAAAAAALIBBAAAA|439577|5758|43986|2449896|2449866|Mr.|Michael|Malone|Y|25|1|1961|SAINT LUCIA||Michael.Malone@LNjGA6.edu|2452417| +4492|AAAAAAAAMIBBAAAA|1119657|3339|39021|2451019|2450989|Sir|Jay|Wheatley|N|18|9|1926|FIJI||Jay.Wheatley@xI2K2b2dM.org|2452572| +4493|AAAAAAAANIBBAAAA||4892|40515|2451204||Miss|Elizabeth||N|19|8|1950||||| +4494|AAAAAAAAOIBBAAAA|1905086|1606|13326|2449852|2449822|Mr.|Carl|James|N|13|10|1932|PORTUGAL||Carl.James@JxpcstusJ5IVI.com|2452349| +4495|AAAAAAAAPIBBAAAA|1214235|1547|9212|2449666|2449636|Ms.|Arthur|Sager|N|2|7|1930|PALAU||Arthur.Sager@NoOu8L5MredLqCrQ.edu|2452641| +4496|AAAAAAAAAJBBAAAA|1349558|5673|7388|2451332|2451302|Ms.|Jana|Anderson|N|30|11|1944|NIUE||Jana.Anderson@6vJVtuhE.org|2452646| +4497|AAAAAAAABJBBAAAA|382193|1708|35223|2449799|2449769|Dr.|Robert|Hanson|Y|1|5|1929|CUBA||Robert.Hanson@1osJs0u21FRPLgB3.com|2452364| +4498|AAAAAAAACJBBAAAA|184473|2570|24267|2451114|2451084|Dr.|Anna|Williams|N|12|6|1990|MEXICO||Anna.Williams@5pVCGv5.edu|2452342| +4499|AAAAAAAADJBBAAAA|893925|4129|33180|2451986|2451956|Ms.|Victoria|Parker|Y|11|1|1968|MARSHALL ISLANDS||Victoria.Parker@INJu8Jhy7OPGzG3B.com|2452469| +4500|AAAAAAAAEJBBAAAA|1396221|2086|1504|2449440|2449410|Mr.|Joe|Welch|N|9|3|1932|FIJI||Joe.Welch@mG47.com|2452285| +4501|AAAAAAAAFJBBAAAA|432382|6275|3415|2450861|2450831|Miss|Jenny|Whitlow|N|7|3|1925|CANADA||Jenny.Whitlow@e0l3I2.com|2452583| +4502|AAAAAAAAGJBBAAAA|1499519|1902|41932|2452182|2452152|Dr.|Frank|Vasquez|Y|26|11|1926|HONG KONG||Frank.Vasquez@N.edu|2452365| +4503|AAAAAAAAHJBBAAAA|120922|6485|11295|2451954|2451924|Dr.|Justin|Johnson|N|19|5|1926|COMOROS||Justin.Johnson@8PIZXXRlH.com|2452482| +4504|AAAAAAAAIJBBAAAA|856403|3588|23620|2449926|2449896|Mr.|Nicholas|Swafford|Y|30|12|1984|BELARUS||Nicholas.Swafford@kf.edu|2452554| +4505|AAAAAAAAJJBBAAAA|974925|715|29705|2452232|2452202|Sir|John|Perkins|Y|20|3|1938|KYRGYZSTAN||John.Perkins@GT64VsAda7fs.com|2452305| +4506|AAAAAAAAKJBBAAAA|1746138|996|29854|2450472|2450442|Mr.|Milton|Rich|N|26|11|1970|CUBA||Milton.Rich@rn4jkbTNGT0T.org|2452475| +4507|AAAAAAAALJBBAAAA|594945|636|46280|2450856|2450826|Sir|Eloy|Miller|Y|22|10|1952|ECUADOR||Eloy.Miller@kHxd7y3Co.org|2452355| +4508|AAAAAAAAMJBBAAAA|626859|3995|23131|2451223|2451193|Miss|Jennifer|Porter|Y|16|3|1953|MAURITANIA||Jennifer.Porter@yyeDaegfYLJ.org|2452437| +4509|AAAAAAAANJBBAAAA|430927|4843|31159|2449841|2449811|Miss|Amanda|Nelson|N|20|7|1949|TOKELAU||Amanda.Nelson@cu.edu|2452400| +4510|AAAAAAAAOJBBAAAA|1210025|6186|23994|2450499|2450469|Mrs.|Beatrice|Garrett|Y|18|11|1947|EL SALVADOR||Beatrice.Garrett@p1bpnrXeCs.edu|2452398| +4511|AAAAAAAAPJBBAAAA|882524|5070|14863|2451687|2451657|Mr.|Larry|Hill|N|9|6|1933|BOUVET ISLAND||Larry.Hill@ntQnupa.edu|2452601| +4512|AAAAAAAAAKBBAAAA|1775447|1452|38704|2451360|2451330|Sir|Frederick|Pacheco|N|18|1|1975|GUYANA||Frederick.Pacheco@q.com|2452478| +4513|AAAAAAAABKBBAAAA|168439|3005|14018|2450926|2450896|Mrs.|Bernice|Green|N|8|12|1930|UGANDA||Bernice.Green@CmUvZkY.org|2452391| +4514|AAAAAAAACKBBAAAA|1234941|1606|12671|2452617|2452587|Dr.|Thomas|Shook|N|14|1|1959|LIBERIA||Thomas.Shook@bMBG6ta1AtfnI.edu|2452535| +4515|AAAAAAAADKBBAAAA|1797245|5094|13790|2449848|2449818|Dr.|Jose|Cutler|Y|23|4|1954|CAPE VERDE||Jose.Cutler@fUzzrnpi40yvrE.com|2452514| +4516|AAAAAAAAEKBBAAAA|1441046|3253|23507|2452274|2452244|Miss|Stephanie|Herron|Y|28|10|1929|MALI||Stephanie.Herron@fQ.org|2452380| +4517|AAAAAAAAFKBBAAAA|219149|1327|22305|2450645|2450615|Mr.|Brendan|Goodman|N|6|7|1941|SAINT HELENA||Brendan.Goodman@P7DHVXEFHd.edu|2452304| +4518|AAAAAAAAGKBBAAAA|1701386|2797|18561|2451291|2451261|Miss|Ashley|Bell|N|15|3|1949|SYRIAN ARAB REPUBLIC||Ashley.Bell@qjs4hHPOfoo3.com|2452596| +4519|AAAAAAAAHKBBAAAA|353235|920|1805|2451966|2451936|Ms.|Linda|Crawford|N|13|2|1924|KIRIBATI||Linda.Crawford@xOf5kYDZapQmj.org|2452598| +4520|AAAAAAAAIKBBAAAA|1835293|5204|47468|2450585|2450555|Sir|Richard|Zavala|N|1|1|1938|WALLIS AND FUTUNA||Richard.Zavala@gOV0cqtrjkfzMHzbSTK.com|2452643| +4521|AAAAAAAAJKBBAAAA|590248|5304|30360|2452204|2452174|Mrs.|Deborah|Stanton|Y|3|4|1960|CZECH REPUBLIC||Deborah.Stanton@QAx.com|2452463| +4522|AAAAAAAAKKBBAAAA|543401|3132|42463|2450554|2450524|Dr.|Kent|Price|N|5|11|1955|ERITREA||Kent.Price@JGaOijngGY.org|2452514| +4523|AAAAAAAALKBBAAAA|1154190|1546|23302|2451739|2451709|Mrs.|Della|Burton|N|20|1|1937|MONTENEGRO||Della.Burton@KbesBR43R2.org|2452430| +4524|AAAAAAAAMKBBAAAA|1738322|5902|10282|2449601|2449571|Miss|Ethel|Bonner|N|31|3|1992|MALTA||Ethel.Bonner@8XAHdRJMMh.edu|2452568| +4525|AAAAAAAANKBBAAAA|378817|1508|23846|2450459|2450429|Ms.|Christy|Corrigan|Y|6|10|1977|SWITZERLAND||Christy.Corrigan@kAl51o4cYbbvpUK.edu|2452609| +4526|AAAAAAAAOKBBAAAA|408088|3296|9704|2452335|2452305|Mrs.|Marlene|Jameson|N|11|12|1977|GUATEMALA||Marlene.Jameson@vcZvvutmVtZ.edu|2452342| +4527|AAAAAAAAPKBBAAAA|479537|2664|38544|2450033|2450003|Dr.|Catherine|Mcclary|N|25|4|1957|JAPAN||Catherine.Mcclary@TRnzsqvKLae0v0ZSc.org|2452379| +4528|AAAAAAAAALBBAAAA|1597512|3118|37924|2451326|2451296|Sir|Barry|George|Y|12|3|1970|PAPUA NEW GUINEA||Barry.George@nJyh1JP6qfU.com|2452547| +4529|AAAAAAAABLBBAAAA|1570712|4253|15794|2449432|2449402|Mr.|Jason|Cornell|Y|10|9|1990|KOREA, REPUBLIC OF||Jason.Cornell@mxJkkJqTiBViAhzkB.com|2452569| +4530|AAAAAAAACLBBAAAA|1255097|2537|17130|2449109|2449079|Mr.|Cory|Morgan|Y|9|5|1990|BAHRAIN||Cory.Morgan@LXMb6FmB8Urt.edu|2452462| +4531|AAAAAAAADLBBAAAA||3659|5997||2449836|Ms.|||N|22|9||||Christine.Chase@6zHlmjuZubupo.edu|2452575| +4532|AAAAAAAAELBBAAAA|193153|3443|4933|2451246|2451216|Mrs.|Barbara|Kendrick|Y|21|7|1933|FRENCH GUIANA||Barbara.Kendrick@I9Au02fSo7SA1o.com|2452472| +4533|AAAAAAAAFLBBAAAA|867449|6096|40343|2451102|2451072|Dr.|Rena|Shields|N|15|10|1939|CHRISTMAS ISLAND||Rena.Shields@MlS2M74FC2Fr7yhdz.org|2452420| +4534|AAAAAAAAGLBBAAAA|1713874|1108|49189|2452121|2452091|Mr.|Lee|Wahl|N|29|10|1959|FRENCH POLYNESIA||Lee.Wahl@4texGdoCuQqC5.edu|2452568| +4535|AAAAAAAAHLBBAAAA|130957|3769|2228|2450456|2450426|Dr.|Amy|Estevez|Y|7|10|1959|GUAM||Amy.Estevez@hsgSHJNTeorB.edu|2452634| +4536|AAAAAAAAILBBAAAA|1024642|1957|2072|2451220|2451190|Mrs.|April|Boggs|N|9|10|1953|MEXICO||April.Boggs@otfyfz.org|2452427| +4537|AAAAAAAAJLBBAAAA|1536554|5974|38947|2452048|2452018|Dr.|Patrick|Magee|Y|14|11|1929|SAINT LUCIA||Patrick.Magee@2Lg551xV3x.com|2452625| +4538|AAAAAAAAKLBBAAAA|908796|1494|37039|2452671|2452641|Mr.|William|Smith|N|26|1|1936|NEW CALEDONIA||William.Smith@uVT7ISo8Da7qaz.com|2452444| +4539|AAAAAAAALLBBAAAA|994445|3970|2695|2452466|2452436|Miss|Verna|Rivera|Y|13|9|1946|CROATIA||Verna.Rivera@ubdski.com|2452432| +4540|AAAAAAAAMLBBAAAA|1255757|690|3223|2452560|2452530|Dr.|Gustavo|Ortiz|N|1|10|1936|HONDURAS||Gustavo.Ortiz@LiVhFuetyh.org|2452343| +4541|AAAAAAAANLBBAAAA|259512|3542|39525|2452315|2452285|Mrs.|Janet|Harris|N|1|7|1939|R�UNION||Janet.Harris@DP2xjcJRKFubV3.com|2452344| +4542|AAAAAAAAOLBBAAAA|1113394|3113|39679|2450792|2450762|Mrs.|Carrie|Miles|Y|22|10|1990|PITCAIRN||Carrie.Miles@O.org|2452327| +4543|AAAAAAAAPLBBAAAA|1445215|6039|46063|2451022|2450992|Ms.|Ethel|Marquez|Y|14|2|1956|AUSTRALIA||Ethel.Marquez@RDSMN.org|2452459| +4544|AAAAAAAAAMBBAAAA|1389968|3159|38797|2449308|2449278|Dr.|David|Ellis|Y|23|8|1963|ANGUILLA||David.Ellis@PlMLmp6bbJ3Ki.org|2452587| +4545|AAAAAAAABMBBAAAA|1707720|2369|41465|2449536|2449506|Dr.|Anthony|Hollis|Y|7|12|1978|DOMINICA||Anthony.Hollis@dnd.edu|2452554| +4546|AAAAAAAACMBBAAAA|929229|9|27636|2451113|2451083|Dr.|Michael|Alvarado|N|27|12|1990|LIBERIA||Michael.Alvarado@A63BvnOldkTj.org|2452320| +4547|AAAAAAAADMBBAAAA|1308605|5664|44723|2451905|2451875|Miss|Teresa|Nelson|N|16|11|1960|GUERNSEY||Teresa.Nelson@xp22Nth8LCPht1e.com|2452392| +4548|AAAAAAAAEMBBAAAA|423076|6649|37869|2450227|2450197|Mrs.|Laura|Stubbs|N|12|2|1966|GRENADA||Laura.Stubbs@8fZuAIOb8oSLg.edu|2452400| +4549|AAAAAAAAFMBBAAAA|1129548|4915|16235|2450903|2450873|Dr.|Charlotte|Bailey|N|1|12|1967|ICELAND||Charlotte.Bailey@u31mkX7AvOAyr.com|2452373| +4550|AAAAAAAAGMBBAAAA|1301404|5058|20665|2450705|2450675|Mr.|Fernando|Wall|Y|16|12|1953|ANTARCTICA||Fernando.Wall@onrR4e.edu|2452300| +4551|AAAAAAAAHMBBAAAA|350477|3245|12813|2449962|2449932|Mr.|William|Rhodes|Y|21|8|1941|CAYMAN ISLANDS||William.Rhodes@smHxjR2r4u.com|2452643| +4552|AAAAAAAAIMBBAAAA|770022|4010|1598|2451509|2451479|Mr.|Clay|Jacks|N|5|10|1937|PAPUA NEW GUINEA||Clay.Jacks@Ui.com|2452337| +4553|AAAAAAAAJMBBAAAA|772475|5711|32147|2452315|2452285|Sir|Shawn|Rivas|Y|18|4|1974|NEW ZEALAND||Shawn.Rivas@tOh0.edu|2452549| +4554|AAAAAAAAKMBBAAAA|43329|5893|25304|2449945|2449915|Dr.|Cedric|Badillo|Y|3|4|1964|CUBA||Cedric.Badillo@6YcGKT.edu|2452361| +4555|AAAAAAAALMBBAAAA|210056|1114|25419|2451340|2451310|Dr.|Salvador|Browning|N|22|11|1926|VANUATU||Salvador.Browning@uLk.org|2452299| +4556|AAAAAAAAMMBBAAAA|1614939|2514|10489|2450858|2450828|Dr.|Alberto|Hass|Y|26|10|1948|BOTSWANA||Alberto.Hass@saz5aRdePv.com|2452463| +4557|AAAAAAAANMBBAAAA|963371|3104|11637|2451496|2451466|Dr.|Eric|Green|Y|9|10|1967|GABON||Eric.Green@6MUFud37NeX9SV.edu|2452572| +4558|AAAAAAAAOMBBAAAA|688119|2137|26602|2450777|2450747|Dr.|Tammy|Wallace|N|12|5|1988|AUSTRALIA||Tammy.Wallace@JR4f.org|2452549| +4559|AAAAAAAAPMBBAAAA|939141|3999|22857|2449649|2449619|Sir|Luis|Salazar|N|9|1|1991|UGANDA||Luis.Salazar@zS7Q.org|2452528| +4560|AAAAAAAAANBBAAAA|1215139|4993|32447|2449733|2449703|Mrs.|Leanna|Brewer|N|12|9|1975|PAKISTAN||Leanna.Brewer@KaPTqpU.org|2452466| +4561|AAAAAAAABNBBAAAA|1914661|302|5896|2449351|2449321|Ms.|Irma|Smith|Y|1|3|1980|SERBIA||Irma.Smith@dOdDOArM.com|2452361| +4562|AAAAAAAACNBBAAAA|1577191|1045|1699|2451634|2451604|Ms.|Joanna|Saavedra|N|16|10|1926|UNITED KINGDOM||Joanna.Saavedra@ChSSHB449r.com|2452394| +4563|AAAAAAAADNBBAAAA|527147|2819|36491|2451938|2451908|Mr.|Tommie|Kearns|Y|12|9|1939|LEBANON||Tommie.Kearns@92rkhLNIIjM.org|2452549| +4564|AAAAAAAAENBBAAAA|871827|3958|34136|2452411|2452381|Miss|Nicole|Martinez|N|31|1|1990|SERBIA||Nicole.Martinez@mTCMr7x.edu|2452520| +4565|AAAAAAAAFNBBAAAA|431754|3631|42261|2449032|2449002|Mr.|Mark|Byrne|N|25|11|1958|BOUVET ISLAND||Mark.Byrne@OclThS14bgV.org|2452426| +4566|AAAAAAAAGNBBAAAA|824367|472|40977|2449487|2449457|Sir|Bruce|Torres|N|6|10|1981|BHUTAN||Bruce.Torres@apdnT2.com|2452559| +4567|AAAAAAAAHNBBAAAA|974034|4486|15338|2449472|2449442|Mr.|Richard|Jones|Y|19|3|1981|BOUVET ISLAND||Richard.Jones@x2go2.org|2452519| +4568|AAAAAAAAINBBAAAA|997478|1797|28516|2452429|2452399|Dr.|Kathleen|Stanford|Y|20|11|1958|TAJIKISTAN||Kathleen.Stanford@DEKZD.com|2452375| +4569|AAAAAAAAJNBBAAAA|285621|2813|46052|2451308|2451278|Sir|David|Bland|Y|27|7|1943|PAPUA NEW GUINEA||David.Bland@ETFbBr7ut9B.com|2452503| +4570|AAAAAAAAKNBBAAAA|346429|4886|5390|2451284|2451254|Mrs.|Shelia|Young|N|23|10|1974|SWITZERLAND||Shelia.Young@xsJrGvSrxF.com|2452553| +4571|AAAAAAAALNBBAAAA|1384316|3859|13402|2452430|2452400|Dr.|Diana|Boyd|N|25|5|1955|GUAM||Diana.Boyd@JVVn77eUSS5zpV.com|2452350| +4572|AAAAAAAAMNBBAAAA|77170|2894|49418|2449130|2449100|Miss|Gertrude|Parrish|N|20|8|1945|BRAZIL||Gertrude.Parrish@LF3DuZyzdUZY5qTv.edu|2452313| +4573|AAAAAAAANNBBAAAA|1698401|423|36092|2452134|2452104|Ms.|Lyndsay|Brooks|Y|17|10|1955|ERITREA||Lyndsay.Brooks@B0DaYFIcUT.edu|2452451| +4574|AAAAAAAAONBBAAAA|569425|6708|11544|2449355|2449325|Ms.|Helen|Payton|Y|3|7|1963|UNITED KINGDOM||Helen.Payton@vL1Tm3IqMln.edu|2452518| +4575|AAAAAAAAPNBBAAAA|780660|4039|48618|2450384|2450354|Dr.|Renee|Hudson|N|21|7|1941|BURUNDI||Renee.Hudson@nb6SLde85piB.org|2452457| +4576|AAAAAAAAAOBBAAAA|810641|3033|33072|2452576|2452546|Miss|Laura|Miranda|Y|18|7|1986|UNITED KINGDOM||Laura.Miranda@uVhQs7JFzlqNaa.com|2452455| +4577|AAAAAAAABOBBAAAA|1845257|410|9163|2450901|2450871|Sir|Raul|Ryan|N|1|4|1931|NICARAGUA||Raul.Ryan@JPhdhoxnAm3ggsP0X7.com|2452325| +4578|AAAAAAAACOBBAAAA|208145|4618|31938|2452291|2452261|Mrs.|Sherri|Clark|Y|1|7|1972|SENEGAL||Sherri.Clark@5RzPRNO.com|2452508| +4579|AAAAAAAADOBBAAAA|1276395|5065|40888|2450551|2450521|Ms.|Elvia|Adams|N|9|2|1940|BELIZE||Elvia.Adams@rqMT.org|2452518| +4580|AAAAAAAAEOBBAAAA|845394|7109|37577|2452159|2452129|Dr.|Mike|Bowling|N|26|10|1943|ESTONIA||Mike.Bowling@TJYvoK.com|2452594| +4581|AAAAAAAAFOBBAAAA|679954|2140|18773|2449954|2449924|Ms.|Dulce|Taylor|Y|6|10|1934|HAITI||Dulce.Taylor@1H.com|2452611| +4582|AAAAAAAAGOBBAAAA|1899228|820|44812|2450800|2450770|Sir|Nicholas|Wyant|N|19|4|1974|TIMOR-LESTE||Nicholas.Wyant@jv2iFcFkyU5.com|2452515| +4583|AAAAAAAAHOBBAAAA|||11656|2450176||Dr.|Elsie|Harrison||4|||URUGUAY||Elsie.Harrison@h0JpZ4nXUH0Be.com|| +4584|AAAAAAAAIOBBAAAA|1411688|697|6459|2452317|2452287|Sir|James|Irvine|Y|31|10|1925|SEYCHELLES||James.Irvine@eJ9bH9G8ReXD.org|2452586| +4585|AAAAAAAAJOBBAAAA|1550387|3986|20785|2450270|2450240|Ms.|Anthony|Wilson|Y|15|5|1991|AZERBAIJAN||Anthony.Wilson@BebfByYiKCcg.com|2452302| +4586|AAAAAAAAKOBBAAAA|873256|5913|35370|2449563|2449533|Ms.|Freda|Skinner|Y|23|3|1951|ANGOLA||Freda.Skinner@IeTk7kd.com|2452612| +4587|AAAAAAAALOBBAAAA|1449045|4207|17016|2449623|2449593|Sir|Carmen|Carr|Y|13|7|1971|GABON||Carmen.Carr@l.edu|2452632| +4588|AAAAAAAAMOBBAAAA|637080|1717|7154|2451650|2451620|Miss|Madelyn|Guerra|Y|16|8|1928|MONGOLIA||Madelyn.Guerra@6I1nOAOcxlvZM.com|2452378| +4589|AAAAAAAANOBBAAAA|872879|5184|47089|2450331|2450301|Dr.|Karen|Watson|N|12|4|1964|GERMANY||Karen.Watson@ooHsYcemgMZ.edu|2452549| +4590|AAAAAAAAOOBBAAAA|1910097|2881|30512|2449043|2449013|Dr.|Francis|Spann|Y|1|1|1956|ARMENIA||Francis.Spann@y60Yp.edu|2452593| +4591|AAAAAAAAPOBBAAAA|707776|5150|16124|2452119|2452089|Dr.|Richard|Corbett|Y|17|11|1942|KOREA, REPUBLIC OF||Richard.Corbett@1Z7jFki81.org|2452434| +4592|AAAAAAAAAPBBAAAA|1360842|7013|32031|2450288|2450258|Dr.|Maxine|Garvin|N|16|2|1958|BELGIUM||Maxine.Garvin@lgO3VVARoF3.org|2452581| +4593|AAAAAAAABPBBAAAA|1105129|867|24158|2449824|2449794|Ms.|Amelia|Blackwell|Y|23|3|1992|MALDIVES||Amelia.Blackwell@3D6v.com|2452436| +4594|AAAAAAAACPBBAAAA|443378|3367|1356|2450873|2450843|Dr.|Marty|Briggs|N|22|3|1950|MOLDOVA, REPUBLIC OF||Marty.Briggs@pt2gZlLBomD.com|2452356| +4595|AAAAAAAADPBBAAAA|1190425|3783|27188|2449359|2449329|Miss|Virginia|Jones|N|25|11|1977|GIBRALTAR||Virginia.Jones@9MmljNyC6.edu|2452403| +4596|AAAAAAAAEPBBAAAA|1335961|2702|34290|2452408|2452378|Dr.|Sandra|Egan|N|23|4|1987|AUSTRIA||Sandra.Egan@UZ1IGZFUNRc38fh.org|2452361| +4597|AAAAAAAAFPBBAAAA|283632|6821|17563|2449854|2449824|Miss|Kimberly|Johnson|Y|19|2|1924|URUGUAY||Kimberly.Johnson@Kj3ufIMOg0FkGM.com|2452632| +4598|AAAAAAAAGPBBAAAA|1422924|2512|44100|2450204|2450174|Mr.|Edward|Adams|N|14|7|1963|GUYANA||Edward.Adams@E8VV.org|2452568| +4599|AAAAAAAAHPBBAAAA|633244|1523|5002|2451749|2451719|Sir|Larry|Lee|Y|26|9|1946|MARSHALL ISLANDS||Larry.Lee@Ez.com|2452378| +4600|AAAAAAAAIPBBAAAA|252353|4790|41980|2450526|2450496|Miss|Cathleen|White|Y|27|9|1947|SAINT HELENA||Cathleen.White@VQa6OTIK7rHq.org|2452305| +4601|AAAAAAAAJPBBAAAA|481310|2164|48618|2451782|2451752|Mr.|Eric|Shore|Y|4|12|1937|TONGA||Eric.Shore@b23K99.edu|2452641| +4602|AAAAAAAAKPBBAAAA|438308|2563|29618|2449800|2449770|Sir|Brian|Henderson|Y|14|4|1965|HAITI||Brian.Henderson@U.edu|2452645| +4603|AAAAAAAALPBBAAAA|610992|2220|25895|2449805|2449775|Mr.|Bernard|Hatfield|Y|11|12|1930|BOUVET ISLAND||Bernard.Hatfield@23Fv.org|2452308| +4604|AAAAAAAAMPBBAAAA|591944|3839|10134|2451097|2451067|Mr.|Robert|Maynard|Y|17|2|1953|BOUVET ISLAND||Robert.Maynard@C2sUq7f9Hmf2Qt1xZ.edu|2452529| +4605|AAAAAAAANPBBAAAA|1489510|2351|5992|2450070|2450040|Sir|Tony|Smith|N|23|6|1968|CANADA||Tony.Smith@XHoC518tya.com|2452432| +4606|AAAAAAAAOPBBAAAA|230571|1629|32240|2451621|2451591|Ms.|Anthony|Barba|Y|28|6|1991|MONACO||Anthony.Barba@eIsJIzSjaHd3ogF.edu|2452301| +4607|AAAAAAAAPPBBAAAA|1368951|5746|9388|2449784|2449754|Dr.|Lillian|Blanco|Y|8|6|1971|FIJI||Lillian.Blanco@BiIA4IkY4.com|2452582| +4608|AAAAAAAAAACBAAAA|1757710|5142|30174|2450080|2450050|Miss|Sara|Burgess|N|7|7|1953|SAUDI ARABIA||Sara.Burgess@gBPTsoF.org|2452314| +4609|AAAAAAAABACBAAAA||6978|22621|||Ms.|Molly|Jones|||3|1942||||2452477| +4610|AAAAAAAACACBAAAA|1684764|3028|47292|2449885|2449855|Dr.|Angelica|Balderas|N|7|4|1973|ARMENIA||Angelica.Balderas@T26erJxVkiAsHpY.org|2452405| +4611|AAAAAAAADACBAAAA|1001068|2185|25749|2450686|2450656|Dr.|Arthur|Jennings|N|11|12|1983|BAHRAIN||Arthur.Jennings@xCKfFo32xde.org|2452479| +4612|AAAAAAAAEACBAAAA|964537|3688|41239|2449325|2449295|Mr.|Marcus|Kelley|N|11|4|1991|MONGOLIA||Marcus.Kelley@P54RCJKPb0hC.com|2452535| +4613|AAAAAAAAFACBAAAA|||10536||2451369||Dustin|||10||1968|||Dustin.Charles@TZTc.edu|| +4614|AAAAAAAAGACBAAAA|1804912|5296|10127|2450803|2450773|Mrs.|Jenny|Bowles|Y|23|12|1958|NORFOLK ISLAND||Jenny.Bowles@IeZX2vD.com|2452612| +4615|AAAAAAAAHACBAAAA|1309799|3850|40748|2451537|2451507|Sir|Steve|Miles|N|12|2|1934|MOZAMBIQUE||Steve.Miles@YFqsVTLx1smaB.org|2452284| +4616|AAAAAAAAIACBAAAA|99285|6153|19021|2450329|2450299|Miss|Gudrun|Stevens|N|12|8|1957|ANGUILLA||Gudrun.Stevens@0sAlcczh7LhCX6.org|2452514| +4617|AAAAAAAAJACBAAAA|569947|4419|18138|2452566|2452536|Dr.|Wilson|Stevens|N|10|2|1931|FIJI||Wilson.Stevens@VoIqmfEffjR0Hp.com|2452636| +4618|AAAAAAAAKACBAAAA|456877|3195|36713|2452361|2452331|Mr.|David|Newell|N|13|10|1980|BELARUS||David.Newell@xQLtJ4Sim.com|2452409| +4619|AAAAAAAALACBAAAA|1650310|3456|21402|2452062|2452032|Dr.|Nancy|Thompson|Y|15|9|1944|GREECE||Nancy.Thompson@N9AJJS2d80.com|2452356| +4620|AAAAAAAAMACBAAAA|432336|5807|31231|2452551|2452521|Miss|Shonda|Cannon|N|21|5|1941|GAMBIA||Shonda.Cannon@5.edu|2452416| +4621|AAAAAAAANACBAAAA|367368|2765|6609|2450432|2450402|Mrs.|Blanche|Sanchez|Y|25|1|1984|PORTUGAL||Blanche.Sanchez@AvDq.com|2452544| +4622|AAAAAAAAOACBAAAA|692008|3719|39383|2451052|2451022|Sir|Lance|Lange|Y|2|10|1960|ANDORRA||Lance.Lange@hrTELiJkmaeaes.edu|2452515| +4623|AAAAAAAAPACBAAAA|1502633|4553|15626|2449724|2449694|Mr.|Edwin|Lewandowski|N|12|6|1951|CHILE||Edwin.Lewandowski@PENSDOYMljAh.com|2452341| +4624|AAAAAAAAABCBAAAA||1306|47243|||Dr.||Miller|N||4|1927|AUSTRALIA|||| +4625|AAAAAAAABBCBAAAA|1020682|6782|33729|2451935|2451905|Sir|Jose|Dodge|Y|16|11|1988|GAMBIA||Jose.Dodge@5mRQbY8Un1.edu|2452518| +4626|AAAAAAAACBCBAAAA|1134367|3735|48350|2451521|2451491|Ms.|Marianne|Davis|Y|12|10|1976|NIGER||Marianne.Davis@1ATLq0n.edu|2452512| +4627|AAAAAAAADBCBAAAA|881673|3951|42676|2451500|2451470|Dr.|Rosetta|Anderson|Y|8|4|1960|BERMUDA||Rosetta.Anderson@z.org|2452554| +4628|AAAAAAAAEBCBAAAA|1839243|4553|34263|2452547|2452517|Miss|Diane|Johnson|Y|11|3|1961|SWAZILAND||Diane.Johnson@hZoHIy.edu|2452534| +4629|AAAAAAAAFBCBAAAA|553253|7155|28657|2449706|2449676|Mrs.|Carolyn|Earley|Y|8|1|1985|ISLE OF MAN||Carolyn.Earley@SBSZRi28ZJJ.org|2452307| +4630|AAAAAAAAGBCBAAAA|103040|3069|21945|2450640|2450610|Dr.|Eleanore|Luna|Y|1|8|1945|VIRGIN ISLANDS, U.S.||Eleanore.Luna@OK1V7mP.edu|2452648| +4631|AAAAAAAAHBCBAAAA|1108564||5242|2450969||Ms.||Watkins||28|2|1985|SEYCHELLES||Ethel.Watkins@EDtGTARsVKR.org|| +4632|AAAAAAAAIBCBAAAA|1258886|2995|43951|2452396|2452366|Ms.|Claudine|Williams|Y|31|12|1987|ALAND ISLANDS||Claudine.Williams@yZ3Y7AX.edu|2452450| +4633|AAAAAAAAJBCBAAAA|401465|3818|47324|2450174|2450144|Ms.|Anita|Baker|N|19|5|1934|LESOTHO||Anita.Baker@ZiUZCgFS.org|2452301| +4634|AAAAAAAAKBCBAAAA|||37010||2452108||Barbara|Dowd||||||||2452606| +4635|AAAAAAAALBCBAAAA|245393|2984|24975|2452331|2452301|Sir|Edgar|Redden|Y|26|11|1973|ISRAEL||Edgar.Redden@CnT.org|2452326| +4636|AAAAAAAAMBCBAAAA|1584014|6307|45000|2449541|2449511|Dr.|Cynthia|Champion|N|2|5|1954|MONTENEGRO||Cynthia.Champion@oZMhQg5QLKIAlCK.org|2452374| +4637|AAAAAAAANBCBAAAA|920465|6001|48690|2451901|2451871|Ms.|Ana|Cary|N|24|1|1969|VENEZUELA||Ana.Cary@imJui63o9G08Nas.com|2452453| +4638|AAAAAAAAOBCBAAAA|1425829|5694|406|2450946|2450916|Mr.|Abraham|White|Y|23|4|1939|CHRISTMAS ISLAND||Abraham.White@i5OJJB.com|2452511| +4639|AAAAAAAAPBCBAAAA|436426|4891|7003|2450617|2450587|Mr.|Drew|Olson|Y|20|1|1960|WALLIS AND FUTUNA||Drew.Olson@LAjlcrXQJE.org|2452558| +4640|AAAAAAAAACCBAAAA|802088|1952|663|2451554|2451524|Mr.|George|Williamson|N|5|4|1949|VANUATU||George.Williamson@Y4EsSHtQGfotiZ.edu|2452383| +4641|AAAAAAAABCCBAAAA|1835492|543|39547|2452082|2452052|Mrs.|Michelle|Kline|N|2|5|1953|COMOROS||Michelle.Kline@PhVbtVsaOuqd.com|2452292| +4642|AAAAAAAACCCBAAAA|1814106|5652|27645|2450223|2450193|Dr.|Dennis|Smith|N|14|2|1973|MAYOTTE||Dennis.Smith@8Vx.org|2452622| +4643|AAAAAAAADCCBAAAA|||32818|||||||||1984||||2452607| +4644|AAAAAAAAECCBAAAA|1472772|5783|13194|2451065|2451035|Ms.|Brynn|Meza|N|29|10|1980|GHANA||Brynn.Meza@tye.org|2452469| +4645|AAAAAAAAFCCBAAAA|1854712|4158|1957|2451329|2451299|Mrs.|Helen|Mccoy|N|15|10|1985|GHANA||Helen.Mccoy@9D1Lqjta.com|2452606| +4646|AAAAAAAAGCCBAAAA|1252612|7087|18762|2452090|2452060|Dr.|Robert|Caldwell|Y|21|3|1954|SAUDI ARABIA||Robert.Caldwell@1nla2nKliyP.com|2452325| +4647|AAAAAAAAHCCBAAAA|401855|1506|17157|2452085|2452055|Dr.|Robert|Dodd|N|2|2|1956|ECUADOR||Robert.Dodd@d.edu|2452554| +4648|AAAAAAAAICCBAAAA|1219185|7149|9913|2450662|2450632|Dr.|Gerald|Hopson|Y|6|1|1945|KOREA, REPUBLIC OF||Gerald.Hopson@5VzboU3g.edu|2452595| +4649|AAAAAAAAJCCBAAAA|1460891|6044|47997|2452071|2452041|Mr.|Cory|Fife|Y|28|8|1973|BANGLADESH||Cory.Fife@6TZE.com|2452585| +4650|AAAAAAAAKCCBAAAA|93869|5050|2231|2450647|2450617|Mr.|Peter|Wilson|Y|24|5|1936|GUAM||Peter.Wilson@KGFamNYr.edu|2452447| +4651|AAAAAAAALCCBAAAA|119581|2531|40281|2449163|2449133|Mr.|Matthew|Morgan|Y|22|7|1962|SIERRA LEONE||Matthew.Morgan@8Z1HHrm.org|2452355| +4652|AAAAAAAAMCCBAAAA|435703|2186|6622|2452595|2452565|Dr.|Leola|Gregory|N|4|9|1981|CUBA||Leola.Gregory@kB3AH.edu|2452429| +4653|AAAAAAAANCCBAAAA|26969|4805|28994|2450129|2450099|Ms.|Elizabeth|Chance|N|11|8|1979|IRELAND||Elizabeth.Chance@PqY9fQ2ZUUx.com|2452391| +4654|AAAAAAAAOCCBAAAA|897068|915|24362|2449435|2449405|Miss|Gayla|Lofton|Y|26|12|1990|SOMALIA||Gayla.Lofton@ND7Dtj.org|2452476| +4655|AAAAAAAAPCCBAAAA|1802936|4138|15740|2452300|2452270|Ms.|Linda|Lee|N|24|5|1991|ISLE OF MAN||Linda.Lee@3xtVTIdHurOsd.edu|2452404| +4656|AAAAAAAAADCBAAAA|479518||39069|2450760|2450730||||N|||||||2452524| +4657|AAAAAAAABDCBAAAA|871075|6486|9544|2450319|2450289|Dr.|Ima|Pugh|Y|3|5|1992|KUWAIT||Ima.Pugh@UTvCFXCe7hkuL.org|2452530| +4658|AAAAAAAACDCBAAAA|514594|890|37539|2451083|2451053|Dr.|Stewart|Delagarza|Y|21|5|1979|EGYPT||Stewart.Delagarza@ouI.edu|2452309| +4659|AAAAAAAADDCBAAAA||568|20495|2452195|2452165||Sybil||Y|7||1941|INDIA||Sybil.Tracy@2hrcKCF.edu|2452485| +4660|AAAAAAAAEDCBAAAA|1626928|6813|38298|2452230|2452200|Sir|Brian|Salas|Y|12|11|1984|IRELAND||Brian.Salas@gHXyIEg39XrLTN.org|2452298| +4661|AAAAAAAAFDCBAAAA|1414256|3933|39610|2450226|2450196|Dr.|Khalilah|Usher|Y|11|10|1963|CAMBODIA||Khalilah.Usher@pFbjEFr0U.edu|2452371| +4662|AAAAAAAAGDCBAAAA|1854058|3855|38052|2451452|2451422|Mr.|John|Williams|N|14|7|1948|ANGOLA||John.Williams@N5CzQ0rC.org|2452407| +4663|AAAAAAAAHDCBAAAA|142615|2287|42359|2449439|2449409|Dr.|Rudy|Green|Y|20|2|1937|BRAZIL||Rudy.Green@ZbI9YdjSGa9EyTcbyXB.com|2452402| +4664|AAAAAAAAIDCBAAAA|5246|6761|37076|2450266|2450236|Miss|Veronica|Williams|N|29|9|1932|SWEDEN||Veronica.Williams@pnAdL1L.org|2452328| +4665|AAAAAAAAJDCBAAAA|1535351|6265|22826|2451852|2451822|Ms.|Natasha|Mcgee|Y|21|3|1934|TONGA||Natasha.Mcgee@fepFqaLooGHtZGZ.org|2452444| +4666|AAAAAAAAKDCBAAAA|1880364|2643|38122|2452051|2452021|Dr.|Susan|Smith|N|20|4|1971|PITCAIRN||Susan.Smith@glYQKEKMBPG.org|2452293| +4667|AAAAAAAALDCBAAAA|1509425|3220|42262|2451058|2451028|Sir|Robbie|Dunlap|Y|13|9|1942|SOUTH AFRICA||Robbie.Dunlap@VP0A.com|2452314| +4668|AAAAAAAAMDCBAAAA|721687|4442|10397|2450436|2450406|Sir|William|Taylor|N|30|10|1981|FRENCH POLYNESIA||William.Taylor@9CV.org|2452501| +4669|AAAAAAAANDCBAAAA|1809574|1257|7740|2451351|2451321|Mr.|Cameron|Richard|N|14|7|1958|KUWAIT||Cameron.Richard@9Choe4BMIeR8ynH9.edu|2452577| +4670|AAAAAAAAODCBAAAA|1360050|5455|48860|2450402|2450372|Dr.|Eleanor|Anderson|Y|26|12|1975|CHINA||Eleanor.Anderson@XDOQ7JGc6kKiOA7.edu|2452473| +4671|AAAAAAAAPDCBAAAA|1110262|2991|22567|2451642|2451612|Dr.|David|Walker|Y|22|9|1939|AUSTRIA||David.Walker@3SaG40F83mNr.edu|2452525| +4672|AAAAAAAAAECBAAAA|824772|6059|17746|2451912|2451882|Mrs.|Antonia|Goode|Y|23|6|1980|ETHIOPIA||Antonia.Goode@GYKAi7aqLN.edu|2452427| +4673|AAAAAAAABECBAAAA||4250|39975|2451775|2451745||Kenneth|Osorio|||2||MOROCCO||Kenneth.Osorio@oNSfii.com|| +4674|AAAAAAAACECBAAAA|670277|3449|18574|2449273|2449243|Mrs.|Jacque|Goolsby|N|14|2|1990|KYRGYZSTAN||Jacque.Goolsby@MUXX41tPF0.org|2452512| +4675|AAAAAAAADECBAAAA|183382|2147|47712|2449523|2449493|Dr.|Eva|Hall|Y|24|4|1954|TRINIDAD AND TOBAGO||Eva.Hall@tDaNJ3hvEgO9SGZ.edu|2452592| +4676|AAAAAAAAEECBAAAA|1534693|2938|45983|2452238|2452208|Dr.|Denise|Gomez|Y|12|1|1933|MALDIVES||Denise.Gomez@3GlO.edu|2452566| +4677|AAAAAAAAFECBAAAA|650049|5319|19036|2451979|2451949|Ms.|Vanessa|Faulkner|Y|14|7|1983|CAPE VERDE||Vanessa.Faulkner@7sg27rmR5Bj.com|2452518| +4678|AAAAAAAAGECBAAAA|467848|2536|23224|2451451|2451421|Mrs.|Carole|Brewer|Y|29|6|1947|SENEGAL||Carole.Brewer@u1mg3K.edu|2452624| +4679|AAAAAAAAHECBAAAA|1272136|4120|24104|2449724|2449694|Mr.|Danny|Strickland|Y|10|4|1960|PANAMA||Danny.Strickland@bynGeGDNtTmss.com|2452601| +4680|AAAAAAAAIECBAAAA|926200|4820|14193|2450851|2450821|Sir|Roger|Thomas|Y|11|8|1934|ANGOLA||Roger.Thomas@id.edu|2452532| +4681|AAAAAAAAJECBAAAA|1669583|3620|2457|2450138|2450108|Dr.|William|Newton|N|7|2|1970|OMAN||William.Newton@hrMBruz.org|2452561| +4682|AAAAAAAAKECBAAAA|1732561|6425|49089|2450415|2450385|Sir|Raymond|Corbett|Y|31|10|1990|TAJIKISTAN||Raymond.Corbett@49mfFrPO.edu|2452343| +4683|AAAAAAAALECBAAAA|1155420|1455|27043|2450284|2450254|Mrs.|Lisa|Young|N|29|1|1949|KIRIBATI||Lisa.Young@TYTl.edu|2452497| +4684|AAAAAAAAMECBAAAA|874504|2379|40607|2450282|2450252|Ms.|Melanie|Johnson|N|23|6|1960|ANTARCTICA||Melanie.Johnson@yS6Q1ZVYVFgviT.com|2452534| +4685|AAAAAAAANECBAAAA|1035122|2665|19361|2452188|2452158|Sir|David|Ballard|N|15|8|1940|TRINIDAD AND TOBAGO||David.Ballard@7YmM5iGyy8Yx6cpq.com|2452489| +4686|AAAAAAAAOECBAAAA|1393963|2131|17899|2450534|2450504|Mrs.|Elouise|Cox|N|2|12|1957|GHANA||Elouise.Cox@qTgzK4p7neU.com|2452362| +4687|AAAAAAAAPECBAAAA|678017|1665|7713|2450923|2450893|Ms.|Minnie|Huddleston|Y|16|7|1939|MALAWI||Minnie.Huddleston@nxfo.edu|2452600| +4688|AAAAAAAAAFCBAAAA|1083602|2352|23055|2449259|2449229|Sir|Lee|Rogers|N|16|3|1947|SOLOMON ISLANDS||Lee.Rogers@AynuGImPz3TFRz9zGOc.org|2452619| +4689|AAAAAAAABFCBAAAA|1190960|2700|46087|2449868|2449838|Sir|Stephen|Steed|Y|14|1|1967|EQUATORIAL GUINEA||Stephen.Steed@IjL0sN.edu|2452502| +4690|AAAAAAAACFCBAAAA|1754630|6830|41830|2450572|2450542|Mr.|Allan|Stewart|N|5|8|1980|JAMAICA||Allan.Stewart@9.org|2452416| +4691|AAAAAAAADFCBAAAA|1520683|1691|35899|2449244|2449214|Dr.|James|Garcia|N|16|3|1975|DJIBOUTI||James.Garcia@XlxM83CfpMM7sE1xfay.edu|2452429| +4692|AAAAAAAAEFCBAAAA|1226187|6180|45326|2450234|2450204|Sir|Jason|Dougherty|N|8|2|1985|SINGAPORE||Jason.Dougherty@qnDCbgaCbDZR.org|2452645| +4693|AAAAAAAAFFCBAAAA|380285|3376|47314|2449683|2449653|Sir|Roger|Larson|N|27|2|1971|GUAM||Roger.Larson@qIe.edu|2452644| +4694|AAAAAAAAGFCBAAAA|13731|3301|20071|2450835|2450805|Mr.|Thomas|Ward|Y|22|9|1986|SLOVENIA||Thomas.Ward@RMuvqB20F4sZ4P.edu|2452509| +4695|AAAAAAAAHFCBAAAA|1304582|36|42737|2449728|2449698|Dr.|Stephen|Peters|Y|12|10|1944|MARTINIQUE||Stephen.Peters@h6JsEdI8C.com|2452391| +4696|AAAAAAAAIFCBAAAA|134237|5689|34341|2452097|2452067|Miss|Katie|Groves|N|27|1|1964|HAITI||Katie.Groves@B96dRrDJskpRyoAt.org|2452487| +4697|AAAAAAAAJFCBAAAA|897219|3117|568|2450583|2450553|Miss|Peggy|Allen|Y|9|3|1946|NAMIBIA||Peggy.Allen@Gu4bIotvn.edu|2452579| +4698|AAAAAAAAKFCBAAAA|835120|395|14870|2449320|2449290|Ms.|Jennifer|Winter|Y|8|4|1937|URUGUAY||Jennifer.Winter@L3OBxIRA9rLQr.com|2452329| +4699|AAAAAAAALFCBAAAA|1831003|4857|10758|2451187|2451157|Mr.|Bobby|Ashley|N|15|10|1944|NEW ZEALAND||Bobby.Ashley@3I7x8Boj5yI.com|2452579| +4700|AAAAAAAAMFCBAAAA|646888|1804|35520|2451509|2451479|Sir|Michael|Schroeder|Y|26|7|1972|MOLDOVA, REPUBLIC OF||Michael.Schroeder@6Eebn9KRO.org|2452355| +4701|AAAAAAAANFCBAAAA|513056|3318|6380|2452353||Dr.|||N|||1929|||Kimberlee.Lawson@r4qIbvkKSC.com|2452321| +4702|AAAAAAAAOFCBAAAA|821695|1836|27177|2449888|2449858|Mr.|Frank|Flanagan|N|24|6|1972|TUVALU||Frank.Flanagan@C17bCcChmnMn09.org|2452530| +4703|AAAAAAAAPFCBAAAA|1305566|3972|45978|2450076|2450046|Miss|Betty|Baker|N|8|5|1974|NETHERLANDS ANTILLES||Betty.Baker@OKu0k1pB9Cn.org|2452593| +4704|AAAAAAAAAGCBAAAA|251899|5868|44990|2449249|2449219|Mr.|James|Sherman|N|7|6|1982|DJIBOUTI||James.Sherman@cG9c4op4R8FG5.edu|2452620| +4705|AAAAAAAABGCBAAAA|1151283|433|13936|2452143|2452113|Ms.|Roxann|Stevens|N|10|12|1969|PITCAIRN||Roxann.Stevens@av.com|2452502| +4706|AAAAAAAACGCBAAAA|1850573|2851|15129|2450298|2450268|Mr.|Chad|Oconnell|Y|1|8|1963|COSTA RICA||Chad.Oconnell@9.edu|2452440| +4707|AAAAAAAADGCBAAAA|1111577|4414|18451|2449170|2449140|Dr.|Grant|Bennett|N|9|4|1945|LIBERIA||Grant.Bennett@EZpAMKUybEpcUUuS.com|2452643| +4708|AAAAAAAAEGCBAAAA|368737|3375|34013|2449871|2449841|Dr.|Daniel|Moffett|N|4|11|1939|BOUVET ISLAND||Daniel.Moffett@dyyjaksIeeAJq8s.com|2452407| +4709|AAAAAAAAFGCBAAAA|84776|2002|8428|2452488|2452458|Dr.|Juana|Bowles|Y|9|2|1974|SLOVENIA||Juana.Bowles@qZRGVUNzNI6a3lOY6.org|2452584| +4710|AAAAAAAAGGCBAAAA|1339137|5640|38896|2450149|2450119|Sir|Michael|Beatty|Y|12|3|1986|CAYMAN ISLANDS||Michael.Beatty@Ck6fmi9FKbyJniklmB.com|2452372| +4711|AAAAAAAAHGCBAAAA|1099746|1520|44141|2449664|2449634|Mr.|Burl|Miller|Y|12|8|1985|TONGA||Burl.Miller@p1tqptPxqJxUi.edu|2452446| +4712|AAAAAAAAIGCBAAAA|1203064|3904|24297|2451763|2451733|Dr.|Micheal|Moody|N|19|7|1972|HUNGARY||Micheal.Moody@bue.org|2452313| +4713|AAAAAAAAJGCBAAAA|1442323|3682|4288|2449983|2449953|Sir|Kevin|Murphy|N|1|1|1932|SENEGAL||Kevin.Murphy@SJiXHRRb8VgmsF.org|2452333| +4714|AAAAAAAAKGCBAAAA|1550598|4342|6886|2449117|2449087|Mr.|Bennett|Reynoso|N|27|10|1975|INDIA||Bennett.Reynoso@Hggf.com|2452488| +4715|AAAAAAAALGCBAAAA|114296|2983|31317|2452649|2452619|Mr.|Garry|Lawrence|N|4|9|1960|MALAYSIA||Garry.Lawrence@JINp.org|2452616| +4716|AAAAAAAAMGCBAAAA|770298|370|46042|2450213|2450183|Ms.|Christy|Mickens|Y|10|11|1933|VANUATU||Christy.Mickens@Utr1mpqLvl.edu|2452549| +4717|AAAAAAAANGCBAAAA|100502|6354|32159|2449841|2449811|Miss|Johana|Hammonds|N|23|8|1991|ALAND ISLANDS||Johana.Hammonds@7636.edu|2452412| +4718|AAAAAAAAOGCBAAAA|857932|7151|33923|2451023|2450993|Dr.|Mark|Stone|N|18|5|1960|UNITED KINGDOM||Mark.Stone@NPCj2RbTqF.org|2452461| +4719|AAAAAAAAPGCBAAAA|1618261|3186|38064|2451054|2451024|Mr.|Francis|Hill|N|11|4|1991|MYANMAR||Francis.Hill@ZeMFsBsuGaNKRNzvJ.com|2452587| +4720|AAAAAAAAAHCBAAAA|1631065|3012|34606|2449868|2449838|Ms.|Anthony|Pitre|Y|14|9|1975|VIET NAM||Anthony.Pitre@lfIq7GI90ZL.com|2452342| +4721|AAAAAAAABHCBAAAA|483867|4441|32262|2450287|2450257|Dr.|Michael|Massey|Y|19|1|1936|FRANCE||Michael.Massey@qAJ8bh1PVLdu9L.org|2452601| +4722|AAAAAAAACHCBAAAA|504012|3814|40016|2449141|2449111|Ms.|Blanche|Thompson|N|25|4|1934|SPAIN||Blanche.Thompson@AlPi.edu|2452645| +4723|AAAAAAAADHCBAAAA|361887|6255|41423|2449289|2449259|Ms.|Barbara|Sherwood|N|8|3|1948|WESTERN SAHARA||Barbara.Sherwood@v.com|2452556| +4724|AAAAAAAAEHCBAAAA|589352|4796|21440|2452182|2452152|Mr.|Norman|Wooley|Y|28|10|1964|EGYPT||Norman.Wooley@aC.com|2452494| +4725|AAAAAAAAFHCBAAAA|318269|3246|43776|2449310|2449280|Mr.|Neil|Hunt|N|9|3|1968|GIBRALTAR||Neil.Hunt@Nc2RKr40NYSE.com|2452362| +4726|AAAAAAAAGHCBAAAA|1251758|581|46204|2451624|2451594|Dr.|Johanna|Bennett|N|19|2|1924|ISRAEL||Johanna.Bennett@So1Dsxl0ppVH.com|2452425| +4727|AAAAAAAAHHCBAAAA|1790015|1611|46807|2449330|2449300|Sir|Dan|Scott|Y|11|11|1981|NETHERLANDS||Dan.Scott@gF.org|2452530| +4728|AAAAAAAAIHCBAAAA|419237|6025|15566|2449400|2449370|Miss|Ann|Barton|N|30|10|1947|URUGUAY||Ann.Barton@BlbEqN.edu|2452526| +4729|AAAAAAAAJHCBAAAA|1739148|721|27494|2449727|2449697|Dr.|Nicholas|Johnson|Y|3|11|1938|WALLIS AND FUTUNA||Nicholas.Johnson@bddhzoeF3XXk.org|2452327| +4730|AAAAAAAAKHCBAAAA|1764910|5953|33396|2452632|2452602|Sir|Richard|Reed|N|9|12|1968|KENYA||Richard.Reed@c7oUx3ViU4FIjjS.org|2452547| +4731|AAAAAAAALHCBAAAA|207863||19306|2452119|2452089|||Frederick||18||||||| +4732|AAAAAAAAMHCBAAAA|838615|2239|49816|2449029|2448999|Dr.|Reynaldo|Butler|N|17|2|1934|KAZAKHSTAN||Reynaldo.Butler@i.edu|2452606| +4733|AAAAAAAANHCBAAAA|516262|6663|21064|2450207|2450177|Dr.|Karen|Brown|N|27|9|1938|GUAM||Karen.Brown@8yS2.com|2452592| +4734|AAAAAAAAOHCBAAAA|1042272|4981|17690|2450804|2450774|Sir|Juan|Chambers|Y|3|7|1927|MALDIVES||Juan.Chambers@GTa.edu|2452637| +4735|AAAAAAAAPHCBAAAA|769805|3094|21983|2449500|2449470|Mr.|Kenneth|Williams|Y|30|4|1969|KYRGYZSTAN||Kenneth.Williams@NCCh2DDDNxJlrN.edu|2452486| +4736|AAAAAAAAAICBAAAA|1839834|5224|31814|2450465|2450435|Miss|Sandra|Cooper|Y|22|9|1987|JAPAN||Sandra.Cooper@EPp4v4meTv.edu|2452601| +4737|AAAAAAAABICBAAAA|616875|6878|42834|2452148|2452118|Dr.|Wayne|Wolfe|N|18|6|1966|SOLOMON ISLANDS||Wayne.Wolfe@qB.com|2452433| +4738|AAAAAAAACICBAAAA|776160|5379|46951|2449066|2449036|Dr.|Joseph|Swanson|N|25|8|1936|CAMBODIA||Joseph.Swanson@JaVSk2lR8q7Ku10.com|2452335| +4739|AAAAAAAADICBAAAA|84761|5693|2625|2449818|2449788|Mr.|Gregory|Martin|Y|15|3|1958|UGANDA||Gregory.Martin@GI78ZP8x.edu|2452621| +4740|AAAAAAAAEICBAAAA|240699|4785|44918|2449097|2449067|Dr.|Philip|Lord|Y|6|5|1939|VANUATU||Philip.Lord@G5pojquZSK.com|2452620| +4741|AAAAAAAAFICBAAAA|127028|3217|23006||||||||||HUNGARY||Nona.Johnson@5Qbl0aCkz2G.edu|| +4742|AAAAAAAAGICBAAAA|665232|6173|13867|2451101|2451071|Dr.|Bobby|Schmid|N|22|5|1953|BURKINA FASO||Bobby.Schmid@LeI2XpLv.edu|2452390| +4743|AAAAAAAAHICBAAAA|21205|417|27676|2450361|2450331|Sir|John|Shelton|N|15|3|1937|SAINT LUCIA||John.Shelton@fVQoJyyi0eE.org|2452423| +4744|AAAAAAAAIICBAAAA|1279913|5168|20823|2451843|2451813|Mr.|Eugene|Jaramillo|N|19|2|1953|COMOROS||Eugene.Jaramillo@FQPoPc7RuuVIK.com|2452290| +4745|AAAAAAAAJICBAAAA|930616|2435|19054|2452269|2452239|Mr.|Ryan|Levine|Y|12|10|1927|ALAND ISLANDS||Ryan.Levine@u9Apei1izF0uF2.org|2452604| +4746|AAAAAAAAKICBAAAA|1673616|1197|29225|2452333|2452303|Sir|Sean|Bostic|N|23|5|1966|ECUADOR||Sean.Bostic@7nK0ioy6pFh677Us.org|2452465| +4747|AAAAAAAALICBAAAA|1593808|4735|40909|2450410|2450380|Miss|Dawn|Marshall|N|18|1|1967|GRENADA||Dawn.Marshall@6sJxuqHVCI.edu|2452377| +4748|AAAAAAAAMICBAAAA|267722|2797|719|2450241|2450211|Sir|Willie|Craig|Y|28|9|1940|FRENCH GUIANA||Willie.Craig@DEsCOaKfkJL0ZAu.com|2452376| +4749|AAAAAAAANICBAAAA|438227|2520|14413|2449150|2449120|Dr.|Carl|Howard|N|20|7|1966|SWITZERLAND||Carl.Howard@K5XuBJd.edu|2452367| +4750|AAAAAAAAOICBAAAA|560684|6756|32828|2450965|2450935|Dr.|Norma|Medeiros|Y|14|2|1958|ANGUILLA||Norma.Medeiros@MnMA4GxEqUXhs.com|2452448| +4751|AAAAAAAAPICBAAAA|1256811|2723|6188|2451259|2451229|Miss|Lora|Cohen|Y|4|6|1980|UKRAINE||Lora.Cohen@SZnaRVu5zj2nef.org|2452555| +4752|AAAAAAAAAJCBAAAA|1076472|2632|28392|||Dr.||||||1977||||2452614| +4753|AAAAAAAABJCBAAAA|853754|5345|40089|2449528|2449498|Sir|James|Williams|N|6|5|1964|FRENCH POLYNESIA||James.Williams@uvdEThstGAUo8VIyn.org|2452459| +4754|AAAAAAAACJCBAAAA|1728627|2165|11703|2450949|2450919|Dr.|Tammy|Crawford|N|1|4|1983|ESTONIA||Tammy.Crawford@EviGzyAMyz.org|2452638| +4755|AAAAAAAADJCBAAAA|909556|183|38810|2451787|2451757|Dr.|Dustin|Leach|Y|6|1|1935|VIET NAM||Dustin.Leach@iQ.com|2452647| +4756|AAAAAAAAEJCBAAAA|1328898|98|15298|2451937|2451907|Sir|Joseph|Worrell|N|20|10|1982|MONACO||Joseph.Worrell@MaXSvvtZ4oy.edu|2452539| +4757|AAAAAAAAFJCBAAAA|1410035|6279|9723|2450835|2450805|Mrs.|Pauline|Ruffin|Y|21|2|1931|SAN MARINO||Pauline.Ruffin@VJ3dyGzA.org|2452457| +4758|AAAAAAAAGJCBAAAA|455149|5470|35791|2450945|2450915|Dr.|Corey|Allen|Y|16|2|1960|BULGARIA||Corey.Allen@jVlyvQy3nXl.edu|2452477| +4759|AAAAAAAAHJCBAAAA|1036377|6017|46810|2449403|2449373|Sir|William|Barnett|N|26|1|1953|SPAIN||William.Barnett@EGGCycHASpJyIhHnU.com|2452569| +4760|AAAAAAAAIJCBAAAA|414417|3969|17921|2451606|2451576|Dr.|Gary|Glenn|Y|6|2|1976|CAPE VERDE||Gary.Glenn@50bCN.com|2452537| +4761|AAAAAAAAJJCBAAAA|1864815|4065|33715|2452327|2452297|Ms.|Lori|Schubert|Y|26|3|1946|CHILE||Lori.Schubert@a2v85dc.edu|2452596| +4762|AAAAAAAAKJCBAAAA|1226123|1586|42202|2450968|2450938|Ms.|Martha|Walker|N|26|4|1929|COSTA RICA||Martha.Walker@3bkiQbHzX1u.edu|2452286| +4763|AAAAAAAALJCBAAAA|1592441|493|14545|2452262|2452232|Dr.|Brandon|Zimmerman|N|6|6|1955|NEPAL||Brandon.Zimmerman@CNlh1BSO3lg2TV2UBDN.edu|2452628| +4764|AAAAAAAAMJCBAAAA|1014572|866|34360|2450967|2450937|Mrs.|Geraldine|Altman|Y|20|10|1982|CHILE||Geraldine.Altman@McIaSUUFJcla.com|2452560| +4765|AAAAAAAANJCBAAAA|609558|7072|27833|2450341|2450311|Miss|Dolly|Henderson|Y|8|1|1946|PITCAIRN||Dolly.Henderson@cPBsij.com|2452340| +4766|AAAAAAAAOJCBAAAA|843955|2096|36684|2452226|2452196|Sir|James|Dempsey|N|31|8|1942|SEYCHELLES||James.Dempsey@lNdo1u8YugYQjL.edu|2452628| +4767|AAAAAAAAPJCBAAAA|368036|5619|43249|2449847|2449817|Mr.|John|Brown|Y|15|3|1966|SLOVENIA||John.Brown@7eOE.org|2452405| +4768|AAAAAAAAAKCBAAAA|434384|950|27869|2451036|2451006|Miss|Marta|John|Y|1|3|1948|TONGA||Marta.John@n1PlDVEmYfX.com|2452441| +4769|AAAAAAAABKCBAAAA|902556|6146|28889|2449339|2449309|Mr.|Barry|Hurley|N|25|2|1981|NEPAL||Barry.Hurley@K.org|2452389| +4770|AAAAAAAACKCBAAAA|198333|1119|21858|2449728|2449698|Dr.|Cesar|Ruff|N|20|3|1981|PHILIPPINES||Cesar.Ruff@V.edu|2452467| +4771|AAAAAAAADKCBAAAA|1801091|6778|14865|2452544|2452514|Mrs.|Sondra|Barnes|N|9|3|1948|SAN MARINO||Sondra.Barnes@C8nBTCM3K.com|2452339| +4772|AAAAAAAAEKCBAAAA|1914680|3217|20107|2450857|2450827|Mr.|Adalberto|Everett|N|20|6|1951|R�UNION||Adalberto.Everett@aq1SvCYcg5.org|2452609| +4773|AAAAAAAAFKCBAAAA|61504|4150|23196|2450482|2450452|Mrs.|Alta|Martin|Y|8|2|1972|FIJI||Alta.Martin@83SG.com|2452353| +4774|AAAAAAAAGKCBAAAA|1617273|3888|26550|2452278|2452248|Mr.|Manuel|Guerrero|Y|4|5|1953|CAPE VERDE||Manuel.Guerrero@Gtf.org|2452295| +4775|AAAAAAAAHKCBAAAA|892443|881|29272|2449519|2449489|Ms.|Elizabeth|Hurst|N|4|12|1935|KYRGYZSTAN||Elizabeth.Hurst@FZ.edu|2452362| +4776|AAAAAAAAIKCBAAAA|674992|455|6318|2452251|2452221|Mr.|Brandon|Torres|N|4|1|1977|UKRAINE||Brandon.Torres@I4Fu03XHB05s.edu|2452544| +4777|AAAAAAAAJKCBAAAA|39825|282|6386|2450216|2450186|Mr.|Robert|Washington|N|17|8|1988|C�TE D'IVOIRE||Robert.Washington@UtgN8JVfxGn86f5t.com|2452525| +4778|AAAAAAAAKKCBAAAA|1650791|920|28296|2450084|2450054|Mrs.|Vivian|Francis|N|24|5|1991|PARAGUAY||Vivian.Francis@5KFbnLj1oYb.edu|2452506| +4779|AAAAAAAALKCBAAAA|1013886|6638|20990|2449079|2449049|Mr.|Scott|Stanton|Y|9|6|1965|PHILIPPINES||Scott.Stanton@5f6hJMNRKyEIhnn.org|2452488| +4780|AAAAAAAAMKCBAAAA||2249|21239||2451767||Sherry|Ybarra||5|10||LESOTHO||Sherry.Ybarra@Oe0m3pkGe2nrVY0.com|2452520| +4781|AAAAAAAANKCBAAAA|735513|91|8222|2449507|2449477|Dr.|Shelia|Vela|N|10|3|1961|VIET NAM||Shelia.Vela@xvF3EaLfeOVZb.com|2452541| +4782|AAAAAAAAOKCBAAAA|1702589|4772|16555|2449829|2449799|Mr.|Larry|Joseph|Y|11|9|1979|INDIA||Larry.Joseph@TMu2QRahoCYJeja7AV.com|2452580| +4783|AAAAAAAAPKCBAAAA|1825339|944|35302|2451972|2451942|Mr.|Brian|Stein|Y|7|6|1965|HAITI||Brian.Stein@9YpeT4kOvPCcqAT.edu|2452564| +4784|AAAAAAAAALCBAAAA|820186|3546|29659|2452338|2452308|Dr.|George|Scott|Y|30|5|1945|PUERTO RICO||George.Scott@Gp7O8SxZXijBUAiSFR.org|2452546| +4785|AAAAAAAABLCBAAAA|359730|802|39521|2451717|2451687|Ms.|Irene|Gill|N|13|3|1969|VIET NAM||Irene.Gill@5057K4dQHDKH0BR6.edu|2452488| +4786|AAAAAAAACLCBAAAA|620193|6821|11049|2449599|2449569|Mr.|Sheldon|Meza|N|9|3|1939|GAMBIA||Sheldon.Meza@FK6slERp0sG.org|2452462| +4787|AAAAAAAADLCBAAAA|754419|2735|17451|2449819|2449789|Sir|Reginald|Green|N|14|2|1936|GREECE||Reginald.Green@RsmlvSRt43o.com|2452298| +4788|AAAAAAAAELCBAAAA|1857457|916|30775|2449358|2449328|Dr.|Peggy|Harper|N|27|9|1953|GUERNSEY||Peggy.Harper@CEZSPK1h.com|2452305| +4789|AAAAAAAAFLCBAAAA|1527921|3646|10571|2450967|2450937|Dr.|Freda|Todd|N|5|6|1945|ZIMBABWE||Freda.Todd@danEshJEQN.com|2452587| +4790|AAAAAAAAGLCBAAAA|78956|6687|20818|2452408|2452378|Ms.|Regina|Patel|N|24|5|1961|CAMEROON||Regina.Patel@6GblAU1ofI.org|2452552| +4791|AAAAAAAAHLCBAAAA|248371|2074|14577|2451200|2451170|Dr.|Dennis|Ashley|N|25|5|1963|CUBA||Dennis.Ashley@XNeLMuc.com|2452306| +4792|AAAAAAAAILCBAAAA|749316|2609|8302|2449183|2449153|Miss|Elizabeth|Simpson|N|7|3|1987|AFGHANISTAN||Elizabeth.Simpson@Ng3d.edu|2452300| +4793|AAAAAAAAJLCBAAAA|779926|3414|38102|2450163|2450133|Dr.|Kathleen|Pearson|N|27|1|1962|UGANDA||Kathleen.Pearson@VShQi1.com|2452510| +4794|AAAAAAAAKLCBAAAA|1694164|5294|18024|2449749|2449719|Dr.|Douglas|Biggs|N|8|9|1932|AUSTRALIA||Douglas.Biggs@siKJq4Tbs.edu|2452309| +4795|AAAAAAAALLCBAAAA|1573934|1525|9069|2451114|2451084|Dr.|Kari|Davis|N|27|7|1966|SOLOMON ISLANDS||Kari.Davis@44ZtuPvjc5ks6.edu|2452366| +4796|AAAAAAAAMLCBAAAA|1033721||11136|2451725|2451695|Dr.|||||2||GUATEMALA||John.Howard@2FPvikS8CVIG.edu|2452546| +4797|AAAAAAAANLCBAAAA|764190|375|10607|2452407|2452377|Sir|Donald|Rodriguez|N|31|1|1957|NORFOLK ISLAND||Donald.Rodriguez@JGgEJmCOSy.com|2452493| +4798|AAAAAAAAOLCBAAAA|240013|6958|32703|2452187|2452157|Sir|Wendell|Batts|Y|8|8|1933|FRANCE||Wendell.Batts@vQYak.com|2452595| +4799|AAAAAAAAPLCBAAAA|1403297|6784|49871|2449442|2449412|Ms.|Antoinette|Schmidt|Y|22|9|1926|SOMALIA||Antoinette.Schmidt@6S0mcADLzuxrLPk.org|2452318| +4800|AAAAAAAAAMCBAAAA|1825264|6781|48651|2452527|2452497|Dr.|Brenda|Shannon|N|18|12|1992|URUGUAY||Brenda.Shannon@GBDUhuNQH.com|2452395| +4801|AAAAAAAABMCBAAAA|697330|2771|15812|2449665|2449635|Ms.|Ashley|Healy|Y|21|8|1928|ZIMBABWE||Ashley.Healy@KnQRJZsOs5M2IkY.edu|2452338| +4802|AAAAAAAACMCBAAAA|878491|2565|45486|2451023|2450993|Dr.|Blake|Thomas|Y|19|3|1972|SAINT HELENA||Blake.Thomas@fj5zyokmNY.com|2452534| +4803|AAAAAAAADMCBAAAA|381893|2796|35252|2451634|2451604|Mr.|Robert|Wyman|Y|23|9|1987|MALI||Robert.Wyman@5yRU7iHI9l.com|2452484| +4804|AAAAAAAAEMCBAAAA|581312|3208|19904|2449137|2449107|Mr.|Britt|Miller|Y|27|8|1926|BELGIUM||Britt.Miller@QGHURPKQENVH70.com|2452602| +4805|AAAAAAAAFMCBAAAA|1576901|6786|21518|2450538|2450508|Ms.|Allen|Wilson|Y|7|9|1963|POLAND||Allen.Wilson@gfZABsOOM2g0.org|2452490| +4806|AAAAAAAAGMCBAAAA|998301|5490|15479|2451543|2451513|Dr.|Lucille|Ortiz|N|21|7|1925|FINLAND||Lucille.Ortiz@mvjicZqso3kX3g.edu|2452601| +4807|AAAAAAAAHMCBAAAA|1180451|552|1853|2450764|2450734|Mrs.|Kathy|Bass|Y|19|2|1936|NAURU||Kathy.Bass@5gmJkURhPTeo.org|2452563| +4808|AAAAAAAAIMCBAAAA|1771766|3175|39431|2449717|2449687|Sir|Burl|Merrill|Y|11|8|1939|MARTINIQUE||Burl.Merrill@oNGTdktriIrSMZ2rsN.com|2452495| +4809|AAAAAAAAJMCBAAAA|746571|2803|18188|2450991|2450961|Sir|Joseph|Tompkins|N|2|1|1943|GREECE||Joseph.Tompkins@aN.com|2452360| +4810|AAAAAAAAKMCBAAAA|745000|3590|9827|2450501|2450471|Mrs.|Emma|Elizondo|N|28|8|1943|PAKISTAN||Emma.Elizondo@iXV1xjHo1.com|2452571| +4811|AAAAAAAALMCBAAAA|1764298|6078|37458|2449308|2449278|Dr.|Brian|Moore|Y|28|2|1969|LITHUANIA||Brian.Moore@c2Xp6a.org|2452633| +4812|AAAAAAAAMMCBAAAA|266873|4572|5071|2449216|2449186|Mr.|Walter|Murphy|Y|7|4|1953|AUSTRIA||Walter.Murphy@2CZKBzkjfi1L.org|2452292| +4813|AAAAAAAANMCBAAAA|921776|6271|25119|2452065|2452035|Sir|Christopher|Arellano|Y|6|1|1947|KENYA||Christopher.Arellano@RaCO78rzbjYHon.com|2452295| +4814|AAAAAAAAOMCBAAAA|1335126|2865|45757|2450785|2450755|Dr.|Lisa|Anderson|Y|28|1|1987|MOLDOVA, REPUBLIC OF||Lisa.Anderson@rt7.edu|2452510| +4815|AAAAAAAAPMCBAAAA|1827323|5712|15254|2449995|2449965|Ms.|Eleanor|Santos|Y|5|3|1955|PARAGUAY||Eleanor.Santos@mdkPycAVX0vFRhts3.edu|2452435| +4816|AAAAAAAAANCBAAAA|298861|2897|18133|2451236|2451206|Mr.|Jose|Jimenez|Y|29|8|1945|NETHERLANDS||Jose.Jimenez@j.org|2452645| +4817|AAAAAAAABNCBAAAA|1744871|5601|33931|2451078|2451048|Mr.|John|Santiago|Y|7|4|1985|SWITZERLAND||John.Santiago@aYtL0E2t8EZbq4j.com|2452296| +4818|AAAAAAAACNCBAAAA|116451|2828|28052|2451476|2451446|Miss|Michelle|Evans|N|16|7|1928|BOUVET ISLAND||Michelle.Evans@8BnRDCb5.org|2452351| +4819|AAAAAAAADNCBAAAA|1389115|1617|38519|2449080|2449050|Sir|Abel|Parsons|Y|3|4|1959|VENEZUELA||Abel.Parsons@CImqOAE4VbV.com|2452561| +4820|AAAAAAAAENCBAAAA|1743482|5511|23587|2449419|2449389|Dr.|Ralph|Brennan|N|23|6|1948|CUBA||Ralph.Brennan@HErNEL30.edu|2452433| +4821|AAAAAAAAFNCBAAAA|1480580|3064|33901|2450627|2450597|Ms.|Anna|Faulk|N|12|10|1956|ICELAND||Anna.Faulk@AnankRqDdnfXkC.com|2452326| +4822|AAAAAAAAGNCBAAAA|410222|1942|34909|2449411|2449381|Dr.|Monica|Martinez|N|1|11|1954|ZAMBIA||Monica.Martinez@qj7.com|2452613| +4823|AAAAAAAAHNCBAAAA|1588297|1315|2686|2450658|2450628|Ms.|Julie|Bock|N|6|2|1945|TAJIKISTAN||Julie.Bock@2GqtFeTASb0pzuO.com|2452509| +4824|AAAAAAAAINCBAAAA|1355917|295|41873|2450232|2450202|Ms.|Ruby|Bullock|N|10|9|1947|MAURITANIA||Ruby.Bullock@7Hu5XC4Z6Q.edu|2452573| +4825|AAAAAAAAJNCBAAAA|363912|76|33393|2449140|2449110|Dr.|Marian|Bryant|N|9|5|1936|BAHRAIN||Marian.Bryant@ufXDl5pg45M.edu|2452568| +4826|AAAAAAAAKNCBAAAA|286284||26613|2451373|||James|Richardson|Y||3|1946||||| +4827|AAAAAAAALNCBAAAA|918837|5825|25613|2450095|2450065|Mr.|James|Massie|Y|27|5|1951|NEW CALEDONIA||James.Massie@HQIKro48HYVCHPsE.org|2452480| +4828|AAAAAAAAMNCBAAAA|94256|1685|17114|2450427|2450397|Mrs.|Amanda|Santiago|Y|4|4|1938|ANGUILLA||Amanda.Santiago@zTKX12pQe.com|2452498| +4829|AAAAAAAANNCBAAAA|1785669|69|14116|2451531|2451501|Sir|Samuel|Ruiz|Y|16|9|1987|PUERTO RICO||Samuel.Ruiz@Ley1xc1eehKjZQ98.edu|2452338| +4830|AAAAAAAAONCBAAAA|623755|3336|48517|2451371|2451341|Mr.|Eric|Switzer|Y|13|1|1970|UNITED STATES||Eric.Switzer@vMv4PiRTzYLayVBb3ne.org|2452394| +4831|AAAAAAAAPNCBAAAA|1392945|4939|28886|2452646|2452616|Mrs.|Chastity|Dodge|Y|22|8|1944|UZBEKISTAN||Chastity.Dodge@KJXX3NB.org|2452356| +4832|AAAAAAAAAOCBAAAA|1885002|765|22928|2451018|2450988|Mr.|Virgil|Ramsey|N|12|6|1955|LATVIA||Virgil.Ramsey@tl4VlPyK47L.org|2452357| +4833|AAAAAAAABOCBAAAA|815849|6211|16220|2451914|2451884|Mr.|Robert|Gilbert|N|28|6|1971|BURUNDI||Robert.Gilbert@RhT4vQtRlSOL.com|2452468| +4834|AAAAAAAACOCBAAAA|1381261|2192|6770|2449559|2449529|Mrs.|Beverly|Patterson|N|3|1|1963|RWANDA||Beverly.Patterson@RV25nHqlZgaRYUoVlRRh.edu|2452523| +4835|AAAAAAAADOCBAAAA|1652684|5727|323|2452499|2452469|Mr.|Bruce|Larson|N|24|4|1975|CYPRUS||Bruce.Larson@TZPTHP0oL0h.com|2452298| +4836|AAAAAAAAEOCBAAAA|1791887|3529|18771|2450234|2450204|Mrs.|Angela|Banks|N|5|9|1978|INDONESIA||Angela.Banks@XbaP.com|2452483| +4837|AAAAAAAAFOCBAAAA|133909|3484|33859|2450956|2450926|Mr.|Steven|Cintron|Y|13|12|1974|KUWAIT||Steven.Cintron@oIAD1qJUpM.com|2452410| +4838|AAAAAAAAGOCBAAAA|627494|1075|49103|2449852|2449822|Mr.|Alan|Phillips|Y|13|8|1965|SWAZILAND||Alan.Phillips@nvKrDMNo7n.org|2452339| +4839|AAAAAAAAHOCBAAAA|1158913|1933|23400|2451886|2451856|Miss|Margaret|Simpson|N|15|4|1946|GREENLAND||Margaret.Simpson@M8oT60jY7p.edu|2452341| +4840|AAAAAAAAIOCBAAAA|933624|5088|18152|2451120|2451090|Mr.|Robert|Riley|Y|18|4|1938|BOLIVIA||Robert.Riley@cJBp.com|2452559| +4841|AAAAAAAAJOCBAAAA|1052357|1298|38639|2450216|2450186|Sir|Robert|Jones|Y|10|6|1961|EQUATORIAL GUINEA||Robert.Jones@cACtKjB9nHd.edu|2452287| +4842|AAAAAAAAKOCBAAAA|1917264|7135|30559|2451122|2451092|Dr.|Rina|Box|N|2|4|1965|GREENLAND||Rina.Box@Avp7uv3JY65Mf.com|2452285| +4843|AAAAAAAALOCBAAAA|||44016|2450225|||||Y|22|9|1989|BRUNEI DARUSSALAM||Nena.Phelps@cCTDhED0F.com|| +4844|AAAAAAAAMOCBAAAA|926497|6347|42550|2452135|2452105|Mrs.|Sandra|Clark|N|5|2|1953|LITHUANIA||Sandra.Clark@d.com|2452579| +4845|AAAAAAAANOCBAAAA|452747|1830|24113|2450372|2450342|Mrs.|Christina|Lang|N|23|6|1938|LEBANON||Christina.Lang@L4tuLIeLA2Yh.edu|2452550| +4846|AAAAAAAAOOCBAAAA|1202698|6862|37026|2452126|2452096|Mr.|John|Ortiz|N|19|5|1941|GIBRALTAR||John.Ortiz@vCoe3PnV.org|2452475| +4847|AAAAAAAAPOCBAAAA|1687510|1076|22242|2452623|2452593|Ms.|Marsha|Delarosa|N|16|12|1938|VENEZUELA||Marsha.Delarosa@mkQ4c3Sx87Rym.com|2452423| +4848|AAAAAAAAAPCBAAAA|1268469|328|16234|2449603|2449573|Ms.|Sallie|Fletcher|Y|22|2|1985|JAMAICA||Sallie.Fletcher@iu1v8Eyns.com|2452298| +4849|AAAAAAAABPCBAAAA|787076|4091|13953|2451637|2451607|Dr.|Juan|Jacobson|N|3|7|1991|GUERNSEY||Juan.Jacobson@fdhQab.com|2452597| +4850|AAAAAAAACPCBAAAA|317387|4799|9411|2449356|2449326|Dr.|Gary|Graham|Y|13|5|1952|QATAR||Gary.Graham@nHp3NzqBAjBgYmb.com|2452506| +4851|AAAAAAAADPCBAAAA||1941|8639||2450994||Eugene||Y||2|||||2452349| +4852|AAAAAAAAEPCBAAAA|407366|5333|45747|2451964|2451934|Ms.|Maryalice|Rivers|Y|26|12|1938|PAPUA NEW GUINEA||Maryalice.Rivers@4b9MXdByVXI.com|2452283| +4853|AAAAAAAAFPCBAAAA|1791007|2701|22795|2451091|2451061|Dr.|Stanley|Gonzalez|Y|25|3|1987|AUSTRIA||Stanley.Gonzalez@72E.org|2452417| +4854|AAAAAAAAGPCBAAAA|294022|2225|21275|2449925|2449895|Dr.|James|Fleming|Y|21|9|1946|PERU||James.Fleming@TzYvYJquZ.com|2452575| +4855|AAAAAAAAHPCBAAAA|400546|2934|10122|2451065|2451035|Sir|Gary|Troy|Y|16|12|1978|MOZAMBIQUE||Gary.Troy@LmNhz7p9FH.edu|2452512| +4856|AAAAAAAAIPCBAAAA||1078|16387||2449911|Dr.|Cesar|Bloom||26||1939|ANTIGUA AND BARBUDA||Cesar.Bloom@IFq2pe9.edu|2452429| +4857|AAAAAAAAJPCBAAAA|1094037|4675|22960|2449210|2449180|Mrs.|Andrew|Bowser|Y|8|5|1954|SWAZILAND||Andrew.Bowser@CL8TqKFDMoKOImdyTD.org|2452424| +4858|AAAAAAAAKPCBAAAA|970053|2795|19537|2451856|2451826|Dr.|Willie|Barnes|N|5|5|1951|HAITI||Willie.Barnes@hIGETCEvYOAlB.edu|2452411| +4859|AAAAAAAALPCBAAAA|796817|3554|36644|2452069|2452039|Mr.|Ignacio|Dean|Y|5|3|1924|GRENADA||Ignacio.Dean@3kQ9axG7BTE6.org|2452539| +4860|AAAAAAAAMPCBAAAA|587928|263|2998|2451916|2451886|Dr.|Earnest|Williams|N|26|1|1987|HAITI||Earnest.Williams@R.com|2452333| +4861|AAAAAAAANPCBAAAA|1369580|6889|39398|2451326||Mrs.|Carolyn||N|5|2|1943||||2452436| +4862|AAAAAAAAOPCBAAAA|1812335|6918|33901|2451790|2451760|Mrs.|Nancy|Wilson|Y|8|7|1931|TUVALU||Nancy.Wilson@oTsPGEa67sFbDs.com|2452328| +4863|AAAAAAAAPPCBAAAA|104512|6427|10295|2450965|2450935|Mr.|David|Hall|N|22|4|1980|KENYA||David.Hall@fv0H4ESzmtYeueoo.com|2452548| +4864|AAAAAAAAAADBAAAA|1096090|6151|5765|2449763|2449733|Sir|Roberto|Jenkins|N|4|4|1973|ZIMBABWE||Roberto.Jenkins@aRx.edu|2452575| +4865|AAAAAAAABADBAAAA|800565|2232|27546|2452020|2451990|Dr.|Juan|Quillen|N|13|2|1965|UGANDA||Juan.Quillen@o.edu|2452566| +4866|AAAAAAAACADBAAAA|1827071|4649|13427|2451250|2451220|Sir|Michael|Chamberlin|N|1|8|1952|MOZAMBIQUE||Michael.Chamberlin@qsdiGbd.edu|2452494| +4867|AAAAAAAADADBAAAA|11842|2294|47146|2452674|2452644|Miss|Cristine|Ruff|Y|13|4|1965|GHANA||Cristine.Ruff@QZJ6RxHIc.org|2452469| +4868|AAAAAAAAEADBAAAA|1462761|4884|33454|2449197|2449167|Dr.|Jeff|Schmidt|N|1|4|1986|MEXICO||Jeff.Schmidt@ysuU7dS4KgEd3kRBJ2.com|2452580| +4869|AAAAAAAAFADBAAAA|1014285|4542|35152|2450381|2450351|Mr.|Thomas|Smith|Y|26|8|1941|ISLE OF MAN||Thomas.Smith@GUVC9gKbh9SBkK.com|2452496| +4870|AAAAAAAAGADBAAAA|1404423|2160|32344|2449758|2449728|Mrs.|Nadia|Miller|N|9|8|1979|ANGUILLA||Nadia.Miller@9qnDSQO5TtxKeBcoA.edu|2452302| +4871|AAAAAAAAHADBAAAA|1511958|3327|12696||2451544|Dr.||Miles|||9||MALDIVES||Gregory.Miles@0Qr6Jq9zYpQbzbTY.org|2452350| +4872|AAAAAAAAIADBAAAA|361445|28|3058|2450046|2450016|Ms.|Vickie|Simon|N|18|1|1974|WESTERN SAHARA||Vickie.Simon@sIbZ4Li3n.org|2452463| +4873|AAAAAAAAJADBAAAA|515341|7005|44269|2451660|2451630|Ms.|Kathy|Mccarthy|N|4|12|1992|C�TE D'IVOIRE||Kathy.Mccarthy@zbfaouQKmzILt.com|2452493| +4874|AAAAAAAAKADBAAAA||1100|20632|||||||30|4||BELGIUM||Lisa.Jones@l02Fj.edu|| +4875|AAAAAAAALADBAAAA|1712209|619|26182|2452356|2452326|Mr.|Lawrence|Henderson|Y|25|8|1960|MALDIVES||Lawrence.Henderson@8xkqhN8eYeV.edu|2452511| +4876|AAAAAAAAMADBAAAA|434817|1712|9054|2449724|2449694|Dr.|Rosa|Smith|Y|20|5|1929|GIBRALTAR||Rosa.Smith@JhOLobPdNHk.org|2452428| +4877|AAAAAAAANADBAAAA|698335|554|32373|2449218|2449188|Miss|Elaine|Brown|Y|18|4|1985|SAN MARINO||Elaine.Brown@fV2z9hfrF3v.org|2452597| +4878|AAAAAAAAOADBAAAA|987152|6333|39067|2449853|2449823|Dr.|Robert|Woodcock|N|3|10|1932|PORTUGAL||Robert.Woodcock@j9705sDzGJrh5bOB3p.com|2452567| +4879|AAAAAAAAPADBAAAA|758150|5992|23434|2450198|2450168|Dr.|Kimberly|Walker|N|30|12|1942|PAPUA NEW GUINEA||Kimberly.Walker@yK9jV.com|2452366| +4880|AAAAAAAAABDBAAAA|1700846|6134|222|2451432|2451402|Mrs.|Susan|Mccormick|N|10|8|1945|MALTA||Susan.Mccormick@0nZCxlEsVFd0zAZ.org|2452303| +4881|AAAAAAAABBDBAAAA|||14494||2449701|Dr.|Clyde|Cox|Y||7||||Clyde.Cox@a0kLKlv.org|| +4882|AAAAAAAACBDBAAAA|750238|251|8664|2451275|2451245|Mr.|Richard|Cartwright|N|4|7|1963|CYPRUS||Richard.Cartwright@HcN.edu|2452430| +4883|AAAAAAAADBDBAAAA|1913959|3754|19549|2450688|2450658|Dr.|Lawrence|Scott|N|19|7|1974|TOGO||Lawrence.Scott@4kq23JA3Brqy.edu|2452309| +4884|AAAAAAAAEBDBAAAA|1743795|4366|30184|2450200|2450170|Mr.|Charles|Smith|Y|16|4|1974|KUWAIT||Charles.Smith@AHCF.com|2452541| +4885|AAAAAAAAFBDBAAAA|1057141|3266|44191|2451312|2451282|Mr.|Timothy|Mccoy|N|10|11|1943|BURUNDI||Timothy.Mccoy@0b0.edu|2452347| +4886|AAAAAAAAGBDBAAAA|788464|4975|34639|2449158|2449128|Mr.|Jack|Esquivel|Y|26|8|1946|SUDAN||Jack.Esquivel@G5pZxJoiaL.org|2452569| +4887|AAAAAAAAHBDBAAAA|36374|3172|17523|2452663|2452633|Mr.|Jose|Drake|N|24|10|1926|PERU||Jose.Drake@5GAUoB.edu|2452394| +4888|AAAAAAAAIBDBAAAA|1617171|1046|3674|2449778|2449748|Sir|David|Blair|N|21|4|1975|HUNGARY||David.Blair@ca05i0LM8NG7s8oi.edu|2452523| +4889|AAAAAAAAJBDBAAAA|1143561|6442|40123|2449863|2449833|Ms.|Glenna|Riley|Y|18|7|1973|SIERRA LEONE||Glenna.Riley@d98mbXpXibLFZ.com|2452383| +4890|AAAAAAAAKBDBAAAA|823721|5327|30886|2451467|2451437|Mr.|Lawrence|Ross|N|2|1|1934|MALDIVES||Lawrence.Ross@XPGa4uJRtP.com|2452284| +4891|AAAAAAAALBDBAAAA|750547|1625|35056|2449524|2449494|Mr.|Marion|Murray|Y|4|4|1929|LESOTHO||Marion.Murray@Xht.edu|2452446| +4892|AAAAAAAAMBDBAAAA|991473|5637|45801|2451418|2451388|Sir|Howard|Luttrell|Y|27|6|1991|UKRAINE||Howard.Luttrell@BnZ7f7nfM.com|2452492| +4893|AAAAAAAANBDBAAAA||3611|25720|||Dr.|Vivian||Y|||1975||||2452582| +4894|AAAAAAAAOBDBAAAA|1439907|6874|26641|2450828|2450798|Miss|Sheri|Ames|N|28|4|1932|MONGOLIA||Sheri.Ames@VqF9VaIMAgc0.org|2452377| +4895|AAAAAAAAPBDBAAAA|168208|1394|5380|2450059|2450029|Miss|Diane|Knott|N|13|12|1947|PALAU||Diane.Knott@ajE0enfsU5i.edu|2452289| +4896|AAAAAAAAACDBAAAA|159233|5704|42975|2451710|2451680|Miss|Norma|Harrison|Y|19|4|1933|ANTIGUA AND BARBUDA||Norma.Harrison@o.com|2452467| +4897|AAAAAAAABCDBAAAA|1655270|4992|25867|2450073|2450043|Miss|Theresa|James|N|7|11|1983|CAYMAN ISLANDS||Theresa.James@CR75eLEJ6ir.com|2452624| +4898|AAAAAAAACCDBAAAA|587472|3003|28487|2451935|2451905|Miss|Violet|Dotson|Y|8|9|1989|GUAM||Violet.Dotson@mMAIrKZ.com|2452346| +4899|AAAAAAAADCDBAAAA|1651966|3629|5245|2451562|2451532|Mr.|Franklin|Stout|Y|23|4|1956|BRUNEI DARUSSALAM||Franklin.Stout@E6Mf.org|2452534| +4900|AAAAAAAAECDBAAAA|1357619|796|7573|2451888|2451858|Sir|David|Hudson|Y|13|1|1937|PARAGUAY||David.Hudson@1.com|2452525| +4901|AAAAAAAAFCDBAAAA|148927|2208|39536|2451077|2451047|Ms.|Karin|Helton|Y|28|6|1989|ICELAND||Karin.Helton@ghoFVcSVIvvIRlhpnv7.com|2452304| +4902|AAAAAAAAGCDBAAAA|1019025|1746|12467|2451862|2451832|Sir|Roger|Williams|Y|14|8|1977|TUVALU||Roger.Williams@OYDt4rEU1P.org|2452291| +4903|AAAAAAAAHCDBAAAA|338381|6585|9189|2449720|2449690|Dr.|Ernest|Thayer|N|5|5|1974|SIERRA LEONE||Ernest.Thayer@HNjbp.org|2452305| +4904|AAAAAAAAICDBAAAA|971129|5118|41860|2451366|2451336|Ms.|Phyllis|Grant|Y|11|6|1970|ALGERIA||Phyllis.Grant@kaA.org|2452524| +4905|AAAAAAAAJCDBAAAA|142931|1829|3127|2451921|2451891|Mr.|Peter|Stahl|N|14|9|1971|NAURU||Peter.Stahl@LYpxz3fCdQHVUI.org|2452494| +4906|AAAAAAAAKCDBAAAA|1340279|3896|21560|2450125|2450095|Mr.|Kevin|William|Y|9|8|1950|MONTSERRAT||Kevin.William@AoHR2vQZXMQ.com|2452604| +4907|AAAAAAAALCDBAAAA|1014163|2255|11147|2451151|2451121|Miss|Tiffany|Smallwood|Y|8|1|1981|BELARUS||Tiffany.Smallwood@QUGHC.edu|2452414| +4908|AAAAAAAAMCDBAAAA|1186070|6861|12099|2450294|2450264|Sir|John|Walden|Y|9|11|1925|BELIZE||John.Walden@pjOEVmZp34n0t.com|2452456| +4909|AAAAAAAANCDBAAAA|1715596|430|38064|2450493|2450463|Mr.|Alphonso|Smith|Y|7|12|1928|NORWAY||Alphonso.Smith@OmOlycOKir.org|2452521| +4910|AAAAAAAAOCDBAAAA|1038686|5447|5408|2450907|2450877|Mr.|Jose|Kenney|N|5|1|1983|CHINA||Jose.Kenney@RB.org|2452628| +4911|AAAAAAAAPCDBAAAA|1424020|245|8023|2450411|2450381|Mr.|Ivan|Sheppard|N|24|12|1954|PAKISTAN||Ivan.Sheppard@gPZPMjLmaO9iH93.com|2452629| +4912|AAAAAAAAADDBAAAA|160678|7195|11359|2449387|2449357|Dr.|Juanita|Blount|N|27|4|1947|ALGERIA||Juanita.Blount@sg5Ne4e963Y0x.edu|2452460| +4913|AAAAAAAABDDBAAAA|944964|543|12969|2452590|2452560|Sir|Alfredo|Morales|Y|24|8|1929|NAMIBIA||Alfredo.Morales@Oi.org|2452620| +4914|AAAAAAAACDDBAAAA|904784|245|38059|2450569|2450539|Sir|Robert|Holmes|N|20|9|1935|BENIN||Robert.Holmes@cur.org|2452555| +4915|AAAAAAAADDDBAAAA|1727269|4522|19822|2450212|2450182|Mr.|Fernando|Carlson|Y|13|2|1935|GIBRALTAR||Fernando.Carlson@qVbgRkGfGo.edu|2452597| +4916|AAAAAAAAEDDBAAAA|247728|2584|17040|2449953|2449923|Ms.|Trina|Wilson|Y|7|9|1959|JORDAN||Trina.Wilson@KFcovtXrX.com|2452594| +4917|AAAAAAAAFDDBAAAA|1610093||13105||2451698|Miss|Crystal|Morgan|N||7|1953|SWITZERLAND||Crystal.Morgan@rx004Nf9NCg.com|| +4918|AAAAAAAAGDDBAAAA|1457241|3320|39046|2451812|2451782|Sir|Randy|Beaulieu|N|15|8|1962|NEW CALEDONIA||Randy.Beaulieu@Sb7JokEr.org|2452585| +4919|AAAAAAAAHDDBAAAA|1616903|3634|43165|2451404|2451374|Mr.|Willie|Mcewen|N|24|11|1950|LATVIA||Willie.Mcewen@A.com|2452361| +4920|AAAAAAAAIDDBAAAA|347030|3439|31095|2452323|2452293|Mr.|Nicholas|Hamilton|Y|13|8|1955|BAHAMAS||Nicholas.Hamilton@JArXAqR.edu|2452624| +4921|AAAAAAAAJDDBAAAA|1262559|6303|46847|2452585|2452555|Dr.|June|Blocker|Y|26|7|1931|MAURITANIA||June.Blocker@16gSop4osBL.edu|2452475| +4922|AAAAAAAAKDDBAAAA|452873|1770|19020|2450372|2450342|Dr.|Milton|Newton|Y|28|11|1951|LATVIA||Milton.Newton@k8rI.edu|2452526| +4923|AAAAAAAALDDBAAAA|563359|835|20311|2449610|2449580|Sir|Joshua|Layman|N|6|2|1961|HONDURAS||Joshua.Layman@jcUDsFZlY3SD.org|2452529| +4924|AAAAAAAAMDDBAAAA|1840079|4174|33069|2452519|2452489|Sir|Andres|Wilson|Y|24|3|1956|UNITED ARAB EMIRATES||Andres.Wilson@Hpd86iS2.org|2452449| +4925|AAAAAAAANDDBAAAA|1614532|3129|47773|2451993|2451963|Ms.|Gladys|Hernandez|N|5|3|1942|UNITED STATES||Gladys.Hernandez@jv7LVgls7Qyvqb.org|2452643| +4926|AAAAAAAAODDBAAAA|1278194|5018|11042|2449505|2449475|Dr.|Thomas|Warren|Y|3|7|1989|ANGOLA||Thomas.Warren@1PNyzJIEfdRlv3T.com|2452555| +4927|AAAAAAAAPDDBAAAA|739582|1826|2517|2451077|2451047|Sir|Jack|Henderson|N|26|1|1960|ISRAEL||Jack.Henderson@kcoHV6pAOH9hU.com|2452507| +4928|AAAAAAAAAEDBAAAA|271755|6472|1942|2452042|2452012|Ms.|Socorro|Knight|Y|7|12|1963|TIMOR-LESTE||Socorro.Knight@stLZikT7kbGKpPUx.org|2452534| +4929|AAAAAAAABEDBAAAA|783335|944|43800|2452490|2452460|Dr.|Pearl|Johnson|N|11|11|1960|TIMOR-LESTE||Pearl.Johnson@M42aAIbm.org|2452403| +4930|AAAAAAAACEDBAAAA|274851|1994|13937|2449851|2449821|Mr.|Kenneth|Johnson|N|18|11|1984|GUADELOUPE||Kenneth.Johnson@jk1NOjL04y.com|2452473| +4931|AAAAAAAADEDBAAAA|341410|2804|14151|2449620|2449590|Dr.|Phillip|Martel|N|16|6|1980|GUADELOUPE||Phillip.Martel@0pr2z6qdCM6ubt0rac.org|2452648| +4932|AAAAAAAAEEDBAAAA|1160133|3289|20892|2450032|2450002|Ms.|Kimberly|Garner|Y|7|9|1976|SAINT HELENA||Kimberly.Garner@rtFX5OADjgZQVq7.com|2452453| +4933|AAAAAAAAFEDBAAAA|512001|3759|43564|2451423|2451393|Mrs.|Linda|Silver|Y|19|12|1928|AZERBAIJAN||Linda.Silver@9YJIadx.com|2452378| +4934|AAAAAAAAGEDBAAAA|479723|4668|46413|2451247|2451217|Dr.|Victor|Ball|N|23|8|1946|MALAWI||Victor.Ball@UHpdrBCvj.org|2452605| +4935|AAAAAAAAHEDBAAAA|1822102|3532|17762|2452172|2452142|Sir|John|Murray|N|2|1|1980|PITCAIRN||John.Murray@iKg3.com|2452626| +4936|AAAAAAAAIEDBAAAA|431476|4135|35641|2449236|2449206|Mr.|Lewis|Bowen|N|22|10|1962|TIMOR-LESTE||Lewis.Bowen@kH5SrNtp0J2y.org|2452418| +4937|AAAAAAAAJEDBAAAA|1579018|5319|12020|2450271|2450241|Dr.|Olga|Baker|N|25|10|1992|ICELAND||Olga.Baker@zQBUNauxzJ4.com|2452438| +4938|AAAAAAAAKEDBAAAA|760995|6|37017|2450605|2450575|Dr.|William|Lang|Y|6|5|1956|FAROE ISLANDS||William.Lang@IqNEJCDm.org|2452644| +4939|AAAAAAAALEDBAAAA|454984|5289|28794|2449525|2449495|Dr.|Travis|Carter|N|15|11|1932|SLOVAKIA||Travis.Carter@fnMq2.edu|2452596| +4940|AAAAAAAAMEDBAAAA|173910|1156|31649|2450751|2450721|Dr.|Robert|Andrews|Y|4|10|1992|GAMBIA||Robert.Andrews@IFkAtG.org|2452393| +4941|AAAAAAAANEDBAAAA|424142|763|20231|2450756|2450726|Dr.|Regina|Gage|Y|4|5|1975|AUSTRIA||Regina.Gage@AqyYh.org|2452601| +4942|AAAAAAAAOEDBAAAA|204814|1213|19212|2449751|2449721|Mrs.|Sherrie|Roberts|N|13|8|1989|NETHERLANDS||Sherrie.Roberts@syx53.com|2452303| +4943|AAAAAAAAPEDBAAAA|273412|5040|7021||||Eric||N|6|8||TOGO||Eric.Hubbard@6nQxV9.com|2452384| +4944|AAAAAAAAAFDBAAAA|472205|1986|47720|2452636|2452606|Dr.|Raymond|Settles|N|22|6|1979|MAYOTTE||Raymond.Settles@0A5f.edu|2452316| +4945|AAAAAAAABFDBAAAA|1598534|1545|43309|2450777|2450747|Mr.|David|Wilcox|Y|27|11|1926|GUATEMALA||David.Wilcox@endTIj.org|2452321| +4946|AAAAAAAACFDBAAAA|554698|3600|48455|2451383|2451353|Dr.|Madeline|Arteaga|N|14|7|1968|SYRIAN ARAB REPUBLIC||Madeline.Arteaga@el35F00zlJa64.org|2452643| +4947|AAAAAAAADFDBAAAA|1351497|1212|6480|2450250|2450220|Sir|Bennie|Smith|N|4|7|1954|AFGHANISTAN||Bennie.Smith@iP4hRhtmo0Hf.edu|2452437| +4948|AAAAAAAAEFDBAAAA|1870638|494|31997|2449927|2449897|Mrs.|Helen|Cartwright|N|3|9|1925|AZERBAIJAN||Helen.Cartwright@3xjU6uUD8h.com|2452502| +4949|AAAAAAAAFFDBAAAA|1076475|1989|28350|2451012|2450982|Miss|Peggy|Heller|Y|19|4|1964|VIET NAM||Peggy.Heller@aENo84mI.com|2452648| +4950|AAAAAAAAGFDBAAAA|936300|5258|569|2450746|2450716|Dr.|Mildred|Latimer|N|30|8|1949|RWANDA||Mildred.Latimer@y1SJFoL2.org|2452377| +4951|AAAAAAAAHFDBAAAA|1610195|1221|14200|2450120|2450090|Dr.|Sherika|Henderson|N|7|12|1937|ISRAEL||Sherika.Henderson@fM0Nu.org|2452503| +4952|AAAAAAAAIFDBAAAA|664681|6654|44980|2451738|2451708|Dr.|Bryan|Jung|Y|2|1|1950|LESOTHO||Bryan.Jung@X8ECvIEeuVLbnHFkj.edu|2452405| +4953|AAAAAAAAJFDBAAAA|1549727|6329|25684|2451172|2451142|Mrs.|Lillie|Jones|N|19|6|1941|BAHAMAS||Lillie.Jones@d0hU9pv9fCOx.com|2452611| +4954|AAAAAAAAKFDBAAAA|1195248|143|6591|2450294|2450264|Mr.|Leonard|Underwood|N|7|3|1960|C�TE D'IVOIRE||Leonard.Underwood@OEkUS0YbE8d59NjlJ.edu|2452629| +4955|AAAAAAAALFDBAAAA|1037637|3451|30865|2450249|2450219|Dr.|Hugh|Ball|Y|27|5|1966|GREENLAND||Hugh.Ball@qI.com|2452414| +4956|AAAAAAAAMFDBAAAA|1814685|1360|17282|2451484|2451454|Mrs.|Ruth|Jones|Y|29|6|1968|GAMBIA||Ruth.Jones@JOdsvXUmQU.edu|2452416| +4957|AAAAAAAANFDBAAAA|1069718|1377|38683|2451216|2451186|Sir|Matthew|Childers|N|19|10|1966|BOLIVIA||Matthew.Childers@c.com|2452441| +4958|AAAAAAAAOFDBAAAA|1138395|3937|41079|2452151|2452121|Sir|Terrence|Ruiz|N|7|2|1945|NIUE||Terrence.Ruiz@UXbhVpmnbF7z3r.org|2452287| +4959|AAAAAAAAPFDBAAAA|515483|5676|47069|2449558|2449528|Sir|Gary|Green|Y|20|1|1957|LESOTHO||Gary.Green@6A0LaYCUE1F2.com|2452310| +4960|AAAAAAAAAGDBAAAA|1247850|3144|45238|2450038|2450008|Sir|Wayne|Johnson|Y|17|1|1969|MOZAMBIQUE||Wayne.Johnson@jLA.com|2452425| +4961|AAAAAAAABGDBAAAA|||43372||2451227||Nova|Tice|Y|8||1984|||Nova.Tice@gn0JOaDX10x.com|| +4962|AAAAAAAACGDBAAAA|144843|3261|8079|2449396|2449366|Dr.|William|Smith|Y|3|4|1979|GUINEA||William.Smith@iRbNB8GAxC.com|2452565| +4963|AAAAAAAADGDBAAAA|430468|3769|40670|2452562|2452532|Mrs.|Dorothy|Garza|N|22|8|1988|LESOTHO||Dorothy.Garza@6Tn1dJ431OQYNTb.org|2452530| +4964|AAAAAAAAEGDBAAAA|1169265|3984|5056|2451921|2451891|Dr.|Harry|Randle|Y|21|5|1936|LIECHTENSTEIN||Harry.Randle@0E.com|2452465| +4965|AAAAAAAAFGDBAAAA|1170278|3353|45981|2450654|2450624|Mrs.|Karen|Rodriguez|N|9|2|1931|GUERNSEY||Karen.Rodriguez@MPMemZaAYoHompEe.org|2452507| +4966|AAAAAAAAGGDBAAAA|1328139|3869|25152|2449408|2449378|Dr.|Seth|Milner|Y|24|7|1974|PUERTO RICO||Seth.Milner@UtyJ1s7.com|2452630| +4967|AAAAAAAAHGDBAAAA|48475|7056|4437|2449410|2449380|Dr.|Willie|Robinson|Y|10|1|1981|VIET NAM||Willie.Robinson@sdyh0Q6ZK3eLfo6R7.edu|2452315| +4968|AAAAAAAAIGDBAAAA|544102|497|35339|2449799|2449769|Dr.|Norman|Fernandez|Y|18|12|1945|BELIZE||Norman.Fernandez@F68aJ4vTZ.edu|2452538| +4969|AAAAAAAAJGDBAAAA|1142339|1552|27918|2449819|2449789|Dr.|Mark|Coleman|N|2|10|1971|UGANDA||Mark.Coleman@TU4o2te6.org|2452642| +4970|AAAAAAAAKGDBAAAA|867304|2198|9539|2450656|2450626|Dr.|Timothy|Francis|Y|19|5|1985|NIGERIA||Timothy.Francis@vZLMhmJ4eUG8eVbcn.edu|2452359| +4971|AAAAAAAALGDBAAAA|704775|2214|12806|2449388|2449358|Mr.|James|Guzman|N|17|3|1977|ARUBA||James.Guzman@ykIKJN3iRaTvGA.org|2452556| +4972|AAAAAAAAMGDBAAAA|1172128|6016|41504|2451146|2451116|Mrs.|Martha|Conway|N|14|9|1967|TIMOR-LESTE||Martha.Conway@VzZJpLQ.com|2452384| +4973|AAAAAAAANGDBAAAA|588366|3098|43077|2451449|2451419|Mrs.|Louann|Kimbrough|N|20|9|1973|GIBRALTAR||Louann.Kimbrough@7yT2AsvLYCd8t.edu|2452516| +4974|AAAAAAAAOGDBAAAA|703420|1428|33436|2452019|2451989|Miss|Margaret|Martens|N|24|5|1926|SAINT HELENA||Margaret.Martens@abBV.org|2452418| +4975|AAAAAAAAPGDBAAAA|1322363|7193|5576|2451647|2451617|Ms.|Andrew|Villanueva|Y|11|4|1976|TUVALU||Andrew.Villanueva@KYB9bTLlmEOT.edu|2452565| +4976|AAAAAAAAAHDBAAAA|914521|3172|13776|2449207|2449177|Mrs.|Marvis|Isaac|N|31|1|1959|AUSTRIA||Marvis.Isaac@hzc5ZqpqdmNcUhbvACI.edu|2452376| +4977|AAAAAAAABHDBAAAA|129517|2165|39566|2451205|2451175|Dr.|Mitch|Gonzalez|N|17|3|1984|CHRISTMAS ISLAND||Mitch.Gonzalez@hBjQRJKII7.com|2452397| +4978|AAAAAAAACHDBAAAA|458801|1188|49622|2452545|2452515|Mr.|Scott|Weaver|Y|23|3|1928|SWITZERLAND||Scott.Weaver@BYG9P.edu|2452431| +4979|AAAAAAAADHDBAAAA|231842|5978|48067|2450698|2450668|Miss|Rebecca|Dominguez|Y|11|1|1984|INDIA||Rebecca.Dominguez@QKjlcORnhVk.com|2452551| +4980|AAAAAAAAEHDBAAAA||3585|33028||||Pauline|Waller|N||11||||Pauline.Waller@AaYgGcz.com|| +4981|AAAAAAAAFHDBAAAA|588075|775|45075|2452190|2452160|Ms.|Christine|Webster|N|14|7|1988|GREECE||Christine.Webster@pQ3DmDuVcl0ZOEl.edu|2452362| +4982|AAAAAAAAGHDBAAAA||2586|15065|2451903|||Jann|||30||1976||||2452405| +4983|AAAAAAAAHHDBAAAA|1579080|4241|24202||2451953||Ricky||Y|||||||| +4984|AAAAAAAAIHDBAAAA|||37217|2449167|||||N|||1966|RUSSIAN FEDERATION||Melvin.Reed@B8Lyimuv.org|| +4985|AAAAAAAAJHDBAAAA|1604592|7120|31648|2449747|2449717|Mr.|Ian|Wilson|N|8|6|1975|MALDIVES||Ian.Wilson@Z.com|2452487| +4986|AAAAAAAAKHDBAAAA|668963|6300|11637|2452463|2452433|Mr.|Jose|Duckworth|Y|16|5|1936|MALDIVES||Jose.Duckworth@ZotIZqPoC8.com|2452475| +4987|AAAAAAAALHDBAAAA|1381733|5762|991|2449681|2449651|Sir|William|Buckner|Y|18|9|1957|FAROE ISLANDS||William.Buckner@a.edu|2452412| +4988|AAAAAAAAMHDBAAAA|730764|699|8234|2449985|2449955|Ms.|Anna|Mcgrath|Y|6|7|1976|UKRAINE||Anna.Mcgrath@cjej.com|2452424| +4989|AAAAAAAANHDBAAAA|1647292|6090|40088|2449261|2449231|Sir|Stephen|England|Y|18|5|1978|UNITED STATES||Stephen.England@o7pmuXiKybRSO.edu|2452617| +4990|AAAAAAAAOHDBAAAA|820493|4027|29561|2452376|2452346|Mr.|Ronald|David|Y|15|1|1986|COMOROS||Ronald.David@q4rS7Dpo8ag.org|2452581| +4991|AAAAAAAAPHDBAAAA|1029651|5689|34974|2452567|2452537|Miss|Melinda|Crane|Y|25|3|1937|SWITZERLAND||Melinda.Crane@qJrL8LtLC6RC.com|2452339| +4992|AAAAAAAAAIDBAAAA|631090|3123|26756|2452367|2452337|Sir|Robert|Riggs|Y|25|3|1983|NAURU||Robert.Riggs@ii.org|2452534| +4993|AAAAAAAABIDBAAAA|1576723|3594|46818|2452341|2452311|Mr.|Bobby|West|N|23|2|1954|BULGARIA||Bobby.West@EomD15Sk9Iln.edu|2452468| +4994|AAAAAAAACIDBAAAA|1307861|6823|7032|2451634|2451604|Mrs.|Arthur|Combs|Y|3|9|1981|LUXEMBOURG||Arthur.Combs@JrLrklIbBRhfK.org|2452570| +4995|AAAAAAAADIDBAAAA|633703|6202|19854|2450890|2450860|Dr.|William|Mattingly|N|29|1|1928|SOLOMON ISLANDS||William.Mattingly@GYM7Kq0spd98Dyn.org|2452601| +4996|AAAAAAAAEIDBAAAA|442660|1530|47600|2452400|2452370|Miss|Asia|Collins|Y|5|7|1928|LITHUANIA||Asia.Collins@JpFE5YAYSzgq.edu|2452357| +4997|AAAAAAAAFIDBAAAA|841937|5221|45705|2452101|2452071|Sir|James|Mendez|Y|3|1|1935|NIUE||James.Mendez@IopXJ.org|2452582| +4998|AAAAAAAAGIDBAAAA|264268|263|44658|2451830|2451800|Mr.|Donald|Cox|N|11|7|1944|MARSHALL ISLANDS||Donald.Cox@DiBi.edu|2452599| +4999|AAAAAAAAHIDBAAAA|1573144|1096|46601|2449860|2449830|Sir|Enrique|Thomas|Y|31|5|1934|BELARUS||Enrique.Thomas@gPAkDrnCD.com|2452610| +5000|AAAAAAAAIIDBAAAA|1533751|3547|18990|2451772|2451742|Mr.|Kyle|Rosenberg|Y|1|12|1946|UZBEKISTAN||Kyle.Rosenberg@q3esRXSLJbdF.org|2452433| +5001|AAAAAAAAJIDBAAAA|1723407|1708|9656|2451498|2451468|Mr.|Chris|Tibbs|Y|10|3|1932|KAZAKHSTAN||Chris.Tibbs@G3.org|2452349| +5002|AAAAAAAAKIDBAAAA|1684104|3084|46417|2452631|2452601|Miss|Anna|Lutz|Y|27|9|1956|SOLOMON ISLANDS||Anna.Lutz@FBNcHYYhfMBT.org|2452600| +5003|AAAAAAAALIDBAAAA|1226644|4688|15485|2451350|2451320|Mr.|Justin|Spencer|N|19|7|1936|LESOTHO||Justin.Spencer@echZ2FDVnjJa.edu|2452514| +5004|AAAAAAAAMIDBAAAA|1023193|6434|13722|2451749|2451719|Sir|Pablo|Munoz|N|6|6|1927|CAMBODIA||Pablo.Munoz@1SExA49oFrMn15V.org|2452500| +5005|AAAAAAAANIDBAAAA|53276|2640|23157|2450599|2450569|Mr.|Mark|Woods|Y|1|6|1962|IRAQ||Mark.Woods@gbZYrke.com|2452299| +5006|AAAAAAAAOIDBAAAA|775488|1471|23854|2450978|2450948|Mrs.|Kathryn|Torrez|Y|1|7|1940|JERSEY||Kathryn.Torrez@sjSkTr0xGXD.edu|2452501| +5007|AAAAAAAAPIDBAAAA|499283|3962|34594|2450113|2450083|Dr.|Terry|Bergman|Y|29|6|1963|MONGOLIA||Terry.Bergman@7fO.edu|2452344| +5008|AAAAAAAAAJDBAAAA|134154|5224|17847|2450478|2450448|Mrs.|Pamela|Connolly|N|30|4|1985|IRELAND||Pamela.Connolly@XQ2NhF9Ec0.com|2452452| +5009|AAAAAAAABJDBAAAA|1246422|5931|29040|2449120|2449090|Mrs.|Elizabeth|Fincher|Y|19|1|1979|SAUDI ARABIA||Elizabeth.Fincher@t7CPbUdIRaTD6y.org|2452322| +5010|AAAAAAAACJDBAAAA|1033780|5417|38607|2451922|2451892|Dr.|Lillian|Thomas|N|29|3|1948|PARAGUAY||Lillian.Thomas@FY5NBQbLP2uSZ6BE.com|2452351| +5011|AAAAAAAADJDBAAAA|1397298|6857|1216|2451792|2451762|Mr.|William|Watson|Y|1|5|1952|NETHERLANDS ANTILLES||William.Watson@PYS.com|2452339| +5012|AAAAAAAAEJDBAAAA|1446256|6089|38275|2451345|2451315|Dr.|Ruby|Thomas|N|16|8|1986|FIJI||Ruby.Thomas@1kfF3TQ.org|2452340| +5013|AAAAAAAAFJDBAAAA|1102004|1033|22862|2452212|2452182|Mrs.|Brandi|Batson|Y|4|5|1932|MAURITIUS||Brandi.Batson@CJuN68QkLfVb.org|2452568| +5014|AAAAAAAAGJDBAAAA|494533|4890|5600|2449982|2449952|Sir|John|Graff|Y|28|2|1941|BELIZE||John.Graff@CtKzDYJqhu6.org|2452313| +5015|AAAAAAAAHJDBAAAA|1902126|1523|16120|2452464|2452434|Dr.|John|Wallace|Y|22|11|1973|ECUADOR||John.Wallace@BPgXFD7Hs7fRgCjd.com|2452513| +5016|AAAAAAAAIJDBAAAA|1425450|1269|39088|2451388|2451358|Dr.|Maxine|Calloway|N|5|4|1928|BELGIUM||Maxine.Calloway@K4v1AqoTaXGNhrUENI.com|2452500| +5017|AAAAAAAAJJDBAAAA|195977|3922|19867|2452448|2452418|Miss|Tamara|Hadden|Y|14|2|1961|ZAMBIA||Tamara.Hadden@HhOfCtB3Uz3mZC0If.com|2452522| +5018|AAAAAAAAKJDBAAAA|517021|3964|36860|2450340|2450310|Mr.|David|Phillips|Y|25|3|1972|LEBANON||David.Phillips@msV.edu|2452520| +5019|AAAAAAAALJDBAAAA|129059|6564|8869|2450517|2450487|Ms.|Monica|Edwards|N|4|6|1965|FIJI||Monica.Edwards@EBpbf1DAVf9a.edu|2452480| +5020|AAAAAAAAMJDBAAAA|||16387|2452463|2452433||Pa|Ledesma|||||VENEZUELA|||| +5021|AAAAAAAANJDBAAAA|367354|6434|3867|2449997|2449967|Dr.|James|Valenzuela|N|21|10|1938|GUINEA||James.Valenzuela@ebmRzCOQbOlXOY.org|2452454| +5022|AAAAAAAAOJDBAAAA|1348199|1812|25378|2451345|2451315|Sir|Robert|Ramos|Y|28|4|1992|R�UNION||Robert.Ramos@0Vpv2EeSO0shnACB.org|2452524| +5023|AAAAAAAAPJDBAAAA|1503010|583|18156|2450881|2450851|Miss|Betty|Calderon|N|8|6|1941|NEPAL||Betty.Calderon@bgAacj7lDj7al.com|2452632| +5024|AAAAAAAAAKDBAAAA|1217145|6453|43361|2450397|2450367|Sir|Jerry|Smith|N|12|7|1966|ROMANIA||Jerry.Smith@shtUqqdvUV.org|2452391| +5025|AAAAAAAABKDBAAAA|95356|5465|6774|2452419|2452389|Miss|Katrina|Creel|Y|23|6|1949|GAMBIA||Katrina.Creel@1z60410Odv4s.edu|2452638| +5026|AAAAAAAACKDBAAAA|768453|4385|29917|2450711|2450681|Miss|Kathy|Kelly|N|15|9|1988|ANTIGUA AND BARBUDA||Kathy.Kelly@o.org|2452339| +5027|AAAAAAAADKDBAAAA|334269|6477|44285|2452397|2452367|Dr.|Edith|Ward|Y|5|6|1977|MONGOLIA||Edith.Ward@V6hejZG.edu|2452420| +5028|AAAAAAAAEKDBAAAA|1702929|4921|4866|2452474|2452444|Dr.|Marilyn|Vann|Y|24|3|1961|SOMALIA||Marilyn.Vann@t7.org|2452600| +5029|AAAAAAAAFKDBAAAA|1662075|3166|13410|2449638|2449608|Mr.|Walter|Simons|N|30|12|1962|NICARAGUA||Walter.Simons@iAlrI52HIJlnRH.edu|2452576| +5030|AAAAAAAAGKDBAAAA|684409|3530|20604|2450210|2450180|Dr.|Jim|Garcia|N|25|6|1926|VIRGIN ISLANDS, U.S.||Jim.Garcia@H5elpMC70DADsh.com|2452357| +5031|AAAAAAAAHKDBAAAA|1679711|7097|1730|2450309|2450279|Miss|Tania|Worley|Y|6|4|1954|MYANMAR||Tania.Worley@sRtkzkrzSE.com|2452336| +5032|AAAAAAAAIKDBAAAA|1190126|2543|36560|2450811|2450781|Ms.|Donna|Pereira|N|15|4|1967|MOZAMBIQUE||Donna.Pereira@fzb8pKCRr.com|2452565| +5033|AAAAAAAAJKDBAAAA|1445805|3540|48428|2449853|2449823|Mr.|Ernest|Gonzales|Y|31|7|1984|MADAGASCAR||Ernest.Gonzales@sO7SSzeZvrIUii.com|2452320| +5034|AAAAAAAAKKDBAAAA|1133546|2621|10496|2452063|2452033|Dr.|Howard|Boone|Y|24|11|1927|GUINEA-BISSAU||Howard.Boone@4.org|2452586| +5035|AAAAAAAALKDBAAAA|1464282|6696|48310|2451223|2451193|Dr.|Regina|Weller|N|28|7|1933|VANUATU||Regina.Weller@gDlbG7EeotNBoM.edu|2452310| +5036|AAAAAAAAMKDBAAAA|1366773|3875|34977|2449084|2449054|Mrs.|Kitty|Hammett|N|7|4|1983|TURKMENISTAN||Kitty.Hammett@nyV6DjenCF.org|2452439| +5037|AAAAAAAANKDBAAAA|||25147||2449979|Mr.|Jack||||3|1975|||Jack.Moss@SmTnG8.com|| +5038|AAAAAAAAOKDBAAAA|1374433|774|49115|2449217|2449187|Sir|Reginald|Gragg|Y|7|9|1926|CAYMAN ISLANDS||Reginald.Gragg@5sf69lRy6eG.org|2452440| +5039|AAAAAAAAPKDBAAAA|1857297|5828|33159|2450785|2450755|Mr.|Robert|George|N|7|6|1929|SUDAN||Robert.George@RMBG.com|2452294| +5040|AAAAAAAAALDBAAAA|154916|6015|3114|2449755|2449725|Mrs.|Barbara|Robertson|N|17|7|1981|BERMUDA||Barbara.Robertson@aLk.com|2452297| +5041|AAAAAAAABLDBAAAA|1590081|1332|7383|2450108|2450078|Miss|Rosa|Giles|Y|1|3|1975|NAMIBIA||Rosa.Giles@YPohX.com|2452429| +5042|AAAAAAAACLDBAAAA|30278|3628|26978|2450522|2450492|Sir|Glenn|Russ|Y|20|9|1945|COSTA RICA||Glenn.Russ@mOQivf9J.org|2452302| +5043|AAAAAAAADLDBAAAA|916589|5826|27910|2451420|2451390|Miss|Aaron|Walker|N|22|4|1939|NICARAGUA||Aaron.Walker@u3orMk8eki1.org|2452533| +5044|AAAAAAAAELDBAAAA|1047614|3713|29012|2449453|2449423|Dr.|Stephen|Benjamin|Y|19|6|1952|SOUTH AFRICA||Stephen.Benjamin@bEKTCIZ6V8xR3mXNP.com|2452428| +5045|AAAAAAAAFLDBAAAA|324846|5151|10256|2450012|2449982|Miss|Holly|Stephens|N|30|1|1951|PAPUA NEW GUINEA||Holly.Stephens@7.edu|2452607| +5046|AAAAAAAAGLDBAAAA|439700|5701|2149|2451352|2451322|Sir|Gerald|Jenson|N|3|2|1976|CAMEROON||Gerald.Jenson@y3yJ0I3KV.org|2452533| +5047|AAAAAAAAHLDBAAAA|915547|6834|49674|2449469|2449439|Sir|Joshua|Green|N|14|12|1988|LIBERIA||Joshua.Green@CBr7UTqlJ0E9HXP.edu|2452636| +5048|AAAAAAAAILDBAAAA|1917352|440|21881|2450551|2450521|Sir|Louis|Muse|N|4|3|1971|JERSEY||Louis.Muse@FksL7CVp3oEGOQxpVMZN.com|2452518| +5049|AAAAAAAAJLDBAAAA|289501|6756|15564|2450045|2450015|Sir|Ronald|Austin|N|11|2|1949|TIMOR-LESTE||Ronald.Austin@Tq1Puys.org|2452525| +5050|AAAAAAAAKLDBAAAA|1419135|4692|25449|2452529|2452499|Mr.|Mark|Freeman|Y|16|11|1974|EQUATORIAL GUINEA||Mark.Freeman@fnGKc.com|2452394| +5051|AAAAAAAALLDBAAAA|1654560|3762|10564|2450649|2450619|Mrs.|Kristine|Stewart|Y|14|1|1989|SRI LANKA||Kristine.Stewart@vcD9bL7lsAQNk.com|2452318| +5052|AAAAAAAAMLDBAAAA|33546|1597|15648|2449214|2449184|Mrs.|Deborah|Garcia|Y|20|7|1979|MAURITIUS||Deborah.Garcia@LYmxOXo4A.org|2452416| +5053|AAAAAAAANLDBAAAA|1136899|4092|44838|2451387|2451357|Ms.|Edwina|Parris|N|18|11|1926|HONDURAS||Edwina.Parris@M2G4iJlpO5Bqkz.edu|2452353| +5054|AAAAAAAAOLDBAAAA|1461145|3766|7987|2450336|2450306|Mr.|Nicholas|Navarro|Y|4|12|1960|TIMOR-LESTE||Nicholas.Navarro@qu8Ht3fmYxZ88U.org|2452524| +5055|AAAAAAAAPLDBAAAA|1833395|4859|13523|2451819|2451789|Dr.|Robert|Tucker|Y|27|9|1979|UZBEKISTAN||Robert.Tucker@EJmK.edu|2452291| +5056|AAAAAAAAAMDBAAAA|1782867||7885|2451140||||||||1937|UNITED KINGDOM||Margaret.Sutton@XK0lgVQbKeXrgyHM7.edu|| +5057|AAAAAAAABMDBAAAA|1778675|2968|34461|2449572|2449542|Mr.|Vincent|Williams|N|6|9|1982|AUSTRIA||Vincent.Williams@2OJE2q15nIP50.edu|2452523| +5058|AAAAAAAACMDBAAAA|960883|6215|9147|2450076|2450046|Miss|Nora|Garcia|Y|14|4|1946|MADAGASCAR||Nora.Garcia@cCV8ZSNZIlII4z.org|2452433| +5059|AAAAAAAADMDBAAAA|1885400|6705|15210|2451756|2451726|Dr.|Anna|Velazquez|Y|13|7|1925|BARBADOS||Anna.Velazquez@xg6vsaOxxKt.edu|2452304| +5060|AAAAAAAAEMDBAAAA|258927|6517|24060|2450071|2450041|Miss|Nancy|Crowe|Y|10|5|1928|SIERRA LEONE||Nancy.Crowe@asq.org|2452323| +5061|AAAAAAAAFMDBAAAA|1172103|1357|44964|2450881|2450851|Dr.|John|Christmas|N|26|8|1953|GIBRALTAR||John.Christmas@sLmRb.com|2452626| +5062|AAAAAAAAGMDBAAAA|360716|82|11448|2449605|2449575|Sir|Nick|Castle|N|28|5|1924|MAURITIUS||Nick.Castle@SlUfArKoR.org|2452283| +5063|AAAAAAAAHMDBAAAA|163865|1259|25053|2450940|2450910|Mr.|||Y|2|9|1941|||Joshua.Bennett@IGfF.com|2452484| +5064|AAAAAAAAIMDBAAAA|1313112|5184|7303|2449434|2449404|Dr.|Walter|Frazier|N|1|10|1970|MAURITANIA||Walter.Frazier@Rj6n8srYjYvmmfdClx.org|2452337| +5065|AAAAAAAAJMDBAAAA|1801419|485|45151|2451819|2451789|Mr.|William|Omalley|Y|28|7|1970|HAITI||William.Omalley@jm9PLedBoaKtN.org|2452607| +5066|AAAAAAAAKMDBAAAA|691674|506|31338|2450853|2450823|Sir|Howard|Bryant|Y|24|3|1965|NIGER||Howard.Bryant@L2MPaHor3.edu|2452572| +5067|AAAAAAAALMDBAAAA|1212790|4189|7941|2451015|2450985|Ms.|Debora|Harris|N|19|9|1931|BRAZIL||Debora.Harris@vrvZ4bGd.edu|2452440| +5068|AAAAAAAAMMDBAAAA|296545|2589|37480|2449825|2449795|Sir|Darrel|Ruffin|N|1|4|1947|VENEZUELA||Darrel.Ruffin@6J0O66RxTl.org|2452286| +5069|AAAAAAAANMDBAAAA|456695|5739|38336|2450527|2450497|Mrs.|Sylvia|Ortiz|Y|20|5|1985|ZIMBABWE||Sylvia.Ortiz@MFOAjx2JNdCVp.com|2452504| +5070|AAAAAAAAOMDBAAAA|1895561|579|25197|2449891|2449861|Ms.|Sue|Martin|Y|23|10|1980|KIRIBATI||Sue.Martin@YP5UsojLVcVul8J.com|2452417| +5071|AAAAAAAAPMDBAAAA|1808486|1117|41395|2449146|2449116|Mr.|Dennis|Cobb|N|20|2|1933|PUERTO RICO||Dennis.Cobb@zJpVdq1I9.edu|2452293| +5072|AAAAAAAAANDBAAAA|155154|6011|68|2451091|2451061|Dr.|Gary|Rivera|N|14|9|1963|BOUVET ISLAND||Gary.Rivera@aY1X.com|2452548| +5073|AAAAAAAABNDBAAAA|660001|6891|15334|2451018|2450988|Mr.|Christopher|Potter|Y|3|4|1941|ETHIOPIA||Christopher.Potter@fmpL0VyKN9.com|2452430| +5074|AAAAAAAACNDBAAAA|114549|3093|10884|2449571|2449541|Mr.|Jimmy|Wade|Y|23|4|1992|UNITED ARAB EMIRATES||Jimmy.Wade@5FvAkj5Ti.edu|2452454| +5075|AAAAAAAADNDBAAAA|1460480|3349|16618||2450626|Ms.|Florence||||3|1927|MOLDOVA, REPUBLIC OF||Florence.Walker@dsSHDM8AyN.org|| +5076|AAAAAAAAENDBAAAA|||28169|2451059|2451029|Miss||Pittman|N|3|7|1945|NAURU|||| +5077|AAAAAAAAFNDBAAAA|1232064|5952|37040|2450380|2450350|Dr.|Michael|Maldonado|Y|17|11|1958|PHILIPPINES||Michael.Maldonado@L4.edu|2452352| +5078|AAAAAAAAGNDBAAAA|607061|2029|2997|2450826|2450796|Dr.|Carlos|Lankford|Y|14|12|1952|ZIMBABWE||Carlos.Lankford@1QkKZSEUXz8q3T.org|2452458| +5079|AAAAAAAAHNDBAAAA|860138|7199|32613|2449662|2449632|Miss|Christina|Lunsford|N|4|5|1949|SWITZERLAND||Christina.Lunsford@vbGUU47pCOAOD1hAn.org|2452381| +5080|AAAAAAAAINDBAAAA|89421|4949|20124|2450683|2450653|Mr.|David|Baker|N|13|11|1930|MONGOLIA||David.Baker@CioZ5J1a8H.edu|2452642| +5081|AAAAAAAAJNDBAAAA|1295670|4987|14751|2449234|2449204|Mr.|Kenneth|Deaton|N|2|4|1964|VIET NAM||Kenneth.Deaton@It30qziSKckT.com|2452303| +5082|AAAAAAAAKNDBAAAA|346043|5576|20662|2451502|2451472|Mrs.|Yolanda|Platt|Y|20|3|1971|EGYPT||Yolanda.Platt@VyHN1.edu|2452409| +5083|AAAAAAAALNDBAAAA|1691968|6351|29534|2450707|2450677|Dr.|Jodi|Fernandez|N|29|1|1967|LUXEMBOURG||Jodi.Fernandez@P6RUNp.edu|2452395| +5084|AAAAAAAAMNDBAAAA|1296954|2273|13639|2451425|2451395|Mrs.|Adrian|Nelson|N|4|1|1985|GABON||Adrian.Nelson@U42bu3MVbp.org|2452349| +5085|AAAAAAAANNDBAAAA|391940|6644|13395|2449352|2449322|Mrs.|Teri|Mcmahon|Y|25|12|1990|TUVALU||Teri.Mcmahon@sO4FOsfo.com|2452554| +5086|AAAAAAAAONDBAAAA|584755|6873|43038|2451641|2451611|Dr.|Dorothy|Nelson|N|15|1|1932|POLAND||Dorothy.Nelson@1VDjR5zi7.org|2452534| +5087|AAAAAAAAPNDBAAAA|1174093|1066|31592|2449674|2449644|Dr.|Thomas|Grace|N|20|11|1969|EL SALVADOR||Thomas.Grace@rM.edu|2452510| +5088|AAAAAAAAAODBAAAA|1081345|565|39785|2450672|2450642|Mr.|Willie|Romero|Y|4|4|1970|BOTSWANA||Willie.Romero@gUeSg0D8vacsZ.org|2452562| +5089|AAAAAAAABODBAAAA|||23635||||Anthony|||11|5||ISLE OF MAN||Anthony.Bynum@JuaQAbhimhaLDhFHT.edu|2452611| +5090|AAAAAAAACODBAAAA|64437|6537|9790|2449521|2449491|Mrs.|Regina|Pittman|N|12|8|1949|MEXICO||Regina.Pittman@DGNhDuR2LOS.edu|2452615| +5091|AAAAAAAADODBAAAA|943770|5472|26774|2449742|2449712|Mrs.|Elizabeth|Spalding|N|27|11|1947|KAZAKHSTAN||Elizabeth.Spalding@ihnjlmxMbs.edu|2452318| +5092|AAAAAAAAEODBAAAA|32760|1341|9922|2450035|2450005|Ms.|Rita|Miller|Y|12|8|1943|MONACO||Rita.Miller@0SX04Hk5ety55fkQ.edu|2452383| +5093|AAAAAAAAFODBAAAA|530920|6950|38980|2449137|2449107|Sir|Ruben|Chase|Y|9|5|1930|ALGERIA||Ruben.Chase@s6QXOhkAAIN.edu|2452483| +5094|AAAAAAAAGODBAAAA|1335854|2155|19303|2451672|2451642|Dr.|Paul|Neuman|Y|14|4|1987|LESOTHO||Paul.Neuman@u2MKk.org|2452610| +5095|AAAAAAAAHODBAAAA|358573|6307|46441|2450529|2450499|Mrs.|Jennie|Rios|N|18|9|1978|NEW CALEDONIA||Jennie.Rios@KTusbGXq.com|2452289| +5096|AAAAAAAAIODBAAAA|996202|6473|37144|2449921|2449891|Mrs.|Elizabeth|Messer|Y|1|9|1970|ISLE OF MAN||Elizabeth.Messer@o3NovQbtiuNB3.edu|2452304| +5097|AAAAAAAAJODBAAAA|272491|266|38088||2451581||Steven|Curtis||2|7|1930|||Steven.Curtis@dBgt9MxO13fi.org|2452437| +5098|AAAAAAAAKODBAAAA|1100559|2647|31935|2451509|2451479|Mr.|Miguel|Roy|N|11|6|1964|PARAGUAY||Miguel.Roy@ea1G7PkBRUh.com|2452557| +5099|AAAAAAAALODBAAAA|1689224|2016|30855|2451131|2451101|Dr.|Carrie|Greene|N|15|9|1960|AUSTRALIA||Carrie.Greene@MkHKPSSJGKPj.com|2452512| +5100|AAAAAAAAMODBAAAA|452902|5227|37552|2450501|2450471|Ms.|Jennifer|Howe|Y|15|12|1964|NIGERIA||Jennifer.Howe@Xk9M4DEPTVO.com|2452393| +5101|AAAAAAAANODBAAAA|857148|2297|39846|2450349|2450319|Ms.|Jane|Albertson|Y|12|3|1940|MAYOTTE||Jane.Albertson@RpfiFALbH9UlrTq2bb.com|2452591| +5102|AAAAAAAAOODBAAAA|1532111|4046|46428|2449210|2449180|Dr.|Frederick|Shields|Y|13|12|1933|ICELAND||Frederick.Shields@8c1VYEZ3YinAD.edu|2452297| +5103|AAAAAAAAPODBAAAA|1655288|3525|28915|2449475|2449445|Ms.|Marie|Glenn|N|9|8|1957|MOLDOVA, REPUBLIC OF||Marie.Glenn@Z3.org|2452620| +5104|AAAAAAAAAPDBAAAA|674895|4293|32444|2451701|2451671|Miss|Paula|Gilmore|Y|28|8|1975|MONTENEGRO||Paula.Gilmore@x1oAHl36lJPyIM.edu|2452292| +5105|AAAAAAAABPDBAAAA|1167132|2069|47853|2452288|2452258|Mr.|Henry|Walker|Y|4|5|1966|ALBANIA||Henry.Walker@4F.com|2452286| +5106|AAAAAAAACPDBAAAA|132504|2101|29298|2450434|2450404|Mr.|Larry|Lewis|Y|14|7|1968|EGYPT||Larry.Lewis@8iotg.com|2452556| +5107|AAAAAAAADPDBAAAA|1259074|4965|41256|2449157|2449127|Sir|Robert|Roman|N|1|3|1970|R�UNION||Robert.Roman@i.org|2452637| +5108|AAAAAAAAEPDBAAAA|534313|3910|45060|2451306|2451276|Sir|Hugh|Powell|N|2|8|1970|AMERICAN SAMOA||Hugh.Powell@hQgnIlbkdb5V.org|2452506| +5109|AAAAAAAAFPDBAAAA|155374|1162|26950|2451621|2451591|Mr.|David|Johnson|N|4|7|1940|BURKINA FASO||David.Johnson@lSLrlCjCO8HlSB8Jc1Gp.edu|2452586| +5110|AAAAAAAAGPDBAAAA|914022|109|34996|2452050|2452020|Mr.|Christopher|Ross|Y|24|10|1949|TUNISIA||Christopher.Ross@5fZB1d0ms.edu|2452589| +5111|AAAAAAAAHPDBAAAA||3510|31231|2449527|||||||7|1952|EGYPT|||2452589| +5112|AAAAAAAAIPDBAAAA|1464527|246|19918|2451732|2451702|Mr.|William|Maupin|N|14|3|1967|PUERTO RICO||William.Maupin@OzlUkyk6Qv.edu|2452316| +5113|AAAAAAAAJPDBAAAA|1033671|226|5981|2452033|2452003|Mr.|Paul|Smith|Y|14|11|1953|SERBIA||Paul.Smith@h0vyCu9DEhq2ec3.com|2452446| +5114|AAAAAAAAKPDBAAAA|806306|2734|36009|2452646|2452616|Mr.|Harry|Sage|N|8|1|1984|IRELAND||Harry.Sage@tbJ5GtAmZR.org|2452483| +5115|AAAAAAAALPDBAAAA|852109|879|12849|2450456|2450426|Dr.|Santiago|Curry|N|18|3|1991|INDIA||Santiago.Curry@sXzCGD.com|2452528| +5116|AAAAAAAAMPDBAAAA|841376|1737|4851|2450564|2450534|Dr.|Charlie|Webb|N|24|5|1977|MAYOTTE||Charlie.Webb@NbkBG.org|2452401| +5117|AAAAAAAANPDBAAAA|667675|1017|10312|2450417|2450387|Ms.|Katherine|Hsu|N|17|4|1948|JAMAICA||Katherine.Hsu@nncEN.com|2452441| +5118|AAAAAAAAOPDBAAAA|698226|4974|2922|2451870|2451840|Dr.|Ivy|Melton|Y|22|7|1943|ETHIOPIA||Ivy.Melton@AKZ5R4DBRUL86.edu|2452494| +5119|AAAAAAAAPPDBAAAA|144924|3978|47785|2452175|2452145|Ms.|Marjorie|Koenig|Y|13|4|1968|SERBIA||Marjorie.Koenig@pPeSTroIRzDM.org|2452628| +5120|AAAAAAAAAAEBAAAA|171394|2135|21276|2450142|2450112|Dr.|Daniel|Holliday|N|21|7|1953|MALAWI||Daniel.Holliday@A4zKH.com|2452646| +5121|AAAAAAAABAEBAAAA|93358|1627|31049|2452335|2452305|Sir|Douglas|Hill|Y|12|6|1948|BOUVET ISLAND||Douglas.Hill@vuUnn2qP.org|2452547| +5122|AAAAAAAACAEBAAAA|||40896|2450767||Sir|||N|||1976|||James.Leon@DmIKbp8xDXRU.com|2452537| +5123|AAAAAAAADAEBAAAA|116740|5715|13937|2451632|2451602|Miss|Jennifer|Bailey|Y|27|4|1965|CANADA||Jennifer.Bailey@8xkNNdCYQbBA4cU.com|2452362| +5124|AAAAAAAAEAEBAAAA|202190|3941|34139|2450093|2450063|Mrs.|Ashley|Moore|N|15|9|1963|UNITED STATES||Ashley.Moore@nBLAXd.org|2452506| +5125|AAAAAAAAFAEBAAAA|92122|1276|22030|2450130|2450100|Ms.|Rita|Long|Y|26|7|1943|ANGOLA||Rita.Long@L2fg8m7QRxu.org|2452594| +5126|AAAAAAAAGAEBAAAA|242695|1566|11190|2451840|2451810|Dr.|Loan|Ross|Y|16|1|1928|PANAMA||Loan.Ross@o6YSZaC.org|2452587| +5127|AAAAAAAAHAEBAAAA|1728187|2608|21782|2450202|2450172|Dr.|Mildred|Jones|N|15|8|1935|MOZAMBIQUE||Mildred.Jones@U3UZac1x.edu|2452316| +5128|AAAAAAAAIAEBAAAA|1482867|5204|41141|2450328|2450298|Sir|Bernard|Matos|Y|11|8|1974|MARTINIQUE||Bernard.Matos@PY4M6E34T.edu|2452495| +5129|AAAAAAAAJAEBAAAA|1797069|4784|47299|2452000|2451970|Dr.|Luis|Branson|N|11|5|1935|CAYMAN ISLANDS||Luis.Branson@XhOoG8Sx1LBzR0HV.org|2452628| +5130|AAAAAAAAKAEBAAAA|459823|3827|38984|2452064|2452034|Miss|Joann|Stock|N|25|11|1953|SPAIN||Joann.Stock@MVbAFXOD70.org|2452387| +5131|AAAAAAAALAEBAAAA|40672|983|5694|2452514|2452484|Dr.|Norman|Gamble|Y|19|9|1945|GUATEMALA||Norman.Gamble@egnhqNR9.edu|2452485| +5132|AAAAAAAAMAEBAAAA|792645|4936|28474|2450542|2450512|Dr.|Rose|Couch|N|27|9|1975|IRELAND||Rose.Couch@5MoF84EifbZs.org|2452408| +5133|AAAAAAAANAEBAAAA|280755|1581|37254|2450493|2450463|Sir|Kelvin|Joseph|Y|23|7|1954|BAHAMAS||Kelvin.Joseph@j2Jgl.edu|2452488| +5134|AAAAAAAAOAEBAAAA|937808|6471|12734|2451723|2451693|Mr.|William|Cronin|Y|27|10|1955|ZIMBABWE||William.Cronin@q4XBn6i3Dyz7PlCm.edu|2452446| +5135|AAAAAAAAPAEBAAAA|1895528|1149|15570|2452593|2452563|Miss|Nicole|Smith|N|21|1|1971|NIGER||Nicole.Smith@EmJp.com|2452450| +5136|AAAAAAAAABEBAAAA|1764261|5988|35668|2452191|2452161|Dr.|Raymond|Robinson|Y|9|1|1929|ALBANIA||Raymond.Robinson@XsB.org|2452380| +5137|AAAAAAAABBEBAAAA|1880776|3853|31283|2451987|2451957|Miss|Annie|Villegas|N|7|1|1937|TURKMENISTAN||Annie.Villegas@U.org|2452442| +5138|AAAAAAAACBEBAAAA|1369473|4649|15825|2449104|2449074|Mr.|Chris|Ford|Y|28|6|1941|ANDORRA||Chris.Ford@9t8.com|2452476| +5139|AAAAAAAADBEBAAAA|386570|1751|9732|2451645|2451615|Dr.|Doris|Plummer|Y|28|6|1927|SOLOMON ISLANDS||Doris.Plummer@AjGpjq8.com|2452621| +5140|AAAAAAAAEBEBAAAA|465363|4534|37258|2449626|2449596|Dr.|Jonathan|Gordon|N|20|8|1938|MARTINIQUE||Jonathan.Gordon@s6hee4BToIc.org|2452311| +5141|AAAAAAAAFBEBAAAA|1746295|5521|37194|2452381|2452351|Sir|Marty|Hoskins|Y|1|5|1933|GRENADA||Marty.Hoskins@Bsjdo7nD9.edu|2452448| +5142|AAAAAAAAGBEBAAAA|287500|1758|40863|2450321|2450291|Mrs.|Arthur|Wilson|N|23|12|1970|ANGOLA||Arthur.Wilson@8zYMCx.org|2452373| +5143|AAAAAAAAHBEBAAAA|1740521|5573|10272|2451458|2451428|Sir|Alphonse|Williams|N|29|1|1959|BURKINA FASO||Alphonse.Williams@SM.com|2452315| +5144|AAAAAAAAIBEBAAAA|1112016|3995|7621|2450073|2450043|Ms.|Susan|Weeks|Y|29|4|1939|SWEDEN||Susan.Weeks@ndLJ3hkjBtJgPau.com|2452338| +5145|AAAAAAAAJBEBAAAA|||2274|2449073|2449043|Sir||Stroud|Y||12|1983|||Ahmed.Stroud@lxAXKDCkJ.edu|| +5146|AAAAAAAAKBEBAAAA|1209705|5456|45607|2451034|2451004|Dr.|Michael|Williams|Y|21|4|1925|MALAWI||Michael.Williams@omY3stfJ1K4T.com|2452340| +5147|AAAAAAAALBEBAAAA|||49100||2452116||Jeremiah|Hylton|N||2|1967|||Jeremiah.Hylton@rNuygH73SnvZn.org|2452496| +5148|AAAAAAAAMBEBAAAA|1216783|3312|25394|2451779|2451749|Mrs.|Judith|Larson|N|30|3|1979|SLOVENIA||Judith.Larson@3EPST77VnMUd.org|2452293| +5149|AAAAAAAANBEBAAAA|638566|6636|39506|2450806|2450776|Sir|William|Lee|Y|24|6|1931|BARBADOS||William.Lee@0epu8uCV34y7TjM32.com|2452386| +5150|AAAAAAAAOBEBAAAA|461554|1511|17115|2452096|2452066|Dr.|Diane|Baker|N|18|11|1948|PANAMA||Diane.Baker@GXN.org|2452370| +5151|AAAAAAAAPBEBAAAA|112066|3658|9297|2450758|||Andrea|Popp|Y|5||1946||||2452468| +5152|AAAAAAAAACEBAAAA|1661008|493|16778|2450733|2450703|Mr.|Seymour|Smith|Y|7|9|1985|RWANDA||Seymour.Smith@GjKFznZYcAuM.org|2452363| +5153|AAAAAAAABCEBAAAA|1161041|1486|35484|2452020|2451990|Ms.|Louise|Lerner|Y|16|4|1930|BURKINA FASO||Louise.Lerner@Ymsk.org|2452393| +5154|AAAAAAAACCEBAAAA|54738|4276|42967|2451354|2451324|Mr.|Phillip|Obrien|Y|23|3|1985|PORTUGAL||Phillip.Obrien@ouJheCj5Tb.com|2452390| +5155|AAAAAAAADCEBAAAA|1652937|2318|26099|2450322|2450292|Mr.|Neal|Burton|N|9|8|1985|ALBANIA||Neal.Burton@EjZBHGg71D.com|2452519| +5156|AAAAAAAAECEBAAAA|44501|769|17837|2452372|2452342|Sir|John|Hardison|N|20|5|1927|SRI LANKA||John.Hardison@YJTxDvJQjD.com|2452553| +5157|AAAAAAAAFCEBAAAA|1405710|5981|45452|2450828|2450798|Ms.|Ann|Prieto|N|1|8|1934|LIECHTENSTEIN||Ann.Prieto@TCYjI2pTUx.edu|2452616| +5158|AAAAAAAAGCEBAAAA|1104756|5027|31745|2452010|2451980|Sir|John|Powell|Y|16|7|1963|INDONESIA||John.Powell@8XKk2oXvhGOFz8dH.com|2452429| +5159|AAAAAAAAHCEBAAAA|1168715|6773|23231|2452207|2452177|Ms.|Francine|Burnett|N|27|8|1991|ARUBA||Francine.Burnett@8JUGFXDmj6NsiA.com|2452394| +5160|AAAAAAAAICEBAAAA|231835|2716|23197|2450990|2450960|Dr.|Roy|Batchelor|N|5|8|1989|SAMOA||Roy.Batchelor@zM0mq9pGT7lq9k.org|2452428| +5161|AAAAAAAAJCEBAAAA|1645123|3360|21597|2452273|2452243|Mr.|Mark|Russo|Y|7|3|1951|CAYMAN ISLANDS||Mark.Russo@n6lm1jU1v.com|2452617| +5162|AAAAAAAAKCEBAAAA|681775|3162|41302|2451771|2451741|Sir|Jerome|Boyd|N|10|11|1931|LUXEMBOURG||Jerome.Boyd@qrJ.com|2452421| +5163|AAAAAAAALCEBAAAA|1561928|3397|3713|2450034|2450004|Dr.|John|Coburn|Y|30|12|1940|BAHRAIN||John.Coburn@y5f.com|2452349| +5164|AAAAAAAAMCEBAAAA|19336|1279|39222|2449079|2449049|Mrs.|Heather|Sheehan|N|30|7|1934|MACAO||Heather.Sheehan@aYJZ6vxSl.com|2452509| +5165|AAAAAAAANCEBAAAA|428870|1180|45043|2452568|2452538|Dr.|Chester|Doyle|N|4|12|1991|MACAO||Chester.Doyle@zbMh5IM8jom.edu|2452289| +5166|AAAAAAAAOCEBAAAA|1671572|5043|514|2451067|2451037|Mrs.|Theresa|Fogle|N|24|1|1966|GABON||Theresa.Fogle@K6BHtlXPRKxAMbf.edu|2452559| +5167|AAAAAAAAPCEBAAAA|1742475|7176|15524|2452276|2452246|Mr.|Warren|Smith|N|31|7|1925|RWANDA||Warren.Smith@p.edu|2452445| +5168|AAAAAAAAADEBAAAA|303414|699|48337|2450541|2450511|Miss|Marie|Jackson|Y|3|4|1972|TURKMENISTAN||Marie.Jackson@MczFYljbM.com|2452593| +5169|AAAAAAAABDEBAAAA|1015048|3276|34704|2450876|2450846|Sir|Mark|Fraser|Y|10|3|1951|MALDIVES||Mark.Fraser@8HVtaBOFKfI.edu|2452513| +5170|AAAAAAAACDEBAAAA|770622|6723|6424|2449805|2449775|Mr.|Gerardo|Lewis|N|5|5|1949|BELGIUM||Gerardo.Lewis@DsACEcFhMRgRUctEM.org|2452505| +5171|AAAAAAAADDEBAAAA|383199|3612|21625|2450686|2450656|Miss|Molly|Flores|Y|31|3|1986|MALDIVES||Molly.Flores@laVu0H44I.com|2452503| +5172|AAAAAAAAEDEBAAAA|525044|3982|39320|2451589|2451559|Sir|Ronald|Escalante|Y|4|12|1947|CYPRUS||Ronald.Escalante@XaXfRTAhAbcr.org|2452617| +5173|AAAAAAAAFDEBAAAA|61006|1164|47769|2452557|2452527|Mr.|Jared|Patterson|N|26|3|1984|MEXICO||Jared.Patterson@agkMtbp20e.edu|2452492| +5174|AAAAAAAAGDEBAAAA|1001199|7026|23206|2452301|2452271|Dr.|Twila|Keith|N|17|3|1936|SIERRA LEONE||Twila.Keith@LEnHXcJby7.edu|2452619| +5175|AAAAAAAAHDEBAAAA|126396|435|19627|2450028|2449998|Dr.|Patrick|Bird|Y|1|1|1936|FRENCH POLYNESIA||Patrick.Bird@x0vAXZuZB1.edu|2452333| +5176|AAAAAAAAIDEBAAAA|1790521|5801|20634|2452002|2451972|Mr.|William|Christian|Y|12|8|1924|TUVALU||William.Christian@eyfenTBr4BEml0k.org|2452430| +5177|AAAAAAAAJDEBAAAA|1264570|6581|592|2451883|2451853|Mr.|Scott|Brown|Y|22|11|1972|ZAMBIA||Scott.Brown@RcF1U6a7JhZh.org|2452578| +5178|AAAAAAAAKDEBAAAA|1500428|6113|4198|2449461|2449431|Ms.|Rebecca|Pierce|Y|2|8|1964|DOMINICA||Rebecca.Pierce@I6XInJxbN9F2ccvMgy.org|2452362| +5179|AAAAAAAALDEBAAAA|411555|1910|35818|2452456|2452426|Mr.|Don|Calhoun|N|1|6|1966|NEW ZEALAND||Don.Calhoun@dv0i8ZEaeSg6m.com|2452313| +5180|AAAAAAAAMDEBAAAA|950927|5777|47780|2452369|2452339|Mr.|Edwin|Jackson|Y|20|10|1985|NICARAGUA||Edwin.Jackson@SisLjRPCxlK.org|2452394| +5181|AAAAAAAANDEBAAAA|245091|1640|2189|2450968|2450938|Sir|Jose|Hartley|N|3|3|1933|NIUE||Jose.Hartley@ePxiJnlQRbzjC.edu|2452438| +5182|AAAAAAAAODEBAAAA|1118952|4067|7135|2451013|2450983|Sir|David|Davis|Y|28|4|1946|NEW CALEDONIA||David.Davis@2SMpLVnKk6HXZFyimY.org|2452642| +5183|AAAAAAAAPDEBAAAA|412417|1568|48766|2450462|2450432|Miss|Kimberly|Vasquez|N|11|10|1962|SWEDEN||Kimberly.Vasquez@4OxXeS74OclQsxtT5aD2.org|2452631| +5184|AAAAAAAAAEEBAAAA|||13130|||Sir|||N||||UNITED ARAB EMIRATES||William.Welsh@Bdmhq9JOUMFksS5v.org|| +5185|AAAAAAAABEEBAAAA|1283444|3617|35390|2449536|2449506|Dr.|Diana|Lovejoy|N|6|6|1961|ETHIOPIA||Diana.Lovejoy@siAMahE3GKg8sM.org|2452425| +5186|AAAAAAAACEEBAAAA|1224217|6129|9241|2450897|2450867|Sir|Frank|Elder|Y|19|6|1947|SENEGAL||Frank.Elder@PeAsshCVPJzak1e.com|2452321| +5187|AAAAAAAADEEBAAAA|1787797|2053|45509|2451855|2451825|Ms.|Carla|Coleman|Y|15|5|1990|RUSSIAN FEDERATION||Carla.Coleman@iRE.edu|2452346| +5188|AAAAAAAAEEEBAAAA|538636|1193|48150|2451186|2451156|Mr.|Ronald|Bolling|Y|20|3|1990|ALGERIA||Ronald.Bolling@BHIOiXu.com|2452405| +5189|AAAAAAAAFEEBAAAA|1272653|425|8950|2452498|2452468|Ms.|Cheryl|Moye|N|19|5|1967|JERSEY||Cheryl.Moye@nHJYyK4nd83.edu|2452569| +5190|AAAAAAAAGEEBAAAA|1848052|6739|8226|2451657|2451627|Ms.|April|Stewart|Y|4|11|1959|MONGOLIA||April.Stewart@824ReVsFqrvT.edu|2452317| +5191|AAAAAAAAHEEBAAAA|1631573|5935|40697|2450864|2450834|Mr.|Peter|Haney|N|15|3|1948|FRENCH GUIANA||Peter.Haney@JnuCLy4MXcJN.com|2452337| +5192|AAAAAAAAIEEBAAAA|1724762|5367|29572|2451023|2450993|Mr.|James|Gilbreath|N|14|3|1960|ECUADOR||James.Gilbreath@Eo1TLmHlMTjeKVGYY.org|2452360| +5193|AAAAAAAAJEEBAAAA|840485|4200|4268|2451473|2451443|Mrs.|Mabel|Cunningham|N|12|6|1938|URUGUAY||Mabel.Cunningham@xGsU6eoeCuvrgy.org|2452466| +5194|AAAAAAAAKEEBAAAA|106484|5236|44435|2450392|2450362|Sir|William|Stearns|N|28|11|1961|DOMINICA||William.Stearns@DnA6LzXODfmPhhnA3V.com|2452317| +5195|AAAAAAAALEEBAAAA|567219|542|22265|2449810|2449780|Mrs.|Victoria|Cotton|Y|12|8|1950|VIET NAM||Victoria.Cotton@aEBKGeDdA.org|2452431| +5196|AAAAAAAAMEEBAAAA|809883|814|45392|2451254|2451224|Mrs.|Lena|Hawkins|Y|13|6|1945|AUSTRIA||Lena.Hawkins@CiC.org|2452619| +5197|AAAAAAAANEEBAAAA|1301681|4096|26158|2450023|2449993|Mrs.|Charmaine|Sosa|N|13|10|1927|INDIA||Charmaine.Sosa@t5fGj.com|2452584| +5198|AAAAAAAAOEEBAAAA|1141838|631|3777|2449704|2449674|Miss|Dorothy|Anderson|N|23|6|1987|AZERBAIJAN||Dorothy.Anderson@Z9hmspBA.com|2452369| +5199|AAAAAAAAPEEBAAAA|1343010|4123|28389|2451597|2451567|Sir|William|Thomas|Y|21|1|1929|COSTA RICA||William.Thomas@BMOADZVCfzzP.com|2452611| +5200|AAAAAAAAAFEBAAAA|980336|5990|5644|2452215|2452185|Dr.||Morris|Y||5||GHANA||Kendall.Morris@CjE.edu|2452597| +5201|AAAAAAAABFEBAAAA|1866289|473|32657|2451732|2451702|Sir|Frances|Smith|Y|19|1|1931|SOMALIA||Frances.Smith@fXutYs.com|2452451| +5202|AAAAAAAACFEBAAAA|279324|954|25831|2450406|2450376|Mr.|Charles|Williams|Y|22|11|1984|POLAND||Charles.Williams@JGu.edu|2452589| +5203|AAAAAAAADFEBAAAA|595455|7132|38456|2452210|2452180|Dr.|Peter|Hall|N|10|4|1940|NETHERLANDS||Peter.Hall@T1E9zs5rCKEi6jF.com|2452341| +5204|AAAAAAAAEFEBAAAA|1394405|1654|47256|2450479|2450449|Mrs.|Catherine|Lang|Y|22|7|1966|NETHERLANDS ANTILLES||Catherine.Lang@r5Y4qRHTBKI.org|2452645| +5205|AAAAAAAAFFEBAAAA|1904517|6275|39633|2452668|2452638|Mr.|James|Johnson|Y|4|5|1926|JAMAICA||James.Johnson@OVuRrbBDuxx1J.com|2452387| +5206|AAAAAAAAGFEBAAAA|1908336|6165|44794|2452526|2452496|Mrs.|Sandra|Stephenson|N|6|1|1953|LIECHTENSTEIN||Sandra.Stephenson@2RKHdFnbeF.com|2452295| +5207|AAAAAAAAHFEBAAAA|535833|6695|31957|2450847|2450817|Miss|Juana|Taylor|N|23|11|1954|QATAR||Juana.Taylor@AE3Hpkhbat6RMOhHx.org|2452368| +5208|AAAAAAAAIFEBAAAA|1061961|2220|44432|2451901|2451871|Sir|Cecil|Johnson|N|29|5|1970|NIGER||Cecil.Johnson@ZH7rp60OBro.com|2452554| +5209|AAAAAAAAJFEBAAAA|125846|2371|35249|2451614|2451584|Mrs.|Dawn|Cantu|N|7|3|1983|FRANCE||Dawn.Cantu@urnJa2dqq.com|2452579| +5210|AAAAAAAAKFEBAAAA|1690477|6346|40262|2451091|2451061|Sir|Timothy|Cheng|Y|29|5|1936|MYANMAR||Timothy.Cheng@t.edu|2452415| +5211|AAAAAAAALFEBAAAA|1046104|3605|26259|2451724|2451694|Mr.|Gregory|Armstrong|N|9|1|1970|CZECH REPUBLIC||Gregory.Armstrong@oPbmRDYfqippc.org|2452561| +5212|AAAAAAAAMFEBAAAA|1241290|5221|25552|2450511|2450481|Mr.|Tony|Chandler|N|31|1|1957|BULGARIA||Tony.Chandler@fzmJNh31uAoVhd.edu|2452455| +5213|AAAAAAAANFEBAAAA|889876|4299|48322|2449637|2449607|Dr.|Alexandria|Cady|Y|4|8|1930|ECUADOR||Alexandria.Cady@aGF8efysfutuJFcKJQ.org|2452300| +5214|AAAAAAAAOFEBAAAA|1077504|2055|44959|2450683|2450653|Dr.|Kara|Horton|Y|3|4|1967|GIBRALTAR||Kara.Horton@KRUz8oN9bTEr7.org|2452367| +5215|AAAAAAAAPFEBAAAA|1870601|5329|26410|2449816|2449786|Sir|Philip|Beasley|N|14|7|1956|BURKINA FASO||Philip.Beasley@6mRb0C7FMOGI.com|2452453| +5216|AAAAAAAAAGEBAAAA|404404|3752|30653|2452094|2452064|Mrs.|Sharon|Porter|Y|25|1|1987|PUERTO RICO||Sharon.Porter@iFK7Z3vz.org|2452284| +5217|AAAAAAAABGEBAAAA|1846171|5091|49164|2450837|2450807|Mr.|Cortez|Sanchez|N|13|3|1954|LESOTHO||Cortez.Sanchez@7TTknY.com|2452619| +5218|AAAAAAAACGEBAAAA|435444|327|29657|2450840|2450810|Dr.|Daniel|Foreman|N|13|4|1958|MALDIVES||Daniel.Foreman@mP5L.org|2452296| +5219|AAAAAAAADGEBAAAA|46927|4424|29474|2449833|2449803|Dr.|Victoria|Mosley|N|17|3|1954|EL SALVADOR||Victoria.Mosley@HTNJnf3RgpJ.com|2452365| +5220|AAAAAAAAEGEBAAAA|606777|3929|38281|2451098|2451068|Sir|Charles|Long|N|17|3|1984|BELGIUM||Charles.Long@xumVqtNUklAsBdGJ8ez.org|2452406| +5221|AAAAAAAAFGEBAAAA|515713|3837|30314|2449757|2449727|Dr.|Mildred|Messer|N|20|2|1967|AMERICAN SAMOA||Mildred.Messer@N.edu|2452599| +5222|AAAAAAAAGGEBAAAA|1864079|1993|12754|2451965|2451935|Miss|Debra|Riggs|Y|7|1|1986|BHUTAN||Debra.Riggs@3t77ZOZ9Q.org|2452640| +5223|AAAAAAAAHGEBAAAA|1426498|6242|14386|2452515|2452485|Dr.|Daniel|Weatherly|N|1|10|1962|ANGOLA||Daniel.Weatherly@mDcAPCY4MJHLK6C.edu|2452456| +5224|AAAAAAAAIGEBAAAA|1784449|2760|45871|2449648|2449618|Sir|Philip|Beach|Y|14|6|1973|KIRIBATI||Philip.Beach@Ftgnm9p7gLAphHz.org|2452621| +5225|AAAAAAAAJGEBAAAA|1628574|2827|41480|2450738|2450708|Mr.|Michael|Crowe|N|2|9|1965|GAMBIA||Michael.Crowe@HFyy3z.edu|2452311| +5226|AAAAAAAAKGEBAAAA|620027|171|22715|2449649|2449619|Dr.|Steven|Knowles|Y|3|3|1929|ANGUILLA||Steven.Knowles@yJqKF4uYK5.edu|2452549| +5227|AAAAAAAALGEBAAAA|472420|6896|36572|2449861|2449831|Ms.|Alicia|Cobb|Y|10|7|1963|ERITREA||Alicia.Cobb@1qUry.org|2452581| +5228|AAAAAAAAMGEBAAAA|1680345|676|12607|2450997|2450967|Mrs.|Gertrude|Stricklin|N|3|12|1986|NAMIBIA||Gertrude.Stricklin@ejsvq.com|2452361| +5229|AAAAAAAANGEBAAAA|628733|4449|38165|2450184|2450154|Sir|Robert|Hay|Y|22|6|1929|AFGHANISTAN||Robert.Hay@h1yzm7F9Cczv4Z.com|2452351| +5230|AAAAAAAAOGEBAAAA|1444068|5172|13029|2451270|2451240|Mr.|Bill|Jefferson|Y|10|6|1982|CYPRUS||Bill.Jefferson@YI.edu|2452618| +5231|AAAAAAAAPGEBAAAA|1659709|888|43714|2450003|2449973|Ms.|Laura|Burgess|N|3|11|1970|MALAYSIA||Laura.Burgess@gbyrMeyv4eA.com|2452483| +5232|AAAAAAAAAHEBAAAA|42510|3432|8307|2450931|2450901|Dr.|Adrienne|King|Y|28|11|1940|NEPAL||Adrienne.King@sog3.edu|2452560| +5233|AAAAAAAABHEBAAAA|1479563|3958|48306|2450553|2450523|Sir|Thomas|Wilson|N|26|12|1955|MARTINIQUE||Thomas.Wilson@U2xeKP6.com|2452328| +5234|AAAAAAAACHEBAAAA|||30061|||||Martinez|||9|1932|TAJIKISTAN||Hayley.Martinez@qEKFyf9YVz3cTr4eQlc.com|| +5235|AAAAAAAADHEBAAAA|100175||10375|2451030|2451000||Barbara|Lawrence||24|9|1952|RWANDA||Barbara.Lawrence@IRZ.com|| +5236|AAAAAAAAEHEBAAAA|301736|2319|42102|2450035|2450005|Mr.|Ryan|Brown|N|13|2|1988|BERMUDA||Ryan.Brown@gtManUIjgr.edu|2452479| +5237|AAAAAAAAFHEBAAAA|316721|2164|8407|2451368|2451338|Ms.|Marcella|Daniels|N|22|1|1974|CAYMAN ISLANDS||Marcella.Daniels@jOa74A6Zp1Qp9.com|2452359| +5238|AAAAAAAAGHEBAAAA|1159089|3251|13413|2452443|2452413|Ms.|Latonya|Torres|Y|30|6|1974|DENMARK||Latonya.Torres@CGG.edu|2452374| +5239|AAAAAAAAHHEBAAAA|542940||40266|2452518|2452488|Sir|Mark||||||MONTSERRAT|||| +5240|AAAAAAAAIHEBAAAA|1591270|4409|33948|2452125|2452095|Sir|Sam|Mcfarlane|N|12|4|1955|MONTENEGRO||Sam.Mcfarlane@nQa2rSgzlkit.org|2452361| +5241|AAAAAAAAJHEBAAAA|31169|1579|21636|2451461|2451431|Dr.|Lottie|Castillo|Y|1|7|1933|SERBIA||Lottie.Castillo@NANXbsZ1Jac3Gua.org|2452444| +5242|AAAAAAAAKHEBAAAA|153166||38870|2450699||Sir|||||6|||||| +5243|AAAAAAAALHEBAAAA|1741361|1456|2304|2451181|2451151|Dr.|Everett|Gonzalez|Y|26|10|1957|COMOROS||Everett.Gonzalez@1Dmkh.edu|2452537| +5244|AAAAAAAAMHEBAAAA|1817727|5620|28313|2450435|2450405|Dr.|John|Payne|N|11|9|1972|FINLAND||John.Payne@YA5jjb.com|2452584| +5245|AAAAAAAANHEBAAAA|1328755|3303|32251|2452646|2452616|Mr.|Walter|Beaudoin|Y|8|2|1982|PAPUA NEW GUINEA||Walter.Beaudoin@CAqfk04OyYmaB1.org|2452298| +5246|AAAAAAAAOHEBAAAA|1698099|1382|27912|2451926|2451896|Dr.|Leonardo|Main|Y|10|10|1991|NAURU||Leonardo.Main@Lcut.com|2452560| +5247|AAAAAAAAPHEBAAAA|307000|5352|16904|2451330|2451300|Dr.|Nellie|Wolf|N|3|10|1931|HUNGARY||Nellie.Wolf@uVoqNtzfcTm.com|2452615| +5248|AAAAAAAAAIEBAAAA|1095365|1043|21568|2452538|2452508|Mr.|Mark|Gonzalez|Y|10|10|1956|UGANDA||Mark.Gonzalez@VfxJD92Bx14.com|2452335| +5249|AAAAAAAABIEBAAAA|1913156|2653|28950|2450809|2450779|Sir|Jonathan|Singleton|Y|8|2|1974|VANUATU||Jonathan.Singleton@lfAx5r5S8xrS5H1.com|2452330| +5250|AAAAAAAACIEBAAAA|840372|3644|33316|2451693|2451663|Mr.|Ronald|Park|N|7|10|1972|SPAIN||Ronald.Park@i.edu|2452637| +5251|AAAAAAAADIEBAAAA|897145|1297|32649|2450017|2449987|Ms.|Janine|Hanson|Y|4|4|1968|CHILE||Janine.Hanson@6VLhZf.org|2452622| +5252|AAAAAAAAEIEBAAAA|434857|3309|5361|2449751|2449721|Ms.|Katherine|Pace|Y|8|3|1980|ALBANIA||Katherine.Pace@VN1.com|2452511| +5253|AAAAAAAAFIEBAAAA|780009|477|44498|2450002|2449972|Mrs.|Martha|Welch|N|13|12|1991|NAMIBIA||Martha.Welch@NXt93Jqhy8u01.com|2452520| +5254|AAAAAAAAGIEBAAAA|1544372|2327|19435|2449866|2449836|Dr.|Anthony|Kerr|Y|2|12|1950|BARBADOS||Anthony.Kerr@fMT.edu|2452490| +5255|AAAAAAAAHIEBAAAA|1167037|3115|48793|2451887|2451857|Sir|Laurence|Blackwell|N|30|1|1977|THAILAND||Laurence.Blackwell@D.com|2452514| +5256|AAAAAAAAIIEBAAAA|343749|4918|4840|2451554|2451524|Mr.|Francis|Harris|N|12|9|1930|AFGHANISTAN||Francis.Harris@T1iRLVF4Ubs.edu|2452285| +5257|AAAAAAAAJIEBAAAA|895459|1866|47120|2451450|2451420|Dr.|Wayne|Evans|Y|12|10|1980|RWANDA||Wayne.Evans@h4xub27YKHDIi.com|2452545| +5258|AAAAAAAAKIEBAAAA|17677|1790|29103|2449554|2449524|Mrs.|Valerie|Ball|Y|18|4|1966|MALDIVES||Valerie.Ball@sgAFsOSUkU.com|2452512| +5259|AAAAAAAALIEBAAAA|10341|5380|22372|2450807|2450777|Miss|Lois|Boyd|Y|4|1|1992|ALGERIA||Lois.Boyd@iRhIDPG.edu|2452326| +5260|AAAAAAAAMIEBAAAA|1280998|5763|37681|2451672|2451642|Sir|Raymond|Nance|Y|12|11|1984|ZAMBIA||Raymond.Nance@nOiMMIeRKehUmn02F.com|2452489| +5261|AAAAAAAANIEBAAAA|1353437|6854|16624|2451598|2451568|Miss|Rosa|Newsome|Y|28|10|1932|MALDIVES||Rosa.Newsome@7VdM.org|2452635| +5262|AAAAAAAAOIEBAAAA|1836173|1336|4099|2451422|2451392|Miss|Janet|Mcrae|Y|11|8|1932|LUXEMBOURG||Janet.Mcrae@7YB.org|2452405| +5263|AAAAAAAAPIEBAAAA|104844|1048|4452|2450525|2450495|Miss|Janet|Diamond|Y|5|2|1947|EGYPT||Janet.Diamond@r8kelIuehmELoc.org|2452361| +5264|AAAAAAAAAJEBAAAA|1878417|389|26008|2449337|2449307|Sir|Vance|Walls|N|4|4|1925|COSTA RICA||Vance.Walls@aX.edu|2452491| +5265|AAAAAAAABJEBAAAA|917448|2092|29983|2450837|2450807|Mrs.|Chanel|Torres|Y|26|8|1933|GUINEA||Chanel.Torres@6vLnmooq.edu|2452615| +5266|AAAAAAAACJEBAAAA|754713|6145|40377|2452559|2452529|Dr.|Karen|Simon|Y|1|2|1951|AMERICAN SAMOA||Karen.Simon@yZQoc43BFY.org|2452648| +5267|AAAAAAAADJEBAAAA|898493|5729|38612|2449241|2449211|Mrs.|Suzette|Bowie|N|11|7|1948|ARUBA||Suzette.Bowie@1sebBb.edu|2452346| +5268|AAAAAAAAEJEBAAAA|1688079|3706|28933|2450192|2450162|Sir|Richard|Burns|Y|26|12|1958|UNITED STATES||Richard.Burns@vBZhoIPyR7lmk.edu|2452438| +5269|AAAAAAAAFJEBAAAA|336687|5755|22023|2450750|2450720|Dr.|Marissa|Pitts|Y|6|12|1935|NETHERLANDS||Marissa.Pitts@QjF8Oy0rxL67ADtO.org|2452399| +5270|AAAAAAAAGJEBAAAA|747486|2032|35778|2449108|2449078|Dr.|Charles|Billups|N|12|4|1942|OMAN||Charles.Billups@H96lYEZ.com|2452381| +5271|AAAAAAAAHJEBAAAA|147264|5307|33703|2450252||Sir||||1||1968|QATAR|||| +5272|AAAAAAAAIJEBAAAA|528212|142|39225|2449448|2449418|Mrs.|Ann|Schultz|N|24|1|1972|BRUNEI DARUSSALAM||Ann.Schultz@paIO6LCshbOhyQx3.com|2452601| +5273|AAAAAAAAJJEBAAAA|1183199|1181|45516|2450204|2450174|Miss|Victoria|Epps|N|8|9|1943|TONGA||Victoria.Epps@jZXxvu4T.org|2452368| +5274|AAAAAAAAKJEBAAAA|1115695|4182|41916|2452246|2452216|Dr.|Kristen|Jennings|N|30|10|1930|YEMEN||Kristen.Jennings@YLcjOHgr3sR60lpi.org|2452583| +5275|AAAAAAAALJEBAAAA|246572|537|18502|2449261|2449231|Dr.|Jennifer|Smith|Y|17|5|1940|PUERTO RICO||Jennifer.Smith@kf4dx70mkMlX.org|2452453| +5276|AAAAAAAAMJEBAAAA|1307388|5113|5991|2451048|2451018|Mr.|Jesse|Rosales|N|9|5|1930|ARMENIA||Jesse.Rosales@aiZBnez2KLhuQ2.edu|2452284| +5277|AAAAAAAANJEBAAAA|527592|4824|8009|2451573|2451543|Sir|Bernard|Parker|Y|10|3|1935|EL SALVADOR||Bernard.Parker@X.org|2452305| +5278|AAAAAAAAOJEBAAAA|1225658|5357|20209|2450581|2450551|Ms.|Vicki|Santos|Y|29|6|1974|GERMANY||Vicki.Santos@Sia5ve6.org|2452405| +5279|AAAAAAAAPJEBAAAA|649621|2218|11262|2450367|2450337|Ms.|Latonya|Garza|N|29|2|1968|UNITED KINGDOM||Latonya.Garza@1ZZ.org|2452301| +5280|AAAAAAAAAKEBAAAA|875315|7186|20657|2450297|2450267|Miss|Margaret|Hernandez|Y|18|9|1964|TONGA||Margaret.Hernandez@d7kssvFtS.edu|2452520| +5281|AAAAAAAABKEBAAAA|401824|6386|26148|2451666|2451636|Dr.|Yolanda|Jones|Y|21|11|1958|CANADA||Yolanda.Jones@VY.com|2452379| +5282|AAAAAAAACKEBAAAA|953371|3995|31866|2451080|2451050|Miss|Kathy|Franco|Y|2|3|1937|SOUTH AFRICA||Kathy.Franco@BbBXc963F1.edu|2452550| +5283|AAAAAAAADKEBAAAA|690909|5123|23737|2450371|2450341|Sir|Gerald|Crockett|Y|29|6|1985|TONGA||Gerald.Crockett@6BgkP6.com|2452612| +5284|AAAAAAAAEKEBAAAA|1198487|5492|36497|2450832|2450802|Sir|Todd|Gonzales|N|15|1|1967|GUADELOUPE||Todd.Gonzales@FMen2YenUXpf.org|2452353| +5285|AAAAAAAAFKEBAAAA|1831456|3056|40703|2449264|2449234|Mr.|Bobby|Austin|N|10|12|1939|BRUNEI DARUSSALAM||Bobby.Austin@Di09LQ.org|2452513| +5286|AAAAAAAAGKEBAAAA|527612|3900|13341|2449521|2449491|Mr.|Leonard|Smith|N|10|3|1949|MONACO||Leonard.Smith@l5g3vHl9UeGHSz4.edu|2452644| +5287|AAAAAAAAHKEBAAAA|676978|4452|43173|2449300|2449270|Mr.|Stuart|Pena|Y|31|12|1961|MOZAMBIQUE||Stuart.Pena@ni9nOQp2LLxfUls2p5.edu|2452575| +5288|AAAAAAAAIKEBAAAA|1125045|6822|28246|2450547|2450517|Dr.|Deborah|Marcus|Y|11|11|1933|ESTONIA||Deborah.Marcus@jPBt43R.org|2452358| +5289|AAAAAAAAJKEBAAAA|307605|4074|42496|2450258|2450228|Ms.|Sandra|Parker|N|19|10|1931|TUNISIA||Sandra.Parker@8jvv.org|2452594| +5290|AAAAAAAAKKEBAAAA|1413420|2502|20784|2451989|2451959|Dr.|Silvia|Smith|Y|30|3|1958|COMOROS||Silvia.Smith@i2M1I5K3vrXIA.org|2452449| +5291|AAAAAAAALKEBAAAA|1853691|1552|12213|2450159|2450129|Dr.|Janet|Miles|Y|4|10|1951|RWANDA||Janet.Miles@Eb.edu|2452527| +5292|AAAAAAAAMKEBAAAA|561690|5262|81|2450067|2450037|Sir|Martin|Waller|N|6|10|1947|SURINAME||Martin.Waller@2KY.com|2452342| +5293|AAAAAAAANKEBAAAA|814775|6126|6630|2451716|2451686|Dr.|Stefan|Mcdonald|N|2|7|1969|AUSTRIA||Stefan.Mcdonald@vOjlrKs8fxX.com|2452438| +5294|AAAAAAAAOKEBAAAA|1826589|4406|44270|2449298|2449268|Mrs.|Veronica|Jones|Y|19|2|1971|ARMENIA||Veronica.Jones@r72.edu|2452447| +5295|AAAAAAAAPKEBAAAA|1198600|3844|43135|2451799|2451769|Mrs.|Amanda|Stewart|Y|2|6|1970|NEW ZEALAND||Amanda.Stewart@EFq2UXksDER.edu|2452488| +5296|AAAAAAAAALEBAAAA|629724|6203|34282|2451186|2451156|Mrs.|Tamika|Baber|N|20|10|1967|ANGOLA||Tamika.Baber@ICZFc2CTjFB1.org|2452625| +5297|AAAAAAAABLEBAAAA|933953|5859|20118|2451969|2451939|Dr.|Jaclyn|Smith|Y|9|5|1954|NORWAY||Jaclyn.Smith@bvUdAboYHla.edu|2452495| +5298|AAAAAAAACLEBAAAA|222185|923|11708|2451403|2451373|Dr.|Frank||Y|9|7|||||| +5299|AAAAAAAADLEBAAAA|1019352|2085|30771|2451669|2451639|Mr.|Charles|Luke|Y|18|6|1967|ARUBA||Charles.Luke@IK0UbsaMyygDCbXJa7qy.org|2452496| +5300|AAAAAAAAELEBAAAA|1617167|2706|29492|2452645|2452615|Mrs.|Dianne|Ross|Y|19|7|1949|ICELAND||Dianne.Ross@D4VH75e9n.org|2452429| +5301|AAAAAAAAFLEBAAAA|231048|5220|13345|2450436|2450406|Ms.|Stacie|Fernandez|Y|31|12|1934|TURKMENISTAN||Stacie.Fernandez@EGj7JssRPFZx5ZQq.com|2452546| +5302|AAAAAAAAGLEBAAAA|1519188|660|37332|2451073|2451043|Mr.|Cody|Bauman|N|20|11|1934|SIERRA LEONE||Cody.Bauman@VA1FP.com|2452311| +5303|AAAAAAAAHLEBAAAA|998945|6378|31364|2451583|2451553|Mrs.|Heather|Mcreynolds|Y|3|10|1972|MONTENEGRO||Heather.Mcreynolds@LHqCpFBdO74NcFQk.com|2452437| +5304|AAAAAAAAILEBAAAA|1837187|2430|26426|2449224|2449194|Sir|Jerome|Murphy|N|10|7|1975|HONG KONG||Jerome.Murphy@RxPnc.com|2452311| +5305|AAAAAAAAJLEBAAAA|424527|5607|24287|2449254|2449224|Mr.|Ronald|Zimmerman|Y|10|4|1950|MADAGASCAR||Ronald.Zimmerman@Ho5mcOx2Oc.com|2452510| +5306|AAAAAAAAKLEBAAAA|1307747|968|36589|2450547|2450517|Ms.|Theresa|Rhodes|N|9|1|1989|ARUBA||Theresa.Rhodes@GIEIEbpy2Jj.com|2452559| +5307|AAAAAAAALLEBAAAA|956386|2159|1932|2451129|2451099|Ms.|Carletta|Slater|N|16|10|1992|ALGERIA||Carletta.Slater@RimIGk9z.com|2452434| +5308|AAAAAAAAMLEBAAAA|221254|5457|30860|2450233|2450203|Miss|Marie|Mccray|N|2|7|1945|INDONESIA||Marie.Mccray@Qy.com|2452299| +5309|AAAAAAAANLEBAAAA|1028682|2153|36424|2450618|2450588|Miss|Valerie|Glover|Y|6|11|1926|IRELAND||Valerie.Glover@1jxAG53O.com|2452463| +5310|AAAAAAAAOLEBAAAA|188699|84|25869|2449579|2449549|Miss|Brenda|Lange|Y|16|1|1965|FRANCE||Brenda.Lange@1CO98Sj.org|2452640| +5311|AAAAAAAAPLEBAAAA|1606769|1061|35753|2450857|2450827|Dr.|Eric|Hogan|N|10|5|1947|QATAR||Eric.Hogan@HbHZVzvPPdxkY4ST5djb.edu|2452498| +5312|AAAAAAAAAMEBAAAA|1438988|1637|4686|2449386|2449356|Mr.|Roger|Mitchell|Y|4|9|1988|EQUATORIAL GUINEA||Roger.Mitchell@R9BgARvhgv.com|2452398| +5313|AAAAAAAABMEBAAAA|429958|2202|19293|2450166|2450136|Dr.|Deborah|Clark|Y|7|10|1953|SPAIN||Deborah.Clark@AaNUI63zTDABSA0.com|2452528| +5314|AAAAAAAACMEBAAAA|975327|3751|29822|2450347|2450317|Dr.|Stephen|Duval|Y|22|9|1979|TOGO||Stephen.Duval@dtLgEZdDK4.org|2452284| +5315|AAAAAAAADMEBAAAA|29335|4256|33224|2449746|2449716|Mr.|Thomas|Miller|Y|5|11|1988|ALGERIA||Thomas.Miller@VboTVUdqle.com|2452462| +5316|AAAAAAAAEMEBAAAA|317131|2650|19369|2451825|2451795|Ms.|Laura|Johnson|Y|14|4|1938|CANADA||Laura.Johnson@V4E.org|2452611| +5317|AAAAAAAAFMEBAAAA|1307371|5728|33285|2449211|2449181|Dr.|Bonnie|Ray|N|19|6|1953|LATVIA||Bonnie.Ray@k9MQvF22I7.edu|2452341| +5318|AAAAAAAAGMEBAAAA|125540|1182|29224|2451175|2451145|Mr.|Duane|Smith|N|16|8|1959|EL SALVADOR||Duane.Smith@NCDagtkrOV.org|2452439| +5319|AAAAAAAAHMEBAAAA|1432525|6049|46020|2449123|2449093|Dr.|Roger|Collins|Y|30|12|1953|ARGENTINA||Roger.Collins@prBeZnZTij.edu|2452601| +5320|AAAAAAAAIMEBAAAA|477185|1650|12915|2449768|2449738|Miss|Emily|Flanagan|Y|15|5|1924|INDIA||Emily.Flanagan@7zpGiRVvesX.org|2452592| +5321|AAAAAAAAJMEBAAAA|1240558|5517|20076|2450036|2450006|Mrs.|Elsie|Escalante|N|24|5|1960|TOKELAU||Elsie.Escalante@Tlj5MgHH.edu|2452414| +5322|AAAAAAAAKMEBAAAA|582258|5765|2025|2450119|2450089|Mr.|Dean|Deaton|N|19|3|1964|IRAQ||Dean.Deaton@SO2Vea2NUbcZX.edu|2452329| +5323|AAAAAAAALMEBAAAA|1823119|2899|44679|2451059|2451029|Mr.|Richard|Henley|Y|2|2|1978|SINGAPORE||Richard.Henley@n.com|2452630| +5324|AAAAAAAAMMEBAAAA|921880|3242|36557|2451553|2451523|Mr.|David|Battaglia|N|31|8|1983|ISLE OF MAN||David.Battaglia@HbUxRE.org|2452547| +5325|AAAAAAAANMEBAAAA|88876|4949|9022|2449200|2449170|Mrs.|Julia|Ellington|N|12|11|1927|BAHRAIN||Julia.Ellington@AthkOE.edu|2452338| +5326|AAAAAAAAOMEBAAAA|763110|5986|36109|2452449|2452419|Mr.|John|Rice|Y|13|3|1983|SLOVAKIA||John.Rice@cZTzjiQFkpo.org|2452648| +5327|AAAAAAAAPMEBAAAA|1404287|2337|9234|2449074|2449044|Dr.|Abigail|Mcdaniel|Y|2|9|1956|JAMAICA||Abigail.Mcdaniel@m5cBAtTk.org|2452340| +5328|AAAAAAAAANEBAAAA|353763|2979|42148|2449184|2449154|Dr.|Michael|Franklin|N|20|8|1949|AZERBAIJAN||Michael.Franklin@POv4eDA.com|2452382| +5329|AAAAAAAABNEBAAAA|1191169|4148|45527|2452384|2452354|Mr.|Thomas|Mathews|N|18|9|1960|CHILE||Thomas.Mathews@vPFGfadH7lxO1fXtx.org|2452350| +5330|AAAAAAAACNEBAAAA|59366|7005|11891|2450312|2450282|Ms.|Julia|Farley|Y|12|11|1985|INDONESIA||Julia.Farley@lxnRPcoz.org|2452360| +5331|AAAAAAAADNEBAAAA|1111526|5000|32827|2449263|2449233|Miss|Karen|Hopkins|N|26|8|1942|ESTONIA||Karen.Hopkins@3nJGeBp61A7RBEb.edu|2452304| +5332|AAAAAAAAENEBAAAA|154752|3363|18727|2452292|2452262|Dr.|Jason|Garrett|N|12|12|1979|RUSSIAN FEDERATION||Jason.Garrett@hEKDDkkkMa.com|2452347| +5333|AAAAAAAAFNEBAAAA|170813|6813|36013|2452002|2451972|Sir|Gary|Allen|N|20|6|1942|SWITZERLAND||Gary.Allen@GE5k1ZYLBic34R7O.com|2452507| +5334|AAAAAAAAGNEBAAAA|1716558|77|19643|2449577|2449547|Ms.|Marilyn|Alvarez|Y|1|9|1973|NEPAL||Marilyn.Alvarez@3fCKBsCFHIv.org|2452416| +5335|AAAAAAAAHNEBAAAA|1034911|5793|40275|2452281|2452251|Sir|Sang|Dunn|N|15|7|1953|URUGUAY||Sang.Dunn@ZAmyYnPmRglAIocRS.edu|2452311| +5336|AAAAAAAAINEBAAAA|647509|1822|1735|2450763|2450733|Sir|Steven|Johnson|Y|8|5|1942|BOTSWANA||Steven.Johnson@epDDB6oZZJ1o.org|2452525| +5337|AAAAAAAAJNEBAAAA|1540341|1740|21186|2451376|2451346|Sir|Robert|Gross|N|10|8|1963|SUDAN||Robert.Gross@HgACsgOHZszV1qZ.edu|2452470| +5338|AAAAAAAAKNEBAAAA|391341|530|47719|2450886|2450856|Sir|Eugene|Porter|N|27|6|1966|TOGO||Eugene.Porter@S8XuSETObJ2.com|2452463| +5339|AAAAAAAALNEBAAAA|49661|2532|32556|2450845|2450815|Miss|Marlene|Mccullough|Y|9|2|1991|GUATEMALA||Marlene.Mccullough@iNqpuBLhhVdjnH.edu|2452395| +5340|AAAAAAAAMNEBAAAA|653883|1235|36587|2451740|2451710|Miss|Anne|Wagner|Y|22|9|1976|EGYPT||Anne.Wagner@nkY3G07a1SZ0pI.edu|2452617| +5341|AAAAAAAANNEBAAAA|1522405|3222|40486|2451104|2451074|Ms.|Gina|Fuller|Y|1|4|1940|MAURITANIA||Gina.Fuller@PRrB5jFCzdZlKc.org|2452328| +5342|AAAAAAAAONEBAAAA|98961|6278|7195|2449147|2449117|Dr.|Ervin|Dean|Y|6|12|1976|TAJIKISTAN||Ervin.Dean@cbfL.com|2452543| +5343|AAAAAAAAPNEBAAAA||6745|4571|||Dr.|||||11||||Julia.Lee@egOjPReGt8Q6E.com|| +5344|AAAAAAAAAOEBAAAA|614894|3879|42817|2451269|2451239|Dr.|Alice|Long|Y|17|7|1961|FRANCE||Alice.Long@TmS3ZN.edu|2452620| +5345|AAAAAAAABOEBAAAA|1585587|5946|27858|2449841|2449811|Mr.|Christopher|Holt|Y|4|11|1947|BRAZIL||Christopher.Holt@rXGdTdxCPMgeV0Uq.org|2452342| +5346|AAAAAAAACOEBAAAA|1768188|1336|20037|2452167|2452137|Dr.|Jerry|Flores|N|1|4|1932|KAZAKHSTAN||Jerry.Flores@sqoUgGCtTUCN.org|2452310| +5347|AAAAAAAADOEBAAAA|819321|5295|20069|2449084|2449054|Dr.|George|Peterson|Y|23|3|1945|JAMAICA||George.Peterson@1XBe99.org|2452611| +5348|AAAAAAAAEOEBAAAA|||39202||||James||Y||1|1969|TOGO||James.Pierce@n836ZHtnVk4.edu|| +5349|AAAAAAAAFOEBAAAA|926318|5866|12661|2450937|2450907|Dr.|Barbara|Hopper|N|1|8|1951|SAUDI ARABIA||Barbara.Hopper@3PitcS.org|2452321| +5350|AAAAAAAAGOEBAAAA|1831344|1991|21038|2452018|2451988|Mr.|Donnie|Leung|Y|1|12|1947|CZECH REPUBLIC||Donnie.Leung@IsC8Z.org|2452601| +5351|AAAAAAAAHOEBAAAA||5588|16108|||Mr.|||N||4||KENYA|||| +5352|AAAAAAAAIOEBAAAA|1017584|838|21692|2450271|2450241|Dr.|John|Salazar|Y|1|7|1963|VIRGIN ISLANDS, U.S.||John.Salazar@rkjBKoM790.com|2452413| +5353|AAAAAAAAJOEBAAAA|163885|1413|16588|2449404|2449374|Miss|Cindy|Means|Y|6|10|1954|ANGOLA||Cindy.Means@Stfr.org|2452600| +5354|AAAAAAAAKOEBAAAA|781389|1210|38584|2450245|2450215|Dr.|John|James|Y|16|7|1944|SAMOA||John.James@XHstjYL7zGbZA9x.com|2452470| +5355|AAAAAAAALOEBAAAA|1161028|3894|36760|2450020|2449990|Dr.|Christopher|Redmond|N|14|9|1966|IRELAND||Christopher.Redmond@OFHONoOYpO.org|2452517| +5356|AAAAAAAAMOEBAAAA||4587|33180|2451770||||Thompson||||1953|||Darlene.Thompson@AtAs.org|| +5357|AAAAAAAANOEBAAAA|21089|1270|44651|2451915|2451885|Mr.|David|Saunders|N|27|2|1980|UZBEKISTAN||David.Saunders@4UboR.com|2452624| +5358|AAAAAAAAOOEBAAAA|1147799|1451|22350|2449519|2449489|Sir|Devon|Wallace|N|3|12|1935|JORDAN||Devon.Wallace@Qu0BhCs.com|2452442| +5359|AAAAAAAAPOEBAAAA|318267|403|23292|||Ms.||Lewis|N||3|1952|PUERTO RICO|||2452588| +5360|AAAAAAAAAPEBAAAA|1661905|2509|19513|2449937|2449907|Mrs.|Jennifer|Polk|N|5|6|1987|SEYCHELLES||Jennifer.Polk@TODkGnP6b4.com|2452605| +5361|AAAAAAAABPEBAAAA|48902|2657|44812|2450671|2450641|Mrs.|Daisy|Gonzales|Y|28|3|1938|LESOTHO||Daisy.Gonzales@xYfgfjlfYATlRYO.edu|2452490| +5362|AAAAAAAACPEBAAAA|1067326|2768|31810|2449189|2449159|Mr.|Marc|Estes|Y|27|5|1924|FIJI||Marc.Estes@AGdx59Ac7rR.com|2452329| +5363|AAAAAAAADPEBAAAA|318684|2172|28460|2450589|2450559|Sir|Gary|Garner|N|14|7|1939|EL SALVADOR||Gary.Garner@AkU73C4H59S.edu|2452381| +5364|AAAAAAAAEPEBAAAA|951091|838|16985|2450747|2450717|Dr.|Virginia|Snyder|Y|15|9|1966|TUVALU||Virginia.Snyder@r.edu|2452366| +5365|AAAAAAAAFPEBAAAA|110563|2313|31263|2451047|2451017|Sir|Charles|Roberts|Y|12|6|1937|CUBA||Charles.Roberts@zOezgkHFpOL.edu|2452554| +5366|AAAAAAAAGPEBAAAA|45873|7173|20073|2450246|2450216|Dr.|Thomas|Garner|Y|22|8|1943|HUNGARY||Thomas.Garner@QG8sr5L1d.edu|2452336| +5367|AAAAAAAAHPEBAAAA|1639440|4659|36725|2450288|2450258|Sir|Kenneth|Munoz|Y|4|6|1975|CHRISTMAS ISLAND||Kenneth.Munoz@N1FBVPt4apuqg.org|2452580| +5368|AAAAAAAAIPEBAAAA|1404651|713|29128|2452616|2452586|Dr.|Claudine|Quezada|N|13|3|1974|MALAYSIA||Claudine.Quezada@HObqUK.org|2452405| +5369|AAAAAAAAJPEBAAAA|619888|6997|28459|2452419|2452389|Mr.|Frank|Johnson|N|10|10|1984|WALLIS AND FUTUNA||Frank.Johnson@QZVsylLftIzQUL2F.com|2452368| +5370|AAAAAAAAKPEBAAAA|780943|6665|9140|2451727|2451697|Dr.|Eldon|Alexander|N|22|7|1978|NAMIBIA||Eldon.Alexander@MC9ZViyINeCo.edu|2452618| +5371|AAAAAAAALPEBAAAA|66711|3000|47642|2450366|2450336|Ms.|Stephanie|Spence|N|16|4|1929|WESTERN SAHARA||Stephanie.Spence@0tr148FE.org|2452628| +5372|AAAAAAAAMPEBAAAA|1583674|4297|46967|2449401|2449371|Mrs.|Laura|Ritter|Y|16|6|1955|BENIN||Laura.Ritter@XYSQ9L.org|2452399| +5373|AAAAAAAANPEBAAAA|1343988|2072|25403|2449843|2449813|Dr.|Robert|Wenzel|Y|8|4|1943|CROATIA||Robert.Wenzel@5uoX3XSn.edu|2452595| +5374|AAAAAAAAOPEBAAAA|665720|476|9985|2451880|2451850|Dr.|Gilbert|Peoples|N|15|7|1943|ISRAEL||Gilbert.Peoples@XoEem9T0jmxos.org|2452316| +5375|AAAAAAAAPPEBAAAA|1048090|568|28293|2451387|2451357|Mrs.|Jennifer|Graham|Y|13|7|1947|FRANCE||Jennifer.Graham@uUunbmvxgR.edu|2452360| +5376|AAAAAAAAAAFBAAAA|1074903|4741|22626|2451675|2451645|Miss|Cora|Merchant|Y|23|2|1991|WALLIS AND FUTUNA||Cora.Merchant@6SsefSmpJh.edu|2452558| +5377|AAAAAAAABAFBAAAA|555507|7113|3072|2450918|2450888|Mrs.|Margaret|Cole|N|18|12|1969|SAINT LUCIA||Margaret.Cole@dD5UayhfEBT9VI.edu|2452490| +5378|AAAAAAAACAFBAAAA|1873207|1819|47231|2451304|2451274|Mr.|Robert|Jones|N|8|12|1938|GRENADA||Robert.Jones@s1.org|2452405| +5379|AAAAAAAADAFBAAAA|1072968|4090|42459|2449369|2449339|Dr.|James|Browning|Y|28|1|1933|BELIZE||James.Browning@ZSVMuz5GUsU6.com|2452343| +5380|AAAAAAAAEAFBAAAA|369681|1145|11762|2452389|2452359|Ms.|Sheron|Harris|Y|22|1|1945|GUATEMALA||Sheron.Harris@9m97nM2oxyM3dTon.org|2452485| +5381|AAAAAAAAFAFBAAAA|1316055|4648|46441|2452618|2452588|Ms.|Melissa|Robbins|Y|3|1|1933|IRELAND||Melissa.Robbins@EOBK4c.org|2452418| +5382|AAAAAAAAGAFBAAAA|713355|733|27264|2449715|2449685|Dr.|Richard|Wilson|Y|11|9|1929|ARUBA||Richard.Wilson@9KEaYmFBdJgVR5.org|2452396| +5383|AAAAAAAAHAFBAAAA|1476588|5470|3199|2451111|2451081|Mrs.|Arthur|Sauer|Y|21|10|1963|ISLE OF MAN||Arthur.Sauer@QUzxYihjnsGU4l.edu|2452390| +5384|AAAAAAAAIAFBAAAA|144984|5965|12099|2449670|2449640|Mr.|Jeffrey|Moore|N|2|4|1977|GUINEA||Jeffrey.Moore@8SH.com|2452501| +5385|AAAAAAAAJAFBAAAA|1274093|3416|8339|2450816|2450786|Sir|Romeo|Martinez|N|7|4|1935|CAMEROON||Romeo.Martinez@803d.edu|2452372| +5386|AAAAAAAAKAFBAAAA|943740|5587|32987|2451169|2451139|Dr.|Caleb|Pannell|Y|24|8|1963|MONTSERRAT||Caleb.Pannell@a.com|2452561| +5387|AAAAAAAALAFBAAAA|741363|3990|21770|2449148|2449118|Mrs.|Ernestine|Yates|Y|25|7|1978|HAITI||Ernestine.Yates@8u.org|2452364| +5388|AAAAAAAAMAFBAAAA|594199|753|2554|2449874|2449844|Ms.|Anthony|Chacon|N|2|12|1935|GEORGIA||Anthony.Chacon@k8.edu|2452407| +5389|AAAAAAAANAFBAAAA|373666|6566|29466|2450290|2450260|Mr.|Carlos|Richard|N|20|8|1926|FRENCH GUIANA||Carlos.Richard@VE9ukc3Grz94Y.com|2452520| +5390|AAAAAAAAOAFBAAAA|183097|3827|27004|2450975|2450945|Mr.|Joshua|Gardner|N|13|6|1963|NEW ZEALAND||Joshua.Gardner@Pd0rPcoy3zb0RtOyzG.edu|2452468| +5391|AAAAAAAAPAFBAAAA|1858870|6768|16266|2449069|2449039|Dr.|Richard|Carr|N|18|12|1926|RUSSIAN FEDERATION||Richard.Carr@zbGsMlM071O2g7.edu|2452637| +5392|AAAAAAAAABFBAAAA|492757|2273|16589|2450482|2450452|Mr.|Fernando|Clayton|Y|17|8|1952|NORFOLK ISLAND||Fernando.Clayton@Z19Raje3DEy.edu|2452423| +5393|AAAAAAAABBFBAAAA|1441945|6309|46320|2449535|2449505|Dr.|Jimmy|Fleming|Y|8|5|1951|ITALY||Jimmy.Fleming@eaaJL.edu|2452404| +5394|AAAAAAAACBFBAAAA|50941|7053|40365|2451804|2451774|Ms.|Alison|Hughes|N|6|1|1951|MALTA||Alison.Hughes@mYdtEMzuXj4J.org|2452432| +5395|AAAAAAAADBFBAAAA|1571027|6400|33726|2452152|2452122|Dr.|Shelley|Ryan|Y|24|12|1930|MACAO||Shelley.Ryan@KXpFbDfmb7h7dlS.edu|2452633| +5396|AAAAAAAAEBFBAAAA|1148326|5600|40921|2451070|2451040|Miss|Mandi|Wilkins|Y|19|6|1929|VENEZUELA||Mandi.Wilkins@ONVxyS1uOcHV6qsUv.com|2452532| +5397|AAAAAAAAFBFBAAAA|838548|5835|25922|2451894|2451864|Ms.|Nancy|Martin|Y|29|7|1925|GUAM||Nancy.Martin@hKddfNtxaZr.edu|2452464| +5398|AAAAAAAAGBFBAAAA|1305937|2343|6349|2452039|2452009|Mr.|John|Henderson|Y|3|2|1930|ICELAND||John.Henderson@e2mf.edu|2452445| +5399|AAAAAAAAHBFBAAAA|891400|3304|27961|2450700|2450670|Miss|Regina|Jordan|N|4|2|1981|POLAND||Regina.Jordan@ehfvzU.org|2452398| +5400|AAAAAAAAIBFBAAAA|670172|5284|47596|2451300|2451270|Mr.|Benny|Flores|N|15|3|1969|LIBERIA||Benny.Flores@Y0USuyFipyP.org|2452526| +5401|AAAAAAAAJBFBAAAA|1281672|3460|8895|2450267|2450237|Dr.|Ruth|Bourne|Y|27|7|1967|JAMAICA||Ruth.Bourne@Vbuge6abZPYkSgSjt.edu|2452613| +5402|AAAAAAAAKBFBAAAA|1645996|2205|7214|2449389|2449359|Dr.|Kori|Moore|Y|12|12|1931|SAMOA||Kori.Moore@g1iBhKbGCJE.org|2452562| +5403|AAAAAAAALBFBAAAA|779516|1391|34066|2451024|2450994|Ms.|Christine|Stevens|N|2|12|1947|ERITREA||Christine.Stevens@m.com|2452450| +5404|AAAAAAAAMBFBAAAA|503016|2068|12367|2451747|2451717|Mrs.|Cristina|Byers|N|19|7|1944|SAN MARINO||Cristina.Byers@QTJ4BjxoG7oNTAEYt.edu|2452528| +5405|AAAAAAAANBFBAAAA|599409|3319|20876|2452505|2452475|Ms.|Rhonda|Talbott|Y|15|7|1931|BRUNEI DARUSSALAM||Rhonda.Talbott@4yhat.org|2452333| +5406|AAAAAAAAOBFBAAAA|1892242|4599|46909|2449430||Mrs.|||Y|||1956|||Tina.Hitchcock@x1N2.edu|2452476| +5407|AAAAAAAAPBFBAAAA|574062|1718|28221|2450186|2450156|Dr.|Marie|Richardson|N|17|12|1938|BOUVET ISLAND||Marie.Richardson@vBU0ITZKNk3To5ec.org|2452545| +5408|AAAAAAAAACFBAAAA|1395415|4908|9601|2452385|2452355|Mrs.|Samantha|Seymour|N|10|3|1926|TRINIDAD AND TOBAGO||Samantha.Seymour@ttIKR.org|2452337| +5409|AAAAAAAABCFBAAAA|976849|2579|24309|2449906|2449876|Sir|James|Mcelroy|N|28|11|1963|ARGENTINA||James.Mcelroy@IqZjHMX.edu|2452623| +5410|AAAAAAAACCFBAAAA|1892626|5224|9022|2450349|2450319|Mr.|Micheal|Lowe|Y|26|4|1937|SRI LANKA||Micheal.Lowe@hNMIsg1PA38hmfVdr.org|2452514| +5411|AAAAAAAADCFBAAAA|724067|817|36367|2452277|||Christopher|Hastings|Y||||||Christopher.Hastings@T.com|| +5412|AAAAAAAAECFBAAAA||4846|7931||2451063|||Shaffer||10|12|||||2452406| +5413|AAAAAAAAFCFBAAAA|1294548|1591|30234|2450517|2450487|Sir|Danny|Rodriquez|Y|25|11|1967|TIMOR-LESTE||Danny.Rodriquez@bt5yam1MY.edu|2452538| +5414|AAAAAAAAGCFBAAAA|603923|4977|14784|2452419|2452389|Mr.|Donald|Dover|Y|5|4|1953|SLOVENIA||Donald.Dover@nH6xA44RTi7.edu|2452613| +5415|AAAAAAAAHCFBAAAA|587670|3842|37661|2452478|2452448|Dr.|Linda|Butler|Y|19|3|1977|ESTONIA||Linda.Butler@Gf1BXT5ayDfrKqoax.com|2452463| +5416|AAAAAAAAICFBAAAA|1247034|4279|31571|2449921|2449891|Dr.|Robert|Perry|Y|8|2|1975|NIGER||Robert.Perry@1.org|2452643| +5417|AAAAAAAAJCFBAAAA|172504|6886|7715|2451187|2451157|Dr.|Nancy|Rios|Y|28|9|1987|NICARAGUA||Nancy.Rios@CDSl9LF9.com|2452639| +5418|AAAAAAAAKCFBAAAA|1189727|3831|16735|2450764|||||N|10|8|1989|HONDURAS||Louis.West@6E5N.com|2452413| +5419|AAAAAAAALCFBAAAA|113719|4461|14403|2449540|2449510|Miss|Cecile|Love|N|23|6|1962|VIRGIN ISLANDS, U.S.||Cecile.Love@clVD5vbAoT.edu|2452438| +5420|AAAAAAAAMCFBAAAA|365664|5876|40088|2451530|2451500|Mr.|James|Rainey|N|14|5|1947|TAJIKISTAN||James.Rainey@yO7Xvh6diGLVLZ00.org|2452449| +5421|AAAAAAAANCFBAAAA|1701666|1134|46955|2450797|2450767|Ms.|Dorothy|Anderson|Y|18|12|1988|JORDAN||Dorothy.Anderson@exIrb9t3.com|2452379| +5422|AAAAAAAAOCFBAAAA|1519567|80|7393|2450400|2450370|Ms.|Janice|Robison|N|25|1|1945|GRENADA||Janice.Robison@B.org|2452514| +5423|AAAAAAAAPCFBAAAA|1866657|4594|47378|2449290|2449260|Sir|Thomas|Harrison|Y|15|3|1952|LITHUANIA||Thomas.Harrison@J8x.com|2452455| +5424|AAAAAAAAADFBAAAA|1810322|1939|46356|2449226|2449196|Dr.|Mark|Williams|Y|10|12|1925|FAROE ISLANDS||Mark.Williams@A.org|2452455| +5425|AAAAAAAABDFBAAAA|1595597|169|38788|2449174|2449144|Sir|Darrell|Clark|Y|10|10|1974|ANTARCTICA||Darrell.Clark@eYMFym8cf.com|2452306| +5426|AAAAAAAACDFBAAAA|515099|3394|17689|2449495|2449465|Mrs.|Diane|Berry|N|9|7|1986|ESTONIA||Diane.Berry@QkgY1jylnb.com|2452341| +5427|AAAAAAAADDFBAAAA|748363|5450|45737|2451021|2450991|Sir|Glenn|Mallory|Y|6|1|1932|VIRGIN ISLANDS, U.S.||Glenn.Mallory@JDaAUVTCmfrMr.org|2452488| +5428|AAAAAAAAEDFBAAAA|1168678|2665|29805|2452146|2452116|Dr.|Betty|Smallwood|Y|14|2|1959|PHILIPPINES||Betty.Smallwood@1IynRi6cBsqXfaNH2P.org|2452294| +5429|AAAAAAAAFDFBAAAA||2685|35326|2451849|||||N|7||1987|MALI||Pedro.Robledo@x0hLbJ4xIgm0eP.com|2452544| +5430|AAAAAAAAGDFBAAAA|219708|1614|27520|2450370|2450340|Dr.|Linwood|Meyers|N|18|6|1941|NAMIBIA||Linwood.Meyers@VjmxATXPvE.edu|2452445| +5431|AAAAAAAAHDFBAAAA|1754895|7083|8918|2452413|2452383|Mr.|David|Davis|N|14|3|1986|UNITED STATES||David.Davis@kXtA9Fe0.edu|2452535| +5432|AAAAAAAAIDFBAAAA|1393874|1488|17502|2449182|2449152|Ms.|Annie|Chambers|N|21|9|1983|CROATIA||Annie.Chambers@EuKuVvI2T.edu|2452647| +5433|AAAAAAAAJDFBAAAA|1155560|3010|18413|2451765|2451735|Dr.|Millard|Hancock|N|6|6|1969|BERMUDA||Millard.Hancock@TMxyjINadf7.com|2452604| +5434|AAAAAAAAKDFBAAAA|||17392||2452168|Mr.|||Y|30|9|1942||||| +5435|AAAAAAAALDFBAAAA|400700|2814|16928|2450254|2450224|Mr.|Simon|Jernigan|Y|4|4|1979|ZIMBABWE||Simon.Jernigan@Equa8JVmQK.edu|2452345| +5436|AAAAAAAAMDFBAAAA|97066|6572|478|2450664|2450634|Dr.|Betty|Shoemaker|Y|6|9|1951|VANUATU||Betty.Shoemaker@Q.com|2452427| +5437|AAAAAAAANDFBAAAA|1812038|2683|36081|2452155|2452125|Mrs.|Adam|Bolton|N|26|8|1988|PHILIPPINES||Adam.Bolton@EpM3998In5CZ7t3nfG.com|2452311| +5438|AAAAAAAAODFBAAAA|364827|667|23307|2451583|2451553|Ms.|Deann|Brown|N|13|5|1987|PORTUGAL||Deann.Brown@3pl2NFvgr.com|2452636| +5439|AAAAAAAAPDFBAAAA|219919|272|39025|2450916|2450886|Dr.|Terrance|Banks|Y|27|4|1975|SLOVENIA||Terrance.Banks@937P2ZH0g.edu|2452610| +5440|AAAAAAAAAEFBAAAA|57153|6456|49359|2452477|2452447|Miss|Luciana|Culver|Y|24|2|1946|MALAWI||Luciana.Culver@jKS.edu|2452489| +5441|AAAAAAAABEFBAAAA|844654|5030|25053|2450642|2450612|Mr.|Ricardo|Ward|Y|30|3|1985|CZECH REPUBLIC||Ricardo.Ward@8qaPgYmjmYOjE5e.edu|2452317| +5442|AAAAAAAACEFBAAAA|734731|1907|46236|2450980|2450950|Dr.|Charles|Hayes|N|6|5|1968|MARTINIQUE||Charles.Hayes@TGTdLMkEiMVe9.edu|2452442| +5443|AAAAAAAADEFBAAAA|372208|5691|20774|2452393|2452363|Miss|Adeline|Johnson|N|11|11|1933|SOUTH AFRICA||Adeline.Johnson@3vAqatnML.org|2452505| +5444|AAAAAAAAEEFBAAAA|742081|3842|18403|2449392|2449362|Dr.|Thomasine|Anders|N|1|8|1991|SOMALIA||Thomasine.Anders@mtpeJPmIncY.org|2452354| +5445|AAAAAAAAFEFBAAAA|1521351|3661|12771|2452292|2452262|Miss|Xiomara|Arnold|N|6|4|1971|YEMEN||Xiomara.Arnold@8P0ieQe3x4.edu|2452447| +5446|AAAAAAAAGEFBAAAA|101265|6116|5014|2452388|2452358|Sir|Clark|Stewart|Y|16|4|1970|ISLE OF MAN||Clark.Stewart@A.edu|2452480| +5447|AAAAAAAAHEFBAAAA|1788780|7057|46487|2452065|2452035|Mr.|James|Finley|Y|27|11|1981|GREENLAND||James.Finley@Sl8iJvXLu2aRIS.org|2452471| +5448|AAAAAAAAIEFBAAAA|121675|6551|28674|2450900|2450870|Dr.|James|Harrington|N|30|8|1989|WALLIS AND FUTUNA||James.Harrington@EM1bBG0r6AcgP.com|2452518| +5449|AAAAAAAAJEFBAAAA|232454|3947|21040|2449533|2449503|Dr.|Alma|Goodman|Y|3|11|1959|NAURU||Alma.Goodman@Y0DjrtZeH3oyvjo5r.com|2452480| +5450|AAAAAAAAKEFBAAAA|1499512|2664|33769|2451582|2451552|Dr.|Jared|Murphy|N|22|6|1951|ARGENTINA||Jared.Murphy@XJ.edu|2452408| +5451|AAAAAAAALEFBAAAA|1015062|2457|49099|2452398|2452368|Dr.|Jerome|Anderson|N|4|7|1976|TURKEY||Jerome.Anderson@PdKdQJsjSC8dO2hE.edu|2452613| +5452|AAAAAAAAMEFBAAAA|378657|1758|6859|2449585|2449555|Mrs.|Susan|Day|N|18|5|1929|BURKINA FASO||Susan.Day@QLX5mKrr9.edu|2452413| +5453|AAAAAAAANEFBAAAA|362923|3691|36859|2450067|2450037|Mr.|Ian|Shaw|Y|18|2|1932|PHILIPPINES||Ian.Shaw@blJax.org|2452387| +5454|AAAAAAAAOEFBAAAA|538112|6023|25534|2450240|2450210|Dr.|Thomas|Smith|N|15|1|1964|NIGERIA||Thomas.Smith@ythrJ3TyF.org|2452472| +5455|AAAAAAAAPEFBAAAA|1281074|1140|6|2450902|2450872|Mr.|Harvey|Clark|N|21|10|1975|KOREA, REPUBLIC OF||Harvey.Clark@bAdnGquApj4.edu|2452421| +5456|AAAAAAAAAFFBAAAA|||23269|2449574||Dr.|Ray|Lee|Y|17|||SLOVAKIA||Ray.Lee@rgESThx67ME.edu|| +5457|AAAAAAAABFFBAAAA|1512812|345|2678|2449467|2449437|Miss|Betty|Cone|N|7|6|1932|ISLE OF MAN||Betty.Cone@XTthDoXQvF.edu|2452646| +5458|AAAAAAAACFFBAAAA|340877|5512|6947|2452096|2452066|Dr.|Myriam|Sykes|N|21|8|1938|TUVALU||Myriam.Sykes@dE3gx41dqpFQ.edu|2452484| +5459|AAAAAAAADFFBAAAA|497297|4398|27546|2451320|2451290|Dr.|Donald|Wells|Y|7|10|1933|GERMANY||Donald.Wells@ZM49sHFGKglickCZjv.com|2452348| +5460|AAAAAAAAEFFBAAAA|390853|6444|6482|2451951|2451921|Sir|Zachary|Mixon|N|16|11|1943|URUGUAY||Zachary.Mixon@UjMnmb.edu|2452369| +5461|AAAAAAAAFFFBAAAA|521869|692|11962|2450693|2450663|Mrs.|Jennifer|Hall|N|20|3|1989|GHANA||Jennifer.Hall@xpG.edu|2452539| +5462|AAAAAAAAGFFBAAAA|516225|4280|4335|2451745|2451715|Sir|Franklin|Pollard|Y|4|12|1968|PARAGUAY||Franklin.Pollard@trZ1OXF8t.org|2452316| +5463|AAAAAAAAHFFBAAAA|802585|5160|24982|2452362|2452332|Dr.|Nicholas|Boyd|N|21|7|1942|MONGOLIA||Nicholas.Boyd@9U4XL44Cjp70ZIk.org|2452580| +5464|AAAAAAAAIFFBAAAA|1633017|1021|29872|2450514|2450484|Dr.|Raymond|Parker|Y|17|8|1963|URUGUAY||Raymond.Parker@COH1G3JDv6ovQyo.edu|2452295| +5465|AAAAAAAAJFFBAAAA|633564|2211|15611|2451291|2451261|Dr.|Joy|Ward|Y|20|10|1962|JORDAN||Joy.Ward@tqYMUGYN.edu|2452606| +5466|AAAAAAAAKFFBAAAA|899641|905|8616|2452592|2452562|Sir|Michael|Andrews|N|24|9|1979|CHILE||Michael.Andrews@k0pXOBzh.com|2452441| +5467|AAAAAAAALFFBAAAA|479365|5499|19633|2451193|2451163|Dr.|Rodrick|Sweat|Y|11|3|1957|GUYANA||Rodrick.Sweat@cSgmI.edu|2452633| +5468|AAAAAAAAMFFBAAAA|500015|3946|36630|2452169|2452139|Sir|Jamie|Trujillo|N|5|1|1935|SLOVENIA||Jamie.Trujillo@405xp3TobJ.edu|2452358| +5469|AAAAAAAANFFBAAAA|581207|550|23846|2452677|2452647|Miss|Debi|Kelsey|N|15|4|1943|SAINT LUCIA||Debi.Kelsey@XM2BlrQ2d3dPluuR.org|2452640| +5470|AAAAAAAAOFFBAAAA|748811|1416|44421|2450522|2450492|Mrs.|Deidra|Lyle|Y|6|2|1924|BELARUS||Deidra.Lyle@iifrPJGbh.com|2452617| +5471|AAAAAAAAPFFBAAAA|707769|2931|29293|2449394|2449364|Dr.|Karen|Huang|Y|13|2|1987|IRELAND||Karen.Huang@CFeJCi.com|2452639| +5472|AAAAAAAAAGFBAAAA|65948|5851|2201|2451388|2451358|Dr.|Charles|Morgan|Y|12|11|1981|BERMUDA||Charles.Morgan@zt2rA1cvt2PR6k.edu|2452373| +5473|AAAAAAAABGFBAAAA|835763|4040|13870|2452130|2452100|Sir|David|Langston|Y|14|11|1933|MALDIVES||David.Langston@COgGvjcmGZd4.com|2452607| +5474|AAAAAAAACGFBAAAA|1099549|3313|38677|2449277|2449247|Sir|Joshua|Mcguire|Y|28|6|1960|MONACO||Joshua.Mcguire@LAF42F.com|2452390| +5475|AAAAAAAADGFBAAAA|1679506|4863|36573|2452436|2452406|Dr.|Richard|Rogers|Y|14|11|1989|SLOVENIA||Richard.Rogers@lZS38VnB.com|2452576| +5476|AAAAAAAAEGFBAAAA|583001|4609|31417|2449038|2449008|Mrs.|Julie|Sellers|N|25|3|1926|HONG KONG||Julie.Sellers@cK.org|2452561| +5477|AAAAAAAAFGFBAAAA|1725008|1824|19087|2449152|2449122|Miss|Marissa|Carmichael|N|15|9|1976|TURKMENISTAN||Marissa.Carmichael@bMtYGC.edu|2452463| +5478|AAAAAAAAGGFBAAAA|1309499|5908|21117|2452502|2452472|Mr.|Rodney|Holmes|N|7|9|1977|BURKINA FASO||Rodney.Holmes@TZBJiV9JD.edu|2452376| +5479|AAAAAAAAHGFBAAAA|966954|2838|25922|2449479|2449449|Sir|Jerome|Robertson|N|14|7|1990|MOROCCO||Jerome.Robertson@9Res5.edu|2452399| +5480|AAAAAAAAIGFBAAAA|981717|3212|30707|2449942|2449912|Dr.|Francine|Rey|Y|29|1|1978|GHANA||Francine.Rey@Xj9L3uZXQHTM0d2K.org|2452600| +5481|AAAAAAAAJGFBAAAA|1521806|6736|47713|2449408|2449378|Dr.|Lakesha|Burke|Y|17|2|1966|TRINIDAD AND TOBAGO||Lakesha.Burke@EVgG3rsuHRJPd.edu|2452600| +5482|AAAAAAAAKGFBAAAA|410200|5495|35990|2450163|2450133|Dr.|Diane|Kane|N|27|12|1930|MALAWI||Diane.Kane@vR9g6yQhCjFopT.com|2452476| +5483|AAAAAAAALGFBAAAA|381233||5974|2451784|||Denae|Bryant|Y|31|7|1962|BAHAMAS||Denae.Bryant@L0HrfIq5.edu|2452459| +5484|AAAAAAAAMGFBAAAA|650455|64|42026|2449433|2449403|Mr.|Ronald|Johnson|Y|14|6|1926|NETHERLANDS ANTILLES||Ronald.Johnson@umqIHtaYe3g.com|2452646| +5485|AAAAAAAANGFBAAAA|1617789|4147|12105|2451041|2451011|Mr.|Oliver|Jernigan|N|4|10|1977|R�UNION||Oliver.Jernigan@dTQf58iLQM.edu|2452358| +5486|AAAAAAAAOGFBAAAA|745075|4412|24949|2451246|2451216|Dr.|Ruben|Higgins|Y|5|3|1981|SAINT HELENA||Ruben.Higgins@OEUYdbNQJBB.edu|2452395| +5487|AAAAAAAAPGFBAAAA|1691186|3137|11888|2449906|2449876|Sir|Donald|Poe|N|8|8|1926|GABON||Donald.Poe@y.org|2452612| +5488|AAAAAAAAAHFBAAAA|645313|4890|29314|2452482|2452452|Ms.|Nina|Cobb|N|6|7|1945|MALAYSIA||Nina.Cobb@gBmv3UCsvkj.com|2452442| +5489|AAAAAAAABHFBAAAA|1566074|2418|34409|2450868|2450838|Sir|Terrance|Davis|N|9|1|1935|MALI||Terrance.Davis@0OxDB.org|2452472| +5490|AAAAAAAACHFBAAAA|332199|1371|8496|2451031|2451001|Dr.|Don|Shaw|Y|16|9|1964|ESTONIA||Don.Shaw@pzCyNO.com|2452342| +5491|AAAAAAAADHFBAAAA|827975|6347|14672|2450566|2450536|Ms.|Myrna|Mitchell|N|4|9|1946|SAINT HELENA||Myrna.Mitchell@K4Z1yGk1z2kruTS.com|2452555| +5492|AAAAAAAAEHFBAAAA|1362516|3447|11531|2449616|2449586|Dr.|Alexander|Waterman|N|4|6|1977|PORTUGAL||Alexander.Waterman@ExbabD5.org|2452552| +5493|AAAAAAAAFHFBAAAA|1670848|4692|49736|2450352|2450322|Mr.|Marcus|Becker|Y|12|12|1984|MALAYSIA||Marcus.Becker@RHo9.org|2452576| +5494|AAAAAAAAGHFBAAAA|214928|6905|30035|2450425|2450395|Dr.|Krystina|Duran|N|17|4|1978|LIBERIA||Krystina.Duran@xBgIbVqtvYI7Vi.com|2452510| +5495|AAAAAAAAHHFBAAAA|386644|4994|39344|2451926|2451896|Miss|Angel|Ramos|Y|18|7|1946|POLAND||Angel.Ramos@CFuO.com|2452547| +5496|AAAAAAAAIHFBAAAA|575944|3004|14417|2451969|2451939|Mr.|Joseph|Sisco|N|17|12|1964|HONG KONG||Joseph.Sisco@4pyJlyizL.edu|2452612| +5497|AAAAAAAAJHFBAAAA|1137674|506|44491|2449196|2449166|Mr.|Raphael|Ramirez|N|6|8|1968|MALTA||Raphael.Ramirez@A84FuBz29fSgnRLMU.org|2452473| +5498|AAAAAAAAKHFBAAAA|361141|3721|27101|2452112|2452082|Sir|Charles|Ragsdale|N|26|10|1976|COMOROS||Charles.Ragsdale@uheQFGxDa.org|2452620| +5499|AAAAAAAALHFBAAAA|45912|876|4687|2449374|2449344|Dr.|Lula|Long|N|24|8|1952|SOLOMON ISLANDS||Lula.Long@v.com|2452324| +5500|AAAAAAAAMHFBAAAA|1179822|5393|14917|2449861|2449831|Miss|Jennifer|Anderson|N|26|5|1945|TURKEY||Jennifer.Anderson@7.com|2452392| +5501|AAAAAAAANHFBAAAA|839207|2148|5136|2451198|2451168|Dr.|James|Ransom|Y|11|10|1970|JERSEY||James.Ransom@HGJpkVHiTaVTkma.com|2452420| +5502|AAAAAAAAOHFBAAAA|243766|3334|46102|2450070|2450040|Ms.|Kina|Joseph|N|9|4|1982|BRUNEI DARUSSALAM||Kina.Joseph@ABEYos8Z.org|2452586| +5503|AAAAAAAAPHFBAAAA|604651|1680|3509|2452269|2452239|Miss|Betty|Pearce|Y|30|4|1951|LESOTHO||Betty.Pearce@l6mntS2qEUXgzU.com|2452356| +5504|AAAAAAAAAIFBAAAA|1708913|1864|10415|2450194|2450164|Miss|Wilma|Lewis|Y|29|5|1983|MONGOLIA||Wilma.Lewis@UhzOtt9uCMhfNdN.org|2452587| +5505|AAAAAAAABIFBAAAA|1032392|4762|41132|2451407|2451377|Mr.|Frederick|Ross|Y|28|1|1961|HONDURAS||Frederick.Ross@xVnVuvrP.org|2452529| +5506|AAAAAAAACIFBAAAA|987846|2698|30703|2449431|2449401|Sir|William|Tyler|N|7|9|1943|SUDAN||William.Tyler@kYdojR6FJ85j1ok1B.edu|2452491| +5507|AAAAAAAADIFBAAAA|1663017|355|45693|2451988|2451958|Dr.|Nathaniel|Chiu|N|19|11|1972|ESTONIA||Nathaniel.Chiu@sK8zcVrcv.edu|2452377| +5508|AAAAAAAAEIFBAAAA|582175|4282|45989|2449632|2449602|Dr.|John|Murphy|Y|3|1|1930|COMOROS||John.Murphy@u.edu|2452423| +5509|AAAAAAAAFIFBAAAA|1382829|1042|11255|2449118|2449088|Ms.|Eleanor|Stewart|Y|8|1|1990|VIET NAM||Eleanor.Stewart@S.org|2452475| +5510|AAAAAAAAGIFBAAAA|1384142|64|41539|2452068|2452038|Ms.|Wanda|Nelson|N|2|2|1938|KOREA, REPUBLIC OF||Wanda.Nelson@Rnu9L7MTQ1Ct6P2.com|2452620| +5511|AAAAAAAAHIFBAAAA|1771903|2932|29161|2452016|2451986|Mrs.|Jill|Horton|Y|10|10|1977|ALGERIA||Jill.Horton@LPdQvBi.edu|2452522| +5512|AAAAAAAAIIFBAAAA|1231108|686|2664|2450861|2450831|Dr.|Rodger|Wallace|Y|23|2|1959|ISRAEL||Rodger.Wallace@LrUoyJ5E09A.edu|2452524| +5513|AAAAAAAAJIFBAAAA|1603167|3725|22236|2449384|2449354|Mr.|Paul|Jackson|Y|11|9|1938|BELARUS||Paul.Jackson@J5k6Gk4VC.com|2452646| +5514|AAAAAAAAKIFBAAAA|89912|368|29224|2451636|2451606|Dr.|Ellie|London|Y|27|12|1979|UZBEKISTAN||Ellie.London@uCnsYtHrRZRB.edu|2452420| +5515|AAAAAAAALIFBAAAA|543536|1602|44115|2452137|2452107|Dr.|Katherine|Ryan|Y|28|11|1941|WALLIS AND FUTUNA||Katherine.Ryan@gpvAgyoN2I5bL.org|2452623| +5516|AAAAAAAAMIFBAAAA|1729904|3523|49142|2452282|2452252|Mr.|Carl|Moore|N|14|12|1940|LITHUANIA||Carl.Moore@rfVNKQ.org|2452589| +5517|AAAAAAAANIFBAAAA|230982|2896|3210|2450898|2450868|Dr.|Daniel|Carr|Y|24|12|1946|GERMANY||Daniel.Carr@3.org|2452392| +5518|AAAAAAAAOIFBAAAA|139555|1648|13341|2451698|2451668|Ms.|Rebecca|Wilson|N|5|11|1965|MOZAMBIQUE||Rebecca.Wilson@Y.org|2452581| +5519|AAAAAAAAPIFBAAAA|1617549|1690|20631|2451508|2451478|Dr.|Kyle|Murphy|N|7|10|1962|DOMINICA||Kyle.Murphy@zGyjDM580rlQoqa.com|2452318| +5520|AAAAAAAAAJFBAAAA|1905243|5482|26921|2451675|2451645|Sir|Humberto|King|N|6|5|1946|C�TE D'IVOIRE||Humberto.King@Mgk1vPOMSfDIUh1ST.com|2452538| +5521|AAAAAAAABJFBAAAA|1135448|6963|31|2449742|2449712|Mrs.|Chloe|Mansfield|Y|5|8|1962|ARGENTINA||Chloe.Mansfield@ZqEV6Ka42X7R7.edu|2452458| +5522|AAAAAAAACJFBAAAA|275689|2216|24343|2451291|2451261|Mr.|Daniel|Burgess|N|10|4|1972|AZERBAIJAN||Daniel.Burgess@gBe.com|2452569| +5523|AAAAAAAADJFBAAAA|1889652|4773|29494|2450728|2450698|Ms.|Robyn|Ferguson|N|26|5|1931|SLOVENIA||Robyn.Ferguson@9e1D.edu|2452408| +5524|AAAAAAAAEJFBAAAA|1248877|3946|8168|2450524|2450494|Dr.|Samuel|Tracy|Y|14|2|1958|HONDURAS||Samuel.Tracy@bPEmmjMprT0.org|2452375| +5525|AAAAAAAAFJFBAAAA|1587718|1509|44100|2450253|2450223|Miss|Albert|Ward|N|28|9|1926|ICELAND||Albert.Ward@tikk3eGFAT.org|2452372| +5526|AAAAAAAAGJFBAAAA|1019141|559|12824|2451685|2451655|Ms.|Nancy|Gilson|Y|17|1|1990|QATAR||Nancy.Gilson@qi56o.com|2452335| +5527|AAAAAAAAHJFBAAAA|1503773|3115|7610|2449596|2449566|Miss|Kathleen|Carr|N|6|2|1933|BELARUS||Kathleen.Carr@3TjGSYI.edu|2452624| +5528|AAAAAAAAIJFBAAAA|812002|3663|47324|2451055|2451025|Dr.|James|Rubio|N|11|3|1950|HONDURAS||James.Rubio@0UKr18pMRs57.com|2452518| +5529|AAAAAAAAJJFBAAAA|679160|5162|29164|2451255|2451225|Mrs.|Betty|Lucero|Y|13|5|1973|BOTSWANA||Betty.Lucero@mL53BpfEHlf7.org|2452595| +5530|AAAAAAAAKJFBAAAA|1758422|3358|341|2449300|2449270|Mr.|Kenneth|Dunn|N|14|9|1937|TURKMENISTAN||Kenneth.Dunn@5FxrSSZxVoa59MEE.org|2452599| +5531|AAAAAAAALJFBAAAA|810571|6412|37053|2452173|2452143|Miss|Amy|Dalton|N|25|4|1969|GUERNSEY||Amy.Dalton@ImfMVLuZpM8.com|2452456| +5532|AAAAAAAAMJFBAAAA|1147572|3394|37616|2450441|2450411|Mrs.|Thelma|Hand|N|16|2|1924|WALLIS AND FUTUNA||Thelma.Hand@sLRuu.org|2452569| +5533|AAAAAAAANJFBAAAA|306191|185|41426|2450726|2450696|Ms.|Donna|Jordan|Y|24|4|1972|INDIA||Donna.Jordan@Ocj1Bcy3vboyZ.edu|2452430| +5534|AAAAAAAAOJFBAAAA|142360|2458|25003|2450370|2450340|Sir|Edward|Dodd|Y|30|4|1950|MONGOLIA||Edward.Dodd@MtkzjVC.edu|2452373| +5535|AAAAAAAAPJFBAAAA|204757|2655|48506|2450007|2449977|Mr.|Thomas|Reilly|N|16|11|1960|DJIBOUTI||Thomas.Reilly@ulStnKHxM6VSYZnH.com|2452619| +5536|AAAAAAAAAKFBAAAA|1676531|4583|16235|2451123|2451093|Dr.|Anna|Palmer|Y|11|7|1982|BULGARIA||Anna.Palmer@yifuS1.org|2452617| +5537|AAAAAAAABKFBAAAA|1409409|1576|35390|2449350|2449320|Dr.|Maurice|Weatherly|Y|8|1|1976|SWAZILAND||Maurice.Weatherly@vtmp9.org|2452590| +5538|AAAAAAAACKFBAAAA|698214|4348|20929|2449202|2449172|Miss|Katherine|Leach|N|7|1|1955|GUERNSEY||Katherine.Leach@oPdR9lHsxrpeg.org|2452403| +5539|AAAAAAAADKFBAAAA|532016|6030|26844|2451437|2451407|Sir|Ralph|Brooks|Y|25|4|1978|VIET NAM||Ralph.Brooks@lceyFRcCj1hqs9Sy.com|2452564| +5540|AAAAAAAAEKFBAAAA||1131|37432||2450968|Miss|Danielle||N|||||||| +5541|AAAAAAAAFKFBAAAA|1391578|968|27605|2449193|2449163|Miss|Sarah|Holland|Y|18|2|1950|TURKMENISTAN||Sarah.Holland@YDg.com|2452552| +5542|AAAAAAAAGKFBAAAA|1542741|6383|29225|2451392|2451362|Mr.|Eduardo|Phillips|N|8|2|1958|YEMEN||Eduardo.Phillips@pk.edu|2452462| +5543|AAAAAAAAHKFBAAAA|||47257|2449359||Sir|||N|30||1971|||Lester.Murphy@PCa9MDj8vxLg4M.edu|2452489| +5544|AAAAAAAAIKFBAAAA|1166988|2947|6153||2450813||Rowena||Y|14|6|||||| +5545|AAAAAAAAJKFBAAAA|913078|6685|38292|2451838|2451808|Sir|Mark|Cox|Y|9|12|1944|MADAGASCAR||Mark.Cox@HbvcqQbgGbi.edu|2452355| +5546|AAAAAAAAKKFBAAAA|1353855|5592|34929|2451910|2451880|Sir|William|Brennan|N|8|5|1964|GREECE||William.Brennan@DyyXBr62P5.edu|2452297| +5547|AAAAAAAALKFBAAAA|318935|6470|25958|2450804|2450774|Mr.|Martin|Stokes|Y|6|3|1957|MONTSERRAT||Martin.Stokes@RXjUxber.com|2452477| +5548|AAAAAAAAMKFBAAAA|1434793|501|17409|2449207|2449177|Mr.|Roger|Wilson|Y|1|8|1930|UNITED ARAB EMIRATES||Roger.Wilson@T4emxIAd3.edu|2452332| +5549|AAAAAAAANKFBAAAA|1343151|665|5826|2451762|2451732|Miss|Anthony|Smith|N|28|2|1953|MAURITANIA||Anthony.Smith@vCOk2ulIMFUk9.org|2452412| +5550|AAAAAAAAOKFBAAAA|322166|6052|21091|2450024|2449994|Dr.|Terry|Smith|Y|27|6|1961|FIJI||Terry.Smith@UgsXNbiIS8YZ5u.edu|2452521| +5551|AAAAAAAAPKFBAAAA|1168712|1730|21548|2450361|2450331|Dr.|Kenneth|Anderson|N|26|9|1962|GAMBIA||Kenneth.Anderson@RJCmnHRMPe.org|2452511| +5552|AAAAAAAAALFBAAAA|763521|3183|31645|2450743|2450713|Dr.|Leann|Medeiros|Y|27|3|1957|MACAO||Leann.Medeiros@zeoLtEGoXI.org|2452356| +5553|AAAAAAAABLFBAAAA|578552|5312|9319|2452048|2452018|Sir|James|Farris|Y|9|4|1954|ARUBA||James.Farris@lO9.edu|2452348| +5554|AAAAAAAACLFBAAAA|383213|1918|15470|2451234|2451204|Miss|Bessie|King|N|26|8|1988|UNITED KINGDOM||Bessie.King@vSXSvnNBv.com|2452387| +5555|AAAAAAAADLFBAAAA|970389|2514|16639|2450497|2450467|Ms.|Laura|Duff|Y|7|5|1985|LESOTHO||Laura.Duff@3jPA54K7s5rJ.com|2452642| +5556|AAAAAAAAELFBAAAA|107961||2913||2451278|||Johnson|N|12|||LIECHTENSTEIN||Amelia.Johnson@dq4DYGlF5L.com|| +5557|AAAAAAAAFLFBAAAA|53230|5834|25591|2452379|2452349|Dr.|John|Carrasco|Y|28|9|1979|CYPRUS||John.Carrasco@38PLkYiyePoPv6F2M.edu|2452307| +5558|AAAAAAAAGLFBAAAA|964467|6727|3387|2449067|2449037|Mrs.|Darlene|Hernandez|Y|14|3|1960|BERMUDA||Darlene.Hernandez@KIE9KaZaitSsQ.com|2452506| +5559|AAAAAAAAHLFBAAAA|912380|6094|17855|2450996|2450966|Mr.|Benjamin|Walton|Y|7|8|1958|MONTSERRAT||Benjamin.Walton@kMS.edu|2452520| +5560|AAAAAAAAILFBAAAA|1106180|623|37866|2449907|2449877|Miss|Donna|Ellison|N|19|4|1984|SAINT LUCIA||Donna.Ellison@AQ.edu|2452448| +5561|AAAAAAAAJLFBAAAA|59258|4264|41223|2450851|2450821|Mrs.|Anita|Barton|N|29|12|1935|ZAMBIA||Anita.Barton@nAYNcHtQQU9iqizjIS.edu|2452534| +5562|AAAAAAAAKLFBAAAA|1173627|5953|34279|2450567|2450537|Mr.|Michael|Allen|N|7|10|1983|AFGHANISTAN||Michael.Allen@4vL1.com|2452541| +5563|AAAAAAAALLFBAAAA|457121|2230|42939|2451917|2451887|Miss|Diane|White|N|5|6|1943|JAPAN||Diane.White@Fn9bHpVqcs567.com|2452525| +5564|AAAAAAAAMLFBAAAA|1454480|6440|42400|2450938|2450908|Mrs.|Jennifer|Mcarthur|N|27|4|1931|GABON||Jennifer.Mcarthur@r3qAzTqURzmpYNs.com|2452450| +5565|AAAAAAAANLFBAAAA|522700|377|17207|2450595||||||25|3|||||2452647| +5566|AAAAAAAAOLFBAAAA|1009084|3678|32272|2451924|2451894|Dr.|Jennifer|Weiss|Y|25|1|1949|LIBERIA||Jennifer.Weiss@zG14prpU4b.com|2452641| +5567|AAAAAAAAPLFBAAAA|||2715||2450225|Sir|||N|15|3|1940|VANUATU|||2452304| +5568|AAAAAAAAAMFBAAAA|132810|975|18740|2450905|2450875|Dr.|Carmela|Shelton|Y|6|12|1983|GRENADA||Carmela.Shelton@pZtj.edu|2452506| +5569|AAAAAAAABMFBAAAA|1583147|5722|34697|2452244|2452214|Sir|Leonel|Stevenson|Y|11|10|1981|CUBA||Leonel.Stevenson@GAezVYanLMFs9.edu|2452422| +5570|AAAAAAAACMFBAAAA|259122|6475|2628|2450372|2450342|Dr.|Joey|Cobbs|Y|13|12|1925|UNITED KINGDOM||Joey.Cobbs@R.com|2452647| +5571|AAAAAAAADMFBAAAA|1721346|2685|17577|2451149|2451119|Ms.|Linda|Parker|N|10|6|1989|CHINA||Linda.Parker@mtIeh7ak1ilK45.com|2452367| +5572|AAAAAAAAEMFBAAAA|1073827|5122|28817|2452478|2452448|Mr.|Richard|Reyes|N|25|6|1940|UGANDA||Richard.Reyes@nmU3SkeE6i.edu|2452635| +5573|AAAAAAAAFMFBAAAA|1285626|7044|46415|2449149|2449119|Miss|Susan|Rowland|Y|3|5|1929|LEBANON||Susan.Rowland@KKG.edu|2452408| +5574|AAAAAAAAGMFBAAAA|706370|3807|35997|2449407|2449377|Miss|Aaron|Jeffrey|N|24|2|1961|SOUTH AFRICA||Aaron.Jeffrey@XtupeIjG9U.edu|2452452| +5575|AAAAAAAAHMFBAAAA|1755456|6339|22869|2449466|2449436|Mr.|Luis|Lundy|N|22|12|1960|SAINT HELENA||Luis.Lundy@7.com|2452320| +5576|AAAAAAAAIMFBAAAA|1474893|4276|46506|2449330|2449300|Miss|Rose|Franklin|N|26|1|1952|MALTA||Rose.Franklin@4ye.com|2452570| +5577|AAAAAAAAJMFBAAAA|1813392|3129|18913|2452199|2452169|Mrs.|Margret|Johnson|N|27|4|1939|SAN MARINO||Margret.Johnson@Mjg2Aqqb.org|2452620| +5578|AAAAAAAAKMFBAAAA|1576578|4389|42600|2450432|2450402|Dr.|Faith|Mason|Y|22|10|1932|GUADELOUPE||Faith.Mason@i9UdGh5L6gB.edu|2452627| +5579|AAAAAAAALMFBAAAA|705866|3933|23600|2452148|2452118|Dr.|Errol|Overby|N|6|3|1962|CHINA||Errol.Overby@bt1sSg6Iv2c9eb0Qm.com|2452503| +5580|AAAAAAAAMMFBAAAA|583402|5133|35664|2450847|2450817|Mrs.|Heidi|Bliss|N|13|7|1982|JERSEY||Heidi.Bliss@7usCs.org|2452352| +5581|AAAAAAAANMFBAAAA|1134864|5768|39425|2450431|2450401|Mrs.|Ruby|Hoffman|Y|23|8|1980|UNITED STATES||Ruby.Hoffman@DhO1EfDU2JJA.org|2452491| +5582|AAAAAAAAOMFBAAAA|361317|1527|32937|2449960|2449930|Ms.|Kathy|Campbell|Y|13|12|1928|MYANMAR||Kathy.Campbell@q2SK43xz.org|2452319| +5583|AAAAAAAAPMFBAAAA|1352426|5817|44055|2452294|2452264|Sir|Gary|Lawrence|Y|15|12|1969|ETHIOPIA||Gary.Lawrence@JoOCaxgiPTNCu3mN.edu|2452370| +5584|AAAAAAAAANFBAAAA|279478|6621|36964|2451647|2451617|Dr.|Juan|Lowery|Y|10|6|1958|DENMARK||Juan.Lowery@tvnbthOMz.edu|2452552| +5585|AAAAAAAABNFBAAAA|1334634|318|29202|2449449|2449419|Mrs.|Margie|Davis|N|13|12|1991|NICARAGUA||Margie.Davis@bQ8NSYZqLX.org|2452335| +5586|AAAAAAAACNFBAAAA|1197936|90|9397|2452452|2452422|Sir|Winston|Gray|N|7|10|1939|PITCAIRN||Winston.Gray@GzFt9H983NNSEJzLBnM.com|2452385| +5587|AAAAAAAADNFBAAAA|1064764|1895|42579|2452666|2452636|Sir|Kevin|Davis|Y|5|11|1972|CAMBODIA||Kevin.Davis@Z.com|2452567| +5588|AAAAAAAAENFBAAAA|273336|1990|49891|2451001|2450971|Dr.|Craig|Epps|Y|20|6|1926|MAYOTTE||Craig.Epps@a1tOl5HPM.edu|2452491| +5589|AAAAAAAAFNFBAAAA|829393|2853|5466|2449319|2449289|Dr.|Annmarie|Black|Y|9|2|1980|SOUTH AFRICA||Annmarie.Black@UN5IIkrib2KynP.com|2452645| +5590|AAAAAAAAGNFBAAAA|668203|3688|28020|2450516|2450486|Sir|Robert|Mitchell|N|20|8|1981|NAMIBIA||Robert.Mitchell@OT0JBLOu7.org|2452297| +5591|AAAAAAAAHNFBAAAA|174926|2666|12453|2450247|2450217|Ms.|Susan|Ward|N|16|8|1983|BELIZE||Susan.Ward@niElfCuZotBdrKk.com|2452504| +5592|AAAAAAAAINFBAAAA|449446|6869|14739|2450962|2450932|Sir|Robert|Armstrong|N|24|8|1959|ARGENTINA||Robert.Armstrong@BXDrKe8ph6rkHZ0P.edu|2452440| +5593|AAAAAAAAJNFBAAAA|133543|6101|10659|2450698|2450668|Dr.|Steve|Mayhew|Y|24|11|1984|TONGA||Steve.Mayhew@nR0cd8EzUjrE.edu|2452436| +5594|AAAAAAAAKNFBAAAA|1612793|4517|28241|2450217|2450187|Dr.|Edward|Wright|N|28|12|1952|ICELAND||Edward.Wright@cCqDDaSyiqHLVofx.com|2452413| +5595|AAAAAAAALNFBAAAA|741997|4846|31295|2449641|2449611|Mrs.|Margo|Moore|N|27|4|1946|TRINIDAD AND TOBAGO||Margo.Moore@1Sht7imlgPiBB0.com|2452400| +5596|AAAAAAAAMNFBAAAA|1744083||13649|2451935|||George|Larkin|||4|||||| +5597|AAAAAAAANNFBAAAA|150542|5469|1149|2449130|2449100|Dr.|Ruth|Carr|Y|21|2|1933|GRENADA||Ruth.Carr@Z.com|2452627| +5598|AAAAAAAAONFBAAAA|1150306|6832|25405|2451000|2450970|Miss|Eleanor|Canfield|Y|8|7|1980|EQUATORIAL GUINEA||Eleanor.Canfield@dVOyBvBgDP.com|2452285| +5599|AAAAAAAAPNFBAAAA||6639|5224|2450018|||||||6|1988|SRI LANKA||Rosetta.Hale@sD.org|2452587| +5600|AAAAAAAAAOFBAAAA|244460|103|31699|2451963|2451933|Ms.|Sabrina|Ortega|Y|13|1|1954|SAUDI ARABIA||Sabrina.Ortega@dTcZJNg.com|2452296| +5601|AAAAAAAABOFBAAAA|1330548|5629|44263|2451102|2451072|Sir|Leonard|Cisneros|N|11|2|1952|BELGIUM||Leonard.Cisneros@U.org|2452644| +5602|AAAAAAAACOFBAAAA|1257207|3949|35891|2449152|2449122|Sir|Dennis|Brown|Y|12|12|1939|ESTONIA||Dennis.Brown@9j0KQUBvvag.com|2452317| +5603|AAAAAAAADOFBAAAA|||40057||||John||N|7|9||||John.Gulley@2p.com|2452449| +5604|AAAAAAAAEOFBAAAA|581682|395|36755|2452425|2452395|Dr.|Michele|Townsend|N|1|5|1944|SOLOMON ISLANDS||Michele.Townsend@pYty.com|2452569| +5605|AAAAAAAAFOFBAAAA|358995|1142|3242|2450438|2450408|Mr.|Jeremy|Grant|Y|14|12|1992|EGYPT||Jeremy.Grant@bhSETnDz3Qk.org|2452596| +5606|AAAAAAAAGOFBAAAA|831795|915|3988|2450571|2450541|Mr.|Charles|Powell|N|29|11|1935|CHILE||Charles.Powell@9TjKPizpE.edu|2452528| +5607|AAAAAAAAHOFBAAAA|662241|6860|4726|||Sir|||Y||1||ISLE OF MAN||John.Covey@hYFyBBfeybyh.edu|2452575| +5608|AAAAAAAAIOFBAAAA||6483|31451|2451024|2450994|Mrs.||Coleman|||8|1957||||2452365| +5609|AAAAAAAAJOFBAAAA|111230|5736|28242|2451547|2451517|Sir|Javier|Larson|Y|10|12|1984|ANTIGUA AND BARBUDA||Javier.Larson@9YVu2nOVKe5z78d.edu|2452555| +5610|AAAAAAAAKOFBAAAA|16009|4900|21791|2450149|2450119|Ms.|Sue|Adams|Y|22|2|1925|FRENCH POLYNESIA||Sue.Adams@KgYj6gR8JY8Z4C4O.edu|2452589| +5611|AAAAAAAALOFBAAAA|1115169|6591|37566|2450077|2450047|Mr.|Neal|Orozco|Y|24|12|1950|NORWAY||Neal.Orozco@q.org|2452523| +5612|AAAAAAAAMOFBAAAA|83033|7150|30020|2451541|2451511|Dr.|Steven|Lloyd|N|16|11|1958|LATVIA||Steven.Lloyd@uBMpUYHdLhe.org|2452330| +5613|AAAAAAAANOFBAAAA|1130350|1949|23101|2450141|2450111|Ms.|Barbara|Pendleton|Y|20|11|1937|ITALY||Barbara.Pendleton@SSV7DDHA0fy.com|2452620| +5614|AAAAAAAAOOFBAAAA|770957|4131|20058|2449028|2448998|Ms.|Kathleen|Williams|N|2|8|1924|MARSHALL ISLANDS||Kathleen.Williams@1k6E8cV0SCgo0Rzb.com|2452497| +5615|AAAAAAAAPOFBAAAA|322321|4133|6792|2450674|2450644|Dr.|Bryce|Kelley|N|13|6|1933|HONG KONG||Bryce.Kelley@Jl4Aqs37lN.com|2452548| +5616|AAAAAAAAAPFBAAAA|1526093|3590|22334|2450552|2450522|Dr.|Jennifer|Chaney|Y|16|4|1931|SLOVAKIA||Jennifer.Chaney@RbAyFmRf8q.com|2452603| +5617|AAAAAAAABPFBAAAA|729902|1500|3545|2451825|2451795|Dr.|Richard|Burchfield|Y|9|8|1935|GRENADA||Richard.Burchfield@VA9CHf0xgKCefSgJE.edu|2452343| +5618|AAAAAAAACPFBAAAA||6900|4489|2451053||Sir||||4|9|||||2452349| +5619|AAAAAAAADPFBAAAA|888893|1566|43839|2452426|2452396|Ms.|Sheila|Massey|N|19|8|1954|MALTA||Sheila.Massey@sshQqmyTHh21FuF7.edu|2452557| +5620|AAAAAAAAEPFBAAAA|950104|2024|15544|2451330|2451300|Sir|Paul|Carlton|N|14|2|1980|FRENCH GUIANA||Paul.Carlton@fhHqpZ.org|2452352| +5621|AAAAAAAAFPFBAAAA|657353|4978|18603|2452556|2452526|Mrs.|Bonnie|Cook|Y|19|7|1929|CAMBODIA||Bonnie.Cook@x3vGNIRFE7s.com|2452600| +5622|AAAAAAAAGPFBAAAA|1065641|1867|40093|2449682|2449652|Ms.|Janie|Rogers|Y|29|7|1957|SOMALIA||Janie.Rogers@IXqfY.org|2452315| +5623|AAAAAAAAHPFBAAAA|1550477|3825|44820|2452299|2452269|Dr.|Jeanne|Stout|N|3|3|1969|RWANDA||Jeanne.Stout@3oG66emdaxJb.com|2452300| +5624|AAAAAAAAIPFBAAAA|963550|4177|2757|2450361|2450331|Dr.|Rene|Smith|N|1|2|1980|MONACO||Rene.Smith@xI0DEs.edu|2452507| +5625|AAAAAAAAJPFBAAAA|306143|1070|7550|2451247|2451217|Mr.|John|Campbell|N|11|11|1957|GUYANA||John.Campbell@thDpxcQ9iUVt4j.com|2452297| +5626|AAAAAAAAKPFBAAAA|109732|3397|26215|2450013|2449983|Dr.|Ronald|Patel|N|19|11|1964|SOLOMON ISLANDS||Ronald.Patel@qLDT5k7axUz.com|2452629| +5627|AAAAAAAALPFBAAAA|17216|1425|32834|2449792|2449762|Sir|Warren|Gross|Y|21|5|1948|ALBANIA||Warren.Gross@4LNu9O1GxK2r4er.edu|2452338| +5628|AAAAAAAAMPFBAAAA|735838|1651|22311|2450827|2450797|Sir|Rudy|Jackson|Y|27|1|1948|DJIBOUTI||Rudy.Jackson@BqXx6TOOoQ.edu|2452589| +5629|AAAAAAAANPFBAAAA|1201440|4467|10247|2450818|2450788|Mr.|Phillip|Walker|N|26|10|1932|ARMENIA||Phillip.Walker@bE8CxpPxmaB.com|2452409| +5630|AAAAAAAAOPFBAAAA|1070032|1254|30476|2450287|2450257|Miss|Blanca|Coffey|Y|19|10|1978|BENIN||Blanca.Coffey@OEtqUeplnAB9.com|2452606| +5631|AAAAAAAAPPFBAAAA|298855|387|47919|2450004|2449974|Dr.|David|Adair|N|16|2|1945|GUADELOUPE||David.Adair@fkinBuXpQ4.org|2452580| +5632|AAAAAAAAAAGBAAAA|871010|1108|6|2452464|2452434|Miss|Viva|Turner|N|5|5|1982|NORWAY||Viva.Turner@Z.com|2452623| +5633|AAAAAAAABAGBAAAA|1236194|6718|2442|2450747|2450717|Ms.|Antonio|Bass|Y|15|2|1976|NEPAL||Antonio.Bass@fEg83dJ1rA.com|2452394| +5634|AAAAAAAACAGBAAAA||2811|4383|2450776|2450746|Ms.|Carolyn||N||1|1979|||Carolyn.Bergeron@yBnPgHpEd3QK61dFn.org|| +5635|AAAAAAAADAGBAAAA|598786|5373|48007|2450252|2450222|Ms.|Helen|Barker|N|10|5|1933|ZAMBIA||Helen.Barker@V8ZT4ncYBuGSK.com|2452378| +5636|AAAAAAAAEAGBAAAA|698577|7111|40845|2451392|2451362|Dr.|Dorothy|Coleman|N|25|3|1985|MEXICO||Dorothy.Coleman@4eEJ9SAr.org|2452371| +5637|AAAAAAAAFAGBAAAA|349515|92|39869|2451358|2451328|Mrs.|Chin|Ballard|Y|20|11|1990|ISRAEL||Chin.Ballard@3kSMKJdbyc6sid.edu|2452573| +5638|AAAAAAAAGAGBAAAA|415816|2018|21420|2449152|2449122|Dr.|Corey|Switzer|Y|1|2|1934|GREECE||Corey.Switzer@ETdSX09o.com|2452540| +5639|AAAAAAAAHAGBAAAA|76537|1757|6624|2449663|2449633|Ms.|Alvina|Johnson|N|5|12|1952|SOMALIA||Alvina.Johnson@ZuI38jKojM.com|2452312| +5640|AAAAAAAAIAGBAAAA|1715459|5051|47490|2450876|2450846|Dr.|Alice|Wells|Y|4|8|1957|FRENCH POLYNESIA||Alice.Wells@Aph.edu|2452444| +5641|AAAAAAAAJAGBAAAA|156633|649|37465|2451735|2451705|Dr.|Pearl|Boisvert|Y|15|5|1933|ANTARCTICA||Pearl.Boisvert@E6bOQOrg5KA.org|2452353| +5642|AAAAAAAAKAGBAAAA|1899926|6893|42260|2450070|2450040|Dr.|Jennie|Mcleod|Y|19|8|1986|PERU||Jennie.Mcleod@ConF0zZH7lAn6Mge.com|2452506| +5643|AAAAAAAALAGBAAAA|1373843|3623|4603|2452008|2451978|Miss|Gwendolyn|Garcia|Y|11|4|1958|MONTENEGRO||Gwendolyn.Garcia@tJjD03akgg.com|2452583| +5644|AAAAAAAAMAGBAAAA|1186662|5557|23252|2449220|2449190|Mrs.|Adam|Chatman|Y|18|7|1943|MONTSERRAT||Adam.Chatman@Lec.org|2452403| +5645|AAAAAAAANAGBAAAA|483353|2914|34428|2449359|2449329|Ms.|Marni|Lyon|Y|24|11|1990|PAPUA NEW GUINEA||Marni.Lyon@hg8XsJZzxXUZkTH.org|2452446| +5646|AAAAAAAAOAGBAAAA|84841|1879|19364|2451367|2451337|Mrs.|Nellie|Hawes|N|29|11|1985|GUINEA||Nellie.Hawes@jZgGtUaog4HC26Qtxj.org|2452493| +5647|AAAAAAAAPAGBAAAA|63064|193|48122|2452451|2452421|Miss|Rena|Moss|N|26|1|1955|ECUADOR||Rena.Moss@Z.edu|2452483| +5648|AAAAAAAAABGBAAAA|1518077|6331|17785|2449962|2449932|Mrs.|Marie|Bravo|Y|11|8|1975|LESOTHO||Marie.Bravo@d0L6F.org|2452299| +5649|AAAAAAAABBGBAAAA|1861272|1894|48916|2452079|2452049|Sir|Mark|Fortin|Y|12|6|1935|CYPRUS||Mark.Fortin@dobYzPV6ag.edu|2452506| +5650|AAAAAAAACBGBAAAA|648284|6365|8046|2450239|2450209|Dr.|Thomas|Robb|Y|6|8|1939|FINLAND||Thomas.Robb@XLl5VBv.org|2452518| +5651|AAAAAAAADBGBAAAA|321867|1393|3766|2451834|2451804|Miss|Phyllis|Kelley|Y|26|12|1988|LATVIA||Phyllis.Kelley@F61trcS.com|2452493| +5652|AAAAAAAAEBGBAAAA|951652|5111|24170|2452452|2452422|Ms.|Carolyn|Kirk|N|18|10|1942|BENIN||Carolyn.Kirk@pCZUaneqFGu1R.com|2452518| +5653|AAAAAAAAFBGBAAAA|1187114|6510|6259|2450035|2450005|Dr.|Chad|Anderson|Y|24|10|1988|GABON||Chad.Anderson@hVsl8vCQ8a9vyTuSKf7.edu|2452329| +5654|AAAAAAAAGBGBAAAA|978500|3724|13701|2449232|2449202|Dr.|Allen|Wilson|N|28|7|1934|NIUE||Allen.Wilson@xmd.org|2452440| +5655|AAAAAAAAHBGBAAAA|665922|3887|27099|2452618|2452588|Sir|Thomas|Stanton|N|18|6|1948|OMAN||Thomas.Stanton@K.com|2452372| +5656|AAAAAAAAIBGBAAAA|1475471|3893|6315|2450539|2450509|Miss|Sara|Hanson|Y|20|4|1926|SOMALIA||Sara.Hanson@q6xXOjF77lfX.edu|2452429| +5657|AAAAAAAAJBGBAAAA|682867|1334|41754|2449410|2449380|Dr.|Craig|Tate|Y|19|6|1929|COMOROS||Craig.Tate@3BQyz84.org|2452625| +5658|AAAAAAAAKBGBAAAA|1781639|3905|16618|2450928|2450898|Sir|Michael|Nowell|N|19|2|1953|CAPE VERDE||Michael.Nowell@EURmdVhpL.com|2452454| +5659|AAAAAAAALBGBAAAA||72|10498|2450933|2450903|Dr.|Israel|||18|11|||||2452383| +5660|AAAAAAAAMBGBAAAA|1426557|4366|3775|2449677|2449647|Ms.|Lola|Messenger|Y|20|6|1931|NAMIBIA||Lola.Messenger@LdQQz92cNVC.edu|2452621| +5661|AAAAAAAANBGBAAAA|872784|3810|39283|2450039|2450009|Mrs.|Kimberlee|Finnegan|Y|20|1|1953|NEW CALEDONIA||Kimberlee.Finnegan@JmbVKGYZpUt.com|2452616| +5662|AAAAAAAAOBGBAAAA|638723||26447|2449780|2449750|Miss|Dina|Raines|Y|19|1|||||2452554| +5663|AAAAAAAAPBGBAAAA|328064|1439|28367|2451368|2451338|Mrs.|Deborah|Hale|Y|28|2|1929|GREECE||Deborah.Hale@aFDGsd.com|2452462| +5664|AAAAAAAAACGBAAAA|1229530|4068|35053|2450799|2450769|Dr.|Hilda|Armenta|N|2|1|1992|MAURITIUS||Hilda.Armenta@pHBBikLrIklh1H.edu|2452407| +5665|AAAAAAAABCGBAAAA|1498083||13534|2450562|||Susan|Spaulding|||4|1958|||Susan.Spaulding@fxCOcNj0ZYiX.com|2452638| +5666|AAAAAAAACCGBAAAA|1538053|3821|48789|2449666|2449636|Ms.|Ana|Jackson|N|9|8|1960|ANTARCTICA||Ana.Jackson@N2AFln2TmQE.com|2452436| +5667|AAAAAAAADCGBAAAA|1699028|1952|20476|2451452|2451422|Dr.|Jonathan|Hein|N|28|9|1980|LATVIA||Jonathan.Hein@zV2JiFy4kZQh.edu|2452326| +5668|AAAAAAAAECGBAAAA|1200177|5930|26083|2450794|2450764|Dr.|Amy|Barrett|N|13|4|1981|GUAM||Amy.Barrett@j9Z2krSbg.com|2452360| +5669|AAAAAAAAFCGBAAAA|250573|5888|13556|2452163|2452133|Sir|Brian|Skidmore|N|11|3|1990|ERITREA||Brian.Skidmore@IIsxOzmR.edu|2452486| +5670|AAAAAAAAGCGBAAAA|1334307|3781|8102|2451508|2451478|Mr.|Peter|Benson|Y|31|10|1934|HONG KONG||Peter.Benson@AHAPaDOVVk8.com|2452367| +5671|AAAAAAAAHCGBAAAA|1057343|1513|8130|2449509|2449479|Dr.|Cecil|Self|N|26|2|1927|CZECH REPUBLIC||Cecil.Self@GqHz8r.com|2452601| +5672|AAAAAAAAICGBAAAA|1467081|6532|22383|2452187|2452157|Mr.|Gordon|Garland|Y|31|5|1953|PERU||Gordon.Garland@sJcOji8ZBZ.org|2452596| +5673|AAAAAAAAJCGBAAAA|1557747|5612|47030|2451259|2451229|Sir|Stephen|Goebel|N|2|5|1965|CZECH REPUBLIC||Stephen.Goebel@Mkf.org|2452369| +5674|AAAAAAAAKCGBAAAA|348563|296|18382|2452582|2452552|Ms.|June|Rhea|N|9|7|1963|CZECH REPUBLIC||June.Rhea@hMH93Y.org|2452442| +5675|AAAAAAAALCGBAAAA|1470127|2537|15347|2450848|2450818|Mr.|Clinton|Simons|Y|6|12|1976|QATAR||Clinton.Simons@y.com|2452308| +5676|AAAAAAAAMCGBAAAA|606908|6343|23731|2452518|2452488|Miss|Angela|Walker|Y|31|5|1937|BRAZIL||Angela.Walker@ShKqtYPCvtN4.com|2452370| +5677|AAAAAAAANCGBAAAA|1714140|6812|6684|2450250|2450220|Sir|Ronald|Wallace|N|14|4|1956|IRAQ||Ronald.Wallace@n8vC5Vq8k.com|2452633| +5678|AAAAAAAAOCGBAAAA|20431|2677|1723|2449995|2449965|Miss|Latrina|Lucero|N|6|7|1973|MALI||Latrina.Lucero@5Y.com|2452333| +5679|AAAAAAAAPCGBAAAA|884641|927|4013|2449531|2449501|Sir|Raymond|Charles|Y|1|7|1929|UNITED KINGDOM||Raymond.Charles@S22EDjSMvKZe.com|2452435| +5680|AAAAAAAAADGBAAAA|1299371|5575|3379|2449459|2449429|Miss|Janette|Smith|N|20|3|1945|GAMBIA||Janette.Smith@79f81ReOcYEVVB7xmp.org|2452579| +5681|AAAAAAAABDGBAAAA|1142385|566|30747|2450761|2450731|Ms.|Rosalie|Gilbert|N|30|6|1973|BAHAMAS||Rosalie.Gilbert@u2H5UatA.org|2452474| +5682|AAAAAAAACDGBAAAA|78131|4325|3719|2452085|2452055|Ms.|Lisa|Miller|N|25|4|1986|PARAGUAY||Lisa.Miller@UqZKKD.com|2452478| +5683|AAAAAAAADDGBAAAA|1838219|683|11147|2450591|2450561|Ms.|Lesia|Ayers|N|1|11|1971|UNITED ARAB EMIRATES||Lesia.Ayers@7T.org|2452455| +5684|AAAAAAAAEDGBAAAA|539773|5823|34673|2451536|2451506|Mr.|Armando|Williams|N|21|6|1935|VANUATU||Armando.Williams@1TbnP3a81KKTp.org|2452462| +5685|AAAAAAAAFDGBAAAA|1806531|7114|18544|2450944|2450914|Miss|Amy|Burns|N|14|5|1970|NEW CALEDONIA||Amy.Burns@tvv.org|2452335| +5686|AAAAAAAAGDGBAAAA|545642|1806|29123|2450246|2450216|Dr.|David|Dorsey|Y|17|4|1938|TRINIDAD AND TOBAGO||David.Dorsey@6MukfuNzyXb.com|2452358| +5687|AAAAAAAAHDGBAAAA|72330|1880|1030|2449239|2449209|Dr.|Marcella|Dowdell|Y|11|1|1951|OMAN||Marcella.Dowdell@zmMrKi7.edu|2452496| +5688|AAAAAAAAIDGBAAAA|||24354|2450925|||Oscar||Y|23|||||Oscar.Peacock@oSpr0UJn0l.com|2452510| +5689|AAAAAAAAJDGBAAAA|65861|1392|26523|2452319|2452289|Mrs.|Alexander|Wyatt|N|19|8|1958|CHINA||Alexander.Wyatt@MIcLu.com|2452411| +5690|AAAAAAAAKDGBAAAA|1128374|2726|40678|2451126|2451096|Ms.|Hazel|Carpenter|Y|20|8|1991|PAPUA NEW GUINEA||Hazel.Carpenter@s1r.edu|2452419| +5691|AAAAAAAALDGBAAAA|1444260|154|4801|2451954|2451924|Dr.|Glen|Glass|N|23|3|1967|BAHAMAS||Glen.Glass@ak6U8Gvbl0I7.com|2452524| +5692|AAAAAAAAMDGBAAAA|1785206|18|7677|2452195|2452165|Sir|Joseph|Gutierrez|Y|17|10|1951|LUXEMBOURG||Joseph.Gutierrez@JS5kLyMhHh8t4.com|2452542| +5693|AAAAAAAANDGBAAAA|1551182|1168|7554|2450398|2450368|Dr.|Jay|Ramirez|N|15|6|1928|CAYMAN ISLANDS||Jay.Ramirez@EtfpHyXp4D2t.com|2452623| +5694|AAAAAAAAODGBAAAA|239239|1426|29166|2450240|2450210|Sir|Harry|Curran|N|11|12|1990|MACAO||Harry.Curran@jATUGkvT6u6ztSh.edu|2452337| +5695|AAAAAAAAPDGBAAAA|1902957|3449|5188|2452316|2452286|Dr.|Sau|Wright|Y|23|2|1924|LITHUANIA||Sau.Wright@D.edu|2452423| +5696|AAAAAAAAAEGBAAAA|630352|1844|536|2449908|2449878|Mr.|Florentino|Gough|N|4|8|1973|ZIMBABWE||Florentino.Gough@jLNKaRUhCsgKYG.org|2452585| +5697|AAAAAAAABEGBAAAA|142395|5946|17497|2452080|2452050|Mr.|Dexter|Butts|N|8|9|1975|LEBANON||Dexter.Butts@ADpdk4FqP2iV.com|2452485| +5698|AAAAAAAACEGBAAAA|1675468|2474|7880|2449464|2449434|Ms.|Catherine|Vasquez|Y|14|2|1953|KOREA, REPUBLIC OF||Catherine.Vasquez@TY6vI.org|2452471| +5699|AAAAAAAADEGBAAAA|442275|141|28854|2450541|2450511|Ms.|Kristine|Carroll|N|1|10|1990|NIGER||Kristine.Carroll@cRzoPF0Y6dOZLEnr.org|2452309| +5700|AAAAAAAAEEGBAAAA||1781|18830|||Mr.|John|Mills||20||1980|||John.Mills@Ccx.com|| +5701|AAAAAAAAFEGBAAAA|513675|5316|18910|2452178|2452148|Mr.|Julio|Lewis|Y|2|4|1940|TRINIDAD AND TOBAGO||Julio.Lewis@hg9sfNFhREvkQvA.edu|2452543| +5702|AAAAAAAAGEGBAAAA|263400||48118|2450798|2450768|||||||||||| +5703|AAAAAAAAHEGBAAAA|407690|2004|13471|2452136|2452106|Mrs.|Fannie|White|Y|18|2|1966|JAPAN||Fannie.White@CTMoKCKz0gZCODezxOn.com|2452585| +5704|AAAAAAAAIEGBAAAA|479284|3159|32634|2450620|2450590|Ms.|Jeannie|Williams|N|20|12|1950|NEW CALEDONIA||Jeannie.Williams@Kid2zXduzXF0jAbnt.com|2452517| +5705|AAAAAAAAJEGBAAAA|141938|6079|37773|2450058|2450028|Miss|Martha|French|Y|13|4|1967|UZBEKISTAN||Martha.French@aKLbHaa2OFs4yzU.edu|2452403| +5706|AAAAAAAAKEGBAAAA||4703|4830|||||Hess||13||1927|PARAGUAY|||| +5707|AAAAAAAALEGBAAAA|11732|5635|49985|2449272|2449242|Mr.|Matthew|Lombardi|Y|10|2|1962|AFGHANISTAN||Matthew.Lombardi@7lx8g9cNpl.edu|2452327| +5708|AAAAAAAAMEGBAAAA|311866|5222|3378|2451470|2451440|Dr.|Jeannette|Martinez|Y|12|1|1938|GREECE||Jeannette.Martinez@ZUptMAC6B4.edu|2452583| +5709|AAAAAAAANEGBAAAA|1864168|4128|9335|2452307|2452277|Sir|Lawrence|Peterson|N|11|9|1971|ALAND ISLANDS||Lawrence.Peterson@TF30JEG.com|2452374| +5710|AAAAAAAAOEGBAAAA|1027542|5996|38450|2452055|2452025|Sir|Norman|Young|N|8|10|1934|HUNGARY||Norman.Young@5U.com|2452555| +5711|AAAAAAAAPEGBAAAA|1240009|3767|45925|2449405|2449375|Mrs.|Manuela|Rushing|N|22|8|1964|MONTENEGRO||Manuela.Rushing@pHEEGbAIEfEaFp5b2.org|2452586| +5712|AAAAAAAAAFGBAAAA|9580|2834|38488|2450896|2450866|Ms.|Mildred|Christopher|Y|13|1|1953|MYANMAR||Mildred.Christopher@Ljs54HupYt.org|2452449| +5713|AAAAAAAABFGBAAAA|718602|5132|18862|2451107|2451077|Mr.|Steve|Johnson|N|3|11|1959|SINGAPORE||Steve.Johnson@Ii0qdqixq02nAIZhk.org|2452385| +5714|AAAAAAAACFGBAAAA|1334275|6016|34604|2451354|2451324|Dr.|Christopher|Demers|N|2|4|1934|ARGENTINA||Christopher.Demers@qCPMT.edu|2452448| +5715|AAAAAAAADFGBAAAA|531985|7057|15282|2451879|2451849|Sir|Eddie|Florez|Y|15|5|1935|C�TE D'IVOIRE||Eddie.Florez@DohkT2kN.com|2452570| +5716|AAAAAAAAEFGBAAAA|1432123|608|1893|2451393|2451363|Dr.|Henry|Vega|N|2|4|1928|RWANDA||Henry.Vega@rg.edu|2452613| +5717|AAAAAAAAFFGBAAAA|390045|6750|39408|2451866|2451836|Mr.|Michael|Fischer|N|10|12|1952|GUERNSEY||Michael.Fischer@AXjbD.com|2452422| +5718|AAAAAAAAGFGBAAAA|1507036|6103|48817|2449708|2449678|Mr.|Alan|Schrader|Y|1|10|1984|TOKELAU||Alan.Schrader@zVrKlURDToSr.com|2452612| +5719|AAAAAAAAHFGBAAAA|1918877|914|9351|2450937|2450907|Sir|James|Rutledge|Y|14|1|1931|BAHAMAS||James.Rutledge@VPNRPD.edu|2452351| +5720|AAAAAAAAIFGBAAAA|1295291|285|3279|2451515|2451485|Miss|Rose|Adams|Y|27|6|1978|WALLIS AND FUTUNA||Rose.Adams@4QcBg.org|2452644| +5721|AAAAAAAAJFGBAAAA|1834393|5679|25718|2451809|2451779|Mr.|James|Kent|N|19|12|1989|CHINA||James.Kent@ojHE3YMHUu1.org|2452389| +5722|AAAAAAAAKFGBAAAA|1230042|4587|13179|2452103|2452073|Miss|Diana|Myers|Y|28|5|1978|PANAMA||Diana.Myers@JzHzBL5aClqbs8R.org|2452555| +5723|AAAAAAAALFGBAAAA|1185765|2879|31640|2452045|2452015|Sir|Wilson|Thompson|N|12|6|1978|KYRGYZSTAN||Wilson.Thompson@8foDDxSx7Q35.edu|2452569| +5724|AAAAAAAAMFGBAAAA|1665363|4697|28784|2452303|2452273|Dr.|Galen|Smith|Y|24|5|1985|PITCAIRN||Galen.Smith@OredhkmOPXRuDC.org|2452293| +5725|AAAAAAAANFGBAAAA|1396203|4813|41376|2450559|2450529|Dr.|Helen|Turner|Y|23|10|1932|IRELAND||Helen.Turner@aF7O7D5mu881k8rh7.edu|2452324| +5726|AAAAAAAAOFGBAAAA|1166577|6806|11082|2449111|2449081|Mrs.|Latasha|Chesser|Y|26|3|1934|ERITREA||Latasha.Chesser@prGH081C2h6rzYUo.edu|2452592| +5727|AAAAAAAAPFGBAAAA|1249372||46007||2452294|Ms.|Ethel||Y||5|1945|CHRISTMAS ISLAND||Ethel.Newby@XnI.com|| +5728|AAAAAAAAAGGBAAAA|320705|978|24051|2449937|2449907|Dr.|Nancy|Sargent|N|20|9|1984|TIMOR-LESTE||Nancy.Sargent@jJgYH0ceKMcUjx6.edu|2452313| +5729|AAAAAAAABGGBAAAA|1637175|2991|39654|2451126|2451096|Miss|Monika|White|Y|6|2|1988|KOREA, REPUBLIC OF||Monika.White@QrCmbjxlxUnH.com|2452535| +5730|AAAAAAAACGGBAAAA|1806975|5236|32650|2452597|2452567|Dr.|Sofia|Braxton|N|30|7|1928|HONG KONG||Sofia.Braxton@tV7rqxSmYz.com|2452595| +5731|AAAAAAAADGGBAAAA|1740200|6073|37946|2451977|2451947|Dr.|Johnny|Wood|Y|4|1|1983|AMERICAN SAMOA||Johnny.Wood@yL8geiOzqG3uGMbnh.org|2452615| +5732|AAAAAAAAEGGBAAAA|1128274|952|16430|2450175|2450145|Dr.|Clifford|Cantrell|N|14|7|1979|GHANA||Clifford.Cantrell@yXjb.com|2452470| +5733|AAAAAAAAFGGBAAAA|1149022|5279|45983|2449239|2449209|Mr.|Craig|Richmond|N|21|12|1975|GUADELOUPE||Craig.Richmond@VIEQ2u4lyvmX.com|2452544| +5734|AAAAAAAAGGGBAAAA|704175|6787|12199|2450204|2450174|Dr.|Timothy|Werner|Y|21|11|1932|TUVALU||Timothy.Werner@z9xTeEkhmM.edu|2452392| +5735|AAAAAAAAHGGBAAAA|1135689|6867|845|2452421|2452391|Sir|Joseph|Goodwin|Y|29|7|1986|SERBIA||Joseph.Goodwin@T615X6QEkDpCT9V.org|2452571| +5736|AAAAAAAAIGGBAAAA|1814770|3233|6496|2449439|2449409|Dr.|Jacob|Hoppe|Y|27|5|1953|BOLIVIA||Jacob.Hoppe@zB9vy.edu|2452589| +5737|AAAAAAAAJGGBAAAA|334252|2199|25746|2451987|2451957|Dr.|Leon|Slater|Y|27|5|1953|JERSEY||Leon.Slater@lFtRaNBNK.com|2452530| +5738|AAAAAAAAKGGBAAAA|1878493|2711|22529|2450986|2450956|Sir|Edward|Wyatt|Y|17|12|1950|MONACO||Edward.Wyatt@eyfc.com|2452576| +5739|AAAAAAAALGGBAAAA|92308|2328|10701|2452259|2452229|Miss|Nadine|Sapp|Y|20|1|1943|NEPAL||Nadine.Sapp@PYS.org|2452593| +5740|AAAAAAAAMGGBAAAA|1514118|317|42069|2449126|2449096|Sir|Stephen|Sears|Y|11|11|1986|MOZAMBIQUE||Stephen.Sears@ajIdvmTlsQy0n.edu|2452484| +5741|AAAAAAAANGGBAAAA|906816|1060|24457|2450259|2450229|Mr.|Jeffrey|Davis|Y|30|8|1935|SINGAPORE||Jeffrey.Davis@8fBteG6SLG.com|2452530| +5742|AAAAAAAAOGGBAAAA|404705|15|3372|2450832|2450802|Sir|Benito|Sparks|N|10|3|1924|INDONESIA||Benito.Sparks@834rAuZhHtBvaRjr.org|2452625| +5743|AAAAAAAAPGGBAAAA|462849|7065|30881|2450498|2450468|Mr.|Oswaldo|Friend|N|1|6|1968|FINLAND||Oswaldo.Friend@88DFNS4OTdHiH.org|2452509| +5744|AAAAAAAAAHGBAAAA|398849|4159|41869|2450095|2450065|Dr.|Alita|Simons|N|8|12|1966|UNITED KINGDOM||Alita.Simons@Qbsd9ROC7ee.org|2452586| +5745|AAAAAAAABHGBAAAA|515706|4950|8194|2449311|2449281|Dr.|Margie|Jackson|Y|29|3|1933|NAURU||Margie.Jackson@TGAcu6VLiOm.org|2452467| +5746|AAAAAAAACHGBAAAA|872463|4189|32315|2449593|2449563|Ms.|Marjorie|Rodriguez|N|1|10|1985|ANDORRA||Marjorie.Rodriguez@20b9m464XcdI.edu|2452455| +5747|AAAAAAAADHGBAAAA|950067|4012|44710|2451595|2451565|Miss|Stephine|Daily|Y|11|11|1924|C�TE D'IVOIRE||Stephine.Daily@kd9zpauvFPQqO0E.edu|2452618| +5748|AAAAAAAAEHGBAAAA|1589290|7030|20017|2449640|2449610|Ms.|Sherri|Phillips|N|20|10|1981|GUERNSEY||Sherri.Phillips@i3d0OE.edu|2452427| +5749|AAAAAAAAFHGBAAAA|87262|5344|49826|2449709|2449679|Dr.|Joshua|Hand|Y|17|12|1927|BRUNEI DARUSSALAM||Joshua.Hand@k7phUu80uE.edu|2452472| +5750|AAAAAAAAGHGBAAAA|1387700|4602|27457|2450316|2450286|Mr.|Kenneth|Bolduc|Y|12|7|1981|SOLOMON ISLANDS||Kenneth.Bolduc@F0YT65Y1hsM77.com|2452292| +5751|AAAAAAAAHHGBAAAA|521736|2606|676|2452648|2452618|Sir|Harry|Moore|Y|18|8|1953|FRENCH GUIANA||Harry.Moore@0k.com|2452413| +5752|AAAAAAAAIHGBAAAA|819934|5540|28346|2449149|2449119|Sir|Timmy|Rinehart|N|19|5|1950|NEPAL||Timmy.Rinehart@aEVOougboNQ04Qbv.edu|2452598| +5753|AAAAAAAAJHGBAAAA|1595430|4256|10682|2450898|2450868|Dr.|Pearlie|Welch|N|19|4|1932|BURUNDI||Pearlie.Welch@mOsqgBaOcPs.edu|2452322| +5754|AAAAAAAAKHGBAAAA|570307|1092|41477|2450047|2450017|Dr.|Felix|Olivas|N|2|10|1985|BELIZE||Felix.Olivas@0zE.edu|2452578| +5755|AAAAAAAALHGBAAAA|1079311|5313|39715|2449184|2449154|Mrs.|Debra|Medley|N|19|7|1953|BERMUDA||Debra.Medley@5XsXI8nKizrC.com|2452459| +5756|AAAAAAAAMHGBAAAA|325225|5723|1021|2452125|2452095|Dr.|Andrew|Hughes|Y|21|7|1953|BERMUDA||Andrew.Hughes@fMbxRhcozPqv.org|2452433| +5757|AAAAAAAANHGBAAAA|632856|6548|15440|2451455|2451425|Sir|Boyd|Hooper|Y|23|8|1986|UNITED ARAB EMIRATES||Boyd.Hooper@697RVJHqeHJ0.edu|2452418| +5758|AAAAAAAAOHGBAAAA|998310|4325|15893|2451142|2451112|Mrs.|Evelyn|Voss|N|13|8|1951|MOROCCO||Evelyn.Voss@1KueftBi7lFHP75.com|2452342| +5759|AAAAAAAAPHGBAAAA|562662|897|21532|2452204|2452174|Dr.|Christine|Grayson|N|29|11|1986|TOKELAU||Christine.Grayson@eGxX8XDtc2Togv6.com|2452574| +5760|AAAAAAAAAIGBAAAA|1602890|6492|38494|2451239|2451209|Dr.|Nancy|Smith|Y|17|8|1925|HAITI||Nancy.Smith@HkPoq3or6R2.edu|2452522| +5761|AAAAAAAABIGBAAAA|1254919|3463|44807|2451356||Dr.||Timm||30|||EQUATORIAL GUINEA|||2452446| +5762|AAAAAAAACIGBAAAA|158886|2633|21627|2451169|2451139|Mr.|Allan|Hudson|Y|2|1|1934|FAROE ISLANDS||Allan.Hudson@Y0RxbF.org|2452479| +5763|AAAAAAAADIGBAAAA|33458|1013|23815|2450700|2450670|Sir|Emmett|Small|Y|30|3|1925|MAURITANIA||Emmett.Small@1znSK9LN.edu|2452400| +5764|AAAAAAAAEIGBAAAA|1613727|5646|15518|2449061|2449031|Miss|Ella|Boyles|Y|9|8|1963|C�TE D'IVOIRE||Ella.Boyles@69scTJGERqMxOF.com|2452555| +5765|AAAAAAAAFIGBAAAA|1064312|6771|48030|2451869|2451839|Sir|Michael|Rivera|N|3|5|1976|CANADA||Michael.Rivera@m9JOF7.com|2452476| +5766|AAAAAAAAGIGBAAAA|1354563|5058|26400|2452421|2452391|Mr.|John|Perry|Y|11|4|1956|MEXICO||John.Perry@N0xHiUcVA.edu|2452462| +5767|AAAAAAAAHIGBAAAA|1786004|525|39046|2450917|2450887|Mrs.|Amber|Burton|Y|1|1|1933|COMOROS||Amber.Burton@oTs9zl2xL.org|2452330| +5768|AAAAAAAAIIGBAAAA|351741|7122|36503|2452144|2452114|Dr.|Jared|Day|N|14|8|1981|SOLOMON ISLANDS||Jared.Day@aMiMAGcS.edu|2452598| +5769|AAAAAAAAJIGBAAAA|725959|5782|38805|2449770|2449740|Mrs.|Carmelina|Harrison|N|22|1|1987|ICELAND||Carmelina.Harrison@q324DeIC7iZJxHZUZ.edu|2452386| +5770|AAAAAAAAKIGBAAAA|990173|3129|28038|2451646|2451616|Ms.|Adele|Richards|N|29|11|1938|TIMOR-LESTE||Adele.Richards@5.org|2452571| +5771|AAAAAAAALIGBAAAA|1254281|748|39568|2451257|2451227|Dr.|Jill|Maxwell|N|14|5|1943|NORWAY||Jill.Maxwell@YB.edu|2452493| +5772|AAAAAAAAMIGBAAAA||6916|9961|||Mr.||Willingham|N|20||1925|MADAGASCAR|||| +5773|AAAAAAAANIGBAAAA|1491974|433|34939|2449782|2449752|Dr.|Herschel|Farley|Y|13|4|1938|UKRAINE||Herschel.Farley@nQcyPRifE.com|2452551| +5774|AAAAAAAAOIGBAAAA|1150875|6474|16073|2452288|2452258|Miss|Joanne|Chisolm|N|3|5|1963|CANADA||Joanne.Chisolm@yVABzgtKToxcco.com|2452607| +5775|AAAAAAAAPIGBAAAA|693563|4971|2332|2451030|2451000|Mr.|Kenneth|Hubbard|N|16|2|1992|C�TE D'IVOIRE||Kenneth.Hubbard@p.edu|2452307| +5776|AAAAAAAAAJGBAAAA|559373|3473|9620|2452214|2452184|Ms.|Martha|Gibson|N|16|7|1970|C�TE D'IVOIRE||Martha.Gibson@vjagfZ9Dp5.edu|2452421| +5777|AAAAAAAABJGBAAAA|303035|2908|18381|2450240|2450210|Dr.|Sara|Wilson|Y|27|4|1942|MONGOLIA||Sara.Wilson@UukAGq.com|2452361| +5778|AAAAAAAACJGBAAAA|630108|3042|24376|2449587|2449557|Dr.|Rudolph|Walsh|Y|30|11|1955|BELIZE||Rudolph.Walsh@2CJ2knjLC.com|2452309| +5779|AAAAAAAADJGBAAAA|51986|477|20968|2450863|2450833|Sir|Heath|Metzger|Y|1|4|1949|ARGENTINA||Heath.Metzger@mvCYhMF.edu|2452319| +5780|AAAAAAAAEJGBAAAA|585206|5508|22963|2449346|2449316|Miss|Crystal|Myers|Y|15|6|1941|EGYPT||Crystal.Myers@QDhsQr68Guh.edu|2452572| +5781|AAAAAAAAFJGBAAAA|54005|502|40999|2452426|2452396|Dr.|John|Montero|N|14|12|1957|LIECHTENSTEIN||John.Montero@IBFU6X6muG2r6pAV.org|2452292| +5782|AAAAAAAAGJGBAAAA|1914062|7024|30433|2449200|2449170|Dr.|Virginia|Davis|Y|25|11|1941|MOZAMBIQUE||Virginia.Davis@KAaBei.edu|2452483| +5783|AAAAAAAAHJGBAAAA|589313|779|37445|2451428|2451398|Sir|Samuel|Elias|N|12|7|1946|QATAR||Samuel.Elias@PCnICgb5Abg.edu|2452290| +5784|AAAAAAAAIJGBAAAA|509694|4596|47545|||Ms.||Porter||5||1978|||Carolyn.Porter@Y339QlukRkp.edu|2452629| +5785|AAAAAAAAJJGBAAAA|990361|7200|18839|2450509|2450479|Mr.|Steven|Lunsford|Y|4|12|1934|LUXEMBOURG||Steven.Lunsford@k2t9rC.com|2452533| +5786|AAAAAAAAKJGBAAAA|1697600||31646||||Matthew|Gonzalez|N||7|1989||||| +5787|AAAAAAAALJGBAAAA|790299|556|34460|2452533|2452503|Dr.|Esther|Coronado|N|14|11|1970|CROATIA||Esther.Coronado@rJk3UTk7dqrt.org|2452477| +5788|AAAAAAAAMJGBAAAA|716414|7012|45963|2450235|2450205|Miss|Madlyn|Smith|N|17|10|1942|GREECE||Madlyn.Smith@2jkNDptMQeu.edu|2452631| +5789|AAAAAAAANJGBAAAA|1864911|1241|19218|2450089|2450059|Mr.|Justin|Church|N|7|8|1971|ZIMBABWE||Justin.Church@FFYs14Jr6C1uUZ.edu|2452623| +5790|AAAAAAAAOJGBAAAA|736906|1783|30260|2451954|2451924|Sir|Robert|Smith|N|27|10|1934|CHRISTMAS ISLAND||Robert.Smith@j4ate667.edu|2452336| +5791|AAAAAAAAPJGBAAAA|1234499|2807|20715|2449614|2449584|Dr.|Gloria|Ramos|Y|5|11|1938|NIUE||Gloria.Ramos@3vplKvapcifuFm.org|2452611| +5792|AAAAAAAAAKGBAAAA|605224|3280|22889|2451591|2451561|Dr.|William|Timmons|N|7|4|1931|HUNGARY||William.Timmons@4XZM.org|2452407| +5793|AAAAAAAABKGBAAAA|1436578|1645|33112|2452215|2452185|Mrs.|Katherine|Hagen|Y|22|10|1928|NAURU||Katherine.Hagen@u7rcN.org|2452646| +5794|AAAAAAAACKGBAAAA|1179398|6019|16287|2450722|2450692|Mr.|John|Farley|Y|15|8|1950|SERBIA||John.Farley@i.com|2452634| +5795|AAAAAAAADKGBAAAA|1915668|4364|23013|2450272|2450242|Sir|Charles|Campos|Y|20|6|1974|SOLOMON ISLANDS||Charles.Campos@CLVPB0IRejD9E.org|2452538| +5796|AAAAAAAAEKGBAAAA|808625|2132|48839|2449543|2449513|Mr.|Patrick|Aguirre|N|30|5|1939|URUGUAY||Patrick.Aguirre@SGv0g45CeuDSZI.edu|2452345| +5797|AAAAAAAAFKGBAAAA|350630|2449|35421|2450254|2450224|Dr.|Tomasa|Arthur|N|24|8|1932|ANTARCTICA||Tomasa.Arthur@nHqSVu6zDaGg.org|2452367| +5798|AAAAAAAAGKGBAAAA|34490|252|14061|2452239|2452209|Mrs.|Virginia|Brown|N|27|2|1953|SOLOMON ISLANDS||Virginia.Brown@bQCzntUK6Vjr5e.com|2452524| +5799|AAAAAAAAHKGBAAAA|414471|3866|15261|2450949|2450919|Dr.|Jonathan|Stoner|Y|3|6|1965|GREENLAND||Jonathan.Stoner@v.edu|2452345| +5800|AAAAAAAAIKGBAAAA|605413|6792|35724|2450873|2450843|Sir|Donald|Vargas|N|15|2|1977|NICARAGUA||Donald.Vargas@MzuBj.edu|2452319| +5801|AAAAAAAAJKGBAAAA|1863490|2253|36263|2451909|2451879|Dr.|Stephen|Davidson|Y|24|4|1951|KUWAIT||Stephen.Davidson@STCUEsfUGxE.edu|2452328| +5802|AAAAAAAAKKGBAAAA|383088|4753|9618|2450978|2450948|Ms.|Susan|Taylor|Y|5|3|1970|AUSTRIA||Susan.Taylor@8zVGf.edu|2452493| +5803|AAAAAAAALKGBAAAA|1416488|5612|10736|2451651|2451621|Miss|Crystal|Diaz|N|22|11|1973|GUERNSEY||Crystal.Diaz@KlPE.edu|2452484| +5804|AAAAAAAAMKGBAAAA|1614678|3350|28022|2449266|2449236|Dr.|Sandra|Wolfe|Y|26|10|1954|MARSHALL ISLANDS||Sandra.Wolfe@1QM.edu|2452382| +5805|AAAAAAAANKGBAAAA|1627714|1228|19942|2452578|2452548|Ms.|Lillian|Roberts|Y|5|8|1971|EGYPT||Lillian.Roberts@DM7fthU.edu|2452544| +5806|AAAAAAAAOKGBAAAA|712619|2988|32110|2450848|2450818|Sir|Joseph|Barnes|Y|15|6|1960|BERMUDA||Joseph.Barnes@ZQkvtzIxqVpV40.com|2452442| +5807|AAAAAAAAPKGBAAAA|449547|5080|19813|2449664|2449634|Ms.|Marian|Buffington|N|15|2|1975|KIRIBATI||Marian.Buffington@S.org|2452513| +5808|AAAAAAAAALGBAAAA|1160244|410|43776|2452213|2452183|Mrs.|Linda|Summers|Y|20|7|1984|BERMUDA||Linda.Summers@toUEdFcUv4tLz.org|2452465| +5809|AAAAAAAABLGBAAAA|997172|579|786||2449659||Jimmy||Y|||1937|||Jimmy.Wilkins@4Orzk0.com|2452579| +5810|AAAAAAAACLGBAAAA|1119658|4823|45235|2450618|2450588|Miss|Alexandria|Gardner|Y|22|8|1974|PARAGUAY||Alexandria.Gardner@I7LkJt6fVrBslIE.org|2452459| +5811|AAAAAAAADLGBAAAA|||18710||2449868|Mr.|Raymond|Chamberlain|N|||1981|MARSHALL ISLANDS||Raymond.Chamberlain@XZk09uUORPT.edu|2452300| +5812|AAAAAAAAELGBAAAA|1051728|4921|42914|2450672|2450642|Ms.|Valerie|Kauffman|Y|3|1|1971|SOUTH AFRICA||Valerie.Kauffman@dEmMTCXRrVIkrib.org|2452643| +5813|AAAAAAAAFLGBAAAA|1179932|1626|33767|2450368|2450338|Dr.|Francisco|Brant|N|27|11|1972|MALAWI||Francisco.Brant@o.edu|2452465| +5814|AAAAAAAAGLGBAAAA|276430||1189||2452227|Dr.||Esposito|Y|||1945|WALLIS AND FUTUNA|||2452318| +5815|AAAAAAAAHLGBAAAA|1279646|872|2898|2451474|2451444|Ms.|Sherri|Toler|N|27|4|1964|LESOTHO||Sherri.Toler@VqvAck2YNMOGZ4sp0Z.com|2452544| +5816|AAAAAAAAILGBAAAA|1201168|1668|26921|2449627|2449597|Mr.|George|Spring|N|4|10|1966|ANDORRA||George.Spring@UorytIciZrSe.edu|2452514| +5817|AAAAAAAAJLGBAAAA|1257638|1473|7755||||||||||||Billy.Robbins@BOP.edu|2452406| +5818|AAAAAAAAKLGBAAAA|612751|4476|35401|2450757|2450727|Mr.|Manuel|Hardy|Y|12|3|1957|BENIN||Manuel.Hardy@uvvftAfLk9mCeee.org|2452615| +5819|AAAAAAAALLGBAAAA|313622|883|19744|2449661|2449631|Mrs.|Barbara|Hastings|N|18|1|1991|DOMINICA||Barbara.Hastings@d.edu|2452556| +5820|AAAAAAAAMLGBAAAA|1395990|1617|31197|2449592|2449562|Dr.|Christopher|Miller|Y|14|4|1964|UNITED STATES||Christopher.Miller@KeNa.edu|2452564| +5821|AAAAAAAANLGBAAAA|391565|2328|6691|2449409|2449379|Miss|Janet|Seaman|Y|28|12|1991|MAURITANIA||Janet.Seaman@yvC0o2p2FL.com|2452635| +5822|AAAAAAAAOLGBAAAA|170410|1549|43888|2452118|2452088|Ms.|Gretchen|Fleming|N|9|5|1947|CYPRUS||Gretchen.Fleming@AEefXLLSynt.com|2452459| +5823|AAAAAAAAPLGBAAAA|1796637|1943|48449|2449278|2449248|Ms.|Rosa|Loving|Y|20|10|1982|TOGO||Rosa.Loving@7FTt5GLfzKraxeZxy.org|2452496| +5824|AAAAAAAAAMGBAAAA|731587|6240|42493|2452360|2452330|Ms.|Joyce|Harris|N|13|1|1946|GUINEA||Joyce.Harris@gs.com|2452547| +5825|AAAAAAAABMGBAAAA|202276|4196|48516|2450442|2450412|Mr.|Michael|Kent|N|11|5|1982|MACAO||Michael.Kent@U8BKiDfqvf.com|2452418| +5826|AAAAAAAACMGBAAAA|1832303|6811|10472|2449306|2449276|Mr.|Thomas|Hunt|N|22|4|1936|SOUTH AFRICA||Thomas.Hunt@k4j9.edu|2452507| +5827|AAAAAAAADMGBAAAA|1273998|6909|3773|2449953|2449923|Dr.|Joshua|Cox|N|4|2|1942|GUATEMALA||Joshua.Cox@fVcpoQtpGts8dt.edu|2452564| +5828|AAAAAAAAEMGBAAAA|163420|1830|27793|2450349|2450319|Mr.|Wayne|Haynes|Y|20|10|1988|DENMARK||Wayne.Haynes@mdzcDx8bSBTvhmu.edu|2452317| +5829|AAAAAAAAFMGBAAAA|920446|296|18099|2451233|2451203|Mrs.|Petra|Peterson|Y|21|11|1955|MARSHALL ISLANDS||Petra.Peterson@jld4CGVMblc3R8BvyjJl.com|2452377| +5830|AAAAAAAAGMGBAAAA|1370198|5631|20264|2452675|2452645|Mrs.|Virginia|Elliott|N|10|4|1925|ANTIGUA AND BARBUDA||Virginia.Elliott@tzSUz50O.com|2452413| +5831|AAAAAAAAHMGBAAAA|1865020|4424|15828|2452637|2452607|Mr.|Luis|Leatherman|N|22|4|1928|SEYCHELLES||Luis.Leatherman@nvOBooUOCp1.com|2452531| +5832|AAAAAAAAIMGBAAAA|943372|3840|38708|2449688|2449658|Dr.|Demetrius|Stewart|N|2|4|1991|NETHERLANDS ANTILLES||Demetrius.Stewart@xkA7sfNYJ191Yos.com|2452406| +5833|AAAAAAAAJMGBAAAA|1229411|3368|16525|2450679|2450649|Mr.|Glenn|Watts|N|18|8|1934|FRENCH POLYNESIA||Glenn.Watts@9P4agIV4kNgHQbD4h.edu|2452526| +5834|AAAAAAAAKMGBAAAA|228615|2471|40727|2450495|2450465|Sir|Dwight|Black|N|12|12|1952|BOLIVIA||Dwight.Black@2PXmGuENjrGFNv.org|2452619| +5835|AAAAAAAALMGBAAAA|68012|1179|8346|2452455|2452425|Dr.|Rebecca|Ellis|N|12|2|1953|ALAND ISLANDS||Rebecca.Ellis@ayrBTM95Cp9.edu|2452609| +5836|AAAAAAAAMMGBAAAA|830187|3706|33625|2451388|2451358|Ms.|Sandra|Jones|N|26|9|1927|URUGUAY||Sandra.Jones@lt5urcqGMhgvQ.edu|2452647| +5837|AAAAAAAANMGBAAAA|798897|3404|11864|2449707|2449677|Sir|Jeffrey|Weiss|Y|13|7|1971|BELIZE||Jeffrey.Weiss@eobLmMOa.org|2452436| +5838|AAAAAAAAOMGBAAAA|1480881|3440|23952|2449048|2449018|Ms.|Ellen|Sanders|Y|11|2|1981|BANGLADESH||Ellen.Sanders@7NB.com|2452305| +5839|AAAAAAAAPMGBAAAA|60178|1176|16574|2450275|2450245|Dr.|Jason|Donaldson|N|8|8|1979|GIBRALTAR||Jason.Donaldson@L.org|2452367| +5840|AAAAAAAAANGBAAAA|780226|4413|24600|2450362|2450332|Mrs.|Irene|Williams|N|23|11|1957|GUINEA-BISSAU||Irene.Williams@sGP4NUxI5HQN8ghi1.org|2452411| +5841|AAAAAAAABNGBAAAA|428414|5466|3271|2451993|2451963|Miss|Samantha|Jones|N|3|6|1949|GUINEA||Samantha.Jones@gd7R0.com|2452309| +5842|AAAAAAAACNGBAAAA|1423635|4920|43758|2452416|2452386|Mr.|Michael|Haley|Y|10|5|1933|SWAZILAND||Michael.Haley@OfAOvhrm.com|2452359| +5843|AAAAAAAADNGBAAAA|1287403|328|31653|2451224|2451194|Mr.|Garry|Pruitt|N|30|11|1944|ANTARCTICA||Garry.Pruitt@g14FE6JzPK.com|2452612| +5844|AAAAAAAAENGBAAAA|456760|2600|22440|2449356|2449326|Dr.|Emily|Aaron|Y|8|4|1950|SAMOA||Emily.Aaron@BLBkB.edu|2452519| +5845|AAAAAAAAFNGBAAAA|1577622|1824|40073|2451287|2451257|Mrs.|Ida|Wheeler|Y|5|4|1987|SAINT HELENA||Ida.Wheeler@CIkG1Bk4spP.org|2452466| +5846|AAAAAAAAGNGBAAAA|1237899||40372|2449493|2449463|Ms.|||N|17|4||||Karen.Blake@Hm.com|2452601| +5847|AAAAAAAAHNGBAAAA|1108655|113|23524|2449336|2449306|Ms.|Andrew|Lloyd|N|21|7|1929|OMAN||Andrew.Lloyd@b9.edu|2452574| +5848|AAAAAAAAINGBAAAA|||2382|2449297||Miss|Sandra||N||4|1980|MAYOTTE|||2452442| +5849|AAAAAAAAJNGBAAAA|1256928|459|3237|2452176|2452146|Mrs.|Lori|Moody|Y|18|9|1929|THAILAND||Lori.Moody@bAX.com|2452442| +5850|AAAAAAAAKNGBAAAA|1101126|3144|17609|2451923|2451893|Miss|Kimberly|Murphy|N|18|5|1970|TUNISIA||Kimberly.Murphy@13FFumDlexaGmkAp.edu|2452620| +5851|AAAAAAAALNGBAAAA|88631|3931|42831|2449574|2449544|Ms.|Nidia|Trotter|N|9|11|1988|BURUNDI||Nidia.Trotter@VU32UeN23GplEJlLM.org|2452552| +5852|AAAAAAAAMNGBAAAA|688019|1546|46626|2452236|2452206|Mr.|Jeffrey|Bradley|N|16|5|1978|PHILIPPINES||Jeffrey.Bradley@nM.edu|2452624| +5853|AAAAAAAANNGBAAAA|1675044|5785|16173|2449564|2449534|Mrs.|Margaret|Gilmore|N|31|5|1986|C�TE D'IVOIRE||Margaret.Gilmore@BeNG3.edu|2452329| +5854|AAAAAAAAONGBAAAA|1541314|742|42923|2451187|2451157|Mrs.|Carissa|Moore|N|4|7|1939|CAPE VERDE||Carissa.Moore@TrAfifJ70.edu|2452461| +5855|AAAAAAAAPNGBAAAA|62037|4261|18485|2449469|2449439|Dr.|Sandra|Reyes|N|29|2|1984|CAYMAN ISLANDS||Sandra.Reyes@NkbChzJ8h5Jy7H7l.com|2452492| +5856|AAAAAAAAAOGBAAAA|809391|4630|1469|2452523|2452493|Mr.|George|Johnston|N|31|8|1988|SURINAME||George.Johnston@uxl3oglEmB1hA.org|2452403| +5857|AAAAAAAABOGBAAAA|1587667|6060|40806|2449712|2449682|Mr.|Jermaine|Rogers|N|6|8|1992|WALLIS AND FUTUNA||Jermaine.Rogers@TeX70rT2U.edu|2452570| +5858|AAAAAAAACOGBAAAA|773555|4854|22600|2449850|2449820|Sir|David|Williams|Y|22|4|1934|INDONESIA||David.Williams@vGbvVeaUe9z6cE.edu|2452411| +5859|AAAAAAAADOGBAAAA|356659|5372|14485|2450028|2449998|Dr.|Ben|Barraza|N|24|6|1969|BELGIUM||Ben.Barraza@92kvdSH.edu|2452610| +5860|AAAAAAAAEOGBAAAA|||28734|||Sir||Ramirez|N|30|||LIBERIA||Hubert.Ramirez@j3MxAebucHMH.com|| +5861|AAAAAAAAFOGBAAAA|731461|5053|17092|2451418|2451388|Dr.|Barbara|Epperson|Y|18|11|1940|ERITREA||Barbara.Epperson@lAyPZelylrALZLME8IQb.edu|2452344| +5862|AAAAAAAAGOGBAAAA|208031|5295|29806|2450307|2450277|Ms.|Irene|Jones|N|9|6|1984|GREECE||Irene.Jones@Z4Anu.org|2452630| +5863|AAAAAAAAHOGBAAAA|1774081|1048|39215|2451239|2451209||Craig||Y|||1925|HUNGARY|||2452436| +5864|AAAAAAAAIOGBAAAA|1060330|1309|23888|2450890|2450860|Dr.|Daniel|Patterson|N|4|4|1937|SAINT HELENA||Daniel.Patterson@FKXAeX.com|2452492| +5865|AAAAAAAAJOGBAAAA|1915735|3915|24848|2449649|2449619|Ms.|Ann|Klein|N|29|10|1934|SWEDEN||Ann.Klein@gHp0YlrZPH.com|2452367| +5866|AAAAAAAAKOGBAAAA|1490933|5362|25038|2450809|2450779|Mr.|Boyd|Mcleod|N|25|7|1948|UNITED ARAB EMIRATES||Boyd.Mcleod@VJYl5k.com|2452531| +5867|AAAAAAAALOGBAAAA|1850999|4324|48449|2450850|2450820|Dr.|Consuelo|Hernandez|Y|10|9|1930|BELIZE||Consuelo.Hernandez@9gNE.edu|2452522| +5868|AAAAAAAAMOGBAAAA|1000843|3863|21283|2449452|2449422|Miss|Vida|Baskin|N|4|10|1973|EGYPT||Vida.Baskin@nys0v84js5gN.com|2452439| +5869|AAAAAAAANOGBAAAA|110495|6554|21813|2450925|2450895|Miss|Maegan|Davis|N|10|9|1941|FRANCE||Maegan.Davis@NHGDHi0Q.edu|2452501| +5870|AAAAAAAAOOGBAAAA|1812484|1476|24468|2449432|2449402|Mr.|Reginald|Cochran|Y|20|1|1929|PERU||Reginald.Cochran@MfI7GeDcXY.edu|2452507| +5871|AAAAAAAAPOGBAAAA|1485165|524|49083|2451345|2451315|Mr.|John|Obrien|Y|10|10|1929|URUGUAY||John.Obrien@mYclfp3MY6tg.edu|2452495| +5872|AAAAAAAAAPGBAAAA|692496|5737|32178|2449416|2449386|Mr.|Robert|Macleod|Y|21|5|1968|IRAQ||Robert.Macleod@oAgdqv5Pr4.edu|2452438| +5873|AAAAAAAABPGBAAAA|487112|2987|3262|2451142|2451112|Sir|Robert|Coon|N|21|12|1960|ALAND ISLANDS||Robert.Coon@MJSaM9XT.org|2452547| +5874|AAAAAAAACPGBAAAA|238426|73|30920|2451158|2451128|Miss|Anne|Huang|N|29|3|1975|HONDURAS||Anne.Huang@DdyM04pykK.com|2452401| +5875|AAAAAAAADPGBAAAA|104649|1787|12925|2451465|2451435|Ms.|Chandra|Miller|N|12|4|1942|TIMOR-LESTE||Chandra.Miller@0UZhysUYhx3.com|2452417| +5876|AAAAAAAAEPGBAAAA|32935|3928|43721|2449036|2449006|Sir|Christoper|Ward|N|24|1|1961|ARUBA||Christoper.Ward@s.edu|2452404| +5877|AAAAAAAAFPGBAAAA|939991|5292|16546|2451841|2451811|Dr.|Dale|Simpson|N|7|6|1957|NORWAY||Dale.Simpson@mBqg.com|2452350| +5878|AAAAAAAAGPGBAAAA|836572|6398|35024|2449110|2449080|Miss|Tracie|Steele|N|24|11|1942|BHUTAN||Tracie.Steele@Tfo0ecGSaF.org|2452631| +5879|AAAAAAAAHPGBAAAA|427581|672|46610|2450166|2450136|Mrs.|Adriene|Betz|N|22|7|1973|EL SALVADOR||Adriene.Betz@5gTMTSBh5H3Z.org|2452383| +5880|AAAAAAAAIPGBAAAA|788804|3524|21844|2449470|2449440|Dr.|Karl|Looney|N|17|2|1977|SLOVENIA||Karl.Looney@GDeMFBQZ3P.org|2452399| +5881|AAAAAAAAJPGBAAAA|189902|658|30167|2452633|2452603|Sir|Edward|Smith|N|23|1|1932|CAMBODIA||Edward.Smith@qLgPMKgkGvRN.com|2452456| +5882|AAAAAAAAKPGBAAAA|1448844|139|33481|2449408|2449378|Ms.|Florence|Carter|N|8|5|1953|ARUBA||Florence.Carter@FbKShla3FFZZk.edu|2452294| +5883|AAAAAAAALPGBAAAA|1422321|5560|35152|2449464|2449434|Ms.|Sylvia|Pike|Y|7|8|1966|LESOTHO||Sylvia.Pike@tltgdLh39.org|2452523| +5884|AAAAAAAAMPGBAAAA|1217714|3143|42972|2450444|2450414|Ms.|Dawn|Gunter|N|6|10|1992|TOGO||Dawn.Gunter@2n1fiaSypj46B.com|2452566| +5885|AAAAAAAANPGBAAAA|14439|5513|24187|2451582|2451552|Mrs.|Melissa|Smith|N|10|12|1958|KAZAKHSTAN||Melissa.Smith@VTix0dYAnIfCLsp.edu|2452328| +5886|AAAAAAAAOPGBAAAA|1914682|5730|14778|2449050|2449020|Miss|Luz|Parks|Y|4|11|1930|ANGOLA||Luz.Parks@fHpjzPkKJuiG.com|2452568| +5887|AAAAAAAAPPGBAAAA|1088861|2783|3058|2451779|2451749|Sir|Fredric|Callaghan|N|26|7|1945|ITALY||Fredric.Callaghan@VLXvkTA1DQJ8q348.org|2452532| +5888|AAAAAAAAAAHBAAAA|46017|835|36853|2451872|2451842|Miss|Anna|Gray|N|21|10|1974|CAYMAN ISLANDS||Anna.Gray@JyhTY7A1eMK0thNh.com|2452309| +5889|AAAAAAAABAHBAAAA|835199|4005|32364|2451338|2451308|Mrs.|Deborah|Willis|Y|9|11|1982|BARBADOS||Deborah.Willis@6yR3JFn9Cb2mj.com|2452434| +5890|AAAAAAAACAHBAAAA|1711705|4665|49225|2449163|2449133|Mr.|James|Powell|N|11|5|1986|ISLE OF MAN||James.Powell@3mKN9Nuj032Al.edu|2452578| +5891|AAAAAAAADAHBAAAA|187683|5163|12243|2449648|2449618|Sir|Mark|Barajas|N|3|11|1985|ZIMBABWE||Mark.Barajas@p9xHn6NgYat.edu|2452394| +5892|AAAAAAAAEAHBAAAA|472695|3240|33385|2452024|2451994|Ms.|Luz|Clark|Y|11|4|1991|SURINAME||Luz.Clark@yBk756mQgDqYNb.edu|2452539| +5893|AAAAAAAAFAHBAAAA|561924|1133|36742|2451498|2451468|Ms.|Laura|Contreras|N|19|9|1948|BOUVET ISLAND||Laura.Contreras@O9ptlUNVBOLqMMdEXd.org|2452507| +5894|AAAAAAAAGAHBAAAA|1210339|4708|13077|2450893|2450863|Sir|Kenneth|Hammond|N|31|5|1991|KAZAKHSTAN||Kenneth.Hammond@LCIv.com|2452531| +5895|AAAAAAAAHAHBAAAA|1725579|155|21931|2451532|2451502|Dr.|John|Landon|Y|9|9|1927|SOLOMON ISLANDS||John.Landon@16OoSp.org|2452581| +5896|AAAAAAAAIAHBAAAA|1090636|5950|30817|2451553|2451523|Dr.|Tameka|Edwards|Y|2|2|1990|TIMOR-LESTE||Tameka.Edwards@xbQBdU.com|2452354| +5897|AAAAAAAAJAHBAAAA|680303|6677|39898|2451170|2451140|Miss|Vonda|Thornton|N|10|8|1929|MALTA||Vonda.Thornton@FH3BzY5ECgsT4.org|2452540| +5898|AAAAAAAAKAHBAAAA|1009293|6574|41902|2452270|2452240|Dr.|Carolyn|Morton|N|30|11|1929|CYPRUS||Carolyn.Morton@mL88PBFQNu7av.edu|2452299| +5899|AAAAAAAALAHBAAAA|990292|6988|26484|2449433|2449403|Ms.|Carin|Law|Y|14|5|1980|TRINIDAD AND TOBAGO||Carin.Law@TBYNOfpKRAjkV.edu|2452582| +5900|AAAAAAAAMAHBAAAA|1835421|4628|31435|2449149|2449119|Dr.|James|Jordan|N|30|7|1975|ISRAEL||James.Jordan@9ab9UZTGROvRKiyC.org|2452385| +5901|AAAAAAAANAHBAAAA|1748366|3262|6605|2451063|2451033|Mr.|John|Johnston|N|13|7|1979|SRI LANKA||John.Johnston@BboLXOi.org|2452413| +5902|AAAAAAAAOAHBAAAA|726477|2222|19967|2449353|2449323|Dr.|Angela|Lawrence|Y|1|5|1962|BARBADOS||Angela.Lawrence@tpKV3d.edu|2452368| +5903|AAAAAAAAPAHBAAAA|710439||4282||||Melvin|||10|12||||Melvin.Sisk@cD7D8i.org|2452545| +5904|AAAAAAAAABHBAAAA|1239430|550|38023|2451564|2451534|Mrs.|Anna|Fulcher|Y|31|3|1965|BERMUDA||Anna.Fulcher@DNffpPFI1a3kR.edu|2452306| +5905|AAAAAAAABBHBAAAA|88407|6600|49993|||Ms.||Lovelace||18||||||| +5906|AAAAAAAACBHBAAAA|1612273|2183|2540|2449076|2449046|Sir|Jeffrey|Hill|N|15|5|1954|IRELAND||Jeffrey.Hill@DQmicHh.edu|2452457| +5907|AAAAAAAADBHBAAAA|1695580|5390|40000|2452633|2452603|Dr.|Phyllis|Gomez|Y|13|4|1978|MYANMAR||Phyllis.Gomez@ITsSO1E2dG1.com|2452580| +5908|AAAAAAAAEBHBAAAA|1702452|875|23745|2452114|2452084|Miss|Joanna|Wells|Y|11|1|1981|SAUDI ARABIA||Joanna.Wells@HvQyfFhiq8U.com|2452380| +5909|AAAAAAAAFBHBAAAA|890155|535|28515|2449571|2449541|Mr.|Timothy|Dye|N|13|1|1971|ZIMBABWE||Timothy.Dye@RrZgtgRh7ynD82.org|2452518| +5910|AAAAAAAAGBHBAAAA|206894|5352|27057|2450913|2450883|Sir|Stanley|Woods|Y|9|11|1934|CUBA||Stanley.Woods@oLRhi38lkm32f1.edu|2452518| +5911|AAAAAAAAHBHBAAAA|1781686|1835|28784|2451243|2451213|Sir|Robert|Rice|N|12|11|1972|BARBADOS||Robert.Rice@58nNpD2vB.org|2452640| +5912|AAAAAAAAIBHBAAAA|1216719|6173|4427|2450094|2450064|Dr.|Dana|Warren|Y|9|7|1989|UKRAINE||Dana.Warren@USrX07p1B.edu|2452513| +5913|AAAAAAAAJBHBAAAA|150081|549|49017|2449676|2449646|Miss|Margaret|Walker|Y|26|4|1970|RWANDA||Margaret.Walker@ltxcViHYYj.com|2452648| +5914|AAAAAAAAKBHBAAAA|379629|3913|10997|2450055|2450025|Miss|Vivian|Calvert|N|3|1|1978|AFGHANISTAN||Vivian.Calvert@L7qHlGAYr9OT.edu|2452335| +5915|AAAAAAAALBHBAAAA|1565552|2674|27396|2449967|2449937|Mr.|Cecil|Kroll|N|27|2|1989|JERSEY||Cecil.Kroll@vu.org|2452356| +5916|AAAAAAAAMBHBAAAA|381427|4600|46220|2450248|2450218|Dr.|Fannie|Gill|Y|31|7|1938|ITALY||Fannie.Gill@J.org|2452299| +5917|AAAAAAAANBHBAAAA|8348|7030|12159|2450956|2450926|Ms.|Elida|Turner|N|16|1|1928|TOGO||Elida.Turner@y.com|2452354| +5918|AAAAAAAAOBHBAAAA|1719842|4451|21460|2450217|2450187|Dr.|Lester|Benson|Y|3|6|1972|PORTUGAL||Lester.Benson@VeGjEuh.com|2452554| +5919|AAAAAAAAPBHBAAAA|3027|5268|19109|2450225|2450195|Mrs.|Susanna|Marrero|Y|16|8|1971|BRAZIL||Susanna.Marrero@cYxeqK5uA7JrUcH2Iz.com|2452463| +5920|AAAAAAAAACHBAAAA|287001|244|23487|2452616|2452586|Ms.|Dorothy|Edwards|N|30|11|1974|QATAR||Dorothy.Edwards@IkMxMp92.com|2452462| +5921|AAAAAAAABCHBAAAA|381607|1876|28945|2449392|2449362|Sir|Kevin|Brown|N|7|2|1931|TUVALU||Kevin.Brown@CEaMuGdL.edu|2452646| +5922|AAAAAAAACCHBAAAA|1580977|3739|43701|2452423|2452393|Dr.|David|Boone|Y|28|1|1981|ALGERIA||David.Boone@DQ1Am23NN2OlE2C.edu|2452398| +5923|AAAAAAAADCHBAAAA|807093|2189|44038|2452548|2452518|Dr.|Angelica|Stokes|N|20|1|1981|GUAM||Angelica.Stokes@BxMATGrmrJX3.com|2452427| +5924|AAAAAAAAECHBAAAA|1438803|3851|13699|2449194|2449164|Mrs.|Olga|Jackson|N|14|4|1926|CAYMAN ISLANDS||Olga.Jackson@J.edu|2452618| +5925|AAAAAAAAFCHBAAAA|708955|4344|1733|2450539|2450509|Sir|Michael|Jackson|N|18|1|1959|SOLOMON ISLANDS||Michael.Jackson@PUsv7IahI.edu|2452359| +5926|AAAAAAAAGCHBAAAA|1327477|6904|39931|2450242|2450212|Mrs.|Melody|Whitlow|N|9|11|1981|BELGIUM||Melody.Whitlow@uqRsDJ.org|2452630| +5927|AAAAAAAAHCHBAAAA|17359|5306|853|2451810|2451780|Sir|Tony|Vaught|Y|10|2|1972|UKRAINE||Tony.Vaught@GyY.org|2452615| +5928|AAAAAAAAICHBAAAA|1006779|7183|35987|2449582|2449552|Sir|Victor|Small|N|7|9|1984|TAJIKISTAN||Victor.Small@sphVJ42YA.com|2452298| +5929|AAAAAAAAJCHBAAAA|218849|716|20936|2449762|2449732|Mrs.|Gena|Vargas|Y|9|4|1966|GREECE||Gena.Vargas@SXxcVBeUTPF9C.edu|2452547| +5930|AAAAAAAAKCHBAAAA|761145|3054|7363|2449305|2449275|Sir|Emil|Paris|Y|17|8|1951|UNITED STATES||Emil.Paris@3l9B.com|2452294| +5931|AAAAAAAALCHBAAAA|1827729|970|48964|2451639|2451609|Mr.|Frederick|Bird|N|3|5|1961|JAMAICA||Frederick.Bird@SrjLqRFoHTIsDx9.org|2452582| +5932|AAAAAAAAMCHBAAAA|1807307|411|27115|2450436|2450406|Dr.|Erik|Lindsay|N|23|10|1982|MONACO||Erik.Lindsay@2pFFIRN7TJ2ej7q.edu|2452561| +5933|AAAAAAAANCHBAAAA|956482|5367|23805|2450948|2450918|Mrs.|Doreen|Gordon|Y|28|12|1973|ZIMBABWE||Doreen.Gordon@uF5nZNXPr9.org|2452499| +5934|AAAAAAAAOCHBAAAA|1628122|3917|24912|2451074|2451044|Mr.|Zachary|Gibbs|Y|11|11|1944|GIBRALTAR||Zachary.Gibbs@64rdqMiOR9zlQoGM7.edu|2452598| +5935|AAAAAAAAPCHBAAAA|669282|3205|41928|2451523|2451493|Dr.|Mark|Moran|Y|15|12|1979|MEXICO||Mark.Moran@nXU8FCLFOEdUZ.edu|2452634| +5936|AAAAAAAAADHBAAAA|1109565|4768|371|2452651|2452621|Ms.|Ruby|Harper|N|7|8|1976|SOMALIA||Ruby.Harper@FuaaEtuGdc.org|2452455| +5937|AAAAAAAABDHBAAAA|110084|3515|25393|2451077|2451047|Mr.|Todd|Nixon|N|28|5|1973|SOLOMON ISLANDS||Todd.Nixon@fSFCB0lVirkldY.com|2452418| +5938|AAAAAAAACDHBAAAA|873811|4930|26049|2449800|2449770|Dr.|Richard|Levi|N|28|4|1978|MALAWI||Richard.Levi@G9bJ0.org|2452554| +5939|AAAAAAAADDHBAAAA|587904|3919|49134|2449094|2449064|Sir|Rogelio|Peters|N|25|9|1940|VANUATU||Rogelio.Peters@GJtGfAf9q6lmQG.com|2452285| +5940|AAAAAAAAEDHBAAAA|681021|4485|41902|2451468|2451438|Ms.|Florence|Aguilera|Y|6|9|1967|CHRISTMAS ISLAND||Florence.Aguilera@Gi14zther.com|2452496| +5941|AAAAAAAAFDHBAAAA|400695|1358|33668|2449382|2449352|Dr.|Robert|Herman|N|20|3|1925|TRINIDAD AND TOBAGO||Robert.Herman@VpOKL.edu|2452318| +5942|AAAAAAAAGDHBAAAA|1504149|3837|21572|2451054|2451024|Mr.|Thomas|Kersey|Y|26|8|1925|UZBEKISTAN||Thomas.Kersey@ANDp8B.org|2452632| +5943|AAAAAAAAHDHBAAAA|222866|7191|15916|2449144|2449114|Ms.|Alysha|Lawson|Y|30|1|1924|CZECH REPUBLIC||Alysha.Lawson@nrhGdV0qnkYm.edu|2452478| +5944|AAAAAAAAIDHBAAAA|903986|24|46062|2449315|2449285|Mrs.|Joann|Charles|N|22|3|1924|AZERBAIJAN||Joann.Charles@0vq.org|2452408| +5945|AAAAAAAAJDHBAAAA|981719|1041|4363|2452293|2452263|Mrs.|Beverly|Fontaine|N|21|9|1955|NAURU||Beverly.Fontaine@MJJM7LqGz4Ux3.com|2452573| +5946|AAAAAAAAKDHBAAAA|5067|5710|13953|2450024|2449994|Dr.|James|Poulin|Y|1|2|1967|GUINEA-BISSAU||James.Poulin@sFKn87nY34FmBUUvojd.com|2452435| +5947|AAAAAAAALDHBAAAA|1889169|2264|23759|2451007|2450977|Mr.|Curtis|Neal|N|22|8|1929|EQUATORIAL GUINEA||Curtis.Neal@mnkBcfyDgf2L.edu|2452518| +5948|AAAAAAAAMDHBAAAA|157045|2184|7723|2451250|2451220|Mr.|John|Baxter|Y|17|5|1981|SWITZERLAND||John.Baxter@1df.org|2452380| +5949|AAAAAAAANDHBAAAA|884394|3070|26546|2449955|2449925|Dr.|Michael|Stevens|Y|6|6|1952|TIMOR-LESTE||Michael.Stevens@dbf.com|2452596| +5950|AAAAAAAAODHBAAAA|1772249|3466|40236|2450557|2450527|Dr.|Ann|Williams|Y|7|4|1982|CAYMAN ISLANDS||Ann.Williams@drnz.org|2452571| +5951|AAAAAAAAPDHBAAAA|277197||4262|||||||23|||BURUNDI||Thomas.Brown@nhc.com|| +5952|AAAAAAAAAEHBAAAA|525821|3195|5722|2450404|2450374|Miss|Catarina|Hayward|N|26|11|1975|UKRAINE||Catarina.Hayward@GoG.org|2452578| +5953|AAAAAAAABEHBAAAA|1884968|2225|43882|2452155|2452125|Ms.|Josephine|Sanchez|Y|10|7|1946|WESTERN SAHARA||Josephine.Sanchez@h.edu|2452352| +5954|AAAAAAAACEHBAAAA|1698872|272|14872|2451008|2450978|Dr.|William|Hirsch|Y|6|2|1934|KOREA, REPUBLIC OF||William.Hirsch@VQPgAKJa5P.edu|2452431| +5955|AAAAAAAADEHBAAAA|753199|1427|46902|2449177|2449147|Miss|Annette|Butcher|Y|18|8|1949|NEW ZEALAND||Annette.Butcher@z.org|2452541| +5956|AAAAAAAAEEHBAAAA||2512|41925|2451327||Miss|||N||8|1979|||Amy.Delgado@L9dXDKfO.org|2452430| +5957|AAAAAAAAFEHBAAAA|707172|6234|36543|2451843|2451813|Mr.|Jose|Craft|Y|27|9|1989|SOLOMON ISLANDS||Jose.Craft@D5zZCoJR8yoUAn4k.org|2452526| +5958|AAAAAAAAGEHBAAAA|312555|1394|17433|2452160||Miss|Anne||N|||1970|GUADELOUPE|||| +5959|AAAAAAAAHEHBAAAA|1103570|7088|32559|2450272|2450242|Sir|Truman|Mccartney|Y|28|4|1937|GRENADA||Truman.Mccartney@el.org|2452537| +5960|AAAAAAAAIEHBAAAA|1409722|1856|5990|2452500|2452470|Dr.|Ethel|Lester|Y|11|11|1991|SLOVAKIA||Ethel.Lester@0frG62D8OiJ2p.com|2452520| +5961|AAAAAAAAJEHBAAAA|1338041|1114|49434|2449792|2449762|Sir|Sylvester|Hutchings|Y|24|4|1929|PANAMA||Sylvester.Hutchings@qyegznlsUCxlb.com|2452628| +5962|AAAAAAAAKEHBAAAA|1846085|4894|44103|2449759|2449729|Mr.|Louis|Moore|Y|26|6|1988|NEPAL||Louis.Moore@ALt12F5vnF9gqFrijqnY.org|2452545| +5963|AAAAAAAALEHBAAAA|1739129|2982|18435|2451532|2451502|Mr.|Christopher|Williams|Y|12|6|1952|NETHERLANDS ANTILLES||Christopher.Williams@XtbRfJ5.org|2452440| +5964|AAAAAAAAMEHBAAAA|869050|6025|27047|2449979|2449949|Dr.|Judy|Hawkins|N|30|10|1953|PANAMA||Judy.Hawkins@QQbg.org|2452464| +5965|AAAAAAAANEHBAAAA|1067285|1551|1770|2452172|2452142|Dr.|Portia|Seibert|N|19|2|1981|PERU||Portia.Seibert@IgbT8LcG2my79S5.edu|2452514| +5966|AAAAAAAAOEHBAAAA|753187|600|18860|2451315|2451285|Dr.|Jerald|Martin|Y|24|8|1965|NIUE||Jerald.Martin@c.org|2452626| +5967|AAAAAAAAPEHBAAAA|205719|1010|19054|2450151|2450121|Mr.|Gerald|Pease|N|3|12|1932|IRAQ||Gerald.Pease@7fE6xPAk.org|2452498| +5968|AAAAAAAAAFHBAAAA|45646|3104|31009|2452645|2452615|Miss|Carla|Sanders|N|5|8|1967|PAPUA NEW GUINEA||Carla.Sanders@ekd.com|2452489| +5969|AAAAAAAABFHBAAAA|183232|5930|30181|2450624|2450594|Miss|Leann|White|Y|28|10|1947|MARSHALL ISLANDS||Leann.White@Hz5ODf7K8Yuj.com|2452399| +5970|AAAAAAAACFHBAAAA|500794|3896|31751|2451722|2451692|Dr.|Samuel|Funderburk|N|1|4|1947|AFGHANISTAN||Samuel.Funderburk@1c2hMF3aeC0jZdr3XS.org|2452596| +5971|AAAAAAAADFHBAAAA|918725|1805|2132|2451839|2451809|Mr.|Michael|Gray|N|6|7|1965|LIBERIA||Michael.Gray@4ubQJ4yD5d.org|2452587| +5972|AAAAAAAAEFHBAAAA|1378699|4472|4645|2451165|2451135|Mr.|Randy|Steward|Y|19|2|1958|KUWAIT||Randy.Steward@H3o6q9OQJjQvKOt.edu|2452607| +5973|AAAAAAAAFFHBAAAA|1053387|4630|37026|2451547|2451517|Mr.|Raul|Hodges|Y|9|2|1944|CYPRUS||Raul.Hodges@US0Z.org|2452431| +5974|AAAAAAAAGFHBAAAA|1094941|5248|22002|2450772||Mrs.|Martha|Jernigan|N|7|5|1953|VIRGIN ISLANDS, U.S.||Martha.Jernigan@na8yogrOufdJ2O.edu|| +5975|AAAAAAAAHFHBAAAA|307740|2331|46122|2450532|2450502|Mrs.|Ruby|Whitney|Y|23|4|1927|POLAND||Ruby.Whitney@KF.com|2452445| +5976|AAAAAAAAIFHBAAAA|865020|2269|3471|2451759|2451729|Sir|Kevin|Horton|N|31|1|1948|VIRGIN ISLANDS, U.S.||Kevin.Horton@yD9GcT3vXZZ5H.org|2452415| +5977|AAAAAAAAJFHBAAAA|39679|2914|34196|2450148|2450118|Miss|Elizabeth|Peck|Y|26|2|1956|AUSTRALIA||Elizabeth.Peck@NQcZxt6lSxHEzXQ.org|2452389| +5978|AAAAAAAAKFHBAAAA|1483684|2902|24073|2450198|2450168|Dr.|Nancy|Humphrey|N|31|12|1992|PANAMA||Nancy.Humphrey@cQt1YqFZSac8HL.com|2452301| +5979|AAAAAAAALFHBAAAA|1004354|5785|38063|2452540|2452510|Dr.|Phillip|Cortes|N|31|10|1977|BANGLADESH||Phillip.Cortes@F75UrKxf6Jo35Npn.org|2452577| +5980|AAAAAAAAMFHBAAAA|1591764|5288|18859|2450933|2450903|Sir|John|Burrell|N|4|4|1982|BHUTAN||John.Burrell@E.org|2452313| +5981|AAAAAAAANFHBAAAA|1182522|3049|39529|2451634|2451604|Dr.|Fred|Byrd|N|11|5|1986|SWITZERLAND||Fred.Byrd@8RKO0ZuESF.org|2452398| +5982|AAAAAAAAOFHBAAAA|1228942|877|34812|2450152|2450122|Mrs.|Amber|Vaughn|Y|28|5|1983|GIBRALTAR||Amber.Vaughn@C.org|2452330| +5983|AAAAAAAAPFHBAAAA|1107242|962|34282|2449154|2449124|Dr.|Jackie|Hoover|Y|22|1|1927|BRUNEI DARUSSALAM||Jackie.Hoover@d5zEPPnDCA.edu|2452459| +5984|AAAAAAAAAGHBAAAA|1879585|2006|6074|2450790|2450760|Dr.|Joann|Martinez|N|13|4|1954|SOLOMON ISLANDS||Joann.Martinez@N.edu|2452587| +5985|AAAAAAAABGHBAAAA|1293255|2170|238|2452446|2452416|Mrs.|Mellissa|Guerrero|Y|27|11|1985|GREECE||Mellissa.Guerrero@ARLSebbQcan.edu|2452372| +5986|AAAAAAAACGHBAAAA|1114226|4120|42814|2451865|2451835|Dr.|Mathew|Downing|N|14|1|1980|FINLAND||Mathew.Downing@0V4eah7izxKpnf.edu|2452369| +5987|AAAAAAAADGHBAAAA||5331|13851||2451090|Dr.||||12|11|1973|||Marilyn.Romero@QpmJJkMlmEDv9k2sbgj.com|| +5988|AAAAAAAAEGHBAAAA|597274|2524|4174||2449064|Mr.||Davis||||1980||||2452383| +5989|AAAAAAAAFGHBAAAA|1734921|2618|6362|2451980|2451950|Miss|Janet|Renner|Y|1|2|1968|SRI LANKA||Janet.Renner@fooHlNh2DTM.org|2452512| +5990|AAAAAAAAGGHBAAAA|138714|5652|42990|2450401|2450371|Mr.|Arden|Strain|Y|2|4|1971|CHRISTMAS ISLAND||Arden.Strain@YTjJ.com|2452512| +5991|AAAAAAAAHGHBAAAA|164466|6614|49609|2449202|2449172|Mr.|Abraham|Bailey|Y|11|11|1961|NETHERLANDS ANTILLES||Abraham.Bailey@Kpr.com|2452562| +5992|AAAAAAAAIGHBAAAA|1235946|970|26041|2451735|2451705|Dr.|James|Hudson|Y|16|3|1989|HONDURAS||James.Hudson@MDzIeTuDAlz9s.org|2452535| +5993|AAAAAAAAJGHBAAAA|1535459|6577|26495|2450126|2450096|Dr.|Ernest|Byrne|Y|4|7|1951|PITCAIRN||Ernest.Byrne@47.com|2452417| +5994|AAAAAAAAKGHBAAAA||5401|46064|2450254||||Garrett||||1981|JAMAICA|||2452329| +5995|AAAAAAAALGHBAAAA|711017|6854|18291|2450718|2450688|Dr.|Herbert|Dumas|N|30|7|1985|EL SALVADOR||Herbert.Dumas@DPZYED5t76.org|2452464| +5996|AAAAAAAAMGHBAAAA|156198|6077|18356|2452449|2452419|Dr.|Dale|Walker|N|8|8|1975|RUSSIAN FEDERATION||Dale.Walker@dmg5p1IZJr37UgNeI48.edu|2452587| +5997|AAAAAAAANGHBAAAA|466525|3860|37948|2450431|2450401|Ms.|Anthony|Hall|N|26|9|1989|BAHAMAS||Anthony.Hall@8kORUS9x.edu|2452518| +5998|AAAAAAAAOGHBAAAA|317176|6244|4112|2451424|2451394|Sir|Jack|Washington|Y|27|1|1978|TUVALU||Jack.Washington@CzbFCx9yzROjQJSL.org|2452334| +5999|AAAAAAAAPGHBAAAA|177930|3208|30086|2451680|2451650|Sir|Damon|Caskey|N|24|8|1944|TONGA||Damon.Caskey@9v0QJrCYmXc.edu|2452529| +6000|AAAAAAAAAHHBAAAA|||49549|2451627|2451597|Mrs.|||Y|1||1972|JORDAN||Lois.Ramos@nE9aRjQjbED.org|| +6001|AAAAAAAABHHBAAAA|896255|4493|15131|2452242|2452212|Dr.|John|Manley|Y|2|7|1977|MAURITIUS||John.Manley@YRKTyQuO1A.org|2452537| +6002|AAAAAAAACHHBAAAA|1455122|686|37003|2452600|2452570|Dr.|William|Barrett|N|21|11|1960|WESTERN SAHARA||William.Barrett@ZLrN.org|2452475| +6003|AAAAAAAADHHBAAAA|369577|2809|31594|2452457|2452427|Miss|Albert|Baker|N|25|5|1957|NETHERLANDS ANTILLES||Albert.Baker@H8ylnlNZ2.com|2452468| +6004|AAAAAAAAEHHBAAAA|191947|4923|9099|2449340|2449310|Mr.|Perry|Young|N|20|11|1978|GAMBIA||Perry.Young@j03RhOuMJfLiRxPih16.edu|2452414| +6005|AAAAAAAAFHHBAAAA|1419762|6845|39175|2451584|2451554|Dr.|Robert|Fuentes|Y|5|2|1939|TIMOR-LESTE||Robert.Fuentes@E2ye2dA3cIP9.edu|2452293| +6006|AAAAAAAAGHHBAAAA|216578|388|11445|2450695|2450665|Mr.|James|Lopez|N|15|9|1949|MARTINIQUE||James.Lopez@0sTFYIL.org|2452422| +6007|AAAAAAAAHHHBAAAA|360220|5342|48911|2450969|2450939|Ms.|Maggie|Lilly|Y|16|5|1988|ALBANIA||Maggie.Lilly@fqD2uFq.edu|2452363| +6008|AAAAAAAAIHHBAAAA|415837|4680|17876|2450966|2450936|Dr.|Melanie|Ware|N|13|6|1976|BRUNEI DARUSSALAM||Melanie.Ware@LgjTVHevAd0.edu|2452573| +6009|AAAAAAAAJHHBAAAA|1320973|6602|15431|2450792|2450762|Sir|Jack|Jackson|Y|21|1|1992|LESOTHO||Jack.Jackson@GGrAeGvr75ENUHm0j.com|2452470| +6010|AAAAAAAAKHHBAAAA|1874963|2328|28893|2449807|2449777|Miss|Andrew|Coleman|Y|17|4|1926|SAUDI ARABIA||Andrew.Coleman@eMrHPvsl6B8eEdv2V.com|2452536| +6011|AAAAAAAALHHBAAAA|1012942|497|16852|2450543|2450513|Mr.|Tim|Mcgee|Y|10|12|1984|COSTA RICA||Tim.Mcgee@2o2l.com|2452614| +6012|AAAAAAAAMHHBAAAA|937078||37326|2450102|||Steve|Mitchell|||6|1985|ZIMBABWE|||| +6013|AAAAAAAANHHBAAAA|1376433|3983|23933|2451063|2451033|Sir|Carlos|Marquez|Y|23|8|1978|HONG KONG||Carlos.Marquez@RCgciYtz5l4b.org|2452490| +6014|AAAAAAAAOHHBAAAA|385402|5880|10311|2451130|2451100|Mr.|Gordon|Hernandez|N|30|3|1948|BERMUDA||Gordon.Hernandez@kUve.com|2452315| +6015|AAAAAAAAPHHBAAAA|89032|246|25860|2449390|2449360|Mr.|William|Barrett|Y|22|4|1976|GUAM||William.Barrett@nNfSTszgeR2zj.org|2452486| +6016|AAAAAAAAAIHBAAAA|1170512|6002|27539|2449725|2449695|Miss|Pearl|Stanley|N|21|7|1970|ALBANIA||Pearl.Stanley@JRq.org|2452293| +6017|AAAAAAAABIHBAAAA|385010|2928|34446|2452281|2452251|Dr.|Micheal|Baker|N|8|11|1948|GUYANA||Micheal.Baker@PC6UM.org|2452512| +6018|AAAAAAAACIHBAAAA|1183888|4162|28036|2451809|2451779|Miss|Lorrie|Chung|N|17|5|1992|ISRAEL||Lorrie.Chung@qjKfAK.org|2452647| +6019|AAAAAAAADIHBAAAA|722101|1649|22844|2452184|2452154|Ms.|Beatrice|Muncy|N|19|3|1974|BHUTAN||Beatrice.Muncy@sBdiG3fbXZfNeUySbYx.edu|2452449| +6020|AAAAAAAAEIHBAAAA|1241953|399|25292|2451645|2451615|Dr.|Angela|Morgan|N|24|9|1953|PERU||Angela.Morgan@X2apsPC.org|2452351| +6021|AAAAAAAAFIHBAAAA||5518|9824|||Dr.||||26|||CAMEROON|||2452419| +6022|AAAAAAAAGIHBAAAA|1805364|5278|2345|2450028|2449998|Dr.|Cynthia|Taylor|Y|13|1|1936|CANADA||Cynthia.Taylor@P6eoVTo4D8K.com|2452469| +6023|AAAAAAAAHIHBAAAA|906571|3176|23781|2449645|2449615|Mr.|Palmer|Billings|Y|20|12|1933|HAITI||Palmer.Billings@3PdYf1pZGB5.org|2452507| +6024|AAAAAAAAIIHBAAAA|1642810|4792|49174|2451050|2451020|Dr.|Sarah|Mock|Y|23|3|1940|MADAGASCAR||Sarah.Mock@uVipj6ir1apTdUO.edu|2452419| +6025|AAAAAAAAJIHBAAAA|773690|5525|42160|2452434|2452404|Ms.|Lucille|Halstead|Y|24|7|1951|BELGIUM||Lucille.Halstead@ekZOmzLyLz.edu|2452505| +6026|AAAAAAAAKIHBAAAA|1425257|324|22991|2450992|2450962|Dr.|Mattie|Douglass|Y|1|4|1935|WALLIS AND FUTUNA||Mattie.Douglass@XC4PYN8pqdfvqqCGfsP.com|2452485| +6027|AAAAAAAALIHBAAAA|152484|5262|19240|2451420|2451390|Dr.|Vern|Carlton|Y|4|3|1953|GHANA||Vern.Carlton@XHS8EcbJg.edu|2452455| +6028|AAAAAAAAMIHBAAAA|597860|4647|39434||2450119|Sir|Roberto|Hodges|N|4|4|1991|||Roberto.Hodges@dicqKcreQ.edu|| +6029|AAAAAAAANIHBAAAA|751318|924|5283|2449145|2449115|Dr.|Clarence|Jones|Y|8|3|1959|LUXEMBOURG||Clarence.Jones@i6.org|2452326| +6030|AAAAAAAAOIHBAAAA|1026942|5570|5751|2450221|2450191|Dr.|Billy|Ryan|N|21|6|1953|DJIBOUTI||Billy.Ryan@xeKMuVB2.org|2452376| +6031|AAAAAAAAPIHBAAAA|1696454|4793|21579|2451101|2451071|Mrs.|Colette|Green|N|19|3|1977|KIRIBATI||Colette.Green@XkFjhqy1comE3CgkM.edu|2452489| +6032|AAAAAAAAAJHBAAAA|978601|1243|33132|2451105|2451075|Mr.|Lee|Combs|Y|31|1|1959|KAZAKHSTAN||Lee.Combs@IEIc.com|2452501| +6033|AAAAAAAABJHBAAAA|484342|5357|44019|2452481|2452451|Sir|Kevin|Miller|Y|22|4|1948|EGYPT||Kevin.Miller@pI7ArEluuqh.com|2452452| +6034|AAAAAAAACJHBAAAA|1224231|5969|49754|2452315|2452285|Mrs.|Melissa|Reilly|Y|8|9|1930|SAUDI ARABIA||Melissa.Reilly@0Yqfny2lFQPsPa.edu|2452545| +6035|AAAAAAAADJHBAAAA|1407774|3470|40366|2450577|2450547|Mr.|William|Mercer|Y|13|11|1926|DENMARK||William.Mercer@EHkm0Z0hDCvK.com|2452318| +6036|AAAAAAAAEJHBAAAA||2846|39980|||Miss|||||4||BURKINA FASO|||2452545| +6037|AAAAAAAAFJHBAAAA|1195949|1540|13978|2450871|2450841|Dr.|Joanne|Short|N|19|1|1934|UKRAINE||Joanne.Short@FQjbYCipCSH227r2.org|2452289| +6038|AAAAAAAAGJHBAAAA|9506|5099|10739|2449895|2449865|Mrs.|Carolyn|Harris|Y|11|8|1983|KUWAIT||Carolyn.Harris@q.com|2452295| +6039|AAAAAAAAHJHBAAAA|1484907|246|6338|2450276|2450246|Mr.|Richard|Brewer|N|12|3|1966|PAPUA NEW GUINEA||Richard.Brewer@E4UjPtQ1lmHsKLzRZ.com|2452625| +6040|AAAAAAAAIJHBAAAA|147678|2457|13636|2451538|2451508|Dr.|Gordon|Fountain|Y|24|12|1953|JORDAN||Gordon.Fountain@yEmYCFOHX.edu|2452481| +6041|AAAAAAAAJJHBAAAA|1215117|5425|14533|2450512|2450482|Sir|Tony|Jones|N|18|4|1928|MYANMAR||Tony.Jones@MzT.org|2452339| +6042|AAAAAAAAKJHBAAAA|116991|4286|2381|2450026|2449996|Mr.|Roderick|Ballard|N|24|9|1979|MAURITANIA||Roderick.Ballard@7nlquhcSxpi29nK3d.com|2452635| +6043|AAAAAAAALJHBAAAA|740071|4882|5693|2450730|2450700|Ms.|Stephanie|Simons|Y|28|6|1968|ITALY||Stephanie.Simons@aFQ9OC4VauS9njfEI.edu|2452620| +6044|AAAAAAAAMJHBAAAA|896401|3111|14221|2451064|2451034|Dr.|Cynthia|Hutton|N|23|12|1969|FAROE ISLANDS||Cynthia.Hutton@FAmxFH10D.com|2452493| +6045|AAAAAAAANJHBAAAA|282447|1496|28545|2450949|2450919|Ms.|Karla|Vasquez|N|29|7|1978|UNITED STATES||Karla.Vasquez@6H3Lrt3h.org|2452647| +6046|AAAAAAAAOJHBAAAA|326660|6080|37500|2451555|2451525|Dr.|Laura|Helm|Y|12|1|1984|UKRAINE||Laura.Helm@HamISPL4xN.com|2452639| +6047|AAAAAAAAPJHBAAAA|928806|1895|42984|2449965|2449935|Sir|Philip|Hall|Y|19|9|1950|SYRIAN ARAB REPUBLIC||Philip.Hall@kacXUR.com|2452637| +6048|AAAAAAAAAKHBAAAA|1120802|4640|19725|2451659|2451629|Dr.|Lincoln|Hart|N|2|8|1928|UKRAINE||Lincoln.Hart@4TgeZT9MOER1K.org|2452298| +6049|AAAAAAAABKHBAAAA|594121|6570|16619|2450816|2450786|Ms.|Rhonda|Bassett|N|26|11|1967|SIERRA LEONE||Rhonda.Bassett@5ECJmQcJ.com|2452390| +6050|AAAAAAAACKHBAAAA|1702143|3395|23847|2449050|2449020|Dr.|Everett|Johnson|N|16|11|1941|GHANA||Everett.Johnson@btN2pNYNJufDa.org|2452558| +6051|AAAAAAAADKHBAAAA|1885431|727|21623|2450082|2450052|Dr.|Christopher|Gross|Y|25|3|1969|GHANA||Christopher.Gross@dhMGmz9uH1.com|2452638| +6052|AAAAAAAAEKHBAAAA|1918137|282|28827|2452369|2452339|Miss|Sherry|Knight|Y|23|12|1971|BRUNEI DARUSSALAM||Sherry.Knight@P9iLAjc8n4kapJx39.edu|2452429| +6053|AAAAAAAAFKHBAAAA|493623|4254|14485|2450377|2450347|Ms.|Margaret|Garza|Y|20|7|1969|ANGOLA||Margaret.Garza@Nz5rVo2on0NMs433rr.com|2452626| +6054|AAAAAAAAGKHBAAAA|640519|5320|8521|2451540|2451510|Dr.|Dennis|Morgan|N|28|10|1951|YEMEN||Dennis.Morgan@F7GhuRpcJ.edu|2452592| +6055|AAAAAAAAHKHBAAAA|1098888|1402|20763|2450761|2450731|Mr.|Charles|Franklin|N|25|8|1987|BOTSWANA||Charles.Franklin@HuERull.com|2452647| +6056|AAAAAAAAIKHBAAAA|354138|569|21847|2451206|2451176|Mr.|Troy|Lawrence|Y|21|6|1936|GHANA||Troy.Lawrence@E3JNQjhy.com|2452324| +6057|AAAAAAAAJKHBAAAA|894608|1453|17545|2451811|2451781|Mrs.|Tanja|Smith|Y|19|5|1926|TAJIKISTAN||Tanja.Smith@CB.com|2452630| +6058|AAAAAAAAKKHBAAAA|548336|6103|20670|2451317|2451287|Ms.|Kimberly|Clark|Y|20|2|1972|MOLDOVA, REPUBLIC OF||Kimberly.Clark@gSflJ6NFsVQRbqPBn.edu|2452314| +6059|AAAAAAAALKHBAAAA|1198009|2629|11296|2450067|2450037|Mrs.|Susan|Morrison|Y|27|8|1971|ALBANIA||Susan.Morrison@D9k.com|2452608| +6060|AAAAAAAAMKHBAAAA|393453|4900|28794|2452551|2452521|Ms.|Anthony|George|Y|9|12|1954|SYRIAN ARAB REPUBLIC||Anthony.George@vtRalx.edu|2452403| +6061|AAAAAAAANKHBAAAA|1116136|5877|46418|2449770|2449740|Dr.|Carlos|Bartlett|N|27|10|1987|GERMANY||Carlos.Bartlett@L3Sz2z8ZVJd5pTkvzU.org|2452423| +6062|AAAAAAAAOKHBAAAA|1331474|5490|14659|2451998|2451968|Ms.|Adam|Heaton|Y|7|11|1991|UNITED KINGDOM||Adam.Heaton@fKbb.com|2452384| +6063|AAAAAAAAPKHBAAAA|361576|2709|22642|2449817|2449787|Mr.|Herbert|Varney|Y|7|10|1924|UNITED KINGDOM||Herbert.Varney@50Aovb7fbt.com|2452592| +6064|AAAAAAAAALHBAAAA|1665169|2679|4142|2452430|2452400|Ms.|Andra|Thomas|N|5|3|1962|OMAN||Andra.Thomas@SXlOdHv.edu|2452639| +6065|AAAAAAAABLHBAAAA|938911|127|5743|2452322|2452292|Dr.|Johnny|Golden|Y|18|2|1955|PITCAIRN||Johnny.Golden@ea5875T7gBL.edu|2452299| +6066|AAAAAAAACLHBAAAA|759310|4932|14154|2450419|2450389|Dr.|Ronald|Hughes|Y|17|5|1951|GABON||Ronald.Hughes@V5l3EzjPFiMNMq.org|2452612| +6067|AAAAAAAADLHBAAAA|1822462|1240|371|2450052|2450022|Sir|Clarence|Henderson|Y|7|1|1949|GUINEA||Clarence.Henderson@Q3FCYG.com|2452634| +6068|AAAAAAAAELHBAAAA|1078666|4455|32244|2452226|2452196|Miss|Felicita|Cross|Y|28|8|1985|INDONESIA||Felicita.Cross@uqcrDufkQmuda7I.com|2452396| +6069|AAAAAAAAFLHBAAAA|1140970|4699|828|2450597|2450567|Ms.|Doris|Gilbreath|N|29|1|1942|PUERTO RICO||Doris.Gilbreath@lpgAEHkOVtRS.com|2452379| +6070|AAAAAAAAGLHBAAAA|1706944|6256|41215|2452001|2451971|Sir|Shawn|Cummings|Y|13|5|1927|FINLAND||Shawn.Cummings@rpxgbvaVHn1Fz.org|2452553| +6071|AAAAAAAAHLHBAAAA|1878047|6861|3275|2450298|2450268|Miss|Minnie|Hidalgo|N|22|2|1972|MYANMAR||Minnie.Hidalgo@O1efP3a6.edu|2452409| +6072|AAAAAAAAILHBAAAA|1082941|2716|8938|2450086|2450056|Dr.|Jonathan|Griffith|N|13|5|1972|SAN MARINO||Jonathan.Griffith@SO367q3NNazg.org|2452306| +6073|AAAAAAAAJLHBAAAA|201519|371|21623|2452284|2452254|Sir|Jason|Smith|Y|16|6|1975|DJIBOUTI||Jason.Smith@1diJiXpEtlaqija.com|2452398| +6074|AAAAAAAAKLHBAAAA|1574816|6666|20069|2452250|2452220|Sir|Geoffrey|Waller|Y|9|11|1938|QATAR||Geoffrey.Waller@FuUobpB.org|2452424| +6075|AAAAAAAALLHBAAAA|570206|5750|19647|2451549|2451519|Mr.|Matthew|Tuggle|Y|15|11|1927|ANGUILLA||Matthew.Tuggle@LlLpNH.edu|2452357| +6076|AAAAAAAAMLHBAAAA|370491|3953|6662|2451292|2451262|Sir|Son|Kendrick|N|3|7|1926|BRUNEI DARUSSALAM||Son.Kendrick@2rY9.com|2452427| +6077|AAAAAAAANLHBAAAA|276368|509|6650|2450323|2450293|Dr.|Andrew|Brady|Y|28|4|1981|TUVALU||Andrew.Brady@AZNqc6ThtAPpmQ8sS.com|2452290| +6078|AAAAAAAAOLHBAAAA|1261867|5629|18991|2452400|2452370|Sir|Daniel|Coleman|N|23|5|1943|CAMEROON||Daniel.Coleman@zmaChsrruSgdazv.edu|2452314| +6079|AAAAAAAAPLHBAAAA|20325|2993|10014|2452541|2452511|Ms.|Deborah|Ward|N|30|1|1976|JORDAN||Deborah.Ward@4ALnfN84xM50Tig.com|2452520| +6080|AAAAAAAAAMHBAAAA|167454|4846|10285|2450019|2449989|Miss|Dina|Nevarez|Y|1|7|1975|VIRGIN ISLANDS, U.S.||Dina.Nevarez@Z5J1UXSalbVOTM.com|2452340| +6081|AAAAAAAABMHBAAAA|1066734|6261|20235|2452451||Miss||Taylor|N||||UNITED STATES|||| +6082|AAAAAAAACMHBAAAA|362522|1207|23539|2452072|2452042|Mrs.|Kristi|Green|N|28|11|1982|FAROE ISLANDS||Kristi.Green@MaIKm2TPjDlvz.com|2452463| +6083|AAAAAAAADMHBAAAA|1626196|1974|1352|2451515|2451485|Ms.|Danyelle|Caldwell|N|26|7|1958|BULGARIA||Danyelle.Caldwell@EVYkpfzaafM4FJR2p.com|2452477| +6084|AAAAAAAAEMHBAAAA|236638|4077|42316|2449656|2449626|Miss|Wendy|Stephens|Y|31|3|1948|KUWAIT||Wendy.Stephens@HSb75Pkvik.com|2452347| +6085|AAAAAAAAFMHBAAAA|1814231|1859|21487|2452119|2452089|Dr.|Christopher|Burk|N|15|9|1969|ARMENIA||Christopher.Burk@tbsMM.org|2452371| +6086|AAAAAAAAGMHBAAAA|901857|1892|2327|2450171|2450141|Dr.|Louie|Brooks|N|9|5|1962|BELIZE||Louie.Brooks@SIexUKCQUkdR.edu|2452483| +6087|AAAAAAAAHMHBAAAA|287865|4087|29462|2449929|2449899|Mrs.|Donna|Trowbridge|Y|2|3|1941|ARMENIA||Donna.Trowbridge@EMRrB.edu|2452331| +6088|AAAAAAAAIMHBAAAA|128674|4675|42276|2451072|2451042|Mrs.|Kathy|Knowles|N|9|5|1984|ARMENIA||Kathy.Knowles@7O.edu|2452552| +6089|AAAAAAAAJMHBAAAA|1680002|3724|14246|2449328|2449298|Ms.|Christine|Sisk|Y|12|1|1946|MONACO||Christine.Sisk@vj.edu|2452336| +6090|AAAAAAAAKMHBAAAA|757268|3893|14565|2449687|2449657|Mr.|Ronald|Hall|Y|16|11|1945|LESOTHO||Ronald.Hall@hX2mz9rx.edu|2452640| +6091|AAAAAAAALMHBAAAA|166804||40742||2449523||Julia|||||||||2452310| +6092|AAAAAAAAMMHBAAAA|1740298|6919|37540|2449490|2449460|Mrs.|Elisabeth|Warren|Y|25|5|1934|FAROE ISLANDS||Elisabeth.Warren@5k3bt15t6Zur1.org|2452402| +6093|AAAAAAAANMHBAAAA|429934|2657|17733|2451448|2451418|Sir|Donald|Maas|N|3|11|1985|JAPAN||Donald.Maas@oncoIncnBHqUDATb.edu|2452520| +6094|AAAAAAAAOMHBAAAA|1879051|1995|14139|2451538|2451508|Ms.|Louise|Cavazos|Y|3|12|1936|JAPAN||Louise.Cavazos@yKKn98qfmF1Vx.com|2452502| +6095|AAAAAAAAPMHBAAAA|557079|2987|34495|2452461|2452431|Ms.|Alma|Stephens|N|8|12|1959|NETHERLANDS||Alma.Stephens@1isYRepVovYRTBx.edu|2452340| +6096|AAAAAAAAANHBAAAA|70587|5978|14969|2449311|2449281|Dr.|Joseph|Long|N|10|11|1951|SYRIAN ARAB REPUBLIC||Joseph.Long@YZ.edu|2452390| +6097|AAAAAAAABNHBAAAA|414197|3419|21841|2450297|2450267|Ms.|Cathy|Grier|N|17|10|1945|NORFOLK ISLAND||Cathy.Grier@izB2z09BdEv1noA6.org|2452638| +6098|AAAAAAAACNHBAAAA|766074|1984|15094|2449429|2449399|Dr.|Jon|Holm|N|7|9|1986|JERSEY||Jon.Holm@NidbMMRf2C.org|2452380| +6099|AAAAAAAADNHBAAAA|1741242|6939|4683|2449308|2449278|Dr.|Russ|Cahill|Y|9|1|1928|NORFOLK ISLAND||Russ.Cahill@zac.org|2452516| +6100|AAAAAAAAENHBAAAA|186311|2825|28586|2450528|2450498|Ms.|Sophia|Rios|N|17|7|1931|ALBANIA||Sophia.Rios@gJb760M.edu|2452447| +6101|AAAAAAAAFNHBAAAA|1410372|4141|42757|2451403|2451373|Sir|Thomas|Cummings|Y|14|11|1952|TUNISIA||Thomas.Cummings@4nM.edu|2452553| +6102|AAAAAAAAGNHBAAAA|741758|6023|7888|2449784|2449754|Dr.|Dominga|Fields|Y|5|8|1934|MOROCCO||Dominga.Fields@Et.org|2452284| +6103|AAAAAAAAHNHBAAAA|1767480|4953|38026|2451438|2451408|Dr.|Wanda|Abraham|Y|18|12|1976|BULGARIA||Wanda.Abraham@jgCU7.org|2452326| +6104|AAAAAAAAINHBAAAA|1756348|3479|3810|2451318|2451288|Sir|Edward|Brown|Y|23|6|1964|GHANA||Edward.Brown@u0nTlIb.com|2452399| +6105|AAAAAAAAJNHBAAAA|422338|2669|28153|2451415|2451385|Ms.|Roberta|Hill|Y|12|3|1940|BOUVET ISLAND||Roberta.Hill@0zOBQUjbVpSIbNxi.edu|2452496| +6106|AAAAAAAAKNHBAAAA|952665||26699|2451321|2451291||||||3|1963|||John.Rodgers@MtOEHAM.com|2452347| +6107|AAAAAAAALNHBAAAA|1547088|3700|3865|2452622|2452592|Dr.|Diann|Johnson|Y|14|6|1960|KIRIBATI||Diann.Johnson@yjF6caG00p.com|2452338| +6108|AAAAAAAAMNHBAAAA|334566|6804|8642|2449477|2449447|Dr.|Missy|Savoy|N|20|9|1959|KOREA, REPUBLIC OF||Missy.Savoy@54n92TxtPB4Xl.edu|2452489| +6109|AAAAAAAANNHBAAAA|248936|233|26330|2449734|2449704|Dr.|Warren|Russell|N|23|12|1950|ERITREA||Warren.Russell@Fytl.edu|2452412| +6110|AAAAAAAAONHBAAAA|524414|6863|21312|2449728|2449698|Mr.|Peter|Gill|Y|18|12|1977|SIERRA LEONE||Peter.Gill@3Md9p8s8Ly3iKD.edu|2452523| +6111|AAAAAAAAPNHBAAAA|1712269|3399|14129|2450014|2449984|Mr.|James|Reyes|N|10|7|1970|QATAR||James.Reyes@M.org|2452376| +6112|AAAAAAAAAOHBAAAA|85409|5641|42273|2449301|2449271|Sir|Thomas|Cohen|N|26|11|1983|KUWAIT||Thomas.Cohen@2LkrxBfVOXbst79Cb.edu|2452553| +6113|AAAAAAAABOHBAAAA|1506146|317|22088|2449872|2449842|Mrs.|Anne|Lapointe|N|15|3|1924|BELGIUM||Anne.Lapointe@0c9tlTg93k.edu|2452480| +6114|AAAAAAAACOHBAAAA|54411|6975|3586|2449943|2449913|Dr.|Kenneth|Preston|Y|25|5|1956|NAURU||Kenneth.Preston@bNQGAylhzjAoM91vXQG.org|2452577| +6115|AAAAAAAADOHBAAAA|1843215|5986|17957|2452543|2452513|Mr.|John|Barrett|Y|16|11|1992|SLOVENIA||John.Barrett@EPQPRLpx.com|2452325| +6116|AAAAAAAAEOHBAAAA|1661385|916|33200|2449503|2449473|Sir|Charley|Sanchez|Y|18|7|1931|SERBIA||Charley.Sanchez@AqkFISyalOeyC.com|2452631| +6117|AAAAAAAAFOHBAAAA|1599977|6191|7605|2452617|2452587|Ms.|Tricia|Anders|N|23|8|1971|TUNISIA||Tricia.Anders@i80mD1p4sP.com|2452584| +6118|AAAAAAAAGOHBAAAA|526740|3001|24462|2452371|2452341|Ms.|Joann|Odonnell|N|7|11|1974|ALAND ISLANDS||Joann.Odonnell@IsOfchE2.edu|2452555| +6119|AAAAAAAAHOHBAAAA|536809|5793|24824|2451267|2451237|Mr.|Thomas|Reid|Y|25|3|1948|ARMENIA||Thomas.Reid@oOJ.com|2452362| +6120|AAAAAAAAIOHBAAAA|565056|870|31484|2451101|2451071|Ms.|Sharon|Peterson|Y|2|5|1979|MONTSERRAT||Sharon.Peterson@4.org|2452490| +6121|AAAAAAAAJOHBAAAA|171854|6570|19239|2451165|2451135|Miss|Sarah|Crabtree|N|27|10|1951|PAKISTAN||Sarah.Crabtree@mpIP4ltNSO2.org|2452366| +6122|AAAAAAAAKOHBAAAA|1101125|4266|17807|2452478|2452448|Dr.|Raul|Vazquez|N|19|2|1971|JAMAICA||Raul.Vazquez@GzkLbpj4Zo.org|2452325| +6123|AAAAAAAALOHBAAAA|272848|2352|36703|2451903|2451873|Miss|Laura|Robinson|N|24|8|1930|ISRAEL||Laura.Robinson@teubxr.edu|2452576| +6124|AAAAAAAAMOHBAAAA|1838028|510|20545|2449135|2449105|Mr.|Lamar|Deal|N|19|1|1987|BOUVET ISLAND||Lamar.Deal@10FEZc8YIh.com|2452567| +6125|AAAAAAAANOHBAAAA|1090512|3629|24299|2452364|2452334|Mrs.|Albert|Acosta|N|11|1|1932|ROMANIA||Albert.Acosta@46Ill.com|2452534| +6126|AAAAAAAAOOHBAAAA|576270|132|31594|2451298|2451268|Ms.|Linda|Maddox|Y|14|7|1932|ERITREA||Linda.Maddox@KsA.com|2452361| +6127|AAAAAAAAPOHBAAAA|120872|6060|26801|2452647|2452617|Mrs.|Linnie|Erickson|N|10|9|1931|UNITED STATES||Linnie.Erickson@jjH0YS0bZEQgNu.org|2452568| +6128|AAAAAAAAAPHBAAAA|1872562|2209|47056|2452403|2452373|Sir|Elbert|Cahill|N|22|12|1944|GUERNSEY||Elbert.Cahill@MadKPDg05nnrVVR.com|2452309| +6129|AAAAAAAABPHBAAAA|1058289|6231|29020|2449173|2449143|Ms.|Kathryn|Wright|N|30|9|1985|GUYANA||Kathryn.Wright@paI1ZknzTAoYIk.edu|2452285| +6130|AAAAAAAACPHBAAAA|687431|3551|1318|2452549|2452519|Mr.|Jimmie|Lewis|Y|16|9|1951|BOLIVIA||Jimmie.Lewis@XeHK.edu|2452571| +6131|AAAAAAAADPHBAAAA|1809212|5240|35668|2451460|2451430|Dr.|Katherine|Hood|Y|21|10|1954|PITCAIRN||Katherine.Hood@uEitadD91v3VVGY.com|2452289| +6132|AAAAAAAAEPHBAAAA||1299|25714|2449660|2449630|Mr.|Emmanuel||||||||Emmanuel.Solis@6vFk1K4Fl4knm.edu|2452596| +6133|AAAAAAAAFPHBAAAA|853325|5667|45256|2452040|2452010|Mr.|Raymond|Parker|N|27|12|1944|VANUATU||Raymond.Parker@bUEXtz3X901jL.org|2452538| +6134|AAAAAAAAGPHBAAAA|1646062|7030|47951|2449158|2449128|Dr.|Edwin|Hutchison|Y|4|11|1954|BELIZE||Edwin.Hutchison@5yQDu9BgT2y0Qfln1NiR.org|2452337| +6135|AAAAAAAAHPHBAAAA|1857177|4224|17671|2451877|2451847|Ms.|Della|Burgess|N|29|10|1977|LIBERIA||Della.Burgess@i8Zf.org|2452376| +6136|AAAAAAAAIPHBAAAA|979829|6119|38762|2451081|2451051|Ms.|Dani|Swain|Y|23|5|1929|ALGERIA||Dani.Swain@pVhgi8DknC8Kk.com|2452457| +6137|AAAAAAAAJPHBAAAA|624155|2746|20537|2449201|2449171|Mr.|Joel|Worthington|Y|28|8|1934|LATVIA||Joel.Worthington@IFMCGrG2xG.com|2452456| +6138|AAAAAAAAKPHBAAAA|1749749|6508|67|2450620|2450590|Mr.|George|Patterson|Y|18|10|1928|CUBA||George.Patterson@Z1Eqjj6QzRHYU.com|2452603| +6139|AAAAAAAALPHBAAAA|1176161|1808|46302|2449817|2449787|Mrs.|Barbara|Mccall|N|2|2|1963|COMOROS||Barbara.Mccall@dze6RKraJR.com|2452523| +6140|AAAAAAAAMPHBAAAA|1429801|2275|45596|2450095|2450065|Miss|Misty|Wood|Y|1|7|1953|MARTINIQUE||Misty.Wood@4OVbIHtleOFUyXq.org|2452329| +6141|AAAAAAAANPHBAAAA|761427|3633|5792|2451769|2451739|Miss|Margaret|Lucas|N|8|4|1927|CROATIA||Margaret.Lucas@1CtqQzonF.com|2452422| +6142|AAAAAAAAOPHBAAAA|1744991|5516|14041|2449035|2449005|Dr.|Charles|Owen|N|12|8|1986|ANTIGUA AND BARBUDA||Charles.Owen@T53PR46azr.com|2452427| +6143|AAAAAAAAPPHBAAAA|86611|1372|8933|2449431|2449401|Sir|Gregory|Williamson|N|16|4|1948|CANADA||Gregory.Williamson@xgr2Xx.edu|2452474| +6144|AAAAAAAAAAIBAAAA|543227|1335|7890|2451374|2451344|Sir|Craig|Scully|N|28|10|1961|TUNISIA||Craig.Scully@nlGNGnArcvn.org|2452441| +6145|AAAAAAAABAIBAAAA|1072049|174|19146|2449445|2449415|Mrs.|Tara|Watts|Y|10|9|1979|KUWAIT||Tara.Watts@BhYeG.edu|2452437| +6146|AAAAAAAACAIBAAAA|631996|5381|32766|2449843|2449813|Sir|Paul|Jones|Y|22|9|1936|MAURITANIA||Paul.Jones@v1g41OpB.org|2452478| +6147|AAAAAAAADAIBAAAA|1283951|90|15382|2451119|2451089|Sir|Sam|Martin|Y|16|7|1960|NAMIBIA||Sam.Martin@MOgasp.edu|2452355| +6148|AAAAAAAAEAIBAAAA|1119941|4251|38114|2450489|2450459|Miss||Troutman||28||1952|ARGENTINA|||2452619| +6149|AAAAAAAAFAIBAAAA|1322101|144|33340|2451851|2451821|Dr.|Matthew|Sandoval|Y|24|6|1935|JORDAN||Matthew.Sandoval@fMRKiEEEsiKFD.org|2452623| +6150|AAAAAAAAGAIBAAAA|821380|5652|11124|2449687|2449657|Dr.|Ryan|Hall|N|3|7|1980|QATAR||Ryan.Hall@G0rkE.com|2452454| +6151|AAAAAAAAHAIBAAAA|990106|2210|11067|2450362|2450332|Dr.|Ronnie|Landrum|Y|5|8|1926|NEPAL||Ronnie.Landrum@8G0jftf1yhUggZmFEhD.com|2452536| +6152|AAAAAAAAIAIBAAAA|174981|2013|21045|2449316|2449286|Miss|Marilyn|Stein|Y|14|10|1948|TAJIKISTAN||Marilyn.Stein@OXv.edu|2452380| +6153|AAAAAAAAJAIBAAAA|907492|1721|17150|2449751|2449721|Miss|Katherine|Doyle|N|11|4|1989|MAYOTTE||Katherine.Doyle@K8n3Zfn24jgyIC6NDpLN.org|2452407| +6154|AAAAAAAAKAIBAAAA|21278|5615|27552|2450291|2450261|Dr.|Angela|Daniels|N|29|1|1968|VIRGIN ISLANDS, U.S.||Angela.Daniels@4N1E8aKX2f.com|2452315| +6155|AAAAAAAALAIBAAAA|1641356|3976|32196|2450653|2450623|Mr.|James|Bray|N|14|11|1934|SPAIN||James.Bray@5p9DLRIQ.edu|2452529| +6156|AAAAAAAAMAIBAAAA|814917|1291|18065|2450094|2450064|Mr.|Stanley|Burks|Y|10|6|1990|CAPE VERDE||Stanley.Burks@iT.org|2452446| +6157|AAAAAAAANAIBAAAA|793290|4804|23565|2452138|2452108|Sir|Michael|Henry|Y|16|11|1966|VANUATU||Michael.Henry@X34ON.edu|2452310| +6158|AAAAAAAAOAIBAAAA|433780|1300|44867|2452051|2452021|Mr.|Roger|Noonan|N|17|5|1960|ICELAND||Roger.Noonan@r2Ux7qoYmdfy.edu|2452325| +6159|AAAAAAAAPAIBAAAA|1470743|1455|48967|2450127|2450097|Ms.|Debra|Hauser|N|14|2|1990|GUERNSEY||Debra.Hauser@guGbYxiB850MfnZ.edu|2452364| +6160|AAAAAAAAABIBAAAA|59040|4149|8858|2450353|2450323|Sir|James|Townsend|N|6|1|1932|GUATEMALA||James.Townsend@I7qeXP2cU.com|2452441| +6161|AAAAAAAABBIBAAAA|1754962|5240|20318|2451002|2450972|Miss|Claudia|Reese|Y|8|5|1966|SOUTH AFRICA||Claudia.Reese@PrnN.org|2452577| +6162|AAAAAAAACBIBAAAA|797813|6631|46696|2452471|2452441|Dr.|Tawana|Ball|Y|9|11|1955|MALI||Tawana.Ball@53d4VJo.org|2452302| +6163|AAAAAAAADBIBAAAA|1096880|52|40871|2450215|2450185|Sir|Byron|Evans|Y|3|6|1951|SWAZILAND||Byron.Evans@X3zeRYBq7a3MVjEbPa4.edu|2452344| +6164|AAAAAAAAEBIBAAAA|1035067|3808|8949|2452589|2452559|Sir|Edward|Keefer|N|24|7|1972|MALAYSIA||Edward.Keefer@FbFVCCFv.org|2452644| +6165|AAAAAAAAFBIBAAAA|331232|5664|16449|2452085|2452055|Sir|Leland|Burdick|Y|2|11|1961|NETHERLANDS||Leland.Burdick@4B0Yc0P4qSR3XEP.edu|2452447| +6166|AAAAAAAAGBIBAAAA|1273813|4312|31611|2450028|2449998|Dr.|Samuel|Jackson|N|6|9|1992|TONGA||Samuel.Jackson@K.com|2452419| +6167|AAAAAAAAHBIBAAAA|219339|3608|2277|2449177|2449147|Ms.|Jennifer|Maher|Y|28|8|1958|COMOROS||Jennifer.Maher@gpIcZbuxA7O1rx.org|2452609| +6168|AAAAAAAAIBIBAAAA|20362|5863|28517|2450704|2450674|Sir|Gregory|Westbrook|Y|21|1|1980|ITALY||Gregory.Westbrook@dASrU01aHI3lCd0V6l.org|2452386| +6169|AAAAAAAAJBIBAAAA|1805069|741|18413|2449075|2449045|Dr.|Alvin|Mcclendon|Y|26|11|1952|UKRAINE||Alvin.Mcclendon@0U.com|2452582| +6170|AAAAAAAAKBIBAAAA|674402|6295|24235|2449398|2449368|Dr.|Melissa|Thomas|Y|2|10|1939|LATVIA||Melissa.Thomas@3.org|2452580| +6171|AAAAAAAALBIBAAAA|412580|4970|23407|2450206|2450176|Dr.|Justin|Guidry|N|25|12|1990|MYANMAR||Justin.Guidry@c.org|2452516| +6172|AAAAAAAAMBIBAAAA|1666623|6543|16063|2449339|2449309|Dr.|Solomon|Peters|N|2|10|1974|MYANMAR||Solomon.Peters@Z.edu|2452529| +6173|AAAAAAAANBIBAAAA|715795|426|32657|2451796|2451766|Miss|Vanessa|Burns|N|18|6|1943|ARMENIA||Vanessa.Burns@qxKBLJ.edu|2452401| +6174|AAAAAAAAOBIBAAAA|1644042|1687|24132|2451085|2451055|Sir|Christopher|Leon|N|17|8|1941|MAYOTTE||Christopher.Leon@KKz7PG.edu|2452465| +6175|AAAAAAAAPBIBAAAA|6688|2783|24078|2450781|2450751|Sir|Kory|Pierce|N|29|11|1973|TUVALU||Kory.Pierce@D.org|2452645| +6176|AAAAAAAAACIBAAAA|1029303|5063|48423|2449188|2449158|Miss|Caroline|Torres|N|16|5|1989|SIERRA LEONE||Caroline.Torres@qR9tU0djEUPHPnO.com|2452473| +6177|AAAAAAAABCIBAAAA|576255|5067|9879|2449801|2449771|Dr.|Elyse|Stewart|N|29|5|1945|ANGOLA||Elyse.Stewart@zjmHx6emK.org|2452425| +6178|AAAAAAAACCIBAAAA|1658631|5069|16354|2449175|2449145|Mr.|John|Sanchez|Y|14|12|1959|INDIA||John.Sanchez@GGHchik.org|2452433| +6179|AAAAAAAADCIBAAAA|822093|2371|41096|2452266|2452236|Mr.|Christopher|Maestas|Y|3|9|1989|LITHUANIA||Christopher.Maestas@DZu072nJqffVQS.edu|2452457| +6180|AAAAAAAAECIBAAAA|38303|6887|17424|2450829|2450799|Mrs.|Iliana|Spencer|N|17|1|1967|MALAWI||Iliana.Spencer@chSlVADXMeL8.com|2452532| +6181|AAAAAAAAFCIBAAAA|364551|5156|8208|2451044|2451014|Mr.|Cody|Richards|Y|15|5|1953|VIET NAM||Cody.Richards@yDvTysURdf4dvREU6bU.edu|2452417| +6182|AAAAAAAAGCIBAAAA|1599314|2346|43378|2449191|2449161|Ms.|Niki|Wilson|N|16|6|1978|COSTA RICA||Niki.Wilson@TrQp.com|2452304| +6183|AAAAAAAAHCIBAAAA|398093|679|29265|2451980|2451950|Miss|Sally|Barbour|Y|25|10|1955|NETHERLANDS ANTILLES||Sally.Barbour@UqRKoQi.com|2452513| +6184|AAAAAAAAICIBAAAA|262560|287|41261|2451220|2451190|Mrs.|Rosa|Franco|Y|21|5|1947|GERMANY||Rosa.Franco@O.org|2452408| +6185|AAAAAAAAJCIBAAAA|515098|3596|447|2449384|2449354|Sir|Calvin|Atkinson|Y|29|11|1925|CANADA||Calvin.Atkinson@JTHUc5Qg2M.edu|2452407| +6186|AAAAAAAAKCIBAAAA|834858|4965|15472|2449669|2449639|Dr.|Luis|Knapp|Y|7|12|1992|SAN MARINO||Luis.Knapp@SU.com|2452552| +6187|AAAAAAAALCIBAAAA|1900399|3971|38790|2449202|2449172|Dr.|George|Brock|Y|8|2|1988|ZAMBIA||George.Brock@Luc.edu|2452610| +6188|AAAAAAAAMCIBAAAA|222550|3404|42611|2451323|2451293|Sir|William|Conger|N|19|3|1984|VIET NAM||William.Conger@NiKX8Fp7g1TmS8GhdM.com|2452492| +6189|AAAAAAAANCIBAAAA|1381919|1790|26814|2450768|2450738|Sir|Freddie|Freese|N|18|10|1978|ZAMBIA||Freddie.Freese@IyGfM7gxjnP.com|2452636| +6190|AAAAAAAAOCIBAAAA|654995|4076|4357|2452255|2452225|Mrs.|Katherine|Howard|Y|6|1|1954|OMAN||Katherine.Howard@2FlC4OnFPI.edu|2452348| +6191|AAAAAAAAPCIBAAAA|1187815|1371|38480|2449280|2449250|Sir|James|Mendez|N|11|12|1943|UZBEKISTAN||James.Mendez@xM.com|2452532| +6192|AAAAAAAAADIBAAAA|1321187|4798|6809|2451716|2451686|Mrs.|Evelyn|Santos|N|25|6|1973|BOLIVIA||Evelyn.Santos@Ep9ciFmF0qB9y.com|2452464| +6193|AAAAAAAABDIBAAAA|607636|1067|14498|2451720|2451690|Dr.|Rose|Lopez|N|5|1|1927|PHILIPPINES||Rose.Lopez@uC.org|2452504| +6194|AAAAAAAACDIBAAAA|1358868|461|29254|2450621|2450591|Sir|Michael|Dunbar|Y|12|4|1936|SAN MARINO||Michael.Dunbar@Tpko60ta7zDx8.edu|2452344| +6195|AAAAAAAADDIBAAAA|378550|4224|38651|2449130|2449100|Dr.|Norman|Lopez|Y|28|2|1976|TUNISIA||Norman.Lopez@yuiYgC.edu|2452356| +6196|AAAAAAAAEDIBAAAA|1423849|3068|41840|2450228|2450198|Mrs.|Rosa|Moss|Y|17|11|1939|GUATEMALA||Rosa.Moss@XRM9maeEy632agM.edu|2452364| +6197|AAAAAAAAFDIBAAAA|1087657|6000|14343|2449171|2449141|Ms.|Arthur|Johnson|Y|11|2|1929|AZERBAIJAN||Arthur.Johnson@D9ETMy.org|2452636| +6198|AAAAAAAAGDIBAAAA|275269|3806|1096|2449484|2449454|Ms.|Christine|Glass|N|7|4|1978|NETHERLANDS||Christine.Glass@XkeXL6PRhk.com|2452358| +6199|AAAAAAAAHDIBAAAA|909904|4989|41227|2449513|2449483|Miss|Michelle|Boone|N|14|7|1991|BENIN||Michelle.Boone@rHuvskJ.com|2452473| +6200|AAAAAAAAIDIBAAAA|1432332|2702|32319|2450575|2450545|Dr.|Deshawn|Jenkins|Y|9|9|1939|LEBANON||Deshawn.Jenkins@94PqAg.com|2452411| +6201|AAAAAAAAJDIBAAAA|468736|2014|44045|2449575|2449545|Dr.|Janet|Taylor|N|10|1|1935|GERMANY||Janet.Taylor@isM81QHZ.edu|2452575| +6202|AAAAAAAAKDIBAAAA|1322712|3047|3382|2449266|2449236|Sir|James|Dean|N|25|12|1959|FRANCE||James.Dean@3CTE.org|2452468| +6203|AAAAAAAALDIBAAAA|1621360|1419|15431|2452598|2452568|Dr.|John|Morgan|N|27|3|1935|UKRAINE||John.Morgan@nQePxb.edu|2452390| +6204|AAAAAAAAMDIBAAAA|1576053|1381|30356|2449977|2449947|Dr.|Frank|Guthrie|Y|14|9|1958|DENMARK||Frank.Guthrie@m.com|2452475| +6205|AAAAAAAANDIBAAAA|411724|5528|5285|2449627|2449597|Mrs.|Kimberlee|Mcneil|Y|9|8|1950|PAPUA NEW GUINEA||Kimberlee.Mcneil@Lqi9G5q1yd9.edu|2452625| +6206|AAAAAAAAODIBAAAA|1538550|4646|30742|2449745|2449715|Miss|Susan|Jordan|N|23|2|1979|MOLDOVA, REPUBLIC OF||Susan.Jordan@s8rvBAt5jP.com|2452425| +6207|AAAAAAAAPDIBAAAA|1585252|5722|22545|2452575|2452545|Miss|April|Shinn|N|21|6|1933|FINLAND||April.Shinn@MfLfziTQKf89.com|2452459| +6208|AAAAAAAAAEIBAAAA|1297171|6781|38838|2450109|2450079|Miss|Julie|Todd|Y|23|9|1964|COMOROS||Julie.Todd@BVKhVcvBOjKppkS2JJ.org|2452620| +6209|AAAAAAAABEIBAAAA|1177647|4776|41505|2450137|2450107|Ms.|Donna|Harrison|N|8|3|1967|FRENCH GUIANA||Donna.Harrison@vKd.edu|2452641| +6210|AAAAAAAACEIBAAAA|1300224|2042|1107|2451372|2451342|Ms.|Rosa|Zaragoza|N|8|5|1964|TUVALU||Rosa.Zaragoza@uGS5IMr.edu|2452641| +6211|AAAAAAAADEIBAAAA|850331|3919|9060|2452170|2452140|Mrs.|Deborah|Thompson|Y|8|11|1965|SOUTH AFRICA||Deborah.Thompson@Bqa3y18a.edu|2452320| +6212|AAAAAAAAEEIBAAAA|1240159|2341|30523|2452109|2452079|Dr.|David|Vaughan|Y|20|10|1985|PERU||David.Vaughan@xEPJHjFeM.com|2452433| +6213|AAAAAAAAFEIBAAAA|73403|449|36295|2449725|2449695|Sir|Howard|Benson|N|14|10|1944|BHUTAN||Howard.Benson@mtYdqXMM6I.org|2452417| +6214|AAAAAAAAGEIBAAAA|1825398||33837||2450626||Ruth||N||9|1975|||Ruth.Kidd@XI2Frdo.edu|| +6215|AAAAAAAAHEIBAAAA|217903|1694|35532|2452295|2452265|Mr.|Charles|Dorris|N|4|2|1970|MONGOLIA||Charles.Dorris@GGSCxRf6XB.com|2452526| +6216|AAAAAAAAIEIBAAAA|1701719|6280|2441|2450183|2450153|Mrs.|Diane|Harrison|N|20|1|1925|FAROE ISLANDS||Diane.Harrison@5.com|2452623| +6217|AAAAAAAAJEIBAAAA|736242|2414|33033|2449445|2449415|Mr.|Mike|Johnson|Y|12|7|1953|MARTINIQUE||Mike.Johnson@1PNbMoCIzxL6M6.com|2452529| +6218|AAAAAAAAKEIBAAAA|178945|1545|2945|2452320|2452290|Dr.|David|Gayle|Y|26|2|1969|LATVIA||David.Gayle@gvMC5UFfLuLBbu8Y.org|2452359| +6219|AAAAAAAALEIBAAAA|1161804|1321|16038|2450726|2450696|Sir|Joseph|Brinkley|Y|10|6|1962|UNITED STATES||Joseph.Brinkley@H0kyOHcms3PgT5.edu|2452576| +6220|AAAAAAAAMEIBAAAA|1816575|4445|9515|2450105|2450075|Dr.|Barbara|Espinoza|N|15|6|1928|MALDIVES||Barbara.Espinoza@AvPomLOCbk2czkn9kpA.org|2452524| +6221|AAAAAAAANEIBAAAA|64491|380|18669|2450418|2450388|Ms.|Janet|Ferguson|Y|1|1|1931|PAPUA NEW GUINEA||Janet.Ferguson@IMDRIOPfmLOB7yL.com|2452619| +6222|AAAAAAAAOEIBAAAA|1094238|3455|22272|2449801|2449771|Dr.|Luigi|Peterson|Y|20|9|1927|GUERNSEY||Luigi.Peterson@aP9CJmMByTzL.org|2452415| +6223|AAAAAAAAPEIBAAAA|725206|5990|8821|2449216|2449186|Mr.|Curtis|Clifford|N|19|3|1930|JERSEY||Curtis.Clifford@K09.edu|2452530| +6224|AAAAAAAAAFIBAAAA|1887936|3834|2885|2452143|2452113|Sir|Art|Kelly|N|22|8|1979|CUBA||Art.Kelly@McFTjl1aUuto.com|2452396| +6225|AAAAAAAABFIBAAAA|952396|5575|9154|2452211|2452181|Ms.|June|Barnes|Y|12|3|1939|TOGO||June.Barnes@dk0y6K.edu|2452643| +6226|AAAAAAAACFIBAAAA|1622885|6458|8878|2450252|2450222|Mr.|William||N|2||1960|||William.Tyler@CID7Qx.com|2452574| +6227|AAAAAAAADFIBAAAA|982326|3619|25359|2449278|2449248|Dr.|Cassie|Evans|Y|3|7|1977|GUYANA||Cassie.Evans@CLr2F.edu|2452475| +6228|AAAAAAAAEFIBAAAA|1857442|1311|1580|2450402|2450372|Sir|Thomas|Williams|N|3|4|1953|BERMUDA||Thomas.Williams@Fq7CED.edu|2452350| +6229|AAAAAAAAFFIBAAAA|1248182|2738|28894|2451673|2451643|Mrs.|Julia|Amundson|N|27|9|1981|BELGIUM||Julia.Amundson@aLv3.com|2452556| +6230|AAAAAAAAGFIBAAAA|1798657|1886|9915|2449457|2449427|Dr.|Kelli|Atkinson|N|22|4|1960|UNITED KINGDOM||Kelli.Atkinson@chsksrzI9Ma4YrP1.com|2452526| +6231|AAAAAAAAHFIBAAAA|1280564|3334|2151|2449258|2449228|Mr.|William|Leblanc|N|6|1|1982|PARAGUAY||William.Leblanc@kYDkGoCBenr.com|2452566| +6232|AAAAAAAAIFIBAAAA|1161905|6542|35626|2450882|2450852|Dr.|Nancy|Little|Y|3|7|1984|VANUATU||Nancy.Little@vIEnOVoKBQIm.org|2452620| +6233|AAAAAAAAJFIBAAAA|732305|1262|34972|2452337|2452307|Sir|Charles|Miller|N|20|5|1956|ALBANIA||Charles.Miller@XdfpCHLd.com|2452490| +6234|AAAAAAAAKFIBAAAA|347386|5323|18507|2450189|2450159|Ms.|Letisha|Jernigan|Y|25|4|1934|R�UNION||Letisha.Jernigan@uLd.edu|2452311| +6235|AAAAAAAALFIBAAAA|373940|1295|3050|2451816|2451786|Sir|Frank|Gunter|Y|12|3|1975|SAINT HELENA||Frank.Gunter@y5EBr2jtsrCdEDuSZ.com|2452397| +6236|AAAAAAAAMFIBAAAA|1810470|3505|41740|2451554|2451524|Sir|Ralph|Betts|N|27|1|1980|IRELAND||Ralph.Betts@M35O8jZu1G.edu|2452522| +6237|AAAAAAAANFIBAAAA|1561515|2071|35827|2452406|2452376|Mr.|Jeffrey|Smith|Y|25|6|1947|FRANCE||Jeffrey.Smith@Gpz8.edu|2452644| +6238|AAAAAAAAOFIBAAAA|1414529|5782|29788|2449363|2449333|Mr.|William|Burnett|N|29|12|1958|GIBRALTAR||William.Burnett@yL6YX.com|2452363| +6239|AAAAAAAAPFIBAAAA|125496|1123|45903|2452561|2452531|Mr.|Gregory|Morgan|Y|7|5|1941|INDIA||Gregory.Morgan@oUaLcs6vRA.com|2452587| +6240|AAAAAAAAAGIBAAAA|1633629|94|12959|2452674|2452644|Mrs.|Debora|Mitchell|Y|28|12|1924|TURKMENISTAN||Debora.Mitchell@4id59RvC4.edu|2452590| +6241|AAAAAAAABGIBAAAA|263559|4713|10714|2451031|2451001|Dr.|Tammy|Downing|Y|1|1|1983|AUSTRALIA||Tammy.Downing@0g91jBQrovOR1xz0p.com|2452354| +6242|AAAAAAAACGIBAAAA|850345|4564|25183|2450870|2450840|Dr.|Sandra|Willis|N|8|8|1987|ERITREA||Sandra.Willis@vQmMo0XMUnRIADUh6.com|2452604| +6243|AAAAAAAADGIBAAAA|1089318|5062|12759|2449806|2449776|Ms.|Gloria|Gonzales|N|21|6|1952|HAITI||Gloria.Gonzales@v4zZ8N.org|2452397| +6244|AAAAAAAAEGIBAAAA|1478418|5811|15157|2450765|2450735|Mrs.|Yolanda|Ayers|Y|4|1|1952|GUINEA||Yolanda.Ayers@4rNIARjrcd.com|2452429| +6245|AAAAAAAAFGIBAAAA|640389|7097|4205|2449538|2449508|Mrs.|Georgia|Beard|N|7|4|1983|AMERICAN SAMOA||Georgia.Beard@tEhg3HXT1QOq6G.com|2452539| +6246|AAAAAAAAGGIBAAAA|986679|5670|661|2450159|2450129|Mrs.|Alice|Rodman|Y|16|9|1939|ZAMBIA||Alice.Rodman@uRcRelpCP.edu|2452505| +6247|AAAAAAAAHGIBAAAA|913346|5821|48696|2451045|2451015|Mrs.|Roslyn|Galloway|Y|8|1|1949|TRINIDAD AND TOBAGO||Roslyn.Galloway@SsN1d8vrJKpkuU.edu|2452360| +6248|AAAAAAAAIGIBAAAA|99698|6121|4144|2449511|2449481|Mr.|James|Perez|N|18|5|1945|COMOROS||James.Perez@B9JOJjnuetmsdQB.com|2452594| +6249|AAAAAAAAJGIBAAAA|939162|5134|42259|2452591|2452561|Ms.|Kathleen|Everett|N|30|10|1928|NICARAGUA||Kathleen.Everett@MIiLi6VM.edu|2452361| +6250|AAAAAAAAKGIBAAAA|1657309|918|46127|2452569|2452539|Dr.|Ryan|Long|N|18|3|1944|GIBRALTAR||Ryan.Long@v6pcmsZM5k.org|2452557| +6251|AAAAAAAALGIBAAAA|28955|1338|28121|2450290|2450260|Miss|Janie|Cummings|N|9|1|1930|NETHERLANDS ANTILLES||Janie.Cummings@DqjDQy.com|2452341| +6252|AAAAAAAAMGIBAAAA|28274|2667|48260|2449215|2449185|Dr.|Alicia|English|N|19|5|1965|AMERICAN SAMOA||Alicia.English@xj.edu|2452305| +6253|AAAAAAAANGIBAAAA|1563057|1502|46347|2452436|2452406|Dr.|Anthony|Greene|N|21|3|1981|ECUADOR||Anthony.Greene@tDZmYDFBXi.com|2452424| +6254|AAAAAAAAOGIBAAAA|1581359|3118|23677|2452323|2452293|Mr.|David|Walker|Y|1|8|1965|KOREA, REPUBLIC OF||David.Walker@iaqAHTE5V8e.com|2452411| +6255|AAAAAAAAPGIBAAAA|1103019|5641|39063|2449491|2449461|Dr.|Larry|Haywood|N|26|4|1932|CAMBODIA||Larry.Haywood@keClo3qHn9C09f.edu|2452400| +6256|AAAAAAAAAHIBAAAA|370829|1526|42270|2450399|2450369|Dr.|James|Kelly|N|15|7|1958|TIMOR-LESTE||James.Kelly@ya04OeAyKgS.org|2452472| +6257|AAAAAAAABHIBAAAA|445405|111|25217|2450230|2450200|Ms.|Athena|Jordon|N|11|4|1953|SWITZERLAND||Athena.Jordon@ahpzRmM59c.com|2452478| +6258|AAAAAAAACHIBAAAA|185874|5002|4083|2452388|2452358|Dr.|Sammy|Taylor|Y|27|4|1977|SUDAN||Sammy.Taylor@dUEC7HViPls.org|2452493| +6259|AAAAAAAADHIBAAAA|1577353|5706|4106|2450012|2449982|Dr.|Willis|Duggan|N|12|11|1929|OMAN||Willis.Duggan@qQkejh0ASGC3.edu|2452418| +6260|AAAAAAAAEHIBAAAA|1189646|1485|36924|2452610|2452580|Mr.|Steven|Perez|Y|28|2|1983|R�UNION||Steven.Perez@dC51iFQa7sL9I8rUh.edu|2452504| +6261|AAAAAAAAFHIBAAAA|1810821|3666|11758|2451891|2451861|Miss|Jessica|Handy|N|24|7|1981|KAZAKHSTAN||Jessica.Handy@X357Y.com|2452383| +6262|AAAAAAAAGHIBAAAA|207778|3323|29621|2450056|2450026|Mrs.|Simone|Murray|Y|23|12|1984|SEYCHELLES||Simone.Murray@fEqF2gZXQVd9.com|2452409| +6263|AAAAAAAAHHIBAAAA|323563|6482|36201|2452104|2452074|Dr.|Karen|Bell|Y|8|2|1964|COMOROS||Karen.Bell@TbcLznT1ruVB.org|2452609| +6264|AAAAAAAAIHIBAAAA|123806|2041|35829|2452042|2452012|Mr.|Elmer|Neal|N|13|8|1978|SEYCHELLES||Elmer.Neal@3LByon4Vam.org|2452593| +6265|AAAAAAAAJHIBAAAA|38662|1311|41207|2450378|2450348|Dr.|Megan|Petty|N|18|6|1966|UKRAINE||Megan.Petty@B5D.com|2452585| +6266|AAAAAAAAKHIBAAAA|1726764|246|46481|2451506|2451476|Mr.|Roger|Wiley|N|27|5|1973|LESOTHO||Roger.Wiley@A.org|2452608| +6267|AAAAAAAALHIBAAAA|1583828|2662|28696|2450488|2450458|Dr.|Margaret|Brazil|Y|23|4|1984|SWAZILAND||Margaret.Brazil@0PBme5cMPJX.org|2452350| +6268|AAAAAAAAMHIBAAAA|||5590||2452578||Arthur|Bragg|||7|1980|HUNGARY|||2452601| +6269|AAAAAAAANHIBAAAA|1309915|2949|46830|2452479|2452449|Ms.|Wanda|Russo|N|6|6|1953|GABON||Wanda.Russo@ZqO.com|2452460| +6270|AAAAAAAAOHIBAAAA|1269733|5156|30932|2450381|2450351|Mr.|Harry|Poole|Y|23|4|1958|LEBANON||Harry.Poole@aJHhyPLka.edu|2452293| +6271|AAAAAAAAPHIBAAAA|625632|493|46507|2451856|2451826|Dr.|Shane|Jones|N|19|5|1983|BAHAMAS||Shane.Jones@G6YSRA.edu|2452520| +6272|AAAAAAAAAIIBAAAA|359398|826|46254|2450304|2450274|Dr.|Michael|Williams|Y|24|2|1936|GUERNSEY||Michael.Williams@YxTpfP6nC0.com|2452497| +6273|AAAAAAAABIIBAAAA|1421433|3439|7892|2452024|2451994|Sir|Patrick|Maki|N|4|7|1957|UNITED ARAB EMIRATES||Patrick.Maki@kMTdMM8gJHJa.edu|2452420| +6274|AAAAAAAACIIBAAAA|240605|417|32982|2450456|2450426|Ms.|Theresa|Bellamy|N|26|2|1945|ALBANIA||Theresa.Bellamy@VYnjQMLN3.org|2452384| +6275|AAAAAAAADIIBAAAA|1570595|37|18326|2450825|2450795|Sir|Michael|Kelly|Y|21|2|1925|CAMBODIA||Michael.Kelly@57VXo5eLcjXj30.com|2452318| +6276|AAAAAAAAEIIBAAAA|926641|4410|37379|2451904|2451874|Mr.|Bobby|Swift|N|18|11|1973|KOREA, REPUBLIC OF||Bobby.Swift@gPD4UMKDNkjeNoMNNQOt.org|2452606| +6277|AAAAAAAAFIIBAAAA|184933|6179|5425|2451390|2451360|Sir|Kenneth|Mcclain|Y|1|12|1991|BENIN||Kenneth.Mcclain@9l4veIistXMtFa.org|2452423| +6278|AAAAAAAAGIIBAAAA|207413|510|2007|2452210|2452180|Ms.|Debra|Perkins|N|26|9|1980|TONGA||Debra.Perkins@5nkM4R.edu|2452454| +6279|AAAAAAAAHIIBAAAA|484916|1970|7024|2449645|2449615|Sir|Steven|Hill|N|20|8|1955|BERMUDA||Steven.Hill@yvsUbEG.com|2452420| +6280|AAAAAAAAIIIBAAAA|989144|1351|13511|2451912|2451882|Mrs.|Tina|Markham|N|22|5|1951|EL SALVADOR||Tina.Markham@5pasJ8P0g.com|2452470| +6281|AAAAAAAAJIIBAAAA|210357|4560|44763|2452198|2452168|Mr.|Alphonse|Sullivan|Y|7|2|1936|JAPAN||Alphonse.Sullivan@YXHB4Z8B15CAy94zC.edu|2452445| +6282|AAAAAAAAKIIBAAAA|1054848|5451|10501|2452132|2452102|Ms.|Susan|Smith|Y|20|11|1933|CAPE VERDE||Susan.Smith@V6Fdqj.com|2452323| +6283|AAAAAAAALIIBAAAA|1696937|6376|33415|2452575|2452545|Ms.|Norma|Speight|Y|20|5|1986|KUWAIT||Norma.Speight@Dlofzxctu1QEu.com|2452581| +6284|AAAAAAAAMIIBAAAA|186290|1171|5652|2451160|2451130|Sir|Jerome|Kramer|Y|9|6|1967|KYRGYZSTAN||Jerome.Kramer@79Ekqba8x9CXD5v.edu|2452436| +6285|AAAAAAAANIIBAAAA|714829|2185|5188|2451462|2451432|Sir|William|Seely|Y|13|2|1927|MYANMAR||William.Seely@ypI.org|2452421| +6286|AAAAAAAAOIIBAAAA|971937|1560|5183|2450980|2450950|Mr.|Jimmy|Morris|N|8|12|1963|AUSTRALIA||Jimmy.Morris@J6OFoFjT.org|2452447| +6287|AAAAAAAAPIIBAAAA|1899441|6135|9629|2450263|2450233|Sir|Richard|Kelly|N|5|3|1989|YEMEN||Richard.Kelly@HJuxIZ6C.com|2452643| +6288|AAAAAAAAAJIBAAAA|1336943|4148|5400|2451981|2451951|Sir|David|Mahoney|Y|21|8|1965|GRENADA||David.Mahoney@eB6.edu|2452592| +6289|AAAAAAAABJIBAAAA|366374|5885|33843|2450587|2450557|Miss|Monica|Guinn|Y|16|7|1979|ANTIGUA AND BARBUDA||Monica.Guinn@66PeS0ebV53.org|2452350| +6290|AAAAAAAACJIBAAAA|611832|194|45403|2449863|2449833|Miss|Griselda|Johnson|Y|1|7|1934|PHILIPPINES||Griselda.Johnson@I.edu|2452456| +6291|AAAAAAAADJIBAAAA|1367526|1218|34510|2451097|2451067|Miss|Mabel|Farr|N|12|12|1988|GAMBIA||Mabel.Farr@gKMBVhCeQb.com|2452353| +6292|AAAAAAAAEJIBAAAA|1877623|847|14118|2450670|2450640|Dr.|Sam|Stuart|Y|8|9|1990|SLOVENIA||Sam.Stuart@fC.edu|2452472| +6293|AAAAAAAAFJIBAAAA|488225|4022|46219|2449440|2449410|Mr.|Gregory|Mathews|Y|21|3|1959|MAURITANIA||Gregory.Mathews@yv6scgGGkvd.org|2452367| +6294|AAAAAAAAGJIBAAAA|1174645|1780|4562|2451662|2451632|Miss|Janet|Thompson|N|13|3|1945|AFGHANISTAN||Janet.Thompson@XJ0p.edu|2452416| +6295|AAAAAAAAHJIBAAAA|692131|2736|37294|2450610|2450580|Miss|Kimberly|Jackson|N|11|5|1951|THAILAND||Kimberly.Jackson@iExSy.com|2452467| +6296|AAAAAAAAIJIBAAAA|1364427|1767|48354|2452606|2452576|Dr.|Jenni|Thompson|N|15|4|1976|FIJI||Jenni.Thompson@bX2RY3sHtVz.com|2452345| +6297|AAAAAAAAJJIBAAAA|177815|2443|5871|2451977|2451947|Dr.|Jacqueline|Bernard|N|5|5|1952|DENMARK||Jacqueline.Bernard@tDBm.edu|2452594| +6298|AAAAAAAAKJIBAAAA|1826711|5983|2455|2449402|2449372|Dr.|Patrick|Toney|Y|3|9|1961|MAURITIUS||Patrick.Toney@G8y1dH9gaRhJQZZs4.edu|2452341| +6299|AAAAAAAALJIBAAAA|575111|1830|30502|2452280|2452250|Ms.|Jasmine|Worth|N|1|2|1941|ISLE OF MAN||Jasmine.Worth@meuGt1St3iKcg5pva.edu|2452575| +6300|AAAAAAAAMJIBAAAA|272572|121|36023|2449344|2449314|Miss|Shawna|Ellis|N|9|2|1980|AMERICAN SAMOA||Shawna.Ellis@0KNeJMOMl8mG.edu|2452333| +6301|AAAAAAAANJIBAAAA|1837582|1403|49271|2450884|2450854|Dr.|Frank|Brantley|N|19|1|1961|CANADA||Frank.Brantley@i.edu|2452404| +6302|AAAAAAAAOJIBAAAA|127655|3439|32070|2449961|2449931|Mrs.|Lois|Wendt|Y|29|7|1969|NEW ZEALAND||Lois.Wendt@dGeSHohQvi1aefcPz.com|2452436| +6303|AAAAAAAAPJIBAAAA|524754|3140|1126|2452160|2452130|Dr.|Barbara|Frazier|N|19|5|1973|SLOVAKIA||Barbara.Frazier@93QuQpLgkGFSkc5uG.edu|2452516| +6304|AAAAAAAAAKIBAAAA|1477304|1231|38405|2449715|2449685|Dr.|Doris|Urbina|Y|17|1|1955|UNITED ARAB EMIRATES||Doris.Urbina@GHG3Z8Th.org|2452523| +6305|AAAAAAAABKIBAAAA|391907|3189|8871|2449069|2449039|Dr.|Amanda|Reed|Y|27|9|1942|BOUVET ISLAND||Amanda.Reed@UVBOiBdduHO.org|2452600| +6306|AAAAAAAACKIBAAAA|1879223|4067|34464|2450990|2450960|Miss|April|Meeks|Y|17|7|1975|GUERNSEY||April.Meeks@MypYke3H.com|2452466| +6307|AAAAAAAADKIBAAAA|1692539|304|16869|2452362|2452332|Ms.|Vivian|Bowers|Y|30|5|1957|GREENLAND||Vivian.Bowers@l.org|2452322| +6308|AAAAAAAAEKIBAAAA|1475607|2030|40453|2451735|2451705|Dr.|Julio|Simmons|N|30|11|1961|BELARUS||Julio.Simmons@q.edu|2452563| +6309|AAAAAAAAFKIBAAAA|662769|5394|14372|2450807|2450777|Sir|Bobby|Arnold|N|14|5|1972|ARGENTINA||Bobby.Arnold@c7KF2jtMqHpNH4H7.edu|2452377| +6310|AAAAAAAAGKIBAAAA|1073552|2364|1929|2450714|2450684|Dr.|Angelica|Cooley|Y|4|2|1931|MONTENEGRO||Angelica.Cooley@bYRgV.com|2452323| +6311|AAAAAAAAHKIBAAAA|124218|2027|11036|2452105|2452075|Dr.|Terry|Wagner|N|1|8|1981|ROMANIA||Terry.Wagner@DzIYKQl1urXiDM.edu|2452378| +6312|AAAAAAAAIKIBAAAA|199069|6977|48485|2452608|2452578|Mrs.|Glenda|Campbell|Y|22|7|1928|HONDURAS||Glenda.Campbell@5hlvdfIZ9FiiQ.edu|2452411| +6313|AAAAAAAAJKIBAAAA|711888|2156|33624|2450543|2450513|Dr.|Eugene|Clark|Y|11|8|1925|INDONESIA||Eugene.Clark@E2.com|2452296| +6314|AAAAAAAAKKIBAAAA|62844|750|42209|2450585|2450555|Mr.|Bernard|Ainsworth|N|8|6|1979|GIBRALTAR||Bernard.Ainsworth@HAFqHvmlL7mi.com|2452543| +6315|AAAAAAAALKIBAAAA|730548|5537|11773|2450565|2450535|Dr.|Bryan|Sowell|N|25|9|1933|BURUNDI||Bryan.Sowell@b.org|2452480| +6316|AAAAAAAAMKIBAAAA|1912759|6224|36489|2451183|2451153|Mrs.|Deanna|Johnson|N|13|12|1961|NEPAL||Deanna.Johnson@qE92HHg.edu|2452497| +6317|AAAAAAAANKIBAAAA|769316|3547|6578|2449136|2449106|Sir|Charles|Brookins|Y|15|1|1936|LESOTHO||Charles.Brookins@btPDmUqv1YiQ.edu|2452507| +6318|AAAAAAAAOKIBAAAA|1520603|895|387|2452152|2452122|Miss|Marie|Deal|Y|30|9|1949|ECUADOR||Marie.Deal@8MaBO.edu|2452382| +6319|AAAAAAAAPKIBAAAA|652816|1256|2449|2451118|2451088|Mrs.|Margaret|Wagner|N|26|10|1958|MAYOTTE||Margaret.Wagner@qdbaPvE3Zp.org|2452379| +6320|AAAAAAAAALIBAAAA|1834670|4802|40887|2449750|2449720|Dr.|Michael|Price|Y|27|7|1980|ROMANIA||Michael.Price@cZllQkjuQtAuh9l4Ir.com|2452490| +6321|AAAAAAAABLIBAAAA|633587|3645|49350|2449889|2449859|Dr.|John|Madden|Y|24|6|1940|NORWAY||John.Madden@22dnxS.edu|2452605| +6322|AAAAAAAACLIBAAAA|1688250|5595|43293|2451680|2451650|Mrs.|Barbara|Bruce|Y|10|5|1948|MEXICO||Barbara.Bruce@2qiFA6GT8B.edu|2452283| +6323|AAAAAAAADLIBAAAA|1791267|22|35624|2450461|2450431|Sir|Joseph|Walker|N|2|2|1979|MONACO||Joseph.Walker@ojvsSVaGhn.edu|2452373| +6324|AAAAAAAAELIBAAAA|1601930|1839|17301|2450307|2450277|Mr.|Jimmy|Ludwig|Y|19|3|1985|GUYANA||Jimmy.Ludwig@nBa3pmpR4Xy.com|2452617| +6325|AAAAAAAAFLIBAAAA|1077205|5483|16514|2450323|2450293|Miss|Anthony|Bryan|Y|10|12|1980|PERU||Anthony.Bryan@SEebR7oYv.org|2452507| +6326|AAAAAAAAGLIBAAAA|547473|6622|9724|2451157|2451127|Mrs.|Catherine|Hagen|N|29|7|1974|INDIA||Catherine.Hagen@AUYc12HmNCAzY.edu|2452302| +6327|AAAAAAAAHLIBAAAA|752876|4744|48259|2451313|2451283|Sir|Charles|Bailey|Y|3|7|1976|ANGOLA||Charles.Bailey@3xvFjdNQru.edu|2452452| +6328|AAAAAAAAILIBAAAA|706098|31|2717|2450739|2450709|Miss|Melissa|Bussey|N|28|10|1963|OMAN||Melissa.Bussey@g6UlzDcobxGlkT7.edu|2452329| +6329|AAAAAAAAJLIBAAAA|1149819|6593|120|2450704|2450674|Sir|Lionel|Ramey|N|12|1|1959|TURKMENISTAN||Lionel.Ramey@RcCBL.org|2452615| +6330|AAAAAAAAKLIBAAAA|805131|1043|48602|2451730|2451700|Mr.|Tim|Adams|Y|23|5|1968|DOMINICA||Tim.Adams@m6SFEq4Ztp1H9I.com|2452451| +6331|AAAAAAAALLIBAAAA|1356363|1439|38162|2451776|2451746|Mr.|Keith|Taylor|Y|18|7|1971|SIERRA LEONE||Keith.Taylor@QVytJnnVfb.com|2452389| +6332|AAAAAAAAMLIBAAAA|||38956|2451677|2451647||||Y|19|5|||||| +6333|AAAAAAAANLIBAAAA|188231|1685|26386|2451459|2451429|Mrs.|Katherine|Teal|N|30|4|1990|MALI||Katherine.Teal@n.com|2452508| +6334|AAAAAAAAOLIBAAAA|502251|5546|12841|2451709|2451679|Sir|John|Miller|Y|3|11|1949|MEXICO||John.Miller@bbxONOYVH.edu|2452387| +6335|AAAAAAAAPLIBAAAA|1389202|578|44707|2450636|2450606|Dr.|Richard|Williams|N|27|3|1947|PERU||Richard.Williams@SM4PnFUPNtkT6IQ.org|2452484| +6336|AAAAAAAAAMIBAAAA|814585|5070|22802|2449895|2449865|Miss|Susan|Seymour|N|5|8|1934|UNITED STATES||Susan.Seymour@y0.edu|2452318| +6337|AAAAAAAABMIBAAAA|357002|1383|4211|2449260|2449230|Dr.|John|Perkins|N|22|5|1985|GAMBIA||John.Perkins@T8.edu|2452319| +6338|AAAAAAAACMIBAAAA||4996|47645|||||Greene|N||10||GUYANA|||2452361| +6339|AAAAAAAADMIBAAAA|1531127|7182|4837|2449827|2449797|Dr.|Latonya|Joyce|N|9|6|1962|NAMIBIA||Latonya.Joyce@PPCtXQgL4I.com|2452502| +6340|AAAAAAAAEMIBAAAA|946496|2808|20320|2451551|2451521|Miss|Judy|Cruz|Y|4|4|1942|NORWAY||Judy.Cruz@s3G.org|2452377| +6341|AAAAAAAAFMIBAAAA|72903|6748|32993|2452031|2452001|Sir|Frank|Clark|Y|2|11|1974|NAMIBIA||Frank.Clark@IqoxEEXZ.org|2452579| +6342|AAAAAAAAGMIBAAAA|1230978|3806|15434|2452374|2452344|Mr.|Ricky|Russell|N|13|12|1963|ISLE OF MAN||Ricky.Russell@ijh0b7IFsbDXmO4Qh.org|2452435| +6343|AAAAAAAAHMIBAAAA|77121|3502|10683|2450369|2450339|Sir|Lewis|Adams|N|2|3|1939|ESTONIA||Lewis.Adams@A37uxADvkzDJTFrDc.com|2452492| +6344|AAAAAAAAIMIBAAAA|1638854|3048|33995|2451240|2451210|Sir|Claudio|Krieger|Y|8|7|1976|CAPE VERDE||Claudio.Krieger@G2Ov4BlbgG4P.org|2452283| +6345|AAAAAAAAJMIBAAAA|1755411||13049|2449808|2449778||Florine||N|28||1949|RWANDA|||| +6346|AAAAAAAAKMIBAAAA|1533310||45801||||David|Archibald|Y|23|||INDONESIA|||2452390| +6347|AAAAAAAALMIBAAAA|431901|5483|38665|2449401|2449371|Ms.|Valerie|Sorrell|N|14|10|1970|C�TE D'IVOIRE||Valerie.Sorrell@pdzO.com|2452585| +6348|AAAAAAAAMMIBAAAA|989670|5461|39652|2450231|2450201|Miss|Gertrude|Land|Y|23|1|1953|OMAN||Gertrude.Land@t2Yzb3xPfI.org|2452524| +6349|AAAAAAAANMIBAAAA|1747109|6684|21012|2452344|2452314|Mr.|Gordon|Castaneda|N|1|10|1935|ALGERIA||Gordon.Castaneda@haMB1JFJVtLR.org|2452636| +6350|AAAAAAAAOMIBAAAA|1730555|7008|4614|2450902|2450872|Mr.|Gerald|Rodgers|N|18|6|1955|MALAYSIA||Gerald.Rodgers@ND7Bd.org|2452390| +6351|AAAAAAAAPMIBAAAA|1598327|5642|41004|2450920|2450890|Mr.|Landon|Madden|N|24|3|1974|SUDAN||Landon.Madden@3ovFSx.com|2452299| +6352|AAAAAAAAANIBAAAA|1039162|6119|276|2449638|2449608|Miss|Luisa|Russell|Y|12|3|1943|SINGAPORE||Luisa.Russell@ZrgyPq.edu|2452583| +6353|AAAAAAAABNIBAAAA|1040899|1099|48899|2451854|2451824|Mr.|Kenneth|Byers|N|9|6|1961|CHRISTMAS ISLAND||Kenneth.Byers@gT.org|2452621| +6354|AAAAAAAACNIBAAAA|||25375||||Stephanie|||||1989||||| +6355|AAAAAAAADNIBAAAA|403452|2888|6724|2452335|2452305|Dr.|Odette|Gonzalez|N|7|8|1948|PANAMA||Odette.Gonzalez@OZ.org|2452556| +6356|AAAAAAAAENIBAAAA|1708634|3798|27715|2450695|2450665|Sir|Edwin|Hoover|N|10|6|1935|UNITED ARAB EMIRATES||Edwin.Hoover@acx4XstB86SSN.edu|2452574| +6357|AAAAAAAAFNIBAAAA|27358|5435|35065|2450670|2450640|Mrs.|Alexander|Walsh|Y|11|10|1989|CAYMAN ISLANDS||Alexander.Walsh@HGvoeA4ALLi.org|2452571| +6358|AAAAAAAAGNIBAAAA||1909|22165|2452235|2452205|||Hahn|||12|1983||||| +6359|AAAAAAAAHNIBAAAA|129|2224|5919|2451406|2451376|Mr.|Johnnie|Horton|N|14|10|1934|TOKELAU||Johnnie.Horton@sddUyctK8HFOHE7iGd.org|2452453| +6360|AAAAAAAAINIBAAAA|621692|156|24220|2449876|2449846|Mrs.|Juanita|Dyson|Y|9|4|1966|TOKELAU||Juanita.Dyson@q7Nk7.edu|2452455| +6361|AAAAAAAAJNIBAAAA|1514663|6642|32621|2449635|2449605|Sir|William|Perry|N|14|2|1935|BURKINA FASO||William.Perry@u0Cf8tOR.com|2452572| +6362|AAAAAAAAKNIBAAAA|355886|3558|13630|2451517|2451487|Mrs.|Kathleen|Fraser|N|8|2|1974|MAURITIUS||Kathleen.Fraser@28VDUV9ac.com|2452427| +6363|AAAAAAAALNIBAAAA|1392986|1848|4710|2452630|2452600|Dr.|Willie|Pierre|Y|24|6|1943|GUATEMALA||Willie.Pierre@xPb.org|2452502| +6364|AAAAAAAAMNIBAAAA|810770|4|34438|2451491|2451461|Miss|Helen|Perez|N|10|4|1944|PHILIPPINES||Helen.Perez@V.org|2452480| +6365|AAAAAAAANNIBAAAA|122637|1400|24210|2449830|2449800|Dr.|Lois|Evans|Y|10|11|1954|WALLIS AND FUTUNA||Lois.Evans@I4EaFVMu2nDa2NqmJ.com|2452463| +6366|AAAAAAAAONIBAAAA|164388|822|6856|2451766|2451736|Mrs.|Sherry|Newton|Y|4|10|1980|SINGAPORE||Sherry.Newton@5iMtk1NjvB7ezvrTf.org|2452479| +6367|AAAAAAAAPNIBAAAA|721686|4641|30396|2450556|2450526|Dr.|Willis|Roberts|N|10|2|1990|ROMANIA||Willis.Roberts@j.edu|2452608| +6368|AAAAAAAAAOIBAAAA|27835|2908|41934|2450536|2450506|Miss|Thuy|Cole|Y|12|5|1978|KAZAKHSTAN||Thuy.Cole@ud56s.edu|2452408| +6369|AAAAAAAABOIBAAAA|129073|7008|28943|2452329|2452299|Ms.|Alice|Joiner|N|24|1|1928|NEW ZEALAND||Alice.Joiner@yIb.org|2452333| +6370|AAAAAAAACOIBAAAA|1347056|6498|46693|2449722|2449692|Ms.|Bernadine|Macias|Y|27|2|1958|SERBIA||Bernadine.Macias@PU.com|2452425| +6371|AAAAAAAADOIBAAAA|1516301|2470|44686|2450983|2450953|Mrs.|Anthony|Lerner|Y|15|2|1990|INDONESIA||Anthony.Lerner@HRoReZDON3Rzl.com|2452382| +6372|AAAAAAAAEOIBAAAA|685694|4063|45585|2449876|2449846|Mr.|Donald|Martin|N|18|10|1975|POLAND||Donald.Martin@OPJtDnujlt.org|2452289| +6373|AAAAAAAAFOIBAAAA|1301573|6484|27795|2451527|2451497|Sir|Alan|Navarrete|Y|18|4|1976|BELARUS||Alan.Navarrete@F5VZyeZF3QDiS2K2P.com|2452379| +6374|AAAAAAAAGOIBAAAA|258271|2731|18791|2449220|2449190|Ms.|Elizabeth|Smith|N|22|7|1926|ALAND ISLANDS||Elizabeth.Smith@o2lcklha1B5kmttF.org|2452559| +6375|AAAAAAAAHOIBAAAA|484719|6324|11082|2451414|2451384|Dr.|Philip|Hatton|N|27|5|1935|BELGIUM||Philip.Hatton@hZEuVXy.org|2452533| +6376|AAAAAAAAIOIBAAAA||5054|13186|||Mrs.|Crystal|Reyes|N|17|6|||||2452466| +6377|AAAAAAAAJOIBAAAA|995226|4314|4202|2449759|2449729|Dr.|Orlando|Ford|N|16|10|1992|ARUBA||Orlando.Ford@REiiQdD8Nh2H.com|2452502| +6378|AAAAAAAAKOIBAAAA|792498|1458|38371|2452610|2452580|Mrs.|Tonya|Pittman|N|20|9|1983|VENEZUELA||Tonya.Pittman@dT.edu|2452603| +6379|AAAAAAAALOIBAAAA|1471667|6427|18434|2451100|2451070|Mrs.|Andrea|Collins|Y|7|5|1939|SWEDEN||Andrea.Collins@vbJnH4YP.edu|2452326| +6380|AAAAAAAAMOIBAAAA|893638|3563|17841|2450222|2450192|Ms.|Sindy|Mendez|Y|7|9|1952|KUWAIT||Sindy.Mendez@ct6cVqTUHn3v1OqQ3d.org|2452345| +6381|AAAAAAAANOIBAAAA|109886|1139|23145|2449895|2449865|Dr.|Scott|Quinn|N|13|2|1973|SWAZILAND||Scott.Quinn@Z1umoac.com|2452618| +6382|AAAAAAAAOOIBAAAA|235877|3531|13938|2450207|2450177|Mr.|Christopher|Blevins|Y|11|5|1950|YEMEN||Christopher.Blevins@yV4C9y5e.org|2452631| +6383|AAAAAAAAPOIBAAAA|381094|5049|33404|2449843|2449813|Dr.|Colin|Chavez|Y|10|11|1987|YEMEN||Colin.Chavez@drFzgfs.com|2452606| +6384|AAAAAAAAAPIBAAAA|339997|1649|12548|2452092|2452062|Mrs.|Berta|Wheeler|Y|25|11|1959|JORDAN||Berta.Wheeler@tiYBbUvl.com|2452557| +6385|AAAAAAAABPIBAAAA|1561889|57|6978|2449484|2449454|Mr.|Christopher|Sands|N|12|4|1950|UGANDA||Christopher.Sands@NJd0F27.com|2452371| +6386|AAAAAAAACPIBAAAA|1740366|3395|32424|2450736|2450706|Sir|James|Hughes|N|2|3|1978|UGANDA||James.Hughes@NXb6Cq.edu|2452292| +6387|AAAAAAAADPIBAAAA|1719131|4079|18627|2449115|2449085|Mr.|Richard|Jacobs|N|19|2|1949|SUDAN||Richard.Jacobs@P3nYMF4uDZZjtAC.org|2452354| +6388|AAAAAAAAEPIBAAAA|1491657|4695|18533|2449282|2449252|Dr.|Michael|Wilson|N|9|8|1969|RWANDA||Michael.Wilson@BRdA0Z4AP3f5hMyb.com|2452553| +6389|AAAAAAAAFPIBAAAA|75203|1081|19759|2449302|2449272|Dr.|John|Harris|N|1|8|1934|ANDORRA||John.Harris@QIQmIdOI06.org|2452522| +6390|AAAAAAAAGPIBAAAA|661402|227|22299|2450987|2450957|Mr.|Chester|Gordon|Y|26|12|1992|SWEDEN||Chester.Gordon@8p3csHkPHf.edu|2452410| +6391|AAAAAAAAHPIBAAAA|898360|5632|11257|2452129|2452099|Dr.|Louis|Lopez|N|4|5|1981|BAHAMAS||Louis.Lopez@AT0IQn5.org|2452389| +6392|AAAAAAAAIPIBAAAA|1659828|5036|10644|2452066|2452036|Sir|George|Lewis|N|17|4|1950|TOGO||George.Lewis@VA1y0f.com|2452335| +6393|AAAAAAAAJPIBAAAA|546893|449|32852|2450419|2450389|Sir|Corey||||2|1987||||| +6394|AAAAAAAAKPIBAAAA|677074|339|14051|2449848|2449818|Ms.|Marilyn|Moore|Y|18|12|1951|LATVIA||Marilyn.Moore@VpGZdl7koVoP.com|2452420| +6395|AAAAAAAALPIBAAAA|847705|4752|31752|2451407|2451377|Dr.|Victoria|Bautista|N|31|12|1925|TIMOR-LESTE||Victoria.Bautista@44HdESQyvz.com|2452589| +6396|AAAAAAAAMPIBAAAA|1404201|4471|12294|2449269|2449239|Ms.|Angela|Moore|N|19|5|1939|ICELAND||Angela.Moore@YJY67OkNFricTH7.edu|2452464| +6397|AAAAAAAANPIBAAAA|1697548|7131|43467|2449193|2449163|Mrs.|Thelma|Cano|N|4|3|1967|FRENCH GUIANA||Thelma.Cano@pxAlkL1TldkQ6.org|2452402| +6398|AAAAAAAAOPIBAAAA|279372|2878|14031|2449858|2449828|Dr.|Martha|Poore|Y|4|10|1949|MALTA||Martha.Poore@pja4CxsOC5Cttj9OBB.edu|2452517| +6399|AAAAAAAAPPIBAAAA|332256|4666|25876|2450195|2450165|Miss|Betty|Porter|Y|7|2|1988|AZERBAIJAN||Betty.Porter@V.edu|2452545| +6400|AAAAAAAAAAJBAAAA|368694|4221|41748|2449117|2449087|Sir|Frank|Johnson|N|5|1|1936|MONACO||Frank.Johnson@aRaLLzKen.com|2452323| +6401|AAAAAAAABAJBAAAA|510597|2266|1888|2451633|2451603|Dr.|Marshall|Wilson|N|12|1|1977|ESTONIA||Marshall.Wilson@qn6S7tjoU7.edu|2452540| +6402|AAAAAAAACAJBAAAA|1880205|1927|2802|2449512|2449482|Mr.|Samuel|Terry|N|22|8|1974|LESOTHO||Samuel.Terry@N8U7PxY.org|2452311| +6403|AAAAAAAADAJBAAAA|1679389|1898|42807|2449698|2449668|Mr.|Shawn|Mclemore|Y|14|5|1967|FRANCE||Shawn.Mclemore@P.com|2452492| +6404|AAAAAAAAEAJBAAAA|1321602|4758|23961|2450934|2450904|Mr.|Roberto|Wood|Y|2|3|1983|ERITREA||Roberto.Wood@JX2il0XCMvNkoe.edu|2452327| +6405|AAAAAAAAFAJBAAAA|1766486|4042|27551|2449333|2449303|Mr.|Harold|Jones|Y|21|9|1933|CHINA||Harold.Jones@1AI.edu|2452352| +6406|AAAAAAAAGAJBAAAA|1199197||3195||2450865||James||Y|27|8|||||2452451| +6407|AAAAAAAAHAJBAAAA|1004800|1224|9648|2449192|2449162|Dr.|Jesus|Perez|Y|7|2|1976|MALTA||Jesus.Perez@p23GAm1Zd0.edu|2452612| +6408|AAAAAAAAIAJBAAAA|1873179|2506|27433|2451621|2451591|Miss|Charlene|James|Y|14|4|1992|SLOVAKIA||Charlene.James@9CtE24BUeNA6.edu|2452406| +6409|AAAAAAAAJAJBAAAA|24232|2237|29889|2451502|2451472|Mr.|Evan|Tran|N|2|4|1924|GUINEA-BISSAU||Evan.Tran@UVh.edu|2452287| +6410|AAAAAAAAKAJBAAAA|673661|2378|23048|2452010|2451980|Dr.|Star|Johnson|N|11|10|1959|MYANMAR||Star.Johnson@CClaxQhUyTr62X.com|2452632| +6411|AAAAAAAALAJBAAAA||3876|5377|2450283||Miss|Vicki||N|24|8||VANUATU|||2452403| +6412|AAAAAAAAMAJBAAAA|978995|4299|2803|2449954|2449924|Sir|Eric|Armstrong|Y|18|6|1939|MADAGASCAR||Eric.Armstrong@Ybs9se0rY3Lfn4fj.edu|2452314| +6413|AAAAAAAANAJBAAAA|129474|686|48345|2450410|2450380|Sir|Chris|Mcclure|Y|4|1|1926|PORTUGAL||Chris.Mcclure@zam2OSerP3XuVBftTG.com|2452640| +6414|AAAAAAAAOAJBAAAA|739390|2883|17976|2451562|2451532|Ms.|Denise|Jones|N|16|10|1952|CYPRUS||Denise.Jones@EKhjR8AbSH4y.org|2452370| +6415|AAAAAAAAPAJBAAAA|582085|6336|49787|2452513|2452483|Mr.|Robert|White|N|10|1|1967|COMOROS||Robert.White@2bc.org|2452477| +6416|AAAAAAAAABJBAAAA|1808521|2795|13869|2452591|2452561|Sir|Eugene|Juarez|N|1|1|1948|RUSSIAN FEDERATION||Eugene.Juarez@s.org|2452398| +6417|AAAAAAAABBJBAAAA|765977|6701|19072|2449848|2449818|Miss|Eleanor|Billingsley|Y|1|9|1990|AFGHANISTAN||Eleanor.Billingsley@4U4heFRhs8.com|2452615| +6418|AAAAAAAACBJBAAAA|1019974|173|24772|2449929|2449899|Mrs.|Kristal|Mcnamara|N|8|1|1935|CANADA||Kristal.Mcnamara@lDb2kFRuq44.edu|2452576| +6419|AAAAAAAADBJBAAAA|362819|786|11153|2452332|2452302|Sir|Eric|Dennis|Y|2|12|1970|COSTA RICA||Eric.Dennis@j5.com|2452329| +6420|AAAAAAAAEBJBAAAA|609295|1488|44034|2450724|2450694|Dr.|Max|Sumner|N|9|2|1957|BURKINA FASO||Max.Sumner@al.org|2452333| +6421|AAAAAAAAFBJBAAAA|1635608|1023|17285|2452484|2452454|Dr.|Dennis|Grant|N|11|12|1954|ITALY||Dennis.Grant@GDRvy9M28n5ht7LD.org|2452467| +6422|AAAAAAAAGBJBAAAA|1238016|4621|15673|2451316|2451286|Sir|James|Keller|N|28|6|1931|HONG KONG||James.Keller@IbutLhcr9hfT1oXbDn.com|2452297| +6423|AAAAAAAAHBJBAAAA|32297|1010|40570|2449118|2449088|Dr.|Linda|Monk|Y|12|11|1932|AMERICAN SAMOA||Linda.Monk@o.edu|2452614| +6424|AAAAAAAAIBJBAAAA||790|4668||2450798|Sir|||Y|8||||||| +6425|AAAAAAAAJBJBAAAA|1913438|6760|20835|2451139|2451109|Mrs.|Geraldine|Allen|Y|13|11|1976|NIGERIA||Geraldine.Allen@jJSEuhnAvaui.com|2452399| +6426|AAAAAAAAKBJBAAAA|1651260|6364|11515|2452618|2452588|Sir|Harold|Jackson|N|29|3|1982|AUSTRALIA||Harold.Jackson@ETbAd6rtMyQRBX2y.org|2452339| +6427|AAAAAAAALBJBAAAA|536783|4668|42106|2449256|2449226|Dr.|Jaime|Barnes|Y|11|8|1970|JAPAN||Jaime.Barnes@Mylf7OJ6HjAjPQyoU.edu|2452333| +6428|AAAAAAAAMBJBAAAA|727179|727|2628|2451735|2451705|Miss|Harmony|Miller|Y|5|10|1929|FRENCH POLYNESIA||Harmony.Miller@SLyE6oznaE.org|2452469| +6429|AAAAAAAANBJBAAAA|127708|2304|44100||2449494|||Nichols||||1984|||Bertha.Nichols@n8VY31.org|| +6430|AAAAAAAAOBJBAAAA|768603|4547|13644|2449633|2449603|Sir|David|Smith|Y|22|8|1928|SAUDI ARABIA||David.Smith@6DlGFTTMJT5daJ.edu|2452549| +6431|AAAAAAAAPBJBAAAA|320811|2972|16333|2450544|2450514|Sir|David|Tipton|N|16|2|1935|DENMARK||David.Tipton@T3KPxszX.edu|2452504| +6432|AAAAAAAAACJBAAAA|929365|5568|15371|2452662|2452632|Ms.|Kimberlee|Collins|Y|28|6|1948|ANGOLA||Kimberlee.Collins@UxdmYVfUD.org|2452464| +6433|AAAAAAAABCJBAAAA|275629|5526|49823|2450770|2450740|Sir|James|Clark|N|14|6|1939|CROATIA||James.Clark@UnxS9PzzFk.edu|2452304| +6434|AAAAAAAACCJBAAAA|30367|4446|48125|2449881|2449851|Dr.|Melina|Bowman|Y|12|2|1987|AUSTRIA||Melina.Bowman@JLrfVI0PHRI6oMs.org|2452385| +6435|AAAAAAAADCJBAAAA|921294|5453|14459|2452020|2451990|Dr.|Ralph|Griffin|Y|7|7|1983|GAMBIA||Ralph.Griffin@E7bRFb4ekU.org|2452641| +6436|AAAAAAAAECJBAAAA|672785|4358|45578|2449832|2449802|Sir|Jerry|Gipson|N|3|6|1973|WESTERN SAHARA||Jerry.Gipson@2JgGTPcqklC1.org|2452394| +6437|AAAAAAAAFCJBAAAA|552011|2267|24076|2450088|2450058|Miss|Rebecca|Copeland|Y|16|10|1990|QATAR||Rebecca.Copeland@zUdIZaMTDggV.org|2452610| +6438|AAAAAAAAGCJBAAAA|1136186|6759|2932|2452255|2452225|Ms.|Fannie|Dowell|N|5|4|1929|NICARAGUA||Fannie.Dowell@I.org|2452381| +6439|AAAAAAAAHCJBAAAA|1415238|6757|41876|2449822|2449792|Dr.|John|Hussey|Y|10|6|1939|ECUADOR||John.Hussey@allFUepKmsPvI.org|2452416| +6440|AAAAAAAAICJBAAAA|682839|1900|20961|2452413|2452383|Dr.|Katherine|Avery|N|31|7|1991|MARTINIQUE||Katherine.Avery@z4BAilQhXo8.org|2452406| +6441|AAAAAAAAJCJBAAAA|969495|1522|13502|2451123|2451093|Mr.|Horace|Washington|Y|1|2|1937|RUSSIAN FEDERATION||Horace.Washington@3VFhn.com|2452455| +6442|AAAAAAAAKCJBAAAA|1175|650|19725|2450354|2450324|Miss|Socorro|Foster|Y|30|3|1988|ISLE OF MAN||Socorro.Foster@uNgcExHlG2kA.edu|2452470| +6443|AAAAAAAALCJBAAAA|816892|3539|28186|2450733|2450703|Mrs.|Regina|Ingram|N|22|4|1961|POLAND||Regina.Ingram@GFAg3NLhXxcIj66x.edu|2452619| +6444|AAAAAAAAMCJBAAAA|1031543|3495|27000||2451421||Domingo|Kaminski|Y|8|8|||||2452604| +6445|AAAAAAAANCJBAAAA|1548152|1665|42378|2450750|2450720|Mr.|Richard|Allen|Y|15|2|1979|PAKISTAN||Richard.Allen@DuSqorobG38bYlkaK.edu|2452360| +6446|AAAAAAAAOCJBAAAA|289196|6382|39521|2451470|2451440|Sir|Samuel|Gannon|Y|2|6|1990|MONACO||Samuel.Gannon@Jz3e4lX002z1ZN7MBa.org|2452459| +6447|AAAAAAAAPCJBAAAA|1436636|4114|22039|2451753|2451723|Dr.|Marian|Owens|N|14|7|1935|BOTSWANA||Marian.Owens@640BI50b5y.com|2452393| +6448|AAAAAAAAADJBAAAA|752309|6974|37286|2452018|2451988|Sir|Andy|Cleveland|Y|28|6|1941|TOKELAU||Andy.Cleveland@uZfzpqE3ummuB8yX.com|2452442| +6449|AAAAAAAABDJBAAAA|1425314|5487|36060|2449649|2449619|Sir|Jesse|Hill|Y|28|1|1953|RUSSIAN FEDERATION||Jesse.Hill@O72Md.com|2452533| +6450|AAAAAAAACDJBAAAA|1646910|6008|30259|2452233|2452203|Ms.|Betty|Cruz|N|15|1|1969|CHINA||Betty.Cruz@DDpGXS56M.edu|2452556| +6451|AAAAAAAADDJBAAAA|5138|7180|6775|2450932|2450902|Miss|Ann|Lucas|N|30|12|1949|IRAQ||Ann.Lucas@NsZI4s.edu|2452363| +6452|AAAAAAAAEDJBAAAA|1336669|268|28314|2449662|2449632|Dr.|Doris|Barrett|Y|29|6|1955|KYRGYZSTAN||Doris.Barrett@Uul237zDJtN.edu|2452334| +6453|AAAAAAAAFDJBAAAA|941549|6894|4592|2449520|2449490|Dr.|Shawn|Welch|N|28|10|1983|EL SALVADOR||Shawn.Welch@b1IRMf9XHz46x.org|2452296| +6454|AAAAAAAAGDJBAAAA|846075|1647|33972|2452514|2452484|Sir|Lawrence|Torrence|Y|20|1|1925|KOREA, REPUBLIC OF||Lawrence.Torrence@660toX.org|2452472| +6455|AAAAAAAAHDJBAAAA|908490|6391|3492|2451068|2451038|Sir|Grady|Webb|Y|19|12|1938|IRELAND||Grady.Webb@cHKpx04GAcnpr.com|2452490| +6456|AAAAAAAAIDJBAAAA|1459865|5131|21150|2449857|2449827|Ms.|Lorraine|Pringle|Y|8|1|1956|BOLIVIA||Lorraine.Pringle@F7bHENFIGrV1N8.com|2452545| +6457|AAAAAAAAJDJBAAAA|1238099|4631|30763|2451930|2451900|Dr.|Melanie|Flores|Y|6|12|1968|CAYMAN ISLANDS||Melanie.Flores@5yxp.com|2452517| +6458|AAAAAAAAKDJBAAAA|418408|6036|2864|2452533|2452503|Sir|Willard|Lynch|Y|10|1|1949|SAINT HELENA||Willard.Lynch@GGaB3LX9LbP6Xk.edu|2452380| +6459|AAAAAAAALDJBAAAA|1284959|485|32058|2451788|2451758|Mrs.|Eva|Lopez|Y|25|11|1963|SAUDI ARABIA||Eva.Lopez@kzPkM8QmyKUy.edu|2452631| +6460|AAAAAAAAMDJBAAAA|466952|3279|18006|2449255|2449225|Miss|Jodi|Cass|N|11|4|1963|YEMEN||Jodi.Cass@rHA9Mo.com|2452456| +6461|AAAAAAAANDJBAAAA|164014|5263|14065|2451232|2451202|Mrs.|Pamela|Scott|N|8|1|1977|DOMINICA||Pamela.Scott@a.edu|2452437| +6462|AAAAAAAAODJBAAAA|280081|6306|29778|2451087|2451057|Miss|Wendy|Vargas|N|7|5|1958|BANGLADESH||Wendy.Vargas@ih0fK3qIkRTy5K.com|2452568| +6463|AAAAAAAAPDJBAAAA|1655208|1797|23986|2449253|2449223|Dr.|Shaun|Meek|N|4|7|1976|BRAZIL||Shaun.Meek@o3.edu|2452616| +6464|AAAAAAAAAEJBAAAA|272560|1807|24207|2452483|2452453|Miss|Virginia|Wolf|Y|14|8|1930|MONTENEGRO||Virginia.Wolf@5lRjNOEtT7cCxc.org|2452598| +6465|AAAAAAAABEJBAAAA|290319|2378|27237|2451258|2451228|Ms.|Neva|Chen|N|13|6|1964|MYANMAR||Neva.Chen@ikQ4SVYUjBjsimL.org|2452627| +6466|AAAAAAAACEJBAAAA|582903|4825|25100|2449319|2449289|Dr.|Jose|Bryant|N|20|11|1953|ARUBA||Jose.Bryant@YNCXU9vfDrqI.org|2452578| +6467|AAAAAAAADEJBAAAA|1828775|3450|29372|2452222|2452192|Mr.|Charles|Sheppard|Y|22|6|1944|BERMUDA||Charles.Sheppard@jeYyG329ezM.com|2452367| +6468|AAAAAAAAEEJBAAAA|1492625|4717|6856|2451754|2451724|Dr.|Wallace|Harris|N|1|12|1953|KYRGYZSTAN||Wallace.Harris@H0neBYVGyj4o.org|2452503| +6469|AAAAAAAAFEJBAAAA|960981|396|3811|2450402|2450372|Miss|Marie|Calderon|N|27|7|1938|DJIBOUTI||Marie.Calderon@0j.org|2452406| +6470|AAAAAAAAGEJBAAAA|1517407|4751|18495|2450570|2450540|Sir|Phillip|Hernandez|N|27|12|1977|TOGO||Phillip.Hernandez@IRlgi0XE8bhYjo.com|2452608| +6471|AAAAAAAAHEJBAAAA|124947|7152|20557|2450018|2449988|Sir|Marvin|Cowart|N|31|3|1947|DENMARK||Marvin.Cowart@l3KGXTY9tBhKPFa.edu|2452387| +6472|AAAAAAAAIEJBAAAA|1898785|6564|34743|2450943|2450913|Dr.|Elsa|Thomas|N|11|4|1950|VANUATU||Elsa.Thomas@eTh.org|2452529| +6473|AAAAAAAAJEJBAAAA|1392590|7084|48378|2451842|2451812|Dr.|Eric|Richard|Y|29|4|1972|BERMUDA||Eric.Richard@xe.edu|2452351| +6474|AAAAAAAAKEJBAAAA|258581|580|20579|2450216|2450186|Ms.|Eleanor|Henderson|N|20|3|1925|BANGLADESH||Eleanor.Henderson@VDKsJMUaeV.com|2452473| +6475|AAAAAAAALEJBAAAA|877068|1679|23890|2451969|2451939|Mrs.|Catherine|Dominguez|Y|14|11|1937|MALAYSIA||Catherine.Dominguez@zLqC4l0bncJQD.org|2452596| +6476|AAAAAAAAMEJBAAAA|455220|2149|45831|2450046|2450016|Sir|Richard|Colon|Y|6|8|1978|RWANDA||Richard.Colon@AvO.edu|2452354| +6477|AAAAAAAANEJBAAAA|741999|6162|24725|2450701|2450671|Ms.|Carolyn|Hinkle|N|12|8|1958|ANTARCTICA||Carolyn.Hinkle@5Yb.org|2452340| +6478|AAAAAAAAOEJBAAAA|560917|5865|45213|2451678|2451648|Sir|Roy|Taylor|N|31|3|1974|ANDORRA||Roy.Taylor@4pnPxF0DrtLB.edu|2452361| +6479|AAAAAAAAPEJBAAAA|1796492|5429|43966|2449797|2449767|Mrs.|Sarah|Guillory|Y|3|10|1989|SWAZILAND||Sarah.Guillory@0NIOnqGSLY2yMrr4.com|2452430| +6480|AAAAAAAAAFJBAAAA|17791|4663|26335|2449984|2449954|Mr.|John|Bowler|N|17|12|1990|CYPRUS||John.Bowler@Ly8eSdcc1O3PaQF75.com|2452373| +6481|AAAAAAAABFJBAAAA|378123|809|19048|2451316|2451286|Dr.|Ervin|Siegel|Y|28|4|1924|ANGOLA||Ervin.Siegel@N.com|2452578| +6482|AAAAAAAACFJBAAAA|1823715|1265|1968|2449097|2449067|Sir|Ronald|Wright|N|27|5|1980|CROATIA||Ronald.Wright@900a2L5TP.com|2452336| +6483|AAAAAAAADFJBAAAA|950326|5564|42049|2452417|2452387|Mrs.|Dawn|Adcock|N|5|12|1945|ARUBA||Dawn.Adcock@rjCzANTBmI.com|2452558| +6484|AAAAAAAAEFJBAAAA|69932|2737|25495|2452467|2452437|Ms.|Ashley|Martin|N|17|6|1985|JAMAICA||Ashley.Martin@jqYy9kFiHLmyGN.org|2452335| +6485|AAAAAAAAFFJBAAAA|132678|389|26100|2452282|2452252|Mrs.|Mildred|Stallings|N|4|6|1987|WALLIS AND FUTUNA||Mildred.Stallings@Xun47.edu|2452620| +6486|AAAAAAAAGFJBAAAA|1035800|2380|48207|2449048|2449018|Sir|Jeffery|Carson|N|16|4|1991|FAROE ISLANDS||Jeffery.Carson@XO.edu|2452524| +6487|AAAAAAAAHFJBAAAA|214033|2301|28216|2451678|2451648|Dr.|Tatyana|Boyer|Y|13|11|1978|IRELAND||Tatyana.Boyer@MEdoQ4nO4k2J1H.com|2452647| +6488|AAAAAAAAIFJBAAAA|31474|3518|17915|2450530|2450500|Mr.|Steven|Wood|Y|1|11|1951|CZECH REPUBLIC||Steven.Wood@BmlVNKqKKeA.org|2452519| +6489|AAAAAAAAJFJBAAAA|1015994|2111|31028|2451523|2451493|Sir|Melvin|Wright|Y|22|1|1954|GIBRALTAR||Melvin.Wright@U65Gp3FTqEOna6g.com|2452580| +6490|AAAAAAAAKFJBAAAA|1508111|3869|39227|2449107|2449077|Mr.|Robert|Beaty|Y|12|4|1976|SRI LANKA||Robert.Beaty@PYButgsKOn3IdOk1oq.edu|2452471| +6491|AAAAAAAALFJBAAAA|465053|849|27398|2451959|2451929|Ms.|Rosalee|Wilson|N|13|12|1945|ALBANIA||Rosalee.Wilson@QegQPMgCpsD.com|2452422| +6492|AAAAAAAAMFJBAAAA|1375959|5123|29840|2450443|2450413|Mrs.|Mabel|Guzman|N|30|1|1959|ZIMBABWE||Mabel.Guzman@lDN2nIqy052x3c5.com|2452299| +6493|AAAAAAAANFJBAAAA|737846|997|5748|2452610|2452580|Mrs.|Margaret|Roberts|N|15|10|1960|PANAMA||Margaret.Roberts@4sYO4NH1o6.edu|2452344| +6494|AAAAAAAAOFJBAAAA|510964|2463|19609|2451775|2451745|Sir|Harry|Bundy|N|18|11|1973|IRELAND||Harry.Bundy@1V9aZqZ5vY.com|2452432| +6495|AAAAAAAAPFJBAAAA|355586|1074|562||2452339||Kevin||||||||Kevin.Thomas@Cb2FejmEx7o.com|| +6496|AAAAAAAAAGJBAAAA|952515|6189|23463|2451525|2451495|Dr.|Jennifer|Rodriguez|N|20|4|1960|ETHIOPIA||Jennifer.Rodriguez@Z5Pe69Sc.edu|2452520| +6497|AAAAAAAABGJBAAAA|1368021|4067|32866|2450673|2450643|Mr.|Juan|Carpenter|N|10|5|1972|LUXEMBOURG||Juan.Carpenter@7htZUIsuSuH.edu|2452568| +6498|AAAAAAAACGJBAAAA|715001|5284|38367|2452200|2452170|Miss|Tami|Sutherland|Y|23|10|1989|MYANMAR||Tami.Sutherland@e.org|2452580| +6499|AAAAAAAADGJBAAAA|1323701|3183|7418|2449281|2449251|Sir|Jesse|Reeves|Y|31|1|1959|VENEZUELA||Jesse.Reeves@m2VJd3.org|2452456| +6500|AAAAAAAAEGJBAAAA|1677825|2481|47090|2450021|2449991|Dr.|Timothy|Thomas|N|1|6|1960|NAURU||Timothy.Thomas@OhpZkYMezY8rQFSzUS.com|2452313| +6501|AAAAAAAAFGJBAAAA|277490|647|44604|2450988|2450958|Dr.|Sharon|Hammond|Y|7|3|1948|UNITED STATES||Sharon.Hammond@Fe9x1105SpIHoCi5CUf.edu|2452604| +6502|AAAAAAAAGGJBAAAA|38443|5817|30791|2449651|2449621|Dr.|Ann|Bostick|Y|4|5|1929|THAILAND||Ann.Bostick@SfrvF.com|2452598| +6503|AAAAAAAAHGJBAAAA|636331|5635|32939|2450081|2450051|Ms.|Deborah|Cruz|N|16|6|1964|SIERRA LEONE||Deborah.Cruz@0GBNqzTa0j.edu|2452420| +6504|AAAAAAAAIGJBAAAA|1342067|2724|27028|2452669|2452639|Ms.|Christina|Weldon|N|24|2|1925|SERBIA||Christina.Weldon@Tba.edu|2452441| +6505|AAAAAAAAJGJBAAAA|1863497|927|10357|2449352|2449322|Ms.|Veronica|King|Y|26|8|1925|WESTERN SAHARA||Veronica.King@4ilGTMi7BPq.edu|2452318| +6506|AAAAAAAAKGJBAAAA|250590|347|46904|2450883|2450853|Dr.|Dwayne|Barron|N|20|5|1956|NETHERLANDS||Dwayne.Barron@3VS1JGH2EEveREtbL.edu|2452610| +6507|AAAAAAAALGJBAAAA|1615494|2257|25820||2451233|Miss||||10|6||||Sherley.Alvarado@EcrAicC0ljpE.edu|2452360| +6508|AAAAAAAAMGJBAAAA|1050293|6533|38373|2452080|2452050|Mrs.|Phyllis|Drake|Y|17|10|1971|BRUNEI DARUSSALAM||Phyllis.Drake@SFoo76zDem1xzKDD.org|2452438| +6509|AAAAAAAANGJBAAAA|199097|4423|15749|2449913|2449883|Dr.|Cheryl|Turner|N|22|8|1926|MARTINIQUE||Cheryl.Turner@brTnNDZ7.edu|2452546| +6510|AAAAAAAAOGJBAAAA|205215|3821|13580|2452131|2452101|Miss|Kathlyn|Abbott|N|7|1|1962|ALGERIA||Kathlyn.Abbott@vH.edu|2452406| +6511|AAAAAAAAPGJBAAAA|816970|283|39996|2451947|2451917|Dr.|Bradley|Turner|N|21|5|1971|MALAYSIA||Bradley.Turner@ZgpMT.com|2452314| +6512|AAAAAAAAAHJBAAAA|73296|6657|3665|2449211|2449181|Sir|David|Stewart|N|13|4|1954|SLOVAKIA||David.Stewart@ulOeH05B6O.com|2452299| +6513|AAAAAAAABHJBAAAA|1519256|839|46422|2451843|2451813|Sir|Ernest|Harper|N|21|12|1934|NEPAL||Ernest.Harper@HQ.org|2452385| +6514|AAAAAAAACHJBAAAA|1743216|6946|42629|2450570|2450540|Mrs.|Diane|Whitehead|Y|7|5|1925|BELARUS||Diane.Whitehead@8NytOzcEBZa2SCInMJ.org|2452365| +6515|AAAAAAAADHJBAAAA|15297|953|19362|2452487|2452457|Mr.|Nathan|Braun|Y|27|7|1992|MALAYSIA||Nathan.Braun@rnYjLuucAz.com|2452300| +6516|AAAAAAAAEHJBAAAA|114380|687|7394|2449809|2449779|Sir|James|Soria|Y|5|8|1938|AZERBAIJAN||James.Soria@VZGRljip.org|2452554| +6517|AAAAAAAAFHJBAAAA|841256|1587|8718|2452675|2452645|Miss|Verna|Mcgowan|N|4|1|1976|PARAGUAY||Verna.Mcgowan@cL8Ks8eOKv.com|2452327| +6518|AAAAAAAAGHJBAAAA|1522359|5331|20750|2450370|2450340|Sir|David|Chaney|Y|21|3|1944|VENEZUELA||David.Chaney@ooSe.edu|2452544| +6519|AAAAAAAAHHJBAAAA|1869370|3485|39647|2450662|2450632|Mr.|Robert|Bird|Y|18|11|1974|SOMALIA||Robert.Bird@Aii3bu2.edu|2452464| +6520|AAAAAAAAIHJBAAAA|1850520|6086|2052|2450391|2450361|Miss|Mercedes|Williams|N|22|2|1948|NETHERLANDS ANTILLES||Mercedes.Williams@aOttg8ueQ3.com|2452332| +6521|AAAAAAAAJHJBAAAA|157604|710|46663|2449286|||Hans||N||8||||Hans.Guidry@yLfulcxNpftbM3.edu|| +6522|AAAAAAAAKHJBAAAA|1388026|1225|48071|2449636|2449606|Mrs.|Carolyn|Baker|Y|10|8|1988|NEPAL||Carolyn.Baker@k8PnqyADZRXs69A1.org|2452590| +6523|AAAAAAAALHJBAAAA|430943|549|38908|2452310|2452280|Mr.|Eric|Minnick|Y|20|7|1985|GUADELOUPE||Eric.Minnick@jOpHgVuo8Dd.edu|2452630| +6524|AAAAAAAAMHJBAAAA|1813816|4161|29353|2452665|2452635|Mr.|David|Poole|N|17|11|1948|LITHUANIA||David.Poole@bgMf.edu|2452496| +6525|AAAAAAAANHJBAAAA|1811700|6742|3337|2449886|2449856|Sir|Irving|Fisher|Y|23|12|1936|ESTONIA||Irving.Fisher@kttYnY1DF7.org|2452437| +6526|AAAAAAAAOHJBAAAA|625996|1358|28673|2451208|2451178|Sir|Hugh|Moore|N|3|9|1971|BARBADOS||Hugh.Moore@bB8Eoq4YfY.org|2452620| +6527|AAAAAAAAPHJBAAAA|592309|56|956|2451031|2451001|Dr.|James|Canada|N|13|6|1931|ICELAND||James.Canada@Ar5bx1d1Ib1ch5.org|2452479| +6528|AAAAAAAAAIJBAAAA|164467|166|4265|2451329|2451299|Dr.|Mark|Bell|N|6|8|1987|ZAMBIA||Mark.Bell@CciAQnv.org|2452620| +6529|AAAAAAAABIJBAAAA|979504|1149|3547|2449163|2449133|Dr.|James|White|N|16|12|1980|IRAQ||James.White@DI8oHLVIxJer5MMljj.org|2452578| +6530|AAAAAAAACIJBAAAA|611455|3633|21011|2449816|2449786|Dr.|Harry|Cook|N|6|10|1986|NICARAGUA||Harry.Cook@t2ZpCJY4uacI.com|2452401| +6531|AAAAAAAADIJBAAAA|225135|3031|23280|2449905|2449875|Dr.|Major|Tate|N|14|12|1966|OMAN||Major.Tate@DqJFHcQSRn8.edu|2452609| +6532|AAAAAAAAEIJBAAAA|1096818|5231|44044|2449110|2449080|Mr.|James|Flowers|Y|9|1|1983|KOREA, REPUBLIC OF||James.Flowers@45AYHb.org|2452367| +6533|AAAAAAAAFIJBAAAA|46688|5681|17474|2451507|2451477|Sir|Theodore|Graham|Y|10|7|1978|UKRAINE||Theodore.Graham@YrN3tMSgEc9TtrUP.org|2452585| +6534|AAAAAAAAGIJBAAAA|338838|6840|24042|2449364|2449334|Ms.|Victoria|Patton|Y|26|2|1966|GUERNSEY||Victoria.Patton@e3tKTb9TREvX.edu|2452406| +6535|AAAAAAAAHIJBAAAA||6071|18959||2450075||Laura|Miller||12||||||| +6536|AAAAAAAAIIJBAAAA|1897669||47491||2452178||Carrie|Pape|Y|||1935|MACAO||Carrie.Pape@ALuAoXT.edu|2452453| +6537|AAAAAAAAJIJBAAAA|550441|185|37176|2452472|2452442|Mr.|Herbert|Martin|Y|24|4|1950|AUSTRIA||Herbert.Martin@A7f7PjiOcO.org|2452518| +6538|AAAAAAAAKIJBAAAA|1672521|1284|9647|2452060|2452030|Ms.|Helen|Hennessey|N|1|9|1924|KOREA, REPUBLIC OF||Helen.Hennessey@bdy7yE8QSF6.edu|2452645| +6539|AAAAAAAALIJBAAAA|923458|4922|27939|2452308|2452278|Sir|Charles|Willis|N|30|11|1926|GERMANY||Charles.Willis@uc364qASl7ZZK.edu|2452454| +6540|AAAAAAAAMIJBAAAA|232971|6454|14737|2451365|2451335|Sir|Daniel|Horn|Y|22|11|1934|KAZAKHSTAN||Daniel.Horn@XgO5U2u5ZDHRcqgXrF.org|2452641| +6541|AAAAAAAANIJBAAAA|163024|5170|30869|2450490|2450460|Mr.|Michael|Jay|N|21|11|1970|CZECH REPUBLIC||Michael.Jay@oz9NBDZcJitC.com|2452396| +6542|AAAAAAAAOIJBAAAA|1854884|2991|48698|2449612|2449582|Mr.|Nicholas|Evans|Y|22|9|1940|YEMEN||Nicholas.Evans@RFSYsdYbKEd.org|2452560| +6543|AAAAAAAAPIJBAAAA|60692|5410|39328|2451022|2450992|Sir|Steven|Shephard|Y|25|8|1967|MAYOTTE||Steven.Shephard@hMd9xGYX9X.org|2452305| +6544|AAAAAAAAAJJBAAAA|1193817|2418|6972|2451290|2451260|Mrs.|Jeannie|Williams|N|18|6|1991|NIUE||Jeannie.Williams@12MUaTTRGfAgvAzBkOSp.edu|2452544| +6545|AAAAAAAABJJBAAAA|296155|2502|16449|2449531|2449501|Mrs.|Cecelia|Grissom|N|9|3|1936|NAMIBIA||Cecelia.Grissom@aoD6.edu|2452549| +6546|AAAAAAAACJJBAAAA|1142493|5412|49084|2450607|2450577|Dr.|Lillian|Williams|N|30|11|1934|HONDURAS||Lillian.Williams@sm2zbNvn9nbt.edu|2452432| +6547|AAAAAAAADJJBAAAA|694029|1319|11809|2450572|2450542|Mrs.|Jeannie|Ray|Y|11|11|1974|RUSSIAN FEDERATION||Jeannie.Ray@4zX5Ojck7u0bceC3tJ.org|2452435| +6548|AAAAAAAAEJJBAAAA|165139|5327|668|2451591|2451561|Ms.|Dorothy|Peterson|Y|11|11|1943|PITCAIRN||Dorothy.Peterson@Qtn1.edu|2452599| +6549|AAAAAAAAFJJBAAAA|1093434|3536|10791|2452402|2452372|Mr.|Daniel|Goldberg|N|22|3|1935|TURKEY||Daniel.Goldberg@JkRJOjneMim.com|2452564| +6550|AAAAAAAAGJJBAAAA|648955|7059|33610|2451731|2451701|Dr.|Darren|Alexander|Y|15|7|1950|NIGERIA||Darren.Alexander@MHrf.org|2452428| +6551|AAAAAAAAHJJBAAAA|1535170|293|48263|2450645|2450615|Sir|Maria|Laroche|Y|29|6|1930|SENEGAL||Maria.Laroche@To56x3zfDUTnR.com|2452383| +6552|AAAAAAAAIJJBAAAA|1405495|3305|37162|2451555|2451525|Ms.|Arnita|Hall|Y|6|7|1961|BELARUS||Arnita.Hall@z8sS3k6cuYumH.com|2452504| +6553|AAAAAAAAJJJBAAAA|1333375|1461|7194|2450304|2450274|Mr.|Emil|Wolf|Y|31|5|1970|KAZAKHSTAN||Emil.Wolf@sHK.org|2452385| +6554|AAAAAAAAKJJBAAAA|1850519|838|25910|2450271|2450241|Sir|Willie|Williamson|N|8|1|1928|SYRIAN ARAB REPUBLIC||Willie.Williamson@7VSQGzlh6yRY.com|2452464| +6555|AAAAAAAALJJBAAAA|490279|431|45336|2451635|2451605|Miss|Winnie|Cates|Y|30|9|1937|CAMBODIA||Winnie.Cates@MAHTzhDP54B.org|2452430| +6556|AAAAAAAAMJJBAAAA|1868297|2590|10883|2449909|2449879|Sir|Carlos|Black|Y|6|2|1939|HUNGARY||Carlos.Black@TpsEedsicIM.com|2452558| +6557|AAAAAAAANJJBAAAA|1077264|4981|13851|2452581|2452551|Mr.|Bernard|Bowman|N|3|11|1932|ISLE OF MAN||Bernard.Bowman@IiPiF.com|2452305| +6558|AAAAAAAAOJJBAAAA|1848468|1658|32895|2449718|2449688|Sir|Steve|Dawson|N|15|12|1989|NIUE||Steve.Dawson@Mx3MaIj5b.edu|2452565| +6559|AAAAAAAAPJJBAAAA|158784|6822|4460|2451894|2451864|Mrs.|Amanda|Zimmerman|N|2|3|1926|NIGERIA||Amanda.Zimmerman@5SaDfFqL86baCAEQjd.com|2452373| +6560|AAAAAAAAAKJBAAAA|1559502|5777|41205|2450485|2450455|Sir|Bobby|Murphy|Y|13|2|1939|FINLAND||Bobby.Murphy@7TJFNl4QoYJDpR.com|2452409| +6561|AAAAAAAABKJBAAAA|740988|2211|20634|2451778|2451748|Dr.|Timothy|Paxton|Y|26|11|1988|OMAN||Timothy.Paxton@qqn.com|2452469| +6562|AAAAAAAACKJBAAAA|1058353|2555|8357|2450115|2450085|Mr.|Steven|Randall|Y|29|11|1927|SURINAME||Steven.Randall@c.edu|2452397| +6563|AAAAAAAADKJBAAAA|706134|2105|46476|2451500|2451470|Miss|Suzanne|Hernandez|N|21|2|1944|EQUATORIAL GUINEA||Suzanne.Hernandez@1LoJCDMJuAH9lBy.org|2452329| +6564|AAAAAAAAEKJBAAAA|644836|5343|14697|2452118|2452088|Mr.|Edmund|Sallee|Y|15|3|1986|TURKEY||Edmund.Sallee@Flk4PnyKTBjvaY0.com|2452392| +6565|AAAAAAAAFKJBAAAA|130799|5049|39299|2449523|2449493|Mr.|August|Sanborn|Y|15|10|1985|BRAZIL||August.Sanborn@qXlLMx.org|2452371| +6566|AAAAAAAAGKJBAAAA|1682808|3431|13654|2452170|2452140|Dr.|Brandon|Walker|Y|19|4|1976|ZAMBIA||Brandon.Walker@ahb1KIs.edu|2452390| +6567|AAAAAAAAHKJBAAAA|1105467|6258|12202|2451364|2451334|Dr.|Frank|Oakley|Y|9|9|1989|PALAU||Frank.Oakley@386zY4LCmR.org|2452631| +6568|AAAAAAAAIKJBAAAA|93551|3049|16547|2452622|2452592|Mr.|Gary|Berry|Y|1|6|1929|MOZAMBIQUE||Gary.Berry@osi0JphBq8VYE.org|2452544| +6569|AAAAAAAAJKJBAAAA|426078|5|16285|2451162|2451132|Dr.|Francis|Bullock|Y|8|2|1969|JORDAN||Francis.Bullock@bJGTy.edu|2452310| +6570|AAAAAAAAKKJBAAAA|1160066|2007|49548|2451475|2451445|Mr.|Erick|Hill|N|20|5|1953|QATAR||Erick.Hill@zik88hgQ9G8hNL2.org|2452641| +6571|AAAAAAAALKJBAAAA|1528721|4591|18414|2451963|2451933|Dr.|Jeremy|Black|Y|2|6|1958|ALAND ISLANDS||Jeremy.Black@p5UDAVBGGnL8G.edu|2452506| +6572|AAAAAAAAMKJBAAAA|536272||23615|2451605||||Jimenez|||10|||||2452405| +6573|AAAAAAAANKJBAAAA|544068|5231|28041|2450030|2450000|Dr.|James|Roundtree|N|19|12|1943|MALI||James.Roundtree@3fh52F8M2VFVQpnVm0.edu|2452358| +6574|AAAAAAAAOKJBAAAA|1234821|5181|48741|2451181|2451151|Miss|Edith|Stone|Y|14|6|1992|MYANMAR||Edith.Stone@LoQ2chXXRNP8yy.com|2452571| +6575|AAAAAAAAPKJBAAAA|128798|6239|16480|2450746|2450716|Mr.|Tommy|Friend|Y|18|10|1978|SWITZERLAND||Tommy.Friend@hXoXnR1.org|2452417| +6576|AAAAAAAAALJBAAAA|516519|3726|38086|2450540|2450510|Ms.|Emily|Battles|Y|15|6|1929|MEXICO||Emily.Battles@p3YEuyu6J7tHuESC.org|2452351| +6577|AAAAAAAABLJBAAAA|302767|1150|42733|2449837|2449807|Mr.|Michael|Villareal|Y|15|2|1981|GUINEA-BISSAU||Michael.Villareal@kmSM84cgknFNNUOsX.com|2452390| +6578|AAAAAAAACLJBAAAA|502029|915|5001|2450527|2450497|Dr.|Carlos|Bowman|Y|18|10|1975|BHUTAN||Carlos.Bowman@lIFuRJfVlKt1K.org|2452355| +6579|AAAAAAAADLJBAAAA|1742385|6242|3299|2449617|2449587|Mr.|Pierre|Foster|Y|22|5|1925|PAKISTAN||Pierre.Foster@CxHmxbiqq.com|2452440| +6580|AAAAAAAAELJBAAAA|1553950|2557|9564|2451922|2451892|Ms.|Janet|George|N|31|1|1983|EQUATORIAL GUINEA||Janet.George@D5PaanfjnJq940pA.edu|2452501| +6581|AAAAAAAAFLJBAAAA|692892|2636|1980|2449742|2449712|Dr.|Kevin|Verdin|N|28|6|1931|CAPE VERDE||Kevin.Verdin@gUrO3OrnYO5Xanz.com|2452358| +6582|AAAAAAAAGLJBAAAA|1052856|735|18214|2451305|2451275|Miss|Lynda|Maxwell|Y|29|6|1949|ROMANIA||Lynda.Maxwell@imqO1rJopoStNbDAfC.org|2452392| +6583|AAAAAAAAHLJBAAAA|544089|1851|4366|2450517|2450487|Ms.|Marian|Roe|Y|4|11|1978|BOTSWANA||Marian.Roe@Fb99UT2XxX3a.edu|2452539| +6584|AAAAAAAAILJBAAAA|1699569|6691|8613|2450244|2450214|Mr.|George|Holmes|Y|13|1|1975|SERBIA||George.Holmes@i91x6aFJuFjx95.com|2452330| +6585|AAAAAAAAJLJBAAAA|364738|1004|35623|2449171|2449141|Mr.|Clinton|Hudson|Y|17|7|1975|CHINA||Clinton.Hudson@GvnAKk192VV.org|2452501| +6586|AAAAAAAAKLJBAAAA|1438139|3893|1647|2450481|2450451|Miss|Irene|Chavarria|Y|22|4|1934|JERSEY||Irene.Chavarria@f59.com|2452477| +6587|AAAAAAAALLJBAAAA|495767|6299|41631|2450855|2450825|Mr.|Geoffrey|Jarrett|Y|3|5|1945|INDONESIA||Geoffrey.Jarrett@GhzPT.com|2452541| +6588|AAAAAAAAMLJBAAAA|303794|1068|29542|2451999|2451969|Dr.|Gina|Williams|Y|17|4|1952|ISRAEL||Gina.Williams@zDf9RCVoi9HOMY.org|2452497| +6589|AAAAAAAANLJBAAAA|694164|1924|26931|2451693|2451663|Ms.|Antoinette|Sullivan|Y|16|11|1952|WALLIS AND FUTUNA||Antoinette.Sullivan@dmHMf1Gruka6tpAO9j.com|2452554| +6590|AAAAAAAAOLJBAAAA|783655|5628|43954|2449065|2449035|Dr.|Sharon|Munoz|N|23|3|1986|GUERNSEY||Sharon.Munoz@Gfn.edu|2452596| +6591|AAAAAAAAPLJBAAAA|646188|5340|26845|2449589|2449559|Sir|Daniel|Hartmann|Y|28|9|1982|WESTERN SAHARA||Daniel.Hartmann@BzoUQ1p.edu|2452534| +6592|AAAAAAAAAMJBAAAA|403892|6230|7768|2449766|2449736|Mr.|Gerald|Ingram|N|18|3|1969|ETHIOPIA||Gerald.Ingram@Dd.com|2452554| +6593|AAAAAAAABMJBAAAA|1175947|4484|45421|2450098|2450068|Dr.|Milton|Varner|Y|18|6|1935|BARBADOS||Milton.Varner@Rf8ogP13aUv6.org|2452304| +6594|AAAAAAAACMJBAAAA|1883137|4928|13919|2451285|2451255|Ms.|Ruby|Rutledge|N|20|4|1992|CHRISTMAS ISLAND||Ruby.Rutledge@dAVBfoxcQPC9dRGpEo.edu|2452377| +6595|AAAAAAAADMJBAAAA|306372|875|24578|2449890|2449860|Mr.|Jeffrey|Jones|N|14|4|1963|AZERBAIJAN||Jeffrey.Jones@KPrEk.com|2452428| +6596|AAAAAAAAEMJBAAAA|1052491|4562|13386|2452531|2452501|Dr.|Jose|Connolly|Y|22|2|1990|AZERBAIJAN||Jose.Connolly@a0AbJNui9buT65.edu|2452285| +6597|AAAAAAAAFMJBAAAA|1842063|6606|37149|2452159|2452129|Sir|Roger|Brewer|Y|28|9|1926|NORFOLK ISLAND||Roger.Brewer@Q5eDE6oFYoHNt4cJb.edu|2452486| +6598|AAAAAAAAGMJBAAAA|1607580|602|33224|2449871|2449841|Mr.|William|Eddy|N|8|4|1985|ANTARCTICA||William.Eddy@oP4Gph9jRxS1.org|2452304| +6599|AAAAAAAAHMJBAAAA|1317926|2881|26989|2450965|2450935|Ms.|Judy|Perez|Y|18|11|1956|SIERRA LEONE||Judy.Perez@OsngjZEDUgNOK.com|2452499| +6600|AAAAAAAAIMJBAAAA|764145|1244|34696|2450199|2450169|Ms.|Lyn|Colon|N|23|9|1963|SLOVAKIA||Lyn.Colon@O55.com|2452412| +6601|AAAAAAAAJMJBAAAA|||13995|2451848|2451818|Miss|Yolanda|Ledford||1||1963|BURKINA FASO|||2452495| +6602|AAAAAAAAKMJBAAAA|1417676|5165|11054|2452424|2452394|Sir|John|Kruse|N|25|9|1950|NEW ZEALAND||John.Kruse@BgZ6x.edu|2452516| +6603|AAAAAAAALMJBAAAA|1818356|5748|15187|2450921|2450891|Mrs.|Katrina|Inman|Y|4|1|1958|NETHERLANDS ANTILLES||Katrina.Inman@No2N.org|2452401| +6604|AAAAAAAAMMJBAAAA|482827|1025|25976|2449690|2449660|Mr.|James|Reeves|N|8|6|1948|FAROE ISLANDS||James.Reeves@U5dOt5e1v23rUMnr.com|2452647| +6605|AAAAAAAANMJBAAAA|56729|4031|9380|2449755|2449725|Sir|Richard|Crane|N|17|3|1981|GIBRALTAR||Richard.Crane@5vuZ.org|2452406| +6606|AAAAAAAAOMJBAAAA|264017|6117|4570|2451162|2451132|Mr.|Thomas|Bryant|Y|20|8|1989|WESTERN SAHARA||Thomas.Bryant@xnozPeY.edu|2452525| +6607|AAAAAAAAPMJBAAAA|1063421|2470|9519|2451555|2451525|Dr.|Amy|Holmes|N|14|9|1964|FRENCH POLYNESIA||Amy.Holmes@IIr8mj7hmSfog.com|2452308| +6608|AAAAAAAAANJBAAAA|1361400|20|37487|2451275|2451245|Mr.|Robert|Simpson|Y|27|9|1973|NIGERIA||Robert.Simpson@DfcmxuQLqvcAPZY.edu|2452471| +6609|AAAAAAAABNJBAAAA|1585809|5496|23983|2451476|2451446|Ms.|Helen|Smith|N|29|5|1936|CYPRUS||Helen.Smith@1L.com|2452501| +6610|AAAAAAAACNJBAAAA|1322339|1992|34445|2449681|2449651|Mrs.|Doris|Wilson|Y|27|12|1939|CAYMAN ISLANDS||Doris.Wilson@58mB7.com|2452345| +6611|AAAAAAAADNJBAAAA|1768033|4926|8572|2452423|2452393|Miss|Cindy|Harmon|Y|20|3|1984|HUNGARY||Cindy.Harmon@GmPBS1z.edu|2452450| +6612|AAAAAAAAENJBAAAA|40757|3203|40449|2452045|2452015|Dr.|Philip|Craft|N|7|3|1964|ARUBA||Philip.Craft@9DM3Pu4uJpP8SdKa.edu|2452302| +6613|AAAAAAAAFNJBAAAA|1638044|1481|44071|2452047|2452017|Sir|Nicholas|Mccutcheon|N|28|9|1954|WALLIS AND FUTUNA||Nicholas.Mccutcheon@CXeFBHcIa64.edu|2452409| +6614|AAAAAAAAGNJBAAAA|1671508|169|830|2451914|2451884|Ms.|Erica|Thompson|N|12|6|1935|QATAR||Erica.Thompson@hAoBEMFDO.com|2452593| +6615|AAAAAAAAHNJBAAAA|894396|4748|23053|2451911|2451881|Sir|William|Miller|Y|10|4|1987|BOUVET ISLAND||William.Miller@V5Xry2R675oBEa.com|2452578| +6616|AAAAAAAAINJBAAAA|1114745|1935|47525|2450589|2450559|Mrs.|Jennifer|Corbett|N|12|6|1943|BERMUDA||Jennifer.Corbett@DV01o8A4cHeyA.org|2452452| +6617|AAAAAAAAJNJBAAAA|1594756|485|18967|2452280|2452250|Dr.|Lisa|Sandoval|Y|4|6|1955|PERU||Lisa.Sandoval@Y4Sp3gS0VlCbg.com|2452571| +6618|AAAAAAAAKNJBAAAA|1206418|5692|29413|2449318|2449288|Mr.|Dean|Pennington|N|9|8|1957|PALAU||Dean.Pennington@mm5xG.com|2452317| +6619|AAAAAAAALNJBAAAA|1576817|68|18119|2451204|2451174|Sir|George|Pacheco|N|4|11|1992|CAMEROON||George.Pacheco@zkoJe.org|2452399| +6620|AAAAAAAAMNJBAAAA|1078720|3000|23280|2452661|2452631|Sir|James|Hoyt|N|12|4|1932|EGYPT||James.Hoyt@u6fbbF7XDbC.com|2452372| +6621|AAAAAAAANNJBAAAA|1511597|6678|38606|2452555|2452525|Miss|Victoria|Green|Y|16|4|1939|ANGUILLA||Victoria.Green@Ddm0ZOgbUngmp.org|2452513| +6622|AAAAAAAAONJBAAAA|1287377|5789|3354|2450966|2450936|Miss|Kristin|Vargas|Y|4|11|1978|BANGLADESH||Kristin.Vargas@2Kb.org|2452373| +6623|AAAAAAAAPNJBAAAA|1541973|5678|183|2451900|2451870|Mr.|Michael|Whitley|N|31|10|1956|BOLIVIA||Michael.Whitley@RSP73P2pqIvE0IF8R.org|2452386| +6624|AAAAAAAAAOJBAAAA|601737|2189|25888|2450274|2450244|Ms.|Jenny|Martin|Y|31|7|1967|SLOVENIA||Jenny.Martin@an.edu|2452500| +6625|AAAAAAAABOJBAAAA|813074|3564|18054|2451679|2451649|Mrs.|Suzanne|Holmes|N|27|4|1972|SEYCHELLES||Suzanne.Holmes@mjD6e6gscHUl44F.com|2452568| +6626|AAAAAAAACOJBAAAA|1705980|679|40131|2452405|2452375|Sir|Robert|Gillette|N|30|12|1990|R�UNION||Robert.Gillette@IfdCFoaPoUT.com|2452647| +6627|AAAAAAAADOJBAAAA|1433694|1256|41665|2452091|2452061|Sir|Francis|Parish|Y|13|4|1969|SAMOA||Francis.Parish@3KZhBDLI.edu|2452404| +6628|AAAAAAAAEOJBAAAA|1238416|5898|22933|2451524|2451494|Dr.|Jeremiah|Howard|N|6|9|1982|FRENCH POLYNESIA||Jeremiah.Howard@dKLM.com|2452348| +6629|AAAAAAAAFOJBAAAA|1915263|14|44365|2450673|2450643|Ms.|Maritza|Johnson|N|10|6|1958|SEYCHELLES||Maritza.Johnson@iVmGXArBcE.org|2452316| +6630|AAAAAAAAGOJBAAAA|1502012|1067|18969|2452295|2452265|Dr.|Richard|Burkhart|N|4|3|1967|MONTSERRAT||Richard.Burkhart@7f6OztkqRCtOYH.com|2452502| +6631|AAAAAAAAHOJBAAAA|1236769|2476|49383|2449358|2449328|Miss|Nancy|Smallwood|Y|13|1|1943|THAILAND||Nancy.Smallwood@CF1F1MPx1y.com|2452456| +6632|AAAAAAAAIOJBAAAA|1159289|5177|41418|2451972|2451942|Mr.|Francisco|Guerra|N|18|9|1963|NIGERIA||Francisco.Guerra@5ONxO61niVgh.org|2452316| +6633|AAAAAAAAJOJBAAAA|1002304|5842|24627|2450404|2450374|Sir|Ronald|Hughes|Y|22|3|1924|WESTERN SAHARA||Ronald.Hughes@GuVAErusl8R.edu|2452604| +6634|AAAAAAAAKOJBAAAA|1853603|1594|1585|2450662|2450632|Dr.|James|Walker|N|9|3|1987|TUVALU||James.Walker@i.com|2452307| +6635|AAAAAAAALOJBAAAA|1796717|5004|21738|2451511|2451481|Ms.|Lisa|Salisbury|N|21|1|1960|PAPUA NEW GUINEA||Lisa.Salisbury@QCAyYP0TFRJ.org|2452490| +6636|AAAAAAAAMOJBAAAA|1400285|5206|34974|2450224|2450194|Dr.|Dianne|Cantu|N|3|12|1962|OMAN||Dianne.Cantu@u3x.org|2452607| +6637|AAAAAAAANOJBAAAA|830555|4776|6997|2450971|2450941|Miss|Linda|Francisco|Y|21|7|1941|MARSHALL ISLANDS||Linda.Francisco@Mi4ut.com|2452451| +6638|AAAAAAAAOOJBAAAA|702044|3260|27676|2449973|2449943|Dr.|Glenda|Wyman|Y|13|4|1982|SLOVAKIA||Glenda.Wyman@eT6R5U7dbsm.edu|2452392| +6639|AAAAAAAAPOJBAAAA|324054|5442|34830|2452459|2452429|Sir|Saul|Hart|N|31|10|1985|ISLE OF MAN||Saul.Hart@SLj.com|2452502| +6640|AAAAAAAAAPJBAAAA|453332|923|4514|2451882|2451852|Sir|John|Gagnon|N|13|2|1989|TAJIKISTAN||John.Gagnon@hcADte.com|2452384| +6641|AAAAAAAABPJBAAAA|250670|6854|32638|2452405|2452375|Sir|Richard|Graham|Y|23|6|1970|BAHAMAS||Richard.Graham@EmtsZeBHfdUcX4K.com|2452597| +6642|AAAAAAAACPJBAAAA|107665|3946|35827|2451417|2451387|Dr.|Colleen|Rose|Y|29|1|1957|ITALY||Colleen.Rose@AQ.edu|2452390| +6643|AAAAAAAADPJBAAAA|652200|1132|45664|2450054|2450024|Miss|Barbara|Ratliff|Y|18|5|1946|SOLOMON ISLANDS||Barbara.Ratliff@rOn8AKkKcs.edu|2452552| +6644|AAAAAAAAEPJBAAAA|388207|1170|18133|2451709|2451679|Miss|Wendy|Pope|N|28|12|1956|BOLIVIA||Wendy.Pope@DfI.org|2452508| +6645|AAAAAAAAFPJBAAAA|1880465|1771|47877|2449693|2449663|Miss|Lisa|Downey|N|12|1|1979|UNITED KINGDOM||Lisa.Downey@E7Su.org|2452537| +6646|AAAAAAAAGPJBAAAA|765614|3018|13975|2449226|2449196|Sir|Jimmy|Mclaughlin|N|21|4|1957|PALAU||Jimmy.Mclaughlin@FroSZ8pNOyTj.edu|2452311| +6647|AAAAAAAAHPJBAAAA|540950|1162|35327|2452665|2452635|Dr.|Kathryn|Johnson|N|9|10|1975|NIUE||Kathryn.Johnson@j5EMgEi7jTJ.org|2452445| +6648|AAAAAAAAIPJBAAAA|695638|3900|39362|2452303|2452273|Miss|Donna|Hill|N|3|5|1957|GERMANY||Donna.Hill@gR0tgh.edu|2452363| +6649|AAAAAAAAJPJBAAAA|1298203|4703|3650|2452622|2452592|Mr.|Christopher|Smith|Y|14|7|1947|COMOROS||Christopher.Smith@NCr5pqDcFPgf1IM8vAfI.org|2452361| +6650|AAAAAAAAKPJBAAAA|352805|2869|37380|2449434|2449404|Sir|Brian|Johnson|N|9|3|1972|PITCAIRN||Brian.Johnson@jdgTXU8kPUjsAL.com|2452440| +6651|AAAAAAAALPJBAAAA|1010936|1701|21666|2449304|2449274|Mrs.|Tammy|King|Y|25|9|1957|SLOVAKIA||Tammy.King@xEROMdNkG.edu|2452610| +6652|AAAAAAAAMPJBAAAA|1313864|976|2088|2449619|2449589|Dr.|Linda|Turner|N|27|5|1981|KENYA||Linda.Turner@xT.com|2452292| +6653|AAAAAAAANPJBAAAA|435154|2666|39378|2449658|2449628|Sir|John|Coy|Y|22|6|1945|NORWAY||John.Coy@CYU7S16qat0Xy1.org|2452608| +6654|AAAAAAAAOPJBAAAA|775986|6942|4698|2451743|2451713|Ms.|Kathleen|Haas|N|27|12|1953|NAURU||Kathleen.Haas@lxnzPBu13fkgM.edu|2452582| +6655|AAAAAAAAPPJBAAAA|707164|2688|19459|2449175|2449145|Dr.|Christine|Shannon|N|30|7|1983|GUERNSEY||Christine.Shannon@G1XsT0lj7pf2Osa6.org|2452494| +6656|AAAAAAAAAAKBAAAA|732048|2847|11646|2451139|2451109|Mrs.|Esther|Mitchell|Y|21|12|1991|KAZAKHSTAN||Esther.Mitchell@arhtMCnshxB.edu|2452533| +6657|AAAAAAAABAKBAAAA|796120|2900|29233|2450654|2450624|Ms.|Tami|Ledbetter|N|4|1|1985|QATAR||Tami.Ledbetter@Z.com|2452603| +6658|AAAAAAAACAKBAAAA|1534829|5093|21757|2450770|2450740|Dr.|David|Akins|Y|18|3|1929|NEW ZEALAND||David.Akins@7n7IZezr91Ic.com|2452547| +6659|AAAAAAAADAKBAAAA|889645|1491|5737|2451637|2451607|Dr.|Jeremy|Garmon|N|9|7|1980|KYRGYZSTAN||Jeremy.Garmon@V4A2ce.com|2452545| +6660|AAAAAAAAEAKBAAAA|1632873|1119|43318|2450179|2450149|Mrs.|Julie|Buckner|N|24|10|1966|PERU||Julie.Buckner@fMA.edu|2452480| +6661|AAAAAAAAFAKBAAAA|1104232|1040|4648|2449157|2449127|Ms.|Darlene|Murphy|Y|22|3|1952|NIGER||Darlene.Murphy@Y3teJl.com|2452352| +6662|AAAAAAAAGAKBAAAA|425606|3715|33471|2452571|2452541|Mr.|Richard|Blankenship|N|9|6|1941|LUXEMBOURG||Richard.Blankenship@Z6ZMPY6EB5uxQDYC.com|2452632| +6663|AAAAAAAAHAKBAAAA|1819897|2859|3021|2450795|2450765|Sir|Kenneth|White|N|25|8|1932|CHILE||Kenneth.White@BD9RXHKMlcf8.edu|2452315| +6664|AAAAAAAAIAKBAAAA|1493827|1807|49434|2449139|2449109|Dr.|Jim|Crowder|N|9|4|1975|NETHERLANDS ANTILLES||Jim.Crowder@2XHQQlrQp1.edu|2452637| +6665|AAAAAAAAJAKBAAAA|611246|3327|26255|2450276|2450246|Sir|Scott|Warner|Y|3|3|1939|NAMIBIA||Scott.Warner@YUxFL2RQQi.edu|2452568| +6666|AAAAAAAAKAKBAAAA|1005243|4764|9838|2451314|2451284|Dr.|Emily|Shelton|Y|24|3|1985|SEYCHELLES||Emily.Shelton@fkv.org|2452419| +6667|AAAAAAAALAKBAAAA|318841|6118|3348|2452453|2452423|Sir|Frederick|Jones|N|24|4|1969|ERITREA||Frederick.Jones@uhgHUMN5vBC.org|2452360| +6668|AAAAAAAAMAKBAAAA|1726612|1453|26247|2449069|2449039|Mrs.|Kimberly|Adair|Y|30|5|1948|KOREA, REPUBLIC OF||Kimberly.Adair@7Uk.edu|2452440| +6669|AAAAAAAANAKBAAAA|882230|3753|21276|2452213|2452183|Dr.|Allan|Anderson|Y|30|11|1972|RUSSIAN FEDERATION||Allan.Anderson@mCQNtmF5NRLS.edu|2452431| +6670|AAAAAAAAOAKBAAAA|1731121|377|32833|2450933|2450903|Dr.|Fred|Johnson|N|23|7|1983|CAMEROON||Fred.Johnson@brkY.edu|2452306| +6671|AAAAAAAAPAKBAAAA|537013|5750|26487|2451072|2451042|Mr.|Sean|Lewis|N|16|8|1964|CHRISTMAS ISLAND||Sean.Lewis@MyKkvM41S5ke3d.edu|2452368| +6672|AAAAAAAAABKBAAAA|1503243|2136|12276|2451626|2451596|Dr.|Deborah|Labbe|N|14|7|1953|THAILAND||Deborah.Labbe@RI.org|2452309| +6673|AAAAAAAABBKBAAAA|817827|1968|14862|2449226|2449196|Dr.|Randy|Leach|Y|4|10|1928|GRENADA||Randy.Leach@puXvbo9I36.edu|2452426| +6674|AAAAAAAACBKBAAAA|1487522|2701|16280|2449628|2449598|Mr.|Robert|Mcclintock|N|9|6|1958|FRENCH GUIANA||Robert.Mcclintock@EzfuuQ43mh4UkF.org|2452287| +6675|AAAAAAAADBKBAAAA|1235698|741|43031|2452007|2451977|Mr.|William|Jolley|Y|25|2|1932|LIECHTENSTEIN||William.Jolley@PBy1CzKERt.edu|2452294| +6676|AAAAAAAAEBKBAAAA|1565815|2652|3136|2450685|2450655|Ms.|Catherine|Langford|N|26|1|1966|MARSHALL ISLANDS||Catherine.Langford@KQqyJV.org|2452519| +6677|AAAAAAAAFBKBAAAA|1298914|4728|27295|2452148|2452118|Mrs.|Barbara|Gunn|N|8|6|1974|MOLDOVA, REPUBLIC OF||Barbara.Gunn@qe.com|2452551| +6678|AAAAAAAAGBKBAAAA|1095663|519|9422|2451600|2451570|Mrs.|Gloria|Macias|Y|22|12|1949|EL SALVADOR||Gloria.Macias@seHtOBe.com|2452438| +6679|AAAAAAAAHBKBAAAA|601326|5497|26915|2450032|2450002|Dr.|Elizabeth|Ridley|N|12|1|1954|SAUDI ARABIA||Elizabeth.Ridley@9EMxI3yzqjaPrrFMz.edu|2452368| +6680|AAAAAAAAIBKBAAAA|683553|1000|12047|2452494|2452464|Dr.|Wendy|Farmer|N|24|11|1974|MACAO||Wendy.Farmer@MzSBIytRZiTIu.com|2452313| +6681|AAAAAAAAJBKBAAAA|273269|4360|3051|2451044|2451014|Dr.|Edmond|Kelley|N|19|6|1953|MALI||Edmond.Kelley@Km0odkBatkbL.edu|2452433| +6682|AAAAAAAAKBKBAAAA|500534|3006|35878|2451138|2451108|Mr.|Cameron|Cameron|N|29|3|1941|TUNISIA||Cameron.Cameron@B2LsQGjc8hdJq9lZhdBS.org|2452449| +6683|AAAAAAAALBKBAAAA|612494|1005|47469|2452055|2452025|Dr.|April|Tomlinson|Y|25|1|1985|ITALY||April.Tomlinson@KXk5B9Gj1pD.edu|2452295| +6684|AAAAAAAAMBKBAAAA|588966|329|20892|2449623|2449593|Miss|Marie|West|Y|24|6|1927|SOMALIA||Marie.West@JeOzyAKbELj7o5x.org|2452528| +6685|AAAAAAAANBKBAAAA|1734126|3978|27030|2452255|2452225|Miss|Caryn|Herndon|N|15|8|1928|VANUATU||Caryn.Herndon@ZtQM.edu|2452407| +6686|AAAAAAAAOBKBAAAA|397228|5381|21181|2449039|2449009|Mr.|George|Frey|Y|1|4|1926|PORTUGAL||George.Frey@d.org|2452340| +6687|AAAAAAAAPBKBAAAA|172871|6042|5613|2449975|2449945|Dr.|Travis|Casey|N|18|9|1963|ZAMBIA||Travis.Casey@S2hB2guDvA0Ch.com|2452341| +6688|AAAAAAAAACKBAAAA|1219469|1946|31176|2452646|2452616|Dr.|John|Weiss|Y|23|6|1978|NEW CALEDONIA||John.Weiss@v6a9vXJR6Mu.com|2452434| +6689|AAAAAAAABCKBAAAA|1796293|2145|21550|2449292|2449262|Mr.|Mitchell|Valencia|Y|31|1|1943|BOTSWANA||Mitchell.Valencia@YZ4VMGC.org|2452412| +6690|AAAAAAAACCKBAAAA|1128506|1934|35809|2451683|2451653|Ms.|Marsha|Hackett|Y|23|12|1958|ERITREA||Marsha.Hackett@VNOHnflxv05X.com|2452616| +6691|AAAAAAAADCKBAAAA|1354015|1762|8309|2449577|2449547|Miss|Lila|Theriot|N|23|9|1964|CHRISTMAS ISLAND||Lila.Theriot@aV.edu|2452507| +6692|AAAAAAAAECKBAAAA|192323|7086|45896|2452515|2452485|Dr.|Lisa|Hyde|Y|22|9|1970|RUSSIAN FEDERATION||Lisa.Hyde@oLRv7rzF.edu|2452618| +6693|AAAAAAAAFCKBAAAA|108560|6141|4075|2451971|2451941|Sir|Ryan|Simpson|N|12|12|1989|MAURITANIA||Ryan.Simpson@yNFd0BbZAa.edu|2452304| +6694|AAAAAAAAGCKBAAAA|749248|467|10086|2450764|2450734|Miss|Ngoc|Rucker|Y|23|3|1924|GREECE||Ngoc.Rucker@zkjranFnVoO.org|2452384| +6695|AAAAAAAAHCKBAAAA|615408|430|21099|2452310|2452280|Sir|Michael|Chaney|N|9|1|1935|PORTUGAL||Michael.Chaney@EFHp1FOUb.org|2452507| +6696|AAAAAAAAICKBAAAA|1540032|5574|316|2451220|2451190|Dr.|Robert|Moore|Y|14|6|1969|VANUATU||Robert.Moore@BQqV.edu|2452340| +6697|AAAAAAAAJCKBAAAA|1035522|2985|42801|2450874|2450844|Dr.|Sammy|Downey|Y|26|3|1952|BELIZE||Sammy.Downey@y0GP9svejPuoR.com|2452582| +6698|AAAAAAAAKCKBAAAA|1607889|824|38912|2450942|2450912|Dr.|Amber|Booth|N|24|9|1958|JAPAN||Amber.Booth@9vtSiT0pUdJyBUxon.org|2452456| +6699|AAAAAAAALCKBAAAA|1040749|2133|5964|2449299|2449269|Mrs.|Kathleen|Rush|N|19|9|1942|GUINEA-BISSAU||Kathleen.Rush@aPuB79zAaYp6JKq.org|2452574| +6700|AAAAAAAAMCKBAAAA|1769165|2507|296|2452041|2452011|Miss|Edith|Petersen|Y|14|5|1952|FINLAND||Edith.Petersen@6H6ZkPieXxd4z.edu|2452525| +6701|AAAAAAAANCKBAAAA|542601|1948|11295|2451325|2451295|Mrs.|Raquel|Marks|N|21|3|1992|ANTIGUA AND BARBUDA||Raquel.Marks@1LO.com|2452631| +6702|AAAAAAAAOCKBAAAA|612183|4274|6318|2449385|2449355|Dr.|Paul|Lopez|Y|5|1|1973|C�TE D'IVOIRE||Paul.Lopez@vtVA7RhkYvo1BRpa8.edu|2452512| +6703|AAAAAAAAPCKBAAAA|1897408|2847|15086|2451033|2451003|Dr.|Adam|Talbot|N|5|8|1978|GAMBIA||Adam.Talbot@10zxOPTu1EqaD.com|2452601| +6704|AAAAAAAAADKBAAAA|1043224|1614|36051|2451378|2451348|Dr.|Kanisha|Hanson|Y|20|8|1956|MALDIVES||Kanisha.Hanson@9A1r5I9a8p.com|2452582| +6705|AAAAAAAABDKBAAAA|28843|4461|22126|2449878|2449848|Miss|Carolyn|Hill|N|27|3|1974|HUNGARY||Carolyn.Hill@l9yUekcp4rO.org|2452466| +6706|AAAAAAAACDKBAAAA|||9900||2448999|Miss||Gill|N|20|||BARBADOS|||| +6707|AAAAAAAADDKBAAAA|265405|1905|15158|2450291|2450261|Sir|Tyler|Wolfe|N|29|9|1958|ISLE OF MAN||Tyler.Wolfe@4kBeho9zUHn.edu|2452472| +6708|AAAAAAAAEDKBAAAA|981819|690|31161|2451822|2451792|Mr.|Joshua|Walters|N|1|4|1990|KENYA||Joshua.Walters@EaIhQ2txJPZrcV.org|2452361| +6709|AAAAAAAAFDKBAAAA|556597|1290|45983|2451447|2451417|Mrs.|Suzanne|Lowe|N|4|2|1991|GUINEA||Suzanne.Lowe@8zS6M5bdSg.edu|2452324| +6710|AAAAAAAAGDKBAAAA|1227574|6071|36168|2449809|2449779|Dr.|Richard|Long|N|2|2|1946|UZBEKISTAN||Richard.Long@pAjeKNQzOUpnB.edu|2452341| +6711|AAAAAAAAHDKBAAAA|1285690|3194|15451|2450893|2450863|Sir|Ronald|Pope|N|7|7|1924|SOMALIA||Ronald.Pope@L5RNZacf1eD99FB.com|2452374| +6712|AAAAAAAAIDKBAAAA|699617|6826|18882|2451205|2451175|Mr.|George|Crawford|Y|9|10|1970|EGYPT||George.Crawford@rbsIPUc5n9.edu|2452485| +6713|AAAAAAAAJDKBAAAA|832366|3109|27052|2450206|2450176|Sir|Tim|Monroe|Y|3|9|1941|ROMANIA||Tim.Monroe@4Vq7M.edu|2452439| +6714|AAAAAAAAKDKBAAAA|702024|3300|12576|2450050|2450020|Mrs.|Lucy|Barber|N|16|5|1950|RWANDA||Lucy.Barber@LjS8EEMClS4oZ.edu|2452576| +6715|AAAAAAAALDKBAAAA|5436|1852|33068|2452259|2452229|Dr.|Latasha|Roldan|N|7|4|1964|TOGO||Latasha.Roldan@ngcnnzO7BKZZtno311.org|2452482| +6716|AAAAAAAAMDKBAAAA|94927|1160|23440|2452512|2452482|Dr.|Mai|Otero|Y|17|6|1935|KENYA||Mai.Otero@gl37DMx3.edu|2452588| +6717|AAAAAAAANDKBAAAA|758113|3450|44787|2451120|2451090|Mr.|John|Contreras|Y|3|8|1989|CHINA||John.Contreras@FN.com|2452462| +6718|AAAAAAAAODKBAAAA||98|4711|2451984||Miss|Angel||Y||4|1976|HONG KONG|||| +6719|AAAAAAAAPDKBAAAA|336842|5515|9341|2449167|2449137|Sir|Eric|Fenton|Y|2|7|1944|KIRIBATI||Eric.Fenton@N.edu|2452399| +6720|AAAAAAAAAEKBAAAA|1426191|4619|9800|2449501|2449471|Miss|Clara|Bryant|N|14|1|1976|NORFOLK ISLAND||Clara.Bryant@K7YsHzhK03j4.edu|2452406| +6721|AAAAAAAABEKBAAAA|79637|5918|43799|2451300|2451270|Dr.|Etta|Whiteside|Y|13|7|1976|BELIZE||Etta.Whiteside@5GPVAe3uxIa7HL.com|2452409| +6722|AAAAAAAACEKBAAAA|884011|6776|35894|2451910|2451880|Ms.|Pamela|Lewis|Y|20|9|1930|SERBIA||Pamela.Lewis@7I.com|2452496| +6723|AAAAAAAADEKBAAAA|1110034|3604|11174|2451158|2451128|Dr.|Timothy|Park|Y|1|6|1946|PORTUGAL||Timothy.Park@ckqoN92FT4.edu|2452456| +6724|AAAAAAAAEEKBAAAA|658849|639|24382|2449719|2449689|Sir|Chad|Hardy|Y|8|7|1957|CUBA||Chad.Hardy@MSEOb2MPlNVOi.edu|2452553| +6725|AAAAAAAAFEKBAAAA|1053398|4285|23525|2451504|2451474|Ms.|Maribeth|Buchanan|N|18|2|1985|WESTERN SAHARA||Maribeth.Buchanan@0Ga0JHZmxCnQi9.edu|2452288| +6726|AAAAAAAAGEKBAAAA|3803|2326|34849|2452426|2452396|Ms.|Dianna|Miller|N|27|4|1924|WALLIS AND FUTUNA||Dianna.Miller@xGu7bZ.edu|2452470| +6727|AAAAAAAAHEKBAAAA|1275981|6694|39407|2450855|2450825|Sir|Napoleon|Davenport|Y|26|12|1954|MONTSERRAT||Napoleon.Davenport@V12XZFlyhqhSaE.edu|2452388| +6728|AAAAAAAAIEKBAAAA|406282|3927|31525|2449397|2449367|Sir|Lawrence|Fischer|N|25|12|1991|TOGO||Lawrence.Fischer@PyJ8XHYcuE.org|2452512| +6729|AAAAAAAAJEKBAAAA|1079054|636|28618|2450654|2450624|Ms.|Nellie|Patrick|N|31|12|1944|UKRAINE||Nellie.Patrick@XYoC3yl4iqnuNK.com|2452493| +6730|AAAAAAAAKEKBAAAA|497791|6543|14542|2450187|2450157|Mrs.|Dorothy|Moss|Y|3|1|1930|FRANCE||Dorothy.Moss@6l3KCkB.com|2452494| +6731|AAAAAAAALEKBAAAA|337892|5463|39787|2451023|2450993|Ms.|Katherine|George|Y|5|8|1930|NAMIBIA||Katherine.George@Gz3VNPcrFNYPn.org|2452378| +6732|AAAAAAAAMEKBAAAA|887966|1555|12391|2452488|2452458|Dr.|Heather|Fox|Y|2|3|1946|LIBERIA||Heather.Fox@41l4dLzCsJliAM43LR.org|2452535| +6733|AAAAAAAANEKBAAAA|1447845|5655|19818|2451255|2451225|Sir|Carlos|Schmidt|Y|15|11|1952|NICARAGUA||Carlos.Schmidt@LRhGdk.org|2452312| +6734|AAAAAAAAOEKBAAAA|1775926|4036|32499|2452431|2452401|Ms.|Adam|Rauch|Y|10|12|1957|GABON||Adam.Rauch@8HKXKO.org|2452637| +6735|AAAAAAAAPEKBAAAA|1729470|3686|13055|2449954|2449924|Dr.|Carol|Garza|Y|26|12|1989|OMAN||Carol.Garza@eG2jBJNXLcRuC5.org|2452504| +6736|AAAAAAAAAFKBAAAA|136083|1052|30821|2452281|2452251|Dr.|David|Guevara|Y|11|10|1951|TURKEY||David.Guevara@5RlqkKedq0.org|2452631| +6737|AAAAAAAABFKBAAAA|115150|293|36189|2450936|2450906|Mr.|Lawrence|Ortiz|N|19|7|1953|C�TE D'IVOIRE||Lawrence.Ortiz@m1AzSq7xpFcysJJHF.edu|2452346| +6738|AAAAAAAACFKBAAAA|620407|6808|856|2451628|2451598|Miss|Lisa|Goodman|N|26|7|1953|KUWAIT||Lisa.Goodman@JlQ3h5b501lI.org|2452621| +6739|AAAAAAAADFKBAAAA|322572|782|13439|2450576|2450546|Dr.|Jason|Andrews|Y|18|6|1946|MALTA||Jason.Andrews@ryPU.com|2452462| +6740|AAAAAAAAEFKBAAAA|1253733|5664|19595|2451117|2451087|Miss|Gladys|Milne|Y|2|9|1980|PHILIPPINES||Gladys.Milne@sCbvfuMznCtkN.com|2452581| +6741|AAAAAAAAFFKBAAAA|751512|4110|43555|2452569|2452539|Mrs.|Donya|Furlong|Y|26|9|1925|SAINT LUCIA||Donya.Furlong@VAm2p3kJiiIGv9yY.edu|2452549| +6742|AAAAAAAAGFKBAAAA|1479702|4793|19037|2452254|2452224|Ms.|Rafaela|Chin|N|16|4|1937|NAURU||Rafaela.Chin@K.com|2452518| +6743|AAAAAAAAHFKBAAAA|155132|729|35526|2451585|2451555|Miss|April|Fuentes|Y|22|7|1927|GABON||April.Fuentes@XBS3zSzhYcuQ.edu|2452406| +6744|AAAAAAAAIFKBAAAA|996097|1846|42795|2450929|2450899|Mr.|Luther|Reed|Y|18|5|1961|VIET NAM||Luther.Reed@DltqBfvBey7.org|2452341| +6745|AAAAAAAAJFKBAAAA|100039|3952|1465|2449889|2449859|Mrs.|Lora|Ramsey|Y|9|2|1973|MALI||Lora.Ramsey@s0jhx7Lho2c.edu|2452505| +6746|AAAAAAAAKFKBAAAA|700256|2138|13173|2450926|2450896|Mr.|Robert|Slaughter|N|6|1|1926|SWITZERLAND||Robert.Slaughter@ip8JZ9Cn0EXaE.edu|2452551| +6747|AAAAAAAALFKBAAAA|1005938|2231|31724|2450713|2450683|Mr.|Joe|Williams|Y|21|7|1986|CUBA||Joe.Williams@sy3pygTJY60QVXNN3.org|2452611| +6748|AAAAAAAAMFKBAAAA||6953|33541|||Dr.|Johnny|Dugger||13|8||TURKMENISTAN||Johnny.Dugger@9FB.org|| +6749|AAAAAAAANFKBAAAA|347243|3424|11205|2450282|2450252|Miss|Carolyn|Barnes|N|4|3|1988|MYANMAR||Carolyn.Barnes@0ClI32iB52.com|2452575| +6750|AAAAAAAAOFKBAAAA|1497701|5200|25099|2450132|2450102|Mr.|Randolph|Horst|Y|15|1|1974|FRENCH GUIANA||Randolph.Horst@OGyTm4zgegham8h.org|2452602| +6751|AAAAAAAAPFKBAAAA|312151|3192|16233|2450353|2450323|Dr.|Marsha|Mack|Y|4|9|1969|PARAGUAY||Marsha.Mack@dVammxFuIcZZ.edu|2452337| +6752|AAAAAAAAAGKBAAAA|546248|5554|27260|2452592|2452562|Ms.|Diann|Washburn|N|27|7|1967|UGANDA||Diann.Washburn@A.com|2452561| +6753|AAAAAAAABGKBAAAA|1041341|616|19091|2450567|2450537|Mr.|Antwan|||6||1969|WESTERN SAHARA|||2452379| +6754|AAAAAAAACGKBAAAA|1333608|2781|3507|2452091|2452061|Ms.|Brenda|Davis|Y|5|12|1925|SENEGAL||Brenda.Davis@ydOC0E0.com|2452623| +6755|AAAAAAAADGKBAAAA|1232131|7195|20659|2450477|2450447|Dr.|Warren|Roberts|N|11|8|1938|GUINEA||Warren.Roberts@SmnO.edu|2452534| +6756|AAAAAAAAEGKBAAAA|412483|1722|30914|2452274|2452244|Ms.|Christine|Maddox|N|5|6|1929|EGYPT||Christine.Maddox@ACAiZGEbL5DH.com|2452353| +6757|AAAAAAAAFGKBAAAA|314125|495|25665|2451565|2451535|Ms.|Betty|Haas|N|23|5|1961|GEORGIA||Betty.Haas@0cDCJJZ8dFzqK9ccsU8.com|2452603| +6758|AAAAAAAAGGKBAAAA|328248|1029|41857|2451123|2451093|Mrs.|Sarah|Long|Y|11|7|1984|JAPAN||Sarah.Long@uDIsxbf9yOGAzfa6.com|2452541| +6759|AAAAAAAAHGKBAAAA|189082|5941|26400|2452661|2452631|Ms.|Marilyn|Vera|Y|25|8|1932|AUSTRALIA||Marilyn.Vera@OQYseLSC80iquZ.org|2452553| +6760|AAAAAAAAIGKBAAAA|1135282|730|2178|2449452|2449422|Mr.|Richard|Richardson|N|10|7|1970|MAURITIUS||Richard.Richardson@56AiuDOCZ3yF27IzY4B.com|2452345| +6761|AAAAAAAAJGKBAAAA|||40441|2452007||Mr.|John|Oliver||8||1973||||2452414| +6762|AAAAAAAAKGKBAAAA|367131|4797|16510|2452509|2452479|Mrs.|Teresa|Gruber|Y|21|9|1947|CAYMAN ISLANDS||Teresa.Gruber@uo2rXCJhdlf2xLBnJ.org|2452359| +6763|AAAAAAAALGKBAAAA|936488|3713|938|2450637|2450607|Dr.|Paul|Williams|Y|21|6|1956|UZBEKISTAN||Paul.Williams@Qra8h5sX8j.org|2452521| +6764|AAAAAAAAMGKBAAAA|1601254|6075|27296|2451354|2451324|Dr.|Marc|Brown|N|19|9|1955|BELIZE||Marc.Brown@A1Gca8uulr.com|2452502| +6765|AAAAAAAANGKBAAAA|1257995|6199|32756|2450361|2450331|Sir|Ramon|Hart|N|16|10|1960|GUAM||Ramon.Hart@uDl3eXS.edu|2452497| +6766|AAAAAAAAOGKBAAAA|437766|1074|35535|2449103|2449073|Mr.|Carl|Garza|Y|11|8|1986|MOROCCO||Carl.Garza@q1ksp.org|2452582| +6767|AAAAAAAAPGKBAAAA|1292543|2726|38909|2449167|2449137|Sir|Thomas|Gilbert|N|24|10|1955|LITHUANIA||Thomas.Gilbert@ucOJrhaBhSkdbe.edu|2452640| +6768|AAAAAAAAAHKBAAAA|359617|5622|41580|2451647|2451617|Miss|Cassandra|Rodriguez|N|18|1|1959|SWAZILAND||Cassandra.Rodriguez@Qa3xM.edu|2452532| +6769|AAAAAAAABHKBAAAA|1321631|2580|12729|2449863|2449833|Sir|Scott|Keys|N|7|11|1986|MOLDOVA, REPUBLIC OF||Scott.Keys@2u5ElF2Gxv8NaJyz9vSp.com|2452328| +6770|AAAAAAAACHKBAAAA|1049051|6866|35834|2450120|2450090|Dr.|Rena|Jacobs|N|6|4|1937|PUERTO RICO||Rena.Jacobs@ah.org|2452400| +6771|AAAAAAAADHKBAAAA|1676528|3740|46552|2452226|2452196|Dr.|Benito|Brown|N|17|4|1990|TONGA||Benito.Brown@3De.com|2452357| +6772|AAAAAAAAEHKBAAAA|1113897|3492|43934|2452284|2452254|Ms.|Georgette|Hobbs|N|22|2|1970|BELARUS||Georgette.Hobbs@h9DQH0G66XDX5bMRPXFp.edu|2452544| +6773|AAAAAAAAFHKBAAAA|1885589|5517|31845|2452460|2452430|Dr.|Ruth|Garner|Y|26|5|1960|KOREA, REPUBLIC OF||Ruth.Garner@fZMU9UIAEcnd0i.org|2452347| +6774|AAAAAAAAGHKBAAAA|1906369|5509|1849|2452299|2452269|Mr.|Mason|Morin|Y|10|6|1989|UNITED STATES||Mason.Morin@Qhaj1L.com|2452546| +6775|AAAAAAAAHHKBAAAA||4927|17900|2450647|2450617||Bessie||||4||UGANDA||Bessie.Evans@4xOqti6pQMH.org|| +6776|AAAAAAAAIHKBAAAA|237770|5799|12181|2450769|2450739|Miss|Kimberly|Holmes|N|27|6|1981|BAHRAIN||Kimberly.Holmes@kmhqZcJ5srCmx.org|2452627| +6777|AAAAAAAAJHKBAAAA|1324510|1230|27950|2449459|2449429|Dr.|Millie|Duncan|N|26|7|1954|TUVALU||Millie.Duncan@FqZp1ItftqBHkSUxTA.edu|2452486| +6778|AAAAAAAAKHKBAAAA|418694|4090|38686||2452616|Dr.||Wu||2|5||||Joseph.Wu@atuvV5sh9lUgCQ.org|| +6779|AAAAAAAALHKBAAAA|1806400|2116|27385|2449950|2449920|Mrs.|Sierra|Gonzales|N|17|12|1977|SOUTH AFRICA||Sierra.Gonzales@H2.org|2452339| +6780|AAAAAAAAMHKBAAAA|1884642|6828|46214|2452006|2451976|Sir|Dana|King|N|21|11|1972|PERU||Dana.King@RI9gX30X.edu|2452586| +6781|AAAAAAAANHKBAAAA|987399|5056|31513|2449402|2449372|Sir|Justin|Thornhill|N|26|5|1932|HONG KONG||Justin.Thornhill@2Phy2jZXXz8kDPA.org|2452388| +6782|AAAAAAAAOHKBAAAA|1555707|3812|36717|2451320|2451290|Sir|Henry|Haley|Y|15|4|1932|RUSSIAN FEDERATION||Henry.Haley@YN1k.org|2452531| +6783|AAAAAAAAPHKBAAAA|510803|6247|16444|2449155|2449125|Mr.|Philip|Shaver|N|4|12|1961|ESTONIA||Philip.Shaver@Le57KQqEBoO5US6.com|2452572| +6784|AAAAAAAAAIKBAAAA|700194|4342|30490|2450885|2450855|Sir|Bill|Rios|Y|7|7|1977|FRENCH POLYNESIA||Bill.Rios@r9.com|2452529| +6785|AAAAAAAABIKBAAAA|1535746|2698|10050|2451170|2451140|Miss|Adam|Baldwin|N|22|4|1979|CAYMAN ISLANDS||Adam.Baldwin@G.com|2452313| +6786|AAAAAAAACIKBAAAA|786806|7132|39058|2449617|2449587|Mrs.|Sue|Dalton|Y|30|10|1972|YEMEN||Sue.Dalton@uft9cyLACG4xxZOZF.edu|2452287| +6787|AAAAAAAADIKBAAAA|1003857|1024|27744|2452022|2451992|Mr.|Wayne|Richards|Y|26|7|1983|ITALY||Wayne.Richards@xknZ.edu|2452599| +6788|AAAAAAAAEIKBAAAA|110596|508|12373|2449867|2449837|Sir|Melvin|Doherty|N|5|10|1987|QATAR||Melvin.Doherty@G0Drhi5yVnc2f.org|2452362| +6789|AAAAAAAAFIKBAAAA|1079309|3230|31598|2452187|2452157|Ms.|Kathryn|Jones|N|5|4|1932|FRANCE||Kathryn.Jones@j83qTI6ALc1GYX.org|2452474| +6790|AAAAAAAAGIKBAAAA|275|2105|44335|2452475|2452445|Ms.|Michele|Gall|N|1|8|1933|CROATIA||Michele.Gall@vZTtHusxZFI5YF.com|2452403| +6791|AAAAAAAAHIKBAAAA|218069|651|29280|2449355|2449325|Miss|Sanda|Miller|N|21|9|1945|NEPAL||Sanda.Miller@LVEnIEZ0xV9.com|2452574| +6792|AAAAAAAAIIKBAAAA|749416|3117|30465|2452032|2452002|Miss|Joyce|Roe|Y|6|1|1961|GABON||Joyce.Roe@BzeyRZ9nyt7M0rsm.org|2452421| +6793|AAAAAAAAJIKBAAAA|387925|740|31508|2450328|2450298|Ms.|Joanne|Schaeffer|N|18|4|1982|NIGERIA||Joanne.Schaeffer@2yAIm.com|2452435| +6794|AAAAAAAAKIKBAAAA|1564074|5568|9015|2451092|2451062|Sir|Ralph|Hernandez|N|5|6|1944|WALLIS AND FUTUNA||Ralph.Hernandez@LE1jor25Xz1.edu|2452566| +6795|AAAAAAAALIKBAAAA|423688|1157|48626|2451468|2451438|Sir|Paul|Murphy|N|30|11|1930|COSTA RICA||Paul.Murphy@7EL3.edu|2452344| +6796|AAAAAAAAMIKBAAAA|409013|3155|4050|2449161|2449131|Ms.|Patsy|Green|N|7|9|1929|VANUATU||Patsy.Green@jkgA.com|2452365| +6797|AAAAAAAANIKBAAAA|280142|2274|41206|2451139|2451109|Mr.|Elroy|Richardson|N|9|4|1974|EGYPT||Elroy.Richardson@y3EaA5PaB0SA.org|2452492| +6798|AAAAAAAAOIKBAAAA|826280|5541|26755|2451752|2451722|Mr.|William|Howard|Y|10|9|1992|TURKEY||William.Howard@Jc9TmJkCio.com|2452513| +6799|AAAAAAAAPIKBAAAA||1040|42394|2452403|2452373|Mrs.|Amy|Guzman|||10|1926|||Amy.Guzman@jxR.edu|| +6800|AAAAAAAAAJKBAAAA|1376728|1910|49492|2452637|2452607|Dr.|Levi|Waller|N|1|7|1986|GREENLAND||Levi.Waller@eRhvSQBCRNesauaToO.org|2452567| +6801|AAAAAAAABJKBAAAA|231997|4981|20812|2449060|2449030|Sir|Bryan|Mayer|Y|5|6|1934|GUYANA||Bryan.Mayer@BSdAVOjSY.com|2452436| +6802|AAAAAAAACJKBAAAA|316734|3978|24869|2451012|2450982|Ms.|Lisa|Stauffer|N|6|6|1957|NIGER||Lisa.Stauffer@y8rb5FeYPuf.com|2452597| +6803|AAAAAAAADJKBAAAA|1618139|6423|41107|2452146|2452116|Dr.|Charlie|Bradshaw|Y|13|9|1988|UNITED KINGDOM||Charlie.Bradshaw@jpRuDR.org|2452526| +6804|AAAAAAAAEJKBAAAA|475019|3254|7089|2450658|2450628|Dr.|Meredith|Clark|Y|17|3|1938|SIERRA LEONE||Meredith.Clark@pjtxClMgxcxD3fSP.edu|2452393| +6805|AAAAAAAAFJKBAAAA|185104|5109|9114|2449639|2449609|Mrs.|Janice|Nelms|Y|10|6|1941|JERSEY||Janice.Nelms@p3U.org|2452347| +6806|AAAAAAAAGJKBAAAA|1365601|2528|29855|2450245|2450215|Sir|Jose|Morales|N|6|6|1962|JERSEY||Jose.Morales@0R1eUiLdShx.com|2452643| +6807|AAAAAAAAHJKBAAAA|915812|5719|1626|2451097|2451067|Ms.|Janice|Fields|Y|7|8|1942|CAPE VERDE||Janice.Fields@H.org|2452424| +6808|AAAAAAAAIJKBAAAA|1393551|4732|11160|2451508|2451478|Dr.|Helen|Barron|N|6|4|1925|MOROCCO||Helen.Barron@32.org|2452568| +6809|AAAAAAAAJJKBAAAA|1769217|3019|8868|2451511|2451481|Mrs.|Rita|Oliver|N|15|10|1925|FRENCH POLYNESIA||Rita.Oliver@3v45Vb1bAdENkoI.com|2452379| +6810|AAAAAAAAKJKBAAAA||3535|29009||2450779|Miss|Renee|Pettway|N|10|11|1927|MOLDOVA, REPUBLIC OF||Renee.Pettway@aQkpXAfIIG84aEZFvVo.edu|2452463| +6811|AAAAAAAALJKBAAAA|394975|712|30096|2452060|2452030|Mrs.|Linda|Shively|Y|16|3|1961|EGYPT||Linda.Shively@0lySYR.org|2452598| +6812|AAAAAAAAMJKBAAAA|95961|1890|44219|2451424|2451394|Mrs.|Elizabeth|Potter|N|11|11|1973|ZAMBIA||Elizabeth.Potter@Hk4uY7SEDZlNgb6MZ.com|2452288| +6813|AAAAAAAANJKBAAAA|778739|3441|23343|2452499|2452469|Dr.|Gregory|Mott|Y|6|4|1926|CANADA||Gregory.Mott@vHovMh.org|2452358| +6814|AAAAAAAAOJKBAAAA|1368906|5139|46125|2449628|2449598|Sir|John|Voigt|N|14|3|1981|MALAYSIA||John.Voigt@511PA7GdyJ3.org|2452490| +6815|AAAAAAAAPJKBAAAA|1100206|524|4795|2449795|2449765|Sir|Emilio|Colwell|Y|26|11|1932|GREECE||Emilio.Colwell@CLjeQkm5Z0IzhBKRfOc.org|2452565| +6816|AAAAAAAAAKKBAAAA|560619|2123|3627|2450161|2450131|Miss|Wanda|Wilder|Y|20|6|1979|PUERTO RICO||Wanda.Wilder@247EE6IzMzvz.com|2452419| +6817|AAAAAAAABKKBAAAA|1593885|6485|30685|2451205|2451175|Sir|Lee|Zavala|Y|12|4|1927|TUVALU||Lee.Zavala@x2yK8BUtxpFnZI.org|2452363| +6818|AAAAAAAACKKBAAAA|1217435|4673|16311|2452281|2452251|Mr.|Robert|Crump|N|17|12|1983|GEORGIA||Robert.Crump@MhFdvQP0fJAAI3P.org|2452526| +6819|AAAAAAAADKKBAAAA|1138422|3800|15978|2452108|2452078|Dr.|Alexander|Doty|N|29|9|1947|CUBA||Alexander.Doty@CPLr06JX.com|2452343| +6820|AAAAAAAAEKKBAAAA|1738324|1004|42884|2449139|2449109|Mr.|Frank|Hill|Y|21|2|1967|SENEGAL||Frank.Hill@XbvYu4LkfbF1tSgGBKO.org|2452363| +6821|AAAAAAAAFKKBAAAA|1693502|311|11120|2450071|2450041|Dr.|Margaret|Finn|N|25|4|1961|UNITED KINGDOM||Margaret.Finn@b26uBizN2cEG.org|2452441| +6822|AAAAAAAAGKKBAAAA|1154849|1578|41907|2449915|2449885|Ms.|Karmen|Ferguson|Y|1|1|1985|SAINT HELENA||Karmen.Ferguson@TZOakt.org|2452578| +6823|AAAAAAAAHKKBAAAA|237446|796|22146|2449388|2449358|Sir|Austin|Lang|Y|17|6|1985|IRELAND||Austin.Lang@OeukJUJz0T.org|2452284| +6824|AAAAAAAAIKKBAAAA|1841942|646|35797|2450028|2449998|Sir|Jason|Anderson|N|1|9|1937|LATVIA||Jason.Anderson@S.org|2452504| +6825|AAAAAAAAJKKBAAAA|1104627|5920|22366|2451111|2451081|Mr.|Richard|Johnson|Y|15|5|1925|ITALY||Richard.Johnson@BREUGni2m4r.com|2452448| +6826|AAAAAAAAKKKBAAAA|1915679|3179|3479|2449137|2449107|Ms.|Anna|Danielson|Y|3|3|1990|TURKEY||Anna.Danielson@Ci.com|2452601| +6827|AAAAAAAALKKBAAAA|1249229|6081|32580|2449930|2449900|Dr.|Josette|Charles|N|22|8|1935|EGYPT||Josette.Charles@1HbTx04x2KUHiTgT.org|2452645| +6828|AAAAAAAAMKKBAAAA|721094|3425|33403|2450588|2450558|Dr.|Howard|Elder|Y|3|1|1960|CANADA||Howard.Elder@HZRtZsUiJk.edu|2452466| +6829|AAAAAAAANKKBAAAA|1479148|3982|5969|2450293|2450263|Mr.|Robert|Welch|Y|14|11|1968|MALAWI||Robert.Welch@QriURduK.org|2452332| +6830|AAAAAAAAOKKBAAAA|632080|2074|38915|2450133|2450103|Miss|Estrella|Vickers|Y|11|4|1974|ROMANIA||Estrella.Vickers@RBjonQnh.org|2452560| +6831|AAAAAAAAPKKBAAAA||5001|4926||2450943|||Keeling|N||3||||Corey.Keeling@C.edu|| +6832|AAAAAAAAALKBAAAA|218664|3461|35458|2450154|2450124|Mrs.|Alex|Gutierrez|Y|16|8|1930|EL SALVADOR||Alex.Gutierrez@xjguP7vsttHKyy.edu|2452406| +6833|AAAAAAAABLKBAAAA|1024757|5436|4915|2452463|2452433|Dr.|Scott|Willis|N|5|9|1939|TAJIKISTAN||Scott.Willis@O5.com|2452366| +6834|AAAAAAAACLKBAAAA|44410|1056|22455|2452610|2452580|Dr.|Jose|Pollard|Y|14|9|1978|MOZAMBIQUE||Jose.Pollard@uPNLLhstC.edu|2452306| +6835|AAAAAAAADLKBAAAA|681982|6352|41922|2449549|2449519|Mr.|Samuel|Marks|N|3|9|1966|MACAO||Samuel.Marks@q0OXMAScq78.com|2452303| +6836|AAAAAAAAELKBAAAA|1108541|2314|35687|2452063|2452033|Mr.|Mathew|Knight|N|7|2|1945|SWAZILAND||Mathew.Knight@MUrgs7zruV5z5.com|2452330| +6837|AAAAAAAAFLKBAAAA|1405309|2402|10295|2450509|2450479|Mr.|Jacob|Dove|N|18|10|1925|PAKISTAN||Jacob.Dove@5ovnszsylaVKp.org|2452319| +6838|AAAAAAAAGLKBAAAA|1583623|1100|36177|2451813|2451783|Mr.|Larry|Dawson|Y|4|12|1965|CHILE||Larry.Dawson@Srub0O.com|2452603| +6839|AAAAAAAAHLKBAAAA|609268|1486|23351|2451451|2451421|Mr.|Curtis|Pereira|Y|26|11|1967|HAITI||Curtis.Pereira@c65XUp768AZ.edu|2452437| +6840|AAAAAAAAILKBAAAA|1052496|3581|35913|2451704|2451674|Ms.|Monique|Johnson|N|29|9|1961|LIECHTENSTEIN||Monique.Johnson@a.org|2452287| +6841|AAAAAAAAJLKBAAAA|310148|2140|39732|2452269|2452239|Dr.|Virginia|Sutton|N|21|12|1925|ROMANIA||Virginia.Sutton@ruoDal9Y9.org|2452645| +6842|AAAAAAAAKLKBAAAA|629029|3216|42203|2450043|2450013|Dr.|Amanda|Walker|Y|27|7|1953|AMERICAN SAMOA||Amanda.Walker@KRN93fLTD.com|2452468| +6843|AAAAAAAALLKBAAAA|69901|5584|420|2450707|2450677|Mr.|Jose|Hannah|N|1|1|1941|AFGHANISTAN||Jose.Hannah@y4y2.org|2452287| +6844|AAAAAAAAMLKBAAAA|1483862|4067|23872|2451937|2451907|Mr.|Robert|Thomas|Y|3|7|1955|SOLOMON ISLANDS||Robert.Thomas@Po7fsFaVjiPa.com|2452359| +6845|AAAAAAAANLKBAAAA|256613|2565|26117|2450875|2450845|Ms.|Nilda|Waite|Y|30|12|1947|ZIMBABWE||Nilda.Waite@42FbPrmIg5Hj.org|2452617| +6846|AAAAAAAAOLKBAAAA|263056|1944|44479|2452157|2452127|Miss|Rhonda|Mcdonald|N|24|8|1945|KAZAKHSTAN||Rhonda.Mcdonald@2YGT0NNB.com|2452521| +6847|AAAAAAAAPLKBAAAA|205504|2766|46028|2452296|2452266|Mrs.|Denise|Thornburg|Y|18|6|1927|PORTUGAL||Denise.Thornburg@crtney5l.edu|2452600| +6848|AAAAAAAAAMKBAAAA|624836|265|13341|2451141|2451111|Dr.|Arthur|Burdette|Y|3|11|1938|DOMINICA||Arthur.Burdette@sDlbMY60TDFMDQqoa.org|2452378| +6849|AAAAAAAABMKBAAAA|860275|2289|21056|2452305|2452275|Ms.|Brandi|Hall|N|9|11|1950|TUNISIA||Brandi.Hall@QB8Nc7j4.com|2452376| +6850|AAAAAAAACMKBAAAA|1048231|6657|39036|2451910|2451880|Ms.|Nicole|Barth|Y|26|6|1966|GUINEA||Nicole.Barth@xpnS5mYGx39eeVPdV.edu|2452492| +6851|AAAAAAAADMKBAAAA|592301|5504|45060|2449237|2449207|Miss|Phyllis|Perkins|Y|10|7|1926|GHANA||Phyllis.Perkins@OntDbRsCHcioSCO78f.com|2452446| +6852|AAAAAAAAEMKBAAAA|1337627|5991|10755|2452266|2452236|Dr.|Deanna|Hines|N|13|1|1991|SOLOMON ISLANDS||Deanna.Hines@a.org|2452501| +6853|AAAAAAAAFMKBAAAA|1626608|1498|5541|2450269|2450239|Mrs.|Ethel|Hopkins|Y|19|12|1936|AMERICAN SAMOA||Ethel.Hopkins@0pysFyEba1OSs.edu|2452307| +6854|AAAAAAAAGMKBAAAA|839193|4771|17949|2450360|2450330|Mrs.|Kristin|Hart|N|25|11|1992|ARMENIA||Kristin.Hart@81Mh.edu|2452534| +6855|AAAAAAAAHMKBAAAA|1600751|4500|32249|2451801|2451771|Mr.|Rafael|Evans|N|2|5|1954|CHRISTMAS ISLAND||Rafael.Evans@OHXuHMs.org|2452488| +6856|AAAAAAAAIMKBAAAA|984690|4371|5|2449964|2449934|Dr.|Jorge|Smith|N|4|2|1935|UGANDA||Jorge.Smith@hkOEhqq7hrZHRN.edu|2452445| +6857|AAAAAAAAJMKBAAAA|585027|3464|24701|2450585|2450555|Mrs.|Nancy|Ingram|Y|4|10|1961|BULGARIA||Nancy.Ingram@z4Ml7j02rT.org|2452318| +6858|AAAAAAAAKMKBAAAA|1429159|4532|47458|2450191|2450161|Mrs.|Jacquline|Lovejoy|N|18|8|1975|MOZAMBIQUE||Jacquline.Lovejoy@xIUktvZJA.edu|2452648| +6859|AAAAAAAALMKBAAAA|1755868|1504|49493|2450263|2450233|Dr.|Anneliese|Reed|Y|6|2|1988|FAROE ISLANDS||Anneliese.Reed@dnvUtrDytzSniP.org|2452533| +6860|AAAAAAAAMMKBAAAA|607819||19240||||Jason|Byrd||24||1991|ANTARCTICA||Jason.Byrd@3fZNr1XT2lh.edu|| +6861|AAAAAAAANMKBAAAA|1542060|5373|43439|2452165|2452135|Dr.|Lisa|Snell|N|17|4|1942|PERU||Lisa.Snell@TK.com|2452392| +6862|AAAAAAAAOMKBAAAA|1897966|3573|40427|2451834|2451804|Mr.|Dennis|Taylor|N|2|9|1988|ARUBA||Dennis.Taylor@m5LtsD.edu|2452544| +6863|AAAAAAAAPMKBAAAA|1748030|2321|9765|2450135|2450105|Dr.|William|Vega|N|24|3|1926|SYRIAN ARAB REPUBLIC||William.Vega@hLtNbaVEv.edu|2452563| +6864|AAAAAAAAANKBAAAA|671778|3760|35695|2452159|2452129|Sir|Jimmy|Nelson|N|18|8|1947|PALAU||Jimmy.Nelson@NMSlkuB.com|2452437| +6865|AAAAAAAABNKBAAAA|690960|1040|39473|2452036|2452006|Dr.|William|Chase|Y|24|10|1970|SWAZILAND||William.Chase@gHdoxP3x2zsB.edu|2452563| +6866|AAAAAAAACNKBAAAA||5807|43518|||Dr.|||Y|8||||||| +6867|AAAAAAAADNKBAAAA|1682663|2927|28791|2451280|2451250|Miss|Ellen|Soto|Y|8|3|1974|FIJI||Ellen.Soto@x1TUfdb7Zqo2yD.edu|2452518| +6868|AAAAAAAAENKBAAAA|533243|378|3560|2450505|2450475|Mr.|Carl|Duran|Y|8|4|1949|YEMEN||Carl.Duran@qa5PaMZAilmxh.org|2452632| +6869|AAAAAAAAFNKBAAAA|230273|6277|5928|2450584|2450554|Mr.|Joseph|Bailey|N|12|7|1949|UGANDA||Joseph.Bailey@SihDtjiFI.edu|2452363| +6870|AAAAAAAAGNKBAAAA|1133654|3471|39288|2450962|2450932|Dr.|Chance|Glenn|Y|8|1|1949|PITCAIRN||Chance.Glenn@fnYBQPjxxgLyt7VEoQ.org|2452579| +6871|AAAAAAAAHNKBAAAA|406084|906|43775|2450081|2450051|Sir|Travis|Munoz|N|5|8|1961|SEYCHELLES||Travis.Munoz@ZgpAQqZiaHdIQD.com|2452342| +6872|AAAAAAAAINKBAAAA|1579313|1228|1896|2449086|2449056|Sir|Leo|Robinson|Y|19|8|1948|THAILAND||Leo.Robinson@N2ADb4KA.com|2452544| +6873|AAAAAAAAJNKBAAAA|1844770|2459|101|2449527|2449497|Sir|Bryan|Stewart|N|31|1|1954|SAMOA||Bryan.Stewart@7.com|2452500| +6874|AAAAAAAAKNKBAAAA|707489|633|41777|2449363|2449333|Sir|Javier|Johnson|N|11|11|1924|BELARUS||Javier.Johnson@ukJlF.edu|2452500| +6875|AAAAAAAALNKBAAAA|1256243|2932|8421|2451966|2451936|Mr.|Leslie|Schneider|Y|17|12|1944|MALAYSIA||Leslie.Schneider@8L1yyLc83hv40.edu|2452459| +6876|AAAAAAAAMNKBAAAA|1417457|6990|16923|2451036|2451006|Dr.|John|Card|Y|10|5|1963|ISLE OF MAN||John.Card@K5JGKk.org|2452487| +6877|AAAAAAAANNKBAAAA|1051077|6100|8676|2451939|2451909|Ms.|Dora|Coyne|N|16|3|1961|SAINT HELENA||Dora.Coyne@At.com|2452308| +6878|AAAAAAAAONKBAAAA|1282433|1638|9643|2449263|2449233|Sir|Edward|Russell|Y|4|12|1985|LIBERIA||Edward.Russell@xL.edu|2452514| +6879|AAAAAAAAPNKBAAAA|954955|1580|17452|2450503|2450473|Ms.|Teresa|Covert|Y|10|11|1960|HUNGARY||Teresa.Covert@t0p.org|2452524| +6880|AAAAAAAAAOKBAAAA|761367|879|43144|2449341|2449311|Dr.|Arthur|Johnson|Y|23|6|1946|GREECE||Arthur.Johnson@lRzA6.com|2452504| +6881|AAAAAAAABOKBAAAA|1054976|1416|1856|2452105|2452075|Ms.|Doris|Mcpherson|N|10|9|1983|LESOTHO||Doris.Mcpherson@leDmYZtBl.com|2452424| +6882|AAAAAAAACOKBAAAA|467392|3122|13652|2449164|2449134|Dr.|Michael|Watson|N|16|12|1985|TUNISIA||Michael.Watson@Kyx8hMfhfy.org|2452419| +6883|AAAAAAAADOKBAAAA|1487312|2715|8871|2451173|2451143|Sir|Michael|Bullock|N|13|5|1979|COMOROS||Michael.Bullock@T8.com|2452415| +6884|AAAAAAAAEOKBAAAA|||4329|2452582|2452552|Mrs.||||||1940|||Betty.Paul@qAi0rARVyhV.org|| +6885|AAAAAAAAFOKBAAAA|50912|5747|28229|2452241|2452211|Miss|Natalie|Allen|N|17|10|1981|SURINAME||Natalie.Allen@QBK0SLgR.com|2452548| +6886|AAAAAAAAGOKBAAAA|10770|6815|45660|2451701|2451671|Mr.|Edward|Fernandez|N|22|7|1933|FIJI||Edward.Fernandez@KM.com|2452418| +6887|AAAAAAAAHOKBAAAA|516665|3865|5065|2450653|2450623|Dr.|Robert|Nugent|Y|6|6|1925|JORDAN||Robert.Nugent@GGHHBYO6CeX.edu|2452473| +6888|AAAAAAAAIOKBAAAA|66737|752|41967|2450941|2450911|Ms.|Samantha|Gilmore|Y|28|3|1987|GIBRALTAR||Samantha.Gilmore@0tbvlBAV.org|2452413| +6889|AAAAAAAAJOKBAAAA|558220|6427|28634|2451488|2451458|Dr.|John|Roberts|Y|21|8|1973|INDIA||John.Roberts@tFZh47pos3C3IPb.org|2452333| +6890|AAAAAAAAKOKBAAAA|401846|2569|3692|2452332|2452302|Mr.|Robert|Dew|N|29|2|1948|CAMEROON||Robert.Dew@4jSqnrtafNPQ3QPt.com|2452310| +6891|AAAAAAAALOKBAAAA|235668|6095|46969|2449403|2449373|Sir|Bradley|Brady|N|6|11|1948|GUINEA-BISSAU||Bradley.Brady@C9i9F6jH.com|2452596| +6892|AAAAAAAAMOKBAAAA|520384|4011|39991|2449175|2449145|Mr.|Thomas|Davis|Y|25|2|1987|JAMAICA||Thomas.Davis@vV.edu|2452471| +6893|AAAAAAAANOKBAAAA|676164|6158|8952|2451235|2451205|Miss|Wendy|Wilson|Y|3|11|1980|TURKEY||Wendy.Wilson@7kV8aaylayicgyXe.org|2452364| +6894|AAAAAAAAOOKBAAAA|1891424|6168|30912|2451895|2451865|Miss|Ethel|Mcclelland|Y|25|10|1953|ALBANIA||Ethel.Mcclelland@qDZqyDJ3jGOeC4YTe.com|2452579| +6895|AAAAAAAAPOKBAAAA|1676874|4806|39938|2452117|2452087|Mr.|Jon|Burke|N|26|7|1966|NORFOLK ISLAND||Jon.Burke@dS.org|2452633| +6896|AAAAAAAAAPKBAAAA|542899|3883|45620|2450695|2450665|Mrs.|Mamie|Smith|N|12|8|1976|TUNISIA||Mamie.Smith@PkdYQ1shNO4.com|2452400| +6897|AAAAAAAABPKBAAAA|1823136|1953|14770|2452348|2452318|Ms.|Margie|Reddy|N|19|4|1973|CROATIA||Margie.Reddy@pgFJ.org|2452536| +6898|AAAAAAAACPKBAAAA|383844|314|17184|2450923|2450893|Mrs.|Marsha|Calhoun|N|14|1|1979|SIERRA LEONE||Marsha.Calhoun@47aEYR6sLTV.com|2452385| +6899|AAAAAAAADPKBAAAA|1073231|523|5462|2450150|2450120|Sir|David|Henderson|N|25|3|1992|NIGER||David.Henderson@lIs1z.edu|2452576| +6900|AAAAAAAAEPKBAAAA|1809823|5658|293|2449686|2449656|Sir|Joey|Barnett|Y|11|11|1990|BARBADOS||Joey.Barnett@BvNQ1.edu|2452521| +6901|AAAAAAAAFPKBAAAA|412210|5155|48500|2449835|2449805|Sir|David|Leon|Y|19|12|1946|BRUNEI DARUSSALAM||David.Leon@2kpTKKFj.org|2452549| +6902|AAAAAAAAGPKBAAAA|1607425|4908|46744|2452442|2452412|Sir|Donald|Doyle|N|19|5|1970|PAPUA NEW GUINEA||Donald.Doyle@1dGoS60Trc5LMtn.org|2452425| +6903|AAAAAAAAHPKBAAAA|1246825|5477|17682|2451835|2451805|Mr.|Billy|Benavides|N|4|2|1934|MONTSERRAT||Billy.Benavides@Sq9S5zkXfkV.edu|2452415| +6904|AAAAAAAAIPKBAAAA|791688|1174|20260|2451356|2451326|Mr.|Frederick|Gallegos|N|16|4|1965|ICELAND||Frederick.Gallegos@RKzm9.edu|2452511| +6905|AAAAAAAAJPKBAAAA|1281019|6399|3332|2450492|2450462|Mr.|Thomas|Cash|N|26|3|1925|MAURITIUS||Thomas.Cash@de79btHUZqtm.com|2452539| +6906|AAAAAAAAKPKBAAAA|695684|849|23661|2450097|2450067|Mr.|Michael|Fahey|N|8|5|1983|LEBANON||Michael.Fahey@3FsKTZlcLtuzK6g1pV.org|2452341| +6907|AAAAAAAALPKBAAAA|1349783|3705|15180|2451452|2451422|Dr.|Luther|Henry|N|13|8|1928|BELARUS||Luther.Henry@LYmBHczMr5ck.org|2452533| +6908|AAAAAAAAMPKBAAAA|321644|1425|2006|2452303|2452273|Miss|Sarah|Landry|Y|22|4|1988|ARGENTINA||Sarah.Landry@Qydmv9uivq.org|2452397| +6909|AAAAAAAANPKBAAAA|620665|4263|48661|2450699|2450669|Miss|Latoya|Martin|N|4|10|1932|CHRISTMAS ISLAND||Latoya.Martin@5G7sZP.org|2452335| +6910|AAAAAAAAOPKBAAAA|1904635|1663|13538|2450208|2450178|Miss|Heather|Flowers|Y|5|6|1979|UKRAINE||Heather.Flowers@h.com|2452528| +6911|AAAAAAAAPPKBAAAA|600992|5529|35266|2452234|2452204|Sir|Brian|Murphy|N|25|1|1968|MONACO||Brian.Murphy@2Y0uSgckRrQ33BzNAs.edu|2452469| +6912|AAAAAAAAAALBAAAA|665911|2558|3605|2451986|2451956|Mr.|Michael|Headley|N|25|5|1952|IRELAND||Michael.Headley@nA0AR.com|2452514| +6913|AAAAAAAABALBAAAA|||20408|||Mr.|Thomas||||||||Thomas.Garcia@EpPBQ8OxUmGKTqA.edu|2452319| +6914|AAAAAAAACALBAAAA|1807667|3009|9151|2452465|2452435|Ms.|Jennifer|Johnson|N|6|11|1929|MONGOLIA||Jennifer.Johnson@pNK0C.org|2452316| +6915|AAAAAAAADALBAAAA|1078622|1944|22066|2449957|2449927|Dr.|Barbara|Jackson|N|12|4|1953|CANADA||Barbara.Jackson@6Q12dh4nDS9Ffh2pl.com|2452527| +6916|AAAAAAAAEALBAAAA|1004010|5074|29691|2451929|2451899|Miss|Carolyn|Ginn|Y|6|1|1960|MONTENEGRO||Carolyn.Ginn@x1jqIqY5bjfC.com|2452503| +6917|AAAAAAAAFALBAAAA|263327|488|26802|2450053|2450023|Dr.|Lucina|Gill|N|13|4|1969|GREECE||Lucina.Gill@xT10a.org|2452543| +6918|AAAAAAAAGALBAAAA|1250037|4190|45543|2452149|2452119|Mr.|James|Brown|N|17|4|1965|MONACO||James.Brown@N1kxz8DFB.com|2452589| +6919|AAAAAAAAHALBAAAA|1117639|2602|21918|2450754|2450724|Mr.|Kent|Dozier|N|26|4|1947|BENIN||Kent.Dozier@HE6rcM7dAGCDn8He.org|2452348| +6920|AAAAAAAAIALBAAAA|1420447|5822|28750|2450861|2450831|Sir|Benjamin|Lord|Y|22|4|1955|SENEGAL||Benjamin.Lord@l1yOckraBKj.org|2452465| +6921|AAAAAAAAJALBAAAA|912036|302|47768|2452532|2452502|Mr.|Stephan|Wall|N|18|2|1987|QATAR||Stephan.Wall@RaVHCtgHHK5nAR.org|2452448| +6922|AAAAAAAAKALBAAAA|1635124|2143|18809|2451130|2451100|Dr.|Larry|Burks|N|17|1|1989|OMAN||Larry.Burks@5MOtCM.edu|2452303| +6923|AAAAAAAALALBAAAA|1814850|5276|22502|2449430|2449400|Sir|John|Frantz|Y|1|10|1938|MOROCCO||John.Frantz@BcfQev9n3JLSg.edu|2452622| +6924|AAAAAAAAMALBAAAA|1599701|4851|4048|2450103|2450073|Dr.|Debora|Hynes|Y|21|11|1957|SLOVAKIA||Debora.Hynes@jbUNK.org|2452530| +6925|AAAAAAAANALBAAAA|329761|1197|23033|2449090|2449060|Sir|Daniel|Sosa|Y|3|9|1972|ANTARCTICA||Daniel.Sosa@VI0Ra2MT3S9.org|2452409| +6926|AAAAAAAAOALBAAAA|1260857|336|13415|2451572|2451542|Sir|William|Hatfield|N|10|4|1936|SWITZERLAND||William.Hatfield@pKju7yr9qIvUHllEQO.com|2452340| +6927|AAAAAAAAPALBAAAA|807817|3393|47369||2449976|Miss||||5||||||| +6928|AAAAAAAAABLBAAAA|207151|2940|6776|2450598|2450568|Miss|Belinda|Johnson|Y|7|4|1989|MALAWI||Belinda.Johnson@6en81tVuG2QAU6.com|2452489| +6929|AAAAAAAABBLBAAAA|1131928|2516|15151|2450221|2450191|Dr.|Laura|Brown|N|31|3|1964|MONTSERRAT||Laura.Brown@mQerASZbABe1Q7.com|2452534| +6930|AAAAAAAACBLBAAAA|738219|4174|25729|2450460|2450430|Mrs.|Karen|Shepherd|Y|1|6|1926|TOKELAU||Karen.Shepherd@fM7.edu|2452303| +6931|AAAAAAAADBLBAAAA|189812|387|35009|2450128|2450098|Dr.|Gary|Rodriguez|Y|18|9|1983|DOMINICA||Gary.Rodriguez@XCI3K90RTCU.com|2452471| +6932|AAAAAAAAEBLBAAAA|1700057|3323|44636|2449102|2449072|Dr.|William|Tuck|N|25|4|1938|BOUVET ISLAND||William.Tuck@IFrcmYIBgRO7.edu|2452441| +6933|AAAAAAAAFBLBAAAA|584084|1868|16852|2449696|2449666|Ms.|Berniece|Mitchell|Y|22|9|1944|PORTUGAL||Berniece.Mitchell@5GV6RioRaUMllTih.edu|2452630| +6934|AAAAAAAAGBLBAAAA|1323674|6416|20758|2450642|2450612|Ms.|Carin|Gonzalez|Y|27|5|1970|SAMOA||Carin.Gonzalez@DHaz4zVFps.com|2452389| +6935|AAAAAAAAHBLBAAAA|1126314|3485|40656|2449846|2449816|Ms.|Cheryl|Jernigan|Y|21|1|1925|NEW CALEDONIA||Cheryl.Jernigan@FtdM9io3vK0q.org|2452488| +6936|AAAAAAAAIBLBAAAA|641267|720|39373|2451023|2450993|Mr.|Enrique|Walker|N|24|6|1975|GREENLAND||Enrique.Walker@Y1ms0RCbD7g9h.com|2452648| +6937|AAAAAAAAJBLBAAAA|1740450|3329|31761|2451198|2451168|Ms.|Jennifer|Wheeler|Y|19|6|1947|CROATIA||Jennifer.Wheeler@y50MPyJzJVv.com|2452603| +6938|AAAAAAAAKBLBAAAA|1220231|3164|20544|2452414|2452384|Dr.|Gordon|Griffin|Y|4|11|1976|MONTENEGRO||Gordon.Griffin@pYn6NjeTgeo.org|2452569| +6939|AAAAAAAALBLBAAAA|1878080|459|28038|2449503|2449473|Mr.|Kyle|Barr|N|8|11|1964|NETHERLANDS ANTILLES||Kyle.Barr@tVYb.org|2452341| +6940|AAAAAAAAMBLBAAAA|1524158|550|6989|2451630|2451600|Sir|James|Valdez|Y|27|1|1930|UNITED KINGDOM||James.Valdez@nE7Ar746LUo3P20xR.org|2452387| +6941|AAAAAAAANBLBAAAA||3096|20191|2449496|||Lisa|Payne|N||10|||||| +6942|AAAAAAAAOBLBAAAA|1369687|115|3390|2449140|2449110|Mrs.|Sandra|Stewart|N|21|1|1989|BOUVET ISLAND||Sandra.Stewart@vQ3DS7.com|2452333| +6943|AAAAAAAAPBLBAAAA|82433|2455|14477|2451134|2451104|Miss|Dawn|Manning|N|18|9|1938|MALAWI||Dawn.Manning@rTZTTQJPu1HdrUMhf.com|2452478| +6944|AAAAAAAAACLBAAAA|66430|1811|31545|2449354|2449324|Dr.|Dorothy|Hansen|Y|25|8|1927|GUINEA-BISSAU||Dorothy.Hansen@X4lcP2kgcEp.com|2452386| +6945|AAAAAAAABCLBAAAA|849352|2906|20860|2449641|2449611|Mrs.|Holly|Harrison|N|12|11|1986|GUERNSEY||Holly.Harrison@Can.edu|2452288| +6946|AAAAAAAACCLBAAAA|1795525|3247|16122|2450253|2450223|Mrs.|Kathryn|Verdin|Y|12|3|1952|PALAU||Kathryn.Verdin@Lkn6n1LSQho.edu|2452384| +6947|AAAAAAAADCLBAAAA|954900|3661|19269|2449921|2449891|Mrs.|Evelyn|Hammett|Y|2|11|1985|HUNGARY||Evelyn.Hammett@0BT64.org|2452304| +6948|AAAAAAAAECLBAAAA|1314021|455|44489|2450183|2450153|Sir|Earl|Rizzo|N|25|2|1945|JAPAN||Earl.Rizzo@bPaO5m3U1oC.org|2452483| +6949|AAAAAAAAFCLBAAAA|194919|1933|31394|2449867|2449837|Mrs.|Hattie|Bates|Y|7|10|1957|NIGERIA||Hattie.Bates@R9.org|2452544| +6950|AAAAAAAAGCLBAAAA|384363|869|25375|2450345|2450315|Dr.|Thomas|Peterson|Y|7|9|1959|PUERTO RICO||Thomas.Peterson@57TSYI9dsi4C0Z5zZ.edu|2452454| +6951|AAAAAAAAHCLBAAAA|1130539|7141|15743|2451330|2451300|Dr.|Ann|Larson|N|4|11|1970|BELIZE||Ann.Larson@m.com|2452326| +6952|AAAAAAAAICLBAAAA|589741|5299|57|2450526|2450496|Ms.|Christine|Cruz|Y|15|11|1988|GUYANA||Christine.Cruz@XkxOztqHZv.edu|2452295| +6953|AAAAAAAAJCLBAAAA|1299872|782|47245|2450983|2450953|||Wooley|||7|1972||||2452346| +6954|AAAAAAAAKCLBAAAA|879645|1259|9226|2451013|2450983|Sir|Robert|Jones|Y|17|5|1981|JORDAN||Robert.Jones@2MU.edu|2452350| +6955|AAAAAAAALCLBAAAA|654427|4771|47468|2450480|2450450|Dr.|Daniel|Foster|N|25|3|1933|MALTA||Daniel.Foster@y8uUr.org|2452628| +6956|AAAAAAAAMCLBAAAA|14580|276|38819|2449455|2449425|Mr.|Curtis|Smith|Y|13|3|1945|KYRGYZSTAN||Curtis.Smith@GakYf.org|2452648| +6957|AAAAAAAANCLBAAAA|210768|5140|44712|2450044|2450014|Sir|Kenneth|Bean|N|9|1|1945|HONDURAS||Kenneth.Bean@A8GpnIIsV6HJhP6NjSZ.org|2452371| +6958|AAAAAAAAOCLBAAAA|1362291|7162|48433|2451675|2451645|Ms.|Hazel|Pickett|N|6|4|1933|CAPE VERDE||Hazel.Pickett@ATl86L68.org|2452290| +6959|AAAAAAAAPCLBAAAA|138590|117|9997|2452086|2452056|Mr.|John|Shivers|N|12|4|1945|ARMENIA||John.Shivers@oK28Qpc0J5n9Mi.org|2452442| +6960|AAAAAAAAADLBAAAA|1464682|6577|13272|2451363|2451333|Sir|Michael|Ojeda|Y|5|12|1954|URUGUAY||Michael.Ojeda@h8jC.org|2452561| +6961|AAAAAAAABDLBAAAA|1035990|6761|27132|2449193|2449163|Dr.|Richelle|Hammonds|Y|19|2|1939|UGANDA||Richelle.Hammonds@cJmeTPETgYjLULFT.edu|2452292| +6962|AAAAAAAACDLBAAAA|1664485|3070|28902|2450374|2450344|Miss|Tiffany|Todd|Y|24|4|1942|NEW CALEDONIA||Tiffany.Todd@x2u0CE5AAfClsS5.org|2452311| +6963|AAAAAAAADDLBAAAA|1375351|2930|26509|2450190|2450160|Sir|Richard|Mathews|Y|18|4|1972|BELGIUM||Richard.Mathews@xOBpc1oSIqsd.edu|2452301| +6964|AAAAAAAAEDLBAAAA|||11128|2450100|2450070||Emily|Obrien|Y||3|1941|ETHIOPIA|||2452427| +6965|AAAAAAAAFDLBAAAA|172281|762|4443|2449405|2449375|Sir|William|Murphy|Y|8|8|1933|INDIA||William.Murphy@X9.org|2452486| +6966|AAAAAAAAGDLBAAAA|638057|4730|7803|2450442|2450412|Dr.|Charles|Huffman|Y|6|2|1983|BAHAMAS||Charles.Huffman@qVeT57I2F1M0egp.org|2452616| +6967|AAAAAAAAHDLBAAAA|1697736|4880|1224|2452187|2452157|Dr.|Miguel|Milne|Y|14|1|1949|GUATEMALA||Miguel.Milne@LN5JQxy.org|2452414| +6968|AAAAAAAAIDLBAAAA|1724368|6520|47472|2450516|2450486|Mrs.|Monica|Bailey|N|3|9|1979|ANTARCTICA||Monica.Bailey@jATAJjh.org|2452404| +6969|AAAAAAAAJDLBAAAA|151963|6665|23014|2450032|2450002|Mrs.|Tania|Garner|Y|30|4|1935|SOLOMON ISLANDS||Tania.Garner@gvkxR0X5tLAe8LYMG.org|2452396| +6970|AAAAAAAAKDLBAAAA|903930|4045|19760|2451494|2451464|Dr.|Joseph|Camp|N|17|8|1928|BELARUS||Joseph.Camp@7XHNMyvS5nz2S.org|2452618| +6971|AAAAAAAALDLBAAAA|215006|1682|29511|2451761|2451731|Mr.|Thomas|Gaines|N|7|3|1943|INDONESIA||Thomas.Gaines@1xXvZhMo3dTELJM7S.edu|2452548| +6972|AAAAAAAAMDLBAAAA|1524724|2439|23401|2452413|2452383|Dr.|Stephanie|Simpson|Y|23|5|1934|ARGENTINA||Stephanie.Simpson@99aKz6a.edu|2452520| +6973|AAAAAAAANDLBAAAA|1141166|3948|45083|2451965|2451935|Ms.|Mitsue|Lynch|Y|2|12|1931|MOZAMBIQUE||Mitsue.Lynch@nA4VOsqmp2.org|2452411| +6974|AAAAAAAAODLBAAAA|1676808|694|11427|2451243|2451213|Mrs.|Jennifer|Allen|Y|14|7|1977|ZIMBABWE||Jennifer.Allen@cOFRb8qGAD6h.com|2452514| +6975|AAAAAAAAPDLBAAAA|254514|674|30434|2451939|2451909|Sir|Rudolph|Frasier|Y|16|1|1970|NEW CALEDONIA||Rudolph.Frasier@nl.edu|2452466| +6976|AAAAAAAAAELBAAAA|1512488|1128|6521|2451066|2451036|Mr.|Juan|Grant|N|14|10|1932|NIGERIA||Juan.Grant@6qcE7Bv4X5rnsIvY.org|2452327| +6977|AAAAAAAABELBAAAA|990293|4336|19179|2452194|2452164|Dr.|Shelly|Forbes|Y|10|10|1975|SPAIN||Shelly.Forbes@U4.com|2452477| +6978|AAAAAAAACELBAAAA|930751|3870|40542|2449064|2449034|Mr.|William|Dunlap|Y|30|11|1953|TUVALU||William.Dunlap@h.edu|2452302| +6979|AAAAAAAADELBAAAA|490543|5551|44606|2451784|2451754|Mr.|Rusty|Coleman|N|9|12|1975|MONGOLIA||Rusty.Coleman@CP2bQCpeIvb.com|2452509| +6980|AAAAAAAAEELBAAAA|476824|1740|45856|2450971|2450941|Dr.|Jose|Martinez|N|19|5|1956|SAINT LUCIA||Jose.Martinez@RSytGIki.com|2452372| +6981|AAAAAAAAFELBAAAA|1253512|1356|8047|2449398|2449368|Dr.|Viola|Allen|N|3|12|1992|JAPAN||Viola.Allen@rGudBIEjVtbNJ.edu|2452579| +6982|AAAAAAAAGELBAAAA|1078700|4818|10089|2449196|2449166|Dr.|Wesley|Hanson|N|25|7|1950|DENMARK||Wesley.Hanson@8lVRujOIT19l67Nat.com|2452582| +6983|AAAAAAAAHELBAAAA|1276622|6660|18745|2449262|2449232|Miss|Iva|Hudson|Y|25|5|1924|ZIMBABWE||Iva.Hudson@7044inZSO.edu|2452547| +6984|AAAAAAAAIELBAAAA|100143|83|6870|2451936|2451906|Dr.|Michelle|Damon|N|7|6|1984|CYPRUS||Michelle.Damon@jt8Jb7LCeyzJZ.edu|2452356| +6985|AAAAAAAAJELBAAAA|985748|1140|30979|2451725|2451695|Mrs.|Thelma|Palmer|Y|26|3|1929|NIGER||Thelma.Palmer@QrFD2XPso45oljRma.org|2452585| +6986|AAAAAAAAKELBAAAA|1235501|3565|23629|2451484|2451454|Ms.|Diane|Fajardo|N|3|4|1945|VENEZUELA||Diane.Fajardo@F9lmGj7Q06OtfUi.org|2452614| +6987|AAAAAAAALELBAAAA|1662787|1265|22143|2450122|2450092|Sir|Robert|Cantwell|N|9|4|1938|ROMANIA||Robert.Cantwell@OFGLUlO0eVoLPfYyEkPL.com|2452456| +6988|AAAAAAAAMELBAAAA|55169|1288|48654|2452674|2452644|Ms.|Elizabeth|Finnegan|Y|20|7|1972|ROMANIA||Elizabeth.Finnegan@VyGdyhRkJ2s.edu|2452647| +6989|AAAAAAAANELBAAAA|1919928|3841|4954|2451151|2451121|Mrs.|Isabelle|Hedges|Y|22|6|1939|THAILAND||Isabelle.Hedges@SU5KZkyU5St.com|2452320| +6990|AAAAAAAAOELBAAAA|1845638|157|37076|2451347|2451317|Dr.|Judith|Seifert|N|2|11|1953|HUNGARY||Judith.Seifert@YEp5Hh.org|2452288| +6991|AAAAAAAAPELBAAAA|1259759||38265||2452336|Dr.|Tara|Johnson|Y|1||1930|SAUDI ARABIA||Tara.Johnson@aEY7npvN2mcQbdY3ui.edu|2452449| +6992|AAAAAAAAAFLBAAAA|1919712|953|35347|2449919|2449889|Sir|Keith|Stern|N|18|2|1925|ICELAND||Keith.Stern@34Hb991eohmd.com|2452549| +6993|AAAAAAAABFLBAAAA|1564566|295|20824|2450992|2450962|Mrs.|Christy|Couch|Y|17|11|1950|NEW ZEALAND||Christy.Couch@Zv2DhO0358J.edu|2452372| +6994|AAAAAAAACFLBAAAA|218991|4319|7294|2451732|2451702|Dr.|Jason|Smith|Y|8|12|1983|CHINA||Jason.Smith@rYnyk.edu|2452423| +6995|AAAAAAAADFLBAAAA|577622|388|32754|2450536|2450506|Mrs.|Tami|Wood|N|16|4|1940|BHUTAN||Tami.Wood@PfKYbS9z0ZPUaV1l.com|2452531| +6996|AAAAAAAAEFLBAAAA|989103|138|24420|2449545|2449515|Mr.|Rex|Fernandez|N|19|9|1958|SOMALIA||Rex.Fernandez@N7aD5YH5k.org|2452310| +6997|AAAAAAAAFFLBAAAA|1554395|5132|24316|2450601|2450571|Sir|Ezequiel|Richardson|Y|21|11|1979|SAN MARINO||Ezequiel.Richardson@Fsp0gt.org|2452499| +6998|AAAAAAAAGFLBAAAA|179949|997|34045|2452360|2452330|Ms.|Aileen|Simmons|N|29|9|1961|PALAU||Aileen.Simmons@O90rSfIOuFPUMSbV2.com|2452461| +6999|AAAAAAAAHFLBAAAA|1545518|2954|30343|2449498|2449468|Dr.|Candice|Robertson|N|17|8|1942|SAN MARINO||Candice.Robertson@7R9qXZQn2Cde.org|2452471| +7000|AAAAAAAAIFLBAAAA|520895|1277|49212|2450078|2450048|Mr.|Brett|Lacy|N|8|8|1976|SWITZERLAND||Brett.Lacy@LVpi0epch8.edu|2452333| +7001|AAAAAAAAJFLBAAAA|863766|3452|39938|2450424|2450394|Mr.|Robert|Alvarado|Y|21|9|1953|BURUNDI||Robert.Alvarado@2TIds8enT.edu|2452422| +7002|AAAAAAAAKFLBAAAA|1847321|3425|17082|2449975|2449945|Mr.|Gary|Fernandez|N|29|6|1968|THAILAND||Gary.Fernandez@DUJVkgUQh0Quyy.org|2452517| +7003|AAAAAAAALFLBAAAA|1571250|4805|17113|2450469|2450439|Dr.|Blanca|Thomas|N|26|8|1990|MONACO||Blanca.Thomas@RzdB0tqM95kzNYDpV.com|2452367| +7004|AAAAAAAAMFLBAAAA|1475870|3117|42317|2450636|2450606|Sir|James|Quintanilla|Y|27|9|1947|NEPAL||James.Quintanilla@BPO.com|2452413| +7005|AAAAAAAANFLBAAAA|1411357|1989|8621|2451382|2451352|Dr.|Roman|Clark|Y|20|3|1987|ESTONIA||Roman.Clark@dYIQQUHdUV9AtAuFta.com|2452631| +7006|AAAAAAAAOFLBAAAA|451077|4503|17129|2449459|2449429|Mr.|Kyle|Gonzalez|N|4|10|1985|BURUNDI||Kyle.Gonzalez@G5en7Ic8I2.org|2452477| +7007|AAAAAAAAPFLBAAAA|1136808|7078|46995|2452626|2452596|Dr.|Cory|Jackson|Y|30|3|1971|BOUVET ISLAND||Cory.Jackson@VYU8JohiZdblJV.com|2452449| +7008|AAAAAAAAAGLBAAAA|18577|942|14107|2451247|2451217|Mrs.|Hilda|Richardson|N|21|2|1926|MEXICO||Hilda.Richardson@oUq6oQniO3Xm81oTghUk.org|2452408| +7009|AAAAAAAABGLBAAAA|1284461|2188|42268|2450448|2450418|Mr.|Mark|Rogers|N|1|5|1990|NIUE||Mark.Rogers@ZuQZ.com|2452362| +7010|AAAAAAAACGLBAAAA|1269247|277|12107|2449240|2449210|Miss|Alvina|Sherman|Y|5|8|1960|MOZAMBIQUE||Alvina.Sherman@BydMrxHMnML3r.org|2452309| +7011|AAAAAAAADGLBAAAA|327575|6127|9479|2451878|2451848|Sir|Mark|Myles|Y|22|12|1961|BULGARIA||Mark.Myles@TGl8jA60.com|2452528| +7012|AAAAAAAAEGLBAAAA|748286|4562|14085|2451621|2451591|Mrs.|Melanie|Cole|Y|18|5|1948|UNITED KINGDOM||Melanie.Cole@tO.org|2452303| +7013|AAAAAAAAFGLBAAAA|1114421|3152|27483|2450268|2450238|Dr.|Peggy|Mott|Y|15|1|1985|ERITREA||Peggy.Mott@OkpefacLLO6u.edu|2452288| +7014|AAAAAAAAGGLBAAAA|1400758|4825|24892|2450475|2450445|Dr.|Edward|Diaz|Y|4|4|1972|CZECH REPUBLIC||Edward.Diaz@4Lnz4YNinPTB.org|2452360| +7015|AAAAAAAAHGLBAAAA|896042|3200|48805|2450650|2450620|Mrs.|Dorothy|Abreu|Y|31|12|1931|GUINEA||Dorothy.Abreu@Tkj9Qcj6e5GflPkv.com|2452437| +7016|AAAAAAAAIGLBAAAA|1407449|5207|10565|2451114|2451084|Dr.|Tim|Martinez|Y|21|3|1981|HONG KONG||Tim.Martinez@xR.edu|2452449| +7017|AAAAAAAAJGLBAAAA|717863|4731|21246|2449964|2449934|Mrs.|Elsie|Mckay|Y|4|10|1987|ETHIOPIA||Elsie.Mckay@ntrhTYBxk4k.edu|2452571| +7018|AAAAAAAAKGLBAAAA|1095953|3349|17498|2452678|2452648|Dr.|Karen|Smith|Y|25|12|1927|RUSSIAN FEDERATION||Karen.Smith@5SzHm.com|2452382| +7019|AAAAAAAALGLBAAAA|445351|6148|42145|2450828|2450798|Ms.|Judy|Blair|N|11|11|1962|NEPAL||Judy.Blair@1u4vtUobDyT.com|2452351| +7020|AAAAAAAAMGLBAAAA|728790|4726|3015|2449335|2449305|Ms.|Wendy|Johnson|Y|25|5|1975|SAMOA||Wendy.Johnson@d70U60FHX.edu|2452590| +7021|AAAAAAAANGLBAAAA|1678912|435|5558|2450380|2450350|Ms.|Brittany|Maldonado|Y|17|9|1969|GREENLAND||Brittany.Maldonado@ar.org|2452483| +7022|AAAAAAAAOGLBAAAA|862507|7003|8346|2452357|2452327|Mrs.|Herlinda|Griffin|N|17|11|1990|FRENCH POLYNESIA||Herlinda.Griffin@VNQVoH7h02Q6OMZZVD.com|2452315| +7023|AAAAAAAAPGLBAAAA|409681|877|5626|2449799|2449769|Sir|Fredrick|Yates|N|30|6|1988|BELARUS||Fredrick.Yates@pMSleghjU.org|2452405| +7024|AAAAAAAAAHLBAAAA|1020752|494|30589|2451491|2451461|Sir|Luis|Numbers|N|15|8|1953|KUWAIT||Luis.Numbers@Hbr.edu|2452598| +7025|AAAAAAAABHLBAAAA|1403459|4047|2635|2450701|2450671|Dr.|Nancy|Comer|Y|21|8|1971|UZBEKISTAN||Nancy.Comer@Pno2UU6HPx8.com|2452477| +7026|AAAAAAAACHLBAAAA|1071572|2510|32695|2452224|2452194|Sir|Eric|King|Y|28|10|1951|MAURITANIA||Eric.King@5Xua9MHpQI.edu|2452349| +7027|AAAAAAAADHLBAAAA|1363660|4553|646|2449828|2449798|Dr.|Gilbert|Vogel|Y|11|12|1934|ESTONIA||Gilbert.Vogel@9e6odp3tB.edu|2452573| +7028|AAAAAAAAEHLBAAAA|1497531|4664|32197|2451396|2451366|Dr.|Marcia|Bailey|Y|14|7|1952|MALI||Marcia.Bailey@I653mU11EMD3dh.edu|2452610| +7029|AAAAAAAAFHLBAAAA|1331605|2849|47340|2450795|2450765|Mrs.|Aaron|Padilla|Y|29|10|1956|UGANDA||Aaron.Padilla@GqeLui.org|2452538| +7030|AAAAAAAAGHLBAAAA|427914|1880|26693|2451409|2451379|Dr.|John|Hyman|N|27|3|1983|NETHERLANDS||John.Hyman@Y.com|2452614| +7031|AAAAAAAAHHLBAAAA|806637|300|29163|2452562|2452532|Sir|Roy|Allison|N|21|7|1976|ARUBA||Roy.Allison@8MJYvcyMLmV.com|2452423| +7032|AAAAAAAAIHLBAAAA|916528|1207|43533|2450118|2450088|Mrs.|Claudine|Orta|Y|10|4|1973|SWAZILAND||Claudine.Orta@CLPh.com|2452599| +7033|AAAAAAAAJHLBAAAA|1794205|823|36994|2451441|2451411|Mr.|Michael|Hill|Y|20|6|1966|EQUATORIAL GUINEA||Michael.Hill@gTUVtL5L53u7Fa.com|2452455| +7034|AAAAAAAAKHLBAAAA|1518901||31047|2451130|2451100|Miss||Wilson|||10||FRANCE|||| +7035|AAAAAAAALHLBAAAA|1579231|4153|48228|2451555|2451525|Mr.|Robert|Riddick|N|12|10|1972|PAKISTAN||Robert.Riddick@yz4jVlOl8jHn.org|2452381| +7036|AAAAAAAAMHLBAAAA|148191|3403|2968|2450045|2450015|Dr.|Brian|Welch|N|29|8|1984|PAKISTAN||Brian.Welch@LmTEV.com|2452318| +7037|AAAAAAAANHLBAAAA|1463443|6009|15775|2451235|2451205|Dr.|Christina|Mchenry|Y|16|5|1941|GUINEA-BISSAU||Christina.Mchenry@aXPr.org|2452385| +7038|AAAAAAAAOHLBAAAA|536821|6884|40594|2451362|2451332|Dr.|Cynthia|Hanson|N|12|6|1961|LESOTHO||Cynthia.Hanson@uBO.edu|2452608| +7039|AAAAAAAAPHLBAAAA|1604050|4407|35964|2449177|2449147|Dr.|Sherry|Moon|Y|2|1|1965|BRAZIL||Sherry.Moon@hnX4fONY1FeYZnfOl.edu|2452426| +7040|AAAAAAAAAILBAAAA|1650873|3843|42013|2450401|2450371|Mrs.|Melinda|Oneal|Y|18|7|1924|MAURITIUS||Melinda.Oneal@arrhMC4keNZl3LX.edu|2452471| +7041|AAAAAAAABILBAAAA|498894|4946|20245|2450016|2449986|Ms.|Julie|Clark|Y|7|3|1970|HUNGARY||Julie.Clark@hA7Rry1g3QyfM0rQiOkz.org|2452349| +7042|AAAAAAAACILBAAAA|356865|2939|48966|2451600|2451570|Sir|Andres|Gilmore|N|27|11|1940|QATAR||Andres.Gilmore@mn0f.org|2452288| +7043|AAAAAAAADILBAAAA|407414|5096|11531|2451040|2451010|Dr.|Jesse|Mabry|Y|27|10|1951|PALAU||Jesse.Mabry@Ctikjy2etiir8.org|2452379| +7044|AAAAAAAAEILBAAAA|632876|2082|4441|2451192|2451162|Ms.|Emily|Williamson|N|2|3|1982|CANADA||Emily.Williamson@uiB.com|2452420| +7045|AAAAAAAAFILBAAAA|861225|2916|48207|2450329|2450299|Dr.|Vivian|Matheny|N|28|2|1990|FRENCH POLYNESIA||Vivian.Matheny@lpFL9T9ZmKAUr1gr.com|2452529| +7046|AAAAAAAAGILBAAAA|951130|5072|22214|2451388|2451358|Dr.|Harold|Motley|Y|16|9|1989|HONDURAS||Harold.Motley@d.org|2452402| +7047|AAAAAAAAHILBAAAA|||41164|||||Earl||||1967|AUSTRIA||John.Earl@7dlY3.com|2452368| +7048|AAAAAAAAIILBAAAA|1826908|3172|636|2449504|2449474|Sir|Mark|Pope|N|20|11|1925|MALI||Mark.Pope@g4B.org|2452436| +7049|AAAAAAAAJILBAAAA|666173|110|4047|2451859|2451829|Dr.|Donald|Smith|Y|15|7|1948|NORFOLK ISLAND||Donald.Smith@zGUCnTaXGETLQpU.org|2452618| +7050|AAAAAAAAKILBAAAA|875683|4300|17270|2449821|2449791|Mr.|Frederick|Abbott|Y|25|5|1979|JAPAN||Frederick.Abbott@pB9salQ.edu|2452512| +7051|AAAAAAAALILBAAAA|120380|5480|6227|2449495|2449465|Mrs.|Alexis|Phillips|N|3|10|1981|FRENCH GUIANA||Alexis.Phillips@7Tr3ezA8Yev6PnNurl.edu|2452416| +7052|AAAAAAAAMILBAAAA|1067648|2304|33807|2452133|2452103|Mr.|Brendan|Rinehart|Y|25|3|1961|NIGER||Brendan.Rinehart@VZoGb9XlqrRUYR.edu|2452472| +7053|AAAAAAAANILBAAAA|1230074|3459|26844|2451700|2451670|Dr.|Tanya|Smith|N|27|10|1932|NEPAL||Tanya.Smith@MR475qzxlm.org|2452433| +7054|AAAAAAAAOILBAAAA|101535|316|37097|2449525|2449495|Mr.|Jerry|James|N|27|11|1956|FAROE ISLANDS||Jerry.James@Re1.edu|2452631| +7055|AAAAAAAAPILBAAAA|1236689|704|40719||||Edward||Y|9||1969|ZIMBABWE|||2452644| +7056|AAAAAAAAAJLBAAAA|249467|1278|33489|2452540|2452510|Mrs.|Adrian|Arias|N|14|7|1965|PORTUGAL||Adrian.Arias@kuIivCdQ7aJvPpy.com|2452441| +7057|AAAAAAAABJLBAAAA|811040|7179|18871|2451867|2451837|Sir|Ivan|Chilton|N|10|9|1972|EL SALVADOR||Ivan.Chilton@HVuMQki6pC8.com|2452443| +7058|AAAAAAAACJLBAAAA|369586|2930|33874|2449764|2449734|Sir|Larry|Borden|N|2|12|1986|ITALY||Larry.Borden@HoutTe.org|2452557| +7059|AAAAAAAADJLBAAAA||2471|27789||2451150||Francis|Shook|N|8|||BOUVET ISLAND|||| +7060|AAAAAAAAEJLBAAAA|1675250|679|12061|2450710|2450680|Miss|Heather|Hayward|N|17|11|1924|SLOVENIA||Heather.Hayward@liNNyb2XIVTyX.edu|2452343| +7061|AAAAAAAAFJLBAAAA|1631738|6068|4398|2449172|2449142|Mr.|Richard|Olvera|Y|20|7|1931|LUXEMBOURG||Richard.Olvera@pMgR687zzrb.edu|2452354| +7062|AAAAAAAAGJLBAAAA|1675516|1067|41818|2451054|2451024|Sir|Judson|Contreras|N|29|12|1972|SYRIAN ARAB REPUBLIC||Judson.Contreras@lDg8ppTpdCevy37.edu|2452592| +7063|AAAAAAAAHJLBAAAA|303343|4201|18212|2450679|2450649|Ms.|Jennifer|Marquez|N|12|11|1946|VENEZUELA||Jennifer.Marquez@Yhqhhisp46.edu|2452636| +7064|AAAAAAAAIJLBAAAA|1657295|7031|25120|2450209|2450179|Mrs.|Cynthia|Dempsey|N|5|12|1963|MOZAMBIQUE||Cynthia.Dempsey@R3Fsh1aeGk.org|2452472| +7065|AAAAAAAAJJLBAAAA|1637700|2193|31486|2450874|2450844|Dr.|Olga|Ferguson|Y|29|4|1946|ANGUILLA||Olga.Ferguson@CnX5KZqam.org|2452635| +7066|AAAAAAAAKJLBAAAA|587945|1102|28706|2449427|2449397|Dr.|Lillian|Trevino|Y|7|3|1983|IRELAND||Lillian.Trevino@ZKtLzclOJfRdIR5.edu|2452611| +7067|AAAAAAAALJLBAAAA|1772156|341|42973|2452045|2452015|Dr.|Joseph|Gray|N|27|9|1966|NORFOLK ISLAND||Joseph.Gray@L.org|2452370| +7068|AAAAAAAAMJLBAAAA|1821738|3323|15308|2449296|2449266|Miss|Majorie|Johnson|N|3|9|1986|THAILAND||Majorie.Johnson@Ox5hFX8hqM7zAqXC.org|2452441| +7069|AAAAAAAANJLBAAAA|292320|3470|23467|2451376|2451346|Mrs.|Karen|Williamson|N|17|9|1983|SOUTH AFRICA||Karen.Williamson@GmZ.org|2452633| +7070|AAAAAAAAOJLBAAAA|486732|3213|29694|2450771|2450741|Miss|Dusti|Taylor|Y|5|11|1991|BENIN||Dusti.Taylor@kMkYBkRjDb.org|2452404| +7071|AAAAAAAAPJLBAAAA|1503365|1136|2107|2450226|2450196|Sir|Gerald|Navarro|N|2|10|1961|SOUTH AFRICA||Gerald.Navarro@f6u08Z0TibIh8l.org|2452392| +7072|AAAAAAAAAKLBAAAA|1667934|759|20384|2451057|2451027|Ms.|Belinda|Miller|N|9|12|1981|BAHAMAS||Belinda.Miller@mUVYD.org|2452326| +7073|AAAAAAAABKLBAAAA|143409|53|37850|2451191|2451161|Sir|Douglas|White|Y|24|2|1940|EQUATORIAL GUINEA||Douglas.White@qimQ4vGoyBI495GUrqI.com|2452408| +7074|AAAAAAAACKLBAAAA|1198465|2043|37221|||Mr.|Gene|Gardner|N|||1970|||Gene.Gardner@ttoNQTsyxZjCn.com|| +7075|AAAAAAAADKLBAAAA|178277|3552|10380|2451102|2451072|Mrs.|Kimberly|Clay|Y|7|11|1935|GUERNSEY||Kimberly.Clay@7eDmpMPKpy94iMrj.com|2452464| +7076|AAAAAAAAEKLBAAAA|94814|4348|6282|2449996|2449966|Sir|Jeffrey|Carter|N|13|12|1944|NORFOLK ISLAND||Jeffrey.Carter@I.com|2452586| +7077|AAAAAAAAFKLBAAAA|1058592|6116|38887|2451852|2451822|Dr.|Douglas|Stewart|N|28|2|1980|BURUNDI||Douglas.Stewart@vrXn5L9kT4drfT.com|2452397| +7078|AAAAAAAAGKLBAAAA|133320|2319|9718|2452182|2452152|Miss|Sondra|Hernandez|N|16|1|1982|BRUNEI DARUSSALAM||Sondra.Hernandez@6ga.edu|2452517| +7079|AAAAAAAAHKLBAAAA|412371|3022|26120|2452061|2452031|Mr.|Roderick|Alvarez|Y|28|10|1985|GAMBIA||Roderick.Alvarez@tGNFjn.edu|2452297| +7080|AAAAAAAAIKLBAAAA|1775925|1754|4383|2451663|2451633|Miss|Faye|Jones|Y|16|5|1928|ISLE OF MAN||Faye.Jones@1g5tA971h2B1jh.org|2452545| +7081|AAAAAAAAJKLBAAAA|1265459|1170|38528|2451119|2451089|Sir|Wayne|Ruiz|N|10|3|1950|CHILE||Wayne.Ruiz@NmEYsh2Bq0L.org|2452402| +7082|AAAAAAAAKKLBAAAA|923026|802|14578|2451905|2451875|Dr.|Marilyn|Boone|Y|11|10|1958|BULGARIA||Marilyn.Boone@TRgXAPdspX0VRNE.edu|2452320| +7083|AAAAAAAALKLBAAAA|1521015|25|26635|2452052|2452022|Mr.|Theo|Daughtry|N|7|7|1944|CHILE||Theo.Daughtry@76.org|2452400| +7084|AAAAAAAAMKLBAAAA|1674941|5793|245|2452375|2452345|Mrs.|Terri|Matthews|Y|18|2|1971|LESOTHO||Terri.Matthews@bp.com|2452527| +7085|AAAAAAAANKLBAAAA|300940|45|44703|2450634|2450604|Dr.|Richard|Fisher|N|24|7|1982|NETHERLANDS ANTILLES||Richard.Fisher@gZyv.com|2452325| +7086|AAAAAAAAOKLBAAAA|1741253|323|14193|2452626|2452596|Mrs.|Rosemary|Williams|Y|1|10|1961|ARGENTINA||Rosemary.Williams@kZ4Q7xyYqHCRAUtJOA7.org|2452371| +7087|AAAAAAAAPKLBAAAA|926750|6801|21252|2450864|2450834|Sir|James|Green|Y|1|1|1981|GREENLAND||James.Green@63ikK.edu|2452468| +7088|AAAAAAAAALLBAAAA|884807|2668|21329|2451823|2451793|Sir|Dennis|Pearce|N|13|12|1983|BRAZIL||Dennis.Pearce@4T7uskrTEk9aVL.org|2452518| +7089|AAAAAAAABLLBAAAA|102718|889|2613|2451542|2451512|Miss|Judith|Garcia|Y|18|7|1968|LESOTHO||Judith.Garcia@beUTB1x28.org|2452291| +7090|AAAAAAAACLLBAAAA|200093|1537|22534|2449767|2449737|Sir|Timothy|Decker|Y|24|7|1935|ISRAEL||Timothy.Decker@YpyqTGDTsvhq.org|2452481| +7091|AAAAAAAADLLBAAAA|631089|3466|10290|2450870|2450840|Dr.|Kevin|Cline|Y|20|11|1979|ANTIGUA AND BARBUDA||Kevin.Cline@P5BN2.org|2452608| +7092|AAAAAAAAELLBAAAA|521582|540|24345|2449083|2449053|Sir|Michael|Sands|Y|14|7|1954|CROATIA||Michael.Sands@hSb.com|2452622| +7093|AAAAAAAAFLLBAAAA|1746876|3591|4706|2449497|2449467|Sir|Frank|Jones|Y|13|10|1979|EQUATORIAL GUINEA||Frank.Jones@HegKNzTUCvX5.edu|2452344| +7094|AAAAAAAAGLLBAAAA|1168820|5680|11787|2449419|2449389|Mrs.|Janice|Macdonald|N|20|12|1942|BANGLADESH||Janice.Macdonald@1jvNICAZI9.edu|2452549| +7095|AAAAAAAAHLLBAAAA|751632|1473|3361|2449371|2449341|Dr.|Martin|Larkin|N|11|3|1986|AMERICAN SAMOA||Martin.Larkin@HvHthvN.edu|2452504| +7096|AAAAAAAAILLBAAAA|921934|4138|35495|2452604|2452574|Sir|Dennis|Miller|N|25|9|1944|BOLIVIA||Dennis.Miller@cz.com|2452509| +7097|AAAAAAAAJLLBAAAA|1515359|2573|42940|2450282|2450252|Sir|David|Robinson|N|10|11|1986|BAHAMAS||David.Robinson@J2mkcldKE9JnJi.com|2452611| +7098|AAAAAAAAKLLBAAAA|1752207|5714|4868|2451306|2451276|Sir|James|Turner|N|12|9|1947|MAYOTTE||James.Turner@qi76hQ9POUcFh26.com|2452300| +7099|AAAAAAAALLLBAAAA|221668|1591|39467|2451931|2451901|Dr.|Sandra|Stewart|Y|23|12|1924|ANDORRA||Sandra.Stewart@u8QXXRJABUXm7Sl4.com|2452648| +7100|AAAAAAAAMLLBAAAA|1773733|6302|14471|2449065|2449035|Sir|Chad|Dawkins|Y|31|10|1963|DOMINICA||Chad.Dawkins@gZpUlgfupbmSv.com|2452594| +7101|AAAAAAAANLLBAAAA|1461722|5262|33955|2451615|2451585|Mr.|Shawn|Norris|Y|14|7|1930|NEPAL||Shawn.Norris@NMcPRvXnMq2oj6rX.com|2452525| +7102|AAAAAAAAOLLBAAAA|233803|5275|14700|2450019|2449989|Dr.|Alfred|Lopez|Y|28|4|1925|GHANA||Alfred.Lopez@CoAvzMaF6Gx.com|2452609| +7103|AAAAAAAAPLLBAAAA|639259|3560|9108|2451659|2451629|Sir|Richard|Valdez|Y|23|7|1961|MOLDOVA, REPUBLIC OF||Richard.Valdez@bYA.edu|2452360| +7104|AAAAAAAAAMLBAAAA|1353581|4541|28301|2450864|2450834|Mr.|Matthew|Williams|Y|23|6|1933|FRENCH GUIANA||Matthew.Williams@K7JugG.com|2452595| +7105|AAAAAAAABMLBAAAA|1614039|3912|2585|2452517|2452487|Ms.|Rosemary|Montano|N|4|4|1924|ISRAEL||Rosemary.Montano@AtUPB5OVRdqC9D.com|2452414| +7106|AAAAAAAACMLBAAAA|1619074|3604|20487|2450888|2450858|Miss|Rachel|Petty|N|19|1|1945|SRI LANKA||Rachel.Petty@KzoCeCphqRd.com|2452491| +7107|AAAAAAAADMLBAAAA|1732175|6495|25968|2449197|2449167|Dr.|Patrick|West|N|24|9|1966|ALAND ISLANDS||Patrick.West@HhY.org|2452321| +7108|AAAAAAAAEMLBAAAA|628988|1487|45986|2449325|2449295|Dr.|Robert|Coulter|N|18|7|1966|ECUADOR||Robert.Coulter@GiLPTJ9K.edu|2452330| +7109|AAAAAAAAFMLBAAAA|1137247|2436|7978|2451468|2451438|Mr.|Charles|Marsh|N|23|8|1968|LITHUANIA||Charles.Marsh@KeoX1b2H.com|2452533| +7110|AAAAAAAAGMLBAAAA|886887|3139|45838|2451386|2451356|Dr.|Edward|Phillips|N|18|6|1970|SRI LANKA||Edward.Phillips@Igr9fuZfHyqloeK5.com|2452605| +7111|AAAAAAAAHMLBAAAA|958797|6359|24368|2451206|2451176|Miss|Eleanor|Miller|N|22|5|1933|NORWAY||Eleanor.Miller@0ThQl5.com|2452292| +7112|AAAAAAAAIMLBAAAA|1655601|5167|5486|2451606|2451576|Mrs.|Margaret|Cheney|N|6|3|1956|SUDAN||Margaret.Cheney@zKsbT.edu|2452356| +7113|AAAAAAAAJMLBAAAA|1875340|836|49485|2450389|2450359|Sir|Benito|Hall|Y|8|1|1954|PHILIPPINES||Benito.Hall@p.edu|2452585| +7114|AAAAAAAAKMLBAAAA|964050|4747|19797|2450895|2450865|Mrs.|Thelma|Page|N|10|5|1968|MALI||Thelma.Page@o.edu|2452308| +7115|AAAAAAAALMLBAAAA|1881275|735|27795|2452152|2452122|Mr.|Nigel|Bynum|N|27|3|1952|AMERICAN SAMOA||Nigel.Bynum@TQi2UH6oBi2zjZ.org|2452412| +7116|AAAAAAAAMMLBAAAA|1441736|2302|39668|2451168|2451138|Ms.|Pok|Ponce|Y|7|9|1970|JAPAN||Pok.Ponce@5m4.org|2452365| +7117|AAAAAAAANMLBAAAA|895864|2067|33470|2450981|2450951|Dr.|Patrick|Bennett|N|21|12|1966|SWITZERLAND||Patrick.Bennett@eBGf2kp11C5tl.com|2452601| +7118|AAAAAAAAOMLBAAAA|981388|3984|30980|2450964|2450934|Mrs.|Betty|Jones|N|2|1|1975|BHUTAN||Betty.Jones@gHBekB8k8h.com|2452631| +7119|AAAAAAAAPMLBAAAA|1310418|6341|47864|2450601|2450571|Dr.|Carlos|Lewis|Y|27|5|1942|NAMIBIA||Carlos.Lewis@JBnYXXqq.org|2452371| +7120|AAAAAAAAANLBAAAA|804539|78|42817|2449348|2449318|Sir|Joey|Card|Y|4|12|1952|DJIBOUTI||Joey.Card@jF8.com|2452501| +7121|AAAAAAAABNLBAAAA|168253|1612|5324|2450410|2450380|Dr.|Amy|Vera|Y|13|2|1967|PORTUGAL||Amy.Vera@8MAcs6yc83.edu|2452541| +7122|AAAAAAAACNLBAAAA|169074|107|24340|2450135|2450105|Mr.|Conrad|Kirkpatrick|Y|1|1|1990|ALBANIA||Conrad.Kirkpatrick@TIYTxrE.edu|2452335| +7123|AAAAAAAADNLBAAAA|43216|828|20440|2450636|2450606|Sir|William|Windsor|N|13|11|1957|EGYPT||William.Windsor@hSEQd7xxYEtlDxSuO.org|2452423| +7124|AAAAAAAAENLBAAAA|888916|5688|47260|2450032|2450002|Ms.|Amanda|Childress|Y|28|6|1960|BAHRAIN||Amanda.Childress@p6lnFYAi3KCnSOaKj.org|2452372| +7125|AAAAAAAAFNLBAAAA|1679478|5716|31954|2451716|2451686|Mr.|Robert|Rice|N|29|1|1972|NIGERIA||Robert.Rice@C.org|2452586| +7126|AAAAAAAAGNLBAAAA|232705|183|10414|2452004|2451974|Dr.|Beatriz|Carrillo|Y|22|1|1969|ISRAEL||Beatriz.Carrillo@SMYTbNDxYkZsJqEDt.org|2452409| +7127|AAAAAAAAHNLBAAAA|1087758|2717|20923|2451574|2451544|Mr.|Ira|Robertson|Y|20|3|1984|THAILAND||Ira.Robertson@3fDRpo06LSZM9Fm.org|2452471| +7128|AAAAAAAAINLBAAAA|1703494|1836|32531|2451529|2451499|Ms.|Valerie|Williams|Y|11|1|1989|CUBA||Valerie.Williams@ja8cBtTpfPhEL405y0.org|2452313| +7129|AAAAAAAAJNLBAAAA|721776|3109|42889|2450909|2450879|Mr.|Mariano|James|N|3|9|1947|SAINT LUCIA||Mariano.James@N.org|2452566| +7130|AAAAAAAAKNLBAAAA|1694503|603|15103|2450933|2450903|Mrs.|Margaret|Estes|Y|15|9|1980|RWANDA||Margaret.Estes@Gygf8d.edu|2452393| +7131|AAAAAAAALNLBAAAA|523863|3551|10792|2449500|2449470|Dr.|Kay|Cardenas|Y|7|1|1936|AMERICAN SAMOA||Kay.Cardenas@DOzgOEl3mQK07i8XJn6.edu|2452535| +7132|AAAAAAAAMNLBAAAA|1915653|3665|16077|2451452|2451422|Miss|Wilhelmina|Randolph|N|17|6|1928|INDONESIA||Wilhelmina.Randolph@xmZbFP8Ka9zh.com|2452557| +7133|AAAAAAAANNLBAAAA|376461|6783|30752|2449415|2449385|Mrs.|Roseanne|Orton|Y|11|7|1931|KYRGYZSTAN||Roseanne.Orton@vXtYsAnjTZ.edu|2452504| +7134|AAAAAAAAONLBAAAA|1864082|3857|18385|2451193|2451163|Ms.|Laura|Baldwin|N|29|12|1932|BAHRAIN||Laura.Baldwin@LpCq96NGM.edu|2452489| +7135|AAAAAAAAPNLBAAAA|567384|4108|19632|2451479|2451449|Dr.|Alfred|Satterfield|Y|26|10|1929|ZIMBABWE||Alfred.Satterfield@IZNv1fKVE9S.com|2452633| +7136|AAAAAAAAAOLBAAAA|316876|2344|32612|2449214|2449184|Mr.|Guillermo|Gray|N|8|7|1963|ESTONIA||Guillermo.Gray@y5g.edu|2452600| +7137|AAAAAAAABOLBAAAA|378801|4577|19953|2451279|2451249|Mr.|Byron|Barnes|Y|11|5|1991|KOREA, REPUBLIC OF||Byron.Barnes@VSr.edu|2452435| +7138|AAAAAAAACOLBAAAA|1063355|5278|34448|2450794|2450764|Dr.|Catherine|Miller|N|3|11|1931|ETHIOPIA||Catherine.Miller@CI1FXBSqgMYROpu2p.org|2452582| +7139|AAAAAAAADOLBAAAA|904333|4466|28952|2451532|2451502|Mr.|Fred|Williams|Y|3|8|1975|LIBERIA||Fred.Williams@G12ncdL.edu|2452561| +7140|AAAAAAAAEOLBAAAA|1056404|1142|34280|2452067|2452037|Dr.|Marcus|Pearson|N|5|1|1940|LIBERIA||Marcus.Pearson@qjQk.org|2452466| +7141|AAAAAAAAFOLBAAAA|1311967|3387|35607|2451743|2451713|Mr.|Jose|Mabry|Y|20|7|1971|MOROCCO||Jose.Mabry@LRgQs7zFoE.edu|2452421| +7142|AAAAAAAAGOLBAAAA|1380878|6388|14298|2450984|2450954|Miss|Kathleen|Stine|N|26|5|1938|SRI LANKA||Kathleen.Stine@AkV.edu|2452579| +7143|AAAAAAAAHOLBAAAA|415031|7003|22222|2449789|2449759|Sir|Jerry|Finch|Y|4|1|1973|MOROCCO||Jerry.Finch@aHd8MI0Cvz.org|2452601| +7144|AAAAAAAAIOLBAAAA|736270|976|46035|2452539|2452509|Mr.|Richard|Cole|N|15|10|1974|FRENCH GUIANA||Richard.Cole@9kbmVbnquFiguZYg.edu|2452366| +7145|AAAAAAAAJOLBAAAA|1746586|2536|34839|2450716|2450686|Dr.|William|Dudley|Y|6|9|1945|VANUATU||William.Dudley@eG5TMPtx2C.com|2452592| +7146|AAAAAAAAKOLBAAAA|1461004||9270|2451412|2451382|Miss||Fisher|||9||TURKMENISTAN|||2452387| +7147|AAAAAAAALOLBAAAA|1756468|6451|34434|2452133|2452103|Miss|Nadine|Randolph|N|18|3|1936|LESOTHO||Nadine.Randolph@7FH5v8I.org|2452648| +7148|AAAAAAAAMOLBAAAA||823|34829|2450764|||Larry|Talley|Y||8|1940|||Larry.Talley@k6gjiZYRs.edu|| +7149|AAAAAAAANOLBAAAA|368248|2788|43328|2452293|2452263|Mr.|Kenneth|Wright|N|1|3|1974|AMERICAN SAMOA||Kenneth.Wright@E.com|2452300| +7150|AAAAAAAAOOLBAAAA|331960|6427|30941|2451151|2451121|Dr.|Cynthia|Blaylock|N|24|12|1926|WESTERN SAHARA||Cynthia.Blaylock@762N0J.com|2452358| +7151|AAAAAAAAPOLBAAAA|1750795|6185|17575|2449088|2449058|Dr.|William|Harrison|N|17|2|1988|GAMBIA||William.Harrison@YRvhJB7KNyfU84.com|2452421| +7152|AAAAAAAAAPLBAAAA|||22563||||Eloise||Y|13|||LITHUANIA||Eloise.Hurst@A55MVbcSr.org|2452309| +7153|AAAAAAAABPLBAAAA|202994|6026|32658|2450568|2450538|Dr.|Emily|Green|Y|22|12|1957|JORDAN||Emily.Green@aJSt5OoPk8T.edu|2452527| +7154|AAAAAAAACPLBAAAA|1887625|4208|3191|2450388|2450358|Dr.|Felix|Bennett|N|26|11|1980|GEORGIA||Felix.Bennett@GA8r6PgzE9C5o.com|2452448| +7155|AAAAAAAADPLBAAAA|1061580|915|15897|2451002|2450972|Dr.|John|Tyree|N|10|4|1934|FRANCE||John.Tyree@rb.org|2452356| +7156|AAAAAAAAEPLBAAAA|407869|6227|28938|2449624|2449594|Mrs.|Melinda|Olsen|N|12|3|1962|BERMUDA||Melinda.Olsen@GvNq.org|2452532| +7157|AAAAAAAAFPLBAAAA|1545972|6066|43313|2451957|2451927|Mr.|Gordon|Lucas|Y|26|1|1962|SOLOMON ISLANDS||Gordon.Lucas@8Hz1JbiCQH01IF.org|2452296| +7158|AAAAAAAAGPLBAAAA|674677|5428|43332|2449957|2449927|Miss|Shelly|Edwards|Y|6|7|1963|TURKMENISTAN||Shelly.Edwards@8fkNA7GtU0j47.org|2452510| +7159|AAAAAAAAHPLBAAAA|437932|4148|21211|2451449|2451419|Mr.|Lawrence|Carlisle|Y|19|3|1968|SERBIA||Lawrence.Carlisle@AzqflHqzNB0I3sEh.edu|2452289| +7160|AAAAAAAAIPLBAAAA|362253|1720|22387|2451590|2451560|Miss|Virginia|Davis|Y|27|10|1982|NAURU||Virginia.Davis@K16t0hi65f.edu|2452593| +7161|AAAAAAAAJPLBAAAA|1089591|5788|21232|2450152|2450122|Mr.|Ben|Metzger|N|3|6|1947|KENYA||Ben.Metzger@hdoQANy.com|2452422| +7162|AAAAAAAAKPLBAAAA|1223976|5341|32306|2449894|2449864|Dr.|Paul|Lee|N|30|7|1974|GERMANY||Paul.Lee@iMbiC.org|2452365| +7163|AAAAAAAALPLBAAAA|1272971|988|18205|2452226|2452196|Mr.|Larry|Rich|Y|4|1|1938|JERSEY||Larry.Rich@0c9tP0Kq.org|2452476| +7164|AAAAAAAAMPLBAAAA|261008|1408|37808|2452431|2452401|Sir|Jason|Clark|N|2|4|1949|FRENCH POLYNESIA||Jason.Clark@N.com|2452303| +7165|AAAAAAAANPLBAAAA|582770|650|44930|2450712|2450682|Mr.|David|Guerra|Y|26|9|1960|RWANDA||David.Guerra@mv9exxj.com|2452432| +7166|AAAAAAAAOPLBAAAA|1432431|5388|5145|2450541|2450511|Dr.|Juan|Cobb|N|11|4|1930|VIET NAM||Juan.Cobb@H.edu|2452489| +7167|AAAAAAAAPPLBAAAA|1721558|917|5476|2449921|2449891|Sir|Michael|Herron|Y|7|2|1964|ALAND ISLANDS||Michael.Herron@Yo.edu|2452619| +7168|AAAAAAAAAAMBAAAA|1338931|3810|30736|2449542|2449512|Ms.|Amanda|Smith|N|16|2|1976|ANTARCTICA||Amanda.Smith@SU32b0aBuA.com|2452564| +7169|AAAAAAAABAMBAAAA|1013289|982|47922|2452230|2452200|Dr.|Curtis|Smith|Y|29|5|1964|IRELAND||Curtis.Smith@gRJXE0Cl0LZvC0g.org|2452557| +7170|AAAAAAAACAMBAAAA|||34557||2450186||||N|18|4|1934|BARBADOS|||2452522| +7171|AAAAAAAADAMBAAAA|1216817|6574|36378|2451480|2451450|Dr.|Fred|Beal|N|5|6|1950|AUSTRIA||Fred.Beal@xSV.org|2452540| +7172|AAAAAAAAEAMBAAAA|991458|3519|34027|2449767|2449737|Ms.|Ella|Rocha|N|4|7|1987|BHUTAN||Ella.Rocha@3f0KTHCnlIJL.org|2452357| +7173|AAAAAAAAFAMBAAAA|1140893|226|43206|2449218|2449188|Mrs.|Becky|Jones|N|30|1|1956|CAMEROON||Becky.Jones@rGHV6sM32r0JOUUl0bS.edu|2452396| +7174|AAAAAAAAGAMBAAAA|92787|4757|33527|2451015|2450985|Sir|Rob|Bolt|Y|13|12|1961|SRI LANKA||Rob.Bolt@O.edu|2452413| +7175|AAAAAAAAHAMBAAAA|1717104|6880|11103|2451864|2451834|Sir|Jesse|Talbot|N|20|1|1973|BAHRAIN||Jesse.Talbot@FIKgFug5OVXbXo.edu|2452382| +7176|AAAAAAAAIAMBAAAA|1874525|206|1040|2450863|2450833|Mr.|Randal|Owens|Y|4|7|1973|LATVIA||Randal.Owens@l.com|2452500| +7177|AAAAAAAAJAMBAAAA|518331|1135|23939|2451947|2451917|Dr.|Elizabeth|Cromwell|N|24|10|1987|SWAZILAND||Elizabeth.Cromwell@sRnQlqvog2g.edu|2452624| +7178|AAAAAAAAKAMBAAAA|1563968|5706|33797|2452211|2452181|Mr.|Jess|Schafer|Y|10|11|1985|OMAN||Jess.Schafer@ZKBR.org|2452551| +7179|AAAAAAAALAMBAAAA|1226866|3487|22108|2449094|2449064|Ms.|Christine|Buckley|Y|21|9|1930|CANADA||Christine.Buckley@LYFb.org|2452386| +7180|AAAAAAAAMAMBAAAA|1096249|3475|31101|2451749|2451719|Ms.|Marian|Harman|N|14|11|1971|NAMIBIA||Marian.Harman@0Zc2R.com|2452346| +7181|AAAAAAAANAMBAAAA|1605168|1273|16178|2452288|2452258|Sir|Salvatore|Villanueva|N|28|5|1953|YEMEN||Salvatore.Villanueva@Reh.edu|2452587| +7182|AAAAAAAAOAMBAAAA|914694|3072|26250||2449093|Mr.||Lentz|N||||PORTUGAL||Humberto.Lentz@vrftmk.edu|| +7183|AAAAAAAAPAMBAAAA|1435822|6339|7703|2451808|2451778|Dr.|Deloris|Garland|N|5|1|1986|MOROCCO||Deloris.Garland@Iifztg.com|2452601| +7184|AAAAAAAAABMBAAAA|1772061|5151|13841|2452165|2452135|Sir|Robert|Holmes|Y|5|9|1949|TOKELAU||Robert.Holmes@B2uBsry4uB2.org|2452579| +7185|AAAAAAAABBMBAAAA|637757|6893|13982|2449373|2449343|Mr.|Ross|Perryman|Y|22|6|1971|SPAIN||Ross.Perryman@2bNbxeQE7Rj9JBJj.edu|2452554| +7186|AAAAAAAACBMBAAAA|301792|138|36931|2449188|2449158|Ms.|Sherry|Rice|N|10|6|1968|WALLIS AND FUTUNA||Sherry.Rice@86yeVN0LXV3axMnL4kyY.edu|2452645| +7187|AAAAAAAADBMBAAAA|1308125|3782|47886|2450003|2449973|Sir|Richard|Deal|Y|8|11|1928|BELGIUM||Richard.Deal@b8Pv.org|2452304| +7188|AAAAAAAAEBMBAAAA|476008|2897|186|2451434|2451404|Sir|Shane|Barber|N|4|12|1971|CZECH REPUBLIC||Shane.Barber@Jz.com|2452391| +7189|AAAAAAAAFBMBAAAA|898820|7169|4271|2452604|2452574|Dr.|Irma|Meyers|Y|25|12|1960|BELIZE||Irma.Meyers@RJFH8UBLsuS7Y.org|2452476| +7190|AAAAAAAAGBMBAAAA|1520903|7080|8057|2451821|2451791|Mr.|Roger|Branch|Y|7|9|1976|TUNISIA||Roger.Branch@P4vhArqt6QjJF.com|2452493| +7191|AAAAAAAAHBMBAAAA|1057719|3634|27149|2449110|2449080|Miss|Kathryn|Autry|Y|26|6|1981|SUDAN||Kathryn.Autry@TJhhZ.com|2452368| +7192|AAAAAAAAIBMBAAAA|1089081|6795|38156|2449491|2449461|Dr.|Donna|Bonilla|N|4|12|1931|MADAGASCAR||Donna.Bonilla@npSMtFUNQnHSZiIQ.edu|2452397| +7193|AAAAAAAAJBMBAAAA|1559055|6439|37023|2451158|2451128|Ms.|Tammy|Brown|Y|25|4|1963|UGANDA||Tammy.Brown@33sSXszNjRk.org|2452504| +7194|AAAAAAAAKBMBAAAA|1164832|3151|23070|2452521|2452491|Sir|Robert|Payne|N|12|8|1986|UKRAINE||Robert.Payne@9.org|2452294| +7195|AAAAAAAALBMBAAAA|1791150|1666|182|2449802|2449772|Dr.|Timmy|Strauss|Y|14|2|1972|SAINT LUCIA||Timmy.Strauss@9v6ZJjmEza.org|2452483| +7196|AAAAAAAAMBMBAAAA|713945|1620|14542|2450978|2450948|Mr.|Scott|Hall|Y|5|8|1929|FIJI||Scott.Hall@hQef.com|2452399| +7197|AAAAAAAANBMBAAAA|885287|3128|16922|2451125|2451095|Dr.|Walter|Lovell|N|19|1|1957|MALI||Walter.Lovell@UI.org|2452377| +7198|AAAAAAAAOBMBAAAA|796260|5941|23975|2451850|2451820|Mr.|Richard|Tibbetts|Y|11|6|1979|ECUADOR||Richard.Tibbetts@XK8St2.com|2452329| +7199|AAAAAAAAPBMBAAAA|1428445|5958|29518|2450228|2450198|Ms.|Jill|Hinton|N|14|11|1963|PUERTO RICO||Jill.Hinton@di3.edu|2452443| +7200|AAAAAAAAACMBAAAA|||39891|2450925||Sir|Ryan||N|3|4|||||2452476| +7201|AAAAAAAABCMBAAAA|1127298|6810|44722|2449759|2449729|Dr.|Willie|Santos|N|15|4|1966|SPAIN||Willie.Santos@5.edu|2452439| +7202|AAAAAAAACCMBAAAA|1629453|1984|42162|2451505|2451475|Mrs.|Juanita|Harris|Y|22|11|1984|NIUE||Juanita.Harris@tHh9Mr8bfRSX.org|2452536| +7203|AAAAAAAADCMBAAAA|1075426|5029|18632|2452166|2452136|Mrs.|Joann|Back|Y|18|7|1926|SOUTH AFRICA||Joann.Back@aFk35A.edu|2452402| +7204|AAAAAAAAECMBAAAA|513846|403|2160|2452515|2452485|Mr.|Gordon|Thompson|N|10|1|1970|MOZAMBIQUE||Gordon.Thompson@6UnNNpLmeB7k0t.com|2452587| +7205|AAAAAAAAFCMBAAAA|287145|3727|45062|2449327|2449297|Mrs.|Eloise|Scott|N|24|2|1949|GUINEA||Eloise.Scott@c.com|2452420| +7206|AAAAAAAAGCMBAAAA||136|23415||2449276|Mr.|||||||||Jeff.White@8uIxkZ7d1xVBo.org|| +7207|AAAAAAAAHCMBAAAA|297890|476|45462|2450374|2450344|Dr.|Christopher|Oneil|Y|23|8|1941|ERITREA||Christopher.Oneil@zzfVZIrgp67ri93Q.com|2452294| +7208|AAAAAAAAICMBAAAA|1387860|432|9004|2449822|2449792|Dr.|Serena|Patterson|N|25|3|1936|HONDURAS||Serena.Patterson@Dzo7kPuKv.com|2452347| +7209|AAAAAAAAJCMBAAAA|1562781|4754|33127|2449071|2449041|Dr.|Kimberly|Hemphill|Y|16|7|1964|VIET NAM||Kimberly.Hemphill@TjTTIQCRVPL.org|2452559| +7210|AAAAAAAAKCMBAAAA|617723|6493|33263|2450888|2450858|Mr.|Robert|Johnson|N|2|4|1966|PUERTO RICO||Robert.Johnson@jo.com|2452484| +7211|AAAAAAAALCMBAAAA|885021|5385|48596|2451089|2451059|Sir|Leonard|Sharp|Y|12|5|1969|SOLOMON ISLANDS||Leonard.Sharp@DrOCs4JGBh4QdNCCz.com|2452323| +7212|AAAAAAAAMCMBAAAA|628481|4171|45814|2451072|2451042|Dr.|Leola|Byrd|N|19|5|1968|GEORGIA||Leola.Byrd@i0UKmM8HNRmr.com|2452566| +7213|AAAAAAAANCMBAAAA|370475|1772|29469|2451876|2451846|Dr.|June|Espinoza|Y|24|8|1956|BURUNDI||June.Espinoza@1i2aplrEO9c8xuC.edu|2452537| +7214|AAAAAAAAOCMBAAAA|163848|3567|10933|2452283|2452253|Mrs.|Linda|Swain|N|12|3|1929|GERMANY||Linda.Swain@rrQ7D7tedH.com|2452384| +7215|AAAAAAAAPCMBAAAA|461719|3442|31565|2449856|2449826|Miss|Suzette|Miller|N|13|6|1979|WALLIS AND FUTUNA||Suzette.Miller@EEADp.org|2452567| +7216|AAAAAAAAADMBAAAA|1191466|4278|37043|2449460|2449430|Miss|Kristin|Gall|N|20|1|1967|AUSTRALIA||Kristin.Gall@HXRJ.org|2452410| +7217|AAAAAAAABDMBAAAA|1429080|2064|34648|2451149|2451119|Sir|Christopher|Romero|Y|15|2|1945|GUATEMALA||Christopher.Romero@pShH2xiiTLTo.edu|2452317| +7218|AAAAAAAACDMBAAAA|989257|851|17645|2449785|2449755|Sir|Steven|Carter|N|31|3|1938|GUAM||Steven.Carter@LP1tNyKsClt7OlmNI5.com|2452288| +7219|AAAAAAAADDMBAAAA|1800690|5343|48749|2449113|2449083|Sir|Joseph|Royal|N|9|9|1931|SPAIN||Joseph.Royal@ua.org|2452411| +7220|AAAAAAAAEDMBAAAA|270110|1159|14961|2452114|2452084|Mr.|Ramon|Smith|N|5|10|1945|PITCAIRN||Ramon.Smith@KxM0XYAb3CrAiRx.edu|2452588| +7221|AAAAAAAAFDMBAAAA|600104|4626|16795||2451711|||Ford|N|3|9||QATAR||Katrina.Ford@qxSffHQcryM7DEZtBY.org|| +7222|AAAAAAAAGDMBAAAA|1443622|7067|6112|2450582|2450552|Sir|Kyle|Flores|N|20|4|1928|ECUADOR||Kyle.Flores@fJkOQiQQ5SlBmp3y8iC.edu|2452568| +7223|AAAAAAAAHDMBAAAA|1409371|5106|35288|2449599|2449569|Mr.|Michael|Layne|N|30|7|1939|CANADA||Michael.Layne@4lqQSecRH.edu|2452311| +7224|AAAAAAAAIDMBAAAA|355375|5254|15086|2450446|2450416|Mrs.|Brenda|Alfaro|N|1|3|1932|KUWAIT||Brenda.Alfaro@H.org|2452540| +7225|AAAAAAAAJDMBAAAA|1262441|5735|44582|2449505|2449475|Ms.|Aaron|Walker|Y|2|6|1948|HONG KONG||Aaron.Walker@8pLCFe071Sp.edu|2452283| +7226|AAAAAAAAKDMBAAAA|1875985|5881|10177|2452411|2452381|Ms.|Anna|Beall|Y|9|2|1944|KUWAIT||Anna.Beall@RtQYXrl9.edu|2452463| +7227|AAAAAAAALDMBAAAA|1053873|3127|27534|2451243|2451213|Dr.|Kenneth|Porter|N|29|6|1989|WALLIS AND FUTUNA||Kenneth.Porter@iHfBIrc9ZHLTgmOZY.org|2452408| +7228|AAAAAAAAMDMBAAAA|1780967|5011|22608|2452392|2452362|Dr.|Ron|Pittman|Y|28|7|1984|TUVALU||Ron.Pittman@uM4.com|2452456| +7229|AAAAAAAANDMBAAAA|1177804|3043|34282|2451335|2451305|Sir|John|Phillips|N|18|8|1968|SWEDEN||John.Phillips@9y8x5Rq7dQ.com|2452295| +7230|AAAAAAAAODMBAAAA|1062343|1449|1708|2452186|2452156|Mrs.|Anna|Busby|Y|26|6|1934|DJIBOUTI||Anna.Busby@VNgdCZOtQbX.com|2452441| +7231|AAAAAAAAPDMBAAAA|1750353|400|16800|2449854|2449824|Sir|John|Boyd|N|13|7|1972|WESTERN SAHARA||John.Boyd@3vtKr.edu|2452495| +7232|AAAAAAAAAEMBAAAA|877256|2100|9791|2450972|2450942|Mrs.|Autumn|Gay|Y|29|8|1938|ICELAND||Autumn.Gay@p6oelJHn0NeKNYC.com|2452416| +7233|AAAAAAAABEMBAAAA|1734760|2836|32912|2451566|2451536|Dr.|Larry|Sherman|N|4|2|1942|MAURITIUS||Larry.Sherman@Xkb.com|2452420| +7234|AAAAAAAACEMBAAAA|1403186|3904|16175|2452629|2452599|Mr.|John|Dennis|N|27|4|1955|ARUBA||John.Dennis@r7Pu3ffqI.com|2452627| +7235|AAAAAAAADEMBAAAA|618027|1282|37737|2450951|2450921|Miss|Donna|Jones|N|4|12|1991|SINGAPORE||Donna.Jones@OsmzvY79.edu|2452486| +7236|AAAAAAAAEEMBAAAA|394486|3386|48733|2450346|2450316|Dr.|Timothy|Mason|Y|20|4|1924|NICARAGUA||Timothy.Mason@nGbnM4r8XYKpD3.com|2452633| +7237|AAAAAAAAFEMBAAAA|1285370|1837|12704|2449688|2449658|Mr.|Stephen|Lewis|N|1|1|1968|GERMANY||Stephen.Lewis@VIS.org|2452539| +7238|AAAAAAAAGEMBAAAA|1691340|853|33519|2450907|2450877|Sir|Lou|Khan|Y|4|10|1974|ICELAND||Lou.Khan@F2ZKA67FmAVYKnnF7fG.com|2452413| +7239|AAAAAAAAHEMBAAAA|1903397|7167|39384|2450250|2450220|Dr.|Eric|Morris|Y|20|11|1964|FRANCE||Eric.Morris@sFCt1DCThdU.com|2452613| +7240|AAAAAAAAIEMBAAAA|22738|4303|1068|2451191|2451161|Mrs.|Olive|Guajardo|Y|26|3|1929|SWAZILAND||Olive.Guajardo@E0FZ3jpEj2.org|2452295| +7241|AAAAAAAAJEMBAAAA|1380212|978|28944|2452416|2452386|Dr.|Stephen|Cole|N|11|8|1950|KYRGYZSTAN||Stephen.Cole@upJhpGytq.org|2452506| +7242|AAAAAAAAKEMBAAAA|1309341|3428|11468|2449626|2449596|Dr.|Edwin|Aguayo|N|28|6|1939|NEW ZEALAND||Edwin.Aguayo@CrdiPncbxg.edu|2452313| +7243|AAAAAAAALEMBAAAA|1503772|4107|1242|2449879|2449849|Mrs.|Kylie|Harrington|N|18|8|1947|SAN MARINO||Kylie.Harrington@SjgmH.edu|2452598| +7244|AAAAAAAAMEMBAAAA|1449003|762|11321|2450837|2450807|Dr.|Michael|Mueller|N|26|8|1936|TIMOR-LESTE||Michael.Mueller@xi1k.com|2452382| +7245|AAAAAAAANEMBAAAA|||13377|2449394|2449364|Mrs.|Christie|Murphy||27||1987||||| +7246|AAAAAAAAOEMBAAAA|1496416|617|24465|2450783|2450753|Mr.|Edwin|Fowler|Y|22|6|1977|MALTA||Edwin.Fowler@UUgJyvcHOzaVJcROC.com|2452441| +7247|AAAAAAAAPEMBAAAA|188906|5026|3546|2451959|2451929|Dr.|Garry|Haywood|N|31|5|1957|KIRIBATI||Garry.Haywood@cMQOxU0o1r.edu|2452325| +7248|AAAAAAAAAFMBAAAA|890204|5317|47230|2451761|2451731|Sir|Robert|Frame|Y|1|11|1929|WESTERN SAHARA||Robert.Frame@akJqVnVaHPYG3s.edu|2452332| +7249|AAAAAAAABFMBAAAA|1298106|845|38784|2449413|2449383|Mr.|Jose|Jones|Y|5|9|1980|MARSHALL ISLANDS||Jose.Jones@FK.org|2452410| +7250|AAAAAAAACFMBAAAA|298376|5037|40451|2450283|2450253|Dr.|Patrick|Dyer|N|30|12|1934|ANTARCTICA||Patrick.Dyer@DdJlsVPhf3LJXgAC.org|2452415| +7251|AAAAAAAADFMBAAAA|933863|3968|24073|2451439|2451409|Dr.|Regina|Green|N|3|12|1988|TRINIDAD AND TOBAGO||Regina.Green@q6Fl018IlnIR.edu|2452532| +7252|AAAAAAAAEFMBAAAA|||6506||2449913|Sir|David|Tisdale|||10|1961||||| +7253|AAAAAAAAFFMBAAAA|876828|1636|31997|2451095|2451065|Dr.|Debra|Manning|N|17|10|1981|FIJI||Debra.Manning@mnqZDJUuPg.org|2452558| +7254|AAAAAAAAGFMBAAAA|1311586|5604|7269|2449712|2449682|Mrs.|Donna|Boren|N|11|9|1988|GABON||Donna.Boren@gbIqO5RcfzpEx5c.edu|2452406| +7255|AAAAAAAAHFMBAAAA|206303|6647|31391|2450413|2450383|Sir|Christopher|Davis|N|22|4|1971|FRENCH GUIANA||Christopher.Davis@DTgaOMH4BFPp.com|2452493| +7256|AAAAAAAAIFMBAAAA|1085745|2392|22239|||Mr.||Smith|Y||||CUBA||William.Smith@dgnM31aqy638nfT.com|| +7257|AAAAAAAAJFMBAAAA|611472|6507|33095|2449365|2449335|Dr.|Donna|Chaney|Y|16|5|1984|FIJI||Donna.Chaney@uIuv0.org|2452530| +7258|AAAAAAAAKFMBAAAA|422155|6431|17954|2452169|2452139|Sir|John|Hoyt|Y|21|8|1932|LESOTHO||John.Hoyt@x2ohgiC.edu|2452355| +7259|AAAAAAAALFMBAAAA|1760821|4697|16126|2449685|2449655|Miss|Michelle|Hammond|N|30|7|1965|CAMBODIA||Michelle.Hammond@33oD2ghinBO03oqS2.org|2452482| +7260|AAAAAAAAMFMBAAAA|1835541|5478|11015|2450005|2449975|Miss|Karen|Dietz|N|4|9|1932|MACAO||Karen.Dietz@F.com|2452307| +7261|AAAAAAAANFMBAAAA|1494924|4432|38387|2452497|2452467|Dr.|Lea|Cameron|Y|13|2|1990|KIRIBATI||Lea.Cameron@Kcg0XyzvT6a.org|2452438| +7262|AAAAAAAAOFMBAAAA|332885|57|32429|2451524|2451494|Dr.|Dorothy|Watson|Y|7|12|1946|AMERICAN SAMOA||Dorothy.Watson@9a1os4fc.edu|2452575| +7263|AAAAAAAAPFMBAAAA|528975|79|49183|2451959|2451929|Sir|Ronald|Levy|N|27|11|1948|PANAMA||Ronald.Levy@Sj6HmZ.org|2452305| +7264|AAAAAAAAAGMBAAAA|704742|2059|38976|2449037|2449007|Mrs.|Susan|Manuel|Y|20|2|1933|BURUNDI||Susan.Manuel@Q1nsR2o.org|2452615| +7265|AAAAAAAABGMBAAAA|636632|5407|9782|2451759|2451729|Mr.|Joshua|Green|Y|18|9|1945|LESOTHO||Joshua.Green@LA2aD.com|2452404| +7266|AAAAAAAACGMBAAAA|234913|1455|39597|2449388|2449358|Dr.|Robert|Gonzalez|Y|10|6|1976|SAUDI ARABIA||Robert.Gonzalez@mm2tqChyndPHjPVx.edu|2452437| +7267|AAAAAAAADGMBAAAA|1361955|2442|16929|2449687|2449657|Miss|Denise|Higgins|N|15|3|1976|CHRISTMAS ISLAND||Denise.Higgins@Vu4a19461.org|2452557| +7268|AAAAAAAAEGMBAAAA|1829688|4382|43123|2450792|2450762|Ms.|Cheryl|Dooley|N|25|4|1941|MARSHALL ISLANDS||Cheryl.Dooley@8jlDal.org|2452301| +7269|AAAAAAAAFGMBAAAA|1067355||29059||2452101|Dr.|Bernard|Jennings|N||||MADAGASCAR||Bernard.Jennings@LX.com|2452382| +7270|AAAAAAAAGGMBAAAA|972067|4422|49977|2452520|2452490|Mr.|Stan|Gilbert|N|9|8|1927|PUERTO RICO||Stan.Gilbert@TErEbUR.org|2452572| +7271|AAAAAAAAHGMBAAAA|1670667|1876|8685|2450694|2450664|Dr.|Crystal|Maples|N|12|5|1933|BENIN||Crystal.Maples@JG3VHCCSLp5523Z.org|2452320| +7272|AAAAAAAAIGMBAAAA|1025097|69|29888|2451942|2451912|Mr.|Lawrence|Hawthorne|N|17|6|1991|SAINT LUCIA||Lawrence.Hawthorne@nkFPGQb1.org|2452361| +7273|AAAAAAAAJGMBAAAA|1568797|4386|47928|2450487|2450457|Ms.|Christine|Dukes|N|5|6|1943|ITALY||Christine.Dukes@D.com|2452322| +7274|AAAAAAAAKGMBAAAA|1641479|4037|28595|2451611|2451581|Sir|Kurt|Farmer|N|5|11|1972|TIMOR-LESTE||Kurt.Farmer@rfjcn48zoy9x.edu|2452299| +7275|AAAAAAAALGMBAAAA|426781|252|33723|2452450|2452420|Sir|Carlton|Mueller|N|23|10|1949|EQUATORIAL GUINEA||Carlton.Mueller@d36Mfknr1SqbNe.edu|2452470| +7276|AAAAAAAAMGMBAAAA|1807479|4007|11735|2450948|2450918|Mr.|Jeffrey|Morris|Y|14|1|1956|LITHUANIA||Jeffrey.Morris@M2LyIjroCc.edu|2452525| +7277|AAAAAAAANGMBAAAA|1025974|2093|49412|2451110|2451080|Mr.|Eddie|Ramirez|N|26|12|1931|TUNISIA||Eddie.Ramirez@1.edu|2452440| +7278|AAAAAAAAOGMBAAAA|1253348|6410|30672|2450753|2450723|Mrs.|Sylvia|Bolduc|N|10|11|1965|GUINEA-BISSAU||Sylvia.Bolduc@OGBDNmHv50PS6.edu|2452302| +7279|AAAAAAAAPGMBAAAA|1111015|6194|34126|2449909|2449879|Sir|Jerry|Walters|N|28|2|1972|VANUATU||Jerry.Walters@rUoR1rXYTVm.com|2452434| +7280|AAAAAAAAAHMBAAAA|897650|256|62|2452406|2452376|Dr.|Laura|Broadway|Y|20|10|1956|ESTONIA||Laura.Broadway@QtQjHvnlum3L.com|2452483| +7281|AAAAAAAABHMBAAAA|760387|2623|38059||2451246|Ms.||||18||1948||||2452423| +7282|AAAAAAAACHMBAAAA|251399|6100|40801|2451836|2451806|Sir|Bradley|Parker|Y|26|5|1926|ERITREA||Bradley.Parker@VBxHqi.org|2452438| +7283|AAAAAAAADHMBAAAA|1778699|6961|12823|2450750|2450720|Mrs.|Adam|Wills|Y|24|8|1964|AUSTRIA||Adam.Wills@T.edu|2452387| +7284|AAAAAAAAEHMBAAAA|484704|3455|9806|2449523|2449493|Mrs.|Benita|Thomas|Y|10|1|1937|LEBANON||Benita.Thomas@U6kjtfUjor5j6Xa.edu|2452361| +7285|AAAAAAAAFHMBAAAA|695665|807|49996|2452441||Mr.|Jacob|Harris|N|||||||2452457| +7286|AAAAAAAAGHMBAAAA||2893|49253|2450253|||||Y|||1974||||2452621| +7287|AAAAAAAAHHMBAAAA|387560|978|11596|2452414|2452384|Mrs.|Bessie|Andrews|N|20|3|1948|SUDAN||Bessie.Andrews@k9uB1ZEgFFH.edu|2452314| +7288|AAAAAAAAIHMBAAAA|1461725|4938|18198|2450838|2450808|Sir|Steven|Storey|Y|1|2|1967|QATAR||Steven.Storey@QdGVDzkOB2lH.edu|2452528| +7289|AAAAAAAAJHMBAAAA|1525195|1455|32115|2452193|2452163|Mr.|Michael|Sanderson|N|20|6|1933|BENIN||Michael.Sanderson@8KV2aR.org|2452355| +7290|AAAAAAAAKHMBAAAA|1258295|1197|40099|2452041|2452011|Dr.|Charles|Khan|N|7|8|1941|CAYMAN ISLANDS||Charles.Khan@uhtEraq.com|2452395| +7291|AAAAAAAALHMBAAAA|873181|1460|3292|2451726|2451696|Dr.|Stuart|Little|Y|7|11|1927|BRUNEI DARUSSALAM||Stuart.Little@aD86Z0ok.com|2452613| +7292|AAAAAAAAMHMBAAAA|520480|2828|33929|2450384|2450354|Ms.|Khalilah|Mattox|Y|18|3|1924|TURKEY||Khalilah.Mattox@7iq8kuIgOO.com|2452296| +7293|AAAAAAAANHMBAAAA|400961|240|43919|2450668|2450638|Mr.|Edward|Walker|Y|22|3|1933|AUSTRALIA||Edward.Walker@pFcYM0kxxV8EO0Mnrs.com|2452553| +7294|AAAAAAAAOHMBAAAA|838585|6898|4458|2451561|2451531|Dr.|Janet|Hebert|Y|30|12|1950|MADAGASCAR||Janet.Hebert@M151aF.org|2452523| +7295|AAAAAAAAPHMBAAAA|110298|4173|43295|2450121|2450091|Mr.|John|Williams|N|20|2|1946|ISRAEL||John.Williams@cqEmiNO4F14.org|2452403| +7296|AAAAAAAAAIMBAAAA|185148|815|7682|2449571|2449541|Dr.|Kristin|Avery|N|29|6|1971|SWAZILAND||Kristin.Avery@ngKKP4AYaL.com|2452375| +7297|AAAAAAAABIMBAAAA||994|13805|2450692||||Estrella|Y|12|||BULGARIA||Alisha.Estrella@iHGUnypBFK8I.org|2452579| +7298|AAAAAAAACIMBAAAA|1096888|1765|9042|2451592|2451562|Dr.|Richard|Hudson|N|12|12|1949|UNITED STATES||Richard.Hudson@H9yQF8.edu|2452381| +7299|AAAAAAAADIMBAAAA|713296|1261|5226|2449989|2449959|Mrs.|Alexis|Gonzales|Y|16|11|1937|FAROE ISLANDS||Alexis.Gonzales@6Ss3TjHQlnp987.org|2452420| +7300|AAAAAAAAEIMBAAAA|1086217|3063|1062|2451546|2451516|Mr.|Joseph|Snow|Y|7|1|1931|HAITI||Joseph.Snow@OmmrI9gZGrX2crtD3.com|2452615| +7301|AAAAAAAAFIMBAAAA|1898421|953|16685|2449476|2449446|Dr.|Mason|Lyon|N|1|3|1935|CAPE VERDE||Mason.Lyon@BERaL34FFcet.edu|2452283| +7302|AAAAAAAAGIMBAAAA|432033|6892|1704|2452653|2452623|Dr.|Ben|Masterson|N|26|4|1969|BOUVET ISLAND||Ben.Masterson@b.org|2452642| +7303|AAAAAAAAHIMBAAAA|1363062|44|25826|2451926|2451896|Dr.|Elizabeth|Morrison|Y|8|4|1944|SAINT HELENA||Elizabeth.Morrison@PdBS9SCkZI.com|2452348| +7304|AAAAAAAAIIMBAAAA|1817643|4826|6809|2451232|2451202|Mrs.|Wendy|Andersen|N|20|3|1949|FRENCH POLYNESIA||Wendy.Andersen@cIturoPemg.edu|2452450| +7305|AAAAAAAAJIMBAAAA|422841|4235|17629|2449967|2449937|Mrs.|Stephanie|Malcolm|N|31|5|1978|NORWAY||Stephanie.Malcolm@e.edu|2452392| +7306|AAAAAAAAKIMBAAAA|1442162|6365|20121|2451109|2451079|Sir|David|Johnson|N|7|6|1966|VANUATU||David.Johnson@DphYOLnU9y.com|2452432| +7307|AAAAAAAALIMBAAAA|1420368|357|44346|2452580|2452550|Dr.|Juan|Harmon|Y|7|8|1961|KYRGYZSTAN||Juan.Harmon@ti.edu|2452323| +7308|AAAAAAAAMIMBAAAA|1127967|1715|17910|2451310|2451280|Sir|Byron|Branch|Y|8|11|1935|BOUVET ISLAND||Byron.Branch@M2BziTdz8rIFe.edu|2452587| +7309|AAAAAAAANIMBAAAA|1194498|4703|34061|2449620|2449590|Ms.|Elfriede|Roldan|N|15|11|1976|RWANDA||Elfriede.Roldan@NX50e9DIbYzCUyi.edu|2452435| +7310|AAAAAAAAOIMBAAAA|615527|6197|28020|2450215|2450185|Sir|Scott|Williamson|N|27|1|1981|EGYPT||Scott.Williamson@BtazuRdUSaLVsCVN7.org|2452391| +7311|AAAAAAAAPIMBAAAA|1752031|4395|20312|2452217|2452187|Miss|Margaret|Gonzalez|N|2|1|1969|ZIMBABWE||Margaret.Gonzalez@a6hNS2sKiG4.edu|2452571| +7312|AAAAAAAAAJMBAAAA|1541804|2196|13741|2452582|2452552|Ms.|Sheila|Mclain|Y|6|2|1959|SPAIN||Sheila.Mclain@1tIBFPIDtQl.edu|2452321| +7313|AAAAAAAABJMBAAAA|1347751|370|26589|2452677|2452647|Mr.|Steven|Carter|Y|1|2|1948|PAPUA NEW GUINEA||Steven.Carter@F.edu|2452363| +7314|AAAAAAAACJMBAAAA|1863278|5043|45208|2452000|2451970|Mr.|Alvin|Simpson|N|16|9|1982|AUSTRALIA||Alvin.Simpson@2bFjQnv.edu|2452459| +7315|AAAAAAAADJMBAAAA|1846259|2927|8748|2452069|2452039|Mr.|Stefan|Coleman|N|19|2|1973|EL SALVADOR||Stefan.Coleman@XSdpIgY8BncG.edu|2452637| +7316|AAAAAAAAEJMBAAAA|1295696|3601|49459|2449524|2449494|Mrs.|Allen|Thompson|N|5|3|1956|SAUDI ARABIA||Allen.Thompson@hGn.com|2452290| +7317|AAAAAAAAFJMBAAAA|1602289|4026|46963|2450830|2450800|Mr.|Patrick|Rasmussen|Y|27|3|1966|MALTA||Patrick.Rasmussen@PTb0FbHauaav.edu|2452325| +7318|AAAAAAAAGJMBAAAA|744922|4727|9732|2452485|2452455|Mr.|Mike|Mcdonald|Y|24|3|1956|SYRIAN ARAB REPUBLIC||Mike.Mcdonald@dSf6kzFmZS049ZD.com|2452508| +7319|AAAAAAAAHJMBAAAA|923658|6549|37793|2449445|2449415|Ms.|Helen|Crockett|N|6|12|1931|JAPAN||Helen.Crockett@5SDmCTa5.com|2452581| +7320|AAAAAAAAIJMBAAAA|528492|5527|9409|2450059|2450029|Dr.|Brian|Ross|N|9|12|1978|SURINAME||Brian.Ross@ox.org|2452370| +7321|AAAAAAAAJJMBAAAA|851179|4087|39348|2452074|2452044|Miss|Gladys|Miranda|Y|16|1|1970|MALDIVES||Gladys.Miranda@4X8CfUuFRx.com|2452543| +7322|AAAAAAAAKJMBAAAA|1276230|4719|12808|2450934|2450904|Mr.|Russell|Barnes|Y|17|3|1966|ZIMBABWE||Russell.Barnes@bRTHc2J1f.com|2452484| +7323|AAAAAAAALJMBAAAA|1691326|4752|32178|2449661|2449631|Dr.|Richard|Clarkson|N|19|12|1985|LIECHTENSTEIN||Richard.Clarkson@2jVHRRfa.org|2452535| +7324|AAAAAAAAMJMBAAAA|910424|3754|237|2449440|2449410|Ms.|Ada|Garcia|N|2|11|1938|TRINIDAD AND TOBAGO||Ada.Garcia@50lznoGubFF.edu|2452479| +7325|AAAAAAAANJMBAAAA|1480426|3725|14977|2451832|2451802|Miss|Kathryn|Carney|Y|8|8|1928|ZIMBABWE||Kathryn.Carney@Ghd.edu|2452347| +7326|AAAAAAAAOJMBAAAA|426540|164|6588|2450763|2450733|Dr.|Steven|Samson|Y|21|8|1932|AMERICAN SAMOA||Steven.Samson@HIjS3B0HVU.org|2452362| +7327|AAAAAAAAPJMBAAAA|2342|5673|43004|2450749|2450719|Dr.|Bok|Stewart|N|13|3|1979|R�UNION||Bok.Stewart@9V.org|2452426| +7328|AAAAAAAAAKMBAAAA|1525898|3101|38257|2449667|2449637|Mr.|Joseph|Watkins|Y|14|12|1975|FINLAND||Joseph.Watkins@LloB.org|2452515| +7329|AAAAAAAABKMBAAAA|196044|7132|15820|2450729|2450699|Mrs.|Kimberly|Fox|Y|17|1|1986|MALI||Kimberly.Fox@efo.edu|2452361| +7330|AAAAAAAACKMBAAAA|146916|4927|23738|2449061|2449031|Dr.|Tom|Sherman|Y|15|7|1948|INDONESIA||Tom.Sherman@57q4Ls0yZlY.com|2452503| +7331|AAAAAAAADKMBAAAA|880481|5290|25299|2451823|2451793|Mr.|Trinidad|Moore|Y|14|5|1966|KENYA||Trinidad.Moore@B3m8V580MTx.edu|2452437| +7332|AAAAAAAAEKMBAAAA|144388|3787|7939|2451650|2451620|Dr.|Brandon|Brooks|Y|1|11|1958|THAILAND||Brandon.Brooks@VUUbdE4rv8L9.com|2452459| +7333|AAAAAAAAFKMBAAAA|1787654|2827|43153|2452374|2452344|Miss|Sherry|Thomas|N|6|1|1971|ANTIGUA AND BARBUDA||Sherry.Thomas@vldF.edu|2452640| +7334|AAAAAAAAGKMBAAAA|32933|2074|25168|2449045|2449015|Miss|Betty|Williams|N|6|10|1976|HUNGARY||Betty.Williams@qIBMMcee.edu|2452444| +7335|AAAAAAAAHKMBAAAA|1874623|5841|47459|2449486|2449456|Dr.|Lillie|Cody|Y|7|6|1940|UZBEKISTAN||Lillie.Cody@DTnx0IhSi.com|2452360| +7336|AAAAAAAAIKMBAAAA|71227|1072|1317|2451752|2451722|Ms.|Suzanne|Damon|Y|14|6|1958|PALAU||Suzanne.Damon@mlx2y.org|2452632| +7337|AAAAAAAAJKMBAAAA|365806|3822|28685|2451720|2451690|Miss|Elizabeth|Rodrigues|Y|5|1|1978|PAKISTAN||Elizabeth.Rodrigues@Oph2.org|2452292| +7338|AAAAAAAAKKMBAAAA|535077|4898|45566|2452451|2452421|Mr.|Thomas|Williams|N|24|12|1930|GABON||Thomas.Williams@sLJJZQ.edu|2452289| +7339|AAAAAAAALKMBAAAA|118874|3181|39888|2451689|2451659|Mrs.|Anne|Powers|Y|6|8|1966|SAINT LUCIA||Anne.Powers@ofIMqe2.com|2452478| +7340|AAAAAAAAMKMBAAAA|1554993|2199|29405|2452195|2452165|Miss|Doris|Thomas|Y|24|8|1982|MYANMAR||Doris.Thomas@0AEev.org|2452319| +7341|AAAAAAAANKMBAAAA|1777759|932|9705|2449641|2449611|Mr.|Richard|King|Y|26|4|1927|ROMANIA||Richard.King@CumCFfmMhh.com|2452564| +7342|AAAAAAAAOKMBAAAA|975614|1917|15773|2451545|2451515|Dr.|Ailene|Moreno|N|25|3|1986|FRENCH GUIANA||Ailene.Moreno@HI7u47s8hHKUqnslnG.edu|2452371| +7343|AAAAAAAAPKMBAAAA|306800|5011|25269|2449944|2449914|Sir|Raymond|Knapp|N|14|3|1930|RUSSIAN FEDERATION||Raymond.Knapp@A3jI7H15HyY.com|2452470| +7344|AAAAAAAAALMBAAAA|640533|3620|9554|2450856|2450826|Sir|Michael|Bustamante|N|6|3|1991|BARBADOS||Michael.Bustamante@uoT1nBy8PeRb.com|2452317| +7345|AAAAAAAABLMBAAAA|1678612|1453|44294|2450480|2450450|Mr.|Isidro|Blount|N|20|11|1942|RUSSIAN FEDERATION||Isidro.Blount@BubDxNTaLdO0lABp7dV.edu|2452572| +7346|AAAAAAAACLMBAAAA|405038|2489|35540|2452134|2452104|Dr.|Tim|Devine|N|20|4|1940|DOMINICA||Tim.Devine@eVxjCMTV7j.org|2452318| +7347|AAAAAAAADLMBAAAA|1197168|1390|37259|2452489|2452459|Ms.|Michelle|Harris|Y|16|1|1976|LUXEMBOURG||Michelle.Harris@YKhM12nsb0V.org|2452347| +7348|AAAAAAAAELMBAAAA|415796|6559|26075|2449922|2449892|Sir|William|Johnson|Y|20|2|1978|UKRAINE||William.Johnson@xZA3mlFd.edu|2452381| +7349|AAAAAAAAFLMBAAAA|506006|5809|45212|2450924|2450894|Dr.|Ernest|Ewing|Y|27|3|1929|SPAIN||Ernest.Ewing@nNZI18Ocbr8Qr8xp.com|2452336| +7350|AAAAAAAAGLMBAAAA|1909773|2493|41613|2450973|2450943|Sir|Joseph|Pierce|N|27|11|1943|ALAND ISLANDS||Joseph.Pierce@tX8LSNuLub7l.com|2452478| +7351|AAAAAAAAHLMBAAAA|1414290|4373|32456|2450652|2450622|Mr.|Carl|Holland|N|11|8|1964|BENIN||Carl.Holland@BiV45AKmC3KE5Bz.org|2452568| +7352|AAAAAAAAILMBAAAA|1780187|1023|49106|2449154|2449124|Dr.|Glenn|Morales|N|23|1|1971|AFGHANISTAN||Glenn.Morales@V.edu|2452508| +7353|AAAAAAAAJLMBAAAA|264474|585|15096|2452405|2452375|Miss|Lillian|Bennett|Y|11|1|1971|ANTARCTICA||Lillian.Bennett@hjDMu6u.com|2452380| +7354|AAAAAAAAKLMBAAAA|1445546|5778|13610|2450127|2450097|Dr.|Paul|Gaither|N|19|2|1970|VIET NAM||Paul.Gaither@xhyTD6Dxdd6Q.com|2452549| +7355|AAAAAAAALLMBAAAA|159761|296|49184|2452534|2452504|Dr.|Eldon|Putman|Y|20|4|1961|POLAND||Eldon.Putman@t9cpHKfqbQte.com|2452452| +7356|AAAAAAAAMLMBAAAA|1059745|1284|40423|2451595|2451565|Sir|Scott|Johnson|N|20|10|1962|GAMBIA||Scott.Johnson@60nqS6kiI.com|2452335| +7357|AAAAAAAANLMBAAAA|1147246|6361|13876|2449268|2449238|Miss|Faith|Hyman|N|26|12|1955|GRENADA||Faith.Hyman@rkxB.com|2452451| +7358|AAAAAAAAOLMBAAAA|710333|1375|13617|2449224|2449194|Miss|Antonia|Haney|N|22|2|1988|NORFOLK ISLAND||Antonia.Haney@tFXDIjaVIL6k64.org|2452327| +7359|AAAAAAAAPLMBAAAA|377011|5900|19189|2451924|2451894|Dr.|Frank|Acevedo|Y|21|1|1932|NIUE||Frank.Acevedo@1Vi8S8b.com|2452435| +7360|AAAAAAAAAMMBAAAA||4157|21016|||Sir|Mathew||||3||||Mathew.Painter@CuKX.org|2452366| +7361|AAAAAAAABMMBAAAA|1690299|6103|43860|2452672|2452642|Sir|Wilfred|Brown|Y|5|5|1952|GUAM||Wilfred.Brown@kppIJee.org|2452581| +7362|AAAAAAAACMMBAAAA|1382209|911|29260|2450550|2450520|Ms.|Gloria|Moore|N|18|1|1942|SWAZILAND||Gloria.Moore@vFR8jdiLiNqVP0j.edu|2452568| +7363|AAAAAAAADMMBAAAA|1742128|1496|7711|2451812|2451782|Dr.|Alfonso|Thomas|N|31|8|1983|MOLDOVA, REPUBLIC OF||Alfonso.Thomas@Q6SyE3prrhalJH.edu|2452299| +7364|AAAAAAAAEMMBAAAA|1299753|3662|5971|2452376|2452346|Ms.|Rita|Carr|N|18|3|1963|HAITI||Rita.Carr@QdhKZ7PQVzR969DRB.com|2452451| +7365|AAAAAAAAFMMBAAAA|1732001|3887|22973|2451503|2451473|Ms.|Terresa|Lacy|N|18|2|1971|FRANCE||Terresa.Lacy@RTj.org|2452636| +7366|AAAAAAAAGMMBAAAA|452044|330|33369|2450561|2450531|Ms.|Monica|Kingsbury|Y|29|10|1965|NIUE||Monica.Kingsbury@B3rHNG6y11.edu|2452370| +7367|AAAAAAAAHMMBAAAA|26833|4206|1979|2450813|2450783|Dr.|Laura|Redmon|N|7|5|1947|SAUDI ARABIA||Laura.Redmon@rOIGE87ikilxmnaoDBJ.edu|2452391| +7368|AAAAAAAAIMMBAAAA|367749|4156|39363|2452137|2452107|Sir|Jeffrey|Mundy|Y|19|6|1987|VIRGIN ISLANDS, U.S.||Jeffrey.Mundy@uqGQHViDEd9yISkoyL.org|2452505| +7369|AAAAAAAAJMMBAAAA|13517|5815|44513|2449037|2449007|Mr.|David|Rose|N|23|5|1975|HONDURAS||David.Rose@LPyS0PgJfSb.edu|2452337| +7370|AAAAAAAAKMMBAAAA|1463704||40010|2452143|||||||12||SAN MARINO|||2452423| +7371|AAAAAAAALMMBAAAA|1833378|304|11183|2449597|2449567|Mrs.|Christina|Gamble|N|29|12|1980|OMAN||Christina.Gamble@G.org|2452387| +7372|AAAAAAAAMMMBAAAA|1766430|685|17740|2450085|2450055|Dr.|Jenny|Bailey|N|28|12|1976|ALBANIA||Jenny.Bailey@i7qP2KDhD8DXMBjvfU.com|2452360| +7373|AAAAAAAANMMBAAAA|2802|2801|45553|2451152|2451122|Miss|Lindy|Lawrence|N|2|10|1945|JORDAN||Lindy.Lawrence@Z1r3gBu64GK.edu|2452563| +7374|AAAAAAAAOMMBAAAA|1494283|4787|29291|2451177|2451147|Mrs.|Kendra|Smith|Y|4|1|1988|NETHERLANDS ANTILLES||Kendra.Smith@0.com|2452467| +7375|AAAAAAAAPMMBAAAA|1367759|2548|14584|2452417|2452387|Mr.|John|Lutz|Y|14|4|1964|PALAU||John.Lutz@CUq6xlv.org|2452405| +7376|AAAAAAAAANMBAAAA|113591|5076|18161|2449436|2449406|Mr.|Joseph|Mason|N|20|4|1931|ESTONIA||Joseph.Mason@uVXtVLpTPB.edu|2452326| +7377|AAAAAAAABNMBAAAA|306204|4640|11076|2450683|2450653|Dr.|Savannah|Bauer|Y|3|3|1991|PAKISTAN||Savannah.Bauer@fR4FusMk2.org|2452601| +7378|AAAAAAAACNMBAAAA|145748|4296|32844|2451694|2451664|Mrs.|Kathryn|Garcia|N|12|2|1951|TONGA||Kathryn.Garcia@t739C.com|2452628| +7379|AAAAAAAADNMBAAAA|1081725|3216|10754|2451777|2451747|Dr.|Holly|Reynolds|Y|25|7|1962|MOLDOVA, REPUBLIC OF||Holly.Reynolds@A4j8oFKCaV.org|2452643| +7380|AAAAAAAAENMBAAAA|777239|2440|44888|2451214|2451184|Ms.|Rosetta|Green|N|1|5|1949|LIBERIA||Rosetta.Green@p8fDAr.edu|2452439| +7381|AAAAAAAAFNMBAAAA|870160|773|586|2450866|2450836|Dr.|Shawnna|Freeland|Y|17|3|1926|DOMINICA||Shawnna.Freeland@QsyGjod1tnU.com|2452490| +7382|AAAAAAAAGNMBAAAA||3861|33976||||Mayra|||||1949||||2452590| +7383|AAAAAAAAHNMBAAAA|381096|5036|49423|2449150|2449120|Mrs.|Heather|Williams|N|4|8|1954|FAROE ISLANDS||Heather.Williams@bCfA.edu|2452577| +7384|AAAAAAAAINMBAAAA|1720803|912|49021|2450866|2450836|Sir|Robert|Stokes|N|26|8|1972|ARMENIA||Robert.Stokes@16.org|2452360| +7385|AAAAAAAAJNMBAAAA|1435293|6149|33405|2451476|2451446|Dr.|Marie|Diaz|N|22|3|1973|SLOVAKIA||Marie.Diaz@XNgITBUYzn6SI.edu|2452538| +7386|AAAAAAAAKNMBAAAA|1171962|3346|17530|2450838|2450808|Ms.|Becky|Brooks|N|21|7|1989|OMAN||Becky.Brooks@rUItByBK4EAP2zP.org|2452303| +7387|AAAAAAAALNMBAAAA|1844995|886|30377|2450684|2450654|Dr.|William|Thomas|Y|26|11|1940|SOLOMON ISLANDS||William.Thomas@hUqh3KpIEHPP.edu|2452511| +7388|AAAAAAAAMNMBAAAA|1189471|3540|40009|2450959|2450929|Ms.|Julienne|Simms|Y|5|12|1960|ITALY||Julienne.Simms@K9D9gYBMFFGD3R.org|2452514| +7389|AAAAAAAANNMBAAAA|949707|3197|32334|2452393|2452363|Mrs.|Rosa|Robertson|N|26|8|1992|MOROCCO||Rosa.Robertson@tSHFHlnZsNIMQkXI.com|2452284| +7390|AAAAAAAAONMBAAAA|967073|6|44139|2451198|2451168|Mrs.|Andre|Chambers|Y|19|8|1937|GUINEA||Andre.Chambers@g1Sz0cU.edu|2452408| +7391|AAAAAAAAPNMBAAAA|1011827|5298|37169|2451724|2451694|Sir|Mike|Luna|Y|2|9|1967|HONDURAS||Mike.Luna@bDvKZ1fvZRRKKtdv.edu|2452624| +7392|AAAAAAAAAOMBAAAA|330806|7182|45558|2451234|2451204|Dr.|Velvet|Taylor|Y|18|1|1963|THAILAND||Velvet.Taylor@H5JNBkN.org|2452383| +7393|AAAAAAAABOMBAAAA|507137|7069|23001|2451301|2451271|Mr.|Harry|Hodges|N|9|6|1952|SEYCHELLES||Harry.Hodges@1C85v.org|2452505| +7394|AAAAAAAACOMBAAAA|106482|4706|20930|2451425|2451395|Dr.|Gerald|Holmes|Y|25|12|1967|R�UNION||Gerald.Holmes@IstR0MK8.com|2452514| +7395|AAAAAAAADOMBAAAA|1391356|6830|29457|2449873|2449843|Sir|Emmanuel|Ashley|Y|21|7|1960|R�UNION||Emmanuel.Ashley@Cgugmp.org|2452553| +7396|AAAAAAAAEOMBAAAA|1129791|4283|39975|2450846|2450816|Mrs.|Belle|Hernandez|N|4|1|1950|CHRISTMAS ISLAND||Belle.Hernandez@OMPA4utalVnrK.edu|2452589| +7397|AAAAAAAAFOMBAAAA|1701386|1512|44885|2452378|2452348|Dr.|Amber|Robinson|N|25|5|1933|BAHAMAS||Amber.Robinson@YLgpdD0lhCJ.edu|2452560| +7398|AAAAAAAAGOMBAAAA|1533141|3550|6406|2451211|2451181|Mr.|Richard|Gault|Y|3|4|1960|ALBANIA||Richard.Gault@rULvKUt.com|2452463| +7399|AAAAAAAAHOMBAAAA|1268751|5861|32571|2449417|2449387|Miss|Jennifer|Adler|N|23|12|1991|AZERBAIJAN||Jennifer.Adler@ONJSCG5xKiJ.com|2452411| +7400|AAAAAAAAIOMBAAAA|1197222|6059|11696|2451411|2451381|Mrs.|Ana|Smith|Y|25|8|1978|GAMBIA||Ana.Smith@kfYBExU9ej2S.edu|2452524| +7401|AAAAAAAAJOMBAAAA|230861|1281|22454|2452264|2452234|Sir|Johnnie|Prater|N|16|1|1976|GUINEA-BISSAU||Johnnie.Prater@4IJBZNdzv3geTV.com|2452398| +7402|AAAAAAAAKOMBAAAA|1876896|6257|25571|2449205|2449175|Ms.|Valerie|Rivera|N|21|5|1992|MARTINIQUE||Valerie.Rivera@h.com|2452612| +7403|AAAAAAAALOMBAAAA|1105919|3272|38178|2451957|2451927|Dr.|Jeff|Ryan|N|26|9|1963|MALAWI||Jeff.Ryan@hE4v.com|2452574| +7404|AAAAAAAAMOMBAAAA||2134|41792|2450402|||Margaret|Banks||13|7||BOTSWANA||Margaret.Banks@QEa2Nd8.edu|| +7405|AAAAAAAANOMBAAAA|1566596|1633|21870|2451882|2451852|Dr.|Tabitha|Moyer|N|21|3|1978|KYRGYZSTAN||Tabitha.Moyer@lQ7v2d1x2vng6t7r.com|2452561| +7406|AAAAAAAAOOMBAAAA|1768292|3035|22578|2451768|2451738|Mrs.|Lori|Abel|Y|20|3|1963|KAZAKHSTAN||Lori.Abel@j9YRuJk.edu|2452356| +7407|AAAAAAAAPOMBAAAA|645360|1251|30295|2450351|2450321|Dr.|Roosevelt|Webb|Y|6|3|1992|MALTA||Roosevelt.Webb@e9DPXnacnur1EcfbxSL.com|2452473| +7408|AAAAAAAAAPMBAAAA|801683|2260|437|2450610|2450580|Ms.|Andrea|Turney|N|18|9|1937|WESTERN SAHARA||Andrea.Turney@DMprD.edu|2452566| +7409|AAAAAAAABPMBAAAA|320751|6508|2265|2450911|2450881|Mrs.|Laurie|Mercado|Y|6|3|1939|GAMBIA||Laurie.Mercado@3eyxv00MBx5Xq.edu|2452305| +7410|AAAAAAAACPMBAAAA|382228|1591|19707|2451604|2451574|Dr.|Carla|Becker|Y|30|11|1928|NEPAL||Carla.Becker@qV.org|2452489| +7411|AAAAAAAADPMBAAAA|1505637|651|29665|2452620|2452590|Ms.|Amanda|Gross|Y|1|8|1981|SLOVENIA||Amanda.Gross@G8Mth.com|2452566| +7412|AAAAAAAAEPMBAAAA|932088|41|17594|2450424|2450394|Sir|Roger|Owens|N|2|9|1949|NORWAY||Roger.Owens@VALeysn0g.org|2452314| +7413|AAAAAAAAFPMBAAAA|1796045|2174|17103|2450565|2450535|Mrs.|Allyson|Knox|N|19|5|1989|SIERRA LEONE||Allyson.Knox@s.edu|2452343| +7414|AAAAAAAAGPMBAAAA|1326455|1563|6624|2449953|2449923|Dr.|Norbert|Williams|Y|14|3|1931|GEORGIA||Norbert.Williams@o7pOo8.edu|2452497| +7415|AAAAAAAAHPMBAAAA|1009182|5679|15329|2450035|2450005|Dr.|Tiana|Olson|Y|11|1|1930|BANGLADESH||Tiana.Olson@MYicEuJAYYUV.com|2452492| +7416|AAAAAAAAIPMBAAAA|1125191|1590|3189|2449999|2449969|Ms.|Ella|Bradley|N|17|11|1986|GUINEA||Ella.Bradley@La.com|2452454| +7417|AAAAAAAAJPMBAAAA|955871|5379|12087|2451961|2451931|Ms.|Dorothy|Reece|N|12|3|1933|KOREA, REPUBLIC OF||Dorothy.Reece@gAR9cKKh.edu|2452482| +7418|AAAAAAAAKPMBAAAA|||41655||2452253|Ms.|Margaret|Garcia|Y|31|7|1946||||| +7419|AAAAAAAALPMBAAAA|1770739|6661|14583|2450572|2450542|Miss|Gloria|Hess|Y|9|12|1966|TUVALU||Gloria.Hess@BSket1mnnEs.org|2452551| +7420|AAAAAAAAMPMBAAAA|1202664|5484|48304|2449032|2449002|Dr.|Laura|Gonzales|N|5|9|1984|NETHERLANDS||Laura.Gonzales@9mHIBhxS6.com|2452287| +7421|AAAAAAAANPMBAAAA|145996||25687|2451979|2451949||Silvia|||2||1953|||Silvia.Ruiz@cn1Ya.edu|2452303| +7422|AAAAAAAAOPMBAAAA|681663|4160|37857|2449674|2449644|Miss|Victoria|Gresham|Y|28|4|1961|GUAM||Victoria.Gresham@EODjiZ.com|2452510| +7423|AAAAAAAAPPMBAAAA|457095|5830|4918|2452207|2452177|Sir|Timothy|Hernandez|Y|6|3|1928|IRELAND||Timothy.Hernandez@ATJFX4HxnNt.edu|2452331| +7424|AAAAAAAAAANBAAAA|849905|5058|24964|2451047|2451017|Ms.|Stephanie|Groves|N|26|8|1978|SEYCHELLES||Stephanie.Groves@SKtbsqi4Hx1.org|2452596| +7425|AAAAAAAABANBAAAA||5461|3546||||||||9||PAPUA NEW GUINEA|||| +7426|AAAAAAAACANBAAAA|595104|6255|9806|2451515|2451485|Miss|Julia|Atkins|Y|23|10|1973|TAJIKISTAN||Julia.Atkins@PcfDVkOsAr.org|2452499| +7427|AAAAAAAADANBAAAA|119879|161|16796|2449312|2449282|Dr.|Vickie|Washington|Y|20|10|1970|GREECE||Vickie.Washington@pr.edu|2452289| +7428|AAAAAAAAEANBAAAA|954361|7043|2525|2450875|2450845|Sir|Joe|Moore|N|1|3|1937|PAKISTAN||Joe.Moore@UiN2M.com|2452447| +7429|AAAAAAAAFANBAAAA|929072|4586|3700|2449770|2449740|Mrs.|Marie|Carter|N|16|5|1969|ANTIGUA AND BARBUDA||Marie.Carter@yVlJu.org|2452517| +7430|AAAAAAAAGANBAAAA|1915844|904|26764|2450018|2449988|Miss|Laura|Jacobson|Y|30|12|1951|CANADA||Laura.Jacobson@T1mCx4hzXg7r.edu|2452409| +7431|AAAAAAAAHANBAAAA|1369406|2324|6442|2449913|2449883|Mr.|David|Morris|Y|8|9|1985|IRAQ||David.Morris@OcPjM.org|2452336| +7432|AAAAAAAAIANBAAAA|159965|6273|36628|2451392|2451362|Mr.|Justin||Y|11|7||NORWAY|||| +7433|AAAAAAAAJANBAAAA|884978|1000|20975|2449216|2449186|Dr.|Barbara|Quick|Y|16|8|1959|ICELAND||Barbara.Quick@UyIsTXDfpKQf2O4q.com|2452365| +7434|AAAAAAAAKANBAAAA|398664|5332|48800|2451923|2451893|Mrs.|Dawn|Cross|N|11|12|1989|PUERTO RICO||Dawn.Cross@0IsYxIei.edu|2452581| +7435|AAAAAAAALANBAAAA|962820|3209|9598|2451630|2451600|Mr.|John|Hurley|N|1|2|1976|SPAIN||John.Hurley@dISOeGMIxSbJ.org|2452395| +7436|AAAAAAAAMANBAAAA|341521|6276|25431|2449708|2449678|Ms.|Andrew|Rice|N|31|10|1930|NIUE||Andrew.Rice@HOJXt0My.org|2452507| +7437|AAAAAAAANANBAAAA|1472438|3435|9679|2450073|2450043|Mrs.|Beatrice|Marlow|N|31|8|1956|VANUATU||Beatrice.Marlow@q4vPfoLh.edu|2452527| +7438|AAAAAAAAOANBAAAA|486959|4021|43083|2450551|2450521|Miss|Melissa|Torres|Y|11|2|1970|NORFOLK ISLAND||Melissa.Torres@7T.org|2452475| +7439|AAAAAAAAPANBAAAA|125795|1590|29032|2450289|2450259|Miss|Helen|Bryant|Y|1|2|1948|TUVALU||Helen.Bryant@EpXS5Qx7B6V.org|2452462| +7440|AAAAAAAAABNBAAAA|120801|5156|36264|2452482|2452452|Dr.|Jason|Velez|N|1|7|1930|LITHUANIA||Jason.Velez@68qHh6iL1JZx.com|2452495| +7441|AAAAAAAABBNBAAAA|648753|3470|32987|2449566|2449536|Mr.|Matthew|Lynch|Y|21|6|1984|C�TE D'IVOIRE||Matthew.Lynch@YnsmBnxuekp8uxfx.com|2452536| +7442|AAAAAAAACBNBAAAA|678107|2579|3804|2452282|2452252|Mrs.|Adeline|Cornish|Y|14|9|1942|MAYOTTE||Adeline.Cornish@FdCZAP1ZMJ9Tg.org|2452451| +7443|AAAAAAAADBNBAAAA|738110|3593|28229|2449765|2449735|Dr.|Bonnie|Myers|N|18|12|1966|TUNISIA||Bonnie.Myers@ttuc0y.com|2452573| +7444|AAAAAAAAEBNBAAAA|349826|3027|9475|2452089|2452059|Mr.|John|Robertson|N|17|12|1956|MOLDOVA, REPUBLIC OF||John.Robertson@fTPG84tVlQtGKS.com|2452527| +7445|AAAAAAAAFBNBAAAA|181276|2467|47211|2452234|2452204|Mr.|Denis|Renfro|N|18|12|1964|GUYANA||Denis.Renfro@TDO22ZNx2ops52by.com|2452516| +7446|AAAAAAAAGBNBAAAA|428360|2060|18160|2449710|2449680|Miss|Alva|Rodriguez|N|19|2|1953|MAURITIUS||Alva.Rodriguez@f.edu|2452492| +7447|AAAAAAAAHBNBAAAA|456822|2683|10897|2451204|2451174|Miss|Wanda|Berry|N|23|3|1966|MAURITIUS||Wanda.Berry@hh7A9Y.org|2452530| +7448|AAAAAAAAIBNBAAAA|1114983|4731|26879|2449203|2449173|Miss|Marjorie|Gray|Y|9|11|1981|MAURITANIA||Marjorie.Gray@G0OgMu4suB1I0.com|2452505| +7449|AAAAAAAAJBNBAAAA|555794|2031|9868|2451903|2451873|Mrs.|Elizabeth|Livingston|N|16|8|1929|CANADA||Elizabeth.Livingston@k.com|2452574| +7450|AAAAAAAAKBNBAAAA|39484|2788|4492|2451881|2451851|Sir|Jay|Harris|Y|19|10|1942|GAMBIA||Jay.Harris@ov1TamH.com|2452444| +7451|AAAAAAAALBNBAAAA|745359|1767|42975|2451966|2451936|Mr.|William|Daniels|N|1|8|1987|BURUNDI||William.Daniels@MuHAdstt.org|2452444| +7452|AAAAAAAAMBNBAAAA|528584|1365|20946|2451223|2451193|Ms.|Sonia|Stacey|Y|10|11|1926|MALDIVES||Sonia.Stacey@uzNNhbUe7rtLLSQ8.edu|2452305| +7453|AAAAAAAANBNBAAAA|1390690|848|32653|2449330|2449300|Sir|Michael|Harrison|Y|29|12|1958|EGYPT||Michael.Harrison@6XVQfIfs7u.com|2452466| +7454|AAAAAAAAOBNBAAAA|122436|5715|34947|2450331|2450301|Mr.|Travis|Paulson|N|16|10|1938|RUSSIAN FEDERATION||Travis.Paulson@zP1BbPzVbRCMBaV5.edu|2452640| +7455|AAAAAAAAPBNBAAAA|118464|1326|36988|2452210|2452180|Mr.|Gerald|Riggs|N|15|3|1968|GREECE||Gerald.Riggs@oe9sUEO6.com|2452646| +7456|AAAAAAAAACNBAAAA|552927|5980|32233|2452157|2452127|Mr.|Danny|Johnson|Y|21|4|1932|HUNGARY||Danny.Johnson@aFBrAlHJc9Ei.org|2452529| +7457|AAAAAAAABCNBAAAA|536646|6553|21260|2451170|2451140|Miss|Ruth|Webb|Y|7|3|1927|NEW CALEDONIA||Ruth.Webb@r0PIJpjOy5.com|2452451| +7458|AAAAAAAACCNBAAAA|913448|4919|37|2450950|2450920|Mrs.|Elizabeth|Miller|Y|11|6|1946|LITHUANIA||Elizabeth.Miller@rTxuAhV2fCvK.edu|2452465| +7459|AAAAAAAADCNBAAAA|1783500|2312|35124|2449053|2449023|Mrs.|Julia|Whitfield|Y|26|1|1932|NORFOLK ISLAND||Julia.Whitfield@M2lX9d6rp.edu|2452531| +7460|AAAAAAAAECNBAAAA|1031576|2467|40205|2449392|2449362|Mrs.|Lita|Orourke|N|23|7|1939|PANAMA||Lita.Orourke@jf9ULNQ.edu|2452396| +7461|AAAAAAAAFCNBAAAA|1023260|2860|5613|2451158|2451128|Dr.|George|Woods|N|22|8|1982|AUSTRALIA||George.Woods@DCnm6.edu|2452318| +7462|AAAAAAAAGCNBAAAA|1156243|4645|3400|2452043|2452013|Mr.|Charles|Bussey|Y|26|2|1973|ARMENIA||Charles.Bussey@mDFY36QeqZbS2TraI5a.org|2452634| +7463|AAAAAAAAHCNBAAAA|294432|2087|2895|2452339|2452309|Mr.|Johnnie|Noyes|Y|5|11|1939|TUVALU||Johnnie.Noyes@kakxV7Ykn9.edu|2452437| +7464|AAAAAAAAICNBAAAA|696762||27299|2451183|2451153|Mrs.|||||||URUGUAY||Barbie.Jackson@nGP6C.com|| +7465|AAAAAAAAJCNBAAAA|564055|161|37020|2452273|2452243|Mrs.|Doretha|Wolford|N|30|12|1941|GUERNSEY||Doretha.Wolford@BCTfOs5lgfQ.org|2452372| +7466|AAAAAAAAKCNBAAAA|259220|3722|37024|2449212|2449182|Mrs.|Helen|Wilson|N|17|12|1957|ALAND ISLANDS||Helen.Wilson@msbB13v.org|2452481| +7467|AAAAAAAALCNBAAAA|768327|1133|2339|2451966|2451936|Dr.|Gregory|Mccall|Y|27|10|1938|GREENLAND||Gregory.Mccall@hMPuuk6c3II.org|2452523| +7468|AAAAAAAAMCNBAAAA|1180777|1698|26572|2449663|2449633|Sir|Ramon|Davidson|Y|29|8|1959|EQUATORIAL GUINEA||Ramon.Davidson@Xbn20lc.com|2452319| +7469|AAAAAAAANCNBAAAA|480102|6445|33541|2452422|2452392|Dr.|April|Evans|Y|29|4|1934|SUDAN||April.Evans@ruGPt9piNYBuRQt.edu|2452462| +7470|AAAAAAAAOCNBAAAA|1690491|321|33704|2451843|2451813|Miss|Teresa|Miller|N|1|6|1965|GUERNSEY||Teresa.Miller@6xBkJIPaL.org|2452642| +7471|AAAAAAAAPCNBAAAA|1809330|2286|3642|2450604|2450574|Dr.|John|Roberson|N|21|1|1980|ROMANIA||John.Roberson@5c6.org|2452458| +7472|AAAAAAAAADNBAAAA|623767|1942|8870|2450583|2450553|Miss|Norma|Nobles|N|20|10|1925|BURKINA FASO||Norma.Nobles@nGe.com|2452536| +7473|AAAAAAAABDNBAAAA|919332|2165|1845|2451637|2451607|Dr.|Jason|Brown|N|12|5|1927|PARAGUAY||Jason.Brown@ONSQKbL5.edu|2452309| +7474|AAAAAAAACDNBAAAA|1640621|861|27901|2449084|2449054|Dr.|Gary|Avila|Y|15|6|1990|LITHUANIA||Gary.Avila@6y1TaBQ3bDX0L.org|2452285| +7475|AAAAAAAADDNBAAAA|1469191|6270|4149|2451818|2451788|Dr.|Lucille|Hawley|Y|20|12|1947|SAUDI ARABIA||Lucille.Hawley@oDrae22bOTG9SsVcNb7U.com|2452450| +7476|AAAAAAAAEDNBAAAA|1689531|6757|8030|2452392|2452362|Mr.|Scott|Williams|Y|11|11|1970|VANUATU||Scott.Williams@KLumBD5sx0Hx.com|2452352| +7477|AAAAAAAAFDNBAAAA|880086|3286|44533|2451531|2451501|Ms.|Catherine|Green|Y|12|7|1970|AMERICAN SAMOA||Catherine.Green@dBhgRtlvLJ4.org|2452539| +7478|AAAAAAAAGDNBAAAA|1749965|4480|45395|2450502|2450472|Miss|Ruth|Ackerman|N|23|6|1941|KOREA, REPUBLIC OF||Ruth.Ackerman@Nsbpd5X4DIvF.edu|2452404| +7479|AAAAAAAAHDNBAAAA|1576968|7180|25792|2450776|2450746|Ms.|Rachael|Hardin|N|24|6|1949|TOGO||Rachael.Hardin@N1Faz4vA8Vf.com|2452313| +7480|AAAAAAAAIDNBAAAA|96301|2466|39031|2451845|2451815|Dr.|Roscoe|Theriault|Y|18|7|1950|SIERRA LEONE||Roscoe.Theriault@CZYtIK7LqKJsirqH.com|2452448| +7481|AAAAAAAAJDNBAAAA|1370448|2814|6380|2452086|2452056|Dr.|Nora|Smith|N|26|5|1942|SYRIAN ARAB REPUBLIC||Nora.Smith@xlNAi.com|2452411| +7482|AAAAAAAAKDNBAAAA|1155235|6373|32311|2449865|2449835|Mrs.|Dora|Johnson|N|30|11|1934|NIGER||Dora.Johnson@ue0c61Pkj.com|2452381| +7483|AAAAAAAALDNBAAAA|1764290|4323|35156|2449702|2449672|Ms.|Xiomara|Williamson|N|14|12|1926|MONTENEGRO||Xiomara.Williamson@zkiRuj1.edu|2452400| +7484|AAAAAAAAMDNBAAAA|1521846|5333|9667|2450690|2450660|Dr.|Basil|Phillips|Y|12|8|1984|AZERBAIJAN||Basil.Phillips@Ud.edu|2452306| +7485|AAAAAAAANDNBAAAA|1556229|2420|48211|2452434|2452404|Ms.|Kathleen|Herndon|N|15|2|1964|EGYPT||Kathleen.Herndon@LRxMGfibtkqZ4j.org|2452300| +7486|AAAAAAAAODNBAAAA|1242352|2344|9536|2451629|2451599|Dr.|Alice|Gomes|N|11|11|1945|FIJI||Alice.Gomes@xKsyhx.edu|2452354| +7487|AAAAAAAAPDNBAAAA|811457|663|18222|2452521|2452491|Dr.|Brenda|Kincaid|Y|15|11|1983|YEMEN||Brenda.Kincaid@CO50Qc5myyY.edu|2452444| +7488|AAAAAAAAAENBAAAA|1240202|410|618|2450349|2450319|Mrs.|Toni|Montemayor|N|12|6|1944|SURINAME||Toni.Montemayor@Dvj.org|2452600| +7489|AAAAAAAABENBAAAA|1199765|6536|12107|2450355|2450325|Dr.|Brady|Scarborough|Y|13|4|1957|UZBEKISTAN||Brady.Scarborough@eXefrXZ.edu|2452377| +7490|AAAAAAAACENBAAAA|1145281|6516|34151|2452049|2452019|Miss|Betty|Robinson|Y|17|3|1975|KOREA, REPUBLIC OF||Betty.Robinson@eHopCLoN1L.edu|2452334| +7491|AAAAAAAADENBAAAA|100497|7195|37046|2450586|2450556|Ms.|Deanna|Dick|Y|6|4|1982|BOLIVIA||Deanna.Dick@bQqH2lCuruKtCA.edu|2452464| +7492|AAAAAAAAEENBAAAA|651984|6819|10807|2449076|2449046|Miss|Susanne|Wilbur|N|5|4|1929|PANAMA||Susanne.Wilbur@ObQkDYr.com|2452395| +7493|AAAAAAAAFENBAAAA|826540|5178|32969|2450162|2450132|Miss|Barbara|Finney|Y|1|8|1969|YEMEN||Barbara.Finney@9SLKgZHdVZmZd.edu|2452515| +7494|AAAAAAAAGENBAAAA|544218|5631|24197|2452304|2452274|Mr.|John|Lowe|Y|19|1|1953|ISLE OF MAN||John.Lowe@tBI4LqVOPg.edu|2452586| +7495|AAAAAAAAHENBAAAA|343953|6635|47717|2449189|2449159|Dr.|Frank|Billiot|N|25|5|1938|MACAO||Frank.Billiot@rDlaSkd8PN.edu|2452537| +7496|AAAAAAAAIENBAAAA|934182|1210|49791|2452542|2452512|Dr.|Raymond|Inman|N|27|3|1977|SAMOA||Raymond.Inman@vlsq5To.org|2452646| +7497|AAAAAAAAJENBAAAA|337222|6527|7832|2449479|2449449|Dr.|Gregory|Bui|N|9|3|1942|POLAND||Gregory.Bui@GGmsRlLX.edu|2452535| +7498|AAAAAAAAKENBAAAA|1040275|492|34709|2452475|2452445|Mr.|Peter|Case|Y|11|4|1945|TOGO||Peter.Case@mEVMF.com|2452423| +7499|AAAAAAAALENBAAAA|334099|1458|10064|2449362|2449332|Sir|Lloyd|Pettis|Y|19|6|1981|UGANDA||Lloyd.Pettis@djTvFX99Pg.com|2452632| +7500|AAAAAAAAMENBAAAA|1600406|5720|24281|2452673|2452643|Dr.|Gary|Courtney|N|23|3|1958|JAMAICA||Gary.Courtney@d2F0jYnzB1t2RCl.edu|2452450| +7501|AAAAAAAANENBAAAA|1008434|4193|11999|2451716|2451686|Mrs.|Katharine|Locke|N|2|10|1980|MALAWI||Katharine.Locke@6AZdv0Edt4326PpmuQP.com|2452375| +7502|AAAAAAAAOENBAAAA|1622563|5405|45847|2451897|2451867|Mr.|David|Reyes|Y|1|7|1934|SINGAPORE||David.Reyes@qcq91HuGF8N.com|2452571| +7503|AAAAAAAAPENBAAAA|1430070|4088|38859|2451121|2451091|Miss|April|Mcintyre|N|28|12|1952|GUATEMALA||April.Mcintyre@P3BtDf.org|2452585| +7504|AAAAAAAAAFNBAAAA|8503|3825|45994|2450247|2450217|Sir|Dale|Harding|Y|18|7|1965|LUXEMBOURG||Dale.Harding@Xh5i.com|2452399| +7505|AAAAAAAABFNBAAAA|315734|4661|39588|2451702|2451672|Mr.|Bryant|Hyde|N|13|7|1931|NORFOLK ISLAND||Bryant.Hyde@9XZ.edu|2452328| +7506|AAAAAAAACFNBAAAA|||17186||||Wayne|||21|1||SWAZILAND|||2452552| +7507|AAAAAAAADFNBAAAA|1367795|1259|2323|2451552|2451522|Dr.|Alex|Gomez|N|21|4|1970|ZAMBIA||Alex.Gomez@NvYYxz0N.edu|2452617| +7508|AAAAAAAAEFNBAAAA|906677|4842|26207|2451167|2451137|Dr.|Linda|Williams|Y|24|10|1971|NEW ZEALAND||Linda.Williams@a4VkG8FU2jdIkQ.edu|2452370| +7509|AAAAAAAAFFNBAAAA|1833899|5863|28627|2451284|2451254|Dr.|Richard|Scott|N|8|9|1929|NIGERIA||Richard.Scott@YsHAE3y4oI.com|2452428| +7510|AAAAAAAAGFNBAAAA|416970|3528|10486|2452530|2452500|Dr.|Debra|Boyce|Y|15|3|1945|ANTARCTICA||Debra.Boyce@8VdNEIm003hDNkEFi.org|2452372| +7511|AAAAAAAAHFNBAAAA|880966|4966|48519|2450579|2450549|Dr.|Thomas|Harris|Y|31|7|1950|DJIBOUTI||Thomas.Harris@CTKE.org|2452467| +7512|AAAAAAAAIFNBAAAA|1710152|3030|28308|2451178|2451148|Mr.|Charles|Curtis|N|24|10|1970|DOMINICA||Charles.Curtis@yVCcQHll2Ujr.edu|2452599| +7513|AAAAAAAAJFNBAAAA|531497|2214|9331|2450543|2450513|Mrs.|Bonnie|Diaz|N|23|1|1968|LUXEMBOURG||Bonnie.Diaz@CQJaV4otvs.org|2452478| +7514|AAAAAAAAKFNBAAAA|745697|6191|47532|2450217|2450187|Dr.|Alberta|Maurer|Y|6|6|1936|NAURU||Alberta.Maurer@POEYvbRy97Y.edu|2452466| +7515|AAAAAAAALFNBAAAA|160670|4460|12697|2452535|2452505|Mr.|David|Boatright|Y|25|9|1942|TURKEY||David.Boatright@yVY.edu|2452406| +7516|AAAAAAAAMFNBAAAA|1459260|6228|15643|2449733|2449703|Mr.|Mario|Matthews|Y|22|9|1925|MAYOTTE||Mario.Matthews@v9RDa1jhApiYSyBXFV.org|2452405| +7517|AAAAAAAANFNBAAAA|572175|3719|13216|2451848|2451818|Sir|Quincy|Carlson|N|8|3|1949|ALGERIA||Quincy.Carlson@xfEmCXjvDNIO.edu|2452432| +7518|AAAAAAAAOFNBAAAA|1385095|2571|33600|2450085|2450055|Miss|Christi|Johnson|Y|28|1|1939|SEYCHELLES||Christi.Johnson@Jos0H3oIK3.org|2452595| +7519|AAAAAAAAPFNBAAAA|1456465|13|47961|2449901|2449871|Sir|Thomas|Hyman|Y|8|2|1943|CUBA||Thomas.Hyman@lTqNSVb.com|2452377| +7520|AAAAAAAAAGNBAAAA|510542|1767|6239|2449778|2449748|Sir|Thomas|Jordon|Y|29|5|1982|NETHERLANDS||Thomas.Jordon@lf4q5nUoMj.com|2452309| +7521|AAAAAAAABGNBAAAA|146668|1691|3114|2450284|2450254|Dr.|Terrell|Ritchie|N|27|11|1982|LIBERIA||Terrell.Ritchie@ysFpRtGH4.edu|2452477| +7522|AAAAAAAACGNBAAAA|1711273|1649|13272|2452085|2452055|Ms.|Katie|Farmer|N|12|12|1971|AFGHANISTAN||Katie.Farmer@XCGl49YqBpU.org|2452444| +7523|AAAAAAAADGNBAAAA|1868752|6604|34856|2450459|2450429|Ms.|Juana|Long|Y|17|1|1941|FINLAND||Juana.Long@fx8eDE6V0tZdZAkq97iS.edu|2452317| +7524|AAAAAAAAEGNBAAAA|1537576|3087|48951|2450285|2450255|Miss|Catherine|Nall|N|1|6|1942|DOMINICA||Catherine.Nall@G6U.com|2452582| +7525|AAAAAAAAFGNBAAAA|214630|5778|17726|2451757|2451727|Dr.|Seth|Mills|N|12|5|1926|BURKINA FASO||Seth.Mills@OrO2ytjE.com|2452394| +7526|AAAAAAAAGGNBAAAA|1243860|2036|3812|2451315|2451285|Mr.|John|Sherrod|Y|9|7|1948|GUAM||John.Sherrod@xZ6ELec.edu|2452623| +7527|AAAAAAAAHGNBAAAA|160145|1799|28253|2452239|2452209|Mr.|George|Taylor|Y|26|7|1959|CUBA||George.Taylor@KTUbGAh.org|2452424| +7528|AAAAAAAAIGNBAAAA|319874|4703|23579|2450633|2450603|Dr.|Denise|Clark|N|4|4|1952|GUATEMALA||Denise.Clark@hUdTj.edu|2452402| +7529|AAAAAAAAJGNBAAAA|1658165|258|1198|2449343|2449313|Mr.|Joseph|Martin|Y|23|11|1970|SYRIAN ARAB REPUBLIC||Joseph.Martin@DalIMkubq7NVQCPt.edu|2452436| +7530|AAAAAAAAKGNBAAAA|418028|5279|36530|2449802|2449772|Sir|Raymond|Todd|Y|29|5|1948|CYPRUS||Raymond.Todd@evrR9h.edu|2452332| +7531|AAAAAAAALGNBAAAA|1891034|5855|49429|2452670|2452640|Mrs.|Betty|Hines|N|20|1|1943|VENEZUELA||Betty.Hines@8P.com|2452526| +7532|AAAAAAAAMGNBAAAA|1050524|5376|14981|2450714|2450684|Dr.|John|Tracy|Y|23|5|1991|BELGIUM||John.Tracy@5R1ryiRztENZ.org|2452388| +7533|AAAAAAAANGNBAAAA|1596129|1052|42328|2449065|2449035|Sir|John|Howard|N|3|12|1953|LITHUANIA||John.Howard@R48S0NLzKh0O4EPxnIs.edu|2452422| +7534|AAAAAAAAOGNBAAAA||6217|6052|||Dr.||Barton||25|||MALTA|||| +7535|AAAAAAAAPGNBAAAA|9490|2826|36515|2452092|2452062|Dr.|Christine|Hunter|N|7|6|1945|PARAGUAY||Christine.Hunter@NP.org|2452608| +7536|AAAAAAAAAHNBAAAA|11681|1624|24367|2451707|2451677|Dr.|Clarence|Ward|Y|23|1|1939|NICARAGUA||Clarence.Ward@5ecUhbEzf6KoFG.org|2452353| +7537|AAAAAAAABHNBAAAA|1455709|2540|40971|2449843|2449813|Dr.|William|Walker|N|20|1|1979|SRI LANKA||William.Walker@h1jsazGV8tu.edu|2452466| +7538|AAAAAAAACHNBAAAA|313405|7018|44231|2452454|2452424|Miss|Janie|Barnes|N|23|3|1947|RUSSIAN FEDERATION||Janie.Barnes@bhny.org|2452319| +7539|AAAAAAAADHNBAAAA|436369|1144|2366|2451634|2451604|Mr.|Tony|Mckinney|Y|1|12|1931|MALAWI||Tony.Mckinney@9ups9pnoy4v.edu|2452339| +7540|AAAAAAAAEHNBAAAA|587120|1425|18530|2451994|2451964|Dr.|Felicitas|Vo|N|3|5|1935|SLOVENIA||Felicitas.Vo@0jXzCBHJix.org|2452394| +7541|AAAAAAAAFHNBAAAA|1463782|4535|35735|2449357|2449327|Mrs.|Beverly|Young|Y|31|5|1957|MALAYSIA||Beverly.Young@oYJCcMrr.com|2452312| +7542|AAAAAAAAGHNBAAAA|1842288|3019|38597|2449135|2449105|Mrs.|Lisa|Horn|N|6|11|1981|GAMBIA||Lisa.Horn@BnJpDr2v9A4.com|2452341| +7543|AAAAAAAAHHNBAAAA|1839415|2962|20909|2450866|2450836|Dr.|William|Lackey|N|4|2|1990|VANUATU||William.Lackey@oU8.org|2452307| +7544|AAAAAAAAIHNBAAAA|416100|6191|13482|2451788|2451758|Sir|Christopher|Carter|Y|20|2|1989|DENMARK||Christopher.Carter@8HoZ5CGDafFzK.org|2452604| +7545|AAAAAAAAJHNBAAAA|985547|2484|45421|2451868|2451838|Mr.|Phillip|Harris|N|16|1|1979|GERMANY||Phillip.Harris@nnT6343yj1yD8O.org|2452407| +7546|AAAAAAAAKHNBAAAA|1161943|5964|26785|2451874|2451844|Dr.|Carl|Wyatt|Y|12|9|1945|INDONESIA||Carl.Wyatt@ynRtMFQM63jZV2At.com|2452555| +7547|AAAAAAAALHNBAAAA|1049123|5840|4157|2450985|2450955|Miss|Lela|Dalton|Y|26|3|1936|BHUTAN||Lela.Dalton@tevyzxs36j5.edu|2452383| +7548|AAAAAAAAMHNBAAAA|647951|4890|43829|2452339|2452309|Mr.|Thomas|Bryant|Y|27|5|1932|GUINEA-BISSAU||Thomas.Bryant@pZYel501s1.edu|2452613| +7549|AAAAAAAANHNBAAAA|352633||29297|2452116|2452086||||||6||CANADA||Michael.Shipp@ILdj6u.com|2452473| +7550|AAAAAAAAOHNBAAAA|1286435|1772|14662|2452235|2452205|Dr.|Michelle|Deaton|N|10|4|1980|CYPRUS||Michelle.Deaton@i9uxZZLQOI5RiXXt9PS.com|2452458| +7551|AAAAAAAAPHNBAAAA|1252795|3398|20754|2450606|2450576|Mrs.|Julianne|Kaiser|N|21|9|1951|SWITZERLAND||Julianne.Kaiser@ddAnICrU.com|2452537| +7552|AAAAAAAAAINBAAAA|929997|3997|350|2450700|2450670|Mr.|Darrin|Cook|Y|9|7|1930|NORWAY||Darrin.Cook@Ef3b6Vc9MsH.edu|2452339| +7553|AAAAAAAABINBAAAA|1387932|1162|27592|2450216|2450186|Sir|Jeffrey|Hillman|N|27|9|1938|ALBANIA||Jeffrey.Hillman@2e.edu|2452314| +7554|AAAAAAAACINBAAAA|675252|1838|30424|2452074|2452044|Sir|Michael|Bruce|N|27|5|1948|ERITREA||Michael.Bruce@Qf7hLOYMklPY4v0u1M.org|2452556| +7555|AAAAAAAADINBAAAA|1580358|4531|30257|2451760|2451730|Mrs.|Teresa|Jones|Y|18|11|1942|GEORGIA||Teresa.Jones@iJHupa.edu|2452495| +7556|AAAAAAAAEINBAAAA||4224|31742|2451061||Mrs.|||N||||AFGHANISTAN||Flor.Vaughn@5VHyMVAPGyi2GGV.org|| +7557|AAAAAAAAFINBAAAA|1738759|2464|47322|2450382|2450352|Miss|Sharon|Gruber|N|8|8|1952|LATVIA||Sharon.Gruber@jTa9djxctsvFtcR.edu|2452541| +7558|AAAAAAAAGINBAAAA|290423|1042|47261|2452232|2452202|Dr.|Margaret|Cole|Y|5|9|1933|IRELAND||Margaret.Cole@BheY3y9p4qCy.edu|2452356| +7559|AAAAAAAAHINBAAAA||1609|25059|2450511|||||N||2|1985|VIET NAM||Penelope.Walker@9vBXH8GQ8FSz.edu|2452502| +7560|AAAAAAAAIINBAAAA|1868542|6825|16308|2450124|2450094|Miss|Arthur|Knapp|N|20|4|1952|CANADA||Arthur.Knapp@QEKMPS.org|2452553| +7561|AAAAAAAAJINBAAAA|1587093|3530|49614|2450487|2450457|Ms.|Kara|Desimone|Y|14|8|1988|BRAZIL||Kara.Desimone@ir82Q08OEu.com|2452625| +7562|AAAAAAAAKINBAAAA|901897|3305|5960|2451190|2451160|Miss|Jennifer|Mullin|Y|16|10|1982|FIJI||Jennifer.Mullin@Ynss.org|2452571| +7563|AAAAAAAALINBAAAA|74236|4999|21362|2450619|2450589|Mr.|John|Shipp|N|13|6|1957|SEYCHELLES||John.Shipp@mGnIQvLmFFoTK6ITmBa.edu|2452291| +7564|AAAAAAAAMINBAAAA|1551218|6303|41388|2449207|2449177|Sir|Howard|Patel|N|25|4|1962|SLOVENIA||Howard.Patel@ka0NMZo34Fy.com|2452431| +7565|AAAAAAAANINBAAAA|1631954|334|39114|2449761|2449731|Miss|Donna|Sampson|Y|21|12|1955|BOUVET ISLAND||Donna.Sampson@8.com|2452378| +7566|AAAAAAAAOINBAAAA|811509|2229|25396|2449273|2449243|Mrs.|Jeanne|Thompson|N|29|10|1960|ECUADOR||Jeanne.Thompson@0.edu|2452296| +7567|AAAAAAAAPINBAAAA|420935|1616|27850|2451781|2451751|Dr.|Elizabeth|Sturm|Y|22|6|1983|UNITED STATES||Elizabeth.Sturm@iqv4Pt.com|2452602| +7568|AAAAAAAAAJNBAAAA|1744828|367|44706|2450763|2450733|Sir|Dave|Mcneely|Y|14|1|1987|INDONESIA||Dave.Mcneely@Da7v7m.org|2452551| +7569|AAAAAAAABJNBAAAA|1605321|4562|28956|2452145|2452115|Dr.|Charles|Miller|N|30|11|1952|RWANDA||Charles.Miller@hYJ9EOsiSk.org|2452480| +7570|AAAAAAAACJNBAAAA|1345730|5914|19345|2452242|2452212|Sir|William|Stewart|Y|4|11|1991|AUSTRIA||William.Stewart@8EQGTuQOhR5.edu|2452421| +7571|AAAAAAAADJNBAAAA|912968|2562|19756|2451811|2451781|Ms.|Sue|Goins|Y|16|7|1981|PUERTO RICO||Sue.Goins@g.edu|2452373| +7572|AAAAAAAAEJNBAAAA|198993|3895|39597|2451675|2451645|Miss|Diana|Weiss|Y|15|2|1977|SUDAN||Diana.Weiss@QFtVnZkFpcDO.org|2452296| +7573|AAAAAAAAFJNBAAAA|322078|266|32901|2450735|2450705|Dr.|Sonia|Buchanan|N|20|6|1930|IRAQ||Sonia.Buchanan@g2m4.org|2452528| +7574|AAAAAAAAGJNBAAAA|1789466|1874|24682|2450157|2450127|Miss|Cathy|Ward|N|14|12|1940|MACAO||Cathy.Ward@22YrmVmZ1P.edu|2452511| +7575|AAAAAAAAHJNBAAAA|1589961|5404|11247|2450140|2450110|Mr.|Wally|Masters|Y|11|12|1950|ANTIGUA AND BARBUDA||Wally.Masters@yBNb8TIiChO.org|2452511| +7576|AAAAAAAAIJNBAAAA|359|7076|19393|2449630|2449600|Dr.|Jacqueline|Byrd|N|19|1|1933|FIJI||Jacqueline.Byrd@kP9FU8k2lTlO.edu|2452503| +7577|AAAAAAAAJJNBAAAA|1759704|5246|10788|2449607|2449577|Mr.|Pedro|Underhill|Y|9|9|1988|PARAGUAY||Pedro.Underhill@nLvN.com|2452538| +7578|AAAAAAAAKJNBAAAA|653124|5579|2434|2451675|2451645|Dr.|Andrew|Thatcher|N|29|11|1969|CYPRUS||Andrew.Thatcher@7Q.org|2452302| +7579|AAAAAAAALJNBAAAA|1133058|6025|49439|2449939|2449909|Dr.|Ryan|Barnes|N|27|1|1939|URUGUAY||Ryan.Barnes@vEYlNOEO2uqHdihg.edu|2452512| +7580|AAAAAAAAMJNBAAAA|475360|4690|12420|2450941|2450911|Mrs.|Julie|Garcia|N|17|7|1954|NETHERLANDS||Julie.Garcia@DnRbBrzv3.com|2452291| +7581|AAAAAAAANJNBAAAA|95745|3851|357|2451270|2451240|Sir|Joseph|Meza|N|28|1|1945|NIGER||Joseph.Meza@Fsee.edu|2452290| +7582|AAAAAAAAOJNBAAAA|1362611|5181|10951|2449369|2449339|Miss|Lori|Crenshaw|Y|9|7|1930|POLAND||Lori.Crenshaw@PQ1TJcq4zv1l.edu|2452303| +7583|AAAAAAAAPJNBAAAA|869521|7004|8534|2451059|2451029|Dr.|Peggy|Caruso|N|12|12|1983|WALLIS AND FUTUNA||Peggy.Caruso@88eThKpfqJ7P4.edu|2452632| +7584|AAAAAAAAAKNBAAAA|633908|3969|18877|||Dr.|Rebecca|Mullins|Y|11||1964|MONGOLIA|||| +7585|AAAAAAAABKNBAAAA|4782|2563|35497|2450715|2450685|Dr.|Bethany|Calhoun|N|28|4|1968|SAN MARINO||Bethany.Calhoun@x.edu|2452339| +7586|AAAAAAAACKNBAAAA|1320244|2177|5710|2450238|2450208|Sir|Laverne|Walters|Y|9|2|1982|DJIBOUTI||Laverne.Walters@aT1edMAEq5pl7s7z14.com|2452434| +7587|AAAAAAAADKNBAAAA|747242|1025|34855|2450902|2450872|Mr.|Stanley|Hughes|Y|5|4|1948|GUERNSEY||Stanley.Hughes@23An1zcSaui.org|2452430| +7588|AAAAAAAAEKNBAAAA||3562|1285||2450177|||||26|||WESTERN SAHARA|||2452620| +7589|AAAAAAAAFKNBAAAA|373920|3422|8124|2452233|2452203|Dr.|Jerry|Shannon|Y|1|7|1928|LATVIA||Jerry.Shannon@Q8OKGjO.org|2452602| +7590|AAAAAAAAGKNBAAAA|622941|725|4509|2451621|2451591|Miss|Kendra|Holder|N|13|6|1980|PERU||Kendra.Holder@j16BGid67xs.edu|2452396| +7591|AAAAAAAAHKNBAAAA|1735787|1160|27780|2450933|2450903|Mr.|Robert|Osborne|Y|6|4|1925|KOREA, REPUBLIC OF||Robert.Osborne@i868.com|2452371| +7592|AAAAAAAAIKNBAAAA|1806981|3105|26429|2451637|2451607|Mr.|Donald|Sullivan|N|5|2|1985|BOTSWANA||Donald.Sullivan@N.edu|2452610| +7593|AAAAAAAAJKNBAAAA|1543602|977|8705|2449980|2449950|Dr.|Rene|Burns|Y|15|7|1927|MALAYSIA||Rene.Burns@68evu1bvptitPhn.org|2452300| +7594|AAAAAAAAKKNBAAAA|1007327|616|31505|2452073|2452043|Ms.|Barbara|Stephens|N|24|6|1989|COMOROS||Barbara.Stephens@mTQgsq.edu|2452438| +7595|AAAAAAAALKNBAAAA|1791041|3408|49775|2451562|2451532|Mrs.|Wendy|Martin|N|28|5|1948|UNITED ARAB EMIRATES||Wendy.Martin@N6HjHrXcCD4qmGu.com|2452413| +7596|AAAAAAAAMKNBAAAA|171552|6492|32185|2451555|2451525|Dr.|John|Stacy|N|8|5|1949|GUINEA-BISSAU||John.Stacy@Dn4mdAtrPN.edu|2452306| +7597|AAAAAAAANKNBAAAA|1352549|6775|29001|2449198|2449168|Dr.|Angel|Hartman|Y|30|4|1971|BARBADOS||Angel.Hartman@m.com|2452648| +7598|AAAAAAAAOKNBAAAA|975921|6970|7619|2449155|2449125|Mr.|Clinton|Mccallister|N|9|7|1931|HONG KONG||Clinton.Mccallister@nga4KsxHggJ.org|2452406| +7599|AAAAAAAAPKNBAAAA|1620417|2619|16774|2452103|2452073|Mrs.|Edith|Batten|Y|18|2|1992|THAILAND||Edith.Batten@2.edu|2452564| +7600|AAAAAAAAALNBAAAA|986707|3151|32442|2449040|2449010|Sir|Sanford|Bernhardt|N|19|5|1935|SURINAME||Sanford.Bernhardt@pS2cDQI3dbTf.com|2452455| +7601|AAAAAAAABLNBAAAA|1101105|3660|22662|2452238|2452208|Dr.|Denis|Wilson|Y|21|6|1990|CHRISTMAS ISLAND||Denis.Wilson@EMxFDExX5RJUYk.edu|2452559| +7602|AAAAAAAACLNBAAAA|130322|920|16718|2451777|2451747|Dr.|Azucena|Riddick|Y|9|6|1969|FINLAND||Azucena.Riddick@B3.com|2452347| +7603|AAAAAAAADLNBAAAA|1254790|7112|31834|2450226|2450196|Dr.|Janet|Mattson|N|9|8|1925|GUAM||Janet.Mattson@1PcrFHRM0xiYNHRVbYY.edu|2452424| +7604|AAAAAAAAELNBAAAA|824037|4103|20122|2449519|2449489|Dr.|Richard|Johnson|N|12|8|1973|VANUATU||Richard.Johnson@OA.org|2452553| +7605|AAAAAAAAFLNBAAAA|1034256|1378|12100|2452071|2452041|Mr.|John|Gonzales|Y|11|3|1948|SWEDEN||John.Gonzales@abJRM4Xz.com|2452466| +7606|AAAAAAAAGLNBAAAA|1010210|2923|10483|2450727|2450697|Dr.|Robert|Acosta|Y|14|12|1987|FAROE ISLANDS||Robert.Acosta@XPkoipvo3Qpv.com|2452327| +7607|AAAAAAAAHLNBAAAA|411567|4156|18698|2451455|2451425|Mr.|Ray|Webster|N|13|1|1972|IRAQ||Ray.Webster@2Tl2H5HP7m.edu|2452623| +7608|AAAAAAAAILNBAAAA|1292046|915|1431|2450610|||Tony|Nettles|N||9||||Tony.Nettles@Jv4XIsQKF.edu|| +7609|AAAAAAAAJLNBAAAA|1817142|3848|33801|2452612|2452582|Dr.|Letha|Brenner|Y|13|11|1957|VIET NAM||Letha.Brenner@dypR2Npo.com|2452561| +7610|AAAAAAAAKLNBAAAA|232273|654|44235|2449984|2449954|Dr.|Terry|Carey|Y|16|2|1952|SUDAN||Terry.Carey@o.org|2452608| +7611|AAAAAAAALLNBAAAA|1152353|5983|8860|2449818|2449788|Dr.|Susan|Pineda|Y|27|7|1942|FRENCH POLYNESIA||Susan.Pineda@EE0a0YUF0GtM.com|2452428| +7612|AAAAAAAAMLNBAAAA|1014556|427|19498|2449613|2449583|Dr.|Bruce|Ramirez|N|20|4|1966|HONDURAS||Bruce.Ramirez@c5fpt0Une.edu|2452358| +7613|AAAAAAAANLNBAAAA|1386955|5028|6701|2452380|2452350|Dr.|Milissa|Alaniz|N|18|12|1985|VENEZUELA||Milissa.Alaniz@dzMatZhCik.com|2452620| +7614|AAAAAAAAOLNBAAAA|1071707|6248|18789|2452131|2452101|Dr.|Clifford|Cook|Y|3|11|1953|MONTSERRAT||Clifford.Cook@m73OJvn.edu|2452285| +7615|AAAAAAAAPLNBAAAA|383224|2591|25605|2449782|2449752|Mr.|Mark|Hodgson|N|3|10|1989|ANGOLA||Mark.Hodgson@cz.org|2452446| +7616|AAAAAAAAAMNBAAAA|1232547|3106|43665|2451899|2451869|Miss|Doris|Mason|N|20|7|1931|SRI LANKA||Doris.Mason@8.edu|2452430| +7617|AAAAAAAABMNBAAAA|1332413|6294|4175|2452099|2452069|Ms.|Susan|Marquez|N|12|3|1937|GUADELOUPE||Susan.Marquez@k7pEM3tFmeypGMrf.com|2452452| +7618|AAAAAAAACMNBAAAA|1166531|1486|34488|2451371|2451341|Mr.|Donald|Ashley|N|21|3|1970|TOGO||Donald.Ashley@0YCR1rAaaody.com|2452417| +7619|AAAAAAAADMNBAAAA|1789446|7106|23566|2449242|2449212|Dr.|Mark|Stephens|N|20|5|1940|VIRGIN ISLANDS, U.S.||Mark.Stephens@Q.org|2452573| +7620|AAAAAAAAEMNBAAAA|886817|6948|2726|2452179|2452149|Sir|Benjamin|Ward|N|29|6|1948|BERMUDA||Benjamin.Ward@D5SJDbIU6n667L5ig.org|2452580| +7621|AAAAAAAAFMNBAAAA|963519|4358|42350|2452130|2452100|Sir|William|Otero|Y|5|10|1948|EGYPT||William.Otero@7y8nRBMxqBg.org|2452319| +7622|AAAAAAAAGMNBAAAA|1688404|1737|39117|2451186|2451156|Dr.|Porfirio|Franklin|Y|16|9|1973|CAMEROON||Porfirio.Franklin@t6Ns82iQcnpEiXuUfO.edu|2452448| +7623|AAAAAAAAHMNBAAAA|435167|4592|1264|2451223|2451193|Dr.|Louis|Spears|Y|29|7|1951|LITHUANIA||Louis.Spears@x.com|2452591| +7624|AAAAAAAAIMNBAAAA|1185389|7099|33609|2449124|2449094|Sir|Daniel|Moore|Y|22|12|1952|CAMEROON||Daniel.Moore@5BR4SpKxv.com|2452594| +7625|AAAAAAAAJMNBAAAA|1095341|2688|1635|2452606|2452576|Dr.|Xiomara|Damron|Y|3|11|1978|CAMBODIA||Xiomara.Damron@LIjc5Aoq.edu|2452456| +7626|AAAAAAAAKMNBAAAA|892637|319|22801|2451770|2451740|Mrs.|Jessica|Ortiz|Y|17|7|1939|TOGO||Jessica.Ortiz@nKRkczD4nCp.org|2452612| +7627|AAAAAAAALMNBAAAA|766893|5329|24565|2452112|2452082|Dr.|Ruth|Conner|N|8|12|1958|RWANDA||Ruth.Conner@68UVCrSGxEE.com|2452390| +7628|AAAAAAAAMMNBAAAA||1390|26947|2451576|2451546|Mr.|||N||12|1932|UNITED KINGDOM||Brian.Pham@01d8IQeuEG.org|2452462| +7629|AAAAAAAANMNBAAAA|1678633|3445|39457|2452394|2452364|Ms.|Charlotte|Jackson|N|6|10|1934|GREENLAND||Charlotte.Jackson@lAjQdxVtNVpQKBG.com|2452394| +7630|AAAAAAAAOMNBAAAA|413035|533|43678|2451236|2451206|Ms.|Dorothy|Denton|Y|18|9|1981|GIBRALTAR||Dorothy.Denton@THQxLo55X.org|2452599| +7631|AAAAAAAAPMNBAAAA|1089125|6373|28868|2449636|2449606|Mr.|Charles|Connors|N|3|2|1948|UZBEKISTAN||Charles.Connors@HizUQUvtI.org|2452452| +7632|AAAAAAAAANNBAAAA|1746444|4848|24805|2451527|2451497|Mrs.|Jeanne|Hartmann|Y|19|3|1981|GAMBIA||Jeanne.Hartmann@C5lR5A5TegnZC.com|2452430| +7633|AAAAAAAABNNBAAAA|874386|1633|38480|2449426|2449396|Mrs.|Cynthia|Stevens|Y|12|3|1947|FAROE ISLANDS||Cynthia.Stevens@8Xzpf2L7kjmb4.edu|2452381| +7634|AAAAAAAACNNBAAAA|136999|740|44081|2449408|2449378|Mr.|Terence|Dew|N|20|8|1955|SOMALIA||Terence.Dew@4a0x.edu|2452322| +7635|AAAAAAAADNNBAAAA|883550|6810|35317|2451763|2451733|Dr.|Heather|Gregory|N|30|1|1930|URUGUAY||Heather.Gregory@fRA0MBXQic7OZHP.com|2452502| +7636|AAAAAAAAENNBAAAA|832241|860|13079|2450634|2450604|Mr.|Floyd|Frey|N|9|8|1937|ARGENTINA||Floyd.Frey@q9oM.org|2452347| +7637|AAAAAAAAFNNBAAAA|1137270|4901|27779|2450193|2450163|Mrs.|Verna|Ponce|N|19|7|1984|LESOTHO||Verna.Ponce@iMbImChtFo.org|2452301| +7638|AAAAAAAAGNNBAAAA|1296026|1036|20893|2449238|2449208|Miss|Dianne|Isaacs|Y|20|12|1962|HONG KONG||Dianne.Isaacs@SPayDpqhmi.org|2452487| +7639|AAAAAAAAHNNBAAAA|1857029|962|17295|2451945|2451915|Sir|Richard|Young|N|2|1|1991|AUSTRALIA||Richard.Young@UZ5Nal7tTI70.edu|2452507| +7640|AAAAAAAAINNBAAAA|1515818|4453|19488|2449260|2449230|Sir|Otto|Mcdaniel|N|28|2|1963|MACAO||Otto.Mcdaniel@uSF.com|2452647| +7641|AAAAAAAAJNNBAAAA|528894|1951|46161|2451454|2451424|Dr.|Jon|Nelson|Y|23|6|1974|NIGER||Jon.Nelson@E56iTYfSf5.com|2452371| +7642|AAAAAAAAKNNBAAAA|1142262|7050|15085|2452136|2452106|Sir|John|Melton|Y|5|6|1986|KIRIBATI||John.Melton@GQIllYdK.edu|2452520| +7643|AAAAAAAALNNBAAAA|800057||46862|2450888|2450858|Ms.||Barrett||8|11|||||| +7644|AAAAAAAAMNNBAAAA|561244|6892|44987|2450031|2450001|Sir|James|Maldonado|N|11|7|1956|CAYMAN ISLANDS||James.Maldonado@Fa.edu|2452346| +7645|AAAAAAAANNNBAAAA|103678|2872|15266|2451817|2451787|Sir|Leroy|Davis|Y|14|4|1982|GABON||Leroy.Davis@Mm7Ci1v3FpdO.org|2452607| +7646|AAAAAAAAONNBAAAA|33905|405|47752|2450720|2450690|Ms.|Lillian|Cobb|Y|3|11|1939|MAYOTTE||Lillian.Cobb@dEmDRr1Rk9.org|2452453| +7647|AAAAAAAAPNNBAAAA|46417|5148|37079|2449779|2449749|Sir|James|Rodriguez|Y|8|10|1963|LITHUANIA||James.Rodriguez@FrqN3aLZ.edu|2452533| +7648|AAAAAAAAAONBAAAA|884234|3171|47755|2451649|2451619|Miss|Laurie|Thrasher|N|7|2|1972|WALLIS AND FUTUNA||Laurie.Thrasher@84Z8Kai1K.org|2452586| +7649|AAAAAAAABONBAAAA|176166|5471|41176|2449494|2449464|Sir|David|Peterson|N|14|2|1951|TONGA||David.Peterson@Uqa.edu|2452376| +7650|AAAAAAAACONBAAAA|506841|5165|41136|2450427|2450397|Mrs.|Helen|Sampson|Y|13|7|1964|TONGA||Helen.Sampson@4Vsnot8n.org|2452515| +7651|AAAAAAAADONBAAAA|783822|6009|48643|2449407|2449377|Ms.|Nancy|Clay|Y|6|10|1970|TOGO||Nancy.Clay@X8jLa.edu|2452590| +7652|AAAAAAAAEONBAAAA|171975|27|5279|2451439|2451409|Dr.|Faye|Robinson|N|10|9|1932|DOMINICA||Faye.Robinson@1NNK06E69S.org|2452340| +7653|AAAAAAAAFONBAAAA|574243|6713|19183|2450265|2450235|Dr.|William|Shook|Y|2|4|1943|BOLIVIA||William.Shook@iEqRI9YB.com|2452378| +7654|AAAAAAAAGONBAAAA|1106288|7034|44180|2449970|2449940|Ms.|Meagan|Massie|N|21|12|1978|TURKMENISTAN||Meagan.Massie@M8tU0QsvD0n.org|2452380| +7655|AAAAAAAAHONBAAAA|169241|4216|40344|2450350|2450320|Miss|Alicia|Young|Y|14|6|1940|VIET NAM||Alicia.Young@741PYp8E1cz7Xm.com|2452639| +7656|AAAAAAAAIONBAAAA|834906|163|26667|2449243|2449213|Mrs.|Tamra|Chapin|N|23|8|1945|BHUTAN||Tamra.Chapin@KIkLsqemQMI.edu|2452628| +7657|AAAAAAAAJONBAAAA|897047|686|49078|2450742|2450712|Sir|Clarence|Hill|Y|28|11|1947|MOZAMBIQUE||Clarence.Hill@Rre16OsMcqbVjifL.org|2452287| +7658|AAAAAAAAKONBAAAA|308353|5946|12739|2450570|2450540|Dr.|Carmelo|Matthews|Y|11|1|1990|PHILIPPINES||Carmelo.Matthews@LeF.edu|2452476| +7659|AAAAAAAALONBAAAA|1645524|1085|20246|2452225|2452195|Miss|Angel|Dougherty|N|8|4|1965|IRAQ||Angel.Dougherty@BIiD8UvR2YQL8.org|2452346| +7660|AAAAAAAAMONBAAAA|628058|4909|47313|2451324|2451294|Sir|Richard|Martinez|N|20|10|1981|TIMOR-LESTE||Richard.Martinez@u6oqNTO1qZ4bco.com|2452343| +7661|AAAAAAAANONBAAAA|1202960|3112|10417|2451292|2451262|Miss|Carolyn|Huffman|N|18|8|1956|CUBA||Carolyn.Huffman@2b1bHMmjY9SH0RQKgd.com|2452464| +7662|AAAAAAAAOONBAAAA|1525972|6986|42000|2451687|2451657|Mr.|Christopher|Page|Y|11|8|1929|GERMANY||Christopher.Page@5nrEcYfCI0IysNHZs.com|2452451| +7663|AAAAAAAAPONBAAAA|1705601|143|24332|2450940|2450910|Mr.|Michael|Leonard|Y|24|12|1962|NEPAL||Michael.Leonard@aLK8O5oayg.org|2452617| +7664|AAAAAAAAAPNBAAAA|412468|3714|23713|2452231|2452201|Miss|Debora|Bennett|Y|19|11|1948|MOROCCO||Debora.Bennett@6rB.org|2452388| +7665|AAAAAAAABPNBAAAA|1690960|6459|7558|2450504|2450474|Mr.|Francisco|Estrada|N|4|12|1948|NEW ZEALAND||Francisco.Estrada@T2rRcTJOAkjlgXypq.edu|2452289| +7666|AAAAAAAACPNBAAAA|||29543||2449298|||Weber||||1978|WESTERN SAHARA|||| +7667|AAAAAAAADPNBAAAA|21184|13|31642|2450010|2449980|Sir|Rodney|Groce|Y|6|12|1964|IRELAND||Rodney.Groce@E.org|2452598| +7668|AAAAAAAAEPNBAAAA|1194635|6004|20573|2450365|2450335|Dr.|Mark|Sullivan|Y|4|9|1946|ECUADOR||Mark.Sullivan@itHiH7BDHDx1a.org|2452491| +7669|AAAAAAAAFPNBAAAA|1049852|390|39328|2452110|2452080|Mr.|Harry|Long|N|23|2|1927|LIBERIA||Harry.Long@r.com|2452411| +7670|AAAAAAAAGPNBAAAA|72894|6837|32333|2450919|2450889|Mrs.|Sibyl|Reynolds|N|4|10|1935|EQUATORIAL GUINEA||Sibyl.Reynolds@PkdBGJ1IDgJb1lV.edu|2452307| +7671|AAAAAAAAHPNBAAAA|1322452|3404|38003|2451581|2451551|Mr.|James|Leon|N|13|11|1982|GABON||James.Leon@1ausIfF.edu|2452307| +7672|AAAAAAAAIPNBAAAA|86407|1100|29840|2451814|2451784|Sir|Bryant|Watts|N|26|10|1964|ZAMBIA||Bryant.Watts@NJU5dirfRgMCO.edu|2452439| +7673|AAAAAAAAJPNBAAAA|691973|7103|7033|2451951|2451921|Mr.|William|Calabrese|N|1|1|1991|ANTARCTICA||William.Calabrese@pP.edu|2452441| +7674|AAAAAAAAKPNBAAAA|960068|3228|43635|2452353|2452323|Mrs.|Pamela|Mendoza|Y|5|2|1951|BULGARIA||Pamela.Mendoza@2ppIgB3U1gDE84hYxdo.edu|2452328| +7675|AAAAAAAALPNBAAAA|518204|6214|35777|2450844|2450814|Sir|Brett|Meeks|N|8|1|1939|NIUE||Brett.Meeks@yG4QJUFL6ohBzy.edu|2452381| +7676|AAAAAAAAMPNBAAAA|486948|2893|15638|2452415|2452385|Dr.|Phyllis|Nelson|N|28|4|1932|KYRGYZSTAN||Phyllis.Nelson@UyV22pZXyA.com|2452566| +7677|AAAAAAAANPNBAAAA|1162754|481|11408|2451803|2451773|Dr.|Donald|Lentz|Y|15|3|1936|SOLOMON ISLANDS||Donald.Lentz@pVlaEZP.com|2452345| +7678|AAAAAAAAOPNBAAAA|1501669|4510|25351|2451940|2451910|Dr.|Benjamin|Boisvert|N|15|12|1992|PORTUGAL||Benjamin.Boisvert@nucM2cLGH6mz5S6.com|2452511| +7679|AAAAAAAAPPNBAAAA|1344032|3514|38596|2452597|2452567|Dr.|Camille|Workman|N|21|11|1976|MALTA||Camille.Workman@6Jnm8S0mPG.edu|2452564| +7680|AAAAAAAAAAOBAAAA|569531|1209|28487|2452453|2452423|Sir|Edwin|Christie|N|5|10|1961|LIECHTENSTEIN||Edwin.Christie@Zmd.org|2452321| +7681|AAAAAAAABAOBAAAA|1666181|5731|38434|2450223|2450193|Miss|Sharron|Vanburen|N|20|11|1967|YEMEN||Sharron.Vanburen@O6Ai.org|2452430| +7682|AAAAAAAACAOBAAAA|1651219|3502|16185|2449925|2449895|Dr.|Rosa|Bland|Y|18|4|1943|SAINT HELENA||Rosa.Bland@7cX4b7ciaScBFFk.com|2452304| +7683|AAAAAAAADAOBAAAA|382293|3157|16021|2451434|2451404|Mr.|Donald|Griffin|Y|24|12|1959|VENEZUELA||Donald.Griffin@rb.com|2452551| +7684|AAAAAAAAEAOBAAAA|1355291|1003|38715|2450033|2450003|Mr.|George|Windsor|N|8|9|1945|GHANA||George.Windsor@IMaERYckzdQJtPPGh.edu|2452394| +7685|AAAAAAAAFAOBAAAA|1212186|6473|42360|2449335|2449305|Dr.|Justin|Schmidt|N|8|8|1977|BANGLADESH||Justin.Schmidt@GNZEiQ0Z.edu|2452385| +7686|AAAAAAAAGAOBAAAA|1376968|2621|23074|2450757|2450727|Dr.|Sabina|Childers|N|26|12|1937|VIET NAM||Sabina.Childers@qn4jGcg7c.edu|2452597| +7687|AAAAAAAAHAOBAAAA|1121428|1500|43125|2452596|2452566|Dr.|Frank|Evans|N|24|1|1935|BERMUDA||Frank.Evans@FynmdQjni.edu|2452299| +7688|AAAAAAAAIAOBAAAA|445335|6020|15152|2450402|2450372|Dr.|Marshall|Yates|N|15|4|1968|IRELAND||Marshall.Yates@nBvduRlCudPbQY.edu|2452488| +7689|AAAAAAAAJAOBAAAA|44275|2300|15078|2451865|2451835|Ms.|Paula|Hughes|N|31|7|1948|NETHERLANDS ANTILLES||Paula.Hughes@SgAzbL3v6.com|2452431| +7690|AAAAAAAAKAOBAAAA|101212|1902|18945|2452481|2452451|Mr.|Dallas|Boyd|N|3|11|1989|ARUBA||Dallas.Boyd@LJI7zdumqB.edu|2452559| +7691|AAAAAAAALAOBAAAA|509516|5038|36207|2451165|2451135|Ms.|Kelli|Williams|N|8|11|1986|FAROE ISLANDS||Kelli.Williams@L.org|2452616| +7692|AAAAAAAAMAOBAAAA|328232|764|37296|2450661|2450631|Sir|Delbert|Oquendo|Y|5|1|1984|CANADA||Delbert.Oquendo@H.edu|2452485| +7693|AAAAAAAANAOBAAAA|1021182|2185|1188|2451853|2451823|Mr.|David|Jeffries|Y|29|7|1992|AMERICAN SAMOA||David.Jeffries@iPByofCu61XSPn1OJ.org|2452558| +7694|AAAAAAAAOAOBAAAA|1092501|1015|9129|2451232|2451202|Mr.|William|Lee|N|23|3|1926|MEXICO||William.Lee@SOFzjtt2J3EbE.com|2452352| +7695|AAAAAAAAPAOBAAAA|243588|6197|3959|2451085|2451055|Ms.|Marietta|Fletcher|Y|14|5|1989|CUBA||Marietta.Fletcher@8180jm20uBom0Gv4i.com|2452349| +7696|AAAAAAAAABOBAAAA||3241|36549|2449791|2449761|Dr.|Susanne|||22|8|||||2452463| +7697|AAAAAAAABBOBAAAA|415848|3565|9635|2449872|2449842|Ms.|Stella|Owens|N|16|7|1976|BULGARIA||Stella.Owens@Cp096N0.com|2452426| +7698|AAAAAAAACBOBAAAA|695085|5750|17615|2449425|2449395|Sir|Douglas|Robertson|N|24|11|1937|HAITI||Douglas.Robertson@9815MMcaFa51KG.edu|2452420| +7699|AAAAAAAADBOBAAAA|220116|1001|16286|2449654|2449624|Miss|Adrian|Fenner|N|6|12|1929|ARGENTINA||Adrian.Fenner@MZ.edu|2452348| +7700|AAAAAAAAEBOBAAAA|743836|4319|27249|2450312|2450282|Miss|Kristina|Bailey|N|3|7|1969|PAPUA NEW GUINEA||Kristina.Bailey@fVKgv1FkXyELPz4q.edu|2452589| +7701|AAAAAAAAFBOBAAAA|908656|5125|33194|2452617|2452587|Dr.|Richard|Anderson|N|1|11|1946|MONGOLIA||Richard.Anderson@c2F7PnL67Yx3.org|2452422| +7702|AAAAAAAAGBOBAAAA|1063648|3482|31716|2451643|2451613|Dr.|Marie|Anderson|N|13|5|1970|ALBANIA||Marie.Anderson@9pGd0jK71.edu|2452528| +7703|AAAAAAAAHBOBAAAA|1294889|6339|34589|2451239|2451209|Mr.|Merrill|Lopez|Y|8|9|1951|IRELAND||Merrill.Lopez@FMXELMX.com|2452401| +7704|AAAAAAAAIBOBAAAA|307395|4572|24102|2450041|2450011|Dr.|David|Delacruz|N|10|10|1986|FRENCH GUIANA||David.Delacruz@86Mh1CvMF1qLBp.org|2452478| +7705|AAAAAAAAJBOBAAAA|1625691|1873|29255|2451758|2451728|Mrs.|Josephine|Arellano|Y|8|12|1954|CANADA||Josephine.Arellano@uJzkIGR6.edu|2452476| +7706|AAAAAAAAKBOBAAAA|826264|2855|36646|2452590|2452560|Ms.|Brenda|Kimball|Y|3|3|1956|SWITZERLAND||Brenda.Kimball@EtQR1KC3.com|2452315| +7707|AAAAAAAALBOBAAAA|1638422|3687|18952|2452531|2452501|Sir|Michael|Ward|Y|8|7|1943|COMOROS||Michael.Ward@uihl1J15Y.org|2452344| +7708|AAAAAAAAMBOBAAAA|59041|4303|43118|2449667|2449637|Sir|Jeff|Hall|Y|24|11|1983|EGYPT||Jeff.Hall@16d44BS5sFOe.com|2452297| +7709|AAAAAAAANBOBAAAA|||32516|||||Banks|Y||12|1965|BURUNDI||Linda.Banks@Leibv0IqtL62xV8d.com|2452355| +7710|AAAAAAAAOBOBAAAA|205640|6057|17321|2452572|2452542|Dr.|James|Mathews|Y|3|9|1975|SOLOMON ISLANDS||James.Mathews@RZtFFJQDSUFH.com|2452562| +7711|AAAAAAAAPBOBAAAA|112902|2617|35579|2451065|2451035|Dr.|Robert|Simmons|N|11|3|1946|GUINEA||Robert.Simmons@K8i6ZsrTua.edu|2452425| +7712|AAAAAAAAACOBAAAA|842111|3868|39578|2450964|2450934|Dr.|Joseph|Hite|N|13|4|1945|NETHERLANDS ANTILLES||Joseph.Hite@XE5rrkKdHYHh.org|2452643| +7713|AAAAAAAABCOBAAAA|847792|414|19083|2451563|2451533|Sir|David|Neal|N|16|4|1929|ISRAEL||David.Neal@YEiJ78yK8mqvl.org|2452561| +7714|AAAAAAAACCOBAAAA|1375816|1666|21916|2452040|2452010|Miss|Dorothy|Martinez|N|23|3|1962|NEPAL||Dorothy.Martinez@O07acDo5eoxR.com|2452569| +7715|AAAAAAAADCOBAAAA|1303691|6399|11460|2450705|2450675|Sir|William|Lindsey|N|27|6|1933|ARGENTINA||William.Lindsey@F.edu|2452443| +7716|AAAAAAAAECOBAAAA|1403010|511|41612|2450808|2450778|Mrs.|Tiffany|Davenport|Y|5|3|1974|MONGOLIA||Tiffany.Davenport@keo0amhaL.org|2452400| +7717|AAAAAAAAFCOBAAAA|1608864|4013|42315|2451315|2451285|Mr.|Pedro|Cuevas|Y|17|1|1938|MAURITANIA||Pedro.Cuevas@cpTNoMAzfPfIuPK.edu|2452407| +7718|AAAAAAAAGCOBAAAA|1235806|2129|13299|2451794|2451764|Mr.|Freddie|Clarke|Y|14|4|1925|GUINEA-BISSAU||Freddie.Clarke@ff3qFsMalA.com|2452307| +7719|AAAAAAAAHCOBAAAA|785398|2714|2477|2450198|2450168|Mr.|William|Estep|Y|28|2|1975|LITHUANIA||William.Estep@U5phSV.com|2452547| +7720|AAAAAAAAICOBAAAA|630849|3000|35104|2450798|2450768|Sir|Bruce|Amundson|Y|21|8|1936|BAHRAIN||Bruce.Amundson@GeZzTLrU.edu|2452611| +7721|AAAAAAAAJCOBAAAA|968707|4639|17724|2452537|2452507|Dr.|Constance|Simmons|N|10|8|1977|FRENCH POLYNESIA||Constance.Simmons@4xL9nVkbNekbD3i3v.org|2452646| +7722|AAAAAAAAKCOBAAAA|610487|2612|14737|2449287|2449257|Dr.|Dolores|Stevens|Y|12|7|1924|ALGERIA||Dolores.Stevens@7P4sLQQShdgO48nyBm.org|2452297| +7723|AAAAAAAALCOBAAAA|495072|1619|7319|2451877|2451847|Dr.|Janet|Larson|Y|11|9|1966|UNITED STATES||Janet.Larson@bfMA.com|2452541| +7724|AAAAAAAAMCOBAAAA|1809883|6093|23150|2449917|2449887|Dr.|Micheal|Chamberlin|Y|4|5|1949|GREENLAND||Micheal.Chamberlin@hKLIHmz.com|2452559| +7725|AAAAAAAANCOBAAAA|793367|6414|23014|2449477|2449447|Sir|Ronald|Sloan|N|18|11|1952|NAURU||Ronald.Sloan@oy5m77p.edu|2452510| +7726|AAAAAAAAOCOBAAAA|1031874|3982|21290|2449333|2449303|Dr.|Ilene|Weeks|N|25|1|1943|PAKISTAN||Ilene.Weeks@zjTcYdF6.org|2452531| +7727|AAAAAAAAPCOBAAAA|1801653|1800|31547|2452027|2451997|Miss|Nancy|Delaney|N|26|10|1930|SLOVENIA||Nancy.Delaney@nP.com|2452547| +7728|AAAAAAAAADOBAAAA|975437|4848|21412|2450791|2450761|Dr.|Daniel|Duarte|N|21|5|1945|SURINAME||Daniel.Duarte@RyDPCIsjau6.com|2452433| +7729|AAAAAAAABDOBAAAA|1367175|373|7618|2450515|2450485|Dr.|Wilfred|Wood|Y|15|10|1982|HUNGARY||Wilfred.Wood@62gAYygQR3.org|2452492| +7730|AAAAAAAACDOBAAAA|1379342|2111|8480|2449382|2449352|Mr.|Richard|Martin|N|17|9|1990|SAINT HELENA||Richard.Martin@TOdt.com|2452338| +7731|AAAAAAAADDOBAAAA|693469|5809|11349|2449508|2449478|Mrs.|Kathryn|Smart|N|29|1|1957|TIMOR-LESTE||Kathryn.Smart@8zDoukVVovth.com|2452560| +7732|AAAAAAAAEDOBAAAA|818502|3847|11639|2451846|2451816|Dr.|Margaret|Cotter|N|8|8|1927|LIBERIA||Margaret.Cotter@Kf4L8s2Hir5h39f6.org|2452593| +7733|AAAAAAAAFDOBAAAA|1132444|1750|23809|2450352|2450322|Dr.|Pearl|Winters|N|30|6|1982|GEORGIA||Pearl.Winters@igpEnzZ.edu|2452621| +7734|AAAAAAAAGDOBAAAA|835942|5779|18463|2452612|2452582|Dr.|James|Whitaker|Y|20|12|1960|ANTIGUA AND BARBUDA||James.Whitaker@EOoToVx9pAyjA7.org|2452521| +7735|AAAAAAAAHDOBAAAA|1586484|4189|21447|2449480|2449450|Sir|Russell|Lewis|Y|21|3|1945|BAHAMAS||Russell.Lewis@UJHdJBcj8kqus329.edu|2452369| +7736|AAAAAAAAIDOBAAAA|346667|962|16904|2451252|2451222|Mr.|Daniel|South|Y|15|4|1932|BOLIVIA||Daniel.South@jG.edu|2452297| +7737|AAAAAAAAJDOBAAAA|1420869|314|30959|2451146|2451116|Dr.|Tammy|Miles|Y|4|10|1992|ANGUILLA||Tammy.Miles@nb31eFan.edu|2452417| +7738|AAAAAAAAKDOBAAAA|47052|3918|41584|2450125|2450095|Sir|Karl|Carpenter|Y|25|8|1977|CHINA||Karl.Carpenter@5Sd.edu|2452444| +7739|AAAAAAAALDOBAAAA|1244692|1544|24796|2449443|2449413|Sir|John|Coon|N|9|5|1986|NEPAL||John.Coon@QvqVD.edu|2452390| +7740|AAAAAAAAMDOBAAAA|1020341|6027|22513|2452128|2452098|Mrs.|Cheryl|Gibbs|N|9|4|1945|TRINIDAD AND TOBAGO||Cheryl.Gibbs@ab18T7AYUTziY1C0J7r.com|2452295| +7741|AAAAAAAANDOBAAAA|1236757|6307|33410|2450151|2450121|Mr.|Neil|Felix|N|14|9|1931|MARTINIQUE||Neil.Felix@TQKkntvdR.edu|2452339| +7742|AAAAAAAAODOBAAAA|1786981|6258|11135|2452481|2452451|Dr.|John|Haywood|N|18|2|1944|SRI LANKA||John.Haywood@NO9yx5j.org|2452606| +7743|AAAAAAAAPDOBAAAA||5824|30840|2451526|2451496|Sir|William||N||8|1953|BAHRAIN|||2452586| +7744|AAAAAAAAAEOBAAAA|17574|3922|17470|2451590|2451560|Dr.|Gracie|Puckett|N|21|7|1983|RUSSIAN FEDERATION||Gracie.Puckett@v2TxPxYiCR.org|2452524| +7745|AAAAAAAABEOBAAAA|36177|1294|15964|2450908|2450878|Dr.|Stefan|Owens|N|7|11|1942|AUSTRIA||Stefan.Owens@102ycH2.edu|2452501| +7746|AAAAAAAACEOBAAAA|1493984|5616|14048|2450967|2450937|Dr.|Brian|Bryson|N|8|2|1930|MOLDOVA, REPUBLIC OF||Brian.Bryson@6f.org|2452423| +7747|AAAAAAAADEOBAAAA||2612|48242||2452532||Melissa|Hobbs|||||NEW ZEALAND|||| +7748|AAAAAAAAEEOBAAAA|1449523|2269|43415|2450465|2450435|Ms.|Sharon|Nguyen|N|6|2|1988|NAURU||Sharon.Nguyen@3b0zptDtEQjo.edu|2452352| +7749|AAAAAAAAFEOBAAAA|171805|2985|1765|2451453|2451423|Ms.|Myung|Windham|N|7|12|1991|KENYA||Myung.Windham@YLuUU.org|2452571| +7750|AAAAAAAAGEOBAAAA|299636|6607|38025|2451618|2451588|Sir|Walter|Dunn|Y|5|11|1931|ANGUILLA||Walter.Dunn@zT0KqOzVOf2.org|2452618| +7751|AAAAAAAAHEOBAAAA|488999|3337|42522|2450096|2450066|Miss|Lucille|Byrd|N|14|5|1933|GUERNSEY||Lucille.Byrd@qPAFBouO3RA82cgl4.com|2452646| +7752|AAAAAAAAIEOBAAAA|1304602|5255|3061|2452582|2452552|Sir|Nathaniel|Grimes|N|19|2|1955|CHINA||Nathaniel.Grimes@ZjIY9ZuyI8t.com|2452532| +7753|AAAAAAAAJEOBAAAA|975203|4548|45893|2449405|2449375|Mrs.|Elizabeth|Brown|Y|7|2|1945|FRANCE||Elizabeth.Brown@Kube.com|2452620| +7754|AAAAAAAAKEOBAAAA|568474|6128|29829|2452556|2452526|Sir|Dean|Tripp|Y|5|8|1968|PITCAIRN||Dean.Tripp@0jybpJtR25sAb.org|2452458| +7755|AAAAAAAALEOBAAAA|1497764|3821|21525|2449876|2449846|Ms.|Judith|Patterson|N|24|5|1975|NICARAGUA||Judith.Patterson@Px9.org|2452512| +7756|AAAAAAAAMEOBAAAA||316|21680|||||Jackson|||||ROMANIA||Ola.Jackson@RxhJLZdbaMR.org|2452357| +7757|AAAAAAAANEOBAAAA|1817450|2669|24735|2451470|2451440|Sir|Milo|Fowler|Y|23|9|1956|GUINEA-BISSAU||Milo.Fowler@ADdj4.org|2452505| +7758|AAAAAAAAOEOBAAAA|410949|1268|41940|2451501|2451471|Dr.|Doris|Lewis|N|18|2|1960|VENEZUELA||Doris.Lewis@PorMqsdr4s.edu|2452439| +7759|AAAAAAAAPEOBAAAA|893846|2402|43405|2451694|2451664|Dr.|Melissa|Goodin|N|8|4|1949|NIGERIA||Melissa.Goodin@c9j2aiB1.org|2452486| +7760|AAAAAAAAAFOBAAAA|1227102|4423|23744|2449873|2449843|Sir|Vernon|Tuttle|N|11|10|1952|ARMENIA||Vernon.Tuttle@4M2NDYpBQoN3.org|2452320| +7761|AAAAAAAABFOBAAAA|1126298|7089|15480|2449828|2449798|Dr.|Robert|Fenton|N|20|7|1979|GERMANY||Robert.Fenton@sJ3a.edu|2452625| +7762|AAAAAAAACFOBAAAA|1597634|3490|16099|2452368|2452338|Dr.|James|Ellis|N|8|5|1938|MALTA||James.Ellis@IcnKpx8sdCvTgsap.edu|2452285| +7763|AAAAAAAADFOBAAAA|1301237|1850|510|2450345|2450315|Sir|Charles|Gregory|N|2|7|1986|ZAMBIA||Charles.Gregory@R4p.org|2452284| +7764|AAAAAAAAEFOBAAAA|1552901|2995|736|2451223|2451193|Dr.|Arthur|Griffin|N|30|3|1985|GABON||Arthur.Griffin@mzKqLL6ZHOrkFB.org|2452318| +7765|AAAAAAAAFFOBAAAA|1824758|284|20657|2451816|2451786|Ms.|Madaline|Garner|Y|6|3|1991|SWEDEN||Madaline.Garner@aRFqGRV.edu|2452317| +7766|AAAAAAAAGFOBAAAA|1224428|3536|691|2451923|2451893|Sir|Robert|Reynolds|N|11|4|1938|SINGAPORE||Robert.Reynolds@vo.edu|2452444| +7767|AAAAAAAAHFOBAAAA|973045|3150|13655|2450861|2450831|Miss|Christine|Romero|N|30|11|1945|SWITZERLAND||Christine.Romero@OBf24JmStIZt.edu|2452312| +7768|AAAAAAAAIFOBAAAA|703856|5037|13018|2452295|2452265|Mrs.|Diana|Bennett|N|30|1|1955|SINGAPORE||Diana.Bennett@uKz7Zq.com|2452641| +7769|AAAAAAAAJFOBAAAA|917892|6807|18122|2452327|2452297|Sir|Christopher|Johnson|Y|4|7|1949|FRANCE||Christopher.Johnson@KoBzo7UEYH.org|2452314| +7770|AAAAAAAAKFOBAAAA|1244727|4845|28210|2450682|2450652|Ms.|Elizabeth|Fish|N|27|3|1981|VENEZUELA||Elizabeth.Fish@sK3.com|2452548| +7771|AAAAAAAALFOBAAAA|356762|3465|9768|2451765|2451735|Dr.|George|Compton|N|21|1|1975|MONTENEGRO||George.Compton@7AhP33.edu|2452442| +7772|AAAAAAAAMFOBAAAA|884421|546|35871|2450605|2450575|Dr.|Pamela|Harding|Y|29|12|1928|ETHIOPIA||Pamela.Harding@Feh1Ug3c31amqYY.edu|2452504| +7773|AAAAAAAANFOBAAAA|788010|4291|44572|2450771|2450741|Mr.|William|Miller|N|1|8|1977|CANADA||William.Miller@kAQYdf6Y07tT.edu|2452375| +7774|AAAAAAAAOFOBAAAA|728925|2579|12680|2450945|2450915|Dr.|Ryan|Borden|Y|23|11|1972|OMAN||Ryan.Borden@V.edu|2452289| +7775|AAAAAAAAPFOBAAAA||5016|36103|2451413|2451383|||||26|6|1951|||Jennie.Daley@tci27r3.com|| +7776|AAAAAAAAAGOBAAAA|273083|4428|33326|2449631|2449601|Dr.|Leonel|Williams|Y|18|2|1954|MEXICO||Leonel.Williams@jHci5MAvbA.com|2452442| +7777|AAAAAAAABGOBAAAA|1678404|4129|43807|2450761|2450731|Dr.|Theodore|Stahl|N|30|7|1949|C�TE D'IVOIRE||Theodore.Stahl@HZxVdCq8D08h.org|2452438| +7778|AAAAAAAACGOBAAAA||5022|38845|2450114||Mrs.|Vicki||N|21|11|1935||||2452483| +7779|AAAAAAAADGOBAAAA|||45640|2451563|||Irma|||1||1934||||2452462| +7780|AAAAAAAAEGOBAAAA|293187|1402|43570|2451781|2451751|Mrs.|Ann|Case|N|21|11|1983|BOTSWANA||Ann.Case@DskXD15eqj1.org|2452378| +7781|AAAAAAAAFGOBAAAA|1337939|1068|35750|2449643|2449613|Mrs.|Barbra|Allen|N|16|5|1924|FIJI||Barbra.Allen@Kk.org|2452352| +7782|AAAAAAAAGGOBAAAA|1613342|388|17387|2450409|2450379|Dr.|Teresa|Tracy|Y|3|1|1939|ICELAND||Teresa.Tracy@bPT.edu|2452439| +7783|AAAAAAAAHGOBAAAA||4005|35570||2451789||Mable|||||1943|SAINT HELENA||Mable.Proffitt@Kzy3JA4ChX.org|2452525| +7784|AAAAAAAAIGOBAAAA|108849|228|5207|2451751|2451721|Dr.|Maurice|Gray|Y|2|7|1969|BURUNDI||Maurice.Gray@S.edu|2452491| +7785|AAAAAAAAJGOBAAAA|1799915|2502|40242|2452124|2452094|Dr.|Carlos|Roach|N|11|3|1968|TIMOR-LESTE||Carlos.Roach@pHi.org|2452620| +7786|AAAAAAAAKGOBAAAA|823051|6951|8404|2451128|2451098|Ms.|Brenda|Brown|N|2|2|1927|GRENADA||Brenda.Brown@STSOlMcChJHy9F4Bn.edu|2452628| +7787|AAAAAAAALGOBAAAA|1181726|6852|10425|2449791|2449761|Mr.|Larry|Berger|Y|13|7|1980|FRENCH GUIANA||Larry.Berger@YdjoHgEVsMK.com|2452402| +7788|AAAAAAAAMGOBAAAA|285994|1298|16249|2449430|2449400|Ms.|Brenda|Benson|N|4|7|1926|SAMOA||Brenda.Benson@uRZ9h00.org|2452464| +7789|AAAAAAAANGOBAAAA|45718|5043|19168|2450679|2450649|Ms.|Betty|Henderson|Y|19|8|1937|TUNISIA||Betty.Henderson@E.com|2452302| +7790|AAAAAAAAOGOBAAAA|1237468|2257|646|2449645|2449615|Mr.|Joshua|Waters|Y|2|10|1932|BAHRAIN||Joshua.Waters@MqsuCygmyRb.edu|2452540| +7791|AAAAAAAAPGOBAAAA|188380|2380|12780|2450818||||||3|5||||Leon.Myrick@geU3xtPZ9s5.com|| +7792|AAAAAAAAAHOBAAAA|1515376|3679|2782|2451489|2451459|Mr.|Armando|King|Y|26|8|1948|ANGUILLA||Armando.King@yhh3yrl1d.edu|2452563| +7793|AAAAAAAABHOBAAAA|156137|6265|41946|2452302|2452272|Mr.|William|Dobbs|Y|10|6|1951|TOGO||William.Dobbs@AaA1sZcX9YzEBEa.org|2452471| +7794|AAAAAAAACHOBAAAA|1061215|3539|17016|2452169|2452139|Dr.|Erik|Pulido|Y|15|10|1975|MARTINIQUE||Erik.Pulido@H.org|2452437| +7795|AAAAAAAADHOBAAAA|1521252|3020|42341|2452480|2452450|Dr.|Carolyn|Garza|Y|19|5|1959|BAHRAIN||Carolyn.Garza@ua.org|2452560| +7796|AAAAAAAAEHOBAAAA|833534|6237|8987|2451183|2451153|Mr.|James|Morris|N|27|10|1959|UNITED KINGDOM||James.Morris@g6MCb7IYn.com|2452361| +7797|AAAAAAAAFHOBAAAA|1803123|5894|49865|2449516|2449486|Miss|Patrica|Ramirez|N|31|8|1951|PITCAIRN||Patrica.Ramirez@8TmsJVhA4ccqkq.com|2452312| +7798|AAAAAAAAGHOBAAAA|1146589|1082|13175|2451468|2451438|Dr.|Thomas|Owens|Y|3|12|1974|SAINT HELENA||Thomas.Owens@z1Si.com|2452441| +7799|AAAAAAAAHHOBAAAA|1038896|1577|45858|2450454|2450424|Mr.|John|Simpson|Y|26|5|1953|BURKINA FASO||John.Simpson@unlF7.edu|2452356| +7800|AAAAAAAAIHOBAAAA|775963|3851|37339|2450350|2450320|Ms.|Dina|Long|N|10|5|1986|KOREA, REPUBLIC OF||Dina.Long@rUav.com|2452411| +7801|AAAAAAAAJHOBAAAA|1489873|6603|9780|2451204|2451174|Miss|Beverlee|Camp|Y|1|7|1946|CAMBODIA||Beverlee.Camp@g7e72Mrp.org|2452425| +7802|AAAAAAAAKHOBAAAA|772273|522|30418|2451468|2451438|Sir|William|Riddle|Y|6|5|1956|ROMANIA||William.Riddle@gYNDjRAGy6SH.com|2452467| +7803|AAAAAAAALHOBAAAA|1423858|6196|31734|2451929|2451899|Mr.|James|Barnes|N|24|12|1946|MAURITIUS||James.Barnes@8uzGudNkKdCBL.com|2452479| +7804|AAAAAAAAMHOBAAAA|1430921|511|21701|2451308|2451278|Dr.|Eileen|Robbins|Y|27|4|1929|CHINA||Eileen.Robbins@Xf6gplAtdktza.org|2452453| +7805|AAAAAAAANHOBAAAA|1510969|6856|11943|2451695|2451665|Dr.|Ronald|Shanks|Y|11|5|1943|FRENCH GUIANA||Ronald.Shanks@F1BOBSuPtnBE.com|2452406| +7806|AAAAAAAAOHOBAAAA|1038988|4074|19513|2449267|2449237|Sir|Richard|Guerrero|N|14|3|1929|SYRIAN ARAB REPUBLIC||Richard.Guerrero@4ym9g5LcYjk2G.edu|2452312| +7807|AAAAAAAAPHOBAAAA|1090259|1343|41227|2449182|2449152|Dr.|Amparo|Hubbard|Y|22|12|1984|DOMINICA||Amparo.Hubbard@7DjdRaCNYBA2HA.com|2452471| +7808|AAAAAAAAAIOBAAAA|1204033|2480|12127|2449522|2449492|Dr.|Martha|Thomas|Y|8|9|1988|MAURITIUS||Martha.Thomas@oaLkYk1akNk5bBDO.org|2452536| +7809|AAAAAAAABIOBAAAA|1496962|3618|35074|2449510|2449480|Mr.|Marion|Davis|Y|18|2|1975|LIECHTENSTEIN||Marion.Davis@E1iSGRiojuQ9mg.org|2452286| +7810|AAAAAAAACIOBAAAA|1811127|201|28922|2450559|2450529|Dr.|Christy|Clayton|Y|14|10|1945|LUXEMBOURG||Christy.Clayton@YoqxS514cQ2Kivy2.edu|2452540| +7811|AAAAAAAADIOBAAAA|1128037|7091|18145|2452397|2452367|Sir|Michael|Ortiz|N|5|5|1929|TAJIKISTAN||Michael.Ortiz@ZdlZ1Vu3CBdS4.com|2452621| +7812|AAAAAAAAEIOBAAAA|611112|6805|20051|2450838|2450808|Dr.|Lisa|Ross|N|19|1|1969|CAYMAN ISLANDS||Lisa.Ross@C4jTshszF4.edu|2452555| +7813|AAAAAAAAFIOBAAAA|35265|869|42120|2449115|2449085|Ms.|Naomi|Ladner|Y|1|7|1925|GAMBIA||Naomi.Ladner@q4.edu|2452467| +7814|AAAAAAAAGIOBAAAA|297009|3134|20012|2451981|2451951|Mrs.|Soon|Riley|N|19|9|1942|GUATEMALA||Soon.Riley@FRc7nvkELJrQOl6A.com|2452334| +7815|AAAAAAAAHIOBAAAA|1055280|1779|12631|2450954|2450924|Mrs.|Brittany|Salazar|Y|24|3|1960|SIERRA LEONE||Brittany.Salazar@oyQqGQCLUq.edu|2452414| +7816|AAAAAAAAIIOBAAAA|1303825|3758|27291|2449772|2449742|Mr.|George|Harrison|N|25|10|1928|PAPUA NEW GUINEA||George.Harrison@HdTJxV03jd.edu|2452416| +7817|AAAAAAAAJIOBAAAA|923059|2424|20189|2451160|2451130|Sir|Scott|White|N|20|5|1930|UZBEKISTAN||Scott.White@DlaykYCTIdd.org|2452645| +7818|AAAAAAAAKIOBAAAA|324269|6184|25699|2449082|2449052|Dr.|Calvin|Williams|N|21|1|1957|HUNGARY||Calvin.Williams@duRcgQ5TF28NZ.org|2452428| +7819|AAAAAAAALIOBAAAA|1604030|6966|29902|2451487|2451457|Dr.|Peter|Luster|N|23|6|1931|UNITED KINGDOM||Peter.Luster@6KySn7jH0EmujRiI.com|2452458| +7820|AAAAAAAAMIOBAAAA|||28523|2450940|2450910|Sir||||||1934|||Warren.Cox@JF1uIhhDdS.org|2452340| +7821|AAAAAAAANIOBAAAA|1253318|6452|29186|2452370|2452340|Dr.|Ken|Matthews|N|9|6|1933|BULGARIA||Ken.Matthews@N6QC8HlhA.edu|2452555| +7822|AAAAAAAAOIOBAAAA|1047123|1380|25637|2451313|2451283|Dr.|Willie|Jones|Y|27|3|1979|PORTUGAL||Willie.Jones@pk1r1F8uiF.edu|2452291| +7823|AAAAAAAAPIOBAAAA|1052789|3809|28029|2449242|2449212|Mrs.|Leila|Burton|Y|11|5|1946|NAURU||Leila.Burton@UpA7A5cNaV7NuXY.org|2452604| +7824|AAAAAAAAAJOBAAAA|1191451|836|13868|2452130|2452100|Miss|Jennifer|Bailey|N|13|11|1940|PARAGUAY||Jennifer.Bailey@XaAJOUdskDTC.org|2452484| +7825|AAAAAAAABJOBAAAA|1772072|4587|9502|2449581|2449551|Dr.|Gilbert|Romero|N|5|2|1968|KENYA||Gilbert.Romero@oaojr67CaQ0u2Db3V.edu|2452626| +7826|AAAAAAAACJOBAAAA|477684|1586|29888|2449371|2449341|Mrs.|Elizabeth|Holmes|N|13|1|1947|ARGENTINA||Elizabeth.Holmes@SYL.edu|2452355| +7827|AAAAAAAADJOBAAAA|967306|2163|17820|2452074|2452044|Dr.|Carl|Morales|N|23|12|1960|GUINEA||Carl.Morales@Pv0.com|2452321| +7828|AAAAAAAAEJOBAAAA|1340372|4069|27766|2451047|2451017|Ms.|Melissa|Burnett|Y|18|5|1972|GREECE||Melissa.Burnett@TmRzO76UMv46.edu|2452417| +7829|AAAAAAAAFJOBAAAA|1416443|6001|36759|2449506|2449476|Dr.|Deborah|Maguire|N|30|1|1966|BURKINA FASO||Deborah.Maguire@I.org|2452552| +7830|AAAAAAAAGJOBAAAA|388581|114|1294|2449104|2449074|Ms.|Tina|Norton|N|7|9|1981|BULGARIA||Tina.Norton@yba2QIxGk.edu|2452380| +7831|AAAAAAAAHJOBAAAA|1025090|3975|3260|2450164|2450134|Mrs.|April|Wallace|N|19|7|1930|FIJI||April.Wallace@9iuUi3U0K.edu|2452382| +7832|AAAAAAAAIJOBAAAA|175133|1404|11675|2452609|2452579|Mrs.|Judith|Calloway|N|19|10|1984|GUAM||Judith.Calloway@t1oi7nK8.edu|2452384| +7833|AAAAAAAAJJOBAAAA|579779|5405|22584|2449331|2449301|Mr.|Fernando|Ramirez|Y|18|12|1960|GUINEA-BISSAU||Fernando.Ramirez@zhKVJp0JaRjTQY.edu|2452591| +7834|AAAAAAAAKJOBAAAA|360987|4617|24200|2451555|2451525|Mrs.|Claudette|Patton|N|5|9|1948|KUWAIT||Claudette.Patton@T81R3JI1KuT6RU.edu|2452334| +7835|AAAAAAAALJOBAAAA|1187433|3440|35047|2449167|2449137|Sir|Joel|Martinez|Y|9|8|1946|ARUBA||Joel.Martinez@D65NAbR49E.com|2452329| +7836|AAAAAAAAMJOBAAAA|1160164|2628|49139|2452517|2452487|Sir|Roy|Miller|N|9|12|1950|FINLAND||Roy.Miller@VXdYGcpYzr.org|2452459| +7837|AAAAAAAANJOBAAAA|1895825|983|1694|2452391|2452361|Sir|Everett|Lloyd|N|18|2|1956|OMAN||Everett.Lloyd@tOg1V1l4Lqae.com|2452288| +7838|AAAAAAAAOJOBAAAA|967519|2298|4277|2449648|2449618|Ms.|Lori|Hunt|Y|18|12|1944|INDIA||Lori.Hunt@NCz3AyntvTrCkxn.com|2452639| +7839|AAAAAAAAPJOBAAAA|||19492|2452642||||Dugas||6|10||||Ana.Dugas@z4uBkf7v7U.edu|| +7840|AAAAAAAAAKOBAAAA|||43014|2452505|2452475|||Smith||6|||||Charles.Smith@6X2zS70yh.edu|2452341| +7841|AAAAAAAABKOBAAAA|1340064|1369|25074|2451935|2451905|Sir|Ronald|Ezell|N|22|4|1941|AFGHANISTAN||Ronald.Ezell@3QczG0QUB.edu|2452306| +7842|AAAAAAAACKOBAAAA|843653|7032|46458|2451891|2451861|Mrs.|Ethel|Spencer|N|25|5|1941|SAUDI ARABIA||Ethel.Spencer@zrg5vqg.com|2452610| +7843|AAAAAAAADKOBAAAA|1726052|4473|31247|2449595|2449565|Dr.|Joseph|Hudgens|Y|24|2|1971|NAURU||Joseph.Hudgens@9kZc05L4.org|2452347| +7844|AAAAAAAAEKOBAAAA|1428848|3306|13011|2451711|2451681|Mr.|Stephen|Bradshaw|Y|2|11|1956|ERITREA||Stephen.Bradshaw@zxKS7pIrXi620i.org|2452510| +7845|AAAAAAAAFKOBAAAA|290291|6508|40772|2449401|2449371|Dr.|George|Steele|Y|12|11|1990|EGYPT||George.Steele@xn7M1Rd.com|2452296| +7846|AAAAAAAAGKOBAAAA|1281053|631|28162|2450628|2450598|Mrs.|Judith|Howard|N|22|9|1989|TURKEY||Judith.Howard@gGfz2k3J.org|2452384| +7847|AAAAAAAAHKOBAAAA|1064485|6149|44213|2452202|2452172|Dr.|Jennifer|Tucker|N|5|9|1966|COMOROS||Jennifer.Tucker@Ma2EjnxdXxjGUAz.org|2452505| +7848|AAAAAAAAIKOBAAAA|539261|2584|35107|2449154|2449124|Dr.|Sandra|Barbee|N|1|8|1960|TUNISIA||Sandra.Barbee@4ZkRV4Dq5hiP4evGZ.com|2452517| +7849|AAAAAAAAJKOBAAAA|1401839|4061|10236|2451935|2451905|Ms.|Cathy|Davis|N|5|12|1962|PHILIPPINES||Cathy.Davis@UBmTnfkD2lyxB9.edu|2452647| +7850|AAAAAAAAKKOBAAAA|911741|6429|21070|2449219|2449189|Dr.|Barbara|Howard|Y|6|9|1940|KIRIBATI||Barbara.Howard@UyZ7Ai6GLfjHnCS.edu|2452540| +7851|AAAAAAAALKOBAAAA|773338|2369|31800|2450740|2450710|Ms.|Latonya|Christian|Y|22|8|1955|TURKEY||Latonya.Christian@tQFeKecsoQ.com|2452405| +7852|AAAAAAAAMKOBAAAA|1525707|1379|36251|2452410|2452380|Mr.|Herbert|Rooney|N|15|10|1940|MYANMAR||Herbert.Rooney@T1cGmQogb53VkLGf.com|2452501| +7853|AAAAAAAANKOBAAAA|1893475|5533|44791|2451945|2451915|Ms.|Katherine|Zapata|Y|25|11|1978|ALBANIA||Katherine.Zapata@CMy8GrF0.org|2452419| +7854|AAAAAAAAOKOBAAAA|1344973||38298|||||Stewart|N||2||GUERNSEY|||| +7855|AAAAAAAAPKOBAAAA|1764992|636|5480|2452084|2452054|Sir|Edward|Zamora|N|13|7|1932|COMOROS||Edward.Zamora@vxPFCfQx.com|2452509| +7856|AAAAAAAAALOBAAAA|259286|4870|22501|2450032|2450002|Mr.|Patrick|Williams|N|18|12|1989|PARAGUAY||Patrick.Williams@DJJg0p.edu|2452385| +7857|AAAAAAAABLOBAAAA|1380738|4649|46607|2452189|2452159|Mr.|Charles|Keen|Y|9|11|1924|PANAMA||Charles.Keen@A4m7Cuh2xaj25dgSC.edu|2452352| +7858|AAAAAAAACLOBAAAA|439078|2545|8859|2449473|2449443|Miss|Brenda|Nagy|Y|15|5|1931|BENIN||Brenda.Nagy@nUg0SUaN1v9vbldP.com|2452629| +7859|AAAAAAAADLOBAAAA|344463|2014|17212|2452595|2452565|Dr.|Naomi|Bailey|Y|14|2|1933|MOROCCO||Naomi.Bailey@RP2kglsJSn.edu|2452341| +7860|AAAAAAAAELOBAAAA|694995|4782|35416|2451024|2450994|Dr.|Sadie|Hampton|N|10|10|1927|PANAMA||Sadie.Hampton@j8QzeRp8UQ4O.com|2452296| +7861|AAAAAAAAFLOBAAAA|1738321|6518|20762|2451338|2451308|Mr.|Thomas|Davis|Y|20|4|1981|INDONESIA||Thomas.Davis@FvcCS0OmpE7Sp.com|2452350| +7862|AAAAAAAAGLOBAAAA|1061498|1281|38664|2450318|2450288|Ms.|Angela|Jones|N|8|6|1971|NICARAGUA||Angela.Jones@1xA81jEThKoHoT.org|2452609| +7863|AAAAAAAAHLOBAAAA|1702549|2360|44971|2449059|2449029|Sir|Steven|Parker|N|2|3|1982|GAMBIA||Steven.Parker@A8IyObHfLcslgs.org|2452338| +7864|AAAAAAAAILOBAAAA|1273883|688|11493|2452660|2452630|Mr.|James|Diaz|Y|19|1|1990|PORTUGAL||James.Diaz@gc0yg.com|2452295| +7865|AAAAAAAAJLOBAAAA|937465|2857|5849|2452636|2452606|Ms.|Virginia|Santos|Y|10|12|1955|KYRGYZSTAN||Virginia.Santos@fNoLinAx94O.com|2452566| +7866|AAAAAAAAKLOBAAAA|381044|6445|45846|2451057|2451027|Ms.|Rose|Roberson|Y|11|7|1968|ETHIOPIA||Rose.Roberson@DuEVOcBTH.edu|2452369| +7867|AAAAAAAALLOBAAAA|1527192|2708|4271|2450816|2450786|Dr.|Dena|Sellers|N|17|2|1970|MAYOTTE||Dena.Sellers@AyGzPf.com|2452393| +7868|AAAAAAAAMLOBAAAA|1205608||42104||||||||7|1986|NEW ZEALAND||Michael.Kelly@ddZd7.com|2452605| +7869|AAAAAAAANLOBAAAA|512202|1130|4058|2450687|2450657|Dr.|Bud|Carson|Y|27|9|1958|BURUNDI||Bud.Carson@yvJ3.org|2452555| +7870|AAAAAAAAOLOBAAAA|1198631|3303|11928|2450832|2450802|Ms.|Darla|Skinner|N|14|11|1990|ARMENIA||Darla.Skinner@JHLj.edu|2452367| +7871|AAAAAAAAPLOBAAAA|1320164|1273|31629|2449803|2449773|Mr.|Christopher|Summers|N|28|1|1948|CHINA||Christopher.Summers@GIIoagiS5ODQ.edu|2452457| +7872|AAAAAAAAAMOBAAAA|598497|6864|27631|2452515|2452485|Mrs.|Virginia|Simpson|Y|26|5|1963|ANGUILLA||Virginia.Simpson@TQqM6bx2PkK.org|2452317| +7873|AAAAAAAABMOBAAAA|1487986|625|28111|2449120|2449090|Dr.|Philip|Burgess|Y|13|6|1984|MALDIVES||Philip.Burgess@TgO.com|2452315| +7874|AAAAAAAACMOBAAAA||3910|4425|2449189|||Angela|Harris||27|6|1953||||2452364| +7875|AAAAAAAADMOBAAAA|1221727|1434|29141|2451194|2451164|Miss|Antonio|Harmon|Y|10|8|1984|OMAN||Antonio.Harmon@A05EP32qvi.edu|2452555| +7876|AAAAAAAAEMOBAAAA|1197733|382|35431|2450117|2450087|Ms.|Adam|Kessler|Y|7|9|1974|PERU||Adam.Kessler@Jo7ufOA7.edu|2452598| +7877|AAAAAAAAFMOBAAAA|286786|1083|45780|2450213|2450183|Mr.|Edward|Chavarria|Y|14|3|1977|TAJIKISTAN||Edward.Chavarria@Zhmbh1kV5JZc.edu|2452518| +7878|AAAAAAAAGMOBAAAA|781262|748|22681|2451235|2451205|Dr.|Victor|Frazier|N|25|3|1988|SLOVENIA||Victor.Frazier@u6BI6DkMjrVBr.com|2452442| +7879|AAAAAAAAHMOBAAAA|1852383|3887|48544|2451347|2451317|Sir|Harry|Jones|N|22|2|1975|NICARAGUA||Harry.Jones@MK.org|2452356| +7880|AAAAAAAAIMOBAAAA|925327|5037|16347|2449373|2449343|Mrs.|Kyung|Miller|Y|22|12|1928|PAKISTAN||Kyung.Miller@I0i8Th.org|2452355| +7881|AAAAAAAAJMOBAAAA|1848010|4901|40545|2451594|2451564|Dr.|Harold|Lyons|Y|11|2|1991|BOUVET ISLAND||Harold.Lyons@FM6t1Ht0r.edu|2452638| +7882|AAAAAAAAKMOBAAAA|405726|5437|34795|2450045|2450015|Miss|Tina|Ruiz|N|6|5|1928|ZAMBIA||Tina.Ruiz@DKeH0.org|2452354| +7883|AAAAAAAALMOBAAAA|1714913|5677|29241|2450940|2450910|Ms.|Jacquelyn|Bryant|N|7|1|1958|NEPAL||Jacquelyn.Bryant@jcuPUymvaG6fiD5i1lq.com|2452628| +7884|AAAAAAAAMMOBAAAA|547354|450|14299|2450448|2450418|Miss|Adeline|Padilla|Y|14|3|1948|FINLAND||Adeline.Padilla@HrVzaH6LTCv.com|2452445| +7885|AAAAAAAANMOBAAAA|1649299|3704|22509|2449977|2449947|Sir|Rudolph|Masterson|Y|22|8|1928|SWITZERLAND||Rudolph.Masterson@23Grbx4f9ZO8fq.com|2452618| +7886|AAAAAAAAOMOBAAAA|1566582|1262|43142|2449880|2449850|Mr.|Richard|Le|Y|29|1|1931|LATVIA||Richard.Le@OVqo.org|2452332| +7887|AAAAAAAAPMOBAAAA|242900|1378|12609|2452218|2452188|Dr.|Diane|Jacobson|N|11|12|1974|ANTARCTICA||Diane.Jacobson@a03t2d.org|2452509| +7888|AAAAAAAAANOBAAAA|1354600||25083||2451279||Robert||Y|13|11|||||2452348| +7889|AAAAAAAABNOBAAAA|70936|6207|31878|2449843|2449813|Mrs.|Rachel||Y|11||1970||||2452284| +7890|AAAAAAAACNOBAAAA|776086|4087|4488|2449315|2449285|Sir|John|Numbers|Y|10|5|1985|NICARAGUA||John.Numbers@g0PP9dX.org|2452549| +7891|AAAAAAAADNOBAAAA|1613232|3783|6334|2449657|2449627|Sir|Joshua|Sherman|N|18|10|1942|BELIZE||Joshua.Sherman@T5tJmnO.com|2452320| +7892|AAAAAAAAENOBAAAA|912432|338|45844|2449945|2449915|Sir|Danial|Tyson|Y|31|10|1924|AZERBAIJAN||Danial.Tyson@RZEe9bn1EbuxS4.org|2452572| +7893|AAAAAAAAFNOBAAAA|1511451|6972|20940|2451450|2451420|Sir|Donnell|Oneil|Y|8|9|1938|KAZAKHSTAN||Donnell.Oneil@GGoU8m.com|2452565| +7894|AAAAAAAAGNOBAAAA|252448|1819|29190|2450972|2450942|Mr.|James|Childs|N|27|6|1935|COSTA RICA||James.Childs@rD5Y3Gux.edu|2452491| +7895|AAAAAAAAHNOBAAAA|1433190|2616|13242|2449170|2449140|Dr.|Alana|Clark|Y|9|6|1947|PUERTO RICO||Alana.Clark@BfPvN.com|2452427| +7896|AAAAAAAAINOBAAAA|6603|2237|42671|2451925|2451895|Ms.|Tammy|Wilson|N|26|4|1942|POLAND||Tammy.Wilson@2Ke5VUAsRfNB.org|2452463| +7897|AAAAAAAAJNOBAAAA|1601708|3397|39533|2450298|2450268|Mrs.|Lela|Wallace|Y|18|10|1932|CHILE||Lela.Wallace@pgESAl.com|2452363| +7898|AAAAAAAAKNOBAAAA|934061|3194|1974|2452304|2452274|Dr.|Kraig|Armstrong|N|16|4|1968|KUWAIT||Kraig.Armstrong@PD9DBO2ichknuVh3lg.edu|2452320| +7899|AAAAAAAALNOBAAAA|913751|4342|38886|2450363|2450333|Sir|John|Kroll|N|13|2|1948|MONGOLIA||John.Kroll@vvIkxhOZh.edu|2452381| +7900|AAAAAAAAMNOBAAAA|1671537|555|45526|2449117|2449087|Mr.|Edward|Lee|N|21|4|1981|CHINA||Edward.Lee@NYfNlR3TJC.org|2452472| +7901|AAAAAAAANNOBAAAA|436131|980|20629|2452332|2452302|Ms.|Lena|Perry|N|26|8|1928|LUXEMBOURG||Lena.Perry@0g3.com|2452467| +7902|AAAAAAAAONOBAAAA|1328118|3259|18447|2449469|2449439|Mr.|Henry|Burnette|N|11|6|1962|BELARUS||Henry.Burnette@GvANMDdepU.org|2452402| +7903|AAAAAAAAPNOBAAAA|||2081||2451834|||||5|12|1944||||2452365| +7904|AAAAAAAAAOOBAAAA|269037|6841|35045|2451850|2451820|Mrs.|Annie|Mckenzie|Y|13|5|1958|VIET NAM||Annie.Mckenzie@Z.edu|2452479| +7905|AAAAAAAABOOBAAAA|936883|3197|45864|2451581|2451551|Mrs.|Stephanie|Reece|N|22|11|1939|GEORGIA||Stephanie.Reece@x2xHg0RpCrh.org|2452590| +7906|AAAAAAAACOOBAAAA|243994|4493|16592|2449431|2449401|Dr.|Robert|Willis|Y|18|7|1956|MONTSERRAT||Robert.Willis@Zv.edu|2452647| +7907|AAAAAAAADOOBAAAA|1017843|544|7692|2452180|2452150|Dr.|Robert|Fried|N|9|6|1962|ANDORRA||Robert.Fried@mXBRNphRhokE.edu|2452309| +7908|AAAAAAAAEOOBAAAA|504262|6333|20004|2449594|2449564|Ms.|Effie|Dove|N|25|12|1933|BOUVET ISLAND||Effie.Dove@v9Ypc.edu|2452488| +7909|AAAAAAAAFOOBAAAA|1885727|5880|10833|2449964|2449934|Ms.|Erica|Hopkins|N|21|8|1991|MALDIVES||Erica.Hopkins@JfRF96NA6cRYJV1s5Ko.edu|2452452| +7910|AAAAAAAAGOOBAAAA|530980|3045|5863|2450265|2450235|Sir|Jerry|Barlow|Y|5|2|1984|EGYPT||Jerry.Barlow@HATmhLzmM.edu|2452380| +7911|AAAAAAAAHOOBAAAA|330284|2771|20905|2450677|2450647|Ms.|Raquel|Mcbride|N|13|5|1943|GUINEA||Raquel.Mcbride@l.com|2452601| +7912|AAAAAAAAIOOBAAAA|1001391|2476|45161|2451875|2451845|Ms.|Maritza|Williams|N|12|8|1977|EQUATORIAL GUINEA||Maritza.Williams@gijuRcZh2B3JbTs.edu|2452389| +7913|AAAAAAAAJOOBAAAA|1566919|3018|2670|2449284|2449254|Mrs.|Dorcas|Rutherford|Y|29|5|1948|SOLOMON ISLANDS||Dorcas.Rutherford@Vg.org|2452497| +7914|AAAAAAAAKOOBAAAA|602299|6759|4066|2450604|2450574|Ms.|Pauline|Ward|N|18|9|1932|EL SALVADOR||Pauline.Ward@aUrpfgAq2pasS.com|2452494| +7915|AAAAAAAALOOBAAAA|1011500|5729|29958|2449317|2449287|Mr.|Louis|Watson|Y|7|12|1986|HONDURAS||Louis.Watson@4Iu66AKN.com|2452423| +7916|AAAAAAAAMOOBAAAA|507570|4544|2271|2451593|2451563|Miss|Flora|Angel|N|20|10|1975|HONG KONG||Flora.Angel@XiCyMcaE8yp.org|2452528| +7917|AAAAAAAANOOBAAAA|28706|5457|23761|2449056|2449026|Dr.|Kyle|Friedman|Y|3|2|1980|BOLIVIA||Kyle.Friedman@jbYYC4X5Sr6Q.com|2452614| +7918|AAAAAAAAOOOBAAAA|490640|3711|4382|2452099|2452069|Sir|John|Mcfadden|N|21|4|1963|YEMEN||John.Mcfadden@tbCnogacm5UaNyf.com|2452453| +7919|AAAAAAAAPOOBAAAA|879687|4147|18207|2450057|2450027|Mr.|Jason|Foster|Y|14|11|1965|ICELAND||Jason.Foster@FENu0j2jnIxtAXGNL7S.edu|2452372| +7920|AAAAAAAAAPOBAAAA|1554459|5497|23735|2450225|2450195|Sir|Lawrence|Benton|N|19|10|1969|LEBANON||Lawrence.Benton@8vJxYx.com|2452606| +7921|AAAAAAAABPOBAAAA|916227|1985|10143|2449835|2449805|Dr.|Charles|Fong|Y|21|4|1927|LESOTHO||Charles.Fong@u7L1PV.org|2452587| +7922|AAAAAAAACPOBAAAA|989582|2153|3171|2450104|2450074|Ms.|Carolyn|Witte|N|13|10|1959|AZERBAIJAN||Carolyn.Witte@suR.org|2452289| +7923|AAAAAAAADPOBAAAA|1131948|5931|48267|2450434|2450404|Dr.|Sarah|Murray|N|30|4|1964|EQUATORIAL GUINEA||Sarah.Murray@hg.edu|2452336| +7924|AAAAAAAAEPOBAAAA|584918|6832|34346|2449908|2449878|Mr.|Thomas|Whitehurst|Y|8|5|1955|PANAMA||Thomas.Whitehurst@pm15Rm04a3ZsTNJr.com|2452626| +7925|AAAAAAAAFPOBAAAA|1451351||18893|||Dr.|||||4|1970|QATAR||Elvira.Gardner@Ei1aNS9ov0.org|2452502| +7926|AAAAAAAAGPOBAAAA|1523472|3108|17144|2450616|2450586|Dr.|Donald|Williams|Y|19|10|1944|KAZAKHSTAN||Donald.Williams@ai74XxhMFEGk4U.org|2452547| +7927|AAAAAAAAHPOBAAAA|1810798|4781|19211|2449334|2449304|Mrs.|Elmira|Blackwell|Y|29|6|1976|PERU||Elmira.Blackwell@VI8Ol4UTlUUAQIkF.com|2452601| +7928|AAAAAAAAIPOBAAAA|1920463|7168|7473|2450804|2450774|Miss|Diane|Zavala|N|26|7|1987|PUERTO RICO||Diane.Zavala@PJ.edu|2452466| +7929|AAAAAAAAJPOBAAAA|692989|2880|19618|2452045|2452015|Sir|Richard|Peck|N|29|7|1946|VIET NAM||Richard.Peck@s9f3O7VAtaIDjJA.com|2452329| +7930|AAAAAAAAKPOBAAAA|1173930|484|5485|2450202|2450172|Sir|Robert|Shepard|Y|16|12|1925|MAYOTTE||Robert.Shepard@DA9fNsd.edu|2452456| +7931|AAAAAAAALPOBAAAA|1914122|1738|3401|2450536|2450506|Miss|Kristie|Judd|N|15|5|1966|SOLOMON ISLANDS||Kristie.Judd@Z72KxrEOIM6E8C.edu|2452491| +7932|AAAAAAAAMPOBAAAA|919517|1797|20524|2450742|2450712|Ms.|Joyce|Mueller|Y|15|9|1974|VIET NAM||Joyce.Mueller@g5F.com|2452624| +7933|AAAAAAAANPOBAAAA|1422689|3906|13876|2451348|2451318|Mr.|Gregory|Hamilton|N|12|10|1977|LESOTHO||Gregory.Hamilton@rYY1pOH84Bn3XlXiq.org|2452349| +7934|AAAAAAAAOPOBAAAA|28872|692|31712|2451126|2451096|Ms.|Lucy|Herrera|Y|15|3|1958|NETHERLANDS||Lucy.Herrera@2CpqTPh11.com|2452511| +7935|AAAAAAAAPPOBAAAA|29608|6028|8401|2450772|2450742|Dr.|Scott|Martinez|Y|19|12|1979|NORFOLK ISLAND||Scott.Martinez@YJ.com|2452443| +7936|AAAAAAAAAAPBAAAA|718892|4711|17730|2450246|2450216|Miss|Nicole|Gonzales|N|23|1|1928|CYPRUS||Nicole.Gonzales@XIBK5mcgs2TRu.org|2452492| +7937|AAAAAAAABAPBAAAA|1040478|6264|46803|2449415|2449385|Sir|Keith|Echols|Y|1|7|1964|TURKMENISTAN||Keith.Echols@pPMeFUGEDHgtl.org|2452354| +7938|AAAAAAAACAPBAAAA|1892854|6043|37679|2449855|2449825|Ms.|Marianne|Fisher|Y|1|10|1966|ALBANIA||Marianne.Fisher@U4a1ndfE6x6RM.com|2452420| +7939|AAAAAAAADAPBAAAA|354869|3966|3063|2450551|2450521|Dr.|Charlene|Dorsey|Y|11|5|1990|HONDURAS||Charlene.Dorsey@DhCTMPPRta3N.edu|2452454| +7940|AAAAAAAAEAPBAAAA|47226|6322|35771|2451162|2451132|Ms.|Christine|Allen|Y|9|11|1945|GUINEA-BISSAU||Christine.Allen@AuLLF.org|2452346| +7941|AAAAAAAAFAPBAAAA|81330|4036|49942|2451957|2451927|Dr.|Geneva|Parker|Y|14|4|1958|UGANDA||Geneva.Parker@1FBOk.org|2452490| +7942|AAAAAAAAGAPBAAAA|1862036|2378|44273|2451992|2451962|Sir|Dennis|Swift|N|6|10|1933|MOROCCO||Dennis.Swift@heUmmnr8mY9fToq6.com|2452629| +7943|AAAAAAAAHAPBAAAA|37087|6073|40212|2451306|2451276|Sir|Burton|Webb|N|28|1|1944|SAINT LUCIA||Burton.Webb@e18y7.org|2452561| +7944|AAAAAAAAIAPBAAAA|1262119|2209|45852|2450497|2450467|Mr.|Earle|Pitts|Y|11|7|1971|VENEZUELA||Earle.Pitts@LknCNvihGkpx.org|2452459| +7945|AAAAAAAAJAPBAAAA|1076920|291|35662|2450860|2450830|Sir|Ernesto|Simon|N|22|6|1934|MALI||Ernesto.Simon@PiTHgxiZ.org|2452586| +7946|AAAAAAAAKAPBAAAA|1864304|311|26271|2451247|2451217|Mrs.|Lori|Lee|Y|9|8|1976|SINGAPORE||Lori.Lee@LlFzaOXCeAk5OrqZ.edu|2452534| +7947|AAAAAAAALAPBAAAA|1341758|1232|44659|2451544|2451514|Miss|Amanda|Couch|N|28|8|1925|SERBIA||Amanda.Couch@UrhKC7c.com|2452440| +7948|AAAAAAAAMAPBAAAA|514653|4735|10464|2452485|2452455|Dr.|Timothy|Knott|N|11|2|1951|LIBERIA||Timothy.Knott@xBVCqj9cgEGHO.org|2452504| +7949|AAAAAAAANAPBAAAA|84776|1859|21695|2451220|2451190|Dr.|Donald|Mckinney|Y|11|12|1967|JORDAN||Donald.Mckinney@TKgEjDbO04.com|2452598| +7950|AAAAAAAAOAPBAAAA|1069331|1382|14186|2449579|2449549|Dr.|William|Handy|N|30|7|1953|NAMIBIA||William.Handy@KEp9uc9LrEF370.org|2452561| +7951|AAAAAAAAPAPBAAAA|717810|832|12607|2451520|2451490|Dr.|Louise|Smith|Y|21|5|1974|MARTINIQUE||Louise.Smith@h.edu|2452438| +7952|AAAAAAAAABPBAAAA|621309|1496|27283|2450676|2450646|Mr.|Juan|Connor|N|20|7|1929|NIUE||Juan.Connor@jgLO3Y5QhaNxl.org|2452342| +7953|AAAAAAAABBPBAAAA|147163||30655|2451314|2451284|Ms.|Amber|||12|11|1939|LESOTHO|||2452492| +7954|AAAAAAAACBPBAAAA|1244913|943|15326|2449853|2449823|Mr.|Greg|Rhodes|Y|14|7|1949|COSTA RICA||Greg.Rhodes@Sf.org|2452562| +7955|AAAAAAAADBPBAAAA|1135301|4794|24334|2452281|2452251|Ms.|Georgette|Holloway|N|5|3|1971|MALTA||Georgette.Holloway@rlBBxcbIs9dy.com|2452623| +7956|AAAAAAAAEBPBAAAA|349680|724|7321|2450137|2450107|Mr.|Richard|Guthrie|Y|4|11|1939|SIERRA LEONE||Richard.Guthrie@bASopQC7UeKG.com|2452373| +7957|AAAAAAAAFBPBAAAA|1524987|6848|23956|2451826|2451796|Dr.|Edward|Jenkins|Y|14|9|1965|ERITREA||Edward.Jenkins@q82vpiG.org|2452550| +7958|AAAAAAAAGBPBAAAA|||48065|||Ms.|Joyce|Kennedy|Y|15|||BENIN||Joyce.Kennedy@zT2DL4546v.com|2452575| +7959|AAAAAAAAHBPBAAAA|1500553|891|27182|2452104|2452074|Dr.|George|Thomas|N|27|2|1925|GREENLAND||George.Thomas@nhbbXSa.org|2452474| +7960|AAAAAAAAIBPBAAAA|1599976|4309|242|2451935|2451905|Dr.|Alejandro|Fierro|Y|7|2|1963|SYRIAN ARAB REPUBLIC||Alejandro.Fierro@N4MegUmf0jtYR.org|2452463| +7961|AAAAAAAAJBPBAAAA|1400333|6770|15207|2452172|2452142|Mr.|Carlton|Daugherty|Y|1|12|1985|CHINA||Carlton.Daugherty@xdFsIt3B.com|2452351| +7962|AAAAAAAAKBPBAAAA|719257|2494|23331|2451163|2451133|Sir|||||||UNITED KINGDOM||John.Rowell@Krnlf6s.com|| +7963|AAAAAAAALBPBAAAA|1587112|5274|37436|2452324|2452294|Mr.|John|Raymond|Y|22|6|1947|SINGAPORE||John.Raymond@aH.com|2452345| +7964|AAAAAAAAMBPBAAAA|1364108|6809|47910|2452124|2452094|Mr.|Percy|Davis|Y|14|11|1980|PUERTO RICO||Percy.Davis@IDXcDEn3A8zOINc4ju.edu|2452546| +7965|AAAAAAAANBPBAAAA|1179500|2300|15500|2451818|2451788|Ms.|Kaci|Mcgowan|Y|15|9|1926|SAUDI ARABIA||Kaci.Mcgowan@8c8DdHJ4.org|2452376| +7966|AAAAAAAAOBPBAAAA|689983|3837|39338|2450757|2450727|Dr.|Matthew|Sullivan|Y|24|11|1992|GUATEMALA||Matthew.Sullivan@X.edu|2452318| +7967|AAAAAAAAPBPBAAAA||2968|31531|||||Dickens|||8|||||| +7968|AAAAAAAAACPBAAAA|179816|1476|42736|2451873|2451843|Mr.|Robert|Russell|Y|4|2|1927|GUINEA-BISSAU||Robert.Russell@z7j.org|2452539| +7969|AAAAAAAABCPBAAAA|516114|4174|11834|2451095|2451065|Dr.|David|Muncy|Y|18|2|1961|BOLIVIA||David.Muncy@0JeVIDeCEqj.com|2452320| +7970|AAAAAAAACCPBAAAA||6309|44620|2449773|2449743|Miss||||29|6||MOROCCO|||| +7971|AAAAAAAADCPBAAAA|1360236|542|16048|2451009|2450979|Dr.|James|Healey|N|1|2|1972|BAHRAIN||James.Healey@2bIR4LbJJsDpVQ51.org|2452506| +7972|AAAAAAAAECPBAAAA|1392474|5261|14001|2449550|2449520|Mr.|George|Arnold|Y|18|5|1950|SLOVAKIA||George.Arnold@j.edu|2452563| +7973|AAAAAAAAFCPBAAAA|1808497|6470|31596|2450580|2450550|Dr.|Timothy|Webster|N|17|3|1948|NIUE||Timothy.Webster@26fGY0SN0vv9Xt81m.org|2452558| +7974|AAAAAAAAGCPBAAAA|101826|4118|35210|2450152|2450122|Ms.|Shayna|Best|Y|30|1|1939|MYANMAR||Shayna.Best@9CQ0h6LXOy.edu|2452630| +7975|AAAAAAAAHCPBAAAA|884555|3752|1302|||Mrs.|Gwen|||||1973||||2452539| +7976|AAAAAAAAICPBAAAA|1510543|19|23942|2451940|2451910|Miss|Teri|Stanley|N|30|12|1983|CUBA||Teri.Stanley@GoI6tJKHuQN8Zf.org|2452519| +7977|AAAAAAAAJCPBAAAA|999434|6336|28085|2452332|2452302|Mrs.|Marjorie|Sullivan|Y|6|10|1954|BARBADOS||Marjorie.Sullivan@t6aAZ.com|2452423| +7978|AAAAAAAAKCPBAAAA|913617|4706|1042|2452673|2452643|Mrs.|Delia|Garland|Y|27|2|1974|RUSSIAN FEDERATION||Delia.Garland@8x.com|2452358| +7979|AAAAAAAALCPBAAAA|938675|340|28405|2450909|2450879|Dr.|Margaret|Baker|Y|5|6|1962|EGYPT||Margaret.Baker@bG4PBfy6Kz.edu|2452520| +7980|AAAAAAAAMCPBAAAA|165765|1638|25158|2449665|||Jacquline||Y|15|||SYRIAN ARAB REPUBLIC|||| +7981|AAAAAAAANCPBAAAA|6173|5|17837|2449212|2449182|Mr.|Edward|Dunn|N|31|5|1968|SOMALIA||Edward.Dunn@ZTPVxBTvatJLO0.edu|2452524| +7982|AAAAAAAAOCPBAAAA|1658204|2713|9975|2451343|2451313|Mr.|Ralph|Dailey|N|19|1|1956|AFGHANISTAN||Ralph.Dailey@qponjNPMag8SG.org|2452625| +7983|AAAAAAAAPCPBAAAA|679952|6035|27271|2452364|2452334|Mr.|Michael|Bingham|Y|8|3|1963|BARBADOS||Michael.Bingham@E.org|2452641| +7984|AAAAAAAAADPBAAAA|830723|4976|36476|2449544|2449514|Mrs.|Lori|Thomas|Y|25|4|1950|GUATEMALA||Lori.Thomas@cP78z8BpVfayBP45j.com|2452324| +7985|AAAAAAAABDPBAAAA|1606433|633|33280|2450205|2450175|Ms.|Lacey|Collins|Y|19|6|1925|ANTARCTICA||Lacey.Collins@F5C6.com|2452426| +7986|AAAAAAAACDPBAAAA|1713547||40515|2450108||Mr.|Jeffery|Pepper||||1949|KOREA, REPUBLIC OF|||2452408| +7987|AAAAAAAADDPBAAAA|149151|6532|43513|2450180|2450150|Mrs.|Marva|Beaudoin|Y|20|8|1958|AZERBAIJAN||Marva.Beaudoin@SGR2keBbMJ8.edu|2452464| +7988|AAAAAAAAEDPBAAAA|786325|4580|835|2450050|2450020|Mrs.|Brenda|Hess|N|29|9|1931|PITCAIRN||Brenda.Hess@ghR5TDXiB2lD.com|2452365| +7989|AAAAAAAAFDPBAAAA|7228|5348|22221|2452425|2452395|Dr.|Cynthia|Bowie|N|7|10|1941|MARSHALL ISLANDS||Cynthia.Bowie@2aycfZMXJV.org|2452397| +7990|AAAAAAAAGDPBAAAA|691471|6862|11853|2452382|2452352|Dr.|Damien|Harman|N|11|2|1948|NIUE||Damien.Harman@EYS.com|2452438| +7991|AAAAAAAAHDPBAAAA|1186342|7150|25605|2450253|2450223|Sir|Merlin|Martin|N|15|9|1925|INDONESIA||Merlin.Martin@V03tl6qBC.edu|2452371| +7992|AAAAAAAAIDPBAAAA|66860|5551|45489|2451822|2451792|Mr.|William|Young|N|26|11|1964|HONG KONG||William.Young@eu47D3ByzNGu.com|2452558| +7993|AAAAAAAAJDPBAAAA|1541467|5489|31911|2450641|2450611|Miss|Patti|Grant|Y|22|5|1972|BELIZE||Patti.Grant@Q24inTF.org|2452504| +7994|AAAAAAAAKDPBAAAA|868576|4661|39593|2451316|2451286|Mrs.|Catalina|Ingram|Y|12|4|1968|BELIZE||Catalina.Ingram@t5ifuHFjtoDD.edu|2452327| +7995|AAAAAAAALDPBAAAA|1774267|2183|35699|2449060|2449030|Ms.|Teresa|Gomez|N|21|9|1960|ANTARCTICA||Teresa.Gomez@LB7jDxX8AgXV.edu|2452480| +7996|AAAAAAAAMDPBAAAA|67051|948|18045|2450053|2450023|Sir|Bryan|Hernandez|N|20|8|1948|ANTARCTICA||Bryan.Hernandez@Xa3.edu|2452422| +7997|AAAAAAAANDPBAAAA|927526|6344|42409|2450297|2450267|Sir|Pat|Foster|Y|27|4|1959|ANGUILLA||Pat.Foster@5ZPB6J2JXpZYi.com|2452329| +7998|AAAAAAAAODPBAAAA|1465594|3782|35123|2449147|2449117|Dr.|Rosetta|Robinson|Y|22|2|1935|HUNGARY||Rosetta.Robinson@35I3UQnjQ.edu|2452552| +7999|AAAAAAAAPDPBAAAA|1880040|2870|39347|2451023|2450993|Mrs.|Cheryl|Chavez|Y|21|2|1967|GERMANY||Cheryl.Chavez@PBXzhbMDbX.edu|2452334| +8000|AAAAAAAAAEPBAAAA|363397|3502|5607|2449283|2449253|Mrs.|Helen|Frank|Y|17|11|1953|TURKEY||Helen.Frank@dqud.com|2452618| +8001|AAAAAAAABEPBAAAA|97534|3842|41797|2450718|2450688|Ms.|Angelina|Rivera|Y|28|1|1963|PARAGUAY||Angelina.Rivera@mQYO9aGm9.org|2452558| +8002|AAAAAAAACEPBAAAA|683455|4732|30856|2451828|2451798|Ms.|Reyna|Mahoney|Y|24|5|1928|IRAQ||Reyna.Mahoney@AN0aqy.org|2452444| +8003|AAAAAAAADEPBAAAA|1510173|5504|35235|2451963|2451933|Dr.|John|Mott|N|28|3|1985|FRENCH POLYNESIA||John.Mott@tozFGLrVxQDs.com|2452360| +8004|AAAAAAAAEEPBAAAA|131084|3519|16354|2451337|2451307|Ms.|Vera|Oreilly|N|29|11|1959|TOGO||Vera.Oreilly@EnCA8Ks.com|2452496| +8005|AAAAAAAAFEPBAAAA|30050|5024|6845|2452085|2452055|Miss|Diane|Carpenter|Y|12|4|1944|LITHUANIA||Diane.Carpenter@NdvHgvVR6d1DZO.edu|2452376| +8006|AAAAAAAAGEPBAAAA|||23691|2450534|2450504||Joshua|Jones|||8||||Joshua.Jones@cRBGGKaKI1nrkG.com|2452338| +8007|AAAAAAAAHEPBAAAA|58801|1318|10282|2450736|2450706|Dr.|Pamela|Roddy|Y|20|11|1984|FAROE ISLANDS||Pamela.Roddy@ZRzT.org|2452292| +8008|AAAAAAAAIEPBAAAA|343774|3687|23553|2449110|2449080|Dr.|Nicholas|Bunting|N|7|10|1943|ARMENIA||Nicholas.Bunting@idQGk4ZSjraZVLs.com|2452411| +8009|AAAAAAAAJEPBAAAA|397035|3844|43853|2450713|2450683|Dr.|Margie|Beatty|Y|10|6|1991|ARUBA||Margie.Beatty@JM8vDiHXBS5.edu|2452523| +8010|AAAAAAAAKEPBAAAA|1024310|5535|27113|2452439|2452409|Mr.|Gary|Bennett|N|26|8|1939|ETHIOPIA||Gary.Bennett@EYRLa4bF3EY.org|2452600| +8011|AAAAAAAALEPBAAAA|1657441|1752|1310|2452383|2452353|Dr.|Nick|Farley|Y|9|9|1960|NIGERIA||Nick.Farley@8nAg.com|2452400| +8012|AAAAAAAAMEPBAAAA|474048|311|18127|2451907|2451877|Miss|Heather|Dillard|Y|12|8|1939|INDONESIA||Heather.Dillard@i8qQH0sG5zUUC.org|2452396| +8013|AAAAAAAANEPBAAAA|1577333|3245|8404|2451815|2451785|Dr.|Donald|Brower|N|29|11|1963|TOKELAU||Donald.Brower@NEEbchZA.com|2452615| +8014|AAAAAAAAOEPBAAAA|1173000|1274|25548|2450932|2450902|Miss|Judy|Vasquez|Y|26|1|1975|SAMOA||Judy.Vasquez@s.org|2452288| +8015|AAAAAAAAPEPBAAAA|356999|1787|15022|2449611|2449581|Sir|Robert|Kane|Y|17|1|1953|SENEGAL||Robert.Kane@BKBm.com|2452286| +8016|AAAAAAAAAFPBAAAA|622419|4640|41614|2449624|2449594|Miss|Julia|Johnson|Y|22|2|1961|ROMANIA||Julia.Johnson@7fBxKDuJXPlV3m.edu|2452290| +8017|AAAAAAAABFPBAAAA|703355|6569|26183|2449299|2449269|Dr.|Beth|Sanches|Y|11|12|1929|CHINA||Beth.Sanches@yhhagS7RXZ9.edu|2452366| +8018|AAAAAAAACFPBAAAA|238407|639|43671|2452474|2452444|Dr.|Dexter|Robson|Y|12|5|1969|GUERNSEY||Dexter.Robson@nksMuGK2H4ytpQkI.edu|2452327| +8019|AAAAAAAADFPBAAAA|138241|4779|2964|2450861|2450831|Mr.|Norman|Green|N|5|10|1987|AMERICAN SAMOA||Norman.Green@gQ4I29Lt.edu|2452392| +8020|AAAAAAAAEFPBAAAA|1034535|2097|33612|2452653|2452623|Dr.|Joshua|Cullen|Y|2|12|1925|MACAO||Joshua.Cullen@5beeSGdz6Oq.com|2452355| +8021|AAAAAAAAFFPBAAAA|1261577|6772|37920|2451200|2451170|Mr.|Roger|Roy|N|16|7|1977|HAITI||Roger.Roy@laITneNFcpc2.org|2452588| +8022|AAAAAAAAGFPBAAAA|887206|3161|21916|2449884|2449854|Miss|Barbara|Lee|Y|11|6|1954|SINGAPORE||Barbara.Lee@nU9iIDBkvqt.edu|2452573| +8023|AAAAAAAAHFPBAAAA|902256|1145|10034|2452569|2452539|Mr.|Emmanuel|Lucas|N|25|12|1943|FRENCH POLYNESIA||Emmanuel.Lucas@xDSR7lDdL7pOrE.org|2452489| +8024|AAAAAAAAIFPBAAAA|1845166|2435|2904|2452317|2452287|Sir|Walton|Mcdonald|Y|8|8|1953|MALTA||Walton.Mcdonald@2G1H.edu|2452442| +8025|AAAAAAAAJFPBAAAA|1920155|6539|16123|2449033|2449003|Mrs.|Annette|Carver|N|13|6|1932|FINLAND||Annette.Carver@v1XNNct.edu|2452368| +8026|AAAAAAAAKFPBAAAA|1036620|2764|15755|2450610|2450580|Sir|Larry|Pipkin|N|4|2|1992|HONG KONG||Larry.Pipkin@jD7aDyzJ5zDjC.org|2452434| +8027|AAAAAAAALFPBAAAA|1780508|1509|26957|2452458|2452428|Sir|James|Lawhorn|N|25|9|1978|MYANMAR||James.Lawhorn@S4aB.org|2452429| +8028|AAAAAAAAMFPBAAAA|825267|1632|11997|2450215|2450185|Sir|Eric|Pineda|N|11|6|1947|NICARAGUA||Eric.Pineda@XK0eXmSu2TbLxpjuD.org|2452452| +8029|AAAAAAAANFPBAAAA|711880|3360|41993|2450525|2450495|Miss|Allen|Bryant|N|2|6|1960|CAPE VERDE||Allen.Bryant@1r0pVvVmFtlj.org|2452466| +8030|AAAAAAAAOFPBAAAA|1584962|3713|48911|2451778|2451748|Sir|Ray|Bennett|N|31|3|1934|PUERTO RICO||Ray.Bennett@V4SS5FJQiulJ.com|2452583| +8031|AAAAAAAAPFPBAAAA|1759302|2191|39080|2452375|2452345|Ms.|Michele|Livingston|N|23|10|1937|MEXICO||Michele.Livingston@5iG52z.org|2452351| +8032|AAAAAAAAAGPBAAAA|424719|2498|19077|2452502|2452472|Dr.|James|Lantz|N|19|2|1959|MAURITIUS||James.Lantz@bajfV0l.org|2452601| +8033|AAAAAAAABGPBAAAA|105404|2072|15940|2452101|2452071|Miss|Eileen|Christopher|Y|24|7|1977|SIERRA LEONE||Eileen.Christopher@YkHfaU7HaSVvCb5.org|2452408| +8034|AAAAAAAACGPBAAAA|46761|1134|754|2451740|2451710|Sir|Robert|Pedigo|N|20|5|1987|SIERRA LEONE||Robert.Pedigo@Ri.edu|2452647| +8035|AAAAAAAADGPBAAAA|1000075|6457|28392|2451360|2451330|Mr.|Jordan|Miller|Y|5|3|1957|MONACO||Jordan.Miller@c8Kve56SKVc.org|2452555| +8036|AAAAAAAAEGPBAAAA|44723|2180|40037|2451502|2451472|Ms.|Rochelle|Collins|Y|17|4|1933|GUYANA||Rochelle.Collins@tQv.com|2452417| +8037|AAAAAAAAFGPBAAAA|699602|1144|5039|2451040|2451010|Sir|David|Spicer|N|20|5|1933|BARBADOS||David.Spicer@32Yr1Tm3i27Aq.edu|2452372| +8038|AAAAAAAAGGPBAAAA|||36077|||Dr.|||N||3||||Doris.Peeples@qftRLXT2.org|2452527| +8039|AAAAAAAAHGPBAAAA|1330110|5168|24626|2451297|2451267|Dr.|Kathleen|Adkins|N|18|1|1977|THAILAND||Kathleen.Adkins@hFpe0cT2.com|2452310| +8040|AAAAAAAAIGPBAAAA|100813|2573|7133|2452461|2452431|Ms.|Sarah|Worley|N|8|10|1991|ANGUILLA||Sarah.Worley@0G6EqXFcvJ.com|2452363| +8041|AAAAAAAAJGPBAAAA|153086|1363|6887|2449648|2449618|Dr.|Devin|Williams|N|15|8|1942|MALTA||Devin.Williams@uzMNm.org|2452389| +8042|AAAAAAAAKGPBAAAA|536760|4450|48940|2450019|2449989|Miss|Aaron|Lawson|Y|4|3|1980|CANADA||Aaron.Lawson@Ctc.edu|2452526| +8043|AAAAAAAALGPBAAAA|816504|6935|48519|2451413|2451383|Mr.|Roy|Fairley|Y|19|12|1951|TIMOR-LESTE||Roy.Fairley@CLvt.edu|2452322| +8044|AAAAAAAAMGPBAAAA|1294728|601|28874|2450304|2450274|Dr.|Douglas|Valdez|N|23|6|1954|SINGAPORE||Douglas.Valdez@rZHJcxqMKm1J.com|2452336| +8045|AAAAAAAANGPBAAAA|395711|5441|30818|2449849|2449819|Ms.|Alex|Keegan|Y|12|10|1939|KENYA||Alex.Keegan@imcE9A7BkVlcO.edu|2452335| +8046|AAAAAAAAOGPBAAAA|80701|2206|32558|2452635|2452605|Dr.|Amy|Edwards|N|22|3|1939|WESTERN SAHARA||Amy.Edwards@I5axCBrQ.com|2452596| +8047|AAAAAAAAPGPBAAAA|1394203|6411|8172|2449868|2449838|Dr.|Martin|Delgado|Y|22|9|1960|BRUNEI DARUSSALAM||Martin.Delgado@6HQm.org|2452583| +8048|AAAAAAAAAHPBAAAA||3817|38663|2450210|||Barbara||||6|1956|||Barbara.Corbett@QzyhShrAyio.org|| +8049|AAAAAAAABHPBAAAA|1325277|5988|40699|2452627|2452597|Dr.|Ann|Greene|Y|27|11|1954|TUNISIA||Ann.Greene@BxJxmud4hTuh.edu|2452594| +8050|AAAAAAAACHPBAAAA|227100|885|3070|2451314|2451284|Dr.|Celeste|Torres|Y|26|12|1975|NEPAL||Celeste.Torres@JYd8Vflgiungl.edu|2452558| +8051|AAAAAAAADHPBAAAA|498897|5276|29208|2452025|2451995|Ms.|Florence|Chan|N|3|3|1968|GUYANA||Florence.Chan@3amnkKTKM06.com|2452349| +8052|AAAAAAAAEHPBAAAA|323119|5120|36674|2450750|2450720|Dr.|Paula|Bearden|Y|24|9|1971|LIBERIA||Paula.Bearden@mAXCpTPtVqk.org|2452518| +8053|AAAAAAAAFHPBAAAA|1402167|4453|11586|2451391|2451361|Ms.|Jeannine|Harms|N|10|6|1941|FIJI||Jeannine.Harms@qJ5SlYMS.edu|2452517| +8054|AAAAAAAAGHPBAAAA|403391|4278|13918|2449787|2449757|Ms.|Judith|Shepherd|Y|16|7|1978|NIGER||Judith.Shepherd@odNaArafQiEht16.com|2452637| +8055|AAAAAAAAHHPBAAAA|110323|1555|41187|2450581|2450551|Dr.|Clifton|Martinez|Y|5|4|1924|KIRIBATI||Clifton.Martinez@7f7maS5LUCGpS8.com|2452391| +8056|AAAAAAAAIHPBAAAA|622321|1424|46174|2450684|2450654|Mr.|Norman|Crenshaw|Y|6|8|1980|FINLAND||Norman.Crenshaw@Gg6vOLm4jRdEYNg.org|2452309| +8057|AAAAAAAAJHPBAAAA|292184|3892|48753|2450697|2450667|Mr.|David|Jones|Y|8|1|1938|LUXEMBOURG||David.Jones@aD2zKsD0VbX.org|2452463| +8058|AAAAAAAAKHPBAAAA|694340|6236|42132|2449763|2449733|Sir|Scott|Jones|Y|7|5|1981|KENYA||Scott.Jones@T6xR0pTFpAl0l.org|2452564| +8059|AAAAAAAALHPBAAAA|1354548|3566|12184|2451744|2451714|Dr.|Ellen|Watson|Y|13|3|1941|UNITED ARAB EMIRATES||Ellen.Watson@4tTNAc3Zo.com|2452489| +8060|AAAAAAAAMHPBAAAA|885147|2528|7509|2451886|2451856|Ms.|Marsha|Spence|Y|26|6|1958|SRI LANKA||Marsha.Spence@LpUSI.org|2452331| +8061|AAAAAAAANHPBAAAA|16211|2774|31690|2451263|2451233|Ms.|Cheryl|Webb|Y|3|4|1989|HONG KONG||Cheryl.Webb@eQZL5TH.edu|2452485| +8062|AAAAAAAAOHPBAAAA|1757653|4239|44378|2450952|2450922|Dr.|Fernando|Martinez|N|19|3|1974|ARMENIA||Fernando.Martinez@ERNX.org|2452312| +8063|AAAAAAAAPHPBAAAA|314219|3851|39483|2451153|2451123|Ms.|Kimberly|Higginbotham|N|22|9|1926|SUDAN||Kimberly.Higginbotham@RObF44ZNZopDf.com|2452305| +8064|AAAAAAAAAIPBAAAA|1238606|6376|36146|2450138|2450108|Mrs.|Eva|Rodriguez|N|4|8|1934|ZIMBABWE||Eva.Rodriguez@0PmgYPBi.com|2452295| +8065|AAAAAAAABIPBAAAA|1588317|5116|36362|2449213|2449183|Mr.|Douglas|Law|N|19|2|1985|SENEGAL||Douglas.Law@yhtfiB.org|2452442| +8066|AAAAAAAACIPBAAAA|40539|2598|26257|2450120|2450090|Mrs.|Norma|Dyer|N|6|2|1963|ANDORRA||Norma.Dyer@fcCCI.com|2452544| +8067|AAAAAAAADIPBAAAA|903264|2383|29149|2450448|2450418|Dr.|Crystal|Ortiz|N|11|10|1972|MONACO||Crystal.Ortiz@PZPiuAfS.com|2452518| +8068|AAAAAAAAEIPBAAAA|1499422|2409|13796|2450119|2450089|Mr.|Dwight|Smith|Y|21|7|1980|ALGERIA||Dwight.Smith@TjAxoOP5s.edu|2452431| +8069|AAAAAAAAFIPBAAAA|1372496|398|38347|2452528|2452498|Ms.|Marsha|Tucker|N|15|1|1945|NETHERLANDS||Marsha.Tucker@u.org|2452607| +8070|AAAAAAAAGIPBAAAA|803990|3005|20067|2449600|2449570|Dr.|Dorthy|Terry|N|22|8|1944|INDIA||Dorthy.Terry@83rFqt8YlIN.com|2452644| +8071|AAAAAAAAHIPBAAAA|1817258|4375|292|2450084|2450054|Mrs.|Ivette|Turner|Y|28|4|1967|SLOVENIA||Ivette.Turner@F87VJJUE.edu|2452393| +8072|AAAAAAAAIIPBAAAA|||9902||||Scott|Williams||20|4|||||| +8073|AAAAAAAAJIPBAAAA|235716|2088|33660|2452071|2452041|Mrs.|Andrew|Rosario|N|3|2|1952|BELIZE||Andrew.Rosario@I5O0BuUoGo7.com|2452552| +8074|AAAAAAAAKIPBAAAA|921105|6911|28364|2449436|2449406|Dr.|Victor|Wilson|N|9|4|1946|FRENCH POLYNESIA||Victor.Wilson@jlfnbzfeA.edu|2452356| +8075|AAAAAAAALIPBAAAA|844978|5882|36833|2449301|2449271|Mr.|Paul|Robinson|N|13|7|1933|QATAR||Paul.Robinson@ehdLF7NaG0Im.org|2452641| +8076|AAAAAAAAMIPBAAAA|462955|3993|9831|2450140|2450110|Sir|Manuel|Rodriguez|Y|25|7|1928|MALDIVES||Manuel.Rodriguez@iFeCKCngcxpD.edu|2452363| +8077|AAAAAAAANIPBAAAA|1168602|5129|33703|2451838|2451808|Mr.|Joshua|Marshall|N|4|9|1980|PANAMA||Joshua.Marshall@M8fs1ktGnzB.edu|2452573| +8078|AAAAAAAAOIPBAAAA|735692|64|28789|2449233|2449203|Sir|Michael|Andres|N|5|12|1964|CAYMAN ISLANDS||Michael.Andres@4llLtvYZ.org|2452291| +8079|AAAAAAAAPIPBAAAA|1331814|6336|40011|2452233|2452203|Ms.|Kathleen|Cordova|N|26|12|1928|BOLIVIA||Kathleen.Cordova@00CK4K9RG4b.edu|2452543| +8080|AAAAAAAAAJPBAAAA|822873||41076|2451101|2451071||||N|4|12|1924||||| +8081|AAAAAAAABJPBAAAA|373493|6456|15372|2450528|2450498|Ms.|Sarah|Zuniga|N|10|4|1951|COMOROS||Sarah.Zuniga@XmBKDm6O3F7j.com|2452296| +8082|AAAAAAAACJPBAAAA|1541712|5260|17608|2449325|2449295|Mr.|Robert|Thompson|Y|4|9|1988|TUVALU||Robert.Thompson@iRaUKlqv588XGox.edu|2452545| +8083|AAAAAAAADJPBAAAA|280836|1420|10004|2452529|2452499|Sir|Gary|Roe|Y|26|11|1925|AZERBAIJAN||Gary.Roe@7Vqe98HfzRf.edu|2452376| +8084|AAAAAAAAEJPBAAAA|1597857|3270|18330|2450662|2450632|Miss|Belinda|Webster|N|9|11|1968|ISLE OF MAN||Belinda.Webster@iv7CebfjvXtL.edu|2452571| +8085|AAAAAAAAFJPBAAAA|961942|4833|18924|2451640|2451610|Mr.|Jeffrey|Bear|N|17|12|1979|CYPRUS||Jeffrey.Bear@sk5Pdk7uEiUmL.edu|2452616| +8086|AAAAAAAAGJPBAAAA|458491|2043|13595|2450380|2450350|Ms.|Nancy|Simmons|N|24|11|1948|GHANA||Nancy.Simmons@ya.org|2452320| +8087|AAAAAAAAHJPBAAAA|1708251|5497|11943|2450190|2450160|Dr.|James|Jefferson|N|21|9|1932|IRAQ||James.Jefferson@zUzF1SK.edu|2452423| +8088|AAAAAAAAIJPBAAAA|1892765|5034|2856|2451461|2451431|Dr.|Jeffery|Lott|N|15|6|1981|ARMENIA||Jeffery.Lott@bA4f8lr.org|2452487| +8089|AAAAAAAAJJPBAAAA|1147939|7014|5242|2450760|2450730|Dr.|Curtis|Ellis|Y|4|11|1928|TOGO||Curtis.Ellis@9dFElIL.edu|2452466| +8090|AAAAAAAAKJPBAAAA|1404255|4537|21219|2450995|2450965|Sir|Harold|Allen|Y|3|7|1958|INDIA||Harold.Allen@NqcE.org|2452481| +8091|AAAAAAAALJPBAAAA|1084020|702|22584|2451949|2451919|Miss|Diane|Chavez|N|24|12|1989|WALLIS AND FUTUNA||Diane.Chavez@e8k.edu|2452628| +8092|AAAAAAAAMJPBAAAA|1908960|4652|43897|2452643|2452613|Dr.|Eugene|Lowman|Y|12|6|1972|VENEZUELA||Eugene.Lowman@yHhqNEcgSzxeFf3DtT.edu|2452510| +8093|AAAAAAAANJPBAAAA|1760384|85|22939|2451930|2451900|Dr.|Kara|Henson|N|28|1|1960|BHUTAN||Kara.Henson@VkLtm0m.com|2452535| +8094|AAAAAAAAOJPBAAAA|759721|2237|15553|2452056|2452026|Miss|Christina|Gray|N|2|5|1939|PALAU||Christina.Gray@7LSjEl7RKy5.edu|2452366| +8095|AAAAAAAAPJPBAAAA|601348|652|31156|2450340|2450310|Mr.|Carrol|Lee|N|12|3|1974|KIRIBATI||Carrol.Lee@CUhxtneSx5C.com|2452489| +8096|AAAAAAAAAKPBAAAA|292754|3310|22313|2449822|2449792|Sir|James|Nelson|Y|18|12|1971|PORTUGAL||James.Nelson@p3CMDZ3Olp1.org|2452311| +8097|AAAAAAAABKPBAAAA|499818|2921|37624|2450808|||Elizabeth||Y|||1959|||Elizabeth.Baker@xpDOuoUI.org|2452639| +8098|AAAAAAAACKPBAAAA|1540421|6691|8161|2451725|2451695|Mrs.|Anthony|Peck|N|12|10|1944|ARGENTINA||Anthony.Peck@FkaxDE9C9qIvR9.com|2452476| +8099|AAAAAAAADKPBAAAA|1094035|53|39671|2450637|2450607|Sir|James|Keller|Y|28|8|1937|MALAYSIA||James.Keller@suDRKZXCa.com|2452437| +8100|AAAAAAAAEKPBAAAA|305209|4071|40980|2451660|2451630|Ms.|Trisha|Ford|N|18|7|1930|BURKINA FASO||Trisha.Ford@YqV.com|2452354| +8101|AAAAAAAAFKPBAAAA|26711|522|41133|2451026|2450996|Sir|Daniel|Garcia|Y|26|7|1974|SUDAN||Daniel.Garcia@E.edu|2452322| +8102|AAAAAAAAGKPBAAAA|815146|1259|20219|2449625|2449595|Sir|Douglas|Taylor|Y|9|8|1924|SEYCHELLES||Douglas.Taylor@ltr1sfKr6Zh.org|2452627| +8103|AAAAAAAAHKPBAAAA|566255|5532|409|2450418|2450388|Dr.|Caroline|Jones|Y|13|11|1970|SAINT LUCIA||Caroline.Jones@02QV.com|2452369| +8104|AAAAAAAAIKPBAAAA|1333993|6388|27183|2451137|2451107|Mr.|Stephen|Vargas|N|17|10|1960|MAYOTTE||Stephen.Vargas@3.edu|2452360| +8105|AAAAAAAAJKPBAAAA|497342|2647|45552|2449139|2449109|Miss|Sally|Arnold|Y|4|9|1990|DENMARK||Sally.Arnold@3Q.com|2452599| +8106|AAAAAAAAKKPBAAAA|1056332|2703|44674|2452321|2452291|Miss|Katherine|Mcgill|Y|9|12|1981|CHILE||Katherine.Mcgill@qzFr74MNvUrH.org|2452303| +8107|AAAAAAAALKPBAAAA|750091|5778|40312|2450865|2450835|Miss|Misty|Bowers|Y|29|3|1932|IRELAND||Misty.Bowers@btOo2sNqhfC3bh.edu|2452419| +8108|AAAAAAAAMKPBAAAA|1219639|1881|40923|2450757|2450727|Dr.|Carole|Comstock|N|5|11|1988|KYRGYZSTAN||Carole.Comstock@44c7HEo10UP.com|2452631| +8109|AAAAAAAANKPBAAAA|1858716|2602|23241|2451470|2451440|Mr.|Paul|Wiseman|Y|20|9|1991|HAITI||Paul.Wiseman@sdllEPYad1.com|2452445| +8110|AAAAAAAAOKPBAAAA|1566020|3778|49936|2452277|2452247|Mrs.|Theresa|Howard|N|20|2|1957|KENYA||Theresa.Howard@U8U.edu|2452499| +8111|AAAAAAAAPKPBAAAA|603479|2601|32570|2450775|2450745|Dr.|John|Nunez|Y|28|5|1951|GAMBIA||John.Nunez@YKmvUeFv80GGFRg.com|2452341| +8112|AAAAAAAAALPBAAAA|1422270|4884|4745|2449756|2449726|Dr.|James|Cope|N|21|2|1977|GREENLAND||James.Cope@qyAl5tjs6xBE7xmu.org|2452377| +8113|AAAAAAAABLPBAAAA|1226025|3279|17780|2451380|2451350|Dr.|Hugh|Miller|N|3|5|1973|JAMAICA||Hugh.Miller@rJ1Prf.org|2452626| +8114|AAAAAAAACLPBAAAA||5541|18760|2451535||Ms.|Flora||||12||SUDAN||Flora.Clark@sRRcDqS5dKtGEg.edu|2452364| +8115|AAAAAAAADLPBAAAA|639862|5732|25555|2449568|2449538|Sir|Michael|Palmer|Y|16|7|1930|ARUBA||Michael.Palmer@UxSKQ5yJ.edu|2452609| +8116|AAAAAAAAELPBAAAA|251295|1468|2888|2449571|2449541|Mr.|Gregory|Ames|N|4|11|1990|TURKEY||Gregory.Ames@NBkd1VoRsCB.edu|2452543| +8117|AAAAAAAAFLPBAAAA|646092|1432|5172|2452411|2452381|Miss|Catherine|Alvarez|Y|28|10|1976|SAN MARINO||Catherine.Alvarez@i7UR6.org|2452496| +8118|AAAAAAAAGLPBAAAA|316582|5039|5143|2449495|2449465|Dr.|Rhonda|Jewell|N|28|2|1926|SOLOMON ISLANDS||Rhonda.Jewell@1kbfUKKlzuxGYC7.org|2452368| +8119|AAAAAAAAHLPBAAAA|728548|2639|40730|2451881|2451851|Mr.|Harry|Davenport|N|6|6|1935|JORDAN||Harry.Davenport@QPe.org|2452563| +8120|AAAAAAAAILPBAAAA|1358939|1441|23896|2452059|2452029|Mr.|Thomas|Peterson|N|2|2|1991|ANGUILLA||Thomas.Peterson@Dg798UDZOs4.com|2452607| +8121|AAAAAAAAJLPBAAAA|430735|1090|8598|2449402|2449372|Mr.|Gordon|Campbell|N|16|4|1931|TURKEY||Gordon.Campbell@rf.com|2452486| +8122|AAAAAAAAKLPBAAAA|1724822|5617|31565|2449800|2449770|Dr.|Joseph|Holman|Y|2|1|1975|GAMBIA||Joseph.Holman@F.com|2452457| +8123|AAAAAAAALLPBAAAA|1885038|6615|4945|2451113|2451083|Sir|Warren|Ladd|Y|30|5|1977|DJIBOUTI||Warren.Ladd@SLcPO.com|2452406| +8124|AAAAAAAAMLPBAAAA|538435|4370|45152|2450663|2450633|Sir|George|Dotson|Y|30|10|1946|ISLE OF MAN||George.Dotson@iMvQ7.org|2452588| +8125|AAAAAAAANLPBAAAA|1101373|6671|21616|2451164|2451134|Dr.|Angela|Smith|Y|12|10|1980|ISRAEL||Angela.Smith@AM2G.com|2452550| +8126|AAAAAAAAOLPBAAAA|786853|3703|25306|2452677|2452647|Mrs.|Kathleen|Morris|N|8|12|1940|ITALY||Kathleen.Morris@jITV2E0paJ.edu|2452501| +8127|AAAAAAAAPLPBAAAA|1445393|5212|35031|2449160|2449130|Mrs.|Sabrina|Haase|Y|23|3|1959|CHINA||Sabrina.Haase@4SZ7Doi9odxX2sQq.edu|2452528| +8128|AAAAAAAAAMPBAAAA|393169|4277|48185|2452012|2451982|Mrs.|Julie|Guerrero|N|30|8|1973|SOLOMON ISLANDS||Julie.Guerrero@PX4zH.com|2452563| +8129|AAAAAAAABMPBAAAA|1043347|3762|18977|2450719|2450689|Sir|Kenneth|Reynolds|Y|31|12|1947|GUERNSEY||Kenneth.Reynolds@LgjmvtgP.org|2452498| +8130|AAAAAAAACMPBAAAA|721843|1479|2315|2450807|2450777|Miss|Allen|Ryan|Y|19|5|1987|MALI||Allen.Ryan@9l9xG1yAY.org|2452384| +8131|AAAAAAAADMPBAAAA|272509|5203|37450|2449387|2449357|Mrs.|Cassandra|Mcgregor|N|25|11|1966|SURINAME||Cassandra.Mcgregor@znI56KtPr.edu|2452482| +8132|AAAAAAAAEMPBAAAA|771855|3425|4195|2450952|2450922|Sir|William|Davis|N|8|7|1974|MOZAMBIQUE||William.Davis@KNn4.edu|2452603| +8133|AAAAAAAAFMPBAAAA|1263557|2147|23137|2450994|2450964|Sir|George|Mcgill|N|14|2|1970|NEW CALEDONIA||George.Mcgill@re4tA.edu|2452546| +8134|AAAAAAAAGMPBAAAA|1326875|3921|26120|2449595|2449565|Dr.|Christopher|Frazer|N|7|8|1959|MONTSERRAT||Christopher.Frazer@TYSojC0Oy6.com|2452318| +8135|AAAAAAAAHMPBAAAA|859760|3533|13306|2450301|2450271|Mr.|Mark|Callahan|Y|26|9|1983|PHILIPPINES||Mark.Callahan@9x.org|2452408| +8136|AAAAAAAAIMPBAAAA|420252|6920|1541|2450733|2450703|Mr.|Mark|Beauchamp|N|16|6|1982|GUYANA||Mark.Beauchamp@N2oTmMVOKfys.org|2452566| +8137|AAAAAAAAJMPBAAAA|1919770|3629|29488||||Norma|Johnson||||1991||||| +8138|AAAAAAAAKMPBAAAA|475170|4979|47541|2451915|2451885|Dr.|Jennifer|Elliott|Y|15|6|1974|DOMINICA||Jennifer.Elliott@RBNV908.com|2452514| +8139|AAAAAAAALMPBAAAA|1901560|4321|33482|2452637|2452607|Mrs.|Pam|Whalen|N|1|4|1934|VENEZUELA||Pam.Whalen@nYes8FU23Vg.edu|2452566| +8140|AAAAAAAAMMPBAAAA|1563674|5140|13359|2449326|2449296|Dr.|John|Evans|N|25|11|1987|PHILIPPINES||John.Evans@cGd3yTeIknoMmqnZ.edu|2452543| +8141|AAAAAAAANMPBAAAA|663635|3274|13266|2451789|2451759|Sir|Joe|James|N|11|5|1934|MONTSERRAT||Joe.James@nJODUKm35A1.com|2452403| +8142|AAAAAAAAOMPBAAAA|861643|2771|39052|2451588|2451558|Miss|Anthony|Fry|Y|23|7|1984|FAROE ISLANDS||Anthony.Fry@x2q7qG53vfDFMplM37D.com|2452606| +8143|AAAAAAAAPMPBAAAA|14951|569|14634|2451604|2451574|Ms.|Debra|Scott|N|5|7|1939|SYRIAN ARAB REPUBLIC||Debra.Scott@dPEpRoPNPJbZ4yuX.com|2452618| +8144|AAAAAAAAANPBAAAA|1660510|5890|35623|2451567|2451537|Dr.|Adam|Enriquez|N|3|12|1930|BAHRAIN||Adam.Enriquez@smeA8.org|2452285| +8145|AAAAAAAABNPBAAAA|1634602|4038|34274|2451022|2450992|Miss|Carla|Wallace|Y|20|10|1968|SENEGAL||Carla.Wallace@BAn2LAQ0eUAZIMZ7.org|2452373| +8146|AAAAAAAACNPBAAAA|||22923|2451657|2451627|Sir|||Y|5|1|1968|NETHERLANDS|||2452506| +8147|AAAAAAAADNPBAAAA|1331335|1477|42419|2451819|2451789|Mr.|Abraham|Bradley|Y|2|12|1979|SAN MARINO||Abraham.Bradley@onZdQp4mS8xy4iixv5.edu|2452588| +8148|AAAAAAAAENPBAAAA|846715|4399|18130|2450384|2450354|Mr.|Kevin|Adams|Y|23|1|1942|GAMBIA||Kevin.Adams@3Yz3ncDLJC.com|2452363| +8149|AAAAAAAAFNPBAAAA|42972|1265|38758|2451959|2451929|Miss|April|Key|N|3|6|1931|VIRGIN ISLANDS, U.S.||April.Key@KGj4KoqDY9Om.com|2452627| +8150|AAAAAAAAGNPBAAAA|1076812|328|13044|2451748|2451718|Sir|Robert|Dean|N|15|5|1931|GUINEA-BISSAU||Robert.Dean@3X5ztQTQ9lV8.edu|2452395| +8151|AAAAAAAAHNPBAAAA|1845681|4013|45145|2451639|2451609|Dr.|Matthew|Folk|N|27|11|1991|NIGER||Matthew.Folk@1p4yseRAoHE2Z3Io.org|2452585| +8152|AAAAAAAAINPBAAAA|1024850|5991|20732|2449512|2449482|Sir|Charles|Tanner|N|22|10|1981|PITCAIRN||Charles.Tanner@Xl.edu|2452530| +8153|AAAAAAAAJNPBAAAA|295878|2303|34681|2452034|2452004|Sir|Michael|Davis|N|25|7|1932|AMERICAN SAMOA||Michael.Davis@GM2lORQqIL0.edu|2452283| +8154|AAAAAAAAKNPBAAAA|1183386|1850|21514|2449554|2449524|Dr.|Christopher|Driggers|Y|13|7|1973|CANADA||Christopher.Driggers@ZyVver.org|2452631| +8155|AAAAAAAALNPBAAAA|2707|6718|34729|2451947|2451917|Mr.|Steven|Mckenna|Y|15|8|1947|MONACO||Steven.Mckenna@iIQTsIG.edu|2452570| +8156|AAAAAAAAMNPBAAAA|1521502|3070|9838|2450858|2450828|Miss|Shawna|Ayala|N|22|4|1985|TAJIKISTAN||Shawna.Ayala@KX79fQNPQfQx5.org|2452397| +8157|AAAAAAAANNPBAAAA|1456297|351|43605|2452216|2452186|Sir|Matthew|Stewart|N|10|5|1975|UNITED ARAB EMIRATES||Matthew.Stewart@QI7utJ7sQr4f.org|2452524| +8158|AAAAAAAAONPBAAAA|1833981|3046|4511|2450201|2450171|Miss|Imogene|Vogel|N|16|4|1951|MADAGASCAR||Imogene.Vogel@so2nPD1MQna.org|2452396| +8159|AAAAAAAAPNPBAAAA|1688923|6221|38319|2449326|2449296|Ms.|Rebecca|Thompson|N|15|3|1942|ZAMBIA||Rebecca.Thompson@Bxg6YjXqe1RQ3.edu|2452526| +8160|AAAAAAAAAOPBAAAA|1259697|2259|41268|2451555|2451525|Mr.|John|King|N|12|5|1950|INDIA||John.King@fl9K.org|2452608| +8161|AAAAAAAABOPBAAAA|88402|6134|45466|2450229|2450199|Sir|Thomas|Day|N|5|12|1983|VENEZUELA||Thomas.Day@LUCtj.org|2452543| +8162|AAAAAAAACOPBAAAA|129786|5200|16214|2451130|2451100|Mrs.|Helen|Mills|Y|30|12|1933|KIRIBATI||Helen.Mills@5SSdEXCRLCcPG.com|2452513| +8163|AAAAAAAADOPBAAAA|449432|806|7827|2450672|2450642|Dr.|Randy|Hernandez|Y|8|9|1938|SENEGAL||Randy.Hernandez@NPojfqd.com|2452569| +8164|AAAAAAAAEOPBAAAA|1277562|1896|15830|2451236|2451206|Dr.|Marian|Reed|N|27|5|1936|ANGOLA||Marian.Reed@AlELYN9XLR.com|2452335| +8165|AAAAAAAAFOPBAAAA|1140975|2986|21797|2450341|2450311|Miss|Vicky|Gross|N|28|8|1928|PHILIPPINES||Vicky.Gross@OuFfrv.com|2452552| +8166|AAAAAAAAGOPBAAAA|747589|749|26756|2452539|2452509|Sir|Michael|Fultz|N|17|12|1987|MEXICO||Michael.Fultz@cCXmqpJpa0SlSBjh.edu|2452362| +8167|AAAAAAAAHOPBAAAA|858097|463|27182|2451297|2451267|Dr.|Irene|Sperry|N|20|2|1932|SLOVENIA||Irene.Sperry@n67MgdTa4BdzFP.com|2452629| +8168|AAAAAAAAIOPBAAAA|1470769|5723|35030|2450734|2450704|Sir|Jesse|Coley|Y|12|10|1931|SAINT HELENA||Jesse.Coley@75lNiGBnxVDeIeszgYp.edu|2452550| +8169|AAAAAAAAJOPBAAAA|459365|5505|26398|2451489|2451459|Ms.|Marcia|Leon|Y|3|6|1964|BURUNDI||Marcia.Leon@hUDASb4J3.edu|2452576| +8170|AAAAAAAAKOPBAAAA|1298838|2683|47281|2450617|2450587|Sir|Nicholas|Harris|Y|14|6|1992|PERU||Nicholas.Harris@rtu1KXpJa5Cj4s.com|2452605| +8171|AAAAAAAALOPBAAAA|162757|6411|36512|2449745|2449715|Mrs.|Letha|Martin|Y|23|11|1991|AZERBAIJAN||Letha.Martin@l0TLci.edu|2452444| +8172|AAAAAAAAMOPBAAAA|18780|3454|27775|2450855|2450825|Ms.|Lori|Garza|Y|27|1|1967|CAMBODIA||Lori.Garza@p9FO8R0fT1TZ5ynvC.org|2452509| +8173|AAAAAAAANOPBAAAA|971539|1131|27523|2450985|2450955|Mr.|Jesus|Allen|N|3|8|1974|OMAN||Jesus.Allen@dBi6Ar5stC.org|2452588| +8174|AAAAAAAAOOPBAAAA|1330636|4477|1965|2449693|2449663|Ms.|Marsha|Funk|Y|29|1|1968|FAROE ISLANDS||Marsha.Funk@Ec0p4B.org|2452339| +8175|AAAAAAAAPOPBAAAA|1684334|609|16084|2452046|2452016|Sir|Mark|Hill|Y|20|11|1968|COMOROS||Mark.Hill@ZT0GZh.org|2452637| +8176|AAAAAAAAAPPBAAAA|844074|6320|7719|2451911|2451881|Ms.|Denise|Johnson|N|24|7|1938|VIRGIN ISLANDS, U.S.||Denise.Johnson@ALY9EZ.com|2452304| +8177|AAAAAAAABPPBAAAA|1621595|5210|6378|2452115|2452085|Dr.|Donald|Cason|N|27|3|1975|CAPE VERDE||Donald.Cason@75kHXHvh.org|2452597| +8178|AAAAAAAACPPBAAAA|1473486|6181|24476|2449361|2449331|Mrs.|Joyce|Hartman|Y|19|11|1930|MONACO||Joyce.Hartman@NiPb.org|2452310| +8179|AAAAAAAADPPBAAAA|701170|2855|31740|2450028|2449998|Mrs.|Lyn|Williams|Y|22|2|1937|MONACO||Lyn.Williams@Q1V13.edu|2452495| +8180|AAAAAAAAEPPBAAAA|306139|5113|41640|2449775|2449745|Miss|Diane|Martin|N|2|8|1938|ALGERIA||Diane.Martin@yEnuk.com|2452571| +8181|AAAAAAAAFPPBAAAA|494764|4487|1763|2452276|2452246|Dr.|Dennis|Herbert|Y|8|6|1992|UKRAINE||Dennis.Herbert@BpmI2l1M1xb6O.edu|2452625| +8182|AAAAAAAAGPPBAAAA|777640|4078|25269|2450813|2450783|Mrs.|Laurie|Davis|N|24|7|1936|FIJI||Laurie.Davis@0jas.org|2452570| +8183|AAAAAAAAHPPBAAAA|942888|4416|5944|2449865|2449835|Dr.|Mark|Coffman|N|15|4|1952|SRI LANKA||Mark.Coffman@1vDkCrrYKL.org|2452429| +8184|AAAAAAAAIPPBAAAA|1662843|2443|48396|2450643|2450613|Dr.|James|Levesque|N|19|4|1942|ROMANIA||James.Levesque@CGlF6hEGe4juq5UiYr.edu|2452630| +8185|AAAAAAAAJPPBAAAA|759769|4193|21864|2449664|2449634|Miss|Donna|Johnson|Y|3|7|1933|GUERNSEY||Donna.Johnson@ziRnPH3.org|2452641| +8186|AAAAAAAAKPPBAAAA|1085147|3696|27045|2450161|2450131|Ms.|Laurie|Davis|Y|20|12|1985|BURUNDI||Laurie.Davis@VFuEqLVGD8EYki6.edu|2452549| +8187|AAAAAAAALPPBAAAA|451690|4143|35396|2452015|2451985|Mrs.|Nilda|Cordova|N|17|8|1965|ROMANIA||Nilda.Cordova@0Q.edu|2452488| +8188|AAAAAAAAMPPBAAAA|1679066|2558|48050|2450756|2450726|Dr.|Gretchen|Duke|N|16|9|1938|MADAGASCAR||Gretchen.Duke@d4GIsM7oLs.org|2452622| +8189|AAAAAAAANPPBAAAA|601839|58|7877|2449282|2449252|Mrs.|Lina|Sanchez|Y|10|6|1931|ISRAEL||Lina.Sanchez@8mif6XJOcS.org|2452605| +8190|AAAAAAAAOPPBAAAA|1401661|6364|8620|2449788|2449758|Dr.|Bobby|Collier|N|13|9|1956|BARBADOS||Bobby.Collier@AhXgF.com|2452464| +8191|AAAAAAAAPPPBAAAA|1910153|4453|34706|2450112|2450082|Mr.|Phillip|Webster|N|31|10|1980|MONTSERRAT||Phillip.Webster@FdpVHykFpqmM9igX.com|2452579| +8192|AAAAAAAAAAACAAAA|963559|4212|818|2450513|2450483|Sir|Mark|Lohr|N|5|5|1973|LIECHTENSTEIN||Mark.Lohr@I3XHBHCDIiR.org|2452592| +8193|AAAAAAAABAACAAAA|1846311|7066|27421|2451747|2451717|Ms.|Grace|Ross|Y|7|6|1955|ALGERIA||Grace.Ross@UqkGyrq84Qxta.org|2452485| +8194|AAAAAAAACAACAAAA|577901|6207|5313||2451169||George|Gipson|Y||6||MAYOTTE||George.Gipson@bs3MXAT.edu|2452591| +8195|AAAAAAAADAACAAAA|547|1837|11127|2450735|2450705|Mrs.|Marjorie|Trimble|N|19|11|1983|NIGER||Marjorie.Trimble@eTEf9bpU3kl.com|2452331| +8196|AAAAAAAAEAACAAAA|187944|4546|35548|2450130|2450100|Dr.|Charles|Roberts|N|15|8|1970|GUINEA||Charles.Roberts@qCZReDrjksBCC.org|2452410| +8197|AAAAAAAAFAACAAAA|827334|301|5668|2452598|2452568|Mrs.|Angela|Donovan|Y|12|11|1982|BAHRAIN||Angela.Donovan@5R0L.org|2452369| +8198|AAAAAAAAGAACAAAA|810904|4314|27277|2450558|2450528|Sir|David|Haskins|N|6|10|1942|MAURITANIA||David.Haskins@BEq3bhGZvsC.org|2452515| +8199|AAAAAAAAHAACAAAA|1208948|2245|14295|2451123|2451093|Ms.|Doris|Melendez|Y|9|11|1975|MONGOLIA||Doris.Melendez@fi.com|2452530| +8200|AAAAAAAAIAACAAAA|1908661|6556|18451|2449529|2449499|Mr.|Kelvin|Sims|N|11|7|1982|GREECE||Kelvin.Sims@YkXNdUFGLCVY0rV.edu|2452525| +8201|AAAAAAAAJAACAAAA|1503224|4722|46667|2451053|2451023|Mrs.|Rosemary|Epperson|Y|5|8|1948|CANADA||Rosemary.Epperson@qtEKd.com|2452425| +8202|AAAAAAAAKAACAAAA|996434|1235|2883|2450369|2450339|Ms.|Viola|Cano|N|29|8|1949|MARSHALL ISLANDS||Viola.Cano@ZEFjdm8dcj.org|2452637| +8203|AAAAAAAALAACAAAA|1188914|1778|18689|2452366|2452336|Dr.|Richard|Goodman|N|11|5|1986|ANGUILLA||Richard.Goodman@7egMLHNSP2CI2.org|2452504| +8204|AAAAAAAAMAACAAAA|1555769|68|29920|2449259|||David|Guzman|N|2|||SINGAPORE||David.Guzman@z.edu|| +8205|AAAAAAAANAACAAAA|940742|7167|439|2449695|2449665|Mrs.|Angela|Simpson|Y|28|9|1950|AZERBAIJAN||Angela.Simpson@hMq9bhfvlZ.edu|2452582| +8206|AAAAAAAAOAACAAAA||7183|46836|||Mr.||Belt|||1|1981||||2452414| +8207|AAAAAAAAPAACAAAA|1229149|6673|41844|2451735|2451705|Dr.|David|Busby|N|24|11|1928|TURKEY||David.Busby@sJsOyUGkC1LFkmI.org|2452411| +8208|AAAAAAAAABACAAAA|38317|2273|4676|2450249|2450219|Sir|George|Esquivel|Y|6|5|1941|ALAND ISLANDS||George.Esquivel@PMbCFNIfaq.org|2452616| +8209|AAAAAAAABBACAAAA|1612162|5408|30654|2451633|2451603|Mr.|Jerry|Thompson|Y|28|7|1960|ESTONIA||Jerry.Thompson@GUVnJS.edu|2452634| +8210|AAAAAAAACBACAAAA|1799926|13|7700|2449585|2449555|Dr.|Nikki|Moody|Y|6|2|1985|GUYANA||Nikki.Moody@V7yOnepTy9.org|2452474| +8211|AAAAAAAADBACAAAA|1855350|4913|14476|2451591|2451561|Mrs.|Alexander|Shackelford|N|27|2|1975|SAINT HELENA||Alexander.Shackelford@xmFbkOM.edu|2452310| +8212|AAAAAAAAEBACAAAA|1051920|5452|10231|2451750|2451720|Sir|David|White|N|22|1|1966|ARGENTINA||David.White@q7FQPvhfO.com|2452536| +8213|AAAAAAAAFBACAAAA|792321|5089|43810|2451172|2451142|Sir|Armando|Hilton|Y|6|10|1935|GUADELOUPE||Armando.Hilton@QaooiEq5q7gbGH84jI0.edu|2452638| +8214|AAAAAAAAGBACAAAA|1183237|4520|9356|2451101|2451071|Dr.|Alta|Meredith|N|15|8|1966|ANTARCTICA||Alta.Meredith@fJ809iAOeMztrUPen.com|2452485| +8215|AAAAAAAAHBACAAAA|518446|5600|49601|2451668|2451638|Mr.|Luis|Levine|Y|26|4|1936|ECUADOR||Luis.Levine@HXYDt.com|2452543| +8216|AAAAAAAAIBACAAAA|678122|1545|23015|2450685|2450655|Mr.|Dale|Richardson|Y|20|2|1945|CZECH REPUBLIC||Dale.Richardson@DNGxxnsp7HgKrv.edu|2452561| +8217|AAAAAAAAJBACAAAA|100863|4973|32730|2451881|2451851|Mr.|Steve|Gomez|N|8|8|1943|UNITED STATES||Steve.Gomez@xaSG.edu|2452437| +8218|AAAAAAAAKBACAAAA|209216|6567|46765|2451717|2451687|Miss|Maura|Campbell|Y|21|10|1950|ETHIOPIA||Maura.Campbell@NYcXCH.com|2452427| +8219|AAAAAAAALBACAAAA|529975|837|7299|2451292|2451262|Mr.|Raymond|Taylor|Y|4|2|1974|TIMOR-LESTE||Raymond.Taylor@KX4CphtlBSEE.edu|2452529| +8220|AAAAAAAAMBACAAAA|193465|1135|25558|2451472|2451442|Sir|Joseph|Bridges|N|30|11|1984|BANGLADESH||Joseph.Bridges@EAuBrBeGrZzq.com|2452515| +8221|AAAAAAAANBACAAAA|22453||29938|2449223|||||N|12|||||Brenda.Rowland@uU.com|2452329| +8222|AAAAAAAAOBACAAAA|515984|6327|40670|2451553|2451523|Sir|Omar|Stiles|Y|15|9|1943|GUATEMALA||Omar.Stiles@GsMysy.edu|2452584| +8223|AAAAAAAAPBACAAAA|902556|5150|7344|2451051|2451021|Sir|Bradley|Mojica|Y|6|5|1972|CANADA||Bradley.Mojica@zMKZVjY.edu|2452560| +8224|AAAAAAAAACACAAAA|1301482|4086|25490|2450569|2450539|Miss|Carrie|Payne|N|16|3|1925|NEPAL||Carrie.Payne@iZPZxgvSg.org|2452516| +8225|AAAAAAAABCACAAAA|1462012|4717|19069|2450020|2449990|Dr.|Elisabeth|Winn|Y|29|5|1976|LATVIA||Elisabeth.Winn@nuOM61gkGsbVHUx4D3.com|2452453| +8226|AAAAAAAACCACAAAA|287658|2682|18106|2452024|2451994|Mr.|Frankie|Jackson|Y|31|1|1939|NORWAY||Frankie.Jackson@B6ma5pBzPh7.edu|2452455| +8227|AAAAAAAADCACAAAA|1493426|2138|33250|2452349|2452319|Ms.|Nancy|Arnold|N|7|9|1976|MADAGASCAR||Nancy.Arnold@x0dmDo7Lv65Kd.com|2452301| +8228|AAAAAAAAECACAAAA|1269124|3432|28140|2450062|2450032|Sir|Timothy|Monroe|Y|8|8|1945|SIERRA LEONE||Timothy.Monroe@GyEq1.org|2452401| +8229|AAAAAAAAFCACAAAA|664410|5791|38772|2451922|2451892|Miss|Patty|Rose|N|12|8|1967|MALAYSIA||Patty.Rose@jbi0byCk8Ll.com|2452538| +8230|AAAAAAAAGCACAAAA|1611827|3221|36215|2449416|2449386|Mr.|Joseph|Chamberlain|N|1|3|1984|COSTA RICA||Joseph.Chamberlain@c5JdN8n00eTuPIg.com|2452418| +8231|AAAAAAAAHCACAAAA|1879415|289|20763|2450618|2450588|Dr.|Robert|Sherwood|Y|14|2|1943|SINGAPORE||Robert.Sherwood@1.org|2452584| +8232|AAAAAAAAICACAAAA|237654|3197|11140|2450076|2450046|Mrs.|Christina|Aaron|N|4|2|1950|NIGER||Christina.Aaron@K.org|2452601| +8233|AAAAAAAAJCACAAAA|1635068|5645|22622|2449844|2449814|Ms.|Brenda|Luis|N|25|12|1929|TURKEY||Brenda.Luis@a.org|2452335| +8234|AAAAAAAAKCACAAAA|824011|7086|23761|2451793|2451763|Mr.|Richard|Hubert|N|20|10|1969|BOUVET ISLAND||Richard.Hubert@Xga0klGjrh.com|2452556| +8235|AAAAAAAALCACAAAA|1434584|2600|33016|2450976|2450946|Dr.|Guillermo|Fritz|Y|25|5|1973|CANADA||Guillermo.Fritz@cRXPVZMs.org|2452633| +8236|AAAAAAAAMCACAAAA|85602|5685|12521|2451799|2451769|Dr.|Melissa|Chastain|N|2|12|1924|GABON||Melissa.Chastain@pHeA2pvjusach5q22.org|2452530| +8237|AAAAAAAANCACAAAA|1476325|2645|12212|2449075|2449045|Mr.|James|Carpenter|N|11|12|1937|SAINT LUCIA||James.Carpenter@nuQoJ6ZlBhTXJ.com|2452542| +8238|AAAAAAAAOCACAAAA|831505|3373|46488|2451074|2451044|Dr.|Rachel|Perry|Y|17|9|1933|NIGER||Rachel.Perry@le.edu|2452565| +8239|AAAAAAAAPCACAAAA|219328|6371|22214|2450108|2450078|Mr.|Gregory|Lu|N|10|5|1985|GUINEA-BISSAU||Gregory.Lu@uOqCCBHjAQ.edu|2452569| +8240|AAAAAAAAADACAAAA|1182677|4518|12006|2450345|2450315|Mrs.|Debra|Welch|N|20|3|1942|LIBERIA||Debra.Welch@uCcdrAQ.com|2452455| +8241|AAAAAAAABDACAAAA|207530|1717|25904|2451071|2451041|Dr.|Wesley|Lee|Y|18|8|1944|UZBEKISTAN||Wesley.Lee@XFQyCJ38Yy9.com|2452335| +8242|AAAAAAAACDACAAAA|173119|754|31189|2449422|2449392|Dr.|Troy|Knox|N|21|12|1938|QATAR||Troy.Knox@0fhyq4uztpXUriItEg3.org|2452576| +8243|AAAAAAAADDACAAAA|499172|7048|571|2451660|2451630|Mrs.|Vickie|Herrera|N|15|12|1945|CYPRUS||Vickie.Herrera@Y8AvYh.edu|2452315| +8244|AAAAAAAAEDACAAAA|907476|486|16343|2449882|2449852|Sir|Colin|Pierce|Y|16|3|1977|MALI||Colin.Pierce@QIzAJ4XO3KNTU.org|2452585| +8245|AAAAAAAAFDACAAAA|917104|4655|43359|2451277|2451247|Dr.|Deborah|Murphy|Y|5|10|1991|SOUTH AFRICA||Deborah.Murphy@Sy1di1S8yxn.org|2452491| +8246|AAAAAAAAGDACAAAA|527288|3713|20491|2450296|2450266|Sir|Juan|Caraway|N|14|3|1927|MOZAMBIQUE||Juan.Caraway@B6LGPfb.edu|2452478| +8247|AAAAAAAAHDACAAAA|||11435||2451892||||Y||4|1934|BARBADOS||Angel.Hughes@FNRDzEddXna1f.org|2452577| +8248|AAAAAAAAIDACAAAA|1829705|2958|33364|2449467|2449437|Mrs.|Elizabeth|Harris|Y|14|11|1939|ZIMBABWE||Elizabeth.Harris@BD3cDhda8.com|2452369| +8249|AAAAAAAAJDACAAAA|865246|5797|49948|2449539|2449509|Mrs.|Susan|Johnson|Y|16|9|1934|GUERNSEY||Susan.Johnson@h6xjZN34Z3vN.edu|2452368| +8250|AAAAAAAAKDACAAAA|1465001|2988|22175|2452375|2452345|Dr.|Joseph|Moore|Y|30|1|1950|LUXEMBOURG||Joseph.Moore@5yyc7ipYsO3Z4.org|2452508| +8251|AAAAAAAALDACAAAA|993528|4081|5034|2451722|2451692|Sir|Joseph|Turner|N|4|2|1930|TRINIDAD AND TOBAGO||Joseph.Turner@P.edu|2452636| +8252|AAAAAAAAMDACAAAA|1209614|3939|6664|2452282|2452252|Dr.|Larry|Robinson|Y|7|3|1929|GIBRALTAR||Larry.Robinson@SyM6MLLsrjc.edu|2452486| +8253|AAAAAAAANDACAAAA|1371243|4139|42635|2449356|2449326|Mr.|Gary|Shirley|Y|5|12|1940|GUINEA||Gary.Shirley@ss9x1MYNRFG9.com|2452498| +8254|AAAAAAAAODACAAAA|919299|5029|33959|2449148|2449118|Mr.|Danny|Folsom|N|25|2|1962|NICARAGUA||Danny.Folsom@cp068641.org|2452346| +8255|AAAAAAAAPDACAAAA|65178|5824|26070|2450066|2450036|Mrs.|Rosalie|Farias|Y|27|12|1971|BENIN||Rosalie.Farias@pUfcbl1vCde.org|2452600| +8256|AAAAAAAAAEACAAAA|167260|4596|37885|2450445|2450415|Mrs.|Vivian|Reid|N|6|4|1956|MOROCCO||Vivian.Reid@7mG1nVDhvoYHyq4.org|2452497| +8257|AAAAAAAABEACAAAA|452485|3365|42198|2450137|2450107|Miss|Megan|Hein|Y|27|3|1964|BOTSWANA||Megan.Hein@hh.edu|2452581| +8258|AAAAAAAACEACAAAA|1671264|6804|4791|2450066|2450036|Mr.|Mark|Smith|N|19|12|1968|KOREA, REPUBLIC OF||Mark.Smith@ync2p7h9rBFtvBnKU.org|2452443| +8259|AAAAAAAADEACAAAA|399020|6627|5034|2451628|2451598|Dr.|Dustin|Baez|N|16|12|1957|PALAU||Dustin.Baez@rXm4Jqxyi4Sr7Te.org|2452300| +8260|AAAAAAAAEEACAAAA|887320|4171|39822|2450165|2450135|Dr.|Matthew|Arthur|Y|10|9|1975|SINGAPORE||Matthew.Arthur@dyrfib.edu|2452626| +8261|AAAAAAAAFEACAAAA|118855|4134|44613|2451555|2451525|Sir|Donald|Quarles|Y|5|4|1967|BURUNDI||Donald.Quarles@ZCkR57CTUC.com|2452567| +8262|AAAAAAAAGEACAAAA|846602|2936|8517|2451211|2451181|Dr.|Leeanna|Fitch|Y|3|5|1961|ZIMBABWE||Leeanna.Fitch@jdM7aIRIkK5.com|2452578| +8263|AAAAAAAAHEACAAAA|923945|5266|29034|2452527|2452497|Mr.|Lee|Saxton|N|17|3|1962|ESTONIA||Lee.Saxton@5gnFcxyYPSaPV.org|2452461| +8264|AAAAAAAAIEACAAAA|1806328|670|33812|2449198|2449168|Sir|Courtney|Sanchez|N|22|4|1973|BELGIUM||Courtney.Sanchez@K4q9n1jxUOiz.org|2452516| +8265|AAAAAAAAJEACAAAA|173155|2303|12251|2449137|2449107|Mr.|Gary|Piazza|Y|6|8|1924|CZECH REPUBLIC||Gary.Piazza@KGRRXhe2i2Y.edu|2452364| +8266|AAAAAAAAKEACAAAA|1825242|1286|20177|2451435|2451405|Mrs.|Alina|Brewer|Y|16|9|1971|GABON||Alina.Brewer@ybLJQQrUEjF5b.edu|2452442| +8267|AAAAAAAALEACAAAA|443868|4312|18756|2450030|2450000|Mrs.|Vera|Castro|Y|31|12|1957|BERMUDA||Vera.Castro@2nNSMGOb0.edu|2452584| +8268|AAAAAAAAMEACAAAA|1106369|3380|4344|2450371|2450341|Mr.|James|Robertson|N|12|8|1952|CAMBODIA||James.Robertson@K.com|2452508| +8269|AAAAAAAANEACAAAA|630096||26828|2452667|2452637||Keith|Austin|Y|28|12||DENMARK|||| +8270|AAAAAAAAOEACAAAA|1629837|3903|33522|2452338|2452308|Dr.|Tommy|Peter|Y|13|8|1981|LIECHTENSTEIN||Tommy.Peter@tJA77Ptl.com|2452362| +8271|AAAAAAAAPEACAAAA|1779380|5043|11506|2451970|2451940|Dr.|Leonard|Willis|Y|8|6|1950|MONGOLIA||Leonard.Willis@R9X0Nqoj.com|2452455| +8272|AAAAAAAAAFACAAAA|1713087|637|35740|2450951|2450921|Ms.|Ruby|Collier|N|5|5|1983|SINGAPORE||Ruby.Collier@xnaqMMHrmP.com|2452310| +8273|AAAAAAAABFACAAAA|988653|5267|35481|2449831|2449801|Mrs.|Hortensia|Cunningham|N|21|12|1975|GUATEMALA||Hortensia.Cunningham@PapsFFDqG0E0RBSn.edu|2452534| +8274|AAAAAAAACFACAAAA|1055681|5835|33542|2449470|2449440|Mr.|Ronald|Anderson|N|19|10|1988|GUATEMALA||Ronald.Anderson@Ty.edu|2452568| +8275|AAAAAAAADFACAAAA|61486|4730|47621|2449336|2449306|Sir|David|Moore|Y|30|4|1980|GERMANY||David.Moore@0pc.org|2452555| +8276|AAAAAAAAEFACAAAA||4686|8254|2451305||Ms.||Anderson|Y|23|5||||Anita.Anderson@1RyNdJg1.edu|2452632| +8277|AAAAAAAAFFACAAAA|1481645|3082|27319|2452517|2452487|Dr.|Joseph|Powell|N|16|3|1936|GREENLAND||Joseph.Powell@lvYfDGgEE.com|2452415| +8278|AAAAAAAAGFACAAAA|1519328|5375|19069|2450987|2450957|Ms.|Ruth|Anthony|Y|11|5|1953|ISLE OF MAN||Ruth.Anthony@Qduu3Tf.com|2452404| +8279|AAAAAAAAHFACAAAA|418812|2122|12185|2451962|2451932|Sir|Warren|Edmond|N|6|1|1967|DJIBOUTI||Warren.Edmond@A5iMvZVF6Nz0nu7B.com|2452349| +8280|AAAAAAAAIFACAAAA|1882381|3516|49898|2450742|2450712|Miss|Ellen|Watson|N|9|1|1957|GIBRALTAR||Ellen.Watson@1s1218Hqx7qTd.org|2452483| +8281|AAAAAAAAJFACAAAA|1022213|4401|8542|2450210|2450180|Ms.|Sheryl|Harris|N|15|7|1937|GUAM||Sheryl.Harris@kSTAsi2advzp5.org|2452424| +8282|AAAAAAAAKFACAAAA|1872382|3635|10488|2451393|2451363|Mr.|John|Jones|Y|18|9|1985|COMOROS||John.Jones@YXvJ.com|2452519| +8283|AAAAAAAALFACAAAA|1834529|6463|23597|2452316|2452286|Mr.|Donald|Harrison|N|26|11|1965|ALGERIA||Donald.Harrison@cOaqkUF6E.edu|2452330| +8284|AAAAAAAAMFACAAAA|1125212|190|22015|2449666|2449636|Sir|James|Winston|Y|15|9|1980|R�UNION||James.Winston@sJoCdatFTKLTk2PmB3G.org|2452306| +8285|AAAAAAAANFACAAAA|46882|1142|9402|2451928|2451898|Dr.|Larry|Welch|N|21|5|1950|PARAGUAY||Larry.Welch@Y90ON.org|2452471| +8286|AAAAAAAAOFACAAAA|701851|4304|26004|2451452|2451422|Ms.|Ami|Dupuis|N|21|5|1980|KAZAKHSTAN||Ami.Dupuis@3l5H6tNiPyKEcn2.org|2452532| +8287|AAAAAAAAPFACAAAA|640679|1827|10975|2449293|2449263|Ms.|Brenda|Lozano|Y|7|12|1965|FRANCE||Brenda.Lozano@Fsse.org|2452632| +8288|AAAAAAAAAGACAAAA|1748738|4316|42647|2450995|2450965|Mrs.|Henrietta|Nagy|Y|27|9|1962|MARSHALL ISLANDS||Henrietta.Nagy@kJmGuGHH9Lr.org|2452309| +8289|AAAAAAAABGACAAAA|1703585|2249|27713|2450607|2450577|Miss|Jennifer|Rendon|N|5|6|1962|NIUE||Jennifer.Rendon@RGLHPezTvOh7dQKAeu.com|2452533| +8290|AAAAAAAACGACAAAA|96699|5469|38639|2449924|2449894|Sir|Luke|Miles|N|9|4|1959|FINLAND||Luke.Miles@1Te8CjggISGoY6OYmS0.org|2452525| +8291|AAAAAAAADGACAAAA|547470|4109|43995|2451843|2451813|Ms.|Beverly|Stroud|Y|24|11|1977|SEYCHELLES||Beverly.Stroud@OiIxGUH8p1o3Rc26.com|2452303| +8292|AAAAAAAAEGACAAAA|1840648|3972|39249|2451003|2450973|Miss|Marilyn|Wright|Y|13|7|1991|NETHERLANDS||Marilyn.Wright@rpRkt8FdhnLUf8y2xR.edu|2452440| +8293|AAAAAAAAFGACAAAA|15387|2612|45790|2450897|2450867|Dr.|Dwight|Miller|Y|27|1|1958|BAHRAIN||Dwight.Miller@GoLUbHz.com|2452520| +8294|AAAAAAAAGGACAAAA|1305162|7062|5809|2452554|2452524|Sir|Francis|Dale|Y|18|4|1940|HAITI||Francis.Dale@0E.org|2452472| +8295|AAAAAAAAHGACAAAA|1321687|1712|26137|2449464|2449434|Sir|Lanny|Ruiz|N|21|4|1966|ICELAND||Lanny.Ruiz@2Z4sV.com|2452455| +8296|AAAAAAAAIGACAAAA|1261815|863|42311|2449288|2449258|Miss|Penny|Osborne|N|1|7|1963|NEPAL||Penny.Osborne@p2KXMv3tUCIhxVoskn.edu|2452323| +8297|AAAAAAAAJGACAAAA|1502423|6715|1854|2450109|2450079|Dr.|Nicholas|Love|Y|3|4|1984|LATVIA||Nicholas.Love@p8nXMO.org|2452618| +8298|AAAAAAAAKGACAAAA|1448844|6480|38116|2451433|2451403|Mr.|Joseph|Snyder|Y|14|1|1945|VIRGIN ISLANDS, U.S.||Joseph.Snyder@gSc2.com|2452452| +8299|AAAAAAAALGACAAAA|474299|3814|23776|2450722|2450692|Ms.|Anne|Waller|N|23|9|1975|RUSSIAN FEDERATION||Anne.Waller@rUIthGBeN1.edu|2452487| +8300|AAAAAAAAMGACAAAA|715456|6676|30743|2449597|2449567|Ms.|Florida|Lopez|N|21|6|1928|NIGERIA||Florida.Lopez@5hcpOT3NIQRu8.org|2452498| +8301|AAAAAAAANGACAAAA|401777|6891|26005|2449177|2449147|Sir|Robert|Smith|Y|28|11|1974|NICARAGUA||Robert.Smith@Kg.com|2452344| +8302|AAAAAAAAOGACAAAA|404566|2136|30553|2450907|2450877|Miss|Lindsy|Bass|N|14|8|1982|CHILE||Lindsy.Bass@SsocJh1CiUO.org|2452638| +8303|AAAAAAAAPGACAAAA|633026|527|9275|2449045|2449015|Mr.|Cliff|Dotson|Y|16|10|1956|ISLE OF MAN||Cliff.Dotson@HDGscbD3cSRUFqK.org|2452642| +8304|AAAAAAAAAHACAAAA|1805377|4646|1115|2450900|2450870|Sir|Owen|Graham|N|10|5|1937|CAMEROON||Owen.Graham@lFtOQ3YZZXK.edu|2452483| +8305|AAAAAAAABHACAAAA|922425|1206|834|2450232|2450202|Mr.|Michael|Yang|N|19|4|1939|PUERTO RICO||Michael.Yang@PMKPDqLANuNX.edu|2452287| +8306|AAAAAAAACHACAAAA|370200|5718|11589|2449410|2449380|Ms.|Annette|Robles|N|6|12|1949|LUXEMBOURG||Annette.Robles@bD8C3.com|2452401| +8307|AAAAAAAADHACAAAA|1662304|6131|37158|2452059|2452029|Dr.|Betty|Shelby|N|16|7|1966|GHANA||Betty.Shelby@uINg.org|2452568| +8308|AAAAAAAAEHACAAAA|1366385|498|45458|2451191|2451161|Dr.|Brandy|Waters|N|25|11|1931|BAHRAIN||Brandy.Waters@GYRB0G8pTxrtg2n.com|2452523| +8309|AAAAAAAAFHACAAAA|342574|3262|48303|2452477|2452447|Mr.|Thomas|Arnold|Y|27|6|1944|BOLIVIA||Thomas.Arnold@Gqf6lxisI83yj.edu|2452309| +8310|AAAAAAAAGHACAAAA|144098|3047|49149|2450681|2450651|Miss|Marlo|Stjohn|N|25|12|1928|CZECH REPUBLIC||Marlo.Stjohn@uF5h3tA8Mg5.com|2452495| +8311|AAAAAAAAHHACAAAA|992905|850|44348|2449614|2449584|Mrs.|Pearl|Jones|N|21|3|1956|SURINAME||Pearl.Jones@qXPheLZ5xZbZKS.edu|2452293| +8312|AAAAAAAAIHACAAAA|503777|4153|36129|2450838|2450808|Mr.|Roger|Rivera|Y|11|8|1946|MALTA||Roger.Rivera@mbE8.com|2452637| +8313|AAAAAAAAJHACAAAA|674501|505|28956|2449495|2449465|Ms.|Nancy|Rose|Y|13|9|1946|MALAYSIA||Nancy.Rose@mBXIpdK.org|2452615| +8314|AAAAAAAAKHACAAAA|708316|95|45121|2452100|2452070|Dr.|Joyce|Wendt|Y|19|6|1972|PANAMA||Joyce.Wendt@PU0NRk5t25DO.edu|2452517| +8315|AAAAAAAALHACAAAA|1095759|1716|48873|2450658|2450628|Mrs.|Sandra|Becker|N|25|3|1925|FIJI||Sandra.Becker@g8g5sr9SguiqyfUlp.org|2452306| +8316|AAAAAAAAMHACAAAA|1676234|4027|28667|2449735|2449705|Ms.|Laura|Foster|Y|1|4|1963|LATVIA||Laura.Foster@rgsBEX96VfZ.org|2452523| +8317|AAAAAAAANHACAAAA|1662314|2097|42680|2450881|2450851|Dr.|Lillian|Roden|Y|21|6|1951|NAMIBIA||Lillian.Roden@3jjl5tSghYCF5.org|2452372| +8318|AAAAAAAAOHACAAAA|236386|80|10574|2451672|2451642|Ms.|Viola|Swartz|N|30|7|1952|GREENLAND||Viola.Swartz@hILXrM6dELu.edu|2452370| +8319|AAAAAAAAPHACAAAA|1179343|1803|15816|2450969|2450939|Miss|Paula|Harless|N|16|11|1933|KIRIBATI||Paula.Harless@J66xVgds6Ua46V.com|2452349| +8320|AAAAAAAAAIACAAAA|1497722|3809|49496|2450800|2450770|Dr.|Bobby|Newman|Y|16|3|1968|ANTARCTICA||Bobby.Newman@20RlB3icTAPO.edu|2452447| +8321|AAAAAAAABIACAAAA|1107770|2500|2877|2450929|2450899|Dr.|George|Cho|Y|25|6|1977|SEYCHELLES||George.Cho@6boTmu.edu|2452293| +8322|AAAAAAAACIACAAAA|105202|5782|31962|2450321|2450291|Mr.|Clyde|Gray|N|29|1|1956|NICARAGUA||Clyde.Gray@ZTn.org|2452569| +8323|AAAAAAAADIACAAAA|872817||47666|2450082|2450052|Dr.|||N|||1972|CROATIA|||2452354| +8324|AAAAAAAAEIACAAAA|1458669|1261|32206|2451901|2451871|Miss|Melinda|Cordova|Y|16|1|1987|ICELAND||Melinda.Cordova@41Vu.org|2452496| +8325|AAAAAAAAFIACAAAA|1491619|319|25819|2452634|2452604|Dr.|Sarah|Garcia|Y|22|3|1977|UNITED STATES||Sarah.Garcia@e0383Go4ZRAxgkXNi.edu|2452392| +8326|AAAAAAAAGIACAAAA|1186301|3528|1555|2451927|2451897|Dr.|Rafael|Miller|N|11|11|1992|UZBEKISTAN||Rafael.Miller@Xy3KcY5pXiFo4F.org|2452422| +8327|AAAAAAAAHIACAAAA|618220|2386|49399|2450269|2450239|Miss|Chandra|Cota|N|13|7|1968|CAPE VERDE||Chandra.Cota@yZ.edu|2452620| +8328|AAAAAAAAIIACAAAA|784759|1061|2163|2452380|2452350|Miss|Kathy|Bowles|Y|11|8|1985|TURKEY||Kathy.Bowles@YnH7otQbFKqqp.org|2452358| +8329|AAAAAAAAJIACAAAA|929237|6781|39525|2451720|2451690|Mrs.|Virginia|Fulton|Y|3|5|1928|SENEGAL||Virginia.Fulton@Ik.edu|2452554| +8330|AAAAAAAAKIACAAAA|1107658|1865|36119|2449271|2449241|Mr.|Jeff|Schmidt|Y|19|7|1943|PORTUGAL||Jeff.Schmidt@U34GF.org|2452355| +8331|AAAAAAAALIACAAAA|63418|2485|7905|2450367|2450337|Dr.|Timothy|Mcwhorter|N|16|9|1964|SAUDI ARABIA||Timothy.Mcwhorter@6mZLY.org|2452358| +8332|AAAAAAAAMIACAAAA|1148180|3654|33975|2450504|2450474|Mr.|Michael|Reardon|Y|8|12|1954|NIGER||Michael.Reardon@3zEG1tSnDnZQuH.org|2452603| +8333|AAAAAAAANIACAAAA|162124|4968|38212|2449816|2449786|Mr.|Cleveland|Terry|N|7|2|1958|BANGLADESH||Cleveland.Terry@mcayAd.edu|2452523| +8334|AAAAAAAAOIACAAAA|935826|1508|8437|2451308|2451278|Dr.|Junior|Brown|N|10|9|1925|BAHRAIN||Junior.Brown@uJr.org|2452529| +8335|AAAAAAAAPIACAAAA|1072198|1677|43909|2451200|2451170|Ms.|Candice|Gary|N|1|3|1980|UNITED KINGDOM||Candice.Gary@qd6.edu|2452403| +8336|AAAAAAAAAJACAAAA|1792490|2682|29274|2451975|2451945|Dr.|Matthew|Caldwell|Y|19|10|1937|NICARAGUA||Matthew.Caldwell@VSR4GPKucq.edu|2452513| +8337|AAAAAAAABJACAAAA|42323|2039|28555|2450988|2450958|Miss|Karen|Valentin|Y|17|10|1959|BENIN||Karen.Valentin@BJ7bpUg9gnAYLJOHgM.com|2452304| +8338|AAAAAAAACJACAAAA|578778|5826|30827|2449058|2449028|Sir|Kraig|Waldron|Y|10|8|1943|SOMALIA||Kraig.Waldron@hMdF6BxPaPIq.com|2452295| +8339|AAAAAAAADJACAAAA|1752002|2268|23734|2450416|2450386|Sir|Jeffrey|Rodriguez|N|1|1|1952|VIET NAM||Jeffrey.Rodriguez@paDkS5Qx133P.org|2452369| +8340|AAAAAAAAEJACAAAA|1566944|2103|42909|2450763|2450733|Dr.|Jennifer|Bryant|N|7|10|1979|RUSSIAN FEDERATION||Jennifer.Bryant@t0Y4yzPs.com|2452392| +8341|AAAAAAAAFJACAAAA|1365287|7140|22193|2451097|2451067|Dr.|Thomas|||9|12|1956|NORFOLK ISLAND|||| +8342|AAAAAAAAGJACAAAA|851397|3855|43913|2450719|2450689|Ms.|Elizabeth|Julian|Y|29|11|1945|R�UNION||Elizabeth.Julian@mS8aqoZ69aMb.com|2452466| +8343|AAAAAAAAHJACAAAA|1569386|1352|518|2450729|2450699|Mrs.|Amber|Russ|N|9|9|1982|TOKELAU||Amber.Russ@vIvY5V.edu|2452573| +8344|AAAAAAAAIJACAAAA|1249039|3325|2811|2450510|2450480|Mr.|Johnny|Dunn|Y|12|8|1985|PAKISTAN||Johnny.Dunn@9.com|2452336| +8345|AAAAAAAAJJACAAAA|764679|2941|15603|2452590|2452560|Mr.|Thomas|Babcock|N|26|3|1931|TOKELAU||Thomas.Babcock@42hy7xk1vf9e.com|2452450| +8346|AAAAAAAAKJACAAAA|1469344|6906|44021|2449175|2449145|Ms.|Rosa|Garcia|N|6|2|1975|KOREA, REPUBLIC OF||Rosa.Garcia@3fPHIk8.org|2452502| +8347|AAAAAAAALJACAAAA|418878|5175|27472|2451160|2451130|Dr.|Nancy|Parsons|N|11|2|1977|JAPAN||Nancy.Parsons@lFEb.edu|2452498| +8348|AAAAAAAAMJACAAAA|821220|5705|39037|2451920|2451890|Mr.|Kenneth|Young|Y|9|6|1924|BELGIUM||Kenneth.Young@Nfnxsta9bUYQr3r4.com|2452455| +8349|AAAAAAAANJACAAAA|461604|6701|25445|2450678|2450648|Miss|Tiffany|Connors|N|6|10|1979|AUSTRALIA||Tiffany.Connors@1qG29u8gcdu.com|2452638| +8350|AAAAAAAAOJACAAAA|1768761|3067|7820||2450263|||||31|3|||||| +8351|AAAAAAAAPJACAAAA|1056303|3478|26253|2451603|2451573|Dr.|Conrad|Solomon|N|15|9|1928|GUADELOUPE||Conrad.Solomon@hUSGQ6LOd.edu|2452647| +8352|AAAAAAAAAKACAAAA|346855|1322|29342|2451492|2451462|Mr.|Rudolph|West|N|30|5|1975|WESTERN SAHARA||Rudolph.West@tMmllDf2a39t.edu|2452480| +8353|AAAAAAAABKACAAAA|1124600|4998|2996|2450414|2450384|Dr.|Joseph|Henderson|N|15|8|1968|MALTA||Joseph.Henderson@m5N.org|2452397| +8354|AAAAAAAACKACAAAA|301643|6575|41107|2449068|2449038|Mr.|Nicholas|Jackson|N|3|11|1933|ERITREA||Nicholas.Jackson@nlE4l.com|2452314| +8355|AAAAAAAADKACAAAA||1281|4137|2451795|||Javier|Rhoades||||1981||||2452528| +8356|AAAAAAAAEKACAAAA|1909919||49865||||Jerrie|Greer||10|10|1982||||2452383| +8357|AAAAAAAAFKACAAAA|1172680|45|27057|2450996|2450966|Dr.|Vernita|Lincoln|Y|19|11|1986|FAROE ISLANDS||Vernita.Lincoln@C8nQjx3XkZECF2l.org|2452447| +8358|AAAAAAAAGKACAAAA|1434543|1956|8969|2449121|2449091|Mr.|Mike|Hitchcock|Y|22|7|1962|IRELAND||Mike.Hitchcock@73YyDlgBz.com|2452287| +8359|AAAAAAAAHKACAAAA|1492318|3386|807|2449134|2449104|Dr.|Jill|Bach|N|25|4|1974|TURKMENISTAN||Jill.Bach@dfxK.edu|2452494| +8360|AAAAAAAAIKACAAAA|1047646|1755|36311|2451419|2451389|Mrs.|Jessica|West|N|8|12|1964|AZERBAIJAN||Jessica.West@kQ4rTArZQimZiKJj4K.edu|2452486| +8361|AAAAAAAAJKACAAAA|1012958|5735|10912|2449166|2449136|Dr.|Arthur|Harris|N|9|3|1925|SEYCHELLES||Arthur.Harris@Sj3QkYzfu.com|2452622| +8362|AAAAAAAAKKACAAAA|651517|3754|8838|2449536|2449506|Dr.|Alan|Dixon|Y|27|3|1949|KOREA, REPUBLIC OF||Alan.Dixon@CPJPCQO8OR.com|2452493| +8363|AAAAAAAALKACAAAA|488005|3911|8617|2450328|2450298|Ms.|Farrah|Pellegrino|Y|3|3|1988|NIGER||Farrah.Pellegrino@BdYBDTFUafUB.com|2452310| +8364|AAAAAAAAMKACAAAA|562119|5459|48446|2449063|2449033|Mr.|Robert|Perry|Y|2|8|1961|GUINEA-BISSAU||Robert.Perry@kDdIL0FCj7Zvr6ST.edu|2452441| +8365|AAAAAAAANKACAAAA|1121466|92|44271|2450612|2450582|Miss|Mabel|Ortiz|Y|21|2|1986|THAILAND||Mabel.Ortiz@XsT6hl1qY.com|2452289| +8366|AAAAAAAAOKACAAAA|613695|3792|45889|2449030|2449000|Ms.|Florence|Peck|N|14|12|1980|CROATIA||Florence.Peck@LLN77.com|2452334| +8367|AAAAAAAAPKACAAAA|1014578|6666|1507|2452598|2452568|Dr.|Hilda|Carter|Y|9|11|1935|FRENCH POLYNESIA||Hilda.Carter@hsUTP10jTv1s.edu|2452583| +8368|AAAAAAAAALACAAAA|799099|5937|12022|2451659|2451629|Sir|Cedrick|Buss|Y|9|8|1990|MALI||Cedrick.Buss@ChOYDTs3zMC.org|2452317| +8369|AAAAAAAABLACAAAA|877338|5198|10611|2451253|2451223|Sir|Edward|Duarte|Y|4|1|1939|ALGERIA||Edward.Duarte@d.org|2452383| +8370|AAAAAAAACLACAAAA|1731263|6563|13063|2451988|2451958|Ms.|Jane|Delossantos|N|7|10|1983|LEBANON||Jane.Delossantos@SRtvTL08c6fUk8fT2ue4.edu|2452518| +8371|AAAAAAAADLACAAAA|666315|760|4805|2452099|2452069|Dr.|Benjamin|Chapman|N|18|12|1952|AUSTRIA||Benjamin.Chapman@6XcooljpCZ0H.edu|2452285| +8372|AAAAAAAAELACAAAA|1676880|3517|25707|2451252|2451222|Dr.|Dean|Dortch|Y|25|6|1965|FRENCH POLYNESIA||Dean.Dortch@pM37zsFoJh.edu|2452295| +8373|AAAAAAAAFLACAAAA|255248|4249|46050|2450747|2450717|Mr.|Brian|Mclean|Y|12|1|1989|IRELAND||Brian.Mclean@HH25qYDL.com|2452457| +8374|AAAAAAAAGLACAAAA|1083055|4724|17993|2451309|2451279|Dr.|Shelby|Catron|Y|30|8|1956|BAHRAIN||Shelby.Catron@buFMd2.org|2452532| +8375|AAAAAAAAHLACAAAA|1647211|2773|3135|2451211|2451181|Dr.|Randy|Bourgeois|N|19|9|1972|KOREA, REPUBLIC OF||Randy.Bourgeois@VdnCkSSChvON158.com|2452514| +8376|AAAAAAAAILACAAAA|1589973|1748|14323|2449311|2449281|Mrs.|Anita|Mcmurray|N|20|4|1992|POLAND||Anita.Mcmurray@DvvhNEsrxnge72.org|2452630| +8377|AAAAAAAAJLACAAAA|1804962|6967|46925|2450563|2450533|Sir|Jason|Meade|N|25|11|1925|GUADELOUPE||Jason.Meade@AZH2cYq.com|2452325| +8378|AAAAAAAAKLACAAAA|1885923|5877|31831|2451803|2451773|Ms.|Anita|Perry|N|5|12|1943|GUINEA-BISSAU||Anita.Perry@d329bete5Br.edu|2452321| +8379|AAAAAAAALLACAAAA|63752|481|21210|2451209|2451179|Sir|Stan|Long|Y|21|6|1956|SAN MARINO||Stan.Long@9.org|2452297| +8380|AAAAAAAAMLACAAAA|11720|3335|30997|2449580|2449550|Sir|Jeff|Jacobs|N|14|7|1949|MONACO||Jeff.Jacobs@qGxyO3lPDHI6.com|2452601| +8381|AAAAAAAANLACAAAA|1143460|5195|25730||||David|Alvarado|||||||David.Alvarado@XDX4SzMVQfu.org|| +8382|AAAAAAAAOLACAAAA|1660198|1962|36173|2449239|2449209|Mr.|Charles|Martin|N|6|9|1970|WESTERN SAHARA||Charles.Martin@UaTK4BSlsV.org|2452296| +8383|AAAAAAAAPLACAAAA|1164007|5034|34861|2451188|2451158|Miss|Freda|Pollard|Y|31|8|1948|SWEDEN||Freda.Pollard@lopmQ0LOFJ54.edu|2452605| +8384|AAAAAAAAAMACAAAA|1568032|1374|47869|2449575|2449545|Dr.|Donald|Travis|N|21|6|1982|ITALY||Donald.Travis@GYA.edu|2452494| +8385|AAAAAAAABMACAAAA|1352314|5809|35304|2452376|2452346|Miss|Jacqueline|Held|Y|16|10|1970|YEMEN||Jacqueline.Held@EKCu.com|2452450| +8386|AAAAAAAACMACAAAA|1394191|6949|48860|2450493|2450463|Mrs.|Shawna|Baxter|N|4|10|1927|TUVALU||Shawna.Baxter@lSGEhlgfjH.com|2452308| +8387|AAAAAAAADMACAAAA|9549|2709|6222|2450246|2450216|Mr.|Robert|Dudley|Y|21|12|1954|ZAMBIA||Robert.Dudley@XZJGCpaG097l.org|2452591| +8388|AAAAAAAAEMACAAAA|1080051|6425|36046|2449106|2449076|Miss|Sharon|Richard|N|4|7|1932|KOREA, REPUBLIC OF||Sharon.Richard@vTix4KD5grMxXt.com|2452322| +8389|AAAAAAAAFMACAAAA|252361|3863|118|2452356|2452326|Sir|Robert|Davis|N|25|8|1956|FAROE ISLANDS||Robert.Davis@TH0F4uZngSF.com|2452515| +8390|AAAAAAAAGMACAAAA|1071993|882|49724|2452113|2452083|Miss|Rosina|Nichols|N|8|8|1933|MAURITIUS||Rosina.Nichols@JontczqqorrZ4Y4.edu|2452460| +8391|AAAAAAAAHMACAAAA|1507213|2485|22157|2449031|2449001|Mrs.|Mae|Horton|N|8|1|1941|UNITED ARAB EMIRATES||Mae.Horton@pTyQtYXyTv4Ad.edu|2452385| +8392|AAAAAAAAIMACAAAA|474541|4161|10981|2449952|2449922|Sir|Jose|Arnold|N|31|5|1952|PITCAIRN||Jose.Arnold@N.org|2452297| +8393|AAAAAAAAJMACAAAA|1029849|6187|24125|2451072|2451042|Mr.|Phillip|Mendoza|N|4|4|1992|IRAQ||Phillip.Mendoza@O5iqH8CDzC6B.com|2452480| +8394|AAAAAAAAKMACAAAA|402467|1025|11983|2449181|2449151|Mrs.|Roberta|Leon|N|23|12|1990|URUGUAY||Roberta.Leon@Ajeq6dp0o.edu|2452340| +8395|AAAAAAAALMACAAAA|1657808|3870|6285|2450528|2450498|Ms.|Diane|Montgomery|Y|15|8|1976|GUERNSEY||Diane.Montgomery@DSl3RX52POMFpvkz.org|2452339| +8396|AAAAAAAAMMACAAAA|718369|5692|20215|2449583|2449553|Miss|Elaine|Knapp|N|14|1|1962|TOKELAU||Elaine.Knapp@5iihEG42BthJgTC.org|2452310| +8397|AAAAAAAANMACAAAA|1775306|1187|48668|2450094|2450064|Dr.|Karen|Bermudez|N|22|5|1939|LESOTHO||Karen.Bermudez@z9jZkR.com|2452545| +8398|AAAAAAAAOMACAAAA|861088|5739|39590|2449303|2449273|Dr.|Paul|Mason|Y|5|6|1934|SYRIAN ARAB REPUBLIC||Paul.Mason@NsGiU1TJK4U3.edu|2452513| +8399|AAAAAAAAPMACAAAA|861231|7001|38771|2449894|2449864|Sir|Daniel|Gulley|Y|24|2|1932|SAUDI ARABIA||Daniel.Gulley@FehjDikXCG.edu|2452417| +8400|AAAAAAAAANACAAAA|1588923|4352|37828|2452261|2452231|Sir|John|Buck|N|7|12|1978|LIBERIA||John.Buck@i.edu|2452336| +8401|AAAAAAAABNACAAAA|979562|2030|26051|2450789|2450759|Mrs.|Janel|Phelps|N|29|1|1968|FRANCE||Janel.Phelps@FLVVgnmcuMBlDf.org|2452328| +8402|AAAAAAAACNACAAAA|261647|5708|34122|2451794|2451764|Dr.|Douglas|Aldridge|N|6|9|1942|MYANMAR||Douglas.Aldridge@BS.com|2452334| +8403|AAAAAAAADNACAAAA|1615141|1216|8091||||Kevin|||21||1936||||2452326| +8404|AAAAAAAAENACAAAA|342871|2425|39979|2451609|2451579|Mrs.|Emiko|Grover|N|19|3|1971|PAPUA NEW GUINEA||Emiko.Grover@aZdUa.com|2452321| +8405|AAAAAAAAFNACAAAA|1495725|1519|32113|2450818|2450788|Dr.|Harry|Scales|Y|4|2|1962|TRINIDAD AND TOBAGO||Harry.Scales@TmgLx4jll2bQ.com|2452606| +8406|AAAAAAAAGNACAAAA|1674306|3106|23059|2450430|2450400|Ms.|Nancy|Wallace|Y|6|7|1964|BOUVET ISLAND||Nancy.Wallace@1NgrYKfG.edu|2452434| +8407|AAAAAAAAHNACAAAA|854549|2079|12809|2450109||Sir||||||1942|||Dwayne.Hackney@HqAli.org|2452579| +8408|AAAAAAAAINACAAAA|795340|5526|37312|2452675|2452645|Miss|Barbara|Richardson|N|3|10|1992|CAYMAN ISLANDS||Barbara.Richardson@pEH5ZbrKykrGV6x.edu|2452646| +8409|AAAAAAAAJNACAAAA|1382052|3716|12133|2450607|2450577|Sir|Larry|Grant|N|8|1|1934|CHRISTMAS ISLAND||Larry.Grant@6XvA0udjj.com|2452567| +8410|AAAAAAAAKNACAAAA|1596789|1696|43003|2449286|2449256|Mrs.|Jacqueline|Petersen|Y|29|12|1957|MOLDOVA, REPUBLIC OF||Jacqueline.Petersen@lz.com|2452441| +8411|AAAAAAAALNACAAAA|688848|3692|8536|2452595|2452565|Miss|Elizabeth|Klein|N|22|3|1952|FIJI||Elizabeth.Klein@hazNvlrtmljlgLbxML6.org|2452546| +8412|AAAAAAAAMNACAAAA|757633|3626|46866|2450631|2450601|Mr.|Edward|Pearl|Y|14|1|1988|AFGHANISTAN||Edward.Pearl@cyknVmy0J.com|2452579| +8413|AAAAAAAANNACAAAA|1895698|5506|27267|2452312|2452282|Sir|Victor|Fitzgerald|N|31|10|1971|ECUADOR||Victor.Fitzgerald@xeneQNQ6Cg.org|2452646| +8414|AAAAAAAAONACAAAA|1907408|3696|19937|2451630|2451600|Mrs.|Taryn|Sample|Y|22|6|1934|BRUNEI DARUSSALAM||Taryn.Sample@aXv.com|2452599| +8415|AAAAAAAAPNACAAAA|1046014|7005|27196|2450587|2450557|Mrs.|Alanna|Smith|Y|6|5|1943|EQUATORIAL GUINEA||Alanna.Smith@epVkpzgrSEO.edu|2452413| +8416|AAAAAAAAAOACAAAA|171332|4794|22256|2449689|2449659|Sir|James|Koch|N|26|7|1935|AZERBAIJAN||James.Koch@UgpVivDbHf4IUuClJ.com|2452599| +8417|AAAAAAAABOACAAAA|1274096|2611|16326|2451570|2451540|Dr.|Donna|Combs|N|17|11|1937|DJIBOUTI||Donna.Combs@HD4k54LDhT0.org|2452578| +8418|AAAAAAAACOACAAAA|732072|1421|22417|2449961|2449931|Dr.|Eric|Riley|N|7|9|1960|ANDORRA||Eric.Riley@yvuiMGAhys.edu|2452588| +8419|AAAAAAAADOACAAAA|885574|3172|15970|2451349|2451319|Dr.|Beatrice|Lackey|Y|25|10|1977|YEMEN||Beatrice.Lackey@lEAsVJQC43cfg.com|2452614| +8420|AAAAAAAAEOACAAAA|19482|2993|26551|2451176|2451146|Dr.|Donald|Stillwell|N|4|1|1941|MONGOLIA||Donald.Stillwell@dqeMgRydZR9f.org|2452454| +8421|AAAAAAAAFOACAAAA|1877572|2769|25966|2452280|2452250|Dr.|John|Green|Y|2|5|1984|INDIA||John.Green@MGdN2SGRoGtHrpvTS7.com|2452323| +8422|AAAAAAAAGOACAAAA|488037|5554|17071|2449970|2449940|Miss|Yolanda|Denton|Y|3|2|1975|UNITED KINGDOM||Yolanda.Denton@tH7MQAegvcO.com|2452540| +8423|AAAAAAAAHOACAAAA|995552|33|48914|2451412|2451382|Mr.|Craig|White|N|7|10|1947|BHUTAN||Craig.White@vCh.org|2452437| +8424|AAAAAAAAIOACAAAA|710204|5205|30559|2450984|2450954|Dr.|Dona|Payne|N|3|5|1980|SLOVAKIA||Dona.Payne@axgNkIQ4J8I.org|2452341| +8425|AAAAAAAAJOACAAAA|1419278|6823|36015|2450905|2450875|Sir|Dale|Charles|N|11|2|1958|NAMIBIA||Dale.Charles@PT0.org|2452480| +8426|AAAAAAAAKOACAAAA|1708568|1888|11655|2449354|2449324|Miss|Tammy|Manns|N|2|4|1965|FRENCH POLYNESIA||Tammy.Manns@1xKkqI7fyA.org|2452627| +8427|AAAAAAAALOACAAAA|1193958|2234|6157|2450549|2450519|Dr.|Margaret|Toscano|Y|27|6|1953|WALLIS AND FUTUNA||Margaret.Toscano@bXD.org|2452295| +8428|AAAAAAAAMOACAAAA|768244|2179|4067|2451565|2451535|Dr.|Tiffany|Cain|Y|16|5|1986|MAURITIUS||Tiffany.Cain@mLCSZv1Ml.com|2452314| +8429|AAAAAAAANOACAAAA|305225|7028|38431|2449708|2449678|Mrs.|Shelly|Kimball|N|23|5|1973|SIERRA LEONE||Shelly.Kimball@o.com|2452407| +8430|AAAAAAAAOOACAAAA|1431155|1075|34948|2451276|2451246|Miss|Ruth|Epps|Y|2|1|1927|SOMALIA||Ruth.Epps@UftTr2K14E.com|2452597| +8431|AAAAAAAAPOACAAAA|392837|5889|21341|2452162|2452132|Ms.|Suzanne|Thompson|Y|22|5|1928|CZECH REPUBLIC||Suzanne.Thompson@bfoBrsQ796Nv.org|2452569| +8432|AAAAAAAAAPACAAAA|1767306|3707|23708|2450768|2450738|Mr.|Joseph|Bruner|Y|8|4|1932|QATAR||Joseph.Bruner@52DaQuKMZAFEMH.edu|2452572| +8433|AAAAAAAABPACAAAA|219222|2099|29729|2450389|2450359|Miss|Carolyn|Bryan|N|1|1|1936|MALAYSIA||Carolyn.Bryan@r7PeJgJiXKO.edu|2452507| +8434|AAAAAAAACPACAAAA|1636373|3806|10486|2451045|2451015|Mrs.|Karen|Thomas|N|30|9|1979|NIGER||Karen.Thomas@XlSByyU4d.com|2452636| +8435|AAAAAAAADPACAAAA|296846|1435|4071|2451299|2451269|Dr.|James|Goodwin|N|1|12|1935|LESOTHO||James.Goodwin@tQ.org|2452597| +8436|AAAAAAAAEPACAAAA|1306998|5389|22980|2449984|2449954|Ms.|Arline|Jefferson|N|11|4|1930|BELGIUM||Arline.Jefferson@V3BFRDgNPT6Up8eu.org|2452565| +8437|AAAAAAAAFPACAAAA|979676|4930|9031|2450866|2450836|Dr.|Dwayne|Watts|N|11|11|1966|MADAGASCAR||Dwayne.Watts@5oqrL.com|2452358| +8438|AAAAAAAAGPACAAAA|46798|6403|20455|2451937|2451907|Dr.|Marlene|Dye|N|16|9|1980|GUINEA-BISSAU||Marlene.Dye@UtjlLYTk4r6iD43DA.com|2452363| +8439|AAAAAAAAHPACAAAA|763659|5464|2869|2449855|2449825|Dr.|Pamela|Carter|N|20|11|1943|BRUNEI DARUSSALAM||Pamela.Carter@ZfT5HSI8q66F0eqI.org|2452639| +8440|AAAAAAAAIPACAAAA|1559225|5614|34403|2451679|2451649|Dr.|Cole|John|Y|15|12|1978|MONTSERRAT||Cole.John@m.edu|2452305| +8441|AAAAAAAAJPACAAAA|12305|1954|38403|2449274|2449244|Miss|Christina|Hernandez|N|3|7|1980|JERSEY||Christina.Hernandez@9YXh.com|2452412| +8442|AAAAAAAAKPACAAAA|902338|6006|32121|2452143||Ms.||Elias||||1967||||2452420| +8443|AAAAAAAALPACAAAA|860392|4857|40475|2451075|2451045|Sir|Nicholas|Crandall|N|13|8|1989|ANTARCTICA||Nicholas.Crandall@KK3YL.com|2452448| +8444|AAAAAAAAMPACAAAA|840260|5287|16761|2449147|2449117|Dr.|Matt|Byrd|N|12|12|1936|NETHERLANDS ANTILLES||Matt.Byrd@iJMcbHg2rf0a.org|2452570| +8445|AAAAAAAANPACAAAA|1138722|3335|748|2451478|2451448|Dr.|Amy|Shafer|Y|20|12|1978|MONTSERRAT||Amy.Shafer@cZC.org|2452426| +8446|AAAAAAAAOPACAAAA|231450|690|12683|2449815|2449785|Sir|Francis|Harris|Y|30|5|1965|BARBADOS||Francis.Harris@7gqIVRE7Pp.org|2452548| +8447|AAAAAAAAPPACAAAA|59418|1176|32181|2451408|2451378|Miss|Jennifer|Simmons|N|8|10|1935|TONGA||Jennifer.Simmons@4aCmRM.org|2452547| +8448|AAAAAAAAAABCAAAA|715055|3886|11917|2451140|2451110|Mr.|William|Vidal|N|3|5|1975|UNITED KINGDOM||William.Vidal@NdsXA.com|2452538| +8449|AAAAAAAABABCAAAA|439802|3468|21087|2452163|2452133|Dr.|Crista|Clifford|N|4|4|1926|AMERICAN SAMOA||Crista.Clifford@jmJ8ZZhcUSJcS.com|2452501| +8450|AAAAAAAACABCAAAA|1823628|4478|13328|2451147|2451117|Dr.|Barbara|Taylor|Y|10|12|1958|GRENADA||Barbara.Taylor@tyVMrHu10POL.org|2452544| +8451|AAAAAAAADABCAAAA|1041389|1051|41160|2452492|2452462|Dr.|Tawanna|Rau|Y|9|7|1960|BELIZE||Tawanna.Rau@8GEYgT5AtiKT.org|2452488| +8452|AAAAAAAAEABCAAAA|910009|7033|41704|2451114|2451084|Dr.|Gregg|Cormier|Y|11|4|1949|BELARUS||Gregg.Cormier@gNIS7FXll58u.org|2452466| +8453|AAAAAAAAFABCAAAA|1262712|5787|16015|2450807|2450777|Dr.|Theodore|Weaver|N|27|3|1929|HONDURAS||Theodore.Weaver@JLrkQmLarCPVp.com|2452538| +8454|AAAAAAAAGABCAAAA|1748329|4757|41185|2450395|2450365|Dr.|Glenn|Stein|N|20|11|1958|LIECHTENSTEIN||Glenn.Stein@Gs1Ck4gVzSmjRvJHGx1.com|2452564| +8455|AAAAAAAAHABCAAAA|394865|5272|18526|2450022|2449992|Miss|Paula|Martin|N|10|10|1971|EGYPT||Paula.Martin@b.edu|2452642| +8456|AAAAAAAAIABCAAAA|1156678|500|27969|2450962|2450932|Mrs.|Katherine|Swann|Y|7|2|1966|WALLIS AND FUTUNA||Katherine.Swann@mRc6efk.org|2452423| +8457|AAAAAAAAJABCAAAA|1621088|1706|37971|2451221|2451191|Mr.|Richard|Hampton|Y|3|12|1936|FIJI||Richard.Hampton@RvaT3Hrs5LY7JJvA.com|2452635| +8458|AAAAAAAAKABCAAAA|273589|3350|22711|2450328|2450298|Dr.|Bernadine|Goodman|Y|8|2|1987|MALAYSIA||Bernadine.Goodman@YR5B3QsP.com|2452515| +8459|AAAAAAAALABCAAAA|238287|1424|13361|2451876|2451846|Ms.|Terri|Jefferson|N|24|9|1992|INDIA||Terri.Jefferson@GAqozr0.edu|2452410| +8460|AAAAAAAAMABCAAAA|740076|1766|47009|2452336|2452306|Dr.|Sara|Vaughan|Y|11|7|1974|TRINIDAD AND TOBAGO||Sara.Vaughan@hUO5.edu|2452647| +8461|AAAAAAAANABCAAAA|737175||47075||2451292||Kendrick|Morales|||4||||Kendrick.Morales@150SKxazLxe1nC.com|| +8462|AAAAAAAAOABCAAAA|1536342|959|41529|2449285|2449255|Ms.|Sandra|Burgess|N|25|12|1935|GUINEA||Sandra.Burgess@0hFTdbrxDsxxHh.edu|2452414| +8463|AAAAAAAAPABCAAAA||928|15744||2450906||Denice|Toledo|Y|||1980|VANUATU|||2452318| +8464|AAAAAAAAABBCAAAA|1812926|5849|6372|2449527|2449497|Miss|Colleen|Masters|N|10|12|1931|SAN MARINO||Colleen.Masters@672rBzu7ogs.com|2452419| +8465|AAAAAAAABBBCAAAA|191801|2220|19839|2452647|2452617|Mrs.|Esther|Fritz|Y|26|9|1940|PAKISTAN||Esther.Fritz@AQtAVhz8l8gK8Cm0L.com|2452289| +8466|AAAAAAAACBBCAAAA|1396622|2188|17001|2450846|2450816|Dr.|Andrea|Gaylord|N|22|2|1926|PAPUA NEW GUINEA||Andrea.Gaylord@VMXfGPbv7dS3YP.edu|2452429| +8467|AAAAAAAADBBCAAAA|424345|6249|26600|2452185|2452155|Dr.|Homer|Hinkle|Y|3|7|1950|SPAIN||Homer.Hinkle@NrG.com|2452525| +8468|AAAAAAAAEBBCAAAA|1758513|3325|49690|2451762|2451732|Dr.|Nicole|Beatty|N|18|12|1932|MARSHALL ISLANDS||Nicole.Beatty@UkGq9cOacOUgf.com|2452631| +8469|AAAAAAAAFBBCAAAA|763110|3004|29251|2450098|2450068|Mrs.|Sharon|Crump|N|18|3|1946|ANGUILLA||Sharon.Crump@aCsSkBXL.edu|2452623| +8470|AAAAAAAAGBBCAAAA|173952|5370|37988|2451779|2451749|Dr.|Roy|Abell|N|13|9|1953|ERITREA||Roy.Abell@RTq4xR3aIDzivZ7.org|2452300| +8471|AAAAAAAAHBBCAAAA|704321|2408|39695|2451744|2451714|Dr.|John|Wallace|Y|28|12|1924|CHRISTMAS ISLAND||John.Wallace@phIpRgHY9rBaFhVy.edu|2452537| +8472|AAAAAAAAIBBCAAAA|974803|6990|28024|2451720|2451690|Dr.|Sandra|Mead|N|29|8|1965|JORDAN||Sandra.Mead@EQM8erG93NkX7Zxpl.org|2452395| +8473|AAAAAAAAJBBCAAAA|167603|5474|14498|2449106|2449076|Sir|Jeff|Rodgers|Y|25|3|1990|BAHAMAS||Jeff.Rodgers@9mf2QmFvzrR.edu|2452331| +8474|AAAAAAAAKBBCAAAA|1143738|4306|43627|2450060|2450030|Sir|Robert|Wilde|N|1|12|1926|NETHERLANDS ANTILLES||Robert.Wilde@GRdF.com|2452326| +8475|AAAAAAAALBBCAAAA|1138172|6197|26622|2449617|2449587|Sir|Clarence|Samuel|Y|25|9|1949|BRAZIL||Clarence.Samuel@IUFgrruE4RKcjD.org|2452345| +8476|AAAAAAAAMBBCAAAA|94749|316|43883|2449817|2449787|Miss|Colleen|Quinn|Y|6|6|1977|FINLAND||Colleen.Quinn@BT5Em3mj9LLX.org|2452352| +8477|AAAAAAAANBBCAAAA|284542|2564|30586|2452566|2452536|Sir|Samuel|Farrell|N|24|3|1960|FINLAND||Samuel.Farrell@Jmp.com|2452337| +8478|AAAAAAAAOBBCAAAA|1875903|6190|430|2449308|2449278|Mrs.|Lori|Allen|Y|14|5|1942|GUAM||Lori.Allen@bHppyixUTVdmQ.edu|2452493| +8479|AAAAAAAAPBBCAAAA|1682279|5401|17676|2451448|2451418|Mrs.|Pauline|Adamson|N|2|7|1970|IRAQ||Pauline.Adamson@dPRnv.com|2452303| +8480|AAAAAAAAACBCAAAA|972755|7122|36689|2449240|2449210|Sir|Jason|Boyd|N|4|7|1927|FINLAND||Jason.Boyd@9zS.org|2452410| +8481|AAAAAAAABCBCAAAA|369866|3332|30798|2450724|2450694|Sir|Billy|Davis|Y|29|3|1965|ALAND ISLANDS||Billy.Davis@SOAEI.com|2452319| +8482|AAAAAAAACCBCAAAA|1573485|4756|16027|2452213|2452183|Miss|Gloria|Banks|N|26|11|1941|SAMOA||Gloria.Banks@BaFKaT.org|2452481| +8483|AAAAAAAADCBCAAAA|950419|4429|12039|2449706|2449676|Sir|Joshua|Findley|Y|4|10|1954|LUXEMBOURG||Joshua.Findley@IEve4j1yjqQbl.com|2452588| +8484|AAAAAAAAECBCAAAA|688928|7148|36568|2452491|2452461|Miss|Vickie|Rainey|Y|24|2|1927|EQUATORIAL GUINEA||Vickie.Rainey@vnmQ4URd3F.org|2452574| +8485|AAAAAAAAFCBCAAAA|1434306|1433|2160|2450274|2450244|Miss|Ione|Farnsworth|N|17|5|1953|TAJIKISTAN||Ione.Farnsworth@ijXb.org|2452458| +8486|AAAAAAAAGCBCAAAA|1261436|6317|28751|2451388|2451358|Sir|William|Spicer|N|18|6|1948|KUWAIT||William.Spicer@hTE8.com|2452626| +8487|AAAAAAAAHCBCAAAA|1418931|1203|23475|2450672|2450642|Sir|Michael|Welch|N|18|3|1986|AFGHANISTAN||Michael.Welch@b.org|2452516| +8488|AAAAAAAAICBCAAAA|1397815|1021|13427|2451839|2451809|Mrs.|Vilma|Hayden|Y|6|4|1967|GUAM||Vilma.Hayden@U8SvZdKBb86e8x13.edu|2452629| +8489|AAAAAAAAJCBCAAAA|568132|6756|43780|2450423|2450393|Ms.|Sara|Harrison|Y|17|3|1960|KIRIBATI||Sara.Harrison@NpsnHihCZSb.edu|2452312| +8490|AAAAAAAAKCBCAAAA|1870151|4538|30172|2449966|2449936|Sir|Oliver|Goodman|N|8|11|1939|EL SALVADOR||Oliver.Goodman@xSXaGJO0yIIkznjay.com|2452500| +8491|AAAAAAAALCBCAAAA|1801558|5930|18749|2452437|2452407|Dr.|Jamie|Prater|N|29|3|1992|SRI LANKA||Jamie.Prater@ccBeOkHKGj.edu|2452571| +8492|AAAAAAAAMCBCAAAA|235217|2588|45112|2451245|2451215|Ms.|Florence|Gonzales|N|29|11|1924|CYPRUS||Florence.Gonzales@1A.org|2452431| +8493|AAAAAAAANCBCAAAA|1762385|1008|41231|2451622|2451592|Ms.|Lillian|Oneal|Y|13|5|1981|LEBANON||Lillian.Oneal@sr1UGE7qKs.com|2452332| +8494|AAAAAAAAOCBCAAAA|17676|2581|26686|2449298|2449268|Mr.|Mark|Trapp|Y|18|7|1953|MALAYSIA||Mark.Trapp@sUT93F.org|2452421| +8495|AAAAAAAAPCBCAAAA|1204517|1678|34439|2450151|2450121|Ms.|Bertha|Martinez|Y|9|8|1953|ARGENTINA||Bertha.Martinez@bZYNLdtCcVcd.com|2452322| +8496|AAAAAAAAADBCAAAA|1672182|993|33851|2449757|2449727|Ms.|Sheila|Pearson|Y|3|10|1973|BELARUS||Sheila.Pearson@mQQuyuktP0LMvG102y.com|2452500| +8497|AAAAAAAABDBCAAAA|1643182||24234|2451036||Ms.|Suzette||N|27||1948||||2452522| +8498|AAAAAAAACDBCAAAA|215048|4628|47808|2450076|2450046|Mr.|Cyrus|Dumont|N|13|8|1980|PAKISTAN||Cyrus.Dumont@UtHBkOEkD.com|2452283| +8499|AAAAAAAADDBCAAAA|1597775|4155|26888|2450069|2450039|Sir|Victor|Ratcliff|Y|13|11|1957|TOKELAU||Victor.Ratcliff@d9DqzyhOgekO.org|2452430| +8500|AAAAAAAAEDBCAAAA|25392|5401|6563|2450604|2450574|Miss|Myrtle|Jackson|N|13|9|1927|BERMUDA||Myrtle.Jackson@rbPIhRJ.com|2452301| +8501|AAAAAAAAFDBCAAAA|1351363|1751|39386|2450567|2450537|Mrs.|Suzanne|Parrish|Y|27|7|1941|FRANCE||Suzanne.Parrish@s649hMpzG.edu|2452416| +8502|AAAAAAAAGDBCAAAA|1620024|5698|45772|2449510|2449480|Mr.|Theodore|Horton|N|23|11|1980|GUINEA-BISSAU||Theodore.Horton@N7a27Q9Zir.org|2452296| +8503|AAAAAAAAHDBCAAAA|1794385|3260|10135|2451663|2451633|Dr.|Jason|Mcreynolds|Y|5|7|1980|KIRIBATI||Jason.Mcreynolds@duqM6sdLTjnZ.com|2452645| +8504|AAAAAAAAIDBCAAAA|1001307|4740|34620|2450951|2450921|Mr.|James|Stone|Y|24|7|1938|THAILAND||James.Stone@SNXIuF3tKgu.org|2452595| +8505|AAAAAAAAJDBCAAAA|719716|6244|31525|2452288|2452258|Sir|Gerard|Davis|N|24|4|1942|COSTA RICA||Gerard.Davis@ajj3lJ.org|2452622| +8506|AAAAAAAAKDBCAAAA||2316|13237|2451865|||Jessica|||5|||CZECH REPUBLIC||Jessica.Payne@zE.org|| +8507|AAAAAAAALDBCAAAA|1268494|1302|9710|2450389|2450359|Sir|Jeremy|Patton|Y|21|7|1974|MALAYSIA||Jeremy.Patton@dIb.com|2452465| +8508|AAAAAAAAMDBCAAAA|1185762|6792|30222|2449728|2449698|Miss|Aaron|Williams|N|28|3|1990|WALLIS AND FUTUNA||Aaron.Williams@ZjVMd.com|2452545| +8509|AAAAAAAANDBCAAAA|388050|3851|3605|2449338|2449308|Dr.|Gloria|Morse|Y|13|8|1957|FINLAND||Gloria.Morse@vblcDfdCepuycovpI.org|2452328| +8510|AAAAAAAAODBCAAAA|132731|6330|40729|2449417|2449387|Mr.|Gary|Davis|N|20|3|1942|SINGAPORE||Gary.Davis@J7Sg.org|2452378| +8511|AAAAAAAAPDBCAAAA|1598054|5393|1932|2450465|2450435|Sir|Jose|Turner|Y|27|3|1967|BHUTAN||Jose.Turner@Ep4fxy14egM.edu|2452622| +8512|AAAAAAAAAEBCAAAA|500885|3634|41536|2451833|2451803|Dr.|Ronald|Haines|Y|11|9|1942|SWITZERLAND||Ronald.Haines@28ePL5AzbcFeR.com|2452478| +8513|AAAAAAAABEBCAAAA|1532482|4183|23084|2450512|2450482|Ms.|Gwendolyn|Rangel|N|10|8|1976|GUYANA||Gwendolyn.Rangel@Cx.org|2452473| +8514|AAAAAAAACEBCAAAA|726390|5877|2564|2451412|2451382|Mr.|Felipe|Strong|Y|28|3|1931|ETHIOPIA||Felipe.Strong@JEc0.com|2452618| +8515|AAAAAAAADEBCAAAA|1235151|2495|17792|2450207|2450177|Ms.|Genevieve|Mendez|Y|24|7|1933|FAROE ISLANDS||Genevieve.Mendez@Mb9iMJ7szkeJ.org|2452458| +8516|AAAAAAAAEEBCAAAA|114821|1049|11277|2452610|2452580|Mr.|Orlando|Dennis|Y|16|1|1973|BERMUDA||Orlando.Dennis@RRkpp7CIhJSgj9Og.com|2452369| +8517|AAAAAAAAFEBCAAAA|302798|4456|22914|2452385|2452355|Mr.|Lyle|Ricci|Y|5|2|1978|MALTA||Lyle.Ricci@OTaLn9TEHh8AqSQCZ.org|2452545| +8518|AAAAAAAAGEBCAAAA|1344894|4826|38758|2451672|2451642|Sir|John|Knight|N|7|4|1931|KENYA||John.Knight@pjvTrkxi.org|2452363| +8519|AAAAAAAAHEBCAAAA|||33298|2451881||||Hudgins||30||1964||||| +8520|AAAAAAAAIEBCAAAA|1369003|4110|7970|2452108|2452078|Mr.|Charles|Miles|N|21|2|1970|TONGA||Charles.Miles@Nece82sS.org|2452412| +8521|AAAAAAAAJEBCAAAA|1597872|2716|1960|2450696|2450666|Ms.|Alise|Canada|N|30|9|1985|SLOVENIA||Alise.Canada@dF3n.com|2452532| +8522|AAAAAAAAKEBCAAAA|836407|6726|47798|2449928|2449898|Miss|Colleen|Wells|Y|17|4|1959|MADAGASCAR||Colleen.Wells@p3VMIF7plzibmo.org|2452623| +8523|AAAAAAAALEBCAAAA|934884|1217|28750|2451753|2451723|Dr.|Dustin|Neal|Y|17|10|1938|SLOVAKIA||Dustin.Neal@aQ.com|2452580| +8524|AAAAAAAAMEBCAAAA|637481|4418|25355|2449532|2449502|Dr.|Inez|Hunter|Y|23|7|1951|GUINEA||Inez.Hunter@znDcsH.com|2452500| +8525|AAAAAAAANEBCAAAA|1313135|4791|17647|2451887|2451857|Miss|Alexander|Harvey|N|5|11|1972|GIBRALTAR||Alexander.Harvey@kO1N5I9C.edu|2452500| +8526|AAAAAAAAOEBCAAAA|661818|7009|41400|2450126|2450096|Ms.|Sarah|Roberts|Y|5|8|1976|FAROE ISLANDS||Sarah.Roberts@eNJtSZB0kMpI.com|2452509| +8527|AAAAAAAAPEBCAAAA|497119|6593|20067|2451438|2451408|Dr.|Kevin|Paul|Y|16|8|1982|INDIA||Kevin.Paul@qfQFO02oqDms38XD.edu|2452485| +8528|AAAAAAAAAFBCAAAA|1813891|831|5771|2449922|2449892|Mr.|Charles|Love|N|13|6|1956|MONTENEGRO||Charles.Love@KoLj9v.org|2452561| +8529|AAAAAAAABFBCAAAA|1556699|7066|14233|2451269|2451239|Miss|Donna|Little|Y|19|10|1958|BARBADOS||Donna.Little@gInV.org|2452575| +8530|AAAAAAAACFBCAAAA|1227954|5664|29821|2452635|2452605|Mrs.|Thelma|Young|Y|20|12|1953|SINGAPORE||Thelma.Young@nm6.com|2452448| +8531|AAAAAAAADFBCAAAA|1593682|5431|9580|2451194|2451164|Miss|Samantha|Harmon|Y|21|9|1988|LEBANON||Samantha.Harmon@GKgmG.org|2452412| +8532|AAAAAAAAEFBCAAAA|1905860|3288|9069|2450472|2450442|Mr.|Joseph|Altman|N|24|10|1967|ETHIOPIA||Joseph.Altman@M3CmIPzhKL6L1q47p.edu|2452622| +8533|AAAAAAAAFFBCAAAA|643963|930|31944|2451840|2451810|Ms.|Michelle|Mccabe|N|17|10|1985|MOZAMBIQUE||Michelle.Mccabe@tv4NKrQaigJqXj.org|2452618| +8534|AAAAAAAAGFBCAAAA|434017|2666|4115|2449802|2449772|Dr.|William|Keller|Y|27|10|1957|BAHAMAS||William.Keller@PK5GpJ9TP315TnX.com|2452514| +8535|AAAAAAAAHFBCAAAA|117317|6670|2023|2452517|2452487|Mrs.|Keri|Bernard|N|22|10|1962|UNITED STATES||Keri.Bernard@zyT8R07glU.edu|2452384| +8536|AAAAAAAAIFBCAAAA|327430|641|45899|2449869|2449839|Dr.|Cassandra|Morrison|N|18|12|1987|ZAMBIA||Cassandra.Morrison@LSRV2.edu|2452470| +8537|AAAAAAAAJFBCAAAA|1708021|2341|3461|2449570|2449540|Mrs.|Rose|King|Y|23|10|1930|GERMANY||Rose.King@sK.edu|2452541| +8538|AAAAAAAAKFBCAAAA|252846|2461|4876|2450319|2450289|Sir|James|Schaefer|N|22|5|1984|SWEDEN||James.Schaefer@dudYk.org|2452367| +8539|AAAAAAAALFBCAAAA|1205309|6484|14059|2452207|2452177|Dr.|Gregory|Dietrich|Y|6|10|1950|MOLDOVA, REPUBLIC OF||Gregory.Dietrich@Dj9UktVXnj2kSdkK.edu|2452433| +8540|AAAAAAAAMFBCAAAA|1886684|1408|36579|2451211|2451181|Dr.|Jonie|Williford|Y|29|7|1926|GREENLAND||Jonie.Williford@z9TzslFvAU.com|2452380| +8541|AAAAAAAANFBCAAAA|1883552|7156|47513|2450082|2450052|Dr.|Angela|Wiseman|N|13|6|1948|SAMOA||Angela.Wiseman@Rtl9Lh0cDQbqVufa.org|2452422| +8542|AAAAAAAAOFBCAAAA|944180|4113|36846|2450095|2450065|Mrs.|Ethel|Schneider|Y|20|3|1941|TIMOR-LESTE||Ethel.Schneider@IfeD7XihKK9h80l.com|2452382| +8543|AAAAAAAAPFBCAAAA|547522|1315|29061|2450149|2450119|Dr.|Susan|Bradshaw|Y|3|2|1984|ALGERIA||Susan.Bradshaw@a4ra5h76GuD.org|2452476| +8544|AAAAAAAAAGBCAAAA|510451|3187|9970|2450034|2450004|Sir|Michael|Dupont|Y|23|6|1979|MALTA||Michael.Dupont@nGlG.org|2452450| +8545|AAAAAAAABGBCAAAA|644776|425|16874|2449545|2449515|Sir|Timothy|Ferrari|Y|7|4|1984|BOTSWANA||Timothy.Ferrari@4XYyg.com|2452317| +8546|AAAAAAAACGBCAAAA|519218|4932|3093|2449638|2449608|Mr.|Rickie|Benitez|Y|19|9|1989|ITALY||Rickie.Benitez@G.com|2452300| +8547|AAAAAAAADGBCAAAA|1723052|6305|21911|2449281|2449251|Mrs.|Elizabeth|Catron|N|13|9|1936|MARSHALL ISLANDS||Elizabeth.Catron@sv6J3gP.org|2452396| +8548|AAAAAAAAEGBCAAAA|1781991||13508|2449170||Dr.|Max||Y|8|2|1957||||| +8549|AAAAAAAAFGBCAAAA|47994|693|46145|2450627|2450597|Miss|Violet|Rivera|N|15|9|1969|SUDAN||Violet.Rivera@0j9TKjcik.com|2452404| +8550|AAAAAAAAGGBCAAAA|546272|6485|5098|2449611|2449581|Sir|Jonathan|Ulrich|Y|16|2|1988|EL SALVADOR||Jonathan.Ulrich@UnR.com|2452636| +8551|AAAAAAAAHGBCAAAA|160719|6661|5756|2449575|2449545|Dr.|Bryan|Adams|Y|14|8|1968|MAYOTTE||Bryan.Adams@vhvss.org|2452629| +8552|AAAAAAAAIGBCAAAA|1193733|2076|6372|2450212|2450182|Dr.|James|Erickson|Y|22|7|1935|GHANA||James.Erickson@db6VEIpu.edu|2452426| +8553|AAAAAAAAJGBCAAAA|93737|6087|6884|2449818|2449788|Miss|Annie|Mack|Y|29|11|1988|TURKMENISTAN||Annie.Mack@KsvT4M8c.org|2452373| +8554|AAAAAAAAKGBCAAAA||5890|38395|||Sir|Phil|Johnson|N||||KAZAKHSTAN||Phil.Johnson@1YImqO.org|| +8555|AAAAAAAALGBCAAAA|1569858|6169|25233|2451793|2451763|Dr.|Debra|Burr|Y|15|4|1937|MACAO||Debra.Burr@DbF0d1czso35AO.edu|2452448| +8556|AAAAAAAAMGBCAAAA|||46147|||Sir|Earl|Riley|Y|4||1927|TUNISIA||Earl.Riley@xxIiMZ3am.com|| +8557|AAAAAAAANGBCAAAA||967|30997|2451482|2451452||Charlotte|Tompkins|||1||||Charlotte.Tompkins@kuzmMazre.com|2452629| +8558|AAAAAAAAOGBCAAAA|73486|5801|38069|2450656|2450626|Mr.|Willie|Gentry|Y|30|12|1964|TUNISIA||Willie.Gentry@0VDCQ9F.edu|2452641| +8559|AAAAAAAAPGBCAAAA|498402|4024|19199|2449405|2449375|Miss|Felicita|Holland|N|14|9|1935|CAMEROON||Felicita.Holland@2U6IIBu7iqjX0.edu|2452490| +8560|AAAAAAAAAHBCAAAA|1221515|355|25995|2449147|2449117|Miss|Fannie|Thomas|N|31|12|1960|GAMBIA||Fannie.Thomas@GfBxGTOT8zXJ08DDl.com|2452493| +8561|AAAAAAAABHBCAAAA|615837|6430|32623|2449055|2449025|Mrs.|Jeanetta|Massie|Y|11|7|1968|NEW ZEALAND||Jeanetta.Massie@QEs4J.org|2452619| +8562|AAAAAAAACHBCAAAA|591673|3589|22705|2450719|2450689|Mrs.|Linda|Kraft|Y|24|2|1945|TUNISIA||Linda.Kraft@dY.org|2452284| +8563|AAAAAAAADHBCAAAA|200975|538|24948|2449417|2449387|Mr.|Sheldon|Buckner|N|8|4|1975|LIECHTENSTEIN||Sheldon.Buckner@b57i3.edu|2452571| +8564|AAAAAAAAEHBCAAAA|616915|4311|32107|2449874|2449844|Dr.|James|Pearson|Y|16|3|1992|FRENCH POLYNESIA||James.Pearson@UIQrJNvHO.org|2452323| +8565|AAAAAAAAFHBCAAAA|915456|5464|20672|2450563|2450533|Mrs.|Lois|Stott|Y|23|2|1944|GRENADA||Lois.Stott@4uRAq.com|2452583| +8566|AAAAAAAAGHBCAAAA|844142|4969|26996|2452346|2452316|Sir|Frank|Royal|N|23|10|1934|ESTONIA||Frank.Royal@c3g1s4yGSfege.edu|2452585| +8567|AAAAAAAAHHBCAAAA|1616166|6194|20622|2449695|2449665|Miss|Kathy|Speed|N|13|10|1948|SOLOMON ISLANDS||Kathy.Speed@xHoszfdzg.edu|2452341| +8568|AAAAAAAAIHBCAAAA|1464582|5957|29332|2450244|2450214|Ms.|Elisabeth|Myers|Y|11|3|1990|TRINIDAD AND TOBAGO||Elisabeth.Myers@s48JLkJ.com|2452579| +8569|AAAAAAAAJHBCAAAA|1477638|2497|32509|2449656|2449626|Sir|David|Bishop|Y|18|2|1934|GUADELOUPE||David.Bishop@2U5.com|2452507| +8570|AAAAAAAAKHBCAAAA|1338477|3094|22558|2451042|2451012|Mrs.|Andrew|Young|Y|2|10|1977|LUXEMBOURG||Andrew.Young@0PH2anbCzlM9B.com|2452514| +8571|AAAAAAAALHBCAAAA|958842|6526|963|2452327|2452297|Sir|James|Crawford|Y|17|9|1945|BRUNEI DARUSSALAM||James.Crawford@LDmPRcKPd6t3XFk95m.edu|2452355| +8572|AAAAAAAAMHBCAAAA|835542|6945|14748|2449990|2449960|Sir|Kevin|Mobley|N|10|6|1992|PAPUA NEW GUINEA||Kevin.Mobley@x232.org|2452288| +8573|AAAAAAAANHBCAAAA|1614771|3667|31410|2450265|2450235|Ms.|Nicole|Reed|Y|19|12|1964|THAILAND||Nicole.Reed@kl7k.com|2452403| +8574|AAAAAAAAOHBCAAAA|835792|4716|5240|2450447|2450417|Ms.|Bertha|James|Y|18|3|1937|HONG KONG||Bertha.James@sMnXy85r2.edu|2452639| +8575|AAAAAAAAPHBCAAAA|1555875|4162|38896|2451202|2451172|Sir|John|Riggins|N|22|4|1924|JAMAICA||John.Riggins@zH7ILqLNYNnN2.com|2452405| +8576|AAAAAAAAAIBCAAAA|57364|1843|35522|2451741|2451711|Mrs.|Melissa|Dougherty|N|12|3|1973|DENMARK||Melissa.Dougherty@DDp7BB0fIPPh.com|2452297| +8577|AAAAAAAABIBCAAAA|1805687|2652|45217|2449858|2449828|Dr.|Deborah|Dunn|Y|16|7|1933|ERITREA||Deborah.Dunn@82bxVdzUgiDUyoRt.org|2452413| +8578|AAAAAAAACIBCAAAA|279359|4843|15880|2451011|2450981|Sir|George|Mason|N|23|2|1939|BAHRAIN||George.Mason@S.com|2452528| +8579|AAAAAAAADIBCAAAA|18962|4273|14431|2450941|2450911|Mr.|Ryan|Johnson|N|30|4|1931|KENYA||Ryan.Johnson@105uRYU8zVv.org|2452478| +8580|AAAAAAAAEIBCAAAA|814463|1396|44329|2451875|2451845|Dr.|Richard|Hannah|N|6|11|1970|WESTERN SAHARA||Richard.Hannah@xziXKxSsHyv.edu|2452589| +8581|AAAAAAAAFIBCAAAA|1044999|1962|25725|2451646|2451616|Mr.|Frank|Mccarty|N|2|2|1930|MACAO||Frank.Mccarty@p1RmF.com|2452554| +8582|AAAAAAAAGIBCAAAA|183800|5699|9951|2449337|2449307|Mrs.|Barbara|Cooper|Y|18|8|1936|BURKINA FASO||Barbara.Cooper@0yX1VRjM7B.edu|2452453| +8583|AAAAAAAAHIBCAAAA|187967|1853|11339|2450934|2450904|Dr.|Tara|Haase|N|12|12|1927|KIRIBATI||Tara.Haase@zKOYGaqGqNUn3.edu|2452410| +8584|AAAAAAAAIIBCAAAA|1324083|6558|10586|2451822|2451792|Mrs.|Annette|Conrad|Y|19|11|1952|ALAND ISLANDS||Annette.Conrad@VLJ69RAnTX.edu|2452309| +8585|AAAAAAAAJIBCAAAA|1727047|1517|22034|2451938|2451908|Mr.|Kenneth|Jarrett|Y|17|3|1941|NETHERLANDS ANTILLES||Kenneth.Jarrett@bm71F4fXKn.edu|2452577| +8586|AAAAAAAAKIBCAAAA|1165496|5613|45777|2451234|2451204|Mr.|Lowell|Palmer|Y|1|10|1981|KIRIBATI||Lowell.Palmer@prni6bjAl.com|2452622| +8587|AAAAAAAALIBCAAAA|1649800|1396|5439|2450427|2450397|Dr.|James|Witt|N|14|8|1981|ISRAEL||James.Witt@nYZ.com|2452364| +8588|AAAAAAAAMIBCAAAA|924896|3819|19495|2451174|2451144|Ms.|Teresa|King|Y|29|11|1941|NEW ZEALAND||Teresa.King@R44A8AJ9u49I6kN.com|2452316| +8589|AAAAAAAANIBCAAAA|651043|6201|24159|2450089|2450059|Dr.|Samuel|Ray|N|30|7|1928|QATAR||Samuel.Ray@QAScl450vQe.com|2452345| +8590|AAAAAAAAOIBCAAAA|196188|126|17264|2451458|2451428|Mrs.|Teresa|Young|Y|18|10|1952|UZBEKISTAN||Teresa.Young@zZkGxYAmjxINux.org|2452342| +8591|AAAAAAAAPIBCAAAA|1467725|4299|19273|2452517|2452487|Mr.|Timothy|Sharp|Y|23|1|1977|ANDORRA||Timothy.Sharp@QEy8z7YOjNtR.edu|2452442| +8592|AAAAAAAAAJBCAAAA|1594630|6715|14800|2449268|2449238|Mr.|Brian|Beard|N|22|10|1986|MAURITANIA||Brian.Beard@X1PU.com|2452441| +8593|AAAAAAAABJBCAAAA|761219|6796|40726|2450057|2450027|Dr.|Melissa|Larson|N|14|2|1968|MYANMAR||Melissa.Larson@fANKqn.edu|2452610| +8594|AAAAAAAACJBCAAAA|1146680|1370|16413|2451280|2451250|Mr.|Patrick|Thompson|N|28|8|1974|VIET NAM||Patrick.Thompson@I4bCArro.org|2452536| +8595|AAAAAAAADJBCAAAA|1690923|1263|45556|2451811|2451781|Dr.|Matthew|Hunt|Y|6|12|1971|JAPAN||Matthew.Hunt@JqBOSBNal.org|2452438| +8596|AAAAAAAAEJBCAAAA|1185206|4128|9050|2452193|2452163|Miss|Jenny|Ketchum|N|22|7|1982|JERSEY||Jenny.Ketchum@7kc4hpY51gQ.edu|2452380| +8597|AAAAAAAAFJBCAAAA|335622|1229|22883|2451777|2451747|Mr.|Max|Williams|N|14|6|1972|BULGARIA||Max.Williams@I9EcNBlECcqoE.com|2452513| +8598|AAAAAAAAGJBCAAAA|488680|4192|38857|2449895|2449865|Dr.|Jerry|Laroche|Y|2|8|1947|BRAZIL||Jerry.Laroche@gU.edu|2452300| +8599|AAAAAAAAHJBCAAAA|1400719|556|5146|2449952|2449922|Sir|James|Hammer|N|2|11|1988|AZERBAIJAN||James.Hammer@id.com|2452509| +8600|AAAAAAAAIJBCAAAA|785495|5618|28541|2449974|2449944|Miss|Genevieve|Williams|N|6|2|1966|SOUTH AFRICA||Genevieve.Williams@MjmibPL.com|2452293| +8601|AAAAAAAAJJBCAAAA|349916|3989|48862|2449258|2449228|Mrs.|Kristin|Bunker|Y|11|11|1966|GUATEMALA||Kristin.Bunker@lGfBhY0uBkVqYB.edu|2452492| +8602|AAAAAAAAKJBCAAAA|718250|1222|38947|2451361|2451331|Miss|Ione|Bailey|N|29|3|1932|GUERNSEY||Ione.Bailey@F50F2kUCLi.edu|2452414| +8603|AAAAAAAALJBCAAAA|114643|5592|15294|2451881|2451851|Ms.|Judith|Baker|N|28|5|1935|MONTSERRAT||Judith.Baker@tZ5N3zn4juNiHXtERk.com|2452533| +8604|AAAAAAAAMJBCAAAA|942705|6577|16449|2451711|2451681|Dr.|Misty|Thompson|N|12|3|1974|AZERBAIJAN||Misty.Thompson@2QyijGD64e.org|2452643| +8605|AAAAAAAANJBCAAAA|1825690|7195|18747|2451403|2451373|Miss|Ruth|Caldwell|N|21|12|1943|GUYANA||Ruth.Caldwell@X1Ok.org|2452632| +8606|AAAAAAAAOJBCAAAA|1128603|3953|25303|2452369|2452339|Mrs.|Nicole|Baggett|N|21|2|1955|NETHERLANDS||Nicole.Baggett@4IrVFvT9irVd.edu|2452413| +8607|AAAAAAAAPJBCAAAA|1640545|19|8293|2451263|2451233|Dr.|Ann|Harvey|Y|12|10|1975|PANAMA||Ann.Harvey@1mOnBfMxrJezvD1.com|2452392| +8608|AAAAAAAAAKBCAAAA|691629|1277|16972|2450443|2450413|Mr.|Lee|Willis|N|9|10|1977|CAPE VERDE||Lee.Willis@ohY.com|2452500| +8609|AAAAAAAABKBCAAAA|1727434|5654|17827|2451922|2451892|Dr.|David|Hughes|N|21|5|1927|KENYA||David.Hughes@NMRP3VP55.org|2452643| +8610|AAAAAAAACKBCAAAA|992814|4828|11195|2450563|2450533|Miss|Brenda|Chapman|Y|29|5|1963|YEMEN||Brenda.Chapman@aVMOcgO.com|2452403| +8611|AAAAAAAADKBCAAAA|1424681|5014|35394|2450544|2450514|Dr.|Karon|Jorgenson|Y|18|8|1941|WESTERN SAHARA||Karon.Jorgenson@ogYx.org|2452584| +8612|AAAAAAAAEKBCAAAA|1113986|2862|4891|2450162|2450132|Dr.|Amy|Lopez|Y|16|5|1955|CAMBODIA||Amy.Lopez@YueRd1J7veM0d.org|2452326| +8613|AAAAAAAAFKBCAAAA|1755391|6615|14686|2449971|2449941|Sir|Howard|Allred|Y|18|9|1969|CAMBODIA||Howard.Allred@9pT7bukgsF.edu|2452487| +8614|AAAAAAAAGKBCAAAA|1340572|5810|2648|2450802|2450772|Sir|John|Stewart|Y|14|5|1954|ANTIGUA AND BARBUDA||John.Stewart@BDLyZzpCSTk0.org|2452383| +8615|AAAAAAAAHKBCAAAA|1123712|4439|49566|2452005|2451975|Miss|Cristal|Barnard|Y|20|1|1950|FINLAND||Cristal.Barnard@7g.edu|2452469| +8616|AAAAAAAAIKBCAAAA|286676|2900|17971|2451359|2451329|Sir|John|Henderson|N|3|8|1938|UGANDA||John.Henderson@ykLa7Y684.com|2452617| +8617|AAAAAAAAJKBCAAAA|696335|4430|49372|2451290|2451260|Miss|Stephanie|Ewing|N|29|4|1942|GUINEA||Stephanie.Ewing@KMgTBs.org|2452486| +8618|AAAAAAAAKKBCAAAA|640460|1479|42451|2452367|2452337|Mr.|Chance|Harris|Y|27|1|1957|SIERRA LEONE||Chance.Harris@Es50218XcqkC5n.org|2452453| +8619|AAAAAAAALKBCAAAA|355285|659|22786|2450143|2450113|Sir|Gary|Johnson|Y|6|7|1987|ISRAEL||Gary.Johnson@C4JjeMeQOe.edu|2452547| +8620|AAAAAAAAMKBCAAAA|233148|3033|10918|2452649|2452619|Ms.|Lori|Nason|N|16|4|1973|LIECHTENSTEIN||Lori.Nason@HPG17ir63uDzTyqjfh.edu|2452548| +8621|AAAAAAAANKBCAAAA|1657381|1470|47559|2451536|2451506|Dr.|Jo|Thompson|Y|1|8|1985|GUERNSEY||Jo.Thompson@uBp7Vtd.edu|2452563| +8622|AAAAAAAAOKBCAAAA|603297|2339|42628|2450431|2450401|Mrs.|Tamara|Mack|Y|12|9|1936|TAJIKISTAN||Tamara.Mack@Ozp636SSSS.org|2452600| +8623|AAAAAAAAPKBCAAAA|1881535|3092|5449|2451377|2451347|Dr.|Clarence|Withrow|Y|15|12|1991|NETHERLANDS||Clarence.Withrow@qjJ.com|2452577| +8624|AAAAAAAAALBCAAAA|1907278|1452|11673|2452246|2452216|Mrs.|Allen|Littlefield|N|8|11|1978|VANUATU||Allen.Littlefield@fjqSSFeN7AapKiU.org|2452339| +8625|AAAAAAAABLBCAAAA|1261233|5740|43400|2451268|2451238|Mrs.|Felicita|Paulson|Y|11|11|1928|INDIA||Felicita.Paulson@YFXYRjr83ShPa8zgdh.org|2452291| +8626|AAAAAAAACLBCAAAA|1314637|4063|24642|2450187|2450157|Dr.|Terry|Rivera|Y|18|2|1992|VIET NAM||Terry.Rivera@5g7tr3ukVoOxUV5D.edu|2452318| +8627|AAAAAAAADLBCAAAA|106721|2326|17578|2451223|2451193|Dr.|Edna|Eaton|N|26|5|1943|WESTERN SAHARA||Edna.Eaton@GxBKj2sGtoXJo.org|2452327| +8628|AAAAAAAAELBCAAAA|1241862|6181|19347|2449627|2449597|Sir|Greg|Cisneros|N|11|3|1971|ITALY||Greg.Cisneros@5Yz0fINrDrDqn3.com|2452309| +8629|AAAAAAAAFLBCAAAA|824336|5333|28377|2449951|2449921|Ms.|Margaret|Roth|Y|12|3|1975|BANGLADESH||Margaret.Roth@a7P.com|2452417| +8630|AAAAAAAAGLBCAAAA|683812|5958|7304|2451685|2451655|Mr.|Adolph|Brown|Y|12|5|1943|ALGERIA||Adolph.Brown@Vjn3JO5.edu|2452394| +8631|AAAAAAAAHLBCAAAA|842828|6137|43846|2450209|2450179|Sir|Russell|Kraus|Y|9|11|1962|KAZAKHSTAN||Russell.Kraus@4MMvgj.org|2452554| +8632|AAAAAAAAILBCAAAA|1583658|4185|45159|2451974|2451944|Miss|Tammy|York|Y|24|4|1938|SENEGAL||Tammy.York@OtX81.edu|2452544| +8633|AAAAAAAAJLBCAAAA|765694|1603|31912|2450852|2450822|Sir|Bryon|Archer|N|14|5|1974|BERMUDA||Bryon.Archer@hPO6opzaKM5liS1a.org|2452283| +8634|AAAAAAAAKLBCAAAA|380837|2435|4880|2450113|2450083|Dr.|Lisa|Thomas|N|19|10|1931|MARSHALL ISLANDS||Lisa.Thomas@3.edu|2452516| +8635|AAAAAAAALLBCAAAA|151921|3214|8154|2450014|2449984|Dr.|Catherine|Baird|Y|24|3|1969|KYRGYZSTAN||Catherine.Baird@OiyZ.com|2452353| +8636|AAAAAAAAMLBCAAAA||1918|22195||||Karla||Y||10||MADAGASCAR||Karla.Feldman@UbTEbpq8zxQcNJ5.org|| +8637|AAAAAAAANLBCAAAA|813241|362|49083|2450739|2450709|Mrs.|Marie|Lane|N|28|9|1977|NAURU||Marie.Lane@U0XlVkZOQy.edu|2452376| +8638|AAAAAAAAOLBCAAAA|1054062|5713|30117|2452278|2452248|Miss|Cheryl|Vest|Y|7|12|1941|FRENCH POLYNESIA||Cheryl.Vest@UiVqhb3dVR.org|2452647| +8639|AAAAAAAAPLBCAAAA|323736|3785|14187|2451173|2451143|Dr.|Michael|Moore|Y|5|9|1969|NICARAGUA||Michael.Moore@3.com|2452444| +8640|AAAAAAAAAMBCAAAA|1858704|4613|4875|2452231|2452201|Dr.|Akiko|Ricketts|N|7|9|1992|MONGOLIA||Akiko.Ricketts@dbPrsPKZK.edu|2452347| +8641|AAAAAAAABMBCAAAA|1093224|3948|22272|2449145|2449115|Dr.|Dave|Clark|Y|19|7|1927|TOGO||Dave.Clark@5BBdRmBSkddy2G.org|2452284| +8642|AAAAAAAACMBCAAAA|1821497|2074|11898|2451124|2451094|Mrs.|Heather|Lynch|Y|15|11|1976|ICELAND||Heather.Lynch@816IdhvYfz8HO.org|2452476| +8643|AAAAAAAADMBCAAAA|1210990|6160|18130|2451823|2451793|Dr.|Andre|Pearson|Y|29|6|1990|PUERTO RICO||Andre.Pearson@RVvZNtcSYT.org|2452334| +8644|AAAAAAAAEMBCAAAA|897569|2533|35544|2450791|2450761|Sir|Daniel|Gleason|Y|12|11|1930|JAPAN||Daniel.Gleason@xvuGk9xBnVPrVhvT.edu|2452397| +8645|AAAAAAAAFMBCAAAA|60238|7080|19922|2449369|2449339|Mrs.|Shelly|Caro|N|23|4|1981|MONTENEGRO||Shelly.Caro@4q2KenOrOhy.edu|2452422| +8646|AAAAAAAAGMBCAAAA|998105|2534|163|2451810|2451780|Ms.|Margaret|Riddle|N|1|6|1933|CAMBODIA||Margaret.Riddle@i5.edu|2452445| +8647|AAAAAAAAHMBCAAAA|1380534|3667|18047|2451795|2451765|Sir|Bruce|Wells|Y|18|4|1971|PAPUA NEW GUINEA||Bruce.Wells@FgprKO.com|2452495| +8648|AAAAAAAAIMBCAAAA|1666782|6553|25124|2449373|2449343|Dr.|Matthew|Cole|N|23|8|1983|PARAGUAY||Matthew.Cole@iOu4YGCA.edu|2452629| +8649|AAAAAAAAJMBCAAAA|78968|536|2457|2452580|2452550|Dr.|Leonard|Wilson|Y|6|2|1968|ICELAND||Leonard.Wilson@YY.edu|2452477| +8650|AAAAAAAAKMBCAAAA|270520|3825|19930|2451801|2451771|Dr.|Betty|Thomas|N|24|7|1977|LIBERIA||Betty.Thomas@H0p7hXhIyYxH.org|2452398| +8651|AAAAAAAALMBCAAAA|1696973|5077|38781|2451900|2451870|Mr.|Issac|Asbury|N|12|3|1946|SIERRA LEONE||Issac.Asbury@6SHbZ.org|2452578| +8652|AAAAAAAAMMBCAAAA|738776|1908|39393|2449950|2449920|Sir|Bradford|Fisher|N|22|5|1931|KUWAIT||Bradford.Fisher@Pl.edu|2452492| +8653|AAAAAAAANMBCAAAA|975009|2422|49035|2451797|2451767|Dr.||Rivers|Y||2|1945||||2452429| +8654|AAAAAAAAOMBCAAAA|535808|5005|29849|2449703|2449673|Dr.|Richard|Alvarado|Y|5|5|1963|MALAYSIA||Richard.Alvarado@NJj1AlVb3zidNP.com|2452586| +8655|AAAAAAAAPMBCAAAA|1905859|6478|7850|2450293|2450263|Ms.|Terri|Medina|Y|31|5|1944|SWITZERLAND||Terri.Medina@V7qCxAenl.com|2452381| +8656|AAAAAAAAANBCAAAA|1826414|4787|10360|2452580|2452550|Sir|David|Mcdowell|N|11|11|1924|DOMINICA||David.Mcdowell@SX3D2IKxmvU4V.edu|2452456| +8657|AAAAAAAABNBCAAAA|840482|5743|7004|2451380|2451350|Mr.|Malcolm|Boggs|Y|29|1|1990|DOMINICA||Malcolm.Boggs@nXurIzS7B9M0.edu|2452614| +8658|AAAAAAAACNBCAAAA|1561260|6171|9482|2452055|2452025|Dr.|Loyd|Snow|Y|8|9|1985|EL SALVADOR||Loyd.Snow@iTJHbHkFbeVX.org|2452499| +8659|AAAAAAAADNBCAAAA|460462|7125|4588|2451907|2451877|Mrs.|Kristen|Spicer|Y|16|2|1970|MALI||Kristen.Spicer@eKcSF.edu|2452555| +8660|AAAAAAAAENBCAAAA|389771|1588|1405|2452097|2452067|Sir|Chris|Kramer|Y|7|12|1958|MALTA||Chris.Kramer@NLfcSgsMUJJZ7H620.com|2452569| +8661|AAAAAAAAFNBCAAAA|1230257|3159|26061|2451000|2450970|Dr.|Jack|Lopez|N|5|4|1928|SWAZILAND||Jack.Lopez@vnm3BjvuBR.org|2452290| +8662|AAAAAAAAGNBCAAAA|1628476|24|45572|2449506|||||N|21||1983|R�UNION||Irene.Benitez@TI7.org|| +8663|AAAAAAAAHNBCAAAA|1734494|3425|36756|2450393|2450363|Mr.|Paul|Darnell|Y|2|6|1962|EGYPT||Paul.Darnell@gcJIT.org|2452399| +8664|AAAAAAAAINBCAAAA|1604024|2983|18742|2452076|2452046|Ms.|Janet|Walsh|Y|7|3|1934|SRI LANKA||Janet.Walsh@n2nH9b6T6oK.edu|2452582| +8665|AAAAAAAAJNBCAAAA|1619906|7087|23468|2450149|2450119|Dr.|Sylvia|Timmons|Y|25|9|1927|LITHUANIA||Sylvia.Timmons@qTSu0YsEXsOM2.edu|2452519| +8666|AAAAAAAAKNBCAAAA|880570|6859|35948|2451422|2451392|Sir|Samuel|Perez|N|25|2|1933|QATAR||Samuel.Perez@rRM5r.com|2452316| +8667|AAAAAAAALNBCAAAA|1550988|4276|25164|2450027|2449997|Miss|Linda|Gentry|N|4|5|1966|SEYCHELLES||Linda.Gentry@cyA01t3z8xIjHqihY.edu|2452515| +8668|AAAAAAAAMNBCAAAA|838892|23|5201|2450752|2450722|Dr.|Suzette|Miller|Y|21|12|1970|MOLDOVA, REPUBLIC OF||Suzette.Miller@DurQMhU.org|2452523| +8669|AAAAAAAANNBCAAAA|322379|709|21486|2449428|2449398|Mr.|Willard|Coleman|N|29|1|1939|UNITED STATES||Willard.Coleman@VZ3NpdHLe2Rk7LoY.com|2452463| +8670|AAAAAAAAONBCAAAA|454743|5643|22896|2451498|2451468|Mr.|Fred|Williams|N|30|3|1943|NAMIBIA||Fred.Williams@x0JDGbnVCJLh.org|2452530| +8671|AAAAAAAAPNBCAAAA|723222|6446|27317|2452271|2452241|Dr.|Ruth|Murray|Y|12|2|1932|KUWAIT||Ruth.Murray@be2VVjddko.com|2452643| +8672|AAAAAAAAAOBCAAAA|850933|5174|49768|2449755|2449725|Miss|Rose|Mueller|N|13|4|1925|SEYCHELLES||Rose.Mueller@z2.org|2452608| +8673|AAAAAAAABOBCAAAA||6040|21013|2451236||||Milam|N|14||1947|||Leah.Milam@CKY7d3lH6g.org|| +8674|AAAAAAAACOBCAAAA|250205|4039|39873|2452222|2452192|Mrs.|Tammy|Howe|N|6|2|1935|BURUNDI||Tammy.Howe@5dcGt06ZpBFMJ0kc.edu|2452319| +8675|AAAAAAAADOBCAAAA|191802|2450|25945|2451263|2451233|Mr.|Clint|Spencer|N|5|9|1982|KUWAIT||Clint.Spencer@rAUK.edu|2452541| +8676|AAAAAAAAEOBCAAAA|1503176|68|21007|2450713|2450683|Dr.|Brian|Lea|Y|13|11|1980|CROATIA||Brian.Lea@H.com|2452485| +8677|AAAAAAAAFOBCAAAA|302023|2913|34603|2449960|2449930|Sir|Richard|Haas|Y|2|3|1944|TONGA||Richard.Haas@fZPDxtnuHC.edu|2452563| +8678|AAAAAAAAGOBCAAAA|1469766|3724|6082|2449727|2449697|Mrs.|Linda|Haas|N|27|5|1948|GHANA||Linda.Haas@z3bzxLEfatSTmI3.org|2452314| +8679|AAAAAAAAHOBCAAAA|719721|1687|33653|2450483|2450453|Dr.|Edna|Brand|Y|21|1|1949|BANGLADESH||Edna.Brand@7KpeXZgUhEd.com|2452453| +8680|AAAAAAAAIOBCAAAA|508682|6609|49783|2449825|2449795|Mr.|Robert|Silva|N|6|12|1980|SAN MARINO||Robert.Silva@OoOGQTY7.org|2452462| +8681|AAAAAAAAJOBCAAAA|293745|2748|7304|2450367|2450337|Mr.|Russell|Hernandez|Y|21|6|1945|BOTSWANA||Russell.Hernandez@qE151KR9LkpEt.org|2452449| +8682|AAAAAAAAKOBCAAAA|1014523|6988|6653|2449067|2449037|Dr.|Kelvin|Buckner|Y|5|1|1989|MOZAMBIQUE||Kelvin.Buckner@JevyKkB4eR5.com|2452545| +8683|AAAAAAAALOBCAAAA|1321765|6209|11453|2450148|2450118|Miss|Amanda|Perdue|Y|11|8|1943|ALBANIA||Amanda.Perdue@2Z1uEqUG2RqMDf.org|2452322| +8684|AAAAAAAAMOBCAAAA|230778|6397|49616|2451152|2451122|Dr.|Earl|Arroyo|N|15|11|1967|DOMINICA||Earl.Arroyo@Mf1xi3UA.org|2452572| +8685|AAAAAAAANOBCAAAA|597683|4514|24945|2449884|2449854|Dr.|James|Hartman|N|6|12|1933|SLOVAKIA||James.Hartman@DOhmyaNAT6pBVv.edu|2452325| +8686|AAAAAAAAOOBCAAAA|1887361|5464|6720|2452146|2452116|Ms.|Michelle|Nunes|N|27|10|1963|LATVIA||Michelle.Nunes@Sm3qbdTPEkZA.edu|2452342| +8687|AAAAAAAAPOBCAAAA|278663|640|46264|2450803|2450773|Sir|Martin|Gibson|N|28|12|1962|EL SALVADOR||Martin.Gibson@ts9TtLyOa.com|2452470| +8688|AAAAAAAAAPBCAAAA|926661|6957|37300|2449213|2449183|Mrs.|Kathleen|Hoffman|Y|7|2|1976|RWANDA||Kathleen.Hoffman@2FTQU5t9t0m.org|2452538| +8689|AAAAAAAABPBCAAAA|1725911|2389|15620|2450757|2450727|Mrs.|Lois|Peebles|Y|27|8|1924|TIMOR-LESTE||Lois.Peebles@tb.org|2452414| +8690|AAAAAAAACPBCAAAA|1477801|3366|42531|2452430|2452400|Sir|Gerald|Carey|Y|8|8|1972|WESTERN SAHARA||Gerald.Carey@LeLzjceHJ.com|2452514| +8691|AAAAAAAADPBCAAAA|1472857|1474|26044|2451262|2451232|Mr.|Harold|Holman|N|24|10|1925|KIRIBATI||Harold.Holman@IqUGMVE9ydq.org|2452478| +8692|AAAAAAAAEPBCAAAA|551742|1294|3199|2450154|2450124|Dr.|William|Herman|Y|16|10|1984|MOROCCO||William.Herman@CuBU1qvOUXbxC7ye.com|2452389| +8693|AAAAAAAAFPBCAAAA|357368|5202|41836|2449699|2449669|Ms.|Christina|Kelly|Y|23|7|1966|NIUE||Christina.Kelly@sxBA6OZ1h.edu|2452323| +8694|AAAAAAAAGPBCAAAA|1550719|3799|45279|2450795|2450765|Dr.|Judy|Herrin|N|1|9|1957|GUATEMALA||Judy.Herrin@0D4Kcnj.com|2452484| +8695|AAAAAAAAHPBCAAAA||2532|48854||2451346|Mr.|Edward|||||1954||||2452617| +8696|AAAAAAAAIPBCAAAA|937446|3865|23484|2452267|2452237|Sir|Preston|Spooner|N|6|6|1940|BOTSWANA||Preston.Spooner@mNfgn.org|2452534| +8697|AAAAAAAAJPBCAAAA|848285|5861|284|2450215|2450185|Mr.|Robert|Berry|Y|13|1|1962|PORTUGAL||Robert.Berry@MYiy75ntGCSr7X.org|2452452| +8698|AAAAAAAAKPBCAAAA|564106|4280|34854|2449530|2449500|Dr.|William|Brown|N|19|12|1978|SAINT LUCIA||William.Brown@UlSkTqgea9RSsST0C.edu|2452517| +8699|AAAAAAAALPBCAAAA|582304|5594|30131|||Mr.|Vince||N|7||||||| +8700|AAAAAAAAMPBCAAAA|1503728|5359|543|2451016|2450986|Mr.|Harry|Tyler|Y|30|5|1942|ANTIGUA AND BARBUDA||Harry.Tyler@p1GSS1VGnF1xDK.edu|2452363| +8701|AAAAAAAANPBCAAAA|1380825||11987|2450488||Dr.||Sullivan|N|9|12||||Timothy.Sullivan@8.org|2452503| +8702|AAAAAAAAOPBCAAAA|947120|6437|20507|2452212|2452182|Dr.|Annie|Patten|N|22|5|1968|NORFOLK ISLAND||Annie.Patten@rr9jl.com|2452628| +8703|AAAAAAAAPPBCAAAA|510573|5378|13736|2452299|2452269|Mrs.|Misty|Duckworth|Y|20|8|1926|ANGOLA||Misty.Duckworth@sbHUejHVXTy0hV6LFv.com|2452313| +8704|AAAAAAAAAACCAAAA|1018997|696|47642|2451767|2451737|Ms.|Jennifer|Gale|N|21|12|1935|TAJIKISTAN||Jennifer.Gale@b2tcmJlX5ubIVtKS.org|2452516| +8705|AAAAAAAABACCAAAA|||10233|2450819|2450789||George|Johnson|Y||10|1953|POLAND||George.Johnson@aDjt5zxNHvcqlpAt6.com|| +8706|AAAAAAAACACCAAAA|856678|1525|35572|2450467|2450437|Dr.|Edward|Moore|N|17|9|1986|YEMEN||Edward.Moore@YdEah11.org|2452321| +8707|AAAAAAAADACCAAAA|1138099|296|30540|2452559|2452529|Mr.|Robert|Bills|Y|12|1|1963|EL SALVADOR||Robert.Bills@CS5AFXZ0Z4.edu|2452646| +8708|AAAAAAAAEACCAAAA|1105430|913|38396|2451851|2451821|Mrs.|Rosa|Goff|N|2|2|1948|FRANCE||Rosa.Goff@EUU.com|2452600| +8709|AAAAAAAAFACCAAAA|1530838|5915|19839|2449390|2449360|Dr.|Bernice|Burns|Y|29|11|1979|JAMAICA||Bernice.Burns@QdzP.edu|2452469| +8710|AAAAAAAAGACCAAAA|911585|4874|7410|2449998|2449968|Sir|Donald|Cook|Y|14|9|1936|PAPUA NEW GUINEA||Donald.Cook@6zB7RYKOdLsVnM.edu|2452404| +8711|AAAAAAAAHACCAAAA|999303|4987|15024|2452126|2452096|Dr.|Iris|Wilson|N|18|9|1980|ISRAEL||Iris.Wilson@UpJ6vfxXu0L.org|2452463| +8712|AAAAAAAAIACCAAAA|959604|5679|38154|2452598|2452568|Miss|Katie|Zamora|Y|7|5|1968|LESOTHO||Katie.Zamora@x4f2CQFHUJeHxcEr2.org|2452511| +8713|AAAAAAAAJACCAAAA|1787329|6581|22233|2449559|2449529|Sir|Robert|Fernandez|N|11|3|1957|AUSTRIA||Robert.Fernandez@3vf4k1ft.org|2452395| +8714|AAAAAAAAKACCAAAA|875065|2370|18875|2451878|2451848|Dr.|Anthony|Young|N|22|5|1989|HAITI||Anthony.Young@SmOKB.edu|2452348| +8715|AAAAAAAALACCAAAA|461768|4358|948|2449677|2449647|Sir|Greg|Young|Y|12|7|1989|SURINAME||Greg.Young@T89.com|2452388| +8716|AAAAAAAAMACCAAAA|185045|5509|48726|2450878|2450848|Dr.|Beverly|Johnson|Y|6|11|1975|VIRGIN ISLANDS, U.S.||Beverly.Johnson@ekxufyJm.org|2452537| +8717|AAAAAAAANACCAAAA|384100|2336|13419|2451602|2451572|Mr.|Brian|Wall|N|29|10|1969|GUAM||Brian.Wall@CSCK.edu|2452633| +8718|AAAAAAAAOACCAAAA|400521|4925|20129|2451099|2451069|Mr.|Eugene|Drake|N|14|8|1960|PITCAIRN||Eugene.Drake@35Drl86ES6.edu|2452416| +8719|AAAAAAAAPACCAAAA|1601543|5201|6012|2449064|2449034|Mr.|Palmer|Reynolds|N|29|3|1983|SWITZERLAND||Palmer.Reynolds@qpARn6UQdMBba.edu|2452366| +8720|AAAAAAAAABCCAAAA|1413730|2881|40785|2450186|2450156|Sir|Ronnie|Morales|N|13|9|1983|UZBEKISTAN||Ronnie.Morales@COBfFIHSdzir8v.edu|2452532| +8721|AAAAAAAABBCCAAAA|346770|1272|38927|2449687|2449657|Ms.|Tamiko|Miller|Y|25|4|1978|KYRGYZSTAN||Tamiko.Miller@mHXCE8392kc2Ig.org|2452613| +8722|AAAAAAAACBCCAAAA|957941|198|48855|2449096|2449066|Miss|Suzette|Durant|N|14|2|1964|SIERRA LEONE||Suzette.Durant@T7LbmjVb2.edu|2452564| +8723|AAAAAAAADBCCAAAA|||31797|2451743|||Opal|Uribe||||1971|CANADA||Opal.Uribe@S8823R1Q.edu|2452578| +8724|AAAAAAAAEBCCAAAA|1328069|2893|26164|2449349|2449319|Sir|Raleigh|Guerrero|N|26|10|1924|URUGUAY||Raleigh.Guerrero@USDTfd2.edu|2452428| +8725|AAAAAAAAFBCCAAAA||4520|35497|2449479||Dr.||Winters|N|27|9|1980||||| +8726|AAAAAAAAGBCCAAAA|454769|5441|30066|2449342|2449312|Mr.|Ray|Hardaway|N|5|10|1956|BURKINA FASO||Ray.Hardaway@zy1.org|2452362| +8727|AAAAAAAAHBCCAAAA|160357|5837|25717|2450223|2450193|Mrs.|Tamara|Mcdonald|N|7|1|1954|TIMOR-LESTE||Tamara.Mcdonald@yezqYKn8h.com|2452523| +8728|AAAAAAAAIBCCAAAA|1168915|5304|28493|2449720|2449690|Sir|Herbert|Randall|N|17|9|1982|ITALY||Herbert.Randall@tmFoT63NjLkqhS.org|2452532| +8729|AAAAAAAAJBCCAAAA|1747579|1010|28488|2450278|2450248|Miss|Virginia|Putman|Y|29|11|1989|SWAZILAND||Virginia.Putman@SdXXGvssFp5xuaPJ.com|2452536| +8730|AAAAAAAAKBCCAAAA|1553096|2284|19425|2452095|2452065|Dr.|Marcus|Pate|Y|10|12|1967|SAINT LUCIA||Marcus.Pate@oC4a.com|2452588| +8731|AAAAAAAALBCCAAAA|1586903|1805|5907|2450931|2450901|Ms.|Hana|Webster|Y|25|10|1938|GREECE||Hana.Webster@QGH849tock2V3ql.com|2452451| +8732|AAAAAAAAMBCCAAAA|566012|475|28538|2451956|2451926|Sir|Larry|Coleman|N|23|11|1977|CAMEROON||Larry.Coleman@r2.org|2452345| +8733|AAAAAAAANBCCAAAA|1630056|803|19997|2450972|2450942|Ms.|Phyllis|Hart|Y|24|5|1930|TURKMENISTAN||Phyllis.Hart@eUa4eQbl2S.edu|2452570| +8734|AAAAAAAAOBCCAAAA||3597|13276||2451414||Michael|Bryson|||10|1953|||Michael.Bryson@J3NbMnChtnEeE.org|2452401| +8735|AAAAAAAAPBCCAAAA|1643701|1667|27233|2450476|2450446|Dr.|Richard|Smith|N|27|2|1952|AZERBAIJAN||Richard.Smith@FdGCenhFEC1N5dct7n.com|2452396| +8736|AAAAAAAAACCCAAAA|1404752|7013|11721|2450403|2450373|Dr.|Albert|Felix|Y|22|5|1940|ARGENTINA||Albert.Felix@72Nj.com|2452448| +8737|AAAAAAAABCCCAAAA|38614|1735|30856|2450079|2450049|Miss|Magdalene|Barber|Y|7|2|1984|TOKELAU||Magdalene.Barber@txyNBVjMkbDu.edu|2452318| +8738|AAAAAAAACCCCAAAA|1206832|493|3307|2449268|2449238|Miss|Janet|Holcombe|Y|5|2|1978|UNITED ARAB EMIRATES||Janet.Holcombe@u8jufXdmiY.com|2452310| +8739|AAAAAAAADCCCAAAA|||11922|||Ms.|Lena|Dunn|||||NORFOLK ISLAND|||| +8740|AAAAAAAAECCCAAAA|679477|89|43501|2449590|2449560|Mr.|Fredrick|Earls|Y|17|4|1972|URUGUAY||Fredrick.Earls@0inzmjMDvh.com|2452615| +8741|AAAAAAAAFCCCAAAA|789244|4519|38194|2452165|2452135|Sir|Paul|Clark|Y|8|2|1992|CUBA||Paul.Clark@5.com|2452351| +8742|AAAAAAAAGCCCAAAA|867891|3494|22250|2451319|2451289|Sir|Ryan|Bragg|Y|4|7|1965|VIRGIN ISLANDS, U.S.||Ryan.Bragg@TsicxN1.org|2452301| +8743|AAAAAAAAHCCCAAAA|1275866|3670|4321|2450617|2450587|Miss|Amy|Rowe|Y|21|7|1947|KYRGYZSTAN||Amy.Rowe@aegFnPlOjU5GG1.com|2452630| +8744|AAAAAAAAICCCAAAA|1133865|1005|40898|2450534|2450504|Mr.|Terry|Harmon|Y|18|4|1931|RUSSIAN FEDERATION||Terry.Harmon@27928.org|2452349| +8745|AAAAAAAAJCCCAAAA|1111914|5461|22608|2451908|2451878|Sir|Boyd|Griffin|N|20|9|1974|ECUADOR||Boyd.Griffin@U2xIQsP.edu|2452601| +8746|AAAAAAAAKCCCAAAA|||27568||2451839|Ms.||Crayton||27|||||Katie.Crayton@5dsVBGM5axmd.org|2452413| +8747|AAAAAAAALCCCAAAA|||43763||2450943|Dr.||Davenport|||9|1932|SOMALIA|||| +8748|AAAAAAAAMCCCAAAA|1210903|197|27938|2449343|2449313|Dr.|Julie|Keys|N|3|9|1960|MONTSERRAT||Julie.Keys@dCef7JY.edu|2452526| +8749|AAAAAAAANCCCAAAA|1351104|5911|36365|2449333|2449303|Mr.|James|Keller|Y|8|5|1960|GUAM||James.Keller@iJkng5JdIvHT96Cd.org|2452513| +8750|AAAAAAAAOCCCAAAA|1485727|2240|42400|2452010|2451980|Miss|Florence|Hernandez|N|14|9|1969|PANAMA||Florence.Hernandez@4m7vpglHQUHv.edu|2452539| +8751|AAAAAAAAPCCCAAAA|251717|4273|4429|2450531|2450501|Ms.|Amanda|Buck|N|6|1|1977|NICARAGUA||Amanda.Buck@sDAGIJ9oxzf2h.com|2452488| +8752|AAAAAAAAADCCAAAA|1907988|6510|49032|2449043|2449013|Sir|Elmer|Southerland|Y|14|3|1967|NICARAGUA||Elmer.Southerland@dG9HZHGjU8.edu|2452623| +8753|AAAAAAAABDCCAAAA|1408620|2923|44026|2450046|2450016|Sir|Bruce|Packer|Y|23|9|1961|PITCAIRN||Bruce.Packer@DajUbB.com|2452396| +8754|AAAAAAAACDCCAAAA|210672|7064|7903|2450959|2450929|Mrs.|Debbie|Holcomb|Y|25|2|1956|VIRGIN ISLANDS, U.S.||Debbie.Holcomb@J.com|2452493| +8755|AAAAAAAADDCCAAAA|848742|513|3484|2452423|2452393|Miss|Gladys|Bryant|Y|16|10|1973|FIJI||Gladys.Bryant@L2LYnp8X.com|2452543| +8756|AAAAAAAAEDCCAAAA|1241534|4905|41876|2450153|2450123|Dr.|Rosemarie|Martin|Y|8|10|1974|BULGARIA||Rosemarie.Martin@iDUkA.edu|2452596| +8757|AAAAAAAAFDCCAAAA|465388|236|23084|2451553|2451523|Dr.|Ruth|Davis|N|9|2|1935|NEW CALEDONIA||Ruth.Davis@fvtJ7dv6i8.com|2452595| +8758|AAAAAAAAGDCCAAAA|1405352|4413|17474|2450697|2450667|Mrs.|Darleen|Benefield|N|13|12|1992|GUYANA||Darleen.Benefield@rVxIU0o8d.com|2452428| +8759|AAAAAAAAHDCCAAAA|1685947|2329|17911|2452568|2452538|Ms.|Inez|Hardy|Y|31|3|1978|FRANCE||Inez.Hardy@cNueOfQdfBGzo.edu|2452319| +8760|AAAAAAAAIDCCAAAA|1303165|2541|4717|2450461|2450431|Mr.|William|Nestor|Y|8|12|1965|TIMOR-LESTE||William.Nestor@lMytChz.org|2452402| +8761|AAAAAAAAJDCCAAAA|138983|1766|5749|2451141|2451111|Dr.|Joshua|Dutton|N|30|8|1982|C�TE D'IVOIRE||Joshua.Dutton@RnY0z.org|2452379| +8762|AAAAAAAAKDCCAAAA|1758540|340|32540|2449635|2449605|Miss|Juanita|Hutto|Y|14|8|1941|UNITED KINGDOM||Juanita.Hutto@ruSAV.org|2452405| +8763|AAAAAAAALDCCAAAA|1212367|384|39201|2452345|2452315|Sir|Scott|Butler|Y|1|7|1963|DENMARK||Scott.Butler@bYmKr7lq02L.com|2452605| +8764|AAAAAAAAMDCCAAAA|1284898|6148|28951|2451999|2451969|Mrs.|Bertha|James|Y|7|11|1977|CYPRUS||Bertha.James@bHJzLUsiikyM1J2Qpe3.org|2452325| +8765|AAAAAAAANDCCAAAA|734738|2748|1911|2450031|2450001|Miss|Dorothy|Koontz|N|11|12|1963|SINGAPORE||Dorothy.Koontz@G11neqo3mYb.org|2452423| +8766|AAAAAAAAODCCAAAA|1000263|5119|33975|2449371|2449341|Miss|Louise|Leyva|Y|25|4|1954|SAINT HELENA||Louise.Leyva@rlTyNFAQlm.edu|2452557| +8767|AAAAAAAAPDCCAAAA|1265028|3563|8851|2452363|2452333|Mrs.|Shawna|Little|N|14|2|1989|CAMBODIA||Shawna.Little@4.com|2452384| +8768|AAAAAAAAAECCAAAA|1603533|2321|5956|2451191|2451161|Ms.|Sandra|Daniel|Y|15|9|1947|SENEGAL||Sandra.Daniel@gPeySOUl8qf5p27SUv.com|2452487| +8769|AAAAAAAABECCAAAA|35113|2760|28760|2450574|2450544|Ms.|Addie|Moses|Y|8|5|1942|CAMEROON||Addie.Moses@UgDFl8.edu|2452299| +8770|AAAAAAAACECCAAAA|1476676|6156|24436|2451736|2451706|Dr.|Silvia|Delarosa|N|20|10|1962|CHRISTMAS ISLAND||Silvia.Delarosa@f2cKXd2.edu|2452418| +8771|AAAAAAAADECCAAAA|1298709|5120|18331|2450121|2450091|Ms.|Marie|Hill|Y|3|12|1954|HAITI||Marie.Hill@Rh8Zb0piATO6Cg.edu|2452412| +8772|AAAAAAAAEECCAAAA|859509|6327|43592|2450657|2450627|Mr.|Bob|Williams|N|5|5|1989|PANAMA||Bob.Williams@nmYxcJiOHD6cPG2.edu|2452507| +8773|AAAAAAAAFECCAAAA|361176|6373|36770|2451308|2451278|Sir|Matthew|Cahill|Y|2|1|1991|LESOTHO||Matthew.Cahill@o78X9y1.org|2452387| +8774|AAAAAAAAGECCAAAA|1502544|5361|38178|2449622|2449592|Sir|Keith|Flores|N|10|5|1927|BARBADOS||Keith.Flores@1TSjnC9lqUQ9EK6.edu|2452636| +8775|AAAAAAAAHECCAAAA|590002|1797|30839|2450039|2450009|Miss|Beatrice|Rountree|Y|6|3|1964|ARUBA||Beatrice.Rountree@nJ8Govnu14a8B.edu|2452361| +8776|AAAAAAAAIECCAAAA|666606|4094|22735|2451879|2451849|Miss|Arletta|Leblanc|Y|9|7|1947|SEYCHELLES||Arletta.Leblanc@O0TfMgQUC1Qb.com|2452295| +8777|AAAAAAAAJECCAAAA|9720|5865|857|2450325|2450295|Ms.|Nellie|Brown|N|6|4|1970|BULGARIA||Nellie.Brown@2rNF1U6q9Y44DMocd.org|2452326| +8778|AAAAAAAAKECCAAAA|371618|6968|18671|2449318|2449288|Sir|Michael|Milton|N|9|2|1955|LIECHTENSTEIN||Michael.Milton@r2L.edu|2452293| +8779|AAAAAAAALECCAAAA|254948|2201|28701|2452333|2452303|Dr.|James|Johnson|Y|26|6|1991|KUWAIT||James.Johnson@i6.org|2452560| +8780|AAAAAAAAMECCAAAA|1210612|2561|13710|2449083|2449053|Dr.|Richard|Joseph|Y|29|2|1940|NIGERIA||Richard.Joseph@3okpCuqz8u1LkAY.org|2452558| +8781|AAAAAAAANECCAAAA|243721|4592|33510|2451318|2451288|Dr.|Michael|Doyle|Y|23|12|1959|ZAMBIA||Michael.Doyle@vykYQv9iXG2SCH.com|2452470| +8782|AAAAAAAAOECCAAAA|596423|6419|28815|2450139|2450109|Mrs.|Marilyn|Boone|Y|15|6|1939|SPAIN||Marilyn.Boone@ig.edu|2452503| +8783|AAAAAAAAPECCAAAA|||12372|2452544||||Turner|||12|1958|DOMINICA|||| +8784|AAAAAAAAAFCCAAAA|814919|4787|21855|2452341|2452311|Mr.|Christopher|Goodman|N|21|4|1932|LATVIA||Christopher.Goodman@mo5U4X3KlGAnXlS.org|2452426| +8785|AAAAAAAABFCCAAAA|529464|3577|7243|2451621|2451591|Ms.|Josephine|Romero|Y|12|9|1947|GUINEA||Josephine.Romero@Ltb6V8.org|2452460| +8786|AAAAAAAACFCCAAAA|1763037|2322|32007|2449778|2449748|Dr.|Clarence|Mulligan|Y|19|3|1956|LATVIA||Clarence.Mulligan@z4.com|2452316| +8787|AAAAAAAADFCCAAAA|91858|4915|1572|2452617|2452587|Sir|Brett|Vargas|N|27|3|1970|TUNISIA||Brett.Vargas@Jr60CaVUbGFVd.org|2452455| +8788|AAAAAAAAEFCCAAAA|1262632|5199|29813|2452601|2452571|Sir|Enrique|Richardson|Y|27|4|1945|MOROCCO||Enrique.Richardson@m2qyVs.org|2452330| +8789|AAAAAAAAFFCCAAAA|940196|1799|39002|2451544|2451514|Ms.|Grace|Shell|Y|4|8|1987|FRANCE||Grace.Shell@zfkdYKOPu.com|2452412| +8790|AAAAAAAAGFCCAAAA|1497876|5610|11297|2451366|2451336|Dr.|Marcus|Silverman|Y|11|11|1955|ISRAEL||Marcus.Silverman@OyjnB9z4zQR3bt.com|2452417| +8791|AAAAAAAAHFCCAAAA|1640869|1424|9932|2451086|2451056|Sir|Nathan|Smith|Y|16|2|1992|HONDURAS||Nathan.Smith@e.edu|2452522| +8792|AAAAAAAAIFCCAAAA|376310|5735|27560|2450056|2450026|Mrs.|Luz|Schuster|Y|10|11|1991|SAUDI ARABIA||Luz.Schuster@LFfOPHp7oEmiJ.com|2452609| +8793|AAAAAAAAJFCCAAAA|1677231|964|33450|2450049|2450019|Mr.|Steve|Lucas|N|15|5|1973|PERU||Steve.Lucas@ksrlu1PkKGnmcNmd.com|2452518| +8794|AAAAAAAAKFCCAAAA|1891722|4976|37077|2450060|2450030|Mr.|Donald|Crawford|Y|9|8|1951|VIET NAM||Donald.Crawford@pVYTYs.com|2452324| +8795|AAAAAAAALFCCAAAA|510800|476|4152|2451789|2451759|Miss|Lori|Kirby|N|2|8|1967|MONTENEGRO||Lori.Kirby@oeCneAYUMDPGu.org|2452515| +8796|AAAAAAAAMFCCAAAA|793175|1465|27402|2452394|2452364|Dr.|Michael|Brown|N|13|5|1992|CHILE||Michael.Brown@fay1dgRjlcob0.org|2452398| +8797|AAAAAAAANFCCAAAA|327496|5412|34175|2452596|2452566|Mrs.|Nina|Collier|N|26|1|1972|FRANCE||Nina.Collier@np9u.edu|2452588| +8798|AAAAAAAAOFCCAAAA|146042|4732|13069|2452613|2452583|Sir|George|Hampton|Y|5|11|1962|ALBANIA||George.Hampton@aV6n7s.com|2452298| +8799|AAAAAAAAPFCCAAAA|147693|2107|39721|2450186|2450156|Mr.|Jose|Inman|N|4|4|1929|GUINEA||Jose.Inman@oKmu2qtPJukyt3QSg.org|2452466| +8800|AAAAAAAAAGCCAAAA|509088|1536|47231|2450658|2450628|Miss|Grace|Saunders|N|2|8|1973|VIRGIN ISLANDS, U.S.||Grace.Saunders@h.edu|2452572| +8801|AAAAAAAABGCCAAAA|862103|5339|18128|2450233|2450203|Sir|Dylan|Mohr|N|5|6|1933|AZERBAIJAN||Dylan.Mohr@VKk.org|2452337| +8802|AAAAAAAACGCCAAAA|1388109|267|18778|2450123|2450093|Mrs.|Sheila|Rawlings|N|12|10|1989|ARGENTINA||Sheila.Rawlings@OngsFA6dBTqyc.com|2452450| +8803|AAAAAAAADGCCAAAA|1786913|4708|24150|2451729|2451699|Miss|Loretta|Moss|N|23|12|1960|GREENLAND||Loretta.Moss@F5npJo9.org|2452492| +8804|AAAAAAAAEGCCAAAA|804911|1934|41710|2452138|2452108|Mr.|Danny|Black|N|19|7|1974|UNITED KINGDOM||Danny.Black@KXZ6.com|2452382| +8805|AAAAAAAAFGCCAAAA|1427109|5079|32202|2451176|2451146|Miss|Sandra|Wood|N|15|2|1988|FRANCE||Sandra.Wood@LZabl0P.com|2452465| +8806|AAAAAAAAGGCCAAAA|1442269|1701|33180|2452026|2451996|Mrs.|Yvette|Gil|Y|29|9|1925|FRENCH GUIANA||Yvette.Gil@Ey7SL7D7gPPtUV3unG.org|2452338| +8807|AAAAAAAAHGCCAAAA|1705545|5869|42240|2449767|2449737|Ms.|Roxanne|Burns|N|28|1|1964|PAPUA NEW GUINEA||Roxanne.Burns@21gZHI53XULq4.org|2452497| +8808|AAAAAAAAIGCCAAAA|877703|464|26778|2450897|2450867|Sir|Isiah|Vanmeter|Y|22|2|1984|LATVIA||Isiah.Vanmeter@P3i.edu|2452534| +8809|AAAAAAAAJGCCAAAA|494354|6995|15989|2451289|2451259|Miss|Cheryl|Hinton|N|22|6|1959|TUVALU||Cheryl.Hinton@zAgaJsKQsdfX.org|2452591| +8810|AAAAAAAAKGCCAAAA|1620182|4424|17866|2451675|2451645|Dr.|Debbie|Mcclain|N|5|2|1955|LIBERIA||Debbie.Mcclain@yFFl2Dnz1dt.org|2452352| +8811|AAAAAAAALGCCAAAA|1031087|4714|2341|2449902|2449872|Dr.|Joe|Wilson|N|4|7|1976|PARAGUAY||Joe.Wilson@2hPXPEfIAry9.edu|2452416| +8812|AAAAAAAAMGCCAAAA|1428449|5656|2816|2449458|2449428|Sir|Jay|Shepard|Y|17|5|1926|VIET NAM||Jay.Shepard@pffSgt3LSXV.edu|2452528| +8813|AAAAAAAANGCCAAAA|255185|3560|18335|2451737|2451707|Ms.|Christine|Davis|N|1|11|1942|ALGERIA||Christine.Davis@FvTfT3uAMYm6yupnj.edu|2452368| +8814|AAAAAAAAOGCCAAAA|1455573|6611|28002|2451646|2451616|Miss|Cecile|Kent|N|21|7|1983|JERSEY||Cecile.Kent@s6dHYGHX.edu|2452332| +8815|AAAAAAAAPGCCAAAA|1148800|1545|43043|2451305|2451275|Dr.|Rhonda|Bradford|N|27|4|1965|ANTIGUA AND BARBUDA||Rhonda.Bradford@BcN6GiVUtE.edu|2452529| +8816|AAAAAAAAAHCCAAAA|191890|6639|42141|2451639|2451609|Mr.|Frank|Livingston|N|2|10|1924|VIRGIN ISLANDS, U.S.||Frank.Livingston@FzYQXsL2.org|2452419| +8817|AAAAAAAABHCCAAAA|1146648|252|20656|2449033|2449003|Dr.|Suzanne|Campbell|N|3|4|1945|HONG KONG||Suzanne.Campbell@b6p0.com|2452643| +8818|AAAAAAAACHCCAAAA|1347237|4001|22765|2449407|2449377|Ms.|Susan|Reece|N|5|7|1992|MOZAMBIQUE||Susan.Reece@V7X3elZNF.com|2452458| +8819|AAAAAAAADHCCAAAA|1356497|6395|44693|2449203|2449173|Dr.|Lisa|Gabriel|N|16|10|1987|CZECH REPUBLIC||Lisa.Gabriel@d8lSJOZITac.edu|2452615| +8820|AAAAAAAAEHCCAAAA|1588019|4218|47264|2449245|2449215|Sir|Leonard|Brown|Y|11|3|1970|MONTENEGRO||Leonard.Brown@30gAASte69o6IzgER8.org|2452351| +8821|AAAAAAAAFHCCAAAA|207290|1054|45618|2452412|2452382|Dr.|Jose|Crisp|N|16|11|1930|JAPAN||Jose.Crisp@ExYaEPO.edu|2452294| +8822|AAAAAAAAGHCCAAAA|136907|1749|45438|2450115|2450085|Mr.|Robert|Mclaughlin|N|8|11|1954|FINLAND||Robert.Mclaughlin@1d3xyl7XyHbzCIVVk1.com|2452416| +8823|AAAAAAAAHHCCAAAA|897988|1011|10387|2452220|2452190|Mrs.|Dawn|Crabtree|N|10|6|1927|CANADA||Dawn.Crabtree@VFC2lBzCgVkdB.com|2452315| +8824|AAAAAAAAIHCCAAAA|640483|2913|42543|2450965|2450935|Dr.|Brandon|Crump|Y|2|10|1934|ANDORRA||Brandon.Crump@6i.edu|2452577| +8825|AAAAAAAAJHCCAAAA|1173141|902|41110|2449231|2449201|Dr.|Nancy|Evans|Y|20|2|1956|JAMAICA||Nancy.Evans@KoXTc1.com|2452630| +8826|AAAAAAAAKHCCAAAA|1515803|4756|26909|2451917|2451887|Sir|Erasmo|Rivera|Y|9|3|1926|BELGIUM||Erasmo.Rivera@Bh5igFy0E1I.edu|2452529| +8827|AAAAAAAALHCCAAAA|809249|6563|11877|2451368|2451338|Ms.|Beverly|Porter|N|13|10|1966|MALI||Beverly.Porter@5EuR94QIEp1Us22A.edu|2452581| +8828|AAAAAAAAMHCCAAAA|1099295|72|35296|2451728|2451698|Miss|Patsy|Fernandez|Y|8|3|1937|SLOVENIA||Patsy.Fernandez@4Iz4t1Zmggol.com|2452467| +8829|AAAAAAAANHCCAAAA|1709444|1098|20752|2452012|2451982|Mr.|Barry|Dunaway|Y|5|6|1973|URUGUAY||Barry.Dunaway@QX.edu|2452628| +8830|AAAAAAAAOHCCAAAA|31585|512|30639|2452208|2452178|Dr.|Marcus|Price|Y|29|5|1936|MAURITANIA||Marcus.Price@zjYFkQ99ha.com|2452518| +8831|AAAAAAAAPHCCAAAA|1901383|2673|27563|2450619|2450589|Miss|Ruth|Simon|N|26|8|1942|GUATEMALA||Ruth.Simon@LCEkTZA7c1GouS6c.edu|2452613| +8832|AAAAAAAAAICCAAAA||6616|10892||2452382|Dr.|||Y|16||1948|||Evelyn.Stewart@A.edu|| +8833|AAAAAAAABICCAAAA|1733103|4503|27503|2450025|2449995|Sir|Michael|Edwards|N|21|6|1988|EGYPT||Michael.Edwards@Uqd93MQ0GO.edu|2452388| +8834|AAAAAAAACICCAAAA|889472|4|20936|2452618|2452588|Dr.|Calvin|Trevino|N|11|11|1952|SINGAPORE||Calvin.Trevino@beRkeCUL2t4.org|2452488| +8835|AAAAAAAADICCAAAA|860641|6255|2269|2452590|2452560|Dr.|James|Mcneal|N|5|5|1952|PAKISTAN||James.Mcneal@BZriTQL5yVBpiEOfF.com|2452422| +8836|AAAAAAAAEICCAAAA|1038588|5819|14603|2450341|2450311|Dr.|Steven|Steele|N|24|2|1925|NEW CALEDONIA||Steven.Steele@Et8DMsf1L67jA.edu|2452439| +8837|AAAAAAAAFICCAAAA|35580|3022|19217|2449416|2449386|Mr.|Jose|Stephenson|Y|5|1|1935|ESTONIA||Jose.Stephenson@yXGt.edu|2452324| +8838|AAAAAAAAGICCAAAA|85106|1925|47591|2449480|2449450|Ms.|Rosalina|Jones|N|6|1|1964|TOKELAU||Rosalina.Jones@kcpsg.com|2452629| +8839|AAAAAAAAHICCAAAA|598562|708|24007|2449111|2449081|Miss|Michelle|Meyer|Y|17|9|1962|PORTUGAL||Michelle.Meyer@uOd62.edu|2452367| +8840|AAAAAAAAIICCAAAA|1136833|4282|27731|2451659|2451629|Mr.|Thomas|Williams|N|21|1|1951|MOLDOVA, REPUBLIC OF||Thomas.Williams@xNbeq.com|2452601| +8841|AAAAAAAAJICCAAAA|26068|2715|19809|2451192|2451162|Sir|John|Larson|Y|22|1|1958|URUGUAY||John.Larson@2HmU6aXNQ0.org|2452604| +8842|AAAAAAAAKICCAAAA|985982|273|37411|2451743|2451713|Dr.|Stanley|Harris|N|1|11|1973|AZERBAIJAN||Stanley.Harris@qHx9ylRvx0.com|2452382| +8843|AAAAAAAALICCAAAA|1080035|3106|27377|2450900|2450870|Mr.|George|Downs|Y|18|7|1924|NIGERIA||George.Downs@us6dybkLEL8l.edu|2452473| +8844|AAAAAAAAMICCAAAA|1429976|1114|34652|2451574|2451544|Ms.|Patti|Evans|N|5|1|1939|CAYMAN ISLANDS||Patti.Evans@zU7GatFYMxsU.edu|2452502| +8845|AAAAAAAANICCAAAA|809639|3179|38319|2452301|2452271|Ms.|Margarette|Thomas|N|16|8|1989|GEORGIA||Margarette.Thomas@deLpgVXtlggknZQ.com|2452484| +8846|AAAAAAAAOICCAAAA|244639|531|3670|2451000|2450970|Dr.|Grace|Murray|Y|31|3|1936|MARSHALL ISLANDS||Grace.Murray@ajMKi4FYTHO.com|2452352| +8847|AAAAAAAAPICCAAAA|165965|2106|10600|2449597|2449567|Mrs.|Marissa|Sheldon|Y|20|5|1990|MADAGASCAR||Marissa.Sheldon@XeM0tFTahhK.org|2452366| +8848|AAAAAAAAAJCCAAAA|90034|886|40538|2450064|2450034|Mr.|Ray|Harris|N|5|8|1991|MYANMAR||Ray.Harris@P9oNe471p4S.org|2452469| +8849|AAAAAAAABJCCAAAA|462054|1321|21788|2449695|2449665|Mr.|James|Johnson|Y|4|6|1942|THAILAND||James.Johnson@IUPmjO0pOIfY.edu|2452382| +8850|AAAAAAAACJCCAAAA|501495|6116|9537|2449626|2449596|Dr.|Alicia|Flores|Y|22|12|1954|ETHIOPIA||Alicia.Flores@lz8jKufVhJtck1.edu|2452302| +8851|AAAAAAAADJCCAAAA|1637439|6341|48043|2449984|2449954|Sir|David|Beavers|Y|11|10|1988|CAMEROON||David.Beavers@AaxEYiycupoZIfzbv.edu|2452394| +8852|AAAAAAAAEJCCAAAA|1045119|79|5086|2450151|2450121|Mr.|Raymond|Whitlock|Y|21|3|1953|NETHERLANDS ANTILLES||Raymond.Whitlock@8YsIkYXq4M.com|2452505| +8853|AAAAAAAAFJCCAAAA|975361|3770|8979|2452272|2452242|Ms.|Ethel|Rivera|N|20|8|1985|PUERTO RICO||Ethel.Rivera@CN4RnQEQvBOn.org|2452500| +8854|AAAAAAAAGJCCAAAA|957665|4448|26784|2452018|2451988|Sir|Darrell|Howell|N|28|4|1942|SOMALIA||Darrell.Howell@kAkNVzaSS.com|2452381| +8855|AAAAAAAAHJCCAAAA|512939|3651|38316|2450249|2450219|Dr.|Mohammad|Schmidt|Y|1|5|1946|SLOVENIA||Mohammad.Schmidt@UJjOhMUbxK5ZnrupCZ.com|2452573| +8856|AAAAAAAAIJCCAAAA|202992|1665|24338|2450808|2450778|Miss|Shari|Griffin|N|28|7|1938|BULGARIA||Shari.Griffin@ftSfTHypvB3oD.edu|2452421| +8857|AAAAAAAAJJCCAAAA|1308561|5208|20702|2449323|2449293|Mr.|Chad|Mccormick|Y|9|11|1958|LITHUANIA||Chad.Mccormick@gkel2UZ0vberM1.edu|2452528| +8858|AAAAAAAAKJCCAAAA|1096316|3920|25077|2449260|2449230|Mr.|Lupe|Jerome|N|5|5|1927|VIRGIN ISLANDS, U.S.||Lupe.Jerome@Kn9huMg3aqYDRiO.edu|2452358| +8859|AAAAAAAALJCCAAAA|1533068|827|11833|2452631|2452601|Mrs.|Leah|Bess|Y|3|1|1967|CYPRUS||Leah.Bess@Hl9joPCFdbvF.org|2452611| +8860|AAAAAAAAMJCCAAAA|1644759|1092|9778|2451132|2451102|Mrs.|Jacqueline|Manns|N|5|5|1982|ERITREA||Jacqueline.Manns@Enqy.org|2452371| +8861|AAAAAAAANJCCAAAA|1248993|5053|33913|2452458|2452428|Mr.|Michael|Hardin|Y|4|2|1946|VIET NAM||Michael.Hardin@p9guEyT3bxqympt4.org|2452548| +8862|AAAAAAAAOJCCAAAA|1221892|5331|12770|2450023|2449993|Sir|Louis|Hartman|N|28|4|1977|NETHERLANDS||Louis.Hartman@Hq1SmRfj3hZ0s1pu.com|2452553| +8863|AAAAAAAAPJCCAAAA|1644814|4035|20809|2449105|2449075|Mr.|Chris|Adams|Y|10|12|1972|BURUNDI||Chris.Adams@j1XXZ6XvIu8i.com|2452607| +8864|AAAAAAAAAKCCAAAA|122731|851|971|2451977|2451947|Mrs.|Edith|Mueller|Y|25|9|1959|AMERICAN SAMOA||Edith.Mueller@NGZBqR.edu|2452542| +8865|AAAAAAAABKCCAAAA|1749302|785|37515|2452036|2452006|Mrs.|Carolyn|Johnson|N|20|7|1931|ESTONIA||Carolyn.Johnson@AKRH3rjafP06SKu.com|2452491| +8866|AAAAAAAACKCCAAAA|119740||10422|2450782|2450752||Janice|Marshall|Y|||||||2452328| +8867|AAAAAAAADKCCAAAA|1488099|6081|14510|2451165|2451135|Miss|Elizabeth|Stone|N|12|7|1942|GUINEA||Elizabeth.Stone@fMZGnuD5.edu|2452472| +8868|AAAAAAAAEKCCAAAA|368284|382|8527|2452175|2452145|Sir|Nathaniel|Sabo|N|22|4|1948|RUSSIAN FEDERATION||Nathaniel.Sabo@RnVQM.org|2452463| +8869|AAAAAAAAFKCCAAAA|514740|1430|2073|2452048|2452018|Mrs.|Diane|Taylor|Y|4|4|1973|MOLDOVA, REPUBLIC OF||Diane.Taylor@aXZ4YAQAPlg.edu|2452624| +8870|AAAAAAAAGKCCAAAA|553807|4692|19977|2450616|2450586|Miss|Veronica|Cook|Y|7|5|1978|VANUATU||Veronica.Cook@THYyppYQLFzu.org|2452302| +8871|AAAAAAAAHKCCAAAA|445880|2109|12738|2451518|2451488|Dr.|Linda|Diamond|Y|11|10|1978|MALDIVES||Linda.Diamond@UjIj9pJDOElE7dG7Z.com|2452403| +8872|AAAAAAAAIKCCAAAA|1356523|189|42655|2450716|2450686|Sir|Carl|Tillman|Y|11|7|1991|NORWAY||Carl.Tillman@guKhdVTT67p1YYY.com|2452610| +8873|AAAAAAAAJKCCAAAA|887332|3206|42633|2451764|2451734|Dr.|Ignacio|Benson|Y|5|7|1944|AMERICAN SAMOA||Ignacio.Benson@Oe.edu|2452520| +8874|AAAAAAAAKKCCAAAA|1811657|891|5772|2451555|2451525|Dr.|James|Faulkner|N|11|5|1987|AMERICAN SAMOA||James.Faulkner@gOJq.com|2452523| +8875|AAAAAAAALKCCAAAA|1198969|1249|17929|2450673|2450643|Mr.|Brandon|Moran|Y|2|12|1973|BURUNDI||Brandon.Moran@dPT.com|2452404| +8876|AAAAAAAAMKCCAAAA|1525334|4784|47481|2452127|2452097|Sir|Charles|Wilkinson|N|23|7|1972|HONG KONG||Charles.Wilkinson@ihEgT.edu|2452517| +8877|AAAAAAAANKCCAAAA|841938|329|32767|2449793|2449763|Sir|Giuseppe|Johnson|N|29|10|1961|TAJIKISTAN||Giuseppe.Johnson@xT5yJhP0Sz9.edu|2452621| +8878|AAAAAAAAOKCCAAAA|1638903|5361|40219|2449661|2449631|Dr.|Allene|Byrd|N|30|9|1929|ANTARCTICA||Allene.Byrd@MFzs3VB66P.com|2452457| +8879|AAAAAAAAPKCCAAAA|53993|5612|23631|2450623|2450593|Mr.|Henry|Davis|Y|26|10|1935|MALAWI||Henry.Davis@0XMOQzeOd5KJU8Q.com|2452383| +8880|AAAAAAAAALCCAAAA|109983|1390|44044|2452239|2452209|Sir|Johnathan|Lopez|Y|19|4|1965|CAMEROON||Johnathan.Lopez@s6NEHQ4iR87hucg.org|2452363| +8881|AAAAAAAABLCCAAAA|679364|4445|6326|2452582|2452552|Dr.|Thomas|Bishop|Y|13|2|1925|GUINEA||Thomas.Bishop@dHms1bC8h.com|2452450| +8882|AAAAAAAACLCCAAAA|1085191|6169|9947|2451085|2451055|Mr.|Frank|Flynn|Y|25|10|1988|EL SALVADOR||Frank.Flynn@Xgg.edu|2452403| +8883|AAAAAAAADLCCAAAA|1838337|2530|44276|2449658|2449628|Mr.|Roger|Morgan|N|24|9|1938|YEMEN||Roger.Morgan@Q.edu|2452408| +8884|AAAAAAAAELCCAAAA|1912734|3354|18774|2449369|2449339|Dr.|Andrew|Baker|N|12|10|1925|SAN MARINO||Andrew.Baker@fU023Dt.edu|2452560| +8885|AAAAAAAAFLCCAAAA||212|34767||2450669|Miss||Guzman|Y|16|5|1978|||Nell.Guzman@69t9e.com|2452383| +8886|AAAAAAAAGLCCAAAA|1277556|1343|25577|2451766|2451736|Miss|Jennifer|Coffey|Y|8|9|1966|NIUE||Jennifer.Coffey@P7lMZqKazpSG.org|2452444| +8887|AAAAAAAAHLCCAAAA|1742540|734|6175|2451404|2451374|Ms.|Juanita|Blevins|Y|4|1|1987|MEXICO||Juanita.Blevins@o.edu|2452488| +8888|AAAAAAAAILCCAAAA|1897252|1883|34467|2449253|2449223|Dr.|Luis|Williams|Y|5|2|1934|INDIA||Luis.Williams@5fnSPLfj5DdLb.edu|2452321| +8889|AAAAAAAAJLCCAAAA|1303709|6246|43536|2451584|2451554|Ms.|Sharon|Cohn|N|29|3|1952|AMERICAN SAMOA||Sharon.Cohn@08Q0qbmhEnkuC.org|2452622| +8890|AAAAAAAAKLCCAAAA|1168585|4064|47543|2451745|2451715|Miss|Ana|Bryant|Y|11|9|1947|PITCAIRN||Ana.Bryant@RYua6aR.org|2452470| +8891|AAAAAAAALLCCAAAA|971015|3062|39509|2450682|2450652|Dr.|Curtis|Martinez|N|31|7|1947|MONGOLIA||Curtis.Martinez@kzZT2qLZjcVRkK.com|2452309| +8892|AAAAAAAAMLCCAAAA|1656373|6360|18700|2450722|2450692|Mr.|Glen|Greenwood|Y|2|2|1926|BELIZE||Glen.Greenwood@jMNTAqOvKya.org|2452309| +8893|AAAAAAAANLCCAAAA|768458|5609|43560|2451420|2451390|Ms.|Katherine|Vogel|Y|24|8|1936|MACAO||Katherine.Vogel@e.edu|2452507| +8894|AAAAAAAAOLCCAAAA|1591017|2734|51|2451335|2451305|Mrs.|Debra|King|Y|25|7|1927|MONTENEGRO||Debra.King@toKC9N6Mgh9z.edu|2452308| +8895|AAAAAAAAPLCCAAAA|1709208|2842|43676|2451700|2451670|Ms.|Francine|Andrade|N|4|6|1961|NAMIBIA||Francine.Andrade@cbVc2NkI.org|2452498| +8896|AAAAAAAAAMCCAAAA|586038|2239|39993|2449848|2449818|Miss|Sandra|Reece|N|23|7|1946|LIBERIA||Sandra.Reece@oB06FhYr.edu|2452560| +8897|AAAAAAAABMCCAAAA|1451717|4665|19166|2451908|2451878|Dr.|John|Harris|Y|5|10|1991|C�TE D'IVOIRE||John.Harris@tno4vrPAlEPJU.com|2452495| +8898|AAAAAAAACMCCAAAA|896678|2769|8107|2451818|2451788|Mr.|Clarence|Walker|Y|23|6|1932|MACAO||Clarence.Walker@rn87Z.org|2452414| +8899|AAAAAAAADMCCAAAA|45586|1342|16698|2451621|2451591|Dr.|James|Metzler|N|17|8|1949|SPAIN||James.Metzler@RTrYVxaQG3ay.com|2452585| +8900|AAAAAAAAEMCCAAAA|1885936|5515|11141|2452489|2452459|Ms.|Lawanda|Wilson|N|15|9|1974|R�UNION||Lawanda.Wilson@LOOo6b4U.edu|2452334| +8901|AAAAAAAAFMCCAAAA|1899241|3474|33660|2451536|2451506|Dr.|Mathew|Harmon|Y|27|10|1945|NEW CALEDONIA||Mathew.Harmon@FJh3RcsUk.edu|2452547| +8902|AAAAAAAAGMCCAAAA|1502705|6804|2103|2449452|2449422|Dr.|Melissa|Williams|Y|15|8|1925|VIRGIN ISLANDS, U.S.||Melissa.Williams@RE8iiP38xk.edu|2452349| +8903|AAAAAAAAHMCCAAAA|49991|7135|2232|2450446|2450416|Mr.|Emilio|Moss|Y|5|8|1968|QATAR||Emilio.Moss@IYr15sDuFRd.edu|2452611| +8904|AAAAAAAAIMCCAAAA|161625|7061|48450|2451019|2450989|Mr.|William|Rendon|Y|20|11|1973|LITHUANIA||William.Rendon@r4epZ3F0k6kves8d.edu|2452577| +8905|AAAAAAAAJMCCAAAA|1108654|1929|48004|2449669|2449639|Miss|Donna|Walton|N|23|12|1991|BULGARIA||Donna.Walton@04atvlIabltOTe.com|2452334| +8906|AAAAAAAAKMCCAAAA|943382|787|17392|2451867|2451837|Ms.|Zaida|Barlow|Y|6|8|1952|PUERTO RICO||Zaida.Barlow@y.edu|2452619| +8907|AAAAAAAALMCCAAAA|1238387|3463|34892|2452215|2452185|Dr.|Tiffanie|Stevenson|N|9|7|1938|BRUNEI DARUSSALAM||Tiffanie.Stevenson@BJTyvUQ3Qrdf2Um.org|2452498| +8908|AAAAAAAAMMCCAAAA|1399702|5367|42232|2449783|2449753|Miss|Trisha|Crawley|Y|13|12|1948|FRENCH GUIANA||Trisha.Crawley@mq.com|2452568| +8909|AAAAAAAANMCCAAAA|1387604|322|45840|2449519|2449489|Sir|Lonnie|Reese|Y|19|8|1969|KUWAIT||Lonnie.Reese@mA8qlEP38AIZvr.org|2452609| +8910|AAAAAAAAOMCCAAAA|884332|4637|35742|2450598|2450568|Ms.|Tina|Humes|Y|2|3|1981|LEBANON||Tina.Humes@2V4Pv49iljj.com|2452617| +8911|AAAAAAAAPMCCAAAA|341157|5189|6021|2451697|2451667|Dr.|Steve|Guzman|Y|25|8|1951|HUNGARY||Steve.Guzman@ZUnuVI8GtnIn.org|2452638| +8912|AAAAAAAAANCCAAAA|273111|1203|22495|2449921|2449891|Ms.|Clara|Hall|N|11|11|1924|FRENCH POLYNESIA||Clara.Hall@OoONg94lftlrL.com|2452385| +8913|AAAAAAAABNCCAAAA|1428547|756|21765|2449684|2449654|Dr.|Lora|Hylton|N|26|12|1965|GEORGIA||Lora.Hylton@rZfkG1UTc8.com|2452525| +8914|AAAAAAAACNCCAAAA|1478169|2275|10781|2451637|2451607|Dr.|Richard|Crump|N|22|5|1965|RUSSIAN FEDERATION||Richard.Crump@hTB.org|2452486| +8915|AAAAAAAADNCCAAAA|481072|3177|19729|2451687|2451657|Mr.|Glenn|Heredia|N|3|5|1947|SUDAN||Glenn.Heredia@4.com|2452358| +8916|AAAAAAAAENCCAAAA|987829|4173|30164|2451680|2451650|Dr.|Teresa|Dawkins|Y|23|2|1989|MONGOLIA||Teresa.Dawkins@nmBm7OBfPX.com|2452371| +8917|AAAAAAAAFNCCAAAA|744979|3896|29613|2450970|2450940|Dr.|Jamie|Coward|N|22|4|1973|MALDIVES||Jamie.Coward@DB.edu|2452327| +8918|AAAAAAAAGNCCAAAA|629309|1474|20086|2450464|2450434|Sir|Michael|Steele|Y|10|10|1976|PERU||Michael.Steele@QdOlAQzTjSD5.edu|2452593| +8919|AAAAAAAAHNCCAAAA|539623|1553|48236|2450286|2450256|Ms.|Sarah|Nelson|N|23|11|1989|ITALY||Sarah.Nelson@2i6X86eBejOk.com|2452586| +8920|AAAAAAAAINCCAAAA||2054|46724|||||||21||1937||||| +8921|AAAAAAAAJNCCAAAA|308115|5778|41441|2449039|2449009|Miss|Dolores|Rosen|Y|13|10|1940|AMERICAN SAMOA||Dolores.Rosen@Y271eJL5CpRx.org|2452548| +8922|AAAAAAAAKNCCAAAA|90743|6818|16940|2449599|2449569|Ms.|Kristan|Adams|N|4|5|1934|GRENADA||Kristan.Adams@smXty6.org|2452341| +8923|AAAAAAAALNCCAAAA|1459715|4584|15053|2451388|2451358|Sir|Terence|Cook|Y|13|8|1926|COMOROS||Terence.Cook@dfju8Eylebkler.edu|2452516| +8924|AAAAAAAAMNCCAAAA|1513429|2818|20207|2449337|2449307|Sir|Henry|Castillo|Y|1|3|1964|WESTERN SAHARA||Henry.Castillo@8qtTMI3E3Ab.org|2452361| +8925|AAAAAAAANNCCAAAA|1801228|6144|23060|2452107|2452077|Dr.|Michael|Richards|Y|30|10|1933|RUSSIAN FEDERATION||Michael.Richards@sLUQ9Y4.edu|2452388| +8926|AAAAAAAAONCCAAAA|845421|2807|28640|2449049|2449019|Ms.|Jessica|Scholl|N|12|5|1976|ALGERIA||Jessica.Scholl@E5JhLkNM.edu|2452476| +8927|AAAAAAAAPNCCAAAA|1532872||41208|2449610|2449580|Mr.||||29||1992||||2452361| +8928|AAAAAAAAAOCCAAAA|1288232|1236|18640|2449948|2449918|Mrs.|Marvel|Greer|Y|19|3|1959|ECUADOR||Marvel.Greer@bCFJ3oAvGvqQCR.com|2452541| +8929|AAAAAAAABOCCAAAA|610373|3505|7037|2452032|2452002|Dr.|Ann|Davis|N|19|2|1927|FRANCE||Ann.Davis@2doNSYb7ge.com|2452395| +8930|AAAAAAAACOCCAAAA|114259|4876|693|2451107|2451077|Miss|Krystal|Elliot|N|28|12|1974|SOLOMON ISLANDS||Krystal.Elliot@XAqgQQ2Ue3L5vNv.org|2452394| +8931|AAAAAAAADOCCAAAA|||9308|2452387|2452357|||Henry|Y||1|1970|||Eugene.Henry@0p7qFLu7.org|| +8932|AAAAAAAAEOCCAAAA|1217651|701|44104|2451674|2451644|Sir|Heath|Fine|Y|21|2|1943|PANAMA||Heath.Fine@5lS.org|2452327| +8933|AAAAAAAAFOCCAAAA|222590|6577|3364|2450449|2450419|Mr.|Byron|Larson|Y|6|7|1990|JERSEY||Byron.Larson@MUTgk1Mhq7sqPmBYvg.edu|2452620| +8934|AAAAAAAAGOCCAAAA|1655070|1937|5604|2450798|2450768|Mr.|Vincent|Reese|Y|11|11|1956|TUNISIA||Vincent.Reese@uz0Qocs.org|2452350| +8935|AAAAAAAAHOCCAAAA|1202051|2302|13706|2449959|2449929|Sir|Michael|Winston|Y|28|2|1949|GIBRALTAR||Michael.Winston@IPsqLQKSnZMnQV.org|2452305| +8936|AAAAAAAAIOCCAAAA|15333|234|33982|2450666|2450636|Dr.|James|Johnson|Y|25|11|1930|COSTA RICA||James.Johnson@RUsXaM.com|2452513| +8937|AAAAAAAAJOCCAAAA|11445|4615|27785|2452337|2452307|Dr.|Nicholas|Ellison|N|14|2|1924|ITALY||Nicholas.Ellison@VrriD2lITip.edu|2452450| +8938|AAAAAAAAKOCCAAAA|752193|1995|19397|2451828|2451798|Dr.|Angela|Beard|N|27|5|1967|FINLAND||Angela.Beard@M2enKls.edu|2452608| +8939|AAAAAAAALOCCAAAA|1719439|4475|29911|2449957|2449927|Miss|Ruth|Simpson|N|15|4|1990|JAPAN||Ruth.Simpson@PXXqMHJnHEtmJiFi.org|2452487| +8940|AAAAAAAAMOCCAAAA|1463512||47927|2449407||||Seward|N|30||1932|SINGAPORE|||2452520| +8941|AAAAAAAANOCCAAAA|468523|6899|47228|2449120|2449090|Mr.|Kerry|Hopper|N|19|6|1952|POLAND||Kerry.Hopper@lMhLc9.org|2452458| +8942|AAAAAAAAOOCCAAAA|1467564|753|25895|2451582|2451552|Ms.|Cheri|Howard|Y|16|11|1978|ICELAND||Cheri.Howard@9jXTGnmvBqQ.edu|2452494| +8943|AAAAAAAAPOCCAAAA|441975|4214|48500|2450845|2450815|Dr.|Mark|Flowers|N|28|3|1948|CUBA||Mark.Flowers@Hls1Jba6Ydhlk.org|2452413| +8944|AAAAAAAAAPCCAAAA|870940|1697|21254||2449218||Pearlie|Scott||||||||| +8945|AAAAAAAABPCCAAAA|858744|1097|38603|2452558|2452528|Mr.|Stephen|Schmidt|N|14|6|1968|SAINT HELENA||Stephen.Schmidt@gxrOa5UrmixPFxIBBpV.edu|2452499| +8946|AAAAAAAACPCCAAAA|431605||29108||||||Y|||1928|TONGA|||| +8947|AAAAAAAADPCCAAAA|1576698|1577|15303|2450921|2450891|Dr.|Robert|Long|Y|28|3|1963|KIRIBATI||Robert.Long@qNNUHLtSb.org|2452339| +8948|AAAAAAAAEPCCAAAA|1586650|2306|9162|2449099|2449069|Sir|Leland|Kirk|N|10|7|1964|JAPAN||Leland.Kirk@A1bCoNlIOsH.com|2452623| +8949|AAAAAAAAFPCCAAAA|896859|6364|15822|2449877|2449847|Dr.|Barbara|Ashe|Y|3|7|1924|JERSEY||Barbara.Ashe@b1Cnf7zlg.com|2452426| +8950|AAAAAAAAGPCCAAAA|474775|5206|16837|2450346|2450316|Ms.|Martha|Mason|N|14|1|1984|SAN MARINO||Martha.Mason@yH.org|2452479| +8951|AAAAAAAAHPCCAAAA|1414777|5566|45155|2449313|2449283|Dr.|Gary|Short|N|24|5|1988|BARBADOS||Gary.Short@nxM2J1omHatS.com|2452603| +8952|AAAAAAAAIPCCAAAA|1605610|6313|2969|2452514|2452484|Dr.|Brian|Wright|N|30|8|1937|SLOVENIA||Brian.Wright@MSNu4ae.com|2452510| +8953|AAAAAAAAJPCCAAAA|910335|2058|45781|2451295|2451265|Dr.|Rick|Creel|Y|4|10|1948|CZECH REPUBLIC||Rick.Creel@PIvPfQnHOetJle.com|2452346| +8954|AAAAAAAAKPCCAAAA|474044|2531|3418|2450634|2450604|Sir|Thomas|Harry|Y|9|7|1932|PUERTO RICO||Thomas.Harry@tyIl.com|2452489| +8955|AAAAAAAALPCCAAAA|914443|5560|1030|2452668|2452638|Mrs.|Floy|Green|N|12|7|1977|SINGAPORE||Floy.Green@5jHBI4c.org|2452480| +8956|AAAAAAAAMPCCAAAA|326687|3665|45843|2452400|2452370|Mrs.|Gina|Dollar|Y|20|5|1951|MOROCCO||Gina.Dollar@4UgcHJ0ZT8749kp.com|2452574| +8957|AAAAAAAANPCCAAAA|1868286|1841|12996|2450088|2450058|Miss|Tammy|Bolduc|N|22|9|1958|SYRIAN ARAB REPUBLIC||Tammy.Bolduc@ddr.com|2452309| +8958|AAAAAAAAOPCCAAAA|95177|6262|30947|2450093|2450063|Dr.|Jack|Williams|N|3|9|1986|SWAZILAND||Jack.Williams@h88paT.com|2452401| +8959|AAAAAAAAPPCCAAAA|703225|751|33477|2451591|2451561|Miss|Sigrid|Miller|N|18|9|1963|AUSTRALIA||Sigrid.Miller@GB.org|2452645| +8960|AAAAAAAAAADCAAAA|118223|3313|44403|2451787|2451757|Sir|Nathan|Smith|Y|20|12|1972|ARMENIA||Nathan.Smith@SJBBOQcZVp0R.com|2452366| +8961|AAAAAAAABADCAAAA|39190|2600|36810|2450196|2450166|Mr.|Blake|Burke|N|16|1|1951|HONG KONG||Blake.Burke@lh8H1dAIXH.org|2452606| +8962|AAAAAAAACADCAAAA|1547890|5707|33877|2451978|2451948|Mr.|Marvin|Allen|Y|26|12|1970|MONACO||Marvin.Allen@OHU8o2VjRUr8HJJeh9.org|2452412| +8963|AAAAAAAADADCAAAA|350658|5873|20688|2450077|2450047|Miss|Lori|Hampton|N|19|1|1954|SLOVAKIA||Lori.Hampton@9ZpH6Rb4pKoi.org|2452357| +8964|AAAAAAAAEADCAAAA|1239618|1334|28297|2451491|2451461|Ms.|Barbara|Babcock|Y|7|1|1977|GUINEA||Barbara.Babcock@pyoXqI18dd580Zs.edu|2452635| +8965|AAAAAAAAFADCAAAA|1134233|4967|43701|2449433|2449403|Ms.|Margery|Kline|Y|6|9|1931|CHRISTMAS ISLAND||Margery.Kline@VKCLmRdj.org|2452359| +8966|AAAAAAAAGADCAAAA|375332|4678|23977|2449722|2449692|Mrs.|Vera|Mcarthur|N|31|5|1931|C�TE D'IVOIRE||Vera.Mcarthur@EjzSoygbiJG.com|2452443| +8967|AAAAAAAAHADCAAAA|1560028|7043|48505|2452600|2452570|Sir|Steve|Jackson|Y|16|10|1992|MYANMAR||Steve.Jackson@2g9fTMjzP.org|2452612| +8968|AAAAAAAAIADCAAAA|1908382|6935|31535|2451040|2451010|Dr.|Robert|Porterfield|N|19|4|1958|TONGA||Robert.Porterfield@6Gk9MAa2nbz.org|2452477| +8969|AAAAAAAAJADCAAAA|1779261|309|42510|2451515|2451485|Dr.|Jimmy|Chavez|N|28|3|1992|MONTSERRAT||Jimmy.Chavez@SiNPJPZqFlb.edu|2452594| +8970|AAAAAAAAKADCAAAA|77704|3007|20757|2452187|2452157|Mr.|Joseph|Hill|Y|8|7|1939|NIGER||Joseph.Hill@KHiBKhZamyIyAimB53K.edu|2452535| +8971|AAAAAAAALADCAAAA|223530|315|28747|2450244|2450214|Sir|Tanner|Weber|N|26|2|1927|SIERRA LEONE||Tanner.Weber@9.edu|2452596| +8972|AAAAAAAAMADCAAAA|830894|813|14685|2449632|2449602|Sir|Dale|Grace|N|30|12|1954|CAYMAN ISLANDS||Dale.Grace@YNmxhiEaSL9smAzQz0.edu|2452584| +8973|AAAAAAAANADCAAAA|119925|4183|7288|2450254|2450224|Mr.|Joshua|Choi|N|10|7|1973|LEBANON||Joshua.Choi@ebVp6g6r0Te5.org|2452495| +8974|AAAAAAAAOADCAAAA|1376912|5340|3368|2452492|2452462|Mrs.|Jacqueline|Leeper|Y|2|7|1961|NAURU||Jacqueline.Leeper@00.edu|2452326| +8975|AAAAAAAAPADCAAAA|407805|1287|41169|2450965|2450935|Sir|Dwayne|Williams|N|3|5|1961|PANAMA||Dwayne.Williams@L3MU.org|2452350| +8976|AAAAAAAAABDCAAAA|218564|4540|19269|2451753|||Ida|Short||||1967|VENEZUELA||Ida.Short@05.org|2452426| +8977|AAAAAAAABBDCAAAA|882685|1553|34421|2449284|2449254|Dr.|June|Thompson|N|17|4|1969|R�UNION||June.Thompson@IpzRx8FaZtE8.com|2452536| +8978|AAAAAAAACBDCAAAA|876931|3687|23643|2450200|2450170|Mr.|Timothy|Davis|N|31|7|1972|AUSTRIA||Timothy.Davis@qGYDvYoGZg6m.com|2452421| +8979|AAAAAAAADBDCAAAA|1816339|6498|704|2450024|2449994|Miss|Miriam|Rice|Y|24|6|1989|VANUATU||Miriam.Rice@lN1uTBX8q8.org|2452630| +8980|AAAAAAAAEBDCAAAA|1587190|6545|44300|2449461|2449431|Mrs.|Lashonda|Hedrick|N|2|6|1956|MOLDOVA, REPUBLIC OF||Lashonda.Hedrick@569PuHpZPFAKQOgP.com|2452376| +8981|AAAAAAAAFBDCAAAA|114525|3042|35959|2450341|2450311|Dr.|John|Parra|Y|6|11|1980|EL SALVADOR||John.Parra@2OkJdRJ1xrCt5I3B.edu|2452407| +8982|AAAAAAAAGBDCAAAA|941868|1251|21881|2449377|2449347|Dr.|John|Napier|Y|8|2|1968|TOKELAU||John.Napier@ko6uUNhmhoc.com|2452425| +8983|AAAAAAAAHBDCAAAA|984761|6351|40788|2449948|2449918|Mr.|James|Brown|N|22|4|1949|HUNGARY||James.Brown@Ejgf34k3M8IIeYd.com|2452558| +8984|AAAAAAAAIBDCAAAA|443598|6707|30169|2449387|2449357|Mrs.|Judy|Cole|N|28|7|1990|BURUNDI||Judy.Cole@76E.edu|2452574| +8985|AAAAAAAAJBDCAAAA|48131|4516|4454|2451417|2451387|Dr.|Jamal|Merritt|N|31|12|1937|TAJIKISTAN||Jamal.Merritt@RDUUQmHEr9TAD.com|2452633| +8986|AAAAAAAAKBDCAAAA|252169|3947|22925|2450014|2449984|Miss|Angelina|Miller|Y|14|9|1937|BELARUS||Angelina.Miller@fjTlMNBRLYR4nM.com|2452412| +8987|AAAAAAAALBDCAAAA|1527862|5678|36694|2451001|2450971|Miss|Thelma|Roy|Y|12|5|1928|CHRISTMAS ISLAND||Thelma.Roy@5Rxf98sKYO02BeK.edu|2452582| +8988|AAAAAAAAMBDCAAAA|1448461|6788|28315|2451603|2451573|Mr.|Leo|Dunne|N|20|1|1925|RWANDA||Leo.Dunne@092bMmxixqYKo.com|2452348| +8989|AAAAAAAANBDCAAAA|950277|3190|30103|2451305|2451275|Dr.|Samuel|Anderson|N|23|6|1971|INDONESIA||Samuel.Anderson@cuqHl0UMh1fHzeQ86.org|2452346| +8990|AAAAAAAAOBDCAAAA|730097|3948|2152|2450364|2450334|Miss|Jennifer|Ellis|N|19|3|1934|MARTINIQUE||Jennifer.Ellis@34q7s.edu|2452618| +8991|AAAAAAAAPBDCAAAA|1639975|5987|25152|2451644|2451614|Dr.|Jeffrey|Broussard|N|11|3|1986|TUVALU||Jeffrey.Broussard@XasmQ2R0p3Du.org|2452481| +8992|AAAAAAAAACDCAAAA|1572084|2162|44240|2450554|2450524|Sir|Richard|Diaz|N|28|4|1971|CHINA||Richard.Diaz@TMYPBAkDv8jx.edu|2452502| +8993|AAAAAAAABCDCAAAA|1864330|5973|36416|2450948|2450918|Dr.|Rhonda|Bond|N|22|6|1967|GUAM||Rhonda.Bond@OrJZsGlLG7UDFJxs.org|2452382| +8994|AAAAAAAACCDCAAAA|1158146|675|9204|2450657|2450627|Dr.|Paula|Miller|N|30|11|1957|DOMINICA||Paula.Miller@c01.com|2452374| +8995|AAAAAAAADCDCAAAA|1617578|7011|11644|2450595|2450565|Ms.|Eva|Davis|Y|8|4|1924|GHANA||Eva.Davis@BCb7SHVzYYBqkyl7G.org|2452340| +8996|AAAAAAAAECDCAAAA|1886207|6515|27176|2450924|2450894|Sir|Calvin|Brady|Y|5|8|1957|PANAMA||Calvin.Brady@2bHANUUTKos.edu|2452554| +8997|AAAAAAAAFCDCAAAA||1869|41768|||||Gabbard|Y|2|2||||Tamara.Gabbard@0k.edu|2452414| +8998|AAAAAAAAGCDCAAAA|655359|5052|5145|2449111|2449081|Mr.|Christopher|Mccann|N|4|5|1956|SOMALIA||Christopher.Mccann@ht6aOQ9Xj.org|2452453| +8999|AAAAAAAAHCDCAAAA|943534|4855|9423|2450698|2450668|Dr.|Patrica|Mcgee|N|30|1|1976|FRENCH GUIANA||Patrica.Mcgee@OZibq0ne2.edu|2452594| +9000|AAAAAAAAICDCAAAA|1211557|1194|32478|2450955|2450925|Dr.|Albert|Kaplan|N|4|8|1990|SOUTH AFRICA||Albert.Kaplan@rGyuF8t7u.com|2452539| +9001|AAAAAAAAJCDCAAAA|1275922|3314|33906|2452225|2452195|Mr.|Benito|Augustine|Y|6|10|1970|LIBERIA||Benito.Augustine@UtUGoB8UhX1.com|2452589| +9002|AAAAAAAAKCDCAAAA|713786|1025|30217|2449667|2449637|Mr.|Ricardo|Stout|Y|3|3|1924|MALTA||Ricardo.Stout@vmeyeVOKGXFUsiaui5I.edu|2452637| +9003|AAAAAAAALCDCAAAA|818659|2336|8013|2451703|2451673|Dr.|Matt|Warden|N|9|10|1951|GEORGIA||Matt.Warden@FyoBXAaVqZR.edu|2452623| +9004|AAAAAAAAMCDCAAAA|1860979|6912|23245|2450591|2450561|Mr.|Kenneth|Ellis|N|17|2|1989|EGYPT||Kenneth.Ellis@dl4z1.edu|2452581| +9005|AAAAAAAANCDCAAAA|101295|6045|1148|2452121|2452091|Mr.|James|Smallwood|N|8|9|1941|THAILAND||James.Smallwood@Phcdq2py5rU.com|2452571| +9006|AAAAAAAAOCDCAAAA|1053108|6978|32048|2450245|2450215|Ms.|Rita|Rice|Y|22|12|1982|MADAGASCAR||Rita.Rice@i.edu|2452384| +9007|AAAAAAAAPCDCAAAA|686093|1732|27371|2449289|2449259|Mrs.|Rebecca|Cason|N|20|10|1983|ROMANIA||Rebecca.Cason@love.com|2452479| +9008|AAAAAAAAADDCAAAA|1390334|2053|30219|2449090|2449060|Dr.|Corey|Scott|Y|27|12|1979|SINGAPORE||Corey.Scott@X1Ng.edu|2452387| +9009|AAAAAAAABDDCAAAA|1389219|5890|29361|2452373|2452343|Dr.|Harold|Cooper|N|7|10|1977|PAKISTAN||Harold.Cooper@ZiyDMpC.com|2452584| +9010|AAAAAAAACDDCAAAA|||13764||||Steven||Y|30||||||| +9011|AAAAAAAADDDCAAAA|1525750|6501|46405|2450804|2450774|Mr.|John|Richardson|N|13|2|1948|GHANA||John.Richardson@VtQnL.com|2452626| +9012|AAAAAAAAEDDCAAAA|454252|4061|42670|2449947|2449917|Ms.|Alex|Sasser|Y|28|7|1932|UNITED ARAB EMIRATES||Alex.Sasser@FNhhIbHj.com|2452430| +9013|AAAAAAAAFDDCAAAA|932260|5966|42119|2452666|2452636|Mr.|Tom|Ogden|N|10|9|1959|NIGERIA||Tom.Ogden@IKr2Vv6Gll4u3J5oF.org|2452463| +9014|AAAAAAAAGDDCAAAA|1028751|6813|23322|2450226|2450196|Mrs.|Krista|Griffith|Y|23|1|1938|POLAND||Krista.Griffith@zGqymZ.org|2452476| +9015|AAAAAAAAHDDCAAAA|1012979||29692||2449931|Sir||Mendoza||15||1987|TUNISIA|||2452461| +9016|AAAAAAAAIDDCAAAA|506524|3929|22534|2450457|2450427|Dr.|George|Bush|Y|8|8|1960|UNITED ARAB EMIRATES||George.Bush@ApnZlahx.com|2452353| +9017|AAAAAAAAJDDCAAAA|992664|3308|28352|2451501|2451471|Mrs.|Amanda|Raymond|Y|19|9|1948|ALBANIA||Amanda.Raymond@VC3SC12aY.edu|2452568| +9018|AAAAAAAAKDDCAAAA|1252500|2090|11680|2449055|2449025|Dr.|Phillip|Irving|Y|23|1|1937|TUNISIA||Phillip.Irving@2VY5gZgyOlU.com|2452445| +9019|AAAAAAAALDDCAAAA|893055|7081|38307|2451869|2451839|Miss|Victoria|Davis|N|17|12|1949|BAHRAIN||Victoria.Davis@EPfUdcN.org|2452301| +9020|AAAAAAAAMDDCAAAA|1424045|7141|47437|2450631|2450601|Ms.|Tonya|Ellis|N|4|9|1974|BOTSWANA||Tonya.Ellis@cbHI2JFSEyno2Ma.org|2452409| +9021|AAAAAAAANDDCAAAA|82389|6491|37346|2451895|2451865|Mr.|Philip|Schneider|Y|27|6|1980|VANUATU||Philip.Schneider@xdvazk8r3Fk75EVp.com|2452519| +9022|AAAAAAAAODDCAAAA|1733196|1485|19774|2450933|2450903|Ms.|Tasha|Pace|Y|22|6|1970|KAZAKHSTAN||Tasha.Pace@x3FirkoP6.com|2452421| +9023|AAAAAAAAPDDCAAAA|1767638|2270|21302|2450582|2450552|Mr.|Darren|Sanders|Y|25|5|1970|KENYA||Darren.Sanders@UnRykI2Tr.org|2452568| +9024|AAAAAAAAAEDCAAAA|6758|3126|5894|2449825|2449795|Mrs.|Rebecca|Sheehan|N|29|10|1956|AMERICAN SAMOA||Rebecca.Sheehan@iy89COhZdc6k.com|2452545| +9025|AAAAAAAABEDCAAAA|1206470|1046|43975|2451130|2451100|Ms.|Elizabeth|Craven|Y|15|8|1978|SOLOMON ISLANDS||Elizabeth.Craven@8cKVbgYtaQJ.org|2452403| +9026|AAAAAAAACEDCAAAA|319934|2815|49343|2450103|2450073|Mr.|Israel|White|N|31|12|1974|KAZAKHSTAN||Israel.White@tkoZ.org|2452414| +9027|AAAAAAAADEDCAAAA|820929|6485|15241|2452326|2452296|Miss|Melinda|Belcher|N|21|4|1930|MONTSERRAT||Melinda.Belcher@yBXNP8Shy9.edu|2452305| +9028|AAAAAAAAEEDCAAAA|1698034|2603|1572|2449795|2449765|Mrs.|Bettye|Jose|N|23|8|1970|FRANCE||Bettye.Jose@EoLFliQq1BL47.edu|2452367| +9029|AAAAAAAAFEDCAAAA|218924|1341|30958|2451920|2451890|Dr.|Dwight|Cole|N|9|2|1927|SYRIAN ARAB REPUBLIC||Dwight.Cole@M3ss6jVBZ.edu|2452568| +9030|AAAAAAAAGEDCAAAA|204696|1456|19016|2452487|2452457|Dr.|Hilton|Ho|N|17|1|1939|SLOVENIA||Hilton.Ho@b0TLUC.com|2452330| +9031|AAAAAAAAHEDCAAAA|1820574|6183|22992|2452024|2451994|Sir|Jason|Grady|N|27|4|1990|PITCAIRN||Jason.Grady@2M9x4do.com|2452413| +9032|AAAAAAAAIEDCAAAA|966807|293|33426|2451515|2451485|Dr.|Jaime|Henson|Y|23|10|1941|FINLAND||Jaime.Henson@azZk.com|2452372| +9033|AAAAAAAAJEDCAAAA|1072141|2700|14506|2450559|2450529|Sir|Thomas|Fernandez|N|8|11|1952|VIRGIN ISLANDS, U.S.||Thomas.Fernandez@az3Ljza3Kzds6dKGu.com|2452540| +9034|AAAAAAAAKEDCAAAA|1775103|566|9582|2449091|2449061|Mr.|Danny|Sammons|N|16|3|1927|RWANDA||Danny.Sammons@LDK.com|2452527| +9035|AAAAAAAALEDCAAAA|278554|7186|41065|2452255|2452225|Ms.|Andrew|Hughes|Y|13|1|1950|TRINIDAD AND TOBAGO||Andrew.Hughes@4CEIIlF8.edu|2452384| +9036|AAAAAAAAMEDCAAAA|284933|6581|7782|2449814|2449784|Sir|Charles|Williams|Y|24|8|1958|BAHAMAS||Charles.Williams@SRNBY7gKVfi.edu|2452470| +9037|AAAAAAAANEDCAAAA|298990|6836|16235|2449988|2449958|Ms.|Debra|Shaw|Y|19|6|1972|CAYMAN ISLANDS||Debra.Shaw@DPGL5rq6lj5.com|2452320| +9038|AAAAAAAAOEDCAAAA|765282|4560|25838|2449335|2449305|Sir|Bryan|Mack|Y|24|10|1964|CHILE||Bryan.Mack@37.org|2452527| +9039|AAAAAAAAPEDCAAAA|410601|5908|37968|2449123|2449093|Dr.|William|Alvarez|N|5|6|1941|KAZAKHSTAN||William.Alvarez@evR1UQ6nq0ajamO9.com|2452628| +9040|AAAAAAAAAFDCAAAA|297528|6823|43718|2450678|2450648|Mrs.|Donna|Brown|N|5|9|1929|SEYCHELLES||Donna.Brown@HqDyEcZR2fVLQQ9k.edu|2452407| +9041|AAAAAAAABFDCAAAA|534231|3214|8197|2452565|2452535|Dr.|Crystal|Dunlap|N|13|9|1959|JORDAN||Crystal.Dunlap@hdHN9B4dyN3.org|2452348| +9042|AAAAAAAACFDCAAAA|153074|2858|31395|2452058|2452028|Sir|Jeffrey|Bailey|Y|23|1|1981|ANGUILLA||Jeffrey.Bailey@hFPBMC7.edu|2452438| +9043|AAAAAAAADFDCAAAA|1649578|3653|49281|2451202|2451172|Ms.|Majorie|Walker|N|8|6|1934|SAMOA||Majorie.Walker@k.org|2452503| +9044|AAAAAAAAEFDCAAAA|362399|7181|9916|2452677|2452647|Mr.|Jorge|Hinton|N|15|7|1964|SWEDEN||Jorge.Hinton@935s.edu|2452324| +9045|AAAAAAAAFFDCAAAA|1907804|1893|43582|2452128|2452098|Mr.|Lynn|Williams|Y|22|2|1965|COMOROS||Lynn.Williams@NIIRLPAaar.org|2452569| +9046|AAAAAAAAGFDCAAAA|1154442|5059|16029|2449791|2449761|Dr.|Cynthia|Smith|Y|4|9|1967|SWEDEN||Cynthia.Smith@JXiZXvrhG5fMZDs.org|2452410| +9047|AAAAAAAAHFDCAAAA|951963|4995|2427|2451557|2451527|Mrs.|Lisa|Mares|N|16|7|1970|MAURITANIA||Lisa.Mares@zq2mCGPycP.org|2452638| +9048|AAAAAAAAIFDCAAAA|1216276|5830|27337|2450912|2450882|Sir|Leonard|Gutierrez|N|23|5|1951|BHUTAN||Leonard.Gutierrez@Z.org|2452520| +9049|AAAAAAAAJFDCAAAA|1094154|5792|35783|2450381|2450351|Sir|Charles|Tucker|N|29|10|1936|GUADELOUPE||Charles.Tucker@As2i36d.edu|2452486| +9050|AAAAAAAAKFDCAAAA|679745|2279|8013|2451886|2451856|Sir|Ronnie|Funk|Y|28|10|1983|SWEDEN||Ronnie.Funk@Fh0EuYJ1o.com|2452413| +9051|AAAAAAAALFDCAAAA|1139619|4041|47663|2451697|2451667|Dr.|Robert|Garris|N|10|9|1981|MACAO||Robert.Garris@edTfJ2Iqxq5r5Vz.com|2452300| +9052|AAAAAAAAMFDCAAAA|1707691|2899|6907|2451047|2451017|Mr.|Bruce|Dow|N|4|11|1974|FIJI||Bruce.Dow@Ovv5c3PcchBu.edu|2452534| +9053|AAAAAAAANFDCAAAA|575681|539|15044|2451088|2451058|Mr.|John|Martin|N|23|2|1987|NIUE||John.Martin@0VDeRqzHR1.edu|2452457| +9054|AAAAAAAAOFDCAAAA|1643047|4930|33613|2452044|2452014|Dr.|Kenneth|Blodgett|Y|8|12|1928|TOKELAU||Kenneth.Blodgett@Uqe9jcY1carF.com|2452429| +9055|AAAAAAAAPFDCAAAA|1237197|29|42061|2451342|2451312|Mr.|Patrick|Thornton|N|6|5|1930|GUATEMALA||Patrick.Thornton@2UnFGlhv9eHD.edu|2452301| +9056|AAAAAAAAAGDCAAAA|1663228|1631|49234|2451501|2451471|Sir|Barney|Clark|N|15|11|1986|DOMINICA||Barney.Clark@NLUq2bmV9ar.edu|2452327| +9057|AAAAAAAABGDCAAAA|1812490|4705|16073|2450791|2450761|Dr.|Barbara|Lara|N|5|11|1931|MARTINIQUE||Barbara.Lara@KUBAErJP.com|2452387| +9058|AAAAAAAACGDCAAAA|1122370|369|21568|2451245|2451215|Ms.|Darlene|Fuller|Y|18|1|1932|BRUNEI DARUSSALAM||Darlene.Fuller@kksAz.com|2452626| +9059|AAAAAAAADGDCAAAA|1626526|4579|1032|2451628|2451598|Dr.|Travis|Murphy|N|8|10|1981|NAMIBIA||Travis.Murphy@x3VSVMqD2XEK01phlTV.edu|2452304| +9060|AAAAAAAAEGDCAAAA|1521254|6866|21725|2452421|2452391|Mrs.|Penny|Simms|N|29|11|1934|GUADELOUPE||Penny.Simms@pv.com|2452509| +9061|AAAAAAAAFGDCAAAA|177207|1350|23093|2449169|2449139|Mrs.|Marlene|Browning|Y|21|4|1942|ICELAND||Marlene.Browning@5ukiX6.com|2452415| +9062|AAAAAAAAGGDCAAAA|1874886|6483|31680|2450566|2450536|Mr.|Zachary|Hendrickson|N|8|6|1930|GUERNSEY||Zachary.Hendrickson@y6.edu|2452334| +9063|AAAAAAAAHGDCAAAA||626|29273||2449158|Dr.|Rickey||||1||||Rickey.Lombard@tVKBhQSfHZSD.org|| +9064|AAAAAAAAIGDCAAAA|1724613|52|7938|2450980|2450950|Miss|Susan|Webb|Y|17|10|1980|PAPUA NEW GUINEA||Susan.Webb@XB.org|2452424| +9065|AAAAAAAAJGDCAAAA|221998|5879|31713|2450069|2450039|Sir|Ralph|Layne|Y|22|5|1935|GUERNSEY||Ralph.Layne@rCfUftXZBqqbYA8.org|2452469| +9066|AAAAAAAAKGDCAAAA|418361|315|6269|2451403|2451373|Ms.|Deborah|Cox|Y|15|5|1964|MADAGASCAR||Deborah.Cox@0voPMuzsv.org|2452373| +9067|AAAAAAAALGDCAAAA|1581963|2421|45232|2451149|2451119|Sir|Thomas|Frazier|N|8|6|1962|MALDIVES||Thomas.Frazier@t.edu|2452357| +9068|AAAAAAAAMGDCAAAA|1409323|6252|26381|2450890|2450860|Mrs.|Juanita|Stewart|N|2|6|1991|SAINT LUCIA||Juanita.Stewart@dzs6MqAPX.edu|2452550| +9069|AAAAAAAANGDCAAAA|408527|1470|46013|2449220|2449190|Sir|Tomas|Lopez|Y|21|6|1945|BOUVET ISLAND||Tomas.Lopez@hgrk5P.org|2452567| +9070|AAAAAAAAOGDCAAAA|1908989|6095|33601|2452242|2452212|Dr.|Helen|Lawrence|N|12|10|1984|BERMUDA||Helen.Lawrence@bprb.edu|2452587| +9071|AAAAAAAAPGDCAAAA|1804026|1000|31927|2452068|2452038|Sir|George|Castaneda|N|6|12|1976|ERITREA||George.Castaneda@TV8hH1BbqQQD.org|2452586| +9072|AAAAAAAAAHDCAAAA|1748016|5154|7845|2449143|2449113|Mr.|Edward|Milam|N|17|10|1938|TUNISIA||Edward.Milam@iY.edu|2452494| +9073|AAAAAAAABHDCAAAA|1313890|2072|29454|2451286|2451256|Dr.|Alan|Moses|N|29|8|1961|NORWAY||Alan.Moses@qpvBnSpNB9kuR4oA.com|2452299| +9074|AAAAAAAACHDCAAAA|1365020|2363|11659|2452259|2452229|Mrs.|Susan|Gardner|Y|9|3|1934|UNITED ARAB EMIRATES||Susan.Gardner@BODMU0edmr.edu|2452524| +9075|AAAAAAAADHDCAAAA|1539689|5495|40148|2452260|2452230|Sir|Mike|Jones|N|14|10|1973|AMERICAN SAMOA||Mike.Jones@B20.org|2452291| +9076|AAAAAAAAEHDCAAAA|678583|420|8610|2450737|2450707|Ms.|Melanie|Mcintyre|Y|15|5|1969|BRAZIL||Melanie.Mcintyre@TumNe0f4bE7bm.edu|2452604| +9077|AAAAAAAAFHDCAAAA|173665|1187|12662|2449340|2449310|Dr.|Scotty|Jones|Y|24|6|1986|TIMOR-LESTE||Scotty.Jones@h3QHn6K9gaARth2Of9.edu|2452362| +9078|AAAAAAAAGHDCAAAA|1487244|5514|49784|2452219|2452189|Dr.|Marjorie|Cahill|Y|13|4|1970|BAHRAIN||Marjorie.Cahill@Nc5D.edu|2452310| +9079|AAAAAAAAHHDCAAAA|137807|2200|27440|2450789|2450759|Mr.|Salvatore|Johnson|N|24|6|1964|DJIBOUTI||Salvatore.Johnson@1X.edu|2452468| +9080|AAAAAAAAIHDCAAAA|250784|3562|39140|2449583|2449553|Dr.|Mickey|Saxon|Y|8|7|1978|COSTA RICA||Mickey.Saxon@4.org|2452283| +9081|AAAAAAAAJHDCAAAA|470956|4777|9751|2450791|2450761|Dr.|Landon|Clements|Y|25|6|1939|MEXICO||Landon.Clements@qoyzjjuQy2fNNGJTMfN.com|2452329| +9082|AAAAAAAAKHDCAAAA|161154|947|19140|2449699|2449669|Sir|Jonathan|Daley|Y|2|5|1951|MONTENEGRO||Jonathan.Daley@O.org|2452388| +9083|AAAAAAAALHDCAAAA|701278|2636|13547|2451369|2451339|Dr.|Amber|Bowman|N|8|7|1959|INDONESIA||Amber.Bowman@xg.com|2452463| +9084|AAAAAAAAMHDCAAAA|935592|431|823|2450801|2450771|Mr.|Richard|Langer|N|6|8|1965|SERBIA||Richard.Langer@4HqFU5sLNbN6QB6xvj.com|2452366| +9085|AAAAAAAANHDCAAAA|1831825|6741|14410|2451123|2451093|Dr.|Jaqueline|Harper|Y|29|1|1951|ALGERIA||Jaqueline.Harper@F.com|2452424| +9086|AAAAAAAAOHDCAAAA|1325500|3575|2403|2450432|2450402|Sir|Ronald|Brown|Y|25|9|1929|BELGIUM||Ronald.Brown@DsGA2af4J.com|2452335| +9087|AAAAAAAAPHDCAAAA|137428|2958|40888|2449238|2449208|Mr.|Kenneth|Moore|Y|8|6|1984|FAROE ISLANDS||Kenneth.Moore@8Of1ebQIt.org|2452467| +9088|AAAAAAAAAIDCAAAA|1733554|6935|5916|2451087|2451057|Dr.|Richard|Mcdowell|Y|5|6|1944|VIRGIN ISLANDS, U.S.||Richard.Mcdowell@vH3AO.com|2452610| +9089|AAAAAAAABIDCAAAA|1498748|1187|13273|2451193|2451163|Dr.|Gary|Diaz|N|16|6|1944|ANTIGUA AND BARBUDA||Gary.Diaz@7S.edu|2452531| +9090|AAAAAAAACIDCAAAA|697548|3580|22459|2449941|2449911|Dr.|Eric|Mcdaniel|N|12|4|1925|CANADA||Eric.Mcdaniel@HVg463T1td6.edu|2452486| +9091|AAAAAAAADIDCAAAA|1032777|3081|14232|2450829|2450799|Mr.|Steven|Taylor|N|18|2|1951|SOLOMON ISLANDS||Steven.Taylor@Oq7Q.com|2452410| +9092|AAAAAAAAEIDCAAAA|851945|5188|22878|2450569|2450539|Sir|Frank|Gomez|N|5|8|1943|FRANCE||Frank.Gomez@d6bFM.com|2452579| +9093|AAAAAAAAFIDCAAAA|1822020|6775|37930|2450265|2450235|Mrs.|Sharon|Castillo|Y|5|5|1971|NETHERLANDS ANTILLES||Sharon.Castillo@d.com|2452326| +9094|AAAAAAAAGIDCAAAA|1436998|3109|10467|2450072|2450042|Sir|Stephen|Sanders|N|4|2|1962|MONTENEGRO||Stephen.Sanders@Ah2sZfxLGailKoybb.edu|2452284| +9095|AAAAAAAAHIDCAAAA|1813631|4796|44808|2451911|2451881|Sir|Robert|Schneider|N|17|11|1930|CANADA||Robert.Schneider@Ji3hFYOlK3.edu|2452346| +9096|AAAAAAAAIIDCAAAA|60233|6349|865|2451550|2451520|Mr.|Charles|Meyer|N|2|10|1929|WALLIS AND FUTUNA||Charles.Meyer@7XMk8Lkf8BtfZxMv1.edu|2452610| +9097|AAAAAAAAJIDCAAAA|975659|5279|47252|2450981|2450951|Miss|Pauline|Tindall|N|2|12|1988|SLOVENIA||Pauline.Tindall@N2A.edu|2452476| +9098|AAAAAAAAKIDCAAAA|1870437|460|48930|2450047|2450017|Mr.|Benjamin|Evans|Y|27|4|1946|MALI||Benjamin.Evans@YqQm4O4Iyb.com|2452510| +9099|AAAAAAAALIDCAAAA|336078|2654|14877|2451174|2451144|Dr.|Norman|Bernstein|N|6|12|1939|WESTERN SAHARA||Norman.Bernstein@1rj7QMMmxykRA4zhL.com|2452308| +9100|AAAAAAAAMIDCAAAA|605464|1080|12743|2449584|2449554|Mrs.|Lisa|Mccormick|N|24|8|1987|SURINAME||Lisa.Mccormick@JFa1FtdB.com|2452630| +9101|AAAAAAAANIDCAAAA|991283|6182|23528|2449045|2449015|Sir|Dennis|Correia|Y|2|7|1953|SWITZERLAND||Dennis.Correia@qzGIEm.edu|2452543| +9102|AAAAAAAAOIDCAAAA|1441115|808|36101|2450315|2450285|Ms.|Marguerite|Martin|Y|9|1|1935|ALGERIA||Marguerite.Martin@uM5Us.edu|2452550| +9103|AAAAAAAAPIDCAAAA|1817013|4503|11993|2452012|2451982|Miss|Joni|Odom|Y|20|7|1988|KENYA||Joni.Odom@N5Al1nMAm8QR.com|2452562| +9104|AAAAAAAAAJDCAAAA|1856647|2054|13908|2450704|2450674|Ms.|Julie|Ayers|N|26|5|1962|ARGENTINA||Julie.Ayers@Vs56vC34cC1.edu|2452442| +9105|AAAAAAAABJDCAAAA|561552|5071|11790|2450801|2450771|Sir|Don|Bradshaw|N|12|8|1983|KAZAKHSTAN||Don.Bradshaw@e1EaA.com|2452405| +9106|AAAAAAAACJDCAAAA|833425|4498|47971|2450121|2450091|Dr.|Diana|Hutcherson|Y|6|9|1970|CAPE VERDE||Diana.Hutcherson@mTOCyu2FXR.org|2452483| +9107|AAAAAAAADJDCAAAA|792778|222|39619|2449892|2449862|Mr.|Darius|Marquez|N|16|6|1981|UNITED STATES||Darius.Marquez@PleSqZ1QM86vz5.org|2452526| +9108|AAAAAAAAEJDCAAAA|1472172|5491|11657|2451455|2451425|Miss|Velma|Poston|N|12|9|1941|CAMBODIA||Velma.Poston@DCN3Grt5CIZ00.org|2452283| +9109|AAAAAAAAFJDCAAAA|847108|1450|29345|2451292|2451262|Sir|Max|Holton|N|13|9|1976|TRINIDAD AND TOBAGO||Max.Holton@T2otoJIdmpPqR.com|2452478| +9110|AAAAAAAAGJDCAAAA|674063|2478|7293|2451883|2451853|Ms.|Araceli|Young|N|16|10|1947|LUXEMBOURG||Araceli.Young@lRbh.com|2452472| +9111|AAAAAAAAHJDCAAAA|1650120|856|7145|2449838|2449808|Ms.|Carla|Manns|Y|1|3|1978|MAURITANIA||Carla.Manns@h5RBOl.org|2452573| +9112|AAAAAAAAIJDCAAAA|933761|2027|14340|2450488|2450458|Miss|Lucille|Mann|Y|15|9|1956|IRAQ||Lucille.Mann@UvcnOfTciXh4d8Vhliz.org|2452432| +9113|AAAAAAAAJJDCAAAA|98866|4363|11507|2450155|2450125|Mrs.|Elizabeth|Mcpherson|N|3|10|1961|CUBA||Elizabeth.Mcpherson@vkKKal.org|2452390| +9114|AAAAAAAAKJDCAAAA|167624|533|27442|2449371|2449341|Ms.|Esther|Fisher|N|15|8|1979|MOLDOVA, REPUBLIC OF||Esther.Fisher@7A1Gfh6XncQz.edu|2452559| +9115|AAAAAAAALJDCAAAA|1103136|5002|35268|2451133|2451103|Dr.|Gary|Griffiths|N|5|4|1980|IRELAND||Gary.Griffiths@rPg9IAdGX7B0VqK8B.com|2452478| +9116|AAAAAAAAMJDCAAAA|1860273|2323|14373|2451190|2451160|Dr.|William|Bailey|N|7|4|1949|MALAYSIA||William.Bailey@dEqC.org|2452367| +9117|AAAAAAAANJDCAAAA|1014355|5480|24605|2452469|2452439|Dr.|Eric|Stockton|Y|24|6|1935|MALAWI||Eric.Stockton@NBy2VNj3U.org|2452523| +9118|AAAAAAAAOJDCAAAA|634092|5054|7663|2449628|2449598|Dr.|James|Collins|N|23|3|1925|UKRAINE||James.Collins@fQbvFDUm8ldXmT6AbJh.org|2452389| +9119|AAAAAAAAPJDCAAAA|940402|6152|47149|2449034|2449004|Dr.|||N||10|1983|ARGENTINA||George.Mckinney@9ZI8fKufVxly.edu|2452455| +9120|AAAAAAAAAKDCAAAA|1788759|2050|34941|2452561|2452531|Dr.|Frank|Gibson|Y|24|2|1957|MONACO||Frank.Gibson@7QmM1QcKqT9.org|2452611| +9121|AAAAAAAABKDCAAAA|1528604|3483|11162|2451793|2451763|Sir|Kenneth|Matney|N|8|12|1970|EL SALVADOR||Kenneth.Matney@zu2lgOM15jjGYV.org|2452562| +9122|AAAAAAAACKDCAAAA|639635|3228|40821|2452184|2452154|Ms.|Victoria|Olsen|Y|30|9|1987|CHRISTMAS ISLAND||Victoria.Olsen@5gShNo8eav9.edu|2452370| +9123|AAAAAAAADKDCAAAA|209972|5697|37375|2450919|2450889|Miss|Karen|Proffitt|Y|25|12|1941|MOLDOVA, REPUBLIC OF||Karen.Proffitt@3arqUjeF377f.com|2452510| +9124|AAAAAAAAEKDCAAAA|1047347|1381|10662|2451150|2451120|Mrs.|Victoria|Morrison|N|23|9|1939|VANUATU||Victoria.Morrison@J.edu|2452505| +9125|AAAAAAAAFKDCAAAA|501684|5037|8837|2452482|2452452|Mr.|Tony|Robinson|Y|18|8|1980|MARTINIQUE||Tony.Robinson@N8Z23Jm.org|2452372| +9126|AAAAAAAAGKDCAAAA|1337753|4454|41508|2450164|2450134|Sir|Harold|Mccarthy|Y|19|7|1924|GEORGIA||Harold.Mccarthy@stQXVvaLlJLOL.org|2452485| +9127|AAAAAAAAHKDCAAAA||845|30826|2450820|2450790||Douglas|Beck||29|9|1926|NEPAL||Douglas.Beck@HfQLP3pP.com|2452450| +9128|AAAAAAAAIKDCAAAA|177858|4600|12542|2449551|2449521|Ms.|Gertrude|Vargas|N|8|8|1988|QATAR||Gertrude.Vargas@3HEsys24.edu|2452523| +9129|AAAAAAAAJKDCAAAA|225475|2134|10793|2452147|2452117|Mrs.|Cathy|Fairley|N|13|9|1978|AUSTRALIA||Cathy.Fairley@K.org|2452592| +9130|AAAAAAAAKKDCAAAA|1663595|3326|22304|2450171|2450141|Miss|Diane|Steward|Y|2|4|1958|BAHAMAS||Diane.Steward@Efn4nvN8ZPvTlAX.com|2452397| +9131|AAAAAAAALKDCAAAA|261326|679|36032|2449568|2449538|Ms.|Marie|Golden|Y|15|5|1953|MALAWI||Marie.Golden@e.com|2452560| +9132|AAAAAAAAMKDCAAAA|1239463|2388|38066|2449940|2449910|Sir|Charles|Holmes|N|9|7|1927|TIMOR-LESTE||Charles.Holmes@mU2Npu8ZOXS.org|2452607| +9133|AAAAAAAANKDCAAAA|1884009|5633|44104|2451311|2451281|Dr.|Robert|Garrett|Y|27|1|1959|FRENCH POLYNESIA||Robert.Garrett@VpvS9X6msV.edu|2452497| +9134|AAAAAAAAOKDCAAAA|1802191|5941|32044|2451927|2451897|Sir|Rodney|Rowan|N|14|2|1979|MALAWI||Rodney.Rowan@8mQCIqkou2JD.org|2452324| +9135|AAAAAAAAPKDCAAAA|1210289|762|38835|2451790|2451760|Ms.|Mitsue|Dixon|Y|24|4|1989|GUERNSEY||Mitsue.Dixon@XuFP8n3cYKld.org|2452290| +9136|AAAAAAAAALDCAAAA|813528|5057|6939|2451271|2451241|Dr.|Vanessa|Smith|Y|22|11|1956|MONGOLIA||Vanessa.Smith@bVHG.org|2452637| +9137|AAAAAAAABLDCAAAA|1588343|1430|36517|2449811|2449781|Ms.|Shara|Smith|N|15|4|1932|GEORGIA||Shara.Smith@f0gv7.com|2452507| +9138|AAAAAAAACLDCAAAA|1041582|5228|44874|2450365|2450335|Mr.|Damon|Boehm|N|12|12|1953|GUADELOUPE||Damon.Boehm@K.edu|2452498| +9139|AAAAAAAADLDCAAAA|1037669|3522|15375|2451327|2451297|Mr.|Leroy|Bassett|N|23|12|1942|ERITREA||Leroy.Bassett@RiiDz.com|2452289| +9140|AAAAAAAAELDCAAAA|510820|6211|21602|2449343|2449313|Sir|Ben|Bright|Y|18|6|1940|UZBEKISTAN||Ben.Bright@04ef6C.com|2452634| +9141|AAAAAAAAFLDCAAAA|1910833|1291|10949|2450732|2450702|Dr.|Benjamin|Kaplan|Y|17|7|1989|MALAWI||Benjamin.Kaplan@DF6huvUF0AU.org|2452339| +9142|AAAAAAAAGLDCAAAA|1670874|4897|31367|2451697|2451667|Sir|James|Warren|Y|16|3|1970|TOKELAU||James.Warren@1VRueRy1f.com|2452359| +9143|AAAAAAAAHLDCAAAA|1875883|942|18401|2449761|2449731|Mr.|Benjamin|Evans|N|25|8|1934|GIBRALTAR||Benjamin.Evans@qICom1mQK8GYQ.edu|2452571| +9144|AAAAAAAAILDCAAAA||1649|33196|||Sir|Timothy||Y|12|10|||||| +9145|AAAAAAAAJLDCAAAA|129504||17739||2449764||||Y||10||GEORGIA||Glenn.Erwin@g0Q3.com|2452613| +9146|AAAAAAAAKLDCAAAA|1296166|672|9131|2450579|2450549|Mr.|Wayne|Frey|N|2|6|1939|INDONESIA||Wayne.Frey@z99Mvr.edu|2452595| +9147|AAAAAAAALLDCAAAA|140316|1057|19398|2450594|2450564|Dr.|Floyd|Allen|N|31|8|1966|TONGA||Floyd.Allen@Kk.com|2452460| +9148|AAAAAAAAMLDCAAAA|1483118|4036|4554|2451186|2451156|Dr.|Delores|Henry|N|23|4|1986|VENEZUELA||Delores.Henry@5.com|2452533| +9149|AAAAAAAANLDCAAAA|499930|5449|32215|2451768|2451738|Ms.|Eve|Williams|Y|21|9|1970|PHILIPPINES||Eve.Williams@I.com|2452388| +9150|AAAAAAAAOLDCAAAA|177391|52|35866|2452299|2452269|Dr.|Grant|Mcintosh|Y|29|12|1991|NEW ZEALAND||Grant.Mcintosh@st0LuHqzqCfCjbc7bUh.org|2452630| +9151|AAAAAAAAPLDCAAAA|978405|6164|5076|2449619|2449589|Mr.|Patrick|Johnson|Y|10|7|1925|PAKISTAN||Patrick.Johnson@lEk.org|2452455| +9152|AAAAAAAAAMDCAAAA|293230|2038|15999|2451715|2451685|Miss|Peggy|Petty|Y|25|8|1947|KIRIBATI||Peggy.Petty@ELg3TP9uM.com|2452632| +9153|AAAAAAAABMDCAAAA|750105|5627|36396|2449633|2449603|Mrs.|Sandra|Galloway|N|21|12|1952|SAUDI ARABIA||Sandra.Galloway@TkQ073gOvOAGf8CBUQ.org|2452511| +9154|AAAAAAAACMDCAAAA|681918|4090|41462|2451782|2451752|Dr.|Timothy|Clark|N|16|10|1947|ITALY||Timothy.Clark@d.org|2452329| +9155|AAAAAAAADMDCAAAA|1431345|5208|19071|2451910|2451880|Mr.|Thomas|Robbins|Y|1|7|1966|GUADELOUPE||Thomas.Robbins@PNOfng.edu|2452373| +9156|AAAAAAAAEMDCAAAA|559299|5423|20794|2449750|2449720|Dr.|Daniel|Andrade|Y|11|11|1927|SIERRA LEONE||Daniel.Andrade@uorr00d4mqFM3.com|2452479| +9157|AAAAAAAAFMDCAAAA|264720|5056|19439|2452468|2452438|Mr.|Frank|Frazier|N|23|3|1989|LITHUANIA||Frank.Frazier@ZSg7qaJeCi9.com|2452295| +9158|AAAAAAAAGMDCAAAA|6750|5135|7167|2450740|2450710|Sir|Randy|Mount|N|26|10|1955|SRI LANKA||Randy.Mount@R7ZTclyf1HUDMqH.org|2452293| +9159|AAAAAAAAHMDCAAAA|1036970|5313|38056|2449604|2449574|Ms.|Sandra|Friedman|N|9|8|1963|IRAQ||Sandra.Friedman@SmDb4EB.org|2452560| +9160|AAAAAAAAIMDCAAAA|1327474|5931|30035|2451336|2451306|Sir|Trevor|Fleming|Y|22|3|1966|ZIMBABWE||Trevor.Fleming@sNoBYIxqcuKP8jA9.org|2452398| +9161|AAAAAAAAJMDCAAAA|145384|5900|13169|2450503|2450473|Miss|Ann|Pimentel|Y|24|5|1968|HONDURAS||Ann.Pimentel@1.org|2452409| +9162|AAAAAAAAKMDCAAAA|1720644|6744|7197|2452386|2452356|Ms.|Louise|Schreiber|Y|4|1|1958|BENIN||Louise.Schreiber@NH.edu|2452635| +9163|AAAAAAAALMDCAAAA|1915728|6810|11939|2449307|2449277|Miss|Laurel|Schmid|N|20|9|1970|NETHERLANDS ANTILLES||Laurel.Schmid@xLzQydxdQt.edu|2452407| +9164|AAAAAAAAMMDCAAAA|1492495|6074|32006|2450323|2450293|Mrs.|Abigail|Davis|Y|22|6|1935|NEW CALEDONIA||Abigail.Davis@p.org|2452450| +9165|AAAAAAAANMDCAAAA|386965|5925|34838|2451357|2451327|Dr.|Cindy|Solis|Y|11|9|1944|KENYA||Cindy.Solis@aZZGeacTM080a.edu|2452469| +9166|AAAAAAAAOMDCAAAA|1517820|5539|11258|2449267|2449237|Mrs.|Malinda|Silva|Y|30|6|1930|SAINT LUCIA||Malinda.Silva@8DFzDbEApHCg.edu|2452293| +9167|AAAAAAAAPMDCAAAA|372773|1244|42468|2449079|2449049|Dr.|Vernon|Spears|N|5|10|1941|FRENCH POLYNESIA||Vernon.Spears@AgCjJ6RAIIU7Kkx2Q.com|2452412| +9168|AAAAAAAAANDCAAAA|1161964|1255|9346|2452565|2452535|Sir|Arnold|Cruz|N|9|3|1972|BURKINA FASO||Arnold.Cruz@kSr2uZIg5y.com|2452401| +9169|AAAAAAAABNDCAAAA|265490|5814|18938|2450773|2450743|Sir|Steven|Gardner|N|21|8|1964|TRINIDAD AND TOBAGO||Steven.Gardner@P8LpdAJNYHUXplJg.org|2452317| +9170|AAAAAAAACNDCAAAA|1032145|1340|12935|2450395|2450365|Mrs.|Lucy|Martinez|Y|24|6|1964|BAHRAIN||Lucy.Martinez@DNXjEuJ.com|2452570| +9171|AAAAAAAADNDCAAAA|1838364|1499|10681|2451481|2451451|Mrs.|Dora|Reid|Y|1|2|1978|SRI LANKA||Dora.Reid@tHBbeEsZ.com|2452647| +9172|AAAAAAAAENDCAAAA|106877|6805|11486|2449873|2449843|Sir|Mark|Johnson|Y|2|4|1963|ISRAEL||Mark.Johnson@uVooVCm9IdVBVNqpL6.edu|2452409| +9173|AAAAAAAAFNDCAAAA|1123166|1234|46502|2451276|2451246|Dr.|Lisa|Buchanan|Y|2|1|1925|SINGAPORE||Lisa.Buchanan@euj.com|2452475| +9174|AAAAAAAAGNDCAAAA|1165749|1860|5124|2452595|2452565|Sir|Rick|Robinson|Y|7|6|1990|MONGOLIA||Rick.Robinson@uH.com|2452301| +9175|AAAAAAAAHNDCAAAA|160536|5224|27865|2451557|2451527|Dr.|Timmy|Franklin|N|6|1|1979|JORDAN||Timmy.Franklin@Mg.com|2452389| +9176|AAAAAAAAINDCAAAA|1558693|3502|11511|2449197|2449167|Miss|Cristal|Gonzales|Y|31|3|1966|BOUVET ISLAND||Cristal.Gonzales@G67u.edu|2452555| +9177|AAAAAAAAJNDCAAAA|1098301|5740|9626|2449115|2449085|Sir|Joseph|Herron|Y|25|5|1928|ZAMBIA||Joseph.Herron@C.com|2452634| +9178|AAAAAAAAKNDCAAAA|1752970|671|37972|2451978|2451948|Dr.|Brian|Pantoja|N|6|2|1965|JAPAN||Brian.Pantoja@kqDB.com|2452322| +9179|AAAAAAAALNDCAAAA|289750|2465|45580|2449390|2449360|Dr.|John|Daniel|Y|19|2|1938|BELIZE||John.Daniel@icH.org|2452325| +9180|AAAAAAAAMNDCAAAA|392708|540|23972|2451580|2451550|Miss|Jill|Ellsworth|Y|12|12|1943|NEW CALEDONIA||Jill.Ellsworth@pY9youQBPe8vC.org|2452367| +9181|AAAAAAAANNDCAAAA|1333058|4704|6703|2452404|2452374|Dr.|Russell|Bennett|Y|17|12|1963|PORTUGAL||Russell.Bennett@Djdt2BKPVo.edu|2452570| +9182|AAAAAAAAONDCAAAA|914806|3647|14088|2450550|2450520|Sir|Russell|Gray|N|3|10|1970|DOMINICA||Russell.Gray@qjhcJUF5UqIKFzGxr1.edu|2452298| +9183|AAAAAAAAPNDCAAAA|863898|760|37622|2449418|2449388|Sir|William|Reese|Y|25|2|1929|TONGA||William.Reese@CBQD0.org|2452302| +9184|AAAAAAAAAODCAAAA|918677|6604|44531|2450802|2450772|Dr.|Judith|Wilson|Y|10|11|1980|PARAGUAY||Judith.Wilson@pzHmgf9tl.com|2452436| +9185|AAAAAAAABODCAAAA|513817|7140|42463|2451856|2451826|Sir|John|Fischer|Y|29|3|1957|ANGUILLA||John.Fischer@US7OYJ.edu|2452572| +9186|AAAAAAAACODCAAAA|954933|641|37996|2451563|2451533|Ms.|Kathie|Jones|Y|23|5|1930|NIGER||Kathie.Jones@6e.edu|2452473| +9187|AAAAAAAADODCAAAA|540071|1541|45411|2451309|2451279|Mr.|Larry|Gilbert|Y|11|7|1988|KENYA||Larry.Gilbert@3fu0Iuqf2.org|2452377| +9188|AAAAAAAAEODCAAAA|1755548|2180|47165|2450399|2450369|Miss|Sara|Ramos|Y|4|1|1978|TURKEY||Sara.Ramos@0ACbz82FE8Za.com|2452455| +9189|AAAAAAAAFODCAAAA|11791|3348|41705|2451657|2451627|Mr.|David|Kelley|N|13|4|1950|BANGLADESH||David.Kelley@LuqRdHJalxa67.com|2452485| +9190|AAAAAAAAGODCAAAA|672249|1125|3601|2452333|2452303|Dr.|Douglas|Mejia|N|21|10|1988|PUERTO RICO||Douglas.Mejia@pnVUzhxE3OxN.edu|2452384| +9191|AAAAAAAAHODCAAAA|324228|5375|46922|2449030|2449000|Mrs.|Marlene|Huerta|Y|25|5|1962|THAILAND||Marlene.Huerta@MDJG.org|2452420| +9192|AAAAAAAAIODCAAAA|819051|2724|11597|2452278|2452248|Mr.|Jack|Bryant|Y|1|5|1983|FRENCH POLYNESIA||Jack.Bryant@AQbUr.edu|2452298| +9193|AAAAAAAAJODCAAAA|477706|5141|31776|2450232|2450202|Mrs.|Nina|Falls|Y|23|5|1934|GREECE||Nina.Falls@cdhSzIICgX.edu|2452454| +9194|AAAAAAAAKODCAAAA|813664|171|28278|2450419|2450389|Mr.|Rodney|Finley|N|31|8|1936|URUGUAY||Rodney.Finley@IoubZG0aAfYx8xUeD.com|2452371| +9195|AAAAAAAALODCAAAA|312413|6570|42374|||||Dooley|N||||SPAIN|||| +9196|AAAAAAAAMODCAAAA|39760|6685|28010|2450671|2450641|Ms.|Edith|Williams|N|18|8|1952|AMERICAN SAMOA||Edith.Williams@v1rqkZXL.org|2452486| +9197|AAAAAAAANODCAAAA|908342|3014|44933|2449709|2449679|Mrs.|Shawna|Macklin|Y|20|4|1929|SENEGAL||Shawna.Macklin@DH0gpaYhHDZen.edu|2452435| +9198|AAAAAAAAOODCAAAA|177267|1416|32412|2452589|2452559|Ms.|Jessica|Cruz|Y|29|9|1926|VIRGIN ISLANDS, U.S.||Jessica.Cruz@LMdzZ5jENpA.com|2452642| +9199|AAAAAAAAPODCAAAA|703336|7134|12132|2451806|2451776|Sir|David|Dodson|N|11|1|1979|MARSHALL ISLANDS||David.Dodson@jZS.org|2452311| +9200|AAAAAAAAAPDCAAAA|1765254|5394|48734|2451232|2451202|Dr.|Paulette|Stamm|N|22|9|1960|GUATEMALA||Paulette.Stamm@ziU0u.edu|2452647| +9201|AAAAAAAABPDCAAAA|1801832|2915|6798|2451078|2451048|Mrs.|Marie|Johnson|Y|25|10|1986|TUVALU||Marie.Johnson@JSXPrbSkFfuCvGojIR.edu|2452437| +9202|AAAAAAAACPDCAAAA|1639553|1120|1487|2450693|2450663|Mr.|Fred|Hicks|N|25|5|1955|ANGUILLA||Fred.Hicks@el.edu|2452516| +9203|AAAAAAAADPDCAAAA|1069128|1776|19624|2451049|2451019|Mrs.|Sonya|Trejo|Y|10|3|1933|GUINEA-BISSAU||Sonya.Trejo@6.org|2452420| +9204|AAAAAAAAEPDCAAAA|1541138|5176|26367|2449149|2449119|Dr.|Vincent|Morey|N|17|2|1924|BELIZE||Vincent.Morey@vrt8Db3fK05tVXyK.edu|2452377| +9205|AAAAAAAAFPDCAAAA|317818|3417|25369|2452284|2452254|Mr.|Vaughn|Walker|N|20|10|1991|BERMUDA||Vaughn.Walker@MNBxeCsqEkcy3g.edu|2452380| +9206|AAAAAAAAGPDCAAAA|79510|3928|40393|2450695|2450665|Mr.|Kevin|Smith|Y|18|8|1970|KAZAKHSTAN||Kevin.Smith@qjiMrFlYDdQPXdZo.com|2452566| +9207|AAAAAAAAHPDCAAAA|345555|2365|1222|2452612|2452582|Mr.|Pedro|Ford|Y|7|4|1984|SWAZILAND||Pedro.Ford@gX8mkyvmNV7mj.com|2452488| +9208|AAAAAAAAIPDCAAAA|1406501|5027|49106|2449063|2449033|Mr.|Eric|Wallace|N|9|9|1968|AZERBAIJAN||Eric.Wallace@60KxkZqE.com|2452420| +9209|AAAAAAAAJPDCAAAA|334729|6340|304|2450503|2450473|Mrs.|Dorothy|Woodcock|Y|11|1|1933|BHUTAN||Dorothy.Woodcock@XUhlr.com|2452381| +9210|AAAAAAAAKPDCAAAA|116953|4886|29219|2451308|2451278|Dr.|Gregory|Johnson|N|10|3|1927|TUNISIA||Gregory.Johnson@lmfX.edu|2452501| +9211|AAAAAAAALPDCAAAA|395095|5189|2036|2449750|2449720|Sir|Stephen|Wilson|Y|25|7|1953|NAURU||Stephen.Wilson@G3.edu|2452308| +9212|AAAAAAAAMPDCAAAA|1884708|2483|11858|2452128|2452098|Miss|Ashley|Smith|N|28|8|1987|FRENCH POLYNESIA||Ashley.Smith@IipQamFs9IgUxtb.edu|2452379| +9213|AAAAAAAANPDCAAAA|401600|4308|20616|2451562|2451532|Ms.|Amy|Foster|N|21|5|1990|MADAGASCAR||Amy.Foster@8CT9K0ITcMUxPBc.com|2452359| +9214|AAAAAAAAOPDCAAAA|693187|986|10832||2452441|Miss|Krista|Small|Y|5|5|1989|||Krista.Small@DHAxkKAaLXbyoj.com|| +9215|AAAAAAAAPPDCAAAA|1250198|886|13127|2451055|2451025|Miss|Gretchen|Whitcomb|N|14|9|1991|BARBADOS||Gretchen.Whitcomb@avqMa16l5K.org|2452300| +9216|AAAAAAAAAAECAAAA|1343237|7047|10|2452061|2452031|Dr.|Henry|Perry|Y|28|11|1958|UNITED KINGDOM||Henry.Perry@CVYJgu3mrui.org|2452484| +9217|AAAAAAAABAECAAAA|1454219|4947|20028|2449310|2449280|Dr.|Evelyn|Friedman|Y|8|3|1964|NORFOLK ISLAND||Evelyn.Friedman@Di5hqrCP.org|2452550| +9218|AAAAAAAACAECAAAA|||11253||||Joshua|||||1927|PITCAIRN||Joshua.Brazil@N.org|| +9219|AAAAAAAADAECAAAA||1579|36825||2451636||Linda|||29|||||Linda.Gordon@Ql69xxi5G69xcq2CoT.edu|2452399| +9220|AAAAAAAAEAECAAAA|1084650|5210|1017|2451885|2451855|Dr.|Vivian|Graves|Y|8|10|1986|NEW ZEALAND||Vivian.Graves@d1oidm0.org|2452594| +9221|AAAAAAAAFAECAAAA|1890080|4562|38244|2451638|2451608|Sir|Charles|Mays|Y|26|4|1979|VENEZUELA||Charles.Mays@NXItZCd93tPo1UDd.org|2452419| +9222|AAAAAAAAGAECAAAA|1019332|1196|42660|2452036|2452006|Mr.|Sheldon|Reiss|N|9|1|1980|ALAND ISLANDS||Sheldon.Reiss@4la.com|2452557| +9223|AAAAAAAAHAECAAAA|1387772|6618|47911|2450112|2450082|Ms.|Adam|Jimenez|N|1|7|1987|MALAYSIA||Adam.Jimenez@nAAyq.org|2452482| +9224|AAAAAAAAIAECAAAA|1366449|1340|30631|2450706|2450676|Mr.|Alvin|Medina|Y|8|10|1978|BURKINA FASO||Alvin.Medina@16G.com|2452487| +9225|AAAAAAAAJAECAAAA|286133|4048|49163|2451276|2451246|Dr.|Linda|Ballard|Y|12|2|1932|UNITED KINGDOM||Linda.Ballard@gLKyUx.org|2452632| +9226|AAAAAAAAKAECAAAA|712327|3668|10162|2452633|2452603|Sir|Milton|Mackey|N|26|5|1990|RUSSIAN FEDERATION||Milton.Mackey@QOPtZd.org|2452343| +9227|AAAAAAAALAECAAAA|1661195|3267|24092|2450468|2450438|Sir|Timothy|Stover|N|31|3|1964|UGANDA||Timothy.Stover@rpt8NVND.edu|2452633| +9228|AAAAAAAAMAECAAAA|726052|1969|21486|2451512|2451482|Dr.|Thomas|Moore|Y|28|9|1976|MALAWI||Thomas.Moore@q2J76Th6sq6bfC28d.com|2452574| +9229|AAAAAAAANAECAAAA|1909004|3120|48036|2449925|2449895|Dr.|Martin|Grubbs|N|31|10|1933|PORTUGAL||Martin.Grubbs@QPerVsUyZX.edu|2452531| +9230|AAAAAAAAOAECAAAA|1689127|2096|46218|2449335|2449305|Dr.|Alan|France|N|14|1|1982|NORFOLK ISLAND||Alan.France@yszz2.edu|2452563| +9231|AAAAAAAAPAECAAAA|949503||43322|||Mr.||Stark||21|5|1970|||Armando.Stark@RTVjtKHUP15jCmH.com|2452477| +9232|AAAAAAAAABECAAAA|911876|464|24699|2449615|2449585|Miss|Estela|Frost|Y|10|1|1959|AFGHANISTAN||Estela.Frost@xgcpR7x.com|2452601| +9233|AAAAAAAABBECAAAA|1080294|3002|48863|2451355|2451325|Ms.|Stacie|Berry|N|27|1|1954|LIECHTENSTEIN||Stacie.Berry@TYYdc8mBNMtbnM.com|2452398| +9234|AAAAAAAACBECAAAA|39724|6279|12081|2451047|2451017|Dr.|John|Mcdonald|N|16|12|1958|ANGUILLA||John.Mcdonald@2Q6P2I.org|2452453| +9235|AAAAAAAADBECAAAA|1044708|6168|49628|2451721|2451691|Mr.|Armando|Neal|N|19|5|1949|PANAMA||Armando.Neal@MeLiKS0.com|2452574| +9236|AAAAAAAAEBECAAAA|683303|1819|5935|2451901|2451871|Dr.|Florence|Light|Y|27|6|1934|SERBIA||Florence.Light@8PCmMV78aRTmlrXNot.com|2452292| +9237|AAAAAAAAFBECAAAA|||20467||2449707||Barbara|||14||1973|TURKEY|||2452294| +9238|AAAAAAAAGBECAAAA|347896|3307|49871|2449609|2449579|Sir|Timothy|Jones|Y|29|3|1958|PHILIPPINES||Timothy.Jones@bPpfGa4HQYVxInSuai.edu|2452493| +9239|AAAAAAAAHBECAAAA|419420|4675|23483|2449719|2449689|Mrs.|Altagracia|Phillips|Y|2|9|1964|AUSTRIA||Altagracia.Phillips@1K37XGU6QbaqmtC.org|2452531| +9240|AAAAAAAAIBECAAAA|1700351|201|15604|2451045|2451015|Dr.|Betty|Ramirez|Y|9|7|1980|SINGAPORE||Betty.Ramirez@35I8mxVK3TJt.edu|2452417| +9241|AAAAAAAAJBECAAAA|1370917|6655|8440|2450824|2450794|Mr.|Francisco|Henson|N|2|4|1950|GERMANY||Francisco.Henson@95mGEV5.org|2452586| +9242|AAAAAAAAKBECAAAA|1076737|1803|36069|2450926|2450896|Ms.|Maurine|Leung|Y|5|7|1938|BOTSWANA||Maurine.Leung@1NtMg.edu|2452409| +9243|AAAAAAAALBECAAAA|335009|4910|42858|2450661|2450631|Dr.|Gregorio|Dunn|Y|16|5|1960|ALBANIA||Gregorio.Dunn@HJ1QSThTybsCq0.org|2452335| +9244|AAAAAAAAMBECAAAA|1502355|3157|11559|2452016|2451986|Mr.|Brian|Guzman|Y|10|6|1963|LEBANON||Brian.Guzman@I4U6tGoDfG.edu|2452382| +9245|AAAAAAAANBECAAAA|1896421|524|25874|2452327|2452297|Mr.|Carl|Conner|N|17|1|1985|ARUBA||Carl.Conner@aHcKFDgT.com|2452578| +9246|AAAAAAAAOBECAAAA|1284599|4769|42539|2450484|2450454|Dr.|William|Hilton|Y|1|9|1980|NIUE||William.Hilton@SEitDe3068nAx.org|2452647| +9247|AAAAAAAAPBECAAAA|1484311|4225|14139|2449928|2449898|Sir|Phillip|Blanchard|N|7|2|1978|NAURU||Phillip.Blanchard@yMGBR.edu|2452380| +9248|AAAAAAAAACECAAAA|1338714|2805|496|2450609|2450579|Ms.|Rosalyn|Martin|Y|26|4|1971|LATVIA||Rosalyn.Martin@pLFKkga8u7qo.com|2452509| +9249|AAAAAAAABCECAAAA|667376|3879|13229|2449314|2449284|Dr.|Amanda|Gates|Y|4|2|1943|MONGOLIA||Amanda.Gates@2Eq6qkJlqt.com|2452476| +9250|AAAAAAAACCECAAAA|1180654|7002|29397|2450091|2450061|Mr.|Norberto|Ricker|Y|2|6|1933|BAHRAIN||Norberto.Ricker@QX8eI91kaSzpSo33G.edu|2452541| +9251|AAAAAAAADCECAAAA|89074|5606|43738|2451972|2451942|Mr.|Jon|Miranda|N|27|6|1966|TIMOR-LESTE||Jon.Miranda@A7jDfef.edu|2452348| +9252|AAAAAAAAECECAAAA|939761||40492|2452226||Mrs.||Schaefer|N|16|2||GREENLAND|||2452478| +9253|AAAAAAAAFCECAAAA|1239325|5487|48096|2450796|2450766|Mr.|Roman|Carpenter|N|4|5|1990|ICELAND||Roman.Carpenter@9exNKkE4zTvebIHV.org|2452343| +9254|AAAAAAAAGCECAAAA|1723648|4420|37992|2449733|2449703|Miss|Coralie|Rust|N|19|7|1961|ALAND ISLANDS||Coralie.Rust@VyIKJjvTR3ZAI.com|2452642| +9255|AAAAAAAAHCECAAAA|579013|5396|32226|2449463|2449433|Sir|John|Nye|Y|23|4|1983|HONG KONG||John.Nye@UQpNNt8LoBZ99.edu|2452327| +9256|AAAAAAAAICECAAAA|1040783|3418|37206|2451283|2451253|Dr.|Theodora|Kelly|N|22|11|1986|NORFOLK ISLAND||Theodora.Kelly@3AlHracJQJX1a.com|2452618| +9257|AAAAAAAAJCECAAAA|1493181|248|42915|2451518|2451488|Dr.|Richard|Crowley|N|16|11|1945|BANGLADESH||Richard.Crowley@CtFxF7Hn57.org|2452499| +9258|AAAAAAAAKCECAAAA|28730|3106|1082|2452091|2452061|Mrs.|Virginia|Alexander|N|4|4|1970|RUSSIAN FEDERATION||Virginia.Alexander@7jf5aAl7TRADHpI.edu|2452486| +9259|AAAAAAAALCECAAAA|1073543|4833|14523|2450562|2450532|Sir|Dennis|Silva|N|29|10|1972|TAJIKISTAN||Dennis.Silva@cjk.com|2452425| +9260|AAAAAAAAMCECAAAA|179559|4916|36320|2452546|2452516|Dr.|Cindy|Phillips|Y|20|10|1979|GUINEA||Cindy.Phillips@fbOMBZ0a3n9BP.org|2452378| +9261|AAAAAAAANCECAAAA|838968|3283|13951|2449461|2449431|Dr.|Kevin|Hart|Y|16|7|1952|GIBRALTAR||Kevin.Hart@UOlguG7s.edu|2452474| +9262|AAAAAAAAOCECAAAA|1141871|1435|21352|2452098|2452068|Miss|Anita|Barth|N|31|7|1948|ALAND ISLANDS||Anita.Barth@smi8eXPhCOQNe.org|2452335| +9263|AAAAAAAAPCECAAAA|1679095|4885|30228|2452330|2452300|Dr.|Roberta|Teel|N|13|12|1968|ALGERIA||Roberta.Teel@N2HaRYoTGd.org|2452301| +9264|AAAAAAAAADECAAAA|1542026|1021|45663|2450901|2450871|Ms.|Pauline|Gunther|Y|10|5|1982|JAPAN||Pauline.Gunther@LqlPinFKh.com|2452506| +9265|AAAAAAAABDECAAAA|381384|7193|15002|2449716|2449686|Miss|Adele|Osburn|N|14|2|1975|GUYANA||Adele.Osburn@q5.com|2452572| +9266|AAAAAAAACDECAAAA||5189|32663|||Dr.||||7|2||||Pamela.Hansen@UPpEXZzhmQy22K0.com|2452450| +9267|AAAAAAAADDECAAAA|76009|6550|26644|2450103|2450073|Sir|James|Boyd|N|26|10|1948|BENIN||James.Boyd@528nmM6KDV.com|2452537| +9268|AAAAAAAAEDECAAAA|321926|4947|35921|2451356|2451326|Mr.|Roscoe|Chapman|Y|13|5|1957|POLAND||Roscoe.Chapman@nTRsdlnTNe.edu|2452313| +9269|AAAAAAAAFDECAAAA|698851|6301|37751|2449335|2449305|Miss|Jennifer|Wilson|N|4|10|1929|MALI||Jennifer.Wilson@NB2ADABxYxXR.edu|2452609| +9270|AAAAAAAAGDECAAAA|1235470|943|18063|2452555|2452525|Ms.|Andrea|Scott|Y|28|9|1963|BOLIVIA||Andrea.Scott@Qfy0feZsnsRm.com|2452424| +9271|AAAAAAAAHDECAAAA|1285696|881|18866|2451622|2451592|Miss|Molly|Stein|Y|29|9|1936|TUNISIA||Molly.Stein@H.edu|2452613| +9272|AAAAAAAAIDECAAAA|1619158|2346|36334|2451590|2451560|Mr.|Jeremy|Aviles|N|26|11|1980|GUADELOUPE||Jeremy.Aviles@erdeR.com|2452419| +9273|AAAAAAAAJDECAAAA|359832|927|20069|2451214|2451184|Sir|Gabriel|Brown|N|10|9|1990|ETHIOPIA||Gabriel.Brown@V8v9DoFafieZDjYi.com|2452422| +9274|AAAAAAAAKDECAAAA|1520299|118|11890||2449468||||||||||Ann.Wallace@t3ixAZNul.com|| +9275|AAAAAAAALDECAAAA|1140997|362|43727|2451256|2451226|Miss|Wanda|Howard|Y|13|1|1977|SERBIA||Wanda.Howard@iQYvS1czTjfqP.com|2452321| +9276|AAAAAAAAMDECAAAA|1865430|4582|36934|2452052|2452022|Mr.|Todd|Foreman|N|20|12|1979|GEORGIA||Todd.Foreman@HA.com|2452580| +9277|AAAAAAAANDECAAAA|69753|2058|12316|2452193|2452163|Dr.|James|Hinds|Y|11|10|1956|BELARUS||James.Hinds@MJXqBl0mECiySHSih.com|2452578| +9278|AAAAAAAAODECAAAA|65837|4813|6636|2451259|2451229|Ms.|Andrew|Garcia|Y|5|4|1981|BAHAMAS||Andrew.Garcia@ofovDS.edu|2452545| +9279|AAAAAAAAPDECAAAA|1235412|3180|18148|2451133|2451103|Ms.|Faye|Rainey|N|26|3|1968|ICELAND||Faye.Rainey@de.org|2452464| +9280|AAAAAAAAAEECAAAA||4470|10664|2452326||Dr.|Anne||Y|25||1957|MALTA|||2452364| +9281|AAAAAAAABEECAAAA|||35133|2450526|2450496|Dr.|William||N|16|1||BERMUDA||William.Griffin@m06lIkn4ziHhO.com|| +9282|AAAAAAAACEECAAAA|1677890|347|41957|2451790|2451760|Mr.|Thomas|Wheeler|Y|11|4|1966|CZECH REPUBLIC||Thomas.Wheeler@TbXnzZfxtUzqd.com|2452516| +9283|AAAAAAAADEECAAAA|619504|6917|45177|2452152|2452122|Dr.|Sylvia|Harris|N|29|11|1926|SEYCHELLES||Sylvia.Harris@Arzq7gfTVgP55UT.com|2452571| +9284|AAAAAAAAEEECAAAA|1765688|6226|45917|2451082|2451052|Ms.|Barbara|Fuentes|N|21|10|1980|NICARAGUA||Barbara.Fuentes@AREpZFj.edu|2452339| +9285|AAAAAAAAFEECAAAA|366037|6308|24226|2449910|2449880|Sir|Kenneth|Kelleher|Y|28|9|1946|ETHIOPIA||Kenneth.Kelleher@KV9UXHAcbXDM73IlANC.com|2452369| +9286|AAAAAAAAGEECAAAA|773475|4503|4731|2450579|2450549|Miss|Joyce|Wilson|N|2|7|1936|CAMBODIA||Joyce.Wilson@d5MMB.org|2452606| +9287|AAAAAAAAHEECAAAA|135124|4689|37901|2452094|2452064|Mrs.|Miriam|Lowery|N|23|4|1932|ALBANIA||Miriam.Lowery@rF1LjZXrNkVmh1sh8.com|2452560| +9288|AAAAAAAAIEECAAAA|1448804|2929|8254|2451107|2451077|Sir|Christopher|Koenig|Y|14|5|1964|JAMAICA||Christopher.Koenig@OniC5el.org|2452520| +9289|AAAAAAAAJEECAAAA|726939|6714|28889|2451681|2451651|Dr.|Anne|Scott|N|20|9|1980|JORDAN||Anne.Scott@xIh6gpu.org|2452456| +9290|AAAAAAAAKEECAAAA|111720|3238|29603|2451184|2451154|Dr.|Jason|Pak|N|30|12|1955|SENEGAL||Jason.Pak@zF.edu|2452624| +9291|AAAAAAAALEECAAAA|135114|2391|33315|2452248|2452218|Sir|Robert|Pena|N|10|11|1987|MALTA||Robert.Pena@2I3GYPgEA.com|2452591| +9292|AAAAAAAAMEECAAAA|1448327|5885|10196|2451671|2451641|Mr.|Bruce|Daly|N|5|3|1925|SAUDI ARABIA||Bruce.Daly@chDqjYS.edu|2452310| +9293|AAAAAAAANEECAAAA|430381|1146|10311|2450639|2450609|Mr.|Donovan|Stevens|N|26|2|1973|SAN MARINO||Donovan.Stevens@h9.org|2452397| +9294|AAAAAAAAOEECAAAA|1282341|4381|43268|2452570|2452540|Mr.|William|Clark|Y|14|3|1979|ANDORRA||William.Clark@4NzO.org|2452550| +9295|AAAAAAAAPEECAAAA|1109352|6736|43367|2452531|2452501|Mr.|Alan|Houck|Y|15|11|1928|BRUNEI DARUSSALAM||Alan.Houck@S0OiEZX9dVZaHX9.edu|2452497| +9296|AAAAAAAAAFECAAAA|975389|3558|44927|2451149|2451119|Mrs.|Dolores|Davila|Y|31|12|1965|HUNGARY||Dolores.Davila@e3Izmlhzu.com|2452503| +9297|AAAAAAAABFECAAAA|1723040|3719|47198|2450427|2450397|Miss|Jesica|Williams|Y|25|1|1949|ERITREA||Jesica.Williams@Y50iDSEUmECYT.com|2452297| +9298|AAAAAAAACFECAAAA|687729|6873|22370|2449292|2449262|Miss|Dora|Hernandez|Y|8|8|1965|MALDIVES||Dora.Hernandez@ikbaI1G4CxxnEOi.org|2452513| +9299|AAAAAAAADFECAAAA|||78|2451609|||||Y|6|5|1933||||| +9300|AAAAAAAAEFECAAAA|1287198|4025|18005|2451829|2451799|Dr.|Tyrone|Shelby|N|23|2|1942|URUGUAY||Tyrone.Shelby@c.edu|2452458| +9301|AAAAAAAAFFECAAAA|1135762|2053|49991|2452609|2452579|Dr.|Elizabeth|Stroud|N|1|12|1928|BOTSWANA||Elizabeth.Stroud@ORU068n58I0BCkDg61.com|2452593| +9302|AAAAAAAAGFECAAAA|633053|198|43696|2451158|2451128|Dr.|Pablo|Hubbard|N|28|6|1932|ESTONIA||Pablo.Hubbard@1z5fQdqQuDRJl8FopG.edu|2452501| +9303|AAAAAAAAHFECAAAA|1448401|7165|682|2452609|2452579|Mrs.|Sherry|Gonzales|Y|8|1|1945|KIRIBATI||Sherry.Gonzales@0gV6.com|2452635| +9304|AAAAAAAAIFECAAAA|1091468|1647|15379|2450465|2450435|Mr.|Michael|Smith|N|9|1|1925|FRENCH GUIANA||Michael.Smith@bxStDKoDfll.edu|2452373| +9305|AAAAAAAAJFECAAAA|618498|5369|25793|2450468|2450438|Mrs.|Gabriella|Slack|N|10|4|1930|ERITREA||Gabriella.Slack@kHgnTvKg3dl.edu|2452470| +9306|AAAAAAAAKFECAAAA|334434|5239|30091|2449029|2448999|Dr.|Landon|Lyon|Y|14|9|1977|BRUNEI DARUSSALAM||Landon.Lyon@4HAvZqb4ty.org|2452638| +9307|AAAAAAAALFECAAAA|1338504|4499|22894|2452104|2452074|Dr.|Jami|Lovell|N|3|5|1979|SIERRA LEONE||Jami.Lovell@98rmHG1nYaX6.org|2452556| +9308|AAAAAAAAMFECAAAA|741825|2847|18669|2449999|2449969|Ms.|Ida|Guess|N|16|3|1961|VIET NAM||Ida.Guess@qOo6yNjBOF4Aod6spy.com|2452426| +9309|AAAAAAAANFECAAAA|1410365|812|1612|2451700|2451670|Dr.|Tyler|Carter|Y|18|9|1968|PUERTO RICO||Tyler.Carter@5tY1a.com|2452634| +9310|AAAAAAAAOFECAAAA|574462|6867|27625|2452301|2452271|Dr.|Amy|Baker|Y|5|2|1939|AUSTRIA||Amy.Baker@OGKPR4dOx.com|2452382| +9311|AAAAAAAAPFECAAAA|1429282|3855|875|2450211|2450181|Sir|John|King|Y|21|12|1932|RUSSIAN FEDERATION||John.King@95f0l.com|2452395| +9312|AAAAAAAAAGECAAAA|717674|2539|12957|2449477|2449447|Sir|Brian|Anderson|Y|13|2|1987|LESOTHO||Brian.Anderson@Iqt6Q5IKuXr.edu|2452510| +9313|AAAAAAAABGECAAAA|212814|3485|37403|2449693|2449663|Dr.|Willian|Beaudoin|N|21|5|1943|ITALY||Willian.Beaudoin@JtggbFCKHAi.org|2452558| +9314|AAAAAAAACGECAAAA|1084701|4776|44851|2450687|2450657|Dr.|Jose|Harris|N|12|6|1990|SAINT HELENA||Jose.Harris@Gy5LfAmS0x0IZ.org|2452340| +9315|AAAAAAAADGECAAAA|1623123|3581|15286|2451389|2451359|Mr.|Thomas|Rinehart|N|21|10|1933|CHINA||Thomas.Rinehart@OiJ.com|2452461| +9316|AAAAAAAAEGECAAAA|774877|6885|27740|2452539|2452509|Ms.|Tessa|Ortiz|Y|27|6|1990|SYRIAN ARAB REPUBLIC||Tessa.Ortiz@ct57hi0IzNZ.com|2452304| +9317|AAAAAAAAFGECAAAA|814918|4601|19484|2450957|2450927|Mrs.|Katrina|Forster|Y|16|4|1957|GUERNSEY||Katrina.Forster@jTIBVkTy.org|2452348| +9318|AAAAAAAAGGECAAAA|930397|2663|36576|2450011|2449981|Miss|Virginia|Davenport|Y|15|4|1952|SERBIA||Virginia.Davenport@8g0ODZiguRkG.org|2452585| +9319|AAAAAAAAHGECAAAA|1170563|1877|9144|2449510|2449480|Sir|Valentine|Sterling|Y|12|5|1949|BRUNEI DARUSSALAM||Valentine.Sterling@LEroPO6.org|2452289| +9320|AAAAAAAAIGECAAAA|423468|5904|8693|2450784|2450754|Mrs.|Kristina|King|N|2|8|1977|IRELAND||Kristina.King@K6.edu|2452294| +9321|AAAAAAAAJGECAAAA|474486|4781|7476|2450004|2449974|Mrs.|Julianna|Burns|N|26|9|1975|LEBANON||Julianna.Burns@54neBc.com|2452584| +9322|AAAAAAAAKGECAAAA|1259566|3141|11916|2451766|2451736|Sir|Ronald|Smith|Y|25|9|1981|BERMUDA||Ronald.Smith@lHS5mFZeDRtdETxaZ5.com|2452636| +9323|AAAAAAAALGECAAAA|1912876|3212|6795|2451629|2451599|Dr.|Boris|Peck|N|15|10|1948|DJIBOUTI||Boris.Peck@3duA.edu|2452547| +9324|AAAAAAAAMGECAAAA|1231062|1061|17013|2451178|2451148||Scott|Jones|Y|26||1937||||| +9325|AAAAAAAANGECAAAA|1312817|7130|20196|2451285|2451255|Mrs.|Janice|Winter|Y|27|12|1965|MADAGASCAR||Janice.Winter@LhYX64Om76z7.edu|2452407| +9326|AAAAAAAAOGECAAAA|1849733|6283|38033|2451798|2451768|Mrs.|Anne|Seay|Y|7|9|1940|MONGOLIA||Anne.Seay@YKmCQFJKjELm6.com|2452397| +9327|AAAAAAAAPGECAAAA|1374853|3778|15950|2449176|2449146|Mrs.|Jennifer|Williams|Y|4|1|1941|ANTIGUA AND BARBUDA||Jennifer.Williams@yIvqN6vhVSt.edu|2452577| +9328|AAAAAAAAAHECAAAA|1531783|3218|33006|2450977|2450947|Ms.|Claire|Delgado|N|14|2|1928|BAHAMAS||Claire.Delgado@QynDbs0TfV.org|2452419| +9329|AAAAAAAABHECAAAA|1595610|4980|9103|2451985|2451955|Mr.|Noel|Anderson|N|26|11|1984|BENIN||Noel.Anderson@xzpG.com|2452515| +9330|AAAAAAAACHECAAAA|1377851|2904|41110|2450935|2450905|Mr.|Richard|Weston|N|7|9|1941|PARAGUAY||Richard.Weston@KAS3SsqighL3K.edu|2452563| +9331|AAAAAAAADHECAAAA|1764879|1237|37595|2449948|2449918|Dr.|Charles|Murray|Y|2|7|1982|EGYPT||Charles.Murray@UDjXFe41QFKQv.com|2452356| +9332|AAAAAAAAEHECAAAA|1763514|6559|10451|2452561|2452531|Mr.|Robert|Carlson|Y|26|6|1986|ALBANIA||Robert.Carlson@8Nnm3.edu|2452386| +9333|AAAAAAAAFHECAAAA|66003|4871|47739|2450181|2450151|Miss|Tanya|Chavez|N|3|7|1979|INDIA||Tanya.Chavez@eDa8p.org|2452634| +9334|AAAAAAAAGHECAAAA|757947|70|4993|2449992|2449962|Dr.|Michael|Wimberly|N|7|7|1940|SERBIA||Michael.Wimberly@ouys7DLDt7p4Dg.edu|2452559| +9335|AAAAAAAAHHECAAAA|1499919|3326|42780|2450542|2450512|Mr.|Leland|Barkley|N|30|5|1929|GRENADA||Leland.Barkley@8OSVI63SLiP41.com|2452458| +9336|AAAAAAAAIHECAAAA|698626|5848|26670|2452486|2452456|Mrs.|Roberta|Reed|N|9|8|1973|ESTONIA||Roberta.Reed@O2QuxgBKZk.com|2452374| +9337|AAAAAAAAJHECAAAA|742852|6033|34414|2450023|2449993|Dr.|Janice|Armstead|Y|27|1|1939|AUSTRALIA||Janice.Armstead@LnhyfSLxHBTI5.com|2452565| +9338|AAAAAAAAKHECAAAA||5708|41925||2450570|Dr.|Dennis|Hill|N|12|5|1985|GEORGIA|||| +9339|AAAAAAAALHECAAAA|1400786|2659|10932|2452441|2452411|Miss|Lakeshia|Lunsford|Y|3|2|1959|ANTARCTICA||Lakeshia.Lunsford@i.edu|2452514| +9340|AAAAAAAAMHECAAAA|104178|5294|2810|2450802|2450772|Dr.|Stephen|Rife|N|25|9|1955|GUAM||Stephen.Rife@Gsf5S6pa0xigSqTaoPz.org|2452611| +9341|AAAAAAAANHECAAAA|1564970|985|40200|2449800|2449770|Ms.|Ruby|Ramos|Y|2|1|1940|MONTSERRAT||Ruby.Ramos@zaOvr3oro739f.edu|2452563| +9342|AAAAAAAAOHECAAAA|315554|5581|29655|2451874|2451844|Miss|Pamela|Cox|Y|25|9|1928|SPAIN||Pamela.Cox@dFtraLLOilAQgZy.com|2452374| +9343|AAAAAAAAPHECAAAA|761254|3005|32709|2449031|2449001|Mr.|Bruce|Brito|Y|3|12|1976|NAMIBIA||Bruce.Brito@PKkhb.edu|2452573| +9344|AAAAAAAAAIECAAAA|29228|5198|34368|2450188|2450158|Mr.|John|Ainsworth|N|6|1|1963|SAINT HELENA||John.Ainsworth@N09mTot.edu|2452530| +9345|AAAAAAAABIECAAAA|||14764||2451432|Mrs.|Vanessa|||6||||||2452629| +9346|AAAAAAAACIECAAAA|1715910|6464|41818|2452581|2452551|Sir|Thomas|Parker|N|7|6|1939|BAHRAIN||Thomas.Parker@eOy816CZ6iKj.org|2452483| +9347|AAAAAAAADIECAAAA|876621|2834|30647|2449924|2449894|Sir|David|Marquez|N|14|6|1927|INDONESIA||David.Marquez@vbLs.com|2452315| +9348|AAAAAAAAEIECAAAA|192566|4844|1122|2450592|2450562|Dr.|Nicholas|Butler|N|11|2|1950|ARGENTINA||Nicholas.Butler@QyyLr589qC.edu|2452464| +9349|AAAAAAAAFIECAAAA|50178|4434|36243|2449823|2449793|Dr.|Robert|Hickman|Y|17|1|1978|MONTENEGRO||Robert.Hickman@0z1I0pnACd.edu|2452339| +9350|AAAAAAAAGIECAAAA|53540|554|29615|2449516|2449486|Sir|Christopher|Grover|Y|20|7|1968|MARSHALL ISLANDS||Christopher.Grover@Ce9GbFnJJKD2t.org|2452585| +9351|AAAAAAAAHIECAAAA|1409947|4893|5039|2449200|2449170|Miss|Adelle|Jenkins|Y|5|6|1976|SENEGAL||Adelle.Jenkins@8ZiQf.org|2452581| +9352|AAAAAAAAIIECAAAA|650229|3584|37339|2449107|2449077|Mr.|Randall|Hernandez|N|8|10|1938|MONTENEGRO||Randall.Hernandez@RNfMlO12ksCAgM.com|2452313| +9353|AAAAAAAAJIECAAAA|644162|5066|12370|2450556|2450526|Dr.|Barry|Oneal|N|25|1|1952|SOMALIA||Barry.Oneal@7zOohPb0.edu|2452623| +9354|AAAAAAAAKIECAAAA|1364402|2686|14970|2449705|2449675|Sir|Logan|King|Y|31|1|1959|ANGUILLA||Logan.King@ba3v3rOKFFoCVkU.org|2452390| +9355|AAAAAAAALIECAAAA|1710606|2610|24614|2449891|2449861|Miss|Georgia|Hall|N|26|2|1986|SAUDI ARABIA||Georgia.Hall@6XgQQiGLTyk.org|2452302| +9356|AAAAAAAAMIECAAAA|1337474|4890|37106|2452137|2452107|Mr.|Lorenzo|Willson|N|24|5|1976|WALLIS AND FUTUNA||Lorenzo.Willson@aR2rPUbJg2.edu|2452419| +9357|AAAAAAAANIECAAAA|1500422|4691|2560|2452206|2452176|Ms.|Temika|Thompson|Y|8|5|1940|SOLOMON ISLANDS||Temika.Thompson@H1Sc.com|2452289| +9358|AAAAAAAAOIECAAAA|1716051|3330|18162|2449879|2449849|Dr.|Gabriel|Jude|N|2|5|1957|HAITI||Gabriel.Jude@gD9V0HRU.org|2452591| +9359|AAAAAAAAPIECAAAA||5011|36852||2449897|||Ellis||23|2|1947|||Frank.Ellis@Fh5JGDbqJXpC.com|2452591| +9360|AAAAAAAAAJECAAAA|370690|3061|17385|2450125|2450095||Christopher||Y||3|1969|ALGERIA|||| +9361|AAAAAAAABJECAAAA|1405382|1854|29526|2449501|2449471|Sir|George|Slayton|N|24|1|1944|WESTERN SAHARA||George.Slayton@JXlvvjRo0rqV6cG8H6.com|2452546| +9362|AAAAAAAACJECAAAA|24790|1365|38693|2451207|2451177|Mr.|Dennis|Abrams|Y|25|9|1926|JAMAICA||Dennis.Abrams@6x7nFO8eYIr6BK.com|2452544| +9363|AAAAAAAADJECAAAA|680592|1886|29580|2452626|2452596|Miss|Aaron|Gibbons|Y|23|5|1964|DJIBOUTI||Aaron.Gibbons@uJJZ2qhDyCCr6t.org|2452588| +9364|AAAAAAAAEJECAAAA|129048|6416|27784|2451321|2451291|Mr.|Larry|Gray|Y|8|3|1927|NIUE||Larry.Gray@fXgoJQo9GfAOcZTPL.com|2452392| +9365|AAAAAAAAFJECAAAA|797558|1298|685|2449049|2449019|Dr.|Bessie|Ellison|N|26|7|1933|TURKMENISTAN||Bessie.Ellison@veblCn9Z0N5jejMzx0y.org|2452316| +9366|AAAAAAAAGJECAAAA|794484|2910|40889|2452196|2452166|Dr.|Stephanie|Falls|Y|28|1|1973|SWAZILAND||Stephanie.Falls@xMSkQ8GdKiUZtBcpqq.org|2452598| +9367|AAAAAAAAHJECAAAA|401325|4980|30988|2451696|2451666|Dr.|James|Daniel|N|29|7|1939|UNITED KINGDOM||James.Daniel@lVkh7DS00JrG.com|2452639| +9368|AAAAAAAAIJECAAAA|1090631|1182|14886|2449979|2449949|Sir|David|Owens|N|7|10|1936|BAHRAIN||David.Owens@AtCMU5jhYlfX7.edu|2452362| +9369|AAAAAAAAJJECAAAA|344508|336|26564|2451918|2451888|Mr.|Thomas|Olsen|Y|12|5|1937|TOGO||Thomas.Olsen@OSibXrYrC8FC7C.com|2452457| +9370|AAAAAAAAKJECAAAA|241416|1304|27592|2449103|2449073|Mrs.|Grace|Smith|Y|17|7|1956|POLAND||Grace.Smith@A1xbh5Irvbjdl.edu|2452354| +9371|AAAAAAAALJECAAAA|1874683|6301|1072|2451760|2451730|Mr.|Robert|Ward|N|5|9|1945|JORDAN||Robert.Ward@DGk50UysYzLHtZ.edu|2452399| +9372|AAAAAAAAMJECAAAA|371660|633|43253|2452439|2452409|Ms.|Albert|Martin|N|21|7|1938|FRENCH GUIANA||Albert.Martin@1Qz6Doziqh.com|2452532| +9373|AAAAAAAANJECAAAA|1091778|6035|21445|2452562|2452532|Mrs.|Pamela|Jones|N|16|7|1942|MADAGASCAR||Pamela.Jones@YjL7SCTkS10I.edu|2452435| +9374|AAAAAAAAOJECAAAA|1508263|6362|27261|2449605|2449575|Dr.|Lois|Martin|Y|21|4|1989|MONGOLIA||Lois.Martin@o151Ni7ibLvoV.com|2452498| +9375|AAAAAAAAPJECAAAA|674291|3793|43295|2452091|2452061|Dr.|Fabian|Wilson|N|20|9|1942|HAITI||Fabian.Wilson@51bM3BiRT.edu|2452568| +9376|AAAAAAAAAKECAAAA|571105|871|12237|2451292|2451262|Mrs.|Anita|Hernandez|Y|16|9|1969|SWEDEN||Anita.Hernandez@oz2iOThsbBPfq.org|2452558| +9377|AAAAAAAABKECAAAA|1391212|2417|24422|2450729|2450699|Miss|Barbara|Ibarra|Y|30|9|1947|TOGO||Barbara.Ibarra@BZg0z4u5C.com|2452375| +9378|AAAAAAAACKECAAAA|641019|3641|30710|2449655|2449625|Miss|Cindy|Snyder|N|4|3|1971|SURINAME||Cindy.Snyder@flv.edu|2452419| +9379|AAAAAAAADKECAAAA|1298670|625|38133|2450667|2450637|Mr.|Manuel|Edwards|N|18|8|1934|TUNISIA||Manuel.Edwards@aRjM8Z.org|2452370| +9380|AAAAAAAAEKECAAAA||2442|3976|2452149|2452119|||||||1960|SINGAPORE||Jodi.Hubert@7YsHXnyazidPmtLQGf.org|| +9381|AAAAAAAAFKECAAAA|1587652|1584|35163|2449098|2449068|Mrs.|Alycia|Knox|N|27|11|1959|SENEGAL||Alycia.Knox@r.edu|2452434| +9382|AAAAAAAAGKECAAAA|1607077|1906|21535|2450055|2450025|Mr.|Gregory|Ingram|Y|19|2|1924|MALAWI||Gregory.Ingram@P2XquiDR.org|2452447| +9383|AAAAAAAAHKECAAAA|216858|1827|37243|2451057|2451027|Dr.|Elliott|Hernandez|N|24|6|1966|SIERRA LEONE||Elliott.Hernandez@L2qHtxTXtDEAGr.edu|2452594| +9384|AAAAAAAAIKECAAAA|113456|3762|47668|2451649|2451619|Miss|Patti|Stanley|N|25|3|1928|KAZAKHSTAN||Patti.Stanley@k19c.edu|2452607| +9385|AAAAAAAAJKECAAAA|1252685|5263|36241|2451213|2451183|Mr.|Joseph|Purdy|N|5|11|1988|ANGUILLA||Joseph.Purdy@dzAIrHrChRGs.com|2452384| +9386|AAAAAAAAKKECAAAA|1727536|746|13603|2449376|2449346|Dr.|Danielle|Donahue|Y|22|4|1938|LUXEMBOURG||Danielle.Donahue@UzN6AKQe5hyyXD.edu|2452568| +9387|AAAAAAAALKECAAAA|732673|1959|21598|2451186|2451156|Mrs.|Julieta|Jenkins|Y|5|1|1982|SYRIAN ARAB REPUBLIC||Julieta.Jenkins@afgi2lqmxB9.org|2452302| +9388|AAAAAAAAMKECAAAA|214400|5717|37034|2449995|2449965|Miss|Ingrid|Mills|N|15|7|1961|CHINA||Ingrid.Mills@HZzjC.org|2452520| +9389|AAAAAAAANKECAAAA|1768526|2005|43884|2452592|2452562|Mrs.|Cindy|Tidwell|N|29|3|1978|UNITED STATES||Cindy.Tidwell@9IvjQ14eyQu.edu|2452344| +9390|AAAAAAAAOKECAAAA|1659688|5676|28017|2450354|2450324|Sir|Christopher|Adkins|N|27|11|1960|PALAU||Christopher.Adkins@0J1LeHlvZk.com|2452295| +9391|AAAAAAAAPKECAAAA|1493506|2202|1298|2451692|2451662|Sir|Christopher|Shaffer|N|24|5|1985|FINLAND||Christopher.Shaffer@EEZP2LGzsEjH.edu|2452484| +9392|AAAAAAAAALECAAAA|1028811|1711|22611|2452527|2452497|Mr.|Jim|Hill|Y|22|11|1992|SLOVENIA||Jim.Hill@kOb5ZxyvIssSSSZ.edu|2452479| +9393|AAAAAAAABLECAAAA|1638486|5341|32996|2451326|2451296|Mrs.|Nancy|Arthur|N|31|12|1936|SURINAME||Nancy.Arthur@6gJikOi5Aeu.edu|2452508| +9394|AAAAAAAACLECAAAA|554195|358|3768|2449225|2449195|Dr.|Kimberly|Morgan|N|27|10|1962|ITALY||Kimberly.Morgan@2REZ.edu|2452608| +9395|AAAAAAAADLECAAAA|488110|4513|41716|2451680|2451650|Mrs.|Melissa|Shrader|Y|21|12|1958|GUINEA-BISSAU||Melissa.Shrader@Gvk9gJrVBdo4gC9.com|2452411| +9396|AAAAAAAAELECAAAA|123217|1296|32222|2449046|2449016|Sir|Glenn|Robinson|N|6|10|1934|GIBRALTAR||Glenn.Robinson@8oblN39Sy0.edu|2452459| +9397|AAAAAAAAFLECAAAA|285346|6444|742|2451030|2451000|Mr.|James|Cross|Y|8|8|1932|ANDORRA||James.Cross@NkFX1sJILnDG637.org|2452338| +9398|AAAAAAAAGLECAAAA|1011018|84|45197|2450468|2450438|Mr.|Brandon|Butler|Y|13|7|1946|ETHIOPIA||Brandon.Butler@yt.edu|2452431| +9399|AAAAAAAAHLECAAAA|872927|6567|13760|2451083|2451053|Dr.|Adam|Curtis|N|9|1|1957|SENEGAL||Adam.Curtis@KoM6.edu|2452498| +9400|AAAAAAAAILECAAAA|1421683|3899|28515|2452255|2452225|Sir|Juan|Simon|Y|1|9|1981|MALI||Juan.Simon@zSyY.edu|2452568| +9401|AAAAAAAAJLECAAAA|579266|2598|7177|2449678|2449648|Mr.|Jeffrey|Langford|N|10|12|1981|AZERBAIJAN||Jeffrey.Langford@jleOZ4y.edu|2452558| +9402|AAAAAAAAKLECAAAA|343673|2326|27779|2452048|2452018|Mrs.|Alexander|Vazquez|Y|16|7|1944|DOMINICA||Alexander.Vazquez@o.com|2452567| +9403|AAAAAAAALLECAAAA|1792517|5561|47358|2451982|2451952|Dr.|Laticia|Whitehead|Y|9|3|1950|WESTERN SAHARA||Laticia.Whitehead@61Vc9xRbiC6EnGscmrq.com|2452588| +9404|AAAAAAAAMLECAAAA|307030|5075|2367|2451475|2451445|Miss|Caryn|Hamilton|N|20|2|1964|MARSHALL ISLANDS||Caryn.Hamilton@qcM.com|2452544| +9405|AAAAAAAANLECAAAA|63301|5962|5023|2449042|2449012|Mr.|Jerry|Russell|Y|19|7|1933|PERU||Jerry.Russell@erq915SrCF7.org|2452463| +9406|AAAAAAAAOLECAAAA|162473|1776|23432|2449451|2449421|Ms.|Betty|Carr|Y|11|4|1981|BELARUS||Betty.Carr@tipeaMlJ7Kv373D.com|2452425| +9407|AAAAAAAAPLECAAAA|1472718|2202|7333|2450775|2450745|Mr.|Richard|Smith|Y|8|7|1954|HONG KONG||Richard.Smith@ffDVu.com|2452284| +9408|AAAAAAAAAMECAAAA|||38200|2452326||Mrs.|Anthony|Cooper|||||SIERRA LEONE|||2452299| +9409|AAAAAAAABMECAAAA|91916|3677|19438|2452096|2452066|Mrs.|Dorthy|Ricks|Y|31|10|1942|MALDIVES||Dorthy.Ricks@admbXMPJYju.org|2452569| +9410|AAAAAAAACMECAAAA|1034737|3707|19924|2450519|2450489|Mrs.|Adam|Elmore|Y|8|7|1952|TAJIKISTAN||Adam.Elmore@9BLItYXpIox0R7.com|2452397| +9411|AAAAAAAADMECAAAA|642125|4560|48548|2452218|2452188|Mr.|Timothy|Adams|N|17|2|1939|AMERICAN SAMOA||Timothy.Adams@9EaPacyl7F5bq5.edu|2452473| +9412|AAAAAAAAEMECAAAA|177305|3937|8216|2449323|2449293|Dr.|Beth|Field|Y|2|11|1933|NAURU||Beth.Field@CgT97UQ.com|2452400| +9413|AAAAAAAAFMECAAAA|1609671|3155|22592|2451280|2451250|Dr.|John|Ferguson|Y|13|11|1942|SEYCHELLES||John.Ferguson@Jzp85cRauEiOUZxD.edu|2452424| +9414|AAAAAAAAGMECAAAA|937852|1954|13609|2450536|2450506|Dr.|Samantha|Lewis|Y|9|2|1949|MALAWI||Samantha.Lewis@2.com|2452608| +9415|AAAAAAAAHMECAAAA|675749|3882|32062|2451283|2451253|Mrs.|Yolanda|Sutton|Y|16|2|1986|CROATIA||Yolanda.Sutton@B3OcG9e61sdS.edu|2452330| +9416|AAAAAAAAIMECAAAA|371150|5416|32082|2449418|2449388|Dr.|Helen|Lee|N|14|7|1927|TURKMENISTAN||Helen.Lee@8AHkdLKf.org|2452556| +9417|AAAAAAAAJMECAAAA|1654426|2095|40517|2449204|2449174|Dr.|Raymond|Kuhn|Y|11|4|1980|AMERICAN SAMOA||Raymond.Kuhn@ZPCA4FvgK.org|2452310| +9418|AAAAAAAAKMECAAAA|1765321|1798|22628|2450061|2450031|Ms.|Diana|Ostrander|Y|14|6|1937|SURINAME||Diana.Ostrander@t9T5smAzo6Tso5rembG.com|2452298| +9419|AAAAAAAALMECAAAA|1012157|35|21513|2450940|2450910|Miss|Brenda|Bond|Y|17|6|1926|EL SALVADOR||Brenda.Bond@ITNMeZz8ZP9Ed.org|2452459| +9420|AAAAAAAAMMECAAAA|1790425|2797|41216|2450797|2450767|Miss|Ashlee|Barber|N|21|10|1970|KENYA||Ashlee.Barber@fQhIMlAkzcGKa.org|2452389| +9421|AAAAAAAANMECAAAA|1168761|6786|44011|2449249|2449219|Sir|Randy|Holmes|N|25|4|1933|QATAR||Randy.Holmes@xzTTTUs5ngz2.edu|2452380| +9422|AAAAAAAAOMECAAAA|207747|2493|49286|2449347|2449317|Sir|Kenneth|Morgan|N|16|6|1981|POLAND||Kenneth.Morgan@zgaIr12RlO1mzLGZ.edu|2452365| +9423|AAAAAAAAPMECAAAA|1209497|890|44636|2449435|2449405|Dr.|Gregory|Wilkins|N|25|4|1957|CAMBODIA||Gregory.Wilkins@SU3BGgE.edu|2452376| +9424|AAAAAAAAANECAAAA|76154|716|3278|2452388|2452358|Dr.|Susanne|Klinger|Y|12|5|1959|QATAR||Susanne.Klinger@7toRk35lhdB.com|2452576| +9425|AAAAAAAABNECAAAA|1818853|5990|41480|2452377|2452347|Dr.|Thomas|Hollingsworth|N|30|9|1960|AFGHANISTAN||Thomas.Hollingsworth@GM6CNHy.org|2452581| +9426|AAAAAAAACNECAAAA|1077038|6493|23148|2449306|2449276|Mr.|Edward|Wilson|Y|1|7|1976|TOGO||Edward.Wilson@GDRYsAekb9P.org|2452582| +9427|AAAAAAAADNECAAAA|490315|4958|26401|2449515|2449485|Mrs.|Anne|Jackson|N|19|5|1983|GUATEMALA||Anne.Jackson@EvnvoCz.com|2452428| +9428|AAAAAAAAENECAAAA|208123|4894|14865|2450529|2450499|Dr.|Robert|Odell|N|12|6|1973|GABON||Robert.Odell@HIHVD8z.org|2452587| +9429|AAAAAAAAFNECAAAA|941685|698|20111|2449683|2449653|Miss|Juliana|Robles|N|30|7|1964|LEBANON||Juliana.Robles@MXoITzAfgP.org|2452635| +9430|AAAAAAAAGNECAAAA|266972|5072|24725|2450414|2450384|Ms.|Kathleen|Jones|N|20|1|1945|SRI LANKA||Kathleen.Jones@1.edu|2452537| +9431|AAAAAAAAHNECAAAA|1360390|4272|47001|2449831|2449801|Miss|Crystal|Foster|N|5|11|1980|PALAU||Crystal.Foster@acLxCPE.com|2452449| +9432|AAAAAAAAINECAAAA|1642736|5813|3690|2452527|2452497|Sir|Samuel|Abrams|N|7|1|1942|BHUTAN||Samuel.Abrams@bDz20RFSjFji6IV.org|2452479| +9433|AAAAAAAAJNECAAAA|824598|775|29752|2450372|2450342|Mrs.|Catherine|Koontz|N|22|10|1951|BAHAMAS||Catherine.Koontz@fsXifcy9y3GoAQ.org|2452512| +9434|AAAAAAAAKNECAAAA|345658|1053|20495|2450091|2450061|Sir|Cody|Perry|Y|2|10|1931|GUAM||Cody.Perry@Cs3Kczd2Ti5.org|2452322| +9435|AAAAAAAALNECAAAA|1288523|2490|19657|2449692|2449662|Mr.|Ramon|Saddler|N|23|8|1945|PITCAIRN||Ramon.Saddler@S0Z.com|2452545| +9436|AAAAAAAAMNECAAAA||3455|6420|||Mrs.||Cohen|||||KYRGYZSTAN||Anthony.Cohen@sUmj1lL50J.edu|| +9437|AAAAAAAANNECAAAA|||11442|2449353|||William||Y|23|3|||||| +9438|AAAAAAAAONECAAAA|97065|5403|5341|2452528|2452498|Dr.|Derrick|Pleasant|Y|2|1|1977|CHINA||Derrick.Pleasant@rY6QMjoqaRun.edu|2452569| +9439|AAAAAAAAPNECAAAA|1554107|1629|11598|2450402|2450372|Dr.|Todd|Hartman|N|19|11|1985|THAILAND||Todd.Hartman@FlgCXkTVal6Pm.com|2452340| +9440|AAAAAAAAAOECAAAA|713440|6100|39960|2449794|2449764|Sir|Mark|Batts|N|17|10|1931|FRENCH POLYNESIA||Mark.Batts@Z3N4R0.org|2452391| +9441|AAAAAAAABOECAAAA|1902417|925|46633|2452063|2452033|Miss|Sally|Henry|Y|23|2|1975|ALGERIA||Sally.Henry@cAagEYRusgp.com|2452314| +9442|AAAAAAAACOECAAAA|964434|304|48974|2451846|2451816|Sir|Michael|Woodbury|Y|10|10|1940|FIJI||Michael.Woodbury@NvzNdR6Zf2eY1.org|2452522| +9443|AAAAAAAADOECAAAA|176601|3047|1493|2451711|2451681|Sir|Thomas|Park|Y|11|3|1987|RUSSIAN FEDERATION||Thomas.Park@oIqPKOGj.com|2452334| +9444|AAAAAAAAEOECAAAA|1221619|97|48575|2450637|2450607|Ms.|Helen|Pittman|Y|27|4|1929|HONDURAS||Helen.Pittman@jFEn6gR.org|2452365| +9445|AAAAAAAAFOECAAAA|||24546||2452309|Mr.||Lawrence|N|16|3|1954|KUWAIT||Drew.Lawrence@VQaUq.org|| +9446|AAAAAAAAGOECAAAA|1494919|5769|619|2452250|2452220|Ms.|Antonietta|Waters|Y|28|3|1946|FIJI||Antonietta.Waters@yT4uhesJXLCepdQe.org|2452439| +9447|AAAAAAAAHOECAAAA|1337920|3791|14062|2450800|2450770|Mrs.|Andrew|Simpson|Y|19|7|1962|ANGOLA||Andrew.Simpson@dznVAR.edu|2452295| +9448|AAAAAAAAIOECAAAA|275334|985|42683|2451239|2451209|Mrs.|Evelyn|Young|N|5|5|1979|TUNISIA||Evelyn.Young@V5T0Mm.edu|2452602| +9449|AAAAAAAAJOECAAAA|1373266|4264|28062|2451227|2451197|Dr.|Barry|Wheeler|Y|10|9|1966|COSTA RICA||Barry.Wheeler@3M6ZTPu.edu|2452570| +9450|AAAAAAAAKOECAAAA|340404|5845|40086|2452384|2452354|Sir|Steven|Vaughn|Y|27|7|1991|NETHERLANDS ANTILLES||Steven.Vaughn@22fkLlUdSVMiK4HrJ.edu|2452376| +9451|AAAAAAAALOECAAAA|1499391|5707|32823|2449369|2449339|Mrs.|Brenda|Simmons|N|20|10|1926|MOROCCO||Brenda.Simmons@k0RLFyVl23bdkAFmBc.org|2452418| +9452|AAAAAAAAMOECAAAA|1049584|1292|6200|2450860|2450830|Mr.|Brandon|Callaghan|Y|5|8|1952|TOGO||Brandon.Callaghan@APdn4rZHVcBN.com|2452546| +9453|AAAAAAAANOECAAAA|18870|346|25375|2452386|2452356|Sir|John|Jones|Y|11|11|1931|FRENCH GUIANA||John.Jones@8MJ.org|2452332| +9454|AAAAAAAAOOECAAAA|1912174|6959|31744|2451172|2451142|Mr.|Edmond|Scott|Y|23|3|1982|VANUATU||Edmond.Scott@6yR4y2KGB.org|2452495| +9455|AAAAAAAAPOECAAAA|1186530|447|31964|2452486|2452456|Dr.|Meredith|Tanner|Y|9|12|1942|PITCAIRN||Meredith.Tanner@G5K.edu|2452381| +9456|AAAAAAAAAPECAAAA|145529|4365|38448|2450777|2450747|Mr.|Raymond|Brown|Y|14|3|1960|ITALY||Raymond.Brown@MmPDZK3ZrA9lsNAf.org|2452615| +9457|AAAAAAAABPECAAAA|202645|4881|37870|2451599|2451569|Dr.|Robert|Pedroza|N|6|3|1961|TOGO||Robert.Pedroza@DP.com|2452443| +9458|AAAAAAAACPECAAAA|1773746|735|33277|2449747|2449717|Sir|Joseph|Mitchell|N|3|5|1943|CUBA||Joseph.Mitchell@7UVSlRL51RzSuzH.org|2452590| +9459|AAAAAAAADPECAAAA|1067594|2938|8329|2449723|2449693|Ms.|Molly|Berger|Y|29|12|1934|THAILAND||Molly.Berger@4beeHtbpFxU1Ad.com|2452303| +9460|AAAAAAAAEPECAAAA|524044|5419|39947|2450331|2450301|Ms.|Holly|Jones|Y|13|9|1953|LATVIA||Holly.Jones@SZYZuvKR8PdV.org|2452585| +9461|AAAAAAAAFPECAAAA|1170299|6535|6418|2451354|2451324|Mrs.|Lisa|Austin|N|23|12|1981|VENEZUELA||Lisa.Austin@qdZtBU84SGLyJR.edu|2452574| +9462|AAAAAAAAGPECAAAA|1332693|1335|42824|2451931|2451901|Mrs.|Janice|Chandler|Y|17|3|1955|CHRISTMAS ISLAND||Janice.Chandler@Df6I2B9Z.org|2452609| +9463|AAAAAAAAHPECAAAA|1201515|3346|2377|2451563|2451533|Sir|Reinaldo|Cohen|Y|29|4|1936|PARAGUAY||Reinaldo.Cohen@L0RAZmKq60G78.edu|2452295| +9464|AAAAAAAAIPECAAAA|642382|7198|61|2451826|2451796|Sir|Tommy|Yoo|N|10|1|1932|CAMBODIA||Tommy.Yoo@HNI5GnrN3FS26i.com|2452648| +9465|AAAAAAAAJPECAAAA|1756116|7021|16726|2449615|2449585|Mr.|Edward|Holland|Y|17|12|1951|LUXEMBOURG||Edward.Holland@V6a9CfrzNXT6s5Z.org|2452389| +9466|AAAAAAAAKPECAAAA|7822|1393|8964|2450114|2450084|Dr.|Catherine|Farr|N|25|10|1959|AFGHANISTAN||Catherine.Farr@kfVi6IGl.com|2452341| +9467|AAAAAAAALPECAAAA|1272377|607|16437|2450084|2450054|Mrs.|Odessa|White|Y|29|12|1933|BOUVET ISLAND||Odessa.White@0yGSkC1.org|2452537| +9468|AAAAAAAAMPECAAAA|1094463|1964|49465|2449812|2449782|Mr.|John|Jeffers|Y|17|3|1931|BULGARIA||John.Jeffers@5.edu|2452284| +9469|AAAAAAAANPECAAAA|1792738|2752|16719|2449393|2449363|Sir|Darrell|Archibald|N|1|2|1974|NORFOLK ISLAND||Darrell.Archibald@UmgdOX.org|2452394| +9470|AAAAAAAAOPECAAAA|1527493|2502|49111|2449423|2449393|Ms.|Annette|Knight|N|14|11|1960|AUSTRIA||Annette.Knight@J3V9qUZbN0NGRxq.com|2452544| +9471|AAAAAAAAPPECAAAA|99543|872|20201|2450931|2450901|Dr.|Richard|Dejesus|Y|22|11|1970|UNITED KINGDOM||Richard.Dejesus@s9ri.org|2452548| +9472|AAAAAAAAAAFCAAAA|1328779|1584|49942|2450585|2450555|Miss|Cynthia|Michaud|Y|23|3|1924|INDIA||Cynthia.Michaud@7k7fivnHl.edu|2452491| +9473|AAAAAAAABAFCAAAA|954775|3620|26465|2449697|2449667|Dr.|Grace|Medina|N|19|1|1990|MEXICO||Grace.Medina@81L.edu|2452578| +9474|AAAAAAAACAFCAAAA|614517|3396|31122|2450574|2450544|Mrs.|Elizabeth|Hayes|N|1|7|1939|GERMANY||Elizabeth.Hayes@COKdCbb7hYX4F.edu|2452536| +9475|AAAAAAAADAFCAAAA|532943|5071|27952|2452245|2452215|Mr.|Randy|Baker|Y|8|6|1964|IRAQ||Randy.Baker@dxpTTLuLP1QCm.org|2452373| +9476|AAAAAAAAEAFCAAAA|747412|1453|5541|2451903|2451873|Dr.|Richard|Quintero|N|7|1|1924|JAPAN||Richard.Quintero@szirzR5BpF.org|2452617| +9477|AAAAAAAAFAFCAAAA|514964|3021|5831|2450494|2450464|Sir|Edward|Lee|Y|6|6|1939|FAROE ISLANDS||Edward.Lee@uSV6MJjMaN.org|2452447| +9478|AAAAAAAAGAFCAAAA|1336863|4427|41944|2452262|2452232|Sir|Gregory|Landry|Y|18|9|1933|GHANA||Gregory.Landry@KAnLcOttKAk.com|2452398| +9479|AAAAAAAAHAFCAAAA|1278017|1185|45106|2449485|2449455|Mrs.|Irene|Wright|Y|23|12|1975|GREECE||Irene.Wright@6MeIiRHH5.edu|2452625| +9480|AAAAAAAAIAFCAAAA|417938|5084|11830|2452221|2452191|Mr.|Arnold|Patterson|Y|3|7|1947|R�UNION||Arnold.Patterson@0Ylp9VzT3kfdC2j.edu|2452296| +9481|AAAAAAAAJAFCAAAA|970623|2719|16462|2451493|2451463|Dr.|Randy|Harris|N|25|10|1959|ESTONIA||Randy.Harris@In0ITXC3YhLk.org|2452298| +9482|AAAAAAAAKAFCAAAA|133078|3334|11190|2451933|2451903|Sir|Richard|Jones|Y|4|4|1956|KUWAIT||Richard.Jones@9z4ehEcH.com|2452513| +9483|AAAAAAAALAFCAAAA|629349|3288|21274|2450378|2450348|Mr.|George|Lynch|N|9|2|1926|GRENADA||George.Lynch@Rh4DOHKUflHMNI.edu|2452383| +9484|AAAAAAAAMAFCAAAA|199590|6987|40384|2450250|2450220|Mr.|William|Musser|N|25|2|1973|HONDURAS||William.Musser@VDT85svNtrO.com|2452465| +9485|AAAAAAAANAFCAAAA|1767089|666|46643|2449031|2449001|Dr.|Michael|Schwartz|N|2|2|1949|YEMEN||Michael.Schwartz@SMjVb.edu|2452319| +9486|AAAAAAAAOAFCAAAA|1542873|2317|18961|2452592|2452562|Ms.|Janet|Figueroa|N|10|5|1950|MONTSERRAT||Janet.Figueroa@Zj8Cg.com|2452468| +9487|AAAAAAAAPAFCAAAA|496540|967|44947|2450059|2450029|Mr.|James|Cummings|Y|2|9|1947|VIRGIN ISLANDS, U.S.||James.Cummings@o.edu|2452506| +9488|AAAAAAAAABFCAAAA|1605349|5209|43916|2450691|2450661|Mr.|Stephen|Hudson|Y|6|10|1978|LIECHTENSTEIN||Stephen.Hudson@b0ekEUzrvIScBqN70.edu|2452450| +9489|AAAAAAAABBFCAAAA|1167574|6931|17488|2451087|2451057|Mrs.|Bonnie|Paradis|N|17|6|1989|NETHERLANDS||Bonnie.Paradis@8Td6kNxNmY7tQqv.edu|2452373| +9490|AAAAAAAACBFCAAAA|1619263|2251|16039|2449744|2449714|Dr.|Georgia|Perry|Y|6|2|1927|OMAN||Georgia.Perry@QYt5dvxyFED.edu|2452381| +9491|AAAAAAAADBFCAAAA|1187922|3544|47166|2451230|2451200|Dr.|Carla|Gibbs|N|10|4|1961|TUNISIA||Carla.Gibbs@G5hDgZHJ2F.edu|2452295| +9492|AAAAAAAAEBFCAAAA|1840078|2677|14429|2449690|2449660|Miss|Catherine|Hammer|Y|26|12|1930|KENYA||Catherine.Hammer@imIdVmsd33LXRHNQ.edu|2452388| +9493|AAAAAAAAFBFCAAAA|334700||27026|2450632||Dr.||Phelps|N||4|1931|||Ulysses.Phelps@FcTAV1GZNC0qBB.edu|| +9494|AAAAAAAAGBFCAAAA|||28848|||Mr.|Jarrod||Y||11|||||2452593| +9495|AAAAAAAAHBFCAAAA|1666822|1548|25474|2452213|2452183|Dr.|James|Wood|Y|18|6|1976|SUDAN||James.Wood@eS.com|2452548| +9496|AAAAAAAAIBFCAAAA|1052131|787|29890|2451018|2450988|Miss|Carolyn|Escobedo|N|17|1|1935|URUGUAY||Carolyn.Escobedo@DCp9uDvHf.org|2452540| +9497|AAAAAAAAJBFCAAAA|525156||24189||2449679|Dr.|||N|1||1942||||2452536| +9498|AAAAAAAAKBFCAAAA|849788|18|46648|2450696|2450666|Sir|Hayden|Franklin|Y|17|3|1934|PAPUA NEW GUINEA||Hayden.Franklin@Rmtcz2tRjQbplz.com|2452559| +9499|AAAAAAAALBFCAAAA|1910483|6209|22782|2450089|2450059|Sir|Frederick|Spivey|N|17|5|1978|ISLE OF MAN||Frederick.Spivey@84ZU6HDgb79E7Xofh.edu|2452378| +9500|AAAAAAAAMBFCAAAA|||11792||2449415|Miss|||||||COSTA RICA||Bernice.Lee@DRgMVbtUfJ.org|| +9501|AAAAAAAANBFCAAAA|1465110|2217|29724|2452205|2452175|Ms.|Serena|Mclean|N|3|1|1973|CAPE VERDE||Serena.Mclean@C9Vbe5nmaKgzDu.org|2452406| +9502|AAAAAAAAOBFCAAAA|542743|5140|16337|2449259|2449229|Mrs.|Abigail|Pappas|N|1|3|1992|HONDURAS||Abigail.Pappas@KFT.com|2452625| +9503|AAAAAAAAPBFCAAAA|1700708|6414|22167|2450825|2450795|Dr.|Robby|Keene|Y|7|1|1932|CYPRUS||Robby.Keene@pcdIyixor.com|2452555| +9504|AAAAAAAAACFCAAAA|479446|5704|43162|2450271|2450241|Miss|Kristen|Bowler|Y|20|8|1939|CUBA||Kristen.Bowler@KmppnFtutXcC.com|2452498| +9505|AAAAAAAABCFCAAAA|873784|3489|747|2451741|2451711|Miss|Sonia|Thompson|Y|13|4|1933|GIBRALTAR||Sonia.Thompson@5G1irPxr45s2P.edu|2452475| +9506|AAAAAAAACCFCAAAA|1574497|1540|19736|2449485|2449455|Ms.|Lora|Templeton|Y|4|4|1972|TURKMENISTAN||Lora.Templeton@2L8Eun0.edu|2452470| +9507|AAAAAAAADCFCAAAA|1491691|5366|4194|2451062|2451032|Ms.|Dorothy|Tate|Y|30|6|1924|PERU||Dorothy.Tate@8Ms6a2I5ndcfjB.org|2452387| +9508|AAAAAAAAECFCAAAA|722309|2043|18977|2450689|2450659|Dr.|Steven|Jackson|N|21|1|1947|KIRIBATI||Steven.Jackson@koppCZ4guNRGx.com|2452384| +9509|AAAAAAAAFCFCAAAA|1078098|6016|6687|2452381|2452351|Sir|Lindsey|Thomas|N|13|6|1987|TOGO||Lindsey.Thomas@KANX9h.edu|2452292| +9510|AAAAAAAAGCFCAAAA|1238170|6515|1307|2451035|2451005|Dr.|Angela|Jones|Y|3|9|1969|SIERRA LEONE||Angela.Jones@F.com|2452411| +9511|AAAAAAAAHCFCAAAA|462094|7051|18855|2452399|2452369|Dr.|Austin|Harmon|Y|18|8|1976|VIRGIN ISLANDS, U.S.||Austin.Harmon@7JI.org|2452498| +9512|AAAAAAAAICFCAAAA|711403|5706|5209|2451420|2451390|Dr.|Michael|Oliver|Y|4|10|1978|SWAZILAND||Michael.Oliver@LdGYgimpZBMt.com|2452593| +9513|AAAAAAAAJCFCAAAA||6361|7449|||||Rust|Y|8|11||||James.Rust@EoHsbVdiOIxcex.org|2452425| +9514|AAAAAAAAKCFCAAAA|1319597|4998|45952||2449237||Marcel|||23||1988||||| +9515|AAAAAAAALCFCAAAA|1343734|4083|47632|2449576|2449546|Dr.|Pamela|Hoff|Y|26|6|1958|LESOTHO||Pamela.Hoff@r67VjUyJNTo69sSNu.com|2452491| +9516|AAAAAAAAMCFCAAAA|924840|5215|15190|2452388|2452358|Dr.|Joshua|Foreman|Y|6|4|1957|RWANDA||Joshua.Foreman@A8VyZz.com|2452360| +9517|AAAAAAAANCFCAAAA|475460|180|30900|2450329|2450299|Mr.|Tom|Clarke|N|10|1|1982|MEXICO||Tom.Clarke@LADhOkei7TpN.com|2452430| +9518|AAAAAAAAOCFCAAAA|142973|4284|46442|2450000|2449970|Mr.|Charles|Rea|N|29|12|1928|TOGO||Charles.Rea@IzVc20Op4q.com|2452583| +9519|AAAAAAAAPCFCAAAA|1590470|1790|39110|2452411|2452381|Dr.|Tina|Wells|N|2|6|1955|BANGLADESH||Tina.Wells@SMVHknBekU.com|2452441| +9520|AAAAAAAAADFCAAAA|47602|1514|45779|2451343|2451313|Mr.|David|Bennett|Y|6|9|1974|C�TE D'IVOIRE||David.Bennett@xvz9bip.edu|2452334| +9521|AAAAAAAABDFCAAAA|463|299|19735|2451596|2451566|Miss|Katrina|Ryan|N|7|12|1974|HUNGARY||Katrina.Ryan@mvNeIlvc9A.org|2452541| +9522|AAAAAAAACDFCAAAA|123348|5781|29014|2449501|2449471|Mr.|Frank|Harper|Y|3|10|1964|EQUATORIAL GUINEA||Frank.Harper@BCEj8aHlhd8.org|2452436| +9523|AAAAAAAADDFCAAAA|508641|1072|13550|2451354|2451324|Sir|Donald|Davis|N|15|12|1991|MONTSERRAT||Donald.Davis@SdDoKv.com|2452294| +9524|AAAAAAAAEDFCAAAA|357277|1881|25685|2450263|2450233|Dr.|Johnathan|Sanchez|N|29|2|1960|LESOTHO||Johnathan.Sanchez@s1OTE1co8M.org|2452621| +9525|AAAAAAAAFDFCAAAA|759257|4568|35879|2450145|2450115|Ms.|Arthur|Mcculloch|Y|22|5|1979|CAMEROON||Arthur.Mcculloch@dEyUtKBk.org|2452310| +9526|AAAAAAAAGDFCAAAA|613810|5926|12042|2451406|2451376|Mr.|Sean|Jackson|N|3|4|1970|PAPUA NEW GUINEA||Sean.Jackson@hbPugvu8nENf.edu|2452456| +9527|AAAAAAAAHDFCAAAA|1474876|6612|25901|2452485|2452455|Dr.|Richard|Santos|N|20|4|1949|MALAYSIA||Richard.Santos@RQ3tAiy.org|2452430| +9528|AAAAAAAAIDFCAAAA|828709|2000|12491|2451435|2451405|Mrs.|Judy|Wilson|N|19|6|1931|MACAO||Judy.Wilson@J5lvOVYlnVqGFJxD.edu|2452531| +9529|AAAAAAAAJDFCAAAA|359999|6627|48902|2450804|2450774|Sir|Nestor|Butts|Y|9|9|1992|PANAMA||Nestor.Butts@U9HYa8UaQTrCi79H0V.com|2452557| +9530|AAAAAAAAKDFCAAAA|261342|3397|27899|2450800|2450770|Dr.|Bradford|Smith|Y|13|3|1950|ISLE OF MAN||Bradford.Smith@RnNExxPef.com|2452614| +9531|AAAAAAAALDFCAAAA|1172972|5251|19977|2451648|2451618|Ms.|Contessa|Fisher|N|9|10|1943|AZERBAIJAN||Contessa.Fisher@3zMxN4MchCYCF9g.edu|2452488| +9532|AAAAAAAAMDFCAAAA|515496|1506|2716|2450797|2450767|Sir|Bobby|Weinberg|Y|18|11|1985|KENYA||Bobby.Weinberg@QZB8Dj1t5.edu|2452534| +9533|AAAAAAAANDFCAAAA|158725|1218|34864|2449500|2449470|Sir|Phillip|Flores|N|14|8|1982|PITCAIRN||Phillip.Flores@lHJJ7nEqBzP.edu|2452601| +9534|AAAAAAAAODFCAAAA|410932|735|38091|2450801|2450771|Dr.|Jeffery|Bird|N|8|12|1967|CROATIA||Jeffery.Bird@yytNO.edu|2452322| +9535|AAAAAAAAPDFCAAAA|1512082|1124|20566|2450958|2450928|Sir|Raymond|Knutson|Y|21|10|1968|CAMBODIA||Raymond.Knutson@jg1cn.edu|2452400| +9536|AAAAAAAAAEFCAAAA|704248|1413|18225|2450881|2450851|Sir|Columbus|Holt|N|29|9|1962|UNITED KINGDOM||Columbus.Holt@gFjgD79RD.edu|2452468| +9537|AAAAAAAABEFCAAAA|1595902|295|2288|2450209|2450179|Ms.|Nellie|Thornhill|Y|30|4|1945|BRAZIL||Nellie.Thornhill@0PjHkycMeJPZ0.org|2452415| +9538|AAAAAAAACEFCAAAA|599852|1141|1230|2451850|2451820|Dr.|Jerry|Snyder|N|4|8|1969|CHILE||Jerry.Snyder@hVJecL3VeBP.org|2452551| +9539|AAAAAAAADEFCAAAA|175891|1387|33283|2449397|2449367|Mr.|Manuel|Barrett|N|12|2|1928|MAURITANIA||Manuel.Barrett@C9vQ9KdXlu.org|2452420| +9540|AAAAAAAAEEFCAAAA|1856839|923|12560|2450183|2450153|Ms.|Mattie|Douglas|Y|8|7|1928|MOLDOVA, REPUBLIC OF||Mattie.Douglas@Ysj1enYvGSRcSq3E.edu|2452527| +9541|AAAAAAAAFEFCAAAA|408179|5111|39996|2451019|2450989|Ms.|Ann|Rogers|N|25|3|1944|NETHERLANDS ANTILLES||Ann.Rogers@A.org|2452396| +9542|AAAAAAAAGEFCAAAA|13030|2376|33369|2449683|2449653|Ms.|Sachiko|Johnson|Y|18|10|1982|SAINT LUCIA||Sachiko.Johnson@HNS5sM4Qg8MXcVKqXE.org|2452287| +9543|AAAAAAAAHEFCAAAA|210365|2389|18813|2449204|2449174|Dr.|Edith|Hare|Y|27|3|1985|UGANDA||Edith.Hare@GfNi4kNmdA9.edu|2452628| +9544|AAAAAAAAIEFCAAAA|900887|4305|11723|2449275|2449245|Mrs.|Margie|Farris|Y|17|8|1960|CAMBODIA||Margie.Farris@S5ooVf.edu|2452400| +9545|AAAAAAAAJEFCAAAA|952040|3386|10779|2450721|2450691|Sir|William|Silver|N|25|1|1976|AFGHANISTAN||William.Silver@lUDVTGilCvUY.org|2452616| +9546|AAAAAAAAKEFCAAAA|531993|1972|25551|2452640|2452610|Mrs.|Albert|Swanson|Y|13|6|1988|ARUBA||Albert.Swanson@joVO2KQJMueKNJ.edu|2452380| +9547|AAAAAAAALEFCAAAA||6863|35677||||Charles||Y||||IRELAND|||| +9548|AAAAAAAAMEFCAAAA|1686413|6707|17401|2451435|2451405|Dr.|Vince|Williamson|Y|6|2|1940|BULGARIA||Vince.Williamson@9z9d6EO.com|2452633| +9549|AAAAAAAANEFCAAAA|1057389|4001|566|2449858|2449828|Mr.|Tom|Mccormick|Y|12|6|1975|PERU||Tom.Mccormick@c6RRAt1cGCHybDt.com|2452386| +9550|AAAAAAAAOEFCAAAA|259644|348|37052|2451069|2451039|Dr.|Kathryn|Torres|Y|18|5|1986|THAILAND||Kathryn.Torres@rV.com|2452366| +9551|AAAAAAAAPEFCAAAA|1058193|3745|24944|2449250|2449220|Sir|Shaun|Cox|Y|1|12|1937|IRAQ||Shaun.Cox@dDJ1.com|2452601| +9552|AAAAAAAAAFFCAAAA|1774447|5507|31367|2449069|2449039|Mr.|George|Brown|Y|15|12|1960|ANTARCTICA||George.Brown@S8GY0C0mSl.org|2452597| +9553|AAAAAAAABFFCAAAA|393436|6673|2310|2449124|2449094|Miss|Sadie|King|Y|26|7|1970|PANAMA||Sadie.King@hx1sFoNJP7HYJb.edu|2452483| +9554|AAAAAAAACFFCAAAA|1855196|3347|18698|2450957|2450927|Miss|Agnes|Gibson|N|17|7|1987|JAMAICA||Agnes.Gibson@KBqPpUba4HFx.edu|2452399| +9555|AAAAAAAADFFCAAAA|1614777|3670|28210|2450034|2450004|Sir|Benito|Fellows|Y|25|5|1930|TONGA||Benito.Fellows@G74vDyzbbthx.org|2452322| +9556|AAAAAAAAEFFCAAAA|195957|2460|42116|2450351|2450321|Ms.|Alexandra|Wiggins|Y|4|4|1972|BERMUDA||Alexandra.Wiggins@57vZnR6nkm8sq.edu|2452382| +9557|AAAAAAAAFFFCAAAA|1236213|1403|7393|2451198|2451168|Mrs.|Ericka|Parry|N|26|1|1955|ISRAEL||Ericka.Parry@fm7NXk7QEdhPZ.com|2452488| +9558|AAAAAAAAGFFCAAAA|947767|3746|20478|2451498|2451468|Sir|Preston|Skinner|N|4|8|1985|KYRGYZSTAN||Preston.Skinner@o.com|2452484| +9559|AAAAAAAAHFFCAAAA|1382097|3642|13540|2450564|2450534|Mrs.|Ruth|Johnson|N|29|11|1977|AUSTRALIA||Ruth.Johnson@CsqDrZdo8v.edu|2452287| +9560|AAAAAAAAIFFCAAAA|1706706|651|39796|2451997|2451967|Sir|Christopher|Burr|Y|1|10|1927|VIET NAM||Christopher.Burr@lmn7OnpA.org|2452624| +9561|AAAAAAAAJFFCAAAA|757480|2493|21459|2452083|2452053|Mr.|Jose|Wilkins|N|3|2|1944|UGANDA||Jose.Wilkins@QxTSevOrQDL5.com|2452516| +9562|AAAAAAAAKFFCAAAA|993267|5942|49464|2450719|2450689|Mrs.|Jennifer|Rhodes|Y|7|4|1942|SEYCHELLES||Jennifer.Rhodes@IZQJUZNBB5Kj2.org|2452406| +9563|AAAAAAAALFFCAAAA|1599619|3989|17912|2449288|2449258|Mrs.|Kathy|Davis|N|13|7|1975|ICELAND||Kathy.Davis@b.com|2452398| +9564|AAAAAAAAMFFCAAAA|1015265|3870|33628|2451381|2451351|Dr.|Kevin|Walling|Y|12|4|1938|MOLDOVA, REPUBLIC OF||Kevin.Walling@i4IIO0EpPO0.com|2452561| +9565|AAAAAAAANFFCAAAA|467783|5879|47928|2451236|2451206|Sir|Tomas|Meyers|Y|29|12|1940|BURKINA FASO||Tomas.Meyers@Jy1rxry7HXdpDXC4.com|2452392| +9566|AAAAAAAAOFFCAAAA|980334|4293|13052|2450186|2450156|Ms.|Wanda|Snow|Y|12|8|1982|AZERBAIJAN||Wanda.Snow@pNq63fDsNko.org|2452287| +9567|AAAAAAAAPFFCAAAA|1765506|4857|15808|2449601|2449571|Miss|Aaron|Collins|N|6|4|1943|ALBANIA||Aaron.Collins@aq1EtiRG0YhreaD.com|2452639| +9568|AAAAAAAAAGFCAAAA|484843|5982|28674|2451944|2451914|Mr.|Eric|Spangler|Y|17|11|1974|CAPE VERDE||Eric.Spangler@1eyHVbieedlO8Pi.com|2452538| +9569|AAAAAAAABGFCAAAA|1889113|946|7179|2450681|2450651|Dr.|Rachel|Berry|N|23|5|1951|QATAR||Rachel.Berry@g1XDNzx4.com|2452558| +9570|AAAAAAAACGFCAAAA|1050873|85|11697|2449502|2449472|Mr.|Edward|Allen|N|25|1|1928|LITHUANIA||Edward.Allen@LAxTxahjT.org|2452442| +9571|AAAAAAAADGFCAAAA|1379137|2280|35470|2451844|2451814|Dr.|Beverly|Langer|Y|9|6|1934|PANAMA||Beverly.Langer@KLZNvClprG27x.com|2452449| +9572|AAAAAAAAEGFCAAAA|1013401|5931|28061|2452153|2452123|Mr.|Jeremy|Phan|Y|31|12|1942|MAURITANIA||Jeremy.Phan@kqtuQSBoLFG.org|2452448| +9573|AAAAAAAAFGFCAAAA|954212|5352|10492|2451811|2451781|Miss|Doris|Denney|N|26|11|1992|VIRGIN ISLANDS, U.S.||Doris.Denney@OzxlxN.com|2452465| +9574|AAAAAAAAGGFCAAAA|455293|3377|31461|2449867|2449837|Dr.|James|Grace|Y|4|6|1928|GEORGIA||James.Grace@Tz.edu|2452594| +9575|AAAAAAAAHGFCAAAA|339970|2015|10860|2450835|2450805|Mrs.|Crystal|Warren|Y|30|3|1925|OMAN||Crystal.Warren@smP0ZFYUo7y46.com|2452285| +9576|AAAAAAAAIGFCAAAA|1569383|1696|32409|2450849|2450819|Dr.|Terrence|Murray|N|1|3|1943|SLOVAKIA||Terrence.Murray@BJ2pyd.org|2452599| +9577|AAAAAAAAJGFCAAAA|973403|6799|18995|2451794|2451764|Sir|Tony|Bartlett|N|17|9|1978|SWEDEN||Tony.Bartlett@69FlyKb2KxLjlcd.com|2452487| +9578|AAAAAAAAKGFCAAAA|1279686|2864|37595|2449164|2449134|Sir|John|Pereira|Y|18|12|1980|PITCAIRN||John.Pereira@6ICXpffJYEX7y.org|2452420| +9579|AAAAAAAALGFCAAAA|1290817|5938|18212|2450496|2450466|Sir|Brett|Holden|N|13|1|1959|ANTIGUA AND BARBUDA||Brett.Holden@XcxLy2U2X4nOS.edu|2452622| +9580|AAAAAAAAMGFCAAAA|552317|4898|47479|2449082|2449052|Ms.|Jennifer|Nall|N|17|6|1969|ARGENTINA||Jennifer.Nall@r8HTPlzDlAsJHqFo.edu|2452401| +9581|AAAAAAAANGFCAAAA|1634173|2657|47094|2450821|2450791|Mr.|William|Hurd|N|19|3|1965|CAMBODIA||William.Hurd@Z7SzbEtvomzFmI5.org|2452429| +9582|AAAAAAAAOGFCAAAA|258270|5103|29536|2452180|2452150|Mr.|Melvin|Chin|Y|26|4|1956|BELIZE||Melvin.Chin@N5Bv4B0T4i.edu|2452325| +9583|AAAAAAAAPGFCAAAA|1279201|337|42273|2452206|2452176|Mrs.|Sylvia|Wolfe|Y|26|8|1933|PERU||Sylvia.Wolfe@ljzkF9xuABVaB9KM.edu|2452399| +9584|AAAAAAAAAHFCAAAA|954082|3683|10084|2452214|2452184|Dr.|Lorena|Aguilar|N|3|5|1950|SAINT HELENA||Lorena.Aguilar@v2E.edu|2452544| +9585|AAAAAAAABHFCAAAA|598811|4922|43833|2452320|2452290|Dr.|Eric|Cooper|Y|25|9|1932|SAMOA||Eric.Cooper@nl9L1iSXd6LyQQb.com|2452482| +9586|AAAAAAAACHFCAAAA|4750|1601|38899|2451272|2451242|Miss|Dorothy|Garland|Y|30|7|1959|ZIMBABWE||Dorothy.Garland@GpJx.org|2452606| +9587|AAAAAAAADHFCAAAA|1405151|6066|33828|2451071|2451041|Ms.|Alexander|Goldstein|Y|13|10|1977|CANADA||Alexander.Goldstein@8eQJJ4dgcpHt.com|2452611| +9588|AAAAAAAAEHFCAAAA|659835|1966|37344|2450710|2450680|Dr.|Luke|Sorenson|N|16|2|1962|AZERBAIJAN||Luke.Sorenson@ptXGjPXjLpbNArN6hL.com|2452341| +9589|AAAAAAAAFHFCAAAA|159454|4711|41521|2450575|2450545|Dr.|Larry|Avila|Y|3|12|1967|TURKEY||Larry.Avila@FlKiG94S0t4R.com|2452564| +9590|AAAAAAAAGHFCAAAA|1680442|29|416|2449604|2449574|Mrs.|Corrine|Everson|N|22|5|1964|BELARUS||Corrine.Everson@RvvAeF1R.edu|2452634| +9591|AAAAAAAAHHFCAAAA|975587|4142|21944|2452154|2452124|Dr.|Alvin|Wertz|Y|16|4|1988|CROATIA||Alvin.Wertz@AjKIg1e5vAI.edu|2452648| +9592|AAAAAAAAIHFCAAAA|1097949|373|47597|2450538|2450508|Dr.|Lael|Swan|N|20|6|1950|HONDURAS||Lael.Swan@EIb.org|2452369| +9593|AAAAAAAAJHFCAAAA|614627|926|25625|2452458|2452428|Mrs.|Shantel|Bryant|Y|5|4|1983|MARSHALL ISLANDS||Shantel.Bryant@cTI4RqGzjA.com|2452472| +9594|AAAAAAAAKHFCAAAA|471549|688|44274|2450743|2450713|Ms.|Lydia|Brewer|N|25|3|1966|BANGLADESH||Lydia.Brewer@CvhOnJUte7qZtB7T.org|2452365| +9595|AAAAAAAALHFCAAAA|450391|6240|42842|2450121|2450091|Dr.|Billy|Sheldon|Y|13|9|1952|PUERTO RICO||Billy.Sheldon@K44XDQ4OdK5H.org|2452444| +9596|AAAAAAAAMHFCAAAA|1845049|2749|13249|2451096|2451066|Mrs.|Angla|Nash|Y|16|3|1924|NIGER||Angla.Nash@XsUD3LTp8L.com|2452361| +9597|AAAAAAAANHFCAAAA|1247797|6547|33297|2451748|2451718|Ms.|Lois|Silva|Y|16|7|1937|NIGER||Lois.Silva@8xfTH0.edu|2452304| +9598|AAAAAAAAOHFCAAAA|1654947|6508|30069|2449337|2449307|Dr.|Geraldine|Childers|N|25|7|1931|ANTIGUA AND BARBUDA||Geraldine.Childers@SY0gfQq.org|2452396| +9599|AAAAAAAAPHFCAAAA|926791|4938|21885|2451256|2451226|Sir|Nick|Watkins|Y|28|9|1956|JAMAICA||Nick.Watkins@xMk3OKoqIYSmoo.edu|2452427| +9600|AAAAAAAAAIFCAAAA|1805995|5418|35994|2452127|2452097|Dr.|Sandy|Sykes|N|5|9|1966|JERSEY||Sandy.Sykes@hDK8ZlayI1.edu|2452499| +9601|AAAAAAAABIFCAAAA|1360725|5663|21401|2452116|2452086|Mr.|Derek|Clifford|N|26|1|1988|OMAN||Derek.Clifford@7RMmsA49sICvEJ08.edu|2452473| +9602|AAAAAAAACIFCAAAA|1147120|5025|8930|2451171|2451141|Dr.|Samuel|Brown|Y|9|4|1976|VIRGIN ISLANDS, U.S.||Samuel.Brown@cXgTtjOHcQpqvT7S1C.edu|2452469| +9603|AAAAAAAADIFCAAAA|580661|6935|16409|2449330|2449300|Dr.|Charles|Chaney|Y|4|6|1944|TONGA||Charles.Chaney@tF.com|2452503| +9604|AAAAAAAAEIFCAAAA|1861598|5449|24947|2450588|2450558|Sir|Marcus|King|N|21|12|1930|TOGO||Marcus.King@dFtt9QZPm89.org|2452417| +9605|AAAAAAAAFIFCAAAA|149298|6172|27799|2450943|2450913|Ms.|Adam|Walls|Y|21|6|1974|BOLIVIA||Adam.Walls@vt5vTIXXJfAc6vOc.org|2452477| +9606|AAAAAAAAGIFCAAAA|1147596|1638|38927|2449218|2449188|Mr.|John|Beard|N|15|3|1925|CZECH REPUBLIC||John.Beard@qRoyc.edu|2452443| +9607|AAAAAAAAHIFCAAAA|1833149|1018|31334|2449074|2449044|Mr.|Miguel|Bradshaw|Y|15|1|1970|CANADA||Miguel.Bradshaw@6tx1N.org|2452559| +9608|AAAAAAAAIIFCAAAA|1047797|5551|4458|2450017|2449987|Dr.|Stephanie|Roe|Y|20|12|1939|AMERICAN SAMOA||Stephanie.Roe@gojZ6tZd6G9cj.com|2452620| +9609|AAAAAAAAJIFCAAAA|1783529|1047|21718|2450301|2450271|Mr.|Jonathan|Arthur|Y|10|1|1951|BELIZE||Jonathan.Arthur@caC98N.com|2452397| +9610|AAAAAAAAKIFCAAAA|89450|3032|214|2450382|2450352|Dr.|Amy|Garcia|N|4|11|1931|TUVALU||Amy.Garcia@naXeFf.edu|2452524| +9611|AAAAAAAALIFCAAAA|664015|4782|2749|2451109|2451079|Mrs.|Dora|Rhodes|N|17|7|1966|MARSHALL ISLANDS||Dora.Rhodes@timbaX2vZ70rH5yP.org|2452488| +9612|AAAAAAAAMIFCAAAA|1657463|2710|18710|2452134|2452104|Dr.|Leonard|Bryan|Y|13|9|1968|GUATEMALA||Leonard.Bryan@ImU0rOB9ACO1ziOR.com|2452534| +9613|AAAAAAAANIFCAAAA|748999|6670|29494|2449838|2449808|Mr.|Earl|Davis|N|25|5|1959|RWANDA||Earl.Davis@79xYkBxY5kiP4.edu|2452478| +9614|AAAAAAAAOIFCAAAA|261429|1667|23505|2450390|2450360|Sir|Peter|Sneed|N|17|8|1945|GEORGIA||Peter.Sneed@qPEvxTEBM.edu|2452557| +9615|AAAAAAAAPIFCAAAA|1376688|2871|16301|2449652|2449622|Dr.|John|Kramer|N|22|6|1940|LIBERIA||John.Kramer@TRbpsGgEvPjXHx.edu|2452328| +9616|AAAAAAAAAJFCAAAA|543872|4432|44192|2450678|2450648|Mr.|Jason|Fender|N|6|4|1953|POLAND||Jason.Fender@2P.com|2452504| +9617|AAAAAAAABJFCAAAA|1391054||14262|||||Coon|||3||BELIZE||Adrian.Coon@8ITUGajiMrTm0G3.org|| +9618|AAAAAAAACJFCAAAA|481227|6166|48258|2449506|2449476|Sir|Roberto|Elliott|N|17|3|1956|VENEZUELA||Roberto.Elliott@1M1Qczcl90EcEPiMqFR.com|2452394| +9619|AAAAAAAADJFCAAAA|1832514|6725|25336|2451795|2451765|Dr.|George|Runyon|Y|23|5|1930|LESOTHO||George.Runyon@NqYH.edu|2452285| +9620|AAAAAAAAEJFCAAAA|22895|355|18968|2451320|2451290|Ms.|Cecila|Fischer|N|10|9|1971|UNITED ARAB EMIRATES||Cecila.Fischer@OIU3q.edu|2452460| +9621|AAAAAAAAFJFCAAAA|1449359|1790|33421|2452191|2452161|Mr.|Darrel|Lawson|Y|25|12|1938|MALAYSIA||Darrel.Lawson@3.edu|2452427| +9622|AAAAAAAAGJFCAAAA|1328758|828|10098|2449609|2449579|Sir|Ken|Flores|Y|31|10|1974|GREECE||Ken.Flores@z0Ck5GvjYp1.edu|2452318| +9623|AAAAAAAAHJFCAAAA|414359|132|47505|2450161|2450131|Mr.|Vernon|Little|N|2|6|1986|KAZAKHSTAN||Vernon.Little@Jq0maaR.org|2452584| +9624|AAAAAAAAIJFCAAAA|1894912|4852|20392|2450093|2450063|Mr.|Conrad|Miller|N|9|12|1969|NEW ZEALAND||Conrad.Miller@QenHIYe8u9faR.edu|2452620| +9625|AAAAAAAAJJFCAAAA|1346020|7096|49372|2451526|2451496|Dr.|Angelina|Stpierre|Y|22|1|1943|HAITI||Angelina.Stpierre@ogF.edu|2452552| +9626|AAAAAAAAKJFCAAAA|249813|963|32960|2452639|2452609|Sir|James|Tate|N|12|5|1944|SWITZERLAND||James.Tate@N74KZXM1PnAzgL.com|2452462| +9627|AAAAAAAALJFCAAAA|||35681|||Dr.|Fernando||Y||4|1982|||Fernando.Washington@AihC2J.com|2452642| +9628|AAAAAAAAMJFCAAAA|606557|2507|34410|2452235|2452205|Ms.|Jacqueline|Haight|N|7|2|1931|BAHRAIN||Jacqueline.Haight@OcQei06c2KThZe.com|2452491| +9629|AAAAAAAANJFCAAAA|172469|5870|24118|2449364|2449334|Sir|Mark|Lawson|N|25|1|1968|PHILIPPINES||Mark.Lawson@Ksbfh0.org|2452492| +9630|AAAAAAAAOJFCAAAA|538280|187|44047|2450412|2450382|Dr.|Janette|Brown|Y|14|5|1928|BURUNDI||Janette.Brown@Pf.edu|2452527| +9631|AAAAAAAAPJFCAAAA|1271396|2553|44110|2451908|2451878|Dr.|Richard|Trent|N|17|8|1944|SWAZILAND||Richard.Trent@TTf72F9oU5Ot.com|2452397| +9632|AAAAAAAAAKFCAAAA|377381|6163|8145|2451842|2451812|Mr.|Clyde|Grant|Y|31|8|1966|PITCAIRN||Clyde.Grant@4E0.com|2452362| +9633|AAAAAAAABKFCAAAA|23413|5144|8522|2449992|2449962|Ms.|Phyllis|Fuchs|Y|4|10|1981|BENIN||Phyllis.Fuchs@30k0ScU7UO.org|2452437| +9634|AAAAAAAACKFCAAAA|1111438|3961|7331|2449204|2449174|Miss|Melissa|Miller|N|28|6|1952|CAMEROON||Melissa.Miller@ieO7FGsqhjKOpM.com|2452438| +9635|AAAAAAAADKFCAAAA|9897|7153|48809|2451188|2451158|Mr.|William|Workman|N|27|12|1950|ISRAEL||William.Workman@Zrl.edu|2452461| +9636|AAAAAAAAEKFCAAAA|458500|5676|52|2452090|2452060|Dr.|Roger|Newman|N|31|8|1953|CROATIA||Roger.Newman@k.edu|2452618| +9637|AAAAAAAAFKFCAAAA|1096259|419|13379|2450508|2450478|Ms.|Kari|Banks|N|7|2|1933|THAILAND||Kari.Banks@H6efKSE1NB9.com|2452359| +9638|AAAAAAAAGKFCAAAA|1725204|751|32651|2450710|2450680|Mr.|Wayne|Pearson|Y|16|3|1985|AFGHANISTAN||Wayne.Pearson@dTGJvhv0IMj.edu|2452326| +9639|AAAAAAAAHKFCAAAA|152330|1768|48853|2451325|2451295|Mr.|Kenneth|Davis|N|8|12|1963|BARBADOS||Kenneth.Davis@o5XLrOFLk.org|2452642| +9640|AAAAAAAAIKFCAAAA|514102|1070|45685|2450866|2450836|Dr.|Edith|Rivera|N|3|1|1987|HAITI||Edith.Rivera@r7CoH4l7k.com|2452312| +9641|AAAAAAAAJKFCAAAA|1297030|2317|9957|2450311|2450281|Ms.|Aaron|Davis|N|6|8|1943|SOLOMON ISLANDS||Aaron.Davis@L58TMQUXh5M.com|2452438| +9642|AAAAAAAAKKFCAAAA|1233486|4298|1957|2451316|2451286|Mr.|Mark|Mcguire|Y|23|6|1966|TOKELAU||Mark.Mcguire@952TERj6RYCz.edu|2452517| +9643|AAAAAAAALKFCAAAA|1852177|5500|40070|2450260|2450230|Mr.|Lawrence|Evans|Y|19|10|1932|FRENCH GUIANA||Lawrence.Evans@Yd3OhO.org|2452582| +9644|AAAAAAAAMKFCAAAA|664956|2158|35760|2449158|2449128|Ms.|Peggy|Paulson|Y|7|6|1952|SEYCHELLES||Peggy.Paulson@lotuD8Q.com|2452569| +9645|AAAAAAAANKFCAAAA|42248|6457|20713|2450406|2450376|Miss|Alice|Long|Y|7|5|1960|ESTONIA||Alice.Long@nivUUM.com|2452472| +9646|AAAAAAAAOKFCAAAA|1149830|4215|2553|2449320|2449290|Miss|Evelyn|Flores|N|19|10|1952|UGANDA||Evelyn.Flores@Y9sgkry.org|2452474| +9647|AAAAAAAAPKFCAAAA|1330991|506|37268|2450927|2450897|Sir|Alejandro|Granados|Y|18|4|1984|VIET NAM||Alejandro.Granados@aX.org|2452454| +9648|AAAAAAAAALFCAAAA|423846||33617|2450018|2449988|||French|||||COMOROS|||2452398| +9649|AAAAAAAABLFCAAAA|1055020|6390|15360|2449609|2449579|Mr.|Alfredo|Tanner|N|2|8|1989|UZBEKISTAN||Alfredo.Tanner@BHh72GG0QFebcf.com|2452631| +9650|AAAAAAAACLFCAAAA|1610561|5330|32146|2449979|2449949|Dr.|Jesus|Greene|N|2|8|1945|MAYOTTE||Jesus.Greene@LNIihQthU2z.org|2452535| +9651|AAAAAAAADLFCAAAA|||45257||||William|Lopez|Y||||GIBRALTAR|||| +9652|AAAAAAAAELFCAAAA|1374942|4535|47728|2449495|2449465|Dr.|Donna|Gordon|N|8|3|1961|QATAR||Donna.Gordon@1.org|2452497| +9653|AAAAAAAAFLFCAAAA|495235|1238|40441|2452246|2452216|Sir|Rick|Allen|Y|28|2|1976|ALBANIA||Rick.Allen@C.com|2452603| +9654|AAAAAAAAGLFCAAAA|1333998|1681|39317|2449717|2449687|Dr.|Nita|Walker|N|10|2|1947|VANUATU||Nita.Walker@aRVQ.edu|2452556| +9655|AAAAAAAAHLFCAAAA|497210|1517|18624|2452047|2452017|Miss|Theresa|Withers|N|7|12|1952|ISRAEL||Theresa.Withers@Bsxgo180.org|2452534| +9656|AAAAAAAAILFCAAAA|365583|2228|16873|2450975|2450945|Dr.|Richard|Cate|Y|11|1|1960|SWITZERLAND||Richard.Cate@z.edu|2452550| +9657|AAAAAAAAJLFCAAAA|1560305|3932|33559|2450355|2450325|Sir|Michael|Burroughs|N|11|11|1977|GUAM||Michael.Burroughs@bQsNFSMxFltH1QQUy.edu|2452525| +9658|AAAAAAAAKLFCAAAA|1536053|4824|41595|2449582|2449552|Miss|Christina|Manson|N|11|8|1977|RUSSIAN FEDERATION||Christina.Manson@90UyoUo2qL4LODd.edu|2452568| +9659|AAAAAAAALLFCAAAA|124920|2005|18290|2452666|2452636|Ms.|Sharon|Harrison|N|13|12|1982|HONG KONG||Sharon.Harrison@AL26lYi0l5.org|2452290| +9660|AAAAAAAAMLFCAAAA|1591317|7175|9433|2452245|2452215|Mr.|Ernesto|Hardman|N|30|3|1983|MOROCCO||Ernesto.Hardman@NmYfE.com|2452387| +9661|AAAAAAAANLFCAAAA|1733552|5189|43226|2452482|2452452|Sir|Tracy|Hamrick|N|15|11|1936|YEMEN||Tracy.Hamrick@e8xKOVkEu3JZo.com|2452455| +9662|AAAAAAAAOLFCAAAA|58707|2620|8533|2451760|2451730|Mr.|Shane|Snyder|Y|12|11|1986|IRAQ||Shane.Snyder@GCcuMJ.edu|2452388| +9663|AAAAAAAAPLFCAAAA|1037641|5370|5499|2451780|2451750|Dr.|Julie|Martinez|Y|14|12|1988|VENEZUELA||Julie.Martinez@O5kKVfBpV02D.com|2452513| +9664|AAAAAAAAAMFCAAAA|550630|3334|45638|2450830|2450800|Miss|Kerrie|Johnson|N|10|3|1985|MALTA||Kerrie.Johnson@0tgL.org|2452412| +9665|AAAAAAAABMFCAAAA|383100|4212|22524|2450584|2450554|Sir|Robert|Martin|N|11|1|1934|SPAIN||Robert.Martin@D0VspbLZsIGZ.org|2452568| +9666|AAAAAAAACMFCAAAA|133723|6103|41283|2450195|2450165|Sir|James|Anderson|N|20|4|1934|CROATIA||James.Anderson@cnoXnOo5rCc1C.com|2452373| +9667|AAAAAAAADMFCAAAA|1876500|5913|3773|2452050|2452020|Mr.|Joshua|Johnson|N|25|4|1969|SENEGAL||Joshua.Johnson@PU.edu|2452439| +9668|AAAAAAAAEMFCAAAA|1252694|1038|7029|2451836|2451806|Miss|Kathy|Shoemaker|N|29|5|1975|TUNISIA||Kathy.Shoemaker@dpH6FGipZRHF.com|2452298| +9669|AAAAAAAAFMFCAAAA|1560394|6370|5593|2449696|2449666|Mrs.|Melissa|Knight|N|4|5|1967|PALAU||Melissa.Knight@VlrtZfradOhB09GI7.com|2452461| +9670|AAAAAAAAGMFCAAAA|318922|6088|44501|2449642|2449612|Mrs.|Carolyn|Fisher|N|19|7|1983|SIERRA LEONE||Carolyn.Fisher@gAyrLjhnji.com|2452648| +9671|AAAAAAAAHMFCAAAA|975178|3854|15330|2450200|2450170|Mrs.|Trina|Johnson|Y|12|7|1930|CAMEROON||Trina.Johnson@U5LXp.com|2452301| +9672|AAAAAAAAIMFCAAAA|309223|219|23262|2451801|2451771|Mr.|Les|Berg|N|11|8|1929|ALBANIA||Les.Berg@0SUtgimMVUeu6Espo.edu|2452602| +9673|AAAAAAAAJMFCAAAA|1616611|1098|42432|2449715|2449685|Dr.|Bryce|Davila|Y|9|11|1957|TOKELAU||Bryce.Davila@mormOocua02k.edu|2452641| +9674|AAAAAAAAKMFCAAAA|1609979|3001|40423|2450727|2450697|Sir|Michael|Macias|N|24|9|1979|JAMAICA||Michael.Macias@qEObm1gU0fFbQ5rk1.com|2452344| +9675|AAAAAAAALMFCAAAA|871437|47|14257|2451628|2451598|Dr.|Edmund|Trevino|N|24|10|1967|CAMBODIA||Edmund.Trevino@ZJq9xrlRl.edu|2452620| +9676|AAAAAAAAMMFCAAAA|7517|3628|35739|2449505|2449475|Mr.|William|Gonzalez|Y|29|11|1980|PUERTO RICO||William.Gonzalez@LNbXC.edu|2452433| +9677|AAAAAAAANMFCAAAA|592527|4734|3341|2451751|2451721|Mr.|Todd|Windham|Y|3|12|1945|TURKMENISTAN||Todd.Windham@CPX7uPO8Nml.com|2452579| +9678|AAAAAAAAOMFCAAAA|472267|8|8596|2451331|2451301|Sir|Adolfo|Collins|Y|18|6|1938|TURKMENISTAN||Adolfo.Collins@Qa2rK8QcJKS46ujmn31.edu|2452369| +9679|AAAAAAAAPMFCAAAA|994630|6224|36618|2449116|2449086|Mr.|Jeff|Bowen|Y|27|2|1951|UNITED ARAB EMIRATES||Jeff.Bowen@7m2lmvYF.edu|2452588| +9680|AAAAAAAAANFCAAAA|528508|5333|39358|2450675|2450645|Miss|Mia|Francis|Y|31|12|1954|LESOTHO||Mia.Francis@2PDllkClDYYpp.com|2452430| +9681|AAAAAAAABNFCAAAA|408661|5182|18456|2449125|2449095|Ms.|Cheryl|Lopez|N|8|12|1932|MACAO||Cheryl.Lopez@utHxR.com|2452400| +9682|AAAAAAAACNFCAAAA|183011|4918|42025|2450536|2450506|Sir|Scott|Rogers|Y|26|8|1944|ANGOLA||Scott.Rogers@p6uNlS91g7qggX1M0.edu|2452630| +9683|AAAAAAAADNFCAAAA|9283|4657|35984|2451936|2451906|Sir|Jeremy|Young|Y|31|1|1926|MACAO||Jeremy.Young@VGlhKS9fu5f6n.com|2452542| +9684|AAAAAAAAENFCAAAA|1864826|217|41608|2450576|2450546|Miss|Pamela|French|Y|7|3|1975|CUBA||Pamela.French@n.org|2452478| +9685|AAAAAAAAFNFCAAAA|1645238|4835|11632|2451102|2451072|Miss|Marilyn|Cook|Y|28|5|1945|ANTIGUA AND BARBUDA||Marilyn.Cook@UFnsxMHGpJtz6q.com|2452374| +9686|AAAAAAAAGNFCAAAA|1100884|6679|4921|2451022|2450992|Mr.|Charles|Barton|Y|4|1|1957|UNITED KINGDOM||Charles.Barton@XCPdz7o25K4jhM.edu|2452445| +9687|AAAAAAAAHNFCAAAA|86539|639|27586|2451225|2451195|Ms.|Tabitha|Singh|Y|2|9|1945|BELARUS||Tabitha.Singh@xH2cN1CFmzKX2nJHBAF.com|2452307| +9688|AAAAAAAAINFCAAAA|1512569|3634|8699|2450602|2450572|||Wasson||||||||2452577| +9689|AAAAAAAAJNFCAAAA|126010|2383|9603|2451487|2451457|Mrs.|Elizabeth|Trevino|Y|27|2|1926|BAHRAIN||Elizabeth.Trevino@NiFbn8tri4u.com|2452336| +9690|AAAAAAAAKNFCAAAA|180599|2130|108|2452128|2452098|Miss|Denise|Hurst|N|19|6|1930|FRENCH POLYNESIA||Denise.Hurst@pAVRx69LSGuqycBt.org|2452380| +9691|AAAAAAAALNFCAAAA|1623791|5232|7873|2452439|2452409|Sir|Donny|Brooks|N|19|8|1948|CUBA||Donny.Brooks@LK0C1.com|2452510| +9692|AAAAAAAAMNFCAAAA|332662|3354|46846|2450086|2450056|Mrs.|Elaina|Williamson|N|5|1|1990|CAMBODIA||Elaina.Williamson@84pcL.com|2452311| +9693|AAAAAAAANNFCAAAA|1515192|5254|40163|2450171|2450141|Mrs.|Paula|Williams|N|7|7|1974|FRENCH GUIANA||Paula.Williams@SVFXGOmVka.com|2452487| +9694|AAAAAAAAONFCAAAA|1273866|1065|42711|2449165|2449135|Mrs.|Margot|Osborne|Y|4|11|1975|NETHERLANDS ANTILLES||Margot.Osborne@pxMdMAU98xszG.com|2452334| +9695|AAAAAAAAPNFCAAAA|1442070|1456|16489|2450683|2450653|Miss|Amy|Large|Y|17|12|1954|SPAIN||Amy.Large@CKVmrVZYQmBPc.org|2452327| +9696|AAAAAAAAAOFCAAAA|900516|2860|15204|2451642|2451612|Dr.|Rene|Thomason|Y|10|1|1974|FRANCE||Rene.Thomason@2Etd85yIdf6C.edu|2452566| +9697|AAAAAAAABOFCAAAA|147607|1699|22591|2450453|2450423|Dr.|Galen|Perry|N|24|1|1959|JORDAN||Galen.Perry@scPYjG8qzC6uB93jf.com|2452353| +9698|AAAAAAAACOFCAAAA|179301|4361|43421|2449665|2449635|Dr.|John|Fairchild|N|1|9|1940|CZECH REPUBLIC||John.Fairchild@XV7AaY1a7v8Chg5NTH.edu|2452369| +9699|AAAAAAAADOFCAAAA|102149|1075|37601|2452016|2451986|Ms.|Emily|Ruiz|N|24|8|1987|FRANCE||Emily.Ruiz@LjsuJDHFSe4Zm5k8.edu|2452292| +9700|AAAAAAAAEOFCAAAA|1693017|97|12777|2450427|2450397|Dr.|Barbara|Hill|N|3|12|1967|NEW CALEDONIA||Barbara.Hill@A9fF7C5KOQnkDLT9DYu.com|2452391| +9701|AAAAAAAAFOFCAAAA|1232104|3683|34015|2449732|2449702|Sir|William|Crouch|N|14|10|1954|FAROE ISLANDS||William.Crouch@Tr3BZuoR.edu|2452442| +9702|AAAAAAAAGOFCAAAA|482655|490|15078|2450877|2450847|Mr.|James|Miller|Y|1|10|1959|INDONESIA||James.Miller@31DQ9zLPKPGBJ.com|2452491| +9703|AAAAAAAAHOFCAAAA|1884935|263|46438|2449394|2449364|Mr.|William|Hoffman|N|25|11|1992|ESTONIA||William.Hoffman@PgL985f0aisp.org|2452363| +9704|AAAAAAAAIOFCAAAA|221675|5598|28321|2452341|2452311|Mrs.|Anita|Rodriguez|N|2|10|1950|CAMBODIA||Anita.Rodriguez@0nsad2NkMHZfxb.com|2452408| +9705|AAAAAAAAJOFCAAAA|85826|4461|19143|2449040|2449010|Ms.|Margaret|Pfeiffer|N|16|8|1977|OMAN||Margaret.Pfeiffer@3Kx7ARupVISGcEstA.com|2452524| +9706|AAAAAAAAKOFCAAAA||2673|41759||2452180||Lillie||Y|31|7|1987||||2452444| +9707|AAAAAAAALOFCAAAA|960039|4359|25879|2450838|2450808|Dr.|Elva|Usher|Y|21|2|1958|BELIZE||Elva.Usher@uetEPlPc8Ha.edu|2452469| +9708|AAAAAAAAMOFCAAAA|1003579|2183|40387|2449893|2449863|Mr.|Timothy|Peters|N|25|7|1936|R�UNION||Timothy.Peters@LnUq.org|2452409| +9709|AAAAAAAANOFCAAAA|502210|3989|40968|2451186|2451156|Dr.|Stephen|Carr|Y|23|9|1992|BELGIUM||Stephen.Carr@5plUY.org|2452542| +9710|AAAAAAAAOOFCAAAA|1070562|538|31267|2449725|2449695|Miss|Christine|Bell|Y|18|11|1938|NETHERLANDS ANTILLES||Christine.Bell@fk.org|2452368| +9711|AAAAAAAAPOFCAAAA|1222482|5002|43122|2451925|2451895|Dr.|Caroline|Jackson|N|24|2|1958|FRENCH GUIANA||Caroline.Jackson@pd7PjYND.org|2452364| +9712|AAAAAAAAAPFCAAAA|175579|748|3964|2450353|2450323|Dr.|Lucy|Smith|N|13|7|1943|FRENCH POLYNESIA||Lucy.Smith@sLrl.com|2452408| +9713|AAAAAAAABPFCAAAA|245132|807|12082|2452415|2452385|Mr.|Cleveland|Harris|Y|4|2|1933|TUVALU||Cleveland.Harris@l2B2KzbZkC7b.com|2452426| +9714|AAAAAAAACPFCAAAA|1203878|4540|27202|2449443|2449413|Mr.|George|Branch|N|26|10|1967|MONGOLIA||George.Branch@8ePRM.com|2452517| +9715|AAAAAAAADPFCAAAA|1918745|6389|8433|2449107|2449077|Dr.|Terence|Prosser|N|22|5|1951|ESTONIA||Terence.Prosser@oT.com|2452284| +9716|AAAAAAAAEPFCAAAA|1552040|2852|45711|2452241|2452211|Dr.|Kenneth|Martinez|N|17|7|1936|SAN MARINO||Kenneth.Martinez@CvId.edu|2452618| +9717|AAAAAAAAFPFCAAAA|1039042|1509|23976|2452447|2452417|Dr.|Theodore|Paul|Y|16|11|1990|ARGENTINA||Theodore.Paul@e.com|2452495| +9718|AAAAAAAAGPFCAAAA|1172993|6924|30855|2449870|2449840|Ms.|Bonnie|Johnson|Y|2|6|1954|GERMANY||Bonnie.Johnson@6OYrrpYmbIYiMhmch.edu|2452341| +9719|AAAAAAAAHPFCAAAA|||3919|2452594|2452564||Jerry|Henderson|Y|20|8||||Jerry.Henderson@52Si.org|2452621| +9720|AAAAAAAAIPFCAAAA|1234014|420|47979|2450781|2450751|Sir|Roberto|Davis|Y|21|2|1961|BELIZE||Roberto.Davis@c3ixyBLGlp0AX.org|2452536| +9721|AAAAAAAAJPFCAAAA|56305|1376|1647|2451495|2451465|Ms.|Blanca|Powell|N|17|8|1991|FIJI||Blanca.Powell@JGsvol.com|2452477| +9722|AAAAAAAAKPFCAAAA|1303297|4263|9229|||Mrs.|Lisa|Negron|Y||6||||Lisa.Negron@rI.org|| +9723|AAAAAAAALPFCAAAA|||11423|2452646|2452616||Gary||N|||||||| +9724|AAAAAAAAMPFCAAAA|620440|310|5380|2451911|2451881|Mr.|Jonathan|Robison|Y|22|10|1932|NEW ZEALAND||Jonathan.Robison@kTAyaz0.edu|2452450| +9725|AAAAAAAANPFCAAAA|948904|835|38366|2451227|2451197|Dr.|Maida|White|N|18|2|1940|MADAGASCAR||Maida.White@7ck0hr19cjeFhnkJ.com|2452530| +9726|AAAAAAAAOPFCAAAA|735287|1979|671|2450386|2450356|Dr.|Eve|Fine|Y|23|2|1924|NEPAL||Eve.Fine@AmUEutqzK1G8.org|2452403| +9727|AAAAAAAAPPFCAAAA|328003|2645|42887|2449812|2449782|Mr.|Nathan|Clark|N|28|11|1965|SIERRA LEONE||Nathan.Clark@4yxU0Mm1Yh77VFsBz.com|2452548| +9728|AAAAAAAAAAGCAAAA|1256716|4508|28300|2450328|2450298|Mrs.|Nichole|Godinez|N|13|4|1953|ALAND ISLANDS||Nichole.Godinez@qa.edu|2452386| +9729|AAAAAAAABAGCAAAA|1066817|4902|43387|2450392|2450362|Dr.|David|Sandoval|Y|27|1|1946|TUNISIA||David.Sandoval@eFhrDStP4.com|2452573| +9730|AAAAAAAACAGCAAAA|1815649|5291|8533|2452217|2452187|Mr.|William|Irving|N|13|7|1959|AUSTRALIA||William.Irving@Vfxj9YtGEKZxl.edu|2452647| +9731|AAAAAAAADAGCAAAA|1883777|5516|12902|2450967|2450937|Ms.|Antonio|Martin|Y|1|3|1988|GUATEMALA||Antonio.Martin@z8QhaAibxP.org|2452477| +9732|AAAAAAAAEAGCAAAA|519288|2485|18529|2449520|2449490|Miss|Geraldine|Martin|Y|3|8|1962|KENYA||Geraldine.Martin@8YaqezL8pm.org|2452372| +9733|AAAAAAAAFAGCAAAA|358148|883|8818|2451800|2451770|Mr.|Keith|Alexander|N|26|5|1925|SAINT LUCIA||Keith.Alexander@NpFrkpmdihqJDz.com|2452437| +9734|AAAAAAAAGAGCAAAA|202790|523|18209|2451832|2451802|Mr.|Rodger|Bryan|Y|4|6|1931|ICELAND||Rodger.Bryan@pPMB.org|2452493| +9735|AAAAAAAAHAGCAAAA|1351486|3673|11860|2452671|2452641|Miss|Lizzie|Smith|N|21|11|1927|INDONESIA||Lizzie.Smith@8c5L880f.org|2452382| +9736|AAAAAAAAIAGCAAAA|1000876|6366|6609|2452607|2452577|Dr.|Julia|Murphy|Y|1|5|1931|UNITED KINGDOM||Julia.Murphy@pos7iKPUBrNT.com|2452532| +9737|AAAAAAAAJAGCAAAA|1812907|4944|47781|2451667|2451637|Ms.|Myrtle|Jenkins|N|16|5|1984|MALTA||Myrtle.Jenkins@M79zNEkIs1z0kJ.edu|2452387| +9738|AAAAAAAAKAGCAAAA|1611903|3561|22019|2450648|2450618|Sir|Norman|Shipman|N|16|6|1969|INDONESIA||Norman.Shipman@5v3Sdd.org|2452328| +9739|AAAAAAAALAGCAAAA|1580373|5488|15756|2449624|2449594|Miss|Jacqueline|Snyder|Y|24|1|1986|TONGA||Jacqueline.Snyder@H0LdI4UQtP87q.edu|2452648| +9740|AAAAAAAAMAGCAAAA|1586416|1433|27650|2452551|2452521|Miss|Anne|Sullivan|N|25|10|1962|AUSTRIA||Anne.Sullivan@5TUaGSg66MbYqCUp.com|2452484| +9741|AAAAAAAANAGCAAAA|1075992|5646|40147|2451658|2451628|Mrs.|Marie|Evans|N|15|3|1924|MEXICO||Marie.Evans@r.org|2452548| +9742|AAAAAAAAOAGCAAAA|305751|778|7852|2449594|2449564|Mr.|Eric|Davis|Y|11|12|1976|ECUADOR||Eric.Davis@EUTpH8se4sMD7dB.com|2452600| +9743|AAAAAAAAPAGCAAAA|1307890|2949|40808|2449308|2449278|Sir|Richard|Fowler|Y|1|11|1951|MALAYSIA||Richard.Fowler@2kpbc4N0PjfSqTC58Z.edu|2452644| +9744|AAAAAAAAABGCAAAA|418175|2062|39549|2449437|2449407|Sir|Curtis|Irby|N|3|2|1938|R�UNION||Curtis.Irby@apHG6DZ41KDVS.edu|2452320| +9745|AAAAAAAABBGCAAAA|1694913|1420|12327|2450244|2450214|Mr.|Joseph|King|N|10|6|1958|TUNISIA||Joseph.King@O8YgeZe7u8D5d.edu|2452441| +9746|AAAAAAAACBGCAAAA|1612233|951|28822|2450947|2450917|Dr.|Jennifer|Whitaker|Y|4|1|1927|TRINIDAD AND TOBAGO||Jennifer.Whitaker@QKA4cPvNf06.com|2452530| +9747|AAAAAAAADBGCAAAA|365525|3931|32784|2452574|2452544|Miss|Louise|Trent|Y|12|4|1980|SAINT HELENA||Louise.Trent@o.com|2452591| +9748|AAAAAAAAEBGCAAAA|1293486|3813|43340|2449775|2449745|Ms.|Brandi|Baptiste|Y|13|3|1929|SIERRA LEONE||Brandi.Baptiste@duVknoa.org|2452330| +9749|AAAAAAAAFBGCAAAA|1045379|2779|20979|2452651|2452621|Sir|Willard|Garcia|N|21|9|1969|AFGHANISTAN||Willard.Garcia@puQoA6m0kGuLyBt.edu|2452293| +9750|AAAAAAAAGBGCAAAA|191433|3699|20365|2449105|2449075|Mrs.|Gloria|Woods|N|13|7|1968|MARSHALL ISLANDS||Gloria.Woods@iGQSseH6Af.org|2452336| +9751|AAAAAAAAHBGCAAAA|153169|3667|47365|2452373|2452343|Ms.|Jennifer|Fischer|Y|11|8|1955|BELARUS||Jennifer.Fischer@M8fPtEfzEgpgs.org|2452613| +9752|AAAAAAAAIBGCAAAA|1312453|2368|15130|2449677|2449647|Sir|Gary|Leonard|Y|25|12|1981|SWAZILAND||Gary.Leonard@nU6OjiglSa7V6tkXAF4B.com|2452520| +9753|AAAAAAAAJBGCAAAA|1596598|4361|12077|2450104|2450074|Dr.|Diane|Henderson|N|28|12|1989|VIRGIN ISLANDS, U.S.||Diane.Henderson@2Q3QRSAhniHUErHx.com|2452611| +9754|AAAAAAAAKBGCAAAA|1772571|5923|9722|2451467|2451437|Dr.|Danielle|Snow|N|8|1|1934|CAMEROON||Danielle.Snow@lB7emDQyJA.org|2452576| +9755|AAAAAAAALBGCAAAA|1377669|6447|44778|2450532|2450502|Dr.|Brian|Mills|Y|26|10|1928|CROATIA||Brian.Mills@nPkLm5Y5vA9c.com|2452286| +9756|AAAAAAAAMBGCAAAA|1359724|2053|41935|2451277|2451247|Mr.|Ryan|Freese|Y|31|12|1931|SWITZERLAND||Ryan.Freese@i9TVS.edu|2452397| +9757|AAAAAAAANBGCAAAA|169607|3718|18182|2449104|2449074|Miss|Shiela|Rivera|Y|9|1|1945|CAPE VERDE||Shiela.Rivera@7lvBq7ZSQbC89.edu|2452490| +9758|AAAAAAAAOBGCAAAA|1648494|288|20529|2450981|2450951|Mr.|James|Smith|Y|28|12|1975|ERITREA||James.Smith@TOBFPAp2JalYH.com|2452545| +9759|AAAAAAAAPBGCAAAA|1110487|4844|32801|2451051|2451021|Miss|Joyce|Hall|N|11|7|1934|GHANA||Joyce.Hall@AAkk4o48d2O.com|2452616| +9760|AAAAAAAAACGCAAAA|6024|388|49905|||Miss|Linda|Hendrix|Y||1||||Linda.Hendrix@FV37Yc16GBE4t.org|2452514| +9761|AAAAAAAABCGCAAAA|156727|5421|9715|2449040|2449010|Mrs.|Michelle|Sullivan|N|25|5|1957|TONGA||Michelle.Sullivan@Q4QQ72dyf1C.edu|2452491| +9762|AAAAAAAACCGCAAAA|277389|812|15435|2452032|2452002|Miss|Roxane|Castillo|Y|21|6|1970|WESTERN SAHARA||Roxane.Castillo@HDCg3dq8lhEZM6.edu|2452372| +9763|AAAAAAAADCGCAAAA|1779399|2123|28094|2450269|2450239|Dr.|Alfred|Farmer|Y|2|10|1941|FRENCH POLYNESIA||Alfred.Farmer@y1SJPKvkZdvJN11bU.edu|2452343| +9764|AAAAAAAAECGCAAAA|676205|4136|47667|2449317|2449287|Sir|Jackson|Wells|N|26|12|1972|SAINT HELENA||Jackson.Wells@Kr7eTVh.com|2452570| +9765|AAAAAAAAFCGCAAAA|515785|2703|26421|2449236|2449206|Mrs.|Elane|Finley|N|10|3|1952|PORTUGAL||Elane.Finley@C0xO29543IL6.edu|2452536| +9766|AAAAAAAAGCGCAAAA|1260128||29331||2449126|Dr.|Toni|Wood|Y||9|1937|||Toni.Wood@MAF3JTEmYZc.edu|2452317| +9767|AAAAAAAAHCGCAAAA|1480596|4799|33002|2449499|2449469|Dr.|Dustin|Howard|N|12|12|1953|ESTONIA||Dustin.Howard@IA42zoNkxxOg8.edu|2452359| +9768|AAAAAAAAICGCAAAA|1605667|1912|29357|2450976|2450946|Ms.|Denise|Rico|Y|23|9|1963|TUVALU||Denise.Rico@tCDzi.com|2452523| +9769|AAAAAAAAJCGCAAAA|1160871|3066|34461|2450317|2450287|Mrs.|Sherry|Reed|N|13|5|1960|MAURITIUS||Sherry.Reed@E2z9DGiz.edu|2452569| +9770|AAAAAAAAKCGCAAAA|1063339|2436|45180|2450785|2450755|Mrs.|Rosemary|Ramsey|N|2|1|1949|SOLOMON ISLANDS||Rosemary.Ramsey@tVPnPyc0hClrp.org|2452349| +9771|AAAAAAAALCGCAAAA|1289229|1515|38248|2450117|2450087|Sir|David|Lira|Y|8|1|1980|CAYMAN ISLANDS||David.Lira@SXd5Z05tiCt75iIshH.org|2452379| +9772|AAAAAAAAMCGCAAAA|1918215|414|27413|2451907|2451877|Dr.|Steven|Lynch|N|13|12|1965|SYRIAN ARAB REPUBLIC||Steven.Lynch@ukdi3Ep.com|2452520| +9773|AAAAAAAANCGCAAAA|77383|6396|31456|2450559|2450529|Miss|Barbara|Booker|N|26|11|1949|TUNISIA||Barbara.Booker@c7d0aN2dfJKp79m657RP.com|2452600| +9774|AAAAAAAAOCGCAAAA|1602672|1522|21219||||Maxine|Echols|N|19||1941|BANGLADESH|||| +9775|AAAAAAAAPCGCAAAA|1105689|5420|39551|2450032|2450002|Dr.|Angela|Mohr|N|4|9|1957|RUSSIAN FEDERATION||Angela.Mohr@nCZD6R.org|2452521| +9776|AAAAAAAAADGCAAAA|1853693|523|40553|2451835|2451805|Dr.|Oneida|Bennett|Y|29|10|1949|URUGUAY||Oneida.Bennett@ln87IbTOl.org|2452640| +9777|AAAAAAAABDGCAAAA|79842|3391|39293|2449875|2449845|Ms.|Meri|Griggs|N|6|4|1972|SERBIA||Meri.Griggs@dpYA3dXx.org|2452538| +9778|AAAAAAAACDGCAAAA|1106610|2644|34278|2452271|2452241|Ms.|Elaine|Martinson|N|22|11|1990|NAMIBIA||Elaine.Martinson@OR4NTNFVHp.org|2452589| +9779|AAAAAAAADDGCAAAA|260843|5118|44546|2450078|2450048|Mrs.|Amanda|Burns|N|29|8|1954|PHILIPPINES||Amanda.Burns@f6.com|2452291| +9780|AAAAAAAAEDGCAAAA|1795706|4447|13588|2450187|2450157|Ms.|Jennifer|Leon|N|20|6|1974|SENEGAL||Jennifer.Leon@6JjKhHQZRhkykpHzK9ox.edu|2452442| +9781|AAAAAAAAFDGCAAAA|355302|3804|25859|2450272|2450242|Dr.|Michael|Morrow|Y|13|8|1959|NIGERIA||Michael.Morrow@Xpd.edu|2452290| +9782|AAAAAAAAGDGCAAAA|756548||9612|||Mr.|||N|14|4|1945|ROMANIA||Raul.Galvan@6hllES3Pa4FlA6.com|2452544| +9783|AAAAAAAAHDGCAAAA|552141|6265|14389|2449721|2449691|Mrs.|Roberta|Alford|N|27|4|1969|HONG KONG||Roberta.Alford@VHkXtfBt3OXv1Ri5Bsm.edu|2452350| +9784|AAAAAAAAIDGCAAAA|126635|5927|21172|2449301|2449271|Mrs.|Helen|Owen|Y|23|10|1952|CROATIA||Helen.Owen@LdYht1EoS.edu|2452444| +9785|AAAAAAAAJDGCAAAA|||31145|2449714|2449684|||Keen|Y||3|||||2452596| +9786|AAAAAAAAKDGCAAAA|1794233|4438|18196|2450576|2450546|Dr.|Annette|Fuller|Y|29|4|1926|ARMENIA||Annette.Fuller@XlaruuOYFtqhP6.com|2452455| +9787|AAAAAAAALDGCAAAA|1420945|5385|4272|2452260|2452230|Dr.|Milly|Cahill|Y|23|4|1935|PUERTO RICO||Milly.Cahill@TqzYDDYQzH.org|2452321| +9788|AAAAAAAAMDGCAAAA|796980|3606|35432|2451446|2451416|Dr.|Larry|Bowers|Y|4|11|1960|BELGIUM||Larry.Bowers@J2SnSY3.com|2452409| +9789|AAAAAAAANDGCAAAA|761884|5883|27882|2449175|2449145|Mrs.|Ethel|Robinson|Y|5|6|1979|SUDAN||Ethel.Robinson@aG2.org|2452471| +9790|AAAAAAAAODGCAAAA|991584|4442|21700|2450433|2450403|Mr.|Barry|Ray|Y|5|6|1963|RWANDA||Barry.Ray@GovUFk.edu|2452354| +9791|AAAAAAAAPDGCAAAA|1131171|1539|30307|2450496|2450466|Dr.|Manuel|Rockwell|N|1|5|1951|KYRGYZSTAN||Manuel.Rockwell@IQc9efm.org|2452546| +9792|AAAAAAAAAEGCAAAA|||21191|||||Kennedy|N||||TOKELAU|||2452416| +9793|AAAAAAAABEGCAAAA|712279||25802|2449580|||Susan|Wilson|Y|9|||TRINIDAD AND TOBAGO||Susan.Wilson@L4Zdvk1g7ZOS.com|2452621| +9794|AAAAAAAACEGCAAAA|640969|5173|26177|2452373|2452343|Sir|George|Stauffer|N|20|6|1975|LUXEMBOURG||George.Stauffer@EKLBDLmoV.org|2452554| +9795|AAAAAAAADEGCAAAA|7149|5934|13519|2451127|2451097|Sir|Michael|Hollis|N|18|9|1987|SRI LANKA||Michael.Hollis@pG.edu|2452466| +9796|AAAAAAAAEEGCAAAA|1500145|5202|20259|2450184|2450154|Miss|Melissa|Montoya|N|26|8|1938|IRAQ||Melissa.Montoya@THTA.org|2452577| +9797|AAAAAAAAFEGCAAAA|677986|1528|23470|2451460|2451430|Mrs.|Cecelia|Childress|N|9|1|1941|BANGLADESH||Cecelia.Childress@ym.edu|2452424| +9798|AAAAAAAAGEGCAAAA|1713809|2236|49016|2452531|2452501|Mr.|Kurtis|Starkey|N|2|3|1957|CHINA||Kurtis.Starkey@l9X7.org|2452356| +9799|AAAAAAAAHEGCAAAA|116936|7041|28699|2451573|2451543|Mrs.|Sue|Bryant|N|4|3|1963|SURINAME||Sue.Bryant@60.edu|2452566| +9800|AAAAAAAAIEGCAAAA|983343|160|44577|2450312|2450282|Miss|Dorothy|Harris|Y|5|4|1959|KUWAIT||Dorothy.Harris@jsJs5vdBMT.com|2452489| +9801|AAAAAAAAJEGCAAAA|1546104|155|32159|2451359|2451329|Dr.|Lavonne|Benner|Y|29|5|1983|SYRIAN ARAB REPUBLIC||Lavonne.Benner@Cb0xUuVHF.org|2452355| +9802|AAAAAAAAKEGCAAAA|1741453|4918|23936|2450352|2450322|Dr.|Michael|Martin|N|11|10|1969|SOUTH AFRICA||Michael.Martin@HeGya.com|2452287| +9803|AAAAAAAALEGCAAAA|1273224|6115|7359|2451843|2451813|Dr.|Shaun|White|N|16|12|1970|MONTSERRAT||Shaun.White@QhFc6eZI.org|2452315| +9804|AAAAAAAAMEGCAAAA|84416|5796|4376|2451714|2451684|Sir|David|Ball|Y|8|8|1956|TURKMENISTAN||David.Ball@o3evPqZ5pVo.org|2452444| +9805|AAAAAAAANEGCAAAA|1069628|5448|34068|2452439|2452409|Dr.|Tyrone|Stanley|N|23|8|1935|YEMEN||Tyrone.Stanley@4vmnmEaEAe9t.com|2452610| +9806|AAAAAAAAOEGCAAAA|1761668|4101|39781|2452300|2452270|Mr.|Joseph|Major|Y|2|8|1943|PAPUA NEW GUINEA||Joseph.Major@TNoARJ718QXbPFx.com|2452454| +9807|AAAAAAAAPEGCAAAA|505259|3515|24571|2451330||Mr.||Salter|Y||10|1926|BELARUS||Clarence.Salter@TG0PbPx8MB7.com|| +9808|AAAAAAAAAFGCAAAA|144647|3773|6852|2449795|2449765|Ms.|Elizabeth|Dabbs|Y|9|8|1932|FAROE ISLANDS||Elizabeth.Dabbs@9pI9DMlAPQ5AU.com|2452569| +9809|AAAAAAAABFGCAAAA|346666|92|6271|2449900|2449870|Miss|Ruth|Hudson|N|25|10|1941|NIGERIA||Ruth.Hudson@BxAuAfQv0LzVJZ30cRV.org|2452379| +9810|AAAAAAAACFGCAAAA|1356028|3073|12518|2452236|2452206|Sir|Brian|Campbell|Y|28|5|1974|UZBEKISTAN||Brian.Campbell@4uzvqMv1qjo5cDsccy.edu|2452377| +9811|AAAAAAAADFGCAAAA|1890643|419|17395|2452591|2452561|Mr.|Scott|Ybarra|Y|31|10|1954|SAMOA||Scott.Ybarra@YJj4rlJVnRupGhrCc.com|2452364| +9812|AAAAAAAAEFGCAAAA|1400545|6647|18396|2449318|2449288|Miss|Sherry|Clark|N|4|4|1987|PORTUGAL||Sherry.Clark@0qbArTtChcfPYQFv.com|2452582| +9813|AAAAAAAAFFGCAAAA|1573354|5568|41735|2452383|2452353|Dr.|Jackson|Miller|Y|20|11|1988|MAURITANIA||Jackson.Miller@iqvGoCNcAX4CPk.edu|2452478| +9814|AAAAAAAAGFGCAAAA|1891338|6308|21278|2449209|2449179|Miss|Leann|Mueller|N|1|11|1932|MONGOLIA||Leann.Mueller@p1A5B6JQTO5.org|2452335| +9815|AAAAAAAAHFGCAAAA|1865468|5312|14309|2449375|2449345|Sir|Peter|Womack|N|8|2|1954|ISRAEL||Peter.Womack@VfPYDC.edu|2452298| +9816|AAAAAAAAIFGCAAAA|199102|5971|46040|2451129|2451099|Ms.|Donna|Williams|N|31|12|1979|JAMAICA||Donna.Williams@E5cPDsZAAVPeF8y69.org|2452370| +9817|AAAAAAAAJFGCAAAA|188085|3789|45093|2450576|2450546|Ms.|Nancy|Stephens|Y|22|3|1931|SWAZILAND||Nancy.Stephens@sFDjZUS.org|2452567| +9818|AAAAAAAAKFGCAAAA|327210|705|45400|2452415|2452385|Sir|Christopher|Poindexter|N|9|12|1977|SPAIN||Christopher.Poindexter@hKmLPC6rrb4.edu|2452540| +9819|AAAAAAAALFGCAAAA|1606990|6992|44594|2452064|2452034|Sir|Roberto|Bell|Y|18|4|1952|UNITED KINGDOM||Roberto.Bell@muBFg7qHxy.org|2452312| +9820|AAAAAAAAMFGCAAAA|1699704|430|22756|2449988|2449958|Ms.|Georgina|Henderson|Y|3|11|1934|ALBANIA||Georgina.Henderson@BQ6ghfG.org|2452570| +9821|AAAAAAAANFGCAAAA|1841529|1292|12608|2449093|2449063|Mr.|Luis|Neal|N|10|5|1938|FIJI||Luis.Neal@y8GSxldYYf.org|2452634| +9822|AAAAAAAAOFGCAAAA|1762369|1984|7999|2450870|2450840|Sir|Timmy|Paris|N|7|7|1940|IRAQ||Timmy.Paris@Pj4c.edu|2452469| +9823|AAAAAAAAPFGCAAAA|1646986|825|49464|2451952|2451922|Miss|Kimberly|Powell|Y|25|4|1941|VIRGIN ISLANDS, U.S.||Kimberly.Powell@8T4Bzoi8iT.com|2452301| +9824|AAAAAAAAAGGCAAAA|1071334|675|44016|2450322|2450292|Dr.|Jennifer|Stephens|Y|20|10|1961|TOGO||Jennifer.Stephens@KHX.edu|2452310| +9825|AAAAAAAABGGCAAAA|605041|6619|13044|2451453|2451423|Sir|Brandon|Mcgregor|N|17|9|1942|ITALY||Brandon.Mcgregor@q8cjZixg9LgA0lkkA.com|2452432| +9826|AAAAAAAACGGCAAAA|1835493|4410|21774|2450716|2450686|Ms.|Heather|Culpepper|Y|7|3|1944|GHANA||Heather.Culpepper@sklekUCyIg.org|2452387| +9827|AAAAAAAADGGCAAAA|867158|3580|26802|2451834|2451804|Sir|Mark|Adams|N|26|11|1992|UZBEKISTAN||Mark.Adams@Yafug4L.org|2452403| +9828|AAAAAAAAEGGCAAAA|1877450|4157|21264|2451039|2451009|Sir|Sean|Hamilton|Y|11|9|1927|KUWAIT||Sean.Hamilton@TjEqOMXyAmiN5.edu|2452625| +9829|AAAAAAAAFGGCAAAA|1244292|2745|35598|2450354|2450324|Mrs.|Melanie|King|N|25|1|1941|MONGOLIA||Melanie.King@CLlJ.edu|2452338| +9830|AAAAAAAAGGGCAAAA|1772307|6126|9192|2450824|2450794|Miss|Loretta|Livingston|N|22|1|1970|SAN MARINO||Loretta.Livingston@mEciGyynaZkUe.edu|2452293| +9831|AAAAAAAAHGGCAAAA|1002380|3633|2886|2451944|2451914|Ms.|Dorothy|Roberts|N|25|9|1968|PHILIPPINES||Dorothy.Roberts@97Bjx.com|2452538| +9832|AAAAAAAAIGGCAAAA|974067|1011|402|2451526|2451496|Mrs.|Dorothy|Thomas|N|30|1|1958|FRANCE||Dorothy.Thomas@ypQhf0QnkP0bpm2.edu|2452472| +9833|AAAAAAAAJGGCAAAA|1493398|5119|22487|2451126|2451096|Dr.|Ismael|Poole|Y|30|7|1936|ROMANIA||Ismael.Poole@oZIGyGv.edu|2452547| +9834|AAAAAAAAKGGCAAAA|609970|1671|35819|2452089|2452059|Sir|David|Craig|N|7|8|1959|ALBANIA||David.Craig@e.edu|2452332| +9835|AAAAAAAALGGCAAAA|1782437|308|2579|2450968|2450938|Sir|Henry|Robinson|Y|21|4|1948|SPAIN||Henry.Robinson@C2metf6mrM.edu|2452597| +9836|AAAAAAAAMGGCAAAA|1613182|6805|12475|2452085|2452055|Mrs.|Cori|Aguirre|Y|8|10|1940|CAMBODIA||Cori.Aguirre@pJhl3vddGxII7oV2Y.org|2452461| +9837|AAAAAAAANGGCAAAA|743412|5962|7712|2450969|2450939|Dr.|Jerry|Nieves|Y|23|12|1978|CZECH REPUBLIC||Jerry.Nieves@KmJ9XuiR3.org|2452480| +9838|AAAAAAAAOGGCAAAA|426265|2128|11638|2451531|2451501|Dr.|Jon|Taylor|N|6|5|1986|SOLOMON ISLANDS||Jon.Taylor@C69J6NNbvpxGIZy.edu|2452417| +9839|AAAAAAAAPGGCAAAA|76849|6000|17151|2449803|2449773|Dr.|Dale|Mcarthur|N|1|4|1949|MEXICO||Dale.Mcarthur@6hQHfC.com|2452571| +9840|AAAAAAAAAHGCAAAA|1099121|4221|16968|2449608|2449578|Dr.|Martha|Williams|N|6|3|1937|SOMALIA||Martha.Williams@zL5VCQ1K75.edu|2452518| +9841|AAAAAAAABHGCAAAA|642059|833|42407|2450891|2450861|Sir|Joshua|Huff|N|26|7|1970|MADAGASCAR||Joshua.Huff@fNprVS7KPjeq.org|2452414| +9842|AAAAAAAACHGCAAAA|1547848|3293|47891|2449405|2449375|Ms.|Diana|Moore|N|2|5|1974|MALDIVES||Diana.Moore@R6ohLFdUNfcnmP0Rq.edu|2452538| +9843|AAAAAAAADHGCAAAA|660520|4328|17289|2452193|2452163|Dr.|Nick|Lawrence|N|16|4|1930|ARMENIA||Nick.Lawrence@rjNcd44Tsm.org|2452343| +9844|AAAAAAAAEHGCAAAA|1449485|1294|16256|2451720|2451690|Mr.|John|Boston|Y|5|2|1925|SAMOA||John.Boston@2ybJ.com|2452636| +9845|AAAAAAAAFHGCAAAA|121391|1305|16419|2450171|2450141|Mrs.|Carla|Muniz|Y|16|11|1958|BENIN||Carla.Muniz@ynRCJVnSal.edu|2452489| +9846|AAAAAAAAGHGCAAAA|606869|3780|9091|2451089|2451059|Sir|Kelly|Bennett|Y|4|5|1938|PHILIPPINES||Kelly.Bennett@1GdpbcOYf.com|2452376| +9847|AAAAAAAAHHGCAAAA|1906089|2419|1596|2452331|2452301|Dr.|Helen|Carney|Y|7|10|1945|OMAN||Helen.Carney@mA7mDVE.edu|2452602| +9848|AAAAAAAAIHGCAAAA|1896104|3640|31576|2450273|2450243|Mr.|Rey|Lee|Y|18|2|1987|GAMBIA||Rey.Lee@KKIsfeTRYQ.com|2452384| +9849|AAAAAAAAJHGCAAAA|749104|1367|36383|2451593|2451563|Ms.|Myrtle|Thompson|Y|29|12|1933|PAPUA NEW GUINEA||Myrtle.Thompson@08VXNFQPD.org|2452429| +9850|AAAAAAAAKHGCAAAA|1367824|7146|45621|2449688|2449658|Dr.|David|Kellogg|N|11|7|1982|PAKISTAN||David.Kellogg@q5flln8a4J8Ekh4LsBs.org|2452494| +9851|AAAAAAAALHGCAAAA|1002885|693|33301|2449885|2449855|Sir|Harvey|Pool|N|30|7|1947|MONACO||Harvey.Pool@Lt7sC6XOXmg33.edu|2452554| +9852|AAAAAAAAMHGCAAAA|917990|1827|42017|2451285|2451255|Mr.|Willard|Singh|N|1|4|1990|AUSTRALIA||Willard.Singh@l.com|2452341| +9853|AAAAAAAANHGCAAAA|809000|536|25625|2450125|2450095|Mr.|Charles|Beck|N|27|4|1954|MALTA||Charles.Beck@RqtxKZSSar.com|2452625| +9854|AAAAAAAAOHGCAAAA|1482550|3007|15511|2451952|2451922|Mrs.|Brittany|Jackson|N|17|4|1991|SOMALIA||Brittany.Jackson@X5sV0MEGe4ueI.com|2452444| +9855|AAAAAAAAPHGCAAAA|1882692|5671|41003|2450889|2450859|Dr.|Theo|Salter|N|6|5|1931|FRANCE||Theo.Salter@Z5vKxQM.com|2452619| +9856|AAAAAAAAAIGCAAAA|477767|1848|14376|2449754|2449724|Ms.|Vicki|Wood|Y|6|6|1981|BELARUS||Vicki.Wood@D1YCbs5Vs2.com|2452347| +9857|AAAAAAAABIGCAAAA|1626142|6121|36853|2452317|2452287|Dr.|James|Hamilton|Y|11|6|1953|OMAN||James.Hamilton@QApLZC3XtyQUEA.edu|2452498| +9858|AAAAAAAACIGCAAAA|211002|1648|39494|2451054|2451024|Dr.|Barbara|Poole|Y|27|5|1963|LIBERIA||Barbara.Poole@XzqlEroG2ak65hPJ.com|2452493| +9859|AAAAAAAADIGCAAAA|||33774|2450169|2450139|Sir|Nicholas||N|||1978|||Nicholas.Harley@c81QVDU2dEKUgSiOJ.com|2452625| +9860|AAAAAAAAEIGCAAAA|253707|2948|8589|2449691|2449661|Miss|Clara|Farmer|Y|30|11|1954|GUATEMALA||Clara.Farmer@XvtlLqioS11bZLVj.com|2452403| +9861|AAAAAAAAFIGCAAAA|1169100|1522|35097|2449926|2449896|Mr.|Timothy|Saylor|N|28|11|1979|CZECH REPUBLIC||Timothy.Saylor@gpk5SM92mzLi31.edu|2452629| +9862|AAAAAAAAGIGCAAAA|106263|2493|3123|2449167|2449137|Dr.|Steven|Warren||||1964|PORTUGAL||Steven.Warren@5KBj9LxsuBoy2r.edu|2452324| +9863|AAAAAAAAHIGCAAAA|1803983|5365|27547|2451830|2451800|Ms.|Cynthia|Keller|Y|19|11|1930|SERBIA||Cynthia.Keller@h.edu|2452450| +9864|AAAAAAAAIIGCAAAA|1362855|2800|25098|2451208|2451178|Sir|Collin|Rector|N|21|6|1972|BOTSWANA||Collin.Rector@duTEU25AAZz58u30.com|2452465| +9865|AAAAAAAAJIGCAAAA|1285786|7016|646|2452333|2452303|Miss|Victoria|Lockwood|N|12|8|1968|CZECH REPUBLIC||Victoria.Lockwood@GnxKgyRFCM.com|2452614| +9866|AAAAAAAAKIGCAAAA|1717951|6414|41324|2451100|2451070|Sir|Jeffery|Murphy|N|19|6|1974|KOREA, REPUBLIC OF||Jeffery.Murphy@UdAzVG5Y1b1CSg7.org|2452567| +9867|AAAAAAAALIGCAAAA|62212|2318|10824|2449127|2449097|Mr.|William|Horner|N|16|4|1928|CZECH REPUBLIC||William.Horner@th56FaGR2.com|2452343| +9868|AAAAAAAAMIGCAAAA|1591611|2084|32110|2449699|2449669|Mrs.|Marcie|Burdette|N|10|5|1962|MALAYSIA||Marcie.Burdette@GYjlhISNycvgqz5.edu|2452603| +9869|AAAAAAAANIGCAAAA|1284470|1879|41641|2451904|2451874|Mr.|Michael|Mueller|Y|4|8|1955|MYANMAR||Michael.Mueller@faJu.com|2452477| +9870|AAAAAAAAOIGCAAAA|1681293|57|7621|2449696|2449666|Dr.|Lucille|Pritchett|N|27|4|1929|INDONESIA||Lucille.Pritchett@3UoQtS5Oz.com|2452364| +9871|AAAAAAAAPIGCAAAA|677553|1711|41869|2449859|2449829|Sir|Joe|Gannon|N|20|5|1935|ZIMBABWE||Joe.Gannon@nHyXexFsliR.org|2452408| +9872|AAAAAAAAAJGCAAAA|548784|3879|39374|2451040|2451010|Dr.|Frank|Davis|N|20|9|1974|MONACO||Frank.Davis@1bV5ofH.edu|2452325| +9873|AAAAAAAABJGCAAAA|1567741|2024|17507|2451666|2451636|Mrs.|Lisa|Hagan|Y|6|12|1964|KIRIBATI||Lisa.Hagan@DLnhXpAViVNg.edu|2452497| +9874|AAAAAAAACJGCAAAA|1149975|389|48850|2452597|2452567|Ms.|Kimberly|Richards|N|1|1|1932|NIUE||Kimberly.Richards@zmFDe.com|2452385| +9875|AAAAAAAADJGCAAAA|1291644|3615|49586|2451666|2451636|Ms.|Nadia|Roach|Y|25|9|1936|ALGERIA||Nadia.Roach@Z8y8c.com|2452633| +9876|AAAAAAAAEJGCAAAA|1401268|102|31336|2451770|2451740|Sir|Raymond|Williams|N|18|6|1990|MALTA||Raymond.Williams@9yElnbIOfx.org|2452518| +9877|AAAAAAAAFJGCAAAA|947102|3310|7518|2451007|2450977|Dr.|Miguel|Mueller|N|26|4|1974|SUDAN||Miguel.Mueller@SOtlkmvFB93.org|2452590| +9878|AAAAAAAAGJGCAAAA|1728412|6482|48199|2449905|2449875|Miss|Dusti|Wilder|Y|5|11|1950|THAILAND||Dusti.Wilder@zoPGeEADzDT6tU5Xr92.com|2452543| +9879|AAAAAAAAHJGCAAAA|1800482|88|8541|2452411|2452381|Mrs.|Grace|Watson|Y|5|1|1941|MACAO||Grace.Watson@X2DOIfKLYgn3.org|2452635| +9880|AAAAAAAAIJGCAAAA|1729332|6931|38483|2452124|2452094|Ms.|Cynthia|Bryant|N|19|6|1947|MARTINIQUE||Cynthia.Bryant@88cV.com|2452586| +9881|AAAAAAAAJJGCAAAA|1549959|3118|49030|2451729|2451699|Dr.|Branden|Briggs|N|29|6|1967|ISLE OF MAN||Branden.Briggs@Jri2SYSIUKq.edu|2452292| +9882|AAAAAAAAKJGCAAAA|629052|6295|7585|2452623|2452593|Dr.|Barbara|Huff|N|4|12|1957|GUADELOUPE||Barbara.Huff@mLNA.edu|2452321| +9883|AAAAAAAALJGCAAAA|1476750|3013|18876|2450300|2450270|Miss|Lan|Doyle|Y|17|5|1946|TUNISIA||Lan.Doyle@RugTYmqeu2Uy.edu|2452522| +9884|AAAAAAAAMJGCAAAA|1422280||5845|2452311||Mrs.|||Y|10|9|1927||||| +9885|AAAAAAAANJGCAAAA|238380|1527|8274|2449106|2449076|Dr.|Ethan|Barrios|N|14|4|1929|CZECH REPUBLIC||Ethan.Barrios@T8fqjrCYGoemVN.org|2452558| +9886|AAAAAAAAOJGCAAAA|288048|3113|4178|2450751|2450721|Ms.|Annie|Austin|Y|20|12|1940|GUINEA-BISSAU||Annie.Austin@MCo.com|2452528| +9887|AAAAAAAAPJGCAAAA|14950|1481|13212|2450619|2450589|Mr.|Steven|Arroyo|Y|28|10|1949|VIET NAM||Steven.Arroyo@NbD0KskDth.edu|2452381| +9888|AAAAAAAAAKGCAAAA|668047|2718|38236|2449198|2449168|Sir|Bob|Fernandez|N|29|9|1968|IRELAND||Bob.Fernandez@uI2Z.com|2452349| +9889|AAAAAAAABKGCAAAA|569790|4370|16150|2449307|2449277|Dr.|Paul|Gonzalez|Y|15|11|1928|MALTA||Paul.Gonzalez@DR.edu|2452444| +9890|AAAAAAAACKGCAAAA|1563886|2519|48705|2451355|2451325|Sir|Bob|Rodriguez|Y|21|4|1986|GUAM||Bob.Rodriguez@mlOXeIop.edu|2452380| +9891|AAAAAAAADKGCAAAA|1737891|3144|7597|2449300|2449270|Dr.|Perry|Fernandez|Y|21|2|1955|NAMIBIA||Perry.Fernandez@CRupNLGfOMbhfS4.org|2452521| +9892|AAAAAAAAEKGCAAAA|83103|1442|19781|2451908|2451878|Miss|Annie|Mccloud|Y|20|12|1933|AFGHANISTAN||Annie.Mccloud@ZMR.com|2452510| +9893|AAAAAAAAFKGCAAAA|744021|5452|14036|2449408|2449378|Dr.|William|Newell|N|27|4|1954|BELGIUM||William.Newell@6N8A3TtnomI.edu|2452550| +9894|AAAAAAAAGKGCAAAA|611613|2861|26909|2452221|2452191|Ms.|Nicole|Morrison|Y|7|5|1990|ALAND ISLANDS||Nicole.Morrison@dKga.com|2452315| +9895|AAAAAAAAHKGCAAAA|479947|7032|7670|2450513|2450483|Dr.|Brian|Davies|Y|11|11|1953|ITALY||Brian.Davies@It9TIpjMxTy8D.com|2452632| +9896|AAAAAAAAIKGCAAAA|137295|1814|35933|2450957|2450927|Mrs.|Dawn|Smith|N|4|8|1932|NEW ZEALAND||Dawn.Smith@dYhK2r0.com|2452492| +9897|AAAAAAAAJKGCAAAA|86999|5901|2803|2450110|2450080|Miss|Deborah|Woodward|N|12|2|1960|BELIZE||Deborah.Woodward@X6M9Z3y7RlrvP.org|2452620| +9898|AAAAAAAAKKGCAAAA|800560|5262|32011|2451302|2451272|Dr.|Jayme|Mendoza|Y|26|8|1954|MOLDOVA, REPUBLIC OF||Jayme.Mendoza@GB88YznZ34.edu|2452535| +9899|AAAAAAAALKGCAAAA|461965|3083|15401|2449652||Miss|Deanna|Chang|Y|19|8|1989|NEW ZEALAND||Deanna.Chang@t7.org|2452435| +9900|AAAAAAAAMKGCAAAA|862181|1854|6727|2451053|2451023|Dr.|Inez|Escobar|Y|8|5|1938|BELIZE||Inez.Escobar@lhiZqrBebyODT.edu|2452373| +9901|AAAAAAAANKGCAAAA|1091080|6462|24197|2452282|2452252|Mrs.|Melinda|Clifton|N|28|6|1974|SWEDEN||Melinda.Clifton@SPcCbueuDN.org|2452596| +9902|AAAAAAAAOKGCAAAA|158528|3311|42094|2450037|2450007|Mr.|Gary|Dotson|Y|8|3|1961|SERBIA||Gary.Dotson@parp5AAdysFcsP1.edu|2452306| +9903|AAAAAAAAPKGCAAAA|121765|5340|27315|2452531|2452501|Dr.|Joseph|Watson|Y|30|1|1978|BELGIUM||Joseph.Watson@iUbqR3tiB.org|2452333| +9904|AAAAAAAAALGCAAAA|114007|5893|49426|2452398|2452368|Mr.|Ryan|Padgett|N|10|7|1980|WESTERN SAHARA||Ryan.Padgett@HM.com|2452584| +9905|AAAAAAAABLGCAAAA|836552|1920|14088|2450614|2450584|Mr.|Mark|Meyers|N|12|2|1976|KENYA||Mark.Meyers@0XUJrSt2Kq.edu|2452472| +9906|AAAAAAAACLGCAAAA|184107|2050|39146|2450118|2450088|Mr.|John|Roberts|N|18|12|1924|IRELAND||John.Roberts@cMZFkbxAf6nBClrea7.org|2452530| +9907|AAAAAAAADLGCAAAA|647841|490|25139|2452194|2452164|Mr.|Hans|Bonner|N|13|4|1925|AZERBAIJAN||Hans.Bonner@JZcrzbMNGLVhiu.org|2452492| +9908|AAAAAAAAELGCAAAA|926570|5787|9804|2452314|2452284|Mrs.|Carolyn|Davis|N|1|12|1970|YEMEN||Carolyn.Davis@e9rGcnd.com|2452488| +9909|AAAAAAAAFLGCAAAA|1083472|3439|35406|2449431|2449401|Mr.|Roberto|Lai|Y|30|7|1936|AUSTRIA||Roberto.Lai@ggbguyxk.com|2452532| +9910|AAAAAAAAGLGCAAAA|1281912||10353||2452017|||Mullins||9|6|||||2452623| +9911|AAAAAAAAHLGCAAAA|1402814|3416|32418|2451746|2451716|Ms.|Lois|Holcomb|N|28|8|1985|CHILE||Lois.Holcomb@j0lMRblEaI.edu|2452586| +9912|AAAAAAAAILGCAAAA|1676963|3273|3183|2449166|2449136|Dr.|Erma|Mitchell|Y|4|3|1988|NORWAY||Erma.Mitchell@l.org|2452364| +9913|AAAAAAAAJLGCAAAA|||19676||||Jane||N|17|3||SYRIAN ARAB REPUBLIC|||| +9914|AAAAAAAAKLGCAAAA|98137|2403|41069|2449184|2449154|Dr.|Vivien|Hernandez|N|25|3|1943|HUNGARY||Vivien.Hernandez@pcolTp0.com|2452420| +9915|AAAAAAAALLGCAAAA|1486801|1440|47679|2451701|2451671|Dr.|Clint|Booth|N|16|2|1966|GUYANA||Clint.Booth@m4DFaKQ8m20P7q.com|2452539| +9916|AAAAAAAAMLGCAAAA|1789343|4931|34519|2449403|2449373|Dr.|Bonnie|Ryan|N|2|6|1926|ALGERIA||Bonnie.Ryan@UF3RAym.org|2452406| +9917|AAAAAAAANLGCAAAA|700469|460|16851|2452160|2452130|Dr.|Virgilio|Alston|N|24|5|1968|URUGUAY||Virgilio.Alston@Fl5z85G1yV.org|2452296| +9918|AAAAAAAAOLGCAAAA|1025977|6805|14833|2452145|2452115|Dr.|June|Dias|N|24|5|1947|BENIN||June.Dias@AOMNmmfyAd.com|2452286| +9919|AAAAAAAAPLGCAAAA|1702374|3820|24066|2449972|2449942|Miss|Renate|Kelly|Y|9|1|1949|AFGHANISTAN||Renate.Kelly@eN3Jf.com|2452346| +9920|AAAAAAAAAMGCAAAA|1418761|508|18655|2451320|2451290|Mr.|Martin|Mack|N|9|2|1981|AFGHANISTAN||Martin.Mack@J1CioPpPsyY2oL.org|2452530| +9921|AAAAAAAABMGCAAAA|120442|6110|25378|2451275|2451245|Mr.|Mark|Pope|Y|19|11|1932|WESTERN SAHARA||Mark.Pope@nlFsp4YUbhQkZZ.com|2452440| +9922|AAAAAAAACMGCAAAA|695845|208|13818|2451313|2451283|Dr.|James|Segura|N|6|7|1927|ITALY||James.Segura@UYXbtje440kFb.com|2452474| +9923|AAAAAAAADMGCAAAA|1626117|1650|7583|2449361|2449331|Miss|Denise|Etheridge|N|9|5|1989|SAMOA||Denise.Etheridge@vhy.edu|2452515| +9924|AAAAAAAAEMGCAAAA|1188190|2540|17243|2452326|2452296|Dr.|Jonathon|Sherman|N|18|2|1991|RWANDA||Jonathon.Sherman@3qQSmjJGEPEZe2p8qc.com|2452639| +9925|AAAAAAAAFMGCAAAA|263549|4012|28122|2451792|2451762|Sir|Steven|Smith|Y|29|8|1957|MONGOLIA||Steven.Smith@M9RumyOrP.com|2452375| +9926|AAAAAAAAGMGCAAAA|585642||24342|||Dr.|Sabrina||Y||8||||Sabrina.Gilbert@i.com|| +9927|AAAAAAAAHMGCAAAA|841906|3385|10064|2449920|2449890|Miss|Ashley|Buck|Y|10|11|1971|KIRIBATI||Ashley.Buck@zktBrRvSOUmTY.edu|2452529| +9928|AAAAAAAAIMGCAAAA|1448948|6966|30137|2451442|2451412|Dr.|William|Reed|Y|16|1|1957|KUWAIT||William.Reed@Z4LB32NQoIdott.org|2452506| +9929|AAAAAAAAJMGCAAAA|1290638|5583|21681|2449035|2449005|Dr.|Keri|Hickson|Y|9|12|1934|PHILIPPINES||Keri.Hickson@PVoHtzPe3zPt.edu|2452396| +9930|AAAAAAAAKMGCAAAA|1132398|3080|3762|2451217|2451187|Sir|Lawrence|Slater|Y|11|3|1945|MONTENEGRO||Lawrence.Slater@AfspKCMjRsx7V5mk.com|2452599| +9931|AAAAAAAALMGCAAAA|1337746|1511|923|2450497|2450467|Ms.|Dorothy|Quick|N|6|5|1980|ICELAND||Dorothy.Quick@sn.edu|2452331| +9932|AAAAAAAAMMGCAAAA|1120291|4087|26330|2450126|2450096|Ms.|Kathryn|Martinez|N|22|9|1986|BOTSWANA||Kathryn.Martinez@N0QH.com|2452437| +9933|AAAAAAAANMGCAAAA|1009026|5568|20910|2450343|2450313|Mrs.|Amy|Johnston|N|14|6|1925|GUAM||Amy.Johnston@oqeD4sBhZtQ5.org|2452386| +9934|AAAAAAAAOMGCAAAA|576524|593|34615|2449495|2449465|Sir|William|Sims|N|4|3|1960|UNITED STATES||William.Sims@EqSKcHKm.com|2452305| +9935|AAAAAAAAPMGCAAAA|681016|249|8282|2451244|2451214|Miss|Jenny|Powell|N|22|12|1986|ALAND ISLANDS||Jenny.Powell@Us1NtLjvJxzrPa2l3.org|2452490| +9936|AAAAAAAAANGCAAAA|477060|5141|36775|2451596|2451566|Ms.|Heidi|Montoya|Y|10|1|1938|BARBADOS||Heidi.Montoya@lZ58cfimcaA2Oxf.com|2452497| +9937|AAAAAAAABNGCAAAA|1780412|7180|6701|2452363|2452333|Dr.|Cheryl|Scott|Y|24|4|1933|GUATEMALA||Cheryl.Scott@hPHosVTIC7urCU.edu|2452578| +9938|AAAAAAAACNGCAAAA|1660523|3899|24828|2451533|2451503|Miss|Barbara|Norris|N|15|6|1981|CZECH REPUBLIC||Barbara.Norris@ZT5zR00l38Rv7ymGIj3.org|2452327| +9939|AAAAAAAADNGCAAAA|1066519|4602|24685|2451280|2451250|Miss|Christine|Sanchez|Y|24|12|1973|CAPE VERDE||Christine.Sanchez@7fT9zfjLq4zX6n.edu|2452443| +9940|AAAAAAAAENGCAAAA|953089|1322|9555|2452163|2452133|Mrs.|Geneva|Lindsay|N|7|8|1956|CHILE||Geneva.Lindsay@J.com|2452296| +9941|AAAAAAAAFNGCAAAA|785589|6959|36729|2451917|2451887|Dr.|Martha|Packer|Y|6|9|1985|PITCAIRN||Martha.Packer@ojRJjIo2BuiZdE.edu|2452590| +9942|AAAAAAAAGNGCAAAA|582582|6105|47628|2452379|2452349|Dr.|Rudy|Staples|N|29|7|1969|PORTUGAL||Rudy.Staples@q1r2CFOc8KTtn.org|2452455| +9943|AAAAAAAAHNGCAAAA|234139|6162|41785|2450254|2450224|Dr.|Robert|Massey|Y|1|9|1971|ESTONIA||Robert.Massey@lGcX5E.org|2452311| +9944|AAAAAAAAINGCAAAA|1866671|6027|19383|2451043|2451013|Ms.|Deborah|Brothers|N|22|5|1978|SLOVENIA||Deborah.Brothers@4UDM6TViIshNs.edu|2452488| +9945|AAAAAAAAJNGCAAAA|820341|6927|17913|2451425|2451395|Dr.|Alexander|Arnold|N|19|8|1962|AMERICAN SAMOA||Alexander.Arnold@iulPi9Xs9hU.org|2452622| +9946|AAAAAAAAKNGCAAAA|1828759|1480|14248|2451461|2451431|Sir|Robert|Marks|N|9|4|1947|CAPE VERDE||Robert.Marks@iLeBotmSmseToNoC.org|2452516| +9947|AAAAAAAALNGCAAAA|1092777|4852|3727|2451748|2451718|Dr.|Harry|Myers|Y|14|11|1928|GRENADA||Harry.Myers@mg4V.org|2452578| +9948|AAAAAAAAMNGCAAAA|1713663|1321|11779|2451829|2451799|Mr.|John|Rosales|Y|12|1|1989|NEW ZEALAND||John.Rosales@VOsd6vb54ig.edu|2452437| +9949|AAAAAAAANNGCAAAA|952595|6581|31367|2449953|2449923|Mrs.|Julia|Murray|N|30|7|1950|QATAR||Julia.Murray@4SrV.edu|2452540| +9950|AAAAAAAAONGCAAAA|228809|6605|23601|2451052|2451022|Dr.|Viola|Malone|N|30|7|1990|GIBRALTAR||Viola.Malone@feNI2SX3VIXbCG.edu|2452316| +9951|AAAAAAAAPNGCAAAA|485204|7121|31438|2451375|2451345|Mr.|Robert|Rankin|N|4|4|1939|CHRISTMAS ISLAND||Robert.Rankin@q3Jh5HDls9fd.org|2452392| +9952|AAAAAAAAAOGCAAAA|1021327||9622|2449101||||||6||1958|WESTERN SAHARA|||2452434| +9953|AAAAAAAABOGCAAAA|413425|1791|23357|2449519|2449489|Dr.|David|Hill|Y|10|8|1962|TIMOR-LESTE||David.Hill@P7aHJe3BaO.edu|2452292| +9954|AAAAAAAACOGCAAAA|1732473|4705|11737|2450157|2450127|Mr.|Arnold|Bell|Y|30|6|1953|TOGO||Arnold.Bell@p8Ns3FP5ZdjOjLg7mUz.com|2452471| +9955|AAAAAAAADOGCAAAA|492225|6782|44892|2449649|2449619|Dr.|Gretchen|Cole|N|10|6|1975|KIRIBATI||Gretchen.Cole@8PLM.com|2452313| +9956|AAAAAAAAEOGCAAAA|1660860|2388|15869|2449927|2449897|Dr.|Deborah|Gonzalez|N|13|10|1931|BOUVET ISLAND||Deborah.Gonzalez@6.com|2452316| +9957|AAAAAAAAFOGCAAAA|618072|1580|32735|2450876|2450846|Mr.|James|Johns|Y|18|2|1978|SLOVAKIA||James.Johns@vJnEpABLHbT.edu|2452583| +9958|AAAAAAAAGOGCAAAA|83744|813|41482|2449228|2449198|Mr.|Chester|Lopez|Y|22|6|1961|HONDURAS||Chester.Lopez@97s.edu|2452301| +9959|AAAAAAAAHOGCAAAA|1385380|1897|9614|2452062|2452032|Sir|Isidro|Gee|N|27|7|1958|MARTINIQUE||Isidro.Gee@LSxGgh.org|2452362| +9960|AAAAAAAAIOGCAAAA|833542|5488|634|2450789|2450759|Dr.|Ann|Thompson|Y|15|1|1967|NIGER||Ann.Thompson@kpdjrooTlsAHgFhs7i.org|2452532| +9961|AAAAAAAAJOGCAAAA|902688|3448|33761|2450752|2450722|Dr.|Terry|Epperson|N|12|7|1924|BAHAMAS||Terry.Epperson@njAkbBf8ebTNGR2d.edu|2452601| +9962|AAAAAAAAKOGCAAAA|1324868|5817|45519|2449409|2449379|Mr.|James|Mancuso|Y|22|1|1975|TONGA||James.Mancuso@LCupzZ9zBz.com|2452286| +9963|AAAAAAAALOGCAAAA|718356|5160|32411|2452457|2452427|Miss|Martha|Strong|N|23|2|1936|MONGOLIA||Martha.Strong@c7r6AUF.edu|2452480| +9964|AAAAAAAAMOGCAAAA|723472|1870|33115|2452384|2452354|Sir|Henry|Fry|Y|4|8|1945|GUYANA||Henry.Fry@OkVdrgrUCN.edu|2452457| +9965|AAAAAAAANOGCAAAA|111160|3372|47984|2451932|2451902|Miss|Norma|Arredondo|Y|18|1|1987|ALGERIA||Norma.Arredondo@e.com|2452359| +9966|AAAAAAAAOOGCAAAA|1187082|2758|43281|2451430|2451400|Sir|Mario|Stewart|N|11|3|1938|AZERBAIJAN||Mario.Stewart@7oQgNK.com|2452643| +9967|AAAAAAAAPOGCAAAA|1412889|6208|49527|2452599|2452569|Mr.|Walter|Wilson|Y|11|7|1961|SEYCHELLES||Walter.Wilson@rfXfM.com|2452603| +9968|AAAAAAAAAPGCAAAA|1176435|5430|28472|2449178|2449148|Ms.|Nancy|Washington|Y|22|1|1976|LIECHTENSTEIN||Nancy.Washington@3dhei6JeTey.com|2452368| +9969|AAAAAAAABPGCAAAA|687033|3853|47026|2451494|2451464|Dr.|Nancy|Bergeron|N|23|9|1959|R�UNION||Nancy.Bergeron@ZvIMNQvej3.edu|2452479| +9970|AAAAAAAACPGCAAAA|1597299|4790|41434|2450554|2450524|Dr.|Leroy|Pearson|N|4|8|1927|PARAGUAY||Leroy.Pearson@rCMgyapKv5TR.com|2452643| +9971|AAAAAAAADPGCAAAA|1655805|3951|28897|2452462|2452432|Sir|Ellis|Castillo|Y|1|9|1947|UNITED ARAB EMIRATES||Ellis.Castillo@FPOPaLFC.org|2452553| +9972|AAAAAAAAEPGCAAAA|102186|1717|18801|2449531|2449501|Dr.|Edwina|Parsons|Y|24|4|1986|WESTERN SAHARA||Edwina.Parsons@s3pTS5F5ubNZ.edu|2452370| +9973|AAAAAAAAFPGCAAAA|1499579|504|15876|2449428|2449398|Ms.|Sheri|Perez|N|2|12|1983|TONGA||Sheri.Perez@Z08Ms.com|2452283| +9974|AAAAAAAAGPGCAAAA|599801|6650|9946|2449669|2449639|Mr.|Charles|Combs|Y|4|5|1988|BURKINA FASO||Charles.Combs@K1cx.org|2452403| +9975|AAAAAAAAHPGCAAAA|699140|3073|29453|2450311|2450281|Dr.|Paul|Roth|N|21|7|1974|MOROCCO||Paul.Roth@eUsp9eBiJ4qyGgrFZ.com|2452287| +9976|AAAAAAAAIPGCAAAA|387915|1363|44411|2450142|2450112|||Johnson|N||10|1952||||| +9977|AAAAAAAAJPGCAAAA|1723432|3884|3089|2449883|2449853|Dr.|Frederick|Still|N|12|9|1975|AFGHANISTAN||Frederick.Still@BYSnkozmqC28ZV.org|2452347| +9978|AAAAAAAAKPGCAAAA|1810589|6616|13443|2452521|2452491|Sir|Collin|Gresham|N|28|1|1990|KYRGYZSTAN||Collin.Gresham@JO72vxmttBx.com|2452571| +9979|AAAAAAAALPGCAAAA||2953|32559||2450260|Mrs.|Karena||||8|1956|MOZAMBIQUE|||2452580| +9980|AAAAAAAAMPGCAAAA|971161|5330|43499|2451058|2451028|Ms.|Ruth|Shaffer|Y|13|4|1935|AZERBAIJAN||Ruth.Shaffer@MehFrR8caqM.edu|2452407| +9981|AAAAAAAANPGCAAAA|1568277|1358|31592|2449568|2449538|Dr.|Norris|Garcia|N|24|4|1989|BRUNEI DARUSSALAM||Norris.Garcia@r3r8.edu|2452311| +9982|AAAAAAAAOPGCAAAA|1888688|4778|32515|2452405|2452375|Dr.|Jacquelyn|Ryan|N|22|6|1942|BELARUS||Jacquelyn.Ryan@b8D77i6F.org|2452567| +9983|AAAAAAAAPPGCAAAA|1524453|3668|2436|2451446|2451416|Mrs.|Arlyne|Burnett|N|25|3|1955|SAN MARINO||Arlyne.Burnett@CUerZpQhJE3MS.com|2452486| +9984|AAAAAAAAAAHCAAAA|1663984|1894|22604|2452678|2452648|Dr.|Brandon|Okeefe|Y|19|10|1992|PALAU||Brandon.Okeefe@nhtEb1JO.org|2452432| +9985|AAAAAAAABAHCAAAA|1660145||36726|2451189|2451159|Sir||||31|7|1967|ECUADOR|||2452516| +9986|AAAAAAAACAHCAAAA|205502|314|40726|2451449|2451419|Mr.|Carlos|Baker|N|20|2|1932|MOZAMBIQUE||Carlos.Baker@xN8UX6zRBTTZmBb.edu|2452313| +9987|AAAAAAAADAHCAAAA|638732|3239|28946|2449438|2449408|Mr.|Everett|Colon|N|19|7|1961|SLOVENIA||Everett.Colon@fvOJL4usrC.org|2452489| +9988|AAAAAAAAEAHCAAAA|426761|2086|13458|2449750|2449720|Dr.|Berry|Torres|N|20|12|1950|COSTA RICA||Berry.Torres@d.com|2452528| +9989|AAAAAAAAFAHCAAAA|321728|4789|38603|2451012|2450982|Ms.|Anna|Koch|Y|8|3|1985|BAHAMAS||Anna.Koch@M0.com|2452327| +9990|AAAAAAAAGAHCAAAA|1651370|4114|23658|2450507|2450477|Mrs.|Rachel|Hogan|N|24|8|1924|PARAGUAY||Rachel.Hogan@Insm0ybo5N.org|2452432| +9991|AAAAAAAAHAHCAAAA|1892669|907|40345|2450048|2450018|Ms.|Anna|Lawson|Y|5|12|1952|ARGENTINA||Anna.Lawson@AZ.com|2452639| +9992|AAAAAAAAIAHCAAAA|70894|685|5676|2452484|2452454|Sir|Frank|Reeder|Y|22|7|1927|NETHERLANDS ANTILLES||Frank.Reeder@8Ac0uHSsKrC6Ycudsrp.edu|2452594| +9993|AAAAAAAAJAHCAAAA|683376|3107|36838|2449996|2449966|Dr.|Irene|Nowell|N|1|5|1992|FINLAND||Irene.Nowell@bBc0DnPABs5.edu|2452348| +9994|AAAAAAAAKAHCAAAA|1021560|1350|740|2450521|2450491|Sir|Dale|Gilbert|N|22|3|1951|MACAO||Dale.Gilbert@lAk7D2YAAR.org|2452508| +9995|AAAAAAAALAHCAAAA|13583|3049|3715|2451678|2451648|Mr.|Wayne|Carpenter|Y|6|10|1945|SWEDEN||Wayne.Carpenter@hHbSTskHfS6Zc5MPQOU.edu|2452625| +9996|AAAAAAAAMAHCAAAA|987569|397|47672|2451318|2451288|Dr.|Kathy|Murray|Y|16|6|1991|CHINA||Kathy.Murray@0QZ2cqXr9klxA7C1Mr.com|2452418| +9997|AAAAAAAANAHCAAAA||3910|26828|||||Herrera|N|7||1991|KYRGYZSTAN||Rachel.Herrera@S54Sy3c8uXtSd.com|2452615| +9998|AAAAAAAAOAHCAAAA|1244968|3170|17725|2449425|2449395|Mr.|William|Hernandez|Y|28|2|1975|YEMEN||William.Hernandez@GAmrXUhzn6E1BCMt.edu|2452451| +9999|AAAAAAAAPAHCAAAA|1222376|3104|21079|2451934|2451904|Mr.|Daniel|Hicks|N|20|5|1989|SIERRA LEONE||Daniel.Hicks@Xjk3lZhPSCLY.edu|2452517| +10000|AAAAAAAAABHCAAAA|727521|4846|1371|2449447|2449417|Ms.|Susan|Barr|Y|6|3|1947|MALTA||Susan.Barr@R.org|2452617| diff --git a/spark-connector/spark-common/src/test/resources/data/tpcds/customer_address/customer_address.dat b/spark-connector/spark-common/src/test/resources/data/tpcds/customer_address/customer_address.dat new file mode 100644 index 00000000000..2490714487f --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/data/tpcds/customer_address/customer_address.dat @@ -0,0 +1,50000 @@ +1|AAAAAAAABAAAAAAA|18|Jackson |Parkway|Suite 280|Fairfield|Maricopa County|AZ|86192|United States|-7|condo| +2|AAAAAAAACAAAAAAA|362|Washington 6th|RD|Suite 80|Fairview|Taos County|NM|85709|United States|-7|condo| +3|AAAAAAAADAAAAAAA|585|Dogwood Washington|Circle|Suite Q|Pleasant Valley|York County|PA|12477|United States|-5|single family| +4|AAAAAAAAEAAAAAAA|111|Smith |Wy|Suite A|Oak Ridge|Kit Carson County|CO|88371|United States|-7|condo| +5|AAAAAAAAFAAAAAAA|31|College |Blvd|Suite 180|Glendale|Barry County|MO|63951|United States|-6|single family| +6|AAAAAAAAGAAAAAAA|59|Williams Sixth|Parkway|Suite 100|Lakeview|Chelan County|WA|98579|United States|-8|single family| +7|AAAAAAAAHAAAAAAA||Hill 7th|Road|Suite U|Farmington|||39145|United States||| +8|AAAAAAAAIAAAAAAA|875|Lincoln |Ct.|Suite Y|Union|Bledsoe County|TN|38721|United States|-5|apartment| +9|AAAAAAAAJAAAAAAA|819|1st Laurel|Ave|Suite 70|New Hope|Perry County|AL|39431|United States|-6|condo| +10|AAAAAAAAKAAAAAAA|851|Woodland Poplar|ST|Suite Y|Martinsville|Haines Borough|AK|90419|United States|-9|condo| +11|AAAAAAAALAAAAAAA|189|13th 2nd|Street|Suite 470|Maple Grove|Madison County|MT|68252|United States|-7|single family| +12|AAAAAAAAMAAAAAAA|76|Ash 8th|Ct.|Suite O|Edgewood|Mifflin County|PA|10069|United States|-5|apartment| +13|AAAAAAAANAAAAAAA|424|Main Second|Ln|Suite 130|Greenville|Noxubee County|MS|51387|United States|-6|single family| +14|AAAAAAAAOAAAAAAA|923|Pine Oak|Dr.|Suite 100||Lipscomb County|TX|77752||-6|| +15|AAAAAAAAPAAAAAAA|314|Spring |Ct.|Suite B|Oakland|Washington County|OH|49843|United States|-5|apartment| +16|AAAAAAAAABAAAAAA|576|Adams Center|Street|Suite J|Valley View|Oldham County|TX|75124|United States|-6|condo| +17|AAAAAAAABBAAAAAA|801|Green |Dr.|Suite 0|Montpelier|Richland County|OH|48930|United States|-5|single family| +18|AAAAAAAACBAAAAAA|460|Maple Spruce|Court|Suite 480|Somerville|Potter County|SD|57783|United States|-7|condo| +19|AAAAAAAADBAAAAAA|611|Wilson |Way|Suite O|Oakdale|Tangipahoa Parish|LA|79584|United States|-6|apartment| +20|AAAAAAAAEBAAAAAA|675|Elm Wilson|Street|Suite I|Hopewell|Williams County|OH|40587|United States|-5|condo| +21|AAAAAAAAFBAAAAAA|294|Jefferson Smith|Ave|Suite 390|Springfield|Marshall County|SD|59303|United States|-7|condo| +22|AAAAAAAAGBAAAAAA|874|Park Second|Avenue|Suite N|Green Acres|Perkins County|NE|67683|United States|-7|single family| +23|AAAAAAAAHBAAAAAA|230|6th Spring|Drive|Suite U|Newtown|Jewell County|KS|61749|United States|-6|single family| +24|AAAAAAAAIBAAAAAA|837|4th |Street|Suite 200|Fairfield|Marin County|CA|96192|United States|-8|single family| +25|AAAAAAAAJBAAAAAA|986|West East|Blvd|Suite F|Pleasant Grove|Coweta County|GA|34136|United States|-5|apartment| +26|AAAAAAAAKBAAAAAA|809|Central |||Springdale|Forsyth County||28883||-5|| +27|AAAAAAAALBAAAAAA|649|Williams |Ct.|Suite 400|Cedar|Utah County|UT|81229|United States|-7|condo| +28|AAAAAAAAMBAAAAAA|992|Birch |Avenue|Suite N|Oak Hill|Montrose County|CO|87838|United States|-7|condo| +29|AAAAAAAANBAAAAAA|572|First Elevnth|Pkwy|Suite 160|Newport|Richland County|IL|61521|United States|-6|single family| +30|AAAAAAAAOBAAAAAA|748|Oak |Avenue|Suite M|Wildwood|Clark County|MO|66871|United States|-6|apartment| +31|AAAAAAAAPBAAAAAA|291|East |Lane|Suite A|Forestville|Walworth County|SD|53027|United States|-7|condo| +32|AAAAAAAAACAAAAAA|243|8th Lake|Wy|Suite G|Oakland|Madison County|NC|29843|United States|-5|condo| +33|AAAAAAAABCAAAAAA|711|3rd 4th|Ave|Suite 10|Pine Grove|Sumter County|GA|34593|United States|-5|apartment| +34|AAAAAAAACCAAAAAA|785|1st Cedar|Ave|Suite 40|Greenville|Russell County|VA|21387|United States|-5|condo| +35|AAAAAAAADCAAAAAA|814|Washington Main|Drive|Suite C|Waterloo|Henry County|VA|21675|United States|-5|single family| +36|AAAAAAAAECAAAAAA|648|South 15th|Court|Suite 410|Lakeview|Lewis County|TN|38579|United States|-6|condo| +37|AAAAAAAAFCAAAAAA|999|4th |Ct.|Suite 250|Sumner|Greenville County|SC|20519|United States|-5|single family| +38|AAAAAAAAGCAAAAAA|912|Second Hickory|Drive|Suite 190|Plainview|Baca County|CO|83683|United States|-7|condo| +39|AAAAAAAAHCAAAAAA|13|10th |Ct.|Suite 280|Hamilton|Lee County|IA|52808|United States|-6|apartment| +40|AAAAAAAAICAAAAAA|369|2nd Maple|Pkwy|Suite H|Bridgeport|Ferry County|WA|95817|United States|-8|single family| +41|AAAAAAAAJCAAAAAA|198|Washington |RD|Suite S|Oakland|Ouray County|CO|89843|United States|-7|apartment| +42|AAAAAAAAKCAAAAAA|884|Oak |Court|Suite F|Riverside|Scott County|MS|59231|United States|-6|single family| +43|AAAAAAAALCAAAAAA|762|Spring |Dr.|Suite F|Sulphur Springs|Jefferson County|IL|68354|United States|-6|condo| +44|AAAAAAAAMCAAAAAA||||Suite 490||O-Brien County||58721|United States|-6|| +45|AAAAAAAANCAAAAAA|206|4th |Street|Suite 120|Union|Washington County|ME|09321|United States|-5|apartment| +46|AAAAAAAAOCAAAAAA|995|Oak |Boulevard|Suite 120|Union Hill|Forrest County|MS|57746|United States|-6|single family| +47|AAAAAAAAPCAAAAAA|776|Washington View|Wy|Suite 60|Pleasant Grove|Mercer County|KY|44136|United States|-5|apartment| +48|AAAAAAAAADAAAAAA|895|Valley |Lane|Suite 480|Salem|James City County|VA|28048|United States|-5|condo| +49|AAAAAAAABDAAAAAA|583|Highland 2nd|Street|Suite 90|Fairfield|York County|ME|06792|United States|-5|apartment| +50|AAAAAAAACDAAAAAA|190|7th |Wy|Suite D|Enterprise|Gray County|KS|61757|United States|-6|single family| +51|AAAAAAAADDAAAAAA|845|5th |Pkwy|Suite F|Green Acres|Hamblen County|TN|37683|United States|-5|single family| +52|AAAAAAAAEDAAAAAA|472|Ridge Park|ST|Suite 60|Oakdale|Muskingum County|OH|49584|United States|-5|single family| +53|AAAAAAAAFDAAAAAA|884|College Franklin|Boulevard|Suite 440|Riverview|McPherson County|SD|59003|United States|-7|single family| +54|AAAAAAAAGDAAAAAA|857|10th 10th|Boulevard|Suite V|Oak Hill|Martin County|NC|27838|United States|-5|single family| +55|AAAAAAAAHDAAAAAA|177|Lee Adams|Way|Suite S|Ashland|Macomb County|MI|44244|United States|-5|condo| +56|AAAAAAAAIDAAAAAA|49|Valley 14th|Wy|Suite V|Woodland|Goodhue County|MN|54854|United States|-6|condo| +57|AAAAAAAAJDAAAAAA|625|10th Oak|RD|Suite J|Enterprise|Bottineau County|ND|51757|United States|-6|single family| +58|AAAAAAAAKDAAAAAA|917|Miller 15th|Parkway|Suite 230|Union Hill|Canyon County|ID|87746|United States|-7|single family| +59|AAAAAAAALDAAAAAA|||Court||Newtown|Valley County|MT||United States|-7|condo| +60|AAAAAAAAMDAAAAAA||Lee |||Forest Hills|Green County|KY|49237|||apartment| +61|AAAAAAAANDAAAAAA|159|Ridge |Boulevard|Suite 20|Sulphur Springs|Laurel County|KY|48354|United States|-5|single family| +62|AAAAAAAAODAAAAAA|925|Lincoln |Lane|Suite 90|Riverdale|Lewis County|WV|29391|United States|-5|condo| +63|AAAAAAAAPDAAAAAA|416|Elevnth Green|Drive|Suite A|Midway|Jewell County|KS|61904|United States|-6|single family| +64|AAAAAAAAAEAAAAAA|546|Meadow |Circle|Suite 90|Macedonia|Trousdale County|TN|31087|United States|-6|apartment| +65|AAAAAAAABEAAAAAA|147|Fourth Lake|RD|Suite 460|Bethel|Iowa County|IA|55281|United States|-6|single family| +66|AAAAAAAACEAAAAAA|238|Pine |Road|Suite 470|Crossroads|Sioux County|NE|60534|United States|-7|single family| +67|AAAAAAAADEAAAAAA|896|11th |Street|Suite B|Flint|Van Buren County|IA|58909|United States|-6|single family| +68|AAAAAAAAEEAAAAAA|240|Laurel Forest|Street|Suite H|Summit|Bledsoe County|TN|30499|United States|-5|single family| +69|AAAAAAAAFEAAAAAA|37|Pine |Drive|Suite H|Glendale|Scott County|KY|43951|United States|-5|single family| +70|AAAAAAAAGEAAAAAA|511|14th Sycamore|Parkway|Suite 210|White Oak|Union County|OH|46668|United States|-5|condo| +71|AAAAAAAAHEAAAAAA|38|Ridge |Cir.|Suite 370|Lakeview|Nottoway County|VA|28579|United States|-5|apartment| +72|AAAAAAAAIEAAAAAA|982|Willow |Blvd|Suite 20|Richville|Cascade County|MT|65945|United States|-7|single family| +73|AAAAAAAAJEAAAAAA|158|Green Broadway|Road|Suite B|Cedar Grove|Kingman County|KS|60411|United States|-6|apartment| +74|AAAAAAAAKEAAAAAA|723|Sycamore |Boulevard|Suite C|Spring Hill|Wapello County|IA|56787|United States|-6|condo| +75|AAAAAAAALEAAAAAA|32|Pine |Wy|Suite 480|Valley View|Cleburne County|AL|35124|United States|-6|single family| +76|AAAAAAAAMEAAAAAA|257|Walnut South|Drive|Suite 420|Shiloh|Menominee County|MI|49275|United States|-5|single family| +77|AAAAAAAANEAAAAAA|151|Hill |Wy|Suite U|Oak Grove|Thomas County|GA|38370|United States|-5|single family| +78|AAAAAAAAOEAAAAAA|857|Railroad |Boulevard|Suite 140|Green Acres|Potter County|SD|57683|United States|-7|single family| +79|AAAAAAAAPEAAAAAA|409|Park 7th|Cir.|Suite U|Farmington|Wayne County|TN|39145|United States|-6|condo| +80|AAAAAAAAAFAAAAAA|298|12th |Way|Suite J|Oakland|Grant County|KY|49843|United States|-6|condo| +81|AAAAAAAABFAAAAAA|953|River Spruce|Pkwy|Suite 0|Union Hill|Brown County|TX|77746|United States|-6|condo| +82|AAAAAAAACFAAAAAA|97|River Main|Ln|Suite L|Buena Vista|Santa Clara County|CA|95752|United States|-8|single family| +83|AAAAAAAADFAAAAAA|410|Spring Poplar|Drive|Suite 20|Bethel|Nueces County|TX|75281|United States|-6|condo| +84|AAAAAAAAEFAAAAAA|886|Sunset |Circle|Suite T|Warwick|Itawamba County|MS|51398|United States|-6|apartment| +85|AAAAAAAAFFAAAAAA|476|Willow |Avenue|Suite R|Five Points|Crawford County|PA|16098|United States|-5|single family| +86|AAAAAAAAGFAAAAAA|699|11th 14th|ST|Suite O|Five Points|Pike County|KY|46098|United States|-5|single family| +87|AAAAAAAAHFAAAAAA||6th |Lane||Maple Grove|Claiborne County||58252|||apartment| +88|AAAAAAAAIFAAAAAA|104|9th |RD|Suite W|Frogtown|Bay County|MI|48784|United States|-5|single family| +89|AAAAAAAAJFAAAAAA|440|Church 7th|Ave|Suite 200|Lakeside|Solano County|CA|99532|United States|-8|apartment| +90|AAAAAAAAKFAAAAAA|892|North East|Drive|Suite U|Centerville|Hettinger County|ND|50059|United States|-6|condo| +91|AAAAAAAALFAAAAAA|936||Wy||Riverside|Murray County|||United States||| +92|AAAAAAAAMFAAAAAA|734|Ash Wilson|Boulevard|Suite V|Newtown|Boise County|ID|81749|United States|-7|condo| +93|AAAAAAAANFAAAAAA|755|Washington Sunset|RD|Suite Q|Walnut Grove|Greenup County|KY|47752|United States|-6|apartment| +94|AAAAAAAAOFAAAAAA|||||Lakeside|Washington County|GA||||| +95|AAAAAAAAPFAAAAAA|571|Hickory River|Ln|Suite M|Woodland|Thayer County|NE|64854|United States|-7|condo| +96|AAAAAAAAAGAAAAAA|246|Cedar Franklin|RD|Suite A|Oakdale|Fulton County|KY|49584|United States|-6|condo| +97|AAAAAAAABGAAAAAA|858|Sunset Main|Way|Suite 320|Enterprise|Hardin County|TX|71757|United States|-6|single family| +98|AAAAAAAACGAAAAAA|346|Walnut |Road|Suite 370|Spring Hill|Labette County|KS|66787|United States|-6|condo| +99|AAAAAAAADGAAAAAA|222|Park View|Drive|Suite U|Wilson|Dickinson County|MI|46971|United States|-5|apartment| +100|AAAAAAAAEGAAAAAA|127|Railroad |Court|Suite 170|Oakland|Perry County|TN|39843|United States|-6|apartment| +101|AAAAAAAAFGAAAAAA|922|Wilson 1st|Ave|Suite C|Bunker Hill|Vernon Parish|LA|70150|United States|-6|condo| +102|AAAAAAAAGGAAAAAA|270|North Fourth|Ct.|Suite 460|Newtown|Bay County|FL|31749|United States|-5|condo| +103|AAAAAAAAHGAAAAAA|467|Maple Chestnut|Ln|Suite 60|Forest Hills|Brown County|WI|59237|United States|-6|condo| +104|AAAAAAAAIGAAAAAA|786|Locust |Drive|Suite V|Jamestown|Madison County|IA|56867|United States|-6|single family| +105|AAAAAAAAJGAAAAAA|960|Franklin |Lane|Suite P|Lebanon|Dallas County|AL|32898|United States|-6|condo| +106|AAAAAAAAKGAAAAAA|992|View Ash|Parkway|Suite J|Forest Hills|Hall County|TX|79237|United States|-6|single family| +107|AAAAAAAALGAAAAAA|128|Laurel Church|Boulevard|Suite 210|Bethel|Grant County|IN|45281|United States|-5|apartment| +108|AAAAAAAAMGAAAAAA|156|West Maple|Street|Suite R|Belmont|Jackson County|OR|90191|United States|-8|single family| +109|AAAAAAAANGAAAAAA|453|Madison 8th|Avenue|Suite 310|Harmony|Aurora County|SD|55804|United States|-6|apartment| +110|AAAAAAAAOGAAAAAA|834|Hill Jackson|ST|Suite N|Hillcrest|Houston County|AL|33003|United States|-6|condo| +111|AAAAAAAAPGAAAAAA|306|Pine Lincoln|Drive|Suite 470|Bethel|Davidson County|NC|25281|United States|-5|condo| +112|AAAAAAAAAHAAAAAA|232|Dogwood |Street|Suite K|Rankin|Harrison County|IA|52621|United States|-6|apartment| +113|AAAAAAAABHAAAAAA|138|Main |Ave|Suite 390|Highland|Page County|IA|59454|United States|-6|apartment| +114|AAAAAAAACHAAAAAA|576|Willow |Road|Suite E|Pleasant Valley|Mohave County|AZ|82477|United States|-7|condo| +115|AAAAAAAADHAAAAAA|543|12th 9th|Ave||Wildwood|Rolette County||||-6|| +116|AAAAAAAAEHAAAAAA|581|Oak |Road|Suite 310|Deerfield|Harvey County|KS|69840|United States|-6|single family| +117|AAAAAAAAFHAAAAAA|593|2nd 5th|Ln|Suite 400|Providence|Curry County|OR|96614|United States|-8|condo| +118|AAAAAAAAGHAAAAAA|185|Wilson |Circle|Suite 270|Union Hill|Washington County|MS|57746|United States|-6|condo| +119|AAAAAAAAHHAAAAAA|525|Cedar |RD|Suite 210|Lakeview|Stark County|OH|48579|United States|-5|apartment| +120|AAAAAAAAIHAAAAAA|741|First |Ave|Suite O|Valley View|Benton County|IN|45124|United States|-5|apartment| +121|AAAAAAAAJHAAAAAA|644|Sixth |Ln|Suite 470|Lakewood|Putnam County|WV|28877|United States|-5|single family| +122|AAAAAAAAKHAAAAAA|||Ave||Woodlawn||MN|||-6|| +123|AAAAAAAALHAAAAAA|319|West View|Blvd|Suite X|Concord|Lagrange County|IN|44107|United States|-5|condo| +124|AAAAAAAAMHAAAAAA|688|Woodland 2nd|Cir.|Suite J|Hopewell|Okanogan County|WA|90587|United States|-8|condo| +125|AAAAAAAANHAAAAAA|74|7th |Lane|Suite 180|New Hope|Santa Clara County|CA|99431|United States|-8|apartment| +126|AAAAAAAAOHAAAAAA|196|Maple Railroad|Road|Suite 120|Arlington|Mahoning County|OH|46557|United States|-5|apartment| +127|AAAAAAAAPHAAAAAA|896|Sunset |Ln|Suite Y|Kingston|Stanton County|KS|64975|United States|-6|single family| +128|AAAAAAAAAIAAAAAA|127|Ridge 9th|Boulevard|Suite B|Sutton|Lorain County|OH|45413|United States|-5|apartment| +129|AAAAAAAABIAAAAAA||Sunset Pine||||Dickson County|TN|||-5|apartment| +130|AAAAAAAACIAAAAAA|194|Pine |ST|Suite J|Salem|Potter County|PA|18048|United States|-5|condo| +131|AAAAAAAADIAAAAAA|884|Woodland Seventh|Road|Suite 200|Green Acres|Cherokee County|OK|77683|United States|-6|condo| +132|AAAAAAAAEIAAAAAA|797|2nd |Ct.|Suite M|Ellisville|Darlington County|SC|26820|United States|-5|condo| +133|AAAAAAAAFIAAAAAA|953|Oak 5th|Boulevard|Suite J|Oakland|Fannin County|TX|79843|United States|-6|single family| +134|AAAAAAAAGIAAAAAA|504|West Hickory|Lane|Suite E|Greenwood|Crawford County|IN|48828|United States|-5|condo| +135|AAAAAAAAHIAAAAAA|279|Oak |Ave|Suite W|Clearview|Thayer County|NE|65495|United States|-7|apartment| +136|AAAAAAAAIIAAAAAA|610|Elm |Wy|Suite 30|Fowler|McLean County|IL|61083|United States|-6|apartment| +137|AAAAAAAAJIAAAAAA|467|Second |RD|Suite 220|Greenwood|Madison County|NE|68828|United States|-7|condo| +138|AAAAAAAAKIAAAAAA|99|Hillcrest 13th|Lane|Suite Q|Belmont|Greenwood County|KS|60191|United States|-6|apartment| +139|AAAAAAAALIAAAAAA|14|Pine Fourth|Ave|Suite 290|Pine Grove|Letcher County|KY|44593|United States|-5|single family| +140|AAAAAAAAMIAAAAAA|837|Madison Washington|Cir.|Suite Q|Union|Ottawa County|OK|78721|United States|-6|single family| +141|AAAAAAAANIAAAAAA|727|Elm Center|Cir.|Suite D|Hartland|Edmunds County|SD|56594|United States|-6|condo| +142|AAAAAAAAOIAAAAAA|268|Maple |Road|Suite 210|Belmont|Bergen County|NJ|00791|United States|-5|single family| +143|AAAAAAAAPIAAAAAA|141|9th |Ct.|Suite R|Edgewood|Burke County|GA|30069|United States|-5|apartment| +144|AAAAAAAAAJAAAAAA|686|Cedar |Cir.|Suite 90|Green Acres|Alpena County|MI|47683|United States|-5|condo| +145|AAAAAAAABJAAAAAA|454|Maple |Street|Suite 190|Wilson|Highland County|VA|26971|United States|-5|single family| +146|AAAAAAAACJAAAAAA|976|Sixth Dogwood|Ct.|Suite L|Highland|Litchfield County|CT|09454|United States|-5|single family| +147|AAAAAAAADJAAAAAA|596|Lake Maple|Blvd|Suite 160|Mount Olive|Hawaii County|HI|98059|United States|-10|apartment| +148|AAAAAAAAEJAAAAAA|731|14th |Boulevard|Suite D|Cedar Grove|Morton County|ND|50411|United States|-6|apartment| +149|AAAAAAAAFJAAAAAA|8|Woodland |Boulevard|Suite 270|Glenwood|Haralson County|GA|33511|United States|-5|apartment| +150|AAAAAAAAGJAAAAAA|121|Fifteenth |Blvd|Suite C|Sunnyside|Lafayette County|AR|71952|United States|-6|apartment| +151|AAAAAAAAHJAAAAAA|125|Fifth |Cir.|Suite 140|Shiloh|Deuel County|NE|69275|United States|-6|apartment| +152|AAAAAAAAIJAAAAAA|448|Sunset |RD|Suite J|Red Hill|Sharkey County|MS|54338|United States|-6|apartment| +153|AAAAAAAAJJAAAAAA|171|River Elm|Parkway|Suite 130|Deerfield|Fulton County|OH|49840|United States|-5|condo| +154|AAAAAAAAKJAAAAAA|264|Hillcrest Lincoln|Boulevard|Suite W|Lakewood|Eureka County|NV|88877|United States|-8|condo| +155|AAAAAAAALJAAAAAA|902|10th |Avenue|Suite 30|Glendale|Jackson County|MI|43951|United States|-5|condo| +156|AAAAAAAAMJAAAAAA|595||||||TX|||-6|single family| +157|AAAAAAAANJAAAAAA|904|Locust Seventh|Court|Suite 340|Shiloh|Clermont County|OH|49275|United States|-5|condo| +158|AAAAAAAAOJAAAAAA|293|Lincoln Willow|Court|Suite 370|Shady Grove|Davidson County|NC|22812|United States|-5|condo| +159|AAAAAAAAPJAAAAAA|544|Spruce |Cir.|Suite J|New Hope|Bradford County|FL|39431|United States|-5|condo| +160|AAAAAAAAAKAAAAAA|676|Jefferson |Lane|Suite 170|Woodland|Fresno County|CA|94854|United States|-8|single family| +161|AAAAAAAABKAAAAAA|518|Center Spruce|Boulevard|Suite 100|Clifton|McDonough County|IL|68014|United States|-6|apartment| +162|AAAAAAAACKAAAAAA|68|Oak Sycamore|Way|Suite 480|Waterloo|Pottawatomie County|KS|61675|United States|-6|condo| +163|AAAAAAAADKAAAAAA|484|Spring |Dr.|Suite 90|Hamilton|Vernon County|WI|52808|United States|-6|single family| +164|AAAAAAAAEKAAAAAA|435|2nd |Boulevard|Suite Q|Enterprise|Sussex County|DE|11757|United States|-5|condo| +165|AAAAAAAAFKAAAAAA|144|Sunset 2nd|Way|Suite A|Woodland|Banner County|NE|64854|United States|-6|condo| +166|AAAAAAAAGKAAAAAA|335|7th |Blvd|Suite 230|Plainview|Taylor County|GA|33683|United States|-5|apartment| +167|AAAAAAAAHKAAAAAA|288|Oak 3rd|Dr.|Suite B|Woodland|Grant County|NE|64854|United States|-6|apartment| +168|AAAAAAAAIKAAAAAA|373|Maple |Court|Suite 450|Providence|Hickory County|MO|66614|United States|-6|condo| +169|AAAAAAAAJKAAAAAA|12|||Suite 50||||||-6|apartment| +170|AAAAAAAAKKAAAAAA|237|Walnut |Way|Suite W|Colonial Heights|Monroe County|MO|63425|United States|-6|single family| +171|AAAAAAAALKAAAAAA|871|Smith Seventh|Way|Suite A|Calhoun|Cook County|MN|56909|United States|-6|condo| +172|AAAAAAAAMKAAAAAA||5th ||Suite 280||Madison County|MO|68054|United States||single family| +173|AAAAAAAANKAAAAAA|715|1st |Dr.|Suite H|Forest Hills|Grayson County|KY|49237|United States|-6|apartment| +174|AAAAAAAAOKAAAAAA|986|Adams |Pkwy|Suite M|Holiday Hills|Garden County|NE|63109|United States|-6|single family| +175|AAAAAAAAPKAAAAAA|557|4th Forest|Cir.|Suite 10|Antioch|Leon County|TX|78605|United States|-6|condo| +176|AAAAAAAAALAAAAAA|302|Jefferson |RD|Suite 140|Midway|Madison County|GA|31904|United States|-5|single family| +177|AAAAAAAABLAAAAAA|853|Dogwood View|Lane|Suite 440|Woodland|Grand County|CO|84854|United States|-7|single family| +178|AAAAAAAACLAAAAAA|592|Davis 14th|Circle|Suite L|Friendship|Winneshiek County|IA|54536|United States|-6|single family| +179|AAAAAAAADLAAAAAA|795|Davis Walnut|Avenue|Suite T|Jackson|Nye County|NV|89583|United States|-8|single family| +180|AAAAAAAAELAAAAAA|264|Railroad 11th|Ct.|Suite R|Midway|Carroll County|IL|61904|United States|-6|single family| +181|AAAAAAAAFLAAAAAA|469|8th |Drive|Suite 250|Greenwood|Osborne County|KS|68828|United States|-6|condo| +182|AAAAAAAAGLAAAAAA|772|Elm |Court|Suite 290|Bayside|Rice County|KS|69550|United States|-6|single family| +183|AAAAAAAAHLAAAAAA|47|Washington |Ave|Suite 310|Lakeside|Moniteau County|MO|69532|United States|-6|apartment| +184|AAAAAAAAILAAAAAA|704|Lincoln |Ln|Suite 340|Flatwoods|Williamson County|TX|74212|United States|-6|condo| +185|AAAAAAAAJLAAAAAA|441|Willow Washington|Cir.|Suite 490|Hillcrest|Pueblo County|CO|83003|United States|-7|condo| +186|AAAAAAAAKLAAAAAA|309|8th |Lane|Suite T|Riverside|Mesa County|CO|89231|United States|-7|apartment| +187|AAAAAAAALLAAAAAA|54|Ash 3rd|Boulevard|Suite 470|Centerville|Mills County|TX|70059|United States|-6|condo| +188|AAAAAAAAMLAAAAAA|798|West Sunset|Pkwy|Suite 160|Harmony|New Haven County|CT|06404|United States|-5|condo| +189|AAAAAAAANLAAAAAA|719|Hillcrest |Parkway|Suite 270|Five Forks|Sanders County|MT|62293|United States|-7|condo| +190|AAAAAAAAOLAAAAAA|928|Lake Lincoln|Pkwy|Suite T|Fremont|Greene County|NC|21851|United States|-5|single family| +191|AAAAAAAAPLAAAAAA|115|12th First|Boulevard|Suite J|Oakwood|Hamilton County|NE|60169|United States|-6|condo| +192|AAAAAAAAAMAAAAAA|639|Maple East|Circle|Suite 340|Walnut Grove|Stokes County|NC|27752|United States|-5|condo| +193|AAAAAAAABMAAAAAA|239|1st |Road|Suite G|Greenville|Hickman County|KY|41387|United States|-6|condo| +194|AAAAAAAACMAAAAAA|990|9th First|Ln|Suite 230|Oak Hill|Union County|AR|77838|United States|-6|single family| +195|AAAAAAAADMAAAAAA|939|Cedar |Court|Suite A|Mount Pleasant|Marion County|GA|31933|United States|-5|single family| +196|AAAAAAAAEMAAAAAA|860|||Suite 270||||||-5|condo| +197|AAAAAAAAFMAAAAAA|200|Park North|Blvd|Suite F|Highland Park|Creek County|OK|76534|United States|-6|condo| +198|AAAAAAAAGMAAAAAA|842|3rd |RD|Suite 280|Pleasant Valley|Pawnee County|KS|62477|United States|-6|condo| +199|AAAAAAAAHMAAAAAA|790|Pine |Street|Suite 400|Lakeview|Somerset County|ME|09179|United States|-5|condo| +200|AAAAAAAAIMAAAAAA|516|6th Spring|Boulevard|Suite K|Marion|McIntosh County|OK|70399|United States|-6|condo| +201|AAAAAAAAJMAAAAAA|517|8th |Parkway|Suite J|Salem|McIntosh County|OK|78048|United States|-6|single family| +202|AAAAAAAAKMAAAAAA|944|Willow |Parkway|Suite L|Unionville|Collin County|TX|71711|United States|-6|apartment| +203|AAAAAAAALMAAAAAA|997|Cedar |Boulevard|Suite T|Mount Zion|Wabash County|IL|68054|United States|-6|apartment| +204|AAAAAAAAMMAAAAAA|901|Locust Main|Blvd|Suite 440|Friendship|Isabella County|MI|44536|United States|-5|condo| +205|AAAAAAAANMAAAAAA|525|North Franklin|Pkwy|Suite 490|Shore Acres|Benewah County|ID|82724|United States|-7|single family| +206|AAAAAAAAOMAAAAAA|381|Third |Blvd|Suite 10|Waterloo|Panola County|MS|51675|United States|-6|single family| +207|AAAAAAAAPMAAAAAA|833|Miller |Pkwy|Suite 450|Blue Springs|Vernon County|WI|54686|United States|-6|single family| +208|AAAAAAAAANAAAAAA|772|Second |Court|Suite S|Lincoln|Albany County|WY|81289|United States|-7|apartment| +209|AAAAAAAABNAAAAAA|335|River |Drive|Suite 210|Sulphur Springs|Mitchell County|NC|28354|United States|-5|condo| +210|AAAAAAAACNAAAAAA|617|Park Hill|Road|Suite W|Shiloh|Jefferson County|KY|49275|United States|-6|apartment| +211|AAAAAAAADNAAAAAA|747|Willow Cedar|Lane|Suite 130|Bridgeport|Lamar County|MS|55817|United States|-6|apartment| +212|AAAAAAAAENAAAAAA|691|9th Mill||Suite 190||Pontotoc County|MS|59431|||| +213|AAAAAAAAFNAAAAAA|351|1st |Blvd|Suite I|Superior|Hidalgo County|NM|82562|United States|-7|condo| +214|AAAAAAAAGNAAAAAA|375|Walnut Second|Lane|Suite 480|Oakland|Halifax County|NC|29843|United States|-5|apartment| +215|AAAAAAAAHNAAAAAA|874|3rd |Street|Suite M|Sunnyside|Palo Alto County|IA|51952|United States|-6|apartment| +216|AAAAAAAAINAAAAAA|469|Maple Laurel|Pkwy|Suite F|Greenwood|Hampton city|VA|28828|United States|-5|condo| +217|AAAAAAAAJNAAAAAA|297|7th |Way|Suite E|White Oak|Tioga County|NY|16668|United States|-5|condo| +218|AAAAAAAAKNAAAAAA|998|Center Dogwood|Lane|Suite I|Oakland|Pine County|MN|59843|United States|-6|single family| +219|AAAAAAAALNAAAAAA|933|Franklin Highland|ST|Suite 380|Woodbury|Stevens County|KS|64489|United States|-6|condo| +220|AAAAAAAAMNAAAAAA|695|1st Oak|Circle|Suite 40|Liberty|Wabaunsee County|KS|63451|United States|-6|single family| +221|AAAAAAAANNAAAAAA|940|5th |Ln|Suite 300|Forest Hills|Kit Carson County|CO|89237|United States|-7|apartment| +222|AAAAAAAAONAAAAAA|393|South |Wy|Suite X|Ashley|Falls County|TX|74324|United States|-6|condo| +223|AAAAAAAAPNAAAAAA|455|First |Ln|Suite 210|Friendship|Cleburne County|AL|34536|United States|-6|apartment| +224|AAAAAAAAAOAAAAAA|94|Lincoln Walnut|Court|Suite 130|Edgewood|Calhoun County|MS|50069|United States|-6|single family| +225|AAAAAAAABOAAAAAA|673|Lincoln Washington|Dr.|Suite K|Altamont|Lake County|IN|49387|United States|-5|condo| +226|AAAAAAAACOAAAAAA|247|Hickory Adams|Way|Suite R|Union|Oldham County|TX|78721|United States|-6|condo| +227|AAAAAAAADOAAAAAA|82|Seventh Fifth|Ln|Suite R|Midway|Fairbanks North Star Borough|AK|91904|United States|-9|apartment| +228|AAAAAAAAEOAAAAAA|776|Madison |ST|Suite A|Woodland|Franklin County|IA|54854|United States|-6|condo| +229|AAAAAAAAFOAAAAAA|376|Ash |Dr.|Suite M|Crossroads|Franklin city|VA|20534|United States|-5|single family| +230|AAAAAAAAGOAAAAAA|313|2nd Eigth|Avenue|Suite 40|Marion|Glascock County|GA|30399|United States|-5|condo| +231|AAAAAAAAHOAAAAAA|560|Lakeview |Way|Suite 130|Deerfield|Union County|KY|49840|United States|-5|condo| +232|AAAAAAAAIOAAAAAA|239|Elm |RD|Suite D|Shady Grove|Harrison County|IN|42812|United States|-5|single family| +233|AAAAAAAAJOAAAAAA|957|Cherry Johnson|Ct.|Suite E|Clinton|Jackson County|NC|28222|United States|-5|condo| +234|AAAAAAAAKOAAAAAA|986|6th Main|Ln|Suite 20|Crossroads|Benton County|IN|40534|United States|-5|condo| +235|AAAAAAAALOAAAAAA|213|Madison |Pkwy|Suite H|Lincoln|Orange County|FL|31289|United States|-5|apartment| +236|AAAAAAAAMOAAAAAA|944|West Center|Cir.|Suite J|Lewisburg|Flagler County|FL|37538|United States|-5|single family| +237|AAAAAAAANOAAAAAA|169|Broadway Oak|Boulevard|Suite V|Marion|Fayette County|AL|30399|United States|-6|apartment| +238|AAAAAAAAOOAAAAAA|566|Williams Lee|Way|Suite 100|Pleasant Hill|Campbell County|TN|33604|United States|-5|single family| +239|AAAAAAAAPOAAAAAA|633|Elm 9th|Dr.|Suite 50|Franklin|Jefferson County|IN|49101|United States|-5|single family| +240|AAAAAAAAAPAAAAAA|8|Valley |Blvd|Suite 470|Concord|Castro County|TX|74107|United States|-6|apartment| +241|AAAAAAAABPAAAAAA|323|Laurel |Way|Suite 50|Wilson|Sioux County|IA|56971|United States|-6|condo| +242|AAAAAAAACPAAAAAA|706|Oak Woodland|Court|Suite C|Sunnyside|Stephenson County|IL|61952|United States|-6|single family| +243|AAAAAAAADPAAAAAA|600|View |Ave|Suite 150|Crossroads|Bennington County|VT|01134|United States|-5|condo| +244|AAAAAAAAEPAAAAAA|501|Dogwood Woodland|Parkway|Suite 180|Ashland|Iredell County|NC|24244|United States|-5|single family| +245|AAAAAAAAFPAAAAAA|187|Park North|Street|Suite 470|Crossroads|Bee County|TX|70534|United States|-6|apartment| +246|AAAAAAAAGPAAAAAA|154|1st |Road|Suite V|Union Hill|Camden County|NJ|08346|United States|-5|apartment| +247|AAAAAAAAHPAAAAAA|310|Sunset |Circle|Suite A|Lakewood|Newport News city|VA|28877|United States|-5|apartment| +248|AAAAAAAAIPAAAAAA|587|Williams Dogwood|Avenue|Suite R|Concord|Paulding County|OH|44107|United States|-5|condo| +249|AAAAAAAAJPAAAAAA|378|First Oak|ST|Suite 390|Five Points|Kauai County|HI|96098|United States|-10|condo| +250|AAAAAAAAKPAAAAAA|882|Ridge Meadow|Parkway|Suite Q|Harmony|Macon County|TN|35804|United States|-6|apartment| +251|AAAAAAAALPAAAAAA|913|First Johnson|Ct.|Suite V|Sugar Hill|Crawford County|IL|65114|United States|-6|apartment| +252|AAAAAAAAMPAAAAAA|753|Center |Cir.|Suite 250|Bunker Hill|Halifax County|VA|20150|United States|-5|apartment| +253|AAAAAAAANPAAAAAA|446|Jefferson Adams|Avenue|Suite O|Liberty|Dinwiddie County|VA|23451|United States|-5|condo| +254|AAAAAAAAOPAAAAAA|271|Highland |Lane|Suite W|Greenfield|Red River Parish|LA|75038|United States|-6|apartment| +255|AAAAAAAAPPAAAAAA|681||Cir.||Concord||MD|||-5|condo| +256|AAAAAAAAAABAAAAA|973|Forest |Dr.|Suite V|Cordova|Union Parish|LA|76938|United States|-6|condo| +257|AAAAAAAABABAAAAA|960|4th |Wy|Suite 200|Ellsworth|Bourbon County|KS|65079|United States|-6|condo| +258|AAAAAAAACABAAAAA|241|Birch |RD|Suite X|Walnut Grove|Pasco County|FL|37752|United States|-5|apartment| +259|AAAAAAAADABAAAAA|888|Sunset |Circle|Suite I|Elkton|Sumter County|FL|33481|United States|-5|condo| +260|AAAAAAAAEABAAAAA|629|13th Cherry|Court|Suite 450|Plainview|Anson County|NC|23683|United States|-5|condo| +261|AAAAAAAAFABAAAAA|911|Second |RD|Suite A|Harmony|Lee County|GA|35804|United States|-5|apartment| +262|AAAAAAAAGABAAAAA|718|Willow |RD|Suite Q|Martinsville|Amherst County|VA|20419|United States|-5|condo| +263|AAAAAAAAHABAAAAA|75|Park Oak|ST|Suite 80|Marion|Nantucket County|MA|00999|United States|-5|apartment| +264|AAAAAAAAIABAAAAA|17|11th |RD|Suite M|Riverdale|Pendleton County|KY|49391|United States|-5|condo| +265|AAAAAAAAJABAAAAA|447|Walnut Fourth|Lane|Suite R|Union|Lincoln County|MO|68721|United States|-6|single family| +266|AAAAAAAAKABAAAAA|940|Second Ridge|Parkway|Suite C|Red Hill|Marshall County|SD|54338|United States|-7|condo| +267|AAAAAAAALABAAAAA|671|Broadway |Circle|Suite 170|Bridgeport|Park County|WY|85817|United States|-7|single family| +268|AAAAAAAAMABAAAAA|938|Cherry |Street|Suite 30|Stringtown|Emmet County|IA|50162|United States|-6|apartment| +269|AAAAAAAANABAAAAA|643|Park Main|Court|Suite Y|Oak Hill|Nuckolls County|NE|67838|United States|-7|single family| +270|AAAAAAAAOABAAAAA|935|Ridge Hill|Blvd|Suite 180|Lakeside|Carroll County|AR|79532|United States|-6|single family| +271|AAAAAAAAPABAAAAA|5|Chestnut Highland|Ct.|Suite 290|Salem|Bedford County|VA|28048|United States|-5|single family| +272|AAAAAAAAABBAAAAA||Hickory ||||Sioux County||68482|United States|-7|| +273|AAAAAAAABBBAAAAA|133|Church |Avenue|Suite F|Providence|Kane County|IL|66614|United States|-6|apartment| +274|AAAAAAAACBBAAAAA|789|Locust |RD|Suite J|Oak Ridge|Sac County|IA|58371|United States|-6|condo| +275|AAAAAAAADBBAAAAA|454|Church |Boulevard|Suite 370|Harmony|Parmer County|TX|75804|United States|-6|apartment| +276|AAAAAAAAEBBAAAAA|767|6th |Road|Suite 430|Franklin|Rockdale County|GA|39101|United States|-5|condo| +277|AAAAAAAAFBBAAAAA|129|Valley |Circle|Suite 410|Ashland|Decatur County|GA|34244|United States|-5|single family| +278|AAAAAAAAGBBAAAAA|877|14th |Wy|Suite 490|Ashland|New Kent County|VA|24244|United States|-5|apartment| +279|AAAAAAAAHBBAAAAA|276|Smith 5th|Street|Suite 270|Jamestown|Martin County|KY|46867|United States|-5|single family| +280|AAAAAAAAIBBAAAAA|288|Walnut 1st|Ln|Suite M|Antioch|Cleveland County|NC|28605|United States|-5|single family| +281|AAAAAAAAJBBAAAAA|716|Oak |Wy|Suite 330|Mount Olive|Windham County|CT|08659|United States|-5|apartment| +282|AAAAAAAAKBBAAAAA|4|Cherry |Ln|Suite L|Buena Vista|Halifax County|VA|25752|United States|-5|apartment| +283|AAAAAAAALBBAAAAA|49|View |Ave|Suite W|Florence|Sevier County|UT|83394|United States|-7|single family| +284|AAAAAAAAMBBAAAAA|766|13th |Ave|Suite 350|Franklin|Ohio County|KY|49101|United States|-5|condo| +285|AAAAAAAANBBAAAAA|764|Adams Main|Pkwy|Suite X|Lincoln|Phelps County|MO|61289|United States|-6|single family| +286|AAAAAAAAOBBAAAAA|103|4th Davis|Way|Suite 390|Jamestown|Douglas County|WI|56867|United States|-6|apartment| +287|AAAAAAAAPBBAAAAA|117|||Suite 30|Mountain View||VA|24466|United States|-5|| +288|AAAAAAAAACBAAAAA|898|15th |RD|Suite 460|Mountain View|Humboldt County|IA|54466|United States|-6|apartment| +289|AAAAAAAABCBAAAAA|612|Davis Birch|Way|Suite W|Webb|Osceola County|MI|40899|United States|-5|single family| +290|AAAAAAAACCBAAAAA|71|Spring West|Lane|Suite 50|Woodland|Grant County|NE|64854|United States|-6|apartment| +291|AAAAAAAADCBAAAAA|638|Main Lakeview|Avenue|Suite P|Oakdale|Storey County|NV|89584|United States|-8|single family| +292|AAAAAAAAECBAAAAA|151|2nd |Drive|Suite C|Unionville|Gray County|KS|61711|United States|-6|single family| +293|AAAAAAAAFCBAAAAA|678|Smith Franklin|Lane|Suite 80|Edgewood|Weber County|UT|80069|United States|-7|condo| +294|AAAAAAAAGCBAAAAA|640|4th Laurel|Boulevard|Suite 340|Enterprise|Peach County|GA|31757|United States|-5|apartment| +295|AAAAAAAAHCBAAAAA|195|West Cherry|Boulevard|Suite S|Enterprise|Whitley County|IN|41757|United States|-5|condo| +296|AAAAAAAAICBAAAAA|999|2nd Main|Drive|Suite W|Greenwood|Ohio County|KY|48828|United States|-5|condo| +297|AAAAAAAAJCBAAAAA|571|Oak Miller|Ln|Suite 290|Newport|Rabun County|GA|31521|United States|-5|single family| +298|AAAAAAAAKCBAAAAA|177|Church Church|Pkwy|Suite X|Macon|Waldo County|ME|00969|United States|-5|condo| +299|AAAAAAAALCBAAAAA|10|Ash 4th|Ave|Suite X|White Oak|Finney County|KS|66668|United States|-6|single family| +300|AAAAAAAAMCBAAAAA|799|10th |Road|Suite 490|Macedonia|Switzerland County|IN|41087|United States|-5|apartment| +301|AAAAAAAANCBAAAAA|660|Lake |ST|Suite 460|Hamilton|Palo Pinto County|TX|72808|United States|-6|apartment| +302|AAAAAAAAOCBAAAAA|401|6th |Wy|Suite 340|Shiloh|Jefferson Davis County|MS|59275|United States|-6|condo| +303|AAAAAAAAPCBAAAAA|49|Adams |Lane|Suite M|Arlington|Jefferson County|WV|26557|United States|-5|single family| +304|AAAAAAAAADBAAAAA|971|View West|Drive|Suite A|Woodland|Green County|WI|54854|United States|-6|apartment| +305|AAAAAAAABDBAAAAA|689|Fourth Ash|Drive|Suite 410|Springdale|La Salle County|IL|68883|United States|-6|single family| +306|AAAAAAAACDBAAAAA|81|Lee 4th|RD|Suite 140|Centerville|Coffee County|GA|30059|United States|-5|single family| +307|AAAAAAAADDBAAAAA|406|Sunset Spring|Street|Suite R|Harmony|Stanton County|NE|65804|United States|-7|single family| +308|AAAAAAAAEDBAAAAA|393|Poplar Lakeview|Boulevard|Suite Q|Bridgeport|York County|NE|65817|United States|-6|single family| +309|AAAAAAAAFDBAAAAA|925|9th |Boulevard|Suite 460|Five Points|Cumberland County|PA|16098|United States|-5|single family| +310|AAAAAAAAGDBAAAAA|349|Cedar Oak|Pkwy|Suite 30|Wilson|Clinton County|IA|56971|United States|-6|condo| +311|AAAAAAAAHDBAAAAA|434|9th Forest|Blvd|Suite W|Marion|Clay County|AL|30399|United States|-6|condo| +312|AAAAAAAAIDBAAAAA|475|Smith |Lane|Suite F|Union Hill|Lake County|TN|37746|United States|-6|apartment| +313|AAAAAAAAJDBAAAAA|660|3rd Poplar|Cir.|Suite L|Lakeside|Teller County|CO|89532|United States|-7|condo| +314|AAAAAAAAKDBAAAAA|256||Ct.|Suite 20|Highland Park|||86534|United States||| +315|AAAAAAAALDBAAAAA|557|Ridge |Circle|Suite 280|Red Hill|Adair County|IA|54338|United States|-6|single family| +316|AAAAAAAAMDBAAAAA|380|West |Parkway|Suite 20|Warwick|Howard County|MO|61398|United States|-6|condo| +317|AAAAAAAANDBAAAAA|603|Main Valley|Street|Suite 90|Valley View|Pontotoc County|OK|75124|United States|-6|single family| +318|AAAAAAAAODBAAAAA|72|Main Hickory|Cir.|Suite H|Greenwood|Muskogee County|OK|78828|United States|-6|apartment| +319|AAAAAAAAPDBAAAAA|682|6th Elevnth|ST|Suite 120|Spring Valley|Columbia County|OR|96060|United States|-8|condo| +320|AAAAAAAAAEBAAAAA|842|Sunset |Ct.|Suite R|Valley View|Walton County|GA|35124|United States|-5|single family| +321|AAAAAAAABEBAAAAA|219|6th |ST|Suite D|Union|Washington County|CO|88721|United States|-7|condo| +322|AAAAAAAACEBAAAAA|132|West |Cir.|Suite R|Pleasant Grove|Guernsey County|OH|44136|United States|-5|condo| +323|AAAAAAAADEBAAAAA|350|Spruce |Avenue|Suite 70|Greenville|Oliver County|ND|51387|United States|-6|condo| +324|AAAAAAAAEEBAAAAA|508|3rd |Dr.|Suite 170|Mountain View|Matanuska-Susitna Borough|AK|94466|United States|-9|condo| +325|AAAAAAAAFEBAAAAA|590|Jackson North|Ln|Suite 20|Maple Grove|Meeker County|MN|58252|United States|-6|condo| +326|AAAAAAAAGEBAAAAA|334|Broadway |RD|Suite 270|Springfield|Platte County|MO|69303|United States|-6|condo| +327|AAAAAAAAHEBAAAAA|636|4th |Way|Suite O|Five Points|Carroll County|MO|66098|United States|-6|apartment| +328|AAAAAAAAIEBAAAAA|561|Jefferson |Lane|Suite 330|Glenwood|Reno County|KS|63511|United States|-6|apartment| +329|AAAAAAAAJEBAAAAA|11|2nd Maple|Ln|Suite S|Riverview|Barren County|KY|49003|United States|-6|apartment| +330|AAAAAAAAKEBAAAAA|627|Madison |RD|Suite 400|Oak Ridge|Suffolk County|MA|08971|United States|-5|single family| +331|AAAAAAAALEBAAAAA|274|Main |Wy|Suite W|Union|Washburn County|WI|58721|United States|-6|apartment| +332|AAAAAAAAMEBAAAAA|622|Spring Church|Street|Suite 450|Woodville|Bingham County|ID|84289|United States|-7|apartment| +333|AAAAAAAANEBAAAAA|629|Cedar |Pkwy|Suite A|Pleasant Valley|Chippewa County|WI|52477|United States|-6|apartment| +334|AAAAAAAAOEBAAAAA|368|Lincoln |Dr.|Suite N|Pomona|Jones County|TX|74153|United States|-6|single family| +335|AAAAAAAAPEBAAAAA|751|8th |Lane|Suite R|Oak Ridge|Perry County|MO|68371|United States|-6|single family| +336|AAAAAAAAAFBAAAAA|878|Oak |Boulevard|Suite 50|Hopewell|Hillsdale County|MI|40587|United States|-5|condo| +337|AAAAAAAABFBAAAAA|262|Sixth |Blvd|Suite 360|Providence|Martin County|TX|76614|United States|-6|apartment| +338|AAAAAAAACFBAAAAA|799|Lake |Way|Suite U|Highland|Surry County|VA|29454|United States|-5|single family| +339|AAAAAAAADFBAAAAA|821|First Cedar|Ave|Suite O|Woodland|Campbell County|VA|24854|United States|-5|condo| +340|AAAAAAAAEFBAAAAA|498|Lake |Way|Suite 490|Spring Valley|Habersham County|GA|36060|United States|-5|single family| +341|AAAAAAAAFFBAAAAA|851|Third |Road|Suite 390|Pine Grove|Vernon County|MO|64593|United States|-6|condo| +342|AAAAAAAAGFBAAAAA|587|Spring |Ave|Suite N|Mountain View|Quitman County|MS|54466|United States|-6|single family| +343|AAAAAAAAHFBAAAAA|195|Second Oak|Avenue|Suite 140|Riverside|Henry County|IA|59231|United States|-6|single family| +344|AAAAAAAAIFBAAAAA|322|||Suite 260|||ND||United States||condo| +345|AAAAAAAAJFBAAAAA|862|Sunset 6th|Dr.|Suite 490|Mount Vernon|Noble County|OK|78482|United States|-6|condo| +346|AAAAAAAAKFBAAAAA|984|East |Drive|Suite V|Elm Grove|Herkimer County|NY|13298|United States|-5|condo| +347|AAAAAAAALFBAAAAA|661|15th Broadway|Ln|Suite 90|Oakdale|Stafford County|VA|29584|United States|-5|apartment| +348|AAAAAAAAMFBAAAAA|49|Sunset |Boulevard|Suite 330|Mount Olive|Etowah County|AL|38059|United States|-6|single family| +349|AAAAAAAANFBAAAAA|191|Second Elm|Ave|Suite 230|Enterprise|Ada County|ID|81757|United States|-7|condo| +350|AAAAAAAAOFBAAAAA|145|West Pine|Blvd|Suite 290|Florence|Lewis County|NY|13394|United States|-5|condo| +351|AAAAAAAAPFBAAAAA|962|1st |Avenue|Suite 130|Five Forks|Mercer County|MO|62293|United States|-6|single family| +352|AAAAAAAAAGBAAAAA|259|Maple |Parkway|Suite 440|Silver Springs|Grant County|WV|24843|United States|-5|single family| +353|AAAAAAAABGBAAAAA|275|Oak |Ct.|Suite K|Union|Nacogdoches County|TX|78721|United States|-6|condo| +354|AAAAAAAACGBAAAAA|245|3rd |Ln|Suite 120|Franklin|Bulloch County|GA|39101|United States|-5|condo| +355|AAAAAAAADGBAAAAA|348|5th |Parkway|Suite 390|Salem|Turner County|GA|38048|United States|-5|condo| +356|AAAAAAAAEGBAAAAA|190|Main |Parkway|Suite 40|Fairview|Greer County|OK|75709|United States|-6|single family| +357|AAAAAAAAFGBAAAAA|801|Hillcrest |RD|Suite R|Wildwood|Crawford County|MO|66871|United States|-6|single family| +358|AAAAAAAAGGBAAAAA|294|6th Williams|RD|Suite T|Deerfield|Franklin County|IN|49840|United States|-5|apartment| +359|AAAAAAAAHGBAAAAA|147|Hickory |Dr.|Suite R|Fairview|Pemiscot County|MO|65709|United States|-6|single family| +360|AAAAAAAAIGBAAAAA|683|9th |Ave|Suite 90|Springfield|Perry County|MO|69303|United States|-6|condo| +361|AAAAAAAAJGBAAAAA|966|West |Blvd|Suite E|Mount Zion|Warren County|NJ|08654|United States|-5|apartment| +362|AAAAAAAAKGBAAAAA|380|View Lincoln|Ct.|Suite S|Valley View|Rutland County|VT|05724|United States|-5|apartment| +363|AAAAAAAALGBAAAAA|341|North |Parkway|Suite 150|Harmony|Clayton County|GA|35804|United States|-5|single family| +364|AAAAAAAAMGBAAAAA|54|Elevnth |Street|Suite B|Woodville|Floyd County|GA|34289|United States|-5|apartment| +365|AAAAAAAANGBAAAAA|315|Fifth Jefferson|Road|Suite 20|Providence|Knox County|KY|46614|United States|-5|condo| +366|AAAAAAAAOGBAAAAA|745|3rd Spruce|Drive|Suite 490|Riverside|Fleming County|KY|49231|United States|-6|condo| +367|AAAAAAAAPGBAAAAA|594|Cherry Main|Road|Suite R|Newport|Montgomery County|TX|71521|United States|-6|single family| +368|AAAAAAAAAHBAAAAA|552|3rd |Road|Suite 50|Clifton|Emmons County|ND|58014|United States|-6|condo| +369|AAAAAAAABHBAAAAA|652|Spring |Circle|Suite E|Pleasant Valley|Charlotte County|VA|22477|United States|-5|apartment| +370|AAAAAAAACHBAAAAA|143|Center |Avenue|Suite A|Arlington|Johnson County|NE|66557|United States|-7|single family| +371|AAAAAAAADHBAAAAA|84|West |Way|Suite 40|Spring Valley|Hubbard County|MN|56060|United States|-6|condo| +372|AAAAAAAAEHBAAAAA|56|Fifth Wilson|Dr.|Suite 90|Pine Grove|Suffolk County|NY|14593|United States|-5|single family| +373|AAAAAAAAFHBAAAAA|273|4th |Circle|Suite E|Red Hill|Trumbull County|OH|44338|United States|-5|single family| +374|AAAAAAAAGHBAAAAA|855|Willow |Street|Suite C|Springfield|Dallas County|IA|59303|United States|-6|condo| +375|AAAAAAAAHHBAAAAA|676|Mill |Way|Suite 100|Spring Grove|Lincoln Parish|LA|76719|United States|-6|condo| +376|AAAAAAAAIHBAAAAA|754|Adams Highland|Pkwy|Suite E|Jackson|Clay County|AR|79583|United States|-6|single family| +377|AAAAAAAAJHBAAAAA|353|Sixth |Ln|Suite 410|Oak Ridge|Charlton County|GA|38371|United States|-5|condo| +378|AAAAAAAAKHBAAAAA|482|Davis |ST|Suite N|Shady Grove|Cheyenne County|KS|62812|United States|-6|apartment| +379|AAAAAAAALHBAAAAA|973|Park Jefferson|Street|Suite J|Georgetown|Scott County|MS|57057|United States|-6|condo| +380|AAAAAAAAMHBAAAAA|407|Cedar |Drive|Suite S|Mount Olive|Oliver County|ND|58059|United States|-6|single family| +381|AAAAAAAANHBAAAAA|954|Lee |RD|Suite 270|Riverdale|Howard County|NE|69391|United States|-7|apartment| +382|AAAAAAAAOHBAAAAA|524|3rd |Ave|Suite 450|Lakewood|Butte County|ID|88877|United States|-7|single family| +383|AAAAAAAAPHBAAAAA|393|Lake Adams|Court|Suite X|Waterloo|Murray County|OK|71675|United States|-6|apartment| +384|AAAAAAAAAIBAAAAA|854|Cedar Mill|Ln|Suite A|Royal|Coffey County|KS|65819|United States|-6|condo| +385|AAAAAAAABIBAAAAA|769|3rd First|RD|Suite N|Ashland|Giles County|VA|24244|United States|-5|single family| +386|AAAAAAAACIBAAAAA|27|6th |Ct.|Suite F|Argyle|Richmond city|VA|28722|United States|-5|apartment| +387|AAAAAAAADIBAAAAA|292|Oak |Boulevard|Suite U|Pleasant Grove|Warren County|GA|34136|United States|-5|condo| +388|AAAAAAAAEIBAAAAA|401|Fourth Valley|Circle|Suite Q|Unionville|Hancock County|IL|61711|United States|-6|apartment| +389|AAAAAAAAFIBAAAAA|516|West |Blvd|Suite 210|Bay View|Bartow County|GA|36457|United States|-5|apartment| +390|AAAAAAAAGIBAAAAA|367|Lee |RD|Suite 170|Concord|Stewart County|TN|34107|United States|-6|single family| +391|AAAAAAAAHIBAAAAA|623|Park |Road|Suite 310|Wilson|Palo Pinto County|TX|76971|United States|-6|single family| +392|AAAAAAAAIIBAAAAA|964|Walnut Hill|ST|Suite 240|Liberty|McDonald County|MO|63451|United States|-6|apartment| +393|AAAAAAAAJIBAAAAA|254|Hill Cedar|Drive|Suite G|Sulphur Springs|DeSoto County|FL|38354|United States|-5|apartment| +394|AAAAAAAAKIBAAAAA|904|Hill |Dr.|Suite K|Riverview|Shiawassee County|MI|49003|United States|-5|condo| +395|AAAAAAAALIBAAAAA|841|Third Main|Lane|Suite A|Brownsville|Bowie County|TX|79310|United States|-6|apartment| +396|AAAAAAAAMIBAAAAA|841|Spruce |Court|Suite 330|Woodlawn|Larimer County|CO|84098|United States|-7|apartment| +397|AAAAAAAANIBAAAAA|114|4th |Lane|Suite 260|Ashland|Mason County|KY|44244|United States|-5|single family| +398|AAAAAAAAOIBAAAAA|49|Lincoln Mill|Wy|Suite 250|Marion|Hunterdon County|NJ|00999|United States|-5|apartment| +399|AAAAAAAAPIBAAAAA|899|Lake Birch|Street|Suite 0|Unionville|Lorain County|OH|41711|United States|-5|single family| +400|AAAAAAAAAJBAAAAA|26|Madison |Ct.|Suite 390|Saratoga|Jackson County|IA|52123|United States|-6|apartment| +401|AAAAAAAABJBAAAAA|352|2nd |Wy|Suite L|Lincoln|Park County|CO|81289|United States|-7|apartment| +402|AAAAAAAACJBAAAAA|521|Church Johnson|Pkwy|Suite 300|Mountain View|O-Brien County|IA|54466|United States|-6|single family| +403|AAAAAAAADJBAAAAA|470|Jefferson |Parkway|Suite 180|Riverside|Hutchinson County|TX|79231|United States|-6|condo| +404|AAAAAAAAEJBAAAAA|185|Sunset |||Fairfield||SD|56192|United States|-7|condo| +405|AAAAAAAAFJBAAAAA|494|2nd Madison|Ave|Suite 150|Belmont|Phelps County|MO|60191|United States|-6|apartment| +406|AAAAAAAAGJBAAAAA|577|11th |Ct.|Suite 380|Pleasant Hill|DeSoto County|FL|33604|United States|-5|single family| +407|AAAAAAAAHJBAAAAA|135|Miller |Avenue|Suite V|Woodland|Scott County|MS|54854|United States|-6|condo| +408|AAAAAAAAIJBAAAAA|62|Maple Hickory|ST|Suite J|Five Points|Hardee County|FL|36098|United States|-5|condo| +409|AAAAAAAAJJBAAAAA|874|Laurel |Road|Suite A|Valley View|Darlington County|SC|25124|United States|-5|single family| +410|AAAAAAAAKJBAAAAA|366|Eigth |Wy|Suite P|Valley View|Nobles County|MN|55124|United States|-6|condo| +411|AAAAAAAALJBAAAAA|488|Locust Walnut|Wy|Suite E|Newport|Valencia County|NM|81521|United States|-7|condo| +412|AAAAAAAAMJBAAAAA|741|Maple |ST|Suite U|White Oak|Cortland County|NY|16668|United States|-5|apartment| +413|AAAAAAAANJBAAAAA|507|Railroad |Ct.|Suite 340|Greenwood|Finney County|KS|68828|United States|-6|single family| +414|AAAAAAAAOJBAAAAA|131|Miller 14th|Dr.|Suite X|Fairview|Laramie County|WY|85709|United States|-7|single family| +415|AAAAAAAAPJBAAAAA|527|Green |Street|Suite F|Sulphur Springs|Luna County|NM|88354|United States|-7|apartment| +416|AAAAAAAAAKBAAAAA|686|2nd |Parkway|Suite 80|Red Hill|Morgan County|KY|44338|United States|-5|condo| +417|AAAAAAAABKBAAAAA|275|Poplar |Boulevard|Suite 140|Oak Grove|Choctaw County|OK|78370|United States|-6|condo| +418|AAAAAAAACKBAAAAA|121|13th |Avenue|Suite T|Montezuma|Jefferson Parish|LA|72150|United States|-6|condo| +419|AAAAAAAADKBAAAAA|817|6th Willow|Cir.|Suite 310|Spring Hill|Davie County|NC|26787|United States|-5|single family| +420|AAAAAAAAEKBAAAAA|234|Sunset |RD|Suite W|Union Hill|El Paso County|TX|77746|United States|-6|apartment| +421|AAAAAAAAFKBAAAAA|996|14th Washington|Dr.|Suite S|Hopewell|Waldo County|ME|01187|United States|-5|apartment| +422|AAAAAAAAGKBAAAAA|4|Jefferson |Cir.|Suite L|Lebanon|Miner County|SD|52898|United States|-7|apartment| +423|AAAAAAAAHKBAAAAA|585|10th Wilson|Parkway|Suite 110|Franklin|Eddy County|NM|89101|United States|-7|single family| +424|AAAAAAAAIKBAAAAA|261|Washington Center|Circle|Suite T|Oak Ridge|Gillespie County|TX|78371|United States|-6|single family| +425|AAAAAAAAJKBAAAAA|441|||Suite U||Dakota County|MN||||apartment| +426|AAAAAAAAKKBAAAAA|536|Main 3rd|Way|Suite G|Wildwood|Clark County|SD|56871|United States|-6|single family| +427|AAAAAAAALKBAAAAA|853|Cherry Smith|Wy|Suite X|Highland|Frontier County|NE|69454|United States|-6|condo| +428|AAAAAAAAMKBAAAAA|369|14th |Blvd|Suite 290|Guilford|Belknap County|NH|05008|United States|-5|apartment| +429|AAAAAAAANKBAAAAA|344|Oak |Circle|Suite S|Five Forks|Montgomery County|GA|32293|United States|-5|single family| +430|AAAAAAAAOKBAAAAA|864|Park South|Circle|Suite 150|Five Points|Nemaha County|KS|66098|United States|-6|condo| +431|AAAAAAAAPKBAAAAA|137|Ninth |Wy|Suite 0|Franklin|Morehouse Parish|LA|79101|United States|-6|single family| +432|AAAAAAAAALBAAAAA|902|Lake |Boulevard|Suite M|Spring Hill|Jo Daviess County|IL|66787|United States|-6|condo| +433|AAAAAAAABLBAAAAA|92|Ash North|Road|Suite D|Union Hill|New London County|CT|08346|United States|-5|condo| +434|AAAAAAAACLBAAAAA|547|11th |Parkway|Suite 70|Lakewood|Pamlico County|NC|28877|United States|-5|condo| +435|AAAAAAAADLBAAAAA|598|3rd |Drive|Suite T|Stringtown|Roosevelt County|NM|80162|United States|-7|single family| +436|AAAAAAAAELBAAAAA|129|Elm 4th|Avenue|Suite 230|Lakewood|Pecos County|TX|78877|United States|-6|apartment| +437|AAAAAAAAFLBAAAAA|218|Davis Spring|Dr.|Suite F|Red Hill|Logan County|OH|44338|United States|-5|single family| +438|AAAAAAAAGLBAAAAA|665|Wilson |Parkway|Suite M|Newtown|Columbia County|WA|91749|United States|-8|single family| +439|AAAAAAAAHLBAAAAA|828|2nd Miller|ST|Suite S|Mechanicsburg|Wheeler County|GA|32219|United States|-5|apartment| +440|AAAAAAAAILBAAAAA|551|Willow |Way|Suite O|Wright|Beauregard Parish|LA|72814|United States|-6|condo| +441|AAAAAAAAJLBAAAAA|919|4th Mill|Blvd|Suite A|Pleasant Grove|Hawaii County|HI|94136|United States|-10|apartment| +442|AAAAAAAAKLBAAAAA|659|Elevnth |Ln|Suite D|New Hope|Walker County|GA|39431|United States|-5|condo| +443|AAAAAAAALLBAAAAA|346|Laurel 7th|Ln|Suite 320|Walnut Grove|Burke County|NC|27752|United States|-5|condo| +444|AAAAAAAAMLBAAAAA|909|15th Elm|Dr.|Suite C|Clinton|Independence County|AR|78222|United States|-6|apartment| +445|AAAAAAAANLBAAAAA|52|Second Highland|Cir.|Suite B|Mount Vernon|Emporia city|VA|28482|United States|-5|apartment| +446|AAAAAAAAOLBAAAAA|598|Madison Davis|Circle|Suite M|Spring Valley|Campbell County|TN|36060|United States|-5|apartment| +447|AAAAAAAAPLBAAAAA|949|Meadow |Blvd|Suite A|Unionville|Rush County|IN|41711|United States|-5|condo| +448|AAAAAAAAAMBAAAAA|189|Highland |Road|Suite M|Guilford|Sampson County|NC|24408|United States|-5|condo| +449|AAAAAAAABMBAAAAA||Highland |Parkway|||||48222|||apartment| +450|AAAAAAAACMBAAAAA|282|||Suite K||Wyoming County|PA||United States|-5|| +451|AAAAAAAADMBAAAAA|828|Lakeview Oak|Parkway|Suite B|Plainview|Pasco County|FL|33683|United States|-5|single family| +452|AAAAAAAAEMBAAAAA|46|Sycamore |Court|Suite 110|Crossroads|Chickasaw County|IA|50534|United States|-6|apartment| +453|AAAAAAAAFMBAAAAA|339|5th |Lane|Suite L|Lone Oak|Watonwan County|MN|56893|United States|-6|condo| +454|AAAAAAAAGMBAAAAA|203|Smith 4th|Lane|Suite S|Shiloh|Okanogan County|WA|99275|United States|-8|condo| +455|AAAAAAAAHMBAAAAA|333|Maple |Parkway|Suite 250|Bunker Hill|Bayfield County|WI|50150|United States|-6|condo| +456|AAAAAAAAIMBAAAAA|36|Smith |Dr.|Suite 370|Buena Vista|Gilmer County|WV|25752|United States|-5|condo| +457|AAAAAAAAJMBAAAAA|136|Poplar |Street|Suite Q|Clinton|Palo Pinto County|TX|78222|United States|-6|apartment| +458|AAAAAAAAKMBAAAAA||Fourth |Street|Suite T|Pine Grove||||||single family| +459|AAAAAAAALMBAAAAA|629|Jackson |Ave|Suite 170|Pleasant Valley|Daniels County|MT|62477|United States|-7|single family| +460|AAAAAAAAMMBAAAAA|730|4th Church|Ln|Suite B|Stringtown|Montgomery County|TX|70162|United States|-6|single family| +461|AAAAAAAANMBAAAAA|719|College |Road|Suite L|Salem|Clark County|NV|88048|United States|-8|single family| +462|AAAAAAAAOMBAAAAA|945|Mill 5th|Ln|Suite 230|Oak Ridge|Jersey County|IL|68371|United States|-6|apartment| +463|AAAAAAAAPMBAAAAA|228|Ridge Fifth|Road|Suite 60|Midway|Ochiltree County|TX|71904|United States|-6|single family| +464|AAAAAAAAANBAAAAA|52|6th Park|Drive|Suite E|Farmington|Johnson County|GA|39145|United States|-5|condo| +465|AAAAAAAABNBAAAAA|102|Mill |Wy|Suite 310|Highland|Canadian County|OK|79454|United States|-6|single family| +466|AAAAAAAACNBAAAAA|702|College |Wy|Suite I|Allentown|Issaquena County|MS|51838|United States|-6|condo| +467|AAAAAAAADNBAAAAA|175|College |Way|Suite H|Centerville|Reynolds County|MO|60059|United States|-6|condo| +468|AAAAAAAAENBAAAAA|662|12th 6th|Lane|Suite 450|Glenwood|Corson County|SD|53511|United States|-6|single family| +469|AAAAAAAAFNBAAAAA|533|Willow |Road|Suite 290|Summit|Catahoula Parish|LA|70499|United States|-6|apartment| +470|AAAAAAAAGNBAAAAA|504|Main |Boulevard|Suite 260|Pleasant Grove|Columbus County|NC|24136|United States|-5|apartment| +471|AAAAAAAAHNBAAAAA|876|12th Ridge|Wy|Suite H|Lakewood|Crawford County|MI|48877|United States|-5|condo| +472|AAAAAAAAINBAAAAA|265|Third |Ave|Suite M|Sunnyside|McCracken County|KY|41952|United States|-5|single family| +473|AAAAAAAAJNBAAAAA|149|Laurel Hill|RD|Suite 110|Oakwood|Juab County|UT|80169|United States|-7|single family| +474|AAAAAAAAKNBAAAAA||||Suite 470|Johnsonville||VT||United States|-5|| +475|AAAAAAAALNBAAAAA|937|Fifth 3rd|Avenue|Suite 240|Hamilton|Stafford County|KS|62808|United States|-6|apartment| +476|AAAAAAAAMNBAAAAA|38|Church |Ct.|Suite X|Marion|Montmorency County|MI|40399|United States|-5|single family| +477|AAAAAAAANNBAAAAA|170|Central |ST|Suite M|Marion|Posey County|IN|40399|United States|-5|apartment| +478|AAAAAAAAONBAAAAA|273|Oak |Cir.|Suite 120|Oak Hill|Blaine County|OK|77838|United States|-6|single family| +479|AAAAAAAAPNBAAAAA|336|Lincoln |Circle|Suite L|Clinton|Shenandoah County|VA|28222|United States|-5|condo| +480|AAAAAAAAAOBAAAAA|709|Franklin |Ave|Suite L|Crossroads|Payne County|OK|70534|United States|-6|single family| +481|AAAAAAAABOBAAAAA|923|8th Maple|Ave|Suite D|Marion|Lawrence County|TN|30399|United States|-6|apartment| +482|AAAAAAAACOBAAAAA|526|Lakeview |Pkwy|Suite 110|Green Acres|Bedford County|PA|17683|United States|-5|condo| +483|AAAAAAAADOBAAAAA|||Ln||||IL||United States||| +484|AAAAAAAAEOBAAAAA|567|9th |Ln|Suite 260|Valley View|Creek County|OK|75124|United States|-6|single family| +485|AAAAAAAAFOBAAAAA|911|Hickory |Ln|Suite F|Blair|Hamilton County|NE|65465|United States|-6|condo| +486|AAAAAAAAGOBAAAAA|128|Wilson Seventh|Lane|Suite B|Summit|Knott County|KY|40499|United States|-5|condo| +487|AAAAAAAAHOBAAAAA|692|View |Circle|Suite R|White Oak|Hubbard County|MN|56668|United States|-6|apartment| +488|AAAAAAAAIOBAAAAA|24|Central |Blvd|Suite 490|Crossroads|Brooks County|GA|30534|United States|-5|single family| +489|AAAAAAAAJOBAAAAA|570|12th Ridge|Circle|Suite 60|Pleasant Grove|Volusia County|FL|34136|United States|-5|apartment| +490|AAAAAAAAKOBAAAAA|3|Main |Ave|Suite O|Bethel|Aroostook County|ME|05881|United States|-5|apartment| +491|AAAAAAAALOBAAAAA|562|5th Smith|Ct.|Suite G|Concord|Marion County|GA|34107|United States|-5|single family| +492|AAAAAAAAMOBAAAAA|283|Willow Davis|Avenue|Suite 480|Mount Vernon|Placer County|CA|98482|United States|-8|apartment| +493|AAAAAAAANOBAAAAA|774|9th |Ave|Suite P|Mountain View|DeKalb County|IL|64466|United States|-6|condo| +494|AAAAAAAAOOBAAAAA|64|Main |Ln|Suite 20|Antioch|Los Alamos County|NM|88605|United States|-7|apartment| +495|AAAAAAAAPOBAAAAA|27|Adams |RD|Suite L|Shiloh|Apache County|AZ|89275|United States|-7|apartment| +496|AAAAAAAAAPBAAAAA|240|Willow Walnut|Avenue|Suite 60|New Hope|Wayne County|UT|89431|United States|-7|single family| +497|AAAAAAAABPBAAAAA|429|5th |Ln|Suite 90|Deerfield|Choctaw County|AL|39840|United States|-6|single family| +498|AAAAAAAACPBAAAAA|682|Ridge |Parkway|Suite I|Waterloo|Sheridan County|ND|51675|United States|-6|condo| +499|AAAAAAAADPBAAAAA|498|Lake |Cir.|Suite 460|Five Points|Cumberland County|IL|66098|United States|-6|single family| +500|AAAAAAAAEPBAAAAA|740|14th Meadow|Street|Suite 10|Clifton|Columbia County|OR|98014|United States|-8|condo| +501|AAAAAAAAFPBAAAAA|894|Hickory Mill|Drive|Suite 220|Ashland|Giles County|VA|24244|United States|-5|condo| +502|AAAAAAAAGPBAAAAA|286|Poplar |Way|Suite 430|Oak Grove|Tazewell County|IL|68370|United States|-6|condo| +503|AAAAAAAAHPBAAAAA|413|Franklin Main|Way|Suite 40|Highland|Blanco County|TX|79454|United States|-6|apartment| +504|AAAAAAAAIPBAAAAA|739|Highland 13th|Wy|Suite 370|Mountain View|Owsley County|KY|44466|United States|-5|apartment| +505|AAAAAAAAJPBAAAAA|842|View |Pkwy|Suite 180|Kingston|Stafford County|KS|64975|United States|-6|single family| +506|AAAAAAAAKPBAAAAA|464|2nd |RD|Suite 470|Enterprise|Watonwan County|MN|51757|United States|-6|apartment| +507|AAAAAAAALPBAAAAA|36|5th |Cir.|Suite 200|Highland|Fairfield County|OH|49454|United States|-5|single family| +508|AAAAAAAAMPBAAAAA|840|Railroad |Drive|Suite G|Hillcrest|Madison County|ID|83003|United States|-7|apartment| +509|AAAAAAAANPBAAAAA|406|Meadow |Ln|Suite P|Shady Grove|Madison County|FL|32812|United States|-5|apartment| +510|AAAAAAAAOPBAAAAA|905|Locust West|Blvd|Suite 100|Shiloh|Orleans County|NY|19275|United States|-5|single family| +511|AAAAAAAAPPBAAAAA||Oak Sunset||||Dallas County|TX||United States||apartment| +512|AAAAAAAAAACAAAAA|487|7th |RD|Suite F|Pleasant Grove|Hamilton County|TX|74136|United States|-6|single family| +513|AAAAAAAABACAAAAA|489|Highland |Street|Suite B|Oak Grove|Ward County|TX|78370|United States|-6|condo| +514|AAAAAAAACACAAAAA|732|Maple |Ln|Suite N|Antioch|Ector County|TX|78605|United States|-6|apartment| +515|AAAAAAAADACAAAAA|44|4th |Pkwy|Suite 20|Riverview|Clallam County|WA|99003|United States|-8|apartment| +516|AAAAAAAAEACAAAAA|347|Lincoln Cedar|Road|Suite M|Oak Hill|Huerfano County|CO|87838|United States|-7|apartment| +517|AAAAAAAAFACAAAAA|795|1st 10th|Pkwy|Suite L|Egypt|Harding County|NM|84921|United States|-7|condo| +518|AAAAAAAAGACAAAAA|98|Miller |Dr.|Suite W|Summit|Adams County|IL|60499|United States|-6|apartment| +519|AAAAAAAAHACAAAAA|521|View |Circle|Suite N|Pleasant Hill|Eureka County|NV|83604|United States|-8|single family| +520|AAAAAAAAIACAAAAA|742|Hickory 9th|Blvd|Suite 200|Macedonia|Hampshire County|WV|21087|United States|-5|single family| +521|AAAAAAAAJACAAAAA|221|8th |Road|Suite 240|Waterloo|Alpine County|CA|91675|United States|-8|condo| +522|AAAAAAAAKACAAAAA|777|Hickory 6th|Street|Suite 140|Whitney|Rosebud County|MT|68339|United States|-7|apartment| +523|AAAAAAAALACAAAAA|775|6th |Drive|Suite H|Lakewood|Lake County|IN|48877|United States|-5|condo| +524|AAAAAAAAMACAAAAA|915|Cedar |Way|Suite 450|Hartland|Henrico County|VA|26594|United States|-5|apartment| +525|AAAAAAAANACAAAAA|607|Main 1st|Boulevard|Suite K|Midway|Gogebic County|MI|41904|United States|-5|apartment| +526|AAAAAAAAOACAAAAA||Jefferson North|Wy||Johnsonville|||||-6|| +527|AAAAAAAAPACAAAAA||Pine Sunset|ST|||Pushmataha County|||United States||apartment| +528|AAAAAAAAABCAAAAA|322|Smith 4th|Boulevard|Suite E|Lakeside|Miller County|MO|69532|United States|-6|condo| +529|AAAAAAAABBCAAAAA|950|Fifth 3rd|Avenue|Suite B|Springdale|Yalobusha County|MS|58883|United States|-6|single family| +530|AAAAAAAACBCAAAAA|928|Williams |Street|Suite H|Oakwood|Concho County|TX|70169|United States|-6|single family| +531|AAAAAAAADBCAAAAA||Thirteenth 1st|||Five Points|||86098|United States||apartment| +532|AAAAAAAAEBCAAAAA||Madison |Street|Suite D||Robertson County|||United States|-5|| +533|AAAAAAAAFBCAAAAA|25|Johnson |ST|Suite 220|Maple Grove|Union County|AR|78252|United States|-6|single family| +534|AAAAAAAAGBCAAAAA|72|Center |Court|Suite W|Clinton|Victoria County|TX|78222|United States|-6|single family| +535|AAAAAAAAHBCAAAAA|210|Cedar |Drive|Suite B|Deerfield|Dade County|MO|69840|United States|-6|single family| +536|AAAAAAAAIBCAAAAA|557|4th Park|Cir.|Suite C|Greenville|Marion County|TX|71387|United States|-6|single family| +537|AAAAAAAAJBCAAAAA|531|Main |Dr.|Suite Y|Five Forks|Wilkes County|NC|22293|United States|-5|single family| +538|AAAAAAAAKBCAAAAA|472|9th Oak|Lane|Suite H|Highland|Clinton County|OH|49454|United States|-5|single family| +539|AAAAAAAALBCAAAAA|667|Smith |Circle|Suite N|Walnut Grove|Holt County|NE|67752|United States|-7|condo| +540|AAAAAAAAMBCAAAAA|886|12th |Avenue|Suite E|Bethel|Taylor County|WV|25281|United States|-5|condo| +541|AAAAAAAANBCAAAAA|158|15th |Boulevard|Suite 260|Oak Hill|Waller County|TX|77838|United States|-6|single family| +542|AAAAAAAAOBCAAAAA|353|Laurel Meadow|Parkway|Suite M|Lebanon|Hamilton County|NY|12898|United States|-5|single family| +543|AAAAAAAAPBCAAAAA|268|Forest |Boulevard|Suite 270|Green Acres|Ozaukee County|WI|57683|United States|-6|condo| +544|AAAAAAAAACCAAAAA|648|Wilson North|Drive|Suite 30|Rosebud|Jasper County|TX|72244|United States|-6|single family| +545|AAAAAAAABCCAAAAA|195|Railroad Oak|Pkwy|Suite 210|Newport|Franklin County|TX|71521|United States|-6|single family| +546|AAAAAAAACCCAAAAA|540|Elm |Circle|Suite V|Mount Olive|Dixie County|FL|38059|United States|-5|apartment| +547|AAAAAAAADCCAAAAA|321|Sixth North|Wy|Suite M|Fox|Pickaway County|OH|40631|United States|-5|apartment| +548|AAAAAAAAECCAAAAA|543|Cherry Hickory|Pkwy|Suite 370|Wilson|Pennington County|SD|56971|United States|-7|condo| +549|AAAAAAAAFCCAAAAA|951|6th |Parkway|Suite 170|Liberty|Alexander County|NC|23451|United States|-5|single family| +550|AAAAAAAAGCCAAAAA|994|Williams |Court|Suite O|Summit|Clear Creek County|CO|80499|United States|-7|single family| +551|AAAAAAAAHCCAAAAA|470|5th |RD|Suite 290|Bethel|Quay County|NM|85281|United States|-7|single family| +552|AAAAAAAAICCAAAAA|460|Park |Wy|Suite 80|Edgewood|Oxford County|ME|00669|United States|-5|condo| +553|AAAAAAAAJCCAAAAA|454|Green |Road|Suite O|Mountain View|Taylor County|FL|34466|United States|-5|apartment| +554|AAAAAAAAKCCAAAAA|731|Birch |Pkwy|Suite 270|Hopewell|Coryell County|TX|70587|United States|-6|condo| +555|AAAAAAAALCCAAAAA|837|Hill |Ln|Suite 180|Hamilton|Amador County|CA|92808|United States|-8|condo| +556|AAAAAAAAMCCAAAAA|762|Smith |Drive|Suite N|Oakland|Curry County|OR|99843|United States|-8|single family| +557|AAAAAAAANCCAAAAA|444|Sixth |Parkway||Union Hill||KS||United States|-6|| +558|AAAAAAAAOCCAAAAA|976|5th |Ave|Suite 130|Lebanon|Centre County|PA|12898|United States|-5|condo| +559|AAAAAAAAPCCAAAAA|300|Lincoln 2nd|Blvd|Suite U|Stratford|Bradford County|FL|36668|United States|-5|single family| +560|AAAAAAAAADCAAAAA|228|View |Ave|Suite I|Mechanicsburg|Owen County|KY|42219|United States|-5|single family| +561|AAAAAAAABDCAAAAA|938|3rd |RD|Suite A|Oak Grove|Van Buren County|IA|58370|United States|-6|condo| +562|AAAAAAAACDCAAAAA|787|4th Second|Street|Suite M|Summit|Rolette County|ND|50499|United States|-6|condo| +563|AAAAAAAADDCAAAAA||Hill Hill||||Livingston County|MI|49454|United States||condo| +564|AAAAAAAAEDCAAAAA|462|Mill |Road|Suite R|Highland|Bradley County|AR|79454|United States|-6|apartment| +565|AAAAAAAAFDCAAAAA|652|Cedar |Drive|Suite 200|Newport|Newton County|MS|51521|United States|-6|apartment| +566|AAAAAAAAGDCAAAAA|92|River 14th|Circle|Suite 390|Summerville|Faulk County|SD|52033|United States|-6|apartment| +567|AAAAAAAAHDCAAAAA|169|Spruce Walnut|RD|Suite B|Friendship|Rosebud County|MT|64536|United States|-7|single family| +568|AAAAAAAAIDCAAAAA|316|Ridge |RD|Suite 430|California|Windham County|VT|00741|United States|-5|condo| +569|AAAAAAAAJDCAAAAA|897|Spruce |Way|Suite X|Ashland|Hocking County|OH|44244|United States|-5|condo| +570|AAAAAAAAKDCAAAAA|878|Fifteenth 2nd|RD|Suite A|Lakewood|Solano County|CA|98877|United States|-8|apartment| +571|AAAAAAAALDCAAAAA|543|Oak 8th|Court|Suite T|Denmark|Belknap County|NH|06176|United States|-5|single family| +572|AAAAAAAAMDCAAAAA|118|Railroad |Street|Suite 60|New Hope|Bailey County|TX|79431|United States|-6|apartment| +573|AAAAAAAANDCAAAAA|891|Church Chestnut|Pkwy|Suite 300|Maple Grove|Menominee County|MI|48252|United States|-5|apartment| +574|AAAAAAAAODCAAAAA|856|Washington Third|ST|Suite F|Macedonia|Dakota County|NE|61087|United States|-6|single family| +575|AAAAAAAAPDCAAAAA|167|Dogwood |Ave|Suite P|Lakeview|Mills County|TX|78579|United States|-6|condo| +576|AAAAAAAAAECAAAAA|682|1st |Dr.|Suite 190|Spring Hill|Travis County|TX|76787|United States|-6|single family| +577|AAAAAAAABECAAAAA|985|Franklin Elm|Boulevard|Suite R|Ellisville|Greene County|OH|46820|United States|-5|condo| +578|AAAAAAAACECAAAAA|550|East |Parkway|Suite M|Enterprise|Buena Vista city|VA|21757|United States|-5|single family| +579|AAAAAAAADECAAAAA|289|Lee |Drive|Suite 310|Oakwood|Decatur County|IA|50169|United States|-6|condo| +580|AAAAAAAAEECAAAAA|853|Eigth |Way|Suite 390|Whitesville|Sacramento County|CA|95903|United States|-8|apartment| +581|AAAAAAAAFECAAAAA|443|9th |ST|Suite 220|Oak Grove|Morehouse Parish|LA|78370|United States|-6|apartment| +582|AAAAAAAAGECAAAAA|264|Ridge Willow|Ln|Suite Q|Pine Grove|Logan County|KS|64593|United States|-6|condo| +583|AAAAAAAAHECAAAAA|809|Oak Laurel|Parkway|Suite H|Five Points|Douglas County|WA|96098|United States|-8|condo| +584|AAAAAAAAIECAAAAA|476|9th Ninth|Ct.|Suite O|Spring Hill|Wood County|OH|46787|United States|-5|condo| +585|AAAAAAAAJECAAAAA|920|Elm 14th|ST|Suite D|Pleasant Grove|Lake County|MT|64136|United States|-7|apartment| +586|AAAAAAAAKECAAAAA|109|Cedar |RD|Suite W|Edgewood|Randolph County|GA|30069|United States|-5|apartment| +587|AAAAAAAALECAAAAA|221|Elm |Drive|Suite Q|Pleasant Grove|Monroe County|TN|34136|United States|-6|single family| +588|AAAAAAAAMECAAAAA|374|Williams |Cir.|Suite P|Crossroads|East Feliciana Parish|LA|70534|United States|-6|apartment| +589|AAAAAAAANECAAAAA|286|Maple Elm|Wy|Suite 10|Oak Hill|Placer County|CA|97838|United States|-8|single family| +590|AAAAAAAAOECAAAAA|332|West Oak|Street|Suite C|Woodland|Lowndes County|AL|34854|United States|-6|single family| +591|AAAAAAAAPECAAAAA|252|North |Road|Suite M|Midway|Kings County|CA|91904|United States|-8|condo| +592|AAAAAAAAAFCAAAAA|307|Highland |Circle|Suite G|Bethel|Wilson County|TN|35281|United States|-5|condo| +593|AAAAAAAABFCAAAAA|223|Seventh Wilson|Lane|Suite 100|Lincoln|Butler County|KS|61289|United States|-6|single family| +594|AAAAAAAACFCAAAAA|978|Mill First|Way|Suite Q|Brookwood|Augusta County|VA|20965|United States|-5|apartment| +595|AAAAAAAADFCAAAAA||6th Fourth||Suite 70||Rich County||||-7|| +596|AAAAAAAAEFCAAAAA|88|Green Highland|Street|Suite T|Oakdale|Cherokee County|GA|39584|United States|-5|single family| +597|AAAAAAAAFFCAAAAA|550|Lake |ST|Suite 460|Waterloo|Scott County|TN|31675|United States|-6|apartment| +598|AAAAAAAAGFCAAAAA|218|Park 4th|Ct.|Suite N|Shady Grove|Madison County|||United States||| +599|AAAAAAAAHFCAAAAA|685|Jackson |Dr.|Suite M|Mount Zion|Ringgold County|IA|58054|United States|-6|apartment| +600|AAAAAAAAIFCAAAAA|447|Main Davis|Dr.|Suite D|Green Acres|Cloud County|KS|67683|United States|-6|apartment| +601|AAAAAAAAJFCAAAAA|192|2nd |Dr.|Suite 230|Fairview|Pamlico County|NC|25709|United States|-5|single family| +602|AAAAAAAAKFCAAAAA|13|Hill Sycamore|Road|Suite 20|Sunnyside|Roosevelt County|MT|61952|United States|-7|apartment| +603|AAAAAAAALFCAAAAA|698|Fifth 1st|ST|Suite U|Bloomingdale|Winnebago County|IL|61824|United States|-6|apartment| +604|AAAAAAAAMFCAAAAA|949|12th |Boulevard|Suite V|Arlington|Tift County|GA|36557|United States|-5|single family| +605|AAAAAAAANFCAAAAA|799|North |Ave|Suite E|Summit|Waushara County|WI|50499|United States|-6|apartment| +606|AAAAAAAAOFCAAAAA|462|Madison |Boulevard|Suite 460|New Hope|Alcona County|MI|49431|United States|-5|apartment| +607|AAAAAAAAPFCAAAAA|324|Sunset |RD|Suite F|Riverview|Beaverhead County|MT|69003|United States|-7|condo| +608|AAAAAAAAAGCAAAAA|600|Fourth |Ct.|Suite D|Union|Sumner County|KS|68721|United States|-6|single family| +609|AAAAAAAABGCAAAAA|870|10th Adams|Boulevard|Suite Y|Mountain View|Wallowa County|OR|94466|United States|-8|single family| +610|AAAAAAAACGCAAAAA|690|Maple |Wy|Suite 40|Marion|Wilson County|KS|60399|United States|-6|condo| +611|AAAAAAAADGCAAAAA||9th ||Suite N|||OH|49843|United States||| +612|AAAAAAAAEGCAAAAA|985|4th Cedar|Ave|Suite 490|Andover|Bates County|MO|61639|United States|-6|condo| +613|AAAAAAAAFGCAAAAA|756|Highland |Ln|Suite 380|Kingston|Camden County|MO|64975|United States|-6|apartment| +614|AAAAAAAAGGCAAAAA|939|Meadow Hickory|Cir.|Suite G|Clifton|Cullman County|AL|38014|United States|-6|condo| +615|AAAAAAAAHGCAAAAA|148|Fourth Elm|Street|Suite W|Oak Hill|Clayton County|GA|37838|United States|-5|condo| +616|AAAAAAAAIGCAAAAA|992|Chestnut 6th|Pkwy|Suite F|Lebanon|Russell County|KY|42898|United States|-5|single family| +617|AAAAAAAAJGCAAAAA|843|Twelfth |Drive|Suite K|Franklin|Runnels County|TX|79101|United States|-6|condo| +618|AAAAAAAAKGCAAAAA|900|Elm Park|Pkwy|Suite 280|Wilson|Hernando County|FL|36971|United States|-5|condo| +619|AAAAAAAALGCAAAAA|749|Mill Maple|Circle|Suite 0|Oak Ridge|Fairfield County|CT|08971|United States|-5|condo| +620|AAAAAAAAMGCAAAAA|305|Williams Walnut|Pkwy|Suite 330|New Hope|Johnson County|KS|69431|United States|-6|condo| +621|AAAAAAAANGCAAAAA|253|Washington Smith|Lane|Suite Q|Rankin|Plumas County|CA|92621|United States|-8|single family| +622|AAAAAAAAOGCAAAAA|720|Birch ||||||26614|United States|-5|| +623|AAAAAAAAPGCAAAAA|555|Pine |Court|Suite M|Red Hill|Garfield County|OK|74338|United States|-6|single family| +624|AAAAAAAAAHCAAAAA|152|View Smith|Ave|Suite P|Harmony|Macon County|IL|65804|United States|-6|single family| +625|AAAAAAAABHCAAAAA|189|Center |RD|Suite 240|Oakdale|Freestone County|TX|79584|United States|-6|condo| +626|AAAAAAAACHCAAAAA|647|||Suite 180|Liberty|Knox County||33451||-6|| +627|AAAAAAAADHCAAAAA|816|Smith Hillcrest|Parkway|Suite E|Red Hill|Rock County|MN|54338|United States|-6|condo| +628|AAAAAAAAEHCAAAAA|634|Maple |Lane|Suite 160|Pleasant Grove|Nelson County|ND|54136|United States|-6|condo| +629|AAAAAAAAFHCAAAAA|388|Sycamore Meadow|Way|Suite 310|Fairview|Little River County|AR|75709|United States|-6|single family| +630|AAAAAAAAGHCAAAAA|172|College Railroad|Dr.|Suite X|Enterprise|Los Angeles County|CA|91757|United States|-8|condo| +631|AAAAAAAAHHCAAAAA|442|Railroad Mill|Ct.|Suite 30|Jamestown|Harrison County|MS|56867|United States|-6|single family| +632|AAAAAAAAIHCAAAAA|11|Green |RD|Suite P|Woodlawn|Lawrence County|IN|44098|United States|-5|apartment| +633|AAAAAAAAJHCAAAAA|971|First West|Ave|Suite K|Maple Grove|Presidio County|TX|78252|United States|-6|apartment| +634|AAAAAAAAKHCAAAAA|556|Hickory |Road|Suite I|Wildwood|DuPage County|IL|66871|United States|-6|condo| +635|AAAAAAAALHCAAAAA|740|Elm |Drive|Suite 380|Forest Hills|Clay County|IL|69237|United States|-6|single family| +636|AAAAAAAAMHCAAAAA|340|6th Lincoln|Road|Suite 60|Shore Acres|Mendocino County|CA|92724|United States|-8|condo| +637|AAAAAAAANHCAAAAA|741|Spring |Road|Suite P|Enterprise|Jefferson County|TN|31757|United States|-6|condo| +638|AAAAAAAAOHCAAAAA|663|Franklin |Drive|Suite 40|Jamestown|Houghton County|MI|46867|United States|-5|single family| +639|AAAAAAAAPHCAAAAA|860|6th Maple|Way|Suite Q|Pleasant Hill|Webster County|IA|53604|United States|-6|single family| +640|AAAAAAAAAICAAAAA|527|Laurel View|Court|Suite 50|Clinton|Berrien County|GA|38222|United States|-5|apartment| +641|AAAAAAAABICAAAAA|986|Valley Willow|Boulevard|Suite 10|Lincoln|Creek County|OK|71289|United States|-6|apartment| +642|AAAAAAAACICAAAAA|928|Walnut 9th|Ln|Suite U|Brownsville|Guadalupe County|TX|79310|United States|-6|single family| +643|AAAAAAAADICAAAAA|908|Spring Maple|Ct.|Suite 380|Marion|Greene County|NY|10399|United States|-5|condo| +644|AAAAAAAAEICAAAAA|485|Valley |Ct.|Suite 30|Mount Zion|Leavenworth County|KS|68054|United States|-6|single family| +645|AAAAAAAAFICAAAAA|464|Elm Franklin|Dr.|Suite 210|Aberdeen|Lyon County|IA|57995|United States|-6|single family| +646|AAAAAAAAGICAAAAA|591|1st West|Circle|Suite 80|Five Points|Gilchrist County|FL|36098|United States|-5|condo| +647|AAAAAAAAHICAAAAA|238|Elm Fourth|Parkway|Suite I|Tipton|Mason County|WV|28773|United States|-5|condo| +648|AAAAAAAAIICAAAAA|267|Fourth |Avenue|Suite G|Sunnyside|Costilla County|CO|81952|United States|-7|condo| +649|AAAAAAAAJICAAAAA|504|Forest Elm|Ln|Suite 270|Mount Zion|La Paz County|AZ|88054|United States|-7|single family| +650|AAAAAAAAKICAAAAA|841|Birch |Pkwy|Suite 390|Red Hill|Harrison County|MS|54338|United States|-6|condo| +651|AAAAAAAALICAAAAA|897|Mill Wilson|Court|Suite A|Cedar Grove|Island County|WA|90411|United States|-8|single family| +652|AAAAAAAAMICAAAAA|899|Sunset Oak|Dr.|Suite 470|Glenwood|Pointe Coupee Parish|LA|73511|United States|-6|condo| +653|AAAAAAAANICAAAAA|801|Elevnth Dogwood|Circle|Suite L|Mount Pleasant|Adams County|PA|11933|United States|-5|condo| +654|AAAAAAAAOICAAAAA|673|Elm |Cir.|Suite 70|Unionville|Caldwell County|TX|71711|United States|-6|condo| +655|AAAAAAAAPICAAAAA|848|Locust 10th|Ln|Suite B|Ellisville|Jefferson Davis County|MS|56820|United States|-6|condo| +656|AAAAAAAAAJCAAAAA|753|Maple 14th|Court|Suite T|Oakdale|Onslow County|NC|29584|United States|-5|apartment| +657|AAAAAAAABJCAAAAA|478|7th |Dr.|Suite L|Bethel|Yuma County|AZ|85281|United States|-7|condo| +658|AAAAAAAACJCAAAAA|231|Wilson 3rd|Ave|Suite 380|Ludlow|Knox County|TX|75566|United States|-6|single family| +659|AAAAAAAADJCAAAAA|324|College North|Drive|Suite Q|Riverview|Cherokee County|NC|29003|United States|-5|single family| +660|AAAAAAAAEJCAAAAA|968|Railroad |Lane|Suite E|Colfax|Jackson County|MI|42565|United States|-5|single family| +661|AAAAAAAAFJCAAAAA|333|Church Hill|Dr.|Suite U|Plainview|Adair County|MO|63683|United States|-6|apartment| +662|AAAAAAAAGJCAAAAA|571|Adams |Ct.|Suite E|Wayland|Muskogee County|OK|75115|United States|-6|single family| +663|AAAAAAAAHJCAAAAA|126|12th |Lane|Suite 430|Union|McMullen County|TX|78721|United States|-6|apartment| +664|AAAAAAAAIJCAAAAA|419|View Miller|Wy|Suite 360|Pleasant Grove|Cobb County|GA|34136|United States|-5|single family| +665|AAAAAAAAJJCAAAAA||River |||Providence||MS||||apartment| +666|AAAAAAAAKJCAAAAA|158|Smith |Cir.|Suite O|Wildwood|Montgomery County|VA|26871|United States|-5|apartment| +667|AAAAAAAALJCAAAAA|494|Spring Third|Boulevard|Suite R|Spring Valley|Phillips County|AR|76060|United States|-6|single family| +668|AAAAAAAAMJCAAAAA|440|College |Ln|Suite 280|Forest Hills|Miner County|SD|59237|United States|-7|condo| +669|AAAAAAAANJCAAAAA|347|3rd |Cir.|Suite 0|Bridgeport|McCracken County|KY|45817|United States|-5|condo| +670|AAAAAAAAOJCAAAAA|736|Mill Miller|Blvd|Suite G|Unionville|Mayes County|OK|71711|United States|-6|apartment| +671|AAAAAAAAPJCAAAAA|508|Hill Washington|Wy|Suite 300|Walnut Grove|Adams County|IA|57752|United States|-6|single family| +672|AAAAAAAAAKCAAAAA|998|Laurel Elm|Parkway|Suite I|Collinsville|Laramie County|WY|82459|United States|-7|apartment| +673|AAAAAAAABKCAAAAA|440|Meadow Chestnut|Blvd|Suite U|Lawrenceville|Otsego County|MI|44462|United States|-5|single family| +674|AAAAAAAACKCAAAAA|434|Main |ST|Suite M|Highland Park|Salem County|NJ|07134|United States|-5|condo| +675|AAAAAAAADKCAAAAA|807|Pine 5th|Road|Suite G|Mountain View|Sabine County|TX|74466|United States|-6|condo| +676|AAAAAAAAEKCAAAAA||Broadway View||Suite 90|Frogtown|Sebastian County|AR|78784|||condo| +677|AAAAAAAAFKCAAAAA|410|Jefferson |Pkwy|Suite Y|Ashland|Crawford County|MI|44244|United States|-5|condo| +678|AAAAAAAAGKCAAAAA|694|Oak |Ave|Suite 350|Bethel|Saguache County|CO|85281|United States|-7|single family| +679|AAAAAAAAHKCAAAAA|379|Spring Eigth|Street|Suite R|Cedar Grove|Wilcox County|GA|30411|United States|-5|single family| +680|AAAAAAAAIKCAAAAA|199|6th Fifteenth|Ave|Suite 230|Woodland|Canadian County|OK|74854|United States|-6|single family| +681|AAAAAAAAJKCAAAAA|283|Wilson |Dr.|Suite W|Union Hill|Saratoga County|NY|17746|United States|-5|condo| +682|AAAAAAAAKKCAAAAA|402|11th |ST|Suite S|Lenox|Winston County|MS|51143|United States|-6|apartment| +683|AAAAAAAALKCAAAAA|775|1st |ST|Suite L|Hardy|Decatur County|IN|45354|United States|-5|condo| +684|AAAAAAAAMKCAAAAA|486|Cedar |Ln|Suite K|Bethel|Titus County|TX|75281|United States|-6|apartment| +685|AAAAAAAANKCAAAAA|866|Mill Third|Parkway|Suite 150|Lone Oak|Menominee County|WI|56893|United States|-6|single family| +686|AAAAAAAAOKCAAAAA|183||||||CA||United States||| +687|AAAAAAAAPKCAAAAA|911|Lee |ST|Suite 110|Union|Putnam County|FL|38721|United States|-5|apartment| +688|AAAAAAAAALCAAAAA|748|4th |ST|Suite I|Highland Park|Licking County|OH|46534|United States|-5|condo| +689|AAAAAAAABLCAAAAA|184|Forest |Ct.|Suite 40|Jackson|Columbia County|WI|59583|United States|-6|apartment| +690|AAAAAAAACLCAAAAA|623|Maple Meadow|Circle|Suite H|Belmont|Minidoka County|ID|80191|United States|-7|apartment| +691|AAAAAAAADLCAAAAA|643|River |Road|Suite G|Arlington|Lincoln County|AR|76557|United States|-6|apartment| +692|AAAAAAAAELCAAAAA|904|Hillcrest |Dr.|Suite U|Buckingham|Rockbridge County|VA|24092|United States|-5|condo| +693|AAAAAAAAFLCAAAAA|343|12th 4th|Cir.|Suite H|Fairfield|District of Columbia|DC|26192|United States|-5|condo| +694|AAAAAAAAGLCAAAAA|554|South |Parkway|Suite E|Red Hill|Rock Island County|IL|64338|United States|-6|condo| +695|AAAAAAAAHLCAAAAA|359|Mill |Circle|Suite 90|Plainview|Crawford County|KS|63683|United States|-6|condo| +696|AAAAAAAAILCAAAAA|224||||Jackson|Jackson County|KS|69583|United States||condo| +697|AAAAAAAAJLCAAAAA|102|West |Street|Suite 410|Lakewood|Borden County|TX|78877|United States|-6|condo| +698|AAAAAAAAKLCAAAAA|433|Railroad Second|Court|Suite 440|Brownsville|Saratoga County|NY|19310|United States|-5|single family| +699|AAAAAAAALLCAAAAA|124|3rd Lee|Pkwy|Suite R|Lakewood|Grundy County|MO|68877|United States|-6|single family| +700|AAAAAAAAMLCAAAAA|190|Jackson |RD|Suite 460|Harmony|Dixon County|NE|65804|United States|-6|apartment| +701|AAAAAAAANLCAAAAA|367|4th 14th|Ave|Suite 370|Marion|Refugio County|TX|70399|United States|-6|condo| +702|AAAAAAAAOLCAAAAA|308|Elm Church|Dr.|Suite 310|Jamestown|La Salle County|TX|76867|United States|-6|single family| +703|AAAAAAAAPLCAAAAA||Fourth ||Suite X||Reynolds County|MO|61289|United States||| +704|AAAAAAAAAMCAAAAA|916|River |Blvd|Suite 10|Brookwood|Cambria County|PA|10965|United States|-5|single family| +705|AAAAAAAABMCAAAAA|72|Woodland |Cir.|Suite C|Bridgeport|Graves County|KY|45817|United States|-6|condo| +706|AAAAAAAACMCAAAAA|387|Wilson Main|Ct.|Suite J|Ashland|Lemhi County|ID|84244|United States|-7|apartment| +707|AAAAAAAADMCAAAAA|293|Willow Maple|ST|Suite 220|Garrison|Madison County|FL|38767|United States|-5|condo| +708|AAAAAAAAEMCAAAAA|646|3rd North|Lane|Suite H|Walnut Grove|Jefferson County|IL|67752|United States|-6|apartment| +709|AAAAAAAAFMCAAAAA|911|4th |Way|Suite 70|Bridgeport|Crisp County|GA|35817|United States|-5|apartment| +710|AAAAAAAAGMCAAAAA|119|Lakeview |Way|Suite B|Crossroads|McNairy County|TN|30534|United States|-6|apartment| +711|AAAAAAAAHMCAAAAA|667|Valley 2nd|Lane|Suite Q|Farmington|Mountrail County|ND|59145|United States|-6|single family| +712|AAAAAAAAIMCAAAAA|56|7th North|Parkway|Suite C|Shiloh|Prince George County|VA|29275|United States|-5|apartment| +713|AAAAAAAAJMCAAAAA|466|East Smith||Suite 440|Newport||MN|51521|United States||single family| +714|AAAAAAAAKMCAAAAA|383|1st 8th|Pkwy|Suite W|Glenwood|Rooks County|KS|63511|United States|-6|single family| +715|AAAAAAAALMCAAAAA|798|Church |Pkwy|Suite Q|Spring Valley|Wilkin County|MN|56060|United States|-6|apartment| +716|AAAAAAAAMMCAAAAA|408|14th |Road|Suite I|Kingston|Mineral County|CO|84975|United States|-7|apartment| +717|AAAAAAAANMCAAAAA|420|Park |Street|Suite L|Oakwood|Lamar County|GA|30169|United States|-5|single family| +718|AAAAAAAAOMCAAAAA|718|Maple |Ave|Suite 80|Enterprise|Taos County|NM|81757|United States|-7|apartment| +719|AAAAAAAAPMCAAAAA|458|7th |Ln|Suite 110|Florence|Monroe County|MI|43394|United States|-5|apartment| +720|AAAAAAAAANCAAAAA|753|Ridge Park|Way|Suite 160|Crossroads|Kitsap County|WA|90534|United States|-8|single family| +721|AAAAAAAABNCAAAAA|579|Elm |Drive|Suite G|Forest Hills|Clearwater County|ID|89237|United States|-7|condo| +722|AAAAAAAACNCAAAAA|669|River Second|Blvd|Suite 40|Deerfield|Liberty County|TX|79840|United States|-6|apartment| +723|AAAAAAAADNCAAAAA|591|Cedar |Circle|Suite N|Allison|Ventura County|CA|94167|United States|-8|apartment| +724|AAAAAAAAENCAAAAA|732|Pine 5th|Ln|Suite I|Sunnyside|Castro County|TX|71952|United States|-6|condo| +725|AAAAAAAAFNCAAAAA|343|Jefferson |Way|Suite 100|Jackson|Marion County|AL|39583|United States|-6|apartment| +726|AAAAAAAAGNCAAAAA|332|Main Hill|Circle|Suite D|Springfield|Gilchrist County|FL|39303|United States|-5|condo| +727|AAAAAAAAHNCAAAAA|806|Sixth River|Road|Suite 190|Pleasant Hill|Meigs County|OH|43604|United States|-5|apartment| +728|AAAAAAAAINCAAAAA|808|11th |Way|Suite H|Golden|Tift County|GA|30411|United States|-5|single family| +729|AAAAAAAAJNCAAAAA|621||Boulevard|Suite K|Leesville|Brewster County|TX|||-6|| +730|AAAAAAAAKNCAAAAA|93|5th Sunset|Ln|Suite P|Valley View|Attala County|MS|55124|United States|-6|condo| +731|AAAAAAAALNCAAAAA|468|2nd |ST|Suite 360|Willow|Rock County|WI|56798|United States|-6|condo| +732|AAAAAAAAMNCAAAAA|673|1st |Pkwy|Suite F|Cedar Grove|Lafayette Parish|LA|70411|United States|-6|condo| +733|AAAAAAAANNCAAAAA|672|Lee |Drive|Suite L|Peoria|Blount County|TN|39818|United States|-5|condo| +734|AAAAAAAAONCAAAAA|877|Park |Pkwy|Suite L|Clinton|Harlan County|NE|68222|United States|-6|condo| +735|AAAAAAAAPNCAAAAA|981|Central |Circle|Suite 390|Mount Zion|Vilas County|WI|58054|United States|-6|condo| +736|AAAAAAAAAOCAAAAA|574|Ridge South|Cir.|Suite 450|Harmony|Elko County|NV|85804|United States|-8|condo| +737|AAAAAAAABOCAAAAA|884|10th Oak|Drive|Suite 140|Belmont|Currituck County|NC|20191|United States|-5|single family| +738|AAAAAAAACOCAAAAA|334|Sycamore |Ln|Suite 400|Glendale|Gray County|KS|63951|United States|-6|single family| +739|AAAAAAAADOCAAAAA|287|Pine Mill|Cir.|Suite 300|Woodlawn|Wood County|WI|54098|United States|-6|apartment| +740|AAAAAAAAEOCAAAAA|790|Second Washington|RD|Suite 280|Concord|Clinton County|OH|44107|United States|-5|single family| +741|AAAAAAAAFOCAAAAA|545|5th Church|Ln|Suite 470|Salem|Henry County|TN|38048|United States|-5|apartment| +742|AAAAAAAAGOCAAAAA|856|Chestnut Jefferson|Lane|Suite X|Wilson|Decatur County|IN|46971|United States|-5|condo| +743|AAAAAAAAHOCAAAAA|514|Maple Ninth|Parkway|Suite 300|Franklin|Brunswick County|NC|29101|United States|-5|single family| +744|AAAAAAAAIOCAAAAA||||Suite 110|||KY||||| +745|AAAAAAAAJOCAAAAA|534|Spring 8th|Dr.|Suite 350|Marion|Harper County|OK|70399|United States|-6|apartment| +746|AAAAAAAAKOCAAAAA|428|6th |Blvd|Suite 30|Valley View|Dakota County|NE|65124|United States|-6|single family| +747|AAAAAAAALOCAAAAA|362|Franklin |Court|Suite 460|Thompsonville|Wasco County|OR|99651|United States|-8|apartment| +748|AAAAAAAAMOCAAAAA|316|Locust Cedar|Way|Suite S|Brownsville|Allen County|IN|49310|United States|-5|apartment| +749|AAAAAAAANOCAAAAA|160|Birch Ridge|Court|Suite F|Shiloh|Okanogan County|WA|99275|United States|-8|condo| +750|AAAAAAAAOOCAAAAA|175|Fifth |Avenue|Suite 170|Oakland|Power County|ID|89843|United States|-7|apartment| +751|AAAAAAAAPOCAAAAA|775|Walnut 3rd|Parkway|Suite 240|Mount Olive|Morgan County|WV|28059|United States|-5|apartment| +752|AAAAAAAAAPCAAAAA|933|12th |Wy|Suite K|Spring Hill|Hartley County|TX|76787|United States|-6|condo| +753|AAAAAAAABPCAAAAA|293|Green |Lane|Suite 410|Valley View|Rock County|WI|55124|United States|-6|single family| +754|AAAAAAAACPCAAAAA|561|East 4th|Road|Suite E|Red Hill|Reno County|KS|64338|United States|-6|condo| +755|AAAAAAAADPCAAAAA|457|6th |Ct.|Suite 0|Mount Zion|Kiowa County|CO|88054|United States|-7|single family| +756|AAAAAAAAEPCAAAAA|917|View Willow|Drive|Suite W|Waterloo|Pulaski County|MO|61675|United States|-6|apartment| +757|AAAAAAAAFPCAAAAA|704|Hillcrest |RD|Suite 310|Marion|Levy County|FL|30399|United States|-5|single family| +758|AAAAAAAAGPCAAAAA|204|Maple |Road|Suite T|Newtown|Caldwell County|MO|61749|United States|-6|condo| +759|AAAAAAAAHPCAAAAA|290|5th |Road|Suite A|Oakdale|Brunswick County|NC|29584|United States|-5|single family| +760|AAAAAAAAIPCAAAAA|176|Seventh Cedar|Way|Suite Y|Wilson|Le Sueur County|MN|56971|United States|-6|single family| +761|AAAAAAAAJPCAAAAA|51|Hill Oak|Lane|Suite P|Hillcrest|Taylor County|WI|53003|United States|-6|condo| +762|AAAAAAAAKPCAAAAA|601|South Third|Boulevard|Suite W|Oak Hill|Telfair County|GA|37838|United States|-5|apartment| +763|AAAAAAAALPCAAAAA|724|Laurel Chestnut|Dr.|Suite U|Greenfield|Florence County|WI|55038|United States|-6|apartment| +764|AAAAAAAAMPCAAAAA|517|Hill |Pkwy|Suite V|Greenfield|Cheyenne County|CO|85038|United States|-7|single family| +765|AAAAAAAANPCAAAAA|176|Jackson |Circle|Suite 460|Salem|Heard County|GA|38048|United States|-5|condo| +766|AAAAAAAAOPCAAAAA|649|6th 2nd|Boulevard|Suite 80|Oak Grove|Faribault County|MN|58370|United States|-6|single family| +767|AAAAAAAAPPCAAAAA|130|Oak River|Avenue|Suite H|Wildwood|San Joaquin County|CA|96871|United States|-8|single family| +768|AAAAAAAAAADAAAAA|479|Johnson Fourth|Road|Suite L|Woodville|Camden County|MO|64289|United States|-6|single family| +769|AAAAAAAABADAAAAA|388|Mill |Lane|Suite 70|Macedonia|Ashland County|OH|41087|United States|-5|condo| +770|AAAAAAAACADAAAAA|220|Chestnut 4th|RD|Suite 300|Oak Ridge|Pipestone County|MN|58371|United States|-6|apartment| +771|AAAAAAAADADAAAAA|555|Church |Lane|Suite T|Valley View|Dickson County|TN|35124|United States|-5|condo| +772|AAAAAAAAEADAAAAA|664|Dogwood |Street|Suite 460|Florence|McDuffie County|GA|33394|United States|-5|condo| +773|AAAAAAAAFADAAAAA|606|Adams |RD|Suite B|Mount Zion|Carroll County|OH|48054|United States|-5|condo| +774|AAAAAAAAGADAAAAA|915|Hill |ST|Suite 400|Shiloh|Laurel County|KY|49275|United States|-5|single family| +775|AAAAAAAAHADAAAAA|415|Poplar |Lane|Suite 130|Marion|Burke County|NC|20399|United States|-5|condo| +776|AAAAAAAAIADAAAAA|315|Walnut 9th|Circle|Suite 230|Oakwood|Richardson County|NE|60169|United States|-7|apartment| +777|AAAAAAAAJADAAAAA|64|Chestnut |Drive|Suite R|Sulphur Springs|Stanton County|NE|68354|United States|-7|single family| +778|AAAAAAAAKADAAAAA|87|Lakeview Locust|Cir.|Suite 180|La Grange|DeKalb County|TN|37941|United States|-5|condo| +779|AAAAAAAALADAAAAA|580|Oak |Road|Suite 290|Green Acres|Chippewa County|MI|47683|United States|-5|condo| +780|AAAAAAAAMADAAAAA|683|View Maple|Boulevard|Suite Y|Oakdale|Washington County|IN|49584|United States|-5|single family| +781|AAAAAAAANADAAAAA|475|View |Lane|Suite 190|Bloomingdale|Carroll County|AR|71824|United States|-6|condo| +782|AAAAAAAAOADAAAAA|701|Lee Davis|ST|Suite U|Red Hill|Wayne County|WV|24338|United States|-5|condo| +783|AAAAAAAAPADAAAAA|515|Forest |Boulevard|Suite 200|Macedonia|Jackson County|AR|71087|United States|-6|apartment| +784|AAAAAAAAABDAAAAA|698|Jefferson |Drive|Suite I|Stringtown|Dillingham Census Area|AK|90162|United States|-9|single family| +785|AAAAAAAABBDAAAAA|490|Main |Ct.|Suite 440|Oakdale|Chattahoochee County|GA|39584|United States|-5|apartment| +786|AAAAAAAACBDAAAAA|852|Park Franklin|Circle|Suite 350|Union|Taos County|NM|88721|United States|-7|apartment| +787|AAAAAAAADBDAAAAA|739|Forest |Drive|Suite F|Wildwood|Cooke County|TX|76871|United States|-6|condo| +788|AAAAAAAAEBDAAAAA|925|Sixth 7th|Ct.|Suite S|Arlington|Robertson County|KY|46557|United States|-5|single family| +789|AAAAAAAAFBDAAAAA|339|Eigth |RD|Suite K|Howell|Rusk County|TX|74854|United States|-6|single family| +790|AAAAAAAAGBDAAAAA|499|13th 3rd|Parkway|Suite X|Bridgeport|Jackson County|GA|35817|United States|-5|condo| +791|AAAAAAAAHBDAAAAA|418|Spring Chestnut|Circle|Suite R|Peoria|Thurston County|NE|69818|United States|-7|single family| +792|AAAAAAAAIBDAAAAA|849|View |Dr.|Suite 390|Concord|Lincoln County|MO|64107|United States|-6|condo| +793|AAAAAAAAJBDAAAAA|260|1st |Ct.|Suite I|Highland|Parke County|IN|49454|United States|-5|condo| +794|AAAAAAAAKBDAAAAA|984|12th Ridge|Avenue|Suite 230|Spring Hill|Pend Oreille County|WA|96787|United States|-8|apartment| +795|AAAAAAAALBDAAAAA|889|||Suite 260||Martin County||32808|United States||single family| +796|AAAAAAAAMBDAAAAA|781|7th Washington|Court|Suite J|Poplar Grove|Henry County|KY|43664|United States|-6|condo| +797|AAAAAAAANBDAAAAA|620|Lakeview Spruce|Blvd|Suite M|Midway|Clallam County|WA|91904|United States|-8|single family| +798|AAAAAAAAOBDAAAAA|722|12th View|Parkway|Suite 400|Fairfield|Fairfax city|VA|26192|United States|-5|single family| +799|AAAAAAAAPBDAAAAA|638|Church |Wy|Suite W|Woodlawn|Big Horn County|WY|84098|United States|-7|condo| +800|AAAAAAAAACDAAAAA|296|1st Railroad|Wy|Suite Q|Valley View|Pasquotank County|NC|25124|United States|-5|condo| +801|AAAAAAAABCDAAAAA|613|Main |RD|Suite 110|Antioch|Augusta County|VA|28605|United States|-5|condo| +802|AAAAAAAACCDAAAAA||Ash |Cir.|Suite 180|Spring Valley||||United States|-7|| +803|AAAAAAAADCDAAAAA|187|West 10th|Circle|Suite 70|Hillcrest|Cattaraugus County|NY|13003|United States|-5|apartment| +804|AAAAAAAAECDAAAAA|47|South Lakeview|Circle|Suite 30|White Oak|Clarion County|PA|16668|United States|-5|apartment| +805|AAAAAAAAFCDAAAAA|149|East 4th|Ave|Suite 420|Bunker Hill|Boyle County|KY|40150|United States|-6|condo| +806|AAAAAAAAGCDAAAAA|143|Johnson |Dr.|Suite I|Glenwood|Staunton city|VA|23511|United States|-5|condo| +807|AAAAAAAAHCDAAAAA|823|Miller Seventh|Way|Suite I|Brownsville|Adams County|ID|89310|United States|-7|single family| +808|AAAAAAAAICDAAAAA|865|13th |Parkway|Suite 340|Woodbury|Lee County|AL|34489|United States|-6|condo| +809|AAAAAAAAJCDAAAAA|216|Elm |Drive|Suite 90|Highland Park|Richmond County|NC|26534|United States|-5|apartment| +810|AAAAAAAAKCDAAAAA|113|Main |Street|Suite 150|Riverview|Crawford County|GA|39003|United States|-5|apartment| +811|AAAAAAAALCDAAAAA|777|Walnut |Dr.|Suite 150|Crossroads|Butts County|GA|30534|United States|-5|apartment| +812|AAAAAAAAMCDAAAAA|880|4th Oak|Wy|Suite 370|Mount Olive|Republic County|KS|68059|United States|-6|condo| +813|AAAAAAAANCDAAAAA|650|Franklin Sixth|Circle|Suite 250|Woodlawn|Walworth County|SD|54098|United States|-7|apartment| +814|AAAAAAAAOCDAAAAA|400|Dogwood 4th|Circle|Suite 220|Bridgeport|Osceola County|IA|55817|United States|-6|apartment| +815|AAAAAAAAPCDAAAAA|473|Johnson |Road|Suite I|Concord|Spotsylvania County|VA|24107|United States|-5|apartment| +816|AAAAAAAAADDAAAAA|513|Ash Center|Way|Suite R|Riverdale|Livingston County|KY|49391|United States|-5|apartment| +817|AAAAAAAABDDAAAAA|503|Ridge |Boulevard|Suite 60|Midway|Dutchess County|NY|11904|United States|-5|single family| +818|AAAAAAAACDDAAAAA|175|Railroad Madison|Pkwy|Suite 380|Woodland|Sac County|IA|54854|United States|-6|single family| +819|AAAAAAAADDDAAAAA|356|Woodland |Lane|Suite 250|Sunnyside|Goodhue County|MN|51952|United States|-6|condo| +820|AAAAAAAAEDDAAAAA|888|5th Ridge|Ct.|Suite P|Sulphur Springs|Yates County|NY|18354|United States|-5|apartment| +821|AAAAAAAAFDDAAAAA|393|5th |Pkwy|Suite D|Mount Olive|Sonoma County|CA|98059|United States|-8|condo| +822|AAAAAAAAGDDAAAAA||Willow 7th|Ct.|Suite E|Spring Hill|Pike County|||United States|-5|| +823|AAAAAAAAHDDAAAAA|514|Highland Madison|ST|Suite A|Lewisburg|Luna County|NM|87538|United States|-7|apartment| +824|AAAAAAAAIDDAAAAA|66|Fifteenth |Pkwy|Suite L|Walnut Grove|Graham County|KS|67752|United States|-6|single family| +825|AAAAAAAAJDDAAAAA|283|Main |Boulevard|Suite N|Oak Hill|La Salle County|TX|77838|United States|-6|apartment| +826|AAAAAAAAKDDAAAAA|119|First Fourth|Dr.|Suite 280|Bridgeport|Limestone County|AL|35817|United States|-6|apartment| +827|AAAAAAAALDDAAAAA|30|Forest 14th|Court|Suite Y|Shady Grove|Cherokee County|TX|72812|United States|-6|apartment| +828|AAAAAAAAMDDAAAAA|175|Cedar 9th|Court|Suite 30|New Hope|Chickasaw County|IA|59431|United States|-6|single family| +829|AAAAAAAANDDAAAAA|8|Spruce 5th|Lane|Suite Y|Riverdale|Whitman County|WA|99391|United States|-8|apartment| +830|AAAAAAAAODDAAAAA|557|5th |Ave|Suite 30|Providence|Pope County|AR|76614|United States|-6|condo| +831|AAAAAAAAPDDAAAAA|240|Green |Blvd|Suite 20|Maple Hill|Henderson County|IL|68095|United States|-6|condo| +832|AAAAAAAAAEDAAAAA|556|Center Main|RD|Suite R|White Oak|Marion County|IL|66668|United States|-6|single family| +833|AAAAAAAABEDAAAAA|349|Mill View|Circle|Suite T|Union Hill|Pipestone County|MN|57746|United States|-6|single family| +834|AAAAAAAACEDAAAAA|715|15th |Street|Suite W|Antioch|Brazos County|TX|78605|United States|-6|condo| +835|AAAAAAAADEDAAAAA|869|12th |Ct.|Suite 90|Waterloo|Arlington County|VA|21675|United States|-5|condo| +836|AAAAAAAAEEDAAAAA|125|Highland |ST|Suite Y|Glendale|Meade County|SD|53951|United States|-7|apartment| +837|AAAAAAAAFEDAAAAA|473|Hillcrest Miller|Street|Suite Y|Lewisburg|Clay County|IN|47538|United States|-5|apartment| +838|AAAAAAAAGEDAAAAA|583|Fourth |Blvd|Suite W|Mount Zion|La Crosse County|WI|58054|United States|-6|single family| +839|AAAAAAAAHEDAAAAA|38|6th 8th|Pkwy|Suite 320|Fairview|Latah County|ID|85709|United States|-7|condo| +840|AAAAAAAAIEDAAAAA|436|Cedar |Wy|Suite 120|Macedonia|Greene County|IN|41087|United States|-5|condo| +841|AAAAAAAAJEDAAAAA|762|Broadway |Avenue|Suite P|Clifton|Liberty County|MT|68014|United States|-7|condo| +842|AAAAAAAAKEDAAAAA|363|Ridge |ST|Suite X|Walnut Grove|Missaukee County|MI|47752|United States|-5|condo| +843|AAAAAAAALEDAAAAA|23|Park Main|Lane|Suite J|Woodville|Cherokee County|IA|54289|United States|-6|condo| +844|AAAAAAAAMEDAAAAA|631|Central North|Pkwy|Suite W|Fairfield|Sandusky County|OH|46192|United States|-5|apartment| +845|AAAAAAAANEDAAAAA|241|12th Lakeview|Boulevard|Suite 290|Oak Ridge|Palo Alto County|IA|58371|United States|-6|condo| +846|AAAAAAAAOEDAAAAA|330|Sycamore Smith|Cir.|Suite 430|Macedonia|Clearwater County|ID|81087|United States|-7|single family| +847|AAAAAAAAPEDAAAAA|235|Poplar Hickory|Court|Suite 310|Belmont|Osborne County|KS|60191|United States|-6|single family| +848|AAAAAAAAAFDAAAAA|548|Birch |Lane|Suite G|Newtown|Hertford County|NC|21749|United States|-5|condo| +849|AAAAAAAABFDAAAAA|103|1st Cedar|Way|Suite 290|Plainview|Ketchikan Gateway Borough|AK|93683|United States|-9|condo| +850|AAAAAAAACFDAAAAA|776|Hillcrest |Pkwy|Suite C|Deerfield|McLeod County|MN|59840|United States|-6|apartment| +851|AAAAAAAADFDAAAAA|669|Hillcrest 14th|Avenue|Suite 210|Five Points|Elko County|NV|86098|United States|-8|apartment| +852|AAAAAAAAEFDAAAAA|270|Lake |Way|Suite W|Oakdale|Wayne County|TN|39584|United States|-6|condo| +853|AAAAAAAAFFDAAAAA|38|Maple |Blvd|Suite 260|Clinton|Brunswick County|VA|28222|United States|-5|condo| +854|AAAAAAAAGFDAAAAA|203|Railroad |Wy|Suite U|Pleasant Valley|Manistee County|MI|42477|United States|-5|apartment| +855|AAAAAAAAHFDAAAAA|114|Walnut Maple|Lane|Suite V|Newport|Wright County|MN|51521|United States|-6|apartment| +856|AAAAAAAAIFDAAAAA|176|Center Ash|Road|Suite 280|Clifton|Franklin County|AR|78014|United States|-6|apartment| +857|AAAAAAAAJFDAAAAA|964|7th Green|Road|Suite 50|Georgetown|Jefferson County|IN|47057|United States|-5|single family| +858|AAAAAAAAKFDAAAAA|658|College |Street|Suite Q|Oakwood|Richland County|WI|50169|United States|-6|single family| +859|AAAAAAAALFDAAAAA|48|Fifth |Blvd|Suite J|Unionville|Poinsett County|AR|71711|United States|-6|single family| +860|AAAAAAAAMFDAAAAA|998|Miller Mill|Pkwy|Suite H|Forest Hills|Atascosa County|TX|79237|United States|-6|single family| +861|AAAAAAAANFDAAAAA|686|Jackson |Drive|Suite 430|Crossroads|Frederick County|VA|20534|United States|-5|condo| +862|AAAAAAAAOFDAAAAA|269|Second Pine|Circle|Suite W|Unionville|Arlington County|VA|21711|United States|-5|apartment| +863|AAAAAAAAPFDAAAAA|315|Hill |Ct.|Suite E|Bunker Hill|Darlington County|SC|20150|United States|-5|condo| +864|AAAAAAAAAGDAAAAA|865|Railroad Fifth|Wy|Suite S|Crossroads|Lackawanna County|PA|10534|United States|-5|single family| +865|AAAAAAAABGDAAAAA|216|Jefferson 4th|Boulevard|Suite X|Centerville|Sedgwick County|CO|80059|United States|-7|single family| +866|AAAAAAAACGDAAAAA|992|3rd Meadow|Blvd|Suite 190|Riverview|Clarke County|IA|59003|United States|-6|single family| +867|AAAAAAAADGDAAAAA|416|10th |Boulevard|Suite R|Calhoun|Miller County|GA|36909|United States|-5|condo| +868|AAAAAAAAEGDAAAAA|50|Oak Jackson|Ave|Suite 270|Green Acres|Huron County|OH|47683|United States|-5|single family| +869|AAAAAAAAFGDAAAAA|821|7th Main|ST|Suite 200|Woodruff|Greene County|MO|64174|United States|-6|single family| +870|AAAAAAAAGGDAAAAA|271|Jefferson |RD|Suite L|Greenville|Letcher County|KY|41387|United States|-5|single family| +871|AAAAAAAAHGDAAAAA|690|River 4th|RD|Suite 100|Plainview|Clark County|MO|63683|United States|-6|condo| +872|AAAAAAAAIGDAAAAA|943|8th |Lane|Suite W|Union|Wheeler County|GA|38721|United States|-5|single family| +873|AAAAAAAAJGDAAAAA|565|Johnson Smith|Boulevard|Suite 120|Mount Vernon|Perry County|PA|18482|United States|-5|apartment| +874|AAAAAAAAKGDAAAAA|601|Green |Dr.|Suite 260|Mountain View|Forsyth County|NC|24466|United States|-5|apartment| +875|AAAAAAAALGDAAAAA|159|Seventh Hickory|Boulevard|Suite S|Sunnyside|Essex County|VA|21952|United States|-5|apartment| +876|AAAAAAAAMGDAAAAA|333|Chestnut |Avenue|Suite 130|Brownsville|Bergen County|NJ|09910|United States|-5|single family| +877|AAAAAAAANGDAAAAA|444|Lake |Boulevard|Suite A|Marion|Macon County|GA|30399|United States|-5|single family| +878|AAAAAAAAOGDAAAAA|294|Highland Main|Boulevard|Suite 250|Highland|Washington County|RI|09454|United States|-5|single family| +879|AAAAAAAAPGDAAAAA|626|Seventh 5th|Ave|Suite U|Oakland|Alamance County|NC|29843|United States|-5|apartment| +880|AAAAAAAAAHDAAAAA|302|Fourth |Avenue|Suite 460|Newport|Carson City|NV|81521|United States|-8|single family| +881|AAAAAAAABHDAAAAA|486|Main |Way|Suite 90|Valley View|Florence County|WI|55124|United States|-6|condo| +882|AAAAAAAACHDAAAAA|411|Pine 2nd|Court|Suite 210|Pinecrest|Webster Parish|LA|79981|United States|-6|apartment| +883|AAAAAAAADHDAAAAA|480|Hickory Highland|Blvd|Suite B|Woodland|Alamance County|NC|24854|United States|-5|condo| +884|AAAAAAAAEHDAAAAA|695|7th 4th|Pkwy|Suite R|Spring Hill|Stevens County|KS|66787|United States|-6|apartment| +885|AAAAAAAAFHDAAAAA|309|10th Fifth|RD|Suite 450|Hopewell|Franklin County|IL|60587|United States|-6|apartment| +886|AAAAAAAAGHDAAAAA|874|Maple |Ave|Suite 90|Valley View|Jackson County|MS|55124|United States|-6|condo| +887|AAAAAAAAHHDAAAAA|131|Hill |Cir.|Suite N|Maple Grove|Pipestone County|MN|58252|United States|-6|apartment| +888|AAAAAAAAIHDAAAAA|63|4th North|Parkway|Suite A|Antioch|Otsego County|MI|48605|United States|-5|single family| +889|AAAAAAAAJHDAAAAA|714|3rd |Ln|Suite D|Edgewood|Conecuh County|AL|30069|United States|-6|single family| +890|AAAAAAAAKHDAAAAA|751|Mill Church|Ave|Suite Y|Arlington|Carson County|TX|76557|United States|-6|single family| +891|AAAAAAAALHDAAAAA|233|Davis |Street|Suite 160|Clearview|Passaic County|NJ|06095|United States|-5|apartment| +892|AAAAAAAAMHDAAAAA|702|2nd 1st|Dr.|Suite 490|Highland Park|Mora County|NM|86534|United States|-7|condo| +893|AAAAAAAANHDAAAAA|302|5th 13th|Boulevard|Suite A|Sunnyside|Martin County|TX|71952|United States|-6|condo| +894|AAAAAAAAOHDAAAAA|234|Woodland Main|Avenue|Suite T|Brownsville|Gila County|AZ|89310|United States|-7|apartment| +895|AAAAAAAAPHDAAAAA|460|Ridge 1st|Dr.|Suite 490|Georgetown|Henry County|IL|67057|United States|-6|single family| +896|AAAAAAAAAIDAAAAA|519|10th |Dr.|Suite 140|Harmony|Stephens County|OK|75804|United States|-6|apartment| +897|AAAAAAAABIDAAAAA|665|9th |Boulevard|Suite D|Whispering Pines|Franklin County|KY|47609|United States|-6|apartment| +898|AAAAAAAACIDAAAAA|56|Lakeview |RD|Suite L|Brunswick|Modoc County|CA|94642|United States|-8|apartment| +899|AAAAAAAADIDAAAAA|755|Jefferson Mill|Cir.|Suite 180|Deerfield|Robertson County|TX|79840|United States|-6|condo| +900|AAAAAAAAEIDAAAAA|605|Lakeview Second|ST|Suite 270|Bunker Hill|Pipestone County|MN|50150|United States|-6|apartment| +901|AAAAAAAAFIDAAAAA|268|Lincoln |Circle|Suite 210|Arlington|Sherman County|KS|66557|United States|-6|apartment| +902|AAAAAAAAGIDAAAAA|638|Pine |Ln|Suite 80|Salem|Sioux County|NE|68048|United States|-7|single family| +903|AAAAAAAAHIDAAAAA|276|Wilson |Ave|Suite Y|Somerville|Pulaski County|VA|27783|United States|-5|single family| +904|AAAAAAAAIIDAAAAA|793|Willow |Way|Suite O|Glenwood|Lee County|GA|33511|United States|-5|apartment| +905|AAAAAAAAJIDAAAAA|873|Maple Lake|Cir.|Suite M|Clinton|Hancock County|IL|68222|United States|-6|apartment| +906|AAAAAAAAKIDAAAAA|584|Park Sycamore|Parkway|Suite 420|Parkwood|Mobile County|AL|31669|United States|-6|apartment| +907|AAAAAAAALIDAAAAA|||Avenue||Pleasant Hill|Conejos County||83604|||| +908|AAAAAAAAMIDAAAAA|176|8th |Way|Suite G|Jackson|Faulkner County|AR|79583|United States|-6|condo| +909|AAAAAAAANIDAAAAA|311|Jefferson Park|ST|Suite S|Maple Grove|Pickens County|GA|38252|United States|-5|apartment| +910|AAAAAAAAOIDAAAAA|59|Washington Walnut|Dr.|Suite 430|Brookwood|Cedar County|MO|60965|United States|-6|apartment| +911|AAAAAAAAPIDAAAAA|727|Oak Davis|Way|Suite C|Mount Vernon|Bedford city|VA|28482|United States|-5|apartment| +912|AAAAAAAAAJDAAAAA|153|Meadow |Wy|Suite 490|Crossroads|Ransom County|ND|50534|United States|-6|apartment| +913|AAAAAAAABJDAAAAA|51|Seventh |Ct.|Suite A|Lakeside|Barton County|MO|69532|United States|-6|apartment| +914|AAAAAAAACJDAAAAA|205|8th Chestnut|Cir.|Suite I|Highland|Fresno County|CA|99454|United States|-8|single family| +915|AAAAAAAADJDAAAAA|260|Maple |Pkwy|Suite J|White Oak|Lackawanna County|PA|16668|United States|-5|condo| +916|AAAAAAAAEJDAAAAA|896|Meadow |Cir.|Suite 320|Georgetown|Multnomah County|OR|97057|United States|-8|apartment| +917|AAAAAAAAFJDAAAAA|869|Johnson 3rd|Ave|Suite 150|Belmont|Newberry County|SC|20191|United States|-5|condo| +918|AAAAAAAAGJDAAAAA|549|10th Third|Court|Suite P|Oak Ridge|Sevier County|AR|78371|United States|-6|condo| +919|AAAAAAAAHJDAAAAA|913|1st Fifth|Ct.|Suite 240|Riverdale|Clatsop County|OR|99391|United States|-8|single family| +920|AAAAAAAAIJDAAAAA|334|Pine Smith|Blvd|Suite 180|Lakewood|Clear Creek County|CO|88877|United States|-7|single family| +921|AAAAAAAAJJDAAAAA|674|Maple Valley|Dr.|Suite 180|Enterprise|Walker County|AL|31757|United States|-6|apartment| +922|AAAAAAAAKJDAAAAA|300|Walnut Elevnth|Ln|Suite 240|Friendship|Adams County|CO|84536|United States|-7|apartment| +923|AAAAAAAALJDAAAAA|115|Park |Ct.|Suite B|Woodland|Hawaii County|HI|94854|United States|-10|condo| +924|AAAAAAAAMJDAAAAA|996|Elm Park|Parkway|Suite 280|Clifton|Warren County|NC|28014|United States|-5|condo| +925|AAAAAAAANJDAAAAA|669||Ct.|||Gadsden County|FL|34975|||condo| +926|AAAAAAAAOJDAAAAA|573|7th Hill|Lane|Suite 170|Chestnut Ridge|Franklin County|AL|37334|United States|-6|condo| +927|AAAAAAAAPJDAAAAA|268|3rd |Court||Providence||AK|||-9|| +928|AAAAAAAAAKDAAAAA|221|Central Cedar|Dr.|Suite 240|Marion|Wayne County|NE|60399|United States|-7|condo| +929|AAAAAAAABKDAAAAA|566|Forest |Pkwy|Suite I|Harmony|Huntington County|IN|45804|United States|-5|single family| +930|AAAAAAAACKDAAAAA|915|Elm 2nd|Drive|Suite 190|Riverdale|Kidder County|ND|59391|United States|-6|condo| +931|AAAAAAAADKDAAAAA|320|Jefferson |Wy|Suite F|Pleasant Valley|Lewis and Clark County|MT|62477|United States|-7|apartment| +932|AAAAAAAAEKDAAAAA|537|Park 4th|Road|Suite T|Newport|Rockbridge County|VA|21521|United States|-5|apartment| +933|AAAAAAAAFKDAAAAA|921|Spring |Street|Suite 260|Mount Pleasant|Kiowa County|KS|61933|United States|-6|single family| +934|AAAAAAAAGKDAAAAA|374|Park Washington|Street|Suite 370|Oak Grove|Minnehaha County|SD|58370|United States|-7|apartment| +935|AAAAAAAAHKDAAAAA|467|Meadow |Ct.|Suite 160|Georgetown|Clay County|IN|47057|United States|-5|condo| +936|AAAAAAAAIKDAAAAA|218|Ninth |Cir.|Suite 240|Ruth|Wabaunsee County|KS|60309|United States|-6|single family| +937|AAAAAAAAJKDAAAAA|28|Jefferson |Lane|Suite 190|Clifton|Lincoln County|OR|98014|United States|-8|single family| +938|AAAAAAAAKKDAAAAA|816|Park Locust|Wy|Suite 170|Hamilton|Gaston County|NC|22808|United States|-5|condo| +939|AAAAAAAALKDAAAAA|107|Cherry Maple|Lane|Suite 250|Enterprise|Carter County|OK|71757|United States|-6|single family| +940|AAAAAAAAMKDAAAAA|28|Lincoln Center|Boulevard|Suite Y|Greenwood|Lancaster County|VA|28828|United States|-5|apartment| +941|AAAAAAAANKDAAAAA|780|South South|Ct.|Suite T|Crossroads|Person County|NC|20534|United States|-5|condo| +942|AAAAAAAAOKDAAAAA|606|Railroad Seventh|RD|Suite 170|Highland|Calhoun County|AR|79454|United States|-6|single family| +943|AAAAAAAAPKDAAAAA|118|Wilson Oak|Dr.|Suite E|Red Hill|Mitchell County|KS|64338|United States|-6|condo| +944|AAAAAAAAALDAAAAA|294|Jackson |Ct.|Suite W|New Town|Watauga County|NC|29634|United States|-5|apartment| +945|AAAAAAAABLDAAAAA|647|Second |Boulevard|Suite 340|Riverdale|Russell County|KY|49391|United States|-5|single family| +946|AAAAAAAACLDAAAAA|294|||Suite N|Belmont||KS|60191|||| +947|AAAAAAAADLDAAAAA|28||||Enterprise|San Bernardino County|CA|91757|United States|-8|single family| +948|AAAAAAAAELDAAAAA|669|||Suite 30|Forest Hills||VA|29237|United States|-5|| +949|AAAAAAAAFLDAAAAA|100|Dogwood Green|ST|Suite 10|Rocky Point|Logan County|ND|51209|United States|-6|single family| +950|AAAAAAAAGLDAAAAA|648|Madison Elm|Lane|Suite 0|Waterloo|Traverse County|MN|51675|United States|-6|condo| +951|AAAAAAAAHLDAAAAA|592|Eigth |Ln|Suite L|Oakwood|Presque Isle County|MI|40169|United States|-5|apartment| +952|AAAAAAAAILDAAAAA|342|Hickory |Cir.|Suite 60|Marion|Orange County|VT|00999|United States|-5|single family| +953|AAAAAAAAJLDAAAAA|639|Birch Highland|Blvd|Suite X|Spring Valley|Keith County|NE|66060|United States|-7|single family| +954|AAAAAAAAKLDAAAAA|820|3rd Walnut|Ct.|Suite 390|Hopewell|Hickory County|MO|60587|United States|-6|condo| +955|AAAAAAAALLDAAAAA|332|Walnut |Parkway|Suite S|Frenchtown|Polk County|TN|32629|United States|-6|apartment| +956|AAAAAAAAMLDAAAAA|253|3rd Maple|Wy|Suite Q|Pleasant Grove|Coos County|NH|04736|United States|-5|single family| +957|AAAAAAAANLDAAAAA|678|Valley 9th|Circle|Suite E|Jackson|Montgomery County|KS|69583|United States|-6|single family| +958|AAAAAAAAOLDAAAAA|530|Sycamore Laurel|Dr.|Suite C|Friendship|Ascension Parish|LA|74536|United States|-6|apartment| +959|AAAAAAAAPLDAAAAA|400|4th |Street|Suite M|Oak Grove|Hyde County|NC|28370|United States|-5|condo| +960|AAAAAAAAAMDAAAAA|780|4th |Ave|Suite T|Spring Hill|Worth County|GA|36787|United States|-5|single family| +961|AAAAAAAABMDAAAAA|866|Park |Pkwy|Suite 240|Oak Grove|Garfield County|OK|78370|United States|-6|single family| +962|AAAAAAAACMDAAAAA|584|8th Main|RD|Suite Y|Pine Grove|Brooks County|TX|74593|United States|-6|condo| +963|AAAAAAAADMDAAAAA|167|Woodland |Pkwy|Suite 400|Deerfield|Bates County|MO|69840|United States|-6|condo| +964|AAAAAAAAEMDAAAAA|242|Jefferson 4th|Street|Suite X|Crossroads|Montgomery County|OH|40534|United States|-5|condo| +965|AAAAAAAAFMDAAAAA|148|1st Birch|Wy|Suite L|Providence|Noble County|IN|46614|United States|-5|condo| +966|AAAAAAAAGMDAAAAA|446|Ash |Ln|Suite R|Jamestown|Billings County|ND|56867|United States|-6|apartment| +967|AAAAAAAAHMDAAAAA|663|Second Lee|Circle|Suite I|Union|Macon County|IL|68721|United States|-6|apartment| +968|AAAAAAAAIMDAAAAA|705|Smith |Ct.|Suite T|Hillcrest|York County|PA|13003|United States|-5|apartment| +969|AAAAAAAAJMDAAAAA|218|Locust |Road|Suite U|Ludlow|Cuming County|NE|65566|United States|-6|condo| +970|AAAAAAAAKMDAAAAA|264|Main |Drive|Suite V|Cedar Grove|Wilcox County|GA|30411|United States|-5|condo| +971|AAAAAAAALMDAAAAA|840|Meadow Main|Ave|Suite 220|Plainview|Dooly County|GA|33683|United States|-5|single family| +972|AAAAAAAAMMDAAAAA||Maple |Circle|Suite Q||Gallatin County|||||single family| +973|AAAAAAAANMDAAAAA|659|Wilson |Parkway|Suite I|Edgewood|Ouachita Parish|LA|70069|United States|-6|condo| +974|AAAAAAAAOMDAAAAA|713|Cedar 3rd|Blvd|Suite R|Pleasant Grove|Pulaski County|GA|34136|United States|-5|condo| +975|AAAAAAAAPMDAAAAA|669|Miller |Avenue|Suite E|Plainview|Christian County|KY|43683|United States|-6|condo| +976|AAAAAAAAANDAAAAA|779|Oak |Blvd|Suite 80|Argyle|Christian County|IL|68722|United States|-6|apartment| +977|AAAAAAAABNDAAAAA|170|River |RD|Suite M|Oak Hill|Wyoming County|PA|17838|United States|-5|apartment| +978|AAAAAAAACNDAAAAA|730|Chestnut |Ave|Suite 400|New Hope|Rooks County|KS|69431|United States|-6|apartment| +979|AAAAAAAADNDAAAAA|250|Maple 4th|Avenue|Suite P|Ashland|Barnwell County|SC|24244|United States|-5|single family| +980|AAAAAAAAENDAAAAA|958|Davis Forest|Court|Suite 110|Newport|Chickasaw County|MS|51521|United States|-6|condo| +981|AAAAAAAAFNDAAAAA|47|Walnut |Ln|Suite 170|Rankin|Fayette County|IN|42621|United States|-5|single family| +982|AAAAAAAAGNDAAAAA|27|6th 10th|ST|Suite H|Glendale|Isanti County|MN|53951|United States|-6|single family| +983|AAAAAAAAHNDAAAAA|519|Center College|Ct.|Suite 200|Newport|Greene County|NY|11521|United States|-5|condo| +984|AAAAAAAAINDAAAAA|928|Willow |Circle|Suite 410|Union|Montgomery County|MO|68721|United States|-6|apartment| +985|AAAAAAAAJNDAAAAA|272|First |Circle|Suite 440|Midway|Johnson County|TX|71904|United States|-6|apartment| +986|AAAAAAAAKNDAAAAA|39|Maple Cedar|Wy|Suite 330|Tanglewood|Bedford County|VA|28994|United States|-5|condo| +987|AAAAAAAALNDAAAAA|73|15th Jackson|Pkwy|Suite C|New Hope|Petersburg city|VA|29431|United States|-5|single family| +988|AAAAAAAAMNDAAAAA|973|West Johnson|Lane|Suite 310|Farmington|Brown County|IL|69145|United States|-6|apartment| +989|AAAAAAAANNDAAAAA|225|Hickory |Circle|Suite 230|Unionville|Fentress County|TN|31711|United States|-5|condo| +990|AAAAAAAAONDAAAAA|644|Poplar Pine|ST|Suite 130|Friendship|Prairie County|AR|74536|United States|-6|single family| +991|AAAAAAAAPNDAAAAA|612|3rd |Circle|Suite 420|Summit|Crawford County|PA|10499|United States|-5|condo| +992|AAAAAAAAAODAAAAA||Laurel ||Suite I||||||-5|apartment| +993|AAAAAAAABODAAAAA|726|River Valley|Wy|Suite 460|Deerfield|DeKalb County|MO|69840|United States|-6|apartment| +994|AAAAAAAACODAAAAA|662|Hill Wilson|Lane|Suite E|Jones|Sarasota County|FL|32686|United States|-5|single family| +995|AAAAAAAADODAAAAA|704|Hillcrest |Boulevard|Suite W|Clinton|Fremont County|CO|88222|United States|-7|single family| +996|AAAAAAAAEODAAAAA|732|Third |Circle|Suite 60|Oakwood|Wise County|TX|70169|United States|-6|condo| +997|AAAAAAAAFODAAAAA||||Suite E|Shady Grove|||72812||-6|| +998|AAAAAAAAGODAAAAA|408|6th |Pkwy|Suite 490|Arlington|Lincoln County|GA|36557|United States|-5|single family| +999|AAAAAAAAHODAAAAA|934|4th Hickory|Street|Suite X|Fayetteville|Miami County|IN|41732|United States|-5|condo| +1000|AAAAAAAAIODAAAAA|287|Cedar Maple|Cir.|Suite 360|Marion|Custer County|OK|70399|United States|-6|single family| +1001|AAAAAAAAJODAAAAA|244|5th 3rd|Avenue|Suite 300|Mount Vernon|Cherokee County|AL|38482|United States|-6|single family| +1002|AAAAAAAAKODAAAAA|692|4th Elm|Dr.|Suite 110|Greenville|Madison County|MT|61387|United States|-7|apartment| +1003|AAAAAAAALODAAAAA|210|Lee Oak|Wy|Suite 20|Belmont|Faulk County|SD|50191|United States|-6|single family| +1004|AAAAAAAAMODAAAAA|235|Ash 13th|Road|Suite I|Wildwood|Dorchester County|MD|26871|United States|-5|single family| +1005|AAAAAAAANODAAAAA|862|Jackson |Ln|Suite X|Pleasant Valley|Beaufort County|SC|22477|United States|-5|single family| +1006|AAAAAAAAOODAAAAA|882|Wilson |Way|Suite 150|Mount Vernon|Bay County|MI|48482|United States|-5|apartment| +1007|AAAAAAAAPODAAAAA|298|Smith |RD|Suite 490|Lakewood|Johnson County|KY|48877|United States|-6|single family| +1008|AAAAAAAAAPDAAAAA|364|12th |Parkway|Suite 80|Greenfield|Daggett County|UT|85038|United States|-7|single family| +1009|AAAAAAAABPDAAAAA|585|4th |Boulevard|Suite 130|Oakdale|Lewis County|TN|39584|United States|-6|condo| +1010|AAAAAAAACPDAAAAA|453|Main Railroad|Street|Suite H|Enterprise|Morgan County|MO|61757|United States|-6|apartment| +1011|AAAAAAAADPDAAAAA|||Court|||Cowlitz County||98828|United States|-8|| +1012|AAAAAAAAEPDAAAAA|19|North |Avenue|Suite 80|Arlington|Shasta County|CA|96557|United States|-8|condo| +1013|AAAAAAAAFPDAAAAA|184|Cedar |Avenue|Suite W|Mount Pleasant|Edwards County|IL|61933|United States|-6|apartment| +1014|AAAAAAAAGPDAAAAA|91|Park |Street|Suite 440|Oak Ridge|Meade County|KY|48371|United States|-5|single family| +1015|AAAAAAAAHPDAAAAA|752|Poplar 6th|RD|Suite S|Brownsville|Saunders County|NE|69310|United States|-7|condo| +1016|AAAAAAAAIPDAAAAA|623|Sunset |Wy|Suite 130|Buena Vista|Dunn County|ND|55752|United States|-6|apartment| +1017|AAAAAAAAJPDAAAAA|223|Park Dogwood|Court|Suite Y|Oak Grove|Leslie County|KY|48370|United States|-5|apartment| +1018|AAAAAAAAKPDAAAAA|954|Mill |Pkwy|Suite 0|Riverview|Llano County|TX|79003|United States|-6|single family| +1019|AAAAAAAALPDAAAAA|110|Lee Jefferson|Blvd|Suite O|Bunker Hill|Kendall County|IL|60150|United States|-6|condo| +1020|AAAAAAAAMPDAAAAA|520|Center |Lane|Suite 270|Newtown|Dallas County|MO|61749|United States|-6|condo| +1021|AAAAAAAANPDAAAAA|751|Ridge ||||Giles County||31387|United States||apartment| +1022|AAAAAAAAOPDAAAAA|333|Walnut |Parkway|Suite 290|Macedonia|Catron County|NM|81087|United States|-7|apartment| +1023|AAAAAAAAPPDAAAAA|882|4th |ST|Suite 160|Lakewood|Chase County|KS|68877|United States|-6|single family| +1024|AAAAAAAAAAEAAAAA|750|Williams Oak|Pkwy|Suite I|Mount Vernon|Deaf Smith County|TX|78482|United States|-6|apartment| +1025|AAAAAAAABAEAAAAA|442|First |Street|Suite 250|Crossroads|Contra Costa County|CA|90534|United States|-8|condo| +1026|AAAAAAAACAEAAAAA|472|2nd Hickory|Cir.|Suite E|Clinton|Fulton County|IL|68222|United States|-6|single family| +1027|AAAAAAAADAEAAAAA|212|Maple |Cir.|Suite 450|Concord|Nassau County|FL|34107|United States|-5|single family| +1028|AAAAAAAAEAEAAAAA|260|Davis |Drive|Suite T|Gum Springs|Concho County|TX|72106|United States|-6|condo| +1029|AAAAAAAAFAEAAAAA|88|10th |RD|Suite 110|Summit|Culpeper County|VA|20499|United States|-5|apartment| +1030|AAAAAAAAGAEAAAAA|571|3rd Wilson|Court|Suite 190|Highland Park|Chesapeake city|VA|26534|United States|-5|condo| +1031|AAAAAAAAHAEAAAAA|881|Oak Ridge|Wy|Suite 390|Riverview|Henry County|IN|49003|United States|-5|apartment| +1032|AAAAAAAAIAEAAAAA|518|River |Ln|Suite 170|Plainview|Newton County|MS|53683|United States|-6|condo| +1033|AAAAAAAAJAEAAAAA|391|Seventh |Blvd|Suite 0|Oakland|De Kalb County|IN|49843|United States|-5|condo| +1034|AAAAAAAAKAEAAAAA|431|Ridge Spring|Wy|Suite 270|Waterloo|Colfax County|NE|61675|United States|-6|apartment| +1035|AAAAAAAALAEAAAAA|419|Valley Maple|Lane|Suite C|White Oak|Saguache County|CO|86668|United States|-7|condo| +1036|AAAAAAAAMAEAAAAA|34|Lake Hickory|Dr.|Suite T|Springdale|Issaquena County|MS|58883|United States|-6|condo| +1037|AAAAAAAANAEAAAAA|148|South |Wy|Suite 310|Wilson|Kimble County|TX|76971|United States|-6|single family| +1038|AAAAAAAAOAEAAAAA|694|Mill First|Wy|Suite 340|Concord|Kit Carson County|CO|84107|United States|-7|condo| +1039|AAAAAAAAPAEAAAAA|358|Hillcrest 10th|RD|Suite L|Sunnyside|Jefferson County|IL|61952|United States|-6|condo| +1040|AAAAAAAAABEAAAAA|354|River 5th|Blvd|Suite 290|Lakewood|Wayne County|IL|68877|United States|-6|single family| +1041|AAAAAAAABBEAAAAA|526|Hillcrest |Street|Suite 440|Oakwood|O-Brien County|IA|50169|United States|-6|apartment| +1042|AAAAAAAACBEAAAAA|774|Park |Parkway|Suite 130|Unionville|Scurry County|TX|71711|United States|-6|single family| +1043|AAAAAAAADBEAAAAA|434|Cherry Sunset|Road|Suite X|Lincoln|La Porte County|IN|41289|United States|-5|single family| +1044|AAAAAAAAEBEAAAAA|498|4th |Drive|Suite 40|Five Points|Petersburg city|VA|26098|United States|-5|single family| +1045|AAAAAAAAFBEAAAAA|678|3rd |Ct.|Suite L|Midway|Polk County|WI|51904|United States|-6|single family| +1046|AAAAAAAAGBEAAAAA|493|Washington Second|Ct.|Suite K|Saratoga|Lawrence County|IL|62123|United States|-6|apartment| +1047|AAAAAAAAHBEAAAAA|2||Cir.|Suite D|Marion|Buena Vista city||20399|||| +1048|AAAAAAAAIBEAAAAA|57|Cedar |Wy|Suite 400|Greenwood|Crawford County|IA|58828|United States|-6|single family| +1049|AAAAAAAAJBEAAAAA|254|Smith |Court|Suite L|Garrison|Virginia Beach city|VA|28767|United States|-5|condo| +1050|AAAAAAAAKBEAAAAA|111|Hillcrest Lincoln|Avenue|Suite M|Centerville|Seneca County|OH|40059|United States|-5|condo| +1051|AAAAAAAALBEAAAAA|369|Cedar |Avenue|Suite 230|Liberty|Cabell County|WV|23451|United States|-5|condo| +1052|AAAAAAAAMBEAAAAA|658|Madison |Circle|Suite 20|Waterloo|Marquette County|WI|51675|United States|-6|single family| +1053|AAAAAAAANBEAAAAA|903|Third Maple|Circle|Suite 210|Tracy|Crockett County|TN|36340|United States|-5|condo| +1054|AAAAAAAAOBEAAAAA|229|Valley |Pkwy|Suite B|Deerfield|Greene County|AL|39840|United States|-6|single family| +1055|AAAAAAAAPBEAAAAA|451|Elm Forest|Dr.|Suite 260|Jamestown|Cedar County|MO|66867|United States|-6|apartment| +1056|AAAAAAAAACEAAAAA|536|Sixth |Way|Suite Q|Royal|Chautauqua County|NY|15819|United States|-5|single family| +1057|AAAAAAAABCEAAAAA|841|Madison |Dr.|Suite N|Stringtown|Mason County|TX|70162|United States|-6|single family| +1058|AAAAAAAACCEAAAAA|475|Johnson |Ct.|Suite J|Harmony|Dearborn County|IN|45804|United States|-5|apartment| +1059|AAAAAAAADCEAAAAA|194|Railroad West|Boulevard|Suite 440|Summit|Franklin County|NY|10499|United States|-5|apartment| +1060|AAAAAAAAECEAAAAA|615|Madison 4th|Ln|Suite 40|Liberty|Mahoning County|OH|43451|United States|-5|condo| +1061|AAAAAAAAFCEAAAAA|383|12th Main|Cir.|Suite 340|White Oak|Bradford County|PA|16668|United States|-5|single family| +1062|AAAAAAAAGCEAAAAA|667|3rd Miller|Blvd|Suite P|Buena Vista|Valencia County|NM|85752|United States|-7|single family| +1063|AAAAAAAAHCEAAAAA|726|Sunset |Blvd|Suite V|Pleasant Grove|Vermilion Parish|LA|74136|United States|-6|single family| +1064|AAAAAAAAICEAAAAA|140|Willow |Ct.|Suite Y|Lakeview|Buchanan County|MO|68579|United States|-6|condo| +1065|AAAAAAAAJCEAAAAA|424|Washington Johnson|Pkwy|Suite 80|Walnut Grove|Nez Perce County|ID|87752|United States|-7|condo| +1066|AAAAAAAAKCEAAAAA|311|Meadow |Dr.|Suite 330|Five Points|Clark County|MO|66098|United States|-6|single family| +1067|AAAAAAAALCEAAAAA|326|Franklin |Cir.|Suite O|Spring Valley|Wadena County|MN|56060|United States|-6|apartment| +1068|AAAAAAAAMCEAAAAA|198|Ridge |Ln|Suite 300|Lakewood|Yolo County|CA|98877|United States|-8|single family| +1069|AAAAAAAANCEAAAAA|956|Oak |Ave|Suite G|Franklin|Shelby County|TN|39101|United States|-6|single family| +1070|AAAAAAAAOCEAAAAA|848|Poplar Park|Drive|Suite T|Mount Zion|Mississippi County|MO|68054|United States|-6|condo| +1071|AAAAAAAAPCEAAAAA|30|Chestnut 7th|RD|Suite 290|Bethesda|Kiowa County|CO|85980|United States|-7|single family| +1072|AAAAAAAAADEAAAAA|684|Willow Meadow|Parkway|Suite 320|Valley View|Bristol Bay Borough|AK|95124|United States|-9|apartment| +1073|AAAAAAAABDEAAAAA|589|1st |Cir.|Suite 440|Pleasant Grove|Carroll County|NH|04736|United States|-5|condo| +1074|AAAAAAAACDEAAAAA|464|13th |Road|Suite 450|Maple Grove|Sunflower County|MS|58252|United States|-6|single family| +1075|AAAAAAAADDEAAAAA|241|3rd Woodland|ST|Suite 110|Centerville|Grayson County|KY|40059|United States|-6|single family| +1076|AAAAAAAAEDEAAAAA|575|East |Ave|Suite 160|Newtown|Harding County|NM|81749|United States|-7|condo| +1077|AAAAAAAAFDEAAAAA|720|West |Ct.|Suite E|Red Hill|Burleigh County|ND|54338|United States|-6|apartment| +1078|AAAAAAAAGDEAAAAA|281|Sycamore |Ln|Suite C|Riverside|Bartholomew County|IN|49231|United States|-5|single family| +1079|AAAAAAAAHDEAAAAA|656|Birch View|Road|Suite D|Woodville|Rensselaer County|NY|14289|United States|-5|apartment| +1080|AAAAAAAAIDEAAAAA|18|Walnut |Dr.|Suite G|Pleasant Hill|Henderson County|KY|43604|United States|-6|single family| +1081|AAAAAAAAJDEAAAAA|921|Lake Fifth|Ln|Suite 370|Adrian|San Mateo County|CA|93301|United States|-8|single family| +1082|AAAAAAAAKDEAAAAA|713|Sixth Elm|Circle|Suite R|Macedonia|Lawrence County|MS|51087|United States|-6|single family| +1083|AAAAAAAALDEAAAAA|565|Sixth |Circle|Suite 400|Lakeview|Livingston County|KY|48579|United States|-5|condo| +1084|AAAAAAAAMDEAAAAA|328|Seventh |ST|Suite 360|Lakewood|Dickinson County|KS|68877|United States|-6|apartment| +1085|AAAAAAAANDEAAAAA|941|Lake Chestnut|Avenue|Suite X|Springdale|Conejos County|CO|88883|United States|-7|single family| +1086|AAAAAAAAODEAAAAA|363|Main |Blvd|Suite R|Berea|Clay County|TN|33804|United States|-5|apartment| +1087|AAAAAAAAPDEAAAAA|384|Franklin |ST|Suite 220|Providence|Charlevoix County|MI|46614|United States|-5|apartment| +1088|AAAAAAAAAEEAAAAA|731|Valley Dogwood|ST|Suite F|Jamestown|Franklin County|OH|46867|United States|-5|condo| +1089|AAAAAAAABEEAAAAA|924|Oak West|Circle|Suite 50|Jackson|Iron County|MO|69583|United States|-6|condo| +1090|AAAAAAAACEEAAAAA|58|3rd |Ln|Suite T|Pleasant Grove|Nuckolls County|NE|64136|United States|-7|single family| +1091|AAAAAAAADEEAAAAA|859|5th Park|RD|Suite 420|Mount Pleasant|Webster Parish|LA|71933|United States|-6|condo| +1092|AAAAAAAAEEEAAAAA|852|Jefferson Washington|Lane|Suite 90|Pine Grove|Ingham County|MI|44593|United States|-5|condo| +1093|AAAAAAAAFEEAAAAA|998|View Cherry|Lane|Suite P|Stringtown|Hardee County|FL|30162|United States|-5|apartment| +1094|AAAAAAAAGEEAAAAA||||Suite 120||||69584|United States||| +1095|AAAAAAAAHEEAAAAA|232|Second |Wy|Suite 120|Louisville|Garza County|TX|74464|United States|-6|condo| +1096|AAAAAAAAIEEAAAAA|826|Ridge |Street|Suite 190|Forest Hills|Webster County|GA|39237|United States|-5|single family| +1097|AAAAAAAAJEEAAAAA|191|Cherry |Ave|Suite T|Shady Grove|Adams County|IN|42812|United States|-5|condo| +1098|AAAAAAAAKEEAAAAA|184|15th |Lane|Suite 220|Mount Vernon|Stephens County|OK|78482|United States|-6|apartment| +1099|AAAAAAAALEEAAAAA|37|Sunset |Lane|Suite 40|Altamont|Ozark County|MO|69387|United States|-6|apartment| +1100|AAAAAAAAMEEAAAAA|318|First |Lane|Suite 290|Kingston|Perry County|AR|74975|United States|-6|single family| +1101|AAAAAAAANEEAAAAA|230|3rd Spring|Road|Suite V|Superior|Craven County|NC|22562|United States|-5|condo| +1102|AAAAAAAAOEEAAAAA|588|Maple |Circle|Suite K|Texas|Haines Borough|AK|93342|United States|-9|single family| +1103|AAAAAAAAPEEAAAAA|628|South Seventh|Avenue|Suite 430|Lincoln|Arenac County|MI|41289|United States|-5|condo| +1104|AAAAAAAAAFEAAAAA|609|2nd |ST|Suite 20|Glencoe|Jefferson County|IL|63396|United States|-6|condo| +1105|AAAAAAAABFEAAAAA|6|2nd Poplar|Boulevard|Suite 180|Warwick|Sierra County|CA|91398|United States|-8|apartment| +1106|AAAAAAAACFEAAAAA|246|Spring |Court|Suite R|Highland Park|Lewis County|WA|96534|United States|-8|single family| +1107|AAAAAAAADFEAAAAA|634|River Pine|Blvd|Suite 30|Mount Olive|Perry County|MS|58059|United States|-6|single family| +1108|AAAAAAAAEFEAAAAA|254|14th |Boulevard|Suite 230|Riverside|Bexar County|TX|79231|United States|-6|condo| +1109|AAAAAAAAFFEAAAAA|591|4th |Ave|Suite 90|Fairview|Gray County|TX|75709|United States|-6|condo| +1110|AAAAAAAAGFEAAAAA|461|Chestnut College|Way|Suite E|Pleasant Grove|Jackson County|AL|34136|United States|-6|condo| +1111|AAAAAAAAHFEAAAAA|908|Wilson |Way|Suite X|Pine Grove|Brunswick County|NC|24593|United States|-5|condo| +1112|AAAAAAAAIFEAAAAA|947|Pine Hillcrest|Way|Suite F|Liberty|Summit County|OH|43451|United States|-5|apartment| +1113|AAAAAAAAJFEAAAAA|313|Seventh |Boulevard|Suite H|Mountain View|Orange County|CA|94466|United States|-8|apartment| +1114|AAAAAAAAKFEAAAAA|133|5th |Street|Suite N|Franklin|Colbert County|AL|39101|United States|-6|single family| +1115|AAAAAAAALFEAAAAA|266|11th |Wy|Suite 470|Pleasant Grove|Georgetown County|SC|24136|United States|-5|single family| +1116|AAAAAAAAMFEAAAAA|620|Jefferson |Avenue|Suite H|Forest Hills|Etowah County|AL|39237|United States|-6|apartment| +1117|AAAAAAAANFEAAAAA|588|North 10th|Ave|Suite P|Bethel|Oswego County|NY|15281|United States|-5|single family| +1118|AAAAAAAAOFEAAAAA|373|Washington |Circle|Suite T|Florence|Lincoln County|OR|93394|United States|-8|condo| +1119|AAAAAAAAPFEAAAAA|285|Eigth |Cir.|Suite I|Shady Grove|Sevier County|UT|82812|United States|-7|single family| +1120|AAAAAAAAAGEAAAAA|600|River |RD|Suite G|Kingston|Jackson County|FL|34975|United States|-5|condo| +1121|AAAAAAAABGEAAAAA|872|Ridge Washington|Pkwy|Suite 200|Waterloo|Wayne County|NC|21675|United States|-5|condo| +1122|AAAAAAAACGEAAAAA|933|Lee |Circle|Suite 280|Mount Vernon|Owsley County|KY|48482|United States|-5|condo| +1123|AAAAAAAADGEAAAAA|462|1st |Ln|Suite 120|Union|Grand County|UT|88721|United States|-7|single family| +1124|AAAAAAAAEGEAAAAA|547|Cherry South|Circle|Suite Y|Stringtown|Cherokee County|OK|70162|United States|-6|single family| +1125|AAAAAAAAFGEAAAAA|100|Mill |Wy|Suite 200|Arlington|Aitkin County|MN|56557|United States|-6|condo| +1126|AAAAAAAAGGEAAAAA|840|Miller |Boulevard|Suite L|Oakland|Tuolumne County|CA|99843|United States|-8|single family| +1127|AAAAAAAAHGEAAAAA|926|6th |Road|Suite F|Woodlawn|Madison County|AR|74098|United States|-6|single family| +1128|AAAAAAAAIGEAAAAA|129|Fifth College|Dr.|Suite B|Five Points|Starke County|IN|46098|United States|-5|condo| +1129|AAAAAAAAJGEAAAAA|78|Lincoln 3rd|Court|Suite 480|Pleasant Hill|Page County|IA|53604|United States|-6|condo| +1130|AAAAAAAAKGEAAAAA|973|Lakeview |Dr.|Suite 340|Oakwood|Bath County|VA|20169|United States|-5|condo| +1131|AAAAAAAALGEAAAAA|639|Pine Hill|Court|Suite 50|Lenox|Vermillion County|IN|41143|United States|-5|single family| +1132|AAAAAAAAMGEAAAAA|853|Seventh Franklin|Parkway|Suite W|Union|Baker County|GA|38721|United States|-5|condo| +1133|AAAAAAAANGEAAAAA|664|4th Main|Way|Suite 390|Belmont|Dakota County|MN|50191|United States|-6|condo| +1134|AAAAAAAAOGEAAAAA||Maple |||Fairview|||15709|United States||| +1135|AAAAAAAAPGEAAAAA|676|Lee Oak|Cir.|Suite 90|Fairview|Kent County|DE|15709|United States|-5|condo| +1136|AAAAAAAAAHEAAAAA|16|Adams |Way|Suite 90|Greenwood|Treasure County|MT|68828|United States|-7|condo| +1137|AAAAAAAABHEAAAAA|218|Birch Sunset|Street|Suite 440|Edgewood|Anderson County|TN|30069|United States|-5|condo| +1138|AAAAAAAACHEAAAAA|134|Jackson Ash|Drive|Suite 280|Georgetown|Holmes County|FL|37057|United States|-5|single family| +1139|AAAAAAAADHEAAAAA|387|Lake Central|Pkwy|Suite L|Centerville|Webster County|NE|60059|United States|-7|apartment| +1140|AAAAAAAAEHEAAAAA|854|Cherry |ST|Suite F|Oak Grove|Screven County|GA|38370|United States|-5|condo| +1141|AAAAAAAAFHEAAAAA|624|Spruce Jefferson|Pkwy|Suite N|Woodland|Platte County|MO|64854|United States|-6|single family| +1142|AAAAAAAAGHEAAAAA|665|Franklin Elm|Road|Suite Y|Union Hill|Seward County|NE|67746|United States|-7|condo| +1143|AAAAAAAAHHEAAAAA||Church ||||||59003|United States|-6|| +1144|AAAAAAAAIHEAAAAA|453|North |Avenue|Suite 170|Clinton|Grant County|WV|28222|United States|-5|single family| +1145|AAAAAAAAJHEAAAAA|244|12th |Avenue|Suite I|Providence|Elbert County|CO|86614|United States|-7|apartment| +1146|AAAAAAAAKHEAAAAA|871|Ridge Central|Parkway|Suite X|Ashland|Cedar County|NE|64244|United States|-6|apartment| +1147|AAAAAAAALHEAAAAA|415|Second South|RD|Suite 450|Newport|Erath County|TX|71521|United States|-6|condo| +1148|AAAAAAAAMHEAAAAA|772|Hickory 1st|Way|Suite W|Union Hill|Macon County|AL|37746|United States|-6|condo| +1149|AAAAAAAANHEAAAAA|193|Jackson |Ct.|Suite F|Springfield|Lemhi County|ID|89303|United States|-7|apartment| +1150|AAAAAAAAOHEAAAAA|873|Lake Broadway|Ave|Suite 450|Cherry Valley|Shelby County|MO|60854|United States|-6|condo| +1151|AAAAAAAAPHEAAAAA|576|East |Avenue|Suite 280|Hillcrest|Cottonwood County|MN|53003|United States|-6|single family| +1152|AAAAAAAAAIEAAAAA|975|6th |ST|Suite 240|Clifton|Hood County|TX|78014|United States|-6|condo| +1153|AAAAAAAABIEAAAAA|697|Sunset Hillcrest|Court|Suite U|Redland|Baldwin County|AL|36343|United States|-6|apartment| +1154|AAAAAAAACIEAAAAA|948|3rd 12th|Road|Suite S|Newtown|Bell County|TX|71749|United States|-6|condo| +1155|AAAAAAAADIEAAAAA|561|Cherry |Blvd|Suite 370|Newport|Pope County|IL|61521|United States|-6|condo| +1156|AAAAAAAAEIEAAAAA|744|Highland |RD|Suite 190|Lakeside|Madison County|MO|69532|United States|-6|condo| +1157|AAAAAAAAFIEAAAAA|916|Cherry Elm|ST|Suite 110|Providence|Clay County|GA|36614|United States|-5|single family| +1158|AAAAAAAAGIEAAAAA|636|South Hill||||Sac County||57838|United States|-6|| +1159|AAAAAAAAHIEAAAAA|81|Green 5th|Boulevard|Suite F|Lincoln|Simpson County|KY|41289|United States|-5|apartment| +1160|AAAAAAAAIIEAAAAA|864|First Main|Ave|Suite O|Marion|Madison County|TN|30399|United States|-6|apartment| +1161|AAAAAAAAJIEAAAAA|364|Main Willow|Wy|Suite 310|Clearview|Beaver County|OK|75495|United States|-6|apartment| +1162|AAAAAAAAKIEAAAAA|128|Wilson |Blvd|Suite F|Clinton|Cameron County|PA|18222|United States|-5|apartment| +1163|AAAAAAAALIEAAAAA|17|East |RD|Suite 280|Fairview|Burleigh County|ND|55709|United States|-6|condo| +1164|AAAAAAAAMIEAAAAA|683|11th |Boulevard|Suite H|Fowler|Mecklenburg County|VA|21083|United States|-5|single family| +1165|AAAAAAAANIEAAAAA|679|Washington |Blvd|Suite 380|Springdale|Rutherford County|TN|38883|United States|-6|apartment| +1166|AAAAAAAAOIEAAAAA|404|Hickory Washington|Circle|Suite M|Oakland|Centre County|PA|19843|United States|-5|apartment| +1167|AAAAAAAAPIEAAAAA|719|Dogwood |Lane|Suite 450|Georgetown|Corson County|SD|57057|United States|-6|apartment| +1168|AAAAAAAAAJEAAAAA|257|Meadow |ST|Suite 290|Walnut Grove|Madera County|CA|97752|United States|-8|single family| +1169|AAAAAAAABJEAAAAA|213|Central Ash|Way|Suite 120|Deerfield|Russell County|VA|29840|United States|-5|condo| +1170|AAAAAAAACJEAAAAA|138|Ash 6th|Circle|Suite 470|Five Forks|Franklin County|OH|42293|United States|-5|apartment| +1171|AAAAAAAADJEAAAAA|395|Franklin Second|Circle|Suite O|Farmington|Vance County|NC|29145|United States|-5|apartment| +1172|AAAAAAAAEJEAAAAA|921|Birch |Cir.|Suite 50|Summerfield|Franklin County|MA|01234|United States|-5|apartment| +1173|AAAAAAAAFJEAAAAA|257|Mill |Blvd|Suite N|Red Hill|Rio Arriba County|NM|84338|United States|-7|condo| +1174|AAAAAAAAGJEAAAAA|188|Ninth Fourth|Wy|Suite 400|Leon|Chester County|PA|10913|United States|-5|condo| +1175|AAAAAAAAHJEAAAAA|650|Green |Circle|Suite 20|Lebanon|Jefferson County|IN|42898|United States|-5|single family| +1176|AAAAAAAAIJEAAAAA||Third ||Suite Q|||||United States|-7|apartment| +1177|AAAAAAAAJJEAAAAA|544|Meadow |Street|Suite 70|Shiloh|Rusk County|WI|59275|United States|-6|apartment| +1178|AAAAAAAAKJEAAAAA|771|Chestnut |Ln|Suite 270|Riverside|Cheyenne County|CO|89231|United States|-7|apartment| +1179|AAAAAAAALJEAAAAA|352|15th Third|Pkwy|Suite 380|Oakwood|Kenosha County|WI|50169|United States|-6|condo| +1180|AAAAAAAAMJEAAAAA|235|10th |Boulevard|Suite 100|Tabor|Sequatchie County|TN|38529|United States|-6|condo| +1181|AAAAAAAANJEAAAAA|257|Elm Laurel|Parkway|Suite 300|Oak Ridge|Yellow Medicine County|MN|58371|United States|-6|condo| +1182|AAAAAAAAOJEAAAAA|899|4th Green|Blvd|Suite D|Springdale|Ellis County|KS|68883|United States|-6|single family| +1183|AAAAAAAAPJEAAAAA|473|2nd |Avenue|Suite 260|Highland Park|Morton County|ND|56534|United States|-6|condo| +1184|AAAAAAAAAKEAAAAA|425|First |Lane|Suite U|Wolf Creek|Ouray County|CO|82455|United States|-7|apartment| +1185|AAAAAAAABKEAAAAA|969|Hill Hickory|Ave|Suite 190|Riverside|Custer County|NE|69231|United States|-6|single family| +1186|AAAAAAAACKEAAAAA|679|River 9th|Lane|Suite 310|Maple Grove|Upson County|GA|38252|United States|-5|single family| +1187|AAAAAAAADKEAAAAA|392|Park College|RD|Suite 170|Liberty|Richland County|OH|43451|United States|-5|apartment| +1188|AAAAAAAAEKEAAAAA|553|1st |Way|Suite B|Waterloo|Madison County|VA|21675|United States|-5|single family| +1189|AAAAAAAAFKEAAAAA|516|Second |ST|Suite E|Fairfield|Dillon County|SC|26192|United States|-5|condo| +1190|AAAAAAAAGKEAAAAA|939|||Suite P||Catawba County|NC||United States||condo| +1191|AAAAAAAAHKEAAAAA|802|Smith |Drive|Suite 0|Oakland|Bradford County|PA|19843|United States|-5|apartment| +1192|AAAAAAAAIKEAAAAA|408|Hickory |Pkwy|Suite M|Pine Grove|Iowa County|IA|54593|United States|-6|condo| +1193|AAAAAAAAJKEAAAAA|577|Forest Birch|Dr.|Suite 280|Liberty|Dawson County|MT|63451|United States|-7|apartment| +1194|AAAAAAAAKKEAAAAA|898|Pine |ST|Suite A|Centerville|Claiborne County|TN|30059|United States|-5|apartment| +1195|AAAAAAAALKEAAAAA|192|Sunset |Street|Suite T|Greenwood|Lawrence County|MO|68828|United States|-6|apartment| +1196|AAAAAAAAMKEAAAAA|749|Locust Johnson|Ct.|Suite J|Newtown|Dundy County|NE|61749|United States|-6|apartment| +1197|AAAAAAAANKEAAAAA|114|Maple |Ln|Suite W|Crossroads|Perry County|AL|30534|United States|-6|apartment| +1198|AAAAAAAAOKEAAAAA|525|Eigth |Drive|Suite X|Pine Grove|Montgomery County|IN|44593|United States|-5|single family| +1199|AAAAAAAAPKEAAAAA|157|Mill Oak|Street|Suite F|Greenfield|Dawson County|NE|65038|United States|-6|apartment| +1200|AAAAAAAAALEAAAAA|589|Second Birch|Pkwy|Suite 280|Shiloh|Clay County|NE|69275|United States|-6|condo| +1201|AAAAAAAABLEAAAAA|247|North Lakeview|RD|Suite F|Liberty|Grant County|KY|43451|United States|-6|condo| +1202|AAAAAAAACLEAAAAA|153|West Cherry|Way|Suite W|Oakwood|Benton County|TN|30169|United States|-5|condo| +1203|AAAAAAAADLEAAAAA|323|7th |Ave|Suite S|Mountain View|Anderson County|TX|74466|United States|-6|single family| +1204|AAAAAAAAELEAAAAA|499|10th View|Blvd|Suite R|Pleasant Hill|Carver County|MN|53604|United States|-6|apartment| +1205|AAAAAAAAFLEAAAAA|12|2nd Valley|Circle|Suite 340|Bunker Hill|Henry County|VA|20150|United States|-5|apartment| +1206|AAAAAAAAGLEAAAAA|155|Second |Court|Suite 270|Oakwood|Effingham County|IL|60169|United States|-6|single family| +1207|AAAAAAAAHLEAAAAA|737|Main Oak|Road|Suite 40|Pleasant Hill|Mille Lacs County|MN|53604|United States|-6|single family| +1208|AAAAAAAAILEAAAAA|378|Franklin 2nd|Ave|Suite 390|Fairfield|Charlotte County|VA|26192|United States|-5|apartment| +1209|AAAAAAAAJLEAAAAA|||||Arthur||IN|45965|||single family| +1210|AAAAAAAAKLEAAAAA|351|Center |RD|Suite 160|Sunnyside|Mecklenburg County|VA|21952|United States|-5|condo| +1211|AAAAAAAALLEAAAAA|712|15th |Way|Suite 290|Salem|Jefferson County|AL|38048|United States|-6|single family| +1212|AAAAAAAAMLEAAAAA|142|Lakeview |Way|Suite 340|Shady Grove|New Kent County|VA|22812|United States|-5|single family| +1213|AAAAAAAANLEAAAAA|584|Thirteenth Hill|Parkway|Suite 50|Glendale|Duval County|FL|33951|United States|-5|apartment| +1214|AAAAAAAAOLEAAAAA|801|Seventh |Blvd|Suite 430|Montpelier|Oxford County|ME|09530|United States|-5|condo| +1215|AAAAAAAAPLEAAAAA|615|Lee 10th|Circle|Suite I|Waterloo|Lake County|CO|81675|United States|-7|apartment| +1216|AAAAAAAAAMEAAAAA|511|Willow Eigth|Ct.|Suite A|Forest Hills|Harper County|OK|79237|United States|-6|apartment| +1217|AAAAAAAABMEAAAAA|237|First |RD|Suite B|Lakeview|Meriwether County|GA|38579|United States|-5|condo| +1218|AAAAAAAACMEAAAAA|145|Franklin |Drive|Suite B|Jamestown|Mitchell County|TX|76867|United States|-6|condo| +1219|AAAAAAAADMEAAAAA|613|Lincoln |Way|Suite 30|Bunker Hill|Manassas city|VA|20150|United States|-5|single family| +1220|AAAAAAAAEMEAAAAA|313|Meadow |Lane|Suite L|Mount Zion|Douglas County|MN|58054|United States|-6|apartment| +1221|AAAAAAAAFMEAAAAA|42|10th 14th|Lane|Suite V|Watkins|Johnston County|NC|21732|United States|-5|single family| +1222|AAAAAAAAGMEAAAAA|563|Broadway 14th|Circle|Suite 10|Mount Vernon|Stafford County|KS|68482|United States|-6|apartment| +1223|AAAAAAAAHMEAAAAA|229|East Main|Wy|Suite E|Hillcrest|Washington County|NY|13003|United States|-5|condo| +1224|AAAAAAAAIMEAAAAA|199||Ct.||||ID|80499|||single family| +1225|AAAAAAAAJMEAAAAA|21|Maple Lakeview|Dr.|Suite 170|New Hope|Bibb County|GA|39431|United States|-5|apartment| +1226|AAAAAAAAKMEAAAAA|201|Maple |Street|Suite C|Mount Olive|Sharp County|AR|78059|United States|-6|apartment| +1227|AAAAAAAALMEAAAAA|527|Cedar Railroad|RD|Suite U|Georgetown|Henry County|IN|47057|United States|-5|single family| +1228|AAAAAAAAMMEAAAAA|42|Davis |Way|Suite 350|Edgewood|Bledsoe County|TN|30069|United States|-5|apartment| +1229|AAAAAAAANMEAAAAA|967|Third Dogwood|Dr.|Suite E|Mount Olive|Franklin County|TN|38059|United States|-5|apartment| +1230|AAAAAAAAOMEAAAAA|701|Jackson Elm|Blvd|Suite 430|Fairview|Union County|AR|75709|United States|-6|apartment| +1231|AAAAAAAAPMEAAAAA|568|Washington Franklin|Ave|Suite B|Argyle|Chisago County|MN|58722|United States|-6|apartment| +1232|AAAAAAAAANEAAAAA|403|9th |Pkwy|Suite 340|Salem|Pike County|GA|38048|United States|-5|apartment| +1233|AAAAAAAABNEAAAAA|515|13th Green|Ln|Suite 30|Riverdale|Fillmore County|MN|59391|United States|-6|apartment| +1234|AAAAAAAACNEAAAAA|418|14th Lincoln|ST|Suite T|Delmar|Mitchell County|KS|63957|United States|-6|condo| +1235|AAAAAAAADNEAAAAA|338|Pine Tenth|Avenue|Suite L|Providence|Jackson County|MN|56614|United States|-6|apartment| +1236|AAAAAAAAENEAAAAA|86|Lake |Dr.|Suite U|Woodland|Lynn County|TX|74854|United States|-6|condo| +1237|AAAAAAAAFNEAAAAA|859|Cedar |Circle|Suite 330|Glenwood|Bibb County|AL|33511|United States|-6|single family| +1238|AAAAAAAAGNEAAAAA|145|5th Forest|Wy|Suite 230|Whitesville|Roscommon County|MI|45903|United States|-5|condo| +1239|AAAAAAAAHNEAAAAA|269|Lincoln Hill|Street|Suite 350|Highland Park|Norton County|KS|66534|United States|-6|apartment| +1240|AAAAAAAAINEAAAAA|671|5th Second|Ct.|Suite B|Mount Vernon|Burnett County|WI|58482|United States|-6|single family| +1241|AAAAAAAAJNEAAAAA|168|Cedar |Blvd|Suite T|Glendale|Crisp County|GA|33951|United States|-5|single family| +1242|AAAAAAAAKNEAAAAA|318|Madison Elm|Drive|Suite H|Cedar Grove|Vilas County|WI|50411|United States|-6|condo| +1243|AAAAAAAALNEAAAAA|710|Central Wilson|Blvd|Suite 110|Waterloo|Barber County|KS|61675|United States|-6|condo| +1244|AAAAAAAAMNEAAAAA|619|Hill |Ave|Suite 230|Oakwood|Harney County|OR|90169|United States|-8|apartment| +1245|AAAAAAAANNEAAAAA|989|North 2nd|Wy|Suite G|Fairview|Vernon County|WI|55709|United States|-6|single family| +1246|AAAAAAAAONEAAAAA|453|Davis Pine|Way|Suite 410|Wildwood|Peach County|GA|36871|United States|-5|condo| +1247|AAAAAAAAPNEAAAAA|964|7th Lincoln|RD|Suite 120|Ferguson|Pulaski County|KY|41821|United States|-5|condo| +1248|AAAAAAAAAOEAAAAA|172|Lincoln Lakeview|Blvd|Suite Q|Lakeside|Brooks County|GA|39532|United States|-5|single family| +1249|AAAAAAAABOEAAAAA|647|Central Second|ST|Suite 20|Greenwood|Sangamon County|IL|68828|United States|-6|apartment| +1250|AAAAAAAACOEAAAAA|653|Willow 5th|Pkwy|Suite 280|Hamilton|Sullivan County|TN|32808|United States|-6|single family| +1251|AAAAAAAADOEAAAAA|906|Poplar |Dr.|Suite E|Mountain View|Harris County|TX|74466|United States|-6|single family| +1252|AAAAAAAAEOEAAAAA|761|Railroad 5th|Lane|Suite F|Crossroads|Grundy County|IA|50534|United States|-6|apartment| +1253|AAAAAAAAFOEAAAAA|919|Elm |Drive|Suite 260|Springfield|Choctaw County|MS|59303|United States|-6|condo| +1254|AAAAAAAAGOEAAAAA|455|View |Circle|Suite D|Springfield|Tazewell County|IL|69303|United States|-6|condo| +1255|AAAAAAAAHOEAAAAA|73|Smith |Wy|Suite 70|Deerfield|Charles City County|VA|29840|United States|-5|apartment| +1256|AAAAAAAAIOEAAAAA|907|Smith Highland|Pkwy|Suite Y|Woodland|Macon County|MO|64854|United States|-6|apartment| +1257|AAAAAAAAJOEAAAAA|414|Cedar |Dr.|Suite W|Oak Hill|Allegheny County|PA|17838|United States|-5|condo| +1258|AAAAAAAAKOEAAAAA|210|Adams First|Parkway|Suite L|Colfax|Orange County|VT|03165|United States|-5|apartment| +1259|AAAAAAAALOEAAAAA|12|8th 11th|Ave|Suite 230|Lakeside|Brown County|KS|69532|United States|-6|apartment| +1260|AAAAAAAAMOEAAAAA|246|Dogwood 12th|Dr.|Suite 70|Jamestown|Marquette County|MI|46867|United States|-5|single family| +1261|AAAAAAAANOEAAAAA|986|Franklin Maple|Drive|Suite X|Mount Zion|Armstrong County|TX|78054|United States|-6|condo| +1262|AAAAAAAAOOEAAAAA|817|1st |Lane|Suite U|Florence|Grant County|WV|23394|United States|-5|condo| +1263|AAAAAAAAPOEAAAAA|146|Fourth Birch|Blvd|Suite 90|Highland Park|Catoosa County|GA|36534|United States|-5|condo| +1264|AAAAAAAAAPEAAAAA|273|1st |ST|Suite X|Enterprise|Richland County|ND|51757|United States|-6|apartment| +1265|AAAAAAAABPEAAAAA|825|First 12th|Circle|Suite 370|Franklin|Traverse County|MN|59101|United States|-6|single family| +1266|AAAAAAAACPEAAAAA|20|Second Hill|Lane|Suite 320|Waterloo|Oldham County|TX|71675|United States|-6|apartment| +1267|AAAAAAAADPEAAAAA|90|Elm Elm|Boulevard|Suite 400|Brownsville|Lyon County|IA|59310|United States|-6|single family| +1268|AAAAAAAAEPEAAAAA|654|Fifth |ST|Suite 310|Green Acres|Prairie County|MT|67683|United States|-7|single family| +1269|AAAAAAAAFPEAAAAA|893|Main College|Pkwy|Suite F|Unionville|Wabaunsee County|KS|61711|United States|-6|single family| +1270|AAAAAAAAGPEAAAAA|919|3rd Lincoln|Drive|Suite B|Concord|Orange County|IN|44107|United States|-5|apartment| +1271|AAAAAAAAHPEAAAAA|378|Main |Cir.|Suite S|Woodland|Lanier County|GA|34854|United States|-5|condo| +1272|AAAAAAAAIPEAAAAA|569|Hill |Road|Suite V|Midway|Yakima County|WA|91904|United States|-8|apartment| +1273|AAAAAAAAJPEAAAAA|438|2nd |Ave|Suite 280|Hamilton|Jasper County|IL|62808|United States|-6|condo| +1274|AAAAAAAAKPEAAAAA|963|Forest |Ct.|Suite 340|Sunnyside|Powhatan County|VA|21952|United States|-5|condo| +1275|AAAAAAAALPEAAAAA|372|Laurel Laurel|Parkway|Suite V|Bethel|Roanoke County|VA|25281|United States|-5|apartment| +1276|AAAAAAAAMPEAAAAA|480|Hillcrest Third|RD|Suite V|Union|Petersburg city|VA|28721|United States|-5|condo| +1277|AAAAAAAANPEAAAAA|58|Smith |Dr.|Suite M|Jamestown|Harper County|KS|66867|United States|-6|condo| +1278|AAAAAAAAOPEAAAAA|968|Broadway |Court|Suite 60|Fairview|Wells County|ND|55709|United States|-6|condo| +1279|AAAAAAAAPPEAAAAA|558|9th College|Avenue|Suite 470|Rutland|Jasper County|MS|58375|United States|-6|apartment| +1280|AAAAAAAAAAFAAAAA|776|2nd |Way|Suite D|Friendship|Mills County|TX|74536|United States|-6|apartment| +1281|AAAAAAAABAFAAAAA|88|5th Sycamore|Road|Suite 80|Franklin|Richland County|WI|59101|United States|-6|condo| +1282|AAAAAAAACAFAAAAA|689|Tenth |Ln|Suite C|Wildwood|Clay County|GA|36871|United States|-5|single family| +1283|AAAAAAAADAFAAAAA|253|River |Circle|Suite 480|Allentown|Fannin County|TX|71838|United States|-6|single family| +1284|AAAAAAAAEAFAAAAA|952|Fourteenth Cherry|Parkway|Suite C|Harmony|Jefferson County|ID|85804|United States|-7|apartment| +1285|AAAAAAAAFAFAAAAA|||||Union|Elmore County|AL|38721|United States||single family| +1286|AAAAAAAAGAFAAAAA|638|||Suite G|Newtown|Assumption Parish||||-6|single family| +1287|AAAAAAAAHAFAAAAA|728|Lincoln |Pkwy|Suite E|Fairfield|Fulton County|OH|46192|United States|-5|condo| +1288|AAAAAAAAIAFAAAAA|917|Oak Lee|Cir.|Suite 90|Brownsville|Reeves County|TX|79310|United States|-6|single family| +1289|AAAAAAAAJAFAAAAA|593|Church |Parkway|Suite 460|Scottsville|Lancaster County|VA|24190|United States|-5|single family| +1290|AAAAAAAAKAFAAAAA|848|Fifth Valley|Ln|Suite 300|Providence|Goodhue County|MN|56614|United States|-6|condo| +1291|AAAAAAAALAFAAAAA|100|Sycamore |Drive|Suite 220|Glenwood|Alcona County|MI|43511|United States|-5|single family| +1292|AAAAAAAAMAFAAAAA|136|Elm Cedar|Court|Suite B|Franklin|Madera County|CA|99101|United States|-8|condo| +1293|AAAAAAAANAFAAAAA|152|Walnut Cedar|Circle|Suite 100|Montezuma|Lawrence County|PA|12150|United States|-5|single family| +1294|AAAAAAAAOAFAAAAA||Ridge Valley|Dr.|Suite 100||||09477|United States|-5|| +1295|AAAAAAAAPAFAAAAA|287|1st |Blvd|Suite G|Mountain View|Caldwell County|MO|64466|United States|-6|single family| +1296|AAAAAAAAABFAAAAA|888|Second |Ave|Suite 310|Crossroads|White County|AR|70534|United States|-6|apartment| +1297|AAAAAAAABBFAAAAA|861|12th Main|RD|Suite G|Waterloo|Wood County|WV|21675|United States|-5|apartment| +1298|AAAAAAAACBFAAAAA|208|15th |Road|Suite 150|Lakewood|Sampson County|NC|28877|United States|-5|apartment| +1299|AAAAAAAADBFAAAAA|140|Meadow 4th|Ave|Suite 260|Five Forks|Adams County|MS|52293|United States|-6|condo| +1300|AAAAAAAAEBFAAAAA|967|15th Center|ST|Suite 180|Jamestown|Calhoun County|MI|46867|United States|-5|single family| +1301|AAAAAAAAFBFAAAAA|46|Wilson Johnson|RD|Suite T|Oakland|Bristol County|MA|09843|United States|-5|condo| +1302|AAAAAAAAGBFAAAAA|695|Park |Circle|Suite P|Greenfield|Russell County|KY|45038|United States|-5|apartment| +1303|AAAAAAAAHBFAAAAA|759|Spring |Dr.|Suite Y|Concord|Rooks County|KS|64107|United States|-6|condo| +1304|AAAAAAAAIBFAAAAA|629|5th |Boulevard|Suite 450|Five Points|Chippewa County|MI|46098|United States|-5|single family| +1305|AAAAAAAAJBFAAAAA|728|Spring |Boulevard|Suite 460|Five Forks|Suffolk County|MA|02893|United States|-5|apartment| +1306|AAAAAAAAKBFAAAAA|116|Second 6th|Avenue|Suite 230|Brownsville|Cobb County|GA|39310|United States|-5|apartment| +1307|AAAAAAAALBFAAAAA|183|Cedar |Ct.|Suite U|Brownsville|Monroe County|IA|59310|United States|-6|apartment| +1308|AAAAAAAAMBFAAAAA|930|Washington |Avenue|Suite 450|Stafford|Claiborne Parish|LA|74980|United States|-6|apartment| +1309|AAAAAAAANBFAAAAA|641|5th ||Suite O|Leland|Ritchie County||29452|United States|-5|condo| +1310|AAAAAAAAOBFAAAAA|198|6th |Way|Suite 480|Lakeside|Atkinson County|GA|39532|United States|-5|single family| +1311|AAAAAAAAPBFAAAAA|172|Johnson |Wy|Suite P|Rosewood|Jackson County|MN|52205|United States|-6|apartment| +1312|AAAAAAAAACFAAAAA|195|View |Street|Suite 410|Freeman|Jasper County|MO|62297|United States|-6|condo| +1313|AAAAAAAABCFAAAAA|218|Church |Ln|Suite N|Woodland|Kalkaska County|MI|44854|United States|-5|condo| +1314|AAAAAAAACCFAAAAA|244|South |Street|Suite 200|Lakeview|Kingfisher County|OK|78579|United States|-6|apartment| +1315|AAAAAAAADCFAAAAA|344|7th Church|Boulevard|Suite I|Sunnyside|Ellis County|KS|61952|United States|-6|condo| +1316|AAAAAAAAECFAAAAA|808|1st |Drive|Suite W|Greenfield|DeKalb County|IL|65038|United States|-6|apartment| +1317|AAAAAAAAFCFAAAAA|489|2nd |ST|Suite 200|Providence|Jefferson County|MT|66614|United States|-7|condo| +1318|AAAAAAAAGCFAAAAA|338|Chestnut |ST|Suite 50|Flat Rock|Sabine Parish|LA|74387|United States|-6|single family| +1319|AAAAAAAAHCFAAAAA|487|Ridge 3rd|Boulevard|Suite 100|Pleasant Hill|Sumter County|SC|23604|United States|-5|apartment| +1320|AAAAAAAAICFAAAAA|787|Poplar Willow|Boulevard|Suite N|Red Hill|Kendall County|IL|64338|United States|-6|apartment| +1321|AAAAAAAAJCFAAAAA|700|Pine Oak|Court|Suite O|Oak Hill|Washington County|PA|17838|United States|-5|apartment| +1322|AAAAAAAAKCFAAAAA|111|8th 5th|Blvd|Suite 100|Springdale|Big Horn County|WY|88883|United States|-7|condo| +1323|AAAAAAAALCFAAAAA|826|7th |Blvd|Suite Y|Forest Hills|Douglas County|MO|69237|United States|-6|condo| +1324|AAAAAAAAMCFAAAAA|215|Sunset |Pkwy|Suite I|Unionville|Riley County|KS|61711|United States|-6|apartment| +1325|AAAAAAAANCFAAAAA|24|Main |Circle|Suite E|Yorktown|Taos County|NM|80732|United States|-7|single family| +1326|AAAAAAAAOCFAAAAA|695|Johnson |Road|Suite 330|Glendale|Sussex County|NJ|04551|United States|-5|apartment| +1327|AAAAAAAAPCFAAAAA|35|Sunset Maple|Ct.|Suite 40|Highland Park|Lincoln County|SD|56534|United States|-7|apartment| +1328|AAAAAAAAADFAAAAA|530|River |Boulevard|Suite Y|Springfield|Burke County|GA|39303|United States|-5|single family| +1329|AAAAAAAABDFAAAAA|854|9th |Lane|Suite W|Shannon|Clark County|WA|94120|United States|-8|apartment| +1330|AAAAAAAACDFAAAAA|||||Arlington|Potter County|PA|||-5|apartment| +1331|AAAAAAAADDFAAAAA|891|Twelfth |Dr.|Suite Q|Wilson|Johnston County|OK|76971|United States|-6|apartment| +1332|AAAAAAAAEDFAAAAA|254|13th |Ave|Suite 40|Lebanon|Cottle County|TX|72898|United States|-6|apartment| +1333|AAAAAAAAFDFAAAAA|639|Washington |ST|Suite H|Union Hill|Davis County|UT|87746|United States|-7|condo| +1334|AAAAAAAAGDFAAAAA|367|Adams |Boulevard|Suite 480|Newtown|King County|TX|71749|United States|-6|single family| +1335|AAAAAAAAHDFAAAAA|520|Cedar Jackson|Cir.|Suite 380|Greenwood|Franklin County|MA|09428|United States|-5|apartment| +1336|AAAAAAAAIDFAAAAA|378|Lincoln |RD|Suite W|Hamilton|Atchison County|KS|62808|United States|-6|apartment| +1337|AAAAAAAAJDFAAAAA|704|6th |Ct.|Suite X|Proctor|Culberson County|TX|78140|United States|-6|single family| +1338|AAAAAAAAKDFAAAAA|697|Railroad 7th|Court|Suite D|Jamestown|Burke County|GA|36867|United States|-5|condo| +1339|AAAAAAAALDFAAAAA|101|3rd |Parkway|Suite P|Mount Vernon|Johnson County|NE|68482|United States|-7|condo| +1340|AAAAAAAAMDFAAAAA|132|Highland 3rd|Court|Suite 170|Mount Olive|Cleburne County|AR|78059|United States|-6|single family| +1341|AAAAAAAANDFAAAAA|543|Lake Elm|Parkway|Suite 330|Oak Hill|Santa Barbara County|CA|97838|United States|-8|condo| +1342|AAAAAAAAODFAAAAA|133|College |Boulevard|Suite R|Crossroads|Nash County|NC|20534|United States|-5|single family| +1343|AAAAAAAAPDFAAAAA|50|Jackson |Blvd|Suite G|Riverview|Franklin County|IA|59003|United States|-6|apartment| +1344|AAAAAAAAAEFAAAAA|258|Hickory ||Suite 300||Buffalo County||59003|||condo| +1345|AAAAAAAABEFAAAAA|530|2nd Walnut|Court|Suite 80|Mount Vernon|Greene County|VA|28482|United States|-5|single family| +1346|AAAAAAAACEFAAAAA|843|Mill 6th|Avenue|Suite D|Spring Valley|Lincoln County|OR|96060|United States|-8|single family| +1347|AAAAAAAADEFAAAAA|116|Oak |ST|Suite Y|Leland|Hyde County|NC|29452|United States|-5|apartment| +1348|AAAAAAAAEEFAAAAA|202|Broadway |Blvd|Suite X|Wilson|Cheyenne County|NE|66971|United States|-6|apartment| +1349|AAAAAAAAFEFAAAAA|261|||||Franklin County|IA|54338||-6|| +1350|AAAAAAAAGEFAAAAA|860|4th |Boulevard|Suite 240|Clinton|Walker County|GA|38222|United States|-5|single family| +1351|AAAAAAAAHEFAAAAA|740|Madison |Drive|Suite 280|Spring Hill|Vinton County|OH|46787|United States|-5|apartment| +1352|AAAAAAAAIEFAAAAA|724|South Railroad|Pkwy|Suite Q|Flatwoods|Pend Oreille County|WA|94212|United States|-8|condo| +1353|AAAAAAAAJEFAAAAA|748|8th |Street|Suite 300|Springdale|Morgan County|WV|28883|United States|-5|single family| +1354|AAAAAAAAKEFAAAAA|276|Green 9th|Wy|Suite 350|Sulphur Springs|Bennett County|SD|58354|United States|-6|single family| +1355|AAAAAAAALEFAAAAA|461|Main Highland|Cir.|Suite 400|Mount Vernon|Avery County|NC|28482|United States|-5|condo| +1356|AAAAAAAAMEFAAAAA|578|6th |Ave|Suite 70|Fairfield|Pacific County|WA|96192|United States|-8|single family| +1357|AAAAAAAANEFAAAAA|365|North Ridge|Boulevard|Suite W|Oak Hill|Lincoln County|KY|47838|United States|-5|single family| +1358|AAAAAAAAOEFAAAAA|289|3rd Ridge|Ave|Suite 390|Lakeside|King William County|VA|29532|United States|-5|single family| +1359|AAAAAAAAPEFAAAAA|746|10th |Ave|Suite 90|Valley View|Jefferson County|AR|75124|United States|-6|single family| +1360|AAAAAAAAAFFAAAAA|947|Woodland |Circle|Suite L|Jamestown|Queen Anne County|MD|26867|United States|-5|apartment| +1361|AAAAAAAABFFAAAAA|615|Willow 1st|Street|Suite F|Wilson|Reynolds County|MO|66971|United States|-6|apartment| +1362|AAAAAAAACFFAAAAA|189|Miller |Pkwy|Suite 350|New Hope|Jones County|MS|59431|United States|-6|apartment| +1363|AAAAAAAADFFAAAAA|||Cir.||Salem|Marquette County||58048|United States|-6|single family| +1364|AAAAAAAAEFFAAAAA|708|Elm South||Suite 0|Franklin|Beaver County||79101|||| +1365|AAAAAAAAFFFAAAAA||Park ||||Lassen County|CA||United States||| +1366|AAAAAAAAGFFAAAAA|963|Lakeview |Court|Suite 90|Salem|Marshall County|KY|48048|United States|-5|condo| +1367|AAAAAAAAHFFAAAAA|396|Maple |Ave|Suite 310|Pomona|Vernon Parish|LA|74153|United States|-6|condo| +1368|AAAAAAAAIFFAAAAA|751|1st |Avenue|Suite T|Union Hill|Perry County|AR|77746|United States|-6|condo| +1369|AAAAAAAAJFFAAAAA|679|Ridge Park|Blvd|Suite O|Bunker Hill|Sampson County|NC|20150|United States|-5|apartment| +1370|AAAAAAAAKFFAAAAA|678|13th Laurel|Ln|Suite 440|Belmont|Tuscaloosa County|AL|30191|United States|-6|apartment| +1371|AAAAAAAALFFAAAAA|379|Madison |Pkwy|Suite 490|Summit|Hamilton County|KS|60499|United States|-6|condo| +1372|AAAAAAAAMFFAAAAA|938|2nd |Pkwy|Suite 50|Elkton|Rusk County|TX|73481|United States|-6|condo| +1373|AAAAAAAANFFAAAAA|487|9th West|Dr.|Suite 490|Belmont|Carter County|KY|40191|United States|-6|condo| +1374|AAAAAAAAOFFAAAAA|509|7th 3rd|Cir.|Suite Q|Gum Springs|Blanco County|TX|72106|United States|-6|apartment| +1375|AAAAAAAAPFFAAAAA|667|Forest |Street|Suite 340|Unionville|Jersey County|IL|61711|United States|-6|condo| +1376|AAAAAAAAAGFAAAAA|579|College |Way|Suite F|Welcome|Tyrrell County|NC|26386|United States|-5|single family| +1377|AAAAAAAABGFAAAAA|802|Poplar |Cir.|Suite L|Oak Grove|Carson County|TX|78370|United States|-6|apartment| +1378|AAAAAAAACGFAAAAA|721|Hillcrest Sycamore|Pkwy|Suite T|Plainview|Calhoun County|WV|23683|United States|-5|condo| +1379|AAAAAAAADGFAAAAA|955|East |Boulevard|Suite S|Saint George|Franklin County|MS|55281|United States|-6|condo| +1380|AAAAAAAAEGFAAAAA|454|West Chestnut|Avenue|Suite 30|Jamestown|Grenada County|MS|56867|United States|-6|single family| +1381|AAAAAAAAFGFAAAAA|499|Fifth |Ln|Suite 90|Woodlawn|Fayette County|AL|34098|United States|-6|apartment| +1382|AAAAAAAAGGFAAAAA|781|7th 1st|Avenue|Suite V|Perkins|Washington County|WI|51852|United States|-6|single family| +1383|AAAAAAAAHGFAAAAA|429|1st Lakeview|Cir.|Suite 390|Springdale|Alcona County|MI|48883|United States|-5|apartment| +1384|AAAAAAAAIGFAAAAA|670|South Second|Wy|Suite 460|Harmony|Durham County|NC|25804|United States|-5|condo| +1385|AAAAAAAAJGFAAAAA|878|Willow |Street|Suite S|Sulphur Springs|Golden Valley County|MT|68354|United States|-7|condo| +1386|AAAAAAAAKGFAAAAA|803|Washington South|Drive|Suite 270|Ashland|Cabarrus County|NC|24244|United States|-5|apartment| +1387|AAAAAAAALGFAAAAA|749|7th Eigth|Court|Suite 410|Woodcrest|New Kent County|VA|24919|United States|-5|single family| +1388|AAAAAAAAMGFAAAAA|231|6th |Blvd|Suite Q|Spring Hill|Jackson County|SD|56787|United States|-7|apartment| +1389|AAAAAAAANGFAAAAA|637|Jefferson |Ave|Suite A|Pleasant Grove|El Dorado County|CA|94136|United States|-8|single family| +1390|AAAAAAAAOGFAAAAA|398|Hill Wilson|Ave|Suite 10|Enterprise|Robertson County|TN|31757|United States|-6|single family| +1391|AAAAAAAAPGFAAAAA|167|Cedar |Lane|Suite 260|Pine Valley|Washington County|FL|38209|United States|-5|condo| +1392|AAAAAAAAAHFAAAAA|218|5th Second|Street|Suite 490|Arlington|Marshall County|IA|56557|United States|-6|single family| +1393|AAAAAAAABHFAAAAA||Main |Street||Farmington|Cass County|NE|69145|United States||condo| +1394|AAAAAAAACHFAAAAA||5th Hill|Pkwy||Unionville|Jones County|||United States||apartment| +1395|AAAAAAAADHFAAAAA|366|Second |ST|Suite 90|Bridgeport|Clay County|IA|55817|United States|-6|condo| +1396|AAAAAAAAEHFAAAAA|739|Spring Ninth|Circle|Suite 130|Friendship|Benton County|IA|||-6|condo| +1397|AAAAAAAAFHFAAAAA|185||Street|Suite D|Franklin|||39101|United States||condo| +1398|AAAAAAAAGHFAAAAA|147|Fifth Forest|Ave|Suite 140|Highland|Marin County|CA|99454|United States|-8|apartment| +1399|AAAAAAAAHHFAAAAA|384|15th 7th|Road|Suite 40|Hopewell|Dyer County|TN|30587|United States|-5|condo| +1400|AAAAAAAAIHFAAAAA|587|Cherry Davis|Wy|Suite K|Unionville|Pondera County|MT|61711|United States|-7|apartment| +1401|AAAAAAAAJHFAAAAA|509|Maple 5th|Road|Suite 350|Deerfield|Manatee County|FL|39840|United States|-5|apartment| +1402|AAAAAAAAKHFAAAAA|919|13th |Street|Suite 440|Shiloh|Morgan County|OH|49275|United States|-5|apartment| +1403|AAAAAAAALHFAAAAA|80|13th |Wy|Suite 80|Midway|Clay County|FL|31904|United States|-5|apartment| +1404|AAAAAAAAMHFAAAAA|316|Broadway |Parkway|Suite 0|Florence|McDuffie County|GA|33394|United States|-5|apartment| +1405|AAAAAAAANHFAAAAA|664|3rd |Boulevard|Suite O|Brookwood|Natrona County|WY|80965|United States|-7|apartment| +1406|AAAAAAAAOHFAAAAA|23|Fourth 5th|Parkway|Suite N|Five Points|Wayne County|NY|16098|United States|-5|condo| +1407|AAAAAAAAPHFAAAAA|698|Highland |Way|Suite A|Carthage|Sheridan County|WY|81529|United States|-7|single family| +1408|AAAAAAAAAIFAAAAA|560|Third Hill|Pkwy|Suite L|Fox|Peach County|GA|30631|United States|-5|condo| +1409|AAAAAAAABIFAAAAA|339|7th |Cir.|Suite S|Springdale|Jones County|IA|58883|United States|-6|apartment| +1410|AAAAAAAACIFAAAAA|280|Forest |Parkway|Suite 340|Kingston|Rutherford County|TN|34975|United States|-6|apartment| +1411|AAAAAAAADIFAAAAA|204|Elm Johnson|Drive|Suite I|Pleasant Hill|Furnas County|NE|63604|United States|-6|apartment| +1412|AAAAAAAAEIFAAAAA|310|Second North|Parkway|Suite N|Enterprise|Kingsbury County|SD|51757|United States|-7|condo| +1413|AAAAAAAAFIFAAAAA|266|Oak 10th|Parkway|Suite W|Bethel|Baltimore County|MD|25281|United States|-5|condo| +1414|AAAAAAAAGIFAAAAA|48|Hillcrest Park|RD|Suite 210|Highland|Harper County|KS|69454|United States|-6|apartment| +1415|AAAAAAAAHIFAAAAA|59|Elm |Road|Suite E|Oak Grove|Taylor County|FL|38370|United States|-5|single family| +1416|AAAAAAAAIIFAAAAA|741|View Oak|Court|Suite J|Highland|Payne County|OK|79454|United States|-6|condo| +1417|AAAAAAAAJIFAAAAA|926|Elm |ST|Suite O|Mount Vernon|Canyon County|ID|88482|United States|-7|single family| +1418|AAAAAAAAKIFAAAAA|706|Lincoln Maple|Lane|Suite 290|Riverview|Chesterfield County|SC|29003|United States|-5|single family| +1419|AAAAAAAALIFAAAAA|25|5th |Cir.|Suite X|Ashland|Crawford County|IN|44244|United States|-5|condo| +1420|AAAAAAAAMIFAAAAA|899|15th Walnut|Lane|Suite 10|Springdale|Barry County|MI|48883|United States|-5|condo| +1421|AAAAAAAANIFAAAAA|530|14th Adams|Ct.|Suite V|Lee|Washakie County|WY|80408|United States|-7|condo| +1422|AAAAAAAAOIFAAAAA|826|Fifth Hickory|Boulevard|Suite N|Harmony|Brown County|TX|75804|United States|-6|condo| +1423|AAAAAAAAPIFAAAAA|622|Meadow |Circle|Suite M|New Hope|Burke County|NC|29431|United States|-5|single family| +1424|AAAAAAAAAJFAAAAA|594|Railroad |Dr.|Suite R|Union|Burnet County|TX|78721|United States|-6|single family| +1425|AAAAAAAABJFAAAAA|66|Willow View|Wy|Suite 320|Oakwood|Yalobusha County|MS|50169|United States|-6|single family| +1426|AAAAAAAACJFAAAAA|579|Spruce Oak|Cir.|Suite 200|Plainview|Warren County|NC|23683|United States|-5|single family| +1427|AAAAAAAADJFAAAAA|577|View |Lane|Suite U|Amherst|Montgomery County|MD|28119|United States|-5|single family| +1428|AAAAAAAAEJFAAAAA|281|5th |Court|Suite A|Marion|Callahan County|TX|70399|United States|-6|condo| +1429|AAAAAAAAFJFAAAAA|320|Poplar Davis|Dr.|Suite 250|Spring Valley|Clay County|KY|46060|United States|-6|condo| +1430|AAAAAAAAGJFAAAAA|963|Jefferson |Ave|Suite W|Florence|Cass County|IL|63394|United States|-6|apartment| +1431|AAAAAAAAHJFAAAAA|456|4th |Wy|Suite 300|Brownsville|Sandusky County|OH|49310|United States|-5|condo| +1432|AAAAAAAAIJFAAAAA|312|Pine Woodland|Ave|Suite I|Edgewood|Prairie County|MT|60069|United States|-7|single family| +1433|AAAAAAAAJJFAAAAA|750|Park 2nd|Pkwy|Suite Y|Lakeview|Benton County|MS|58579|United States|-6|condo| +1434|AAAAAAAAKJFAAAAA|38|Locust Davis||Suite 0|Bunker Hill|||||-5|| +1435|AAAAAAAALJFAAAAA|453|Birch |Blvd|Suite J|Midway|Crawford County|IL|61904|United States|-6|condo| +1436|AAAAAAAAMJFAAAAA|785|Poplar Lake|Street|Suite X|Antioch|Union County|SC|28605|United States|-5|apartment| +1437|AAAAAAAANJFAAAAA|728|3rd Central|Way|Suite G|Woodland|Stephens County|GA|34854|United States|-5|apartment| +1438|AAAAAAAAOJFAAAAA|502|Hickory Forest|Way|Suite R|Oak Grove|Tyler County|TX|78370|United States|-6|single family| +1439|AAAAAAAAPJFAAAAA|651|Central |Ln|Suite C|Fairfield|Orange County|CA|96192|United States|-8|single family| +1440|AAAAAAAAAKFAAAAA|172|Walnut Tenth|Way|Suite 140|Wildwood|Albany County|WY|86871|United States|-7|apartment| +1441|AAAAAAAABKFAAAAA|377|Pine Broadway|Blvd|Suite 300|Clifton|Clay County|GA|38014|United States|-5|condo| +1442|AAAAAAAACKFAAAAA|83|First 14th|Lane|Suite 120|Antioch|Gem County|ID|88605|United States|-7|apartment| +1443|AAAAAAAADKFAAAAA|168|Lakeview 2nd|Court|Suite 280|Indian Village|White County|IN|41075|United States|-5|single family| +1444|AAAAAAAAEKFAAAAA|460|4th |Road|Suite L|Cedar Grove|Fall River County|SD|50411|United States|-6|condo| +1445|AAAAAAAAFKFAAAAA|580|Washington Sixth|Court|Suite 410|Bunker Hill|Garden County|NE|60150|United States|-6|condo| +1446|AAAAAAAAGKFAAAAA|477|Hickory Laurel|Circle|Suite B|Concord|Chase County|NE|64107|United States|-6|condo| +1447|AAAAAAAAHKFAAAAA|950|Park |ST|Suite 230|Crossroads|Monroe County|PA|10534|United States|-5|apartment| +1448|AAAAAAAAIKFAAAAA|432|1st Railroad|Way|Suite 120|Weldon|Harris County|TX|76277|United States|-6|condo| +1449|AAAAAAAAJKFAAAAA|895|Highland Mill|Dr.|Suite 20|Woodlawn|Ellis County|OK|74098|United States|-6|condo| +1450|AAAAAAAAKKFAAAAA|89|Dogwood |Dr.|Suite F|Deerfield|Carbon County|UT|89840|United States|-7|apartment| +1451|AAAAAAAALKFAAAAA|805||Boulevard|Suite 270|Philadelphia|||55591|United States||| +1452|AAAAAAAAMKFAAAAA|176|South Wilson|Road|Suite G|Kingston|Humboldt County|NV|84975|United States|-8|condo| +1453|AAAAAAAANKFAAAAA|769|Park |Court|Suite T|Shore Acres|Orange County|TX|72724|United States|-6|condo| +1454|AAAAAAAAOKFAAAAA|114|Spring Oak|Dr.|Suite 250|Tanglewood|Walsh County|ND|58994|United States|-6|single family| +1455|AAAAAAAAPKFAAAAA|562|7th |Cir.|Suite 340|Shady Grove|Brule County|SD|52812|United States|-6|single family| +1456|AAAAAAAAALFAAAAA|955|Johnson Lee|Ln|Suite 80|Clifton|Cleveland County|OK|78014|United States|-6|apartment| +1457|AAAAAAAABLFAAAAA||Park Dogwood|||Sulphur Springs||TX|78354|United States||single family| +1458|AAAAAAAACLFAAAAA|285|Church |Wy|Suite I|Lebanon|Luna County|NM|82898|United States|-7|single family| +1459|AAAAAAAADLFAAAAA|565|Eigth Lee|Ave|Suite 170|White Oak|Newaygo County|MI|46668|United States|-5|apartment| +1460|AAAAAAAAELFAAAAA|27|College 3rd|Boulevard|Suite 150|Tyrone|Roanoke city|VA|21201|United States|-5|single family| +1461|AAAAAAAAFLFAAAAA|15|Valley |Drive|Suite P|Stringtown|Fentress County|TN|30162|United States|-5|apartment| +1462|AAAAAAAAGLFAAAAA|110|Sycamore Meadow|Blvd|Suite M|Forest Hills|Pepin County|WI|59237|United States|-6|single family| +1463|AAAAAAAAHLFAAAAA|82|Third First|Ln|Suite H|Bunker Hill|Otter Tail County|MN|50150|United States|-6|apartment| +1464|AAAAAAAAILFAAAAA|869|Walnut Third|Dr.|Suite C|Jamestown|Curry County|NM|86867|United States|-7|single family| +1465|AAAAAAAAJLFAAAAA|383|5th 7th|Way|Suite E|Oak Grove|Eureka County|NV|88370|United States|-8|condo| +1466|AAAAAAAAKLFAAAAA|60|10th |Avenue|Suite Q|Red Hill|Boone County|MO|64338|United States|-6|condo| +1467|AAAAAAAALLFAAAAA|541||Dr.|Suite 300|West Liberty|||14752|United States|-5|| +1468|AAAAAAAAMLFAAAAA|703|1st Railroad|Way|Suite J|Buena Vista|Montgomery County|OH|45752|United States|-5|single family| +1469|AAAAAAAANLFAAAAA|869|Hill |Circle|Suite H|Lakeview|Lubbock County|TX|78579|United States|-6|single family| +1470|AAAAAAAAOLFAAAAA|356|Franklin |Street|Suite 450|Plainview|Clay County|IN|43683|United States|-5|single family| +1471|AAAAAAAAPLFAAAAA|199|6th Hill|Avenue|Suite D|Hillcrest|Monroe County|IL|63003|United States|-6|apartment| +1472|AAAAAAAAAMFAAAAA|80|Locust Walnut|Circle|Suite W|Valley View|Major County|OK|75124|United States|-6|condo| +1473|AAAAAAAABMFAAAAA|740|Wilson |Street|Suite B|Union|Sevier County|AR|78721|United States|-6|apartment| +1474|AAAAAAAACMFAAAAA|352|Hillcrest ||Suite V|Oak Grove|||78370|United States|-6|single family| +1475|AAAAAAAADMFAAAAA|699|Mill |Lane|Suite H|Floyd|Worcester County|MA|03835|United States|-5|apartment| +1476|AAAAAAAAEMFAAAAA|193|Park Davis|Ln|Suite W|Springdale|Charlotte County|FL|38883|United States|-5|condo| +1477|AAAAAAAAFMFAAAAA|241|Johnson |Drive|Suite 210|Fairview|Clermont County|OH|45709|United States|-5|condo| +1478|AAAAAAAAGMFAAAAA|297|Franklin 15th|ST|Suite 450|Fayetteville|Johnson County|GA|31732|United States|-5|apartment| +1479|AAAAAAAAHMFAAAAA|975|Center 4th|Ln|Suite L|Edgewood|King William County|VA|20069|United States|-5|condo| +1480|AAAAAAAAIMFAAAAA|734|Lincoln |Pkwy||Glenville|Newaygo County||||-5|single family| +1481|AAAAAAAAJMFAAAAA|705|Birch Miller|RD|Suite I|Mount Vernon|Penobscot County|ME|09082|United States|-5|condo| +1482|AAAAAAAAKMFAAAAA|536|Highland |Avenue|Suite 20|Springdale|Irwin County|GA|38883|United States|-5|single family| +1483|AAAAAAAALMFAAAAA|630|Sixth River|Avenue|Suite 330|Lakeside|Calhoun County|IA|59532|United States|-6|single family| +1484|AAAAAAAAMMFAAAAA|841|9th South|Lane|Suite L|Hillcrest|Victoria County|TX|73003|United States|-6|single family| +1485|AAAAAAAANMFAAAAA|525|Main Main|Wy|Suite 100|Mountain View|Vermillion County|IN|44466|United States|-5|single family| +1486|AAAAAAAAOMFAAAAA|729|Broadway Park|Wy|Suite T|Lebanon|Marion County|AL|32898|United States|-6|condo| +1487|AAAAAAAAPMFAAAAA|437|Central Wilson|Parkway|Suite Y|Concord|Comanche County|OK|74107|United States|-6|apartment| +1488|AAAAAAAAANFAAAAA|124|West Seventh|Road|Suite C|Georgetown|Greer County|OK|77057|United States|-6|single family| +1489|AAAAAAAABNFAAAAA|478|Johnson Main|Lane|Suite H|Redland|Johnston County|NC|26343|United States|-5|apartment| +1490|AAAAAAAACNFAAAAA|773|Fourth |ST|Suite X|Hamilton|Mayes County|OK|72808|United States|-6|apartment| +1491|AAAAAAAADNFAAAAA|459|Highland |Dr.|Suite S|Arlington|Jackson County|TX|76557|United States|-6|condo| +1492|AAAAAAAAENFAAAAA|123|Madison |Avenue|Suite 390|Redland|Stanton County|KS|66343|United States|-6|single family| +1493|AAAAAAAAFNFAAAAA|412|Franklin |Ct.|Suite 70|Buena Vista|Poinsett County|AR|75752|United States|-6|single family| +1494|AAAAAAAAGNFAAAAA|404|Washington Third|Blvd|Suite B|Plainview|Walton County|GA|33683|United States|-5|single family| +1495|AAAAAAAAHNFAAAAA|322|Oak West|Drive|Suite 200|Forest Hills|Jefferson County|WI|59237|United States|-6|single family| +1496|AAAAAAAAINFAAAAA|809|Cedar 5th|Ave|Suite 90|Midway|Nobles County|MN|51904|United States|-6|single family| +1497|AAAAAAAAJNFAAAAA|472|8th West|Avenue|Suite T|Ashland|Grady County|GA|34244|United States|-5|single family| +1498|AAAAAAAAKNFAAAAA|198|Main |Circle|Suite 180|Oak Ridge|Surry County|VA|28371|United States|-5|condo| +1499|AAAAAAAALNFAAAAA|976|Jefferson |RD|Suite C|Clinton|Washington County|ME|08822|United States|-5|apartment| +1500|AAAAAAAAMNFAAAAA|476|11th Center|Ave|Suite 430|Kingston|Garland County|AR|74975|United States|-6|condo| +1501|AAAAAAAANNFAAAAA|338|Washington Main|ST|Suite K|Stringtown|Muscogee County|GA|30162|United States|-5|apartment| +1502|AAAAAAAAONFAAAAA|608|Park |Cir.|Suite W|Forest Hills|Sullivan County|TN|39237|United States|-6|condo| +1503|AAAAAAAAPNFAAAAA|883|Railroad Highland|Road|Suite 180|Clifton|Broome County|NY|18014|United States|-5|condo| +1504|AAAAAAAAAOFAAAAA|68|Spring Jackson|Circle|Suite T|Greenville|Howard County|NE|61387|United States|-7|condo| +1505|AAAAAAAABOFAAAAA|998|Center Lee|Avenue|Suite 320|Sulphur Springs|Jackson County|MI|48354|United States|-5|single family| +1506|AAAAAAAACOFAAAAA|904|Chestnut Church|Wy|Suite 320|Mount Zion|Cass County|IN|48054|United States|-5|condo| +1507|AAAAAAAADOFAAAAA|177|8th Willow|Dr.|Suite N|Greenfield|Clarendon County|SC|25038|United States|-5|condo| +1508|AAAAAAAAEOFAAAAA|801|Lakeview Mill|Parkway|Suite 60|Highland|Jenkins County|GA|39454|United States|-5|single family| +1509|AAAAAAAAFOFAAAAA|995|Wilson |Ln|Suite D|Farmington|Richardson County|NE|69145|United States|-7|apartment| +1510|AAAAAAAAGOFAAAAA|336|Pine 1st|Ct.|Suite 390|Bridgeport|Bremer County|IA|55817|United States|-6|single family| +1511|AAAAAAAAHOFAAAAA|919|Birch |Avenue|Suite 360|Spring Valley|Fond du Lac County|WI|56060|United States|-6|apartment| +1512|AAAAAAAAIOFAAAAA|567|Meadow Third|Court|Suite T|Bethel|Ottawa County|KS|65281|United States|-6|condo| +1513|AAAAAAAAJOFAAAAA|276|Second |Parkway|Suite N|Woodland|Shelby County|IA|54854|United States|-6|apartment| +1514|AAAAAAAAKOFAAAAA|632|Church Highland|Blvd|Suite I|Union Hill|Adams County|OH|47746|United States|-5|single family| +1515|AAAAAAAALOFAAAAA|510|River South|Pkwy|Suite 110|Antioch|Pamlico County|NC|28605|United States|-5|apartment| +1516|AAAAAAAAMOFAAAAA|716|Pine |Way|Suite 240|Lakeside|Spokane County|WA|99532|United States|-8|apartment| +1517|AAAAAAAANOFAAAAA|546|Church |Lane|Suite 30|Gravel Hill|Fleming County|KY|41944|United States|-6|apartment| +1518|AAAAAAAAOOFAAAAA|447|Adams Washington|Way|Suite 160|Oakdale|Liberty County|TX|79584|United States|-6|apartment| +1519|AAAAAAAAPOFAAAAA|576|Railroad |Dr.|Suite 250|Oakwood|Union County|IN|40169|United States|-5|apartment| +1520|AAAAAAAAAPFAAAAA|723|Central Birch|RD|Suite 180|Oakwood|Caldwell County|KY|40169|United States|-6|single family| +1521|AAAAAAAABPFAAAAA||Forest Ridge|||Oak Hill||TX|77838|United States|-6|apartment| +1522|AAAAAAAACPFAAAAA|348|Broadway Spring|Court|Suite 370|Five Forks|Gregg County|TX|72293|United States|-6|condo| +1523|AAAAAAAADPFAAAAA|764|1st |Blvd|Suite 310|Forest Hills|Knox County|NE|69237|United States|-7|single family| +1524|AAAAAAAAEPFAAAAA|564|Main |Way|Suite 120|Bethel|Hunt County|TX|75281|United States|-6|apartment| +1525|AAAAAAAAFPFAAAAA|832|Willow Ash|ST|Suite B|Oak Ridge|Lake County|SD|58371|United States|-7|single family| +1526|AAAAAAAAGPFAAAAA|586|Lake |Blvd|Suite 200|Oakdale|Marion County|SC|29584|United States|-5|apartment| +1527|AAAAAAAAHPFAAAAA|200|11th |Circle|Suite 300|Clinton|Pike County|GA|38222|United States|-5|condo| +1528|AAAAAAAAIPFAAAAA|509|9th Williams|Pkwy|Suite M|Providence|Roane County|TN|36614|United States|-6|single family| +1529|AAAAAAAAJPFAAAAA|355|Dogwood |Boulevard|Suite C|New Hope|Pulaski County|GA|39431|United States|-5|single family| +1530|AAAAAAAAKPFAAAAA|54|Park Washington|Boulevard|Suite 140|Newport|Aitkin County|MN|51521|United States|-6|single family| +1531|AAAAAAAALPFAAAAA|28|Elm Cedar|Lane|Suite K|Summit|Ozaukee County|WI|50499|United States|-6|single family| +1532|AAAAAAAAMPFAAAAA|455|Sycamore Broadway|Blvd|Suite 240|Bethel|York County|ME|05881|United States|-5|single family| +1533|AAAAAAAANPFAAAAA|49|4th |Way|Suite E|Midway|Grant County|NM|81904|United States|-7|single family| +1534|AAAAAAAAOPFAAAAA|79|Fourth |Lane|Suite W|Oak Ridge|Washington County|MN|58371|United States|-6|single family| +1535|AAAAAAAAPPFAAAAA|484|First Forest|Parkway|Suite 310|Pleasant Grove|Waller County|TX|74136|United States|-6|apartment| +1536|AAAAAAAAAAGAAAAA|53|Sixth Pine|ST|Suite E|Bethel|Cache County|UT|85281|United States|-7|apartment| +1537|AAAAAAAABAGAAAAA|583|Spruce Oak|Road|Suite 200|Cedar Grove|York County|PA|10411|United States|-5|apartment| +1538|AAAAAAAACAGAAAAA|291|Wilson Mill|ST|Suite P|Providence|Greenwood County|KS|66614|United States|-6|apartment| +1539|AAAAAAAADAGAAAAA|649|4th |Boulevard|Suite 410|Midway|Harrison County|IN|41904|United States|-5|apartment| +1540|AAAAAAAAEAGAAAAA|104|Pine |Avenue|Suite 220|Plainview|Fairfield County|CT|04283|United States|-5|single family| +1541|AAAAAAAAFAGAAAAA|581|Lincoln Church|Dr.|Suite W|Midway|Chouteau County|MT|61904|United States|-7|apartment| +1542|AAAAAAAAGAGAAAAA|118|Chestnut Church|Street|Suite 430|Fairfield|Choctaw County|OK|76192|United States|-6|single family| +1543|AAAAAAAAHAGAAAAA|148|Maple Park|Pkwy|Suite 420|Spring Valley|Jones County|IA|56060|United States|-6|single family| +1544|AAAAAAAAIAGAAAAA|242|4th |Parkway|Suite N|Maple Grove|Milam County|TX|78252|United States|-6|single family| +1545|AAAAAAAAJAGAAAAA|203|Walnut Pine|Lane|Suite 150|Deerfield|Clark County|KY|49840|United States|-6|single family| +1546|AAAAAAAAKAGAAAAA||7th 4th|Road|Suite H||Cortland County|||||apartment| +1547|AAAAAAAALAGAAAAA|487|Main Highland|Cir.|Suite 40|Jackson|Madison County|NC|29583|United States|-5|single family| +1548|AAAAAAAAMAGAAAAA|814|Johnson Jefferson|Lane|Suite 80|Oakland|Wright County|MO|69843|United States|-6|condo| +1549|AAAAAAAANAGAAAAA|18|Williams |Ave|Suite 350|Marion|Queens County|NY|10399|United States|-5|single family| +1550|AAAAAAAAOAGAAAAA|205|Spruce East|Lane|Suite 430|Enterprise|Rockingham County|VA|21757|United States|-5|condo| +1551|AAAAAAAAPAGAAAAA|965|Park |Court|Suite 90|Union Hill|Giles County|VA|27746|United States|-5|apartment| +1552|AAAAAAAAABGAAAAA|337|Hill Wilson|Dr.|Suite 410|Antioch|Larue County|KY|48605|United States|-5|single family| +1553|AAAAAAAABBGAAAAA|559|Fourth |Lane|Suite 160|Clifton|Lake County|MI|48014|United States|-5|single family| +1554|AAAAAAAACBGAAAAA|271|Lee |Street|Suite 110|Lakeside|Lincoln County|ME|09532|United States|-5|single family| +1555|AAAAAAAADBGAAAAA|660|Maple Sunset|ST|Suite G|Ryan|Swain County|NC|20525|United States|-5|condo| +1556|AAAAAAAAEBGAAAAA|54|Willow View|Ave|Suite 490|Springdale|Howard County|AR|78883|United States|-6|condo| +1557|AAAAAAAAFBGAAAAA|35|Cherry 5th|Road|Suite W|Five Forks|Larimer County|CO|82293|United States|-7|apartment| +1558|AAAAAAAAGBGAAAAA|96|9th |Parkway|Suite 30|Mount Zion|Edmonson County|KY|48054|United States|-6|single family| +1559|AAAAAAAAHBGAAAAA|971|8th Davis|Blvd|Suite 440|Pine Grove|Jackson County|TX|74593|United States|-6|apartment| +1560|AAAAAAAAIBGAAAAA|743|Franklin |ST|Suite 170|Greenville|Clayton County|IA|51387|United States|-6|condo| +1561|AAAAAAAAJBGAAAAA|939|Cedar |Circle|Suite W|Fairview|McCone County|MT|65709|United States|-7|condo| +1562|AAAAAAAAKBGAAAAA|199|Third |Boulevard|Suite 470|Greenville|Lake County|MN|51387|United States|-6|condo| +1563|AAAAAAAALBGAAAAA|695|Jefferson |Drive|Suite 80|Clinton|Sumter County|FL|38222|United States|-5|apartment| +1564|AAAAAAAAMBGAAAAA|730|East Walnut|Cir.|Suite I|Lincoln|Grand Forks County|ND|51289|United States|-6|condo| +1565|AAAAAAAANBGAAAAA|544|Broadway |Avenue|Suite 280|Sunnyside|Dunklin County|MO|61952|United States|-6|condo| +1566|AAAAAAAAOBGAAAAA|282|1st |Wy|Suite E|Arlington|Carroll County|TN|36557|United States|-5|condo| +1567|AAAAAAAAPBGAAAAA|571|Franklin |Dr.|Suite V|Shiloh|Gates County|NC|29275|United States|-5|single family| +1568|AAAAAAAAACGAAAAA||North |Circle|Suite 180|Jamestown|Missoula County|||||| +1569|AAAAAAAABCGAAAAA|749|10th |Blvd|Suite E|Jamestown|Jessamine County|KY|46867|United States|-6|apartment| +1570|AAAAAAAACCGAAAAA|618|Highland Madison|Drive|Suite 470|Forest Hills|Franklin County|WA|99237|United States|-8|condo| +1571|AAAAAAAADCGAAAAA|776|15th 8th|Way|Suite L|Five Forks|Escambia County|FL|32293|United States|-5|single family| +1572|AAAAAAAAECGAAAAA|607|Hillcrest Main|Ave|Suite Y|Riverview|Clinton County|MI|49003|United States|-5|apartment| +1573|AAAAAAAAFCGAAAAA|52|Elm |Ln|Suite E|Woodville|Addison County|VT|04889|United States|-5|condo| +1574|AAAAAAAAGCGAAAAA|69|15th |Blvd|Suite J|Dallas|Madison County|IL|63628|United States|-6|single family| +1575|AAAAAAAAHCGAAAAA|874|Washington Elm|Ct.|Suite 460|Five Forks|Wallace County|KS|62293|United States|-6|apartment| +1576|AAAAAAAAICGAAAAA|124|Second 4th|Dr.|Suite 440|Harmony|Newton County|MS|55804|United States|-6|single family| +1577|AAAAAAAAJCGAAAAA|486|Forest |Avenue|Suite 430|Union Hill|Clay County|SD|57746|United States|-6|condo| +1578|AAAAAAAAKCGAAAAA|556|Willow |Drive|Suite 200|Warwick|Sagadahoc County|ME|01998|United States|-5|single family| +1579|AAAAAAAALCGAAAAA|998|6th |Parkway|Suite J|Unionville|Bryan County|OK|71711|United States|-6|apartment| +1580|AAAAAAAAMCGAAAAA|594|Oak Ash|Pkwy|Suite 250|Liberty|Perry County|IL|63451|United States|-6|condo| +1581|AAAAAAAANCGAAAAA|811|Park |Blvd|Suite O|Kingston|Liberty County|FL|34975|United States|-5|apartment| +1582|AAAAAAAAOCGAAAAA|381|Park Spring|Blvd|Suite N|Mount Pleasant|Lamb County|TX|71933|United States|-6|condo| +1583|AAAAAAAAPCGAAAAA|457|3rd Highland|Parkway|Suite 50|Concord|Hand County|SD|54107|United States|-7|single family| +1584|AAAAAAAAADGAAAAA|619|Jefferson Hill|Road|Suite C|Riverview|Montgomery County|AL|39003|United States|-6|single family| +1585|AAAAAAAABDGAAAAA|994|13th |Blvd|Suite J|Mount Olive|Polk County|TN|38059|United States|-6|condo| +1586|AAAAAAAACDGAAAAA|983|Fifth |Road|Suite 470|Fairview|Robertson County|KY|45709|United States|-5|condo| +1587|AAAAAAAADDGAAAAA|546|Spring 4th|Ct.|Suite 410|Newport|Meade County|KY|41521|United States|-5|single family| +1588|AAAAAAAAEDGAAAAA|882|Main |Parkway|Suite O|Maple Grove|Scott County|MO|68252|United States|-6|apartment| +1589|AAAAAAAAFDGAAAAA|417|Spruce |Circle|Suite U|Woodville|Greenwood County|KS|64289|United States|-6|condo| +1590|AAAAAAAAGDGAAAAA|212|1st 4th|Ave|Suite 480|Riverside|Moultrie County|IL|69231|United States|-6|condo| +1591|AAAAAAAAHDGAAAAA|324|6th |Ct.|Suite Y|Hillcrest|Twiggs County|GA|33003|United States|-5|condo| +1592|AAAAAAAAIDGAAAAA|425|1st |Parkway|Suite 170|Springdale|Sublette County|WY|88883|United States|-7|single family| +1593|AAAAAAAAJDGAAAAA|501|4th Woodland|Drive|Suite 460|Edgewood|Custer County|CO|80069|United States|-7|apartment| +1594|AAAAAAAAKDGAAAAA|4|2nd Miller|Lane|Suite V|Red Hill|Columbia County|WI|54338|United States|-6|condo| +1595|AAAAAAAALDGAAAAA|597|Chestnut |Dr.|Suite H|Oak Grove|Dorchester County|MD|28370|United States|-5|single family| +1596|AAAAAAAAMDGAAAAA|77|Seventh |Wy|Suite 130|Walnut Grove|Wichita County|KS|67752|United States|-6|condo| +1597|AAAAAAAANDGAAAAA|191|11th |Dr.|Suite W|Saratoga|Clay County|AR|72123|United States|-6|condo| +1598|AAAAAAAAODGAAAAA|754|Jackson Spring|Way|Suite 200|Oak Ridge|Ottawa County|KS|68371|United States|-6|condo| +1599|AAAAAAAAPDGAAAAA|693|West |Parkway|Suite 110|Lakeside|Hardeman County|TX|79532|United States|-6|single family| +1600|AAAAAAAAAEGAAAAA|424|Johnson Railroad|Wy|Suite 470|Mount Pleasant|Merrick County|NE|61933|United States|-7|apartment| +1601|AAAAAAAABEGAAAAA|524|Birch Church|Ct.|Suite 90|Oakdale|Morgan County|WV|29584|United States|-5|single family| +1602|AAAAAAAACEGAAAAA|852|Pine Hillcrest|Pkwy|Suite C|Unionville|Harford County|MD|21711|United States|-5|apartment| +1603|AAAAAAAADEGAAAAA|515|3rd Washington|Circle|Suite P|Spring Grove|Ellis County|OK|76719|United States|-6|apartment| +1604|AAAAAAAAEEGAAAAA|234|Church |Way|Suite W|Richardson|Tom Green County|TX|77687|United States|-6|apartment| +1605|AAAAAAAAFEGAAAAA|561|Valley |Court|Suite C|Highland|Monroe County|PA|19454|United States|-5|apartment| +1606|AAAAAAAAGEGAAAAA|426|Hillcrest |Ln|Suite C|Kingston|Marshall County|WV|24975|United States|-5|single family| +1607|AAAAAAAAHEGAAAAA|594|Ridge 5th|Blvd|Suite 450|Greenwood|McMinn County|TN|38828|United States|-6|condo| +1608|AAAAAAAAIEGAAAAA|417|Lee |Ave|Suite X|Leesburg|Hall County|GA|35605|United States|-5|condo| +1609|AAAAAAAAJEGAAAAA|776|5th Fourth|Pkwy|Suite C|Spring Hill|Washington County|MS|56787|United States|-6|apartment| +1610|AAAAAAAAKEGAAAAA|313|1st |Lane|Suite 290|Union|Holt County|MO|68721|United States|-6|condo| +1611|AAAAAAAALEGAAAAA|868|9th View|RD|Suite 10|Lakeside|Wyandotte County|KS|69532|United States|-6|condo| +1612|AAAAAAAAMEGAAAAA|8|Maple Williams|Ct.|Suite C|Unionville|Osborne County|KS|61711|United States|-6|condo| +1613|AAAAAAAANEGAAAAA|304|Jackson |ST|Suite 50|Walnut Grove|McKenzie County|ND|57752|United States|-6|single family| +1614|AAAAAAAAOEGAAAAA|185|Hill Williams|Circle|Suite 240|Burton|Gilmer County|GA|35469|United States|-5|apartment| +1615|AAAAAAAAPEGAAAAA|717|Poplar 8th|Pkwy|Suite 450|Lakeview|Porter County|IN|48579|United States|-5|apartment| +1616|AAAAAAAAAFGAAAAA|46|Dogwood 1st|Drive|Suite 160|Summit|Miller County|AR|70499|United States|-6|condo| +1617|AAAAAAAABFGAAAAA|5|Spring 3rd|Road|Suite E|Jamestown|Will County|IL|66867|United States|-6|condo| +1618|AAAAAAAACFGAAAAA|819|Lake |Blvd|Suite 80|Philadelphia|Esmeralda County|NV|85591|United States|-8|single family| +1619|AAAAAAAADFGAAAAA|406|Sunset Maple|Boulevard|Suite Q|Oakwood|Richmond County|GA|30169|United States|-5|condo| +1620|AAAAAAAAEFGAAAAA|849|11th |ST|Suite 450|Arthur|Bath County|VA|25965|United States|-5|single family| +1621|AAAAAAAAFFGAAAAA|635|Central Forest|Pkwy|Suite D|Oakdale|Montague County|TX|79584|United States|-6|single family| +1622|AAAAAAAAGFGAAAAA|618|1st Ash|Blvd|Suite 360|Woodville|Pike County|PA|14289|United States|-5|apartment| +1623|AAAAAAAAHFGAAAAA|643|South |Parkway|Suite V|Highland|Dorchester County|SC|29454|United States|-5|apartment| +1624|AAAAAAAAIFGAAAAA|248|Jefferson |Avenue|Suite 40|Walnut Grove|Webster County|MO|67752|United States|-6|condo| +1625|AAAAAAAAJFGAAAAA|115|1st |Ave|Suite T|Belmont|Larue County|KY|40191|United States|-5|condo| +1626|AAAAAAAAKFGAAAAA|753|Johnson |Road|Suite 340|Waterloo|Scott County|IL|61675|United States|-6|apartment| +1627|AAAAAAAALFGAAAAA|647|2nd |Street|Suite F|Bethel|Mercer County|PA|15281|United States|-5|condo| +1628|AAAAAAAAMFGAAAAA|709|Lee Thirteenth|Cir.|Suite 80|Hamilton|Marshall County|TN|32808|United States|-6|single family| +1629|AAAAAAAANFGAAAAA|119|Dogwood |Ave|Suite 100|Oak Grove|Copiah County|MS|58370|United States|-6|single family| +1630|AAAAAAAAOFGAAAAA||5th 4th|Cir.|Suite 290|Ashland||IA|||-6|| +1631|AAAAAAAAPFGAAAAA|256|Dogwood |Court|Suite 230|Five Points|Lawrence County|PA|16098|United States|-5|condo| +1632|AAAAAAAAAGGAAAAA|12|Meadow |ST|Suite 440|Waterloo|Montour County|PA|11675|United States|-5|single family| +1633|AAAAAAAABGGAAAAA|838|Mill Oak|Road|Suite P|Hopewell|Lamar County|AL|30587|United States|-6|apartment| +1634|AAAAAAAACGGAAAAA|172|12th |Ln|Suite H|Bridgeport|Lamar County|TX|75817|United States|-6|condo| +1635|AAAAAAAADGGAAAAA|886|Cherry |Parkway|Suite 390|Union Hill|Washington County|OR|97746|United States|-8|apartment| +1636|AAAAAAAAEGGAAAAA|223|Maple |Street|Suite 370|Mountain View|Johnston County|OK|74466|United States|-6|condo| +1637|AAAAAAAAFGGAAAAA|998|Laurel Park|Street|Suite 350|Oakland|Gray County|KS|69843|United States|-6|apartment| +1638|AAAAAAAAGGGAAAAA|762|Poplar |Circle|Suite 380|Forestville|Poquoson city|VA|23027|United States|-5|apartment| +1639|AAAAAAAAHGGAAAAA|81|Ridge Forest|Circle|Suite 430|Oakwood|Queens County|NY|10169|United States|-5|single family| +1640|AAAAAAAAIGGAAAAA|314|12th |Circle|Suite 300|Forest Hills|Douglas County|MN|59237|United States|-6|apartment| +1641|AAAAAAAAJGGAAAAA|502|12th |Avenue|Suite 60|Weldon|Platte County|MO|66277|United States|-6|apartment| +1642|AAAAAAAAKGGAAAAA|594|Main |Avenue|Suite Q|Lebanon|Geary County|KS|62898|United States|-6|single family| +1643|AAAAAAAALGGAAAAA|351|View Oak|Road|Suite 420|Kimball|Floyd County|GA|33595|United States|-5|apartment| +1644|AAAAAAAAMGGAAAAA|13|Broadway 7th|Pkwy|Suite M|Williamsville|Petroleum County|MT|68754|United States|-7|single family| +1645|AAAAAAAANGGAAAAA|896|View 10th|Ave|Suite E|Greenfield|Calhoun County|IA|55038|United States|-6|single family| +1646|AAAAAAAAOGGAAAAA|131|Third |Drive|Suite I|Forest Hills|Lawrence County|PA|19237|United States|-5|apartment| +1647|AAAAAAAAPGGAAAAA|922|5th |Circle|Suite U|White Oak|Jefferson County|MT|66668|United States|-7|apartment| +1648|AAAAAAAAAHGAAAAA|87|Hickory Adams|Cir.|Suite K|Pleasant Hill|Elko County|NV|83604|United States|-8|condo| +1649|AAAAAAAABHGAAAAA|493|Green |Lane|Suite L|Green Acres|Decatur County|TN|37683|United States|-5|condo| +1650|AAAAAAAACHGAAAAA|277|Oak 2nd|Ct.|Suite C|Jacksonville|Palo Pinto County|TX|78223|United States|-6|condo| +1651|AAAAAAAADHGAAAAA|313|Laurel Pine|Road|Suite X|Brookwood|Oswego County|NY|10965|United States|-5|apartment| +1652|AAAAAAAAEHGAAAAA|130|Oak |Circle|Suite K|Macedonia|Atoka County|OK|71087|United States|-6|condo| +1653|AAAAAAAAFHGAAAAA|732|Green 5th|Boulevard|Suite 410|Sunnyside|McDonald County|MO|61952|United States|-6|apartment| +1654|AAAAAAAAGHGAAAAA|374|15th |Pkwy|Suite 60|Pleasant Grove|Gila County|AZ|84136|United States|-7|condo| +1655|AAAAAAAAHHGAAAAA|546|Forest |Wy|Suite N|Pleasant Valley|Rockcastle County|KY|42477|United States|-5|single family| +1656|AAAAAAAAIHGAAAAA|460|Pine Park|Dr.|Suite S|Glendale|Hidalgo County|TX|73951|United States|-6|apartment| +1657|AAAAAAAAJHGAAAAA|966|Hill Meadow|Cir.|Suite 50|Farmington|Montgomery County|IA|59145|United States|-6|single family| +1658|AAAAAAAAKHGAAAAA|640|Broadway Tenth|Pkwy|Suite C|Clifton|Eddy County|ND|58014|United States|-6|apartment| +1659|AAAAAAAALHGAAAAA|583|7th |Way|Suite 200|Hopewell|Dent County|MO|60587|United States|-6|single family| +1660|AAAAAAAAMHGAAAAA|969|15th 15th|Court|Suite 50|Riverview|Conway County|AR|79003|United States|-6|condo| +1661|AAAAAAAANHGAAAAA|322|12th |Lane|Suite T|Bridgeport|Anoka County|MN|55817|United States|-6|condo| +1662|AAAAAAAAOHGAAAAA|956|12th Lincoln|Court|Suite 370|Ellisville|Lauderdale County|TN|36820|United States|-6|apartment| +1663|AAAAAAAAPHGAAAAA|988|3rd |Cir.|Suite 360|Mount Vernon|Gilchrist County|FL|38482|United States|-5|condo| +1664|AAAAAAAAAIGAAAAA|970|Fifth |Court|Suite 380|Greenville|Perquimans County|NC|21387|United States|-5|apartment| +1665|AAAAAAAABIGAAAAA|151|Railroad |Drive|Suite 420|Crossroads|Reagan County|TX|70534|United States|-6|apartment| +1666|AAAAAAAACIGAAAAA|793|Hill |Street|Suite 220|Georgetown|Eagle County|CO|87057|United States|-7|single family| +1667|AAAAAAAADIGAAAAA|536|7th |Court|Suite 40|Georgetown|Socorro County|NM|87057|United States|-7|condo| +1668|AAAAAAAAEIGAAAAA|321|Valley Maple|Ct.|Suite 430|Bunker Hill|Gordon County|GA|30150|United States|-5|single family| +1669|AAAAAAAAFIGAAAAA|||Street|Suite 100||Otoe County|NE|61521|United States||apartment| +1670|AAAAAAAAGIGAAAAA|497|Jefferson Washington|Ave|Suite D|Wilson|Nantucket County|MA|07571|United States|-5|single family| +1671|AAAAAAAAHIGAAAAA|862|Birch |Wy|Suite 90|Brunswick|Wyandotte County|KS|64642|United States|-6|single family| +1672|AAAAAAAAIIGAAAAA|716|Center Johnson|Parkway|Suite G|Summit|Fairbanks North Star Borough|AK|90499|United States|-9|condo| +1673|AAAAAAAAJIGAAAAA|271|Willow Jefferson|Dr.|Suite 270|Lakeside|Tom Green County|TX|79532|United States|-6|apartment| +1674|AAAAAAAAKIGAAAAA|||Parkway|Suite A|Belmont|Fairfield County|CT|00791|||| +1675|AAAAAAAALIGAAAAA|8|Elm |Way|Suite I|Brownsville|Greenup County|KY|49310|United States|-6|apartment| +1676|AAAAAAAAMIGAAAAA|768|Maple Second|Dr.|Suite 40|Redland|Eddy County|ND|56343|United States|-6|single family| +1677|AAAAAAAANIGAAAAA|393|Third |Cir.|Suite X|Oak Grove|Mariposa County|CA|98370|United States|-8|condo| +1678|AAAAAAAAOIGAAAAA|381|North |Court|Suite 390|Pleasant Grove|Clay County|IN|44136|United States|-5|condo| +1679|AAAAAAAAPIGAAAAA|580|Hickory Sixth|Pkwy|Suite H|Jamestown|Dawson County|MT|66867|United States|-7|single family| +1680|AAAAAAAAAJGAAAAA|530|Sunset Second|Way|Suite 40|Lincoln|Elbert County|GA|31289|United States|-5|single family| +1681|AAAAAAAABJGAAAAA||Lee Main||Suite 350|Springdale||||United States||| +1682|AAAAAAAACJGAAAAA|422|10th |Drive|Suite 230|Bridgeport|Morrow County|OH|45817|United States|-5|condo| +1683|AAAAAAAADJGAAAAA|179|Smith |Drive|Suite 60|Mount Pleasant|Elmore County|AL|31933|United States|-6|apartment| +1684|AAAAAAAAEJGAAAAA|164|Jackson |Cir.|Suite Q|Franklin|Sherman County|TX|79101|United States|-6|condo| +1685|AAAAAAAAFJGAAAAA|278|Williams Park|Way|Suite 400|Jenkins|Wright County|MN|57292|United States|-6|condo| +1686|AAAAAAAAGJGAAAAA|900|Ninth |Blvd|Suite 190|Pleasant Valley|Madison County|NY|12477|United States|-5|condo| +1687|AAAAAAAAHJGAAAAA|145|Locust |Boulevard|Suite 130|Clinton|Clay County|KY|48222|United States|-6|condo| +1688|AAAAAAAAIJGAAAAA|675|Hillcrest 4th|Avenue|Suite Y|Woodlawn|Carroll County|MS|54098|United States|-6|apartment| +1689|AAAAAAAAJJGAAAAA|481|North 3rd|Street|Suite U|Wildwood|Jefferson County|IL|66871|United States|-6|single family| +1690|AAAAAAAAKJGAAAAA|187|14th Ridge|Parkway|Suite L|Newtown|Douglas County|WI|51749|United States|-6|single family| +1691|AAAAAAAALJGAAAAA|967|Valley |Ct.|Suite X|Unionville|Cortland County|NY|11711|United States|-5|condo| +1692|AAAAAAAAMJGAAAAA|502|Sunset Johnson|Boulevard|Suite N|Providence|Clinton County|MO|66614|United States|-6|single family| +1693|AAAAAAAANJGAAAAA|580|Fourth Highland|Way|Suite R|Fairfield|Henry County|IN|46192|United States|-5|condo| +1694|AAAAAAAAOJGAAAAA||Church |Court||Richfield||FL||United States|-5|apartment| +1695|AAAAAAAAPJGAAAAA|19|Madison Hickory|Court|Suite S|Kingston|Niagara County|NY|14975|United States|-5|single family| +1696|AAAAAAAAAKGAAAAA|703|River West|Dr.|Suite 270|Buena Vista|Hunterdon County|NJ|06352|United States|-5|single family| +1697|AAAAAAAABKGAAAAA|425|Sycamore |Street|Suite F|Bunker Hill|Champaign County|IL|60150|United States|-6|condo| +1698|AAAAAAAACKGAAAAA|142|Dogwood Mill|Lane|Suite R|Arlington|Coffee County|AL|36557|United States|-6|single family| +1699|AAAAAAAADKGAAAAA|15|Cherry |Wy|Suite L|Wilson|Pike County|IN|46971|United States|-5|apartment| +1700|AAAAAAAAEKGAAAAA|901|Highland Davis|Ct.|Suite 420|Lakewood|Washington County|TX|78877|United States|-6|condo| +1701|AAAAAAAAFKGAAAAA|303|3rd Hill|Blvd|Suite M|Liberty|Multnomah County|OR|93451|United States|-8|apartment| +1702|AAAAAAAAGKGAAAAA|252|Fourth |Boulevard|Suite 70|Lincoln|Madera County|CA|91289|United States|-8|apartment| +1703|AAAAAAAAHKGAAAAA|37|Miller Franklin|Drive|Suite R|Jackson|Anderson County|TX|79583|United States|-6|condo| +1704|AAAAAAAAIKGAAAAA|492|Walnut 1st|Avenue|Suite 420|Fairfield|Pecos County|TX|76192|United States|-6|apartment| +1705|AAAAAAAAJKGAAAAA|788|Elm 11th|Ct.|Suite 170|Waterloo|Brule County|SD|51675|United States|-6|condo| +1706|AAAAAAAAKKGAAAAA|274|Washington |Parkway|Suite J|Union|Roosevelt County|NM|88721|United States|-7|single family| +1707|AAAAAAAALKGAAAAA|278|Franklin |Dr.|Suite S|Silver Springs|Bay County|FL|34843|United States|-5|apartment| +1708|AAAAAAAAMKGAAAAA|828|Oak Jefferson|Circle|Suite 480|Glenwood|Eau Claire County|WI|53511|United States|-6|condo| +1709|AAAAAAAANKGAAAAA|607|13th |Ave|Suite F|Red Hill|Montgomery County|MO|64338|United States|-6|condo| +1710|AAAAAAAAOKGAAAAA|109|First 6th|Circle|Suite F|Riverside|Juneau County|WI|59231|United States|-6|condo| +1711|AAAAAAAAPKGAAAAA|469|Third |Lane|Suite 20|Georgetown|Valley County|ID|87057|United States|-7|single family| +1712|AAAAAAAAALGAAAAA|220|Walnut |Circle|Suite O|Salem|Ziebach County|SD|58048|United States|-6|single family| +1713|AAAAAAAABLGAAAAA|759|Cedar Green|Parkway|Suite G|Midway|Wallace County|KS|61904|United States|-6|condo| +1714|AAAAAAAACLGAAAAA|223|Green |Circle|Suite H|Spring Valley|Clinton County|NY|16060|United States|-5|single family| +1715|AAAAAAAADLGAAAAA|922|Ridge 4th|Dr.|Suite B|Georgetown|Cass County|IA|57057|United States|-6|single family| +1716|AAAAAAAAELGAAAAA|682|13th |||||VA||United States|-5|single family| +1717|AAAAAAAAFLGAAAAA|695|Eigth Oak|Ave|Suite 350|Springfield|Torrance County|NM|89303|United States|-7|single family| +1718|AAAAAAAAGLGAAAAA||3rd North|Cir.|Suite Y|Florence|Lenoir County||23394|United States||| +1719|AAAAAAAAHLGAAAAA|962|4th Williams|Ct.|Suite 0|Richardson|Marion County|KS|67687|United States|-6|condo| +1720|AAAAAAAAILGAAAAA|630|Birch Oak|Parkway|Suite 210|Greenville|Dixon County|NE|61387|United States|-6|apartment| +1721|AAAAAAAAJLGAAAAA|490|Davis 10th|ST|Suite D|||ID|||-7|| +1722|AAAAAAAAKLGAAAAA|428|Hillcrest |Boulevard|Suite 320|Unionville|Emery County|UT|81711|United States|-7|single family| +1723|AAAAAAAALLGAAAAA|855|Cherry Chestnut|Ave|Suite Q|Hopewell|Power County|ID|80587|United States|-7|single family| +1724|AAAAAAAAMLGAAAAA|493|Madison |Wy|Suite D|Woodlawn|Dixon County|NE|64098|United States|-6|single family| +1725|AAAAAAAANLGAAAAA|424|Pine |Road|Suite L|Unionville|Lincoln County|WA|91711|United States|-8|single family| +1726|AAAAAAAAOLGAAAAA|216|7th |Circle|Suite 280|Mechanicsburg|Bristol city|VA|22219|United States|-5|apartment| +1727|AAAAAAAAPLGAAAAA|959|10th |Lane|Suite 410|Plainview|Yell County|AR|73683|United States|-6|single family| +1728|AAAAAAAAAMGAAAAA|882|8th |Ct.|Suite 130|Red Hill|Phillips County|AR|74338|United States|-6|apartment| +1729|AAAAAAAABMGAAAAA|240|Washington |Street|Suite I|Philadelphia|Upshur County|WV|25591|United States|-5|apartment| +1730|AAAAAAAACMGAAAAA|117|Third Jackson|Circle|Suite W|Friendship|Baker County|OR|94536|United States|-8|condo| +1731|AAAAAAAADMGAAAAA|||||||||United States||| +1732|AAAAAAAAEMGAAAAA|504|3rd |Way|Suite N|Jackson|Bartow County|GA|39583|United States|-5|condo| +1733|AAAAAAAAFMGAAAAA|140|Third |Court|Suite D|Cedar Grove|Wise County|TX|70411|United States|-6|single family| +1734|AAAAAAAAGMGAAAAA|313|11th 10th|Blvd|Suite Y|Lebanon|Matanuska-Susitna Borough|AK|92898|United States|-9|condo| +1735|AAAAAAAAHMGAAAAA|366|5th |Ln|Suite U|Lakewood|Cloud County|KS|68877|United States|-6|condo| +1736|AAAAAAAAIMGAAAAA|453|Park |Boulevard|Suite 410|Forest Hills|Sullivan County|PA|19237|United States|-5|single family| +1737|AAAAAAAAJMGAAAAA|746|Adams |Street|Suite 430|Five Points|Moody County|SD|56098|United States|-7|apartment| +1738|AAAAAAAAKMGAAAAA|||||White Oak|Cedar County||66668|United States|-6|| +1739|AAAAAAAALMGAAAAA|770|Jefferson |Way|Suite Q|Fairfield|Pitt County|NC|26192|United States|-5|apartment| +1740|AAAAAAAAMMGAAAAA|839|Hillcrest 5th|Ave|Suite S|Spring Hill|Mineral County|NV|86787|United States|-8|condo| +1741|AAAAAAAANMGAAAAA|334|Ash |Ave|Suite 280|Blair|Martin County|IN|45465|United States|-5|single family| +1742|AAAAAAAAOMGAAAAA|614|Third Lake|Avenue|Suite R|Highland|DeSoto County|FL|39454|United States|-5|single family| +1743|AAAAAAAAPMGAAAAA|370|2nd Sunset|Blvd|Suite A|Riverside|Bath County|VA|29231|United States|-5|single family| +1744|AAAAAAAAANGAAAAA|4|Green |Street|Suite B|Fowler|Storey County|NV|81083|United States|-8|apartment| +1745|AAAAAAAABNGAAAAA|977|Jackson Sunset|Wy|Suite 230|Woodland|Lafourche Parish|LA|74854|United States|-6|condo| +1746|AAAAAAAACNGAAAAA|583|Ridge |Blvd|Suite 250|Pomona|Osceola County|FL|34153|United States|-5|apartment| +1747|AAAAAAAADNGAAAAA|138|Fifth Sixth|Lane|Suite O|Buena Vista|Cottle County|TX|75752|United States|-6|single family| +1748|AAAAAAAAENGAAAAA|619|Second |Circle|Suite 190|Belmont|Barbour County|WV|20191|United States|-5|apartment| +1749|AAAAAAAAFNGAAAAA|814|South |Street|Suite 60|Harmony|Union County|IA|55804|United States|-6|condo| +1750|AAAAAAAAGNGAAAAA|504|Smith |Cir.|Suite L|Clifton|Oconee County|SC|28014|United States|-5|condo| +1751|AAAAAAAAHNGAAAAA|936|2nd |Dr.|Suite 480|Summit|Hot Springs County|WY|80499|United States|-7|condo| +1752|AAAAAAAAINGAAAAA|896|Forest |Street|Suite 350|Jamestown|Tunica County|MS|56867|United States|-6|single family| +1753|AAAAAAAAJNGAAAAA|156|15th Miller|Lane|Suite Q|Glenwood|Crawford County|PA|13511|United States|-5|condo| +1754|AAAAAAAAKNGAAAAA|464|Pine |Ave|Suite O|Brownsville|DeWitt County|TX|79310|United States|-6|condo| +1755|AAAAAAAALNGAAAAA|539|Madison Cherry|Ave|Suite 460|Flatwoods|Roane County|TN|34212|United States|-6|apartment| +1756|AAAAAAAAMNGAAAAA|692|Lake |Road|Suite U|Hopewell|Oakland County|MI|40587|United States|-5|single family| +1757|AAAAAAAANNGAAAAA|347|Second |Dr.|Suite W|Stringtown|Franklin County|GA|30162|United States|-5|single family| +1758|AAAAAAAAONGAAAAA|753|Franklin Lincoln|Cir.|Suite T|Mount Vernon|Collier County|FL|38482|United States|-5|apartment| +1759|AAAAAAAAPNGAAAAA|127|Williams 5th|Ave|Suite 280|Walnut Grove|Burke County|GA|37752|United States|-5|condo| +1760|AAAAAAAAAOGAAAAA|745|View |Wy|Suite N|Brunswick|Martin County|FL|34642|United States|-5|apartment| +1761|AAAAAAAABOGAAAAA|859|Park Williams|Way|Suite A|Wilson|Clarendon County|SC|26971|United States|-5|single family| +1762|AAAAAAAACOGAAAAA|||Dr.|||Jewell County|KS|64593|United States|-6|single family| +1763|AAAAAAAADOGAAAAA|303|10th Adams|Ave|Suite D|Mountain View|Carroll County|IN|44466|United States|-5|apartment| +1764|AAAAAAAAEOGAAAAA|155|7th |Road|Suite 90|Salem|Riley County|KS|68048|United States|-6|condo| +1765|AAAAAAAAFOGAAAAA|639|Lincoln |Lane|Suite 210|Jamestown|Concordia Parish|LA|76867|United States|-6|single family| +1766|AAAAAAAAGOGAAAAA|396|Locust Hill|Ave|Suite L|Summit|Santa Clara County|CA|90499|United States|-8|single family| +1767|AAAAAAAAHOGAAAAA|211|5th Park|ST|Suite 180|Newport|Carroll County|GA|31521|United States|-5|condo| +1768|AAAAAAAAIOGAAAAA|397|1st Green|ST|Suite 150|Pleasant Grove|Bon Homme County|SD|54136|United States|-6|single family| +1769|AAAAAAAAJOGAAAAA|744|Locust Park|Ct.|Suite 110|Lakeview|Clay County|FL|38579|United States|-5|single family| +1770|AAAAAAAAKOGAAAAA|104|Davis |Drive|Suite 80|Jackson|Audubon County|IA|59583|United States|-6|condo| +1771|AAAAAAAALOGAAAAA|820|Laurel |Lane|Suite 250|Hopewell|Jefferson County|KS|60587|United States|-6|apartment| +1772|AAAAAAAAMOGAAAAA|287|Jackson |Court|Suite A|Springfield|Marion County|AL|39303|United States|-6|apartment| +1773|AAAAAAAANOGAAAAA|793|First Elm|Avenue|Suite 280|Summit|McIntosh County|ND|50499|United States|-6|condo| +1774|AAAAAAAAOOGAAAAA|178|Williams 4th|Cir.|Suite 470|Centerville|Auglaize County|OH|40059|United States|-5|single family| +1775|AAAAAAAAPOGAAAAA|58|8th |Drive|Suite 460|Fairview|Greeley County|KS|65709|United States|-6|condo| +1776|AAAAAAAAAPGAAAAA|158|Maple Main|ST|Suite 390|New Hope|Geneva County|AL|39431|United States|-6|single family| +1777|AAAAAAAABPGAAAAA|414|5th |Ct.|Suite 220|Hamilton|Hampton city|VA|22808|United States|-5|condo| +1778|AAAAAAAACPGAAAAA|670|Lake |Wy|Suite 280|Pleasant Valley|Cumberland County|PA|12477|United States|-5|apartment| +1779|AAAAAAAADPGAAAAA|749|Seventh |Lane|Suite U|Union|Starr County|TX|78721|United States|-6|condo| +1780|AAAAAAAAEPGAAAAA|740|Maple 14th|Ct.|Suite 70|Unionville|Marshall County|SD|51711|United States|-7|condo| +1781|AAAAAAAAFPGAAAAA|541|East Williams|Street|Suite O|Spring Valley|Itawamba County|MS|56060|United States|-6|apartment| +1782|AAAAAAAAGPGAAAAA|598|Park |ST|Suite W|Antioch|Patrick County|VA|28605|United States|-5|single family| +1783|AAAAAAAAHPGAAAAA|585|12th |Drive|Suite 360|Friendship|Titus County|TX|74536|United States|-6|single family| +1784|AAAAAAAAIPGAAAAA|617|Hill |Cir.|Suite W|White Oak|Sussex County|VA|26668|United States|-5|condo| +1785|AAAAAAAAJPGAAAAA|342|Williams 7th|Way|Suite M|Providence|Berkeley County|SC|26614|United States|-5|condo| +1786|AAAAAAAAKPGAAAAA|148|10th |Boulevard|Suite 300|Marion|Butler County|KS|60399|United States|-6|apartment| +1787|AAAAAAAALPGAAAAA|605|1st |Avenue|Suite O|Lakewood|Johnston County|NC|28877|United States|-5|condo| +1788|AAAAAAAAMPGAAAAA|2|3rd 7th|Avenue|Suite X|Mountain View|Summit County|CO|84466|United States|-7|single family| +1789|AAAAAAAANPGAAAAA|493|Third |Avenue|Suite R|Sulphur Springs|McLean County|KY|48354|United States|-5|condo| +1790|AAAAAAAAOPGAAAAA|554|First |Ct.|Suite 80|Thompsonville|Moody County|SD|59651|United States|-7|condo| +1791|AAAAAAAAPPGAAAAA|581|10th |Dr.|Suite S|Oakdale|Cheyenne County|CO|89584|United States|-7|apartment| +1792|AAAAAAAAAAHAAAAA|635|Lake Walnut|ST|Suite Q|Fairview|Ward County|ND|55709|United States|-6|single family| +1793|AAAAAAAABAHAAAAA|571|Oak |Ln|Suite C|Lebanon|Hampton County|SC|22898|United States|-5|single family| +1794|AAAAAAAACAHAAAAA|770|Valley Ridge|Ct.|Suite 280|Sulphur Springs|Jefferson County|MO|68354|United States|-6|apartment| +1795|AAAAAAAADAHAAAAA|166|Hill Sycamore|Avenue|Suite 30|Shiloh|McHenry County|IL|69275|United States|-6|single family| +1796|AAAAAAAAEAHAAAAA|76|Spring Main|Pkwy|Suite B|Glenwood|Natchitoches Parish|LA|73511|United States|-6|single family| +1797|AAAAAAAAFAHAAAAA|805|Hillcrest Main|Street|Suite 0|Mount Zion||||United States||single family| +1798|AAAAAAAAGAHAAAAA|915|Lake Lake|Circle|Suite C|Fairview|Napa County|CA|95709|United States|-8|single family| +1799|AAAAAAAAHAHAAAAA|271|Poplar |Drive|Suite 380|Forest|Potter County|PA|17537|United States|-5|single family| +1800|AAAAAAAAIAHAAAAA|513|11th 9th|Avenue|Suite Q|Hopewell|Victoria County|TX|70587|United States|-6|apartment| +1801|AAAAAAAAJAHAAAAA|382|Church |Way|Suite X|Union|Honolulu County|HI|98721|United States|-10|single family| +1802|AAAAAAAAKAHAAAAA|514|River Williams|Boulevard|Suite 330|Greenwood|Cherokee County|OK|78828|United States|-6|single family| +1803|AAAAAAAALAHAAAAA|809|Hickory |ST|Suite 200|Belmont|Edgar County|IL|60191|United States|-6|condo| +1804|AAAAAAAAMAHAAAAA|948|14th View|RD|Suite 150|Oakwood|Norfolk city|VA|20169|United States|-5|single family| +1805|AAAAAAAANAHAAAAA|728|4th |Ave|Suite 220|Delmar|Marion County|GA|33957|United States|-5|single family| +1806|AAAAAAAAOAHAAAAA|294||Ave||||IA||United States||| +1807|AAAAAAAAPAHAAAAA|288|12th Birch|Ct.|Suite G|Oakdale|Lehigh County|PA|19584|United States|-5|single family| +1808|AAAAAAAAABHAAAAA|38|Central |Road|Suite 100|Bridgeport|Arkansas County|AR|75817|United States|-6|apartment| +1809|AAAAAAAABBHAAAAA|36|Seventh First|Lane|Suite 160|Lincoln|King William County|VA|21289|United States|-5|condo| +1810|AAAAAAAACBHAAAAA|859|Johnson |Blvd|Suite 400|Pleasant Hill|Kent County|MD|23604|United States|-5|single family| +1811|AAAAAAAADBHAAAAA|922|Smith |Boulevard|Suite K|Enterprise|Jefferson County|WI|51757|United States|-6|apartment| +1812|AAAAAAAAEBHAAAAA||West ||Suite 440||||28370||-5|| +1813|AAAAAAAAFBHAAAAA|483|Third Church|Parkway|Suite R|Georgetown|Boone County|MO|67057|United States|-6|single family| +1814|AAAAAAAAGBHAAAAA|345|8th |ST|Suite B|Franklin|Cherokee County|IA|59101|United States|-6|single family| +1815|AAAAAAAAHBHAAAAA|947|Lake Twelfth|Way|Suite A|Liberty|Page County|VA|23451|United States|-5|condo| +1816|AAAAAAAAIBHAAAAA|785|Maple West|ST|Suite J|Unionville|Barnstable County|MA|02311|United States|-5|single family| +1817|AAAAAAAAJBHAAAAA|709|Hickory Pine|Ave|Suite V|Plainview|Meigs County|OH|43683|United States|-5|condo| +1818|AAAAAAAAKBHAAAAA|490|4th Hillcrest|Cir.|Suite H|Blue Springs|Ashe County|NC|24686|United States|-5|apartment| +1819|AAAAAAAALBHAAAAA|27|Cherry Chestnut|Drive|Suite T|Enterprise|Geauga County|OH|41757|United States|-5|condo| +1820|AAAAAAAAMBHAAAAA|591|Johnson |Road|Suite Y|Oakdale|Morris County|TX|79584|United States|-6|single family| +1821|AAAAAAAANBHAAAAA|209|Fourth Forest|RD|Suite E|Liberty|Hamilton County|KS|63451|United States|-6|condo| +1822|AAAAAAAAOBHAAAAA|439|Chestnut 3rd|Blvd|Suite T|Edgewater|Marion County|AL|30635|United States|-6|single family| +1823|AAAAAAAAPBHAAAAA|570|First 12th|Road|Suite 30|Carpenter|Shannon County|MO|61147|United States|-6|single family| +1824|AAAAAAAAACHAAAAA|72|Hillcrest |RD|Suite 140|Walnut Grove|Las Animas County|CO|87752|United States|-7|condo| +1825|AAAAAAAABCHAAAAA|335|Miller North|Lane|Suite 40|Green Acres|Pepin County|WI|57683|United States|-6|condo| +1826|AAAAAAAACCHAAAAA|138|Lake Willow|Cir.|Suite G|Riverview|Muscogee County|GA|39003|United States|-5|apartment| +1827|AAAAAAAADCHAAAAA|388|Woodland Jackson|Circle|Suite 210|Wildwood|Winneshiek County|IA|56871|United States|-6|condo| +1828|AAAAAAAAECHAAAAA|198|Hickory |Drive|Suite 270|Marion|Pitt County|NC|20399|United States|-5|single family| +1829|AAAAAAAAFCHAAAAA|677|Seventh |Ave|Suite F|Woodville|Canadian County|OK|74289|United States|-6|condo| +1830|AAAAAAAAGCHAAAAA|80|Laurel 3rd|Cir.|Suite W|Walnut Grove|Montgomery County|TN|37752|United States|-6|apartment| +1831|AAAAAAAAHCHAAAAA|850|Lake 13th|RD|Suite U|Lewisville|Bosque County|TX|70056|United States|-6|apartment| +1832|AAAAAAAAICHAAAAA|||Ct.|Suite E|Birmingham||NE||||| +1833|AAAAAAAAJCHAAAAA|935|7th |Dr.|Suite A|Maple Grove|Bureau County|IL|68252|United States|-6|condo| +1834|AAAAAAAAKCHAAAAA|314|1st |Road|Suite N|Shiloh|Missoula County|MT|69275|United States|-7|apartment| +1835|AAAAAAAALCHAAAAA|514|Lee Pine|Ln|Suite 90|Belmont|Clay County|KY|40191|United States|-6|condo| +1836|AAAAAAAAMCHAAAAA|76|1st |Lane|Suite N|Unionville|Humboldt County|IA|51711|United States|-6|single family| +1837|AAAAAAAANCHAAAAA|713||||||MI||United States||single family| +1838|AAAAAAAAOCHAAAAA|502|5th |Road|Suite G|Green Acres|Otsego County|MI|47683|United States|-5|condo| +1839|AAAAAAAAPCHAAAAA|859|Sunset 10th|Cir.|Suite 160|Edgewood|Fulton County|KY|40069|United States|-6|single family| +1840|AAAAAAAAADHAAAAA|205|Valley Birch|Blvd|Suite T|Salem|Campbell County|SD|58048|United States|-6|apartment| +1841|AAAAAAAABDHAAAAA||Fourth Franklin|RD||Sunnyside|||31952||-5|apartment| +1842|AAAAAAAACDHAAAAA|943|Madison Washington|Wy|Suite 150|Bunker Hill|Costilla County|CO|80150|United States|-7|single family| +1843|AAAAAAAADDHAAAAA|407|6th Spring|Lane|Suite 310|Sleepy Hollow|Osage County|KS|63592|United States|-6|apartment| +1844|AAAAAAAAEDHAAAAA|174|Spring Jackson|Court|Suite 240|Lakeside|Dade County|MO|69532|United States|-6|apartment| +1845|AAAAAAAAFDHAAAAA|890|Main West|Ln|Suite C|Bennett|Houston County|TN|31715|United States|-5|apartment| +1846|AAAAAAAAGDHAAAAA|823|Jackson |Circle|Suite N|Mount Pleasant|Madison County|OH|41933|United States|-5|single family| +1847|AAAAAAAAHDHAAAAA|859|1st Miller|Cir.|Suite 410|Pleasant Valley|Winkler County|TX|72477|United States|-6|apartment| +1848|AAAAAAAAIDHAAAAA|895|Meadow Lee|Blvd|Suite K|Bridgeport|Rogers County|OK|75817|United States|-6|condo| +1849|AAAAAAAAJDHAAAAA|213|River Fourth|Way|Suite C|Woodlawn|Bedford city|VA|24098|United States|-5|single family| +1850|AAAAAAAAKDHAAAAA|279|6th |Dr.|Suite 440|Newtown|Kenedy County|TX|71749|United States|-6|condo| +1851|AAAAAAAALDHAAAAA|664|Forest Oak|Street|Suite 70|Macedonia|Yalobusha County|MS|51087|United States|-6|apartment| +1852|AAAAAAAAMDHAAAAA|937|11th |Road|Suite 490|Mount Vernon|Frio County|TX|78482|United States|-6|single family| +1853|AAAAAAAANDHAAAAA|777|Ridge 4th|Circle|Suite 230|Murray|Kings County|CA|92150|United States|-8|apartment| +1854|AAAAAAAAODHAAAAA|827|Oak |Cir.|Suite 420|White Hall|Nuckolls County|NE|66955|United States|-7|apartment| +1855|AAAAAAAAPDHAAAAA|378|Main Fifth|Boulevard|Suite I|Wilson|Traverse County|MN|56971|United States|-6|condo| +1856|AAAAAAAAAEHAAAAA|342|Central Laurel|Way|Suite L|Sulphur Springs|Polk County|MN|58354|United States|-6|single family| +1857|AAAAAAAABEHAAAAA|295|Locust Main|Court|Suite B|Newport|Erie County|NY|11521|United States|-5|condo| +1858|AAAAAAAACEHAAAAA|817|8th 9th|Ln|Suite 70|Lebanon|Prentiss County|MS|52898|United States|-6|condo| +1859|AAAAAAAADEHAAAAA|588|Park |Court|Suite D|Saint George|Vance County|NC|25281|United States|-5|condo| +1860|AAAAAAAAEEHAAAAA|402|View |Avenue|Suite N|Oak Grove|Avery County|NC|28370|United States|-5|condo| +1861|AAAAAAAAFEHAAAAA|533|Green Cedar|Drive|Suite R|Summerville|Jefferson County|TX|72033|United States|-6|apartment| +1862|AAAAAAAAGEHAAAAA||15th |Blvd|Suite 370||Beltrami County||57838||-6|| +1863|AAAAAAAAHEHAAAAA|504|Miller Mill|Wy|Suite 490|Oakland|Sumner County|TN|39843|United States|-6|condo| +1864|AAAAAAAAIEHAAAAA|844|Cherry Washington|Ln|Suite 370|Woodville|Washakie County|WY|84289|United States|-7|apartment| +1865|AAAAAAAAJEHAAAAA|861|6th Main|Dr.|Suite 300|Concord|San Bernardino County|CA|94107|United States|-8|apartment| +1866|AAAAAAAAKEHAAAAA|371|Forest Railroad|Wy|||Marion County||64167|United States|-6|| +1867|AAAAAAAALEHAAAAA|707|15th |Drive|Suite F|Stringtown|Tipton County|TN|30162|United States|-6|condo| +1868|AAAAAAAAMEHAAAAA|896|Lakeview |Dr.|Suite 380|Newtown|Dewey County|SD|51749|United States|-6|single family| +1869|AAAAAAAANEHAAAAA|715|Park Miller|Court|Suite N|Midway|Mills County|TX|71904|United States|-6|condo| +1870|AAAAAAAAOEHAAAAA|161|River 9th|Ave|Suite Y|Concord|Lee County|AR|74107|United States|-6|condo| +1871|AAAAAAAAPEHAAAAA|998|Green |Parkway|Suite 50|Oakland|Tioga County|PA|19843|United States|-5|single family| +1872|AAAAAAAAAFHAAAAA|675|Valley |Cir.|Suite C|Shady Grove|Ward County|ND|52812|United States|-6|apartment| +1873|AAAAAAAABFHAAAAA|271|Hill |ST|Suite 380|Warwick|Buena Vista city|VA|21398|United States|-5|single family| +1874|AAAAAAAACFHAAAAA|241|Valley 5th|Road|Suite W|Mount Zion|Denali Borough|AK|98054|United States|-9|apartment| +1875|AAAAAAAADFHAAAAA|613|Walnut Locust|Cir.|Suite B|Waterloo|Monmouth County|NJ|02275|United States|-5|condo| +1876|AAAAAAAAEFHAAAAA|471|Poplar |Wy|Suite R|Oakwood|Cherokee County|AL|30169|United States|-6|single family| +1877|AAAAAAAAFFHAAAAA|383|Park View|RD|Suite Q|Wright|Yell County|AR|72814|United States|-6|condo| +1878|AAAAAAAAGFHAAAAA|642|Wilson 8th|Wy|Suite 60|Macedonia|Sublette County|WY|81087|United States|-7|single family| +1879|AAAAAAAAHFHAAAAA|309|1st |Ln|Suite 460|Bunker Hill|Plumas County|CA|90150|United States|-8|apartment| +1880|AAAAAAAAIFHAAAAA|||||||NJ|09831|United States|-5|| +1881|AAAAAAAAJFHAAAAA|211|Walnut Hill|Boulevard|Suite 390|Providence|Carroll County|MO|66614|United States|-6|apartment| +1882|AAAAAAAAKFHAAAAA|883|Chestnut Walnut|Way|Suite 410|Highland|Dewey County|SD|59454|United States|-6|condo| +1883|AAAAAAAALFHAAAAA|446|Ridge Hickory|Avenue|Suite A|Bridgeport|Hockley County|TX|75817|United States|-6|apartment| +1884|AAAAAAAAMFHAAAAA|63|Second 1st|Way|Suite 220|Franklin|Marshall County|IL|69101|United States|-6|single family| +1885|AAAAAAAANFHAAAAA|254|Washington Birch|Parkway|Suite 270|Pleasant Hill|Judith Basin County|MT|63604|United States|-7|apartment| +1886|AAAAAAAAOFHAAAAA|22|College |Avenue|Suite B|Glenwood|Sumner County|TN|33511|United States|-6|single family| +1887|AAAAAAAAPFHAAAAA|802|Madison |Pkwy|Suite C|Georgetown|Pendleton County|WV|27057|United States|-5|apartment| +1888|AAAAAAAAAGHAAAAA|608|Poplar |Ct.|Suite W|Arlington|Mariposa County|CA|96557|United States|-8|apartment| +1889|AAAAAAAABGHAAAAA|251|Madison 3rd|Ave|Suite 380|Harmony|Webster County|GA|35804|United States|-5|single family| +1890|AAAAAAAACGHAAAAA|123|5th Cedar|Drive|Suite R|Crossroads|Wicomico County|MD|20534|United States|-5|condo| +1891|AAAAAAAADGHAAAAA|543|Oak Miller|Way|Suite S|Union Hill|McMinn County|TN|37746|United States|-6|condo| +1892|AAAAAAAAEGHAAAAA|79|Valley |Cir.|Suite 310|Oakdale|Jefferson County|KS|69584|United States|-6|apartment| +1893|AAAAAAAAFGHAAAAA||||Suite V|Stewart|Sac County||58041|||| +1894|AAAAAAAAGGHAAAAA|631|North |RD|Suite 480|Plainview|Cheshire County|NH|04283|United States|-5|condo| +1895|AAAAAAAAHGHAAAAA|670|10th |Cir.|Suite C|Glenwood|Wilson County|KS|63511|United States|-6|single family| +1896|AAAAAAAAIGHAAAAA|609|Washington Wilson|Court|Suite 200|Five Forks|Dickson County|TN|32293|United States|-5|condo| +1897|AAAAAAAAJGHAAAAA||North 12th|Way|Suite M|Mount Pleasant|Live Oak County|||||| +1898|AAAAAAAAKGHAAAAA|98|South |Dr.|Suite 400|Rosewood|Cass County|ND|52205|United States|-6|condo| +1899|AAAAAAAALGHAAAAA|929|Hickory |Boulevard|Suite A|Arcola|Story County|IA|51654|United States|-6|single family| +1900|AAAAAAAAMGHAAAAA||||Suite D|||WY|85709|United States||| +1901|AAAAAAAANGHAAAAA|583|Poplar Lakeview|Court|Suite 320|Riverdale|Marquette County|WI|59391|United States|-6|apartment| +1902|AAAAAAAAOGHAAAAA|955|Valley |Circle|Suite X|Valley View|Bernalillo County|NM|85124|United States|-7|apartment| +1903|AAAAAAAAPGHAAAAA|967|9th |Ln|Suite A|Franklin|Williamson County|TX|79101|United States|-6|apartment| +1904|AAAAAAAAAHHAAAAA|507|Oak |Way|Suite I|Highland|Penobscot County|ME|09454|United States|-5|condo| +1905|AAAAAAAABHHAAAAA|403|Church |Dr.|Suite P|Five Points|Montour County|PA|16098|United States|-5|single family| +1906|AAAAAAAACHHAAAAA|531|Mill |Circle|Suite N|Antioch|Pontotoc County|MS|58605|United States|-6|condo| +1907|AAAAAAAADHHAAAAA|185|Valley Poplar|Lane|Suite S|Walnut Grove|Coffey County|KS|67752|United States|-6|apartment| +1908|AAAAAAAAEHHAAAAA|771|Cedar Main|Ct.|Suite H|Stringtown|Davis County|IA|50162|United States|-6|single family| +1909|AAAAAAAAFHHAAAAA|631|Chestnut 7th|Ave|Suite 330|Empire|Clay County|WV|24145|United States|-5|apartment| +1910|AAAAAAAAGHHAAAAA|599|Washington Franklin|Ct.|Suite 240|Plainview|Weber County|UT|83683|United States|-7|condo| +1911|AAAAAAAAHHHAAAAA|333|Cherry 2nd|ST|Suite 270|Glenwood|Kent County|DE|13511|United States|-5|apartment| +1912|AAAAAAAAIHHAAAAA|867|Church Smith|ST|Suite G|Woodville|Ogemaw County|MI|44289|United States|-5|condo| +1913|AAAAAAAAJHHAAAAA|683|3rd |Ct.|Suite F|Wilson|Faulkner County|AR|76971|United States|-6|condo| +1914|AAAAAAAAKHHAAAAA|112|Green |Parkway|Suite 180|Buena Vista|Montcalm County|MI|45752|United States|-5|single family| +1915|AAAAAAAALHHAAAAA|353|Lincoln 7th|RD|Suite G|Bethel|Gilliam County|OR|95281|United States|-8|apartment| +1916|AAAAAAAAMHHAAAAA|800|2nd |Lane|Suite 490|Oak Ridge|Hooker County|NE|68371|United States|-7|condo| +1917|AAAAAAAANHHAAAAA|885|Park Maple|Pkwy|Suite 270|Pleasant Valley|Perry County|AR|72477|United States|-6|single family| +1918|AAAAAAAAOHHAAAAA|159|Lake |RD|Suite A|Oak Grove|Clay County|NC|28370|United States|-5|single family| +1919|AAAAAAAAPHHAAAAA|95|Birch |Parkway|Suite T|Oak Ridge|Wilson County|KS|68371|United States|-6|condo| +1920|AAAAAAAAAIHAAAAA|929|Main |Parkway|Suite 460|Avon|Currituck County|NC|20069|United States|-5|single family| +1921|AAAAAAAABIHAAAAA|839|Franklin Laurel|Boulevard|Suite 90|Macedonia|Dawson County|TX|71087|United States|-6|condo| +1922|AAAAAAAACIHAAAAA|698|Park Ash|Avenue|Suite D|Stringtown|Atoka County|OK|70162|United States|-6|apartment| +1923|AAAAAAAADIHAAAAA|896|East |Avenue|Suite 40|Clinton|Belknap County|NH|08822|United States|-5|single family| +1924|AAAAAAAAEIHAAAAA|927|Main 7th|Cir.|Suite V|Clifton|Caroline County|MD|28014|United States|-5|apartment| +1925|AAAAAAAAFIHAAAAA|380|North |Blvd|Suite D|Bridgeport|Ozaukee County|WI|55817|United States|-6|single family| +1926|AAAAAAAAGIHAAAAA|661|5th |Avenue|Suite P|Lakewood|Kent County|TX|78877|United States|-6|single family| +1927|AAAAAAAAHIHAAAAA|78|Adams |Drive|Suite R|Maple Grove|Harrison County|IA|58252|United States|-6|condo| +1928|AAAAAAAAIIHAAAAA|168|6th Main|Blvd|Suite A|Deerfield|Cass County|TX|79840|United States|-6|apartment| +1929|AAAAAAAAJIHAAAAA|912|3rd Miller|Ln|Suite M|Tabor|Wake County|NC|28529|United States|-5|apartment| +1930|AAAAAAAAKIHAAAAA|164|View |ST|Suite 210|Georgetown|Gila County|AZ|87057|United States|-7|single family| +1931|AAAAAAAALIHAAAAA|132|Main |Dr.|Suite 450|Oak Hill|Meade County|KY|47838|United States|-5|apartment| +1932|AAAAAAAAMIHAAAAA|298|4th |Way|Suite A|Deerfield|Clark County|IL|69840|United States|-6|apartment| +1933|AAAAAAAANIHAAAAA|750|Fourth |Blvd|Suite D|Springfield|Sheboygan County|WI|59303|United States|-6|condo| +1934|AAAAAAAAOIHAAAAA|363|6th |RD|Suite 490|Lakeview|DeBaca County|NM|88579|United States|-7|condo| +1935|AAAAAAAAPIHAAAAA|353|Hickory First|Circle|Suite 410|Woodlawn|Chilton County|AL|34098|United States|-6|single family| +1936|AAAAAAAAAJHAAAAA|229|Highland |Pkwy|Suite 0|Lakeside|Gray County|KS|69532|United States|-6|apartment| +1937|AAAAAAAABJHAAAAA|655|Jackson |Dr.|Suite D|Forest Hills|Sequoyah County|OK|79237|United States|-6|single family| +1938|AAAAAAAACJHAAAAA|640|8th Second|Street|Suite R|Antioch|Zapata County|TX|78605|United States|-6|apartment| +1939|AAAAAAAADJHAAAAA|978|1st |Cir.|Suite 400|Stringtown|Franklin County|OH|40162|United States|-5|single family| +1940|AAAAAAAAEJHAAAAA|481|Chestnut Adams|RD|Suite X|Newport|DeBaca County|NM|81521|United States|-7|apartment| +1941|AAAAAAAAFJHAAAAA|847|Railroad Mill|Dr.|Suite S|Scotland|Charlottesville city|VA|26454|United States|-5|condo| +1942|AAAAAAAAGJHAAAAA|331|Broadway |Street|Suite 440|Fairfield|Middlesex County|NJ|06792|United States|-5|apartment| +1943|AAAAAAAAHJHAAAAA|381|Birch Dogwood|Dr.|Suite 260|Mount Zion|Howard County|NE|68054|United States|-7|condo| +1944|AAAAAAAAIJHAAAAA|795|South Church|Ave|Suite L|Red Hill|Mercer County|PA|14338|United States|-5|apartment| +1945|AAAAAAAAJJHAAAAA|754|9th 1st|Way|Suite M|Hopewell|Alleghany County|VA|20587|United States|-5|apartment| +1946|AAAAAAAAKJHAAAAA|776|Madison Davis|Drive|Suite 180|Hillcrest|Stafford County|VA|23003|United States|-5|apartment| +1947|AAAAAAAALJHAAAAA|61|Smith |Ave|Suite A|Nichols|Antrim County|MI|47940|United States|-5|single family| +1948|AAAAAAAAMJHAAAAA|903|Willow North|RD|Suite A|Lakeside|New Haven County|CT|09532|United States|-5|condo| +1949|AAAAAAAANJHAAAAA|484|7th Wilson|Avenue|Suite 450|Green Acres|Ravalli County|MT|67683|United States|-7|apartment| +1950|AAAAAAAAOJHAAAAA|231|Broadway |Court|Suite 370|Highland Park|Beltrami County|MN|56534|United States|-6|apartment| +1951|AAAAAAAAPJHAAAAA|455|Forest |Circle|Suite V|Arlington|Gladwin County|MI|46557|United States|-5|condo| +1952|AAAAAAAAAKHAAAAA|204|Smith |Pkwy|Suite M|Bethel|Taliaferro County|GA|35281|United States|-5|apartment| +1953|AAAAAAAABKHAAAAA|181|12th |Court|Suite E|Springfield|Webster County|MS|59303|United States|-6|apartment| +1954|AAAAAAAACKHAAAAA|788|Washington 11th|Parkway|Suite 380|Woodcrest|Apache County|AZ|84919|United States|-7|single family| +1955|AAAAAAAADKHAAAAA|994|Second Central|Ave|Suite D|Glendale|Kiowa County|CO|83951|United States|-7|apartment| +1956|AAAAAAAAEKHAAAAA|941|Fifth |Road|Suite P|Summit|Kershaw County|SC|20499|United States|-5|single family| +1957|AAAAAAAAFKHAAAAA|454|Mill Poplar|Ave|Suite 10|Brookwood|Marshall County|IA|50965|United States|-6|apartment| +1958|AAAAAAAAGKHAAAAA|525|Spruce |Way|Suite A|Greenwood|Ross County|OH|48828|United States|-5|apartment| +1959|AAAAAAAAHKHAAAAA|817|Washington |Pkwy|Suite J|Oak Hill|Christian County|MO|67838|United States|-6|apartment| +1960|AAAAAAAAIKHAAAAA|897|3rd Birch|Street|Suite A|Pleasant Valley|Knox County|IL|62477|United States|-6|apartment| +1961|AAAAAAAAJKHAAAAA|749|Woodland Railroad|ST|Suite 450|Forest Hills|Mora County|NM|89237|United States|-7|apartment| +1962|AAAAAAAAKKHAAAAA|308|Walnut 2nd|Pkwy|Suite B|Red Hill|Prince Edward County|VA|24338|United States|-5|apartment| +1963|AAAAAAAALKHAAAAA|767|11th Maple|RD|Suite D|Buena Vista|Macon County|AL|35752|United States|-6|condo| +1964|AAAAAAAAMKHAAAAA|745|Washington Hillcrest|Pkwy|Suite 150|Bethel|Mecklenburg County|VA|25281|United States|-5|apartment| +1965|AAAAAAAANKHAAAAA|||Road|Suite M|Five Forks|||52293||-7|| +1966|AAAAAAAAOKHAAAAA|190|First Fifth|Blvd|Suite 310|Centerville|Steele County|MN|50059|United States|-6|apartment| +1967|AAAAAAAAPKHAAAAA|534|Park Spruce|Cir.|Suite R|Waterloo|Jefferson County|GA|31675|United States|-5|apartment| +1968|AAAAAAAAALHAAAAA|237|Park |Dr.|Suite N|White Oak|Caldwell County|KY|46668|United States|-6|condo| +1969|AAAAAAAABLHAAAAA|746|Meadow |Avenue|Suite 180|Bath|Chelan County|WA|90573|United States|-8|condo| +1970|AAAAAAAACLHAAAAA|500|Mill |ST|Suite 300|Forest Hills|Montgomery County|AR|79237|United States|-6|single family| +1971|AAAAAAAADLHAAAAA|453|Pine |Street|||Gilmer County||21757|United States||condo| +1972|AAAAAAAAELHAAAAA|790|Ridge Jefferson|Circle|Suite K|Waterloo|San Benito County|CA|91675|United States|-8|single family| +1973|AAAAAAAAFLHAAAAA|3|Dogwood |Ct.|Suite P|Antioch|Sheboygan County|WI|58605|United States|-6|apartment| +1974|AAAAAAAAGLHAAAAA|139|Main Laurel|Road|Suite 250|Pleasant Hill|Hughes County|OK|73604|United States|-6|condo| +1975|AAAAAAAAHLHAAAAA|942|3rd Mill|Cir.|Suite E|Buena Vista|Ben Hill County|GA|35752|United States|-5|single family| +1976|AAAAAAAAILHAAAAA|952|Sycamore |Court|Suite 180|Glenwood|Herkimer County|NY|13511|United States|-5|apartment| +1977|AAAAAAAAJLHAAAAA|229|Green 5th|Avenue|Suite 180|Woodland|Chippewa County|MN|54854|United States|-6|condo| +1978|AAAAAAAAKLHAAAAA|452|View |Street|Suite 390|Buena Vista|Laurel County|KY|45752|United States|-5|apartment| +1979|AAAAAAAALLHAAAAA|289|Lakeview |Wy|Suite 30|Bennett|Evangeline Parish|LA|71715|United States|-6|apartment| +1980|AAAAAAAAMLHAAAAA|903|Elm |Ln|Suite 70|Glendale|Colfax County|NE|63951|United States|-6|condo| +1981|AAAAAAAANLHAAAAA|753|Spring Spring|Blvd|Suite A|Plainview|Rolette County|ND|53683|United States|-6|apartment| +1982|AAAAAAAAOLHAAAAA|251|Green |Circle|Suite M|Deerfield|Webster County|NE|69840|United States|-7|condo| +1983|AAAAAAAAPLHAAAAA|884|Lincoln |Road|Suite Y|Edgewood|New Madrid County|MO|60069|United States|-6|single family| +1984|AAAAAAAAAMHAAAAA|628|5th |Ave|Suite J|New Hope|Iredell County|NC|29431|United States|-5|single family| +1985|AAAAAAAABMHAAAAA|265|Johnson |Ln|Suite 380|Maple Grove|Rawlins County|KS|68252|United States|-6|condo| +1986|AAAAAAAACMHAAAAA|177|15th |Pkwy|Suite J|New Hope|Edmunds County|SD|59431|United States|-6|apartment| +1987|AAAAAAAADMHAAAAA||Franklin |Ct.||||TX|74466|United States|-6|single family| +1988|AAAAAAAAEMHAAAAA|182|North Willow|Blvd|Suite 210|Pleasant Hill|Lebanon County|PA|13604|United States|-5|condo| +1989|AAAAAAAAFMHAAAAA|270|Fourth Jefferson|Blvd|Suite 200|Lincoln|Johnson County|TN|31289|United States|-6|apartment| +1990|AAAAAAAAGMHAAAAA|524|Cedar Davis|Cir.|Suite 230|Bridgeport|Polk County|TN|35817|United States|-6|single family| +1991|AAAAAAAAHMHAAAAA|87|Forest |Ct.|Suite 450|Springdale|Washington County|PA|18883|United States|-5|single family| +1992|AAAAAAAAIMHAAAAA|190|View Davis|Ave|Suite 330|Sulphur Springs|Breathitt County|KY|48354|United States|-6|single family| +1993|AAAAAAAAJMHAAAAA|527|Hill Jefferson|Avenue|Suite 400|Spring Valley|Power County|ID|86060|United States|-7|single family| +1994|AAAAAAAAKMHAAAAA|321|East |Road|Suite 440|Bear Creek|Putnam County|IN|43075|United States|-5|single family| +1995|AAAAAAAALMHAAAAA|171|East Johnson|Cir.|Suite 170|Harmony|Gasconade County|MO|65804|United States|-6|single family| +1996|AAAAAAAAMMHAAAAA|692|Mill |Dr.|Suite 80|Mount Olive|Sioux County|NE|68059|United States|-7|single family| +1997|AAAAAAAANMHAAAAA|29|Ash |Avenue|Suite 320|Riverside|Douglas County|GA|39231|United States|-5|apartment| +1998|AAAAAAAAOMHAAAAA|497|Cherry North|Lane|Suite O|Clifton|Van Zandt County|TX|78014|United States|-6|apartment| +1999|AAAAAAAAPMHAAAAA|909|Church |Ct.|Suite 340|Hillcrest|Blair County|PA|13003|United States|-5|single family| +2000|AAAAAAAAANHAAAAA|615|10th |Blvd|Suite 150|Oakdale|Glynn County|GA|39584|United States|-5|apartment| +2001|AAAAAAAABNHAAAAA|23|Pine |Lane|Suite 280|Golden|Sheboygan County|WI|50411|United States|-6|condo| +2002|AAAAAAAACNHAAAAA|576|2nd Main|Lane|Suite 70|Friendship|Greene County|OH|44536|United States|-5|single family| +2003|AAAAAAAADNHAAAAA|40|Willow |Pkwy|Suite 150|Greenfield|Montgomery County|AL|35038|United States|-6|apartment| +2004|AAAAAAAAENHAAAAA|835|Eigth |Circle|Suite Q|Five Forks|Price County|WI|52293|United States|-6|apartment| +2005|AAAAAAAAFNHAAAAA|227|Birch |RD|Suite 320|Union Hill|Wallace County|KS|67746|United States|-6|single family| +2006|AAAAAAAAGNHAAAAA|882|Dogwood |Street|Suite L|Oak Ridge|Kerr County|TX|78371|United States|-6|condo| +2007|AAAAAAAAHNHAAAAA|644|Lincoln |Street|Suite H|Spring Grove|Mecklenburg County|NC|26719|United States|-5|apartment| +2008|AAAAAAAAINHAAAAA|195|Highland Locust|Road|Suite 490|Shannon|Lincoln County|OR|94120|United States|-8|single family| +2009|AAAAAAAAJNHAAAAA|706|Maple |Street|Suite M|Georgetown|Wright County|MN|57057|United States|-6|single family| +2010|AAAAAAAAKNHAAAAA|734|Main |Ln|Suite Q|Argyle|Sioux County|IA|58722|United States|-6|condo| +2011|AAAAAAAALNHAAAAA|320|Wilson |Ln|Suite 330|Riverview|Cross County|AR|79003|United States|-6|apartment| +2012|AAAAAAAAMNHAAAAA|742|Spruce West|Ave|Suite 430|Farmington|Hettinger County|ND|59145|United States|-6|single family| +2013|AAAAAAAANNHAAAAA|794|Laurel Franklin|Lane|Suite 360|Pine Grove|Charles Mix County|SD|54593|United States|-6|apartment| +2014|AAAAAAAAONHAAAAA|400|Broadway |Wy|Suite P|Gilbert|Lake County|FL|35183|United States|-5|single family| +2015|AAAAAAAAPNHAAAAA|737|Maple |Drive|Suite 240|Fairview|Charlottesville city|VA|25709|United States|-5|apartment| +2016|AAAAAAAAAOHAAAAA|514|Johnson 1st|Parkway|Suite 20|Providence|Menifee County|KY|46614|United States|-5|apartment| +2017|AAAAAAAABOHAAAAA|387|6th Oak|ST|Suite 200|Arlington|Wythe County|VA|26557|United States|-5|condo| +2018|AAAAAAAACOHAAAAA|509|Franklin |Drive|Suite 300|Pine Grove|Franklin County|IA|54593|United States|-6|condo| +2019|AAAAAAAADOHAAAAA|338|Park 4th|Blvd|Suite O|Wilton|Greenwood County|SC|26997|United States|-5|single family| +2020|AAAAAAAAEOHAAAAA|258|8th Dogwood|Way|Suite E|Wildwood|Logan County|CO|86871|United States|-7|condo| +2021|AAAAAAAAFOHAAAAA|952|Franklin Mill|Pkwy|Suite B|Woodville|Fort Bend County|TX|74289|United States|-6|apartment| +2022|AAAAAAAAGOHAAAAA|351|Cherry 3rd|Parkway|Suite R|Arlington|Midland County|TX|76557|United States|-6|condo| +2023|AAAAAAAAHOHAAAAA|19|Sycamore Adams|Dr.|Suite U|Centerville|Knox County|MO|60059|United States|-6|single family| +2024|AAAAAAAAIOHAAAAA|636|Hickory |Ave|Suite 400|Shady Grove|Lafayette Parish|LA|72812|United States|-6|apartment| +2025|AAAAAAAAJOHAAAAA|5||Way|Suite E|New Hope|Tyler County|WV||||apartment| +2026|AAAAAAAAKOHAAAAA|563|Park East|RD|Suite S|Bridgeport|Manassas city|VA|25817|United States|-5|condo| +2027|AAAAAAAALOHAAAAA|808|Poplar 9th|Ave|Suite 10|Springdale|Frederick County|MD|28883|United States|-5|single family| +2028|AAAAAAAAMOHAAAAA|370|Spring |Cir.|Suite 20|Union Hill|Jackson County|MI|47746|United States|-5|condo| +2029|AAAAAAAANOHAAAAA|959|Railroad Cedar|Circle|Suite 360|Bunker Hill|Greene County|VA|20150|United States|-5|condo| +2030|AAAAAAAAOOHAAAAA|1000|Wilson Woodland|Lane|Suite 370|Sunnyside|Middlesex County|MA|02552|United States|-5|single family| +2031|AAAAAAAAPOHAAAAA|381|Center |Lane|Suite 290|Hopewell|Portage County|OH|40587|United States|-5|single family| +2032|AAAAAAAAAPHAAAAA|30|Hillcrest |Way|Suite 320|Union Hill|Phelps County|NE|67746|United States|-7|single family| +2033|AAAAAAAABPHAAAAA|486|Willow |Ave|Suite V|Lakewood|Carlton County|MN|58877|United States|-6|single family| +2034|AAAAAAAACPHAAAAA|26|13th |Circle|Suite 70|Spring Hill|Hancock County|MS|56787|United States|-6|condo| +2035|AAAAAAAADPHAAAAA|223|Jefferson 12th|Avenue|Suite 330|New Hope|Nantucket County|MA|09431|United States|-5|condo| +2036|AAAAAAAAEPHAAAAA|486|4th Spruce|Dr.|Suite 330|Buena Vista|Isabella County|MI|45752|United States|-5|condo| +2037|AAAAAAAAFPHAAAAA|111|12th |Road|Suite K|Nottingham|Oktibbeha County|MS|54074|United States|-6|single family| +2038|AAAAAAAAGPHAAAAA|451|Main 3rd|Street|Suite 200|Newport|Wirt County|WV|21521|United States|-5|condo| +2039|AAAAAAAAHPHAAAAA|899|||Suite O||Oktibbeha County|||United States|-6|| +2040|AAAAAAAAIPHAAAAA|128|North First|Street|Suite U|Wildwood|Jewell County|KS|66871|United States|-6|apartment| +2041|AAAAAAAAJPHAAAAA|707|West Walnut|Way|Suite 310|Shiloh|Dawson County|TX|79275|United States|-6|condo| +2042|AAAAAAAAKPHAAAAA|713|8th |Pkwy|Suite D|Hardy|Fannin County|GA|35354|United States|-5|condo| +2043|AAAAAAAALPHAAAAA|413|Pine |Drive|Suite 440|Liberty|Whiteside County|IL|63451|United States|-6|single family| +2044|AAAAAAAAMPHAAAAA|873|River |Lane|Suite 60|Oak Hill|Morrow County|OH|47838|United States|-5|apartment| +2045|AAAAAAAANPHAAAAA|374|Oak Main|Wy|Suite F|Stringtown|Isle of Wight County|VA|20162|United States|-5|condo| +2046|AAAAAAAAOPHAAAAA|41|Cedar 3rd|Ave|Suite U|Hillcrest|De Witt County|IL|63003|United States|-6|single family| +2047|AAAAAAAAPPHAAAAA|789|Spring |Blvd|Suite N|Sunnyside|Calaveras County|CA|91952|United States|-8|condo| +2048|AAAAAAAAAAIAAAAA|801|View |RD|Suite 160|Mount Olive|Mayes County|OK|78059|United States|-6|condo| +2049|AAAAAAAABAIAAAAA|927|Washington 10th|Blvd|Suite 220|Clifton|Renville County|MN|58014|United States|-6|single family| +2050|AAAAAAAACAIAAAAA|663|Third Mill|RD|Suite F|Lebanon|Knox County|MO|62898|United States|-6|single family| +2051|AAAAAAAADAIAAAAA|650|Lincoln |Street|Suite U|Brownsville|Park County|MT|69310|United States|-7|single family| +2052|AAAAAAAAEAIAAAAA|592|Walnut |ST|Suite 370|Mountain View|Clay County|AR|74466|United States|-6|single family| +2053|AAAAAAAAFAIAAAAA|133|Dogwood |Ct.|Suite 410|Salem|Livingston County|IL|68048|United States|-6|condo| +2054|AAAAAAAAGAIAAAAA|384|Main 12th|Street|Suite P|Newport|Wayne County|WV|21521|United States|-5|single family| +2055|AAAAAAAAHAIAAAAA|385|5th |Road|Suite 100|Hopewell|Franklin County|FL|30587|United States|-5|condo| +2056|AAAAAAAAIAIAAAAA|5|Williams Maple|Parkway|Suite 400|Liberty|Palo Alto County|IA|53451|United States|-6|single family| +2057|AAAAAAAAJAIAAAAA|643|Sunset 7th|Lane|Suite 260|Waterloo|Venango County|PA|11675|United States|-5|single family| +2058|AAAAAAAAKAIAAAAA|703|Lake |RD|Suite 460|Bunker Hill|Moore County|TN|30150|United States|-6|apartment| +2059|AAAAAAAALAIAAAAA|938|Lee Main|RD|Suite 430|Oakwood|Clay County|KS|60169|United States|-6|single family| +2060|AAAAAAAAMAIAAAAA|620|Smith Forest|Circle|Suite C|Kingston|Harrisonburg city|VA|24975|United States|-5|apartment| +2061|AAAAAAAANAIAAAAA|819|Center Main|ST|Suite K|Edgewood|Smith County|TX|70069|United States|-6|single family| +2062|AAAAAAAAOAIAAAAA|200|13th |Parkway|Suite 420|Woodlawn|Van Buren County|MI|44098|United States|-5|condo| +2063|AAAAAAAAPAIAAAAA|898|South Maple|Avenue|Suite 60|Hillside|Marion County|AR|73094|United States|-6|single family| +2064|AAAAAAAAABIAAAAA|966|Highland |Lane|Suite I|Belmont|Henrico County|VA|20191|United States|-5|condo| +2065|AAAAAAAABBIAAAAA|767|Walnut Hickory|Dr.|Suite 110|Riverside|Barnes County|ND|59231|United States|-6|single family| +2066|AAAAAAAACBIAAAAA|473|15th |Dr.|Suite 210|Spring Hill|Murray County|GA|36787|United States|-5|apartment| +2067|AAAAAAAADBIAAAAA|970|9th Pine|Road|Suite D|Friendship|Woodson County|KS|64536|United States|-6|single family| +2068|AAAAAAAAEBIAAAAA||Cedar ||Suite 440|Oakland|Sac County||59843|||apartment| +2069|AAAAAAAAFBIAAAAA|932|8th |Street|Suite D|Providence|Chatham County|NC|26614|United States|-5|single family| +2070|AAAAAAAAGBIAAAAA|567|Elm |RD|Suite R|Shady Grove|Russell County|AL|32812|United States|-6|condo| +2071|AAAAAAAAHBIAAAAA|978|Hickory First|Dr.|Suite 390|Maple Hill|Union County|GA|38095|United States|-5|single family| +2072|AAAAAAAAIBIAAAAA|721|Lincoln Park|Blvd|Suite 150|Georgetown|Valley County|MT|67057|United States|-7|apartment| +2073|AAAAAAAAJBIAAAAA|344|South Forest|Way|Suite 90|Maple Grove|Lenawee County|MI|48252|United States|-5|condo| +2074|AAAAAAAAKBIAAAAA|375|2nd Main|Drive|Suite G|Mount Pleasant|Fayette County|GA|31933|United States|-5|single family| +2075|AAAAAAAALBIAAAAA|493|Ridge |Parkway|Suite 140|Leon|Hopkins County|KY|40913|United States|-6|condo| +2076|AAAAAAAAMBIAAAAA|357|Lee |Court|Suite H|Pleasant Hill|Presidio County|TX|73604|United States|-6|apartment| +2077|AAAAAAAANBIAAAAA|426|First 13th|Pkwy|Suite J|Fairfield|Sevier County|AR|76192|United States|-6|apartment| +2078|AAAAAAAAOBIAAAAA|107|North |Drive|Suite V|Bunker Hill|Grant County|WV|20150|United States|-5|condo| +2079|AAAAAAAAPBIAAAAA|618|Jackson |Avenue|Suite 20|Summit|Fayette County|PA|10499|United States|-5|single family| +2080|AAAAAAAAACIAAAAA|502|Ridge |Cir.|Suite N|Buena Vista|Madison County|FL|35752|United States|-5|single family| +2081|AAAAAAAABCIAAAAA|285|Highland Main|Road|Suite 230|Harmony|Warren County|MO|65804|United States|-6|apartment| +2082|AAAAAAAACCIAAAAA|718|Smith |RD|Suite N|Kimball|Lamoille County|VT|04195|United States|-5|condo| +2083|AAAAAAAADCIAAAAA|623|Spruce |Circle|Suite W|Ferguson|Appanoose County|IA|51821|United States|-6|apartment| +2084|AAAAAAAAECIAAAAA|168|Adams 1st|Road|Suite P|Spring Hill|Santa Clara County|CA|96787|United States|-8|single family| +2085|AAAAAAAAFCIAAAAA|463|Chestnut Spring|Ct.|Suite C|Crossroads|Douglas County|IL|60534|United States|-6|apartment| +2086|AAAAAAAAGCIAAAAA|116|Maple |Circle|Suite 130|Stringtown|Mora County|NM|80162|United States|-7|condo| +2087|AAAAAAAAHCIAAAAA|341|Cedar |ST|Suite D|Oak Grove|Pike County|GA|38370|United States|-5|apartment| +2088|AAAAAAAAICIAAAAA|677|Hillcrest |Ave|Suite 80|Plainview|Coos County|OR|93683|United States|-8|apartment| +2089|AAAAAAAAJCIAAAAA|872|Valley |Ln|Suite 480|Buena Vista|Washington County|IN|45752|United States|-5|condo| +2090|AAAAAAAAKCIAAAAA|985|Park |RD|Suite 460|Green Acres|Lafayette County|WI|57683|United States|-6|condo| +2091|AAAAAAAALCIAAAAA|496|Birch |Way|Suite 410|Mountain View|Wasco County|OR|94466|United States|-8|apartment| +2092|AAAAAAAAMCIAAAAA|954|Seventh |Court|Suite 290|Newport|Caldwell County|NC|21521|United States|-5|single family| +2093|AAAAAAAANCIAAAAA|392|Williams |Wy|Suite L|Lone Pine|Butler County|KY|47441|United States|-6|single family| +2094|AAAAAAAAOCIAAAAA|114|Oak |Road|Suite Q|Clinton|Union County|FL|38222|United States|-5|apartment| +2095|AAAAAAAAPCIAAAAA|611|West 15th|Lane|Suite G|Hillcrest|Columbus County|NC|23003|United States|-5|condo| +2096|AAAAAAAAADIAAAAA|926|Central |Ct.|Suite M|Bloomingdale|Edmonson County|KY|41824|United States|-6|apartment| +2097|AAAAAAAABDIAAAAA|941|West 12th|Avenue|Suite 190|Marion|Marlboro County|SC|20399|United States|-5|apartment| +2098|AAAAAAAACDIAAAAA||Birch Williams|Parkway|Suite B||Hamilton County|||United States||| +2099|AAAAAAAADDIAAAAA|644||Road|Suite 160|Dallas|Marshall County||53628|||| +2100|AAAAAAAAEDIAAAAA|738|5th |Pkwy|Suite 460|Edgewood|Gooding County|ID|80069|United States|-7|apartment| +2101|AAAAAAAAFDIAAAAA|811|Spring 4th|Drive|Suite 170|Greenwood|Miner County|SD|58828|United States|-7|condo| +2102|AAAAAAAAGDIAAAAA|800|Maple Hill|Circle|Suite 490|Fairview|Lee County|IL|65709|United States|-6|single family| +2103|AAAAAAAAHDIAAAAA|731|7th Park|Lane|Suite V|Spring Hill|Schenectady County|NY|16787|United States|-5|apartment| +2104|AAAAAAAAIDIAAAAA|129|North |Avenue|Suite H|Hillcrest|Jerauld County|SD|53003|United States|-7|apartment| +2105|AAAAAAAAJDIAAAAA|615|Ash Railroad|Cir.|Suite S|Farmersville|Christian County|KY|49305|United States|-6|condo| +2106|AAAAAAAAKDIAAAAA|855|Willow 3rd|Way|Suite L|Pleasant Valley|Zavala County|TX|72477|United States|-6|condo| +2107|AAAAAAAALDIAAAAA|643|Spring |Ave|Suite 480|Bridgeport|Centre County|PA|15817|United States|-5|condo| +2108|AAAAAAAAMDIAAAAA|549|Jackson 12th|Way|Suite 420|Riverside|Kandiyohi County|MN|59231|United States|-6|condo| +2109|AAAAAAAANDIAAAAA|766|Hill Central|Street|Suite 280|Five Points|Keya Paha County|NE|66098|United States|-7|apartment| +2110|AAAAAAAAODIAAAAA|626|Cedar Hillcrest|Dr.|Suite D|Crossroads|Kent County|RI|01134|United States|-5|condo| +2111|AAAAAAAAPDIAAAAA|161|River Ash|Way|Suite 470|Woodlawn|Monroe County|WI|54098|United States|-6|single family| +2112|AAAAAAAAAEIAAAAA|446|Poplar |Dr.|Suite 110|Lebanon|Knox County|KY|42898|United States|-5|single family| +2113|AAAAAAAABEIAAAAA|165|Chestnut |ST|Suite U|Oak Grove|Calhoun County|MS|58370|United States|-6|single family| +2114|AAAAAAAACEIAAAAA|429|South Fifth|Dr.|Suite V|Lakeside|Mobile County|AL|39532|United States|-6|condo| +2115|AAAAAAAADEIAAAAA|528|Maple Ridge|Way|Suite 330|Pleasant Grove|Gallatin County|KY|44136|United States|-6|apartment| +2116|AAAAAAAAEEIAAAAA|361|Elm |Dr.|Suite 30|Hillcrest|Okmulgee County|OK|73003|United States|-6|single family| +2117|AAAAAAAAFEIAAAAA|158|Central |Drive|Suite I|Midway|Brevard County|FL|31904|United States|-5|single family| +2118|AAAAAAAAGEIAAAAA|461|Park Hickory|Circle|Suite R|Willis|Owen County|KY|46788|United States|-5|condo| +2119|AAAAAAAAHEIAAAAA|31|Forest |Avenue|Suite 200|Five Forks|Brunswick County|NC|22293|United States|-5|apartment| +2120|AAAAAAAAIEIAAAAA|567|Broadway |Dr.|Suite G|Gilmore|Putnam County|WV|25464|United States|-5|single family| +2121|AAAAAAAAJEIAAAAA|86|South Valley|Circle|Suite H|Crossroads|Ozaukee County|WI|50534|United States|-6|condo| +2122|AAAAAAAAKEIAAAAA|271|Poplar Lakeview|Blvd|Suite 460|Marion|Lamoille County|VT|00999|United States|-5|condo| +2123|AAAAAAAALEIAAAAA|22|Lincoln Cedar|Pkwy|Suite C|Bethel|Summers County|WV|25281|United States|-5|apartment| +2124|AAAAAAAAMEIAAAAA||Miller Johnson|Ln|Suite 40|||||||single family| +2125|AAAAAAAANEIAAAAA|269|Dogwood Church|Cir.|Suite 430|Florence|DeSoto County|MS|53394|United States|-6|apartment| +2126|AAAAAAAAOEIAAAAA|98|3rd |Street|Suite 260|Buena Vista|Citrus County|FL|35752|United States|-5|single family| +2127|AAAAAAAAPEIAAAAA|791|Chestnut Main|Ln|Suite 70|Plainview|Monroe County|GA|33683|United States|-5|single family| +2128|AAAAAAAAAFIAAAAA|13|Sunset |Ave|Suite E|Waterloo|Washtenaw County|MI|41675|United States|-5|apartment| +2129|AAAAAAAABFIAAAAA||Forest Highland|Dr.|Suite G||Custer County||55709||-6|single family| +2130|AAAAAAAACFIAAAAA|310|Davis Park|Way|Suite V|Mount Pleasant|Fulton County|NY|11933|United States|-5|condo| +2131|AAAAAAAADFIAAAAA|618|1st Spring|Ct.|Suite 120|Shiloh|Lincoln County|NM|89275|United States|-7|condo| +2132|AAAAAAAAEFIAAAAA|812|4th 7th|RD|Suite X|Woodlawn|Cleburne County|AR|74098|United States|-6|condo| +2133|AAAAAAAAFFIAAAAA|909|Ninth |Street|Suite 250|Highland Park|Douglas County|WI|56534|United States|-6|apartment| +2134|AAAAAAAAGFIAAAAA|414|Mill |Lane|Suite 350|Deerfield|Kit Carson County|CO|89840|United States|-7|apartment| +2135|AAAAAAAAHFIAAAAA|367|Sunset Birch|Dr.|Suite 170|Sulphur Springs|Augusta County|VA|28354|United States|-5|condo| +2136|AAAAAAAAIFIAAAAA|222|First South|Street|Suite 470|Pleasant Valley|Cheyenne County|NE|62477|United States|-6|single family| +2137|AAAAAAAAJFIAAAAA|853|Hill |Dr.|Suite 420|Glenwood|Hood River County|OR|93511|United States|-8|apartment| +2138|AAAAAAAAKFIAAAAA|388|Adams |Blvd|Suite 420|Oakland|Rockcastle County|KY|49843|United States|-5|apartment| +2139|AAAAAAAALFIAAAAA|513|Dogwood |Street|Suite 40|Farmersville|Marion County|SC|29305|United States|-5|apartment| +2140|AAAAAAAAMFIAAAAA|138|Highland |Ave|Suite 350|Wildwood|McCracken County|KY|46871|United States|-5|condo| +2141|AAAAAAAANFIAAAAA|204|10th |Way|Suite K|Spring Valley|Scott County|IN|46060|United States|-5|condo| +2142|AAAAAAAAOFIAAAAA|414|Elm |Road|Suite M|Yorktown|Yellowstone County|MT|60732|United States|-7|condo| +2143|AAAAAAAAPFIAAAAA|724|Park |RD|Suite S|Lebanon|Campbell County|KY|42898|United States|-6|apartment| +2144|AAAAAAAAAGIAAAAA|840|Lakeview View|Avenue|Suite W|Plainview|Pender County|NC|23683|United States|-5|apartment| +2145|AAAAAAAABGIAAAAA|105|Sunset Walnut|Pkwy|Suite 380|Hillcrest|Schenectady County|NY|13003|United States|-5|single family| +2146|AAAAAAAACGIAAAAA|2|North |Ln|Suite V|Unionville|Rio Grande County|CO|81711|United States|-7|condo| +2147|AAAAAAAADGIAAAAA|355|10th 5th|Dr.|Suite 220|Greenville|Jones County|MS|51387|United States|-6|condo| +2148|AAAAAAAAEGIAAAAA|316|1st |Court|Suite 490|Unionville|Washington County|CO|81711|United States|-7|condo| +2149|AAAAAAAAFGIAAAAA|941|4th Poplar|RD|Suite H|Greenville|Stephens County|OK|71387|United States|-6|apartment| +2150|AAAAAAAAGGIAAAAA|148|Woodland Ash|Dr.|Suite B|Hamilton|Allegany County|MD|22808|United States|-5|single family| +2151|AAAAAAAAHGIAAAAA|622|Adams |ST|Suite 320|Woodville|Union County|SD|54289|United States|-7|condo| +2152|AAAAAAAAIGIAAAAA|240|East |Pkwy|Suite 400|Bridgeport|Indian River County|FL|35817|United States|-5|single family| +2153|AAAAAAAAJGIAAAAA|876|12th 6th|Ct.|Suite 450|Maple Grove|Coahoma County|MS|58252|United States|-6|condo| +2154|AAAAAAAAKGIAAAAA|82|Mill Spruce|Wy|Suite B|Providence|Newton County|AR|76614|United States|-6|condo| +2155|AAAAAAAALGIAAAAA|145|Laurel |ST|Suite K|Wilson|Franklin County|NE|66971|United States|-6|condo| +2156|AAAAAAAAMGIAAAAA|937|Fifth |Court|Suite A|Oak Grove|Dawson County|NE|68370|United States|-6|apartment| +2157|AAAAAAAANGIAAAAA|739|Franklin Lake|Parkway|Suite 350|Oakwood|Otsego County|NY|10169|United States|-5|condo| +2158|AAAAAAAAOGIAAAAA|119|Adams College|Street|Suite A|Mount Pleasant|Steele County|MN|51933|United States|-6|condo| +2159|AAAAAAAAPGIAAAAA|104|Chestnut Wilson|Cir.|Suite 360|Forest Hills|Calhoun County|TX|79237|United States|-6|apartment| +2160|AAAAAAAAAHIAAAAA|395|Birch |Blvd|Suite A|Salem|Edmunds County|SD|58048|United States|-6|condo| +2161|AAAAAAAABHIAAAAA|233|Highland |Blvd|Suite 100|Hazelwood|Goodhue County|MN|51206|United States|-6|condo| +2162|AAAAAAAACHIAAAAA|322|West 6th|Pkwy|Suite D|Woodland|Cleveland County|NC|24854|United States|-5|condo| +2163|AAAAAAAADHIAAAAA|401|Mill Adams|Court|Suite 320|Mount Zion|Golden Valley County|MT|68054|United States|-7|single family| +2164|AAAAAAAAEHIAAAAA|123|Jefferson |Lane|Suite 90|Lincoln|Greene County|PA|11289|United States|-5|condo| +2165|AAAAAAAAFHIAAAAA|314|Main |Ln|Suite 400|Bayview|Keith County|NE|69672|United States|-7|apartment| +2166|AAAAAAAAGHIAAAAA|406|Forest |Wy|Suite D|Liberty|Alpine County|CA|93451|United States|-8|single family| +2167|AAAAAAAAHHIAAAAA|775|Second |Parkway|Suite E|Greenfield|Bethel Census Area|AK|95038|United States|-9|apartment| +2168|AAAAAAAAIHIAAAAA|403|Ridge Maple|RD|Suite 90|Forest Hills|Winkler County|TX|79237|United States|-6|condo| +2169|AAAAAAAAJHIAAAAA|585|Madison |Circle|Suite N|Ashland|Union County|IN|44244|United States|-5|single family| +2170|AAAAAAAAKHIAAAAA|758|7th |Boulevard|Suite F|Brownsville|Hopkins County|KY|49310|United States|-6|condo| +2171|AAAAAAAALHIAAAAA|123|Ninth River|Circle|Suite 480|Stringtown|Pacific County|WA|90162|United States|-8|single family| +2172|AAAAAAAAMHIAAAAA|51|Fourth |Circle|Suite 50|Martinsville|Blount County|TN|30419|United States|-5|single family| +2173|AAAAAAAANHIAAAAA|875|Smith 1st|Wy|||Haines Borough|AK|98721||-9|single family| +2174|AAAAAAAAOHIAAAAA|884||||Oak Grove||PA|18370|||condo| +2175|AAAAAAAAPHIAAAAA|36|Davis |RD|Suite 460|Franklin|Manatee County|FL|39101|United States|-5|condo| +2176|AAAAAAAAAIIAAAAA|608|Seventh |Drive|Suite 70|Union|Heard County|GA|38721|United States|-5|condo| +2177|AAAAAAAABIIAAAAA|182|North |Ave|Suite L|Mount Zion|Pickett County|TN|38054|United States|-6|single family| +2178|AAAAAAAACIIAAAAA|47|Oak Center|Wy|Suite M|Jamestown|Blackford County|IN|46867|United States|-5|single family| +2179|AAAAAAAADIIAAAAA|400|Pine South|Drive|Suite G|Springdale|Cache County|UT|88883|United States|-7|apartment| +2180|AAAAAAAAEIIAAAAA|862|13th |Blvd|Suite H|Lincoln|Cabarrus County|NC|21289|United States|-5|condo| +2181|AAAAAAAAFIIAAAAA|819|Washington Cherry|Way|Suite R|Pine Grove|Cass County|MN|54593|United States|-6|apartment| +2182|AAAAAAAAGIIAAAAA|610|Dogwood Meadow|Ave|Suite 390|Woodland|Lafayette County|AR|74854|United States|-6|condo| +2183|AAAAAAAAHIIAAAAA|779|Forest Washington|ST|Suite A|Belmont|Loudon County|TN|30191|United States|-6|single family| +2184|AAAAAAAAIIIAAAAA|232|River |ST|Suite 100|Globe|Greene County|IA|59614|United States|-6|condo| +2185|AAAAAAAAJIIAAAAA|786|Fourteenth Lakeview|Cir.|Suite J|Lakeview|San Mateo County|CA|98579|United States|-8|single family| +2186|AAAAAAAAKIIAAAAA|781|Dogwood |Street|Suite 20|Pleasant Hill|Antrim County|MI|43604|United States|-5|condo| +2187|AAAAAAAALIIAAAAA|216|Dogwood |Lane|Suite 80|Clifton|Lackawanna County|PA|18014|United States|-5|apartment| +2188|AAAAAAAAMIIAAAAA|486|Lake |Blvd|Suite D|Clinton|Columbia County|OR|98222|United States|-8|single family| +2189|AAAAAAAANIIAAAAA|819|7th |Way|Suite C|Freeman|Simpson County|KY|42297|United States|-5|apartment| +2190|AAAAAAAAOIIAAAAA|213|Hill |Pkwy|Suite R|Whitney|Calumet County|WI|58339|United States|-6|single family| +2191|AAAAAAAAPIIAAAAA|245|Sunset |Dr.|Suite H|Waterloo|Columbia County|WA|91675|United States|-8|single family| +2192|AAAAAAAAAJIAAAAA|31|North |Street|Suite 160|Freeport|Butler County|NE|61844|United States|-6|condo| +2193|AAAAAAAABJIAAAAA|359|Center |Blvd|Suite X|Lincoln|Phelps County|NE|61289|United States|-7|single family| +2194|AAAAAAAACJIAAAAA|890|3rd Oak|Way|Suite D|Clifton|Heard County|GA|38014|United States|-5|single family| +2195|AAAAAAAADJIAAAAA|761|Seventh Park|Ct.|Suite C|Pleasant Grove|Jefferson County|AL|34136|United States|-6|single family| +2196|AAAAAAAAEJIAAAAA|924|Walnut North|Ave|Suite S|Mount Vernon|Amherst County|VA|28482|United States|-5|apartment| +2197|AAAAAAAAFJIAAAAA|592|Hillcrest |Avenue|Suite N|Maple Grove|Muscogee County|GA|38252|United States|-5|single family| +2198|AAAAAAAAGJIAAAAA|672|4th |Pkwy|Suite 200|Cedar Grove|Simpson County|MS|50411|United States|-6|apartment| +2199|AAAAAAAAHJIAAAAA|983|Cedar Johnson|Cir.|Suite 360|Edgewood|Jenkins County|GA|30069|United States|-5|condo| +2200|AAAAAAAAIJIAAAAA|973|Park |Drive|Suite D|Walnut Grove|Dickinson County|KS|67752|United States|-6|single family| +2201|AAAAAAAAJJIAAAAA|93|Ridge Railroad|Avenue|Suite 450|Lebanon|Shelby County|AL|32898|United States|-6|single family| +2202|AAAAAAAAKJIAAAAA|271|Park Spring|Dr.|Suite 20|Wilson|Blackford County|IN|46971|United States|-5|apartment| +2203|AAAAAAAALJIAAAAA|116|Spring Third|Cir.|Suite 280|Woodland Park|Boone County|WV|21934|United States|-5|apartment| +2204|AAAAAAAAMJIAAAAA|984|Oak 6th|RD|Suite M|Bethel|Gogebic County|MI|45281|United States|-5|condo| +2205|AAAAAAAANJIAAAAA|404|Railroad 7th|Cir.|Suite 180|Enterprise|Kewaunee County|WI|51757|United States|-6|single family| +2206|AAAAAAAAOJIAAAAA|939|Main |Parkway|Suite 160|Mount Vernon|Pearl River County|MS|58482|United States|-6|apartment| +2207|AAAAAAAAPJIAAAAA|142|3rd |Avenue|Suite 10|Greenfield|Winn Parish|LA|75038|United States|-6|apartment| +2208|AAAAAAAAAKIAAAAA|126|Walnut |Ct.|Suite 410|Jackson|Scott County|KY|49583|United States|-5|condo| +2209|AAAAAAAABKIAAAAA|576|Main Washington|Boulevard|Suite X|Hopewell|Jefferson County|NY|10587|United States|-5|condo| +2210|AAAAAAAACKIAAAAA|661|Oak Chestnut|Parkway|Suite I|Union|Golden Valley County|MT|68721|United States|-7|single family| +2211|AAAAAAAADKIAAAAA|56|2nd |Street|Suite 490|Waterloo|Linn County|IA|51675|United States|-6|apartment| +2212|AAAAAAAAEKIAAAAA|571|Davis 4th|Circle|Suite 320|Ashland|Socorro County|NM|84244|United States|-7|apartment| +2213|AAAAAAAAFKIAAAAA|320|4th Sixth|Parkway|Suite 400|Forest Hills|Bracken County|KY|49237|United States|-6|apartment| +2214|AAAAAAAAGKIAAAAA|600|Center |Ct.|Suite 250|Wildwood|Cloud County|KS|66871|United States|-6|single family| +2215|AAAAAAAAHKIAAAAA|515|12th |Parkway|Suite J|Morgantown|Pickaway County|OH|49193|United States|-5|condo| +2216|AAAAAAAAIKIAAAAA|706|Wilson |Avenue|Suite 490|Riceville|Beckham County|OK|75867|United States|-6|apartment| +2217|AAAAAAAAJKIAAAAA|929|2nd 4th|Dr.|Suite X|Amity|Warren County|IA|50766|United States|-6|condo| +2218|AAAAAAAAKKIAAAAA|355|Sixth |Circle|Suite F|Union Hill|Chickasaw County|MS|57746|United States|-6|single family| +2219|AAAAAAAALKIAAAAA|776|Lakeview Oak|Avenue|Suite 300|Midway|Warren County|IL|61904|United States|-6|single family| +2220|AAAAAAAAMKIAAAAA|461|Highland 4th|Ave|Suite U|Oakdale|Thomas County|GA|39584|United States|-5|apartment| +2221|AAAAAAAANKIAAAAA|977|Highland |Circle|Suite G|Glenwood|Forest County|PA|13511|United States|-5|apartment| +2222|AAAAAAAAOKIAAAAA|917||Street|Suite 350||||||-6|| +2223|AAAAAAAAPKIAAAAA|395|North |Avenue|Suite M|Woodbury|Polk County|GA|34489|United States|-5|condo| +2224|AAAAAAAAALIAAAAA|692|Central |Way|Suite 270|Riverdale|Dougherty County|GA|39391|United States|-5|condo| +2225|AAAAAAAABLIAAAAA|72|Birch |Street|Suite I|Mountain View|Grant County|KS|64466|United States|-6|apartment| +2226|AAAAAAAACLIAAAAA|254|8th |RD|Suite D|Greenville|Loup County|NE|61387|United States|-7|single family| +2227|AAAAAAAADLIAAAAA|761|North Cherry|Wy|Suite 480|Highland|Johnston County|NC|29454|United States|-5|single family| +2228|AAAAAAAAELIAAAAA|203|Hickory |Court|Suite 280|Bunker Hill|Bergen County|NJ|00750|United States|-5|single family| +2229|AAAAAAAAFLIAAAAA|891|2nd Ridge|Ln|Suite O|Bunker Hill|Lawrence County|MS|50150|United States|-6|condo| +2230|AAAAAAAAGLIAAAAA||||||Queen Anne County|MD||||condo| +2231|AAAAAAAAHLIAAAAA|703|West |Boulevard|Suite 340|Liberty|Marion County|IN|43451|United States|-5|single family| +2232|AAAAAAAAILIAAAAA|758|Fourth Third|Circle|Suite U|Mount Olive|Jefferson County|OR|98059|United States|-8|single family| +2233|AAAAAAAAJLIAAAAA|417|Ridge Oak|Blvd|Suite 470|Cedar Grove|Trempealeau County|WI|50411|United States|-6|condo| +2234|AAAAAAAAKLIAAAAA|469|Second |Dr.||||AL|34536|United States||single family| +2235|AAAAAAAALLIAAAAA|732|Poplar |Pkwy|Suite R|Georgetown|Burke County|NC|27057|United States|-5|apartment| +2236|AAAAAAAAMLIAAAAA|597|Hillcrest Elm|Drive||Spring Hill||||||single family| +2237|AAAAAAAANLIAAAAA|889|Lee 3rd|Dr.|Suite W|Oak Grove|Carson County|TX|78370|United States|-6|apartment| +2238|AAAAAAAAOLIAAAAA|374|Seventh |Ct.|Suite 330|Sunnyside|Houston County|TX|71952|United States|-6|apartment| +2239|AAAAAAAAPLIAAAAA|810|View Spruce|Ct.|Suite 140|Shiloh|Marshall County|IN|49275|United States|-5|single family| +2240|AAAAAAAAAMIAAAAA||7th ||||||60499|||apartment| +2241|AAAAAAAABMIAAAAA|199|Jackson 3rd|Road|Suite F|Centerville|Dorchester County|SC|20059|United States|-5|apartment| +2242|AAAAAAAACMIAAAAA|230|1st 4th|Ct.|Suite 240|Unionville|Johnson County|AR|71711|United States|-6|single family| +2243|AAAAAAAADMIAAAAA|294|Jackson |RD|Suite 420|Deerfield|Fillmore County|MN|59840|United States|-6|condo| +2244|AAAAAAAAEMIAAAAA|227|Railroad Tenth|Wy|Suite 330|Woodville|Carroll County|IA|54289|United States|-6|single family| +2245|AAAAAAAAFMIAAAAA|209|5th Hill|Dr.|Suite J|Oakwood|Martin County|FL|30169|United States|-5|single family| +2246|AAAAAAAAGMIAAAAA|791|Jackson |Cir.|Suite O|Farmington|Weld County|CO|89145|United States|-7|apartment| +2247|AAAAAAAAHMIAAAAA|471|Sycamore |Boulevard|Suite 130|Lakewood|Steele County|ND|58877|United States|-6|apartment| +2248|AAAAAAAAIMIAAAAA|383|Sixth |Street|Suite O|Clifton|Manitowoc County|WI|58014|United States|-6|apartment| +2249|AAAAAAAAJMIAAAAA|239|Lee |Pkwy|Suite 220|Florence|Crawford County|GA|33394|United States|-5|apartment| +2250|AAAAAAAAKMIAAAAA|838|Pine |Lane|Suite T|Arcola|Clinch County|GA|31654|United States|-5|single family| +2251|AAAAAAAALMIAAAAA|59|View |Dr.|Suite D|Mount Zion|Wright County|MO|68054|United States|-6|condo| +2252|AAAAAAAAMMIAAAAA|847|Cedar 4th|Circle|Suite 480|Lebanon|Lake County|IN|42898|United States|-5|condo| +2253|AAAAAAAANMIAAAAA|574|7th |Ln|Suite A|New Hope|Montcalm County|MI|49431|United States|-5|condo| +2254|AAAAAAAAOMIAAAAA|329|Ridge 14th|Street|Suite Y|Riverside|King George County|VA|29231|United States|-5|single family| +2255|AAAAAAAAPMIAAAAA|574|Poplar 3rd|Lane|Suite I|Liberty|Belknap County|NH|04051|United States|-5|apartment| +2256|AAAAAAAAANIAAAAA|329|Franklin Park|Wy|Suite 10|Woodville|Hubbard County|MN|54289|United States|-6|apartment| +2257|AAAAAAAABNIAAAAA|865|Chestnut Maple|ST|Suite B|Oakwood|Douglas County|NE|60169|United States|-6|single family| +2258|AAAAAAAACNIAAAAA|672|Central Smith|Drive|Suite 340|Edgewood|Rio Blanco County|CO|80069|United States|-7|condo| +2259|AAAAAAAADNIAAAAA|757|Ash |Ct.|Suite 240|Jamestown|Amherst County|VA|26867|United States|-5|condo| +2260|AAAAAAAAENIAAAAA|888|10th River|Road|Suite 40|Newport|Billings County|ND|51521|United States|-6|condo| +2261|AAAAAAAAFNIAAAAA|505|Oak |Pkwy|Suite M|Valley View|Howell County|MO|65124|United States|-6|condo| +2262|AAAAAAAAGNIAAAAA|694|Highland |Ct.|Suite 210|Plainview|Jefferson County|CO|83683|United States|-7|apartment| +2263|AAAAAAAAHNIAAAAA|605|5th |Street|Suite A|Valley View|Gonzales County|TX|75124|United States|-6|single family| +2264|AAAAAAAAINIAAAAA|739|Lincoln Jackson|Way|Suite 240|Pleasant Valley|Kane County|IL|62477|United States|-6|single family| +2265|AAAAAAAAJNIAAAAA|347|Green |Boulevard|Suite 380|Harmony|Macon County|IL|65804|United States|-6|single family| +2266|AAAAAAAAKNIAAAAA|476|Williams |Boulevard|Suite E|Mountain View|Harney County|OR|94466|United States|-8|single family| +2267|AAAAAAAALNIAAAAA|148|12th |Blvd|Suite D|Forest Hills|Dodge County|GA|39237|United States|-5|single family| +2268|AAAAAAAAMNIAAAAA|483|Main |Blvd|Suite 300|Greenville|La Salle County|IL|61387|United States|-6|condo| +2269|AAAAAAAANNIAAAAA|592|9th Dogwood|Dr.|Suite 200|Edgewood|Sarpy County|NE|60069|United States|-7|single family| +2270|AAAAAAAAONIAAAAA|956|Madison Meadow|Court|Suite 190|Highland|Grant County|AR|79454|United States|-6|condo| +2271|AAAAAAAAPNIAAAAA|730|Fifth |ST|Suite 20|Newport|Jackson County|SD|51521|United States|-7|apartment| +2272|AAAAAAAAAOIAAAAA|11|Elm ||Suite 270||Washington County|OR||||| +2273|AAAAAAAABOIAAAAA|953|East |Way|Suite 120|Lakeside|Marion County|OH|49532|United States|-5|condo| +2274|AAAAAAAACOIAAAAA|864|Center |Road|Suite U|Clifton|Morgan County|AL|38014|United States|-6|apartment| +2275|AAAAAAAADOIAAAAA|22|Pine 7th|Ln|Suite 280|Woodland|Lane County|KS|64854|United States|-6|apartment| +2276|AAAAAAAAEOIAAAAA||Locust |Avenue|Suite 100|Belmont|||||-6|| +2277|AAAAAAAAFOIAAAAA|276|Central Maple|Pkwy|Suite T|Oak Grove|Cass County|TX|78370|United States|-6|apartment| +2278|AAAAAAAAGOIAAAAA|27|Maple 15th|Lane|Suite 330|Highland Park|Mitchell County|IA|56534|United States|-6|condo| +2279|AAAAAAAAHOIAAAAA|184|Pine |Blvd|Suite 470|Concord|Carroll County|IL|64107|United States|-6|apartment| +2280|AAAAAAAAIOIAAAAA|756|1st Lake|Lane|Suite 200|Adrian|Addison County|VT|03901|United States|-5|single family| +2281|AAAAAAAAJOIAAAAA||River Jackson||Suite J||Waupaca County|||||| +2282|AAAAAAAAKOIAAAAA|959|6th Hickory|Way|Suite P|Friendship|Wayne County|MI|44536|United States|-5|single family| +2283|AAAAAAAALOIAAAAA|987|Dogwood Sycamore|Blvd|Suite F|Frogtown|Newport County|RI|09384|United States|-5|single family| +2284|AAAAAAAAMOIAAAAA|389|Ninth |RD|Suite 100|Belmont|Lee County|MS|50191|United States|-6|single family| +2285|AAAAAAAANOIAAAAA|342|Sunset Birch|Dr.|Suite 320|Antioch|Grant County|NE|68605|United States|-6|apartment| +2286|AAAAAAAAOOIAAAAA|401|South |RD|Suite O|Centerville|Grant County|KS|60059|United States|-6|single family| +2287|AAAAAAAAPOIAAAAA|666|Highland |ST|Suite 70|Greenfield|Collier County|FL|35038|United States|-5|single family| +2288|AAAAAAAAAPIAAAAA|883|2nd |Boulevard|Suite 290|Mount Zion|Ascension Parish|LA|78054|United States|-6|condo| +2289|AAAAAAAABPIAAAAA|755|13th |Avenue|Suite 280|Franklin|Adair County|KY|49101|United States|-6|single family| +2290|AAAAAAAACPIAAAAA|109|Lake 11th|Street|Suite 420|Lakeview|Pine County|MN|58579|United States|-6|apartment| +2291|AAAAAAAADPIAAAAA|712|Smith Maple|ST|Suite 80|Westminster|Cooke County|TX|76549|United States|-6|condo| +2292|AAAAAAAAEPIAAAAA|136|Oak |Cir.|Suite F|Salem|Judith Basin County|MT|68048|United States|-7|single family| +2293|AAAAAAAAFPIAAAAA|51|Meadow Second|Drive|Suite 220|Oak Ridge|Fayette County|GA|38371|United States|-5|single family| +2294|AAAAAAAAGPIAAAAA|538|2nd Jefferson|Circle|Suite 210|Hardy|Little River County|AR|75354|United States|-6|apartment| +2295|AAAAAAAAHPIAAAAA|976|Elm Miller|Dr.|Suite M|Jackson|Terrell County|TX|79583|United States|-6|apartment| +2296|AAAAAAAAIPIAAAAA|837|Fourth Washington|Wy|Suite G|Mount Olive|Kimble County|TX|78059|United States|-6|apartment| +2297|AAAAAAAAJPIAAAAA|783|Washington Sunset|Pkwy|Suite 270|Oak Grove|Lyon County|IA|58370|United States|-6|apartment| +2298|AAAAAAAAKPIAAAAA|607|Main |Way|Suite E|Waterloo|Ionia County|MI|41675|United States|-5|single family| +2299|AAAAAAAALPIAAAAA|24|Cherry Fourth|Circle|Suite B|Colfax|Golden Valley County|ND|52565|United States|-6|apartment| +2300|AAAAAAAAMPIAAAAA|345|Franklin Miller|Wy|Suite 170|Bridgeport|Cape May County|NJ|06417|United States|-5|single family| +2301|AAAAAAAANPIAAAAA|251|Miller South|Wy|Suite 320|Shiloh|Mitchell County|GA|39275|United States|-5|condo| +2302|AAAAAAAAOPIAAAAA|835|West Main|Ave|Suite 310|Frogtown|Ouachita Parish|LA|78784|United States|-6|apartment| +2303|AAAAAAAAPPIAAAAA|121|South |Parkway|Suite 290|Enterprise|Jones County|NC|21757|United States|-5|apartment| +2304|AAAAAAAAAAJAAAAA|618|Lincoln Hillcrest|Way|Suite O|Enterprise|Clark County|SD|51757|United States|-6|apartment| +2305|AAAAAAAABAJAAAAA|450|Lee |Wy|Suite 310|Ashland|Madison County|NY|14244|United States|-5|apartment| +2306|AAAAAAAACAJAAAAA|98|Washington Spring|RD|Suite 410|Jones|Sumner County|KS|62686|United States|-6|condo| +2307|AAAAAAAADAJAAAAA|596|Smith |Ct.|Suite N|Shady Grove|Oconee County|||||| +2308|AAAAAAAAEAJAAAAA|1|East |Lane|Suite 170|Walnut Grove|Grant County|KS|67752|United States|-6|apartment| +2309|AAAAAAAAFAJAAAAA|363|Second Main|Blvd|Suite L|Fairfield|Amador County|CA|96192|United States|-8|single family| +2310|AAAAAAAAGAJAAAAA|868|Sunset Spruce|Avenue|Suite A|Greenwood|Harris County|TX|78828|United States|-6|single family| +2311|AAAAAAAAHAJAAAAA|409|Park Davis|Lane|Suite 20|Springdale|Chippewa County|MI|48883|United States|-5|apartment| +2312|AAAAAAAAIAJAAAAA|453|Spring Lincoln|Ln|Suite 80|Cedar Grove|Newport News city|VA|20411|United States|-5|apartment| +2313|AAAAAAAAJAJAAAAA|637|East |Road|Suite 110|Glenwood|Livingston County|MO|63511|United States|-6|condo| +2314|AAAAAAAAKAJAAAAA|64|Smith 5th|Way|Suite Q|Shiloh|Lake of the Woods County|MN|59275|United States|-6|condo| +2315|AAAAAAAALAJAAAAA|581|Woodland |Dr.|Suite F|Oak Ridge|Golden Valley County|MT|68371|United States|-7|single family| +2316|AAAAAAAAMAJAAAAA|119|Pine Hillcrest|Avenue|Suite 70|Oak Ridge|Nolan County|TX|78371|United States|-6|condo| +2317|AAAAAAAANAJAAAAA|374|Third |RD|Suite T|Shiloh|Richmond County|GA|39275|United States|-5|single family| +2318|AAAAAAAAOAJAAAAA|141|10th Maple|Ct.|Suite 460|Ashland|Carter County|OK|74244|United States|-6|single family| +2319|AAAAAAAAPAJAAAAA|586|College |Lane|Suite 390|Mount Zion|Beaufort County|SC|28054|United States|-5|single family| +2320|AAAAAAAAABJAAAAA|380||Wy|Suite W|||IL|61087|United States||| +2321|AAAAAAAABBJAAAAA|207|8th |Blvd|Suite 380|Johnsonville|Drew County|AR|77745|United States|-6|single family| +2322|AAAAAAAACBJAAAAA|814|Dogwood West|Dr.|Suite X|Antioch|Wilkinson County|GA|38605|United States|-5|apartment| +2323|AAAAAAAADBJAAAAA|658|Main |Way|Suite 50|Farmersville|Mercer County|IL|69305|United States|-6|apartment| +2324|AAAAAAAAEBJAAAAA|868|6th Church|Lane|Suite 30|Newport|Oneida County|NY|11521|United States|-5|apartment| +2325|AAAAAAAAFBJAAAAA|811|Fourth |Court|Suite 130|Shady Grove|Berrien County|GA|32812|United States|-5|condo| +2326|AAAAAAAAGBJAAAAA|240|4th 1st|Pkwy|Suite C|Union Hill|Natchitoches Parish|LA|77746|United States|-6|condo| +2327|AAAAAAAAHBJAAAAA|342|Washington Valley|Parkway|Suite 460|Springfield|Bottineau County|ND|59303|United States|-6|apartment| +2328|AAAAAAAAIBJAAAAA|521|Railroad West|Ln|Suite T|Macedonia|Suffolk city|VA|21087|United States|-5|condo| +2329|AAAAAAAAJBJAAAAA|850|College Elm|Road|Suite O|Pleasant Valley|Washington County|ID|82477|United States|-7|single family| +2330|AAAAAAAAKBJAAAAA|328|3rd 3rd|Ave|Suite 250|Springfield|Moore County|NC|29303|United States|-5|single family| +2331|AAAAAAAALBJAAAAA|841|Eigth |Road|Suite 120|Frogtown|Wright County|MN|58784|United States|-6|condo| +2332|AAAAAAAAMBJAAAAA|623|Third Jefferson|Dr.|Suite X|Midway|Carroll County|IA|51904|United States|-6|condo| +2333|AAAAAAAANBJAAAAA|220|Dogwood View|RD|Suite P|Union Hill|Harrison County|WV|27746|United States|-5|condo| +2334|AAAAAAAAOBJAAAAA|476|Ninth |Wy|Suite 110|Pleasant Grove|Pawnee County|KS|64136|United States|-6|apartment| +2335|AAAAAAAAPBJAAAAA|807|Park |Wy|Suite S|Harvey|Colusa County|CA|95858|United States|-8|single family| +2336|AAAAAAAAACJAAAAA|211|Green Mill|Ln|Suite K|Forest Hills|Jones County|IA|59237|United States|-6|apartment| +2337|AAAAAAAABCJAAAAA|968|Central |Circle|Suite C|Plainview|Holmes County|FL|33683|United States|-5|apartment| +2338|AAAAAAAACCJAAAAA|739|Franklin Miller|RD|Suite 230|Walnut Grove|Elliott County|KY|47752|United States|-6|condo| +2339|AAAAAAAADCJAAAAA|170|Elm |Street|Suite 280|Riverview|Winkler County|TX|79003|United States|-6|single family| +2340|AAAAAAAAECJAAAAA|519|10th |Avenue|Suite T|Concord|Carter County|TN|34107|United States|-5|single family| +2341|AAAAAAAAFCJAAAAA|56|Green Woodland|RD|Suite 230|Springdale|Jackson County|IA|58883|United States|-6|single family| +2342|AAAAAAAAGCJAAAAA|979|Wilson |Street|Suite 120|White Oak|Winston County|MS|56668|United States|-6|apartment| +2343|AAAAAAAAHCJAAAAA|373|Dogwood |Cir.|Suite 170|Springdale|Carson City|NV|88883|United States|-8|condo| +2344|AAAAAAAAICJAAAAA|922|Second |Cir.|Suite 440|Rolling Hills|Stafford County|VA|27272|United States|-5|condo| +2345|AAAAAAAAJCJAAAAA|664|Spring |Boulevard|Suite Y|Clinton|Rusk County|TX|78222|United States|-6|apartment| +2346|AAAAAAAAKCJAAAAA|645|4th |Street|Suite 150|Jamestown|Warren County|IL|66867|United States|-6|condo| +2347|AAAAAAAALCJAAAAA|786|East Madison|RD|Suite L|Johnsonville|Prince William County|VA|27745|United States|-5|apartment| +2348|AAAAAAAAMCJAAAAA|868|Maple Ridge|Boulevard|Suite B|Lone Pine|Decatur County|GA|37441|United States|-5|condo| +2349|AAAAAAAANCJAAAAA|440|Smith |Lane|Suite N|Lakewood|Custer County|CO|88877|United States|-7|condo| +2350|AAAAAAAAOCJAAAAA|211|Maple |Parkway|Suite A|Greenwood|Pope County|MN|58828|United States|-6|single family| +2351|AAAAAAAAPCJAAAAA|561|Main First|Lane|Suite 80|Hubbard|Broome County|NY|16291|United States|-5|single family| +2352|AAAAAAAAADJAAAAA|998|North First|Road|Suite F|Providence|Huntingdon County|PA|16614|United States|-5|single family| +2353|AAAAAAAABDJAAAAA|747|Wilson |Ct.|Suite N|Marion|Jackson County|KS|60399|United States|-6|apartment| +2354|AAAAAAAACDJAAAAA|333|Park |Avenue|Suite J|Lebanon|Virginia Beach city|VA|22898|United States|-5|single family| +2355|AAAAAAAADDJAAAAA|149|Highland Chestnut|Cir.|Suite J|Mount Zion|Warren County|NC|28054|United States|-5|apartment| +2356|AAAAAAAAEDJAAAAA|142|8th ||Suite B|Riverdale|Hickman County|TN|39391||-5|condo| +2357|AAAAAAAAFDJAAAAA|292|15th Pine|Lane|Suite C|Belmont|Dubois County|IN|40191|United States|-5|single family| +2358|AAAAAAAAGDJAAAAA|550|12th |Street|Suite D|Pleasant Grove|Parker County|TX|74136|United States|-6|apartment| +2359|AAAAAAAAHDJAAAAA|489|Mill |Street|Suite 120|Jamestown|Seneca County|OH|46867|United States|-5|apartment| +2360|AAAAAAAAIDJAAAAA|889|4th Park|Street|Suite V|Forestville|Hawaii County|HI|93027|United States|-10|apartment| +2361|AAAAAAAAJDJAAAAA|41|Elm |Ct.|Suite I|Woodland|Clay County|SD|54854|United States|-6|apartment| +2362|AAAAAAAAKDJAAAAA|714|River 9th|Ave|Suite E|Clifton|Middlesex County|VA|28014|United States|-5|condo| +2363|AAAAAAAALDJAAAAA|978|4th Hill|RD|Suite C|Granite|Addison County|VT|06884|United States|-5|apartment| +2364|AAAAAAAAMDJAAAAA|673|View Franklin|Cir.|Suite K|Centerville|Harmon County|OK|70059|United States|-6|apartment| +2365|AAAAAAAANDJAAAAA|447|4th |Ave|Suite V|Forest Hills|Adams County|WA|99237|United States|-8|single family| +2366|AAAAAAAAODJAAAAA|730|Second |Cir.|Suite C|Louisville|Franklin County|KS|64464|United States|-6|single family| +2367|AAAAAAAAPDJAAAAA|633|South Pine|ST|Suite P|Arlington|Decatur County|TN|36557|United States|-5|single family| +2368|AAAAAAAAAEJAAAAA|275|Smith Third|Parkway|Suite 430|Plainview|Potter County|SD|53683|United States|-7|condo| +2369|AAAAAAAABEJAAAAA|21|11th Johnson|Way|Suite 370|Montague|Greeley County|KS|64062|United States|-6|apartment| +2370|AAAAAAAACEJAAAAA|172|Fifth View|Road|Suite 80|Harmony|Duchesne County|UT|85804|United States|-7|single family| +2371|AAAAAAAADEJAAAAA|63|Chestnut |Wy|Suite L|Jamestown|Cooper County|MO|66867|United States|-6|single family| +2372|AAAAAAAAEEJAAAAA|119|Woodland |Lane|Suite R|Shiloh|Lexington city|VA|29275|United States|-5|apartment| +2373|AAAAAAAAFEJAAAAA|447|Eigth |Way|Suite 150|Crossroads|Montezuma County|CO|80534|United States|-7|condo| +2374|AAAAAAAAGEJAAAAA|827|Spruce |ST|Suite 360|Lakeview|Powhatan County|VA|28579|United States|-5|apartment| +2375|AAAAAAAAHEJAAAAA|871|Sycamore Valley|Lane|Suite K|Fairfield|Sumter County|GA|36192|United States|-5|single family| +2376|AAAAAAAAIEJAAAAA|547|Meadow Wilson|Boulevard|Suite 210|Bunker Hill|Hamilton County|IN|40150|United States|-5|apartment| +2377|AAAAAAAAJEJAAAAA|734|Franklin Fifth|Court|Suite 340|Riverside|Collin County|TX|79231|United States|-6|condo| +2378|AAAAAAAAKEJAAAAA|725|4th Birch|Parkway|Suite O|Calhoun|Grayson County|KY|46909|United States|-6|condo| +2379|AAAAAAAALEJAAAAA|19|Franklin |Court|Suite 50|Macedonia|Chippewa County|WI|51087|United States|-6|apartment| +2380|AAAAAAAAMEJAAAAA|177|Oak |Ln|Suite G|Cedar Grove|Lyman County|SD|50411|United States|-7|apartment| +2381|AAAAAAAANEJAAAAA|726|Third |Dr.|Suite F|Sulphur Springs|Logan County|ND|58354|United States|-6|apartment| +2382|AAAAAAAAOEJAAAAA|13|Lake |Lane|Suite 360|Greenwood|Bremer County|IA|58828|United States|-6|apartment| +2383|AAAAAAAAPEJAAAAA|628|11th Second|Street|Suite V|Lee|Fremont County|IA|50408|United States|-6|condo| +2384|AAAAAAAAAFJAAAAA|160|Lincoln |Street|Suite H|Bunker Hill|Doniphan County|KS|60150|United States|-6|condo| +2385|AAAAAAAABFJAAAAA|144|Second Tenth|Court|Suite M|Lakeview|Midland County|TX|78579|United States|-6|apartment| +2386|AAAAAAAACFJAAAAA|394|5th Poplar|Ave|Suite 410|Jerome|Logan County|OH|49920|United States|-5|single family| +2387|AAAAAAAADFJAAAAA|85|3rd Central|RD|Suite W|Wildwood|Appling County|GA|36871|United States|-5|condo| +2388|AAAAAAAAEFJAAAAA|447|Main |Ave|Suite 330|Jackson|Sierra County|CA|99583|United States|-8|apartment| +2389|AAAAAAAAFFJAAAAA|927|Green |Drive|Suite 470|Macedonia|Grayson County|TX|71087|United States|-6|condo| +2390|AAAAAAAAGFJAAAAA|653|Maple |Ave|Suite P|Woodlawn|Johnston County|NC|24098|United States|-5|single family| +2391|AAAAAAAAHFJAAAAA||4th Poplar||Suite 350||San Benito County|CA||United States||single family| +2392|AAAAAAAAIFJAAAAA|842|Dogwood 10th|Ln|Suite C|Red Hill|Frio County|TX|74338|United States|-6|single family| +2393|AAAAAAAAJFJAAAAA|541|6th South|Cir.|Suite T|Waterloo|Polk County|MO|61675|United States|-6|single family| +2394|AAAAAAAAKFJAAAAA|285|Jackson |Street|Suite T|Hopewell|Marquette County|WI|50587|United States|-6|apartment| +2395|AAAAAAAALFJAAAAA|741|Walnut |Ct.|Suite 410|Plainview|Sioux County|NE|63683|United States|-7|condo| +2396|AAAAAAAAMFJAAAAA|381|Sycamore Hickory|Wy|Suite J|Spring Hill|Madison County|TX|76787|United States|-6|apartment| +2397|AAAAAAAANFJAAAAA|81|6th Jackson|Blvd|Suite 10|Wildwood|Monroe County|GA|36871|United States|-5|condo| +2398|AAAAAAAAOFJAAAAA|815|Railroad |Dr.|Suite W|Glenwood|Pittsylvania County|VA|23511|United States|-5|condo| +2399|AAAAAAAAPFJAAAAA|361|12th |Ave|Suite 50|Edgewood|Calhoun County|MS|50069|United States|-6|apartment| +2400|AAAAAAAAAGJAAAAA|68|Elm Lakeview|Way|Suite I|Oakwood|Lamar County|TX|70169|United States|-6|condo| +2401|AAAAAAAABGJAAAAA|879|Lakeview |Circle|Suite N|Bridgeport|Jackson County|MS|55817|United States|-6|condo| +2402|AAAAAAAACGJAAAAA|191|Ash Maple|Boulevard|Suite 240|Plainview|Los Alamos County|NM|83683|United States|-7|single family| +2403|AAAAAAAADGJAAAAA|5|River Park|Way|Suite 470|Cedar Grove|Oconto County|WI|50411|United States|-6|apartment| +2404|AAAAAAAAEGJAAAAA|930|Elm Lake|Street|Suite A|Oak Grove|Hamblen County|TN|38370|United States|-5|single family| +2405|AAAAAAAAFGJAAAAA|213|Fifth |Dr.|Suite 300|Glenwood|Howard County|MO|63511|United States|-6|apartment| +2406|AAAAAAAAGGJAAAAA|423|6th Hill|Wy|Suite T|Highland|Chilton County|AL|39454|United States|-6|apartment| +2407|AAAAAAAAHGJAAAAA|582|Walnut |ST|Suite G|Enterprise|Genesee County|MI|41757|United States|-5|apartment| +2408|AAAAAAAAIGJAAAAA|842|Mill |Ct.|Suite I|Cedar Grove|Jefferson Davis Parish|LA|70411|United States|-6|apartment| +2409|AAAAAAAAJGJAAAAA|331|Lakeview Valley|Boulevard|Suite P|||||||| +2410|AAAAAAAAKGJAAAAA|808|Sixth 2nd|Road|Suite B|Franklin|Erath County|TX|79101|United States|-6|single family| +2411|AAAAAAAALGJAAAAA|992|Birch |||||TX||||condo| +2412|AAAAAAAAMGJAAAAA|563|Second Sunset|Court|Suite 380|Pine Grove|Gibson County|IN|44593|United States|-5|apartment| +2413|AAAAAAAANGJAAAAA|649|15th Oak|ST|Suite 80|Ludlow|Ohio County|KY|45566|United States|-5|single family| +2414|AAAAAAAAOGJAAAAA|858|Laurel Highland|Court|Suite W|Pine Grove|Hockley County|TX|74593|United States|-6|condo| +2415|AAAAAAAAPGJAAAAA|264|6th Mill|Pkwy|Suite 20|Hamilton|Montgomery County|GA|32808|United States|-5|condo| +2416|AAAAAAAAAHJAAAAA|343|Jefferson Franklin|ST|Suite F|Riverside|Mercer County|MO|69231|United States|-6|apartment| +2417|AAAAAAAABHJAAAAA|919|North College|Ave|Suite G|Fairview|Crawford County|IL|65709|United States|-6|single family| +2418|AAAAAAAACHJAAAAA|185|7th |Boulevard|Suite J|Lebanon|Park County|WY|82898|United States|-7|condo| +2419|AAAAAAAADHJAAAAA|675|East 6th|Drive|Suite 380|Valley View|Simpson County|MS|55124|United States|-6|apartment| +2420|AAAAAAAAEHJAAAAA|931|6th |Ave|Suite R|Fairfield|Meigs County|TN|36192|United States|-6|apartment| +2421|AAAAAAAAFHJAAAAA|205|Willow |ST|Suite 140|Spring Valley|Georgetown County|SC|26060|United States|-5|apartment| +2422|AAAAAAAAGHJAAAAA|26|12th |Way|Suite M|Unionville|Wasatch County|UT|81711|United States|-7|apartment| +2423|AAAAAAAAHHJAAAAA|933|Chestnut Lakeview|Boulevard|Suite 140|Enterprise|Humphreys County|MS|51757|United States|-6|single family| +2424|AAAAAAAAIHJAAAAA|18|Hickory |Circle|Suite 430|Pine Grove|Ware County|GA|34593|United States|-5|single family| +2425|AAAAAAAAJHJAAAAA|60|5th |RD|Suite V|Lakeview|Greenbrier County|WV|28579|United States|-5|condo| +2426|AAAAAAAAKHJAAAAA|987|4th |Drive|Suite L|Blue Springs|Napa County|CA|94686|United States|-8|condo| +2427|AAAAAAAALHJAAAAA|577|Wilson Meadow|Blvd|Suite 330|Mount Vernon|Hockley County|TX|78482|United States|-6|condo| +2428|AAAAAAAAMHJAAAAA|523|Laurel Birch|Blvd|Suite I|Bethel|Hocking County|OH|45281|United States|-5|apartment| +2429|AAAAAAAANHJAAAAA|469|Adams |Dr.|Suite 490|Hillcrest|Morgan County|UT|83003|United States|-7|single family| +2430|AAAAAAAAOHJAAAAA|310|View |Avenue|Suite 200|Pine Grove|Green County|WI|54593|United States|-6|condo| +2431|AAAAAAAAPHJAAAAA|807|Sunset Green|RD|Suite 80|Perkins|Stillwater County|MT|61852|United States|-7|condo| +2432|AAAAAAAAAIJAAAAA|687|River |Cir.|Suite 210|Mount Zion|Burnett County|WI|58054|United States|-6|apartment| +2433|AAAAAAAABIJAAAAA|821|Eigth Wilson|ST|Suite H|Greenfield|Blanco County|TX|75038|United States|-6|condo| +2434|AAAAAAAACIJAAAAA|727|Lake 8th|Parkway|Suite 0|Spring Hill|James City County|VA|26787|United States|-5|apartment| +2435|AAAAAAAADIJAAAAA|342|Cedar |Cir.|Suite 160|Maple Grove|Douglas County|KS|68252|United States|-6|single family| +2436|AAAAAAAAEIJAAAAA|927|4th |Wy|Suite H|Fairfield|Crittenden County|KY|46192|United States|-6|condo| +2437|AAAAAAAAFIJAAAAA|||Lane|Suite W|Wildwood||KY||United States|-6|| +2438|AAAAAAAAGIJAAAAA|796|5th |Cir.|Suite Y|Wright|Guilford County|NC|22814|United States|-5|apartment| +2439|AAAAAAAAHIJAAAAA|430|Forest |Circle|Suite V|Woodland|Lafayette County|WI|54854|United States|-6|single family| +2440|AAAAAAAAIIJAAAAA|433|Lake |Parkway|Suite 360|Deer Park|Alexandria city|VA|29829|United States|-5|apartment| +2441|AAAAAAAAJIJAAAAA|256|Hillcrest |Circle|Suite 100|Crossroads|Walker County|GA|30534|United States|-5|single family| +2442|AAAAAAAAKIJAAAAA|313|Broadway 10th|Ct.|Suite J|Green Acres|Cross County|AR|77683|United States|-6|single family| +2443|AAAAAAAALIJAAAAA|484|Wilson Smith|Avenue|Suite B|Five Points|Carson City|NV|86098|United States|-8|condo| +2444|AAAAAAAAMIJAAAAA|96|Fourth Fifth|Dr.|Suite 150|Spring Valley|Laclede County|MO|66060|United States|-6|single family| +2445|AAAAAAAANIJAAAAA|928|7th |Way|Suite 450|Deerfield|Valencia County|NM|89840|United States|-7|single family| +2446|AAAAAAAAOIJAAAAA|439|Cedar Hill|Ave|Suite 220|Valley View|Custer County|ID|85124|United States|-7|condo| +2447|AAAAAAAAPIJAAAAA|162|13th Jackson|Blvd|Suite V|Union|Valencia County|NM|88721|United States|-7|condo| +2448|AAAAAAAAAJJAAAAA|927|Fourth 2nd||||McCracken County||48054|||apartment| +2449|AAAAAAAABJJAAAAA|241|First Seventh|RD|Suite J|Bridgeport|Lawrence County|IN|45817|United States|-5|apartment| +2450|AAAAAAAACJJAAAAA|839|6th Cherry|Ave|Suite 490|Somerville|Fentress County|TN|37783|United States|-5|single family| +2451|AAAAAAAADJJAAAAA|401|Highland |Boulevard|Suite 380|Bunker Hill|Martin County|KY|40150|United States|-5|apartment| +2452|AAAAAAAAEJJAAAAA|449|Railroad |Cir.|Suite 290|Pleasant Hill|Grant County|WI|53604|United States|-6|apartment| +2453|AAAAAAAAFJJAAAAA|474|7th |Lane|Suite W|Newtown|Flagler County|FL|31749|United States|-5|apartment| +2454|AAAAAAAAGJJAAAAA|664|4th Main|Cir.|Suite 30|Farmington|Augusta County|VA|29145|United States|-5|condo| +2455|AAAAAAAAHJJAAAAA|886|West |Ave|Suite G|Spring Hill|Caribou County|ID|86787|United States|-7|single family| +2456|AAAAAAAAIJJAAAAA|41|2nd |Ave|Suite 150|Providence|Lagrange County|IN|46614|United States|-5|single family| +2457|AAAAAAAAJJJAAAAA||Oak Poplar||||Wheeler County|||||apartment| +2458|AAAAAAAAKJJAAAAA|981|Willow |Circle|Suite H|Tanglewood|Grady County|OK|78994|United States|-6|condo| +2459|AAAAAAAALJJAAAAA|561|Lee |Wy|Suite 90|Shiloh|Barren County|KY|49275|United States|-6|condo| +2460|AAAAAAAAMJJAAAAA|243|River |Street|Suite W|Hamilton|Scotland County|MO|62808|United States|-6|apartment| +2461|AAAAAAAANJJAAAAA|458|Pine |Street|Suite 70|Springfield|Montour County|PA|19303|United States|-5|apartment| +2462|AAAAAAAAOJJAAAAA|72|Ridge |Way|Suite O|Greenwood|Woodward County|OK|78828|United States|-6|single family| +2463|AAAAAAAAPJJAAAAA|177|1st |Parkway|Suite 340|Antioch|Washington County|FL|38605|United States|-5|apartment| +2464|AAAAAAAAAKJAAAAA|122|12th 14th|Way|Suite 200|Spring Valley|Rooks County|KS|66060|United States|-6|condo| +2465|AAAAAAAABKJAAAAA|380|West |Boulevard|Suite V|Cedar Grove|Grand County|CO|80411|United States|-7|apartment| +2466|AAAAAAAACKJAAAAA|12|Chestnut |Lane|Suite O|White Oak|Shelby County|AL|36668|United States|-6|apartment| +2467|AAAAAAAADKJAAAAA|342|3rd Hill|Dr.|Suite M|Rockwood|Shelby County|TX|71545|United States|-6|condo| +2468|AAAAAAAAEKJAAAAA|666|Seventh 2nd|Wy|Suite L|Bridgeport|Hoke County|NC|25817|United States|-5|condo| +2469|AAAAAAAAFKJAAAAA|99|Johnson |Ln|Suite 390|Waterloo|Carter County|OK|71675|United States|-6|apartment| +2470|AAAAAAAAGKJAAAAA|400|Johnson Hickory|Boulevard|Suite S|Highland Park|Logan County|OH|46534|United States|-5|single family| +2471|AAAAAAAAHKJAAAAA|43|1st 4th|Way|Suite 190|Pleasant Grove|Camden County|GA|34136|United States|-5|apartment| +2472|AAAAAAAAIKJAAAAA|345|5th |Lane|Suite 90|Jackson|Rhea County|TN|39583|United States|-6|apartment| +2473|AAAAAAAAJKJAAAAA|253|South Meadow|Boulevard|Suite 20|Pine Grove|Wilkes County|GA|34593|United States|-5|condo| +2474|AAAAAAAAKKJAAAAA|458|Park Walnut|ST|Suite 470|Oakland|Shelby County|IN|49843|United States|-5|apartment| +2475|AAAAAAAALKJAAAAA|995|4th Lee|Way|Suite 480|Lakewood|Sheridan County|NE|68877|United States|-7|condo| +2476|AAAAAAAAMKJAAAAA|969|Sycamore Birch|Road|Suite 300|Pomona|Payne County|OK|74153|United States|-6|single family| +2477|AAAAAAAANKJAAAAA|25|Park |Road|Suite 160|Franklin|Wilson County|TX|79101|United States|-6|single family| +2478|AAAAAAAAOKJAAAAA|500|Third |Wy|Suite 400|Lakeside|Chautauqua County|KS|69532|United States|-6|condo| +2479|AAAAAAAAPKJAAAAA|298|Dogwood |Ct.|Suite 80|Rosebud|Seward County|KS|62244|United States|-6|condo| +2480|AAAAAAAAALJAAAAA|626|North |Ln|Suite V|Woodlawn|Lafourche Parish|LA|74098|United States|-6|apartment| +2481|AAAAAAAABLJAAAAA|42|Lincoln |Dr.|Suite 170|Lone Pine|Amador County|CA|97441|United States|-8|single family| +2482|AAAAAAAACLJAAAAA|443|Green |Cir.|Suite 260|Highland Park|Sussex County|NJ|07134|United States|-5|apartment| +2483|AAAAAAAADLJAAAAA|800|Wilson Meadow|Pkwy|Suite 40|Kingston|Mayes County|OK|74975|United States|-6|condo| +2484|AAAAAAAAELJAAAAA|184|South Ninth|Road|Suite 440|Hamilton|Union County|OH|42808|United States|-5|single family| +2485|AAAAAAAAFLJAAAAA|159|Walnut Main|Lane|Suite C|Lakewood|Coos County|NH|09477|United States|-5|apartment| +2486|AAAAAAAAGLJAAAAA|839|Fourth Pine|Boulevard|Suite Q|Buena Vista|Columbiana County|OH|45752|United States|-5|condo| +2487|AAAAAAAAHLJAAAAA|18|Main |Parkway|Suite F|Oakland|Marion County|WV|29843|United States|-5|condo| +2488|AAAAAAAAILJAAAAA|14|Elm Smith|RD|Suite 170|Lakewood|Stanton County|KS|68877|United States|-6|condo| +2489|AAAAAAAAJLJAAAAA|330|5th |Ct.|Suite 470|Oak Grove|Smith County|MS|58370|United States|-6|condo| +2490|AAAAAAAAKLJAAAAA|527|River |Lane|Suite N|White Oak|Matagorda County|TX|76668|United States|-6|apartment| +2491|AAAAAAAALLJAAAAA|349|Lake Mill|Circle|Suite C|Woodville|Gordon County|GA|34289|United States|-5|single family| +2492|AAAAAAAAMLJAAAAA|855|Cedar |Boulevard|Suite 210|Jackson|Winnebago County|IA|59583|United States|-6|apartment| +2493|AAAAAAAANLJAAAAA|35|Walnut Lee|Street|Suite Y|Walnut Grove|Carlton County|MN|57752|United States|-6|single family| +2494|AAAAAAAAOLJAAAAA|948|Wilson Pine|Blvd|Suite J|Clinton|Colonial Heights city|VA|28222|United States|-5|single family| +2495|AAAAAAAAPLJAAAAA|416|Lake 8th|Blvd|Suite D|Riverdale|Strafford County|NH|09991|United States|-5|apartment| +2496|AAAAAAAAAMJAAAAA|511|Second |ST|Suite 60|Clinton|Henry County|VA|28222|United States|-5|single family| +2497|AAAAAAAABMJAAAAA|941|Jackson 6th|Circle|Suite 180|Belmont|Culpeper County|VA|20191|United States|-5|apartment| +2498|AAAAAAAACMJAAAAA|124|Spring Fourth|Avenue|Suite 400|Oak Ridge|Montgomery County|TN|38371|United States|-6|single family| +2499|AAAAAAAADMJAAAAA|264|4th |Ln|Suite 170|Arlington|Allen County|KS|66557|United States|-6|condo| +2500|AAAAAAAAEMJAAAAA|359|First Lee|Ave|Suite 200|Pleasant Valley|Walthall County|MS|52477|United States|-6|condo| +2501|AAAAAAAAFMJAAAAA||Washington ||Suite T|||||||apartment| +2502|AAAAAAAAGMJAAAAA|341|8th Main|Way|Suite 430|Hopewell|Brown County|IN|40587|United States|-5|apartment| +2503|AAAAAAAAHMJAAAAA|336|1st East|Way|Suite I|Mountain View|Lafayette Parish|LA|74466|United States|-6|apartment| +2504|AAAAAAAAIMJAAAAA|569|View Adams|Circle|Suite F|Bethel|Pepin County|WI|55281|United States|-6|single family| +2505|AAAAAAAAJMJAAAAA|751|2nd Hickory|ST|Suite H|Shady Grove|Macon County|IL|62812|United States|-6|single family| +2506|AAAAAAAAKMJAAAAA|391|Elevnth Sixth|Blvd|Suite Y|Jamestown|Jackson County|IL|66867|United States|-6|condo| +2507|AAAAAAAALMJAAAAA|22|3rd |Road|Suite S|Belmont|Kossuth County|IA|50191|United States|-6|apartment| +2508|AAAAAAAAMMJAAAAA|769|Meadow |Drive|Suite O|Carthage|Carter County|OK|71529|United States|-6|apartment| +2509|AAAAAAAANMJAAAAA|126|Park |ST|Suite 30|Sulphur Springs|Grant County|WV|28354|United States|-5|apartment| +2510|AAAAAAAAOMJAAAAA|393|Railroad |Dr.|Suite P|Buckingham|Jefferson County|WA|94092|United States|-8|condo| +2511|AAAAAAAAPMJAAAAA|438|14th Second|Blvd|Suite Y|Highland Park|Thomas County|NE|66534|United States|-7|single family| +2512|AAAAAAAAANJAAAAA|588|Central |Avenue|Suite 50|Harvey|Wilkinson County|MS|55858|United States|-6|apartment| +2513|AAAAAAAABNJAAAAA|200|15th |RD|Suite 70|Oakwood|Bibb County|AL|30169|United States|-6|apartment| +2514|AAAAAAAACNJAAAAA|901|11th Madison|RD|Suite 100|Birmingham|Logan County|KS|63372|United States|-6|condo| +2515|AAAAAAAADNJAAAAA|238|Washington Third|Cir.|Suite C|Liberty|Henry County|IN|43451|United States|-5|condo| +2516|AAAAAAAAENJAAAAA|529|Cherry 5th|Ln|Suite 400|Forest Hills|Blaine County|MT|69237|United States|-7|single family| +2517|AAAAAAAAFNJAAAAA|713|Elm Hill|Circle|Suite 170|New Hope|Allen County|KY|49431|United States|-6|single family| +2518|AAAAAAAAGNJAAAAA|826|5th |Avenue|Suite J|Franklin|Pine County|MN|59101|United States|-6|condo| +2519|AAAAAAAAHNJAAAAA|834|Sixth |ST|Suite 130|Brownsville|Randolph County|AR|79310|United States|-6|condo| +2520|AAAAAAAAINJAAAAA|91|Ash Jefferson||Suite 270||Ben Hill County||30150|United States|-5|single family| +2521|AAAAAAAAJNJAAAAA|470|Central Chestnut|Parkway|Suite 110|Marion|Nolan County|TX|70399|United States|-6|single family| +2522|AAAAAAAAKNJAAAAA|594|Woodland |Circle|Suite R|Spring Valley|Jones County|MS|56060|United States|-6|single family| +2523|AAAAAAAALNJAAAAA|622|2nd Main|Avenue|Suite 210|Fairfield|Polk County|NC|26192|United States|-5|apartment| +2524|AAAAAAAAMNJAAAAA|621|4th Railroad|Cir.|Suite 450|Springdale|Burnet County|TX|78883|United States|-6|single family| +2525|AAAAAAAANNJAAAAA|731|4th |Court|Suite 40|Buena Vista|Freeborn County|MN|55752|United States|-6|condo| +2526|AAAAAAAAONJAAAAA|64|Poplar East|Ct.|Suite 180|Highland Park|Daviess County|IN|46534|United States|-5|apartment| +2527|AAAAAAAAPNJAAAAA|317|Maple Johnson|Ct.|Suite 460|Glenwood|Piscataquis County|ME|04111|United States|-5|apartment| +2528|AAAAAAAAAOJAAAAA|298|Highland Railroad|Avenue|Suite 150|Oakwood|Douglas County|NV|80169|United States|-8|apartment| +2529|AAAAAAAABOJAAAAA|313|Chestnut |Court|Suite A|Ashland|Hopkins County|KY|44244|United States|-6|condo| +2530|AAAAAAAACOJAAAAA|76|South Third|Dr.|Suite K|Shiloh|Wheeler County|TX|79275|United States|-6|apartment| +2531|AAAAAAAADOJAAAAA||Lee |||Mount Olive|Richland County|SC|28059||-5|| +2532|AAAAAAAAEOJAAAAA|806|Oak |Cir.|Suite C|Lakeside|Val Verde County|TX|79532|United States|-6|condo| +2533|AAAAAAAAFOJAAAAA|536|Sixth Hill|Cir.|Suite R|Bridgeport|Grayson County|KY|45817|United States|-6|single family| +2534|AAAAAAAAGOJAAAAA|494|11th 4th|Road|Suite P|White Oak|Cherokee County|KS|66668|United States|-6|apartment| +2535|AAAAAAAAHOJAAAAA|615|Second |Road|Suite 50|Thompsonville|Wayne County|NE|69651|United States|-7|condo| +2536|AAAAAAAAIOJAAAAA|104|9th |Cir.|Suite 240|Riverside|McPherson County|KS|69231|United States|-6|condo| +2537|AAAAAAAAJOJAAAAA|160|10th |Pkwy|Suite C|Walnut Grove|New Haven County|CT|08352|United States|-5|single family| +2538|AAAAAAAAKOJAAAAA|696|14th |RD|Suite 160|Deerfield|Putnam County|IN|49840|United States|-5|single family| +2539|AAAAAAAALOJAAAAA|195|Hill Railroad|Ln|Suite 0|Glenwood|Unicoi County|TN|33511|United States|-6|apartment| +2540|AAAAAAAAMOJAAAAA|33|Lake Willow|Boulevard|Suite 40|Bridgeport|Washington County|NE|65817|United States|-7|condo| +2541|AAAAAAAANOJAAAAA|705|Seventh |Avenue|Suite 0|Oakdale|Columbia County|WA|99584|United States|-8|condo| +2542|AAAAAAAAOOJAAAAA|578|Pine |Pkwy|Suite 430|Ruth|Macon County|GA|30309|United States|-5|condo| +2543|AAAAAAAAPOJAAAAA|492|First 6th|Ct.|Suite 280|Fairfield|Valencia County|NM|86192|United States|-7|apartment| +2544|AAAAAAAAAPJAAAAA|474|River |Wy|Suite 320|Five Points|Mineral County|WV|26098|United States|-5|apartment| +2545|AAAAAAAABPJAAAAA|729|Main 4th|Boulevard|Suite R|Salem|Issaquena County|MS|58048|United States|-6|apartment| +2546|AAAAAAAACPJAAAAA|635|Mill |Boulevard|Suite 470|Lincoln|Vernon County|WI|51289|United States|-6|apartment| +2547|AAAAAAAADPJAAAAA|219|Washington Ridge|Road|Suite A|Mount Vernon|Richland County|MT|68482|United States|-7|single family| +2548|AAAAAAAAEPJAAAAA|534|View Miller|Parkway|Suite 50|Unionville|Massac County|IL|61711|United States|-6|apartment| +2549|AAAAAAAAFPJAAAAA|890|Spring |Pkwy|Suite D|Summit|Sandusky County|OH|40499|United States|-5|apartment| +2550|AAAAAAAAGPJAAAAA|459|Jefferson Hickory|Ave|Suite N|Peru|Madison County|MO|60302|United States|-6|condo| +2551|AAAAAAAAHPJAAAAA|619|Fifth 12th|Cir.|Suite X|Springdale|Cooper County|MO|68883|United States|-6|condo| +2552|AAAAAAAAIPJAAAAA|802|Willow Elm|ST|Suite W|Farmington|Crawford County|KS|69145|United States|-6|single family| +2553|AAAAAAAAJPJAAAAA|212|5th |Cir.|Suite 370|Arlington|Suwannee County|FL|36557|United States|-5|single family| +2554|AAAAAAAAKPJAAAAA|641|Jackson Meadow|Way|Suite 130|Ashland|Brown County|TX|74244|United States|-6|apartment| +2555|AAAAAAAALPJAAAAA|843|Wilson |Lane|Suite K|Lakeview|Madison County|MO|68579|United States|-6|apartment| +2556|AAAAAAAAMPJAAAAA|534|2nd Cedar|Way|Suite 130|Fairfield|Clinton County|KY|46192|United States|-6|condo| +2557|AAAAAAAANPJAAAAA|648|Laurel |Circle|Suite 480|Sheffield|Rogers County|OK|76896|United States|-6|condo| +2558|AAAAAAAAOPJAAAAA|877|West Walnut|Circle|Suite H|Highland Park|Washington County|MS|56534|United States|-6|single family| +2559|AAAAAAAAPPJAAAAA|659|West First|Ln|Suite 410|Riverdale|Jefferson County|NY|19391|United States|-5|single family| +2560|AAAAAAAAAAKAAAAA|805|Main |Cir.|Suite A|Midway|Yazoo County|MS|51904|United States|-6|condo| +2561|AAAAAAAABAKAAAAA|352|Hillcrest Forest|Ln|Suite S|Arlington|Cook County|IL|66557|United States|-6|apartment| +2562|AAAAAAAACAKAAAAA|762|5th |Lane|Suite X|Lakeview|Dare County|NC|28579|United States|-5|single family| +2563|AAAAAAAADAKAAAAA|37|Church |Parkway|Suite 10|Phoenix|Swisher County|TX|72276|United States|-6|single family| +2564|AAAAAAAAEAKAAAAA|599|East |Circle|Suite J|Aberdeen|Nye County|NV|87995|United States|-8|apartment| +2565|AAAAAAAAFAKAAAAA|411|West 5th|Court|Suite 150|Springdale|Clermont County|OH|48883|United States|-5|apartment| +2566|AAAAAAAAGAKAAAAA|970|Sycamore |Cir.|Suite 200|Highland|Nuckolls County|NE|69454|United States|-7|single family| +2567|AAAAAAAAHAKAAAAA|682|Birch Smith|Wy|Suite H|Woodbine|Franklin County|TX|74253|United States|-6|apartment| +2568|AAAAAAAAIAKAAAAA|822|13th |Court|Suite P|Lakeside|Polk County|WI|59532|United States|-6|condo| +2569|AAAAAAAAJAKAAAAA|472|Eigth |Blvd|Suite Y|Thompsonville|Lee County|IL|69651|United States|-6|condo| +2570|AAAAAAAAKAKAAAAA|157|11th |ST|Suite 390|Hamilton|Delaware County|OH|42808|United States|-5|single family| +2571|AAAAAAAALAKAAAAA|10|3rd Eigth|Dr.|Suite D|Marion|Franklin County|VA|20399|United States|-5|condo| +2572|AAAAAAAAMAKAAAAA|999|Spring Hill|Street|Suite 140|Friendship|Aitkin County|MN|54536|United States|-6|single family| +2573|AAAAAAAANAKAAAAA|69|River |Circle|Suite W|Wilson|Lawrence County|TN|36971|United States|-6|condo| +2574|AAAAAAAAOAKAAAAA|867|Green |Way|Suite 120|Summit|Belmont County|OH|40499|United States|-5|single family| +2575|AAAAAAAAPAKAAAAA|75|Jefferson |Dr.|Suite 0|Pleasant Valley|Wirt County|WV|22477|United States|-5|single family| +2576|AAAAAAAAABKAAAAA|15|3rd 2nd|Way|Suite G|Pleasant Grove|Montrose County|CO|84136|United States|-7|condo| +2577|AAAAAAAABBKAAAAA|860|9th 2nd|RD|Suite Y|Fairfield|Jefferson County|WA|96192|United States|-8|single family| +2578|AAAAAAAACBKAAAAA|978|Green 4th|Boulevard|Suite O|Salem|Delta County|MI|48048|United States|-5|apartment| +2579|AAAAAAAADBKAAAAA|680|Spring |Street|Suite J|Maple Grove|Taylor County|TX|78252|United States|-6|apartment| +2580|AAAAAAAAEBKAAAAA||Hillcrest Pine||Suite 470|Fairview|Lampasas County|TX|75709|United States|-6|| +2581|AAAAAAAAFBKAAAAA|607|South Central|ST|Suite L|Edgewood|Scott County|TN|30069|United States|-6|apartment| +2582|AAAAAAAAGBKAAAAA|410|3rd Woodland|Way|Suite R|Bunker Hill|Maricopa County|AZ|80150|United States|-7|apartment| +2583|AAAAAAAAHBKAAAAA|825|11th |RD|Suite 20|Enterprise|Henry County|IN|41757|United States|-5|single family| +2584|AAAAAAAAIBKAAAAA|28|4th |Avenue|Suite 140|Lone Pine|Boyd County|NE|67441|United States|-6|condo| +2585|AAAAAAAAJBKAAAAA|629|Oak |Ct.|Suite O|Greenfield|Newton County|GA|35038|United States|-5|apartment| +2586|AAAAAAAAKBKAAAAA|773|Willow Main|Circle|Suite 0|Fairview|Madison County|ID|85709|United States|-7|single family| +2587|AAAAAAAALBKAAAAA|812|Walnut South|Avenue|Suite N|Cedar Grove|Lake County|CA|90411|United States|-8|apartment| +2588|AAAAAAAAMBKAAAAA|393|Cherry |Drive|Suite N|Fairview|Stark County|ND|55709|United States|-6|single family| +2589|AAAAAAAANBKAAAAA|493|9th |Ln|Suite R|Mount Zion|Elk County|PA|18054|United States|-5|single family| +2590|AAAAAAAAOBKAAAAA|372|Park |Drive|Suite O|Crossroads|Midland County|TX|70534|United States|-6|condo| +2591|AAAAAAAAPBKAAAAA|98|Pine 7th|Circle|Suite 340|Cedar Grove|Washington County|MO|60411|United States|-6|condo| +2592|AAAAAAAAACKAAAAA|273|Sunset Walnut|ST|Suite 350|Centerville|Martin County|KY|40059|United States|-5|condo| +2593|AAAAAAAABCKAAAAA|456|Dogwood South|Court|Suite 150|Hopewell|Ralls County|MO|60587|United States|-6|apartment| +2594|AAAAAAAACCKAAAAA|922|Cedar Lake|Wy|Suite T|Hillcrest|Sumter County|AL|33003|United States|-6|single family| +2595|AAAAAAAADCKAAAAA|203|Jefferson |Boulevard|Suite D|Edgewood|Woodson County|KS|60069|United States|-6|single family| +2596|AAAAAAAAECKAAAAA|406|Woodland |Circle|Suite M|New Hope|Powhatan County|VA|29431|United States|-5|condo| +2597|AAAAAAAAFCKAAAAA|326|Main Oak|Circle|Suite 0|Pleasant Valley|Meeker County|MN|52477|United States|-6|apartment| +2598|AAAAAAAAGCKAAAAA|914|Willow Washington|ST|Suite N|Glenwood|Lavaca County|TX|73511|United States|-6|condo| +2599|AAAAAAAAHCKAAAAA|227|Maple Franklin|Drive|Suite 20|Blue Springs|New York County|NY|14686|United States|-5|apartment| +2600|AAAAAAAAICKAAAAA|847|Sunset Ash|Ct.|Suite 270|Blue Springs|Duchesne County|UT|84686|United States|-7|single family| +2601|AAAAAAAAJCKAAAAA|388|9th Maple|RD|Suite M|Farmington|Middlesex County|NJ|09745|United States|-5|single family| +2602|AAAAAAAAKCKAAAAA|366|Church Wilson|Boulevard|Suite O|Deerfield|Hidalgo County|TX|79840|United States|-6|condo| +2603|AAAAAAAALCKAAAAA|672|Hickory |Avenue|Suite U|Maple Grove|Solano County|CA|98252|United States|-8|condo| +2604|AAAAAAAAMCKAAAAA|847|8th Oak|Circle|Suite D|Red Hill|Cameron County|TX|74338|United States|-6|condo| +2605|AAAAAAAANCKAAAAA|253|Railroad |Drive|Suite B|Greenfield|Madison County|KY|45038|United States|-5|apartment| +2606|AAAAAAAAOCKAAAAA|6|7th |Wy|Suite U|Hopewell|Garvin County|OK|70587|United States|-6|condo| +2607|AAAAAAAAPCKAAAAA|611|Third Elm|Wy|Suite P|Newtown|McDonald County|MO|61749|United States|-6|condo| +2608|AAAAAAAAADKAAAAA|925|5th Laurel|Ct.|Suite A|Hopewell|Benson County|ND|50587|United States|-6|single family| +2609|AAAAAAAABDKAAAAA|315|Ash |Ave|Suite Y|Arlington|Jewell County|KS|66557|United States|-6|condo| +2610|AAAAAAAACDKAAAAA|441|Meadow 10th|RD|Suite 490|Bethel|Noble County|OK|75281|United States|-6|condo| +2611|AAAAAAAADDKAAAAA|414|Ridge |Cir.|Suite R|Lebanon|Ottawa County|KS|62898|United States|-6|condo| +2612|AAAAAAAAEDKAAAAA||Spring ||Suite 40|Hamilton|Chase County||62808|||condo| +2613|AAAAAAAAFDKAAAAA|888|Cherry |Ct.|Suite 150|Providence|Galveston County|TX|76614|United States|-6|single family| +2614|AAAAAAAAGDKAAAAA|87|1st Jackson|Cir.|Suite 390|Newtown|Fayette County|KY|41749|United States|-6|condo| +2615|AAAAAAAAHDKAAAAA|815|Elm Oak|Circle|Suite 370|Greenwood|Greer County|OK|78828|United States|-6|single family| +2616|AAAAAAAAIDKAAAAA|536|Sunset |Ct.|Suite E|Maple Grove|Gilliam County|OR|98252|United States|-8|condo| +2617|AAAAAAAAJDKAAAAA|439|1st |RD|Suite 330|Salem|Clarke County|MS|58048|United States|-6|condo| +2618|AAAAAAAAKDKAAAAA|46|Meadow |ST|Suite 130|Fairview|Gem County|ID|85709|United States|-7|apartment| +2619|AAAAAAAALDKAAAAA|927|Washington |Pkwy|Suite J|Frankfort|Copiah County|MS|59681|United States|-6|condo| +2620|AAAAAAAAMDKAAAAA|101|4th 2nd|Lane|Suite J|Deerfield|Florence County|WI|59840|United States|-6|apartment| +2621|AAAAAAAANDKAAAAA|140|Spruce |Street|Suite A|Glendale|Spink County|SD|53951|United States|-7|condo| +2622|AAAAAAAAODKAAAAA|866|||||Lancaster County|SC|26098|United States||condo| +2623|AAAAAAAAPDKAAAAA|601|4th Fourth|Ave|Suite H|Crossroads|Evangeline Parish|LA|70534|United States|-6|condo| +2624|AAAAAAAAAEKAAAAA|672|Pine |Street|Suite 70|Farmington|Portage County|OH|49145|United States|-5|single family| +2625|AAAAAAAABEKAAAAA|461|Sunset Highland|Street|Suite H|Newtown|Bay County|FL|31749|United States|-5|single family| +2626|AAAAAAAACEKAAAAA|473|Williams |ST|Suite Q|Farmington|Graves County|KY|49145|United States|-6|single family| +2627|AAAAAAAADEKAAAAA|299|5th Church|Pkwy|Suite K|Lincoln|Otero County|NM|81289|United States|-7|single family| +2628|AAAAAAAAEEKAAAAA|386|Railroad |Lane|Suite U|Springdale|Sangamon County|IL|68883|United States|-6|single family| +2629|AAAAAAAAFEKAAAAA|311|Lee |Way|Suite I|Hillcrest|Fannin County|GA|33003|United States|-5|apartment| +2630|AAAAAAAAGEKAAAAA|73|10th 3rd|Blvd|Suite R|Friendship|Highland County|VA|24536|United States|-5|condo| +2631|AAAAAAAAHEKAAAAA|237|Third |Blvd|Suite 240|Centerville|Quay County|NM|80059|United States|-7|single family| +2632|AAAAAAAAIEKAAAAA|314|Center Elm|Ln|Suite P|Lakewood|Lawrence County|IN|48877|United States|-5|condo| +2633|AAAAAAAAJEKAAAAA|525|4th |Lane|Suite O|Riverdale|Gogebic County|MI|49391|United States|-5|apartment| +2634|AAAAAAAAKEKAAAAA|438|2nd Oak|Way|Suite T|Red Hill|Butler County|NE|64338|United States|-6|apartment| +2635|AAAAAAAALEKAAAAA|831|College Park|Ct.|Suite V|Riverside|Lafayette County|MS|59231|United States|-6|apartment| +2636|AAAAAAAAMEKAAAAA|91|Sunset Lincoln|Ln|Suite 390|Unionville|Tift County|GA|31711|United States|-5|apartment| +2637|AAAAAAAANEKAAAAA|916|Elm First|Drive|Suite 90|Jackson|Okmulgee County|OK|79583|United States|-6|single family| +2638|AAAAAAAAOEKAAAAA|266|Hill |Ct.|Suite E|Randolph|Jewell County|KS|62729|United States|-6|apartment| +2639|AAAAAAAAPEKAAAAA|237|Forest |Road|Suite 490|Union|Houston County|MN|58721|United States|-6|condo| +2640|AAAAAAAAAFKAAAAA|261|Sixth 4th|Blvd|Suite 90|Edgewood|Howard County|IA|50069|United States|-6|apartment| +2641|AAAAAAAABFKAAAAA|||Parkway||||TX||United States||apartment| +2642|AAAAAAAACFKAAAAA|698|Park Hickory|Parkway|Suite 350|Salem|DuPage County|IL|68048|United States|-6|single family| +2643|AAAAAAAADFKAAAAA|177|2nd Ash|Road|Suite C|Mount Zion|Bee County|TX|78054|United States|-6|apartment| +2644|AAAAAAAAEFKAAAAA||||Suite 320|Woodville||AL|||-6|| +2645|AAAAAAAAFFKAAAAA|502|1st South||Suite I|Wildwood|Tolland County|CT|07471|United States||apartment| +2646|AAAAAAAAGFKAAAAA|986|West Fifth|Blvd|Suite 140|Oak Ridge|Currituck County|NC|28371|United States|-5|apartment| +2647|AAAAAAAAHFKAAAAA|888|Park Chestnut|Pkwy|Suite 150|Union Hill|Miner County|SD|57746|United States|-7|apartment| +2648|AAAAAAAAIFKAAAAA|78|2nd Second|Lane|Suite Q|Mountain View|Hutchinson County|SD|54466|United States|-7|condo| +2649|AAAAAAAAJFKAAAAA|939|15th Sixth|Parkway|Suite X|Greenville|Warren County|MS|51387|United States|-6|single family| +2650|AAAAAAAAKFKAAAAA|754|Main Park|Ln|Suite P|Glenwood|Lamar County|TX|73511|United States|-6|single family| +2651|AAAAAAAALFKAAAAA|768|3rd |Avenue|Suite D|Andover|Pike County|MS|51639|United States|-6|single family| +2652|AAAAAAAAMFKAAAAA|384|3rd |Way|Suite 30|Forest Hills|Monroe County|AL|39237|United States|-6|condo| +2653|AAAAAAAANFKAAAAA|425|Johnson |Ct.|Suite 170|Greenwood|Clay County|IN|48828|United States|-5|single family| +2654|AAAAAAAAOFKAAAAA|671|Central Meadow|Cir.|Suite 190|Lakewood|Liberty County|FL|38877|United States|-5|apartment| +2655|AAAAAAAAPFKAAAAA|679|Jackson |Dr.|Suite 490|Antioch|Winston County|MS|58605|United States|-6|condo| +2656|AAAAAAAAAGKAAAAA|512|Cherry |Dr.|Suite W|San Jose|Adams County|PA|18003|United States|-5|apartment| +2657|AAAAAAAABGKAAAAA|140|3rd Park|Ct.|Suite 420|||GA||||single family| +2658|AAAAAAAACGKAAAAA||Hickory 3rd|Parkway|Suite M|Macedonia|||31087|United States||single family| +2659|AAAAAAAADGKAAAAA|416|5th |Blvd|Suite T|Providence|Washtenaw County|MI|46614|United States|-5|condo| +2660|AAAAAAAAEGKAAAAA|366|Center 14th|Wy|Suite B|Harmony|Scott County|MS|55804|United States|-6|apartment| +2661|AAAAAAAAFGKAAAAA|253|12th |RD|Suite 290|Jamestown|Mellette County|SD|56867|United States|-7|single family| +2662|AAAAAAAAGGKAAAAA|26|Davis Highland|Boulevard|Suite 140|Bunker Hill|Santa Barbara County|CA|90150|United States|-8|condo| +2663|AAAAAAAAHGKAAAAA|255|6th Willow|Ln|Suite J|Red Hill|White County|AR|74338|United States|-6|apartment| +2664|AAAAAAAAIGKAAAAA|33|6th Smith|RD|Suite V|Lakeview|Roosevelt County|MT|68579|United States|-7|condo| +2665|AAAAAAAAJGKAAAAA|546|13th 12th|Dr.|Suite 170|Riverview|Winnebago County|IL|69003|United States|-6|single family| +2666|AAAAAAAAKGKAAAAA|111|Cherry |Lane|Suite 70|Bethel|Cheshire County|NH|05881|United States|-5|apartment| +2667|AAAAAAAALGKAAAAA|886|5th |RD|Suite M|Brownsville|Hudson County|NJ|09910|United States|-5|single family| +2668|AAAAAAAAMGKAAAAA|63|South |RD|Suite R|Greenfield|Harmon County|OK|75038|United States|-6|single family| +2669|AAAAAAAANGKAAAAA|17|Sunset Third|Blvd|Suite 210|Hamilton|Franklin County|OH|42808|United States|-5|condo| +2670|AAAAAAAAOGKAAAAA|678|Ridge 8th|Parkway|Suite P|Westminster|Aleutians West Census Area|AK|96549|United States|-9|single family| +2671|AAAAAAAAPGKAAAAA|473|Ridge |Cir.|Suite H|Hopewell|Costilla County|CO|80587|United States|-7|condo| +2672|AAAAAAAAAHKAAAAA|666|Smith |Way|Suite O|Woodville|Christian County|MO|64289|United States|-6|apartment| +2673|AAAAAAAABHKAAAAA|536|7th |Ave|Suite 20|Phoenix|Washington County|IL|62276|United States|-6|condo| +2674|AAAAAAAACHKAAAAA|93||||Acme|Charles Mix County||50164|United States|-6|| +2675|AAAAAAAADHKAAAAA|309|Forest Pine|Pkwy|Suite T|Maple Grove|Roosevelt County|MT|68252|United States|-7|condo| +2676|AAAAAAAAEHKAAAAA|299|1st |Road|Suite I|Oakdale|Flagler County|FL|39584|United States|-5|single family| +2677|AAAAAAAAFHKAAAAA|416|North |Way|Suite P|Macedonia|Washington Parish|LA|71087|United States|-6|condo| +2678|AAAAAAAAGHKAAAAA|52|Elm |Street|Suite 190|Glendale|Madison County|NY|13951|United States|-5|apartment| +2679|AAAAAAAAHHKAAAAA|753|River 7th|Wy|Suite 490|Sutton|Bland County|VA|25413|United States|-5|condo| +2680|AAAAAAAAIHKAAAAA|199|Cedar Jackson|Drive|Suite 110|Riverside|Bond County|IL|69231|United States|-6|apartment| +2681|AAAAAAAAJHKAAAAA|452|View |Parkway|Suite 270|Providence|Boone County|WV|26614|United States|-5|apartment| +2682|AAAAAAAAKHKAAAAA|803|Lee 6th|RD|Suite 190|Valley View|Monroe County|WI|55124|United States|-6|condo| +2683|AAAAAAAALHKAAAAA|754|Oak |Cir.|Suite 310|Pleasant Grove|Putnam County|TN|34136|United States|-6|apartment| +2684|AAAAAAAAMHKAAAAA|485|13th |Pkwy|Suite 230|Deerfield|Plymouth County|MA|09840|United States|-5|condo| +2685|AAAAAAAANHKAAAAA|662|Ridge |Way|Suite 170|Lee|Essex County|VT|01008|United States|-5|apartment| +2686|AAAAAAAAOHKAAAAA|854|Pine |RD|Suite 450|Union Hill|Cherokee County|TX|77746|United States|-6|apartment| +2687|AAAAAAAAPHKAAAAA|742|Willow |Ave|Suite 140|Shiloh|Washington Parish|LA|79275|United States|-6|condo| +2688|AAAAAAAAAIKAAAAA|446|Sixth |Wy|Suite 480|Glenwood|Chatham County|NC|23511|United States|-5|condo| +2689|AAAAAAAABIKAAAAA|757|1st |Drive|Suite 320|Arlington|Red Lake County|MN|56557|United States|-6|single family| +2690|AAAAAAAACIKAAAAA|175|Railroad Hill|RD|Suite Y|Lee|El Dorado County|CA|90408|United States|-8|condo| +2691|AAAAAAAADIKAAAAA|934|Willow |Wy|Suite L|Nebo|York County|VA|20728|United States|-5|single family| +2692|AAAAAAAAEIKAAAAA|258|North Main|Street|Suite X|Deerfield|Marengo County|AL|39840|United States|-6|condo| +2693|AAAAAAAAFIKAAAAA|252|Oak |Road|Suite 250|Glendale|Shelby County|IA|53951|United States|-6|apartment| +2694|AAAAAAAAGIKAAAAA||||Suite X|Union||GA||United States||condo| +2695|AAAAAAAAHIKAAAAA|925|Railroad |Parkway|Suite W|Sunnyside|Claiborne County|TN|31952|United States|-5|condo| +2696|AAAAAAAAIIKAAAAA|615|||Suite J|Bay View||TX|76457|||| +2697|AAAAAAAAJIKAAAAA|110|First |Wy|Suite 330|Plainview|Bland County|VA|23683|United States|-5|condo| +2698|AAAAAAAAKIKAAAAA|375|Seventh |Circle|Suite A|Glenwood|Erie County|NY|13511|United States|-5|apartment| +2699|AAAAAAAALIKAAAAA|861|3rd |Dr.|Suite C|Highland|Lake County|FL|39454|United States|-5|single family| +2700|AAAAAAAAMIKAAAAA|3|South Pine|Parkway|Suite I|Edgewood|Garfield County|NE|60069|United States|-6|single family| +2701|AAAAAAAANIKAAAAA|545|East |Court|Suite 390|Liberty|Lake of the Woods County|MN|53451|United States|-6|single family| +2702|AAAAAAAAOIKAAAAA|819|9th Mill|Pkwy|Suite 180|Oakland|Roane County|WV|29843|United States|-5|single family| +2703|AAAAAAAAPIKAAAAA|979|10th Cherry|Road|Suite W|Hamilton|McCurtain County|OK|72808|United States|-6|single family| +2704|AAAAAAAAAJKAAAAA|886|8th |Avenue|Suite 110|Enterprise|Delaware County|IN|41757|United States|-5|condo| +2705|AAAAAAAABJKAAAAA|489|Main |Ct.|Suite K|Newport|Addison County|VT|02121|United States|-5|apartment| +2706|AAAAAAAACJKAAAAA|775|Ridge Franklin|Boulevard|Suite 140|Wildwood|Dawson County|TX|76871|United States|-6|single family| +2707|AAAAAAAADJKAAAAA|721|Cedar |Wy|Suite Y|Farmington|Lamar County|AL|39145|United States|-6|single family| +2708|AAAAAAAAEJKAAAAA|562|Ridge Locust|Avenue|Suite M|Frankfort|Carroll County|MS|59681|United States|-6|single family| +2709|AAAAAAAAFJKAAAAA|441|2nd Jackson|RD|Suite 300|Pleasant Grove|Clinton County|MO|64136|United States|-6|single family| +2710|AAAAAAAAGJKAAAAA|506|South 2nd|Ave|Suite 380|Farmersville|Harrisonburg city|VA|29305|United States|-5|single family| +2711|AAAAAAAAHJKAAAAA|698|Dogwood Dogwood|Dr.|Suite 140|Springdale|Hawaii County|HI|98883|United States|-10|single family| +2712|AAAAAAAAIJKAAAAA|906|Maple Chestnut|Road|Suite I|Walnut Grove|Colfax County|NE|67752|United States|-6|single family| +2713|AAAAAAAAJJKAAAAA|347|Second |Blvd|Suite F|Wildwood|Bradford County|FL|36871|United States|-5|single family| +2714|AAAAAAAAKJKAAAAA|139|Park Second|Parkway|Suite D|New Hope|Rockingham County|NC|29431|United States|-5|condo| +2715|AAAAAAAALJKAAAAA|728|Chestnut Railroad|Boulevard|Suite 180|Summit|Kanawha County|WV|20499|United States|-5|apartment| +2716|AAAAAAAAMJKAAAAA|450|Fourth View|RD|Suite 180|Mount Zion|Claiborne County|MS|58054|United States|-6|apartment| +2717|AAAAAAAANJKAAAAA|629|College |Ct.|Suite T|Spring Valley|Curry County|OR|96060|United States|-8|apartment| +2718|AAAAAAAAOJKAAAAA|437|5th |Court|Suite F|Lakeside|Wyoming County|NY|19532|United States|-5|apartment| +2719|AAAAAAAAPJKAAAAA||Oak Williams|Court|Suite 100||Sedgwick County||81521|United States|-7|condo| +2720|AAAAAAAAAKKAAAAA|562|View 9th|Dr.|Suite Y|Mount Olive|Pulaski County|IL|68059|United States|-6|condo| +2721|AAAAAAAABKKAAAAA|101|River |Wy|Suite 440|Lakeside|Brown County|NE|69532|United States|-6|single family| +2722|AAAAAAAACKKAAAAA|673|Oak Willow|Boulevard|Suite M|Shiloh|Kent County|MI|49275|United States|-5|single family| +2723|AAAAAAAADKKAAAAA|887|Third 5th|Drive|Suite R|Arlington|Denver County|CO|86557|United States|-7|condo| +2724|AAAAAAAAEKKAAAAA|970|7th |Drive|Suite 160|Riverdale|Okmulgee County|OK|79391|United States|-6|apartment| +2725|AAAAAAAAFKKAAAAA|341|14th Ash|RD|Suite 450|Hamilton|Person County|NC|22808|United States|-5|single family| +2726|AAAAAAAAGKKAAAAA|645|Green |Boulevard|Suite P|Wildwood|Fergus County|MT|66871|United States|-7|condo| +2727|AAAAAAAAHKKAAAAA|572|4th |RD|Suite D|Marion|Benewah County|ID|80399|United States|-7|condo| +2728|AAAAAAAAIKKAAAAA|381|7th |Street|Suite W|Cedar Grove|Marion County|AL|30411|United States|-6|apartment| +2729|AAAAAAAAJKKAAAAA|466|1st |Wy|Suite 100|Fairview|Dunn County|ND|55709|United States|-6|apartment| +2730|AAAAAAAAKKKAAAAA|689|Madison |Court|Suite P|Valley View|Fallon County|MT|65124|United States|-7|single family| +2731|AAAAAAAALKKAAAAA|131|Cedar |Drive|Suite 250|Bunker Hill|Morrill County|NE|60150|United States|-7|condo| +2732|AAAAAAAAMKKAAAAA|540|Miller Maple|Circle|Suite A|Green Acres|Moultrie County|IL|67683|United States|-6|single family| +2733|AAAAAAAANKKAAAAA|362|Second |Parkway|Suite I|Belmont|Lafayette County|MS|50191|United States|-6|apartment| +2734|AAAAAAAAOKKAAAAA|426|Spring 13th|Court|Suite 210|Jackson|Saginaw County|MI|49583|United States|-5|single family| +2735|AAAAAAAAPKKAAAAA|323|East |Lane|Suite N|Oakwood|Franklin County|ME|00769|United States|-5|apartment| +2736|AAAAAAAAALKAAAAA|943|Oak |Cir.|Suite Y|Pleasant Grove|Allen County|KS|64136|United States|-6|single family| +2737|AAAAAAAABLKAAAAA||Main |||||DE|18048||-5|apartment| +2738|AAAAAAAACLKAAAAA|377|Lee Park|Avenue|Suite 290|Jackson|Shackelford County|TX|79583|United States|-6|condo| +2739|AAAAAAAADLKAAAAA|684|Lake 6th|Ln|Suite J|Lincoln|Benton County|IN|41289|United States|-5|single family| +2740|AAAAAAAAELKAAAAA|330|River |Ln|Suite 450|Edgewood|Bennett County|SD|50069|United States|-6|apartment| +2741|AAAAAAAAFLKAAAAA|280|Poplar Fifth|Dr.|Suite 420|Bethel|Evangeline Parish|LA|75281|United States|-6|condo| +2742|AAAAAAAAGLKAAAAA|857|Cedar Birch|Parkway|Suite G|Harmony|Clay County|SD|55804|United States|-6|single family| +2743|AAAAAAAAHLKAAAAA|449|Johnson |RD|Suite K|Kingston|Mason County|KY|44975|United States|-5|apartment| +2744|AAAAAAAAILKAAAAA|866|Forest Hill|Street|Suite T|Spring Valley|Johnson County|GA|36060|United States|-5|condo| +2745|AAAAAAAAJLKAAAAA|308|Park |Ave|Suite 180|Antioch|Benton County|TN|38605|United States|-5|condo| +2746|AAAAAAAAKLKAAAAA|263|Oak |Dr.|Suite E|Union Hill|San Bernardino County|CA|97746|United States|-8|single family| +2747|AAAAAAAALLKAAAAA|758|Broadway |Circle|Suite V|Highland Park|Pecos County|TX|76534|United States|-6|apartment| +2748|AAAAAAAAMLKAAAAA|239|Hillcrest |Ln|Suite Y|Sulphur Springs|Rusk County|TX|78354|United States|-6|apartment| +2749|AAAAAAAANLKAAAAA|887|Ash Williams|Wy|Suite N|Friendship|Rutherford County|TN|34536|United States|-6|condo| +2750|AAAAAAAAOLKAAAAA|369|Third Wilson|Blvd|Suite 430|Lucas|Bradford County|FL|34554|United States|-5|condo| +2751|AAAAAAAAPLKAAAAA|564|Fifth 3rd|Ave|Suite 340|Carthage|Stone County|MS|51529|United States|-6|single family| +2752|AAAAAAAAAMKAAAAA|798|Main Hill|Dr.|Suite W|Oak Grove|Wayne County|WV|28370|United States|-5|condo| +2753|AAAAAAAABMKAAAAA|374|Washington Dogwood|Lane|Suite 60|Belmont|Harrisonburg city|VA|20191|United States|-5|condo| +2754|AAAAAAAACMKAAAAA|483|Maple |Drive|Suite M|Woodville|Lycoming County|PA|14289|United States|-5|condo| +2755|AAAAAAAADMKAAAAA|34|Cedar 2nd|Ln|Suite 430|Buena Vista|York County|NE|65752|United States|-6|condo| +2756|AAAAAAAAEMKAAAAA|512||||||||United States||apartment| +2757|AAAAAAAAFMKAAAAA|89|15th Second|Wy|Suite 30|Ashland|Crawford County|IN|44244|United States|-5|apartment| +2758|AAAAAAAAGMKAAAAA|973|1st 1st|Ave|Suite 170|Hopewell|Marion County|WV|20587|United States|-5|apartment| +2759|AAAAAAAAHMKAAAAA|747|4th |Pkwy|Suite 90|Pleasant Hill|Jerauld County|SD|53604|United States|-7|single family| +2760|AAAAAAAAIMKAAAAA|320|North |Street|Suite W|Harmony|Tippah County|MS|55804|United States|-6|apartment| +2761|AAAAAAAAJMKAAAAA|644|Church |Road|Suite Y|Glenwood|Gaston County|NC|23511|United States|-5|apartment| +2762|AAAAAAAAKMKAAAAA|64|Adams |RD|Suite 390|Greenville|Mitchell County|TX|71387|United States|-6|condo| +2763|AAAAAAAALMKAAAAA|23|Fourth |Ln|Suite A|Lincoln|Sullivan County|MO|61289|United States|-6|apartment| +2764|AAAAAAAAMMKAAAAA|87||Blvd||||CO|||-7|apartment| +2765|AAAAAAAANMKAAAAA|710|Ridge |Drive|Suite 250|Mount Pleasant|Yukon-Koyukuk Census Area|AK|91933|United States|-9|single family| +2766|AAAAAAAAOMKAAAAA|240|Second 5th|Pkwy|Suite W|Hillcrest|Chambers County|AL|33003|United States|-6|condo| +2767|AAAAAAAAPMKAAAAA|269|Maple 1st|Boulevard|Suite 300|Lakeview|Louisa County|VA|28579|United States|-5|condo| +2768|AAAAAAAAANKAAAAA|12|8th |Way|Suite H|Enterprise|Walker County|TX|71757|United States|-6|single family| +2769|AAAAAAAABNKAAAAA|6|11th West|Lane|Suite G|Brownsville|Clark County|MO|69310|United States|-6|condo| +2770|AAAAAAAACNKAAAAA|668|River |ST|Suite 10|Red Hill|Tillman County|OK|74338|United States|-6|condo| +2771|AAAAAAAADNKAAAAA|612|2nd Elm|Pkwy|Suite 50|Lakeview|Lanier County|GA|38579|United States|-5|apartment| +2772|AAAAAAAAENKAAAAA|339|Adams Davis|Avenue|Suite X|Greenwood|Oswego County|NY|18828|United States|-5|single family| +2773|AAAAAAAAFNKAAAAA|222|3rd |Boulevard|Suite 280|Fairfield|Wade Hampton Census Area|AK|96192|United States|-9|condo| +2774|AAAAAAAAGNKAAAAA|932|Smith |Court|Suite 390|Sunnyside|Jasper County|IA|51952|United States|-6|condo| +2775|AAAAAAAAHNKAAAAA|432|Spruce |Dr.|Suite D|Sunnyside|Island County|WA|91952|United States|-8|single family| +2776|AAAAAAAAINKAAAAA|917|3rd Main|Boulevard|Suite 190|Liberty|Baxter County|AR|73451|United States|-6|apartment| +2777|AAAAAAAAJNKAAAAA|443|1st 12th|Street|Suite N|Buena Vista|Hughes County|SD|55752|United States|-7|apartment| +2778|AAAAAAAAKNKAAAAA|71|Valley Poplar|Drive|Suite S|Valley View|Mariposa County|CA|95124|United States|-8|single family| +2779|AAAAAAAALNKAAAAA|711|Pine Hillcrest|Street|Suite E|San Jose|Pike County|IL|68003|United States|-6|condo| +2780|AAAAAAAAMNKAAAAA|556|Elm Second|ST|Suite E|Pine Grove|Washakie County|WY|84593|United States|-7|condo| +2781|AAAAAAAANNKAAAAA|834|Park ||||Dona Ana County|||||| +2782|AAAAAAAAONKAAAAA|377|North |Lane|Suite 440|Oak Ridge|Claiborne Parish|LA|78371|United States|-6|condo| +2783|AAAAAAAAPNKAAAAA|334|8th 7th|Parkway|Suite 320|Greenwood|Le Sueur County|MN|58828|United States|-6|single family| +2784|AAAAAAAAAOKAAAAA|249|Elm |RD|Suite R|Georgetown|Crisp County|GA|37057|United States|-5|condo| +2785|AAAAAAAABOKAAAAA|773|Cherry Miller|Street|Suite 470|Glendale|Furnas County|NE|63951|United States|-6|condo| +2786|AAAAAAAACOKAAAAA|807|Forest Lakeview|Road|Suite 110|Wilson|Woodson County|KS|66971|United States|-6|single family| +2787|AAAAAAAADOKAAAAA|114|Second |Boulevard|Suite 50|Summit|Craig County|VA|20499|United States|-5|apartment| +2788|AAAAAAAAEOKAAAAA|921|Spruce |Parkway|Suite 140|Edgewood|Greenlee County|AZ|80069|United States|-7|single family| +2789|AAAAAAAAFOKAAAAA|278|Lakeview |Boulevard|Suite 50|Five Points|Walworth County|WI|56098|United States|-6|condo| +2790|AAAAAAAAGOKAAAAA|505|Ninth |Parkway|Suite G|Springfield|Woods County|OK|79303|United States|-6|condo| +2791|AAAAAAAAHOKAAAAA|990|Lincoln |Cir.|Suite C|Deerfield|Webster County|MO|69840|United States|-6|apartment| +2792|AAAAAAAAIOKAAAAA|655|Birch |Boulevard|Suite C|Hopewell|Henry County|IA|50587|United States|-6|apartment| +2793|AAAAAAAAJOKAAAAA|707|Poplar Lakeview|RD|Suite 190|Wildwood|Sargent County|ND|56871|United States|-6|apartment| +2794|AAAAAAAAKOKAAAAA|39|Broadway Main|Parkway|Suite 10|Page|Mecosta County|MI|40296|United States|-5|apartment| +2795|AAAAAAAALOKAAAAA|238|West |Drive|Suite 270|Mechanicsburg|Covington city|VA|22219|United States|-5|single family| +2796|AAAAAAAAMOKAAAAA|629|Mill Spruce|Blvd|Suite 220|Clifton|Knox County|OH|48014|United States|-5|single family| +2797|AAAAAAAANOKAAAAA|331|Ridge View|RD|Suite 140|Clinton|Mariposa County|CA|98222|United States|-8|single family| +2798|AAAAAAAAOOKAAAAA|322|Jefferson Thirteenth|Boulevard|Suite S|Glenwood|Rush County|IN|43511|United States|-5|condo| +2799|AAAAAAAAPOKAAAAA|979|South 4th|ST|Suite 460|Midway|Iron County|MI|41904|United States|-5|condo| +2800|AAAAAAAAAPKAAAAA|442||RD|Suite 80|||||United States|-5|| +2801|AAAAAAAABPKAAAAA|528|Fifth Center|Ln|Suite U|Five Forks|Tazewell County|VA|22293|United States|-5|single family| +2802|AAAAAAAACPKAAAAA|325|Hillcrest College|Dr.|Suite 150|Allentown|Holt County|NE|61838|United States|-7|condo| +2803|AAAAAAAADPKAAAAA|924|Walnut View|Lane|Suite L|Woodlawn|Lafayette County|FL|34098|United States|-5|single family| +2804|AAAAAAAAEPKAAAAA|305|Jackson |Street|Suite X|Springfield|Thurston County|NE|69303|United States|-7|condo| +2805|AAAAAAAAFPKAAAAA|851|Washington Park|Ln|Suite 160|Green Acres|Chautauqua County|NY|17683|United States|-5|apartment| +2806|AAAAAAAAGPKAAAAA|43|Oak |Road|Suite L|Lakeside|Madison County|GA|39532|United States|-5|apartment| +2807|AAAAAAAAHPKAAAAA|66|9th River|Avenue|Suite U|Springfield|Ada County|ID|89303|United States|-7|condo| +2808|AAAAAAAAIPKAAAAA|43|Elm |Avenue|Suite 70|Union Hill|Mason County|IL|67746|United States|-6|condo| +2809|AAAAAAAAJPKAAAAA|541|Second Highland|Boulevard|Suite D|Ashland|Tipton County|TN|34244|United States|-6|condo| +2810|AAAAAAAAKPKAAAAA|228|Franklin |Court|Suite N|Five Points|Harper County|KS|66098|United States|-6|apartment| +2811|AAAAAAAALPKAAAAA|213|Cherry |Ct.|Suite M|Kingston|Missoula County|MT|64975|United States|-7|condo| +2812|AAAAAAAAMPKAAAAA||Railroad |Way||Oak Grove||||United States||apartment| +2813|AAAAAAAANPKAAAAA|735|11th |Lane|Suite W|Spring Valley|Caroline County|VA|26060|United States|-5|single family| +2814|AAAAAAAAOPKAAAAA|933|Valley |Lane|Suite 410|Providence|Clatsop County|OR|96614|United States|-8|condo| +2815|AAAAAAAAPPKAAAAA|329|River |Pkwy|Suite 190|Highland Park|Jefferson Davis County|MS|56534|United States|-6|condo| +2816|AAAAAAAAAALAAAAA|765|8th Church|Circle|Suite T|Stringtown|Hunt County|TX|70162|United States|-6|apartment| +2817|AAAAAAAABALAAAAA|254|Poplar |Lane|Suite J|Sulphur Springs|Madison County|ID|88354|United States|-7|apartment| +2818|AAAAAAAACALAAAAA|225|Smith North|Avenue|Suite 40|Hamilton|Bradley County|TN|32808|United States|-5|single family| +2819|AAAAAAAADALAAAAA|381|Spring Elm|Drive|Suite E|Forest Hills|Columbiana County|OH|49237|United States|-5|single family| +2820|AAAAAAAAEALAAAAA||Ridge 4th|Parkway||Five Points||||United States|-5|single family| +2821|AAAAAAAAFALAAAAA|220|Highland Main|RD|Suite R|Pleasant Valley|Jasper County|TX|72477|United States|-6|apartment| +2822|AAAAAAAAGALAAAAA|460|Poplar Cedar|Ct.|Suite 460|Five Points|Franklin County|AL|36098|United States|-6|single family| +2823|AAAAAAAAHALAAAAA|60|Sycamore |Circle|Suite 290|Brownsville|Stonewall County|TX|79310|United States|-6|condo| +2824|AAAAAAAAIALAAAAA|786|Wilson |Road|Suite V|Shiloh|Westmoreland County|PA|19275|United States|-5|apartment| +2825|AAAAAAAAJALAAAAA|242|Center |Ct.|Suite X|Friendship|Jim Hogg County|TX|74536|United States|-6|apartment| +2826|AAAAAAAAKALAAAAA|385|Walnut |Street|Suite 330|Unionville|Grayson County|KY|41711|United States|-6|condo| +2827|AAAAAAAALALAAAAA|933|Hill |Drive|Suite 250|Bethel|Pittsburg County|OK|75281|United States|-6|condo| +2828|AAAAAAAAMALAAAAA|593|Church Ash|Dr.|Suite 350|Union|Greensville County|VA|28721|United States|-5|single family| +2829|AAAAAAAANALAAAAA|624|Third |ST|Suite Y|Shady Grove|Strafford County|NH|03412|United States|-5|condo| +2830|AAAAAAAAOALAAAAA|114|Forest |Circle|Suite I|Sutton|Bernalillo County|NM|85413|United States|-7|single family| +2831|AAAAAAAAPALAAAAA|213|Johnson |Pkwy|Suite Y|Jamestown|Crowley County|CO|86867|United States|-7|condo| +2832|AAAAAAAAABLAAAAA|497|Oak Elm|Road|Suite H|Marion|Nelson County|VA|20399|United States|-5|condo| +2833|AAAAAAAABBLAAAAA|678|Cedar |Parkway|Suite 230|Springfield|Briscoe County|TX|79303|United States|-6|single family| +2834|AAAAAAAACBLAAAAA|351|7th Lake|Parkway|Suite 270|Antioch|Iowa County|WI|58605|United States|-6|condo| +2835|AAAAAAAADBLAAAAA|452|Chestnut 13th|Lane|Suite Y|Valley View|Dunn County|WI|55124|United States|-6|apartment| +2836|AAAAAAAAEBLAAAAA|755|Smith |Parkway|Suite 480|Pleasant Hill|Shawnee County|KS|63604|United States|-6|apartment| +2837|AAAAAAAAFBLAAAAA|664|Jackson 7th|ST|Suite 430|Cedar Grove|Clinton County|IA|50411|United States|-6|single family| +2838|AAAAAAAAGBLAAAAA|925|Mill |Ln|Suite 260|Hopewell|Cass County|NE|60587|United States|-6|apartment| +2839|AAAAAAAAHBLAAAAA|309|Pine |Dr.|Suite 390|Floyd|Ford County|IL|63235|United States|-6|single family| +2840|AAAAAAAAIBLAAAAA|888|2nd Seventh|Street|Suite G|Mount Zion|Windsor County|VT|08654|United States|-5|apartment| +2841|AAAAAAAAJBLAAAAA|336|Davis Church|Circle|Suite M|Oakland|Parmer County|TX|79843|United States|-6|apartment| +2842|AAAAAAAAKBLAAAAA|345|Willow |Blvd|Suite E|Bethel|Freestone County|TX|75281|United States|-6|single family| +2843|AAAAAAAALBLAAAAA|285|6th |Circle|Suite 160|Mount Pleasant|Murray County|GA|31933|United States|-5|single family| +2844|AAAAAAAAMBLAAAAA|148|Cedar Madison|RD|Suite Y|Spring Hill|Chariton County|MO|66787|United States|-6|condo| +2845|AAAAAAAANBLAAAAA|521|Maple 3rd|Boulevard|Suite U|Mountain View|Howard County|TX|74466|United States|-6|condo| +2846|AAAAAAAAOBLAAAAA|550|Main |Blvd|Suite T|Hopewell|Wade Hampton Census Area|AK|90587|United States|-9|single family| +2847|AAAAAAAAPBLAAAAA|532|Davis Wilson|Road|Suite X|Webb|Haines Borough|AK|90899|United States|-9|apartment| +2848|AAAAAAAAACLAAAAA|127|Park |Parkway|Suite 30|Bethel|Putnam County|WV|25281|United States|-5|condo| +2849|AAAAAAAABCLAAAAA|10|Poplar |Drive|Suite 320|Springfield|Madison County|MT|69303|United States|-7|single family| +2850|AAAAAAAACCLAAAAA|666|Mill Jackson|Pkwy|Suite 410|Five Points|Lee County|GA|36098|United States|-5|condo| +2851|AAAAAAAADCLAAAAA|821|Cedar Washington|Street|Suite I|Oak Hill|Franklin County|IA|57838|United States|-6|apartment| +2852|AAAAAAAAECLAAAAA|301|Wilson |Ct.|Suite R|Belmont|Boone County|NE|60191|United States|-6|condo| +2853|AAAAAAAAFCLAAAAA|296|Mill |Dr.|Suite M|Union|Walsh County|ND|58721|United States|-6|single family| +2854|AAAAAAAAGCLAAAAA|706|Cedar |Ct.|Suite Q|Spring Hill|Fairbanks North Star Borough|AK|96787|United States|-9|apartment| +2855|AAAAAAAAHCLAAAAA|137|Jefferson Lee|Blvd|Suite W|Lakeville|Latah County|ID|88811|United States|-7|single family| +2856|AAAAAAAAICLAAAAA|552|||Suite B|Pleasant Valley||TX||United States|-6|| +2857|AAAAAAAAJCLAAAAA|216|1st |RD|Suite 170|Farmington|Roanoke city|VA|29145|United States|-5|single family| +2858|AAAAAAAAKCLAAAAA|543|Walnut Railroad|Ct.|Suite C|Lebanon|Arenac County|MI|42898|United States|-5|apartment| +2859|AAAAAAAALCLAAAAA||Park |||||TN|39145|United States|-6|| +2860|AAAAAAAAMCLAAAAA|975|4th |RD|Suite W|Greenfield|Harrison County|MS|55038|United States|-6|single family| +2861|AAAAAAAANCLAAAAA|799|Park |Way|Suite W|Oak Grove|Boulder County|CO|88370|United States|-7|single family| +2862|AAAAAAAAOCLAAAAA|1000|Birch Walnut|Circle|Suite 300|Florence|Perry County|OH|43394|United States|-5|single family| +2863|AAAAAAAAPCLAAAAA||Meadow |Street|Suite H||Bourbon County|KS|||-6|| +2864|AAAAAAAAADLAAAAA|84|Laurel 2nd|Drive|Suite B|Summit|Mineral County|CO|80499|United States|-7|single family| +2865|AAAAAAAABDLAAAAA|85|Sycamore |Street|Suite W|Brownsville|Sequoyah County|OK|79310|United States|-6|apartment| +2866|AAAAAAAACDLAAAAA|145|3rd |Ct.|Suite U|Brownsville|Grant County|WI|59310|United States|-6|single family| +2867|AAAAAAAADDLAAAAA||Lake Smith|RD||Ellsworth||||||| +2868|AAAAAAAAEDLAAAAA|209|Railroad 7th|Road|Suite 310|Ashland|Flagler County|FL|34244|United States|-5|apartment| +2869|AAAAAAAAFDLAAAAA|561|Lake Spring|Dr.|Suite H|Red Hill|Shelby County|AL|34338|United States|-6|condo| +2870|AAAAAAAAGDLAAAAA|654|Madison |Pkwy|Suite 410|Spring Valley|Cuming County|NE|66060|United States|-6|single family| +2871|AAAAAAAAHDLAAAAA|881|Center 9th|Parkway|Suite P|Antioch|Brown County|KS|68605|United States|-6|condo| +2872|AAAAAAAAIDLAAAAA|721||Drive|||||60534||-6|| +2873|AAAAAAAAJDLAAAAA|203|Green 13th|Ave|Suite 210|Pine Grove|Arlington County|VA|24593|United States|-5|condo| +2874|AAAAAAAAKDLAAAAA|941|Hickory Locust|Street|Suite 190|Springfield|Alamance County|NC|29303|United States|-5|apartment| +2875|AAAAAAAALDLAAAAA|660|Pine |RD|Suite N|Spring Valley|Kiowa County|OK|76060|United States|-6|single family| +2876|AAAAAAAAMDLAAAAA|9|Elm |Way|Suite 160|Woodville|Henry County|VA|24289|United States|-5|condo| +2877|AAAAAAAANDLAAAAA|229|Birch |Street|Suite 250|Pleasant Hill|Carson County|TX|73604|United States|-6|single family| +2878|AAAAAAAAODLAAAAA|163|12th |Circle|Suite H|Franklin|Lincoln County|NE|69101|United States|-7|apartment| +2879|AAAAAAAAPDLAAAAA|303|Walnut 1st|Parkway|Suite K|Oak Ridge|Marion County|IL|68371|United States|-6|condo| +2880|AAAAAAAAAELAAAAA|196|12th |Way|Suite U|Oakdale|Mifflin County|PA|19584|United States|-5|condo| +2881|AAAAAAAABELAAAAA|873|North North|Boulevard|Suite N|Greenfield|Chattahoochee County|GA|35038|United States|-5|single family| +2882|AAAAAAAACELAAAAA|183|Cedar |Parkway|Suite 290|Riverdale|Clermont County|OH|49391|United States|-5|apartment| +2883|AAAAAAAADELAAAAA|832|Park |Parkway|Suite B|Pleasant Grove|Delaware County|OH|44136|United States|-5|condo| +2884|AAAAAAAAEELAAAAA|732|Lincoln |Ln|Suite W|Plainview|Gallatin County|MT|63683|United States|-7|single family| +2885|AAAAAAAAFELAAAAA|822|7th |Dr.|Suite J|Providence|Livingston Parish|LA|76614|United States|-6|condo| +2886|AAAAAAAAGELAAAAA|719|Highland Third|Blvd|Suite E|Farmington|Lavaca County|TX|79145|United States|-6|single family| +2887|AAAAAAAAHELAAAAA|984|Sycamore 6th|RD|Suite 320|Bunker Hill|Middlesex County|VA|20150|United States|-5|condo| +2888|AAAAAAAAIELAAAAA|758|North |RD|Suite X|Summerfield|Williamsburg County|SC|20634|United States|-5|apartment| +2889|AAAAAAAAJELAAAAA|824|Second Lee|Drive|Suite O|Buena Vista|Chouteau County|MT|65752|United States|-7|apartment| +2890|AAAAAAAAKELAAAAA|170|Park |Ct.|Suite P|Friendship|Twiggs County|GA|34536|United States|-5|single family| +2891|AAAAAAAALELAAAAA|308|12th Franklin|Ct.|Suite J|Kingston|Harrisonburg city|VA|24975|United States|-5|apartment| +2892|AAAAAAAAMELAAAAA|620|1st |RD|Suite O|Bridgeport|Nuckolls County|NE|65817|United States|-7|single family| +2893|AAAAAAAANELAAAAA|90|4th |Circle|Suite I|Mount Zion|Jasper County|SC|28054|United States|-5|apartment| +2894|AAAAAAAAOELAAAAA|767|1st |Drive|Suite 130|Ludlow|San Mateo County|CA|95566|United States|-8|apartment| +2895|AAAAAAAAPELAAAAA|282|Sunset |Ave|Suite X|Ashland|Kane County|UT|84244|United States|-7|single family| +2896|AAAAAAAAAFLAAAAA|720|Miller Miller|Ct.|Suite E|Belleview|Bath County|KY|40492|United States|-6|condo| +2897|AAAAAAAABFLAAAAA|637|2nd |Wy|Suite 440|Concord|Houston County|AL|34107|United States|-6|single family| +2898|AAAAAAAACFLAAAAA|567|3rd |Street|Suite 320|Friendship|Duplin County|NC|24536|United States|-5|single family| +2899|AAAAAAAADFLAAAAA|94|10th South|Dr.|Suite 80|Walnut Grove|Chatham County|NC|27752|United States|-5|apartment| +2900|AAAAAAAAEFLAAAAA|643|Elm Second|Ave|Suite V|Elkton|Drew County|AR|73481|United States|-6|condo| +2901|AAAAAAAAFFLAAAAA|461|Main Elm|Boulevard|Suite 450|Red Hill|Polk County|FL|34338|United States|-5|condo| +2902|AAAAAAAAGFLAAAAA|692|Lincoln |Boulevard|Suite 330|Lamont|Cumberland County|NJ|00804|United States|-5|condo| +2903|AAAAAAAAHFLAAAAA|795|1st |Way|Suite 130|Franklin|Greenbrier County|WV|29101|United States|-5|apartment| +2904|AAAAAAAAIFLAAAAA|871|5th |Street|Suite B|Plainview|San Benito County|CA|93683|United States|-8|apartment| +2905|AAAAAAAAJFLAAAAA|424|West |Circle|Suite 40|Jackson|Grant County|OK|79583|United States|-6|single family| +2906|AAAAAAAAKFLAAAAA|995|River |Cir.|Suite 450|Church Hill|Harding County|SD|53790|United States|-7|single family| +2907|AAAAAAAALFLAAAAA|615|4th |Lane|Suite Y|Jamestown|Union County|OR|96867|United States|-8|apartment| +2908|AAAAAAAAMFLAAAAA|558|Birch |Parkway|Suite P|Friendship|Webb County|TX|74536|United States|-6|single family| +2909|AAAAAAAANFLAAAAA|649|2nd 5th|Way|Suite 10|Cedar Grove|Warren County|KY|40411|United States|-5|apartment| +2910|AAAAAAAAOFLAAAAA|741|Chestnut |Dr.|Suite Q|California|Coffee County|AL|30141|United States|-6|apartment| +2911|AAAAAAAAPFLAAAAA|164|Park |Circle|Suite 50|Griffin|Washington County|KY|41204|United States|-5|apartment| +2912|AAAAAAAAAGLAAAAA|344|Lakeview |Lane|Suite 370|Hamilton|Pitkin County|CO|82808|United States|-7|condo| +2913|AAAAAAAABGLAAAAA|82|11th |ST|Suite F|Bethel|Douglas County|NV|85281|United States|-8|single family| +2914|AAAAAAAACGLAAAAA|399|First |Lane|Suite 130|Highland Park|Menifee County|KY|46534|United States|-5|apartment| +2915|AAAAAAAADGLAAAAA|964|Cedar |Dr.|Suite P|Cedar Grove|Ross County|OH|40411|United States|-5|single family| +2916|AAAAAAAAEGLAAAAA|733|Cedar Park|Way|Suite E|Shiloh|Barnwell County|SC|29275|United States|-5|single family| +2917|AAAAAAAAFGLAAAAA|795|Mill Dogwood|RD|Suite O|Brownsville|Clay County|WV|29310|United States|-5|single family| +2918|AAAAAAAAGGLAAAAA|739|Birch |Ave|Suite W|Spring Hill|Schuylkill County|PA|16787|United States|-5|single family| +2919|AAAAAAAAHGLAAAAA|846|3rd Second|Court|Suite G|Spring Valley|Snyder County|PA|16060|United States|-5|condo| +2920|AAAAAAAAIGLAAAAA|699|5th Maple|Blvd|Suite 60|Fairview|Leelanau County|MI|45709|United States|-5|apartment| +2921|AAAAAAAAJGLAAAAA|66|6th |Way|Suite L|Kingston|Lagrange County|IN|44975|United States|-5|condo| +2922|AAAAAAAAKGLAAAAA|173|Woodland |Road|Suite T|Bunker Hill|Candler County|GA|30150|United States|-5|apartment| +2923|AAAAAAAALGLAAAAA|387|Lake 1st|RD|Suite U|Woodlawn|Doddridge County|WV|24098|United States|-5|apartment| +2924|AAAAAAAAMGLAAAAA|398|View 7th|ST|Suite 50|Sulphur Springs|Henderson County|IL|68354|United States|-6|single family| +2925|AAAAAAAANGLAAAAA|209|Laurel Main|Cir.|Suite 490|Union|La Salle County|IL|68721|United States|-6|single family| +2926|AAAAAAAAOGLAAAAA||5th Washington||Suite R||||90169||-8|apartment| +2927|AAAAAAAAPGLAAAAA|427|Smith |Street|Suite A|Richville|Reynolds County|MO|65945|United States|-6|condo| +2928|AAAAAAAAAHLAAAAA|166|10th |Lane|Suite 300|Bridgeport|Harnett County|NC|25817|United States|-5|condo| +2929|AAAAAAAABHLAAAAA|434|Highland |Blvd|Suite 180|Friendship|Hardin County|IA|54536|United States|-6|condo| +2930|AAAAAAAACHLAAAAA|528|1st Mill|Cir.|Suite O|Lebanon|Clark County|WA|92898|United States|-8|condo| +2931|AAAAAAAADHLAAAAA|55|12th Cedar|Drive|Suite Y|Newtown|Mayes County|OK|71749|United States|-6|condo| +2932|AAAAAAAAEHLAAAAA|349|Meadow |Road|Suite 30|Stringtown|Belmont County|OH|40162|United States|-5|condo| +2933|AAAAAAAAFHLAAAAA|514|8th |Lane|Suite 270|Wilson|Comanche County|OK|76971|United States|-6|apartment| +2934|AAAAAAAAGHLAAAAA|197|Central West|Avenue|Suite M|Newtown|Llano County|TX|71749|United States|-6|condo| +2935|AAAAAAAAHHLAAAAA|405|Miller |Circle|Suite 400|Allison|Union County|NM|84167|United States|-7|apartment| +2936|AAAAAAAAIHLAAAAA|659|Chestnut 12th|Parkway|Suite H|Union|Dixie County|FL|38721|United States|-5|single family| +2937|AAAAAAAAJHLAAAAA|334|South Dogwood|Parkway|Suite 370|Clifton|Clinton County|IA|58014|United States|-6|single family| +2938|AAAAAAAAKHLAAAAA|242|North 3rd|Pkwy|Suite M|Oak Ridge|Marion County|GA|38371|United States|-5|single family| +2939|AAAAAAAALHLAAAAA||Main View|||Riverdale||NC|29391|United States||condo| +2940|AAAAAAAAMHLAAAAA|907|5th 8th|Avenue|Suite I|Pleasant Grove|Nash County|NC|24136|United States|-5|apartment| +2941|AAAAAAAANHLAAAAA|474|8th Main|Pkwy|Suite 370|Glendale|Woodford County|KY|43951|United States|-6|apartment| +2942|AAAAAAAAOHLAAAAA|613|9th |Court|Suite 490|Greenfield|Dyer County|TN|35038|United States|-5|apartment| +2943|AAAAAAAAPHLAAAAA|153|Maple |Wy|Suite N|Newport|Dauphin County|PA|11521|United States|-5|single family| +2944|AAAAAAAAAILAAAAA|225|Jefferson Lakeview|Ln|Suite Y|Oakwood|Pulaski County|VA|20169|United States|-5|apartment| +2945|AAAAAAAABILAAAAA|636|Wilson |Avenue|Suite 350|Gladstone|Bolivar County|MS|50894|United States|-6|condo| +2946|AAAAAAAACILAAAAA|522|Cedar 3rd|Pkwy|Suite 480|Oak Ridge|Neosho County|KS|68371|United States|-6|single family| +2947|AAAAAAAADILAAAAA|819|Wilson Woodland|Cir.|Suite M|Fairview|Auglaize County|OH|45709|United States|-5|condo| +2948|AAAAAAAAEILAAAAA|905|13th |Court|Suite I|Oakland|Ravalli County|MT|69843|United States|-7|single family| +2949|AAAAAAAAFILAAAAA|945|Cherry |Blvd|Suite 370|Church Hill|Cass County|NE|63790|United States|-6|single family| +2950|AAAAAAAAGILAAAAA|446|Adams |Pkwy|Suite 190|Lakeview|Benewah County|ID|88579|United States|-7|apartment| +2951|AAAAAAAAHILAAAAA|135|13th |Court|Suite W|Harmony|Irwin County|GA|35804|United States|-5|single family| +2952|AAAAAAAAIILAAAAA|812|Laurel Elm|Ln|Suite N|Shady Grove|Essex County|NJ|03412|United States|-5|condo| +2953|AAAAAAAAJILAAAAA|893|Chestnut |Ave|Suite 90|Pleasant Valley|Petroleum County|MT|62477|United States|-7|single family| +2954|AAAAAAAAKILAAAAA|369|10th Elm|Blvd|Suite F|Greenfield|Marion County|TX|75038|United States|-6|single family| +2955|AAAAAAAALILAAAAA|196|Pine |Pkwy|Suite K|Lakewood|Allamakee County|IA|58877|United States|-6|condo| +2956|AAAAAAAAMILAAAAA|70|8th |Pkwy|Suite 50|Hillcrest|Sharp County|AR|73003|United States|-6|condo| +2957|AAAAAAAANILAAAAA|137|Cedar |Road|Suite 180|Clifton|Smith County|KS|68014|United States|-6|condo| +2958|AAAAAAAAOILAAAAA|735|Pine |RD|Suite Q|Lakeview|McKenzie County|ND|58579|United States|-6|condo| +2959|AAAAAAAAPILAAAAA|25|12th |Cir.|Suite G|Buckingham|Cedar County|NE|64092|United States|-6|apartment| +2960|AAAAAAAAAJLAAAAA|476|Fourth |Pkwy|Suite 310|Buena Vista|Androscoggin County|ME|06352|United States|-5|condo| +2961|AAAAAAAABJLAAAAA|918|Maple 6th|Circle|Suite 10|Highland|Franklin County|VA|29454|United States|-5|condo| +2962|AAAAAAAACJLAAAAA|749|First |Blvd|Suite 410|Roscoe|McLean County|ND|51854|United States|-6|single family| +2963|AAAAAAAADJLAAAAA|652|Walnut 10th|ST|Suite M|Unionville|Noble County|OK|71711|United States|-6|apartment| +2964|AAAAAAAAEJLAAAAA|992|Spring |Cir.|Suite 320|Pine Grove|Douglas County|WA|94593|United States|-8|apartment| +2965|AAAAAAAAFJLAAAAA|19|East Spruce|Ave|Suite J|Pine Grove|Ripley County|MO|64593|United States|-6|apartment| +2966|AAAAAAAAGJLAAAAA|978|Adams Mill|Boulevard|Suite 70|Stringtown|Neosho County|KS|60162|United States|-6|apartment| +2967|AAAAAAAAHJLAAAAA|442|Hickory Railroad|Boulevard|Suite 350|Fairfield|Winchester city|VA|26192|United States|-5|apartment| +2968|AAAAAAAAIJLAAAAA|197|Laurel Park|Way|Suite M|Oakwood|Matanuska-Susitna Borough|AK|90169|United States|-9|condo| +2969|AAAAAAAAJJLAAAAA|208|Poplar 1st|RD|Suite C|Mount Olive|Schleicher County|TX|78059|United States|-6|single family| +2970|AAAAAAAAKJLAAAAA|321|Pine Woodland|Avenue|Suite 40|Five Points|Spalding County|GA|36098|United States|-5|apartment| +2971|AAAAAAAALJLAAAAA|62|Fifth |Pkwy|Suite S|Newtown|Tioga County|NY|11749|United States|-5|single family| +2972|AAAAAAAAMJLAAAAA|533|Dogwood Lake|Boulevard|Suite K|Hamilton|Appanoose County|IA|52808|United States|-6|condo| +2973|AAAAAAAANJLAAAAA|526|Maple |Blvd|Suite 440|Glendale|Florence County|WI|53951|United States|-6|single family| +2974|AAAAAAAAOJLAAAAA|897|Wilson |Blvd|Suite 180|Greenville|Washington County|ME|01987|United States|-5|condo| +2975|AAAAAAAAPJLAAAAA|450|Forest |Road|Suite T|Glenwood|Canadian County|OK|73511|United States|-6|apartment| +2976|AAAAAAAAAKLAAAAA|235|||||||||-6|single family| +2977|AAAAAAAABKLAAAAA|326|Sycamore Park|Avenue|Suite 90|Unionville|White County|GA|31711|United States|-5|condo| +2978|AAAAAAAACKLAAAAA|689|Cherry North|Wy|Suite 220|Ashley|Graham County|KS|64324|United States|-6|apartment| +2979|AAAAAAAADKLAAAAA|109|Franklin 6th|Lane|Suite 420|Sulphur Springs|Halifax County|VA|28354|United States|-5|condo| +2980|AAAAAAAAEKLAAAAA|305|Spring Oak|ST|Suite F|Indian Village|Van Zandt County|TX|71075|United States|-6|single family| +2981|AAAAAAAAFKLAAAAA|273|5th Center|Boulevard|Suite 60|Plainview|Talbot County|MD|23683|United States|-5|single family| +2982|AAAAAAAAGKLAAAAA|716|11th |Lane|Suite 320|Belmont|Watonwan County|MN|50191|United States|-6|apartment| +2983|AAAAAAAAHKLAAAAA|59|10th |Ave|Suite 470|Oak Hill|Taylor County|WI|57838|United States|-6|apartment| +2984|AAAAAAAAIKLAAAAA|367|Hill |Circle|Suite H|Whispering Pines|Jenkins County|GA|37609|United States|-5|condo| +2985|AAAAAAAAJKLAAAAA|687|Jackson Dogwood|Street|Suite 170|Brownsville|Live Oak County|TX|79310|United States|-6|condo| +2986|AAAAAAAAKKLAAAAA|446|Main Johnson|Court|Suite C|Hardy|Anderson County|TN|35354|United States|-5|apartment| +2987|AAAAAAAALKLAAAAA|785|Washington Tenth|Circle|Suite 160|Pleasant Valley|Portsmouth city|VA|22477|United States|-5|apartment| +2988|AAAAAAAAMKLAAAAA|921|Pine Sunset|ST|Suite 400|Walnut Grove|DeKalb County|MO|67752|United States|-6|single family| +2989|AAAAAAAANKLAAAAA|183|15th Willow|Ct.|Suite 80|Glendale|Talladega County|AL|33951|United States|-6|condo| +2990|AAAAAAAAOKLAAAAA|51|14th Hill|Cir.|Suite C|Mount Olive|Calumet County|WI|58059|United States|-6|condo| +2991|AAAAAAAAPKLAAAAA|177|South |Wy|Suite 250|Harmony|Bryan County|GA|35804|United States|-5|apartment| +2992|AAAAAAAAALLAAAAA|582|Cherry 13th|Ln|Suite 450|Greenville|Washington County|KS|61387|United States|-6|single family| +2993|AAAAAAAABLLAAAAA|239|North Smith|Lane|Suite 450|Wilton|Mitchell County|TX|76997|United States|-6|condo| +2994|AAAAAAAACLLAAAAA|904|Willow 8th|Wy|Suite 460|Mount Vernon|Allamakee County|IA|58482|United States|-6|apartment| +2995|AAAAAAAADLLAAAAA|303|Green |Lane|Suite 110|Mount Zion|Wichita County|KS|68054|United States|-6|apartment| +2996|AAAAAAAAELLAAAAA|188|Dogwood Fifth|Ct.|Suite 320|Spring Valley|Muscatine County|IA|56060|United States|-6|single family| +2997|AAAAAAAAFLLAAAAA|173|Ash |Court|Suite 390|Forest Hills|Cumberland County|TN|39237|United States|-5|apartment| +2998|AAAAAAAAGLLAAAAA|286|Main Fourth|Ave|Suite 50|Jackson|Hancock County|MS|59583|United States|-6|single family| +2999|AAAAAAAAHLLAAAAA|122|Jackson |Way|Suite 150|Mount Olive|Adair County|OK|78059|United States|-6|condo| +3000|AAAAAAAAILLAAAAA|644|5th Sixth|Street|Suite T|Cedar Grove|Brunswick County|VA|20411|United States|-5|apartment| +3001|AAAAAAAAJLLAAAAA|469|Sunset |Dr.|Suite 100|Cedar Grove|Wilkinson County|MS|50411|United States|-6|apartment| +3002|AAAAAAAAKLLAAAAA|579|Church Johnson|Ave|Suite R|Fairview|Piatt County|IL|65709|United States|-6|single family| +3003|AAAAAAAALLLAAAAA|575|Ridge |Wy|Suite K|Valley View|Davidson County|TN|35124|United States|-5|condo| +3004|AAAAAAAAMLLAAAAA|959|Valley |Dr.|Suite 0|Hamilton|Darlington County|SC|22808|United States|-5|condo| +3005|AAAAAAAANLLAAAAA|847|Sunset |Blvd|Suite M|Springfield|Pocahontas County|IA|59303|United States|-6|single family| +3006|AAAAAAAAOLLAAAAA|946|Spruce 14th|Circle|Suite 480|New Hope|Greer County|OK|79431|United States|-6|single family| +3007|AAAAAAAAPLLAAAAA|329|Park 6th|Drive|Suite E|Unionville|Vernon Parish|LA|71711|United States|-6|single family| +3008|AAAAAAAAAMLAAAAA|956|3rd Spring|Way|Suite 100|Macedonia|Cass County|MI|41087|United States|-5|apartment| +3009|AAAAAAAABMLAAAAA|816|East |Way|Suite 180|Hopewell|Washington County|VT|01187|United States|-5|condo| +3010|AAAAAAAACMLAAAAA|848|Pine 4th|ST|Suite F|Franklin|Mobile County|AL|39101|United States|-6|condo| +3011|AAAAAAAADMLAAAAA|||Ct.||Fairview|Aleutians East Borough|AK|95709|United States||condo| +3012|AAAAAAAAEMLAAAAA|400|Birch 2nd|Court|Suite X|Greenfield|Gallatin County|KY|45038|United States|-6|condo| +3013|AAAAAAAAFMLAAAAA|466|Adams |Ave|Suite 100|Salem|Oconee County|SC|28048|United States|-5|condo| +3014|AAAAAAAAGMLAAAAA|310|Jackson Third|Court|Suite 360|Oakland|Barrow County|GA|39843|United States|-5|apartment| +3015|AAAAAAAAHMLAAAAA|90|Forest Green|Cir.||Edgewood|||||-7|| +3016|AAAAAAAAIMLAAAAA|76|View |Ct.|Suite 20|Altamont|Foard County|TX|79387|United States|-6|single family| +3017|AAAAAAAAJMLAAAAA|961|Pine College|Cir.|Suite 40|Maple Grove|Houston County|MN|58252|United States|-6|condo| +3018|AAAAAAAAKMLAAAAA|296|3rd |Avenue|Suite 110|Fairview|Franklin County|NC|25709|United States|-5|apartment| +3019|AAAAAAAALMLAAAAA|870|West |Cir.|Suite I|Five Points|Osage County|OK|76098|United States|-6|apartment| +3020|AAAAAAAAMMLAAAAA|21|Third |Lane|Suite X|Friendship|Tioga County|NY|14536|United States|-5|single family| +3021|AAAAAAAANMLAAAAA|438|Spring |Lane|Suite 80|Oak Hill|Cerro Gordo County|IA|57838|United States|-6|condo| +3022|AAAAAAAAOMLAAAAA|490|1st |Circle|Suite X|Green Acres|Somerset County|MD|27683|United States|-5|single family| +3023|AAAAAAAAPMLAAAAA|626|Fourth Railroad|Dr.|Suite U|Liberty|Harrison County|MS|53451|United States|-6|single family| +3024|AAAAAAAAANLAAAAA|244|Second |Wy|Suite P|Greenwood|Larimer County|CO|88828|United States|-7|single family| +3025|AAAAAAAABNLAAAAA|90|Mill |Street|Suite 360|Woodville|Lyon County|MN|54289|United States|-6|condo| +3026|AAAAAAAACNLAAAAA|107|Jefferson Fifth|Ct.|Suite W|Concord|Alger County|MI|44107|United States|-5|apartment| +3027|AAAAAAAADNLAAAAA|740|Fifth Pine|Boulevard|Suite Y|Sunnyside|Jackson County|FL|31952|United States|-5|single family| +3028|AAAAAAAAENLAAAAA|639|8th Hill|Ct.|Suite C|Forest Hills|Wheatland County|MT|69237|United States|-7|apartment| +3029|AAAAAAAAFNLAAAAA|137|Adams Seventh|Blvd|Suite 40|Hamilton|East Feliciana Parish|LA|72808|United States|-6|apartment| +3030|AAAAAAAAGNLAAAAA|641|Hickory College|Cir.|Suite B|Sunnyside|Fayette County|TX|71952|United States|-6|condo| +3031|AAAAAAAAHNLAAAAA|163|Central |Ave|Suite P|Brownsville|Scioto County|OH|49310|United States|-5|apartment| +3032|AAAAAAAAINLAAAAA|814|Church |Way|Suite K|Hillcrest|Oldham County|KY|43003|United States|-5|single family| +3033|AAAAAAAAJNLAAAAA|513|9th 1st|Way|Suite O|Spring Valley|Niobrara County|WY|86060|United States|-7|single family| +3034|AAAAAAAAKNLAAAAA|38|Main Green|Wy|Suite 160|Brownsville|Washington County|TN|39310|United States|-6|single family| +3035|AAAAAAAALNLAAAAA|520|12th Central|Lane|Suite 70|Mount Zion|Manassas city|VA|28054|United States|-5|condo| +3036|AAAAAAAAMNLAAAAA|326|College |Blvd|Suite 180|Hamilton|San Mateo County|CA|92808|United States|-8|single family| +3037|AAAAAAAANNLAAAAA|668|||Suite 280||Lemhi County|ID|89532|||| +3038|AAAAAAAAONLAAAAA|539|Hickory |Ct.|Suite N|Red Hill|Newberry County|SC|24338|United States|-5|single family| +3039|AAAAAAAAPNLAAAAA|11|East |RD|Suite W|Franklin|Roseau County|MN|59101|United States|-6|condo| +3040|AAAAAAAAAOLAAAAA|645|Oak Hillcrest|ST|Suite A|Shady Grove|Newton County|GA|32812|United States|-5|apartment| +3041|AAAAAAAABOLAAAAA|59|First East|Drive|Suite X|Crossroads|Nobles County|MN|50534|United States|-6|condo| +3042|AAAAAAAACOLAAAAA|956|7th South|Road|Suite U|Shiloh|Aleutians West Census Area|AK|99275|United States|-9|apartment| +3043|AAAAAAAADOLAAAAA|568|Spruce South|Way|Suite J|Clinton|Belmont County|OH|48222|United States|-5|condo| +3044|AAAAAAAAEOLAAAAA|336|Second |Wy|Suite V|Washington Heights|Claiborne County|MS|58167|United States|-6|single family| +3045|AAAAAAAAFOLAAAAA|25|Forest 7th|Boulevard|Suite 0|Sulphur Springs|Clinton County|MO|68354|United States|-6|single family| +3046|AAAAAAAAGOLAAAAA|440|Oak Meadow|Avenue|Suite 350|Highland Park|Mingo County|WV|26534|United States|-5|apartment| +3047|AAAAAAAAHOLAAAAA|241|Second Jackson|Circle|Suite P|Greenwood|Carver County|MN|58828|United States|-6|single family| +3048|AAAAAAAAIOLAAAAA|898|Main |Avenue|Suite L|Hopewell|Ingham County|MI|40587|United States|-5|single family| +3049|AAAAAAAAJOLAAAAA|441|Pine |Blvd|Suite R|Lincoln|Hamilton County|TN|31289|United States|-5|apartment| +3050|AAAAAAAAKOLAAAAA|423|Locust Sixth|Avenue|Suite 310|Plainview|Pittsburg County|OK|73683|United States|-6|single family| +3051|AAAAAAAALOLAAAAA|341|Park East|Blvd|Suite 260|Bath|Van Wert County|OH|40573|United States|-5|single family| +3052|AAAAAAAAMOLAAAAA|289|6th |Drive|Suite L|Gum Springs|Wood County|OH|42106|United States|-5|apartment| +3053|AAAAAAAANOLAAAAA|829|Oak |Pkwy|Suite V|Concord|Campbell County|SD|54107|United States|-6|apartment| +3054|AAAAAAAAOOLAAAAA|456|5th |Ave|Suite U|Wyoming|Windham County|CT|00816|United States|-5|single family| +3055|AAAAAAAAPOLAAAAA|70|Maple |Lane|Suite M|Farmington|Itasca County|MN|59145|United States|-6|single family| +3056|AAAAAAAAAPLAAAAA|959|Central ||||Haskell County|TX||United States|-6|| +3057|AAAAAAAABPLAAAAA|829|Miller |Parkway|Suite L|Lakeview|Laurens County|SC|28579|United States|-5|condo| +3058|AAAAAAAACPLAAAAA|951|4th Jefferson|Boulevard|Suite 120|Lincoln|Mecklenburg County|VA|21289|United States|-5|apartment| +3059|AAAAAAAADPLAAAAA|382|Main 8th|ST|Suite 130|Woodville|Currituck County|NC|24289|United States|-5|single family| +3060|AAAAAAAAEPLAAAAA|554|Cherry Second|Ln|Suite 450|Fayetteville|Malheur County|OR|91732|United States|-8|single family| +3061|AAAAAAAAFPLAAAAA|769|Seventh Park|Parkway|Suite 100|Oakdale|Cherokee County|NC|29584|United States|-5|apartment| +3062|AAAAAAAAGPLAAAAA|32|Jackson |Pkwy|Suite 70|Arthur|Washington County|WI|55965|United States|-6|single family| +3063|AAAAAAAAHPLAAAAA|507|Railroad |Ave|Suite X|Mount Vernon|Hyde County|SD|58482|United States|-7|single family| +3064|AAAAAAAAIPLAAAAA|862|Poplar |Lane|Suite B|Florence|Saluda County|SC|23394|United States|-5|apartment| +3065|AAAAAAAAJPLAAAAA|793|Jackson Jackson|RD|Suite 0|Summit|Washington County|WI|50499|United States|-6|single family| +3066|AAAAAAAAKPLAAAAA|198|Cherry Davis|Ave|Suite R|Plainview|Pitt County|NC|23683|United States|-5|condo| +3067|AAAAAAAALPLAAAAA|868|Sunset Lake|Pkwy|Suite 400|Clinton|Trimble County|KY|48222|United States|-5|single family| +3068|AAAAAAAAMPLAAAAA|607|Lincoln |Parkway|Suite B|Pine Valley|Breathitt County|KY|48209|United States|-6|apartment| +3069|AAAAAAAANPLAAAAA|198|4th 6th|Pkwy|Suite X|Fairfield|Jackson County|OR|96192|United States|-8|apartment| +3070|AAAAAAAAOPLAAAAA|561|Fifth |Ave|Suite 30|Unionville|Seward County|KS|61711|United States|-6|single family| +3071|AAAAAAAAPPLAAAAA|409|Fifth North|Cir.|Suite 450|Harmony|Jackson County|AR|75804|United States|-6|single family| +3072|AAAAAAAAAAMAAAAA|446|Cedar Railroad|Pkwy|Suite 260|Green Acres|Holmes County|FL|37683|United States|-5|apartment| +3073|AAAAAAAABAMAAAAA|459|4th |Drive|Suite P|Allentown|Caldwell County|MO|61838|United States|-6|single family| +3074|AAAAAAAACAMAAAAA|||RD||Clinton|||78222||-6|| +3075|AAAAAAAADAMAAAAA|416|Spring View|Ct.|Suite I|Greenfield|Woodson County|KS|65038|United States|-6|condo| +3076|AAAAAAAAEAMAAAAA|717|Spring Washington|Drive|Suite 130|Lincoln|Deuel County|NE|61289|United States|-6|condo| +3077|AAAAAAAAFAMAAAAA|36|Mill Second|Cir.|Suite 470|Arthur|Williamson County|TN|35965|United States|-5|condo| +3078|AAAAAAAAGAMAAAAA|544|Jefferson |Blvd|Suite B|Lebanon|Aleutians East Borough|AK|92898|United States|-9|condo| +3079|AAAAAAAAHAMAAAAA|316|Walnut Seventh|Circle|Suite S|Five Forks|Rock County|WI|52293|United States|-6|single family| +3080|AAAAAAAAIAMAAAAA|280|River Wilson|Ave|Suite 60|Five Forks|Washington County|TN|32293|United States|-6|single family| +3081|AAAAAAAAJAMAAAAA|786|Poplar Lincoln|Parkway|Suite 0|Union Hill|Rowan County|KY|47746|United States|-5|single family| +3082|AAAAAAAAKAMAAAAA|304|Washington |Pkwy|Suite 430|Concord|Winnebago County|IA|54107|United States|-6|single family| +3083|AAAAAAAALAMAAAAA|473|River Lincoln|Boulevard|Suite B|Ellsworth|Bartow County|GA|35079|United States|-5|apartment| +3084|AAAAAAAAMAMAAAAA|927|Church Elm|Pkwy|Suite 90|Riverview|Greeley County|KS|69003|United States|-6|condo| +3085|AAAAAAAANAMAAAAA|521|Ash Oak|Street|Suite 310|Bethel|Rio Arriba County|NM|85281|United States|-7|condo| +3086|AAAAAAAAOAMAAAAA|476|Jackson Mill|Dr.|Suite S|Wildwood|Chariton County|MO|66871|United States|-6|single family| +3087|AAAAAAAAPAMAAAAA|516|Johnson Willow|Road|Suite N|Frogtown|Shasta County|CA|98784|United States|-8|condo| +3088|AAAAAAAAABMAAAAA|585|First Locust|Dr.|Suite 310|Harmony|Upshur County|TX|75804|United States|-6|condo| +3089|AAAAAAAABBMAAAAA|277|6th 6th|Ct.|Suite A|Green Acres|Treasure County|MT|67683|United States|-7|single family| +3090|AAAAAAAACBMAAAAA|269|Ridge Cedar|Wy|Suite R|Bunker Hill|Northampton County|VA|20150|United States|-5|apartment| +3091|AAAAAAAADBMAAAAA|395|3rd |Pkwy|Suite U|Stringtown|San Miguel County|CO|80162|United States|-7|condo| +3092|AAAAAAAAEBMAAAAA|686|3rd |Street|Suite C|Salem|Wabash County|IN|48048|United States|-5|condo| +3093|AAAAAAAAFBMAAAAA|459|Center |Dr.|Suite 240|Five Forks|Caribou County|ID|82293|United States|-7|apartment| +3094|AAAAAAAAGBMAAAAA|374|Walnut 1st|Pkwy|Suite 240|Springfield|Ripley County|IN|49303|United States|-5|apartment| +3095|AAAAAAAAHBMAAAAA|645|Seventh |ST|Suite 230|Lincoln|Rockdale County|GA|31289|United States|-5|single family| +3096|AAAAAAAAIBMAAAAA|744|Madison |Parkway|Suite X|Mount Vernon|Kodiak Island Borough|AK|98482|United States|-9|apartment| +3097|AAAAAAAAJBMAAAAA|531|Spring 6th|Blvd|Suite 440|Antioch|Adair County|MO|68605|United States|-6|single family| +3098|AAAAAAAAKBMAAAAA|59|3rd 1st|Road|Suite 160|Providence|Boone County|KY|46614|United States|-6|apartment| +3099|AAAAAAAALBMAAAAA|43|Sycamore |Way|Suite L|Union|Harrison County|WV|28721|United States|-5|apartment| +3100|AAAAAAAAMBMAAAAA|502|Green |Wy|Suite C|Whitesville|Lucas County|IA|55903|United States|-6|condo| +3101|AAAAAAAANBMAAAAA|241|1st |RD|Suite 280|Maple Grove|Collier County|FL|38252|United States|-5|single family| +3102|AAAAAAAAOBMAAAAA|211|Washington |Boulevard|Suite D|Waterloo|Pine County|MN|51675|United States|-6|condo| +3103|AAAAAAAAPBMAAAAA|607|Walnut 6th|Wy|Suite W|Mount Pleasant|Jo Daviess County|IL|61933|United States|-6|condo| +3104|AAAAAAAAACMAAAAA|235|Ridge 11th|Ave|Suite Q|Maple Grove|Charlotte County|VA|28252|United States|-5|apartment| +3105|AAAAAAAABCMAAAAA||Cedar ||||Monroe County|||||single family| +3106|AAAAAAAACCMAAAAA|235|Hickory |Way|Suite R|Mount Pleasant|Kidder County|ND|51933|United States|-6|apartment| +3107|AAAAAAAADCMAAAAA|753|Jackson |Avenue|Suite 60|Pine Grove|Middlesex County|MA|05193|United States|-5|single family| +3108|AAAAAAAAECMAAAAA|552|Central |Ct.|Suite 20|Buena Vista|Roosevelt County|MT|65752|United States|-7|apartment| +3109|AAAAAAAAFCMAAAAA|719|Laurel Franklin|Way|Suite L|Shiloh|Hood County|TX|79275|United States|-6|single family| +3110|AAAAAAAAGCMAAAAA|179|Jackson |Circle|Suite S|Spring Hill|Grainger County|TN|36787|United States|-5|condo| +3111|AAAAAAAAHCMAAAAA|997|11th |Ct.|Suite 410|Bunker Hill|Jim Hogg County|TX|70150|United States|-6|single family| +3112|AAAAAAAAICMAAAAA|464|Pine West|Wy|Suite 230|Fairview|Clearfield County|PA|15709|United States|-5|apartment| +3113|AAAAAAAAJCMAAAAA|20|Lee Ridge|Court|Suite U|Unionville|Cumberland County|PA|11711|United States|-5|condo| +3114|AAAAAAAAKCMAAAAA|81|Pine |Ave|Suite 110|Pleasant Hill|Cayuga County|NY|13604|United States|-5|apartment| +3115|AAAAAAAALCMAAAAA|253|Cherry Eigth|ST|Suite 120|Five Forks|Douglas County|MN|52293|United States|-6|apartment| +3116|AAAAAAAAMCMAAAAA|972|Oak |Circle|Suite 130|Mountain View|Richland County|WI|54466|United States|-6|condo| +3117|AAAAAAAANCMAAAAA|213|4th |Way|Suite I|Jackson|Breckinridge County|KY|49583|United States|-6|condo| +3118|AAAAAAAAOCMAAAAA|939|Dogwood 2nd|Lane|Suite J|Union|Worcester County|MD|28721|United States|-5|condo| +3119|AAAAAAAAPCMAAAAA|969|Third Ridge|Blvd|Suite O|Jackson|Genesee County|MI|49583|United States|-5|condo| +3120|AAAAAAAAADMAAAAA|84|Spring |Road|Suite 220|Ellsworth|Monroe County|MS|55079|United States|-6|condo| +3121|AAAAAAAABDMAAAAA|849|First |Ave|Suite X|Antioch|Carter County|KY|48605|United States|-6|condo| +3122|AAAAAAAACDMAAAAA|302|Center |Blvd|Suite I|Star|Humboldt County|CA|90725|United States|-8|apartment| +3123|AAAAAAAADDMAAAAA|314|Church Madison|Road|Suite 230|Oak Hill|Sheridan County|WY|87838|United States|-7|single family| +3124|AAAAAAAAEDMAAAAA|977|Eigth |RD|Suite 270|Forest Hills|Walker County|GA|39237|United States|-5|single family| +3125|AAAAAAAAFDMAAAAA|327|Church Elm|||||||||apartment| +3126|AAAAAAAAGDMAAAAA|662|4th |Street|Suite 120|Aberdeen|Beckham County|OK|77995|United States|-6|single family| +3127|AAAAAAAAHDMAAAAA|78|Lincoln Smith|Ave|Suite 420|Greenwood|Otter Tail County|MN|58828|United States|-6|apartment| +3128|AAAAAAAAIDMAAAAA|952|Third Johnson|Lane|Suite B|Wildwood|Howard County|AR|76871|United States|-6|single family| +3129|AAAAAAAAJDMAAAAA|677|Walnut |Drive|Suite 300|Lakeside|Ventura County|CA|99532|United States|-8|apartment| +3130|AAAAAAAAKDMAAAAA|||Avenue||Providence|Marion County|WV|26614|||| +3131|AAAAAAAALDMAAAAA|103|Cedar Oak|Way|Suite 70|Highland|Cameron County|TX|79454|United States|-6|single family| +3132|AAAAAAAAMDMAAAAA|234|First Park|Pkwy|Suite N|Newport|Buffalo County|SD|51521|United States|-6|apartment| +3133|AAAAAAAANDMAAAAA|854|Johnson South|Ln|Suite 460|Riverside|Cleveland County|AR|79231|United States|-6|apartment| +3134|AAAAAAAAODMAAAAA|510|Seventh Spring|Ct.|Suite 60|Georgetown|Yazoo County|MS|57057|United States|-6|single family| +3135|AAAAAAAAPDMAAAAA|796|Cherry |Way|Suite 330|Oakland|DeSoto County|FL|39843|United States|-5|single family| +3136|AAAAAAAAAEMAAAAA|202|5th 13th|Pkwy|Suite O|Paxton|Idaho County|ID|85669|United States|-7|condo| +3137|AAAAAAAABEMAAAAA|817|East Sixth|Lane|Suite S|Green Acres|Wallowa County|OR|97683|United States|-8|apartment| +3138|AAAAAAAACEMAAAAA|866|Franklin |Cir.|Suite 10|Glendale|Scott County|MS|53951|United States|-6|apartment| +3139|AAAAAAAADEMAAAAA|982|Ash 8th|Ave|Suite H|Five Forks|Douglas County|WA|92293|United States|-8|apartment| +3140|AAAAAAAAEEMAAAAA|147|Pine |Avenue|Suite A|Oakdale|Grafton County|NH|09584|United States|-5|single family| +3141|AAAAAAAAFEMAAAAA|205|Washington |||Green Acres|Buena Vista city|VA|27683|United States|-5|apartment| +3142|AAAAAAAAGEMAAAAA|245|Oak 2nd|Ave|Suite S|Newport|Mercer County|MO|61521|United States|-6|apartment| +3143|AAAAAAAAHEMAAAAA||Hill ||Suite Q|Greenwood||TX||United States|-6|| +3144|AAAAAAAAIEMAAAAA|897|3rd River|Wy|Suite I|Greenville|Rock County|WI|51387|United States|-6|single family| +3145|AAAAAAAAJEMAAAAA|327|Fifth |RD|Suite 180|Wyoming|Atchison County|MO|60216|United States|-6|apartment| +3146|AAAAAAAAKEMAAAAA|633|Cedar Jackson|Drive|Suite 50|Acme|Buena Vista County|IA|50164|United States|-6|single family| +3147|AAAAAAAALEMAAAAA|807|Maple |RD|Suite Y|Oak Ridge|Brown County|OH|48371|United States|-5|single family| +3148|AAAAAAAAMEMAAAAA|214|Highland |Dr.|Suite 160|Sunnyside|Montgomery County|IL|61952|United States|-6|apartment| +3149|AAAAAAAANEMAAAAA|354|Center Davis|RD|Suite 440|Valley View|La Salle Parish|LA|75124|United States|-6|condo| +3150|AAAAAAAAOEMAAAAA|299|Jefferson ||Suite Q|Clinton|||58222|||condo| +3151|AAAAAAAAPEMAAAAA|561|Mill Adams|Ln|Suite A|Centerville|Prentiss County|MS|50059|United States|-6|single family| +3152|AAAAAAAAAFMAAAAA|515|Park 4th|Road|Suite 350|Sulphur Springs|Rockcastle County|KY|48354|United States|-5|condo| +3153|AAAAAAAABFMAAAAA|886|Forest First|Ln|Suite 250|Lebanon|Miller County|AR|72898|United States|-6|condo| +3154|AAAAAAAACFMAAAAA|||Avenue|Suite 260|Deerfield|||69840|United States|-6|apartment| +3155|AAAAAAAADFMAAAAA|899|Willow |Cir.|Suite I|Smith|Unicoi County|TN|37317|United States|-6|apartment| +3156|AAAAAAAAEFMAAAAA|887|5th |Wy|Suite 190|Mount Zion|Moniteau County|MO|68054|United States|-6|apartment| +3157|AAAAAAAAFFMAAAAA|417|Woodland 10th|Road|Suite 370|Green Acres|Wapello County|IA|57683|United States|-6|apartment| +3158|AAAAAAAAGFMAAAAA|979|Fifth Church|Ln|Suite O|Newport|Rooks County|KS|61521|United States|-6|apartment| +3159|AAAAAAAAHFMAAAAA|589|West |Ct.|Suite 480|Riverside|Richland County|SC|29231|United States|-5|apartment| +3160|AAAAAAAAIFMAAAAA|388|West |RD|Suite M|Sheridan|King County|WA|97219|United States|-8|condo| +3161|AAAAAAAAJFMAAAAA|732|Meadow |Court|Suite 370|Jackson|Midland County|MI|49583|United States|-5|condo| +3162|AAAAAAAAKFMAAAAA|294|Washington |RD|Suite 360|Pleasant Hill|Dade County|GA|33604|United States|-5|single family| +3163|AAAAAAAALFMAAAAA|535|Lake |Cir.|Suite E|Riverview|Adams County|OH|49003|United States|-5|condo| +3164|AAAAAAAAMFMAAAAA|343|Jackson Mill|Circle|Suite 300|Shady Grove|Tyler County|TX|72812|United States|-6|single family| +3165|AAAAAAAANFMAAAAA|3|Ridge |Court|Suite 100|Lone Oak|Washington County|IL|66893|United States|-6|condo| +3166|AAAAAAAAOFMAAAAA|821|2nd View|RD|Suite 60|Jackson|Scotland County|NC|29583|United States|-5|apartment| +3167|AAAAAAAAPFMAAAAA|97|Pine |Dr.|Suite P|Siloam|Shelby County|KY|48948|United States|-5|apartment| +3168|AAAAAAAAAGMAAAAA|778|Johnson |Circle|Suite G|Wildwood|Anoka County|MN|56871|United States|-6|apartment| +3169|AAAAAAAABGMAAAAA|247|Dogwood |Dr.|Suite 300|Harmony|Santa Cruz County|CA|95804|United States|-8|condo| +3170|AAAAAAAACGMAAAAA|371|Walnut |Parkway|Suite 410|Jamestown|Trousdale County|TN|36867|United States|-6|apartment| +3171|AAAAAAAADGMAAAAA|680|South |Road|Suite T|Riverdale|Roanoke city|VA|29391|United States|-5|apartment| +3172|AAAAAAAAEGMAAAAA|558|7th Broadway|RD|Suite P|Pine Grove|Thomas County|KS|64593|United States|-6|single family| +3173|AAAAAAAAFGMAAAAA|509|Main |Way|Suite N|Harmony|Polk County|MO|65804|United States|-6|single family| +3174|AAAAAAAAGGMAAAAA|616|Pine |Court|Suite F|Montague|Pointe Coupee Parish|LA|74062|United States|-6|condo| +3175|AAAAAAAAHGMAAAAA|65|Wilson Second|Lane|Suite 30|Jamestown|Piatt County|IL|66867|United States|-6|condo| +3176|AAAAAAAAIGMAAAAA|642|Washington |Way|Suite 230|Oak Hill|Pike County|IL|67838|United States|-6|single family| +3177|AAAAAAAAJGMAAAAA|876|Franklin |RD|Suite I|Pleasant Hill|Attala County|MS|53604|United States|-6|condo| +3178|AAAAAAAAKGMAAAAA|618|Mill River|Dr.|Suite G|Sulphur Springs|Polk County|MO|68354|United States|-6|apartment| +3179|AAAAAAAALGMAAAAA|207|Fifth |RD|Suite D|Newtown|Hillsdale County|MI|41749|United States|-5|single family| +3180|AAAAAAAAMGMAAAAA|4|7th Elm|Circle|Suite X|Salem|Upson County|GA|38048|United States|-5|condo| +3181|AAAAAAAANGMAAAAA|909|2nd |Cir.|Suite J|Clifton|Bond County|IL|68014|United States|-6|apartment| +3182|AAAAAAAAOGMAAAAA|252|Main Wilson|Lane|Suite 30|Spring Hill|Barnstable County|MA|07387|United States|-5|single family| +3183|AAAAAAAAPGMAAAAA|427|Lake 13th|Boulevard|Suite I|Lakeside|Brown County|KS|69532|United States|-6|single family| +3184|AAAAAAAAAHMAAAAA|73|Locust Lakeview|Blvd|Suite V|Unionville|Aleutians West Census Area|AK|91711|United States|-9|single family| +3185|AAAAAAAABHMAAAAA|581|Fourth 10th|Way|Suite E|Salem|DeKalb County|IL|68048|United States|-6|single family| +3186|AAAAAAAACHMAAAAA|963|Forest |Dr.|Suite X|Ashland|Preston County|WV|24244|United States|-5|apartment| +3187|AAAAAAAADHMAAAAA|829|Walnut |Lane|Suite 440|Riverview|Eddy County|ND|59003|United States|-6|condo| +3188|AAAAAAAAEHMAAAAA|896|Seventh 5th|Road|Suite 280|Cedar Grove|Esmeralda County|NV|80411|United States|-8|condo| +3189|AAAAAAAAFHMAAAAA|706|Hill Lincoln|Street|Suite E|Forest Hills|Barton County|KS|69237|United States|-6|condo| +3190|AAAAAAAAGHMAAAAA|198|Church |Avenue|Suite 450|Mount Vernon|Kearney County|NE|68482|United States|-7|single family| +3191|AAAAAAAAHHMAAAAA|149|First Cherry|RD|Suite Y|Waterloo|Page County|IA|51675|United States|-6|apartment| +3192|AAAAAAAAIHMAAAAA|849|Cedar |Circle|Suite 110|Buena Vista|Clinton County|PA|15752|United States|-5|apartment| +3193|AAAAAAAAJHMAAAAA|146|Spruce |Ln|Suite Q|Clearview|Brown County|IN|45495|United States|-5|condo| +3194|AAAAAAAAKHMAAAAA|205|2nd |Avenue|Suite P|Ashland|Meade County|KY|44244|United States|-5|apartment| +3195|AAAAAAAALHMAAAAA|483|Willow |Circle|Suite C|Georgetown|Fulton County|AR|77057|United States|-6|condo| +3196|AAAAAAAAMHMAAAAA|378|12th |Wy|Suite 470|Garrison|Sutton County|TX|78767|United States|-6|single family| +3197|AAAAAAAANHMAAAAA||||Suite E|Plainview||GA|||-5|apartment| +3198|AAAAAAAAOHMAAAAA|473|5th Davis|Circle|Suite M|Springdale|Saline County|MO|68883|United States|-6|apartment| +3199|AAAAAAAAPHMAAAAA|827|||Suite S|Farmington|Whitley County|||United States||condo| +3200|AAAAAAAAAIMAAAAA|574|Jackson 3rd|Dr.|Suite K|Wilson|Grady County|GA|36971|United States|-5|apartment| +3201|AAAAAAAABIMAAAAA|187|Main |Road|Suite 440|Concord|Johnson County|IL|64107|United States|-6|condo| +3202|AAAAAAAACIMAAAAA|725|Cedar Meadow|Cir.|Suite 130|Five Points|Lincoln County|NV|86098|United States|-8|apartment| +3203|AAAAAAAADIMAAAAA|460|2nd Walnut|Dr.|Suite Y|Farmington|Kings County|NY|19145|United States|-5|single family| +3204|AAAAAAAAEIMAAAAA|682|Highland |Ln|Suite 170|Woodland|Yellow Medicine County|MN|54854|United States|-6|condo| +3205|AAAAAAAAFIMAAAAA|873|Thirteenth |Court|Suite 420|Maple Grove|Siskiyou County|CA|98252|United States|-8|apartment| +3206|AAAAAAAAGIMAAAAA|296|Oak |Cir.|Suite 220|Flatwoods|Ellis County|KS|64212|United States|-6|single family| +3207|AAAAAAAAHIMAAAAA|465|Ridge 6th|ST|Suite C|Crossroads|Clinton County|IL|60534|United States|-6|condo| +3208|AAAAAAAAIIMAAAAA|873|Hill |Lane|Suite 200|White Oak|Sullivan County|NH|07268|United States|-5|apartment| +3209|AAAAAAAAJIMAAAAA|426|Laurel |Cir.|Suite P|Shady Grove|Gulf County|FL|32812|United States|-5|condo| +3210|AAAAAAAAKIMAAAAA|902|Miller Maple||Suite 100||Lawrence County|||United States|-5|| +3211|AAAAAAAALIMAAAAA|316|3rd |Wy|Suite Y|White Oak|Talladega County|AL|36668|United States|-6|apartment| +3212|AAAAAAAAMIMAAAAA|98|Spruce Pine|ST|Suite 380|Salem|Collier County|FL|38048|United States|-5|apartment| +3213|AAAAAAAANIMAAAAA|913|12th 2nd|Boulevard|Suite E|Oakdale|Galveston County|TX|79584|United States|-6|condo| +3214|AAAAAAAAOIMAAAAA|157|Oak Third|Court|Suite 470|Maple Grove|Valley County|NE|68252|United States|-7|apartment| +3215|AAAAAAAAPIMAAAAA|751|1st Walnut|Cir.|Suite N|Harmony|Fleming County|KY|45804|United States|-6|apartment| +3216|AAAAAAAAAJMAAAAA|823|Oak |Dr.|Suite 420|Cedar Grove|Lynchburg city|VA|20411|United States|-5|apartment| +3217|AAAAAAAABJMAAAAA|318|Hickory |Cir.|Suite K|Union Hill|Chesterfield County|SC|27746|United States|-5|apartment| +3218|AAAAAAAACJMAAAAA|1000|Walnut |Road|Suite 220|Oak Grove|Huron County|OH|48370|United States|-5|apartment| +3219|AAAAAAAADJMAAAAA|383|7th |Court|Suite P|Bethel|Scotts Bluff County|NE|65281|United States|-7|single family| +3220|AAAAAAAAEJMAAAAA|35|Fourth North|Parkway|Suite 50|Riverview|Rains County|TX|79003|United States|-6|single family| +3221|AAAAAAAAFJMAAAAA|545|||Suite 80||Lawrence County|PA||United States|-5|single family| +3222|AAAAAAAAGJMAAAAA|435|3rd |Circle|Suite Q|Glendale|Fulton County|OH|43951|United States|-5|single family| +3223|AAAAAAAAHJMAAAAA|747|Meadow |Way|Suite R|Belmont|Otsego County|MI|40191|United States|-5|condo| +3224|AAAAAAAAIJMAAAAA|714|||Suite P|Highland Park|Natchitoches Parish|LA|76534||-6|single family| +3225|AAAAAAAAJJMAAAAA|20|Lincoln 12th|Dr.|Suite 140|Lakeside|Whitman County|WA|99532|United States|-8|single family| +3226|AAAAAAAAKJMAAAAA|791|3rd |Drive|Suite 240|Edgewood|Muscogee County|GA|30069|United States|-5|single family| +3227|AAAAAAAALJMAAAAA|305|Center Park|Road|Suite U|Oakdale|Lake County|OH|49584|United States|-5|apartment| +3228|AAAAAAAAMJMAAAAA|248|Lee |Road|Suite A|Oakland|Major County|OK|79843|United States|-6|condo| +3229|AAAAAAAANJMAAAAA||9th Hill||||Greene County||76867|United States||condo| +3230|AAAAAAAAOJMAAAAA|918|Lake |Drive|Suite T|Shelby|Lincoln County|GA|36575|United States|-5|condo| +3231|AAAAAAAAPJMAAAAA|706|Lakeview |Ct.|Suite A|Edgewood|Oneida County|NY|10069|United States|-5|condo| +3232|AAAAAAAAAKMAAAAA|369|Wilson |Court|Suite 110|Sulphur Springs|Jim Wells County|TX|78354|United States|-6|condo| +3233|AAAAAAAABKMAAAAA|124|Forest |Parkway|Suite 60|Bethel|Hamilton County|OH|45281|United States|-5|condo| +3234|AAAAAAAACKMAAAAA|318|3rd Williams|Ct.|Suite 60|Wayland|Nacogdoches County|TX|75115|United States|-6|single family| +3235|AAAAAAAADKMAAAAA|964|Forest Hickory|Boulevard|Suite O|Liberty|Presque Isle County|MI|43451|United States|-5|single family| +3236|AAAAAAAAEKMAAAAA|835|Pine Oak|Ln|Suite 380|Fairfield|Roosevelt County|MT|66192|United States|-7|apartment| +3237|AAAAAAAAFKMAAAAA|229|Spring Hickory|Pkwy|Suite 170|Franklin|Millard County|UT|89101|United States|-7|condo| +3238|AAAAAAAAGKMAAAAA|751|Third |Drive|Suite S|Georgetown|Chelan County|WA|97057|United States|-8|single family| +3239|AAAAAAAAHKMAAAAA|42|South |Ct.|Suite 70|New Hope|Bradley County|TN|39431|United States|-5|apartment| +3240|AAAAAAAAIKMAAAAA|249|1st |Way|Suite X|Denver|Tipton County|IN|45547|United States|-5|condo| +3241|AAAAAAAAJKMAAAAA|419|Highland Church|Ave|Suite B|Antioch|Burleson County|TX|78605|United States|-6|apartment| +3242|AAAAAAAAKKMAAAAA|444|Forest |Court|Suite 370|Florence|Lander County|NV|83394|United States|-8|single family| +3243|AAAAAAAALKMAAAAA|734|Woodland Williams|Drive|Suite 330|Fairfield|Fayette County|OH|46192|United States|-5|single family| +3244|AAAAAAAAMKMAAAAA|489|Elm |Street|Suite 340|Summit|Butler County|OH|40499|United States|-5|single family| +3245|AAAAAAAANKMAAAAA|827|Highland North|ST|Suite 10|Five Forks|Clallam County|WA|92293|United States|-8|condo| +3246|AAAAAAAAOKMAAAAA|176|Seventh |Drive|Suite X|Waterloo|Hettinger County|ND|51675|United States|-6|apartment| +3247|AAAAAAAAPKMAAAAA|127|Woodland |Ct.|Suite 340|Oakdale|Waynesboro city|VA|29584|United States|-5|apartment| +3248|AAAAAAAAALMAAAAA|881|Spring 4th|Lane|Suite Y|New Hope|Boyd County|NE|69431|United States|-6|apartment| +3249|AAAAAAAABLMAAAAA|591|Cedar Center|Court|Suite R|Union Hill|Charles County|MD|27746|United States|-5|single family| +3250|AAAAAAAACLMAAAAA|238|Laurel |Ln|Suite R|Pleasant Hill|Furnas County|NE|63604|United States|-6|condo| +3251|AAAAAAAADLMAAAAA|108|Spring Wilson|Ct.|Suite F|Glenwood|Macon County|GA|33511|United States|-5|apartment| +3252|AAAAAAAAELMAAAAA|373|Pine Elm|Parkway|Suite W|Oakdale|Lee County|IL|69584|United States|-6|apartment| +3253|AAAAAAAAFLMAAAAA|614|Maple |ST|Suite R|Stringtown|Bennett County|SD|50162|United States|-6|condo| +3254|AAAAAAAAGLMAAAAA|718|Johnson 12th|Street|Suite 270|Highland Park|Sussex County|DE|16534|United States|-5|single family| +3255|AAAAAAAAHLMAAAAA|913|Smith |Circle|Suite 420|Enterprise|Macon County|GA|31757|United States|-5|apartment| +3256|AAAAAAAAILMAAAAA|991|4th |Avenue|Suite O|Pleasant Grove|Mason County|TX|74136|United States|-6|single family| +3257|AAAAAAAAJLMAAAAA|831|Oak Eigth|Ln|Suite 330|Calhoun|Scott County|IL|66909|United States|-6|single family| +3258|AAAAAAAAKLMAAAAA|698|10th |Pkwy|Suite 430|Sulphur Springs|Garfield County|CO|88354|United States|-7|apartment| +3259|AAAAAAAALLMAAAAA|949|7th 2nd|Pkwy|Suite X|Woodland|San Diego County|CA|94854|United States|-8|condo| +3260|AAAAAAAAMLMAAAAA|295|Meadow |Street|Suite B|Hamilton|Emanuel County|GA|32808|United States|-5|single family| +3261|AAAAAAAANLMAAAAA|742|Railroad |Blvd|Suite L|Centerville|Cheyenne County|CO|80059|United States|-7|single family| +3262|AAAAAAAAOLMAAAAA|622|Main South|Boulevard|Suite A|Mountain View|Jones County|IA|54466|United States|-6|apartment| +3263|AAAAAAAAPLMAAAAA|571|Oak Williams|Wy|Suite 250|Bolton|Houston County|AL|35957|United States|-6|single family| +3264|AAAAAAAAAMMAAAAA|930|3rd |RD|Suite M|Pleasant Hill|New Castle County|DE|13604|United States|-5|single family| +3265|AAAAAAAABMMAAAAA|609||||Five Points||SC|26098|United States||single family| +3266|AAAAAAAACMMAAAAA|825|Broadway 15th|Cir.|Suite 410|Clinton|Fulton County|KY|48222|United States|-6|apartment| +3267|AAAAAAAADMMAAAAA|4|Fourth Cherry|Avenue|Suite 280|Green Acres|Brule County|SD|57683|United States|-6|apartment| +3268|AAAAAAAAEMMAAAAA|311|Smith |RD|Suite 450|Bunker Hill|Wayne County|IL|60150|United States|-6|apartment| +3269|AAAAAAAAFMMAAAAA|549|2nd |Ct.|Suite 390|Plainview|Prince William County|VA|23683|United States|-5|condo| +3270|AAAAAAAAGMMAAAAA|52|10th |Dr.|Suite 470|Sunnyside|Clay County|NE|61952|United States|-6|condo| +3271|AAAAAAAAHMMAAAAA|73|Church Walnut|Cir.|Suite 430|Woodlawn|Tolland County|CT|04698|United States|-5|condo| +3272|AAAAAAAAIMMAAAAA|219|Miller Maple|Avenue|Suite W|Shady Grove|Lee County|NC|22812|United States|-5|single family| +3273|AAAAAAAAJMMAAAAA|314|9th |Circle|Suite 130|Friendship|Neshoba County|MS|54536|United States|-6|condo| +3274|AAAAAAAAKMMAAAAA|903|Woodland Sunset|Drive|Suite 40|Franklin|Kennebec County|ME|09701|United States|-5|condo| +3275|AAAAAAAALMMAAAAA|746|College South|Ave|Suite 10|Riverside|Baldwin County|GA|39231|United States|-5|condo| +3276|AAAAAAAAMMMAAAAA|294|Broadway Oak|Road|Suite 400|Mechanicsburg|O-Brien County|IA|52219|United States|-6|single family| +3277|AAAAAAAANMMAAAAA|645|Wilson |Parkway|Suite 70|Enterprise|Clinton County|KY|41757|United States|-6|apartment| +3278|AAAAAAAAOMMAAAAA|968|Park Church|Court|Suite G|Arlington|Stone County|MO|66557|United States|-6|apartment| +3279|AAAAAAAAPMMAAAAA|890|Valley Highland|Avenue|Suite 110|Harmony|Crane County|TX|75804|United States|-6|single family| +3280|AAAAAAAAANMAAAAA|553|Jackson |Avenue|Suite 250|Williamsville|Green County|KY|48754|United States|-6|condo| +3281|AAAAAAAABNMAAAAA||||Suite O|Centerville|Raleigh County|||United States||| +3282|AAAAAAAACNMAAAAA|662|Center Main|Boulevard|Suite 80|Friendship|Rice County|MN|54536|United States|-6|apartment| +3283|AAAAAAAADNMAAAAA|491|Franklin Green|Way|Suite 190|Walnut Grove|Nelson County|KY|47752|United States|-5|single family| +3284|AAAAAAAAENMAAAAA|954|Cherry Sycamore|RD|Suite 360|Shiloh|Pulaski County|VA|29275|United States|-5|condo| +3285|AAAAAAAAFNMAAAAA|118|Oak |Pkwy|Suite L|Woodlawn|Cheshire County|NH|04698|United States|-5|single family| +3286|AAAAAAAAGNMAAAAA|934|Wilson |Dr.|Suite 380|Woodland Park|Allen County|IN|41934|United States|-5|single family| +3287|AAAAAAAAHNMAAAAA|277|Highland East|ST|Suite 470|West Liberty|Morgan County|MO|64752|United States|-6|condo| +3288|AAAAAAAAINMAAAAA|177|Lincoln |Road|Suite 210|Red Hill|James City County|VA|24338|United States|-5|single family| +3289|AAAAAAAAJNMAAAAA|479|Spruce Sunset|Ct.|||Hartley County|TX||United States||single family| +3290|AAAAAAAAKNMAAAAA|998|Williams North|Pkwy|Suite 270|Lebanon|Conejos County|CO|82898|United States|-7|apartment| +3291|AAAAAAAALNMAAAAA|967|Cedar |Drive|Suite B|Lakeside|Wapello County|IA|59532|United States|-6|condo| +3292|AAAAAAAAMNMAAAAA|867|2nd |Parkway|Suite N|Ellsworth|Perry County|AR|75079|United States|-6|single family| +3293|AAAAAAAANNMAAAAA|273|Lake |Dr.|Suite O|Providence|Mountrail County|ND|56614|United States|-6|condo| +3294|AAAAAAAAONMAAAAA|617|Main |Ct.|Suite G|Shady Grove|Hancock County|WV|22812|United States|-5|single family| +3295|AAAAAAAAPNMAAAAA|74|Church |Pkwy|Suite I|Riverside|Cook County|MN|59231|United States|-6|single family| +3296|AAAAAAAAAOMAAAAA|176|2nd |Ln|Suite 290|Riverview|Northampton County|NC|29003|United States|-5|condo| +3297|AAAAAAAABOMAAAAA|653|Davis Fourth|Ln|Suite O|Hillcrest|Walker County|TX|73003|United States|-6|apartment| +3298|AAAAAAAACOMAAAAA|495|Mill |Ct.|Suite A|Lebanon|Monroe County|IN|42898|United States|-5|condo| +3299|AAAAAAAADOMAAAAA|540|6th |Boulevard|Suite E|Unionville|Rowan County|NC|21711|United States|-5|single family| +3300|AAAAAAAAEOMAAAAA|386|Railroad Smith|Wy|Suite 200|Riverside|Childress County|TX|79231|United States|-6|condo| +3301|AAAAAAAAFOMAAAAA|104|Sunset |Ln|Suite D|Summit|Orange County|FL|30499|United States|-5|apartment| +3302|AAAAAAAAGOMAAAAA|676|10th Main|Court|Suite U|Florence|Kaufman County|TX|73394|United States|-6|condo| +3303|AAAAAAAAHOMAAAAA|702|Ash Jackson|Cir.|Suite R|San Jose|Seminole County|OK|78003|United States|-6|single family| +3304|AAAAAAAAIOMAAAAA|559|North 9th|||Hopewell|Grand Traverse County|MI||||| +3305|AAAAAAAAJOMAAAAA|773|Williams |ST|Suite V|Crossroads|Yellow Medicine County|MN|50534|United States|-6|condo| +3306|AAAAAAAAKOMAAAAA|834|View |Dr.|Suite 100|Harmony|Dauphin County|PA|15804|United States|-5|condo| +3307|AAAAAAAALOMAAAAA|734|East Adams|Drive|Suite S|Mount Pleasant|Wilson County|TN|31933|United States|-5|single family| +3308|AAAAAAAAMOMAAAAA|370|8th |Pkwy|Suite 260|Enterprise|Waller County|TX|71757|United States|-6|condo| +3309|AAAAAAAANOMAAAAA|678|Lee |Road|Suite L|White Oak|Monroe County|MO|66668|United States|-6|single family| +3310|AAAAAAAAOOMAAAAA|659|Johnson Green|Dr.|Suite 240|Oakland|Edwards County|IL|69843|United States|-6|condo| +3311|AAAAAAAAPOMAAAAA|231|View Birch|Drive|Suite F|Oak Hill|Martinsville city|VA|27838|United States|-5|condo| +3312|AAAAAAAAAPMAAAAA|23|1st |Circle|Suite N|Plainview|Schenectady County|NY|13683|United States|-5|single family| +3313|AAAAAAAABPMAAAAA|361|2nd |Way|Suite 50|Florence|Marshall County|KY|43394|United States|-5|single family| +3314|AAAAAAAACPMAAAAA|675|13th Church|Street|Suite T|Oak Grove|Orleans County|VT|08970|United States|-5|condo| +3315|AAAAAAAADPMAAAAA|583|5th Chestnut|Dr.|Suite D|Warwick|Madison County|IN|41398|United States|-5|single family| +3316|AAAAAAAAEPMAAAAA|1|Sunset Davis|Parkway|Suite P|Lone Oak|Winkler County|TX|76893|United States|-6|single family| +3317|AAAAAAAAFPMAAAAA|140|Spruce Hickory|Pkwy|Suite E|Spring Hill|Kenton County|KY|46787|United States|-5|apartment| +3318|AAAAAAAAGPMAAAAA|896|Hill Third|Street|Suite C|Green Acres|Walton County|GA|37683|United States|-5|apartment| +3319|AAAAAAAAHPMAAAAA|22|Sunset |Boulevard|Suite Y|Ashland|Logan County|KY|44244|United States|-5|apartment| +3320|AAAAAAAAIPMAAAAA|687|Church Lake|Parkway|Suite E|Summit|Leflore County|MS|50499|United States|-6|single family| +3321|AAAAAAAAJPMAAAAA|882|Church |Cir.|Suite 250|Friendship|Cherokee County|SC|24536|United States|-5|apartment| +3322|AAAAAAAAKPMAAAAA|754|Hillcrest Chestnut|Dr.|Suite 0|Georgetown|Lee County|TX|77057|United States|-6|condo| +3323|AAAAAAAALPMAAAAA|877|Washington |RD|Suite 400|Lawrenceville|Westchester County|NY|14462|United States|-5|single family| +3324|AAAAAAAAMPMAAAAA|357|Fourth |Drive|Suite 340|Bridgeport|McLean County|ND|55817|United States|-6|single family| +3325|AAAAAAAANPMAAAAA|13|Mill |Ave|Suite 220|Midway|Galax city|VA|21904|United States|-5|single family| +3326|AAAAAAAAOPMAAAAA|789|4th |Pkwy|Suite 70|Belmont|Lincoln County|SD|50191|United States|-7|condo| +3327|AAAAAAAAPPMAAAAA|336|Park Cedar|Boulevard|Suite 130|Ferguson|Riley County|KS|61821|United States|-6|condo| +3328|AAAAAAAAAANAAAAA|730|West Lakeview|Drive|Suite 320|Riverdale|Kingfisher County|OK|79391|United States|-6|apartment| +3329|AAAAAAAABANAAAAA|170|Washington |Circle|Suite 480|Highland Park|Tallahatchie County|MS|56534|United States|-6|single family| +3330|AAAAAAAACANAAAAA|363|Jefferson |Cir.|Suite P|Brownsville|Boyd County|KY|49310|United States|-6|single family| +3331|AAAAAAAADANAAAAA|216|Madison |Circle|Suite 50|Oakwood|Wayne County|WV|20169|United States|-5|single family| +3332|AAAAAAAAEANAAAAA|295|Spring |Street|Suite 250|Springfield|Winston County|AL|39303|United States|-6|single family| +3333|AAAAAAAAFANAAAAA||Ridge Willow|Avenue|||Benton County|||United States|-6|condo| +3334|AAAAAAAAGANAAAAA|44|Ridge 3rd|Way|Suite 20|Oak Hill|Polk County|OR|97838|United States|-8|apartment| +3335|AAAAAAAAHANAAAAA|102|Oak Lake|Ct.|Suite V|Centerville|Leavenworth County|KS|60059|United States|-6|apartment| +3336|AAAAAAAAIANAAAAA|375|3rd Tenth|Boulevard|Suite B|Wildwood|Albany County|NY|16871|United States|-5|apartment| +3337|AAAAAAAAJANAAAAA|192|Lincoln Franklin|ST|Suite 340|The Meadows|York County|ME|00626|United States|-5|condo| +3338|AAAAAAAAKANAAAAA|329|Fifth View|Avenue|Suite O|Walnut Grove|Webb County|TX|77752|United States|-6|apartment| +3339|AAAAAAAALANAAAAA|317|Cedar Maple|Blvd|Suite 50|Forest Hills|Anderson County|KS|69237|United States|-6|single family| +3340|AAAAAAAAMANAAAAA|7|Ridge 2nd|Road|Suite 490|Glendale|Jenkins County|GA|33951|United States|-5|condo| +3341|AAAAAAAANANAAAAA|453|Main |Road|Suite T|Unionville|Cortland County|NY|11711|United States|-5|single family| +3342|AAAAAAAAOANAAAAA|159|Jackson 2nd|Blvd|Suite 210|Shady Grove|Jefferson Davis Parish|LA|72812|United States|-6|condo| +3343|AAAAAAAAPANAAAAA|913|River Smith|Ln|Suite U|Greenville|Hamilton County|KS|61387|United States|-6|single family| +3344|AAAAAAAAABNAAAAA|196|Hill |Lane|Suite W|Riverdale|Posey County|IN|49391|United States|-5|condo| +3345|AAAAAAAABBNAAAAA|18|Lincoln 8th|ST|Suite 30|Oak Hill|Middlesex County|NJ|08438|United States|-5|apartment| +3346|AAAAAAAACBNAAAAA|77|Adams |Court|Suite G|New Boston|Sumter County|GA|34484|United States|-5|single family| +3347|AAAAAAAADBNAAAAA|97|3rd Park|ST|Suite 160|Woodville|Kingfisher County|OK|74289|United States|-6|condo| +3348|AAAAAAAAEBNAAAAA|872|Green Park|Ln|Suite 20|Green Acres|Upshur County|WV|27683|United States|-5|apartment| +3349|AAAAAAAAFBNAAAAA|919|West Chestnut|Court|Suite X|Fairview|Oneida County|WI|55709|United States|-6|apartment| +3350|AAAAAAAAGBNAAAAA|966|Sixth Madison|Avenue|Suite 290|Hamilton|Seward County|KS|62808|United States|-6|single family| +3351|AAAAAAAAHBNAAAAA|2|Park |Ln|Suite 390|Woodlawn|Durham County|NC|24098|United States|-5|apartment| +3352|AAAAAAAAIBNAAAAA|575||||Winslow|Tooele County||88525|United States||single family| +3353|AAAAAAAAJBNAAAAA|30|Willow |Wy|Suite 230|Farmington|Crisp County|GA|39145|United States|-5|single family| +3354|AAAAAAAAKBNAAAAA|885|5th Lincoln|Ln|Suite C|Highland|Ripley County|IN|49454|United States|-5|apartment| +3355|AAAAAAAALBNAAAAA|240|First |Wy|Suite 330|Woodlawn|Sangamon County|IL|64098|United States|-6|single family| +3356|AAAAAAAAMBNAAAAA|709|Second |Street|Suite 440|Westminster|DeKalb County|GA|36549|United States|-5|condo| +3357|AAAAAAAANBNAAAAA|811|Birch Tenth|Lane|Suite A|Brookwood|San Juan County|UT|80965|United States|-7|condo| +3358|AAAAAAAAOBNAAAAA|958|Hill Third|Court|Suite S|Unionville|Clark County|KY|41711|United States|-6|single family| +3359|AAAAAAAAPBNAAAAA|18|Sunset 1st|Blvd|Suite R|Silver Creek|Tripp County|SD|54546|United States|-7|single family| +3360|AAAAAAAAACNAAAAA|855|Mill |Avenue|Suite A|Mount Pleasant|Cumberland County|NJ|02533|United States|-5|condo| +3361|AAAAAAAABCNAAAAA|873|Central |Street|Suite B|Georgetown|Branch County|MI|47057|United States|-5|condo| +3362|AAAAAAAACCNAAAAA|||Court|Suite 90||Jim Hogg County||75752|||apartment| +3363|AAAAAAAADCNAAAAA|||Street|Suite D||Wayne County||21675|United States||condo| +3364|AAAAAAAAECNAAAAA|649|7th |ST|Suite H|Riverdale|Forsyth County|NC|29391|United States|-5|apartment| +3365|AAAAAAAAFCNAAAAA|64|View |Way|Suite 220|Lebanon|Clarion County|PA|12898|United States|-5|condo| +3366|AAAAAAAAGCNAAAAA|807|Hill |Street|Suite B|Forest Hills|Appomattox County|VA|29237|United States|-5|single family| +3367|AAAAAAAAHCNAAAAA|596|10th Main|Wy|Suite 340|Unionville|Milam County|TX|71711|United States|-6|apartment| +3368|AAAAAAAAICNAAAAA|851|River |Dr.|Suite B|Five Points|Stutsman County|ND|56098|United States|-6|apartment| +3369|AAAAAAAAJCNAAAAA|123|Park |Ln|Suite 310|Oak Hill|Pickaway County|OH|47838|United States|-5|single family| +3370|AAAAAAAAKCNAAAAA|213|Eigth |ST|Suite N|Florence|Sequatchie County|TN|33394|United States|-6|condo| +3371|AAAAAAAALCNAAAAA|311|Elm |ST|Suite P|Jamestown|Galax city|VA|26867|United States|-5|condo| +3372|AAAAAAAAMCNAAAAA|450|Center Railroad|Street|Suite 120|Adrian|Steuben County|IN|43301|United States|-5|single family| +3373|AAAAAAAANCNAAAAA|197|South |Ln|Suite 90|Antioch|Cayuga County|NY|18605|United States|-5|single family| +3374|AAAAAAAAOCNAAAAA|343|Highland Jackson|Cir.|Suite J|Georgetown|Sevier County|AR|77057|United States|-6|apartment| +3375|AAAAAAAAPCNAAAAA|52|College View|Parkway|Suite R|Lincoln|Deuel County|NE|61289|United States|-6|single family| +3376|AAAAAAAAADNAAAAA|187|Lakeview |Pkwy|Suite 210|Carthage|Butler County|KY|41529|United States|-6|condo| +3377|AAAAAAAABDNAAAAA|974|North |Street|Suite P|Oak Grove|Boone County|KY|48370|United States|-6|condo| +3378|AAAAAAAACDNAAAAA|533|Park |Avenue|Suite O|Hillcrest|Montrose County|CO|83003|United States|-7|single family| +3379|AAAAAAAADDNAAAAA|501|6th |Road|Suite X|Salem|Jasper County|MO|68048|United States|-6|apartment| +3380|AAAAAAAAEDNAAAAA|23|9th Railroad|Court|Suite 10|Springdale|Appanoose County|IA|58883|United States|-6|condo| +3381|AAAAAAAAFDNAAAAA|284|Green |Court|Suite 160|Union|Jefferson County|TX|78721|United States|-6|condo| +3382|AAAAAAAAGDNAAAAA|550|5th Spring|Road|Suite 400|Stratford|Brazoria County|TX|76668|United States|-6|apartment| +3383|AAAAAAAAHDNAAAAA|671|Broadway 4th|Blvd|Suite 330|Union|Sussex County|VA|28721|United States|-5|condo| +3384|AAAAAAAAIDNAAAAA|379|3rd |Way|Suite M|Five Forks|Howard County|IN|42293|United States|-5|condo| +3385|AAAAAAAAJDNAAAAA|669|Green |Cir.|Suite Q|Red Hill|Strafford County|NH|04938|United States|-5|single family| +3386|AAAAAAAAKDNAAAAA|419|Spring Fourth|Ln|Suite Y|Woodlawn|Williams County|ND|54098|United States|-6|condo| +3387|AAAAAAAALDNAAAAA|986|Spruce |Pkwy|Suite 420|White Oak|Titus County|TX|76668|United States|-6|apartment| +3388|AAAAAAAAMDNAAAAA|178|Cherry |Cir.|Suite 380|Valley View|Door County|WI|55124|United States|-6|single family| +3389|AAAAAAAANDNAAAAA|188|9th |Blvd|Suite 340|Summit|Dawson County|GA|30499|United States|-5|apartment| +3390|AAAAAAAAODNAAAAA|555|3rd |Lane|Suite T|Newtown|Adams County|IN|41749|United States|-5|single family| +3391|AAAAAAAAPDNAAAAA|280|Jackson Jefferson|Road|Suite R|Jamestown|Park County|WY|86867|United States|-7|apartment| +3392|AAAAAAAAAENAAAAA|883|14th Forest|Street|Suite 140|Lakeview|Reynolds County|MO|68579|United States|-6|apartment| +3393|AAAAAAAABENAAAAA|432|Wilson Highland|Road|Suite W|Highland Park|Ogemaw County|MI|46534|United States|-5|single family| +3394|AAAAAAAACENAAAAA|938|Mill |RD|Suite 0|Wildwood|Calhoun County|SC|26871|United States|-5|apartment| +3395|AAAAAAAADENAAAAA|203|Hillcrest |Circle|Suite 330|Shiloh|Humboldt County|IA|59275|United States|-6|single family| +3396|AAAAAAAAEENAAAAA|865|Locust Third|ST|Suite 90|Mount Pleasant|Bourbon County|KY|41933|United States|-6|single family| +3397|AAAAAAAAFENAAAAA|982|1st |Court|Suite B|Bridgeport|Sanborn County|SD|55817|United States|-7|condo| +3398|AAAAAAAAGENAAAAA|294|Forest 2nd|Drive|Suite U|Walnut Grove|Tazewell County|IL|67752|United States|-6|single family| +3399|AAAAAAAAHENAAAAA|734|Tenth Ash|Avenue|Suite Q|Pleasant Grove|Cheboygan County|MI|44136|United States|-5|single family| +3400|AAAAAAAAIENAAAAA|892|Spring East|Ln|Suite 0|Salem|Jefferson County|IN|48048|United States|-5|single family| +3401|AAAAAAAAJENAAAAA|11|4th Second|Dr.|Suite 250|Maple Hill|Shelby County|OH|48095|United States|-5|single family| +3402|AAAAAAAAKENAAAAA|479|Adams 9th|Parkway|Suite 340|Crossroads|Wyoming County|NY|10534|United States|-5|single family| +3403|AAAAAAAALENAAAAA|||||Fairview|Paulding County|||United States|-5|| +3404|AAAAAAAAMENAAAAA|880|Jackson |Circle|Suite 160|Rocky Point|Washington County|OH|41209|United States|-5|apartment| +3405|AAAAAAAANENAAAAA|283|View |Ave|Suite B|Greenwood|Haskell County|OK|78828|United States|-6|condo| +3406|AAAAAAAAOENAAAAA|672|4th Lake|ST|Suite 180|Oak Hill|Clark County|IN|47838|United States|-5|single family| +3407|AAAAAAAAPENAAAAA|530|Davis North|Blvd|Suite 130|Summit|Miner County|SD|50499|United States|-7|single family| +3408|AAAAAAAAAFNAAAAA|643|3rd |Cir.|Suite S|Five Points|Steuben County|IN|46098|United States|-5|apartment| +3409|AAAAAAAABFNAAAAA|992|4th |Ct.|Suite 20|Oakland|Washington County|ID|89843|United States|-7|apartment| +3410|AAAAAAAACFNAAAAA|583|Jefferson |Drive|Suite 180|Hardy|Lonoke County|AR|75354|United States|-6|single family| +3411|AAAAAAAADFNAAAAA|376||Way|Suite 440|Forest Hills||WA||United States||single family| +3412|AAAAAAAAEFNAAAAA|291|7th South|RD|Suite R|Marion|Pleasants County|WV|20399|United States|-5|single family| +3413|AAAAAAAAFFNAAAAA|349|Elm 7th|Court|Suite D|Oakland|Letcher County|KY|49843|United States|-5|single family| +3414|AAAAAAAAGFNAAAAA|767|Fifth |Ct.|Suite 200|Glendale|Bear Lake County|ID|83951|United States|-7|single family| +3415|AAAAAAAAHFNAAAAA|254|Third 3rd|Wy|Suite F|Belmont|Morgan County|GA|30191|United States|-5|condo| +3416|AAAAAAAAIFNAAAAA|484|3rd |Way|Suite 370|Mount Pleasant|Stutsman County|ND|51933|United States|-6|single family| +3417|AAAAAAAAJFNAAAAA|231|Cedar Cedar|Pkwy|Suite 90|Newport|Schoolcraft County|MI|41521|United States|-5|apartment| +3418|AAAAAAAAKFNAAAAA|115|Sunset |Street|Suite B|Friendship|Taylor County|WI|54536|United States|-6|apartment| +3419|AAAAAAAALFNAAAAA|617|Third |Street|Suite 10|Kingston|Cottle County|TX|74975|United States|-6|condo| +3420|AAAAAAAAMFNAAAAA|437|Lake Cedar|Street|Suite E|Westminster|Coffee County|AL|36549|United States|-6|single family| +3421|AAAAAAAANFNAAAAA|405|Hickory Willow|Cir.|Suite 290|The Meadows|Mahaska County|IA|50026|United States|-6|apartment| +3422|AAAAAAAAOFNAAAAA|58|1st |RD|Suite N|Fairview|Collier County|FL|35709|United States|-5|condo| +3423|AAAAAAAAPFNAAAAA|370|Pine Birch|Ct.|Suite C|Concord|Mills County|IA|54107|United States|-6|single family| +3424|AAAAAAAAAGNAAAAA|956|7th |Way|Suite 260|Plainview|Linn County|IA|53683|United States|-6|single family| +3425|AAAAAAAABGNAAAAA|677|Fifth Wilson|ST|Suite R|Clearview|Dukes County|MA|06095|United States|-5|single family| +3426|AAAAAAAACGNAAAAA|897|Sixth Hillcrest|ST|Suite 120|Ashland|Clay County|NC|24244|United States|-5|apartment| +3427|AAAAAAAADGNAAAAA|202|Lincoln |Way|Suite I|Greenwood|Wallowa County|OR|98828|United States|-8|condo| +3428|AAAAAAAAEGNAAAAA|94|Forest |Parkway|Suite G|Jackson|Siskiyou County|CA|99583|United States|-8|single family| +3429|AAAAAAAAFGNAAAAA|788|3rd |Dr.|Suite M|Whitney|Knox County|TX|78339|United States|-6|condo| +3430|AAAAAAAAGGNAAAAA|154|Willow Highland|Blvd|Suite I|Bunker Hill|Lawrence County|IN|40150|United States|-5|single family| +3431|AAAAAAAAHGNAAAAA|899|Oak Hill|Boulevard|Suite 200|Providence|Boone County|IN|46614|United States|-5|condo| +3432|AAAAAAAAIGNAAAAA|421|9th River|Wy|Suite 60|Forest Hills|Franklin County|KY|49237|United States|-6|condo| +3433|AAAAAAAAJGNAAAAA|261|Elevnth |Way|Suite 470|Shady Grove|Mohave County|AZ|82812|United States|-7|apartment| +3434|AAAAAAAAKGNAAAAA|450|Forest |Street|Suite V|Five Forks|Dallas County|IA|52293|United States|-6|single family| +3435|AAAAAAAALGNAAAAA|395|Lake |Parkway|Suite J|Mount Olive|Spencer County|IN|48059|United States|-5|single family| +3436|AAAAAAAAMGNAAAAA|641|Lincoln Park|Avenue|Suite 450|Brownsville|Henry County|AL|39310|United States|-6|apartment| +3437|AAAAAAAANGNAAAAA||Church Elm||Suite 130||Jones County||56557|United States||single family| +3438|AAAAAAAAOGNAAAAA|57|View |Lane|Suite 400|Oak Ridge|Cullman County|AL|38371|United States|-6|condo| +3439|AAAAAAAAPGNAAAAA|765|Lee Valley|Court|Suite 300|New Hope|Brown County|MN|59431|United States|-6|single family| +3440|AAAAAAAAAHNAAAAA|429|Mill First|Way|Suite 40|Macedonia|Aitkin County|MN|51087|United States|-6|condo| +3441|AAAAAAAABHNAAAAA|355|Maple |Court|Suite 200|Salem|Crittenden County|AR|78048|United States|-6|condo| +3442|AAAAAAAACHNAAAAA|809|Hill |Street|Suite 80|Fairfield|San Juan County|WA|96192|United States|-8|condo| +3443|AAAAAAAADHNAAAAA|648|South |Street|Suite N|Bridgeport|Bartow County|GA|35817|United States|-5|condo| +3444|AAAAAAAAEHNAAAAA|150|Pine |Blvd|Suite C|Five Forks|Cooper County|MO|62293|United States|-6|condo| +3445|AAAAAAAAFHNAAAAA|259|Maple |ST|Suite U|Lakewood|Hampshire County|MA|09477|United States|-5|apartment| +3446|AAAAAAAAGHNAAAAA|957|Laurel 11th|Boulevard|Suite K|Greenfield|Curry County|NM|85038|United States|-7|apartment| +3447|AAAAAAAAHHNAAAAA|465|4th Cherry|Ln|Suite L|Woodlawn|McPherson County|NE|64098|United States|-7|condo| +3448|AAAAAAAAIHNAAAAA|455|Jackson Hillcrest|Blvd|Suite 110|Crossroads|Arenac County|MI|40534|United States|-5|condo| +3449|AAAAAAAAJHNAAAAA|714|Pine |Way|Suite 120|Newport|Meade County|SD|51521|United States|-7|single family| +3450|AAAAAAAAKHNAAAAA|80|North |Wy|Suite P|Oakwood|Richland County|IL|60169|United States|-6|condo| +3451|AAAAAAAALHNAAAAA|610|14th Jackson|Drive|Suite 410|Newport|Iroquois County|IL|61521|United States|-6|condo| +3452|AAAAAAAAMHNAAAAA|5|4th |Pkwy|Suite J|Mount Vernon|Aitkin County|MN|58482|United States|-6|single family| +3453|AAAAAAAANHNAAAAA|814|1st |ST|Suite 300|Sulphur Springs|Bailey County|TX|78354|United States|-6|single family| +3454|AAAAAAAAOHNAAAAA||Hill |Blvd||||||United States|-5|apartment| +3455|AAAAAAAAPHNAAAAA|583||Dr.|Suite B||Marshall County|KY|40913||-5|| +3456|AAAAAAAAAINAAAAA|635|13th |Parkway|Suite 370|Stringtown|Hampden County|MA|00762|United States|-5|apartment| +3457|AAAAAAAABINAAAAA|804|Hickory |Lane|Suite 170|Flint Hill|Collin County|TX|77934|United States|-6|apartment| +3458|AAAAAAAACINAAAAA|819|Elm |Parkway|Suite H|Edgewood|Clallam County|WA|90069|United States|-8|apartment| +3459|AAAAAAAADINAAAAA|503|Sixth |Boulevard|Suite 90|Post Oak|Suwannee County|FL|38567|United States|-5|condo| +3460|AAAAAAAAEINAAAAA|464|Walnut |Dr.|Suite 240|Pine Grove|Minidoka County|ID|84593|United States|-7|apartment| +3461|AAAAAAAAFINAAAAA|203|Walnut |Boulevard|Suite E|Mount Olive|Laramie County|WY|88059|United States|-7|single family| +3462|AAAAAAAAGINAAAAA|306|Tenth |Blvd|Suite 460|Kingston|Brown County|OH|44975|United States|-5|single family| +3463|AAAAAAAAHINAAAAA|42|Lee North|Dr.|Suite Y|Harvey|Scott County|IL|65858|United States|-6|condo| +3464|AAAAAAAAIINAAAAA|939|North 1st|RD|Suite 450|Springfield|Bibb County|AL|39303|United States|-6|apartment| +3465|AAAAAAAAJINAAAAA|277|Church |Road|Suite 90|Sawyer|Nicollet County|MN|56045|United States|-6|apartment| +3466|AAAAAAAAKINAAAAA|59|Ash |ST|Suite A|Ashland|Pecos County|TX|74244|United States|-6|apartment| +3467|AAAAAAAALINAAAAA|986|1st Jefferson|Way|Suite 310|Mount Olive|Rapides Parish|LA|78059|United States|-6|condo| +3468|AAAAAAAAMINAAAAA|336|Lee 5th|Ave|Suite 20|Union Hill|Coryell County|TX|77746|United States|-6|condo| +3469|AAAAAAAANINAAAAA|614|Davis |Way|Suite M|Fairview|Kenosha County|WI|55709|United States|-6|condo| +3470|AAAAAAAAOINAAAAA|517|Adams |Street|Suite 80|Ashland|Bristol County|RI|04844|United States|-5|condo| +3471|AAAAAAAAPINAAAAA|510|Jefferson |Ct.|Suite U|Lincoln|North Slope Borough|AK|91289|United States|-9|single family| +3472|AAAAAAAAAJNAAAAA|884|Broadway Cherry|Wy|Suite 210|Florence|Randall County|TX|73394|United States|-6|condo| +3473|AAAAAAAABJNAAAAA|171|Smith |ST|Suite 480|Lakeside|Chester County|PA|19532|United States|-5|single family| +3474|AAAAAAAACJNAAAAA|974|View 1st|Drive|Suite Y|Saint Johns|Grays Harbor County|WA|95717|United States|-8|apartment| +3475|AAAAAAAADJNAAAAA|632|Center Third|Cir.|Suite J|Lakeside|Cherokee County|OK|79532|United States|-6|condo| +3476|AAAAAAAAEJNAAAAA|834|Oak |Street|Suite U|Hamilton|Logan County|WV|22808|United States|-5|condo| +3477|AAAAAAAAFJNAAAAA|482|Washington Third|Way|Suite C|Red Hill|Ashley County|AR|74338|United States|-6|condo| +3478|AAAAAAAAGJNAAAAA|644|1st Oak|Drive|Suite K|Forest Hills|Austin County|TX|79237|United States|-6|apartment| +3479|AAAAAAAAHJNAAAAA|412|Johnson Lake|Lane|Suite 80|Waterloo|San Saba County|TX|71675|United States|-6|condo| +3480|AAAAAAAAIJNAAAAA|699|Jefferson |Way|Suite F|Clinton|Mower County|MN|58222|United States|-6|condo| +3481|AAAAAAAAJJNAAAAA|126|Meadow |Dr.|Suite B|Jerome|Stark County|OH|49920|United States|-5|apartment| +3482|AAAAAAAAKJNAAAAA|276|6th Miller|Lane|Suite 220|Lebanon|Owsley County|KY|42898|United States|-5|single family| +3483|AAAAAAAALJNAAAAA|137|Second |Boulevard|Suite 210|Summit|Hoke County|NC|20499|United States|-5|apartment| +3484|AAAAAAAAMJNAAAAA|234|Cedar 6th|Way|Suite Y|Highland Park|Allamakee County|IA|56534|United States|-6|single family| +3485|AAAAAAAANJNAAAAA|466|Forest |Lane|Suite 410|Summit|Harding County|SD|50499|United States|-7|single family| +3486|AAAAAAAAOJNAAAAA|173|South Forest|Road|Suite 100|Pleasant Hill|Jackson County|SD|53604|United States|-7|apartment| +3487|AAAAAAAAPJNAAAAA|646|Adams Meadow|Drive|Suite 230|Lakewood|Kent County|MD|28877|United States|-5|apartment| +3488|AAAAAAAAAKNAAAAA|3|Pine Twelfth|Cir.|Suite 30|Union Hill|Morris County|NJ|08346|United States|-5|condo| +3489|AAAAAAAABKNAAAAA|911|South |Ct.|Suite X|Hopewell|Franklin County|KS|60587|United States|-6|single family| +3490|AAAAAAAACKNAAAAA|||Court|Suite X|Highland|Sierra County|CA||United States|-8|single family| +3491|AAAAAAAADKNAAAAA|282|Madison Ridge|Wy|Suite L|Oak Grove|Faulkner County|AR|78370|United States|-6|single family| +3492|AAAAAAAAEKNAAAAA|175|Lakeview 13th|Ct.|Suite F|Wildwood|Ford County|KS|66871|United States|-6|condo| +3493|AAAAAAAAFKNAAAAA|669|Fourth |Parkway|Suite 380|Georgetown|Johnson County|AR|77057|United States|-6|single family| +3494|AAAAAAAAGKNAAAAA|415|Eigth |Wy|Suite B|Farmington|Patrick County|VA|29145|United States|-5|apartment| +3495|AAAAAAAAHKNAAAAA|645|Johnson |Pkwy|Suite H|Mount Olive|Lapeer County|MI|48059|United States|-5|condo| +3496|AAAAAAAAIKNAAAAA|413|Main |Lane|Suite 210|Green Acres|Maury County|TN|37683|United States|-6|condo| +3497|AAAAAAAAJKNAAAAA|165|Elm |Cir.|Suite I|Oakwood|Floyd County|TX|70169|United States|-6|apartment| +3498|AAAAAAAAKKNAAAAA|870|Cedar Adams|Avenue|Suite T|Riverside|Itasca County|MN|59231|United States|-6|single family| +3499|AAAAAAAALKNAAAAA|894|Poplar Adams|Pkwy|Suite O|Bethany|Early County|GA|35460|United States|-5|apartment| +3500|AAAAAAAAMKNAAAAA|596|Poplar Pine|RD|Suite T|Unionville|Decatur County|GA|31711|United States|-5|single family| +3501|AAAAAAAANKNAAAAA|506|Second Fourth|Road|Suite 20|Springfield|Larimer County|CO|89303|United States|-7|single family| +3502|AAAAAAAAOKNAAAAA|233|Sunset Main|Avenue|Suite U|Superior|Franklin County|WA|92562|United States|-8|single family| +3503|AAAAAAAAPKNAAAAA|654|Park Chestnut|ST|Suite S|Bunker Hill|Ripley County|MO|60150|United States|-6|single family| +3504|AAAAAAAAALNAAAAA|34|Oak |Circle|Suite 320|Millbrook|Prince Edward County|VA|27529|United States|-5|apartment| +3505|AAAAAAAABLNAAAAA|605|Jackson |Road|Suite 410|Franklin|Citrus County|FL|39101|United States|-5|single family| +3506|AAAAAAAACLNAAAAA|288|Birch |RD|Suite U|Oakland|Madison County|ID|89843|United States|-7|single family| +3507|AAAAAAAADLNAAAAA|461|Center Second|Pkwy|Suite 310|Oakdale|Champaign County|OH|49584|United States|-5|single family| +3508|AAAAAAAAELNAAAAA|760|Fifth Hill|Wy|Suite 0|Forest Hills|Wahkiakum County|WA|99237|United States|-8|apartment| +3509|AAAAAAAAFLNAAAAA|417|East 4th|Wy|Suite 310|Mount Zion|Oliver County|ND|58054|United States|-6|single family| +3510|AAAAAAAAGLNAAAAA|925|7th 13th|Road|Suite P|Glenwood|Staunton city|VA|23511|United States|-5|single family| +3511|AAAAAAAAHLNAAAAA|862|9th Jackson|ST|Suite 10|Waterloo|Whitley County|IN|41675|United States|-5|condo| +3512|AAAAAAAAILNAAAAA|444|Hillcrest North|Ct.|Suite 440|Centerville|Montgomery County|IA|50059|United States|-6|apartment| +3513|AAAAAAAAJLNAAAAA|904|3rd Highland|Parkway|Suite X|Lincoln|Gloucester County|NJ|01889|United States|-5|condo| +3514|AAAAAAAAKLNAAAAA|824|Oak |Court|Suite L|Brentwood|Mills County|IA|54188|United States|-6|apartment| +3515|AAAAAAAALLNAAAAA|102|Spruce Fourth|Cir.|Suite 460|Jamestown|Larue County|KY|46867|United States|-5|condo| +3516|AAAAAAAAMLNAAAAA|236|Main |Dr.|Suite G|Walnut Grove|Montgomery County|TX|77752|United States|-6|single family| +3517|AAAAAAAANLNAAAAA|525|Park Maple|Ct.|Suite 370|Forest Hills|Yankton County|SD|59237|United States|-6|condo| +3518|AAAAAAAAOLNAAAAA|471|View Pine|Road|Suite 390|Stewart|Dallas County|IA|58041|United States|-6|single family| +3519|AAAAAAAAPLNAAAAA|726|Main Walnut|ST|Suite 290|Antioch|Harmon County|OK|78605|United States|-6|apartment| +3520|AAAAAAAAAMNAAAAA|607|Park Lake|Dr.|Suite V|Adrian|Sabine County|TX|73301|United States|-6|condo| +3521|AAAAAAAABMNAAAAA|956|Lake Church|RD|Suite K|Springfield|Lackawanna County|PA|19303|United States|-5|single family| +3522|AAAAAAAACMNAAAAA|203|Railroad |Drive|Suite 440|Clinton|Karnes County|TX|78222|United States|-6|apartment| +3523|AAAAAAAADMNAAAAA|653|Jefferson 3rd|Parkway|Suite 110|Roy|Thayer County|NE|60744|United States|-7|apartment| +3524|AAAAAAAAEMNAAAAA|798|College Spruce|Court|Suite B|Red Hill|Allen County|KS|64338|United States|-6|single family| +3525|AAAAAAAAFMNAAAAA|292|Lee Park|Road|Suite G|Carthage|Kanawha County|WV|21529|United States|-5|condo| +3526|AAAAAAAAGMNAAAAA|675|Church 7th|ST|Suite 70|Hillcrest|Pope County|IL|63003|United States|-6|condo| +3527|AAAAAAAAHMNAAAAA|716|Mill 1st|Way|Suite 260|Green Acres|Hopewell city|VA|27683|United States|-5|single family| +3528|AAAAAAAAIMNAAAAA|33|4th |Drive|Suite K|Greenwood|Comanche County|OK|78828|United States|-6|single family| +3529|AAAAAAAAJMNAAAAA|312|Adams |RD|Suite 90|Edgewood|Manassas city|VA|20069|United States|-5|single family| +3530|AAAAAAAAKMNAAAAA|904|14th |Lane|Suite 490|Oneida|Dougherty County|GA|34027|United States|-5|apartment| +3531|AAAAAAAALMNAAAAA|832|First Wilson|Cir.|Suite B|Riverview|Saratoga County|NY|19003|United States|-5|condo| +3532|AAAAAAAAMMNAAAAA|467|Jefferson |RD|Suite X|Mount Vernon|Meagher County|MT|68482|United States|-7|single family| +3533|AAAAAAAANMNAAAAA|598|Eigth Main|Drive|Suite 470|Greenwood|Tate County|MS|58828|United States|-6|apartment| +3534|AAAAAAAAOMNAAAAA||Dogwood |Blvd|Suite A||Clackamas County|OR|91087||-8|apartment| +3535|AAAAAAAAPMNAAAAA|641|Madison |Ct.|Suite T|Green Acres|Campbell County|TN|37683|United States|-5|condo| +3536|AAAAAAAAANNAAAAA|325|Lakeview |Drive|Suite X|Providence|Webster County|KY|46614|United States|-5|condo| +3537|AAAAAAAABNNAAAAA|39|Lakeview |Wy|Suite 380|Bethel|Hendry County|FL|35281|United States|-5|condo| +3538|AAAAAAAACNNAAAAA|103|Johnson |Circle|Suite 290|Lincoln|Henry County|GA|31289|United States|-5|single family| +3539|AAAAAAAADNNAAAAA|545|Washington Sunset|Boulevard|Suite C|Five Points|McKean County|PA|16098|United States|-5|condo| +3540|AAAAAAAAENNAAAAA|827|7th Dogwood|Circle|Suite 190|Concord|Lincoln County|CO|84107|United States|-7|condo| +3541|AAAAAAAAFNNAAAAA|654|Ridge 10th|Parkway|Suite T|Jackson|Cowlitz County|WA|99583|United States|-8|apartment| +3542|AAAAAAAAGNNAAAAA|236|5th Lake|Blvd|Suite 250|Sulphur Springs|Franklin County|KS|68354|United States|-6|condo| +3543|AAAAAAAAHNNAAAAA|561|12th 1st|Dr.|Suite G|Brownsville|Walthall County|MS|59310|United States|-6|single family| +3544|AAAAAAAAINNAAAAA|492|View Oak|Avenue|Suite C|Walnut Grove|Houston County|GA|37752|United States|-5|apartment| +3545|AAAAAAAAJNNAAAAA|19|Ridge |Lane|Suite 390|Belmont|Kauai County|HI|90191|United States|-10|condo| +3546|AAAAAAAAKNNAAAAA|511|Hill Jackson|Avenue|Suite 220|Shady Grove|Orange County|IN|42812|United States|-5|single family| +3547|AAAAAAAALNNAAAAA|981|Park |Ln|Suite U|Newport|DeKalb County|MO|61521|United States|-6|condo| +3548|AAAAAAAAMNNAAAAA|709|Sycamore |Blvd|Suite 120|White Oak|Alleghany County|VA|26668|United States|-5|single family| +3549|AAAAAAAANNNAAAAA|673|Hickory |Ln|Suite S|Union Hill|Howard County|IA|57746|United States|-6|single family| +3550|AAAAAAAAONNAAAAA|853|3rd |Parkway|Suite T|Clifton|Madison County|ID|88014|United States|-7|single family| +3551|AAAAAAAAPNNAAAAA|518|Jefferson Lincoln|Circle|Suite 180|Midway|Cleburne County|AL|31904|United States|-6|condo| +3552|AAAAAAAAAONAAAAA|86|12th 4th|Ct.|Suite 390|Bear Creek|Itasca County|MN|53075|United States|-6|single family| +3553|AAAAAAAABONAAAAA|809|Smith 2nd|Ct.|Suite 270|Harmony|Cavalier County|ND|55804|United States|-6|apartment| +3554|AAAAAAAACONAAAAA|671|Green Cedar|Way|Suite 460|Bridgeport|Polk County|AR|75817|United States|-6|single family| +3555|AAAAAAAADONAAAAA||Woodland Sycamore||Suite Y||Custer County|SD|56614||-6|| +3556|AAAAAAAAEONAAAAA|544|College Willow|Blvd|Suite 230|Lakeside|Dimmit County|TX|79532|United States|-6|single family| +3557|AAAAAAAAFONAAAAA|316|Sycamore Laurel|Parkway|Suite 80|Oakdale|Woodford County|KY|49584|United States|-6|apartment| +3558|AAAAAAAAGONAAAAA|105|Third |Ln|Suite T|Leon|Edgar County|IL|60913|United States|-6|condo| +3559|AAAAAAAAHONAAAAA|99|Cedar |Avenue|Suite 50|Bunker Hill|Taylor County|WI|50150|United States|-6|apartment| +3560|AAAAAAAAIONAAAAA|953|Fourth |Road|Suite 230|Sulphur Springs|Baker County|OR|98354|United States|-8|single family| +3561|AAAAAAAAJONAAAAA|207|Davis Hill|Ln|Suite 280|Ryan|Wilkin County|MN|50525|United States|-6|condo| +3562|AAAAAAAAKONAAAAA|597|10th |Ave|Suite 370|Providence|Russell County|AL|36614|United States|-6|condo| +3563|AAAAAAAALONAAAAA|508|Wilson |Avenue|Suite L|Jamestown|Obion County|TN|36867|United States|-6|apartment| +3564|AAAAAAAAMONAAAAA|473|Washington Main|Ave|Suite F|Pleasant Hill|Russell County|KY|43604|United States|-5|single family| +3565|AAAAAAAANONAAAAA|369|Cedar |Ln|Suite 50|Waterloo|Rio Grande County|CO|81675|United States|-7|single family| +3566|AAAAAAAAOONAAAAA|501|Second |Boulevard|Suite 250|Oak Grove|Boone County|IA|58370|United States|-6|condo| +3567|AAAAAAAAPONAAAAA|561|First 2nd|Wy|Suite O|Walnut Grove|Delta County|MI|47752|United States|-5|single family| +3568|AAAAAAAAAPNAAAAA|113|Chestnut 1st|Ln|Suite 40|Springfield|Oakland County|MI|49303|United States|-5|apartment| +3569|AAAAAAAABPNAAAAA|295|Second |Street|Suite F|Centerville|Accomack County|VA|20059|United States|-5|single family| +3570|AAAAAAAACPNAAAAA|568|Third Valley|Ct.|Suite T|Belmont|Clearwater County|ID|80191|United States|-7|single family| +3571|AAAAAAAADPNAAAAA|427|Elevnth Main|Wy|Suite H|Riverview|Duchesne County|UT|89003|United States|-7|apartment| +3572|AAAAAAAAEPNAAAAA|779|First Park|Ln|Suite 460|Friendship|Wheatland County|MT|64536|United States|-7|single family| +3573|AAAAAAAAFPNAAAAA|746|East Park|Road|Suite C|Lakeview|Daviess County|IN|48579|United States|-5|condo| +3574|AAAAAAAAGPNAAAAA|||Ln||Forest Hills||MO|69237||-6|| +3575|AAAAAAAAHPNAAAAA|890|Hill Ridge|Ave|Suite Q|Oakdale|San Diego County|CA|99584|United States|-8|single family| +3576|AAAAAAAAIPNAAAAA|620|Hill Hillcrest|Avenue|Suite K|Marion|Hart County|KY|40399|United States|-6|condo| +3577|AAAAAAAAJPNAAAAA|322|Highland First|ST|Suite K|Five Points|Sampson County|NC|26098|United States|-5|condo| +3578|AAAAAAAAKPNAAAAA|391|Valley Adams|Parkway|Suite C|Bridgeport|Hot Springs County|WY|85817|United States|-7|apartment| +3579|AAAAAAAALPNAAAAA|469|Center |Cir.|Suite Q|Macedonia|Penobscot County|ME|01687|United States|-5|single family| +3580|AAAAAAAAMPNAAAAA||Spruce Broadway||||Brooke County||||-5|condo| +3581|AAAAAAAANPNAAAAA|83|Lake Locust|Ct.|Suite Q|Woodland|Clarke County|MS|54854|United States|-6|apartment| +3582|AAAAAAAAOPNAAAAA|803|Madison Sunset|Way|Suite V|Forest Hills|Carson County|TX|79237|United States|-6|condo| +3583|AAAAAAAAPPNAAAAA|917|Meadow Maple|Dr.|Suite 160|Ashland|Appanoose County|IA|54244|United States|-6|condo| +3584|AAAAAAAAAAOAAAAA|9|Ninth Pine|Wy|Suite 150|White Oak|Wadena County|MN|56668|United States|-6|condo| +3585|AAAAAAAABAOAAAAA|229|Chestnut |Dr.|Suite 450|Weldon|Caroline County|MD|26277|United States|-5|condo| +3586|AAAAAAAACAOAAAAA|779|15th Birch|Ave|Suite Y|Macedonia|Humphreys County|MS|51087|United States|-6|single family| +3587|AAAAAAAADAOAAAAA|637|Park |Parkway|Suite X|Oak Hill|Harding County|SD|57838|United States|-7|apartment| +3588|AAAAAAAAEAOAAAAA|44|12th Railroad|Dr.|Suite B|Spring Hill|Clearwater County|MN|56787|United States|-6|condo| +3589|AAAAAAAAFAOAAAAA|736|Broadway Dogwood|Ln|Suite O|Bunker Hill|Troup County|GA|30150|United States|-5|apartment| +3590|AAAAAAAAGAOAAAAA|712|Sunset 11th|RD|Suite P|Lakeside|Cherokee County|OK|79532|United States|-6|single family| +3591|AAAAAAAAHAOAAAAA|157|East |Wy|Suite I|Fairview|Overton County|TN|35709|United States|-6|single family| +3592|AAAAAAAAIAOAAAAA|294|Washington Hill|Road|Suite T|Midway|Ripley County|MO|61904|United States|-6|apartment| +3593|AAAAAAAAJAOAAAAA|573|13th 2nd|ST|Suite R|Ashland|Putnam County|OH|44244|United States|-5|condo| +3594|AAAAAAAAKAOAAAAA|874|Pine |Wy|Suite C|Oak Hill|Oconto County|WI|57838|United States|-6|apartment| +3595|AAAAAAAALAOAAAAA|85|Fourteenth Third|Blvd|Suite L|Clinton|Lincoln County|KY|48222|United States|-5|single family| +3596|AAAAAAAAMAOAAAAA|27|Walnut |Ln|Suite 190|Deerfield|Wetzel County|WV|29840|United States|-5|condo| +3597|AAAAAAAANAOAAAAA|377|Sixth Railroad|Dr.|Suite V|Highland|Carbon County|MT|69454|United States|-7|apartment| +3598|AAAAAAAAOAOAAAAA|99|Lake |Boulevard|Suite 320|Antioch|Hocking County|OH|48605|United States|-5|single family| +3599|AAAAAAAAPAOAAAAA|218|Lake College|Dr.|Suite 230|Clifton|Effingham County|IL|68014|United States|-6|condo| +3600|AAAAAAAAABOAAAAA|173|Walnut 1st|Avenue|Suite 290|Maple Grove|Oconee County|GA|38252|United States|-5|apartment| +3601|AAAAAAAABBOAAAAA|556|Main North|Drive|Suite P|Oakwood|Crawford County|MI|40169|United States|-5|apartment| +3602|AAAAAAAACBOAAAAA|350|13th |Avenue|Suite 460|Oakland|Cumberland County|KY|49843|United States|-6|apartment| +3603|AAAAAAAADBOAAAAA|558|1st Sixth|Ln|Suite P|Hamilton|Idaho County|ID|82808|United States|-7|condo| +3604|AAAAAAAAEBOAAAAA|409|Davis Oak|Dr.|Suite G|Unionville|Ida County|IA|51711|United States|-6|apartment| +3605|AAAAAAAAFBOAAAAA|636|9th 13th|Road|Suite D|Harmony|Jackson County|KS|65804|United States|-6|condo| +3606|AAAAAAAAGBOAAAAA|496|Spring 10th|Ave|Suite 410|Greenwood|Forsyth County|NC|28828|United States|-5|apartment| +3607|AAAAAAAAHBOAAAAA|492|Maple |Court|Suite 300|Brooks|Worcester County|MA|01184|United States|-5|single family| +3608|AAAAAAAAIBOAAAAA|740|Elm |Ln|Suite 230|Union Hill|Oneida County|WI|57746|United States|-6|single family| +3609|AAAAAAAAJBOAAAAA|879|Lake Lake|Cir.|Suite G|Clinton|Rosebud County|MT|68222|United States|-7|single family| +3610|AAAAAAAAKBOAAAAA||Cedar ||Suite X|||VA|24098|||| +3611|AAAAAAAALBOAAAAA|985|Hickory Forest|Street|Suite V|Marion|Yakutat Borough|AK|90399|United States|-9|apartment| +3612|AAAAAAAAMBOAAAAA|380|Woodland Dogwood|Cir.|Suite J|Royal|Converse County|WY|85819|United States|-7|condo| +3613|AAAAAAAANBOAAAAA|873|Church Cherry|Wy|Suite 300|Oakdale|Ingham County|MI|49584|United States|-5|apartment| +3614|AAAAAAAAOBOAAAAA||||Suite 280|Lee||MO|60408|United States||apartment| +3615|AAAAAAAAPBOAAAAA|866|Fourth Locust|Blvd|Suite 400|Shiloh|Sharkey County|MS|59275|United States|-6|condo| +3616|AAAAAAAAACOAAAAA|105|Locust |ST|Suite 470|Green Acres|Lafayette County|AR|77683|United States|-6|apartment| +3617|AAAAAAAABCOAAAAA|625|Poplar Church|Circle|Suite M|Kingston|Washington County|PA|14975|United States|-5|condo| +3618|AAAAAAAACCOAAAAA|868|West |Ave|Suite 230|Greenville|Kershaw County|SC|21387|United States|-5|apartment| +3619|AAAAAAAADCOAAAAA|774|5th |Lane|Suite 20|Youngstown|Mohave County|AZ|80001|United States|-7|condo| +3620|AAAAAAAAECOAAAAA|877|7th |Lane|Suite P|Liberty|Lamb County|TX|73451|United States|-6|apartment| +3621|AAAAAAAAFCOAAAAA|31|Oak Church|Pkwy|Suite 480|Greenfield|Chatham County|NC|25038|United States|-5|apartment| +3622|AAAAAAAAGCOAAAAA|487|Lake |Ct.|Suite 120|Highland Park|Jackson County|AR|76534|United States|-6|single family| +3623|AAAAAAAAHCOAAAAA|523|Maple |ST|Suite K|Bethesda|Hendry County|FL|35980|United States|-5|apartment| +3624|AAAAAAAAICOAAAAA|770|Birch Lakeview|RD|Suite X|Highland Park|Bernalillo County|NM|86534|United States|-7|single family| +3625|AAAAAAAAJCOAAAAA|494|Second |Dr.|Suite A|Clifton|Lamar County|MS|58014|United States|-6|condo| +3626|AAAAAAAAKCOAAAAA|46|13th |Parkway|Suite 140|Kingston|Lawrence County|KY|44975|United States|-5|single family| +3627|AAAAAAAALCOAAAAA|320|4th |Parkway|Suite U|Union Hill|Garfield County|MT|67746|United States|-7|single family| +3628|AAAAAAAAMCOAAAAA|320|Cedar Dogwood|Road|Suite J|Mount Vernon|Giles County|TN|38482|United States|-5|single family| +3629|AAAAAAAANCOAAAAA|184|4th 14th|ST|Suite F|Bunker Hill|Calhoun County|TX|70150|United States|-6|single family| +3630|AAAAAAAAOCOAAAAA|74|Sunset Jackson|Blvd|Suite W|Friendship|Knox County|TN|34536|United States|-6|single family| +3631|AAAAAAAAPCOAAAAA|712|Second |RD|Suite X|Wilson|Franklin County|KY|46971|United States|-6|single family| +3632|AAAAAAAAADOAAAAA||||Suite G||Dade County|MO|69532||-6|condo| +3633|AAAAAAAABDOAAAAA|421|Spruce |Pkwy|Suite M|Sunnyside|Lagrange County|IN|41952|United States|-5|condo| +3634|AAAAAAAACDOAAAAA||||Suite 460|||||||| +3635|AAAAAAAADDOAAAAA|991|1st |Court|Suite V|Harmony|Pepin County|WI|55804|United States|-6|condo| +3636|AAAAAAAAEDOAAAAA|28|6th |ST|Suite 470|Harmony|Robertson County|TX|75804|United States|-6|single family| +3637|AAAAAAAAFDOAAAAA|908|Chestnut |Circle|Suite C|Lakeside|Chester County|PA|19532|United States|-5|apartment| +3638|AAAAAAAAGDOAAAAA|430|Green |Lane|Suite 400|Oakland|Breckinridge County|KY|49843|United States|-6|condo| +3639|AAAAAAAAHDOAAAAA|255|River Central|Ave|Suite 260|Lakeside|Pulaski County|IN|49532|United States|-5|single family| +3640|AAAAAAAAIDOAAAAA|113|Franklin Park|Parkway|Suite 450|Fairview|Wright County|MO|65709|United States|-6|condo| +3641|AAAAAAAAJDOAAAAA|329|5th |Drive|Suite 450|Jackson|Hunterdon County|NJ|09583|United States|-5|condo| +3642|AAAAAAAAKDOAAAAA|258|Second Poplar|Pkwy|Suite 180|Florence|Henderson County|NC|23394|United States|-5|single family| +3643|AAAAAAAALDOAAAAA|903|Franklin |Street|Suite 310|Pine Grove|Bristol Bay Borough|AK|94593|United States|-9|single family| +3644|AAAAAAAAMDOAAAAA|152|Spruce Miller|Parkway|Suite J|Lincoln|Sequatchie County|TN|31289|United States|-6|apartment| +3645|AAAAAAAANDOAAAAA|713|Park |RD|Suite U|Walnut Grove|Carter County|MT|67752|United States|-7|single family| +3646|AAAAAAAAODOAAAAA|865|Cedar |Avenue|Suite S|Newtown|Hutchinson County|SD|51749|United States|-7|condo| +3647|AAAAAAAAPDOAAAAA|988|Lake Seventh|Wy|Suite O|Greendale|Otoe County|NE|69343|United States|-7|apartment| +3648|AAAAAAAAAEOAAAAA|424|Mill Williams|Ave|Suite 100|Redland|Red River County|TX|76343|United States|-6|apartment| +3649|AAAAAAAABEOAAAAA|823|Park 14th|Drive|Suite 440|New Hope|Randolph County|MO|69431|United States|-6|condo| +3650|AAAAAAAACEOAAAAA|695|7th 9th|Parkway|Suite 200|Stringtown|Cimarron County|OK|70162|United States|-6|single family| +3651|AAAAAAAADEOAAAAA|556|Wilson Highland|Pkwy|Suite R|Forest Hills|King County|WA|99237|United States|-8|single family| +3652|AAAAAAAAEEOAAAAA|931|Second Maple|Ct.|Suite O|Hillcrest|Sabine County|TX|73003|United States|-6|single family| +3653|AAAAAAAAFEOAAAAA|93|3rd 7th|Parkway|Suite 330|Ashland|Routt County|CO|84244|United States|-7|single family| +3654|AAAAAAAAGEOAAAAA|228|Dogwood |Avenue|Suite 330|Lakeview|Buffalo County|NE|68579|United States|-6|apartment| +3655|AAAAAAAAHEOAAAAA|916|Forest 5th|Boulevard|Suite C|Friendship|Wallowa County|OR|94536|United States|-8|single family| +3656|AAAAAAAAIEOAAAAA|111|Sunset |Way|Suite D|Woodbine|Woodbury County|IA|54253|United States|-6|single family| +3657|AAAAAAAAJEOAAAAA|817|Williams 3rd|Street|Suite 410|Wright|Yadkin County|NC|22814|United States|-5|condo| +3658|AAAAAAAAKEOAAAAA|236|12th 11th|Boulevard|Suite 260|Hamilton|Pepin County|WI|52808|United States|-6|condo| +3659|AAAAAAAALEOAAAAA||||Suite 160|||||United States|-5|apartment| +3660|AAAAAAAAMEOAAAAA|153|Lake River|Drive|Suite 210|Bear Creek|Woodruff County|AR|73075|United States|-6|single family| +3661|AAAAAAAANEOAAAAA|873|Cherry |Pkwy|Suite J|Florence|Saunders County|NE|63394|United States|-7|condo| +3662|AAAAAAAAOEOAAAAA|752|6th |Circle|Suite 280|Summit|Steele County|MN|50499|United States|-6|single family| +3663|AAAAAAAAPEOAAAAA|698|10th |Ct.|Suite 280|Lakewood|Jackson County|KY|48877|United States|-6|apartment| +3664|AAAAAAAAAFOAAAAA|450|15th |Dr.|Suite 210|Oak Grove|Mercer County|ND|58370|United States|-6|condo| +3665|AAAAAAAABFOAAAAA|665|Eigth Washington|Ct.|Suite Y|Page|Minidoka County|ID|80296|United States|-7|single family| +3666|AAAAAAAACFOAAAAA|852|Lake |Avenue|Suite 230|Maple Grove|Pettis County|MO|68252|United States|-6|single family| +3667|AAAAAAAADFOAAAAA||Maple |Dr.|Suite 360|Antioch||||United States|-5|| +3668|AAAAAAAAEFOAAAAA|820|4th |Ct.|Suite K|Arlington|Carter County|MT|66557|United States|-7|apartment| +3669|AAAAAAAAFFOAAAAA|534|Sixth Sycamore|Road|Suite S|Maple Grove|Grand Isle County|VT|08852|United States|-5|apartment| +3670|AAAAAAAAGFOAAAAA|976|West |Court|Suite D|Crossroads|Scotland County|NC|20534|United States|-5|single family| +3671|AAAAAAAAHFOAAAAA|477|East |Drive|Suite S|Georgetown|Jeff Davis County|TX|77057|United States|-6|single family| +3672|AAAAAAAAIFOAAAAA|557|Adams |Boulevard|Suite F|Glendale|Union County|NM|83951|United States|-7|single family| +3673|AAAAAAAAJFOAAAAA|567|Meadow Hillcrest|Ct.|Suite 110|Bunker Hill|Camden County|GA|30150|United States|-5|apartment| +3674|AAAAAAAAKFOAAAAA|291|Fifth Lincoln|Avenue|Suite H|Hopewell|Campbell County|SD|50587|United States|-6|single family| +3675|AAAAAAAALFOAAAAA|921|Third |Pkwy|Suite 10|Mountain View|Gonzales County|TX|74466|United States|-6|apartment| +3676|AAAAAAAAMFOAAAAA|317|Adams Maple|ST|Suite 150|Edgewood|Grant County|KS|60069|United States|-6|apartment| +3677|AAAAAAAANFOAAAAA|682|Franklin Spruce|Wy|Suite Q|Kingston|Eaton County|MI|44975|United States|-5|apartment| +3678|AAAAAAAAOFOAAAAA|766|Central |Way|Suite M|Sunnyside|Umatilla County|OR|91952|United States|-8|single family| +3679|AAAAAAAAPFOAAAAA|201|Davis |Court|Suite T|Buena Vista|Quitman County|MS|55752|United States|-6|apartment| +3680|AAAAAAAAAGOAAAAA|966|Mill Meadow|Court|Suite B|Glendale|McCulloch County|TX|73951|United States|-6|single family| +3681|AAAAAAAABGOAAAAA|582|Oak Central|Boulevard|Suite 450|Red Bank|Scott County|IN|44975|United States|-5|single family| +3682|AAAAAAAACGOAAAAA|359||Ct.|Suite 360|Georgetown|Luce County|MI|47057|United States||| +3683|AAAAAAAADGOAAAAA|241|Pine View|Ln|Suite H|Lakewood|Archuleta County|CO|88877|United States|-7|condo| +3684|AAAAAAAAEGOAAAAA|949|Madison Johnson|Dr.|Suite 360|Spring Grove|Yakima County|WA|96719|United States|-8|apartment| +3685|AAAAAAAAFGOAAAAA|423|Sycamore 9th|Ct.|Suite A|Shady Grove|Middlesex County|CT|03412|United States|-5|single family| +3686|AAAAAAAAGGOAAAAA|||||Farmington||IN||United States|-5|single family| +3687|AAAAAAAAHGOAAAAA|379|3rd College|Cir.|Suite 70|Spring Hill|Kent County|TX|76787|United States|-6|condo| +3688|AAAAAAAAIGOAAAAA|11|Ridge |Boulevard|Suite C|Jamestown|Clinch County|GA|36867|United States|-5|apartment| +3689|AAAAAAAAJGOAAAAA|537|Eigth 12th|Pkwy|Suite M|Forest Hills|Carroll County|AR|79237|United States|-6|apartment| +3690|AAAAAAAAKGOAAAAA|292|Broadway South|Blvd|Suite 400|Deerfield|Brooks County|GA|39840|United States|-5|single family| +3691|AAAAAAAALGOAAAAA|669|Ash Valley|Avenue|Suite 480|Brownsville|Dewey County|OK|79310|United States|-6|condo| +3692|AAAAAAAAMGOAAAAA|314|Mill 12th|Dr.|Suite U|Buena Vista|Rush County|KS|65752|United States|-6|condo| +3693|AAAAAAAANGOAAAAA|699|North Wilson|Circle|Suite X|Lakeside|Fauquier County|VA|29532|United States|-5|condo| +3694|AAAAAAAAOGOAAAAA|1|Church Second|Road|Suite P|Greenfield|Macon County|MO|65038|United States|-6|single family| +3695|AAAAAAAAPGOAAAAA|952|Mill Sunset|Boulevard|Suite 140|Cedar Grove|Boone County|AR|70411|United States|-6|single family| +3696|AAAAAAAAAHOAAAAA|679|South Main|Street|Suite U|Concord|Harnett County|NC|24107|United States|-5|single family| +3697|AAAAAAAABHOAAAAA|989|Sunset Oak|Ave|Suite 240|Salem|Scioto County|OH|48048|United States|-5|single family| +3698|AAAAAAAACHOAAAAA|424|Johnson |Way|Suite J|Forest Hills|Randolph County|IL|69237|United States|-6|apartment| +3699|AAAAAAAADHOAAAAA|472|Park 6th|Boulevard|Suite N|Lakeview|Wayne County|OH|48579|United States|-5|apartment| +3700|AAAAAAAAEHOAAAAA|320|Meadow |Wy|Suite C|Spring Hill|Ottawa County|OK|76787|United States|-6|single family| +3701|AAAAAAAAFHOAAAAA|374|Pine |Parkway|Suite 440|Green Acres|Ulster County|NY|17683|United States|-5|condo| +3702|AAAAAAAAGHOAAAAA|935|River Church|Pkwy|Suite 110|Springdale|Clearwater County|MN|58883|United States|-6|single family| +3703|AAAAAAAAHHOAAAAA|985|Park |Street|Suite 0|Wilson|Lauderdale County|MS|56971|United States|-6|apartment| +3704|AAAAAAAAIHOAAAAA|542|Hillcrest |Boulevard|Suite B|Buena Vista|Wayne County|NC|25752|United States|-5|single family| +3705|AAAAAAAAJHOAAAAA|914|Johnson Elm|Dr.|Suite A|Spring Hill|Perry County|TN|36787|United States|-6|condo| +3706|AAAAAAAAKHOAAAAA|597|Oak Franklin|Wy|Suite D|Riverview|Grundy County|IL|69003|United States|-6|condo| +3707|AAAAAAAALHOAAAAA|425|4th Park|Blvd|Suite W|Five Forks|Somerset County|MD|22293|United States|-5|apartment| +3708|AAAAAAAAMHOAAAAA|58|Maple Ridge|Ln|Suite 350|Arlington|Butler County|KY|46557|United States|-6|single family| +3709|AAAAAAAANHOAAAAA|897|14th Mill|Ave|Suite 20|Buena Vista|Fayette County|GA|35752|United States|-5|apartment| +3710|AAAAAAAAOHOAAAAA|565|12th |Drive|Suite Y|Liberty|Yolo County|CA|93451|United States|-8|apartment| +3711|AAAAAAAAPHOAAAAA|574|Lake Twelfth|Pkwy|Suite U|Mount Olive|Weston County|WY|88059|United States|-7|condo| +3712|AAAAAAAAAIOAAAAA|959|6th 3rd|Lane|Suite K|Riverside|Clinton County|KY|49231|United States|-6|condo| +3713|AAAAAAAABIOAAAAA|462|Madison |Drive|Suite 440|Sunnyside|Union County|NJ|02552|United States|-5|single family| +3714|AAAAAAAACIOAAAAA|203|Second First|ST|Suite X|Riverdale|Ottawa County|KS|69391|United States|-6|condo| +3715|AAAAAAAADIOAAAAA|464|Poplar 5th|Drive|Suite K|Oak Ridge|Essex County|VT|08971|United States|-5|condo| +3716|AAAAAAAAEIOAAAAA|222|9th |RD|Suite T|Greenfield|Oconto County|WI|55038|United States|-6|apartment| +3717|AAAAAAAAFIOAAAAA|433|River Spruce|Circle|Suite V|Sunnyside|Crockett County|TX|71952|United States|-6|condo| +3718|AAAAAAAAGIOAAAAA|||RD|||Cameron Parish|LA||||single family| +3719|AAAAAAAAHIOAAAAA|40|14th |Road|Suite N|Enterprise|Latimer County|OK|71757|United States|-6|apartment| +3720|AAAAAAAAIIOAAAAA|404|Miller |Cir.|Suite N|Edgewood|Union County|KY|40069|United States|-5|condo| +3721|AAAAAAAAJIOAAAAA|597|Williams |ST|Suite 140|Greenwood|Christian County|IL|68828|United States|-6|apartment| +3722|AAAAAAAAKIOAAAAA|320|Ridge Sixth|Avenue|Suite V|Franklin|Vernon County|MO|69101|United States|-6|single family| +3723|AAAAAAAALIOAAAAA|189|Spruce |Avenue|Suite 30|Hopewell|Nelson County|VA|20587|United States|-5|condo| +3724|AAAAAAAAMIOAAAAA|4|Chestnut Chestnut|Wy|Suite U|Oakwood|Valley County|MT|60169|United States|-7|apartment| +3725|AAAAAAAANIOAAAAA|193|Washington |Lane|Suite D|Fairfield|Union County|NC|26192|United States|-5|single family| +3726|AAAAAAAAOIOAAAAA|877|12th Seventh|Way|Suite 220|Oakdale|Eureka County|NV|89584|United States|-8|single family| +3727|AAAAAAAAPIOAAAAA|444|4th Railroad|ST|Suite J|Spring Hill|Eureka County|NV|86787|United States|-8|apartment| +3728|AAAAAAAAAJOAAAAA|176|Meadow Miller|Ave|Suite 20|Glenwood|Sullivan County|MO|63511|United States|-6|apartment| +3729|AAAAAAAABJOAAAAA|408|Walnut |Ct.|Suite P|Deerfield|Dauphin County|PA|19840|United States|-5|condo| +3730|AAAAAAAACJOAAAAA|80|View |Boulevard|Suite 220|Waterloo|Bolivar County|MS|51675|United States|-6|apartment| +3731|AAAAAAAADJOAAAAA|957|Cherry |Wy|Suite B|Lakeside|Wyoming County|PA|19532|United States|-5|single family| +3732|AAAAAAAAEJOAAAAA|604|Dogwood |Blvd|Suite J|New Hope|Monroe County|WI|59431|United States|-6|single family| +3733|AAAAAAAAFJOAAAAA|650|8th |Ln|Suite M|Macedonia|Yamhill County|OR|91087|United States|-8|condo| +3734|AAAAAAAAGJOAAAAA|692|College |Pkwy|Suite 330|Pleasant Hill|Saline County|NE|63604|United States|-7|condo| +3735|AAAAAAAAHJOAAAAA|229|Woodland Oak|RD|Suite 340|Richfield|Castro County|TX|76196|United States|-6|apartment| +3736|AAAAAAAAIJOAAAAA|369|Hickory Johnson|Road|Suite O|Florence|Maui County|HI|93394|United States|-10|single family| +3737|AAAAAAAAJJOAAAAA|371|2nd |Wy|Suite 380|Forest Hills|Noble County|OH|49237|United States|-5|single family| +3738|AAAAAAAAKJOAAAAA|102|Poplar |Drive|Suite T|Arlington|Mathews County|VA|26557|United States|-5|condo| +3739|AAAAAAAALJOAAAAA|727|4th Maple|Dr.|Suite B|Mount Zion|Victoria County|TX|78054|United States|-6|single family| +3740|AAAAAAAAMJOAAAAA|565|Mill |Road|Suite 480|Salem|Transylvania County|NC|28048|United States|-5|condo| +3741|AAAAAAAANJOAAAAA|440|12th Oak|Cir.|Suite 460|Pleasant Valley|Hemphill County|TX|72477|United States|-6|apartment| +3742|AAAAAAAAOJOAAAAA|750|Hillcrest |Wy|Suite I|Hamilton|Rice County|KS|62808|United States|-6|apartment| +3743|AAAAAAAAPJOAAAAA|608|Third 5th|Blvd|Suite 420|Friendship|Grainger County|TN|34536|United States|-5|single family| +3744|AAAAAAAAAKOAAAAA|716|Ash |Ct.|Suite C|Greenwood|McCurtain County|OK|78828|United States|-6|condo| +3745|AAAAAAAABKOAAAAA|157|Poplar 9th|Ave|Suite P|New Town|Bryan County|OK|79634|United States|-6|condo| +3746|AAAAAAAACKOAAAAA|507|College |Pkwy|Suite 110|Glenwood|Washington County|ME|04111|United States|-5|condo| +3747|AAAAAAAADKOAAAAA|777|Fourth Davis|Circle|Suite F|Phoenix|Marion County|KS|62276|United States|-6|condo| +3748|AAAAAAAAEKOAAAAA|668|Miller |ST|Suite 490|Harmony|San Francisco County|CA|95804|United States|-8|single family| +3749|AAAAAAAAFKOAAAAA|785|Poplar |Cir.|Suite 20|White Oak|Cherokee County|NC|26668|United States|-5|apartment| +3750|AAAAAAAAGKOAAAAA|73|Sunset |Blvd|Suite 370|Whitney|Wabash County|IL|68339|United States|-6|condo| +3751|AAAAAAAAHKOAAAAA|229|Railroad |Court|Suite 120|Cedar Grove|Lyon County|IA|50411|United States|-6|apartment| +3752|AAAAAAAAIKOAAAAA|461|Willow 14th|Ln|Suite 400|Bethel|Noxubee County|MS|55281|United States|-6|apartment| +3753|AAAAAAAAJKOAAAAA|973|Elevnth |Avenue|Suite 360|Ashland|Burnett County|WI|54244|United States|-6|single family| +3754|AAAAAAAAKKOAAAAA|434|Lake |Lane|Suite C|Waterloo|New Castle County|DE|11675|United States|-5|condo| +3755|AAAAAAAALKOAAAAA|983|Hillcrest Forest|ST|Suite 370|Valley View|Crisp County|GA|35124|United States|-5|single family| +3756|AAAAAAAAMKOAAAAA|619|15th Valley|Lane|Suite 290|Riverview|Calumet County|WI|59003|United States|-6|single family| +3757|AAAAAAAANKOAAAAA|143|6th 2nd|Lane|Suite 350|Enterprise|Collingsworth County|TX|71757|United States|-6|apartment| +3758|AAAAAAAAOKOAAAAA|609|Spring Laurel|Cir.||Pleasant Grove|Erie County|OH|44136||-5|| +3759|AAAAAAAAPKOAAAAA|428|Seventh Meadow|Ln|Suite P|Arlington|Jefferson County|KY|46557|United States|-6|condo| +3760|AAAAAAAAALOAAAAA|580|Ridge |ST|Suite A|Parkwood|Winneshiek County|IA|51669|United States|-6|single family| +3761|AAAAAAAABLOAAAAA|403|Spring |Road|Suite 360|Stringtown|Madison County|VA|20162|United States|-5|single family| +3762|AAAAAAAACLOAAAAA|||Ave|Suite V||Humboldt County|NV||United States||condo| +3763|AAAAAAAADLOAAAAA|56|Highland |Dr.|Suite D|Oak Hill|Malheur County|OR|97838|United States|-8|condo| +3764|AAAAAAAAELOAAAAA|96|Jackson |Ct.|Suite S|Millwood|Mason County|IL|67319|United States|-6|apartment| +3765|AAAAAAAAFLOAAAAA|870|Hillcrest |Parkway|Suite 250|Liberty|Kearny County|KS|63451|United States|-6|apartment| +3766|AAAAAAAAGLOAAAAA|329|Mill 4th|Ave|Suite 280|Spring Grove|Rice County|MN|56719|United States|-6|apartment| +3767|AAAAAAAAHLOAAAAA|176|Washington Oak|Ave|Suite 430|Lebanon|Nacogdoches County|TX|72898|United States|-6|condo| +3768|AAAAAAAAILOAAAAA|794|Mill |Ct.|Suite 280|Antioch|Alcona County|MI|48605|United States|-5|condo| +3769|AAAAAAAAJLOAAAAA|730||Circle||Sulphur Springs||CA|98354|United States||| +3770|AAAAAAAAKLOAAAAA|636|2nd |Ct.|Suite S|Highland Park|Hancock County|KY|46534|United States|-6|apartment| +3771|AAAAAAAALLOAAAAA|522|Fourth |Dr.|Suite E|Mountain View|Salem city|VA|24466|United States|-5|condo| +3772|AAAAAAAAMLOAAAAA|533|Spring |Road|Suite R|Hopewell|Starr County|TX|70587|United States|-6|apartment| +3773|AAAAAAAANLOAAAAA|157|Lincoln 3rd|Dr.|Suite 420|Crossroads|Yell County|AR|70534|United States|-6|apartment| +3774|AAAAAAAAOLOAAAAA|783|3rd Maple|Road|Suite S|Riverview|Linn County|MO|69003|United States|-6|condo| +3775|AAAAAAAAPLOAAAAA||Pine ||Suite Y|New Hope||WV||United States||| +3776|AAAAAAAAAMOAAAAA|831|Fifth |Cir.|Suite 200|Oak Ridge|Shelby County|IL|68371|United States|-6|single family| +3777|AAAAAAAABMOAAAAA|462|East |Avenue|Suite 30|Woodland|Geauga County|OH|44854|United States|-5|apartment| +3778|AAAAAAAACMOAAAAA|128|12th |Wy|Suite V|Wilson|Effingham County|GA|36971|United States|-5|apartment| +3779|AAAAAAAADMOAAAAA|251|Valley Oak|Way|Suite 230|Mount Pleasant|Washington County|VT|02533|United States|-5|apartment| +3780|AAAAAAAAEMOAAAAA|423|Hillcrest Washington|Circle|Suite R|Mount Pleasant|Grant County|KY|41933|United States|-6|single family| +3781|AAAAAAAAFMOAAAAA|609|2nd Lincoln|Avenue|Suite 300|Oakwood|Howard County|IN|40169|United States|-5|apartment| +3782|AAAAAAAAGMOAAAAA|528|Smith |Street|Suite 70|Belmont|Stoddard County|MO|60191|United States|-6|single family| +3783|AAAAAAAAHMOAAAAA|914|Woodland Highland|Boulevard|Suite 350|Summerville|Gosper County|NE|62033|United States|-6|condo| +3784|AAAAAAAAIMOAAAAA|496|8th |ST|Suite 60|Plainville|Brown County|NE|66115|United States|-6|condo| +3785|AAAAAAAAJMOAAAAA|547|Seventh |Lane|Suite Q|Clifton|Cass County|IN|48014|United States|-5|single family| +3786|AAAAAAAAKMOAAAAA|855|Seventh Pine|ST|Suite 460|Oak Grove|Potter County|PA|18370|United States|-5|single family| +3787|AAAAAAAALMOAAAAA|160|East Pine|Pkwy|Suite P|Enterprise|Leslie County|KY|41757|United States|-5|condo| +3788|AAAAAAAAMMOAAAAA|17|Third Washington|Wy|Suite N|Spring Valley|Jerome County|ID|86060|United States|-7|single family| +3789|AAAAAAAANMOAAAAA|876|Walnut |Lane|Suite B|Five Forks|Tift County|GA|32293|United States|-5|single family| +3790|AAAAAAAAOMOAAAAA|431|Locust |RD|Suite O|Summit|Warren County|VA|20499|United States|-5|apartment| +3791|AAAAAAAAPMOAAAAA|694|3rd Hill|Dr.|Suite I|Shady Grove|Tillamook County|OR|92812|United States|-8|apartment| +3792|AAAAAAAAANOAAAAA|982|2nd |Wy|Suite 140|Sulphur Springs|Columbia County|PA|18354|United States|-5|condo| +3793|AAAAAAAABNOAAAAA|927|Mill Woodland|Lane|Suite H|Belmont|Nicholas County|WV|20191|United States|-5|apartment| +3794|AAAAAAAACNOAAAAA|836|Church 2nd|Ln|Suite 150|Glenwood|Washington County|MN|53511|United States|-6|condo| +3795|AAAAAAAADNOAAAAA|187|13th |Ln|Suite M|Oak Hill|Ottawa County|MI|47838|United States|-5|condo| +3796|AAAAAAAAENOAAAAA|966|2nd |Boulevard|Suite 430|Brownsville|Russell County|KS|69310|United States|-6|single family| +3797|AAAAAAAAFNOAAAAA|895|5th 7th|||Fairview|Montgomery County||55709|||apartment| +3798|AAAAAAAAGNOAAAAA|370|1st |Ln|Suite P|Lakeside|Converse County|WY|89532|United States|-7|single family| +3799|AAAAAAAAHNOAAAAA|238|Forest |ST|Suite 370|Unionville|Union County|IN|41711|United States|-5|single family| +3800|AAAAAAAAINOAAAAA|44|East |Wy|Suite 340|White Oak|Park County|MT|66668|United States|-7|single family| +3801|AAAAAAAAJNOAAAAA|278|Woodland |Blvd|Suite A|Pleasant Valley|Clinton County|IN|42477|United States|-5|condo| +3802|AAAAAAAAKNOAAAAA|570|Meadow 6th|Court|Suite K|Spring Valley|Guilford County|NC|26060|United States|-5|single family| +3803|AAAAAAAALNOAAAAA|878|Cedar |Ln|Suite 250|Pleasant Grove|Lehigh County|PA|14136|United States|-5|condo| +3804|AAAAAAAAMNOAAAAA|41|5th Valley|Ct.|Suite 250|Green Acres|Sanpete County|UT|87683|United States|-7|apartment| +3805|AAAAAAAANNOAAAAA|842|Lincoln College|Ave|Suite 250|Hopewell|Panola County|TX|70587|United States|-6|condo| +3806|AAAAAAAAONOAAAAA|414|Oak |Court|Suite 120|Deerfield|Wheeler County|NE|69840|United States|-7|apartment| +3807|AAAAAAAAPNOAAAAA|639|7th First|Dr.|Suite 430|Woodlawn|Mecklenburg County|VA|24098|United States|-5|condo| +3808|AAAAAAAAAOOAAAAA|513|Lee |Dr.|Suite Q|Lincoln|Emmons County|ND|51289|United States|-6|apartment| +3809|AAAAAAAABOOAAAAA|30|14th Park|Court|Suite K|Shiloh|Todd County|MN|59275|United States|-6|single family| +3810|AAAAAAAACOOAAAAA|30|9th |Avenue|Suite 20|Jamestown|Mercer County|MO|66867|United States|-6|condo| +3811|AAAAAAAADOOAAAAA|98|Jackson |Blvd|Suite D|Pleasant Grove|Wadena County|MN|54136|United States|-6|condo| +3812|AAAAAAAAEOOAAAAA|685|Fourth Church|Court|Suite 210|Woodville|Lunenburg County|VA|24289|United States|-5|apartment| +3813|AAAAAAAAFOOAAAAA|648|Ash |Avenue|Suite 210|Hopewell|Edgecombe County|NC|20587|United States|-5|apartment| +3814|AAAAAAAAGOOAAAAA|540|Poplar 4th|Blvd|Suite P|Bunker Hill|Kingfisher County|OK|70150|United States|-6|condo| +3815|AAAAAAAAHOOAAAAA|700|12th |Drive|Suite 70|Lincoln|Salt Lake County|UT|81289|United States|-7|single family| +3816|AAAAAAAAIOOAAAAA|961|Mill |Pkwy|Suite P|Pleasant Hill|Lee County|GA|33604|United States|-5|apartment| +3817|AAAAAAAAJOOAAAAA|141|Railroad |Lane|Suite 60|Fisher|La Salle Parish|LA|72819|United States|-6|condo| +3818|AAAAAAAAKOOAAAAA|925|||Suite B|Waterloo||||United States||| +3819|AAAAAAAALOOAAAAA|367|Smith |Dr.|Suite 290|Hillcrest|Knox County|ME|03603|United States|-5|condo| +3820|AAAAAAAAMOOAAAAA|336|Franklin Lincoln|Pkwy|Suite F|Glenwood|Baxter County|AR|73511|United States|-6|single family| +3821|AAAAAAAANOOAAAAA|476|First |Blvd|Suite 250|Mount Olive|Ravalli County|MT|68059|United States|-7|apartment| +3822|AAAAAAAAOOOAAAAA|98|Main |Circle|Suite F|Murphy|Beltrami County|MN|52105|United States|-6|condo| +3823|AAAAAAAAPOOAAAAA|710|11th |Circle|Suite V|Mountain View|Clark County|IN|44466|United States|-5|apartment| +3824|AAAAAAAAAPOAAAAA|496|12th |Wy|Suite X|Jamestown|Mackinac County|MI|46867|United States|-5|condo| +3825|AAAAAAAABPOAAAAA|772|Meadow |Blvd|Suite G|Five Points|Mercer County|PA|16098|United States|-5|condo| +3826|AAAAAAAACPOAAAAA|907|Main |Wy|Suite 190|Lebanon|Lubbock County|TX|72898|United States|-6|apartment| +3827|AAAAAAAADPOAAAAA|568|Poplar 4th|Parkway|Suite 20|Wildwood|Drew County|AR|76871|United States|-6|condo| +3828|AAAAAAAAEPOAAAAA|491|3rd |ST|Suite 430|Greenwood|Franklin County|ID|88828|United States|-7|single family| +3829|AAAAAAAAFPOAAAAA|125|Ridge |Cir.|Suite R|Bethel|Green Lake County|WI|55281|United States|-6|apartment| +3830|AAAAAAAAGPOAAAAA|640|Lee North|RD|Suite I|Farmington|Warrick County|IN|49145|United States|-5|single family| +3831|AAAAAAAAHPOAAAAA|384|Church North|Cir.|Suite K|Summit|Iron County|MI|40499|United States|-5|single family| +3832|AAAAAAAAIPOAAAAA|43|Mill East|Dr.|Suite S|Denmark|Rensselaer County|NY|15576|United States|-5|apartment| +3833|AAAAAAAAJPOAAAAA|158|Church Center|Circle|Suite T|Hopewell|Manassas Park city|VA|20587|United States|-5|condo| +3834|AAAAAAAAKPOAAAAA|272|Park |Pkwy|Suite 20|Newport|Wilson County|KS|61521|United States|-6|single family| +3835|AAAAAAAALPOAAAAA|435|Johnson 11th|Parkway|Suite 100|Union|Kandiyohi County|MN|58721|United States|-6|single family| +3836|AAAAAAAAMPOAAAAA|463|Main |Ave|Suite 190|Waterloo|Somerset County|PA|11675|United States|-5|single family| +3837|AAAAAAAANPOAAAAA|960|Third Willow|Ln|Suite B|Lakeside|Campbell County|KY|49532|United States|-6|single family| +3838|AAAAAAAAOPOAAAAA|745|Church |Ln|Suite 100|Cherokee|Madison County|AL|35436|United States|-6|apartment| +3839|AAAAAAAAPPOAAAAA|220|5th |Way|Suite 440|Greenville|Tuscola County|MI|41387|United States|-5|single family| +3840|AAAAAAAAAAPAAAAA|460|5th |Blvd|Suite F|Woodville|Woodson County|KS|64289|United States|-6|single family| +3841|AAAAAAAABAPAAAAA||||Suite D|Lakewood||||United States|-5|| +3842|AAAAAAAACAPAAAAA|683|Miller 15th|Boulevard|Suite T|Belmont|Butte County|ID|80191|United States|-7|apartment| +3843|AAAAAAAADAPAAAAA|100|Church |Parkway|Suite R|Lakeside|Delaware County|NY|19532|United States|-5|apartment| +3844|AAAAAAAAEAPAAAAA|412|Lake |Parkway|Suite 0|Oakwood|Wise County|VA|20169|United States|-5|condo| +3845|AAAAAAAAFAPAAAAA|647|Third |Boulevard|Suite 380|Green Acres|Orleans County|VT|08283|United States|-5|condo| +3846|AAAAAAAAGAPAAAAA|442|Church |Dr.|Suite 470|Clifton|Davie County|NC|28014|United States|-5|condo| +3847|AAAAAAAAHAPAAAAA|374|Ridge |Avenue|Suite 70|Riverdale|Clay County|SD|59391|United States|-6|apartment| +3848|AAAAAAAAIAPAAAAA|435|First |Street|Suite V|Hamilton|Sequoyah County|OK|72808|United States|-6|apartment| +3849|AAAAAAAAJAPAAAAA|417|Seventh Fifth|Wy|Suite A|Antioch|Tuscaloosa County|AL|38605|United States|-6|condo| +3850|AAAAAAAAKAPAAAAA|685|15th |Street|Suite C|Lebanon|Niobrara County|WY|82898|United States|-7|condo| +3851|AAAAAAAALAPAAAAA|607|2nd |Boulevard|Suite 340|Colfax|Williamsburg County|SC|22565|United States|-5|condo| +3852|AAAAAAAAMAPAAAAA|42|Park |ST|Suite 280|Fairview|Phillips County|KS|65709|United States|-6|apartment| +3853|AAAAAAAANAPAAAAA|598|Jefferson 8th|Street|Suite I|Five Forks|Barton County|MO|62293|United States|-6|single family| +3854|AAAAAAAAOAPAAAAA|827|10th South|Parkway|Suite C|Valley View|Comal County|TX|75124|United States|-6|single family| +3855|AAAAAAAAPAPAAAAA|796|5th |Ct.|Suite 440|Pleasant Valley|Union County|SC|22477|United States|-5|apartment| +3856|AAAAAAAAABPAAAAA|152|Park Highland|Wy|Suite 70|Concord|Mills County|TX|74107|United States|-6|condo| +3857|AAAAAAAABBPAAAAA|252|6th |Drive|Suite 20|Providence|Marshall County|AL|36614|United States|-6|apartment| +3858|AAAAAAAACBPAAAAA|311|Church |Road|Suite 200|Green Acres|Faulkner County|AR|77683|United States|-6|condo| +3859|AAAAAAAADBPAAAAA|431|Davis |Drive|Suite U|Newtown|Kauai County|HI|91749|United States|-10|single family| +3860|AAAAAAAAEBPAAAAA|333|8th Pine|Wy|Suite 40|Oak Grove|Lewis County|WA|98370|United States|-8|apartment| +3861|AAAAAAAAFBPAAAAA|999|Second |Pkwy|Suite E|Lakeview|Marshall County|WV|28579|United States|-5|apartment| +3862|AAAAAAAAGBPAAAAA|863|Park Railroad|Ln|Suite 90|Salem|Seneca County|OH|48048|United States|-5|single family| +3863|AAAAAAAAHBPAAAAA|343|Lake Cherry|Road|Suite 230|Mountain View|Jasper County|MO|64466|United States|-6|condo| +3864|AAAAAAAAIBPAAAAA||||Suite 270|||||United States||| +3865|AAAAAAAAJBPAAAAA|120|2nd Dogwood|Dr.|Suite P|Greenville|Duval County|FL|31387|United States|-5|single family| +3866|AAAAAAAAKBPAAAAA|267|View Spruce|Road|Suite F|Jackson|Fayette County|IA|59583|United States|-6|apartment| +3867|AAAAAAAALBPAAAAA|971|Miller |ST|Suite T|Walnut Grove|Clarke County|VA|27752|United States|-5|apartment| +3868|AAAAAAAAMBPAAAAA|32|2nd |Avenue|Suite 130|Newtown|San Juan County|UT|81749|United States|-7|condo| +3869|AAAAAAAANBPAAAAA|461|Elm 13th|Pkwy|Suite 80|Lakewood|Reeves County|TX|78877|United States|-6|apartment| +3870|AAAAAAAAOBPAAAAA|522|Meadow |Ct.|Suite O|New Hope|Shannon County|SD|59431|United States|-7|condo| +3871|AAAAAAAAPBPAAAAA|154|South |RD|Suite 490|Kingston|Roberts County|SD|54975|United States|-7|condo| +3872|AAAAAAAAACPAAAAA|980|View |Circle|Suite 490|Glenwood|Chaffee County|CO|83511|United States|-7|condo| +3873|AAAAAAAABCPAAAAA|332|12th |Ln|Suite A|Crossroads|Williams County|ND|50534|United States|-6|condo| +3874|AAAAAAAACCPAAAAA|256|5th |Drive|Suite 120|Union City|Orleans County|NY|18087|United States|-5|condo| +3875|AAAAAAAADCPAAAAA|394|Birch Laurel|Ln|Suite 450|Antioch|Calloway County|KY|48605|United States|-6|single family| +3876|AAAAAAAAECPAAAAA|350|Park View|Avenue|Suite K|Jerome|Clay County|WV|29920|United States|-5|single family| +3877|AAAAAAAAFCPAAAAA|509|Williams |Way|Suite W|Wilson|Cayuga County|NY|16971|United States|-5|condo| +3878|AAAAAAAAGCPAAAAA|30|Laurel |Ct.|Suite 450|Summit|Knox County|OH|40499|United States|-5|apartment| +3879|AAAAAAAAHCPAAAAA|780|Washington |Ln|Suite 300|Hartland|Richmond County|NC|26594|United States|-5|apartment| +3880|AAAAAAAAICPAAAAA|165|Lake Main|Street|Suite 150|Deerfield|Taylor County|FL|39840|United States|-5|condo| +3881|AAAAAAAAJCPAAAAA|195|11th Park|ST|Suite 210|Sullivan|Putnam County|NY|10451|United States|-5|condo| +3882|AAAAAAAAKCPAAAAA|810|4th Maple|Street|Suite 320|Sherwood Forest|Adams County|IA|56702|United States|-6|apartment| +3883|AAAAAAAALCPAAAAA|594|View |Boulevard|Suite 280|Hopewell|Caldwell County|TX|70587|United States|-6|single family| +3884|AAAAAAAAMCPAAAAA|979|13th |Ave|Suite 270|Lebanon|Chickasaw County|MS|52898|United States|-6|single family| +3885|AAAAAAAANCPAAAAA|697|3rd Main|Circle|Suite 280|Union Hill|Washtenaw County|MI|47746|United States|-5|condo| +3886|AAAAAAAAOCPAAAAA|247|East Walnut|Pkwy|Suite 490|Newport|Decatur County|KS|61521|United States|-6|single family| +3887|AAAAAAAAPCPAAAAA|818|Elm |Ct.|Suite 30|Sulphur Springs|Marion County|MO|68354|United States|-6|single family| +3888|AAAAAAAAADPAAAAA|377|11th |Lane|Suite T|Pine Grove|Carson City|NV|84593|United States|-8|single family| +3889|AAAAAAAABDPAAAAA|77|Maple |Street|Suite W|Riverside|Knox County|MO|69231|United States|-6|single family| +3890|AAAAAAAACDPAAAAA|765|Willow Lake|Ln|Suite 220|Acme|Yalobusha County|MS|50164|United States|-6|condo| +3891|AAAAAAAADDPAAAAA|979|6th 3rd|Ave|Suite A|||ME||United States||apartment| +3892|AAAAAAAAEDPAAAAA|840|Locust |Cir.|Suite 190|Hamburg|Platte County|WY|82230|United States|-7|single family| +3893|AAAAAAAAFDPAAAAA|837|Johnson |Lane|Suite U|La Grange|Sargent County|ND|57941|United States|-6|apartment| +3894|AAAAAAAAGDPAAAAA||Walnut Maple|Way|Suite V|Webb|Anderson County||||-6|apartment| +3895|AAAAAAAAHDPAAAAA|14|Willow |RD|Suite D|Woodbury|Hempstead County|AR|74489|United States|-6|apartment| +3896|AAAAAAAAIDPAAAAA|944|First 4th|Circle|Suite U|Green Acres|Saline County|IL|67683|United States|-6|condo| +3897|AAAAAAAAJDPAAAAA|844|Meadow 15th|Dr.|Suite 480|Pleasant Valley|Lincoln County|SD|52477|United States|-7|single family| +3898|AAAAAAAAKDPAAAAA|763|Madison 4th|Avenue|Suite 300|Bethel|Washington County|OR|95281|United States|-8|condo| +3899|AAAAAAAALDPAAAAA|749|9th Wilson|Parkway|Suite 470|Springdale|Marshall County|MS|58883|United States|-6|condo| +3900|AAAAAAAAMDPAAAAA|816|Walnut Lake|Way|Suite 490|Centerville|Perry County|KY|40059|United States|-5|condo| +3901|AAAAAAAANDPAAAAA|87|Fifth |Lane|Suite S|Pleasant Hill|Clay County|NE|63604|United States|-6|apartment| +3902|AAAAAAAAODPAAAAA|828|Valley Forest|Parkway|Suite R|Ashland|Swift County|MN|54244|United States|-6|single family| +3903|AAAAAAAAPDPAAAAA|353|Valley Maple|Drive|Suite S|Winslow|Pitkin County|CO|88525|United States|-7|apartment| +3904|AAAAAAAAAEPAAAAA|7|4th |ST|Suite 130|Macon|Polk County|GA|30369|United States|-5|condo| +3905|AAAAAAAABEPAAAAA|867|7th Park|Ct.|Suite C|Springdale|Livingston County|MI|48883|United States|-5|apartment| +3906|AAAAAAAACEPAAAAA|928|Central |Parkway|Suite P|Stewart|Craven County|NC|28041|United States|-5|single family| +3907|AAAAAAAADEPAAAAA|572|Johnson Franklin|Blvd|Suite Q|Clifton|Jones County|NC|28014|United States|-5|apartment| +3908|AAAAAAAAEEPAAAAA|539|Spring |Dr.|Suite 180|Plainview|Madison County|TN|33683|United States|-6|condo| +3909|AAAAAAAAFEPAAAAA|362|Second |Parkway|Suite 80|Greenfield|Outagamie County|WI|55038|United States|-6|single family| +3910|AAAAAAAAGEPAAAAA|756|East Seventh|Circle|Suite 420|Woodville|Carroll County|TN|34289|United States|-5|single family| +3911|AAAAAAAAHEPAAAAA|709|Fifth Third|RD|Suite 60|White Hall|Fresno County|CA|96955|United States|-8|condo| +3912|AAAAAAAAIEPAAAAA|994|3rd |Way|Suite 120|Bunker Hill|Coos County|OR|90150|United States|-8|apartment| +3913|AAAAAAAAJEPAAAAA|587|Johnson Spruce|Wy|Suite E|Glenwood|Santa Rosa County|FL|33511|United States|-5|single family| +3914|AAAAAAAAKEPAAAAA|15|Elm 9th|Cir.|Suite 240|Ashland|Caldwell Parish|LA|74244|United States|-6|condo| +3915|AAAAAAAALEPAAAAA|876|Oak |Blvd|Suite F|Kingston|Louisa County|IA|54975|United States|-6|apartment| +3916|AAAAAAAAMEPAAAAA|806|||Suite 120||||69584|United States|-7|| +3917|AAAAAAAANEPAAAAA|539|Elm |Drive|Suite X|Unionville|Scott County|TN|31711|United States|-6|single family| +3918|AAAAAAAAOEPAAAAA|89|Cherry |Street|Suite 10|Greenfield|Dawes County|NE|65038|United States|-6|condo| +3919|AAAAAAAAPEPAAAAA|697|Fourteenth |Pkwy|Suite G|Springfield|El Dorado County|CA|99303|United States|-8|condo| +3920|AAAAAAAAAFPAAAAA|440|Elm |Blvd|Suite 230|Lakeview|Wood County|WV|28579|United States|-5|single family| +3921|AAAAAAAABFPAAAAA|959|Miller |RD|Suite 100|Pleasant Hill|Anoka County|MN|53604|United States|-6|single family| +3922|AAAAAAAACFPAAAAA|282|14th 2nd|Ct.|Suite M|Clifford|Brookings County|SD|58164|United States|-6|apartment| +3923|AAAAAAAADFPAAAAA|258|View 1st|Street|Suite 460|Oakland|Wilkinson County|MS|59843|United States|-6|condo| +3924|AAAAAAAAEFPAAAAA|740|Third |RD|Suite O|Pleasant Valley|Dade County|GA|32477|United States|-5|apartment| +3925|AAAAAAAAFFPAAAAA|308|Willow North|Ln|Suite 160|Fairfield|Jefferson County|CO|86192|United States|-7|condo| +3926|AAAAAAAAGFPAAAAA|297|Williams South|Wy|Suite 470|Farmington|Lewis County|KY|49145|United States|-5|condo| +3927|AAAAAAAAHFPAAAAA|605|Park |Avenue|Suite W|Fox|Terrebonne Parish|LA|70631|United States|-6|condo| +3928|AAAAAAAAIFPAAAAA|718|Hickory Davis|Ave|Suite M|New Town|Wilson County|NC|29634|United States|-5|apartment| +3929|AAAAAAAAJFPAAAAA|393||Ln|Suite 190|Lebanon||||United States|-6|apartment| +3930|AAAAAAAAKFPAAAAA|680|View Lake|Way|Suite 80|Stratford|White County|GA|36668|United States|-5|apartment| +3931|AAAAAAAALFPAAAAA|991|8th Main|Wy|Suite J|Harmony|Blount County|AL|35804|United States|-6|apartment| +3932|AAAAAAAAMFPAAAAA|238|Franklin |Lane|Suite 90|Fairfield|Valdez-Cordova Census Area|AK|96192|United States|-9|condo| +3933|AAAAAAAANFPAAAAA|297|Fourth Spruce|Avenue|Suite F|Clifton|Kossuth County|IA|58014|United States|-6|single family| +3934|AAAAAAAAOFPAAAAA|965|Hickory 6th|Lane|Suite H|Collinsville|Washington County|IA|52459|United States|-6|condo| +3935|AAAAAAAAPFPAAAAA|783|Church |Court|Suite 340|Jamestown|Hopkins County|KY|46867|United States|-6|condo| +3936|AAAAAAAAAGPAAAAA|688|Lincoln View|Street|Suite F|Freeman|Acadia Parish|LA|72297|United States|-6|single family| +3937|AAAAAAAABGPAAAAA|626|Eigth Jackson|Ln|Suite H|Highland Park|Woodson County|KS|66534|United States|-6|single family| +3938|AAAAAAAACGPAAAAA|362|Third |Blvd|Suite 490|Lakewood|Pocahontas County|IA|58877|United States|-6|condo| +3939|AAAAAAAADGPAAAAA|219|5th |Ct.|Suite U|Mount Zion|Sac County|IA|58054|United States|-6|condo| +3940|AAAAAAAAEGPAAAAA|945|7th |Road|Suite L|Welcome|Morgan County|TN|36386|United States|-6|single family| +3941|AAAAAAAAFGPAAAAA|643|Hill Hickory|Street|Suite Y|Spring Hill|Broome County|NY|16787|United States|-5|condo| +3942|AAAAAAAAGGPAAAAA|991|Church Lake|RD|Suite 400|Hartland|Panola County|TX|76594|United States|-6|apartment| +3943|AAAAAAAAHGPAAAAA|773|River 4th|Boulevard|Suite 470|Pleasant Grove|Marquette County|MI|44136|United States|-5|condo| +3944|AAAAAAAAIGPAAAAA|606|Madison Forest|Wy|Suite S|Liberty|Dillingham Census Area|AK|93451|United States|-9|single family| +3945|AAAAAAAAJGPAAAAA|377|Lincoln |Street|Suite L|Georgetown|Caldwell County|NC|27057|United States|-5|single family| +3946|AAAAAAAAKGPAAAAA|676|Birch |Ln|Suite W|Clinton|Itawamba County|MS|58222|United States|-6|single family| +3947|AAAAAAAALGPAAAAA|128|Washington |Dr.|Suite L|Newport|Monroe County|MO|61521|United States|-6|single family| +3948|AAAAAAAAMGPAAAAA|607|View |Drive|||Real County||79584|United States||apartment| +3949|AAAAAAAANGPAAAAA|745|Franklin Hickory|ST|Suite 460|Farmington|DeKalb County|TN|39145|United States|-5|condo| +3950|AAAAAAAAOGPAAAAA|649|4th |Blvd|Suite N|New Hope|Morrill County|NE|69431|United States|-7|apartment| +3951|AAAAAAAAPGPAAAAA|838|Ridge |Parkway|Suite C|Blanchard|Denver County|CO|85985|United States|-7|single family| +3952|AAAAAAAAAHPAAAAA|30|6th |Pkwy|Suite 390|Woodlawn|Richland County|ND|54098|United States|-6|apartment| +3953|AAAAAAAABHPAAAAA|948|Laurel |Street|Suite 60|Springdale|Winn Parish|LA|78883|United States|-6|apartment| +3954|AAAAAAAACHPAAAAA|613|Jefferson |Lane|Suite 230|Valley View|Marlboro County|SC|25124|United States|-5|single family| +3955|AAAAAAAADHPAAAAA|64|Adams |RD|Suite K|Oak Ridge|Ottawa County|OH|48371|United States|-5|single family| +3956|AAAAAAAAEHPAAAAA|208|Broadway |Ct.|Suite 400|Union|Henderson County|TN|38721|United States|-5|single family| +3957|AAAAAAAAFHPAAAAA|328|2nd Second|Cir.|Suite 330|Clinton|Bradford County|FL|38222|United States|-5|condo| +3958|AAAAAAAAGHPAAAAA|160|6th Mill|Avenue|Suite P|Woodville|Jefferson County|TN|34289|United States|-6|condo| +3959|AAAAAAAAHHPAAAAA|979|Hill |Circle|Suite B|Colonial Heights|Walsh County|ND|53425|United States|-6|single family| +3960|AAAAAAAAIHPAAAAA|843|Washington Lincoln|ST|Suite 250|Elkton|Wasatch County|UT|83481|United States|-7|condo| +3961|AAAAAAAAJHPAAAAA|893|10th Willow|Pkwy|Suite O|Wildwood|Clay County|NC|26871|United States|-5|single family| +3962|AAAAAAAAKHPAAAAA|998|Washington |Court|Suite 140|Mount Vernon|Mitchell County|NC|28482|United States|-5|single family| +3963|AAAAAAAALHPAAAAA|13|Fourth |Street|Suite 140|Crossroads|Malheur County|OR|90534|United States|-8|single family| +3964|AAAAAAAAMHPAAAAA|190|Maple Willow|Pkwy|Suite G|Fairfield|Santa Cruz County|AZ|86192|United States|-7|condo| +3965|AAAAAAAANHPAAAAA|802|Hill Center|RD|Suite S|Riverview|Graham County|KS|69003|United States|-6|single family| +3966|AAAAAAAAOHPAAAAA|596|Second Pine|Pkwy|Suite 200|Hopewell|Johnson County|TX|70587|United States|-6|condo| +3967|AAAAAAAAPHPAAAAA|125|5th |Wy|Suite 70|Greenville|Ritchie County|WV|21387|United States|-5|single family| +3968|AAAAAAAAAIPAAAAA|148|2nd Main|RD|Suite X|Omega|McIntosh County|ND|56364|United States|-6|apartment| +3969|AAAAAAAABIPAAAAA|458|Cedar |Circle|Suite 10|Five Points|McIntosh County|ND|56098|United States|-6|condo| +3970|AAAAAAAACIPAAAAA|875|Hickory |Circle|Suite M|New Hope|Ouachita County|AR|79431|United States|-6|apartment| +3971|AAAAAAAADIPAAAAA|496|Second |Street|Suite 210|Yorktown|Lanier County|GA|30732|United States|-5|condo| +3972|AAAAAAAAEIPAAAAA|148|Highland |Ct.|Suite 460|New Hope|Dolores County|CO|89431|United States|-7|condo| +3973|AAAAAAAAFIPAAAAA|465|Cedar Green|Way|Suite K|Midway|Wyoming County|PA|11904|United States|-5|single family| +3974|AAAAAAAAGIPAAAAA|677|Smith |Dr.|Suite B|Milo|Dodge County|NE|60116|United States|-6|apartment| +3975|AAAAAAAAHIPAAAAA|703|Ash 11th|Avenue|Suite 270|Greenfield|Hockley County|TX|75038|United States|-6|apartment| +3976|AAAAAAAAIIPAAAAA|654|Davis |Blvd|Suite 130|Lakeville|Becker County|MN|58811|United States|-6|condo| +3977|AAAAAAAAJIPAAAAA|670|Dogwood Park|RD|Suite 230|Mountain View|Stewart County|TN|34466|United States|-6|single family| +3978|AAAAAAAAKIPAAAAA|731|South College|Boulevard|Suite D|Brownsville|Gallia County|OH|49310|United States|-5|single family| +3979|AAAAAAAALIPAAAAA|203|Willow Spruce|Street|Suite E|Springfield|Newton County|MO|69303|United States|-6|apartment| +3980|AAAAAAAAMIPAAAAA|915|Hill Cedar|Lane|Suite 50|Marion|Johnson County|KY|40399|United States|-6|apartment| +3981|AAAAAAAANIPAAAAA|720|Maple Sunset|Wy|Suite V|Payne|Fentress County|TN|36134|United States|-5|apartment| +3982|AAAAAAAAOIPAAAAA|355|Forest Church|Blvd|Suite 190|Argyle|Adams County|IN|48722|United States|-5|condo| +3983|AAAAAAAAPIPAAAAA|989|First Wilson|Circle|Suite W|Langdon|Roger Mills County|OK|70852|United States|-6|apartment| +3984|AAAAAAAAAJPAAAAA|544|10th 1st|Ct.|Suite 380|Pleasant Valley|Idaho County|ID|82477|United States|-7|apartment| +3985|AAAAAAAABJPAAAAA|413|Elevnth |Court|Suite 110|Bunker Hill|Daggett County|UT|80150|United States|-7|condo| +3986|AAAAAAAACJPAAAAA|439|Cedar Green|Pkwy|Suite 430|Macedonia|Fairfax County|VA|21087|United States|-5|apartment| +3987|AAAAAAAADJPAAAAA|431|Oak 4th|Boulevard|Suite E|Jackson|Charlottesville city|VA|29583|United States|-5|condo| +3988|AAAAAAAAEJPAAAAA|266|9th |RD|Suite R|Stewart|Wilkinson County|MS|58041|United States|-6|single family| +3989|AAAAAAAAFJPAAAAA|268|South Third|Pkwy|Suite Y|Hillcrest|Jefferson County|IA|53003|United States|-6|condo| +3990|AAAAAAAAGJPAAAAA|229|Franklin Church|Pkwy|Suite 370|Harmony|Allen County|OH|45804|United States|-5|apartment| +3991|AAAAAAAAHJPAAAAA|688|Main Green|Ln|Suite 260|Lakeside|Polk County|NC|29532|United States|-5|condo| +3992|AAAAAAAAIJPAAAAA|951|Railroad Highland|Way|Suite F|Sulphur Springs|Scott County|IA|58354|United States|-6|single family| +3993|AAAAAAAAJJPAAAAA|137|Lee |Pkwy|Suite V|Shady Grove|Marion County|IL|62812|United States|-6|apartment| +3994|AAAAAAAAKJPAAAAA|877|Adams Oak||Suite 480|Midway|Miller County|AR|||-6|| +3995|AAAAAAAALJPAAAAA|295|Lincoln |Circle|Suite M|Georgetown|Calhoun County|WV|27057|United States|-5|apartment| +3996|AAAAAAAAMJPAAAAA|800|Park Walnut|RD|Suite K|Mount Olive|Hendricks County|IN|48059|United States|-5|condo| +3997|AAAAAAAANJPAAAAA|985|2nd |Boulevard|Suite 150|Cedar Grove|Bedford County|TN|30411|United States|-5|apartment| +3998|AAAAAAAAOJPAAAAA|888|Fifth |Ave|Suite W|Belmont|Little River County|AR|70191|United States|-6|condo| +3999|AAAAAAAAPJPAAAAA|922|Main 2nd|Blvd|Suite N|Wilson|Lumpkin County|GA|36971|United States|-5|apartment| +4000|AAAAAAAAAKPAAAAA|236|Spring First|Blvd|Suite 40|Woodville|Jackson County|MN|54289|United States|-6|apartment| +4001|AAAAAAAABKPAAAAA|415|Woodland |Parkway|Suite C|Liberty|Orleans County|NY|13451|United States|-5|condo| +4002|AAAAAAAACKPAAAAA|884|Spruce |Ct.|Suite 180|Fayetteville|Monroe County|MI|41732|United States|-5|condo| +4003|AAAAAAAADKPAAAAA|935|View College|Dr.|Suite T|Lakeview|Darlington County|SC|28579|United States|-5|single family| +4004|AAAAAAAAEKPAAAAA|240|Green |Pkwy|Suite 370|Ashland|Audrain County|MO|64244|United States|-6|single family| +4005|AAAAAAAAFKPAAAAA|8|2nd Fourth|Ln|Suite 0|Centerville|Lyon County|NV|80059|United States|-8|condo| +4006|AAAAAAAAGKPAAAAA|670|4th |Drive|Suite 120|Clinton|Nance County|NE|68222|United States|-7|apartment| +4007|AAAAAAAAHKPAAAAA|932|3rd |Circle|Suite 180|Pine Grove|Armstrong County|TX|74593|United States|-6|apartment| +4008|AAAAAAAAIKPAAAAA|135|Ash Second|Parkway|Suite G|Fairfield|Hartley County|TX|76192|United States|-6|single family| +4009|AAAAAAAAJKPAAAAA|449|13th |Road|Suite S|Morgantown|Rock County|NE|69193|United States|-7|apartment| +4010|AAAAAAAAKKPAAAAA|218|Laurel North|ST|Suite W|Cedar Grove|Lincoln County|AR|70411|United States|-6|apartment| +4011|AAAAAAAALKPAAAAA|22|Cedar Seventh|Parkway|Suite L|White Oak|Poquoson city|VA|26668|United States|-5|condo| +4012|AAAAAAAAMKPAAAAA||Maple Valley|Parkway|Suite W||||55858|United States|-6|single family| +4013|AAAAAAAANKPAAAAA|361|Cedar |Blvd|Suite 470|Oak Hill|Barry County|MI|47838|United States|-5|apartment| +4014|AAAAAAAAOKPAAAAA|420|6th |Street|Suite 40|Highland Park|Decatur County|TN|36534|United States|-5|single family| +4015|AAAAAAAAPKPAAAAA|207|Cherry |RD|Suite A|Greenwood|Grant County|KY|48828|United States|-6|apartment| +4016|AAAAAAAAALPAAAAA|643|5th Lake|Court|Suite F|Five Forks|Lyon County|KY|42293|United States|-5|apartment| +4017|AAAAAAAABLPAAAAA|747|5th Washington|RD|Suite 30|Liberty|King William County|VA|23451|United States|-5|apartment| +4018|AAAAAAAACLPAAAAA|996|Hill |Drive|Suite J|Bunker Hill|Washington County|UT|80150|United States|-7|condo| +4019|AAAAAAAADLPAAAAA|276|Ridge |Parkway|Suite 140|Arlington|Ada County|ID|86557|United States|-7|condo| +4020|AAAAAAAAELPAAAAA|58|7th Ninth|RD|Suite 80|Highland Park|Swain County|NC|26534|United States|-5|condo| +4021|AAAAAAAAFLPAAAAA|309|8th |Pkwy|Suite 10|Oak Ridge|Bullitt County|KY|48371|United States|-6|apartment| +4022|AAAAAAAAGLPAAAAA|239|Railroad |Pkwy|Suite 150|Woodland|Isanti County|MN|54854|United States|-6|apartment| +4023|AAAAAAAAHLPAAAAA|46|Oak |ST|Suite 180|Siloam|Fayette County|PA|18948|United States|-5|single family| +4024|AAAAAAAAILPAAAAA|302|Laurel |RD|Suite K|Mount Pleasant|Gosper County|NE|61933|United States|-6|condo| +4025|AAAAAAAAJLPAAAAA|497|Park River|Wy|Suite M|Sunnyside|Chesapeake city|VA|21952|United States|-5|single family| +4026|AAAAAAAAKLPAAAAA|810|Poplar Green|Drive|Suite 320|Midway|Lincoln County|WA|91904|United States|-8|single family| +4027|AAAAAAAALLPAAAAA|387|Fifth |Lane|Suite 260|Lebanon|Benton County|MO|62898|United States|-6|single family| +4028|AAAAAAAAMLPAAAAA|369|Johnson |Pkwy|Suite M|Willow|Yuma County|CO|86798|United States|-7|condo| +4029|AAAAAAAANLPAAAAA|922|5th |Parkway|Suite D|Spring Hill|Otsego County|NY|16787|United States|-5|condo| +4030|AAAAAAAAOLPAAAAA|273|Main Poplar|Ave|Suite 270|Bridgeport|Dickson County|TN|35817|United States|-5|apartment| +4031|AAAAAAAAPLPAAAAA|239|Sunset |Pkwy|Suite 120|California|Oldham County|TX|70141|United States|-6|apartment| +4032|AAAAAAAAAMPAAAAA|60|1st Valley|Way|Suite T|Deerfield|Yamhill County|OR|99840|United States|-8|apartment| +4033|AAAAAAAABMPAAAAA|684|Second |ST|Suite M|Lakeside|Knox County|OH|49532|United States|-5|single family| +4034|AAAAAAAACMPAAAAA|220|Lake Sunset|Drive|Suite 250|Highland Park|Shasta County|CA|96534|United States|-8|apartment| +4035|AAAAAAAADMPAAAAA|888|Cedar |RD|Suite U|Liberty|Lynchburg city|VA|23451|United States|-5|condo| +4036|AAAAAAAAEMPAAAAA|82|Maple |Lane|Suite 150|Highland|Lafayette County|FL|39454|United States|-5|single family| +4037|AAAAAAAAFMPAAAAA|637|Lee 7th|Way|Suite 310|Bethel|Woodford County|KY|45281|United States|-6|single family| +4038|AAAAAAAAGMPAAAAA|780|3rd Third|Blvd|Suite 340|Maple Grove|Blaine County|NE|68252|United States|-6|apartment| +4039|AAAAAAAAHMPAAAAA|915|6th |Dr.|Suite G|Midway|Russell County|KY|41904|United States|-5|single family| +4040|AAAAAAAAIMPAAAAA|25|Lake |Ct.|Suite 480|Jackson|Owsley County|KY|49583|United States|-5|apartment| +4041|AAAAAAAAJMPAAAAA|383|Fourth |Ave|Suite V|Buena Vista|Contra Costa County|CA|95752|United States|-8|single family| +4042|AAAAAAAAKMPAAAAA|592|9th 9th|Parkway|Suite X|Florence|Bradley County|TN|33394|United States|-5|condo| +4043|AAAAAAAALMPAAAAA|347|River Main|Lane|Suite 110|Hopewell|Caldwell County|TX|70587|United States|-6|condo| +4044|AAAAAAAAMMPAAAAA|321|Dogwood Lake|Wy|Suite X|Stafford|Adams County|WI|54980|United States|-6|condo| +4045|AAAAAAAANMPAAAAA|726|3rd |Street|Suite 360|White Oak|Tulsa County|OK|76668|United States|-6|apartment| +4046|AAAAAAAAOMPAAAAA|760|River 1st|Dr.|Suite 220|Bridgeport|Tallahatchie County|MS|55817|United States|-6|single family| +4047|AAAAAAAAPMPAAAAA|224|4th |Cir.|Suite B|Shiloh|Willacy County|TX|79275|United States|-6|single family| +4048|AAAAAAAAANPAAAAA|497|Poplar Ash|Avenue|Suite 470|Mountain View|Stark County|IL|64466|United States|-6|single family| +4049|AAAAAAAABNPAAAAA|469|Ridge Second|Street|Suite S|Hillcrest|Oconee County|GA|33003|United States|-5|apartment| +4050|AAAAAAAACNPAAAAA|||Ave|Suite 10|||MN|58883|United States|-6|| +4051|AAAAAAAADNPAAAAA|786|7th |Parkway|Suite 360|Adams|Chautauqua County|KS|60986|United States|-6|condo| +4052|AAAAAAAAENPAAAAA|639|Third Park|Dr.|Suite 270|Jamestown|Eaton County|MI|46867|United States|-5|single family| +4053|AAAAAAAAFNPAAAAA|246|Fifth |Way|Suite 260|Oakdale|Adams County|WA|99584|United States|-8|apartment| +4054|AAAAAAAAGNPAAAAA|959|Third Chestnut|Wy|Suite S|Antioch|Atkinson County|GA|38605|United States|-5|apartment| +4055|AAAAAAAAHNPAAAAA|70|Ridge |ST|Suite G|Enterprise|Niagara County|NY|11757|United States|-5|condo| +4056|AAAAAAAAINPAAAAA|350|East |Wy|Suite F|Greenfield|Cocke County|TN|35038|United States|-5|single family| +4057|AAAAAAAAJNPAAAAA|995|Cherry |Ave|Suite V|Concord|Storey County|NV|84107|United States|-8|single family| +4058|AAAAAAAAKNPAAAAA|762|Oak |Wy|Suite I|Sumner|Lenawee County|MI|40519|United States|-5|single family| +4059|AAAAAAAALNPAAAAA|909|Lake |Drive|Suite 70|Newport|Brown County|IL|61521|United States|-6|single family| +4060|AAAAAAAAMNPAAAAA|94|Sycamore West|Dr.|Suite T|Glenwood|Randolph County|MO|63511|United States|-6|single family| +4061|AAAAAAAANNPAAAAA|326|Valley East|Pkwy|Suite G|Bethel|Greenbrier County|WV|25281|United States|-5|condo| +4062|AAAAAAAAONPAAAAA|262|Maple |Court|Suite L|Doyle|Osceola County|MI|48434|United States|-5|single family| +4063|AAAAAAAAPNPAAAAA|452|Seventh |Lane|Suite F|Richardson|Edgecombe County|NC|27687|United States|-5|condo| +4064|AAAAAAAAAOPAAAAA|505|Lincoln 6th|RD|Suite 20|Shamrock|Los Alamos County|NM|83879|United States|-7|single family| +4065|AAAAAAAABOPAAAAA|405|Washington Williams|Blvd|Suite 300|Franklin|Columbus County|NC|29101|United States|-5|apartment| +4066|AAAAAAAACOPAAAAA|110|Miller Oak|RD|Suite 280|Bunker Hill|Natchitoches Parish|LA|70150|United States|-6|apartment| +4067|AAAAAAAADOPAAAAA|203|Jackson |Parkway|Suite R|Guthrie|Lebanon County|PA|11423|United States|-5|condo| +4068|AAAAAAAAEOPAAAAA|148|Washington 9th|Drive|Suite 100|Woodville|Garfield County|UT|84289|United States|-7|apartment| +4069|AAAAAAAAFOPAAAAA||Railroad 1st|||Springdale|Grant County|WA|98883|||| +4070|AAAAAAAAGOPAAAAA|740|Lincoln 12th|Pkwy|Suite D|Newtown|Jefferson County|AL|31749|United States|-6|single family| +4071|AAAAAAAAHOPAAAAA|336|Washington Washington|Road|Suite U|Bethel|Rio Grande County|CO|85281|United States|-7|single family| +4072|AAAAAAAAIOPAAAAA|159|Eigth Valley|Pkwy|Suite 450|Forest Hills|Guadalupe County|TX|79237|United States|-6|condo| +4073|AAAAAAAAJOPAAAAA|186|6th North|Court|Suite 60|Kingston|Williamson County|TX|74975|United States|-6|single family| +4074|AAAAAAAAKOPAAAAA|662|Dogwood |Lane|Suite Q|Concord|Hamilton County|OH|44107|United States|-5|apartment| +4075|AAAAAAAALOPAAAAA|886|5th |Ct.|Suite 330|Forest Hills|Chemung County|NY|19237|United States|-5|single family| +4076|AAAAAAAAMOPAAAAA|695|Ridge Railroad|Court|Suite R|Lebanon|Bremer County|IA|52898|United States|-6|apartment| +4077|AAAAAAAANOPAAAAA|683|11th 11th|Pkwy|Suite R|Crossroads|Franklin County|KY|40534|United States|-6|apartment| +4078|AAAAAAAAOOPAAAAA|835|Jefferson |Dr.|Suite 10|Deerfield|Caribou County|ID|89840|United States|-7|apartment| +4079|AAAAAAAAPOPAAAAA|520|Elm First|Ct.|Suite 190|Guilford|Chaves County|NM|84408|United States|-7|condo| +4080|AAAAAAAAAPPAAAAA|936|3rd |Way|Suite 330|Wildwood|Nottoway County|VA|26871|United States|-5|single family| +4081|AAAAAAAABPPAAAAA|30|Eigth |Circle|Suite C|Providence|Santa Cruz County|AZ|86614|United States|-7|condo| +4082|AAAAAAAACPPAAAAA|958|View |Lane|Suite V|Forest Hills|Murray County|GA|39237|United States|-5|single family| +4083|AAAAAAAADPPAAAAA|832|View Sycamore|Road|Suite 440|Spring Valley|Worth County|GA|36060|United States|-5|condo| +4084|AAAAAAAAEPPAAAAA|738|Walnut |Road|Suite M|Hamilton|Stevens County|KS|62808|United States|-6|apartment| +4085|AAAAAAAAFPPAAAAA|82|Cedar |Dr.|Suite D|Plainview|Banks County|GA|33683|United States|-5|apartment| +4086|AAAAAAAAGPPAAAAA|783|12th |Blvd|Suite 420|Ashland|Dale County|AL|34244|United States|-6|apartment| +4087|AAAAAAAAHPPAAAAA|188|West Main|Lane|Suite F|Pinecrest|Mahaska County|IA|59981|United States|-6|condo| +4088|AAAAAAAAIPPAAAAA|441|North Forest|ST|Suite W|Brownsville|Merrick County|NE|69310|United States|-7|apartment| +4089|AAAAAAAAJPPAAAAA|917|Smith Washington|Cir.|Suite W|Lakewood|Columbiana County|OH|48877|United States|-5|apartment| +4090|AAAAAAAAKPPAAAAA|690|Laurel Hill|Circle|Suite 200|Stringtown|Wexford County|MI|40162|United States|-5|apartment| +4091|AAAAAAAALPPAAAAA|536|Cedar First|Dr.|Suite 130|Lincoln|Monroe County|IN|41289|United States|-5|single family| +4092|AAAAAAAAMPPAAAAA|116|4th |Circle|Suite F|Arlington|Northumberland County|PA|16557|United States|-5|single family| +4093|AAAAAAAANPPAAAAA|672|College Third|Dr.|Suite S|Pisgah|Winneshiek County|IA|52539|United States|-6|single family| +4094|AAAAAAAAOPPAAAAA|125|Third Locust|Ave|Suite T|Glenville|Denver County|CO|83445|United States|-7|single family| +4095|AAAAAAAAPPPAAAAA|689|Second |Road|Suite W|Unionville|Roanoke city|VA|21711|United States|-5|single family| +4096|AAAAAAAAAAABAAAA|226|3rd |Dr.|Suite 120|Bunker Hill|Washington Parish|LA|70150|United States|-6|single family| +4097|AAAAAAAABAABAAAA|358|Smith 8th|Dr.|Suite I|Stafford|Orange County|VA|24980|United States|-5|single family| +4098|AAAAAAAACAABAAAA|862|Madison 3rd|Cir.|Suite O|Riverdale|Buchanan County|MO|69391|United States|-6|single family| +4099|AAAAAAAADAABAAAA|358|Valley 13th|ST|Suite 340|Mountain View|Elkhart County|IN|44466|United States|-5|single family| +4100|AAAAAAAAEAABAAAA|450|Davis Meadow|Cir.|Suite U|Brentwood|Alamance County|NC|24188|United States|-5|condo| +4101|AAAAAAAAFAABAAAA|79|9th |Pkwy|Suite W|Ruth|Burke County|NC|20309|United States|-5|apartment| +4102|AAAAAAAAGAABAAAA|142|Second |Road|Suite O|Franklin|Yazoo County|MS|59101|United States|-6|condo| +4103|AAAAAAAAHAABAAAA|17|Ash Oak|Ln|Suite H|Springdale|Accomack County|VA|28883|United States|-5|condo| +4104|AAAAAAAAIAABAAAA|99|Park Seventh|Ln|Suite 140|Spring Hill|Grundy County|TN|36787|United States|-5|apartment| +4105|AAAAAAAAJAABAAAA|220|Smith |Wy|Suite L|Florence|Chaffee County|CO|83394|United States|-7|single family| +4106|AAAAAAAAKAABAAAA|770|Oak |Drive|Suite M|Florence|Plymouth County|IA|53394|United States|-6|condo| +4107|AAAAAAAALAABAAAA|846||Ct.|Suite P||Custer County|NE|69431||-6|apartment| +4108|AAAAAAAAMAABAAAA|487|Fourth Smith|Dr.|Suite A|Springdale|Fannin County|TX|78883|United States|-6|condo| +4109|AAAAAAAANAABAAAA||Church |Ave||||KS|||-6|| +4110|AAAAAAAAOAABAAAA|897|8th |Parkway|Suite J|Bridgeport|Millard County|UT|85817|United States|-7|single family| +4111|AAAAAAAAPAABAAAA|35|Willow Lincoln|Wy|Suite G|Riverview|Brule County|SD|59003|United States|-6|condo| +4112|AAAAAAAAABABAAAA|260|Second Sixth|Cir.|Suite M|Oak Hill|Oglethorpe County|GA|37838|United States|-5|single family| +4113|AAAAAAAABBABAAAA|989|Johnson Main|Boulevard|Suite V|Green Acres|Meade County|SD|57683|United States|-7|apartment| +4114|AAAAAAAACBABAAAA|539|Spruce Ridge|Drive|Suite E|New Hope|Hutchinson County|TX|79431|United States|-6|single family| +4115|AAAAAAAADBABAAAA|417|1st Fifth|Lane|Suite 80|Plainview|East Feliciana Parish|LA|73683|United States|-6|apartment| +4116|AAAAAAAAEBABAAAA|233|Church Johnson|Circle|Suite V|Bethel|Ottawa County|OH|45281|United States|-5|single family| +4117|AAAAAAAAFBABAAAA|384|Hill |Lane|Suite K|Shady Grove|Warren County|MS|52812|United States|-6|apartment| +4118|AAAAAAAAGBABAAAA|960|Park |Parkway|Suite N|Liberty|Moore County|NC|23451|United States|-5|single family| +4119|AAAAAAAAHBABAAAA|473|5th South|Parkway|Suite O|Five Points|San Benito County|CA|96098|United States|-8|condo| +4120|AAAAAAAAIBABAAAA|253|5th |Blvd|Suite 170|Roxbury|Bryan County|GA|35508|United States|-5|condo| +4121|AAAAAAAAJBABAAAA|883|14th |Boulevard|Suite 420|Salem|Bollinger County|MO|68048|United States|-6|single family| +4122|AAAAAAAAKBABAAAA|983|Davis 5th|Drive|Suite 160|Georgetown|Merced County|CA|97057|United States|-8|single family| +4123|AAAAAAAALBABAAAA|415|Ash |Ave|Suite 210|Franklin|Dona Ana County|NM|89101|United States|-7|single family| +4124|AAAAAAAAMBABAAAA|684|Ash Madison|Avenue|Suite 10|Centerville|Nueces County|TX|70059|United States|-6|single family| +4125|AAAAAAAANBABAAAA|745|Jefferson |Avenue|Suite 260|Wildwood|Jackson County|NC|26871|United States|-5|single family| +4126|AAAAAAAAOBABAAAA|567|Woodland |RD|Suite X|Oakland|Mercer County|PA|19843|United States|-5|apartment| +4127|AAAAAAAAPBABAAAA|42|Laurel |Cir.|Suite C|Five Points|Clearwater County|MN|56098|United States|-6|apartment| +4128|AAAAAAAAACABAAAA|440|Washington 8th|Street|Suite L|Spring Valley|Henry County|OH|46060|United States|-5|apartment| +4129|AAAAAAAABCABAAAA|29|Forest Fifth|Court|Suite 140|New Hope|Weakley County|TN|39431|United States|-6|apartment| +4130|AAAAAAAACCABAAAA|354|2nd Hill|Dr.|Suite G|Kingston|Morehouse Parish|LA|74975|United States|-6|apartment| +4131|AAAAAAAADCABAAAA|953|Mill |Ave|Suite K|Stringtown|Halifax County|NC|20162|United States|-5|single family| +4132|AAAAAAAAECABAAAA|887|First |Boulevard|Suite 100|Cedar Grove|Marshall County|KS|60411|United States|-6|condo| +4133|AAAAAAAAFCABAAAA|44|Walnut |RD|Suite 250|Stringtown|Webster County|MS|50162|United States|-6|condo| +4134|AAAAAAAAGCABAAAA|444|Spring |Cir.|Suite M|Cedar Grove|Wilkes County|NC|20411|United States|-5|apartment| +4135|AAAAAAAAHCABAAAA|524|Center Pine|Circle|Suite W|Woodland|Warren County|PA|14854|United States|-5|apartment| +4136|AAAAAAAAICABAAAA|517|Lee |Ln|Suite V|Five Forks|Surry County|VA|22293|United States|-5|apartment| +4137|AAAAAAAAJCABAAAA|619|Fifth |Ct.|Suite S|Riverdale|Floyd County|IN|49391|United States|-5|condo| +4138|AAAAAAAAKCABAAAA|227|Adams |Dr.|Suite 380|Greenwood|Wilkinson County|GA|38828|United States|-5|apartment| +4139|AAAAAAAALCABAAAA|622|1st Hickory|Blvd|Suite S|Pine Grove|Lafourche Parish|LA|74593|United States|-6|condo| +4140|AAAAAAAAMCABAAAA|496|3rd |Ave|Suite H|Lee|Burnet County|TX|70408|United States|-6|apartment| +4141|AAAAAAAANCABAAAA|313|4th |Street|Suite 240|Red Hill|Graves County|KY|44338|United States|-6|apartment| +4142|AAAAAAAAOCABAAAA|56|Cherry Wilson|Way|Suite 110|La Grange|Franklin County|MO|67941|United States|-6|condo| +4143|AAAAAAAAPCABAAAA|92|Pine Johnson|Dr.|Suite 490|Centerville|Buchanan County|MO|60059|United States|-6|apartment| +4144|AAAAAAAAADABAAAA|491|Park |Drive|Suite C|Maple Grove|Schuyler County|MO|68252|United States|-6|single family| +4145|AAAAAAAABDABAAAA|415|Jackson |Wy|Suite 360|Spring Valley|Prince Edward County|VA|26060|United States|-5|apartment| +4146|AAAAAAAACDABAAAA|882|11th |Dr.|Suite 350|Plainville|Randolph County|IN|46115|United States|-5|apartment| +4147|AAAAAAAADDABAAAA|23|Spring |ST|Suite 90|Edgewood|Lexington County|SC|20069|United States|-5|condo| +4148|AAAAAAAAEDABAAAA|857|12th First|Court|Suite 270||Butler County||51521|United States|-6|| +4149|AAAAAAAAFDABAAAA|859|Jefferson |Ave|Suite O|Kingston|Vigo County|IN|44975|United States|-5|apartment| +4150|AAAAAAAAGDABAAAA|714|Cherry |Way|Suite I|Unionville|Todd County|MN|51711|United States|-6|single family| +4151|AAAAAAAAHDABAAAA|780|Madison |Ave|Suite 150|Newport|Sussex County|NJ|02121|United States|-5|single family| +4152|AAAAAAAAIDABAAAA|389|8th Meadow|Blvd|Suite 20|New Hope|Millard County|UT|89431|United States|-7|condo| +4153|AAAAAAAAJDABAAAA|94|South |RD|Suite E|Mount Vernon|Henry County|AL|38482|United States|-6|apartment| +4154|AAAAAAAAKDABAAAA|16|Cedar |Ln|Suite I|Sulphur Springs|Salt Lake County|UT|88354|United States|-7|condo| +4155|AAAAAAAALDABAAAA|5|Lincoln Highland|Ct.|Suite 190|Hopewell|Wyoming County|WV|20587|United States|-5|condo| +4156|AAAAAAAAMDABAAAA|448|5th Madison|Wy|Suite 130|Langdon|Trempealeau County|WI|50852|United States|-6|condo| +4157|AAAAAAAANDABAAAA|439|4th Washington|Wy|Suite 200|Woodville|Centre County|PA|14289|United States|-5|condo| +4158|AAAAAAAAODABAAAA|567|Ridge |Wy|Suite O|Woodbury|Potter County|SD|54489|United States|-7|condo| +4159|AAAAAAAAPDABAAAA|691|Woodland |Drive|Suite 110|Bunker Hill|Mahaska County|IA|50150|United States|-6|single family| +4160|AAAAAAAAAEABAAAA|157|Meadow Woodland|Circle|Suite 480|Oak Grove|Campbell County|KY|48370|United States|-6|apartment| +4161|AAAAAAAABEABAAAA|779|Chestnut |ST|Suite 440|Georgetown|Cocke County|TN|37057|United States|-5|apartment| +4162|AAAAAAAACEABAAAA|653|Lee |Ct.|Suite S|Cedar Grove|Logan County|ND|50411|United States|-6|condo| +4163|AAAAAAAADEABAAAA|436|Meadow |Ct.|Suite 490|Spring Hill|McKenzie County|ND|56787|United States|-6|condo| +4164|AAAAAAAAEEABAAAA|627|Franklin Hillcrest|Way|Suite 270|Mount Olive|Brazoria County|TX|78059|United States|-6|condo| +4165|AAAAAAAAFEABAAAA|816|Mill Second|Lane|Suite 450|Omega|Latimer County|OK|76364|United States|-6|condo| +4166|AAAAAAAAGEABAAAA|146|Cedar Ninth|Road|Suite 20|Hillcrest|Scotland County|NC|23003|United States|-5|apartment| +4167|AAAAAAAAHEABAAAA|206|13th Mill|Ln|Suite 80|Greenville|Harris County|GA|31387|United States|-5|condo| +4168|AAAAAAAAIEABAAAA|863|12th Twelfth||Suite E|Fairview||MS|||-6|| +4169|AAAAAAAAJEABAAAA|432|Center Center|Ave|Suite K|Fairview|Hays County|TX|75709|United States|-6|condo| +4170|AAAAAAAAKEABAAAA|961|Main |Avenue|Suite M|Pleasant Grove|Manassas city|VA|24136|United States|-5|apartment| +4171|AAAAAAAALEABAAAA|497||Road|||Crockett County|||||| +4172|AAAAAAAAMEABAAAA|145|Walnut Tenth|Dr.|Suite 110|Walnut Grove|Guadalupe County|TX|77752|United States|-6|condo| +4173|AAAAAAAANEABAAAA|147|Hill |Lane|Suite 220|Crossroads|Jefferson County|ID|80534|United States|-7|single family| +4174|AAAAAAAAOEABAAAA|852|Sunset |Boulevard|Suite P|Glendale|Pecos County|TX|73951|United States|-6|apartment| +4175|AAAAAAAAPEABAAAA||Oak |Avenue|Suite 310||Cooke County||74136|United States|-6|single family| +4176|AAAAAAAAAFABAAAA|25|Chestnut |Wy|Suite 400|Midway|Hardeman County|TN|31904|United States|-5|condo| +4177|AAAAAAAABFABAAAA|775|Tenth Cherry|Boulevard|Suite 60|Pine Grove|Transylvania County||24593|United States||| +4178|AAAAAAAACFABAAAA|552|12th Lakeview|RD|Suite B|Glendale|Ellis County|KS|63951|United States|-6|condo| +4179|AAAAAAAADFABAAAA|618|Fifth West|Wy|Suite 20|Wilson|Delta County|TX|76971|United States|-6|condo| +4180|AAAAAAAAEFABAAAA|399|Lakeview Lincoln|Ave|Suite A|Caledonia|Augusta County|VA|27411|United States|-5|apartment| +4181|AAAAAAAAFFABAAAA|902|First Main|Wy|Suite S|Jerome|Fayette County|TX|79920|United States|-6|apartment| +4182|AAAAAAAAGFABAAAA|214|Fifth |Circle|Suite 370|Summit|Nevada County|CA|90499|United States|-8|condo| +4183|AAAAAAAAHFABAAAA|75|Third |RD|Suite A|Newtown|Johnson County|IN|41749|United States|-5|single family| +4184|AAAAAAAAIFABAAAA|165|South |Parkway|Suite F|Hillcrest|King County|TX|73003|United States|-6|condo| +4185|AAAAAAAAJFABAAAA|643|Franklin Spruce|Ave|Suite L|Stringtown|Billings County|ND|50162|United States|-6|condo| +4186|AAAAAAAAKFABAAAA|623|Third Center|Lane|Suite T|Montpelier|Genesee County|MI|48930|United States|-5|apartment| +4187|AAAAAAAALFABAAAA|659|Ridge |Street|Suite 160|Union Hill|Wilkinson County|GA|37746|United States|-5|condo| +4188|AAAAAAAAMFABAAAA|326|Eigth |Circle|Suite 90|Bethel|Avery County|NC|25281|United States|-5|single family| +4189|AAAAAAAANFABAAAA|970|Main |Avenue|Suite P|Lakeview|Yellowstone County|MT|68579|United States|-7|single family| +4190|AAAAAAAAOFABAAAA|826|Elm 14th|Wy|Suite 50|Lebanon|Kenosha County|WI|52898|United States|-6|condo| +4191|AAAAAAAAPFABAAAA|817|Ninth South|Lane|Suite N|Springdale|Ohio County|WV|28883|United States|-5|condo| +4192|AAAAAAAAAGABAAAA|146|Mill |Lane|Suite 290|Oneida|Butte County|ID|84027|United States|-7|apartment| +4193|AAAAAAAABGABAAAA|425|6th |Ave|Suite 260|Lakewood|Orange County|NY|18877|United States|-5|condo| +4194|AAAAAAAACGABAAAA|586|First |Cir.|Suite 300|Mount Olive|Chambers County|TX|78059|United States|-6|single family| +4195|AAAAAAAADGABAAAA|423|Third Ash|Ct.|Suite 470|West Liberty|Summers County|WV|24752|United States|-5|apartment| +4196|AAAAAAAAEGABAAAA|684|Franklin Johnson|Blvd|Suite N|Midway||MT|61904|United States||| +4197|AAAAAAAAFGABAAAA|551|8th Ridge|Boulevard|Suite 180|Lakewood|Stevens County|WA|98877|United States|-8|single family| +4198|AAAAAAAAGGABAAAA|779|7th |Road|Suite M|Bunker Hill|Teton County|MT|60150|United States|-7|apartment| +4199|AAAAAAAAHGABAAAA|703|3rd |Wy|Suite 240|Marion|Oswego County|NY|10399|United States|-5|apartment| +4200|AAAAAAAAIGABAAAA|432|Franklin Pine|Ct.|Suite H|Newport|Otero County|NM|81521|United States|-7|apartment| +4201|AAAAAAAAJGABAAAA|398|Valley |Blvd|Suite X|Clinton|Stephens County|GA|38222|United States|-5|condo| +4202|AAAAAAAAKGABAAAA|604|Park |Blvd|Suite I|Marion|Perkins County|SD|50399|United States|-7|condo| +4203|AAAAAAAALGABAAAA|3|Fifth |Drive|Suite 330|Five Forks|Garfield County|OK|72293|United States|-6|single family| +4204|AAAAAAAAMGABAAAA|212|Cherry |Lane|Suite 390|Shady Grove|Midland County|MI|42812|United States|-5|single family| +4205|AAAAAAAANGABAAAA|612|Mill |Wy|Suite 240|Jamestown|Custer County|ID|86867|United States|-7|condo| +4206|AAAAAAAAOGABAAAA|434|Maple |Cir.|Suite R|Riverdale|Norton County|KS|69391|United States|-6|single family| +4207|AAAAAAAAPGABAAAA|437|Mill Mill|Court|Suite 400|Lakewood|Butler County|IA|58877|United States|-6|apartment| +4208|AAAAAAAAAHABAAAA|784|11th |Parkway|Suite V|Enterprise|Lee County|FL|31757|United States|-5|condo| +4209|AAAAAAAABHABAAAA|481|Cedar |Way|Suite 380|Arlington|Swain County|NC|26557|United States|-5|condo| +4210|AAAAAAAACHABAAAA|642|Pine 9th|Drive|Suite J|Antioch|Delaware County|OK|78605|United States|-6|apartment| +4211|AAAAAAAADHABAAAA|472|10th |Parkway|Suite V|Harmony|Jefferson Davis County|MS|55804|United States|-6|single family| +4212|AAAAAAAAEHABAAAA|781|8th 3rd|Court|Suite 370|Union|Gentry County|MO|68721|United States|-6|condo| +4213|AAAAAAAAFHABAAAA||Spring |||||SD|||-7|| +4214|AAAAAAAAGHABAAAA|596|12th |Wy|Suite 180|Hillcrest|Caroline County|MD|23003|United States|-5|condo| +4215|AAAAAAAAHHABAAAA|685|Mill Maple|Parkway|Suite C|Greenville|Finney County|KS|61387|United States|-6|single family| +4216|AAAAAAAAIHABAAAA|375|Broadway 5th|Street|Suite O|Pleasant Valley|Edwards County|TX|72477|United States|-6|single family| +4217|AAAAAAAAJHABAAAA|251|Main |Street|Suite Y|Greenwood|Clay County|TN|38828|United States|-5|single family| +4218|AAAAAAAAKHABAAAA|273|Park |Boulevard|Suite P|Oakwood|Barry County|MO|60169|United States|-6|single family| +4219|AAAAAAAALHABAAAA|850|3rd |Street|Suite D|Shiloh|Creek County|OK|79275|United States|-6|condo| +4220|AAAAAAAAMHABAAAA|853|Elm |Boulevard|Suite U|Oak Grove|Carroll County|KY|48370|United States|-6|apartment| +4221|AAAAAAAANHABAAAA|942|Sunset |Blvd|Suite S|Concord|Wilkes County|GA|34107|United States|-5|single family| +4222|AAAAAAAAOHABAAAA|482|7th 5th|Boulevard|Suite U|White Oak|Liberty County|FL|36668|United States|-5|apartment| +4223|AAAAAAAAPHABAAAA|778|7th |Avenue|Suite F|Liberty|Huerfano County|CO|83451|United States|-7|condo| +4224|AAAAAAAAAIABAAAA|341|13th |Drive|Suite 390|Lone Oak|Polk County|FL|36893|United States|-5|condo| +4225|AAAAAAAABIABAAAA|194|7th Second|Street|Suite L|Enterprise|McMullen County|TX|71757|United States|-6|condo| +4226|AAAAAAAACIABAAAA|669|13th Lee|Way|Suite J|Five Forks|Nance County|NE|62293|United States|-7|single family| +4227|AAAAAAAADIABAAAA|261|Valley South|Way|Suite T|Appleton|Broadwater County|MT|64240|United States|-7|apartment| +4228|AAAAAAAAEIABAAAA|765|Highland |Cir.|Suite K|Springtown|Marion County|MS|59858|United States|-6|apartment| +4229|AAAAAAAAFIABAAAA|936|6th 1st|Blvd|Suite Y|Omega|Chautauqua County|KS|66364|United States|-6|single family| +4230|AAAAAAAAGIABAAAA|213|Adams |Road|Suite A|Pleasant Valley|Perry County|TN|32477|United States|-6|single family| +4231|AAAAAAAAHIABAAAA|583|10th 1st|Way|Suite H|Fairfield|Duval County|FL|36192|United States|-5|apartment| +4232|AAAAAAAAIIABAAAA|541|West 15th|Ave|Suite 340|Delmar|Mercer County|IL|63957|United States|-6|single family| +4233|AAAAAAAAJIABAAAA|956|1st Main|Ct.|Suite 130|Hopewell|Carbon County|UT|80587|United States|-7|condo| +4234|AAAAAAAAKIABAAAA|202|Laurel |Road|Suite P|Clifton|Moffat County|CO|88014|United States|-7|single family| +4235|AAAAAAAALIABAAAA|129|Jefferson Elm|Ave|Suite 350|Georgetown|Morgan County|CO|87057|United States|-7|apartment| +4236|AAAAAAAAMIABAAAA|616|View Dogwood|Ct.|Suite O|Fairfield|Benton County|MS|56192|United States|-6|condo| +4237|AAAAAAAANIABAAAA|395|2nd |Wy|Suite 490|Mountain View|Bee County|TX|74466|United States|-6|single family| +4238|AAAAAAAAOIABAAAA|572|Maple |Drive|Suite 410|Plainview|Kent County|RI|04283|United States|-5|condo| +4239|AAAAAAAAPIABAAAA|494|Jefferson Chestnut|Ct.|Suite 300|Jamestown|Lincoln County|ME|07467|United States|-5|apartment| +4240|AAAAAAAAAJABAAAA|652|8th Highland|Blvd|Suite Y|Jenkins|Montgomery County|MO|67292|United States|-6|apartment| +4241|AAAAAAAABJABAAAA|874|College Church|Ct.|Suite 270|Mount Pleasant|Sabine County|TX|71933|United States|-6|apartment| +4242|AAAAAAAACJABAAAA|||Avenue||Greenville|Franklin County|||United States|-6|| +4243|AAAAAAAADJABAAAA|722|Maple 5th|Drive|Suite 270|Pleasant Hill|Fulton County|GA|33604|United States|-5|apartment| +4244|AAAAAAAAEJABAAAA|464|Seventh |Ct.|Suite F|Wildwood|Nicholas County|KY|46871|United States|-5|apartment| +4245|AAAAAAAAFJABAAAA|24|Fifteenth Sunset|Way|Suite 210|Plainview|Horry County|SC|23683|United States|-5|single family| +4246|AAAAAAAAGJABAAAA|538|West |Pkwy|Suite U|Wildwood|Lee County|IA|56871|United States|-6|single family| +4247|AAAAAAAAHJABAAAA|137|Ash |Dr.|Suite J|Kingston|Ohio County|KY|44975|United States|-5|apartment| +4248|AAAAAAAAIJABAAAA||Tenth ||||Culpeper County|VA||United States|-5|apartment| +4249|AAAAAAAAJJABAAAA|977|Hickory Main|RD|Suite 130|Arlington|Wapello County|IA|56557|United States|-6|apartment| +4250|AAAAAAAAKJABAAAA|10|3rd |Dr.|Suite O|Union|Ohio County|WV|28721|United States|-5|condo| +4251|AAAAAAAALJABAAAA|936|Poplar |Drive|Suite X|Belmont|Wilson County|KS|60191|United States|-6|single family| +4252|AAAAAAAAMJABAAAA|892|Woodland 4th|ST|Suite 120|Unionville||IN||||| +4253|AAAAAAAANJABAAAA|565|East 1st|Ct.|Suite B|Oak Hill|Brunswick County|VA|27838|United States|-5|apartment| +4254|AAAAAAAAOJABAAAA|103|Center |Blvd|Suite 130|Newport|Morehouse Parish|LA|71521|United States|-6|condo| +4255|AAAAAAAAPJABAAAA|419|East Johnson|Wy|Suite F|Marion|Barton County|KS|60399|United States|-6|condo| +4256|AAAAAAAAAKABAAAA|717|Elm |Circle|Suite 80|Springhill|Allegany County|NY|14602|United States|-5|single family| +4257|AAAAAAAABKABAAAA|96|Central Lake|Court|Suite S|Jackson|Crawford County|MI|49583|United States|-5|single family| +4258|AAAAAAAACKABAAAA|343|8th |ST|Suite 70|New Hope|Anderson County|TN|39431|United States|-5|condo| +4259|AAAAAAAADKABAAAA|201|1st |RD|Suite Y|Riverview|Burke County|NC|29003|United States|-5|single family| +4260|AAAAAAAAEKABAAAA|882|South |Way|Suite 320|Buffalo|Santa Clara County|CA|97098|United States|-8|single family| +4261|AAAAAAAAFKABAAAA|||Parkway||Five Points|Anne Arundel County|||||| +4262|AAAAAAAAGKABAAAA|484|6th |Street|Suite H|Jamestown|Cherokee County|KS|66867|United States|-6|condo| +4263|AAAAAAAAHKABAAAA|999|Elevnth |Blvd|Suite S|Bunker Hill|Jasper County|IL|60150|United States|-6|single family| +4264|AAAAAAAAIKABAAAA|965|Main Wilson|Cir.|Suite 310|Aberdeen|Conecuh County|AL|37995|United States|-6|condo| +4265|AAAAAAAAJKABAAAA|277|East West|Boulevard|Suite P|Brookville|Loup County|NE|63524|United States|-7|single family| +4266|AAAAAAAAKKABAAAA|180||Ave||Springfield|Frederick County|||||| +4267|AAAAAAAALKABAAAA|474|Lincoln First|Lane|Suite 150|Green Acres|Floyd County|TX|77683|United States|-6|apartment| +4268|AAAAAAAAMKABAAAA|626|Oak Oak|Lane|Suite E|Spring Valley|Early County|GA|36060|United States|-5|condo| +4269|AAAAAAAANKABAAAA|616|Hillcrest Park|Boulevard|Suite 180|Greenwood|Rockwall County|TX|78828|United States|-6|apartment| +4270|AAAAAAAAOKABAAAA|919|Oak 11th|Blvd|Suite 270|Spring Valley|Holmes County|MS|56060|United States|-6|apartment| +4271|AAAAAAAAPKABAAAA|90|Adams |Dr.|Suite 480|Fairview|Terry County|TX|75709|United States|-6|single family| +4272|AAAAAAAAALABAAAA|326|Second Cedar|Way|Suite B|Shady Grove|Putnam County|OH|42812|United States|-5|single family| +4273|AAAAAAAABLABAAAA|169|Chestnut |Way|Suite 310|Sulphur Springs|Mitchell County|TX|78354|United States|-6|apartment| +4274|AAAAAAAACLABAAAA|599|Park Oak|Pkwy|Suite X|Richville|Terrell County|TX|75945|United States|-6|condo| +4275|AAAAAAAADLABAAAA|350|Park |Dr.|Suite 180|Rosebud|Seminole County|FL|32244|United States|-5|condo| +4276|AAAAAAAAELABAAAA|832|Park Center|ST|Suite G|Howell|Searcy County|AR|74854|United States|-6|condo| +4277|AAAAAAAAFLABAAAA|840|North 12th|Parkway|Suite H|Brookville|Juniata County|PA|13524|United States|-5|apartment| +4278|AAAAAAAAGLABAAAA|173|Chestnut |Ct.|Suite M|Fairfield|Eddy County|NM|86192|United States|-7|apartment| +4279|AAAAAAAAHLABAAAA|716|6th |Court|Suite A|Buena Vista|Platte County|MO|65752|United States|-6|single family| +4280|AAAAAAAAILABAAAA|235|Hill |Pkwy|Suite X|Riverdale|Sarpy County|NE|69391|United States|-7|condo| +4281|AAAAAAAAJLABAAAA|884|13th |Lane|Suite Q|Bunker Hill|Calhoun County|WV|20150|United States|-5|apartment| +4282|AAAAAAAAKLABAAAA|442|Church Second|Court|Suite 190|Riverview|Drew County|AR|79003|United States|-6|condo| +4283|AAAAAAAALLABAAAA|591|11th Park|Blvd|Suite 130|Lakewood|Washington County|UT|88877|United States|-7|condo| +4284|AAAAAAAAMLABAAAA|81|Railroad |Court|Suite 220|Five Points|Maricopa County|AZ|86098|United States|-7|single family| +4285|AAAAAAAANLABAAAA|338|Lee Ninth|Ln|Suite 210|Woodland|Stephens County|GA|34854|United States|-5|single family| +4286|AAAAAAAAOLABAAAA|833|7th |RD|Suite 40|Red Hill|Washington County|NE|64338|United States|-7|apartment| +4287|AAAAAAAAPLABAAAA|367|Park |Wy|Suite 320|Sullivan|Ontonagon County|MI|40451|United States|-5|condo| +4288|AAAAAAAAAMABAAAA|485|Seventh |Avenue|Suite E|Franklin|Newaygo County|MI|49101|United States|-5|condo| +4289|AAAAAAAABMABAAAA|455|Main |Ave|Suite H|Deerfield|Culberson County|TX|79840|United States|-6|single family| +4290|AAAAAAAACMABAAAA|466|6th |Dr.|Suite N|Mount Zion|Virginia Beach city|VA|28054|United States|-5|apartment| +4291|AAAAAAAADMABAAAA|874|Hickory Highland|Court|Suite D|Jamestown|Lumpkin County|GA|36867|United States|-5|single family| +4292|AAAAAAAAEMABAAAA|168|3rd 1st|Ct.|Suite 180|Highland|Houston County|GA|39454|United States|-5|apartment| +4293|AAAAAAAAFMABAAAA|907|Hillcrest |Boulevard|Suite X|Buena Vista|Wood County|WV|25752|United States|-5|single family| +4294|AAAAAAAAGMABAAAA|572|Wilson |Court|Suite C|Five Points|Rankin County|MS|56098|United States|-6|single family| +4295|AAAAAAAAHMABAAAA|351|Park 10th|Cir.|Suite C|Pleasant Grove|Nicholas County|KY|44136|United States|-5|single family| +4296|AAAAAAAAIMABAAAA|103|Adams 4th|Wy|Suite H|Union|Deuel County|NE|68721|United States|-6|apartment| +4297|AAAAAAAAJMABAAAA|745|2nd |ST|Suite S|Sunnyside|Dixon County|NE|61952|United States|-6|condo| +4298|AAAAAAAAKMABAAAA|184|1st Church|Way|Suite U|Oak Ridge|Rush County|IN|48371|United States|-5|single family| +4299|AAAAAAAALMABAAAA|224|Wilson |Street|Suite 150|Kingston|Johnson County|TN|34975|United States|-6|condo| +4300|AAAAAAAAMMABAAAA|507|Jefferson Lincoln|Blvd|Suite 70|Crossroads|Lycoming County|PA|10534|United States|-5|apartment| +4301|AAAAAAAANMABAAAA|171|Smith |Lane|Suite H|Union Hill|Beaver County|UT|87746|United States|-7|single family| +4302|AAAAAAAAOMABAAAA|49|Main |Avenue|Suite 380|Lebanon|Cook County|IL|62898|United States|-6|apartment| +4303|AAAAAAAAPMABAAAA|268|Woodland Main|Avenue|Suite 480|Highland Park|Bastrop County|TX|76534|United States|-6|single family| +4304|AAAAAAAAANABAAAA|972|4th Railroad|Cir.|Suite R|Crossroads|DuPage County|IL|60534|United States|-6|condo| +4305|AAAAAAAABNABAAAA|454|Lee |Dr.|Suite 210|Edgewood|Concordia Parish|LA|70069|United States|-6|single family| +4306|AAAAAAAACNABAAAA|423|Spruce |ST|Suite J|Woodlawn|Comanche County|KS|64098|United States|-6|apartment| +4307|AAAAAAAADNABAAAA|821|Fifth |Way|Suite 120|Quincy|Dinwiddie County|VA|23868|United States|-5|single family| +4308|AAAAAAAAENABAAAA|447|Railroad |Circle|Suite 200|Brownsville|Jim Wells County|TX|79310|United States|-6|condo| +4309|AAAAAAAAFNABAAAA||Pine Birch|Boulevard||Hillcrest|||53003||-6|apartment| +4310|AAAAAAAAGNABAAAA|411|Smith |Court|Suite 330|New Hope|Perry County|KY|49431|United States|-5|single family| +4311|AAAAAAAAHNABAAAA|104|Main Williams|Avenue|Suite V|Concord|Grenada County|MS|54107|United States|-6|apartment| +4312|AAAAAAAAINABAAAA|677|Miller |Avenue|Suite B|Glenwood|Page County|IA|53511|United States|-6|apartment| +4313|AAAAAAAAJNABAAAA|185|Miller |Ct.|Suite 130|Fairfield|Palo Alto County|IA|56192|United States|-6|condo| +4314|AAAAAAAAKNABAAAA|724|Sixth Second|Street|Suite 160|Indian Village|Duplin County|NC|21075|United States|-5|condo| +4315|AAAAAAAALNABAAAA|171|Forest |Ct.|Suite 460|Greenville|Dewey County|SD|51387|United States|-6|apartment| +4316|AAAAAAAAMNABAAAA|986|Center Second|Blvd|Suite 70|Salem|Worcester County|MA|08648|United States|-5|single family| +4317|AAAAAAAANNABAAAA|272|Hickory Broadway|Circle|Suite A|Shiloh|Hart County|GA|39275|United States|-5|apartment| +4318|AAAAAAAAONABAAAA|450|Maple Main|Avenue|Suite L|Glendale|Cleveland County|AR|73951|United States|-6|apartment| +4319|AAAAAAAAPNABAAAA|120|Lakeview |Circle|Suite 410|Edgewood|Worcester County|MD|20069|United States|-5|condo| +4320|AAAAAAAAAOABAAAA|123|Ash |Wy|Suite 90|Franklin|Lafayette Parish|LA|79101|United States|-6|condo| +4321|AAAAAAAABOABAAAA|147|East Main|Pkwy|Suite B|Cedar Grove|Dickens County|TX|70411|United States|-6|single family| +4322|AAAAAAAACOABAAAA|540|Park |Blvd|Suite B|Shady Grove|Augusta County|VA|22812|United States|-5|single family| +4323|AAAAAAAADOABAAAA|296|Maple Ninth|Parkway|Suite 330|Ashland|Union County|IN|44244|United States|-5|condo| +4324|AAAAAAAAEOABAAAA|320|Miller Second|Road|Suite C|Franklin|Marquette County|MI|49101|United States|-5|single family| +4325|AAAAAAAAFOABAAAA|964|Cedar |Lane|Suite 330|Woodland|Manassas Park city|VA|24854|United States|-5|apartment| +4326|AAAAAAAAGOABAAAA|295|Hickory 9th|Pkwy|Suite 10|Spring Hill|Fulton County|IL|66787|United States|-6|apartment| +4327|AAAAAAAAHOABAAAA|917|Hillcrest |Boulevard|Suite M|Springfield|De Soto Parish|LA|79303|United States|-6|apartment| +4328|AAAAAAAAIOABAAAA|458|Lake Willow|Ln|Suite X|Crossroads|Grant County|KS|60534|United States|-6|single family| +4329|AAAAAAAAJOABAAAA|248|Church Valley|Circle|Suite U|Brownsville|Carroll County|KY|49310|United States|-6|condo| +4330|AAAAAAAAKOABAAAA|578|2nd Twelfth|Avenue|Suite W|Riverview|Fayette County|IA|59003|United States|-6|condo| +4331|AAAAAAAALOABAAAA|408|Fourth |Wy|Suite B|Jamestown|Dearborn County|IN|46867|United States|-5|apartment| +4332|AAAAAAAAMOABAAAA|430|Elm |Lane|Suite 260|Greenfield|Lumpkin County|GA|35038|United States|-5|apartment| +4333|AAAAAAAANOABAAAA|401|12th Spruce|Boulevard|Suite U|Spring Valley|Sedgwick County|KS|66060|United States|-6|apartment| +4334|AAAAAAAAOOABAAAA|278|Green Lake|ST|Suite Y|Jamestown|Yuma County|AZ|86867|United States|-7|single family| +4335|AAAAAAAAPOABAAAA|959|Park |Street|Suite 470|Macedonia|Cumberland County|PA|11087|United States|-5|single family| +4336|AAAAAAAAAPABAAAA|970|Willow Cedar|Blvd|Suite Q|Oakdale|Levy County|FL|39584|United States|-5|condo| +4337|AAAAAAAABPABAAAA|301|Eigth Hickory|Lane|Suite 230|Collinsville|Rusk County|WI|52459|United States|-6|single family| +4338|AAAAAAAACPABAAAA|206|Seventh Railroad|ST|Suite H|Kingston|Modoc County|CA|94975|United States|-8|condo| +4339|AAAAAAAADPABAAAA|825|Pine |Blvd|Suite 20|Highland Park|Cleveland County|AR|76534|United States|-6|single family| +4340|AAAAAAAAEPABAAAA|947|Park 4th|Wy|Suite 480|Liberty|Crow Wing County|MN|53451|United States|-6|single family| +4341|AAAAAAAAFPABAAAA|901|Maple |Ln|Suite 40|Newtown|Archuleta County|CO|81749|United States|-7|condo| +4342|AAAAAAAAGPABAAAA|526|North |Ln|Suite K|Pleasant Hill|Houghton County|MI|43604|United States|-5|single family| +4343|AAAAAAAAHPABAAAA|749|6th Railroad|RD|Suite B|Mount Zion|Hutchinson County|TX|78054|United States|-6|condo| +4344|AAAAAAAAIPABAAAA||||Suite 110|||||United States||single family| +4345|AAAAAAAAJPABAAAA|852|Hill |Avenue|Suite 380|Lakeside|Garrett County|MD|29532|United States|-5|apartment| +4346|AAAAAAAAKPABAAAA|802|Hill Jackson|RD|Suite J|Spring Valley|Perkins County|NE|66060|United States|-7|condo| +4347|AAAAAAAALPABAAAA|596|1st |Wy|Suite 10|Deerfield|Bolivar County|MS|59840|United States|-6|apartment| +4348|AAAAAAAAMPABAAAA|440|Central 12th|RD|Suite T|Valley View|Colonial Heights city|VA|25124|United States|-5|apartment| +4349|AAAAAAAANPABAAAA|675|Cedar |RD|Suite A|Glenwood|Perry County|IN|43511|United States|-5|single family| +4350|AAAAAAAAOPABAAAA|150|7th |Wy|Suite 410|Greenfield|Ohio County|WV|25038|United States|-5|condo| +4351|AAAAAAAAPPABAAAA|407|5th |Dr.|Suite 340|Shady Grove|Pawnee County|KS|62812|United States|-6|condo| +4352|AAAAAAAAAABBAAAA|946|Church Spring|RD|Suite 0|Pleasant Valley|Beltrami County|MN|52477|United States|-6|apartment| +4353|AAAAAAAABABBAAAA|537|11th Maple|Wy|Suite 270|Greenwood|Vernon Parish|LA|78828|United States|-6|apartment| +4354|AAAAAAAACABBAAAA|441|South |Drive|Suite 330|Milo|Brooks County|GA|30116|United States|-5|condo| +4355|AAAAAAAADABBAAAA|55|2nd |Court|Suite V|Friendship|Kearney County|NE|64536|United States|-7|apartment| +4356|AAAAAAAAEABBAAAA|631|Franklin Mill|ST|Suite N|Birmingham|Miami County|IN|43372|United States|-5|condo| +4357|AAAAAAAAFABBAAAA|710|12th |Blvd|Suite N|Fairfield|Bowman County|ND|56192|United States|-6|single family| +4358|AAAAAAAAGABBAAAA|72|Williams Highland|Ct.|Suite 50|Woodlawn|Musselshell County|MT|64098|United States|-7|single family| +4359|AAAAAAAAHABBAAAA|986|Sunset |Court|Suite 280|Pine Grove|Queens County|NY|14593|United States|-5|single family| +4360|AAAAAAAAIABBAAAA|393|Washington |Court|Suite 20|Waterloo|Knox County|TN|31675|United States|-6|apartment| +4361|AAAAAAAAJABBAAAA|643|Sunset |Court|Suite D|Crossroads|Trinity County|CA|90534|United States|-8|condo| +4362|AAAAAAAAKABBAAAA|232|Green Meadow|Lane||Fairfield|Jefferson County|PA||United States|-5|condo| +4363|AAAAAAAALABBAAAA|186|Jefferson 4th|Cir.|Suite D|Unionville|Cumberland County|VA|21711|United States|-5|single family| +4364|AAAAAAAAMABBAAAA|645|1st 7th|Pkwy|Suite O|Springdale|Coleman County|TX|78883|United States|-6|apartment| +4365|AAAAAAAANABBAAAA|684|3rd Washington|Cir.|Suite X|Valley View|Harrison County|IN|45124|United States|-5|condo| +4366|AAAAAAAAOABBAAAA|954|8th |Pkwy|Suite 30|Hillcrest|Bleckley County|GA|33003|United States|-5|single family| +4367|AAAAAAAAPABBAAAA|458|Sunset Ninth|Wy|Suite 330|Brownsville|Rowan County|NC|29310|United States|-5|condo| +4368|AAAAAAAAABBBAAAA|433|Sunset |Lane|Suite 10|Pleasant Valley|Martin County|MN|52477|United States|-6|apartment| +4369|AAAAAAAABBBBAAAA|77|2nd |Blvd|Suite 450|Pleasant Grove|Hendricks County|IN|44136|United States|-5|condo| +4370|AAAAAAAACBBBAAAA|327|Main Second|Pkwy|Suite R|Valley View|Tompkins County|NY|15124|United States|-5|single family| +4371|AAAAAAAADBBBAAAA|583|6th West|Boulevard|Suite H|Oak Grove|Candler County|GA|38370|United States|-5|apartment| +4372|AAAAAAAAEBBBAAAA|63|Williams Fourth|Ave|Suite 290|Oakland|Ochiltree County|TX|79843|United States|-6|condo| +4373|AAAAAAAAFBBBAAAA|90|Locust 1st|Cir.|Suite G|Fairview|Lamoille County|VT|06309|United States|-5|apartment| +4374|AAAAAAAAGBBBAAAA|555|Washington Valley|Blvd|Suite F|Greenfield|Rutherford County|NC|25038|United States|-5|condo| +4375|AAAAAAAAHBBBAAAA|999|Cedar Pine|Lane|Suite 290|Pleasant Grove|Ashley County|AR|74136|United States|-6|single family| +4376|AAAAAAAAIBBBAAAA|||||||NY|19231|United States|-5|| +4377|AAAAAAAAJBBBAAAA|655|Center Center|Court|Suite G|Edgewood|Baker County|OR|90069|United States|-8|single family| +4378|AAAAAAAAKBBBAAAA|600|Williams |Court|Suite D|Edgewood|Jefferson Davis Parish|LA|70069|United States|-6|single family| +4379|AAAAAAAALBBBAAAA|158|6th |Ave|Suite Y|White Oak|Aransas County|TX|76668|United States|-6|condo| +4380|AAAAAAAAMBBBAAAA|913|Jackson |Ct.|Suite 130|Oakland|Coconino County|AZ|89843|United States|-7|condo| +4381|AAAAAAAANBBBAAAA|666|Elm Second|Ln|Suite J|Mount Olive|Richmond city|VA|28059|United States|-5|condo| +4382|AAAAAAAAOBBBAAAA|1000|Pine |Pkwy|Suite 40|Glenwood|Ottawa County|OH|43511|United States|-5|apartment| +4383|AAAAAAAAPBBBAAAA|639|South Madison|Drive|Suite 90|Woodland|Sullivan County|NH|05454|United States|-5|condo| +4384|AAAAAAAAACBBAAAA|243|4th |Cir.|Suite G|Red Hill|Clay County|TX|74338|United States|-6|condo| +4385|AAAAAAAABCBBAAAA|459|Dogwood Washington|Circle|Suite D|Hillcrest|Vinton County|OH|43003|United States|-5|apartment| +4386|AAAAAAAACCBBAAAA|38|Walnut |Parkway|Suite K|Enterprise|Schoolcraft County|MI|41757|United States|-5|condo| +4387|AAAAAAAADCBBAAAA|776|6th Fifth|Avenue|Suite G|Oneida|Sullivan County|MO|64027|United States|-6|condo| +4388|AAAAAAAAECBBAAAA|355|Railroad 9th|Parkway|Suite Q|Franklin|White County|IL|69101|United States|-6|condo| +4389|AAAAAAAAFCBBAAAA|146|8th East|Street|Suite A|Woodlawn|Florence County|SC|24098|United States|-5|apartment| +4390|AAAAAAAAGCBBAAAA|7|Lake |Street|Suite Y|New Hope|Cecil County|MD|29431|United States|-5|condo| +4391|AAAAAAAAHCBBAAAA|593|Poplar 11th|Drive|Suite Y|Bethel|Caldwell Parish|LA|75281|United States|-6|condo| +4392|AAAAAAAAICBBAAAA|42|Elm 6th|Dr.|Suite 470|Riverview|Jay County|IN|49003|United States|-5|condo| +4393|AAAAAAAAJCBBAAAA|993|2nd |Ln|Suite 420|Woodlawn|Pitkin County|CO|84098|United States|-7|condo| +4394|AAAAAAAAKCBBAAAA|149|Park |Lane|Suite M|Unionville|White Pine County|NV|81711|United States|-8|single family| +4395|AAAAAAAALCBBAAAA|684|Jefferson Hill|Court|Suite C|Birmingham|Yell County|AR|73372|United States|-6|condo| +4396|AAAAAAAAMCBBAAAA|646|Cherry |Parkway|Suite 440|White Oak|Lincoln County|KY|46668|United States|-5|condo| +4397|AAAAAAAANCBBAAAA|||Boulevard|Suite D|Elizabeth|Hampton County|||United States||| +4398|AAAAAAAAOCBBAAAA|213|Sunset |Drive|Suite 140|Oak Hill|Bath County|KY|47838|United States|-6|condo| +4399|AAAAAAAAPCBBAAAA|339|7th |Ln|Suite 60|Greenfield|Vernon Parish|LA|75038|United States|-6|apartment| +4400|AAAAAAAAADBBAAAA|500|Williams 10th|Drive|Suite D|Sulphur Springs|Polk County|MO|68354|United States|-6|condo| +4401|AAAAAAAABDBBAAAA|338|West |Dr.|Suite 200|Cedar Grove|Franklin County|NC|20411|United States|-5|single family| +4402|AAAAAAAACDBBAAAA|402|2nd |Parkway|Suite J|Buena Vista|Franklin County|FL|35752|United States|-5|condo| +4403|AAAAAAAADDBBAAAA|962|North |Ln|Suite L|Cedar Grove|Newport News city|VA|20411|United States|-5|apartment| +4404|AAAAAAAAEDBBAAAA|891|11th |Drive|Suite 450|Midway|Monroe County|IN|41904|United States|-5|condo| +4405|AAAAAAAAFDBBAAAA|779|6th Ridge||||Laclede County|MO|64466|||apartment| +4406|AAAAAAAAGDBBAAAA|973|Highland |Circle|Suite 60|Sunnyside|Holt County|MO|61952|United States|-6|condo| +4407|AAAAAAAAHDBBAAAA|385|7th Pine|Road|Suite 310|Kelly|Weld County|CO|82738|United States|-7|condo| +4408|AAAAAAAAIDBBAAAA|495|Elevnth |Way|Suite J|Clifton|Marin County|CA|98014|United States|-8|single family| +4409|AAAAAAAAJDBBAAAA|777||Way|Suite W|Unionville||||United States|-6|| +4410|AAAAAAAAKDBBAAAA||Cherry ||Suite Q|Mountain View||OH|||-5|| +4411|AAAAAAAALDBBAAAA|1|Meadow 7th|Dr.|Suite 400|Red Hill|Adams County|WA|94338|United States|-8|condo| +4412|AAAAAAAAMDBBAAAA||Pine North|Way|Suite H||||10191|United States|-5|| +4413|AAAAAAAANDBBAAAA|106|Fourth Lakeview|Ln|Suite F|Ashland|Washita County|OK|74244|United States|-6|single family| +4414|AAAAAAAAODBBAAAA|646|Cedar |Road|Suite M|Cedar Grove|San Miguel County|CO|80411|United States|-7|apartment| +4415|AAAAAAAAPDBBAAAA||Park Elm|Boulevard||Pleasant Grove||||United States|-5|| +4416|AAAAAAAAAEBBAAAA|587|Adams Fifth|Dr.|Suite 430|Springfield|Massac County|IL|69303|United States|-6|single family| +4417|AAAAAAAABEBBAAAA|900|Washington Mill|Pkwy|Suite 80|Lincoln|McPherson County|SD|51289|United States|-7|condo| +4418|AAAAAAAACEBBAAAA|659|Poplar |Wy|Suite K|Plainview|Multnomah County|OR|93683|United States|-8|condo| +4419|AAAAAAAADEBBAAAA|138|Hickory Oak|Lane|Suite S|Edgewood|Hardin County|IL|60069|United States|-6|single family| +4420|AAAAAAAAEEBBAAAA|612|Center |Wy|Suite L|Farmington|Caldwell Parish|LA|79145|United States|-6|condo| +4421|AAAAAAAAFEBBAAAA|643|Main Ash|Ln|Suite Q|Union Hill|Lancaster County|PA|17746|United States|-5|single family| +4422|AAAAAAAAGEBBAAAA|634|14th |Cir.|Suite Y|Riverside|Teller County|CO|89231|United States|-7|condo| +4423|AAAAAAAAHEBBAAAA|714|Sycamore |Circle|Suite 80|Hubbard|Madison County|NE|66291|United States|-7|single family| +4424|AAAAAAAAIEBBAAAA|654|Cherry Maple|Ln|Suite 220|Red Hill||LA|74338||-6|| +4425|AAAAAAAAJEBBAAAA|360|Franklin |Ct.|Suite Y|Red Hill|Kershaw County|SC|24338|United States|-5|single family| +4426|AAAAAAAAKEBBAAAA|291|8th |Lane|Suite 440|Vienna|Erie County|NY|15119|United States|-5|apartment| +4427|AAAAAAAALEBBAAAA|615|Central Second|Lane|Suite 150|Pleasant Valley|Monroe County|MS|52477|United States|-6|single family| +4428|AAAAAAAAMEBBAAAA|591|4th Central|Ln|Suite T|Riverview|Halifax County|NC|29003|United States|-5|apartment| +4429|AAAAAAAANEBBAAAA|490|Highland |Cir.|Suite 420|Stringtown|Dawson County|GA|30162|United States|-5|single family| +4430|AAAAAAAAOEBBAAAA|341|Walnut |Boulevard|Suite 70|Oak Grove|Las Animas County|CO|88370|United States|-7|condo| +4431|AAAAAAAAPEBBAAAA|714|Willow Park|Cir.|Suite 30|Ashland|Kodiak Island Borough|AK|94244|United States|-9|condo| +4432|AAAAAAAAAFBBAAAA|443|Pine |Dr.|Suite W|New Town|Pittsylvania County|VA|29634|United States|-5|condo| +4433|AAAAAAAABFBBAAAA|531|South Center|Blvd|Suite 360|Friendship|Butler County|MO|64536|United States|-6|apartment| +4434|AAAAAAAACFBBAAAA|366|Fifth 4th|Drive||Providence|Dixon County|NE|||-6|| +4435|AAAAAAAADFBBAAAA|469||ST|Suite F|Hopewell||NE||United States||condo| +4436|AAAAAAAAEFBBAAAA|147|14th |Pkwy|Suite 310|Springfield|Dorchester County|MD|29303|United States|-5|apartment| +4437|AAAAAAAAFFBBAAAA|734|Ridge |Circle|Suite T|Wildwood|Carbon County|PA|16871|United States|-5|single family| +4438|AAAAAAAAGFBBAAAA|854|Smith Hickory|Ct.|Suite I|Shady Grove|Franklin County|FL|32812|United States|-5|condo| +4439|AAAAAAAAHFBBAAAA|445|Lake |Court|Suite 50|Stringtown|Bates County|MO|60162|United States|-6|condo| +4440|AAAAAAAAIFBBAAAA|632|5th |Cir.|Suite 280|New Hope|Williamson County|TX|79431|United States|-6|single family| +4441|AAAAAAAAJFBBAAAA|825|Central Washington|Street|Suite M|Crossroads|Lexington city|VA|20534|United States|-5|condo| +4442|AAAAAAAAKFBBAAAA|748||RD|Suite 270||Burke County|ND||United States||single family| +4443|AAAAAAAALFBBAAAA|722|Highland |Avenue|Suite 110|Pine Valley|Beaver County|UT|88209|United States|-7|single family| +4444|AAAAAAAAMFBBAAAA|464|8th |Parkway|Suite 420|Liberty|Island County|WA|93451|United States|-8|apartment| +4445|AAAAAAAANFBBAAAA|60|2nd |Ct.|Suite 190|Welcome|Scotts Bluff County|NE|66386|United States|-7|single family| +4446|AAAAAAAAOFBBAAAA|450|Madison 2nd|Street|Suite 240|Springdale|Brown County|MN|58883|United States|-6|condo| +4447|AAAAAAAAPFBBAAAA|497|Sunset |Parkway|Suite 180|Riverdale|Jefferson County|MO|69391|United States|-6|single family| +4448|AAAAAAAAAGBBAAAA|635|Hickory |Ct.|Suite 10|Mount Olive|McCulloch County|TX|78059|United States|-6|apartment| +4449|AAAAAAAABGBBAAAA|990|Maple Chestnut|RD|Suite 240|Wilson|Alpine County|CA|96971|United States|-8|condo| +4450|AAAAAAAACGBBAAAA|834|11th Pine|Wy|Suite 90|Buena Vista|James City County|VA|25752|United States|-5|condo| +4451|AAAAAAAADGBBAAAA|381|View 5th|ST|Suite B|Fairview|Lee County|AR|75709|United States|-6|single family| +4452|AAAAAAAAEGBBAAAA|908|Hill |Pkwy|Suite D|Greenfield|Choctaw County|MS|55038|United States|-6|single family| +4453|AAAAAAAAFGBBAAAA|989|Davis |Pkwy|Suite K|Union|Escambia County|FL|38721|United States|-5|condo| +4454|AAAAAAAAGGBBAAAA|125|Second Lake|Court|Suite 80|Five Points|Webster County|WV|26098|United States|-5|apartment| +4455|AAAAAAAAHGBBAAAA|508|Meadow |Court|Suite P|Georgetown|James City County|VA|27057|United States|-5|apartment| +4456|AAAAAAAAIGBBAAAA|865|Pine 6th|Way|Suite 180|Perkins|Huerfano County|CO|81852|United States|-7|apartment| +4457|AAAAAAAAJGBBAAAA|588|4th 1st|Avenue|Suite U|Springfield|McPherson County|SD|59303|United States|-7|condo| +4458|AAAAAAAAKGBBAAAA|275|Lee |Boulevard|Suite 450|Shiloh|Barry County|MI|49275|United States|-5|condo| +4459|AAAAAAAALGBBAAAA|918|6th Church|Dr.|Suite Q|Midway|Denver County|CO|81904|United States|-7|apartment| +4460|AAAAAAAAMGBBAAAA|470|6th Ash|Road|Suite 90|Oakwood|Montgomery County|AL|30169|United States|-6|apartment| +4461|AAAAAAAANGBBAAAA|41|15th Birch|Drive|Suite 230|Clinton|Schuyler County|IL|68222|United States|-6|single family| +4462|AAAAAAAAOGBBAAAA|165|Adams |Drive|Suite 20|Midway|Clay County|GA|31904|United States|-5|condo| +4463|AAAAAAAAPGBBAAAA|628|Central Valley|Ct.|Suite 230|Plainview|Pike County|MO|63683|United States|-6|single family| +4464|AAAAAAAAAHBBAAAA|786|Woodland |Cir.|Suite U|New Hope|Gosper County|NE|69431|United States|-6|apartment| +4465|AAAAAAAABHBBAAAA|973|Washington |ST|Suite C|Brownsville|Washington County|ID|89310|United States|-7|apartment| +4466|AAAAAAAACHBBAAAA|||Pkwy||Oak Hill|Kennebec County|ME||United States||condo| +4467|AAAAAAAADHBBAAAA|227|Laurel 12th|Drive|Suite 170|Stringtown|Carter County|MT|60162|United States|-7|apartment| +4468|AAAAAAAAEHBBAAAA|705|Park Walnut|Wy|Suite P|New Hope|Roosevelt County|NM|89431|United States|-7|condo| +4469|AAAAAAAAFHBBAAAA|276|Elm 2nd|RD|Suite 480|Oakdale|Otsego County|MI|49584|United States|-5|condo| +4470|AAAAAAAAGHBBAAAA|136|5th Jefferson|Avenue|Suite 50|Hillcrest|Gove County|KS|63003|United States|-6|apartment| +4471|AAAAAAAAHHBBAAAA|609|Williams |Ct.|Suite F|Tremont|Hancock County|MS|59515|United States|-6|apartment| +4472|AAAAAAAAIHBBAAAA|328|4th Cherry|Boulevard|Suite H|Greenfield|Goodhue County|MN|55038|United States|-6|single family| +4473|AAAAAAAAJHBBAAAA|757|Park Willow|Dr.|Suite 150|Crossroads|Hardeman County|TX|70534|United States|-6|apartment| +4474|AAAAAAAAKHBBAAAA|107|Third 1st|Street|Suite E|Walnut Grove|Sonoma County|CA|97752|United States|-8|apartment| +4475|AAAAAAAALHBBAAAA|65|Cherry Jackson|Circle|Suite V|Shady Grove|Perkins County|NE|62812|United States|-7|single family| +4476|AAAAAAAAMHBBAAAA|289|Poplar Cherry|Avenue|Suite 270|Bunker Hill|Aleutians West Census Area|AK|90150|United States|-9|single family| +4477|AAAAAAAANHBBAAAA|499|Davis 2nd|ST|Suite 400|Pine Grove|Mingo County|WV|24593|United States|-5|apartment| +4478|AAAAAAAAOHBBAAAA|5|Dogwood Laurel|Drive|Suite M|Valley View|Karnes County|TX|75124|United States|-6|single family| +4479|AAAAAAAAPHBBAAAA|325|North |Drive|Suite X|Concord|Lincoln County|AR|74107|United States|-6|single family| +4480|AAAAAAAAAIBBAAAA|248|Railroad 2nd|Wy|Suite D|Union|Douglas County|OR|98721|United States|-8|condo| +4481|AAAAAAAABIBBAAAA|766|Church |Street|Suite D|Shady Grove|Lincoln County|GA|32812|United States|-5|apartment| +4482|AAAAAAAACIBBAAAA|396|Sunset Franklin|Street|Suite 430|Brownsville|Auglaize County|OH|49310|United States|-5|single family| +4483|AAAAAAAADIBBAAAA|73|Sixth |Ave|Suite 0|Whispering Pines|Emery County|UT|87609|United States|-7|single family| +4484|AAAAAAAAEIBBAAAA|143|Main |Blvd|Suite 350|Waterloo|Floyd County|GA|31675|United States|-5|single family| +4485|AAAAAAAAFIBBAAAA|657|6th |Road|Suite 110|Farmington|Dodge County|WI|59145|United States|-6|condo| +4486|AAAAAAAAGIBBAAAA|663|Locust |ST|Suite Q|Newport|Huron County|MI|41521|United States|-5|condo| +4487|AAAAAAAAHIBBAAAA|848|Sunset |Avenue|Suite W|Riverside|Cass County|MO|69231|United States|-6|condo| +4488|AAAAAAAAIIBBAAAA|889|9th |Boulevard|Suite 150|Oakland|Hocking County|OH|49843|United States|-5|condo| +4489|AAAAAAAAJIBBAAAA|498|6th |Circle|Suite E|Ashland|Neshoba County|MS|54244|United States|-6|apartment| +4490|AAAAAAAAKIBBAAAA|31|11th |Ct.|Suite J|Ashland|Lake and Peninsula Borough|AK|94244|United States|-9|single family| +4491|AAAAAAAALIBBAAAA|264|1st River|Circle|Suite N|Marion|Jewell County|KS|60399|United States|-6|single family| +4492|AAAAAAAAMIBBAAAA|42|Lincoln 1st|Avenue|Suite 190|Lakewood|Boulder County|CO|88877|United States|-7|condo| +4493|AAAAAAAANIBBAAAA|738|West |Cir.|Suite 340|Salem|Jackson County|IA|58048|United States|-6|apartment| +4494|AAAAAAAAOIBBAAAA|497|Ridge 1st|Ct.|Suite C|Oakwood|Erie County|OH|40169|United States|-5|single family| +4495|AAAAAAAAPIBBAAAA|716|Main |Road|Suite Y|Hamilton|Harvey County|KS|62808|United States|-6|apartment| +4496|AAAAAAAAAJBBAAAA|740|River 9th|Wy|Suite 420|Lakeview|Park County|MT|68579|United States|-7|condo| +4497|AAAAAAAABJBBAAAA|184|Center |Parkway|Suite B|Glendale|Niobrara County|WY|83951|United States|-7|condo| +4498|AAAAAAAACJBBAAAA|1000|Third |Boulevard|Suite 420|Crossroads|Wilkin County|MN|50534|United States|-6|single family| +4499|AAAAAAAADJBBAAAA|935|Lake Lake|Pkwy|Suite 400|Newport|Hood River County|OR|91521|United States|-8|apartment| +4500|AAAAAAAAEJBBAAAA|1000|Ridge |Blvd|Suite F|Murray|Minnehaha County|SD|52150|United States|-7|apartment| +4501|AAAAAAAAFJBBAAAA|3|Maple |Boulevard|Suite 230|Pleasant Valley|Waynesboro city|VA|22477|United States|-5|condo| +4502|AAAAAAAAGJBBAAAA|241|Jefferson 6th|Ln|Suite O|Clinton|Nottoway County|VA|28222|United States|-5|single family| +4503|AAAAAAAAHJBBAAAA|666|7th Maple|RD|Suite S|Brownsville|Perry County|TN|39310|United States|-6|condo| +4504|AAAAAAAAIJBBAAAA|403|Dogwood Second|Cir.|Suite 210|Reno|Major County|OK|70344|United States|-6|apartment| +4505|AAAAAAAAJJBBAAAA|560|Ash River|Wy|Suite 390|Wilson|Bracken County|KY|46971|United States|-6|condo| +4506|AAAAAAAAKJBBAAAA|992|1st |Court|Suite O|Harmony|Dauphin County|PA|15804|United States|-5|single family| +4507|AAAAAAAALJBBAAAA|206|Park |Road|Suite O|California|Stark County|ND|50141|United States|-6|condo| +4508|AAAAAAAAMJBBAAAA|860|Williams |Pkwy|Suite E|Oak Hill|Moffat County|CO|87838|United States|-7|single family| +4509|AAAAAAAANJBBAAAA|967|Maple Eigth|Circle|Suite 0|Hillcrest|Stark County|OH|43003|United States|-5|single family| +4510|AAAAAAAAOJBBAAAA|397|14th Mill|Cir.|Suite O|Lakeview|Daviess County|IN|48579|United States|-5|apartment| +4511|AAAAAAAAPJBBAAAA|412|Park |Dr.|Suite 350|Glendale|Riverside County|CA|93951|United States|-8|apartment| +4512|AAAAAAAAAKBBAAAA|769|West |RD|Suite 490|Florence|Sherman County|TX|73394|United States|-6|single family| +4513|AAAAAAAABKBBAAAA|111|Hill |Ave|Suite F|Somerville|Sunflower County|MS|57783|United States|-6|single family| +4514|AAAAAAAACKBBAAAA|771|Fourth |Dr.|Suite J|Andover|Plaquemines Parish|LA|71639|United States|-6|apartment| +4515|AAAAAAAADKBBAAAA|310||RD||Bunker Hill|Wayne County|PA|||-5|| +4516|AAAAAAAAEKBBAAAA|468|Davis Park|Wy|Suite V|Providence|McDowell County|NC|26614|United States|-5|apartment| +4517|AAAAAAAAFKBBAAAA|404|Center Second|RD|Suite V|Buena Vista|Lamoille County|VT|06352|United States|-5|apartment| +4518|AAAAAAAAGKBBAAAA|753|Lincoln |Drive|Suite V|Sunnyside|Harlan County|KY|41952|United States|-6|apartment| +4519|AAAAAAAAHKBBAAAA|701|4th Second|Ln|Suite N|Greenwood|Lake County|MI|48828|United States|-5|apartment| +4520|AAAAAAAAIKBBAAAA|165|8th |Avenue|Suite H|New Hope|Wayne County|NE|69431|United States|-7|apartment| +4521|AAAAAAAAJKBBAAAA|808|Oak |Ln|Suite D|Hopewell|Alameda County|CA|90587|United States|-8|single family| +4522|AAAAAAAAKKBBAAAA|224|5th |RD|Suite 260|Riverdale|Norfolk County|MA|09991|United States|-5|condo| +4523|AAAAAAAALKBBAAAA|52|Poplar Hill|Lane|Suite L|Mount Zion|Anne Arundel County|MD|28054|United States|-5|single family| +4524|AAAAAAAAMKBBAAAA|996|10th |Wy|Suite 60|Green Acres|Delta County|CO|87683|United States|-7|single family| +4525|AAAAAAAANKBBAAAA|870|First |Lane|Suite 30|Walnut Grove|Lake County|MT|67752|United States|-7|condo| +4526|AAAAAAAAOKBBAAAA|797|Oak |ST|Suite F|Scottsville|Lincoln County|WA|94190|United States|-8|apartment| +4527|AAAAAAAAPKBBAAAA|46|View River|Cir.|Suite D|Marion|Fredericksburg city|VA|20399|United States|-5|apartment| +4528|AAAAAAAAALBBAAAA|632|Park View|Street|Suite 250|Fairfield|Fremont County|IA|56192|United States|-6|condo| +4529|AAAAAAAABLBBAAAA|440|Elm |Pkwy|Suite 410|Walnut Grove|Williamson County|TN|37752|United States|-5|condo| +4530|AAAAAAAACLBBAAAA|33|Washington |Wy|Suite 10|Glenwood|Macon County|TN|33511|United States|-6|condo| +4531|AAAAAAAADLBBAAAA|706|Franklin College|Blvd|Suite N|Lee|Mercer County|NJ|01008|United States|-5|condo| +4532|AAAAAAAAELBBAAAA|276|View Jackson|Street|Suite 230|Pleasant Grove|Juniata County|PA|14136|United States|-5|apartment| +4533|AAAAAAAAFLBBAAAA|134|Ridge 7th|Court|Suite J|Green Acres|Magoffin County|KY|47683|United States|-5|condo| +4534|AAAAAAAAGLBBAAAA|2|7th |Pkwy|Suite K|Florence|Christian County|IL|63394|United States|-6|single family| +4535|AAAAAAAAHLBBAAAA|254|Woodland |Pkwy|Suite C|Hillcrest|Marshall County|KS|63003|United States|-6|apartment| +4536|AAAAAAAAILBBAAAA|360|Forest |RD|Suite 260|Edgewood|Dickinson County|IA|50069|United States|-6|single family| +4537|AAAAAAAAJLBBAAAA|908|Maple Valley|Lane|Suite 40|Lakewood|Hamblen County|TN|38877|United States|-5|condo| +4538|AAAAAAAAKLBBAAAA|531|Walnut 15th|Circle|Suite 180|Lincoln|Robertson County|TX|71289|United States|-6|apartment| +4539|AAAAAAAALLBBAAAA|592|14th |RD|Suite 480|Harmony|Floyd County|VA|25804|United States|-5|apartment| +4540|AAAAAAAAMLBBAAAA|674|Jefferson |Court|Suite 250|Five Forks|Leon County|FL|32293|United States|-5|condo| +4541|AAAAAAAANLBBAAAA|813|Cherry South|Ave|Suite W|Providence|Barnes County|ND|56614|United States|-6|condo| +4542|AAAAAAAAOLBBAAAA|888|Jefferson |Avenue|Suite N|Green Acres|Yuba County|CA|97683|United States|-8|single family| +4543|AAAAAAAAPLBBAAAA|648|Sycamore |Court|Suite L|Buckingham|La Salle County|TX|74092|United States|-6|single family| +4544|AAAAAAAAAMBBAAAA|461|5th |Parkway|Suite 20|Centerville|Bonneville County|ID|80059|United States|-7|apartment| +4545|AAAAAAAABMBBAAAA|276|Lakeview |Blvd|Suite 370|Woodlawn|East Feliciana Parish|LA|74098|United States|-6|condo| +4546|AAAAAAAACMBBAAAA|826|2nd |Court|Suite H|Riverdale|Jasper County|IN|49391|United States|-5|single family| +4547|AAAAAAAADMBBAAAA|840|Cherry Hill|Ct.|Suite 130|Centerville|Elbert County|GA|30059|United States|-5|apartment| +4548|AAAAAAAAEMBBAAAA|734|North 4th|Street|Suite 430|Pine Grove|Fremont County|CO|84593|United States|-7|condo| +4549|AAAAAAAAFMBBAAAA|30|Laurel Meadow|Circle|Suite 180|Riverdale|Galveston County|TX|79391|United States|-6|single family| +4550|AAAAAAAAGMBBAAAA|908|First |Parkway|Suite 150|Pleasant Hill|Perry County|IN|43604|United States|-5|apartment| +4551|AAAAAAAAHMBBAAAA|491|9th Poplar|Ave|Suite B|Lincoln|Evans County|GA|31289|United States|-5|apartment| +4552|AAAAAAAAIMBBAAAA|651|Fifth 12th|Dr.|Suite M|Springfield|DeBaca County|NM|89303|United States|-7|apartment| +4553|AAAAAAAAJMBBAAAA|528|Wilson |Boulevard|Suite 80|Edgewood|Gulf County|FL|30069|United States|-5|condo| +4554|AAAAAAAAKMBBAAAA|273|Mill |Wy|Suite Y|Mount Olive|Coles County|IL|68059|United States|-6|condo| +4555|AAAAAAAALMBBAAAA|233|4th |Parkway|Suite B|Buena Vista|McPherson County|SD|55752|United States|-7|condo| +4556|AAAAAAAAMMBBAAAA|656|Williams Green|Parkway|Suite 360|Lakeside|Brookings County|SD|59532|United States|-6|condo| +4557|AAAAAAAANMBBAAAA|29|11th 4th|Pkwy|Suite J|Walnut Grove|Walker County|GA|37752|United States|-5|condo| +4558|AAAAAAAAOMBBAAAA|313|7th Center|Court|Suite 250|Woodville|Pershing County|NV|84289|United States|-8|condo| +4559|AAAAAAAAPMBBAAAA|997|Park |Street|Suite P|Jackson|Emanuel County|GA|39583|United States|-5|apartment| +4560|AAAAAAAAANBBAAAA|171|Spring East|Street|Suite 430|Macedonia|Mecosta County|MI|41087|United States|-5|single family| +4561|AAAAAAAABNBBAAAA|128|Oak |Drive|Suite G|Hardy|Fayette County|KY|45354|United States|-6|single family| +4562|AAAAAAAACNBBAAAA|700|Main |ST|Suite 490|Harmony|Wayne County|TN|35804|United States|-6|condo| +4563|AAAAAAAADNBBAAAA|354|Second Fourth|Lane|Suite 200|Pleasant Grove|Eaton County|MI|44136|United States|-5|single family| +4564|AAAAAAAAENBBAAAA|657|10th |Pkwy|Suite Y|Walnut Grove|Lemhi County|ID|87752|United States|-7|apartment| +4565|AAAAAAAAFNBBAAAA|568|9th Hickory|Dr.|Suite 0|Sulphur Springs|Morton County|KS|68354|United States|-6|condo| +4566|AAAAAAAAGNBBAAAA|572|East |Boulevard|Suite M|Concord|Daviess County|IN|44107|United States|-5|condo| +4567|AAAAAAAAHNBBAAAA|768|Main |RD|Suite J|Florence|Alexander County|NC|23394|United States|-5|single family| +4568|AAAAAAAAINBBAAAA|948|Walnut Locust|Ln|Suite X|Smith|Quitman County|MS|57317|United States|-6|condo| +4569|AAAAAAAAJNBBAAAA|680|1st Chestnut|RD|Suite 150|Kingston|Hillsdale County|MI|44975|United States|-5|condo| +4570|AAAAAAAAKNBBAAAA|429|Pine Main|Wy|Suite 350|Cedar Grove|Stanley County|SD|50411|United States|-7|apartment| +4571|AAAAAAAALNBBAAAA|263|Third |Court|Suite 400|Plainville|Pratt County|KS|66115|United States|-6|apartment| +4572|AAAAAAAAMNBBAAAA|904|Laurel 1st|Court|Suite E|Westminster|Wells County|IN|46549|United States|-5|apartment| +4573|AAAAAAAANNBBAAAA|28|15th Main|Wy|Suite C|Mountain View|Thurston County|WA|94466|United States|-8|single family| +4574|AAAAAAAAONBBAAAA|291|Highland |Wy|Suite 470|Green Acres|Somerset County|NJ|08283|United States|-5|single family| +4575|AAAAAAAAPNBBAAAA|39|Third Oak|Ave|Suite 330|Red Hill|San Diego County|CA|94338|United States|-8|condo| +4576|AAAAAAAAAOBBAAAA|883|4th 11th|Cir.|Suite G|Five Forks|Perry County|AR|72293|United States|-6|single family| +4577|AAAAAAAABOBBAAAA|555|14th |Drive|Suite 290|Harmony|Saline County|KS|65804|United States|-6|condo| +4578|AAAAAAAACOBBAAAA|326|Spruce |Road|Suite 390|Mountain View||NY||||apartment| +4579|AAAAAAAADOBBAAAA|308|Washington |Boulevard|Suite 70|Jamestown|Cass County|MN|56867|United States|-6|apartment| +4580|AAAAAAAAEOBBAAAA|307|7th Hillcrest|Street|Suite 70|Wildwood|Union County|GA|36871|United States|-5|apartment| +4581|AAAAAAAAFOBBAAAA|659|Spruce |Lane|Suite C|Oak Grove|Whitfield County|GA|38370|United States|-5|apartment| +4582|AAAAAAAAGOBBAAAA|249|1st Main|Lane|Suite X|Bridgeport|Simpson County|KY|45817|United States|-5|apartment| +4583|AAAAAAAAHOBBAAAA|765|Park Park|Parkway|Suite 270|Greenwood|Fayette County|AL|38828|United States|-6|single family| +4584|AAAAAAAAIOBBAAAA|787|View |Court|Suite 370|Sunnyside|Cherry County|NE|61952|United States|-6|apartment| +4585|AAAAAAAAJOBBAAAA|15|15th |Road|Suite 100|Arlington|Brevard County|FL|36557|United States|-5|condo| +4586|AAAAAAAAKOBBAAAA|229|Maple Hickory|Road|Suite N|Pine Grove|Warren County|NC|24593|United States|-5|single family| +4587|AAAAAAAALOBBAAAA|168|Chestnut Third|Boulevard|Suite 70|Greenville|Ramsey County|MN|51387|United States|-6|apartment| +4588|AAAAAAAAMOBBAAAA|366|15th |Ln|Suite 420|Fairview|Cleveland County|OK|75709|United States|-6|apartment| +4589|AAAAAAAANOBBAAAA|||||White Oak|Frederick County|MD|26668||-5|single family| +4590|AAAAAAAAOOBBAAAA|432|9th Maple|Boulevard|Suite J|Springfield|Pike County|MS|59303|United States|-6|apartment| +4591|AAAAAAAAPOBBAAAA||4th 13th|Dr.||Hopewell||||United States||| +4592|AAAAAAAAAPBBAAAA|817|Lincoln |Ln|Suite K|Caledonia|Scotland County|MO|67411|United States|-6|single family| +4593|AAAAAAAABPBBAAAA|854|Lake |Ave|Suite 20|White Oak|Kent County|RI|07268|United States|-5|condo| +4594|AAAAAAAACPBBAAAA|510|Jefferson Main|Ct.|Suite Y|Hamilton|Green County|KY|42808|United States|-6|condo| +4595|AAAAAAAADPBBAAAA|599|2nd |Way|Suite D|Enterprise|Tioga County|NY|11757|United States|-5|single family| +4596|AAAAAAAAEPBBAAAA|328|Dogwood Laurel|Boulevard|Suite S|Pleasant Grove|Hunt County|TX|74136|United States|-6|apartment| +4597|AAAAAAAAFPBBAAAA|225|Sunset |Ln|Suite K|Greendale|Carbon County|WY|89343|United States|-7|apartment| +4598|AAAAAAAAGPBBAAAA|194|Hill Woodland|Parkway|Suite 140|Greenfield|Washington County|IL|65038|United States|-6|apartment| +4599|AAAAAAAAHPBBAAAA|645|Park Fourteenth|Circle|Suite F|Wilson|Van Buren County|MI|46971|United States|-5|single family| +4600|AAAAAAAAIPBBAAAA|814|Hickory Wilson|RD|Suite H|Hillcrest|Iosco County|MI|43003|United States|-5|apartment| +4601|AAAAAAAAJPBBAAAA|443|6th Smith|Ct.|Suite W|Crossroads|Jackson County|CO|80534|United States|-7|apartment| +4602|AAAAAAAAKPBBAAAA|500|Center |Drive|Suite I|Harmony|Rolette County|ND|55804|United States|-6|condo| +4603|AAAAAAAALPBBAAAA|771|4th |Pkwy|Suite H|Sulphur Springs|Cheyenne County|KS|68354|United States|-6|apartment| +4604|AAAAAAAAMPBBAAAA|640|Adams Maple|Avenue|Suite 300|Bay View|Mahnomen County|MN|56457|United States|-6|apartment| +4605|AAAAAAAANPBBAAAA|475|Fifth |Blvd|Suite O|Forest Hills|Webster County|IA|59237|United States|-6|condo| +4606|AAAAAAAAOPBBAAAA|468|Smith |Way|Suite 410|Mount Zion|Shawano County|WI|58054|United States|-6|condo| +4607|AAAAAAAAPPBBAAAA|185|West Lake|Ct.|Suite 360|Deerfield|Hubbard County|MN|59840|United States|-6|single family| +4608|AAAAAAAAAACBAAAA|345|6th 3rd|Way|Suite P|Riverview|Long County|GA|39003|United States|-5|condo| +4609|AAAAAAAABACBAAAA|614|Oak Oak|Blvd|Suite 360|Providence|Sanders County|MT|66614|United States|-7|single family| +4610|AAAAAAAACACBAAAA|430|Johnson Jefferson|Parkway|Suite 200|Midway|Mayes County|OK|71904|United States|-6|apartment| +4611|AAAAAAAADACBAAAA|261|Spring |Ave|Suite A|Maple Grove|Nelson County|ND|58252|United States|-6|condo| +4612|AAAAAAAAEACBAAAA|239|Pine Walnut|Boulevard|Suite E|Unionville|Morgan County|TN|31711|United States|-6|apartment| +4613|AAAAAAAAFACBAAAA|620|Tenth |Court|Suite N|Providence|Sharkey County|MS|56614|United States|-6|condo| +4614|AAAAAAAAGACBAAAA|496|15th |Street|Suite 390|Sunnyside|Boyd County|KY|41952|United States|-6|single family| +4615|AAAAAAAAHACBAAAA|974|Hillcrest |Way|Suite 90|Midway|Wood County|OH|41904|United States|-5|apartment| +4616|AAAAAAAAIACBAAAA|946|Jefferson 8th|Circle|Suite 240|Woodville|San Augustine County|TX|74289|United States|-6|apartment| +4617|AAAAAAAAJACBAAAA|97|8th |Wy|Suite 330|Highland Park|Weakley County|TN|36534|United States|-6|condo| +4618|AAAAAAAAKACBAAAA|609|Hill 5th|RD|Suite K|Springfield|Chase County|KS|69303|United States|-6|condo| +4619|AAAAAAAALACBAAAA|983|Hill West|Parkway|Suite U|Springdale|Autauga County|AL|38883|United States|-6|single family| +4620|AAAAAAAAMACBAAAA|420|Park |Ave|Suite 250|Macedonia|Chippewa County|MN|51087|United States|-6|condo| +4621|AAAAAAAANACBAAAA|705|Center Locust|Circle|Suite W|Five Points|McNairy County|TN|36098|United States|-6|condo| +4622|AAAAAAAAOACBAAAA|281|Cedar |Dr.|Suite B|Edwards|Caldwell County|MO|61409|United States|-6|apartment| +4623|AAAAAAAAPACBAAAA|807|Oak |RD|Suite C|Lakewood|Fillmore County|NE|68877|United States|-6|apartment| +4624|AAAAAAAAABCBAAAA|640|Hickory 2nd|Street|Suite O|Oak Ridge|Piute County|UT|88371|United States|-7|apartment| +4625|AAAAAAAABBCBAAAA|978|10th |Parkway|Suite V|Edgewood|Wilbarger County|TX|70069|United States|-6|apartment| +4626|AAAAAAAACBCBAAAA|545|Eigth View|Avenue|Suite U|Shady Grove|LaMoure County|ND|52812|United States|-6|single family| +4627|AAAAAAAADBCBAAAA|741|Lakeview |Circle|Suite B|Providence|Columbia County|PA|16614|United States|-5|single family| +4628|AAAAAAAAEBCBAAAA|350|6th |Circle|Suite 440|Newport|Camden County|NJ|02121|United States|-5|single family| +4629|AAAAAAAAFBCBAAAA|64||Ln|Suite 460||Daniels County|MT|69003|United States|-7|| +4630|AAAAAAAAGBCBAAAA|521|Lee |Blvd|Suite 270|Plainville|Grant County|WI|56115|United States|-6|apartment| +4631|AAAAAAAAHBCBAAAA|261|Franklin |Parkway|Suite 340|Woodville|Magoffin County|KY|44289|United States|-5|single family| +4632|AAAAAAAAIBCBAAAA|407|Third South|Parkway|Suite 300|Spring Hill|Halifax County|VA|26787|United States|-5|condo| +4633|AAAAAAAAJBCBAAAA|401|Willow |Lane|Suite 270|Enterprise|Ziebach County|SD|51757|United States|-6|single family| +4634|AAAAAAAAKBCBAAAA|956|Davis Park|Parkway|Suite 30|Lincoln|Crittenden County|KY|41289|United States|-6|single family| +4635|AAAAAAAALBCBAAAA|583|Pine |Dr.|Suite 310|Midway|East Baton Rouge Parish|LA|71904|United States|-6|condo| +4636|AAAAAAAAMBCBAAAA|281|6th South|Ct.|Suite 440|Edgewood|Coosa County|AL|30069|United States|-6|apartment| +4637|AAAAAAAANBCBAAAA|767|Washington |ST|Suite 320|Midway|Colfax County|NE|61904|United States|-6|apartment| +4638|AAAAAAAAOBCBAAAA|656|Center Pine|Avenue|Suite X|Marion|Logan County|OK|70399|United States|-6|condo| +4639|AAAAAAAAPBCBAAAA|980|Washington 2nd|Lane|Suite 270|Oakwood|Ness County|KS|60169|United States|-6|single family| +4640|AAAAAAAAACCBAAAA|23|14th |Wy|Suite 280|Valley View|Kewaunee County|WI|55124|United States|-6|apartment| +4641|AAAAAAAABCCBAAAA|882|4th Maple|Boulevard|Suite E|Woodbine|Franklin County|AL|34253|United States|-6|apartment| +4642|AAAAAAAACCCBAAAA|944|Fourth Hill|Road|Suite I|Green Acres|Woodbury County|IA|57683|United States|-6|condo| +4643|AAAAAAAADCCBAAAA|586|Third Adams|Ave|Suite D|Franklin|Jefferson County|FL|39101|United States|-5|apartment| +4644|AAAAAAAAECCBAAAA|62|Hickory Madison|Ct.|Suite 160|Lebanon|Laurel County|KY|42898|United States|-5|apartment| +4645|AAAAAAAAFCCBAAAA|153|Lee Sycamore|Ct.|Suite 40|Sleepy Hollow|Buckingham County|VA|23592|United States|-5|apartment| +4646|AAAAAAAAGCCBAAAA|792|2nd |Street|Suite T|Hopewell|Early County|GA|30587|United States|-5|condo| +4647|AAAAAAAAHCCBAAAA|763|Lincoln |Ln|Suite 180|Clinton|Pocahontas County|IA|58222|United States|-6|single family| +4648|AAAAAAAAICCBAAAA|194|Chestnut |Dr.|Suite S|Hamilton|Riley County|KS|62808|United States|-6|single family| +4649|AAAAAAAAJCCBAAAA|305|Oak |RD|Suite H|Mountain View|Garfield County|WA|94466|United States|-8|single family| +4650|AAAAAAAAKCCBAAAA|438|8th |Parkway|Suite G|Mount Zion|Henry County|TN|38054|United States|-5|single family| +4651|AAAAAAAALCCBAAAA|269|Oak |Blvd|Suite 470|Sulphur Springs|Plumas County|CA|98354|United States|-8|single family| +4652|AAAAAAAAMCCBAAAA|865|Maple |Street|Suite H|Plainview|Charlton County|GA|33683|United States|-5|single family| +4653|AAAAAAAANCCBAAAA|23|Lincoln 5th|Lane|Suite 250|Edgewood|Adams County|OH|40069|United States|-5|condo| +4654|AAAAAAAAOCCBAAAA||Hill Main|Avenue||||MS||||| +4655|AAAAAAAAPCCBAAAA|229|Maple Highland|Street|Suite O|Mount Olive|Los Angeles County|CA|98059|United States|-8|condo| +4656|AAAAAAAAADCBAAAA|210||Avenue||Fairview||||United States|-7|| +4657|AAAAAAAABDCBAAAA|649|View |Blvd|Suite 230|Sunnyside|Adair County|KY|41952|United States|-6|apartment| +4658|AAAAAAAACDCBAAAA|658|1st |Drive|Suite 10|Midway|Arenac County|MI|41904|United States|-5|single family| +4659|AAAAAAAADDCBAAAA|136|Spring Second|Parkway|Suite P|Centerville|Highlands County|FL|30059|United States|-5|single family| +4660|AAAAAAAAEDCBAAAA|813|Third |Drive|Suite 400|Deerfield|Carter County|KY|49840|United States|-6|apartment| +4661|AAAAAAAAFDCBAAAA|174|Spring Washington|Circle|Suite F|Jackson|Clinton County|MO|69583|United States|-6|single family| +4662|AAAAAAAAGDCBAAAA|425|Lakeview |Circle|Suite 410|Freeman|Roberts County|SD|52297|United States|-7|single family| +4663|AAAAAAAAHDCBAAAA|52|5th 11th|Blvd|Suite 410|Wilson|Ripley County|MO|66971|United States|-6|single family| +4664|AAAAAAAAIDCBAAAA|630|Smith |ST|Suite F|Fairfield|Lawrence County|AR|76192|United States|-6|apartment| +4665|AAAAAAAAJDCBAAAA|368|Railroad |Ave|Suite 170|Shiloh|Laurens County|GA|39275|United States|-5|condo| +4666|AAAAAAAAKDCBAAAA|893|Franklin |Ave|Suite 330|Kingston|Mason County|KY|44975|United States|-5|single family| +4667|AAAAAAAALDCBAAAA|155|Willow |Wy|Suite 100|Shiloh|Seminole County|OK|79275|United States|-6|apartment| +4668|AAAAAAAAMDCBAAAA|502|Jackson |Road|Suite 10|Fairfield|Crawford County|IN|46192|United States|-5|apartment| +4669|AAAAAAAANDCBAAAA|545|Pine |Drive|Suite U|Rockwood|Osage County|OK|71545|United States|-6|condo| +4670|AAAAAAAAODCBAAAA|2|Franklin |Road|Suite O|Springfield|Trigg County|KY|49303|United States|-5|apartment| +4671|AAAAAAAAPDCBAAAA|323|Highland Franklin|Avenue|Suite 270|Jackson|Kent County|TX|79583|United States|-6|apartment| +4672|AAAAAAAAAECBAAAA|582|5th Highland|Cir.|Suite V|Greenville|Webster County|GA|31387|United States|-5|single family| +4673|AAAAAAAABECBAAAA|577|Cherry Birch|Lane|Suite C|Mount Pleasant|Craig County|OK|71933|United States|-6|condo| +4674|AAAAAAAACECBAAAA|337|Fourth 6th|Avenue|Suite W|Oak Ridge|Storey County|NV|88371|United States|-8|single family| +4675|AAAAAAAADECBAAAA|||Lane|||Pennington County|MN|54338|United States||| +4676|AAAAAAAAEECBAAAA|692|Maple |Ct.|Suite X|Centerville|San Juan County|CO|80059|United States|-7|apartment| +4677|AAAAAAAAFECBAAAA|892|5th |Dr.|Suite 80|Jamestown|Dolores County|CO|86867|United States|-7|single family| +4678|AAAAAAAAGECBAAAA|657|North |Street|Suite 100|Shady Grove|Wasatch County|UT|82812|United States|-7|condo| +4679|AAAAAAAAHECBAAAA|498|Sixth |Ave|Suite B|Greenville|Knox County|IN|41387|United States|-5|condo| +4680|AAAAAAAAIECBAAAA|470|Spruce 10th|Ct.|Suite 190|Oak Hill|Elkhart County|IN|47838|United States|-5|single family| +4681|AAAAAAAAJECBAAAA|770|Church Hill|Dr.|Suite B|Harmony|Brewster County|TX|75804|United States|-6|apartment| +4682|AAAAAAAAKECBAAAA|546|8th |Cir.|Suite 50|Newtown|Jefferson County|IA|51749|United States|-6|condo| +4683|AAAAAAAALECBAAAA|862|Broadway |Boulevard|Suite T|Greenville|Porter County|IN|41387|United States|-5|condo| +4684|AAAAAAAAMECBAAAA|811|Meadow Mill|Boulevard|Suite 210|Edgewood|Bureau County|IL|60069|United States|-6|single family| +4685|AAAAAAAANECBAAAA|630|Hickory 9th|Drive|Suite 360|Maple Grove|Marquette County|WI|58252|United States|-6|apartment| +4686|AAAAAAAAOECBAAAA|13|River Church|Parkway|Suite N|Lincoln|Grady County|GA|31289|United States|-5|apartment| +4687|AAAAAAAAPECBAAAA|816|Hickory Oak|Cir.|Suite 410|Payne|Santa Barbara County|CA|96134|United States|-8|single family| +4688|AAAAAAAAAFCBAAAA|715|Third Johnson|Avenue|Suite 150|Buena Vista|Upshur County|WV|25752|United States|-5|condo| +4689|AAAAAAAABFCBAAAA|467|Oak 4th|Wy|Suite P|Enterprise|Lehigh County|PA|11757|United States|-5|apartment| +4690|AAAAAAAACFCBAAAA|748|Hill Highland|Blvd|Suite E|Liberty|Gilpin County|CO|83451|United States|-7|single family| +4691|AAAAAAAADFCBAAAA|914|Ash Lake|Cir.|Suite K|Berea|Oglethorpe County|GA|33804|United States|-5|apartment| +4692|AAAAAAAAEFCBAAAA|905|4th 7th|Lane|Suite C|Oak Grove|Natrona County|WY|88370|United States|-7|apartment| +4693|AAAAAAAAFFCBAAAA|705|Park 2nd|Avenue|Suite 50|Cedar Grove|Chattooga County|GA|30411|United States|-5|condo| +4694|AAAAAAAAGFCBAAAA|418|10th |Ct.|Suite 160|Fairfield|Hopkins County|KY|46192|United States|-6|apartment| +4695|AAAAAAAAHFCBAAAA|773|Oak Davis|Street|Suite 160|Lakeview|Cheshire County|NH|09179|United States|-5|condo| +4696|AAAAAAAAIFCBAAAA|115|||Suite 470|Florence|||73394|United States||single family| +4697|AAAAAAAAJFCBAAAA|17|Locust |Circle|Suite O|Watkins|Madison County|IN|41732|United States|-5|single family| +4698|AAAAAAAAKFCBAAAA|745|3rd |Ave|Suite U|Oakdale|Lake of the Woods County|MN|59584|United States|-6|single family| +4699|AAAAAAAALFCBAAAA|827|Franklin Pine|Circle|Suite 360|Springfield|Matagorda County|TX|79303|United States|-6|condo| +4700|AAAAAAAAMFCBAAAA|867|9th Elm|Way|Suite C|Friendship|Morgan County|WV|24536|United States|-5|apartment| +4701|AAAAAAAANFCBAAAA|272|Sixth |Way|Suite H|Centerville|Ogemaw County|MI|40059|United States|-5|condo| +4702|AAAAAAAAOFCBAAAA|287|Railroad |Street|Suite L|Harmony|Lancaster County|NE|65804|United States|-7|single family| +4703|AAAAAAAAPFCBAAAA|431|5th Woodland|Way|Suite 330|Pleasant Hill|Beckham County|OK|73604|United States|-6|single family| +4704|AAAAAAAAAGCBAAAA|606|Locust |Boulevard|Suite 140|Amherst|Bell County|KY|48119|United States|-6|condo| +4705|AAAAAAAABGCBAAAA|575|Elevnth View|Way|Suite 210|Red Hill|Montgomery County|KY|44338|United States|-5|condo| +4706|AAAAAAAACGCBAAAA|401|Park 4th|Ct.|Suite 80|Salem|Logan County|OK|78048|United States|-6|apartment| +4707|AAAAAAAADGCBAAAA|454|4th Woodland|Ct.|Suite D|Oakwood|Morgan County|GA|30169|United States|-5|single family| +4708|AAAAAAAAEGCBAAAA|786|Spruce Pine|Circle|Suite 190|Deerfield|Pike County|OH|49840|United States|-5|single family| +4709|AAAAAAAAFGCBAAAA|760|Ninth |Dr.|Suite T|Gum Springs|Worcester County|MA|02706|United States|-5|single family| +4710|AAAAAAAAGGCBAAAA|816|View |Dr.|Suite 360|Plainview||VA|23683|United States|-5|apartment| +4711|AAAAAAAAHGCBAAAA|581|Hillcrest 2nd|Cir.|Suite 30|Crossroads|Cheshire County|NH|01134|United States|-5|condo| +4712|AAAAAAAAIGCBAAAA|548|Green Wilson|Cir.|Suite O|Crossroads|Fisher County|TX|70534|United States|-6|condo| +4713|AAAAAAAAJGCBAAAA|307|9th Washington|Dr.|Suite T|Oak Grove|Hardeman County|TN|38370|United States|-5|condo| +4714|AAAAAAAAKGCBAAAA|335|Maple |Circle|Suite N|Salem|Big Stone County|MN|58048|United States|-6|condo| +4715|AAAAAAAALGCBAAAA|852|Railroad |ST|Suite 380|Bethel|Holmes County|FL|35281|United States|-5|apartment| +4716|AAAAAAAAMGCBAAAA|277|Cedar Cherry|Dr.|Suite T|Kingston|Broome County|NY|14975|United States|-5|condo| +4717|AAAAAAAANGCBAAAA|279|Park Oak|Pkwy|Suite V|Mount Pleasant|Adair County|MO|61933|United States|-6|apartment| +4718|AAAAAAAAOGCBAAAA|983|Locust |Road|Suite B|Marion|Mercer County|OH|40399|United States|-5|condo| +4719|AAAAAAAAPGCBAAAA|79|4th Maple|Ln|Suite B|Crossroads|Blair County|PA|10534|United States|-5|single family| +4720|AAAAAAAAAHCBAAAA|607|12th 13th|Ave|Suite G|Lincoln|La Salle Parish|LA|71289|United States|-6|condo| +4721|AAAAAAAABHCBAAAA|984|Willow Sunset|RD|Suite S|Silver Creek|Perry County|IL|64546|United States|-6|apartment| +4722|AAAAAAAACHCBAAAA|108|Hill Woodland|Parkway|Suite 40|Mountain View|Cape May County|NJ|05066|United States|-5|apartment| +4723|AAAAAAAADHCBAAAA|463|Pine 15th|Road|Suite F|Arlington|Ritchie County|WV|26557|United States|-5|condo| +4724|AAAAAAAAEHCBAAAA|20|Park |Avenue|Suite B|Indian Village|Snyder County|PA|11075|United States|-5|condo| +4725|AAAAAAAAFHCBAAAA|317|Poplar |Lane|Suite V|New Hope|Lake County|IL|69431|United States|-6|condo| +4726|AAAAAAAAGHCBAAAA|9|4th |Ave|Suite C|Valley View|Jefferson County|IL|65124|United States|-6|apartment| +4727|AAAAAAAAHHCBAAAA|189|3rd |Parkway|Suite 70|Edgewood|Orangeburg County|SC|20069|United States|-5|condo| +4728|AAAAAAAAIHCBAAAA|524|12th Cedar|Way|Suite B|Springfield|Sullivan County|TN|39303|United States|-6|condo| +4729|AAAAAAAAJHCBAAAA|488|Railroad Eigth|RD|Suite N|Newtown|Schuyler County|MO|61749|United States|-6|condo| +4730|AAAAAAAAKHCBAAAA|621|2nd 10th|ST|Suite Q|Walnut Grove|Mills County|IA|57752|United States|-6|single family| +4731|AAAAAAAALHCBAAAA|981|Fifth Church|Road|Suite X|Greenwood|Wyandotte County|KS|68828|United States|-6|single family| +4732|AAAAAAAAMHCBAAAA|698|Wilson |Ave|Suite 460|Pleasant Grove|Simpson County|MS|54136|United States|-6|apartment| +4733|AAAAAAAANHCBAAAA|457|Park River|Street|Suite 170|Woodville|Dawson County|GA|34289|United States|-5|condo| +4734|AAAAAAAAOHCBAAAA|694|Franklin |Pkwy|Suite 160|Oakland|Bottineau County|ND|59843|United States|-6|single family| +4735|AAAAAAAAPHCBAAAA|725|Elm |Ave|Suite L|Maple Grove|Scotland County|NC|28252|United States|-5|condo| +4736|AAAAAAAAAICBAAAA|61|Locust 7th|Avenue|Suite F|Five Points|Wood County|WV|26098|United States|-5|single family| +4737|AAAAAAAABICBAAAA|990|6th |Ct.|Suite G|Oakwood|Pitkin County|CO|80169|United States|-7|apartment| +4738|AAAAAAAACICBAAAA|275|14th |Street|Suite Y|Fairview|Aitkin County|MN|55709|United States|-6|single family| +4739|AAAAAAAADICBAAAA|426|Washington |Road|Suite 380|Springfield|Miami County|KS|69303|United States|-6|condo| +4740|AAAAAAAAEICBAAAA|494|Highland |Pkwy|Suite 60|Farmington|Rutherford County|TN|39145|United States|-6|single family| +4741|AAAAAAAAFICBAAAA|832|Jackson Cedar|Road|Suite H|Greenwood|Haines Borough|AK|98828|United States|-9|condo| +4742|AAAAAAAAGICBAAAA|427|11th |Ct.|Suite A|Greenville|Forsyth County|GA|31387|United States|-5|single family| +4743|AAAAAAAAHICBAAAA|61|Park Williams|Road|Suite T|Harmon|Kandiyohi County|MN|55623|United States|-6|apartment| +4744|AAAAAAAAIICBAAAA|489|Washington 10th|Pkwy|Suite 430|Lebanon|New Castle County|DE|12898|United States|-5|single family| +4745|AAAAAAAAJICBAAAA|419|9th 11th|Street|Suite S|Summit|Mellette County|SD|50499|United States|-7|apartment| +4746|AAAAAAAAKICBAAAA|10|1st |Pkwy|Suite 70|Oakdale|Dorchester County|SC|29584|United States|-5|apartment| +4747|AAAAAAAALICBAAAA|44|Poplar 2nd|Ln|Suite O|Walnut Grove|Cumberland County|NC|27752|United States|-5|apartment| +4748|AAAAAAAAMICBAAAA||Park ||Suite 80|||||||single family| +4749|AAAAAAAANICBAAAA|424|Green |Avenue|Suite F|Woodland|Franklin County|AL|34854|United States|-6|apartment| +4750|AAAAAAAAOICBAAAA|19|Third |Court|Suite 320|Five Forks|Dallas County|AL|32293|United States|-6|condo| +4751|AAAAAAAAPICBAAAA|411|Lake |Avenue|Suite M|Newport|McMinn County|TN|31521|United States|-6|single family| +4752|AAAAAAAAAJCBAAAA|431|Pine |Court|Suite 430|Shiloh|Iron County|UT|89275|United States|-7|apartment| +4753|AAAAAAAABJCBAAAA|395|Second Fifth|Ln|Suite H|Five Forks|McCracken County|KY|42293|United States|-5|condo| +4754|AAAAAAAACJCBAAAA|703|1st |Parkway|Suite 130|Union Hill|Dutchess County|NY|17746|United States|-5|condo| +4755|AAAAAAAADJCBAAAA|998|4th |Blvd|Suite G|Wilson|Pickens County|SC|26971|United States|-5|single family| +4756|AAAAAAAAEJCBAAAA|617|5th Spring|Ave|Suite W|Pleasant Grove|Boone County|MO|64136|United States|-6|condo| +4757|AAAAAAAAFJCBAAAA|941|View 7th|Parkway|Suite D|Highland|Alleghany County|NC|29454|United States|-5|single family| +4758|AAAAAAAAGJCBAAAA|585|Laurel |Ct.|Suite X|Summit|Lancaster County|PA|10499|United States|-5|single family| +4759|AAAAAAAAHJCBAAAA|48|Wilson Meadow||Suite D|Deerfield|||59840||-6|| +4760|AAAAAAAAIJCBAAAA|130|2nd |Circle|Suite P|Fairfield|Wexford County|MI|46192|United States|-5|apartment| +4761|AAAAAAAAJJCBAAAA|234|Valley Jefferson|Blvd|Suite J|Riverview|Mason County|WV|29003|United States|-5|single family| +4762|AAAAAAAAKJCBAAAA|266|Ridge |Cir.|Suite 130|Fairview|Putnam County|IN|45709|United States|-5|apartment| +4763|AAAAAAAALJCBAAAA|734|3rd Elm|Drive|Suite D|Glendale|Greene County|NY|13951|United States|-5|condo| +4764|AAAAAAAAMJCBAAAA|821|Valley |Ave|Suite K|Woodlawn|Covington city|VA|24098|United States|-5|condo| +4765|AAAAAAAANJCBAAAA|458|Miller |Ave|Suite 180|Buena Vista|Mingo County|WV|25752|United States|-5|apartment| +4766|AAAAAAAAOJCBAAAA|667|Broadway Railroad|Road|Suite 440|Belfast|Franklin County|WA|90125|United States|-8|apartment| +4767|AAAAAAAAPJCBAAAA|951|College First|Road|Suite P|Brownsville|Wilkes County|NC|29310|United States|-5|apartment| +4768|AAAAAAAAAKCBAAAA|87|Maple |Drive|Suite 360|Mountain View|Candler County|GA|34466|United States|-5|apartment| +4769|AAAAAAAABKCBAAAA|704|Laurel |Ave|Suite 110|Bridgeport|Sumner County|TN|35817|United States|-6|condo| +4770|AAAAAAAACKCBAAAA|469|Lakeview |Street|Suite 280|Woodville|Clallam County|WA|94289|United States|-8|single family| +4771|AAAAAAAADKCBAAAA|862|Davis |Street|Suite 50|Liberty|Hamilton County|KS|63451|United States|-6|condo| +4772|AAAAAAAAEKCBAAAA|38|Smith |Wy|Suite 340|New Salem|Isanti County|MN|59568|United States|-6|apartment| +4773|AAAAAAAAFKCBAAAA||Franklin Franklin|Boulevard|Suite 440||Miller County|AR||||| +4774|AAAAAAAAGKCBAAAA|995|2nd |RD|Suite 380|Concord|Ellis County|KS|64107|United States|-6|condo| +4775|AAAAAAAAHKCBAAAA|894|South |Ave|Suite 310|Midway|Marshall County|KS|61904|United States|-6|apartment| +4776|AAAAAAAAIKCBAAAA|679|Franklin |Boulevard|Suite G|Waterloo|Buchanan County|IA|51675|United States|-6|single family| +4777|AAAAAAAAJKCBAAAA|612|5th Elm|Boulevard|Suite D|Riverdale|Fayette County|GA|39391|United States|-5|condo| +4778|AAAAAAAAKKCBAAAA|731|Seventh College|Way|Suite P|Concord|Tift County|GA|34107|United States|-5|single family| +4779|AAAAAAAALKCBAAAA|386|Lee |Ave|Suite 170|Newtown|Calhoun County|IL|61749|United States|-6|condo| +4780|AAAAAAAAMKCBAAAA|829|Walnut Woodland|Parkway|Suite M|Hopewell|Chouteau County|MT|60587|United States|-7|condo| +4781|AAAAAAAANKCBAAAA|511|Forest |Way|Suite 170|Leon|Garfield County|OK|70913|United States|-6|condo| +4782|AAAAAAAAOKCBAAAA|611|Maple Madison|Drive|Suite 210|Hamilton|Clay County|KS|62808|United States|-6|apartment| +4783|AAAAAAAAPKCBAAAA|451|12th South|Court|Suite 20|White Oak|Faulk County|SD|56668|United States|-6|condo| +4784|AAAAAAAAALCBAAAA|965|Oak |Drive|Suite 140|Oak Hill|Alpine County|CA|97838|United States|-8|single family| +4785|AAAAAAAABLCBAAAA|599|Third Cherry|Court|Suite 10|Berea|Kitsap County|WA|93804|United States|-8|apartment| +4786|AAAAAAAACLCBAAAA|11|Oak Railroad|ST|Suite G|Hopewell|Zapata County|TX|70587|United States|-6|single family| +4787|AAAAAAAADLCBAAAA|719|Johnson |Ave|Suite U|Marion|Ketchikan Gateway Borough|AK|90399|United States|-9|condo| +4788|AAAAAAAAELCBAAAA|217|Valley Woodland|Pkwy|Suite W|Newport|Cherokee County|TX|71521|United States|-6|single family| +4789|AAAAAAAAFLCBAAAA|415||Ct.|||Pocahontas County|||United States||| +4790|AAAAAAAAGLCBAAAA|55|Maple Highland|Cir.|Suite M|Pine Grove|Benton County|WA|94593|United States|-8|condo| +4791|AAAAAAAAHLCBAAAA|2|Miller |RD|Suite 60|Franklin|Barbour County|WV|29101|United States|-5|apartment| +4792|AAAAAAAAILCBAAAA|511|6th Main|RD|Suite K|Woodland|Oconto County|WI|54854|United States|-6|apartment| +4793|AAAAAAAAJLCBAAAA|13|Pine Mill|Way|Suite 430|Mountain View|Murray County|OK|74466|United States|-6|condo| +4794|AAAAAAAAKLCBAAAA|24|Lincoln |Avenue|Suite 360|Oak Grove|Graves County|KY|48370|United States|-6|apartment| +4795|AAAAAAAALLCBAAAA|633|Pine |Road|Suite 230|Mountain View|Dane County|WI|54466|United States|-6|condo| +4796|AAAAAAAAMLCBAAAA|319|Church Oak|Court|Suite 220|New Hope|Simpson County|MS|59431|United States|-6|apartment| +4797|AAAAAAAANLCBAAAA|530|Maple Spring|Cir.|Suite 70|Oak Hill|San Juan County|CO|87838|United States|-7|condo| +4798|AAAAAAAAOLCBAAAA|429|First |Dr.|Suite B|Mountain View|Cherokee County|GA|34466|United States|-5|apartment| +4799|AAAAAAAAPLCBAAAA|385|Cedar |||||NE|||-6|| +4800|AAAAAAAAAMCBAAAA||||||Phelps County|NE|||-7|apartment| +4801|AAAAAAAABMCBAAAA|547|Main Lee|Blvd|Suite D|Glenwood|Lyon County|NV|83511|United States|-8|apartment| +4802|AAAAAAAACMCBAAAA|774|Fourth |Parkway|Suite 200|Highland|Hayes County|NE|69454|United States|-6|condo| +4803|AAAAAAAADMCBAAAA|752|Fourth Birch|Street|Suite 410|White Oak|McLean County|IL|66668|United States|-6|condo| +4804|AAAAAAAAEMCBAAAA|925|Locust Fifth|ST|Suite P|Brownsville|Clark County|OH|49310|United States|-5|single family| +4805|AAAAAAAAFMCBAAAA|945|Hillcrest |Way|Suite P|Mountain View|Morgan County|WV|24466|United States|-5|condo| +4806|AAAAAAAAGMCBAAAA|907|West |Cir.|Suite A|Sunnyside|Preble County|OH|41952|United States|-5|apartment| +4807|AAAAAAAAHMCBAAAA|526|Cherry Lincoln|Pkwy|Suite L|Oakwood|Bradley County|AR|70169|United States|-6|single family| +4808|AAAAAAAAIMCBAAAA|736|Spruce Dogwood|Boulevard|Suite 70|Georgetown|Jim Hogg County|TX|77057|United States|-6|single family| +4809|AAAAAAAAJMCBAAAA|514|15th |Drive|Suite 110|Jackson|Creek County|OK|79583|United States|-6|condo| +4810|AAAAAAAAKMCBAAAA|941|Sunset South|Ct.|Suite X|Sunnyside|Garrard County|KY|41952|United States|-6|condo| +4811|AAAAAAAALMCBAAAA|797|7th Walnut|Cir.|Suite W|Arlington|Codington County|SD|56557|United States|-6|apartment| +4812|AAAAAAAAMMCBAAAA|66|Davis Elm|Ave|Suite 30|Lakewood|Washington County|TN|38877|United States|-6|single family| +4813|AAAAAAAANMCBAAAA|305|Sunset |Circle|Suite 270|Jamestown|Perquimans County|NC|26867|United States|-5|single family| +4814|AAAAAAAAOMCBAAAA|274|Lee |Avenue|Suite 390|Riverside|Charles Mix County|SD|59231|United States|-6|single family| +4815|AAAAAAAAPMCBAAAA|535|10th Sunset|Ave|Suite S|Crossroads|Jones County|NC|20534|United States|-5|single family| +4816|AAAAAAAAANCBAAAA|849|6th |RD|Suite 430|Springfield|Carbon County|UT|89303|United States|-7|condo| +4817|AAAAAAAABNCBAAAA|405|Meadow |RD|Suite 10|Bridgeport|Gallia County|OH|45817|United States|-5|single family| +4818|AAAAAAAACNCBAAAA|615|Main |ST|Suite 300|Gary|Christian County|IL|60418|United States|-6|single family| +4819|AAAAAAAADNCBAAAA|86|Spring 3rd|Court|Suite 460|Greenfield|Butler County|OH|45038|United States|-5|apartment| +4820|AAAAAAAAENCBAAAA|299|Forest 1st|Ln|Suite Y|Jackson|Harford County|MD|29583|United States|-5|condo| +4821|AAAAAAAAFNCBAAAA|226|River |Pkwy|Suite 30|Hopewell|Pike County|IL|60587|United States|-6|apartment| +4822|AAAAAAAAGNCBAAAA|143|Lee |Ln|Suite 440|Springfield|Haywood County|TN|39303|United States|-5|apartment| +4823|AAAAAAAAHNCBAAAA|197|Ash |Ln|Suite 20|Wilson|Forsyth County|NC|26971|United States|-5|apartment| +4824|AAAAAAAAINCBAAAA|422|Birch 5th|Cir.|Suite 250|Riverdale|McPherson County|NE|69391|United States|-7|condo| +4825|AAAAAAAAJNCBAAAA|201|Pine |Dr.|Suite N|Cedar Grove|Pettis County|MO|60411|United States|-6|condo| +4826|AAAAAAAAKNCBAAAA|72|Walnut 4th|Blvd|Suite V|Pine Grove|Phillips County|CO|84593|United States|-7|apartment| +4827|AAAAAAAALNCBAAAA|595|3rd 4th|Street|Suite D|Clifton|Fayette County|PA|18014|United States|-5|single family| +4828|AAAAAAAAMNCBAAAA|113|Dogwood Chestnut|Court|Suite 180|Brownsville|Jefferson County|KS|69310|United States|-6|apartment| +4829|AAAAAAAANNCBAAAA|520|Elm |Road|Suite 30|Five Forks|Grainger County|TN|32293|United States|-5|single family| +4830|AAAAAAAAONCBAAAA|314|Laurel Fifth|Street|Suite 270|Wilson|Grant County|OR|96971|United States|-8|apartment| +4831|AAAAAAAAPNCBAAAA|545|Sixth |RD|Suite U|Plainview|Long County|GA|33683|United States|-5|apartment| +4832|AAAAAAAAAOCBAAAA|542|2nd |Parkway|Suite F|Green Acres|Chattooga County|GA|37683|United States|-5|single family| +4833|AAAAAAAABOCBAAAA|444|Hickory |Ct.|Suite 330|Maple Grove|Portage County|WI|58252|United States|-6|single family| +4834|AAAAAAAACOCBAAAA|498|Park Wilson|Road|Suite 210|Floyd|Kiowa County|KS|63235|United States|-6|apartment| +4835|AAAAAAAADOCBAAAA|356|Woodland Central|Ct.|Suite 200|Cedar Grove|Susquehanna County|PA|10411|United States|-5|condo| +4836|AAAAAAAAEOCBAAAA|558|Church Washington|Blvd|Suite 70|Friendship|Milam County|TX|74536|United States|-6|apartment| +4837|AAAAAAAAFOCBAAAA|73|Pine Spruce|Cir.|Suite W|Fairview|Denver County|CO|85709|United States|-7|single family| +4838|AAAAAAAAGOCBAAAA|315|Madison |Circle|Suite R|Jenkins|Antrim County|MI|47292|United States|-5|single family| +4839|AAAAAAAAHOCBAAAA|665|9th |Ct.|Suite 280|Highland Park|Riverside County|CA|96534|United States|-8|apartment| +4840|AAAAAAAAIOCBAAAA|39|11th |Parkway|Suite I|Hamilton|Columbia County|PA|12808|United States|-5|condo| +4841|AAAAAAAAJOCBAAAA|1|West 5th|Boulevard|Suite 180|Glenwood|Sheridan County|WY|83511|United States|-7|condo| +4842|AAAAAAAAKOCBAAAA|937|Lincoln |Street|Suite J|Cedar Grove|Kent County|RI|01011|United States|-5|apartment| +4843|AAAAAAAALOCBAAAA|628|Hickory Hillcrest|Ln|Suite E|Deerfield|Todd County|KY|49840|United States|-5|condo| +4844|AAAAAAAAMOCBAAAA|536|Woodland |Ct.|Suite A|Green Acres|Petroleum County|MT|67683|United States|-7|condo| +4845|AAAAAAAANOCBAAAA|461|Highland Ash|||||||United States|-6|condo| +4846|AAAAAAAAOOCBAAAA|||Way|Suite 180|Newport|||41521||-5|condo| +4847|AAAAAAAAPOCBAAAA|148|Washington |Ln|Suite Y|Green Acres|Yukon-Koyukuk Census Area|AK|97683|United States|-9|single family| +4848|AAAAAAAAAPCBAAAA|799|Lake |Court|Suite K|Acme|Clay County|IL|60164|United States|-6|condo| +4849|AAAAAAAABPCBAAAA|619|Railroad |Ct.|Suite E|Lincoln|Wabash County|IL|61289|United States|-6|single family| +4850|AAAAAAAACPCBAAAA|353|Ridge South|Lane|Suite 180|Lakeside|Greene County|AR|79532|United States|-6|single family| +4851|AAAAAAAADPCBAAAA|454|8th Spring|Blvd|Suite Y|Riverview|Ontario County|NY|19003|United States|-5|single family| +4852|AAAAAAAAEPCBAAAA|666|4th Oak|Way|Suite 380|Providence|Elk County|PA|16614|United States|-5|apartment| +4853|AAAAAAAAFPCBAAAA|300|Hillcrest Johnson|Pkwy|Suite 370|Spring Hill|Crane County|TX|76787|United States|-6|single family| +4854|AAAAAAAAGPCBAAAA|621|Second Elm|RD|Suite F|Lakewood|Jefferson County|IA|58877|United States|-6|single family| +4855|AAAAAAAAHPCBAAAA|386|West |Drive|Suite 380|Mount Olive|Clear Creek County|CO|88059|United States|-7|condo| +4856|AAAAAAAAIPCBAAAA|88|9th Hillcrest|Parkway|Suite I|Waterloo|Hillsborough County|NH|02275|United States|-5|single family| +4857|AAAAAAAAJPCBAAAA|2|Main Fourth|Drive|Suite J|Five Points|Hendricks County|IN|46098|United States|-5|apartment| +4858|AAAAAAAAKPCBAAAA|918|Jackson Laurel|RD|Suite H|Tanglewood|Boone County|MO|68994|United States|-6|single family| +4859|AAAAAAAALPCBAAAA|889|Hill |Ct.|Suite D|Lake Forest|Iroquois County|IL|66000|United States|-6|condo| +4860|AAAAAAAAMPCBAAAA|744|North Jefferson|Ct.|Suite G|Mount Olive|Valdez-Cordova Census Area|AK|98059|United States|-9|single family| +4861|AAAAAAAANPCBAAAA|510|4th Walnut|Blvd|Suite M|Mountain View|Nolan County|TX|74466|United States|-6|apartment| +4862|AAAAAAAAOPCBAAAA|501|||Suite R|Harmony|Cooke County|TX|75804|United States|-6|| +4863|AAAAAAAAPPCBAAAA|991|Sixth |Circle|Suite H|Pleasant Hill|Franklin County|ME|04204|United States|-5|single family| +4864|AAAAAAAAAADBAAAA|968|Adams Valley|Road|Suite 420|Salem|Douglas County|NV|88048|United States|-8|single family| +4865|AAAAAAAABADBAAAA|769|Ridge South|Parkway|Suite S|Martinsville|Telfair County|GA|30419|United States|-5|apartment| +4866|AAAAAAAACADBAAAA|391|Lee 9th|Wy|Suite Y|Avery|Jefferson County|FL|30194|United States|-5|single family| +4867|AAAAAAAADADBAAAA|116|Washington Cherry|Street|Suite Y|Riceville|Colquitt County|GA|35867|United States|-5|apartment| +4868|AAAAAAAAEADBAAAA|694|7th |Lane|Suite 110|Maple Grove|Runnels County|TX|78252|United States|-6|condo| +4869|AAAAAAAAFADBAAAA|105|Elm Chestnut|Avenue|Suite R|Brownsville|Putnam County|TN|39310|United States|-6|condo| +4870|AAAAAAAAGADBAAAA|833|13th |ST|Suite D|Oakdale|Gage County|NE|69584|United States|-6|single family| +4871|AAAAAAAAHADBAAAA|517|Hill |Ave|Suite 300|Lincoln|Currituck County|NC|21289|United States|-5|condo| +4872|AAAAAAAAIADBAAAA|632|||Suite 360|Waterloo||||United States|-6|| +4873|AAAAAAAAJADBAAAA|731|Lake Green|Drive|Suite Y|Centerville|Washington County|AR|70059|United States|-6|condo| +4874|AAAAAAAAKADBAAAA|393|View |Ct.|Suite 40|New Hope|Menard County|TX|79431|United States|-6|single family| +4875|AAAAAAAALADBAAAA|838|Poplar |Street|Suite K|Mount Pleasant|Clark County|SD|51933|United States|-6|condo| +4876|AAAAAAAAMADBAAAA|839|13th View|Way|Suite 390|Springdale|Mason County|WA|98883|United States|-8|condo| +4877|AAAAAAAANADBAAAA|528|7th Lakeview|Street|Suite 90|Greenfield|Vilas County|WI|55038|United States|-6|apartment| +4878|AAAAAAAAOADBAAAA|717|4th Maple|Pkwy|Suite 50|Deerfield|Stafford County|KS|69840|United States|-6|single family| +4879|AAAAAAAAPADBAAAA|669|Lee 4th|Road|Suite 40|Harmon|Forest County|WI|55623|United States|-6|condo| +4880|AAAAAAAAABDBAAAA|558|4th |Dr.|Suite 430|Oak Ridge|Cheatham County|TN|38371|United States|-5|apartment| +4881|AAAAAAAABBDBAAAA|880|3rd Center|Boulevard|Suite 350|Brownsville|Fulton County|OH|49310|United States|-5|single family| +4882|AAAAAAAACBDBAAAA|675|Cedar Highland|Ln|Suite 300|Jamestown|Boundary County|ID|86867|United States|-7|single family| +4883|AAAAAAAADBDBAAAA|149|View Williams|Ct.|Suite L|Mount Pleasant|Washington County|GA|31933|United States|-5|single family| +4884|AAAAAAAAEBDBAAAA|773|Dogwood Church|Dr.|Suite K|Buena Vista|El Dorado County|CA|95752|United States|-8|apartment| +4885|AAAAAAAAFBDBAAAA|61|Laurel 1st|Street|Suite 210|Spring Valley|Weld County|CO|86060|United States|-7|single family| +4886|AAAAAAAAGBDBAAAA|437|Birch |Ln|Suite W|Fairfield|White Pine County|NV|86192|United States|-8|single family| +4887|AAAAAAAAHBDBAAAA|260|Ash |Drive|Suite 350|Newport|Rutland County|VT|02121|United States|-5|single family| +4888|AAAAAAAAIBDBAAAA||Woodland 5th|Ct.||Riverview|Harford County|MD||||single family| +4889|AAAAAAAAJBDBAAAA|823|4th 8th||Suite J|Edgewood|Troup County||30069|United States||condo| +4890|AAAAAAAAKBDBAAAA|207|Church |Street|Suite 370|Bunker Hill|Saguache County|CO|80150|United States|-7|condo| +4891|AAAAAAAALBDBAAAA|980|Jackson 3rd|Dr.|Suite X|Five Points|Meagher County|MT|66098|United States|-7|condo| +4892|AAAAAAAAMBDBAAAA|272|Park Washington|Court|Suite 390|Bridgeport|Johnston County|OK|75817|United States|-6|apartment| +4893|AAAAAAAANBDBAAAA|133|Oak Pine|Wy|Suite 460|Antioch|Winchester city|VA|28605|United States|-5|single family| +4894|AAAAAAAAOBDBAAAA|965|3rd Pine|Dr.|Suite 250|Summit|Beaufort County|NC|20499|United States|-5|apartment| +4895|AAAAAAAAPBDBAAAA|668|Sunset |Ave|Suite 310|Farmington|Delaware County|IN|49145|United States|-5|apartment| +4896|AAAAAAAAACDBAAAA|181|Lincoln |Cir.|Suite I|White Oak|DeKalb County|MO|66668|United States|-6|condo| +4897|AAAAAAAABCDBAAAA|412|11th Main|Dr.|Suite 370|Brentwood|Matagorda County|TX|74188|United States|-6|single family| +4898|AAAAAAAACCDBAAAA|61|Poplar Ridge|Blvd|Suite T|Lucas|Clay County|FL|34554|United States|-5|condo| +4899|AAAAAAAADCDBAAAA|833|South |Ct.|Suite 0|Oak Ridge|Nottoway County|VA|28371|United States|-5|single family| +4900|AAAAAAAAECDBAAAA|273|Church Sycamore|Boulevard|Suite 350|Woodland|Caldwell Parish|LA|74854|United States|-6|condo| +4901|AAAAAAAAFCDBAAAA|114|River Lakeview|Street|Suite 360|Oakdale|Huron County|OH|49584|United States|-5|single family| +4902|AAAAAAAAGCDBAAAA|964|10th |Pkwy|Suite O|Lebanon|Marion County|KS|62898|United States|-6|single family| +4903|AAAAAAAAHCDBAAAA|742|6th |Blvd|Suite 340|Newport|El Paso County|CO|81521|United States|-7|condo| +4904|AAAAAAAAICDBAAAA|70|Madison Elm|Avenue|Suite F|Hillcrest|Montgomery County|NY|13003|United States|-5|condo| +4905|AAAAAAAAJCDBAAAA|680|Elevnth |Lane|Suite M|Edgewood|Juneau Borough|AK|90069|United States|-9|condo| +4906|AAAAAAAAKCDBAAAA|990|Church |ST|Suite J|Farmington|Starke County|IN|49145|United States|-5|apartment| +4907|AAAAAAAALCDBAAAA|584|||||Multnomah County|OR||United States|-8|single family| +4908|AAAAAAAAMCDBAAAA|247|Park |Circle|Suite B|Stringtown|Niagara County|NY|10162|United States|-5|apartment| +4909|AAAAAAAANCDBAAAA|553|Walnut |ST|Suite W|Oakdale|Knox County|TX|79584|United States|-6|condo| +4910|AAAAAAAAOCDBAAAA|492|14th |Parkway|Suite 100|Shiloh|Waynesboro city|VA|29275|United States|-5|condo| +4911|AAAAAAAAPCDBAAAA|576|Church 9th|Lane|Suite C|Springdale|Camden County|GA|38883|United States|-5|apartment| +4912|AAAAAAAAADDBAAAA|679|Oak Valley|Way|Suite 110|Five Points|White County|IN|46098|United States|-5|apartment| +4913|AAAAAAAABDDBAAAA|252|Oak 2nd|Cir.|Suite 60|Fairview|Winneshiek County|IA|55709|United States|-6|apartment| +4914|AAAAAAAACDDBAAAA|439|3rd |Pkwy|Suite U|Belmont|Douglas County|OR|90191|United States|-8|single family| +4915|AAAAAAAADDDBAAAA|645|Elm 2nd|Ave|Suite P|Oakwood|Toombs County|GA|30169|United States|-5|condo| +4916|AAAAAAAAEDDBAAAA|15|Forest |Wy|Suite 410|Clinton|Worth County|MO|68222|United States|-6|single family| +4917|AAAAAAAAFDDBAAAA|426|Jackson Oak|Dr.|Suite 320|Greenville|Greenwood County|KS|61387|United States|-6|single family| +4918|AAAAAAAAGDDBAAAA|812|6th 10th|Dr.|Suite 240|Forest Hills|Sherman County|NE|69237|United States|-7|single family| +4919|AAAAAAAAHDDBAAAA|474|Oak |ST|Suite J|Pleasant Grove|Pike County|MO|64136|United States|-6|single family| +4920|AAAAAAAAIDDBAAAA|608|2nd Church|Street|Suite 390|Centerville|Prentiss County|MS|50059|United States|-6|condo| +4921|AAAAAAAAJDDBAAAA|290|Johnson Jackson|Ave|Suite S|Deerfield|King County|WA|99840|United States|-8|single family| +4922|AAAAAAAAKDDBAAAA|779|Elm |Dr.|Suite V|Highland|Randolph County|WV|29454|United States|-5|condo| +4923|AAAAAAAALDDBAAAA|275|8th |Way|Suite 200|Riverview|Wagoner County|OK|79003|United States|-6|single family| +4924|AAAAAAAAMDDBAAAA|891|Cedar Lee|Pkwy|Suite O|Springdale|Jackson County|AR|78883|United States|-6|apartment| +4925|AAAAAAAANDDBAAAA|957|Willow |Street|Suite 300|Florence|Hardin County|TN|33394|United States|-5|single family| +4926|AAAAAAAAODDBAAAA|394|West |Way|Suite M|Shiloh|Jefferson County|MO|69275|United States|-6|condo| +4927|AAAAAAAAPDDBAAAA|30|11th |Road|Suite 390|White Oak|San Francisco County|CA|96668|United States|-8|apartment| +4928|AAAAAAAAAEDBAAAA||||||Comal County||70008|||single family| +4929|AAAAAAAABEDBAAAA|25|Jackson 1st|Court|Suite X|Springdale|Coffey County|KS|68883|United States|-6|condo| +4930|AAAAAAAACEDBAAAA|95|Locust |Road|Suite 60|Quincy|Laurens County|GA|33868|United States|-5|apartment| +4931|AAAAAAAADEDBAAAA|621|1st |Cir.|Suite M|Salem|Taylor County|TX|78048|United States|-6|condo| +4932|AAAAAAAAEEDBAAAA|267|Franklin |Court|Suite H|Midway|Karnes County|TX|71904|United States|-6|condo| +4933|AAAAAAAAFEDBAAAA|201|Cherry Eigth|Parkway|Suite M|Greenwood|Fremont County|WY|88828|United States|-7|apartment| +4934|AAAAAAAAGEDBAAAA|346|Elevnth Locust|Drive|Suite Q|Liberty|Edgecombe County|NC|23451|United States|-5|single family| +4935|AAAAAAAAHEDBAAAA|120|1st 10th|Ln|Suite 310|Fairbanks|Lamar County|MS|56653|United States|-6|single family| +4936|AAAAAAAAIEDBAAAA|416|Hill Lake|Circle|Suite A|Oakwood|McCone County|MT|60169|United States|-7|single family| +4937|AAAAAAAAJEDBAAAA|776|Wilson |Way|Suite 30|Jackson|Harney County|OR|99583|United States|-8|single family| +4938|AAAAAAAAKEDBAAAA|483|Franklin |Pkwy|Suite 160|Wilson|Bristol city|VA|26971|United States|-5|single family| +4939|AAAAAAAALEDBAAAA|113|Park 4th|Dr.|Suite W|New Hope|Armstrong County|TX|79431|United States|-6|condo| +4940|AAAAAAAAMEDBAAAA|641|Forest |ST|Suite 210|Bunker Hill|Pasquotank County|NC|20150|United States|-5|condo| +4941|AAAAAAAANEDBAAAA|239|Washington |Street|Suite 430|Springfield|Cherokee County|SC|29303|United States|-5|apartment| +4942|AAAAAAAAOEDBAAAA|837|Park |Ave|Suite 240|Jackson|Adams County|MS|59583|United States|-6|single family| +4943|AAAAAAAAPEDBAAAA|491|Fourth |Pkwy|Suite 80|Clifton|Flagler County|FL|38014|United States|-5|apartment| +4944|AAAAAAAAAFDBAAAA|40|Sunset |Dr.|Suite M|Hopewell|Buchanan County|IA|50587|United States|-6|apartment| +4945|AAAAAAAABFDBAAAA|851|Center Cherry|Road|Suite 200|Wesley|Bristol city|VA|21218|United States|-5|single family| +4946|AAAAAAAACFDBAAAA|431|7th |RD|Suite 60|Mount Olive|Switzerland County|IN|48059|United States|-5|apartment| +4947|AAAAAAAADFDBAAAA||||Suite 410||Haywood County|NC|||-5|apartment| +4948|AAAAAAAAEFDBAAAA||Railroad |RD|Suite W||||50150|United States|-7|| +4949|AAAAAAAAFFDBAAAA|115|Central |Ct.|Suite 90|Hillcrest|Georgetown County|SC|23003|United States|-5|condo| +4950|AAAAAAAAGFDBAAAA|146|Forest Railroad||Suite S|Spring Valley|Pope County||56060|United States|-6|single family| +4951|AAAAAAAAHFDBAAAA|944|Johnson |Pkwy|Suite 410|Lee|Wasatch County|UT|80408|United States|-7|condo| +4952|AAAAAAAAIFDBAAAA|73|West |RD|Suite 110|Enterprise|Wyoming County|WV|21757|United States|-5|single family| +4953|AAAAAAAAJFDBAAAA|798|Fifteenth |Cir.|Suite H|Clifton|Palo Pinto County|TX|78014|United States|-6|apartment| +4954|AAAAAAAAKFDBAAAA|463|1st |Wy|Suite 10|Mount Vernon|Hancock County|IL|68482|United States|-6|apartment| +4955|AAAAAAAALFDBAAAA|785|Main |Ct.|Suite 240|Foster|Polk County|MN|58587|United States|-6|single family| +4956|AAAAAAAAMFDBAAAA|839|Green Eigth|Road|Suite 250|Deerfield|Screven County|GA|39840|United States|-5|condo| +4957|AAAAAAAANFDBAAAA|667|7th |Ln|Suite 460|Hamilton|Churchill County|NV|82808|United States|-8|condo| +4958|AAAAAAAAOFDBAAAA|223|Mill |Street|Suite U|Buena Vista|Jefferson County|ID|85752|United States|-7|apartment| +4959|AAAAAAAAPFDBAAAA|372|11th |Road|Suite 160|Kingston|Rice County|MN|54975|United States|-6|apartment| +4960|AAAAAAAAAGDBAAAA|62|Willow |Dr.|Suite 420|Buena Vista|Clarke County|VA|25752|United States|-5|apartment| +4961|AAAAAAAABGDBAAAA|44|2nd |Cir.|Suite 110|Jackson|Trinity County|CA|99583|United States|-8|apartment| +4962|AAAAAAAACGDBAAAA|735|Center Park|Avenue|Suite 140|Union|Rockcastle County|KY|48721|United States|-5|condo| +4963|AAAAAAAADGDBAAAA|680|Adams 7th|Court|Suite V|Spring Valley|Livingston County|MI|46060|United States|-5|single family| +4964|AAAAAAAAEGDBAAAA|533|Green |Parkway|Suite 280|Florence|Hernando County|FL|33394|United States|-5|single family| +4965|AAAAAAAAFGDBAAAA|943|View Third|Circle|Suite 180|Hillcrest|Big Horn County|WY|83003|United States|-7|condo| +4966|AAAAAAAAGGDBAAAA|117|Ridge South|Dr.|Suite 100|Woodlawn|Mahaska County|IA|54098|United States|-6|single family| +4967|AAAAAAAAHGDBAAAA|657|Fifth |Boulevard|Suite 70|Lake Forest|Monroe County|OH|46000|United States|-5|single family| +4968|AAAAAAAAIGDBAAAA|242|Lee Cedar|Ave|Suite M|Georgetown|Washington County|PA|17057|United States|-5|apartment| +4969|AAAAAAAAJGDBAAAA|507|Sunset Cherry|Ct.|Suite C|Antioch|Dallas County|MO|68605|United States|-6|apartment| +4970|AAAAAAAAKGDBAAAA|370|Park Walnut|RD|Suite W|Newtown|Miami County|KS|61749|United States|-6|single family| +4971|AAAAAAAALGDBAAAA|102|Willow |Wy|Suite T|Enterprise|Clay County|KY|41757|United States|-6|apartment| +4972|AAAAAAAAMGDBAAAA|916|Oak Oak|Dr.|Suite Y|Springdale|Adair County|KY|48883|United States|-6|condo| +4973|AAAAAAAANGDBAAAA|125|Sunset 3rd|Ln|Suite 260|Lakeside|Putnam County|FL|39532|United States|-5|single family| +4974|AAAAAAAAOGDBAAAA|255|12th |Ln|Suite 410|Greenville|Lexington city|VA|21387|United States|-5|single family| +4975|AAAAAAAAPGDBAAAA|573|College Fourth|ST|Suite F|Lincoln|Maries County|MO|61289|United States|-6|apartment| +4976|AAAAAAAAAHDBAAAA|619|Cedar |Cir.|Suite N|Cedar Grove|Camden County|GA|30411|United States|-5|single family| +4977|AAAAAAAABHDBAAAA|306|Hickory |Parkway|Suite X|Mountain View|Gallatin County|KY|44466|United States|-6|single family| +4978|AAAAAAAACHDBAAAA|65|College Main|Drive|Suite Y|Newtown|Placer County|CA|91749|United States|-8|apartment| +4979|AAAAAAAADHDBAAAA|299|Main Main|Dr.|Suite 470|Red Hill|Kenosha County|WI|54338|United States|-6|apartment| +4980|AAAAAAAAEHDBAAAA|149|Maple 4th|Way|Suite 190|Forest Hills|Hardin County|OH|49237|United States|-5|single family| +4981|AAAAAAAAFHDBAAAA|488|3rd Park|Court|Suite D|Fairfield|Okfuskee County|OK|76192|United States|-6|apartment| +4982|AAAAAAAAGHDBAAAA|141|Woodland Park|Road|Suite 40|Mount Pleasant|Carlisle County|KY|41933|United States|-6|condo| +4983|AAAAAAAAHHDBAAAA|382|View Maple|Lane|Suite B|Riverside|Roanoke County|VA|29231|United States|-5|apartment| +4984|AAAAAAAAIHDBAAAA||Hill Spring|Circle|Suite 490||||47683|||| +4985|AAAAAAAAJHDBAAAA|275|15th |Court|Suite 490|Globe|Union County|NM|89614|United States|-7|single family| +4986|AAAAAAAAKHDBAAAA|612|Locust Adams|Blvd|Suite P|Greenwood|Gilpin County|CO|88828|United States|-7|condo| +4987|AAAAAAAALHDBAAAA|793|Fourth |Way|Suite 290|Lakewood|Ashland County|WI|58877|United States|-6|condo| +4988|AAAAAAAAMHDBAAAA|523|Washington |Blvd|Suite 440|Union|Benton County|TN|38721|United States|-5|apartment| +4989|AAAAAAAANHDBAAAA|794|Oak |Circle|Suite X|Lakeside|Carroll County|VA|29532|United States|-5|single family| +4990|AAAAAAAAOHDBAAAA|229|Franklin |Avenue|Suite 390|Springfield|Nantucket County|MA|09903|United States|-5|single family| +4991|AAAAAAAAPHDBAAAA|433|1st |ST|Suite 350|Antioch|Wells County|ND|58605|United States|-6|condo| +4992|AAAAAAAAAIDBAAAA|36|9th Fifth|RD|Suite B|Glenwood|Marion County|FL|33511|United States|-5|condo| +4993|AAAAAAAABIDBAAAA|514|Smith |Dr.|Suite M|Shady Grove|Knox County|OH|42812|United States|-5|single family| +4994|AAAAAAAACIDBAAAA|91|South |ST|Suite P|Stringtown|Ziebach County|SD|50162|United States|-6|condo| +4995|AAAAAAAADIDBAAAA|181|Miller 4th|Avenue|Suite 70|Belmont|Laramie County|WY|80191|United States|-7|single family| +4996|AAAAAAAAEIDBAAAA|552|River Lake|Wy|Suite G|Wildwood|Erie County|NY|16871|United States|-5|single family| +4997|AAAAAAAAFIDBAAAA|105|Second Park|Ave|Suite O|Pleasant Valley|Trego County|KS|62477|United States|-6|single family| +4998|AAAAAAAAGIDBAAAA|121|Lincoln Fourth|Drive|Suite U|Salem|Charlotte County|FL|38048|United States|-5|condo| +4999|AAAAAAAAHIDBAAAA|219|Lake |Ln|Suite A|Franklin|Winkler County|TX|79101|United States|-6|apartment| +5000|AAAAAAAAIIDBAAAA|962|West Eigth|ST|Suite R|Pine Grove|Claiborne Parish|LA|74593|United States|-6|single family| +5001|AAAAAAAAJIDBAAAA|106|Hill |Pkwy|Suite 200|Ingleside|Duplin County|NC|24356|United States|-5|single family| +5002|AAAAAAAAKIDBAAAA|556|Hill |Cir.|Suite 250|Woodville|Pottawatomie County|KS|64289|United States|-6|apartment| +5003|AAAAAAAALIDBAAAA|873|11th |Blvd|Suite 360|Oakdale|Blackford County|IN|49584|United States|-5|single family| +5004|AAAAAAAAMIDBAAAA|775|Walnut Smith|ST|Suite 360|Spring Hill|Fayette County|KY|46787|United States|-6|condo| +5005|AAAAAAAANIDBAAAA|517|Williams 11th|Ct.|Suite G|Five Points|Lincoln County|CO|86098|United States|-7|condo| +5006|AAAAAAAAOIDBAAAA|709|||Suite 230||||||-6|apartment| +5007|AAAAAAAAPIDBAAAA|720|Sycamore |RD|Suite 360|Prosperity|Menifee County|KY|49089|United States|-5|condo| +5008|AAAAAAAAAJDBAAAA|644|Main |Lane|Suite M|Greenwood|Escambia County|AL|38828|United States|-6|apartment| +5009|AAAAAAAABJDBAAAA|105|Third |Pkwy|Suite 260|Mount Vernon|Tooele County|UT|88482|United States|-7|apartment| +5010|AAAAAAAACJDBAAAA|146|Davis |Ct.|Suite 340|Kingston|Stevens County|WA|94975|United States|-8|apartment| +5011|AAAAAAAADJDBAAAA|487|Woodland |Cir.|Suite E|Hopewell|Jackson County|TN|30587|United States|-5|apartment| +5012|AAAAAAAAEJDBAAAA|905|Park Church|Ct.|Suite J|Clifton|Barbour County|AL|38014|United States|-6|apartment| +5013|AAAAAAAAFJDBAAAA|585|Cherry Jefferson|Boulevard|Suite 300|Red Hill|Randolph County|MO|64338|United States|-6|single family| +5014|AAAAAAAAGJDBAAAA||||||Delaware County|OK|74466|||| +5015|AAAAAAAAHJDBAAAA|88|Eigth |Ave|Suite 430|Macedonia|Harlan County|KY|41087|United States|-6|single family| +5016|AAAAAAAAIJDBAAAA|760|Spring |Street|Suite P|Marion|Pierce County|GA|30399|United States|-5|single family| +5017|AAAAAAAAJJDBAAAA|785|Washington |Pkwy|Suite 360|Bethel|Union County|IA|55281|United States|-6|condo| +5018|AAAAAAAAKJDBAAAA|912|Locust |Street|Suite A|Silver Springs|Hardin County|TN|34843|United States|-5|condo| +5019|AAAAAAAALJDBAAAA|498|Third Sixth|Court|Suite K|Greenfield|Humphreys County|TN|35038|United States|-5|condo| +5020|AAAAAAAAMJDBAAAA|590|Davis |Ln|Suite P|Newtown|Amelia County|VA|21749|United States|-5|apartment| +5021|AAAAAAAANJDBAAAA|610|River Lincoln|Blvd|Suite P|Valley View|Washington County|ME|05724|United States|-5|apartment| +5022|AAAAAAAAOJDBAAAA|605|Lincoln |Cir.|Suite 390|Milan|Pepin County|WI|56697|United States|-6|single family| +5023|AAAAAAAAPJDBAAAA|519|8th Maple|Blvd|Suite T|Woodlawn|Gloucester County|VA|24098|United States|-5|condo| +5024|AAAAAAAAAKDBAAAA|281|8th |Pkwy|Suite G|Rosewood|Harding County|NM|82205|United States|-7|condo| +5025|AAAAAAAABKDBAAAA|650|Sycamore |Ln|Suite K|Highland|Honolulu County|HI|99454|United States|-10|apartment| +5026|AAAAAAAACKDBAAAA|606|Wilson Park|Cir.|Suite 270|Oak Grove|San Augustine County|TX|78370|United States|-6|apartment| +5027|AAAAAAAADKDBAAAA|882|Adams Oak|Road|Suite G|Brownsville|Fillmore County|MN|59310|United States|-6|single family| +5028|AAAAAAAAEKDBAAAA|666|View 13th|Boulevard|Suite 260|Belmont|Lake and Peninsula Borough|AK|90191|United States|-9|condo| +5029|AAAAAAAAFKDBAAAA|381|Mill |Dr.|Suite 180|Maywood|McClain County|OK|75681|United States|-6|apartment| +5030|AAAAAAAAGKDBAAAA|877|Washington |Blvd|Suite S|Hillcrest|Columbia County|GA|33003|United States|-5|single family| +5031|AAAAAAAAHKDBAAAA|207|1st Walnut|Ave|Suite A|Georgetown|Fremont County|WY|87057|United States|-7|apartment| +5032|AAAAAAAAIKDBAAAA|207||Parkway|Suite K||||46867|||apartment| +5033|AAAAAAAAJKDBAAAA|271|Park |Avenue|Suite K|Ashland|Stone County|MO|64244|United States|-6|single family| +5034|AAAAAAAAKKDBAAAA|315|East Oak|Road|Suite 480|Springdale|Chisago County|MN|58883|United States|-6|condo| +5035|AAAAAAAALKDBAAAA|228|1st |Blvd|Suite 10|Harmony|Winnebago County|WI|55804|United States|-6|condo| +5036|AAAAAAAAMKDBAAAA|863|Ash 2nd|Ct.|Suite 20|Brookwood|Kleberg County|TX|70965|United States|-6|single family| +5037|AAAAAAAANKDBAAAA|787|Birch |Road|Suite C|Fairview|Dunn County|WI|55709|United States|-6|single family| +5038|AAAAAAAAOKDBAAAA|614|East |ST|Suite K|Newtown|Butler County|PA|11749|United States|-5|apartment| +5039|AAAAAAAAPKDBAAAA|537|4th |Ave|Suite 350|Kingston|Thurston County|NE|64975|United States|-7|single family| +5040|AAAAAAAAALDBAAAA|513|Cedar |Ln|Suite V|Greenwood|Allegan County|MI|48828|United States|-5|apartment| +5041|AAAAAAAABLDBAAAA|818|Cedar Hillcrest|Blvd|Suite J|Oakwood|Marengo County|AL|30169|United States|-6|apartment| +5042|AAAAAAAACLDBAAAA|599|Tenth |Road|Suite 300|Waterloo|Hardee County|FL|31675|United States|-5|apartment| +5043|AAAAAAAADLDBAAAA|828|Davis |RD|Suite 320|Oak Hill|Faulk County|SD|57838|United States|-6|apartment| +5044|AAAAAAAAELDBAAAA|311|7th Park|Ct.|Suite A|Deerfield|Pike County|KY|49840|United States|-5|condo| +5045|AAAAAAAAFLDBAAAA|4|Cedar Walnut|Drive|Suite M|Five Points|Contra Costa County|CA|96098|United States|-8|condo| +5046|AAAAAAAAGLDBAAAA|251|Elm Valley|RD|Suite 490|Lebanon|Pacific County|WA|92898|United States|-8|apartment| +5047|AAAAAAAAHLDBAAAA|432|Washington |Blvd|Suite 220|Franklin|Florence County|WI|59101|United States|-6|apartment| +5048|AAAAAAAAILDBAAAA|12|Pine |Ave|Suite 230|Spring Valley|Burke County|NC|26060|United States|-5|condo| +5049|AAAAAAAAJLDBAAAA|256|East Poplar|Ave|Suite X|Pleasant Valley|Linn County|OR|92477|United States|-8|single family| +5050|AAAAAAAAKLDBAAAA|682|Main |Ave|Suite F|Oakland|Henrico County|VA|29843|United States|-5|single family| +5051|AAAAAAAALLDBAAAA|913||||Georgetown|Somerset County||27057|||| +5052|AAAAAAAAMLDBAAAA|933||Ln||Centerville|||70059|United States|-6|| +5053|AAAAAAAANLDBAAAA|526|Church 1st|Court|Suite 310|Lakewood|Andrew County|MO|68877|United States|-6|apartment| +5054|AAAAAAAAOLDBAAAA|292|Cedar Center|Way|Suite 200|Centerville|Terrell County|TX|70059|United States|-6|apartment| +5055|AAAAAAAAPLDBAAAA|639|Hickory Second|Ln|Suite M|Pleasant Grove|Chouteau County|MT|64136|United States|-7|condo| +5056|AAAAAAAAAMDBAAAA|737|9th |Wy|Suite 490|Five Forks|Wayne County|TN|32293|United States|-6|apartment| +5057|AAAAAAAABMDBAAAA|341|3rd |Drive|Suite Q|Newport|McCormick County|SC|21521|United States|-5|apartment| +5058|AAAAAAAACMDBAAAA|374|8th Pine|Court|Suite 10|Woodland|Berkeley County|SC|24854|United States|-5|apartment| +5059|AAAAAAAADMDBAAAA|264|West Franklin|Road|Suite 90|Fairfield|Scurry County|TX|76192|United States|-6|condo| +5060|AAAAAAAAEMDBAAAA|773|Hill |Ave|Suite 80|Mount Vernon|New York County|NY|18482|United States|-5|apartment| +5061|AAAAAAAAFMDBAAAA|700|Lee Broadway|Street|Suite P|Sunnyside|Coos County|OR|91952|United States|-8|apartment| +5062|AAAAAAAAGMDBAAAA|844|Wilson |Street|Suite 240|Mount Vernon|Westchester County|NY|18482|United States|-5|apartment| +5063|AAAAAAAAHMDBAAAA|981|Hill |Wy|Suite C|Unionville|Park County|CO|81711|United States|-7|condo| +5064|AAAAAAAAIMDBAAAA|44|11th Oak|Court|Suite 110|Five Forks|Merrimack County|NH|02893|United States|-5|single family| +5065|AAAAAAAAJMDBAAAA|869|Seventh Lake|Cir.|Suite 300|Stringtown|Marion County|OR|90162|United States|-8|condo| +5066|AAAAAAAAKMDBAAAA|601|Fifth Maple|Pkwy|Suite F|Brownsville|Simpson County|KY|49310|United States|-5|condo| +5067|AAAAAAAALMDBAAAA|158|Davis |Blvd|Suite X|Arlington|Wright County|IA|56557|United States|-6|single family| +5068|AAAAAAAAMMDBAAAA|527|Wilson Maple|Boulevard|Suite 160|Pine Grove|Prentiss County|MS|54593|United States|-6|single family| +5069|AAAAAAAANMDBAAAA|123|View Sycamore|Boulevard|Suite 340|Forest Hills|Door County|WI|59237|United States|-6|apartment| +5070|AAAAAAAAOMDBAAAA|423|Ridge River|Lane|Suite F|Hurricane|Vermilion Parish|LA|77644|United States|-6|apartment| +5071|AAAAAAAAPMDBAAAA|80|Forest |Wy|Suite 90|Riverview|Kiowa County|OK|79003|United States|-6|apartment| +5072|AAAAAAAAANDBAAAA|66|Spring |Street|Suite 140|Lebanon|Jefferson County|AL|32898|United States|-6|single family| +5073|AAAAAAAABNDBAAAA|985|River |Cir.|Suite 300|Jamestown|Miller County|GA|36867|United States|-5|apartment| +5074|AAAAAAAACNDBAAAA|7|Lee Walnut|Wy|Suite G|Pleasant Grove|Wilkes County|GA|34136|United States|-5|condo| +5075|AAAAAAAADNDBAAAA|729|3rd |Drive|Suite A|Plainview|Kittson County|MN|53683|United States|-6|condo| +5076|AAAAAAAAENDBAAAA|118|7th |Street|Suite Y|Friendship|Escambia County|FL|34536|United States|-5|condo| +5077|AAAAAAAAFNDBAAAA|680|Spruce 5th|Ct.|Suite 260|Fowler|Carroll County|IN|41083|United States|-5|condo| +5078|AAAAAAAAGNDBAAAA|911|Church |Parkway|Suite 190|Hopewell|Forest County|WI|50587|United States|-6|apartment| +5079|AAAAAAAAHNDBAAAA|123|3rd |Road|Suite H|Riverdale|Breckinridge County|||United States|-6|apartment| +5080|AAAAAAAAINDBAAAA|436||Avenue|||Orleans Parish|LA|79237|||| +5081|AAAAAAAAJNDBAAAA|462|6th 1st|Dr.|Suite Y|Oak Grove|Lynchburg city|VA|28370|United States|-5|single family| +5082|AAAAAAAAKNDBAAAA|109|Sycamore |Circle|Suite 110|Union Hill|Teller County|CO|87746|United States|-7|apartment| +5083|AAAAAAAALNDBAAAA|828|||||||||-7|| +5084|AAAAAAAAMNDBAAAA|349|6th |Court|Suite V|Woodland Hills|Lincoln County|OK|71254|United States|-6|apartment| +5085|AAAAAAAANNDBAAAA|974|1st |Blvd|Suite I|Sulphur Springs|Middlesex County|VA|28354|United States|-5|single family| +5086|AAAAAAAAONDBAAAA|306|Laurel |Blvd||||IN|44136||-5|| +5087|AAAAAAAAPNDBAAAA|270|Johnson Walnut|Blvd|Suite O|Pine Grove|Pettis County|MO|64593|United States|-6|apartment| +5088|AAAAAAAAAODBAAAA|13|East |Drive|Suite 220|Union Hill|Scott County|IL|67746|United States|-6|single family| +5089|AAAAAAAABODBAAAA|474|15th |Dr.|Suite 80|Summit|Grant Parish|LA|70499|United States|-6|apartment| +5090|AAAAAAAACODBAAAA|||Wy||Red Hill|||||-6|single family| +5091|AAAAAAAADODBAAAA|826|Park |Parkway|Suite 270|Harmon|Cloud County|KS|65623|United States|-6|single family| +5092|AAAAAAAAEODBAAAA|90|Williams 3rd|Street|Suite 400|Summit|Karnes County|TX|70499|United States|-6|single family| +5093|AAAAAAAAFODBAAAA|605|Fifth |Parkway|Suite 240|Clinton|New Hanover County|NC|28222|United States|-5|single family| +5094|AAAAAAAAGODBAAAA|377|1st Forest|Court|Suite 430|Kingston|Lewis County|WV|24975|United States|-5|single family| +5095|AAAAAAAAHODBAAAA|525|Meadow |Way|Suite H|Edgewood|Victoria County|TX|70069|United States|-6|single family| +5096|AAAAAAAAIODBAAAA|967|North Forest|Avenue|Suite 280|Lakeview|Greene County|IA|58579|United States|-6|condo| +5097|AAAAAAAAJODBAAAA|92|Church Fourth|Dr.|Suite K|Oak Hill|Lorain County|OH|47838|United States|-5|condo| +5098|AAAAAAAAKODBAAAA|665|Second |Road|Suite H|Midway|Baker County|OR|91904|United States|-8|single family| +5099|AAAAAAAALODBAAAA|155|Cedar Walnut|Dr.|Suite U|Riverside|Crowley County|CO|89231|United States|-7|single family| +5100|AAAAAAAAMODBAAAA|495|2nd Sunset|Ave|Suite 400|Lakeview|Meriwether County|GA|38579|United States|-5|condo| +5101|AAAAAAAANODBAAAA||12th Spruce|Way|Suite R||||||-6|apartment| +5102|AAAAAAAAOODBAAAA|413|8th |Dr.|Suite 470|Maple Grove|Union County|NJ|08852|United States|-5|single family| +5103|AAAAAAAAPODBAAAA|702|2nd |Way|Suite E|Pleasant Hill|Brown County|OH|43604|United States|-5|condo| +5104|AAAAAAAAAPDBAAAA|198|Oak |ST|Suite K|Macon|Pittsburg County|OK|70369|United States|-6|single family| +5105|AAAAAAAABPDBAAAA|354|Smith Green|RD|Suite 450|Shady Grove|Scott County|VA|22812|United States|-5|apartment| +5106|AAAAAAAACPDBAAAA|312|Madison 7th|Ct.|Suite G|Newport|Chester County|SC|21521|United States|-5|apartment| +5107|AAAAAAAADPDBAAAA|568|7th |Court|Suite X|Floyd|Major County|OK|73235|United States|-6|condo| +5108|AAAAAAAAEPDBAAAA|73|Oak |Wy|Suite H|Franklin|Mason County|KY|49101|United States|-5|single family| +5109|AAAAAAAAFPDBAAAA|941|View |Dr.|Suite 100|Sunnyside|Allen County|OH|41952|United States|-5|single family| +5110|AAAAAAAAGPDBAAAA|561|Lake |Dr.|Suite 480|Farmington|Kings County|CA|99145|United States|-8|condo| +5111|AAAAAAAAHPDBAAAA|293|Cedar Main|Pkwy|Suite U|Clinton|Harrison County|MS|58222|United States|-6|apartment| +5112|AAAAAAAAIPDBAAAA|981|Franklin 4th|Ln|Suite 440|Bridgeport|Independence County|AR|75817|United States|-6|apartment| +5113|AAAAAAAAJPDBAAAA|729|9th |Boulevard|Suite U|Fairview|San Miguel County|NM|85709|United States|-7|apartment| +5114|AAAAAAAAKPDBAAAA|673|Hill |Pkwy|Suite C|Mount Olive|Pulaski County|GA|38059|United States|-5|condo| +5115|AAAAAAAALPDBAAAA|669|Pine Park|Ave|Suite G|Concord|Choctaw County|OK|74107|United States|-6|apartment| +5116|AAAAAAAAMPDBAAAA|479|Second Fifth|Circle|Suite P|Greenville|Mineral County|WV|21387|United States|-5|condo| +5117|AAAAAAAANPDBAAAA|372|13th Cedar|Cir.|Suite R|Tabor|Dixie County|FL|38529|United States|-5|single family| +5118|AAAAAAAAOPDBAAAA|591|Cherry |Dr.|Suite 300|Hamilton|Marshall County|IL|62808|United States|-6|condo| +5119|AAAAAAAAPPDBAAAA|807|Fifth Main|Boulevard|Suite 180|Lakeview|Fulton County|KY|48579|United States|-6|apartment| +5120|AAAAAAAAAAEBAAAA|334|Lincoln 1st|Cir.|Suite 470|Pleasant Grove|Tillman County|OK|74136|United States|-6|condo| +5121|AAAAAAAABAEBAAAA|87|11th View|Boulevard|Suite I|Riverview|Howell County|MO|69003|United States|-6|single family| +5122|AAAAAAAACAEBAAAA|343|Hickory |RD|Suite P|Centerville|Love County|OK|70059|United States|-6|apartment| +5123|AAAAAAAADAEBAAAA|280|1st |Pkwy|Suite 390|Green Acres|Knox County|NE|67683|United States|-7|condo| +5124|AAAAAAAAEAEBAAAA|554|Dogwood |Blvd|Suite A|Oakwood|Meade County|KS|60169|United States|-6|condo| +5125|AAAAAAAAFAEBAAAA|212|Hill |Drive|Suite 230|Ferguson|Addison County|VT|02421|United States|-5|single family| +5126|AAAAAAAAGAEBAAAA|638|Pine Birch|Parkway|Suite 290|Florence|Jefferson County|WA|93394|United States|-8|condo| +5127|AAAAAAAAHAEBAAAA|824|9th |Blvd|Suite 450|Ashland|Canadian County|OK|74244|United States|-6|single family| +5128|AAAAAAAAIAEBAAAA|482|Laurel |Pkwy|Suite R|Glendale|Effingham County|IL|63951|United States|-6|apartment| +5129|AAAAAAAAJAEBAAAA|200|1st |ST|Suite 440|White Oak|Zapata County|TX|76668|United States|-6|apartment| +5130|AAAAAAAAKAEBAAAA|873|Cedar North|Ct.|Suite 110|Edgewood|Monroe County|KY|40069|United States|-5|apartment| +5131|AAAAAAAALAEBAAAA|67|14th |Parkway|Suite 490|White Oak|Polk County|NC|26668|United States|-5|condo| +5132|AAAAAAAAMAEBAAAA|757|Spring |Pkwy|Suite 410|Oak Ridge|Gregg County|TX|78371|United States|-6|apartment| +5133|AAAAAAAANAEBAAAA|414|Pine |Blvd|Suite H|Lakeside|Newton County|MS|59532|United States|-6|single family| +5134|AAAAAAAAOAEBAAAA|709|7th |Drive|Suite X|Riverdale|Franklin County|TN|39391|United States|-5|apartment| +5135|AAAAAAAAPAEBAAAA|553|Lake Lake|Parkway|Suite 30|Forest Hills|Nassau County|FL|39237|United States|-5|apartment| +5136|AAAAAAAAABEBAAAA|238|South |Drive|Suite S|Stringtown|Tipton County|TN|30162|United States|-6|single family| +5137|AAAAAAAABBEBAAAA|211|Park Elm|Wy||Spring Hill|Jackson County|IN|46787|||| +5138|AAAAAAAACBEBAAAA|799|Third |Ct.|Suite 150|Glenwood|Harrison County|TX|73511|United States|-6|apartment| +5139|AAAAAAAADBEBAAAA|376|Pine East|Pkwy|Suite C|Wildwood|Parker County|TX|76871|United States|-6|single family| +5140|AAAAAAAAEBEBAAAA|154|Locust Sunset|Cir.|Suite 150|Springdale|Glynn County|GA|38883|United States|-5|apartment| +5141|AAAAAAAAFBEBAAAA|772|Fifth 13th|Way|Suite U|Farmington|Sandoval County|NM|89145|United States|-7|apartment| +5142|AAAAAAAAGBEBAAAA|714|Walnut |Court|Suite 80|Pleasant Hill|Hood River County|OR|93604|United States|-8|condo| +5143|AAAAAAAAHBEBAAAA|984|9th Maple|Way|Suite 400|Marion|Polk County|NC|20399|United States|-5|condo| +5144|AAAAAAAAIBEBAAAA|132|2nd Oak|Ave|Suite H|Oak Hill|Granville County|NC|27838|United States|-5|condo| +5145|AAAAAAAAJBEBAAAA|385|Franklin |Court|Suite 280|Greenfield|Sullivan County|MO|65038|United States|-6|apartment| +5146|AAAAAAAAKBEBAAAA|90|5th |Street|Suite 120|Springfield|Lincoln County|GA|39303|United States|-5|apartment| +5147|AAAAAAAALBEBAAAA|70|Elm Meadow|Circle|Suite S|Jackson|Dakota County|MN|59583|United States|-6|apartment| +5148|AAAAAAAAMBEBAAAA|604|2nd Laurel|Ct.|Suite 240|Union|McClain County|OK|78721|United States|-6|condo| +5149|AAAAAAAANBEBAAAA|754|Chestnut |Boulevard|Suite 340|Cedar Grove|Campbell County|VA|20411|United States|-5|single family| +5150|AAAAAAAAOBEBAAAA|69|Mill |ST|Suite D|Oakdale|Fairfield County|CT|09584|United States|-5|condo| +5151|AAAAAAAAPBEBAAAA|290|View Pine|RD|Suite B|Five Points|Carroll County|MD|26098|United States|-5|condo| +5152|AAAAAAAAACEBAAAA|866|South |Court|Suite 220|Hidden Valley|Heard County|GA|35521|United States|-5|condo| +5153|AAAAAAAABCEBAAAA|661|11th Hillcrest|Court|Suite H|Pinhook|Lincoln County|ID|89398|United States|-7|single family| +5154|AAAAAAAACCEBAAAA|431|Oak 13th|Wy|Suite 260|Greenfield|Leavenworth County|KS|65038|United States|-6|apartment| +5155|AAAAAAAADCEBAAAA|66|Birch College|Drive|Suite 390|Marion|Plumas County|CA|90399|United States|-8|single family| +5156|AAAAAAAAECEBAAAA|352|Lee |Circle|Suite 50|Edgewood|San Benito County|CA|90069|United States|-8|condo| +5157|AAAAAAAAFCEBAAAA|982|11th |Boulevard|Suite I|Shannon|Logan County|NE|64120|United States|-7|apartment| +5158|AAAAAAAAGCEBAAAA|230|Lake |RD|Suite C|Snug Harbor|Hood County|TX|77936|United States|-6|single family| +5159|AAAAAAAAHCEBAAAA|909|Ridge |Way|Suite C|Westgate|Fountain County|IN|42366|United States|-5|apartment| +5160|AAAAAAAAICEBAAAA|69|9th |Cir.|Suite 430|Pleasant Valley|Red River Parish|LA|72477|United States|-6|apartment| +5161|AAAAAAAAJCEBAAAA|172|West |Boulevard|Suite R|Spring Hill|Wood County|TX|76787|United States|-6|single family| +5162|AAAAAAAAKCEBAAAA|604|1st Cedar|Blvd|Suite 40|Pleasant Valley|Westchester County|NY|12477|United States|-5|condo| +5163|AAAAAAAALCEBAAAA|710|Cherry |Dr.|Suite 480|Lakeside|Ramsey County|MN|59532|United States|-6|single family| +5164|AAAAAAAAMCEBAAAA|842|Sunset Church|Ct.|Suite Y|Oakland|Jefferson County|GA|39843|United States|-5|apartment| +5165|AAAAAAAANCEBAAAA|686|Green Jackson|Blvd|Suite F|Marion|Sussex County|DE|10399|United States|-5|condo| +5166|AAAAAAAAOCEBAAAA|783|Lincoln |Pkwy|Suite 320|Bunker Hill|Hart County|KY|40150|United States|-6|condo| +5167|AAAAAAAAPCEBAAAA|905|Hill |Drive|Suite R|Stringtown|McCurtain County|OK|70162|United States|-6|condo| +5168|AAAAAAAAADEBAAAA|773|Wilson 12th|Pkwy|Suite U|Brownsville|Fayette County|TX|79310|United States|-6|single family| +5169|AAAAAAAABDEBAAAA|752|||Suite 460||Smith County|||United States||apartment| +5170|AAAAAAAACDEBAAAA|489|6th Valley|Road|Suite 220|Red Hill|Shelby County|MO|64338|United States|-6|condo| +5171|AAAAAAAADDEBAAAA|480|Ash First|Road|Suite 10|Sheffield|Hooker County|NE|66896|United States|-7|apartment| +5172|AAAAAAAAEDEBAAAA|454|Fourth |Pkwy|Suite 270|Riverside|Decatur County|IA|59231|United States|-6|single family| +5173|AAAAAAAAFDEBAAAA|609|Adams 5th|RD|Suite 380|Walnut Grove|Lake County|OR|97752|United States|-8|condo| +5174|AAAAAAAAGDEBAAAA|724|Willow |Wy|Suite H|Crossroads|Lafayette County|FL|30534|United States|-5|apartment| +5175|AAAAAAAAHDEBAAAA|824|Elm 2nd|Lane|Suite 330|Spring Valley|Big Horn County|MT|66060|United States|-7|apartment| +5176|AAAAAAAAIDEBAAAA|617|North Miller|||||MI|46060|United States||| +5177|AAAAAAAAJDEBAAAA|80|Cherry |Court|Suite L|Pleasant Grove|Medina County|TX|74136|United States|-6|condo| +5178|AAAAAAAAKDEBAAAA|261|Lake Lincoln|Parkway|Suite R|Oakland|Silver Bow County|MT|69843|United States|-7|condo| +5179|AAAAAAAALDEBAAAA|143|Meadow College|Lane|Suite 310|Farmington|Prince William County|VA|29145|United States|-5|single family| +5180|AAAAAAAAMDEBAAAA|643|Jefferson |Lane|Suite J|Riverview|Mitchell County|TX|79003|United States|-6|single family| +5181|AAAAAAAANDEBAAAA|968|Elm Smith|Court|Suite C|Lebanon|Adair County|IA|52898|United States|-6|single family| +5182|AAAAAAAAODEBAAAA|76|Locust |Way|Suite 400|Deerfield|Haralson County|GA|39840|United States|-5|condo| +5183|AAAAAAAAPDEBAAAA|855|Park Johnson|Court|Suite P|Oak Ridge|Prince William County|VA|28371|United States|-5|condo| +5184|AAAAAAAAAEEBAAAA|287|Adams |Cir.|Suite L|Westgate|Denali Borough|AK|92366|United States|-9|apartment| +5185|AAAAAAAABEEBAAAA|754|Cedar |Avenue|Suite 90|Liberty|Grainger County|TN|33451|United States|-5|condo| +5186|AAAAAAAACEEBAAAA|809|Railroad Cherry|Street|Suite 50|Spring Hill|Robertson County|TX|76787|United States|-6|condo| +5187|AAAAAAAADEEBAAAA|4|Lakeview |Lane|Suite C|Glenwood|Owsley County|KY|43511|United States|-5|single family| +5188|AAAAAAAAEEEBAAAA|139|Lee 2nd|Lane|Suite 490|Glendale|Oconee County|GA|33951|United States|-5|apartment| +5189|AAAAAAAAFEEBAAAA|651|Adams Sunset|Lane|Suite 450|Spring Hill|Ouachita Parish|LA|76787|United States|-6|condo| +5190|AAAAAAAAGEEBAAAA|672|Sixth |Ct.|Suite I|Pleasant Hill|Dauphin County|PA|13604|United States|-5|apartment| +5191|AAAAAAAAHEEBAAAA|381|Park Cedar|Ave|Suite 130|Centerville|Navajo County|AZ|80059|United States|-7|condo| +5192|AAAAAAAAIEEBAAAA|478|3rd River|Drive|Suite 430|Salem|Covington city|VA|28048|United States|-5|single family| +5193|AAAAAAAAJEEBAAAA|355|Miller Hill|Road|Suite W|Shady Grove|Crawford County|IA|52812|United States|-6|single family| +5194|AAAAAAAAKEEBAAAA|548|First 10th|Cir.|Suite 180|Greenwood|Randolph County|NC|28828|United States|-5|condo| +5195|AAAAAAAALEEBAAAA|190|Green 12th|Avenue|Suite 50|Green Acres|Beltrami County|MN|57683|United States|-6|condo| +5196|AAAAAAAAMEEBAAAA|850|Woodland Jefferson|Lane|Suite 180|Plainview|Esmeralda County|NV|83683|United States|-8|apartment| +5197|AAAAAAAANEEBAAAA|740|Fifth 7th|Parkway|Suite S|Wilson|Chippewa County|MN|56971|United States|-6|condo| +5198|AAAAAAAAOEEBAAAA|765|View Washington|Parkway|Suite 420|Glendale|Johnson County|MO|63951|United States|-6|apartment| +5199|AAAAAAAAPEEBAAAA|614|Maple |Way|Suite 110|Florence|Erie County|NY|13394|United States|-5|single family| +5200|AAAAAAAAAFEBAAAA|||Drive|Suite I|Riverview|Dubuque County|||United States||single family| +5201|AAAAAAAABFEBAAAA|880|South Church|Parkway|Suite X|Red Hill|Whiteside County|IL|64338|United States|-6|apartment| +5202|AAAAAAAACFEBAAAA|944|East Sixth|Cir.|Suite K|Deerfield|Perry County|AR|79840|United States|-6|single family| +5203|AAAAAAAADFEBAAAA|555|Smith |Boulevard|Suite H|Brentwood|Chippewa County|MI|44188|United States|-5|apartment| +5204|AAAAAAAAEFEBAAAA|755|Sunset |Ave|Suite C|Pleasant Hill|Fall River County|SD|53604|United States|-6|apartment| +5205|AAAAAAAAFFEBAAAA|553|Elm |Way|Suite 120|Oakwood|Jefferson County|PA|10169|United States|-5|apartment| +5206|AAAAAAAAGFEBAAAA|23|Meadow North|ST|Suite 240|Wilson|McCormick County|SC|26971|United States|-5|condo| +5207|AAAAAAAAHFEBAAAA|445|Laurel 8th|Court|Suite 100|Springtown|Lawrence County|SD|59858|United States|-7|single family| +5208|AAAAAAAAIFEBAAAA|827|Park South|Drive|Suite I|Oak Hill|Bristol County|RI|08438|United States|-5|single family| +5209|AAAAAAAAJFEBAAAA|670|Mill |Boulevard|Suite K|Lincoln|Gogebic County|MI|41289|United States|-5|apartment| +5210|AAAAAAAAKFEBAAAA|908|View Hill|Ave|Suite F|Centerville|Polk County|TX|70059|United States|-6|apartment| +5211|AAAAAAAALFEBAAAA|161|Second |Ave|Suite 60|Lone Pine|Jackson County|TX|77441|United States|-6|condo| +5212|AAAAAAAAMFEBAAAA|804|Maple Third|Ct.|Suite 190|Concord|Aitkin County|MN|54107|United States|-6|apartment| +5213|AAAAAAAANFEBAAAA|95|Park |Cir.|Suite T|Pine Valley|Lincoln County|KS|68209|United States|-6|condo| +5214|AAAAAAAAOFEBAAAA|850|Jefferson Spring|RD|Suite 330|Glenwood|Elbert County|CO|83511|United States|-7|condo| +5215|AAAAAAAAPFEBAAAA|162|Park |Road|Suite T|Wilson|Clay County|NE|66971|United States|-6|condo| +5216|AAAAAAAAAGEBAAAA|966|Woodland Pine|Street|Suite K|Oakland|Gibson County|TN|39843|United States|-5|apartment| +5217|AAAAAAAABGEBAAAA|157|Mill |Cir.|Suite B|Brownsville|Union County|SC|29310|United States|-5|apartment| +5218|AAAAAAAACGEBAAAA|248|Maple |Boulevard|Suite X|Church Hill|Warren County|PA|13790|United States|-5|single family| +5219|AAAAAAAADGEBAAAA|306|Chestnut Sunset|Ct.|Suite Y|Stringtown|Madison County|NE|60162|United States|-7|condo| +5220|AAAAAAAAEGEBAAAA|719|Mill View|Lane|Suite N|Five Forks|Pocahontas County|IA|52293|United States|-6|apartment| +5221|AAAAAAAAFGEBAAAA|161|Lake |Parkway|Suite 80|White Oak|Gallia County|OH|46668|United States|-5|apartment| +5222|AAAAAAAAGGEBAAAA|508|11th |Pkwy|Suite P|Mount Zion|Thomas County|NE|68054|United States|-7|single family| +5223|AAAAAAAAHGEBAAAA|65|South Cherry|Parkway|Suite 430|Plainview|Webster County|IA|53683|United States|-6|single family| +5224|AAAAAAAAIGEBAAAA|320|Jefferson |Way|Suite L|Marion|Duchesne County|UT|80399|United States|-7|condo| +5225|AAAAAAAAJGEBAAAA|844|Cherry |Court|Suite 340|Mount Olive|Lincoln County|AR|78059|United States|-6|single family| +5226|AAAAAAAAKGEBAAAA|511|||Suite 30||Stillwater County|MT|68095||-7|| +5227|AAAAAAAALGEBAAAA|962|2nd College|Pkwy|Suite 490|Farmington|Cook County|MN|59145|United States|-6|condo| +5228|AAAAAAAAMGEBAAAA||Valley Hickory||||Manassas Park city||21087|United States|-5|apartment| +5229|AAAAAAAANGEBAAAA|222|Fifth Park|Wy|Suite J|Glenwood|Butler County|MO|63511|United States|-6|condo| +5230|AAAAAAAAOGEBAAAA|608|8th 13th|Ln|Suite Y|Belmont|Vinton County|OH|40191|United States|-5|apartment| +5231|AAAAAAAAPGEBAAAA|777|Elm Franklin|Cir.|Suite Y|Antioch|Hamblen County|TN|38605|United States|-5|apartment| +5232|AAAAAAAAAHEBAAAA|310|11th |ST|Suite Y|Stringtown|Cedar County|NE|60162|United States|-6|condo| +5233|AAAAAAAABHEBAAAA|785|Park 12th|Wy|Suite 420|Springdale|Ballard County|KY|48883|United States|-6|condo| +5234|AAAAAAAACHEBAAAA|69|Third |Ct.|Suite E|Mount Zion|Norton city|VA|28054|United States|-5|condo| +5235|AAAAAAAADHEBAAAA|617|Meadow |Way|Suite 440|Green Acres|Pershing County|NV|87683|United States|-8|apartment| +5236|AAAAAAAAEHEBAAAA|328|Sunset Smith|Drive|Suite 210|Deerfield|Starke County|IN|49840|United States|-5|condo| +5237|AAAAAAAAFHEBAAAA|167|1st |Lane|Suite O|Friendship|Windham County|VT|05136|United States|-5|single family| +5238|AAAAAAAAGHEBAAAA|711|Sunset Fifth|Boulevard|Suite 170|Macedonia|Bottineau County|ND|51087|United States|-6|apartment| +5239|AAAAAAAAHHEBAAAA|221|Smith Woodland|Ct.|Suite F|Harmony|Chatham County|GA|35804|United States|-5|single family| +5240|AAAAAAAAIHEBAAAA|13|South Franklin|Circle|Suite 470|Summit|Red Willow County|NE|60499|United States|-7|apartment| +5241|AAAAAAAAJHEBAAAA|625|Johnson |ST|Suite 160|Enterprise|Presidio County|TX|71757|United States|-6|condo| +5242|AAAAAAAAKHEBAAAA|658|Hill |ST|Suite 80|Mountain View|Queen Anne County|MD|24466|United States|-5|condo| +5243|AAAAAAAALHEBAAAA|871|Lakeview |Parkway|Suite A|Calhoun|Scotland County|NC|26909|United States|-5|apartment| +5244|AAAAAAAAMHEBAAAA|431|Church Sixth|Ave|Suite J|Mount Vernon|Santa Fe County|NM|88482|United States|-7|apartment| +5245|AAAAAAAANHEBAAAA|246|Sunset |Court|Suite U|Wilton|Hancock County|GA|36997|United States|-5|single family| +5246|AAAAAAAAOHEBAAAA|612|Ridge Washington|ST|Suite 90|Edgewood|Bienville Parish|LA|70069|United States|-6|condo| +5247|AAAAAAAAPHEBAAAA|313|Lee Lake|Ct.|Suite 290|Sunnyside|Hillsborough County|NH|02552|United States|-5|apartment| +5248|AAAAAAAAAIEBAAAA|43|Jackson 2nd|Pkwy|Suite S|Oakdale|Johnston County|OK|79584|United States|-6|single family| +5249|AAAAAAAABIEBAAAA|317|Williams Valley|Road|Suite 220|Mount Pleasant|Calhoun County|TX|71933|United States|-6|condo| +5250|AAAAAAAACIEBAAAA|168|Fifth |Ln|Suite F|Enterprise|Yazoo County|MS|51757|United States|-6|apartment| +5251|AAAAAAAADIEBAAAA|202|Church 9th|Ct.|Suite P|Walnut Grove|Carroll County|MD|27752|United States|-5|condo| +5252|AAAAAAAAEIEBAAAA|94|Willow |Circle|Suite T|Maple Grove|Dallas County|TX|78252|United States|-6|condo| +5253|AAAAAAAAFIEBAAAA|442|Walnut |Parkway|Suite 180|Ashland|Charlevoix County|MI|44244|United States|-5|apartment| +5254|AAAAAAAAGIEBAAAA|94|Park |Drive|Suite 30|Marion|Jersey County|IL|60399|United States|-6|apartment| +5255|AAAAAAAAHIEBAAAA|874|Fourth Willow|ST|Suite G|Maple Grove|Cascade County|MT|68252|United States|-7|apartment| +5256|AAAAAAAAIIEBAAAA|438|Willow |ST|Suite O|Forest Hills|Isanti County|MN|59237|United States|-6|apartment| +5257|AAAAAAAAJIEBAAAA|15|Washington Central|ST|Suite 480|Glenwood|Perry County|MO|63511|United States|-6|apartment| +5258|AAAAAAAAKIEBAAAA|||Ln|Suite C|Clifton|Sandusky County|OH||United States||single family| +5259|AAAAAAAALIEBAAAA|93|Poplar Second|Court|Suite 310|Riverdale|Sumter County|FL|39391|United States|-5|single family| +5260|AAAAAAAAMIEBAAAA|904|Sycamore Highland|Drive|Suite S|Concord|Rensselaer County|NY|14107|United States|-5|single family| +5261|AAAAAAAANIEBAAAA|623|Seventh Washington|Pkwy|Suite 190|Cedar Grove|Pasco County|FL|30411|United States|-5|single family| +5262|AAAAAAAAOIEBAAAA|67|1st Park|Street|Suite U|Greenfield|Trousdale County|TN|35038|United States|-6|single family| +5263|AAAAAAAAPIEBAAAA|755|Williams |Street|Suite 100|Oakwood|Salem city|VA|20169|United States|-5|single family| +5264|AAAAAAAAAJEBAAAA|358|Lincoln |Cir.|Suite 0|Antioch|Dearborn County|IN|48605|United States|-5|condo| +5265|AAAAAAAABJEBAAAA|872|4th |Court|Suite F|Pine Grove|Barren County|KY|44593|United States|-6|single family| +5266|AAAAAAAACJEBAAAA|292|Lakeview Forest|Road|Suite T|Union Hill|Jackson County|AR|77746|United States|-6|apartment| +5267|AAAAAAAADJEBAAAA|60|East |Lane|Suite R|Mount Olive|Calvert County|MD|28059|United States|-5|condo| +5268|AAAAAAAAEJEBAAAA|14|Park Washington|Court|Suite 130|Sunnyside|Carroll County|NH|02552|United States|-5|single family| +5269|AAAAAAAAFJEBAAAA|396|Jackson 1st|Avenue|Suite R|Union|Lancaster County|PA|18721|United States|-5|condo| +5270|AAAAAAAAGJEBAAAA|953||||Fairfield|Davie County|NC|26192|United States||| +5271|AAAAAAAAHJEBAAAA|617|Elevnth |Avenue|Suite R|Forest|Clarke County|GA|37537|United States|-5|condo| +5272|AAAAAAAAIJEBAAAA|521|Birch |Ave|Suite T|Woodland|Wilkes County|NC|24854|United States|-5|condo| +5273|AAAAAAAAJJEBAAAA||||||Mecklenburg County||21852|United States|-5|| +5274|AAAAAAAAKJEBAAAA|738|Central |RD|Suite U|Belmont|Carroll County|VA|20191|United States|-5|single family| +5275|AAAAAAAALJEBAAAA|999|East |Lane|Suite 90|Lakewood|Morgan County|AL|38877|United States|-6|apartment| +5276|AAAAAAAAMJEBAAAA|854|4th |Road|Suite W|Shiloh|Stark County|IL|69275|United States|-6|apartment| +5277|AAAAAAAANJEBAAAA|452|Forest Maple|Lane|Suite O|Unionville|Buffalo County|WI|51711|United States|-6|single family| +5278|AAAAAAAAOJEBAAAA||Woodland Hickory||Suite 230|Kingston|||54975|United States||apartment| +5279|AAAAAAAAPJEBAAAA|630|Main |RD|Suite L|Florence|Cumberland County|IL|63394|United States|-6|single family| +5280|AAAAAAAAAKEBAAAA|623|South Jefferson|Ave|Suite H|Leland|Norton County|KS|69452|United States|-6|single family| +5281|AAAAAAAABKEBAAAA|727|13th |Ave|Suite E|Brownsville|Oldham County|KY|49310|United States|-5|apartment| +5282|AAAAAAAACKEBAAAA|890|View |Parkway|Suite 430|Red Hill|Turner County|GA|34338|United States|-5|apartment| +5283|AAAAAAAADKEBAAAA|330||ST||Hillcrest|Grand Traverse County|MI|43003|||apartment| +5284|AAAAAAAAEKEBAAAA|380|Lakeview |Boulevard|Suite 400|New Hope|Lake County|SD|59431|United States|-7|condo| +5285|AAAAAAAAFKEBAAAA|276|Walnut |RD|Suite 480|Marion|Chickasaw County|MS|50399|United States|-6|single family| +5286|AAAAAAAAGKEBAAAA|911|6th Center|Lane|Suite 470|Pleasant Grove|Washington County|TN|34136|United States|-6|single family| +5287|AAAAAAAAHKEBAAAA|357|Lake |Boulevard|Suite T|Johnsonville|Sioux County|IA|57745|United States|-6|apartment| +5288|AAAAAAAAIKEBAAAA|9|Mill Second|Ln|Suite P|Hamilton|Madison Parish|LA|72808|United States|-6|single family| +5289|AAAAAAAAJKEBAAAA|551|Jefferson |Road|||Bollinger County|||United States||| +5290|AAAAAAAAKKEBAAAA|545|Seventh Park|Blvd|Suite V|Wildwood|Atkinson County|GA|36871|United States|-5|single family| +5291|AAAAAAAALKEBAAAA|408|Main |Ct.|Suite 170|Midway|Craven County|NC|21904|United States|-5|condo| +5292|AAAAAAAAMKEBAAAA|274|Ridge |Cir.|Suite S|Deerfield|Oregon County|MO|69840|United States|-6|apartment| +5293|AAAAAAAANKEBAAAA|424|Mill Center|Cir.|Suite 200|Pierce|Jefferson County|AR|73360|United States|-6|single family| +5294|AAAAAAAAOKEBAAAA|457|Miller 14th|Avenue|Suite 330|Lincoln|Shasta County|CA|91289|United States|-8|single family| +5295|AAAAAAAAPKEBAAAA|||||Greenfield|||85038||-7|| +5296|AAAAAAAAALEBAAAA|215|East Church|ST|Suite X|Oak Ridge|Blackford County|IN|48371|United States|-5|single family| +5297|AAAAAAAABLEBAAAA|647|Fifth Sycamore|Way|Suite N|Harmony|Licking County|OH|45804|United States|-5|single family| +5298|AAAAAAAACLEBAAAA|883|Williams |Road|Suite 260|Woodlawn|Waynesboro city|VA|24098|United States|-5|single family| +5299|AAAAAAAADLEBAAAA|612|Broadway East|Lane|Suite T|Brownsville|Anderson County|KY|49310|United States|-6|condo| +5300|AAAAAAAAELEBAAAA|716|Main |Avenue|Suite C|Bethel|Matanuska-Susitna Borough|AK|95281|United States|-9|apartment| +5301|AAAAAAAAFLEBAAAA|866|Highland |Pkwy|Suite U|Macedonia|McLean County|KY|41087|United States|-5|condo| +5302|AAAAAAAAGLEBAAAA|557|Mill Elm|RD|Suite 230|Mountain View|Washington County|AL|34466|United States|-6|condo| +5303|AAAAAAAAHLEBAAAA|189|Sixth |ST|Suite O|Jackson|Dixon County|NE|69583|United States|-6|condo| +5304|AAAAAAAAILEBAAAA|118|14th |Dr.|Suite R|Greenfield|Mitchell County|KS|65038|United States|-6|condo| +5305|AAAAAAAAJLEBAAAA|458|8th 10th|Avenue|Suite 40|Stringtown|Switzerland County|IN|40162|United States|-5|condo| +5306|AAAAAAAAKLEBAAAA|449|Main |Parkway|Suite 20|Kirkland|Moore County|TN|37896|United States|-6|condo| +5307|AAAAAAAALLEBAAAA|439|2nd |Dr.|Suite 450|Kelly|Randolph County|GA|32738|United States|-5|apartment| +5308|AAAAAAAAMLEBAAAA|562|Sunset |Wy|Suite 90|Ashley|Fulton County|NY|14324|United States|-5|single family| +5309|AAAAAAAANLEBAAAA|79|Fourth |Parkway|Suite P|Appleton|Hancock County|IN|44240|United States|-5|single family| +5310|AAAAAAAAOLEBAAAA|264|River Central|Drive|Suite N|Salem|Fulton County|GA|38048|United States|-5|single family| +5311|AAAAAAAAPLEBAAAA|531|Green Cherry|Way|Suite V|Highland|Monroe County|IA|59454|United States|-6|condo| +5312|AAAAAAAAAMEBAAAA|484|College 6th|Street|Suite N|Highland Park|Bossier Parish|LA|76534|United States|-6|apartment| +5313|AAAAAAAABMEBAAAA|112|Wilson Maple|Drive|Suite 50|Riverview|Winkler County|TX|79003|United States|-6|single family| +5314|AAAAAAAACMEBAAAA||3rd |Ave|Suite 250||Graham County|||United States|-5|| +5315|AAAAAAAADMEBAAAA|96|Jefferson River|Circle|Suite 140|Bunker Hill|Craven County|NC|20150|United States|-5|apartment| +5316|AAAAAAAAEMEBAAAA|427|Second |RD|Suite 320|Jamestown|San Bernardino County|CA|96867|United States|-8|condo| +5317|AAAAAAAAFMEBAAAA|329|6th Highland|Drive|Suite L|Pine Grove|Chambers County|TX|74593|United States|-6|single family| +5318|AAAAAAAAGMEBAAAA|922|Elm |Boulevard|Suite F|Jamestown|Roseau County|MN|56867|United States|-6|condo| +5319|AAAAAAAAHMEBAAAA|613|6th Adams|Circle|Suite D|Stringtown|Floyd County|TX|70162|United States|-6|apartment| +5320|AAAAAAAAIMEBAAAA|175|9th |ST|Suite 410|Jackson|Sampson County|NC|29583|United States|-5|single family| +5321|AAAAAAAAJMEBAAAA|625|Madison Johnson|Blvd|Suite Y|Newtown|Hancock County|IL|61749|United States|-6|single family| +5322|AAAAAAAAKMEBAAAA|434|Valley Main|Lane|Suite L|Mount Olive|Marion County|MS|58059|United States|-6|single family| +5323|AAAAAAAALMEBAAAA|701|Cherry Lincoln|RD|Suite 80|Marion|Polk County|NC|20399|United States|-5|condo| +5324|AAAAAAAAMMEBAAAA|857|4th Railroad|RD|Suite 90|Sunnyside|Jay County|IN|41952|United States|-5|apartment| +5325|AAAAAAAANMEBAAAA|52|Madison |RD|Suite 60|Pleasant Grove|Moffat County|CO|84136|United States|-7|apartment| +5326|AAAAAAAAOMEBAAAA|752|Lee Lincoln|Cir.|Suite 120|Buena Vista|Kemper County|MS|55752|United States|-6|single family| +5327|AAAAAAAAPMEBAAAA|195|5th |Ln|Suite 90|Morgantown|Hancock County|KY|49193|United States|-6|condo| +5328|AAAAAAAAANEBAAAA|404|14th North|Court|Suite 150|Montezuma|Dearborn County|IN|42150|United States|-5|apartment| +5329|AAAAAAAABNEBAAAA|205|Maple |Court|Suite 130|Indian Village|Panola County|MS|51075|United States|-6|apartment| +5330|AAAAAAAACNEBAAAA|495|Madison |Ave|Suite 110|Liberty|Baltimore city|MD|23451|United States|-5|condo| +5331|AAAAAAAADNEBAAAA|492|Jefferson |Cir.|Suite 410|Mountain View|Taylor County|TX|74466|United States|-6|single family| +5332|AAAAAAAAENEBAAAA|888|Franklin |Cir.|Suite 90|Riverside|Banner County|NE|69231|United States|-6|single family| +5333|AAAAAAAAFNEBAAAA|558|4th Sycamore|Pkwy|Suite 350|Springfield|Lee County|AR|79303|United States|-6|single family| +5334|AAAAAAAAGNEBAAAA|253|14th |Street|Suite J|Hamilton|Conejos County|CO|82808|United States|-7|single family| +5335|AAAAAAAAHNEBAAAA|778|Locust Church|Street|Suite 410|Summit|Orange County|CA|90499|United States|-8|condo| +5336|AAAAAAAAINEBAAAA|395|Eigth |ST|Suite 320|Waterloo|Cocke County|TN|31675|United States|-5|condo| +5337|AAAAAAAAJNEBAAAA|511|Laurel 9th|Wy|Suite 350|Jamestown|Chariton County|MO|66867|United States|-6|single family| +5338|AAAAAAAAKNEBAAAA|146|East 6th|Ln|Suite O|Lakeside|Lancaster County|PA|19532|United States|-5|apartment| +5339|AAAAAAAALNEBAAAA|278|1st Spruce|Road|Suite W|White Oak|Imperial County|CA|96668|United States|-8|condo| +5340|AAAAAAAAMNEBAAAA|89|13th Walnut|Avenue|Suite 0|Forest Hills|Gooding County|ID|89237|United States|-7|condo| +5341|AAAAAAAANNEBAAAA|966|View 5th|RD|Suite 360|Newport|Iowa County|IA|51521|United States|-6|single family| +5342|AAAAAAAAONEBAAAA||||Suite B||Gadsden County||30587|||| +5343|AAAAAAAAPNEBAAAA|122|Main College|Court|Suite 230|Sulphur Springs|Conejos County|CO|88354|United States|-7|single family| +5344|AAAAAAAAAOEBAAAA|646|4th Lincoln|Way|Suite 110|Willow|Cayuga County|NY|16798|United States|-5|single family| +5345|AAAAAAAABOEBAAAA|806|Cherry |Ave|Suite O|Bethel|Winkler County|TX|75281|United States|-6|condo| +5346|AAAAAAAACOEBAAAA|922|1st |Road|Suite 280|Highland|Glenn County|CA|99454|United States|-8|apartment| +5347|AAAAAAAADOEBAAAA|915|Mill |||Springdale|Beltrami County||58883|United States||| +5348|AAAAAAAAEOEBAAAA|147|4th |Way|Suite 470|White Oak|Placer County|CA|96668|United States|-8|single family| +5349|AAAAAAAAFOEBAAAA|257|Spring Chestnut|Circle|Suite 390|Lawrenceville|Marinette County|WI|54462|United States|-6|condo| +5350|AAAAAAAAGOEBAAAA||Jackson |ST|Suite Y|||MO|67838|||single family| +5351|AAAAAAAAHOEBAAAA|182|2nd |Drive|Suite 390|Newtown|Morrison County|MN|51749|United States|-6|single family| +5352|AAAAAAAAIOEBAAAA|684|Miller 6th|Pkwy|Suite 420|Union Hill|Miller County|AR|77746|United States|-6|apartment| +5353|AAAAAAAAJOEBAAAA|518|Lake |Ln|Suite O|Jackson|Cleburne County|AL|39583|United States|-6|apartment| +5354|AAAAAAAAKOEBAAAA|771|Elm Oak|RD|Suite G|Deerfield|Jefferson County|AL|39840|United States|-6|single family| +5355|AAAAAAAALOEBAAAA|343|South Ash|Ct.|Suite 70|Henry|Northumberland County|PA|13318|United States|-5|apartment| +5356|AAAAAAAAMOEBAAAA|168|Jackson |Way|Suite F|Lebanon|Rio Grande County|CO|82898|United States|-7|single family| +5357|AAAAAAAANOEBAAAA|611|1st Main|Avenue|Suite 230|White Oak|Jones County|IA|56668|United States|-6|single family| +5358|AAAAAAAAOOEBAAAA|627|Third |Cir.|Suite C|Oak Hill|Fairfield County|SC|27838|United States|-5|condo| +5359|AAAAAAAAPOEBAAAA|194|Meadow |Circle|Suite 450|Oak Grove|Sangamon County|IL|68370|United States|-6|condo| +5360|AAAAAAAAAPEBAAAA|879|Second First|Street|Suite 120|Five Forks|Tuscaloosa County|AL|32293|United States|-6|apartment| +5361|AAAAAAAABPEBAAAA|139|Central |Circle|Suite 210|Edgewood|Lewis and Clark County|MT|60069|United States|-7|apartment| +5362|AAAAAAAACPEBAAAA|664|South |Road|Suite 40|Forest Hills|Thayer County|NE|69237|United States|-7|single family| +5363|AAAAAAAADPEBAAAA|730|Sixth |Drive|Suite 390|Glenwood|Charlotte County|VA|23511|United States|-5|single family| +5364|AAAAAAAAEPEBAAAA|70|Locust |Blvd|Suite 120|Plainview|DeSoto County|FL|33683|United States|-5|condo| +5365|AAAAAAAAFPEBAAAA|738|Maple |Ln|Suite D|Newtown|El Dorado County|CA|91749|United States|-8|condo| +5366|AAAAAAAAGPEBAAAA|77|View |Drive|Suite 110|Green Acres|Iron County|UT|87683|United States|-7|condo| +5367|AAAAAAAAHPEBAAAA|911|Pine Hill|Street|Suite 340|Oak Ridge|Ohio County|IN|48371|United States|-5|single family| +5368|AAAAAAAAIPEBAAAA|778|Park |Wy|Suite 240|Liberty|Carroll County|TN|33451|United States|-5|single family| +5369|AAAAAAAAJPEBAAAA|245|Davis |Pkwy|Suite R|Saint James|Iberia Parish|LA|75799|United States|-6|single family| +5370|AAAAAAAAKPEBAAAA|212|Oak |Parkway|Suite A|Mount Pleasant|Blount County|TN|31933|United States|-5|condo| +5371|AAAAAAAALPEBAAAA|976|Woodland |Dr.|Suite 190|Hamilton|Benson County|ND|52808|United States|-6|single family| +5372|AAAAAAAAMPEBAAAA|835|Jackson |Road|Suite S|Spring Valley|Cortland County|NY|16060|United States|-5|single family| +5373|AAAAAAAANPEBAAAA|845|3rd |Lane|Suite R|Plainview|Collingsworth County|TX|73683|United States|-6|condo| +5374|AAAAAAAAOPEBAAAA|370|Pine |Circle|Suite J|Harmony|Gilliam County|OR|95804|United States|-8|condo| +5375|AAAAAAAAPPEBAAAA|604|Fifth |Circle|Suite 270|Union City|Franklin County|MS|58087|United States|-6|condo| +5376|AAAAAAAAAAFBAAAA||Hill Jefferson||Suite V||Callaway County||68371|United States||| +5377|AAAAAAAABAFBAAAA|384|West |Circle|Suite 320|Highland|Nash County|NC|29454|United States|-5|condo| +5378|AAAAAAAACAFBAAAA|874|Highland |Ave|Suite 330|Lone Oak|Cuyahoga County|OH|46893|United States|-5|condo| +5379|AAAAAAAADAFBAAAA|656|Hill |Cir.|Suite G|Riverview|Fremont County|ID|89003|United States|-7|condo| +5380|AAAAAAAAEAFBAAAA|614|Cedar |Lane|Suite 80|Ashley|Woodford County|IL|64324|United States|-6|apartment| +5381|AAAAAAAAFAFBAAAA|457|Johnson Jackson|Drive|Suite 200|Mount Zion|Kenai Peninsula Borough|AK|98054|United States|-9|single family| +5382|AAAAAAAAGAFBAAAA|81|Church |Pkwy|Suite 160|Summit|Berkeley County|WV|20499|United States|-5|condo| +5383|AAAAAAAAHAFBAAAA|676|Locust |Circle|Suite 120|Antioch|Harding County|SD|58605|United States|-7|single family| +5384|AAAAAAAAIAFBAAAA|954|Spring |Court|Suite C|Sunnyside|Jackson County|IN|41952|United States|-5|apartment| +5385|AAAAAAAAJAFBAAAA|590|Locust First|Parkway|Suite T|Greenwood|Madison County|MT|68828|United States|-7|condo| +5386|AAAAAAAAKAFBAAAA|998|Lincoln |RD|Suite O|Enterprise|Winston County|AL|31757|United States|-6|apartment| +5387|AAAAAAAALAFBAAAA|988|14th |Circle|Suite 30|Buena Vista|Sanders County|MT|65752|United States|-7|apartment| +5388|AAAAAAAAMAFBAAAA|125|Poplar Ridge|Circle|Suite O|Pleasant Valley|Calhoun County|MI|42477|United States|-5|single family| +5389|AAAAAAAANAFBAAAA|395|13th |Wy|Suite P|New Hope|Walworth County|WI|59431|United States|-6|condo| +5390|AAAAAAAAOAFBAAAA|367|Laurel Church|Blvd|Suite 260|Greenfield|Greenwood County|SC|25038|United States|-5|condo| +5391|AAAAAAAAPAFBAAAA|878|View |Drive|Suite Y|Providence|Meagher County|MT|66614|United States|-7|single family| +5392|AAAAAAAAABFBAAAA|948|Sunset South|Ln|Suite Y|Oakwood|Sierra County|CA|90169|United States|-8|single family| +5393|AAAAAAAABBFBAAAA|883|Spring Ridge|Circle|Suite 180|Jackson|Franklin County|KY|49583|United States|-6|apartment| +5394|AAAAAAAACBFBAAAA|684|Cedar |Court|Suite D|Hillcrest|Richardson County|NE|63003|United States|-7|condo| +5395|AAAAAAAADBFBAAAA|521|Spring |Dr.|Suite 30|Jamestown|Polk County|IA|56867|United States|-6|single family| +5396|AAAAAAAAEBFBAAAA|435|North Smith|Pkwy|Suite 30|Riverside|Fairfax County|VA|29231|United States|-5|condo| +5397|AAAAAAAAFBFBAAAA|641|11th 6th|Pkwy|Suite K|Woodville|Bastrop County|TX|74289|United States|-6|apartment| +5398|AAAAAAAAGBFBAAAA|813|1st Valley|Parkway|Suite F|Sleepy Hollow|San Bernardino County|CA|93592|United States|-8|single family| +5399|AAAAAAAAHBFBAAAA|583|Spring East|ST|Suite B|Riverview|Orange County|NC|29003|United States|-5|apartment| +5400|AAAAAAAAIBFBAAAA|922|Second |Road|Suite A|Fairbanks|Amite County|MS|56653|United States|-6|single family| +5401|AAAAAAAAJBFBAAAA|892|11th Cherry|Street|Suite O|Mount Pleasant|Henry County|AL|31933|United States|-6|single family| +5402|AAAAAAAAKBFBAAAA|55|Jefferson |Circle|Suite G|New Hope|Amelia County|VA|29431|United States|-5|apartment| +5403|AAAAAAAALBFBAAAA|446|Park |Ave|Suite E|Hamilton|McIntosh County|OK|72808|United States|-6|single family| +5404|AAAAAAAAMBFBAAAA|753|East Adams|ST|Suite 390|Empire|Nassau County|NY|14145|United States|-5|condo| +5405|AAAAAAAANBFBAAAA|46|Poplar Cherry|Boulevard|Suite G|Sawyer|Loup County|NE|66045|United States|-7|condo| +5406|AAAAAAAAOBFBAAAA|102|Johnson |Blvd|Suite M|Hamilton|Cottonwood County|MN|52808|United States|-6|condo| +5407|AAAAAAAAPBFBAAAA|174|Smith |Ave|Suite B|Walnut Grove|Tippah County|MS|57752|United States|-6|single family| +5408|AAAAAAAAACFBAAAA|104|10th |Lane|Suite W|Franklin|Steele County|ND|59101|United States|-6|single family| +5409|AAAAAAAABCFBAAAA|589|1st Hillcrest|Street|Suite 470|Georgetown|Lebanon County|PA|17057|United States|-5|apartment| +5410|AAAAAAAACCFBAAAA|204|Pine 4th|Ct.|Suite O|Midway|Nuckolls County|NE|61904|United States|-7|single family| +5411|AAAAAAAADCFBAAAA|272|Central Cedar|Lane|Suite W|Forest Hills|Johnston County|NC|29237|United States|-5|apartment| +5412|AAAAAAAAECFBAAAA|52|Walnut Jefferson|Court|Suite 190|Salem|Dallas County|AL|38048|United States|-6|apartment| +5413|AAAAAAAAFCFBAAAA|3|Cedar |ST|Suite 140|Sulphur Springs|Burke County|ND|58354|United States|-6|single family| +5414|AAAAAAAAGCFBAAAA|518|3rd Lee|Dr.|Suite F|Liberty|Lampasas County|TX|73451|United States|-6|condo| +5415|AAAAAAAAHCFBAAAA|736|Franklin River|Ln|Suite I|Ridgeville|Miller County|GA|39306|United States|-5|apartment| +5416|AAAAAAAAICFBAAAA|534|Church |Drive|Suite 320|Edgewood|Trego County|KS|60069|United States|-6|apartment| +5417|AAAAAAAAJCFBAAAA|672|13th |Way|Suite 390|Bunker Hill|Limestone County|AL|30150|United States|-6|single family| +5418|AAAAAAAAKCFBAAAA|675|Franklin |Cir.|Suite 10|Unionville|Petersburg city|VA|21711|United States|-5|single family| +5419|AAAAAAAALCFBAAAA|716|Walnut Main|Drive|Suite 40|Crossroads|Washington County|NY|10534|United States|-5|apartment| +5420|AAAAAAAAMCFBAAAA|203|9th Lee|Court|Suite 60|Hopewell|Rio Blanco County|CO|80587|United States|-7|single family| +5421|AAAAAAAANCFBAAAA|267|2nd College|Cir.|Suite B|Perkins|Walsh County|ND|51852|United States|-6|condo| +5422|AAAAAAAAOCFBAAAA|934|Third Franklin|Ln|Suite 90|Clinton|Douglas County|NE|68222|United States|-6|single family| +5423|AAAAAAAAPCFBAAAA|9|1st Park|Street|Suite N|Friendship|Bond County|IL|64536|United States|-6|condo| +5424|AAAAAAAAADFBAAAA|227|13th Green|Parkway|Suite E|Roy|Dickson County|TN|30744|United States|-5|apartment| +5425|AAAAAAAABDFBAAAA|823|Sunset |Lane|Suite M|Georgetown|Stephens County|GA|37057|United States|-5|condo| +5426|AAAAAAAACDFBAAAA|902|Poplar |Circle|Suite 120|Riverdale|Cotton County|OK|79391|United States|-6|condo| +5427|AAAAAAAADDFBAAAA|778|Woodland |Blvd|Suite 430|Enterprise|Meriwether County|GA|31757|United States|-5|single family| +5428|AAAAAAAAEDFBAAAA|535|Walnut |Dr.|Suite C|Mount Pleasant|Jackson County|WI|51933|United States|-6|apartment| +5429|AAAAAAAAFDFBAAAA|667|1st |Cir.|Suite 30|Newport|Anderson County|TN|31521|United States|-5|apartment| +5430|AAAAAAAAGDFBAAAA|778|Walnut 7th|ST|Suite U|Friendship|Seneca County|NY|14536|United States|-5|apartment| +5431|AAAAAAAAHDFBAAAA|666|Willow Sycamore|Court|Suite J|Edgewater|Linn County|OR|90635|United States|-8|condo| +5432|AAAAAAAAIDFBAAAA|746|Laurel Central|Street|Suite 170|Riverdale|Hillsborough County|FL|39391|United States|-5|single family| +5433|AAAAAAAAJDFBAAAA|543|Johnson |Drive|Suite 400|Centerville|Beaufort County|NC|20059|United States|-5|single family| +5434|AAAAAAAAKDFBAAAA|190|Sycamore Dogwood|Way|Suite B|Ashland|Lake County|MN|54244|United States|-6|apartment| +5435|AAAAAAAALDFBAAAA|177|Ridge |Avenue|Suite H|Pleasant Valley|Potter County|PA|12477|United States|-5|condo| +5436|AAAAAAAAMDFBAAAA|538|4th 3rd|Street|Suite 470|Shannon|Franklin County|NE|64120|United States|-6|single family| +5437|AAAAAAAANDFBAAAA|843|Park 6th|Drive|Suite Q|Enterprise|Otero County|CO|81757|United States|-7|apartment| +5438|AAAAAAAAODFBAAAA|628|Hill |Lane|Suite Y|Oakwood|Cloud County|KS|60169|United States|-6|condo| +5439|AAAAAAAAPDFBAAAA|796|Eigth Spring|Dr.|Suite K|Unionville|Potter County|PA|11711|United States|-5|apartment| +5440|AAAAAAAAAEFBAAAA|154|8th |Wy|Suite 90|Kingston|Dale County|AL|34975|United States|-6|single family| +5441|AAAAAAAABEFBAAAA|760|Lincoln |Pkwy|Suite E|Bridgeport|Henderson County|KY|45817|United States|-6|condo| +5442|AAAAAAAACEFBAAAA|590|5th Chestnut|Ln|Suite S|Jamestown|Phillips County|CO|86867|United States|-7|single family| +5443|AAAAAAAADEFBAAAA|760|8th |Road|Suite G|Union|Logan County|OK|78721|United States|-6|condo| +5444|AAAAAAAAEEFBAAAA|845|Second |ST|Suite S|Georgetown|Edwards County|TX|77057|United States|-6|apartment| +5445|AAAAAAAAFEFBAAAA|41|9th Church|Ct.|Suite X|Red Hill|Lafourche Parish|LA|74338|United States|-6|condo| +5446|AAAAAAAAGEFBAAAA|913|Oak |Dr.|Suite 470|Flat Rock|Lexington County|SC|24387|United States|-5|apartment| +5447|AAAAAAAAHEFBAAAA|967|Ridge |Court|Suite 280|Mountain View|Colfax County|NM|84466|United States|-7|apartment| +5448|AAAAAAAAIEFBAAAA|371|5th Highland|Blvd|Suite 80|Unionville|Garfield County|CO|81711|United States|-7|condo| +5449|AAAAAAAAJEFBAAAA|964|Laurel |Parkway|Suite I|Maple Grove|Woodruff County|AR|78252|United States|-6|single family| +5450|AAAAAAAAKEFBAAAA|83|Hill |Road|Suite 400|Valley View|Mono County|CA|95124|United States|-8|single family| +5451|AAAAAAAALEFBAAAA|70|View |Drive|Suite R|Riverview|Goshen County|WY|89003|United States|-7|single family| +5452|AAAAAAAAMEFBAAAA|982|5th Maple|Dr.|Suite 280|Farmington|Coles County|IL|69145|United States|-6|condo| +5453|AAAAAAAANEFBAAAA|850|Meadow |ST|Suite 330|Red Hill|Lancaster County|PA|14338|United States|-5|apartment| +5454|AAAAAAAAOEFBAAAA|816|Maple 5th|Circle|Suite L|Spring Hill|Baylor County|TX|76787|United States|-6|single family| +5455|AAAAAAAAPEFBAAAA|703|Church College|Ln|Suite 420|Oakdale|Lowndes County|MS|59584|United States|-6|single family| +5456|AAAAAAAAAFFBAAAA|107|Third |Street|Suite F|Newport|Boone County|WV|21521|United States|-5|single family| +5457|AAAAAAAABFFBAAAA|544|Lake |Wy|Suite 180|Arlington|Monroe County|MS|56557|United States|-6|apartment| +5458|AAAAAAAACFFBAAAA|126|Central |Court|Suite G|Providence|Indiana County|PA|16614|United States|-5|condo| +5459|AAAAAAAADFFBAAAA|47|First Elm|RD|Suite H|Jamestown|Cass County|IA|56867|United States|-6|condo| +5460|AAAAAAAAEFFBAAAA|193|Broadway Washington|Court|Suite 10|Winslow|Cheyenne County|NE|68525|United States|-6|apartment| +5461|AAAAAAAAFFFBAAAA|498|Main Railroad|Avenue|Suite Y|Green Acres|Lake County|CA|97683|United States|-8|single family| +5462|AAAAAAAAGFFBAAAA|512|Green |Parkway|Suite 50|Gilmore|Johnson County|TN|35464|United States|-6|single family| +5463|AAAAAAAAHFFBAAAA|75|Railroad Madison|Avenue|Suite 0|Lebanon|Grant County|WI|52898|United States|-6|condo| +5464|AAAAAAAAIFFBAAAA|220|Third Chestnut|Way|Suite 40|Brownsville|Indiana County|PA|19310|United States|-5|apartment| +5465|AAAAAAAAJFFBAAAA|883|Railroad |Drive|Suite E|Oakwood|Hickman County|KY|40169|United States|-6|apartment| +5466|AAAAAAAAKFFBAAAA|521|Central Church|Lane|Suite 100|Riverdale|Edwards County|IL|69391|United States|-6|single family| +5467|AAAAAAAALFFBAAAA|959|7th |Boulevard|Suite 480|Jamestown|Rolette County|ND|56867|United States|-6|condo| +5468|AAAAAAAAMFFBAAAA|69|Smith |Ln|Suite 450|Newport|Kauai County|HI|91521|United States|-10|condo| +5469|AAAAAAAANFFBAAAA|87|Oak Ridge|Road|Suite 480|Clifton|Sullivan County|TN|38014|United States|-6|condo| +5470|AAAAAAAAOFFBAAAA|660|Oak |Drive|Suite W|Gary|De Witt County|IL|60418|United States|-6|condo| +5471|AAAAAAAAPFFBAAAA||Madison |Avenue|||||49532||-5|| +5472|AAAAAAAAAGFBAAAA|483|2nd |Ave|Suite 290|Morris|Texas County|OK|76696|United States|-6|single family| +5473|AAAAAAAABGFBAAAA|36|Second Hill|Ln|Suite C|Mountain View|Roosevelt County|NM|84466|United States|-7|apartment| +5474|AAAAAAAACGFBAAAA|827|Valley Lincoln|Boulevard|Suite 380|Harmony|Nelson County|ND|55804|United States|-6|condo| +5475|AAAAAAAADGFBAAAA|296|Valley View|Ave|Suite Q|Five Forks|Taliaferro County|GA|32293|United States|-5|apartment| +5476|AAAAAAAAEGFBAAAA|400|Miller View|Boulevard|Suite 320|Kingston|Nolan County|TX|74975|United States|-6|condo| +5477|AAAAAAAAFGFBAAAA|67|Lake East|ST|Suite 60|Valley View|Cleveland County|AR|75124|United States|-6|condo| +5478|AAAAAAAAGGFBAAAA|781|2nd |Wy|Suite 410|Georgetown|Monroe County|IL|67057|United States|-6|condo| +5479|AAAAAAAAHGFBAAAA|507|6th View|Wy|Suite 280|Shiloh|Steele County|ND|59275|United States|-6|apartment| +5480|AAAAAAAAIGFBAAAA|261|East |Street|Suite 440|Stringtown|Gladwin County|MI|40162|United States|-5|single family| +5481|AAAAAAAAJGFBAAAA|150|Seventh |Wy|Suite P|Red Hill|Polk County|TX|74338|United States|-6|single family| +5482|AAAAAAAAKGFBAAAA|261|15th |Cir.|Suite D|Springfield|Johnston County|NC|29303|United States|-5|apartment| +5483|AAAAAAAALGFBAAAA|874|Jackson Willow|Road|Suite 260|Clinton|Cobb County|GA|38222|United States|-5|condo| +5484|AAAAAAAAMGFBAAAA|660|7th |Ave|Suite 460|Sleepy Hollow|Delaware County|OH|43592|United States|-5|condo| +5485|AAAAAAAANGFBAAAA|838|Jefferson |Way|Suite F|Maple Grove|Kit Carson County|CO|88252|United States|-7|single family| +5486|AAAAAAAAOGFBAAAA|499|Hill |Cir.|Suite 0|Farmington|Dauphin County|PA|19145|United States|-5|condo| +5487|AAAAAAAAPGFBAAAA|988|2nd Chestnut|Boulevard|Suite J|Pleasant Grove|Aleutians West Census Area|AK|94136|United States|-9|condo| +5488|AAAAAAAAAHFBAAAA|463|Johnson |Avenue|Suite 150|Riverdale|Hopewell city|VA|29391|United States|-5|single family| +5489|AAAAAAAABHFBAAAA|335|4th Adams|Blvd|Suite S|Union Hill|Terrell County|TX|77746|United States|-6|single family| +5490|AAAAAAAACHFBAAAA|110|Railroad 10th|RD|Suite X|Midway|Saline County|MO|61904|United States|-6|apartment| +5491|AAAAAAAADHFBAAAA|847|Second |Drive|Suite G|Pleasant Grove|Chautauqua County|KS|64136|United States|-6|apartment| +5492|AAAAAAAAEHFBAAAA|886|Hill |Boulevard|Suite N|Pine Grove|Woodward County|OK|74593|United States|-6|apartment| +5493|AAAAAAAAFHFBAAAA|937|Elm Cherry|ST|Suite 220|Greenfield|Yolo County|CA|95038|United States|-8|condo| +5494|AAAAAAAAGHFBAAAA|910|Oak Lee|Parkway|Suite B|White Oak|Wyoming County|WV|26668|United States|-5|condo| +5495|AAAAAAAAHHFBAAAA|502|3rd Miller|Pkwy|Suite X|Five Forks|Hubbard County|MN|52293|United States|-6|apartment| +5496|AAAAAAAAIHFBAAAA|342|8th |Way|Suite 450|Springdale|Lancaster County|NE|68883|United States|-7|apartment| +5497|AAAAAAAAJHFBAAAA|132|Forest 6th|ST|Suite D|Pomona|Montgomery County|VA|24153|United States|-5|apartment| +5498|AAAAAAAAKHFBAAAA|95|Locust Washington|Avenue|Suite L|Georgetown|Polk County|IA|57057|United States|-6|apartment| +5499|AAAAAAAALHFBAAAA|161|Park |Court|Suite W|Adrian|Williamson County|TX|73301|United States|-6|apartment| +5500|AAAAAAAAMHFBAAAA||Pine 6th|ST|||Oregon County||68252|United States||| +5501|AAAAAAAANHFBAAAA|886|Johnson 1st|Court|Suite X|Post Oak|Johnson County|IL|68567|United States|-6|condo| +5502|AAAAAAAAOHFBAAAA|999|Mill 3rd|Ct.|Suite R|Georgetown|Carroll County|MS|57057|United States|-6|apartment| +5503|AAAAAAAAPHFBAAAA|966|Elm |Road|Suite 100|Cedar Hill|Traill County|ND|50254|United States|-6|single family| +5504|AAAAAAAAAIFBAAAA|656|8th Cherry|Ln|Suite 260|Union|Luzerne County|PA|18721|United States|-5|single family| +5505|AAAAAAAABIFBAAAA|304|15th Johnson|Pkwy|Suite S|Midway|Cullman County|AL|31904|United States|-6|single family| +5506|AAAAAAAACIFBAAAA|596|3rd |Ln|Suite Y|Oak Hill|Tuolumne County|CA|97838|United States|-8|apartment| +5507|AAAAAAAADIFBAAAA|688|Cherry |Parkway|Suite 80|Freeport|Monongalia County|WV|21844|United States|-5|condo| +5508|AAAAAAAAEIFBAAAA|120|13th West|Ave|Suite W|Mount Olive|Moore County|TX|78059|United States|-6|apartment| +5509|AAAAAAAAFIFBAAAA|850|5th Meadow|Wy|Suite 430|Snug Harbor|Horry County|SC|27936|United States|-5|condo| +5510|AAAAAAAAGIFBAAAA|265|Ridge |Drive|Suite T|Oak Ridge|Silver Bow County|MT|68371|United States|-7|apartment| +5511|AAAAAAAAHIFBAAAA|519|Valley |Wy|Suite F|Jackson|Pike County|GA|39583|United States|-5|single family| +5512|AAAAAAAAIIFBAAAA|913|Lake 4th|Ave|Suite 280|Harmony|Hale County|AL|35804|United States|-6|condo| +5513|AAAAAAAAJIFBAAAA|209|Birch |Pkwy|Suite G|Forest Hills|Nacogdoches County|TX|79237|United States|-6|apartment| +5514|AAAAAAAAKIFBAAAA|721|Davis |Dr.|Suite L|Sunnyside|Waller County|TX|71952|United States|-6|apartment| +5515|AAAAAAAALIFBAAAA|959|15th |Ct.|Suite G|Bethel|Clay County|TX|75281|United States|-6|apartment| +5516|AAAAAAAAMIFBAAAA|295|11th Sunset|Avenue|Suite B|Friendship|Montrose County|CO|84536|United States|-7|single family| +5517|AAAAAAAANIFBAAAA|969|Sunset |Drive|Suite 440|Springfield|Tarrant County|TX|79303|United States|-6|apartment| +5518|AAAAAAAAOIFBAAAA|496|7th |Lane|Suite 250|Mount Olive|Charlottesville city|VA|28059|United States|-5|apartment| +5519|AAAAAAAAPIFBAAAA|590|Adams ||Suite 190|Wildwood|Ottawa County|OK|76871|United States||condo| +5520|AAAAAAAAAJFBAAAA|673|Johnson Adams|Ct.|Suite Q|Plainview|Otsego County|NY|13683|United States|-5|single family| +5521|AAAAAAAABJFBAAAA|848|6th 2nd|Wy|Suite E|Mountain View|Chatham County|GA|34466|United States|-5|single family| +5522|AAAAAAAACJFBAAAA|546|Ridge Cedar|Avenue|Suite 240|Mount Pleasant|Ouachita County|AR|71933|United States|-6|single family| +5523|AAAAAAAADJFBAAAA|630|7th |RD|Suite T|Wilson|Henry County|KY|46971|United States|-6|single family| +5524|AAAAAAAAEJFBAAAA|||Lane|||||||-6|| +5525|AAAAAAAAFJFBAAAA|460|Woodland Mill|||Lincoln|Livingston County|KY||||apartment| +5526|AAAAAAAAGJFBAAAA|323|Mill Sunset|Cir.|Suite D|Friendship|Prentiss County|MS|54536|United States|-6|condo| +5527|AAAAAAAAHJFBAAAA|109|Pine Church|Road|Suite 130|Unionville|Belmont County|OH|41711|United States|-5|apartment| +5528|AAAAAAAAIJFBAAAA|2|Pine Spruce|Court|Suite 60|Brownsville|Rawlins County|KS|69310|United States|-6|condo| +5529|AAAAAAAAJJFBAAAA|150|||Suite 20|Pleasant Hill|Ward County|ND||United States|-6|condo| +5530|AAAAAAAAKJFBAAAA|853|Elevnth |Road|Suite F|Providence|Sussex County|VA|26614|United States|-5|apartment| +5531|AAAAAAAALJFBAAAA|872|Oak |Dr.|Suite 260|Liberty|Big Horn County|WY|83451|United States|-7|condo| +5532|AAAAAAAAMJFBAAAA|280|8th South|Boulevard|Suite 70|Mount Pleasant|Decatur County|KS|61933|United States|-6|single family| +5533|AAAAAAAANJFBAAAA|164|Locust |Avenue|Suite Y|Macedonia|Mineral County|CO|81087|United States|-7|condo| +5534|AAAAAAAAOJFBAAAA||||Suite 390|||||||| +5535|AAAAAAAAPJFBAAAA|438|West |ST|Suite 290|Highland Park|Red Willow County|NE|66534|United States|-7|condo| +5536|AAAAAAAAAKFBAAAA|221|Washington Hickory|Parkway|Suite 180|Fairfield|Russell County|KY|46192|United States|-5|single family| +5537|AAAAAAAABKFBAAAA|243|7th Sixth|Ave|Suite I|Marion|Dauphin County|PA|10399|United States|-5|apartment| +5538|AAAAAAAACKFBAAAA|191|Eigth |Avenue|Suite I|Centerville|Hale County|AL|30059|United States|-6|apartment| +5539|AAAAAAAADKFBAAAA|257|Smith |Wy|Suite Y|Oakwood|Oxford County|ME|00769|United States|-5|apartment| +5540|AAAAAAAAEKFBAAAA|365|2nd |ST|Suite 230|Summit|Lincoln County|WY|80499|United States|-7|single family| +5541|AAAAAAAAFKFBAAAA|839|Church Maple|Road|Suite R|Stringtown|Dolores County|CO|80162|United States|-7|condo| +5542|AAAAAAAAGKFBAAAA|486|Green College|Pkwy|Suite 380|Oakwood|Rhea County|TN|30169|United States|-6|apartment| +5543|AAAAAAAAHKFBAAAA|299|Oak Chestnut|Ct.|Suite I|Concord|Saline County|AR|74107|United States|-6|condo| +5544|AAAAAAAAIKFBAAAA|637|Adams |Pkwy|Suite 260|Oakwood|Hudspeth County|TX|70169|United States|-6|single family| +5545|AAAAAAAAJKFBAAAA|779|Tenth Main|Avenue|Suite 10|Edgewood|Hardin County|OH|40069|United States|-5|apartment| +5546|AAAAAAAAKKFBAAAA|6|9th Adams|Blvd|Suite 410|Oak Hill|Real County|TX|77838|United States|-6|apartment| +5547|AAAAAAAALKFBAAAA|460|Green |Lane|Suite 420|Bunker Hill|Jefferson County|TN|30150|United States|-6|single family| +5548|AAAAAAAAMKFBAAAA|754|Oak Church|Boulevard|Suite M|Wildwood|Sargent County|ND|56871|United States|-6|apartment| +5549|AAAAAAAANKFBAAAA|232|Franklin |Street|Suite 40|Fairfield|Atchison County|KS|66192|United States|-6|condo| +5550|AAAAAAAAOKFBAAAA|863|Church |Street|Suite 180|Crossroads|Jim Hogg County|TX|70534|United States|-6|single family| +5551|AAAAAAAAPKFBAAAA|674|3rd 4th|Dr.|Suite U|Oakdale|Chautauqua County|NY|19584|United States|-5|single family| +5552|AAAAAAAAALFBAAAA|786|Fourth Cedar|Street|Suite 350|Springfield|New London County|CT|09903|United States|-5|condo| +5553|AAAAAAAABLFBAAAA|674|Laurel |Parkway|Suite R|Five Points|Mellette County|SD|56098|United States|-7|single family| +5554|AAAAAAAACLFBAAAA|1|2nd Main|Parkway|Suite K|Forest Hills|Carbon County|UT|89237|United States|-7|single family| +5555|AAAAAAAADLFBAAAA|126|Highland Park|Court|Suite E|San Jose|King George County|VA|28003|United States|-5|single family| +5556|AAAAAAAAELFBAAAA|127|Washington |Drive|Suite S|Lakewood|Washington County|PA|18877|United States|-5|apartment| +5557|AAAAAAAAFLFBAAAA|5|Cedar Locust|Dr.|Suite 40|Riverside|Oscoda County|MI|49231|United States|-5|single family| +5558|AAAAAAAAGLFBAAAA|581|Jefferson Second|Parkway|Suite P|Oakwood|Gila County|AZ|80169|United States|-7|condo| +5559|AAAAAAAAHLFBAAAA|821|Ninth |Cir.|Suite Q|Riverside|Crawford County|WI|59231|United States|-6|single family| +5560|AAAAAAAAILFBAAAA|244|Franklin |Road|Suite K|Edgewood|Oktibbeha County|MS|50069|United States|-6|condo| +5561|AAAAAAAAJLFBAAAA|936|Adams Mill|Wy|Suite Y|Brownsville|Tuscaloosa County|AL|39310|United States|-6|apartment| +5562|AAAAAAAAKLFBAAAA|777|Cherry |ST|Suite F|Post Oak|Pulaski County|VA|28567|United States|-5|single family| +5563|AAAAAAAALLFBAAAA|463|Miller |Circle|Suite D|Brownsville|Hill County|MT|69310|United States|-7|apartment| +5564|AAAAAAAAMLFBAAAA|247|Second |Dr.|Suite 480|Green Acres|Ida County|IA|57683|United States|-6|single family| +5565|AAAAAAAANLFBAAAA|312|Hillcrest 6th|Ln|Suite 10|Shiloh|Graham County|AZ|89275|United States|-7|apartment| +5566|AAAAAAAAOLFBAAAA|766|Main Park|Pkwy|Suite C|Lebanon|Island County|WA|92898|United States|-8|single family| +5567|AAAAAAAAPLFBAAAA|58|Hill |Dr.|Suite 220|Oak Grove|Lamar County|GA|38370|United States|-5|single family| +5568|AAAAAAAAAMFBAAAA|567|River |Pkwy|Suite 140|Shiloh|Nome Census Area|AK|99275|United States|-9|apartment| +5569|AAAAAAAABMFBAAAA|537|4th Chestnut|Ln|Suite F|Hillcrest|Greer County|OK|73003|United States|-6|single family| +5570|AAAAAAAACMFBAAAA|412|Spring |Way|Suite 250|Fairfield|Washington County|FL|36192|United States|-5|condo| +5571|AAAAAAAADMFBAAAA|873|College Laurel|Boulevard|Suite R|Newtown|Hall County|NE|61749|United States|-6|condo| +5572|AAAAAAAAEMFBAAAA|927|Tenth Willow|Pkwy|Suite Y|Valley View|Lake County|IN|45124|United States|-5|apartment| +5573|AAAAAAAAFMFBAAAA|102|Elm Center|Dr.|Suite 110|Summit|Stewart County|GA|30499|United States|-5|single family| +5574|AAAAAAAAGMFBAAAA|444|6th |Lane|Suite C|Saint Johns|Edwards County|KS|65717|United States|-6|single family| +5575|AAAAAAAAHMFBAAAA|||RD|||Oregon County|MO|61904|United States||| +5576|AAAAAAAAIMFBAAAA|128|3rd First|Wy|Suite 230|Granite|Chelan County|WA|96284|United States|-8|single family| +5577|AAAAAAAAJMFBAAAA|142|Oak |RD|Suite E|Slabtown|Graves County|KY|44422|United States|-6|apartment| +5578|AAAAAAAAKMFBAAAA|742|13th Ridge|Avenue|Suite 390|Providence|Union County|IA|56614|United States|-6|condo| +5579|AAAAAAAALMFBAAAA|945|5th |Dr.|Suite B|Greenfield|Hooker County|NE|65038|United States|-7|single family| +5580|AAAAAAAAMMFBAAAA|764|Cedar |Dr.|Suite 20|Wildwood|Macon County|TN|36871|United States|-6|single family| +5581|AAAAAAAANMFBAAAA|735|7th |Ave|Suite M|Centerville|Essex County|NY|10059|United States|-5|apartment| +5582|AAAAAAAAOMFBAAAA|585|Forest Lake|Drive|Suite 350|Centerville|Buena Vista city|VA|20059|United States|-5|condo| +5583|AAAAAAAAPMFBAAAA|11|Highland |Street|Suite U|Harmony|Shannon County|MO|65804|United States|-6|single family| +5584|AAAAAAAAANFBAAAA|378|College |Way|Suite 50|Pleasant Grove|Berrien County|MI|44136|United States|-5|apartment| +5585|AAAAAAAABNFBAAAA|71|Spruce Davis|Ave|Suite V|Hamilton|Jefferson County|OK|72808|United States|-6|single family| +5586|AAAAAAAACNFBAAAA|261|Willow Second|Wy|Suite 60|Spring Valley|Benton County|OR|96060|United States|-8|condo| +5587|AAAAAAAADNFBAAAA|285|Pine Hill|Dr.|Suite M|Five Points|Marion County|OR|96098|United States|-8|condo| +5588|AAAAAAAAENFBAAAA|335|Broadway 2nd|Road|Suite 90|Springfield|Isabella County|MI|49303|United States|-5|condo| +5589|AAAAAAAAFNFBAAAA|564|Third Walnut|Lane|Suite 260|Woodlawn|Bollinger County|MO|64098|United States|-6|condo| +5590|AAAAAAAAGNFBAAAA|716|Pine Sycamore|Court|Suite G|Adrian|Effingham County|IL|63301|United States|-6|single family| +5591|AAAAAAAAHNFBAAAA|285|Johnson Main|Dr.|Suite X|Hopewell|Nicholas County|KY|40587|United States|-5|apartment| +5592|AAAAAAAAINFBAAAA|606|Hickory Chestnut|Ct.|Suite R|Woodlawn|Greenlee County|AZ|84098|United States|-7|condo| +5593|AAAAAAAAJNFBAAAA|217|Mill Poplar|Way|Suite 450|Bridgeport|Juneau Borough|AK|95817|United States|-9|condo| +5594|AAAAAAAAKNFBAAAA|73|Washington Hickory|Way|Suite Y|Carpenter|Red River Parish|LA|71147|United States|-6|apartment| +5595|AAAAAAAALNFBAAAA|671|Sunset 10th|Ct.|Suite 220|Rock Hill|King and Queen County|VA|21364|United States|-5|single family| +5596|AAAAAAAAMNFBAAAA|197|Elm Central|Ct.|Suite S|Ashland|Archer County|TX|74244|United States|-6|single family| +5597|AAAAAAAANNFBAAAA|855|Sunset |Way|Suite R|Stringtown|Madison County|AL|30162|United States|-6|single family| +5598|AAAAAAAAONFBAAAA|277|North |Court|Suite 180|Concord|Marshall County|WV|24107|United States|-5|single family| +5599|AAAAAAAAPNFBAAAA|200|East |Cir.|Suite 0|Harmony|Carroll County|MO|65804|United States|-6|apartment| +5600|AAAAAAAAAOFBAAAA|360|Railroad |RD|Suite 180|Wildwood|Marion County|FL|36871|United States|-5|apartment| +5601|AAAAAAAABOFBAAAA|267|River Chestnut|Cir.|Suite 460|Oakdale|Doniphan County|KS|69584|United States|-6|single family| +5602|AAAAAAAACOFBAAAA|130|8th |Pkwy|Suite P|Little River|Curry County|OR|90319|United States|-8|apartment| +5603|AAAAAAAADOFBAAAA|443|Jackson Oak|Parkway|Suite 310|Lebanon|Murray County|GA|32898|United States|-5|condo| +5604|AAAAAAAAEOFBAAAA|746|Davis Hill|Drive|Suite S|Mount Olive|Morrill County|NE|68059|United States|-7|single family| +5605|AAAAAAAAFOFBAAAA|784|7th |Road|Suite M|Oak Grove|Bracken County|KY|48370|United States|-6|apartment| +5606|AAAAAAAAGOFBAAAA|927||RD||Newport|Cross County|AR||||| +5607|AAAAAAAAHOFBAAAA|413|Laurel Ash|ST|Suite 320|Macedonia|Leelanau County|MI|41087|United States|-5|single family| +5608|AAAAAAAAIOFBAAAA|19|Walnut Madison|Blvd|Suite 390|Lebanon|Park County|MT|62898|United States|-7|single family| +5609|AAAAAAAAJOFBAAAA|80|Main Adams|Ln|Suite J|Five Forks|Cass County|IL|62293|United States|-6|apartment| +5610|AAAAAAAAKOFBAAAA|613|River 14th|RD|Suite 460|Woodland|Pulaski County|MO|64854|United States|-6|single family| +5611|AAAAAAAALOFBAAAA|848|Dogwood 12th|Cir.|Suite C|Brownsville|Cameron County|PA|19310|United States|-5|apartment| +5612|AAAAAAAAMOFBAAAA|171|9th Johnson|Lane|Suite W|Pleasant Valley|Humboldt County|CA|92477|United States|-8|condo| +5613|AAAAAAAANOFBAAAA|798|Pine Washington|Boulevard|Suite 400|Stratford|Glenn County|CA|96668|United States|-8|condo| +5614|AAAAAAAAOOFBAAAA|27|Ridge |ST|Suite 270|Farmington|Webster County|MS|59145|United States|-6|condo| +5615|AAAAAAAAPOFBAAAA|328|Williams |Way|Suite 410|Deerfield|Summit County|OH|49840|United States|-5|condo| +5616|AAAAAAAAAPFBAAAA|959|Jefferson Laurel|Road|Suite 50|Farmington|Bradley County|TN|39145|United States|-5|condo| +5617|AAAAAAAABPFBAAAA|620|Second |Blvd|Suite 60|Franklin|Bates County|MO|69101|United States|-6|single family| +5618|AAAAAAAACPFBAAAA|602|Williams |Ct.|Suite 250|Centerville|Major County|OK|70059|United States|-6|single family| +5619|AAAAAAAADPFBAAAA|884|Highland |Lane|Suite 300|Lakewood|Boone County|KY|48877|United States|-6|single family| +5620|AAAAAAAAEPFBAAAA||Washington |||Buena Vista||||||| +5621|AAAAAAAAFPFBAAAA|561|Woodland |ST|Suite 420|Liberty|Putnam County|OH|43451|United States|-5|apartment| +5622|AAAAAAAAGPFBAAAA|782|Lake |Way|Suite 320|Edgewood|Johnson County|TN|30069|United States|-6|condo| +5623|AAAAAAAAHPFBAAAA|56|Mill Broadway|Blvd|Suite 40|Macedonia|DeKalb County|TN|31087|United States|-5|condo| +5624|AAAAAAAAIPFBAAAA|353|Second Church|Street|Suite F|Shiloh|Lewis County|ID|89275|United States|-7|condo| +5625|AAAAAAAAJPFBAAAA|338|12th |RD|Suite 400|Lakeview|Washington County|TN|38579|United States|-6|single family| +5626|AAAAAAAAKPFBAAAA|460|Oak 1st|Avenue|Suite U|Riverside|Fairfax city|VA|29231|United States|-5|single family| +5627|AAAAAAAALPFBAAAA|133|Maple Williams|ST|Suite 120|Hillcrest|Chippewa County|WI|53003|United States|-6|single family| +5628|AAAAAAAAMPFBAAAA|387|Madison Ridge|Parkway|Suite 140|Glenwood|Cameron County|PA|13511|United States|-5|single family| +5629|AAAAAAAANPFBAAAA|885|15th |Road|Suite C|Bethel|Golden Valley County|MT|65281|United States|-7|apartment| +5630|AAAAAAAAOPFBAAAA|227|4th |RD|Suite 270|Bethel|Colleton County|SC|25281|United States|-5|apartment| +5631|AAAAAAAAPPFBAAAA|67|Spring Forest|Boulevard|Suite 20|Fairfield|Morgan County|TN|36192|United States|-6|apartment| +5632|AAAAAAAAAAGBAAAA|628|Eigth Park|Lane|Suite 400|Forest Hills|Baltimore County|MD|29237|United States|-5|single family| +5633|AAAAAAAABAGBAAAA|463|North |Court|Suite 270|Maple Grove|Boyle County|KY|48252|United States|-6|condo| +5634|AAAAAAAACAGBAAAA|115|Railroad |Parkway|Suite X|Appleton|Lincoln County|MN|54240|United States|-6|single family| +5635|AAAAAAAADAGBAAAA|419|South |Wy|Suite 320|Glendale|Franklin County|WA|93951|United States|-8|apartment| +5636|AAAAAAAAEAGBAAAA|20|Jefferson |Street|Suite 290|Macedonia|Cimarron County|OK|71087|United States|-6|apartment| +5637|AAAAAAAAFAGBAAAA|19|Hillcrest Adams|Court|Suite S|Maple Grove|Multnomah County|OR|98252|United States|-8|single family| +5638|AAAAAAAAGAGBAAAA|218|4th |Cir.|Suite 380|Riverview|Fairfield County|CT|09603|United States|-5|single family| +5639|AAAAAAAAHAGBAAAA||Johnson ||Suite K|Enterprise||TN|31757|United States|-5|| +5640|AAAAAAAAIAGBAAAA|382|Jackson Fifth|Ln|Suite 190|Riverview|Lamar County|MS|59003|United States|-6|single family| +5641|AAAAAAAAJAGBAAAA|655|10th Sixth|Ct.|Suite 150|Shady Grove|Franklin County|NC|22812|United States|-5|single family| +5642|AAAAAAAAKAGBAAAA|199|College |ST|Suite D|Waterloo|Keith County|NE|61675|United States|-7|condo| +5643|AAAAAAAALAGBAAAA|972|Washington |Wy|Suite 320|Edgewood|Pawnee County|NE|60069|United States|-7|apartment| +5644|AAAAAAAAMAGBAAAA|162|Railroad |Ave|Suite T|Bridgeport|Jackson County|NC|25817|United States|-5|condo| +5645|AAAAAAAANAGBAAAA|178|1st |Boulevard|Suite I|Lakewood|Grafton County|NH|09477|United States|-5|single family| +5646|AAAAAAAAOAGBAAAA|601||Way|||||99584||-8|apartment| +5647|AAAAAAAAPAGBAAAA|960|Maple Main|Cir.|Suite 40|Union Hill|Hancock County|MS|57746|United States|-6|single family| +5648|AAAAAAAAABGBAAAA|815|Washington |Circle|Suite 420|Greenfield|Muscatine County|IA|55038|United States|-6|condo| +5649|AAAAAAAABBGBAAAA|182|Cedar |Ct.|Suite W|Spring Hill|Sheridan County|KS|66787|United States|-6|apartment| +5650|AAAAAAAACBGBAAAA|995|1st |Cir.|Suite 410|Highland Park|Louisa County|IA|56534|United States|-6|single family| +5651|AAAAAAAADBGBAAAA|991|9th Meadow|Circle|Suite A|Wildwood|Butler County|AL|36871|United States|-6|single family| +5652|AAAAAAAAEBGBAAAA|210|Lakeview |Dr.|Suite C|Oakland|Rhea County|TN|39843|United States|-6|condo| +5653|AAAAAAAAFBGBAAAA|444|5th Main|Way|Suite U|Mount Vernon|Fredericksburg city|VA|28482|United States|-5|condo| +5654|AAAAAAAAGBGBAAAA|51|Hillcrest |Blvd|Suite 430|Greenville|Essex County|NJ|01987|United States|-5|single family| +5655|AAAAAAAAHBGBAAAA|316|Maple |Boulevard|Suite 420|Mount Pleasant|Richland County|MT|61933|United States|-7|apartment| +5656|AAAAAAAAIBGBAAAA|450|Poplar Elm|Blvd|Suite X|Deerfield|Mitchell County|IA|59840|United States|-6|single family| +5657|AAAAAAAAJBGBAAAA|259|Second Railroad|Way|Suite K|Highland|Alexandria city|VA|29454|United States|-5|condo| +5658|AAAAAAAAKBGBAAAA|537|Tenth |Wy|Suite 100|Lebanon|Saratoga County|NY|12898|United States|-5|condo| +5659|AAAAAAAALBGBAAAA|541|Mill Cedar|ST|Suite A|Wildwood|Andrew County|MO|66871|United States|-6|condo| +5660|AAAAAAAAMBGBAAAA|453|Washington Cherry|Ln|Suite L|Buena Vista|Itasca County|MN|55752|United States|-6|condo| +5661|AAAAAAAANBGBAAAA|541|Madison |Ave|Suite Y|Antioch|Madison County|IN|48605|United States|-5|single family| +5662|AAAAAAAAOBGBAAAA|755|Cedar Oak|Ln|Suite 370|Riverview|Simpson County|MS|59003|United States|-6|apartment| +5663|AAAAAAAAPBGBAAAA|986|Spring |Ln|Suite 260|Marion|Prentiss County|MS|50399|United States|-6|condo| +5664|AAAAAAAAACGBAAAA|88|Williams Fourth|Blvd|Suite A|New Hope|Gilpin County|CO|89431|United States|-7|single family| +5665|AAAAAAAABCGBAAAA|698|Sunset Sunset|Parkway|Suite C|Glendale|Ashley County|AR|73951|United States|-6|apartment| +5666|AAAAAAAACCGBAAAA|461|Jackson View|Court|Suite 340|Wildwood|Sharp County|AR|76871|United States|-6|apartment| +5667|AAAAAAAADCGBAAAA|641|Washington Park|Road|Suite G|Midway|Caldwell County|MO|61904|United States|-6|condo| +5668|AAAAAAAAECGBAAAA|686|3rd Ridge|Ct.|Suite 460|Greenville|Dade County|GA|31387|United States|-5|single family| +5669|AAAAAAAAFCGBAAAA|989|Church |Boulevard|Suite S|Riverdale|Colusa County|CA|99391|United States|-8|single family| +5670|AAAAAAAAGCGBAAAA|566|Sycamore |Ave|Suite W|Pleasant Valley|Pope County|MN|52477|United States|-6|apartment| +5671|AAAAAAAAHCGBAAAA|1000|Pine |Blvd|Suite 480|Fairview|Lawrence County|IN|45709|United States|-5|apartment| +5672|AAAAAAAAICGBAAAA|463|Franklin Miller|ST|Suite 0|River Oaks|Butts County|GA|38075|United States|-5|apartment| +5673|AAAAAAAAJCGBAAAA|497|Johnson Spring|Circle|Suite 330|Parkwood|Nowata County|OK|71669|United States|-6|single family| +5674|AAAAAAAAKCGBAAAA|990|Park |Avenue|Suite B|Bunker Hill|Okfuskee County|OK|70150|United States|-6|single family| +5675|AAAAAAAALCGBAAAA|688|Third |Circle|Suite R|Lebanon|Cassia County|ID|82898|United States|-7|condo| +5676|AAAAAAAAMCGBAAAA|420|Eigth Hillcrest|Avenue|Suite 160|Williamsville|Kent County|DE|18754|United States|-5|condo| +5677|AAAAAAAANCGBAAAA|498|Park Hickory|Way|Suite 250|Edgewood|Beckham County|OK|70069|United States|-6|single family| +5678|AAAAAAAAOCGBAAAA|750|Tenth Lake|Blvd|Suite C|Florence|Jasper County|IA|53394|United States|-6|condo| +5679|AAAAAAAAPCGBAAAA|362|Washington |Circle|Suite 150|Deerfield|Hampton County|SC|29840|United States|-5|condo| +5680|AAAAAAAAADGBAAAA|328|7th |Circle|Suite 360|Glenwood|Gunnison County|CO|83511|United States|-7|apartment| +5681|AAAAAAAABDGBAAAA|922|Adams Green|Ln|Suite 220|Nichols|Clearfield County|PA|17940|United States|-5|apartment| +5682|AAAAAAAACDGBAAAA|222|Maple |Ln|Suite 370|Glenwood|Knox County|TX|73511|United States|-6|single family| +5683|AAAAAAAADDGBAAAA|176|Third |Circle|Suite G|Bunker Hill|Red Lake County|MN|50150|United States|-6|condo| +5684|AAAAAAAAEDGBAAAA|361|Spruce |Avenue|Suite 190|Hamilton|Robertson County|TX|72808|United States|-6|apartment| +5685|AAAAAAAAFDGBAAAA|595|10th Highland|Street|Suite 260|Jackson|Marshall County|SD|59583|United States|-7|condo| +5686|AAAAAAAAGDGBAAAA|423|Smith Sycamore|RD|Suite X|Brownsville|Valley County|ID|89310|United States|-7|condo| +5687|AAAAAAAAHDGBAAAA|614|South |Pkwy|Suite T|Concord|Tarrant County|TX|74107|United States|-6|apartment| +5688|AAAAAAAAIDGBAAAA|59|3rd 3rd|Circle|Suite 200|Highland Park|Adair County|IA|56534|United States|-6|apartment| +5689|AAAAAAAAJDGBAAAA|373|East |Court|Suite 220|Snug Harbor|Hoke County|NC|27936|United States|-5|apartment| +5690|AAAAAAAAKDGBAAAA|284|2nd Center|Court|Suite 60|Summit|Rappahannock County|VA|20499|United States|-5|condo| +5691|AAAAAAAALDGBAAAA|333|6th |RD|Suite 420|Harmony|Lafourche Parish|LA|75804|United States|-6|apartment| +5692|AAAAAAAAMDGBAAAA|739|Sycamore Ridge|ST|Suite M|Cedar Grove|Brooke County|WV|20411|United States|-5|condo| +5693|AAAAAAAANDGBAAAA|253|Maple |Blvd|Suite 170|Valley View|Jeff Davis County|TX|75124|United States|-6|apartment| +5694|AAAAAAAAODGBAAAA|209|Second 9th|Avenue|Suite V|Greenfield|Estill County|KY|45038|United States|-6|apartment| +5695|AAAAAAAAPDGBAAAA|831|1st |Street|Suite 190|Five Points|Graham County|KS|66098|United States|-6|single family| +5696|AAAAAAAAAEGBAAAA|148|||Suite P|Five Points|Mifflin County|PA|16098||-5|condo| +5697|AAAAAAAABEGBAAAA|917|3rd |Dr.|Suite 420|Wilson|Monroe County|MS|56971|United States|-6|single family| +5698|AAAAAAAACEGBAAAA|883|Park Spring|Ln|Suite 370|Lebanon|Stewart County|TN|32898|United States|-6|single family| +5699|AAAAAAAADEGBAAAA|248|Willow Eigth|Wy|Suite 100|Ashland|Coal County|OK|74244|United States|-6|condo| +5700|AAAAAAAAEEGBAAAA|696|Central South|Court|Suite J|Glendale|Stark County|IL|63951|United States|-6|single family| +5701|AAAAAAAAFEGBAAAA|363|View Broadway|Street|Suite 30|Macedonia|Trinity County|CA|91087|United States|-8|apartment| +5702|AAAAAAAAGEGBAAAA|781|Maple ||||Windsor County|VT|09903|United States|-5|apartment| +5703|AAAAAAAAHEGBAAAA|641|Hillcrest |Parkway|Suite A|Marion|Warren County|IL|60399|United States|-6|single family| +5704|AAAAAAAAIEGBAAAA|998|8th |Way|Suite R|Maple Grove|Clear Creek County|CO|88252|United States|-7|apartment| +5705|AAAAAAAAJEGBAAAA|527|Pine Madison|Court|Suite 120|Lakeview|Van Wert County|OH|48579|United States|-5|condo| +5706|AAAAAAAAKEGBAAAA|829|2nd Lincoln|ST|Suite L|Proctor|Phillips County|AR|78140|United States|-6|condo| +5707|AAAAAAAALEGBAAAA|604||RD|Suite 310||||||-6|condo| +5708|AAAAAAAAMEGBAAAA|73|Dogwood |Dr.|Suite X|Greenville|Sanilac County|MI|41387|United States|-5|single family| +5709|AAAAAAAANEGBAAAA|377|View |Avenue|Suite N|Waterloo|Hanson County|SD|51675|United States|-7|single family| +5710|AAAAAAAAOEGBAAAA|60|Pine Willow|Circle|Suite J|Oak Ridge|Saline County|IL|68371|United States|-6|condo| +5711|AAAAAAAAPEGBAAAA|690|6th 3rd|Drive|Suite G|Lakeside|Lebanon County|PA|19532|United States|-5|apartment| +5712|AAAAAAAAAFGBAAAA|662|Meadow South|||Green Acres|Valley County|||||| +5713|AAAAAAAABFGBAAAA|342|6th |Parkway|Suite 290|Union City|Grant County|IN|48087|United States|-5|apartment| +5714|AAAAAAAACFGBAAAA|265|Williams Central|Boulevard|Suite A|Wildwood|Harmon County|OK|76871|United States|-6|single family| +5715|AAAAAAAADFGBAAAA|786|Main |Cir.|Suite L|Sulphur Springs|Caldwell County|KY|48354|United States|-6|condo| +5716|AAAAAAAAEFGBAAAA|735|2nd Mill|Blvd|Suite G|Macedonia|Windham County|VT|01687|United States|-5|condo| +5717|AAAAAAAAFFGBAAAA|884|Green |Pkwy|Suite 450|Greenwood|Fulton County|OH|48828|United States|-5|condo| +5718|AAAAAAAAGFGBAAAA|777|7th |Road|Suite D|The Meadows|Washington County|KS|60026|United States|-6|single family| +5719|AAAAAAAAHFGBAAAA|596|Lee |Ln|Suite C|Hillcrest|Harlan County|KY|43003|United States|-6|apartment| +5720|AAAAAAAAIFGBAAAA|74|Cedar Hill|Drive|Suite Q|Shiloh|Macon County|NC|29275|United States|-5|apartment| +5721|AAAAAAAAJFGBAAAA|547|Hickory Oak|Cir.|Suite Y|Hillcrest|Lee County|AR|73003|United States|-6|single family| +5722|AAAAAAAAKFGBAAAA|368|Highland Smith|Road|Suite 40|Providence|Chesterfield County|SC|26614|United States|-5|condo| +5723|AAAAAAAALFGBAAAA|113|Chestnut |Road|Suite 450|Oakdale|Westmoreland County|VA|29584|United States|-5|condo| +5724|AAAAAAAAMFGBAAAA|682|Walnut Railroad|Cir.|Suite I|Belmont|Big Horn County|MT|60191|United States|-7|apartment| +5725|AAAAAAAANFGBAAAA|140|Fifth North|Cir.|Suite 130|Wilson|Texas County|MO|66971|United States|-6|single family| +5726|AAAAAAAAOFGBAAAA|543|1st First|Ave|Suite L|Friendship|Sheboygan County|WI|54536|United States|-6|condo| +5727|AAAAAAAAPFGBAAAA|663|Sycamore Madison|Parkway|Suite S|Belmont|Jefferson County|TX|70191|United States|-6|single family| +5728|AAAAAAAAAGGBAAAA|435|Lake |Ave|Suite 420|Waterloo|Rockingham County|NC|21675|United States|-5|condo| +5729|AAAAAAAABGGBAAAA|608|10th |Court|Suite A|Highland Park|Dooly County|GA|36534|United States|-5|single family| +5730|AAAAAAAACGGBAAAA|302|Ash Main|Lane|Suite N|Edgewood|Dallam County|TX|70069|United States|-6|single family| +5731|AAAAAAAADGGBAAAA|433|View |Pkwy|Suite 390|Union Hill|Putnam County|TN|37746|United States|-6|condo| +5732|AAAAAAAAEGGBAAAA|940|Meadow |Boulevard|Suite Q|Sunnyside|Hardin County|KY|41952|United States|-6|condo| +5733|AAAAAAAAFGGBAAAA|685|Center Church|Avenue|Suite C|Oakland|Wayne County|TN|39843|United States|-6|apartment| +5734|AAAAAAAAGGGBAAAA|599|Fifth |Cir.|Suite D|Plainville|Roscommon County|MI|46115|United States|-5|single family| +5735|AAAAAAAAHGGBAAAA|198|Ridge |RD|Suite 160|Riverside|Roane County|TN|39231|United States|-6|single family| +5736|AAAAAAAAIGGBAAAA||Fifth Poplar||||Poquoson city|VA|||-5|| +5737|AAAAAAAAJGGBAAAA|124|7th Lincoln|Court|Suite 440|Spring Hill|Macomb County|MI|46787|United States|-5|apartment| +5738|AAAAAAAAKGGBAAAA|708|First |Wy|Suite B|Jackson|Washington County|ID|89583|United States|-7|single family| +5739|AAAAAAAALGGBAAAA|207|Center College|Cir.|Suite N|White Oak|Washington Parish|LA|76668|United States|-6|condo| +5740|AAAAAAAAMGGBAAAA|602|13th 7th|Parkway|Suite 310|Tyrone|Martin County|FL|31201|United States|-5|apartment| +5741|AAAAAAAANGGBAAAA|122|South Franklin|Boulevard|Suite H|Bunker Hill|Livingston County|KY|40150|United States|-5|apartment| +5742|AAAAAAAAOGGBAAAA|847|Maple |Way|Suite J|Georgetown|Effingham County|IL|67057|United States|-6|condo| +5743|AAAAAAAAPGGBAAAA|711|Ridge 1st|Parkway|Suite 70|Arlington|Nash County|NC|26557|United States|-5|apartment| +5744|AAAAAAAAAHGBAAAA|858|Lakeview |Way|Suite G|Riverside|Upton County|TX|79231|United States|-6|single family| +5745|AAAAAAAABHGBAAAA|530|South |RD|Suite 410|Midway|Mercer County|NJ|02504|United States|-5|apartment| +5746|AAAAAAAACHGBAAAA|529|Meadow Lincoln|Circle|Suite 150|Summit|East Feliciana Parish|LA|70499|United States|-6|single family| +5747|AAAAAAAADHGBAAAA|738|Maple 9th|Cir.|Suite 380|Hillcrest|Sherman County|KS|63003|United States|-6|apartment| +5748|AAAAAAAAEHGBAAAA|456|Church Washington|Dr.|Suite 90|Woodville|Refugio County|TX|74289|United States|-6|apartment| +5749|AAAAAAAAFHGBAAAA|957|Ash Ninth|Boulevard|Suite 260|The Meadows|Le Flore County|OK|70026|United States|-6|apartment| +5750|AAAAAAAAGHGBAAAA|814|Johnson River|Street|Suite X|Franklin|Kane County|UT|89101|United States|-7|apartment| +5751|AAAAAAAAHHGBAAAA|175|River |Wy|Suite 320|New Hope|Madison County|TX|79431|United States|-6|condo| +5752|AAAAAAAAIHGBAAAA|535|2nd Sixth|RD|Suite M|Unionville|Ralls County|MO|61711|United States|-6|apartment| +5753|AAAAAAAAJHGBAAAA|112|Lakeview Jackson|Wy|Suite B|Cedar Grove|Newport News city|VA|20411|United States|-5|apartment| +5754|AAAAAAAAKHGBAAAA|108|2nd Wilson|Lane|Suite 410|Wilson|Rutherford County|TN|36971|United States|-6|single family| +5755|AAAAAAAALHGBAAAA|186|10th Johnson|Boulevard|Suite 330|Wildwood|Cumberland County|VA|26871|United States|-5|condo| +5756|AAAAAAAAMHGBAAAA|54|Broadway |Dr.|Suite 40|Waterloo|Nobles County|MN|51675|United States|-6|apartment| +5757|AAAAAAAANHGBAAAA|283|Willow Hickory|RD|Suite B|Bridgeport|Metcalfe County|KY|45817|United States|-5|single family| +5758|AAAAAAAAOHGBAAAA|731|Adams |Boulevard|Suite 160|Oak Hill|Loving County|TX|77838|United States|-6|condo| +5759|AAAAAAAAPHGBAAAA|361|Highland Second|Street|Suite 220|Oakdale|Swisher County|TX|79584|United States|-6|apartment| +5760|AAAAAAAAAIGBAAAA|388|Mill Fifteenth|Circle|Suite 190|Pine Grove|Grainger County|TN|34593|United States|-5|apartment| +5761|AAAAAAAABIGBAAAA|878|Sixth |Road|Suite 200|Pleasant Valley|Winston County|AL|32477|United States|-6|apartment| +5762|AAAAAAAACIGBAAAA|987|Park |Way|Suite 430|Fairview|Cayuga County|NY|15709|United States|-5|condo| +5763|AAAAAAAADIGBAAAA|125|4th |Parkway|Suite A|Riverside|Lincoln County|MT|69231|United States|-7|condo| +5764|AAAAAAAAEIGBAAAA|468|Laurel |Cir.|Suite 480|Reno|Yavapai County|AZ|80344|United States|-7|apartment| +5765|AAAAAAAAFIGBAAAA|864|Park Woodland|Ave|Suite 420|Bloomingdale|San Jacinto County|TX|71824|United States|-6|single family| +5766|AAAAAAAAGIGBAAAA|210|11th Spruce|ST|Suite M|Providence|Boone County|WV|26614|United States|-5|apartment| +5767|AAAAAAAAHIGBAAAA|457|Sunset |Road|Suite C|New Hope|Toombs County|GA|39431|United States|-5|single family| +5768|AAAAAAAAIIGBAAAA|902|Railroad |Court|Suite N|Jackson|Marshall County|AL|39583|United States|-6|condo| +5769|AAAAAAAAJIGBAAAA|531|Elm |Parkway|Suite C|Mount Olive|Edwards County|IL|68059|United States|-6|apartment| +5770|AAAAAAAAKIGBAAAA|946|Sunset |Parkway|Suite 110|Fairview|Garfield County|WA|95709|United States|-8|condo| +5771|AAAAAAAALIGBAAAA|273|Willow 3rd|Avenue|Suite K|Cedar Grove|Perry County|OH|40411|United States|-5|condo| +5772|AAAAAAAAMIGBAAAA|603|Smith Second|Road|Suite 130|Fairfield|Meagher County|MT|66192|United States|-7|condo| +5773|AAAAAAAANIGBAAAA|317|Third |Dr.|Suite F|Bethel|Atlantic County|NJ|05881|United States|-5|apartment| +5774|AAAAAAAAOIGBAAAA|887|10th |Wy|Suite W|Shiloh|Falls County|TX|79275|United States|-6|apartment| +5775|AAAAAAAAPIGBAAAA|460|Main Chestnut|Road|Suite S|Mount Olive|Scott County|IL|68059|United States|-6|condo| +5776|AAAAAAAAAJGBAAAA|658|Elm |Street|Suite R|Highland|Tuscaloosa County|AL|39454|United States|-6|apartment| +5777|AAAAAAAABJGBAAAA|988|Railroad |Street|Suite 310|Hillcrest|Cowlitz County|WA|93003|United States|-8|single family| +5778|AAAAAAAACJGBAAAA|392|Hickory |Circle|Suite 320|Woodcrest|DeKalb County|GA|34919|United States|-5|condo| +5779|AAAAAAAADJGBAAAA|||Wy|Suite 140|Farmington|Prince George County|VA|29145|United States|-5|| +5780|AAAAAAAAEJGBAAAA|456|Davis |Lane|Suite Y|Mount Pleasant|Lee County|IA|51933|United States|-6|single family| +5781|AAAAAAAAFJGBAAAA||10th 11th|||Macedonia|Preble County|OH||||| +5782|AAAAAAAAGJGBAAAA|591|15th 8th|Dr.|Suite I|Saint Clair|Powhatan County|VA|25294|United States|-5|single family| +5783|AAAAAAAAHJGBAAAA|944|Park |ST|Suite L|Greenville|Cumberland County|ME|01987|United States|-5|single family| +5784|AAAAAAAAIJGBAAAA|622|Oak |Parkway|Suite W|Mount Pleasant|Idaho County|ID|81933|United States|-7|single family| +5785|AAAAAAAAJJGBAAAA|331|Washington |Street|Suite M|Little River|Berkeley County|WV|20319|United States|-5|condo| +5786|AAAAAAAAKJGBAAAA|721|Cedar Hill|Ct.|Suite 50|Riverview|Traill County|ND|59003|United States|-6|apartment| +5787|AAAAAAAALJGBAAAA|197|Valley First|Court|Suite 30|Deerfield|Colonial Heights city|VA|29840|United States|-5|condo| +5788|AAAAAAAAMJGBAAAA|786|Pine College|Boulevard|Suite U|Riverside|Delta County|TX|79231|United States|-6|condo| +5789|AAAAAAAANJGBAAAA|454|Locust 11th|Lane|Suite S|Hamilton|Minnehaha County|SD|52808|United States|-7|condo| +5790|AAAAAAAAOJGBAAAA|417|Maple Highland|Ave|Suite E|Ashland|Marshall County|WV|24244|United States|-5|condo| +5791|AAAAAAAAPJGBAAAA|587|Walnut |Drive|Suite L|Fairview|Blaine County|MT|65709|United States|-7|condo| +5792|AAAAAAAAAKGBAAAA|991|North Lake|Street|Suite K|Sherwood Forest|Lincoln County|ME|07302|United States|-5|single family| +5793|AAAAAAAABKGBAAAA|977|Eigth Lake|Ct.|Suite Q|Concord|Mills County|TX|74107|United States|-6|condo| +5794|AAAAAAAACKGBAAAA|986|1st |Blvd|Suite B|Oakdale|Banner County|NE|69584|United States|-6|apartment| +5795|AAAAAAAADKGBAAAA|124|5th Hickory|Boulevard|Suite 60|Maple Grove|Cumberland County|IL|68252|United States|-6|single family| +5796|AAAAAAAAEKGBAAAA|195|Park 13th|Street|Suite L|Rolling Hills|Mille Lacs County|MN|57272|United States|-6|condo| +5797|AAAAAAAAFKGBAAAA|922|View |Way|Suite 180|Marion|Grant County|WA|90399|United States|-8|single family| +5798|AAAAAAAAGKGBAAAA|790|Pine Central|Street|Suite T|Lakewood|Jefferson County|NE|68877|United States|-7|apartment| +5799|AAAAAAAAHKGBAAAA|518|4th |Dr.|Suite J|Five Forks|Bradford County|FL|32293|United States|-5|apartment| +5800|AAAAAAAAIKGBAAAA|57|Maple |Ln|||Riley County||69145||-6|| +5801|AAAAAAAAJKGBAAAA||1st ||Suite F|Glenwood|Stephens County|||||| +5802|AAAAAAAAKKGBAAAA|||RD|Suite 300|Oak Ridge|Kalamazoo County|MI|||-5|apartment| +5803|AAAAAAAALKGBAAAA|826|2nd |Ct.|Suite S|Riverside|Washington County|ME|09831|United States|-5|condo| +5804|AAAAAAAAMKGBAAAA|824|6th Walnut|Cir.|Suite 30|Lakeside|Rappahannock County|VA|29532|United States|-5|single family| +5805|AAAAAAAANKGBAAAA|462|8th Church|Cir.|Suite 250|Ashland|East Baton Rouge Parish|LA|74244|United States|-6|apartment| +5806|AAAAAAAAOKGBAAAA|273|Railroad |Parkway|Suite 290|Georgetown|Powell County|KY|47057|United States|-5|condo| +5807|AAAAAAAAPKGBAAAA|645|3rd |Ave|Suite 450|Midway|Burt County|NE|61904|United States|-6|condo| +5808|AAAAAAAAALGBAAAA|903|South |Circle|Suite 170|Mount Vernon|Harrison County|IA|58482|United States|-6|single family| +5809|AAAAAAAABLGBAAAA|850|Lake |Lane|Suite 90||Clark County|AR|78041|||| +5810|AAAAAAAACLGBAAAA|409|Sunset |Avenue|Suite 150|Georgetown|Kauai County|HI|97057|United States|-10|single family| +5811|AAAAAAAADLGBAAAA|652|Park |Street|Suite 440|Bunker Hill|Jerome County|ID|80150|United States|-7|condo| +5812|AAAAAAAAELGBAAAA|512|Fifth Jackson|Street|Suite 50|Hillcrest|Sullivan County|MO|63003|United States|-6|single family| +5813|AAAAAAAAFLGBAAAA|44|5th Hill|Wy|Suite 20|Bunker Hill|Stanton County|KS|60150|United States|-6|single family| +5814|AAAAAAAAGLGBAAAA|154|Second Green|Ln|Suite 160|Woodville|Juab County|UT|84289|United States|-7|condo| +5815|AAAAAAAAHLGBAAAA|425|Elm North|Ct.|Suite 330|Lakeside|Washington County|KS|69532|United States|-6|condo| +5816|AAAAAAAAILGBAAAA|776|Highland Lee|Lane|Suite N|Frankfort|Buffalo County|NE|69681|United States|-6|condo| +5817|AAAAAAAAJLGBAAAA|871|Madison Second|Boulevard|Suite 60|Brownsville|Irwin County|GA|39310|United States|-5|single family| +5818|AAAAAAAAKLGBAAAA|942|Hillcrest |Way|Suite 70|White Oak|Calhoun County|GA|36668|United States|-5|apartment| +5819|AAAAAAAALLGBAAAA|759|Elm 8th|Lane|Suite 190|Brownsville|Pickens County|AL|39310|United States|-6|single family| +5820|AAAAAAAAMLGBAAAA|637|Mill |Avenue|||Isanti County|MN||United States||condo| +5821|AAAAAAAANLGBAAAA|926|1st |Avenue|Suite 320|Valley View|Clay County|TN|35124|United States|-5|single family| +5822|AAAAAAAAOLGBAAAA|854|2nd |Way|Suite 300|Friendship|Faulk County|SD|54536|United States|-6|condo| +5823|AAAAAAAAPLGBAAAA|139|River Main|Pkwy|Suite B|Pine Grove|Braxton County|WV|24593|United States|-5|apartment| +5824|AAAAAAAAAMGBAAAA|564|Green Central|Ct.|Suite 190|Red Bank|Rains County|TX|74975|United States|-6|condo| +5825|AAAAAAAABMGBAAAA|238|Railroad Spruce|ST|Suite 100|Friendship|Franklin County|IL|64536|United States|-6|condo| +5826|AAAAAAAACMGBAAAA|44|Maple Hillcrest|Circle|Suite 390|Rossville|Halifax County|NC|22503|United States|-5|condo| +5827|AAAAAAAADMGBAAAA|393|Wilson |Road|Suite 390|New Hope|Poinsett County|AR|79431|United States|-6|single family| +5828|AAAAAAAAEMGBAAAA|972|Fourteenth |Street|Suite 70|Bridgeport|Aiken County|SC|25817|United States|-5|condo| +5829|AAAAAAAAFMGBAAAA|398|View North|Drive|Suite B|Shiloh|Abbeville County|SC|29275|United States|-5|apartment| +5830|AAAAAAAAGMGBAAAA|647|Cherry |Ave|Suite B|Bear Creek|Grant Parish|LA|73075|United States|-6|single family| +5831|AAAAAAAAHMGBAAAA|736|Broadway Laurel|Drive|Suite 460|Gary|Willacy County|TX|70418|United States|-6|single family| +5832|AAAAAAAAIMGBAAAA|150|Central 9th|Blvd|Suite C|Lakeside|Chaffee County|CO|89532|United States|-7|apartment| +5833|AAAAAAAAJMGBAAAA|457|Sunset Wilson|Dr.|Suite X|Pleasant Valley|Hand County|SD|52477|United States|-7|single family| +5834|AAAAAAAAKMGBAAAA|11|15th Lee|Drive|Suite M|Deerfield|Columbiana County|OH|49840|United States|-5|apartment| +5835|AAAAAAAALMGBAAAA|367|Broadway Forest|Avenue|Suite 90|Edgewood|Marion County|IA|50069|United States|-6|apartment| +5836|AAAAAAAAMMGBAAAA|482|College |Road|Suite D|Woodcrest|Hamilton County|NE|64919|United States|-6|single family| +5837|AAAAAAAANMGBAAAA|||Boulevard||Mount Olive|Cortland County||18059|United States||single family| +5838|AAAAAAAAOMGBAAAA|119|Washington Lincoln|Ln|Suite D|Elkton|Jefferson County|GA|33481|United States|-5|condo| +5839|AAAAAAAAPMGBAAAA|720|Willow Hill|Wy|Suite 290|Valley View|Campbell County|VA|25124|United States|-5|apartment| +5840|AAAAAAAAANGBAAAA|680|Poplar Walnut|Boulevard|Suite 440|Glendale|Independence County|AR|73951|United States|-6|condo| +5841|AAAAAAAABNGBAAAA|543|Maple 3rd|Avenue|Suite 110|Shiloh|Sioux County|ND|59275|United States|-6|apartment| +5842|AAAAAAAACNGBAAAA|602|1st |Ct.|Suite D|Springfield|Audrain County|MO|69303|United States|-6|condo| +5843|AAAAAAAADNGBAAAA|347|Hill 3rd|Circle|Suite V|Shady Grove|Wayne County|GA|32812|United States|-5|single family| +5844|AAAAAAAAENGBAAAA|157|Lee Franklin|Ct.|Suite B|Glenwood|Hockley County|TX|73511|United States|-6|single family| +5845|AAAAAAAAFNGBAAAA|371|Third 5th|Pkwy|Suite P|Sulphur Springs|Marathon County|WI|58354|United States|-6|condo| +5846|AAAAAAAAGNGBAAAA|948|Fifth Dogwood|Wy|Suite K|Riverview|Coles County|IL|69003|United States|-6|single family| +5847|AAAAAAAAHNGBAAAA|788|Park |Cir.|Suite N|Fairfield|Dawson County|MT|66192|United States|-7|apartment| +5848|AAAAAAAAINGBAAAA|710|Oak |ST|Suite 260|Hamilton|La Plata County|CO|82808|United States|-7|single family| +5849|AAAAAAAAJNGBAAAA|381|Sunset College|Court|Suite 10|Mount Olive|Prowers County|CO|88059|United States|-7|condo| +5850|AAAAAAAAKNGBAAAA|208|8th |Lane|Suite S|Pleasant Valley|Randolph County|IL|62477|United States|-6|apartment| +5851|AAAAAAAALNGBAAAA|225|Franklin |Street|Suite T|Brownsville|Onslow County|NC|29310|United States|-5|single family| +5852|AAAAAAAAMNGBAAAA|336|11th Washington|Avenue|Suite C|Oak Hill|Kendall County|IL|67838|United States|-6|condo| +5853|AAAAAAAANNGBAAAA|478|View Main|Ave|Suite O|Lakewood|Somerset County|NJ|09477|United States|-5|single family| +5854|AAAAAAAAONGBAAAA|156|Railroad Second|Parkway|Suite 450|Little River|Smith County|KS|60319|United States|-6|apartment| +5855|AAAAAAAAPNGBAAAA||7th ||Suite 290||Whiteside County|IL|64975|United States|-6|apartment| +5856|AAAAAAAAAOGBAAAA|183|Main 3rd|Circle|Suite S|Concord|West Feliciana Parish|LA|74107|United States|-6|condo| +5857|AAAAAAAABOGBAAAA|178|7th |Drive|Suite 370|Providence|Butler County|KS|66614|United States|-6|single family| +5858|AAAAAAAACOGBAAAA|739|Park |Pkwy|Suite 250|Pleasant Grove|Rockbridge County|VA|24136|United States|-5|single family| +5859|AAAAAAAADOGBAAAA|451|Park Madison|Wy|Suite 440|Pine Grove|Salem County|NJ|05193|United States|-5|single family| +5860|AAAAAAAAEOGBAAAA|165|Locust Ridge|Ave|Suite 370|New Hope|Henry County|IN|49431|United States|-5|single family| +5861|AAAAAAAAFOGBAAAA|108|Hill |Cir.|Suite E|Bethel|Calumet County|WI|55281|United States|-6|single family| +5862|AAAAAAAAGOGBAAAA|796|Johnson |Ct.|Suite X|Providence|Calhoun County|IL|66614|United States|-6|condo| +5863|AAAAAAAAHOGBAAAA|538|12th 2nd|Road|Suite P|Sunnyside|Goshen County|WY|81952|United States|-7|apartment| +5864|AAAAAAAAIOGBAAAA|964|Smith |Drive|Suite 240|Mount Vernon|Pitkin County|CO|88482|United States|-7|apartment| +5865|AAAAAAAAJOGBAAAA|756|Jefferson 12th|Way|Suite C|Providence|Hanson County|SD|56614|United States|-7|condo| +5866|AAAAAAAAKOGBAAAA|274|Lincoln |ST|Suite 360|Greenwood|Belknap County|NH|09428|United States|-5|condo| +5867|AAAAAAAALOGBAAAA|711|||Suite E|Valley View|Howard County|MD|25124||-5|single family| +5868|AAAAAAAAMOGBAAAA|898|Walnut Walnut|Ct.|Suite 470|New Hope|Ector County|TX|79431|United States|-6|condo| +5869|AAAAAAAANOGBAAAA|809|Cherry |Boulevard|Suite I|Shiloh|Frederick County|MD|29275|United States|-5|single family| +5870|AAAAAAAAOOGBAAAA|215|Maple |ST|Suite S|Valley View|Brown County|KS|65124|United States|-6|apartment| +5871|AAAAAAAAPOGBAAAA|608|Center Spring|Road|Suite P|Jamestown|Sunflower County|MS|56867|United States|-6|single family| +5872|AAAAAAAAAPGBAAAA|880|Madison |Lane|Suite 40|Newport|Reno County|KS|61521|United States|-6|condo| +5873|AAAAAAAABPGBAAAA|673|Poplar |Blvd|Suite N|Buena Vista|Allegany County|NY|15752|United States|-5|condo| +5874|AAAAAAAACPGBAAAA|97|Hill View|Way|Suite 270|Pleasant Valley|Kankakee County|IL|62477|United States|-6|condo| +5875|AAAAAAAADPGBAAAA|4|Lakeview |Road|Suite 200|Forest Hills|Maricopa County|AZ|89237|United States|-7|single family| +5876|AAAAAAAAEPGBAAAA|69|Cherry Cedar|Ct.|Suite 110|Unionville|Madison County|IA|51711|United States|-6|condo| +5877|AAAAAAAAFPGBAAAA|111|Sycamore |Cir.|Suite 460|Oakwood|Morrow County|OR|90169|United States|-8|single family| +5878|AAAAAAAAGPGBAAAA|169|12th |ST|Suite P|White Oak|Northumberland County|VA|26668|United States|-5|condo| +5879|AAAAAAAAHPGBAAAA|892|Hill Highland|Blvd|Suite U|Lee|Custer County|MT|60408|United States|-7|condo| +5880|AAAAAAAAIPGBAAAA|253|Lincoln Mill|RD|Suite L||Hendricks County|||||| +5881|AAAAAAAAJPGBAAAA|634|Hill |Drive|Suite 310|Enterprise|Houston County|TX|71757|United States|-6|condo| +5882|AAAAAAAAKPGBAAAA|372|Lincoln Hill|RD|Suite 160|Pleasant Valley|Modoc County|CA|92477|United States|-8|single family| +5883|AAAAAAAALPGBAAAA|775|Valley |Street|Suite 260|Five Forks|Ouray County|CO|82293|United States|-7|condo| +5884|AAAAAAAAMPGBAAAA||||||Wasatch County||80191||-7|| +5885|AAAAAAAANPGBAAAA|819|Lakeview |Drive|Suite E|Lakeview|Yates County|NY|18579|United States|-5|condo| +5886|AAAAAAAAOPGBAAAA|766|Dogwood Cedar|Wy|Suite 100|Sulphur Springs|Grant County|IN|48354|United States|-5|apartment| +5887|AAAAAAAAPPGBAAAA|349|7th Park|Ave|Suite B|Belmont|Perry County|TN|30191|United States|-6|apartment| +5888|AAAAAAAAAAHBAAAA|671|Broadway Maple|Ave|Suite 380|Edgewood|Tucker County|WV|20069|United States|-5|condo| +5889|AAAAAAAABAHBAAAA|657|7th |Parkway|Suite 320|Five Forks|Sevier County|UT|82293|United States|-7|single family| +5890|AAAAAAAACAHBAAAA|334|River |Parkway|Suite S|Springfield|Onslow County|NC|29303|United States|-5|single family| +5891|AAAAAAAADAHBAAAA|329|||Suite G|||GA||United States||| +5892|AAAAAAAAEAHBAAAA|995|Forest Lake|Road|Suite Q|Clinton|Phelps County|NE|68222|United States|-7|single family| +5893|AAAAAAAAFAHBAAAA|200|12th 12th|Pkwy|Suite 240|New Hope|Garfield County|CO|89431|United States|-7|single family| +5894|AAAAAAAAGAHBAAAA|825|Lake Lakeview|Avenue|Suite H|Highland|Brown County|OH|49454|United States|-5|apartment| +5895|AAAAAAAAHAHBAAAA|575|Broadway Maple|Dr.|Suite 430|Macedonia|Pamlico County|NC|21087|United States|-5|condo| +5896|AAAAAAAAIAHBAAAA|170|Jackson |ST|Suite A|Concord|Carson County|TX|74107|United States|-6|condo| +5897|AAAAAAAAJAHBAAAA|654|Sycamore Ninth|Street|Suite 300|Mount Zion|Grant Parish|LA|78054|United States|-6|apartment| +5898|AAAAAAAAKAHBAAAA|655|Poplar 3rd|Ct.|Suite C|Newtown|Fayette County|TN|31749|United States|-5|apartment| +5899|AAAAAAAALAHBAAAA|624|Woodland North|ST|||Sacramento County|CA||||| +5900|AAAAAAAAMAHBAAAA|23|Highland |ST|Suite 150|Farmersville|Goochland County|VA|29305|United States|-5|apartment| +5901|AAAAAAAANAHBAAAA|628|Willow |Avenue|Suite S|Red Hill|Copiah County|MS|54338|United States|-6|single family| +5902|AAAAAAAAOAHBAAAA|783|6th Laurel|Circle|Suite 420|Lebanon|Haakon County|SD|52898|United States|-7|apartment| +5903|AAAAAAAAPAHBAAAA||||||||86614||-8|condo| +5904|AAAAAAAAABHBAAAA|834|View Hillcrest|Wy|Suite X|Greenwood|Montgomery County|KS|68828|United States|-6|single family| +5905|AAAAAAAABBHBAAAA|33|Woodland |Cir.|Suite T|Oak Hill|Wayne County|UT|87838|United States|-7|apartment| +5906|AAAAAAAACBHBAAAA|473|Hill |Ln|Suite 230|Brooklyn|Holt County|NE|61103|United States|-7|apartment| +5907|AAAAAAAADBHBAAAA|152|6th Third|Drive|Suite 340|Walnut Grove|Henry County|VA|27752|United States|-5|apartment| +5908|AAAAAAAAEBHBAAAA|972|Spruce Cherry|Cir.|Suite 140|Bunker Hill|Sully County|SD|50150|United States|-7|condo| +5909|AAAAAAAAFBHBAAAA|906|Main 5th|RD|Suite W|Bridgeport|Northampton County|NC|25817|United States|-5|apartment| +5910|AAAAAAAAGBHBAAAA|578|8th |Boulevard|Suite 110|Liberty|Kane County|UT|83451|United States|-7|single family| +5911|AAAAAAAAHBHBAAAA|71|Williams 4th|Dr.|Suite S|Hillcrest|Mathews County|VA|23003|United States|-5|single family| +5912|AAAAAAAAIBHBAAAA|392|Mill ||Suite I|Oakwood|Pike County|GA||United States||single family| +5913|AAAAAAAAJBHBAAAA|930|4th |Court|Suite 350|Springdale|Lumpkin County|GA|38883|United States|-5|apartment| +5914|AAAAAAAAKBHBAAAA|316|1st |Ave|Suite 150|White Oak|Guilford County|NC|26668|United States|-5|condo| +5915|AAAAAAAALBHBAAAA|364|Meadow |Wy|Suite V|Greenville|Clay County|AR|71387|United States|-6|apartment| +5916|AAAAAAAAMBHBAAAA|999|Washington |Way|Suite 300|Howell|Dale County|AL|34854|United States|-6|apartment| +5917|AAAAAAAANBHBAAAA|147|8th 13th|RD|Suite Y|Farmington|Lincoln County|WV|29145|United States|-5|apartment| +5918|AAAAAAAAOBHBAAAA|467|Cherry |Cir.|Suite 140|Brownsville|Grant County|KS|69310|United States|-6|condo| +5919|AAAAAAAAPBHBAAAA|741|Park Chestnut|Ct.|Suite U|Newtown|Holmes County|MS|51749|United States|-6|single family| +5920|AAAAAAAAACHBAAAA|157|Hillcrest Lincoln|Circle|Suite R|Pineville|Hunterdon County|NJ|09951|United States|-5|single family| +5921|AAAAAAAABCHBAAAA|980|Madison Twelfth|Lane|Suite O|Macedonia|Sullivan County|PA|11087|United States|-5|single family| +5922|AAAAAAAACCHBAAAA|947|Maple Lake|Dr.|Suite 70|Bridgeport|Tyler County|TX|75817|United States|-6|condo| +5923|AAAAAAAADCHBAAAA|420|Ninth |Ln|Suite Y|Woodlawn|Pike County|PA|14098|United States|-5|apartment| +5924|AAAAAAAAECHBAAAA|73|Railroad |Boulevard|Suite N|Mount Vernon|Wilcox County|AL|38482|United States|-6|single family| +5925|AAAAAAAAFCHBAAAA|243|Fourth 5th|Pkwy|Suite C|Newtown|Fairfield County|OH|41749|United States|-5|single family| +5926|AAAAAAAAGCHBAAAA|519|Chestnut |Blvd|Suite 160|Mount Olive|Lincoln County|MS|58059|United States|-6|condo| +5927|AAAAAAAAHCHBAAAA|499|Lee 13th|Cir.|Suite W|Oakland|Sedgwick County|KS|69843|United States|-6|apartment| +5928|AAAAAAAAICHBAAAA|338|Fourth |ST|Suite L|Unionville|Osceola County|IA|51711|United States|-6|single family| +5929|AAAAAAAAJCHBAAAA|72|Central Laurel|Parkway|Suite 410|Lakeside|Peach County|GA|39532|United States|-5|single family| +5930|AAAAAAAAKCHBAAAA|232|1st |Ln|Suite Y|Bethel|Boone County|MO|65281|United States|-6|single family| +5931|AAAAAAAALCHBAAAA|775|Fifth Cedar|Avenue|Suite K|Florence|Warren County|TN|33394|United States|-6|single family| +5932|AAAAAAAAMCHBAAAA|566|Forest |Boulevard|Suite S|Lincoln|Crawford County|MI|41289|United States|-5|apartment| +5933|AAAAAAAANCHBAAAA|854|Birch |Pkwy|Suite 400|Jackson|Mahaska County|IA|59583|United States|-6|apartment| +5934|AAAAAAAAOCHBAAAA|564|Spring |Road|Suite Y|Mount Olive|Curry County|NM|88059|United States|-7|apartment| +5935|AAAAAAAAPCHBAAAA|70|Park |Way|Suite F|Maple Grove|Union County|GA|38252|United States|-5|apartment| +5936|AAAAAAAAADHBAAAA|827|Central |Wy|Suite 170|Hamilton|Chatham County|GA|32808|United States|-5|condo| +5937|AAAAAAAABDHBAAAA|852|Wilson 4th|Street|Suite 270|Woodlawn|Caroline County|MD|24098|United States|-5|condo| +5938|AAAAAAAACDHBAAAA|986|Park |Ct.|Suite 50|Wright|Washington County|WI|52814|United States|-6|condo| +5939|AAAAAAAADDHBAAAA|245|Sycamore Forest|RD|Suite J|Spring Valley|Union County|KY|46060|United States|-5|condo| +5940|AAAAAAAAEDHBAAAA|932|Elm Wilson|Circle|Suite B|Oakwood|Twin Falls County|ID|80169|United States|-7|condo| +5941|AAAAAAAAFDHBAAAA|601|5th Cedar|Avenue|Suite 130|Vienna|Jenkins County|GA|35119|United States|-5|single family| +5942|AAAAAAAAGDHBAAAA|718|Elm 2nd|Drive|Suite H|Shiloh|Kittson County|MN|59275|United States|-6|single family| +5943|AAAAAAAAHDHBAAAA|398|Sunset |Drive|Suite 260|Jackson|Appanoose County|IA|59583|United States|-6|single family| +5944|AAAAAAAAIDHBAAAA|585|Church |RD|Suite 230|Forest Hills|Northwest Arctic Borough|AK|99237|United States|-9|apartment| +5945|AAAAAAAAJDHBAAAA|681|1st 9th|Cir.|Suite D|Woodlawn|Stone County|MO|64098|United States|-6|apartment| +5946|AAAAAAAAKDHBAAAA|440|Jefferson Jackson|Pkwy|Suite I|Oak Hill|Powell County|KY|47838|United States|-5|condo| +5947|AAAAAAAALDHBAAAA|660|Johnson |Road|Suite E|Jamestown|Garvin County|OK|76867|United States|-6|condo| +5948|AAAAAAAAMDHBAAAA|823|Sycamore 4th|Dr.|Suite 250|Hopewell|Hamlin County|SD|50587|United States|-7|single family| +5949|AAAAAAAANDHBAAAA|560|2nd |Blvd|Suite 320|Bethel|Cuyahoga County|OH|45281|United States|-5|single family| +5950|AAAAAAAAODHBAAAA|||Court|Suite V|Jamestown|Carroll County|||United States|-6|| +5951|AAAAAAAAPDHBAAAA|589|Third |Ln|Suite Q|Fairfield|Eagle County|CO|86192|United States|-7|apartment| +5952|AAAAAAAAAEHBAAAA|676|15th 12th|Parkway|Suite 110|Shiloh|Jefferson County|KY|49275|United States|-6|single family| +5953|AAAAAAAABEHBAAAA|726|Ash Ash|Blvd|Suite 420|Arlington|Stone County|MS|56557|United States|-6|apartment| +5954|AAAAAAAACEHBAAAA|251|Ninth Maple|Cir.|Suite H|Summit|Yuma County|AZ|80499|United States|-7|condo| +5955|AAAAAAAADEHBAAAA|822|Williams |||||||United States||apartment| +5956|AAAAAAAAEEHBAAAA|102|Fourth |Pkwy|Suite C|Union Hill|Monroe County|WI|57746|United States|-6|single family| +5957|AAAAAAAAFEHBAAAA|685|Birch Railroad|RD|Suite E|Rock Springs|Macon County|GA|37018|United States|-5|single family| +5958|AAAAAAAAGEHBAAAA|835|Sunset |Ct.|Suite F|Crossroads|Clinton County|MI|40534|United States|-5|apartment| +5959|AAAAAAAAHEHBAAAA|453|Church |Cir.|Suite S|Bunker Hill|Lewis County|WV|20150|United States|-5|condo| +5960|AAAAAAAAIEHBAAAA|891|Locust |Cir.|Suite I|Enterprise|Llano County|TX|71757|United States|-6|apartment| +5961|AAAAAAAAJEHBAAAA|313|Lincoln Oak|Wy|Suite 120|Green Acres|Wayne County|IA|57683|United States|-6|single family| +5962|AAAAAAAAKEHBAAAA|482|Highland |Pkwy|Suite 120|Lakewood|Wood County|TX|78877|United States|-6|single family| +5963|AAAAAAAALEHBAAAA|844|Lake West|Ave|Suite M|Lebanon|Madison County|TX|72898|United States|-6|condo| +5964|AAAAAAAAMEHBAAAA||9th Miller||Suite I||Yakima County|WA|95709|||apartment| +5965|AAAAAAAANEHBAAAA|306|Fourth |Avenue|Suite H|Union|Grayson County|VA|28721|United States|-5|condo| +5966|AAAAAAAAOEHBAAAA|509|Cherry |Ct.|Suite 110|Waterloo|McKenzie County|ND|51675|United States|-6|apartment| +5967|AAAAAAAAPEHBAAAA|867|Main |ST|Suite 340|Belmont|Sheridan County|ND|50191|United States|-6|condo| +5968|AAAAAAAAAFHBAAAA|210|1st |Way|Suite 190|Lebanon|Giles County|VA|22898|United States|-5|apartment| +5969|AAAAAAAABFHBAAAA|745|Church Smith|RD|Suite 340|Union|Floyd County|VA|28721|United States|-5|single family| +5970|AAAAAAAACFHBAAAA|241|Meadow Fifth|Drive|Suite 20|Oakwood|Sacramento County|CA|90169|United States|-8|apartment| +5971|AAAAAAAADFHBAAAA|807|View |Ln|Suite 360|Shiloh|Echols County|GA|39275|United States|-5|single family| +5972|AAAAAAAAEFHBAAAA|357|3rd |Cir.|Suite W|Fairview|Shiawassee County|MI|45709|United States|-5|apartment| +5973|AAAAAAAAFFHBAAAA||Third ||Suite 490||||80534||-7|single family| +5974|AAAAAAAAGFHBAAAA|438|River |Pkwy|Suite 140|Floyd|Clay County|NE|63235|United States|-6|condo| +5975|AAAAAAAAHFHBAAAA|690|2nd 3rd|Ct.|Suite Y|Cedar Grove|Martin County|KY|40411|United States|-5|apartment| +5976|AAAAAAAAIFHBAAAA|919|1st |RD|Suite Y|Harmony|Tallapoosa County|AL|35804|United States|-6|single family| +5977|AAAAAAAAJFHBAAAA|784|Madison |Ct.|Suite 370|Lakewood|Spotsylvania County|VA|28877|United States|-5|apartment| +5978|AAAAAAAAKFHBAAAA|518|Park |ST|Suite V|Macedonia|Noble County|OK|71087|United States|-6|condo| +5979|AAAAAAAALFHBAAAA|529|Elm |Cir.|Suite 180|Union Hill|Allegheny County|PA|17746|United States|-5|single family| +5980|AAAAAAAAMFHBAAAA|692|5th |Court|Suite 130|Edgewood|Bartholomew County|IN|40069|United States|-5|apartment| +5981|AAAAAAAANFHBAAAA|125|6th |Drive|Suite 290|Blanchard|Elko County|NV|85985|United States|-8|apartment| +5982|AAAAAAAAOFHBAAAA|65|5th |Ct.|Suite U|Hamilton|Fulton County|IL|62808|United States|-6|apartment| +5983|AAAAAAAAPFHBAAAA|280|Park Valley|Drive|Suite Q|Bethel|Montgomery County|IL|65281|United States|-6|condo| +5984|AAAAAAAAAGHBAAAA|987|Central |Street|Suite W|Plainview|Mercer County|NJ|04283|United States|-5|single family| +5985|AAAAAAAABGHBAAAA|724|Meadow 1st|Avenue|Suite 110|Wildwood|Gulf County|FL|36871|United States|-5|condo| +5986|AAAAAAAACGHBAAAA|847|East |Ave|Suite 80|Red Hill|Dearborn County|IN|44338|United States|-5|single family| +5987|AAAAAAAADGHBAAAA|613|5th |Way|Suite Q|Pine Grove|San Jacinto County|TX|74593|United States|-6|condo| +5988|AAAAAAAAEGHBAAAA|101|North |Ct.|Suite R|Lebanon|Wayne County|OH|42898|United States|-5|condo| +5989|AAAAAAAAFGHBAAAA|855|View |Ave|Suite 50|Lakewood|Washington County|AL|38877|United States|-6|condo| +5990|AAAAAAAAGGHBAAAA|443|Forest |Avenue|Suite M|Riceville|Menominee County|WI|55867|United States|-6|apartment| +5991|AAAAAAAAHGHBAAAA|703|7th Cedar|Court|Suite K|Hamilton|Logan County|ND|52808|United States|-6|condo| +5992|AAAAAAAAIGHBAAAA|||Street|Suite R|||MS|52477|||| +5993|AAAAAAAAJGHBAAAA|793|Laurel |Avenue|Suite 120|Wilson|Lackawanna County|PA|16971|United States|-5|apartment| +5994|AAAAAAAAKGHBAAAA|647|Sunset |Parkway|Suite H|Antioch|Woodson County|KS|68605|United States|-6|apartment| +5995|AAAAAAAALGHBAAAA|832|9th Mill|Cir.|Suite P|Valley View|Walton County|GA|35124|United States|-5|single family| +5996|AAAAAAAAMGHBAAAA|749|East |Way|Suite R|Crossroads|Allen Parish|LA|70534|United States|-6|condo| +5997|AAAAAAAANGHBAAAA|872|Birch Hill|Pkwy|Suite 420|Highland Park|Gaston County|NC|26534|United States|-5|condo| +5998|AAAAAAAAOGHBAAAA|245|1st 6th|Cir.|Suite 230|Providence|Rolette County|ND|56614|United States|-6|single family| +5999|AAAAAAAAPGHBAAAA|661|Cherry |Pkwy|Suite D|Page|Oneida County|ID|80296|United States|-7|single family| +6000|AAAAAAAAAHHBAAAA|670|View |Boulevard|Suite U|Enterprise|Laramie County|WY|81757|United States|-7|apartment| +6001|AAAAAAAABHHBAAAA|483|Cherry West|Parkway|Suite K|Harmony|San Augustine County|TX|75804|United States|-6|single family| +6002|AAAAAAAACHHBAAAA|809|Washington Dogwood||Suite Q|Wildwood|Orange County||76871|||single family| +6003|AAAAAAAADHHBAAAA|860|Woodland Central|Ct.|Suite 240|Brownsville|Kings County|NY|19310|United States|-5|condo| +6004|AAAAAAAAEHHBAAAA|601|5th |Ln|Suite 190|Mount Vernon|Amite County|MS|58482|United States|-6|single family| +6005|AAAAAAAAFHHBAAAA|397|Wilson |Dr.|Suite F|Farmersville|Coweta County|GA|39305|United States|-5|single family| +6006|AAAAAAAAGHHBAAAA|907|Franklin |RD|Suite 330|Friendship|Otero County|CO|84536|United States|-7|apartment| +6007|AAAAAAAAHHHBAAAA|984|First |Road|Suite 310|Riverside|Yell County|AR|79231|United States|-6|apartment| +6008|AAAAAAAAIHHBAAAA|718|Lake Elevnth|Avenue|Suite 130|Riverside|Gloucester County|VA|29231|United States|-5|apartment| +6009|AAAAAAAAJHHBAAAA||3rd |Blvd|Suite H||Johnson County|AR|73951|||| +6010|AAAAAAAAKHHBAAAA||Maple ||Suite 430|||PA|||-5|single family| +6011|AAAAAAAALHHBAAAA|240|Washington |Lane|Suite J|Burns|Frontier County|NE|65272|United States|-6|single family| +6012|AAAAAAAAMHHBAAAA|29|1st Mill|Circle|Suite Q|Cedar Grove|Douglas County|NV|80411|United States|-8|condo| +6013|AAAAAAAANHHBAAAA|947|Third Green|Court|Suite 490|Hopewell|Monongalia County|WV|20587|United States|-5|apartment| +6014|AAAAAAAAOHHBAAAA|562|Third 11th|Dr.|Suite B|Ryan|Texas County|OK|70525|United States|-6|single family| +6015|AAAAAAAAPHHBAAAA|922|Ninth |Circle|Suite 80|Riverside|Beaufort County|SC|29231|United States|-5|apartment| +6016|AAAAAAAAAIHBAAAA|598|Railroad |ST|Suite O|Mountain View|Okanogan County|WA|94466|United States|-8|condo| +6017|AAAAAAAABIHBAAAA|815|Lincoln |RD|Suite 190|Plainview|Boone County|IN|43683|United States|-5|apartment| +6018|AAAAAAAACIHBAAAA|984|Park Sycamore|Blvd|Suite 160|Elizabeth|Lee County|GA|32935|United States|-5|apartment| +6019|AAAAAAAADIHBAAAA|||Ln|Suite 320||Menominee County|WI|||-6|single family| +6020|AAAAAAAAEIHBAAAA|781|Park Lincoln|Ct.|Suite 470|Buena Vista|Uvalde County|TX|75752|United States|-6|apartment| +6021|AAAAAAAAFIHBAAAA|623|View Second|Dr.|Suite 390|New Hope|Coryell County|TX|79431|United States|-6|single family| +6022|AAAAAAAAGIHBAAAA|933|Pine Maple|RD|Suite 270|Guthrie|Maries County|MO|61423|United States|-6|condo| +6023|AAAAAAAAHIHBAAAA|564|Johnson |Blvd|Suite 20|Elm Grove|Jasper County|TX|73298|United States|-6|single family| +6024|AAAAAAAAIIHBAAAA|242|Lake West|Circle|Suite P|Fairfield|Pulaski County|IN|46192|United States|-5|apartment| +6025|AAAAAAAAJIHBAAAA|936|Ridge Fourth|Road|Suite N|Harmony|Roosevelt County|NM|85804|United States|-7|apartment| +6026|AAAAAAAAKIHBAAAA|911|7th |Parkway|Suite S|Highland|Chesapeake city|VA|29454|United States|-5|condo| +6027|AAAAAAAALIHBAAAA|596|Mill |Wy|Suite 380|Stringtown|Ben Hill County|GA|30162|United States|-5|apartment| +6028|AAAAAAAAMIHBAAAA|532|Fifth View|Wy|Suite M|Forest Hills|Lincoln County|WI|59237|United States|-6|condo| +6029|AAAAAAAANIHBAAAA|126|Hill |Blvd|Suite C|Fairfield|Shelby County|TN|36192|United States|-6|single family| +6030|AAAAAAAAOIHBAAAA|465|River Lake|Cir.|Suite F|Sulphur Springs|Mayes County|OK|78354|United States|-6|single family| +6031|AAAAAAAAPIHBAAAA|225|First West|Parkway|Suite 130|Glendale|Hardeman County|TX|73951|United States|-6|apartment| +6032|AAAAAAAAAJHBAAAA|736|Lake |Court|Suite 400|Belmont|Cumberland County|VA|20191|United States|-5|single family| +6033|AAAAAAAABJHBAAAA|993|Birch |Pkwy|Suite 330|Mountain View|Fresno County|CA|94466|United States|-8|condo| +6034|AAAAAAAACJHBAAAA|803|Sycamore |Ave|Suite C|Woodville|Vernon County|MO|64289|United States|-6|condo| +6035|AAAAAAAADJHBAAAA|213|6th |RD|Suite 160|Bridgeport|Mills County|TX|75817|United States|-6|apartment| +6036|AAAAAAAAEJHBAAAA|924|Oak Forest|Pkwy|Suite I|Lincoln|Olmsted County|MN|51289|United States|-6|single family| +6037|AAAAAAAAFJHBAAAA|353|Dogwood East|Road|Suite W|Denmark|Dare County|NC|25576|United States|-5|single family| +6038|AAAAAAAAGJHBAAAA|452|Cedar |Ave|Suite 460|Mount Zion|Travis County|TX|78054|United States|-6|apartment| +6039|AAAAAAAAHJHBAAAA|833|South North|Ln|Suite 450|Fairfield|Grady County|GA|36192|United States|-5|apartment| +6040|AAAAAAAAIJHBAAAA|894|Sixth |Court|Suite S|Five Points|Clearwater County|MN|56098|United States|-6|condo| +6041|AAAAAAAAJJHBAAAA|||||Newtown|Barton County|KS|||-6|| +6042|AAAAAAAAKJHBAAAA|953|Park |Pkwy|Suite 10|White Oak|Tippecanoe County|IN|46668|United States|-5|apartment| +6043|AAAAAAAALJHBAAAA|930|Highland South|Boulevard|Suite Y|Woodville|Monroe County|MO|64289|United States|-6|single family| +6044|AAAAAAAAMJHBAAAA|84|7th Green|Parkway|Suite 420|Riverdale|Grayson County|TX|79391|United States|-6|condo| +6045|AAAAAAAANJHBAAAA|119|Spring |ST|Suite 130|Pleasant Valley|Columbus County|NC|22477|United States|-5|condo| +6046|AAAAAAAAOJHBAAAA|662|Jefferson |ST|Suite 220|Bloomingdale|Pondera County|MT|61824|United States|-7|single family| +6047|AAAAAAAAPJHBAAAA|387|4th |Wy|Suite 150|Spring Valley|Cibola County|NM|86060|United States|-7|single family| +6048|AAAAAAAAAKHBAAAA|676|Sunset |Parkway|Suite D|Shore Acres|Pasquotank County|NC|22724|United States|-5|apartment| +6049|AAAAAAAABKHBAAAA|122|Jackson |Pkwy|Suite 410|Oneida|Twin Falls County|ID|84027|United States|-7|condo| +6050|AAAAAAAACKHBAAAA|209|Miller |ST|Suite 330|Jackson|Kendall County|IL|69583|United States|-6|condo| +6051|AAAAAAAADKHBAAAA|699|Park Williams|Lane|Suite 30|Wildwood|Baltimore County|MD|26871|United States|-5|single family| +6052|AAAAAAAAEKHBAAAA|494|15th |ST|Suite 230|Union|Gates County|NC|28721|United States|-5|single family| +6053|AAAAAAAAFKHBAAAA|52|Williams Lake|Circle|Suite 410|Macedonia|Las Animas County|CO|81087|United States|-7|condo| +6054|AAAAAAAAGKHBAAAA||Broadway |||Lakeside||OH||United States||single family| +6055|AAAAAAAAHKHBAAAA|902|Miller Green|Road|Suite V|Mountain View|Alleghany County|NC|24466|United States|-5|condo| +6056|AAAAAAAAIKHBAAAA|533|Maple |Drive|Suite 340|Shady Grove|Itasca County|MN|52812|United States|-6|single family| +6057|AAAAAAAAJKHBAAAA|533||Ln|||Jefferson County|||||apartment| +6058|AAAAAAAAKKHBAAAA|235|Park |Dr.|Suite X|Pleasant Hill|Bland County|VA|23604|United States|-5|apartment| +6059|AAAAAAAALKHBAAAA|594|View Park|RD|Suite W|Riverdale|Lac qui Parle County|MN|59391|United States|-6|apartment| +6060|AAAAAAAAMKHBAAAA|352|Fifth Cedar|Avenue|Suite L|Midway|Jackson County|MI|41904|United States|-5|condo| +6061|AAAAAAAANKHBAAAA|923|Sunset |Cir.|Suite K|Walnut Grove|Fayette County|IA|57752|United States|-6|apartment| +6062|AAAAAAAAOKHBAAAA|338|Mill |Parkway|Suite 440|Harmony|Payne County|OK|75804|United States|-6|apartment| +6063|AAAAAAAAPKHBAAAA|838|Johnson |Drive|Suite T|New Hope|Flathead County|MT|69431|United States|-7|apartment| +6064|AAAAAAAAALHBAAAA|298|Ridge |Street|Suite 220|Deerfield|Henderson County|KY|49840|United States|-6|condo| +6065|AAAAAAAABLHBAAAA|543|North |Ln|Suite A|Arlington|Washington County|AR|76557|United States|-6|condo| +6066|AAAAAAAACLHBAAAA|462|Lincoln 1st|Wy|Suite X|Crossroads|Canyon County|ID|80534|United States|-7|condo| +6067|AAAAAAAADLHBAAAA|98|South 1st|Court|Suite 230|Shady Grove|Summit County|OH|42812|United States|-5|apartment| +6068|AAAAAAAAELHBAAAA|772|3rd Washington|Cir.|Suite Y|White Oak|Nassau County|FL|36668|United States|-5|condo| +6069|AAAAAAAAFLHBAAAA|874|Cedar |Way|Suite 320|Spring Hill|Andrew County|MO|66787|United States|-6|condo| +6070|AAAAAAAAGLHBAAAA|449|Adams Johnson|Dr.|Suite 460|Farmington|Litchfield County|CT|09745|United States|-5|apartment| +6071|AAAAAAAAHLHBAAAA|800|Hickory |Circle|Suite P|Riverdale|Pocahontas County|WV|29391|United States|-5|single family| +6072|AAAAAAAAILHBAAAA|699|Main Walnut|Pkwy|Suite P|Salem|Marshall County|KY|48048|United States|-5|condo| +6073|AAAAAAAAJLHBAAAA|981|First Poplar|Drive|Suite 230|Oak Hill|Nelson County|ND|57838|United States|-6|apartment| +6074|AAAAAAAAKLHBAAAA|279|Woodland |Boulevard|Suite W|Mount Vernon|Washington County|VA|28482|United States|-5|condo| +6075|AAAAAAAALLHBAAAA|632|Ash South|Ave|Suite O|Riverview|Greensville County|VA|29003|United States|-5|single family| +6076|AAAAAAAAMLHBAAAA|814|Maple |Court|Suite 410|Milan|Brooks County|GA|36697|United States|-5|condo| +6077|AAAAAAAANLHBAAAA|855|Woodland |Wy|Suite D|Highland|Chippewa County|WI|59454|United States|-6|condo| +6078|AAAAAAAAOLHBAAAA|776|2nd Second|ST|Suite O|Woodville|Ben Hill County|GA|34289|United States|-5|single family| +6079|AAAAAAAAPLHBAAAA|881|Williams Davis|Avenue|Suite L|Springdale|Lapeer County|MI|48883|United States|-5|condo| +6080|AAAAAAAAAMHBAAAA|28|8th Park|RD|Suite 360|Florence|Fairfield County|OH|43394|United States|-5|single family| +6081|AAAAAAAABMHBAAAA|210|1st |Parkway|Suite L|Edgewood|Pawnee County|KS|60069|United States|-6|condo| +6082|AAAAAAAACMHBAAAA||||Suite 490|Five Points|Howell County|MO||||single family| +6083|AAAAAAAADMHBAAAA|161|6th |Street|Suite G|Tanglewood|Pickett County|TN|38994|United States|-6|apartment| +6084|AAAAAAAAEMHBAAAA|873|Forest |ST|Suite 170|Oak Hill|Harnett County|NC|27838|United States|-5|single family| +6085|AAAAAAAAFMHBAAAA|||Boulevard|||Cloud County|KS|68059||-6|| +6086|AAAAAAAAGMHBAAAA|152|Third |Road|Suite 230|Enterprise|El Dorado County|CA|91757|United States|-8|condo| +6087|AAAAAAAAHMHBAAAA|641|Hickory |Way|Suite L|Hunter|Ellsworth County|KS|64615|United States|-6|condo| +6088|AAAAAAAAIMHBAAAA|760|West |Boulevard|Suite C|Lakewood|Laurens County|SC|28877|United States|-5|condo| +6089|AAAAAAAAJMHBAAAA|252|Franklin |ST|Suite 90|Sulphur Springs|Idaho County|ID|88354|United States|-7|single family| +6090|AAAAAAAAKMHBAAAA|209|Locust Ridge|ST|Suite 410|Mountain View|Chaves County|NM|84466|United States|-7|apartment| +6091|AAAAAAAALMHBAAAA|420|Seventh |Lane|Suite 340|Antioch|Guthrie County|IA|58605|United States|-6|apartment| +6092|AAAAAAAAMMHBAAAA|607|Franklin Ridge|Street|Suite 200|Summit|San Bernardino County|CA|90499|United States|-8|apartment| +6093|AAAAAAAANMHBAAAA|45|Washington |Way|Suite J|Greenville|Decatur County|GA|31387|United States|-5|apartment| +6094|AAAAAAAAOMHBAAAA|959|East |Ct.|Suite 400|Unionville|Yamhill County|OR|91711|United States|-8|apartment| +6095|AAAAAAAAPMHBAAAA|137|Lakeview 4th|Lane|Suite R|Greenfield|Lake County|TN|35038|United States|-6|apartment| +6096|AAAAAAAAANHBAAAA|932|Maple |Avenue|Suite 300|Glendale|Butts County|GA|33951|United States|-5|single family| +6097|AAAAAAAABNHBAAAA|659|North Maple|RD|Suite A|Five Forks|Lee County|IA|52293|United States|-6|condo| +6098|AAAAAAAACNHBAAAA|40|9th 5th|Road|Suite T|Newtown|Tom Green County|TX|71749|United States|-6|single family| +6099|AAAAAAAADNHBAAAA|991|Adams Main|Wy|Suite 130|Avery|Sherman County|KS|60194|United States|-6|apartment| +6100|AAAAAAAAENHBAAAA|689|Sunset |Ct.|Suite 340|Elm Grove|Trigg County|KY|43298|United States|-5|apartment| +6101|AAAAAAAAFNHBAAAA|939||ST||||KY||United States|-6|condo| +6102|AAAAAAAAGNHBAAAA|908|Hillcrest |RD|Suite P|Crossroads|Charlotte County|FL|30534|United States|-5|single family| +6103|AAAAAAAAHNHBAAAA|609|Maple Birch|Court|Suite D|Sunnyside|Benton County|MS|51952|United States|-6|single family| +6104|AAAAAAAAINHBAAAA|674|Lincoln |Dr.|Suite 20|Lebanon|Calhoun County|GA|32898|United States|-5|single family| +6105|AAAAAAAAJNHBAAAA|141|1st |Court|Suite N|Midway|Granville County|NC|21904|United States|-5|single family| +6106|AAAAAAAAKNHBAAAA|895|Locust Second|ST|Suite 70|Franklin|Lewis and Clark County|MT|69101|United States|-7|condo| +6107|AAAAAAAALNHBAAAA|895|Park |Street|Suite 110|Belmont|Decatur County|KS|60191|United States|-6|single family| +6108|AAAAAAAAMNHBAAAA|920|Oak |ST|Suite 380|Belmont|Fort Bend County|TX|70191|United States|-6|apartment| +6109|AAAAAAAANNHBAAAA|146|Hill |Dr.|Suite 50|Colonial Heights|Logan County|AR|73425|United States|-6|apartment| +6110|AAAAAAAAONHBAAAA|491|Poplar |Ct.|Suite Q|Lebanon|Marshall County|KY|42898|United States|-5|apartment| +6111|AAAAAAAAPNHBAAAA|9|12th 1st|Cir.|Suite J|Harmony|Hancock County|IN|45804|United States|-5|condo| +6112|AAAAAAAAAOHBAAAA|766|Madison |Ave|Suite L|Ashland|Wallace County|KS|64244|United States|-6|single family| +6113|AAAAAAAABOHBAAAA|738|Cedar |Ct.|Suite 280|White Plains|DeKalb County|AL|36622|United States|-6|condo| +6114|AAAAAAAACOHBAAAA|87|First Cherry|Wy|Suite O|Acme|Freestone County|TX|70164|United States|-6|apartment| +6115|AAAAAAAADOHBAAAA|491|Center Pine|Way|Suite T|Macedonia|Phillips County|AR|71087|United States|-6|condo| +6116|AAAAAAAAEOHBAAAA|815|Willow |Parkway|Suite E|New Hope|Dixon County|NE|69431|United States|-6|single family| +6117|AAAAAAAAFOHBAAAA|321|Meadow Johnson|ST|Suite 200|Bunker Hill|Polk County|TX|70150|United States|-6|single family| +6118|AAAAAAAAGOHBAAAA|804|4th |RD|Suite U|Oakdale|Ripley County|MO|69584|United States|-6|single family| +6119|AAAAAAAAHOHBAAAA|549|Hickory |Avenue|Suite 410|Jamestown|Gulf County|FL|36867|United States|-5|apartment| +6120|AAAAAAAAIOHBAAAA|757|Lee Mill|Boulevard|Suite 200|Fairbanks|Appling County|GA|36653|United States|-5|condo| +6121|AAAAAAAAJOHBAAAA|34|6th East|Parkway|Suite C|Valley View|Butts County|GA|35124|United States|-5|single family| +6122|AAAAAAAAKOHBAAAA|943|View Eigth|ST|Suite 140|Red Oak|Hughes County|SD|55018|United States|-7|apartment| +6123|AAAAAAAALOHBAAAA|579|6th |Wy|Suite X|Jamestown|Franklin County|IA|56867|United States|-6|condo| +6124|AAAAAAAAMOHBAAAA|16|Madison Third|Wy|Suite 430|Liberty|Effingham County|IL|63451|United States|-6|single family| +6125|AAAAAAAANOHBAAAA|246|Cherry |Lane|Suite 460|Edgewood|Clay County|KY|40069|United States|-6|single family| +6126|AAAAAAAAOOHBAAAA|102|Poplar |Ct.|Suite D|Mountain View|Ascension Parish|LA|74466|United States|-6|apartment| +6127|AAAAAAAAPOHBAAAA|543|Highland |Wy|Suite V|Buena Vista|Cass County|NE|65752|United States|-6|apartment| +6128|AAAAAAAAAPHBAAAA|696|North Chestnut|Way|Suite B|San Jose|Spalding County|GA|38003|United States|-5|condo| +6129|AAAAAAAABPHBAAAA|925|9th West|Lane|Suite 410|Lakewood|Greene County|TN|38877|United States|-5|condo| +6130|AAAAAAAACPHBAAAA|392|Hillcrest Central|RD|Suite B|Pleasant Valley|Cabell County|WV|22477|United States|-5|condo| +6131|AAAAAAAADPHBAAAA|210|Spring |Street|Suite 380|Lakeview|Hayes County|NE|68579|United States|-6|apartment| +6132|AAAAAAAAEPHBAAAA|132|8th Pine|Pkwy|Suite L|Oakwood|Pipestone County|MN|50169|United States|-6|single family| +6133|AAAAAAAAFPHBAAAA|374|5th Meadow|Court|Suite 330|Highland|Edmunds County|SD|59454|United States|-6|condo| +6134|AAAAAAAAGPHBAAAA|704|Cherry |Ave|Suite 310|Belfast|Matagorda County|TX|70125|United States|-6|single family| +6135|AAAAAAAAHPHBAAAA|579|Eigth 4th|Way|Suite P|Pleasant Grove|Custer County|MT|64136|United States|-7|condo| +6136|AAAAAAAAIPHBAAAA||River |Parkway|Suite G|Bridgeport||OK|75817|United States||| +6137|AAAAAAAAJPHBAAAA|930|5th |Way|Suite A|Riverside|Piatt County|IL|69231|United States|-6|condo| +6138|AAAAAAAAKPHBAAAA|938|10th Main|Boulevard|Suite 40|Spring Valley|Douglas County|NV|86060|United States|-8|condo| +6139|AAAAAAAALPHBAAAA|581|Cedar Main|ST|Suite Q|Pine Grove|Duval County|TX|74593|United States|-6|condo| +6140|AAAAAAAAMPHBAAAA|||Ave|||Tyler County|||||| +6141|AAAAAAAANPHBAAAA|94|Cherry 10th|Ln|Suite 390|Five Points|Peach County|GA|36098|United States|-5|single family| +6142|AAAAAAAAOPHBAAAA|231|9th |Road|Suite 440|Salem|Pulaski County|GA|38048|United States|-5|condo| +6143|AAAAAAAAPPHBAAAA|890|Forest Spring|Drive||||FL||United States||apartment| +6144|AAAAAAAAAAIBAAAA|404|7th |Street|Suite L|Oakland|Jasper County|GA|39843|United States|-5|condo| +6145|AAAAAAAABAIBAAAA|468|Wilson 4th|Parkway|Suite 440|Five Forks|Greene County|MS|52293|United States|-6|apartment| +6146|AAAAAAAACAIBAAAA|585|River |Drive|Suite 490|Oak Ridge|Middlesex County|VA|28371|United States|-5|condo| +6147|AAAAAAAADAIBAAAA|627|Hillcrest |Boulevard|Suite P|Belmont|Nelson County|ND|50191|United States|-6|apartment| +6148|AAAAAAAAEAIBAAAA|235|South |Lane|Suite 410|Woodville|Hemphill County|TX|74289|United States|-6|apartment| +6149|AAAAAAAAFAIBAAAA|227|Forest View|Circle|Suite U|Lakeview|Bee County|TX|78579|United States|-6|single family| +6150|AAAAAAAAGAIBAAAA|154|Hill Park|Blvd|Suite T|Oakland|Warren County|KY|49843|United States|-5|apartment| +6151|AAAAAAAAHAIBAAAA|991|Willow |Ln|Suite 340|Wilson|Kenedy County|TX|76971|United States|-6|apartment| +6152|AAAAAAAAIAIBAAAA|773|7th Franklin|Way|Suite 220|Centerville|Autauga County|AL|30059|United States|-6|apartment| +6153|AAAAAAAAJAIBAAAA|704|Valley Walnut|Ave|Suite N|Macedonia|Charlotte County|FL|31087|United States|-5|apartment| +6154|AAAAAAAAKAIBAAAA|998|Elm Oak|Parkway|Suite A|Five Forks|Marshall County|AL|32293|United States|-6|single family| +6155|AAAAAAAALAIBAAAA|577|Pine Park|Court|Suite 450|Arlington|Pepin County|WI|56557|United States|-6|single family| +6156|AAAAAAAAMAIBAAAA|917|View Park|Court|Suite 240|Waterloo|Dawson County|TX|71675|United States|-6|single family| +6157|AAAAAAAANAIBAAAA|114|West West|Boulevard|Suite 300|Arlington|Bastrop County|TX|76557|United States|-6|single family| +6158|AAAAAAAAOAIBAAAA|301|13th |Court|Suite 460|Crossroads|Graham County|AZ|80534|United States|-7|single family| +6159|AAAAAAAAPAIBAAAA|721|Eigth |Wy|Suite W|Farmington|Holmes County|FL|39145|United States|-5|condo| +6160|AAAAAAAAABIBAAAA|26|River |Drive|Suite H|Forestville|San Juan County|NM|83027|United States|-7|apartment| +6161|AAAAAAAABBIBAAAA|297|Ash East|Pkwy|Suite E|Jackson|Chaffee County|CO|89583|United States|-7|apartment| +6162|AAAAAAAACBIBAAAA|344|Smith Valley|ST|Suite B|Lakeside|Estill County|KY|49532|United States|-6|apartment| +6163|AAAAAAAADBIBAAAA|399|Adams |ST|Suite 340|Bethesda|Hale County|TX|75980|United States|-6|apartment| +6164|AAAAAAAAEBIBAAAA|682|Lee Wilson|Ct.|Suite 320|Mountain View|Colorado County|TX|74466|United States|-6|condo| +6165|AAAAAAAAFBIBAAAA|780|East 11th|Blvd|Suite 10|Lakewood|Bourbon County|KY|48877|United States|-6|apartment| +6166|AAAAAAAAGBIBAAAA|246|2nd Tenth|Wy|Suite 10|Wilson|Culberson County|TX|76971|United States|-6|single family| +6167|AAAAAAAAHBIBAAAA|187|Cherry |RD|Suite H|Lincoln|Carteret County|NC|21289|United States|-5|single family| +6168|AAAAAAAAIBIBAAAA|942|Lakeview 7th|Way|Suite M|Shady Grove|Orange County|NY|12812|United States|-5|apartment| +6169|AAAAAAAAJBIBAAAA|615|South Lake|Drive|Suite 250|Lakewood|Yancey County|NC|28877|United States|-5|apartment| +6170|AAAAAAAAKBIBAAAA|82|South |RD|Suite R|Riverdale|Albany County|WY|89391|United States|-7|apartment| +6171|AAAAAAAALBIBAAAA|698|Elm Lincoln|Ave|Suite L|Shady Grove|Perry County|TN|32812|United States|-6|apartment| +6172|AAAAAAAAMBIBAAAA|968|Woodland 4th|Boulevard|Suite Y|Greenfield|Albany County|NY|15038|United States|-5|condo| +6173|AAAAAAAANBIBAAAA|468|Second |Drive|Suite L|Price|Wyandot County|OH|43824|United States|-5|condo| +6174|AAAAAAAAOBIBAAAA|304|Madison |RD|Suite 230|Hamilton|Pendleton County|WV|22808|United States|-5|apartment| +6175|AAAAAAAAPBIBAAAA|492|View |Dr.|Suite 140|Ashland|Umatilla County|OR|94244|United States|-8|condo| +6176|AAAAAAAAACIBAAAA|249|Railroad Central|Blvd|Suite 480|Silver City|Caldwell County|KY|43405|United States|-6|apartment| +6177|AAAAAAAABCIBAAAA|346|Fifth |Wy|Suite 220|Cedar Grove|Alexander County|NC|20411|United States|-5|single family| +6178|AAAAAAAACCIBAAAA|981|Highland Fourth|Circle|Suite L|Plainview|Multnomah County|OR|93683|United States|-8|condo| +6179|AAAAAAAADCIBAAAA|426|5th |Way|Suite J|Mount Olive|Bannock County|ID|88059|United States|-7|apartment| +6180|AAAAAAAAECIBAAAA|598|Main Third|Cir.|Suite 70|Union|Grant County|SD|58721|United States|-6|single family| +6181|AAAAAAAAFCIBAAAA|796|4th |Pkwy|Suite 370|Marion|Ontario County|NY|10399|United States|-5|single family| +6182|AAAAAAAAGCIBAAAA|123|Center |Court|Suite T|Jamestown|Polk County|TN|36867|United States|-6|condo| +6183|AAAAAAAAHCIBAAAA|814|Pine Willow|Parkway|Suite L|Midway|Marion County|GA|31904|United States|-5|single family| +6184|AAAAAAAAICIBAAAA|732|2nd |Ct.|Suite 430|Greenfield|Heard County|GA|35038|United States|-5|apartment| +6185|AAAAAAAAJCIBAAAA|420|Forest Walnut|Drive|Suite X|Woodland|Curry County|OR|94854|United States|-8|apartment| +6186|AAAAAAAAKCIBAAAA|753|Willow Second|Way|Suite Y|Louisville|Thomas County|GA|34464|United States|-5|condo| +6187|AAAAAAAALCIBAAAA|143|Ridge |Dr.|Suite 160|Franklin|Camp County|TX|79101|United States|-6|apartment| +6188|AAAAAAAAMCIBAAAA|907|Davis Second|Wy|Suite Q|Bethel|Ashland County|WI|55281|United States|-6|condo| +6189|AAAAAAAANCIBAAAA|809|Maple South|Ave|Suite M|Riverview|Rappahannock County|VA|29003|United States|-5|single family| +6190|AAAAAAAAOCIBAAAA|875|Sunset 7th|Avenue|Suite Y|Lewisburg|Montgomery County|AL|37538|United States|-6|condo| +6191|AAAAAAAAPCIBAAAA|682|Cedar Willow|Court|Suite J|Friendship|Knox County|IN|44536|United States|-5|apartment| +6192|AAAAAAAAADIBAAAA|153|Hickory Elm|Parkway|Suite C|Jamestown|Jefferson County|AR|76867|United States|-6|apartment| +6193|AAAAAAAABDIBAAAA|478|Williams |Ave|Suite 40|Pleasant Valley|Wayne County|MO|62477|United States|-6|single family| +6194|AAAAAAAACDIBAAAA|534|Wilson Forest|Dr.|Suite M|Georgetown|Fulton County|KY|47057|United States|-6|single family| +6195|AAAAAAAADDIBAAAA|361|Pine |Ct.|Suite 140|Spring Hill|Yuba County|CA|96787|United States|-8|single family| +6196|AAAAAAAAEDIBAAAA|641|12th |Pkwy|Suite W|Deerfield|Ogle County|IL|69840|United States|-6|single family| +6197|AAAAAAAAFDIBAAAA|443|Lee Third|Parkway|Suite I|Oak Hill|Sarasota County|FL|37838|United States|-5|apartment| +6198|AAAAAAAAGDIBAAAA|858|Main |Circle|Suite 260|Greenwood|Oktibbeha County|MS|58828|United States|-6|apartment| +6199|AAAAAAAAHDIBAAAA|793|8th |Ct.|Suite 430|Wildwood|Nelson County|ND|56871|United States|-6|condo| +6200|AAAAAAAAIDIBAAAA|430|West Seventh|Ct.|Suite W|Wildwood|Grady County|GA|36871|United States|-5|single family| +6201|AAAAAAAAJDIBAAAA|736|Cedar Green|Blvd|Suite V|Williamsville|Washington County|NC|28754|United States|-5|apartment| +6202|AAAAAAAAKDIBAAAA|704|Maple |Court|Suite Y|Fairfield|Nodaway County|MO|66192|United States|-6|single family| +6203|AAAAAAAALDIBAAAA|247|5th |Pkwy|Suite 170|Marion|Wood County|TX|70399|United States|-6|single family| +6204|AAAAAAAAMDIBAAAA|271|Walnut |Pkwy|Suite 270|Valley View|Craighead County|AR|75124|United States|-6|apartment| +6205|AAAAAAAANDIBAAAA|633|1st |RD|Suite 30|Newtown|Isanti County|MN|51749|United States|-6|apartment| +6206|AAAAAAAAODIBAAAA|227|Chestnut Park|Parkway|Suite 10|Salem|Fayette County|PA|18048|United States|-5|condo| +6207|AAAAAAAAPDIBAAAA|112|Pine Lincoln|RD|Suite T|Brownsville|McHenry County|ND|59310|United States|-6|apartment| +6208|AAAAAAAAAEIBAAAA|285|Davis |Wy|Suite 250|Summit|Cheshire County|NH|01099|United States|-5|condo| +6209|AAAAAAAABEIBAAAA|756|2nd Johnson|Ct.|Suite N|White Oak|Burke County|ND|56668|United States|-6|condo| +6210|AAAAAAAACEIBAAAA|852|Park |ST|Suite D|Pine Grove|Hot Spring County|AR|74593|United States|-6|single family| +6211|AAAAAAAADEIBAAAA|759|Green |Street|Suite T|Valley View|Ashland County|WI|55124|United States|-6|apartment| +6212|AAAAAAAAEEIBAAAA|937|Johnson Adams|Avenue|Suite Y|Mount Zion|Paulding County|OH|48054|United States|-5|condo| +6213|AAAAAAAAFEIBAAAA|153|Miller |Way|Suite 260|Lakeview|Pike County|MO|68579|United States|-6|single family| +6214|AAAAAAAAGEIBAAAA|77|1st |Drive|Suite 110|Oak Grove|Emanuel County|GA|38370|United States|-5|single family| +6215|AAAAAAAAHEIBAAAA|953|Cedar Franklin|Road|Suite I|Woodland|Dixie County|FL|34854|United States|-5|condo| +6216|AAAAAAAAIEIBAAAA|743|Forest |RD|Suite 360|Phoenix|Jefferson County|FL|32276|United States|-5|apartment| +6217|AAAAAAAAJEIBAAAA|255|8th |Circle|Suite 380|Bridgeport|Alexander County|IL|65817|United States|-6|single family| +6218|AAAAAAAAKEIBAAAA|934|15th Oak|Ave|Suite N|Riverdale|Dade County|GA|39391|United States|-5|apartment| +6219|AAAAAAAALEIBAAAA|382|Pine |Court|Suite H|Highland|Sierra County|CA|99454|United States|-8|condo| +6220|AAAAAAAAMEIBAAAA|540|Woodland |Ln|Suite 240|Pleasant Valley|Lake of the Woods County|MN|52477|United States|-6|apartment| +6221|AAAAAAAANEIBAAAA|981|Chestnut |RD|Suite 490|Tipton|Merced County|CA|98773|United States|-8|apartment| +6222|AAAAAAAAOEIBAAAA|77|3rd |Ave|Suite N|Bunker Hill|Jackson County|OK|70150|United States|-6|apartment| +6223|AAAAAAAAPEIBAAAA|515|Elm Second|Cir.|Suite 310|Lakeview|Greene County|AR|78579|United States|-6|single family| +6224|AAAAAAAAAFIBAAAA|535|North |Ln|Suite T|Bunker Hill|Polk County|TN|30150|United States|-6|apartment| +6225|AAAAAAAABFIBAAAA|417|Mill |Avenue|Suite P|Fairview|McLean County|KY|45709|United States|-5|single family| +6226|AAAAAAAACFIBAAAA|529|Pine Railroad|Pkwy|Suite 370|Summit|Marshall County|IL|60499|United States|-6|apartment| +6227|AAAAAAAADFIBAAAA|772|Dogwood |Drive|Suite Y|Springdale|Lafayette County|FL|38883|United States|-5|condo| +6228|AAAAAAAAEFIBAAAA|491|Sycamore Hillcrest|RD|Suite 360|Hillcrest|Bradford County|FL|33003|United States|-5|single family| +6229|AAAAAAAAFFIBAAAA|49|Maple |Circle|Suite 220|Bridgeport|Macon County|TN|35817|United States|-6|apartment| +6230|AAAAAAAAGFIBAAAA|341|Center Locust|Cir.|Suite U|Woodland|Crisp County|GA|34854|United States|-5|single family| +6231|AAAAAAAAHFIBAAAA|849|Spring 5th|Court|Suite H|Edgewood|Mason County|MI|40069|United States|-5|apartment| +6232|AAAAAAAAIFIBAAAA|787|View |Drive|Suite U|Five Forks|Meade County|KY|42293|United States|-5|condo| +6233|AAAAAAAAJFIBAAAA||Ridge ||Suite 10|Oak Hill|Macon County|IL|||-6|apartment| +6234|AAAAAAAAKFIBAAAA|836|Elm Hill|Wy|Suite U|Union|Brunswick County|NC|28721|United States|-5|condo| +6235|AAAAAAAALFIBAAAA|135|Jackson Adams|Road|Suite Q|Lincoln|Grant County|NM|81289|United States|-7|apartment| +6236|AAAAAAAAMFIBAAAA|957|West |ST|Suite Q|Waterloo|Beaver County|PA|11675|United States|-5|condo| +6237|AAAAAAAANFIBAAAA|842|8th 5th|Avenue|Suite 180|Clinton|Page County|VA|28222|United States|-5|single family| +6238|AAAAAAAAOFIBAAAA|214|Locust |Parkway|Suite 350|Bridgeport|Franklin County|TX|75817|United States|-6|condo| +6239|AAAAAAAAPFIBAAAA|878|Railroad Chestnut|Boulevard|Suite C|Ashland|Grant County|AR|74244|United States|-6|single family| +6240|AAAAAAAAAGIBAAAA|595|3rd Main|Wy|Suite 320|Georgetown|Wilson County|NC|27057|United States|-5|apartment| +6241|AAAAAAAABGIBAAAA|109|Spring |Ln|Suite Y|Murray|De Witt County|IL|62150|United States|-6|apartment| +6242|AAAAAAAACGIBAAAA||||Suite 10|||KY|||-6|apartment| +6243|AAAAAAAADGIBAAAA|907|Lakeview |Dr.|Suite 290|Pine Grove|Beaufort County|SC|24593|United States|-5|condo| +6244|AAAAAAAAEGIBAAAA|291|Ash 4th|Street|Suite E|Florence|Washington County|VT|03994|United States|-5|single family| +6245|AAAAAAAAFGIBAAAA|464|Park |Lane|Suite 150|Jamestown|Indian River County|FL|36867|United States|-5|single family| +6246|AAAAAAAAGGIBAAAA|769|Oak |Cir.|Suite O|Antioch|Pointe Coupee Parish|LA|78605|United States|-6|single family| +6247|AAAAAAAAHGIBAAAA|308|Lincoln Lincoln|Ct.|Suite 40|Lakeside|Nevada County|CA|99532|United States|-8|condo| +6248|AAAAAAAAIGIBAAAA|337|Locust Smith|Circle|Suite 160|Mount Pleasant|Franklin County|IN|41933|United States|-5|condo| +6249|AAAAAAAAJGIBAAAA|612|Central |Dr.|Suite 220|Oak Hill|Tipton County|IN|47838|United States|-5|single family| +6250|AAAAAAAAKGIBAAAA|159|Birch Maple|Avenue|Suite 40|Sunnyside|Marion County|OH|41952|United States|-5|single family| +6251|AAAAAAAALGIBAAAA|755|13th |Court|Suite 210|Spring Hill|Madison County|ID|86787|United States|-7|condo| +6252|AAAAAAAAMGIBAAAA||||Suite 80|Arlington|||46557|||single family| +6253|AAAAAAAANGIBAAAA|359|Cherry 2nd|ST|Suite T|Riverdale|Douglas County|NV|89391|United States|-8|apartment| +6254|AAAAAAAAOGIBAAAA|527|Lake |RD|Suite 180|Snug Harbor|Alameda County|CA|97936|United States|-8|condo| +6255|AAAAAAAAPGIBAAAA|6|6th Smith|Ave|Suite X|Jackson|Okeechobee County|FL|39583|United States|-5|single family| +6256|AAAAAAAAAHIBAAAA|880|Forest |Cir.|Suite T|Enterprise|Stewart County|TN|31757|United States|-6|condo| +6257|AAAAAAAABHIBAAAA|900|Central 11th|Ave|Suite 470|Newtown|Floyd County|KY|41749|United States|-6|apartment| +6258|AAAAAAAACHIBAAAA|85|Hickory Williams|Lane|Suite D|Woodlawn|Gordon County|GA|34098|United States|-5|condo| +6259|AAAAAAAADHIBAAAA|821|Hill Church|Blvd|Suite 260|Florence|Cheatham County|TN|33394|United States|-5|single family| +6260|AAAAAAAAEHIBAAAA|||RD|Suite Q||Henry County|||United States||apartment| +6261|AAAAAAAAFHIBAAAA|137|College |Boulevard|Suite 20|Mount Olive|Coconino County|AZ|88059|United States|-7|single family| +6262|AAAAAAAAGHIBAAAA|817|9th |Street|Suite G|Newtown|Williamsburg city|VA|21749|United States|-5|apartment| +6263|AAAAAAAAHHIBAAAA|461|Lake Sunset|Way|Suite A|Mount Zion|Estill County|KY|48054|United States|-6|apartment| +6264|AAAAAAAAIHIBAAAA|800|7th |Cir.|Suite 320|Highland Park|Kosciusko County|IN|46534|United States|-5|apartment| +6265|AAAAAAAAJHIBAAAA|894|Elevnth |Parkway|Suite K|Brookville|Powell County|KY|43524|United States|-5|single family| +6266|AAAAAAAAKHIBAAAA|848|Spring Adams|Circle|Suite 40|Newport|Edgefield County|SC|21521|United States|-5|single family| +6267|AAAAAAAALHIBAAAA|868|Laurel Elm|Lane|Suite 320|Lakeville|Hampshire County|WV|28811|United States|-5|apartment| +6268|AAAAAAAAMHIBAAAA|242|Elevnth Park|Ave|Suite 110|Antioch|Houston County|TN|38605|United States|-5|apartment| +6269|AAAAAAAANHIBAAAA|957|Sixth |Lane|Suite 260|Hillcrest|Rio Blanco County|CO|83003|United States|-7|condo| +6270|AAAAAAAAOHIBAAAA|48|College 3rd|Ct.|Suite 470|Buena Vista|Perry County|IL|65752|United States|-6|condo| +6271|AAAAAAAAPHIBAAAA|333|Central |Ct.|Suite G|Sunnyside|Lycoming County|PA|11952|United States|-5|condo| +6272|AAAAAAAAAIIBAAAA|425|Second First|Circle|Suite W|Lakeview|Somerset County|MD|28579|United States|-5|apartment| +6273|AAAAAAAABIIBAAAA|710|1st Davis|Ct.|Suite R|Summit|Richmond County|GA|30499|United States|-5|single family| +6274|AAAAAAAACIIBAAAA|453|Meadow River|Wy|Suite B|Lakeside|Sherman County|KS|69532|United States|-6|apartment| +6275|AAAAAAAADIIBAAAA|491|Main 3rd|Wy|Suite V|Five Points|Jackson County|SD|56098|United States|-7|single family| +6276|AAAAAAAAEIIBAAAA|793|6th 6th|Cir.|Suite L|Oak Grove|Gray County|KS|68370|United States|-6|condo| +6277|AAAAAAAAFIIBAAAA|23|Williams Franklin|Drive|Suite 270|Hopewell|Lane County|KS|60587|United States|-6|condo| +6278|AAAAAAAAGIIBAAAA|509|13th 7th|Ln|Suite S|Leesville|Madison County|GA|35423|United States|-5|condo| +6279|AAAAAAAAHIIBAAAA|168|6th |Road|Suite 120|Jamestown|Meade County|KY|46867|United States|-5|single family| +6280|AAAAAAAAIIIBAAAA|375|Mill Main|Dr.|Suite F|Pleasant Hill|Floyd County|IA|53604|United States|-6|apartment| +6281|AAAAAAAAJIIBAAAA|689|Washington |Court|Suite 180|Salem|Rooks County|KS|68048|United States|-6|condo| +6282|AAAAAAAAKIIBAAAA|510|1st |Blvd|Suite 130|Floyd|Marion County|MS|53235|United States|-6|apartment| +6283|AAAAAAAALIIBAAAA|387|12th |Dr.|Suite M|Harmony|Cherokee County|GA|35804|United States|-5|condo| +6284|AAAAAAAAMIIBAAAA|850|Elm Park|Court|Suite S|Walnut Grove||CA|97752|United States|-8|| +6285|AAAAAAAANIIBAAAA|815|4th |Parkway|Suite W|Oak Ridge|Taylor County|KY|48371|United States|-5|apartment| +6286|AAAAAAAAOIIBAAAA|682|Sunset |Dr.|Suite 440|Greenfield|Lewis County|WA|95038|United States|-8|single family| +6287|AAAAAAAAPIIBAAAA|984|13th 8th|Lane|Suite L|Unionville|Washington County|VT|02311|United States|-5|condo| +6288|AAAAAAAAAJIBAAAA|680|Park Highland|Parkway|Suite J|Lebanon|Madison County|AR|72898|United States|-6|condo| +6289|AAAAAAAABJIBAAAA|697|Lincoln |RD|Suite Y|Newtown|Deaf Smith County|TX|71749|United States|-6|condo| +6290|AAAAAAAACJIBAAAA|314|Johnson Wilson|Lane|Suite S|Mount Olive|Hawkins County|TN|38059|United States|-5|single family| +6291|AAAAAAAADJIBAAAA|239|Sunset Maple|Dr.|Suite 270|Tabor|Crook County|OR|98529|United States|-8|single family| +6292|AAAAAAAAEJIBAAAA|456|Jackson |Way|Suite 430|Clinton|Shelby County|IA|58222|United States|-6|apartment| +6293|AAAAAAAAFJIBAAAA|66|Cherry |Way|Suite 70|Macedonia|Watauga County|NC|21087|United States|-5|apartment| +6294|AAAAAAAAGJIBAAAA|337|Pine |RD|Suite 330|Midway|Comanche County|KS|61904|United States|-6|condo| +6295|AAAAAAAAHJIBAAAA|802|West |Ave|Suite 480|Pleasant Hill|Surry County|NC|23604|United States|-5|condo| +6296|AAAAAAAAIJIBAAAA|679|Lake Ash|Avenue|Suite 0|Hillcrest|Taylor County|GA|33003|United States|-5|condo| +6297|AAAAAAAAJJIBAAAA|270|Smith |Avenue|Suite D|Mount Olive|Gwinnett County|GA|38059|United States|-5|single family| +6298|AAAAAAAAKJIBAAAA|479|River |ST|Suite C|Plainville|Tift County|GA|36115|United States|-5|apartment| +6299|AAAAAAAALJIBAAAA|684|Washington First|Ln|Suite G|Jamestown|Garland County|AR|76867|United States|-6|single family| +6300|AAAAAAAAMJIBAAAA|668|Main Jefferson|Avenue|Suite A|Hamilton|Maricopa County|AZ|82808|United States|-7|apartment| +6301|AAAAAAAANJIBAAAA|748|Smith |Ave|Suite Y|Saint James|Stanly County|NC|25799|United States|-5|single family| +6302|AAAAAAAAOJIBAAAA|367|North |Dr.|Suite 220|Florence|Wilkinson County|MS|53394|United States|-6|single family| +6303|AAAAAAAAPJIBAAAA|329|Sixth Hill|Drive|Suite 350|Oakwood|Morgan County|TN|30169|United States|-6|condo| +6304|AAAAAAAAAKIBAAAA|516|Miller |Parkway|Suite 30|Kingston|Lewis and Clark County|MT|64975|United States|-7|single family| +6305|AAAAAAAABKIBAAAA|231|First |Ct.|Suite 440|Centerville|Forsyth County|NC|20059|United States|-5|apartment| +6306|AAAAAAAACKIBAAAA|174|Mill Second|ST|Suite M|Riverdale|Cattaraugus County|NY|19391|United States|-5|single family| +6307|AAAAAAAADKIBAAAA|892|Spring 9th|Lane|Suite 240|Carpenter|Columbus County|NC|21147|United States|-5|single family| +6308|AAAAAAAAEKIBAAAA|377|6th |Road|Suite Q|Pine Grove|Metcalfe County|KY|44593|United States|-5|condo| +6309|AAAAAAAAFKIBAAAA|104|Thirteenth |Cir.|Suite Q|Sulphur Springs|Fulton County|KY|48354|United States|-6|single family| +6310|AAAAAAAAGKIBAAAA|548|Church 15th|Boulevard|Suite 160|Elizabeth|Butte County|CA|92935|United States|-8|single family| +6311|AAAAAAAAHKIBAAAA|197|Forest |Ct.|Suite 20|Oak Ridge|Newport County|RI|08971|United States|-5|single family| +6312|AAAAAAAAIKIBAAAA|534|Franklin |Court|Suite 10|Five Points|Brown County|NE|66098|United States|-6|single family| +6313|AAAAAAAAJKIBAAAA|454|East First|Road|Suite T|Jackson|Columbiana County|OH|49583|United States|-5|condo| +6314|AAAAAAAAKKIBAAAA|22|Birch |Ave|Suite 250|Flint|Pickaway County|OH|48909|United States|-5|single family| +6315|AAAAAAAALKIBAAAA|855|First |Circle|Suite U|Sulphur Springs|Armstrong County|PA|18354|United States|-5|apartment| +6316|AAAAAAAAMKIBAAAA|933|Railroad Eigth|Street|Suite Y|Buena Vista|Monroe County|MO|65752|United States|-6|condo| +6317|AAAAAAAANKIBAAAA|408|Ridge Smith|Court|Suite 460|Greenwood|Wilson County|NC|28828|United States|-5|condo| +6318|AAAAAAAAOKIBAAAA|102|Maple |Avenue|Suite H|Nichols|Washburn County|WI|57940|United States|-6|condo| +6319|AAAAAAAAPKIBAAAA|832|Eigth |Road|Suite P|Liberty|Ware County|GA|33451|United States|-5|single family| +6320|AAAAAAAAALIBAAAA|427|Johnson |Road|Suite 310|Hillcrest|Rockwall County|TX|73003|United States|-6|apartment| +6321|AAAAAAAABLIBAAAA|417|Elm 4th|Wy|Suite 170|Five Points|Columbiana County|OH|46098|United States|-5|apartment| +6322|AAAAAAAACLIBAAAA|796||Cir.|Suite 190|Oak Grove|||||-5|| +6323|AAAAAAAADLIBAAAA|263|9th Walnut|Road|Suite 200|Clifton|Trinity County|TX|78014|United States|-6|single family| +6324|AAAAAAAAELIBAAAA|29|2nd |Road|Suite B|Spring Valley|Las Animas County|CO|86060|United States|-7|single family| +6325|AAAAAAAAFLIBAAAA|956|College Locust|Street|Suite Y|Pleasant Hill|Snyder County|PA|13604|United States|-5|condo| +6326|AAAAAAAAGLIBAAAA|708|View West|Way|Suite 480|Oak Ridge|Wichita County|KS|68371|United States|-6|single family| +6327|AAAAAAAAHLIBAAAA|925||Ln|||Troup County||31821||-5|apartment| +6328|AAAAAAAAILIBAAAA|564|Main Park|Street|Suite N|Lenox|Benton County|AR|71143|United States|-6|apartment| +6329|AAAAAAAAJLIBAAAA|474|North |Way|Suite 190|Deerfield|Clinton County|MI|49840|United States|-5|condo| +6330|AAAAAAAAKLIBAAAA|383|Walnut |Wy|Suite 390|Harmon|Sac County|IA|55623|United States|-6|apartment| +6331|AAAAAAAALLIBAAAA|283|Main |Avenue|Suite Y|Brownsville|Perry County|MS|59310|United States|-6|apartment| +6332|AAAAAAAAMLIBAAAA|704|Maple 13th|Boulevard|Suite C|Mount Vernon|Ouachita County|AR|78482|United States|-6|condo| +6333|AAAAAAAANLIBAAAA|524|2nd Church|Blvd|Suite M|Shelby|Lake County|CA|96575|United States|-8|condo| +6334|AAAAAAAAOLIBAAAA|387|Main |Street|Suite 360|Riverdale|Weld County|CO|89391|United States|-7|apartment| +6335|AAAAAAAAPLIBAAAA|807|Ash Franklin|Circle|Suite 80|Florence|La Porte County|IN|43394|United States|-5|single family| +6336|AAAAAAAAAMIBAAAA|865|Central Park|Dr.|Suite 420|Brownsville|Washington County|VA|29310|United States|-5|apartment| +6337|AAAAAAAABMIBAAAA|188|Williams |Ave|Suite L|Friendship|Allamakee County|IA|54536|United States|-6|single family| +6338|AAAAAAAACMIBAAAA|343|Third Maple|Wy|Suite V|White Oak|Yavapai County|AZ|86668|United States|-7|condo| +6339|AAAAAAAADMIBAAAA|188|Johnson Tenth|Ave|Suite 240|Highland|Barnes County|ND|59454|United States|-6|condo| +6340|AAAAAAAAEMIBAAAA|620|East |Lane|Suite 300|Sunnyside|Harris County|GA|31952|United States|-5|single family| +6341|AAAAAAAAFMIBAAAA|813|Highland |Parkway|Suite B|Greenville|Coffee County|AL|31387|United States|-6|condo| +6342|AAAAAAAAGMIBAAAA|249|South |Ave|Suite 40|Highland|Morgan County|GA|39454|United States|-5|single family| +6343|AAAAAAAAHMIBAAAA|185|River 2nd|Drive|Suite 330|Marion|Golden Valley County|ND|50399|United States|-6|single family| +6344|AAAAAAAAIMIBAAAA|909|Pine 12th|Lane|Suite M|Clifton|Brantley County|GA|38014|United States|-5|condo| +6345|AAAAAAAAJMIBAAAA|604|West |Ct.|Suite P|Georgetown|Saline County|AR|77057|United States|-6|condo| +6346|AAAAAAAAKMIBAAAA|180|3rd Central|Way|Suite 290|Parkwood|Wade Hampton Census Area|AK|91669|United States|-9|apartment| +6347|AAAAAAAALMIBAAAA|48|5th Williams|Drive|Suite 40|Valley View|Lake County|IN|45124|United States|-5|condo| +6348|AAAAAAAAMMIBAAAA|476|Oak 3rd|Circle|Suite P|Pleasant Grove|Dodge County|GA|34136|United States|-5|condo| +6349|AAAAAAAANMIBAAAA|221|Adams |Ave|Suite 190|Lakeside|Obion County|TN|39532|United States|-6|apartment| +6350|AAAAAAAAOMIBAAAA|513|Franklin |Road|Suite Q|Oakdale|Lawrence County|IL|69584|United States|-6|condo| +6351|AAAAAAAAPMIBAAAA|70|14th Spring|Ln|Suite E|Springdale|Greenwood County|KS|68883|United States|-6|single family| +6352|AAAAAAAAANIBAAAA|403|13th Elm|Parkway|Suite X|Woodland|Dubuque County|IA|54854|United States|-6|apartment| +6353|AAAAAAAABNIBAAAA|197|Fourth |Cir.|Suite 260|Highland Park|Falls County|TX|76534|United States|-6|condo| +6354|AAAAAAAACNIBAAAA|944|Madison |Blvd|Suite Q|Springdale|Douglas County|IL|68883|United States|-6|condo| +6355|AAAAAAAADNIBAAAA|498|Maple |Way|Suite N|Plainview|Platte County|WY|83683|United States|-7|apartment| +6356|AAAAAAAAENIBAAAA|680|||Suite I|Unionville|Montgomery County|TX||||condo| +6357|AAAAAAAAFNIBAAAA|225|Oak Broadway|Blvd|Suite 410|Oak Hill|Jackson County|TN|37838|United States|-5|apartment| +6358|AAAAAAAAGNIBAAAA|376|Franklin Second|RD|Suite 220|Enterprise|Crittenden County|AR|71757|United States|-6|single family| +6359|AAAAAAAAHNIBAAAA|565|4th |ST|Suite G|Arlington|Morrow County|OR|96557|United States|-8|single family| +6360|AAAAAAAAINIBAAAA||Woodland North|Pkwy|Suite 200|||TX|||-6|single family| +6361|AAAAAAAAJNIBAAAA|303|Davis 4th|Ln|Suite H|Springfield|Nolan County|TX|79303|United States|-6|single family| +6362|AAAAAAAAKNIBAAAA|974|Third |Pkwy|Suite 90|New Hope|Catron County|NM|89431|United States|-7|condo| +6363|AAAAAAAALNIBAAAA|525|Washington |Parkway|Suite 90|Sulphur Springs|Pender County|NC|28354|United States|-5|condo| +6364|AAAAAAAAMNIBAAAA|594|Center Fourth|Circle|Suite 370|Macedonia|Orange County|VA|21087|United States|-5|condo| +6365|AAAAAAAANNIBAAAA|845|Lee Ash|Ln|Suite 100|Midway|Crawford County|IN|41904|United States|-5|condo| +6366|AAAAAAAAONIBAAAA|18|Hickory Davis|Wy|Suite M|Enterprise|Jones County|NC|21757|United States|-5|single family| +6367|AAAAAAAAPNIBAAAA|211|Sixth |Lane|Suite T|Georgetown|Kalkaska County|MI|47057|United States|-5|single family| +6368|AAAAAAAAAOIBAAAA|134|View |RD|Suite G|Clinton|Newaygo County|MI|48222|United States|-5|condo| +6369|AAAAAAAABOIBAAAA|343|East |Parkway|Suite 270|Lincoln|Manitowoc County|WI|51289|United States|-6|single family| +6370|AAAAAAAACOIBAAAA|188|Adams |Pkwy|Suite 10|Greenville|Baker County|FL|31387|United States|-5|apartment| +6371|AAAAAAAADOIBAAAA|207|Mill Locust|Circle|Suite 160|Bennett|Okeechobee County|FL|31715|United States|-5|apartment| +6372|AAAAAAAAEOIBAAAA|95|River |RD|Suite C|Millbrook|Quay County|NM|87529|United States|-7|condo| +6373|AAAAAAAAFOIBAAAA|505|7th Lake|Dr.|Suite N|Riverview|Chattahoochee County|GA|39003|United States|-5|condo| +6374|AAAAAAAAGOIBAAAA|785|Miller |Cir.|Suite P|Pleasant Valley|Taylor County|KY|42477|United States|-5|single family| +6375|AAAAAAAAHOIBAAAA|287|Valley Adams|Pkwy|Suite L|Newtown|Auglaize County|OH|41749|United States|-5|single family| +6376|AAAAAAAAIOIBAAAA|||ST|Suite 190|||VA||United States|-5|condo| +6377|AAAAAAAAJOIBAAAA|492|Wilson Third|Wy|Suite 260|Crossroads|Karnes County|TX|70534|United States|-6|single family| +6378|AAAAAAAAKOIBAAAA|962|North |Ave|Suite Y|Newtown|Screven County|GA|31749|United States|-5|apartment| +6379|AAAAAAAALOIBAAAA|734|6th |Ct.|Suite D|Lakeside|Hancock County|WV|29532|United States|-5|condo| +6380|AAAAAAAAMOIBAAAA|776|Dogwood Ash|Avenue|Suite D|La Grange|Santa Fe County|NM|87941|United States|-7|condo| +6381|AAAAAAAANOIBAAAA|784|Maple |Road|Suite 460|Shiloh|Meigs County|OH|49275|United States|-5|single family| +6382|AAAAAAAAOOIBAAAA|130|Forest Park|Ct.|Suite 100|Sulphur Springs|Webster County|IA|58354|United States|-6|apartment| +6383|AAAAAAAAPOIBAAAA|389|Cherry |Court|Suite 270|Maywood|Concordia Parish|LA|75681|United States|-6|single family| +6384|AAAAAAAAAPIBAAAA|95|Adams |ST|Suite F|Marion|Wheeler County|GA|30399|United States|-5|condo| +6385|AAAAAAAABPIBAAAA|486|Second 1st|Parkway|Suite A|Hillcrest|Casey County|KY|43003|United States|-6|apartment| +6386|AAAAAAAACPIBAAAA|974|Johnson Church|Road|Suite 350|Winslow|Sharkey County|MS|58525|United States|-6|single family| +6387|AAAAAAAADPIBAAAA|472|||Suite S||Henderson County||31521|United States||| +6388|AAAAAAAAEPIBAAAA|822|Main |Dr.|Suite R|Five Forks|Henry County|OH|42293|United States|-5|condo| +6389|AAAAAAAAFPIBAAAA|809|Maple Oak|Court|Suite 330|Oak Grove|Elkhart County|IN|48370|United States|-5|apartment| +6390|AAAAAAAAGPIBAAAA|86|5th |Boulevard|Suite 260|Crossroads|Saline County|KS|60534|United States|-6|condo| +6391|AAAAAAAAHPIBAAAA|548|Oak |Ct.|Suite 100|Newport|Beadle County|SD|51521|United States|-6|apartment| +6392|AAAAAAAAIPIBAAAA|244|Oak |Cir.|Suite C|Newport|Fluvanna County|VA|21521|United States|-5|condo| +6393|AAAAAAAAJPIBAAAA|5|West |Dr.|Suite B|Stringtown|Louisa County|VA|20162|United States|-5|single family| +6394|AAAAAAAAKPIBAAAA|509|Green |Ln|Suite 0|Mount Pleasant|Russell County|KS|61933|United States|-6|apartment| +6395|AAAAAAAALPIBAAAA|654|Central |Pkwy|Suite U|Greenwood|Pennington County|MN|58828|United States|-6|apartment| +6396|AAAAAAAAMPIBAAAA|362|Fourth Pine|RD|Suite 220|Highland Park|Platte County|NE|66534|United States|-7|apartment| +6397|AAAAAAAANPIBAAAA|142|Hill |Cir.|Suite G|Four Points|Liberty County|MT|61216|United States|-7|single family| +6398|AAAAAAAAOPIBAAAA|732|Fourth |Lane|Suite 80|Crossroads|Union Parish|LA|70534|United States|-6|apartment| +6399|AAAAAAAAPPIBAAAA|597|10th |Drive|Suite 360|Jamestown|San Saba County|TX|76867|United States|-6|apartment| +6400|AAAAAAAAAAJBAAAA|639|7th |Road|Suite 480|Sulphur Springs|Gallatin County|KY|48354|United States|-6|apartment| +6401|AAAAAAAABAJBAAAA|538|Eigth |Blvd|Suite 250|Clifton|Park County|MT|68014|United States|-7|apartment| +6402|AAAAAAAACAJBAAAA|8|Ridge Jackson|Cir.|Suite D|White Oak|Yellow Medicine County|MN|56668|United States|-6|single family| +6403|AAAAAAAADAJBAAAA|667|Locust Walnut|Road|Suite W|Wilson|Tipton County|TN|36971|United States|-6|single family| +6404|AAAAAAAAEAJBAAAA|908|Mill Jefferson|Way|Suite 140|New Hope|Morgan County|AL|39431|United States|-6|condo| +6405|AAAAAAAAFAJBAAAA|351|North |Avenue|Suite J|New Hope|Haskell County|TX|79431|United States|-6|apartment| +6406|AAAAAAAAGAJBAAAA|327|5th Willow|Wy|Suite 50|Valley View|Lincoln County|KS|65124|United States|-6|apartment| +6407|AAAAAAAAHAJBAAAA|125|Cedar |Boulevard|Suite T|Bethel|Hampton County|SC|25281|United States|-5|apartment| +6408|AAAAAAAAIAJBAAAA|860|Miller 4th|ST|Suite 260|Milo|Garfield County|OK|70116|United States|-6|single family| +6409|AAAAAAAAJAJBAAAA|352|Adams Madison|Parkway|Suite G|Highland Park|Ringgold County|IA|56534|United States|-6|single family| +6410|AAAAAAAAKAJBAAAA|951|Dogwood |Drive|Suite O|Mount Olive|Chaves County|NM|88059|United States|-7|condo| +6411|AAAAAAAALAJBAAAA|710|10th 2nd|Dr.|Suite U|Harmony|Howard County|IA|55804|United States|-6|apartment| +6412|AAAAAAAAMAJBAAAA|202|Main Park|Ct.|Suite A|Fairfield|Ogle County|IL|66192|United States|-6|apartment| +6413|AAAAAAAANAJBAAAA|194|Dogwood Eigth|Ct.|Suite 160|Snug Harbor|Burnett County|WI|57936|United States|-6|apartment| +6414|AAAAAAAAOAJBAAAA|419|Williams |Road|Suite P|Oak Hill|Live Oak County|TX|77838|United States|-6|apartment| +6415|AAAAAAAAPAJBAAAA|846|3rd |ST|Suite 470|Lakeview|Metcalfe County|KY|48579|United States|-5|apartment| +6416|AAAAAAAAABJBAAAA|||Boulevard|Suite B|Oakland|Todd County||59843|United States|-6|| +6417|AAAAAAAABBJBAAAA|894|Jackson |Dr.|Suite N|Plainview|Iron County|MO|63683|United States|-6|single family| +6418|AAAAAAAACBJBAAAA|615|Wilson |Circle|Suite U|Maywood|Spotsylvania County|VA|25681|United States|-5|single family| +6419|AAAAAAAADBJBAAAA|866|Maple |RD|Suite O|Oak Grove|Piscataquis County|ME|08970|United States|-5|condo| +6420|AAAAAAAAEBJBAAAA|748|13th |Drive|Suite R|Highland Park|Grayson County|KY|46534|United States|-6|apartment| +6421|AAAAAAAAFBJBAAAA|618|Park |Boulevard|Suite Y|Walnut Grove|Curry County|NM|87752|United States|-7|apartment| +6422|AAAAAAAAGBJBAAAA|138|Railroad Spring|Blvd|Suite 380|Pleasant Hill|Lewis County|KY|43604|United States|-5|single family| +6423|AAAAAAAAHBJBAAAA|480|Lee |Lane|Suite 210|Highland Park|Ingham County|MI|46534|United States|-5|condo| +6424|AAAAAAAAIBJBAAAA|69|Dogwood 12th|Street|Suite Q|Florence|Cottonwood County|MN|53394|United States|-6|apartment| +6425|AAAAAAAAJBJBAAAA|273|Washington |Court|Suite E|Friendship|Wise County|TX|74536|United States|-6|apartment| +6426|AAAAAAAAKBJBAAAA|968|Park |Court|Suite 0|Florence|Carter County|MT|63394|United States|-7|single family| +6427|AAAAAAAALBJBAAAA|82|10th |Street|Suite X|Five Forks|Bullock County|AL|32293|United States|-6|apartment| +6428|AAAAAAAAMBJBAAAA|529|Willow |Drive|Suite N|Spring Hill|Scott County|IA|56787|United States|-6|condo| +6429|AAAAAAAANBJBAAAA|380|7th |Circle|Suite 230|Bunker Hill|Richmond County|NC|20150|United States|-5|condo| +6430|AAAAAAAAOBJBAAAA|754|12th Hickory|Avenue|Suite 70|Edgewood|Carroll County|IN|40069|United States|-5|single family| +6431|AAAAAAAAPBJBAAAA|129|5th Spruce|Drive|Suite F|Springdale|Adams County|WI|58883|United States|-6|condo| +6432|AAAAAAAAACJBAAAA|830|Wilson Central|Lane|Suite M|Farmington|Jefferson County|KS|69145|United States|-6|condo| +6433|AAAAAAAABCJBAAAA|678|Fourth Poplar|Dr.|Suite 410|Mount Vernon|Polk County|FL|38482|United States|-5|condo| +6434|AAAAAAAACCJBAAAA|833|South |Lane|Suite F|Spring Valley|Oakland County|MI|46060|United States|-5|condo| +6435|AAAAAAAADCJBAAAA|947|1st Miller|Dr.|Suite U|Georgetown|Jasper County|MS|57057|United States|-6|single family| +6436|AAAAAAAAECJBAAAA|||Boulevard|Suite S|Shiloh||OH|49275|||| +6437|AAAAAAAAFCJBAAAA|834|Spring Poplar|Ct.|Suite W|Midway|Madison County|NE|61904|United States|-7|single family| +6438|AAAAAAAAGCJBAAAA|847|14th Broadway|Boulevard|Suite 0|Riverview|Gogebic County|MI|49003|United States|-5|apartment| +6439|AAAAAAAAHCJBAAAA|986|Miller |Ave|Suite 240|Bethel|Crook County|OR|95281|United States|-8|apartment| +6440|AAAAAAAAICJBAAAA||Valley |||Mountain View|Gilchrist County|FL||||condo| +6441|AAAAAAAAJCJBAAAA|584|Park Spruce|Wy|Suite B|Highland Park|Grenada County|MS|56534|United States|-6|single family| +6442|AAAAAAAAKCJBAAAA|22|Cherry 6th|Avenue|Suite R|Harmony|Union County|PA|15804|United States|-5|single family| +6443|AAAAAAAALCJBAAAA|162|Dogwood |Cir.|Suite 370|Woodlawn|Piute County|UT|84098|United States|-7|apartment| +6444|AAAAAAAAMCJBAAAA|21|2nd Adams|Avenue|Suite 450|Crossroads|Garvin County|OK|70534|United States|-6|apartment| +6445|AAAAAAAANCJBAAAA|83|Madison |Pkwy|Suite 230|Lakeside|Lucas County|IA|59532|United States|-6|apartment| +6446|AAAAAAAAOCJBAAAA|63|7th Williams|Pkwy|Suite W|Fairfield|Boyle County|KY|46192|United States|-6|apartment| +6447|AAAAAAAAPCJBAAAA|432|Sunset |Blvd|Suite V|Mountain View|Bandera County|TX|74466|United States|-6|condo| +6448|AAAAAAAAADJBAAAA|853|Woodland Green|Ln|Suite 120|Blanchard|Baldwin County|AL|35985|United States|-6|apartment| +6449|AAAAAAAABDJBAAAA|614|Central Second|RD|Suite 330|Woodville|Washoe County|NV|84289|United States|-8|apartment| +6450|AAAAAAAACDJBAAAA|309|Maple |Cir.|Suite 180|Oak Ridge|Petersburg city|VA|28371|United States|-5|apartment| +6451|AAAAAAAADDJBAAAA|591|Pine |Cir.|Suite 300|Ruth|Harris County|TX|70309|United States|-6|condo| +6452|AAAAAAAAEDJBAAAA|215|Washington |Ln|Suite 450|Shiloh|Hendricks County|IN|49275|United States|-5|single family| +6453|AAAAAAAAFDJBAAAA|53|North |Road|Suite C|Brownsville|Sumter County|GA|39310|United States|-5|single family| +6454|AAAAAAAAGDJBAAAA|214|3rd Meadow|ST|Suite 420|Lakeview|Cortland County|NY|18579|United States|-5|condo| +6455|AAAAAAAAHDJBAAAA|618|Tenth |Dr.|Suite 110|Big Creek|Lunenburg County|VA|25386|United States|-5|single family| +6456|AAAAAAAAIDJBAAAA|799|10th Cedar|Ln|Suite Y|Glendale|Rockbridge County|VA|23951|United States|-5|apartment| +6457|AAAAAAAAJDJBAAAA|896|Oak Smith|Wy|Suite 130|Riverview|Jefferson County|OR|99003|United States|-8|apartment| +6458|AAAAAAAAKDJBAAAA|167|South |RD|Suite 20|Clinton|Taylor County|IA|58222|United States|-6|condo| +6459|AAAAAAAALDJBAAAA|706|Highland |Drive|Suite 230|Vienna|Aiken County|SC|25119|United States|-5|condo| +6460|AAAAAAAAMDJBAAAA|603|Meadow |Parkway|Suite 90|Cedar Grove|Washtenaw County|MI|40411|United States|-5|apartment| +6461|AAAAAAAANDJBAAAA|378|Lincoln |Court|Suite 170|Lakeside|Moore County|NC|29532|United States|-5|single family| +6462|AAAAAAAAODJBAAAA|512|Davis |Pkwy|Suite 110|Antioch|Gentry County|MO|68605|United States|-6|single family| +6463|AAAAAAAAPDJBAAAA|345|West |Dr.|Suite 310|Woodland|Van Buren County|AR|74854|United States|-6|condo| +6464|AAAAAAAAAEJBAAAA|977|4th |Court|Suite W|Pine Valley|Fayette County|WV|28209|United States|-5|condo| +6465|AAAAAAAABEJBAAAA|734|Maple |Ct.|Suite 130|Fairview|Phillips County|MT|65709|United States|-7|condo| +6466|AAAAAAAACEJBAAAA|61|5th |Dr.|Suite T|Pine Grove|Knott County|KY|44593|United States|-5|apartment| +6467|AAAAAAAADEJBAAAA|620|Church |Circle|Suite W|Springdale|Sampson County|NC|28883|United States|-5|condo| +6468|AAAAAAAAEEJBAAAA|746|View |Court|Suite 60|Oakdale|Bath County|KY|49584|United States|-6|apartment| +6469|AAAAAAAAFEJBAAAA|350|4th |Parkway|Suite 90|Shady Grove|Dare County|NC|22812|United States|-5|apartment| +6470|AAAAAAAAGEJBAAAA|810|3rd Elm|Lane|Suite S|Clinton|Cuyahoga County|OH|48222|United States|-5|apartment| +6471|AAAAAAAAHEJBAAAA|934|Miller Mill|Parkway|Suite K|Farmington|Miami County|KS|69145|United States|-6|apartment| +6472|AAAAAAAAIEJBAAAA|37|Elm River|Court|Suite B|Centerville|Stone County|AR|70059|United States|-6|condo| +6473|AAAAAAAAJEJBAAAA|797|4th |RD|Suite 250|New Hope|Boone County|AR|79431|United States|-6|apartment| +6474|AAAAAAAAKEJBAAAA|994|Sunset |Ave|Suite N|Roy|San Diego County|CA|90744|United States|-8|apartment| +6475|AAAAAAAALEJBAAAA|379|Highland Lincoln|Lane|Suite 150|Red Hill|Tulare County|CA|94338|United States|-8|condo| +6476|AAAAAAAAMEJBAAAA|107|6th Oak|ST|Suite 390|Dallas|Erie County|NY|13628|United States|-5|apartment| +6477|AAAAAAAANEJBAAAA|568|15th Birch|Drive|Suite A|Springfield|Stevens County|KS|69303|United States|-6|apartment| +6478|AAAAAAAAOEJBAAAA|579|1st |ST|Suite 240|Forest Hills|Custer County|OK|79237|United States|-6|single family| +6479|AAAAAAAAPEJBAAAA|295|South |Ct.|Suite X|Riverview|Summit County|UT|89003|United States|-7|single family| +6480|AAAAAAAAAFJBAAAA|582|Washington |Boulevard|Suite 430|Edgewood|Waynesboro city|VA|20069|United States|-5|single family| +6481|AAAAAAAABFJBAAAA|||Ct.||Mount Olive||||United States||condo| +6482|AAAAAAAACFJBAAAA|405|Sunset |Way|Suite 40|Brownsville|Hanson County|SD|59310|United States|-7|single family| +6483|AAAAAAAADFJBAAAA|709|9th |Wy|Suite S|Lincoln|Yukon-Koyukuk Census Area|AK|91289|United States|-9|apartment| +6484|AAAAAAAAEFJBAAAA|741|Park Dogwood|Parkway|Suite 270|Union|McDowell County|WV|28721|United States|-5|apartment| +6485|AAAAAAAAFFJBAAAA|707|Chestnut River|Boulevard|Suite 210|Hopewell|Pittsburg County|OK|70587|United States|-6|condo| +6486|AAAAAAAAGFJBAAAA|754|Jackson |Dr.|Suite Y|Ashley|Hooker County|NE|64324|United States|-7|apartment| +6487|AAAAAAAAHFJBAAAA|107|Walnut 2nd|ST|Suite 270|Oak Grove|Stanton County|NE|68370|United States|-7|apartment| +6488|AAAAAAAAIFJBAAAA|651|Franklin |Street|Suite B|Green Acres|Licking County|OH|47683|United States|-5|condo| +6489|AAAAAAAAJFJBAAAA|990|Hillcrest |Parkway|Suite W|Centerville|Redwood County|MN|50059|United States|-6|condo| +6490|AAAAAAAAKFJBAAAA|832|Spring |Dr.|Suite H|Macedonia|Raleigh County|WV|21087|United States|-5|single family| +6491|AAAAAAAALFJBAAAA|715|Ninth |Ct.|Suite P|Glenwood|Scott County|IL|63511|United States|-6|single family| +6492|AAAAAAAAMFJBAAAA|423|Main Valley|Lane|Suite 0|Mount Pleasant|Hudson County|NJ|02533|United States|-5|apartment| +6493|AAAAAAAANFJBAAAA|481|Third |Cir.|Suite 280|Plainview|Wyandotte County|KS|63683|United States|-6|condo| +6494|AAAAAAAAOFJBAAAA|463|Jefferson Jackson|Avenue|Suite 400|Spring Hill|Cottle County|TX|76787|United States|-6|apartment| +6495|AAAAAAAAPFJBAAAA|12|Walnut Main|Cir.|Suite 250|Woodland|Glades County|FL|34854|United States|-5|apartment| +6496|AAAAAAAAAGJBAAAA|596|Lincoln |Drive|Suite M|Crossroads|Austin County|TX|70534|United States|-6|single family| +6497|AAAAAAAABGJBAAAA|963|9th |ST|Suite C|Montezuma|Custer County|ID|82150|United States|-7|apartment| +6498|AAAAAAAACGJBAAAA|850|Railroad |Ln|Suite 410|Florence|Harney County|OR|93394|United States|-8|apartment| +6499|AAAAAAAADGJBAAAA|360|Willow |Parkway|Suite P|Sulphur Springs|Calhoun County|MS|58354|United States|-6|apartment| +6500|AAAAAAAAEGJBAAAA|778|West Smith|Way|Suite 100|Glenwood|Aleutians West Census Area|AK|93511|United States|-9|condo| +6501|AAAAAAAAFGJBAAAA|||Dr.|Suite 100||Honolulu County|HI||United States||condo| +6502|AAAAAAAAGGJBAAAA|582|River 3rd|Avenue|Suite L|Riverside|Haywood County|NC|29231|United States|-5|apartment| +6503|AAAAAAAAHGJBAAAA|784|Pine |Ln|Suite 320|Clinton|Hooker County|NE|68222|United States|-7|apartment| +6504|AAAAAAAAIGJBAAAA|67|15th |Wy|Suite 0|Macedonia|Lavaca County|TX|71087|United States|-6|condo| +6505|AAAAAAAAJGJBAAAA|310|6th |Cir.|Suite V|Lakeside|Person County|NC|29532|United States|-5|condo| +6506|AAAAAAAAKGJBAAAA|658|East |Parkway|Suite 330|Springdale|Randolph County|MO|68883|United States|-6|apartment| +6507|AAAAAAAALGJBAAAA|598|Walnut |Avenue|Suite 160|Sunnyside|Tipton County|IN|41952|United States|-5|condo| +6508|AAAAAAAAMGJBAAAA|750|Oak |ST|Suite H|Lakeview|Collin County|TX|78579|United States|-6|single family| +6509|AAAAAAAANGJBAAAA|360|Willow |Court|Suite 440|Valley View|Washington County|FL|35124|United States|-5|apartment| +6510|AAAAAAAAOGJBAAAA|592|Williams Poplar|Ave|Suite O|Sulphur Springs|Bay County|FL|38354|United States|-5|apartment| +6511|AAAAAAAAPGJBAAAA|183|Church |Wy|Suite R|La Grange|Ross County|OH|47941|United States|-5|condo| +6512|AAAAAAAAAHJBAAAA|709|13th Franklin|Avenue|Suite 460|Oak Grove|Fayette County|IN|48370|United States|-5|single family| +6513|AAAAAAAABHJBAAAA|595|Oak Seventh|Wy|Suite 290|Oak Ridge|Haskell County|OK|78371|United States|-6|single family| +6514|AAAAAAAACHJBAAAA|50|Walnut College|Ct.|Suite 100|Summit|Newton County|IN|40499|United States|-5|single family| +6515|AAAAAAAADHJBAAAA|209|7th Lake|Blvd|Suite 180|Youngstown|Cherokee County|KS|60001|United States|-6|condo| +6516|AAAAAAAAEHJBAAAA|102|Railroad Broadway|Road|Suite M|Maple Grove|Siskiyou County|CA|98252|United States|-8|single family| +6517|AAAAAAAAFHJBAAAA|16|Franklin Sixth|Boulevard|Suite C|Oak Hill|La Paz County|AZ|87838|United States|-7|condo| +6518|AAAAAAAAGHJBAAAA|642|Willow 1st|Dr.|Suite W|Sunnyside|Monroe County|IL|61952|United States|-6|single family| +6519|AAAAAAAAHHJBAAAA|582|13th Pine|Wy|Suite T|Hurricane|Kent County|MD|27644|United States|-5|condo| +6520|AAAAAAAAIHJBAAAA|148|Main |RD|Suite 80|Redland|Albany County|NY|16343|United States|-5|condo| +6521|AAAAAAAAJHJBAAAA|||Ave|||Harding County|NM|81654|United States|-7|| +6522|AAAAAAAAKHJBAAAA|272|Elevnth East|Ln|Suite 120|Stringtown|Zavala County|TX|70162|United States|-6|condo| +6523|AAAAAAAALHJBAAAA|857|Spruce |Drive|Suite R|Oakland|Crawford County|WI|59843|United States|-6|apartment| +6524|AAAAAAAAMHJBAAAA|179||Street|Suite 20||Williamson County|TX||United States||apartment| +6525|AAAAAAAANHJBAAAA|214|Valley Sunset|Lane|Suite 190|Oakdale|Tioga County|NY|19584|United States|-5|apartment| +6526|AAAAAAAAOHJBAAAA|315|Spring Highland|Wy|Suite 20|Pleasant Hill|Fairfield County|OH|43604|United States|-5|condo| +6527|AAAAAAAAPHJBAAAA|688|Third River|Wy|Suite H|Plainview|Pennington County|MN|53683|United States|-6|condo| +6528|AAAAAAAAAIJBAAAA|785|Park Sycamore|Way|Suite 80|Wolf Creek|Ashe County|NC|22455|United States|-5|condo| +6529|AAAAAAAABIJBAAAA|483|Sycamore |Blvd|Suite 250|Springfield|Nemaha County|NE|69303|United States|-7|condo| +6530|AAAAAAAACIJBAAAA|818|Third Maple|Parkway|Suite P|Antioch|Leake County|MS|58605|United States|-6|single family| +6531|AAAAAAAADIJBAAAA|237|Central Davis|Ave|Suite I|Oakwood|Hidalgo County|NM|80169|United States|-7|apartment| +6532|AAAAAAAAEIJBAAAA|965|2nd |Wy|Suite 60|Edgewood|Pitkin County|CO|80069|United States|-7|condo| +6533|AAAAAAAAFIJBAAAA|485|Smith Cherry|Ave|Suite R|Newport|Currituck County|NC|21521|United States|-5|single family| +6534|AAAAAAAAGIJBAAAA|355|West |Drive|Suite J|Woodville|Clinton County|MI|44289|United States|-5|condo| +6535|AAAAAAAAHIJBAAAA|428|Ridge |Road|Suite K|Sulphur Springs|York County|PA|18354|United States|-5|single family| +6536|AAAAAAAAIIJBAAAA|800|12th Central|Court|Suite B|Lone Pine|Bienville Parish|LA|77441|United States|-6|condo| +6537|AAAAAAAAJIJBAAAA|864|Pine |RD|Suite 50|Wilton|Uintah County|UT|86997|United States|-7|condo| +6538|AAAAAAAAKIJBAAAA|232|Davis |Ct.|Suite O|Avery|Henry County|TN|30194|United States|-5|apartment| +6539|AAAAAAAALIJBAAAA|677|Church 3rd|ST|Suite 410|Shannon|Caribou County|ID|84120|United States|-7|single family| +6540|AAAAAAAAMIJBAAAA|121|Hill |Circle|Suite 160|Arlington|Anchorage Borough|AK|96557|United States|-9|condo| +6541|AAAAAAAANIJBAAAA|918|11th South|Ln|Suite N|Oak Ridge|Haywood County|TN|38371|United States|-5|condo| +6542|AAAAAAAAOIJBAAAA|796|Walnut |Lane|Suite 60|Clinton|Cobb County|GA|38222|United States|-5|apartment| +6543|AAAAAAAAPIJBAAAA|795|Cherry |RD|Suite 200|Greenfield|Lowndes County|AL|35038|United States|-6|apartment| +6544|AAAAAAAAAJJBAAAA|971|Oak |Court|Suite 60|Summit|Pike County|IL|60499|United States|-6|apartment| +6545|AAAAAAAABJJBAAAA|938|Church Railroad|Blvd|Suite 170|Crossroads|Dutchess County|NY|10534|United States|-5|single family| +6546|AAAAAAAACJJBAAAA|831|Fifth Washington|ST|Suite M|Woodville|Monroe County|WI|54289|United States|-6|single family| +6547|AAAAAAAADJJBAAAA|905|Jackson |Boulevard|Suite B|Florence|Carter County|MT|63394|United States|-7|condo| +6548|AAAAAAAAEJJBAAAA|447|Ridge |Parkway|Suite W|Springfield|Columbia County|WI|59303|United States|-6|condo| +6549|AAAAAAAAFJJBAAAA|323|Main |Blvd|Suite Q|Hamilton|Hill County|TX|72808|United States|-6|condo| +6550|AAAAAAAAGJJBAAAA|457||Street||Five Points||||United States|-5|| +6551|AAAAAAAAHJJBAAAA|328|Elm |Dr.|Suite M|New Hope|Genesee County|NY|19431|United States|-5|condo| +6552|AAAAAAAAIJJBAAAA|624|Broadway Hill|Pkwy|Suite J|Macedonia|Edwards County|TX|71087|United States|-6|single family| +6553|AAAAAAAAJJJBAAAA|440|Hickory |Blvd|Suite 40|Fairview|Dorchester County|MD|25709|United States|-5|single family| +6554|AAAAAAAAKJJBAAAA|541|View 12th|Street|Suite 290|Shiloh|Philadelphia County|PA|19275|United States|-5|condo| +6555|AAAAAAAALJJBAAAA|903|Hillcrest Third|Way|Suite A|Antioch|Wabaunsee County|KS|68605|United States|-6|apartment| +6556|AAAAAAAAMJJBAAAA|757|Wilson Maple|Dr.|Suite J|Lawrenceville|Summit County|UT|84462|United States|-7|condo| +6557|AAAAAAAANJJBAAAA|188|2nd Fourth|Court|Suite X|Greenwood|Columbia County|OR|98828|United States|-8|single family| +6558|AAAAAAAAOJJBAAAA|791|4th |Wy|Suite 380|Newtown|Umatilla County|OR|91749|United States|-8|apartment| +6559|AAAAAAAAPJJBAAAA|239|Sunset Highland|Avenue|Suite L|Liberty|Trigg County|KY|43451|United States|-5|condo| +6560|AAAAAAAAAKJBAAAA|909|West View|Avenue|Suite U|Riverside|Johnson County|IL|69231|United States|-6|apartment| +6561|AAAAAAAABKJBAAAA|55|Eigth |Ct.|Suite I|Spring Hill|Baraga County|MI|46787|United States|-5|single family| +6562|AAAAAAAACKJBAAAA|407|1st |Pkwy|Suite I|Golden|Kerr County|TX|70411|United States|-6|single family| +6563|AAAAAAAADKJBAAAA|522|Lakeview |Circle|Suite E|Hopewell|Butler County|PA|10587|United States|-5|condo| +6564|AAAAAAAAEKJBAAAA|760|Wilson |Way|Suite 140|Mount Zion|Petroleum County|MT|68054|United States|-7|single family| +6565|AAAAAAAAFKJBAAAA|594|Second |Ln|Suite 370|Springfield|Pine County|MN|59303|United States|-6|single family| +6566|AAAAAAAAGKJBAAAA|607|Third Highland|Circle|Suite K|Lakeside|Norton city|VA|29532|United States|-5|apartment| +6567|AAAAAAAAHKJBAAAA|274|Eigth First|Ave|Suite E|White Oak|Starke County|IN|46668|United States|-5|apartment| +6568|AAAAAAAAIKJBAAAA|419|First Fifth|Ct.|Suite 100|Fairfield|Chippewa County|MI|46192|United States|-5|single family| +6569|AAAAAAAAJKJBAAAA|944|1st Jackson|Court|Suite H|Oakland|Laurel County|KY|49843|United States|-5|single family| +6570|AAAAAAAAKKJBAAAA|205|Fourth |Ln|Suite 130|Valley View|Mitchell County|NC|25124|United States|-5|single family| +6571|AAAAAAAALKJBAAAA|35|Jackson 2nd|Cir.|Suite 270|Clearwater|Jim Hogg County|TX|79534|United States|-6|apartment| +6572|AAAAAAAAMKJBAAAA|812|Hill |Blvd|Suite T|Oakdale|Lancaster County|NE|69584|United States|-7|single family| +6573|AAAAAAAANKJBAAAA|51|Main Broadway|Lane|Suite Q|Union|Texas County|OK|78721|United States|-6|condo| +6574|AAAAAAAAOKJBAAAA|958|Cedar South|Parkway|Suite 310|Clearwater|San Luis Obispo County|CA|99534|United States|-8|apartment| +6575|AAAAAAAAPKJBAAAA|550|West Maple|Court|Suite G|Gladstone|Rutherford County|TN|30894|United States|-6|apartment| +6576|AAAAAAAAALJBAAAA|4|3rd Jefferson|Boulevard|Suite J|Riverdale|Northampton County|PA|19391|United States|-5|single family| +6577|AAAAAAAABLJBAAAA|273|Lake |Way|Suite 190|Lakewood|Edgar County|IL|68877|United States|-6|single family| +6578|AAAAAAAACLJBAAAA|32|Madison Willow|Parkway|Suite 460|Springfield|Madison County|VA|29303|United States|-5|condo| +6579|AAAAAAAADLJBAAAA||Central |Drive||Clinton||CO|88222|United States||apartment| +6580|AAAAAAAAELJBAAAA|489|8th Williams|Court|Suite A|Concord|Livingston County|NY|14107|United States|-5|apartment| +6581|AAAAAAAAFLJBAAAA||||Suite P||Marion County||39391|United States|-5|| +6582|AAAAAAAAGLJBAAAA|90|Mill |Circle|Suite 430|Belmont|Williamsburg County|SC|20191|United States|-5|condo| +6583|AAAAAAAAHLJBAAAA|490|View Broadway|Court|Suite 200|Stringtown|Putnam County|NY|10162|United States|-5|condo| +6584|AAAAAAAAILJBAAAA|884||Drive|Suite 10|||TX||United States|-6|| +6585|AAAAAAAAJLJBAAAA|300|Cherry |Ln|Suite 270|Mount Vernon|Boone County|AR|78482|United States|-6|condo| +6586|AAAAAAAAKLJBAAAA|433|Oak |Avenue|Suite O|Bunker Hill|Sumter County|AL|30150|United States|-6|single family| +6587|AAAAAAAALLJBAAAA|451|1st East|Boulevard|Suite U|Pine Grove|Isabella County|MI|44593|United States|-5|condo| +6588|AAAAAAAAMLJBAAAA|624|Poplar Davis|Pkwy|Suite E|Springdale|Dubois County|IN|48883|United States|-5|single family| +6589|AAAAAAAANLJBAAAA|268|3rd |Parkway|Suite 230|Union Hill|Walworth County|SD|57746|United States|-7|condo| +6590|AAAAAAAAOLJBAAAA|872|7th |Ct.|Suite 310|Union Hill|Livingston County|KY|47746|United States|-5|condo| +6591|AAAAAAAAPLJBAAAA|804|2nd |Street|Suite J|Lakewood|King County|TX|78877|United States|-6|single family| +6592|AAAAAAAAAMJBAAAA|63|South |Road|Suite M|Green Acres|Shelby County|OH|47683|United States|-5|single family| +6593|AAAAAAAABMJBAAAA|767|Pine |Way|Suite 280|Pleasant Valley|Fort Bend County|TX|72477|United States|-6|apartment| +6594|AAAAAAAACMJBAAAA||First ||Suite F|Lakewood|||58877||-6|| +6595|AAAAAAAADMJBAAAA|311|Valley |Road|Suite L|Pleasant Hill|Orange County|NY|13604|United States|-5|apartment| +6596|AAAAAAAAEMJBAAAA|179|Johnson |Cir.|Suite G|Farmington|Hartford County|CT|09745|United States|-5|condo| +6597|AAAAAAAAFMJBAAAA|484|Wilson Main|Way|Suite B|Summit|Milwaukee County|WI|50499|United States|-6|apartment| +6598|AAAAAAAAGMJBAAAA|754|Franklin Spring|Lane|Suite 10|Spring Valley|Goliad County|TX|76060|United States|-6|apartment| +6599|AAAAAAAAHMJBAAAA|388|13th |Ave|Suite 200|Little River|Pickens County|SC|20319|United States|-5|apartment| +6600|AAAAAAAAIMJBAAAA|235|1st |Parkway|Suite G|Greenfield|Grand County|CO|85038|United States|-7|single family| +6601|AAAAAAAAJMJBAAAA|36|Broadway |Drive|Suite 100|Ashland|Pawnee County|OK|74244|United States|-6|apartment| +6602|AAAAAAAAKMJBAAAA|19|Chestnut Ash|Pkwy|Suite B|Pleasant Hill|Hardin County|OH|43604|United States|-5|single family| +6603|AAAAAAAALMJBAAAA|396|Birch |RD|Suite 390|Unionville|Blanco County|TX|71711|United States|-6|condo| +6604|AAAAAAAAMMJBAAAA|664|13th Elm|Avenue|Suite 60|Mount Pleasant|Cascade County|MT|61933|United States|-7|single family| +6605|AAAAAAAANMJBAAAA|425|Cherry |Ln|Suite U|Oakland|Stoddard County|MO|69843|United States|-6|condo| +6606|AAAAAAAAOMJBAAAA|217|Church |Dr.|Suite A|Brownsville|Warren County|GA|39310|United States|-5|apartment| +6607|AAAAAAAAPMJBAAAA|469|Oak |Pkwy|Suite W|Newport|Dyer County|TN|31521|United States|-5|apartment| +6608|AAAAAAAAANJBAAAA|978|River |Court|Suite H|Concord|Marshall County|MS|54107|United States|-6|condo| +6609|AAAAAAAABNJBAAAA|622|River |RD|Suite 450|Hamilton|Union County|SC|22808|United States|-5|single family| +6610|AAAAAAAACNJBAAAA|174|Second Fourth|Road|Suite C|Woodlawn|Buchanan County|MO|64098|United States|-6|condo| +6611|AAAAAAAADNJBAAAA|929|7th Hill|Blvd|Suite 450|Woodcrest|Union Parish|LA|74919|United States|-6|apartment| +6612|AAAAAAAAENJBAAAA|610|Wilson |Ave|Suite 20|Antioch|Chippewa County|MI|48605|United States|-5|condo| +6613|AAAAAAAAFNJBAAAA|247|Green |Avenue|||Craig County|VA|29275|United States||| +6614|AAAAAAAAGNJBAAAA|326|9th |Boulevard|Suite 180|Pleasant Hill|Perry County|AR|73604|United States|-6|apartment| +6615|AAAAAAAAHNJBAAAA|282|7th |Court|Suite 40|Ashland|Hamilton County|IN|44244|United States|-5|apartment| +6616|AAAAAAAAINJBAAAA|338|Valley |Way|Suite 160|||WY|89101|United States||| +6617|AAAAAAAAJNJBAAAA|202|Walnut |Ct.|Suite 90|Sutton|La Salle County|IL|65413|United States|-6|single family| +6618|AAAAAAAAKNJBAAAA|536|Cherry |Street|Suite 490|Sulphur Springs|Divide County|ND|58354|United States|-6|apartment| +6619|AAAAAAAALNJBAAAA|752|Railroad |Ct.|Suite T|Indian Village|Custer County|CO|81075|United States|-7|apartment| +6620|AAAAAAAAMNJBAAAA|552|Forest Spruce|ST|Suite O|Jamestown|Ottawa County|KS|66867|United States|-6|apartment| +6621|AAAAAAAANNJBAAAA|190|River Lake|Ave|Suite 140|Oak Hill|Spokane County|WA|97838|United States|-8|apartment| +6622|AAAAAAAAONJBAAAA|472|Dogwood Second|RD|Suite 170|Waterloo|New Haven County|CT|02275|United States|-5|single family| +6623|AAAAAAAAPNJBAAAA|611|Third Spruce|Parkway|Suite E|Springdale|Campbell County|WY|88883|United States|-7|condo| +6624|AAAAAAAAAOJBAAAA|847|View |Road|Suite U|Mount Vernon|Washington County|KY|48482|United States|-5|single family| +6625|AAAAAAAABOJBAAAA|365|14th |Court|Suite 210|Midway|Monongalia County|WV|21904|United States|-5|condo| +6626|AAAAAAAACOJBAAAA|270|13th |Boulevard|Suite 390|Unionville|Dawson County|NE|61711|United States|-6|single family| +6627|AAAAAAAADOJBAAAA|682|Second Mill|Cir.|Suite Q|Friendship|Columbia County|PA|14536|United States|-5|single family| +6628|AAAAAAAAEOJBAAAA|724|Highland South|Lane|Suite H|Providence|Alleghany County|NC|26614|United States|-5|apartment| +6629|AAAAAAAAFOJBAAAA|15|Sunset |Ave|Suite J|Belmont|Green County|KY|40191|United States|-6|single family| +6630|AAAAAAAAGOJBAAAA|469|Main Forest|RD|Suite Q|Woodville|Neshoba County|MS|54289|United States|-6|single family| +6631|AAAAAAAAHOJBAAAA|386|Broadway South|Parkway|Suite 440|Spring Hill|Banner County|NE|66787|United States|-6|condo| +6632|AAAAAAAAIOJBAAAA|525|Park Maple|Boulevard|Suite R|New Hope|Santa Cruz County|CA|99431|United States|-8|single family| +6633|AAAAAAAAJOJBAAAA|56|Sycamore |Cir.|Suite T|Riverview|Sandusky County|OH|49003|United States|-5|single family| +6634|AAAAAAAAKOJBAAAA|624|Spring |Wy|Suite B|Highland Park|Grant County|OR|96534|United States|-8|single family| +6635|AAAAAAAALOJBAAAA|247|Locust |Cir.|Suite 390|Oak Ridge|Freeborn County|MN|58371|United States|-6|apartment| +6636|AAAAAAAAMOJBAAAA|965|5th |ST|Suite 230|Highland|Shelby County|KY|49454|United States|-5|apartment| +6637|AAAAAAAANOJBAAAA|984|Walnut |Boulevard|Suite I|Harmony|Bates County|MO|65804|United States|-6|condo| +6638|AAAAAAAAOOJBAAAA|421|Hickory Johnson|Cir.|Suite K|Glendale|Dakota County|NE|63951|United States|-6|single family| +6639|AAAAAAAAPOJBAAAA|696|7th |RD|Suite G|Oak Hill|Middlesex County|VA|27838|United States|-5|apartment| +6640|AAAAAAAAAPJBAAAA|4|Railroad |Boulevard|Suite 450|Five Points|Union County|NC|26098|United States|-5|single family| +6641|AAAAAAAABPJBAAAA|315|Walnut 14th|Ln|Suite I|Forest Hills|San Miguel County|NM|89237|United States|-7|single family| +6642|AAAAAAAACPJBAAAA|333|North 4th|Drive|Suite F|Woodville|Waushara County|WI|54289|United States|-6|condo| +6643|AAAAAAAADPJBAAAA|814|First |Dr.|Suite 250|Highland Park|Teller County|CO|86534|United States|-7|condo| +6644|AAAAAAAAEPJBAAAA|505|2nd Pine|ST|Suite I|Valley View|Alamosa County|CO|85124|United States|-7|condo| +6645|AAAAAAAAFPJBAAAA|411|Fourth |Circle|Suite 380|Wildwood|Armstrong County|TX|76871|United States|-6|single family| +6646|AAAAAAAAGPJBAAAA|947|Railroad Cedar|Blvd|Suite 190|Birmingham|Jefferson County|WV|23372|United States|-5|single family| +6647|AAAAAAAAHPJBAAAA|118|2nd Birch|Ct.|Suite D|Lakewood|Lincoln County|OK|78877|United States|-6|single family| +6648|AAAAAAAAIPJBAAAA|701|View |Lane|Suite G|Riverdale|Williams County|ND|59391|United States|-6|single family| +6649|AAAAAAAAJPJBAAAA|691|Birch Central|Avenue|Suite 340|Church Hill|Harding County|NM|83790|United States|-7|condo| +6650|AAAAAAAAKPJBAAAA|459|11th Walnut|Street|Suite J|Hopewell|Covington city|VA|20587|United States|-5|condo| +6651|AAAAAAAALPJBAAAA|609|Main |Dr.|Suite 270|Sulphur Springs|Dixie County|FL|38354|United States|-5|apartment| +6652|AAAAAAAAMPJBAAAA|638|Park Green|Ct.|Suite 270|Lakeside|Wayne County|GA|39532|United States|-5|single family| +6653|AAAAAAAANPJBAAAA|195|10th Washington|Dr.|Suite 170|Evans|Todd County|KY|42284|United States|-5|single family| +6654|AAAAAAAAOPJBAAAA|864|Miller |Drive|Suite 340|Pomona|Elmore County|AL|34153|United States|-6|apartment| +6655|AAAAAAAAPPJBAAAA|384|Lakeview |Way|Suite 60|Plainview|Sheridan County|ND|53683|United States|-6|apartment| +6656|AAAAAAAAAAKBAAAA|566|Lee |Avenue|Suite 200|Westgate|Miami County|OH|42366|United States|-5|single family| +6657|AAAAAAAABAKBAAAA|528|4th View|Ct.|Suite 410|Hillcrest|Stephens County|OK|73003|United States|-6|condo| +6658|AAAAAAAACAKBAAAA|228|Park Ash|Road|Suite F|Oakwood|Harper County|OK|70169|United States|-6|condo| +6659|AAAAAAAADAKBAAAA|304|Spring |Wy|Suite Q|Glendale|Gunnison County|CO|83951|United States|-7|single family| +6660|AAAAAAAAEAKBAAAA|551|Oak |Avenue|Suite 170|Rolling Hills|Colorado County|TX|77272|United States|-6|single family| +6661|AAAAAAAAFAKBAAAA|344|Fourth Oak|Parkway|Suite 260|Providence|Kingfisher County|OK|76614|United States|-6|single family| +6662|AAAAAAAAGAKBAAAA|306|Hickory |RD|Suite 180|Woodland|Plaquemines Parish|LA|74854|United States|-6|apartment| +6663|AAAAAAAAHAKBAAAA|271|Walnut 1st|Boulevard|Suite W|Providence|Wetzel County|WV|26614|United States|-5|condo| +6664|AAAAAAAAIAKBAAAA|362|Third |Avenue|Suite 270|Northwood|Waushara County|WI|54104|United States|-6|apartment| +6665|AAAAAAAAJAKBAAAA|299|Birch Sunset||Suite 320|||MN|59583|United States|-6|condo| +6666|AAAAAAAAKAKBAAAA|372|Miller |Road|Suite I|Riverdale|Beadle County|SD|59391|United States|-6|condo| +6667|AAAAAAAALAKBAAAA|||Way|||Perry County|PA||United States|-5|| +6668|AAAAAAAAMAKBAAAA|221|Franklin |Avenue|Suite 310|Franklin|Augusta County|VA|29101|United States|-5|condo| +6669|AAAAAAAANAKBAAAA|868|12th Main|Lane|Suite 60|Highland Park|Osceola County|IA|56534|United States|-6|apartment| +6670|AAAAAAAAOAKBAAAA|838|Miller Jackson|Street|Suite J|Bunker Hill|Grant Parish|LA|70150|United States|-6|single family| +6671|AAAAAAAAPAKBAAAA|572|Ridge Davis|Boulevard|Suite B|Florence|Koochiching County|MN|53394|United States|-6|condo| +6672|AAAAAAAAABKBAAAA|939|North Hickory|Lane|Suite K|Harmony|Chelan County|WA|95804|United States|-8|apartment| +6673|AAAAAAAABBKBAAAA|314|Walnut Spruce|Ct.|Suite 270|Valley View|Rock Island County|IL|65124|United States|-6|apartment| +6674|AAAAAAAACBKBAAAA|678|Valley 6th|Court|Suite 240|Lincoln|Hot Spring County|AR|71289|United States|-6|single family| +6675|AAAAAAAADBKBAAAA|819|Williams |Boulevard|Suite Q|Greenwood|Koochiching County|MN|58828|United States|-6|apartment| +6676|AAAAAAAAEBKBAAAA|126|15th |Lane|Suite D|Woodlawn|Monroe County|MO|64098|United States|-6|apartment| +6677|AAAAAAAAFBKBAAAA|540|8th 14th|Dr.|Suite S|Saratoga|Comanche County|OK|72123|United States|-6|single family| +6678|AAAAAAAAGBKBAAAA|596|View |Parkway|Suite I|Waterloo|Dundy County|NE|61675|United States|-6|apartment| +6679|AAAAAAAAHBKBAAAA|548|Davis River|Boulevard|Suite W|Glendale|Newton County|AR|73951|United States|-6|apartment| +6680|AAAAAAAAIBKBAAAA|494|Valley |ST|Suite Q|Oak Grove|Becker County|MN|58370|United States|-6|apartment| +6681|AAAAAAAAJBKBAAAA|36|Hill Meadow|Parkway|Suite 70|Crossroads|Franklin County|IL|60534|United States|-6|condo| +6682|AAAAAAAAKBKBAAAA|680|Oak |Ln|Suite 50|Centerville|Stokes County|NC|20059|United States|-5|single family| +6683|AAAAAAAALBKBAAAA|135|Green First|ST||||ID||United States|-7|condo| +6684|AAAAAAAAMBKBAAAA|162|Willow |Court|Suite N|Oakwood|Miami County|KS|60169|United States|-6|single family| +6685|AAAAAAAANBKBAAAA|161|South 6th|Avenue|Suite Q|Stringtown|Newton County|IN|40162|United States|-5|single family| +6686|AAAAAAAAOBKBAAAA|182|1st Eigth|Ct.|Suite I|Greenville|Chippewa County|MI|41387|United States|-5|apartment| +6687|AAAAAAAAPBKBAAAA|634|7th 1st|Court|Suite V|Providence|Yakima County|WA|96614|United States|-8|single family| +6688|AAAAAAAAACKBAAAA|184|14th Second|Pkwy|Suite 190|Oak Ridge|Carter County|MT|68371|United States|-7|apartment| +6689|AAAAAAAABCKBAAAA|247|Locust 8th|Boulevard|Suite Q|Riverview|Hamilton County|KS|69003|United States|-6|condo| +6690|AAAAAAAACCKBAAAA|821|Jackson |Avenue|Suite 220|Highland|Montgomery County|TX|79454|United States|-6|single family| +6691|AAAAAAAADCKBAAAA|347|College |Circle|Suite 70|Enterprise|Fulton County|PA|11757|United States|-5|condo| +6692|AAAAAAAAECKBAAAA|858|1st North|Avenue|Suite 460|Green Acres|Charlotte County|FL|37683|United States|-5|condo| +6693|AAAAAAAAFCKBAAAA|329|4th Poplar|Circle|Suite X|Nichols|Calcasieu Parish|LA|77940|United States|-6|condo| +6694|AAAAAAAAGCKBAAAA|606|Elm Oak|Lane|Suite 30|Providence|Chelan County|WA|96614|United States|-8|single family| +6695|AAAAAAAAHCKBAAAA|738|Second |Pkwy|Suite O|Red Hill|Yankton County|SD|54338|United States|-6|condo| +6696|AAAAAAAAICKBAAAA|322|Third Broadway|Blvd|Suite 20|Pine Grove|Athens County|OH|44593|United States|-5|single family| +6697|AAAAAAAAJCKBAAAA|859|Sunset |Drive|Suite 390|Oakdale|Cheyenne County|CO|89584|United States|-7|condo| +6698|AAAAAAAAKCKBAAAA|941|Wilson |Cir.|Suite P|Lakewood|Hood County|TX|78877|United States|-6|condo| +6699|AAAAAAAALCKBAAAA|939|Forest |Ave|Suite 370|Woodlawn|Bienville Parish|LA|74098|United States|-6|condo| +6700|AAAAAAAAMCKBAAAA|335|Second Hillcrest|ST|Suite 430|Florence|Marion County|SC|23394|United States|-5|apartment| +6701|AAAAAAAANCKBAAAA|185|Maple Valley|Street|Suite 490|Wildwood|Schoolcraft County|MI|46871|United States|-5|apartment| +6702|AAAAAAAAOCKBAAAA|470|Dogwood Birch|Street|Suite A|Hopewell|York County|VA|20587|United States|-5|apartment| +6703|AAAAAAAAPCKBAAAA||Lincoln ||Suite L|||||United States|-5|condo| +6704|AAAAAAAAADKBAAAA|333|Walnut |Pkwy|Suite Y|Hartland|Clay County|MS|56594|United States|-6|condo| +6705|AAAAAAAABDKBAAAA|646|Spring Ninth|Street|Suite 50|Riley|Clinton County|MI|41692|United States|-5|single family| +6706|AAAAAAAACDKBAAAA|280|Railroad 6th|Ave|Suite 220|Waterloo|Jefferson County|OH|41675|United States|-5|single family| +6707|AAAAAAAADDKBAAAA|703|Jackson |Cir.|Suite G|Belmont|Clay County|SD|50191|United States|-6|condo| +6708|AAAAAAAAEDKBAAAA|266|West |Dr.|Suite 310|Florence|Lowndes County|MS|53394|United States|-6|condo| +6709|AAAAAAAAFDKBAAAA||Oak |Street|Suite 230||Crawford County||48605||-5|apartment| +6710|AAAAAAAAGDKBAAAA|111|Ridge |Avenue|Suite 170|Pine Grove|Natchitoches Parish|LA|74593|United States|-6|condo| +6711|AAAAAAAAHDKBAAAA|276|11th |Street|Suite R|New Hope|Saluda County|SC|29431|United States|-5|apartment| +6712|AAAAAAAAIDKBAAAA|238|Hill Hill|Blvd|Suite L|Marion|Leon County|FL|30399|United States|-5|single family| +6713|AAAAAAAAJDKBAAAA|518|Elm 4th|Ave|Suite 120|Pleasant Grove|Evangeline Parish|LA|74136|United States|-6|condo| +6714|AAAAAAAAKDKBAAAA|711|5th |Wy|Suite 170|Quincy|Garfield County|UT|83868|United States|-7|apartment| +6715|AAAAAAAALDKBAAAA|327|Jackson Elm|RD|Suite 10|Paxton|Haywood County|TN|35669|United States|-5|apartment| +6716|AAAAAAAAMDKBAAAA|188|Elm |Court|Suite M|Shady Grove|Haskell County|KS|62812|United States|-6|apartment| +6717|AAAAAAAANDKBAAAA|194|Poplar |Drive|Suite O|Pleasant Hill|Butler County|AL|33604|United States|-6|single family| +6718|AAAAAAAAODKBAAAA|304|Washington |Lane|Suite G|Antioch|Wilkinson County|MS|58605|United States|-6|single family| +6719|AAAAAAAAPDKBAAAA|409|Oak |Cir.|Suite U|Oakdale|Sumter County|SC|29584|United States|-5|apartment| +6720|AAAAAAAAAEKBAAAA|650|Cherry |Lane|Suite S|Highland|Montour County|PA|19454|United States|-5|single family| +6721|AAAAAAAABEKBAAAA|333|2nd |Cir.|Suite M|Florence|New London County|CT|03994|United States|-5|apartment| +6722|AAAAAAAACEKBAAAA|771|Walnut |Street|Suite 280|Deerfield|Kandiyohi County|MN|59840|United States|-6|apartment| +6723|AAAAAAAADEKBAAAA|875|Chestnut |Ave|Suite 140|Jones|Crawford County|OH|42686|United States|-5|condo| +6724|AAAAAAAAEEKBAAAA||Pine |Court|Suite I|Cedar Grove|Izard County||||-6|| +6725|AAAAAAAAFEKBAAAA|294|Park |Lane|Suite C|Greenville|Gordon County|GA|31387|United States|-5|apartment| +6726|AAAAAAAAGEKBAAAA|184|Lincoln Pine|Wy|Suite 150|Somerville|Glacier County|MT|67783|United States|-7|single family| +6727|AAAAAAAAHEKBAAAA|835|15th 5th|Lane|Suite 190|Hartland|Clearwater County|ID|86594|United States|-7|apartment| +6728|AAAAAAAAIEKBAAAA|921|Park 9th|Way|Suite 10|Pine Grove|Wilkes County|GA|34593|United States|-5|single family| +6729|AAAAAAAAJEKBAAAA|459|Davis |RD|Suite V|Harmony|Wicomico County|MD|25804|United States|-5|condo| +6730|AAAAAAAAKEKBAAAA|||Pkwy|Suite N|Red Hill|Cascade County|MT|64338|United States||apartment| +6731|AAAAAAAALEKBAAAA|20|Seventh |Ct.|Suite Q|Hillcrest|Anderson County|SC|23003|United States|-5|apartment| +6732|AAAAAAAAMEKBAAAA|738|Park |Blvd|Suite 0|Pleasant Valley|Ford County|KS|62477|United States|-6|condo| +6733|AAAAAAAANEKBAAAA|229|Second Park|RD|Suite 0|Lakeview|Davidson County|NC|28579|United States|-5|apartment| +6734|AAAAAAAAOEKBAAAA|673|Birch Lake|Drive|Suite 340|Mountain View|Arkansas County|AR|74466|United States|-6|condo| +6735|AAAAAAAAPEKBAAAA|58|Hill |Lane|Suite Y|Woodland Park|Dorchester County|MD|21934|United States|-5|single family| +6736|AAAAAAAAAFKBAAAA|564|Lake Ash|Boulevard|Suite 160|Hillcrest|Hamlin County|SD|53003|United States|-7|condo| +6737|AAAAAAAABFKBAAAA|125|Forest Center|Avenue|Suite V|Mount Vernon|Langlade County|WI|58482|United States|-6|apartment| +6738|AAAAAAAACFKBAAAA|187|2nd |Avenue|Suite 480|Mount Pleasant|San Joaquin County|CA|91933|United States|-8|apartment| +6739|AAAAAAAADFKBAAAA|294|Valley Meadow|Cir.|Suite 0|Greenwood|Bienville Parish|LA|78828|United States|-6|single family| +6740|AAAAAAAAEFKBAAAA|427|Madison |Road|Suite 40|Siloam|Garrett County|MD|28948|United States|-5|single family| +6741|AAAAAAAAFFKBAAAA|572|Miller |Ln|Suite I|Mount Pleasant|Nolan County|TX|71933|United States|-6|apartment| +6742|AAAAAAAAGFKBAAAA|291|5th |Drive|Suite 380|Wildwood|Grand County|CO|86871|United States|-7|apartment| +6743|AAAAAAAAHFKBAAAA|173|Madison Railroad|Ave|Suite I|Lakewood|Avery County|NC|28877|United States|-5|apartment| +6744|AAAAAAAAIFKBAAAA|955|View Valley|Ct.|Suite 340|Pleasant Valley|Edwards County|TX|72477|United States|-6|condo| +6745|AAAAAAAAJFKBAAAA|197|Elm |Avenue|Suite R|Parkwood|Pennington County|MN|51669|United States|-6|apartment| +6746|AAAAAAAAKFKBAAAA|348|Park |Lane|Suite 210|Wilson|Pleasants County|WV|26971|United States|-5|single family| +6747|AAAAAAAALFKBAAAA|578|Willow |Circle|Suite Y|Woodville|Merrimack County|NH|04889|United States|-5|single family| +6748|AAAAAAAAMFKBAAAA|320|8th |Cir.|Suite 140|Summit|Marquette County|MI|40499|United States|-5|single family| +6749|AAAAAAAANFKBAAAA|994|Jackson |Boulevard|Suite 270|Georgetown|Christian County|MO|67057|United States|-6|apartment| +6750|AAAAAAAAOFKBAAAA|823|1st 7th|Dr.|Suite L|Ashland|Mason County|TX|74244|United States|-6|apartment| +6751|AAAAAAAAPFKBAAAA|304|6th |Way|Suite 400|Edgewood|Accomack County|VA|20069|United States|-5|apartment| +6752|AAAAAAAAAGKBAAAA|295|Walnut |Street|Suite M|Salem|Polk County|TX|78048|United States|-6|condo| +6753|AAAAAAAABGKBAAAA|613|Center |Parkway|Suite F|Forestville|Rusk County|WI|53027|United States|-6|single family| +6754|AAAAAAAACGKBAAAA|301|Spring |Avenue|Suite 160|Highland Park|Wallowa County|OR|96534|United States|-8|condo| +6755|AAAAAAAADGKBAAAA|167|Oak |Boulevard|Suite 450|Concord|Yankton County|SD|54107|United States|-6|condo| +6756|AAAAAAAAEGKBAAAA|298|Sixth |Avenue|Suite R|Hamilton|Monona County|IA|52808|United States|-6|condo| +6757|AAAAAAAAFGKBAAAA|666|First |Ct.|Suite X|Highland Park|Camden County|NJ|07134|United States|-5|apartment| +6758|AAAAAAAAGGKBAAAA|765|14th Forest|RD|Suite 220|Valley View|Clarke County|VA|25124|United States|-5|condo| +6759|AAAAAAAAHGKBAAAA|810|6th Miller|Way|Suite 220|Bunker Hill|Winnebago County|IL|60150|United States|-6|condo| +6760|AAAAAAAAIGKBAAAA|949|Birch |Boulevard|Suite T|Friendship|Marquette County|MI|44536|United States|-5|apartment| +6761|AAAAAAAAJGKBAAAA|27|Spruce Sycamore|Ln|Suite T|Maple Grove|Florence County|SC|28252|United States|-5|condo| +6762|AAAAAAAAKGKBAAAA|981|Maple Johnson|Boulevard|Suite 260|Five Forks|Lawrence County|TN|32293|United States|-6|apartment| +6763|AAAAAAAALGKBAAAA|456||||Oakdale||IL||United States|-6|| +6764|AAAAAAAAMGKBAAAA|||Ct.|Suite 40|||MN||United States|-6|| +6765|AAAAAAAANGKBAAAA|120|Sycamore Cedar|Street|Suite X|Oak Ridge|Lyon County|NV|88371|United States|-8|apartment| +6766|AAAAAAAAOGKBAAAA|721|Park |Boulevard|Suite 10|Newtown|Morgan County|KY|41749|United States|-5|apartment| +6767|AAAAAAAAPGKBAAAA|974|Franklin 4th|Court|Suite 260|Mount Zion|Summers County|WV|28054|United States|-5|apartment| +6768|AAAAAAAAAHKBAAAA|851|Third |Blvd|Suite 190|Lebanon|Overton County|TN|32898|United States|-6|condo| +6769|AAAAAAAABHKBAAAA|96|Chestnut 6th|Blvd|Suite 290|Union|Huntingdon County|PA|18721|United States|-5|apartment| +6770|AAAAAAAACHKBAAAA|441|5th 7th|Blvd|Suite G|Brownsville|Armstrong County|PA|19310|United States|-5|apartment| +6771|AAAAAAAADHKBAAAA|671|4th |Court|Suite 160|Salem|Faribault County|MN|58048|United States|-6|condo| +6772|AAAAAAAAEHKBAAAA|445|Center |Circle|Suite U|Deerfield|Live Oak County|TX|79840|United States|-6|apartment| +6773|AAAAAAAAFHKBAAAA|528|Madison |Boulevard|Suite 20|Springdale|Butler County|KS|68883|United States|-6|condo| +6774|AAAAAAAAGHKBAAAA|517|Laurel North|Ave|Suite 30|Union|Mercer County|ND|58721|United States|-6|condo| +6775|AAAAAAAAHHKBAAAA|449|13th |Ave|Suite M|Springfield|Valencia County|NM|89303|United States|-7|single family| +6776|AAAAAAAAIHKBAAAA|389|Davis |Court|Suite J|Bunker Hill|Kane County|IL|60150|United States|-6|apartment| +6777|AAAAAAAAJHKBAAAA|451|Valley Sunset|Ln|Suite 120|Oakland|Houston County|MN|59843|United States|-6|condo| +6778|AAAAAAAAKHKBAAAA|906|Park |Wy|Suite 230|Red Hill|Carter County|MT|64338|United States|-7|single family| +6779|AAAAAAAALHKBAAAA|309|Ash Elm|Parkway|Suite A|Forest Hills|Westmoreland County|PA|19237|United States|-5|apartment| +6780|AAAAAAAAMHKBAAAA|715|Main Madison|Street|Suite N|Wilson|Amador County|CA|96971|United States|-8|apartment| +6781|AAAAAAAANHKBAAAA|584|7th |Ct.|Suite L|Vienna|Crawford County|AR|75119|United States|-6|single family| +6782|AAAAAAAAOHKBAAAA|980|Spring |Court|Suite T|Shady Grove|Amelia County|VA|22812|United States|-5|single family| +6783|AAAAAAAAPHKBAAAA|506|Johnson Ridge|Wy|Suite O|Glendale|Teller County|CO|83951|United States|-7|condo| +6784|AAAAAAAAAIKBAAAA|475|Maple |Ct.|Suite 380|Whispering Pines|Mecosta County|MI|47609|United States|-5|single family| +6785|AAAAAAAABIKBAAAA|803|Cedar Central|Circle|Suite R|Glendale|Jefferson County|KS|63951|United States|-6|condo| +6786|AAAAAAAACIKBAAAA|188|Church Wilson|Wy|Suite 80|Springfield|Crawford County|MI|49303|United States|-5|apartment| +6787|AAAAAAAADIKBAAAA|101|Pine Jackson|Avenue|Suite Q|Woodbine|Live Oak County|TX|74253|United States|-6|single family| +6788|AAAAAAAAEIKBAAAA|912|2nd 11th|Court|Suite O|Macedonia|Beadle County|SD|51087|United States|-6|apartment| +6789|AAAAAAAAFIKBAAAA|66|Cedar Hickory|Lane|Suite E|Jamestown|Multnomah County|OR|96867|United States|-8|single family| +6790|AAAAAAAAGIKBAAAA|533|14th |Dr.|Suite 70|Pleasant Valley|Calhoun County|FL|32477|United States|-5|condo| +6791|AAAAAAAAHIKBAAAA|770|9th Church|Drive|Suite 180|Oak Grove|Presque Isle County|MI|48370|United States|-5|condo| +6792|AAAAAAAAIIKBAAAA|574|Birch Walnut|Lane|Suite 190|Glenwood|Shiawassee County|MI|43511|United States|-5|condo| +6793|AAAAAAAAJIKBAAAA|541|9th 4th|Street|Suite H|Five Points|Carter County|KY|46098|United States|-6|single family| +6794|AAAAAAAAKIKBAAAA|615|East Church|Cir.|Suite J|Highland|Le Sueur County|MN|59454|United States|-6|single family| +6795|AAAAAAAALIKBAAAA|730|Sycamore |Road|Suite 310|Hillcrest|Franklin County|KY|43003|United States|-6|condo| +6796|AAAAAAAAMIKBAAAA|947|Hill |Lane|Suite O|Newport|Rice County|MN|51521|United States|-6|condo| +6797|AAAAAAAANIKBAAAA|667|Church Cedar|Boulevard|Suite I|Forest Hills|Elmore County|AL|39237|United States|-6|apartment| +6798|AAAAAAAAOIKBAAAA|816|9th |ST|Suite 50|Springdale|Fairfax County|VA|28883|United States|-5|single family| +6799|AAAAAAAAPIKBAAAA|||Blvd|||Knott County||41749|United States||single family| +6800|AAAAAAAAAJKBAAAA|272|Sixth View|Wy|Suite J|Springfield|El Dorado County|CA|99303|United States|-8|condo| +6801|AAAAAAAABJKBAAAA|753|Sunset Church|Parkway|Suite 480|Kingston|Wythe County|VA|24975|United States|-5|apartment| +6802|AAAAAAAACJKBAAAA|908|Willow |Blvd|Suite A|Newport|Rockingham County|VA|21521|United States|-5|condo| +6803|AAAAAAAADJKBAAAA|644|Spruce Birch|Ln|Suite 30|Midway|Aurora County|SD|51904|United States|-6|single family| +6804|AAAAAAAAEJKBAAAA|678|4th Miller|Circle|Suite B|Glendale|Lenoir County|NC|23951|United States|-5|single family| +6805|AAAAAAAAFJKBAAAA|813|Maple Elm|Drive|Suite X|Riverview|Fairfax County|VA|29003|United States|-5|single family| +6806|AAAAAAAAGJKBAAAA|531|Franklin Fifth|Avenue|Suite G|Antioch|Lawrence County|AR|78605|United States|-6|apartment| +6807|AAAAAAAAHJKBAAAA|564|Madison |Way|Suite N|Waterloo|Lyon County|IA|51675|United States|-6|apartment| +6808|AAAAAAAAIJKBAAAA|938|2nd View|Cir.|Suite N|Lakeside|Gem County|ID|89532|United States|-7|apartment| +6809|AAAAAAAAJJKBAAAA|325|East Willow|ST|Suite N|Clinton|Montgomery County|KS|68222|United States|-6|apartment| +6810|AAAAAAAAKJKBAAAA|448|Park |Ct.|Suite 20|Lincoln|San Bernardino County|CA|91289|United States|-8|apartment| +6811|AAAAAAAALJKBAAAA|73|Valley Fourth|Boulevard|Suite L|Helena|Penobscot County|ME|02249|United States|-5|apartment| +6812|AAAAAAAAMJKBAAAA|439|Davis Valley|Blvd|Suite 210|Union Hill|Lawrence County|MS|57746|United States|-6|condo| +6813|AAAAAAAANJKBAAAA|155|Spring |Dr.|Suite I|White Oak|Mackinac County|MI|46668|United States|-5|single family| +6814|AAAAAAAAOJKBAAAA|624|Center Birch|Lane|Suite J|Newtown|Hardee County|FL|31749|United States|-5|condo| +6815|AAAAAAAAPJKBAAAA|565|Highland |Drive|Suite 270|Shady Grove|Story County|IA|52812|United States|-6|single family| +6816|AAAAAAAAAKKBAAAA|211|Oak |Way|Suite G|Concord|Henry County|TN|34107|United States|-5|single family| +6817|AAAAAAAABKKBAAAA|68|Pine |RD|Suite G|Woodlawn|Martin County|IN|44098|United States|-5|condo| +6818|AAAAAAAACKKBAAAA|380|Cherry 5th|Lane|Suite X|Mount Zion|Cerro Gordo County|IA|58054|United States|-6|apartment| +6819|AAAAAAAADKKBAAAA|213|Church |ST|Suite 460|Arlington|Rio Blanco County|CO|86557|United States|-7|single family| +6820|AAAAAAAAEKKBAAAA|738|First |Pkwy|Suite 110|Shiloh|Furnas County|NE|69275|United States|-6|single family| +6821|AAAAAAAAFKKBAAAA|343|13th |Road|Suite 260|Clifton|Hempstead County|AR|78014|United States|-6|condo| +6822|AAAAAAAAGKKBAAAA|840|Madison Forest|Ct.|Suite I|Bunker Hill|Columbia County|GA|30150|United States|-5|single family| +6823|AAAAAAAAHKKBAAAA|768|Johnson |Drive|Suite 450|Concord|Harrison County|IN|44107|United States|-5|apartment| +6824|AAAAAAAAIKKBAAAA|833|Second |Parkway|Suite 240|Georgetown|Bethel Census Area|AK|97057|United States|-9|single family| +6825|AAAAAAAAJKKBAAAA|605|6th Highland|Way|Suite 90|Ruth|Bernalillo County|NM|80309|United States|-7|single family| +6826|AAAAAAAAKKKBAAAA|31|5th Oak|Circle|Suite 390|Greenwood|Decatur County||48828|United States|-5|| +6827|AAAAAAAALKKBAAAA|90|Center Poplar|Pkwy|Suite C|Mountain View|Monroe County|OH|44466|United States|-5|apartment| +6828|AAAAAAAAMKKBAAAA|80|Hill Cherry|Dr.|Suite G|Crossroads|Harmon County|OK|70534|United States|-6|condo| +6829|AAAAAAAANKKBAAAA|378|10th Third|Avenue|Suite 160|Union|Crittenden County|KY|48721|United States|-6|single family| +6830|AAAAAAAAOKKBAAAA|523|Maple Church|Lane|Suite 250|Belleview|Pierce County|ND|50492|United States|-6|apartment| +6831|AAAAAAAAPKKBAAAA|327|11th Central|Parkway|Suite D|White Oak|Glynn County|GA|36668|United States|-5|single family| +6832|AAAAAAAAALKBAAAA|364|Spruce Tenth|Pkwy|Suite O|Concord|Winn Parish|LA|74107|United States|-6|apartment| +6833|AAAAAAAABLKBAAAA|327|Sixth |Way|Suite I|Marion|York County|SC|20399|United States|-5|apartment| +6834|AAAAAAAACLKBAAAA|794|15th 1st|Cir.|Suite 120|Salem|Manassas city|VA|28048|United States|-5|single family| +6835|AAAAAAAADLKBAAAA|641|Maple 3rd|Ave||Red Hill||CA|94338||-8|condo| +6836|AAAAAAAAELKBAAAA|759|Maple Lee|Ln|Suite 490|Lebanon|Kenedy County|TX|72898|United States|-6|apartment| +6837|AAAAAAAAFLKBAAAA|324|Main Church|Lane|Suite A|Mount Pleasant|Salem County|NJ|02533|United States|-5|apartment| +6838|AAAAAAAAGLKBAAAA|216|Maple West|Circle|Suite 20|Woodlawn|Belknap County|NH|04698|United States|-5|single family| +6839|AAAAAAAAHLKBAAAA|961|Elm Hill|Ct.|Suite U|Providence|Hand County|SD|56614|United States|-7|condo| +6840|AAAAAAAAILKBAAAA|995|South |Circle|Suite U|Union|Hanover County|VA|28721|United States|-5|condo| +6841|AAAAAAAAJLKBAAAA|275|Willow 3rd|Court|Suite 390|Brentwood|Logan County|IL|64188|United States|-6|single family| +6842|AAAAAAAAKLKBAAAA|252|1st |Circle|Suite B|Clinton|Miami County|KS|68222|United States|-6|single family| +6843|AAAAAAAALLKBAAAA|544|Second |Parkway|Suite G|Roy|Jerome County|ID|80744|United States|-7|single family| +6844|AAAAAAAAMLKBAAAA|550|8th Third|Wy|Suite 430|Waterloo|Madison County|MS|51675|United States|-6|single family| +6845|AAAAAAAANLKBAAAA|630|Dogwood |Cir.|Suite O|Hillcrest|Gaines County|TX|73003|United States|-6|single family| +6846|AAAAAAAAOLKBAAAA|661|Pine |Wy|Suite L|Walnut Grove|Iroquois County|IL|67752|United States|-6|apartment| +6847|AAAAAAAAPLKBAAAA|764|Locust |Boulevard|Suite M|New Hope|Lowndes County|GA|39431|United States|-5|single family| +6848|AAAAAAAAAMKBAAAA|740|14th |Lane|Suite 210|Oakland|Oklahoma County|OK|79843|United States|-6|single family| +6849|AAAAAAAABMKBAAAA|125|Ridge Washington|Blvd|Suite N|New Hope|Covington County|MS|59431|United States|-6|apartment| +6850|AAAAAAAACMKBAAAA|137|Pine |Drive|Suite 10|Sulphur Springs|Hill County|MT|68354|United States|-7|single family| +6851|AAAAAAAADMKBAAAA|643|Washington |Dr.|Suite T|Oak Ridge|Cass County|MI|48371|United States|-5|apartment| +6852|AAAAAAAAEMKBAAAA|112|Center Sunset|ST|Suite 290|Fairfield|Lewis and Clark County|MT|66192|United States|-7|apartment| +6853|AAAAAAAAFMKBAAAA|793|Third 9th|Drive|Suite 80|Lincoln|Gosper County|NE|61289|United States|-6|apartment| +6854|AAAAAAAAGMKBAAAA|598|Spruce South|Dr.|Suite 80|Clifton|Ector County|TX|78014|United States|-6|single family| +6855|AAAAAAAAHMKBAAAA|446|3rd |Lane|Suite 390|Oak Hill|Halifax County|VA|27838|United States|-5|apartment| +6856|AAAAAAAAIMKBAAAA|897|3rd West|Parkway|Suite I|Bethel|Clay County|SD|55281|United States|-6|apartment| +6857|AAAAAAAAJMKBAAAA|655|Center Washington|Road|Suite H|Shady Grove|Cass County|MN|52812|United States|-6|single family| +6858|AAAAAAAAKMKBAAAA|813|Central Maple|Road|Suite W|Sulphur Springs|Wilkin County|MN|58354|United States|-6|single family| +6859|AAAAAAAALMKBAAAA|990|Lee |Circle|Suite K|Hamilton|Bernalillo County|NM|82808|United States|-7|condo| +6860|AAAAAAAAMMKBAAAA|226|Maple |Parkway|Suite U|Bunker Hill|Sandusky County|OH|40150|United States|-5|condo| +6861|AAAAAAAANMKBAAAA|630|Forest Elm|Dr.|Suite 470|Oak Hill|Wise County|TX|77838|United States|-6|single family| +6862|AAAAAAAAOMKBAAAA|848|Meadow |Boulevard|Suite L|Belmont|Coffee County|GA|30191|United States|-5|single family| +6863|AAAAAAAAPMKBAAAA|363|Lake Oak|Road|Suite F|Spring Hill|Cibola County|NM|86787|United States|-7|single family| +6864|AAAAAAAAANKBAAAA|578|6th South|Parkway|Suite 130|Wildwood|Graves County|KY|46871|United States|-6|apartment| +6865|AAAAAAAABNKBAAAA|599|5th 15th|Parkway|Suite A|Kingston|Sierra County|NM|84975|United States|-7|condo| +6866|AAAAAAAACNKBAAAA|197|North |ST|Suite 320|Walnut Grove|Clark County|WA|97752|United States|-8|single family| +6867|AAAAAAAADNKBAAAA|744|Cherry |Ln|Suite T|Mount Pleasant|Knox County|IL|61933|United States|-6|single family| +6868|AAAAAAAAENKBAAAA|832|Oak |Cir.|Suite 160|Harmony|Elk County|KS|65804|United States|-6|single family| +6869|AAAAAAAAFNKBAAAA|269|Fourth 3rd|Parkway|Suite N|Jackson|Mitchell County|GA|39583|United States|-5|condo| +6870|AAAAAAAAGNKBAAAA|216|13th |Road|Suite 280|Oak Ridge|Portage County|OH|48371|United States|-5|condo| +6871|AAAAAAAAHNKBAAAA|260|Center Cedar|Drive|Suite L|Oak Hill|Johnson County|NE|67838|United States|-7|apartment| +6872|AAAAAAAAINKBAAAA|435|2nd |Way|Suite 450|Bridgeport|Gates County|NC|25817|United States|-5|condo| +6873|AAAAAAAAJNKBAAAA|922|North 2nd|Dr.|Suite 40|Bunker Hill|Allen County|IN|40150|United States|-5|single family| +6874|AAAAAAAAKNKBAAAA|579|Lake |Wy|Suite N|Springfield|Scott County|TN|39303|United States|-6|single family| +6875|AAAAAAAALNKBAAAA|986|4th |Pkwy|Suite 120|Lakeview|Franklin County|MS|58579|United States|-6|condo| +6876|AAAAAAAAMNKBAAAA|378|Dogwood |Parkway|Suite Y|Fairbanks|Iron County|WI|56653|United States|-6|condo| +6877|AAAAAAAANNKBAAAA|184|Hillcrest Miller|Way|Suite F|Hamilton|Covington County|MS|52808|United States|-6|apartment| +6878|AAAAAAAAONKBAAAA|873|Oak |Dr.|Suite G|Georgetown|Putnam County|GA|37057|United States|-5|condo| +6879|AAAAAAAAPNKBAAAA|360|Poplar Oak|Blvd|Suite 430|Union|Durham County|NC|28721|United States|-5|apartment| +6880|AAAAAAAAAOKBAAAA|959|Second |Circle|Suite 340|Springfield|Crawford County|IL|69303|United States|-6|single family| +6881|AAAAAAAABOKBAAAA|646|4th |Pkwy|Suite K|Greenwood|Miami County|OH|48828|United States|-5|condo| +6882|AAAAAAAACOKBAAAA|890|Second Oak|Parkway|Suite T|Mount Pleasant|Dallas County|MO|61933|United States|-6|condo| +6883|AAAAAAAADOKBAAAA|429|South |Ct.|Suite 160|Waterloo|Humboldt County|CA|91675|United States|-8|condo| +6884|AAAAAAAAEOKBAAAA|456|Franklin Oak|Drive|Suite 170|Murray|Trimble County|KY|42150|United States|-5|apartment| +6885|AAAAAAAAFOKBAAAA|116|Cedar Highland|Pkwy|Suite O|||||||single family| +6886|AAAAAAAAGOKBAAAA|256|9th |Drive|Suite 140|Oak Hill|Rio Blanco County|CO|87838|United States|-7|single family| +6887|AAAAAAAAHOKBAAAA|937|View 3rd|Circle|Suite F|Liberty|Pike County|AR|73451|United States|-6|single family| +6888|AAAAAAAAIOKBAAAA|219|Laurel |Way|Suite 380|Hopewell|Bay County|MI|40587|United States|-5|apartment| +6889|AAAAAAAAJOKBAAAA|218|Fourth |Dr.|Suite T|Bethel|Trinity County|CA|95281|United States|-8|condo| +6890|AAAAAAAAKOKBAAAA|997|Park Miller|Wy|Suite 430|Prosperity|Mitchell County|NC|29089|United States|-5|apartment| +6891|AAAAAAAALOKBAAAA|325|Davis 6th|Parkway|Suite 160|Newport|Noble County|OH|41521|United States|-5|apartment| +6892|AAAAAAAAMOKBAAAA|452|Hill |Ct.|Suite J|Bethel|Moffat County|CO|85281|United States|-7|condo| +6893|AAAAAAAANOKBAAAA|745|Cedar |ST|Suite S|Roxbury|Howard County|IA|55508|United States|-6|apartment| +6894|AAAAAAAAOOKBAAAA|290|Maple West|Boulevard|Suite 110|Greenfield|Muhlenberg County|KY|45038|United States|-5|apartment| +6895|AAAAAAAAPOKBAAAA||Jefferson ||Suite W|Riverside||UT||United States|-7|| +6896|AAAAAAAAAPKBAAAA|100|Park 8th|Court|Suite 100|Belmont|Bosque County|TX|70191|United States|-6|apartment| +6897|AAAAAAAABPKBAAAA|949|Davis Maple|Parkway|Suite 120|Lakeside|Rensselaer County|NY|19532|United States|-5|single family| +6898|AAAAAAAACPKBAAAA|413|1st |Ave|Suite K|Bethel|Morrison County|MN|55281|United States|-6|condo| +6899|AAAAAAAADPKBAAAA|274|14th Central|Parkway|Suite 40|Woodland Park|Montgomery County|IA|51934|United States|-6|single family| +6900|AAAAAAAAEPKBAAAA|64|Elm 7th|Road|Suite 490|Hopewell|Baker County|OR|90587|United States|-8|single family| +6901|AAAAAAAAFPKBAAAA|512|Cedar Hickory|Blvd|Suite 480|Centerville|Lea County|NM|80059|United States|-7|condo| +6902|AAAAAAAAGPKBAAAA|685|Walnut 7th|ST|Suite 450|Bunker Hill|Wagoner County|OK|70150|United States|-6|condo| +6903|AAAAAAAAHPKBAAAA|229|River |ST|Suite 370|Jamestown|Cook County|GA|36867|United States|-5|single family| +6904|AAAAAAAAIPKBAAAA|193|10th |Ct.|Suite N|Antioch|Saguache County|CO|88605|United States|-7|single family| +6905|AAAAAAAAJPKBAAAA|172|||Suite 390|Buena Vista|Mayes County|||||single family| +6906|AAAAAAAAKPKBAAAA|341|Ash |Blvd|Suite V|Forest Hills|Jefferson County|IL|69237|United States|-6|single family| +6907|AAAAAAAALPKBAAAA|263|Park |Avenue|Suite 280|Ridgeville|Brantley County|GA|39306|United States|-5|single family| +6908|AAAAAAAAMPKBAAAA|716|Park |Ln|Suite 150|Spring Grove|Natrona County|WY|86719|United States|-7|apartment| +6909|AAAAAAAANPKBAAAA|272|Oak |Ln|Suite 450|Montague|Accomack County|VA|24062|United States|-5|apartment| +6910|AAAAAAAAOPKBAAAA|183|15th |Ln|Suite P|Greenwood|Clay County|GA|38828|United States|-5|condo| +6911|AAAAAAAAPPKBAAAA|880|Ash |Ln|Suite J|Liberty|Pawnee County|OK|73451|United States|-6|single family| +6912|AAAAAAAAAALBAAAA|544|3rd |Circle|Suite 190|Unionville|Escambia County|AL|31711|United States|-6|apartment| +6913|AAAAAAAABALBAAAA|881|6th |Way|Suite 120|Lakeview|Martin County|MN|58579|United States|-6|condo| +6914|AAAAAAAACALBAAAA|518|Davis |Pkwy|Suite W|Oak Grove|Saluda County|SC|28370|United States|-5|condo| +6915|AAAAAAAADALBAAAA|438|View Madison|Blvd|Suite 100|Bethel|Lubbock County|TX|75281|United States|-6|single family| +6916|AAAAAAAAEALBAAAA|274|College |Cir.|Suite 120|Pleasant Hill|Trego County|KS|63604|United States|-6|apartment| +6917|AAAAAAAAFALBAAAA|468|Park 15th|Cir.|Suite X|Stringtown|Hayes County|NE|60162|United States|-6|single family| +6918|AAAAAAAAGALBAAAA||4th ||||Nodaway County|||||| +6919|AAAAAAAAHALBAAAA|686|Sunset |Drive|Suite P|Woodlawn|Lea County|NM|84098|United States|-7|condo| +6920|AAAAAAAAIALBAAAA|756|Chestnut Adams|Wy|Suite 400|Valley View|Allen County|OH|45124|United States|-5|single family| +6921|AAAAAAAAJALBAAAA|347|4th |Wy|Suite 270|Derby|Schuylkill County|PA|17702|United States|-5|apartment| +6922|AAAAAAAAKALBAAAA|411|Johnson |Cir.|Suite 150|Amherst|Morrill County|NE|68119|United States|-7|single family| +6923|AAAAAAAALALBAAAA|309|Tenth Woodland|Ct.|Suite 140|Farmington|Union County|IA|59145|United States|-6|condo| +6924|AAAAAAAAMALBAAAA|687|Birch 13th|Circle|Suite D|Crossroads|Iroquois County|IL|60534|United States|-6|single family| +6925|AAAAAAAANALBAAAA|826|Cherry |Boulevard|Suite 290|Mount Zion|Noble County|IN|48054|United States|-5|apartment| +6926|AAAAAAAAOALBAAAA|276|1st 3rd|Ln|Suite 330|Glenwood|Bernalillo County|NM|83511|United States|-7|single family| +6927|AAAAAAAAPALBAAAA|215|Smith 11th|Avenue|Suite T|Riverside|Huerfano County|CO|89231|United States|-7|condo| +6928|AAAAAAAAABLBAAAA|523|Main |Cir.|Suite G|Lakeview|Houston County|TX|78579|United States|-6|apartment| +6929|AAAAAAAABBLBAAAA|837|South |Cir.|Suite 350|Brownsville|Pitkin County|CO|89310|United States|-7|single family| +6930|AAAAAAAACBLBAAAA|515|Main |Circle|Suite O|Frogtown|Reeves County|TX|78784|United States|-6|single family| +6931|AAAAAAAADBLBAAAA|929|1st |Court|Suite 270|Highland|Garfield County|NE|69454|United States|-6|condo| +6932|AAAAAAAAEBLBAAAA|||Road|||Montgomery County|MO|||-6|apartment| +6933|AAAAAAAAFBLBAAAA|757|East |Road|Suite 250|Langdon|Caldwell County|NC|20852|United States|-5|single family| +6934|AAAAAAAAGBLBAAAA|469|Jackson |Street|Suite 180|Union Hill|Baylor County|TX|77746|United States|-6|condo| +6935|AAAAAAAAHBLBAAAA|204|3rd |Blvd|Suite 200|Spring Hill|Leon County|TX|76787|United States|-6|single family| +6936|AAAAAAAAIBLBAAAA|509|Cherry Lincoln|Lane|Suite 140|Griffin|Randolph County|NC|21204|United States|-5|single family| +6937|AAAAAAAAJBLBAAAA|252|Walnut |Road|Suite 240|Pleasant Grove|Mineral County|CO|84136|United States|-7|condo| +6938|AAAAAAAAKBLBAAAA|308|Adams Railroad|Dr.|Suite U|Florence|Aitkin County|MN|53394|United States|-6|apartment| +6939|AAAAAAAALBLBAAAA|550|Madison 6th|Avenue|Suite 130|Blair|Camas County|ID|85465|United States|-7|single family| +6940|AAAAAAAAMBLBAAAA|503|6th College|Ave|Suite P|Walnut Grove|Green Lake County|WI|57752|United States|-6|condo| +6941|AAAAAAAANBLBAAAA|854|Fourth Madison|Street|Suite 100|Glenwood|Carroll County|AR|73511|United States|-6|single family| +6942|AAAAAAAAOBLBAAAA|951|Hillcrest Miller|Parkway|Suite 50|Oak Grove|Carter County|KY|48370|United States|-6|condo| +6943|AAAAAAAAPBLBAAAA|100|East |Court|Suite 480|Springdale|Bandera County|TX|78883|United States|-6|condo| +6944|AAAAAAAAACLBAAAA|564|Ridge |Dr.|Suite 200|Bunker Hill|Botetourt County|VA|20150|United States|-5|apartment| +6945|AAAAAAAABCLBAAAA|648|Highland Second|Court|Suite 400|Arlington|Bronx County|NY|16557|United States|-5|single family| +6946|AAAAAAAACCLBAAAA|193|Center Lincoln|Lane|Suite J|Mount Pleasant|Whitley County|IN|41933|United States|-5|condo| +6947|AAAAAAAADCLBAAAA|479|1st Hickory|Ln|Suite 230|Riverside|Lee County|MS|59231|United States|-6|single family| +6948|AAAAAAAAECLBAAAA|752|Adams |Drive|Suite 430|Pleasant Valley|Val Verde County|TX|72477|United States|-6|single family| +6949|AAAAAAAAFCLBAAAA|608|8th |Court|Suite S|Jackson|Union County|NC|29583|United States|-5|condo| +6950|AAAAAAAAGCLBAAAA|37|First Lincoln|Ave|Suite 320|Deerfield|Stillwater County|MT|69840|United States|-7|apartment| +6951|AAAAAAAAHCLBAAAA|580|Williams Main|RD|Suite 120|Riverview|Sumter County|SC|29003|United States|-5|apartment| +6952|AAAAAAAAICLBAAAA|279|Sunset Park|Pkwy|Suite 260|Kingston|Page County|VA|24975|United States|-5|single family| +6953|AAAAAAAAJCLBAAAA||College Lake||Suite Q|Five Points|McKenzie County||56098|United States|-6|| +6954|AAAAAAAAKCLBAAAA|260|3rd Main|Ave|Suite 320|Hamilton|Dickens County|TX|72808|United States|-6|single family| +6955|AAAAAAAALCLBAAAA|606|Sycamore |RD|Suite B|Willow|Lawrence County|TN|36798|United States|-6|condo| +6956|AAAAAAAAMCLBAAAA|717|Adams |Street|Suite 430|Antioch|Allen County|OH|48605|United States|-5|condo| +6957|AAAAAAAANCLBAAAA|35|Cherry Locust|Cir.|Suite T|Woodland|Jackson County|MO|64854|United States|-6|condo| +6958|AAAAAAAAOCLBAAAA|685|Miller 2nd|Road|Suite D|Oak Ridge|Clay County|IL|68371|United States|-6|apartment| +6959|AAAAAAAAPCLBAAAA|599|5th Lakeview|Lane|Suite W|Ashland|Davidson County|TN|34244|United States|-5|apartment| +6960|AAAAAAAAADLBAAAA|872|West |Ave|Suite C|Oakwood|Bland County|VA|20169|United States|-5|single family| +6961|AAAAAAAABDLBAAAA|765|Broadway |Cir.|Suite L|Lakeville|Coosa County|AL|38811|United States|-6|single family| +6962|AAAAAAAACDLBAAAA|681|Jefferson |Pkwy|Suite Q|Fairview|Gogebic County|MI|45709|United States|-5|apartment| +6963|AAAAAAAADDLBAAAA|384|Meadow |Dr.|Suite 200|Providence|Pike County|MO|66614|United States|-6|apartment| +6964|AAAAAAAAEDLBAAAA|227|Walnut Sycamore|Blvd|Suite 240|New Hope|Cottle County|TX|79431|United States|-6|apartment| +6965|AAAAAAAAFDLBAAAA|740|Seventh |Ln|Suite G|Ashland|Sullivan County|MO|64244|United States|-6|single family| +6966|AAAAAAAAGDLBAAAA|563|5th 6th|Pkwy|Suite C|Mount Zion|Cayuga County|NY|18054|United States|-5|single family| +6967|AAAAAAAAHDLBAAAA|498|Maple |Ave|Suite 210|Mount Olive|Halifax County|NC|28059|United States|-5|single family| +6968|AAAAAAAAIDLBAAAA|329|Hickory Smith|Ave|Suite U|Macedonia|Harrison County|MO|61087|United States|-6|condo| +6969|AAAAAAAAJDLBAAAA|313|Washington Franklin|Wy|Suite P|River Oaks|Grays Harbor County|WA|98075|United States|-8|apartment| +6970|AAAAAAAAKDLBAAAA|620|Ash |Street|Suite 440|Hubbard|Monongalia County|WV|26291|United States|-5|apartment| +6971|AAAAAAAALDLBAAAA|232|Mill |Lane|Suite 400|Ashland|Mingo County|WV|24244|United States|-5|apartment| +6972|AAAAAAAAMDLBAAAA|221|Mill Highland|RD|Suite 210|Maple Grove|Lancaster County|SC|28252|United States|-5|condo| +6973|AAAAAAAANDLBAAAA|770|Adams Railroad|Ct.|Suite A|Midway|Lewis and Clark County|MT|61904|United States|-7|single family| +6974|AAAAAAAAODLBAAAA|471|Hickory |Drive|Suite 490|Newtown|Marshall County|AL|31749|United States|-6|apartment| +6975|AAAAAAAAPDLBAAAA|12|Elm North|Pkwy|Suite 370|Hamilton|Cedar County|NE|62808|United States|-6|single family| +6976|AAAAAAAAAELBAAAA|906|Davis Hillcrest|Boulevard|Suite Q|Greenville|Allen County|KY|41387|United States|-6|single family| +6977|AAAAAAAABELBAAAA|625|Jackson Locust|Ave|Suite J|Plainview|Webster County|WV|23683|United States|-5|condo| +6978|AAAAAAAACELBAAAA|509|Lincoln |Parkway|Suite U|Union|Morgan County|IN|48721|United States|-5|condo| +6979|AAAAAAAADELBAAAA|356|9th |RD|Suite A|Riverside|Jay County|IN|49231|United States|-5|apartment| +6980|AAAAAAAAEELBAAAA|286|View Johnson|Ln|Suite F|Pleasant Valley|Preston County|WV|22477|United States|-5|condo| +6981|AAAAAAAAFELBAAAA|616|Hill |Wy|Suite F|Springdale|Cheshire County|NH|09483|United States|-5|single family| +6982|AAAAAAAAGELBAAAA|444|10th |Ave|Suite 70|Spring Hill|Rogers County|OK|76787|United States|-6|single family| +6983|AAAAAAAAHELBAAAA|117|First Pine|Pkwy|Suite K|Providence|Pawnee County|NE|66614|United States|-7|single family| +6984|AAAAAAAAIELBAAAA|233|Lee |Street|Suite T|Hamilton|Valley County|MT|62808|United States|-7|condo| +6985|AAAAAAAAJELBAAAA|266|Maple 1st|Avenue|Suite 250|Oak Grove|Las Animas County|CO|88370|United States|-7|apartment| +6986|AAAAAAAAKELBAAAA|549|Elevnth Railroad|Ln|Suite 170|Harmony|Kitsap County|WA|95804|United States|-8|condo| +6987|AAAAAAAALELBAAAA|66|Center Cedar|Drive|Suite G|Edgewood|Lamoille County|VT|00669|United States|-5|condo| +6988|AAAAAAAAMELBAAAA|174|Railroad |Parkway|Suite I|Greenwood|Autauga County|AL|38828|United States|-6|apartment| +6989|AAAAAAAANELBAAAA|167|Birch |Road|Suite 400|Lakewood|Crawford County|IN|48877|United States|-5|apartment| +6990|AAAAAAAAOELBAAAA|998|11th Sunset|Way|Suite T|Maple Grove|Menominee County|WI|58252|United States|-6|condo| +6991|AAAAAAAAPELBAAAA|628|Jackson Poplar|Parkway|Suite 270|Clifton|Appanoose County|IA|58014|United States|-6|condo| +6992|AAAAAAAAAFLBAAAA|438|Elevnth |Ln|Suite Q|Woodville|Travis County|TX|74289|United States|-6|single family| +6993|AAAAAAAABFLBAAAA|623|Adams |Avenue|Suite 40|Pine Grove|Clay County|MN|54593|United States|-6|apartment| +6994|AAAAAAAACFLBAAAA|385|8th Franklin|RD|Suite R|Pleasant Grove|Maverick County|TX|74136|United States|-6|apartment| +6995|AAAAAAAADFLBAAAA|840|Hill |ST|Suite 260|Mount Vernon|Franklin city|VA|28482|United States|-5|condo| +6996|AAAAAAAAEFLBAAAA|367|Hillcrest |Drive|Suite 340|Deerfield|Logan County|KS|69840|United States|-6|single family| +6997|AAAAAAAAFFLBAAAA|951|Dogwood |Ln|Suite Q|Pleasant Grove|Culberson County|TX|74136|United States|-6|condo| +6998|AAAAAAAAGFLBAAAA|313|Seventh 5th|Drive|Suite F|Brownsville|Anderson County|TX|79310|United States|-6|single family| +6999|AAAAAAAAHFLBAAAA|249|Birch Valley|Ln|Suite L|Harmony|Erie County|OH|45804|United States|-5|single family| +7000|AAAAAAAAIFLBAAAA|289|Jackson Pine|Dr.|Suite G|Forest Hills|Grenada County|MS|59237|United States|-6|single family| +7001|AAAAAAAAJFLBAAAA|553|11th Oak|Boulevard|Suite B|White Oak|Shasta County|CA|96668|United States|-8|condo| +7002|AAAAAAAAKFLBAAAA|475|Cherry 5th|Pkwy|Suite 230|Georgetown|Bee County|TX|77057|United States|-6|single family| +7003|AAAAAAAALFLBAAAA|247|Park North|Street|Suite F|Mount Vernon|Story County|IA|58482|United States|-6|apartment| +7004|AAAAAAAAMFLBAAAA|538|Spring Lee|Boulevard|Suite V|Leesville|El Dorado County|CA|95423|United States|-8|single family| +7005|AAAAAAAANFLBAAAA|243|Cedar Sunset|Court|Suite 50|Bridgeport|Sanpete County|UT|85817|United States|-7|apartment| +7006|AAAAAAAAOFLBAAAA|182|Spring Lee|Ct.|Suite G|Lewis|Bennett County|SD|57066|United States|-6|apartment| +7007|AAAAAAAAPFLBAAAA|549|Jackson 3rd|Street|Suite 490|Spring Hill|Perkins County|NE|66787|United States|-7|apartment| +7008|AAAAAAAAAGLBAAAA|164|Lee Lincoln|Avenue|Suite C|Glendale|Buffalo County|WI|53951|United States|-6|apartment| +7009|AAAAAAAABGLBAAAA|696|Main |Street|Suite U|Oakland|Jackson County|OK|79843|United States|-6|single family| +7010|AAAAAAAACGLBAAAA|116|Smith ||||Jefferson County||32293|||| +7011|AAAAAAAADGLBAAAA|310|South Locust|Dr.|Suite 350|Valley View|Osage County|OK|75124|United States|-6|single family| +7012|AAAAAAAAEGLBAAAA|140|5th Lake|Blvd|Suite 380|Newport|Craighead County|AR|71521|United States|-6|single family| +7013|AAAAAAAAFGLBAAAA|628|12th |Cir.|Suite O|Bethel|Lowndes County|GA|35281|United States|-5|single family| +7014|AAAAAAAAGGLBAAAA|27|West Mill|Blvd|Suite V|Waterloo|Harding County|NM|81675|United States|-7|condo| +7015|AAAAAAAAHGLBAAAA|509|Locust Laurel|Cir.|Suite 360|Springfield|Johnson County|GA|39303|United States|-5|single family| +7016|AAAAAAAAIGLBAAAA|333|College Lincoln|ST|Suite U|Sycamore|Cherokee County|AL|34090|United States|-6|condo| +7017|AAAAAAAAJGLBAAAA|722|Second |Wy|Suite R|Clinton|Irion County|TX|78222|United States|-6|condo| +7018|AAAAAAAAKGLBAAAA|765|Ridge |Drive|Suite M|Lakeview|Alleghany County|VA|28579|United States|-5|single family| +7019|AAAAAAAALGLBAAAA||||Suite 430|Lakewood|||18877|United States|-5|| +7020|AAAAAAAAMGLBAAAA|444|Jefferson Railroad|Ave|Suite P|Globe|Jasper County|MS|59614|United States|-6|condo| +7021|AAAAAAAANGLBAAAA|633|Sunset Hill|Ln|Suite V|Shiloh|Lycoming County|PA|19275|United States|-5|condo| +7022|AAAAAAAAOGLBAAAA|170|Meadow |Ave|Suite 150|Bunker Hill|Goochland County|VA|20150|United States|-5|single family| +7023|AAAAAAAAPGLBAAAA|571|Forest |Circle|Suite 110|Red Hill|Orange County|NY|14338|United States|-5|apartment| +7024|AAAAAAAAAHLBAAAA|36|4th Wilson|Pkwy|Suite C|Bethel|Wade Hampton Census Area|AK|95281|United States|-9|apartment| +7025|AAAAAAAABHLBAAAA|683|Ridge Mill||Suite 120|||WI|||-6|| +7026|AAAAAAAACHLBAAAA|848|Park 4th|Circle|Suite 480|Four Points|Livingston County|KY|41216|United States|-5|single family| +7027|AAAAAAAADHLBAAAA|784|Second Eigth|Wy|Suite 200|Springdale|Martin County|NC|28883|United States|-5|apartment| +7028|AAAAAAAAEHLBAAAA|949|Williams Willow|Blvd|Suite 150|Unionville|Hughes County|OK|71711|United States|-6|apartment| +7029|AAAAAAAAFHLBAAAA|477|Birch |Avenue|Suite 300|Jackson|Kidder County|ND|59583|United States|-6|condo| +7030|AAAAAAAAGHLBAAAA|465|Center Pine|Road|Suite 120|Carter|Telfair County|GA|30919|United States|-5|apartment| +7031|AAAAAAAAHHLBAAAA|59|Hickory Park|Blvd|Suite 10|Kingston|Rosebud County|MT|64975|United States|-7|condo| +7032|AAAAAAAAIHLBAAAA|782|Ash |Avenue|Suite 280|Belmont||ID|80191|United States|-7|condo| +7033|AAAAAAAAJHLBAAAA|707|Chestnut |Avenue|Suite N|Bethel|Houston County|GA|35281|United States|-5|single family| +7034|AAAAAAAAKHLBAAAA|225|Jackson |Ave|Suite 90|Harmony|Sierra County|CA|95804|United States|-8|single family| +7035|AAAAAAAALHLBAAAA|181|9th Railroad|Ln|Suite 30|Clifford|Oconee County|SC|28164|United States|-5|apartment| +7036|AAAAAAAAMHLBAAAA|716|Walnut Fifth|Avenue|Suite B|Brownsville|Stafford County|KS|69310|United States|-6|apartment| +7037|AAAAAAAANHLBAAAA|496|||Suite 80||Sevier County|||United States|-7|| +7038|AAAAAAAAOHLBAAAA|652|15th River|Ave|Suite N|Shady Grove|Nevada County|AR|72812|United States|-6|apartment| +7039|AAAAAAAAPHLBAAAA|807|Laurel Green|Drive|Suite T|Hopewell|Monroe County|KY|40587|United States|-5|condo| +7040|AAAAAAAAAILBAAAA|832|8th Jackson|RD|Suite 220|Clinton|Ashland County|WI|58222|United States|-6|apartment| +7041|AAAAAAAABILBAAAA|741|Forest Cedar|Road|Suite 400|Shannon|Indiana County|PA|14120|United States|-5|condo| +7042|AAAAAAAACILBAAAA||Cedar 6th|Ave||Maple Grove|McKean County|PA||United States||single family| +7043|AAAAAAAADILBAAAA|696|West Maple|Street|Suite K|Vienna|Jackson County|MO|65119|United States|-6|single family| +7044|AAAAAAAAEILBAAAA|954|14th |Drive|Suite X|Valley View|Leon County|FL|35124|United States|-5|apartment| +7045|AAAAAAAAFILBAAAA|315|Spring |Cir.|Suite 430|Harmony|Montgomery County|OH|45804|United States|-5|condo| +7046|AAAAAAAAGILBAAAA|764|Spring 12th|Street|Suite 230|Oakland|Deschutes County|OR|99843|United States|-8|apartment| +7047|AAAAAAAAHILBAAAA|461|Center Elm|Boulevard|Suite O|Sulphur Springs|Clay County|FL|38354|United States|-5|single family| +7048|AAAAAAAAIILBAAAA|821|Lake Locust|Pkwy|Suite 380|Friendship|Morrill County|NE|64536|United States|-7|single family| +7049|AAAAAAAAJILBAAAA|100|Locust 1st|Ln|Suite 180|Sulphur Springs|Brantley County|GA|38354|United States|-5|single family| +7050|AAAAAAAAKILBAAAA|451|Washington Jackson|Ave|Suite 90|Green Acres|Franklin County|MO|67683|United States|-6|apartment| +7051|AAAAAAAALILBAAAA|159|Railroad Hill|Boulevard|Suite V|Lakeview|Somerset County|MD|28579|United States|-5|single family| +7052|AAAAAAAAMILBAAAA|587|Cedar |Lane|Suite J|Oak Hill|Morgan County|MO|67838|United States|-6|apartment| +7053|AAAAAAAANILBAAAA|961|Main Jackson|Ln|Suite 470|Pleasant Grove|Mingo County|WV|24136|United States|-5|condo| +7054|AAAAAAAAOILBAAAA|365|Dogwood |Way|Suite I|Fairfield|Echols County|GA|36192|United States|-5|single family| +7055|AAAAAAAAPILBAAAA|460|Ninth River|Street|Suite N|Summit|Gallia County|OH|40499|United States|-5|single family| +7056|AAAAAAAAAJLBAAAA|749|Cherry Hickory|Boulevard|Suite 420|Wildwood|Daggett County|UT|86871|United States|-7|condo| +7057|AAAAAAAABJLBAAAA|944|Oak |Wy|Suite 330|Five Points|Dearborn County|IN|46098|United States|-5|condo| +7058|AAAAAAAACJLBAAAA|300|Locust |Road|Suite W|Franklin|San Miguel County|CO|89101|United States|-7|single family| +7059|AAAAAAAADJLBAAAA|66|Elm Highland|Ave|Suite 440|Fairview|Sweetwater County|WY|85709|United States|-7|single family| +7060|AAAAAAAAEJLBAAAA|909|Adams |Ln|Suite S|Wildwood|Rockingham County|NH|07471|United States|-5|condo| +7061|AAAAAAAAFJLBAAAA|680|5th South|Dr.|Suite 170|Unionville|Banner County|NE|61711|United States|-6|condo| +7062|AAAAAAAAGJLBAAAA|212|Spring |Blvd|Suite R|Howell|Leake County|MS|54854|United States|-6|apartment| +7063|AAAAAAAAHJLBAAAA|203|Spring |Street|Suite 350|Monroe|San Juan County|WA|93984|United States|-8|apartment| +7064|AAAAAAAAIJLBAAAA|11|Adams |Blvd|Suite S|Macedonia|Yuma County|CO|81087|United States|-7|single family| +7065|AAAAAAAAJJLBAAAA|115|Madison |Parkway|Suite 60|Highland Park|Johnson County|TX|76534|United States|-6|condo| +7066|AAAAAAAAKJLBAAAA|140|Johnson |Wy|Suite J|Oakwood|Presidio County|TX|70169|United States|-6|apartment| +7067|AAAAAAAALJLBAAAA|97|Poplar |Dr.|Suite 370|Fairfield|Glasscock County|TX|76192|United States|-6|condo| +7068|AAAAAAAAMJLBAAAA|20|Main |Dr.|Suite 40|Farmington|Liberty County|FL|39145|United States|-5|apartment| +7069|AAAAAAAANJLBAAAA|949|4th Elm|RD|Suite R|Woodbine|Martin County|MN|54253|United States|-6|apartment| +7070|AAAAAAAAOJLBAAAA|762|||||De Soto Parish|||||condo| +7071|AAAAAAAAPJLBAAAA|876|Hill |Way|Suite D|Murray|Furnas County|NE|62150|United States|-6|condo| +7072|AAAAAAAAAKLBAAAA|285|South |Court|Suite B|Crossroads|Morris County|KS|60534|United States|-6|single family| +7073|AAAAAAAABKLBAAAA|703|Dogwood |Lane|Suite A|Forest Hills|Pamlico County|NC|29237|United States|-5|condo| +7074|AAAAAAAACKLBAAAA|916|Spring Lee|Parkway|Suite Q|Waterloo|Presidio County|TX|71675|United States|-6|apartment| +7075|AAAAAAAADKLBAAAA|80|Woodland 5th|Street|Suite 10|Hopewell|New Castle County|DE|10587|United States|-5|single family| +7076|AAAAAAAAEKLBAAAA|991|6th Williams|Street|Suite 360|Union|Hamilton County|NE|68721|United States|-6|single family| +7077|AAAAAAAAFKLBAAAA|183|9th |Ln|Suite 350|Hopewell|Tioga County|PA|10587|United States|-5|condo| +7078|AAAAAAAAGKLBAAAA|309|2nd |Ln|Suite 430|Lakeside|Larimer County|CO|89532|United States|-7|single family| +7079|AAAAAAAAHKLBAAAA|84|Woodland |Drive|Suite A|Liberty|Wright County|IA|53451|United States|-6|single family| +7080|AAAAAAAAIKLBAAAA|285|Jefferson Spring|Street|Suite 380|Highland Park|Hancock County|TN|36534|United States|-5|apartment| +7081|AAAAAAAAJKLBAAAA|5|Fifth |Boulevard|Suite 150|Union Hill|Lincoln County|OK|77746|United States|-6|apartment| +7082|AAAAAAAAKKLBAAAA|581|Walnut Sycamore|Circle|Suite R|Valley View|Neshoba County|MS|55124|United States|-6|apartment| +7083|AAAAAAAALKLBAAAA|136|East |Boulevard|Suite U|Antioch|Hopewell city|VA|28605|United States|-5|apartment| +7084|AAAAAAAAMKLBAAAA|||||Washington Heights|||98167|||| +7085|AAAAAAAANKLBAAAA|685|Walnut |Pkwy|Suite N|Belmont|Manassas Park city|VA|20191|United States|-5|condo| +7086|AAAAAAAAOKLBAAAA|334|Jackson 5th|Cir.|Suite Q|Sulphur Springs|Manassas city|VA|28354|United States|-5|condo| +7087|AAAAAAAAPKLBAAAA|67|Central |Court|Suite 90|Five Forks|Switzerland County|IN|42293|United States|-5|condo| +7088|AAAAAAAAALLBAAAA|901|Oak |Circle|Suite I|Five Forks|Sabine County|TX|72293|United States|-6|condo| +7089|AAAAAAAABLLBAAAA|537|Jefferson |Ln|Suite A|Stringtown|Henderson County|NC|20162|United States|-5|apartment| +7090|AAAAAAAACLLBAAAA|117|Miller |Pkwy|Suite V|Lakewood|Pendleton County|KY|48877|United States|-5|condo| +7091|AAAAAAAADLLBAAAA|833|Main |Ln|Suite I|Cherry Valley|Southampton County|VA|20854|United States|-5|apartment| +7092|AAAAAAAAELLBAAAA|400|Lake Willow|Court|Suite G|Mount Zion|Minidoka County|ID|88054|United States|-7|apartment| +7093|AAAAAAAAFLLBAAAA|704|Mill Hickory|Court|Suite 190|Tipton|Latah County|ID|88773|United States|-7|condo| +7094|AAAAAAAAGLLBAAAA|738|Hill |Ct.|Suite 490|Highland|Newport County|RI|09454|United States|-5|single family| +7095|AAAAAAAAHLLBAAAA|478|Davis |Ave|Suite 460|Oakdale|Graham County|NC|29584|United States|-5|condo| +7096|AAAAAAAAILLBAAAA|496|Sycamore Dogwood|Ln|Suite L|Florence|Gilpin County|CO|83394|United States|-7|single family| +7097|AAAAAAAAJLLBAAAA|198|4th Ash|Pkwy|Suite M|Riverdale|Vernon Parish|LA|79391|United States|-6|condo| +7098|AAAAAAAAKLLBAAAA|321|Chestnut |Way|Suite 340|Newport|Runnels County|TX|71521|United States|-6|apartment| +7099|AAAAAAAALLLBAAAA|848|Mill Sycamore|Cir.|Suite O|Providence|Cumberland County|NC|26614|United States|-5|condo| +7100|AAAAAAAAMLLBAAAA|208|Jackson |Road|Suite 380|Spring Valley|Crittenden County|AR|76060|United States|-6|single family| +7101|AAAAAAAANLLBAAAA|881|Central Wilson|Drive|Suite X|Bethel|Kalamazoo County|MI|45281|United States|-5|apartment| +7102|AAAAAAAAOLLBAAAA|276|9th |Ave|Suite 130|Bridgeport|Breathitt County|KY|45817|United States|-6|apartment| +7103|AAAAAAAAPLLBAAAA|507|12th Washington|Street|Suite L|Glendale|Anderson County|TX|73951|United States|-6|condo| +7104|AAAAAAAAAMLBAAAA|846|Valley Second|Street|Suite 240|Riverview|Ingham County|MI|49003|United States|-5|apartment| +7105|AAAAAAAABMLBAAAA|104|Sycamore |RD|Suite L|Deerfield|Trumbull County|OH|49840|United States|-5|condo| +7106|AAAAAAAACMLBAAAA|411|Elm Main|Road|Suite 20|Clifton|Bannock County|ID|88014|United States|-7|single family| +7107|AAAAAAAADMLBAAAA|762|6th Woodland|Blvd|Suite 490|Woodland|Perry County|AR|74854|United States|-6|single family| +7108|AAAAAAAAEMLBAAAA|831|First Oak|Drive|Suite N|Crossroads|Richmond County|GA|30534|United States|-5|single family| +7109|AAAAAAAAFMLBAAAA|145|Spring Main|Ct.|Suite 480|Providence|Saluda County|SC|26614|United States|-5|single family| +7110|AAAAAAAAGMLBAAAA||Hill Hickory|Parkway|Suite E|Greenfield||MS||||| +7111|AAAAAAAAHMLBAAAA|608|Ridge |Blvd|Suite M|Woodland|Big Stone County|MN|54854|United States|-6|condo| +7112|AAAAAAAAIMLBAAAA|538|Hickory |Avenue|Suite S|Shiloh|Boone County|IN|49275|United States|-5|apartment| +7113|AAAAAAAAJMLBAAAA|957|Center |Ave|Suite 410|Enterprise|Putnam County|IL|61757|United States|-6|condo| +7114|AAAAAAAAKMLBAAAA|462|Main |RD|Suite Q|Pine Grove|Erie County|OH|44593|United States|-5|single family| +7115|AAAAAAAALMLBAAAA|915|Sycamore |Street|Suite 70|Woodlawn|Sheridan County|MT|64098|United States|-7|condo| +7116|AAAAAAAAMMLBAAAA|212|3rd Park|Ln|Suite C|Valley View|Texas County|OK|75124|United States|-6|apartment| +7117|AAAAAAAANMLBAAAA|639|Hickory |Lane|Suite 270|Mountain View|Charles City County|VA|24466|United States|-5|single family| +7118|AAAAAAAAOMLBAAAA|592|Valley 8th|Wy|Suite V|Fairview|Bertie County|NC|25709|United States|-5|apartment| +7119|AAAAAAAAPMLBAAAA|144|1st 1st|Blvd|Suite R|Jackson|Pike County|OH|49583|United States|-5|single family| +7120|AAAAAAAAANLBAAAA|391|3rd |Ct.|Suite N|Liberty|Wirt County|WV|23451|United States|-5|apartment| +7121|AAAAAAAABNLBAAAA|319|Hickory |Ct.|Suite 370|Hopewell|Nuckolls County|NE|60587|United States|-7|condo| +7122|AAAAAAAACNLBAAAA|589|West Sixth|Ave|Suite X|Five Points|Stanton County|NE|66098|United States|-7|condo| +7123|AAAAAAAADNLBAAAA|747|West |Way|Suite J|Fairfield|Irwin County|GA|36192|United States|-5|apartment| +7124|AAAAAAAAENLBAAAA|426|4th |Wy|Suite 200|Mount Pleasant|Keweenaw County|MI|41933|United States|-5|condo| +7125|AAAAAAAAFNLBAAAA|142|10th |Parkway|Suite 140|Hazelwood|Oneida County|ID|81206|United States|-7|single family| +7126|AAAAAAAAGNLBAAAA|463|Maple |Parkway|Suite Y|Walnut Grove|Henry County|VA|27752|United States|-5|condo| +7127|AAAAAAAAHNLBAAAA|174|Ash Main|Court|Suite 190|Buena Vista|Osage County|MO|65752|United States|-6|single family| +7128|AAAAAAAAINLBAAAA|876|Poplar Cedar|Avenue|Suite 240|Newtown|Loudon County|TN|31749|United States|-6|condo| +7129|AAAAAAAAJNLBAAAA|133|5th 6th|ST|Suite G|Wildwood|Stephens County|TX|76871|United States|-6|apartment| +7130|AAAAAAAAKNLBAAAA|899|Adams Elm|Blvd|Suite M|Springfield|Mineral County|WV|29303|United States|-5|apartment| +7131|AAAAAAAALNLBAAAA|95|5th Main|Way|Suite 160|Five Forks|Calhoun County|WV|22293|United States|-5|single family| +7132|AAAAAAAAMNLBAAAA|831|10th Mill|Circle|Suite 420|Liberty|Montague County|TX|73451|United States|-6|apartment| +7133|AAAAAAAANNLBAAAA|758|Miller 2nd|Wy|Suite 120|Deerfield|Pickett County|TN|39840|United States|-6|apartment| +7134|AAAAAAAAONLBAAAA|316|2nd 11th|Dr.|Suite 450|Woodlawn|Richland County|MT|64098|United States|-7|condo| +7135|AAAAAAAAPNLBAAAA|874|Maple 1st|Ln|Suite 250|Plainview|Sullivan County|PA|13683|United States|-5|condo| +7136|AAAAAAAAAOLBAAAA|625|Forest Poplar|Ln|Suite 400|Mount Olive|Hardee County|FL|38059|United States|-5|condo| +7137|AAAAAAAABOLBAAAA|850|Ash |Way|Suite 180|Maple Grove|Stone County|AR|78252|United States|-6|single family| +7138|AAAAAAAACOLBAAAA|250|Chestnut Elm|Wy|Suite 450|Pine Grove|Monroe County|MS|54593|United States|-6|single family| +7139|AAAAAAAADOLBAAAA|363|Spring |Circle|Suite 130|Shiloh|Aroostook County|ME|09875|United States|-5|single family| +7140|AAAAAAAAEOLBAAAA|842|Smith |Blvd|Suite 110|Hamilton|Stevens County|WA|92808|United States|-8|single family| +7141|AAAAAAAAFOLBAAAA|986|Willow Laurel|Blvd|Suite Q|Wildwood|Fergus County|MT|66871|United States|-7|single family| +7142|AAAAAAAAGOLBAAAA|201|Madison |Road|Suite 350|Pine Grove|Adams County|IA|54593|United States|-6|apartment| +7143|AAAAAAAAHOLBAAAA||||Suite 60|Providence|||||-6|apartment| +7144|AAAAAAAAIOLBAAAA|282|3rd Mill|Avenue|Suite 10|Centerville|Mercer County|KY|40059|United States|-5|condo| +7145|AAAAAAAAJOLBAAAA|249|Central Spruce|RD|Suite C|Gum Springs|Wheeler County|OR|92106|United States|-8|condo| +7146|AAAAAAAAKOLBAAAA|414|Broadway Mill|Lane|Suite 480|Newtown|Marshall County|AL|31749|United States|-6|apartment| +7147|AAAAAAAALOLBAAAA|156|Meadow View|Way|Suite 220|Redland|Uintah County|UT|86343|United States|-7|apartment| +7148|AAAAAAAAMOLBAAAA|388|5th Poplar|Boulevard|Suite 430|Lucas|Seneca County|NY|14554|United States|-5|apartment| +7149|AAAAAAAANOLBAAAA|679|North |Avenue|Suite 380|Bear Creek|Big Horn County|MT|63075|United States|-7|single family| +7150|AAAAAAAAOOLBAAAA|29|Lincoln |Wy|Suite 160|Pine Grove|Clark County|WI|54593|United States|-6|single family| +7151|AAAAAAAAPOLBAAAA|847|North |Ct.|Suite 310|Oak Grove|Montgomery County|GA|38370|United States|-5|single family| +7152|AAAAAAAAAPLBAAAA|989|Hill Cedar|Court|Suite G|Lakeview|Screven County|GA|38579|United States|-5|single family| +7153|AAAAAAAABPLBAAAA|245|Fifth Lake|Pkwy|Suite B|Lincoln|Crawford County|MO|61289|United States|-6|apartment| +7154|AAAAAAAACPLBAAAA|642|Davis |Boulevard|Suite V|Derby|Carroll County|MO|67702|United States|-6|apartment| +7155|AAAAAAAADPLBAAAA|895|Spring |ST|Suite D|Calhoun|Hutchinson County|SD|56909|United States|-7|single family| +7156|AAAAAAAAEPLBAAAA|13|4th |Avenue|Suite X|Shiloh|Warren County|TN|39275|United States|-6|apartment| +7157|AAAAAAAAFPLBAAAA||Hickory View||||||48014|United States||| +7158|AAAAAAAAGPLBAAAA|457|Madison Center|Dr.|Suite 330|Riverside|Beaufort County|NC|29231|United States|-5|condo| +7159|AAAAAAAAHPLBAAAA||Church |||Pleasant Valley|||32477|United States|-5|| +7160|AAAAAAAAIPLBAAAA|707|Second 8th|Boulevard|Suite 220|Price|Jenkins County|GA|33824|United States|-5|condo| +7161|AAAAAAAAJPLBAAAA|323|Highland |Blvd|Suite 60|Concord|Rockwall County|TX|74107|United States|-6|condo| +7162|AAAAAAAAKPLBAAAA|305|Maple Fifth|Street|Suite 200|Deerfield|Paulding County|GA|39840|United States|-5|single family| +7163|AAAAAAAALPLBAAAA|516|Church Pine|Street|Suite S|Red Hill|Ingham County|MI|44338|United States|-5|condo| +7164|AAAAAAAAMPLBAAAA|173|Park Maple|Road|Suite Y|Sulphur Springs|Thomas County|KS|68354|United States|-6|condo| +7165|AAAAAAAANPLBAAAA|290|West Madison|RD|Suite 50|Mountain View|Woodford County|KY|44466|United States|-6|single family| +7166|AAAAAAAAOPLBAAAA|318|Sunset |Circle|Suite 360|Springfield|Crowley County|CO|89303|United States|-7|single family| +7167|AAAAAAAAPPLBAAAA|894|2nd Second|Ct.|Suite 110|Blanchard|Grant County|OK|75985|United States|-6|condo| +7168|AAAAAAAAAAMBAAAA|214|Pine Sixth|Cir.|Suite 390|Spring Valley|Boone County|NE|66060|United States|-6|condo| +7169|AAAAAAAABAMBAAAA|155|4th Highland|Circle|Suite S|White Oak|Jefferson County|MO|66668|United States|-6|condo| +7170|AAAAAAAACAMBAAAA|858|||||Van Buren County|AR||||single family| +7171|AAAAAAAADAMBAAAA|742|Johnson |Circle|Suite M|Bethel|Bourbon County|KS|65281|United States|-6|single family| +7172|AAAAAAAAEAMBAAAA|609|Laurel Walnut|Court|Suite 470|Lee|Pulaski County|VA|20408|United States|-5|condo| +7173|AAAAAAAAFAMBAAAA|56|3rd Cherry|Cir.|Suite 270|Pine Grove|Madison County|AR|74593|United States|-6|condo| +7174|AAAAAAAAGAMBAAAA|717|Cherry Hickory|Way|Suite L|Deerfield|Monterey County|CA|99840|United States|-8|condo| +7175|AAAAAAAAHAMBAAAA|696|Highland Meadow|ST|Suite 130|White Oak|Rock County|MN|56668|United States|-6|single family| +7176|AAAAAAAAIAMBAAAA|209|Fourth |Boulevard|Suite 200|Marion|Montgomery County|TX|70399|United States|-6|apartment| +7177|AAAAAAAAJAMBAAAA|239|Fourth |Cir.|Suite 100|Lakeview|Beaufort County|NC|28579|United States|-5|condo| +7178|AAAAAAAAKAMBAAAA|704|Forest |Way|Suite 360|Farmington|Iroquois County|IL|69145|United States|-6|single family| +7179|AAAAAAAALAMBAAAA|729|Madison |Dr.|Suite 370|Cedar Grove|Johnston County|NC|20411|United States|-5|apartment| +7180|AAAAAAAAMAMBAAAA|9|Cedar Main|Cir.|Suite H|Ruth|Wagoner County|OK|70309|United States|-6|condo| +7181|AAAAAAAANAMBAAAA|553|Johnson Lee|Ct.|Suite 430|Salem|Weld County|CO|88048|United States|-7|apartment| +7182|AAAAAAAAOAMBAAAA|815|Hill |Way|Suite M|Sunnyside|Washington County|OH|41952|United States|-5|apartment| +7183|AAAAAAAAPAMBAAAA|282|Franklin |ST|Suite 360|Oakwood|Chaffee County|CO|80169|United States|-7|single family| +7184|AAAAAAAAABMBAAAA|625|Center |Boulevard|Suite 340|Hamilton|Tuscaloosa County|AL|32808|United States|-6|apartment| +7185|AAAAAAAABBMBAAAA|999|Smith Jefferson|Way|Suite E|Mountain View|Linn County|MO|64466|United States|-6|condo| +7186|AAAAAAAACBMBAAAA|506|Maple |Road|Suite 320|Forest Hills|Winona County|MN|59237|United States|-6|condo| +7187|AAAAAAAADBMBAAAA|5|8th 2nd|Cir.|Suite 130|Bridgeport|Sheridan County|ND|55817|United States|-6|apartment| +7188|AAAAAAAAEBMBAAAA|193|North Sunset|Avenue|Suite 260|Centerville|Holt County|MO|60059|United States|-6|apartment| +7189|AAAAAAAAFBMBAAAA|973|Center Oak|Court|Suite 440|Sumner|Licking County|OH|40519|United States|-5|apartment| +7190|AAAAAAAAGBMBAAAA|458|Williams Pine|Ct.|Suite D|White Oak|Spencer County|IN|46668|United States|-5|single family| +7191|AAAAAAAAHBMBAAAA|3|Miller 3rd|Pkwy|Suite V|Fairview|Shelby County|KY|45709|United States|-5|single family| +7192|AAAAAAAAIBMBAAAA|800|View College|Wy|Suite N|Bear Creek|Walthall County|MS|53075|United States|-6|condo| +7193|AAAAAAAAJBMBAAAA|388|Chestnut 9th|Drive|Suite M|Belmont|Clay County|FL|30191|United States|-5|condo| +7194|AAAAAAAAKBMBAAAA|332|Lake |Parkway|Suite A|Buena Vista|Greene County|GA|35752|United States|-5|apartment| +7195|AAAAAAAALBMBAAAA|979|Broadway |Lane|Suite O|Shady Grove|McMinn County|TN|32812|United States|-6|condo| +7196|AAAAAAAAMBMBAAAA|13|Williams 10th|Court|Suite C|Red Bank|Warren County|MS|54975|United States|-6|condo| +7197|AAAAAAAANBMBAAAA|414|Walnut |RD|Suite I|Greenwood|Deer Lodge County|MT|68828|United States|-7|condo| +7198|AAAAAAAAOBMBAAAA|835|Twelfth Hill|Blvd|Suite 420|Fairfield|Seward County|KS|66192|United States|-6|apartment| +7199|AAAAAAAAPBMBAAAA|202|Washington |Wy|Suite 60|Pleasant Grove|Lewis and Clark County|MT|64136|United States|-7|apartment| +7200|AAAAAAAAACMBAAAA|37|Cedar 9th|Wy|Suite 100|Cedar Grove|Wallace County|KS|60411|United States|-6|single family| +7201|AAAAAAAABCMBAAAA|907|Valley |Ave|Suite 250|Brownsville|Buchanan County|VA|29310|United States|-5|apartment| +7202|AAAAAAAACCMBAAAA|398|Woodland 13th|Boulevard|Suite Y|Forest Hills|Doddridge County|WV|29237|United States|-5|apartment| +7203|AAAAAAAADCMBAAAA|289|2nd 1st|Blvd|Suite 430|Florence|Mason County|TX|73394|United States|-6|apartment| +7204|AAAAAAAAECMBAAAA|127|10th |Drive|Suite Q|Wildwood|Roanoke city|VA|26871|United States|-5|single family| +7205|AAAAAAAAFCMBAAAA|242|Third |Cir.|Suite J|Summit|Morgan County|WV|20499|United States|-5|apartment| +7206|AAAAAAAAGCMBAAAA|210|Birch |Pkwy|Suite O|Springfield|Corson County|SD|59303|United States|-6|apartment| +7207|AAAAAAAAHCMBAAAA|371|Ridge 5th|Lane|Suite 350|Mountain View|Modoc County|CA|94466|United States|-8|condo| +7208|AAAAAAAAICMBAAAA|912|Main |Road|Suite J|Concord|Boundary County|ID|84107|United States|-7|apartment| +7209|AAAAAAAAJCMBAAAA|282|Cherry |Ln|Suite Q|Wildwood|Washington County|GA|36871|United States|-5|apartment| +7210|AAAAAAAAKCMBAAAA|935|Cherry |Wy|Suite 210|Unionville|Carroll County|AR|71711|United States|-6|apartment| +7211|AAAAAAAALCMBAAAA|108|South |Court|Suite A|Concord|Crook County|OR|94107|United States|-8|single family| +7212|AAAAAAAAMCMBAAAA|474|Johnson ||Suite 480||||||-6|| +7213|AAAAAAAANCMBAAAA|156|4th |Road|Suite 170|Riverside|Jasper County|IL|69231|United States|-6|condo| +7214|AAAAAAAAOCMBAAAA|349|Lake |Way|Suite 480|Page|Orangeburg County|SC|20296|United States|-5|condo| +7215|AAAAAAAAPCMBAAAA|916|Willow West|Avenue|Suite K|Glenwood|Ottawa County|KS|63511|United States|-6|single family| +7216|AAAAAAAAADMBAAAA||||Suite 380||Dickey County|ND|54289|||condo| +7217|AAAAAAAABDMBAAAA|563|Second Sycamore|Circle|Suite I|Plainville|Watonwan County|MN|56115|United States|-6|condo| +7218|AAAAAAAACDMBAAAA|406|2nd |ST|Suite E|Cedar Grove|Fountain County|IN|40411|United States|-5|apartment| +7219|AAAAAAAADDMBAAAA|463|River Madison|Avenue|Suite F|Woodville|Centre County|PA|14289|United States|-5|apartment| +7220|AAAAAAAAEDMBAAAA|473|Sunset 7th|Court|Suite C|Clifton|Collier County|FL|38014|United States|-5|condo| +7221|AAAAAAAAFDMBAAAA|349|Walnut |Court|Suite Y|Four Points|Miami County|IN|41216|United States|-5|condo| +7222|AAAAAAAAGDMBAAAA|64|Franklin |Ln|Suite 90|Mount Olive|Ashland County|OH|48059|United States|-5|single family| +7223|AAAAAAAAHDMBAAAA|357|Jackson |Avenue|Suite 20|Wildwood|Hughes County|OK|76871|United States|-6|apartment| +7224|AAAAAAAAIDMBAAAA|495||Way|Suite 50|Five Forks|||52293||-6|| +7225|AAAAAAAAJDMBAAAA|254|Twelfth Meadow|Ct.|Suite E|Liberty|Mingo County|WV|23451|United States|-5|single family| +7226|AAAAAAAAKDMBAAAA|81|Lake |Avenue|Suite 20|Mount Vernon|Monmouth County|NJ|09082|United States|-5|condo| +7227|AAAAAAAALDMBAAAA|826|Willow Forest|Circle|Suite T|Walnut Grove|Taylor County|WI|57752|United States|-6|condo| +7228|AAAAAAAAMDMBAAAA|971|12th Jackson|Ct.|Suite 320|Harmony|Issaquena County|MS|55804|United States|-6|condo| +7229|AAAAAAAANDMBAAAA|894|Park |Ave|Suite J|Antioch|Macon County|TN|38605|United States|-6|single family| +7230|AAAAAAAAODMBAAAA|176|Railroad 6th|Lane|Suite 290|Clinton|Allen County|KS|68222|United States|-6|apartment| +7231|AAAAAAAAPDMBAAAA|986|Sycamore |Drive|Suite 330|Pleasant Hill|Jackson County|MO|63604|United States|-6|apartment| +7232|AAAAAAAAAEMBAAAA||Jackson |Wy|||||||-6|single family| +7233|AAAAAAAABEMBAAAA||||||Kershaw County|SC|||-5|| +7234|AAAAAAAACEMBAAAA|775|Birch |Ln|Suite J|Somerset|DeBaca County|NM|85580|United States|-7|apartment| +7235|AAAAAAAADEMBAAAA|290|Dogwood |Pkwy|Suite 340|Clinton|Cloud County|KS|68222|United States|-6|condo| +7236|AAAAAAAAEEMBAAAA|802|Spring |Parkway|Suite 240|Riverside|Morgan County|OH|49231|United States|-5|apartment| +7237|AAAAAAAAFEMBAAAA|474|Seventh |Blvd|Suite 380|Franklin|Madison County|NY|19101|United States|-5|apartment| +7238|AAAAAAAAGEMBAAAA|448|1st River|Ct.|Suite 350|Mount Vernon|Sumter County|FL|38482|United States|-5|apartment| +7239|AAAAAAAAHEMBAAAA|801|Washington |Dr.|Suite K|Buena Vista|Kodiak Island Borough|AK|95752|United States|-9|single family| +7240|AAAAAAAAIEMBAAAA|417|Oak 4th|Blvd|Suite 30|Highland|Wyoming County|WV|29454|United States|-5|apartment| +7241|AAAAAAAAJEMBAAAA|105|East Second|Way|Suite 480|Enterprise|Orange County|VT|02357|United States|-5|single family| +7242|AAAAAAAAKEMBAAAA|23|Pine 10th|Ln|Suite 110|Mount Pleasant|Wood County|WI|51933|United States|-6|single family| +7243|AAAAAAAALEMBAAAA|819|Jackson |Wy|Suite S|Hillcrest|Ripley County|IN|43003|United States|-5|single family| +7244|AAAAAAAAMEMBAAAA|115|Center |Pkwy|Suite H|Clifton|Elmore County|AL|38014|United States|-6|apartment| +7245|AAAAAAAANEMBAAAA|231|5th Fourth|Ave|Suite 460|Union Hill|Griggs County|ND|57746|United States|-6|apartment| +7246|AAAAAAAAOEMBAAAA|737|1st |Road|Suite T|Oak Ridge|Randolph County|MO|68371|United States|-6|single family| +7247|AAAAAAAAPEMBAAAA|905|Hill |Cir.|Suite S|Jamestown|Washington County|OR|96867|United States|-8|condo| +7248|AAAAAAAAAFMBAAAA|573|Jefferson |Road|Suite Q|Valley View|Starke County|IN|45124|United States|-5|apartment| +7249|AAAAAAAABFMBAAAA|942|Lake 14th|Circle|Suite 360|Fairview|Medina County|TX|75709|United States|-6|condo| +7250|AAAAAAAACFMBAAAA|680|1st Ninth|Court|Suite 350|Jackson|Gibson County|TN|39583|United States|-5|apartment| +7251|AAAAAAAADFMBAAAA|623|Center |Cir.|Suite N|Valley View|Isabella County|MI|45124|United States|-5|apartment| +7252|AAAAAAAAEFMBAAAA|881|Birch 4th|Cir.|Suite 290|Liberty|Monroe County|NY|13451|United States|-5|apartment| +7253|AAAAAAAAFFMBAAAA|421|Willow Cedar|Parkway|Suite X|Galena|Greene County|GA|34369|United States|-5|apartment| +7254|AAAAAAAAGFMBAAAA|472|Oak |Pkwy|Suite 150|Clifton|Washington County|MD|28014|United States|-5|condo| +7255|AAAAAAAAHFMBAAAA|431|Hillcrest Lake|Avenue|Suite 290|Mount Vernon|Baxter County|AR|78482|United States|-6|condo| +7256|AAAAAAAAIFMBAAAA|704|Church |Pkwy|Suite X|Glendale|Cumberland County|PA|13951|United States|-5|apartment| +7257|AAAAAAAAJFMBAAAA|894|Walnut Mill|Parkway|Suite 210|Springdale|Pike County|GA|38883|United States|-5|single family| +7258|AAAAAAAAKFMBAAAA|164|14th |RD|Suite 100|Fair Oaks|Oklahoma County|OK|76431|United States|-6|condo| +7259|AAAAAAAALFMBAAAA|511|Main Park|Way|Suite C|Farmington|Williamson County|TX|79145|United States|-6|condo| +7260|AAAAAAAAMFMBAAAA|50|Walnut Valley|Cir.|Suite 400|Spring Hill|Oktibbeha County|MS|56787|United States|-6|condo| +7261|AAAAAAAANFMBAAAA|731|Walnut North|Wy|Suite B|Phillips|Freestone County|TX|78617|United States|-6|single family| +7262|AAAAAAAAOFMBAAAA|840|Birch Highland|Avenue|Suite C|Midway|Floyd County|VA|21904|United States|-5|single family| +7263|AAAAAAAAPFMBAAAA|191|Jefferson |Avenue|Suite 80|Altamont|Dickinson County|IA|59387|United States|-6|condo| +7264|AAAAAAAAAGMBAAAA|209|4th First|Lane|Suite 240|Granite|Yuba County|CA|96284|United States|-8|condo| +7265|AAAAAAAABGMBAAAA|349|Elm Fifth|Drive|Suite 280|Sugar Hill|Leon County|TX|75114|United States|-6|apartment| +7266|AAAAAAAACGMBAAAA|853|3rd Oak|Blvd|Suite 240|Riverside|Obion County|TN|39231|United States|-6|apartment| +7267|AAAAAAAADGMBAAAA|178|12th 11th|Boulevard|Suite W|Springfield|Dona Ana County|NM|89303|United States|-7|single family| +7268|AAAAAAAAEGMBAAAA|792|Walnut Jefferson|Ln|Suite O|Woodville|Coles County|IL|64289|United States|-6|condo| +7269|AAAAAAAAFGMBAAAA|809|3rd Green|Road|Suite D|Woodlawn|Geary County|KS|64098|United States|-6|condo| +7270|AAAAAAAAGGMBAAAA|336|Madison 8th|Lane|Suite P|Waterloo|Otero County|CO|81675|United States|-7|single family| +7271|AAAAAAAAHGMBAAAA|514|Cedar |Boulevard|Suite W|Pleasant Valley|Slope County|ND|52477|United States|-6|apartment| +7272|AAAAAAAAIGMBAAAA|371|Fourth 6th|Dr.|Suite 400|Lakewood|Yuma County|AZ|88877|United States|-7|single family| +7273|AAAAAAAAJGMBAAAA|974|10th Park|Cir.|Suite 180|Concord|Brooks County|GA|34107|United States|-5|condo| +7274|AAAAAAAAKGMBAAAA|766|Sycamore Spring|Boulevard|Suite S|Brownsville|Morgan County|IL|69310|United States|-6|condo| +7275|AAAAAAAALGMBAAAA|617|Green Third|Road|Suite 40|Westgate|Bedford County|TN|32366|United States|-5|single family| +7276|AAAAAAAAMGMBAAAA|395|Main Ridge|Court|Suite I|Midway|Richmond County|NC|21904|United States|-5|apartment| +7277|AAAAAAAANGMBAAAA|981|Miller Poplar|Parkway|Suite 160|Stringtown|Lake County|SD|50162|United States|-7|condo| +7278|AAAAAAAAOGMBAAAA|340|Lee |Avenue|Suite 60|Parkwood|Modoc County|CA|91669|United States|-8|condo| +7279|AAAAAAAAPGMBAAAA|637|Lincoln |Ln|Suite 20|Lakeview|Henry County|IL|68579|United States|-6|condo| +7280|AAAAAAAAAHMBAAAA|703|2nd 13th|Ln|Suite W|Jackson|Belknap County|NH|09583|United States|-5|single family| +7281|AAAAAAAABHMBAAAA|585|Locust 13th|Wy|Suite Y|Five Points|San Diego County|CA|96098|United States|-8|apartment| +7282|AAAAAAAACHMBAAAA|690|8th |Boulevard|Suite 140|Red Hill|King and Queen County|VA|24338|United States|-5|condo| +7283|AAAAAAAADHMBAAAA|595|Chestnut South|Parkway|Suite 30|Highland|Lowndes County|MS|59454|United States|-6|single family| +7284|AAAAAAAAEHMBAAAA||Main |RD||Highland||CA||||| +7285|AAAAAAAAFHMBAAAA|530|Third Broadway|ST|Suite M|Pine Grove|Falls Church city|VA|24593|United States|-5|condo| +7286|AAAAAAAAGHMBAAAA|239|Poplar |Boulevard|Suite 490|Lakewood|Baylor County|TX|78877|United States|-6|apartment| +7287|AAAAAAAAHHMBAAAA|972|11th |Drive|Suite 60|Farmington|Lynchburg city|VA|29145|United States|-5|single family| +7288|AAAAAAAAIHMBAAAA|849|Woodland |Circle|Suite 190|Bunker Hill|Ashe County|NC|20150|United States|-5|apartment| +7289|AAAAAAAAJHMBAAAA|917|11th Center|Road|Suite 320|Valley View|Windsor County|VT|05724|United States|-5|single family| +7290|AAAAAAAAKHMBAAAA|415|Broadway |Drive|Suite V|Belmont|Yates County|NY|10191|United States|-5|single family| +7291|AAAAAAAALHMBAAAA|416|South |Dr.|Suite 210|Providence|Lac qui Parle County|MN|56614|United States|-6|single family| +7292|AAAAAAAAMHMBAAAA|34|4th |Ave|Suite B|Lincoln|McPherson County|SD|51289|United States|-7|apartment| +7293|AAAAAAAANHMBAAAA|872|Franklin |Court|Suite S|Arthur|Jefferson County|IN|45965|United States|-5|condo| +7294|AAAAAAAAOHMBAAAA|617|Oak Park|Way|Suite I|Forestville|White Pine County|NV|83027|United States|-8|apartment| +7295|AAAAAAAAPHMBAAAA|994|Spruce |Lane|Suite 140|Cedar Grove|Van Buren County|MI|40411|United States|-5|condo| +7296|AAAAAAAAAIMBAAAA|803|Fourth |Drive|Suite Y|Sharon|Lincoln County|NE|64029|United States|-7|apartment| +7297|AAAAAAAABIMBAAAA|973|Davis |Court|Suite R|Enterprise|Burke County|NC|21757|United States|-5|condo| +7298|AAAAAAAACIMBAAAA|289|Highland |Street|Suite 150|Mount Zion|Douglas County|IL|68054|United States|-6|condo| +7299|AAAAAAAADIMBAAAA|597|7th Johnson|Blvd|Suite W|Deerfield|Humboldt County|IA|59840|United States|-6|single family| +7300|AAAAAAAAEIMBAAAA|93|6th |Circle|Suite 150|Fairview|Washington County|KS|65709|United States|-6|single family| +7301|AAAAAAAAFIMBAAAA|193|2nd Third|Ln|Suite 480|Oak Grove|Leslie County|KY|48370|United States|-5|apartment| +7302|AAAAAAAAGIMBAAAA|747|Jackson |Street|Suite V|Bunker Hill|Baxter County|AR|70150|United States|-6|apartment| +7303|AAAAAAAAHIMBAAAA|542|Green |Pkwy|Suite 350|Woodland|La Plata County|CO|84854|United States|-7|single family| +7304|AAAAAAAAIIMBAAAA|793|5th |Court|Suite N|Riverside|Jefferson County|GA|39231|United States|-5|single family| +7305|AAAAAAAAJIMBAAAA|269|8th |Cir.|Suite T|Granite|Pulaski County|KY|46284|United States|-5|condo| +7306|AAAAAAAAKIMBAAAA|92|Sunset |Avenue|Suite H|Friendship|Atascosa County|TX|74536|United States|-6|condo| +7307|AAAAAAAALIMBAAAA|386|Lee |Boulevard|Suite 250|Bunker Hill|Hardin County|TN|30150|United States|-5|single family| +7308|AAAAAAAAMIMBAAAA|858|Main |Road|Suite 0|Oakdale|Green Lake County|WI|59584|United States|-6|condo| +7309|AAAAAAAANIMBAAAA|591|View |Ln|Suite 30|Pleasant Valley|Hillsborough County|NH|03077|United States|-5|condo| +7310|AAAAAAAAOIMBAAAA|378|West Center|Ln|Suite U|Harmony|Eddy County|NM|85804|United States|-7|apartment| +7311|AAAAAAAAPIMBAAAA|581|Highland South|RD|Suite 170|Altamont|Coos County|OR|99387|United States|-8|apartment| +7312|AAAAAAAAAJMBAAAA|363|Miller |Ln|Suite N|New Hope|Lubbock County|TX|79431|United States|-6|apartment| +7313|AAAAAAAABJMBAAAA||6th Ridge|Ct.|||Pershing County|||United States|-8|| +7314|AAAAAAAACJMBAAAA|860|North |Street|Suite E|Edgewood|Iron County|WI|50069|United States|-6|apartment| +7315|AAAAAAAADJMBAAAA|278|North |Pkwy|Suite 360|Sulphur Springs|Miller County|GA|38354|United States|-5|single family| +7316|AAAAAAAAEJMBAAAA|578|Birch |Pkwy|Suite W|Oakland|Roger Mills County|OK|79843|United States|-6|apartment| +7317|AAAAAAAAFJMBAAAA|337|Spring |Ave|Suite O|Woodlawn|Whitley County|IN|44098|United States|-5|single family| +7318|AAAAAAAAGJMBAAAA|547|Willow |Road|Suite N|Hamilton|Auglaize County|OH|42808|United States|-5|apartment| +7319|AAAAAAAAHJMBAAAA|762|Maple |Road|Suite 140|Clifton|Hickman County|KY|48014|United States|-6|single family| +7320|AAAAAAAAIJMBAAAA|403|Hill |Road|Suite 430|Bunker Hill|Caswell County|NC|20150|United States|-5|single family| +7321|AAAAAAAAJJMBAAAA|299|Hickory |Street|Suite 80|Pleasant Valley|Door County|WI|52477|United States|-6|condo| +7322|AAAAAAAAKJMBAAAA|199|11th Laurel|Ln|Suite Y|Jamestown|Lane County|OR|96867|United States|-8|condo| +7323|AAAAAAAALJMBAAAA|644|Third Hillcrest|Way|Suite 450|Fairview|Griggs County|ND|55709|United States|-6|single family| +7324|AAAAAAAAMJMBAAAA|539|5th |Road|Suite G|Avoca|Leake County|MS|50540|United States|-6|apartment| +7325|AAAAAAAANJMBAAAA|54|Sunset |Pkwy|Suite 280|Franklin|Prince Edward County|VA|29101|United States|-5|apartment| +7326|AAAAAAAAOJMBAAAA|444|Lake Woodland|Parkway|Suite R|Ashley|Hooker County|NE|64324|United States|-7|apartment| +7327|AAAAAAAAPJMBAAAA|942|Hillcrest River|Dr.|Suite 460|Brentwood|Windham County|CT|04788|United States|-5|single family| +7328|AAAAAAAAAKMBAAAA|712|Meadow 3rd|RD|Suite 230|Sulphur Springs|Teton County|MT|68354|United States|-7|condo| +7329|AAAAAAAABKMBAAAA|742|Locust |Court|Suite H|Walnut Grove|Garfield County|WA|97752|United States|-8|apartment| +7330|AAAAAAAACKMBAAAA|302|Maple Elm|ST|Suite X|Friendship|Dillon County|SC|24536|United States|-5|apartment| +7331|AAAAAAAADKMBAAAA|996|Hill Williams|Drive|Suite 30|Antioch|Douglas County|NE|68605|United States|-6|apartment| +7332|AAAAAAAAEKMBAAAA|377|Miller Hickory|Lane|Suite 370|Hillcrest|Waushara County|WI|53003|United States|-6|condo| +7333|AAAAAAAAFKMBAAAA|747|Valley |Dr.|Suite O|Bridgeport|Laurel County|KY|45817|United States|-5|single family| +7334|AAAAAAAAGKMBAAAA|399|North Laurel|Ave|Suite 50|Riverview|Pennington County|SD|59003|United States|-7|single family| +7335|AAAAAAAAHKMBAAAA|173|Ash |Ct.|Suite 360|Forest Hills|Harnett County|NC|29237|United States|-5|condo| +7336|AAAAAAAAIKMBAAAA|190|North Second|Circle|Suite Y|Stringtown|Carlton County|MN|50162|United States|-6|apartment| +7337|AAAAAAAAJKMBAAAA|402|Broadway |RD|Suite 220|Sunnyside|Tyler County|TX|71952|United States|-6|single family| +7338|AAAAAAAAKKMBAAAA|5|Fifth |Boulevard|Suite J|Friendship|Scott County|MN|54536|United States|-6|condo| +7339|AAAAAAAALKMBAAAA|643|1st |Ct.|Suite 430|Providence|Sherman County|OR|96614|United States|-8|single family| +7340|AAAAAAAAMKMBAAAA|442|Railroad |Avenue|Suite D|Wilton|Jim Hogg County|TX|76997|United States|-6|condo| +7341|AAAAAAAANKMBAAAA||||Suite 390||Beauregard Parish|||||single family| +7342|AAAAAAAAOKMBAAAA|449|West |Ct.|Suite W|Edgewood|Caldwell County|MO|60069|United States|-6|condo| +7343|AAAAAAAAPKMBAAAA|724|Adams |Lane|Suite 280|Pleasant Grove|Beaufort County|SC|24136|United States|-5|apartment| +7344|AAAAAAAAALMBAAAA|951|5th 10th|Street|Suite 470|Roy|Van Buren County|IA|50744|United States|-6|single family| +7345|AAAAAAAABLMBAAAA|386|Elevnth 9th|Pkwy|Suite 230|Newtown|Bradford County|PA|11749|United States|-5|apartment| +7346|AAAAAAAACLMBAAAA|70|Walnut Hill|Way|Suite B|Gilmore|Nolan County|TX|75464|United States|-6|single family| +7347|AAAAAAAADLMBAAAA|598|Highland Chestnut|Way|Suite 60|Superior|Pike County|PA|12562|United States|-5|apartment| +7348|AAAAAAAAELMBAAAA|55|Church |Circle|Suite 30|Newtown|Lyon County|IA|51749|United States|-6|apartment| +7349|AAAAAAAAFLMBAAAA|786|First |Avenue|Suite H|Hopewell|Polk County|TN|30587|United States|-6|single family| +7350|AAAAAAAAGLMBAAAA|777|Church East|Avenue|Suite 480|Lakeview|Fairfield County|OH|48579|United States|-5|condo| +7351|AAAAAAAAHLMBAAAA|828|Second |Ave|Suite 140|Thompsonville|Logan County|IL|69651|United States|-6|apartment| +7352|AAAAAAAAILMBAAAA|878|Ridge Jackson|Avenue|Suite 240|Woodland|Searcy County|AR|74854|United States|-6|single family| +7353|AAAAAAAAJLMBAAAA|747|Spruce Spring|Parkway|Suite 290|Union|Hopewell city|VA|28721|United States|-5|condo| +7354|AAAAAAAAKLMBAAAA|||||Bay View|Graves County|KY|46457||-6|| +7355|AAAAAAAALLMBAAAA|968|Willow Washington|Parkway|Suite T|Pleasant Valley|Fayette County|IL|62477|United States|-6|condo| +7356|AAAAAAAAMLMBAAAA|329|Wilson |Street|Suite P|Farmington|Chippewa County|MN|59145|United States|-6|apartment| +7357|AAAAAAAANLMBAAAA|797|8th |Circle|Suite 100|Jamestown|Marion County|FL|36867|United States|-5|condo| +7358|AAAAAAAAOLMBAAAA|775|6th |Avenue|Suite 360|Deerfield|Rankin County|MS|59840|United States|-6|condo| +7359|AAAAAAAAPLMBAAAA|429|Locust |RD|Suite S|Oak Grove|Northumberland County|VA|28370|United States|-5|single family| +7360|AAAAAAAAAMMBAAAA|827|Mill |||Sulphur Springs|||||-8|| +7361|AAAAAAAABMMBAAAA|470|Hill |Drive|Suite 480|Riverdale|Coahoma County|MS|59391|United States|-6|apartment| +7362|AAAAAAAACMMBAAAA|888|Lake |Blvd|Suite 160|Welcome|Monroe County|FL|36386|United States|-5|condo| +7363|AAAAAAAADMMBAAAA|820|Lake Franklin|Road|Suite T|Jackson|Oneida County|WI|59583|United States|-6|condo| +7364|AAAAAAAAEMMBAAAA|811|1st Wilson|||Marion|||30399||-5|| +7365|AAAAAAAAFMMBAAAA|17|Cedar Spruce|Street|Suite 170|Highland|Jefferson County|WA|99454|United States|-8|apartment| +7366|AAAAAAAAGMMBAAAA|906|Smith Cedar|Parkway|Suite 90|Newtown|Buffalo County|WI|51749|United States|-6|apartment| +7367|AAAAAAAAHMMBAAAA|694|Sixth Lee|Dr.|Suite 310|Texas|Richmond city|VA|23342|United States|-5|condo| +7368|AAAAAAAAIMMBAAAA|752|Main Green|ST|Suite 140|Plainview|Dickens County|TX|73683|United States|-6|single family| +7369|AAAAAAAAJMMBAAAA|990|Cedar |Pkwy|Suite 380|Oakdale|Sherman County|NE|69584|United States|-7|apartment| +7370|AAAAAAAAKMMBAAAA|280|East Jefferson|Ct.|Suite S|Newport|Chester County|PA|11521|United States|-5|apartment| +7371|AAAAAAAALMMBAAAA|413|Second |Circle|Suite 350|Shady Grove|Modoc County|CA|92812|United States|-8|condo| +7372|AAAAAAAAMMMBAAAA|439|5th Hill|Ave|Suite 420|Lebanon|Midland County|TX|72898|United States|-6|condo| +7373|AAAAAAAANMMBAAAA|308|2nd |Boulevard|Suite 330|Springdale|Craighead County|AR|78883|United States|-6|condo| +7374|AAAAAAAAOMMBAAAA|777|View |Road|Suite 260|Brownsville|Prince William County|VA|29310|United States|-5|single family| +7375|AAAAAAAAPMMBAAAA|344|14th Elm|Parkway|Suite 100|Bunker Hill|Sibley County|MN|50150|United States|-6|apartment| +7376|AAAAAAAAANMBAAAA|212|4th |Pkwy|Suite H|Salem|Guthrie County|IA|58048|United States|-6|apartment| +7377|AAAAAAAABNMBAAAA|267|12th |ST|Suite I|Murphy|Newberry County|SC|22105|United States|-5|apartment| +7378|AAAAAAAACNMBAAAA|53|Washington Park|ST|Suite 480|Nottingham|Saline County|MO|64074|United States|-6|condo| +7379|AAAAAAAADNMBAAAA|197|North North|Avenue|Suite 230|Mount Zion|Bristol County|RI|08654|United States|-5|condo| +7380|AAAAAAAAENMBAAAA|770|6th 11th|Pkwy|Suite J|Clifton|Collingsworth County|TX|78014|United States|-6|condo| +7381|AAAAAAAAFNMBAAAA|68|12th |Ln|Suite 260|Roy|Lincoln County|WV|20744|United States|-5|condo| +7382|AAAAAAAAGNMBAAAA|862|Main Main|Court|Suite D|Farmington|Stokes County|NC|29145|United States|-5|single family| +7383|AAAAAAAAHNMBAAAA|820|Central |Boulevard|Suite 400|Oakdale|Warren County|OH|49584|United States|-5|apartment| +7384|AAAAAAAAINMBAAAA|616|1st 14th|Lane|Suite Q|Salem|Palo Pinto County|TX|78048|United States|-6|single family| +7385|AAAAAAAAJNMBAAAA|955|Park Maple|Ave|Suite E|Hopewell|Carbon County|PA|10587|United States|-5|condo| +7386|AAAAAAAAKNMBAAAA|571|Center |Ave|Suite U|Pleasant Grove|Santa Clara County|CA|94136|United States|-8|condo| +7387|AAAAAAAALNMBAAAA|233|Jefferson |Circle|Suite 40|Walnut Grove|Union County|FL|37752|United States|-5|condo| +7388|AAAAAAAAMNMBAAAA|772|Lincoln |Way|Suite S|Hamilton|Hardin County|IA|52808|United States|-6|condo| +7389|AAAAAAAANNMBAAAA|105|13th |Parkway|Suite 250|Red Hill|Preston County|WV|24338|United States|-5|condo| +7390|AAAAAAAAONMBAAAA|505|9th Broadway|Lane|Suite 400|Lakeside|Fluvanna County|VA|29532|United States|-5|single family| +7391|AAAAAAAAPNMBAAAA||Second |||Kingston|Franklin County|NC|24975|United States|-5|| +7392|AAAAAAAAAOMBAAAA|181|Railroad Jefferson|Parkway|Suite 470|Riverside|Gloucester County|NJ|09831|United States|-5|single family| +7393|AAAAAAAABOMBAAAA|348|12th North|Circle|Suite I|Sulphur Springs|Pueblo County|CO|88354|United States|-7|condo| +7394|AAAAAAAACOMBAAAA|952|Oak |Ct.|Suite C|Clinton|Williamsburg city|VA|28222|United States|-5|apartment| +7395|AAAAAAAADOMBAAAA|470|North |ST|Suite X|Edgewood|Ector County|TX|70069|United States|-6|apartment| +7396|AAAAAAAAEOMBAAAA|220|Sycamore |Wy|Suite 340|Stringtown|Jefferson County|WA|90162|United States|-8|single family| +7397|AAAAAAAAFOMBAAAA|701|Cedar |Circle|Suite 340|Mountain View|Ashland County|OH|44466|United States|-5|condo| +7398|AAAAAAAAGOMBAAAA|945|Dogwood Seventh|Pkwy|Suite O|Brownsville|Moffat County|CO|89310|United States|-7|single family| +7399|AAAAAAAAHOMBAAAA|788|Pine Poplar|Court|Suite G|Five Points|Kingsbury County|SD|56098|United States|-7|single family| +7400|AAAAAAAAIOMBAAAA|934|River |ST|Suite 170|Greenville|Clearwater County|MN|51387|United States|-6|condo| +7401|AAAAAAAAJOMBAAAA|632|9th |Lane|Suite 70|Fairfield|Vermilion Parish|LA|76192|United States|-6|condo| +7402|AAAAAAAAKOMBAAAA|593|8th |Ln|Suite A|Crossroads|Mineral County|MT|60534|United States|-7|single family| +7403|AAAAAAAALOMBAAAA|822|River Birch|Ln|Suite H|Fairview|Rock County|MN|55709|United States|-6|apartment| +7404|AAAAAAAAMOMBAAAA|907|View Seventh|Parkway|Suite 150|Glenwood|Mackinac County|MI|43511|United States|-5|apartment| +7405|AAAAAAAANOMBAAAA|588|Maple |Cir.|Suite K|Pleasant Hill|Garfield County|WA|93604|United States|-8|apartment| +7406|AAAAAAAAOOMBAAAA|32|First |Parkway|Suite T|Riverview|Bradford County|FL|39003|United States|-5|condo| +7407|AAAAAAAAPOMBAAAA|503|Birch Poplar|Road|Suite 110|Oak Grove|Maui County|HI|98370|United States|-10|single family| +7408|AAAAAAAAAPMBAAAA|1000|Mill |Cir.|Suite 180|Lenox|Cloud County|KS|61143|United States|-6|condo| +7409|AAAAAAAABPMBAAAA|201|Ash Hill|Drive|Suite J|Brookwood|Gallatin County|KY|40965|United States|-6|condo| +7410|AAAAAAAACPMBAAAA|239|Church |Drive|Suite J|Pleasant Hill|Wyoming County|WV|23604|United States|-5|condo| +7411|AAAAAAAADPMBAAAA|841|Elm Cherry|Boulevard|Suite 270|Buena Vista|Wyoming County|WV|25752|United States|-5|condo| +7412|AAAAAAAAEPMBAAAA|106|3rd Railroad|Road|Suite 0|Oak Hill|Pinellas County|FL|37838|United States|-5|condo| +7413|AAAAAAAAFPMBAAAA|801|Main |Cir.|Suite W|Wilson|Knox County|IN|46971|United States|-5|single family| +7414|AAAAAAAAGPMBAAAA|180|Birch Locust|RD|Suite Y|Buena Vista|Carroll County|OH|45752|United States|-5|single family| +7415|AAAAAAAAHPMBAAAA|736|10th |Drive|Suite L|Friendship|Johnson County|KS|64536|United States|-6|apartment| +7416|AAAAAAAAIPMBAAAA|340|Birch |Ave|Suite 50|Harvey|Cleburne County|AR|75858|United States|-6|apartment| +7417|AAAAAAAAJPMBAAAA|741|Fourth |Wy|Suite Y|Mount Zion|Carson City|NV|88054|United States|-8|single family| +7418|AAAAAAAAKPMBAAAA|486|5th Elm|Wy|Suite A|Enterprise|Jasper County|MS|51757|United States|-6|single family| +7419|AAAAAAAALPMBAAAA|612|Franklin 2nd|Cir.|Suite E|Ashland|Lyon County|IA|54244|United States|-6|condo| +7420|AAAAAAAAMPMBAAAA|875|Wilson Central|Street|Suite K|Arlington|New London County|CT|07157|United States|-5|single family| +7421|AAAAAAAANPMBAAAA|792|Forest |Pkwy|Suite H|Pleasant Valley|San Miguel County|NM|82477|United States|-7|condo| +7422|AAAAAAAAOPMBAAAA|414|Lee Franklin|Blvd|Suite 140|Edgewood|Harrison County|MS|50069|United States|-6|single family| +7423|AAAAAAAAPPMBAAAA|108|Walnut Forest|Court|Suite S|Lincoln|Pike County|AR|71289|United States|-6|condo| +7424|AAAAAAAAAANBAAAA|256|Main Jefferson|Street|Suite 430|Wildwood|New Kent County|VA|26871|United States|-5|condo| +7425|AAAAAAAABANBAAAA|64|13th |Ln|Suite 240|Cedar Grove|Randolph County|IN|40411|United States|-5|condo| +7426|AAAAAAAACANBAAAA||||Suite 230|Greenville||GA||United States|-5|single family| +7427|AAAAAAAADANBAAAA|600|Green Church|Dr.|Suite P|Wilson|Union County|NC|26971|United States|-5|single family| +7428|AAAAAAAAEANBAAAA|602|Tenth Williams|Dr.|Suite M|Glendale|Mercer County|MO|63951|United States|-6|single family| +7429|AAAAAAAAFANBAAAA|454|Miller Smith|Blvd|Suite 240|Oakwood|Kalamazoo County|MI|40169|United States|-5|single family| +7430|AAAAAAAAGANBAAAA|405|Lee Park|Ave|Suite H|Montague|Taylor County|IA|54062|United States|-6|condo| +7431|AAAAAAAAHANBAAAA|525|3rd |Parkway|Suite S|Summit|Sullivan County|IN|40499|United States|-5|apartment| +7432|AAAAAAAAIANBAAAA|533|Hill Valley|Drive|Suite S|Mount Zion|Montgomery County|TX|78054|United States|-6|apartment| +7433|AAAAAAAAJANBAAAA|968|Main Poplar|Blvd|Suite 480|Reno|Washington County|MO|60344|United States|-6|single family| +7434|AAAAAAAAKANBAAAA|490|Maple Birch|Lane|Suite P|Springfield|Edgefield County|SC|29303|United States|-5|single family| +7435|AAAAAAAALANBAAAA|28|Main Pine|Way|Suite B|Greenfield|Kanawha County|WV|25038|United States|-5|apartment| +7436|AAAAAAAAMANBAAAA|484|Sunset Elm|Ct.|Suite K|Enterprise|Bradford County|FL|31757|United States|-5|apartment| +7437|AAAAAAAANANBAAAA|597|5th River|Ct.|Suite M|Lincoln|Lincoln County|WV|21289|United States|-5|single family| +7438|AAAAAAAAOANBAAAA|818|8th |Pkwy|Suite 490|Spring Hill|Lafourche Parish|LA|76787|United States|-6|apartment| +7439|AAAAAAAAPANBAAAA|579|5th |Wy|Suite 260|Johnson|Roanoke County|VA|29105|United States|-5|condo| +7440|AAAAAAAAABNBAAAA|489|College |Ave|Suite M|Jackson|Durham County|NC|29583|United States|-5|single family| +7441|AAAAAAAABBNBAAAA|242|Hill 1st|Wy|Suite 440|Stewart|Giles County|TN|38041|United States|-5|single family| +7442|AAAAAAAACBNBAAAA|527|Cedar Lake|Avenue|Suite R|Mount Pleasant|McLennan County|TX|71933|United States|-6|apartment| +7443|AAAAAAAADBNBAAAA|313|Franklin |Parkway|Suite I|Sawyer|Owyhee County|ID|86045|United States|-7|condo| +7444|AAAAAAAAEBNBAAAA|130|Forest |Circle|Suite 0|Springdale|Elmore County|ID|88883|United States|-7|apartment| +7445|AAAAAAAAFBNBAAAA|34|Cherry 5th|Way|Suite H|Summit|Florence County|WI|50499|United States|-6|condo| +7446|AAAAAAAAGBNBAAAA|408|Madison Lake|Cir.|Suite T|Mount Vernon|Crawford County|IA|58482|United States|-6|single family| +7447|AAAAAAAAHBNBAAAA|775|Church Fourth|Ln|Suite 290|Clinton|Stafford County|VA|28222|United States|-5|single family| +7448|AAAAAAAAIBNBAAAA|635|7th |Court|Suite A|Florence|Meade County|SD|53394|United States|-7|condo| +7449|AAAAAAAAJBNBAAAA|813|North 10th|Circle|Suite X|Shiloh|Cherokee County|TX|79275|United States|-6|single family| +7450|AAAAAAAAKBNBAAAA|281|Green |Cir.|Suite 60|Georgetown|Beaver County|PA|17057|United States|-5|condo| +7451|AAAAAAAALBNBAAAA|260|Maple Williams|Pkwy|Suite X|Fairview|White County|IL|65709|United States|-6|apartment| +7452|AAAAAAAAMBNBAAAA|464|Locust |Road|Suite 140|Woodland|Scott County|IN|44854|United States|-5|condo| +7453|AAAAAAAANBNBAAAA|175|13th Third|Street|Suite 200|Hillcrest|Palm Beach County|FL|33003|United States|-5|condo| +7454|AAAAAAAAOBNBAAAA|173|Williams South|Boulevard|Suite X|Five Points|Worth County|GA|36098|United States|-5|condo| +7455|AAAAAAAAPBNBAAAA|499|10th Church|Cir.|Suite 480|Saint Clair|Hancock County|TN|35294|United States|-5|single family| +7456|AAAAAAAAACNBAAAA|471|Mill Sunset|RD|Suite 410|Lakeside|Iowa County|WI|59532|United States|-6|condo| +7457|AAAAAAAABCNBAAAA|339|Third Cherry|Parkway|Suite O|Union Hill|Routt County|CO|87746|United States|-7|single family| +7458|AAAAAAAACCNBAAAA|804|Birch |Ln|Suite 60|Bridgeport|Atchison County|KS|65817|United States|-6|apartment| +7459|AAAAAAAADCNBAAAA|589|Fourth |Lane|Suite X|Shady Grove|Fairfield County|OH|42812|United States|-5|single family| +7460|AAAAAAAAECNBAAAA|109|Woodland |Court|Suite O|Concord|Van Buren County|IA|54107|United States|-6|condo| +7461|AAAAAAAAFCNBAAAA|304|Sycamore 1st|Ct.|Suite 0|Woodlawn|Porter County|IN|44098|United States|-5|apartment| +7462|AAAAAAAAGCNBAAAA|838|West |Drive|Suite 450|Springdale|Leflore County|MS|58883|United States|-6|single family| +7463|AAAAAAAAHCNBAAAA|82|Spruce 12th|ST|Suite 310|Pine Grove|Pembina County|ND|54593|United States|-6|single family| +7464|AAAAAAAAICNBAAAA|411|Cedar |Boulevard|Suite D|Woodland|Calhoun County|SC|24854|United States|-5|apartment| +7465|AAAAAAAAJCNBAAAA|562|Second |Blvd|Suite B|Macedonia|Red River County|TX|71087|United States|-6|single family| +7466|AAAAAAAAKCNBAAAA|997|Lincoln College|Drive|Suite T|Pleasant Valley|Franklin County|MS|52477|United States|-6|apartment| +7467|AAAAAAAALCNBAAAA|586|Williams |ST|Suite L|Florence|Jefferson County|TN|33394|United States|-6|apartment| +7468|AAAAAAAAMCNBAAAA|702|Jackson 1st|Drive|Suite G|Lakeview|McNairy County|TN|38579|United States|-6|condo| +7469|AAAAAAAANCNBAAAA|940|Ridge |Pkwy|Suite L|Green Acres|Beauregard Parish|LA|77683|United States|-6|condo| +7470|AAAAAAAAOCNBAAAA|434|Poplar River|Way|Suite P|Pleasant Hill|Lenoir County|NC|23604|United States|-5|single family| +7471|AAAAAAAAPCNBAAAA|870|11th |Lane|Suite P|Wayland|Walton County|GA|35115|United States|-5|apartment| +7472|AAAAAAAAADNBAAAA|931|Oak South|Ln|Suite F|Bridgeport|Union County|OR|95817|United States|-8|apartment| +7473|AAAAAAAABDNBAAAA|989|Maple Lake|Boulevard|Suite W|Forest Hills|Vermilion County|IL|69237|United States|-6|single family| +7474|AAAAAAAACDNBAAAA|903|Lee Fifth|Way|Suite 380|Centerville|Oakland County|MI|40059|United States|-5|apartment| +7475|AAAAAAAADDNBAAAA|77|Sixth |Street|Suite V|Proctor|Concordia Parish|LA|78140|United States|-6|apartment| +7476|AAAAAAAAEDNBAAAA|984|10th |Road|Suite 0|Maple Grove|Montgomery County|AR|78252|United States|-6|single family| +7477|AAAAAAAAFDNBAAAA|751|Fourth |Blvd|Suite 260|Lakewood|Marion County|WV|28877|United States|-5|apartment| +7478|AAAAAAAAGDNBAAAA|15|Main |Court|Suite U|Woodville|Coles County|IL|64289|United States|-6|apartment| +7479|AAAAAAAAHDNBAAAA|23|Church 10th|Way|Suite P|Lakeside|Barry County|MO|69532|United States|-6|single family| +7480|AAAAAAAAIDNBAAAA|827|Washington Williams|Pkwy|Suite F|White Oak|Todd County|KY|46668|United States|-5|condo| +7481|AAAAAAAAJDNBAAAA|||||||VA|27746|||| +7482|AAAAAAAAKDNBAAAA|282|Lincoln Mill|Drive|Suite F|Woodland|Camden County|NC|24854|United States|-5|condo| +7483|AAAAAAAALDNBAAAA|45|View |Boulevard|Suite G|Farmington|Otsego County|MI|49145|United States|-5|apartment| +7484|AAAAAAAAMDNBAAAA|676|||Suite 330||Bronx County|||||| +7485|AAAAAAAANDNBAAAA|407|South 8th|Court|Suite J|Lebanon|Lake County|OR|92898|United States|-8|single family| +7486|AAAAAAAAODNBAAAA|491|14th |Boulevard|Suite W|Mount Zion|Sullivan County|NH|08654|United States|-5|single family| +7487|AAAAAAAAPDNBAAAA|771|Third |Cir.|Suite K|Concord|Ogemaw County|MI|44107|United States|-5|apartment| +7488|AAAAAAAAAENBAAAA|475|Woodland Maple|Parkway|Suite W|Hamilton|McCone County|MT|62808|United States|-7|apartment| +7489|AAAAAAAABENBAAAA|374|Wilson 15th|Court|Suite 90|Mount Zion|Iron County|WI|58054|United States|-6|condo| +7490|AAAAAAAACENBAAAA|860||Street|||Ottawa County|||||| +7491|AAAAAAAADENBAAAA|767|5th Laurel|Parkway|Suite Q|Woodland|Montour County|PA|14854|United States|-5|apartment| +7492|AAAAAAAAEENBAAAA|955|Walnut |Road|Suite G|Adrian|Steele County|ND|53301|United States|-6|single family| +7493|AAAAAAAAFENBAAAA|152|River Second|Pkwy|Suite 280|White Oak|Huron County|MI|46668|United States|-5|single family| +7494|AAAAAAAAGENBAAAA|203|Ninth Elm|Street|Suite M|Peoria|Miller County|MO|69818|United States|-6|condo| +7495|AAAAAAAAHENBAAAA|742|Meadow |Wy|Suite B|Georgetown|Otoe County|NE|67057|United States|-7|single family| +7496|AAAAAAAAIENBAAAA|131|6th Washington|ST|Suite 30|Sunnyside|Gibson County|IN|41952|United States|-5|condo| +7497|AAAAAAAAJENBAAAA|623|Hickory Park|Ave|Suite 350|Lakewood|Lee County|IL|68877|United States|-6|condo| +7498|AAAAAAAAKENBAAAA|386|Sycamore |Circle|Suite 480|Mount Zion|Stokes County|NC|28054|United States|-5|condo| +7499|AAAAAAAALENBAAAA|476|Walnut Miller|Street|Suite 250|Oak Hill|Christian County|MO|67838|United States|-6|apartment| +7500|AAAAAAAAMENBAAAA|573|1st |Dr.|Suite G|Liberty|Clear Creek County|CO|83451|United States|-7|condo| +7501|AAAAAAAANENBAAAA|647|Laurel Washington|Boulevard|Suite 280|Greenfield|Ontonagon County|MI|45038|United States|-5|single family| +7502|AAAAAAAAOENBAAAA|433|2nd |Street|Suite 490|Clifton|Woods County|OK|78014|United States|-6|apartment| +7503|AAAAAAAAPENBAAAA|485|Cedar River|Ln|Suite 220|Blair|Karnes County|TX|75465|United States|-6|single family| +7504|AAAAAAAAAFNBAAAA|481|First |ST|Suite 100|Enterprise|Chatham County|NC|21757|United States|-5|apartment| +7505|AAAAAAAABFNBAAAA|668|Park |Wy|Suite X|Williamsville|Throckmorton County|TX|78754|United States|-6|single family| +7506|AAAAAAAACFNBAAAA|876|Smith 3rd|Boulevard|Suite 290|Five Points|Putnam County|MO|66098|United States|-6|apartment| +7507|AAAAAAAADFNBAAAA|8|Oak Sixth|Avenue|Suite D|Pine Grove|Pottawattamie County|IA|54593|United States|-6|single family| +7508|AAAAAAAAEFNBAAAA|15|Johnson Cherry|Ln|Suite T|Crossroads|Steele County|ND|50534|United States|-6|single family| +7509|AAAAAAAAFFNBAAAA|837|9th |Drive|Suite R|Hillcrest|Berkeley County|WV|23003|United States|-5|apartment| +7510|AAAAAAAAGFNBAAAA|207|Spring 5th|Circle|Suite 470|Valley View|Union County|IA|55124|United States|-6|condo| +7511|AAAAAAAAHFNBAAAA|571|Lake |RD|Suite 430|Newtown|Lincoln County|NV|81749|United States|-8|single family| +7512|AAAAAAAAIFNBAAAA|177|Franklin 6th|Way|Suite 60|Highland|Hickman County|KY|49454|United States|-6|single family| +7513|AAAAAAAAJFNBAAAA|887|Third Sycamore|Avenue|Suite U|Walnut Grove|Richland County|MT|67752|United States|-7|condo| +7514|AAAAAAAAKFNBAAAA|387|Second Church|Road|Suite 490|Highland Park|Lewis and Clark County|MT|66534|United States|-7|apartment| +7515|AAAAAAAALFNBAAAA|938|Chestnut South|Cir.|Suite 30|Mount Olive|Northampton County|NC|28059|United States|-5|apartment| +7516|AAAAAAAAMFNBAAAA|305|Green |Court|Suite 210|Lewis|Warren County|GA|37066|United States|-5|condo| +7517|AAAAAAAANFNBAAAA|963|9th |Drive|Suite J|Pine Grove|Carbon County|MT|64593|United States|-7|single family| +7518|AAAAAAAAOFNBAAAA|422|4th Spruce|ST|Suite E|Antioch|Tripp County|SD|58605|United States|-7|apartment| +7519|AAAAAAAAPFNBAAAA|||Circle|Suite A|Jackson|McCurtain County|OK||United States||apartment| +7520|AAAAAAAAAGNBAAAA|807|River Pine|Road|Suite 440|Greenfield|Moody County|SD|55038|United States|-7|apartment| +7521|AAAAAAAABGNBAAAA|279|Jefferson 5th|Circle|Suite V|Greenville|District of Columbia|DC|21387|United States|-5|single family| +7522|AAAAAAAACGNBAAAA|462|Meadow Broadway|Cir.|Suite 330|Oak Grove|Greenbrier County|WV|28370|United States|-5|single family| +7523|AAAAAAAADGNBAAAA|219|West |RD|Suite U|Springfield|Washington County|RI|09903|United States|-5|condo| +7524|AAAAAAAAEGNBAAAA|10|11th |Boulevard|Suite C|Marion|Fayette County|TN|30399|United States|-5|single family| +7525|AAAAAAAAFGNBAAAA|215|Fifth Broadway|Ln|Suite C|Lakeside|New Madrid County|MO|69532|United States|-6|condo| +7526|AAAAAAAAGGNBAAAA|354|Main |Blvd|Suite N|Plainview|Henderson County|TX|73683|United States|-6|single family| +7527|AAAAAAAAHGNBAAAA|559|Maple Hillcrest|RD|Suite V|Greenville|Menifee County|KY|41387|United States|-5|single family| +7528|AAAAAAAAIGNBAAAA|708|Railroad Lake|Drive||||VA|29431|United States|-5|| +7529|AAAAAAAAJGNBAAAA|300|2nd Cedar|Ln|Suite 100|Greenfield|Fulton County|IL|65038|United States|-6|single family| +7530|AAAAAAAAKGNBAAAA|924|14th 3rd|Boulevard|Suite 180|Red Hill|Indiana County|PA|14338|United States|-5|single family| +7531|AAAAAAAALGNBAAAA|99|Ridge |Boulevard|Suite 110|Concord|New Castle County|DE|14107|United States|-5|apartment| +7532|AAAAAAAAMGNBAAAA|431|Miller Park|Dr.|Suite 130|Stringtown|Livingston County|MI|40162|United States|-5|condo| +7533|AAAAAAAANGNBAAAA|825|Spring 5th|Road|Suite V|Carpenter|Labette County|KS|61147|United States|-6|condo| +7534|AAAAAAAAOGNBAAAA|913|Hill 5th|Road|Suite R|Union|Hennepin County|MN|58721|United States|-6|apartment| +7535|AAAAAAAAPGNBAAAA|226|6th |Way|Suite K|Liberty|Slope County|ND|53451|United States|-6|condo| +7536|AAAAAAAAAHNBAAAA|243|Lakeview 7th|Ave|Suite 70|Woodland|Manatee County|FL|34854|United States|-5|apartment| +7537|AAAAAAAABHNBAAAA|662|Forest 15th|Dr.|Suite 390|Midway|Throckmorton County|TX|71904|United States|-6|apartment| +7538|AAAAAAAACHNBAAAA|797|Pine |Lane|Suite 300|Valley View|Butler County|OH|45124|United States|-5|single family| +7539|AAAAAAAADHNBAAAA|757|5th 3rd|Ct.|Suite L|Whispering Pines|Grant County|OK|77609|United States|-6|condo| +7540|AAAAAAAAEHNBAAAA|946|West |Pkwy|Suite W|Somerville|Scott County|TN|37783|United States|-6|condo| +7541|AAAAAAAAFHNBAAAA|960|12th |Street|Suite Q|Friendship|Union County|NM|84536|United States|-7|apartment| +7542|AAAAAAAAGHNBAAAA|320|Willow Hickory|Ct.|Suite 40|Riverside|Roscommon County|MI|49231|United States|-5|single family| +7543|AAAAAAAAHHNBAAAA|188|10th Ridge|Road|Suite P|Superior|Gallatin County|KY|42562|United States|-6|single family| +7544|AAAAAAAAIHNBAAAA|285|11th Pine|ST|Suite G|Weldon|Bibb County|GA|36277|United States|-5|condo| +7545|AAAAAAAAJHNBAAAA|562|Tenth |Wy|Suite V|Sunnyside|Clinton County|IL|61952|United States|-6|apartment| +7546|AAAAAAAAKHNBAAAA|865|Lincoln |Road|Suite 120|Edgewood|Cross County|AR|70069|United States|-6|single family| +7547|AAAAAAAALHNBAAAA|739|Spring Second|Way|Suite U|Glendale|Hopkins County|TX|73951|United States|-6|condo| +7548|AAAAAAAAMHNBAAAA|275|Sunset |Court|Suite 350|Crossroads|McDonough County|IL|60534|United States|-6|apartment| +7549|AAAAAAAANHNBAAAA|225|Willow Center|Ct.|Suite 0|Maple Grove|Columbia County|WA|98252|United States|-8|single family| +7550|AAAAAAAAOHNBAAAA|204|Sunset Eigth|Parkway|Suite 490|Maple Grove|Carlton County|MN|58252|United States|-6|condo| +7551|AAAAAAAAPHNBAAAA|353|5th |Street|Suite 340|Lakewood|Bergen County|NJ|09477|United States|-5|condo| +7552|AAAAAAAAAINBAAAA|935|Washington |Circle|Suite V|Stringtown|Sussex County|VA|20162|United States|-5|apartment| +7553|AAAAAAAABINBAAAA|526|11th River|Boulevard|Suite S|Ashland|Johnson County|TX|74244|United States|-6|apartment| +7554|AAAAAAAACINBAAAA|212|West |ST|Suite I|Friendship|Fayette County|KY|44536|United States|-6|apartment| +7555|AAAAAAAADINBAAAA|209|Park Lee|ST|Suite H|La Grange|Henry County|KY|47941|United States|-6|condo| +7556|AAAAAAAAEINBAAAA|916|9th |RD|Suite 450|Oak Ridge|Wilson County|TN|38371|United States|-5|single family| +7557|AAAAAAAAFINBAAAA|292|15th |Way|Suite 190|Indian Village|James City County|VA|21075|United States|-5|single family| +7558|AAAAAAAAGINBAAAA|75|Third |Circle|Suite 60|Friendship|Coos County|OR|94536|United States|-8|apartment| +7559|AAAAAAAAHINBAAAA|62|Lincoln Pine|Avenue|Suite 390|Saint Johns|Miller County|GA|35717|United States|-5|condo| +7560|AAAAAAAAIINBAAAA|435|Poplar |Ct.|Suite 0|Simpson|Scioto County|OH|49865|United States|-5|apartment| +7561|AAAAAAAAJINBAAAA|833|Main |Pkwy|Suite 270|Forest Hills|Wood County|TX|79237|United States|-6|condo| +7562|AAAAAAAAKINBAAAA|116|Woodland Elm|Avenue|Suite R|Unionville|Johnson County|TX|71711|United States|-6|single family| +7563|AAAAAAAALINBAAAA|383|6th 2nd|Cir.|Suite O|Deerfield|Searcy County|AR|79840|United States|-6|condo| +7564|AAAAAAAAMINBAAAA|530|Fourth |Street|||||||-7|apartment| +7565|AAAAAAAANINBAAAA|51|Maple Lake|Wy|Suite 330|Plainview|Grant County|SD|53683|United States|-6|single family| +7566|AAAAAAAAOINBAAAA|32|Cedar Lee|RD|Suite 70|Royal|Columbia County|AR|75819|United States|-6|apartment| +7567|AAAAAAAAPINBAAAA|573|8th |Dr.|Suite 380|Friendship|Hoke County|NC|24536|United States|-5|condo| +7568|AAAAAAAAAJNBAAAA|302|Cedar Lake|Court|Suite 160|Glenwood|Jefferson County|CO|83511|United States|-7|condo| +7569|AAAAAAAABJNBAAAA|187|East |Parkway|Suite 130|Midway|Meagher County|MT|61904|United States|-7|condo| +7570|AAAAAAAACJNBAAAA|575|Cedar |Drive|Suite K|Jackson|Davidson County|NC|29583|United States|-5|single family| +7571|AAAAAAAADJNBAAAA|246|Third Park|Way|Suite K|Brownsville|Marion County|TX|79310|United States|-6|apartment| +7572|AAAAAAAAEJNBAAAA|507|Oak Center|Road|Suite 390|Shiloh|Pickens County|GA|39275|United States|-5|apartment| +7573|AAAAAAAAFJNBAAAA|755|Fourth Walnut|Boulevard|Suite T|Wildwood|Adams County|PA|16871|United States|-5|apartment| +7574|AAAAAAAAGJNBAAAA|445|Sycamore |Ln|Suite 340|Jamestown|Rock County|MN|56867|United States|-6|single family| +7575|AAAAAAAAHJNBAAAA|745|Green Willow|Street|Suite 70|Shady Grove|Augusta County|VA|22812|United States|-5|single family| +7576|AAAAAAAAIJNBAAAA|||Parkway||||IL|63003||-6|single family| +7577|AAAAAAAAJJNBAAAA|399|Park College|Pkwy|Suite A|Edgewood|Brown County|MN|50069|United States|-6|apartment| +7578|AAAAAAAAKJNBAAAA|702|Mill |Cir.|Suite N|Waterloo|Chittenden County|VT|02275|United States|-5|condo| +7579|AAAAAAAALJNBAAAA|281|12th Locust|Boulevard|Suite 300|Liberty|Switzerland County|IN|43451|United States|-5|condo| +7580|AAAAAAAAMJNBAAAA|777|South Madison|Blvd|Suite V|Newtown|Orange County|TX|71749|United States|-6|condo| +7581|AAAAAAAANJNBAAAA|384|North |Ave|Suite D|Mount Pleasant|Macon County|MO|61933|United States|-6|apartment| +7582|AAAAAAAAOJNBAAAA|406|Hill |Pkwy|Suite 70|Greenfield|Chester County|TN|35038|United States|-5|single family| +7583|AAAAAAAAPJNBAAAA|14|Fourth Oak|ST|Suite K|Fairfield|Bourbon County|KY|46192|United States|-6|condo| +7584|AAAAAAAAAKNBAAAA|509|Broadway North|Pkwy|Suite 430|Oakland|Apache County|AZ|89843|United States|-7|apartment| +7585|AAAAAAAABKNBAAAA|640|Jackson West|Dr.|Suite I|Harmony|York County|SC|25804|United States|-5|condo| +7586|AAAAAAAACKNBAAAA|597|Smith |Drive|Suite 130|Lincoln|Walton County|FL|31289|United States|-5|condo| +7587|AAAAAAAADKNBAAAA|413|Cedar |Ln|Suite E|Union|Hardin County|KY|48721|United States|-6|apartment| +7588|AAAAAAAAEKNBAAAA|782|9th |Dr.|Suite 240|Sugar Hill|Kerr County|TX|75114|United States|-6|single family| +7589|AAAAAAAAFKNBAAAA|349|Locust |Way|Suite A|Pine Grove|Petroleum County|MT|64593|United States|-7|apartment| +7590|AAAAAAAAGKNBAAAA|560|Green 5th|Cir.|Suite 280|Antioch|Eagle County|CO|88605|United States|-7|single family| +7591|AAAAAAAAHKNBAAAA|29|Park Wilson|Road|Suite 450|Lakeside|Hayes County|NE|69532|United States|-6|condo| +7592|AAAAAAAAIKNBAAAA|522|Wilson |Avenue|Suite X|Summit|Elmore County|AL|30499|United States|-6|apartment| +7593|AAAAAAAAJKNBAAAA|838|Lincoln 9th|Cir.|Suite J|Union City|Chenango County|NY|18087|United States|-5|single family| +7594|AAAAAAAAKKNBAAAA|714|7th |Ave|Suite F|Woodbury|Gila County|AZ|84489|United States|-7|condo| +7595|AAAAAAAALKNBAAAA|897|Valley Madison|Wy|Suite 120|Riverside|Adams County|IN|49231|United States|-5|single family| +7596|AAAAAAAAMKNBAAAA|659|Third |Way|Suite X|Hopewell|Staunton city|VA|20587|United States|-5|single family| +7597|AAAAAAAANKNBAAAA|722|Cedar |Wy|Suite G|Salem|Cross County|AR|78048|United States|-6|apartment| +7598|AAAAAAAAOKNBAAAA|562|Church 3rd|Avenue|Suite V|Mount Olive|Macon County|GA|38059|United States|-5|single family| +7599|AAAAAAAAPKNBAAAA|178|Hill Green|Cir.|Suite F|Jackson|Nottoway County|VA|29583|United States|-5|single family| +7600|AAAAAAAAALNBAAAA|131|4th |Street|Suite N|Forest Hills|Ware County|GA|39237|United States|-5|condo| +7601|AAAAAAAABLNBAAAA|13|Washington 11th|Blvd|Suite D|Shiloh|Sedgwick County|CO|89275|United States|-7|condo| +7602|AAAAAAAACLNBAAAA|154|Hickory Chestnut|Pkwy|Suite 100|Forest Hills|Tyler County|TX|79237|United States|-6|condo| +7603|AAAAAAAADLNBAAAA|85|Hickory |Way|Suite M|Riverdale|Sampson County|NC|29391|United States|-5|single family| +7604|AAAAAAAAELNBAAAA||15th Johnson|Parkway|Suite H||||39003||-5|| +7605|AAAAAAAAFLNBAAAA|638|Madison |Drive|Suite 280|Spring Valley|Lawrence County|TN|36060|United States|-6|single family| +7606|AAAAAAAAGLNBAAAA|540|Highland |RD|Suite Q|Galena|Marion County|IA|54369|United States|-6|single family| +7607|AAAAAAAAHLNBAAAA|308|Willow Lincoln|ST|Suite 410|Lakewood|Marion County|TX|78877|United States|-6|single family| +7608|AAAAAAAAILNBAAAA|644|4th Sixth|ST|Suite 60|Riverview|Routt County|CO|89003|United States|-7|apartment| +7609|AAAAAAAAJLNBAAAA|879|Railroad Ridge|Avenue|Suite 300|Oakdale|Custer County|OK|79584|United States|-6|single family| +7610|AAAAAAAAKLNBAAAA||Park |Drive||Union|Utah County||||-7|condo| +7611|AAAAAAAALLNBAAAA|613|Cedar Ash|Parkway|Suite P|Riverview|El Paso County|CO|89003|United States|-7|condo| +7612|AAAAAAAAMLNBAAAA|758|Lincoln |Court|Suite R|Oak Hill|Stonewall County|TX|77838|United States|-6|apartment| +7613|AAAAAAAANLNBAAAA|609|3rd North|Parkway|Suite 450|Maple Grove|Howard County|TX|78252|United States|-6|single family| +7614|AAAAAAAAOLNBAAAA|937|Washington |Ave|Suite 280|Arlington|Hillsdale County|MI|46557|United States|-5|single family| +7615|AAAAAAAAPLNBAAAA|405|6th Hill|Ct.|Suite F|Marion|Camas County|ID|80399|United States|-7|single family| +7616|AAAAAAAAAMNBAAAA|650|Sunset Miller|Parkway|Suite 250|Five Forks|Clinton County|IN|42293|United States|-5|condo| +7617|AAAAAAAABMNBAAAA|371|1st |Court|Suite 340|Newtown|Dallas County|IA|51749|United States|-6|single family| +7618|AAAAAAAACMNBAAAA|907|Hillcrest 4th|Circle|Suite 470|Fairfield|Owyhee County|ID|86192|United States|-7|single family| +7619|AAAAAAAADMNBAAAA|790|9th 3rd|Court|Suite 390|Waterloo|Harrison County|KY|41675|United States|-6|single family| +7620|AAAAAAAAEMNBAAAA|96|Sixth Smith|Avenue|Suite 340|Wilson|Okeechobee County|FL|36971|United States|-5|single family| +7621|AAAAAAAAFMNBAAAA|763|Mill Tenth|Road|Suite 370|Highland|Reno County|KS|69454|United States|-6|apartment| +7622|AAAAAAAAGMNBAAAA|844|Cedar |Boulevard|Suite 450|Clinton|Guadalupe County|NM|88222|United States|-7|condo| +7623|AAAAAAAAHMNBAAAA|948|Johnson |Cir.|Suite 190|Union|Trinity County|TX|78721|United States|-6|single family| +7624|AAAAAAAAIMNBAAAA|807|Williams Birch|RD|Suite 240|Oak Grove|Wyoming County|WV|28370|United States|-5|single family| +7625|AAAAAAAAJMNBAAAA|525|Spring 2nd|Road|Suite 470|Clinton|Caswell County|NC|28222|United States|-5|apartment| +7626|AAAAAAAAKMNBAAAA|75|East Lakeview|Drive|Suite V|Edgewood|Macon County|TN|30069|United States|-6|apartment| +7627|AAAAAAAALMNBAAAA|286|West Highland|Wy|Suite 10|Valley View|El Dorado County|CA|95124|United States|-8|condo| +7628|AAAAAAAAMMNBAAAA|956|Lake |Avenue|Suite A|Pleasant Valley|Box Butte County|NE|62477|United States|-6|condo| +7629|AAAAAAAANMNBAAAA|656|Elevnth |Ln|Suite 420|Greenfield|Delaware County|IN|45038|United States|-5|single family| +7630|AAAAAAAAOMNBAAAA|512|Madison 14th|Drive|Suite 260|Lincoln|Monroe County|MS|51289|United States|-6|condo| +7631|AAAAAAAAPMNBAAAA|30|5th |Ct.|Suite 480|Richville|Fresno County|CA|95945|United States|-8|condo| +7632|AAAAAAAAANNBAAAA|202|Green Miller|Pkwy|Suite P|Unionville|Henry County|IA|51711|United States|-6|condo| +7633|AAAAAAAABNNBAAAA|957|Railroad Johnson|Way|Suite A|Belmont|Colonial Heights city|VA|20191|United States|-5|apartment| +7634|AAAAAAAACNNBAAAA|525|5th |Boulevard|Suite O|Salem|Menard County|IL|68048|United States|-6|single family| +7635|AAAAAAAADNNBAAAA|104|View Lincoln|Wy|Suite O|Springfield|Broward County|FL|39303|United States|-5|single family| +7636|AAAAAAAAENNBAAAA|512|Tenth |Pkwy|Suite S|Harmony|Stafford County|KS|65804|United States|-6|apartment| +7637|AAAAAAAAFNNBAAAA|875|1st Locust|Parkway|Suite 60|Springdale|Oconee County|GA|38883|United States|-5|single family| +7638|AAAAAAAAGNNBAAAA|437|4th |Road|Suite M|Union|Brown County|WI|58721|United States|-6|single family| +7639|AAAAAAAAHNNBAAAA|46|Wilson Maple|Blvd|Suite D|Shore Acres|Marlboro County|SC|22724|United States|-5|apartment| +7640|AAAAAAAAINNBAAAA|485|Washington Davis|Road|Suite Q|Clinton|Reagan County|TX|78222|United States|-6|apartment| +7641|AAAAAAAAJNNBAAAA|827|Pine Wilson|Circle|Suite F|Union|Pierce County|GA|38721|United States|-5|single family| +7642|AAAAAAAAKNNBAAAA|594|Spruce |Avenue|Suite T|Crossroads|Abbeville County|SC|20534|United States|-5|single family| +7643|AAAAAAAALNNBAAAA||Jefferson |ST|Suite X||Carroll County||47746|United States|-5|apartment| +7644|AAAAAAAAMNNBAAAA|107|Locust Second|Way|Suite 320|Newtown|Cheyenne County|CO|81749|United States|-7|apartment| +7645|AAAAAAAANNNBAAAA|89|1st Highland|Street|Suite N|Green Acres|Fresno County|CA|97683|United States|-8|single family| +7646|AAAAAAAAONNBAAAA|110||Lane|Suite I|||SD|58059|United States||| +7647|AAAAAAAAPNNBAAAA|999|Williams Cedar|Way|Suite 20|Oakdale|Tazewell County|VA|29584|United States|-5|single family| +7648|AAAAAAAAAONBAAAA|949|Walnut 7th|Blvd|Suite E|Green Acres|Caddo Parish|LA|77683|United States|-6|condo| +7649|AAAAAAAABONBAAAA|751|Pine North|Street|Suite J|Spring Valley|Tuscarawas County|OH|46060|United States|-5|apartment| +7650|AAAAAAAACONBAAAA|676|Sunset |Circle|Suite 140|Lakeside|Wapello County|IA|59532|United States|-6|condo| +7651|AAAAAAAADONBAAAA|549|Pine View|Avenue|Suite F|Sulphur Springs|Warren County|TN|38354|United States|-6|apartment| +7652|AAAAAAAAEONBAAAA|274|Cedar |Pkwy|Suite T|Jenkins|Jasper County|SC|27292|United States|-5|apartment| +7653|AAAAAAAAFONBAAAA|292|Maple Jackson|Boulevard|Suite 270|Midway|Culberson County|TX|71904|United States|-6|condo| +7654|AAAAAAAAGONBAAAA|577|Park |ST|Suite E|Oak Hill|Woodbury County|IA|57838|United States|-6|apartment| +7655|AAAAAAAAHONBAAAA|512|15th |Way|Suite J|Greenfield|Lexington County|SC|25038|United States|-5|apartment| +7656|AAAAAAAAIONBAAAA|809|Lakeview Williams|Avenue|Suite M|Oak Grove|West Carroll Parish|LA|78370|United States|-6|condo| +7657|AAAAAAAAJONBAAAA|366|Highland |ST|Suite R|Forest Hills|Teton County|WY|89237|United States|-7|condo| +7658|AAAAAAAAKONBAAAA|585|2nd |Court|Suite 130|Lakeview|Bradley County|TN|38579|United States|-5|apartment| +7659|AAAAAAAALONBAAAA|528|Hillcrest |Boulevard|Suite U|Riverdale|Jefferson County|IN|49391|United States|-5|apartment| +7660|AAAAAAAAMONBAAAA|158|2nd Cedar|Circle|Suite W|Pleasant Grove|Gilliam County|OR|94136|United States|-8|single family| +7661|AAAAAAAANONBAAAA|590|Ridge |Street|Suite 460|Pine Grove|Emmet County|MI|44593|United States|-5|apartment| +7662|AAAAAAAAOONBAAAA|291|Walnut |Ln|Suite W|Oakdale|Polk County|IA|59584|United States|-6|apartment| +7663|AAAAAAAAPONBAAAA|135|Spring River|Way|Suite 370|Sulphur Springs|Ness County|KS||||apartment| +7664|AAAAAAAAAPNBAAAA|777|Valley Hill|Parkway|Suite 280|Jamestown|Vernon County|WI|56867|United States|-6|apartment| +7665|AAAAAAAABPNBAAAA||||Suite B|Springdale||IN|48883|United States||| +7666|AAAAAAAACPNBAAAA|815|11th |Ct.|Suite 260|Gum Springs|Kalamazoo County|MI|42106|United States|-5|condo| +7667|AAAAAAAADPNBAAAA|724|Park |Boulevard|Suite 180|Woodland|Pawnee County|OK|74854|United States|-6|apartment| +7668|AAAAAAAAEPNBAAAA|213||ST|Suite J|Oakdale|Pennington County|||United States||| +7669|AAAAAAAAFPNBAAAA|275|Wilson |Ave|Suite 270|Waterloo|Union County|IL|61675|United States|-6|condo| +7670|AAAAAAAAGPNBAAAA|636|Spring |Wy|Suite S|Red Hill|Marquette County|MI|44338|United States|-5|condo| +7671|AAAAAAAAHPNBAAAA|755|10th |Ln|Suite 490|Crossroads|Cavalier County|ND|50534|United States|-6|condo| +7672|AAAAAAAAIPNBAAAA|793|Tenth |RD|Suite G|Highland|Gray County|KS|69454|United States|-6|apartment| +7673|AAAAAAAAJPNBAAAA|357|Valley Eigth|Boulevard|Suite I|Springfield|Essex County|VA|29303|United States|-5|apartment| +7674|AAAAAAAAKPNBAAAA|530|Maple |Pkwy|Suite O|Stringtown|Doddridge County|WV|20162|United States|-5|condo| +7675|AAAAAAAALPNBAAAA|148|13th Lincoln|Street|Suite T|Bunker Hill|Jefferson County|CO|80150|United States|-7|apartment| +7676|AAAAAAAAMPNBAAAA|837|Church |Dr.|Suite 360|Weldon|Woodward County|OK|76277|United States|-6|single family| +7677|AAAAAAAANPNBAAAA|208|Spruce Woodland|Street|Suite 490|Pleasant Grove|Lake County|OR|94136|United States|-8|apartment| +7678|AAAAAAAAOPNBAAAA|720|Lake Center|Boulevard|Suite R|Morris|Christian County|IL|66696|United States|-6|single family| +7679|AAAAAAAAPPNBAAAA|280|Lake |Road|Suite A|Walnut Grove|Nottoway County|VA|27752|United States|-5|apartment| +7680|AAAAAAAAAAOBAAAA|877|Fifth 9th|ST|Suite X|Kingston|Bennington County|VT|05575|United States|-5|single family| +7681|AAAAAAAABAOBAAAA|805|Sunset |Ln|Suite T|Riverview|Sublette County|WY|89003|United States|-7|single family| +7682|AAAAAAAACAOBAAAA|989|Cherry 1st|Parkway|Suite H|Hopewell|Mendocino County|CA|90587|United States|-8|condo| +7683|AAAAAAAADAOBAAAA|113|Forest Miller|Avenue|Suite T|Pleasant Hill|Nicholas County|WV|23604|United States|-5|apartment| +7684|AAAAAAAAEAOBAAAA|568|Ridge Walnut|Dr.|Suite 260|Walnut Grove|Routt County|CO|87752|United States|-7|single family| +7685|AAAAAAAAFAOBAAAA|452|Miller |Wy|Suite J|Berea|Webster County|IA|53804|United States|-6|apartment| +7686|AAAAAAAAGAOBAAAA||||Suite 120|Midway|||41904||-5|| +7687|AAAAAAAAHAOBAAAA|167|1st Fifth|Court|Suite E|Bridgeport|Amelia County|VA|25817|United States|-5|condo| +7688|AAAAAAAAIAOBAAAA|457|Main Chestnut|Road|Suite 80|Arlington|Horry County|SC|26557|United States|-5|single family| +7689|AAAAAAAAJAOBAAAA|654|Locust |Way|Suite W|Richardson|Palo Pinto County|TX|77687|United States|-6|condo| +7690|AAAAAAAAKAOBAAAA|278|10th |Court|Suite 380|Lakeside|Teton County|MT|69532|United States|-7|condo| +7691|AAAAAAAALAOBAAAA|364|2nd |Ln|Suite 440|Delmar|Emmet County|IA|53957|United States|-6|single family| +7692|AAAAAAAAMAOBAAAA|390|Fifth Lakeview|Parkway|Suite 250|Thompsonville|Morrow County|OH|49651|United States|-5|apartment| +7693|AAAAAAAANAOBAAAA|638|Fourth |Cir.|Suite 250|Lebanon|Clay County|GA|32898|United States|-5|condo| +7694|AAAAAAAAOAOBAAAA|509|11th |Pkwy|Suite Q|Highland|Scioto County|OH|49454|United States|-5|single family| +7695|AAAAAAAAPAOBAAAA|282|Cedar Oak|Boulevard|Suite P|Plainview|Cumberland County|IL|63683|United States|-6|single family| +7696|AAAAAAAAABOBAAAA|333|Fifth |Avenue|Suite 260|Florence|Jasper County|IN|43394|United States|-5|condo| +7697|AAAAAAAABBOBAAAA|391|Spring Valley|Court|Suite A|Springfield|Caldwell County|NC|29303|United States|-5|apartment| +7698|AAAAAAAACBOBAAAA|99|1st |ST|Suite C|Berea|Van Buren County|AR|73804|United States|-6|condo| +7699|AAAAAAAADBOBAAAA|441|13th |Avenue|Suite R|Providence|Highland County|OH|46614|United States|-5|single family| +7700|AAAAAAAAEBOBAAAA|719|Wilson 7th|Ave|Suite J|Waterloo|Jefferson County|WA|91675|United States|-8|condo| +7701|AAAAAAAAFBOBAAAA|869|Valley 2nd|Court|Suite N|Union Hill|Hancock County|MS|57746|United States|-6|condo| +7702|AAAAAAAAGBOBAAAA|853|Sycamore |Circle|Suite 150|Shiloh|Calhoun County|AR|79275|United States|-6|single family| +7703|AAAAAAAAHBOBAAAA|248|2nd |Pkwy|Suite D|Lakeview|Middlesex County|CT|09179|United States|-5|condo| +7704|AAAAAAAAIBOBAAAA|914|Spring Williams|Street|Suite R|Shiloh|DeBaca County|NM|89275|United States|-7|apartment| +7705|AAAAAAAAJBOBAAAA|975|Third Lincoln|Way|Suite T|Wilson|Marion County|AL|36971|United States|-6|apartment| +7706|AAAAAAAAKBOBAAAA|55|Ridge 1st|Street|Suite 380|Crossroads|Chester County|SC|20534|United States|-5|single family| +7707|AAAAAAAALBOBAAAA|614|Walnut Maple|Ln|Suite 230|Brownsville|Baca County|CO|89310|United States|-7|single family| +7708|AAAAAAAAMBOBAAAA|827|5th Center|Way|Suite 210|Post Oak|Wake County|NC|28567|United States|-5|apartment| +7709|AAAAAAAANBOBAAAA|363|Laurel Woodland|Pkwy|Suite S|Pleasant Grove|Jasper County|TX|74136|United States|-6|single family| +7710|AAAAAAAAOBOBAAAA|54|Lincoln 2nd|Ct.|Suite S|Unionville|Lincoln County|NC|21711|United States|-5|condo| +7711|AAAAAAAAPBOBAAAA|121|Park |Court|Suite 50|Concord|Williamson County|TX|74107|United States|-6|condo| +7712|AAAAAAAAACOBAAAA|99|10th |Pkwy|Suite 360|Ashland|Beaufort County|NC|24244|United States|-5|condo| +7713|AAAAAAAABCOBAAAA|742|Elm Oak|Ct.|Suite 190|Valley View|Eau Claire County|WI|55124|United States|-6|apartment| +7714|AAAAAAAACCOBAAAA|449|Fifth Forest|Pkwy|Suite 10|Pleasant Valley|Chattahoochee County|GA|32477|United States|-5|apartment| +7715|AAAAAAAADCOBAAAA|369|West Jefferson|Drive|Suite G|Hamilton|Middlesex County|NJ|03408|United States|-5|single family| +7716|AAAAAAAAECOBAAAA|765|Washington 2nd|ST|Suite 310|Florence|Ritchie County|WV|23394|United States|-5|single family| +7717|AAAAAAAAFCOBAAAA|390|Franklin |Ln|Suite M|Stringtown|Kiowa County|CO|80162|United States|-7|condo| +7718|AAAAAAAAGCOBAAAA|967|Jackson 5th|Court|Suite S|Liberty|Buncombe County|NC|23451|United States|-5|condo| +7719|AAAAAAAAHCOBAAAA|764|Center |Drive|Suite 290|Cedar Grove|Ascension Parish|LA|70411|United States|-6|apartment| +7720|AAAAAAAAICOBAAAA|484|Forest |Court|Suite 240|Pinecrest|Sonoma County|CA|99981|United States|-8|single family| +7721|AAAAAAAAJCOBAAAA|528|Spruce |Circle|Suite 240|Plainview|Independence County|AR|73683|United States|-6|condo| +7722|AAAAAAAAKCOBAAAA|612|Cherry |Pkwy|Suite K|Pinecrest|Coffey County|KS|69981|United States|-6|apartment| +7723|AAAAAAAALCOBAAAA|894|Dogwood 14th|Street|Suite D|Bunker Hill|DeBaca County|NM|80150|United States|-7|single family| +7724|AAAAAAAAMCOBAAAA|78|Johnson |Ln|Suite H|Liberty|Nome Census Area|AK|93451|United States|-9|condo| +7725|AAAAAAAANCOBAAAA|544|Ash |Boulevard|Suite G|Enterprise|Benzie County|MI|41757|United States|-5|apartment| +7726|AAAAAAAAOCOBAAAA|404|Elm |Lane|Suite G|Red Hill|Monterey County|CA|94338|United States|-8|condo| +7727|AAAAAAAAPCOBAAAA|705|Smith |Boulevard|Suite S|Lebanon|Hitchcock County|NE|62898|United States|-6|condo| +7728|AAAAAAAAADOBAAAA|376|12th |RD|Suite 220|Pine Grove|Hennepin County|MN|54593|United States|-6|condo| +7729|AAAAAAAABDOBAAAA|205|Hickory |Parkway|Suite 60|Westminster|Carroll County|TN|36549|United States|-5|single family| +7730|AAAAAAAACDOBAAAA|943|Church |Cir.|Suite 340|Bethel|Macomb County|MI|45281|United States|-5|apartment| +7731|AAAAAAAADDOBAAAA|172|8th Cherry|Ln|Suite 410|Highland|Wabash County|IL|69454|United States|-6|apartment| +7732|AAAAAAAAEDOBAAAA|623|Spruce Laurel|Lane|Suite T|Forest Hills|Haskell County|TX|79237|United States|-6|apartment| +7733|AAAAAAAAFDOBAAAA|337|Pine 3rd|Pkwy|Suite L|Lincoln|Franklin County|NC|21289|United States|-5|condo| +7734|AAAAAAAAGDOBAAAA|3|Miller 3rd|Ct.|Suite J|Bunker Hill|Lamb County|TX|70150|United States|-6|condo| +7735|AAAAAAAAHDOBAAAA|950|Lake Church|Dr.|Suite 210|Greenville|Randolph County|GA|31387|United States|-5|condo| +7736|AAAAAAAAIDOBAAAA|891|Fourth Meadow|Ave|Suite C|Jackson|Ray County|MO|69583|United States|-6|apartment| +7737|AAAAAAAAJDOBAAAA|939|Maple Cherry|Street|Suite 250|Summit|Barry County|MI|40499|United States|-5|single family| +7738|AAAAAAAAKDOBAAAA|46|Lincoln Mill|RD|Suite J|Bunker Hill|Menominee County|WI|50150|United States|-6|apartment| +7739|AAAAAAAALDOBAAAA|324|Forest |Pkwy|Suite 150|Spring Valley|Broward County|FL|36060|United States|-5|condo| +7740|AAAAAAAAMDOBAAAA|549|4th Lakeview|Boulevard|Suite N|Buena Vista|Gallatin County|IL|65752|United States|-6|condo| +7741|AAAAAAAANDOBAAAA|448|Wilson |Court|Suite 180|Crossroads|Inyo County|CA|90534|United States|-8|condo| +7742|AAAAAAAAODOBAAAA|599|4th |Dr.|Suite E|Woodville|Cherokee County|GA|34289|United States|-5|condo| +7743|AAAAAAAAPDOBAAAA|927|Second |Lane|Suite 190|Waterloo|Van Buren County|MI|41675|United States|-5|apartment| +7744|AAAAAAAAAEOBAAAA|578|Jefferson |Cir.|Suite 190|Georgetown|Miller County|GA|37057|United States|-5|apartment| +7745|AAAAAAAABEOBAAAA|61|6th Meadow|Wy|Suite 450|Roscoe|Clark County|IL|61854|United States|-6|condo| +7746|AAAAAAAACEOBAAAA|550|South |ST|Suite 380|Cedar Grove|Crockett County|TX|70411|United States|-6|apartment| +7747|AAAAAAAADEOBAAAA|629|Jefferson Washington|Blvd|Suite 260|Edgewood|Kittitas County|WA|90069|United States|-8|apartment| +7748|AAAAAAAAEEOBAAAA|619|Lincoln Washington|Avenue|Suite 110|Edgewood|Claiborne County|MS|50069|United States|-6|condo| +7749|AAAAAAAAFEOBAAAA|676|11th |Wy|Suite C|Lakewood|Beaverhead County|MT|68877|United States|-7|single family| +7750|AAAAAAAAGEOBAAAA|41|Locust |Road|Suite E|Chestnut Ridge|Buchanan County|MO|67334|United States|-6|apartment| +7751|AAAAAAAAHEOBAAAA|138|Williams 7th|Dr.|Suite 130|Sullivan|Pike County|MS|50451|United States|-6|apartment| +7752|AAAAAAAAIEOBAAAA|133|3rd |Cir.|Suite 480|Oak Hill|Paulding County|OH|47838|United States|-5|single family| +7753|AAAAAAAAJEOBAAAA|577|Highland |Ave|Suite I|Unionville|Kiowa County|KS|61711|United States|-6|condo| +7754|AAAAAAAAKEOBAAAA|295|Wilson Green|ST|Suite O|Hillcrest|Koochiching County|MN|53003|United States|-6|condo| +7755|AAAAAAAALEOBAAAA|119|Laurel |Dr.|Suite R|Jamestown|Bay County|FL|36867|United States|-5|condo| +7756|AAAAAAAAMEOBAAAA|853|First |Boulevard|Suite T|Price|Oscoda County|MI|43824|United States|-5|condo| +7757|AAAAAAAANEOBAAAA|615|Cherry |Cir.|Suite U|Centerville|Pocahontas County|IA|50059|United States|-6|condo| +7758|AAAAAAAAOEOBAAAA|887|Washington Woodland|Avenue|Suite L|Clinton|Highland County|VA|28222|United States|-5|single family| +7759|AAAAAAAAPEOBAAAA|749|Fifth 8th|Circle|Suite 200|Edgewood|Craighead County|AR|70069|United States|-6|single family| +7760|AAAAAAAAAFOBAAAA|991|Jackson |ST|Suite 380|Five Forks|Hampshire County|WV|22293|United States|-5|condo| +7761|AAAAAAAABFOBAAAA|765|14th |Lane|Suite Q|White Oak|Brown County|KS|66668|United States|-6|condo| +7762|AAAAAAAACFOBAAAA|488|First |Ave|Suite H|Kingston|Prince Edward County|VA|24975|United States|-5|condo| +7763|AAAAAAAADFOBAAAA|846|Park 3rd|ST|Suite 270|Glendale|Cole County|MO|63951|United States|-6|apartment| +7764|AAAAAAAAEFOBAAAA|460|4th |ST|Suite O|Richville|Lynchburg city|VA|25945|United States|-5|single family| +7765|AAAAAAAAFFOBAAAA|33|Meadow |Dr.|Suite 130|Pleasant Valley|Orange County|VA|22477|United States|-5|single family| +7766|AAAAAAAAGFOBAAAA|||||Greenville|Dent County|||||single family| +7767|AAAAAAAAHFOBAAAA|402|Oak |Dr.|Suite 10|Riverview|Lawrence County|AR|79003|United States|-6|apartment| +7768|AAAAAAAAIFOBAAAA|865|3rd Jefferson|Pkwy|Suite 70|Lebanon|Snyder County|PA|12898|United States|-5|apartment| +7769|AAAAAAAAJFOBAAAA|56|Wilson 14th|Wy|Suite S|Summit|Cook County|MN|50499|United States|-6|single family| +7770|AAAAAAAAKFOBAAAA|31|Walnut |Wy|Suite 320|Friendship|Newton County|IN|44536|United States|-5|apartment| +7771|AAAAAAAALFOBAAAA|91|View |Avenue|Suite J|Hillcrest|Crook County|OR|93003|United States|-8|apartment| +7772|AAAAAAAAMFOBAAAA|495|7th |Lane|Suite X|Unionville|Caswell County|NC|21711|United States|-5|single family| +7773|AAAAAAAANFOBAAAA|984|7th |Blvd|Suite Y|Bay View|Victoria County|TX|76457|United States|-6|single family| +7774|AAAAAAAAOFOBAAAA|803|Washington Highland|Wy|Suite 460|Midway|Unicoi County|TN|31904|United States|-6|condo| +7775|AAAAAAAAPFOBAAAA|8|Oak |Ct.|Suite T|Oakland|Coconino County|AZ|89843|United States|-7|condo| +7776|AAAAAAAAAGOBAAAA|546|Twelfth |Way|Suite 110|Providence|Lyman County|SD|56614|United States|-7|condo| +7777|AAAAAAAABGOBAAAA|733|Third Williams|Ave|Suite 320|Walnut Grove|Bedford County|TN|37752|United States|-5|condo| +7778|AAAAAAAACGOBAAAA|407|15th |RD|Suite 310|Spring Valley|El Paso County|TX|76060|United States|-6|condo| +7779|AAAAAAAADGOBAAAA|664|Ash |Blvd|Suite N|Lakeside|Oconto County|WI|59532|United States|-6|single family| +7780|AAAAAAAAEGOBAAAA|827|15th 2nd|Pkwy|Suite 420|Oakland|Park County|MT|69843|United States|-7|single family| +7781|AAAAAAAAFGOBAAAA|302|Main Park|Pkwy|Suite C|Woodbury|Jefferson County|TX|74489|United States|-6|condo| +7782|AAAAAAAAGGOBAAAA|217|Eigth Chestnut|Wy|Suite L|Five Points|Missaukee County|MI|46098|United States|-5|condo| +7783|AAAAAAAAHGOBAAAA|950|Williams Lake|Court|Suite 0|Kingston|Lonoke County|AR|74975|United States|-6|apartment| +7784|AAAAAAAAIGOBAAAA|488|Maple Ridge|Wy|Suite 280|Wilson|San Patricio County|TX|76971|United States|-6|single family| +7785|AAAAAAAAJGOBAAAA|459||Ave|||||||-6|single family| +7786|AAAAAAAAKGOBAAAA|21||Drive|Suite 370|Mount Pleasant|Manassas Park city|VA|21933|United States||| +7787|AAAAAAAALGOBAAAA|923|Pine |Court|Suite 120|Newport|Fayette County|IN|41521|United States|-5|condo| +7788|AAAAAAAAMGOBAAAA|245|14th |Ave|Suite 160|Mountain View|Highlands County|FL|34466|United States|-5|single family| +7789|AAAAAAAANGOBAAAA|791|Laurel |Court|Suite 290|Spring Grove|Rawlins County|KS|66719|United States|-6|apartment| +7790|AAAAAAAAOGOBAAAA|571|Jackson |Cir.|Suite D|Greenfield|Hamilton County|OH|45038|United States|-5|condo| +7791|AAAAAAAAPGOBAAAA|933|Sunset |Wy|Suite 250|Franklin|Duplin County|NC|29101|United States|-5|single family| +7792|AAAAAAAAAHOBAAAA|606|Third View|Circle|Suite 370|Langdon|Donley County|TX|70852|United States|-6|condo| +7793|AAAAAAAABHOBAAAA|714|Sunset Spruce|ST|Suite 440|Red Hill|Orange County|CA|94338|United States|-8|condo| +7794|AAAAAAAACHOBAAAA|860|3rd View|Wy|Suite 450|Plainview|Muskegon County|MI|43683|United States|-5|condo| +7795|AAAAAAAADHOBAAAA|884|Poplar |Circle|Suite 360|Concord|Little River County|AR|74107|United States|-6|apartment| +7796|AAAAAAAAEHOBAAAA|686|Park |ST|Suite 250|Montague|Mason County|IL|64062|United States|-6|condo| +7797|AAAAAAAAFHOBAAAA|515|Main |RD|Suite 310|Lakewood|Tensas Parish|LA|78877|United States|-6|single family| +7798|AAAAAAAAGHOBAAAA|3|11th |Blvd|Suite 180|Wildwood|Clay County|NE|66871|United States|-6|condo| +7799|AAAAAAAAHHOBAAAA|701|Sixth |Lane|Suite W|Jamestown|Wood County|WV|26867|United States|-5|single family| +7800|AAAAAAAAIHOBAAAA|959|Pine |Way|Suite K|Delmar|Talladega County|AL|33957|United States|-6|single family| +7801|AAAAAAAAJHOBAAAA|946|3rd 1st|Ln|Suite 350|Highland Park|Scott County|MN|56534|United States|-6|single family| +7802|AAAAAAAAKHOBAAAA|132||RD||Mountain View|Lipscomb County|TX|74466||-6|condo| +7803|AAAAAAAALHOBAAAA|329|Pine |RD|Suite 210|Greenfield|Huntingdon County|PA|15038|United States|-5|apartment| +7804|AAAAAAAAMHOBAAAA|385|Forest Cherry|Avenue|Suite 0|Union City|Fond du Lac County|WI|58087|United States|-6|apartment| +7805|AAAAAAAANHOBAAAA|252|Meadow |ST|Suite H|Woodville|Morton County|KS|64289|United States|-6|apartment| +7806|AAAAAAAAOHOBAAAA|861|Center |Lane|Suite P|Walnut Grove|Lincoln County|CO|87752|United States|-7|apartment| +7807|AAAAAAAAPHOBAAAA|627|Madison |ST|Suite H|Bridgeport|Ozark County|MO|65817|United States|-6|single family| +7808|AAAAAAAAAIOBAAAA|335|9th |Blvd|Suite 250|Glenwood|Pottawattamie County|IA|53511|United States|-6|condo| +7809|AAAAAAAABIOBAAAA|954|Lake West|Avenue|Suite J|Elm Grove|Lowndes County|MS|53298|United States|-6|single family| +7810|AAAAAAAACIOBAAAA|150|5th Walnut|Boulevard|Suite 30|Pleasant Grove|Saluda County|SC|24136|United States|-5|apartment| +7811|AAAAAAAADIOBAAAA|293|Cherry Main|Court|Suite 410|Walnut Grove|Scotland County|NC|27752|United States|-5|single family| +7812|AAAAAAAAEIOBAAAA|43|||||Pierce County|ND|55709|United States||apartment| +7813|AAAAAAAAFIOBAAAA|905|Willow |Way|Suite T|Mount Pleasant|Washington County|TX|71933|United States|-6|condo| +7814|AAAAAAAAGIOBAAAA|985|Ridge |ST|Suite 120|Fairfield|Tom Green County|TX|76192|United States|-6|apartment| +7815|AAAAAAAAHIOBAAAA|768|View |Blvd|Suite 130|Antioch|Santa Cruz County|CA|98605|United States|-8|single family| +7816|AAAAAAAAIIOBAAAA|966|Maple Williams|Cir.|Suite K|Newtown|Chambers County|TX|71749|United States|-6|apartment| +7817|AAAAAAAAJIOBAAAA|628|Woodland Sunset|Court|Suite 0|Centerville|Valley County|ID|80059|United States|-7|apartment| +7818|AAAAAAAAKIOBAAAA|456|Ninth |Ln|Suite 220|Arthur|Vernon Parish|LA|75965|United States|-6|apartment| +7819|AAAAAAAALIOBAAAA|414|Main |Wy|Suite 120|Belmont|Miami County|OH|40191|United States|-5|single family| +7820|AAAAAAAAMIOBAAAA|895|Park |Ave|Suite 260|Wilson|Ritchie County|WV|26971|United States|-5|single family| +7821|AAAAAAAANIOBAAAA|618|4th |Way|Suite 470|Woodland|Pike County|OH|44854|United States|-5|single family| +7822|AAAAAAAAOIOBAAAA|700|Mill |Pkwy|Suite 370|Riverview|Lincoln County|ID|89003|United States|-7|single family| +7823|AAAAAAAAPIOBAAAA|228|North |Blvd|Suite L|Newtown|Taylor County|TX|71749|United States|-6|apartment| +7824|AAAAAAAAAJOBAAAA|313|Sixth Highland|ST|Suite J|Buena Vista|Johnston County|OK|75752|United States|-6|condo| +7825|AAAAAAAABJOBAAAA|431|Washington |Ln|Suite N|Clifton|Linn County|OR|98014|United States|-8|single family| +7826|AAAAAAAACJOBAAAA|981|Johnson |Dr.|Suite I|Oak Ridge|Beckham County|OK|78371|United States|-6|condo| +7827|AAAAAAAADJOBAAAA|573|Johnson Central|RD|Suite 50|Newtown|Tuscola County|MI|41749|United States|-5|apartment| +7828|AAAAAAAAEJOBAAAA|653|Jefferson Seventh|Road|Suite H|Jackson|Summit County|CO|89583|United States|-7|apartment| +7829|AAAAAAAAFJOBAAAA|756|Chestnut |Court|Suite 140|Edgewood|Harrison County|TX|70069|United States|-6|condo| +7830|AAAAAAAAGJOBAAAA|489|Spring Ridge|Cir.|Suite T|Jamestown|Lee County|AR|76867|United States|-6|apartment| +7831|AAAAAAAAHJOBAAAA|424|6th Laurel|Cir.|Suite 350|Springfield|Lewis County|WA|99303|United States|-8|apartment| +7832|AAAAAAAAIJOBAAAA|744|Pine |Wy|Suite R|Edgewood|Iron County|UT|80069|United States|-7|condo| +7833|AAAAAAAAJJOBAAAA|35|Madison View|Blvd|Suite P|King|Lenawee County|MI|40008|United States|-5|condo| +7834|AAAAAAAAKJOBAAAA|11|1st |RD|Suite N|Lakewood|Waynesboro city|VA|28877|United States|-5|single family| +7835|AAAAAAAALJOBAAAA|280|Hickory River|Ct.|Suite J|Clifton|Butler County|MO|68014|United States|-6|single family| +7836|AAAAAAAAMJOBAAAA|467|Ninth |Circle|Suite T|Mount Pleasant|Cannon County|TN|31933|United States|-5|apartment| +7837|AAAAAAAANJOBAAAA|868|Wilson |ST|Suite P|Riverside|Bullitt County|KY|49231|United States|-6|condo| +7838|AAAAAAAAOJOBAAAA|322|4th |Boulevard|Suite Y|Fairfield|Dorchester County|SC|26192|United States|-5|apartment| +7839|AAAAAAAAPJOBAAAA|738|View 12th|ST|Suite 50|Pleasant Hill|Saguache County|CO|83604|United States|-7|condo| +7840|AAAAAAAAAKOBAAAA|581|Poplar Meadow|Dr.|Suite H|Greenville|Schuyler County|IL|61387|United States|-6|condo| +7841|AAAAAAAABKOBAAAA|496|Walnut Meadow|Parkway|Suite 270|Mount Olive|Barbour County|AL|38059|United States|-6|apartment| +7842|AAAAAAAACKOBAAAA||Washington ||Suite 340|Stringtown|||20162|United States|-5|single family| +7843|AAAAAAAADKOBAAAA|733|Valley Williams|Ct.|Suite 120|Murray|Lincoln County|WV|22150|United States|-5|single family| +7844|AAAAAAAAEKOBAAAA|239|7th Birch|Avenue|Suite 410|Five Forks|Ferry County|WA|92293|United States|-8|apartment| +7845|AAAAAAAAFKOBAAAA|524|Dogwood View|Court|Suite A|Unionville|Butler County|NE|61711|United States|-6|apartment| +7846|AAAAAAAAGKOBAAAA|492|Washington Park|Road|Suite B|Five Forks|Harrison County|TX|72293|United States|-6|condo| +7847|AAAAAAAAHKOBAAAA|362|Jefferson |ST|Suite 270|Oak Grove|Chatham County|NC|28370|United States|-5|apartment| +7848|AAAAAAAAIKOBAAAA|405|3rd |ST|Suite 160|Newport|Camden County|MO|61521|United States|-6|apartment| +7849|AAAAAAAAJKOBAAAA|682|Cedar 6th|Dr.|Suite X|Glendale|Riley County|KS|63951|United States|-6|condo| +7850|AAAAAAAAKKOBAAAA|759|Lincoln Lincoln|Dr.|Suite U|Allison|Teton County|WY|84167|United States|-7|apartment| +7851|AAAAAAAALKOBAAAA|885|Pine Third|Lane|Suite Q|Mount Olive|Oxford County|ME|08659|United States|-5|single family| +7852|AAAAAAAAMKOBAAAA|608|Fourth Washington|Cir.|Suite G|Riverdale|Treasure County|MT|69391|United States|-7|apartment| +7853|AAAAAAAANKOBAAAA|761|10th Mill|Dr.|Suite 360|Hillcrest|Owen County|IN|43003|United States|-5|single family| +7854|AAAAAAAAOKOBAAAA|218|5th |Ave|Suite B|Maple Grove|Foard County|TX|78252|United States|-6|apartment| +7855|AAAAAAAAPKOBAAAA|201|South |Boulevard|Suite R|Pleasant Valley|Appling County|GA|32477|United States|-5|single family| +7856|AAAAAAAAALOBAAAA|70|Tenth |Boulevard|Suite N|Greenville|Hunterdon County|NJ|01987|United States|-5|single family| +7857|AAAAAAAABLOBAAAA|687|1st |Street|Suite R|Sunnyside|Franklin County|TX|71952|United States|-6|apartment| +7858|AAAAAAAACLOBAAAA|472|River Willow|Court|Suite 30|Oakdale|Wakulla County|FL|39584|United States|-5|apartment| +7859|AAAAAAAADLOBAAAA|319|Lake |Pkwy|Suite 180|Saint Paul|Orange County|FL|35720|United States|-5|condo| +7860|AAAAAAAAELOBAAAA|420|North 14th|Cir.|Suite 260|Wildwood|Campbell County|SD|56871|United States|-6|apartment| +7861|AAAAAAAAFLOBAAAA|358|Railroad Spring|Street|Suite 30|Church Hill|Crisp County|GA|33790|United States|-5|single family| +7862|AAAAAAAAGLOBAAAA|312|Sunset Ash|Court|Suite 460|Lawrenceville|Wetzel County|WV|24462|United States|-5|condo| +7863|AAAAAAAAHLOBAAAA|647|Walnut Elm||Suite W|Avery|||90194|United States|-8|| +7864|AAAAAAAAILOBAAAA|877|Lee Jefferson|Ct.|Suite 90|Oak Hill|LaMoure County|ND|57838|United States|-6|single family| +7865|AAAAAAAAJLOBAAAA|940|12th Williams|Court|Suite V|Arlington|Bucks County|PA|16557|United States|-5|apartment| +7866|AAAAAAAAKLOBAAAA|971|Park |Parkway|Suite A|Mount Zion|Richland County|SC|28054|United States|-5|condo| +7867|AAAAAAAALLOBAAAA|494|Main 15th|Boulevard|Suite G|Greenwood|Moore County|TN|38828|United States|-6|apartment| +7868|AAAAAAAAMLOBAAAA|386|View |Dr.|Suite 450|Walnut Grove|Fremont County|ID|87752|United States|-7|single family| +7869|AAAAAAAANLOBAAAA|333|Main |Court|Suite 160|Florence|Erie County|PA|13394|United States|-5|single family| +7870|AAAAAAAAOLOBAAAA|761|River |Dr.|Suite 300|Pine Grove|Belmont County|OH|44593|United States|-5|single family| +7871|AAAAAAAAPLOBAAAA|887|Third |Ct.|Suite W|Sulphur Springs|Clay County|GA|38354|United States|-5|apartment| +7872|AAAAAAAAAMOBAAAA|453|1st |Cir.|Suite 480|Hamilton|Tangipahoa Parish|LA|72808|United States|-6|condo| +7873|AAAAAAAABMOBAAAA|889|Oak Lee|ST|Suite 440|Shore Acres|Bulloch County|GA|32724|United States|-5|condo| +7874|AAAAAAAACMOBAAAA|335|Lake |Cir.|Suite 310|Hillcrest|Grant County|KY|43003|United States|-6|condo| +7875|AAAAAAAADMOBAAAA|721|Elevnth Jackson|Pkwy|Suite S|Thompsonville|Los Alamos County|NM|89651|United States|-7|apartment| +7876|AAAAAAAAEMOBAAAA|224|Cedar 2nd|Cir.|Suite 450|Horton|Kearny County|KS|61869|United States|-6|single family| +7877|AAAAAAAAFMOBAAAA|798|Park |Ln|Suite 20|Waterloo|Island County|WA|91675|United States|-8|condo| +7878|AAAAAAAAGMOBAAAA|932|Walnut Walnut|Parkway|Suite 440|Crossroads|Dent County|MO|60534|United States|-6|apartment| +7879|AAAAAAAAHMOBAAAA|524|Meadow |Road|Suite 190|Lakeview|Marshall County|WV|28579|United States|-5|apartment| +7880|AAAAAAAAIMOBAAAA|15|Hillcrest |Road|Suite 390|Fairview|Phillips County|AR|75709|United States|-6|apartment| +7881|AAAAAAAAJMOBAAAA|895|Sycamore 1st|Street|Suite 90|Oak Grove|Millard County|UT|88370|United States|-7|condo| +7882|AAAAAAAAKMOBAAAA|480|Adams Pine|Ln|Suite 10|Aberdeen|Douglas County|WI|57995|United States|-6|single family| +7883|AAAAAAAALMOBAAAA|840|12th |Way|Suite C|Cedar Grove|Caledonia County|VT|01011|United States|-5|condo| +7884|AAAAAAAAMMOBAAAA|138|South |Ln|Suite 410|Clinton|Hancock County|OH|48222|United States|-5|apartment| +7885|AAAAAAAANMOBAAAA|175|Park College|Wy|Suite K|Hopewell|Bradford County|FL|30587|United States|-5|single family| +7886|AAAAAAAAOMOBAAAA|672|3rd |RD|Suite 140|Plainview|Grays Harbor County|WA|93683|United States|-8|condo| +7887|AAAAAAAAPMOBAAAA|357|North 7th|Way|Suite O|Enterprise|Sheridan County|KS|61757|United States|-6|single family| +7888|AAAAAAAAANOBAAAA|444|Washington |Circle|Suite 40|Oak Grove|Seward County|KS|68370|United States|-6|condo| +7889|AAAAAAAABNOBAAAA|318|Main Main|Circle|Suite 60|Valley View|Lawrence County|PA|15124|United States|-5|single family| +7890|AAAAAAAACNOBAAAA|649|Second |Ln|Suite W|Franklin|Morgan County|TN|39101|United States|-6|condo| +7891|AAAAAAAADNOBAAAA|408|Smith |ST|Suite 310|Hamilton|Cumberland County|IL|62808|United States|-6|single family| +7892|AAAAAAAAENOBAAAA|696|Walnut Fourth|Wy|Suite S|Mountain View|Coleman County|TX|74466|United States|-6|condo| +7893|AAAAAAAAFNOBAAAA|598|Cedar Highland|Avenue|Suite 30|Harmony|Martin County|KY|45804|United States|-5|apartment| +7894|AAAAAAAAGNOBAAAA|462|Lincoln View|Pkwy|Suite E|Bethel|Lincoln County|AR|75281|United States|-6|condo| +7895|AAAAAAAAHNOBAAAA|599|Oak |Street|Suite 190|Liberty|Hocking County|OH|43451|United States|-5|apartment| +7896|AAAAAAAAINOBAAAA|421|3rd Eigth|Wy|Suite T|Lewisburg|La Salle County|IL|67538|United States|-6|single family| +7897|AAAAAAAAJNOBAAAA|495|Cedar |Court|Suite G|Glenwood|Manatee County|FL|33511|United States|-5|condo| +7898|AAAAAAAAKNOBAAAA|809|Pine |RD|Suite Q|Edgewood|Carroll County|IN|40069|United States|-5|single family| +7899|AAAAAAAALNOBAAAA|263|Meadow Jackson|Blvd|Suite 470|Salem|Shiawassee County|MI|48048|United States|-5|apartment| +7900|AAAAAAAAMNOBAAAA|546|Church Williams|RD|Suite 0|Oakland|Ripley County|MO|69843|United States|-6|condo| +7901|AAAAAAAANNOBAAAA|413|West Sixth|Road|Suite 390|Siloam|Smith County|TX|78948|United States|-6|single family| +7902|AAAAAAAAONOBAAAA|397|5th |Ln|Suite 100|Sunnyside|Chase County|NE|61952|United States|-6|condo| +7903|AAAAAAAAPNOBAAAA|97|Park 10th|Boulevard|Suite 140|Pleasant Hill|Jersey County|IL|63604|United States|-6|single family| +7904|AAAAAAAAAOOBAAAA|163|South 15th|Circle|Suite 230|Post Oak|Crisp County|GA|38567|United States|-5|apartment| +7905|AAAAAAAABOOBAAAA|712|10th First|Lane|Suite R|Maple Grove|Orangeburg County|SC|28252|United States|-5|condo| +7906|AAAAAAAACOOBAAAA|928|4th |Cir.|Suite Q|Hopewell|Kerr County|TX|70587|United States|-6|apartment| +7907|AAAAAAAADOOBAAAA|362|Railroad Main|Pkwy|Suite 20|Springdale|Allegan County|MI|48883|United States|-5|apartment| +7908|AAAAAAAAEOOBAAAA|886|Jefferson |Avenue|Suite W|Belmont|Lake County|SD|50191|United States|-7|single family| +7909|AAAAAAAAFOOBAAAA|865|Spruce |ST|Suite L|Franklin|Limestone County|TX|79101|United States|-6|single family| +7910|AAAAAAAAGOOBAAAA|68|Franklin Park|Blvd|Suite E|Farmington|Carter County|MT|69145|United States|-7|apartment| +7911|AAAAAAAAHOOBAAAA|771|South |Circle|Suite 410|Columbus|Gates County|NC|23622|United States|-5|single family| +7912|AAAAAAAAIOOBAAAA|649|3rd East|Road|Suite 310|Arlington|Quitman County|MS|56557|United States|-6|condo| +7913|AAAAAAAAJOOBAAAA|983|Pine |Circle|Suite 410|Crossroads|Lee County|MS|50534|United States|-6|apartment| +7914|AAAAAAAAKOOBAAAA|191|||Suite H||||||-7|| +7915|AAAAAAAALOOBAAAA|156|12th Railroad|Ln|Suite W|Bennett|Dillingham Census Area|AK|91715|United States|-9|condo| +7916|AAAAAAAAMOOBAAAA|789|Main |Court|Suite R|Springfield|Terrebonne Parish|LA|79303|United States|-6|single family| +7917|AAAAAAAANOOBAAAA|47|Lincoln Spruce|Dr.|Suite 240|Brownsville|Orange County|VT|09910|United States|-5|single family| +7918|AAAAAAAAOOOBAAAA|274|Forest Lake|Street|Suite R|Pine Grove|Walker County|AL|34593|United States|-6|apartment| +7919|AAAAAAAAPOOBAAAA|901|Pine |Parkway|Suite 400|Greenwood|Oceana County|MI|48828|United States|-5|condo| +7920|AAAAAAAAAPOBAAAA|985|Maple Third|Court|Suite 0|Newport|Johnson County|IL|61521|United States|-6|apartment| +7921|AAAAAAAABPOBAAAA|627|Franklin Green|Ln|Suite I|Buena Vista|Tolland County|CT|06352|United States|-5|apartment| +7922|AAAAAAAACPOBAAAA|666|Wilson View|Circle|Suite 320|Harmony|Adams County|ND|55804|United States|-6|apartment| +7923|AAAAAAAADPOBAAAA|408|10th Hill|Dr.|Suite 420|Dewey|San Juan County|NM|81160|United States|-7|apartment| +7924|AAAAAAAAEPOBAAAA|223|Mill |RD|Suite M|Oakland|Haines Borough|AK|99843|United States|-9|condo| +7925|AAAAAAAAFPOBAAAA|286|Spring |Street|Suite S|Pine Grove|New Madrid County|MO|64593|United States|-6|condo| +7926|AAAAAAAAGPOBAAAA|363|Park 5th|Circle|Suite O|Union Hill|Las Animas County|CO|87746|United States|-7|condo| +7927|AAAAAAAAHPOBAAAA|355|Washington Park|Boulevard|Suite P|Crossroads|Tazewell County|IL|60534|United States|-6|condo| +7928|AAAAAAAAIPOBAAAA|||||Walnut Grove||MA||United States|-5|| +7929|AAAAAAAAJPOBAAAA|457|Lincoln |ST|Suite 130|Mount Vernon|Franklin Parish|LA|78482|United States|-6|apartment| +7930|AAAAAAAAKPOBAAAA|464|Walnut |Cir.|Suite G|Belmont|Hopewell city|VA|20191|United States|-5|apartment| +7931|AAAAAAAALPOBAAAA|239|2nd North|Avenue|Suite L|Mountain View|Guernsey County|OH|44466|United States|-5|apartment| +7932|AAAAAAAAMPOBAAAA|319|Elm |Circle|Suite 450|Oakland|Clark County|MO|69843|United States|-6|single family| +7933|AAAAAAAANPOBAAAA||||Suite I|Woodland||||||condo| +7934|AAAAAAAAOPOBAAAA|288|2nd |Court|Suite G|Spring Hill|Robertson County|TX|76787|United States|-6|apartment| +7935|AAAAAAAAPPOBAAAA|165|Washington 1st|Circle|Suite H|Buena Vista|Ford County|IL|65752|United States|-6|single family| +7936|AAAAAAAAAAPBAAAA|373|Lakeview College|Drive|Suite L|Harmony|Williamson County|TX|75804|United States|-6|single family| +7937|AAAAAAAABAPBAAAA|546|7th Mill|Parkway|Suite I|Cedar Grove|Twin Falls County|ID|80411|United States|-7|single family| +7938|AAAAAAAACAPBAAAA|276|Franklin |Avenue|Suite 50|Pleasant Grove|Mobile County|AL|34136|United States|-6|condo| +7939|AAAAAAAADAPBAAAA|711|Sunset |Pkwy|Suite 0|Fairview|Sussex County|VA|25709|United States|-5|condo| +7940|AAAAAAAAEAPBAAAA|138|River Maple|Pkwy|Suite T|Lebanon|Morris County|TX|72898|United States|-6|condo| +7941|AAAAAAAAFAPBAAAA|477|Elm |Lane|Suite O|Vista|Lee County|MS|54694|United States|-6|apartment| +7942|AAAAAAAAGAPBAAAA|495|1st Franklin|Wy|Suite 320|Greenfield|Tioga County|PA|15038|United States|-5|condo| +7943|AAAAAAAAHAPBAAAA|803|Cedar |Boulevard|Suite 440|Marion|Darke County|OH|40399|United States|-5|single family| +7944|AAAAAAAAIAPBAAAA|820|Lake Main|Avenue|Suite L|River Oaks|Roosevelt County|NM|88075|United States|-7|apartment| +7945|AAAAAAAAJAPBAAAA|||Lane|Suite H|Lakewood||ND|58877|United States|-6|condo| +7946|AAAAAAAAKAPBAAAA|214|Johnson South||Suite F|New Hope|||09431|||| +7947|AAAAAAAALAPBAAAA|76|2nd |Lane|Suite 140|Ashley|Roberts County|TX|74324|United States|-6|condo| +7948|AAAAAAAAMAPBAAAA|175|Lincoln 6th|RD|Suite 310|Union|Garden County|NE|68721|United States|-6|single family| +7949|AAAAAAAANAPBAAAA|754|Hill |Court|Suite J|Cedar Grove|Zavala County|TX|70411|United States|-6|single family| +7950|AAAAAAAAOAPBAAAA|118|3rd |Parkway|Suite H|Lakewood|Cherry County|NE|68877|United States|-6|single family| +7951|AAAAAAAAPAPBAAAA|919|3rd First|Boulevard|Suite K|Glendale|Perry County|KY|43951|United States|-5|apartment| +7952|AAAAAAAAABPBAAAA|748|Meadow South|Street|Suite T|Marion|Anchorage Borough|AK|90399|United States|-9|apartment| +7953|AAAAAAAABBPBAAAA|291|10th 6th|Wy|Suite 210|Pleasant Hill|Haskell County|OK|73604|United States|-6|condo| +7954|AAAAAAAACBPBAAAA|939|Park Lake|Street|Suite Q|New Hope|Woodward County|OK|79431|United States|-6|condo| +7955|AAAAAAAADBPBAAAA|549|1st 4th|Blvd|Suite 200|Highland|Millard County|UT|89454|United States|-7|single family| +7956|AAAAAAAAEBPBAAAA|462|Ridge 11th|Boulevard|Suite 90|Edgewood|Dutchess County|NY|10069|United States|-5|condo| +7957|AAAAAAAAFBPBAAAA|451|7th Poplar|Street|Suite 440|Woodruff|Sanborn County|SD|54174|United States|-7|apartment| +7958|AAAAAAAAGBPBAAAA|156|Park Pine|Lane|Suite O|Greenfield|Howard County|TX|75038|United States|-6|condo| +7959|AAAAAAAAHBPBAAAA|243|Central |Ave|Suite M|Wolf Creek|Hickman County|TN|32455|United States|-5|apartment| +7960|AAAAAAAAIBPBAAAA|512|Fifth Hillcrest|Dr.|Suite 140|Mount Vernon|Warren County|MO|68482|United States|-6|condo| +7961|AAAAAAAAJBPBAAAA|702|Main View|Ave|Suite Y|Blair|Orange County|NC|25465|United States|-5|single family| +7962|AAAAAAAAKBPBAAAA|510|Central |Ln|Suite H|Bunker Hill|Jasper County|IN|40150|United States|-5|condo| +7963|AAAAAAAALBPBAAAA|887|Main Forest|Boulevard|Suite I|Highland|Linn County|IA|59454|United States|-6|condo| +7964|AAAAAAAAMBPBAAAA|||Avenue|Suite 190|Tabor||TN|38529|United States|-5|| +7965|AAAAAAAANBPBAAAA|938|Walnut Sycamore|Pkwy|Suite Y|Mount Vernon|Forest County|PA|18482|United States|-5|apartment| +7966|AAAAAAAAOBPBAAAA|501|Cedar |Street|Suite 470|Five Forks|Butler County|OH|42293|United States|-5|apartment| +7967|AAAAAAAAPBPBAAAA|65|Church Park|Road|Suite 220|Bethel|Stokes County|NC|25281|United States|-5|single family| +7968|AAAAAAAAACPBAAAA|171|Lake |Blvd|Suite L|Stringtown|Hodgeman County|KS|60162|United States|-6|single family| +7969|AAAAAAAABCPBAAAA|320|6th 8th|Ln|Suite O|Macedonia|Hamilton County|TX|71087|United States|-6|apartment| +7970|AAAAAAAACCPBAAAA|129|13th Elm|Circle|Suite 200|Saint James|Union County|MS|55799|United States|-6|single family| +7971|AAAAAAAADCPBAAAA|57||Boulevard|Suite S|Union||TX||||| +7972|AAAAAAAAECPBAAAA|272|Lake Mill|Circle|Suite 210|Waterloo|Dundy County|NE|61675|United States|-6|condo| +7973|AAAAAAAAFCPBAAAA|360|Valley |Blvd|Suite U|Pleasant Hill|McKenzie County|ND|53604|United States|-6|single family| +7974|AAAAAAAAGCPBAAAA|633|3rd Pine|Avenue|Suite 160|Glenwood|Uintah County|UT|83511|United States|-7|apartment| +7975|AAAAAAAAHCPBAAAA|520|Wilson |ST|Suite L|Centerville|Chase County|NE|60059|United States|-6|condo| +7976|AAAAAAAAICPBAAAA|793|Hill 5th|RD|Suite P|Union Hill|Maries County|MO|67746|United States|-6|single family| +7977|AAAAAAAAJCPBAAAA|880|Fifth |Blvd|Suite 460|Lincoln|Ashtabula County|OH|41289|United States|-5|condo| +7978|AAAAAAAAKCPBAAAA|145|Walnut |Street|Suite 40|White Oak|Simpson County|MS|56668|United States|-6|apartment| +7979|AAAAAAAALCPBAAAA|671|6th |Cir.|Suite U|Oakland|Wood County|WI|59843|United States|-6|single family| +7980|AAAAAAAAMCPBAAAA|477|Highland |Avenue|Suite W|Mount Olive|Warren County|OH|48059|United States|-5|condo| +7981|AAAAAAAANCPBAAAA|||Cir.|Suite 440|Riverdale|Tillamook County||99391|||single family| +7982|AAAAAAAAOCPBAAAA|901|12th |Lane|Suite S|Waterloo|Big Horn County|MT|61675|United States|-7|single family| +7983|AAAAAAAAPCPBAAAA|559|10th |Parkway|Suite 400|Bunker Hill|Hyde County|SD|50150|United States|-7|apartment| +7984|AAAAAAAAADPBAAAA|666|Ridge Park|Ave|Suite 180|Plainview|Cleburne County|AL|33683|United States|-6|apartment| +7985|AAAAAAAABDPBAAAA|251|6th |Cir.|Suite J|Marion|Kenosha County|WI|50399|United States|-6|apartment| +7986|AAAAAAAACDPBAAAA|692|Main |RD|Suite G|Greenwood|Montgomery County|NY|18828|United States|-5|condo| +7987|AAAAAAAADDPBAAAA||7th Railroad||Suite W|Tabor|||||-6|| +7988|AAAAAAAAEDPBAAAA|933||Drive|Suite C|Bunker Hill|||||-7|single family| +7989|AAAAAAAAFDPBAAAA|125|Park Main|Avenue|Suite D|Belmont|San Miguel County|NM|80191|United States|-7|single family| +7990|AAAAAAAAGDPBAAAA|523|Lee 14th|Lane|Suite 90|White Oak|Glynn County|GA|36668|United States|-5|condo| +7991|AAAAAAAAHDPBAAAA|485|Center Woodland|Avenue|Suite S|Forest Hills|Northwest Arctic Borough|AK|99237|United States|-9|apartment| +7992|AAAAAAAAIDPBAAAA|179|Oak |Cir.|Suite 470|Mount Pleasant|Santa Cruz County|CA|91933|United States|-8|condo| +7993|AAAAAAAAJDPBAAAA|390|Railroad |Ct.|Suite E|Oak Hill|Lyon County|KS|67838|United States|-6|single family| +7994|AAAAAAAAKDPBAAAA|114|10th |Way|Suite R|Hamilton|Rockdale County|GA|32808|United States|-5|apartment| +7995|AAAAAAAALDPBAAAA|280|Williams 14th|Court|Suite N|Fairview|Shasta County|CA|95709|United States|-8|apartment| +7996|AAAAAAAAMDPBAAAA|879|6th |Blvd|Suite 100|Kingston|Searcy County|AR|74975|United States|-6|single family| +7997|AAAAAAAANDPBAAAA|407|Elm |Street|Suite 400|Midway|Buffalo County|NE|61904|United States|-6|single family| +7998|AAAAAAAAODPBAAAA|954|Green Spring|Circle|Suite B|Harmony|Pulaski County|GA|35804|United States|-5|single family| +7999|AAAAAAAAPDPBAAAA|893|Davis East|RD|Suite 480|Spring Valley|Gasconade County|MO|66060|United States|-6|condo| +8000|AAAAAAAAAEPBAAAA|56|Walnut Washington|Ave|Suite 50|Salem|Ford County|KS|68048|United States|-6|condo| +8001|AAAAAAAABEPBAAAA||Jefferson |||Waterloo|Eddy County||51675||-6|| +8002|AAAAAAAACEPBAAAA|806|2nd 3rd|Dr.|Suite 120|Summerville|Lynn County|TX|72033|United States|-6|condo| +8003|AAAAAAAADEPBAAAA|769|Railroad |Court|Suite 430|Shiloh|Iowa County|WI|59275|United States|-6|condo| +8004|AAAAAAAAEEPBAAAA|965|East |Dr.|Suite N|Smyrna|Erath County|TX|78119|United States|-6|single family| +8005|AAAAAAAAFEPBAAAA|417|Johnson |Way|Suite F|Lakeview|Loup County|NE|68579|United States|-7|apartment| +8006|AAAAAAAAGEPBAAAA|920|Hill |Way|Suite 260|Oakdale|Yamhill County|OR|99584|United States|-8|apartment| +8007|AAAAAAAAHEPBAAAA|600|View |Wy|Suite E|Riverside|Dakota County|NE|69231|United States|-6|apartment| +8008|AAAAAAAAIEPBAAAA|249|Laurel |Ave|Suite 480|Kingston|Trempealeau County|WI|54975|United States|-6|single family| +8009|AAAAAAAAJEPBAAAA|101|Cedar Park|Road|Suite 160|Greenwood|Scotland County|NC|28828|United States|-5|single family| +8010|AAAAAAAAKEPBAAAA|216|Maple Walnut|Ave|Suite 310|Greenfield|Glascock County|GA|35038|United States|-5|single family| +8011|AAAAAAAALEPBAAAA|148|14th Cherry|RD|Suite 240|Cedar Grove|Tipton County|TN|30411|United States|-6|condo| +8012|AAAAAAAAMEPBAAAA|729|Main |RD|Suite E|Cherry Valley|Butler County|OH|40854|United States|-5|condo| +8013|AAAAAAAANEPBAAAA|600|Woodland |Drive|Suite 260|Concord|Greensville County|VA|24107|United States|-5|single family| +8014|AAAAAAAAOEPBAAAA|267|Sunset |Cir.|Suite 180|Avery|Hamilton County|NY|10194|United States|-5|single family| +8015|AAAAAAAAPEPBAAAA|517|Willow |Street|Suite 70|Arlington|Clark County|WA|96557|United States|-8|condo| +8016|AAAAAAAAAFPBAAAA|184|Third |Wy|Suite 70|Leon|Frederick County|VA|20913|United States|-5|apartment| +8017|AAAAAAAABFPBAAAA|107|Park Ridge|Ct.|Suite 390|Highland|Mendocino County|CA|99454|United States|-8|apartment| +8018|AAAAAAAACFPBAAAA|895|6th |Ln|Suite B|Mount Pleasant|Shelby County|TN|31933|United States|-6|condo| +8019|AAAAAAAADFPBAAAA|792|Poplar |Street|Suite 420|Summerville|Aitkin County|MN|52033|United States|-6|apartment| +8020|AAAAAAAAEFPBAAAA|219|Oak |Pkwy|Suite O|Johnsonville|Greene County|TN|37745|United States|-5|condo| +8021|AAAAAAAAFFPBAAAA|434|Johnson |Road|Suite B|Harmony|Lawrence County|OH|45804|United States|-5|apartment| +8022|AAAAAAAAGFPBAAAA|822|3rd |Wy|Suite C|Clinton|Jeff Davis County|GA|38222|United States|-5|single family| +8023|AAAAAAAAHFPBAAAA|299|Park 14th|Lane|Suite 250|Wesley|Gadsden County|FL|31218|United States|-5|single family| +8024|AAAAAAAAIFPBAAAA|236|Fifth |Road|Suite 410|Fairview|DeBaca County|NM|85709|United States|-7|condo| +8025|AAAAAAAAJFPBAAAA|834|Washington Laurel|Street|Suite T|Farmington|Morris County|TX|79145|United States|-6|single family| +8026|AAAAAAAAKFPBAAAA|967|3rd |Boulevard|Suite 260|Newport|Russell County|KS|61521|United States|-6|condo| +8027|AAAAAAAALFPBAAAA|441|Central Cedar|RD|Suite P|Mount Olive|Carlton County|MN|58059|United States|-6|condo| +8028|AAAAAAAAMFPBAAAA|311|5th |Way|Suite R|Five Forks|Leon County|TX|72293|United States|-6|condo| +8029|AAAAAAAANFPBAAAA|458|7th Madison|Blvd|Suite G|Mountain View|Lackawanna County|PA|14466|United States|-5|single family| +8030|AAAAAAAAOFPBAAAA|393|Main Poplar|RD|Suite O|Gravel Hill|Monroe County|NY|11944|United States|-5|condo| +8031|AAAAAAAAPFPBAAAA|897|3rd Church|Wy|Suite I|Fairview|Van Wert County|OH|45709|United States|-5|condo| +8032|AAAAAAAAAGPBAAAA||||Suite 450||Wasatch County|UT|||-7|| +8033|AAAAAAAABGPBAAAA|221|Birch Forest|Way|Suite K|Highland|Sherman County|KS|69454|United States|-6|condo| +8034|AAAAAAAACGPBAAAA|586|Cedar |Dr.|Suite 250|Waterloo|Cook County|GA|31675|United States|-5|apartment| +8035|AAAAAAAADGPBAAAA|327|Adams |ST|Suite D|Salem|Harlan County|NE|68048|United States|-6|apartment| +8036|AAAAAAAAEGPBAAAA|600|3rd |ST|Suite Y|Bridgeport|Dickinson County|MI|45817|United States|-5|single family| +8037|AAAAAAAAFGPBAAAA|868|Highland |Ave|Suite E|Oakdale|Traverse County|MN|59584|United States|-6|condo| +8038|AAAAAAAAGGPBAAAA|969|4th Elm|Ln|Suite C|Somerville|Denali Borough|AK|97783|United States|-9|apartment| +8039|AAAAAAAAHGPBAAAA||Locust ||Suite W||Williamson County|TX||United States||condo| +8040|AAAAAAAAIGPBAAAA|789|North |Lane|Suite F|Lincoln|Ashtabula County|OH|41289|United States|-5|condo| +8041|AAAAAAAAJGPBAAAA|406||Court||Franklin||||||| +8042|AAAAAAAAKGPBAAAA|257|Dogwood Broadway|Parkway|Suite K|Sunnyside|Ascension Parish|LA|71952|United States|-6|single family| +8043|AAAAAAAALGPBAAAA|558|Elm Adams|Circle|Suite 260|Sunnyside|Jackson County|TX|71952|United States|-6|apartment| +8044|AAAAAAAAMGPBAAAA|158|10th Smith|Street|Suite 440|Lakeside|Santa Cruz County|CA|99532|United States|-8|condo| +8045|AAAAAAAANGPBAAAA|893|Oak |Road|Suite M|Oak Grove|Dodge County|GA|38370|United States|-5|condo| +8046|AAAAAAAAOGPBAAAA|506|9th Oak|Ave|Suite 320|Georgetown|Ogemaw County|MI|47057|United States|-5|apartment| +8047|AAAAAAAAPGPBAAAA|797|Cedar Forest|Way|Suite L|Marion|Calhoun County|FL|30399|United States|-5|condo| +8048|AAAAAAAAAHPBAAAA|146|Oak 12th|Way|Suite S|Georgetown|Cass County|IL|67057|United States|-6|condo| +8049|AAAAAAAABHPBAAAA|320|Elm Highland|Ln|Suite T|Oakwood|Worth County|MO|60169|United States|-6|apartment| +8050|AAAAAAAACHPBAAAA|89|10th 4th|Avenue|Suite 260|Red Hill|McPherson County|SD|54338|United States|-7|single family| +8051|AAAAAAAADHPBAAAA|954|Woodland Johnson|Avenue|Suite X|Summit|Coconino County|AZ|80499|United States|-7|single family| +8052|AAAAAAAAEHPBAAAA|664|Elm Johnson|Parkway|Suite O|Clinton|Bedford County|VA|28222|United States|-5|apartment| +8053|AAAAAAAAFHPBAAAA|806|Meadow |Street|Suite 440|Union Hill|Rutherford County|TN|37746|United States|-6|single family| +8054|AAAAAAAAGHPBAAAA|805|Oak Meadow|ST|Suite 250|Buena Vista|Montgomery County|IA|55752|United States|-6|condo| +8055|AAAAAAAAHHPBAAAA|431||||Deerfield||||United States||apartment| +8056|AAAAAAAAIHPBAAAA||Madison View|Ct.||||||||| +8057|AAAAAAAAJHPBAAAA|823|15th Hickory|Ave|Suite 410|Friendship|Miami County|OH|44536|United States|-5|single family| +8058|AAAAAAAAKHPBAAAA|594|Fourteenth |Street|Suite 480|Hillcrest|Steele County|MN|53003|United States|-6|condo| +8059|AAAAAAAALHPBAAAA|982|Jackson Jackson|Ln|Suite 430|Wilson|Providence County|RI|07571|United States|-5|single family| +8060|AAAAAAAAMHPBAAAA|513|4th Main|Ln|Suite W|Lenox|Pontotoc County|OK|71143|United States|-6|condo| +8061|AAAAAAAANHPBAAAA|736|Spring |Parkway|Suite P|Buena Vista|Lewis and Clark County|MT|65752|United States|-7|condo| +8062|AAAAAAAAOHPBAAAA|68|Second |Parkway|Suite 150|Wright|Brown County|WI|52814|United States|-6|apartment| +8063|AAAAAAAAPHPBAAAA|864|1st 1st|Ave|Suite 360|Kingston|Jackson County|WI|54975|United States|-6|apartment| +8064|AAAAAAAAAIPBAAAA|604|2nd |Ct.|Suite Y|Concord|Middlesex County|VA|24107|United States|-5|single family| +8065|AAAAAAAABIPBAAAA|698|Adams |Court|Suite H|Green Acres|Caddo Parish|LA|77683|United States|-6|single family| +8066|AAAAAAAACIPBAAAA|299|Franklin |Drive|Suite U|Warwick|Taos County|NM|81398|United States|-7|condo| +8067|AAAAAAAADIPBAAAA|681|Sunset Oak|Avenue|Suite M|Maple Grove|Dawson County|NE|68252|United States|-6|apartment| +8068|AAAAAAAAEIPBAAAA|855|Meadow |Circle|Suite U|Enterprise|Talbot County|GA|31757|United States|-5|single family| +8069|AAAAAAAAFIPBAAAA|513|9th River|Boulevard|Suite S|Brooklyn|Real County|TX|71103|United States|-6|condo| +8070|AAAAAAAAGIPBAAAA|545|Lake |Way|Suite A|Salem|Itasca County|MN|58048|United States|-6|condo| +8071|AAAAAAAAHIPBAAAA|506|Woodland Spring|Way|Suite N|Mount Olive|Botetourt County|VA|28059|United States|-5|condo| +8072|AAAAAAAAIIPBAAAA|751|Park |Avenue|Suite D|Lakewood|Shelby County|OH|48877|United States|-5|single family| +8073|AAAAAAAAJIPBAAAA|714|Fourth Church|Avenue|Suite T|Avery|Page County|IA|50194|United States|-6|apartment| +8074|AAAAAAAAKIPBAAAA|725|Washington |Pkwy|Suite F|Liberty|Hudspeth County|TX|73451|United States|-6|condo| +8075|AAAAAAAALIPBAAAA|280|4th Main|Court|Suite 180|Maple Grove|Bon Homme County|SD|58252|United States|-6|condo| +8076|AAAAAAAAMIPBAAAA||Oak Jackson|||Cordova|Eaton County|MI||United States||condo| +8077|AAAAAAAANIPBAAAA|688|Church |Wy|Suite 0|Woodland|Alpine County|CA|94854|United States|-8|single family| +8078|AAAAAAAAOIPBAAAA|140|Second |Ln|Suite A|Unionville|Orangeburg County|SC|21711|United States|-5|condo| +8079|AAAAAAAAPIPBAAAA|627|11th |Parkway|Suite 130|Stringtown|Idaho County|ID|80162|United States|-7|single family| +8080|AAAAAAAAAJPBAAAA|783|Tenth Ash|Court|Suite R|Macedonia|Monroe County|MO|61087|United States|-6|condo| +8081|AAAAAAAABJPBAAAA|172|Wilson |Circle|Suite C|Franklin|Schoharie County|NY|19101|United States|-5|single family| +8082|AAAAAAAACJPBAAAA|493|7th 4th|Avenue|Suite V|Liberty|Eagle County|CO|83451|United States|-7|apartment| +8083|AAAAAAAADJPBAAAA|274|Pine Walnut|Dr.|Suite C|Glendale|Bureau County|IL|63951|United States|-6|single family| +8084|AAAAAAAAEJPBAAAA|658|First Highland|Street|Suite G|Greenfield|Gilpin County|CO|85038|United States|-7|single family| +8085|AAAAAAAAFJPBAAAA|579|West |Blvd|Suite C|Lincoln|Harding County|NM|81289|United States|-7|apartment| +8086|AAAAAAAAGJPBAAAA|771|Forest Main|RD|Suite 30|Concord|Lancaster County|VA|24107|United States|-5|condo| +8087|AAAAAAAAHJPBAAAA|397|Railroad 2nd|Boulevard|Suite 100|Kirkland|Pushmataha County|OK|77896|United States|-6|condo| +8088|AAAAAAAAIJPBAAAA|438|Miller |Wy|Suite 260|Green Acres|Jackson County|MN|57683|United States|-6|apartment| +8089|AAAAAAAAJJPBAAAA|301|Spring View|Wy|Suite 340|Salem|Gates County|NC|28048|United States|-5|apartment| +8090|AAAAAAAAKJPBAAAA|506|North Dogwood|Pkwy|Suite F|Newtown|Pinal County|AZ|81749|United States|-7|apartment| +8091|AAAAAAAALJPBAAAA|148|6th |Street|Suite 10|Midway|Toole County|MT|61904|United States|-7|apartment| +8092|AAAAAAAAMJPBAAAA|711|Eigth |Parkway|Suite T|Newtown|Elmore County|ID|81749|United States|-7|condo| +8093|AAAAAAAANJPBAAAA|||||Shiloh|Coos County||09875||-5|single family| +8094|AAAAAAAAOJPBAAAA|966|Cherry |Ln|Suite Y|Unionville|Chester County|TN|31711|United States|-5|apartment| +8095|AAAAAAAAPJPBAAAA|482|10th |Ave|Suite 450|Liberty|Lampasas County|TX|73451|United States|-6|condo| +8096|AAAAAAAAAKPBAAAA|626|5th |Court|Suite 230|Ashley|Bracken County|KY|44324|United States|-6|single family| +8097|AAAAAAAABKPBAAAA|503|Chestnut |Ln|Suite D|Forest Hills|Hartley County|TX|79237|United States|-6|single family| +8098|AAAAAAAACKPBAAAA|782|Sunset Green|Court|Suite X|Clifton|Hardeman County|TX|78014|United States|-6|single family| +8099|AAAAAAAADKPBAAAA|9|2nd Central|Avenue|Suite 220|Harmony|Osage County|OK|75804|United States|-6|apartment| +8100|AAAAAAAAEKPBAAAA|133|Center Chestnut|Ave|Suite 340|Fisher|Breckinridge County|KY|42819|United States|-6|apartment| +8101|AAAAAAAAFKPBAAAA|996|Willow |Drive|Suite 260|Brownsville|Hamilton County|TN|39310|United States|-5|condo| +8102|AAAAAAAAGKPBAAAA|355|West Spring|Circle|Suite 370|Fairfield|Campbell County|WY|86192|United States|-7|single family| +8103|AAAAAAAAHKPBAAAA|215|Pine Spruce|Boulevard|Suite 100|Wildwood|Bienville Parish|LA|76871|United States|-6|single family| +8104|AAAAAAAAIKPBAAAA|441|Meadow Ridge|Parkway|Suite U|Shady Grove|Stillwater County|MT|62812|United States|-7|condo| +8105|AAAAAAAAJKPBAAAA|115|Pine |Cir.|Suite N|Glenwood|Perkins County|NE|63511|United States|-7|apartment| +8106|AAAAAAAAKKPBAAAA|580|Jefferson |Drive|Suite H|Riverdale|Somervell County|TX|79391|United States|-6|apartment| +8107|AAAAAAAALKPBAAAA|382|Fourth |Avenue|Suite E|Wilson|Dallam County|TX|76971|United States|-6|single family| +8108|AAAAAAAAMKPBAAAA|387|5th 8th|Court|Suite M|Midway|Ouachita Parish|LA|71904|United States|-6|condo| +8109|AAAAAAAANKPBAAAA||South 7th|Ct.|Suite 330|||CO||United States||| +8110|AAAAAAAAOKPBAAAA|495|Birch North|Road|Suite 280|Hamilton|Johnson County|GA|32808|United States|-5|apartment| +8111|AAAAAAAAPKPBAAAA|638|Ash Johnson|Avenue|Suite O|Union|Harrisonburg city|VA|28721|United States|-5|single family| +8112|AAAAAAAAALPBAAAA|51|Cedar Johnson|Dr.|Suite 290|Adrian|Wayne County|UT|83301|United States|-7|single family| +8113|AAAAAAAABLPBAAAA|140|Ridge |Avenue|Suite T|Fairview|Hancock County|IA|55709|United States|-6|single family| +8114|AAAAAAAACLPBAAAA|431|2nd Elm|Blvd|Suite 10|||||United States||| +8115|AAAAAAAADLPBAAAA|728|Second Washington|Ave|Suite G|Oakland|Blaine County|OK|79843|United States|-6|single family| +8116|AAAAAAAAELPBAAAA|343|Chestnut |Cir.|Suite 230|Newport|Baldwin County|GA|31521|United States|-5|single family| +8117|AAAAAAAAFLPBAAAA|180|4th |Drive|Suite A|Jackson|San Diego County|CA|99583|United States|-8|condo| +8118|AAAAAAAAGLPBAAAA|8|8th 6th|Pkwy|Suite N|Oak Ridge|Dubois County|IN|48371|United States|-5|condo| +8119|AAAAAAAAHLPBAAAA|106|View Hickory|Ave|Suite L|Buena Vista|De Kalb County|IN|45752|United States|-5|apartment| +8120|AAAAAAAAILPBAAAA|533|Jefferson |Wy|Suite 300|Maple Grove|Hansford County|TX|78252|United States|-6|single family| +8121|AAAAAAAAJLPBAAAA|979|2nd 10th|Pkwy|Suite 490|Greenville|Johnson County|IN|41387|United States|-5|condo| +8122|AAAAAAAAKLPBAAAA|17|Fifth |Lane||Pleasant Grove|Butler County|NE|64136|||apartment| +8123|AAAAAAAALLPBAAAA|722|Central |Boulevard|Suite 30|Sulphur Springs|Hancock County|ME|08954|United States|-5|condo| +8124|AAAAAAAAMLPBAAAA|429|First Hickory|Cir.|Suite 460|Silver Creek|Sully County|SD|54546|United States|-7|apartment| +8125|AAAAAAAANLPBAAAA|389|Park North|Ct.|Suite 90|Pleasant Valley|Hamilton County|NE|62477|United States|-6|single family| +8126|AAAAAAAAOLPBAAAA|422|3rd 1st|Ct.|Suite X|Forestville|Roosevelt County|NM|83027|United States|-7|apartment| +8127|AAAAAAAAPLPBAAAA|978|Hickory Forest|RD|Suite I|Ashland|Collin County|TX|74244|United States|-6|single family| +8128|AAAAAAAAAMPBAAAA|513|12th |Drive|Suite 190|Buena Vista|Roane County|TN|35752|United States|-6|condo| +8129|AAAAAAAABMPBAAAA|656|Park Johnson|Road|Suite G|Flatwoods|Delaware County|OH|44212|United States|-5|apartment| +8130|AAAAAAAACMPBAAAA|387|6th Maple|Court|Suite F|Springdale|Wilcox County|AL|38883|United States|-6|single family| +8131|AAAAAAAADMPBAAAA|140|View Madison|Boulevard|Suite 220|Cedar Grove|Steele County|ND|50411|United States|-6|condo| +8132|AAAAAAAAEMPBAAAA|178|Johnson Smith|Dr.|Suite M|Ellisville|Greene County|OH|46820|United States|-5|condo| +8133|AAAAAAAAFMPBAAAA|172|Pine Ridge|Boulevard|Suite Y|Newport|Bossier Parish|LA|71521|United States|-6|apartment| +8134|AAAAAAAAGMPBAAAA|219|Railroad Park|Ct.|Suite 360|Oakwood|Faribault County|MN|50169|United States|-6|apartment| +8135|AAAAAAAAHMPBAAAA|196|Franklin |Pkwy|Suite 430|Macedonia|Wright County|IA|51087|United States|-6|single family| +8136|AAAAAAAAIMPBAAAA|394|Poplar |Ct.|Suite V|Bunker Hill|Randolph County|AL|30150|United States|-6|condo| +8137|AAAAAAAAJMPBAAAA|421|Lakeview Pine|Drive|Suite 200|Riverview|Hale County|TX|79003|United States|-6|apartment| +8138|AAAAAAAAKMPBAAAA||Cedar Johnson|||||||United States||single family| +8139|AAAAAAAALMPBAAAA||North 4th|Circle|||Morgan County|TN|33604|||apartment| +8140|AAAAAAAAMMPBAAAA||Johnson ||Suite W|Lawrenceville|Lewis County|||||| +8141|AAAAAAAANMPBAAAA|484|Cherry 11th|Court|Suite J|Sunnyside|Chesapeake city|VA|21952|United States|-5|single family| +8142|AAAAAAAAOMPBAAAA|987|13th |Court|Suite C|Hamilton|Calhoun County|SC|22808|United States|-5|single family| +8143|AAAAAAAAPMPBAAAA|932|Sunset |Cir.|Suite F|Florence|Crittenden County|KY|43394|United States|-6|single family| +8144|AAAAAAAAANPBAAAA|55|Adams |Drive|Suite 180|Stringtown|Oconee County|SC|20162|United States|-5|apartment| +8145|AAAAAAAABNPBAAAA|851|Lake View|ST|Suite Y|Mount Pleasant|Russell County|VA|21933|United States|-5|single family| +8146|AAAAAAAACNPBAAAA|759|Smith |Parkway|Suite 170|Pleasant Grove|Ionia County|MI|44136|United States|-5|condo| +8147|AAAAAAAADNPBAAAA|414|Sunset |Ct.|Suite A|Farmington|Luce County|MI|49145|United States|-5|single family| +8148|AAAAAAAAENPBAAAA|998|Washington Sunset|Wy|Suite 40|Liberty|Butler County|KS|63451|United States|-6|single family| +8149|AAAAAAAAFNPBAAAA|59|Main Hickory|Avenue|Suite 280|Blair|Lewis County|NY|15465|United States|-5|condo| +8150|AAAAAAAAGNPBAAAA|842|Jefferson 5th|Ave|Suite 370|Wilson|Quay County|NM|86971|United States|-7|single family| +8151|AAAAAAAAHNPBAAAA|43|9th Railroad|Way|Suite V|Five Points|Providence County|RI|06698|United States|-5|single family| +8152|AAAAAAAAINPBAAAA|247|Church Railroad|Ave|Suite V|Pleasant Valley|Lipscomb County|TX|72477|United States|-6|condo| +8153|AAAAAAAAJNPBAAAA|82|Mill |Wy|Suite 320|Crossroads|Portsmouth city|VA|20534|United States|-5|single family| +8154|AAAAAAAAKNPBAAAA|518|7th |Drive|Suite N|Five Points|Real County|TX|76098|United States|-6|single family| +8155|AAAAAAAALNPBAAAA|180|Ash |Boulevard|Suite I|Brownsville|Hunt County|TX|79310|United States|-6|condo| +8156|AAAAAAAAMNPBAAAA|961|7th |Circle|Suite 30|Florence|Montcalm County|MI|43394|United States|-5|single family| +8157|AAAAAAAANNPBAAAA|316|2nd Park|Way|Suite 160|Pleasant Hill|Franklin County|NY|13604|United States|-5|apartment| +8158|AAAAAAAAONPBAAAA|960|5th |RD|Suite 380|Hillcrest|Lake County|MI|43003|United States|-5|condo| +8159|AAAAAAAAPNPBAAAA|588|Lincoln |Dr.|Suite 120|Plainview|Cuming County|NE|63683|United States|-6|apartment| +8160|AAAAAAAAAOPBAAAA|694|Jefferson 11th|Way|Suite 180|Woodville|Wasatch County|UT|84289|United States|-7|single family| +8161|AAAAAAAABOPBAAAA|586|7th |Drive|Suite S|Liberty|Buffalo County|SD|53451|United States|-6|single family| +8162|AAAAAAAACOPBAAAA|700|Poplar |Street|Suite 220|Franklin|Adams County|IA|59101|United States|-6|condo| +8163|AAAAAAAADOPBAAAA|425|College Woodland|Avenue|Suite 120|Brownsville|New York County|NY|19310|United States|-5|single family| +8164|AAAAAAAAEOPBAAAA|106|Washington Ridge|Avenue|Suite 460|Highland Park|Sanpete County|UT|86534|United States|-7|single family| +8165|AAAAAAAAFOPBAAAA|396|12th West|Road|Suite 230|Pine Grove|Union County|SD|54593|United States|-7|apartment| +8166|AAAAAAAAGOPBAAAA|338|9th |Road|Suite S|Brownsville|Sweetwater County|WY|89310|United States|-7|condo| +8167|AAAAAAAAHOPBAAAA|717|Washington |RD|Suite 310|Pleasant Hill|Baker County|FL|33604|United States|-5|single family| +8168|AAAAAAAAIOPBAAAA|80|First Meadow|Avenue|Suite O|Rosewood|Greeley County|NE|62205|United States|-6|apartment| +8169|AAAAAAAAJOPBAAAA|747|Hill Cedar|Boulevard|Suite V|Red Oak|Lake County|OH|45018|United States|-5|apartment| +8170|AAAAAAAAKOPBAAAA|||Road|Suite M||McDonald County|MO|65709|||| +8171|AAAAAAAALOPBAAAA|243|Hill |RD|Suite M|Antioch|Muscatine County|IA|58605|United States|-6|apartment| +8172|AAAAAAAAMOPBAAAA|447|East First|Drive|Suite 120|Providence|Hot Springs County|WY|86614|United States|-7|apartment| +8173|AAAAAAAANOPBAAAA|693|South Sunset|Court|Suite 370|Hamilton|Greene County|AR|72808|United States|-6|condo| +8174|AAAAAAAAOOPBAAAA|952|Hill Madison|RD|Suite J|Plainview|Irwin County|GA|33683|United States|-5|single family| +8175|AAAAAAAAPOPBAAAA|299|3rd |Circle|Suite J|Lebanon|Smith County|MS|52898|United States|-6|condo| +8176|AAAAAAAAAPPBAAAA|10|River North|Boulevard|Suite Q|Maple Grove|Natchitoches Parish|LA|78252|United States|-6|single family| +8177|AAAAAAAABPPBAAAA|776|4th Lincoln|ST|Suite U|Union Hill|Clay County|FL|37746|United States|-5|condo| +8178|AAAAAAAACPPBAAAA|704|Elm Main|Circle|Suite D|Florence|Wheeler County|NE|63394|United States|-7|single family| +8179|AAAAAAAADPPBAAAA|954|Main Poplar|Wy|Suite C|Lincoln|Jones County|MS|51289|United States|-6|single family| +8180|AAAAAAAAEPPBAAAA|825|5th Washington|Parkway|Suite K|Deerfield|Bacon County|GA|39840|United States|-5|single family| +8181|AAAAAAAAFPPBAAAA|764|4th 3rd|Wy|Suite 440|Freeman|Polk County|OR|92297|United States|-8|single family| +8182|AAAAAAAAGPPBAAAA|965|Cherry Third|Ave|Suite 230|Greenwood|Montgomery County|MD|28828|United States|-5|apartment| +8183|AAAAAAAAHPPBAAAA|169|Church |Dr.|Suite V|New Hope|Wayne County|UT|89431|United States|-7|apartment| +8184|AAAAAAAAIPPBAAAA|731|Adams Lakeview|Wy|Suite R|Riverside|Pottawatomie County|KS|69231|United States|-6|condo| +8185|AAAAAAAAJPPBAAAA|775|Sunset Sycamore|Ct.|Suite 130|Brownsville|Fayette County|IN|49310|United States|-5|condo| +8186|AAAAAAAAKPPBAAAA|147|Church 6th|Road|Suite I|Hopewell|Trego County|KS|60587|United States|-6|apartment| +8187|AAAAAAAALPPBAAAA|480|8th |RD|Suite O|Ashland|McCracken County|KY|44244|United States|-5|single family| +8188|AAAAAAAAMPPBAAAA|8|Main 3rd|Ave|Suite 20|Lakeside|Tioga County|PA|19532|United States|-5|apartment| +8189|AAAAAAAANPPBAAAA|718|Center Green|Pkwy|Suite S|Plainview|Lassen County|CA|93683|United States|-8|apartment| +8190|AAAAAAAAOPPBAAAA|96|Elm Hill|Boulevard|Suite A|Burns|Newberry County|SC|25272|United States|-5|apartment| +8191|AAAAAAAAPPPBAAAA|590|2nd |Avenue|Suite Y|Valley View|Burleson County|TX|75124|United States|-6|single family| +8192|AAAAAAAAAAACAAAA|232|8th Sunset|Court|Suite L|Mountain View|Swift County|MN|54466|United States|-6|condo| +8193|AAAAAAAABAACAAAA||Chestnut |||Waterloo|Montmorency County|||United States|-5|single family| +8194|AAAAAAAACAACAAAA|218|First Hillcrest|Ct.|Suite O|Lakeview|Douglas County|CO|88579|United States|-7|apartment| +8195|AAAAAAAADAACAAAA|39|Lee Adams|Wy|Suite G|Unionville|Somervell County|TX|71711|United States|-6|apartment| +8196|AAAAAAAAEAACAAAA|518|Adams 2nd|Cir.|Suite R|Macedonia|Warren County|KY|41087|United States|-5|single family| +8197|AAAAAAAAFAACAAAA|2|Elm College|Pkwy|Suite 130|Pleasant Hill|Hancock County|ME|04204|United States|-5|condo| +8198|AAAAAAAAGAACAAAA|211|Elm |Ct.|Suite 340|Midway|Henrico County|VA|21904|United States|-5|single family| +8199|AAAAAAAAHAACAAAA|505|9th |Ave|Suite K|Florence|Clackamas County|OR|93394|United States|-8|single family| +8200|AAAAAAAAIAACAAAA|865|5th |Ln|Suite R|Guthrie|Montmorency County|MI|41423|United States|-5|condo| +8201|AAAAAAAAJAACAAAA|891|9th |ST|Suite E|Summit|Northampton County|VA|20499|United States|-5|single family| +8202|AAAAAAAAKAACAAAA|856|Birch Forest|Cir.|Suite 430|Pleasant Valley|Kendall County|TX|72477|United States|-6|single family| +8203|AAAAAAAALAACAAAA|965|Davis Hillcrest|Circle|Suite T|Stringtown|Gilliam County|OR|90162|United States|-8|apartment| +8204|AAAAAAAAMAACAAAA|590|Green Center|Pkwy|Suite 380|New Town|Latimer County|OK|79634|United States|-6|condo| +8205|AAAAAAAANAACAAAA|606|Park |Blvd|Suite G|Woodlawn|Somerset County|MD|24098|United States|-5|single family| +8206|AAAAAAAAOAACAAAA|750|Miller Elm|Ave|Suite M|Bridgeport|Perry County|MO|65817|United States|-6|single family| +8207|AAAAAAAAPAACAAAA|60||Ct.|||Newton County|AR|||-6|condo| +8208|AAAAAAAAABACAAAA|799|Williams |Cir.|Suite 350|Mount Zion|Twiggs County|GA|38054|United States|-5|apartment| +8209|AAAAAAAABBACAAAA|97|Forest Wilson|Way|Suite 0|Ashland|Tioga County|PA|14244|United States|-5|apartment| +8210|AAAAAAAACBACAAAA|972|Highland |Pkwy|Suite 350|Concord|Steuben County|NY|14107|United States|-5|condo| +8211|AAAAAAAADBACAAAA|270|College 2nd|Dr.|Suite 270|Oakdale|Park County|WY|89584|United States|-7|single family| +8212|AAAAAAAAEBACAAAA|166|Dogwood |Street|Suite 380|Providence|Jasper County|TX|76614|United States|-6|apartment| +8213|AAAAAAAAFBACAAAA|111|Willow 14th|Cir.|Suite L|Mount Olive|Tyler County|TX|78059|United States|-6|single family| +8214|AAAAAAAAGBACAAAA|40|Johnson |Drive|Suite M|Montpelier|King County|TX|78930|United States|-6|apartment| +8215|AAAAAAAAHBACAAAA|622|First Sycamore|Lane|Suite 40|Franklin|Jasper County|SC|29101|United States|-5|condo| +8216|AAAAAAAAIBACAAAA|61|Park |Dr.|Suite 250|Belmont|Rice County|MN|50191|United States|-6|single family| +8217|AAAAAAAAJBACAAAA|181|9th Cedar|Ave|Suite V|Salem|Stephenson County|IL|68048|United States|-6|condo| +8218|AAAAAAAAKBACAAAA|304|Lee Sunset|Circle|Suite 240|Ashland|McDowell County|NC|24244|United States|-5|condo| +8219|AAAAAAAALBACAAAA|470|Main ||||Blaine County||66668|United States||single family| +8220|AAAAAAAAMBACAAAA|354|East Cherry||||Dixon County|NE||||condo| +8221|AAAAAAAANBACAAAA|278|Hill 2nd|Drive|Suite 230|Granite|Geauga County|OH|46284|United States|-5|single family| +8222|AAAAAAAAOBACAAAA|48|First |Ln|Suite E|Sulphur Springs|Wade Hampton Census Area|AK|98354|United States|-9|condo| +8223|AAAAAAAAPBACAAAA|119|Laurel |Street|Suite 250|Caledonia|Parker County|TX|77411|United States|-6|condo| +8224|AAAAAAAAACACAAAA|744|Third 3rd|Circle|Suite L|Richfield|Wabaunsee County|KS|66196|United States|-6|apartment| +8225|AAAAAAAABCACAAAA|31|Eigth Lakeview|Ave|Suite 190|Jamestown|Dakota County|MN|56867|United States|-6|condo| +8226|AAAAAAAACCACAAAA|428|6th Ridge|Cir.|Suite U|Point Pleasant|Roscommon County|MI|44749|United States|-5|single family| +8227|AAAAAAAADCACAAAA|789|Central South|RD|Suite 210|Mount Zion|Sandoval County|NM|88054|United States|-7|single family| +8228|AAAAAAAAECACAAAA|953|8th |Lane|Suite L|Arcola|Seward County|NE|61654|United States|-7|single family| +8229|AAAAAAAAFCACAAAA|225|Oak |Blvd|Suite 240|Springdale|Iosco County|MI|48883|United States|-5|condo| +8230|AAAAAAAAGCACAAAA|933|Lake |ST|Suite L|Springdale|Woodson County|KS|68883|United States|-6|single family| +8231|AAAAAAAAHCACAAAA|839|Central |Circle|Suite 460|Friendship|Edwards County|IL|64536|United States|-6|apartment| +8232|AAAAAAAAICACAAAA|490|Williams Sixth|Road|Suite A|Lakeville|Spencer County|IN|48811|United States|-5|single family| +8233|AAAAAAAAJCACAAAA|75|Woodland Park|Ave|Suite 330|Woodville|New Castle County|DE|14289|United States|-5|condo| +8234|AAAAAAAAKCACAAAA|39|Spring |Circle|Suite K|Crossroads|Lauderdale County|TN|30534|United States|-6|condo| +8235|AAAAAAAALCACAAAA|661|Sunset 7th|Wy|Suite 260|Stringtown|De Soto Parish|LA|70162|United States|-6|condo| +8236|AAAAAAAAMCACAAAA|233|View 2nd|Cir.|Suite 270|Crossroads|Grant County|MN|50534|United States|-6|condo| +8237|AAAAAAAANCACAAAA|105|Ash 14th|Way|Suite 240|Greenfield|Davis County|UT|85038|United States|-7|single family| +8238|AAAAAAAAOCACAAAA|126|College |Parkway|Suite L|Woodlawn|Aransas County|TX|74098|United States|-6|condo| +8239|AAAAAAAAPCACAAAA|862|14th |Avenue|Suite M|Mount Zion|Brunswick County|NC|28054|United States|-5|single family| +8240|AAAAAAAAADACAAAA|88|10th |Road|Suite 140|Greenville|Washington County|CO|81387|United States|-7|apartment| +8241|AAAAAAAABDACAAAA|946|Central 15th|Cir.|Suite Q|Buena Vista|Wayne County|IN|45752|United States|-5|apartment| +8242|AAAAAAAACDACAAAA|122|Dogwood |Street|Suite 60|Jamestown|Presidio County|TX|76867|United States|-6|condo| +8243|AAAAAAAADDACAAAA|482|10th Washington|Pkwy|Suite U|Antioch|Sunflower County|MS|58605|United States|-6|condo| +8244|AAAAAAAAEDACAAAA|79|Highland Walnut|Ct.|Suite O|Clifton|Quay County|NM|88014|United States|-7|condo| +8245|AAAAAAAAFDACAAAA|172|Spruce 2nd|Lane|Suite T|Lebanon|Grady County|GA|32898|United States|-5|apartment| +8246|AAAAAAAAGDACAAAA|240|5th |Ct.|Suite 190|Union Hill|Pierce County|GA|37746|United States|-5|single family| +8247|AAAAAAAAHDACAAAA|447|6th 3rd|Ave|Suite E|Evans|Le Flore County|OK|72284|United States|-6|condo| +8248|AAAAAAAAIDACAAAA|656|Center |Circle|Suite 10|Valley View|Eau Claire County|WI|55124|United States|-6|single family| +8249|AAAAAAAAJDACAAAA|756|Lincoln |Ave|Suite E|Jackson|Clay County|MN|59583|United States|-6|condo| +8250|AAAAAAAAKDACAAAA|752|1st Hickory|Court|Suite 360|Newtown|Litchfield County|CT|02349|United States|-5|condo| +8251|AAAAAAAALDACAAAA|815|Lake Park|Dr.|Suite W|Oak Grove|Brown County|SD|58370|United States|-6|condo| +8252|AAAAAAAAMDACAAAA|143|13th |Court|Suite 110|Shady Grove|Pottawatomie County|OK|72812|United States|-6|apartment| +8253|AAAAAAAANDACAAAA||Pine |Lane|Suite M|||MN|55817|United States|-6|condo| +8254|AAAAAAAAODACAAAA|142|Spring |Lane|Suite 460|Pleasant Hill|Ouachita Parish|LA|73604|United States|-6|single family| +8255|AAAAAAAAPDACAAAA|454|Willow 9th|Ln|Suite O|Riverdale|Butts County|GA|39391|United States|-5|single family| +8256|AAAAAAAAAEACAAAA|116|West |Way|Suite 280|Antioch|Perry County|AR|78605|United States|-6|single family| +8257|AAAAAAAABEACAAAA|897|1st |Circle|Suite 380|Hamilton|Fairbanks North Star Borough|AK|92808|United States|-9|apartment| +8258|AAAAAAAACEACAAAA|338|3rd Green|Ln|Suite R|Pleasant Valley|Manistee County|MI|42477|United States|-5|single family| +8259|AAAAAAAADEACAAAA|875|Highland |Boulevard|Suite 390|Enterprise|Clarendon County|SC|21757|United States|-5|apartment| +8260|AAAAAAAAEEACAAAA|197|Cherry Spring|Parkway|Suite 450|Franklin|Pike County|MS|59101|United States|-6|condo| +8261|AAAAAAAAFEACAAAA|868|2nd |Pkwy|Suite 380|Hopewell|Worth County|IA|50587|United States|-6|single family| +8262|AAAAAAAAGEACAAAA|309|Railroad |Dr.|Suite 70||Brown County|||United States||| +8263|AAAAAAAAHEACAAAA|385|10th |Ct.|Suite H|Sunnyside|Dorchester County|SC|21952|United States|-5|apartment| +8264|AAAAAAAAIEACAAAA|260|Jefferson |Avenue|Suite P|Highland|Beaufort County|NC|29454|United States|-5|apartment| +8265|AAAAAAAAJEACAAAA|152|Spruce |Road|Suite 50|Brownsville|Grays Harbor County|WA|99310|United States|-8|single family| +8266|AAAAAAAAKEACAAAA|471|Meadow |Cir.|Suite K|Ridgeville|Calhoun County|WV|29306|United States|-5|single family| +8267|AAAAAAAALEACAAAA|400|5th Johnson|Avenue|Suite Y|Stratford|Marshall County|TN|36668|United States|-6|condo| +8268|AAAAAAAAMEACAAAA|430|7th |Drive|Suite D|Forest Hills|Carroll County|IA|59237|United States|-6|apartment| +8269|AAAAAAAANEACAAAA|511|Fifth |Lane|Suite 200|Ashland|Baker County|FL|34244|United States|-5|apartment| +8270|AAAAAAAAOEACAAAA|4|Valley |Ln|Suite T|Centerville|Woods County|OK|70059|United States|-6|condo| +8271|AAAAAAAAPEACAAAA|962|Hill Center|Cir.|Suite W|Glenwood|Barton County|MO|63511|United States|-6|apartment| +8272|AAAAAAAAAFACAAAA|366|Lake College|Blvd|Suite U|Kingston|Roanoke County|VA|24975|United States|-5|single family| +8273|AAAAAAAABFACAAAA|368|Walnut Willow|Ln|Suite F|Five Forks|Madison County|ID|82293|United States|-7|single family| +8274|AAAAAAAACFACAAAA|693|Valley |Pkwy|Suite M|Spring Hill|Sitka Borough|AK|96787|United States|-9|condo| +8275|AAAAAAAADFACAAAA|100|4th Adams|Cir.|Suite I|Pleasant Grove|Franklin city|VA|24136|United States|-5|apartment| +8276|AAAAAAAAEFACAAAA|343|Fourth |Way|Suite A|Mount Pleasant|Marion County|IA|51933|United States|-6|single family| +8277|AAAAAAAAFFACAAAA|152|2nd Cherry|Ln|Suite 300|Summit|Warren County|PA|10499|United States|-5|single family| +8278|AAAAAAAAGFACAAAA||Second |Dr.||||MO||United States|-6|single family| +8279|AAAAAAAAHFACAAAA|812|Fourth |Wy|Suite J|Union Hill|Greenwood County|KS|67746|United States|-6|condo| +8280|AAAAAAAAIFACAAAA|709|Washington |Dr.|Suite N|Lebanon|Loup County|NE|62898|United States|-7|condo| +8281|AAAAAAAAJFACAAAA|544|Williams Elm|Wy|Suite K|Morgantown|Lake County|MI|49193|United States|-5|condo| +8282|AAAAAAAAKFACAAAA|33|Locust Sixth|Dr.|Suite 50|Rockwood|Berkeley County|WV|21545|United States|-5|single family| +8283|AAAAAAAALFACAAAA|476|Lincoln |Ct.|Suite V|Forest Hills|Union County|IN|49237|United States|-5|apartment| +8284|AAAAAAAAMFACAAAA|337|Park |Ave|Suite W|Providence|Bristol County|RI|07214|United States|-5|condo| +8285|AAAAAAAANFACAAAA|44|Elm Hickory|Wy|Suite K|Oak Hill|Obion County|TN|37838|United States|-6|apartment| +8286|AAAAAAAAOFACAAAA|581|Central |Boulevard|Suite J|Oak Ridge|Lee County|IL|68371|United States|-6|apartment| +8287|AAAAAAAAPFACAAAA|946|River Main|Ct.|Suite V|Centerville|Muscatine County|IA|50059|United States|-6|single family| +8288|AAAAAAAAAGACAAAA|345|Birch |Street|Suite 200|Concord|Roanoke County|VA|24107|United States|-5|condo| +8289|AAAAAAAABGACAAAA|233|3rd |Cir.|Suite 190|Arlington|Beaver County|UT|86557|United States|-7|condo| +8290|AAAAAAAACGACAAAA|305|Sunset Fifteenth|ST|Suite L|Five Points|Hancock County|ME|06698|United States|-5|single family| +8291|AAAAAAAADGACAAAA|785|Hickory 10th|Cir.|Suite 80|Bunker Hill|Chautauqua County|KS|60150|United States|-6|apartment| +8292|AAAAAAAAEGACAAAA|357|Washington Spruce|Way|Suite W|Spring Valley|Kossuth County|IA|56060|United States|-6|single family| +8293|AAAAAAAAFGACAAAA||7th Lake||Suite 160|Riverdale|Duval County|TX|||-6|apartment| +8294|AAAAAAAAGGACAAAA|20|Smith 5th|Ct.|Suite I|Macon|Pennington County|MN|50369|United States|-6|single family| +8295|AAAAAAAAHGACAAAA|50|Broadway |Ave|Suite P|Georgetown|Crosby County|TX|77057|United States|-6|apartment| +8296|AAAAAAAAIGACAAAA|821|Lee |Dr.|Suite D|Stringtown|Scotts Bluff County|NE|60162|United States|-7|single family| +8297|AAAAAAAAJGACAAAA|||Ln|Suite 130||Passaic County|NJ|08654|||| +8298|AAAAAAAAKGACAAAA|924|5th Davis|Wy|Suite 190|Forest Hills|Zavala County|TX|79237|United States|-6|single family| +8299|AAAAAAAALGACAAAA|665|Woodland |Cir.|Suite 250|Harmony|Kossuth County|IA|55804|United States|-6|single family| +8300|AAAAAAAAMGACAAAA|70|Railroad Smith|RD|Suite 410|Stringtown|Austin County|TX|70162|United States|-6|single family| +8301|AAAAAAAANGACAAAA|520|Locust 3rd|RD|Suite 180|New Hope|New York County|NY|19431|United States|-5|single family| +8302|AAAAAAAAOGACAAAA|767|12th 2nd|Pkwy|Suite D|Unionville|McPherson County|NE|61711|United States|-7|single family| +8303|AAAAAAAAPGACAAAA|425|Ash |Ct.|Suite E|Brownsville|Boundary County|ID|89310|United States|-7|apartment| +8304|AAAAAAAAAHACAAAA|626|Lee |Road|Suite S|Providence|Lowndes County|GA|36614|United States|-5|condo| +8305|AAAAAAAABHACAAAA|894|11th |Wy|Suite 450|Walnut Grove|Windsor County|VT|08352|United States|-5|apartment| +8306|AAAAAAAACHACAAAA|621|Third Park|Avenue|Suite 330|Jamestown|Chambers County|TX|76867|United States|-6|condo| +8307|AAAAAAAADHACAAAA|862|Madison |Boulevard|Suite N||Nicholas County||||-5|| +8308|AAAAAAAAEHACAAAA|497|Main 14th|Blvd|Suite X|Union|Tolland County|CT|09321|United States|-5|condo| +8309|AAAAAAAAFHACAAAA|662|3rd Maple|Drive|Suite Q|Belmont|Otsego County|MI|40191|United States|-5|single family| +8310|AAAAAAAAGHACAAAA|811|Sycamore |Court|Suite H|Jackson|Butler County|OH|49583|United States|-5|condo| +8311|AAAAAAAAHHACAAAA|525|5th Pine|Street|Suite K|Spring Hill|Iberia Parish|LA|76787|United States|-6|single family| +8312|AAAAAAAAIHACAAAA|736|Oak East|Circle|Suite 160|Walnut Grove|Sumner County|KS|67752|United States|-6|apartment| +8313|AAAAAAAAJHACAAAA|993|Broadway |Street|Suite 40|Newtown|Boulder County|CO|81749|United States|-7|apartment| +8314|AAAAAAAAKHACAAAA|537|Ninth |Cir.|Suite 350|Unionville|Bernalillo County|NM|81711|United States|-7|apartment| +8315|AAAAAAAALHACAAAA|577|Adams |Road|Suite S|Waterloo|Mingo County|WV|21675|United States|-5|single family| +8316|AAAAAAAAMHACAAAA|168|Maple |Avenue|Suite T|Pleasant Grove|Monroe County|IN|44136|United States|-5|single family| +8317|AAAAAAAANHACAAAA|51|Hickory |Way|Suite 460|Springfield|Saline County|MO|69303|United States|-6|apartment| +8318|AAAAAAAAOHACAAAA|646|1st |Ave|Suite 90|Providence|Cheatham County|TN|36614|United States|-5|apartment| +8319|AAAAAAAAPHACAAAA|302|Park West|Avenue|Suite 130|Highland Park|Yalobusha County|MS|56534|United States|-6|apartment| +8320|AAAAAAAAAIACAAAA|127|Locust |Ln|Suite 420|Macedonia|Noble County|OK|71087|United States|-6|single family| +8321|AAAAAAAABIACAAAA|898|South |RD|Suite H|Salem|Sweetwater County|WY|88048|United States|-7|condo| +8322|AAAAAAAACIACAAAA|466|Dogwood |Circle|Suite 270|Liberty|Lincoln County|NC|23451|United States|-5|condo| +8323|AAAAAAAADIACAAAA|700|Washington |Ln|Suite 70|Pleasant Grove|Sumter County|FL|34136|United States|-5|apartment| +8324|AAAAAAAAEIACAAAA|138|Ninth Railroad|Ct.|Suite 320|Weldon|Washington County|AR|76277|United States|-6|single family| +8325|AAAAAAAAFIACAAAA|44|Cedar Oak|Road|Suite 80|Belleville|Newport News city|VA|22924|United States|-5|apartment| +8326|AAAAAAAAGIACAAAA|744|Park Sunset|Cir.|Suite 430|Five Points|Santa Cruz County|AZ|86098|United States|-7|apartment| +8327|AAAAAAAAHIACAAAA|256|Park |Blvd|Suite 110|Marion|Jones County|NC|20399|United States|-5|single family| +8328|AAAAAAAAIIACAAAA|711|14th Green|Cir.|Suite 210|Kingston|Medina County|TX|74975|United States|-6|apartment| +8329|AAAAAAAAJIACAAAA|798|Williams |Wy|Suite 340|Montpelier|Issaquena County|MS|58930|United States|-6|apartment| +8330|AAAAAAAAKIACAAAA|254|Pine |Drive|Suite X|Brownsville|Perry County|PA|19310|United States|-5|apartment| +8331|AAAAAAAALIACAAAA|899|Sunset Adams|Drive|Suite 190|Mount Olive|Colfax County|NE|68059|United States|-6|single family| +8332|AAAAAAAAMIACAAAA|260|Willow 6th|Street|Suite D|Greenwood|Thomas County|NE|68828|United States|-7|condo| +8333|AAAAAAAANIACAAAA|911|Park Lake|Ave|Suite 450|Buena Vista|Cleveland County|AR|75752|United States|-6|single family| +8334|AAAAAAAAOIACAAAA|754|Poplar Poplar|Drive|Suite 240|Plainview|Yuba County|CA|93683|United States|-8|apartment| +8335|AAAAAAAAPIACAAAA|252|Elm Meadow|Way|Suite 60|Pleasant Grove|Indian River County|FL|34136|United States|-5|condo| +8336|AAAAAAAAAJACAAAA|706|Johnson |Street|Suite 430|Concord|Contra Costa County|CA|94107|United States|-8|single family| +8337|AAAAAAAABJACAAAA|463|Willow |Circle|Suite L|Mount Vernon|Ouachita Parish|LA|78482|United States|-6|condo| +8338|AAAAAAAACJACAAAA|318|Mill College|Road|Suite M|Pleasant Hill|Gilpin County|CO|83604|United States|-7|apartment| +8339|AAAAAAAADJACAAAA|994|Maple |Parkway|Suite L|Fairview|Franklin County|NY|15709|United States|-5|apartment| +8340|AAAAAAAAEJACAAAA|243|Forest |Pkwy|Suite T|Marion|York County|PA|10399|United States|-5|single family| +8341|AAAAAAAAFJACAAAA|185|Elevnth North|Drive|Suite P|Greenville|Fairfield County|OH|41387|United States|-5|single family| +8342|AAAAAAAAGJACAAAA|389|Madison |Ln|Suite 180|Springdale|Camden County|NC|28883|United States|-5|condo| +8343|AAAAAAAAHJACAAAA|540|Hillcrest Franklin|Street|Suite B|Woodville|Northumberland County|PA|14289|United States|-5|condo| +8344|AAAAAAAAIJACAAAA|657|Second Seventh|Wy|Suite V|Concord|Morgan County|OH|44107|United States|-5|condo| +8345|AAAAAAAAJJACAAAA|684|1st Woodland|RD|Suite 440|Greenwood|Big Horn County|WY|88828|United States|-7|apartment| +8346|AAAAAAAAKJACAAAA|232|First |Lane|Suite I|Edgewood|Atkinson County|GA|30069|United States|-5|apartment| +8347|AAAAAAAALJACAAAA|887|Mill Willow|Road|Suite I|Greenwood|Catron County|NM|88828|United States|-7|apartment| +8348|AAAAAAAAMJACAAAA|576|7th |Avenue|Suite 150|Macedonia|Johnson County|NE|61087|United States|-7|condo| +8349|AAAAAAAANJACAAAA|611|Center |Street|Suite W|Liberty|Madison County|ID|83451|United States|-7|single family| +8350|AAAAAAAAOJACAAAA|908|8th |Ave|Suite O|Enterprise|Bradford County|PA|11757|United States|-5|condo| +8351|AAAAAAAAPJACAAAA|99|Oak |Dr.|Suite K|Westgate|Bladen County|NC|22366|United States|-5|condo| +8352|AAAAAAAAAKACAAAA|518|Johnson |Blvd|Suite W|Centerville|Dooly County|GA|30059|United States|-5|single family| +8353|AAAAAAAABKACAAAA|920|Lake |Ln|Suite O|Providence|Goliad County|TX|76614|United States|-6|condo| +8354|AAAAAAAACKACAAAA|50|14th Smith|Ct.|Suite 50|Wright|Wichita County|TX|72814|United States|-6|condo| +8355|AAAAAAAADKACAAAA|701|Washington |Avenue|Suite 360|Buena Vista|Boise County|ID|85752|United States|-7|apartment| +8356|AAAAAAAAEKACAAAA|920|14th Poplar|Parkway|Suite 120|Brookville|Roger Mills County|OK|73524|United States|-6|apartment| +8357|AAAAAAAAFKACAAAA|75|4th Oak|Parkway|Suite G|Marion|Multnomah County|OR|90399|United States|-8|condo| +8358|AAAAAAAAGKACAAAA|112|Adams Jefferson|Pkwy|Suite 40|Spring Hill|Rockwall County|TX|76787|United States|-6|condo| +8359|AAAAAAAAHKACAAAA|764|Main |RD|Suite 300|Stringtown|Columbia County|WI|50162|United States|-6|condo| +8360|AAAAAAAAIKACAAAA|978|Chestnut 12th|RD|Suite 400|Glendale|Haskell County|KS|63951|United States|-6|apartment| +8361|AAAAAAAAJKACAAAA|317|Twelfth Church|Way|Suite R|Buena Vista|Dallas County|IA|55752|United States|-6|apartment| +8362|AAAAAAAAKKACAAAA|762|9th |Avenue|Suite E|Maple Grove|Claiborne Parish|LA|78252|United States|-6|condo| +8363|AAAAAAAALKACAAAA|713|Highland Green|Cir.|Suite 150|Clifton|Grant County|OK|78014|United States|-6|condo| +8364|AAAAAAAAMKACAAAA|914|Church |Court|Suite F|Union Hill|Iron County|MO|67746|United States|-6|single family| +8365|AAAAAAAANKACAAAA|148|Green |Wy|Suite 250|Franklin|Cass County|MO|69101|United States|-6|apartment| +8366|AAAAAAAAOKACAAAA|57|15th Locust|ST|Suite E|Bridgeport|Strafford County|NH|06417|United States|-5|condo| +8367|AAAAAAAAPKACAAAA|245|Second South|Drive|Suite H|Oak Hill|Liberty County|GA|37838|United States|-5|apartment| +8368|AAAAAAAAALACAAAA|257|Elm Railroad|Blvd|Suite 70|Plainview|Island County|WA|93683|United States|-8|condo| +8369|AAAAAAAABLACAAAA|530|Mill Fourteenth|Avenue|Suite L|Red Oak|DeKalb County|MO|65018|United States|-6|apartment| +8370|AAAAAAAACLACAAAA|398|First |Ln|Suite F|Riverdale|Alfalfa County|OK|79391|United States|-6|condo| +8371|AAAAAAAADLACAAAA|358|West |RD|Suite R|Oakwood|Clay County|MS|50169|United States|-6|condo| +8372|AAAAAAAAELACAAAA|857|Ninth Johnson|Court|Suite K|Pine Grove|Santa Barbara County|CA|94593|United States|-8|condo| +8373|AAAAAAAAFLACAAAA|245|Spring Railroad|Way|Suite S|White Oak|Wyoming County|WV|26668|United States|-5|single family| +8374|AAAAAAAAGLACAAAA|842|South |Parkway|Suite C|Greenwood|Choctaw County|AL|38828|United States|-6|apartment| +8375|AAAAAAAAHLACAAAA|948|12th |Cir.|Suite 470|Green Acres|Grant County|OR|97683|United States|-8|condo| +8376|AAAAAAAAILACAAAA|795|First Davis|Wy|Suite Q|Murray|Ellis County|TX|72150|United States|-6|single family| +8377|AAAAAAAAJLACAAAA|76|Park |Cir.|Suite 310|Crossroads|Vermillion County|IN|40534|United States|-5|apartment| +8378|AAAAAAAAKLACAAAA|507|Smith |Avenue|Suite I|Mount Pleasant|Dixon County|NE|61933|United States|-6|condo| +8379|AAAAAAAALLACAAAA|690|Park |Wy|Suite 310|Woodlawn|Beaufort County|NC|24098|United States|-5|condo| +8380|AAAAAAAAMLACAAAA|965|Fifth Pine|Ln|Suite 100|Greenville|Houston County|AL|31387|United States|-6|condo| +8381|AAAAAAAANLACAAAA|698|5th |ST|Suite 290|Westgate|Lake County|OR|92366|United States|-8|condo| +8382|AAAAAAAAOLACAAAA|890|Wilson |Road|Suite 450|Bunker Hill|Lincoln County|KY|40150|United States|-5|apartment| +8383|AAAAAAAAPLACAAAA|982|Oak Lake|RD|Suite 230|Sulphur Springs|Middlesex County|CT|08954|United States|-5|condo| +8384|AAAAAAAAAMACAAAA|706|Maple Washington|Ave|Suite 330|Greenwood|Greene County|TN|38828|United States|-5|single family| +8385|AAAAAAAABMACAAAA|558|Walnut 14th|Dr.|Suite D|Louisville|Washington County|AR|74464|United States|-6|single family| +8386|AAAAAAAACMACAAAA|941|Adams ||Suite G|Riverside|Montgomery County||||-6|| +8387|AAAAAAAADMACAAAA|752|Main |Boulevard|Suite 140|Clifton|Campbell County|VA|28014|United States|-5|single family| +8388|AAAAAAAAEMACAAAA|155|Smith |Drive|Suite O|Shiloh|Monroe County|PA|19275|United States|-5|condo| +8389|AAAAAAAAFMACAAAA|128|Spruce |Circle|Suite Q|Mount Olive|Raleigh County|WV|28059|United States|-5|condo| +8390|AAAAAAAAGMACAAAA|637|Walnut |Way|Suite N|Clifton|Newton County|GA|38014|United States|-5|condo| +8391|AAAAAAAAHMACAAAA|739|Davis Hill|Boulevard|Suite 450|Bunker Hill|Jackson County|FL|30150|United States|-5|apartment| +8392|AAAAAAAAIMACAAAA|283|Jackson |Boulevard|Suite E|Antioch|Arthur County|NE|68605|United States|-6|apartment| +8393|AAAAAAAAJMACAAAA|716|East ||Suite 120|Concord|Douglas County|||||| +8394|AAAAAAAAKMACAAAA|309|Third |Ln|Suite X|Bunker Hill|Columbia County|WA|90150|United States|-8|condo| +8395|AAAAAAAALMACAAAA|61|Ash |Ln|Suite R|Hopewell|Nevada County|CA|90587|United States|-8|apartment| +8396|AAAAAAAAMMACAAAA|842|13th First|Avenue|Suite 210|Green Acres|Terrell County|GA|37683|United States|-5|apartment| +8397|AAAAAAAANMACAAAA|322|Spring East|Blvd|Suite W|Oak Grove|Eureka County|NV|88370|United States|-8|single family| +8398|AAAAAAAAOMACAAAA|832|Ridge |Street|Suite 340|Wildwood|Oldham County|TX|76871|United States|-6|apartment| +8399|AAAAAAAAPMACAAAA|188|Hickory Oak|Ave|Suite 350|Oak Ridge|Butte County|ID|88371|United States|-7|single family| +8400|AAAAAAAAANACAAAA||2nd Central|ST||||VA||United States||| +8401|AAAAAAAABNACAAAA|456|Willow Sunset|Blvd|Suite 320|Deerfield|Cortland County|NY|19840|United States|-5|apartment| +8402|AAAAAAAACNACAAAA|390|Ash |Ln|Suite 90|Pine Ridge|Flagler County|FL|35319|United States|-5|apartment| +8403|AAAAAAAADNACAAAA|482|Lee |RD|Suite U|Colfax|Payette County|ID|82565|United States|-7|single family| +8404|AAAAAAAAENACAAAA|590|Chestnut Park|Wy|Suite 400|Tyrone|Jasper County|MS|51201|United States|-6|apartment| +8405|AAAAAAAAFNACAAAA|292|Lake Adams|Parkway|Suite X|Woodlawn|Madison County|VA|24098|United States|-5|single family| +8406|AAAAAAAAGNACAAAA|799|Davis Cedar|Parkway|Suite M|Farmington|Missoula County|MT|69145|United States|-7|single family| +8407|AAAAAAAAHNACAAAA|318|Woodland Highland|Ct.|Suite 280|Glenwood|Hall County|TX|73511|United States|-6|single family| +8408|AAAAAAAAINACAAAA|742|Center 15th|Blvd|Suite 50|Sunnyside|Morrison County|MN|51952|United States|-6|apartment| +8409|AAAAAAAAJNACAAAA|431|Franklin ||Suite M|Oakdale||OH|||-5|| +8410|AAAAAAAAKNACAAAA|673|Fourteenth 2nd|Ct.||Greenfield|Isanti County||55038||-6|| +8411|AAAAAAAALNACAAAA|735|View Hickory|Pkwy|Suite 430|Pleasant Valley|Aransas County|TX|72477|United States|-6|condo| +8412|AAAAAAAAMNACAAAA|223|Church |Ln|Suite 110|Star|Sauk County|WI|50725|United States|-6|single family| +8413|AAAAAAAANNACAAAA|||Wy||Fairview|||||-8|| +8414|AAAAAAAAONACAAAA|||Way|||Cumberland County|NC|27746|United States||| +8415|AAAAAAAAPNACAAAA|641|Jackson |Court|Suite U|Riley|Garfield County|UT|81692|United States|-7|single family| +8416|AAAAAAAAAOACAAAA|912|College Jackson|Drive|Suite P|Woodland|Washington County|AL|34854|United States|-6|condo| +8417|AAAAAAAABOACAAAA|72|Second Eigth|Cir.|Suite 330|Woodville|Seward County|NE|64289|United States|-7|single family| +8418|AAAAAAAACOACAAAA|777|||Suite W|Greenville|Saunders County|||||| +8419|AAAAAAAADOACAAAA|364|Forest Pine|Lane|Suite A|Edgewood|Montgomery County|IN|40069|United States|-5|single family| +8420|AAAAAAAAEOACAAAA|917|Jefferson |Drive|Suite Y|Greenfield|Crawford County|MO|65038|United States|-6|condo| +8421|AAAAAAAAFOACAAAA|291|South Willow|Lane|Suite B|Ashland|Hardin County|OH|44244|United States|-5|single family| +8422|AAAAAAAAGOACAAAA|432|Railroad |Boulevard|Suite 120|Oak Grove|DeSoto County|FL|38370|United States|-5|condo| +8423|AAAAAAAAHOACAAAA|140|Lincoln |Avenue|Suite M|Sunnyside|Henry County|TN|31952|United States|-5|apartment| +8424|AAAAAAAAIOACAAAA|48|Center |Lane|Suite 480|Franklin|Warren County|MS|59101|United States|-6|single family| +8425|AAAAAAAAJOACAAAA|899|Ridge 4th|Circle|Suite 230|Stringtown|Forsyth County|NC|20162|United States|-5|single family| +8426|AAAAAAAAKOACAAAA|912|Main |Ln|Suite 390|Antioch|Atoka County|OK|78605|United States|-6|single family| +8427|AAAAAAAALOACAAAA|487|Third |Cir.|Suite D|White Oak|Caroline County|VA|26668|United States|-5|apartment| +8428|AAAAAAAAMOACAAAA|495|Sycamore |Parkway|Suite D|Mount Vernon|Kendall County|TX|78482|United States|-6|apartment| +8429|AAAAAAAANOACAAAA|788|Hillcrest |Wy|Suite C|Rutland|Garvin County|OK|78375|United States|-6|condo| +8430|AAAAAAAAOOACAAAA|52|5th 10th|Dr.|Suite Q|Jamestown|Modoc County|CA|96867|United States|-8|apartment| +8431|AAAAAAAAPOACAAAA|701|Chestnut Fourteenth|Pkwy|Suite 260|Riverdale|Colbert County|AL|39391|United States|-6|condo| +8432|AAAAAAAAAPACAAAA|39|3rd Poplar|Road|Suite W|Freeman|Gentry County|MO|62297|United States|-6|condo| +8433|AAAAAAAABPACAAAA|302|3rd Washington|Circle|Suite 130|Sharon|Escambia County|FL|34029|United States|-5|condo| +8434|AAAAAAAACPACAAAA|649|Jackson |Lane|Suite J|Belmont|Ketchikan Gateway Borough|AK|90191|United States|-9|apartment| +8435|AAAAAAAADPACAAAA|390|Park |Court|Suite 300|Jamestown|Fulton County|KY|46867|United States|-6|condo| +8436|AAAAAAAAEPACAAAA|250|Lincoln Highland|Pkwy|Suite 350|Hamilton|Monroe County|MS|52808|United States|-6|condo| +8437|AAAAAAAAFPACAAAA|677|Jackson |Lane|Suite Y|Hidden Valley|De Kalb County|IN|45521|United States|-5|single family| +8438|AAAAAAAAGPACAAAA|806|Spring |Cir.|Suite 150|Riverdale|Chouteau County|MT|69391|United States|-7|single family| +8439|AAAAAAAAHPACAAAA|763|Oak College|Dr.|Suite 210|River Oaks|Clinch County|GA|38075|United States|-5|apartment| +8440|AAAAAAAAIPACAAAA|98|4th |Circle|Suite 320|Union Hill|Brown County|OH|47746|United States|-5|apartment| +8441|AAAAAAAAJPACAAAA|766|4th Oak|Blvd|Suite 240|Clifton|Lawrence County|MS|58014|United States|-6|single family| +8442|AAAAAAAAKPACAAAA|199|3rd |Ct.|Suite K|Riceville|Ouray County|CO|85867|United States|-7|single family| +8443|AAAAAAAALPACAAAA|49|Third Chestnut|Court|Suite J|Lincoln|Henderson County|TN|31289|United States|-5|apartment| +8444|AAAAAAAAMPACAAAA|520|14th Maple|Drive|Suite S|Fairfield|Montgomery County|IA|56192|United States|-6|single family| +8445|AAAAAAAANPACAAAA|442|Sycamore |Drive|Suite 370|Greenwood|Jefferson County|FL|38828|United States|-5|condo| +8446|AAAAAAAAOPACAAAA|142|1st Poplar|Parkway|Suite X|Liberty|Falls Church city|VA|23451|United States|-5|single family| +8447|AAAAAAAAPPACAAAA|716|Cherry Franklin|Ct.|Suite 100|Gum Springs|Clark County|AR|72106|United States|-6|condo| +8448|AAAAAAAAAABCAAAA|856|Lake 2nd|Avenue|Suite D|Mount Zion|Paulding County|GA|38054|United States|-5|single family| +8449|AAAAAAAABABCAAAA|333|Central |Dr.|Suite N|Concord|Polk County|MN|54107|United States|-6|condo| +8450|AAAAAAAACABCAAAA|326|South |Ave|Suite 20|Woodland|Eddy County|ND|54854|United States|-6|apartment| +8451|AAAAAAAADABCAAAA|420|Highland |Lane|Suite 460|Five Points|Cheyenne County|KS|66098|United States|-6|apartment| +8452|AAAAAAAAEABCAAAA|846|Lake |Boulevard|Suite 0|Woodbine|Union County|MS|54253|United States|-6|apartment| +8453|AAAAAAAAFABCAAAA|113|Green |Lane|Suite V|Fairview|Columbia County|NY|15709|United States|-5|condo| +8454|AAAAAAAAGABCAAAA|488|Park |ST|Suite 40|Crossroads|Hamlin County|SD|50534|United States|-7|single family| +8455|AAAAAAAAHABCAAAA|762|Cedar 1st|Blvd|Suite S|Florence|Suffolk County|MA|03994|United States|-5|condo| +8456|AAAAAAAAIABCAAAA|808|Lake South|Pkwy|Suite E|Galena|Madison County|MO|64369|United States|-6|apartment| +8457|AAAAAAAAJABCAAAA||Hill Park||Suite F|Kingston|Tishomingo County||54975||-6|| +8458|AAAAAAAAKABCAAAA|289|Laurel Sunset|Wy|Suite 200|Ashland|Gilmer County|GA|34244|United States|-5|condo| +8459|AAAAAAAALABCAAAA|34|Smith Johnson|Road|Suite C|Forest Hills|Putnam County|IL|69237|United States|-6|single family| +8460|AAAAAAAAMABCAAAA|607|Cedar Walnut|Parkway|Suite K|Five Forks|Orange County|CA|92293|United States|-8|condo| +8461|AAAAAAAANABCAAAA|834|7th Dogwood|Circle|Suite B|Pleasant Valley|Putnam County|MO|62477|United States|-6|apartment| +8462|AAAAAAAAOABCAAAA|836|South |Lane|Suite 440|Woodville|Campbell County|WY|84289|United States|-7|single family| +8463|AAAAAAAAPABCAAAA|454|Franklin |Ave|Suite W|Red Hill|Franklin County|MA|04938|United States|-5|condo| +8464|AAAAAAAAABBCAAAA|422|Jackson |Way|Suite 90|Shady Grove|Belknap County|NH|03412|United States|-5|condo| +8465|AAAAAAAABBBCAAAA|965|Johnson Jackson|Road|Suite G|Sanford|Polk County|NC|29737|United States|-5|apartment| +8466|AAAAAAAACBBCAAAA||Valley Broadway|RD||Riverdale|Butler County|||United States||| +8467|AAAAAAAADBBCAAAA|341|Park |RD|Suite X|Lebanon|Bayfield County|WI|52898|United States|-6|apartment| +8468|AAAAAAAAEBBCAAAA|540|Locust 9th|Drive|Suite I|Glendale|Dawson County|TX|73951|United States|-6|condo| +8469|AAAAAAAAFBBCAAAA|587|Sixth First|Ct.|Suite 400|Lebanon|Baxter County|AR|72898|United States|-6|single family| +8470|AAAAAAAAGBBCAAAA|992|4th Spruce|Wy|Suite B|Pleasant Hill|Boone County|IA|53604|United States|-6|condo| +8471|AAAAAAAAHBBCAAAA|698|Birch |Circle|Suite G|Maple Grove|Avery County|NC|28252|United States|-5|single family| +8472|AAAAAAAAIBBCAAAA|938|Madison |Boulevard|Suite Y|Walnut Grove|Johnson County|KS|67752|United States|-6|single family| +8473|AAAAAAAAJBBCAAAA|732|Hickory Pine|ST|Suite 230|Newport|Imperial County|CA|91521|United States|-8|apartment| +8474|AAAAAAAAKBBCAAAA|192|Wilson |Road|Suite U|Oakwood|Columbia County|AR|70169|United States|-6|apartment| +8475|AAAAAAAALBBCAAAA|581|Laurel |Lane|Suite D|Glenwood|Charlton County|GA|33511|United States|-5|condo| +8476|AAAAAAAAMBBCAAAA|4|Meadow |Circle|Suite 100|Newtown|Marlboro County|SC|21749|United States|-5|condo| +8477|AAAAAAAANBBCAAAA|653|Smith 5th|Wy|Suite 30|Enterprise|Buchanan County|VA|21757|United States|-5|apartment| +8478|AAAAAAAAOBBCAAAA|970|Woodland |Boulevard|Suite B|Buena Vista|Crawford County|IN|45752|United States|-5|condo| +8479|AAAAAAAAPBBCAAAA|755|Walnut |Boulevard|Suite 0|Woodlawn|Lawrence County|SD|54098|United States|-7|condo| +8480|AAAAAAAAACBCAAAA|984|Willow |Dr.|Suite 410||Benewah County||80411|United States|-7|apartment| +8481|AAAAAAAABCBCAAAA|222|Davis |Pkwy|Suite B|Allentown|Bleckley County|GA|31838|United States|-5|condo| +8482|AAAAAAAACCBCAAAA|305|Jackson |Ave|Suite D|Oakley|York County|PA|19698|United States|-5|single family| +8483|AAAAAAAADCBCAAAA|579|Chestnut |ST|Suite S|Riverside|Wood County|OH|49231|United States|-5|single family| +8484|AAAAAAAAECBCAAAA|132|9th Jackson|Parkway|Suite A|Clinton|West Carroll Parish|LA|78222|United States|-6|apartment| +8485|AAAAAAAAFCBCAAAA|946||Dr.|Suite E||Torrance County|NM|||-7|| +8486|AAAAAAAAGCBCAAAA|365|5th |Pkwy|Suite 340|Red Hill|Niagara County|NY|14338|United States|-5|apartment| +8487|AAAAAAAAHCBCAAAA|70|Oak |Wy|Suite E|Glendale|Polk County|AR|73951|United States|-6|apartment| +8488|AAAAAAAAICBCAAAA|862|Pine |Drive|Suite 490|Mount Olive|McKinley County|NM|88059|United States|-7|apartment| +8489|AAAAAAAAJCBCAAAA|861|Walnut 13th|Ln|Suite Y|Macedonia|Noxubee County|MS|51087|United States|-6|apartment| +8490|AAAAAAAAKCBCAAAA|547|Seventh 10th||Suite 90|Mountain View|||||-6|condo| +8491|AAAAAAAALCBCAAAA|317|2nd |RD|Suite 110|Riverside|Goodhue County|MN|59231|United States|-6|apartment| +8492|AAAAAAAAMCBCAAAA|351|15th 1st|Circle|Suite 280|Montague|Blaine County|ID|84062|United States|-7|condo| +8493|AAAAAAAANCBCAAAA|710|East Wilson|Way|Suite V|Fairview|Vernon Parish|LA|75709|United States|-6|single family| +8494|AAAAAAAAOCBCAAAA|353|Pine Jefferson|Blvd|Suite 200|Shiloh|Skamania County|WA|99275|United States|-8|single family| +8495|AAAAAAAAPCBCAAAA|690|Ridge |Court|Suite 120|Shiloh|Brown County|TX|79275|United States|-6|condo| +8496|AAAAAAAAADBCAAAA|223|11th |Blvd|||DeWitt County|TX|74289|United States||| +8497|AAAAAAAABDBCAAAA|676|North |Ct.|Suite J|Spring Hill|Boone County|IL|66787|United States|-6|apartment| +8498|AAAAAAAACDBCAAAA|780|Sunset |Boulevard|Suite R|Stringtown|Talladega County|AL|30162|United States|-6|condo| +8499|AAAAAAAADDBCAAAA|389|Third Laurel|Dr.|Suite 270|Buena Vista|Walker County|TX|75752|United States|-6|condo| +8500|AAAAAAAAEDBCAAAA|83|West |Street|Suite 60|Oak Ridge|Hughes County|SD|58371|United States|-7|condo| +8501|AAAAAAAAFDBCAAAA|||Circle|||||56534|||single family| +8502|AAAAAAAAGDBCAAAA|523|Pine |Ln|Suite F|Oakdale|Jackson County|WV|29584|United States|-5|apartment| +8503|AAAAAAAAHDBCAAAA|593|Pine |Parkway|Suite V|Summit|Fayette County|TN|30499|United States|-5|single family| +8504|AAAAAAAAIDBCAAAA|806|View |RD|Suite X|Clinton|Cherokee County|OK|78222|United States|-6|condo| +8505|AAAAAAAAJDBCAAAA|648|Mill Sunset|Avenue|Suite S|Mountain View|Covington County|MS|54466|United States|-6|condo| +8506|AAAAAAAAKDBCAAAA|650|River Cedar|Dr.|Suite 420|Red Hill|Worth County|GA|34338|United States|-5|apartment| +8507|AAAAAAAALDBCAAAA|200|6th |Wy|Suite 450|Union Hill|Pierce County|NE|67746|United States|-7|apartment| +8508|AAAAAAAAMDBCAAAA|62|River 1st|Cir.|Suite 150|Springtown|Shelby County|TX|79858|United States|-6|single family| +8509|AAAAAAAANDBCAAAA|296|Main Washington|Ave|Suite 40|Georgetown|Montgomery County|GA|37057|United States|-5|single family| +8510|AAAAAAAAODBCAAAA|32|River |Road|Suite O|Deerfield|Fulton County|PA|19840|United States|-5|apartment| +8511|AAAAAAAAPDBCAAAA|918|Meadow Sycamore|Pkwy|Suite U|Macedonia|Duplin County|NC|21087|United States|-5|single family| +8512|AAAAAAAAAEBCAAAA|386||||||KS|68059||-6|| +8513|AAAAAAAABEBCAAAA|258|Smith 11th|Ln|Suite D|Pine Grove|Blount County|AL|34593|United States|-6|single family| +8514|AAAAAAAACEBCAAAA|147|Lakeview Park|Blvd|Suite K|Bethel|DeWitt County|TX|75281|United States|-6|apartment| +8515|AAAAAAAADEBCAAAA|612|6th |Lane|Suite D|Newtown|Obion County|TN|31749|United States|-6|single family| +8516|AAAAAAAAEEBCAAAA|332|Broadway |Court|Suite 380|Woodland|Giles County|VA|24854|United States|-5|condo| +8517|AAAAAAAAFEBCAAAA|487|Hill Ridge|Avenue|Suite 40|Greenfield|Bourbon County|KY|45038|United States|-6|single family| +8518|AAAAAAAAGEBCAAAA|870|Park 3rd|Ln|Suite 10|Pleasant Grove|Lewis County|TN|34136|United States|-6|apartment| +8519|AAAAAAAAHEBCAAAA|525|Walnut South|Ln|Suite Q|Midway|Otsego County|MI|41904|United States|-5|condo| +8520|AAAAAAAAIEBCAAAA|296|Spring Miller|Road|Suite 340|Pleasant Hill|Maury County|TN|33604|United States|-6|apartment| +8521|AAAAAAAAJEBCAAAA|687|East |Dr.|Suite K|Lakewood|Broadwater County|MT|68877|United States|-7|single family| +8522|AAAAAAAAKEBCAAAA|751|3rd |Ave|Suite L|Jamestown|Jennings County|IN|46867|United States|-5|single family| +8523|AAAAAAAALEBCAAAA|183|Pine 1st|Street|Suite 200|Shady Grove|Washington County|WI|52812|United States|-6|apartment| +8524|AAAAAAAAMEBCAAAA|334|Oak |Blvd|Suite X|Providence|Johnson County|TN|36614|United States|-6|apartment| +8525|AAAAAAAANEBCAAAA|484|Maple Eigth|Wy|Suite 300|Mount Olive|Victoria County|TX|78059|United States|-6|apartment| +8526|AAAAAAAAOEBCAAAA|684|Spring |Dr.|Suite J|Mountain View|Honolulu County|HI|94466|United States|-10|apartment| +8527|AAAAAAAAPEBCAAAA|979|Adams Oak|Ln|Suite 60|Jamestown|Mille Lacs County|MN|56867|United States|-6|condo| +8528|AAAAAAAAAFBCAAAA|524|Cherry Lincoln|RD|Suite 160|Lebanon|Chariton County|MO|62898|United States|-6|condo| +8529|AAAAAAAABFBCAAAA|637|Washington Walnut|Ln|Suite O|Hamilton|Wayne County|IN|42808|United States|-5|condo| +8530|AAAAAAAACFBCAAAA|177|3rd |Parkway|Suite 440|Carpenter|Clark County|MO|61147|United States|-6|single family| +8531|AAAAAAAADFBCAAAA|850|Cherry |Blvd|Suite K|Plainview|Butler County|KS|63683|United States|-6|single family| +8532|AAAAAAAAEFBCAAAA|569|Willow Ridge|Lane|Suite D|Macedonia|Okanogan County|WA|91087|United States|-8|condo| +8533|AAAAAAAAFFBCAAAA|994|11th North||||||||-6|condo| +8534|AAAAAAAAGFBCAAAA|12|Sixth |ST|Suite 390|Salem|Audubon County|IA|58048|United States|-6|apartment| +8535|AAAAAAAAHFBCAAAA|466|Ridge |Ln|Suite B|Bethel|Cotton County|OK|75281|United States|-6|condo| +8536|AAAAAAAAIFBCAAAA|885|4th |Drive|Suite 230|Morgantown|Iron County|UT|89193|United States|-7|single family| +8537|AAAAAAAAJFBCAAAA|719|River 9th|Pkwy|Suite O|Clinton|Butler County|KS|68222|United States|-6|single family| +8538|AAAAAAAAKFBCAAAA|704|Washington |Parkway|Suite 240|Stafford|Johnson County|GA|34980|United States|-5|single family| +8539|AAAAAAAALFBCAAAA|637|Maple 11th|Road|Suite 210|Liberty|Irion County|TX|73451|United States|-6|single family| +8540|AAAAAAAAMFBCAAAA|591|Poplar |Way|Suite 20|Mountain View|Renville County|MN|54466|United States|-6|apartment| +8541|AAAAAAAANFBCAAAA||||||Martin County|FL|36098||-5|| +8542|AAAAAAAAOFBCAAAA|979|Locust Ninth|Avenue|Suite I|Union|Yancey County|NC|28721|United States|-5|single family| +8543|AAAAAAAAPFBCAAAA|699|14th Wilson|Cir.|Suite 90|Springdale|Hamilton County|NY|18883|United States|-5|single family| +8544|AAAAAAAAAGBCAAAA|298|Elm |Pkwy|Suite 250|Union|Ogemaw County|MI|48721|United States|-5|single family| +8545|AAAAAAAABGBCAAAA|843|Main Ash|Court|Suite S|Riverview|Cooper County|MO|69003|United States|-6|apartment| +8546|AAAAAAAACGBCAAAA|757|Park Oak|RD|Suite 210|Crystal|Charleston County|SC|25258|United States|-5|single family| +8547|AAAAAAAADGBCAAAA|73|Locust |RD|Suite Q|Riverside|Union County|IL|69231|United States|-6|condo| +8548|AAAAAAAAEGBCAAAA|206|Ridge Main|Cir.|Suite 150|Oak Grove|Alpena County|MI|48370|United States|-5|condo| +8549|AAAAAAAAFGBCAAAA|275|Fourth Seventh|Street|Suite 260|Tracy|Covington city|VA|26340|United States|-5|single family| +8550|AAAAAAAAGGBCAAAA||Park 1st|||Macedonia|||51087|||single family| +8551|AAAAAAAAHGBCAAAA|927|Valley |RD|Suite P|Walnut Grove|Columbia County|FL|37752|United States|-5|apartment| +8552|AAAAAAAAIGBCAAAA|53|Cedar Hill|Road|Suite 140|Union Hill|Arthur County|NE|67746|United States|-6|single family| +8553|AAAAAAAAJGBCAAAA|777|Miller 14th|Road|Suite 30|Brownsville|Shelby County|KY|49310|United States|-5|condo| +8554|AAAAAAAAKGBCAAAA|810|South |Court|Suite J|Forest Hills|Berks County|PA|19237|United States|-5|single family| +8555|AAAAAAAALGBCAAAA||Cedar |Ln||Spring Valley||||||| +8556|AAAAAAAAMGBCAAAA|480|Washington |ST|Suite I|Westgate|Pottawatomie County|OK|72366|United States|-6|single family| +8557|AAAAAAAANGBCAAAA|34|Elm Adams|Way|Suite 160|Glendale|Pottawatomie County|OK|73951|United States|-6|single family| +8558|AAAAAAAAOGBCAAAA|938|Walnut |Lane|Suite X|Arlington|Newton County|MS|56557|United States|-6|condo| +8559|AAAAAAAAPGBCAAAA|512|Railroad |Ave|Suite R|Newport|Pike County|GA|31521|United States|-5|condo| +8560|AAAAAAAAAHBCAAAA|297|Railroad Oak|Blvd|Suite J|Oak Ridge|Bourbon County|KY|48371|United States|-6|single family| +8561|AAAAAAAABHBCAAAA||Elevnth 9th||Suite T|Langdon||LA|||-6|single family| +8562|AAAAAAAACHBCAAAA|306|Chestnut Mill|Avenue|Suite X|Mount Zion|Augusta County|VA|28054|United States|-5|single family| +8563|AAAAAAAADHBCAAAA|127|9th |Road|Suite P|Oak Grove|Rockcastle County|KY|48370|United States|-5|single family| +8564|AAAAAAAAEHBCAAAA|142|First 1st|Circle|Suite 70|Richfield|Okeechobee County|FL|36196|United States|-5|condo| +8565|AAAAAAAAFHBCAAAA|143|Forest Hillcrest|Drive|Suite F|Five Points|Washington County|FL|36098|United States|-5|apartment| +8566|AAAAAAAAGHBCAAAA|791|2nd Dogwood|Wy|Suite W|Jackson|Hardeman County|TX|79583|United States|-6|single family| +8567|AAAAAAAAHHBCAAAA|871|Johnson Lincoln|ST|Suite 280|Mount Olive|Washington County|GA|38059|United States|-5|condo| +8568|AAAAAAAAIHBCAAAA|448|Cedar |Ave|Suite H|Enterprise|Cumberland County|NJ|02357|United States|-5|single family| +8569|AAAAAAAAJHBCAAAA|2|2nd |Street|Suite K|Oakwood|Hardin County|KY|40169|United States|-6|single family| +8570|AAAAAAAAKHBCAAAA|167|15th |Street|Suite C|Union Hill|Tuscola County|MI|47746|United States|-5|apartment| +8571|AAAAAAAALHBCAAAA|222|Pine Railroad|Cir.|Suite W|Glenwood|Meade County|SD|53511|United States|-7|condo| +8572|AAAAAAAAMHBCAAAA|510|River West|Drive|Suite 230|Lebanon|Franklin County|TN|32898|United States|-5|condo| +8573|AAAAAAAANHBCAAAA|810|Sunset Woodland|RD|Suite 490|Bridgeport|Harper County|KS|65817|United States|-6|single family| +8574|AAAAAAAAOHBCAAAA||11th Franklin|||Riverside|||49231|||| +8575|AAAAAAAAPHBCAAAA|801|West 2nd|Court|Suite 440|Ashland|San Benito County|CA|94244|United States|-8|single family| +8576|AAAAAAAAAIBCAAAA|515|11th View|Street|Suite 310|Harmon|Washington County|WI|55623|United States|-6|condo| +8577|AAAAAAAABIBCAAAA|986|Hill |Parkway|Suite 230|New Hope|Edmunds County|SD|59431|United States|-6|apartment| +8578|AAAAAAAACIBCAAAA|927|7th First|Cir.|Suite N|Franklin|East Feliciana Parish|LA|79101|United States|-6|single family| +8579|AAAAAAAADIBCAAAA|316|1st |ST|Suite 320|Riceville|Union County|SC|25867|United States|-5|apartment| +8580|AAAAAAAAEIBCAAAA|844|Lincoln |Blvd|Suite 400|Royal|Rush County|IN|45819|United States|-5|condo| +8581|AAAAAAAAFIBCAAAA|205|Valley 11th|Dr.|Suite X|Lone Pine|Creek County|OK|77441|United States|-6|single family| +8582|AAAAAAAAGIBCAAAA|142|Sycamore |Court|Suite 230|Lincoln|Alexander County|NC|21289|United States|-5|condo| +8583|AAAAAAAAHIBCAAAA|109|Pine |Court|Suite B|Union|Grimes County|TX|78721|United States|-6|condo| +8584|AAAAAAAAIIBCAAAA|666|Franklin |Street|Suite 50|Mount Zion|Scotland County|NC|28054|United States|-5|single family| +8585|AAAAAAAAJIBCAAAA|277|Ridge |Wy|Suite U|Concord|Dallas County|AR|74107|United States|-6|apartment| +8586|AAAAAAAAKIBCAAAA|392|Hillcrest Adams|Ln|Suite V|Waterloo|Shiawassee County|MI|41675|United States|-5|condo| +8587|AAAAAAAALIBCAAAA|740|Chestnut Locust|ST|Suite 220|Midway|Dougherty County|GA|31904|United States|-5|apartment| +8588|AAAAAAAAMIBCAAAA|76|Fifth Chestnut|Parkway|Suite 320|Valley View|Lincoln County|GA|35124|United States|-5|condo| +8589|AAAAAAAANIBCAAAA|157|4th |Way|Suite 80|Belleview|Hickman County|KY|40492|United States|-6|apartment| +8590|AAAAAAAAOIBCAAAA|186|Maple Sycamore|Way|Suite 230|Riverside|Audrain County|MO|69231|United States|-6|condo| +8591|AAAAAAAAPIBCAAAA|959|2nd |Court|Suite V|Greenwood|Butte County|CA|98828|United States|-8|single family| +8592|AAAAAAAAAJBCAAAA|355|Fourth |Parkway|Suite 90|Greenwood|Allendale County|SC|28828|United States|-5|single family| +8593|AAAAAAAABJBCAAAA||Spring |Avenue|Suite 330|||SD|56997|United States|-7|condo| +8594|AAAAAAAACJBCAAAA|867|2nd |Parkway|Suite 10|Guilford|Swain County|NC|24408|United States|-5|condo| +8595|AAAAAAAADJBCAAAA|752|Spruce 2nd|Wy|Suite 40|Spring Hill|Lauderdale County|MS|56787|United States|-6|apartment| +8596|AAAAAAAAEJBCAAAA|841|Lakeview |Wy|Suite Y|Buena Vista|Teton County|WY|85752|United States|-7|single family| +8597|AAAAAAAAFJBCAAAA|290|Church |Dr.|Suite 310|Clifton|Allen County|KY|48014|United States|-6|apartment| +8598|AAAAAAAAGJBCAAAA|977|Walnut |Street|Suite R|Gary|Howell County|MO|60418|United States|-6|condo| +8599|AAAAAAAAHJBCAAAA|642|Tenth 7th|Blvd|Suite S|Crossroads|Sumter County|FL|30534|United States|-5|apartment| +8600|AAAAAAAAIJBCAAAA|702|Locust |Street|Suite N|Valley View|Windham County|CT|05724|United States|-5|condo| +8601|AAAAAAAAJJBCAAAA|991|Railroad Jackson|Parkway|Suite E|Oakdale|Ocean County|NJ|09584|United States|-5|single family| +8602|AAAAAAAAKJBCAAAA|462|Lee West|Ln|Suite 170|Brownsville|Crane County|TX|79310|United States|-6|apartment| +8603|AAAAAAAALJBCAAAA|101|Wilson |Blvd|Suite S|Waterloo|Wilbarger County|TX|71675|United States|-6|condo| +8604|AAAAAAAAMJBCAAAA|531|Oak 7th|Cir.|Suite L|New Hope|Matagorda County|TX|79431|United States|-6|apartment| +8605|AAAAAAAANJBCAAAA|169|5th Jefferson|Street|Suite I|Deerfield|Suffolk city|VA|29840|United States|-5|apartment| +8606|AAAAAAAAOJBCAAAA|950|Forest |Ave|Suite 250|Shannon|Spalding County|GA|34120|United States|-5|single family| +8607|AAAAAAAAPJBCAAAA|632|6th |ST|Suite P|Green Acres|Colfax County|NM|87683|United States|-7|condo| +8608|AAAAAAAAAKBCAAAA|828|Hillcrest 13th|Drive|Suite J|Marion|Giles County|VA|20399|United States|-5|condo| +8609|AAAAAAAABKBCAAAA|873|Park |Pkwy|Suite R|Glendale|Sitka Borough|AK|93951|United States|-9|condo| +8610|AAAAAAAACKBCAAAA|699|Franklin |Street|Suite 0|White Oak|Wythe County|VA|26668|United States|-5|condo| +8611|AAAAAAAADKBCAAAA||||Suite M|Springfield|||||-5|condo| +8612|AAAAAAAAEKBCAAAA|53|Elm Lake|Blvd|Suite 220|Salem|Seminole County|GA|38048|United States|-5|single family| +8613|AAAAAAAAFKBCAAAA|550|Mill |Ln|Suite C|Maple Grove|Warren County|NY|18252|United States|-5|single family| +8614|AAAAAAAAGKBCAAAA|7|Washington Hill|Drive|Suite K|Lakeside|Griggs County|ND|59532|United States|-6|condo| +8615|AAAAAAAAHKBCAAAA|218|10th |ST|Suite 470|Summit|Huron County|MI|40499|United States|-5|condo| +8616|AAAAAAAAIKBCAAAA|699|||Suite 370|||MO|60399|||single family| +8617|AAAAAAAAJKBCAAAA|285|Park |Street|Suite W|Farmington|Williamsburg city|VA|29145|United States|-5|single family| +8618|AAAAAAAAKKBCAAAA|485|Chestnut Washington|Road|Suite 370|Five Points|Warren County|IA|56098|United States|-6|apartment| +8619|AAAAAAAALKBCAAAA|817|Second |Ave|Suite 10|Providence|Harney County|OR|96614|United States|-8|single family| +8620|AAAAAAAAMKBCAAAA|350|Cedar Laurel|Circle|Suite 150|Fowler|Jasper County|IL|61083|United States|-6|single family| +8621|AAAAAAAANKBCAAAA|959|Dogwood |Ave|Suite 280|Scottsville|Nez Perce County|ID|84190|United States|-7|apartment| +8622|AAAAAAAAOKBCAAAA|652|South |Dr.|Suite G|Lincoln|Perry County|TN|31289|United States|-6|single family| +8623|AAAAAAAAPKBCAAAA|288|Hill |Ct.|Suite C|Jamestown|Wirt County|WV|26867|United States|-5|apartment| +8624|AAAAAAAAALBCAAAA|828|Wilson |Boulevard|Suite 90|Mount Vernon|Murray County|GA|38482|United States|-5|apartment| +8625|AAAAAAAABLBCAAAA|611|Ash Cherry|Lane|Suite H|Mountain View|Cross County|AR|74466|United States|-6|apartment| +8626|AAAAAAAACLBCAAAA|902|Washington Park|Ave|Suite B|Riverview|Coshocton County|OH|49003|United States|-5|condo| +8627|AAAAAAAADLBCAAAA|625|Church 2nd|Wy|Suite A|Greenfield|Kleberg County|TX|75038|United States|-6|single family| +8628|AAAAAAAAELBCAAAA|336|3rd |Parkway|Suite 400|Brownsville|Suffolk County|MA|09910|United States|-5|single family| +8629|AAAAAAAAFLBCAAAA|299|Pine |Circle|Suite 270|Deerfield|Owen County|KY|49840|United States|-5|apartment| +8630|AAAAAAAAGLBCAAAA|396|Hillcrest |Lane|Suite 490|Salem|Camas County|ID|88048|United States|-7|condo| +8631|AAAAAAAAHLBCAAAA|323|Highland Spring|Ln|Suite 340|Mount Pleasant|DeBaca County|NM|81933|United States|-7|condo| +8632|AAAAAAAAILBCAAAA|59|Forest Jefferson|Lane|Suite D|Woodville|Alexander County|NC|24289|United States|-5|single family| +8633|AAAAAAAAJLBCAAAA|907|Green North|Ct.|Suite N|Farmington|Placer County|CA|99145|United States|-8|single family| +8634|AAAAAAAAKLBCAAAA|154|Main Park|RD|Suite J|Walnut Grove|Allegany County|MD|27752|United States|-5|apartment| +8635|AAAAAAAALLBCAAAA|795|4th Wilson|Ln|Suite T|Kingston|Glades County|FL|34975|United States|-5|condo| +8636|AAAAAAAAMLBCAAAA|407|Lincoln Davis|Road|Suite 400|Valley View|Elbert County|GA|35124|United States|-5|single family| +8637|AAAAAAAANLBCAAAA|447|2nd 7th|Ave|Suite B|Belleview|Walker County|AL|30492|United States|-6|condo| +8638|AAAAAAAAOLBCAAAA|419|Second Center|Wy|Suite C|Glendale|Guthrie County|IA|53951|United States|-6|apartment| +8639|AAAAAAAAPLBCAAAA|141|Cherry Main|Parkway|Suite S|Willow|Susquehanna County|PA|16798|United States|-5|single family| +8640|AAAAAAAAAMBCAAAA|978|Park |Ln|Suite Y|Five Points|Chittenden County|VT|06698|United States|-5|condo| +8641|AAAAAAAABMBCAAAA|241|Oak Park|Street|Suite V|Macedonia|Walker County|TX|71087|United States|-6|condo| +8642|AAAAAAAACMBCAAAA|116|1st Johnson|Road|Suite W|Five Forks|Pontotoc County|OK|72293|United States|-6|apartment| +8643|AAAAAAAADMBCAAAA|31|Park |Cir.|Suite W|Centerville|Muskogee County|OK|70059|United States|-6|single family| +8644|AAAAAAAAEMBCAAAA|887|Maple First|Parkway|Suite Y|Georgetown|Berks County|PA|17057|United States|-5|apartment| +8645|AAAAAAAAFMBCAAAA|369|Davis Cherry|Boulevard|Suite 110|Oak Hill|Carroll County|OH|47838|United States|-5|single family| +8646|AAAAAAAAGMBCAAAA|687|12th |Boulevard|Suite 370|Riverside|Shasta County|CA|99231|United States|-8|condo| +8647|AAAAAAAAHMBCAAAA|857|2nd Woodland|Dr.|Suite 450|Welcome|Jackson County|OH|46386|United States|-5|condo| +8648|AAAAAAAAIMBCAAAA|96|11th Hillcrest|Pkwy|Suite 490|Woodlawn|Orange County|VA|24098|United States|-5|apartment| +8649|AAAAAAAAJMBCAAAA|579|1st Spring|Drive|Suite A|Stewart|Aiken County|SC|28041|United States|-5|single family| +8650|AAAAAAAAKMBCAAAA|120|Eigth |Parkway|Suite U|Spring Valley|Sheboygan County|WI|56060|United States|-6|condo| +8651|AAAAAAAALMBCAAAA|61|Meadow ||Suite 140|Crossroads|Pierce County|NE||||condo| +8652|AAAAAAAAMMBCAAAA|22|Railroad |Pkwy|Suite 460|Union|Vinton County|OH|48721|United States|-5|single family| +8653|AAAAAAAANMBCAAAA|247|View |Boulevard|Suite 370|Riverdale|Coos County|NH|09991|United States|-5|single family| +8654|AAAAAAAAOMBCAAAA|874|Railroad |Avenue|Suite 50|Dewey|Carter County|MO|61160|United States|-6|single family| +8655|AAAAAAAAPMBCAAAA|251|Meadow |Pkwy|Suite 40|Green Acres|Dougherty County|GA|37683|United States|-5|condo| +8656|AAAAAAAAANBCAAAA|350|Walnut |Lane|Suite C|Salem|Lafayette County|WI|58048|United States|-6|condo| +8657|AAAAAAAABNBCAAAA|756|3rd Williams|Ct.|Suite Q|Ashland|Greene County|IA|54244|United States|-6|condo| +8658|AAAAAAAACNBCAAAA|242|Elm Pine|Blvd|Suite 180|Pine Grove|Jackson County|SD|54593|United States|-7|condo| +8659|AAAAAAAADNBCAAAA|208|Woodland |Lane|Suite F|Post Oak|Loudon County|TN|38567|United States|-6|apartment| +8660|AAAAAAAAENBCAAAA|811|Chestnut |Street|Suite 150|Vista|King George County|VA|24694|United States|-5|condo| +8661|AAAAAAAAFNBCAAAA|788|South |Ave|Suite H|Mount Pleasant|Howard County|IN|41933|United States|-5|apartment| +8662|AAAAAAAAGNBCAAAA|173|Second Fourth|Circle|Suite 480|Fairview|Washington County|IA|55709|United States|-6|apartment| +8663|AAAAAAAAHNBCAAAA|821|Jackson |ST|Suite Q|Woodruff|Valdez-Cordova Census Area|AK|94174|United States|-9|single family| +8664|AAAAAAAAINBCAAAA|157|East |Drive|Suite E|Riverview|Charlotte County|FL|39003|United States|-5|condo| +8665|AAAAAAAAJNBCAAAA|309|Adams |Ave|Suite U|Crossroads|Fayette County|PA|10534|United States|-5|single family| +8666|AAAAAAAAKNBCAAAA|822|Elm |Boulevard|Suite W|Mount Vernon|Swift County|MN|58482|United States|-6|condo| +8667|AAAAAAAALNBCAAAA|268|Dogwood |Parkway|Suite X|Wildwood|Bossier Parish|LA|76871|United States|-6|single family| +8668|AAAAAAAAMNBCAAAA|215|3rd Valley|Cir.|Suite 460|Oakdale|Clear Creek County|CO|89584|United States|-7|single family| +8669|AAAAAAAANNBCAAAA|468|Oak Elm|Way|Suite Y|Westminster|Tyler County|TX|76549|United States|-6|single family| +8670|AAAAAAAAONBCAAAA|473||||||||||| +8671|AAAAAAAAPNBCAAAA|628|Elevnth Fifth|Parkway|Suite 60|Deerfield|Donley County|TX|79840|United States|-6|apartment| +8672|AAAAAAAAAOBCAAAA|923|Walnut |Wy|Suite 290|Bridgeport|Morris County|NJ|06417|United States|-5|apartment| +8673|AAAAAAAABOBCAAAA|459|Park |Way|Suite W|Lebanon|Bullitt County|KY|42898|United States|-6|apartment| +8674|AAAAAAAACOBCAAAA|740|Lake |Way|Suite 250|Fairview|Lemhi County|ID|85709|United States|-7|condo| +8675|AAAAAAAADOBCAAAA|797|Railroad Davis|Court|Suite 340|Wilson|Pike County|GA|36971|United States|-5|single family| +8676|AAAAAAAAEOBCAAAA|657|View North|Cir.|Suite 190|Union|Eddy County|ND|58721|United States|-6|apartment| +8677|AAAAAAAAFOBCAAAA|719|Tenth Cedar|Cir.|Suite E|Edgewood|Dallas County|MO|60069|United States|-6|condo| +8678|AAAAAAAAGOBCAAAA|693|Third 1st|ST|Suite H|Rossville|Mille Lacs County|MN|52503|United States|-6|condo| +8679|AAAAAAAAHOBCAAAA|6|Oak Sunset|ST|Suite A|Hopewell|Gibson County|TN|30587|United States|-5|condo| +8680|AAAAAAAAIOBCAAAA|16|Willow Adams|Ct.|Suite 20|Pleasant Valley|Wayne County|PA|12477|United States|-5|apartment| +8681|AAAAAAAAJOBCAAAA|253|Jefferson |Circle|Suite O|Oakwood|King and Queen County|VA|20169|United States|-5|condo| +8682|AAAAAAAAKOBCAAAA|640|Davis |ST|Suite 230|Clinton|Multnomah County|OR|98222|United States|-8|apartment| +8683|AAAAAAAALOBCAAAA|595|West |Boulevard|Suite W|Lincoln|Comanche County|OK|71289|United States|-6|single family| +8684|AAAAAAAAMOBCAAAA|540|11th Woodland|Pkwy|Suite J|Oak Hill|Bradford County|FL|37838|United States|-5|condo| +8685|AAAAAAAANOBCAAAA|369|Elm Lake|Boulevard|Suite F|Highland|Rowan County|KY|49454|United States|-5|apartment| +8686|AAAAAAAAOOBCAAAA|285|Highland Railroad|Circle|Suite 240|Deerfield|Coryell County|TX|79840|United States|-6|apartment| +8687|AAAAAAAAPOBCAAAA|294|Sixth Park|Dr.|Suite R|Oakdale|Waller County|TX|79584|United States|-6|condo| +8688|AAAAAAAAAPBCAAAA|349|Lake |Drive|Suite 30|Jamestown|Dixie County|FL|36867|United States|-5|condo| +8689|AAAAAAAABPBCAAAA|357|Second |Boulevard|Suite T|Mount Vernon|Trinity County|CA|98482|United States|-8|single family| +8690|AAAAAAAACPBCAAAA|494|First |Boulevard|Suite 220|New Hope|Wapello County|IA|59431|United States|-6|apartment| +8691|AAAAAAAADPBCAAAA|523|Third |Circle|Suite 420|Oakdale|Bristol County|MA|09584|United States|-5|single family| +8692|AAAAAAAAEPBCAAAA|131|Spring Cedar|Blvd|Suite 240|Union|Warren County|IN|48721|United States|-5|apartment| +8693|AAAAAAAAFPBCAAAA|518|Ash |Ln|Suite 380|Walnut Grove|East Feliciana Parish|LA|77752|United States|-6|single family| +8694|AAAAAAAAGPBCAAAA|160|Pine |Lane|Suite 280|Greenbrier|Williamson County|IL|67854|United States|-6|single family| +8695|AAAAAAAAHPBCAAAA|375|Miller Seventh|Pkwy|Suite A|Pleasant Valley|Wabash County|IN|42477|United States|-5|single family| +8696|AAAAAAAAIPBCAAAA|336|Lakeview |Blvd|Suite 40|Union|McDowell County|WV|28721|United States|-5|single family| +8697|AAAAAAAAJPBCAAAA|293|Williams |Blvd|Suite 150|Argyle|Red River County|TX|78722|United States|-6|single family| +8698|AAAAAAAAKPBCAAAA|835|Smith |RD|Suite Q|Pleasant Valley|Escambia County|AL|32477|United States|-6|apartment| +8699|AAAAAAAALPBCAAAA|532|Ridge |Circle|Suite K|Riverview|Cass County|IA|59003|United States|-6|apartment| +8700|AAAAAAAAMPBCAAAA|553|Chestnut |Lane|Suite N|Union Hill|DeKalb County|IL|67746|United States|-6|apartment| +8701|AAAAAAAANPBCAAAA|986|South Green|Blvd|Suite 330|Highland Park|Sherman County|KS|66534|United States|-6|apartment| +8702|AAAAAAAAOPBCAAAA|874|Maple Mill|Avenue|Suite V|Oakdale|Washington County|OR|99584|United States|-8|apartment| +8703|AAAAAAAAPPBCAAAA|658|Oak |Wy|Suite 0|Galena|Jefferson County|IA|54369|United States|-6|apartment| +8704|AAAAAAAAAACCAAAA|743|Sunset |ST|Suite K|Crossroads|Hoke County|NC|20534|United States|-5|apartment| +8705|AAAAAAAABACCAAAA|368|13th |Ct.|Suite 170|Farmington|Copiah County|MS|59145|United States|-6|condo| +8706|AAAAAAAACACCAAAA|605|Lakeview Poplar|Parkway|Suite K|Woodlawn|Brazos County|TX|74098|United States|-6|condo| +8707|AAAAAAAADACCAAAA|919|Washington |Lane||||KY||||| +8708|AAAAAAAAEACCAAAA|603|3rd Madison|Circle|Suite 180|Highland Park|Teton County|ID|86534|United States|-7|single family| +8709|AAAAAAAAFACCAAAA|358|Spruce Maple|RD|Suite Q|Enterprise|Polk County|NE|61757|United States|-7|condo| +8710|AAAAAAAAGACCAAAA|271|Willow South|Parkway|Suite K|Pleasant Hill|Orange County|IN|43604|United States|-5|condo| +8711|AAAAAAAAHACCAAAA|764|Spruce |Ct.|Suite 50|Shiloh|Levy County|FL|39275|United States|-5|single family| +8712|AAAAAAAAIACCAAAA||||Suite G||Shelby County|MO|65709||-6|| +8713|AAAAAAAAJACCAAAA|193|East Cedar|Pkwy|Suite I|Riverdale|Calhoun County|AL|39391|United States|-6|apartment| +8714|AAAAAAAAKACCAAAA|591|Woodland |Cir.|Suite S|Clifford|Hamlin County|SD|58164|United States|-7|condo| +8715|AAAAAAAALACCAAAA|265|Park Main|Blvd|Suite C|Floyd|Manatee County|FL|33235|United States|-5|condo| +8716|AAAAAAAAMACCAAAA|505|Washington Wilson|Dr.|Suite L|Plainview|Perkins County|SD|53683|United States|-7|condo| +8717|AAAAAAAANACCAAAA|94|Tenth |Ct.|Suite D|Springdale|Shawnee County|KS|68883|United States|-6|apartment| +8718|AAAAAAAAOACCAAAA|504|Madison Johnson|Wy|Suite B|Marion|Washington County|NE|60399|United States|-7|condo| +8719|AAAAAAAAPACCAAAA|245|3rd |Circle|Suite R|Woodville|Calhoun County|IA|54289|United States|-6|single family| +8720|AAAAAAAAABCCAAAA|455|Mill |Lane|Suite 460|Valley View|Pike County|IN|45124|United States|-5|single family| +8721|AAAAAAAABBCCAAAA|458|Railroad |RD|Suite L|Midway|Henry County|AL|31904|United States|-6|apartment| +8722|AAAAAAAACBCCAAAA|260|Dogwood View|Drive|Suite L|Plainview|Gregory County|SD|53683|United States|-6|single family| +8723|AAAAAAAADBCCAAAA|787|Woodland Laurel|Street|Suite L|Highland|Greene County|GA|39454|United States|-5|condo| +8724|AAAAAAAAEBCCAAAA|595|Hickory Adams|Ct.|Suite 180|Lakeview|McDowell County|WV|28579|United States|-5|condo| +8725|AAAAAAAAFBCCAAAA|74|Eigth |Dr.|Suite X|Pleasant Valley|Navajo County|AZ|82477|United States|-7|condo| +8726|AAAAAAAAGBCCAAAA|348|Franklin View|Circle|Suite Y|King|Hooker County|NE|60008|United States|-7|condo| +8727|AAAAAAAAHBCCAAAA|190|View Green|Avenue|Suite 430|Lakeside|Hartford County|CT|09532|United States|-5|condo| +8728|AAAAAAAAIBCCAAAA|735|Dogwood |Boulevard|Suite M|Glenwood|Uinta County|WY|83511|United States|-7|single family| +8729|AAAAAAAAJBCCAAAA|216|East |ST|Suite A|Unionville|McLean County|ND|51711|United States|-6|single family| +8730|AAAAAAAAKBCCAAAA|123|13th |Ave|Suite Q|Oakdale|Scurry County|TX|79584|United States|-6|apartment| +8731|AAAAAAAALBCCAAAA|47|Mill 7th|Lane|Suite 270|Crossroads|Lagrange County|IN|40534|United States|-5|single family| +8732|AAAAAAAAMBCCAAAA|255|Oak Laurel|Dr.|Suite 10|Riverdale|Tipton County|TN|39391|United States|-6|condo| +8733|AAAAAAAANBCCAAAA|174|South |Pkwy|Suite 430|Macedonia|Fremont County|IA|51087|United States|-6|apartment| +8734|AAAAAAAAOBCCAAAA|235|Park Chestnut|Blvd|Suite 60|Parker|Fountain County|IN|41373|United States|-5|apartment| +8735|AAAAAAAAPBCCAAAA|135|Highland Main|Drive|Suite 110|Newport|Wayne County|IL|61521|United States|-6|single family| +8736|AAAAAAAAACCCAAAA|767|Oak |Lane|Suite 460|Oakwood|Potter County|SD|50169|United States|-7|apartment| +8737|AAAAAAAABCCCAAAA|947|Lake Lakeview|Wy|Suite 390|Riverview|Dukes County|MA|09603|United States|-5|condo| +8738|AAAAAAAACCCCAAAA|270|Lee Railroad|Parkway|Suite 190|Maple Grove|Butte County|ID|88252|United States|-7|single family| +8739|AAAAAAAADCCCAAAA|572|Forest |ST|Suite G|Hillcrest|Cole County|MO|63003|United States|-6|single family| +8740|AAAAAAAAECCCAAAA|467|Main Center|Cir.|Suite L|Brunswick|Martin County|IN|44642|United States|-5|apartment| +8741|AAAAAAAAFCCCAAAA|810|11th Seventh|Ln|Suite 220|Hopewell|Dixie County|FL|30587|United States|-5|apartment| +8742|AAAAAAAAGCCCAAAA|141|Madison |RD|Suite 170|Friendship|Salt Lake County|UT|84536|United States|-7|single family| +8743|AAAAAAAAHCCCAAAA|188|Chestnut First|Circle|Suite 150|Ashley|Elko County|NV|84324|United States|-8|single family| +8744|AAAAAAAAICCCAAAA|864|Spruce |Dr.|Suite 170|Enterprise|Mississippi County|AR|71757|United States|-6|apartment| +8745|AAAAAAAAJCCCAAAA|310|Mill |Pkwy|Suite A|Highland Park|George County|MS|56534|United States|-6|single family| +8746|AAAAAAAAKCCCAAAA|111|Center Cedar|Drive|Suite N|Hamilton|Tarrant County|TX|72808|United States|-6|condo| +8747|AAAAAAAALCCCAAAA||Eigth ||Suite D||Paulding County|GA|35124|United States|-5|condo| +8748|AAAAAAAAMCCCAAAA|890|Walnut Hillcrest|Blvd|Suite S|Buena Vista|Seminole County|FL|35752|United States|-5|apartment| +8749|AAAAAAAANCCCAAAA|577|Elm 6th|Street|Suite 440|White Oak|Chaffee County|CO|86668|United States|-7|single family| +8750|AAAAAAAAOCCCAAAA|236|Main |Cir.|Suite G|Mount Olive|Campbell County|SD|58059|United States|-6|single family| +8751|AAAAAAAAPCCCAAAA|830|Forest 7th|Ave|Suite W|Oakland|Crook County|OR|99843|United States|-8|single family| +8752|AAAAAAAAADCCAAAA|920|Ninth Maple|Drive|Suite 430|Greenville|McCurtain County|OK|71387|United States|-6|condo| +8753|AAAAAAAABDCCAAAA|24|Fifth Adams|Lane|Suite 320|Whitney|Huron County|OH|48339|United States|-5|single family| +8754|AAAAAAAACDCCAAAA|999|6th |Blvd|Suite V|Pleasant Grove|Dillingham Census Area|AK|94136|United States|-9|condo| +8755|AAAAAAAADDCCAAAA|840|Jefferson Valley|Ln|Suite M|Lakeview|Tippah County|MS|58579|United States|-6|condo| +8756|AAAAAAAAEDCCAAAA|725|15th |Parkway|Suite 270|Lincoln|Redwood County|MN|51289|United States|-6|single family| +8757|AAAAAAAAFDCCAAAA|559|Cedar Walnut|Court|Suite V|Edgewood|Lafourche Parish|LA|70069|United States|-6|single family| +8758|AAAAAAAAGDCCAAAA|781|Oak |Blvd|Suite 380|Springfield|Escambia County|AL|39303|United States|-6|single family| +8759|AAAAAAAAHDCCAAAA|103|North South|Parkway|Suite A|Arlington|Early County|GA|36557|United States|-5|apartment| +8760|AAAAAAAAIDCCAAAA|306|Mill |Parkway|Suite 280|Union|Sutter County|CA|98721|United States|-8|apartment| +8761|AAAAAAAAJDCCAAAA|320|Elm Forest|Ln|Suite S|Springfield|Baker County|FL|39303|United States|-5|single family| +8762|AAAAAAAAKDCCAAAA|12|Cedar |Way|Suite 370|Maple Grove|Murray County|GA|38252|United States|-5|single family| +8763|AAAAAAAALDCCAAAA||||||Edgecombe County|NC||||| +8764|AAAAAAAAMDCCAAAA|806|First West|Court|Suite 240|Summit|Ballard County|KY|40499|United States|-6|single family| +8765|AAAAAAAANDCCAAAA|573|Washington Smith|Cir.|Suite W|Lakeside|Dinwiddie County|VA|29532|United States|-5|condo| +8766|AAAAAAAAODCCAAAA|741|Washington Elm|Dr.|Suite 210|Hamilton|Radford city|VA|22808|United States|-5|apartment| +8767|AAAAAAAAPDCCAAAA|77|Main |Cir.|Suite 160|Glenwood|Allegheny County|PA|13511|United States|-5|apartment| +8768|AAAAAAAAAECCAAAA|95|9th |Blvd|Suite Y|Oakland|Emmet County|IA|59843|United States|-6|condo| +8769|AAAAAAAABECCAAAA|433|Locust Forest|Ct.|Suite O|Sulphur Springs|Pickens County|GA|38354|United States|-5|single family| +8770|AAAAAAAACECCAAAA|723|College |RD|Suite 190|Glendale|Nemaha County|KS|63951|United States|-6|single family| +8771|AAAAAAAADECCAAAA|833|Woodland |Court|Suite 480|Harmony|Cimarron County|OK|75804|United States|-6|apartment| +8772|AAAAAAAAEECCAAAA|624|4th |Ct.|Suite Q|Woodville||CO|84289|United States||condo| +8773|AAAAAAAAFECCAAAA|214|10th |Blvd|Suite W|Mountain View|Barnstable County|MA|05066|United States|-5|single family| +8774|AAAAAAAAGECCAAAA|329|4th River|Blvd|Suite D|Midway|Ontario County|NY|11904|United States|-5|apartment| +8775|AAAAAAAAHECCAAAA|618|Johnson Johnson|Street|Suite 300|Green Acres|Webster County|MO|67683|United States|-6|condo| +8776|AAAAAAAAIECCAAAA|660|Miller Lincoln|Way|Suite I|Lebanon|Somerset County|PA|12898|United States|-5|apartment| +8777|AAAAAAAAJECCAAAA|587|Second Adams|Drive|Suite O|Buena Vista|Indian River County|FL|35752|United States|-5|apartment| +8778|AAAAAAAAKECCAAAA|880|View Lincoln|Ct.|Suite M|Hamilton|Big Horn County|WY|82808|United States|-7|apartment| +8779|AAAAAAAALECCAAAA|524|Elm Maple|Blvd|Suite B|Summit|Richmond County|GA|30499|United States|-5|condo| +8780|AAAAAAAAMECCAAAA|915|Elm |Pkwy|Suite 340|Woodlawn|Montgomery County|OH|44098|United States|-5|single family| +8781|AAAAAAAANECCAAAA|317|Fifth Center|Cir.|Suite 320|Plainview|Hamilton County|OH|43683|United States|-5|condo| +8782|AAAAAAAAOECCAAAA|184|Washington |Ct.|Suite 420|Union|Johnson County|KS|68721|United States|-6|single family| +8783|AAAAAAAAPECCAAAA|||Cir.||Kingston|Montgomery County||74975|United States|-6|single family| +8784|AAAAAAAAAFCCAAAA||Wilson Second|||||NE||||single family| +8785|AAAAAAAABFCCAAAA|561|South |Court|Suite I|Woodland Park|Lafourche Parish|LA|71934|United States|-6|single family| +8786|AAAAAAAACFCCAAAA|844|Laurel |Lane|Suite X|||PA|14854|United States||apartment| +8787|AAAAAAAADFCCAAAA|947|Davis Park|Ave|Suite Y|Lakewood|Nez Perce County|ID|88877|United States|-7|single family| +8788|AAAAAAAAEFCCAAAA|497|Davis 6th|Avenue|Suite 250|Waterloo|Stevens County|MN|51675|United States|-6|apartment| +8789|AAAAAAAAFFCCAAAA|617|Johnson Adams|Circle|Suite 290|Highland Park|Terry County|TX|76534|United States|-6|apartment| +8790|AAAAAAAAGFCCAAAA|615|Pine Washington|Drive|Suite 380|King|Newport County|RI|00608|United States|-5|apartment| +8791|AAAAAAAAHFCCAAAA|397|Green |RD|Suite Q|Woodland|Sullivan County|TN|34854|United States|-6|single family| +8792|AAAAAAAAIFCCAAAA|808|3rd Mill|Avenue|Suite Y|Oakdale|Otero County|CO|89584|United States|-7|apartment| +8793|AAAAAAAAJFCCAAAA|7|11th Tenth|RD|Suite 470|Springdale|Rockingham County|NH|09483|United States|-5|single family| +8794|AAAAAAAAKFCCAAAA|453|Fifth |Wy|Suite 130|Bunker Hill|Hill County|MT|60150|United States|-7|single family| +8795|AAAAAAAALFCCAAAA|541|Railroad |Drive|Suite 120|Kingston|Erie County|OH|44975|United States|-5|condo| +8796|AAAAAAAAMFCCAAAA|828|Ninth 3rd|Pkwy|Suite 220|Oakwood|Liberty County|FL|30169|United States|-5|condo| +8797|AAAAAAAANFCCAAAA|197|Woodland View|Ct.|Suite 380|Highland|Limestone County|TX|79454|United States|-6|condo| +8798|AAAAAAAAOFCCAAAA|942|Oak |Ct.|Suite P|Midway|Grundy County|IA|51904|United States|-6|apartment| +8799|AAAAAAAAPFCCAAAA|455|River |Road|Suite L|Walnut Grove|Colorado County|TX|77752|United States|-6|apartment| +8800|AAAAAAAAAGCCAAAA|39|Twelfth |Road|Suite W|Allentown|Onondaga County|NY|11838|United States|-5|single family| +8801|AAAAAAAABGCCAAAA|338|East |Way|Suite 40|Woodland|Caroline County|VA|24854|United States|-5|apartment| +8802|AAAAAAAACGCCAAAA|673|East Pine|Circle|Suite W|Union|Kendall County|IL|68721|United States|-6|apartment| +8803|AAAAAAAADGCCAAAA|311|Main |Way|Suite 90|Five Forks|Brown County|IN|42293|United States|-5|apartment| +8804|AAAAAAAAEGCCAAAA|30|Forest |Dr.|Suite S|Woodlawn|Garza County|TX|74098|United States|-6|single family| +8805|AAAAAAAAFGCCAAAA|931|Sycamore 2nd|Road|Suite E|Oak Ridge|Butler County|IA|58371|United States|-6|single family| +8806|AAAAAAAAGGCCAAAA|83|Sycamore 5th|Dr.|Suite P|Jackson|Citrus County|FL|39583|United States|-5|single family| +8807|AAAAAAAAHGCCAAAA|191|Walnut |Boulevard|Suite X|Clifton|Summers County|WV|28014|United States|-5|apartment| +8808|AAAAAAAAIGCCAAAA|6|East Center|Circle|Suite W|Adrian|Garden County|NE|63301|United States|-6|single family| +8809|AAAAAAAAJGCCAAAA|154|West |Street|Suite 0|Newport|West Baton Rouge Parish|LA|71521|United States|-6|condo| +8810|AAAAAAAAKGCCAAAA|984|Walnut Walnut|Ct.|Suite 200|Bethel|Douglas County|GA|35281|United States|-5|apartment| +8811|AAAAAAAALGCCAAAA|712|Fourth |Court|Suite 140|Montague|San Augustine County|TX|74062|United States|-6|apartment| +8812|AAAAAAAAMGCCAAAA|617|Second |Drive|Suite T|Summit|Passaic County|NJ|01099|United States|-5|condo| +8813|AAAAAAAANGCCAAAA|917|5th 14th|Street|Suite 80|Providence|Parmer County|TX|76614|United States|-6|apartment| +8814|AAAAAAAAOGCCAAAA|216|Wilson 1st|Wy|Suite 450|Greenfield|Haskell County|TX|75038|United States|-6|condo| +8815|AAAAAAAAPGCCAAAA|372|Sixth Birch|Street|Suite U|Shady Grove|Appling County|GA|32812|United States|-5|condo| +8816|AAAAAAAAAHCCAAAA||Church River||Suite 460|Riverside|Edwards County|IL||United States|-6|single family| +8817|AAAAAAAABHCCAAAA|756|6th |Ln|Suite T|Glendale|Noble County|IN|43951|United States|-5|condo| +8818|AAAAAAAACHCCAAAA|811|Walnut |RD|Suite S|Summit|Rappahannock County|VA|20499|United States|-5|condo| +8819|AAAAAAAADHCCAAAA|31|Broadway |Circle|Suite F|Harmon|Dukes County|MA|06223|United States|-5|apartment| +8820|AAAAAAAAEHCCAAAA|191|Second |Boulevard|Suite H|Salem|Logan County|ND|58048|United States|-6|condo| +8821|AAAAAAAAFHCCAAAA|183|7th |Ct.|Suite 70|Springfield|Dinwiddie County|VA|29303|United States|-5|apartment| +8822|AAAAAAAAGHCCAAAA|63|South Lincoln|Ct.|Suite 20|Liberty|Washington County|KY|43451|United States|-5|apartment| +8823|AAAAAAAAHHCCAAAA|926|Church Washington|Way|Suite P|Hamilton|Jim Wells County|TX|72808|United States|-6|apartment| +8824|AAAAAAAAIHCCAAAA|887|3rd |Pkwy|Suite 370|Union|Guernsey County|OH|48721|United States|-5|condo| +8825|AAAAAAAAJHCCAAAA|411|5th Central|Avenue|Suite B|Martinsville|Warren County|MS|50419|United States|-6|apartment| +8826|AAAAAAAAKHCCAAAA|753|Laurel |Wy|Suite 90|Greenwood|Dade County|MO|68828|United States|-6|apartment| +8827|AAAAAAAALHCCAAAA|196|Lincoln South|Dr.|Suite 480|Maple Grove|Ripley County|IN|48252|United States|-5|condo| +8828|AAAAAAAAMHCCAAAA|155|Birch |Avenue|Suite 400|Wilson|Bell County|KY|46971|United States|-6|apartment| +8829|AAAAAAAANHCCAAAA|390|Spring |Lane|Suite 200|White Oak|Scott County|KS|66668|United States|-6|condo| +8830|AAAAAAAAOHCCAAAA|995|Mill 10th|Dr.|Suite W|Siloam|Grant County|MN|58948|United States|-6|apartment| +8831|AAAAAAAAPHCCAAAA|480|11th Washington|Boulevard|Suite I|Stratford|Powell County|MT|66668|United States|-7|apartment| +8832|AAAAAAAAAICCAAAA|804|Willow Jefferson|Ave|Suite C|Oak Ridge|Fredericksburg city|VA|28371|United States|-5|single family| +8833|AAAAAAAABICCAAAA|651|North 3rd|Avenue|Suite 300|Jamestown|Honolulu County|HI|96867|United States|-10|condo| +8834|AAAAAAAACICCAAAA|258|Lake |Ct.|Suite F|Lakeside|Kewaunee County|WI|59532|United States|-6|apartment| +8835|AAAAAAAADICCAAAA|78|Oak Maple|Road|Suite R|Mount Zion|Sweet Grass County|MT|68054|United States|-7|apartment| +8836|AAAAAAAAEICCAAAA|796|Pine Davis|Circle|Suite 170|Pine Grove|Branch County|MI|44593|United States|-5|apartment| +8837|AAAAAAAAFICCAAAA|702|4th Fifth|ST|Suite 210|Mount Pleasant|Wheeler County|NE|61933|United States|-7|apartment| +8838|AAAAAAAAGICCAAAA|740|9th |RD|Suite A|Cedar Grove|Cedar County|IA|50411|United States|-6|condo| +8839|AAAAAAAAHICCAAAA|100|Sixth Park|Road|Suite 70|Mount Olive|Bamberg County|SC|28059|United States|-5|apartment| +8840|AAAAAAAAIICCAAAA|796|4th East|Parkway|Suite C|Glenwood|Sullivan County|MO|63511|United States|-6|apartment| +8841|AAAAAAAAJICCAAAA|759|Franklin |Lane|Suite 290|Summit|Salt Lake County|UT|80499|United States|-7|single family| +8842|AAAAAAAAKICCAAAA|483|Jefferson Walnut|Dr.|Suite F|Riverside|Greeley County|KS|69231|United States|-6|single family| +8843|AAAAAAAALICCAAAA|91|Franklin Valley|ST|Suite U|Edgewood|Allamakee County|IA|50069|United States|-6|condo| +8844|AAAAAAAAMICCAAAA|606|South 3rd|Ln|Suite P|Summit|Edgar County|IL|60499|United States|-6|single family| +8845|AAAAAAAANICCAAAA|382|||Suite Q||Highland County|OH||||| +8846|AAAAAAAAOICCAAAA|891|Washington |RD|Suite Y|Newtown|Pondera County|MT|61749|United States|-7|condo| +8847|AAAAAAAAPICCAAAA|103|Locust Green|Parkway|Suite N|Farmington|Upshur County|TX|79145|United States|-6|condo| +8848|AAAAAAAAAJCCAAAA|418|Lee |Ave|Suite B|Concord|Smith County|TN|34107|United States|-6|single family| +8849|AAAAAAAABJCCAAAA|890|Poplar |ST|Suite R|Highland Park|Montgomery County|KY|46534|United States|-5|apartment| +8850|AAAAAAAACJCCAAAA|866|14th |Avenue|Suite 20|Hamilton|Shoshone County|ID|82808|United States|-7|single family| +8851|AAAAAAAADJCCAAAA|195|Lake |Avenue|Suite A|Bridgeport|Clatsop County|OR|95817|United States|-8|single family| +8852|AAAAAAAAEJCCAAAA|215|1st Smith|Ave|Suite G|Sunnyside|Cheyenne County|CO|81952|United States|-7|apartment| +8853|AAAAAAAAFJCCAAAA|715|Oak |Lane|Suite O|Mount Vernon|Stewart County|GA|38482|United States|-5|apartment| +8854|AAAAAAAAGJCCAAAA||||Suite 250|||MT||United States||| +8855|AAAAAAAAHJCCAAAA|351|3rd |Drive|Suite 20|Cedar Grove|Lancaster County|SC|20411|United States|-5|single family| +8856|AAAAAAAAIJCCAAAA|430|Second |Wy|Suite 380|Four Points|Dewey County|SD|51216|United States|-6|single family| +8857|AAAAAAAAJJCCAAAA|576|Mill |Dr.|Suite 460|Pinhook|Hyde County|SD|59398|United States|-7|apartment| +8858|AAAAAAAAKJCCAAAA|546|Park Pine|Cir.|Suite H|Riverview|Jefferson County|PA|19003|United States|-5|single family| +8859|AAAAAAAALJCCAAAA|722|Washington Church|Road|Suite 60|Clifton|Tattnall County|GA|38014|United States|-5|single family| +8860|AAAAAAAAMJCCAAAA|803|Fourth Park|Ct.|Suite R|Belmont|Yakima County|WA|90191|United States|-8|single family| +8861|AAAAAAAANJCCAAAA|334|College |RD|Suite X|Woodlawn|Butler County|KY|44098|United States|-6|apartment| +8862|AAAAAAAAOJCCAAAA|281|4th |Drive|Suite 310|Lebanon|Trimble County|KY|42898|United States|-5|condo| +8863|AAAAAAAAPJCCAAAA|248|Oak View|ST|Suite N|Midway|Seminole County|FL|31904|United States|-5|condo| +8864|AAAAAAAAAKCCAAAA|478|7th |Ct.|Suite 460|Summit|Shannon County|SD|50499|United States|-7|condo| +8865|AAAAAAAABKCCAAAA|796|Willow |Drive|Suite J|Pleasant Grove|Beaufort County|NC|24136|United States|-5|condo| +8866|AAAAAAAACKCCAAAA|246|Smith Main|Circle|Suite U|Carpenter|Titus County|TX|71147|United States|-6|condo| +8867|AAAAAAAADKCCAAAA|272|South |Way|Suite U|Lakeside|Denver County|CO|89532|United States|-7|apartment| +8868|AAAAAAAAEKCCAAAA|797|4th |Wy|Suite V|White Oak|Pender County|NC|26668|United States|-5|apartment| +8869|AAAAAAAAFKCCAAAA|783|5th |Avenue|Suite G|White Oak|Pawnee County|OK|76668|United States|-6|single family| +8870|AAAAAAAAGKCCAAAA|933|Twelfth Highland|Court|Suite 0|Midway|Anchorage Borough|AK|91904|United States|-9|single family| +8871|AAAAAAAAHKCCAAAA|104|Davis 1st|Pkwy|Suite X|Bennett|Davison County|SD|51715|United States|-6|single family| +8872|AAAAAAAAIKCCAAAA|962|4th |Ln|Suite 270|Lakeview|Wyoming County|NY|18579|United States|-5|condo| +8873|AAAAAAAAJKCCAAAA|942|Lincoln Elm|Court|Suite 290|Springfield|Wabash County|IL|69303|United States|-6|single family| +8874|AAAAAAAAKKCCAAAA|713|14th Park|Ct.|Suite N|Enterprise|Coos County|OR|91757|United States|-8|single family| +8875|AAAAAAAALKCCAAAA|381|Willow |Drive|Suite F|Jamestown|Humphreys County|MS|56867|United States|-6|condo| +8876|AAAAAAAAMKCCAAAA|296|Oak |Way|Suite N|Hopewell|Wilcox County|AL|30587|United States|-6|condo| +8877|AAAAAAAANKCCAAAA|713|7th |Circle|Suite U|Wilson|Lanier County|GA|36971|United States|-5|single family| +8878|AAAAAAAAOKCCAAAA|622|Cherry 2nd|Ave|Suite 70|Harmony|Marion County|MS|55804|United States|-6|condo| +8879|AAAAAAAAPKCCAAAA|641|Elm Oak|Dr.|Suite X|Green Acres|Barber County|KS|67683|United States|-6|single family| +8880|AAAAAAAAALCCAAAA|83|Ash |Pkwy|Suite 130|Leesville|Adams County|WI|55423|United States|-6|condo| +8881|AAAAAAAABLCCAAAA|252|8th |RD|Suite 120|Midway|Jackson County|MI|41904|United States|-5|apartment| +8882|AAAAAAAACLCCAAAA|319|Fourth |Parkway|Suite 150|Pleasant Hill|Putnam County|NY|13604|United States|-5|single family| +8883|AAAAAAAADLCCAAAA|790|Mill West|Ln|Suite P|Barnes|Alcona County|MI|43788|United States|-5|condo| +8884|AAAAAAAAELCCAAAA|223|Ninth |Drive|Suite T|Sunnyside|Rutland County|VT|02552|United States|-5|apartment| +8885|AAAAAAAAFLCCAAAA|37|Second |Pkwy|Suite 100|Arlington|Montgomery County|MO|66557|United States|-6|apartment| +8886|AAAAAAAAGLCCAAAA|267|Main |Avenue|Suite 50|Woodland|Mercer County|MO|64854|United States|-6|single family| +8887|AAAAAAAAHLCCAAAA|173|Williams |Court|Suite 100|Riverside|Bollinger County|MO|69231|United States|-6|condo| +8888|AAAAAAAAILCCAAAA|631|Church 5th|RD|Suite F|Mount Pleasant|Monterey County|CA|91933|United States|-8|single family| +8889|AAAAAAAAJLCCAAAA|404|5th |Cir.|Suite 140|Union|Summit County|UT|88721|United States|-7|condo| +8890|AAAAAAAAKLCCAAAA|956|Broadway Lee|Blvd|Suite 450|Shiloh|Livingston County|MI|49275|United States|-5|apartment| +8891|AAAAAAAALLCCAAAA|302|Oak Central|Pkwy|Suite A|Hamilton|Guadalupe County|NM|82808|United States|-7|single family| +8892|AAAAAAAAMLCCAAAA|619|Chestnut |Dr.|Suite O|Centerville|Walla Walla County|WA|90059|United States|-8|apartment| +8893|AAAAAAAANLCCAAAA|835|Jackson |Pkwy|Suite N|Hamilton|Cotton County|OK|72808|United States|-6|apartment| +8894|AAAAAAAAOLCCAAAA|678|Lee Hillcrest|Blvd|Suite 110|Marion|Smith County|TX|70399|United States|-6|apartment| +8895|AAAAAAAAPLCCAAAA|699|2nd |Dr.|Suite X|Empire|Dutchess County|NY|14145|United States|-5|apartment| +8896|AAAAAAAAAMCCAAAA|574|Lakeview |Street|Suite 0|Providence|Bibb County|GA|36614|United States|-5|single family| +8897|AAAAAAAABMCCAAAA|466|8th 2nd|Street|Suite R|Hamilton|Irion County|TX|72808|United States|-6|single family| +8898|AAAAAAAACMCCAAAA|12|Ninth |Court|Suite 270|Brownsville|Woodward County|OK|79310|United States|-6|condo| +8899|AAAAAAAADMCCAAAA|90|Lake |Cir.|Suite 240|Shiloh|Decatur County|IA|59275|United States|-6|apartment| +8900|AAAAAAAAEMCCAAAA|899|Wilson |Wy|Suite X|Bunker Hill|Bureau County|IL|60150|United States|-6|apartment| +8901|AAAAAAAAFMCCAAAA|459|Park Jefferson|Circle|Suite 180|Red Hill|San Saba County|TX|74338|United States|-6|condo| +8902|AAAAAAAAGMCCAAAA|463|Lake |Blvd|Suite G|Springfield|Brown County|IL|69303|United States|-6|condo| +8903|AAAAAAAAHMCCAAAA|687|Lee Oak|Boulevard|Suite 140|Jackson|Lawrence County|TN|39583|United States|-6|single family| +8904|AAAAAAAAIMCCAAAA|169|Jefferson |Ave|Suite S|Hillcrest|Manatee County|FL|33003|United States|-5|apartment| +8905|AAAAAAAAJMCCAAAA|682|River |Parkway|Suite 450|Liberty|Breathitt County|KY|43451|United States|-6|single family| +8906|AAAAAAAAKMCCAAAA|996|10th |Drive|Suite 0|Saint Clair|McCook County|SD|55294|United States|-7|apartment| +8907|AAAAAAAALMCCAAAA|825|Davis |Way|Suite 340|Valley View|Schuyler County|NY|15124|United States|-5|single family| +8908|AAAAAAAAMMCCAAAA|43|7th |Boulevard|Suite P|Glenwood|Vernon Parish|LA|73511|United States|-6|single family| +8909|AAAAAAAANMCCAAAA|198|Center |Parkway|Suite M|Franklin|Morris County|NJ|09701|United States|-5|condo| +8910|AAAAAAAAOMCCAAAA|449|Lee Green|Circle|Suite Q|Newport|Chisago County|MN|51521|United States|-6|condo| +8911|AAAAAAAAPMCCAAAA|131|7th Park|Wy|Suite 140|Silver Springs|Caribou County|ID|84843|United States|-7|single family| +8912|AAAAAAAAANCCAAAA|949|West |Pkwy|Suite M|Pine Valley|Glascock County|GA|38209|United States|-5|condo| +8913|AAAAAAAABNCCAAAA|561|Lake Seventh|Circle|Suite 80|Wildwood|Coleman County|TX|76871|United States|-6|single family| +8914|AAAAAAAACNCCAAAA|737|Wilson |Road|Suite H|Oak Grove|Benton County|MS|58370|United States|-6|apartment| +8915|AAAAAAAADNCCAAAA|120|13th Willow|Drive|Suite 300|Greenfield|Campbell County|SD|55038|United States|-6|condo| +8916|AAAAAAAAENCCAAAA|385|Lake Maple|Parkway|Suite 70|Salem|Boone County|NE|68048|United States|-6|single family| +8917|AAAAAAAAFNCCAAAA|880|View |Ave|Suite V||Camden County|||United States|-5|single family| +8918|AAAAAAAAGNCCAAAA|710|View Dogwood|Parkway|Suite X|Walnut Grove|Mitchell County|IA|57752|United States|-6|single family| +8919|AAAAAAAAHNCCAAAA|341|East 9th|Road|Suite 350|Shiloh|Iron County|MI|49275|United States|-5|single family| +8920|AAAAAAAAINCCAAAA|705|Chestnut |Wy|Suite 40|Yorktown|Colleton County|SC|20732|United States|-5|single family| +8921|AAAAAAAAJNCCAAAA|71|4th View|Way|Suite 350|Glendale|Faulk County|SD|53951|United States|-6|condo| +8922|AAAAAAAAKNCCAAAA|469|Smith |Pkwy|Suite I|Cedar Grove|Washington County|PA|10411|United States|-5|single family| +8923|AAAAAAAALNCCAAAA|750|Railroad Seventh|Way|Suite R|Highland Park|Lewis County|ID|86534|United States|-7|condo| +8924|AAAAAAAAMNCCAAAA|189|1st South|Wy|Suite V|Summit|Douglas County|SD|50499|United States|-6|condo| +8925|AAAAAAAANNCCAAAA|232|Lake South|Pkwy|Suite 0|Fairview|Stark County|IL|65709|United States|-6|condo| +8926|AAAAAAAAONCCAAAA|738|Madison Ninth|Circle|Suite C|Highland|Coryell County|TX|79454|United States|-6|condo| +8927|AAAAAAAAPNCCAAAA|807|East Spring|Avenue|Suite E|Clinton|Webb County|TX|78222|United States|-6|apartment| +8928|AAAAAAAAAOCCAAAA|72|Maple Hill|Boulevard|Suite A|Glendale|DeWitt County|TX|73951|United States|-6|apartment| +8929|AAAAAAAABOCCAAAA|616|Lakeview Forest|Circle|Suite H|Red Hill|Manatee County|FL|34338|United States|-5|apartment| +8930|AAAAAAAACOCCAAAA|167|Second 6th|Boulevard|Suite I|Forest Hills|Mineral County|CO|89237|United States|-7|apartment| +8931|AAAAAAAADOCCAAAA|799|Laurel Spruce|Ct.|Suite 370|Crossroads|Goochland County|VA|20534|United States|-5|single family| +8932|AAAAAAAAEOCCAAAA|534|Ridge Sixth|Road|Suite 340|Lakeview|Phillips County|MT|68579|United States|-7|apartment| +8933|AAAAAAAAFOCCAAAA|689|5th |Lane|Suite A|Hopewell|Lake County|CO|80587|United States|-7|apartment| +8934|AAAAAAAAGOCCAAAA|286|Sunset Laurel|Avenue|Suite 20|Greenwood|Clearfield County|PA|18828|United States|-5|single family| +8935|AAAAAAAAHOCCAAAA|240|Elm |Pkwy|Suite X|Union|Jones County|NC|28721|United States|-5|condo| +8936|AAAAAAAAIOCCAAAA|16|2nd |Lane|Suite D|Florence|Coffey County|KS|63394|United States|-6|condo| +8937|AAAAAAAAJOCCAAAA|903|Forest 7th|Boulevard|Suite 210|Sulphur Springs|Waupaca County|WI|58354|United States|-6|single family| +8938|AAAAAAAAKOCCAAAA|215||Cir.|Suite 150|||SC||||| +8939|AAAAAAAALOCCAAAA|782|Johnson River|Pkwy|Suite 380|Macedonia|Jefferson County|OR|91087|United States|-8|apartment| +8940|AAAAAAAAMOCCAAAA|268|Williams |Ct.|Suite E|Florence|Assumption Parish|LA|73394|United States|-6|single family| +8941|AAAAAAAANOCCAAAA|83|Washington Second|RD|Suite B|Deerfield|Martin County|KY|49840|United States|-5|condo| +8942|AAAAAAAAOOCCAAAA|38|Green First|Wy|Suite 130|Newport|Zavala County|TX|71521|United States|-6|condo| +8943|AAAAAAAAPOCCAAAA|836|Birch |Court|Suite 10|Florence|Lincoln County|AR|73394|United States|-6|single family| +8944|AAAAAAAAAPCCAAAA|698|Forest East|Parkway|Suite W|Pleasant Hill|Hand County|SD|53604|United States|-7|apartment| +8945|AAAAAAAABPCCAAAA|598|Lincoln Spring|Ave|Suite P|Leland|Middlesex County|MA|09452|United States|-5|apartment| +8946|AAAAAAAACPCCAAAA|474|Park Walnut|Cir.|Suite 440|Woodlawn|Finney County|KS|64098|United States|-6|single family| +8947|AAAAAAAADPCCAAAA|751|Sunset |Boulevard|Suite 260|Highland|Anne Arundel County|MD|29454|United States|-5|single family| +8948|AAAAAAAAEPCCAAAA|253|Sunset Mill|Ln|Suite G|Woodville|Polk County|OR|94289|United States|-8|condo| +8949|AAAAAAAAFPCCAAAA|856|Cedar 3rd|Circle|Suite 370|Newtown|Coffey County|KS|61749|United States|-6|apartment| +8950|AAAAAAAAGPCCAAAA|140|Lincoln |Ct.|Suite 310|Buckhorn|Winneshiek County|IA|55458|United States|-6|condo| +8951|AAAAAAAAHPCCAAAA|805|7th |Pkwy|Suite 30|Cherry Valley|Payette County|ID|80854|United States|-7|condo| +8952|AAAAAAAAIPCCAAAA|214|7th Ninth|Street|Suite 270|Glenwood|Randolph County|IL|63511|United States|-6|apartment| +8953|AAAAAAAAJPCCAAAA|518|Woodland |Blvd|Suite 320|Summit||TX||United States|-6|apartment| +8954|AAAAAAAAKPCCAAAA|88|Lakeview |Court|Suite V|Georgetown|Knox County|KY|47057|United States|-5|single family| +8955|AAAAAAAALPCCAAAA|454|River Wilson|RD|Suite L|Oak Ridge|Barry County|MO|68371|United States|-6|apartment| +8956|AAAAAAAAMPCCAAAA|411|6th Woodland|Street|Suite 230|Franklin|Champaign County|IL|69101|United States|-6|condo| +8957|AAAAAAAANPCCAAAA|409|1st |Ct.|Suite 10|Hillcrest|Beckham County|OK|73003|United States|-6|single family| +8958|AAAAAAAAOPCCAAAA|668|11th Lakeview|Wy|Suite 210|Greenfield|Plymouth County|MA|05638|United States|-5|condo| +8959|AAAAAAAAPPCCAAAA|513|Railroad |Dr.|Suite Q|Ferguson|Chester County|PA|11821|United States|-5|condo| +8960|AAAAAAAAAADCAAAA|976|College Lincoln|Ln|Suite 330|Chelsea|Bremer County|IA|55888|United States|-6|single family| +8961|AAAAAAAABADCAAAA|484|Third |Drive|Suite Q|Lebanon|Hardy County|WV|22898|United States|-5|single family| +8962|AAAAAAAACADCAAAA|907|Washington |Wy|Suite X|Bunker Hill|Etowah County|AL|30150|United States|-6|single family| +8963|AAAAAAAADADCAAAA|463|8th Center|Cir.|Suite M|Gravel Hill|Bowie County|TX|71944|United States|-6|apartment| +8964|AAAAAAAAEADCAAAA|264|Mill |Road|Suite 220|Jackson|Woodward County|OK|79583|United States|-6|condo| +8965|AAAAAAAAFADCAAAA|717|Wilson |Parkway|Suite 410|Newport|Montgomery County|PA|11521|United States|-5|single family| +8966|AAAAAAAAGADCAAAA|123|Wilson Birch|Boulevard|Suite G|Wildwood|Idaho County|ID|86871|United States|-7|single family| +8967|AAAAAAAAHADCAAAA|740|Center First|Wy|Suite 200|Clinton|Anchorage Borough|AK|98222|United States|-9|single family| +8968|AAAAAAAAIADCAAAA|432|View |Pkwy|Suite W|Five Forks|Perry County|TN|32293|United States|-6|apartment| +8969|AAAAAAAAJADCAAAA|181|2nd Willow|Ave|Suite M|Springhill|Lincoln County|WI|54602|United States|-6|apartment| +8970|AAAAAAAAKADCAAAA|103|Jackson Fourth|Ave|Suite F|Brownsville|Comal County|TX|79310|United States|-6|apartment| +8971|AAAAAAAALADCAAAA|547|3rd |Boulevard|Suite 290|Sunnyside|Union County|MS|51952|United States|-6|single family| +8972|AAAAAAAAMADCAAAA|487|5th |ST|Suite 160|Harmony|Vermilion Parish|LA|75804|United States|-6|single family| +8973|AAAAAAAANADCAAAA|257|View |Cir.|Suite L|Enterprise|Daviess County|KY|41757|United States|-6|single family| +8974|AAAAAAAAOADCAAAA|283|5th |ST|Suite 190|Pleasant Valley|Emmet County|MI|42477|United States|-5|apartment| +8975|AAAAAAAAPADCAAAA|274|1st |RD|Suite 380|Greenville|Lincoln County|AR|71387|United States|-6|single family| +8976|AAAAAAAAABDCAAAA|441|Meadow Pine|Ct.|Suite 190|Greenfield|McCormick County|SC|25038|United States|-5|apartment| +8977|AAAAAAAABBDCAAAA|590|4th |Street|Suite 480|Buena Vista|Moore County|NC|25752|United States|-5|condo| +8978|AAAAAAAACBDCAAAA|391|Chestnut |Court|Suite D|Edgewood|Monroe County|GA|30069|United States|-5|single family| +8979|AAAAAAAADBDCAAAA|446|4th 6th|Way|Suite W|Bethel|Bedford County|PA|15281|United States|-5|condo| +8980|AAAAAAAAEBDCAAAA|130|Lincoln |Avenue|Suite O|Marion|Steele County|MN|50399|United States|-6|apartment| +8981|AAAAAAAAFBDCAAAA|627|Center Wilson|Court|Suite 90|Spring Valley|Summit County|OH|46060|United States|-5|single family| +8982|AAAAAAAAGBDCAAAA|574|First |||Lakewood|Osborne County|KS||United States||| +8983|AAAAAAAAHBDCAAAA|519|Spruce Locust|Road|Suite S|Franklin|Concordia Parish|LA|79101|United States|-6|apartment| +8984|AAAAAAAAIBDCAAAA|133|Lake |Pkwy|Suite Y|Liberty|Woodford County|KY|43451|United States|-6|single family| +8985|AAAAAAAAJBDCAAAA|677|Third |Parkway|Suite 150|Fairview|Ashe County|NC|25709|United States|-5|condo| +8986|AAAAAAAAKBDCAAAA|141|Spring Valley|Ct.|Suite 490|Good Hope|Daviess County|IN|42711|United States|-5|apartment| +8987|AAAAAAAALBDCAAAA|995|Pine Main|Avenue|Suite 60|Fayetteville|Wabaunsee County|KS|61732|United States|-6|single family| +8988|AAAAAAAAMBDCAAAA|244|Franklin Broadway|Street|Suite J|Plainview|Putnam County|FL|33683|United States|-5|apartment| +8989|AAAAAAAANBDCAAAA|879|Poplar Locust|Avenue|Suite 310|Glendale|Gallia County|OH|43951|United States|-5|single family| +8990|AAAAAAAAOBDCAAAA|866|Madison 2nd|Street|Suite U|Springfield|Brown County|IN|49303|United States|-5|single family| +8991|AAAAAAAAPBDCAAAA|404|Jackson Sixth|Pkwy|Suite X|Pleasant Hill|Clackamas County|OR|93604|United States|-8|single family| +8992|AAAAAAAAACDCAAAA|29|Lake |Pkwy|Suite Y|Sunnyside|Vigo County|IN|41952|United States|-5|apartment| +8993|AAAAAAAABCDCAAAA|230|Lake |Blvd|Suite 480|Redland|Dundy County|NE|66343|United States|-6|single family| +8994|AAAAAAAACCDCAAAA|923|View 14th|Court|Suite 400|Bethel|Colonial Heights city|VA|25281|United States|-5|single family| +8995|AAAAAAAADCDCAAAA|104|4th |Cir.|Suite 360|Guthrie|Washington County|PA|11423|United States|-5|apartment| +8996|AAAAAAAAECDCAAAA|||Cir.|Suite 70|Antioch||TN||||| +8997|AAAAAAAAFCDCAAAA|175|Elm |Circle|Suite I|Bridgeport|Hand County|SD|55817|United States|-7|single family| +8998|AAAAAAAAGCDCAAAA|917|Elm |Wy|Suite B|Hamilton|Ashland County|OH|42808|United States|-5|condo| +8999|AAAAAAAAHCDCAAAA|163|5th |Cir.|Suite M|Glenwood|Hancock County|OH|43511|United States|-5|single family| +9000|AAAAAAAAICDCAAAA|393|Third Franklin|Road|Suite 120|Vienna|Richmond County|NY|15119|United States|-5|apartment| +9001|AAAAAAAAJCDCAAAA|169|2nd |Drive|Suite E|Unionville|Northampton County|NC|21711|United States|-5|condo| +9002|AAAAAAAAKCDCAAAA|268|River Fifth|ST|Suite 390|Stringtown|Madera County|CA|90162|United States|-8|condo| +9003|AAAAAAAALCDCAAAA|544|Main Birch|Ln|Suite 420|Frenchtown|Taliaferro County|GA|32629|United States|-5|single family| +9004|AAAAAAAAMCDCAAAA|310|Lincoln |Boulevard|Suite 480|Cedar Grove|Dakota County|NE|60411|United States|-6|apartment| +9005|AAAAAAAANCDCAAAA|957|Church |Avenue|Suite 340|Forest Hills|Stonewall County|TX|79237|United States|-6|apartment| +9006|AAAAAAAAOCDCAAAA|79|Elm |Ln|Suite N|Shiloh|Stephens County|GA|39275|United States|-5|apartment| +9007|AAAAAAAAPCDCAAAA|97||||Wildwood|||46871|||| +9008|AAAAAAAAADDCAAAA|632|Willow Adams|Avenue|Suite W|Belmont|Skamania County|WA|90191|United States|-8|condo| +9009|AAAAAAAABDDCAAAA|428|Maple |Circle|Suite U|Oakland|Richmond city|VA|29843|United States|-5|single family| +9010|AAAAAAAACDDCAAAA|934|Laurel Maple|Street|Suite 30|Louisville|Fulton County|GA|34464|United States|-5|single family| +9011|AAAAAAAADDDCAAAA|459|Park Hillcrest|Boulevard|Suite 50|Florence|Fulton County|NY|13394|United States|-5|apartment| +9012|AAAAAAAAEDDCAAAA|||Ln|Suite 190|Midway||KY|41904|||| +9013|AAAAAAAAFDDCAAAA|729|Hickory |Blvd|Suite K|Bridgeport|Lorain County|OH|45817|United States|-5|apartment| +9014|AAAAAAAAGDDCAAAA|568|Sycamore Dogwood|Pkwy|Suite U|Oak Hill|Morrill County|NE|67838|United States|-7|apartment| +9015|AAAAAAAAHDDCAAAA|44|4th Spruce|Dr.|Suite 490|Mountain View|Lake and Peninsula Borough|AK|94466|United States|-9|condo| +9016|AAAAAAAAIDDCAAAA|356||Parkway|||||56668|||| +9017|AAAAAAAAJDDCAAAA|900|15th Railroad|Dr.|Suite 220|Harvey|Halifax County|NC|25858|United States|-5|apartment| +9018|AAAAAAAAKDDCAAAA|647|Chestnut Chestnut|Road|Suite T|Woodlawn|Webster County|NE|64098|United States|-7|apartment| +9019|AAAAAAAALDDCAAAA|303|12th Ridge|Cir.|Suite V|Highland Park|Calhoun County|GA|36534|United States|-5|single family| +9020|AAAAAAAAMDDCAAAA|57|Laurel View|Circle|Suite U|Mount Pleasant|Harnett County|NC|21933|United States|-5|single family| +9021|AAAAAAAANDDCAAAA|652|Jackson Birch|RD|Suite U|Antioch|Bradley County|AR|78605|United States|-6|apartment| +9022|AAAAAAAAODDCAAAA|567|Broadway 6th|Cir.|Suite R|Fisher|King George County|VA|22819|United States|-5|single family| +9023|AAAAAAAAPDDCAAAA|174|Lakeview Fifth|Lane|Suite 130|Red Hill|Sangamon County|IL|64338|United States|-6|apartment| +9024|AAAAAAAAAEDCAAAA|679|Cedar Cherry|Street|Suite X|Greenville|Middlesex County|MA|01987|United States|-5|apartment| +9025|AAAAAAAABEDCAAAA|615|3rd Oak|Circle|Suite 240|Red Hill|Navajo County|AZ|84338|United States|-7|apartment| +9026|AAAAAAAACEDCAAAA|884|Spring Madison|Cir.|Suite V|Glendale|Shiawassee County|MI|43951|United States|-5|apartment| +9027|AAAAAAAADEDCAAAA|971|Walnut Adams|Way|Suite 70|Oak Hill|Grant County|OK|77838|United States|-6|single family| +9028|AAAAAAAAEEDCAAAA|891|West |Way|Suite 220|Woodville|Clinton County|PA|14289|United States|-5|condo| +9029|AAAAAAAAFEDCAAAA|491|4th |Pkwy|Suite 300|Georgetown|San Augustine County|TX|77057|United States|-6|condo| +9030|AAAAAAAAGEDCAAAA||9th Dogwood||Suite P|||KY|42898||-6|| +9031|AAAAAAAAHEDCAAAA|879|6th Chestnut|Ct.|Suite 310|Walnut Grove|East Feliciana Parish|LA|77752|United States|-6|condo| +9032|AAAAAAAAIEDCAAAA|420|Broadway |Ct.|Suite L|Nottingham|Milam County|TX|74074|United States|-6|single family| +9033|AAAAAAAAJEDCAAAA|856|Pine |Circle|Suite W|Florence|Cape May County|NJ|03994|United States|-5|apartment| +9034|AAAAAAAAKEDCAAAA|480|View Highland|Court|Suite E|Brownsville|Nez Perce County|ID|89310|United States|-7|apartment| +9035|AAAAAAAALEDCAAAA|851|6th |||Crossroads||ID|80534|||| +9036|AAAAAAAAMEDCAAAA|939|Cedar Woodland|Parkway|Suite L|Macedonia|Ohio County|WV|21087|United States|-5|apartment| +9037|AAAAAAAANEDCAAAA|705|Spring |Ave|Suite D|Bath|Deuel County|NE|60573|United States|-6|condo| +9038|AAAAAAAAOEDCAAAA|425|Park |RD|Suite F|Buena Vista|Randolph County|GA|35752|United States|-5|apartment| +9039|AAAAAAAAPEDCAAAA|873|Woodland |Pkwy|Suite M|Maple Grove|Henry County|TN|38252|United States|-5|single family| +9040|AAAAAAAAAFDCAAAA|661|Cedar Church|Road|Suite X|Woodland|Jackson County|AL|34854|United States|-6|apartment| +9041|AAAAAAAABFDCAAAA|654|Main |Blvd|Suite O|New Hope|Chester County|SC|29431|United States|-5|apartment| +9042|AAAAAAAACFDCAAAA|301|Spring |Dr.|Suite 220|Sunnyside|Taylor County|TX|71952|United States|-6|condo| +9043|AAAAAAAADFDCAAAA|370|Birch Locust|Street|Suite 290|Franklin|Hopkins County|TX|79101|United States|-6|condo| +9044|AAAAAAAAEFDCAAAA|685|15th Johnson|Road|Suite 440|Shiloh|Gillespie County|TX|79275|United States|-6|condo| +9045|AAAAAAAAFFDCAAAA|83|Cherry Lincoln|Pkwy|Suite E|Waterloo|Anderson County|KS|61675|United States|-6|single family| +9046|AAAAAAAAGFDCAAAA|55|Elm |Drive|Suite 0|Glendale|Platte County|NE|63951|United States|-7|apartment| +9047|AAAAAAAAHFDCAAAA|514|Williams |Pkwy|Suite 480|Jackson|Haywood County|NC|29583|United States|-5|single family| +9048|AAAAAAAAIFDCAAAA|97|Chestnut Central|Ave|Suite 420|Friendship|Windham County|CT|05136|United States|-5|condo| +9049|AAAAAAAAJFDCAAAA|306|Hill |Way|Suite Q|Antioch|Jefferson County|IA|58605|United States|-6|single family| +9050|AAAAAAAAKFDCAAAA|22|4th |Parkway|Suite 310|Oneida|Essex County|MA|04627|United States|-5|condo| +9051|AAAAAAAALFDCAAAA|882|Elm Pine|Lane|Suite 300|Plainview|Hampton city|VA|23683|United States|-5|apartment| +9052|AAAAAAAAMFDCAAAA|941|2nd |Dr.|Suite Y|Spring Hill|Fremont County|CO|86787|United States|-7|condo| +9053|AAAAAAAANFDCAAAA|381|Highland |Court|Suite V|New Hope|Lamar County|GA|39431|United States|-5|condo| +9054|AAAAAAAAOFDCAAAA|255|5th |Ave|Suite 140|Clifton|Anne Arundel County|MD|28014|United States|-5|condo| +9055|AAAAAAAAPFDCAAAA|368|Willow |Pkwy|Suite N|Bath|Boone County|NE|60573|United States|-6|apartment| +9056|AAAAAAAAAGDCAAAA|235|Lincoln |Avenue|Suite 20|Mount Olive|Orange County|VA|28059|United States|-5|single family| +9057|AAAAAAAABGDCAAAA|190|Dogwood Elm|Way|Suite Y|Oakdale|Tangipahoa Parish|LA|79584|United States|-6|condo| +9058|AAAAAAAACGDCAAAA|916|5th |Wy|Suite C|Bunker Hill|Louisa County|IA|50150|United States|-6|single family| +9059|AAAAAAAADGDCAAAA|36|Third |Parkway|Suite 430|Summit|Bedford County|VA|20499|United States|-5|single family| +9060|AAAAAAAAEGDCAAAA|114|Cedar Fifth|Ct.|Suite 130|Macedonia|Smith County|KS|61087|United States|-6|condo| +9061|AAAAAAAAFGDCAAAA|790|Lakeview |ST|Suite E|Valley View|Lunenburg County|VA|25124|United States|-5|single family| +9062|AAAAAAAAGGDCAAAA|85|Hill Church|ST|Suite 420|Parkwood|Calhoun County|GA|31669|United States|-5|apartment| +9063|AAAAAAAAHGDCAAAA|75|Cherry |Blvd|Suite I|Valley View|Saline County|KS|65124|United States|-6|single family| +9064|AAAAAAAAIGDCAAAA|428|College |Wy|Suite 300|Highland Park|Emmet County|IA|56534|United States|-6|apartment| +9065|AAAAAAAAJGDCAAAA|123|Laurel |Way|Suite 40|Sunnyside|Winona County|MN|51952|United States|-6|apartment| +9066|AAAAAAAAKGDCAAAA|166|8th |Road|Suite 470|Edgewood|Whitley County|KY|40069|United States|-5|single family| +9067|AAAAAAAALGDCAAAA|23|Center |Ct.|Suite 250|Glenwood|Cottle County|TX|73511|United States|-6|apartment| +9068|AAAAAAAAMGDCAAAA|276|5th |Parkway|Suite 130|Newport|Pembina County|ND|51521|United States|-6|condo| +9069|AAAAAAAANGDCAAAA|912|Green |Wy|Suite 230|Crossroads|Box Elder County|UT|80534|United States|-7|single family| +9070|AAAAAAAAOGDCAAAA|110|6th |Avenue|Suite H|Oakdale|Shelby County|IN|49584|United States|-5|condo| +9071|AAAAAAAAPGDCAAAA|745|7th Washington|Drive|Suite B|Harmony|Contra Costa County|CA|95804|United States|-8|condo| +9072|AAAAAAAAAHDCAAAA|508|Willow Poplar|Parkway|Suite 410|Crossroads|Gaines County|TX|70534|United States|-6|apartment| +9073|AAAAAAAABHDCAAAA||Madison ||Suite 410|Spring Hill||OR|96787|United States||apartment| +9074|AAAAAAAACHDCAAAA|536|Highland Sixth|Ln|Suite 330|Spring Hill|Camden County|MO|66787|United States|-6|apartment| +9075|AAAAAAAADHDCAAAA|317|Fifth Smith|Dr.|Suite 220|Harmony|Montgomery County|NY|15804|United States|-5|apartment| +9076|AAAAAAAAEHDCAAAA|727|Oak East|Cir.|Suite 240|Valley View|Wayne County|WV|25124|United States|-5|condo| +9077|AAAAAAAAFHDCAAAA|525|Cherry Woodland|Wy|Suite 320|Five Points|Danville city|VA|26098|United States|-5|apartment| +9078|AAAAAAAAGHDCAAAA|54|Park |Pkwy|Suite 120|Liberty|Scott County|MS|53451|United States|-6|apartment| +9079|AAAAAAAAHHDCAAAA|885|Green |Parkway|Suite V|Brownsville|Boone County|KY|49310|United States|-6|condo| +9080|AAAAAAAAIHDCAAAA|177|Broadway |Road|Suite Y|Union Hill|Jackson County|WV|27746|United States|-5|single family| +9081|AAAAAAAAJHDCAAAA|665|14th |Cir.|Suite 20|Lakewood|Cumberland County|IL|68877|United States|-6|single family| +9082|AAAAAAAAKHDCAAAA|409|Second 11th||Suite A|Mountain View|Cass County|MO||||| +9083|AAAAAAAALHDCAAAA|352|Fourth Cherry|Dr.|Suite R|Macedonia|Skamania County|WA|91087|United States|-8|single family| +9084|AAAAAAAAMHDCAAAA|318|Oak Central|Cir.|Suite 80|Oak Ridge|Pike County|AL|38371|United States|-6|condo| +9085|AAAAAAAANHDCAAAA|431|9th 2nd|Ct.|Suite R|Walnut Grove|Onslow County|NC|27752|United States|-5|single family| +9086|AAAAAAAAOHDCAAAA|145|Sunset |Wy|Suite 340|Centerville|Johnson County|NE|60059|United States|-7|apartment| +9087|AAAAAAAAPHDCAAAA|700|Pine East|ST|Suite 220|Providence|Colfax County|NM|86614|United States|-7|apartment| +9088|AAAAAAAAAIDCAAAA|495|Woodland North|Avenue|Suite S|Woodlawn|Pickens County|SC|24098|United States|-5|condo| +9089|AAAAAAAABIDCAAAA|599|North Railroad|Circle|Suite 30|Woodville|Jones County|NC|24289|United States|-5|apartment| +9090|AAAAAAAACIDCAAAA|546|15th |Avenue|Suite N|Georgetown|Butte County|CA|97057|United States|-8|single family| +9091|AAAAAAAADIDCAAAA|369|Green 8th|Avenue|Suite U|Hopewell|Snyder County|PA|10587|United States|-5|condo| +9092|AAAAAAAAEIDCAAAA|146|Jefferson |ST|Suite E|Farmington|Vinton County|OH|49145|United States|-5|apartment| +9093|AAAAAAAAFIDCAAAA|389|Valley Railroad||Suite B|||MN||||| +9094|AAAAAAAAGIDCAAAA|840|9th Woodland|||Kingston|Hill County||74975||-6|single family| +9095|AAAAAAAAHIDCAAAA|||Ct.|||||59532||-6|single family| +9096|AAAAAAAAIIDCAAAA|107|Elm |Dr.|Suite 160|Union Hill|Tift County|GA|37746|United States|-5|condo| +9097|AAAAAAAAJIDCAAAA|784|Main Meadow|Cir.|Suite 110|Deerfield|Bradford County|PA|19840|United States|-5|condo| +9098|AAAAAAAAKIDCAAAA|944|2nd |Drive|Suite 490|Jackson|Woodford County|IL|69583|United States|-6|condo| +9099|AAAAAAAALIDCAAAA||Pine ||Suite 60||Stearns County|MN|58054|United States|-6|condo| +9100|AAAAAAAAMIDCAAAA|101|4th Franklin|Cir.|Suite 110|Crossroads|Johnson County|KY|40534|United States|-6|apartment| +9101|AAAAAAAANIDCAAAA|333|First 2nd|Pkwy|Suite O|Spring Hill|Warren County|IL|66787|United States|-6|apartment| +9102|AAAAAAAAOIDCAAAA|602|Birch |Ave|Suite 380|Highland|Glades County|FL|39454|United States|-5|single family| +9103|AAAAAAAAPIDCAAAA|661|North |Circle|Suite 450|Prosperity|Ford County|IL|69089|United States|-6|single family| +9104|AAAAAAAAAJDCAAAA||14th |Wy|||Trimble County|||United States|-5|| +9105|AAAAAAAABJDCAAAA|187|Poplar |Way|Suite I|Spring Hill|Union County|SC|26787|United States|-5|apartment| +9106|AAAAAAAACJDCAAAA|893|Church Williams|Drive|Suite 50|Carthage|Itawamba County|MS|51529|United States|-6|condo| +9107|AAAAAAAADJDCAAAA|641|Sunset |Lane|Suite 220|Leon|Santa Cruz County|AZ|80913|United States|-7|apartment| +9108|AAAAAAAAEJDCAAAA|797|7th Elm|Ct.|Suite I|Kingston|Sandoval County|NM|84975|United States|-7|single family| +9109|AAAAAAAAFJDCAAAA|965|Walnut |Avenue|Suite 290|Springfield|Kent County|MD|29303|United States|-5|single family| +9110|AAAAAAAAGJDCAAAA|595|Ninth 2nd|Ave|Suite N|Spring Valley|Marshall County|OK|76060|United States|-6|single family| +9111|AAAAAAAAHJDCAAAA|842|Park Main|Pkwy|Suite J|Greenfield|Hart County|KY|45038|United States|-6|apartment| +9112|AAAAAAAAIJDCAAAA|470|Lincoln |Circle|Suite N|Sulphur Springs|Kiowa County|KS|68354|United States|-6|apartment| +9113|AAAAAAAAJJDCAAAA|172|College |Parkway|Suite J|Arlington|DeKalb County|AL|36557|United States|-6|condo| +9114|AAAAAAAAKJDCAAAA|835|Woodland 1st|Blvd|Suite B|Five Points|Meigs County|OH|46098|United States|-5|condo| +9115|AAAAAAAALJDCAAAA|613|5th South|Blvd|Suite W|Marion|Grant County|SD|50399|United States|-6|single family| +9116|AAAAAAAAMJDCAAAA|374|Green Ninth|Parkway|Suite 250|Mapleton|Washita County|OK|79787|United States|-6|single family| +9117|AAAAAAAANJDCAAAA|989|Locust |Court|Suite 10|Shiloh|Limestone County|AL|39275|United States|-6|apartment| +9118|AAAAAAAAOJDCAAAA|51|College |Court|Suite C|Mount Pleasant|Hinsdale County|CO|81933|United States|-7|condo| +9119|AAAAAAAAPJDCAAAA|333|West Third|Parkway|Suite 50|Five Points|Benton County|AR|76098|United States|-6|single family| +9120|AAAAAAAAAKDCAAAA|352|Sycamore |Boulevard|Suite R|Florence|Roosevelt County|MT|63394|United States|-7|single family| +9121|AAAAAAAABKDCAAAA|773|East |ST|Suite 110|Carpenter|Walker County|TX|71147|United States|-6|apartment| +9122|AAAAAAAACKDCAAAA|595|Fourteenth Lakeview|Street|Suite E|Woodland|Elkhart County|IN|44854|United States|-5|condo| +9123|AAAAAAAADKDCAAAA||Walnut |||Salem||PA|18048|United States||single family| +9124|AAAAAAAAEKDCAAAA|482|Second River|Blvd||Greenfield||WI||United States||apartment| +9125|AAAAAAAAFKDCAAAA|473|Park Meadow|Blvd|Suite 160|Bunker Hill|Craighead County|AR|70150|United States|-6|condo| +9126|AAAAAAAAGKDCAAAA|896|Spruce |Road|Suite 250|Glendale|Rockingham County|NC|23951|United States|-5|condo| +9127|AAAAAAAAHKDCAAAA|938|3rd North|Drive|Suite 180|Oak Ridge|Johnston County|NC|28371|United States|-5|single family| +9128|AAAAAAAAIKDCAAAA|319|Central Ninth|Road|Suite 390|Springfield|Warren County|VA|29303|United States|-5|single family| +9129|AAAAAAAAJKDCAAAA|140|Washington |Cir.|Suite Q|Oak Ridge|Chippewa County|MI|48371|United States|-5|condo| +9130|AAAAAAAAKKDCAAAA|425|Laurel Eigth|Parkway|Suite T|Hamilton|Pocahontas County|IA|52808|United States|-6|single family| +9131|AAAAAAAALKDCAAAA|579|Madison |Cir.|Suite 200|Enterprise|Wasco County|OR|91757|United States|-8|apartment| +9132|AAAAAAAAMKDCAAAA|826|Seventh Maple|ST|Suite 130|Highland Park|Gillespie County|TX|76534|United States|-6|condo| +9133|AAAAAAAANKDCAAAA|547|9th |Way|Suite X|Shiloh|Union County|AR|79275|United States|-6|single family| +9134|AAAAAAAAOKDCAAAA|211|Broadway Sunset|Parkway|Suite Q|Mount Zion|Windham County|CT|08654|United States|-5|apartment| +9135|AAAAAAAAPKDCAAAA|15|Adams |Lane|Suite 30|Lakeview|Worcester County|MD|28579|United States|-5|single family| +9136|AAAAAAAAALDCAAAA|636|Hill ||||Lyon County||50573||-6|| +9137|AAAAAAAABLDCAAAA|978|Fourth Hickory|Dr.|Suite 220|Mountain View|Perkins County|SD|54466|United States|-7|apartment| +9138|AAAAAAAACLDCAAAA|735|Chestnut |Pkwy|Suite 200|Farmington|Carbon County|WY|89145|United States|-7|apartment| +9139|AAAAAAAADLDCAAAA|771|Chestnut Forest|Ave|Suite 470|Greenville|Marion County|OR|91387|United States|-8|condo| +9140|AAAAAAAAELDCAAAA|134|Davis |Lane|Suite 50|Bunker Hill|Perry County|PA|10150|United States|-5|condo| +9141|AAAAAAAAFLDCAAAA|911|Locust Railroad|Court|Suite X|San Jose|McKean County|PA|18003|United States|-5|apartment| +9142|AAAAAAAAGLDCAAAA|915|2nd |Way|Suite 440|Newtown|Humphreys County|MS|51749|United States|-6|single family| +9143|AAAAAAAAHLDCAAAA|93|Hill |Wy|Suite 10|Pleasant Grove|Jones County|MS|54136|United States|-6|single family| +9144|AAAAAAAAILDCAAAA|63|11th Birch|Pkwy|Suite D|Crossroads|Jones County|IA|50534|United States|-6|condo| +9145|AAAAAAAAJLDCAAAA|878|Fourth |Court|Suite C|Mount Olive|Pointe Coupee Parish|LA|78059|United States|-6|single family| +9146|AAAAAAAAKLDCAAAA|146|Miller |Blvd|Suite 250|Lone Oak|Live Oak County|TX|76893|United States|-6|single family| +9147|AAAAAAAALLDCAAAA|402|Madison Church|Boulevard|Suite 190|Oak Hill|Grant County|WI|57838|United States|-6|condo| +9148|AAAAAAAAMLDCAAAA|710|Broadway |Parkway|Suite 470|Andover|Albemarle County|VA|21639|United States|-5|single family| +9149|AAAAAAAANLDCAAAA|103|Willow |Boulevard|Suite 380|Florence|Leon County|FL|33394|United States|-5|apartment| +9150|AAAAAAAAOLDCAAAA|993|Forest |Pkwy|Suite 100|Springdale|Sanborn County|SD|58883|United States|-7|apartment| +9151|AAAAAAAAPLDCAAAA|143|Ash Fifth|Ct.|Suite 410|Red Hill|Oswego County|NY|14338|United States|-5|apartment| +9152|AAAAAAAAAMDCAAAA|188|First |Street|Suite H|Georgetown|Blue Earth County|MN|57057|United States|-6|single family| +9153|AAAAAAAABMDCAAAA|622|Pine |Street|Suite 370|Union|Suwannee County|FL|38721|United States|-5|apartment| +9154|AAAAAAAACMDCAAAA|691|5th |Dr.|Suite F|Oak Hill|Jack County|TX|77838|United States|-6|apartment| +9155|AAAAAAAADMDCAAAA|338|Lincoln |Boulevard|Suite 80|Arthur|Granite County|MT|65965|United States|-7|apartment| +9156|AAAAAAAAEMDCAAAA|12|Fourth Main|ST|Suite U|Union|LaMoure County|ND|58721|United States|-6|condo| +9157|AAAAAAAAFMDCAAAA|499|6th |Court|Suite 470|Crossroads|Whitley County|KY|40534|United States|-5|single family| +9158|AAAAAAAAGMDCAAAA|161|View |Boulevard|Suite W|Clinton|Hughes County|OK|78222|United States|-6|apartment| +9159|AAAAAAAAHMDCAAAA|196|4th Cherry|Circle|Suite X|Clinton|Taos County|NM|88222|United States|-7|condo| +9160|AAAAAAAAIMDCAAAA|465|Walnut |Ln|Suite T|Stringtown|Imperial County|CA|90162|United States|-8|single family| +9161|AAAAAAAAJMDCAAAA|914|5th |Drive|Suite W|Edgewood|Grafton County|NH|00669|United States|-5|condo| +9162|AAAAAAAAKMDCAAAA|19|South |Lane|Suite 50|Belmont|Granville County|NC|20191|United States|-5|single family| +9163|AAAAAAAALMDCAAAA|431|Jackson Park|Way|Suite 480|Five Points|Sullivan County|NY|16098|United States|-5|single family| +9164|AAAAAAAAMMDCAAAA|802|Mill |Drive|Suite N|Pine Grove|Clark County|WA|94593|United States|-8|apartment| +9165|AAAAAAAANMDCAAAA|224|Park |Dr.|Suite 400|Enterprise|Harper County|OK|71757|United States|-6|apartment| +9166|AAAAAAAAOMDCAAAA|208|4th Oak|ST|Suite J|Springfield|Cleveland County|OK|79303|United States|-6|single family| +9167|AAAAAAAAPMDCAAAA|521|Spring |Parkway|Suite 110|Greenville|Hardin County|KY|41387|United States|-6|apartment| +9168|AAAAAAAAANDCAAAA|380||Avenue|Suite 130||Oneida County|||||apartment| +9169|AAAAAAAABNDCAAAA|567|8th |Ln|Suite I|Centerville|Cherokee County|TX|70059|United States|-6|apartment| +9170|AAAAAAAACNDCAAAA|39|Hillcrest |ST|Suite 220|Macedonia|Sandoval County|NM|81087|United States|-7|apartment| +9171|AAAAAAAADNDCAAAA|391|Ninth Forest|Court|Suite X|Brownsville|Hopkins County|KY|49310|United States|-6|apartment| +9172|AAAAAAAAENDCAAAA|321|North |Dr.|Suite 230|Oakwood|Monroe County|KY|40169|United States|-5|single family| +9173|AAAAAAAAFNDCAAAA|298|Laurel |Cir.|Suite 40|New Hope|Union County|TN|39431|United States|-6|apartment| +9174|AAAAAAAAGNDCAAAA|514|Church |Avenue|Suite F|Oakland|Van Buren County|AR|79843|United States|-6|apartment| +9175|AAAAAAAAHNDCAAAA|471|Lincoln |Drive|Suite T|Salem|Clay County|MS|58048|United States|-6|single family| +9176|AAAAAAAAINDCAAAA|635|College Third|RD|Suite 360|Riverview|Palm Beach County|FL|39003|United States|-5|apartment| +9177|AAAAAAAAJNDCAAAA|61|View 6th|RD|Suite T|Harmony|Comanche County|TX|75804|United States|-6|single family| +9178|AAAAAAAAKNDCAAAA|838|15th Maple|Lane|Suite K|Oneida|Boone County|WV|24027|United States|-5|apartment| +9179|AAAAAAAALNDCAAAA|357|Sixth Ash|Wy|Suite V|Mountain View|Liberty County|FL|34466|United States|-5|condo| +9180|AAAAAAAAMNDCAAAA|317|Dogwood Maple|Dr.|Suite T|Cedar Grove|Wood County|TX|70411|United States|-6|apartment| +9181|AAAAAAAANNDCAAAA|503|North |Ave|Suite 420|Winslow|Crook County|WY|88525|United States|-7|apartment| +9182|AAAAAAAAONDCAAAA|826|9th |Ave|Suite N|Concord|Lee County|IL|64107|United States|-6|apartment| +9183|AAAAAAAAPNDCAAAA|213|3rd 1st|Blvd|Suite T|Glendale|Sacramento County|CA|93951|United States|-8|apartment| +9184|AAAAAAAAAODCAAAA|998|Hill Sycamore|Way|Suite N|New Hope|Suffolk city|VA|29431|United States|-5|single family| +9185|AAAAAAAABODCAAAA|917|Pine 10th|Road|Suite M|Newport|Cottle County|TX|71521|United States|-6|single family| +9186|AAAAAAAACODCAAAA||7th Spruce|ST|||Traill County|ND|52898|United States|-6|single family| +9187|AAAAAAAADODCAAAA|498|4th |Court|Suite A|Pine Grove|Liberty County|TX|74593|United States|-6|apartment| +9188|AAAAAAAAEODCAAAA|604|Main Smith|Avenue|Suite 60|Hopewell|Freeborn County|MN|50587|United States|-6|apartment| +9189|AAAAAAAAFODCAAAA|715|Park 3rd|Circle|Suite 170|Mount Pleasant|Glenn County|CA|91933|United States|-8|condo| +9190|AAAAAAAAGODCAAAA|921|7th |Pkwy|Suite 250|New Hope|Caribou County|ID|89431|United States|-7|single family| +9191|AAAAAAAAHODCAAAA|566|Pine |Road||Five Forks|Powell County||||-5|| +9192|AAAAAAAAIODCAAAA|912|Lee |Ln|Suite A|Woodland|Clark County|AR|74854|United States|-6|single family| +9193|AAAAAAAAJODCAAAA|665|View Cherry|Ln|Suite 460|Woodlawn|Marion County|IL|64098|United States|-6|condo| +9194|AAAAAAAAKODCAAAA|686|Oak Lee|ST|Suite 450|Maple Grove|Sedgwick County|CO|88252|United States|-7|condo| +9195|AAAAAAAALODCAAAA|219|13th Spring|Drive|Suite L|Lebanon|Parmer County|TX|72898|United States|-6|apartment| +9196|AAAAAAAAMODCAAAA|343|Railroad |Ln|Suite 220|Sulphur Springs|Guilford County|NC|28354|United States|-5|condo| +9197|AAAAAAAANODCAAAA|848|Adams |Parkway|Suite X|Newport|Linn County|IA|51521|United States|-6|apartment| +9198|AAAAAAAAOODCAAAA|673|Main Jackson|Road|Suite F|Union|Lincoln County|MT|68721|United States|-7|condo| +9199|AAAAAAAAPODCAAAA|530|Hillcrest 8th|Ln|Suite 230|Union|Lincoln County|MT|68721|United States|-7|apartment| +9200|AAAAAAAAAPDCAAAA|130|5th Meadow|Ln|Suite 100|Buena Vista|Lancaster County|PA|15752|United States|-5|single family| +9201|AAAAAAAABPDCAAAA|182|Forest Oak|Way|Suite 350|Union Hill|Cedar County|IA|57746|United States|-6|apartment| +9202|AAAAAAAACPDCAAAA|335|Franklin |Wy|Suite 440|Lebanon|Mineral County|CO|82898|United States|-7|apartment| +9203|AAAAAAAADPDCAAAA|1|Hillcrest Main|Drive|Suite 60|Sunnyside|Quitman County|MS|51952|United States|-6|apartment| +9204|AAAAAAAAEPDCAAAA|136|River ||Suite 280|Stafford||KY|44980|||single family| +9205|AAAAAAAAFPDCAAAA|566|Madison Oak|||||||United States|-5|single family| +9206|AAAAAAAAGPDCAAAA|565|Lake South|Cir.|Suite U|Shady Grove|Cheyenne County|KS|62812|United States|-6|condo| +9207|AAAAAAAAHPDCAAAA|649|8th Woodland|Circle|Suite 160|Silver Springs|Waseca County|MN|54843|United States|-6|single family| +9208|AAAAAAAAIPDCAAAA|120|Williams Second|Ln|Suite 300|Summerfield|Alexander County|NC|20634|United States|-5|condo| +9209|AAAAAAAAJPDCAAAA|92|1st 14th|Street|Suite 170|Brownsville|Anderson County|KY|49310|United States|-6|condo| +9210|AAAAAAAAKPDCAAAA|210|11th |Circle|Suite F|Avery|Carter County|TN|30194|United States|-5|condo| +9211|AAAAAAAALPDCAAAA|373|Walnut |Avenue|Suite 310|Waterloo|Dickinson County|KS|61675|United States|-6|apartment| +9212|AAAAAAAAMPDCAAAA|729|West Oak|Blvd|Suite 380|Oakdale|Tift County|GA|39584|United States|-5|apartment| +9213|AAAAAAAANPDCAAAA|105|Jefferson 7th|RD|Suite D|Georgetown|Wyoming County|NY|17057|United States|-5|condo| +9214|AAAAAAAAOPDCAAAA|266|Meadow |Circle|Suite O|Summit|Haskell County|TX|70499|United States|-6|condo| +9215|AAAAAAAAPPDCAAAA|891|3rd |Lane|Suite 100|Glenwood|Henry County|KY|43511|United States|-6|apartment| +9216|AAAAAAAAAAECAAAA||Center ||||||51904|United States||| +9217|AAAAAAAABAECAAAA|720|Jackson |RD|Suite 50|Plainview|Austin County|TX|73683|United States|-6|apartment| +9218|AAAAAAAACAECAAAA|988|Sycamore |Pkwy|Suite Y|Walnut Grove|Scott County|KY|47752|United States|-5|single family| +9219|AAAAAAAADAECAAAA|529|Walnut |ST|Suite D|Greenfield|Indian River County|FL|35038|United States|-5|condo| +9220|AAAAAAAAEAECAAAA|181|Jackson |Lane|||Morris County||01987|United States|-5|| +9221|AAAAAAAAFAECAAAA|952|12th |Wy|Suite 70||||77057|United States|-6|apartment| +9222|AAAAAAAAGAECAAAA|547|Hillcrest |Ln|Suite W|Pleasant Valley|Grady County|GA|32477|United States|-5|single family| +9223|AAAAAAAAHAECAAAA|557|Lake 6th|Drive|Suite 390|Hamilton|Venango County|PA|12808|United States|-5|single family| +9224|AAAAAAAAIAECAAAA|466|Hillcrest Fourth|Blvd|Suite 200|Pine Grove|Iron County|MI|44593|United States|-5|single family| +9225|AAAAAAAAJAECAAAA|946|Willow |Court|Suite 210|Wilson|Putnam County|OH|46971|United States|-5|apartment| +9226|AAAAAAAAKAECAAAA|703|Hillcrest |Parkway|Suite 170|Newport|Lyman County|SD|51521|United States|-7|single family| +9227|AAAAAAAALAECAAAA|278|Sunset 7th|Drive|Suite X|Plainview|Scott County|IL|63683|United States|-6|single family| +9228|AAAAAAAAMAECAAAA|915|Oak Hillcrest|Pkwy|Suite M|Tanglewood|Buchanan County|IA|58994|United States|-6|single family| +9229|AAAAAAAANAECAAAA|182|Ash Railroad|Way|Suite 0|Oakland|Buena Vista County|IA|59843|United States|-6|apartment| +9230|AAAAAAAAOAECAAAA|79|Forest Park|RD|Suite T|Springfield|Kitsap County|WA|99303|United States|-8|apartment| +9231|AAAAAAAAPAECAAAA|288|Birch South|Avenue|Suite U|Lincoln|Liberty County|TX|71289|United States|-6|condo| +9232|AAAAAAAAABECAAAA|492|14th |Road|Suite 70|Kirkland|Cooke County|TX|77896|United States|-6|condo| +9233|AAAAAAAABBECAAAA|70|Washington Church|Avenue|Suite O|Clinton|Hampden County|MA|08822|United States|-5|condo| +9234|AAAAAAAACBECAAAA|452|6th 7th|Parkway|Suite R|Five Forks|Cimarron County|OK|72293|United States|-6|single family| +9235|AAAAAAAADBECAAAA|715||||Enterprise|||41757|United States|-5|| +9236|AAAAAAAAEBECAAAA|952|12th Pine|Ct.|Suite K|Oakdale|Amador County|CA|99584|United States|-8|apartment| +9237|AAAAAAAAFBECAAAA|469|View Sixth|Street|Suite 250|Arlington|Durham County|NC|26557|United States|-5|condo| +9238|AAAAAAAAGBECAAAA|31|Washington Valley|Pkwy|Suite 230|Unionville|Stafford County|VA|21711|United States|-5|single family| +9239|AAAAAAAAHBECAAAA|318|View Main|Ln|Suite Q|Highland Park|San Juan County|WA|96534|United States|-8|apartment| +9240|AAAAAAAAIBECAAAA|585|Railroad |Ave|Suite 240|Williamsville|Bradford County|FL|38754|United States|-5|single family| +9241|AAAAAAAAJBECAAAA|924|2nd |Avenue|Suite N|Edgewood|Okmulgee County|OK|70069|United States|-6|single family| +9242|AAAAAAAAKBECAAAA|547|Woodland |Cir.|Suite G|Hamilton|Benson County|ND|52808|United States|-6|condo| +9243|AAAAAAAALBECAAAA|896|Walnut Meadow|Boulevard|Suite 380|Wildwood|Montezuma County|CO|86871|United States|-7|single family| +9244|AAAAAAAAMBECAAAA|946|Sunset Third|Street|Suite M|Plainview|Burnett County|WI|53683|United States|-6|single family| +9245|AAAAAAAANBECAAAA||||||||14338|United States||| +9246|AAAAAAAAOBECAAAA|617|Davis Center|Way|Suite 370|Oak Hill|Tioga County|NY|17838|United States|-5|condo| +9247|AAAAAAAAPBECAAAA|102|Lake |Boulevard|Suite J|Union Hill|San Miguel County|NM|87746|United States|-7|single family| +9248|AAAAAAAAACECAAAA|217|View |Avenue|Suite O|Georgetown|Webster County|KY|47057|United States|-5|single family| +9249|AAAAAAAABCECAAAA||Fifth |Drive|||La Salle Parish|LA||United States||| +9250|AAAAAAAACCECAAAA|336|4th Church|Avenue|Suite 70|White Oak|Oconee County|GA|36668|United States|-5|condo| +9251|AAAAAAAADCECAAAA|494|Oak |ST|Suite 450|Edgewood|Jones County|GA|30069|United States|-5|single family| +9252|AAAAAAAAECECAAAA|578|14th |Cir.|Suite B|Stringtown|Tattnall County|GA|30162|United States|-5|condo| +9253|AAAAAAAAFCECAAAA|9|Madison |Cir.|Suite F|Marion|Kershaw County|SC|20399|United States|-5|single family| +9254|AAAAAAAAGCECAAAA|159|Center 5th|Court|Suite U|Friendship|Meigs County|TN|34536|United States|-6|apartment| +9255|AAAAAAAAHCECAAAA|14|Johnson Green|Pkwy|Suite D|Enterprise|Webster County|MO|61757|United States|-6|single family| +9256|AAAAAAAAICECAAAA|462|Jefferson |Way|Suite 150|Barnes|Henderson County|KY|43788|United States|-6|apartment| +9257|AAAAAAAAJCECAAAA|392|Johnson |Lane|Suite 60|Antioch|Gallia County|OH|48605|United States|-5|condo| +9258|AAAAAAAAKCECAAAA|567|Spring |Court|Suite W|Midway|Grant County|OR|91904|United States|-8|condo| +9259|AAAAAAAALCECAAAA|27|Ninth Elm|Street|Suite H|Five Points|Wheeler County|OR|96098|United States|-8|single family| +9260|AAAAAAAAMCECAAAA|137|North Park|Avenue|Suite 390|Highland Park|Cibola County|NM|86534|United States|-7|condo| +9261|AAAAAAAANCECAAAA|499|4th Washington|Ave|Suite N|Bridgeport|Washburn County|WI|55817|United States|-6|condo| +9262|AAAAAAAAOCECAAAA|200|Willow |Circle|Suite 370|Macedonia|Iron County|UT|81087|United States|-7|single family| +9263|AAAAAAAAPCECAAAA|||Way||Farmington|Lea County||||-7|single family| +9264|AAAAAAAAADECAAAA|852|15th Oak|Lane|Suite 240|Pleasant Valley|Lamar County|GA|32477|United States|-5|apartment| +9265|AAAAAAAABDECAAAA|231|Park |Road|Suite 140|Summit|Pawnee County|KS|60499|United States|-6|single family| +9266|AAAAAAAACDECAAAA|52|5th |Lane|Suite 400|Summit|Middlesex County|NJ|01099|United States|-5|single family| +9267|AAAAAAAADDECAAAA|565|Maple Central|Road|Suite 440|Woodlawn|Clay County|SD|54098|United States|-6|condo| +9268|AAAAAAAAEDECAAAA|675|Second Church|ST|Suite 340|Fairview|Petroleum County|MT|65709|United States|-7|condo| +9269|AAAAAAAAFDECAAAA|249|West |Road|Suite O|Hopewell|Palo Alto County|IA|50587|United States|-6|apartment| +9270|AAAAAAAAGDECAAAA|393|5th |Cir.|Suite Y|Kingston|Suffolk County|NY|14975|United States|-5|apartment| +9271|AAAAAAAAHDECAAAA|958|Church Highland|Dr.|Suite 460|Marion|Dutchess County|NY|10399|United States|-5|single family| +9272|AAAAAAAAIDECAAAA|385|Eigth |Blvd|Suite 100|Oak Ridge|McCormick County|SC|28371|United States|-5|single family| +9273|AAAAAAAAJDECAAAA|643|14th 1st|Street|Suite 340|Sunnyside|Ionia County|MI|41952|United States|-5|apartment| +9274|AAAAAAAAKDECAAAA|285|8th Maple|Dr.|Suite D|Stringtown|Doniphan County|KS|60162|United States|-6|single family| +9275|AAAAAAAALDECAAAA|257|1st |Dr.|Suite Y|Union Hill|Washtenaw County|MI|47746|United States|-5|apartment| +9276|AAAAAAAAMDECAAAA|271|Center |Drive|Suite 440|Newtown|Cibola County|NM|81749|United States|-7|apartment| +9277|AAAAAAAANDECAAAA|410|7th Center|Road|Suite S|New Hope|Wood County|WV|29431|United States|-5|condo| +9278|AAAAAAAAODECAAAA|787|Maple |Cir.|Suite 140|Shady Grove|Clay County|NC|22812|United States|-5|apartment| +9279|AAAAAAAAPDECAAAA|157|South 4th|Parkway|Suite 380|Thompsonville|Saline County|NE|69651|United States|-7|condo| +9280|AAAAAAAAAEECAAAA|529|Laurel Railroad|Lane|Suite T|Morris|Clackamas County|OR|96696|United States|-8|condo| +9281|AAAAAAAABEECAAAA|148|Smith |Road|Suite 330|Fox|DeKalb County|GA|30631|United States|-5|condo| +9282|AAAAAAAACEECAAAA|394|Laurel |Ct.|Suite E|Blue Springs|Rockingham County|VA|24686|United States|-5|single family| +9283|AAAAAAAADEECAAAA|554|Smith Main|RD|Suite C|Georgetown|McCracken County|KY|47057|United States|-5|condo| +9284|AAAAAAAAEEECAAAA|526|Jefferson |Wy|Suite I|Springfield|Pitt County|NC|29303|United States|-5|apartment| +9285|AAAAAAAAFEECAAAA|124|Main |Boulevard|Suite O|Oakland|Tangipahoa Parish|LA|79843|United States|-6|single family| +9286|AAAAAAAAGEECAAAA|305|11th Ash|Wy|Suite K|Empire|Clark County|IL|64145|United States|-6|apartment| +9287|AAAAAAAAHEECAAAA|393|Miller Lincoln|Drive|Suite Y|Buena Vista|Oceana County|MI|45752|United States|-5|apartment| +9288|AAAAAAAAIEECAAAA|32|7th |Circle|Suite I|Bethel|Douglas County|KS|65281|United States|-6|condo| +9289|AAAAAAAAJEECAAAA|583|4th Railroad|Street|Suite N|Five Forks|Southampton County|VA|22293|United States|-5|apartment| +9290|AAAAAAAAKEECAAAA|5|Second Madison|Ln|Suite V|Woodland|Trumbull County|OH|44854|United States|-5|single family| +9291|AAAAAAAALEECAAAA|199|Sixth |Dr.|Suite S|Cedar Grove|Warren County|PA|10411|United States|-5|condo| +9292|AAAAAAAAMEECAAAA|41|East |Pkwy|Suite X|Lakeview|Ontonagon County|MI|48579|United States|-5|single family| +9293|AAAAAAAANEECAAAA|894|Main 9th|Blvd|Suite 140|Friendship|Brooke County|WV|24536|United States|-5|condo| +9294|AAAAAAAAOEECAAAA|176|Cedar |Ln|Suite B|Oak Ridge|Accomack County|VA|28371|United States|-5|single family| +9295|AAAAAAAAPEECAAAA|861|Franklin Park|Boulevard|Suite 310|Highland|Macon County|IL|69454|United States|-6|condo| +9296|AAAAAAAAAFECAAAA|686|Lincoln |Drive|Suite 140|Pleasant Valley|Grand Isle County|VT|03077|United States|-5|apartment| +9297|AAAAAAAABFECAAAA|668|Locust Oak|Pkwy|Suite 380|Bridgeport|Essex County|NJ|06417|United States|-5|single family| +9298|AAAAAAAACFECAAAA|805|Railroad Fifth|ST|Suite S|Valley View|Sevier County|UT|85124|United States|-7|apartment| +9299|AAAAAAAADFECAAAA|531|8th |Way|Suite F|Clifton|Macon County|AL|38014|United States|-6|apartment| +9300|AAAAAAAAEFECAAAA|852|Davis |Circle|Suite Q|Newport|Worcester County|MD|21521|United States|-5|apartment| +9301|AAAAAAAAFFECAAAA|59|1st |Wy|Suite T|Newport|Washington County|OK|71521|United States|-6|apartment| +9302|AAAAAAAAGFECAAAA|||Lane||Jamestown|Tipton County|TN|36867|United States||single family| +9303|AAAAAAAAHFECAAAA|79|||Suite W|||GA|32812|United States||apartment| +9304|AAAAAAAAIFECAAAA|427||Dr.|||Gooding County|ID||United States|-7|condo| +9305|AAAAAAAAJFECAAAA|666|Hill |Dr.|Suite I|Jackson|Marion County|IL|69583|United States|-6|condo| +9306|AAAAAAAAKFECAAAA|83|Sycamore |Ct.|Suite T|Frankfort|Chester County|SC|29681|United States|-5|apartment| +9307|AAAAAAAALFECAAAA|766|Second Church|Way|Suite C|Clinton|Boone County|AR|78222|United States|-6|single family| +9308|AAAAAAAAMFECAAAA|916|Hill Elm|Dr.|Suite 180|Liberty|Spokane County|WA|93451|United States|-8|single family| +9309|AAAAAAAANFECAAAA|341|Cedar Sunset|Avenue|Suite C|Unionville|Clay County|KY|41711|United States|-6|apartment| +9310|AAAAAAAAOFECAAAA|645|Washington |Ct.|Suite 250|Jamestown|Crockett County|TN|36867|United States|-5|apartment| +9311|AAAAAAAAPFECAAAA|669|Green Central|Pkwy|Suite G|Sulphur Springs|Portage County|OH|48354|United States|-5|condo| +9312|AAAAAAAAAGECAAAA|72|6th |Pkwy|Suite 180|Hopewell|Union County|NJ|01187|United States|-5|condo| +9313|AAAAAAAABGECAAAA|383|Adams |Pkwy|Suite 340|Oakdale|Jackson County|KS|69584|United States|-6|condo| +9314|AAAAAAAACGECAAAA|516|Willow |Ave|Suite P|Saint James|Blanco County|TX|75799|United States|-6|condo| +9315|AAAAAAAADGECAAAA|177|4th |Lane|Suite 240|Buena Vista|Lamar County|MS|55752|United States|-6|condo| +9316|AAAAAAAAEGECAAAA|454|2nd Smith|ST|Suite 30|Marion|York County|VA|20399|United States|-5|single family| +9317|AAAAAAAAFGECAAAA|944|Miller Valley|Parkway|Suite R|Shiloh|Shelby County|IN|49275|United States|-5|condo| +9318|AAAAAAAAGGECAAAA|435|Ridge Highland|Ave|Suite L|Springfield|Natrona County|WY|89303|United States|-7|single family| +9319|AAAAAAAAHGECAAAA|416|Mill Center|Court|Suite 340|Forest|Crawford County|IL|67537|United States|-6|apartment| +9320|AAAAAAAAIGECAAAA|856|15th Jefferson|Avenue|Suite 120|Waterloo|Barton County|KS|61675|United States|-6|apartment| +9321|AAAAAAAAJGECAAAA|249|Elm |Ave|Suite 250|Shady Grove|Wheeler County|GA|32812|United States|-5|apartment| +9322|AAAAAAAAKGECAAAA|153|Davis View|Lane|Suite T|Greenwood|Knox County|NE|68828|United States|-7|apartment| +9323|AAAAAAAALGECAAAA|777|2nd West|Pkwy|Suite 140|Brownsville|Barnes County|ND|59310|United States|-6|apartment| +9324|AAAAAAAAMGECAAAA|15|Meadow Woodland|Court|Suite A|Johnsonville|Patrick County|VA|27745|United States|-5|condo| +9325|AAAAAAAANGECAAAA|652|Lake Ridge|Ln|Suite 120|Providence|Belmont County|OH|46614|United States|-5|condo| +9326|AAAAAAAAOGECAAAA|649|5th Sunset|Way|Suite 0|Riverside|Lancaster County|VA|29231|United States|-5|condo| +9327|AAAAAAAAPGECAAAA|514|3rd Sixth|Way|Suite E|Marion|Wayne County|NC|20399|United States|-5|condo| +9328|AAAAAAAAAHECAAAA|269|Hillcrest |Lane|Suite 350|Deerfield|Sanborn County|SD|59840|United States|-7|apartment| +9329|AAAAAAAABHECAAAA|959|Ash Lake|Road|Suite L|Hopewell|Pike County|MO|60587|United States|-6|single family| +9330|AAAAAAAACHECAAAA|265|3rd Dogwood|Wy|Suite Q|Pierce|Lake County|TN|33360|United States|-6|condo| +9331|AAAAAAAADHECAAAA|5|West |Ln|Suite 20|Crossroads|Union County|PA|10534|United States|-5|apartment| +9332|AAAAAAAAEHECAAAA|221||ST|Suite 460||Van Buren County||55124||-6|| +9333|AAAAAAAAFHECAAAA|231|Green Forest|Way|Suite 300|Spring Valley|Greensville County|VA|26060|United States|-5|apartment| +9334|AAAAAAAAGHECAAAA|594|Miller |Road|Suite F|Shiloh|Audrain County|MO|69275|United States|-6|condo| +9335|AAAAAAAAHHECAAAA|546|1st |Wy|Suite T|Oak Hill|Floyd County|GA|37838|United States|-5|single family| +9336|AAAAAAAAIHECAAAA|615|Jackson Meadow|Court|Suite A|Wildwood|Clay County|MO|66871|United States|-6|apartment| +9337|AAAAAAAAJHECAAAA|863|Walnut Washington|Blvd|Suite 330|Clifton|Chariton County|MO|68014|United States|-6|condo| +9338|AAAAAAAAKHECAAAA|654|Sunset Ridge|Lane|Suite B|Marion|Saline County|AR|70399|United States|-6|apartment| +9339|AAAAAAAALHECAAAA|118|4th Pine|Ct.|Suite 240|Spring Hill|Brunswick County|NC|26787|United States|-5|single family| +9340|AAAAAAAAMHECAAAA|572|Second |Street|Suite V|White Oak|Snohomish County|WA|96668|United States|-8|apartment| +9341|AAAAAAAANHECAAAA|173|||Suite 280|Lakewood|Huntington County|IN|48877||-5|condo| +9342|AAAAAAAAOHECAAAA|72|West |Court|Suite 70|Point Pleasant|Overton County|TN|34749|United States|-6|apartment| +9343|AAAAAAAAPHECAAAA|215|Sunset |Way|Suite G|Georgetown|Deuel County|SD|57057|United States|-6|apartment| +9344|AAAAAAAAAIECAAAA|959|Sixth Sycamore|Ln|Suite O|Sunnyside|Clarke County|IA|51952|United States|-6|condo| +9345|AAAAAAAABIECAAAA|322|Oak Cedar|Street|Suite B|Ashland|Canyon County|ID|84244|United States|-7|condo| +9346|AAAAAAAACIECAAAA|433|Broadway View|Ln|Suite 30|Sulphur Springs|Saline County|NE|68354|United States|-7|single family| +9347|AAAAAAAADIECAAAA|252|Washington Elevnth|Drive|Suite 80|Glendale|Oneida County|NY|13951|United States|-5|single family| +9348|AAAAAAAAEIECAAAA|789|Jefferson Sixth|RD|Suite 60|Lincoln|Park County|CO|81289|United States|-7|apartment| +9349|AAAAAAAAFIECAAAA|592|North |Blvd|Suite 190|Glendale|Wyoming County|WV|23951|United States|-5|single family| +9350|AAAAAAAAGIECAAAA|612|Spruce |ST|Suite 320|Millbrook|Kanawha County|WV|27529|United States|-5|apartment| +9351|AAAAAAAAHIECAAAA|566|Cedar 6th|Street|Suite V|Bridgeport|Henry County|TN|35817|United States|-5|single family| +9352|AAAAAAAAIIECAAAA|196|Cherry |Dr.|Suite 230|New Hope|McCracken County|KY|49431|United States|-5|single family| +9353|AAAAAAAAJIECAAAA|700|Oak Wilson|||Marion|||40399|United States|-5|| +9354|AAAAAAAAKIECAAAA|523|Second 11th|Drive|Suite 350|Franklin|Aleutians West Census Area|AK|99101|United States|-9|condo| +9355|AAAAAAAALIECAAAA|780|Pine |Boulevard|Suite E|Woodville|Le Sueur County|MN|54289|United States|-6|single family| +9356|AAAAAAAAMIECAAAA|669|5th 14th|Parkway|Suite R|Ellsworth|Baldwin County|GA|35079|United States|-5|single family| +9357|AAAAAAAANIECAAAA|368|Sycamore 10th|Boulevard|Suite 420|Edgewood|Monterey County|CA|90069|United States|-8|apartment| +9358|AAAAAAAAOIECAAAA|77|13th |Parkway|Suite 110|Oak Ridge|Klamath County|OR|98371|United States|-8|single family| +9359|AAAAAAAAPIECAAAA|177|River Ridge|Circle|Suite 100|Franklin|Nicholas County|WV|29101|United States|-5|condo| +9360|AAAAAAAAAJECAAAA||Franklin Davis|ST|Suite K|Five Forks|Hall County||62293|||apartment| +9361|AAAAAAAABJECAAAA|197|Elm 8th|Blvd|Suite 80|Unionville|Caldwell Parish|LA|71711|United States|-6|apartment| +9362|AAAAAAAACJECAAAA|513|View Hillcrest|Drive|Suite A|Clifton|Choctaw County|OK|78014|United States|-6|apartment| +9363|AAAAAAAADJECAAAA|26|Jackson 6th|Dr.|Suite 30|Belmont|Humboldt County|CA|90191|United States|-8|apartment| +9364|AAAAAAAAEJECAAAA|645|Ash East|Circle|Suite 140|Kingston|Boyd County|KY|44975|United States|-6|condo| +9365|AAAAAAAAFJECAAAA|981|Walnut Woodland|Ave|Suite J|New Hope|Harding County|NM|89431|United States|-7|condo| +9366|AAAAAAAAGJECAAAA|545|Miller Adams|ST|Suite Y|Macedonia|Culpeper County|VA|21087|United States|-5|condo| +9367|AAAAAAAAHJECAAAA|365|4th Valley|Dr.|Suite O|California|Union County|PA|10141|United States|-5|apartment| +9368|AAAAAAAAIJECAAAA|261|Main |Circle|Suite 250|Springdale|Otsego County|NY|18883|United States|-5|single family| +9369|AAAAAAAAJJECAAAA|74|Walnut Spruce|Blvd|Suite 270|Cherry Valley|Stark County|IL|60854|United States|-6|single family| +9370|AAAAAAAAKJECAAAA|409|Oak |Dr.|Suite B|Florence|Fairfield County|SC|23394|United States|-5|apartment| +9371|AAAAAAAALJECAAAA|458|1st 3rd|ST|Suite 180|Oak Grove|Johnson County|NE|68370|United States|-7|condo| +9372|AAAAAAAAMJECAAAA|62|4th |Parkway|Suite 150|Hamilton|Lincoln County|GA|32808|United States|-5|apartment| +9373|AAAAAAAANJECAAAA|28|8th Highland|ST|Suite B|Enterprise|Washoe County|NV|81757|United States|-8|single family| +9374|AAAAAAAAOJECAAAA|678|Adams Main|Lane|Suite N|Doyle|Sierra County|CA|98434|United States|-8|single family| +9375|AAAAAAAAPJECAAAA|484|Johnson Poplar|Pkwy|Suite 170|Brentwood|Guadalupe County|TX|74188|United States|-6|apartment| +9376|AAAAAAAAAKECAAAA||||Suite 420|Redland|Gilliam County||96343||-8|| +9377|AAAAAAAABKECAAAA|121|7th 8th|Cir.|Suite D|Farmington|Broward County|FL|39145|United States|-5|single family| +9378|AAAAAAAACKECAAAA|493|Madison |Road|Suite 180|Friendship|Nez Perce County|ID|84536|United States|-7|condo| +9379|AAAAAAAADKECAAAA|512|Cedar 3rd|Parkway|Suite W|Belmont|Park County|WY|80191|United States|-7|condo| +9380|AAAAAAAAEKECAAAA|410|Main |Court|Suite 360|Bunker Hill|Washington County|FL|30150|United States|-5|condo| +9381|AAAAAAAAFKECAAAA|948|Spring Wilson|Ave|Suite 360|Glenwood|Henrico County|VA|23511|United States|-5|single family| +9382|AAAAAAAAGKECAAAA|956|Hickory Park|Drive|Suite U|Sulphur Springs|Metcalfe County|KY|48354|United States|-5|condo| +9383|AAAAAAAAHKECAAAA|932|Lincoln |Way|Suite 120|Clinton|Gregory County|SD|58222|United States|-6|condo| +9384|AAAAAAAAIKECAAAA|597|Meadow Elevnth|Way|Suite 70|Lebanon|San Miguel County|NM|82898|United States|-7|single family| +9385|AAAAAAAAJKECAAAA|484|Spring Main|Wy|Suite V|Mount Vernon|Lake County|CO|88482|United States|-7|apartment| +9386|AAAAAAAAKKECAAAA|515|4th Sycamore|Avenue|Suite K|Ashland|Knox County|TX|74244|United States|-6|condo| +9387|AAAAAAAALKECAAAA|34|Adams Franklin|Cir.|Suite N|Franklin|Keokuk County|IA|59101|United States|-6|apartment| +9388|AAAAAAAAMKECAAAA|741|Wilson |Wy|Suite T|Spring Hill|Beaufort County|SC|26787|United States|-5|apartment| +9389|AAAAAAAANKECAAAA|426|Ridge Oak|Pkwy|Suite L|Pleasant Valley|Monroe County|GA|32477|United States|-5|condo| +9390|AAAAAAAAOKECAAAA|182|15th Walnut|Circle|Suite 160|Salem|Knox County|TX|78048|United States|-6|condo| +9391|AAAAAAAAPKECAAAA|573|7th 1st|Wy|Suite J|Ashland|Lucas County|IA|54244|United States|-6|apartment| +9392|AAAAAAAAALECAAAA|968|9th |Ln|Suite 290|Wilson|Jackson County|IA|56971|United States|-6|single family| +9393|AAAAAAAABLECAAAA|519|First |Dr.|Suite K|Edgewood|Grant County|WI|50069|United States|-6|single family| +9394|AAAAAAAACLECAAAA|28|Fourth 15th|Cir.|Suite 30|Hillcrest|Winston County|AL|33003|United States|-6|condo| +9395|AAAAAAAADLECAAAA|991|Oak Jackson|Blvd|Suite C|Fairview|Lamar County|MS|55709|United States|-6|apartment| +9396|AAAAAAAAELECAAAA|728|Ninth |Blvd|Suite 310|Murphy|Washington County|RI|02705|United States|-5|apartment| +9397|AAAAAAAAFLECAAAA|84|Washington Second|Avenue|Suite R|Franklin|Hamilton County|IL|69101|United States|-6|condo| +9398|AAAAAAAAGLECAAAA|871|First Valley|Ln|Suite 120|Antioch|Etowah County|AL|38605|United States|-6|apartment| +9399|AAAAAAAAHLECAAAA|574|Central Jackson|Dr.|Suite A|Rutland|Hot Springs County|WY|88375|United States|-7|apartment| +9400|AAAAAAAAILECAAAA|755|Lake |Wy|Suite O|Walnut Grove|Harrison County|MO|67752|United States|-6|single family| +9401|AAAAAAAAJLECAAAA|87|Cedar College|Ln|Suite S||||71387||-6|| +9402|AAAAAAAAKLECAAAA|401|Fourth |Road|Suite 370|Jamestown|Hoke County|NC|26867|United States|-5|apartment| +9403|AAAAAAAALLECAAAA|292|Park |Wy|Suite 220|Franklin|Maries County|MO|69101|United States|-6|condo| +9404|AAAAAAAAMLECAAAA|296|Tenth |Pkwy|Suite 470|Woodlawn|Lee County|NC|24098|United States|-5|condo| +9405|AAAAAAAANLECAAAA|34|Madison Lake||Suite 300||||31398||-6|single family| +9406|AAAAAAAAOLECAAAA|688|Lincoln |Parkway|Suite Q|Plainview|Hall County|GA|33683|United States|-5|single family| +9407|AAAAAAAAPLECAAAA|719|Smith |RD|Suite M|Empire|Oglethorpe County|GA|34145|United States|-5|single family| +9408|AAAAAAAAAMECAAAA|540|Central |Parkway|Suite X|Pleasant Valley|Weber County|UT|82477|United States|-7|single family| +9409|AAAAAAAABMECAAAA|795|Main Fourth|Dr.|Suite 240|Green Acres|Delta County|TX|77683|United States|-6|condo| +9410|AAAAAAAACMECAAAA|676|Franklin Spruce|Way|Suite 380|Greenwood|Broward County|FL|38828|United States|-5|apartment| +9411|AAAAAAAADMECAAAA|481|2nd |Ct.|Suite 200|Harmony|Starke County|IN|45804|United States|-5|condo| +9412|AAAAAAAAEMECAAAA|137|8th East|Way|Suite M|Red Hill|DeWitt County|TX|74338|United States|-6|single family| +9413|AAAAAAAAFMECAAAA|73|Fourth Pine|Road|Suite 330|Hopewell|Baldwin County|GA|30587|United States|-5|single family| +9414|AAAAAAAAGMECAAAA|861|Central Second|Blvd|Suite H|Riverview|Howard County|IA|59003|United States|-6|single family| +9415|AAAAAAAAHMECAAAA|772|Park |Ln|Suite V|Glenwood|Rockland County|NY|13511|United States|-5|single family| +9416|AAAAAAAAIMECAAAA|83|6th Adams|Wy|Suite 210|Liberty|Meigs County|OH|43451|United States|-5|single family| +9417|AAAAAAAAJMECAAAA|67|8th Main|ST|Suite 410|Blanchard|Union County|MS|55985|United States|-6|condo| +9418|AAAAAAAAKMECAAAA|639|Johnson Main|Boulevard|Suite G|Unionville|Cheatham County|TN|31711|United States|-5|single family| +9419|AAAAAAAALMECAAAA|901|Lincoln |Avenue|Suite B|Unionville|Montgomery County|MD|21711|United States|-5|single family| +9420|AAAAAAAAMMECAAAA|935|Maple |Avenue|Suite G|Leland|Custer County|SD|59452|United States|-6|apartment| +9421|AAAAAAAANMECAAAA|794|Madison 3rd|RD|Suite 320|Walnut Grove|Oregon County|MO|67752|United States|-6|apartment| +9422|AAAAAAAAOMECAAAA|371|Mill Mill|Lane|Suite 180|Springfield|Warren County|OH|49303|United States|-5|single family| +9423|AAAAAAAAPMECAAAA|303|Green 6th|Circle|Suite O|Clifton|Delta County|MI|48014|United States|-5|single family| +9424|AAAAAAAAANECAAAA|894|1st |Court|Suite 30|Mount Vernon|Muhlenberg County|KY|48482|United States|-5|single family| +9425|AAAAAAAABNECAAAA|475|15th Ash|Way|Suite 290|Unionville|San Juan County|NM|81711|United States|-7|apartment| +9426|AAAAAAAACNECAAAA|70|Franklin Fourth|ST|Suite L|Kingston|Monroe County|MO|64975|United States|-6|single family| +9427|AAAAAAAADNECAAAA|465|Washington |Parkway|Suite V|Newport|Benton County|OR|91521|United States|-8|condo| +9428|AAAAAAAAENECAAAA|469|Hickory Walnut|Ct.|Suite U|Bennett|Ross County|OH|41715|United States|-5|condo| +9429|AAAAAAAAFNECAAAA|263|6th |Road|Suite O|Deerfield|Queen Anne County|MD|29840|United States|-5|condo| +9430|AAAAAAAAGNECAAAA|644|West 1st|Lane|Suite V|Sunnyside|Todd County|KY|41952|United States|-5|single family| +9431|AAAAAAAAHNECAAAA|729|Johnson Hickory|Avenue|Suite P|Greenfield|Harlan County|KY|45038|United States|-6|single family| +9432|AAAAAAAAINECAAAA|315|Woodland Washington|Court|Suite L|Summit|Berrien County|MI|40499|United States|-5|condo| +9433|AAAAAAAAJNECAAAA|749|7th |Drive|Suite F|Ashland|Sumter County|FL|34244|United States|-5|single family| +9434|AAAAAAAAKNECAAAA|56|Fourth |Court|Suite G|Lakeside|Madison County|AR|79532|United States|-6|condo| +9435|AAAAAAAALNECAAAA|791|North |Road|Suite 210|Hopewell|Marshall County|KY|40587|United States|-5|single family| +9436|AAAAAAAAMNECAAAA|153|Hickory 1st|Ave|Suite 450|Benton|Washington County|OH|41549|United States|-5|condo| +9437|AAAAAAAANNECAAAA|21|Walnut |Pkwy|Suite 260|Walnut Grove|Allen County|IN|47752|United States|-5|apartment| +9438|AAAAAAAAONECAAAA|173|Maple |Pkwy|Suite D|Lakeside|Jones County|MS|59532|United States|-6|single family| +9439|AAAAAAAAPNECAAAA|723|North |Lane|Suite D|Perkins|Trinity County|TX|71852|United States|-6|single family| +9440|AAAAAAAAAOECAAAA||Cedar 8th||Suite P||||76098|United States||| +9441|AAAAAAAABOECAAAA|554|Walnut |RD|Suite M|Mount Olive|New Haven County|CT|08659|United States|-5|single family| +9442|AAAAAAAACOECAAAA|551|Jefferson Johnson|Lane|Suite 160|Sawyer|White County|IL|66045|United States|-6|condo| +9443|AAAAAAAADOECAAAA|724|Ridge |RD|Suite A|Omega|Edmonson County|KY|46364|United States|-6|apartment| +9444|AAAAAAAAEOECAAAA|692|Ridge |Street|Suite C|Roscoe|Sheridan County|WY|81854|United States|-7|apartment| +9445|AAAAAAAAFOECAAAA|948|9th 14th|Way|Suite 230|Spring Hill|Bedford County|TN|36787|United States|-5|apartment| +9446|AAAAAAAAGOECAAAA|562|South |Way|Suite U|Bethel|Pacific County|WA|95281|United States|-8|condo| +9447|AAAAAAAAHOECAAAA|867|8th Hill|Street||Mount Pleasant|Union County|SC|21933|United States||condo| +9448|AAAAAAAAIOECAAAA|462|South |Ct.|Suite E|Florence|Hartford County|CT|03994|United States|-5|condo| +9449|AAAAAAAAJOECAAAA|91|Lake |Pkwy|Suite L|Springfield|Sanilac County|MI|49303|United States|-5|apartment| +9450|AAAAAAAAKOECAAAA|426|Spruce |Avenue|Suite V|Wilson|Gem County|ID|86971|United States|-7|single family| +9451|AAAAAAAALOECAAAA|860|Jackson 9th|Ave|Suite S|Green Acres|Barrow County|GA|37683|United States|-5|condo| +9452|AAAAAAAAMOECAAAA|373||||Five Forks||NE||United States||| +9453|AAAAAAAANOECAAAA|224|Highland Sycamore|Court|Suite 180|Summit|Alger County|MI|40499|United States|-5|single family| +9454|AAAAAAAAOOECAAAA|835|Sycamore Hickory|Dr.|Suite C|Oak Hill|Carbon County|UT|87838|United States|-7|apartment| +9455|AAAAAAAAPOECAAAA|388|Center |Pkwy|Suite 190|Summit|Greene County|TN|30499|United States|-5|apartment| +9456|AAAAAAAAAPECAAAA|98|Park 9th|Lane|Suite H|Jackson|Orange County|VT|09583|United States|-5|apartment| +9457|AAAAAAAABPECAAAA|920|Elm |ST|Suite J|Spring Hill|Mitchell County|NC|26787|United States|-5|condo| +9458|AAAAAAAACPECAAAA|725|Center Ash|Ct.|Suite 0|Carter|Jackson County|KS|60919|United States|-6|single family| +9459|AAAAAAAADPECAAAA|676|Williams Meadow|Wy|Suite 410|Union|Terrell County|GA|38721|United States|-5|condo| +9460|AAAAAAAAEPECAAAA||Adams |Lane|||||43451|||| +9461|AAAAAAAAFPECAAAA|923|Hillcrest |Pkwy|Suite 430|Perkins|Madison County|GA|31852|United States|-5|apartment| +9462|AAAAAAAAGPECAAAA|29|Cedar |Wy|Suite C|Gilmore|Itawamba County|MS|55464|United States|-6|condo| +9463|AAAAAAAAHPECAAAA|479|Fourth |RD|Suite 130|Mount Zion|Dade County|GA|38054|United States|-5|apartment| +9464|AAAAAAAAIPECAAAA|984|2nd |Wy|Suite 0|Lebanon|Monongalia County|WV|22898|United States|-5|single family| +9465|AAAAAAAAJPECAAAA|681|15th |Dr.|Suite 360|Oak Hill|Wright County|MN|57838|United States|-6|apartment| +9466|AAAAAAAAKPECAAAA|801|Jefferson Cedar|Way|Suite 190|Greenwood|Rice County|KS|68828|United States|-6|apartment| +9467|AAAAAAAALPECAAAA|978|14th |Drive|Suite D|Oakwood|Brown County|OH|40169|United States|-5|condo| +9468|AAAAAAAAMPECAAAA|719|Lake Cedar|Blvd|Suite Q|Marion|Drew County|AR|70399|United States|-6|apartment| +9469|AAAAAAAANPECAAAA|653|Elm |Wy|Suite 480|Lebanon|McDonough County|IL|62898|United States|-6|single family| +9470|AAAAAAAAOPECAAAA|889|Center |Road|Suite 170|Wilson|Newport News city|VA|26971|United States|-5|single family| +9471|AAAAAAAAPPECAAAA|849|Ridge Second|Ave||New Hope|Kimball County|||United States||| +9472|AAAAAAAAAAFCAAAA|781|East |Boulevard|Suite V|Clinton|Pershing County|NV|88222|United States|-8|condo| +9473|AAAAAAAABAFCAAAA|988|Fourth 1st|Ave|Suite T|Plainview|Fremont County|ID|83683|United States|-7|condo| +9474|AAAAAAAACAFCAAAA|210|Pine 6th|Lane|Suite 290|Pleasant Grove|Livingston County|MO|64136|United States|-6|single family| +9475|AAAAAAAADAFCAAAA|366|12th 3rd|Boulevard|Suite W|Oakdale|Bucks County|PA|19584|United States|-5|single family| +9476|AAAAAAAAEAFCAAAA|268|Hill Smith|Ln|Suite 130|Newport|Morgan County|IL|61521|United States|-6|apartment| +9477|AAAAAAAAFAFCAAAA|472|Hill Smith|Drive|Suite I|Denmark|Chemung County|NY|15576|United States|-5|apartment| +9478|AAAAAAAAGAFCAAAA|872|Spruce Main|Boulevard|Suite 270|Long Branch|Chattahoochee County|GA|37682|United States|-5|condo| +9479|AAAAAAAAHAFCAAAA|630|15th |Ave|Suite 290|Mount Pleasant|Knox County|NE|61933|United States|-7|condo| +9480|AAAAAAAAIAFCAAAA|6|Locust |Ln|Suite C|Springfield|Lagrange County|IN|49303|United States|-5|condo| +9481|AAAAAAAAJAFCAAAA|740|Broadway Dogwood|Cir.|Suite 220|Hamilton|Bartholomew County|IN|42808|United States|-5|apartment| +9482|AAAAAAAAKAFCAAAA|702|Willow College|Way|Suite J|Union|Adams County|PA|18721|United States|-5|single family| +9483|AAAAAAAALAFCAAAA|152|Center 2nd|Street|Suite 460|Sulphur Springs|Lenoir County|NC|28354|United States|-5|apartment| +9484|AAAAAAAAMAFCAAAA|208|Hickory Jackson|Boulevard|Suite 230|Hillcrest|Dickson County|TN|33003|United States|-5|apartment| +9485|AAAAAAAANAFCAAAA|975|Willow |RD|Suite X|Riverdale|Franklin County|MO|69391|United States|-6|apartment| +9486|AAAAAAAAOAFCAAAA|977|East |Lane|||Seneca County|NY|12808|United States|-5|condo| +9487|AAAAAAAAPAFCAAAA|98|Lakeview |Cir.|Suite 430|Oakland|Sherburne County|MN|59843|United States|-6|single family| +9488|AAAAAAAAABFCAAAA|143|Maple |ST|Suite 430|Shiloh|Yalobusha County|MS|59275|United States|-6|condo| +9489|AAAAAAAABBFCAAAA|||Pkwy||Arlington||||United States||condo| +9490|AAAAAAAACBFCAAAA|743|Seventh Locust|Road|Suite W|Union|Terrell County|GA|38721|United States|-5|condo| +9491|AAAAAAAADBFCAAAA|368|Jackson Pine|Ave|Suite 320|Antioch|Reno County|KS|68605|United States|-6|apartment| +9492|AAAAAAAAEBFCAAAA|857|Forest Elm|Parkway|Suite A|Union|Wyoming County|NY|18721|United States|-5|single family| +9493|AAAAAAAAFBFCAAAA|992|Lakeview |Blvd|Suite P|Brownsville|Okaloosa County|FL|39310|United States|-5|single family| +9494|AAAAAAAAGBFCAAAA|669|Lee |Way|Suite C|Springdale|Canyon County|ID|88883|United States|-7|single family| +9495|AAAAAAAAHBFCAAAA|183|Main |Ave|Suite 220|Highland|Laclede County|MO|69454|United States|-6|condo| +9496|AAAAAAAAIBFCAAAA|683|Jefferson |Lane|Suite G|Oak Hill|Winona County|MN|57838|United States|-6|single family| +9497|AAAAAAAAJBFCAAAA|309|4th Laurel|RD|Suite 350|Providence|Crawford County|AR|76614|United States|-6|condo| +9498|AAAAAAAAKBFCAAAA|986|Mill |Cir.|Suite S|Oak Ridge|Lincoln County|WA|98371|United States|-8|single family| +9499|AAAAAAAALBFCAAAA|680|Third Jefferson|Dr.|Suite D|Woodland|Claiborne County|TN|34854|United States|-5|single family| +9500|AAAAAAAAMBFCAAAA|735|Main Oak|Blvd|Suite Q|Salem|Sagadahoc County|ME|08648|United States|-5|condo| +9501|AAAAAAAANBFCAAAA|187|Main |Wy|Suite C|Royal|Taylor County|GA|35819|United States|-5|single family| +9502|AAAAAAAAOBFCAAAA|523|View |ST|Suite H|Wilson|Wabasha County|MN|56971|United States|-6|single family| +9503|AAAAAAAAPBFCAAAA|86|14th Ninth|RD|Suite 110|White Oak|Eddy County|NM|86668|United States|-7|condo| +9504|AAAAAAAAACFCAAAA|437|9th West|Pkwy|Suite C|Wilson|White County|IL|66971|United States|-6|condo| +9505|AAAAAAAABCFCAAAA|161|Sunset |Cir.|Suite H|Lakeside|Louisa County|IA|59532|United States|-6|condo| +9506|AAAAAAAACCFCAAAA|821|Park |Circle|Suite I|Forest|Madison County|IL|67537|United States|-6|apartment| +9507|AAAAAAAADCFCAAAA|49|West View|Ln|Suite V|Richardson|Umatilla County|OR|97687|United States|-8|condo| +9508|AAAAAAAAECFCAAAA|642|Valley |RD|Suite 140|Plainview|Logan County|KY|43683|United States|-5|apartment| +9509|AAAAAAAAFCFCAAAA|854|Seventh |Road|Suite Q|Clinton|Muskegon County|MI|48222|United States|-5|single family| +9510|AAAAAAAAGCFCAAAA|160|Oak Eigth|Ln|Suite O|Farmington|Mercer County|OH|49145|United States|-5|single family| +9511|AAAAAAAAHCFCAAAA|281|Main |Ave|Suite 460|Hamilton|Stevens County|MN|52808|United States|-6|single family| +9512|AAAAAAAAICFCAAAA|614|First |Parkway|Suite T|Arlington|Poinsett County|AR|76557|United States|-6|single family| +9513|AAAAAAAAJCFCAAAA|307|Sycamore Lincoln|Street|Suite 480|Salem|Grayson County|VA|28048|United States|-5|apartment| +9514|AAAAAAAAKCFCAAAA|148|Ash Mill|Boulevard|Suite R|Concord|Scott County|IL|64107|United States|-6|single family| +9515|AAAAAAAALCFCAAAA|891|Washington Miller|Boulevard|Suite Y|Jamestown|Windsor County|VT|07467|United States|-5|apartment| +9516|AAAAAAAAMCFCAAAA|262|Sycamore |Drive|Suite O|Clinton|Pickett County|TN|38222|United States|-6|condo| +9517|AAAAAAAANCFCAAAA|919|Center Lee|Ln|Suite 260|Valley View|Colbert County|AL|35124|United States|-6|apartment| +9518|AAAAAAAAOCFCAAAA|360|1st |Avenue|Suite V|Hamilton|Llano County|TX|72808|United States|-6|single family| +9519|AAAAAAAAPCFCAAAA|750|Ridge Hill|Way|Suite V|Greenfield|McMinn County|TN|35038|United States|-6|single family| +9520|AAAAAAAAADFCAAAA|999|10th 9th|Circle|Suite 480|Clinton|Fremont County|CO|88222|United States|-7|single family| +9521|AAAAAAAABDFCAAAA|79|Lee |RD|Suite 170|Springdale|Wilcox County|GA|38883|United States|-5|single family| +9522|AAAAAAAACDFCAAAA|842|Oak |Cir.|Suite I|Leesburg|Yuba County|CA|95605|United States|-8|condo| +9523|AAAAAAAADDFCAAAA||Hillcrest |Road||Salem|Maui County||||-10|| +9524|AAAAAAAAEDFCAAAA|989|Railroad |Ln|Suite R|Oak Hill|Cheyenne County|NE|67838|United States|-6|apartment| +9525|AAAAAAAAFDFCAAAA||Smith Spring|Court|Suite 200|Providence||||||apartment| +9526|AAAAAAAAGDFCAAAA|348|Eigth |Drive|Suite 0|Newport|Waukesha County|WI|51521|United States|-6|condo| +9527|AAAAAAAAHDFCAAAA|780|9th |Drive|Suite 430|Hillcrest|Mason County|WV|23003|United States|-5|apartment| +9528|AAAAAAAAIDFCAAAA|617|7th River|Dr.|Suite Y|Greenville|Paulding County|GA|31387|United States|-5|apartment| +9529|AAAAAAAAJDFCAAAA|796|Miller |Way|Suite D|Glendale|Northwest Arctic Borough|AK|93951|United States|-9|single family| +9530|AAAAAAAAKDFCAAAA|142|Franklin 1st|Cir.|Suite 250|Unionville|Pierce County|WA|91711|United States|-8|condo| +9531|AAAAAAAALDFCAAAA|974|10th Walnut|Ln|Suite I|Jackson|Potter County|SD|59583|United States|-7|apartment| +9532|AAAAAAAAMDFCAAAA|689|Locust |Circle|Suite L|Summit|Crawford County|KS|60499|United States|-6|apartment| +9533|AAAAAAAANDFCAAAA|5|Miller Spring|Wy|Suite 420|Mount Vernon|Beckham County|OK|78482|United States|-6|condo| +9534|AAAAAAAAODFCAAAA|191|Washington |Blvd|Suite 110|Red Hill|Houston County|AL|34338|United States|-6|apartment| +9535|AAAAAAAAPDFCAAAA|923|Willow |ST|Suite 350|Webb|Randolph County|NC|20899|United States|-5|apartment| +9536|AAAAAAAAAEFCAAAA|363|Green Lincoln|Road|Suite 40|Springfield|Baxter County|AR|79303|United States|-6|apartment| +9537|AAAAAAAABEFCAAAA|607|Church |ST|Suite S|Greenwood|Chisago County|MN|58828|United States|-6|apartment| +9538|AAAAAAAACEFCAAAA|766|Park Walnut|Boulevard|Suite J|Leesburg|Calcasieu Parish|LA|75605|United States|-6|condo| +9539|AAAAAAAADEFCAAAA|408|1st |Avenue|Suite J|Greenwood|Trimble County|KY|48828|United States|-5|single family| +9540|AAAAAAAAEEFCAAAA|325|Elm Broadway|ST|Suite 430|Salem|Stephens County|GA|38048|United States|-5|condo| +9541|AAAAAAAAFEFCAAAA|21|Sycamore |Court|Suite 370|Fairfield|Montgomery County|MS|56192|United States|-6|single family| +9542|AAAAAAAAGEFCAAAA|749|3rd Spruce|Road|Suite V|Highland Park|Snyder County|PA|16534|United States|-5|condo| +9543|AAAAAAAAHEFCAAAA|853|Johnson |Ave|Suite J|Walnut Grove|Franklin County|ME|08352|United States|-5|single family| +9544|AAAAAAAAIEFCAAAA|202|Forest |Dr.|Suite X|Jackson|Dodge County|MN|59583|United States|-6|condo| +9545|AAAAAAAAJEFCAAAA|997|Railroad Walnut||Suite 80|Wilson||KY|46971|||| +9546|AAAAAAAAKEFCAAAA|352|Maple |Blvd|Suite L|Buena Vista|Halifax County|NC|25752|United States|-5|single family| +9547|AAAAAAAALEFCAAAA|22|Hill Fourth|Blvd|Suite C|Arlington|Moore County|TX|76557|United States|-6|condo| +9548|AAAAAAAAMEFCAAAA|346|Miller 6th|Boulevard|Suite 430|Harmony|Tulsa County|OK|75804|United States|-6|single family| +9549|AAAAAAAANEFCAAAA|469|View |Street|Suite 400|Oak Grove|Pierce County|GA|38370|United States|-5|condo| +9550|AAAAAAAAOEFCAAAA|167|13th |Drive|Suite K|Pleasant Grove|Sussex County|NJ|04736|United States|-5|single family| +9551|AAAAAAAAPEFCAAAA|833|Second |Ct.|Suite X|White Oak|Callahan County|TX|76668|United States|-6|apartment| +9552|AAAAAAAAAFFCAAAA|465|Laurel Highland|Wy|Suite B|Mountain View|Collin County|TX|74466|United States|-6|condo| +9553|AAAAAAAABFFCAAAA|474|15th 1st|Way|Suite F|Valley View|Cole County|MO|65124|United States|-6|apartment| +9554|AAAAAAAACFFCAAAA|958|Sunset 15th|Blvd|Suite 430|Salem|Cleveland County|OK|78048|United States|-6|apartment| +9555|AAAAAAAADFFCAAAA|99|5th |Wy|Suite 480|Red Oak|Todd County|SD|55018|United States|-7|apartment| +9556|AAAAAAAAEFFCAAAA|531|Elm |RD|Suite 400|Forest Hills|Goochland County|VA|29237|United States|-5|condo| +9557|AAAAAAAAFFFCAAAA|408|Valley Sycamore|Blvd|Suite 460|Oakwood|Marion County|KY|40169|United States|-5|condo| +9558|AAAAAAAAGFFCAAAA|158|Lincoln |Cir.|Suite 340|Pleasant Valley|Irwin County|GA|32477|United States|-5|apartment| +9559|AAAAAAAAHFFCAAAA|680|Elm |Road|Suite 160|Morris|Carbon County|PA|16696|United States|-5|single family| +9560|AAAAAAAAIFFCAAAA|409|Railroad Park|Drive|Suite D|Springfield|Lincoln County|WY|89303|United States|-7|single family| +9561|AAAAAAAAJFFCAAAA|805|3rd |Ct.|Suite O|Fairfield|Garfield County|OK|76192|United States|-6|single family| +9562|AAAAAAAAKFFCAAAA|711|Fifth Maple|Lane|Suite 420|Clifford|Marquette County|MI|48164|United States|-5|condo| +9563|AAAAAAAALFFCAAAA|279|Green |Blvd|Suite T|Belmont|Dallas County|AL|30191|United States|-6|condo| +9564|AAAAAAAAMFFCAAAA|70|Central 11th|Boulevard|Suite Q|Riverdale|Blackford County|IN|49391|United States|-5|apartment| +9565|AAAAAAAANFFCAAAA|82|6th 2nd|Boulevard|Suite 50|Concord|Burke County|NC|24107|United States|-5|condo| +9566|AAAAAAAAOFFCAAAA|788|12th Lake|Parkway|Suite I|Bridgeport|Roanoke County|VA|25817|United States|-5|apartment| +9567|AAAAAAAAPFFCAAAA|221|Woodland |Boulevard|Suite 50|Plainview|Kossuth County|IA|53683|United States|-6|condo| +9568|AAAAAAAAAGFCAAAA|866|Walnut |Ln|Suite 250|New Hope|Tolland County|CT|09431|United States|-5|apartment| +9569|AAAAAAAABGFCAAAA|251|7th |RD|Suite X|Edgewood|Cooke County|TX|70069|United States|-6|condo| +9570|AAAAAAAACGFCAAAA|473|Washington Church|Boulevard|Suite F|Shiloh|Tooele County|UT|89275|United States|-7|condo| +9571|AAAAAAAADGFCAAAA|357|Forest River|Blvd|Suite W|Oakland|Chisago County|MN|59843|United States|-6|apartment| +9572|AAAAAAAAEGFCAAAA|940|Lee |Wy|Suite M|Rosebud|Palo Pinto County|TX|72244|United States|-6|condo| +9573|AAAAAAAAFGFCAAAA|869|Seventh 5th|Ave|Suite F|Oak Grove|Henry County|AL|38370|United States|-6|apartment| +9574|AAAAAAAAGGFCAAAA|962|Woodland Cherry|Street|Suite Q|Florence|Carter County|OK|73394|United States|-6|single family| +9575|AAAAAAAAHGFCAAAA|809|Valley Second|Pkwy|Suite 10|Oak Hill|Macon County|NC|27838|United States|-5|condo| +9576|AAAAAAAAIGFCAAAA|147|Oak |Ct.|Suite 400|Lebanon|Owsley County|KY|42898|United States|-5|condo| +9577|AAAAAAAAJGFCAAAA|540|Spruce |Ct.|Suite M||||||-5|single family| +9578|AAAAAAAAKGFCAAAA|367|Third |ST|Suite 450|Midway|Indian River County|FL|31904|United States|-5|condo| +9579|AAAAAAAALGFCAAAA|210|Main Cherry|Circle|Suite X|Tanglewood|Onslow County|NC|28994|United States|-5|condo| +9580|AAAAAAAAMGFCAAAA|222|View Sunset|Ln|Suite S|Waterloo|Bryan County|GA|31675|United States|-5|condo| +9581|AAAAAAAANGFCAAAA|760|Third Jefferson|Drive|Suite O|Ryan|Greene County|GA|30525|United States|-5|single family| +9582|AAAAAAAAOGFCAAAA|384|7th Birch|Boulevard|Suite H|Greenwood|Seminole County|FL|38828|United States|-5|condo| +9583|AAAAAAAAPGFCAAAA|771|Chestnut |RD|Suite 440|Reno|Wagoner County|OK|70344|United States|-6|single family| +9584|AAAAAAAAAHFCAAAA|976|Cherry |RD|Suite Y|Mountain View|Henry County|OH|44466|United States|-5|single family| +9585|AAAAAAAABHFCAAAA|591|Williams |Drive|Suite 130|Friendship|Monmouth County|NJ|05136|United States|-5|condo| +9586|AAAAAAAACHFCAAAA|825|Railroad |Ct.|Suite N|Mount Pleasant|Nueces County|TX|71933|United States|-6|single family| +9587|AAAAAAAADHFCAAAA|144|Railroad |Wy|Suite 410|Cedar Grove|Castro County|TX|70411|United States|-6|single family| +9588|AAAAAAAAEHFCAAAA|105|Railroad Miller|ST|Suite Y|Waterloo|Lake County|MN|51675|United States|-6|apartment| +9589|AAAAAAAAFHFCAAAA|511|Park Willow|Pkwy|Suite 410|Oak Ridge|Giles County|VA|28371|United States|-5|apartment| +9590|AAAAAAAAGHFCAAAA|958|Johnson 11th|Blvd|Suite V|Greenfield|Lenoir County|NC|25038|United States|-5|apartment| +9591|AAAAAAAAHHFCAAAA|9|1st Laurel|Ct.|Suite 240|Mount Vernon|Glenn County|CA|98482|United States|-8|single family| +9592|AAAAAAAAIHFCAAAA|944|Park |ST|Suite F|Unionville|Dewey County|SD|51711|United States|-6|single family| +9593|AAAAAAAAJHFCAAAA|623|Park Pine|Blvd|Suite 400|Mount Pleasant|Acadia Parish|LA|71933|United States|-6|single family| +9594|AAAAAAAAKHFCAAAA|602|3rd |Ln|Suite 490|Bethel|Jackson County|NC|25281|United States|-5|single family| +9595|AAAAAAAALHFCAAAA|656|Church |Street|Suite A|Summit|Dewey County|OK|70499|United States|-6|apartment| +9596|AAAAAAAAMHFCAAAA|567|Cherry |Road|Suite 480|Franklin|Pacific County|WA|99101|United States|-8|apartment| +9597|AAAAAAAANHFCAAAA|166|Fourth West|Parkway|Suite H|Five Points|Apache County|AZ|86098|United States|-7|single family| +9598|AAAAAAAAOHFCAAAA|279|11th Pine|RD|Suite 480|Greenwood|Jewell County|KS|68828|United States|-6|condo| +9599|AAAAAAAAPHFCAAAA|515|East |Dr.|Suite 200|Springfield|Dyer County|TN|39303|United States|-5|condo| +9600|AAAAAAAAAIFCAAAA|492|East Meadow|Court|Suite 280|Florence|Fulton County|KY|43394|United States|-6|condo| +9601|AAAAAAAABIFCAAAA|512|Highland Smith|Dr.|Suite 460|Fairview|Southampton County|VA|25709|United States|-5|apartment| +9602|AAAAAAAACIFCAAAA|911|Wilson |Boulevard|Suite O|Brownsville|Jackson County|GA|39310|United States|-5|apartment| +9603|AAAAAAAADIFCAAAA|973|Hill Park|Cir.|Suite H|Riverview|Cheboygan County|MI|49003|United States|-5|condo| +9604|AAAAAAAAEIFCAAAA||||Suite U||White County||79145||-6|| +9605|AAAAAAAAFIFCAAAA|837|Hill |Lane|Suite 370|Granite|Perry County|AR|76284|United States|-6|condo| +9606|AAAAAAAAGIFCAAAA|854|Ash |Wy|Suite U|Riverview|Valley County|MT|69003|United States|-7|single family| +9607|AAAAAAAAHIFCAAAA|13|Willow |Ave|Suite Y|Belmont|Tangipahoa Parish|LA|70191|United States|-6|single family| +9608|AAAAAAAAIIFCAAAA||||Suite Q||Hays County||79003||-6|apartment| +9609|AAAAAAAAJIFCAAAA|16|11th 6th|Ave|Suite J|Oakland|Clayton County|GA|39843|United States|-5|condo| +9610|AAAAAAAAKIFCAAAA|310|Williams Birch|Ct.|Suite Q|Five Points|Polk County|IA|56098|United States|-6|condo| +9611|AAAAAAAALIFCAAAA|525|Lakeview |Wy|Suite Q|Newtown|Butler County|OH|41749|United States|-5|single family| +9612|AAAAAAAAMIFCAAAA|503|Green Sixth|Ln|Suite I|Friendship|Newton County|MS|54536|United States|-6|condo| +9613|AAAAAAAANIFCAAAA|751|Wilson Fifth|Ln|Suite 420|Mountain View|Big Horn County|MT|64466|United States|-7|single family| +9614|AAAAAAAAOIFCAAAA|908|Maple Seventh|ST|Suite 270|Pine Grove|Bonneville County|ID|84593|United States|-7|apartment| +9615|AAAAAAAAPIFCAAAA|400|Jefferson ||||Washington County|TN|||-6|single family| +9616|AAAAAAAAAJFCAAAA|||||Woodlawn|Roberts County||||-6|condo| +9617|AAAAAAAABJFCAAAA|107|5th Hickory|Road|Suite W|White Oak|Franklin County|IN|46668|United States|-5|condo| +9618|AAAAAAAACJFCAAAA|974|Hill Fifth|Boulevard|Suite 170|Oak Ridge|Passaic County|NJ|08971|United States|-5|condo| +9619|AAAAAAAADJFCAAAA|208|Willow Woodland|ST|Suite 60|Bay View|Kane County|IL|66457|United States|-6|condo| +9620|AAAAAAAAEJFCAAAA|561|Main 13th|Parkway|Suite 270|Crossroads|Clay County|NC|20534|United States|-5|single family| +9621|AAAAAAAAFJFCAAAA|625|Hill |Boulevard|Suite G|Lakeview|Taney County|MO|68579|United States|-6|single family| +9622|AAAAAAAAGJFCAAAA|955|3rd Center|Drive|Suite G|Valley View|Adams County|WA|95124|United States|-8|condo| +9623|AAAAAAAAHJFCAAAA|109|Ash 6th|ST|Suite 120|Tracy|Staunton city|VA|26340|United States|-5|condo| +9624|AAAAAAAAIJFCAAAA|149|3rd |Blvd|Suite 400|Wildwood|Hancock County|IL|66871|United States|-6|condo| +9625|AAAAAAAAJJFCAAAA|344|Davis Locust|Cir.|Suite 0|Florence|Wilkes County|NC|23394|United States|-5|condo| +9626|AAAAAAAAKJFCAAAA|805|First |Ave|Suite 270|Plainville|Custer County|SD|56115|United States|-6|apartment| +9627|AAAAAAAALJFCAAAA|380||Ln|Suite 410|Pine Grove||KS||||condo| +9628|AAAAAAAAMJFCAAAA|48|Central |Blvd|Suite I|Ashland|Galveston County|TX|74244|United States|-6|apartment| +9629|AAAAAAAANJFCAAAA|158|Woodland |ST|Suite 130|Oakdale|Okeechobee County|FL|39584|United States|-5|single family| +9630|AAAAAAAAOJFCAAAA|806|Park Wilson|Avenue|Suite 470|Arlington|Shelby County|OH|46557|United States|-5|apartment| +9631|AAAAAAAAPJFCAAAA|188|Hillcrest |Way|Suite Y|Cedar Grove|Nemaha County|NE|60411|United States|-7|single family| +9632|AAAAAAAAAKFCAAAA|9|Park Lake|Cir.|Suite 330|Riverview|Barbour County|WV|29003|United States|-5|apartment| +9633|AAAAAAAABKFCAAAA|10|Center 9th|ST|Suite 180|Ashland|Wheeler County|GA|34244|United States|-5|condo| +9634|AAAAAAAACKFCAAAA|937|3rd |Parkway|Suite 20|Salem|Morgan County|IN|48048|United States|-5|condo| +9635|AAAAAAAADKFCAAAA|959|Poplar |Ln|Suite D|Hillcrest|Worcester County|MD|23003|United States|-5|condo| +9636|AAAAAAAAEKFCAAAA|440|10th |Avenue|Suite 450|Five Points|Stephens County|OK|76098|United States|-6|apartment| +9637|AAAAAAAAFKFCAAAA|145|10th 2nd|ST|Suite 400|Wildwood|DeKalb County|GA|36871|United States|-5|single family| +9638|AAAAAAAAGKFCAAAA|60|1st |Cir.|Suite W|Union|Thomas County|KS|68721|United States|-6|apartment| +9639|AAAAAAAAHKFCAAAA|202|Johnson |Boulevard|Suite 350|Freedom|Muskingum County|OH|41651|United States|-5|condo| +9640|AAAAAAAAIKFCAAAA|756|3rd 15th|Road|Suite 280|Cedar Grove|Wilson County|TN|30411|United States|-5|single family| +9641|AAAAAAAAJKFCAAAA|643|Cedar |Circle|Suite F|Valley View|Bell County|TX|75124|United States|-6|apartment| +9642|AAAAAAAAKKFCAAAA|768|Sycamore Mill|Lane|Suite F|Ferguson|Columbus County|NC|21821|United States|-5|single family| +9643|AAAAAAAALKFCAAAA|877|Ash View|Court|Suite R|Macedonia|Northwest Arctic Borough|AK|91087|United States|-9|single family| +9644|AAAAAAAAMKFCAAAA|728|Pine |Dr.|Suite 380|Brownsville|Johnson County|NE|69310|United States|-7|condo| +9645|AAAAAAAANKFCAAAA|895|Park |Ave|Suite S|Liberty|Cass County|ND|53451|United States|-6|apartment| +9646|AAAAAAAAOKFCAAAA|111|Willow |Boulevard|Suite L|Red Hill|Clark County|IL|64338|United States|-6|single family| +9647|AAAAAAAAPKFCAAAA|281||Dr.|||Petroleum County|MT|69398|United States|-7|| +9648|AAAAAAAAALFCAAAA|760|Woodland 1st|Dr.|Suite E|Summit|Wayne County|WV|20499|United States|-5|condo| +9649|AAAAAAAABLFCAAAA|150|Broadway |Drive|Suite C|Greenfield|Humboldt County|CA|95038|United States|-8|single family| +9650|AAAAAAAACLFCAAAA|255|West 9th|Drive|Suite H|Jackson|Neshoba County|MS|59583|United States|-6|apartment| +9651|AAAAAAAADLFCAAAA|378|Woodland |Ave|Suite 230|White Oak|Pendleton County|WV|26668|United States|-5|apartment| +9652|AAAAAAAAELFCAAAA|993|Smith |Way|Suite 390|Shaw|Howell County|MO|60618|United States|-6|apartment| +9653|AAAAAAAAFLFCAAAA|429|Cherry |Court|Suite R|Oak Hill|Gilmer County|GA|37838|United States|-5|single family| +9654|AAAAAAAAGLFCAAAA|277|Spruce 1st|Ave|Suite A|Spring Valley|Randolph County|WV|26060|United States|-5|apartment| +9655|AAAAAAAAHLFCAAAA|759|Wilson |RD|Suite L|Harmony|Carroll County|IL|65804|United States|-6|condo| +9656|AAAAAAAAILFCAAAA|508|Spruce Valley|Wy|Suite F|Union|Wayne County|GA|38721|United States|-5|apartment| +9657|AAAAAAAAJLFCAAAA|536|Second Mill|Cir.|Suite U|Bethel|Trinity County|CA|95281|United States|-8|condo| +9658|AAAAAAAAKLFCAAAA|744|Hill |Dr.|Suite 120|Summit|Reeves County|TX|70499|United States|-6|single family| +9659|AAAAAAAALLFCAAAA|607|15th |Ct.|Suite 250|Springfield|Crenshaw County|AL|39303|United States|-6|single family| +9660|AAAAAAAAMLFCAAAA|109|Smith View|Parkway|Suite H|Concord|Wabasha County|MN|54107|United States|-6|single family| +9661|AAAAAAAANLFCAAAA|696|12th Broadway|Lane|Suite 320|Centerville|Tooele County|UT|80059|United States|-7|condo| +9662|AAAAAAAAOLFCAAAA|883|Eigth 10th|Way|Suite 120|Riverview|Carroll County|AR|79003|United States|-6|single family| +9663|AAAAAAAAPLFCAAAA|425|Railroad Pine|Way|Suite G|Maple Grove|Gallatin County|IL|68252|United States|-6|condo| +9664|AAAAAAAAAMFCAAAA|16|11th 14th|Way|Suite 70|Springfield|Centre County|PA|19303|United States|-5|single family| +9665|AAAAAAAABMFCAAAA||Lee Maple||Suite 450||Macon County|||United States|-5|| +9666|AAAAAAAACMFCAAAA|15|2nd |Way|Suite J|Mount Zion|Bourbon County|KS|68054|United States|-6|condo| +9667|AAAAAAAADMFCAAAA|661|Fourth Washington|Avenue|Suite 150|Brownsville|Stoddard County|MO|69310|United States|-6|apartment| +9668|AAAAAAAAEMFCAAAA|391|6th Poplar|Circle|Suite 300|Glendale|Shawano County|WI|53951|United States|-6|condo| +9669|AAAAAAAAFMFCAAAA|9|Main Lee|Ct.|Suite F|Bunker Hill|Essex County|VA|20150|United States|-5|condo| +9670|AAAAAAAAGMFCAAAA|440|Elm |||Franklin|||49101||-5|condo| +9671|AAAAAAAAHMFCAAAA|342|Adams |Lane|Suite 270|Mount Zion|Dale County|AL|38054|United States|-6|single family| +9672|AAAAAAAAIMFCAAAA|496|1st Railroad|Way|Suite L|Newtown|Elliott County|KY|41749|United States|-6|single family| +9673|AAAAAAAAJMFCAAAA|232|Pine |Street|Suite C|Georgetown|Madison County|FL|37057|United States|-5|single family| +9674|AAAAAAAAKMFCAAAA|310|13th Mill|Avenue|Suite K|Lakeside|Rice County|MN|59532|United States|-6|single family| +9675|AAAAAAAALMFCAAAA|974|8th |Lane|Suite 340|Hopewell|Wilson County|NC|20587|United States|-5|single family| +9676|AAAAAAAAMMFCAAAA|764|First 11th|Ave|Suite 410|Saint Clair|Columbia County|AR|75294|United States|-6|condo| +9677|AAAAAAAANMFCAAAA|436|River Railroad|RD|Suite M|Clinton|Somervell County|TX|78222|United States|-6|apartment| +9678|AAAAAAAAOMFCAAAA|315|Eigth |Road|Suite 330|Union|Morgan County|KY|48721|United States|-5|apartment| +9679|AAAAAAAAPMFCAAAA|640|Lake Cedar|Ln|Suite 390|Enterprise|Walworth County|WI|51757|United States|-6|condo| +9680|AAAAAAAAANFCAAAA|466|2nd |Way|Suite H|Summit|Lafayette Parish|LA|70499|United States|-6|apartment| +9681|AAAAAAAABNFCAAAA|369|Church |Ln|Suite 290|Oakwood|Jefferson County|CO|80169|United States|-7|condo| +9682|AAAAAAAACNFCAAAA|52||Boulevard|Suite 250|Newtown||||United States|-5|| +9683|AAAAAAAADNFCAAAA|107|Williams |Ct.|Suite O|Greenwood|Prairie County|AR|78828|United States|-6|single family| +9684|AAAAAAAAENFCAAAA|653|Walnut |Parkway|Suite J|Mount Olive|Castro County|TX|78059|United States|-6|apartment| +9685|AAAAAAAAFNFCAAAA|75|Fourth |Ct.|Suite E|Red Hill|Tyler County|WV|24338|United States|-5|single family| +9686|AAAAAAAAGNFCAAAA|18|1st |Cir.|Suite 190|Maple Grove|Queen Anne County|MD|28252|United States|-5|apartment| +9687|AAAAAAAAHNFCAAAA|304|Forest Dogwood|Wy|Suite X|Edgewood|Clark County|IN|40069|United States|-5|condo| +9688|AAAAAAAAINFCAAAA|746|3rd 3rd|RD|Suite 370|Sunnyside|Haskell County|KS|61952|United States|-6|condo| +9689|AAAAAAAAJNFCAAAA|489|Pine Smith|Dr.|Suite X|Bunker Hill|Switzerland County|IN|40150|United States|-5|condo| +9690|AAAAAAAAKNFCAAAA|203|West Ridge|Drive|Suite A|Sunnyside|Ashland County|OH|41952|United States|-5|apartment| +9691|AAAAAAAALNFCAAAA|286|Cedar |Ln|Suite B|Liberty|Juab County|UT|83451|United States|-7|single family| +9692|AAAAAAAAMNFCAAAA|913|Maple |Parkway|Suite 40|Springdale|Caldwell County|TX|78883|United States|-6|apartment| +9693|AAAAAAAANNFCAAAA|269|Twelfth |Circle|Suite 70|Jamestown|Rapides Parish|LA|76867|United States|-6|apartment| +9694|AAAAAAAAONFCAAAA|470|Washington 4th|Way|Suite 420|Unionville|Goochland County|VA|21711|United States|-5|condo| +9695|AAAAAAAAPNFCAAAA|940|Hillcrest |Circle|Suite I|Red Hill|Maricopa County|AZ|84338|United States|-7|single family| +9696|AAAAAAAAAOFCAAAA|412|Oak |Street|Suite 10|Lebanon|McDowell County|WV|22898|United States|-5|apartment| +9697|AAAAAAAABOFCAAAA|641|Oak 12th|Parkway|Suite C|Hillcrest|Concho County|TX|73003|United States|-6|single family| +9698|AAAAAAAACOFCAAAA|475|Hickory |Ln|Suite 220|Farmington|Tulsa County|OK|79145|United States|-6|single family| +9699|AAAAAAAADOFCAAAA|974|Jefferson Franklin|Ln|Suite 440|Saint James|Norton city|VA|25799|United States|-5|condo| +9700|AAAAAAAAEOFCAAAA|719|Third |Avenue|Suite 270|Saint James|Bath County|VA|25799|United States|-5|condo| +9701|AAAAAAAAFOFCAAAA|242|Spring |Road|Suite 170|Pine Grove|Lyman County|SD|54593|United States|-7|single family| +9702|AAAAAAAAGOFCAAAA|98|Birch Washington|Parkway|Suite Y|Greenwood|Louisa County|IA|58828|United States|-6|apartment| +9703|AAAAAAAAHOFCAAAA|277|Second Spruce|Lane|Suite 460|Spring Valley|Washington County|NC|26060|United States|-5|single family| +9704|AAAAAAAAIOFCAAAA|526|15th First|RD|Suite K|Clinton|Montague County|TX|78222|United States|-6|condo| +9705|AAAAAAAAJOFCAAAA|809|Chestnut 5th|Dr.|Suite 180|Kimball|Hot Spring County|AR|73595|United States|-6|single family| +9706|AAAAAAAAKOFCAAAA|387|Hillcrest |Parkway|Suite N|Woodlawn|Union County|IA|54098|United States|-6|apartment| +9707|AAAAAAAALOFCAAAA|39|Third |Avenue|Suite M|Pomona|Traill County|ND|54153|United States|-6|apartment| +9708|AAAAAAAAMOFCAAAA|462|Lincoln |Wy|Suite H|Shore Acres|Iosco County|MI|42724|United States|-5|single family| +9709|AAAAAAAANOFCAAAA|214|Jackson |Circle|Suite 170|Midway|Murray County|MN|51904|United States|-6|apartment| +9710|AAAAAAAAOOFCAAAA|698|Hill 11th|Drive|Suite W|Ashland|Nobles County|MN|54244|United States|-6|apartment| +9711|AAAAAAAAPOFCAAAA|742|South Johnson|Avenue|Suite E|Riverside|Lafayette County|FL|39231|United States|-5|condo| +9712|AAAAAAAAAPFCAAAA|128|Madison |Lane|Suite 60|Edgewood|Stephenson County|IL|60069|United States|-6|single family| +9713|AAAAAAAABPFCAAAA|44|Hill First|Way|Suite U|Parkwood|Nez Perce County|ID|81669|United States|-7|single family| +9714|AAAAAAAACPFCAAAA|911|9th Madison|Lane|Suite 390|Ashland|Jackson County|TN|34244|United States|-5|single family| +9715|AAAAAAAADPFCAAAA|592|Green Main|Dr.|Suite 250|Mount Vernon|Randall County|TX|78482|United States|-6|condo| +9716|AAAAAAAAEPFCAAAA|46|Cedar Davis|Drive|Suite 380|Cedar Grove|Roanoke County|VA|20411|United States|-5|single family| +9717|AAAAAAAAFPFCAAAA|463|1st Lake|Wy|Suite T|Fairview|Franklin County|NE|65709|United States|-6|condo| +9718|AAAAAAAAGPFCAAAA|812|South Chestnut|Blvd|Suite C|Blanchard|McCook County|SD|55985|United States|-7|single family| +9719|AAAAAAAAHPFCAAAA|42|1st Sunset|Lane|Suite G|Marion|Newton County|GA|30399|United States|-5|single family| +9720|AAAAAAAAIPFCAAAA|519|Cedar |Ln|Suite I||Van Wert County|OH|42812||-5|single family| +9721|AAAAAAAAJPFCAAAA|340|Pine |Court|Suite 260|Concord|Manatee County|FL|34107|United States|-5|single family| +9722|AAAAAAAAKPFCAAAA|78|Walnut |Court|Suite 280|King|Cass County|IN|40008|United States|-5|apartment| +9723|AAAAAAAALPFCAAAA|237|North |ST|Suite A|Midway|Kendall County|IL|61904|United States|-6|condo| +9724|AAAAAAAAMPFCAAAA|968|Oak |Drive|Suite H|Tabor|Menifee County|KY|48529|United States|-5|single family| +9725|AAAAAAAANPFCAAAA|6|7th Sycamore|Cir.|Suite 420|Ashland|Marlboro County|SC|24244|United States|-5|apartment| +9726|AAAAAAAAOPFCAAAA|919|Washington |Ln|Suite V|Highland Park|Nueces County|TX|76534|United States|-6|single family| +9727|AAAAAAAAPPFCAAAA|630|Spring Madison|RD|Suite U|Riverview|Colfax County|NE|69003|United States|-6|condo| +9728|AAAAAAAAAAGCAAAA|258|10th 3rd|Dr.|Suite 250|Union Hill|Grundy County|IA|57746|United States|-6|condo| +9729|AAAAAAAABAGCAAAA|212|Birch |Street|Suite B|Valley View|Moniteau County|MO|65124|United States|-6|apartment| +9730|AAAAAAAACAGCAAAA|574|Park Washington|Way|Suite U|Caledonia|Yellowstone County|MT|67411|United States|-7|condo| +9731|AAAAAAAADAGCAAAA|803|Spruce Dogwood|||New Hope||MT||United States|-7|apartment| +9732|AAAAAAAAEAGCAAAA|196|2nd |Drive|Suite 240|Kirkland|Jasper County|IN|47896|United States|-5|single family| +9733|AAAAAAAAFAGCAAAA|199|Ash |Pkwy|Suite A|Riverdale|Fillmore County|MN|59391|United States|-6|single family| +9734|AAAAAAAAGAGCAAAA|978|Lincoln 11th|Street|Suite H|Fairfield|Dubois County|IN|46192|United States|-5|condo| +9735|AAAAAAAAHAGCAAAA||Sycamore |Dr.||Shiloh||WV||United States|-5|apartment| +9736|AAAAAAAAIAGCAAAA|817|Park Wilson|Court|Suite 360|Jamestown|Powell County|KY|46867|United States|-5|apartment| +9737|AAAAAAAAJAGCAAAA|550|Oak 10th|Ct.|Suite L|Oakland|Greene County|AL|39843|United States|-6|condo| +9738|AAAAAAAAKAGCAAAA|98|Wilson |Wy|Suite M|Liberty|McKenzie County|ND|53451|United States|-6|apartment| +9739|AAAAAAAALAGCAAAA|680|Locust 15th|Blvd|Suite 240|Friendship|McIntosh County|OK|74536|United States|-6|apartment| +9740|AAAAAAAAMAGCAAAA|515|Second Central|Ave|Suite T|Farmington|Houston County|TN|39145|United States|-5|condo| +9741|AAAAAAAANAGCAAAA|443|Adams |Way|Suite V|Wildwood|Wichita County|KS|66871|United States|-6|apartment| +9742|AAAAAAAAOAGCAAAA|220|Laurel |Circle|Suite 260|Pine Grove|Rapides Parish|LA|74593|United States|-6|apartment| +9743|AAAAAAAAPAGCAAAA|475|Oak Laurel|Circle|Suite J|Woodlawn|Grundy County|MO|64098|United States|-6|single family| +9744|AAAAAAAAABGCAAAA|150|Oak |Parkway|Suite Q|Clinton|Aransas County|TX|78222|United States|-6|apartment| +9745|AAAAAAAABBGCAAAA|779|Washington |Court|Suite 450|Oakwood|Reagan County|TX|70169|United States|-6|single family| +9746|AAAAAAAACBGCAAAA|441|3rd |Dr.|Suite M|Mount Vernon|Moore County|NC|28482|United States|-5|condo| +9747|AAAAAAAADBGCAAAA|269|Maple Adams|Street|Suite 30|Chestnut Ridge|Talladega County|AL|37334|United States|-6|condo| +9748|AAAAAAAAEBGCAAAA|889|Jackson |Blvd|Suite 220|Jackson|Chickasaw County|IA|59583|United States|-6|condo| +9749|AAAAAAAAFBGCAAAA|400|3rd |Way|Suite J|Woodland|Anderson County|TX|74854|United States|-6|single family| +9750|AAAAAAAAGBGCAAAA|591|7th Park|Way|Suite G|Waterloo|Forest County|WI|51675|United States|-6|apartment| +9751|AAAAAAAAHBGCAAAA|630|Dogwood Elm|Dr.|Suite 200|Woodlawn|Scotland County|MO|64098|United States|-6|apartment| +9752|AAAAAAAAIBGCAAAA|746|||Suite P|Carthage|Ascension Parish|LA||United States|-6|| +9753|AAAAAAAAJBGCAAAA|631|Mill Williams|Street|Suite S|Tremont|Westchester County|NY|19515|United States|-5|condo| +9754|AAAAAAAAKBGCAAAA|523|Ash |Ct.|Suite 410|White Oak|Westmoreland County|VA|26668|United States|-5|single family| +9755|AAAAAAAALBGCAAAA|888|8th |Circle|Suite 340|Ryan|Roberts County|SD|50525|United States|-7|condo| +9756|AAAAAAAAMBGCAAAA|50|Ridge |Drive|Suite 430|Centerville|Perry County|MO|60059|United States|-6|apartment| +9757|AAAAAAAANBGCAAAA|668|3rd Maple|ST|Suite I|Oakwood|Prince George County|MD|20169|United States|-5|condo| +9758|AAAAAAAAOBGCAAAA|936|6th Johnson|Avenue|Suite 490|Riverside|Meigs County|OH|49231|United States|-5|apartment| +9759|AAAAAAAAPBGCAAAA|738|6th Third|Pkwy|Suite 390|Spring Valley|Porter County|IN|46060|United States|-5|apartment| +9760|AAAAAAAAACGCAAAA|761|East |Ln|Suite 380|Lebanon|Wake County|NC|22898|United States|-5|condo| +9761|AAAAAAAABCGCAAAA|979|6th South|Drive|Suite 120|Oakdale|Buchanan County|VA|29584|United States|-5|apartment| +9762|AAAAAAAACCGCAAAA|731|1st Park|Dr.|Suite 200|Five Forks|Genesee County|MI|42293|United States|-5|single family| +9763|AAAAAAAADCGCAAAA|377|Madison Maple|Boulevard|Suite X|Waterloo|Saline County|NE|61675|United States|-7|condo| +9764|AAAAAAAAECGCAAAA|417|1st 1st|Dr.|Suite N|Macedonia|Chattooga County|GA|31087|United States|-5|apartment| +9765|AAAAAAAAFCGCAAAA|169|15th Center|Road|Suite O|Waterloo|Mesa County|CO|81675|United States|-7|condo| +9766|AAAAAAAAGCGCAAAA|272|7th Park|Lane|Suite 90|Shiloh|Livingston Parish|LA|79275|United States|-6|condo| +9767|AAAAAAAAHCGCAAAA|725|First |Ave|Suite 430|Marion|Ottawa County|MI|40399|United States|-5|condo| +9768|AAAAAAAAICGCAAAA|67|6th Valley|Drive|Suite C|Montezuma|Kingman County|KS|62150|United States|-6|condo| +9769|AAAAAAAAJCGCAAAA|675|Spring Lee|Way|Suite K|Walnut Grove|Madison County|MT|67752|United States|-7|apartment| +9770|AAAAAAAAKCGCAAAA|569|Sycamore Woodland|Parkway|Suite 310|Flint|Hardin County|KY|48909|United States|-6|condo| +9771|AAAAAAAALCGCAAAA|525|2nd Park|Wy|Suite A|Kingston|Bradford County|FL|34975|United States|-5|condo| +9772|AAAAAAAAMCGCAAAA|787|14th |Dr.|Suite K|Mount Pleasant|Sumner County|KS|61933|United States|-6|condo| +9773|AAAAAAAANCGCAAAA|631|||||Union County|NM||United States||| +9774|AAAAAAAAOCGCAAAA|116|Hill |Dr.|Suite 290|Whitesville|Nemaha County|KS|65903|United States|-6|apartment| +9775|AAAAAAAAPCGCAAAA|683|North |Drive|Suite 430|Liberty|Carter County|MO|63451|United States|-6|single family| +9776|AAAAAAAAADGCAAAA|131||Road||Hopewell|Winston County||50587|United States|-6|condo| +9777|AAAAAAAABDGCAAAA|886|Sycamore 7th|Avenue|Suite 300|Mountain View|Summers County|WV|24466|United States|-5|single family| +9778|AAAAAAAACDGCAAAA|681|Forest Lee|Court|Suite Y|Kingston|Comanche County|KS|64975|United States|-6|single family| +9779|AAAAAAAADDGCAAAA|604|Elevnth |Avenue|Suite 210|Sunnyside|Haskell County|OK|71952|United States|-6|apartment| +9780|AAAAAAAAEDGCAAAA|590|7th |Way|Suite H|Clinton|Jefferson County|OK|78222|United States|-6|apartment| +9781|AAAAAAAAFDGCAAAA|189|Main Chestnut|Pkwy|Suite C|Fairview|Racine County|WI|55709|United States|-6|condo| +9782|AAAAAAAAGDGCAAAA|692|Highland Lincoln|Ct.|Suite 370|Oak Ridge|Somerset County|ME|08971|United States|-5|condo| +9783|AAAAAAAAHDGCAAAA|395|1st Maple|Parkway|Suite W|Clifford|Knox County|MO|68164|United States|-6|apartment| +9784|AAAAAAAAIDGCAAAA|744|Forest Cherry|Pkwy|Suite M|Buena Vista|Rowan County|KY|45752|United States|-5|condo| +9785|AAAAAAAAJDGCAAAA|245|5th 6th|Court|Suite 440|Middletown|Lanier County|GA|31422|United States|-5|apartment| +9786|AAAAAAAAKDGCAAAA|383|Lincoln 13th|Lane|Suite 390|Arlington|Bowie County|TX|76557|United States|-6|single family| +9787|AAAAAAAALDGCAAAA|138|Cedar |Way|Suite 280|Oak Ridge|Sumter County|AL|38371|United States|-6|apartment| +9788|AAAAAAAAMDGCAAAA|610|Fourth Central|Drive|Suite V|Green Acres|Crosby County|TX|77683|United States|-6|condo| +9789|AAAAAAAANDGCAAAA|404|1st |Way|Suite 60|Midway|Yuma County|AZ|81904|United States|-7|apartment| +9790|AAAAAAAAODGCAAAA|426|Locust |Dr.|Suite R|Union|Grundy County|IA|58721|United States|-6|apartment| +9791|AAAAAAAAPDGCAAAA|162|Lincoln Washington|Cir.|Suite 370|Bunker Hill|Prentiss County|MS|50150|United States|-6|single family| +9792|AAAAAAAAAEGCAAAA|863|3rd Adams|Avenue|Suite K|Lone Pine|Alameda County|CA|97441|United States|-8|apartment| +9793|AAAAAAAABEGCAAAA|782|9th |Lane|Suite Q|Lakewood|Lincoln County|ME|09477|United States|-5|condo| +9794|AAAAAAAACEGCAAAA|264|Railroad |Dr.|Suite 150|Pinecrest|Ashland County|OH|49981|United States|-5|apartment| +9795|AAAAAAAADEGCAAAA|722|Ridge |Road|Suite 130|Ridgeville|Wadena County|MN|59306|United States|-6|apartment| +9796|AAAAAAAAEEGCAAAA|161|5th |Parkway|Suite 40|Riverside|Clay County|WV|29231|United States|-5|condo| +9797|AAAAAAAAFEGCAAAA|760|Ridge Main|Lane|Suite S|Woodville|Clay County|GA|34289|United States|-5|single family| +9798|AAAAAAAAGEGCAAAA|849|Church |Dr.|Suite C|Highland|Kosciusko County|IN|49454|United States|-5|single family| +9799|AAAAAAAAHEGCAAAA|714|Lake Meadow|Ave|Suite V|Buena Vista|Haywood County|NC|25752|United States|-5|condo| +9800|AAAAAAAAIEGCAAAA|15|Ridge |Avenue|Suite X|Riverside|Quitman County|MS|59231|United States|-6|single family| +9801|AAAAAAAAJEGCAAAA|180|Lake |Road|Suite T|Crossroads|Sumter County|SC|20534|United States|-5|apartment| +9802|AAAAAAAAKEGCAAAA|666|3rd |Way|Suite 260|Oak Hill|McCracken County|KY|47838|United States|-5|single family| +9803|AAAAAAAALEGCAAAA|21|Locust Elevnth|ST|Suite 290|Shady Grove|Wayne County|OH|42812|United States|-5|apartment| +9804|AAAAAAAAMEGCAAAA|725|Smith |Ct.|Suite V|Harmony|Noble County|OK|75804|United States|-6|single family| +9805|AAAAAAAANEGCAAAA|765|Lake 14th|RD|Suite V|Jackson|Seneca County|OH|49583|United States|-5|condo| +9806|AAAAAAAAOEGCAAAA|815|14th Sycamore|Street|Suite C|Lakeside|Kiowa County|OK|79532|United States|-6|single family| +9807|AAAAAAAAPEGCAAAA|281|Maple |Ln|Suite 20|Kingston|Lamar County|MS|54975|United States|-6|single family| +9808|AAAAAAAAAFGCAAAA|805|14th Center|Blvd|Suite 340|Shiloh|Benton County|MN|59275|United States|-6|single family| +9809|AAAAAAAABFGCAAAA|115|Railroad College|Blvd||Shiloh|Boone County|||United States||apartment| +9810|AAAAAAAACFGCAAAA|592|Sunset |RD|Suite I|Mount Pleasant|Sherburne County|MN|51933|United States|-6|single family| +9811|AAAAAAAADFGCAAAA|982|6th |Ave|Suite 70|Mount Pleasant|Sebastian County|AR|71933|United States|-6|apartment| +9812|AAAAAAAAEFGCAAAA|916|Pine 2nd|ST|Suite C|Oakdale|Mercer County|PA|19584|United States|-5|single family| +9813|AAAAAAAAFFGCAAAA|133|Mill Jefferson|Lane|Suite P|Highland|Sawyer County|WI|59454|United States|-6|apartment| +9814|AAAAAAAAGFGCAAAA|985|4th |Parkway|Suite 220|Farmington|Erie County|NY|19145|United States|-5|condo| +9815|AAAAAAAAHFGCAAAA|503|West |RD|Suite J|Ellsworth|Osage County|KS|65079|United States|-6|single family| +9816|AAAAAAAAIFGCAAAA|233|Elm Spring|Ln|Suite 200|Oakwood|Mineral County|WV|20169|United States|-5|single family| +9817|AAAAAAAAJFGCAAAA|451|Second Ninth|Ave|Suite A|Red Hill|Bucks County|PA|14338|United States|-5|apartment| +9818|AAAAAAAAKFGCAAAA|88|Oak Fourth|Court|Suite 440|Birmingham|Benton County|IA|53372|United States|-6|single family| +9819|AAAAAAAALFGCAAAA|888|West |RD|Suite N|Fairview|Lowndes County|MS|55709|United States|-6|single family| +9820|AAAAAAAAMFGCAAAA|754|Davis |Boulevard|Suite Y|Liberty|Jasper County|SC|23451|United States|-5|apartment| +9821|AAAAAAAANFGCAAAA|300|Willow Sycamore|Blvd|Suite 380|Liberty|Twiggs County|GA|33451|United States|-5|apartment| +9822|AAAAAAAAOFGCAAAA|947|Church Ash|ST|Suite K|Salem|Galveston County|TX|78048|United States|-6|single family| +9823|AAAAAAAAPFGCAAAA|417|Main Fourth|Lane|Suite K|Oakdale|Brunswick County|VA|29584|United States|-5|single family| +9824|AAAAAAAAAGGCAAAA|758|Sunset |Cir.|Suite 480|Highland|Pembina County|ND|59454|United States|-6|condo| +9825|AAAAAAAABGGCAAAA|846|Sixth Center|Road|Suite 290|Union|Webster County|IA|58721|United States|-6|single family| +9826|AAAAAAAACGGCAAAA|448|10th |Wy|Suite F|Belmont|Barnes County|ND|50191|United States|-6|apartment| +9827|AAAAAAAADGGCAAAA|165|Green |Lane|Suite 100|Wildwood|Lafayette Parish|LA|76871|United States|-6|apartment| +9828|AAAAAAAAEGGCAAAA|627|College |Street|Suite N|Macedonia|Marshall County|KS|61087|United States|-6|condo| +9829|AAAAAAAAFGGCAAAA|165|5th |Dr.|Suite 60|Oakdale|King George County|VA|29584|United States|-5|condo| +9830|AAAAAAAAGGGCAAAA|473|Walnut Dogwood|Cir.|Suite 20|Jackson|Rockdale County|GA|39583|United States|-5|single family| +9831|AAAAAAAAHGGCAAAA|47|Sycamore Fourth|Lane|Suite A|Stringtown|Onslow County|NC|20162|United States|-5|condo| +9832|AAAAAAAAIGGCAAAA|619|Highland |RD|Suite 70|White Oak|Grant County|SD|56668|United States|-6|single family| +9833|AAAAAAAAJGGCAAAA|538|Maple Jackson|Street|Suite 50|Harmony|Floyd County|GA|35804|United States|-5|apartment| +9834|AAAAAAAAKGGCAAAA|670|Meadow |Cir.|Suite 100|Springfield|Barton County|MO|69303|United States|-6|apartment| +9835|AAAAAAAALGGCAAAA|195|Locust |Way|Suite R|Lebanon|Harrison County|MS|52898|United States|-6|apartment| +9836|AAAAAAAAMGGCAAAA|463|Franklin |Blvd|Suite E|Lakeside|Oneida County|WI|59532|United States|-6|condo| +9837|AAAAAAAANGGCAAAA|668|Park 13th|RD|Suite 30|Pine Grove|Metcalfe County|KY|44593|United States|-5|single family| +9838|AAAAAAAAOGGCAAAA|864|Spring |Court|Suite X|Oak Grove|Power County|ID|88370|United States|-7|condo| +9839|AAAAAAAAPGGCAAAA|771|Dogwood Laurel|Blvd|Suite 470|Liberty|Bronx County|NY|13451|United States|-5|single family| +9840|AAAAAAAAAHGCAAAA|870|Ash Park|Ave|Suite 300|Lakeview|Harris County|GA|38579|United States|-5|single family| +9841|AAAAAAAABHGCAAAA|124|Center |Parkway|Suite 40|Oak Ridge|Cassia County|ID|88371|United States|-7|apartment| +9842|AAAAAAAACHGCAAAA|92|Smith |Blvd|Suite 400|Clifton|Wheeler County|OR|98014|United States|-8|single family| +9843|AAAAAAAADHGCAAAA|145|Park |RD|Suite Q|Farmington|Todd County|SD|59145|United States|-7|apartment| +9844|AAAAAAAAEHGCAAAA|453|Pine |Boulevard|Suite K|Springfield|Audrain County|MO|69303|United States|-6|single family| +9845|AAAAAAAAFHGCAAAA|559|Oak Locust|ST|Suite C|Deerfield|Haakon County|SD|59840|United States|-7|apartment| +9846|AAAAAAAAGHGCAAAA|621|Pine |Road|Suite 220|Pleasant Valley|Owen County|IN|42477|United States|-5|single family| +9847|AAAAAAAAHHGCAAAA|268|Chestnut Poplar|Ln|Suite P|Greenfield|Muskegon County|MI|45038|United States|-5|apartment| +9848|AAAAAAAAIHGCAAAA|374|West |Lane|Suite N|Newtown|Robertson County|TN|31749|United States|-6|condo| +9849|AAAAAAAAJHGCAAAA|363|Lincoln |Wy|Suite L|Crossroads|Yavapai County|AZ|80534|United States|-7|single family| +9850|AAAAAAAAKHGCAAAA|853|Walnut |RD|Suite 0|Springfield|Lexington County|SC|29303|United States|-5|apartment| +9851|AAAAAAAALHGCAAAA|125|Lee 8th|Avenue|Suite H|Mount Pleasant|Karnes County|TX|71933|United States|-6|apartment| +9852|AAAAAAAAMHGCAAAA|728|West Hillcrest|Ave|Suite 220|Oak Grove|Greene County|OH|48370|United States|-5|single family| +9853|AAAAAAAANHGCAAAA|609|Cherry 7th|Ln|Suite B|Riverview|Carroll County|IN|49003|United States|-5|single family| +9854|AAAAAAAAOHGCAAAA|698|Main |Street|Suite 10|Lincoln|Barton County|MO|61289|United States|-6|condo| +9855|AAAAAAAAPHGCAAAA|384|Mill |Ln|Suite V|Mount Zion|Swain County|NC|28054|United States|-5|single family| +9856|AAAAAAAAAIGCAAAA|101|First |Road|Suite G|Florence|Pulaski County|VA|23394|United States|-5|single family| +9857|AAAAAAAABIGCAAAA||||Suite 350|Pleasant Hill|Meade County|KS||United States||| +9858|AAAAAAAACIGCAAAA|505|Sixth Williams|Pkwy|Suite D|Walnut Grove|McKean County|PA|17752|United States|-5|single family| +9859|AAAAAAAADIGCAAAA|409|River |Street|Suite B|Pleasant Grove|Tioga County|NY|14136|United States|-5|apartment| +9860|AAAAAAAAEIGCAAAA|888|5th Second|Dr.|Suite 120|Sulphur Springs|Yadkin County|NC|28354|United States|-5|single family| +9861|AAAAAAAAFIGCAAAA|493|Park |Ln|Suite 160|Bridgeport|Lawrence County|AL|35817|United States|-6|single family| +9862|AAAAAAAAGIGCAAAA|419|Church Lincoln|Ct.|Suite T|Riverside|Ketchikan Gateway Borough|AK|99231|United States|-9|apartment| +9863|AAAAAAAAHIGCAAAA|741|Church |RD|Suite 310|Millbrook|Franklin County|KS|67529|United States|-6|apartment| +9864|AAAAAAAAIIGCAAAA|957|View Center|RD|Suite 270|Forest Hills|Aleutians West Census Area|AK|99237|United States|-9|condo| +9865|AAAAAAAAJIGCAAAA|67|Smith 8th|Wy|Suite C|Highland Park|Martin County|KY|46534|United States|-5|apartment| +9866|AAAAAAAAKIGCAAAA|898|2nd |Boulevard|Suite 30|Glendale|Phillips County|CO|83951|United States|-7|condo| +9867|AAAAAAAALIGCAAAA|283|Mill 4th|ST|Suite 80|Maple Grove|Hancock County|IA|58252|United States|-6|condo| +9868|AAAAAAAAMIGCAAAA|986|Spruce |Ln|Suite 400|Five Points|Richmond city|VA|26098|United States|-5|single family| +9869|AAAAAAAANIGCAAAA|258|Mill |Cir.|Suite 150|Edgewood|Polk County|NC|20069|United States|-5|condo| +9870|AAAAAAAAOIGCAAAA|520|Maple |Ave|Suite 190|Five Points|San Francisco County|CA|96098|United States|-8|single family| +9871|AAAAAAAAPIGCAAAA|948|3rd Spring|RD|Suite I|Sullivan|Henry County|IA|50451|United States|-6|apartment| +9872|AAAAAAAAAJGCAAAA|||RD|Suite 430||||74107|United States||condo| +9873|AAAAAAAABJGCAAAA|850|Meadow |RD|Suite O|Concord|Union County|KY|44107|United States|-5|single family| +9874|AAAAAAAACJGCAAAA|593|Lee |Ln|Suite M|Buena Vista|Geneva County|AL|35752|United States|-6|condo| +9875|AAAAAAAADJGCAAAA|283|Main |Court|Suite A|Fairview|Woodson County|KS|65709|United States|-6|single family| +9876|AAAAAAAAEJGCAAAA|887|Spring Washington|Parkway|Suite 70|Mount Olive|Lancaster County|NE|68059|United States|-7|single family| +9877|AAAAAAAAFJGCAAAA|545|Elevnth Main|Lane|Suite 160|Hamilton|Schuyler County|MO|62808|United States|-6|single family| +9878|AAAAAAAAGJGCAAAA|707|Mill |ST|Suite 80|Springfield|Steele County|MN|59303|United States|-6|single family| +9879|AAAAAAAAHJGCAAAA|||Drive|Suite 480||Bland County||24466|United States||apartment| +9880|AAAAAAAAIJGCAAAA|425|Washington |Street|Suite O|Union|Palm Beach County|FL|38721|United States|-5|single family| +9881|AAAAAAAAJJGCAAAA|628|Third |Street|Suite V|Five Points|Hot Springs County|WY|86098|United States|-7|single family| +9882|AAAAAAAAKJGCAAAA|338|Sunset |Drive|Suite Q|Fairfield|Monroe County|NY|16192|United States|-5|single family| +9883|AAAAAAAALJGCAAAA|436|Lake Hill|Dr.|Suite P|Bridgeport|Faribault County|MN|55817|United States|-6|condo| +9884|AAAAAAAAMJGCAAAA|21|Elm 2nd|Drive|Suite 220|White Oak|Freestone County|TX|76668|United States|-6|condo| +9885|AAAAAAAANJGCAAAA|600|First |Court|Suite J|Springfield|Stevens County|KS|69303|United States|-6|single family| +9886|AAAAAAAAOJGCAAAA|313|Oak |Blvd|Suite 350|Hillcrest|Deschutes County|OR|93003|United States|-8|single family| +9887|AAAAAAAAPJGCAAAA||||Suite I||Lake County|||United States|-7|single family| +9888|AAAAAAAAAKGCAAAA|231|Hickory Meadow|Blvd|Suite 320|Wayland|Carter County|KY|45115|United States|-6|condo| +9889|AAAAAAAABKGCAAAA|270|3rd 1st|Pkwy|Suite 140|Lincoln|De Witt County|IL|61289|United States|-6|single family| +9890|AAAAAAAACKGCAAAA|884|Ridge Maple|Blvd|Suite 80|Midway|Martin County|TX|71904|United States|-6|apartment| +9891|AAAAAAAADKGCAAAA|261|1st Hillcrest|Way|Suite 90|Bennett|Knox County|NE|61715|United States|-7|single family| +9892|AAAAAAAAEKGCAAAA|85|12th 11th|Ct.|Suite G|Union|Dukes County|MA|09321|United States|-5|condo| +9893|AAAAAAAAFKGCAAAA|||ST||Glendale|Ottawa County|OH|43951|United States|-5|single family| +9894|AAAAAAAAGKGCAAAA|89|2nd Poplar|Ave|Suite 240|Jamestown|Whatcom County|WA|96867|United States|-8|single family| +9895|AAAAAAAAHKGCAAAA|38|6th |Wy|Suite 250|Red Hill|Wayne County|TN|34338|United States|-6|single family| +9896|AAAAAAAAIKGCAAAA|697|Smith Oak|Pkwy|Suite 220|Spring Hill|Beaver County|OK|76787|United States|-6|condo| +9897|AAAAAAAAJKGCAAAA|283|Church Jackson|Ln|Suite 370|Riverside|Throckmorton County|TX|79231|United States|-6|single family| +9898|AAAAAAAAKKGCAAAA|680|Hickory Maple|Pkwy|Suite I|Mountain View|Wilson County|TX|74466|United States|-6|apartment| +9899|AAAAAAAALKGCAAAA|836|Woodland |Pkwy|Suite 70|Peru|Asotin County|WA|90302|United States|-8|condo| +9900|AAAAAAAAMKGCAAAA|490|West Main|Court|Suite C|Mount Vernon|Hardy County|WV|28482|United States|-5|condo| +9901|AAAAAAAANKGCAAAA|493|1st 3rd|RD|Suite A|White Hall|Roseau County|MN|56955|United States|-6|condo| +9902|AAAAAAAAOKGCAAAA|159|Lincoln 8th|Ave|Suite 100|Five Points|Henderson County|TX|76098|United States|-6|single family| +9903|AAAAAAAAPKGCAAAA|819|Ridge |Boulevard|Suite L|Clifton|Pike County|KY|48014|United States|-5|apartment| +9904|AAAAAAAAALGCAAAA|651|Laurel |Circle|Suite 410|Lone Oak|Lubbock County|TX|76893|United States|-6|apartment| +9905|AAAAAAAABLGCAAAA|690|1st |Way|Suite 340|Woodlawn|Montgomery County|MD|24098|United States|-5|condo| +9906|AAAAAAAACLGCAAAA|426|Fourth |Parkway|Suite K|Georgetown|Wheeler County|NE|67057|United States|-7|apartment| +9907|AAAAAAAADLGCAAAA|249|Poplar Green|Boulevard|Suite 360|Farmington|Hutchinson County|SD|59145|United States|-7|condo| +9908|AAAAAAAAELGCAAAA|682|College |Ct.|Suite O|Glenwood|Northampton County|NC|23511|United States|-5|single family| +9909|AAAAAAAAFLGCAAAA|959|Oak Williams|Circle|Suite T|Mount Pleasant|Pipestone County|MN|51933|United States|-6|apartment| +9910|AAAAAAAAGLGCAAAA|315|Jefferson 7th|RD|Suite S|Wildwood|Riley County|KS|66871|United States|-6|single family| +9911|AAAAAAAAHLGCAAAA|493|3rd |Circle|Suite B|Red Hill|Isle of Wight County|VA|24338|United States|-5|condo| +9912|AAAAAAAAILGCAAAA|767|Elm |Boulevard|Suite G|Lebanon|Kauai County|HI|92898|United States|-10|single family| +9913|AAAAAAAAJLGCAAAA|363|Smith |Circle|Suite 280|Valley View|Bourbon County|KY|45124|United States|-6|apartment| +9914|AAAAAAAAKLGCAAAA|530|Center Oak|Road|Suite D|Tanglewood|Cullman County|AL|38994|United States|-6|condo| +9915|AAAAAAAALLGCAAAA|516|View |Avenue|Suite 340|Saratoga|Brooke County|WV|22123|United States|-5|apartment| +9916|AAAAAAAAMLGCAAAA|200|Williams |Ct.|Suite W|Stringtown|Cheyenne County|NE|60162|United States|-6|apartment| +9917|AAAAAAAANLGCAAAA|70|Lincoln Willow|Blvd|Suite 250|Oakdale|Davie County|NC|29584|United States|-5|apartment| +9918|AAAAAAAAOLGCAAAA|908|5th 3rd|Dr.|Suite 210|Fairview|Knox County|ME|06309|United States|-5|single family| +9919|AAAAAAAAPLGCAAAA|65|Hillcrest |RD|Suite L|Red Hill|Grant County|KY|44338|United States|-6|apartment| +9920|AAAAAAAAAMGCAAAA|602|Forest 6th|Drive|Suite R|Red Hill|Mobile County|AL|34338|United States|-6|apartment| +9921|AAAAAAAABMGCAAAA|382|Center |Cir.|Suite Q|Vienna|Bollinger County|MO|65119|United States|-6|condo| +9922|AAAAAAAACMGCAAAA|644|Poplar |Parkway|Suite R|Williamsville|Vermillion County|IN|48754|United States|-5|single family| +9923|AAAAAAAADMGCAAAA|231|10th West|Way|Suite S|Highland|Clinton County|IA|59454|United States|-6|condo| +9924|AAAAAAAAEMGCAAAA|658|Hill Sunset|Lane|Suite E|Georgetown|Durham County|NC|27057|United States|-5|condo| +9925|AAAAAAAAFMGCAAAA|21|Center |Dr.|Suite Y|Oak Grove|Benton County|AR|78370|United States|-6|condo| +9926|AAAAAAAAGMGCAAAA|230|East |Way|Suite Y|Five Points|Bibb County|AL|36098|United States|-6|single family| +9927|AAAAAAAAHMGCAAAA|201|Smith 2nd|Cir.|Suite G|Mount Zion|Greensville County|VA|28054|United States|-5|condo| +9928|AAAAAAAAIMGCAAAA|954|Seventh |Circle|Suite 130|Lakeside|Mahaska County|IA|59532|United States|-6|condo| +9929|AAAAAAAAJMGCAAAA|773|5th Walnut|Parkway|Suite Q|Marion|Gaines County|TX|70399|United States|-6|apartment| +9930|AAAAAAAAKMGCAAAA|795|Locust |Lane||Tanglewood|Clark County|IL||||apartment| +9931|AAAAAAAALMGCAAAA|759|First Fourth|Ave|Suite 270|Clinton|Richmond County|VA|28222|United States|-5|apartment| +9932|AAAAAAAAMMGCAAAA|176|Railroad |Pkwy|Suite G|Hillcrest|Utah County|UT|83003|United States|-7|condo| +9933|AAAAAAAANMGCAAAA|724|Oak |Blvd|Suite 330|Newport|Middlesex County|MA|02121|United States|-5|apartment| +9934|AAAAAAAAOMGCAAAA|123|Elm Second|Avenue|Suite S|Waterloo|Moffat County|CO|81675|United States|-7|condo| +9935|AAAAAAAAPMGCAAAA|234|8th |Road|Suite 150|Belmont|Fayette County|WV|20191|United States|-5|condo| +9936|AAAAAAAAANGCAAAA|842|Maple |Circle|Suite 310|Fairview|Nicholas County|KY|45709|United States|-5|single family| +9937|AAAAAAAABNGCAAAA|774|Adams Highland|Blvd|Suite F|Guilford|Hamilton County|OH|44408|United States|-5|apartment| +9938|AAAAAAAACNGCAAAA|740|Spring |Pkwy|Suite 60|Ludlow|Centre County|PA|15566|United States|-5|single family| +9939|AAAAAAAADNGCAAAA|310||Ct.|Suite 250|Fairview|Lincoln County|||United States||single family| +9940|AAAAAAAAENGCAAAA|683|Cedar |Parkway|Suite K|Salem|Washington County|IL|68048|United States|-6|condo| +9941|AAAAAAAAFNGCAAAA|405|Cherry 3rd|Cir.|Suite 460|Plainview|Lewis County|ID|83683|United States|-7|condo| +9942|AAAAAAAAGNGCAAAA|314|Williams |ST|Suite G|Sulphur Springs|Warren County|MO|68354|United States|-6|single family| +9943|AAAAAAAAHNGCAAAA|590|Thirteenth View|ST|Suite O|Bunker Hill|Montgomery County|GA|30150|United States|-5|single family| +9944|AAAAAAAAINGCAAAA|948|Franklin Johnson|Avenue|Suite 0|Hopewell|Norfolk city|VA|20587|United States|-5|condo| +9945|AAAAAAAAJNGCAAAA|104|Johnson Cedar|ST|Suite D|Woodville|Montgomery County|KY|44289|United States|-5|single family| +9946|AAAAAAAAKNGCAAAA|562|Johnson Forest|Ave|Suite N|Greenfield|Logan County|CO|85038|United States|-7|condo| +9947|AAAAAAAALNGCAAAA|494|Highland West|Dr.|Suite G|White Oak|Pickens County|GA|36668|United States|-5|apartment| +9948|AAAAAAAAMNGCAAAA|398|View |Pkwy|Suite 390|Summit|Frontier County|NE|60499|United States|-6|apartment| +9949|AAAAAAAANNGCAAAA|325|Franklin Park|Dr.|Suite 230|Maple Grove|Goliad County|TX|78252|United States|-6|apartment| +9950|AAAAAAAAONGCAAAA|829|Elm Lakeview||||Sawyer County||56060|United States||condo| +9951|AAAAAAAAPNGCAAAA|773|North |Circle|Suite J|Greenville|Chickasaw County|IA|51387|United States|-6|condo| +9952|AAAAAAAAAOGCAAAA|525|Oak South|RD|Suite S|Five Forks|Snohomish County|WA|92293|United States|-8|single family| +9953|AAAAAAAABOGCAAAA|644|Main |Boulevard|Suite 40|Bridgeport|Warren County|IN|45817|United States|-5|condo| +9954|AAAAAAAACOGCAAAA|320|Center |Court|Suite 450|Lakeside|Logan County|AR|79532|United States|-6|condo| +9955|AAAAAAAADOGCAAAA|448|Davis Mill|Ln|Suite Q|Lakeview|Payette County|ID|88579|United States|-7|single family| +9956|AAAAAAAAEOGCAAAA|176|Washington |Parkway|Suite S|Highland|Bowie County|TX|79454|United States|-6|single family| +9957|AAAAAAAAFOGCAAAA|335|9th Lake|Court|Suite D|Doyle|Rains County|TX|78434|United States|-6|condo| +9958|AAAAAAAAGOGCAAAA|556|Sunset Ninth|ST|Suite 50|Mount Vernon|Lincoln County|SD|58482|United States|-7|condo| +9959|AAAAAAAAHOGCAAAA|760|Hill College|Cir.|Suite U|Franklin|Bailey County|TX|79101|United States|-6|apartment| +9960|AAAAAAAAIOGCAAAA|513|10th |Dr.|Suite 370|Four Points|Sharp County|AR|71216|United States|-6|condo| +9961|AAAAAAAAJOGCAAAA|493|West |Ct.|Suite 120|Five Forks|Allen County|KY|42293|United States|-6|condo| +9962|AAAAAAAAKOGCAAAA||Miller Lake|Parkway|Suite R|Woodville||KS|64289|United States|-6|| +9963|AAAAAAAALOGCAAAA|552|13th 2nd|Court|Suite 280|Doyle|Nelson County|VA|28434|United States|-5|condo| +9964|AAAAAAAAMOGCAAAA|840|Dogwood Railroad|Blvd|Suite V|Elba|Carlisle County|KY|40262|United States|-6|single family| +9965|AAAAAAAANOGCAAAA|755|Ninth Jefferson|Wy|Suite 340|Union Hill|Forsyth County|GA|37746|United States|-5|single family| +9966|AAAAAAAAOOGCAAAA|91|Lee Cherry|ST|Suite 300|Globe|Cherokee County|NC|29614|United States|-5|apartment| +9967|AAAAAAAAPOGCAAAA|103|Railroad |Wy|Suite L|Waterloo|Bergen County|NJ|02275|United States|-5|condo| +9968|AAAAAAAAAPGCAAAA|743|South Pine|ST|Suite 420|Waterloo|Allen County|IN|41675|United States|-5|single family| +9969|AAAAAAAABPGCAAAA|837|Pine Poplar|RD|Suite 110|Hamilton|Gibson County|IN|42808|United States|-5|apartment| +9970|AAAAAAAACPGCAAAA|827|Cherry Sycamore|Road|Suite I|Jackson|Curry County|OR|99583|United States|-8|apartment| +9971|AAAAAAAADPGCAAAA|203|Sixth |Dr.|Suite 460|Oakwood|Jeff Davis County|GA|30169|United States|-5|apartment| +9972|AAAAAAAAEPGCAAAA|698|Mill 1st|Road|Suite C|Newport|Baca County|CO|81521|United States|-7|condo| +9973|AAAAAAAAFPGCAAAA|439|4th Main|Wy|Suite 70|Saint Johns|Chautauqua County|NY|15717|United States|-5|single family| +9974|AAAAAAAAGPGCAAAA|408|Park 4th|Ave|Suite U|Doyle|Jewell County|KS|68434|United States|-6|condo| +9975|AAAAAAAAHPGCAAAA|162|Adams Railroad|Drive|Suite M|Bethel|Camp County|TX|75281|United States|-6|apartment| +9976|AAAAAAAAIPGCAAAA|705|Spring 8th|Lane|Suite Q|Deerfield|Little River County|AR|79840|United States|-6|condo| +9977|AAAAAAAAJPGCAAAA|883|Madison Park|Circle|Suite J|Clinton|Missaukee County|MI|48222|United States|-5|single family| +9978|AAAAAAAAKPGCAAAA|706|Lee Jackson|Lane|Suite 280|Oakwood|Davidson County|NC|20169|United States|-5|single family| +9979|AAAAAAAALPGCAAAA|859|East |Dr.|Suite 320|Bunker Hill|Jasper County|IL|60150|United States|-6|single family| +9980|AAAAAAAAMPGCAAAA|817|Elm 10th|Wy|Suite F|Franklin|Custer County|SD|59101|United States|-6|apartment| +9981|AAAAAAAANPGCAAAA|537|River |Ct.|Suite 480|Liberty|Wabaunsee County|KS|63451|United States|-6|apartment| +9982|AAAAAAAAOPGCAAAA|184|First Park|Ave|Suite 270|Spring Hill|Plumas County|CA|96787|United States|-8|apartment| +9983|AAAAAAAAPPGCAAAA|445|Williams Main|Ct.|Suite H|Edgewood|Vilas County|WI|50069|United States|-6|condo| +9984|AAAAAAAAAAHCAAAA|367|6th |Dr.|Suite G|Deerfield|Raleigh County|WV|29840|United States|-5|condo| +9985|AAAAAAAABAHCAAAA|590|Hickory |Blvd|Suite D|Red Hill|Greenwood County|SC|24338|United States|-5|single family| +9986|AAAAAAAACAHCAAAA|902|11th Cedar|Boulevard|Suite S|Maywood|Cheyenne County|NE|65681|United States|-6|condo| +9987|AAAAAAAADAHCAAAA|544|9th Central|Drive|Suite 80|Pleasant Hill|Gilpin County|CO|83604|United States|-7|apartment| +9988|AAAAAAAAEAHCAAAA|823|North Second|Way|Suite 230|New Town|Inyo County|CA|99634|United States|-8|condo| +9989|AAAAAAAAFAHCAAAA|289|Valley |Circle|Suite 210|Fairview|Clifton Forge city|VA|25709|United States|-5|apartment| +9990|AAAAAAAAGAHCAAAA|464|Ridge |Road|Suite 440|Macedonia|Bates County|MO|61087|United States|-6|apartment| +9991|AAAAAAAAHAHCAAAA|693|Ash |RD|Suite 170|Wilson|Van Buren County|AR|76971|United States|-6|condo| +9992|AAAAAAAAIAHCAAAA|532|13th 4th|Way|Suite 300|Liberty|Dallas County|AL|33451|United States|-6|condo| +9993|AAAAAAAAJAHCAAAA|190|Poplar West|Cir.|Suite F|Cedar Grove|Oglethorpe County|GA|30411|United States|-5|apartment| +9994|AAAAAAAAKAHCAAAA|593|Davis Center|Parkway|Suite I|Bethel|McPherson County|SD|55281|United States|-7|condo| +9995|AAAAAAAALAHCAAAA|256|Birch Hillcrest|Blvd|Suite 230|Hopewell|Sussex County|DE|10587|United States|-5|single family| +9996|AAAAAAAAMAHCAAAA|971|Third |Wy|Suite U|Lakeside|Gregory County|SD|59532|United States|-6|apartment| +9997|AAAAAAAANAHCAAAA|294|Center 3rd|Road|Suite 190|Lakeview|Webster County|KY|48579|United States|-5|single family| +9998|AAAAAAAAOAHCAAAA|534|Franklin 2nd|Boulevard|Suite 220|Woodville|Grant County|WI|54289|United States|-6|apartment| +9999|AAAAAAAAPAHCAAAA|983|Park |Way|Suite 40|Flatwoods|Hudspeth County|TX|74212|United States|-6|apartment| +10000|AAAAAAAAABHCAAAA|327|Fifth Meadow|Court|Suite 320|Glenwood|Martin County|NC|23511|United States|-5|single family| +10001|AAAAAAAABBHCAAAA|347|Highland Sunset|Way|Suite V|Greenville|Hill County|MT|61387|United States|-7|apartment| +10002|AAAAAAAACBHCAAAA|609|Smith |Parkway|Suite 230|Summit|Ferry County|WA|90499|United States|-8|single family| +10003|AAAAAAAADBHCAAAA|661|Spring View|Parkway|Suite 420|Ashland|Newton County|IN|44244|United States|-5|single family| +10004|AAAAAAAAEBHCAAAA|65|Second |Pkwy|Suite 20|White Oak|Baltimore city|MD|26668|United States|-5|single family| +10005|AAAAAAAAFBHCAAAA|575|Oak 15th|Street|Suite 80|Franklin|Franklin County|MA|09701|United States|-5|condo| +10006|AAAAAAAAGBHCAAAA|676|Pine 1st|Lane|Suite 480|Sunnyside|Gallatin County|MT|61952|United States|-7|single family| +10007|AAAAAAAAHBHCAAAA|869|Oak |Circle|Suite 40|Valley View|Monona County|IA|55124|United States|-6|apartment| +10008|AAAAAAAAIBHCAAAA|930|First View|Parkway|Suite 330|Woodlawn|Smith County|KS|64098|United States|-6|apartment| +10009|AAAAAAAAJBHCAAAA|199|Johnson Fifth|Street|Suite C|Plainview|Madison County|MO|63683|United States|-6|apartment| +10010|AAAAAAAAKBHCAAAA|692|Poplar 3rd|Ln|Suite A|Lincoln|Vilas County|WI|51289|United States|-6|apartment| +10011|AAAAAAAALBHCAAAA|581|6th Franklin|Pkwy|Suite 0|Oakwood|Lincoln County|MT|60169|United States|-7|condo| +10012|AAAAAAAAMBHCAAAA|894|12th Park|Cir.|Suite G|Green Acres|Nance County|NE|67683|United States|-7|apartment| +10013|AAAAAAAANBHCAAAA|23|Jefferson 1st|Ct.|Suite W|Mount Vernon|Hood County|TX|78482|United States|-6|single family| +10014|AAAAAAAAOBHCAAAA|757|Jefferson ||Suite 70||Sumter County|GA|38605|||| +10015|AAAAAAAAPBHCAAAA|255|Maple |Pkwy|Suite 450|New Hope|Hood River County|OR|99431|United States|-8|single family| +10016|AAAAAAAAACHCAAAA|753|First Broadway|Circle|Suite N|Sulphur Springs|Porter County|IN|48354|United States|-5|single family| +10017|AAAAAAAABCHCAAAA|257|4th |Lane|Suite 250|Maple Grove|Decatur County|IN|48252|United States|-5|single family| +10018|AAAAAAAACCHCAAAA|240|Main |Blvd|Suite 200|Sulphur Springs|Douglas County|GA|38354|United States|-5|apartment| +10019|AAAAAAAADCHCAAAA|133|8th Birch|Parkway|Suite C|Unionville|Mille Lacs County|MN|51711|United States|-6|single family| +10020|AAAAAAAAECHCAAAA|843|4th |Ln|Suite R|Salem|Newaygo County|MI|48048|United States|-5|condo| +10021|AAAAAAAAFCHCAAAA|996|6th Johnson|Parkway|Suite S|Concord|Blue Earth County|MN|54107|United States|-6|single family| +10022|AAAAAAAAGCHCAAAA|646|North |Ave|Suite W|Jamestown|Carter County|OK|76867|United States|-6|condo| +10023|AAAAAAAAHCHCAAAA|252|3rd |Parkway|Suite 460|Glendale|Dakota County|MN|53951|United States|-6|apartment| +10024|AAAAAAAAICHCAAAA|473|West |Ln|Suite 370|Mount Olive|Ferry County|WA|98059|United States|-8|condo| +10025|AAAAAAAAJCHCAAAA||||Suite 310||Terry County|||United States|-6|apartment| +10026|AAAAAAAAKCHCAAAA|698|Forest |Ave|Suite E|Springfield|Dawson County|TX|79303|United States|-6|condo| +10027|AAAAAAAALCHCAAAA||||Suite 30||McLennan County||76614|United States|-6|single family| +10028|AAAAAAAAMCHCAAAA|965|Meadow |ST|Suite N|Woodrow|Weston County|WY|84273|United States|-7|apartment| +10029|AAAAAAAANCHCAAAA|292|Oak |Street|Suite K|Clifton|Leon County|FL|38014|United States|-5|single family| +10030|AAAAAAAAOCHCAAAA|590|12th |Road|Suite 110|Greenwood|Lonoke County|AR|78828|United States|-6|single family| +10031|AAAAAAAAPCHCAAAA||Center View|Circle||Georgetown||||United States|-5|single family| +10032|AAAAAAAAADHCAAAA|454|Jefferson Pine|Circle|Suite 100|Sunnyside|Yates County|NY|11952|United States|-5|single family| +10033|AAAAAAAABDHCAAAA|875|Hillcrest |RD|Suite K|Oakdale|Early County|GA|39584|United States|-5|apartment| +10034|AAAAAAAACDHCAAAA|308|Willow |Avenue|Suite N|Oak Hill|Cameron County|PA|17838|United States|-5|condo| +10035|AAAAAAAADDHCAAAA|61|13th Adams|Street|Suite A|Littleton|Richland County|IL|66074|United States|-6|apartment| +10036|AAAAAAAAEDHCAAAA|107|Fifth Sunset|Lane|Suite 280|Clinton|Colfax County|NM|88222|United States|-7|condo| +10037|AAAAAAAAFDHCAAAA|74|4th Sycamore|Cir.|Suite 290|Bethel|Vilas County|WI|55281|United States|-6|condo| +10038|AAAAAAAAGDHCAAAA|377|Cedar |Boulevard|Suite C|Hopewell|Zapata County|TX|70587|United States|-6|apartment| +10039|AAAAAAAAHDHCAAAA|997|Oak |Boulevard|Suite 390|Arlington|McLeod County|MN|56557|United States|-6|apartment| +10040|AAAAAAAAIDHCAAAA|817|Washington North|Wy|Suite T|Five Points|Childress County|TX|76098|United States|-6|single family| +10041|AAAAAAAAJDHCAAAA|445|Elm |RD|Suite W|Point Pleasant|Hardin County|IL|64749|United States|-6|apartment| +10042|AAAAAAAAKDHCAAAA|491|Maple |Court|Suite R|Pleasant Hill|Marshall County|IN|43604|United States|-5|single family| +10043|AAAAAAAALDHCAAAA|111|Maple Oak|Way|Suite O|Buena Vista|Lexington County|SC|25752|United States|-5|apartment| +10044|AAAAAAAAMDHCAAAA||||Suite 290|Ashland||TX||||single family| +10045|AAAAAAAANDHCAAAA|907|Lee |Ln|Suite N|Carter|Plaquemines Parish|LA|70919|United States|-6|apartment| +10046|AAAAAAAAODHCAAAA|234|First |Street|Suite S|Bridgeport|Summit County|UT|85817|United States|-7|condo| +10047|AAAAAAAAPDHCAAAA|553|Meadow |Court|Suite N|Wildwood|Moody County|SD|56871|United States|-7|apartment| +10048|AAAAAAAAAEHCAAAA|277|Davis Jackson|Street|Suite J|Jamestown|Mineral County|NV|86867|United States|-8|condo| +10049|AAAAAAAABEHCAAAA|530|Church |Dr.|Suite 60|Spring Hill|Utah County|UT|86787|United States|-7|apartment| +10050|AAAAAAAACEHCAAAA|515|Fourth Center|ST|Suite R|New Hope|Coffey County|KS|69431|United States|-6|condo| +10051|AAAAAAAADEHCAAAA|230|5th 5th|Blvd|Suite 340|California|Blaine County|OK|70141|United States|-6|apartment| +10052|AAAAAAAAEEHCAAAA|886|Main |Wy|Suite 20|Antioch|Jasper County|IA|58605|United States|-6|apartment| +10053|AAAAAAAAFEHCAAAA|136|Jackson Woodland|Parkway|Suite 180|Union|Buckingham County|VA|28721|United States|-5|apartment| +10054|AAAAAAAAGEHCAAAA|898|12th |Cir.|Suite T|Georgetown|Wayne County|WV|27057|United States|-5|single family| +10055|AAAAAAAAHEHCAAAA||Main 9th|||Deerfield|||69840||-7|| +10056|AAAAAAAAIEHCAAAA|636|Highland Ridge|Circle|Suite J|Belmont|Fergus County|MT|60191|United States|-7|single family| +10057|AAAAAAAAJEHCAAAA|573|Walnut Locust|Cir.|Suite C|Saint Clair|Dewey County|OK|75294|United States|-6|condo| +10058|AAAAAAAAKEHCAAAA|320|Hillcrest |Road|Suite 10|Kingston|Montgomery County|GA|34975|United States|-5|single family| +10059|AAAAAAAALEHCAAAA|109|6th |Way|Suite 390|Friendship|Hardeman County|TN|34536|United States|-5|condo| +10060|AAAAAAAAMEHCAAAA|121|Sycamore Second|Cir.|Suite 470|Harmony|Washington County|OR|95804|United States|-8|apartment| +10061|AAAAAAAANEHCAAAA|297|Poplar |Ave|Suite 90|Clinton|Bent County|CO|88222|United States|-7|single family| +10062|AAAAAAAAOEHCAAAA|512|5th 11th|Ave|Suite B|Hamilton|Madison County|IL|62808|United States|-6|single family| +10063|AAAAAAAAPEHCAAAA||Main Birch|Parkway|Suite 40|Buena Vista|Lincoln County|||||| +10064|AAAAAAAAAFHCAAAA|328|Lincoln |Street|Suite 310|Belmont|Jackson County|AR|70191|United States|-6|condo| +10065|AAAAAAAABFHCAAAA||Lake 2nd|||||||United States|-6|apartment| +10066|AAAAAAAACFHCAAAA|770|Ridge |Pkwy|Suite 300|Ashland|Lyon County|KY|44244|United States|-5|apartment| +10067|AAAAAAAADFHCAAAA|489|13th |Drive|Suite E|Lebanon|Campbell County|TN|32898|United States|-5|apartment| +10068|AAAAAAAAEFHCAAAA|397|Lake Willow|Parkway|Suite E|Kingston||||United States||single family| +10069|AAAAAAAAFFHCAAAA|264|Walnut |Drive|||Wabaunsee County||61904|United States||single family| +10070|AAAAAAAAGFHCAAAA|696|4th Mill|ST|Suite B|Sulphur Springs|Barnwell County|SC|28354|United States|-5|single family| +10071|AAAAAAAAHFHCAAAA|303|River Cedar|Street|Suite D|Mount Vernon|Pulaski County|KY|48482|United States|-5|single family| +10072|AAAAAAAAIFHCAAAA|417|Cherry |Parkway|Suite E|White Oak|Emmet County|MI|46668|United States|-5|condo| +10073|AAAAAAAAJFHCAAAA|147|10th Laurel|Avenue|Suite 0|Woodlawn|Anderson County|KS|64098|United States|-6|apartment| +10074|AAAAAAAAKFHCAAAA|494|Oak Thirteenth|Street|Suite 160|Greenfield|Live Oak County|TX|75038|United States|-6|single family| +10075|AAAAAAAALFHCAAAA|243|Spring Forest|Wy|Suite A|Riverside|Smith County|TN|39231|United States|-6|apartment| +10076|AAAAAAAAMFHCAAAA|306|Miller Jefferson|Dr.|Suite X|Globe|Carroll County|IN|49614|United States|-5|apartment| +10077|AAAAAAAANFHCAAAA|280|Park |Street|Suite 120|Cedar Grove|Jackson County|MO|60411|United States|-6|apartment| +10078|AAAAAAAAOFHCAAAA|241|Ash |Road|Suite 400|Bethel|Sussex County|DE|15281|United States|-5|condo| +10079|AAAAAAAAPFHCAAAA|195|Sunset |Ave|Suite W|Washington Heights|Meigs County|TN|38167|United States|-6|condo| +10080|AAAAAAAAAGHCAAAA|32|Sunset Fourth|Ln|Suite O|Greenfield|Benson County|ND|55038|United States|-6|condo| +10081|AAAAAAAABGHCAAAA|330|Forest Railroad|Boulevard|Suite M|Five Points|Polk County|GA|36098|United States|-5|apartment| +10082|AAAAAAAACGHCAAAA|169|View |Ave|Suite Y|Fairfield|Chippewa County|MN|56192|United States|-6|single family| +10083|AAAAAAAADGHCAAAA|286|Cherry |Drive|Suite S|Liberty|Rice County|MN|53451|United States|-6|apartment| +10084|AAAAAAAAEGHCAAAA|911|Pine |Street|Suite F|Welcome|Johnston County|OK|76386|United States|-6|apartment| +10085|AAAAAAAAFGHCAAAA|684|South Wilson|Street|Suite F|Mountain View|Lafourche Parish|LA|74466|United States|-6|apartment| +10086|AAAAAAAAGGHCAAAA|69|Lakeview |ST|Suite X|Sunnyside|Montcalm County|MI|41952|United States|-5|apartment| +10087|AAAAAAAAHGHCAAAA|897|North Spruce|Parkway|Suite 260|Five Points|Franklin County|FL|36098|United States|-5|apartment| +10088|AAAAAAAAIGHCAAAA|305|Railroad |Ct.|Suite T|Glenwood|Gallia County|OH|43511|United States|-5|single family| +10089|AAAAAAAAJGHCAAAA|789|Hillcrest |Ln|Suite Y|Oak Ridge|Kent County|DE|18371|United States|-5|condo| +10090|AAAAAAAAKGHCAAAA|384|Center |Blvd|Suite V|Howell|Hickman County|KY|44854|United States|-6|condo| +10091|AAAAAAAALGHCAAAA|20|6th Oak|Circle|Suite 170|Union|Morgan County|KY|48721|United States|-5|apartment| +10092|AAAAAAAAMGHCAAAA|15|View |Pkwy|Suite B|Riverside|Valdez-Cordova Census Area|AK|99231|United States|-9|single family| +10093|AAAAAAAANGHCAAAA||4th Elm||Suite E|||AZ||United States||single family| +10094|AAAAAAAAOGHCAAAA||5th |||Mount Olive||SC||||condo| +10095|AAAAAAAAPGHCAAAA|144|West 1st|Avenue|Suite 280|Oak Ridge|Steele County|MN|58371|United States|-6|condo| +10096|AAAAAAAAAHHCAAAA|749|10th 1st|Ave|Suite 420|Sutton|Mille Lacs County|MN|55413|United States|-6|single family| +10097|AAAAAAAABHHCAAAA|844|North |ST|Suite U|Lebanon|Butts County|GA|32898|United States|-5|single family| +10098|AAAAAAAACHHCAAAA|222|Johnson |Boulevard|Suite 400|Lakeside|Nelson County|KY|49532|United States|-5|condo| +10099|AAAAAAAADHHCAAAA|985|Woodland |Drive|Suite 450|Woodlawn|Clearwater County|MN|54098|United States|-6|single family| +10100|AAAAAAAAEHHCAAAA|28|Lee |Lane|Suite 460|Spring Valley|Posey County|IN|46060|United States|-5|apartment| +10101|AAAAAAAAFHHCAAAA|819|Sixth |Pkwy|Suite 100|Silver Creek|Matanuska-Susitna Borough|AK|94546|United States|-9|condo| +10102|AAAAAAAAGHHCAAAA|795|Davis Miller|Court|Suite 230|Antioch|Beauregard Parish|LA|78605|United States|-6|apartment| +10103|AAAAAAAAHHHCAAAA|373|Green Mill|Court|Suite L|Glendale|Harrison County|KY|43951|United States|-6|single family| +10104|AAAAAAAAIHHCAAAA|281|Pine 6th|Ct.|Suite G|Woodville|Howard County|MD|24289|United States|-5|condo| +10105|AAAAAAAAJHHCAAAA|908|Hill |Road|Suite 360|Springfield|DeKalb County|MO|69303|United States|-6|condo| +10106|AAAAAAAAKHHCAAAA|342|Willow |Ave|Suite 130|Franklin|Newton County|GA|39101|United States|-5|condo| +10107|AAAAAAAALHHCAAAA|890|Woodland Sixth|Parkway|Suite G|Forestville|McIntosh County|OK|73027|United States|-6|single family| +10108|AAAAAAAAMHHCAAAA|915|Lincoln Locust|Circle|Suite O|Shady Grove|Bear Lake County|ID|82812|United States|-7|condo| +10109|AAAAAAAANHHCAAAA|204|Oak |Drive|Suite B|Green Acres|Scotts Bluff County|NE|67683|United States|-7|single family| +10110|AAAAAAAAOHHCAAAA|398|13th |Wy|Suite F|Westgate|Boone County|IN|42366|United States|-5|apartment| +10111|AAAAAAAAPHHCAAAA|139|11th 7th|ST|Suite W|Summit|Stafford County|KS|60499|United States|-6|apartment| +10112|AAAAAAAAAIHCAAAA|444|Jackson |Dr.|Suite W|Lakeside|Clay County|IA|59532|United States|-6|apartment| +10113|AAAAAAAABIHCAAAA|539|Hickory |Blvd|Suite M|Glenwood|Northampton County|VA|23511|United States|-5|condo| +10114|AAAAAAAACIHCAAAA|81|Wilson |Ave|Suite M|Lakewood|Barton County|KS|68877|United States|-6|apartment| +10115|AAAAAAAADIHCAAAA|676|14th Valley|Dr.|Suite H|Bridgeport|Fannin County|GA|35817|United States|-5|single family| +10116|AAAAAAAAEIHCAAAA|520|2nd View|Court|Suite 110|Walnut Grove|Winkler County|TX|77752|United States|-6|condo| +10117|AAAAAAAAFIHCAAAA|453|Franklin |RD|Suite 40|Friendship|Gila County|AZ|84536|United States|-7|single family| +10118|AAAAAAAAGIHCAAAA|921|Sycamore |Way|Suite 160|Highland|Stone County|MS|59454|United States|-6|apartment| +10119|AAAAAAAAHIHCAAAA|487|Lake Birch|Boulevard|Suite 380|Jamestown|Douglas County|WI|56867|United States|-6|single family| +10120|AAAAAAAAIIHCAAAA|992|Park |Dr.|Suite I|Fairfield|Williamsburg County|SC|26192|United States|-5|condo| +10121|AAAAAAAAJIHCAAAA|603|Birch |Drive|Suite 220|Crossroads|Haywood County|NC|20534|United States|-5|condo| +10122|AAAAAAAAKIHCAAAA|440|3rd |Street|Suite J|Brownsville|Tallahatchie County|MS|59310|United States|-6|condo| +10123|AAAAAAAALIHCAAAA|57|Elm |Ave|Suite 40|Sulphur Springs|Itasca County|MN|58354|United States|-6|apartment| +10124|AAAAAAAAMIHCAAAA|593|Walnut |Cir.|Suite W|Cedar Grove|Franklin County|WA|90411|United States|-8|single family| +10125|AAAAAAAANIHCAAAA||Elm Willow|Cir.|Suite 40||Comanche County||68930|United States|-6|condo| +10126|AAAAAAAAOIHCAAAA|102|13th |ST|Suite C|Wilson|Kenosha County|WI|56971|United States|-6|apartment| +10127|AAAAAAAAPIHCAAAA|711|Lakeview Sunset|Boulevard|Suite 10|Springdale|Shenandoah County|VA|28883|United States|-5|single family| +10128|AAAAAAAAAJHCAAAA|393|5th Lee|Boulevard|Suite Q|Buena Vista|Monroe County|FL|35752|United States|-5|single family| +10129|AAAAAAAABJHCAAAA|702|Park |Ct.|Suite 290|Clifton|Griggs County|ND|58014|United States|-6|single family| +10130|AAAAAAAACJHCAAAA|539|Spring |Parkway|Suite O|Winslow|Shelby County|OH|48525|United States|-5|condo| +10131|AAAAAAAADJHCAAAA|669|Walnut |Lane|Suite K|Lakewood|Pulaski County|VA|28877|United States|-5|single family| +10132|AAAAAAAAEJHCAAAA|221|13th Park|Dr.|Suite X|Newtown|Stafford County|VA|21749|United States|-5|apartment| +10133|AAAAAAAAFJHCAAAA|974|2nd 5th|Ave|Suite 90|Oak Ridge|Platte County|MO|68371|United States|-6|condo| +10134|AAAAAAAAGJHCAAAA|930|Wilson Railroad|Avenue|Suite 150|Buena Vista|Perkins County|NE|65752|United States|-7|apartment| +10135|AAAAAAAAHJHCAAAA|386|6th 12th|Dr.|Suite 70|Pine Grove|Williamson County|IL|64593|United States|-6|single family| +10136|AAAAAAAAIJHCAAAA|176|Locust |Boulevard|Suite C|Cedar Grove|Mahaska County|IA|50411|United States|-6|apartment| +10137|AAAAAAAAJJHCAAAA|824|Highland 6th|Way|Suite 470|Florence|Haskell County|OK|73394|United States|-6|apartment| +10138|AAAAAAAAKJHCAAAA|346|First 9th|Lane|Suite 370|Carpenter|Spencer County|KY|41147|United States|-5|single family| +10139|AAAAAAAALJHCAAAA|20|Smith Lakeview|Ave|Suite I|Harmony|Essex County|VA|25804|United States|-5|apartment| +10140|AAAAAAAAMJHCAAAA|498|Spruce |Avenue|Suite 270|New Hope|Gilliam County|OR|99431|United States|-8|condo| +10141|AAAAAAAANJHCAAAA|676|Cedar |Road|Suite A|Hamilton|Baker County|OR|92808|United States|-8|single family| +10142|AAAAAAAAOJHCAAAA|875|Sycamore 10th|ST|Suite 170|Riverdale|Florence County|SC|29391|United States|-5|condo| +10143|AAAAAAAAPJHCAAAA|237|Spring Hillcrest|Avenue|Suite 430|Macedonia|Cheyenne County|CO|81087|United States|-7|condo| +10144|AAAAAAAAAKHCAAAA|323|Sunset |Ave|Suite 80|Florence|Boyd County|NE|63394|United States|-6|apartment| +10145|AAAAAAAABKHCAAAA|743|View River|Way|Suite X|Waterloo|Colleton County|SC|21675|United States|-5|condo| +10146|AAAAAAAACKHCAAAA|478|Main |Road|Suite D|Saint George|Pike County|AL|35281|United States|-6|single family| +10147|AAAAAAAADKHCAAAA|837|Lincoln 2nd|Circle|Suite H|Clifton|Woodruff County|AR|78014|United States|-6|single family| +10148|AAAAAAAAEKHCAAAA|256|Park Walnut|Lane|Suite X|Waterloo|Garrett County|MD|21675|United States|-5|single family| +10149|AAAAAAAAFKHCAAAA|261|Church |ST|Suite 250|Union|Clatsop County|OR|98721|United States|-8|apartment| +10150|AAAAAAAAGKHCAAAA|328|Lake Church|Ct.|Suite D|Fairfield|Union County|NC|26192|United States|-5|single family| +10151|AAAAAAAAHKHCAAAA|18|Birch |Parkway|Suite 30|Deerfield|Richland County|SC|29840|United States|-5|single family| +10152|AAAAAAAAIKHCAAAA|521|6th Lake|Dr.|Suite 190|Greenville|Chambers County|AL|31387|United States|-6|single family| +10153|AAAAAAAAJKHCAAAA|227|3rd |Ave|Suite 50|Five Forks|Coosa County|AL|32293|United States|-6|apartment| +10154|AAAAAAAAKKHCAAAA|484|Park |Ct.|Suite K|Belmont|Lake County|FL|30191|United States|-5|single family| +10155|AAAAAAAALKHCAAAA|353|4th North|ST|Suite 270|Newport|Lincoln County|WI|51521|United States|-6|single family| +10156|AAAAAAAAMKHCAAAA|777|First Jefferson|Parkway|Suite 10|Pleasant Valley|Linn County|MO|62477|United States|-6|single family| +10157|AAAAAAAANKHCAAAA|664|Cherry |Blvd|Suite 240|Woodbury|Susquehanna County|PA|14489|United States|-5|condo| +10158|AAAAAAAAOKHCAAAA|306|2nd Oak|Parkway|Suite C|Lakeville|Allen County|IN|48811|United States|-5|apartment| +10159|AAAAAAAAPKHCAAAA|840|Adams Walnut|RD|Suite 60|Franklin|Waldo County|ME|09701|United States|-5|condo| +10160|AAAAAAAAALHCAAAA|789|Willow Wilson|Avenue|Suite 240|Lakewood|Barnwell County|SC|28877|United States|-5|apartment| +10161|AAAAAAAABLHCAAAA|634|Park South|Circle|Suite A|Bayside|Howard County|AR|79550|United States|-6|condo| +10162|AAAAAAAACLHCAAAA|930|Fifth |Way|Suite 110|Farmington|Dare County|NC|29145|United States|-5|apartment| +10163|AAAAAAAADLHCAAAA|335|Hickory |Wy|Suite 490|Harmony|Fairfield County|CT|06404|United States|-5|single family| +10164|AAAAAAAAELHCAAAA|446|Poplar Park|Parkway|Suite 90|Lakeview|Hutchinson County|TX|78579|United States|-6|single family| +10165|AAAAAAAAFLHCAAAA|636|Lake |ST|Suite 480|Woodlawn|Jackson County|WV|24098|United States|-5|apartment| +10166|AAAAAAAAGLHCAAAA|908|Church |Wy|Suite 450|White Oak|Richmond County|GA|36668|United States|-5|single family| +10167|AAAAAAAAHLHCAAAA|437|View |Court|Suite V|Green Acres|Madison County|AR|77683|United States|-6|single family| +10168|AAAAAAAAILHCAAAA|527|River |Court|Suite 360|Riverview|Clallam County|WA|99003|United States|-8|condo| +10169|AAAAAAAAJLHCAAAA|746|4th 13th|Road|Suite G|Clifton|Owsley County|KY|48014|United States|-5|apartment| +10170|AAAAAAAAKLHCAAAA|215|8th Lincoln|Lane|Suite T|Greenfield|Woodbury County|IA|55038|United States|-6|single family| +10171|AAAAAAAALLHCAAAA|479|Railroad Park|ST|Suite 400|Lakeside|Ferry County|WA|99532|United States|-8|single family| +10172|AAAAAAAAMLHCAAAA|655|11th 1st|Cir.|Suite 240|Five Forks|Columbus County|NC|22293|United States|-5|single family| +10173|AAAAAAAANLHCAAAA|3|10th Spruce|Parkway|Suite U|Edgewood|DeBaca County|NM|80069|United States|-7|condo| +10174|AAAAAAAAOLHCAAAA||Sunset Woodland|||Sulphur Springs|Hocking County|OH|48354|||| +10175|AAAAAAAAPLHCAAAA|132|First |Parkway|Suite D|Brownsville|Shelby County|KY|49310|United States|-5|apartment| +10176|AAAAAAAAAMHCAAAA|191|Hickory |Road|Suite S|Shady Grove|Edwards County|KS|62812|United States|-6|condo| +10177|AAAAAAAABMHCAAAA|837|Lake 2nd|RD|Suite 450|White Oak|Dorchester County|SC|26668|United States|-5|condo| +10178|AAAAAAAACMHCAAAA|358|Laurel |Way|Suite 290|Oakdale|Cache County|UT|89584|United States|-7|single family| +10179|AAAAAAAADMHCAAAA|275|Meadow |Blvd|Suite B|Lebanon|Greer County|OK|72898|United States|-6|apartment| +10180|AAAAAAAAEMHCAAAA|759|Pine 5th|ST|Suite X|Brownsville|Webster County|WV|29310|United States|-5|condo| +10181|AAAAAAAAFMHCAAAA|50|Forest |Ave|Suite 260|Evans|Bath County|VA|22284|United States|-5|condo| +10182|AAAAAAAAGMHCAAAA|575||Wy|Suite B|||||United States||| +10183|AAAAAAAAHMHCAAAA|539|Park |Ln|Suite 350|Silver Creek|Sumter County|SC|24546|United States|-5|single family| +10184|AAAAAAAAIMHCAAAA|639|Railroad |ST|Suite 370|Fairfield|Nueces County|TX|76192|United States|-6|apartment| +10185|AAAAAAAAJMHCAAAA|27|Elm |Road||Belleview|Rockingham County|NC|20492|||apartment| +10186|AAAAAAAAKMHCAAAA|352|1st |Cir.|Suite W|Shady Grove|King William County|VA|22812|United States|-5|condo| +10187|AAAAAAAALMHCAAAA|118|Hickory |ST|Suite X|Maple Grove|Bonneville County|ID|88252|United States|-7|condo| +10188|AAAAAAAAMMHCAAAA|873|First 11th|Avenue|Suite N|Edgewood|Greenwood County|KS|60069|United States|-6|apartment| +10189|AAAAAAAANMHCAAAA|280|East Park|RD|Suite P|Union|Washington County|MD|28721|United States|-5|apartment| +10190|AAAAAAAAOMHCAAAA|921|Sunset Jackson|Circle|Suite I|Lincoln|Cuyahoga County|OH|41289|United States|-5|apartment| +10191|AAAAAAAAPMHCAAAA|361|12th |Court|Suite G|Unionville|Washington County|PA|11711|United States|-5|condo| +10192|AAAAAAAAANHCAAAA|599|River |Way|Suite 10|Oakwood|Prentiss County|MS|50169|United States|-6|condo| +10193|AAAAAAAABNHCAAAA|246|Washington West|Court|Suite S|Whitney|Harper County|OK|78339|United States|-6|single family| +10194|AAAAAAAACNHCAAAA|940|14th Green|ST||||||||apartment| +10195|AAAAAAAADNHCAAAA|317|Ash Maple|Pkwy|Suite A|Hamilton|Reno County|KS|62808|United States|-6|apartment| +10196|AAAAAAAAENHCAAAA|309|Spring |ST|Suite 20|Spring Hill|Meigs County|TN|36787|United States|-6|condo| +10197|AAAAAAAAFNHCAAAA|900|North Woodland|Boulevard|Suite 20|Howell|Montgomery County|AR|74854|United States|-6|condo| +10198|AAAAAAAAGNHCAAAA|162|Chestnut |Drive|Suite 10|Jamestown|Baldwin County|AL|36867|United States|-6|single family| +10199|AAAAAAAAHNHCAAAA|96|View 15th|Cir.|Suite 420|Hopewell|Reno County|KS|60587|United States|-6|single family| +10200|AAAAAAAAINHCAAAA|208|Valley 2nd|Parkway|Suite 190|Hillcrest|Saline County|AR|73003|United States|-6|apartment| +10201|AAAAAAAAJNHCAAAA|974|Walnut 3rd|ST|Suite N|Enterprise|Pawnee County|NE|61757|United States|-7|single family| +10202|AAAAAAAAKNHCAAAA|287|Willow |Ave|Suite W|Brookville|Sibley County|MN|53524|United States|-6|condo| +10203|AAAAAAAALNHCAAAA|326|View Spruce|Avenue|Suite 240|Fairview|Berkeley County|SC|25709|United States|-5|condo| +10204|AAAAAAAAMNHCAAAA|221|Woodland |Ln|Suite B|Hamilton|Woodruff County|AR|72808|United States|-6|single family| +10205|AAAAAAAANNHCAAAA|720|4th |RD|Suite 490|Buena Vista|Gilliam County|OR|95752|United States|-8|single family| +10206|AAAAAAAAONHCAAAA|880|Spruce Pine|Avenue|Suite 320|Belmont|Tompkins County|NY|10191|United States|-5|condo| +10207|AAAAAAAAPNHCAAAA|543|Cedar Washington|Street|Suite 300|Maple Grove|Henry County|OH|48252|United States|-5|condo| +10208|AAAAAAAAAOHCAAAA|48|Pine 10th|Court|Suite I|River Oaks|Niagara County|NY|18075|United States|-5|single family| +10209|AAAAAAAABOHCAAAA|85|Mill 10th|Ave|Suite 290|Woodlawn|Tarrant County|TX|74098|United States|-6|apartment| +10210|AAAAAAAACOHCAAAA|150|First 13th|Drive|Suite P|Oak Hill|Hendry County|FL|37838|United States|-5|apartment| +10211|AAAAAAAADOHCAAAA|129|Washington |Dr.|Suite 320|Union Hill|McHenry County|IL|67746|United States|-6|condo| +10212|AAAAAAAAEOHCAAAA|895|Lincoln South|Ct.|Suite 90|Clinton|Worcester County|MA|08822|United States|-5|single family| +10213|AAAAAAAAFOHCAAAA|644|Birch Spruce|RD|Suite 140|Riverdale|Monmouth County|NJ|09991|United States|-5|single family| +10214|AAAAAAAAGOHCAAAA|162|River |Wy|||Clay County|||United States||| +10215|AAAAAAAAHOHCAAAA|610|North |Blvd|Suite 240|Fairfield|White County|GA|36192|United States|-5|apartment| +10216|AAAAAAAAIOHCAAAA|863|Spruce Meadow|ST|Suite W|Woodville|Blanco County|TX|74289|United States|-6|condo| +10217|AAAAAAAAJOHCAAAA|30||||||||United States|-7|| +10218|AAAAAAAAKOHCAAAA|863|Thirteenth Hill|Way|Suite 230|Mount Olive|Forest County|PA|18059|United States|-5|single family| +10219|AAAAAAAALOHCAAAA|497|Pine |Way|Suite 440|Mount Zion|Gloucester County|NJ|08654|United States|-5|condo| +10220|AAAAAAAAMOHCAAAA|535|Spring |Lane|Suite 490|Cedar Grove|Mason County|WA|90411|United States|-8|apartment| +10221|AAAAAAAANOHCAAAA|41|North First|Dr.|Suite E|Willis|Sumner County|TN|36788|United States|-6|condo| +10222|AAAAAAAAOOHCAAAA|931|Cherry |Court|Suite 340|Spring Valley|Porter County|IN|46060|United States|-5|condo| +10223|AAAAAAAAPOHCAAAA|383|Elm Locust|Road|Suite C|Marion|San Miguel County|NM|80399|United States|-7|single family| +10224|AAAAAAAAAPHCAAAA|909|8th Seventh|Ct.|Suite 170|Oakdale|Neshoba County|MS|59584|United States|-6|single family| +10225|AAAAAAAABPHCAAAA|74|1st |Drive|Suite 240|Springdale|San Miguel County|CO|88883|United States|-7|apartment| +10226|AAAAAAAACPHCAAAA|328|College Elm|Avenue|Suite U|Arlington|Trego County|KS|66557|United States|-6|single family| +10227|AAAAAAAADPHCAAAA|574|Park |Parkway|Suite Q|Enterprise|Custer County|OK|71757|United States|-6|condo| +10228|AAAAAAAAEPHCAAAA|232|Third Cedar|Pkwy|Suite M|Hillcrest|Middlesex County|NJ|03603|United States|-5|apartment| +10229|AAAAAAAAFPHCAAAA|54|Adams Church|Court|Suite Y|Wayland|Judith Basin County|MT|65115|United States|-7|condo| +10230|AAAAAAAAGPHCAAAA|621|Walnut |RD|Suite L|Lakewood|Gaston County|NC|28877|United States|-5|condo| +10231|AAAAAAAAHPHCAAAA|89|Lincoln North|Ave|Suite 150|Forestville|O-Brien County|IA|53027|United States|-6|apartment| +10232|AAAAAAAAIPHCAAAA|794|Hill Elm|Lane|Suite 440|Edgewood|Johnson County|AR|70069|United States|-6|condo| +10233|AAAAAAAAJPHCAAAA|127|Johnson Broadway|Parkway|Suite 210|Lincoln|Montezuma County|CO|81289|United States|-7|apartment| +10234|AAAAAAAAKPHCAAAA|||Blvd|Suite 400|Sunnyside||GA|31952|United States|-5|single family| +10235|AAAAAAAALPHCAAAA|306|Park |RD|Suite H|Lakeside|Belmont County|OH|49532|United States|-5|condo| +10236|AAAAAAAAMPHCAAAA|491|View |Ct.|Suite 470|Patterson|DeBaca County|NM|83175|United States|-7|condo| +10237|AAAAAAAANPHCAAAA|223|Cedar |Court|Suite 230|Bayside|Lipscomb County|TX|79550|United States|-6|single family| +10238|AAAAAAAAOPHCAAAA|458|Cherry Wilson|RD|Suite L|Marion|Carter County|MO|60399|United States|-6|apartment| +10239|AAAAAAAAPPHCAAAA|565|11th |Blvd|Suite O|Lakeview|Concordia Parish|LA|78579|United States|-6|condo| +10240|AAAAAAAAAAICAAAA|330|North Cedar|Road|Suite 100|Oakdale|Parke County|IN|49584|United States|-5|single family| +10241|AAAAAAAABAICAAAA|636|Hillcrest |Parkway|Suite 240|New Hope|Hardin County|KY|49431|United States|-6|single family| +10242|AAAAAAAACAICAAAA|301|Fourth |Boulevard|Suite 470|Shelby|Rich County|UT|86575|United States|-7|condo| +10243|AAAAAAAADAICAAAA|454|College |Avenue|Suite 260|Allentown|Catoosa County|GA|31838|United States|-5|single family| +10244|AAAAAAAAEAICAAAA|22|11th Cedar|Parkway|Suite 260|Clinton|Hamblen County|TN|38222|United States|-5|single family| +10245|AAAAAAAAFAICAAAA|425|View |Way|Suite T|Sulphur Springs|Dillingham Census Area|AK|98354|United States|-9|condo| +10246|AAAAAAAAGAICAAAA|164|Cedar First|Pkwy|Suite 250|Greenfield|Jim Wells County|TX|75038|United States|-6|apartment| +10247|AAAAAAAAHAICAAAA||Elm Park|||Liberty|Sherburne County|MN|53451|||apartment| +10248|AAAAAAAAIAICAAAA|507|7th |Wy|Suite N|Wildwood|Jerauld County|SD|56871|United States|-7|single family| +10249|AAAAAAAAJAICAAAA|669|Washington |Blvd|Suite G|Green Acres|Cumberland County|NC|27683|United States|-5|apartment| +10250|AAAAAAAAKAICAAAA|386|||Suite S||Bryan County||76614||-6|single family| +10251|AAAAAAAALAICAAAA|87|Sunset 4th|Dr.|Suite T|Green Acres|Mercer County|WV|27683|United States|-5|condo| +10252|AAAAAAAAMAICAAAA|780|Second Elevnth|Way|Suite Y|Fairfield|Steele County|MN|56192|United States|-6|apartment| +10253|AAAAAAAANAICAAAA|251|Laurel Jefferson|Pkwy|Suite A|New Hope|Barry County|MO|69431|United States|-6|condo| +10254|AAAAAAAAOAICAAAA|398|Lincoln Mill|Dr.|Suite K|Clifton|Renville County|ND|58014|United States|-6|condo| +10255|AAAAAAAAPAICAAAA|922|Adams |Way|Suite 300|Harmony|Dodge County|GA|35804|United States|-5|single family| +10256|AAAAAAAAABICAAAA|881|9th Seventh|Ln|Suite V|Hopewell|Pike County|MO|60587|United States|-6|condo| +10257|AAAAAAAABBICAAAA|996|River |Ln|Suite 470|Spring Valley|Cecil County|MD|26060|United States|-5|apartment| +10258|AAAAAAAACBICAAAA|46|North |Wy|Suite O|Oakland|Morgan County|OH|49843|United States|-5|apartment| +10259|AAAAAAAADBICAAAA|616|9th 9th|Ln|Suite V|Spring Hill|Lee County|GA|36787|United States|-5|condo| +10260|AAAAAAAAEBICAAAA|429|3rd 1st|Avenue|Suite 340|Crossroads|Richmond County|NC|20534|United States|-5|single family| +10261|AAAAAAAAFBICAAAA|100|Oak View|Wy|Suite 320|Florence|Jennings County|IN|43394|United States|-5|condo| +10262|AAAAAAAAGBICAAAA|318|Jefferson |Lane|Suite 480|Fairfield|Adams County|IN|46192|United States|-5|condo| +10263|AAAAAAAAHBICAAAA|808|Sixth |Wy|Suite H|Antioch|Wise County|VA|28605|United States|-5|condo| +10264|AAAAAAAAIBICAAAA|||||||SC|24466|United States|-5|condo| +10265|AAAAAAAAJBICAAAA|660|View |Cir.|Suite 30|Jackson|Erie County|NY|19583|United States|-5|apartment| +10266|AAAAAAAAKBICAAAA|778|Cedar |Avenue|Suite D|Belfast|McLean County|ND|50125|United States|-6|condo| +10267|AAAAAAAALBICAAAA|781|Highland 10th|Ave|Suite K|White Oak|Polk County|NC|26668|United States|-5|condo| +10268|AAAAAAAAMBICAAAA|914|15th |Ave|Suite M|Oak Grove|Lucas County|IA|58370|United States|-6|condo| +10269|AAAAAAAANBICAAAA|617|5th |Cir.|Suite O|Valley View|Jackson County|AR|75124|United States|-6|condo| +10270|AAAAAAAAOBICAAAA|917|Dogwood |Court|Suite 110|Friendship|Washington County|KY|44536|United States|-5|single family| +10271|AAAAAAAAPBICAAAA|275|Laurel |Road|Suite 100|Fairview|Chisago County|MN|55709|United States|-6|apartment| +10272|AAAAAAAAACICAAAA|264|Pine |Street|Suite 360|Lincoln|Medina County|OH|41289|United States|-5|single family| +10273|AAAAAAAABCICAAAA|127|Main |Wy|Suite 120|Midway|Scott County|VA|21904|United States|-5|single family| +10274|AAAAAAAACCICAAAA|972|West Cedar|Boulevard|Suite S|Liberty|Jo Daviess County|IL|63451|United States|-6|apartment| +10275|AAAAAAAADCICAAAA|166|Main Third|RD|Suite 420|Mount Pleasant|Highland County|VA|21933|United States|-5|condo| +10276|AAAAAAAAECICAAAA|838|12th |Ave|Suite 440|Spring Hill|Butler County|AL|36787|United States|-6|single family| +10277|AAAAAAAAFCICAAAA|101|Johnson Hill|ST|Suite 460|Union|Buffalo County|WI|58721|United States|-6|apartment| +10278|AAAAAAAAGCICAAAA|540|3rd College|RD|Suite 260|Belmont|Rankin County|MS|50191|United States|-6|condo| +10279|AAAAAAAAHCICAAAA|395|Chestnut Second|RD|Suite 50|Kirkland|Benton County|IN|47896|United States|-5|single family| +10280|AAAAAAAAICICAAAA|807|Second First|Cir.|Suite 110|Five Points|Hall County|GA|36098|United States|-5|apartment| +10281|AAAAAAAAJCICAAAA|691|3rd |Circle|Suite 130|Oak Grove|Pratt County|KS|68370|United States|-6|single family| +10282|AAAAAAAAKCICAAAA|26|Spring |Ct.|Suite J|Green Acres|Oneida County|NY|17683|United States|-5|condo| +10283|AAAAAAAALCICAAAA|944|Williams |Parkway|Suite 290|Georgetown|Schley County|GA|37057|United States|-5|apartment| +10284|AAAAAAAAMCICAAAA|658|9th |Circle|Suite N|Sunnyside|Mitchell County|GA|31952|United States|-5|single family| +10285|AAAAAAAANCICAAAA|812|3rd Lincoln|RD|Suite J|Wildwood|Saunders County|NE|66871|United States|-7|apartment| +10286|AAAAAAAAOCICAAAA|949|Cedar |RD|Suite 0|Lakewood|Randolph County|IN|48877|United States|-5|apartment| +10287|AAAAAAAAPCICAAAA|832|10th |RD|Suite R|Woodville|Sheridan County|MT|64289|United States|-7|single family| +10288|AAAAAAAAADICAAAA|502|Fifth |Road|Suite 250|Lakewood|Nelson County|KY|48877|United States|-5|single family| +10289|AAAAAAAABDICAAAA|468|Lakeview |Cir.|Suite 110|Fisher|Telfair County|GA|32819|United States|-5|apartment| +10290|AAAAAAAACDICAAAA|213|13th |Dr.|Suite G|Springdale|Queens County|NY|18883|United States|-5|apartment| +10291|AAAAAAAADDICAAAA|927|Valley |RD|Suite 80|Concord|Victoria County|TX|74107|United States|-6|condo| +10292|AAAAAAAAEDICAAAA|783|Seventh |Street|Suite R|White Oak|Scott County|TN|36668|United States|-6|condo| +10293|AAAAAAAAFDICAAAA|524|Church Hickory|Circle|Suite O|Plainview|Wyoming County|WV|23683|United States|-5|apartment| +10294|AAAAAAAAGDICAAAA|486|Adams Lake|Parkway|Suite 90|Plainview|Tallahatchie County|MS|53683|United States|-6|single family| +10295|AAAAAAAAHDICAAAA|31|Davis |Ave|Suite Y|Glenville|Brunswick County|NC|23445|United States|-5|apartment| +10296|AAAAAAAAIDICAAAA|391|Maple |Blvd|Suite 170|Woodville|Coconino County|AZ|84289|United States|-7|condo| +10297|AAAAAAAAJDICAAAA|831|1st |Circle|Suite 80|Marion|Wheeler County|OR|90399|United States|-8|apartment| +10298|AAAAAAAAKDICAAAA|193|View Main|Wy|Suite 410|New Hope|Liberty County|TX|79431|United States|-6|condo| +10299|AAAAAAAALDICAAAA|887|4th Madison|Street|Suite W|Shady Grove|Atkinson County|GA|32812|United States|-5|apartment| +10300|AAAAAAAAMDICAAAA|792|Laurel Ninth|Ct.|Suite 390|Lakeside|Uintah County|UT|89532|United States|-7|condo| +10301|AAAAAAAANDICAAAA|381|Fifth |Street|Suite 310|Fairview|Iosco County|MI|45709|United States|-5|condo| +10302|AAAAAAAAODICAAAA|||Drive||Liberty||VA|23451|United States|-5|apartment| +10303|AAAAAAAAPDICAAAA|397|5th |Cir.|Suite D|Brownsville|Ozaukee County|WI|59310|United States|-6|single family| +10304|AAAAAAAAAEICAAAA|840|Johnson |Cir.|Suite 0|Maple Grove|Putnam County|OH|48252|United States|-5|condo| +10305|AAAAAAAABEICAAAA|746|North 12th|Blvd|Suite Q|Pleasant Valley|Chicot County|AR|72477|United States|-6|single family| +10306|AAAAAAAACEICAAAA|873|Pine |RD|Suite 460|Lakeside|Humboldt County|IA|59532|United States|-6|condo| +10307|AAAAAAAADEICAAAA|63|Lake Ash|Cir.|Suite 280|Green Acres|Schoharie County|NY|17683|United States|-5|single family| +10308|AAAAAAAAEEICAAAA|384|4th |Ct.|Suite I|Pleasant Grove|Wheeler County|TX|74136|United States|-6|single family| +10309|AAAAAAAAFEICAAAA|672|Fourth Railroad|Parkway|Suite D|Mountain View|Washoe County|NV|84466|United States|-8|apartment| +10310|AAAAAAAAGEICAAAA|482|View Lake|Boulevard|Suite R|Newport|Colquitt County|GA|31521|United States|-5|apartment| +10311|AAAAAAAAHEICAAAA|494|Wilson |Dr.|Suite 220|Pleasant Hill|Bucks County|PA|13604|United States|-5|single family| +10312|AAAAAAAAIEICAAAA|917|4th |Wy|Suite X|Waterloo|Yellow Medicine County|MN|51675|United States|-6|apartment| +10313|AAAAAAAAJEICAAAA|179|Main |Way|Suite Y|Sulphur Springs|Lincoln County|OR|98354|United States|-8|condo| +10314|AAAAAAAAKEICAAAA|169|Lee |ST|Suite 170|Kingston|Rains County|TX|74975|United States|-6|condo| +10315|AAAAAAAALEICAAAA|376|Davis Sycamore|ST|Suite 120|Enterprise|Jefferson County|MO|61757|United States|-6|condo| +10316|AAAAAAAAMEICAAAA|529|9th Washington|ST|Suite 400|Maple Grove|Adams County|IL|68252|United States|-6|apartment| +10317|AAAAAAAANEICAAAA|136|Dogwood Smith|Road|Suite P|Riverdale|Taylor County|KY|49391|United States|-5|condo| +10318|AAAAAAAAOEICAAAA|897|Sycamore Pine|Parkway|Suite M|Waterloo|Limestone County|AL|31675|United States|-6|apartment| +10319|AAAAAAAAPEICAAAA|892|Main Second|Way|Suite 430|Clifton|Boyd County|NE|68014|United States|-6|single family| +10320|AAAAAAAAAFICAAAA|865|First Maple|Dr.|Suite 150|Union Hill|Lapeer County|MI|47746|United States|-5|condo| +10321|AAAAAAAABFICAAAA|826|Elm |Dr.|Suite P|Sulphur Springs|Davidson County|NC|28354|United States|-5|single family| +10322|AAAAAAAACFICAAAA|489|Locust Locust|Ln|Suite 0|Lakeside|Pacific County|WA|99532|United States|-8|single family| +10323|AAAAAAAADFICAAAA|269|10th |Road|Suite M|Friendship|Culpeper County|VA|24536|United States|-5|condo| +10324|AAAAAAAAEFICAAAA|153|Park |Boulevard|Suite 90|Greenfield|Lincoln County|SD|55038|United States|-7|apartment| +10325|AAAAAAAAFFICAAAA|881|14th |RD|Suite W|Edgewood|Titus County|TX|70069|United States|-6|condo| +10326|AAAAAAAAGFICAAAA|691|6th |Avenue|Suite M|Sulphur Springs|Cass County|IA|58354|United States|-6|single family| +10327|AAAAAAAAHFICAAAA|307|Ninth 3rd|Lane|Suite J|Walnut Grove|Howard County|MD|27752|United States|-5|single family| +10328|AAAAAAAAIFICAAAA|65|3rd |Court|Suite C|Farmington|Columbia County|NY|19145|United States|-5|condo| +10329|AAAAAAAAJFICAAAA|274|||Suite H||||40587|United States||apartment| +10330|AAAAAAAAKFICAAAA|505|10th |Ct.|Suite G|Highland|Craig County|OK|79454|United States|-6|single family| +10331|AAAAAAAALFICAAAA|749|8th Lake|Road|Suite 100|Spring Hill|Gilmer County|GA|36787|United States|-5|condo| +10332|AAAAAAAAMFICAAAA|469|Sunset |Circle|Suite J|Greenwood|Garfield County|UT|88828|United States|-7|single family| +10333|AAAAAAAANFICAAAA|901|15th Jefferson||Suite 310|||MN||||single family| +10334|AAAAAAAAOFICAAAA|843|Central Meadow|RD|Suite 420|Maple Grove|Raleigh County|WV|28252|United States|-5|apartment| +10335|AAAAAAAAPFICAAAA|479|Pine 3rd|Avenue|Suite 50|Walnut Grove|Halifax County|VA|27752|United States|-5|apartment| +10336|AAAAAAAAAGICAAAA|447|Main |Dr.|Suite 20|Sunnyside|Rice County|MN|51952|United States|-6|condo| +10337|AAAAAAAABGICAAAA|136|First Park|RD|Suite 220|Clinton|Twin Falls County|ID|88222|United States|-7|single family| +10338|AAAAAAAACGICAAAA|243|Sixth South|Ln|Suite 440|Oakdale|San Francisco County|CA|99584|United States|-8|condo| +10339|AAAAAAAADGICAAAA|89|Pine Elevnth|Lane|Suite G|Clinton|Nelson County|KY|48222|United States|-5|condo| +10340|AAAAAAAAEGICAAAA|291|Park Franklin|Street|Suite 430|Oakdale|Hand County|SD|59584|United States|-7|condo| +10341|AAAAAAAAFGICAAAA|298|13th |Pkwy|Suite P|Newport|Jackson County|CO|81521|United States|-7|single family| +10342|AAAAAAAAGGICAAAA|576|3rd Washington|Way|Suite 180|Walnut Grove|Pierce County|ND|57752|United States|-6|condo| +10343|AAAAAAAAHGICAAAA|764|First |Court|Suite F|Silver Springs|Wood County|TX|74843|United States|-6|single family| +10344|AAAAAAAAIGICAAAA|646|10th |RD|Suite 250|Kingston|Cass County|NE|64975|United States|-6|single family| +10345|AAAAAAAAJGICAAAA|602|15th Lincoln|Avenue|Suite 330|Edgewood|Marion County|SC|20069|United States|-5|condo| +10346|AAAAAAAAKGICAAAA|185|East 3rd|Dr.|Suite G|Lakewood|Ziebach County|SD|58877|United States|-6|single family| +10347|AAAAAAAALGICAAAA|610|Second Jackson|Dr.|Suite 290|Carpenter|Skamania County|WA|91147|United States|-8|single family| +10348|AAAAAAAAMGICAAAA|121|2nd |Pkwy|Suite 290|Lincoln|Hancock County|GA|31289|United States|-5|condo| +10349|AAAAAAAANGICAAAA|625|3rd |Boulevard|Suite W|Springfield|Christian County|KY|49303|United States|-6|apartment| +10350|AAAAAAAAOGICAAAA|828|Park |Street|Suite A|Mountain View|Hancock County|IL|64466|United States|-6|condo| +10351|AAAAAAAAPGICAAAA|712|First |Cir.|Suite 350|Union|Grays Harbor County|WA|98721|United States|-8|condo| +10352|AAAAAAAAAHICAAAA|286|Jefferson Cherry|Parkway|Suite 490|Newport|Aroostook County|ME|02121|United States|-5|single family| +10353|AAAAAAAABHICAAAA|976|Franklin |Parkway|Suite H|Deerfield|Beaver County|UT|89840|United States|-7|single family| +10354|AAAAAAAACHICAAAA|610|First North|Avenue|Suite 400|Enterprise|Nicholas County|WV|21757|United States|-5|condo| +10355|AAAAAAAADHICAAAA|118|Laurel |Cir.|Suite M|Farmington|Livingston County|KY|49145|United States|-5|single family| +10356|AAAAAAAAEHICAAAA|190|Williams |Street|Suite 190|Pine Grove|Gasconade County|MO|64593|United States|-6|apartment| +10357|AAAAAAAAFHICAAAA|433|Broadway |Boulevard|Suite 460|Enterprise|Cleburne County|AR|71757|United States|-6|single family| +10358|AAAAAAAAGHICAAAA|2|Chestnut Hill|Court|Suite 350|Silver Creek|Jackson County|TN|34546|United States|-5|single family| +10359|AAAAAAAAHHICAAAA|825|Cedar |Ave|Suite V|Woodville|Erath County|TX|74289|United States|-6|apartment| +10360|AAAAAAAAIHICAAAA|489|6th Fourth|Blvd|Suite 150|Centerville|Pinal County|AZ|80059|United States|-7|single family| +10361|AAAAAAAAJHICAAAA|881|Meadow |Wy|Suite 200|Union|Bon Homme County|SD|58721|United States|-6|apartment| +10362|AAAAAAAAKHICAAAA|782|Park Second||Suite 410|Georgetown|Sitka Borough|AK||United States||apartment| +10363|AAAAAAAALHICAAAA|140|Fourth 9th|Drive|Suite W|Newtown|Pulaski County|MO|61749|United States|-6|apartment| +10364|AAAAAAAAMHICAAAA|728|Park 5th|Road|Suite Y|Harmony|Scurry County|TX|75804|United States|-6|condo| +10365|AAAAAAAANHICAAAA|153|Hickory |ST|Suite 280|Lincoln|Ogemaw County|MI|41289|United States|-5|condo| +10366|AAAAAAAAOHICAAAA|230|Main |Avenue|Suite F|Cedar Grove|Dutchess County|NY|10411|United States|-5|condo| +10367|AAAAAAAAPHICAAAA|781|5th Chestnut|Road|Suite 170|Sulphur Springs|Glasscock County|TX|78354|United States|-6|apartment| +10368|AAAAAAAAAIICAAAA|994|Tenth Oak|Pkwy|Suite G|Glenwood|Lake and Peninsula Borough|AK|93511|United States|-9|apartment| +10369|AAAAAAAABIICAAAA|859|Second Church|Pkwy|Suite L|Red Hill|Gulf County|FL|34338|United States|-5|apartment| +10370|AAAAAAAACIICAAAA|615|Main |RD|Suite 150|Red Hill|Norfolk County|MA|04938|United States|-5|apartment| +10371|AAAAAAAADIICAAAA|894|5th |Lane|Suite V|Riverside|DeWitt County|TX|79231|United States|-6|apartment| +10372|AAAAAAAAEIICAAAA|186|Willow Dogwood|Circle|Suite F|Colonial Heights|Cross County|AR|73425|United States|-6|condo| +10373|AAAAAAAAFIICAAAA|983|Miller |Lane|Suite H|Mount Vernon|Barnwell County|SC|28482|United States|-5|single family| +10374|AAAAAAAAGIICAAAA|209|Maple |Dr.|Suite N|Valley View|Laurel County|KY|45124|United States|-5|single family| +10375|AAAAAAAAHIICAAAA|718|Woodland |Court|Suite 240|Oakland|Charles City County|VA|29843|United States|-5|apartment| +10376|AAAAAAAAIIICAAAA|476||Ln||White Oak|||26668||-5|| +10377|AAAAAAAAJIICAAAA|450|First |Lane|Suite T|Pine Grove|Genesee County|MI|44593|United States|-5|single family| +10378|AAAAAAAAKIICAAAA|836|8th Miller|Way|Suite T|Concord|Woodward County|OK|74107|United States|-6|condo| +10379|AAAAAAAALIICAAAA|909|View |Blvd|Suite 300|Unionville|York County|SC|21711|United States|-5|single family| +10380|AAAAAAAAMIICAAAA|41|8th |Blvd|Suite W|Jamestown|Iroquois County|IL|66867|United States|-6|apartment| +10381|AAAAAAAANIICAAAA|472|Spruce |Dr.|Suite 340|Macedonia|Kern County|CA|91087|United States|-8|condo| +10382|AAAAAAAAOIICAAAA|891|Ridge 4th|Blvd|Suite 180|Concord|LaMoure County|ND|54107|United States|-6|condo| +10383|AAAAAAAAPIICAAAA|686|9th |Cir.|Suite S|Pomona|El Dorado County|CA|94153|United States|-8|condo| +10384|AAAAAAAAAJICAAAA|115|Madison |Ct.|Suite 180|Providence|Washington County|IL|66614|United States|-6|apartment| +10385|AAAAAAAABJICAAAA|904|Lake |Boulevard|Suite 30|Arlington|Grundy County|IL|66557|United States|-6|single family| +10386|AAAAAAAACJICAAAA|154|1st |Road|Suite 480|Belmont|Whatcom County|WA|90191|United States|-8|single family| +10387|AAAAAAAADJICAAAA|364|Elm Mill|Boulevard|Suite V|Salem|Putnam County|NY|18048|United States|-5|apartment| +10388|AAAAAAAAEJICAAAA||Ash |Lane|Suite 110|Newtown||OK||||| +10389|AAAAAAAAFJICAAAA|748|3rd |Court|Suite N|Glendale|Todd County|SD|53951|United States|-7|apartment| +10390|AAAAAAAAGJICAAAA|737|Elm |Pkwy|Suite 100|Forest Hills|Clay County|KS|69237|United States|-6|condo| +10391|AAAAAAAAHJICAAAA|46|Mill Hillcrest|Lane|Suite 280|Five Points|Rapides Parish|LA|76098|United States|-6|apartment| +10392|AAAAAAAAIJICAAAA|531|Poplar Lincoln|Ct.|Suite 470|Greenwood|Oswego County|NY|18828|United States|-5|condo| +10393|AAAAAAAAJJICAAAA|||Blvd|Suite X|Tanglewood|Unicoi County|TN||United States|-6|condo| +10394|AAAAAAAAKJICAAAA|225|4th |Wy|Suite 400|Antioch|Oneida County|ID|88605|United States|-7|single family| +10395|AAAAAAAALJICAAAA|248|East |Court|Suite J|Waterloo|Escambia County|FL|31675|United States|-5|condo| +10396|AAAAAAAAMJICAAAA|570|Ridge |Ct.|Suite 420|Farmington|McDowell County|NC|29145|United States|-5|single family| +10397|AAAAAAAANJICAAAA|||Parkway||||ND|||-6|| +10398|AAAAAAAAOJICAAAA|438|West 11th|Parkway|Suite W|Hillcrest|Trinity County|TX|73003|United States|-6|condo| +10399|AAAAAAAAPJICAAAA|943|First |Street|Suite 180|Gray|Washington County|CO|80468|United States|-7|condo| +10400|AAAAAAAAAKICAAAA|128|Third East|Boulevard|Suite 360|Roxbury|Floyd County|GA|35508|United States|-5|single family| +10401|AAAAAAAABKICAAAA|701|Pine |Ln|Suite 440|Greenville|Logan County|WV|21387|United States|-5|single family| +10402|AAAAAAAACKICAAAA|342|14th |Road|Suite 100|Edgewood|Logan County|OH|40069|United States|-5|single family| +10403|AAAAAAAADKICAAAA|119|West |Avenue|Suite 200|Salem|Ferry County|WA|98048|United States|-8|condo| +10404|AAAAAAAAEKICAAAA||River ||||Dillon County||26192|United States|-5|| +10405|AAAAAAAAFKICAAAA|715|Walnut |Ln|Suite A|Saint Clair|Rains County|TX|75294|United States|-6|apartment| +10406|AAAAAAAAGKICAAAA|707|Wilson 3rd|RD|Suite 440|Concord|Ouachita County|AR|74107|United States|-6|apartment| +10407|AAAAAAAAHKICAAAA|751|Dogwood Willow|ST|Suite 230|Franklin|Ottawa County|MI|49101|United States|-5|single family| +10408|AAAAAAAAIKICAAAA|26|8th |Wy|Suite Q|Sunnyside|Berrien County|MI|41952|United States|-5|single family| +10409|AAAAAAAAJKICAAAA|700|Lake |Lane|Suite 50|Franklin|Williams County|OH|49101|United States|-5|condo| +10410|AAAAAAAAKKICAAAA|798|11th |Ave|Suite 250|Rossville|Malheur County|OR|92503|United States|-8|condo| +10411|AAAAAAAALKICAAAA|647|4th Spring|Drive|Suite 140|Woodland|Clark County|WA|94854|United States|-8|apartment| +10412|AAAAAAAAMKICAAAA|14|Spring 2nd|Blvd|Suite T|Providence|Stephenson County|IL|66614|United States|-6|condo| +10413|AAAAAAAANKICAAAA|50|2nd Woodland|Pkwy|Suite 480|Oakwood|Hinds County|MS|50169|United States|-6|condo| +10414|AAAAAAAAOKICAAAA|523|East View|Ln|Suite J|Greenfield|Mountrail County|ND|55038|United States|-6|apartment| +10415|AAAAAAAAPKICAAAA|349|Adams |Road|Suite V|Union Hill|Colusa County|CA|97746|United States|-8|apartment| +10416|AAAAAAAAALICAAAA|833|Eigth Maple|Ave|Suite E|Hillcrest|Goshen County|WY|83003|United States|-7|single family| +10417|AAAAAAAABLICAAAA|265|Lake |Ave|Suite 100|Farmington|Fulton County|AR|79145|United States|-6|condo| +10418|AAAAAAAACLICAAAA|553|Hill Railroad|Drive|Suite 110|Glendale|Plymouth County|IA|53951|United States|-6|condo| +10419|AAAAAAAADLICAAAA|412|Cedar |Blvd|Suite M|Hillcrest|Jefferson County|TX|73003|United States|-6|single family| +10420|AAAAAAAAELICAAAA|174|Fourth 10th|Circle|Suite O|Argyle|McDowell County|WV|28722|United States|-5|apartment| +10421|AAAAAAAAFLICAAAA|140|11th 8th|Ct.|Suite 10|Friendship|Fayette County|AL|34536|United States|-6|apartment| +10422|AAAAAAAAGLICAAAA|956|1st |Ave|Suite 290|Kingston|Amite County|MS|54975|United States|-6|single family| +10423|AAAAAAAAHLICAAAA|337|15th 7th|ST|Suite N|Highland|Allen County|KY|49454|United States|-6|apartment| +10424|AAAAAAAAILICAAAA|667|Valley Meadow|Drive|Suite 70|Hopewell|Nottoway County|VA|20587|United States|-5|condo| +10425|AAAAAAAAJLICAAAA|746|Jackson |Circle|Suite 240|Oak Grove|McIntosh County|ND|58370|United States|-6|condo| +10426|AAAAAAAAKLICAAAA|783|3rd 7th|Avenue|Suite B|Stringtown|Orange County|VA|20162|United States|-5|apartment| +10427|AAAAAAAALLICAAAA|602|Madison Maple|Court|Suite 350|Five Points|Eddy County|NM|86098|United States|-7|single family| +10428|AAAAAAAAMLICAAAA|205|Meadow 13th|Street|Suite B|Greenwood|Hernando County|FL|38828|United States|-5|single family| +10429|AAAAAAAANLICAAAA|34|12th Walnut|Boulevard|Suite H|Woodlawn|Cambria County|PA|14098|United States|-5|single family| +10430|AAAAAAAAOLICAAAA|856|Washington 8th|ST|Suite 100|White Oak|Allendale County|SC|26668|United States|-5|single family| +10431|AAAAAAAAPLICAAAA|723|Ridge |Ct.|Suite 230|Liberty|Mercer County|NJ|04051|United States|-5|condo| +10432|AAAAAAAAAMICAAAA|487|5th |Drive|Suite A|Bethel|Noble County|OK|75281|United States|-6|condo| +10433|AAAAAAAABMICAAAA|442|Franklin Central|Circle|Suite 140|Belmont|Pepin County|WI|50191|United States|-6|condo| +10434|AAAAAAAACMICAAAA|521|15th Chestnut|Ln|Suite 70|Plainview|Newton County|IN|43683|United States|-5|condo| +10435|AAAAAAAADMICAAAA|236|East ||Suite A||Decatur County||62477|United States|-6|| +10436|AAAAAAAAEMICAAAA|606|Washington Cedar|Dr.|Suite A|Five Forks|Leake County|MS|52293|United States|-6|single family| +10437|AAAAAAAAFMICAAAA|446|Eigth |Ave|Suite 380|Five Forks|Lowndes County|AL|32293|United States|-6|condo| +10438|AAAAAAAAGMICAAAA|478|Hillcrest 9th|Cir.|Suite 190|Clifton|Racine County|WI|58014|United States|-6|condo| +10439|AAAAAAAAHMICAAAA|332|North |Circle|Suite 140|Union Hill|Jefferson County|IA|57746|United States|-6|single family| +10440|AAAAAAAAIMICAAAA|902|Laurel |Dr.|Suite 210|Pleasant Hill|Macon County|NC|23604|United States|-5|condo| +10441|AAAAAAAAJMICAAAA|24|Locust Spruce|Boulevard|Suite G|Sulphur Springs|Green County|KY|48354|United States|-6|single family| +10442|AAAAAAAAKMICAAAA|110|East Williams|Street|Suite 420|Paxton|Colorado County|TX|75669|United States|-6|single family| +10443|AAAAAAAALMICAAAA|15|Fourth Poplar|Street|Suite 280|Glenwood|Bell County|KY|43511|United States|-6|condo| +10444|AAAAAAAAMMICAAAA|302|Lee |Street|Suite H|Summit|Cabarrus County|NC|20499|United States|-5|single family| +10445|AAAAAAAANMICAAAA|589|Spruce Central|Circle|Suite X|Shiloh|Napa County|CA|99275|United States|-8|condo| +10446|AAAAAAAAOMICAAAA|752|Broadway Hill|Street|Suite I|Springfield|Cherokee County|NC|29303|United States|-5|apartment| +10447|AAAAAAAAPMICAAAA|873|Ash Fifth|||Nottingham|Lamar County|TX|74074|||single family| +10448|AAAAAAAAANICAAAA|515|Ridge 5th|Street|Suite A|Providence|Greenwood County|KS|66614|United States|-6|single family| +10449|AAAAAAAABNICAAAA|619|Park Washington|Ln|Suite U|Clinton|Logan County|IL|68222|United States|-6|apartment| +10450|AAAAAAAACNICAAAA|373|Elm |Lane|Suite 60|Walnut Grove|Grundy County|MO|67752|United States|-6|apartment| +10451|AAAAAAAADNICAAAA|997|Hill |Street|Suite 320|Bridgeport|Sanilac County|MI|45817|United States|-5|apartment| +10452|AAAAAAAAENICAAAA|228|Central 15th|Road|Suite 170|Centerville|Ramsey County|MN|50059|United States|-6|condo| +10453|AAAAAAAAFNICAAAA|312||Road|Suite Y|||GA||United States||single family| +10454|AAAAAAAAGNICAAAA|391|Pine |Street|Suite N|Florence|Allegany County|MD|23394|United States|-5|condo| +10455|AAAAAAAAHNICAAAA|567|Elevnth |Circle|Suite 400|Wilson|||46971|||| +10456|AAAAAAAAINICAAAA|470|West |Ave|Suite 430|Lakewood|Jackson County|IA|58877|United States|-6|single family| +10457|AAAAAAAAJNICAAAA|311|Cedar |Pkwy|Suite 270|Greenfield|Labette County|KS|65038|United States|-6|single family| +10458|AAAAAAAAKNICAAAA|863|Spruce Park|Pkwy|Suite 50|Mount Pleasant|Moffat County|CO|81933|United States|-7|condo| +10459|AAAAAAAALNICAAAA|454|Hickory 3rd|Ct.|Suite 80|Mount Pleasant|Lincoln County|NV|81933|United States|-8|single family| +10460|AAAAAAAAMNICAAAA|219|Miller Hill|Ln|Suite 410|Watkins|Spink County|SD|51732|United States|-7|condo| +10461|AAAAAAAANNICAAAA|379|Davis Franklin|ST|Suite 470|Lebanon|Conejos County|CO|82898|United States|-7|single family| +10462|AAAAAAAAONICAAAA|690|Broadway |Wy|Suite I|Liberty|Wagoner County|OK|73451|United States|-6|apartment| +10463|AAAAAAAAPNICAAAA|397|3rd |Ct.|Suite 350|Forest Hills|Cascade County|MT|69237|United States|-7|condo| +10464|AAAAAAAAAOICAAAA|186|Hickory |Cir.|Suite V|Buena Vista|Jefferson County|CO|85752|United States|-7|apartment| +10465|AAAAAAAABOICAAAA|669|Third Smith|Boulevard|Suite 40|Mount Zion|Bradford County|FL|38054|United States|-5|apartment| +10466|AAAAAAAACOICAAAA|63|Laurel Dogwood|Ave|Suite 300|Oak Hill|Kings County|CA|97838|United States|-8|condo| +10467|AAAAAAAADOICAAAA|877|Maple |Blvd|Suite 60|Waterloo|Ford County|KS|61675|United States|-6|condo| +10468|AAAAAAAAEOICAAAA|921|Lee |ST|Suite 410|Green Acres|Wirt County|WV|27683|United States|-5|apartment| +10469|AAAAAAAAFOICAAAA|947|Railroad Oak|Court|Suite T|Georgetown|Limestone County|AL|37057|United States|-6|single family| +10470|AAAAAAAAGOICAAAA|198|Hickory 5th|RD|Suite C|Fairview|Wheatland County|MT|65709|United States|-7|apartment| +10471|AAAAAAAAHOICAAAA|156|5th Hickory|Lane|Suite L|Jackson|Van Buren County|MI|49583|United States|-5|single family| +10472|AAAAAAAAIOICAAAA|59|4th Elm|Parkway|Suite 480|Forest Hills|Branch County|MI|49237|United States|-5|apartment| +10473|AAAAAAAAJOICAAAA|499|Birch First|Road|Suite U|Stringtown|Monroe County|NY|10162|United States|-5|condo| +10474|AAAAAAAAKOICAAAA|279|Hickory Willow|Lane|Suite 360|Cedar Grove|Mason County|IL|60411|United States|-6|apartment| +10475|AAAAAAAALOICAAAA|895|East Oak|Blvd|Suite 50|Clifton|Madison County|IA|58014|United States|-6|single family| +10476|AAAAAAAAMOICAAAA|724|Madison 6th|Ct.|Suite C|Marion|Fannin County|GA|30399|United States|-5|single family| +10477|AAAAAAAANOICAAAA|600|6th |Pkwy|Suite F|Franklin|Washington County|OH|49101|United States|-5|single family| +10478|AAAAAAAAOOICAAAA|61|College Maple|RD|Suite O|Macedonia|Coconino County|AZ|81087|United States|-7|condo| +10479|AAAAAAAAPOICAAAA|166|Mill |Parkway|Suite H|Summerville|Somerset County|MD|22033|United States|-5|condo| +10480|AAAAAAAAAPICAAAA|672|Railroad Laurel|RD|Suite E|Sulphur Springs|Rowan County|KY|48354|United States|-5|apartment| +10481|AAAAAAAABPICAAAA|854|Central View|Parkway|Suite R|Jamestown|Washoe County|NV|86867|United States|-8|apartment| +10482|AAAAAAAACPICAAAA|943|14th Park|Drive|Suite 50|Brookwood|Lincoln County|ME|01565|United States|-5|apartment| +10483|AAAAAAAADPICAAAA|976|Pine |Lane|Suite 70|Philadelphia|Dewey County|OK|75591|United States|-6|apartment| +10484|AAAAAAAAEPICAAAA|267|Third 8th|Pkwy|Suite 330|Five Points|Owsley County|KY|46098|United States|-5|condo| +10485|AAAAAAAAFPICAAAA|624|Hickory Pine|Lane|Suite D|Buena Vista|Hempstead County|AR|75752|United States|-6|single family| +10486|AAAAAAAAGPICAAAA|332|Hill |Cir.|Suite D|Brownsville|New Hanover County|NC|29310|United States|-5|condo| +10487|AAAAAAAAHPICAAAA|432|Sycamore Washington|Parkway|Suite 360|Mount Pleasant|Uinta County|WY|81933|United States|-7|condo| +10488|AAAAAAAAIPICAAAA|278|Lincoln |Avenue|Suite 80|Union Hill|Livingston County|IL|67746|United States|-6|condo| +10489|AAAAAAAAJPICAAAA|608|Meadow |Ln|Suite 420|Clifton|Lewis County|KY|48014|United States|-5|condo| +10490|AAAAAAAAKPICAAAA|955|||||Washita County||70399||-6|condo| +10491|AAAAAAAALPICAAAA|738|Hickory |Court|Suite C|Concord|Owsley County|KY|44107|United States|-5|single family| +10492|AAAAAAAAMPICAAAA|||Ct.|Suite N|Mount Pleasant||MT|61933|||single family| +10493|AAAAAAAANPICAAAA|524|Highland |Boulevard|Suite 0|Belmont|Stevens County|WA|90191|United States|-8|single family| +10494|AAAAAAAAOPICAAAA|803|7th Main|Cir.|Suite 450|Highland|White County|IL|69454|United States|-6|apartment| +10495|AAAAAAAAPPICAAAA|970|Woodland Jackson|Dr.|Suite F|Spring Valley|Butler County|PA|16060|United States|-5|apartment| +10496|AAAAAAAAAAJCAAAA|358|Davis |Dr.|Suite 250|Mount Zion|Hancock County|GA|38054|United States|-5|single family| +10497|AAAAAAAABAJCAAAA|3|12th |Wy|Suite 190|Shiloh|Pleasants County|WV|29275|United States|-5|apartment| +10498|AAAAAAAACAJCAAAA|765|Highland Hill|RD|Suite E|Edgewood|Montgomery County|IA|50069|United States|-6|single family| +10499|AAAAAAAADAJCAAAA|982|Railroad 13th|Way|Suite 450|Summerville|Transylvania County|NC|22033|United States|-5|apartment| +10500|AAAAAAAAEAJCAAAA|986|Second |Court|Suite T|Glenwood|Tunica County|MS|53511|United States|-6|apartment| +10501|AAAAAAAAFAJCAAAA|936|Highland |RD|Suite R|Sulphur Springs|Richland County|WI|58354|United States|-6|condo| +10502|AAAAAAAAGAJCAAAA|256|College |Avenue|Suite T|Mount Vernon|White County|GA|38482|United States|-5|condo| +10503|AAAAAAAAHAJCAAAA|873|Park |Avenue|Suite O|Wilson|Nowata County|OK|76971|United States|-6|apartment| +10504|AAAAAAAAIAJCAAAA|895|10th Hillcrest|Drive|Suite 30|Superior|Onslow County|NC|22562|United States|-5|single family| +10505|AAAAAAAAJAJCAAAA|134|East |Street|Suite F|Spring Valley|Lee County|IL|66060|United States|-6|condo| +10506|AAAAAAAAKAJCAAAA|337|Birch |Ln|Suite 350|Edgewood|Rains County|TX|70069|United States|-6|single family| +10507|AAAAAAAALAJCAAAA||14th Park|Dr.|Suite 170|Valley View|||||-5|| +10508|AAAAAAAAMAJCAAAA|369|Williams Fifth|Circle|Suite G|Mountain View|Oregon County|MO|64466|United States|-6|condo| +10509|AAAAAAAANAJCAAAA|733|8th |Blvd|Suite A|Hamilton|Lebanon County|PA|12808|United States|-5|apartment| +10510|AAAAAAAAOAJCAAAA|254|Meadow Park|Dr.|Suite 400|Brownsville|Yuma County|CO|89310|United States|-7|condo| +10511|AAAAAAAAPAJCAAAA|576|Franklin 9th|Wy|Suite R|New Hope|Koochiching County|MN|59431|United States|-6|single family| +10512|AAAAAAAAABJCAAAA|||Dr.||||OH|49310|||| +10513|AAAAAAAABBJCAAAA|605|Cedar |RD|Suite 260|Springdale|Garfield County|NE|68883|United States|-6|condo| +10514|AAAAAAAACBJCAAAA|457|South |Dr.|||Dallam County|TX||United States||condo| +10515|AAAAAAAADBJCAAAA|514|Woodland 13th|Wy|Suite P|Mount Zion|Sheridan County|ND|58054|United States|-6|apartment| +10516|AAAAAAAAEBJCAAAA|358|Second 7th|Court|Suite 470|Greenwood|Fulton County|PA|18828|United States|-5|condo| +10517|AAAAAAAAFBJCAAAA|442|View Oak|Circle|Suite 480|Summit|Webster County|GA|30499|United States|-5|condo| +10518|AAAAAAAAGBJCAAAA|446|Forest |Way|Suite D|Georgetown|Harrison County|KY|47057|United States|-6|condo| +10519|AAAAAAAAHBJCAAAA|823|4th Meadow|Drive|Suite I|Thompsonville|Kandiyohi County|MN|59651|United States|-6|apartment| +10520|AAAAAAAAIBJCAAAA|466|View |Wy|Suite 260|Midway|Monroe County|KY|41904|United States|-5|condo| +10521|AAAAAAAAJBJCAAAA|688|Hillcrest Main|Road|Suite 140|Washington Heights|Miller County|GA|38167|United States|-5|apartment| +10522|AAAAAAAAKBJCAAAA|351|Willow |Drive|Suite 110|Payne|Tishomingo County|MS|56134|United States|-6|single family| +10523|AAAAAAAALBJCAAAA|776|South Valley|Drive|Suite 350|Belmont|Green County|WI|50191|United States|-6|condo| +10524|AAAAAAAAMBJCAAAA|320|Main |Lane|Suite W|Ridgeville|Vermilion Parish|LA|79306|United States|-6|single family| +10525|AAAAAAAANBJCAAAA|600|Chestnut Railroad|Ct.|Suite 110|Shiloh|Highland County|OH|49275|United States|-5|single family| +10526|AAAAAAAAOBJCAAAA|495|Ridge |RD|Suite 300|Lakeview|Hancock County|KY|48579|United States|-6|single family| +10527|AAAAAAAAPBJCAAAA|594|River |Court|Suite J|Wilson|Monroe County|TN|36971|United States|-6|single family| +10528|AAAAAAAAACJCAAAA|876|Highland |ST|Suite 170|Ferndale|Stephens County|TX|72731|United States|-6|apartment| +10529|AAAAAAAABCJCAAAA|191|Spring |Road|Suite K|Belmont|Sutter County|CA|90191|United States|-8|single family| +10530|AAAAAAAACCJCAAAA|278|Lakeview |Ct.|Suite Q|Greenwood|Johnson County|KY|48828|United States|-6|single family| +10531|AAAAAAAADCJCAAAA|642|Franklin |Ave|Suite 420|Lincoln|Orleans County|NY|11289|United States|-5|single family| +10532|AAAAAAAAECJCAAAA|332|Forest |Ave|Suite 160|Antioch|Pulaski County|IN|48605|United States|-5|apartment| +10533|AAAAAAAAFCJCAAAA|35|Third |Wy|Suite K|Green Acres|Herkimer County|NY|17683|United States|-5|single family| +10534|AAAAAAAAGCJCAAAA|109|Tenth 4th|Cir.|Suite S|Shady Grove|Grundy County|TN|32812|United States|-5|apartment| +10535|AAAAAAAAHCJCAAAA|108|Central |Ln|Suite T|Liberty|Golden Valley County|ND|53451|United States|-6|apartment| +10536|AAAAAAAAICJCAAAA|409|Park 1st|Ave|Suite 80|Bridgeport|Baldwin County|GA|35817|United States|-5|single family| +10537|AAAAAAAAJCJCAAAA|508|First |Ct.|Suite 270|Oak Grove|Pembina County|ND|58370|United States|-6|single family| +10538|AAAAAAAAKCJCAAAA|797|3rd |Lane|Suite 40|Antioch|Mobile County|AL|38605|United States|-6|condo| +10539|AAAAAAAALCJCAAAA|405|1st |Boulevard|Suite R|Mount Zion|Parmer County|TX|78054|United States|-6|apartment| +10540|AAAAAAAAMCJCAAAA|793|Pine Spruce|Ln|Suite 150|Woodland|Dallas County|AL|34854|United States|-6|condo| +10541|AAAAAAAANCJCAAAA|142|1st |Avenue|Suite 220|Greenfield|Knox County|IN|45038|United States|-5|apartment| +10542|AAAAAAAAOCJCAAAA|376|Hillcrest |Ave|Suite 440|White Plains|Crockett County|TX|76622|United States|-6|apartment| +10543|AAAAAAAAPCJCAAAA|442|Lake View|Road|Suite 470|Walnut|Clarke County|MS|56245|United States|-6|apartment| +10544|AAAAAAAAADJCAAAA|63|4th |Ave|Suite X|Parkwood|Kandiyohi County|MN|51669|United States|-6|condo| +10545|AAAAAAAABDJCAAAA|326|2nd |RD|Suite 140|Lakeside|Valley County|MT|69532|United States|-7|apartment| +10546|AAAAAAAACDJCAAAA|179|Williams |Circle|Suite 0|Fairfield|Ionia County|MI|46192|United States|-5|condo| +10547|AAAAAAAADDJCAAAA|330|Elm 6th|Circle|Suite D|Five Forks|Gasconade County|MO|62293|United States|-6|condo| +10548|AAAAAAAAEDJCAAAA|344|Sixth |ST|Suite A||Oceana County||41854|||| +10549|AAAAAAAAFDJCAAAA|310|Elm |Ln|Suite 340|Valley View|Harford County|MD|25124|United States|-5|apartment| +10550|AAAAAAAAGDJCAAAA|631|Locust |ST|Suite B|Willow|New Castle County|DE|16798|United States|-5|apartment| +10551|AAAAAAAAHDJCAAAA|555|Second |Blvd|Suite 270|Belmont|Berkshire County|MA|00791|United States|-5|condo| +10552|AAAAAAAAIDJCAAAA|152|Lake |Court|Suite Q|Lakeside|Lee County|SC|29532|United States|-5|condo| +10553|AAAAAAAAJDJCAAAA|309|12th |Wy|Suite M|Woodland|Forsyth County|GA|34854|United States|-5|condo| +10554|AAAAAAAAKDJCAAAA|131|Third |Ave|Suite J|Green Acres|Green County|KY|47683|United States|-6|apartment| +10555|AAAAAAAALDJCAAAA|568|Second |Avenue|Suite 130|Fairview|Lemhi County|ID|85709|United States|-7|apartment| +10556|AAAAAAAAMDJCAAAA|447|River Madison|Wy|Suite 370|Fairview|Haywood County|NC|25709|United States|-5|single family| +10557|AAAAAAAANDJCAAAA|192|Jefferson Lee|Wy|Suite V|Highland|Crenshaw County|AL|39454|United States|-6|single family| +10558|AAAAAAAAODJCAAAA|280|Valley |Cir.|Suite N|Clinton|Fannin County|TX|78222|United States|-6|single family| +10559|AAAAAAAAPDJCAAAA|131|8th |Circle|Suite 260|Highland Park|Greene County|IN|46534|United States|-5|apartment| +10560|AAAAAAAAAEJCAAAA|576|Laurel |Ave|Suite M|Georgetown|Hancock County|WV|27057|United States|-5|apartment| +10561|AAAAAAAABEJCAAAA|24|East Third|Ave|Suite 150|Superior|Wakulla County|FL|32562|United States|-5|apartment| +10562|AAAAAAAACEJCAAAA|54|Maple |Dr.|Suite E|Springfield|Milam County|TX|79303|United States|-6|single family| +10563|AAAAAAAADEJCAAAA|211|4th Walnut|Ct.|Suite F|Pleasant Grove|Karnes County|TX|74136|United States|-6|apartment| +10564|AAAAAAAAEEJCAAAA|98|Pine |Ave|Suite 290|Mount Pleasant|Van Buren County|IA|51933|United States|-6|apartment| +10565|AAAAAAAAFEJCAAAA|42|Sycamore |Avenue|Suite I|Highland|Sangamon County|IL|69454|United States|-6|single family| +10566|AAAAAAAAGEJCAAAA|703|Ninth |Drive|Suite D|Centerville|Granville County|NC|20059|United States|-5|condo| +10567|AAAAAAAAHEJCAAAA|987|Center Jackson|Street|Suite K|Hillcrest|Philadelphia County|PA|13003|United States|-5|single family| +10568|AAAAAAAAIEJCAAAA|116|Park |Boulevard|Suite K|Deerfield|Webster County|IA|59840|United States|-6|single family| +10569|AAAAAAAAJEJCAAAA|101|West College|Ct.|Suite M|Union Hill|Providence County|RI|08346|United States|-5|single family| +10570|AAAAAAAAKEJCAAAA|558|3rd |Avenue|Suite A|Oakland|Walton County|FL|39843|United States|-5|single family| +10571|AAAAAAAALEJCAAAA|211|Central |Street|Suite 220|Florence|Caledonia County|VT|03994|United States|-5|apartment| +10572|AAAAAAAAMEJCAAAA|241|Fourth |Way|Suite 380|Spring Hill|Hood County|TX|76787|United States|-6|apartment| +10573|AAAAAAAANEJCAAAA|756|11th Sycamore|Ct.|Suite 480|Lakeview|New London County|CT|09179|United States|-5|apartment| +10574|AAAAAAAAOEJCAAAA|705|Cedar Highland|Street|Suite 370|King|Sangamon County|IL|60008|United States|-6|single family| +10575|AAAAAAAAPEJCAAAA|475|9th |Circle|Suite 310|Mount Zion|Olmsted County|MN|58054|United States|-6|condo| +10576|AAAAAAAAAFJCAAAA|937|Main 2nd|Ave|Suite P|Texas|Tuscaloosa County|AL|33342|United States|-6|condo| +10577|AAAAAAAABFJCAAAA|614|Oak |Cir.|Suite 450|Rolling Hills|Jefferson Davis Parish|LA|77272|United States|-6|condo| +10578|AAAAAAAACFJCAAAA|617|5th Hill|Court|Suite F|Rosewood|Madison County|IL|62205|United States|-6|single family| +10579|AAAAAAAADFJCAAAA|654|8th Pine|Parkway|Suite X|Saint Clair|Reno County|KS|65294|United States|-6|single family| +10580|AAAAAAAAEFJCAAAA|533|Maple Sixth|Ln|Suite 470|Lenox|Randolph County|IL|61143|United States|-6|single family| +10581|AAAAAAAAFFJCAAAA|20|Oak Birch|Avenue|Suite C|Wildwood|Allegany County|NY|16871|United States|-5|condo| +10582|AAAAAAAAGFJCAAAA|810|6th Cherry|Cir.|Suite 470|Winslow|Wyoming County|PA|18525|United States|-5|apartment| +10583|AAAAAAAAHFJCAAAA|239|Main Cherry|Drive|Suite N|Liberty|Stokes County|NC|23451|United States|-5|single family| +10584|AAAAAAAAIFJCAAAA|849|Fifth |||||MN|58877|United States||apartment| +10585|AAAAAAAAJFJCAAAA|102|4th 3rd|Circle|Suite Y|Waterloo|Hale County|TX|71675|United States|-6|condo| +10586|AAAAAAAAKFJCAAAA|500|Maple Main|Lane|Suite G|Deerfield|Hunterdon County|NJ|09840|United States|-5|condo| +10587|AAAAAAAALFJCAAAA|663|Walnut |RD|Suite 130|Wilson|Oldham County|KY|46971|United States|-5|single family| +10588|AAAAAAAAMFJCAAAA|516|1st |Ct.|Suite Q|Clinton|Marshall County|IL|68222|United States|-6|apartment| +10589|AAAAAAAANFJCAAAA|197|Poplar Hickory|Cir.|Suite J|Florence|Randolph County|WV|23394|United States|-5|single family| +10590|AAAAAAAAOFJCAAAA|147|North |Dr.|Suite 230|Kingston|Fauquier County|VA|24975|United States|-5|condo| +10591|AAAAAAAAPFJCAAAA|970|Lincoln |Avenue|Suite 300|Forest Hills|Carbon County|MT|69237|United States|-7|apartment| +10592|AAAAAAAAAGJCAAAA|550|Smith |Pkwy|Suite 310|Franklin|Crawford County|IA|59101|United States|-6|single family| +10593|AAAAAAAABGJCAAAA|463|9th |Pkwy|Suite I|Fairview|Ouachita County|AR|75709|United States|-6|apartment| +10594|AAAAAAAACGJCAAAA|22|8th |Pkwy|Suite 60|Woodville|Wexford County|MI|44289|United States|-5|condo| +10595|AAAAAAAADGJCAAAA|221|Maple Elm|Ct.|Suite 150|Highland Park|Suwannee County|FL|36534|United States|-5|condo| +10596|AAAAAAAAEGJCAAAA|32|Mill |Ct.|Suite Q|Wilson|Rich County|UT|86971|United States|-7|single family| +10597|AAAAAAAAFGJCAAAA|141|Fifth |Court|Suite 40|Newport|Clinton County|PA|11521|United States|-5|apartment| +10598|AAAAAAAAGGJCAAAA|162|Madison |Way|Suite 10|Woodville|Umatilla County|OR|94289|United States|-8|condo| +10599|AAAAAAAAHGJCAAAA|652|South Washington|Parkway|Suite 430|Pine Grove|Chattooga County|GA|34593|United States|-5|apartment| +10600|AAAAAAAAIGJCAAAA|524|North Ash|Parkway|Suite 180|Greenwood|Genesee County|NY|18828|United States|-5|single family| +10601|AAAAAAAAJGJCAAAA|11|Valley Fifth|Circle|Suite 410|Franklin|Centre County|PA|19101|United States|-5|condo| +10602|AAAAAAAAKGJCAAAA|728|Pine Valley|Ln|Suite 370|Wolf Creek|Berkshire County|MA|03055|United States|-5|apartment| +10603|AAAAAAAALGJCAAAA|226|2nd |Road|Suite 30|Post Oak|Colfax County|NE|68567|United States|-6|condo| +10604|AAAAAAAAMGJCAAAA|776|5th First|Wy|Suite X|Lakeside|Crawford County|IA|59532|United States|-6|single family| +10605|AAAAAAAANGJCAAAA|707|2nd |Circle|Suite A|Mount Olive|Waseca County|MN|58059|United States|-6|condo| +10606|AAAAAAAAOGJCAAAA|703|Pine |Wy|Suite J|Harmony|Pickens County|SC|25804|United States|-5|apartment| +10607|AAAAAAAAPGJCAAAA|226|Central Hickory|Blvd|Suite O|Florence|Anderson County|KY|43394|United States|-6|apartment| +10608|AAAAAAAAAHJCAAAA|446|3rd Cedar|Court|Suite J|Pleasant Hill|Yolo County|CA|93604|United States|-8|condo| +10609|AAAAAAAABHJCAAAA|761|Cherry |Ave|Suite V|Jackson|Decatur County|IN|49583|United States|-5|condo| +10610|AAAAAAAACHJCAAAA|985|Cedar Oak|Pkwy|Suite 300|Elizabeth|Burnett County|WI|52935|United States|-6|single family| +10611|AAAAAAAADHJCAAAA|144|10th |Ct.|Suite Q|Oak Grove|Frederick County|MD|28370|United States|-5|single family| +10612|AAAAAAAAEHJCAAAA|349|Third Maple|Ave|Suite 30|Harmony|Livingston County|NY|15804|United States|-5|condo| +10613|AAAAAAAAFHJCAAAA|806|Walnut |Street|Suite D|Bethel|Polk County|AR|75281|United States|-6|single family| +10614|AAAAAAAAGHJCAAAA|134|Park |RD|Suite 310|Franklin|Hamilton County|IN|49101|United States|-5|apartment| +10615|AAAAAAAAHHJCAAAA|388|Cedar |Blvd|Suite E|Five Points|Storey County|NV|86098|United States|-8|apartment| +10616|AAAAAAAAIHJCAAAA|71|Elm Lee|Street|Suite R|Springfield|Custer County|NE|69303|United States|-6|apartment| +10617|AAAAAAAAJHJCAAAA|856|Main Church|Avenue|Suite 0|Oakdale|Wyoming County|WV|29584|United States|-5|single family| +10618|AAAAAAAAKHJCAAAA|218|Railroad Main|Street|Suite 340|Spring Hill|Arenac County|MI|46787|United States|-5|apartment| +10619|AAAAAAAALHJCAAAA|701|Park |Street|Suite 480|Woodville|Lucas County|OH|44289|United States|-5|single family| +10620|AAAAAAAAMHJCAAAA|172|4th Mill|Wy|Suite P|Highland Park|Spink County|SD|56534|United States|-7|single family| +10621|AAAAAAAANHJCAAAA|377|Davis |Parkway|Suite 280|Spring Hill|Jefferson Davis County|MS|56787|United States|-6|single family| +10622|AAAAAAAAOHJCAAAA|592|Jefferson |Lane|Suite 470|Woodland|Sumner County|KS|64854|United States|-6|condo| +10623|AAAAAAAAPHJCAAAA|700|Smith |Road|Suite Q|Hamilton|Archuleta County|CO|82808|United States|-7|single family| +10624|AAAAAAAAAIJCAAAA|396|Third Elm|Dr.|Suite 240|Glenwood|Conejos County|CO|83511|United States|-7|apartment| +10625|AAAAAAAABIJCAAAA|438|8th Park|Pkwy|Suite B|Springfield|Nelson County|VA|29303|United States|-5|single family| +10626|AAAAAAAACIJCAAAA|173|Mill Meadow|Road|Suite 50|Sutton|Panola County|TX|75413|United States|-6|apartment| +10627|AAAAAAAADIJCAAAA|138|10th First|Pkwy|Suite 410|Arlington|Albany County|WY|86557|United States|-7|apartment| +10628|AAAAAAAAEIJCAAAA|468|Park |Way|Suite E|Jackson|Brunswick County|NC|29583|United States|-5|apartment| +10629|AAAAAAAAFIJCAAAA|970|Locust 1st|Ave|Suite C|Bridgeport|Wharton County|TX|75817|United States|-6|apartment| +10630|AAAAAAAAGIJCAAAA|403|4th |Circle|Suite 300|Highland Park|Linn County|KS|66534|United States|-6|single family| +10631|AAAAAAAAHIJCAAAA|817|View |Circle|Suite 220|Fairfield|Atchison County|KS|66192|United States|-6|apartment| +10632|AAAAAAAAIIJCAAAA||3rd 1st||Suite M|Sulphur Springs|Shelby County||38354|United States|-6|| +10633|AAAAAAAAJIJCAAAA|941|||Suite 480|Unionville||MO|61711|||apartment| +10634|AAAAAAAAKIJCAAAA|905|Hickory Madison|Lane|Suite 420|Lakeview|Leflore County|MS|58579|United States|-6|condo| +10635|AAAAAAAALIJCAAAA|501|Pine Elm|Drive|Suite E|Summit|Chattooga County|GA|30499|United States|-5|apartment| +10636|AAAAAAAAMIJCAAAA|148|Mill |Circle|Suite Y|Leesville|Valencia County|NM|85423|United States|-7|condo| +10637|AAAAAAAANIJCAAAA|452|Second |Way|Suite O|Pine Grove|Tyler County|WV|24593|United States|-5|apartment| +10638|AAAAAAAAOIJCAAAA|612|Green |Blvd|Suite 200|Oakwood|Nottoway County|VA|20169|United States|-5|single family| +10639|AAAAAAAAPIJCAAAA|804|Dogwood |Way|Suite 150|Lakeview|Webster County|MO|68579|United States|-6|apartment| +10640|AAAAAAAAAJJCAAAA|68|Lakeview |Cir.|Suite S|Riverdale|Lincoln County|OK|79391|United States|-6|apartment| +10641|AAAAAAAABJJCAAAA|553|South |Lane|Suite 140|Millbrook|Camden County|NC|27529|United States|-5|apartment| +10642|AAAAAAAACJJCAAAA|813|Lincoln 4th|Wy|Suite K|Glendale|Missaukee County|MI|43951|United States|-5|apartment| +10643|AAAAAAAADJJCAAAA|128|1st Second|Blvd|Suite C|Deerfield|Bremer County|IA|59840|United States|-6|condo| +10644|AAAAAAAAEJJCAAAA|926|Church |Parkway|Suite L|Wildwood|Lafayette County|MS|56871|United States|-6|single family| +10645|AAAAAAAAFJJCAAAA|12|5th |Dr.|Suite 170|Ashland|Portage County|OH|44244|United States|-5|condo| +10646|AAAAAAAAGJJCAAAA|775|Williams Green|Lane|Suite N|Clinton|Miller County|AR|78222|United States|-6|condo| +10647|AAAAAAAAHJJCAAAA|628|Park |Court|Suite T|Mountain View|Coweta County|GA|34466|United States|-5|single family| +10648|AAAAAAAAIJJCAAAA|755|8th River|Boulevard|Suite 150|Wildwood|Colfax County|NE|66871|United States|-6|single family| +10649|AAAAAAAAJJJCAAAA|629|Hickory 4th|Drive|Suite 70|Oak Grove|Glasscock County|TX|78370|United States|-6|single family| +10650|AAAAAAAAKJJCAAAA|200|1st Second|Pkwy|Suite J|Mount Olive|Franklin Parish|LA|78059|United States|-6|condo| +10651|AAAAAAAALJJCAAAA|||Wy|Suite P|Georgetown|McLean County||57057|United States|-6|apartment| +10652|AAAAAAAAMJJCAAAA|288|Cedar |Pkwy|Suite 380|Friendship|Macomb County|MI|44536|United States|-5|apartment| +10653|AAAAAAAANJJCAAAA|684|Cedar Elevnth|Drive|Suite Q|Ryan|Fisher County|TX|70525|United States|-6|condo| +10654|AAAAAAAAOJJCAAAA|918|Broadway |Parkway|Suite 230|Pleasant Grove|Williamsburg County|SC|24136|United States|-5|apartment| +10655|AAAAAAAAPJJCAAAA|936|3rd |Avenue|Suite 490|Bear Creek|Jefferson County|OK|73075|United States|-6|single family| +10656|AAAAAAAAAKJCAAAA|971|Seventh |Wy|Suite I|Oakwood|Payne County|OK|70169|United States|-6|condo| +10657|AAAAAAAABKJCAAAA|145|Railroad Forest|Street|Suite 460|Brookwood|Jefferson County|NY|10965|United States|-5|condo| +10658|AAAAAAAACKJCAAAA|581|Mill |Ln|Suite 80|Highland|Marshall County|WV|29454|United States|-5|apartment| +10659|AAAAAAAADKJCAAAA|573|View 5th|ST|Suite 410|Riverside|Macon County|NC|29231|United States|-5|apartment| +10660|AAAAAAAAEKJCAAAA|315|Sunset Lee|||Glendale|||53951|United States||apartment| +10661|AAAAAAAAFKJCAAAA|761|Sycamore Sunset|Lane|Suite 360|Farmington|York County|NE|69145|United States|-6|single family| +10662|AAAAAAAAGKJCAAAA|383|Spring |Pkwy|Suite R|Spring Hill|Ringgold County|IA|56787|United States|-6|apartment| +10663|AAAAAAAAHKJCAAAA|64|Smith |Ct.|Suite U|Sulphur Springs|Perry County|PA|18354|United States|-5|apartment| +10664|AAAAAAAAIKJCAAAA|457|Twelfth |Dr.|Suite V|Sunnyside|Hancock County|IN|41952|United States|-5|single family| +10665|AAAAAAAAJKJCAAAA|568|Ridge Tenth|Drive|Suite O|Bethel|Wagoner County|OK|75281|United States|-6|single family| +10666|AAAAAAAAKKJCAAAA|443|Walnut |Court|Suite 20|Pine Grove|Lyon County|MN|54593|United States|-6|apartment| +10667|AAAAAAAALKJCAAAA|874|Ridge |Way|Suite U|Oak Hill|Pittsylvania County|VA|27838|United States|-5|condo| +10668|AAAAAAAAMKJCAAAA|118|Main North|ST|Suite V|Oak Grove|Clay County|TN|38370|United States|-5|condo| +10669|AAAAAAAANKJCAAAA|446|2nd |Circle|Suite 330|Oakland|Mercer County|IL|69843|United States|-6|single family| +10670|AAAAAAAAOKJCAAAA|945|First 8th|Blvd|Suite 450|Georgetown|Fairfax County|VA|27057|United States|-5|single family| +10671|AAAAAAAAPKJCAAAA|781|Laurel |Lane|Suite 250|Highland Park|DeKalb County|IL|66534|United States|-6|single family| +10672|AAAAAAAAALJCAAAA||Main Park|Dr.|Suite 70|Riverdale|Lincoln County||||-5|apartment| +10673|AAAAAAAABLJCAAAA|646|Broadway Park|Court|Suite W|Ashland|Penobscot County|ME|04844|United States|-5|condo| +10674|AAAAAAAACLJCAAAA|707|Smith Fourth|Dr.|Suite 260|Hillcrest|Covington city|VA|23003|United States|-5|condo| +10675|AAAAAAAADLJCAAAA|601|Maple |Drive|Suite K|Belmont|Routt County|CO|80191|United States|-7|apartment| +10676|AAAAAAAAELJCAAAA|508|1st |Dr.|Suite 220|Edgewood|Murray County|GA|30069|United States|-5|condo| +10677|AAAAAAAAFLJCAAAA|737|Chestnut South|Cir.|Suite Y|Greenwood|Sarpy County|NE|68828|United States|-7|condo| +10678|AAAAAAAAGLJCAAAA|648|9th Jefferson|Parkway|Suite 60|Five Points|Marion County|IL|66098|United States|-6|single family| +10679|AAAAAAAAHLJCAAAA|236|Hillcrest North|Street|Suite 260|Bunker Hill|Ashland County|OH|40150|United States|-5|condo| +10680|AAAAAAAAILJCAAAA|797|5th 8th|Street|Suite 140|Lebanon|Hamilton County|IA|52898|United States|-6|apartment| +10681|AAAAAAAAJLJCAAAA|141|Central |Cir.|Suite Y|New Hope|Moody County|SD|59431|United States|-7|apartment| +10682|AAAAAAAAKLJCAAAA|704|Lake Lincoln|Circle|Suite 220|New Hope|Stevens County|KS|69431|United States|-6|single family| +10683|AAAAAAAALLJCAAAA|261|9th Main|RD|Suite 480|Denmark|Northampton County|VA|25576|United States|-5|condo| +10684|AAAAAAAAMLJCAAAA|550|Broadway Lake|Circle|Suite 190|Forest|Gentry County|MO|67537|United States|-6|apartment| +10685|AAAAAAAANLJCAAAA|329|Park |Wy|Suite 90|Riverside|Milwaukee County|WI|59231|United States|-6|condo| +10686|AAAAAAAAOLJCAAAA|698|Franklin Railroad|Boulevard|Suite F|Hopewell|Webster County|MO|60587|United States|-6|condo| +10687|AAAAAAAAPLJCAAAA|72|4th |Boulevard|Suite J|Hillcrest|Lancaster County|VA|23003|United States|-5|apartment| +10688|AAAAAAAAAMJCAAAA|617|||||Washita County|OK||United States||single family| +10689|AAAAAAAABMJCAAAA|472|8th |Avenue|Suite 80|Oakdale|Dawson County|GA|39584|United States|-5|apartment| +10690|AAAAAAAACMJCAAAA|598|11th |Pkwy|Suite 440|Plainview|Knox County|TN|33683|United States|-6|single family| +10691|AAAAAAAADMJCAAAA|730|5th 1st|Court|Suite 320|Hardy|Monroe County|IA|55354|United States|-6|condo| +10692|AAAAAAAAEMJCAAAA|514|Lake 9th|Dr.|Suite 130|Florence|Furnas County|NE|63394|United States|-6|single family| +10693|AAAAAAAAFMJCAAAA|665|Maple |ST|Suite 130|Royal|Brown County|KS|65819|United States|-6|apartment| +10694|AAAAAAAAGMJCAAAA|366|Ridge Woodland|Street|Suite O|Liberty|Bland County|VA|23451|United States|-5|single family| +10695|AAAAAAAAHMJCAAAA||Highland Dogwood||Suite 150|Brownsville|Green County||||-6|single family| +10696|AAAAAAAAIMJCAAAA|708|Mill Washington|Lane|Suite W|Pine Grove|Monroe County|WI|54593|United States|-6|apartment| +10697|AAAAAAAAJMJCAAAA|915|North |Drive|Suite 40|Highland|Shannon County|MO|69454|United States|-6|apartment| +10698|AAAAAAAAKMJCAAAA|231|First Spring|Drive|Suite 440|Sunnyside|Orange County|IN|41952|United States|-5|single family| +10699|AAAAAAAALMJCAAAA|135|14th |Road|Suite 130|Arlington|Gloucester County|NJ|07157|United States|-5|apartment| +10700|AAAAAAAAMMJCAAAA|824|Chestnut Railroad|Blvd|Suite V|Forest Hills|Carver County|MN|59237|United States|-6|single family| +10701|AAAAAAAANMJCAAAA|374|7th Central|Street|Suite H|Glendale|Mercer County|OH|43951|United States|-5|apartment| +10702|AAAAAAAAOMJCAAAA|777|Washington Spring|Ln|Suite W|Riverside|Merrimack County|NH|09831|United States|-5|condo| +10703|AAAAAAAAPMJCAAAA|405|Franklin |Boulevard|Suite 280|Mountain View|Schuyler County|IL|64466|United States|-6|single family| +10704|AAAAAAAAANJCAAAA|449|Dogwood |Ave|Suite 160|Shady Grove|Fulton County|GA|32812|United States|-5|condo| +10705|AAAAAAAABNJCAAAA|551|Highland Lake|Parkway|Suite 400|Newport|Bonner County|ID|81521|United States|-7|condo| +10706|AAAAAAAACNJCAAAA|770|West |Drive|Suite 290|Hopewell|Lake of the Woods County|MN|50587|United States|-6|apartment| +10707|AAAAAAAADNJCAAAA|993|Central Birch|Ave|Suite K|Oak Grove|Sullivan County|TN|38370|United States|-6|single family| +10708|AAAAAAAAENJCAAAA|184|1st |Wy|Suite 100|Glendale|Prince George County|VA|23951|United States|-5|single family| +10709|AAAAAAAAFNJCAAAA|841|Dogwood Broadway|Dr.|Suite D|Mount Vernon|Kaufman County|TX|78482|United States|-6|apartment| +10710|AAAAAAAAGNJCAAAA|973|Chestnut |Court|Suite I|Antioch|Clallam County|WA|98605|United States|-8|condo| +10711|AAAAAAAAHNJCAAAA|802|Woodland Park|ST|Suite I|Highland Park|Wayne County|MI|46534|United States|-5|single family| +10712|AAAAAAAAINJCAAAA|995|4th Third|Ave|Suite X|Oakland|Travis County|TX|79843|United States|-6|condo| +10713|AAAAAAAAJNJCAAAA|123|Williams |Ct.|Suite Q|Jackson|Valley County|ID|89583|United States|-7|single family| +10714|AAAAAAAAKNJCAAAA|959|9th ||Suite 220|Wilson|Forest County|WI|56971||-6|single family| +10715|AAAAAAAALNJCAAAA|543|3rd |Circle|Suite M|Belmont|Andrews County|TX|70191|United States|-6|apartment| +10716|AAAAAAAAMNJCAAAA|476|Green |Blvd|Suite 460|Pine Grove|Davis County|IA|54593|United States|-6|apartment| +10717|AAAAAAAANNJCAAAA|339|Central |Lane|Suite 20|Mount Pleasant|Carlisle County|KY|41933|United States|-6|condo| +10718|AAAAAAAAONJCAAAA|669|Washington Lake|Ave|Suite A|Empire|Bonneville County|ID|84145|United States|-7|condo| +10719|AAAAAAAAPNJCAAAA|716|Spruce |Blvd|Suite 40|Peru|Okfuskee County|OK|70302|United States|-6|condo| +10720|AAAAAAAAAOJCAAAA|151|Cherry |Drive|Suite 350|Crossroads|Benton County|IN|40534|United States|-5|condo| +10721|AAAAAAAABOJCAAAA|441|Oak Elm|Blvd|Suite 80|Red Hill|Crawford County|GA|34338|United States|-5|apartment| +10722|AAAAAAAACOJCAAAA|955|Locust |Circle|Suite U|Glenville|Wabash County|IL|63445|United States|-6|condo| +10723|AAAAAAAADOJCAAAA|55|Miller |Boulevard|Suite T|Maple Grove|Dickens County|TX|78252|United States|-6|single family| +10724|AAAAAAAAEOJCAAAA||Railroad Church|Drive||||||United States|-5|| +10725|AAAAAAAAFOJCAAAA|95|View |Street|Suite 110|Kingston|Menominee County|WI|54975|United States|-6|condo| +10726|AAAAAAAAGOJCAAAA|941|Cedar 2nd|Street|Suite K|Riceville|Hamilton County|TX|75867|United States|-6|apartment| +10727|AAAAAAAAHOJCAAAA|117|Hickory |Cir.|Suite 360|Gladstone|Union County|SD|50894|United States|-7|condo| +10728|AAAAAAAAIOJCAAAA|867|5th |Drive|Suite 380|Plainview|Greenbrier County|WV|23683|United States|-5|condo| +10729|AAAAAAAAJOJCAAAA|944|Sycamore 3rd|Lane|Suite 90|Lincoln|Traverse County|MN|51289|United States|-6|apartment| +10730|AAAAAAAAKOJCAAAA|465||||||||United States||| +10731|AAAAAAAALOJCAAAA|439|Jackson |Court|Suite Q|Red Hill|Crawford County|OH|44338|United States|-5|condo| +10732|AAAAAAAAMOJCAAAA|349||Street|Suite P||Yankton County||53445|||single family| +10733|AAAAAAAANOJCAAAA|217|10th 1st|Drive|Suite 280|Hamilton|Valley County|ID|82808|United States|-7|apartment| +10734|AAAAAAAAOOJCAAAA|238|Walnut |Street|Suite 20|Mount Pleasant|Renville County|ND|51933|United States|-6|single family| +10735|AAAAAAAAPOJCAAAA||Miller ||||Pasquotank County||24244|United States|-5|single family| +10736|AAAAAAAAAPJCAAAA|955|Highland View|ST|Suite J|Summit|Clarke County|MS|50499|United States|-6|single family| +10737|AAAAAAAABPJCAAAA|977|Sunset |ST|Suite 50|Georgetown|Dickens County|TX|77057|United States|-6|single family| +10738|AAAAAAAACPJCAAAA|617|Lake Highland|Street|Suite 280|Oakland|Iron County|WI|59843|United States|-6|apartment| +10739|AAAAAAAADPJCAAAA|877|Willow |Blvd|Suite H|Maple Grove|Manitowoc County|WI|58252|United States|-6|single family| +10740|AAAAAAAAEPJCAAAA|989|14th Laurel|Court|Suite 50|Five Forks|Linn County|KS|62293|United States|-6|condo| +10741|AAAAAAAAFPJCAAAA|775|Poplar 6th|Cir.|Suite 210|Mount Zion|Wyoming County|NY|18054|United States|-5|single family| +10742|AAAAAAAAGPJCAAAA|873|Oak 15th|Dr.|Suite 60|Wyoming|Crockett County|TX|70216|United States|-6|condo| +10743|AAAAAAAAHPJCAAAA|870|Railroad |Ave|Suite 260|Oak Hill|Hendry County|FL|37838|United States|-5|apartment| +10744|AAAAAAAAIPJCAAAA|175|Wilson Walnut|Street|Suite 140|Burns|Treutlen County|GA|35272|United States|-5|apartment| +10745|AAAAAAAAJPJCAAAA|312|Sycamore |Drive|Suite 320|Riverdale|Copiah County|MS|59391|United States|-6|apartment| +10746|AAAAAAAAKPJCAAAA|64|Wilson |Court|Suite 350|Glenwood|Madison County|GA|33511|United States|-5|condo| +10747|AAAAAAAALPJCAAAA|826|5th West|Ct.|Suite T|Crossroads|Knox County|IL|60534|United States|-6|apartment| +10748|AAAAAAAAMPJCAAAA|426|Walnut Mill|Way|Suite 100|Paxton|Blaine County|NE|65669|United States|-6|single family| +10749|AAAAAAAANPJCAAAA|548|Park Spruce|Avenue|Suite D|Spring Valley|Johnson County|NE|66060|United States|-7|apartment| +10750|AAAAAAAAOPJCAAAA|284|South |Ave|Suite O|Liberty|Clark County|IL|63451|United States|-6|single family| +10751|AAAAAAAAPPJCAAAA|44|Eigth |Boulevard|Suite 150|Midway|Crawford County|IN|41904|United States|-5|apartment| +10752|AAAAAAAAAAKCAAAA|360|Davis |Boulevard|Suite 60|Crossroads|Bullock County|AL|30534|United States|-6|condo| +10753|AAAAAAAABAKCAAAA|780|West |Way|Suite 230|Jackson|Torrance County|NM|89583|United States|-7|single family| +10754|AAAAAAAACAKCAAAA|846|View First|Lane|Suite 260|Crossroads|Cass County|TX|70534|United States|-6|apartment| +10755|AAAAAAAADAKCAAAA|631|Cedar Sunset|Dr.|Suite I|Saratoga|Muhlenberg County|KY|42123|United States|-5|condo| +10756|AAAAAAAAEAKCAAAA|85|Sixth |Parkway|Suite N|Five Points|Treasure County|MT|66098|United States|-7|apartment| +10757|AAAAAAAAFAKCAAAA|807|Willow Lincoln|Dr.|Suite E|Forest Hills|Washington County|CO|89237|United States|-7|condo| +10758|AAAAAAAAGAKCAAAA|286|Valley |Circle|Suite 380|Shady Grove|Scott County|MS|52812|United States|-6|apartment| +10759|AAAAAAAAHAKCAAAA|726|Fourteenth |Parkway|Suite 30|Springfield|Caledonia County|VT|09903|United States|-5|single family| +10760|AAAAAAAAIAKCAAAA|38|River Miller|Drive|Suite 80|Sulphur Springs|Charlotte County|FL|38354|United States|-5|single family| +10761|AAAAAAAAJAKCAAAA|201|River |Boulevard|Suite 140|Spring Valley|Alpena County|MI|46060|United States|-5|apartment| +10762|AAAAAAAAKAKCAAAA|222|Forest |Parkway|Suite N|Spring Hill|Kimble County|TX|76787|United States|-6|single family| +10763|AAAAAAAALAKCAAAA|231|Park |Cir.|Suite 170|Glendale|Austin County|TX|73951|United States|-6|condo| +10764|AAAAAAAAMAKCAAAA|483|7th |Cir.|Suite Y|Forest Hills|Rock County|MN|59237|United States|-6|condo| +10765|AAAAAAAANAKCAAAA|9|Lake Madison|Way|||||||-8|condo| +10766|AAAAAAAAOAKCAAAA|535|2nd |ST|Suite 0|Bethel|Stanton County|NE|65281|United States|-7|condo| +10767|AAAAAAAAPAKCAAAA|626|12th Spring|Lane|Suite B|Clifton|North Slope Borough|AK|98014|United States|-9|single family| +10768|AAAAAAAAABKCAAAA|977|Walnut Elm|Pkwy|Suite 350|Midway|Johnson County|NE|61904|United States|-7|apartment| +10769|AAAAAAAABBKCAAAA|332|Sunset |Circle|Suite 470|Hamilton|Deaf Smith County|TX|72808|United States|-6|single family| +10770|AAAAAAAACBKCAAAA|626|Franklin 2nd|Ave|Suite J|Highland|Winnebago County|IL|69454|United States|-6|single family| +10771|AAAAAAAADBKCAAAA|797|East |Pkwy|Suite 90|Macedonia|Calhoun County|SC|21087|United States|-5|single family| +10772|AAAAAAAAEBKCAAAA|386|13th 12th|Ave|Suite 460|Shady Grove|Okfuskee County|OK|72812|United States|-6|condo| +10773|AAAAAAAAFBKCAAAA|595|5th Fifth|Cir.|Suite N|Shiloh|White County|AR|79275|United States|-6|condo| +10774|AAAAAAAAGBKCAAAA|956|6th Washington|Ln|Suite I|Oak Hill|Larimer County|CO|87838|United States|-7|condo| +10775|AAAAAAAAHBKCAAAA|313|Ridge Hill|Avenue|Suite U|Mountain View|Charles City County|VA|24466|United States|-5|apartment| +10776|AAAAAAAAIBKCAAAA|825|Lincoln 2nd|Road|Suite S|Red Hill|Elk County|KS|64338|United States|-6|condo| +10777|AAAAAAAAJBKCAAAA|257|Davis Elm|Road|Suite 120|New Hope|Rutherford County|TN|39431|United States|-6|single family| +10778|AAAAAAAAKBKCAAAA||2nd Park||Suite 260|||||United States||apartment| +10779|AAAAAAAALBKCAAAA|349|4th |Boulevard|Suite 200|Salem|Georgetown County|SC|28048|United States|-5|single family| +10780|AAAAAAAAMBKCAAAA|954|5th |Parkway|Suite D|Oakwood|Houston County|MN|50169|United States|-6|apartment| +10781|AAAAAAAANBKCAAAA|404|Washington |Drive|Suite 430|Hamilton|Meigs County|OH|42808|United States|-5|apartment| +10782|AAAAAAAAOBKCAAAA|452|Seventh 11th|Avenue|Suite A|Plainview|Hunterdon County|NJ|04283|United States|-5|condo| +10783|AAAAAAAAPBKCAAAA|249|Broadway |Blvd|Suite 250|White Oak|Wilcox County|AL|36668|United States|-6|apartment| +10784|AAAAAAAAACKCAAAA|599|Sixth |Street|Suite 70|Riverdale|Hughes County|OK|79391|United States|-6|single family| +10785|AAAAAAAABCKCAAAA|657|Sycamore Park|Blvd|Suite W|Greenfield|Mercer County|NJ|05638|United States|-5|single family| +10786|AAAAAAAACCKCAAAA|981|3rd |Pkwy|Suite C|Unionville|Lee County|NC|21711|United States|-5|condo| +10787|AAAAAAAADCKCAAAA|781|Main |Ave|Suite B|Midway|Zavala County|TX|71904|United States|-6|apartment| +10788|AAAAAAAAECKCAAAA|630|11th Lake|Blvd||Spring Valley||TX|76060|United States||condo| +10789|AAAAAAAAFCKCAAAA|770|Oak ||Suite 290|Mount Vernon|||38482|United States||condo| +10790|AAAAAAAAGCKCAAAA|616|2nd |Way|Suite O|Clifton|Claiborne County|MS|58014|United States|-6|apartment| +10791|AAAAAAAAHCKCAAAA|533|Main |Avenue|Suite L|Riverview|LaMoure County|ND|59003|United States|-6|condo| +10792|AAAAAAAAICKCAAAA|733|Elm |Boulevard|Suite 70|Belmont|Mendocino County|CA|90191|United States|-8|apartment| +10793|AAAAAAAAJCKCAAAA|744|Lakeview 2nd|Street|Suite L|Bridgeport|Walker County|AL|35817|United States|-6|single family| +10794|AAAAAAAAKCKCAAAA|299|Maple |Parkway|Suite O|Wildwood|Boise County|ID|86871|United States|-7|apartment| +10795|AAAAAAAALCKCAAAA|964|Lee 5th|Way|Suite G|Glendale|Sharkey County|MS|53951|United States|-6|condo| +10796|AAAAAAAAMCKCAAAA|930|Elm Jackson|Ln|Suite 140|Greenfield|Dewey County|SD|55038|United States|-6|single family| +10797|AAAAAAAANCKCAAAA|787|1st |Avenue|Suite G|Walnut Grove|Somerset County|ME|08352|United States|-5|apartment| +10798|AAAAAAAAOCKCAAAA|738|Williams 2nd|Court|Suite 270|Woodland||||United States||| +10799|AAAAAAAAPCKCAAAA|504|Ridge Dogwood|Circle|Suite 0|Centerville|Crawford County|IN|40059|United States|-5|apartment| +10800|AAAAAAAAADKCAAAA|493|Central |Way|Suite 180|Greenfield|Holmes County|MS|55038|United States|-6|apartment| +10801|AAAAAAAABDKCAAAA|263|North |Avenue|Suite 300|Georgetown|Clearfield County|PA|17057|United States|-5|apartment| +10802|AAAAAAAACDKCAAAA|461|Birch |Street|Suite 0|Newtown|Henry County|OH|41749|United States|-5|apartment| +10803|AAAAAAAADDKCAAAA|848|Poplar Meadow|Way|Suite 70|Lakeside|Elk County|PA|19532|United States|-5|single family| +10804|AAAAAAAAEDKCAAAA|161|Elm ||Suite D||Bexar County|TX|||-6|single family| +10805|AAAAAAAAFDKCAAAA|977|Hickory Spruce|Blvd|Suite 60|Crossroads|Carlisle County|KY|40534|United States|-6|apartment| +10806|AAAAAAAAGDKCAAAA|258|Pine Washington|RD|Suite 90|Wilson|Prince William County|VA|26971|United States|-5|apartment| +10807|AAAAAAAAHDKCAAAA|612|Fourth Lake|ST|Suite 230|Clifton|Salt Lake County|UT|88014|United States|-7|condo| +10808|AAAAAAAAIDKCAAAA|736|East |Road|Suite G|Walnut Grove|Worcester County|MD|27752|United States|-5|single family| +10809|AAAAAAAAJDKCAAAA|747|Lake View|Ave|Suite 170|Hamilton|Bath County|VA|22808|United States|-5|condo| +10810|AAAAAAAAKDKCAAAA|386|East 7th|Wy|Suite 370|Shady Grove|Yavapai County|AZ|82812|United States|-7|condo| +10811|AAAAAAAALDKCAAAA|175|9th Ridge|Avenue|Suite 280|Five Forks|Hendry County|FL|32293|United States|-5|condo| +10812|AAAAAAAAMDKCAAAA|813|3rd Cherry|Street|Suite O|Saint George|Warren County|MS|55281|United States|-6|condo| +10813|AAAAAAAANDKCAAAA|358|10th Adams|Ct.|Suite 310|Greenville|Oliver County|ND|51387|United States|-6|apartment| +10814|AAAAAAAAODKCAAAA|447|Sixth |Ln|Suite 410|Union|Coffee County|AL|38721|United States|-6|single family| +10815|AAAAAAAAPDKCAAAA|222|Sunset First|Street|Suite A|Five Points|Calhoun County|SC|26098|United States|-5|single family| +10816|AAAAAAAAAEKCAAAA|282|Franklin Eigth|ST|Suite R|Pleasant Valley|Bristol Bay Borough|AK|92477|United States|-9|condo| +10817|AAAAAAAABEKCAAAA|879|2nd |Pkwy|Suite F|Lakeview|Wilson County|TN|38579|United States|-5|single family| +10818|AAAAAAAACEKCAAAA|243||Circle||Crossroads|Buffalo County|NE||United States||condo| +10819|AAAAAAAADEKCAAAA|571|Cedar |Way|Suite E|Forest Hills|Otero County|NM|89237|United States|-7|apartment| +10820|AAAAAAAAEEKCAAAA||Hill |Dr.||Providence|||36614|United States||apartment| +10821|AAAAAAAAFEKCAAAA|106|4th |Boulevard|Suite W|Sulphur Springs|Hudson County|NJ|08954|United States|-5|apartment| +10822|AAAAAAAAGEKCAAAA|536|Birch |Ave|Suite 130|Page|Manitowoc County|WI|50296|United States|-6|single family| +10823|AAAAAAAAHEKCAAAA|892|15th |Ln|Suite F|Friendship|Moore County|NC|24536|United States|-5|condo| +10824|AAAAAAAAIEKCAAAA|764|Williams |RD|Suite 380|Waterloo|San Francisco County|CA|91675|United States|-8|single family| +10825|AAAAAAAAJEKCAAAA|682|Pine South|Avenue|Suite 420|Woodland|Spalding County|GA|34854|United States|-5|apartment| +10826|AAAAAAAAKEKCAAAA|650|Center |Street|Suite 390|Mount Vernon|Medina County|TX|78482|United States|-6|single family| +10827|AAAAAAAALEKCAAAA|776|Hill |Blvd|Suite 100|Liberty|Grundy County|MO|63451|United States|-6|apartment| +10828|AAAAAAAAMEKCAAAA|303|West Cedar|RD|Suite X|Riverdale|Chittenden County|VT|09991|United States|-5|condo| +10829|AAAAAAAANEKCAAAA|377|River |RD|Suite Q|Enterprise|Churchill County|NV|81757|United States|-8|apartment| +10830|AAAAAAAAOEKCAAAA|279|Central |Court|Suite J|Five Forks|Arenac County|MI|42293|United States|-5|single family| +10831|AAAAAAAAPEKCAAAA|772|Sixth |Ave|Suite 450|Florence|Washington County|IA|53394|United States|-6|apartment| +10832|AAAAAAAAAFKCAAAA|608|Ridge |Avenue|Suite L|Valley View|Bradford County|PA|15124|United States|-5|condo| +10833|AAAAAAAABFKCAAAA|250|Washington Walnut|Court|Suite F|Florence|Franklin County|AR|73394|United States|-6|single family| +10834|AAAAAAAACFKCAAAA|889|Lincoln |Cir.|Suite 90|Fairfield|Saginaw County|MI|46192|United States|-5|single family| +10835|AAAAAAAADFKCAAAA|204|Central West|RD|Suite P|Belfast|Liberty County|GA|30125|United States|-5|single family| +10836|AAAAAAAAEFKCAAAA|292|Walnut Hillcrest|Lane|Suite 440|Colonial Heights|Meade County|SD|53425|United States|-7|condo| +10837|AAAAAAAAFFKCAAAA|838|North Ash|Drive|Suite 270|Enterprise|Potter County|SD|51757|United States|-7|apartment| +10838|AAAAAAAAGFKCAAAA|31|Oak Jefferson|Way|Suite 400|Franklin|Nottoway County|VA|29101|United States|-5|apartment| +10839|AAAAAAAAHFKCAAAA|508|Chestnut Railroad|Lane|Suite 450|Union Hill|Hopewell city|VA|27746|United States|-5|single family| +10840|AAAAAAAAIFKCAAAA|494|Washington Meadow|Wy|Suite 70|Crossroads|Costilla County|CO|80534|United States|-7|condo| +10841|AAAAAAAAJFKCAAAA|917|Madison |ST|Suite 80|Bethel|Plymouth County|MA|05881|United States|-5|single family| +10842|AAAAAAAAKFKCAAAA|362|10th Tenth|Street|Suite 0|Macedonia|Crawford County|MI|41087|United States|-5|apartment| +10843|AAAAAAAALFKCAAAA|540|5th South|Lane|Suite 60|Spring Valley|Nez Perce County|ID|86060|United States|-7|single family| +10844|AAAAAAAAMFKCAAAA|314|2nd |Wy|Suite Y|Carpenter|Taylor County|GA|31147|United States|-5|single family| +10845|AAAAAAAANFKCAAAA|142|Hillcrest |Dr.|Suite 250|Five Forks|Peoria County|IL|62293|United States|-6|single family| +10846|AAAAAAAAOFKCAAAA|517|Spring Valley|Lane|Suite O|Marion|Lackawanna County|PA|10399|United States|-5|single family| +10847|AAAAAAAAPFKCAAAA|521|Cedar |RD|Suite 420|Mount Olive|Sierra County|NM|88059|United States|-7|condo| +10848|AAAAAAAAAGKCAAAA|9||Dr.||||IN||United States||single family| +10849|AAAAAAAABGKCAAAA|758|Johnson Wilson|Pkwy|Suite 60|Oakdale|Allegany County|MD|29584|United States|-5|single family| +10850|AAAAAAAACGKCAAAA|658|Main |Court|Suite 130|Lakeside|Coffey County|KS|69532|United States|-6|single family| +10851|AAAAAAAADGKCAAAA|596|Pine |Cir.|||Wilkinson County|MS|55804||-6|apartment| +10852|AAAAAAAAEGKCAAAA|376|Main Mill|Avenue|Suite 470|Highland|Hancock County|IL|69454|United States|-6|condo| +10853|AAAAAAAAFGKCAAAA|95|Wilson |Parkway|Suite L|Pine Grove|Allegany County|MD|24593|United States|-5|single family| +10854|AAAAAAAAGGKCAAAA|54|Washington |Circle|Suite I|Ashland|Clay County|TN|34244|United States|-5|apartment| +10855|AAAAAAAAHGKCAAAA|400|6th Dogwood|RD|Suite D|Arlington|Phillips County|CO|86557|United States|-7|condo| +10856|AAAAAAAAIGKCAAAA|933|Hill First|Cir.|Suite P|Pleasant Grove|Pendleton County|WV|24136|United States|-5|single family| +10857|AAAAAAAAJGKCAAAA|160|8th Green|Ave|Suite 220|Mount Olive|Chester County|TN|38059|United States|-5|apartment| +10858|AAAAAAAAKGKCAAAA|759|Williams |Cir.|Suite 270|Prosperity|Newton County|MO|69089|United States|-6|condo| +10859|AAAAAAAALGKCAAAA|202|Jefferson |Dr.|Suite 360|Greenville|Pondera County|MT|61387|United States|-7|single family| +10860|AAAAAAAAMGKCAAAA|160|Willow 13th|ST|Suite G|Lebanon|Hood County|TX|72898|United States|-6|single family| +10861|AAAAAAAANGKCAAAA|760|West |RD|Suite 450|Five Forks|Nelson County|ND|52293|United States|-6|apartment| +10862|AAAAAAAAOGKCAAAA|683|2nd Mill|Drive|Suite K|Harmony|Blackford County|IN|45804|United States|-5|apartment| +10863|AAAAAAAAPGKCAAAA|197|Main West|Way|Suite J|Kingston|Monroe County|IN|44975|United States|-5|single family| +10864|AAAAAAAAAHKCAAAA|417|Broadway 2nd|Drive|Suite 170|Glendale|Taylor County|IA|53951|United States|-6|apartment| +10865|AAAAAAAABHKCAAAA|1000|Meadow |Way|Suite 60|Highland Park|Moffat County|CO|86534|United States|-7|condo| +10866|AAAAAAAACHKCAAAA|90|Meadow Franklin|Lane|Suite 70|Shaw|Henry County|KY|40618|United States|-6|single family| +10867|AAAAAAAADHKCAAAA|29||Cir.|||Audubon County|IA|59101||-6|| +10868|AAAAAAAAEHKCAAAA|366|West 8th|Drive|Suite 340|Pleasant Hill|Perkins County|SD|53604|United States|-7|apartment| +10869|AAAAAAAAFHKCAAAA|32|View |Road|Suite 480|Concord|Horry County|SC|24107|United States|-5|single family| +10870|AAAAAAAAGHKCAAAA|645|Lee |Boulevard|Suite R|Unionville|Okmulgee County|OK|71711|United States|-6|apartment| +10871|AAAAAAAAHHKCAAAA|142|Meadow |Road|Suite 460|Woodville|Montgomery County|GA|34289|United States|-5|single family| +10872|AAAAAAAAIHKCAAAA|357|Green |ST|Suite D|Friendship|Navajo County|AZ|84536|United States|-7|apartment| +10873|AAAAAAAAJHKCAAAA|614|Willow |Parkway|Suite W|Concord|Todd County|MN|54107|United States|-6|condo| +10874|AAAAAAAAKHKCAAAA|820|Valley 2nd|Avenue|Suite X|Concord|Lincoln County|CO|84107|United States|-7|single family| +10875|AAAAAAAALHKCAAAA|874|11th |Court|Suite 80|Springfield|Sequoyah County|OK|79303|United States|-6|single family| +10876|AAAAAAAAMHKCAAAA|326|4th |Avenue|Suite 450|Oak Ridge|Cabarrus County|NC|28371|United States|-5|single family| +10877|AAAAAAAANHKCAAAA|67|Pine |Wy|Suite 480|Shelby|Craven County|NC|26575|United States|-5|apartment| +10878|AAAAAAAAOHKCAAAA|331|Williams North|Wy|Suite S|Barnes|Colonial Heights city|VA|23788|United States|-5|condo| +10879|AAAAAAAAPHKCAAAA|385|Highland Fourth|Lane|Suite 140|Plainview|Union County|NC|23683|United States|-5|apartment| +10880|AAAAAAAAAIKCAAAA|542|Davis |Dr.|Suite N|Spring Hill|Hardee County|FL|36787|United States|-5|condo| +10881|AAAAAAAABIKCAAAA|864|Poplar |Road|Suite 70|Peoria|Ripley County|MO|69818|United States|-6|condo| +10882|AAAAAAAACIKCAAAA|361|3rd East|Dr.|Suite 270|Belmont|Hancock County|MS|50191|United States|-6|single family| +10883|AAAAAAAADIKCAAAA|389|Lincoln |Ave|Suite N|Spring Valley|West Baton Rouge Parish|LA|76060|United States|-6|apartment| +10884|AAAAAAAAEIKCAAAA|324|Oak Lakeview|Way|Suite 290|Providence|Perry County|MS|56614|United States|-6|single family| +10885|AAAAAAAAFIKCAAAA|179|Lake |Lane|Suite 390|Antioch|Palm Beach County|FL|38605|United States|-5|single family| +10886|AAAAAAAAGIKCAAAA|997|12th |Blvd|Suite S|Jamestown|Mercer County|MO|66867|United States|-6|single family| +10887|AAAAAAAAHIKCAAAA|546|Pine Valley|Drive|Suite O|Lakeside|Calhoun County|IA|59532|United States|-6|apartment| +10888|AAAAAAAAIIKCAAAA|978|5th |Parkway|Suite D|Clinton|Glascock County|GA|38222|United States|-5|single family| +10889|AAAAAAAAJIKCAAAA|619|Davis |Court|Suite 380|Greenwood|Nantucket County|MA|09428|United States|-5|apartment| +10890|AAAAAAAAKIKCAAAA|5|Johnson 2nd|Street|Suite 10|Lakeville|Cannon County|TN|38811|United States|-5|condo| +10891|AAAAAAAALIKCAAAA|90|3rd |Street|Suite O|Greenville|Bladen County|NC|21387|United States|-5|apartment| +10892|AAAAAAAAMIKCAAAA|96|Center Meadow|Court|Suite 160|Belmont|Dixon County|NE|60191|United States|-6|condo| +10893|AAAAAAAANIKCAAAA|995|Park |Road|Suite 270|Oakland|Washington County|NY|19843|United States|-5|condo| +10894|AAAAAAAAOIKCAAAA|459|Hillcrest |Wy|Suite 120|Harmony|Charles Mix County|SD|55804|United States|-6|condo| +10895|AAAAAAAAPIKCAAAA|765|Oak Pine|Lane|Suite 0|Marion|Sawyer County|WI|50399|United States|-6|condo| +10896|AAAAAAAAAJKCAAAA|619|Elm |Circle|Suite 0|Waterloo|Greene County|OH|41675|United States|-5|apartment| +10897|AAAAAAAABJKCAAAA|951|Walnut |Blvd||Fairview|Jackson County|FL|35709|United States||| +10898|AAAAAAAACJKCAAAA|323|River Washington|RD|Suite 200|Greenfield|Morgan County|UT|85038|United States|-7|condo| +10899|AAAAAAAADJKCAAAA|305|12th |Road|Suite 220|Greenfield|Highland County|OH|45038|United States|-5|single family| +10900|AAAAAAAAEJKCAAAA|4|Woodland 5th|Boulevard|Suite 10|Greenfield|Ouachita County|AR|75038|United States|-6|condo| +10901|AAAAAAAAFJKCAAAA|283|View |Wy|Suite K|Wilson|Tallapoosa County|AL|36971|United States|-6|apartment| +10902|AAAAAAAAGJKCAAAA|535|Pine Forest|Dr.|Suite 110|Greenwood|Nemaha County|KS|68828|United States|-6|condo| +10903|AAAAAAAAHJKCAAAA|256|Maple 15th|Court|Suite T|Lakeview|Mills County|IA|58579|United States|-6|apartment| +10904|AAAAAAAAIJKCAAAA|581|8th |Drive|Suite 10|Belmont|Miner County|SD|50191|United States|-7|apartment| +10905|AAAAAAAAJJKCAAAA|236|Church 6th|RD|Suite F|Providence|Muskingum County|OH|46614|United States|-5|condo| +10906|AAAAAAAAKJKCAAAA|515|Ash |Cir.|Suite 30|Jamestown|Burlington County|NJ|07467|United States|-5|apartment| +10907|AAAAAAAALJKCAAAA|241|Railroad Sunset|ST|Suite G|Pleasant Hill|McCurtain County|OK|73604|United States|-6|apartment| +10908|AAAAAAAAMJKCAAAA|254|Lakeview Hill|Lane|Suite 60|Belmont|Crawford County|WI|50191|United States|-6|apartment| +10909|AAAAAAAANJKCAAAA|510|Walnut |Dr.|Suite P|Wilson|Campbell County|VA|26971|United States|-5|apartment| +10910|AAAAAAAAOJKCAAAA|239|9th Lincoln|Pkwy|Suite 140|Mount Pleasant|Wirt County|WV|21933|United States|-5|single family| +10911|AAAAAAAAPJKCAAAA||Forest |Lane|Suite A||Yellowstone County|MT|66457|United States||condo| +10912|AAAAAAAAAKKCAAAA|239|Birch Mill|Drive|Suite 300|Franklin|Johnson County|IL|69101|United States|-6|single family| +10913|AAAAAAAABKKCAAAA|292|East 3rd|Blvd|Suite P|Spring Hill|Knox County|TX|76787|United States|-6|apartment| +10914|AAAAAAAACKKCAAAA|991|Maple |Street|Suite P|Fairfield|Stutsman County|ND|56192|United States|-6|single family| +10915|AAAAAAAADKKCAAAA|73|Fourth |Lane|Suite 420|Jackson|Limestone County|TX|79583|United States|-6|apartment| +10916|AAAAAAAAEKKCAAAA|40|Lake |Pkwy|Suite R|Fairfield|Greeley County|KS|66192|United States|-6|condo| +10917|AAAAAAAAFKKCAAAA|18|Spruce |Circle|Suite R|Red Hill|Watauga County|NC|24338|United States|-5|apartment| +10918|AAAAAAAAGKKCAAAA|284|Cedar Lake|Dr.|Suite 20|Buena Vista|Dodge County|NE|65752|United States|-6|single family| +10919|AAAAAAAAHKKCAAAA|897|View |Dr.|Suite G|Newtown|Keya Paha County|NE|61749|United States|-7|apartment| +10920|AAAAAAAAIKKCAAAA|581|Dogwood |ST|Suite 280|Pleasant Hill|Lee County|MS|53604|United States|-6|apartment| +10921|AAAAAAAAJKKCAAAA|863|Lee Tenth|Road|Suite 420|Florence|Wilkes County|NC|23394|United States|-5|apartment| +10922|AAAAAAAAKKKCAAAA|200|Elm Birch|Pkwy|Suite A|Concord|Madison County|ID|84107|United States|-7|apartment| +10923|AAAAAAAALKKCAAAA|902|Ridge |ST|Suite 150|Mountain View|Douglas County|MN|54466|United States|-6|single family| +10924|AAAAAAAAMKKCAAAA|79|Birch Laurel|Blvd|Suite F|Providence|Bonneville County|ID|86614|United States|-7|condo| +10925|AAAAAAAANKKCAAAA|992|Park 5th|RD|Suite W|Lakewood|Rock County|WI|58877|United States|-6|single family| +10926|AAAAAAAAOKKCAAAA|105|5th Railroad|RD|Suite 440|Woodlawn|Dorchester County|SC|24098|United States|-5|apartment| +10927|AAAAAAAAPKKCAAAA|846|Lee |Road|Suite R|Oakdale|Latimer County|OK|79584|United States|-6|single family| +10928|AAAAAAAAALKCAAAA|156|Pine |Ln|Suite 320|Highland Park|Meade County|KY|46534|United States|-5|single family| +10929|AAAAAAAABLKCAAAA|564|Fifth |Parkway|Suite L|Ferguson|Baker County|FL|31821|United States|-5|apartment| +10930|AAAAAAAACLKCAAAA|646|Seventh Forest|Lane|Suite 90|Five Points|Tuscola County|MI|46098|United States|-5|apartment| +10931|AAAAAAAADLKCAAAA|42|College Lakeview|Parkway|Suite P|Frenchtown|Pinellas County|FL|32629|United States|-5|single family| +10932|AAAAAAAAELKCAAAA|702|Fifth Fifth|Avenue|Suite 180|Newport|Fleming County|KY|41521|United States|-6|condo| +10933|AAAAAAAAFLKCAAAA|372|Maple |Road|Suite 370|Mountain View|Oregon County|MO|64466|United States|-6|condo| +10934|AAAAAAAAGLKCAAAA|169|Park Cherry|Drive|Suite 290|Forest Hills|Conejos County|CO|89237|United States|-7|apartment| +10935|AAAAAAAAHLKCAAAA|240|Pine Broadway|Ct.|Suite 70|Highland|Cayuga County|NY|19454|United States|-5|single family| +10936|AAAAAAAAILKCAAAA|290|Chestnut |Parkway|Suite 460|Wilson|Washington County|UT|86971|United States|-7|condo| +10937|AAAAAAAAJLKCAAAA|246|Maple |Road|Suite 230|Macedonia|Cass County|IA|51087|United States|-6|apartment| +10938|AAAAAAAAKLKCAAAA|161|Pine |Circle|Suite 470|Five Forks|Halifax County|VA|22293|United States|-5|single family| +10939|AAAAAAAALLKCAAAA|||Street||Evans||MS|52284|United States|-6|| +10940|AAAAAAAAMLKCAAAA||||Suite 260|||||United States||| +10941|AAAAAAAANLKCAAAA|368|Thirteenth |Dr.|Suite 70|Summit|Niobrara County|WY|80499|United States|-7|single family| +10942|AAAAAAAAOLKCAAAA|520|College Oak|||||||United States||condo| +10943|AAAAAAAAPLKCAAAA|179|Oak 14th|Ln|Suite 270|Mount Zion|Marshall County|IN|48054|United States|-5|single family| +10944|AAAAAAAAAMKCAAAA|600|Oak |ST|Suite Q|Shiloh|Custer County|SD|59275|United States|-6|condo| +10945|AAAAAAAABMKCAAAA|327|Lincoln |Ave|Suite 390|Union|Lee County|FL|38721|United States|-5|condo| +10946|AAAAAAAACMKCAAAA||Forest West||||Anson County||24244|United States||| +10947|AAAAAAAADMKCAAAA|148|10th Birch|Parkway|Suite P|Antioch|Houston County|GA|38605|United States|-5|apartment| +10948|AAAAAAAAEMKCAAAA|411|Walnut |Court|Suite 340|Liberty|Trinity County|TX|73451|United States|-6|single family| +10949|AAAAAAAAFMKCAAAA|425|Sunset |Ln|Suite O|Mount Vernon|Denali Borough|AK|98482|United States|-9|condo| +10950|AAAAAAAAGMKCAAAA|507|Wilson Sunset|Lane|Suite H|Harmony|Pleasants County|WV|25804|United States|-5|apartment| +10951|AAAAAAAAHMKCAAAA|496|Sunset 1st|RD|Suite I|Tipton|Harrison County|MO|68773|United States|-6|condo| +10952|AAAAAAAAIMKCAAAA|706|Church |Way|Suite E|Summit|Morgan County|IL|60499|United States|-6|apartment| +10953|AAAAAAAAJMKCAAAA|626|Fourth Valley|Blvd|Suite V|Lewis|Grant County|ND|57066|United States|-6|single family| +10954|AAAAAAAAKMKCAAAA|575|6th First|Drive|Suite L|Westwood|Cullman County|AL|31715|United States|-6|condo| +10955|AAAAAAAALMKCAAAA|751|Elevnth |ST|Suite 480|Crossroads|Armstrong County|TX|70534|United States|-6|condo| +10956|AAAAAAAAMMKCAAAA|951|6th |RD|Suite 50|Hillcrest|Goochland County|VA|23003|United States|-5|apartment| +10957|AAAAAAAANMKCAAAA|880|Lee 4th|Drive|Suite Y|Waterloo|Lawrence County|AR|71675|United States|-6|apartment| +10958|AAAAAAAAOMKCAAAA|549|3rd |RD|Suite 140|Cedar Grove|Des Moines County|IA|50411|United States|-6|apartment| +10959|AAAAAAAAPMKCAAAA|897|Jackson |Cir.|Suite 90|Oak Ridge|Smith County|KS|68371|United States|-6|condo| +10960|AAAAAAAAANKCAAAA|945|Washington Elm|Blvd|Suite 350|Spring Hill|Edmunds County|SD|56787|United States|-6|apartment| +10961|AAAAAAAABNKCAAAA|61|7th |Boulevard|Suite 310|Enterprise|Tipton County|IN|41757|United States|-5|single family| +10962|AAAAAAAACNKCAAAA|455|College Walnut|Dr.|Suite 20|Mount Vernon|Owsley County|KY|48482|United States|-5|apartment| +10963|AAAAAAAADNKCAAAA|590|East First|Cir.|Suite S|Shady Grove|Turner County|GA|32812|United States|-5|apartment| +10964|AAAAAAAAENKCAAAA|515|Willow 9th|Cir.|Suite S|Lakeside|North Slope Borough|AK|99532|United States|-9|condo| +10965|AAAAAAAAFNKCAAAA|497|Second |Way|Suite L|Oakdale|Marion County|IL|69584|United States|-6|condo| +10966|AAAAAAAAGNKCAAAA|469|5th South|RD|Suite N|Chestnut Ridge|Saline County|KS|67334|United States|-6|apartment| +10967|AAAAAAAAHNKCAAAA|497|10th |ST|Suite N|Lakewood|Spotsylvania County|VA|28877|United States|-5|condo| +10968|AAAAAAAAINKCAAAA|930|West Elm|Court|Suite 380|Shiloh|Hyde County|NC|29275|United States|-5|apartment| +10969|AAAAAAAAJNKCAAAA|410|Sixth Elm|Circle|Suite Y|Kingston|Cimarron County|OK|74975|United States|-6|condo| +10970|AAAAAAAAKNKCAAAA||2nd |Parkway|Suite C||Steuben County|NY|14136|||single family| +10971|AAAAAAAALNKCAAAA|926|12th |Avenue|Suite K|Caledonia|Lee County|AL|37411|United States|-6|condo| +10972|AAAAAAAAMNKCAAAA|17|Ash Jackson|ST|Suite 120|Lakeside|Menard County|IL|69532|United States|-6|condo| +10973|AAAAAAAANNKCAAAA|514|Railroad |Road|Suite F|Ashland|Crittenden County|KY|44244|United States|-6|condo| +10974|AAAAAAAAONKCAAAA|298|Oak Main|Circle|Suite O|Providence|Wells County|ND|56614|United States|-6|apartment| +10975|AAAAAAAAPNKCAAAA|730|Willow Third|Dr.|Suite 320|Glenwood|Henrico County|VA|23511|United States|-5|apartment| +10976|AAAAAAAAAOKCAAAA|628|Willow Meadow|Lane|Suite Q|Oak Ridge|Sauk County|WI|58371|United States|-6|condo| +10977|AAAAAAAABOKCAAAA|729|Main |Parkway|Suite 180|Woodville|Washington County|MN|54289|United States|-6|apartment| +10978|AAAAAAAACOKCAAAA|506|Wilson Maple|Boulevard|Suite N|Greenfield|Yuba County|CA|95038|United States|-8|condo| +10979|AAAAAAAADOKCAAAA|395|Jackson Ridge|Pkwy|Suite P|Sulphur Springs|Houston County|TN|38354|United States|-5|apartment| +10980|AAAAAAAAEOKCAAAA|231|Center |Boulevard|Suite H|Oak Grove|Dewey County|SD|58370|United States|-6|condo| +10981|AAAAAAAAFOKCAAAA|370|||Suite G|Georgetown|Kit Carson County|CO|87057|||apartment| +10982|AAAAAAAAGOKCAAAA|738|Main Ninth|Dr.|Suite Y|Liberty|Blackford County|IN|43451|United States|-5|apartment| +10983|AAAAAAAAHOKCAAAA|604|Washington Lincoln|Cir.|Suite 110|Fairview|Warren County|PA|15709|United States|-5|apartment| +10984|AAAAAAAAIOKCAAAA|265|Washington 1st|Ave|Suite L|Fairfield|Haralson County|GA|36192|United States|-5|condo| +10985|AAAAAAAAJOKCAAAA|918|Sunset |Street|Suite R|Greenfield|Throckmorton County|TX|75038|United States|-6|condo| +10986|AAAAAAAAKOKCAAAA|454|Main |Ln|Suite 20|Enterprise|Madera County|CA|91757|United States|-8|single family| +10987|AAAAAAAALOKCAAAA|653|Woodland 6th|Blvd|Suite 480|Concord|Jefferson County|WI|54107|United States|-6|single family| +10988|AAAAAAAAMOKCAAAA|562|Forest |Drive|Suite I|Five Forks|Washington County|GA|32293|United States|-5|single family| +10989|AAAAAAAANOKCAAAA|327|Park 6th|Street|Suite 240|Bethel|Defiance County|OH|45281|United States|-5|apartment| +10990|AAAAAAAAOOKCAAAA|963|Maple Meadow|Court|Suite 230|Vista|Fulton County|OH|44694|United States|-5|apartment| +10991|AAAAAAAAPOKCAAAA|492|Pine 12th|Road|Suite I|Greenville|Frontier County|NE|61387|United States|-6|condo| +10992|AAAAAAAAAPKCAAAA|538|Center Lincoln|Way|Suite R|Greenwood|Warren County|MO|68828|United States|-6|condo| +10993|AAAAAAAABPKCAAAA|763|Spring |Court|Suite J|Mount Olive|Barry County|MO|68059|United States|-6|single family| +10994|AAAAAAAACPKCAAAA|550|Oak |Drive|Suite T|Bridgeport|Mercer County|IL|65817|United States|-6|single family| +10995|AAAAAAAADPKCAAAA||||Suite O|Union Hill|Chowan County|NC||United States||| +10996|AAAAAAAAEPKCAAAA|291|Mill 4th|Court|Suite 80|Newport|Las Animas County|CO|81521|United States|-7|single family| +10997|AAAAAAAAFPKCAAAA|210|Oak |Lane|Suite 360|Clinton|Palo Pinto County|TX|78222|United States|-6|condo| +10998|AAAAAAAAGPKCAAAA|403|Jackson Smith|ST|Suite 30|Unionville|Carter County|OK|71711|United States|-6|condo| +10999|AAAAAAAAHPKCAAAA||Eigth |Street||Georgetown|Vermilion Parish||77057|United States|-6|| +11000|AAAAAAAAIPKCAAAA|496|Lake |Blvd|Suite 40|Shiloh|Jackson County|TX|79275|United States|-6|apartment| +11001|AAAAAAAAJPKCAAAA|997|River |Boulevard|Suite K|Woodland|Noxubee County|MS|54854|United States|-6|apartment| +11002|AAAAAAAAKPKCAAAA|989|Cedar |Pkwy|Suite H|Mountain View|Pontotoc County|OK|74466|United States|-6|apartment| +11003|AAAAAAAALPKCAAAA|65|Madison 4th|ST|Suite 340|Florence|Fayette County|PA|13394|United States|-5|condo| +11004|AAAAAAAAMPKCAAAA|133|Spruce Jefferson|Court|Suite U|Centerville|Wichita County|KS|60059|United States|-6|condo| +11005|AAAAAAAANPKCAAAA|549|Chestnut Ash|ST|Suite 150|Union|Montcalm County|MI|48721|United States|-5|condo| +11006|AAAAAAAAOPKCAAAA|212|Chestnut 2nd|RD|Suite A|Bunker Hill|Vanderburgh County|IN|40150|United States|-5|apartment| +11007|AAAAAAAAPPKCAAAA|754|Willow |Boulevard|Suite 80|Mountain View|Mayes County|OK|74466|United States|-6|apartment| +11008|AAAAAAAAAALCAAAA|699|Main 6th|Dr.|Suite 490|Jamestown|Custer County|MT|66867|United States|-7|condo| +11009|AAAAAAAABALCAAAA|459|View Twelfth|Drive|Suite J|Bunker Hill|Summers County|WV|20150|United States|-5|apartment| +11010|AAAAAAAACALCAAAA|379|Sunset North|Boulevard|Suite 230|Kingston|Irion County|TX|74975|United States|-6|single family| +11011|AAAAAAAADALCAAAA|452|View 4th|Street|Suite B|Glenwood|Wasatch County|UT|83511|United States|-7|single family| +11012|AAAAAAAAEALCAAAA|383|6th Johnson|Ave|Suite L|New Hope|Outagamie County|WI|59431|United States|-6|condo| +11013|AAAAAAAAFALCAAAA|630|7th |Avenue|Suite Y|Summit|Seneca County|OH|40499|United States|-5|single family| +11014|AAAAAAAAGALCAAAA|65|Pine Cedar|Wy|Suite 340|Spring Hill|Carbon County|UT|86787|United States|-7|condo| +11015|AAAAAAAAHALCAAAA|706|Woodland |Way|Suite 230|White Oak|Evans County|GA|36668|United States|-5|apartment| +11016|AAAAAAAAIALCAAAA|668|3rd |Ave|Suite 390|Mount Zion|McMinn County|TN|38054|United States|-6|single family| +11017|AAAAAAAAJALCAAAA|647|Chestnut Chestnut|Ave|Suite G|Newport|Benton County|IN|41521|United States|-5|single family| +11018|AAAAAAAAKALCAAAA|153|11th Green|Road|Suite 370|Oak Ridge|Mesa County|CO|88371|United States|-7|apartment| +11019|AAAAAAAALALCAAAA|554|Sunset 15th|Ave|Suite F|Ashland|Blaine County|NE|64244|United States|-6|condo| +11020|AAAAAAAAMALCAAAA|692|Third |Parkway|Suite S|Friendship|Bayfield County|WI|54536|United States|-6|condo| +11021|AAAAAAAANALCAAAA|500|2nd |Boulevard|Suite 200|Concord|Boone County|WV|24107|United States|-5|single family| +11022|AAAAAAAAOALCAAAA|863|Lake |Blvd|Suite A|Shiloh|Union County|MS|59275|United States|-6|single family| +11023|AAAAAAAAPALCAAAA|544|15th |Way|Suite S|Spring Hill|Stephens County|TX|76787|United States|-6|apartment| +11024|AAAAAAAAABLCAAAA|355|Sunset |ST|Suite A|Forest Hills|Monroe County|OH|49237|United States|-5|condo| +11025|AAAAAAAABBLCAAAA|840|Church |Street|Suite 480|Brownsville|Polk County|WI|59310|United States|-6|condo| +11026|AAAAAAAACBLCAAAA|81|Center |Court|Suite 480|Red Hill|Comanche County|OK|74338|United States|-6|apartment| +11027|AAAAAAAADBLCAAAA|220|1st |Court|Suite 280|Oak Hill|Montgomery County|AL|37838|United States|-6|apartment| +11028|AAAAAAAAEBLCAAAA|828|||Suite J||Prairie County|MT|||-7|| +11029|AAAAAAAAFBLCAAAA|815|Dogwood |Way|Suite U|Franklin|Mellette County|SD|59101|United States|-7|apartment| +11030|AAAAAAAAGBLCAAAA|377|12th Church|Ct.|Suite V|Red Hill|Montgomery County|IN|44338|United States|-5|single family| +11031|AAAAAAAAHBLCAAAA|43|Park 13th|Drive|Suite 330|Hardy|Terry County|TX|75354|United States|-6|condo| +11032|AAAAAAAAIBLCAAAA|390|Fourth Davis|Drive|Suite J|Georgetown|Warren County|OH|47057|United States|-5|apartment| +11033|AAAAAAAAJBLCAAAA|163|14th South|Cir.|Suite 100|Woodlawn|Mountrail County|ND|54098|United States|-6|condo| +11034|AAAAAAAAKBLCAAAA|272|Third Main|Drive|Suite S|Woodland|Clinton County|IA|54854|United States|-6|single family| +11035|AAAAAAAALBLCAAAA|441|Main 2nd|Way|Suite C|Bunker Hill|Isle of Wight County|VA|20150|United States|-5|condo| +11036|AAAAAAAAMBLCAAAA|63|Davis Williams|Court|Suite 160|Westgate|Clarke County|MS|52366|United States|-6|apartment| +11037|AAAAAAAANBLCAAAA|238|Lakeview 5th|Wy|Suite 140|Pleasant Hill|Montgomery County|PA|13604|United States|-5|apartment| +11038|AAAAAAAAOBLCAAAA|228|River Davis|Wy|Suite 180|Woodlawn|McCracken County|KY|44098|United States|-5|single family| +11039|AAAAAAAAPBLCAAAA|611|Elm |Parkway|Suite 460|Florence|Porter County|IN|43394|United States|-5|single family| +11040|AAAAAAAAACLCAAAA|388|3rd Center|Street|Suite 170|Highland|Coos County|OR|99454|United States|-8|condo| +11041|AAAAAAAABCLCAAAA|683|Cherry |Dr.|Suite 210|Midway|Harrison County|KY|41904|United States|-6|apartment| +11042|AAAAAAAACCLCAAAA|683|Second |Parkway|Suite 470|Smith|Franklin County|IN|47317|United States|-5|apartment| +11043|AAAAAAAADCLCAAAA|904|Center |Drive|Suite O|Hopewell|Jefferson County|TN|30587|United States|-6|condo| +11044|AAAAAAAAECLCAAAA|371|Maple |Circle|Suite 430|Walnut Grove|Walker County|TX|77752|United States|-6|condo| +11045|AAAAAAAAFCLCAAAA|677|Walnut Elm|Wy|Suite 230|Bethel|Marshall County|IL|65281|United States|-6|single family| +11046|AAAAAAAAGCLCAAAA|788|Main 5th|Road|Suite 490|Hopewell|Colquitt County|GA|30587|United States|-5|single family| +11047|AAAAAAAAHCLCAAAA|339|Miller |Street|Suite B|Antioch|Harrison County|OH|48605|United States|-5|single family| +11048|AAAAAAAAICLCAAAA|690|Spring |RD|Suite M|Oakwood|Greeley County|NE|60169|United States|-6|apartment| +11049|AAAAAAAAJCLCAAAA|50|Main Mill|Circle|Suite X|Woodville|Halifax County|VA|24289|United States|-5|apartment| +11050|AAAAAAAAKCLCAAAA|975|View |Street|Suite 340|Union|Brookings County|SD|58721|United States|-6|condo| +11051|AAAAAAAALCLCAAAA|330|Miller Hickory|Ave|Suite 450|Jamestown|Greene County|PA|16867|United States|-5|apartment| +11052|AAAAAAAAMCLCAAAA|406|Main |Pkwy|Suite H|Jackson|Los Alamos County|NM|89583|United States|-7|apartment| +11053|AAAAAAAANCLCAAAA|50|5th |Ct.|Suite B|Plainview|Umatilla County|OR|93683|United States|-8|condo| +11054|AAAAAAAAOCLCAAAA|248|Park |Road|Suite 110|Green Acres|Richland County|IL|67683|United States|-6|condo| +11055|AAAAAAAAPCLCAAAA|987|Eigth Cherry|Pkwy|Suite 380|Mount Pleasant|Cotton County|OK|71933|United States|-6|apartment| +11056|AAAAAAAAADLCAAAA|382|Hickory South|ST|Suite 290|Mount Zion|Yellow Medicine County|MN|58054|United States|-6|apartment| +11057|AAAAAAAABDLCAAAA|452|Cedar Sycamore|Drive|Suite O|Millbrook|Washington County|RI|08129|United States|-5|condo| +11058|AAAAAAAACDLCAAAA||Birch 5th|||Riverdale||MT|69391|United States||| +11059|AAAAAAAADDLCAAAA|165|Central |Cir.|Suite 310|Shady Grove|Lee County|AL|32812|United States|-6|apartment| +11060|AAAAAAAAEDLCAAAA|832|Adams |Cir.|Suite 50|Bunker Hill|Coke County|TX|70150|United States|-6|single family| +11061|AAAAAAAAFDLCAAAA|968|8th |Lane|Suite W|Macedonia|Traverse County|MN|51087|United States|-6|apartment| +11062|AAAAAAAAGDLCAAAA|860|Mill |Parkway|Suite 440|Liberty|Dorchester County|SC|23451|United States|-5|single family| +11063|AAAAAAAAHDLCAAAA|75|Adams |Street|Suite 260|Cedar Grove|Orange County|TX|70411|United States|-6|single family| +11064|AAAAAAAAIDLCAAAA|269|Maple Sixth|Street|Suite N|Farmington|Red Willow County|NE|69145|United States|-7|single family| +11065|AAAAAAAAJDLCAAAA|750|Tenth |Parkway|Suite X|Newton|Bureau County|IL|61345|United States|-6|single family| +11066|AAAAAAAAKDLCAAAA|490|10th |Ct.|Suite 280|Sheffield|Grainger County|TN|36896|United States|-5|single family| +11067|AAAAAAAALDLCAAAA|540|Hickory Spruce|RD|Suite 330|Ashland|Lawrence County|OH|44244|United States|-5|condo| +11068|AAAAAAAAMDLCAAAA|476|Main Locust|Blvd|Suite Q|Tipton|Webster County|NE|68773|United States|-7|condo| +11069|AAAAAAAANDLCAAAA|555|Spring North|ST|Suite O|Lakewood|Montgomery County|KS|68877|United States|-6|apartment| +11070|AAAAAAAAODLCAAAA|937|3rd Main|Ave|Suite 200|Hillcrest|Marion County|TN|33003|United States|-6|apartment| +11071|AAAAAAAAPDLCAAAA|847|Pine |ST|Suite 210|Griffin|Park County|MT|61204|United States|-7|single family| +11072|AAAAAAAAAELCAAAA|758|Adams Third|Ct.|Suite 290|Salem|Hamilton County|FL|38048|United States|-5|condo| +11073|AAAAAAAABELCAAAA|870|Ridge Adams|Ln|Suite 480|Wildwood|Jackson County|OH|46871|United States|-5|single family| +11074|AAAAAAAACELCAAAA|957|Spruce 6th|Parkway|Suite 140|Enterprise|Snyder County|PA|11757|United States|-5|single family| +11075|AAAAAAAADELCAAAA|710|Central 7th|Cir.|Suite 230|Green Acres|Seminole County|GA|37683|United States|-5|condo| +11076|AAAAAAAAEELCAAAA|491|Park |Parkway|Suite W|Cedar Grove|Albany County|WY|80411|United States|-7|apartment| +11077|AAAAAAAAFELCAAAA|817|First |Dr.|Suite 60|Glenwood|Louisa County|VA|23511|United States|-5|condo| +11078|AAAAAAAAGELCAAAA|882|Davis Pine|Pkwy|Suite 120|Summit|Whitley County|KY|40499|United States|-5|condo| +11079|AAAAAAAAHELCAAAA|912|8th |Dr.|Suite X|Jamestown|Ellsworth County|KS|66867|United States|-6|apartment| +11080|AAAAAAAAIELCAAAA|622|9th |Ln|Suite 360|Clifton|Macon County|TN|38014|United States|-6|single family| +11081|AAAAAAAAJELCAAAA|837|Highland |Pkwy|Suite L|Providence|Williamsburg County|SC|26614|United States|-5|apartment| +11082|AAAAAAAAKELCAAAA|2|Railroad 2nd|Road|Suite 160|Shady Grove|Washington County|ID|82812|United States|-7|single family| +11083|AAAAAAAALELCAAAA|253|Davis 2nd|Road|Suite 210|Littleton|Robertson County|TN|36074|United States|-6|single family| +11084|AAAAAAAAMELCAAAA|649|Fourth |Pkwy|Suite R|Sutton|Hertford County|NC|25413|United States|-5|single family| +11085|AAAAAAAANELCAAAA|524|Hill |ST|Suite 190|Spring Hill|Lewis County|WV|26787|United States|-5|single family| +11086|AAAAAAAAOELCAAAA|421|Walnut |Pkwy|Suite S|Ashton|Champaign County|IL|69981|United States|-6|condo| +11087|AAAAAAAAPELCAAAA|488|Oak |Wy|Suite 430|Woodlawn|Martin County|IN|44098|United States|-5|apartment| +11088|AAAAAAAAAFLCAAAA|187|Johnson |Cir.|Suite 480|Fairview|San Saba County|TX|75709|United States|-6|condo| +11089|AAAAAAAABFLCAAAA|876|Church 1st|Cir.|Suite 310|Oakland|Perry County|TN|39843|United States|-6|condo| +11090|AAAAAAAACFLCAAAA|951|4th Main|Boulevard|Suite 120|Clinton|Noble County|OH|48222|United States|-5|condo| +11091|AAAAAAAADFLCAAAA|160|Johnson Park||Suite H||||57671|United States||| +11092|AAAAAAAAEFLCAAAA|288|Tenth Main|Lane|Suite 410|Lakeside|Cass County|MN|59532|United States|-6|condo| +11093|AAAAAAAAFFLCAAAA|515|6th |Parkway|Suite J|Howell|Lawrence County|IN|44854|United States|-5|single family| +11094|AAAAAAAAGFLCAAAA|879|Pine |Blvd|Suite 180|Edgewood|Edmunds County|SD|50069|United States|-6|apartment| +11095|AAAAAAAAHFLCAAAA|520|7th |ST|Suite P|Fairview|Marshall County|IA|55709|United States|-6|condo| +11096|AAAAAAAAIFLCAAAA|215|6th |Parkway|Suite 320|Lone Oak|Broadwater County|MT|66893|United States|-7|apartment| +11097|AAAAAAAAJFLCAAAA|473|Locust Sycamore|Cir.|Suite 70|Mountain View|McIntosh County|OK|74466|United States|-6|condo| +11098|AAAAAAAAKFLCAAAA|364||Lane|Suite 330|Salem|Fairbanks North Star Borough|||||apartment| +11099|AAAAAAAALFLCAAAA||||Suite 180||Wayne County||13451|||| +11100|AAAAAAAAMFLCAAAA|824|Park |Cir.|Suite I|Springfield|Buncombe County|NC|29303|United States|-5|apartment| +11101|AAAAAAAANFLCAAAA|116|Locust |Ct.|Suite T|Enterprise|Osceola County|MI|41757|United States|-5|condo| +11102|AAAAAAAAOFLCAAAA||Birch Church||Suite 420||Adams County|IN||United States||| +11103|AAAAAAAAPFLCAAAA|512|1st Center|Lane|Suite U|Union Hill|Vermilion County|IL|67746|United States|-6|condo| +11104|AAAAAAAAAGLCAAAA|556|Smith |Avenue|Suite R|Mountain View|Randolph County|IL|64466|United States|-6|single family| +11105|AAAAAAAABGLCAAAA|830|College |Street|Suite T|Greenfield|Halifax County|VA|25038|United States|-5|single family| +11106|AAAAAAAACGLCAAAA|469|Fourth |Parkway|Suite 80|Oakland|Washington County|MO|69843|United States|-6|condo| +11107|AAAAAAAADGLCAAAA|716|12th |Lane|Suite 210|Newport|Pacific County|WA|91521|United States|-8|single family| +11108|AAAAAAAAEGLCAAAA|577|Ninth |Pkwy|Suite 40|Glendale|Schleicher County|TX|73951|United States|-6|single family| +11109|AAAAAAAAFGLCAAAA|209|Lakeview |Court|Suite H|Owens|Broome County|NY|12324|United States|-5|condo| +11110|AAAAAAAAGGLCAAAA|302|14th Sycamore|Pkwy|Suite 80|Oakwood|Sierra County|CA|90169|United States|-8|condo| +11111|AAAAAAAAHGLCAAAA|347|Spring Ridge|Pkwy|Suite S|Mount Pleasant|Elmore County|ID|81933|United States|-7|apartment| +11112|AAAAAAAAIGLCAAAA|655|Main |Circle|Suite T|Hidden Valley|Breathitt County|KY|45521|United States|-6|apartment| +11113|AAAAAAAAJGLCAAAA|318|South Davis|Cir.|Suite 130|Liberty|Pulaski County|VA|23451|United States|-5|apartment| +11114|AAAAAAAAKGLCAAAA|685|9th |Ln|Suite 320|Valley View|Lowndes County|MS|55124|United States|-6|single family| +11115|AAAAAAAALGLCAAAA|547|Hickory Cedar|Ave|Suite 170|Pleasant Hill|Shelby County|KY|43604|United States|-5|single family| +11116|AAAAAAAAMGLCAAAA|369|15th |Blvd|Suite 320|Sulphur Springs|Shenandoah County|VA|28354|United States|-5|single family| +11117|AAAAAAAANGLCAAAA|809|Mill 3rd|Boulevard|Suite J|Cedar Grove|Weston County|WY|80411|United States|-7|condo| +11118|AAAAAAAAOGLCAAAA|440|Mill |Drive|Suite V|Walnut Grove|Arlington County|VA|27752|United States|-5|condo| +11119|AAAAAAAAPGLCAAAA|757|Walnut |ST|Suite Q|Brownsville|Chippewa County|MN|59310|United States|-6|condo| +11120|AAAAAAAAAHLCAAAA|522|Elm |Way|Suite 40|Kingston|Mono County|CA|94975|United States|-8|apartment| +11121|AAAAAAAABHLCAAAA|267|Broadway |Drive|Suite 190|Antioch|Monroe County|NY|18605|United States|-5|single family| +11122|AAAAAAAACHLCAAAA|105|Main |Parkway|Suite G|Siloam|Franklin County|AL|38948|United States|-6|apartment| +11123|AAAAAAAADHLCAAAA|874|Lincoln Ash|Way|Suite N|Liberty|Poquoson city|VA|23451|United States|-5|single family| +11124|AAAAAAAAEHLCAAAA|619|Maple |Ln|Suite 40|Deerfield|Dixon County|NE|69840|United States|-6|single family| +11125|AAAAAAAAFHLCAAAA|500|River |Ct.|Suite 110|Jackson|Lyon County|KS|69583|United States|-6|condo| +11126|AAAAAAAAGHLCAAAA|298|Hickory |Street|Suite X|Chestnut Ridge|Gwinnett County|GA|37334|United States|-5|apartment| +11127|AAAAAAAAHHLCAAAA|273|Woodland Sunset|ST|Suite 460|Glenwood|Buncombe County|NC|23511|United States|-5|apartment| +11128|AAAAAAAAIHLCAAAA||Lincoln |||Oak Grove|Crook County|WY|88370|||| +11129|AAAAAAAAJHLCAAAA|475|North |Wy|Suite 280|Marion|Union County|MS|50399|United States|-6|apartment| +11130|AAAAAAAAKHLCAAAA|657|Miller |Drive|Suite G|Spring Valley|Powell County|KY|46060|United States|-5|apartment| +11131|AAAAAAAALHLCAAAA|43|Locust |Cir.|Suite Q|Waterloo|Cass County|MI|41675|United States|-5|single family| +11132|AAAAAAAAMHLCAAAA|957|Railroad |Road|Suite H|Bridgeport|Hennepin County|MN|55817|United States|-6|single family| +11133|AAAAAAAANHLCAAAA|191|Mill Sycamore|Ct.|Suite X|Franklin|Guilford County|NC|29101|United States|-5|single family| +11134|AAAAAAAAOHLCAAAA|988|Seventh |Road|Suite 140|Newport|Aroostook County|ME|02121|United States|-5|apartment| +11135|AAAAAAAAPHLCAAAA|972|1st |ST|Suite 380|Edgewood|Amador County|CA|90069|United States|-8|apartment| +11136|AAAAAAAAAILCAAAA|242|Pine Oak|Ln|Suite G|Mount Vernon|Lamoille County|VT|09082|United States|-5|condo| +11137|AAAAAAAABILCAAAA|98|Hickory 3rd|Street|Suite M|Spring Hill|Ralls County|MO|66787|United States|-6|condo| +11138|AAAAAAAACILCAAAA|286|East Elm|Ave|Suite 180|Walnut Grove|Madison County|NE|67752|United States|-7|apartment| +11139|AAAAAAAADILCAAAA|410|Wilson |Wy|Suite 370|Franklin|Indiana County|PA|19101|United States|-5|condo| +11140|AAAAAAAAEILCAAAA|84|Oak |Wy|Suite 360|Newtown|Boise County|ID|81749|United States|-7|apartment| +11141|AAAAAAAAFILCAAAA|368|14th Main|ST|Suite 270|Deerfield|Winneshiek County|IA|59840|United States|-6|condo| +11142|AAAAAAAAGILCAAAA|705||||Fairfield|Dale County|AL|36192|||| +11143|AAAAAAAAHILCAAAA|980|Twelfth |Ln|Suite R|Lakeside|McPherson County|KS|69532|United States|-6|single family| +11144|AAAAAAAAIILCAAAA|67|3rd Pine|Boulevard|Suite 360|Salem|Isle of Wight County|VA|28048|United States|-5|condo| +11145|AAAAAAAAJILCAAAA|596|14th |Dr.|Suite R|Springfield|Flagler County|FL|39303|United States|-5|single family| +11146|AAAAAAAAKILCAAAA|158|Willow |Parkway|Suite Q|Riverside|Stephens County|OK|79231|United States|-6|single family| +11147|AAAAAAAALILCAAAA|132|Pine |Lane|Suite U|Marion|Shelby County|KY|40399|United States|-5|condo| +11148|AAAAAAAAMILCAAAA|715|9th |Lane|Suite S|Hopewell|Morgan County|MO|60587|United States|-6|apartment| +11149|AAAAAAAANILCAAAA|19|South |Wy|Suite G|Liberty|Baxter County|AR|73451|United States|-6|single family| +11150|AAAAAAAAOILCAAAA|188|Williams |ST|Suite 160|Woodlawn|Shiawassee County|MI|44098|United States|-5|apartment| +11151|AAAAAAAAPILCAAAA|856|Pine Willow|Lane|Suite P|Salem|Socorro County|NM|88048|United States|-7|single family| +11152|AAAAAAAAAJLCAAAA|733|Ridge Laurel||Suite N|Hopewell|Sherburne County|||||| +11153|AAAAAAAABJLCAAAA||15th |Road|Suite 190|||NV||United States||apartment| +11154|AAAAAAAACJLCAAAA|163|Jackson Wilson|Drive|Suite 330|Edgewood|Barry County|MI|40069|United States|-5|single family| +11155|AAAAAAAADJLCAAAA|301|Willow |Boulevard|Suite 280|Cedar Grove|Mecklenburg County|VA|20411|United States|-5|single family| +11156|AAAAAAAAEJLCAAAA|912|Sixth ||Suite B|Enterprise|||21757|United States||condo| +11157|AAAAAAAAFJLCAAAA|822|Washington First|ST|Suite 200|Oakdale|Caldwell County|MO|69584|United States|-6|single family| +11158|AAAAAAAAGJLCAAAA|499|Wilson Park|Circle|Suite Q|Wilson|Giles County|TN|36971|United States|-5|apartment| +11159|AAAAAAAAHJLCAAAA|988|14th |Cir.|Suite L|Owens|Sullivan County|IN|42324|United States|-5|condo| +11160|AAAAAAAAIJLCAAAA|868|Lincoln |Blvd|Suite A|Riverdale|McDonald County|MO|69391|United States|-6|single family| +11161|AAAAAAAAJJLCAAAA|613|College Walnut|Circle|Suite V|Concord|Burleigh County|ND|54107|United States|-6|condo| +11162|AAAAAAAAKJLCAAAA|97|6th |Court|Suite 10|Union|Dunn County|ND|58721|United States|-6|single family| +11163|AAAAAAAALJLCAAAA|189|Fifteenth |Road|Suite V|Spring Valley|Elbert County|GA|36060|United States|-5|condo| +11164|AAAAAAAAMJLCAAAA|334|View |Boulevard|Suite 410|Edgewood|Livingston Parish|LA|70069|United States|-6|apartment| +11165|AAAAAAAANJLCAAAA|371|Ash 13th|Ave|Suite 270|Kingston|Taos County|NM|84975|United States|-7|condo| +11166|AAAAAAAAOJLCAAAA|123|Green Mill|RD|Suite L|Pleasant Valley|Briscoe County|TX|72477|United States|-6|apartment| +11167|AAAAAAAAPJLCAAAA|979|Spring |Pkwy|Suite 110|Tanglewood|Marshall County|TN|38994|United States|-6|single family| +11168|AAAAAAAAAKLCAAAA|841|3rd Franklin|Blvd|Suite U|Providence|Shelby County|TX|76614|United States|-6|single family| +11169|AAAAAAAABKLCAAAA|91|7th |Way|Suite 360|Macon|Shelby County|AL|30369|United States|-6|condo| +11170|AAAAAAAACKLCAAAA|985|Center Pine|Parkway|Suite I|Leesville|Fulton County|GA|35423|United States|-5|single family| +11171|AAAAAAAADKLCAAAA|263|2nd |ST|Suite 420|Union|Woods County|OK|78721|United States|-6|condo| +11172|AAAAAAAAEKLCAAAA|50|15th Forest|Wy|Suite 220|Sunnyside|Weakley County|TN|31952|United States|-6|condo| +11173|AAAAAAAAFKLCAAAA|816|Center Church|Cir.|Suite 390|Glenwood|Tehama County|CA|93511|United States|-8|single family| +11174|AAAAAAAAGKLCAAAA||||Suite 390|Five Points||NY|||-5|| +11175|AAAAAAAAHKLCAAAA|372|View Poplar|Court|Suite 0|Oak Grove|Clinton County|IA|58370|United States|-6|condo| +11176|AAAAAAAAIKLCAAAA|240|3rd |Circle|Suite O|Lakeview|Alfalfa County|OK|78579|United States|-6|apartment| +11177|AAAAAAAAJKLCAAAA|204|Forest |Way|Suite 200|Marion|Platte County|WY|80399|United States|-7|apartment| +11178|AAAAAAAAKKLCAAAA|636|Spring Elevnth|Road|Suite B|Oak Hill|Edgefield County|SC|27838|United States|-5|single family| +11179|AAAAAAAALKLCAAAA|643|4th |Way|Suite D|Glenwood|Franklin County|AL|33511|United States|-6|single family| +11180|AAAAAAAAMKLCAAAA|401|Green |Way|Suite J|Crossroads|Vernon County|MO|60534|United States|-6|single family| +11181|AAAAAAAANKLCAAAA|985|Center Spring|Boulevard|Suite 140|Lebanon|Newport County|RI|03498|United States|-5|single family| +11182|AAAAAAAAOKLCAAAA|279||Street|Suite 110||||70534|United States|-6|| +11183|AAAAAAAAPKLCAAAA|602|Church |Ln|Suite X|Red Hill|Burke County|ND|54338|United States|-6|single family| +11184|AAAAAAAAALLCAAAA|632|Walnut River||Suite 320||Wabasha County|MN|56060|||single family| +11185|AAAAAAAABLLCAAAA|231|Jackson View|Wy|Suite U|Oakland|Cabell County|WV|29843|United States|-5|single family| +11186|AAAAAAAACLLCAAAA|581|Woodland |Boulevard|Suite 380|Jackson|Lexington County|SC|29583|United States|-5|apartment| +11187|AAAAAAAADLLCAAAA|264|Sunset Meadow|Street|Suite 310|Brownsville|Hardin County|KY|49310|United States|-6|apartment| +11188|AAAAAAAAELLCAAAA|||Way|Suite 180|||||United States|-5|| +11189|AAAAAAAAFLLCAAAA|166|Dogwood |Dr.|Suite 280|Summit|Northwest Arctic Borough|AK|90499|United States|-9|apartment| +11190|AAAAAAAAGLLCAAAA|15|10th |RD|Suite G|Aberdeen|Gilmer County|WV|27995|United States|-5|condo| +11191|AAAAAAAAHLLCAAAA|43|Center |Avenue|Suite 450|Edgewood|Van Buren County|MI|40069|United States|-5|single family| +11192|AAAAAAAAILLCAAAA|59|Oak |Boulevard|Suite 430|Macedonia|Halifax County|VA|21087|United States|-5|single family| +11193|AAAAAAAAJLLCAAAA||4th ||Suite X|Wolf Creek||WA|92455|||condo| +11194|AAAAAAAAKLLCAAAA|847|Lincoln |Road|Suite J|Hamilton|Larimer County|CO|82808|United States|-7|single family| +11195|AAAAAAAALLLCAAAA|331|Park |RD|Suite 290|Belmont|Knox County|NE|60191|United States|-7|apartment| +11196|AAAAAAAAMLLCAAAA|145|8th 12th|Dr.|Suite X|Bridgeport|Lake County|SD|55817|United States|-7|single family| +11197|AAAAAAAANLLCAAAA|895|Hillcrest Birch|Lane|Suite 330|Enterprise|Bienville Parish|LA|71757|United States|-6|condo| +11198|AAAAAAAAOLLCAAAA|813|River Meadow|Way|Suite T|Bunker Hill|Fulton County|OH|40150|United States|-5|single family| +11199|AAAAAAAAPLLCAAAA|649|Church 3rd|Ave|Suite C|White Oak|Gates County|NC|26668|United States|-5|apartment| +11200|AAAAAAAAAMLCAAAA|265|College |Ave|Suite 370|Sulphur Springs|Lafayette County|FL|38354|United States|-5|condo| +11201|AAAAAAAABMLCAAAA|405|Highland |Road|Suite N|Spring Valley|Fayette County|IN|46060|United States|-5|single family| +11202|AAAAAAAACMLCAAAA|312|14th |Road|Suite 460|Gum Springs|Garfield County|UT|82106|United States|-7|apartment| +11203|AAAAAAAADMLCAAAA|418|Willow 9th|Blvd|Suite L|Spring Valley|Jackson County||56060|United States|-6|condo| +11204|AAAAAAAAEMLCAAAA|992|Second |Cir.|Suite 40|Jamestown|Coosa County|AL|36867|United States|-6|condo| +11205|AAAAAAAAFMLCAAAA|743|Church |Way|Suite L|Springfield|Jewell County|KS|69303|United States|-6|apartment| +11206|AAAAAAAAGMLCAAAA|658|First |Ln|Suite 310|Five Points|Edwards County|TX|76098|United States|-6|apartment| +11207|AAAAAAAAHMLCAAAA|612|Cherry 1st|Circle|Suite 180|Union Hill|Houston County|TX|77746|United States|-6|apartment| +11208|AAAAAAAAIMLCAAAA|668|Green |Wy||Springfield||LA|||-6|| +11209|AAAAAAAAJMLCAAAA|667|Lincoln |Wy|Suite I|Lebanon|Gregg County|TX|72898|United States|-6|condo| +11210|AAAAAAAAKMLCAAAA|634|Fifth Jackson|Circle|Suite 320|Spring Valley|Barrow County|GA|36060|United States|-5|condo| +11211|AAAAAAAALMLCAAAA|270|Ash |Street|Suite S|Antioch|Madison County|IN|48605|United States|-5|apartment| +11212|AAAAAAAAMMLCAAAA|572|5th Oak|Pkwy|Suite L|Flint|Jay County|IN|48909|United States|-5|single family| +11213|AAAAAAAANMLCAAAA|35|Sycamore 14th|Dr.|Suite L|Macedonia|Douglas County|WI|51087|United States|-6|single family| +11214|AAAAAAAAOMLCAAAA|424|Madison Highland|Boulevard|Suite Q|Pine Grove|Barren County|KY|44593|United States|-6|single family| +11215|AAAAAAAAPMLCAAAA|215|Cherry |Ln|Suite Y|Shady Grove|Lane County|OR|92812|United States|-8|single family| +11216|AAAAAAAAANLCAAAA|900|Hill Chestnut|Road|Suite N|Mount Olive|McIntosh County|ND|58059|United States|-6|single family| +11217|AAAAAAAABNLCAAAA|177|Ridge Spruce|Drive||Clearwater|||59534|United States||apartment| +11218|AAAAAAAACNLCAAAA|898|Lakeview |Cir.|Suite 430|Forest Hills|Iosco County|MI|49237|United States|-5|condo| +11219|AAAAAAAADNLCAAAA|156|Second Mill|Court|Suite 460|Crossroads|Del Norte County|CA|90534|United States|-8|single family| +11220|AAAAAAAAENLCAAAA|358|Mill |Avenue|Suite 10|Red Hill|Warren County|MO|64338|United States|-6|single family| +11221|AAAAAAAAFNLCAAAA|741|1st |Lane|Suite 140|Macedonia|Madison County|IL|61087|United States|-6|condo| +11222|AAAAAAAAGNLCAAAA|419|Willow 5th|RD|Suite 230|Edgewood|Green County|KY|40069|United States|-6|single family| +11223|AAAAAAAAHNLCAAAA|29|Johnson Woodland|Way|Suite L|Dallas|Roberts County|TX|73628|United States|-6|single family| +11224|AAAAAAAAINLCAAAA|414|Railroad 2nd|Boulevard|Suite 210|Antioch|Bernalillo County|NM|88605|United States|-7|apartment| +11225|AAAAAAAAJNLCAAAA|643|River Miller|Cir.|Suite 420|Ashland|Tillamook County|OR|94244|United States|-8|apartment| +11226|AAAAAAAAKNLCAAAA|56|Hill Sycamore|Pkwy|Suite 470|Mount Zion|Berkshire County|MA|08654|United States|-5|single family| +11227|AAAAAAAALNLCAAAA|80|6th |Ave|Suite H|Somerville|Butler County|KY|47783|United States|-6|condo| +11228|AAAAAAAAMNLCAAAA|287|South |Wy|Suite 430|Buena Vista|Montgomery County|PA|15752|United States|-5|condo| +11229|AAAAAAAANNLCAAAA|687|6th |Blvd|Suite V|Lakeview|Cass County|NE|68579|United States|-6|single family| +11230|AAAAAAAAONLCAAAA|766|5th |RD|Suite 470|New Hope|Morris County|KS|69431|United States|-6|condo| +11231|AAAAAAAAPNLCAAAA|905|Hickory |Circle|Suite 120|Butler|Jackson County|MI|46607|United States|-5|condo| +11232|AAAAAAAAAOLCAAAA|348|4th |Drive|Suite 0|Lakeside|Polk County|MO|69532|United States|-6|condo| +11233|AAAAAAAABOLCAAAA|190|Railroad 15th|Road|Suite 210|Union Hill|Chester County|SC|27746|United States|-5|condo| +11234|AAAAAAAACOLCAAAA|443|Spring Valley|Boulevard|Suite T|Valley View|Calhoun County|FL|35124|United States|-5|single family| +11235|AAAAAAAADOLCAAAA|52|Ninth 3rd|Wy|Suite A|Bridgeport|Lanier County|GA|35817|United States|-5|single family| +11236|AAAAAAAAEOLCAAAA|417|Poplar 3rd|ST|Suite N|Arlington|Lawrence County|IN|46557|United States|-5|condo| +11237|AAAAAAAAFOLCAAAA|933|Jackson Lake|Dr.|Suite 330|Florence|Monterey County|CA|93394|United States|-8|condo| +11238|AAAAAAAAGOLCAAAA|522|Lake 12th|Court|Suite 190|Concord|Hardin County|IA|54107|United States|-6|condo| +11239|AAAAAAAAHOLCAAAA|924|Elm Main|Circle|Suite 350|Florence|Lamar County|TX|73394|United States|-6|condo| +11240|AAAAAAAAIOLCAAAA|653|Pine Hill|Avenue|Suite 420|Perkins|Washburn County|WI|51852|United States|-6|single family| +11241|AAAAAAAAJOLCAAAA|127|7th 2nd|Wy|Suite D|Union|Elkhart County|IN|48721|United States|-5|single family| +11242|AAAAAAAAKOLCAAAA|346|Pine |Boulevard|Suite H|Mount Olive|Wilson County|TX|78059|United States|-6|condo| +11243|AAAAAAAALOLCAAAA|822|Williams |Court|Suite U|Bridgeport|Mississippi County|MO|65817|United States|-6|apartment| +11244|AAAAAAAAMOLCAAAA|402|Lincoln |Ln|Suite M|Crossroads|Custer County|ID|80534|United States|-7|apartment| +11245|AAAAAAAANOLCAAAA|366|College |Parkway|Suite 340|Farmington|Butler County|KS|69145|United States|-6|apartment| +11246|AAAAAAAAOOLCAAAA|737|Spruce |Street|Suite L|Arlington|Sioux County|NE|66557|United States|-7|condo| +11247|AAAAAAAAPOLCAAAA|||Pkwy||Woodlawn||IA|54098|United States||apartment| +11248|AAAAAAAAAPLCAAAA|656|Oak Second||Suite E|||MS|54466|||apartment| +11249|AAAAAAAABPLCAAAA|943|Seventh Forest|Dr.|Suite D|Buena Vista|Crawford County|OH|45752|United States|-5|condo| +11250|AAAAAAAACPLCAAAA|685|Lake |Avenue|Suite S|Hazelwood|Poinsett County|AR|71206|United States|-6|apartment| +11251|AAAAAAAADPLCAAAA|118|5th |Street|Suite Q|Cedar Grove|Lincoln County|ID|80411|United States|-7|apartment| +11252|AAAAAAAAEPLCAAAA|548|Spruce |Drive|Suite 390|Lincoln|Carter County|TN|31289|United States|-5|condo| +11253|AAAAAAAAFPLCAAAA|630|River |Ln|Suite N|Oak Ridge|Hickman County|KY|48371|United States|-6|apartment| +11254|AAAAAAAAGPLCAAAA|559|15th |Court|Suite W|Cedar Grove|Woodford County|IL|60411|United States|-6|single family| +11255|AAAAAAAAHPLCAAAA|997|Washington Third|ST|Suite U|Woodland|Elk County|KS|64854|United States|-6|single family| +11256|AAAAAAAAIPLCAAAA|84|Elm Hill|ST|Suite 180|Union|Lea County|NM|88721|United States|-7|single family| +11257|AAAAAAAAJPLCAAAA|812|Spring |Ln|Suite 230|Lakewood|Washburn County|WI|58877|United States|-6|single family| +11258|AAAAAAAAKPLCAAAA|40|First Center|Wy|Suite 400|Fairfield|Madison County|ID|86192|United States|-7|apartment| +11259|AAAAAAAALPLCAAAA|312|7th 10th|Street|Suite X|Jamestown|Pickett County|TN|36867|United States|-6|single family| +11260|AAAAAAAAMPLCAAAA|109|Smith South|Circle|Suite I|Salem|Upton County|TX|78048|United States|-6|condo| +11261|AAAAAAAANPLCAAAA|83|Chestnut Madison|Boulevard|Suite 350|Marion|McLeod County|MN|50399|United States|-6|condo| +11262|AAAAAAAAOPLCAAAA|173||Circle||Buena Vista||IA|55752|||condo| +11263|AAAAAAAAPPLCAAAA|972|12th |Street|Suite 140|Mount Vernon|Marlboro County|SC|28482|United States|-5|condo| +11264|AAAAAAAAAAMCAAAA|724|Sycamore 10th|Street|Suite 190|Waterloo|Putnam County|MO|61675|United States|-6|single family| +11265|AAAAAAAABAMCAAAA|730|Elm |Court|Suite Y|Waterloo|Carroll County|IA|51675|United States|-6|single family| +11266|AAAAAAAACAMCAAAA|848|Green |ST|Suite Q|Stringtown|Washburn County|WI|50162|United States|-6|apartment| +11267|AAAAAAAADAMCAAAA|251|Cherry First|Pkwy|Suite 100|Union|Hill County|MT|68721|United States|-7|condo| +11268|AAAAAAAAEAMCAAAA|525|3rd Hill|Dr.|Suite W|Hillcrest|Union County|NM|83003|United States|-7|condo| +11269|AAAAAAAAFAMCAAAA|14|Maple |Lane|Suite B|Sutton|Dickson County|TN|35413|United States|-5|single family| +11270|AAAAAAAAGAMCAAAA|814|Green |Wy|Suite 480|Oakdale|Lamar County|MS|59584|United States|-6|single family| +11271|AAAAAAAAHAMCAAAA|486|Johnson |Ave|Suite O|Oakwood|Decatur County|IA|50169|United States|-6|single family| +11272|AAAAAAAAIAMCAAAA|573|Fourth Cedar|Road|Suite Y|Salem|East Feliciana Parish|LA|78048|United States|-6|single family| +11273|AAAAAAAAJAMCAAAA|890|Mill Johnson|Circle|Suite X|Lakeside|Adams County|ND|59532|United States|-6|condo| +11274|AAAAAAAAKAMCAAAA|407|View |Pkwy|Suite G|Smith|Cerro Gordo County|IA|57317|United States|-6|single family| +11275|AAAAAAAALAMCAAAA|552|River Hill|Wy|Suite 210|Five Points|Lake County|IL|66098|United States|-6|condo| +11276|AAAAAAAAMAMCAAAA|654|Seventh Green|Avenue|Suite 450|Union|Houston County|GA|38721|United States|-5|apartment| +11277|AAAAAAAANAMCAAAA|286|Hickory Oak|Circle|Suite W|Newport|Winnebago County|IA|51521|United States|-6|single family| +11278|AAAAAAAAOAMCAAAA|988|4th 1st|Lane|Suite P|Sulphur Springs|Mitchell County|GA|38354|United States|-5|single family| +11279|AAAAAAAAPAMCAAAA|825|Green |Court|Suite E|Woodville|Bedford city|VA|24289|United States|-5|single family| +11280|AAAAAAAAABMCAAAA|914|8th |Way|Suite 140|Langdon|Alameda County|CA|90852|United States|-8|single family| +11281|AAAAAAAABBMCAAAA|586|Maple |Street|Suite 260|Forest Hills|Crawford County|IN|49237|United States|-5|condo| +11282|AAAAAAAACBMCAAAA|485|Washington Green|RD|Suite U|Shiloh|Hart County|GA|39275|United States|-5|single family| +11283|AAAAAAAADBMCAAAA|456|Spruce Spring|Lane|Suite S|Marion|Spencer County|KY|40399|United States|-5|single family| +11284|AAAAAAAAEBMCAAAA|711|Lincoln |Wy|Suite Y|Chestnut Ridge|Independence County|AR|77334|United States|-6|condo| +11285|AAAAAAAAFBMCAAAA|88|Central Cedar|ST|Suite T|Kingston|Washington County|PA|14975|United States|-5|apartment| +11286|AAAAAAAAGBMCAAAA|733|North Sunset|Drive|Suite 440|Highland Park|Covington city|VA|26534|United States|-5|apartment| +11287|AAAAAAAAHBMCAAAA|18|11th |Street|Suite 450|Newtown|Buffalo County|SD|51749|United States|-6|single family| +11288|AAAAAAAAIBMCAAAA|4|Fourth |Avenue|Suite F|Walnut|Stutsman County|ND|56245|United States|-6|condo| +11289|AAAAAAAAJBMCAAAA|167|Lakeview Green|Parkway|Suite B|Springdale|Amador County|CA|98883|United States|-8|single family| +11290|AAAAAAAAKBMCAAAA|10|Wilson Third|Road|Suite R|Friendship|Tuscarawas County|OH|44536|United States|-5|single family| +11291|AAAAAAAALBMCAAAA|354|15th South|Court|Suite Y|Summit|Essex County|VT|01099|United States|-5|apartment| +11292|AAAAAAAAMBMCAAAA|437|Davis |Boulevard|Suite 400|Pulaski|Ford County|IL|68674|United States|-6|single family| +11293|AAAAAAAANBMCAAAA|193|Lee Park|Cir.|Suite E|Mount Olive|Anderson County|SC|28059|United States|-5|single family| +11294|AAAAAAAAOBMCAAAA|768|Fifteenth Wilson|Avenue|Suite N|Highland Park|Skagit County|WA|96534|United States|-8|apartment| +11295|AAAAAAAAPBMCAAAA|897|5th Johnson|Circle|Suite 460|Harmony|Carroll County|GA|35804|United States|-5|single family| +11296|AAAAAAAAACMCAAAA|502|Davis |RD|Suite C|Woodland|Paulding County|GA|34854|United States|-5|condo| +11297|AAAAAAAABCMCAAAA|55|Broadway Locust|Cir.|Suite Q|Bethel|Coahoma County|MS|55281|United States|-6|condo| +11298|AAAAAAAACCMCAAAA|945|Miller Spring|Street|Suite 270|Unionville|Crawford County|IN|41711|United States|-5|single family| +11299|AAAAAAAADCMCAAAA|657|12th |RD|Suite H|Riverdale|Union County|SD|59391|United States|-7|single family| +11300|AAAAAAAAECMCAAAA|855|Adams Franklin|Avenue|Suite 440|Edgewood|Fresno County|CA|90069|United States|-8|condo| +11301|AAAAAAAAFCMCAAAA|944|8th |Ct.|Suite 300|Springdale|Van Zandt County|TX|78883|United States|-6|condo| +11302|AAAAAAAAGCMCAAAA|271|Lake Broadway|Road|Suite F|Lakeside|Aransas County|TX|79532|United States|-6|single family| +11303|AAAAAAAAHCMCAAAA|312|10th |Pkwy|Suite K|Midway|Lunenburg County|VA|21904|United States|-5|apartment| +11304|AAAAAAAAICMCAAAA|247|Railroad |Blvd|Suite 250|Wilson|Jackson County|KS|66971|United States|-6|single family| +11305|AAAAAAAAJCMCAAAA|284|Miller |Court|Suite 50|Woodville|Drew County|AR|74289|United States|-6|apartment| +11306|AAAAAAAAKCMCAAAA|294|Main |Ave|Suite I|Blair|Pulaski County|IL|65465|United States|-6|apartment| +11307|AAAAAAAALCMCAAAA|78|College Birch|Street|Suite 180|Waterloo|Buchanan County|MO|61675|United States|-6|condo| +11308|AAAAAAAAMCMCAAAA|142|Walnut River|RD|Suite 140|Woodville|Union Parish|LA|74289|United States|-6|single family| +11309|AAAAAAAANCMCAAAA|76|Second |Ct.|Suite F|Bunker Hill|Hood River County|OR|90150|United States|-8|condo| +11310|AAAAAAAAOCMCAAAA|551|Elm Jackson|Wy|Suite 240|Mount Pleasant|Muhlenberg County|KY|41933|United States|-5|condo| +11311|AAAAAAAAPCMCAAAA|168|Second Willow|Ln|Suite J|Farmington|Renville County|MN|59145|United States|-6|condo| +11312|AAAAAAAAADMCAAAA|769|Walnut |Parkway|Suite Y|Owens|Tulare County|CA|92324|United States|-8|single family| +11313|AAAAAAAABDMCAAAA|894|Laurel Madison|Wy|Suite L|Shiloh|Berrien County|MI|49275|United States|-5|apartment| +11314|AAAAAAAACDMCAAAA|755|Ridge |Dr.|Suite 130|Oneida|Rio Grande County|CO|84027|United States|-7|apartment| +11315|AAAAAAAADDMCAAAA|825|Mill |Ln|Suite 170|Crystal|Power County|ID|85258|United States|-7|condo| +11316|AAAAAAAAEDMCAAAA|583|Cedar |Blvd|Suite T|Glenwood|Marshall County|MS|53511|United States|-6|condo| +11317|AAAAAAAAFDMCAAAA|616|14th |Court|Suite 260|Lakewood|Fulton County|AR|78877|United States|-6|apartment| +11318|AAAAAAAAGDMCAAAA|44|Railroad |Ave|Suite 120|Marion|Cherokee County|OK|70399|United States|-6|apartment| +11319|AAAAAAAAHDMCAAAA|408|Second Hill|Ln|Suite 150|Valley View|Logan County|CO|85124|United States|-7|condo| +11320|AAAAAAAAIDMCAAAA|231|Third Pine|Street|Suite 80|Rolling Hills|Gibson County|TN|37272|United States|-5|single family| +11321|AAAAAAAAJDMCAAAA|926|15th Maple|Ln|Suite S|Lincoln|Vermilion County|IL|61289|United States|-6|single family| +11322|AAAAAAAAKDMCAAAA|301|8th 13th|Wy|Suite 470|Smith|Rock County|WI|57317|United States|-6|condo| +11323|AAAAAAAALDMCAAAA|849|Birch |Blvd|Suite 300|Georgetown|Greenbrier County|WV|27057|United States|-5|apartment| +11324|AAAAAAAAMDMCAAAA|632|Johnson Washington|Road|Suite H|Springdale|Wythe County|VA|28883|United States|-5|condo| +11325|AAAAAAAANDMCAAAA|868|Valley Railroad|Avenue|Suite Q|Highland|Jefferson County|OR|99454|United States|-8|condo| +11326|AAAAAAAAODMCAAAA|529|10th |Pkwy|Suite 80|Clifton|Graves County|KY|48014|United States|-6|apartment| +11327|AAAAAAAAPDMCAAAA|111|Ridge Birch|Street|Suite 190|Stringtown|Cache County|UT|80162|United States|-7|condo| +11328|AAAAAAAAAEMCAAAA|499|Elm |RD|Suite W|Rockwood|Russell County|VA|21545|United States|-5|apartment| +11329|AAAAAAAABEMCAAAA|244|Lakeview |Blvd|Suite 70|Thompsonville|Carlton County|MN|59651|United States|-6|condo| +11330|AAAAAAAACEMCAAAA|689|Dogwood |ST|Suite C|Mount Vernon|Red River County|TX|78482|United States|-6|single family| +11331|AAAAAAAADEMCAAAA|743|East Fifth|Avenue|Suite D|Springdale|Walker County|TX|78883|United States|-6|single family| +11332|AAAAAAAAEEMCAAAA|434|Washington Center|Ln|Suite Q|Flint Hill|Montgomery County|TN|37934|United States|-6|single family| +11333|AAAAAAAAFEMCAAAA|866|Franklin |Road|Suite 60|Farmington|Logan County|ND|59145|United States|-6|condo| +11334|AAAAAAAAGEMCAAAA|607|Pine |Ct.|Suite 130|Midway|Delaware County|IA|51904|United States|-6|condo| +11335|AAAAAAAAHEMCAAAA|788|3rd Park|Avenue|Suite D|Summit|Delaware County|OH|40499|United States|-5|single family| +11336|AAAAAAAAIEMCAAAA|264|3rd Broadway|Ct.|Suite 360|Waterloo|Warren County|VA|21675|United States|-5|apartment| +11337|AAAAAAAAJEMCAAAA|420|Forest |Cir.|Suite T|Greenville|Kent County|DE|11387|United States|-5|single family| +11338|AAAAAAAAKEMCAAAA|864|Main |Way|Suite H|New Hope|Grafton County|NH|09431|United States|-5|apartment| +11339|AAAAAAAALEMCAAAA|547|7th |Lane|Suite G|Jackson|Laurel County|KY|49583|United States|-5|condo| +11340|AAAAAAAAMEMCAAAA|737|Hillcrest Woodland|Parkway|Suite 360|Mount Vernon|Okfuskee County|OK|78482|United States|-6|condo| +11341|AAAAAAAANEMCAAAA|769|Green |Road|Suite 120|Antioch|Crawford County|IN|48605|United States|-5|apartment| +11342|AAAAAAAAOEMCAAAA|538|Walnut |Wy|Suite 100|Arlington|Ramsey County|ND|56557|United States|-6|condo| +11343|AAAAAAAAPEMCAAAA|762|North Lake|Drive|Suite B|Florence|Marshall County|KS|63394|United States|-6|condo| +11344|AAAAAAAAAFMCAAAA|8|Mill |Dr.|Suite P|Vista|Lac qui Parle County|MN|54694|United States|-6|single family| +11345|AAAAAAAABFMCAAAA|474|1st Adams|Ave|Suite 360|Hidden Valley|Ray County|MO|65521|United States|-6|single family| +11346|AAAAAAAACFMCAAAA|195|Mill Smith|Cir.|Suite 160|Friendship|Monroe County|FL|34536|United States|-5|apartment| +11347|AAAAAAAADFMCAAAA|464|5th |Avenue|Suite 280|Sutton|Jones County|IA|55413|United States|-6|condo| +11348|AAAAAAAAEFMCAAAA|709|Elm |Avenue|Suite 190|Greenville|Trimble County|KY|41387|United States|-5|single family| +11349|AAAAAAAAFFMCAAAA|532|Church Highland|Ave|Suite 320|Summit|Lincoln County|GA|30499|United States|-5|condo| +11350|AAAAAAAAGFMCAAAA||Adams |Cir.||||NC|24240||-5|| +11351|AAAAAAAAHFMCAAAA|203|15th |Lane|Suite H|Enterprise|Todd County|SD|51757|United States|-7|apartment| +11352|AAAAAAAAIFMCAAAA|541|Miller |Ct.|Suite 340|Jackson|Shoshone County|ID|89583|United States|-7|apartment| +11353|AAAAAAAAJFMCAAAA|537|Fifteenth Spring|Lane|Suite 60|Harmony|Ravalli County|MT|65804|United States|-7|single family| +11354|AAAAAAAAKFMCAAAA|982|9th |Road|Suite H|Martinsville|Madison County|ID|80419|United States|-7|single family| +11355|AAAAAAAALFMCAAAA|218|Park |Ave|Suite B|New Hope|Union County|SC|29431|United States|-5|single family| +11356|AAAAAAAAMFMCAAAA|578|River |Road|Suite R|Antioch|Madison County|AL|38605|United States|-6|condo| +11357|AAAAAAAANFMCAAAA|804|Ridge |Ave|Suite 190|Avoca|Saline County|NE|60540|United States|-7|single family| +11358|AAAAAAAAOFMCAAAA|681|Church 10th|Pkwy|Suite E|Cedar Grove|Buchanan County|IA|50411|United States|-6|apartment| +11359|AAAAAAAAPFMCAAAA|292|North |Road|Suite 320|Edgewood|Cherokee County|NC|20069|United States|-5|apartment| +11360|AAAAAAAAAGMCAAAA|422|Fourth 11th|Street|Suite 230|Nichols|Nez Perce County|ID|87940|United States|-7|apartment| +11361|AAAAAAAABGMCAAAA||||Suite K|Shady Grove|||||-5|apartment| +11362|AAAAAAAACGMCAAAA|995|3rd Dogwood|Parkway|Suite 470|Harmony|Gaines County|TX|75804|United States|-6|single family| +11363|AAAAAAAADGMCAAAA|524|Wilson Maple|Pkwy|Suite M|Antioch|White County|IL|68605|United States|-6|single family| +11364|AAAAAAAAEGMCAAAA|606|15th Church|Court|Suite 370|Newport|Carroll County|KY|41521|United States|-6|apartment| +11365|AAAAAAAAFGMCAAAA|162|Broadway Chestnut|Court|Suite O|Oakdale|Alpena County|MI|49584|United States|-5|single family| +11366|AAAAAAAAGGMCAAAA|37|Spring |Pkwy|Suite 100|Lebanon|Beaver County|UT|82898|United States|-7|apartment| +11367|AAAAAAAAHGMCAAAA|563|Highland |Circle|Suite M|Stafford|Nolan County|TX|74980|United States|-6|apartment| +11368|AAAAAAAAIGMCAAAA|168|Fourteenth Elm|Ct.|Suite 30|Highland Park|Richmond County|NY|16534|United States|-5|single family| +11369|AAAAAAAAJGMCAAAA|826|Miller |Drive|Suite O|Sunnyside|Carroll County|KY|41952|United States|-6|apartment| +11370|AAAAAAAAKGMCAAAA|346|Second |Wy|Suite 120|Pleasant Hill|Baldwin County|GA|33604|United States|-5|apartment| +11371|AAAAAAAALGMCAAAA|186|Miller |Parkway|Suite K|Union|Providence County|RI|09321|United States|-5|apartment| +11372|AAAAAAAAMGMCAAAA|592|Spruce Center|Lane|Suite E|Concord|West Carroll Parish|LA|74107|United States|-6|apartment| +11373|AAAAAAAANGMCAAAA|911|Cedar |Ln|Suite 260|Woodville|Fayette County|IA|54289|United States|-6|single family| +11374|AAAAAAAAOGMCAAAA|611|Laurel |RD|Suite 10|Highland Park|Tallapoosa County|AL|36534|United States|-6|apartment| +11375|AAAAAAAAPGMCAAAA|599|North |Cir.|Suite Y|Lakeview|Bladen County|NC|28579|United States|-5|condo| +11376|AAAAAAAAAHMCAAAA|833||||Highland Park||OK|76534||-6|condo| +11377|AAAAAAAABHMCAAAA|880|Main Center|Pkwy|Suite 20|Winslow|Washington County|MD|28525|United States|-5|single family| +11378|AAAAAAAACHMCAAAA|612|Cedar Washington|Ct.|Suite 220|Union Hill|McPherson County|SD|57746|United States|-7|single family| +11379|AAAAAAAADHMCAAAA|310|Cherry Madison|Boulevard|Suite G|Red Hill|Santa Barbara County|CA|94338|United States|-8|single family| +11380|AAAAAAAAEHMCAAAA|657|Wilson East|RD|Suite S|Liberty|Aleutians West Census Area|AK|93451|United States|-9|single family| +11381|AAAAAAAAFHMCAAAA|383||Cir.|||Cheyenne County||60492|United States||single family| +11382|AAAAAAAAGHMCAAAA|192|Maple |Avenue|Suite 350|Liberty|Cocke County|TN|33451|United States|-5|single family| +11383|AAAAAAAAHHMCAAAA|403|Fourth |Ln|Suite Q|Ashland|Hardin County|IA|54244|United States|-6|apartment| +11384|AAAAAAAAIHMCAAAA||Center |Dr.|Suite 270|Wilton||||||| +11385|AAAAAAAAJHMCAAAA|222|1st Washington|Ave|Suite M|Jones|Rockbridge County|VA|22686|United States|-5|condo| +11386|AAAAAAAAKHMCAAAA|607|Sunset Washington|Way|Suite D|Waterloo|Elbert County|CO|81675|United States|-7|single family| +11387|AAAAAAAALHMCAAAA|133|6th 6th|Blvd|Suite D|Woodville|Emmons County|ND|54289|United States|-6|apartment| +11388|AAAAAAAAMHMCAAAA|427|1st |Lane|Suite P|Altamont|Litchfield County|CT|09987|United States|-5|condo| +11389|AAAAAAAANHMCAAAA|985|Second |Ln|Suite 480|Belmont|Vermilion County|IL|60191|United States|-6|apartment| +11390|AAAAAAAAOHMCAAAA|721|7th Washington|Pkwy|Suite 340|Shady Grove|McNairy County|TN|32812|United States|-6|apartment| +11391|AAAAAAAAPHMCAAAA|632|Willow |Ave|Suite B|Harmony|Parke County|IN|45804|United States|-5|condo| +11392|AAAAAAAAAIMCAAAA|541|Lake |ST|Suite 70|Hamilton|Rusk County|TX|72808|United States|-6|single family| +11393|AAAAAAAABIMCAAAA|106|Highland Poplar|Avenue|Suite 130|Riverside|Wakulla County|FL|39231|United States|-5|condo| +11394|AAAAAAAACIMCAAAA|655|Church |ST|Suite 40|Stringtown|Valley County|ID|80162|United States|-7|apartment| +11395|AAAAAAAADIMCAAAA|864|Elm Jackson|Street|Suite 150|Woodville|McLennan County|TX|74289|United States|-6|condo| +11396|AAAAAAAAEIMCAAAA|519|Main 2nd|Drive|Suite I|Salem|Saunders County|NE|68048|United States|-7|single family| +11397|AAAAAAAAFIMCAAAA|39|1st |Avenue|Suite V|Deerfield|Elmore County|AL|39840|United States|-6|single family| +11398|AAAAAAAAGIMCAAAA|1000|Park |Ave|Suite 330|Riverside|Wabash County|IN|49231|United States|-5|condo| +11399|AAAAAAAAHIMCAAAA|167|Oak Railroad|RD|||Decatur County|IN||United States|-5|| +11400|AAAAAAAAIIMCAAAA|352|4th |Avenue|Suite 20|Stringtown|Hutchinson County|SD|50162|United States|-7|condo| +11401|AAAAAAAAJIMCAAAA|138|Adams 2nd|Ln|Suite H|Five Points|Virginia Beach city|VA|26098|United States|-5|condo| +11402|AAAAAAAAKIMCAAAA|141|Forest 13th|Lane|Suite F|Florence|Somervell County|TX|73394|United States|-6|single family| +11403|AAAAAAAALIMCAAAA|162|Ridge 9th|RD|Suite R|Springdale|Osceola County|FL|38883|United States|-5|apartment| +11404|AAAAAAAAMIMCAAAA|757|9th |Dr.|Suite T|Georgetown|Jackson County|MO|67057|United States|-6|single family| +11405|AAAAAAAANIMCAAAA|952|Adams Sycamore|Lane|Suite 180|Salem|Norfolk city|VA|28048|United States|-5|condo| +11406|AAAAAAAAOIMCAAAA|774|Park 5th|Parkway|Suite G|Oak Ridge|Lamb County|TX|78371|United States|-6|apartment| +11407|AAAAAAAAPIMCAAAA|495|Lake 14th|Lane|Suite 40|Oak Hill|Worth County|MO|67838|United States|-6|single family| +11408|AAAAAAAAAJMCAAAA|812|Green |Ave|Suite Y|Mountain View|Brown County|SD|54466|United States|-6|single family| +11409|AAAAAAAABJMCAAAA|268|View Locust|Drive|Suite Y|Burns|Rockland County|NY|15272|United States|-5|single family| +11410|AAAAAAAACJMCAAAA|365|Ridge |Circle|Suite G|Union Hill|DeKalb County|IL|67746|United States|-6|condo| +11411|AAAAAAAADJMCAAAA|264|14th 9th|Wy|Suite Y|Arlington|Dubois County|IN|46557|United States|-5|condo| +11412|AAAAAAAAEJMCAAAA|367||Drive|Suite Q|Prosperity|Saginaw County|||United States|-5|single family| +11413|AAAAAAAAFJMCAAAA||1st |Boulevard|Suite G||Osage County||70913|United States|-6|| +11414|AAAAAAAAGJMCAAAA|767|7th Green|Road|Suite 460|Lebanon|Beckham County|OK|72898|United States|-6|single family| +11415|AAAAAAAAHJMCAAAA|254|Highland |Parkway|Suite I|Lakewood|Solano County|CA|98877|United States|-8|single family| +11416|AAAAAAAAIJMCAAAA|455|Madison |Drive|Suite V|Lebanon|Hancock County|TN|32898|United States|-5|single family| +11417|AAAAAAAAJJMCAAAA|802|Pine Main|Wy|Suite P|Walnut Grove|Rooks County|KS|67752|United States|-6|single family| +11418|AAAAAAAAKJMCAAAA|227|Elm Park|Cir.|Suite E|Union Hill|Bryan County|GA|37746|United States|-5|condo| +11419|AAAAAAAALJMCAAAA|379|Cedar 7th|Court|Suite T|Woodville|Bureau County|IL|64289|United States|-6|apartment| +11420|AAAAAAAAMJMCAAAA|87|7th |Wy|Suite 430|Hamilton|Kosciusko County|IN|42808|United States|-5|apartment| +11421|AAAAAAAANJMCAAAA|396|Laurel |Lane|Suite 270|Edgewood|Cedar County|MO|60069|United States|-6|single family| +11422|AAAAAAAAOJMCAAAA|704|4th Highland|Way|Suite K|Ridgewood|Middlesex County|CT|07347|United States|-5|condo| +11423|AAAAAAAAPJMCAAAA|506|Walnut |Ln|Suite O|Oakwood|Clinch County|GA|30169|United States|-5|apartment| +11424|AAAAAAAAAKMCAAAA|906|West Highland|Circle|Suite 410|Jackson|Marion County|AL|39583|United States|-6|apartment| +11425|AAAAAAAABKMCAAAA|261|Walnut |Court|Suite O|Kirkland|Harrison County|IA|57896|United States|-6|apartment| +11426|AAAAAAAACKMCAAAA|664|Highland |Dr.|Suite 50|Cedar Grove|Park County|WY|80411|United States|-7|apartment| +11427|AAAAAAAADKMCAAAA|427|5th |Ct.|Suite 40|Green Acres|Golden Valley County|ND|57683|United States|-6|apartment| +11428|AAAAAAAAEKMCAAAA|566|Jackson |Circle|Suite 250|Highland|Black Hawk County|IA|59454|United States|-6|apartment| +11429|AAAAAAAAFKMCAAAA|970|Lincoln |Court|Suite 150|Lake Forest|Albany County|NY|16000|United States|-5|single family| +11430|AAAAAAAAGKMCAAAA|80|Third Cedar|Boulevard|Suite W|Oakdale|Teton County|ID|89584|United States|-7|single family| +11431|AAAAAAAAHKMCAAAA|955|2nd |Ct.|Suite 360|Unionville|Tuolumne County|CA|91711|United States|-8|condo| +11432|AAAAAAAAIKMCAAAA|877|First Birch|Way|Suite 420|Waterloo|Stanton County|KS|61675|United States|-6|single family| +11433|AAAAAAAAJKMCAAAA|522|Willow |Lane|Suite K|Greenfield|Lane County|KS|65038|United States|-6|condo| +11434|AAAAAAAAKKMCAAAA|392|6th Mill|Ln|Suite H|Newport|Perry County|KY|41521|United States|-5|single family| +11435|AAAAAAAALKMCAAAA|138|2nd 2nd|Avenue|Suite F|Newport|La Salle County|IL|61521|United States|-6|condo| +11436|AAAAAAAAMKMCAAAA|719|Third Forest|Way|Suite O|Five Points|Sherman County|OR|96098|United States|-8|apartment| +11437|AAAAAAAANKMCAAAA|915|Pine |Blvd|Suite 310|Summit|Cameron County|PA|10499|United States|-5|single family| +11438|AAAAAAAAOKMCAAAA|948|Dogwood Washington|Ave|Suite M|Enterprise|Garvin County|OK|71757|United States|-6|apartment| +11439|AAAAAAAAPKMCAAAA|450|Hickory Railroad|Avenue|Suite H|Buena Vista|Morgan County|UT|85752|United States|-7|condo| +11440|AAAAAAAAALMCAAAA|175|Hickory |RD|Suite 470|Mount Vernon|Taylor County|KY|48482|United States|-5|single family| +11441|AAAAAAAABLMCAAAA|996|3rd |Parkway|Suite I|Church Hill|Addison County|VT|04390|United States|-5|apartment| +11442|AAAAAAAACLMCAAAA|433|Mill 11th|Avenue|Suite N|Sulphur Springs|Ripley County|MO|68354|United States|-6|apartment| +11443|AAAAAAAADLMCAAAA|16|Lake |Parkway|Suite S|Florence|Le Sueur County|MN|53394|United States|-6|condo| +11444|AAAAAAAAELMCAAAA|269|12th Poplar|Drive|Suite D|Oakwood|Crawford County|MO|60169|United States|-6|single family| +11445|AAAAAAAAFLMCAAAA|5|Cherry |Cir.|Suite M|Wilson|Wayne County|WV|26971|United States|-5|condo| +11446|AAAAAAAAGLMCAAAA|538||Dr.|Suite H||||55038|||| +11447|AAAAAAAAHLMCAAAA|119||||Belmont||||United States||| +11448|AAAAAAAAILMCAAAA|539|Forest |RD|Suite I|Woodland|Collingsworth County|TX|74854|United States|-6|apartment| +11449|AAAAAAAAJLMCAAAA|683|Second |Wy|Suite 470|Deerfield|Clark County|WA|99840|United States|-8|apartment| +11450|AAAAAAAAKLMCAAAA||College 4th|Ave|Suite L|Jackson||GA|39583|||| +11451|AAAAAAAALLMCAAAA|465|Valley 1st|Circle|Suite 480|Stringtown|Iowa County|IA|50162|United States|-6|apartment| +11452|AAAAAAAAMLMCAAAA|552|Lincoln Spruce|Road|Suite 180|Woodlawn|Crockett County|TN|34098|United States|-5|single family| +11453|AAAAAAAANLMCAAAA|304|Valley Birch|Blvd|Suite 160|Springfield|Sumter County|AL|39303|United States|-6|condo| +11454|AAAAAAAAOLMCAAAA|101|12th |Boulevard|Suite 320|Forest Hills|Jefferson County|MO|69237|United States|-6|apartment| +11455|AAAAAAAAPLMCAAAA|307|Dogwood Main|ST|Suite M|Concord|Coos County|NH|04707|United States|-5|single family| +11456|AAAAAAAAAMMCAAAA|386|East |Cir.|Suite X|Harmony|Olmsted County|MN|55804|United States|-6|condo| +11457|AAAAAAAABMMCAAAA|803|9th Hickory|Wy|Suite 130|Wildwood|Columbia County|PA|16871|United States|-5|single family| +11458|AAAAAAAACMMCAAAA|579|Main Park|Road|Suite 220|Pinhook|Redwood County|MN|59398|United States|-6|apartment| +11459|AAAAAAAADMMCAAAA|20|Forest |Avenue|Suite K|Greenwood|Montgomery County|IL|68828|United States|-6|condo| +11460|AAAAAAAAEMMCAAAA|672|South 4th|Ct.|Suite 120|Lakeside|Rockland County|NY|19532|United States|-5|apartment| +11461|AAAAAAAAFMMCAAAA|429|Franklin |ST|Suite N|Oak Grove|Sanilac County|MI|48370|United States|-5|condo| +11462|AAAAAAAAGMMCAAAA|37|Cedar Cedar|Blvd|Suite D|Oak Grove|Berks County|PA|18370|United States|-5|condo| +11463|AAAAAAAAHMMCAAAA|||Wy|Suite W|Lakeside|Charleston County|SC||United States||apartment| +11464|AAAAAAAAIMMCAAAA|1|Sunset |Parkway|Suite E|Prosperity|Grand County|UT|89089|United States|-7|single family| +11465|AAAAAAAAJMMCAAAA|882|13th |Blvd|Suite 0|Rossville|Liberty County|TX|72503|United States|-6|condo| +11466|AAAAAAAAKMMCAAAA|294|3rd |Court|Suite X|Union|Windsor County|VT|09321|United States|-5|single family| +11467|AAAAAAAALMMCAAAA|815|Elm Spruce|Parkway|Suite E|Watkins|Grady County|GA|31732|United States|-5|condo| +11468|AAAAAAAAMMMCAAAA|472|5th |Circle|Suite E|Mount Olive|Hampshire County|WV|28059|United States|-5|single family| +11469|AAAAAAAANMMCAAAA|299|1st 12th|Parkway|Suite 70|Five Forks|Lowndes County|MS|52293|United States|-6|apartment| +11470|AAAAAAAAOMMCAAAA|476|Second |Road|Suite 400|Welcome|Meigs County|OH|46386|United States|-5|single family| +11471|AAAAAAAAPMMCAAAA|509|River Spruce|ST|Suite 60|Centerville|Ontario County|NY|10059|United States|-5|apartment| +11472|AAAAAAAAANMCAAAA|380|West Meadow|Way|Suite J|Greenville|Otoe County|NE|61387|United States|-7|single family| +11473|AAAAAAAABNMCAAAA|113|Maple 1st|Avenue|Suite 80|Harmon|Bon Homme County|SD|55623|United States|-6|apartment| +11474|AAAAAAAACNMCAAAA|372|Washington |RD|Suite C|Mount Olive|Essex County|VA|28059|United States|-5|condo| +11475|AAAAAAAADNMCAAAA|787|Washington |Road|Suite K|Five Points|Coal County|OK|76098|United States|-6|apartment| +11476|AAAAAAAAENMCAAAA|295|West |Ln|Suite 370|Springdale|Clark County|NV|88883|United States|-8|apartment| +11477|AAAAAAAAFNMCAAAA|203|15th |RD|Suite 180|Lebanon|Butler County|NE|62898|United States|-6|condo| +11478|AAAAAAAAGNMCAAAA|512|Birch Jefferson|Parkway|Suite 170|Five Points|Spencer County|KY|46098|United States|-5|apartment| +11479|AAAAAAAAHNMCAAAA|780|Spring Seventh|Road|Suite 370|Mount Vernon|Garfield County|CO|88482|United States|-7|apartment| +11480|AAAAAAAAINMCAAAA|315|9th |RD|Suite N|Shady Grove|Northwest Arctic Borough|AK|92812|United States|-9|apartment| +11481|AAAAAAAAJNMCAAAA|3|1st |ST|Suite 330|Cordova|Floyd County|KY|46938|United States|-6|condo| +11482|AAAAAAAAKNMCAAAA|809|Wilson Ash|Ct.|Suite W|Fernwood|McCook County|SD|52291|United States|-7|apartment| +11483|AAAAAAAALNMCAAAA|709|South Jackson|Cir.|Suite W|Oak Grove|Clay County|NC|28370|United States|-5|single family| +11484|AAAAAAAAMNMCAAAA|669|Smith |Boulevard|Suite 70|Mount Pleasant|Champaign County|IL|61933|United States|-6|condo| +11485|AAAAAAAANNMCAAAA|649|Birch |Blvd|Suite I|Mount Pleasant|Perry County|OH|41933|United States|-5|condo| +11486|AAAAAAAAONMCAAAA|291|East Mill|Cir.|Suite 210|Lakeville|Sherburne County|MN|58811|United States|-6|condo| +11487|AAAAAAAAPNMCAAAA|143|Laurel North|Drive|Suite 10|Oak Grove|Greene County|AL|38370|United States|-6|single family| +11488|AAAAAAAAAOMCAAAA|304|Ash Broadway|Parkway|Suite 290|Summit|Corson County|SD|50499|United States|-6|apartment| +11489|AAAAAAAABOMCAAAA|128|Main Elm|Dr.|Suite 220|Friendship|Stanton County|NE|64536|United States|-7|apartment| +11490|AAAAAAAACOMCAAAA|85|Washington |Ct.|Suite 150|Harmony|Rapides Parish|LA|75804|United States|-6|apartment| +11491|AAAAAAAADOMCAAAA|92|Hickory |Drive|Suite O|Oak Hill|Fayette County|TN|37838|United States|-5|single family| +11492|AAAAAAAAEOMCAAAA|710|Green |Circle|Suite Q|Forest Hills|Stonewall County|TX|79237|United States|-6|single family| +11493|AAAAAAAAFOMCAAAA|421|11th |Blvd|Suite 80|Providence|Spotsylvania County|VA|26614|United States|-5|condo| +11494|AAAAAAAAGOMCAAAA|126|14th 2nd|Wy|Suite E|Golden|Navarro County|TX|70411|United States|-6|condo| +11495|AAAAAAAAHOMCAAAA|792|Lee |Ct.|Suite U|Springdale|Seminole County|FL|38883|United States|-5|single family| +11496|AAAAAAAAIOMCAAAA|834|3rd |Ave|Suite 170|Arlington|Decatur County|IN|46557|United States|-5|condo| +11497|AAAAAAAAJOMCAAAA|54|Maple College|Dr.|Suite 50|Stringtown|Texas County|MO|60162|United States|-6|apartment| +11498|AAAAAAAAKOMCAAAA|851|14th Sycamore|Boulevard|Suite D|Brownsville|Linn County|KS|69310|United States|-6|single family| +11499|AAAAAAAALOMCAAAA|31|14th |Lane|Suite 480|Macedonia|Chatham County|NC|21087|United States|-5|single family| +11500|AAAAAAAAMOMCAAAA|889|1st Laurel|Way|Suite U|Pleasant Hill|Idaho County|ID|83604|United States|-7|condo| +11501|AAAAAAAANOMCAAAA|720|Park |Street|Suite D|Hazelwood|King County|WA|91206|United States|-8|condo| +11502|AAAAAAAAOOMCAAAA|453|Fifth |RD|Suite G|Tremont|Custer County|NE|69515|United States|-6|condo| +11503|AAAAAAAAPOMCAAAA|291|Ridge Willow|Pkwy|Suite 430|Springdale|Pierce County|WI|58883|United States|-6|apartment| +11504|AAAAAAAAAPMCAAAA|956|Mill |RD|Suite 250|Oakdale|Dimmit County|TX|79584|United States|-6|apartment| +11505|AAAAAAAABPMCAAAA|406|Tenth 15th||Suite E|Lincoln|Warren County|VA|21289|United States|-5|| +11506|AAAAAAAACPMCAAAA|807|Ninth |Court|Suite 430|Providence|Tallapoosa County|AL|36614|United States|-6|condo| +11507|AAAAAAAADPMCAAAA|703|3rd Valley|Ave|Suite 470|Bunker Hill|Gloucester County|VA|20150|United States|-5|single family| +11508|AAAAAAAAEPMCAAAA|812|Cherry |Boulevard|Suite 30|Newtown|Orange County|CA|91749|United States|-8|condo| +11509|AAAAAAAAFPMCAAAA|845|1st Hill|Wy|Suite 360|Lakeview|Tipton County|TN|38579|United States|-6|condo| +11510|AAAAAAAAGPMCAAAA|413|2nd |Pkwy|Suite 10|Redland|Lancaster County|NE|66343|United States|-7|apartment| +11511|AAAAAAAAHPMCAAAA|113|Maple Cedar|Way|Suite R|Union Hill|Jasper County|TX|77746|United States|-6|apartment| +11512|AAAAAAAAIPMCAAAA|714|5th West|Court|Suite 360|Clinton|Greer County|OK|78222|United States|-6|apartment| +11513|AAAAAAAAJPMCAAAA|382|6th |ST|Suite K|Newtown|Luce County|MI|41749|United States|-5|apartment| +11514|AAAAAAAAKPMCAAAA|509|Railroad Jefferson|Lane|Suite L|Jamestown|Powell County|MT|66867|United States|-7|apartment| +11515|AAAAAAAALPMCAAAA|73|11th 11th|Lane|Suite L|Five Forks|Honolulu County|HI|92293|United States|-10|single family| +11516|AAAAAAAAMPMCAAAA|769|Sycamore 7th|Ct.|Suite F|Riverdale|Lanier County|GA|39391|United States|-5|single family| +11517|AAAAAAAANPMCAAAA|38|Cedar 2nd|Drive|Suite 50|Shiloh|Rock County|WI|59275|United States|-6|apartment| +11518|AAAAAAAAOPMCAAAA|832|Jefferson Pine|RD|Suite 460|Crossroads|Seminole County|OK|70534|United States|-6|condo| +11519|AAAAAAAAPPMCAAAA|404||||Deerfield||MO|69840|United States|-6|| +11520|AAAAAAAAAANCAAAA|483|Poplar River|Dr.|Suite J|Fairfield|Fannin County|TX|76192|United States|-6|condo| +11521|AAAAAAAABANCAAAA|998|Lakeview Lake|Boulevard|Suite M|Spring Valley|Berks County|PA|16060|United States|-5|single family| +11522|AAAAAAAACANCAAAA|97|Washington |Road|Suite 10|San Jose|Wise County|TX|78003|United States|-6|condo| +11523|AAAAAAAADANCAAAA|939|Lincoln North|Pkwy|Suite 270|Oak Grove|Washington County|MD|28370|United States|-5|condo| +11524|AAAAAAAAEANCAAAA|715|Lake |Boulevard|Suite 60|New Hope|Boone County|WV|29431|United States|-5|apartment| +11525|AAAAAAAAFANCAAAA|667|5th Jackson|Circle|Suite L|Clinton|Jefferson County|OH|48222|United States|-5|single family| +11526|AAAAAAAAGANCAAAA||Second |Road|||Richland County|WI||||condo| +11527|AAAAAAAAHANCAAAA|54|Tenth |Ct.|Suite X|Kingston|Gilchrist County|FL|34975|United States|-5|single family| +11528|AAAAAAAAIANCAAAA|477|7th |Cir.|Suite B|Lincoln|Isabella County|MI|41289|United States|-5|single family| +11529|AAAAAAAAJANCAAAA|797|East |Blvd|Suite 320|Wilson|Cerro Gordo County|IA|56971|United States|-6|condo| +11530|AAAAAAAAKANCAAAA|441|3rd Chestnut|RD|Suite V|Enterprise|O-Brien County|IA|51757|United States|-6|apartment| +11531|AAAAAAAALANCAAAA|8|12th |Way|Suite 30|Lakeside|Rutherford County|NC|29532|United States|-5|single family| +11532|AAAAAAAAMANCAAAA|304|Broadway |Parkway|Suite C|Pleasant Hill|Madison County|AL|33604|United States|-6|apartment| +11533|AAAAAAAANANCAAAA|53|3rd |Boulevard|Suite 480|Maple Grove|Phillips County|AR|78252|United States|-6|condo| +11534|AAAAAAAAOANCAAAA|935|Valley |Court|Suite K|Hopewell|Mille Lacs County|MN|50587|United States|-6|single family| +11535|AAAAAAAAPANCAAAA|583|North |Ln|Suite 370|Pine Valley|Calhoun County|IA|58209|United States|-6|apartment| +11536|AAAAAAAAABNCAAAA|889|7th 2nd|Boulevard|Suite Y|Cedar Grove|Salt Lake County|UT|80411|United States|-7|single family| +11537|AAAAAAAABBNCAAAA|392|Green |Road|Suite S|Brownsville|Tolland County|CT|09910|United States|-5|apartment| +11538|AAAAAAAACBNCAAAA|406|Second Highland|Circle|Suite I|Oakdale|Hickman County|KY|49584|United States|-6|apartment| +11539|AAAAAAAADBNCAAAA|304|3rd Third|Way|Suite L|Clifton|Cole County|MO|68014|United States|-6|single family| +11540|AAAAAAAAEBNCAAAA|573|Center |Street|Suite 0|Marion|Boulder County|CO|80399|United States|-7|single family| +11541|AAAAAAAAFBNCAAAA|780|West Woodland|ST|Suite K|Wilton|Carroll County|TN|36997|United States|-5|single family| +11542|AAAAAAAAGBNCAAAA|743|Davis |Street|Suite B|Oak Hill|Kosciusko County|IN|47838|United States|-5|single family| +11543|AAAAAAAAHBNCAAAA|748|Walnut Meadow|Avenue|Suite 140|Woodland|Caswell County|NC|24854|United States|-5|condo| +11544|AAAAAAAAIBNCAAAA|743|Green |Blvd|Suite N|Lincoln|Allegany County|NY|11289|United States|-5|condo| +11545|AAAAAAAAJBNCAAAA|454|Washington 1st|Boulevard|Suite I|Mount Zion|Sequoyah County|OK|78054|United States|-6|apartment| +11546|AAAAAAAAKBNCAAAA|121|Lakeview |Boulevard|Suite 190|Spring Hill|Presidio County|TX|76787|United States|-6|apartment| +11547|AAAAAAAALBNCAAAA|614|Oak Walnut|Lane|Suite D|Jackson|Fulton County|GA|39583|United States|-5|condo| +11548|AAAAAAAAMBNCAAAA|282|Second |Ave|Suite 180|Bunker Hill|Grady County|OK|70150|United States|-6|single family| +11549|AAAAAAAANBNCAAAA|943|Railroad Lake|Ln|Suite D|River Oaks|Deer Lodge County|MT|68075|United States|-7|condo| +11550|AAAAAAAAOBNCAAAA|158|Railroad |Circle|Suite G|Walnut Grove|Halifax County|NC|27752|United States|-5|apartment| +11551|AAAAAAAAPBNCAAAA|579|Church |Way|Suite E|Mount Vernon|Cocke County|TN|38482|United States|-5|apartment| +11552|AAAAAAAAACNCAAAA|720|1st Maple|Ave|Suite O|Summit|Ouray County|CO|80499|United States|-7|apartment| +11553|AAAAAAAABCNCAAAA|||Drive||Valley View|||45124|United States||| +11554|AAAAAAAACCNCAAAA|866|Washington Main|Ave|Suite Y|Oak Grove|San Bernardino County|CA|98370|United States|-8|condo| +11555|AAAAAAAADCNCAAAA|370|3rd South|Ave|Suite C|Enterprise|Big Horn County|WY|81757|United States|-7|apartment| +11556|AAAAAAAAECNCAAAA|630|Spruce Park|Way|Suite B|Lincoln|Onondaga County|NY|11289|United States|-5|single family| +11557|AAAAAAAAFCNCAAAA|990|West Cedar|Ave|Suite 400|Shiloh|Fresno County|CA|99275|United States|-8|single family| +11558|AAAAAAAAGCNCAAAA|556|Broadway |Circle|Suite B||Buchanan County|||||single family| +11559|AAAAAAAAHCNCAAAA|756|First 4th|Wy|Suite 120|Lakewood|Barry County|MO|68877|United States|-6|apartment| +11560|AAAAAAAAICNCAAAA|9|Center |Circle|Suite T|Edgewood|Shannon County|SD|50069|United States|-7|condo| +11561|AAAAAAAAJCNCAAAA|766|Sunset 12th|Ave|Suite G|Liberty|Cherokee County|AL|33451|United States|-6|condo| +11562|AAAAAAAAKCNCAAAA|358|Green Park|Ln|Suite M|Arlington|Hettinger County|ND|56557|United States|-6|apartment| +11563|AAAAAAAALCNCAAAA|653|Ridge Jefferson|Wy|Suite 280|New Hope|Hughes County|SD|59431|United States|-7|apartment| +11564|AAAAAAAAMCNCAAAA|231|Maple |Street|Suite 40|New Hope|Bristol County|RI|09431|United States|-5|apartment| +11565|AAAAAAAANCNCAAAA|726|6th |Boulevard|Suite S|Mount Pleasant|Graham County|NC|21933|United States|-5|condo| +11566|AAAAAAAAOCNCAAAA|941|Sixth Lincoln|Pkwy|Suite 470|Red Hill|Potter County|TX|74338|United States|-6|condo| +11567|AAAAAAAAPCNCAAAA|977|Woodland |Drive|Suite 480|Belmont|Perry County|MS|50191|United States|-6|apartment| +11568|AAAAAAAAADNCAAAA|806|1st First|Avenue|Suite R|Riverside|Cowlitz County|WA|99231|United States|-8|single family| +11569|AAAAAAAABDNCAAAA|819|Spring Jackson|Lane|Suite R|Newport|Obion County|TN|31521|United States|-6|condo| +11570|AAAAAAAACDNCAAAA|342|Sycamore |Parkway|Suite 450|Friendship|Beaverhead County|MT|64536|United States|-7|condo| +11571|AAAAAAAADDNCAAAA|656|River 3rd|Avenue|Suite M|Red Hill|Phelps County|MO|64338|United States|-6|apartment| +11572|AAAAAAAAEDNCAAAA|833|Fifth Poplar|Pkwy|Suite Y|Springfield|Oldham County|TX|79303|United States|-6|apartment| +11573|AAAAAAAAFDNCAAAA|639|Park Hill|Boulevard|Suite F|Woodland|Corson County|SD|54854|United States|-6|single family| +11574|AAAAAAAAGDNCAAAA|643|Ash |Lane|Suite 40|Newtown|Breathitt County|KY|41749|United States|-6|single family| +11575|AAAAAAAAHDNCAAAA|466|View Main|Street|Suite 200|Glendale|Santa Cruz County|CA|93951|United States|-8|apartment| +11576|AAAAAAAAIDNCAAAA|163|2nd Lake|Blvd|Suite X|Glendale|Saline County|NE|63951|United States|-7|single family| +11577|AAAAAAAAJDNCAAAA|485|Seventh 2nd|Avenue|Suite B|Five Points|Wise County|TX|76098|United States|-6|condo| +11578|AAAAAAAAKDNCAAAA|940|Cedar |Road|Suite I|Springfield|Kinney County|TX|79303|United States|-6|condo| +11579|AAAAAAAALDNCAAAA|978|11th 8th|Street|Suite X|Bethel|Sutter County|CA|95281|United States|-8|apartment| +11580|AAAAAAAAMDNCAAAA|846|Central |Lane|Suite 390|Stafford|Washington County|ME|05580|United States|-5|apartment| +11581|AAAAAAAANDNCAAAA|579|Dogwood Third|Road|Suite 290|Sunnyside|Madison County|MO|61952|United States|-6|apartment| +11582|AAAAAAAAODNCAAAA|959|Hillcrest |Lane|Suite N|Jamestown|York County|ME|07467|United States|-5|condo| +11583|AAAAAAAAPDNCAAAA|521|College |Cir.|Suite 20|Riverview|Buckingham County|VA|29003|United States|-5|condo| +11584|AAAAAAAAAENCAAAA|605|Cedar |RD|Suite G|Greenville|Claiborne Parish|LA|71387|United States|-6|single family| +11585|AAAAAAAABENCAAAA|454|First First|Boulevard|Suite 50|Providence|Hillsdale County|MI|46614|United States|-5|single family| +11586|AAAAAAAACENCAAAA|63|||||||46060|United States|-5|single family| +11587|AAAAAAAADENCAAAA|597|Spring Lincoln|Wy|Suite W|Pomona|Harlan County|NE|64153|United States|-6|condo| +11588|AAAAAAAAEENCAAAA|105|Maple |ST|Suite D|Riverdale|Powell County|KY|49391|United States|-5|single family| +11589|AAAAAAAAFENCAAAA|183|Park Ridge|Ln|Suite 130|Fairfield|Wyoming County|PA|16192|United States|-5|condo| +11590|AAAAAAAAGENCAAAA|254|Cedar 5th|Ave|Suite 250|Red Hill|Winnebago County|WI|54338|United States|-6|apartment| +11591|AAAAAAAAHENCAAAA|127|Park View|Lane|Suite 450|Gravel Hill|Rockcastle County|KY|41944|United States|-5|single family| +11592|AAAAAAAAIENCAAAA|937|Cherry 9th|Avenue|Suite J|Mount Vernon|Hudson County|NJ|09082|United States|-5|condo| +11593|AAAAAAAAJENCAAAA||Williams |Drive||Five Forks|||||-6|| +11594|AAAAAAAAKENCAAAA|643|Railroad |Avenue|Suite 470|Greenville|Dodge County|WI|51387|United States|-6|condo| +11595|AAAAAAAALENCAAAA|887|Willow |Ave|Suite A|Newtown|Childress County|TX|71749|United States|-6|apartment| +11596|AAAAAAAAMENCAAAA|557|Cherry Poplar|Ct.|Suite K|Farmington|Hamilton County|TX|79145|United States|-6|condo| +11597|AAAAAAAANENCAAAA|833|Sunset East|Circle|Suite 480|Pinhook|Monroe County|FL|39398|United States|-5|single family| +11598|AAAAAAAAOENCAAAA|433|East |Ave|Suite 250|Pleasant Grove|Grant County|ND|54136|United States|-6|apartment| +11599|AAAAAAAAPENCAAAA|308|Valley Main|Parkway|Suite 80|Parkwood|Colleton County|SC|21669|United States|-5|single family| +11600|AAAAAAAAAFNCAAAA|264|East Woodland|Circle|Suite X|Woodland|Coffee County|AL|34854|United States|-6|condo| +11601|AAAAAAAABFNCAAAA|626|Maple Pine|Street|Suite 230|Mount Zion|Marion County|TX|78054|United States|-6|condo| +11602|AAAAAAAACFNCAAAA|570|Pine 7th|Ave|Suite F|Pleasant Hill|Brown County|IL|63604|United States|-6|apartment| +11603|AAAAAAAADFNCAAAA|866|10th |Parkway|Suite K|White Oak|Greene County|MO|66668|United States|-6|condo| +11604|AAAAAAAAEFNCAAAA|699|Hill 2nd|Ave|Suite 350|Cedar Grove|Gladwin County|MI|40411|United States|-5|condo| +11605|AAAAAAAAFFNCAAAA||||||||||-6|single family| +11606|AAAAAAAAGFNCAAAA|140|Third Second|Cir.|Suite J|Hillcrest|Dickens County|TX|73003|United States|-6|single family| +11607|AAAAAAAAHFNCAAAA|678|Pine |Drive|Suite 100|Lakewood|Howard County|MD|28877|United States|-5|apartment| +11608|AAAAAAAAIFNCAAAA|189|Jackson 7th|Parkway|Suite 360|Lakewood|Dade County|GA|38877|United States|-5|single family| +11609|AAAAAAAAJFNCAAAA|516|Forest Chestnut|Pkwy|Suite X|Kingston|Appling County|GA|34975|United States|-5|apartment| +11610|AAAAAAAAKFNCAAAA|757|Hickory |Ln|Suite I|Antioch|Waynesboro city|VA|28605|United States|-5|condo| +11611|AAAAAAAALFNCAAAA|251|Jefferson Fifth|Blvd|Suite E|Five Points|Bradford County|PA|16098|United States|-5|single family| +11612|AAAAAAAAMFNCAAAA|974|3rd 4th|Wy|Suite 450|Spring Hill|Shannon County|SD|56787|United States|-7|condo| +11613|AAAAAAAANFNCAAAA|930|6th Park|Circle|Suite 150|Springfield|Platte County|MO|69303|United States|-6|single family| +11614|AAAAAAAAOFNCAAAA|746|15th Davis|Wy|Suite 130|Flint|Montgomery County|VA|28909|United States|-5|condo| +11615|AAAAAAAAPFNCAAAA|795|Meadow |RD|Suite 290|Salem|Waushara County|WI|58048|United States|-6|apartment| +11616|AAAAAAAAAGNCAAAA|767|Miller Lake|Blvd|Suite G|Riverside|Meade County|KY|49231|United States|-5|condo| +11617|AAAAAAAABGNCAAAA|153|North |Way|Suite 180|Harmony|Cannon County|TN|35804|United States|-5|apartment| +11618|AAAAAAAACGNCAAAA|562|Smith |Blvd|Suite 370|Oak Ridge|Choctaw County|MS|58371|United States|-6|apartment| +11619|AAAAAAAADGNCAAAA|569|Johnson Adams|Pkwy|Suite 280|Sunnyside|Greene County|AR|71952|United States|-6|condo| +11620|AAAAAAAAEGNCAAAA|22|Lakeview |ST|Suite 400|Highland Park|Scott County|MN|56534|United States|-6|condo| +11621|AAAAAAAAFGNCAAAA|521|Highland |Dr.|Suite 50|Welcome|Black Hawk County|IA|56386|United States|-6|condo| +11622|AAAAAAAAGGNCAAAA|801|Washington 4th|RD|Suite 280|Bunker Hill|Green Lake County|WI|50150|United States|-6|condo| +11623|AAAAAAAAHGNCAAAA|421|Main Walnut|Lane|Suite D|Point Pleasant|San Patricio County|TX|74749|United States|-6|condo| +11624|AAAAAAAAIGNCAAAA|515|Davis North|Parkway|Suite E|Springdale|Franklin County|IL|68883|United States|-6|condo| +11625|AAAAAAAAJGNCAAAA|117|Franklin |Avenue|Suite G|Pleasant Grove|Sumter County|AL|34136|United States|-6|condo| +11626|AAAAAAAAKGNCAAAA|799|Hillcrest Maple|RD|Suite 0|Newport|Clarke County|VA|21521|United States|-5|condo| +11627|AAAAAAAALGNCAAAA|690|West |Wy|Suite V|Glendale|Bexar County|TX|73951|United States|-6|condo| +11628|AAAAAAAAMGNCAAAA|253|1st Chestnut|Circle|Suite L|Oak Ridge|Nevada County|CA|98371|United States|-8|single family| +11629|AAAAAAAANGNCAAAA|903|Lake |Road|Suite L|Shiloh|Carroll County|OH|49275|United States|-5|apartment| +11630|AAAAAAAAOGNCAAAA|686|River |Road|Suite S|Allison|Hancock County|IA|54167|United States|-6|single family| +11631|AAAAAAAAPGNCAAAA|685|1st |Circle|Suite 480|Lakeside|Murray County|GA|39532|United States|-5|apartment| +11632|AAAAAAAAAHNCAAAA|260|11th 6th|Ln|Suite K|Riverside|Williamson County|IL|69231|United States|-6|single family| +11633|AAAAAAAABHNCAAAA|656|6th Lee|Road|Suite 450|Harmony|Bristol Bay Borough|AK|95804|United States|-9|condo| +11634|AAAAAAAACHNCAAAA|195|Center 3rd|Lane|Suite 430|Deerfield|Giles County|TN|39840|United States|-5|single family| +11635|AAAAAAAADHNCAAAA|523|Oak Fourth|Drive|Suite 70|Lakeview|Steuben County|IN|48579|United States|-5|condo| +11636|AAAAAAAAEHNCAAAA|728|6th |ST|Suite F|Oak Hill|Montgomery County|MS|57838|United States|-6|single family| +11637|AAAAAAAAFHNCAAAA|950|Elm |Circle|Suite I|Wesley|Hill County|MT|61218|United States|-7|single family| +11638|AAAAAAAAGHNCAAAA|544|Sunset |Street|Suite 30|Newtown|Van Buren County|IA|51749|United States|-6|apartment| +11639|AAAAAAAAHHNCAAAA|741|Smith 3rd|Parkway|Suite K|Mount Vernon|Mora County|NM|88482|United States|-7|apartment| +11640|AAAAAAAAIHNCAAAA|200|2nd |RD|Suite N|Clifton|Jasper County|IN|48014|United States|-5|condo| +11641|AAAAAAAAJHNCAAAA|777|West Fourteenth|Drive|Suite 400|Springfield|Adair County|OK|79303|United States|-6|single family| +11642|AAAAAAAAKHNCAAAA|726|South |Road|Suite 460|Deerfield|Petroleum County|MT|69840|United States|-7|condo| +11643|AAAAAAAALHNCAAAA|378|Locust Sycamore|Boulevard|Suite 480|Lakeview|Vilas County|WI|58579|United States|-6|single family| +11644|AAAAAAAAMHNCAAAA|34|West |Way|Suite A|Lakeside|Clay County|TN|39532|United States|-5|apartment| +11645|AAAAAAAANHNCAAAA|763|Ninth Second|Dr.|Suite 440|Riverview|Chicot County|AR|79003|United States|-6|condo| +11646|AAAAAAAAOHNCAAAA|110|First |Parkway|Suite K|Plainview|Jim Hogg County|TX|73683|United States|-6|single family| +11647|AAAAAAAAPHNCAAAA|464|South |Court|Suite 210|Farmington|Greenbrier County|WV|29145|United States|-5|condo| +11648|AAAAAAAAAINCAAAA|||Ln|Suite O|Providence|Ouray County||86614|United States|-7|condo| +11649|AAAAAAAABINCAAAA|426|Spring Lakeview|Lane|Suite O|Plainview|Rutherford County|NC|23683|United States|-5|apartment| +11650|AAAAAAAACINCAAAA||Johnson Davis|||Franklin||PA||||| +11651|AAAAAAAADINCAAAA|167|Chestnut |Avenue|Suite 210|Jackson|Owsley County|KY|49583|United States|-5|single family| +11652|AAAAAAAAEINCAAAA|880|4th |Pkwy|Suite O|Farmington|Clay County|FL|39145|United States|-5|apartment| +11653|AAAAAAAAFINCAAAA|452|East |Road|Suite 0|Crossroads|Terrell County|GA|30534|United States|-5|single family| +11654|AAAAAAAAGINCAAAA|407|Maple |Road|Suite B|Bridgeport|Campbell County|KY|45817|United States|-6|apartment| +11655|AAAAAAAAHINCAAAA|407|East |Wy|Suite I|Saratoga|Attala County|MS|52123|United States|-6|single family| +11656|AAAAAAAAIINCAAAA|780|Second Dogwood|Street|Suite 470|Oak Hill|Caribou County|ID|87838|United States|-7|single family| +11657|AAAAAAAAJINCAAAA|326|Church |RD|Suite V|Maple Grove|Broome County|NY|18252|United States|-5|condo| +11658|AAAAAAAAKINCAAAA|756|Meadow |Parkway|Suite 180|Spring Valley|Adams County|WA|96060|United States|-8|apartment| +11659|AAAAAAAALINCAAAA|996|South Franklin|Avenue|Suite G|Spring Hill|Davie County|NC|26787|United States|-5|apartment| +11660|AAAAAAAAMINCAAAA|631|Sycamore |Parkway|Suite O|Mount Tabor|Mille Lacs County|MN|57702|United States|-6|single family| +11661|AAAAAAAANINCAAAA|656|6th Maple|RD|Suite P|Newtown|Los Angeles County|CA|91749|United States|-8|apartment| +11662|AAAAAAAAOINCAAAA|202|5th |Dr.|Suite 430|Wilson|Brown County|WI|56971|United States|-6|single family| +11663|AAAAAAAAPINCAAAA|272|Hillcrest Center|RD|Suite M|White Oak|Elko County|NV|86668|United States|-8|apartment| +11664|AAAAAAAAAJNCAAAA|29|Davis |Circle|Suite 130|Mount Vernon|Pulaski County|AR|78482|United States|-6|condo| +11665|AAAAAAAABJNCAAAA|863|Smith First|Blvd|Suite 450|Unionville|Greeley County|NE|61711|United States|-6|condo| +11666|AAAAAAAACJNCAAAA|613|Spruce |Cir.|Suite Y|Belmont|Tolland County|CT|00791|United States|-5|single family| +11667|AAAAAAAADJNCAAAA|181|Lee |Ct.|Suite 240|Vienna|Lafayette County|MS|55119|United States|-6|apartment| +11668|AAAAAAAAEJNCAAAA|91|Park |Court|Suite N|Antioch|Jefferson County|MT|68605|United States|-7|apartment| +11669|AAAAAAAAFJNCAAAA|232|Wilson 3rd|ST|Suite O|Plainview|Alachua County|FL|33683|United States|-5|condo| +11670|AAAAAAAAGJNCAAAA|829|Davis Second|Circle|Suite G|Woodlawn|Mercer County|PA|14098|United States|-5|apartment| +11671|AAAAAAAAHJNCAAAA|218|Ninth |Lane|Suite W|Midway|Madison County|IN|41904|United States|-5|condo| +11672|AAAAAAAAIJNCAAAA|596|Pine Spruce|Drive|Suite M|Union Hill|Saguache County|CO|87746|United States|-7|condo| +11673|AAAAAAAAJJNCAAAA|388|West |Road|Suite 240|Enterprise|Yellowstone County|MT|61757|United States|-7|apartment| +11674|AAAAAAAAKJNCAAAA|287|Hillcrest |Dr.|Suite 50|Forest Hills|Clay County|NE|69237|United States|-6|apartment| +11675|AAAAAAAALJNCAAAA|163|Willow Tenth|Blvd|Suite 350|Little River|Houston County|TN|30319|United States|-5|single family| +11676|AAAAAAAAMJNCAAAA|126|Hill |Lane|Suite 440|Five Points|Dundy County|NE|66098|United States|-6|single family| +11677|AAAAAAAANJNCAAAA|538|Mill Valley|Blvd|Suite A||Hancock County|||United States||apartment| +11678|AAAAAAAAOJNCAAAA|187|College |Cir.|Suite C|Woodville|Norton city|VA|24289|United States|-5|condo| +11679|AAAAAAAAPJNCAAAA|385|Pine Chestnut|ST|Suite W|Tyrone|Randolph County|WV|21201|United States|-5|single family| +11680|AAAAAAAAAKNCAAAA|912|Walnut Cedar|Parkway|Suite 80|Clifton|Lincoln County|SD|58014|United States|-7|single family| +11681|AAAAAAAABKNCAAAA|421|Willow |Parkway|Suite I|Antioch|Horry County|SC|28605|United States|-5|single family| +11682|AAAAAAAACKNCAAAA|989|Fourth River|Dr.|Suite 160|Pine Grove|Tangipahoa Parish|LA|74593|United States|-6|condo| +11683|AAAAAAAADKNCAAAA|530|Poplar |RD|Suite 210|Springdale|Orange County|TX|78883|United States|-6|single family| +11684|AAAAAAAAEKNCAAAA|91|Elm |Drive|Suite A|Springdale|Gray County|KS|68883|United States|-6|condo| +11685|AAAAAAAAFKNCAAAA|594|Johnson |RD|Suite C|Pleasant Grove|Meriwether County|GA|34136|United States|-5|apartment| +11686|AAAAAAAAGKNCAAAA|242|Fourth |Road|Suite 40|Jackson|Clark County|IL|69583|United States|-6|apartment| +11687|AAAAAAAAHKNCAAAA|867|Elm |Lane|Suite 0|Springfield|Lawrence County|IN|49303|United States|-5|single family| +11688|AAAAAAAAIKNCAAAA|509|Poplar Williams|Way|Suite 460|Kingston|Waupaca County|WI|54975|United States|-6|single family| +11689|AAAAAAAAJKNCAAAA|698|11th Mill|Road|Suite K|Forest Hills|Hamlin County|SD|59237|United States|-7|single family| +11690|AAAAAAAAKKNCAAAA|198|Cedar |Ln|Suite H|Guilford|Union County|KY|44408|United States|-5|single family| +11691|AAAAAAAALKNCAAAA|111|Pine |ST|Suite U|Hillsboro|Charles City County|VA|22532|United States|-5|apartment| +11692|AAAAAAAAMKNCAAAA|809|Madison |Ln|Suite 450|Five Points|DuPage County|IL|66098|United States|-6|apartment| +11693|AAAAAAAANKNCAAAA|291|Valley 12th|Cir.|Suite N|Acme|Rutland County|VT|00764|United States|-5|condo| +11694|AAAAAAAAOKNCAAAA|591|11th |Wy|Suite K|Sheffield|Bottineau County|ND|56896|United States|-6|apartment| +11695|AAAAAAAAPKNCAAAA|522|North |ST|Suite 60|Edgewood|Cocke County|TN|30069|United States|-5|single family| +11696|AAAAAAAAALNCAAAA|41|Sycamore Franklin|Ln|Suite 460|Sunnyside|Ouray County|CO|81952|United States|-7|apartment| +11697|AAAAAAAABLNCAAAA|217|South 2nd|Road|Suite 300|Brownsville|Baca County|CO|89310|United States|-7|apartment| +11698|AAAAAAAACLNCAAAA|433|Seventh |Circle|Suite X|Brownsville|Harper County|KS|69310|United States|-6|single family| +11699|AAAAAAAADLNCAAAA|696|Maple |Boulevard|Suite H|Cedar Grove|Tippah County|MS|50411|United States|-6|condo| +11700|AAAAAAAAELNCAAAA|505|Sixth |Road|Suite O|Willis|Dougherty County|GA|36788|United States|-5|condo| +11701|AAAAAAAAFLNCAAAA|42|Jackson Miller|ST|Suite U|Hamilton|Apache County|AZ|82808|United States|-7|single family| +11702|AAAAAAAAGLNCAAAA|784|East |Cir.|Suite R|Union Hill|Van Buren County|TN|37746|United States|-6|apartment| +11703|AAAAAAAAHLNCAAAA|54|Jackson First|Drive|Suite 380|Glendale|Fremont County|CO|83951|United States|-7|condo| +11704|AAAAAAAAILNCAAAA|450|1st 1st|Blvd|Suite M|Riverside|Orleans Parish|LA|79231|United States|-6|single family| +11705|AAAAAAAAJLNCAAAA|126|Park Sunset|Way|Suite U|White Oak|Aurora County|SD|56668|United States|-6|single family| +11706|AAAAAAAAKLNCAAAA|267|Main |Road|Suite N|Harmony|Acadia Parish|LA|75804|United States|-6|condo| +11707|AAAAAAAALLNCAAAA|662|Washington Park|Cir.|Suite S|Oak Grove|Forest County|WI|58370|United States|-6|single family| +11708|AAAAAAAAMLNCAAAA|442|Lincoln |Ln|Suite 450|Red Hill|Chickasaw County|IA|54338|United States|-6|condo| +11709|AAAAAAAANLNCAAAA|979|Valley 11th|Blvd|Suite 340|Centerville|Waukesha County|WI|50059|United States|-6|apartment| +11710|AAAAAAAAOLNCAAAA|40|Sunset Park|Wy|Suite 420|Salem|Wayne County|NC|28048|United States|-5|single family| +11711|AAAAAAAAPLNCAAAA|808|Fourth |Ct.|Suite 130|Crossroads|Clackamas County|OR|90534|United States|-8|condo| +11712|AAAAAAAAAMNCAAAA|928|Ridge 10th|Ct.|Suite G|Marion|Nicollet County|MN|50399|United States|-6|condo| +11713|AAAAAAAABMNCAAAA|216|Johnson Elm||||||69583|United States|-6|| +11714|AAAAAAAACMNCAAAA|335|Hillcrest |Blvd|Suite 30|Murray|Pueblo County|CO|82150|United States|-7|condo| +11715|AAAAAAAADMNCAAAA|325|Hill |Street|Suite X|Pleasant Hill|Washington County|NC|23604|United States|-5|condo| +11716|AAAAAAAAEMNCAAAA|810|Third |RD|Suite B|Deerfield|Chase County|KS|69840|United States|-6|apartment| +11717|AAAAAAAAFMNCAAAA||||Suite 240|Pine Grove|||64593||-6|| +11718|AAAAAAAAGMNCAAAA|348|Lake |Lane|Suite R|Lebanon|De Soto Parish|LA|72898|United States|-6|condo| +11719|AAAAAAAAHMNCAAAA|514|Main |Ave|Suite B|Greenville|Osage County|KS|61387|United States|-6|condo| +11720|AAAAAAAAIMNCAAAA|772|Main |Wy|Suite 60|Blanchard|Hughes County|SD|55985|United States|-7|apartment| +11721|AAAAAAAAJMNCAAAA|871|6th View|Ave|Suite 320|Oakwood|Orange County|NY|10169|United States|-5|single family| +11722|AAAAAAAAKMNCAAAA|895|Cedar 2nd|Ct.|Suite 220|Harmon|Galax city|VA|25623|United States|-5|apartment| +11723|AAAAAAAALMNCAAAA|12|Birch |Ave|Suite 50|Jamestown|Laurens County|SC|26867|United States|-5|single family| +11724|AAAAAAAAMMNCAAAA|402|Lake Hickory|Ln|Suite 350|Hamilton|Sumner County|TN|32808|United States|-6|apartment| +11725|AAAAAAAANMNCAAAA|630|Lincoln |Lane|Suite 330|Hillcrest|Warren County|TN|33003|United States|-6|single family| +11726|AAAAAAAAOMNCAAAA|127|Hill |Pkwy|Suite 360|New Hope|Tate County|MS|59431|United States|-6|condo| +11727|AAAAAAAAPMNCAAAA|798|3rd |Circle|Suite W|Friendship|Jackson County|CO|84536|United States|-7|apartment| +11728|AAAAAAAAANNCAAAA|167|Forest |Drive|Suite Y|Glendale|White County|AR|73951|United States|-6|condo| +11729|AAAAAAAABNNCAAAA|529|Main Elm|Ct.|Suite 110|Friendship|Rowan County|KY|44536|United States|-5|single family| +11730|AAAAAAAACNNCAAAA|897|Spring |Pkwy|Suite M|Concord|Rock County|MN|54107|United States|-6|condo| +11731|AAAAAAAADNNCAAAA|52|5th |Road|Suite 380|Gladstone|Gregg County|TX|70894|United States|-6|apartment| +11732|AAAAAAAAENNCAAAA|794|Center |Wy|Suite D|White Oak|Clay County|WV|26668|United States|-5|single family| +11733|AAAAAAAAFNNCAAAA|14|Walnut West|Ct.|Suite 80|Greenwood|Santa Rosa County|FL|38828|United States|-5|single family| +11734|AAAAAAAAGNNCAAAA|342|1st |Ct.|Suite E|Red Hill|Stephens County|TX|74338|United States|-6|condo| +11735|AAAAAAAAHNNCAAAA|||Avenue|Suite W|Riverside||FL|39231|||| +11736|AAAAAAAAINNCAAAA|505|Center |Parkway|Suite 420|Green Acres|Ketchikan Gateway Borough|AK|97683|United States|-9|single family| +11737|AAAAAAAAJNNCAAAA|381|College |Boulevard|Suite Y|Wilson|Susquehanna County|PA|16971|United States|-5|apartment| +11738|AAAAAAAAKNNCAAAA|920|East |Road|Suite N|Mount Vernon|Monmouth County|NJ|09082|United States|-5|apartment| +11739|AAAAAAAALNNCAAAA|337|Ash |Parkway|Suite 60|Springfield|Clay County|SD|59303|United States|-6|single family| +11740|AAAAAAAAMNNCAAAA|577|Park 3rd|Way|Suite X|Concord|Kinney County|TX|74107|United States|-6|apartment| +11741|AAAAAAAANNNCAAAA|440|12th |Street|Suite 260|Lakeside|Lincoln County|ID|89532|United States|-7|apartment| +11742|AAAAAAAAONNCAAAA|877|River |Road|Suite 250|Union|Boone County|WV|28721|United States|-5|apartment| +11743|AAAAAAAAPNNCAAAA|590|Walnut Central|Dr.|Suite 50|Providence|Portage County|WI|56614|United States|-6|apartment| +11744|AAAAAAAAAONCAAAA|219|Main Jefferson|Ln|Suite K|Price|Union County|NM|83824|United States|-7|apartment| +11745|AAAAAAAABONCAAAA|506|View |Blvd|Suite 90|Fremont|Washtenaw County|MI|41851|United States|-5|single family| +11746|AAAAAAAACONCAAAA|153|Pine 11th|Cir.|Suite H|Kingston|Oliver County|ND|54975|United States|-6|apartment| +11747|AAAAAAAADONCAAAA|447|Smith |Drive|Suite 190|Littleton|Chippewa County|MN|56074|United States|-6|condo| +11748|AAAAAAAAEONCAAAA|101|River |Court|Suite 100|Shiloh|Pershing County|NV|89275|United States|-8|apartment| +11749|AAAAAAAAFONCAAAA|211|15th |Lane|Suite N|Shady Grove|Calhoun County|MI|42812|United States|-5|condo| +11750|AAAAAAAAGONCAAAA|865|6th College|Road|Suite 340|Oak Ridge|Logan County|IL|68371|United States|-6|condo| +11751|AAAAAAAAHONCAAAA|845|Park 6th|Parkway|Suite A|Florence|Ralls County|MO|63394|United States|-6|single family| +11752|AAAAAAAAIONCAAAA|407|Davis |Cir.|Suite P|Perkins|Lauderdale County|AL|31852|United States|-6|single family| +11753|AAAAAAAAJONCAAAA|886|Park |Ct.|Suite 50|Cherry Valley|Hamilton County|TX|70854|United States|-6|condo| +11754|AAAAAAAAKONCAAAA|604|3rd 8th|Parkway|Suite 290|Jones|Johnson County|GA|32686|United States|-5|condo| +11755|AAAAAAAALONCAAAA|216|Dogwood Cherry|Avenue|Suite I|Lakeside|Orange County|FL|39532|United States|-5|single family| +11756|AAAAAAAAMONCAAAA|47|Washington |Cir.|Suite 150|Forest Hills|Sumter County|AL|39237|United States|-6|apartment| +11757|AAAAAAAANONCAAAA|983|Jefferson Cherry|Court|Suite R|Mount Vernon|Muhlenberg County|KY|48482|United States|-5|single family| +11758|AAAAAAAAOONCAAAA|476|Jefferson Railroad|Circle|Suite 80|Newport|Marion County|SC|21521|United States|-5|single family| +11759|AAAAAAAAPONCAAAA|82|West Cedar|Street|Suite L|Hopewell|Haakon County|SD|50587|United States|-7|apartment| +11760|AAAAAAAAAPNCAAAA|967|Center |Ct.|Suite P|Howell|Robertson County|TX|74854|United States|-6|single family| +11761|AAAAAAAABPNCAAAA|573|Cherry Jackson|Ave|Suite D|Five Forks|Ford County|KS|62293|United States|-6|condo| +11762|AAAAAAAACPNCAAAA|995|Forest Church|Blvd|Suite 330|Lebanon|Chaves County|NM|82898|United States|-7|single family| +11763|AAAAAAAADPNCAAAA|626|Spring |Parkway|Suite D|Greenfield|Kern County|CA|95038|United States|-8|apartment| +11764|AAAAAAAAEPNCAAAA|223|Sunset 4th|Ct.|Suite 180|Stringtown|Val Verde County|TX|70162|United States|-6|condo| +11765|AAAAAAAAFPNCAAAA|678|Hickory Lake|Pkwy|Suite 80|Buena Vista|Woodward County|OK|75752|United States|-6|apartment| +11766|AAAAAAAAGPNCAAAA|637|12th |Wy|Suite S|Wilson|Chautauqua County|KS|66971|United States|-6|condo| +11767|AAAAAAAAHPNCAAAA|412|River |RD|Suite Q|Fox|Fairfax County|VA|20631|United States|-5|condo| +11768|AAAAAAAAIPNCAAAA|577|Elm |Parkway|Suite M|Hamilton|Modoc County|CA|92808|United States|-8|apartment| +11769|AAAAAAAAJPNCAAAA|7|3rd |Ave|Suite 410|Bridgeport|DeSoto County|FL|35817|United States|-5|condo| +11770|AAAAAAAAKPNCAAAA|922|Valley |Cir.|Suite H|Newtown|Steuben County|NY|11749|United States|-5|apartment| +11771|AAAAAAAALPNCAAAA|413|Washington Church|Boulevard|Suite 170|Springtown|Sioux County|IA|59858|United States|-6|apartment| +11772|AAAAAAAAMPNCAAAA|984|Sixth 15th|Road|Suite X|Arlington|Benton County|MN|56557|United States|-6|single family| +11773|AAAAAAAANPNCAAAA|998|Valley |RD|Suite 340|Mount Olive|Hamilton County|IN|48059|United States|-5|single family| +11774|AAAAAAAAOPNCAAAA|541|Cedar 4th|Drive|Suite 130|Greenwood|Montgomery County|KY|48828|United States|-5|condo| +11775|AAAAAAAAPPNCAAAA|534|Park |Ln|Suite 50|Bayview|La Paz County|AZ|89672|United States|-7|condo| +11776|AAAAAAAAAAOCAAAA|941|Laurel |Street|Suite S|Maple Grove|Alpena County|MI|48252|United States|-5|condo| +11777|AAAAAAAABAOCAAAA|247|10th |Ln|Suite 130|Bunker Hill|Lassen County|CA|90150|United States|-8|single family| +11778|AAAAAAAACAOCAAAA|15|6th |Boulevard|Suite 290|Spring Hill|Tallapoosa County|AL|36787|United States|-6|condo| +11779|AAAAAAAADAOCAAAA|658|4th Jefferson|Pkwy|Suite 310|Fairfield|Cherry County|NE|66192|United States|-6|condo| +11780|AAAAAAAAEAOCAAAA|452|Birch |Ln|Suite J|Concord|Grant County|SD|54107|United States|-6|apartment| +11781|AAAAAAAAFAOCAAAA|461|6th Hickory|Street|Suite Q|Shiloh|Banks County|GA|39275|United States|-5|apartment| +11782|AAAAAAAAGAOCAAAA|468|Ash |Drive|Suite E|Brentwood|Ontonagon County|MI|44188|United States|-5|apartment| +11783|AAAAAAAAHAOCAAAA|112|Walnut Davis|Pkwy|Suite 230|Mountain View|Missoula County|MT|64466|United States|-7|condo| +11784|AAAAAAAAIAOCAAAA|883|Davis |RD|Suite G|Berea|Athens County|OH|43804|United States|-5|condo| +11785|AAAAAAAAJAOCAAAA|103||Cir.||Shady Grove|Lavaca County|TX|72812|||| +11786|AAAAAAAAKAOCAAAA|57|Pine Miller|Cir.|Suite W|Wilson|Jenkins County|GA|36971|United States|-5|single family| +11787|AAAAAAAALAOCAAAA|506|Jackson |Parkway|Suite W|Centerville|Goodhue County|MN|50059|United States|-6|single family| +11788|AAAAAAAAMAOCAAAA|619|Sixth |Drive|Suite 450|Franklin|Garfield County|MT|69101|United States|-7|single family| +11789|AAAAAAAANAOCAAAA|663|Lincoln Church|Boulevard|Suite 70|Shady Grove|Marshall County|WV|22812|United States|-5|condo| +11790|AAAAAAAAOAOCAAAA|78|Church |Dr.|Suite K|Pleasant Grove|Haskell County|OK|74136|United States|-6|condo| +11791|AAAAAAAAPAOCAAAA|739|7th |Pkwy|Suite 180|Greenville|Mayes County|OK|71387|United States|-6|apartment| +11792|AAAAAAAAABOCAAAA|340|Sycamore Elm|Street|Suite J|Hamilton|Bernalillo County|NM|82808|United States|-7|apartment| +11793|AAAAAAAABBOCAAAA|297|Davis Meadow|Road|Suite Y|Shady Grove|Menard County|TX|72812|United States|-6|apartment| +11794|AAAAAAAACBOCAAAA|975|Johnson |Avenue|Suite 290|Lebanon|Claiborne Parish|LA|72898|United States|-6|single family| +11795|AAAAAAAADBOCAAAA|95|Park Maple|Ln|Suite 330|Forest Hills|Osborne County|KS|69237|United States|-6|single family| +11796|AAAAAAAAEBOCAAAA|278|Sunset North|Dr.|Suite M|Mount Zion|Mason County|WV|28054|United States|-5|single family| +11797|AAAAAAAAFBOCAAAA|759|3rd Broadway|Drive|Suite K|Franklin|Macon County|GA|39101|United States|-5|single family| +11798|AAAAAAAAGBOCAAAA|374|Broadway |Circle|Suite 390|Deerfield|Lawrence County|SD|59840|United States|-7|single family| +11799|AAAAAAAAHBOCAAAA|396||Pkwy|Suite 40|||||United States||| +11800|AAAAAAAAIBOCAAAA|973|4th Railroad|Road|Suite G|Pleasant Hill|Mitchell County|NC|23604|United States|-5|condo| +11801|AAAAAAAAJBOCAAAA|316|Center Cherry|Circle|Suite L|Pleasant Valley|Richardson County|NE|62477|United States|-7|apartment| +11802|AAAAAAAAKBOCAAAA|289|North 4th|Wy|Suite C|Plainview|Burke County|ND|53683|United States|-6|condo| +11803|AAAAAAAALBOCAAAA|662|Jackson |Court|Suite 60|Riverview|Randolph County|AR|79003|United States|-6|apartment| +11804|AAAAAAAAMBOCAAAA|||RD||Bethel|Gage County|NE||||| +11805|AAAAAAAANBOCAAAA|735|Ridge Locust|Ct.|Suite L|Marion|Rock County|NE|60399|United States|-7|condo| +11806|AAAAAAAAOBOCAAAA|575|4th |Ct.|Suite E|New Hope|Lawrence County|MO|69431|United States|-6|apartment| +11807|AAAAAAAAPBOCAAAA|149|First Sixth|Cir.|Suite G|Sunnyside|Brown County|MN|51952|United States|-6|condo| +11808|AAAAAAAAACOCAAAA|532|Main |Way|Suite 290|Belmont|King County|TX|70191|United States|-6|apartment| +11809|AAAAAAAABCOCAAAA|114|Ash |Cir.|Suite N|Ashland|Hampshire County|MA|04844|United States|-5|condo| +11810|AAAAAAAACCOCAAAA|669|Spruce |Pkwy|Suite V|Glendale|Kalamazoo County|MI|43951|United States|-5|apartment| +11811|AAAAAAAADCOCAAAA|290|Spring |Pkwy|Suite 190|Mountain View|Crisp County|GA|34466|United States|-5|condo| +11812|AAAAAAAAECOCAAAA|188|Hill |Dr.|Suite 50|Greenfield|Barnwell County|SC|25038|United States|-5|apartment| +11813|AAAAAAAAFCOCAAAA|82|Franklin Spruce|Lane|Suite 260|Valley View|Mason County|KY|45124|United States|-5|condo| +11814|AAAAAAAAGCOCAAAA|586|Smith |Circle|Suite B|Forestville|Adams County|MS|53027|United States|-6|single family| +11815|AAAAAAAAHCOCAAAA|789|||Suite 450|Hillcrest|San Augustine County|TX||United States|-6|condo| +11816|AAAAAAAAICOCAAAA|333|Sunset East|Ct.|Suite E|Lincoln|Lincoln County|ME|01889|United States|-5|single family| +11817|AAAAAAAAJCOCAAAA|914|Second |Ct.|Suite X|Hamilton|Stanislaus County|CA|92808|United States|-8|single family| +11818|AAAAAAAAKCOCAAAA|601|14th |Drive|Suite 90|Jamestown|Swisher County|TX|76867|United States|-6|apartment| +11819|AAAAAAAALCOCAAAA|906|Main |Wy|Suite 40|Riverview|Renville County|ND|59003|United States|-6|single family| +11820|AAAAAAAAMCOCAAAA|342|Wilson Highland|Road|Suite 410|Salem|Pierce County|GA|38048|United States|-5|single family| +11821|AAAAAAAANCOCAAAA|876|3rd |Wy|Suite W|Concord|Red River Parish|LA|74107|United States|-6|apartment| +11822|AAAAAAAAOCOCAAAA|659|4th Twelfth|Cir.|Suite 220|Liberty|Gallatin County|MT|63451|United States|-7|apartment| +11823|AAAAAAAAPCOCAAAA|574|Lake |Blvd|Suite 360|Jackson|Phillips County|MT|69583|United States|-7|apartment| +11824|AAAAAAAAADOCAAAA|824|6th 14th|Dr.|Suite L|Oak Grove|San Miguel County|NM|88370|United States|-7|apartment| +11825|AAAAAAAABDOCAAAA|362|Elm 7th|Lane|Suite 200|Oak Ridge|Taylor County|FL|38371|United States|-5|single family| +11826|AAAAAAAACDOCAAAA|522|Hillcrest |Ln|Suite F|Farmington|Morris County|KS|69145|United States|-6|condo| +11827|AAAAAAAADDOCAAAA|899|8th |Circle|Suite J|Glendale|Bowman County|ND|53951|United States|-6|condo| +11828|AAAAAAAAEDOCAAAA|481|Broadway 13th|ST|Suite B|Prosperity|Jackson County|TN|39089|United States|-5|apartment| +11829|AAAAAAAAFDOCAAAA|485|Park |Road|Suite 240|Adrian|Otoe County|NE|63301|United States|-7|single family| +11830|AAAAAAAAGDOCAAAA|292|Elm |Avenue|Suite W|Union|Baker County|OR|98721|United States|-8|apartment| +11831|AAAAAAAAHDOCAAAA|830|Valley Chestnut|Pkwy|Suite W|Ashland|Lincoln County|NM|84244|United States|-7|single family| +11832|AAAAAAAAIDOCAAAA|183|Ash |Way|Suite 50|Ashland|Page County|VA|24244|United States|-5|apartment| +11833|AAAAAAAAJDOCAAAA|335|||Suite U|||PA||United States||apartment| +11834|AAAAAAAAKDOCAAAA|517|Church |Road|Suite Y|Mount Olive|Shenandoah County|VA|28059|United States|-5|condo| +11835|AAAAAAAALDOCAAAA|158|Franklin |Blvd|Suite 70|Concord|Lancaster County|SC|24107|United States|-5|single family| +11836|AAAAAAAAMDOCAAAA|107|Spring 9th|Ln|Suite 310|Hopewell|Seward County|NE|60587|United States|-7|apartment| +11837|AAAAAAAANDOCAAAA|202|East |RD|Suite 380|Lincoln|Montgomery County|KS|61289|United States|-6|apartment| +11838|AAAAAAAAODOCAAAA|129|Hill Laurel|Cir.|Suite W|Buena Vista|Lewis County|WA|95752|United States|-8|apartment| +11839|AAAAAAAAPDOCAAAA|990|Highland River|Circle|Suite Y|Spring Valley|Marion County|IA|56060|United States|-6|condo| +11840|AAAAAAAAAEOCAAAA|710|Meadow College|Dr.|Suite 60|Oak Hill|Culpeper County|VA|27838|United States|-5|single family| +11841|AAAAAAAABEOCAAAA|671|8th Smith|Pkwy|Suite 480|Clifton|Cherokee County|KS|68014|United States|-6|condo| +11842|AAAAAAAACEOCAAAA|817|Maple 9th|Avenue|Suite I|Ashland|Deaf Smith County|TX|74244|United States|-6|single family| +11843|AAAAAAAADEOCAAAA|616|Hillcrest 1st|Court|Suite O|Marion|Columbia County|GA|30399|United States|-5|apartment| +11844|AAAAAAAAEEOCAAAA|570|Jackson Mill|Ct.|Suite J|Appleton|Tehama County|CA|94240|United States|-8|condo| +11845|AAAAAAAAFEOCAAAA|107|Green 10th|Ln|Suite R|Edgewood|Coweta County|GA|30069|United States|-5|apartment| +11846|AAAAAAAAGEOCAAAA|360|4th Center|Street|Suite O|Union|Highlands County|FL|38721|United States|-5|condo| +11847|AAAAAAAAHEOCAAAA|313|Sycamore |Parkway|Suite I|Allison|San Bernardino County|CA|94167|United States|-8|apartment| +11848|AAAAAAAAIEOCAAAA|42|Poplar |Way|Suite I|Walnut Grove|Salem city|VA|27752|United States|-5|single family| +11849|AAAAAAAAJEOCAAAA|361|Mill |Drive|Suite 300|Greenville|Durham County|NC|21387|United States|-5|apartment| +11850|AAAAAAAAKEOCAAAA|490|Railroad Franklin|RD|Suite 460|Clinton|Campbell County|VA|28222|United States|-5|single family| +11851|AAAAAAAALEOCAAAA|467|2nd Oak|Street|Suite L|Fairview|Sutter County|CA|95709|United States|-8|apartment| +11852|AAAAAAAAMEOCAAAA|221|12th 1st|Parkway|Suite 170|Sunnyside|Kiowa County|OK|71952|United States|-6|condo| +11853|AAAAAAAANEOCAAAA|972|Maple Church|RD|Suite 390|Crossroads|Thomas County|NE|60534|United States|-7|single family| +11854|AAAAAAAAOEOCAAAA|473|Hill |Drive|Suite Q|Woodville|Harper County|KS|64289|United States|-6|condo| +11855|AAAAAAAAPEOCAAAA|696|Fifth |Ct.|Suite 130|Buena Vista|Osborne County|KS|65752|United States|-6|single family| +11856|AAAAAAAAAFOCAAAA|322|Oak |Blvd|Suite 230|Crossroads|Saluda County|SC|20534|United States|-5|condo| +11857|AAAAAAAABFOCAAAA|809|14th |Way|Suite O|Glenwood|Perry County|AR|73511|United States|-6|single family| +11858|AAAAAAAACFOCAAAA|258|Walnut Cherry|Boulevard|Suite C|Five Points|Lander County|NV|86098|United States|-8|condo| +11859|AAAAAAAADFOCAAAA|594|11th |Ave|Suite 170|New Hope|Clinton County|PA|19431|United States|-5|apartment| +11860|AAAAAAAAEFOCAAAA|446|2nd Ash|Avenue|Suite 180|Bunker Hill|Pickens County|SC|20150|United States|-5|condo| +11861|AAAAAAAAFFOCAAAA|466|1st 2nd|Blvd|Suite 70|Florence|Green County|KY|43394|United States|-6|condo| +11862|AAAAAAAAGFOCAAAA|25|Lakeview Oak|RD|Suite 210|Mount Zion|Montrose County|CO|88054|United States|-7|apartment| +11863|AAAAAAAAHFOCAAAA|981|Jackson |Blvd|Suite Q|Riverside|Douglas County|OR|99231|United States|-8|apartment| +11864|AAAAAAAAIFOCAAAA|478|Park West|Parkway|Suite G|Fairview|Lincoln County|SD|55709|United States|-7|single family| +11865|AAAAAAAAJFOCAAAA|363|Ash |Blvd|Suite C|Forest Hills|Riverside County|CA|99237|United States|-8|condo| +11866|AAAAAAAAKFOCAAAA|316|Jefferson Main|Ct.|Suite 370|Brownsville|Lincoln County|MO|69310|United States|-6|single family| +11867|AAAAAAAALFOCAAAA|726|Elm |Cir.|Suite 150|Edgewood|Musselshell County|MT|60069|United States|-7|single family| +11868|AAAAAAAAMFOCAAAA|238|Locust |Drive|Suite H|Hamilton|Columbia County|PA|12808|United States|-5|condo| +11869|AAAAAAAANFOCAAAA|39|3rd |Dr.|Suite A|Forest Hills|Franklin County|ID|89237|United States|-7|condo| +11870|AAAAAAAAOFOCAAAA|13|Oak Meadow|Road|Suite 280|Lakewood|Leslie County|KY|48877|United States|-5|apartment| +11871|AAAAAAAAPFOCAAAA|747|Adams |Blvd|Suite X|Wildwood|McPherson County|SD|56871|United States|-7|single family| +11872|AAAAAAAAAGOCAAAA||Spring Birch||Suite N|||ID|89584||-7|single family| +11873|AAAAAAAABGOCAAAA|204|Dogwood |Boulevard|Suite 490|Wildwood|Rock County|MN|56871|United States|-6|single family| +11874|AAAAAAAACGOCAAAA|117|Ninth Ridge|Way|Suite W|Mount Pleasant|Chesterfield County|SC|21933|United States|-5|single family| +11875|AAAAAAAADGOCAAAA|852|Johnson |Ln|Suite R|Bunker Hill|Essex County|NY|10150|United States|-5|condo| +11876|AAAAAAAAEGOCAAAA|214|12th River|Lane|Suite L|Spring Valley|Pershing County|NV|86060|United States|-8|apartment| +11877|AAAAAAAAFGOCAAAA|147|Walnut First|Avenue|Suite F|Five Points|Marion County|MS|56098|United States|-6|apartment| +11878|AAAAAAAAGGOCAAAA|447|Maple |Avenue|Suite I|Red Oak|Smyth County|VA|25018|United States|-5|apartment| +11879|AAAAAAAAHGOCAAAA|390|Cherry Third|Pkwy|Suite 90|Hillcrest|Lewis County|TN|33003|United States|-6|single family| +11880|AAAAAAAAIGOCAAAA|143|14th Main|Cir.|Suite P|Buena Vista|Alexander County|NC|25752|United States|-5|single family| +11881|AAAAAAAAJGOCAAAA|792|View 5th|Court|Suite X|Spring Hill|Shelby County|OH|46787|United States|-5|condo| +11882|AAAAAAAAKGOCAAAA|706|Ridge Ash|RD|Suite H|Macedonia|Nassau County|NY|11087|United States|-5|apartment| +11883|AAAAAAAALGOCAAAA|142|Davis 15th|Parkway|Suite B|Five Points|Dickinson County|KS|66098|United States|-6|single family| +11884|AAAAAAAAMGOCAAAA|850|Jackson West|Way|Suite 280|Woodlawn|Wabaunsee County|KS|64098|United States|-6|apartment| +11885|AAAAAAAANGOCAAAA|186|Maple Second|Drive|Suite 110|Antioch|Lee County|FL|38605|United States|-5|apartment| +11886|AAAAAAAAOGOCAAAA|115|Church |Ct.|Suite 380|Fowler|Garza County|TX|71083|United States|-6|single family| +11887|AAAAAAAAPGOCAAAA|436|Spruce 1st|Court|Suite K|Oak Hill|Douglas County|IL|67838|United States|-6|apartment| +11888|AAAAAAAAAHOCAAAA|169|11th |Ln|Suite 480|Spring Valley|Webster Parish|LA|76060|United States|-6|apartment| +11889|AAAAAAAABHOCAAAA|151|Lincoln |Blvd|Suite S|Glendale|McHenry County|IL|63951|United States|-6|single family| +11890|AAAAAAAACHOCAAAA|233|Ridge 4th|Cir.|Suite 180|Riverview|Lauderdale County|TN|39003|United States|-6|single family| +11891|AAAAAAAADHOCAAAA|955|Davis Jefferson|Court|Suite 50|New Hope|Kimble County|TX|79431|United States|-6|single family| +11892|AAAAAAAAEHOCAAAA|165|12th Park|Ln|Suite 280|Mountain View|Guadalupe County|TX|74466|United States|-6|apartment| +11893|AAAAAAAAFHOCAAAA|718|Pine Meadow|Blvd|Suite L|San Jose|Missaukee County|MI|48003|United States|-5|apartment| +11894|AAAAAAAAGHOCAAAA|441|North Maple|Pkwy|Suite 80|Centerville|Kaufman County|TX|70059|United States|-6|single family| +11895|AAAAAAAAHHOCAAAA|157|6th |ST|Suite 190|Jamestown|Newton County|AR|76867|United States|-6|single family| +11896|AAAAAAAAIHOCAAAA|278|Woodland |Way|Suite 100|Woodland|Hot Springs County|WY|84854|United States|-7|apartment| +11897|AAAAAAAAJHOCAAAA|826|Main 8th|RD|Suite L|Edgewood|Pierce County|ND|50069|United States|-6|single family| +11898|AAAAAAAAKHOCAAAA|369|Hickory |Way|Suite N|Edgewood|Hickman County|KY|40069|United States|-6|condo| +11899|AAAAAAAALHOCAAAA|505|4th |Cir.|Suite 80|Shady Grove|Chester County|PA|12812|United States|-5|apartment| +11900|AAAAAAAAMHOCAAAA|933|Adams Williams|Lane|Suite 60|Arlington|Boulder County|CO|86557|United States|-7|condo| +11901|AAAAAAAANHOCAAAA|601|View Church|Parkway|Suite S|Pleasant Grove|Shawano County|WI|54136|United States|-6|condo| +11902|AAAAAAAAOHOCAAAA|854|Johnson Maple|Avenue|Suite I|Shady Grove|Saline County|KS|62812|United States|-6|condo| +11903|AAAAAAAAPHOCAAAA|636|First |Wy|Suite D|Liberty|Highlands County|FL|33451|United States|-5|apartment| +11904|AAAAAAAAAIOCAAAA|871|Cedar Lee|Ln|Suite G|Riverside|Steuben County|NY|19231|United States|-5|single family| +11905|AAAAAAAABIOCAAAA|401|Third Chestnut|Ave|Suite Q|Mount Zion|Webster County|KY|48054|United States|-5|single family| +11906|AAAAAAAACIOCAAAA|744|Willow Franklin|Ave|Suite 10|Woodville|Arenac County|MI|44289|United States|-5|condo| +11907|AAAAAAAADIOCAAAA|26|College Johnson|Lane|Suite 310|Mount Vernon|Menominee County|MI|48482|United States|-5|single family| +11908|AAAAAAAAEIOCAAAA|841|Cherry Spring|Dr.|Suite 60|Woodville|Putnam County|IN|44289|United States|-5|apartment| +11909|AAAAAAAAFIOCAAAA|687|Center Sunset|Ln|Suite 200|Enterprise|Schley County|GA|31757|United States|-5|single family| +11910|AAAAAAAAGIOCAAAA|173|Park |Way|Suite H|Red Hill|Cannon County|TN|34338|United States|-5|condo| +11911|AAAAAAAAHIOCAAAA|363|Smith |Parkway|Suite 170|Clifton|Gaston County|NC|28014|United States|-5|single family| +11912|AAAAAAAAIIOCAAAA|835|Johnson Railroad|Street|Suite 40|Mount Olive|Ketchikan Gateway Borough|AK|98059|United States|-9|apartment| +11913|AAAAAAAAJIOCAAAA|531|College |ST|Suite O|Waterloo|Dallas County|MO|61675|United States|-6|single family| +11914|AAAAAAAAKIOCAAAA|631|Laurel |Lane|Suite 260|Woodville|Emmons County|ND|54289|United States|-6|condo| +11915|AAAAAAAALIOCAAAA|442|5th |Circle|Suite 280|Oak Ridge|Nowata County|OK|78371|United States|-6|condo| +11916|AAAAAAAAMIOCAAAA||Walnut |Parkway||Sunnyside||KS|61952|||| +11917|AAAAAAAANIOCAAAA|255|Main Hillcrest|Road|Suite O|Unionville|Lunenburg County|VA|21711|United States|-5|single family| +11918|AAAAAAAAOIOCAAAA|981|4th |Court|Suite 120|Woodland|Fayette County|PA|14854|United States|-5|single family| +11919|AAAAAAAAPIOCAAAA||Walnut Lincoln|Avenue|||Bedford County|||United States||| +11920|AAAAAAAAAJOCAAAA|||Pkwy|Suite O|Newport|Walla Walla County|WA||||apartment| +11921|AAAAAAAABJOCAAAA|707|Pine Walnut|Street|Suite 100|Wildwood|Todd County|KY|46871|United States|-5|single family| +11922|AAAAAAAACJOCAAAA|363|1st Ridge|Street|Suite 230|Buena Vista|Camden County|GA|35752|United States|-5|apartment| +11923|AAAAAAAADJOCAAAA|536|South Wilson|Ave|Suite 340|Lincoln|Sanborn County|SD|51289|United States|-7|apartment| +11924|AAAAAAAAEJOCAAAA|915|Seventh |Wy|Suite B|Oakland|Crosby County|TX|79843|United States|-6|condo| +11925|AAAAAAAAFJOCAAAA|123|Hickory Broadway|Road|Suite 490|Union|Pierce County|GA|38721|United States|-5|condo| +11926|AAAAAAAAGJOCAAAA|950|Jefferson Walnut|Ave|Suite 180|Sunnyside|Walton County|FL|31952|United States|-5|condo| +11927|AAAAAAAAHJOCAAAA|710|North Ash|Court|Suite 420|Webb|Massac County|IL|60899|United States|-6|single family| +11928|AAAAAAAAIJOCAAAA|759|Washington 13th|Blvd|Suite R|Pleasant Hill|Doniphan County|KS|63604|United States|-6|condo| +11929|AAAAAAAAJJOCAAAA|676|Park |Lane|Suite K|Farmington|Washington County|NC|29145|United States|-5|condo| +11930|AAAAAAAAKJOCAAAA|343|7th 1st|Court|Suite C|Ridgeway|Washington County|ID|86708|United States|-7|apartment| +11931|AAAAAAAALJOCAAAA|195|East |Avenue|Suite 190|Riverdale|Dolores County|CO|89391|United States|-7|single family| +11932|AAAAAAAAMJOCAAAA|725|Franklin |Blvd|Suite 400|Greenville|Jackson County|SD|51387|United States|-7|apartment| +11933|AAAAAAAANJOCAAAA|738|6th Walnut|Pkwy|Suite 280|Florence|Hardin County|IL|63394|United States|-6|single family| +11934|AAAAAAAAOJOCAAAA|560|Oak Hill|Ave|Suite 110|Fairview|Lawrence County|KY|45709|United States|-5|apartment| +11935|AAAAAAAAPJOCAAAA|830|Hill Hickory|Boulevard|Suite U|New Hope|Essex County|VA|29431|United States|-5|single family| +11936|AAAAAAAAAKOCAAAA|22|Laurel Railroad|RD|Suite V|Riverside|Kings County|CA|99231|United States|-8|apartment| +11937|AAAAAAAABKOCAAAA|504|14th |Pkwy|Suite 130|Salem|Lake County|FL|38048|United States|-5|apartment| +11938|AAAAAAAACKOCAAAA|110|3rd |Way|Suite 0|Oak Hill|San Bernardino County|CA|97838|United States|-8|condo| +11939|AAAAAAAADKOCAAAA|473|Walnut Ash|Circle|Suite C|Fairview|Mississippi County|MO|65709|United States|-6|single family| +11940|AAAAAAAAEKOCAAAA|80|Central Ridge|ST|Suite H|Oak Hill|Torrance County|NM|87838|United States|-7|apartment| +11941|AAAAAAAAFKOCAAAA|800|Fourth |Ct.|Suite 180|Summit|Clark County|OH|40499|United States|-5|apartment| +11942|AAAAAAAAGKOCAAAA|725|3rd Pine|Street|Suite J|Fairview|Christian County|KY|45709|United States|-6|apartment| +11943|AAAAAAAAHKOCAAAA|317|Wilson Forest|Circle|Suite 330|Lincoln|Missaukee County|MI|41289|United States|-5|apartment| +11944|AAAAAAAAIKOCAAAA|677|8th |RD|Suite J|Lincoln|Ogle County|IL|61289|United States|-6|single family| +11945|AAAAAAAAJKOCAAAA|495|13th West|Blvd|Suite 0|Waterloo|Ochiltree County|TX|71675|United States|-6|apartment| +11946|AAAAAAAAKKOCAAAA|||Way|Suite 450|Franklin|Wilbarger County|TX|79101||-6|| +11947|AAAAAAAALKOCAAAA|220|Park 13th|Road|Suite 110|Oak Ridge|Lucas County|OH|48371|United States|-5|single family| +11948|AAAAAAAAMKOCAAAA|33|3rd Second|Dr.|Suite 310|Union|Ingham County|MI|48721|United States|-5|single family| +11949|AAAAAAAANKOCAAAA|729|11th Main|Court|Suite 330|Glenwood|Carbon County|PA|13511|United States|-5|condo| +11950|AAAAAAAAOKOCAAAA|934|Jefferson |Circle|Suite 320|Frogtown|Hamilton County|IL|68784|United States|-6|apartment| +11951|AAAAAAAAPKOCAAAA|699|Washington Fourth|Wy|Suite 40|Stringtown|Perry County|IL|60162|United States|-6|single family| +11952|AAAAAAAAALOCAAAA|598|Cherry |Dr.|Suite 30|Greenwood|Hamilton County|TX|78828|United States|-6|single family| +11953|AAAAAAAABLOCAAAA|825|4th |Ave|Suite G|Oak Ridge|Boise County|ID|88371|United States|-7|single family| +11954|AAAAAAAACLOCAAAA|333|Ash First|ST|Suite 300|Union Hill|Gooding County|ID|87746|United States|-7|condo| +11955|AAAAAAAADLOCAAAA|909|First |Ave|Suite 190|Enterprise|York County|NE|61757|United States|-6|apartment| +11956|AAAAAAAAELOCAAAA|263|Davis |Road|Suite L|Pleasant Grove|New London County|CT|04736|United States|-5|apartment| +11957|AAAAAAAAFLOCAAAA|956|Hill Walnut|Cir.|Suite E|Ellsworth|Garland County|AR|75079|United States|-6|condo| +11958|AAAAAAAAGLOCAAAA|141|Sunset Green|Way|Suite 220|Shiloh|Grundy County|MO|69275|United States|-6|single family| +11959|AAAAAAAAHLOCAAAA|28|View |Cir.|Suite 430|Glenwood|Pike County|OH|43511|United States|-5|condo| +11960|AAAAAAAAILOCAAAA|993|3rd 5th|Ave|Suite U|Edgewater|Vermilion County|IL|60635|United States|-6|condo| +11961|AAAAAAAAJLOCAAAA|33||Pkwy||Lakeview||KS|68579|||condo| +11962|AAAAAAAAKLOCAAAA|587|15th Central|Lane|Suite 490|Arlington|Morgan County|WV|26557|United States|-5|apartment| +11963|AAAAAAAALLOCAAAA|||Court|Suite 370|Pleasant Grove|Wood County|OH|44136||-5|apartment| +11964|AAAAAAAAMLOCAAAA|727|Jackson |ST|Suite 0|Bunker Hill|Roberts County|SD|50150|United States|-7|single family| +11965|AAAAAAAANLOCAAAA|356|10th |Circle|Suite J|Fairfield|Coleman County|TX|76192|United States|-6|single family| +11966|AAAAAAAAOLOCAAAA|971|Hillcrest |||Taft|Nez Perce County|ID|80589|||condo| +11967|AAAAAAAAPLOCAAAA|687|Central |Way|Suite T|Marion|Tishomingo County|MS|50399|United States|-6|single family| +11968|AAAAAAAAAMOCAAAA|455|Oak Ash|ST|Suite H|Midway|Llano County|TX|71904|United States|-6|single family| +11969|AAAAAAAABMOCAAAA|474|North |RD|Suite 380|Maple Grove|Osceola County|IA|58252|United States|-6|single family| +11970|AAAAAAAACMOCAAAA|317|Adams |Drive|Suite 460|Greenfield|Iosco County|MI|45038|United States|-5|single family| +11971|AAAAAAAADMOCAAAA|553|Madison |Cir.|Suite 470|Roy|Pottawatomie County|OK|70744|United States|-6|condo| +11972|AAAAAAAAEMOCAAAA|841|East Main|Cir.|Suite G|Buena Vista|East Baton Rouge Parish|LA|75752|United States|-6|condo| +11973|AAAAAAAAFMOCAAAA|328|Miller Jackson|Blvd|Suite V|Morgantown|Mifflin County|PA|19193|United States|-5|condo| +11974|AAAAAAAAGMOCAAAA|284|Center |Lane|Suite Y|Oakland|Lyon County|IA|59843|United States|-6|single family| +11975|AAAAAAAAHMOCAAAA|986|Main Park|Dr.|Suite U|Clinton|Macon County|IL|68222|United States|-6|apartment| +11976|AAAAAAAAIMOCAAAA|207|2nd View|Street|Suite I|Lincoln|Washington County|CO|81289|United States|-7|single family| +11977|AAAAAAAAJMOCAAAA|942|Railroad Jefferson|Way|Suite V|Glenwood|Banks County|GA|33511|United States|-5|single family| +11978|AAAAAAAAKMOCAAAA|731|Oak |Circle|Suite 280|Newport|Buchanan County|MO|61521|United States|-6|single family| +11979|AAAAAAAALMOCAAAA|563|7th Park|Ln|Suite 340|Farmington|Greene County|GA|39145|United States|-5|condo| +11980|AAAAAAAAMMOCAAAA|115|Central |Road|Suite V|Friendship|Wilcox County|GA|34536|United States|-5|single family| +11981|AAAAAAAANMOCAAAA|882|Washington 13th|Street|Suite 190|San Jose|Mitchell County|NC|28003|United States|-5|apartment| +11982|AAAAAAAAOMOCAAAA|874|Main Center|Ct.|Suite 250|Salem|Saline County|IL|68048|United States|-6|single family| +11983|AAAAAAAAPMOCAAAA||South |RD||Cedar Grove|||70411|United States|-6|| +11984|AAAAAAAAANOCAAAA|564|Eigth Dogwood|RD|Suite I|Woodland|McKenzie County|ND|54854|United States|-6|condo| +11985|AAAAAAAABNOCAAAA|616|Park |Cir.|Suite 340|Buena Vista|Hamilton County|NY|15752|United States|-5|single family| +11986|AAAAAAAACNOCAAAA|933|Pine Meadow|Wy|Suite 260|Maple Grove|Pawnee County|NE|68252|United States|-7|single family| +11987|AAAAAAAADNOCAAAA|666|East |ST|Suite 310|Five Forks|Talladega County|AL|32293|United States|-6|single family| +11988|AAAAAAAAENOCAAAA|621|Lake Fifth|Cir.|Suite 260|Woodville|Jasper County|IA|54289|United States|-6|condo| +11989|AAAAAAAAFNOCAAAA|284|Woodland |Street|Suite 450|Riverview|Washington County|AR|79003|United States|-6|apartment| +11990|AAAAAAAAGNOCAAAA|||Dr.||||||||| +11991|AAAAAAAAHNOCAAAA|325|Valley Main|Avenue|Suite 350|Lenox|Burleigh County|ND|51143|United States|-6|condo| +11992|AAAAAAAAINOCAAAA|170|Lincoln Highland|ST|Suite 330|Oakwood|Smith County|KS|60169|United States|-6|single family| +11993|AAAAAAAAJNOCAAAA|392|East |RD|Suite G|New Hope|Stephens County|OK|79431|United States|-6|single family| +11994|AAAAAAAAKNOCAAAA|219|Hickory 12th|Parkway|Suite H|Highland Park|Boyle County|KY|46534|United States|-6|apartment| +11995|AAAAAAAALNOCAAAA|430|9th |Way|Suite 360|Friendship|Clay County|NE|64536|United States|-6|condo| +11996|AAAAAAAAMNOCAAAA|918|College |Ave|Suite 230|Brownsville|Nez Perce County|ID|89310|United States|-7|single family| +11997|AAAAAAAANNOCAAAA|186|Valley Johnson|Lane|Suite 410|Forest Hills|Franklin County|AR|79237|United States|-6|single family| +11998|AAAAAAAAONOCAAAA|165|Seventh |Ln|Suite 130|Wildwood|Randolph County|GA|36871|United States|-5|apartment| +11999|AAAAAAAAPNOCAAAA|515|Lake |Lane|Suite J|Cedar Grove|Phelps County|NE|60411|United States|-7|condo| +12000|AAAAAAAAAOOCAAAA|558|Railroad Hill|Court|Suite C|Woodland|Wicomico County|MD|24854|United States|-5|single family| +12001|AAAAAAAABOOCAAAA|757|Elm |Wy|Suite 60|Springdale|Fairfax city|VA|28883|United States|-5|single family| +12002|AAAAAAAACOOCAAAA|426|Locust |Court|Suite P|Frenchtown|Palo Pinto County|TX|72629|United States|-6|single family| +12003|AAAAAAAADOOCAAAA|386|Adams |RD|Suite B|Hillcrest|Oconee County|SC|23003|United States|-5|apartment| +12004|AAAAAAAAEOOCAAAA|821|Church |RD|Suite 0|Oak Hill|Sioux County|IA|57838|United States|-6|single family| +12005|AAAAAAAAFOOCAAAA|370|Center Oak|Avenue|Suite N|Belmont|Grant County|KS|60191|United States|-6|condo| +12006|AAAAAAAAGOOCAAAA|907|Lee Miller|Avenue|Suite 280|Elba|Bourbon County|KS|60262|United States|-6|condo| +12007|AAAAAAAAHOOCAAAA|99|Wilson First|Street|Suite 90|Bethel|Bollinger County|MO|65281|United States|-6|condo| +12008|AAAAAAAAIOOCAAAA|737|Adams |Way|Suite 20|Bunker Hill|Lipscomb County|TX|70150|United States|-6|condo| +12009|AAAAAAAAJOOCAAAA|513|East |Ave|Suite W|Green Acres|Johnson County|NE|67683|United States|-7|apartment| +12010|AAAAAAAAKOOCAAAA|769|Main Spring|Court|Suite Y|Frankfort|Greenup County|KY|49681|United States|-6|single family| +12011|AAAAAAAALOOCAAAA|934|Center |Way|Suite 70|Shiloh|Bath County|VA|29275|United States|-5|single family| +12012|AAAAAAAAMOOCAAAA|||Ln|Suite F|Shore Acres|Johnston County||22724|United States||| +12013|AAAAAAAANOOCAAAA|114|Hickory |Lane|Suite 50|Oakwood|Lafayette County|MO|60169|United States|-6|apartment| +12014|AAAAAAAAOOOCAAAA|216|11th Lincoln|Lane|Suite 200|Cherry Valley|Greene County|IA|50854|United States|-6|condo| +12015|AAAAAAAAPOOCAAAA|216|Locust |Way|Suite 40|Lakeside|Sagadahoc County|ME|09532|United States|-5|single family| +12016|AAAAAAAAAPOCAAAA|179|Sunset Ash|Wy|Suite H|Greenwood|Waldo County|ME|09428|United States|-5|single family| +12017|AAAAAAAABPOCAAAA|595|1st |Avenue|Suite N|Harmon|Somervell County|TX|75623|United States|-6|condo| +12018|AAAAAAAACPOCAAAA|877|12th |RD|Suite 10|Clinton|Cottonwood County|MN|58222|United States|-6|condo| +12019|AAAAAAAADPOCAAAA|929|5th East|Ln|Suite 50|Highland|Montgomery County|IA|59454|United States|-6|apartment| +12020|AAAAAAAAEPOCAAAA|914|Mill |Lane|Suite 420|Liberty|Jerome County|ID|83451|United States|-7|single family| +12021|AAAAAAAAFPOCAAAA|415|North |Dr.|Suite 330|Salem|Ellsworth County|KS|68048|United States|-6|apartment| +12022|AAAAAAAAGPOCAAAA|198|8th Highland|Avenue|Suite 70|Lincoln|De Kalb County|IN|41289|United States|-5|apartment| +12023|AAAAAAAAHPOCAAAA|655|4th |Wy|Suite P|Bridgeport|Winnebago County|IL|65817|United States|-6|apartment| +12024|AAAAAAAAIPOCAAAA|870|11th |RD|Suite 430|Clinton|Loudoun County|VA|28222|United States|-5|condo| +12025|AAAAAAAAJPOCAAAA||11th |Road||||GA||||| +12026|AAAAAAAAKPOCAAAA|416|Forest |Drive|Suite F|Spring Valley|Ralls County|MO|66060|United States|-6|condo| +12027|AAAAAAAALPOCAAAA|992|Valley |Pkwy|Suite X|Marion|Monterey County|CA|90399|United States|-8|apartment| +12028|AAAAAAAAMPOCAAAA|191|Jefferson 3rd|Dr.|Suite C|Liberty|Ware County|GA|33451|United States|-5|condo| +12029|AAAAAAAANPOCAAAA|444|Franklin 7th|Cir.|Suite 170|Providence|Costilla County|CO|86614|United States|-7|single family| +12030|AAAAAAAAOPOCAAAA|869|Wilson Locust|Avenue|Suite U|Oakdale|Seward County|NE|69584|United States|-7|apartment| +12031|AAAAAAAAPPOCAAAA|817|Mill |Circle|Suite W|Stringtown|Red River County|TX|70162|United States|-6|single family| +12032|AAAAAAAAAAPCAAAA|48|Fourth |Way|Suite 270|Ashland|Jefferson Parish|LA|74244|United States|-6|apartment| +12033|AAAAAAAABAPCAAAA|485|4th |Road|Suite B|Star|Manassas city|VA|20725|United States|-5|apartment| +12034|AAAAAAAACAPCAAAA|677|Church 2nd|Drive|Suite 130|Indian Village|Craig County|OK|71075|United States|-6|apartment| +12035|AAAAAAAADAPCAAAA|637|Hillcrest |Circle|Suite M|Macedonia|Bates County|MO|61087|United States|-6|single family| +12036|AAAAAAAAEAPCAAAA|22|3rd |Parkway|Suite 320|Midway|Monroe County|AR|71904|United States|-6|apartment| +12037|AAAAAAAAFAPCAAAA|295|1st Fourth|Drive|Suite 330|Florence|Emmet County|IA|53394|United States|-6|condo| +12038|AAAAAAAAGAPCAAAA|323|Jackson Green|RD|Suite Q|Wildwood|Alleghany County|NC|26871|United States|-5|condo| +12039|AAAAAAAAHAPCAAAA|203|Elm |Court|Suite 20|Maple Grove|Grundy County|IL|68252|United States|-6|apartment| +12040|AAAAAAAAIAPCAAAA|926|Forest Smith|Parkway|Suite 360|Spring Hill|Lee County|AR|76787|United States|-6|apartment| +12041|AAAAAAAAJAPCAAAA|332|Park Laurel|Avenue|Suite I|Union|Decatur County|KS|68721|United States|-6|apartment| +12042|AAAAAAAAKAPCAAAA|422|Jackson View|Avenue|Suite 140|Georgetown|Hickman County|TN|37057|United States|-5|apartment| +12043|AAAAAAAALAPCAAAA|675|6th View|RD|Suite W|Oak Ridge|Whatcom County|WA|98371|United States|-8|apartment| +12044|AAAAAAAAMAPCAAAA|538|Sunset Park|ST|Suite 190|Pleasant Hill|Madison County|IN|43604|United States|-5|single family| +12045|AAAAAAAANAPCAAAA|339|South |Ave|Suite 220|Buena Vista|Guadalupe County|TX|75752|United States|-6|single family| +12046|AAAAAAAAOAPCAAAA|171|Davis |Lane|Suite 70|Greenville|Carbon County|WY|81387|United States|-7|apartment| +12047|AAAAAAAAPAPCAAAA|749|5th Highland|ST|Suite 90|Clearwater|Gregg County|TX|79534|United States|-6|condo| +12048|AAAAAAAAABPCAAAA|5|Washington Franklin|Cir.|Suite P|Five Forks|Lee County|AR|72293|United States|-6|condo| +12049|AAAAAAAABBPCAAAA|893|4th |Cir.|Suite V|Farmington|Edwards County|IL|69145|United States|-6|single family| +12050|AAAAAAAACBPCAAAA|897|2nd Church|Boulevard|Suite E|Belmont|Mahoning County|OH|40191|United States|-5|apartment| +12051|AAAAAAAADBPCAAAA|822|Johnson Oak|Pkwy|Suite X|Waterloo|Trinity County|TX|71675|United States|-6|condo| +12052|AAAAAAAAEBPCAAAA|131|Cedar |Boulevard|Suite B|Harmony|Lawrence County|OH|45804|United States|-5|apartment| +12053|AAAAAAAAFBPCAAAA|964|Center Lake|Cir.|Suite V|Oakdale|Oktibbeha County|MS|59584|United States|-6|single family| +12054|AAAAAAAAGBPCAAAA|636|College |Ln|Suite 10|Wildwood|Hansford County|TX|76871|United States|-6|apartment| +12055|AAAAAAAAHBPCAAAA|237|Hickory |RD|Suite 420|Woodlawn|Monterey County|CA|94098|United States|-8|apartment| +12056|AAAAAAAAIBPCAAAA|292|9th 2nd|Court|Suite 420|Williamsville|McIntosh County|GA|38754|United States|-5|apartment| +12057|AAAAAAAAJBPCAAAA|183|10th |Cir.|Suite U|Five Points|Ohio County|KY|46098|United States|-5|apartment| +12058|AAAAAAAAKBPCAAAA|981|Broadway Hickory|Ave|Suite J|Riverside|District of Columbia|DC|29231|United States|-5|single family| +12059|AAAAAAAALBPCAAAA|536|Ridge |Parkway|Suite X|Newtown|Perry County|PA|11749|United States|-5|single family| +12060|AAAAAAAAMBPCAAAA|396|View Cedar|Court|Suite E|Pleasant Grove|Dyer County|TN|34136|United States|-5|single family| +12061|AAAAAAAANBPCAAAA|293|Spruce Franklin|Cir.|Suite R|Summit|Richmond city|VA|20499|United States|-5|single family| +12062|AAAAAAAAOBPCAAAA|440|Sixth 5th|Wy|Suite G|Riverdale|Prince William County|VA|29391|United States|-5|apartment| +12063|AAAAAAAAPBPCAAAA|518|4th |ST|Suite 210|Jerome|Mason County|TX|79920|United States|-6|condo| +12064|AAAAAAAAACPCAAAA||Fifth Fifth||||Lawrence County||60499||-6|single family| +12065|AAAAAAAABCPCAAAA|448|Spring Third|Drive|Suite 10|Enterprise|Wichita County|TX|71757|United States|-6|apartment| +12066|AAAAAAAACCPCAAAA|719|Davis Spring|Ave|Suite 360|Midway|Juab County|UT|81904|United States|-7|single family| +12067|AAAAAAAADCPCAAAA|203|Adams Ash|RD|Suite 320|Antioch|Manitowoc County|WI|58605|United States|-6|single family| +12068|AAAAAAAAECPCAAAA|825|Main |Circle|Suite 190|Mount Zion|Berrien County|GA|38054|United States|-5|apartment| +12069|AAAAAAAAFCPCAAAA|334|Main |Circle|Suite X|Pine Grove|Monroe County|WV|24593|United States|-5|single family| +12070|AAAAAAAAGCPCAAAA|788|Walnut Forest|Lane|Suite 440|Spring Valley|Jefferson County|OH|46060|United States|-5|apartment| +12071|AAAAAAAAHCPCAAAA|92|11th |Cir.|Suite 220|Wilson|Val Verde County|TX|76971|United States|-6|apartment| +12072|AAAAAAAAICPCAAAA|770|Central |RD|Suite 400|Lincoln|Stephens County|GA|31289|United States|-5|apartment| +12073|AAAAAAAAJCPCAAAA|679|River |Wy|Suite H|Ashland|Lenoir County|NC|24244|United States|-5|apartment| +12074|AAAAAAAAKCPCAAAA|149|College 3rd|Court|Suite 350|Centerville|Douglas County|NV|80059|United States|-8|apartment| +12075|AAAAAAAALCPCAAAA|254|Green Williams|Circle|Suite D|Brownsville|Gilchrist County|FL|39310|United States|-5|single family| +12076|AAAAAAAAMCPCAAAA|25|Cedar |Wy|Suite 10|Friendship|Howell County|MO|64536|United States|-6|apartment| +12077|AAAAAAAANCPCAAAA|603|4th |Parkway|Suite 0|Hopewell|Union County|MS|50587|United States|-6|apartment| +12078|AAAAAAAAOCPCAAAA|797|View |Pkwy|Suite 220|Lakewood|Butler County|KS|68877|United States|-6|single family| +12079|AAAAAAAAPCPCAAAA|329|Hill |Boulevard|Suite C|Kingston|Stephens County|GA|34975|United States|-5|condo| +12080|AAAAAAAAADPCAAAA|275|Second |Circle|Suite U|Liberty|Alpena County|MI|43451|United States|-5|condo| +12081|AAAAAAAABDPCAAAA||Center Maple|||Watkins|San Diego County|||||single family| +12082|AAAAAAAACDPCAAAA||||Suite 350|Colonial Heights|Jersey County|IL|||-6|apartment| +12083|AAAAAAAADDPCAAAA|416|Spruce |ST|Suite X|Clinton|Sumter County|AL|38222|United States|-6|apartment| +12084|AAAAAAAAEDPCAAAA|684|Hill |RD|Suite 420|Royal|Lafayette Parish|LA|75819|United States|-6|condo| +12085|AAAAAAAAFDPCAAAA|271|Laurel |Boulevard|Suite 150|Glendale|Butler County|OH|43951|United States|-5|single family| +12086|AAAAAAAAGDPCAAAA|783|Franklin Cherry|Road|Suite 120|Maple Grove|Washington County|OH|48252|United States|-5|apartment| +12087|AAAAAAAAHDPCAAAA|143|Willow 1st|Cir.|Suite 140|Shannon|Searcy County|AR|74120|United States|-6|condo| +12088|AAAAAAAAIDPCAAAA|167|North Cedar|Boulevard|Suite 20|Ashland||KY||||single family| +12089|AAAAAAAAJDPCAAAA|686|Woodland West|Boulevard|Suite N|Oakdale|Choctaw County|OK|79584|United States|-6|condo| +12090|AAAAAAAAKDPCAAAA|89|Willow |Ct.|Suite I|Waterloo|Zapata County|TX|71675|United States|-6|condo| +12091|AAAAAAAALDPCAAAA|808|10th |ST|Suite 340|Kingston|Marin County|CA|94975|United States|-8|single family| +12092|AAAAAAAAMDPCAAAA|350|Elm Meadow|Pkwy|Suite 130|Lakeside|Kidder County|ND|59532|United States|-6|single family| +12093|AAAAAAAANDPCAAAA|773|Mill Sunset|Blvd|Suite B|Sunnyside|Wise County|TX|71952|United States|-6|apartment| +12094|AAAAAAAAODPCAAAA|509|4th Birch|Lane|Suite 180|Brownsville|Montmorency County|MI|49310|United States|-5|condo| +12095|AAAAAAAAPDPCAAAA|643|Sycamore 5th|Circle|Suite T|Riverside|Clark County|MO|69231|United States|-6|apartment| +12096|AAAAAAAAAEPCAAAA|931|Mill |Court|Suite 50|Franklin|Grady County|OK|79101|United States|-6|apartment| +12097|AAAAAAAABEPCAAAA|615|Franklin 8th|Parkway|Suite 180|Kelly|Arlington County|VA|22738|United States|-5|single family| +12098|AAAAAAAACEPCAAAA|979|3rd |Ln|Suite T|Reno|Madison County|OH|40344|United States|-5|condo| +12099|AAAAAAAADEPCAAAA|910|Park Johnson|Road|Suite 280|California|Manistee County|MI|40141|United States|-5|single family| +12100|AAAAAAAAEEPCAAAA|756|6th 13th|Parkway|Suite C|Five Points|Marquette County|MI|46098|United States|-5|apartment| +12101|AAAAAAAAFEPCAAAA|731|Forest |Road|Suite P|Farmington|Benton County|IN|49145|United States|-5|condo| +12102|AAAAAAAAGEPCAAAA|685|College Smith|Circle|Suite 70|Woodlawn|La Salle County|IL|64098|United States|-6|single family| +12103|AAAAAAAAHEPCAAAA|242|Pine |Drive|Suite D|Greenfield|Emporia city|VA|25038|United States|-5|single family| +12104|AAAAAAAAIEPCAAAA|620|Pine |Street|Suite 380|Highland Park|Neshoba County|MS|56534|United States|-6|single family| +12105|AAAAAAAAJEPCAAAA|91|First |Cir.|Suite U|Jerome|Catahoula Parish|LA|79920|United States|-6|single family| +12106|AAAAAAAAKEPCAAAA|673|Oak Washington|Court|Suite E|Fayetteville|Ascension Parish|LA|71732|United States|-6|single family| +12107|AAAAAAAALEPCAAAA|219|Oak Valley|Pkwy|Suite 360|Midway|Clarke County|IA|51904|United States|-6|single family| +12108|AAAAAAAAMEPCAAAA|826|15th Eigth|Ave|Suite 300|Oak Ridge|Kenosha County|WI|58371|United States|-6|apartment| +12109|AAAAAAAANEPCAAAA|121|Elm |Ct.|Suite 410|Glenwood|Goliad County|TX|73511|United States|-6|apartment| +12110|AAAAAAAAOEPCAAAA|280|Park |Wy|Suite T|Stringtown|Steuben County|NY|10162|United States|-5|condo| +12111|AAAAAAAAPEPCAAAA|333|Woodland Miller|Avenue|Suite S|Walnut Grove|Carroll County|MS|57752|United States|-6|apartment| +12112|AAAAAAAAAFPCAAAA|730|Cedar |Dr.|Suite Y|Arlington|Todd County|KY|46557|United States|-5|apartment| +12113|AAAAAAAABFPCAAAA|32|Walnut East|Pkwy|Suite 90|Fairfield|Chouteau County|MT|66192|United States|-7|single family| +12114|AAAAAAAACFPCAAAA|541|Main |Pkwy|Suite 190|Florence|Hendricks County|IN|43394|United States|-5|condo| +12115|AAAAAAAADFPCAAAA|949|Valley |Way|Suite 20|Glendale|Grant County|AR|73951|United States|-6|condo| +12116|AAAAAAAAEFPCAAAA|107|Eigth |Road|Suite K|Clinton|Bear Lake County|ID|88222|United States|-7|condo| +12117|AAAAAAAAFFPCAAAA|801|Washington Dogwood|Ct.|Suite U|Greenfield|Red Willow County|NE|65038|United States|-7|condo| +12118|AAAAAAAAGFPCAAAA|766|North |Parkway|Suite U|Lakewood|Levy County|FL|38877|United States|-5|apartment| +12119|AAAAAAAAHFPCAAAA|614|4th |Ct.|Suite 330|Brownsville|Perry County|PA|19310|United States|-5|apartment| +12120|AAAAAAAAIFPCAAAA|921|Ash |Way|Suite J|Riverview|Robertson County|KY|49003|United States|-5|apartment| +12121|AAAAAAAAJFPCAAAA|750|Pine Main|Ave|Suite 220|Farmington|Mitchell County|TX|79145|United States|-6|apartment| +12122|AAAAAAAAKFPCAAAA|778|Highland |Way|Suite A|Forest Hills|Jackson County|CO|89237|United States|-7|condo| +12123|AAAAAAAALFPCAAAA|6|10th Second|Boulevard|Suite 330|Friendship|Edwards County|KS|64536|United States|-6|single family| +12124|AAAAAAAAMFPCAAAA|19|Main Oak|Way|Suite S|Bunker Hill|Atascosa County|TX|70150|United States|-6|condo| +12125|AAAAAAAANFPCAAAA|284|Twelfth Walnut|Ave|Suite Y|Liberty|Pulaski County|VA||United States|-5|single family| +12126|AAAAAAAAOFPCAAAA|690|Valley 11th|Way|Suite X|Cedar Grove|Jefferson Davis County|MS|50411|United States|-6|condo| +12127|AAAAAAAAPFPCAAAA|660|2nd Walnut|Circle|Suite S|Pleasant Valley|Scott County|IA|52477|United States|-6|condo| +12128|AAAAAAAAAGPCAAAA|440|Lake |Dr.|Suite S|Mount Pleasant|Moffat County|CO|81933|United States|-7|single family| +12129|AAAAAAAABGPCAAAA|73|Lakeview |Dr.|Suite T|Bethel|Bremer County|IA|55281|United States|-6|single family| +12130|AAAAAAAACGPCAAAA|839|Cedar First|ST|Suite P|Woodville|Hendricks County|IN|44289|United States|-5|single family| +12131|AAAAAAAADGPCAAAA|50|View |ST|Suite 170|Jackson|Cloud County|KS|69583|United States|-6|apartment| +12132|AAAAAAAAEGPCAAAA|705|7th Maple|Wy|Suite B|Woodland|Lumpkin County|GA|34854|United States|-5|condo| +12133|AAAAAAAAFGPCAAAA|926|Elm Mill|Boulevard|Suite 270|Oakdale|Parker County|TX|79584|United States|-6|condo| +12134|AAAAAAAAGGPCAAAA|688|Center |Pkwy|Suite 50|Lakeside|Twin Falls County|ID|89532|United States|-7|condo| +12135|AAAAAAAAHGPCAAAA||||Suite M||Gasconade County|||||| +12136|AAAAAAAAIGPCAAAA|920|Spruce Second|Drive|Suite 400|Clinton|Windham County|VT|08822|United States|-5|condo| +12137|AAAAAAAAJGPCAAAA|338|Mill |RD|Suite W|Brownsville|Robertson County|TX|79310|United States|-6|apartment| +12138|AAAAAAAAKGPCAAAA|405|Lake Cherry|Blvd|Suite W|Bunker Hill|Gates County|NC|20150|United States|-5|apartment| +12139|AAAAAAAALGPCAAAA|61|Maple |Boulevard|Suite R|Oak Grove|Cleburne County|AL|38370|United States|-6|single family| +12140|AAAAAAAAMGPCAAAA|637|Smith |Parkway|Suite O|Red Hill|Union County|TN|34338|United States|-6|condo| +12141|AAAAAAAANGPCAAAA|906|Hickory |Ct.|Suite F|Amherst|Collier County|FL|38119|United States|-5|apartment| +12142|AAAAAAAAOGPCAAAA|683|10th |Wy|Suite G|Sulphur Springs|Brooks County|GA|38354|United States|-5|single family| +12143|AAAAAAAAPGPCAAAA|924|Hillcrest |ST|Suite 360|Summit|Sedgwick County|CO|80499|United States|-7|apartment| +12144|AAAAAAAAAHPCAAAA|55|9th 3rd|Road|Suite 340|Plainview|Cooper County|MO|63683|United States|-6|single family| +12145|AAAAAAAABHPCAAAA|499|6th 4th|Lane|Suite 330|Jones|Northwest Arctic Borough|AK|92686|United States|-9|condo| +12146|AAAAAAAACHPCAAAA|716|4th |Ln|Suite K|Glendale|Los Angeles County|CA|93951|United States|-8|apartment| +12147|AAAAAAAADHPCAAAA|676|Woodland 1st|Ln|Suite 80|Royal|Henrico County|VA|25819|United States|-5|condo| +12148|AAAAAAAAEHPCAAAA|931|Willow 4th|Ave|Suite T|Mount Pleasant|Kenedy County|TX|71933|United States|-6|apartment| +12149|AAAAAAAAFHPCAAAA|429|Mill North|Avenue|Suite 400|King|Jeff Davis County|GA|30008|United States|-5|apartment| +12150|AAAAAAAAGHPCAAAA|237||Way|||Cobb County|||||apartment| +12151|AAAAAAAAHHPCAAAA|415|Franklin |ST|Suite H|Valley View|Kimball County|NE|65124|United States|-7|condo| +12152|AAAAAAAAIHPCAAAA|188|2nd Church|Ave|Suite S|Centerville|Abbeville County|SC|20059|United States|-5|apartment| +12153|AAAAAAAAJHPCAAAA|807|Maple Ninth|Road|Suite V|Riverview|Lincoln County|NC|29003|United States|-5|condo| +12154|AAAAAAAAKHPCAAAA|960|College |RD|Suite 490|Midway|Rich County|UT|81904|United States|-7|condo| +12155|AAAAAAAALHPCAAAA|966|Ninth |Avenue|Suite H|Lakeview|Cedar County|MO|68579|United States|-6|apartment| +12156|AAAAAAAAMHPCAAAA|623|View Madison|Drive|Suite 190|Mount Vernon|Jackson County|FL|38482|United States|-5|apartment| +12157|AAAAAAAANHPCAAAA|492|Oak Hill|Parkway|Suite K|Sulphur Springs|Douglas County|WA|98354|United States|-8|condo| +12158|AAAAAAAAOHPCAAAA|72|Adams |Cir.|Suite D|Marion|Heard County|GA|30399|United States|-5|apartment| +12159|AAAAAAAAPHPCAAAA|404|Lee |Cir.|Suite H|Woodlawn|Churchill County|NV|84098|United States|-8|apartment| +12160|AAAAAAAAAIPCAAAA|675|Washington Lincoln|Drive|Suite 40|Griffin|Richland County|SC|21204|United States|-5|apartment| +12161|AAAAAAAABIPCAAAA|560|Madison Sycamore|Ln|Suite 100|White Oak|Wayne County|NC|26668|United States|-5|single family| +12162|AAAAAAAACIPCAAAA|753|Railroad North|Court|Suite U|Buena Vista|Cannon County|TN|35752|United States|-5|single family| +12163|AAAAAAAADIPCAAAA|826|North |Road|Suite 50|Summit|Victoria County|TX|70499|United States|-6|apartment| +12164|AAAAAAAAEIPCAAAA|702|Lakeview Hill|Street|Suite 490|Oakdale|Glades County|FL|39584|United States|-5|single family| +12165|AAAAAAAAFIPCAAAA|583|Main |Wy|Suite Q|Blue Springs|Greene County|OH|44686|United States|-5|condo| +12166|AAAAAAAAGIPCAAAA|890|Madison Hill|Ln|Suite 260|Farmington|Franklin County|FL|39145|United States|-5|single family| +12167|AAAAAAAAHIPCAAAA|136|Spruce 9th|Court|Suite J|Florence|George County|MS|53394|United States|-6|apartment| +12168|AAAAAAAAIIPCAAAA|98|12th 5th|Circle|Suite O|Five Forks|Walker County|AL|32293|United States|-6|single family| +12169|AAAAAAAAJIPCAAAA|488|South |Circle|Suite 140|Sulphur Springs|Coleman County|TX|78354|United States|-6|apartment| +12170|AAAAAAAAKIPCAAAA|963|Wilson South|Circle|Suite B|Springdale|Boundary County|ID|88883|United States|-7|apartment| +12171|AAAAAAAALIPCAAAA|708|Highland Walnut|Cir.|Suite 150|Clifton|Jasper County|IL|68014|United States|-6|single family| +12172|AAAAAAAAMIPCAAAA|184|Maple Seventh|Pkwy|Suite 380|New Hope|Carroll County|MS|59431|United States|-6|condo| +12173|AAAAAAAANIPCAAAA|381|Birch Broadway|Parkway|Suite U|Riverview|Morgan County|IN|49003|United States|-5|single family| +12174|AAAAAAAAOIPCAAAA|406||||Summit|Newton County||40499|United States||| +12175|AAAAAAAAPIPCAAAA|183|Ridge Sunset|Road|Suite B|Marion|Dickinson County|KS|60399|United States|-6|single family| +12176|AAAAAAAAAJPCAAAA|910|Valley |Circle|Suite U|Unionville|Lamar County|MS|51711|United States|-6|apartment| +12177|AAAAAAAABJPCAAAA|779|5th Thirteenth|Court|Suite D|White Oak|Colusa County|CA|96668|United States|-8|single family| +12178|AAAAAAAACJPCAAAA|94|Hickory |Boulevard|Suite 230|Wyoming|Sunflower County|MS|50216|United States|-6|apartment| +12179|AAAAAAAADJPCAAAA|105|Third Valley|Blvd|Suite 110|Bethel|Carlton County|MN|55281|United States|-6|single family| +12180|AAAAAAAAEJPCAAAA|987|5th 6th|Avenue|Suite P|Kingston|Chenango County|NY|14975|United States|-5|single family| +12181|AAAAAAAAFJPCAAAA|264|Valley |Cir.|Suite K|Waterloo|Arthur County|NE|61675|United States|-6|apartment| +12182|AAAAAAAAGJPCAAAA|369|First View|Ave|Suite A|Bethel|Gloucester County|NJ|05881|United States|-5|condo| +12183|AAAAAAAAHJPCAAAA|815|Cherry West|Court|Suite 200|Mount Vernon|Caroline County|VA|28482|United States|-5|condo| +12184|AAAAAAAAIJPCAAAA|327|Ash |ST|Suite 40|Springfield|Tattnall County|GA|39303|United States|-5|apartment| +12185|AAAAAAAAJJPCAAAA|6|Woodland Park|Boulevard|Suite X|Pine Grove|Fairfield County|SC|24593|United States|-5|single family| +12186|AAAAAAAAKJPCAAAA|409|West Church|Boulevard|Suite 470|Lakeview|Mitchell County|IA|58579|United States|-6|apartment| +12187|AAAAAAAALJPCAAAA|668|West Laurel|Pkwy|Suite M|Mount Olive|Quitman County|GA|38059|United States|-5|single family| +12188|AAAAAAAAMJPCAAAA|360|Green |ST|Suite W|Kingston|Wood County|TX|74975|United States|-6|apartment| +12189|AAAAAAAANJPCAAAA|638|Fourth |Ln|Suite X|Walnut Grove|Wilkes County|NC|27752|United States|-5|apartment| +12190|AAAAAAAAOJPCAAAA|97|Seventh |Road|Suite 10|Mountain View|Inyo County|CA|94466|United States|-8|condo| +12191|AAAAAAAAPJPCAAAA|704|9th Washington|Way|Suite M|Mountain View|Arkansas County|AR|74466|United States|-6|condo| +12192|AAAAAAAAAKPCAAAA|105|Maple Tenth|Wy|Suite K|Pleasant Valley|Grady County|OK|72477|United States|-6|single family| +12193|AAAAAAAABKPCAAAA|89|Sycamore South|Ct.|Suite T|Greenfield|Buckingham County|VA|25038|United States|-5|single family| +12194|AAAAAAAACKPCAAAA|443|Madison |Ave|Suite S|Louisville|Webster County|NE|64464|United States|-7|apartment| +12195|AAAAAAAADKPCAAAA|243|Ninth |Circle|Suite 260|Glendale|Carroll County|MD|23951|United States|-5|apartment| +12196|AAAAAAAAEKPCAAAA|90|Wilson |RD|Suite Y|Frogtown|Daviess County|KY|48784|United States|-6|condo| +12197|AAAAAAAAFKPCAAAA|783|1st Central|Dr.|Suite 360|Concord|Wood County|WI|54107|United States|-6|condo| +12198|AAAAAAAAGKPCAAAA|833|Walnut Forest|Parkway|Suite W|Edgewood|Travis County|TX|70069|United States|-6|apartment| +12199|AAAAAAAAHKPCAAAA|715|Maple Cedar|Way|Suite F|Leon|Bastrop County|TX|70913|United States|-6|single family| +12200|AAAAAAAAIKPCAAAA|936|3rd |Ct.|Suite T|Ryan|Dawes County|NE|60525|United States|-6|apartment| +12201|AAAAAAAAJKPCAAAA|525|Wilson |Ln|Suite 460|Lone Pine|Sherman County|TX|77441|United States|-6|condo| +12202|AAAAAAAAKKPCAAAA|133|5th |Circle|Suite F|Ashland|Broome County|NY|14244|United States|-5|apartment| +12203|AAAAAAAALKPCAAAA|592|Thirteenth |Circle|Suite D|Glendale|Athens County|OH|43951|United States|-5|condo| +12204|AAAAAAAAMKPCAAAA|509|5th Washington|Road|Suite W|Pleasant Grove|Robertson County|TN|34136|United States|-6|single family| +12205|AAAAAAAANKPCAAAA|101|Adams Washington|Boulevard|Suite G|Clinton|Rapides Parish|LA|78222|United States|-6|condo| +12206|AAAAAAAAOKPCAAAA|758|Sunset Lake|Circle|Suite Y|Sumner|Rowan County|NC|20519|United States|-5|apartment| +12207|AAAAAAAAPKPCAAAA|557|Pine |Parkway|Suite 410|Smith|Fulton County|PA|17317|United States|-5|apartment| +12208|AAAAAAAAALPCAAAA|390|Pine |RD|Suite 240|Georgetown|Lewis County|MO|67057|United States|-6|condo| +12209|AAAAAAAABLPCAAAA|394|Birch Franklin|Drive|Suite 110|Bridgeport|Moultrie County|IL|65817|United States|-6|condo| +12210|AAAAAAAACLPCAAAA|28|Lake |RD|Suite M|Florence|Iron County|UT|83394|United States|-7|condo| +12211|AAAAAAAADLPCAAAA|381|First |Cir.|Suite 120|Clifton|Dickey County|ND|58014|United States|-6|apartment| +12212|AAAAAAAAELPCAAAA|969|Sixth |Court|Suite 380|Granite|Oceana County|MI|46284|United States|-5|condo| +12213|AAAAAAAAFLPCAAAA|747|7th West|Court|Suite 0|Liberty|Washington County|AL|33451|United States|-6|single family| +12214|AAAAAAAAGLPCAAAA|966|East |Court|Suite 240|Liberty|Rio Blanco County|CO|83451|United States|-7|apartment| +12215|AAAAAAAAHLPCAAAA|104|Fifth |Pkwy|Suite 280|Bear Creek|Surry County|VA|23075|United States|-5|apartment| +12216|AAAAAAAAILPCAAAA|625|Spruce |RD|Suite 40||Lewis County||66871|||| +12217|AAAAAAAAJLPCAAAA|316|Valley Tenth|Drive|Suite 160|Pine Grove|Harmon County|OK|74593|United States|-6|single family| +12218|AAAAAAAAKLPCAAAA|893|Willow East|Drive|Suite 0|Cedar Grove|Montgomery County|NC|20411|United States|-5|single family| +12219|AAAAAAAALLPCAAAA|235|2nd |Boulevard|Suite J|Oakwood|Crawford County|OH|40169|United States|-5|apartment| +12220|AAAAAAAAMLPCAAAA|800|Spring Williams|Road|Suite 50|Pine Grove|Wayne County|UT|84593|United States|-7|single family| +12221|AAAAAAAANLPCAAAA|777|15th |Street|Suite 120|Woodville|Clay County|TX|74289|United States|-6|apartment| +12222|AAAAAAAAOLPCAAAA|443|Woodland Lake|Road|Suite K|Bethel|Robertson County|KY|45281|United States|-5|apartment| +12223|AAAAAAAAPLPCAAAA|963|Chestnut |Blvd|Suite 350|Macedonia|Ochiltree County|TX|71087|United States|-6|single family| +12224|AAAAAAAAAMPCAAAA|823|6th |Drive|Suite I|Oakland|Adair County|MO|69843|United States|-6|single family| +12225|AAAAAAAABMPCAAAA|478|Spruce Ridge|Circle|Suite 430|Bunker Hill|Clear Creek County|CO|80150|United States|-7|single family| +12226|AAAAAAAACMPCAAAA|||Court|Suite 450||||47752||-5|| +12227|AAAAAAAADMPCAAAA|990|Locust |Road|Suite Y|Red Hill|Talbot County|GA|34338|United States|-5|condo| +12228|AAAAAAAAEMPCAAAA|199|Maple |Blvd|Suite 430|Riverdale|Shawnee County|KS|69391|United States|-6|apartment| +12229|AAAAAAAAFMPCAAAA|629|Fourth Madison|Ct.|Suite 360|Plainview|Glades County|FL|33683|United States|-5|condo| +12230|AAAAAAAAGMPCAAAA|281|Franklin |Ave|Suite 290|Cedar Grove|Essex County|NJ|01011|United States|-5|apartment| +12231|AAAAAAAAHMPCAAAA|480|6th Park|RD|Suite 360|Riverview|Lincoln County|WY|89003|United States|-7|single family| +12232|AAAAAAAAIMPCAAAA|423|6th |Avenue|Suite V|Marion|Adair County|IA|50399|United States|-6|single family| +12233|AAAAAAAAJMPCAAAA|119|First |Blvd|Suite 370|Valley View|Houston County|AL|35124|United States|-6|single family| +12234|AAAAAAAAKMPCAAAA|890|West |Ave|Suite J|Stringtown|Muscatine County|IA|50162|United States|-6|apartment| +12235|AAAAAAAALMPCAAAA|355|Hill |Boulevard|Suite H|Riley|Chattooga County|GA|31692|United States|-5|apartment| +12236|AAAAAAAAMMPCAAAA|378|Park Lincoln|Drive|Suite D|Shady Grove|Emanuel County|GA|32812|United States|-5|apartment| +12237|AAAAAAAANMPCAAAA|105|Elm |Blvd|Suite K|Spring Valley|DeBaca County|NM|86060|United States|-7|single family| +12238|AAAAAAAAOMPCAAAA|981|Lake Meadow|Cir.|Suite N|Oakdale|Vernon County|WI|59584|United States|-6|condo| +12239|AAAAAAAAPMPCAAAA|280|Highland Park|Way|Suite 490|Centerville|Liberty County|TX|70059|United States|-6|single family| +12240|AAAAAAAAANPCAAAA|372|Jefferson Adams|Ct.|Suite 130|Jamestown|Boone County|NE|66867|United States|-6|condo| +12241|AAAAAAAABNPCAAAA|627|4th |Wy|Suite V|Five Points|Bland County|VA|26098|United States|-5|apartment| +12242|AAAAAAAACNPCAAAA|853|Park Chestnut|ST|Suite 0|Farmington|Strafford County|NH|09745|United States|-5|apartment| +12243|AAAAAAAADNPCAAAA|118|West 4th|Ct.|Suite 260|Oakdale|Calhoun County|MS|59584|United States|-6|single family| +12244|AAAAAAAAENPCAAAA|718|3rd |Circle|Suite A|Providence|Lawrence County|TN|36614|United States|-6|condo| +12245|AAAAAAAAFNPCAAAA|621|Church |Circle|Suite E|Bridgeport|West Feliciana Parish|LA|75817|United States|-6|condo| +12246|AAAAAAAAGNPCAAAA|501|4th 6th|Way|Suite N|Leesville|Pulaski County|IL|65423|United States|-6|apartment| +12247|AAAAAAAAHNPCAAAA|181|Elm Hillcrest|ST|Suite 160|Oakdale|Richmond city|VA|29584|United States|-5|condo| +12248|AAAAAAAAINPCAAAA|384|Park 1st|ST|Suite C|Highland Park|Mendocino County|CA|96534|United States|-8|apartment| +12249|AAAAAAAAJNPCAAAA|403|Chestnut Maple|Parkway|Suite 250|Newtown|Mecklenburg County|VA|21749|United States|-5|condo| +12250|AAAAAAAAKNPCAAAA|395|Sunset |Court|Suite W|Forest Hills|Houston County|AL|39237|United States|-6|condo| +12251|AAAAAAAALNPCAAAA|247|1st 5th|Cir.|Suite Y|Liberty|Erath County|TX|73451|United States|-6|condo| +12252|AAAAAAAAMNPCAAAA|335|West |Ave|Suite 270|Riverview|Prairie County|AR|79003|United States|-6|apartment| +12253|AAAAAAAANNPCAAAA|135|South |Blvd|Suite 90|Florence|Stutsman County|ND|53394|United States|-6|condo| +12254|AAAAAAAAONPCAAAA|870|6th 5th|Circle|Suite 270|Riverside|Orange County|VT|09831|United States|-5|apartment| +12255|AAAAAAAAPNPCAAAA|182|Park Jackson|RD|Suite I|Elba|Sheridan County|NE|60262|United States|-7|condo| +12256|AAAAAAAAAOPCAAAA|791|12th |Avenue|Suite E||Carbon County|WY|87272|United States||| +12257|AAAAAAAABOPCAAAA|111|Highland |Ct.|Suite V|Springdale|Renville County|MN|58883|United States|-6|apartment| +12258|AAAAAAAACOPCAAAA|381|View Park|Lane|Suite 40|Providence|LaMoure County|ND|56614|United States|-6|condo| +12259|AAAAAAAADOPCAAAA|290|Elevnth |Boulevard|Suite 280|Forest Hills|Washington County|WI|59237|United States|-6|single family| +12260|AAAAAAAAEOPCAAAA|159|Johnson |RD|Suite S|Midway|Newton County|MS|51904|United States|-6|apartment| +12261|AAAAAAAAFOPCAAAA|317|Miller 2nd|Pkwy|Suite B|Mount Olive|Hand County|SD|58059|United States|-7|single family| +12262|AAAAAAAAGOPCAAAA|612|Cherry |Way|Suite 450|Shady Grove|Dona Ana County|NM|82812|United States|-7|condo| +12263|AAAAAAAAHOPCAAAA|930|13th Main|Dr.|Suite L|Clinton|Monroe County|IA|58222|United States|-6|condo| +12264|AAAAAAAAIOPCAAAA|839|5th 7th|Blvd|Suite 90|White Oak|Jefferson County|TX|76668|United States|-6|apartment| +12265|AAAAAAAAJOPCAAAA|320|Washington |Ave|Suite V|Lebanon|Adams County|ID|82898|United States|-7|condo| +12266|AAAAAAAAKOPCAAAA|713|Fourteenth |Boulevard|Suite 490|Sumner|Amherst County|VA|20519|United States|-5|single family| +12267|AAAAAAAALOPCAAAA|547|Pine Adams|Road|Suite L|Edgewood|Decatur County|IN|40069|United States|-5|apartment| +12268|AAAAAAAAMOPCAAAA|492|4th Franklin|Ct.|Suite 330|Waterloo|Knox County|NE|61675|United States|-7|condo| +12269|AAAAAAAANOPCAAAA|825|2nd |Ln|Suite V|Bethel|Holmes County|OH|45281|United States|-5|condo| +12270|AAAAAAAAOOPCAAAA|447|10th View|Lane|Suite 450|Providence|Merrick County|NE|66614|United States|-7|condo| +12271|AAAAAAAAPOPCAAAA|462|Sunset |Drive|Suite P|Hamilton|Manassas city|VA|22808|United States|-5|apartment| +12272|AAAAAAAAAPPCAAAA|390|Willow |Ln|Suite E|Springdale|Carbon County|MT|68883|United States|-7|apartment| +12273|AAAAAAAABPPCAAAA|175|Second Cherry|Ave|Suite K|Farmington|Crawford County|OH|49145|United States|-5|apartment| +12274|AAAAAAAACPPCAAAA|971|Central |Street|Suite Y|Mount Olive|Columbiana County|OH|48059|United States|-5|condo| +12275|AAAAAAAADPPCAAAA|142|Oak 7th|Circle|Suite 330|Bridgeport|Tallapoosa County|AL|35817|United States|-6|single family| +12276|AAAAAAAAEPPCAAAA|197|Chestnut 6th|Road|Suite Y|Newport|Chester County|PA|11521|United States|-5|single family| +12277|AAAAAAAAFPPCAAAA|390|Spruce Maple|Blvd|Suite J|Mount Olive|Covington city|VA|28059|United States|-5|condo| +12278|AAAAAAAAGPPCAAAA|784|Elevnth Jackson|Lane|Suite 190|Brookwood|Jackson County|WV|20965|United States|-5|apartment| +12279|AAAAAAAAHPPCAAAA|175|4th River|Lane|Suite A|Shiloh|Randolph County|WV|29275|United States|-5|single family| +12280|AAAAAAAAIPPCAAAA|4|Sycamore |ST|Suite K|Harmony|Schoharie County|NY|15804|United States|-5|condo| +12281|AAAAAAAAJPPCAAAA|871|Williams Wilson|Road|Suite 490|Jamestown|Alleghany County|VA|26867|United States|-5|single family| +12282|AAAAAAAAKPPCAAAA|589|||Suite 460|Centerville|Baylor County||70059||-6|single family| +12283|AAAAAAAALPPCAAAA|322|Main |Drive|Suite 300|Shiloh|Prairie County|MT|69275|United States|-7|condo| +12284|AAAAAAAAMPPCAAAA|579|Elm Maple|Ct.|Suite 0|Glendale|Wood County|WV|23951|United States|-5|condo| +12285|AAAAAAAANPPCAAAA|715||Ct.|Suite N||||||-5|condo| +12286|AAAAAAAAOPPCAAAA|895|Ridge Oak|Blvd|Suite L|Clifton|Mercer County|IL|68014|United States|-6|condo| +12287|AAAAAAAAPPPCAAAA|26|Sunset |Lane|Suite D|Plainview|Hampton County|SC|23683|United States|-5|condo| +12288|AAAAAAAAAAADAAAA|347|Jackson Ridge|Pkwy|Suite 210|Plainville|Cass County|ND|56115|United States|-6|apartment| +12289|AAAAAAAABAADAAAA||Chestnut Main|||Buena Vista|Sevier County||75752|United States|-6|single family| +12290|AAAAAAAACAADAAAA|605|Dogwood 8th|Drive|Suite W|Glendale|Howard County|IA|53951|United States|-6|single family| +12291|AAAAAAAADAADAAAA|469|Hill Meadow|Boulevard|Suite 240|Burns|Washington County|KS|65272|United States|-6|condo| +12292|AAAAAAAAEAADAAAA|531|Jackson |Street|Suite G|Lakeview|Lake County|MI|48579|United States|-5|apartment| +12293|AAAAAAAAFAADAAAA|540|Church |Cir.|Suite 300|Arlington|Barron County|WI|56557|United States|-6|single family| +12294|AAAAAAAAGAADAAAA|112|Main Johnson|Wy|Suite 260|Buena Vista|Clatsop County|OR|95752|United States|-8|condo| +12295|AAAAAAAAHAADAAAA|149|Spring |RD|Suite 380|Highland Park|Barnwell County|SC|26534|United States|-5|condo| +12296|AAAAAAAAIAADAAAA|||Street|||Sully County|SD|56534|||apartment| +12297|AAAAAAAAJAADAAAA|676|East Walnut|Ave|Suite B|Pleasant Hill|Wyandotte County|KS|63604|United States|-6|condo| +12298|AAAAAAAAKAADAAAA|781|Spring |Wy|Suite X|Five Points|Adams County|WI|56098|United States|-6|single family| +12299|AAAAAAAALAADAAAA|168|Pine 4th|Drive|Suite 410|Highland|Pearl River County|MS|59454|United States|-6|single family| +12300|AAAAAAAAMAADAAAA|673|Lake 5th|Dr.|Suite E|Summit|Crook County|OR|90499|United States|-8|condo| +12301|AAAAAAAANAADAAAA|746|Meadow Elm|Cir.|Suite J|Pleasant Valley|Arlington County|VA|22477|United States|-5|condo| +12302|AAAAAAAAOAADAAAA|98|Laurel |Circle|Suite Q|Oneida|Crook County|WY|84027|United States|-7|condo| +12303|AAAAAAAAPAADAAAA|436||Drive|||Prince William County|VA|||-5|single family| +12304|AAAAAAAAABADAAAA|64|Adams |Cir.|Suite 480|Bunker Hill|Hamblen County|TN|30150|United States|-5|condo| +12305|AAAAAAAABBADAAAA|61|2nd Washington|Ave|Suite F|Plainview|Catahoula Parish|LA|73683|United States|-6|single family| +12306|AAAAAAAACBADAAAA|226|Oak Washington|Blvd||||GA||United States||| +12307|AAAAAAAADBADAAAA|927|Chestnut |Parkway|Suite L|Summit|San Jacinto County|TX|70499|United States|-6|single family| +12308|AAAAAAAAEBADAAAA|362|South View|Ln|Suite F|Oak Ridge|Tillman County|OK|78371|United States|-6|apartment| +12309|AAAAAAAAFBADAAAA|824|5th Maple|Way|Suite O|Highland|Bethel Census Area|AK|99454|United States|-9|apartment| +12310|AAAAAAAAGBADAAAA|655|Jefferson |Wy|Suite P|Florence|Oconee County|GA|33394|United States|-5|apartment| +12311|AAAAAAAAHBADAAAA|972|13th |RD|Suite 290|Highland|Harper County|OK|79454|United States|-6|condo| +12312|AAAAAAAAIBADAAAA|406|Smith |Ave|Suite D|Pleasant Valley|Pike County|MO|62477|United States|-6|condo| +12313|AAAAAAAAJBADAAAA|810|Chestnut Second|Dr.|Suite 310|Hamilton|Florence County|WI|52808|United States|-6|single family| +12314|AAAAAAAAKBADAAAA|359|9th |RD|Suite 300|Waterloo|Marion County|IL|61675|United States|-6|apartment| +12315|AAAAAAAALBADAAAA|617|Oak |Drive|Suite 30|Maple Grove|Rockwall County|TX|78252|United States|-6|single family| +12316|AAAAAAAAMBADAAAA|827|Williams |ST|Suite 20|Oak Hill|Powell County|KY|47838|United States|-5|single family| +12317|AAAAAAAANBADAAAA|381|7th Fourth|Ln|Suite S|Bethel|Greenup County|KY|45281|United States|-6|condo| +12318|AAAAAAAAOBADAAAA|392|Mill Johnson|Blvd|Suite X|Salem|Lake County|TN|38048|United States|-6|condo| +12319|AAAAAAAAPBADAAAA|283|Miller |Cir.|Suite 0|Oakland|Montague County|TX|79843|United States|-6|single family| +12320|AAAAAAAAACADAAAA|474|Park |Parkway|Suite 320|Mountain View|Carbon County|WY|84466|United States|-7|single family| +12321|AAAAAAAABCADAAAA|401|Chestnut 2nd|ST|Suite I|Macedonia|Clayton County|GA|31087|United States|-5|single family| +12322|AAAAAAAACCADAAAA|619|Woodland |Pkwy|Suite W|Union|Kingman County|KS|68721|United States|-6|condo| +12323|AAAAAAAADCADAAAA|168|9th Main|Way|Suite 200|Golden|Clarke County|AL|30411|United States|-6|apartment| +12324|AAAAAAAAECADAAAA|7|3rd |Ct.|Suite A|Jackson|Carroll County|MS|59583|United States|-6|condo| +12325|AAAAAAAAFCADAAAA|779|Tenth |Avenue|Suite R|Woodland|Johnson County|MO|64854|United States|-6|apartment| +12326|AAAAAAAAGCADAAAA|20|Sycamore |Lane|Suite L|Deerfield|Modoc County|CA|99840|United States|-8|apartment| +12327|AAAAAAAAHCADAAAA|251|3rd |Road|Suite K|Sugar Hill|Floyd County|KY|45114|United States|-6|condo| +12328|AAAAAAAAICADAAAA|67|8th |Ln|Suite I|Glendale|||43951|||| +12329|AAAAAAAAJCADAAAA|43|Washington |Parkway|Suite 10|Ashland|McMullen County|TX|74244|United States|-6|apartment| +12330|AAAAAAAAKCADAAAA|41|8th |Boulevard|Suite 80|Liberty|McIntosh County|GA|33451|United States|-5|single family| +12331|AAAAAAAALCADAAAA|20|Oak College|Dr.|Suite L|Providence|Storey County|NV|86614|United States|-8|apartment| +12332|AAAAAAAAMCADAAAA|245|Cedar |Circle|Suite I|Buena Vista|Fremont County|CO|85752|United States|-7|condo| +12333|AAAAAAAANCADAAAA|374|Highland View|Ln|Suite 60|Deerfield|Barton County|KS|69840|United States|-6|apartment| +12334|AAAAAAAAOCADAAAA|12|5th Hill|ST|Suite 400|Crossroads|Lamar County|AL|30534|United States|-6|single family| +12335|AAAAAAAAPCADAAAA|449|Pine |Ave|Suite I|Concord|Dawson County|TX|74107|United States|-6|condo| +12336|AAAAAAAAADADAAAA|787|Franklin Second|Avenue|Suite O|Fairfield|Taylor County|GA|36192|United States|-5|condo| +12337|AAAAAAAABDADAAAA|906|Smith Lee|Ave|Suite A|Harmony|Juniata County|PA|15804|United States|-5|condo| +12338|AAAAAAAACDADAAAA|107|3rd |Avenue|Suite D|New Hope|Whitfield County|GA|39431|United States|-5|single family| +12339|AAAAAAAADDADAAAA|314|Broadway |Cir.|Suite T|Oakland|Beauregard Parish|LA|79843|United States|-6|single family| +12340|AAAAAAAAEDADAAAA|608|Washington |Boulevard|Suite B|Florence|Ketchikan Gateway Borough|AK|93394|United States|-9|condo| +12341|AAAAAAAAFDADAAAA|272|Main Franklin|Cir.|Suite 440|Pleasant Valley|Iowa County|WI|52477|United States|-6|apartment| +12342|AAAAAAAAGDADAAAA|60|Sunset |Parkway|Suite A|Oak Grove|Windsor County|VT|08970|United States|-5|single family| +12343|AAAAAAAAHDADAAAA|372|||Suite P||Jackson County|||||single family| +12344|AAAAAAAAIDADAAAA|329|Forest |Court|Suite 70|Lakeville|Meagher County|MT|68811|United States|-7|condo| +12345|AAAAAAAAJDADAAAA|625|Chestnut 2nd|Pkwy|Suite 290|Springfield|Raleigh County|WV|29303|United States|-5|single family| +12346|AAAAAAAAKDADAAAA|831|Miller Jefferson|Circle|Suite 320|Marion|Morris County|KS|60399|United States|-6|single family| +12347|AAAAAAAALDADAAAA|27|13th |Court|Suite J|Woodville|Leon County|TX|74289|United States|-6|single family| +12348|AAAAAAAAMDADAAAA|900|9th |Way|Suite F|Union Hill|Dickinson County|KS|67746|United States|-6|apartment| +12349|AAAAAAAANDADAAAA|845|Forest Adams|Circle|Suite X|Liberty|Bailey County|TX|73451|United States|-6|apartment| +12350|AAAAAAAAODADAAAA|23|Seventh Smith|Circle|Suite 90|Greenwood|Pierce County|WI|58828|United States|-6|apartment| +12351|AAAAAAAAPDADAAAA|141|Oak |Boulevard|Suite 160|Mount Olive|Vinton County|OH|48059|United States|-5|single family| +12352|AAAAAAAAAEADAAAA|748|Cedar Cherry|Way|Suite T|Yorktown|Ashe County|NC|20732|United States|-5|condo| +12353|AAAAAAAABEADAAAA|479|6th |Ln|Suite P|Woodville|Botetourt County|VA|24289|United States|-5|single family| +12354|AAAAAAAACEADAAAA|897|Central Cedar|Ln|Suite 270|Oakland|Wheeler County|GA|39843|United States|-5|apartment| +12355|AAAAAAAADEADAAAA|319|Maple |Pkwy|Suite R|Midway|Chesapeake city|VA|21904|United States|-5|apartment| +12356|AAAAAAAAEEADAAAA|798|4th |Circle|Suite 490|Woodlawn|Grayson County|VA|24098|United States|-5|single family| +12357|AAAAAAAAFEADAAAA|316|Valley |Ct.|Suite M|White Oak|Lenoir County|NC|26668|United States|-5|condo| +12358|AAAAAAAAGEADAAAA|77|1st |Road||Newtown|||02349||-5|| +12359|AAAAAAAAHEADAAAA|756|15th |Ln|Suite 210|Mount Zion|Morgan County|OH|48054|United States|-5|apartment| +12360|AAAAAAAAIEADAAAA|15|10th |ST|Suite E|Georgetown|Kleberg County|TX|77057|United States|-6|single family| +12361|AAAAAAAAJEADAAAA|194|Lake |Pkwy|Suite F|Oak Hill|Lowndes County|MS|57838|United States|-6|single family| +12362|AAAAAAAAKEADAAAA|140|Davis |Lane|Suite B|Mount Olive|Cibola County|NM|88059|United States|-7|single family| +12363|AAAAAAAALEADAAAA|623|Walnut |Pkwy|Suite O|Friendship|Outagamie County|WI|54536|United States|-6|condo| +12364|AAAAAAAAMEADAAAA|192|Fifth |ST|Suite 0|Glenwood|San Juan County|NM|83511|United States|-7|single family| +12365|AAAAAAAANEADAAAA|458|Lee |Lane|Suite D|Oak Ridge|Caldwell County|TX|78371|United States|-6|apartment| +12366|AAAAAAAAOEADAAAA|373|Madison |Way|Suite D|Woodruff|Livingston County|MO|64174|United States|-6|apartment| +12367|AAAAAAAAPEADAAAA|652|Hickory Elm|ST|Suite 150|Bath|Franklin County|PA|10573|United States|-5|single family| +12368|AAAAAAAAAFADAAAA|54|3rd Oak|Blvd|Suite C|Riverview|Pawnee County|OK|79003|United States|-6|apartment| +12369|AAAAAAAABFADAAAA|169|South |Wy|Suite 400|Centerville|Union County|OR|90059|United States|-8|apartment| +12370|AAAAAAAACFADAAAA|492|Franklin Hickory|Boulevard|Suite Y|Brownsville|Lane County|KS|69310|United States|-6|apartment| +12371|AAAAAAAADFADAAAA|887|Main |Way|Suite F|Deerfield|DeWitt County|TX|79840|United States|-6|apartment| +12372|AAAAAAAAEFADAAAA|141|14th Davis|Pkwy|Suite 10|Mount Vernon|Winona County|MN|58482|United States|-6|single family| +12373|AAAAAAAAFFADAAAA|736|Cedar 14th|Wy|Suite 140|Red Hill|Harrison County|IN|44338|United States|-5|single family| +12374|AAAAAAAAGFADAAAA|555|Mill |Boulevard|Suite U|Woodbury|Putnam County|IN|44489|United States|-5|condo| +12375|AAAAAAAAHFADAAAA|958|Valley 2nd|Pkwy|Suite 230|Maple Grove|Livingston Parish|LA|78252|United States|-6|apartment| +12376|AAAAAAAAIFADAAAA|349|College |Pkwy|Suite O|Shiloh|Walla Walla County|WA|99275|United States|-8|condo| +12377|AAAAAAAAJFADAAAA|276|Wilson |Lane|Suite U|Jackson|Payette County|ID|89583|United States|-7|apartment| +12378|AAAAAAAAKFADAAAA|582|13th |Blvd|Suite V|Perkins|Okaloosa County|FL|31852|United States|-5|single family| +12379|AAAAAAAALFADAAAA|711|Broadway |Blvd|Suite 100|Woodlawn|Union County|NC|24098|United States|-5|condo| +12380|AAAAAAAAMFADAAAA|145|Oak |Lane|Suite H|Washington Heights|King William County|VA|28167|United States|-5|apartment| +12381|AAAAAAAANFADAAAA|318|4th |Way|Suite W|Kingston|Glascock County|GA|34975|United States|-5|apartment| +12382|AAAAAAAAOFADAAAA|52|Locust |Cir.|Suite 190|Mount Olive|Cape Girardeau County|MO|68059|United States|-6|apartment| +12383|AAAAAAAAPFADAAAA|48|Hickory |Circle|Suite 230|Oak Hill|Lenawee County|MI|47838|United States|-5|apartment| +12384|AAAAAAAAAGADAAAA|509|Main Center|Wy|Suite 370|Freeman|Renville County|ND|52297|United States|-6|single family| +12385|AAAAAAAABGADAAAA|417|Smith |Road|Suite C|Clifton|Harrison County|MS|58014|United States|-6|condo| +12386|AAAAAAAACGADAAAA|647|Hill |Cir.|Suite 90|Sunnyside|Warren County|MO|61952|United States|-6|apartment| +12387|AAAAAAAADGADAAAA|242|Fourth 13th|Ln|Suite 30|Jamestown|Lubbock County|TX|76867|United States|-6|condo| +12388|AAAAAAAAEGADAAAA|867|Lincoln |Ln|Suite 40|Oakdale|Clark County|NV|89584|United States|-8|single family| +12389|AAAAAAAAFGADAAAA||||Suite 460|Midway||||||| +12390|AAAAAAAAGGADAAAA|547|3rd |Circle|Suite 130|Centerville|Ashland County|WI|50059|United States|-6|condo| +12391|AAAAAAAAHGADAAAA|877|Oak |Drive|Suite V|Fairfield|Door County|WI|56192|United States|-6|condo| +12392|AAAAAAAAIGADAAAA|377|6th |ST|Suite O|Crossroads|Claiborne Parish|LA|70534|United States|-6|single family| +12393|AAAAAAAAJGADAAAA|501|Locust |Lane|Suite G|Sunnyside|Marshall County|KY|41952|United States|-5|apartment| +12394|AAAAAAAAKGADAAAA|501|Williams North|Ct.|Suite H|Union Hill|Maricopa County|AZ|87746|United States|-7|condo| +12395|AAAAAAAALGADAAAA||Highland Franklin|||Five Forks||OH||||| +12396|AAAAAAAAMGADAAAA|47|Seventh River|Street|Suite O|Riverview|Lenoir County|NC|29003|United States|-5|single family| +12397|AAAAAAAANGADAAAA|825|Washington |Blvd|Suite S|Greenfield|Dorchester County|MD|25038|United States|-5|condo| +12398|AAAAAAAAOGADAAAA|840|5th Railroad|Circle|Suite U|Ashland|Clay County|NC|24244|United States|-5|apartment| +12399|AAAAAAAAPGADAAAA|211|6th Laurel|Ave|Suite M|Elba|Erath County|TX|70262|United States|-6|single family| +12400|AAAAAAAAAHADAAAA|305|Lake |Lane|Suite 130|Buena Vista|Liberty County|TX|75752|United States|-6|single family| +12401|AAAAAAAABHADAAAA|92|8th |Road|Suite 290|Lakewood|Reynolds County|MO|68877|United States|-6|single family| +12402|AAAAAAAACHADAAAA|301|Central Spring|RD|Suite X|Five Points|Archuleta County|CO|86098|United States|-7|condo| +12403|AAAAAAAADHADAAAA|639|Church |ST|Suite M|Clinton|Wibaux County|MT|68222|United States|-7|condo| +12404|AAAAAAAAEHADAAAA|869|Williams |Wy|Suite 260|Oakdale|La Salle County|IL|69584|United States|-6|apartment| +12405|AAAAAAAAFHADAAAA||South Fifth|ST|Suite X|Maple Grove|Avery County|||||condo| +12406|AAAAAAAAGHADAAAA|202|Mill Spring|Pkwy|Suite A|Union|Williams County|OH|48721|United States|-5|condo| +12407|AAAAAAAAHHADAAAA|396|Willow Seventh|Road|Suite 340|Walnut Grove|Greene County|MO|67752|United States|-6|apartment| +12408|AAAAAAAAIHADAAAA|369|Main 1st|Pkwy|Suite O|Shiloh|Isle of Wight County|VA|29275|United States|-5|single family| +12409|AAAAAAAAJHADAAAA|11|Green |Cir.|Suite B|Green Valley|Choctaw County|AL|38337|United States|-6|apartment| +12410|AAAAAAAAKHADAAAA|375|Washington |RD|Suite 10|Spring Valley|Stokes County|NC|26060|United States|-5|apartment| +12411|AAAAAAAALHADAAAA|972|Cedar |Way|Suite 490|Gravel Hill|Benton County|OR|91944|United States|-8|single family| +12412|AAAAAAAAMHADAAAA|949|Valley Cherry|Wy|Suite 30|Quincy|Pitkin County|CO|83868|United States|-7|condo| +12413|AAAAAAAANHADAAAA|838|Fifth |Ln|Suite L|Greenwood|Leon County|TX|78828|United States|-6|condo| +12414|AAAAAAAAOHADAAAA|846|Cherry Miller|Ct.|Suite 340|Lakeside|Gallatin County|MT|69532|United States|-7|condo| +12415|AAAAAAAAPHADAAAA|164|4th |Drive|Suite Y|Oakland|Hidalgo County|NM|89843|United States|-7|condo| +12416|AAAAAAAAAIADAAAA|904|Cedar |Ave|Suite 370|Edgewood|Washington County|GA|30069|United States|-5|apartment| +12417|AAAAAAAABIADAAAA|301|Sycamore |Cir.|Suite P|Wayland|McMinn County|TN|35115|United States|-6|single family| +12418|AAAAAAAACIADAAAA|752|First |Way|Suite S|Riverside|Benton County|WA|99231|United States|-8|single family| +12419|AAAAAAAADIADAAAA|605|Wilson Central|Ln|Suite 30|Oak Ridge|Davie County|NC|28371|United States|-5|condo| +12420|AAAAAAAAEIADAAAA|29|Lincoln Main|Circle|Suite 300|Guthrie|Nelson County|KY|41423|United States|-5|single family| +12421|AAAAAAAAFIADAAAA|973|5th |Drive|Suite 130|Woodlawn|Fulton County|IL|64098|United States|-6|single family| +12422|AAAAAAAAGIADAAAA|54|7th |Pkwy|Suite 140|Cedar Grove|Yuma County|CO|80411|United States|-7|single family| +12423|AAAAAAAAHIADAAAA|823|Green |Pkwy|Suite 330|Spring Valley|Napa County|CA|96060|United States|-8|single family| +12424|AAAAAAAAIIADAAAA|812|Willow Fourth|Lane|Suite 430|Oakland|Crittenden County|AR|79843|United States|-6|condo| +12425|AAAAAAAAJIADAAAA|863|North |Ln|Suite 490|Deerfield|Broome County|NY|19840|United States|-5|apartment| +12426|AAAAAAAAKIADAAAA|211|Lakeview |RD|Suite 480|Jamestown|Tippah County|MS|56867|United States|-6|apartment| +12427|AAAAAAAALIADAAAA|133|Main Spring|Ct.|Suite 280||||69145|||apartment| +12428|AAAAAAAAMIADAAAA|618|8th Main|Ave|Suite 470|Woodville|Floyd County|GA|34289|United States|-5|single family| +12429|AAAAAAAANIADAAAA|811|Hillcrest |ST|Suite D|Providence|Terrell County|TX|76614|United States|-6|apartment| +12430|AAAAAAAAOIADAAAA|852|Main 2nd|Way|Suite P|Midway|Logan County|NE|61904|United States|-7|single family| +12431|AAAAAAAAPIADAAAA|946|Laurel |Boulevard|Suite 100|Fairview|Salem city|VA|25709|United States|-5|apartment| +12432|AAAAAAAAAJADAAAA|289|Cherry |Dr.|Suite 10|White Oak|Presque Isle County|MI|46668|United States|-5|single family| +12433|AAAAAAAABJADAAAA|160|Johnson View|Avenue|Suite M|Salem|Waynesboro city|VA|28048|United States|-5|single family| +12434|AAAAAAAACJADAAAA|921|Cedar |Court|Suite 170|Highland|Clearwater County|ID|89454|United States|-7|apartment| +12435|AAAAAAAADJADAAAA|938|Williams Broadway|Pkwy|Suite 80|Red Hill|Morgan County|IN|44338|United States|-5|apartment| +12436|AAAAAAAAEJADAAAA|523|View |Ave|Suite A|Pleasant Hill|Calloway County|KY|43604|United States|-6|apartment| +12437|AAAAAAAAFJADAAAA|239|East |Blvd|Suite D|Oak Ridge|Washington County|ME|08971|United States|-5|single family| +12438|AAAAAAAAGJADAAAA|47|Laurel Birch|Avenue|Suite 280|White City|Val Verde County|TX|76704|United States|-6|single family| +12439|AAAAAAAAHJADAAAA|793|5th Highland|Pkwy|Suite F|Lakeside|Lawrence County|OH|49532|United States|-5|single family| +12440|AAAAAAAAIJADAAAA|532|Church |Blvd|Suite 180|Garrison|Posey County|IN|48767|United States|-5|single family| +12441|AAAAAAAAJJADAAAA|167|Williams |Ln|Suite 410|Gilmore|Prairie County|AR|75464|United States|-6|single family| +12442|AAAAAAAAKJADAAAA|518|Broadway |Wy|Suite 190|Edgewood|Clarion County|PA|10069|United States|-5|apartment| +12443|AAAAAAAALJADAAAA|122|Valley |Way|Suite K|Enterprise|Mason County|IL|61757|United States|-6|single family| +12444|AAAAAAAAMJADAAAA|513|North |ST|Suite 130|Sunnyside|Ramsey County|ND|51952|United States|-6|condo| +12445|AAAAAAAANJADAAAA|550|12th Birch|RD|Suite E|Newtown|Santa Barbara County|CA|91749|United States|-8|condo| +12446|AAAAAAAAOJADAAAA|145|River Lake|Ave|Suite 100|Kingston|Independence County|AR|74975|United States|-6|apartment| +12447|AAAAAAAAPJADAAAA|319|Hill |Wy|Suite 400|Mountain View|Barber County|KS|64466|United States|-6|apartment| +12448|AAAAAAAAAKADAAAA|694|South |Ln|Suite J|Shelby|Warren County|MS|56575|United States|-6|single family| +12449|AAAAAAAABKADAAAA|100|Center Franklin|Circle|Suite 490|Pleasant Grove|Washington County|RI|04736|United States|-5|single family| +12450|AAAAAAAACKADAAAA|304|Jefferson |Lane|Suite 300|Crossroads|Henry County|IA|50534|United States|-6|apartment| +12451|AAAAAAAADKADAAAA|459|Cedar |Street|Suite L|Mount Vernon|Jefferson County|WI|58482|United States|-6|condo| +12452|AAAAAAAAEKADAAAA|1000|East |Drive|Suite R|Antioch|Wapello County|IA|58605|United States|-6|apartment| +12453|AAAAAAAAFKADAAAA|977|Maple |Way|Suite 150|Walnut Grove|Pickens County|GA|37752|United States|-5|single family| +12454|AAAAAAAAGKADAAAA|159|8th Elm|Boulevard|Suite 60|Wilson|James City County|VA|26971|United States|-5|single family| +12455|AAAAAAAAHKADAAAA|95|Davis 13th|Boulevard|Suite 470|Newtown|Providence County|RI|02349|United States|-5|single family| +12456|AAAAAAAAIKADAAAA|409|Poplar |Ln|Suite 220|Florence|Rich County|UT|83394|United States|-7|apartment| +12457|AAAAAAAAJKADAAAA|319|10th Oak|Wy|Suite K|Plainville|Alcorn County|MS|56115|United States|-6|condo| +12458|AAAAAAAAKKADAAAA|689|14th |Ave|Suite T|Antioch|Montgomery County|VA|28605|United States|-5|condo| +12459|AAAAAAAALKADAAAA|844|Lee |Lane|Suite S|Indian Village|Onslow County|NC|21075|United States|-5|single family| +12460|AAAAAAAAMKADAAAA|472|Oak |ST|Suite 370|Green Acres|Hardin County|KY|47683|United States|-6|condo| +12461|AAAAAAAANKADAAAA|294|Fifth Mill|Ct.|Suite 400|Macedonia|Brewster County|TX|71087|United States|-6|apartment| +12462|AAAAAAAAOKADAAAA|447|Walnut 1st|Circle|Suite F|Providence|Otero County|CO|86614|United States|-7|apartment| +12463|AAAAAAAAPKADAAAA|963|14th |Drive|Suite 40|Wildwood|Columbia County|FL|36871|United States|-5|apartment| +12464|AAAAAAAAALADAAAA|585|Seventh Second|Drive|Suite O|Woodland|Tehama County|CA|94854|United States|-8|condo| +12465|AAAAAAAABLADAAAA|731|Woodland |Parkway|Suite 350|Woodville|Cumberland County|ME|04889|United States|-5|condo| +12466|AAAAAAAACLADAAAA|642|10th |Ln|Suite 360|Chestnut Ridge|Perry County|MO|67334|United States|-6|condo| +12467|AAAAAAAADLADAAAA|811|8th |Wy|Suite X|Riverview|Pennington County|MN|59003|United States|-6|single family| +12468|AAAAAAAAELADAAAA|146|Spring |Court|Suite E|Hurricane|Talbot County|MD|27644|United States|-5|condo| +12469|AAAAAAAAFLADAAAA|30|Central |Circle|Suite 110|Fairfield|Upton County|TX|76192|United States|-6|single family| +12470|AAAAAAAAGLADAAAA|545|Birch East|Road|Suite 80|Marion|Covington County|AL|30399|United States|-6|condo| +12471|AAAAAAAAHLADAAAA|471|Elm |Ave|Suite 30|Crossroads|Kandiyohi County|MN|50534|United States|-6|single family| +12472|AAAAAAAAILADAAAA|||Lane|Suite 380||Marshall County|KS|||-6|condo| +12473|AAAAAAAAJLADAAAA|215|Railroad |RD|Suite 60|Five Points|Camas County|ID|86098|United States|-7|single family| +12474|AAAAAAAAKLADAAAA|425|Poplar |ST|Suite C|Greenville|San Diego County|CA|91387|United States|-8|single family| +12475|AAAAAAAALLADAAAA|391|Main |Circle|Suite 0|Richville|Lincoln County|CO|85945|United States|-7|apartment| +12476|AAAAAAAAMLADAAAA|55|5th |Street|Suite 280|Ashland|Charleston County|SC|24244|United States|-5|condo| +12477|AAAAAAAANLADAAAA|436|Miller |Circle|Suite 90|Lakewood|Hamilton County|IA|58877|United States|-6|condo| +12478|AAAAAAAAOLADAAAA|969|Forest |Court|Suite 420|Riverview|Camas County|ID|89003|United States|-7|condo| +12479|AAAAAAAAPLADAAAA|576|Spruce Center|Circle|Suite T|Shady Grove|Santa Cruz County|CA|92812|United States|-8|single family| +12480|AAAAAAAAAMADAAAA|880|Oak Elm|Cir.|Suite B|Lebanon|Howard County|IA|52898|United States|-6|apartment| +12481|AAAAAAAABMADAAAA|168|View Pine|Cir.|Suite F|Red Hill|Greene County|IL|64338|United States|-6|apartment| +12482|AAAAAAAACMADAAAA|443|Forest Woodland|Cir.|Suite 90|Spring Hill|McPherson County|KS|66787|United States|-6|condo| +12483|AAAAAAAADMADAAAA|893|Cherry Railroad|Wy|Suite O|Jackson|Orange County|VA|29583|United States|-5|condo| +12484|AAAAAAAAEMADAAAA|684|8th Cedar|Wy|Suite L|Wildwood|Jefferson County|MO|66871|United States|-6|single family| +12485|AAAAAAAAFMADAAAA|558|Lakeview |Dr.|Suite M|Oak Grove|Warren County|MS|58370|United States|-6|condo| +12486|AAAAAAAAGMADAAAA|242|Maple |Lane|Suite 450|Greenwood|Clay County|NE|68828|United States|-6|apartment| +12487|AAAAAAAAHMADAAAA|577|Washington |Ave|Suite M|Greenville|Cheboygan County|MI|41387|United States|-5|condo| +12488|AAAAAAAAIMADAAAA|215|Adams |Street|Suite C|Summit|Tioga County|PA|10499|United States|-5|apartment| +12489|AAAAAAAAJMADAAAA|211|Hill 10th|Drive|Suite B|Fairview|Columbiana County|OH|45709|United States|-5|apartment| +12490|AAAAAAAAKMADAAAA|599||Pkwy||Macedonia|||31087|United States|-5|| +12491|AAAAAAAALMADAAAA|989|River 7th|Court|Suite 20|Summit|Perry County|AL|30499|United States|-6|single family| +12492|AAAAAAAAMMADAAAA|254|Pine First|Parkway|Suite F|Wildwood|Accomack County|VA|26871|United States|-5|single family| +12493|AAAAAAAANMADAAAA|861|3rd Jefferson|Pkwy|Suite 130|Kingston|Crook County|OR|94975|United States|-8|condo| +12494|AAAAAAAAOMADAAAA|812|Park |Ct.|Suite U|Oakdale|Kiowa County|OK|79584|United States|-6|single family| +12495|AAAAAAAAPMADAAAA|97|Church Lakeview|Court|Suite L|Clifton|McCone County|MT|68014|United States|-7|apartment| +12496|AAAAAAAAANADAAAA|371|Lakeview 12th|Road|Suite O|Riverview|Wilkes County|GA|39003|United States|-5|single family| +12497|AAAAAAAABNADAAAA|127|Eigth Hillcrest|ST|Suite 210|Kingston|San Juan County|WA|94975|United States|-8|single family| +12498|AAAAAAAACNADAAAA|14|View Green|Drive|Suite 140|Brownsville|Cobb County|GA|39310|United States|-5|condo| +12499|AAAAAAAADNADAAAA|369|Fifth Highland|Court|Suite 480|Mount Zion|Wetzel County|WV|28054|United States|-5|condo| +12500|AAAAAAAAENADAAAA|956|Poplar |Ln|Suite M|Mountain View|Berrien County|GA|34466|United States|-5|single family| +12501|AAAAAAAAFNADAAAA|65|Lakeview |Wy||||||United States||condo| +12502|AAAAAAAAGNADAAAA|1|3rd |Lane|Suite E|Pleasant Grove|Rockingham County|NC|24136|United States|-5|condo| +12503|AAAAAAAAHNADAAAA|80|Franklin |Court|Suite I|Hillcrest|Vanderburgh County|IN|43003|United States|-5|condo| +12504|AAAAAAAAINADAAAA|228|Adams Chestnut|Ln|Suite 40|Cedar Grove|Liberty County|MT|60411|United States|-7|apartment| +12505|AAAAAAAAJNADAAAA|796|Wilson |ST|Suite 380|Enterprise|Lewis and Clark County|MT|61757|United States|-7|apartment| +12506|AAAAAAAAKNADAAAA|62|Ninth Pine|Street|Suite E|Lakeview|Republic County|KS|68579|United States|-6|single family| +12507|AAAAAAAALNADAAAA|156|Madison Lee|Way|Suite 490|Welcome|Pottawattamie County|IA|56386|United States|-6|condo| +12508|AAAAAAAAMNADAAAA|427|Pine |Cir.|Suite C|Shady Grove|Dunklin County|MO|62812|United States|-6|single family| +12509|AAAAAAAANNADAAAA|146|Smith Elm|Avenue|Suite 430|Saint George|Watonwan County|MN|55281|United States|-6|condo| +12510|AAAAAAAAONADAAAA|838|Elm |Cir.|Suite 10|Greenville|Musselshell County|MT|61387|United States|-7|condo| +12511|AAAAAAAAPNADAAAA|353|Franklin |Parkway|Suite 210|Union|Jackson County|GA|38721|United States|-5|condo| +12512|AAAAAAAAAOADAAAA|708|Lee Ash|Boulevard|Suite 0|Unionville|Twin Falls County|ID|81711|United States|-7|condo| +12513|AAAAAAAABOADAAAA|672|2nd |Ct.|Suite 350|Mount Pleasant|Hancock County|OH|41933|United States|-5|single family| +12514|AAAAAAAACOADAAAA|928|Locust |Parkway|Suite 480|Fairfield|Madison County|VA|26192|United States|-5|condo| +12515|AAAAAAAADOADAAAA|223|14th |Pkwy|Suite J|Martinsville|Genesee County|NY|10419|United States|-5|apartment| +12516|AAAAAAAAEOADAAAA|219|College Pine|Pkwy|Suite 80|Wildwood|Oneida County|NY|16871|United States|-5|condo| +12517|AAAAAAAAFOADAAAA|517|Center 7th|Lane|Suite 30|Hamilton|Jackson County|IL|62808|United States|-6|single family| +12518|AAAAAAAAGOADAAAA|239|11th Dogwood|Boulevard|Suite W|Bath|Arkansas County|AR|70573|United States|-6|single family| +12519|AAAAAAAAHOADAAAA|385|Seventh |Pkwy|Suite 280|Fairview|Highlands County|FL|35709|United States|-5|single family| +12520|AAAAAAAAIOADAAAA|379|Railroad |Avenue|Suite Q|Concord|Manatee County|FL|34107|United States|-5|condo| +12521|AAAAAAAAJOADAAAA|48|Jefferson |Drive|Suite 140|Newport|Yavapai County|AZ|81521|United States|-7|apartment| +12522|AAAAAAAAKOADAAAA|794|Smith 6th|Street|Suite Y|Friendship|Clay County|IA|54536|United States|-6|apartment| +12523|AAAAAAAALOADAAAA|675|West |Court|Suite N|Lakewood|Ritchie County|WV|28877|United States|-5|apartment| +12524|AAAAAAAAMOADAAAA|158|Poplar Locust|Boulevard|Suite 390|Sulphur Springs|Appanoose County|IA|58354|United States|-6|single family| +12525|AAAAAAAANOADAAAA|480|North |Dr.|Suite 340|Bunker Hill|Ware County|GA|30150|United States|-5|single family| +12526|AAAAAAAAOOADAAAA|898|Highland |Ln|Suite 330|Farmington|Muhlenberg County|KY|49145|United States|-5|single family| +12527|AAAAAAAAPOADAAAA|916|1st |Road|Suite O|Brownsville|Montgomery County|IL|69310|United States|-6|condo| +12528|AAAAAAAAAPADAAAA|94|Smith |Ct.|Suite U|Sunnyside|Torrance County|NM|81952|United States|-7|condo| +12529|AAAAAAAABPADAAAA|319|Lee 4th|Street|Suite E|Bunker Hill|Cumberland County|NC|20150|United States|-5|condo| +12530|AAAAAAAACPADAAAA|283|Dogwood |Street|Suite 430|Wilson|Crosby County|TX|76971|United States|-6|single family| +12531|AAAAAAAADPADAAAA|130|Highland |Dr.|Suite U|Woodland|Colfax County|NM|84854|United States|-7|single family| +12532|AAAAAAAAEPADAAAA|460|Woodland |RD|Suite I|Glendale|Union County|AR|73951|United States|-6|single family| +12533|AAAAAAAAFPADAAAA|692|3rd 8th|Pkwy|Suite A|Carthage|Barber County|KS|61529|United States|-6|condo| +12534|AAAAAAAAGPADAAAA|570|Birch |Way|Suite 400|Plainview|Northampton County|NC|23683|United States|-5|apartment| +12535|AAAAAAAAHPADAAAA|189||Cir.||||NE||||single family| +12536|AAAAAAAAIPADAAAA|123|Park Miller|Drive|Suite 70|Glendale|DeSoto County|FL|33951|United States|-5|single family| +12537|AAAAAAAAJPADAAAA|402|Williams 15th|Way|Suite P|Jackson|Campbell County|KY|49583|United States|-6|condo| +12538|AAAAAAAAKPADAAAA|679|Park |Street|Suite U|Enterprise|Buffalo County|SD|51757|United States|-6|apartment| +12539|AAAAAAAALPADAAAA|877|Williams |Lane|Suite I|Siloam|Jefferson Davis Parish|LA|78948|United States|-6|single family| +12540|AAAAAAAAMPADAAAA|341|Railroad |ST|Suite J|Woodbury|Culpeper County|VA|24489|United States|-5|single family| +12541|AAAAAAAANPADAAAA|721|River Locust|Ln|Suite B|Marion|Wayne County|WV|20399|United States|-5|condo| +12542|AAAAAAAAOPADAAAA|920|9th |Ct.|Suite A|Union Hill|Franklin County|TX|77746|United States|-6|condo| +12543|AAAAAAAAPPADAAAA|20|5th 3rd|RD|Suite 360|Oak Hill|Mohave County|AZ|87838|United States|-7|condo| +12544|AAAAAAAAAABDAAAA|978|Spring |Cir.|Suite 270|Jamestown|Calcasieu Parish|LA|76867|United States|-6|apartment| +12545|AAAAAAAABABDAAAA|224|South Elm|Circle|Suite F|Mount Olive|Aroostook County|ME|08659|United States|-5|single family| +12546|AAAAAAAACABDAAAA|425|Park 1st|Street|Suite V|Clinton|Cook County|MN|58222|United States|-6|condo| +12547|AAAAAAAADABDAAAA|303|6th Pine|Drive|Suite N|Green Acres|Osborne County|KS|67683|United States|-6|single family| +12548|AAAAAAAAEABDAAAA|339|Central Maple|Ave|Suite G|Edgewood|Summit County|UT|80069|United States|-7|condo| +12549|AAAAAAAAFABDAAAA||||Suite 460||Clay County|IL|||-6|condo| +12550|AAAAAAAAGABDAAAA|585|Woodland |Court|Suite 10|Silver Creek|Union County|SC|24546|United States|-5|single family| +12551|AAAAAAAAHABDAAAA|551|Third |Way|Suite 40|Hillcrest|Amelia County|VA|23003|United States|-5|condo| +12552|AAAAAAAAIABDAAAA|761|Elm |Ct.|Suite 480|Belmont|Cache County|UT|80191|United States|-7|condo| +12553|AAAAAAAAJABDAAAA|928|5th Forest|Way|Suite 30|Oakdale|Menominee County|MI|49584|United States|-5|apartment| +12554|AAAAAAAAKABDAAAA|345|Sunset 5th|Dr.|Suite E|Oakwood|Nassau County|NY|10169|United States|-5|apartment| +12555|AAAAAAAALABDAAAA|278|2nd Pine|Street|Suite 30|Deerfield|Tattnall County|GA|39840|United States|-5|single family| +12556|AAAAAAAAMABDAAAA|672|Third |ST|Suite 270|Crossroads|Ionia County|MI|40534|United States|-5|apartment| +12557|AAAAAAAANABDAAAA|42|Walnut Fifth|Blvd|Suite 370|Shiloh|Henry County|OH|49275|United States|-5|apartment| +12558|AAAAAAAAOABDAAAA|129|Mill |Circle|Suite E|Kingston|Pike County|AR|74975|United States|-6|condo| +12559|AAAAAAAAPABDAAAA|479|Spruce 3rd|RD|Suite 90|Lakeview|Oklahoma County|OK|78579|United States|-6|single family| +12560|AAAAAAAAABBDAAAA|849|Park |Avenue|Suite E|Ashland|Coal County|OK|74244|United States|-6|condo| +12561|AAAAAAAABBBDAAAA|402|Sycamore |Way|Suite B|Clinton|Oscoda County|MI|48222|United States|-5|single family| +12562|AAAAAAAACBBDAAAA|174|Cedar |Street|Suite 390|Ashland|Horry County|SC|24244|United States|-5|apartment| +12563|AAAAAAAADBBDAAAA|516|Maple |Wy|Suite A|Spring Valley|Gratiot County|MI|46060|United States|-5|condo| +12564|AAAAAAAAEBBDAAAA|164|Park Spring|Boulevard|Suite T|Mountain View|Holmes County|FL|34466|United States|-5|apartment| +12565|AAAAAAAAFBBDAAAA|881|First River|Circle|Suite F|Fairview|Ontario County|NY|15709|United States|-5|apartment| +12566|AAAAAAAAGBBDAAAA|900|Cherry East|Parkway|Suite T|Deerfield|Bayfield County|WI|59840|United States|-6|apartment| +12567|AAAAAAAAHBBDAAAA|400|West 9th|Ave|Suite 490|Greenfield|Union County|SC|25038|United States|-5|single family| +12568|AAAAAAAAIBBDAAAA|118|||Suite 400||Robertson County|KY|49583|United States||| +12569|AAAAAAAAJBBDAAAA|441|Birch Cedar|Blvd|Suite 50|Cedar Grove|Grant County|SD|50411|United States|-6|single family| +12570|AAAAAAAAKBBDAAAA|682|Birch |ST|Suite Q|Edgewood|Coweta County|GA|30069|United States|-5|single family| +12571|AAAAAAAALBBDAAAA|519|9th |Blvd|Suite 300|Mountain View|Preston County|WV|24466|United States|-5|single family| +12572|AAAAAAAAMBBDAAAA|284|10th |Wy|Suite W|Springfield|Pondera County|MT|69303|United States|-7|condo| +12573|AAAAAAAANBBDAAAA|90|Cherry |Court|Suite 190|Belleview|Mountrail County|ND|50492|United States|-6|apartment| +12574|AAAAAAAAOBBDAAAA|166|15th |Road|Suite I|Georgetown|Ottawa County|KS|67057|United States|-6|single family| +12575|AAAAAAAAPBBDAAAA|473|Elm Lincoln|Circle|Suite N|Antioch|Borden County|TX|78605|United States|-6|condo| +12576|AAAAAAAAACBDAAAA|||||Bridgeport|Scott County||55817|United States||| +12577|AAAAAAAABCBDAAAA|624|Elevnth |Wy|Suite 20|Unionville|Carroll County|OH|41711|United States|-5|apartment| +12578|AAAAAAAACCBDAAAA|713|Davis Williams|Lane|Suite 330|Providence|Crawford County|MO|66614|United States|-6|condo| +12579|AAAAAAAADCBDAAAA|518|Jackson Railroad|ST|Suite 150|Brownsville|Stephenson County|IL|69310|United States|-6|condo| +12580|AAAAAAAAECBDAAAA|351|Jackson 2nd|Court|Suite I|Lebanon|Grant County|WI|52898|United States|-6|single family| +12581|AAAAAAAAFCBDAAAA|845|Lake Park|Court|Suite P|Mount Vernon|Navajo County|AZ|88482|United States|-7|single family| +12582|AAAAAAAAGCBDAAAA|80|Lake |Way|Suite J|Oakland|Lincoln County|MT|69843|United States|-7|single family| +12583|AAAAAAAAHCBDAAAA|203|3rd Washington|Blvd|Suite 220|Mount Vernon|Sioux County|IA|58482|United States|-6|single family| +12584|AAAAAAAAICBDAAAA|824|7th |Parkway|Suite R|Bridgeport|Union County|OH|45817|United States|-5|single family| +12585|AAAAAAAAJCBDAAAA|488|Spruce |ST|Suite W|Jackson|Reagan County|TX|79583|United States|-6|condo| +12586|AAAAAAAAKCBDAAAA|890|Willow |Parkway|Suite U|Liberty|Clark County|MO|63451|United States|-6|condo| +12587|AAAAAAAALCBDAAAA|409|Walnut |Ave|Suite N|Lakeview|Carteret County|NC|28579|United States|-5|apartment| +12588|AAAAAAAAMCBDAAAA|896|Maple |Drive|Suite P|Plainview|Ector County|TX|73683|United States|-6|apartment| +12589|AAAAAAAANCBDAAAA|800|2nd |Ave|Suite 190|Lakeview|Gloucester County|VA|28579|United States|-5|apartment| +12590|AAAAAAAAOCBDAAAA|657|Railroad |Way|Suite D|White Oak|Macomb County|MI|46668|United States|-5|apartment| +12591|AAAAAAAAPCBDAAAA|207|||Suite K|Red Hill||LA||||| +12592|AAAAAAAAADBDAAAA|510|Dogwood Sycamore|Street|Suite B|Centerville|Evans County|GA|30059|United States|-5|apartment| +12593|AAAAAAAABDBDAAAA|454|Park |Blvd|Suite 470|Mount Vernon|Ziebach County|SD|58482|United States|-6|apartment| +12594|AAAAAAAACDBDAAAA|1000|6th Laurel|Cir.|Suite 330|Oakland|Clackamas County|OR|99843|United States|-8|condo| +12595|AAAAAAAADDBDAAAA|393|Main Park|Pkwy|Suite 110|Elizabeth|Benzie County|MI|42935|United States|-5|single family| +12596|AAAAAAAAEDBDAAAA|128|Dogwood Locust|Lane|Suite 390|Sulphur Springs|Hancock County|ME|08954|United States|-5|single family| +12597|AAAAAAAAFDBDAAAA|479|West |Blvd|Suite K|Avery|Russell County|AL|30194|United States|-6|apartment| +12598|AAAAAAAAGDBDAAAA|452|Lakeview Elm|Wy|Suite 330|Wilson|Banner County|NE|66971|United States|-6|condo| +12599|AAAAAAAAHDBDAAAA|284|Smith Main|Drive|Suite M|Wilson|Branch County|MI|46971|United States|-5|condo| +12600|AAAAAAAAIDBDAAAA|396|Laurel 4th|Boulevard|Suite 20|Oak Hill|Kearny County|KS|67838|United States|-6|single family| +12601|AAAAAAAAJDBDAAAA|911|9th Seventh|Circle|Suite H|Belmont|Columbia County|WI|50191|United States|-6|apartment| +12602|AAAAAAAAKDBDAAAA|823|Fourth |Wy|Suite C|Brownsville|Quay County|NM|89310|United States|-7|single family| +12603|AAAAAAAALDBDAAAA|507|Maple Cedar|ST|Suite 490|Perkins|Pierce County|NE|61852|United States|-7|single family| +12604|AAAAAAAAMDBDAAAA|338|Third Wilson|Ln|Suite 360|Tyrone|Douglas County|WI|51201|United States|-6|single family| +12605|AAAAAAAANDBDAAAA|74|6th South|Dr.|Suite 140|Wildwood|Chesterfield County|SC|26871|United States|-5|single family| +12606|AAAAAAAAODBDAAAA|364|Cedar |Blvd|Suite 320|Hillcrest|Polk County|TX|73003|United States|-6|single family| +12607|AAAAAAAAPDBDAAAA|748|Second Williams|Cir.|Suite 410|Denmark|Cuming County|NE|65576|United States|-6|condo| +12608|AAAAAAAAAEBDAAAA|293|Locust |Blvd|Suite B|Salem|Clark County|OH|48048|United States|-5|single family| +12609|AAAAAAAABEBDAAAA|547|Poplar |Circle|Suite M|Union Hill|Baraga County|MI|47746|United States|-5|condo| +12610|AAAAAAAACEBDAAAA|123|Mill 2nd|Blvd|Suite 490|New Hope|Grundy County|IL|69431|United States|-6|apartment| +12611|AAAAAAAADEBDAAAA|641|Main |Drive|Suite B|Union Hill|Saline County|KS|67746|United States|-6|apartment| +12612|AAAAAAAAEEBDAAAA|691|Maple Forest|Way|Suite 160|Five Forks|Early County|GA|32293|United States|-5|condo| +12613|AAAAAAAAFEBDAAAA|816|Poplar |Drive|Suite 390|Woodlawn|DeKalb County|GA|34098|United States|-5|apartment| +12614|AAAAAAAAGEBDAAAA|880|Church 2nd|Way|Suite C|Buena Vista|Seminole County|OK|75752|United States|-6|condo| +12615|AAAAAAAAHEBDAAAA|467|2nd |Dr.|Suite A|Deerfield|Phillips County|MT|69840|United States|-7|apartment| +12616|AAAAAAAAIEBDAAAA|240|12th Walnut|Boulevard|Suite 80|Shelby|Long County|GA|36575|United States|-5|condo| +12617|AAAAAAAAJEBDAAAA|21|Williams Oak|ST|Suite W|Florence|Clay County|KY|43394|United States|-6|apartment| +12618|AAAAAAAAKEBDAAAA|316|4th |Court|Suite F|Deerfield|Etowah County|AL|39840|United States|-6|condo| +12619|AAAAAAAALEBDAAAA|917|2nd |Street|Suite L|Oakdale|Johnson County|IA|59584|United States|-6|apartment| +12620|AAAAAAAAMEBDAAAA|967|4th |Circle|Suite H|Wildwood|Braxton County|WV|26871|United States|-5|single family| +12621|AAAAAAAANEBDAAAA|479|Johnson |Drive|Suite J|Lincoln|Ouachita County|AR|71289|United States|-6|single family| +12622|AAAAAAAAOEBDAAAA|410|Woodland |Pkwy|Suite O|Unionville|Twin Falls County|ID|81711|United States|-7|apartment| +12623|AAAAAAAAPEBDAAAA|21|Spring |Drive|Suite R|Shady Grove|New Madrid County|MO|62812|United States|-6|apartment| +12624|AAAAAAAAAFBDAAAA|560|4th Ridge|Ave|Suite 370|Clifton|Webster County|KY|48014|United States|-5|condo| +12625|AAAAAAAABFBDAAAA|203|1st 7th|Circle|Suite J|Crossroads|Garfield County|CO|80534|United States|-7|condo| +12626|AAAAAAAACFBDAAAA|962|Mill East|Drive|Suite G|Mount Vernon|Walthall County|MS|58482|United States|-6|apartment| +12627|AAAAAAAADFBDAAAA|847|Meadow Washington|Ave|Suite 440|Stringtown|Wibaux County|MT|60162|United States|-7|condo| +12628|AAAAAAAAEFBDAAAA|321|Ash 2nd|Ln|Suite 460|Whitney|Hunterdon County|NJ|08939|United States|-5|apartment| +12629|AAAAAAAAFFBDAAAA|281|Maple |RD|Suite 360|Hopewell|Rock Island County|IL|60587|United States|-6|single family| +12630|AAAAAAAAGFBDAAAA|595|Third 13th|Parkway|Suite H|Bunker Hill|Ottawa County|OH|40150|United States|-5|apartment| +12631|AAAAAAAAHFBDAAAA|794|Madison |Court|Suite L|Valley View|Box Butte County|NE|65124|United States|-6|condo| +12632|AAAAAAAAIFBDAAAA|432|5th 10th|Ave|Suite T|Oak Hill|Starke County|IN|47838|United States|-5|apartment| +12633|AAAAAAAAJFBDAAAA|484|6th Walnut|Drive|Suite I|Willow|Cass County|ND|56798|United States|-6|single family| +12634|AAAAAAAAKFBDAAAA||||||Starr County|TX|||-6|| +12635|AAAAAAAALFBDAAAA|459|Sunset |ST|Suite B|Plainville|Poquoson city|VA|26115|United States|-5|apartment| +12636|AAAAAAAAMFBDAAAA|383|Valley Sixth|Wy|Suite 210|Oakdale|Oscoda County|MI|49584|United States|-5|single family| +12637|AAAAAAAANFBDAAAA|154|Poplar |Wy|Suite B|Pleasant Grove|Genesee County|NY|14136|United States|-5|apartment| +12638|AAAAAAAAOFBDAAAA|641|6th Second|Wy|Suite H|Antioch|Jackson County|TN|38605|United States|-5|apartment| +12639|AAAAAAAAPFBDAAAA|316|Johnson |RD|Suite 90|Fairview|Monroe County|OH|45709|United States|-5|apartment| +12640|AAAAAAAAAGBDAAAA|754|4th |Ave|Suite 170|Hillcrest|Pittsburg County|OK|73003|United States|-6|single family| +12641|AAAAAAAABGBDAAAA|108|Birch |Ave|Suite U|Sleepy Hollow|Johnston County|NC|23592|United States|-5|apartment| +12642|AAAAAAAACGBDAAAA|84|Elevnth |Court|Suite N|Newtown|Pend Oreille County|WA|91749|United States|-8|single family| +12643|AAAAAAAADGBDAAAA|10|Sunset |Ln|Suite 210|Woodland|Lemhi County|ID|84854|United States|-7|condo| +12644|AAAAAAAAEGBDAAAA|273|Adams Hillcrest|Cir.|Suite 280|Oak Ridge|Pulaski County|IL|68371|United States|-6|single family| +12645|AAAAAAAAFGBDAAAA|293|Highland Maple|Court|Suite S|Millbrook|Massac County|IL|67529|United States|-6|apartment| +12646|AAAAAAAAGGBDAAAA|793|Wilson |Pkwy|Suite 470|Shiloh|Greene County|MS|59275|United States|-6|apartment| +12647|AAAAAAAAHGBDAAAA|940|South |Pkwy|Suite H|Burns|Davidson County|NC|25272|United States|-5|condo| +12648|AAAAAAAAIGBDAAAA|711|Smith Fourth|Parkway|Suite 490|Five Points|Juneau Borough|AK|96098|United States|-9|condo| +12649|AAAAAAAAJGBDAAAA|581|3rd Jackson|Blvd|Suite 230|Forest Hills|Comanche County|OK|79237|United States|-6|single family| +12650|AAAAAAAAKGBDAAAA|397|10th Spring|Circle|Suite 380|Mount Vernon|Hampton County|SC|28482|United States|-5|condo| +12651|AAAAAAAALGBDAAAA|438|Maple First|Way|Suite 120|Hamilton|Bibb County|GA|32808|United States|-5|condo| +12652|AAAAAAAAMGBDAAAA|363|8th |Court|Suite X|Bethel|Mineral County|NV|85281|United States|-8|single family| +12653|AAAAAAAANGBDAAAA|705|Cedar 3rd|Cir.|Suite V|Brownsville|Logan County|NE|69310|United States|-7|apartment| +12654|AAAAAAAAOGBDAAAA|633|Davis 10th|Ave|Suite 210|Franklin|Judith Basin County|MT|69101|United States|-7|apartment| +12655|AAAAAAAAPGBDAAAA|946||||||||United States|-6|| +12656|AAAAAAAAAHBDAAAA|112|4th Woodland|Drive|Suite T|Lakeview|Caldwell County|NC|28579|United States|-5|apartment| +12657|AAAAAAAABHBDAAAA|30|Hickory Spring|Blvd|Suite F|Five Forks|Scott County|IN|42293|United States|-5|condo| +12658|AAAAAAAACHBDAAAA|893|4th Hillcrest|Avenue|Suite 20|Stafford|Lauderdale County|TN|34980|United States|-6|apartment| +12659|AAAAAAAADHBDAAAA|881|River |Cir.|Suite 240|Forest Hills|Clark County|KS|69237|United States|-6|condo| +12660|AAAAAAAAEHBDAAAA|898|First |Street|Suite 130|Lakeside|Anoka County|MN|59532|United States|-6|condo| +12661|AAAAAAAAFHBDAAAA|968|5th |Pkwy|Suite 150|Jackson|Corson County|SD|59583|United States|-6|apartment| +12662|AAAAAAAAGHBDAAAA|348|Hickory 5th|Blvd|Suite 400|Glenwood|Covington city|VA|23511|United States|-5|condo| +12663|AAAAAAAAHHBDAAAA|749|11th |Lane|Suite G|Hillcrest|Burnett County|WI|53003|United States|-6|single family| +12664|AAAAAAAAIHBDAAAA|360|Ridge ||||Chippewa County|WI|51147|||apartment| +12665|AAAAAAAAJHBDAAAA|587|Lincoln |Drive|Suite P||||46697|||single family| +12666|AAAAAAAAKHBDAAAA|738|Second |Parkway|Suite 90|Hillcrest|Allegany County|NY|13003|United States|-5|condo| +12667|AAAAAAAALHBDAAAA|633|Cedar Jackson|RD|Suite 190|Morgantown|Lewis County|WA|99193|United States|-8|apartment| +12668|AAAAAAAAMHBDAAAA|92|9th Oak|Lane|Suite Q|Jackson|Pike County|KY|49583|United States|-5|apartment| +12669|AAAAAAAANHBDAAAA|168|Pine |Boulevard|Suite 50|Riverside|Vermillion County|IN|49231|United States|-5|single family| +12670|AAAAAAAAOHBDAAAA|1000|Fourth Church|Blvd|Suite E|Stringtown|Newton County|MS|50162|United States|-6|single family| +12671|AAAAAAAAPHBDAAAA|303|Lake 1st|Way|Suite 290|Wildwood|Yalobusha County|MS|56871|United States|-6|condo| +12672|AAAAAAAAAIBDAAAA|111|Spring Central|Lane|Suite K|White Oak|Jefferson County|ID|86668|United States|-7|single family| +12673|AAAAAAAABIBDAAAA|628|East |RD|Suite X|Spring Valley|Robeson County|NC|26060|United States|-5|apartment| +12674|AAAAAAAACIBDAAAA|93|North 1st|Wy|Suite C|Greendale|Jim Wells County|TX|79343|United States|-6|apartment| +12675|AAAAAAAADIBDAAAA|430|Main View|Ln|Suite 110|Greenville|Alexandria city|VA|21387|United States|-5|apartment| +12676|AAAAAAAAEIBDAAAA|84|Sunset |Wy|Suite 410|Bunker Hill|Switzerland County|IN|40150|United States|-5|condo| +12677|AAAAAAAAFIBDAAAA|815|West |Cir.|Suite D|Pleasant Valley|DeKalb County|TN|32477|United States|-5|condo| +12678|AAAAAAAAGIBDAAAA|107|Williams Park|Court|Suite 300|Wildwood|Huerfano County|CO|86871|United States|-7|single family| +12679|AAAAAAAAHIBDAAAA|784|Central |Wy|Suite 450|Wildwood|Cass County|TX|76871|United States|-6|apartment| +12680|AAAAAAAAIIBDAAAA|943|3rd Sycamore|||Oak Hill|Franklin County||77838|United States|-6|| +12681|AAAAAAAAJIBDAAAA|184|9th 11th|Avenue|Suite O|Bethel|Jackson County|FL|35281|United States|-5|single family| +12682|AAAAAAAAKIBDAAAA|117|Ridge |Blvd|Suite M|Forest Hills|Clinton County|IL|69237|United States|-6|condo| +12683|AAAAAAAALIBDAAAA|371|Center Main|Lane|Suite 0|Oakwood|Golden Valley County|MT|60169|United States|-7|apartment| +12684|AAAAAAAAMIBDAAAA|788|Lakeview |Avenue|Suite W|Lenox|Newton County|MO|61143|United States|-6|condo| +12685|AAAAAAAANIBDAAAA|19|Sycamore Main|Boulevard|Suite 0|Unionville|Edmunds County|SD|51711|United States|-6|single family| +12686|AAAAAAAAOIBDAAAA|936|First Washington|Drive|Suite 280|Edgewood|Cache County|UT|80069|United States|-7|condo| +12687|AAAAAAAAPIBDAAAA|8|4th Spruce|Boulevard|Suite T|Wildwood|Champaign County|IL|66871|United States|-6|apartment| +12688|AAAAAAAAAJBDAAAA|72|Lakeview Jackson|RD|Suite 90|Lisbon|Passaic County|NJ|05125|United States|-5|single family| +12689|AAAAAAAABJBDAAAA|511|13th Davis|Avenue|Suite 410|Hamilton|Calumet County|WI|52808|United States|-6|condo| +12690|AAAAAAAACJBDAAAA|489|River 1st|Way|Suite R|Farmington|Valdez-Cordova Census Area|AK|99145|United States|-9|condo| +12691|AAAAAAAADJBDAAAA|770|Pine Meadow|Boulevard|Suite X|Sunrise|DeSoto County|MS|51590|United States|-6|apartment| +12692|AAAAAAAAEJBDAAAA|116|14th |Avenue|Suite J|Clinton|Baylor County|TX|78222|United States|-6|single family| +12693|AAAAAAAAFJBDAAAA|609|5th |RD|Suite 450|Buena Vista|Cheshire County|NH|06352|United States|-5|apartment| +12694|AAAAAAAAGJBDAAAA|815|4th North|Dr.|Suite 420|Edgewood|Chippewa County|MN|50069|United States|-6|single family| +12695|AAAAAAAAHJBDAAAA|354|Laurel |ST|Suite Q|Wildwood|Middlesex County|MA|07471|United States|-5|condo| +12696|AAAAAAAAIJBDAAAA|128|Chestnut Hill|Boulevard|Suite U|New Hope|Greenbrier County|WV|29431|United States|-5|apartment| +12697|AAAAAAAAJJBDAAAA|827|Lake Mill|Lane|Suite 350|Philadelphia|Hubbard County|MN|55591|United States|-6|single family| +12698|AAAAAAAAKJBDAAAA|325|7th |Road|Suite U|Liberty|Pima County|AZ|83451|United States|-7|apartment| +12699|AAAAAAAALJBDAAAA|683|11th |Drive|Suite E|Brownsville|Bandera County|TX|79310|United States|-6|single family| +12700|AAAAAAAAMJBDAAAA|373|14th Cherry|Court|Suite C|Four Points|Knox County|TX|71216|United States|-6|single family| +12701|AAAAAAAANJBDAAAA|166|Meadow |Blvd|Suite 220|Union|Okeechobee County|FL|38721|United States|-5|single family| +12702|AAAAAAAAOJBDAAAA|800|Fourth |Drive|Suite R|Newport|Marion County|SC|21521|United States|-5|condo| +12703|AAAAAAAAPJBDAAAA|227|Elm |Parkway|Suite 50|Five Points|Johnson County|NE|66098|United States|-7|condo| +12704|AAAAAAAAAKBDAAAA|379|Forest Laurel|Boulevard|Suite O|Oakdale|Pontotoc County|MS|59584|United States|-6|single family| +12705|AAAAAAAABKBDAAAA|687|Birch |Pkwy|Suite J|Oak Ridge|Sheridan County|WY|88371|United States|-7|apartment| +12706|AAAAAAAACKBDAAAA|420|12th Hill|Court|Suite 340|Clinton|Wheeler County|OR|98222|United States|-8|condo| +12707|AAAAAAAADKBDAAAA|542|Eigth North|RD|Suite S|Delta|Cottle County|TX|74090|United States|-6|apartment| +12708|AAAAAAAAEKBDAAAA|332|Church |Court|Suite 150|Greenville|Jackson County|MS|51387|United States|-6|apartment| +12709|AAAAAAAAFKBDAAAA|340|Locust Sixth|RD|Suite M|Glenwood|Hillsborough County|FL|33511|United States|-5|apartment| +12710|AAAAAAAAGKBDAAAA|913|Willow Woodland|Blvd|Suite 40|New Hope|Webster County|MS|59431|United States|-6|single family| +12711|AAAAAAAAHKBDAAAA|563|Church 9th||Suite 130|Oak Ridge|Marion County|FL||||| +12712|AAAAAAAAIKBDAAAA|127|Lake |Avenue|Suite 320|Sulphur Springs|Jackson County|KS|68354|United States|-6|condo| +12713|AAAAAAAAJKBDAAAA|488|Chestnut |Lane|Suite M|Springfield|Pottawatomie County|KS|69303|United States|-6|apartment| +12714|AAAAAAAAKKBDAAAA|92|Second |Road|Suite C|Bethel|Green County|WI|55281|United States|-6|condo| +12715|AAAAAAAALKBDAAAA|212|Park Central|Pkwy|Suite A|Clinton|Linn County|OR|98222|United States|-8|condo| +12716|AAAAAAAAMKBDAAAA|||||||MT||United States||condo| +12717|AAAAAAAANKBDAAAA|562|Williams |Circle|Suite T|Concord|Camden County|MO|64107|United States|-6|condo| +12718|AAAAAAAAOKBDAAAA|544|5th Park|Drive|Suite W|Kingston|Charlottesville city|VA|24975|United States|-5|condo| +12719|AAAAAAAAPKBDAAAA|186|Lake |Court|Suite 290|Scottsville|Sullivan County|NH|04790|United States|-5|apartment| +12720|AAAAAAAAALBDAAAA|98|Smith |ST|Suite 430|Pleasant Hill|Huerfano County|CO|83604|United States|-7|apartment| +12721|AAAAAAAABLBDAAAA|481|7th |Wy|Suite P|Hillcrest|Pierce County|WI|53003|United States|-6|condo| +12722|AAAAAAAACLBDAAAA|906|Jefferson |Parkway|Suite 410|Kirkland|Nemaha County|NE|67896|United States|-7|single family| +12723|AAAAAAAADLBDAAAA|556|Sycamore View|Ln|Suite 400|Brentwood|Ramsey County|MN|54188|United States|-6|condo| +12724|AAAAAAAAELBDAAAA|906|13th 8th|Wy|Suite C|Bethel|Kiowa County|KS|65281|United States|-6|apartment| +12725|AAAAAAAAFLBDAAAA|833|Johnson |Ct.|Suite S|Lakeview|Lauderdale County|TN|38579|United States|-6|condo| +12726|AAAAAAAAGLBDAAAA|645|14th |Ct.|Suite 340|Sunshine|Screven County|GA|31525|United States|-5|apartment| +12727|AAAAAAAAHLBDAAAA|634|9th |Road|Suite F|Winfield|Linn County|IA|58423|United States|-6|apartment| +12728|AAAAAAAAILBDAAAA|106|Cherry Dogwood|Wy|Suite R|Farmington|Gaston County|NC|29145|United States|-5|condo| +12729|AAAAAAAAJLBDAAAA|590|Valley |Blvd|Suite 50|Greenville|Sharp County|AR|71387|United States|-6|single family| +12730|AAAAAAAAKLBDAAAA|94|View Elm|ST|Suite 470|Fairview|Texas County|OK|75709|United States|-6|single family| +12731|AAAAAAAALLBDAAAA|569|Wilson Adams|Pkwy|Suite 310|Forest Hills|Humboldt County|NV|89237|United States|-8|single family| +12732|AAAAAAAAMLBDAAAA|475|Locust Franklin|Avenue|Suite S|Highland Park|Butler County|NE|66534|United States|-6|condo| +12733|AAAAAAAANLBDAAAA|990|Spring |Parkway|Suite Q|Highland Park|Dare County|NC|26534|United States|-5|single family| +12734|AAAAAAAAOLBDAAAA|111|Pine Cedar|ST|Suite 270|Tremont|Taylor County|KY|49515|United States|-5|single family| +12735|AAAAAAAAPLBDAAAA|780|Main |Ct.|Suite G|Spring Valley|Lauderdale County|TN|36060|United States|-6|apartment| +12736|AAAAAAAAAMBDAAAA|555|9th |Ln|Suite 40|Lincoln|Montrose County|CO|81289|United States|-7|condo| +12737|AAAAAAAABMBDAAAA|913|Forest |Court|Suite Q|Deerfield|Menard County|IL|69840|United States|-6|apartment| +12738|AAAAAAAACMBDAAAA|438|North Main|Court|Suite X|Mount Pleasant|Minnehaha County|SD|51933|United States|-7|single family| +12739|AAAAAAAADMBDAAAA|99|West Smith|Pkwy|Suite H|Friendship|Clark County|WA|94536|United States|-8|apartment| +12740|AAAAAAAAEMBDAAAA|945|Valley |Street|Suite 190|Concord|Crawford County|IN|44107|United States|-5|single family| +12741|AAAAAAAAFMBDAAAA|268|Madison |Ct.|Suite N|Ashland|Schuyler County|IL|64244|United States|-6|apartment| +12742|AAAAAAAAGMBDAAAA|932|Miller |Street|Suite 420|Deerfield|Mountrail County|ND|59840|United States|-6|single family| +12743|AAAAAAAAHMBDAAAA|629|River Birch|RD|Suite I|Mount Olive|Bartow County|GA|38059|United States|-5|condo| +12744|AAAAAAAAIMBDAAAA|746|Lee Park|Boulevard|Suite M|Highland Park|Stafford County|VA|26534|United States|-5|single family| +12745|AAAAAAAAJMBDAAAA|376|Main |Dr.|Suite T|Arlington|Burleson County|TX|76557|United States|-6|condo| +12746|AAAAAAAAKMBDAAAA|52|Elm 12th|Ave|Suite K|Wesley|Lehigh County|PA|11218|United States|-5|condo| +12747|AAAAAAAALMBDAAAA|513|Willow Cherry|Avenue|Suite T|Wildwood|Prince William County|VA|26871|United States|-5|condo| +12748|AAAAAAAAMMBDAAAA|336|Spring View|Court|Suite 490|Oak Ridge|Douglas County|KS|68371|United States|-6|condo| +12749|AAAAAAAANMBDAAAA|233|Walnut |Ave|Suite 360|Forestville|Calhoun County|IA|53027|United States|-6|apartment| +12750|AAAAAAAAOMBDAAAA|255|3rd |Ave|Suite 80|New Hope|Alachua County|FL|39431|United States|-5|single family| +12751|AAAAAAAAPMBDAAAA|873|Davis |Cir.|Suite Q|Jamestown|San Patricio County|TX|76867|United States|-6|single family| +12752|AAAAAAAAANBDAAAA|865|5th Mill|Road|Suite C|Providence|Duplin County|NC|26614|United States|-5|single family| +12753|AAAAAAAABNBDAAAA|764|Lakeview Jackson|Ct.|Suite N|Unionville|Santa Barbara County|CA|91711|United States|-8|apartment| +12754|AAAAAAAACNBDAAAA|854|Jefferson |RD|Suite L|Five Points|Alexander County|IL|66098|United States|-6|single family| +12755|AAAAAAAADNBDAAAA|273|Hillcrest Fourth|Wy|Suite 250|Oak Grove|Boone County|WV|28370|United States|-5|single family| +12756|AAAAAAAAENBDAAAA|986|Lee |RD|Suite 410|Derby|Real County|TX|77702|United States|-6|single family| +12757|AAAAAAAAFNBDAAAA|875|8th Park|Ln|Suite 460|Wilson|Union County|OR|96971|United States|-8|apartment| +12758|AAAAAAAAGNBDAAAA|247|6th |Ln|Suite 390|Woodland|Sanilac County|MI|44854|United States|-5|apartment| +12759|AAAAAAAAHNBDAAAA|459|Railroad |Wy|Suite 210|Providence|Scott County|MN|56614|United States|-6|single family| +12760|AAAAAAAAINBDAAAA|15|Cedar Lee|RD|Suite J|Jamestown|Ramsey County|MN|56867|United States|-6|apartment| +12761|AAAAAAAAJNBDAAAA|590|Lakeview Laurel|Dr.|Suite 330|Florence|Jennings County|IN|43394|United States|-5|single family| +12762|AAAAAAAAKNBDAAAA|617|Elm |Parkway|Suite Q|Fairview|Crosby County|TX|75709|United States|-6|apartment| +12763|AAAAAAAALNBDAAAA|733|Spring |Avenue|Suite R|Springfield|Wayne County|UT|89303|United States|-7|condo| +12764|AAAAAAAAMNBDAAAA|131|Ninth |Ave|Suite 210|Salem|Brooks County|GA|38048|United States|-5|apartment| +12765|AAAAAAAANNBDAAAA|615|6th Cherry|Street|Suite 30|Bridgeport|New Madrid County|MO|65817|United States|-6|apartment| +12766|AAAAAAAAONBDAAAA|742|Ridge |Cir.|Suite 80|Plainview|Bolivar County|MS|53683|United States|-6|apartment| +12767|AAAAAAAAPNBDAAAA|499|West Cedar|Blvd|Suite 390|Riverside|Skamania County|WA|99231|United States|-8|single family| +12768|AAAAAAAAAOBDAAAA|395|Hill |Blvd|Suite N|Highland Park|Vigo County|IN|46534|United States|-5|single family| +12769|AAAAAAAABOBDAAAA|485|Sunset |Court|Suite 50|Newport|Love County|OK|71521|United States|-6|apartment| +12770|AAAAAAAACOBDAAAA|397|Walnut |Pkwy|Suite N|Shady Grove|Alger County|MI|42812|United States|-5|apartment| +12771|AAAAAAAADOBDAAAA|841|Woodland |Blvd|Suite S|Williamsville|Isabella County|MI|48754|United States|-5|single family| +12772|AAAAAAAAEOBDAAAA|803|3rd 2nd|ST|Suite T|Richardson|Potter County|PA|17687|United States|-5|single family| +12773|AAAAAAAAFOBDAAAA|964|Walnut |Avenue|Suite 370|Walnut Grove|Marion County|IN|47752|United States|-5|single family| +12774|AAAAAAAAGOBDAAAA|198|Willow |Pkwy|Suite 180|Harmon|Dolores County|CO|85623|United States|-7|condo| +12775|AAAAAAAAHOBDAAAA|471|5th Madison|Drive|Suite 50|Liberty|McCone County|MT|63451|United States|-7|single family| +12776|AAAAAAAAIOBDAAAA|758|Spruce |Street|Suite E|Antioch|Black Hawk County|IA|58605|United States|-6|apartment| +12777|AAAAAAAAJOBDAAAA|197|Ridge Park|Avenue|Suite D|Farmington|Antrim County|MI|49145|United States|-5|apartment| +12778|AAAAAAAAKOBDAAAA|733|4th Park|ST|Suite 250|Forest Hills|Calhoun County|AR|79237|United States|-6|apartment| +12779|AAAAAAAALOBDAAAA|789|Sycamore Lakeview|Ln|Suite 40|Oak Grove|Davis County|IA|58370|United States|-6|single family| +12780|AAAAAAAAMOBDAAAA|183|Oak Jefferson|Ave|Suite E|Greenville|Routt County|CO|81387|United States|-7|single family| +12781|AAAAAAAANOBDAAAA|920|Madison |Circle|Suite 410|Plainview|Deaf Smith County|TX|73683|United States|-6|apartment| +12782|AAAAAAAAOOBDAAAA|980|Main 4th|Drive|Suite V|Greenwood|Washington County|WI|58828|United States|-6|apartment| +12783|AAAAAAAAPOBDAAAA||4th |Circle|Suite S|Plainview|Washington County|KS|63683|United States|-6|condo| +12784|AAAAAAAAAPBDAAAA|138|8th Oak|Court|Suite 160|Brownsville|Emmet County|IA|59310|United States|-6|condo| +12785|AAAAAAAABPBDAAAA|971|Pine Johnson|ST|Suite 120|Liberty|Alcorn County|MS|53451|United States|-6|condo| +12786|AAAAAAAACPBDAAAA|671|North Second|Dr.|Suite 210|Highland|Armstrong County|PA|19454|United States|-5|apartment| +12787|AAAAAAAADPBDAAAA|30|Cedar First|Parkway|Suite 110|Unionville|Lauderdale County|MS|51711|United States|-6|apartment| +12788|AAAAAAAAEPBDAAAA|516|Johnson Ash|Way|Suite 200|Deerfield|Buffalo County|SD|59840|United States|-6|condo| +12789|AAAAAAAAFPBDAAAA|258|6th |Court|Suite 410|Clifton|Aleutians East Borough|AK|98014|United States|-9|apartment| +12790|AAAAAAAAGPBDAAAA|748|4th |Boulevard|Suite 370|Bridgeport|Overton County|TN|35817|United States|-6|apartment| +12791|AAAAAAAAHPBDAAAA|808|Pine Washington|Blvd|Suite 340|Pine Grove|Lexington city|VA|24593|United States|-5|single family| +12792|AAAAAAAAIPBDAAAA|13|10th |Blvd|Suite 490|Fairview|Shasta County|CA|95709|United States|-8|single family| +12793|AAAAAAAAJPBDAAAA|879|Third |Avenue|Suite 200|Lakeside|Cassia County|ID|89532|United States|-7|single family| +12794|AAAAAAAAKPBDAAAA|115|Lakeview East|Pkwy|Suite I|Greenfield|Lyon County|NV|85038|United States|-8|apartment| +12795|AAAAAAAALPBDAAAA|497|Thirteenth 2nd|Drive|Suite 70|Georgetown|Lamar County|GA|37057|United States|-5|apartment| +12796|AAAAAAAAMPBDAAAA|230|1st |Blvd|Suite L|Fairview|Sampson County|NC|25709|United States|-5|apartment| +12797|AAAAAAAANPBDAAAA|490|West Meadow|Court|Suite P|Green Acres|Lackawanna County|PA|17683|United States|-5|apartment| +12798|AAAAAAAAOPBDAAAA|747|Maple |Boulevard|Suite 160|Mount Pleasant|Union County|IN|41933|United States|-5|condo| +12799|AAAAAAAAPPBDAAAA|841|Lincoln |Dr.|Suite 150|Fairbanks|Donley County|TX|76653|United States|-6|condo| +12800|AAAAAAAAAACDAAAA|980|Center Maple|Ct.|Suite A|Forest Hills|Stewart County|GA|39237|United States|-5|condo| +12801|AAAAAAAABACDAAAA|22|Elm Railroad|Blvd|Suite 160|Franklin|Oktibbeha County|MS|59101|United States|-6|single family| +12802|AAAAAAAACACDAAAA|539|4th Sycamore|Drive|Suite G|Lakeside|Real County|TX|79532|United States|-6|apartment| +12803|AAAAAAAADACDAAAA||Central |Dr.||||PA||United States|-5|condo| +12804|AAAAAAAAEACDAAAA|289|West Park|Ave|Suite R|White Oak|Jessamine County|KY|46668|United States|-6|single family| +12805|AAAAAAAAFACDAAAA|862|Forest |Avenue|Suite 480|Woodville|Susquehanna County|PA|14289|United States|-5|single family| +12806|AAAAAAAAGACDAAAA|763|Johnson |Boulevard|Suite 250|Glenwood|Lincoln County|AR|73511|United States|-6|single family| +12807|AAAAAAAAHACDAAAA|727|14th |RD|Suite 230|Woodville|Morton County|KS|64289|United States|-6|apartment| +12808|AAAAAAAAIACDAAAA|759|Seventh |Ln|Suite 0|Mason|Jackson County|MO|61185|United States|-6|condo| +12809|AAAAAAAAJACDAAAA||Cedar Pine||||||20499||-5|apartment| +12810|AAAAAAAAKACDAAAA|528|North Highland|Drive|Suite J|Antioch|Lyon County|IA|58605|United States|-6|condo| +12811|AAAAAAAALACDAAAA|873|Broadway |Cir.|Suite 300|Wilson|Susquehanna County|PA|16971|United States|-5|single family| +12812|AAAAAAAAMACDAAAA|56|Oak Eigth|Cir.|Suite D|Centerville|Nuckolls County|NE|60059|United States|-7|single family| +12813|AAAAAAAANACDAAAA|653|5th |Boulevard|Suite Y|Crossroads|Buchanan County|VA|20534|United States|-5|condo| +12814|AAAAAAAAOACDAAAA|492|Adams Smith|Ave|Suite 490|Walnut Grove|Fairfax County|VA|27752|United States|-5|single family| +12815|AAAAAAAAPACDAAAA|472|Oak Broadway|Pkwy|Suite O|Lincoln|Seminole County|GA|31289|United States|-5|condo| +12816|AAAAAAAAABCDAAAA|740|Mill Forest|Wy|Suite G|Salem|Henry County|IN|48048|United States|-5|single family| +12817|AAAAAAAABBCDAAAA|456|Willow |Court|Suite 220|Lewis|Somerset County|MD|27066|United States|-5|apartment| +12818|AAAAAAAACBCDAAAA|324|Pine |Wy|Suite 170|Warwick|McKinley County|NM|81398|United States|-7|single family| +12819|AAAAAAAADBCDAAAA|773|Hill |Drive|Suite E|Salem|Treasure County|MT|68048|United States|-7|apartment| +12820|AAAAAAAAEBCDAAAA|69|9th |ST|Suite 440|Leland|Carbon County|WY|89452|United States|-7|single family| +12821|AAAAAAAAFBCDAAAA|874|Cedar |RD|Suite 420|Parkwood|Whiteside County|IL|61669|United States|-6|single family| +12822|AAAAAAAAGBCDAAAA|229|Pine 1st|Ave|Suite F|Clifton|Gogebic County|MI|48014|United States|-5|apartment| +12823|AAAAAAAAHBCDAAAA|967|View ||||Greene County||||-5|single family| +12824|AAAAAAAAIBCDAAAA|225|2nd Third|Cir.|Suite G|Oak Grove|Duval County|TX|78370|United States|-6|apartment| +12825|AAAAAAAAJBCDAAAA|64|Fourth Tenth|Avenue|Suite H|Riverside|Mariposa County|CA|99231|United States|-8|single family| +12826|AAAAAAAAKBCDAAAA|316|14th Main|||Newtown||||United States|-5|| +12827|AAAAAAAALBCDAAAA|917|Sycamore |Road|||Holt County||69003|United States||| +12828|AAAAAAAAMBCDAAAA|648|Walnut |Blvd|Suite W|Springdale|Foster County|ND|58883|United States|-6|single family| +12829|AAAAAAAANBCDAAAA|824|15th |Way|Suite O|Greenfield|Grant County|OK|75038|United States|-6|apartment| +12830|AAAAAAAAOBCDAAAA|60|Spruce Pine|Avenue|Suite R|Mount Olive|Cattaraugus County|NY|18059|United States|-5|single family| +12831|AAAAAAAAPBCDAAAA|219|Spring Poplar|RD|Suite C|Phoenix|Blaine County|NE|62276|United States|-6|condo| +12832|AAAAAAAAACCDAAAA|860|Hill |Parkway|Suite 20|Woodville|Hendry County|FL|34289|United States|-5|single family| +12833|AAAAAAAABCCDAAAA|819|12th 7th|ST|Suite W|Lebanon|Marion County|IL|62898|United States|-6|condo| +12834|AAAAAAAACCCDAAAA|3|11th |Ave|Suite O|Sunnyside|Berrien County|MI|41952|United States|-5|single family| +12835|AAAAAAAADCCDAAAA|255|Pine |Ct.|Suite 380|Florence|Jefferson County|MT|63394|United States|-7|apartment| +12836|AAAAAAAAECCDAAAA|651|Laurel |Pkwy|Suite B|Rockwood|Hand County|SD|51545|United States|-7|single family| +12837|AAAAAAAAFCCDAAAA|532|Lake 4th|Pkwy|Suite 20|Mount Pleasant|Pickens County|AL|31933|United States|-6|single family| +12838|AAAAAAAAGCCDAAAA|238|6th College|Avenue|Suite 160|Greenfield|Camden County|MO|65038|United States|-6|single family| +12839|AAAAAAAAHCCDAAAA|692|5th Williams|Pkwy|Suite C|Marion|Fairfax city|VA|20399|United States|-5|apartment| +12840|AAAAAAAAICCDAAAA|77|7th |Court|Suite H|Lebanon|Multnomah County|OR|92898|United States|-8|apartment| +12841|AAAAAAAAJCCDAAAA|248|12th |Court|Suite 150|Vance|Wasco County|OR|90268|United States|-8|apartment| +12842|AAAAAAAAKCCDAAAA|216|3rd |Court|Suite 110|Reno|Sanilac County|MI|40344|United States|-5|apartment| +12843|AAAAAAAALCCDAAAA|805|2nd Main|Drive|Suite 430|Georgetown|Liberty County|TX|77057|United States|-6|apartment| +12844|AAAAAAAAMCCDAAAA|984|6th 5th|RD|Suite 50|White Oak|Shelby County|IN|46668|United States|-5|single family| +12845|AAAAAAAANCCDAAAA|109|6th |Parkway|Suite 170|Florence|Geauga County|OH|43394|United States|-5|apartment| +12846|AAAAAAAAOCCDAAAA|299|Park Smith|Ave|Suite 180|Oak Grove|Macon County|TN|38370|United States|-6|single family| +12847|AAAAAAAAPCCDAAAA|591|Lake |Pkwy|Suite D|Greenville|Montrose County|CO|81387|United States|-7|condo| +12848|AAAAAAAAADCDAAAA|243|Hill Elm|Street|Suite 160|Union|Hickman County|TN|38721|United States|-5|condo| +12849|AAAAAAAABDCDAAAA|691|Fourth Spruce|Road|Suite T|Jackson|Hot Spring County|AR|79583|United States|-6|single family| +12850|AAAAAAAACDCDAAAA|407|College |Wy|Suite 190|Whitney|Dickenson County|VA|28339|United States|-5|condo| +12851|AAAAAAAADDCDAAAA|772|Spring Hill|Ct.|Suite 460|Bunker Hill|Taylor County|KY|40150|United States|-5|apartment| +12852|AAAAAAAAEDCDAAAA|728|13th Main|Ave|Suite 320|Jackson|Coconino County|AZ|89583|United States|-7|condo| +12853|AAAAAAAAFDCDAAAA|552|Lincoln |Street|Suite 100|Ashland|McMullen County|TX|74244|United States|-6|apartment| +12854|AAAAAAAAGDCDAAAA|934|Jefferson Jefferson|Drive|Suite 30|Oakland|Goshen County|WY|89843|United States|-7|condo| +12855|AAAAAAAAHDCDAAAA|484|1st 2nd|Dr.|Suite 80|Jackson|Osceola County|MI|49583|United States|-5|single family| +12856|AAAAAAAAIDCDAAAA|720|Spring |Road|Suite G|Maple Grove|Randolph County|AR|78252|United States|-6|single family| +12857|AAAAAAAAJDCDAAAA|676|Johnson |Circle|Suite 190|Newtown|Charlotte County|FL|31749|United States|-5|single family| +12858|AAAAAAAAKDCDAAAA|166|North |Way|Suite V|Concord|Rice County|MN|54107|United States|-6|condo| +12859|AAAAAAAALDCDAAAA|869|Railroad South|Lane|Suite O|Georgetown|Yellow Medicine County|MN|57057|United States|-6|single family| +12860|AAAAAAAAMDCDAAAA|700|Cherry Center|Road|Suite O|Marion|Meriwether County|GA|30399|United States|-5|condo| +12861|AAAAAAAANDCDAAAA|167|Meadow Park|Way|Suite M|Pleasant Valley|Appomattox County|VA|22477|United States|-5|single family| +12862|AAAAAAAAODCDAAAA|279|Washington |Wy|Suite W|Walnut Grove|Texas County|MO|67752|United States|-6|single family| +12863|AAAAAAAAPDCDAAAA|900|3rd |Blvd|Suite E|Hillcrest|Stanton County|NE|63003|United States|-7|single family| +12864|AAAAAAAAAECDAAAA|665|Maple |Drive|Suite 470|Fairfield|Oneida County|NY|16192|United States|-5|condo| +12865|AAAAAAAABECDAAAA|579|9th |ST|Suite 120|Edgewood|Cameron Parish|LA|70069|United States|-6|condo| +12866|AAAAAAAACECDAAAA|566|Sixth Williams|Circle|Suite 210|Liberty|Clayton County|GA|33451|United States|-5|apartment| +12867|AAAAAAAADECDAAAA|44|4th |Ave|Suite X|Friendship|Uinta County|WY|84536|United States|-7|condo| +12868|AAAAAAAAEECDAAAA|742|9th |Circle|Suite 70|Cedar Grove|Wabash County|IL|60411|United States|-6|apartment| +12869|AAAAAAAAFECDAAAA|190|13th 15th|Court|Suite H|Friendship|Jerome County|ID|84536|United States|-7|condo| +12870|AAAAAAAAGECDAAAA|683|Main |ST|Suite 490|Arlington|Clay County|KY|46557|United States|-6|condo| +12871|AAAAAAAAHECDAAAA|982|Lincoln |Pkwy|Suite I|Hopewell|Lac qui Parle County|MN|50587|United States|-6|apartment| +12872|AAAAAAAAIECDAAAA|158|Forest 9th|Boulevard|Suite G|Harmony|Benson County|ND|55804|United States|-6|condo| +12873|AAAAAAAAJECDAAAA|376|Willow |Road|Suite 0|Oak Hill|Brule County|SD|57838|United States|-6|condo| +12874|AAAAAAAAKECDAAAA|962|4th |Avenue|Suite 260|Shiloh|Churchill County|NV|89275|United States|-8|apartment| +12875|AAAAAAAALECDAAAA|41|Davis 3rd|Wy|Suite 60|Oakdale|Polk County|TX|79584|United States|-6|single family| +12876|AAAAAAAAMECDAAAA|466|Seventh |RD|Suite 390|Newport|Jackson County|OK|71521|United States|-6|condo| +12877|AAAAAAAANECDAAAA|258|12th North|Ln|Suite Y|Sulphur Springs|Seward County|NE|68354|United States|-7|single family| +12878|AAAAAAAAOECDAAAA|670|Johnson |Pkwy|Suite 130|Greenville|Sabine County|TX|71387|United States|-6|single family| +12879|AAAAAAAAPECDAAAA|356|Spring |Blvd|Suite 100|Midway|Otsego County|MI|41904|United States|-5|single family| +12880|AAAAAAAAAFCDAAAA|953|Madison 9th|ST|Suite P|Wyoming|Tishomingo County|MS|50216|United States|-6|single family| +12881|AAAAAAAABFCDAAAA|450|Church Oak|Parkway|Suite 400|Forest|Cowlitz County|WA|97537|United States|-8|apartment| +12882|AAAAAAAACFCDAAAA|109|2nd |Drive|Suite U|Woodbury|Jackson County|AR|74489|United States|-6|single family| +12883|AAAAAAAADFCDAAAA|22|Williams Oak|Avenue|Suite 480|Hardy|Clinton County|NY|15354|United States|-5|single family| +12884|AAAAAAAAEFCDAAAA|583|Railroad Third|Circle|Suite U|Centerville|Solano County|CA|90059|United States|-8|apartment| +12885|AAAAAAAAFFCDAAAA|122|West |Pkwy|Suite Y|Oakwood|Coffee County|AL|30169|United States|-6|condo| +12886|AAAAAAAAGFCDAAAA||Wilson Oak||Suite 200|Adrian||||United States||condo| +12887|AAAAAAAAHFCDAAAA|508|Madison Third|RD|Suite M|Union Hill|Union County|OR|97746|United States|-8|apartment| +12888|AAAAAAAAIFCDAAAA|867|Davis |Ave|Suite 430|Lakeview|Hubbard County|MN|58579|United States|-6|condo| +12889|AAAAAAAAJFCDAAAA|226|Johnson Main|Circle|Suite 140|Providence|Neshoba County|MS|56614|United States|-6|single family| +12890|AAAAAAAAKFCDAAAA|152|7th |ST|Suite 450|Cumberland|Colorado County|TX|78971|United States|-6|condo| +12891|AAAAAAAALFCDAAAA|705|12th Johnson|Road|Suite F|Bethel|Bradley County|TN|35281|United States|-5|condo| +12892|AAAAAAAAMFCDAAAA|145|Spring Highland|Pkwy|Suite 170|Edgewood|Lipscomb County|TX|70069|United States|-6|apartment| +12893|AAAAAAAANFCDAAAA|172|6th College|Boulevard|Suite W|Stringtown|Owsley County|KY|40162|United States|-5|apartment| +12894|AAAAAAAAOFCDAAAA||Birch Third|Dr.||Salem|Hot Springs County|WY||United States||| +12895|AAAAAAAAPFCDAAAA|28|Elm 7th|Parkway|Suite Q|Belfast|Genesee County|MI|40125|United States|-5|single family| +12896|AAAAAAAAAGCDAAAA||||Suite P||Trinity County|TX|||-6|condo| +12897|AAAAAAAABGCDAAAA|160|Park Lincoln|Wy|Suite T|Wayland|Coshocton County|OH|45115|United States|-5|single family| +12898|AAAAAAAACGCDAAAA|413|Lake Railroad|Circle|Suite Y|Ashland|Chambers County|AL|34244|United States|-6|condo| +12899|AAAAAAAADGCDAAAA|552|3rd |Road|Suite P|Enterprise|Shenandoah County|VA|21757|United States|-5|condo| +12900|AAAAAAAAEGCDAAAA|712|Chestnut 1st|Avenue|Suite X|Cedar Grove|Lancaster County|NE|60411|United States|-7|single family| +12901|AAAAAAAAFGCDAAAA|878|Church 5th|Cir.|Suite S|Clifton|Morgan County|OH|48014|United States|-5|apartment| +12902|AAAAAAAAGGCDAAAA|775|Center 13th|Drive|Suite 350|Glenwood|Jefferson County|MS|53511|United States|-6|apartment| +12903|AAAAAAAAHGCDAAAA|545|Church Fourth|Cir.|Suite 60|Five Points|Harvey County|KS|66098|United States|-6|apartment| +12904|AAAAAAAAIGCDAAAA|665|Laurel Washington|Drive|Suite 360|Summit|Allegan County|MI|40499|United States|-5|condo| +12905|AAAAAAAAJGCDAAAA|445|6th |Wy|Suite R|Lakeville|Hempstead County|AR|78811|United States|-6|single family| +12906|AAAAAAAAKGCDAAAA||Sunset Meadow|Road|||Gates County|NC||||| +12907|AAAAAAAALGCDAAAA|770|7th Madison|Road|Suite 180|Stringtown|Colusa County|CA|90162|United States|-8|condo| +12908|AAAAAAAAMGCDAAAA|222|3rd |Way|Suite 390|Shiloh|Jefferson County|OH|49275|United States|-5|condo| +12909|AAAAAAAANGCDAAAA|433|Hill |Ct.|Suite E|Clinton|Renville County|MN|58222|United States|-6|single family| +12910|AAAAAAAAOGCDAAAA|616|Forest |Circle|Suite R|Oakwood|Columbia County|FL|30169|United States|-5|condo| +12911|AAAAAAAAPGCDAAAA|975|Wilson Dogwood|Road|Suite Q|Midway|Valley County|NE|61904|United States|-7|condo| +12912|AAAAAAAAAHCDAAAA|632|13th North|Cir.|Suite L|Hurricane|Jackson County|TX|77644|United States|-6|single family| +12913|AAAAAAAABHCDAAAA|885|Birch |Ct.|Suite P|Lakeview|Crane County|TX|78579|United States|-6|condo| +12914|AAAAAAAACHCDAAAA|27|Third Locust|Parkway|Suite 110|Harmony|Buckingham County|VA|25804|United States|-5|apartment| +12915|AAAAAAAADHCDAAAA|277|Church 3rd|Parkway|Suite 350|Farmington|Bossier Parish|LA|79145|United States|-6|apartment| +12916|AAAAAAAAEHCDAAAA|554|Chestnut |Cir.|Suite 110|Mount Zion|Adams County|IN|48054|United States|-5|single family| +12917|AAAAAAAAFHCDAAAA|716|3rd |Street|Suite U|Bethel|Coffee County|TN|35281|United States|-5|single family| +12918|AAAAAAAAGHCDAAAA|84|Lakeview |Ln|Suite H|Green Acres|McDonough County|IL|67683|United States|-6|single family| +12919|AAAAAAAAHHCDAAAA|883|Valley |Drive|Suite J|Macedonia|Lanier County|GA|31087|United States|-5|single family| +12920|AAAAAAAAIHCDAAAA|205|Spring Lake|Road|Suite 170|Oak Hill|Rankin County|MS|57838|United States|-6|apartment| +12921|AAAAAAAAJHCDAAAA|520|Miller Lincoln|Street|Suite 0|Union|Runnels County|TX|78721|United States|-6|single family| +12922|AAAAAAAAKHCDAAAA|807|Park |Way|Suite 460|Highland|Pleasants County|WV|29454|United States|-5|single family| +12923|AAAAAAAALHCDAAAA|754|Mill Second|Circle|Suite N|Mount Pleasant|Parke County|IN|41933|United States|-5|condo| +12924|AAAAAAAAMHCDAAAA|293|Walnut Park|Blvd|Suite P|Hamilton|Floyd County|GA|32808|United States|-5|apartment| +12925|AAAAAAAANHCDAAAA|451|Lincoln |Drive|Suite 190|Greenfield|Franklin County|ME|05638|United States|-5|condo| +12926|AAAAAAAAOHCDAAAA|||||Stringtown||VA|20162|United States||| +12927|AAAAAAAAPHCDAAAA|761|First Washington|Street|Suite I|Glendale|Lycoming County|PA|13951|United States|-5|single family| +12928|AAAAAAAAAICDAAAA|918|Williams 9th|Parkway|Suite 60|Oakwood|Marion County|KS|60169|United States|-6|condo| +12929|AAAAAAAABICDAAAA|196|Hickory |Road|Suite 450|Walnut Grove|Yavapai County|AZ|87752|United States|-7|single family| +12930|AAAAAAAACICDAAAA|187|Ninth Fourth|Road|Suite V|Mount Zion|Thomas County|KS|68054|United States|-6|single family| +12931|AAAAAAAADICDAAAA|129|||||||64136|||single family| +12932|AAAAAAAAEICDAAAA|115|Williams |Dr.|Suite 140|Red Hill|Douglas County|WA|94338|United States|-8|single family| +12933|AAAAAAAAFICDAAAA|908|Forest |Lane|Suite U|Clinton|Morgan County|IL|68222|United States|-6|condo| +12934|AAAAAAAAGICDAAAA|659|Church |Avenue|Suite R|Newtown|Cass County|ND|51749|United States|-6|apartment| +12935|AAAAAAAAHICDAAAA|262|Davis |Street|Suite 360|Greenfield|Adams County|IA|55038|United States|-6|single family| +12936|AAAAAAAAIICDAAAA|412|3rd |Boulevard|Suite 440|Shady Grove|Huron County|MI|42812|United States|-5|condo| +12937|AAAAAAAAJICDAAAA|79|Fourth |Street|Suite 60|Amity|Scotts Bluff County|NE|60766|United States|-7|single family| +12938|AAAAAAAAKICDAAAA|127|Seventh 1st|RD|Suite M|Stringtown|Suffolk city|VA|20162|United States|-5|single family| +12939|AAAAAAAALICDAAAA|196|1st |Way|Suite E|Crystal|Bullock County|AL|35258|United States|-6|apartment| +12940|AAAAAAAAMICDAAAA|20|Park Franklin|Ave|Suite P|Arlington|McCracken County|KY|46557|United States|-5|condo| +12941|AAAAAAAANICDAAAA|389|Cedar Ash|Pkwy|Suite 160|Harmony|Merced County|CA|95804|United States|-8|apartment| +12942|AAAAAAAAOICDAAAA|946|Wilson Sunset|Ct.|Suite 270|Clifton|Macoupin County|IL|68014|United States|-6|condo| +12943|AAAAAAAAPICDAAAA|127|Ridge Maple|Boulevard|Suite 350|Pine Grove|Cumberland County|NC|24593|United States|-5|apartment| +12944|AAAAAAAAAJCDAAAA|35|Woodland Spring|Blvd|Suite 350|Liberty|Greene County|IA|53451|United States|-6|condo| +12945|AAAAAAAABJCDAAAA|457|Washington 6th|Court|Suite R|Green Acres|Randolph County|NC|27683|United States|-5|apartment| +12946|AAAAAAAACJCDAAAA|722|Spring |Avenue|Suite B|Stringtown|Belmont County|OH|40162|United States|-5|condo| +12947|AAAAAAAADJCDAAAA|168|Meadow 3rd|Pkwy|Suite 380|Farmington|Chesapeake city|VA|29145|United States|-5|apartment| +12948|AAAAAAAAEJCDAAAA|615|10th Church|Dr.|Suite 220|Greenwood|Jefferson County|IA|58828|United States|-6|apartment| +12949|AAAAAAAAFJCDAAAA|885|Railroad |Way|Suite M|Somerville|Niagara County|NY|17783|United States|-5|condo| +12950|AAAAAAAAGJCDAAAA|317|Spring |Ave|Suite W|Oak Grove|De Witt County|IL|68370|United States|-6|condo| +12951|AAAAAAAAHJCDAAAA|141|Jackson |Circle|Suite 130|Hurricane|Otter Tail County|MN|57644|United States|-6|condo| +12952|AAAAAAAAIJCDAAAA|513|Sixth Church|Drive|Suite S|Sunnyside|Hamilton County|IA|51952|United States|-6|single family| +12953|AAAAAAAAJJCDAAAA|426|Jackson |Way|Suite W|Summit|Huntingdon County|PA|10499|United States|-5|single family| +12954|AAAAAAAAKJCDAAAA|728|8th Madison|Wy|Suite J|Union Hill|Boone County|WV|27746|United States|-5|condo| +12955|AAAAAAAALJCDAAAA|595|Hill Walnut|Drive|Suite M|Clifton|Otero County|NM|88014|United States|-7|single family| +12956|AAAAAAAAMJCDAAAA|73|12th |Dr.|Suite Q|Antioch|Marion County|MS|58605|United States|-6|apartment| +12957|AAAAAAAANJCDAAAA|818|Cedar |Street|Suite 260|Sullivan|Owen County|KY|40451|United States|-5|single family| +12958|AAAAAAAAOJCDAAAA|102|Woodland 11th|Street|Suite 380|Charleston|Lamar County|MS|58018|United States|-6|single family| +12959|AAAAAAAAPJCDAAAA|728|Miller Poplar|Drive|Suite R|Crossroads|Monroe County|TN|30534|United States|-6|condo| +12960|AAAAAAAAAKCDAAAA|836|Second Hickory|Blvd|Suite 350|Mountain View|Marion County|MS|54466|United States|-6|apartment| +12961|AAAAAAAABKCDAAAA|508|Church |Ave|Suite 220|Plainview|Mineral County|NV|83683|United States|-8|apartment| +12962|AAAAAAAACKCDAAAA|586|Pine |Ln|Suite 220|Walnut Grove|Potter County|TX|77752|United States|-6|apartment| +12963|AAAAAAAADKCDAAAA|674|Broadway |Blvd|Suite H|Hopewell|Glasscock County|TX|70587|United States|-6|apartment| +12964|AAAAAAAAEKCDAAAA|216|Valley |Pkwy|Suite C|Bridgeport|Hockley County|TX|75817|United States|-6|condo| +12965|AAAAAAAAFKCDAAAA|188|South First|Road|Suite K|Oak Ridge|San Francisco County|CA|98371|United States|-8|single family| +12966|AAAAAAAAGKCDAAAA|524|14th |Court|Suite D|Summit|San Patricio County|TX|70499|United States|-6|single family| +12967|AAAAAAAAHKCDAAAA|541|2nd |Parkway|Suite 20|Unionville|Morris County|TX|71711|United States|-6|single family| +12968|AAAAAAAAIKCDAAAA|250|Park View|Blvd|Suite Q|Ashland|Camas County|ID|84244|United States|-7|single family| +12969|AAAAAAAAJKCDAAAA|737|13th 9th|Wy|Suite J|Wilson|Evans County|GA|36971|United States|-5|condo| +12970|AAAAAAAAKKCDAAAA||North West|||Shiloh||||United States|-5|apartment| +12971|AAAAAAAALKCDAAAA|149|Hillcrest First|Drive|Suite C|Oakwood|Schuylkill County|PA|10169|United States|-5|single family| +12972|AAAAAAAAMKCDAAAA|171|5th |Wy|Suite 70|Providence|Clermont County|OH|46614|United States|-5|apartment| +12973|AAAAAAAANKCDAAAA|144|Fifth |Road|Suite H|Westgate|Kalamazoo County|MI|42366|United States|-5|condo| +12974|AAAAAAAAOKCDAAAA|154|5th |Boulevard|Suite 20|Concord|Buchanan County|IA|54107|United States|-6|single family| +12975|AAAAAAAAPKCDAAAA|171|Fourth |Blvd|Suite A|Union|Frederick County|VA|28721|United States|-5|single family| +12976|AAAAAAAAALCDAAAA|453|Meadow |Ave|Suite 210|Riverdale|Clay County|TN|39391|United States|-5|apartment| +12977|AAAAAAAABLCDAAAA|179|13th |Dr.|Suite 350|Hillcrest|Sullivan County|NY|13003|United States|-5|condo| +12978|AAAAAAAACLCDAAAA|643||Cir.||Lakewood|Blanco County||78877||-6|| +12979|AAAAAAAADLCDAAAA|441|West |Ln|Suite S|Centerville|Harrison County|OH|40059|United States|-5|single family| +12980|AAAAAAAAELCDAAAA|420|Laurel |Ln|Suite 70|Woodbury|Lubbock County|TX|74489|United States|-6|condo| +12981|AAAAAAAAFLCDAAAA|601|Hillcrest |Dr.|Suite K|Four Points|Kingman County|KS|61216|United States|-6|single family| +12982|AAAAAAAAGLCDAAAA|289|Laurel Washington|Ct.|Suite 100|Newport|Hart County|KY|41521|United States|-6|apartment| +12983|AAAAAAAAHLCDAAAA|45|Mill Jefferson|Road|Suite 40|White Oak|Cumberland County|NJ|07268|United States|-5|apartment| +12984|AAAAAAAAILCDAAAA|190|Walnut Railroad|Court|Suite X|Lakewood|Wood County|WV|28877|United States|-5|single family| +12985|AAAAAAAAJLCDAAAA|552|South 10th|Way|Suite A|Cedar Grove|Delaware County|PA|10411|United States|-5|single family| +12986|AAAAAAAAKLCDAAAA|194|Meadow |Road|Suite V|Bethel|Chase County|KS|65281|United States|-6|single family| +12987|AAAAAAAALLCDAAAA|950|Adams Walnut|Ln|Suite 180|Cedar Grove|Alexandria city|VA|20411|United States|-5|condo| +12988|AAAAAAAAMLCDAAAA|537|Poplar |Ave|Suite K|Five Forks|Warren County|MO|62293|United States|-6|apartment| +12989|AAAAAAAANLCDAAAA|7|Cedar |Street|Suite J|Clifton|Sumter County|SC|28014|United States|-5|single family| +12990|AAAAAAAAOLCDAAAA|255|Jefferson |Lane|Suite 100|Mount Vernon|Waynesboro city|VA|28482|United States|-5|apartment| +12991|AAAAAAAAPLCDAAAA|300|Pine Lake|Drive|Suite 250|Big Creek|Becker County|MN|55386|United States|-6|single family| +12992|AAAAAAAAAMCDAAAA|757|College 3rd|Blvd|Suite Y|Lakeside|Coleman County|TX|79532|United States|-6|apartment| +12993|AAAAAAAABMCDAAAA|20|Pine Park|Parkway|Suite K|Pomona|Louisa County|VA|24153|United States|-5|apartment| +12994|AAAAAAAACMCDAAAA|792|Cedar Seventh|RD|Suite A|Lakeview|Cherokee County|GA|38579|United States|-5|single family| +12995|AAAAAAAADMCDAAAA|377|Eigth Dogwood|Drive|Suite K|Woodville|Williamson County|TX|74289|United States|-6|condo| +12996|AAAAAAAAEMCDAAAA|205|Washington Main|RD|Suite O|White Oak|Darlington County|SC|26668|United States|-5|single family| +12997|AAAAAAAAFMCDAAAA|236|Lincoln |Ave|Suite 380|Shelby|Jim Hogg County|TX|76575|United States|-6|condo| +12998|AAAAAAAAGMCDAAAA|413|Adams Oak|Street|Suite 60|Fairfield|Thurston County|WA|96192|United States|-8|condo| +12999|AAAAAAAAHMCDAAAA|395|Lake |Parkway||||UT||||| +13000|AAAAAAAAIMCDAAAA|401|Highland Lake|Parkway|Suite V|Riverdale|Uintah County|UT|89391|United States|-7|single family| +13001|AAAAAAAAJMCDAAAA|648|Main |Lane|Suite Y|Oak Grove|Langlade County|WI|58370|United States|-6|apartment| +13002|AAAAAAAAKMCDAAAA|305|Walnut |Blvd|Suite 380|Buena Vista|Wythe County|VA|25752|United States|-5|condo| +13003|AAAAAAAALMCDAAAA|28|Lake Cherry|Parkway|Suite 290|Oakwood|Madison County|NE|60169|United States|-7|single family| +13004|AAAAAAAAMMCDAAAA|47|Railroad Elevnth|Drive|Suite Y|Glendale|Perry County|PA|13951|United States|-5|single family| +13005|AAAAAAAANMCDAAAA|138|West |Street|Suite 170|Providence|Prince William County|VA|26614|United States|-5|condo| +13006|AAAAAAAAOMCDAAAA|187|Ridge Poplar|Lane|Suite 270|Lakeside|McDowell County|WV|29532|United States|-5|single family| +13007|AAAAAAAAPMCDAAAA|587|Walnut Fourth|Wy|Suite K|Unionville|Atchison County|MO|61711|United States|-6|apartment| +13008|AAAAAAAAANCDAAAA|279|Poplar |Street|Suite 40|Hamilton|Cherokee County|TX|72808|United States|-6|single family| +13009|AAAAAAAABNCDAAAA|673|5th Miller|Blvd|Suite 210|Lincoln|Gratiot County|MI|41289|United States|-5|single family| +13010|AAAAAAAACNCDAAAA|820|Cedar Fifth|Ave|Suite T|Shady Grove|Motley County|TX|72812|United States|-6|apartment| +13011|AAAAAAAADNCDAAAA|395|2nd Walnut|Lane|Suite 220|Stewart|Waushara County|WI|58041|United States|-6|apartment| +13012|AAAAAAAAENCDAAAA|617|1st Cherry|Ave|Suite 350|Hopewell|Grant County|ND|50587|United States|-6|apartment| +13013|AAAAAAAAFNCDAAAA|989|Main |Road|Suite Q|Belmont|Duplin County|NC|20191|United States|-5|single family| +13014|AAAAAAAAGNCDAAAA|507|8th |Court|Suite D|Lebanon|Crook County|WY|82898|United States|-7|condo| +13015|AAAAAAAAHNCDAAAA|540|1st Main|Blvd|Suite 320|Webb|Dundy County|NE|60899|United States|-6|single family| +13016|AAAAAAAAINCDAAAA|722|Center Birch|Lane|Suite I|Liberty|Henry County|TN|33451|United States|-5|apartment| +13017|AAAAAAAAJNCDAAAA|741|Elm |Ct.|Suite 330|Johnsonville|Stanley County|SD|57745|United States|-7|condo| +13018|AAAAAAAAKNCDAAAA|237|Adams |||Lakeville|Kingman County||68811||-6|| +13019|AAAAAAAALNCDAAAA|893|Meadow |RD|Suite D|Arlington|Callahan County|TX|76557|United States|-6|apartment| +13020|AAAAAAAAMNCDAAAA|636|2nd |Road|Suite F|Riverdale|Columbus County|NC|29391|United States|-5|condo| +13021|AAAAAAAANNCDAAAA|984|15th Madison|Road|Suite 220|Maple Grove|Brazoria County|TX|78252|United States|-6|single family| +13022|AAAAAAAAONCDAAAA|207|2nd |Pkwy|Suite 140|Lincoln|Isabella County|MI|41289|United States|-5|condo| +13023|AAAAAAAAPNCDAAAA|688|Maple |Road|Suite E|Highland Park|Newton County|MS|56534|United States|-6|single family| +13024|AAAAAAAAAOCDAAAA|70|Jackson 10th|Lane|Suite J|Five Points|Halifax County|NC|26098|United States|-5|condo| +13025|AAAAAAAABOCDAAAA|391|Main |Parkway|Suite 450|Oakwood|Robeson County|NC|20169|United States|-5|condo| +13026|AAAAAAAACOCDAAAA|522|1st Center|ST|Suite F|Kingston|Patrick County|VA|24975|United States|-5|single family| +13027|AAAAAAAADOCDAAAA|860|7th |Boulevard|Suite C|Lakeview|Carson City|NV|88579|United States|-8|apartment| +13028|AAAAAAAAEOCDAAAA|371|Davis Lee|Ave|Suite T|Riverdale|Anderson County|TX|79391|United States|-6|condo| +13029|AAAAAAAAFOCDAAAA|937|Ridge Eigth|Court|Suite 400|Pine Grove|Meade County|KY|44593|United States|-5|single family| +13030|AAAAAAAAGOCDAAAA|910|West |Circle|Suite 80|Oak Grove|Houston County|GA|38370|United States|-5|condo| +13031|AAAAAAAAHOCDAAAA|67|Park Lake|Court|Suite Y|Mount Olive|Kay County|OK|78059|United States|-6|single family| +13032|AAAAAAAAIOCDAAAA|328|West |RD|Suite 460|Walnut Grove|Hamilton County|KS|67752|United States|-6|single family| +13033|AAAAAAAAJOCDAAAA|330|4th Hillcrest|Ave|Suite 30|Sunnyside|Walker County|AL|31952|United States|-6|condo| +13034|AAAAAAAAKOCDAAAA|578|Smith |ST|Suite M|Fairfield|Briscoe County|TX|76192|United States|-6|apartment| +13035|AAAAAAAALOCDAAAA|56|Seventh |Court|Suite T|Crossroads|Tucker County|WV|20534|United States|-5|single family| +13036|AAAAAAAAMOCDAAAA|817|1st |Blvd|Suite D|Bethel|Gratiot County|MI|45281|United States|-5|condo| +13037|AAAAAAAANOCDAAAA|78|Church Cedar|Circle|Suite 80|Cedar Grove|Comal County|TX|70411|United States|-6|single family| +13038|AAAAAAAAOOCDAAAA|452|12th 6th|Road|Suite 0|Griffin|Cedar County|NE|61204|United States|-6|condo| +13039|AAAAAAAAPOCDAAAA|443|Smith North|Cir.|Suite C|Wildwood|McLeod County|MN|56871|United States|-6|apartment| +13040|AAAAAAAAAPCDAAAA|928|5th View|Dr.|Suite B|Woodland|Baltimore County|MD|24854|United States|-5|apartment| +13041|AAAAAAAABPCDAAAA|991|Lee Spring|Avenue|Suite H|Buena Vista|Alcona County|MI|45752|United States|-5|apartment| +13042|AAAAAAAACPCDAAAA|595|Lincoln 6th|Dr.|Suite 180|Wilson|Lorain County|OH|46971|United States|-5|apartment| +13043|AAAAAAAADPCDAAAA|330|Lakeview Sixth||Suite 100||Bosque County|||United States|-6|| +13044|AAAAAAAAEPCDAAAA|686|5th |Parkway|Suite K|New Hope|Hardin County|IL|69431|United States|-6|apartment| +13045|AAAAAAAAFPCDAAAA|939|Willow East|Circle|Suite H|Wildwood|Campbell County|SD|56871|United States|-6|apartment| +13046|AAAAAAAAGPCDAAAA|143|First 8th|Drive|Suite 20|Dewey|Garza County|TX|71160|United States|-6|single family| +13047|AAAAAAAAHPCDAAAA|225|Walnut |Blvd|Suite 60|Oak Hill|Oneida County|ID|87838|United States|-7|apartment| +13048|AAAAAAAAIPCDAAAA|428|Meadow |Blvd|Suite G|Willis|Pinal County|AZ|86788|United States|-7|single family| +13049|AAAAAAAAJPCDAAAA|260|Willow |Pkwy|Suite 140|Arthur|Claiborne County|MS|55965|United States|-6|single family| +13050|AAAAAAAAKPCDAAAA|71|Walnut |Blvd|Suite 220|Woodruff|Hart County|GA|34174|United States|-5|apartment| +13051|AAAAAAAALPCDAAAA|285|Hill |Cir.|Suite 10|Lakeside|Hitchcock County|NE|69532|United States|-6|single family| +13052|AAAAAAAAMPCDAAAA|725|Pine |Boulevard|Suite 80|Perkins|Somerset County|MD|21852|United States|-5|single family| +13053|AAAAAAAANPCDAAAA|604|4th Birch|Ave|Suite D|Clifton|Aroostook County|ME|08614|United States|-5|condo| +13054|AAAAAAAAOPCDAAAA|377|Woodland |RD|Suite 250|Meadowbrook|Fillmore County|NE|60156|United States|-6|condo| +13055|AAAAAAAAPPCDAAAA|777|Cedar 1st|Court|||Tioga County|||United States|-5|apartment| +13056|AAAAAAAAAADDAAAA|547|||Suite Q|Brownsville|Cooper County|||United States||single family| +13057|AAAAAAAABADDAAAA|206|Lake |Avenue|Suite 40|Cedar Grove|Midland County|TX|70411|United States|-6|condo| +13058|AAAAAAAACADDAAAA|974|10th Oak|Pkwy|Suite W|Willow|Van Wert County|OH|46798|United States|-5|apartment| +13059|AAAAAAAADADDAAAA|771|Lincoln |Wy|Suite 440|Friendship|Scott County|IN|44536|United States|-5|condo| +13060|AAAAAAAAEADDAAAA|534|Broadway 9th|Avenue|Suite 80|Woodville|Gilmer County|WV|24289|United States|-5|condo| +13061|AAAAAAAAFADDAAAA|894|Mill Maple|Ave|Suite 70|Lakeside|Loup County|NE|69532|United States|-7|condo| +13062|AAAAAAAAGADDAAAA|423|Spring Tenth|Boulevard|Suite P|Lincoln|Dickenson County|VA|21289|United States|-5|condo| +13063|AAAAAAAAHADDAAAA|148|Davis |Way|Suite 310|Highland Park|Lawrence County|AL|36534|United States|-6|single family| +13064|AAAAAAAAIADDAAAA|239|Center |Parkway|Suite 440|Pleasant Valley|King County|WA|92477|United States|-8|single family| +13065|AAAAAAAAJADDAAAA|116|6th |Boulevard|Suite 300|Crossroads|Hamilton County|IL|60534|United States|-6|condo| +13066|AAAAAAAAKADDAAAA|83|Cherry |Avenue|Suite L|Lebanon|Will County|IL|62898|United States|-6|condo| +13067|AAAAAAAALADDAAAA|9|Park Eigth|Court|Suite R|Macedonia|Monroe County|AL|31087|United States|-6|apartment| +13068|AAAAAAAAMADDAAAA|210|5th |Wy|Suite 0|Bethel|Gray County|KS|65281|United States|-6|single family| +13069|AAAAAAAANADDAAAA|803|Center First|Wy|Suite V|Buena Vista|Allen County|KS|65752|United States|-6|apartment| +13070|AAAAAAAAOADDAAAA|302|Woodland |Wy|Suite F|Oak Grove|Becker County|MN|58370|United States|-6|condo| +13071|AAAAAAAAPADDAAAA|766|1st |Ave|Suite 440|Indian Village|Yellow Medicine County|MN|51075|United States|-6|condo| +13072|AAAAAAAAABDDAAAA|83|5th 4th|Ln|Suite D|Woodville|Kittson County|MN|54289|United States|-6|single family| +13073|AAAAAAAABBDDAAAA|280|Main 11th|Ct.|Suite 310|Harmony|Plumas County|CA|95804|United States|-8|apartment| +13074|AAAAAAAACBDDAAAA|118|Locust |Avenue|Suite A|Shady Grove|Sequoyah County|OK|72812|United States|-6|single family| +13075|AAAAAAAADBDDAAAA|808|Franklin Oak|Blvd|Suite Y|Concord|Middlesex County|NJ|04707|United States|-5|single family| +13076|AAAAAAAAEBDDAAAA|371|10th |Drive|Suite X|Jamestown|Houston County|AL|36867|United States|-6|single family| +13077|AAAAAAAAFBDDAAAA|112|Park Forest|Boulevard|Suite 420|Buena Vista|Mineral County|NV|85752|United States|-8|apartment| +13078|AAAAAAAAGBDDAAAA|768|Main |Avenue|Suite J|Bay View|Okfuskee County|OK|76457|United States|-6|apartment| +13079|AAAAAAAAHBDDAAAA|117|Johnson |Boulevard|Suite M|Mount Olive|Polk County|GA|38059|United States|-5|condo| +13080|AAAAAAAAIBDDAAAA|236|Smith |Wy|Suite 110|Glenwood|Shoshone County|ID|83511|United States|-7|single family| +13081|AAAAAAAAJBDDAAAA|167|Mill |Avenue|Suite 170|White Oak|Vance County|NC|26668|United States|-5|condo| +13082|AAAAAAAAKBDDAAAA|1|1st |Street|Suite 390|Arthur|Freeborn County|MN|55965|United States|-6|single family| +13083|AAAAAAAALBDDAAAA|||Ave|Suite 250|||PA||United States||single family| +13084|AAAAAAAAMBDDAAAA|264|North Williams|Road|Suite 220|Shady Grove|Lyon County|NV|82812|United States|-8|condo| +13085|AAAAAAAANBDDAAAA|782|Fourth |Road|Suite 480|Centerville|Baca County|CO|80059|United States|-7|apartment| +13086|AAAAAAAAOBDDAAAA|604|Park Lee|Drive|Suite F|Oak Hill|Ellis County|KS|67838|United States|-6|apartment| +13087|AAAAAAAAPBDDAAAA|961|Ridge Mill|Ct.|Suite 150|Sunnyside|Sanpete County|UT|81952|United States|-7|single family| +13088|AAAAAAAAACDDAAAA|311|Third |Cir.|Suite 490|Bridgeport|Matanuska-Susitna Borough|AK|95817|United States|-9|apartment| +13089|AAAAAAAABCDDAAAA|496|Jackson Park|Circle|Suite 120|Hazelwood|Sanders County|MT|61206|United States|-7|condo| +13090|AAAAAAAACCDDAAAA|84|Oak Lee|RD|Suite E|Forest Hills|Barnes County|ND|59237|United States|-6|single family| +13091|AAAAAAAADCDDAAAA|361|7th Walnut|Ln|Suite X|Florence|DeBaca County|NM|83394|United States|-7|condo| +13092|AAAAAAAAECDDAAAA|452|Spring Hill|Ct.|Suite 320|Spring Valley|Posey County|IN|46060|United States|-5|condo| +13093|AAAAAAAAFCDDAAAA|852|Church |Boulevard|Suite F|Springtown|Buffalo County|WI|59858|United States|-6|single family| +13094|AAAAAAAAGCDDAAAA|213|3rd Mill|Ct.|Suite R|Highland Park|Pulaski County|IL|66534|United States|-6|condo| +13095|AAAAAAAAHCDDAAAA|548|Madison |Dr.|Suite 190|Brownsville|Caldwell County|KY|49310|United States|-6|condo| +13096|AAAAAAAAICDDAAAA|733|Park Lake|RD|Suite 160|Franklin|Cavalier County|ND|59101|United States|-6|apartment| +13097|AAAAAAAAJCDDAAAA||12th Lake|||Friendship||LA|74536|||single family| +13098|AAAAAAAAKCDDAAAA|991|Center 9th|Dr.|Suite U|Mechanicsburg|Brown County|IL|62219|United States|-6|condo| +13099|AAAAAAAALCDDAAAA|234|Fifth |Court|Suite W|Blanchard|Clay County|KY|45985|United States|-6|single family| +13100|AAAAAAAAMCDDAAAA|950|Wilson Eigth|Avenue|Suite W|Fairfield|Miller County|GA|36192|United States|-5|single family| +13101|AAAAAAAANCDDAAAA|||Ave||Page||||United States||single family| +13102|AAAAAAAAOCDDAAAA|895|Hickory Broadway|Circle|Suite 10|Glendale|Jackson County|TX|73951|United States|-6|single family| +13103|AAAAAAAAPCDDAAAA|212|Second Poplar|Ln|Suite 90|Springdale|Gove County|KS|68883|United States|-6|apartment| +13104|AAAAAAAAADDDAAAA|670|Lake 7th|Ln|Suite U|Poplar Springs|Millard County|UT|81811|United States|-7|condo| +13105|AAAAAAAABDDDAAAA|326|Forest |Cir.|Suite M|Wolf Creek|Seminole County|FL|32455|United States|-5|apartment| +13106|AAAAAAAACDDDAAAA|828|River 13th|Lane|Suite 300|Bethel|Stewart County|TN|35281|United States|-6|apartment| +13107|AAAAAAAADDDDAAAA|967|Forest |Dr.|Suite 390|Wilson|Frederick County|MD|26971|United States|-5|condo| +13108|AAAAAAAAEDDDAAAA|404|3rd |Court|Suite 60|Brownsville|Langlade County|WI|59310|United States|-6|single family| +13109|AAAAAAAAFDDDAAAA|408|Park |Drive|Suite 300|Springfield|Early County|GA|39303|United States|-5|apartment| +13110|AAAAAAAAGDDDAAAA|708|Highland 8th|Wy|Suite R|Riley|Geneva County|AL|31692|United States|-6|single family| +13111|AAAAAAAAHDDDAAAA|799|Davis Ridge|Wy|Suite U|Belmont|Lincoln County|AR|70191|United States|-6|apartment| +13112|AAAAAAAAIDDDAAAA|458|10th |Street|Suite R|Arlington|Pleasants County|WV|26557|United States|-5|apartment| +13113|AAAAAAAAJDDDAAAA|605|Railroad |ST|Suite E|||VA|22935||-5|| +13114|AAAAAAAAKDDDAAAA|532|Lee |Parkway|Suite 250|Marion|Hancock County|IA|50399|United States|-6|single family| +13115|AAAAAAAALDDDAAAA|686|Church Chestnut|Court|Suite 390|Centerville|Hamilton County|TX|70059|United States|-6|condo| +13116|AAAAAAAAMDDDAAAA|923|Meadow Sunset|Circle|Suite 490|Jamestown|Clay County|MS|56867|United States|-6|single family| +13117|AAAAAAAANDDDAAAA|969|Mill |Way|Suite F|Highland|Alfalfa County|OK|79454|United States|-6|condo| +13118|AAAAAAAAODDDAAAA|603|Meadow |Street|Suite W|Hopewell|Sibley County|MN|50587|United States|-6|single family| +13119|AAAAAAAAPDDDAAAA|953|Valley Franklin|Way|Suite 130|Clifton|Mendocino County|CA|98014|United States|-8|apartment| +13120|AAAAAAAAAEDDAAAA|422|4th |Avenue|Suite 400|Marion|Chester County|PA|10399|United States|-5|condo| +13121|AAAAAAAABEDDAAAA|102|2nd |Circle|Suite Q|Whispering Pines|Jackson County|TN|37609|United States|-5|condo| +13122|AAAAAAAACEDDAAAA|789|15th Hickory|Ln|Suite 310|Pleasant Valley|Audubon County|IA|52477|United States|-6|apartment| +13123|AAAAAAAADEDDAAAA|417|Pine Second|Ave|Suite 120|Forest Hills|Anderson County|KY|49237|United States|-6|single family| +13124|AAAAAAAAEEDDAAAA|181|1st |Blvd|Suite 10|Kingston|La Salle County|IL|64975|United States|-6|apartment| +13125|AAAAAAAAFEDDAAAA|737|||Suite 120|Sunnyside|Lincoln County|ME|02552|||| +13126|AAAAAAAAGEDDAAAA|333|4th |Parkway|Suite 460|Mountain View|Kanabec County|MN|54466|United States|-6|apartment| +13127|AAAAAAAAHEDDAAAA|322|||Suite 190||Bonneville County||84749|United States||apartment| +13128|AAAAAAAAIEDDAAAA|678|Highland 9th|Ct.|Suite 100|Woodville|Palo Pinto County|TX|74289|United States|-6|condo| +13129|AAAAAAAAJEDDAAAA|468|3rd Sycamore|Road|Suite H|Greenville|Marshall County|MS|51387|United States|-6|single family| +13130|AAAAAAAAKEDDAAAA|202|Valley Spring|Wy|Suite 450|Summit|Allegany County|NY|10499|United States|-5|single family| +13131|AAAAAAAALEDDAAAA|602|Washington Highland|Ave|Suite 250|Providence|Bon Homme County|SD|56614|United States|-6|single family| +13132|AAAAAAAAMEDDAAAA|558|West River|Street|Suite 290|Valley View|Stillwater County|MT|65124|United States|-7|single family| +13133|AAAAAAAANEDDAAAA|254|||Suite 140|Deerfield||||United States|-6|| +13134|AAAAAAAAOEDDAAAA|116|Center |Blvd|Suite A|Clinton|Putnam County|IL|68222|United States|-6|single family| +13135|AAAAAAAAPEDDAAAA|855|Miller 6th|Lane|Suite 100|Caledonia|Sumner County|KS|67411|United States|-6|single family| +13136|AAAAAAAAAFDDAAAA|134|Sunset Cherry|Avenue|Suite 360|Unionville|Franklin County|VA|21711|United States|-5|condo| +13137|AAAAAAAABFDDAAAA|257|Madison |Blvd|Suite X|Riverside|Minidoka County|ID|89231|United States|-7|condo| +13138|AAAAAAAACFDDAAAA|451|Green |Lane|Suite K|Jackson|Hall County|TX|79583|United States|-6|single family| +13139|AAAAAAAADFDDAAAA|719|Lake Madison|Circle|Suite 130|Salem|Hill County|TX|78048|United States|-6|single family| +13140|AAAAAAAAEFDDAAAA|498|Chestnut Johnson|Road|Suite 300|Oakland|Peoria County|IL|69843|United States|-6|single family| +13141|AAAAAAAAFFDDAAAA|24|Meadow Elevnth|Blvd|Suite D|Elkton|Horry County|SC|23481|United States|-5|apartment| +13142|AAAAAAAAGFDDAAAA|25|Locust |Blvd|Suite 440|Lebanon|Hancock County|OH|42898|United States|-5|condo| +13143|AAAAAAAAHFDDAAAA|123|Lincoln Spring|Road|Suite 390|Sutton|Carroll County|KY|45413|United States|-6|condo| +13144|AAAAAAAAIFDDAAAA|538|Johnson |Ct.|Suite 360|Five Points|McLean County|ND|56098|United States|-6|condo| +13145|AAAAAAAAJFDDAAAA|508|Davis 2nd|RD|Suite 60|Highland|Washington County|MO|69454|United States|-6|condo| +13146|AAAAAAAAKFDDAAAA|247|River |Wy|Suite 270|Lebanon|Lake County|TN|32898|United States|-6|condo| +13147|AAAAAAAALFDDAAAA|7|Washington Center|Drive|Suite Q|Bunker Hill|Bureau County|IL|60150|United States|-6|single family| +13148|AAAAAAAAMFDDAAAA|752|Park |Lane|Suite 460|Friendship|Dinwiddie County|VA|24536|United States|-5|single family| +13149|AAAAAAAANFDDAAAA|67|Williams Church|Road|Suite 270|Deerfield|Osage County|KS|69840|United States|-6|condo| +13150|AAAAAAAAOFDDAAAA|588|Sycamore |Dr.|Suite 310|Buena Vista|Seminole County|GA|35752|United States|-5|condo| +13151|AAAAAAAAPFDDAAAA|399|Ash 1st|Ave|Suite 110|Doyle|Davidson County|TN|38434|United States|-5|single family| +13152|AAAAAAAAAGDDAAAA|910|Main |ST|Suite G|Farmington|Stephens County|TX|79145|United States|-6|single family| +13153|AAAAAAAABGDDAAAA||View |Ct.|||Lynchburg city||29584||-5|| +13154|AAAAAAAACGDDAAAA||Cedar |||Kingston|Lowndes County|GA|34975|United States||| +13155|AAAAAAAADGDDAAAA|941|North |Boulevard|Suite R|Oakland|Guadalupe County|TX|79843|United States|-6|condo| +13156|AAAAAAAAEGDDAAAA|237|Oak Park|Blvd|Suite 220|Salem|San Juan County|CO|88048|United States|-7|condo| +13157|AAAAAAAAFGDDAAAA|911|Eigth |Circle|Suite 310|Stringtown|Stillwater County|MT|60162|United States|-7|single family| +13158|AAAAAAAAGGDDAAAA|926|Williams Second|Court|Suite J|Lebanon|Houston County|AL|32898|United States|-6|apartment| +13159|AAAAAAAAHGDDAAAA|859|Woodland Ash|RD|Suite X|Greenfield|Fayette County|IL|65038|United States|-6|condo| +13160|AAAAAAAAIGDDAAAA|662|Highland 1st|Ln|Suite 90|Lebanon|Norton city|VA|22898|United States|-5|apartment| +13161|AAAAAAAAJGDDAAAA|936|Adams |ST|Suite M|Parkwood|Forsyth County|GA|31669|United States|-5|apartment| +13162|AAAAAAAAKGDDAAAA|269|4th Adams|Ct.|Suite 470|Highland|Richmond city|VA|29454|United States|-5|single family| +13163|AAAAAAAALGDDAAAA|772|Ridge |Pkwy|Suite L|Springdale|Yakutat Borough|AK|98883|United States|-9|condo| +13164|AAAAAAAAMGDDAAAA|397|Washington |Ct.|Suite 70|Weldon|Wyoming County|NY|16277|United States|-5|apartment| +13165|AAAAAAAANGDDAAAA|903|West |Circle|Suite 190|Bath|Berrien County|GA|30573|United States|-5|single family| +13166|AAAAAAAAOGDDAAAA|115|First |Lane|Suite 170|Price|Salem County|NJ|04424|United States|-5|condo| +13167|AAAAAAAAPGDDAAAA|993|Smith Pine|Court|Suite B|Greenwood|Halifax County|NC|28828|United States|-5|apartment| +13168|AAAAAAAAAHDDAAAA|916|3rd |Way|Suite K|Riverdale|Schenectady County|NY|19391|United States|-5|single family| +13169|AAAAAAAABHDDAAAA|887|9th |Dr.|Suite F|Hamilton|Attala County|MS|52808|United States|-6|condo| +13170|AAAAAAAACHDDAAAA|519|Pine |Wy|Suite 230|Woodville|Jackson County|AR|74289|United States|-6|single family| +13171|AAAAAAAADHDDAAAA|141|Church |Road|Suite 420|Oakdale|Amite County|MS|59584|United States|-6|single family| +13172|AAAAAAAAEHDDAAAA|979|Mill Wilson|Ave|Suite C|Farmington|Lewis County|NY|19145|United States|-5|condo| +13173|AAAAAAAAFHDDAAAA|486|Wilson |Ct.|Suite Q|Carthage|Utah County|UT|81529|United States|-7|apartment| +13174|AAAAAAAAGHDDAAAA|900|Walnut Hickory|Cir.|Suite 470|Maple Grove|Tyrrell County|NC|28252|United States|-5|apartment| +13175|AAAAAAAAHHDDAAAA|747|North Valley|Street|Suite 20|Franklin|Chippewa County|MI|49101|United States|-5|single family| +13176|AAAAAAAAIHDDAAAA|77|Railroad |Way|Suite E|Florence|Newton County|TX|73394|United States|-6|apartment| +13177|AAAAAAAAJHDDAAAA|466|Fifth |Ave|Suite 220|Centerville|Garfield County|OK|70059|United States|-6|apartment| +13178|AAAAAAAAKHDDAAAA|801|Williams |Drive|Suite B|Wilson|Emmet County|MI|46971|United States|-5|apartment| +13179|AAAAAAAALHDDAAAA|404|Spring |Ct.|Suite B|Riverside|Johnson County|WY|89231|United States|-7|single family| +13180|AAAAAAAAMHDDAAAA|225|Church 10th|Ct.|Suite 120|Plainville|Barnstable County|MA|06715|United States|-5|apartment| +13181|AAAAAAAANHDDAAAA|379|Smith |Cir.|Suite G|Five Forks|Briscoe County|TX|72293|United States|-6|single family| +13182|AAAAAAAAOHDDAAAA|82|Eigth Green|Ave|Suite 320|Cedar Grove|Floyd County|IN|40411|United States|-5|single family| +13183|AAAAAAAAPHDDAAAA|510||Cir.||Clinton|Dade County|GA||United States|-5|apartment| +13184|AAAAAAAAAIDDAAAA|815|Park |Drive|Suite 310|Fairview|Daviess County|MO|65709|United States|-6|condo| +13185|AAAAAAAABIDDAAAA|952|Fourth |Cir.|Suite M|Pleasant Valley|Merrimack County|NH|03077|United States|-5|condo| +13186|AAAAAAAACIDDAAAA|117|Washington Walnut|Ln|Suite E|Rossville|Allendale County|SC|22503|United States|-5|apartment| +13187|AAAAAAAADIDDAAAA|790|3rd |ST|Suite J|Woodrow|Humboldt County|IA|54273|United States|-6|condo| +13188|AAAAAAAAEIDDAAAA|989|First Tenth|Cir.|Suite 450|Red Hill|Pickens County|AL|34338|United States|-6|apartment| +13189|AAAAAAAAFIDDAAAA|615|Chestnut Spring|Drive|Suite B|White Oak|Kenedy County|TX|76668|United States|-6|condo| +13190|AAAAAAAAGIDDAAAA|439|2nd Cedar|Blvd|Suite 230|Union Hill|Beaufort County|NC|27746|United States|-5|condo| +13191|AAAAAAAAHIDDAAAA|358|Second Maple|ST|Suite 130|Springfield|Seminole County|OK|79303|United States|-6|single family| +13192|AAAAAAAAIIDDAAAA|251|Ridge Center|Lane|Suite D|Hillcrest|Greene County|IL|63003|United States|-6|single family| +13193|AAAAAAAAJIDDAAAA|897|Willow |Drive|Suite F|Shady Grove|East Feliciana Parish|LA|72812|United States|-6|apartment| +13194|AAAAAAAAKIDDAAAA|452|East |RD|Suite E|Oak Grove|Lake County|IL|68370|United States|-6|condo| +13195|AAAAAAAALIDDAAAA|452|Elm 12th|Circle|Suite 190|Plainview|Bernalillo County|NM|83683|United States|-7|apartment| +13196|AAAAAAAAMIDDAAAA|941|North 15th|Cir.|Suite G|Kingston|Madison County|MO|64975|United States|-6|condo| +13197|AAAAAAAANIDDAAAA|741|Pine 2nd|Blvd|Suite 90|Five Forks|Franklin city|VA|22293|United States|-5|apartment| +13198|AAAAAAAAOIDDAAAA|195|Adams First|Pkwy|Suite I|Highland Park|Hennepin County|MN|56534|United States|-6|apartment| +13199|AAAAAAAAPIDDAAAA|740|Mill |Blvd|Suite P|Hillcrest|Carter County|MT|63003|United States|-7|condo| +13200|AAAAAAAAAJDDAAAA|366|Railroad |Ln|Suite Y|Ashley|Muskingum County|OH|44324|United States|-5|single family| +13201|AAAAAAAABJDDAAAA|888|Cedar Fifth|Avenue|Suite 420|Liberty|Citrus County|FL|33451|United States|-5|condo| +13202|AAAAAAAACJDDAAAA|438|Oak |Cir.|Suite Q|Pleasant Grove|Pierce County|GA|34136|United States|-5|single family| +13203|AAAAAAAADJDDAAAA|158|Park Highland|Court|Suite 400|Oakdale|Rockcastle County|KY|49584|United States|-5|single family| +13204|AAAAAAAAEJDDAAAA|854|South |Circle|Suite 130|Harmony|Suffolk County|MA|06404|United States|-5|condo| +13205|AAAAAAAAFJDDAAAA|526|Sunset 2nd|Ln|Suite 360|Newtown|Cumberland County|KY|41749|United States|-6|apartment| +13206|AAAAAAAAGJDDAAAA|791|East River|Cir.|Suite M|Lakeview|Coweta County|GA|38579|United States|-5|condo| +13207|AAAAAAAAHJDDAAAA|958|10th |Court|Suite 330|Oakland|Cherokee County|NC|29843|United States|-5|single family| +13208|AAAAAAAAIJDDAAAA|955|Hillcrest Lincoln|Way|Suite 300|Mount Vernon|Elliott County|KY|48482|United States|-6|single family| +13209|AAAAAAAAJJDDAAAA|336|North |Circle|Suite V|Summit|Essex County|NJ|01099|United States|-5|single family| +13210|AAAAAAAAKJDDAAAA|666|Railroad Lincoln|Cir.|Suite 440|Riverdale|Page County|IA|59391|United States|-6|single family| +13211|AAAAAAAALJDDAAAA|102|9th North|Road|Suite 430|Johnsonville|Montgomery County|MO|67745|United States|-6|apartment| +13212|AAAAAAAAMJDDAAAA|777|Mill Ridge|Street|Suite F|Salem|Johnson County|TX|78048|United States|-6|apartment| +13213|AAAAAAAANJDDAAAA|663|Maple Williams|Lane|Suite R|White Oak|Allegany County|MD|26668|United States|-5|apartment| +13214|AAAAAAAAOJDDAAAA|197|Highland 4th|Blvd|Suite 150|Kingston|Gogebic County|MI|44975|United States|-5|single family| +13215|AAAAAAAAPJDDAAAA|150|Maple |Road|Suite 190|New Hope|Montgomery County|AL|39431|United States|-6|condo| +13216|AAAAAAAAAKDDAAAA|988|5th |ST|Suite E|Lakeview|Wabash County|IL|68579|United States|-6|apartment| +13217|AAAAAAAABKDDAAAA|348|Dogwood |Wy|Suite 90|Pine Grove|Dent County|MO|64593|United States|-6|single family| +13218|AAAAAAAACKDDAAAA|890|1st |Pkwy|Suite 100|Belmont|Labette County|KS|60191|United States|-6|apartment| +13219|AAAAAAAADKDDAAAA|35|Lake |ST|Suite F|Deerfield|Canyon County|ID|89840|United States|-7|apartment| +13220|AAAAAAAAEKDDAAAA|642|Lincoln |Road|Suite N|Wilson|Adams County|OH|46971|United States|-5|condo| +13221|AAAAAAAAFKDDAAAA|670|Maple |Way|Suite 200|Union Hill|Montgomery County|AL|37746|United States|-6|apartment| +13222|AAAAAAAAGKDDAAAA|94|Spring |Pkwy|Suite R|Martinsville|Keweenaw County|MI|40419|United States|-5|condo| +13223|AAAAAAAAHKDDAAAA|462|South |Court|Suite 280|Union|Los Angeles County|CA|98721|United States|-8|single family| +13224|AAAAAAAAIKDDAAAA|109|Oak |Boulevard|Suite 80|Maple Grove|Yuma County|AZ|88252|United States|-7|apartment| +13225|AAAAAAAAJKDDAAAA|||||||MI||||apartment| +13226|AAAAAAAAKKDDAAAA|892|8th |Street|Suite H|White Oak|Chambers County|TX|76668|United States|-6|apartment| +13227|AAAAAAAALKDDAAAA|426|Lakeview 7th|Dr.|Suite 120|Newport|Carroll County|IL|61521|United States|-6|condo| +13228|AAAAAAAAMKDDAAAA|984|Pine Hickory|Ave|Suite 50|Conway|Harmon County|OK|74360|United States|-6|single family| +13229|AAAAAAAANKDDAAAA|143|East Pine|Cir.|Suite 300|Red Hill|Essex County|VT|04938|United States|-5|condo| +13230|AAAAAAAAOKDDAAAA|707|Hillcrest Chestnut|Way|Suite E|Clifton|Van Zandt County|TX|78014|United States|-6|condo| +13231|AAAAAAAAPKDDAAAA|984|Valley |Ave|Suite P|Oneida|Estill County|KY|44027|United States|-6|condo| +13232|AAAAAAAAALDDAAAA|200|Fourth |ST|Suite O|Farmington|Jefferson County|WA|99145|United States|-8|condo| +13233|AAAAAAAABLDDAAAA|333|View 8th|Dr.|Suite 270|Sunnyside|Simpson County|KY|41952|United States|-5|single family| +13234|AAAAAAAACLDDAAAA|172|15th Mill|Blvd|Suite O|Lakeview|Loudon County|TN|38579|United States|-6|condo| +13235|AAAAAAAADLDDAAAA|687|Walnut Elm|RD|Suite V|Marion|Breckinridge County|KY|40399|United States|-6|apartment| +13236|AAAAAAAAELDDAAAA|965|Park Second|Drive|Suite T|Kingston|Monroe County|WV|24975|United States|-5|single family| +13237|AAAAAAAAFLDDAAAA|683|View Sunset|Ct.|Suite V|Valley View|Jasper County|TX|75124|United States|-6|single family| +13238|AAAAAAAAGLDDAAAA|613|Pine |RD|Suite 60|Hopewell|Murray County|OK|70587|United States|-6|apartment| +13239|AAAAAAAAHLDDAAAA|550|Center |Street|Suite I|Wilson|Estill County|KY|46971|United States|-6|condo| +13240|AAAAAAAAILDDAAAA|300|Sycamore |Wy|Suite 330|Greenville|Shannon County|SD|51387|United States|-7|single family| +13241|AAAAAAAAJLDDAAAA|209|||Suite V|Wildwood|Ventura County|CA|96871|United States||apartment| +13242|AAAAAAAAKLDDAAAA|145||Circle|||San Juan County||80162|United States||apartment| +13243|AAAAAAAALLDDAAAA|956|Laurel Park|Blvd|Suite 420|Belmont|Colquitt County|GA|30191|United States|-5|single family| +13244|AAAAAAAAMLDDAAAA|865|Park Central|Dr.|Suite 10|New Hope|Renville County|ND|59431|United States|-6|condo| +13245|AAAAAAAANLDDAAAA|108|East |Circle|Suite U|Macedonia|Barton County|KS|61087|United States|-6|single family| +13246|AAAAAAAAOLDDAAAA|629|Park |Street|Suite E|Highland Park|Waushara County|WI|56534|United States|-6|apartment| +13247|AAAAAAAAPLDDAAAA|610|Eigth Third|Ln|Suite 400|Jamestown|Kendall County|TX|76867|United States|-6|single family| +13248|AAAAAAAAAMDDAAAA|912|8th Locust|Court|Suite M|Spring Hill|Greenup County|KY|46787|United States|-6|condo| +13249|AAAAAAAABMDDAAAA|250|Williams 12th|Parkway|Suite 10|Clifton|Hancock County|KY|48014|United States|-6|single family| +13250|AAAAAAAACMDDAAAA|262|Williams |RD|Suite H|Greenfield|Carbon County|PA|15038|United States|-5|condo| +13251|AAAAAAAADMDDAAAA|633|Valley Lincoln|ST|Suite 260|Edgewater|Warren County|GA|30635|United States|-5|apartment| +13252|AAAAAAAAEMDDAAAA|863|Park |Wy|Suite 120|Dewey|Adams County|WI|51160|United States|-6|apartment| +13253|AAAAAAAAFMDDAAAA|197|East |Dr.|Suite K|Woodrow|Davie County|NC|24273|United States|-5|apartment| +13254|AAAAAAAAGMDDAAAA|910|13th |RD|Suite 180|Salem|Catahoula Parish|LA|78048|United States|-6|single family| +13255|AAAAAAAAHMDDAAAA|649|Second |Circle|Suite E|Wildwood|Petersburg city|VA|26871|United States|-5|condo| +13256|AAAAAAAAIMDDAAAA|17|Davis Walnut|Avenue|Suite M|Summit|Mitchell County|GA|30499|United States|-5|apartment| +13257|AAAAAAAAJMDDAAAA|361|Hillcrest Park|Ave|Suite J|Seneca|Winston County|MS|59726|United States|-6|single family| +13258|AAAAAAAAKMDDAAAA|465|View Pine|Road|Suite W|Antioch|Atchison County|KS|68605|United States|-6|single family| +13259|AAAAAAAALMDDAAAA|800|Jefferson |Way|Suite 200|Hamilton|Windham County|VT|03408|United States|-5|single family| +13260|AAAAAAAAMMDDAAAA|609|11th |Parkway|Suite G|Macedonia|Lewis County|WV|21087|United States|-5|apartment| +13261|AAAAAAAANMDDAAAA|905|Green Oak|Boulevard|Suite 110|Vista|Hancock County|GA|34694|United States|-5|condo| +13262|AAAAAAAAOMDDAAAA|913|11th |Ave|Suite 270|Lakeside|Fairfax County|VA|29532|United States|-5|single family| +13263|AAAAAAAAPMDDAAAA|928|2nd Highland|Court|Suite Q|Green Acres|Swisher County|TX|77683|United States|-6|single family| +13264|AAAAAAAAANDDAAAA|366|Dogwood |ST|Suite 30|Pleasant Hill|Issaquena County|MS|53604|United States|-6|condo| +13265|AAAAAAAABNDDAAAA|387|Valley 7th|Circle|Suite 160|Stringtown|Shiawassee County|MI|40162|United States|-5|apartment| +13266|AAAAAAAACNDDAAAA|961|Fifteenth |Wy|Suite 430|Quincy|Ohio County|IN|43868|United States|-5|condo| +13267|AAAAAAAADNDDAAAA|359|Maple |RD|Suite 260|Mount Pleasant|Webster County|WV|21933|United States|-5|apartment| +13268|AAAAAAAAENDDAAAA|975|East |||||||United States||| +13269|AAAAAAAAFNDDAAAA|636|Davis Maple|RD|Suite 300|Mount Zion|Cerro Gordo County|IA|58054|United States|-6|condo| +13270|AAAAAAAAGNDDAAAA|634|Park 4th|Wy|Suite 320|Greenville|Yukon-Koyukuk Census Area|AK|91387|United States|-9|condo| +13271|AAAAAAAAHNDDAAAA|698|Broadway 12th|Road|Suite 0|Maple Grove|Franklin County|VT|08852|United States|-5|apartment| +13272|AAAAAAAAINDDAAAA|163|Valley Sunset|Ln|Suite 210|Kingston|Goodhue County|MN|54975|United States|-6|apartment| +13273|AAAAAAAAJNDDAAAA|457|Main Highland|Ave|Suite 210|Belmont|Goodhue County|MN|50191|United States|-6|single family| +13274|AAAAAAAAKNDDAAAA|321|Pine Locust|Lane|Suite 90|Oak Grove|Clark County|AR|78370|United States|-6|condo| +13275|AAAAAAAALNDDAAAA|213|Pine 1st|ST|Suite B|Allentown|Missaukee County|MI|41838|United States|-5|condo| +13276|AAAAAAAAMNDDAAAA|522|Church 5th|ST|Suite N|Kingston|Muskegon County|MI|44975|United States|-5|single family| +13277|AAAAAAAANNDDAAAA|902|Cedar |Court|Suite A|Oak Ridge|Lincoln County|WI|58371|United States|-6|apartment| +13278|AAAAAAAAONDDAAAA|942|West |Ave|Suite 450|Clifford|Gilmer County|WV|28164|United States|-5|apartment| +13279|AAAAAAAAPNDDAAAA|167|Maple |Ln|Suite N|Friendship|Colusa County|CA|94536|United States|-8|condo| +13280|AAAAAAAAAODDAAAA|480|Oak |RD|Suite J|Riverdale|De Witt County|IL|69391|United States|-6|single family| +13281|AAAAAAAABODDAAAA|986|Third |Drive|Suite 150|Shiloh|Houston County|MN|59275|United States|-6|apartment| +13282|AAAAAAAACODDAAAA|924|Washington |Pkwy|Suite 450|Sulphur Springs|Boyd County|NE|68354|United States|-6|apartment| +13283|AAAAAAAADODDAAAA|951|2nd Woodland|Circle|Suite 0|Kingston|Monroe County|AL|34975|United States|-6|single family| +13284|AAAAAAAAEODDAAAA|466|3rd |Way|Suite 480|Stringtown|Blaine County|NE|60162|United States|-6|single family| +13285|AAAAAAAAFODDAAAA|800|Second |Avenue|Suite O|Pleasant Grove|Madison County|MT|64136|United States|-7|single family| +13286|AAAAAAAAGODDAAAA|783|Lee |Parkway|Suite 50|Ruth|Wayne County|GA|30309|United States|-5|condo| +13287|AAAAAAAAHODDAAAA|862|11th |Avenue|Suite 190|Chestnut Ridge|Jones County|NC|27334|United States|-5|single family| +13288|AAAAAAAAIODDAAAA|621|Johnson Miller|Drive|Suite 440|Waterloo|Beltrami County|MN|51675|United States|-6|condo| +13289|AAAAAAAAJODDAAAA|813|Center |Way|Suite H|Providence|Grant County|KS|66614|United States|-6|condo| +13290|AAAAAAAAKODDAAAA|469|Walnut |Avenue|Suite 460|Macedonia|Monroe County|TN|31087|United States|-6|condo| +13291|AAAAAAAALODDAAAA|541|Park |Pkwy|Suite 430|Woodlawn|Marquette County|WI|54098|United States|-6|apartment| +13292|AAAAAAAAMODDAAAA|148|Locust |Blvd|Suite X|Oakland|Ransom County|ND|59843|United States|-6|condo| +13293|AAAAAAAANODDAAAA|701|Elm Oak|Ln|Suite Q|Cedar Grove|Bingham County|ID|80411|United States|-7|condo| +13294|AAAAAAAAOODDAAAA|35|View 2nd|Way|Suite M|Newtown|San Joaquin County|CA|91749|United States|-8|condo| +13295|AAAAAAAAPODDAAAA|261|Main Oak|Way|Suite 190|Woodville|Nantucket County|MA|04889|United States|-5|single family| +13296|AAAAAAAAAPDDAAAA|921|Railroad Laurel|Pkwy|Suite 110|Bunker Hill|O-Brien County|IA|50150|United States|-6|condo| +13297|AAAAAAAABPDDAAAA|37|Williams |Wy|Suite 430|Belmont|Wyoming County|PA|10191|United States|-5|single family| +13298|AAAAAAAACPDDAAAA|830|2nd 9th|Avenue|Suite 70|Five Points|Los Angeles County|CA|96098|United States|-8|apartment| +13299|AAAAAAAADPDDAAAA|90|View Broadway|Court|Suite 0|Brownsville|Walton County|GA|39310|United States|-5|apartment| +13300|AAAAAAAAEPDDAAAA|250|North Highland|Ct.|Suite 340|Bunker Hill|Calloway County|KY|40150|United States|-6|single family| +13301|AAAAAAAAFPDDAAAA|397|Meadow |Blvd|Suite 60|Bethel|York County|PA|15281|United States|-5|single family| +13302|AAAAAAAAGPDDAAAA|447|Sunset Cherry|Court|Suite U|White Oak|Scott County|MN|56668|United States|-6|single family| +13303|AAAAAAAAHPDDAAAA|876|Sixth |RD|Suite 160|Spring Hill|Franklin Parish|LA|76787|United States|-6|apartment| +13304|AAAAAAAAIPDDAAAA|160|Fourth |Circle|Suite P|Four Points|Liberty County|TX|71216|United States|-6|single family| +13305|AAAAAAAAJPDDAAAA|278|7th Miller|Ave|Suite 150|Red Hill|Huerfano County|CO|84338|United States|-7|single family| +13306|AAAAAAAAKPDDAAAA|832|Hill |Pkwy|Suite Y|Wildwood|Wheatland County|MT|66871|United States|-7|apartment| +13307|AAAAAAAALPDDAAAA|409|Hill |Blvd|Suite X|Deerfield|Kauai County|HI|99840|United States|-10|condo| +13308|AAAAAAAAMPDDAAAA|309|Ninth |Boulevard|Suite 320|Union|Banner County|NE|68721|United States|-6|single family| +13309|AAAAAAAANPDDAAAA|5|East |Parkway|Suite R|Green Acres|Pinellas County|FL|37683|United States|-5|single family| +13310|AAAAAAAAOPDDAAAA|637|West Maple|Drive|Suite E|Union|Campbell County|VA|28721|United States|-5|condo| +13311|AAAAAAAAPPDDAAAA|779|Fourth |Circle|Suite J|Floyd|Mississippi County|MO|63235|United States|-6|apartment| +13312|AAAAAAAAAAEDAAAA|226|Elm Railroad|RD|Suite G|Pine Grove|Fluvanna County|VA|24593|United States|-5|apartment| +13313|AAAAAAAABAEDAAAA|616|Railroad |Avenue|Suite W|Harmon|Smith County|TN|35623|United States|-6|condo| +13314|AAAAAAAACAEDAAAA|652|7th |Boulevard|Suite 260|Tracy|Winona County|MN|56340|United States|-6|condo| +13315|AAAAAAAADAEDAAAA|748|Walnut Broadway|Street|Suite W|Centerville|Wilkes County|GA|30059|United States|-5|single family| +13316|AAAAAAAAEAEDAAAA|484|Jefferson |Ln|Suite 130|Fayetteville|Durham County|NC|21732|United States|-5|condo| +13317|AAAAAAAAFAEDAAAA|104|12th Highland|Wy|Suite H|Watkins|Johnson County|KS|61732|United States|-6|condo| +13318|AAAAAAAAGAEDAAAA|557|Lee Williams|Blvd|Suite 400|Sunnyside|Meade County|SD|51952|United States|-7|single family| +13319|AAAAAAAAHAEDAAAA|116||||||KS|||-6|| +13320|AAAAAAAAIAEDAAAA|521|Wilson |Blvd|Suite 150|Salem|Gem County|ID|88048|United States|-7|apartment| +13321|AAAAAAAAJAEDAAAA|754|Pine First|Way|Suite S|Bethel|Otter Tail County|MN|55281|United States|-6|condo| +13322|AAAAAAAAKAEDAAAA|888|View |Ln|Suite W|Oak Ridge|Fremont County|IA|58371|United States|-6|apartment| +13323|AAAAAAAALAEDAAAA|771|Walnut Pine|Avenue|Suite 480|Aberdeen|Aiken County|SC|27995|United States|-5|single family| +13324|AAAAAAAAMAEDAAAA|409|Elm West|Ave|Suite M|Lakeview|Jefferson County|OH|48579|United States|-5|apartment| +13325|AAAAAAAANAEDAAAA|365|Wilson |Boulevard|Suite 190|Glenwood|Mahaska County|IA|53511|United States|-6|condo| +13326|AAAAAAAAOAEDAAAA|735|9th Hill|Cir.|Suite 460|Mount Olive|Lee County|AL|38059|United States|-6|condo| +13327|AAAAAAAAPAEDAAAA|886|River Birch|Road|Suite 440|Five Points|Anderson County|TN|36098|United States|-5|condo| +13328|AAAAAAAAABEDAAAA|213|Willow |Pkwy|Suite 260|Newport|Boyd County|NE|61521|United States|-6|apartment| +13329|AAAAAAAABBEDAAAA|541|First Third|Cir.|Suite W|Lakeside|Laclede County|MO|69532|United States|-6|apartment| +13330|AAAAAAAACBEDAAAA|627|Dogwood |Dr.|Suite 400|Fairview|Alleghany County|NC|25709|United States|-5|apartment| +13331|AAAAAAAADBEDAAAA|318|Ash |Blvd|Suite 120|Cumberland|Noble County|OK|78971|United States|-6|apartment| +13332|AAAAAAAAEBEDAAAA|804|Woodland |RD|Suite 60|Mount Zion|Clay County|AL|38054|United States|-6|condo| +13333|AAAAAAAAFBEDAAAA|494|4th 3rd|Cir.|Suite T|Wilson|Avoyelles Parish|LA|76971|United States|-6|condo| +13334|AAAAAAAAGBEDAAAA|200|4th |Boulevard|Suite 90|Oak Hill|Richmond County|NC|27838|United States|-5|apartment| +13335|AAAAAAAAHBEDAAAA|232|Main Railroad|Circle|Suite A|Pine Grove|Deer Lodge County|MT|64593|United States|-7|single family| +13336|AAAAAAAAIBEDAAAA|444|View |Boulevard|Suite N|Riverside|Falls County|TX|79231|United States|-6|single family| +13337|AAAAAAAAJBEDAAAA|247|Washington View|Court|Suite 90|Cedar Grove|Peach County|GA|30411|United States|-5|single family| +13338|AAAAAAAAKBEDAAAA|645|First Spring|Blvd|Suite L|White Oak|Calhoun County|SC|26668|United States|-5|single family| +13339|AAAAAAAALBEDAAAA||||Suite 20|Fairview|Cherokee County||35709|||single family| +13340|AAAAAAAAMBEDAAAA|991|Hillcrest River|Cir.|Suite B|Greenfield|Bon Homme County|SD|55038|United States|-6|single family| +13341|AAAAAAAANBEDAAAA|109|Smith |Lane|Suite I|Pleasant Valley|Willacy County|TX|72477|United States|-6|single family| +13342|AAAAAAAAOBEDAAAA|915|Meadow |Circle|Suite 250|Macedonia|Wilkinson County|MS|51087|United States|-6|condo| +13343|AAAAAAAAPBEDAAAA|159|Second Williams|Cir.|Suite T|Pleasant Valley|Castro County|TX|72477|United States|-6|condo| +13344|AAAAAAAAACEDAAAA|537|Hill 2nd|Avenue|Suite 490|Leland|Calhoun County|MS|59452|United States|-6|single family| +13345|AAAAAAAABCEDAAAA|786|Park First|Avenue|Suite 360|Oakdale|Stevens County|MN|59584|United States|-6|single family| +13346|AAAAAAAACCEDAAAA|732|Walnut |Parkway|Suite Y|Spring Hill|Polk County|TX|76787|United States|-6|condo| +13347|AAAAAAAADCEDAAAA|460|Chestnut |Pkwy|Suite H|Crossroads|Cortland County|NY|10534|United States|-5|condo| +13348|AAAAAAAAECEDAAAA|824|15th 5th|Street|Suite 120|Wildwood|Sutter County|CA|96871|United States|-8|apartment| +13349|AAAAAAAAFCEDAAAA|104|Lake Main|Drive|Suite T|Union Hill|Florence County|SC|27746|United States|-5|single family| +13350|AAAAAAAAGCEDAAAA|703|5th |ST|Suite 180|Georgetown|Kaufman County|TX|77057|United States|-6|condo| +13351|AAAAAAAAHCEDAAAA|353|Lee |Lane|Suite B|New Hope|Jewell County|KS|69431|United States|-6|single family| +13352|AAAAAAAAICEDAAAA|621|Lee Ridge|Lane|Suite 370|Shady Grove|Lewis County|MO|62812|United States|-6|condo| +13353|AAAAAAAAJCEDAAAA|66|Jackson 2nd|Boulevard|Suite 240|Riverside|Yakutat Borough|AK|99231|United States|-9|apartment| +13354|AAAAAAAAKCEDAAAA|532|Park |Ave|Suite W|Newport|Covington County|AL|31521|United States|-6|apartment| +13355|AAAAAAAALCEDAAAA|310|Main Fifth|Parkway|Suite Q|Yorktown|Alger County|MI|40732|United States|-5|single family| +13356|AAAAAAAAMCEDAAAA|446|Chestnut |Ave|Suite S|Pleasant Grove|Shelby County|IA|54136|United States|-6|single family| +13357|AAAAAAAANCEDAAAA|808|Wilson 3rd|Road|Suite W|Concord|Clarke County|IA|54107|United States|-6|single family| +13358|AAAAAAAAOCEDAAAA|91|Woodland |Pkwy|Suite 430|Concord|Martin County|MN|54107|United States|-6|condo| +13359|AAAAAAAAPCEDAAAA|121|Jefferson Hill|Blvd|Suite S|Post Oak|Navajo County|AZ|88567|United States|-7|apartment| +13360|AAAAAAAAADEDAAAA|510|Hickory Third|Way|Suite 320|Macedonia|Sequatchie County|TN|31087|United States|-6|apartment| +13361|AAAAAAAABDEDAAAA|677|3rd Cedar|Way|Suite T|Pleasant Grove|Natrona County|WY|84136|United States|-7|apartment| +13362|AAAAAAAACDEDAAAA|45|Center |Blvd|Suite S|Edgewood|Jackson County|IL|60069|United States|-6|condo| +13363|AAAAAAAADDEDAAAA|534|1st Davis|Pkwy|Suite S|Florence|Martin County|NC|23394|United States|-5|single family| +13364|AAAAAAAAEDEDAAAA|245|Jackson 5th|Court|Suite L|Elba|Hill County|TX|70262|United States|-6|single family| +13365|AAAAAAAAFDEDAAAA|276|6th |Parkway|Suite T|Crossroads|Barry County|MI|40534|United States|-5|single family| +13366|AAAAAAAAGDEDAAAA|249|Green Chestnut|Lane|Suite 370|Westgate|Ottawa County|OK|72366|United States|-6|condo| +13367|AAAAAAAAHDEDAAAA|278|Highland |Boulevard|Suite L|Greenfield|Dutchess County|NY|15038|United States|-5|apartment| +13368|AAAAAAAAIDEDAAAA|244|2nd |||Walnut Grove|Suffolk County|NY|||-5|| +13369|AAAAAAAAJDEDAAAA|130|1st |Dr.|Suite A|Florence|Fayette County|IA|53394|United States|-6|apartment| +13370|AAAAAAAAKDEDAAAA|491|Meadow Madison|Dr.|Suite 60|Mansfield|Phelps County|NE|68430|United States|-7|condo| +13371|AAAAAAAALDEDAAAA|284|Washington Spruce|Street|Suite 430|Harmony|Moore County|TX|75804|United States|-6|condo| +13372|AAAAAAAAMDEDAAAA|474|Jefferson South|Blvd|Suite 130|Forest Hills|Allen County|KS|69237|United States|-6|condo| +13373|AAAAAAAANDEDAAAA|249|Oak |Circle|Suite 70|Mount Vernon|San Patricio County|TX|78482|United States|-6|single family| +13374|AAAAAAAAODEDAAAA|278|Oak |Blvd|Suite A|Mount Pleasant|Kent County|DE|11933|United States|-5|single family| +13375|AAAAAAAAPDEDAAAA|640|Broadway 10th|Drive|Suite 70|Buena Vista|Fulton County|NY|15752|United States|-5|single family| +13376|AAAAAAAAAEEDAAAA|661|13th Laurel|RD|Suite 330|Shiloh|Echols County|GA|39275|United States|-5|condo| +13377|AAAAAAAABEEDAAAA|441|5th Green|Ave|Suite X|Maple Grove|Park County|CO|88252|United States|-7|condo| +13378|AAAAAAAACEEDAAAA|631|Walnut |Dr.|Suite 30|Fairview|Marshall County|KS|65709|United States|-6|apartment| +13379|AAAAAAAADEEDAAAA|29|Main |RD|Suite E|White Oak|Travis County|TX|76668|United States|-6|apartment| +13380|AAAAAAAAEEEDAAAA|497|Williams |Ct.|Suite 20|Spring Valley|Holmes County|FL|36060|United States|-5|condo| +13381|AAAAAAAAFEEDAAAA||Davis Elm|||||||United States|-6|single family| +13382|AAAAAAAAGEEDAAAA|949|Sycamore |Blvd|Suite L|Roscoe|Lake County|OR|91854|United States|-8|single family| +13383|AAAAAAAAHEEDAAAA|296|Hill Lake|Pkwy|Suite 270|Five Points|Robertson County|TN|36098|United States|-6|apartment| +13384|AAAAAAAAIEEDAAAA|561|Davis Washington|Pkwy|Suite 370|Woodlawn|Manassas Park city|VA|24098|United States|-5|condo| +13385|AAAAAAAAJEEDAAAA|492|Hill 4th|Court|Suite 440|Riverdale|Pawnee County|OK|79391|United States|-6|apartment| +13386|AAAAAAAAKEEDAAAA|169|Cedar Cherry|Road|Suite 180|Lincoln|Nottoway County|VA|21289|United States|-5|single family| +13387|AAAAAAAALEEDAAAA|598|Mill |Pkwy|Suite X|Brownsville|Fairfax city|VA|29310|United States|-5|apartment| +13388|AAAAAAAAMEEDAAAA|435|Birch |RD|Suite 40|Shiloh|Gloucester County|NJ|09875|United States|-5|condo| +13389|AAAAAAAANEEDAAAA|11|6th Church|Wy|Suite P|Harmony|Val Verde County|TX|75804|United States|-6|apartment| +13390|AAAAAAAAOEEDAAAA|812|Central Maple|Ct.|Suite 370|Springdale|Halifax County|NC|28883|United States|-5|single family| +13391|AAAAAAAAPEEDAAAA|708|4th Ninth|Wy|Suite Y|Lincoln|Nueces County|TX|71289|United States|-6|condo| +13392|AAAAAAAAAFEDAAAA|690|Ridge |Wy|Suite 200|Maple Grove|Dodge County|WI|58252|United States|-6|apartment| +13393|AAAAAAAABFEDAAAA|478|Oak |RD|Suite F|Salem|Lemhi County|ID|88048|United States|-7|single family| +13394|AAAAAAAACFEDAAAA|591|Chestnut Fifth|Lane|Suite 450|Bethel|Brown County|SD|55281|United States|-6|apartment| +13395|AAAAAAAADFEDAAAA|326|Poplar |Ct.|Suite J|Franklin|Merrimack County|NH|09701|United States|-5|condo| +13396|AAAAAAAAEFEDAAAA|973|Elm Park|ST|Suite 400|Harmony|Fulton County|OH|45804|United States|-5|condo| +13397|AAAAAAAAFFEDAAAA|276|Third |Blvd|Suite U|Friendship|Texas County|MO|64536|United States|-6|apartment| +13398|AAAAAAAAGFEDAAAA|431|8th 5th|Blvd|Suite 140|Mount Vernon|Wyandot County|OH|48482|United States|-5|apartment| +13399|AAAAAAAAHFEDAAAA|701|Adams |RD|Suite 470|Sunnyside|Prince William County|VA|21952|United States|-5|condo| +13400|AAAAAAAAIFEDAAAA|647|Ash |Way|Suite 430|Clinton|Osceola County|IA|58222|United States|-6|single family| +13401|AAAAAAAAJFEDAAAA|29|View 9th|Cir.|Suite 160|Franklin|Mason County|WV|29101|United States|-5|single family| +13402|AAAAAAAAKFEDAAAA|822|Franklin |Avenue|Suite 170|Lakeview|Monterey County|CA|98579|United States|-8|apartment| +13403|AAAAAAAALFEDAAAA|305|Third Spring|Ct.|Suite 210|Woodville|Polk County|AR|74289|United States|-6|single family| +13404|AAAAAAAAMFEDAAAA||15th |Blvd|Suite 410|Riverview|||||-6|| +13405|AAAAAAAANFEDAAAA|191|Pine |Cir.|Suite U|Shore Acres|Jackson Parish|LA|72724|United States|-6|condo| +13406|AAAAAAAAOFEDAAAA|660|Birch |Way|Suite S|Hopewell|Lincoln County|NC|20587|United States|-5|single family| +13407|AAAAAAAAPFEDAAAA|979|1st Park|Dr.|Suite Q|Hopewell|Minidoka County|ID|80587|United States|-7|apartment| +13408|AAAAAAAAAGEDAAAA|798|Elm Davis|Parkway|Suite 260|Salem|O-Brien County|IA|58048|United States|-6|condo| +13409|AAAAAAAABGEDAAAA|387|Green Cherry|RD|Suite A|Florence|Spencer County|KY|43394|United States|-5|condo| +13410|AAAAAAAACGEDAAAA|529|Adams |ST|Suite X|Unionville|Carson County|TX|71711|United States|-6|single family| +13411|AAAAAAAADGEDAAAA|212|Davis |Street|Suite 130|Macedonia|Delaware County|NY|11087|United States|-5|apartment| +13412|AAAAAAAAEGEDAAAA|657|||Suite Q|Springfield|Tippah County||||-6|single family| +13413|AAAAAAAAFGEDAAAA|372|14th |Pkwy|Suite Y|Rossville|Douglas County|KS|62503|United States|-6|condo| +13414|AAAAAAAAGGEDAAAA|43|Park 8th|Dr.|Suite W|Pleasant Valley|Lee County|KY|42477|United States|-5|condo| +13415|AAAAAAAAHGEDAAAA|392|Laurel 7th|Parkway|Suite 140|Sunnyside|Hudson County|NJ|02552|United States|-5|condo| +13416|AAAAAAAAIGEDAAAA|||Road|||Williams County||48059|||| +13417|AAAAAAAAJGEDAAAA|953|View |Drive|Suite N|Brownsville|Lincoln County|WI|59310|United States|-6|condo| +13418|AAAAAAAAKGEDAAAA|937|Smith Washington|Road|Suite 410|New Hope|Otsego County|NY|19431|United States|-5|apartment| +13419|AAAAAAAALGEDAAAA|534|Mill 4th|Drive|Suite 50|Bradford|Hamlin County|SD|51184|United States|-7|condo| +13420|AAAAAAAAMGEDAAAA|174|Locust Railroad|Ave|Suite 290|Marion|Pickens County|SC|20399|United States|-5|apartment| +13421|AAAAAAAANGEDAAAA|14|Johnson |RD|Suite V|Unionville|Marengo County|AL|31711|United States|-6|condo| +13422|AAAAAAAAOGEDAAAA|53|Park |Street|Suite 40|Edgewood|Paulding County|OH|40069|United States|-5|single family| +13423|AAAAAAAAPGEDAAAA|574|View |Street|Suite V|Woodland|Ouachita County|AR|74854|United States|-6|condo| +13424|AAAAAAAAAHEDAAAA||South Chestnut|Ct.|Suite 120|Walnut Grove|Carter County|KY|47752|United States||apartment| +13425|AAAAAAAABHEDAAAA|304|Elm Seventh|Blvd|Suite H|Summit|Scott County|VA|20499|United States|-5|apartment| +13426|AAAAAAAACHEDAAAA|682|West |Avenue|Suite 390|Liberty|Kearney County|NE|63451|United States|-7|apartment| +13427|AAAAAAAADHEDAAAA|755|Johnson |Boulevard|Suite P|Newport|Polk County|MN|51521|United States|-6|apartment| +13428|AAAAAAAAEHEDAAAA|138|Maple |Blvd|Suite 60|Arthur|Fleming County|KY|45965|United States|-6|single family| +13429|AAAAAAAAFHEDAAAA||Pine Meadow|Street|Suite 270||DeSoto County|||||apartment| +13430|AAAAAAAAGHEDAAAA|378|View Chestnut|RD|Suite 240|Jamestown|Campbell County|SD|56867|United States|-6|apartment| +13431|AAAAAAAAHHEDAAAA|811|View |Way|Suite 50|Five Forks|Tipton County|TN|32293|United States|-6|apartment| +13432|AAAAAAAAIHEDAAAA|115|Spring Jackson|Road|Suite 40|Lakeside|DeKalb County|GA|39532|United States|-5|single family| +13433|AAAAAAAAJHEDAAAA|77|7th College|ST|Suite D|Unionville|Brazos County|TX|71711|United States|-6|single family| +13434|AAAAAAAAKHEDAAAA|281|1st Park|Pkwy|Suite Q|Lone Pine|Hidalgo County|TX|77441|United States|-6|apartment| +13435|AAAAAAAALHEDAAAA|214|Birch |RD|Suite F|Farmington|Prince Edward County|VA|29145|United States|-5|single family| +13436|AAAAAAAAMHEDAAAA|998|12th |ST|Suite 150|Liberty|Fayette County|WV|23451|United States|-5|single family| +13437|AAAAAAAANHEDAAAA|989|Maple Mill|Cir.|Suite 160|Greenfield|Belknap County|NH|05638|United States|-5|apartment| +13438|AAAAAAAAOHEDAAAA|953|North |Court|Suite 370|Mount Pleasant|Deer Lodge County|MT|61933|United States|-7|single family| +13439|AAAAAAAAPHEDAAAA|149|Hickory |Pkwy|Suite P|Valley View|Aransas County|TX|75124|United States|-6|apartment| +13440|AAAAAAAAAIEDAAAA|240|Elm Park|Wy|Suite L|Glendale|Greene County|AL|33951|United States|-6|apartment| +13441|AAAAAAAABIEDAAAA|880|West |RD|Suite B|Unionville|Erath County|TX|71711|United States|-6|apartment| +13442|AAAAAAAACIEDAAAA|440|Birch Walnut|Way|Suite 140|Deerfield|Stillwater County|MT|69840|United States|-7|single family| +13443|AAAAAAAADIEDAAAA|587|Sixth Spring|Lane|Suite 250|Marion|Clinton County|OH|40399|United States|-5|apartment| +13444|AAAAAAAAEIEDAAAA|345|Smith |Blvd|Suite 290|Springfield|Montague County|TX|79303|United States|-6|apartment| +13445|AAAAAAAAFIEDAAAA|463|East Lincoln|Wy|Suite 30|Five Points|Rosebud County|MT|66098|United States|-7|apartment| +13446|AAAAAAAAGIEDAAAA|345|Sixth Maple|Street|Suite 210|Midway|Poquoson city|VA|21904|United States|-5|single family| +13447|AAAAAAAAHIEDAAAA|71|1st 2nd|Ave|Suite G|Woodland|Lancaster County|NE|64854|United States|-7|condo| +13448|AAAAAAAAIIEDAAAA|2|8th 3rd|Avenue|Suite F|Spring Hill|Trimble County|KY|46787|United States|-5|single family| +13449|AAAAAAAAJIEDAAAA|445|Lincoln |Dr.|Suite D|Deerfield|Bourbon County|KS|69840|United States|-6|single family| +13450|AAAAAAAAKIEDAAAA|530|Elevnth |Circle|Suite 160|Kirkland|Garrett County|MD|27896|United States|-5|condo| +13451|AAAAAAAALIEDAAAA|722|First |Way|Suite G|Philadelphia|Oswego County|NY|15591|United States|-5|condo| +13452|AAAAAAAAMIEDAAAA|10|4th |Cir.|Suite N|Red Hill|Harrison County|MS|54338|United States|-6|single family| +13453|AAAAAAAANIEDAAAA|266|Dogwood |Blvd|Suite F|Royal|Fisher County|TX|75819|United States|-6|condo| +13454|AAAAAAAAOIEDAAAA|876|Sunset |Drive|Suite 290|Oakwood|Angelina County|TX|70169|United States|-6|condo| +13455|AAAAAAAAPIEDAAAA|889|Meadow 11th|ST|Suite 100|Pleasant Valley|Montgomery County|IA|52477|United States|-6|apartment| +13456|AAAAAAAAAJEDAAAA|606|Wilson |Blvd|Suite 170|Hillcrest|Essex County|NY|13003|United States|-5|apartment| +13457|AAAAAAAABJEDAAAA|670|6th Maple|Lane|Suite O|Mountain View|Russell County|KS|64466|United States|-6|apartment| +13458|AAAAAAAACJEDAAAA|250|Park Hickory|Pkwy|Suite R|Brownsville|Madison County|IN|49310|United States|-5|apartment| +13459|AAAAAAAADJEDAAAA|491||Circle|Suite N|Wayland|Washington County|NY|15115|||apartment| +13460|AAAAAAAAEJEDAAAA|529|Lakeview 15th|Pkwy|Suite 430|Superior|McLean County|IL|62562|United States|-6|condo| +13461|AAAAAAAAFJEDAAAA|638|Lake Elm|Ct.|Suite 260|Mount Olive|New York County|NY|18059|United States|-5|apartment| +13462|AAAAAAAAGJEDAAAA|199|6th |Parkway|Suite S|Lincoln|Forest County|PA|11289|United States|-5|single family| +13463|AAAAAAAAHJEDAAAA|387|Church Valley|Parkway|Suite 460|Enterprise|Larimer County|CO|81757|United States|-7|condo| +13464|AAAAAAAAIJEDAAAA|892|8th Lake|Ln|Suite V|White Oak|Rice County|MN|56668|United States|-6|apartment| +13465|AAAAAAAAJJEDAAAA|921|Broadway Franklin|Wy|Suite 70|Shady Grove|Daviess County|MO|62812|United States|-6|apartment| +13466|AAAAAAAAKJEDAAAA|32|Main |Ln|Suite W|Liberty|Appomattox County|VA|23451|United States|-5|condo| +13467|AAAAAAAALJEDAAAA|476|Railroad Franklin|Street|Suite X|Plainview|Breckinridge County|KY|43683|United States|-6|single family| +13468|AAAAAAAAMJEDAAAA|393|Birch |Blvd|Suite T|Macedonia|Stanly County|NC|21087|United States|-5|condo| +13469|AAAAAAAANJEDAAAA|1|Spring |Avenue|Suite 100|Highland Park|Noble County|IN|46534|United States|-5|single family| +13470|AAAAAAAAOJEDAAAA|119|Eigth |Dr.|Suite W|Fairview|Wilkes County|GA|35709|United States|-5|single family| +13471|AAAAAAAAPJEDAAAA||Second Washington|Wy||Elkton||WV|||-5|apartment| +13472|AAAAAAAAAKEDAAAA|944|Lakeview |Road|Suite 40|Wildwood|Manitowoc County|WI|56871|United States|-6|single family| +13473|AAAAAAAABKEDAAAA|732|View |Ct.|Suite S|Woodland|Bayfield County|WI|54854|United States|-6|single family| +13474|AAAAAAAACKEDAAAA|186|Main Spring|ST|Suite I|Centerville|Wood County|TX|70059|United States|-6|condo| +13475|AAAAAAAADKEDAAAA|199|Ninth Ridge|Ct.|Suite 140|Valley View|Benton County|MS|55124|United States|-6|single family| +13476|AAAAAAAAEKEDAAAA|212|Wilson |Circle|Suite 320|Edgewood|Switzerland County|IN|40069|United States|-5|single family| +13477|AAAAAAAAFKEDAAAA|597|Central Center|Way|Suite N|Greenville|Mercer County|WV|21387|United States|-5|condo| +13478|AAAAAAAAGKEDAAAA|456|5th South|RD|Suite 30|Jamestown|Suwannee County|FL|36867|United States|-5|condo| +13479|AAAAAAAAHKEDAAAA|435|Ash |Boulevard|Suite 20|Centerville|Polk County|MN|50059|United States|-6|single family| +13480|AAAAAAAAIKEDAAAA|27|Sunset Poplar|Street|Suite G|Oak Grove|Wilbarger County|TX|78370|United States|-6|apartment| +13481|AAAAAAAAJKEDAAAA|406|1st |RD|Suite 60|Springdale|Lincoln County|TN|38883|United States|-6|apartment| +13482|AAAAAAAAKKEDAAAA|121|12th 7th|Wy|Suite L|Bridgeport|Washakie County|WY|85817|United States|-7|single family| +13483|AAAAAAAALKEDAAAA|644|Johnson Spring|Wy|Suite I|Five Points|Stanton County|NE|66098|United States|-7|condo| +13484|AAAAAAAAMKEDAAAA|419|Davis Center|Ln|Suite 320|Ashley|White County|TN|34324|United States|-6|condo| +13485|AAAAAAAANKEDAAAA|611|6th |Boulevard|Suite A|Lincoln|Newton County|AR|71289|United States|-6|condo| +13486|AAAAAAAAOKEDAAAA|926|Meadow Green|Court|Suite J|Concord|Coryell County|TX|74107|United States|-6|apartment| +13487|AAAAAAAAPKEDAAAA|779|Center |RD|Suite L|Riverside|Oneida County|WI|59231|United States|-6|single family| +13488|AAAAAAAAALEDAAAA|291|Washington |Lane|Suite 350|Hopewell|Leelanau County|MI|40587|United States|-5|single family| +13489|AAAAAAAABLEDAAAA|630|11th |Street|Suite O|Edgewood|Williamson County|IL|60069|United States|-6|apartment| +13490|AAAAAAAACLEDAAAA|565|Davis ||Suite 170||Clinch County|||United States||| +13491|AAAAAAAADLEDAAAA|67|Center |RD|Suite 280|Newport|Elmore County|AL|31521|United States|-6|single family| +13492|AAAAAAAAELEDAAAA|351|Forest Park|Blvd|Suite N|Newport|Mineral County|MT|61521|United States|-7|single family| +13493|AAAAAAAAFLEDAAAA|637|Washington 4th|ST|Suite 40|Forest Hills|Crook County|WY|89237|United States|-7|apartment| +13494|AAAAAAAAGLEDAAAA|54|Maple |Street|Suite S|Centerville|Henry County|KY|40059|United States|-6|apartment| +13495|AAAAAAAAHLEDAAAA|595|Davis |Ave|Suite N|Marion|Gaines County|TX|70399|United States|-6|single family| +13496|AAAAAAAAILEDAAAA|497|Valley |Street|Suite 230|Salem|Wasco County|OR|98048|United States|-8|condo| +13497|AAAAAAAAJLEDAAAA|31|Lincoln Poplar|Pkwy|Suite N|White Oak|Van Buren County|AR|76668|United States|-6|apartment| +13498|AAAAAAAAKLEDAAAA|917|West Second|ST|Suite K|Waterloo|Franklin County|NE|61675|United States|-6|apartment| +13499|AAAAAAAALLEDAAAA|258|3rd |Lane|Suite 10|Friendship|Jefferson County|OK|74536|United States|-6|single family| +13500|AAAAAAAAMLEDAAAA|647|Elm Lake|Pkwy||Oak Ridge|Richmond County||18371|United States||| +13501|AAAAAAAANLEDAAAA|564|4th Jackson|Road|Suite R|Ashland|Woods County|OK|74244|United States|-6|apartment| +13502|AAAAAAAAOLEDAAAA|558|Cedar |Cir.|Suite 400|Pleasant Hill|Dakota County|MN|53604|United States|-6|condo| +13503|AAAAAAAAPLEDAAAA|504|Seventh |Wy|Suite N|Green Acres|Laurens County|GA|37683|United States|-5|single family| +13504|AAAAAAAAAMEDAAAA|327|Maple Spring|Lane|Suite K|Wildwood|Montgomery County|OH|46871|United States|-5|single family| +13505|AAAAAAAABMEDAAAA|516|Pine Birch|Way|Suite W|Ashland|Hall County|NE|64244|United States|-6|condo| +13506|AAAAAAAACMEDAAAA|786|Poplar 3rd|RD|Suite 270|Clifton|San Juan County|WA|98014|United States|-8|condo| +13507|AAAAAAAADMEDAAAA|272|Maple 12th|Avenue|Suite 150|Brownsville|La Crosse County|WI|59310|United States|-6|apartment| +13508|AAAAAAAAEMEDAAAA|823|Dogwood Cedar|Ave|Suite 370|Spring Hill|Jackson County|IA|56787|United States|-6|apartment| +13509|AAAAAAAAFMEDAAAA|192|Green First|Road|Suite T|Georgetown|Geary County|KS|67057|United States|-6|single family| +13510|AAAAAAAAGMEDAAAA|839|Sycamore Forest|Ave|Suite K|Spring Valley|Menard County|IL|66060|United States|-6|apartment| +13511|AAAAAAAAHMEDAAAA|557|Oak Fourth|Cir.|Suite C|Kingston|East Feliciana Parish|LA|74975|United States|-6|apartment| +13512|AAAAAAAAIMEDAAAA|346|West |Court|Suite 300|Oakdale|Crockett County|TN|39584|United States|-5|condo| +13513|AAAAAAAAJMEDAAAA|545|Meadow |RD|Suite X|Crystal|DeKalb County|MO|65258|United States|-6|condo| +13514|AAAAAAAAKMEDAAAA|730|Hickory Wilson|Street|Suite 340|Friendship|Hockley County|TX|74536|United States|-6|single family| +13515|AAAAAAAALMEDAAAA||Lincoln Wilson||||Highlands County|FL|35124|||| +13516|AAAAAAAAMMEDAAAA|114|Wilson |Lane|Suite J|New Hope|Knox County|KY|49431|United States|-5|condo| +13517|AAAAAAAANMEDAAAA|177|8th Walnut|Ct.|Suite 300|Riverview|Jenkins County|GA|39003|United States|-5|single family| +13518|AAAAAAAAOMEDAAAA|11|Adams 1st|Cir.|Suite T|Franklin|Essex County|NY|19101|United States|-5|apartment| +13519|AAAAAAAAPMEDAAAA|261|4th River|ST|Suite T|Green Acres|Sheboygan County|WI|57683|United States|-6|condo| +13520|AAAAAAAAANEDAAAA|892|Johnson |Pkwy|Suite 320|Fairfield|Mora County|NM|86192|United States|-7|single family| +13521|AAAAAAAABNEDAAAA|579|Elm |Road|Suite 480|Clifton|Wichita County|KS|68014|United States|-6|single family| +13522|AAAAAAAACNEDAAAA|139|5th |Wy|Suite P|Springdale|Koochiching County|MN|58883|United States|-6|condo| +13523|AAAAAAAADNEDAAAA|26|5th Davis|Ct.|Suite X|Jackson|San Miguel County|CO|89583|United States|-7|apartment| +13524|AAAAAAAAENEDAAAA|506|13th |Ave|Suite J|Winslow|Burleson County|TX|78525|United States|-6|condo| +13525|AAAAAAAAFNEDAAAA|43|Highland |Ct.|Suite 290|Riceville|Stanly County|NC|25867|United States|-5|apartment| +13526|AAAAAAAAGNEDAAAA|672|Willow Hickory|Ct.|Suite Y|Jackson|Lafayette County|FL|39583|United States|-5|condo| +13527|AAAAAAAAHNEDAAAA|697|Park |Road|Suite R|Saratoga|Lenoir County|NC|22123|United States|-5|condo| +13528|AAAAAAAAINEDAAAA|823|Oak |Ave|Suite O|Superior|Emmet County|MI|42562|United States|-5|single family| +13529|AAAAAAAAJNEDAAAA|554|Locust Railroad|ST|Suite 150|Buena Vista|Wicomico County|MD|25752|United States|-5|condo| +13530|AAAAAAAAKNEDAAAA|487|Meadow |Boulevard|Suite L|Sunnyside|Marshall County|KY|41952|United States|-5|apartment| +13531|AAAAAAAALNEDAAAA|606|Lakeview Fourth|||White Oak||KS|||-6|| +13532|AAAAAAAAMNEDAAAA|418|Main |RD|Suite E|Clifton|Hughes County|SD|58014|United States|-7|apartment| +13533|AAAAAAAANNEDAAAA|941|Elm Pine|Way|Suite E|Lebanon|Dickinson County|IA|52898|United States|-6|condo| +13534|AAAAAAAAONEDAAAA|958|Lee |Cir.|Suite 0|Liberty|Ogle County|IL|63451|United States|-6|condo| +13535|AAAAAAAAPNEDAAAA|323|5th |Parkway|Suite T|Plainview|Frederick County|VA|23683|United States|-5|single family| +13536|AAAAAAAAAOEDAAAA|839|Woodland Park|Pkwy|Suite 380|Ashland|Winston County|AL|34244|United States|-6|apartment| +13537|AAAAAAAABOEDAAAA|397|Park |Ave|Suite 0|Valley View|Lake and Peninsula Borough|AK|95124|United States|-9|single family| +13538|AAAAAAAACOEDAAAA|510|Pine |Ave|Suite 130|Leland|Queen Anne County|MD|29452|United States|-5|single family| +13539|AAAAAAAADOEDAAAA||2nd Lincoln|Dr.|||||70069|United States||apartment| +13540|AAAAAAAAEOEDAAAA|695|Main Meadow|Street|Suite 0|Cedar Grove|Boone County|KY|40411|United States|-6|apartment| +13541|AAAAAAAAFOEDAAAA|289|9th |Avenue|Suite 370|Oak Ridge|Fayette County|PA|18371|United States|-5|single family| +13542|AAAAAAAAGOEDAAAA|178|14th 6th|Ln|Suite H|Sulphur Springs|Allamakee County|IA|58354|United States|-6|single family| +13543|AAAAAAAAHOEDAAAA||11th Pine||Suite 470||||40587|United States|-5|| +13544|AAAAAAAAIOEDAAAA|166|Hickory Second|Court|Suite 20|Roscoe|Jefferson County|AL|31854|United States|-6|apartment| +13545|AAAAAAAAJOEDAAAA|845|Johnson |Wy|Suite C|Farmington|Sevier County|UT|89145|United States|-7|condo| +13546|AAAAAAAAKOEDAAAA|325|Adams |Avenue|Suite E|Fairfield|Weakley County|TN|36192|United States|-6|single family| +13547|AAAAAAAALOEDAAAA|886|Central Poplar|Pkwy|Suite T|Mountain View|Brewster County|TX|74466|United States|-6|apartment| +13548|AAAAAAAAMOEDAAAA|361|Forest Locust|Street|Suite D|Pine Grove|Kanabec County|MN|54593|United States|-6|apartment| +13549|AAAAAAAANOEDAAAA|617|1st Oak|Dr.|Suite 10|Mount Zion|Emmet County|MI|48054|United States|-5|apartment| +13550|AAAAAAAAOOEDAAAA|766|Maple |Ct.|Suite B|Waterloo|Calloway County|KY|41675|United States|-6|apartment| +13551|AAAAAAAAPOEDAAAA|812|Walnut |Cir.|Suite V|Midway|Robertson County|KY|41904|United States|-5|condo| +13552|AAAAAAAAAPEDAAAA|961|Meadow Hickory|Circle|Suite 490|Highland|Greenlee County|AZ|89454|United States|-7|condo| +13553|AAAAAAAABPEDAAAA|579|Washington 3rd|Drive|Suite 70|New Hope|Sheridan County|MT|69431|United States|-7|single family| +13554|AAAAAAAACPEDAAAA|353|Park |Way|Suite R|Fairfield|Perkins County|SD|56192|United States|-7|condo| +13555|AAAAAAAADPEDAAAA|931|Lincoln |Ln|Suite 400|Oakland|Taliaferro County|GA|39843|United States|-5|condo| +13556|AAAAAAAAEPEDAAAA|437|15th 11th|Avenue|Suite 460|Wildwood|Charles Mix County|SD|56871|United States|-6|condo| +13557|AAAAAAAAFPEDAAAA|34|Pine |Boulevard|Suite 330|Woodlawn|Hartford County|CT|04698|United States|-5|apartment| +13558|AAAAAAAAGPEDAAAA|433|Church Lee|Lane|Suite 440|Jackson|Montcalm County|MI|49583|United States|-5|apartment| +13559|AAAAAAAAHPEDAAAA|370|Madison Center|RD|Suite 490|Oak Hill|Grundy County|IA|57838|United States|-6|condo| +13560|AAAAAAAAIPEDAAAA|290|14th |Cir.|Suite 160|Lakeside|Weston County|WY|89532|United States|-7|single family| +13561|AAAAAAAAJPEDAAAA|47|14th Thirteenth|Ave|Suite O|Woodville|Putnam County|IN|44289|United States|-5|condo| +13562|AAAAAAAAKPEDAAAA|722|Williams Valley|Pkwy|Suite 90|Greenfield|Prentiss County|MS|55038|United States|-6|condo| +13563|AAAAAAAALPEDAAAA|974|Poplar Forest|Boulevard|Suite E|Pine Grove|Colfax County|NE|64593|United States|-6|single family| +13564|AAAAAAAAMPEDAAAA|486|13th |Circle|Suite S|Crossroads|Grant County|ND|50534|United States|-6|single family| +13565|AAAAAAAANPEDAAAA|595|Fifth |Cir.|Suite 140|Riverside|Dutchess County|NY|19231|United States|-5|condo| +13566|AAAAAAAAOPEDAAAA|901|2nd 15th|RD|Suite N|West Liberty|Aurora County|SD|54752|United States|-6|condo| +13567|AAAAAAAAPPEDAAAA|591|West Sixth|Blvd|Suite A|Hopewell|Hall County|GA|30587|United States|-5|single family| +13568|AAAAAAAAAAFDAAAA|580|Hickory |Ave|Suite I|Lakeview|Platte County|WY|88579|United States|-7|apartment| +13569|AAAAAAAABAFDAAAA|868|Lincoln |Parkway|Suite E|Highland||||United States||condo| +13570|AAAAAAAACAFDAAAA|974|Second 3rd|Ave|Suite L|Sunnyside|Wilkinson County|GA|31952|United States|-5|apartment| +13571|AAAAAAAADAFDAAAA|110|Washington Third|ST|Suite T|Newtown|Presidio County|TX|71749|United States|-6|condo| +13572|AAAAAAAAEAFDAAAA|612|Oak |Avenue|Suite 300|Lakeview|Hot Springs County|WY|88579|United States|-7|single family| +13573|AAAAAAAAFAFDAAAA|699|Jackson Woodland|Ave|Suite C|Thompsonville|Clinton County|MO|69651|United States|-6|condo| +13574|AAAAAAAAGAFDAAAA|428|First South|Court|Suite I|Florence|Clinton County|IN|43394|United States|-5|condo| +13575|AAAAAAAAHAFDAAAA|427|Lincoln |Parkway|Suite P|Oak Ridge|Grant County|OK|78371|United States|-6|apartment| +13576|AAAAAAAAIAFDAAAA|905|Ridge Johnson|Court|Suite S|Peru|Houston County|TX|70302|United States|-6|condo| +13577|AAAAAAAAJAFDAAAA|84|Walnut Woodland|Boulevard|Suite 170|Hillcrest|Douglas County|MN|53003|United States|-6|single family| +13578|AAAAAAAAKAFDAAAA|693|Williams Lincoln|Ln|Suite O|Pleasant Hill|Crawford County|MI|43604|United States|-5|condo| +13579|AAAAAAAALAFDAAAA|86|7th |Wy|Suite P|Spring Hill|Clear Creek County|CO|86787|United States|-7|single family| +13580|AAAAAAAAMAFDAAAA|918|7th Washington|Wy|Suite L|Highland|Churchill County|NV|89454|United States|-8|single family| +13581|AAAAAAAANAFDAAAA|7|Spring Davis|Drive|Suite 70|Belmont|Shackelford County|TX|70191|United States|-6|apartment| +13582|AAAAAAAAOAFDAAAA|37|Washington Cedar|RD|Suite Y|Antioch|Santa Cruz County|AZ|88605|United States|-7|apartment| +13583|AAAAAAAAPAFDAAAA|481|12th |Avenue|Suite 490|Edgewood|Murray County|OK|70069|United States|-6|apartment| +13584|AAAAAAAAABFDAAAA|362|Valley |Road|Suite A|Greenfield|Ashe County|NC|25038|United States|-5|condo| +13585|AAAAAAAABBFDAAAA|989|5th |Way|Suite 150|Farmington|Napa County|CA|99145|United States|-8|apartment| +13586|AAAAAAAACBFDAAAA|608|15th |Road|Suite 320|Bunker Hill|Macon County|NC|20150|United States|-5|single family| +13587|AAAAAAAADBFDAAAA|244|10th |Court|Suite 160|Pleasant Valley|Vernon County|WI|52477|United States|-6|condo| +13588|AAAAAAAAEBFDAAAA|168|Davis |Blvd|Suite S|Deerfield|Santa Fe County|NM|89840|United States|-7|condo| +13589|AAAAAAAAFBFDAAAA|758|Lincoln Church|Dr.|Suite 250|Midway|Wirt County|WV|21904|United States|-5|apartment| +13590|AAAAAAAAGBFDAAAA|481|North |Circle|Suite F|Brownsville|Wilkinson County|MS|59310|United States|-6|condo| +13591|AAAAAAAAHBFDAAAA|722|Spring Church|Pkwy|Suite 160|Collinsville|Jerome County|ID|82459|United States|-7|single family| +13592|AAAAAAAAIBFDAAAA|793|Ridge Park|||Vienna|||55119|United States||apartment| +13593|AAAAAAAAJBFDAAAA|239|View |ST|Suite 470|Valley View|Marshall County|WV|25124|United States|-5|single family| +13594|AAAAAAAAKBFDAAAA|460|Jackson |Avenue|Suite 40|Union|Randolph County|NC|28721|United States|-5|apartment| +13595|AAAAAAAALBFDAAAA|315|13th |Court|Suite 450|Glenwood|Chickasaw County|MS|53511|United States|-6|apartment| +13596|AAAAAAAAMBFDAAAA|223|Washington Jefferson|Ct.|Suite G|Springfield|Potter County|PA|19303|United States|-5|apartment| +13597|AAAAAAAANBFDAAAA|436|11th Lake|Road|Suite 310|Pleasant Grove|Ellis County|OK|74136|United States|-6|apartment| +13598|AAAAAAAAOBFDAAAA|719|River |Street|Suite 70|Altamont|Hays County|TX|79387|United States|-6|apartment| +13599|AAAAAAAAPBFDAAAA|366|Madison Spruce|Ln|Suite 490|Riverside|Roosevelt County|MT|69231|United States|-7|apartment| +13600|AAAAAAAAACFDAAAA|606|Seventh |Circle|Suite N|Milan|Trimble County|KY|46697|United States|-5|condo| +13601|AAAAAAAABCFDAAAA|3|5th |RD|Suite 360|Glenwood|Wichita County|KS|63511|United States|-6|apartment| +13602|AAAAAAAACCFDAAAA|458|Mill |Boulevard|Suite 160|Salem|Lyon County|MN|58048|United States|-6|condo| +13603|AAAAAAAADCFDAAAA|720|Chestnut |Blvd|Suite 0|Riverside|Susquehanna County|PA|19231|United States|-5|apartment| +13604|AAAAAAAAECFDAAAA|554|Birch |RD|Suite 390|Sulphur Springs|Lincoln County|WA|98354|United States|-8|single family| +13605|AAAAAAAAFCFDAAAA|608|Third |Street|Suite F|Woodlawn|Sierra County|CA|94098|United States|-8|apartment| +13606|AAAAAAAAGCFDAAAA|999|College Meadow|Pkwy|Suite 300|Jamestown|Lea County|NM|86867|United States|-7|apartment| +13607|AAAAAAAAHCFDAAAA|578|Sunset Sixth|Dr.|Suite 270|Florence|Allen County|KY|43394|United States|-6|condo| +13608|AAAAAAAAICFDAAAA|327|Franklin 3rd|ST|Suite M|Four Points|Marshall County|AL|31216|United States|-6|single family| +13609|AAAAAAAAJCFDAAAA|461|First Smith|Lane|Suite S|Shiloh|Oconee County|SC|29275|United States|-5|condo| +13610|AAAAAAAAKCFDAAAA|688|Walnut |Wy|Suite H|Edgewood|Jewell County|KS|60069|United States|-6|condo| +13611|AAAAAAAALCFDAAAA|154|5th |Wy|Suite N|Walnut Grove|Lynchburg city|VA|27752|United States|-5|apartment| +13612|AAAAAAAAMCFDAAAA|39|3rd Tenth|ST|Suite X|Harmony|Tom Green County|TX|75804|United States|-6|single family| +13613|AAAAAAAANCFDAAAA|379|Lake Hill|Avenue|Suite 340|Bethel|Kinney County|TX|75281|United States|-6|apartment| +13614|AAAAAAAAOCFDAAAA|656|South |Dr.|Suite 160|Oak Grove|Kittson County|MN|58370|United States|-6|single family| +13615|AAAAAAAAPCFDAAAA||Park Ash|ST|Suite 280|Summerville||TX|72033|United States|-6|apartment| +13616|AAAAAAAAADFDAAAA|678|View Birch|Blvd|Suite E|Belmont|Greensville County|VA|20191|United States|-5|apartment| +13617|AAAAAAAABDFDAAAA|881|1st |Dr.|Suite G|Deerfield|Sharp County|AR|79840|United States|-6|apartment| +13618|AAAAAAAACDFDAAAA|953|Hill Cherry|Avenue|Suite I|Greenfield|Marshall County|IL|65038|United States|-6|apartment| +13619|AAAAAAAADDFDAAAA|357|Seventh |Boulevard|Suite 300|Lincoln|Randolph County|GA|31289|United States|-5|condo| +13620|AAAAAAAAEDFDAAAA|649|Ridge Broadway|Ct.|Suite A|Cedar Grove|Lipscomb County|TX|70411|United States|-6|condo| +13621|AAAAAAAAFDFDAAAA|280|Highland |Lane|Suite 470|Hillcrest|Atlantic County|NJ|03603|United States|-5|condo| +13622|AAAAAAAAGDFDAAAA|493|Center Cherry|Parkway|Suite 10|Leland|White Pine County|NV|89452|United States|-8|apartment| +13623|AAAAAAAAHDFDAAAA|631|Fourth 14th|Boulevard|Suite 30|Waterloo|San Jacinto County|TX|71675|United States|-6|apartment| +13624|AAAAAAAAIDFDAAAA|668|Jefferson Ridge|Ct.|Suite S|Star|Cherokee County|KS|60725|United States|-6|single family| +13625|AAAAAAAAJDFDAAAA|584|7th Hickory|Street|Suite 290|Walnut Grove|Reeves County|TX|77752|United States|-6|single family| +13626|AAAAAAAAKDFDAAAA|941|10th Wilson|Cir.|Suite O|Lakeside|Breathitt County|KY|49532|United States|-6|condo| +13627|AAAAAAAALDFDAAAA|434|4th Cedar|Street|Suite 40|Wildwood|Caldwell County|MO|66871|United States|-6|apartment| +13628|AAAAAAAAMDFDAAAA|967|Railroad |Circle|Suite E|Walnut Grove|Vermilion County|IL|67752|United States|-6|single family| +13629|AAAAAAAANDFDAAAA|875|Hill |Circle|Suite P|Shiloh|Fremont County|IA|59275|United States|-6|single family| +13630|AAAAAAAAODFDAAAA|153|Cedar Spring|Street|Suite P|Pleasant Valley|Broadwater County|MT|62477|United States|-7|apartment| +13631|AAAAAAAAPDFDAAAA|257|Washington |Cir.|Suite 10|Shady Grove|Oliver County|ND|52812|United States|-6|apartment| +13632|AAAAAAAAAEFDAAAA|199|Poplar Davis|Pkwy|Suite 250|Pleasant Hill|Beadle County|SD|53604|United States|-6|condo| +13633|AAAAAAAABEFDAAAA|648|Oak Oak|Dr.|||||10534|United States||| +13634|AAAAAAAACEFDAAAA|616|Lincoln 1st|Avenue|Suite 60|Woodville|Alcona County|MI|44289|United States|-5|single family| +13635|AAAAAAAADEFDAAAA|557|Highland |Court|Suite Y|Florence|Windham County|CT|03994|United States|-5|apartment| +13636|AAAAAAAAEEFDAAAA|566|Sycamore Lincoln|Dr.|Suite J|Shady Grove|Elmore County|ID|82812|United States|-7|single family| +13637|AAAAAAAAFEFDAAAA|563|Second Madison|ST|Suite 200|Fairbanks|Cherokee County|AL|36653|United States|-6|single family| +13638|AAAAAAAAGEFDAAAA|539|Park Walnut|Circle|Suite A|Riverside|Saguache County|CO|89231|United States|-7|condo| +13639|AAAAAAAAHEFDAAAA|143|6th |Way|Suite 80|Marion|Park County|WY|80399|United States|-7|apartment| +13640|AAAAAAAAIEFDAAAA|329|Park |Lane|Suite 160|Springfield|DeKalb County|IL|69303|United States|-6|single family| +13641|AAAAAAAAJEFDAAAA|580|Laurel Elm|Cir.|Suite U|Waterloo|Lowndes County|MS|51675|United States|-6|single family| +13642|AAAAAAAAKEFDAAAA|786|Mill 7th|Court|Suite 490|Pleasant Grove|Ralls County|MO|64136|United States|-6|condo| +13643|AAAAAAAALEFDAAAA|996|Hill |Parkway|Suite 340|Springfield|Bryan County|GA|39303|United States|-5|condo| +13644|AAAAAAAAMEFDAAAA||Hill Franklin|||||TN|39843|United States||apartment| +13645|AAAAAAAANEFDAAAA|216|Railroad |ST|Suite F|Jamestown|Mecosta County|MI|46867|United States|-5|apartment| +13646|AAAAAAAAOEFDAAAA|78|East Twelfth|Ln|Suite 230|Glendale|Stone County|AR|73951|United States|-6|apartment| +13647|AAAAAAAAPEFDAAAA|921|Davis |Lane|Suite 30|Lakeview|Delaware County|PA|18579|United States|-5|single family| +13648|AAAAAAAAAFFDAAAA|220|Birch ||Suite T||Wagoner County|OK|||-6|| +13649|AAAAAAAABFFDAAAA|633|South Sunset|Ct.|Suite E||Iredell County|NC||||apartment| +13650|AAAAAAAACFFDAAAA|791|Jackson Hill|Court|Suite 20|Maple Grove|Madison County|IA|58252|United States|-6|single family| +13651|AAAAAAAADFFDAAAA|85|Lincoln 8th|Dr.|Suite 370|Greenville|Bottineau County|ND|51387|United States|-6|single family| +13652|AAAAAAAAEFFDAAAA|41|2nd |Parkway|Suite 190|Highland Park|Washington County|PA|16534|United States|-5|condo| +13653|AAAAAAAAFFFDAAAA|957|Church Laurel|Parkway|Suite V|Glenwood|Kemper County|MS|53511|United States|-6|single family| +13654|AAAAAAAAGFFDAAAA|140|1st |Cir.|Suite 10|Spring Valley|Jefferson County|IA|56060|United States|-6|condo| +13655|AAAAAAAAHFFDAAAA|759|8th |Ct.|Suite Y|Buena Vista|Clay County|IL|65752|United States|-6|apartment| +13656|AAAAAAAAIFFDAAAA|816|Poplar Park|Road|Suite 160|Lakeview|Okanogan County|WA|98579|United States|-8|condo| +13657|AAAAAAAAJFFDAAAA|594|Ridge Park|Wy|Suite 300|Macedonia|Emmons County|ND|51087|United States|-6|single family| +13658|AAAAAAAAKFFDAAAA||Park 2nd||Suite O|Lincoln|Pottawattamie County|IA||||| +13659|AAAAAAAALFFDAAAA|693|1st Washington|Wy|Suite 180|Red Hill|Long County|GA|34338|United States|-5|condo| +13660|AAAAAAAAMFFDAAAA|876|8th 9th|Ct.|Suite F|Sulphur Springs|Salem city|VA|28354|United States|-5|condo| +13661|AAAAAAAANFFDAAAA|208|Elm |Dr.|Suite P|Midway|Clinton County|IN|41904|United States|-5|apartment| +13662|AAAAAAAAOFFDAAAA|826|Mill Sunset|Circle|Suite M|Riverview|Clayton County|IA|59003|United States|-6|apartment| +13663|AAAAAAAAPFFDAAAA|656|Sixth 4th|Pkwy|Suite U|Trenton|Kern County|CA|99566|United States|-8|single family| +13664|AAAAAAAAAGFDAAAA|255|Elm |Way|Suite J|Lebanon|Boone County|IA|52898|United States|-6|condo| +13665|AAAAAAAABGFDAAAA|493||Parkway|Suite D|Five Points|Skamania County|||||| +13666|AAAAAAAACGFDAAAA|12|Locust |RD|Suite M|Cedar Grove|Wayne County|IL|60411|United States|-6|single family| +13667|AAAAAAAADGFDAAAA|100|Oak |RD|Suite Y|Macedonia|Dutchess County|NY|11087|United States|-5|single family| +13668|AAAAAAAAEGFDAAAA|773|South |Avenue|Suite B|Bridgeport|Newton County|GA|35817|United States|-5|apartment| +13669|AAAAAAAAFGFDAAAA|467|10th |Cir.|Suite A|Macedonia|Archuleta County|CO|81087|United States|-7|condo| +13670|AAAAAAAAGGFDAAAA|307|Eigth East|Lane|Suite 370|Pine Grove|Ransom County|ND|54593|United States|-6|apartment| +13671|AAAAAAAAHGFDAAAA|724|West 1st|Ln|Suite 270|Nichols|Venango County|PA|17940|United States|-5|apartment| +13672|AAAAAAAAIGFDAAAA|908|Smith Church|Circle|Suite 320|Arlington|McHenry County|ND|56557|United States|-6|condo| +13673|AAAAAAAAJGFDAAAA|818|Walnut Cherry|ST|Suite K|Greenfield|Oscoda County|MI|45038|United States|-5|single family| +13674|AAAAAAAAKGFDAAAA|138|Willow |Blvd|Suite 460|Riverdale|Burleson County|TX|79391|United States|-6|condo| +13675|AAAAAAAALGFDAAAA|311|Walnut |Parkway|Suite I|Lakeside|Chittenden County|VT|09532|United States|-5|single family| +13676|AAAAAAAAMGFDAAAA|487|Washington |RD|Suite E|Midway|Mercer County|IL|61904|United States|-6|apartment| +13677|AAAAAAAANGFDAAAA|734|8th Birch|Road|Suite 180|Plainview|Clay County|WV|23683|United States|-5|condo| +13678|AAAAAAAAOGFDAAAA|218|Sunset |Road|Suite 350|Piney Grove|Bureau County|IL|68274|United States|-6|apartment| +13679|AAAAAAAAPGFDAAAA|951|Madison West|Road|Suite R|Spring Hill|Bledsoe County|TN|36787|United States|-5|condo| +13680|AAAAAAAAAHFDAAAA|714|Dogwood |Street|Suite P|Pleasant Hill|Yates County|NY|13604|United States|-5|condo| +13681|AAAAAAAABHFDAAAA|935|2nd Maple|Pkwy|Suite O|Oakland|Monroe County|MO|69843|United States|-6|single family| +13682|AAAAAAAACHFDAAAA|688|Railroad 4th|Dr.|Suite 70|Pleasant Grove|Cape May County|NJ|04736|United States|-5|apartment| +13683|AAAAAAAADHFDAAAA|702|Wilson Second|Dr.|Suite 470|Stringtown|Columbiana County|OH|40162|United States|-5|apartment| +13684|AAAAAAAAEHFDAAAA|751|12th Center|Ct.|Suite 100|Pleasant Valley|Richmond County|GA|32477|United States|-5|condo| +13685|AAAAAAAAFHFDAAAA|956|Sycamore Cedar|Dr.|Suite 180|Lincoln|King and Queen County|VA|21289|United States|-5|condo| +13686|AAAAAAAAGHFDAAAA|636|Spring 7th|Circle|Suite 420|Riverdale|Boyle County|KY|49391|United States|-6|apartment| +13687|AAAAAAAAHHFDAAAA|350|Dogwood 12th|Ave|Suite 320|Forestville|Bullock County|AL|33027|United States|-6|condo| +13688|AAAAAAAAIHFDAAAA|402|Park |ST|Suite E|Lakewood|Arapahoe County|CO|88877|United States|-7|single family| +13689|AAAAAAAAJHFDAAAA|232|1st |||Friendship|New Castle County|DE|14536|||condo| +13690|AAAAAAAAKHFDAAAA|964|Smith |RD|Suite Y|Riverside|Irwin County|GA|39231|United States|-5|condo| +13691|AAAAAAAALHFDAAAA|850|Washington 13th|Ln|Suite I|Valley View|Suffolk County|MA|05724|United States|-5|condo| +13692|AAAAAAAAMHFDAAAA|489|Hill |Ave|Suite L|Wilson|Davis County|IA|56971|United States|-6|condo| +13693|AAAAAAAANHFDAAAA|76|Mill |Pkwy|Suite S|Lincoln|DeKalb County|TN|31289|United States|-5|apartment| +13694|AAAAAAAAOHFDAAAA|214|Wilson Smith|Road|Suite 320|Quincy|Columbiana County|OH|43868|United States|-5|condo| +13695|AAAAAAAAPHFDAAAA|285|Central |Street|Suite S|Maple Grove|Logan County|CO|88252|United States|-7|apartment| +13696|AAAAAAAAAIFDAAAA|||||Cedar Grove|Greenlee County||80411|United States|-7|single family| +13697|AAAAAAAABIFDAAAA|858|Johnson |Boulevard|Suite 230|Marion|Union County|NM|80399|United States|-7|apartment| +13698|AAAAAAAACIFDAAAA|630|Valley |Blvd|Suite G|Farmington|Caroline County|VA|29145|United States|-5|condo| +13699|AAAAAAAADIFDAAAA|522|First Second|Way|Suite Y|Lebanon|Bartholomew County|IN|42898|United States|-5|apartment| +13700|AAAAAAAAEIFDAAAA|370|12th Hillcrest|Cir.|Suite 210|Spring Hill|Camden County|NC|26787|United States|-5|condo| +13701|AAAAAAAAFIFDAAAA|514|2nd Green|ST|Suite 300|Pleasant Valley|Falls Church city|VA|22477|United States|-5|single family| +13702|AAAAAAAAGIFDAAAA|529|Jefferson |Parkway|Suite J|Oakwood|Anoka County|MN|50169|United States|-6|single family| +13703|AAAAAAAAHIFDAAAA|821|Hillcrest |Circle|Suite Q|Marion|Floyd County|GA|30399|United States|-5|apartment| +13704|AAAAAAAAIIFDAAAA|46|1st Ridge|Drive|Suite C|Leesville|Murray County|OK|75423|United States|-6|condo| +13705|AAAAAAAAJIFDAAAA|493|Birch Lakeview|Parkway|Suite H|Five Points|Harding County|SD|56098|United States|-7|condo| +13706|AAAAAAAAKIFDAAAA|397|Meadow Maple|Circle|Suite 170|Wildwood|Uintah County|UT|86871|United States|-7|apartment| +13707|AAAAAAAALIFDAAAA|756|Pine |Blvd|Suite J|Spring Valley|Columbus County|NC|26060|United States|-5|apartment| +13708|AAAAAAAAMIFDAAAA|571|Main |Ave|Suite B|Mount Zion|Troup County|GA|38054|United States|-5|apartment| +13709|AAAAAAAANIFDAAAA|693|8th Cedar|Avenue|Suite F|Glenwood|Washington County|IA|53511|United States|-6|condo| +13710|AAAAAAAAOIFDAAAA|227|College Hillcrest|Dr.|Suite 0|Willis|Moultrie County|IL|66788|United States|-6|apartment| +13711|AAAAAAAAPIFDAAAA|970|Laurel |Boulevard|Suite O|Griffin|Ballard County|KY|41204|United States|-6|apartment| +13712|AAAAAAAAAJFDAAAA|495|6th |Circle|Suite N|Oakwood|Saline County|AR|70169|United States|-6|condo| +13713|AAAAAAAABJFDAAAA|182|Smith |Wy|Suite B|Mount Pleasant|Morgan County|IN|41933|United States|-5|condo| +13714|AAAAAAAACJFDAAAA|833|Hickory 4th|Road|Suite 50|Freeport|Fauquier County|VA|21844|United States|-5|condo| +13715|AAAAAAAADJFDAAAA|240|Cherry |Wy|Suite R|Crossroads|Payette County|ID|80534|United States|-7|apartment| +13716|AAAAAAAAEJFDAAAA|660|Tenth |ST|Suite R|Franklin|Belknap County|NH|09701|United States|-5|apartment| +13717|AAAAAAAAFJFDAAAA|88|Woodland Washington|Pkwy|Suite T|Marion|Lewis County|TN|30399|United States|-6|apartment| +13718|AAAAAAAAGJFDAAAA|716|15th |Ln|Suite 330|Maple Grove|Red River Parish|LA|78252|United States|-6|apartment| +13719|AAAAAAAAHJFDAAAA|258|Pine Second|Drive|Suite H|Clinton|Brevard County|FL|38222|United States|-5|condo| +13720|AAAAAAAAIJFDAAAA|514|11th 4th|Way|Suite 180|Clinton|Wichita County|KS|68222|United States|-6|apartment| +13721|AAAAAAAAJJFDAAAA|455|Second View|Avenue|Suite 280|Hopewell|Greene County|PA|10587|United States|-5|condo| +13722|AAAAAAAAKJFDAAAA|377|Jefferson |Court|Suite 130|Hopewell|Dodge County|MN|50587|United States|-6|apartment| +13723|AAAAAAAALJFDAAAA|783|Cherry Hill|ST|Suite 10|Oak Grove|Moultrie County|IL|68370|United States|-6|single family| +13724|AAAAAAAAMJFDAAAA|933|Park Sixth|ST|Suite M|Harmony|Green Lake County|WI|55804|United States|-6|apartment| +13725|AAAAAAAANJFDAAAA|500|Second Forest|Blvd|Suite 10|Salem|Washington County|NE|68048|United States|-7|single family| +13726|AAAAAAAAOJFDAAAA|626|Hillcrest Spring|Blvd|Suite T|Antioch|Hopewell city|VA|28605|United States|-5|single family| +13727|AAAAAAAAPJFDAAAA|358|Washington |Drive|Suite 150|Wildwood|Gila County|AZ|86871|United States|-7|single family| +13728|AAAAAAAAAKFDAAAA|746|Lake |Parkway|Suite 250|Hillcrest|Phelps County|MO|63003|United States|-6|condo| +13729|AAAAAAAABKFDAAAA|964|Maple |Dr.|Suite P|Five Forks|Grant County|WV|22293|United States|-5|single family| +13730|AAAAAAAACKFDAAAA|477|Highland |Parkway|Suite 360|Carthage|Portage County|OH|41529|United States|-5|single family| +13731|AAAAAAAADKFDAAAA|969|Oak Smith|ST|Suite 370|Belmont|Walla Walla County|WA|90191|United States|-8|apartment| +13732|AAAAAAAAEKFDAAAA|174|Lake 6th|Ln|Suite 230|Mount Vernon|Grant County|WI|58482|United States|-6|condo| +13733|AAAAAAAAFKFDAAAA|571|5th 1st|RD|Suite 100|Perkins|Grainger County|TN|31852|United States|-5|condo| +13734|AAAAAAAAGKFDAAAA|815|Eigth 14th|ST|Suite C|Forest Hills|Banner County|NE|69237|United States|-6|apartment| +13735|AAAAAAAAHKFDAAAA|236|13th |Lane|Suite T|Greenfield|Jasper County|GA|35038|United States|-5|condo| +13736|AAAAAAAAIKFDAAAA|384|Ash |Court|Suite S|Unionville|Shawnee County|KS|61711|United States|-6|apartment| +13737|AAAAAAAAJKFDAAAA|22|Third |Street|Suite B|Greenwood|Sibley County|MN|58828|United States|-6|condo| +13738|AAAAAAAAKKFDAAAA|890|Thirteenth |Court|Suite W|Newport|Allen County|OH|41521|United States|-5|apartment| +13739|AAAAAAAALKFDAAAA|470|Birch Chestnut|ST|Suite 310|Union Hill|Bayfield County|WI|57746|United States|-6|condo| +13740|AAAAAAAAMKFDAAAA|659|7th |Ln|Suite C|Newport|Licking County|OH|41521|United States|-5|apartment| +13741|AAAAAAAANKFDAAAA|530|5th 9th|Ct.|Suite 40|Omega|Richmond County|NC|26364|United States|-5|single family| +13742|AAAAAAAAOKFDAAAA|689|Fifth Spruce|Dr.|Suite T|Phoenix|Branch County|MI|42276|United States|-5|condo| +13743|AAAAAAAAPKFDAAAA|717|Highland Meadow|Court|Suite 200|Crossroads|Taliaferro County|GA|30534|United States|-5|single family| +13744|AAAAAAAAALFDAAAA|35|4th Church|Ln|Suite H|Concord|White County|GA|34107|United States|-5|single family| +13745|AAAAAAAABLFDAAAA|881|Cedar |Court|Suite K|Buena Vista|Gilmer County|GA|35752|United States|-5|condo| +13746|AAAAAAAACLFDAAAA||8th ||Suite B|||||United States||| +13747|AAAAAAAADLFDAAAA|650|Johnson 6th|Wy|Suite 300|Brownsville|Luna County|NM|89310|United States|-7|apartment| +13748|AAAAAAAAELFDAAAA|186|Elm |Dr.|Suite 0|Clifton|Niobrara County|WY|88014|United States|-7|single family| +13749|AAAAAAAAFLFDAAAA|680|View |Dr.|Suite W|Summerville|Lafayette County|MO|62033|United States|-6|condo| +13750|AAAAAAAAGLFDAAAA|626|Elm Elm|Road|Suite Y|Springfield|Hall County|GA|39303|United States|-5|apartment| +13751|AAAAAAAAHLFDAAAA|860|Broadway Park|RD|Suite 470|Belmont|Prince William County|VA|20191|United States|-5|single family| +13752|AAAAAAAAILFDAAAA|169|North |Wy|Suite 350|Green Acres|Hoke County|NC|27683|United States|-5|apartment| +13753|AAAAAAAAJLFDAAAA|607|River Meadow|Lane|Suite 80|Owens|Lancaster County|SC|22324|United States|-5|apartment| +13754|AAAAAAAAKLFDAAAA|14|Miller |Street|Suite 100|Freeman|Whitman County|WA|92297|United States|-8|apartment| +13755|AAAAAAAALLFDAAAA|114|Hill |Avenue|Suite 90|Marion|Henry County|IA|50399|United States|-6|apartment| +13756|AAAAAAAAMLFDAAAA|279|Main Hill|ST|Suite 350|Oak Grove|Falls Church city|VA|28370|United States|-5|condo| +13757|AAAAAAAANLFDAAAA|956|3rd Park|Circle|Suite 490|Lakewood|Crosby County|TX|78877|United States|-6|single family| +13758|AAAAAAAAOLFDAAAA|363|Pine |Boulevard|Suite 110|Pleasant Hill|Shelby County|IA|53604|United States|-6|single family| +13759|AAAAAAAAPLFDAAAA|967|Elm |Court|Suite X|Arlington|Hamilton County|KS|66557|United States|-6|single family| +13760|AAAAAAAAAMFDAAAA|96|1st |Lane|Suite Q|Midway|Lowndes County|GA|31904|United States|-5|condo| +13761|AAAAAAAABMFDAAAA|815|Birch |Boulevard|Suite 80|Lucas|Lynchburg city|VA|24554|United States|-5|apartment| +13762|AAAAAAAACMFDAAAA|302|6th 3rd|Circle|Suite 270|Birmingham|Weston County|WY|83372|United States|-7|condo| +13763|AAAAAAAADMFDAAAA|325|Franklin Chestnut|Ln|Suite V|Greenwood|Ida County|IA|58828|United States|-6|apartment| +13764|AAAAAAAAEMFDAAAA|211|Broadway Franklin|Way|Suite 460|Freeman|Rockland County|NY|12297|United States|-5|apartment| +13765|AAAAAAAAFMFDAAAA|357|8th |Blvd|Suite N|Arlington|Perkins County|NE|66557|United States|-7|condo| +13766|AAAAAAAAGMFDAAAA|119|Chestnut Lee|Ct.|Suite 180|Shiloh|Page County|IA|59275|United States|-6|condo| +13767|AAAAAAAAHMFDAAAA|715|Seventh |RD|Suite A|Wildwood|Pope County|MN|56871|United States|-6|condo| +13768|AAAAAAAAIMFDAAAA|697|Spruce 5th|Parkway|Suite V|Ruth|McClain County|OK|70309|United States|-6|single family| +13769|AAAAAAAAJMFDAAAA|533|5th |ST|Suite 190|Lincoln|Culberson County|TX|71289|United States|-6|single family| +13770|AAAAAAAAKMFDAAAA|929|Main 8th|Wy|Suite 310|Farmington|Jay County|IN|49145|United States|-5|apartment| +13771|AAAAAAAALMFDAAAA|163|9th Ridge|Blvd|Suite 170|Newtown|Newton County|MS|51749|United States|-6|condo| +13772|AAAAAAAAMMFDAAAA|952|3rd Cedar|Road|Suite F|Five Forks|Fremont County|CO|82293|United States|-7|single family| +13773|AAAAAAAANMFDAAAA|80|Fifth |Ct.|Suite 390|Bunker Hill|Winston County|AL|30150|United States|-6|single family| +13774|AAAAAAAAOMFDAAAA|194|Washington |Parkway|Suite 440|Church Hill|Graham County|AZ|83790|United States|-7|condo| +13775|AAAAAAAAPMFDAAAA|466|Williams |ST|Suite C|Springdale|Franklin County|VT|09483|United States|-5|condo| +13776|AAAAAAAAANFDAAAA|552|Hickory Sycamore|Court|Suite P|Kingston|Hopkins County|TX|74975|United States|-6|apartment| +13777|AAAAAAAABNFDAAAA|632|Lee |Lane|Suite 160|||MS|56098|United States|-6|| +13778|AAAAAAAACNFDAAAA|213|North Park|Ave|Suite S|Newtown|Lassen County|CA|91749|United States|-8|condo| +13779|AAAAAAAADNFDAAAA|738|6th |Parkway|Suite N|Riverview|Lincoln County|MS|59003|United States|-6|single family| +13780|AAAAAAAAENFDAAAA|425|||Suite R|Lakeside||NE|69532|United States|-7|apartment| +13781|AAAAAAAAFNFDAAAA|151|Elm Adams|Pkwy|Suite Q|Forest Hills|Henry County|KY|49237|United States|-6|apartment| +13782|AAAAAAAAGNFDAAAA|306|River |RD|Suite 220|Stringtown|Crockett County|TX|70162|United States|-6|apartment| +13783|AAAAAAAAHNFDAAAA|585|2nd College|RD|Suite 130|Hurricane|Wake County|NC|27644|United States|-5|apartment| +13784|AAAAAAAAINFDAAAA|723|4th 8th|Lane|Suite K|Marion|Nolan County|TX|70399|United States|-6|condo| +13785|AAAAAAAAJNFDAAAA|561|Johnson |Cir.|Suite 210|Plainview|Camden County|NC|23683|United States|-5|condo| +13786|AAAAAAAAKNFDAAAA|499|7th |Ct.|Suite 200|White Oak|Wapello County|IA|56668|United States|-6|single family| +13787|AAAAAAAALNFDAAAA|217|Spruce |Ct.|Suite 40|Golden|Marlboro County|SC|20411|United States|-5|apartment| +13788|AAAAAAAAMNFDAAAA|59|6th |Drive|Suite 440|Pleasant Hill|Hidalgo County|NM|83604|United States|-7|condo| +13789|AAAAAAAANNFDAAAA|293|Madison 4th|RD|Suite D|Mountain View|Thurston County|NE|64466|United States|-7|single family| +13790|AAAAAAAAONFDAAAA|329|View |Parkway|Suite W|Unionville|Dallas County|AR|71711|United States|-6|single family| +13791|AAAAAAAAPNFDAAAA|373|Madison |Way|Suite L|Green Acres|Pulaski County|AR|77683|United States|-6|condo| +13792|AAAAAAAAAOFDAAAA|655|Willow |Avenue|Suite D|Pleasant Valley|San Francisco County|CA|92477|United States|-8|condo| +13793|AAAAAAAABOFDAAAA|824|Birch |Street|Suite 10|Ashland|Clinton County|IL|64244|United States|-6|single family| +13794|AAAAAAAACOFDAAAA|909|Willow 5th|RD|Suite 100|Deerfield|Pocahontas County|IA|59840|United States|-6|single family| +13795|AAAAAAAADOFDAAAA|216|Johnson Chestnut|RD|Suite U|Farmington|Dolores County|CO|89145|United States|-7|condo| +13796|AAAAAAAAEOFDAAAA|981|Hill |Court|Suite 160|Lakewood|Bureau County|IL|68877|United States|-6|condo| +13797|AAAAAAAAFOFDAAAA|120|5th |Circle|Suite 230|Crossroads|Ozark County|MO|60534|United States|-6|condo| +13798|AAAAAAAAGOFDAAAA|580|3rd Mill|Ln|Suite 160|Providence|Cowlitz County|WA|96614|United States|-8|apartment| +13799|AAAAAAAAHOFDAAAA|54|Meadow Lincoln|Ct.|Suite Q|Mount Zion|Barnwell County|SC|28054|United States|-5|single family| +13800|AAAAAAAAIOFDAAAA|929|Hill North|Way|Suite G|Greenville|Hutchinson County|TX|71387|United States|-6|condo| +13801|AAAAAAAAJOFDAAAA|5|1st |Dr.|Suite L|Mount Pleasant|Texas County|OK|71933|United States|-6|apartment| +13802|AAAAAAAAKOFDAAAA|544|Maple |Avenue|Suite 390|Cedar Grove|Ripley County|MO|60411|United States|-6|single family| +13803|AAAAAAAALOFDAAAA|993|Johnson Lakeview|Ln|Suite X|Lincoln|Boone County|IN|41289|United States|-5|single family| +13804|AAAAAAAAMOFDAAAA|753|Spruce 4th|Avenue|Suite A|Belmont|Finney County|KS|60191|United States|-6|apartment| +13805|AAAAAAAANOFDAAAA|83|Locust Pine|Street|Suite A|Five Forks|Angelina County|TX|72293|United States|-6|single family| +13806|AAAAAAAAOOFDAAAA|190|Lee Walnut|Circle||Oakland||VA|29843|United States||| +13807|AAAAAAAAPOFDAAAA|749|Main Madison|Dr.|Suite G|Macedonia|Yell County|AR|71087|United States|-6|apartment| +13808|AAAAAAAAAPFDAAAA|698|Elm 9th|||||VA||United States||condo| +13809|AAAAAAAABPFDAAAA|972|13th |ST|Suite V|Unionville|Roanoke city|VA|21711|United States|-5|apartment| +13810|AAAAAAAACPFDAAAA|416|Lincoln |Boulevard|Suite S|San Jose|Tioga County|PA|18003|United States|-5|apartment| +13811|AAAAAAAADPFDAAAA|902|Dogwood Park|Ln|Suite 420|Oakwood|Hardin County|TN|30169|United States|-5|condo| +13812|AAAAAAAAEPFDAAAA|793|Hill |Circle|Suite I|Oak Hill|Barry County|MI|47838|United States|-5|condo| +13813|AAAAAAAAFPFDAAAA|852|Highland Poplar|Circle|Suite U|Marion|Henry County|VA|20399|United States|-5|single family| +13814|AAAAAAAAGPFDAAAA|323|Sixth Mill|Way|Suite 260|Brownsville|DeKalb County|MO|69310|United States|-6|condo| +13815|AAAAAAAAHPFDAAAA|290|4th 7th|Way|Suite J|Wilson|Vermillion County|IN|46971|United States|-5|condo| +13816|AAAAAAAAIPFDAAAA|343|Green 7th|Circle|Suite G|Providence|Clinton County|NY|16614|United States|-5|single family| +13817|AAAAAAAAJPFDAAAA|538|Hill Chestnut|Boulevard|Suite 70|Franklin|Ventura County|CA|99101|United States|-8|single family| +13818|AAAAAAAAKPFDAAAA|330|Poplar |Wy|Suite Q|Hamilton|Sumter County|FL|32808|United States|-5|condo| +13819|AAAAAAAALPFDAAAA|123|Hillcrest Sunset|Court|Suite N|Greenwood|Power County|ID|88828|United States|-7|apartment| +13820|AAAAAAAAMPFDAAAA|961|Laurel |Dr.|Suite 20|Murray|Larimer County|CO|82150|United States|-7|apartment| +13821|AAAAAAAANPFDAAAA|765|Davis 1st|Boulevard|Suite 410|Pleasant Valley|Hampden County|MA|03077|United States|-5|apartment| +13822|AAAAAAAAOPFDAAAA|140|Maple 1st|Drive|Suite 140|Ashland|Spencer County|KY|44244|United States|-5|apartment| +13823|AAAAAAAAPPFDAAAA|972|Sycamore |Pkwy|Suite P|Florence|Unicoi County|TN|33394|United States|-6|condo| +13824|AAAAAAAAAAGDAAAA|754|College North|Blvd|Suite 200|Sunnyside|Emmet County|MI|41952|United States|-5|single family| +13825|AAAAAAAABAGDAAAA|129|Hickory |Boulevard|Suite P|Riverside|Union County|MS|59231|United States|-6|condo| +13826|AAAAAAAACAGDAAAA|137|First |Road|Suite T|Brownsville|Deaf Smith County|TX|79310|United States|-6|single family| +13827|AAAAAAAADAGDAAAA|412|Ridge Willow|Parkway|Suite 470|Maple Grove|DeSoto County|MS|58252|United States|-6|apartment| +13828|AAAAAAAAEAGDAAAA|768|Birch |RD|Suite H|Spring Hill|Wagoner County|OK|76787|United States|-6|condo| +13829|AAAAAAAAFAGDAAAA|774|Washington Oak|Lane|Suite O|Hillcrest|Fleming County|KY|43003|United States|-6|apartment| +13830|AAAAAAAAGAGDAAAA|565|15th First|Boulevard|Suite X|Farmington|Barnes County|ND|59145|United States|-6|condo| +13831|AAAAAAAAHAGDAAAA|61|Lakeview 11th|Pkwy|Suite 370|Green Acres|Clallam County|WA|97683|United States|-8|apartment| +13832|AAAAAAAAIAGDAAAA|348|Hill Chestnut|Ct.|Suite 320|Sunnyside|Crow Wing County|MN|51952|United States|-6|condo| +13833|AAAAAAAAJAGDAAAA|755|Ninth View|Ct.|Suite 20|Antioch|Costilla County|CO|88605|United States|-7|single family| +13834|AAAAAAAAKAGDAAAA|533|Washington Cherry|Parkway|Suite 130|Riverside|Rusk County|TX|79231|United States|-6|single family| +13835|AAAAAAAALAGDAAAA|801|4th |Ct.|Suite 170|Lebanon|Metcalfe County|KY|42898|United States|-5|single family| +13836|AAAAAAAAMAGDAAAA|585|4th Center|Dr.|Suite 260|Woodville|Motley County|TX|74289|United States|-6|condo| +13837|AAAAAAAANAGDAAAA|||ST|Suite P||Knox County|KY|40059|United States|-5|| +13838|AAAAAAAAOAGDAAAA|316|15th |RD|Suite S|Cedar Grove|Lampasas County|TX|70411|United States|-6|single family| +13839|AAAAAAAAPAGDAAAA|286|Hickory Main|Parkway||Mountain View||OK|74466|United States||| +13840|AAAAAAAAABGDAAAA|883|8th Jackson|Way|Suite K|Lakeview|Medina County|OH|48579|United States|-5|apartment| +13841|AAAAAAAABBGDAAAA|792|Pine Jefferson|Lane|Suite 400|Deerfield|Renville County|MN|59840|United States|-6|apartment| +13842|AAAAAAAACBGDAAAA|238|7th Second|Court|Suite 250|Georgetown|Mason County|WA|97057|United States|-8|apartment| +13843|AAAAAAAADBGDAAAA|251|Franklin |Drive|Suite 220|Forest Hills|Jackson County|AL|39237|United States|-6|single family| +13844|AAAAAAAAEBGDAAAA|323|North |Road|Suite W|Highland|Lowndes County|GA|39454|United States|-5|apartment| +13845|AAAAAAAAFBGDAAAA|86|First |Cir.|Suite 100|Maple Grove|Columbia County|FL|38252|United States|-5|apartment| +13846|AAAAAAAAGBGDAAAA|945|Sycamore |Road|Suite U|Carter|Anderson County|SC|20919|United States|-5|single family| +13847|AAAAAAAAHBGDAAAA|940|Spruce |Cir.|Suite 460|Bridgeport|Whiteside County|IL|65817|United States|-6|single family| +13848|AAAAAAAAIBGDAAAA|965|Walnut |Dr.|Suite 190|Providence|Winnebago County|IL|66614|United States|-6|apartment| +13849|AAAAAAAAJBGDAAAA|39|Franklin |Ave|Suite P|Kingston|Guthrie County|IA|54975|United States|-6|condo| +13850|AAAAAAAAKBGDAAAA|953|Central Main|Ln|Suite 380|Midway|Hancock County|IN|41904|United States|-5|apartment| +13851|AAAAAAAALBGDAAAA|574|Forest |Avenue|Suite O|Wright|Greene County|MO|62814|United States|-6|apartment| +13852|AAAAAAAAMBGDAAAA|569|Green |Court|Suite 310|Bethel|Riley County|KS|65281|United States|-6|single family| +13853|AAAAAAAANBGDAAAA|654|Madison Ridge|Ct.|Suite 190|Bay View|Bee County|TX|76457|United States|-6|condo| +13854|AAAAAAAAOBGDAAAA|450|View Eigth|Street|Suite E|Glenwood|Cherry County|NE|63511|United States|-6|condo| +13855|AAAAAAAAPBGDAAAA|726|14th |Cir.|Suite 180|Woodland Park|Whatcom County|WA|91934|United States|-8|single family| +13856|AAAAAAAAACGDAAAA|643|Lake Meadow|Ln|Suite C|Morgantown|Defiance County|OH|49193|United States|-5|condo| +13857|AAAAAAAABCGDAAAA|222|Sixth |Court|Suite Y|Welcome|Garrett County|MD|26386|United States|-5|single family| +13858|AAAAAAAACCGDAAAA|573|View Smith|Ave|Suite U|Woodbury|Rockcastle County|KY|44489|United States|-5|apartment| +13859|AAAAAAAADCGDAAAA|384|Hill Pine|Road|Suite M|Glenwood|Chase County|NE|63511|United States|-6|apartment| +13860|AAAAAAAAECGDAAAA|513|Laurel Third|Drive|Suite A|Oak Grove|Marshall County|IN|48370|United States|-5|apartment| +13861|AAAAAAAAFCGDAAAA|469|Fourth Park|Pkwy|Suite S|Salem|Cherry County|NE|68048|United States|-6|single family| +13862|AAAAAAAAGCGDAAAA|434|Birch |Dr.|Suite 460|Farmington|Anderson County|KS|69145|United States|-6|apartment| +13863|AAAAAAAAHCGDAAAA|372|West |Dr.|Suite N|Springdale|Logan County|CO|88883|United States|-7|apartment| +13864|AAAAAAAAICGDAAAA|720|2nd |RD|Suite C|Buena Vista|Palm Beach County|FL|35752|United States|-5|apartment| +13865|AAAAAAAAJCGDAAAA|503|Johnson |Parkway|Suite D|Centerville|Cherokee County|OK|70059|United States|-6|single family| +13866|AAAAAAAAKCGDAAAA|304|Mill |Circle|Suite 210|Wyoming|Richardson County|NE|60216|United States|-7|condo| +13867|AAAAAAAALCGDAAAA|834|Fourth 15th|Blvd|Suite K|Clinton|Essex County|NJ|08822|United States|-5|single family| +13868|AAAAAAAAMCGDAAAA|428|Oak |Ln|Suite S|Pine Grove|Lincoln County|TN|34593|United States|-6|single family| +13869|AAAAAAAANCGDAAAA|624|Oak |RD|Suite 360|Ashland|Telfair County|GA|34244|United States|-5|apartment| +13870|AAAAAAAAOCGDAAAA|884|8th |Way|Suite O|Lebanon|McCormick County|SC|22898|United States|-5|apartment| +13871|AAAAAAAAPCGDAAAA|590|7th Cherry|Court|Suite K|Lebanon|Green County|WI|52898|United States|-6|condo| +13872|AAAAAAAAADGDAAAA|241|Park 10th|Pkwy|Suite 80|Shady Grove|Sublette County|WY|82812|United States|-7|condo| +13873|AAAAAAAABDGDAAAA|876|Jefferson Hillcrest|Cir.|Suite 110|Unionville|Sarpy County|NE|61711|United States|-7|condo| +13874|AAAAAAAACDGDAAAA|512|Jackson Ash|Avenue|Suite 200|Berea|Buchanan County|IA|53804|United States|-6|single family| +13875|AAAAAAAADDGDAAAA|547|Lake Ridge|Street|Suite L|Buena Vista|Fulton County|PA|15752|United States|-5|apartment| +13876|AAAAAAAAEDGDAAAA|846|1st |Circle|Suite O|Union|Columbia County|GA|38721|United States|-5|condo| +13877|AAAAAAAAFDGDAAAA|8|4th |Court|Suite L|Georgetown|Newton County|IN|47057|United States|-5|condo| +13878|AAAAAAAAGDGDAAAA|55|Ridge Pine|ST|Suite X|Lebanon|New London County|CT|03498|United States|-5|single family| +13879|AAAAAAAAHDGDAAAA|247|Lake |Boulevard|Suite 170|Fox|Hernando County|FL|30631|United States|-5|single family| +13880|AAAAAAAAIDGDAAAA|549|Main |Cir.|Suite 370|Riverview|Monroe County|MI|49003|United States|-5|apartment| +13881|AAAAAAAAJDGDAAAA|60|Park 6th|Lane|Suite 440|Maple Grove|Kern County|CA|98252|United States|-8|single family| +13882|AAAAAAAAKDGDAAAA|479|Walnut |Circle|Suite P|Woodlawn|Drew County|AR|74098|United States|-6|condo| +13883|AAAAAAAALDGDAAAA|899|Madison Jackson|Ln|Suite M|Pine Grove|Irion County|TX|74593|United States|-6|single family| +13884|AAAAAAAAMDGDAAAA|495|College Sunset|Ln|Suite U|Hamilton|Jackson County|WV|22808|United States|-5|apartment| +13885|AAAAAAAANDGDAAAA|496|Miller |Drive|Suite 400|Florence|Johnson County|IA|53394|United States|-6|condo| +13886|AAAAAAAAODGDAAAA|978|Pine |RD|Suite E|Crossroads|Arthur County|NE|60534|United States|-6|single family| +13887|AAAAAAAAPDGDAAAA|690|5th |Pkwy|Suite K|Clifton|Coosa County|AL|38014|United States|-6|apartment| +13888|AAAAAAAAAEGDAAAA|956|Central |Circle|Suite 200|Wildwood|Pasco County|FL|36871|United States|-5|single family| +13889|AAAAAAAABEGDAAAA|261|Lake |Drive|Suite O|Highland|Spencer County|IN|49454|United States|-5|condo| +13890|AAAAAAAACEGDAAAA|57|7th |Street|Suite X|Enterprise|Jefferson County|OH|41757|United States|-5|apartment| +13891|AAAAAAAADEGDAAAA|269|Adams College|Ln|Suite 470|Arlington|Knox County|OH|46557|United States|-5|single family| +13892|AAAAAAAAEEGDAAAA|727|Oak Wilson|Wy|Suite 200|Plainview|Ripley County|MO|63683|United States|-6|single family| +13893|AAAAAAAAFEGDAAAA|945|North View|Drive|Suite 80|Belmont|Grand County|CO|80191|United States|-7|apartment| +13894|AAAAAAAAGEGDAAAA|410|7th |Way|Suite 70|Providence|York County|SC|26614|United States|-5|single family| +13895|AAAAAAAAHEGDAAAA|193|7th Spring|Drive|Suite 260|Union|Upton County|TX|78721|United States|-6|condo| +13896|AAAAAAAAIEGDAAAA|69|Cherry |ST|Suite J|Bunker Hill|Hardeman County|TX|70150|United States|-6|apartment| +13897|AAAAAAAAJEGDAAAA|538|Oak Smith|Ave|Suite 90|Unionville|Walworth County|WI|51711|United States|-6|single family| +13898|AAAAAAAAKEGDAAAA|181|Birch Broadway|Court|Suite 330|Oak Grove|Jay County|IN|48370|United States|-5|apartment| +13899|AAAAAAAALEGDAAAA|9|Lee Washington|Wy|Suite P|Highland|Hancock County|ME|09454|United States|-5|condo| +13900|AAAAAAAAMEGDAAAA|592|Davis 13th|Road|Suite J|Cedar Grove|Custer County|ID|80411|United States|-7|apartment| +13901|AAAAAAAANEGDAAAA|526|2nd |Dr.|Suite 480|Edgewood|Greene County|PA|10069|United States|-5|single family| +13902|AAAAAAAAOEGDAAAA|241|Sunset |Boulevard|Suite 270|Mount Zion|Haywood County|TN|38054|United States|-5|apartment| +13903|AAAAAAAAPEGDAAAA|979|8th 6th|Road|Suite F|Whitney|Adams County|PA|18339|United States|-5|single family| +13904|AAAAAAAAAFGDAAAA|265|River |Circle|Suite 40|Hopewell|Baker County|OR|90587|United States|-8|single family| +13905|AAAAAAAABFGDAAAA|347|9th |Way|Suite 270|Mount Olive|Ohio County|KY|48059|United States|-5|condo| +13906|AAAAAAAACFGDAAAA|5|Maple |RD|Suite 290|Newport|Sioux County|ND|51521|United States|-6|apartment| +13907|AAAAAAAADFGDAAAA|775|Third |Boulevard|Suite 200|Stringtown|Bedford city|VA|20162|United States|-5|single family| +13908|AAAAAAAAEFGDAAAA|418|Second |Court|Suite U|Plainview|Morrow County|OR|93683|United States|-8|apartment| +13909|AAAAAAAAFFGDAAAA|639|Hickory |Lane|Suite 440|Walnut Grove|Lewis County|NY|17752|United States|-5|single family| +13910|AAAAAAAAGFGDAAAA|136|12th |Drive|Suite 320|Griffin|Jefferson Parish|LA|71204|United States|-6|single family| +13911|AAAAAAAAHFGDAAAA|698|Miller Main|Ave|Suite 300|Oak Hill|Riverside County|CA|97838|United States|-8|single family| +13912|AAAAAAAAIFGDAAAA|630|4th First|Ct.|Suite K|Crossroads|Garvin County|OK|70534|United States|-6|condo| +13913|AAAAAAAAJFGDAAAA|478|Jefferson |Way|Suite Y|Highland Park|Muscatine County|IA|56534|United States|-6|apartment| +13914|AAAAAAAAKFGDAAAA|930|Washington |Ct.|Suite 230|Valley View|Buffalo County|WI|55124|United States|-6|single family| +13915|AAAAAAAALFGDAAAA|453|Cedar |Road|Suite 70|Franklin|Dinwiddie County|VA|29101|United States|-5|apartment| +13916|AAAAAAAAMFGDAAAA|698|4th |Pkwy|Suite N|Jenkins|Prairie County|MT|67292|United States|-7|condo| +13917|AAAAAAAANFGDAAAA|449|Sixth Valley|Lane|Suite L|Woodville|Camden County|MO|64289|United States|-6|condo| +13918|AAAAAAAAOFGDAAAA|201|Woodland 5th|Ct.|Suite F|Providence|DeKalb County|GA|36614|United States|-5|apartment| +13919|AAAAAAAAPFGDAAAA|492|Park |Pkwy|Suite T|Mount Olive|Marion County|MS|58059|United States|-6|apartment| +13920|AAAAAAAAAGGDAAAA||||Suite T|Woodland||TX|74854||-6|| +13921|AAAAAAAABGGDAAAA|895|Oak |Ln|Suite 310|Bridgeport|Vermilion Parish|LA|75817|United States|-6|single family| +13922|AAAAAAAACGGDAAAA|481|Mill |Ave|Suite 110|Greenville|Alcona County|MI|41387|United States|-5|single family| +13923|AAAAAAAADGGDAAAA|691|Fourth 2nd|Way|Suite D|Highland|Washington County|MD|29454|United States|-5|apartment| +13924|AAAAAAAAEGGDAAAA|439|Sixth Cherry|RD|Suite 250|Macedonia|Nolan County|TX|71087|United States|-6|apartment| +13925|AAAAAAAAFGGDAAAA|122|2nd Maple|Boulevard|Suite 270|Glenwood|Pendleton County|KY|43511|United States|-5|single family| +13926|AAAAAAAAGGGDAAAA|610|Main |Pkwy|Suite M|Pleasant Valley|Blaine County|MT|62477|United States|-7|condo| +13927|AAAAAAAAHGGDAAAA|652|11th Lake|Road|Suite 160|Howell|Erie County|OH|44854||-5|apartment| +13928|AAAAAAAAIGGDAAAA|459|Washington Railroad|ST|Suite Q|Georgetown|Perry County|TN|37057|United States|-6|condo| +13929|AAAAAAAAJGGDAAAA|941|Maple Wilson|Dr.|Suite A|Green Acres|Barnstable County|MA|08283|United States|-5|apartment| +13930|AAAAAAAAKGGDAAAA|848|9th Railroad|Ave|Suite 310|Unionville|Shelby County|IL|61711|United States|-6|apartment| +13931|AAAAAAAALGGDAAAA|231|Lake |RD|Suite N|Red Hill|Walworth County|SD|54338|United States|-7|condo| +13932|AAAAAAAAMGGDAAAA|451|Cherry Franklin|Street|Suite 340|Pine Grove|Morrow County|OH|44593|United States|-5|condo| +13933|AAAAAAAANGGDAAAA|250|Center Cherry|Avenue|Suite 80|Newport|Wilkinson County|GA|31521|United States|-5|condo| +13934|AAAAAAAAOGGDAAAA|870|Hill Railroad|RD|Suite 170|Sulphur Springs|Pontotoc County|MS|58354|United States|-6|single family| +13935|AAAAAAAAPGGDAAAA|585|Cedar Washington|Way|Suite 80|Littleton|Nodaway County|MO|66074|United States|-6|apartment| +13936|AAAAAAAAAHGDAAAA|218|Elm |Parkway|Suite E|White Oak|Izard County|AR|76668|United States|-6|single family| +13937|AAAAAAAABHGDAAAA|684|Johnson |Ave|Suite Y|Sulphur Springs|Saline County|AR|78354|United States|-6|apartment| +13938|AAAAAAAACHGDAAAA|981|Cedar 4th|Blvd|Suite B|Springdale|Pawnee County|NE|68883|United States|-7|apartment| +13939|AAAAAAAADHGDAAAA|1|Ash Railroad|Cir.|Suite L|Snug Harbor|Ballard County|KY|47936|United States|-6|condo| +13940|AAAAAAAAEHGDAAAA|736|Hill Lincoln|Avenue|Suite J|Glenville|Lincoln County|KY|43445|United States|-5|apartment| +13941|AAAAAAAAFHGDAAAA|820|Ash |Lane|Suite G|Glendale|Teton County|MT|63951|United States|-7|condo| +13942|AAAAAAAAGHGDAAAA|264|North East|Wy|Suite 30|Mount Pleasant|Cherokee County|OK|71933|United States|-6|single family| +13943|AAAAAAAAHHGDAAAA|462|1st |Ln|Suite C|Farmington|Santa Barbara County|CA|99145|United States|-8|apartment| +13944|AAAAAAAAIHGDAAAA|||||Mount Vernon|||48482|||| +13945|AAAAAAAAJHGDAAAA|310|Johnson |Blvd|Suite R|Ashland|Clay County|SD|54244|United States|-6|apartment| +13946|AAAAAAAAKHGDAAAA|148|2nd 4th|Dr.|Suite B|Plainview|Harper County|KS|63683|United States|-6|single family| +13947|AAAAAAAALHGDAAAA|532||||Hamilton|||92808|United States||| +13948|AAAAAAAAMHGDAAAA|601|Park Birch|Dr.|Suite 30|Fairfield|Grayson County|VA|26192|United States|-5|apartment| +13949|AAAAAAAANHGDAAAA|195|Church |Ct.|Suite 340|Greenwood|Yazoo County|MS|58828|United States|-6|condo| +13950|AAAAAAAAOHGDAAAA|418|8th |Lane|Suite I|Buena Vista|Lancaster County|SC|25752|United States|-5|condo| +13951|AAAAAAAAPHGDAAAA|334|Second |Street|Suite 100|Bethel|Butler County|MO|65281|United States|-6|single family| +13952|AAAAAAAAAIGDAAAA|481|Ninth |Way|Suite 490|Pleasant Grove|Gillespie County|TX|74136|United States|-6|apartment| +13953|AAAAAAAABIGDAAAA|99|Elm |Dr.|Suite 360|Reno|Riley County|KS|60344|United States|-6|condo| +13954|AAAAAAAACIGDAAAA|521|Jackson West|Wy|Suite 330|Mount Pleasant|Bourbon County|KS|61933|United States|-6|condo| +13955|AAAAAAAADIGDAAAA|303|View Jefferson|Drive|Suite I|Red Bank|Rusk County|WI|54975|United States|-6|condo| +13956|AAAAAAAAEIGDAAAA|849|Maple |Court|Suite V|Bethel|Hendricks County|IN|45281|United States|-5|single family| +13957|AAAAAAAAFIGDAAAA|103|1st |Dr.|Suite G|Woodville|Adams County|ID|84289|United States|-7|single family| +13958|AAAAAAAAGIGDAAAA|1|Spruce Pine|Lane|Suite 310|Hillcrest|Glynn County|GA|33003|United States|-5|single family| +13959|AAAAAAAAHIGDAAAA|812|First Central|Street|Suite H|Brownsville|Limestone County|TX|79310|United States|-6|single family| +13960|AAAAAAAAIIGDAAAA|226|Second Lake|Lane|Suite U|Fairfield|Knott County|KY|46192|United States|-5|condo| +13961|AAAAAAAAJIGDAAAA|678|13th |ST|Suite 0|Brownsville|Delta County|MI|49310|United States|-5|apartment| +13962|AAAAAAAAKIGDAAAA|185|Birch |Avenue|Suite N|Belmont|Tolland County|CT|00791|United States|-5|apartment| +13963|AAAAAAAALIGDAAAA|754|Woodland |Court|Suite 330|Smith|Alleghany County|VA|27317|United States|-5|apartment| +13964|AAAAAAAAMIGDAAAA|914|Ash 5th|Avenue|Suite 470|Lakeview|Finney County|KS|68579|United States|-6|condo| +13965|AAAAAAAANIGDAAAA|287|North 9th|ST|Suite 210|Oakwood|Oswego County|NY|10169|United States|-5|apartment| +13966|AAAAAAAAOIGDAAAA|924|Sunset |Blvd|Suite X|Mount Vernon|McIntosh County|ND|58482|United States|-6|condo| +13967|AAAAAAAAPIGDAAAA|581|Broadway |Road|Suite 370|Highland Park|Richmond city|VA|26534|United States|-5|condo| +13968|AAAAAAAAAJGDAAAA|20|Pine |Boulevard|Suite 460|Texas|Clark County|WI|53342|United States|-6|condo| +13969|AAAAAAAABJGDAAAA|856|North |Way|Suite K|Greenwood|Rockdale County|GA|38828|United States|-5|apartment| +13970|AAAAAAAACJGDAAAA|451|8th |ST|Suite W|Harmony|Mercer County|ND|55804|United States|-6|single family| +13971|AAAAAAAADJGDAAAA|967|2nd 1st|Ct.|Suite 100|Lakewood|Elk County|KS|68877|United States|-6|condo| +13972|AAAAAAAAEJGDAAAA|981|North Elm|Pkwy|Suite 400|Wilson|Phelps County|MO|66971|United States|-6|condo| +13973|AAAAAAAAFJGDAAAA|876|Central South|ST|Suite 360|Owens|Aroostook County|ME|02924|United States|-5|apartment| +13974|AAAAAAAAGJGDAAAA|512|Fifth Locust|Wy|Suite 100|Waterloo|Holmes County|MS|51675|United States|-6|condo| +13975|AAAAAAAAHJGDAAAA|601|Maple 6th|Parkway|Suite U|Price|Anderson County|KY|43824|United States|-6|apartment| +13976|AAAAAAAAIJGDAAAA|605|Second |Parkway|Suite 100|Antioch|Coweta County|GA|38605|United States|-5|single family| +13977|AAAAAAAAJJGDAAAA|736|Wilson |Cir.|Suite S|Red Hill|Lane County|OR|94338|United States|-8|condo| +13978|AAAAAAAAKJGDAAAA|232|Lee Miller|Pkwy|Suite M|Valley View|Delaware County|OH|45124|United States|-5|single family| +13979|AAAAAAAALJGDAAAA|987|Main |Street|Suite 40|Clifton|Caroline County|MD|28014|United States|-5|apartment| +13980|AAAAAAAAMJGDAAAA|451|2nd |Circle|Suite 10|Pleasant Hill|Rusk County|TX|73604|United States|-6|condo| +13981|AAAAAAAANJGDAAAA|996|6th |Parkway|Suite 80|Oak Grove|Emmet County|IA|58370|United States|-6|apartment| +13982|AAAAAAAAOJGDAAAA|125|Williams Main|Pkwy|Suite Y|Farmington|Payette County|ID|89145|United States|-7|single family| +13983|AAAAAAAAPJGDAAAA|404|3rd |Street|Suite 360|Belmont|Ida County|IA|50191|United States|-6|single family| +13984|AAAAAAAAAKGDAAAA|195|Railroad Pine|Way|Suite 40|Spring Valley|Fayette County|AL|36060|United States|-6|apartment| +13985|AAAAAAAABKGDAAAA|996|Oak 5th|ST|Suite L|Bethel|Garfield County|OK|75281|United States|-6|single family| +13986|AAAAAAAACKGDAAAA|403|9th |Ct.|Suite 270|Clifford|Lea County|NM|88164|United States||| +13987|AAAAAAAADKGDAAAA|806||Dr.|Suite K||Alachua County||32477|United States||| +13988|AAAAAAAAEKGDAAAA|663|Church 14th|Avenue|Suite 160|Springdale|Eaton County|MI|48883|United States|-5|condo| +13989|AAAAAAAAFKGDAAAA|798|4th 8th|Drive|Suite V|Five Forks|Amador County|CA|92293|United States|-8|single family| +13990|AAAAAAAAGKGDAAAA|778|3rd Walnut|Street|Suite K|Oakland|Jenkins County|GA|39843|United States|-5|apartment| +13991|AAAAAAAAHKGDAAAA|622|7th 2nd|Boulevard|Suite 210|Sulphur Springs|Polk County|TX|78354|United States|-6|apartment| +13992|AAAAAAAAIKGDAAAA|812|Dogwood |Drive|Suite W|Red Hill|Warrick County|IN|44338|United States|-5|condo| +13993|AAAAAAAAJKGDAAAA|683|Chestnut |Cir.|Suite U|Macedonia|Cobb County|GA|31087|United States|-5|single family| +13994|AAAAAAAAKKGDAAAA|155|Third |RD|Suite P|Antioch|Barbour County|WV|28605|United States|-5|apartment| +13995|AAAAAAAALKGDAAAA|22|3rd |Road|Suite F|Stringtown|Valley County|NE|60162|United States|-7|condo| +13996|AAAAAAAAMKGDAAAA|278|Johnson |Dr.|Suite V|Concord|Okaloosa County|FL|34107|United States|-5|condo| +13997|AAAAAAAANKGDAAAA|38|Hill Maple|Boulevard|Suite 310|Franklin|Green County|WI|59101|United States|-6|single family| +13998|AAAAAAAAOKGDAAAA|627|Pine Wilson|Way|Suite X|Buena Vista|Clark County|AR|75752|United States|-6|single family| +13999|AAAAAAAAPKGDAAAA|85|Mill |Circle|Suite 250|Concord|Smith County|TN|34107|United States|-6|apartment| +14000|AAAAAAAAALGDAAAA|515|Smith |Cir.|Suite 360|Pleasant Grove|Polk County|AR|74136|United States|-6|single family| +14001|AAAAAAAABLGDAAAA|889|Chestnut Miller|Way|Suite S|Enterprise|Box Butte County|NE|61757|United States|-6|apartment| +14002|AAAAAAAACLGDAAAA|861|1st View|Parkway|Suite 140|Hamilton|Lubbock County|TX|72808|United States|-6|apartment| +14003|AAAAAAAADLGDAAAA|154|Spring |Avenue|Suite J|Sunnyside|Comal County|TX|71952|United States|-6|single family| +14004|AAAAAAAAELGDAAAA|934|Valley |Street|Suite 350|Chestnut Ridge|Wibaux County|MT|67334|United States|-7|single family| +14005|AAAAAAAAFLGDAAAA|227|Park Thirteenth|Dr.|Suite I|Mount Pleasant|Delaware County|IN|41933|United States|-5|single family| +14006|AAAAAAAAGLGDAAAA|450|Dogwood 9th|Ln|Suite 380|Glenwood|Poinsett County|AR|73511|United States|-6|single family| +14007|AAAAAAAAHLGDAAAA|542|Poplar 9th|Parkway|Suite O|Pleasant Grove|Essex County|NJ|04736|United States|-5|condo| +14008|AAAAAAAAILGDAAAA|640|3rd Washington|Circle|Suite 360|Deerfield|Sebastian County|AR|79840|United States|-6|single family| +14009|AAAAAAAAJLGDAAAA|367|1st Cherry|Cir.|Suite 490|Lee|Faulkner County|AR|70408|United States|-6|single family| +14010|AAAAAAAAKLGDAAAA|456|Lincoln 10th|Lane|Suite K|Red Hill|Wirt County|WV|24338|United States|-5|single family| +14011|AAAAAAAALLGDAAAA|114|Highland Main|Wy|Suite K|Shady Grove|Dewey County|OK|72812|United States|-6|single family| +14012|AAAAAAAAMLGDAAAA|522|6th |Avenue|Suite S|Fairview|Livingston County|NY|15709|United States|-5|apartment| +14013|AAAAAAAANLGDAAAA|869|13th |Road|Suite 320|Franklin|Schuyler County|MO|69101|United States|-6|condo| +14014|AAAAAAAAOLGDAAAA|297|Center 2nd|Circle|Suite 10|Silver Creek|Bandera County|TX|74546|United States|-6|single family| +14015|AAAAAAAAPLGDAAAA|757|Park |Court|Suite H|Riverview|Perry County|IN|49003|United States|-5|single family| +14016|AAAAAAAAAMGDAAAA|946|Pine |Wy|Suite T|Littleton|Lapeer County|MI|46074|United States|-5|condo| +14017|AAAAAAAABMGDAAAA|1000|Main |Blvd|Suite 280|Oak Hill|Keweenaw County|MI|47838|United States|-5|condo| +14018|AAAAAAAACMGDAAAA|600|9th Park|Street|Suite D|Greenwood|Wyandot County|OH|48828|United States|-5|apartment| +14019|AAAAAAAADMGDAAAA|558|Smith |Dr.|Suite 260|Midway|Bear Lake County|ID|81904|United States|-7|apartment| +14020|AAAAAAAAEMGDAAAA|378|6th |Road|Suite G|Mount Zion|Sterling County|TX|78054|United States|-6|apartment| +14021|AAAAAAAAFMGDAAAA|898|10th 13th|Wy|Suite E|Hopewell|Taylor County|IA|50587|United States|-6|single family| +14022|AAAAAAAAGMGDAAAA|440|Park |Circle|Suite 100|Oak Grove|Winneshiek County|IA|58370|United States|-6|apartment| +14023|AAAAAAAAHMGDAAAA|781|Hickory |Road|Suite R|Guthrie|Montgomery County|PA|11423|United States|-5|condo| +14024|AAAAAAAAIMGDAAAA|449|Spruce Hickory|Ave|Suite 10|Pleasant Valley|Sampson County|NC|22477|United States|-5|apartment| +14025|AAAAAAAAJMGDAAAA|639||Circle|||||55124|United States||apartment| +14026|AAAAAAAAKMGDAAAA|330|Ash 1st|Circle|Suite Q|Valley View|Spokane County|WA|95124|United States|-8|apartment| +14027|AAAAAAAALMGDAAAA|990|Dogwood Spring|Circle|Suite C|Antioch|Monroe County|AL|38605|United States|-6|single family| +14028|AAAAAAAAMMGDAAAA|66|8th |Road|Suite H|Belmont|Neshoba County|MS|50191|United States|-6|apartment| +14029|AAAAAAAANMGDAAAA|131|14th Lake|Road|Suite J|Pleasant Hill|Mahoning County|OH|43604|United States|-5|apartment| +14030|AAAAAAAAOMGDAAAA|565|Wilson 8th|Drive|Suite 300|Wilson|Ohio County|KY|46971|United States|-5|single family| +14031|AAAAAAAAPMGDAAAA|342|Lee Sunset|Circle|Suite 160|Pleasant Grove|Lamb County|TX|74136|United States|-6|single family| +14032|AAAAAAAAANGDAAAA|464|View |Road|Suite H|Fairfield|Harper County|KS|66192|United States|-6|apartment| +14033|AAAAAAAABNGDAAAA|936|Pine Mill|Lane|Suite F|Red Hill|Houston County|TN|34338|United States|-5|single family| +14034|AAAAAAAACNGDAAAA|641|||Suite D||Vernon Parish|LA||United States|-6|| +14035|AAAAAAAADNGDAAAA|747|4th Park|RD|Suite H|Sumner|Frontier County|NE|60519|United States|-6|apartment| +14036|AAAAAAAAENGDAAAA|665|Sixth |ST|Suite A|Antioch|Le Sueur County|MN|58605|United States|-6|condo| +14037|AAAAAAAAFNGDAAAA|606|Ash |Boulevard|Suite W|Hamilton|Bradley County|TN|32808|United States|-5|apartment| +14038|AAAAAAAAGNGDAAAA|820|Madison |Wy|Suite E|Mountain View|Henderson County|IL|64466|United States|-6|apartment| +14039|AAAAAAAAHNGDAAAA|485|View Fifth|RD|Suite N|Brownsville|Concho County|TX|79310|United States|-6|condo| +14040|AAAAAAAAINGDAAAA|334|Walnut Pine|Street|Suite X|Forest Hills|Franklin County|WA|99237|United States|-8|single family| +14041|AAAAAAAAJNGDAAAA|||Street|Suite Y|Maple Grove|||38252|United States|-5|| +14042|AAAAAAAAKNGDAAAA|812|Oak |Street|Suite 400|Crossroads|Perry County|TN|30534|United States|-6|single family| +14043|AAAAAAAALNGDAAAA|530|Spring Fourth|Avenue|Suite B|Scottsville|Muskegon County|MI|44190|United States|-5|single family| +14044|AAAAAAAAMNGDAAAA|1|1st |Lane|Suite 310|Mount Vernon|Beckham County|OK|78482|United States|-6|condo| +14045|AAAAAAAANNGDAAAA|446|Cherry |Court|Suite O|Sunnyside|Spokane County|WA|91952|United States|-8|apartment| +14046|AAAAAAAAONGDAAAA|406|Ridge |Avenue|Suite P|Ryan|Boone County|IL|60525|United States|-6|condo| +14047|AAAAAAAAPNGDAAAA|||Dr.|Suite U||||||-6|single family| +14048|AAAAAAAAAOGDAAAA|815|Oak |Ave|Suite C|Highland Park|Washington County|CO|86534|United States|-7|single family| +14049|AAAAAAAABOGDAAAA|745|Elm Fourth|Pkwy|Suite 300|Hopewell|Webster County|MO|60587|United States|-6|single family| +14050|AAAAAAAACOGDAAAA|105|Cedar |Drive|Suite N|Wilson|Hernando County|FL|36971|United States|-5|apartment| +14051|AAAAAAAADOGDAAAA|228|Miller Madison|RD|Suite P|Green Acres|Shelby County|IN|47683|United States|-5|apartment| +14052|AAAAAAAAEOGDAAAA|||Lane|Suite U||San Juan County||98014|||condo| +14053|AAAAAAAAFOGDAAAA|584|Spring |Avenue|Suite 80|Cedar Grove|Winnebago County|WI|50411|United States|-6|apartment| +14054|AAAAAAAAGOGDAAAA|319|11th Green|Street|Suite 430|Glendale|Wilcox County|AL|33951|United States|-6|apartment| +14055|AAAAAAAAHOGDAAAA|960|Hill Jackson|Dr.|Suite U|Woodland|Walworth County|SD|54854|United States|-7|apartment| +14056|AAAAAAAAIOGDAAAA|209|First 11th|Street|Suite S|Pleasant Grove|Cumberland County|IL|64136|United States|-6|single family| +14057|AAAAAAAAJOGDAAAA|467|Franklin Oak|Pkwy|Suite 410|Jackson|Bremer County|IA|59583|United States|-6|single family| +14058|AAAAAAAAKOGDAAAA||||Suite 220||||80162||-7|apartment| +14059|AAAAAAAALOGDAAAA|698|Spring 1st|Street|Suite 70|Bunker Hill|San Patricio County|TX|70150|United States|-6|condo| +14060|AAAAAAAAMOGDAAAA|977|4th |Circle|Suite U|Green Acres|Assumption Parish|LA|77683|United States|-6|single family| +14061|AAAAAAAANOGDAAAA|685|Laurel |Avenue|Suite 110|Red Hill|Mineral County|MT|64338|United States|-7|apartment| +14062|AAAAAAAAOOGDAAAA|660|Williams Highland|Ave|Suite 480|Clifton|Brevard County|FL|38014|United States|-5|single family| +14063|AAAAAAAAPOGDAAAA|628|Maple Lakeview|ST|Suite 0|Centerville|Flathead County|MT|60059|United States|-7|apartment| +14064|AAAAAAAAAPGDAAAA|502|2nd Highland|RD|Suite 420|Fisher|Putnam County|MO|62819|United States|-6|apartment| +14065|AAAAAAAABPGDAAAA|479|Wilson Cherry|Road|Suite 0|Fairfield|Middlesex County|VA|26192|United States|-5|condo| +14066|AAAAAAAACPGDAAAA|931|Park |Parkway|Suite 170|Farmington|Mille Lacs County|MN|59145|United States|-6|apartment| +14067|AAAAAAAADPGDAAAA|862|Sunset |Avenue|Suite 360|Fairview|Calhoun County|IL|65709|United States|-6|single family| +14068|AAAAAAAAEPGDAAAA|625|View Birch|Street|Suite A|Riverview|Los Angeles County|CA|99003|United States|-8|condo| +14069|AAAAAAAAFPGDAAAA|908|Spruce Hillcrest|Pkwy|Suite 420|Woodland|Atkinson County|GA|34854|United States|-5|apartment| +14070|AAAAAAAAGPGDAAAA|333|Oak |RD|Suite X|Hillcrest|Boone County|KY|43003|United States|-6|condo| +14071|AAAAAAAAHPGDAAAA|565|Spruce Williams|Blvd|Suite 130|Delmar|Alger County|MI|43957|United States|-5|apartment| +14072|AAAAAAAAIPGDAAAA|720|7th |Ct.|Suite 290|Concord|Davidson County|NC|24107|United States|-5|apartment| +14073|AAAAAAAAJPGDAAAA|580|13th |ST|Suite Y|Wilson|Custer County|MT|66971|United States|-7|condo| +14074|AAAAAAAAKPGDAAAA|228|Dogwood |Ave|Suite 30|Bloomingdale|Hamilton County|TX|71824|United States|-6|condo| +14075|AAAAAAAALPGDAAAA|314|Hill View|Drive|Suite A|Edgewood|Taylor County|KY|40069|United States|-5|condo| +14076|AAAAAAAAMPGDAAAA|245|Forest |Court|Suite 70|Pleasant Valley|Door County|WI|52477|United States|-6|apartment| +14077|AAAAAAAANPGDAAAA|232|Madison |Circle|Suite B|Stringtown|Upshur County|TX|70162|United States|-6|single family| +14078|AAAAAAAAOPGDAAAA|145|11th |Boulevard|Suite 180|Tracy|Atchison County|KS|66340|United States|-6|apartment| +14079|AAAAAAAAPPGDAAAA||3rd 4th||Suite H|||TX||United States||| +14080|AAAAAAAAAAHDAAAA|292||Drive||Pleasant Grove|Alcona County||44136|||| +14081|AAAAAAAABAHDAAAA|131|Spruce Elm|Parkway|Suite 380|Wildwood|Wichita County|TX|76871|United States|-6|apartment| +14082|AAAAAAAACAHDAAAA|793|Pine |Wy|Suite 180|Jackson|Beaverhead County|MT|69583|United States|-7|condo| +14083|AAAAAAAADAHDAAAA|187|Woodland |RD|Suite X|Pleasant Grove|Park County|MT|64136|United States|-7|condo| +14084|AAAAAAAAEAHDAAAA|54|Laurel Hickory|Dr.|Suite Y|Shady Grove|Stephens County|TX|72812|United States|-6|apartment| +14085|AAAAAAAAFAHDAAAA|291|Valley |||Belmont||KY|40191|||apartment| +14086|AAAAAAAAGAHDAAAA|724|Jefferson |RD|Suite P|Fairview|Pine County|MN|55709|United States|-6|apartment| +14087|AAAAAAAAHAHDAAAA|106|Spruce East|Parkway|Suite H|Kingston|Malheur County|OR|94975|United States|-8|apartment| +14088|AAAAAAAAIAHDAAAA|27|North |Parkway|Suite 310|Tremont|Leake County|MS|59515|United States|-6|apartment| +14089|AAAAAAAAJAHDAAAA|992|Dogwood 10th|Ave|Suite H|Marion|Emmet County|IA|50399|United States|-6|condo| +14090|AAAAAAAAKAHDAAAA|932|College |Ct.|Suite M|Stringtown|Stevens County|WA|90162|United States|-8|apartment| +14091|AAAAAAAALAHDAAAA|274|View Cedar|Court|Suite 340|Valley View|Pacific County|WA|95124|United States|-8|condo| +14092|AAAAAAAAMAHDAAAA|804|Spruce |Boulevard|Suite N|Macedonia|Duplin County|NC|21087|United States|-5|apartment| +14093|AAAAAAAANAHDAAAA|904|Willow |Parkway|Suite 140|Salem|Santa Fe County|NM|88048|United States|-7|condo| +14094|AAAAAAAAOAHDAAAA||1st |Cir.|Suite 270|||SD||||apartment| +14095|AAAAAAAAPAHDAAAA|735|Railroad View|Dr.|Suite 300|Georgetown|Trego County|KS|67057|United States|-6|apartment| +14096|AAAAAAAAABHDAAAA|174|8th |RD|Suite W|Oakdale|Jefferson County|FL|39584|United States|-5|condo| +14097|AAAAAAAABBHDAAAA|807|Smith |Dr.|Suite N|Andover|Lexington city|VA|21639|United States|-5|apartment| +14098|AAAAAAAACBHDAAAA|305|Cherry |Ct.|Suite P|Arlington|Tuscola County|MI|46557|United States|-5|single family| +14099|AAAAAAAADBHDAAAA|954|5th |Court|Suite 220|Leesville|Franklin County|MS|55423|United States|-6|apartment| +14100|AAAAAAAAEBHDAAAA|177|First |ST|Suite D|Edgewood|Fayette County|TX|70069|United States|-6|single family| +14101|AAAAAAAAFBHDAAAA|608|Woodland Hickory|Circle|Suite O|Stringtown|Strafford County|NH|00762|United States|-5|apartment| +14102|AAAAAAAAGBHDAAAA|509|Broadway Walnut|Drive|Suite 180|Bridgeport|Boone County|AR|75817|United States|-6|condo| +14103|AAAAAAAAHBHDAAAA|168|Spring Park|Parkway|Suite A|Clifton|Jackson County|MS|58014|United States|-6|apartment| +14104|AAAAAAAAIBHDAAAA|878|Second Meadow|Blvd|Suite 490|Glenwood|Lafayette County|MO|63511|United States|-6|condo| +14105|AAAAAAAAJBHDAAAA|532|River Birch|Avenue|Suite 440|Allison|Lowndes County|AL|34167|United States|-6|single family| +14106|AAAAAAAAKBHDAAAA|134|Hill |RD|Suite S|Jackson|Laurel County|KY|49583|United States|-5|single family| +14107|AAAAAAAALBHDAAAA|400|Main Maple|Way|Suite E|Pleasant Hill|Greene County|PA|13604|United States|-5|condo| +14108|AAAAAAAAMBHDAAAA|465|West Pine|Lane|Suite I|Wildwood|Kearny County|KS|66871|United States|-6|single family| +14109|AAAAAAAANBHDAAAA|221|College |Ct.|Suite J|Crossroads|Merrimack County|NH|01134|United States|-5|single family| +14110|AAAAAAAAOBHDAAAA|215|15th |Wy|Suite T|Macedonia|Love County|OK|71087|United States|-6|single family| +14111|AAAAAAAAPBHDAAAA|239|Hill Meadow|Dr.|Suite 110|Cedar Grove|Peoria County|IL|60411|United States|-6|single family| +14112|AAAAAAAAACHDAAAA|604|Hillcrest |Pkwy|Suite 450|Five Forks|Athens County|OH|42293|United States|-5|single family| +14113|AAAAAAAABCHDAAAA|741|College |Court|Suite E|Friendship|Shelby County|AL|34536|United States|-6|apartment| +14114|AAAAAAAACCHDAAAA|326|5th |Dr.|Suite J|Hamilton|Finney County|KS|62808|United States|-6|condo| +14115|AAAAAAAADCHDAAAA|967|Lee |Ct.|Suite 350|Waterloo|Wallowa County|OR|91675|United States|-8|apartment| +14116|AAAAAAAAECHDAAAA|764|Oak |Circle|Suite V|Marion|Berkeley County|SC|20399|United States|-5|apartment| +14117|AAAAAAAAFCHDAAAA|157|Main Lake|RD|Suite 420|Mount Zion|Tattnall County|GA|38054|United States|-5|apartment| +14118|AAAAAAAAGCHDAAAA|193|Chestnut 13th|Drive|Suite 100|Allison|Arapahoe County|CO|84167|United States|-7|single family| +14119|AAAAAAAAHCHDAAAA|153|Jefferson |ST|Suite 280|Glendale|Allegheny County|PA|13951|United States|-5|apartment| +14120|AAAAAAAAICHDAAAA||2nd Jackson||||Iron County|||||single family| +14121|AAAAAAAAJCHDAAAA|778|Main Davis|Ave|Suite H|Salem|Elkhart County|IN|48048|United States|-5|condo| +14122|AAAAAAAAKCHDAAAA||10th Oak||Suite 50|||||||| +14123|AAAAAAAALCHDAAAA|356|Railroad |Street|Suite T|Avoca|Big Horn County|MT|60540|United States|-7|condo| +14124|AAAAAAAAMCHDAAAA|560|Lee |Dr.|Suite 300|Union|Bradley County|TN|38721|United States|-5|apartment| +14125|AAAAAAAANCHDAAAA|215|Madison Church|Way|Suite 190|Salem|Woodbury County|IA|58048|United States|-6|single family| +14126|AAAAAAAAOCHDAAAA|265|7th |Ln|Suite B|Enterprise|Moniteau County|MO|61757|United States|-6|single family| +14127|AAAAAAAAPCHDAAAA|507|Willow |Avenue|Suite Y|Green Acres|Hansford County|TX|77683|United States|-6|apartment| +14128|AAAAAAAAADHDAAAA|743|3rd 7th|Drive|Suite 180|Macedonia|Catahoula Parish|LA|71087|United States|-6|condo| +14129|AAAAAAAABDHDAAAA|72|Central |Street|Suite E|Belleville|Jasper County|IA|52924|United States|-6|condo| +14130|AAAAAAAACDHDAAAA|369|Jefferson Forest|Pkwy|Suite 250|Salem|Nacogdoches County|TX|78048|United States|-6|single family| +14131|AAAAAAAADDHDAAAA|89|Pine Broadway|Ave|Suite S|Greenwood|Pinellas County|FL|38828|United States|-5|apartment| +14132|AAAAAAAAEDHDAAAA|41|Ridge River|Avenue|Suite 170|Lakeside|Fulton County|NY|19532|United States|-5|apartment| +14133|AAAAAAAAFDHDAAAA|93|6th |Road|Suite 200|Green Acres|Adams County|MS|57683|United States|-6|condo| +14134|AAAAAAAAGDHDAAAA|764|Laurel Valley|Boulevard|Suite 410|Spring Hill|Mitchell County|IA|56787|United States|-6|condo| +14135|AAAAAAAAHDHDAAAA|569|Maple 11th|Court|Suite V|Oak Hill|Wayne County|IA|57838|United States|-6|condo| +14136|AAAAAAAAIDHDAAAA|725|1st |Ave|Suite 40|Providence|Traill County|ND|56614|United States|-6|single family| +14137|AAAAAAAAJDHDAAAA|845|Main South|Boulevard|Suite 360|Pine Grove|Carroll County|NH|05193|United States|-5|apartment| +14138|AAAAAAAAKDHDAAAA|578|Meadow |Parkway|Suite 130|Shiloh|Rockland County|NY|19275|United States|-5|apartment| +14139|AAAAAAAALDHDAAAA|883|11th View|Ave|Suite X|Green Acres|Tuscola County|MI|47683|United States|-5|single family| +14140|AAAAAAAAMDHDAAAA|562|Seventh Elm|Boulevard|Suite 480|Stringtown|Manitowoc County|WI|50162|United States|-6|condo| +14141|AAAAAAAANDHDAAAA|300|Hill |Ln|Suite Q|Walnut Grove|Bennett County|SD|57752|United States|-6|single family| +14142|AAAAAAAAODHDAAAA|898|Sixth |Avenue||Highland Park|Ochiltree County|||United States||single family| +14143|AAAAAAAAPDHDAAAA|678|Lake |Boulevard|Suite 430|Unionville|Perry County|PA|11711|United States|-5|single family| +14144|AAAAAAAAAEHDAAAA|283|Main Broadway|Way|Suite A|Enterprise|Socorro County|NM|81757|United States|-7|apartment| +14145|AAAAAAAABEHDAAAA|925|Pine Main|Cir.|Suite U|Arlington|Ohio County|IN|46557|United States|-5|condo| +14146|AAAAAAAACEHDAAAA|991|Adams |RD|Suite P|Oak Ridge|Juneau County|WI|58371|United States|-6|apartment| +14147|AAAAAAAADEHDAAAA|340|Elm |Ln|Suite 160|Pine Grove|Jackson County|MN|54593|United States|-6|apartment| +14148|AAAAAAAAEEHDAAAA|271|Pine Park|Pkwy|Suite 340|Midway|Knox County|KY|41904|United States|-5|single family| +14149|AAAAAAAAFEHDAAAA|416|Locust |Street|Suite T|Bethel|Macoupin County|IL|65281|United States|-6|single family| +14150|AAAAAAAAGEHDAAAA|||Ln|||Crawford County|IL||United States|-6|condo| +14151|AAAAAAAAHEHDAAAA|703|Birch Mill|Court|Suite H|Red Hill|Hinds County|MS|54338|United States|-6|apartment| +14152|AAAAAAAAIEHDAAAA|466|Park 4th|Avenue|Suite L|Jackson|Wyoming County|WV|29583|United States|-5|apartment| +14153|AAAAAAAAJEHDAAAA|975|Sycamore 13th|Road|Suite 410|Wildwood|Shelby County|MO|66871|United States|-6|single family| +14154|AAAAAAAAKEHDAAAA|889|7th Park|Circle|Suite 120|Brunswick|Lake County|CO|84642|United States|-7|condo| +14155|AAAAAAAALEHDAAAA|893|Miller |Dr.|Suite 480|Post Oak|Juab County|UT|88567|United States|-7|single family| +14156|AAAAAAAAMEHDAAAA|383|Wilson Smith|Drive||Plainview||||United States||apartment| +14157|AAAAAAAANEHDAAAA|648|Willow 2nd|Road|Suite 150|Woodbury|Greeley County|NE|64489|United States|-6|single family| +14158|AAAAAAAAOEHDAAAA|999|Hill |Wy|Suite G|Lakeview|Washburn County|WI|58579|United States|-6|single family| +14159|AAAAAAAAPEHDAAAA|250|Smith Cherry|ST|Suite 400|Hillcrest|Carver County|MN|53003|United States|-6|apartment| +14160|AAAAAAAAAFHDAAAA|336|Chestnut |Street|Suite 40|Glenwood|Corson County|SD|53511|United States|-6|condo| +14161|AAAAAAAABFHDAAAA|901|Spring 3rd|Pkwy|Suite 320|Kingston|McCook County|SD|54975|United States|-7|condo| +14162|AAAAAAAACFHDAAAA|888|Church 6th|Way|Suite 430|New Hope|Indiana County|PA|19431|United States|-5|apartment| +14163|AAAAAAAADFHDAAAA|296|Cedar |Circle|Suite 320|Waterloo|Page County|IA|51675|United States|-6|single family| +14164|AAAAAAAAEFHDAAAA|315|Railroad Lincoln|Avenue|Suite 260|Five Points|Rutland County|VT|06698|United States|-5|single family| +14165|AAAAAAAAFFHDAAAA|618||||Deerfield|Wayne County|||||| +14166|AAAAAAAAGFHDAAAA|391|College |Parkway|Suite 390|Florence|Douglas County|NV|83394|United States|-8|apartment| +14167|AAAAAAAAHFHDAAAA|425|Third |Dr.|Suite B|Buena Vista|Calumet County|WI|55752|United States|-6|single family| +14168|AAAAAAAAIFHDAAAA|||ST|Suite 430|Bayview|Perry County|AR|79672|United States||condo| +14169|AAAAAAAAJFHDAAAA|730|5th |Circle|Suite N|Mount Zion|Big Horn County|MT|68054|United States|-7|condo| +14170|AAAAAAAAKFHDAAAA|161|Sixth 6th|Way|Suite 250|Ashland|Skagit County|WA|94244|United States|-8|apartment| +14171|AAAAAAAALFHDAAAA|455|Second |Way|Suite 200|Pleasant Hill|Forest County|WI|53604|United States|-6|single family| +14172|AAAAAAAAMFHDAAAA|947|Chestnut |Ave|Suite 370|Greenville|Osborne County|KS|61387|United States|-6|single family| +14173|AAAAAAAANFHDAAAA|11|Park |Circle|Suite 210|Macedonia|Pondera County|MT|61087|United States|-7|apartment| +14174|AAAAAAAAOFHDAAAA|214|Spring |Cir.|Suite 410|Greenville|McPherson County|KS|61387|United States|-6|apartment| +14175|AAAAAAAAPFHDAAAA|86|6th |RD|Suite C|Walnut Grove|Benton County|IN|47752|United States|-5|condo| +14176|AAAAAAAAAGHDAAAA|612|Smith Lakeview|Drive|Suite 250|Thompsonville|Monroe County|AR|79651|United States|-6|apartment| +14177|AAAAAAAABGHDAAAA|347|Maple |ST|Suite X|Clifton|Placer County|CA|98014|United States|-8|apartment| +14178|AAAAAAAACGHDAAAA|626|Sycamore 2nd|Ln|Suite J|Oakdale|Mahaska County|IA|59584|United States|-6|condo| +14179|AAAAAAAADGHDAAAA|344|3rd Valley|Lane|Suite 210|Lewis|Cheboygan County|MI|47066|United States|-5|apartment| +14180|AAAAAAAAEGHDAAAA|923|Willow Madison|Ct.|Suite A|Hillcrest|Hancock County|TN|33003|United States|-5|single family| +14181|AAAAAAAAFGHDAAAA|164|Maple Lake|Wy|Suite P|Woodville|Kanabec County|MN|54289|United States|-6|single family| +14182|AAAAAAAAGGHDAAAA|700|8th |Drive|Suite Q|Farmington|Queen Anne County|MD|29145|United States|-5|apartment| +14183|AAAAAAAAHGHDAAAA|403|2nd Jefferson|ST|Suite B|Concord|Caroline County|VA|24107|United States|-5|condo| +14184|AAAAAAAAIGHDAAAA|372|Walnut Sunset|Wy|Suite W|Clifton|Craven County|NC|28014|United States|-5|condo| +14185|AAAAAAAAJGHDAAAA||||Suite 50|Lebanon|||52898|||| +14186|AAAAAAAAKGHDAAAA|901|West |Lane|Suite 330|Williamsville|Eastland County|TX|78754|United States|-6|single family| +14187|AAAAAAAALGHDAAAA|660|14th |RD|Suite Q|Walnut Grove|Rockbridge County|VA|27752|United States|-5|apartment| +14188|AAAAAAAAMGHDAAAA|184|Oak Washington|Boulevard|Suite 90|Oak Hill|Ritchie County|WV|27838|United States|-5|single family| +14189|AAAAAAAANGHDAAAA|75|Sunset View|Ave|Suite U|Liberty|McCulloch County|TX|73451|United States|-6|condo| +14190|AAAAAAAAOGHDAAAA|838|West |Cir.|Suite F|Pleasant Grove|Nuckolls County|NE|64136|United States|-7|single family| +14191|AAAAAAAAPGHDAAAA|4|Lee |Ave|Suite H|Union Hill|Cherokee County|TX|77746|United States|-6|apartment| +14192|AAAAAAAAAHHDAAAA|809|8th |Cir.|Suite J|Blair|Fairfield County|CT|06065|United States|-5|single family| +14193|AAAAAAAABHHDAAAA|338|Laurel Green|Dr.|Suite 220|Oakwood|Dewey County|OK|70169|United States|-6|single family| +14194|AAAAAAAACHHDAAAA|559|4th |RD|Suite 70|Clifton|McCone County|MT|68014|United States|-7|condo| +14195|AAAAAAAADHHDAAAA|357|Lake West|Boulevard|Suite D|Friendship|Wicomico County|MD|24536|United States|-5|condo| +14196|AAAAAAAAEHHDAAAA|753|Church |Cir.|Suite M|Georgetown|Meade County|SD|57057|United States|-7|condo| +14197|AAAAAAAAFHHDAAAA|736|Franklin |Ln|Suite 10|Sunnyside|Arlington County|VA|21952|United States|-5|apartment| +14198|AAAAAAAAGHHDAAAA|181|South Forest|Parkway|Suite 170|Westminster|Schleicher County|TX|76549|United States|-6|single family| +14199|AAAAAAAAHHHDAAAA|879|Elm |Dr.|Suite 100|Hubbard|Trousdale County|TN|36291|United States|-6|condo| +14200|AAAAAAAAIHHDAAAA|716|Park Broadway|Ave|Suite 420|Oakland|Halifax County|NC|29843|United States|-5|condo| +14201|AAAAAAAAJHHDAAAA|567|Washington Lee|Ct.|Suite C|White Oak|Morgan County|OH|46668|United States|-5|single family| +14202|AAAAAAAAKHHDAAAA|171|Park |Court|Suite X|Lakeview|Early County|GA|38579|United States|-5|apartment| +14203|AAAAAAAALHHDAAAA|378|Woodland Johnson|ST|Suite 370|Summerville|Morris County|TX|72033|United States|-6|condo| +14204|AAAAAAAAMHHDAAAA|74|Valley |Parkway|Suite J|Buena Vista|Wayne County|IN|45752|United States|-5|single family| +14205|AAAAAAAANHHDAAAA|467|Railroad |Ave|Suite 40|Riverview|Marengo County|AL|39003|United States|-6|condo| +14206|AAAAAAAAOHHDAAAA|696|Fourth Smith|Ct.|Suite X|Salem|Washington County|RI|08648|United States|-5|apartment| +14207|AAAAAAAAPHHDAAAA|136|13th 4th|Cir.|Suite 270|Lincoln|West Carroll Parish|LA|71289|United States|-6|apartment| +14208|AAAAAAAAAIHDAAAA|293|4th Pine|Way|Suite I|Wildwood|Kitsap County|WA|96871|United States|-8|single family| +14209|AAAAAAAABIHDAAAA|609|12th |Cir.|Suite 350|Enterprise|Mohave County|AZ|81757|United States|-7|condo| +14210|AAAAAAAACIHDAAAA|3|Main |Parkway|Suite O|Franklin|Duplin County|NC|29101|United States|-5|condo| +14211|AAAAAAAADIHDAAAA|717|Elm |Drive|Suite 90|Cedar Grove|Talbot County|GA|30411|United States|-5|condo| +14212|AAAAAAAAEIHDAAAA|690|Main Ridge|Lane|Suite 360|Friendship|Simpson County|MS|54536|United States|-6|apartment| +14213|AAAAAAAAFIHDAAAA||||Suite 340|Concord||IL|64107|||apartment| +14214|AAAAAAAAGIHDAAAA|846|Johnson Fourth|Ln|Suite A|White Oak|Independence County|AR|76668|United States|-6|condo| +14215|AAAAAAAAHIHDAAAA|24|Pine |Ave|Suite Y|Farmington|Williamson County|IL|69145|United States|-6|condo| +14216|AAAAAAAAIIHDAAAA|34|Pine |Parkway|Suite 390|Pine Grove|Palm Beach County|FL|34593|United States|-5|apartment| +14217|AAAAAAAAJIHDAAAA|770|Pine Oak|Lane|Suite 490|Lebanon|Swift County|MN|52898|United States|-6|single family| +14218|AAAAAAAAKIHDAAAA|526|Pine Broadway|Boulevard|Suite Y|Lakeside|Cherokee County|TX|79532|United States|-6|single family| +14219|AAAAAAAALIHDAAAA|414|Park Birch|Way|Suite H|Springfield|Brown County|NE|69303|United States|-6|single family| +14220|AAAAAAAAMIHDAAAA|756|Cedar |Drive|Suite G|Springfield|Lake County|SD|59303|United States|-7|apartment| +14221|AAAAAAAANIHDAAAA|530|Lincoln |Way|Suite X|Friendship|Madison County|AR|74536|United States|-6|apartment| +14222|AAAAAAAAOIHDAAAA|764|Smith Chestnut|Ln|Suite 190|Oakdale|Putnam County|GA|39584|United States|-5|condo| +14223|AAAAAAAAPIHDAAAA|696|3rd Adams|Circle|Suite E|Fillmore|Sweetwater County|WY|88556|United States|-7|condo| +14224|AAAAAAAAAJHDAAAA|548|2nd |Pkwy|Suite I|Five Points|La Crosse County|WI|56098|United States|-6|single family| +14225|AAAAAAAABJHDAAAA|971|Walnut Ridge|Circle|Suite H|Forest Hills|Mayes County|OK|79237|United States|-6|condo| +14226|AAAAAAAACJHDAAAA|939|Franklin |Road|Suite 90|Union|Calhoun County|WV|28721|United States|-5|apartment| +14227|AAAAAAAADJHDAAAA|300|Ninth 6th|ST|Suite W|Oakland|Washington County|ME|09843|United States|-5|apartment| +14228|AAAAAAAAEJHDAAAA|644|Broadway 6th|Avenue|Suite 70|Buena Vista|Marion County|MO|65752|United States|-6|single family| +14229|AAAAAAAAFJHDAAAA|393|6th Locust|Blvd|Suite 240|Edgewood|Newton County|MS|50069|United States|-6|single family| +14230|AAAAAAAAGJHDAAAA|337|Sycamore Second|Pkwy|Suite 280|Oakland|Rockbridge County|VA|29843|United States|-5|condo| +14231|AAAAAAAAHJHDAAAA|374|Park |ST|Suite 360|Lebanon|Allen Parish|LA|72898|United States|-6|condo| +14232|AAAAAAAAIJHDAAAA|401|Elm Lincoln|Parkway|Suite S|Buckingham|Columbia County|WI|54092|United States|-6|apartment| +14233|AAAAAAAAJJHDAAAA|134|Main 12th|Cir.|Suite 250|||TN||United States|-5|condo| +14234|AAAAAAAAKJHDAAAA|885|Meadow |Parkway|Suite P|Concord|White County|IL|64107|United States|-6|condo| +14235|AAAAAAAALJHDAAAA|19|Mill Park|||Georgetown|||27057|United States|-5|| +14236|AAAAAAAAMJHDAAAA|277|Wilson |Ave|Suite N|Newport|Summit County|CO|81521|United States|-7|single family| +14237|AAAAAAAANJHDAAAA|53|Cedar |Cir.|Suite 160|Milo|York County|SC|20116|United States|-5|apartment| +14238|AAAAAAAAOJHDAAAA|543|Sunset Locust|ST|Suite U|Fairview|Martin County|NC|25709|United States|-5|apartment| +14239|AAAAAAAAPJHDAAAA|824|Hillcrest |ST|Suite 60|Woodland|Claiborne County|TN|34854|United States|-5|condo| +14240|AAAAAAAAAKHDAAAA|270|Central |Ave|Suite G|Ashland|Pike County|MO|64244|United States|-6|single family| +14241|AAAAAAAABKHDAAAA|653|Elm Green|Street||Georgetown|Grand Forks County|ND|57057||-6|| +14242|AAAAAAAACKHDAAAA|54|Pine |Circle|Suite V|Liberty|Marengo County|AL|33451|United States|-6|apartment| +14243|AAAAAAAADKHDAAAA|391|1st Franklin|Drive|Suite 480|Darlington|Cambria County|PA|15167|United States|-5|condo| +14244|AAAAAAAAEKHDAAAA|466|Main 3rd|Avenue|Suite J|Riverside|Oregon County|MO|69231|United States|-6|single family| +14245|AAAAAAAAFKHDAAAA|105|Williams 11th|Avenue|Suite 470|Sulphur Springs|Washington County|NY|18354|United States|-5|single family| +14246|AAAAAAAAGKHDAAAA|765|Main |Street|Suite L|Crossroads|Fayette County|TN|30534|United States|-5|single family| +14247|AAAAAAAAHKHDAAAA|362||Blvd|Suite 100||Potter County|||||| +14248|AAAAAAAAIKHDAAAA|862|Elm Wilson|ST|Suite T|Kimball|Hardin County|TX|73595|United States|-6|single family| +14249|AAAAAAAAJKHDAAAA|263|Spruce Forest|Drive|Suite W|Shelby|Lincoln County|MT|66575|United States|-7|apartment| +14250|AAAAAAAAKKHDAAAA|417|Elevnth Park|Road|Suite U|Walnut Grove|Webb County|TX|77752|United States|-6|condo| +14251|AAAAAAAALKHDAAAA|151|Lincoln |Ct.|Suite 130|Pine Grove|Morgan County|IN|44593|United States|-5|single family| +14252|AAAAAAAAMKHDAAAA|508|Birch |Dr.|Suite 360|Amity|East Feliciana Parish|LA|70766|United States|-6|condo| +14253|AAAAAAAANKHDAAAA|810|3rd |Boulevard|Suite L|Weldon|Musselshell County|MT|66277|United States|-7|single family| +14254|AAAAAAAAOKHDAAAA|461|View |Blvd|Suite Y|Kingston|Craven County|NC|24975|United States|-5|condo| +14255|AAAAAAAAPKHDAAAA|808|1st |Blvd|Suite 230|Church Hill|Fremont County|IA|53790|United States|-6|single family| +14256|AAAAAAAAALHDAAAA|453|Railroad |Circle|Suite 10|Union Hill|Weber County|UT|87746|United States|-7|condo| +14257|AAAAAAAABLHDAAAA|159|South |ST|Suite 400|New Hope|Dade County|MO|69431|United States|-6|apartment| +14258|AAAAAAAACLHDAAAA|47|Oak |Ave|Suite V|Wilton|Bee County|TX|76997|United States|-6|single family| +14259|AAAAAAAADLHDAAAA|973|Ridge Second|Drive|Suite S|Farmington|Garfield County|NE|69145|United States|-6|single family| +14260|AAAAAAAAELHDAAAA|109|Adams 8th|Court|Suite 410|Clinton|Gray County|KS|68222|United States|-6|apartment| +14261|AAAAAAAAFLHDAAAA|342|12th |Court|Suite 70|Wildwood|Steele County|ND|56871|United States|-6|single family| +14262|AAAAAAAAGLHDAAAA|141|5th |Ln|Suite V|Shiloh|Wirt County|WV|29275|United States|-5|single family| +14263|AAAAAAAAHLHDAAAA|54|Ridge |Boulevard|Suite Q|Hillcrest|Kidder County|ND|53003|United States|-6|apartment| +14264|AAAAAAAAILHDAAAA|428|Hill 12th|Ct.|Suite 340|Oak Hill|Pasquotank County|NC|27838|United States|-5|apartment| +14265|AAAAAAAAJLHDAAAA|307|College |Blvd|Suite P|Hillcrest|San Miguel County|NM|83003|United States|-7|apartment| +14266|AAAAAAAAKLHDAAAA|198|Pine |Ln|Suite V|Lakewood|Menifee County|KY|48877|United States|-5|single family| +14267|AAAAAAAALLHDAAAA|251|Oak Cherry|RD|Suite 280|Spring Valley|Chaves County|NM|86060|United States|-7|condo| +14268|AAAAAAAAMLHDAAAA|17|Lake Sixth|Ln|Suite 460|Concord|Schleicher County|TX|74107|United States|-6|condo| +14269|AAAAAAAANLHDAAAA|655|Washington South|Ct.|Suite X|Bethel|Forsyth County|GA|35281|United States|-5|single family| +14270|AAAAAAAAOLHDAAAA|111|Sunset |RD|Suite 230|Oak Ridge|Kearney County|NE|68371|United States|-7|apartment| +14271|AAAAAAAAPLHDAAAA|417|5th Forest|Ct.|Suite T|Union|Buckingham County|VA|28721|United States|-5|single family| +14272|AAAAAAAAAMHDAAAA|600|North |Way|Suite U|Rosewood|Moore County|NC|22205|United States|-5|single family| +14273|AAAAAAAABMHDAAAA|881|Smith View|ST|Suite T|Fairfield|Hardin County|TN|36192|United States|-5|apartment| +14274|AAAAAAAACMHDAAAA|165|Washington 11th|ST|Suite Y|Mountain View|Douglas County|MO|64466|United States|-6|condo| +14275|AAAAAAAADMHDAAAA|925||Dr.|Suite C|||MS|||-6|| +14276|AAAAAAAAEMHDAAAA|1000|Smith |Ave|Suite 310|Walnut Grove|Banks County|GA|37752|United States|-5|condo| +14277|AAAAAAAAFMHDAAAA|584|4th Maple|Boulevard|Suite I|Welcome|Polk County|MO|66386|United States|-6|apartment| +14278|AAAAAAAAGMHDAAAA|249|3rd |Cir.|Suite 280|Oakwood|Goshen County|WY|80169|United States|-7|condo| +14279|AAAAAAAAHMHDAAAA|25|Main |Lane|Suite 0|Maple Grove|Leon County|TX|78252|United States|-6|single family| +14280|AAAAAAAAIMHDAAAA|52|Green |Way|Suite 270|Bunker Hill|Harper County|OK|70150|United States|-6|apartment| +14281|AAAAAAAAJMHDAAAA|400|Church First|Avenue|Suite 330|Edgewood|Calloway County|KY|40069|United States|-6|condo| +14282|AAAAAAAAKMHDAAAA|||Cir.||Friendship|Ozaukee County|||||single family| +14283|AAAAAAAALMHDAAAA|394|Franklin |Way|Suite 230|Middletown|Simpson County|MS|51422|United States|-6|condo| +14284|AAAAAAAAMMHDAAAA|228|College |Wy|Suite H|Crossroads|Page County|VA|20534|United States|-5|condo| +14285|AAAAAAAANMHDAAAA|360|8th |Cir.|Suite 120|Cedar Grove|Brantley County|GA|30411|United States|-5|single family| +14286|AAAAAAAAOMHDAAAA|180|Lakeview |Drive|Suite 360|Flat Rock|Andrews County|TX|74387|United States|-6|single family| +14287|AAAAAAAAPMHDAAAA|593|Lakeview Main|Drive|Suite 250|Woodville|Lewis County|WV|24289|United States|-5|single family| +14288|AAAAAAAAANHDAAAA|376|Broadway |Pkwy|Suite J|Siloam|Piatt County|IL|68948|United States|-6|condo| +14289|AAAAAAAABNHDAAAA|790|Smith Lincoln|Boulevard|Suite 320|Woodbury|Brazos County|TX|74489|United States|-6|condo| +14290|AAAAAAAACNHDAAAA|548|Fourth |Cir.|Suite 380|Ashland|Bartow County|GA|34244|United States|-5|apartment| +14291|AAAAAAAADNHDAAAA|501|10th |Wy|Suite L|Woodville|Reno County|KS|64289|United States|-6|apartment| +14292|AAAAAAAAENHDAAAA|144|2nd |Dr.|Suite 250|Macedonia|Dillingham Census Area|AK|91087|United States|-9|apartment| +14293|AAAAAAAAFNHDAAAA|610|11th |Drive|Suite 90|Springfield|Clallam County|WA|99303|United States|-8|condo| +14294|AAAAAAAAGNHDAAAA|265|Poplar Oak|Parkway|Suite E|Mount Zion|Madison County|AL|38054|United States|-6|condo| +14295|AAAAAAAAHNHDAAAA|11|Williams |Cir.|Suite J|Wildwood|Mono County|CA|96871|United States|-8|apartment| +14296|AAAAAAAAINHDAAAA|||||Walnut Grove|Walker County|GA|37752|||| +14297|AAAAAAAAJNHDAAAA|225|Walnut Poplar|Lane|Suite X|Macedonia|McDonough County|IL|61087|United States|-6|condo| +14298|AAAAAAAAKNHDAAAA|521|Mill |Way|Suite 160|Harmony|Harrison County|MS|55804|United States|-6|single family| +14299|AAAAAAAALNHDAAAA|504|Fourth |Ave|Suite 340|Pine Grove|Georgetown County|SC|24593|United States|-5|condo| +14300|AAAAAAAAMNHDAAAA|25||||Greenfield|||15038|United States||apartment| +14301|AAAAAAAANNHDAAAA|487|3rd |Dr.|Suite J|Bunker Hill|De Witt County|IL|60150|United States|-6|condo| +14302|AAAAAAAAONHDAAAA|253|South Forest|ST|Suite G|White Oak|Morgan County|KY|46668|United States|-5|condo| +14303|AAAAAAAAPNHDAAAA|972|Railroad 4th|Lane|Suite J|Harmony|Brown County|SD|55804|United States|-6|apartment| +14304|AAAAAAAAAOHDAAAA|622|3rd |Road|Suite 480|Piney Grove|Fayette County|GA|38274|United States|-5|single family| +14305|AAAAAAAABOHDAAAA|529|View 1st|Cir.|Suite P|Newtown|Rockbridge County|VA|21749|United States|-5|apartment| +14306|AAAAAAAACOHDAAAA|55|6th Smith|Blvd|Suite W|Enterprise|Florence County|WI|51757|United States|-6|apartment| +14307|AAAAAAAADOHDAAAA|214|First |Ln|Suite 230|Frankfort|Ben Hill County|GA|39681|United States|-5|apartment| +14308|AAAAAAAAEOHDAAAA|927|Lincoln |Avenue|Suite V|Riverdale|Nantucket County|MA|09991|United States|-5|single family| +14309|AAAAAAAAFOHDAAAA|550|Laurel Laurel|Pkwy|Suite Y|Hillcrest|Calloway County|KY|43003|United States|-6|apartment| +14310|AAAAAAAAGOHDAAAA|19|East 10th|Court|Suite 340|Mountain View|Cheatham County|TN|34466|United States|-5|condo| +14311|AAAAAAAAHOHDAAAA|202|Cherry |Ln|Suite W|Spring Hill|Montgomery County|GA|36787|United States|-5|single family| +14312|AAAAAAAAIOHDAAAA|327|Smith College|Court|Suite 140|Pleasant Hill|Fayette County|TX|73604|United States|-6|condo| +14313|AAAAAAAAJOHDAAAA|414|Birch |Lane|Suite V|Georgetown|Boone County|IN|47057|United States|-5|apartment| +14314|AAAAAAAAKOHDAAAA|852|Highland 15th|Avenue|Suite 280|Wilson|Childress County|TX|76971|United States|-6|condo| +14315|AAAAAAAALOHDAAAA|45|Willow Main|Road|Suite L|Jamestown|Gibson County|TN|36867|United States|-5|single family| +14316|AAAAAAAAMOHDAAAA|526|Park Adams|Parkway|Suite 370|Oakwood|Atoka County|OK|70169|United States|-6|single family| +14317|AAAAAAAANOHDAAAA|858|West |Boulevard|Suite 330|Oakwood|Shenandoah County|VA|20169|United States|-5|condo| +14318|AAAAAAAAOOHDAAAA|995|2nd Oak|Boulevard|Suite 110|Buena Vista|Collingsworth County|TX|75752|United States|-6|condo| +14319|AAAAAAAAPOHDAAAA|935|Lake Willow|Circle|Suite 160|Midway|Scott County|MO|61904|United States|-6|apartment| +14320|AAAAAAAAAPHDAAAA|966|Ridge 3rd|Drive|Suite O|Deerfield|Clay County|IN|49840|United States|-5|single family| +14321|AAAAAAAABPHDAAAA||Chestnut |Boulevard|||Osage County|MO||United States||apartment| +14322|AAAAAAAACPHDAAAA||Washington Smith|Cir.|||||||-6|single family| +14323|AAAAAAAADPHDAAAA|975|||Suite C||Osage County|OK|74289|||| +14324|AAAAAAAAEPHDAAAA|861|Walnut Park|Pkwy|Suite B|Highland Park|Buffalo County|SD|56534|United States|-6|condo| +14325|AAAAAAAAFPHDAAAA|319|Spring |Court|Suite 100|Providence|Loudon County|TN|36614|United States|-6|condo| +14326|AAAAAAAAGPHDAAAA|980|1st |Drive|Suite P|Oak Grove|Guthrie County|IA|58370|United States|-6|condo| +14327|AAAAAAAAHPHDAAAA|632|South |Way|Suite 310|Newport|Sitka Borough|AK|91521|United States|-9|condo| +14328|AAAAAAAAIPHDAAAA|622|11th Laurel|Ct.|Suite W|Oak Grove|Montgomery County|GA|38370|United States|-5|single family| +14329|AAAAAAAAJPHDAAAA|126|Chestnut 5th|Street|Suite X|Lebanon|Multnomah County|OR|92898|United States|-8|apartment| +14330|AAAAAAAAKPHDAAAA|576|First |Drive|Suite 380|Greenwood|Burnett County|WI|58828|United States|-6|apartment| +14331|AAAAAAAALPHDAAAA|42|River |RD|Suite 490|Shady Grove|Coahoma County|MS|52812|United States|-6|condo| +14332|AAAAAAAAMPHDAAAA|661|7th Chestnut|Dr.|Suite T|Greenfield|Macon County|||United States||condo| +14333|AAAAAAAANPHDAAAA|287|Fifth 12th|Circle|Suite A|Maple Grove|Lamoille County|VT|08852|United States|-5|apartment| +14334|AAAAAAAAOPHDAAAA|488|Highland |ST|Suite U|Walnut Grove|Jefferson County|MS|57752|United States|-6|apartment| +14335|AAAAAAAAPPHDAAAA|114|Washington First|Wy|Suite 330|Friendship|Logan County|WV|24536|United States|-5|apartment| +14336|AAAAAAAAAAIDAAAA|463|Elm |Way|Suite 390|Cedar Grove|Oneida County|NY|10411|United States|-5|condo| +14337|AAAAAAAABAIDAAAA|788|Oak |RD|Suite 330|Lincoln|Carver County|MN|51289|United States|-6|apartment| +14338|AAAAAAAACAIDAAAA|226|Davis Sunset|Road|Suite J|Mount Olive|Lynn County|TX|78059|United States|-6|single family| +14339|AAAAAAAADAIDAAAA|831|Railroad 14th|Way|Suite X|Springfield|Navajo County|AZ|89303|United States|-7|apartment| +14340|AAAAAAAAEAIDAAAA|277|Walnut |Lane|Suite 160|Shiloh|Cedar County|NE|69275|United States|-6|single family| +14341|AAAAAAAAFAIDAAAA|187|Forest |Road|Suite 240|Tipton|Pike County|AL|38773|United States|-6|single family| +14342|AAAAAAAAGAIDAAAA|565|Woodland |Dr.|Suite G|Fairview|Greer County|OK|75709|United States|-6|condo| +14343|AAAAAAAAHAIDAAAA|939|Fourth |Ln|Suite 0|Woodland|Sarasota County|FL|34854|United States|-5|single family| +14344|AAAAAAAAIAIDAAAA|103|Second 1st|Lane|Suite M|Clearview|Cherokee County|NC|25495|United States|-5|apartment| +14345|AAAAAAAAJAIDAAAA|794|Main Davis|Court|Suite V|Sulphur Springs|Kalkaska County|MI|48354|United States|-5|single family| +14346|AAAAAAAAKAIDAAAA|418|8th |Avenue|Suite J|Greenville|Greene County|NC|21387|United States|-5|condo| +14347|AAAAAAAALAIDAAAA|319|1st |Ave|Suite F|Walnut|Woods County|OK|76245|United States|-6|condo| +14348|AAAAAAAAMAIDAAAA|799|Park North|Court|Suite 110|Macedonia|Teton County|WY|81087|United States|-7|single family| +14349|AAAAAAAANAIDAAAA|309|Hill West|RD|Suite S|Mount Pleasant|Rockcastle County|KY|41933|United States|-5|single family| +14350|AAAAAAAAOAIDAAAA|921|Maple |Dr.|Suite 60|Mount Pleasant|Sherman County|NE|61933|United States|-7|single family| +14351|AAAAAAAAPAIDAAAA|872|Lake |Boulevard|Suite N|Oak Ridge|Albany County|WY|88371|United States|-7|single family| +14352|AAAAAAAAABIDAAAA|419|Smith |Ave|Suite 310|Pinecrest|Wood County|WI|59981|United States|-6|condo| +14353|AAAAAAAABBIDAAAA|201|Main Birch|Drive|Suite 140|Wilson|Wabasha County|MN|56971|United States|-6|apartment| +14354|AAAAAAAACBIDAAAA|348|2nd |Pkwy|Suite N|Sunnyside|Mississippi County|AR|71952|United States|-6|condo| +14355|AAAAAAAADBIDAAAA|878|7th |ST|Suite F|Mount Pleasant|Coffey County|KS|61933|United States|-6|condo| +14356|AAAAAAAAEBIDAAAA|724|Elm Central|ST|Suite 40|Murray|Burke County|NC|22150|United States|-5|condo| +14357|AAAAAAAAFBIDAAAA|409|Johnson Ridge|Parkway|Suite 290|Bridgeport|Ziebach County|SD|55817|United States|-6|apartment| +14358|AAAAAAAAGBIDAAAA|25|South Lake|Blvd|Suite Q|Plainview|Tioga County|NY|13683|United States|-5|condo| +14359|AAAAAAAAHBIDAAAA|559|View |Boulevard|Suite R|Greenfield|Izard County|AR|75038|United States|-6|single family| +14360|AAAAAAAAIBIDAAAA|133|Hickory |Wy|Suite S|Star|Thayer County|NE|60725|United States|-7|apartment| +14361|AAAAAAAAJBIDAAAA|366|Church Lincoln|Ln|Suite G|White Oak|Ogle County|IL|66668|United States|-6|single family| +14362|AAAAAAAAKBIDAAAA|120|East |Ct.|Suite 490|Pleasant Grove|Webster County|MS|54136|United States|-6|single family| +14363|AAAAAAAALBIDAAAA|431|Hillcrest Sixth|Ave|Suite 410|New Hope|Polk County|IA|59431|United States|-6|apartment| +14364|AAAAAAAAMBIDAAAA|418|Ridge Railroad|Lane|Suite 210|Newtown|Dade County|MO|61749|United States|-6|condo| +14365|AAAAAAAANBIDAAAA|423|Park College|Cir.|Suite 280|Spring Hill|Nowata County|OK|76787|United States|-6|apartment| +14366|AAAAAAAAOBIDAAAA|192|Central |Blvd|Suite L|Providence|Todd County|MN|56614|United States|-6|condo| +14367|AAAAAAAAPBIDAAAA|900|4th Spring|Wy|Suite 320|Pine Grove|Twiggs County|GA|34593|United States|-5|condo| +14368|AAAAAAAAACIDAAAA|668|Johnson |Street|Suite R|Waterloo|Maries County|MO|61675|United States|-6|condo| +14369|AAAAAAAABCIDAAAA|160|Maple Meadow|Circle|Suite 30|Ashland|Chisago County|MN|54244|United States|-6|condo| +14370|AAAAAAAACCIDAAAA|462|Park |Avenue|Suite 340|Newport|Charlton County|GA|31521|United States|-5|single family| +14371|AAAAAAAADCIDAAAA|280|Elm Church|Circle|Suite 170|Farmington|Wise County|VA|29145|United States|-5|single family| +14372|AAAAAAAAECIDAAAA|39|First |Ln|Suite 330|Forest Hills|Caldwell County|KY|49237|United States|-6|single family| +14373|AAAAAAAAFCIDAAAA|581|Main Cherry|Circle|Suite 350|Bunker Hill|Grand Forks County|ND|50150|United States|-6|condo| +14374|AAAAAAAAGCIDAAAA|142|Laurel Wilson|Avenue|Suite S|Marion|Scioto County|OH|40399|United States|-5|apartment| +14375|AAAAAAAAHCIDAAAA|||Ln|||||28579|||apartment| +14376|AAAAAAAAICIDAAAA|39|Walnut 2nd|Pkwy|Suite 100|Bunker Hill|Custer County|MT|60150|United States|-7|apartment| +14377|AAAAAAAAJCIDAAAA|258|Third |Blvd|Suite F|Hillcrest|Wayne County|PA|13003|United States|-5|single family| +14378|AAAAAAAAKCIDAAAA|343|1st |Cir.|Suite 110|Lebanon|Bath County|KY|42898|United States|-6|single family| +14379|AAAAAAAALCIDAAAA|239|Church Sunset|Road|Suite 110|Oak Ridge|Ventura County|CA|98371|United States|-8|single family| +14380|AAAAAAAAMCIDAAAA|976|Smith |Blvd|Suite W|Brownsville|Lee County|KY|49310|United States|-5|condo| +14381|AAAAAAAANCIDAAAA|256|Lincoln Fourth|ST|Suite 490|Highland Park|Merrick County|NE|66534|United States|-7|single family| +14382|AAAAAAAAOCIDAAAA|33|Jackson |Way|Suite 420|Union|Green County|KY|48721|United States|-6|single family| +14383|AAAAAAAAPCIDAAAA|877|Forest |RD|Suite 250|Highland|Sabine Parish|LA|79454|United States|-6|condo| +14384|AAAAAAAAADIDAAAA|581|2nd Johnson|Avenue|Suite 190|Macedonia|Crawford County|KS|61087|United States|-6|condo| +14385|AAAAAAAABDIDAAAA|136|Adams |ST|Suite 430|Mount Pleasant|Lincoln County|NV|81933|United States|-8|single family| +14386|AAAAAAAACDIDAAAA|953|8th |Wy|Suite 200|Shiloh|Rockwall County|TX|79275|United States|-6|condo| +14387|AAAAAAAADDIDAAAA|688|Mill |Court|Suite D|Plainview|Divide County|ND|53683|United States|-6|single family| +14388|AAAAAAAAEDIDAAAA|195|Walnut Ash|Pkwy|Suite 310|Salem|Clinton County|IA|58048|United States|-6|apartment| +14389|AAAAAAAAFDIDAAAA|289|First 9th|Drive|Suite K|Enterprise|Madison County|OH|41757|United States|-5|condo| +14390|AAAAAAAAGDIDAAAA|906|Jackson |Ct.|Suite 20|Edgewood|Kiowa County|KS|60069|United States|-6|apartment| +14391|AAAAAAAAHDIDAAAA|195|Lake |Circle|Suite 170|||ID|86719|||| +14392|AAAAAAAAIDIDAAAA|995|Pine |Way|Suite C|Lakeside|Pontotoc County|MS|59532|United States|-6|apartment| +14393|AAAAAAAAJDIDAAAA|833|Forest |Way|Suite 330|Five Forks|Bleckley County|GA|32293|United States|-5|apartment| +14394|AAAAAAAAKDIDAAAA|871|Elm |Street|Suite 310|Fairfield|Monroe County|FL|36192|United States|-5|apartment| +14395|AAAAAAAALDIDAAAA|493|Jackson |Court|Suite H|Spring Valley|Rooks County|KS|66060|United States|-6|condo| +14396|AAAAAAAAMDIDAAAA|712|Spring 1st|Blvd|Suite D|Plainview|Robertson County|TN|33683|United States|-6|condo| +14397|AAAAAAAANDIDAAAA|366|Williams Dogwood|Ln|Suite K|Pine Grove|Hamilton County|TN|34593|United States|-5|apartment| +14398|AAAAAAAAODIDAAAA|905|East |Way|Suite I|Greenfield|Franklin city|VA|25038|United States|-5|single family| +14399|AAAAAAAAPDIDAAAA|339|Hill Laurel|Way|Suite D|Georgetown|Fayette County|TN|37057|United States|-5|condo| +14400|AAAAAAAAAEIDAAAA|341|Jefferson Oak|Pkwy|Suite I|Pleasant Hill|Corson County|SD|53604|United States|-6|apartment| +14401|AAAAAAAABEIDAAAA|336|Woodland Cherry|Pkwy|Suite 330|Riverside|Lane County|OR|99231|United States|-8|single family| +14402|AAAAAAAACEIDAAAA|466|Hill 8th|Way|Suite L|Salem|Pike County|PA|18048|United States|-5|apartment| +14403|AAAAAAAADEIDAAAA|961|Smith |Lane|Suite 170|Edgewood|Lander County|NV|80069|United States|-8|condo| +14404|AAAAAAAAEEIDAAAA|131|4th Hill|Circle|Suite K|Belmont|Haywood County|TN|30191|United States|-5|condo| +14405|AAAAAAAAFEIDAAAA|104|Second Johnson|Pkwy|Suite R|Wildwood|Nye County|NV|86871|United States|-8|single family| +14406|AAAAAAAAGEIDAAAA|55|Laurel |Road|Suite 200|Friendship|Pulaski County|IN|44536|United States|-5|single family| +14407|AAAAAAAAHEIDAAAA|770|Hickory 11th|Circle|Suite 110|Providence|Douglas County|OR|96614|United States|-8|condo| +14408|AAAAAAAAIEIDAAAA|321|Woodland |Wy|Suite M|Clinton|Perry County|MS|58222|United States|-6|condo| +14409|AAAAAAAAJEIDAAAA|963|Williams Birch|Drive|Suite 410|Edgewood|Franklin County|KS|60069|United States|-6|condo| +14410|AAAAAAAAKEIDAAAA|195|Second |Way|Suite W|Mountain View|Texas County|MO|64466|United States|-6|condo| +14411|AAAAAAAALEIDAAAA|688|Sunset |Parkway|Suite R|Clinton|Teton County|ID|88222|United States|-7|condo| +14412|AAAAAAAAMEIDAAAA|863|West View|Avenue|Suite P|Shiloh|McMullen County|TX|79275|United States|-6|condo| +14413|AAAAAAAANEIDAAAA|852|Hill Lake|Circle|Suite R|Pleasant Valley|Warren County|GA|32477|United States|-5|condo| +14414|AAAAAAAAOEIDAAAA|637|View |Ave|Suite 0|Highland Park|Montgomery County|OH|46534|United States|-5|condo| +14415|AAAAAAAAPEIDAAAA|144|Oak Fourteenth|Drive|Suite N|Chestnut Ridge|Cache County|UT|87334|United States|-7|condo| +14416|AAAAAAAAAFIDAAAA|257|Willow |Pkwy|Suite R|Spring Valley|Santa Cruz County|AZ|86060|United States|-7|single family| +14417|AAAAAAAABFIDAAAA|734|Elm |Road|Suite J|Bethel|Lewis County|TN|35281|United States|-6|single family| +14418|AAAAAAAACFIDAAAA|183|Washington |Drive|Suite 460|Pumpkin Center|Anson County|NC|27721|United States|-5|condo| +14419|AAAAAAAADFIDAAAA|250|South 6th|Road|Suite 320|Clinton|Hancock County|OH|48222|United States|-5|apartment| +14420|AAAAAAAAEFIDAAAA|250|Lee |ST|Suite I|Enterprise|Hancock County|WV|21757|United States|-5|apartment| +14421|AAAAAAAAFFIDAAAA|342|Maple 6th|Blvd|Suite 110|Salem|Union County|OH|48048|United States|-5|single family| +14422|AAAAAAAAGFIDAAAA|445|6th |Way|Suite M|Bridgeport|Benson County|ND|55817|United States|-6|apartment| +14423|AAAAAAAAHFIDAAAA|869|Lee Hickory|Ave|Suite 110|Riverside|Victoria County|TX|79231|United States|-6|apartment| +14424|AAAAAAAAIFIDAAAA|304|Jefferson Williams|Ct.|Suite 230|Woodville|Camden County|NC|24289|United States|-5|single family| +14425|AAAAAAAAJFIDAAAA|158|Smith Cedar|Boulevard|Suite 350|Mount Zion|Buchanan County|IA|58054|United States|-6|apartment| +14426|AAAAAAAAKFIDAAAA|30|2nd |Way|Suite F|Wilson|Weakley County|TN|36971|United States|-6|apartment| +14427|AAAAAAAALFIDAAAA|562|Railroad Forest|Dr.|Suite 20|Deerfield|Ferry County|WA|99840|United States|-8|apartment| +14428|AAAAAAAAMFIDAAAA|||Blvd|Suite W||Calhoun County|WV|29431|||single family| +14429|AAAAAAAANFIDAAAA|273|Pine 2nd|Parkway|Suite 270|Sleepy Hollow|Scott County|IA|53592|United States|-6|condo| +14430|AAAAAAAAOFIDAAAA|590|Pine Jefferson|Ct.|Suite M|White Oak|Caroline County|VA|26668|United States|-5|single family| +14431|AAAAAAAAPFIDAAAA|281|Oak |Court|Suite 330|Waterloo|Jefferson County|IN|41675|United States|-5|condo| +14432|AAAAAAAAAGIDAAAA|844|Lake |Avenue|Suite P|Highland Park|Pratt County|KS|66534|United States|-6|single family| +14433|AAAAAAAABGIDAAAA|606|Willow |Avenue|Suite P|Greenfield|Northumberland County|VA|25038|United States|-5|apartment| +14434|AAAAAAAACGIDAAAA|881|Valley |Wy|Suite 270|Saint Johns|Ontario County|NY|15717|United States|-5|apartment| +14435|AAAAAAAADGIDAAAA|551|Green |Lane|Suite I|Sunnyside|White Pine County|NV|81952|United States|-8|condo| +14436|AAAAAAAAEGIDAAAA|352|Central |ST|Suite 120|Mount Olive|Daviess County|IN|48059|United States|-5|single family| +14437|AAAAAAAAFGIDAAAA|28|Smith |Pkwy|Suite 340|Jamestown|Stearns County|MN|56867|United States|-6|apartment| +14438|AAAAAAAAGGIDAAAA|841|Elm |Cir.|Suite F|Hillcrest|Wilkin County|MN|53003|United States|-6|single family| +14439|AAAAAAAAHGIDAAAA|426|Smith |Street|Suite W|Stringtown|Niagara County|NY|10162|United States|-5|single family| +14440|AAAAAAAAIGIDAAAA|126|Park Ash|Ave|Suite E|Millbrook|Butte County|ID|87529|United States|-7|condo| +14441|AAAAAAAAJGIDAAAA|407|Highland Spring|Avenue|Suite G|Salem|Saline County|IL|68048|United States|-6|single family| +14442|AAAAAAAAKGIDAAAA|205|6th Hillcrest|Cir.|Suite Q|Riverside|Flathead County|MT|69231|United States|-7|single family| +14443|AAAAAAAALGIDAAAA|501|5th |Court|Suite 400|Shady Grove|Yavapai County|AZ|82812|United States|-7|apartment| +14444|AAAAAAAAMGIDAAAA|155|15th |Dr.|Suite 420|Summit|Jeff Davis County|GA|30499|United States|-5|condo| +14445|AAAAAAAANGIDAAAA|749|Main |Blvd|Suite 70|Highland|Craig County|VA|29454|United States|-5|single family| +14446|AAAAAAAAOGIDAAAA|||Lane||Hillcrest|||23003|||| +14447|AAAAAAAAPGIDAAAA|90|Second |ST|Suite 220|Lakewood|Worth County|GA|38877|United States|-5|single family| +14448|AAAAAAAAAHIDAAAA|321|Valley |RD|Suite 190|Liberty|Belmont County|OH|43451|United States|-5|condo| +14449|AAAAAAAABHIDAAAA|234|Lee |Cir.|Suite J|Five Points|Lafayette County|WI|56098|United States|-6|apartment| +14450|AAAAAAAACHIDAAAA|876|Mill |Boulevard|Suite Y|Maple Grove|Hayes County|NE|68252|United States|-6|condo| +14451|AAAAAAAADHIDAAAA|597|Lee |Lane|Suite 460|Greenville|Buchanan County|IA|51387|United States|-6|apartment| +14452|AAAAAAAAEHIDAAAA|59|Hillcrest Jackson|Ct.|Suite 340|Belmont|Park County|MT|60191|United States|-7|condo| +14453|AAAAAAAAFHIDAAAA|312|Park Wilson|Blvd|Suite P|Pine Grove|Ellis County|TX|74593|United States|-6|apartment| +14454|AAAAAAAAGHIDAAAA|300|Ridge Center|Dr.|Suite 100|Spring Hill|La Salle Parish|LA|76787|United States|-6|single family| +14455|AAAAAAAAHHIDAAAA|329|Williams 7th|Pkwy|Suite C|Belmont|Lawrence County|IN|40191|United States|-5|single family| +14456|AAAAAAAAIHIDAAAA|259|Ridge Washington|Way|Suite B|Northwood|Calhoun County|TX|74104|United States|-6|single family| +14457|AAAAAAAAJHIDAAAA|388|Williams Fifth|Blvd|Suite C|Martinsville|Otero County|NM|80419|United States|-7|condo| +14458|AAAAAAAAKHIDAAAA|771|Oak |Way|Suite 290|Wesley|Kings County|CA|91218|United States|-8|single family| +14459|AAAAAAAALHIDAAAA|173|Oak |Avenue|Suite W|Lakeview|Lake County|CO|88579|United States|-7|apartment| +14460|AAAAAAAAMHIDAAAA|995|6th |Court|Suite 120|Sutton|Garfield County|CO|85413|United States|-7|condo| +14461|AAAAAAAANHIDAAAA|312|Seventh |Dr.|Suite 340|Hopewell|Spalding County|GA|30587|United States|-5|single family| +14462|AAAAAAAAOHIDAAAA|821|2nd |Lane|Suite 250|Phoenix|Greenbrier County|WV|22276|United States|-5|condo| +14463|AAAAAAAAPHIDAAAA|636|Spruce Pine|Blvd|Suite 200|Kingston|Peach County|GA|34975|United States|-5|single family| +14464|AAAAAAAAAIIDAAAA|358|14th |Pkwy|Suite 410|Plainview|Rockingham County|NC|23683|United States|-5|apartment| +14465|AAAAAAAABIIDAAAA|167|Oak Cedar|Street|Suite 400|Stringtown|Dewey County|OK|70162|United States|-6|condo| +14466|AAAAAAAACIIDAAAA|277|Maple |||Concord|||54107|United States|-6|| +14467|AAAAAAAADIIDAAAA|67|11th |Drive|Suite L|Providence|Lorain County|OH|46614|United States|-5|condo| +14468|AAAAAAAAEIIDAAAA|259|Hillcrest |ST|Suite D|Riverdale|Mississippi County|AR|79391|United States|-6|condo| +14469|AAAAAAAAFIIDAAAA|92|Center |RD|Suite W|Pomona|Green County|WI|54153|United States|-6|condo| +14470|AAAAAAAAGIIDAAAA|486|10th |Wy|Suite 340|Georgetown|Highland County|OH|47057|United States|-5|single family| +14471|AAAAAAAAHIIDAAAA|886|9th |Road|Suite O|Belmont|Chippewa County|WI|50191|United States|-6|apartment| +14472|AAAAAAAAIIIDAAAA|824|Mill |Way|Suite Q|Clifton|Socorro County|NM|88014|United States|-7|apartment| +14473|AAAAAAAAJIIDAAAA|391|5th |Ct.|Suite P|Mount Pleasant|Pierce County|ND|51933|United States|-6|condo| +14474|AAAAAAAAKIIDAAAA|103|Woodland |Lane|Suite 200|Bunker Hill|Harper County|KS|60150|United States|-6|single family| +14475|AAAAAAAALIIDAAAA|625|3rd Main|Parkway|Suite Q|Georgetown|DeWitt County|TX|77057|United States|-6|condo| +14476|AAAAAAAAMIIDAAAA|971|Jackson |Circle|Suite 200|Oak Ridge|Arlington County|VA|28371|United States|-5|apartment| +14477|AAAAAAAANIIDAAAA|997|First 4th|Way|Suite A|Marion|Poquoson city|VA|20399|United States|-5|apartment| +14478|AAAAAAAAOIIDAAAA|533|Willow |Pkwy|Suite D|Omega|Ness County|KS|66364|United States|-6|condo| +14479|AAAAAAAAPIIDAAAA|419|6th Maple|ST|Suite 170|Enterprise|Butler County|NE|61757|United States|-6|condo| +14480|AAAAAAAAAJIDAAAA|191|Laurel Sunset|Street|Suite 210|Pleasant Hill|Calhoun County|WV|23604|United States|-5|apartment| +14481|AAAAAAAABJIDAAAA|919|West |RD|Suite P|Springfield|Piscataquis County|ME|09903|United States|-5|apartment| +14482|AAAAAAAACJIDAAAA|862|Main Williams|Ct.|Suite 180|Greenfield|Jewell County|KS|65038|United States|-6|single family| +14483|AAAAAAAADJIDAAAA|564|5th |Cir.|Suite U|Waterloo|Renville County|ND|51675|United States|-6|single family| +14484|AAAAAAAAEJIDAAAA|381|Ridge |Boulevard|Suite 320|Wayland|Concho County|TX|75115|United States|-6|apartment| +14485|AAAAAAAAFJIDAAAA|503|Oak Pine|Wy|Suite 400|Harmony|Logan County|CO|85804|United States|-7|single family| +14486|AAAAAAAAGJIDAAAA|827|Lake |Avenue|Suite 120|Sunnyside|Paulding County|OH|41952|United States|-5|apartment| +14487|AAAAAAAAHJIDAAAA|984|Oak |Parkway|Suite J|Lebanon|Cleveland County|OK|72898|United States|-6|condo| +14488|AAAAAAAAIJIDAAAA|204|Smith |Wy|Suite H|Midway|Outagamie County|WI|51904|United States|-6|condo| +14489|AAAAAAAAJJIDAAAA|652|Park |Court|Suite D|Hopewell|Mayes County|OK|70587|United States|-6|apartment| +14490|AAAAAAAAKJIDAAAA|834|6th |||Glendale|Wasatch County||83951|||condo| +14491|AAAAAAAALJIDAAAA|154|View |Avenue|Suite 160|Derby|Etowah County|AL|37702|United States|-6|condo| +14492|AAAAAAAAMJIDAAAA|637|12th |Lane|Suite I|Sunnyside|Story County|IA|51952|United States|-6|condo| +14493|AAAAAAAANJIDAAAA|52|North 13th|Avenue|Suite 70|Shiloh|Taney County|MO|69275|United States|-6|apartment| +14494|AAAAAAAAOJIDAAAA|945|Franklin |Road|Suite O|Plainview|Cherokee County|AL|33683|United States|-6|condo| +14495|AAAAAAAAPJIDAAAA|245|North Ninth|Wy|Suite 110|Greenfield|Waushara County|WI|55038|United States|-6|single family| +14496|AAAAAAAAAKIDAAAA|37|Spring |Dr.|Suite G|Walnut Grove|Franklin County|TN|37752|United States|-5|condo| +14497|AAAAAAAABKIDAAAA|371|Sixth Sycamore|Boulevard|Suite 0|Arcola|Lancaster County|SC|21654|United States|-5|apartment| +14498|AAAAAAAACKIDAAAA|816|9th |Circle|Suite 330|Maple Grove|Stone County|MO|68252|United States|-6|apartment| +14499|AAAAAAAADKIDAAAA|55|Church Maple|Pkwy|Suite V|Pine Grove|Doniphan County|KS|64593|United States|-6|condo| +14500|AAAAAAAAEKIDAAAA|595|Spruce |Ct.|Suite 80|Red Hill|Cherokee County|KS|64338|United States|-6|single family| +14501|AAAAAAAAFKIDAAAA|704|10th |Dr.|Suite 200|New Hope|Antelope County|NE|69431|United States|-6|single family| +14502|AAAAAAAAGKIDAAAA|432|9th Center|Road|Suite J|Five Forks|Dawson County|MT|62293|United States|-7|apartment| +14503|AAAAAAAAHKIDAAAA|254|First Hill|Way|Suite C|Enterprise|Dickinson County|IA|51757|United States|-6|single family| +14504|AAAAAAAAIKIDAAAA|703|Laurel |Dr.|Suite 300|Oak Hill|Muhlenberg County|KY|47838|United States|-5|single family| +14505|AAAAAAAAJKIDAAAA|879|Lincoln Sycamore|Dr.|Suite 330|Wilson|Franklin County|OH|46971|United States|-5|apartment| +14506|AAAAAAAAKKIDAAAA|924|||Suite 260|||||United States|-6|single family| +14507|AAAAAAAALKIDAAAA|801|Maple Hickory|Ct.|Suite 40|Midway|Isanti County|MN|51904|United States|-6|apartment| +14508|AAAAAAAAMKIDAAAA|207|Green Seventh|ST|Suite A|Spring Hill|Upshur County|WV|26787|United States|-5|condo| +14509|AAAAAAAANKIDAAAA|43|Franklin |Dr.|Suite 20|Newtown|Fisher County|TX|71749|United States|-6|condo| +14510|AAAAAAAAOKIDAAAA|823|Johnson |Ct.|Suite J|New Hope|Clearwater County|ID|89431|United States|-7|condo| +14511|AAAAAAAAPKIDAAAA|750|Miller |Boulevard|Suite 160|Franklin|Vigo County|IN|49101|United States|-5|single family| +14512|AAAAAAAAALIDAAAA|36|6th Locust|Court|Suite 190|Liberty|Addison County|VT|04051|United States|-5|apartment| +14513|AAAAAAAABLIDAAAA|234|Ash |Court|Suite K|Waterloo|Effingham County|GA|31675|United States|-5|apartment| +14514|AAAAAAAACLIDAAAA|935|Park Washington|ST|Suite P|Deerfield|Pulaski County|MO|69840|United States|-6|single family| +14515|AAAAAAAADLIDAAAA|343|Pine |Drive|Suite 160|Salem|Overton County|TN|38048|United States|-6|single family| +14516|AAAAAAAAELIDAAAA|278||||Pine Grove|Socorro County||84593|United States|-7|single family| +14517|AAAAAAAAFLIDAAAA|173|Sixth 8th|Wy|Suite 350|Georgetown|San Francisco County|CA|97057|United States|-8|condo| +14518|AAAAAAAAGLIDAAAA|494|Elm Thirteenth|Boulevard|Suite 180|Buena Vista|Tate County|MS|55752|United States|-6|single family| +14519|AAAAAAAAHLIDAAAA|627|View Mill|Avenue|Suite 160|Crossroads|Hamilton County|FL|30534|United States|-5|single family| +14520|AAAAAAAAILIDAAAA|481|Highland |Ct.|Suite U|Woodland|Nevada County|CA|94854|United States|-8|condo| +14521|AAAAAAAAJLIDAAAA|771|Lincoln |Lane|Suite U|Kingston|Northampton County|PA|14975|United States|-5|apartment| +14522|AAAAAAAAKLIDAAAA|459|5th Church|Dr.|Suite 490|Friendship|Waldo County|ME|05136|United States|-5|single family| +14523|AAAAAAAALLIDAAAA|773|Johnson Oak|RD|Suite 10|Liberty|Jo Daviess County|IL|63451|United States|-6|apartment| +14524|AAAAAAAAMLIDAAAA|937|Smith East|Circle|Suite 240|Midway|Martin County|KY|41904|United States|-5|apartment| +14525|AAAAAAAANLIDAAAA|40|1st 3rd|Ct.|Suite P|Mountain View|Elkhart County|IN|44466|United States|-5|single family| +14526|AAAAAAAAOLIDAAAA|918|Broadway |Street|Suite P|Glenwood|Randolph County|WV|23511|United States|-5|apartment| +14527|AAAAAAAAPLIDAAAA|66|Twelfth Woodland|Cir.|Suite P|Clifton|Otter Tail County|MN|58014|United States|-6|condo| +14528|AAAAAAAAAMIDAAAA|730|River 11th|Pkwy|Suite 160|Georgetown|Cortland County|NY|17057|United States|-5|apartment| +14529|AAAAAAAABMIDAAAA|951|Poplar |Boulevard|Suite 120|Valley View|Faulk County|SD|55124|United States|-6|condo| +14530|AAAAAAAACMIDAAAA|333|7th |Street|Suite K|White Oak|Darke County|OH|46668|United States|-5|apartment| +14531|AAAAAAAADMIDAAAA|355|13th |Street|Suite 190|Bunker Hill|Ector County|TX|70150|United States|-6|condo| +14532|AAAAAAAAEMIDAAAA|166|Williams |Parkway|Suite 260|Plainview|Franklin County|MS|53683|United States|-6|condo| +14533|AAAAAAAAFMIDAAAA|525|Highland Main|Wy|Suite 70|Saratoga|Dauphin County|PA|12123|United States|-5|condo| +14534|AAAAAAAAGMIDAAAA|608|Church Locust|Road|Suite S|Woodville|Auglaize County|OH|44289|United States|-5|apartment| +14535|AAAAAAAAHMIDAAAA|63|Oak Birch|Road|Suite V|Fisher|Presque Isle County|MI|42819|United States|-5|apartment| +14536|AAAAAAAAIMIDAAAA|745|Mill 13th|Boulevard|Suite 360|Friendship|Placer County|CA|94536|United States|-8|condo| +14537|AAAAAAAAJMIDAAAA|275|1st Ash|Lane|Suite 40|Cedar Grove|Perry County|KY|40411|United States|-5|single family| +14538|AAAAAAAAKMIDAAAA|875|5th |Wy|Suite L|Buena Vista|Steuben County|NY|15752|United States|-5|apartment| +14539|AAAAAAAALMIDAAAA|380|Forest Mill|Road|Suite 310|Arlington|Morgan County|OH|46557|United States|-5|condo| +14540|AAAAAAAAMMIDAAAA|221|1st |RD|Suite A|Riverside|Vernon County|WI|59231|United States|-6|apartment| +14541|AAAAAAAANMIDAAAA|423|10th 3rd|Pkwy|Suite 460|Valley View|Calhoun County|GA|35124|United States|-5|apartment| +14542|AAAAAAAAOMIDAAAA|184|Davis |Drive|Suite 360|Highland Park|Milam County|TX|76534|United States|-6|single family| +14543|AAAAAAAAPMIDAAAA|||||Edgewood|Coos County|OR|90069||-8|| +14544|AAAAAAAAANIDAAAA|457|3rd |Dr.|Suite 440|Pine Grove|Crawford County|MI|44593|United States|-5|single family| +14545|AAAAAAAABNIDAAAA|968|Center |Wy|Suite 300|Lakewood|Petroleum County|MT|68877|United States|-7|apartment| +14546|AAAAAAAACNIDAAAA|482|River |Wy|Suite 100|Harvey|Keweenaw County|MI|45858|United States|-5|single family| +14547|AAAAAAAADNIDAAAA|409|Cedar |Cir.|Suite U|Wildwood|Ballard County|KY|46871|United States|-6|apartment| +14548|AAAAAAAAENIDAAAA|591|View |ST|Suite 170|Georgetown|Sharkey County|MS|57057|United States|-6|apartment| +14549|AAAAAAAAFNIDAAAA|145|Central |Circle|Suite 40|Wilson|Westchester County|NY|16971|United States|-5|condo| +14550|AAAAAAAAGNIDAAAA|545|Highland |Road|Suite J|Springfield|Sandoval County|NM|89303|United States|-7|single family| +14551|AAAAAAAAHNIDAAAA|331|12th Locust|Road|Suite N|Greenfield|Sierra County|CA|95038|United States|-8|condo| +14552|AAAAAAAAINIDAAAA|855|Pine |Circle|Suite 120|Greenville|Greene County|NC|21387|United States|-5|apartment| +14553|AAAAAAAAJNIDAAAA|648|4th 12th|Way|Suite 400|Union|Palo Pinto County|TX|78721|United States|-6|condo| +14554|AAAAAAAAKNIDAAAA|659|5th |Pkwy|Suite D|Oakland|Otter Tail County|MN|59843|United States|-6|condo| +14555|AAAAAAAALNIDAAAA|43|Ash Willow|Boulevard|Suite N|Pleasant Hill|Treasure County|MT|63604|United States|-7|apartment| +14556|AAAAAAAAMNIDAAAA|970|Cedar River|Avenue|Suite Y|Bunker Hill|Highland County|OH|40150|United States|-5|condo| +14557|AAAAAAAANNIDAAAA|514|4th Washington|Lane|Suite O|Valley View|Robeson County|NC|25124|United States|-5|condo| +14558|AAAAAAAAONIDAAAA|276|Park Park|Lane|Suite D|Oakland|Clay County|SD|59843|United States|-6|condo| +14559|AAAAAAAAPNIDAAAA|765|Elm West|Drive|Suite 130|Harmony|Hardeman County|TX|75804|United States|-6|single family| +14560|AAAAAAAAAOIDAAAA|707|Smith |Pkwy|Suite 310|Oak Hill|Lawrence County|OH|47838|United States|-5|single family| +14561|AAAAAAAABOIDAAAA|761|6th Central|Wy|Suite C|Kingston|Harrison County|IA|54975|United States|-6|single family| +14562|AAAAAAAACOIDAAAA|646|Poplar |ST|Suite F|Providence|Ravalli County|MT|66614|United States|-7|single family| +14563|AAAAAAAADOIDAAAA|580|Walnut West|Cir.|Suite K|Woodland|Charles City County|VA|24854|United States|-5|condo| +14564|AAAAAAAAEOIDAAAA|912|4th Lincoln|Circle|Suite H|Edgewood|Pocahontas County|WV|20069|United States|-5|apartment| +14565|AAAAAAAAFOIDAAAA|292|1st 8th|Road|Suite G|Crossroads|Platte County|NE|60534|United States|-7|apartment| +14566|AAAAAAAAGOIDAAAA|933|Broadway Cedar|Blvd|Suite 350|Newtown|Panola County|MS|51749|United States|-6|condo| +14567|AAAAAAAAHOIDAAAA|182|Maple 4th|Ct.|Suite N|Highland Park|Gaines County|TX|76534|United States|-6|condo| +14568|AAAAAAAAIOIDAAAA|226|Center Sunset|Ln|Suite S|Stringtown|Adams County|IA|50162|United States|-6|apartment| +14569|AAAAAAAAJOIDAAAA|819|Poplar 5th|Drive|Suite X|Greenville|Jackson County|MN|51387|United States|-6|condo| +14570|AAAAAAAAKOIDAAAA|323|Sunset |Blvd|Suite 490|Ellsworth|Winchester city|VA|25079|United States|-5|apartment| +14571|AAAAAAAALOIDAAAA|592|Chestnut |ST|Suite B|Carthage|Roscommon County|MI|41529|United States|-5|condo| +14572|AAAAAAAAMOIDAAAA|987|Park |Parkway|Suite 50|Georgetown|Tuscarawas County|OH|47057|United States|-5|single family| +14573|AAAAAAAANOIDAAAA|982|||Suite K|Walnut Grove|Lexington County|||United States|-5|| +14574|AAAAAAAAOOIDAAAA|898|Wilson Birch|Road|Suite X|Greenwood|Monroe County|MI|48828|United States|-5|single family| +14575|AAAAAAAAPOIDAAAA|67|West Spring|Ave|Suite B|Enterprise|Gosper County|NE|61757|United States|-6|single family| +14576|AAAAAAAAAPIDAAAA|715|Fourth |Drive|Suite 370|Sunnyside|Early County|GA|31952|United States|-5|single family| +14577|AAAAAAAABPIDAAAA|989|Oak Cedar|Court|Suite 370|Liberty|Bibb County|AL|33451|United States|-6|single family| +14578|AAAAAAAACPIDAAAA|981|River |Wy|Suite 250|Sunnyside|Marion County|IA|51952|United States|-6|single family| +14579|AAAAAAAADPIDAAAA|629|Forest |Circle|Suite G|Waterloo|Cherokee County|SC|21675|United States|-5|single family| +14580|AAAAAAAAEPIDAAAA|283|1st Oak|Ct.|Suite 250|Waterloo|Clark County||81675|||| +14581|AAAAAAAAFPIDAAAA||Second |Ave|||Johnson County||43604||-6|single family| +14582|AAAAAAAAGPIDAAAA|984|Maple |Road|Suite F|Maywood|Madison County|AL|35681|United States|-6|single family| +14583|AAAAAAAAHPIDAAAA|776|Walnut |Ct.|Suite 10|Belmont|San Augustine County|TX|70191|United States|-6|apartment| +14584|AAAAAAAAIPIDAAAA|477|East |Lane|Suite F|Mount Olive|Caldwell Parish|LA|78059|United States|-6|condo| +14585|AAAAAAAAJPIDAAAA|345|Spring |Way|Suite 430|Fairview|Fayette County|IN|45709|United States|-5|single family| +14586|AAAAAAAAKPIDAAAA|488|Fifth Cedar|Road|Suite N|Carpenter|Hansford County|TX|71147|United States|-6|apartment| +14587|AAAAAAAALPIDAAAA|850|12th |Boulevard|Suite E|Ashland|Coryell County|TX|74244|United States|-6|single family| +14588|AAAAAAAAMPIDAAAA|738|Park 1st|Dr.|Suite C|Concord|Prince Edward County|VA|24107|United States|-5|single family| +14589|AAAAAAAANPIDAAAA|594|Willow |Avenue|Suite 120|Washington Heights|Kittitas County|WA|98167|United States|-8|apartment| +14590|AAAAAAAAOPIDAAAA|612|Washington Church|Road|Suite 400|Woodland|Henry County|TN|34854|United States|-5|condo| +14591|AAAAAAAAPPIDAAAA|730|Walnut 5th|Ave|Suite 160|Antioch|Whiteside County|IL|68605|United States|-6|apartment| +14592|AAAAAAAAAAJDAAAA|913|First |RD|Suite S|Five Forks|Miami County|KS|62293|United States|-6|apartment| +14593|AAAAAAAABAJDAAAA|722|11th |Dr.|Suite 110|Glenwood|Staunton city|VA|23511|United States|-5|apartment| +14594|AAAAAAAACAJDAAAA|202|Broadway |ST|Suite N|Glendale|Swift County|MN|53951|United States|-6|apartment| +14595|AAAAAAAADAJDAAAA|691|11th Maple|Pkwy|Suite N|Pleasant Hill|Brunswick County|VA|23604|United States|-5|single family| +14596|AAAAAAAAEAJDAAAA|211|Sunset Main|Ln|Suite I|Pleasant Grove|Labette County|KS|64136|United States|-6|apartment| +14597|AAAAAAAAFAJDAAAA|494|Laurel Hill|RD|Suite 290|Union|Chaffee County|CO|88721|United States|-7|condo| +14598|AAAAAAAAGAJDAAAA|845|Elevnth Williams|Court|Suite Q|Hamilton|Greene County|NY|12808|United States|-5|single family| +14599|AAAAAAAAHAJDAAAA|408|Smith Locust|Circle|Suite 460|Flint|Uinta County|WY|88909|United States|-7|condo| +14600|AAAAAAAAIAJDAAAA|898|Adams Willow|Dr.|Suite 130|Ashland|Will County|IL|64244|United States|-6|condo| +14601|AAAAAAAAJAJDAAAA|739|2nd |Way|Suite 90|Providence|Crow Wing County|MN|56614|United States|-6|single family| +14602|AAAAAAAAKAJDAAAA|571|4th |Way|Suite 0|Oak Ridge|Sheridan County|ND|58371|United States|-6|apartment| +14603|AAAAAAAALAJDAAAA|364|Spring Oak|Ave|Suite 400|Pleasant Hill|Cheshire County|NH|04204|United States|-5|apartment| +14604|AAAAAAAAMAJDAAAA|856|Hill North|Ct.|Suite X|Midway|San Juan County|WA|91904|United States|-8|apartment| +14605|AAAAAAAANAJDAAAA|412|Second |Court|Suite H|Edgewood|Nevada County|CA|90069|United States|-8|condo| +14606|AAAAAAAAOAJDAAAA|853|Williams |Court|Suite O|Spring Hill|Mellette County|SD|56787|United States|-7|single family| +14607|AAAAAAAAPAJDAAAA|65|Main |Wy|Suite O|Clinton|Gallatin County|MT|68222|United States|-7|condo| +14608|AAAAAAAAABJDAAAA|131|Lincoln |Circle|Suite V|Oak Hill|Walker County|AL|37838|United States|-6|apartment| +14609|AAAAAAAABBJDAAAA|843|Jefferson Adams|ST|Suite 160|Siloam|Chelan County|WA|98948|United States|-8|apartment| +14610|AAAAAAAACBJDAAAA|776|Madison 1st|Blvd|Suite O|Springtown|Martin County|IN|49858|United States|-5|single family| +14611|AAAAAAAADBJDAAAA|849|Mill Oak|Parkway|Suite P|Newport|Appanoose County|IA|51521|United States|-6|apartment| +14612|AAAAAAAAEBJDAAAA|553|Sycamore Valley|Ave|Suite 100|Woodlawn|Jackson County|WI|54098|United States|-6|single family| +14613|AAAAAAAAFBJDAAAA|972|Dogwood Lake|Drive|Suite 240|Montpelier|Hartford County|CT|09530|United States|-5|apartment| +14614|AAAAAAAAGBJDAAAA|354|14th Walnut|Ave|Suite 190|Ashland|Upson County|GA|34244|United States|-5|apartment| +14615|AAAAAAAAHBJDAAAA|576|Poplar |Parkway|Suite 230|Liberty|Fulton County|AR|73451|United States|-6|single family| +14616|AAAAAAAAIBJDAAAA|771|3rd |Drive|Suite 40|Mount Olive|Darlington County|SC|28059|United States|-5|apartment| +14617|AAAAAAAAJBJDAAAA|242|4th |Circle|Suite 20|Shady Grove|McDuffie County|GA|32812|United States|-5|condo| +14618|AAAAAAAAKBJDAAAA|509|Cedar Second|Cir.|Suite R|Glendale|Polk County|IA|53951|United States|-6|condo| +14619|AAAAAAAALBJDAAAA|447|Miller Spring|Cir.|Suite 60|Hopewell|Stevens County|WA|90587|United States|-8|condo| +14620|AAAAAAAAMBJDAAAA|358|Elm |Blvd|Suite E|Lebanon|Barton County|KS|62898|United States|-6|apartment| +14621|AAAAAAAANBJDAAAA|510|Park |Pkwy|Suite 140||Buncombe County|NC||United States||apartment| +14622|AAAAAAAAOBJDAAAA|404|15th |Dr.|Suite 250|Pine Grove|Coles County|IL|64593|United States|-6|condo| +14623|AAAAAAAAPBJDAAAA|610|Chestnut Park|Ln|Suite 300|Enterprise|Campbell County|TN|31757|United States|-5|single family| +14624|AAAAAAAAACJDAAAA|525|Johnson |Road|Suite Y|Richville|McIntosh County|ND|55945|United States|-6|apartment| +14625|AAAAAAAABCJDAAAA|281|Ash |Cir.|Suite V|Green Acres|Cumberland County|IL|67683|United States|-6|condo| +14626|AAAAAAAACCJDAAAA|578|Washington |Parkway|Suite M|Arlington|Esmeralda County|NV|86557|United States|-8|condo| +14627|AAAAAAAADCJDAAAA|576|Green |Road|Suite 130|Oakland|Archuleta County|CO|89843|United States|-7|single family| +14628|AAAAAAAAECJDAAAA|915|Cherry |Boulevard|Suite Y|Mountain View|Mathews County|VA|24466|United States|-5|apartment| +14629|AAAAAAAAFCJDAAAA|248|Church East|Ave|Suite X|Bath|Douglas County|KS|60573|United States|-6|single family| +14630|AAAAAAAAGCJDAAAA|491|3rd |Lane|Suite 100|Sunnyside|Slope County|ND|51952|United States|-6|condo| +14631|AAAAAAAAHCJDAAAA|700|Miller ||Suite 290|||MI||||| +14632|AAAAAAAAICJDAAAA|857|Dogwood |Ave|Suite K|Marion|Lycoming County|PA|10399|United States|-5|apartment| +14633|AAAAAAAAJCJDAAAA|305|Church |Drive|Suite T|Green Acres|Vernon Parish|LA|77683|United States|-6|apartment| +14634|AAAAAAAAKCJDAAAA|639||Boulevard|Suite A|Highland Park|Lake and Peninsula Borough|||United States||| +14635|AAAAAAAALCJDAAAA|589|Highland |Parkway|Suite 40|Unionville|Yellow Medicine County|MN|51711|United States|-6|single family| +14636|AAAAAAAAMCJDAAAA|650|Main |Road|Suite H|Woodville|De Witt County|IL|64289|United States|-6|condo| +14637|AAAAAAAANCJDAAAA|481|Maple |RD|Suite C|Royal|Racine County|WI|55819|United States|-6|condo| +14638|AAAAAAAAOCJDAAAA|911|South North|Ln|Suite V|Deerfield|Rowan County|NC|29840|United States|-5|apartment| +14639|AAAAAAAAPCJDAAAA|534|9th Central|Lane|Suite 330|Clifton|Dutchess County|NY|18014|United States|-5|single family| +14640|AAAAAAAAADJDAAAA|104|Hickory |Ln|Suite R|Concord|Ontario County|NY|14107|United States|-5|single family| +14641|AAAAAAAABDJDAAAA|625|Dogwood College|Lane|Suite J|Oakdale|District of Columbia|DC|29584|United States|-5|apartment| +14642|AAAAAAAACDJDAAAA|647|Ninth |Drive|Suite G|Spring Hill|Sandusky County|OH|46787|United States|-5|single family| +14643|AAAAAAAADDJDAAAA|730|Main Locust|Ln|Suite 330|Glenwood|Lawrence County|AL|33511|United States|-6|apartment| +14644|AAAAAAAAEDJDAAAA|290|View |RD|Suite 450|Cedar Grove|Lewis County|NY|10411|United States|-5|condo| +14645|AAAAAAAAFDJDAAAA|369|Lake |Wy|Suite 20|Spring Valley|Campbell County|VA|26060|United States|-5|apartment| +14646|AAAAAAAAGDJDAAAA|322|Seventh Mill|Dr.|Suite 60|Kingston|Loving County|TX|74975|United States|-6|condo| +14647|AAAAAAAAHDJDAAAA|858|10th Oak|Circle|Suite J|Cedar Grove|Hancock County|KY|40411|United States|-6|apartment| +14648|AAAAAAAAIDJDAAAA|955|West |Ave|Suite 490|Union|Bosque County|TX|78721|United States|-6|apartment| +14649|AAAAAAAAJDJDAAAA|605|Spring |Avenue|Suite I|Frankfort|Monroe County|TN|39681|United States|-6|single family| +14650|AAAAAAAAKDJDAAAA|992|3rd Sixth|Way|Suite 400|Hillcrest|Morris County|KS|63003|United States|-6|condo| +14651|AAAAAAAALDJDAAAA|877|River |ST|Suite 350|Wilson|Nicholas County|WV|26971|United States|-5|apartment| +14652|AAAAAAAAMDJDAAAA|176|Pine |Lane|Suite X|Marion|Crosby County|TX|70399|United States|-6|apartment| +14653|AAAAAAAANDJDAAAA|63|3rd Spring|Way|Suite 190|Oakdale|Hamilton County|KS|69584|United States|-6|single family| +14654|AAAAAAAAODJDAAAA|63|Miller Elm|Court|Suite 240|Marion|Catahoula Parish|LA|70399|United States|-6|apartment| +14655|AAAAAAAAPDJDAAAA|544|Central |Ave|Suite 290|Woodland|Stone County|MO|64854|United States|-6|single family| +14656|AAAAAAAAAEJDAAAA|398|Spring |Road|Suite L|Oak Hill|Sullivan County|TN|37838|United States|-6|condo| +14657|AAAAAAAABEJDAAAA|54|Jackson |Wy|Suite 460|Hillcrest|Bradford County|PA|13003|United States|-5|single family| +14658|AAAAAAAACEJDAAAA|890|4th Pine|Avenue|Suite 360|Greenville|Oklahoma County|OK|71387|United States|-6|single family| +14659|AAAAAAAADEJDAAAA|668|Willow 4th|Ln|Suite 0|Warwick|Yancey County|NC|21398|United States|-5|apartment| +14660|AAAAAAAAEEJDAAAA|330|North |Cir.|Suite Y|Springfield|Daviess County|IN|49303|United States|-5|apartment| +14661|AAAAAAAAFEJDAAAA|940|15th Birch|Dr.|Suite I|Union Hill|Kent County|DE|17746|United States|-5|apartment| +14662|AAAAAAAAGEJDAAAA|378|Birch Sunset|Parkway|Suite V|Woodland|James City County|VA|24854|United States|-5|single family| +14663|AAAAAAAAHEJDAAAA|894|River Meadow|Circle|Suite M|Concord|Routt County|CO|84107|United States|-7|condo| +14664|AAAAAAAAIEJDAAAA|235|Cedar Sixth|Street|Suite A|Springfield|Fulton County|IL|69303|United States|-6|condo| +14665|AAAAAAAAJEJDAAAA|54|Highland |Avenue|Suite V|Marietta|Richland County|IL|61178|United States|-6|single family| +14666|AAAAAAAAKEJDAAAA|724|Hillcrest |ST|Suite L|Pleasant Hill|Pike County|KY|43604|United States|-5|condo| +14667|AAAAAAAALEJDAAAA|93|Maple Valley|Ln|Suite W|Jamestown|Dillingham Census Area|AK|96867|United States|-9|single family| +14668|AAAAAAAAMEJDAAAA|308|12th 2nd|ST|Suite 450|Glenwood|Coffee County|AL|33511|United States|-6|condo| +14669|AAAAAAAANEJDAAAA|933|Washington |Wy|Suite J|Stringtown|Rutland County|VT|00762|United States|-5|single family| +14670|AAAAAAAAOEJDAAAA|484|Oak |Lane|Suite 0|Enterprise|Union County|GA|31757|United States|-5|apartment| +14671|AAAAAAAAPEJDAAAA|179|Washington Chestnut|Dr.|Suite W|Jackson|Columbia County|GA|39583|United States|-5|condo| +14672|AAAAAAAAAFJDAAAA||||Suite 130|||IN|41675|||| +14673|AAAAAAAABFJDAAAA|893|Mill |Pkwy|Suite K|Glendale|Tama County|IA|53951|United States|-6|condo| +14674|AAAAAAAACFJDAAAA|807|3rd |Street|Suite 90|Deerfield|Pitt County|NC|29840|United States|-5|single family| +14675|AAAAAAAADFJDAAAA|301|River |Ct.|Suite R|Edgewood|Pasquotank County|NC|20069|United States|-5|condo| +14676|AAAAAAAAEFJDAAAA|367|Park Maple|Wy|Suite 40|Mount Zion|Henrico County|VA|28054|United States|-5|condo| +14677|AAAAAAAAFFJDAAAA|401|Johnson Elm|Drive|Suite 290|Oakland|Lowndes County|MS|59843|United States|-6|apartment| +14678|AAAAAAAAGFJDAAAA|296|Hill |Road|Suite 270|Jackson|Clay County|TN|39583|United States|-5|condo| +14679|AAAAAAAAHFJDAAAA|519|2nd |Blvd|Suite 100|Bunker Hill|Langlade County|WI|50150|United States|-6|single family| +14680|AAAAAAAAIFJDAAAA|199|Highland 10th|Circle|Suite B|Riverview|Ramsey County|MN|59003|United States|-6|single family| +14681|AAAAAAAAJFJDAAAA|517|Locust West|Lane|Suite 310|New Hope|Kandiyohi County|MN|59431|United States|-6|condo| +14682|AAAAAAAAKFJDAAAA|137|Main Oak|Boulevard|Suite G|Highland Park|Dodge County|WI|56534|United States|-6|single family| +14683|AAAAAAAALFJDAAAA|949|Fourth View|Lane|Suite R|Brownsville|Jefferson County|FL|39310|United States|-5|apartment| +14684|AAAAAAAAMFJDAAAA|993|2nd View|Avenue|Suite V|Macedonia|Elko County|NV|81087|United States|-8|condo| +14685|AAAAAAAANFJDAAAA|997|4th |Pkwy|Suite K|Concord|Cooke County|TX|74107|United States|-6|single family| +14686|AAAAAAAAOFJDAAAA|6|Park Pine|Court|Suite I|Smith|Clinton County|KY|47317|United States|-6|condo| +14687|AAAAAAAAPFJDAAAA|428|Main |Parkway|Suite 370|Spring Hill|Merrimack County|NH|07387|United States|-5|apartment| +14688|AAAAAAAAAGJDAAAA|242|Church 14th|Court|Suite I|Concord|Dodge County|WI|54107|United States|-6|condo| +14689|AAAAAAAABGJDAAAA|912|Oak Madison|Ct.|Suite U|Brownsville|Madison County|IA|59310|United States|-6|apartment| +14690|AAAAAAAACGJDAAAA|692|Spring |Street|Suite L|Proctor|Orange County|FL|38140|United States|-5|apartment| +14691|AAAAAAAADGJDAAAA|304|Center Seventh|Pkwy|Suite P|Greenville|Pend Oreille County|WA|91387|United States|-8|apartment| +14692|AAAAAAAAEGJDAAAA|431|Jackson |Wy|Suite E|Ashley|Harding County|NM|84324|United States|-7|single family| +14693|AAAAAAAAFGJDAAAA|832|Jackson |RD|Suite 40|Woodville|Morris County|KS|64289|United States|-6|single family| +14694|AAAAAAAAGGJDAAAA|456|Center |Cir.|Suite A|Bridgeport|Wilson County|TX|75817|United States|-6|apartment| +14695|AAAAAAAAHGJDAAAA|793|Walnut |Court|Suite 270|Lakewood|Bell County|KY|48877|United States|-6|single family| +14696|AAAAAAAAIGJDAAAA|232|Main |Pkwy|Suite U|Unionville|Upshur County|TX|71711|United States|-6|single family| +14697|AAAAAAAAJGJDAAAA|700|Pine Elm|Road|Suite U|Oak Grove|Union County|AR|78370|United States|-6|condo| +14698|AAAAAAAAKGJDAAAA|786|Laurel Hill|Blvd|Suite Y|Riverside|Lincoln County|OR|99231|United States|-8|apartment| +14699|AAAAAAAALGJDAAAA|532|Adams |RD|Suite 40|Hillcrest|Crosby County|TX|73003|United States|-6|apartment| +14700|AAAAAAAAMGJDAAAA|537|Park Broadway|Lane|Suite 50|Carlton|Newport News city|VA|20918|United States|-5|single family| +14701|AAAAAAAANGJDAAAA|447|Pine Cedar|Cir.|Suite 210|Union|Allegan County|MI|48721|United States|-5|single family| +14702|AAAAAAAAOGJDAAAA|174|Park Park|Ct.|Suite H|Green Acres|Schley County|GA|37683|United States|-5|single family| +14703|AAAAAAAAPGJDAAAA|359|5th View|Ln|Suite T|Highland Park|Benton County|IA|56534|United States|-6|condo| +14704|AAAAAAAAAHJDAAAA|269|9th Meadow|Boulevard|Suite 490|Oakland|Terrell County|TX|79843|United States|-6|condo| +14705|AAAAAAAABHJDAAAA|748|Cherry Woodland|Boulevard|Suite 120|Five Forks|Muskogee County|OK|72293|United States|-6|apartment| +14706|AAAAAAAACHJDAAAA|578|Chestnut |Lane|Suite 180|Macedonia|Juab County|UT|81087|United States|-7|condo| +14707|AAAAAAAADHJDAAAA|290|Mill |Ave|Suite 310|Forest Hills|Newport News city|VA|29237|United States|-5|apartment| +14708|AAAAAAAAEHJDAAAA|440|Park |Avenue|Suite 250|Hidden Valley|Terrebonne Parish|LA|75521|United States|-6|single family| +14709|AAAAAAAAFHJDAAAA|638|View Main|Cir.|Suite M|Springfield|Iberia Parish|LA|79303|United States|-6|single family| +14710|AAAAAAAAGHJDAAAA|460|3rd Center|Boulevard|Suite 380|Glenwood|Putnam County|IL|63511|United States|-6|single family| +14711|AAAAAAAAHHJDAAAA|387|Oak |Boulevard|Suite 460|Five Forks|Orange County|CA|92293|United States|-8|condo| +14712|AAAAAAAAIHJDAAAA|515|Lake |Ave|Suite 390|Morgantown|Hamilton County|IN|49193|United States|-5|condo| +14713|AAAAAAAAJHJDAAAA|911|Smith |RD|Suite T|Pleasant Hill|Cooke County|TX|73604|United States|-6|condo| +14714|AAAAAAAAKHJDAAAA|235|East West|Wy|Suite 160|Summit|Logan County|KS|60499|United States|-6|single family| +14715|AAAAAAAALHJDAAAA|288|Church Chestnut|Boulevard|Suite 150|Walnut|Vernon County|WI|56245|United States|-6|apartment| +14716|AAAAAAAAMHJDAAAA|45|Willow |Court|Suite J|Woodlawn|Lauderdale County|TN|34098|United States|-6|condo| +14717|AAAAAAAANHJDAAAA|242|1st |Cir.|Suite E|Enterprise|Cayuga County|NY|11757|United States|-5|single family| +14718|AAAAAAAAOHJDAAAA|144|10th Walnut|Circle|Suite Q|Hopewell|Corson County|SD|50587|United States|-6|condo| +14719|AAAAAAAAPHJDAAAA|727|8th |Way|Suite H|Maple Grove|Quitman County|GA|38252|United States|-5|single family| +14720|AAAAAAAAAIJDAAAA|672|Sunset Spring|||||||United States|-7|condo| +14721|AAAAAAAABIJDAAAA|723|Jackson |ST|Suite 60|Oakdale|Essex County|VT|09584|United States|-5|apartment| +14722|AAAAAAAACIJDAAAA|725|Oak Cedar|Lane|Suite J|Franklin|Jackson County|IL|69101|United States|-6|single family| +14723|AAAAAAAADIJDAAAA|682|Miller |Pkwy|Suite F|Glenwood|Somerset County|PA|13511|United States|-5|condo| +14724|AAAAAAAAEIJDAAAA||Fifth College||||Washington County||||-5|| +14725|AAAAAAAAFIJDAAAA|||Lane||Woodville|Dillon County||||-5|| +14726|AAAAAAAAGIJDAAAA|975|Ridge Oak|||Marion|||10399|United States||single family| +14727|AAAAAAAAHIJDAAAA|120|8th |Ave|Suite Y|Summit|Effingham County|GA|30499|United States|-5|condo| +14728|AAAAAAAAIIJDAAAA|120|Mill |Ave|Suite F|Mount Olive|Nolan County|TX|78059|United States|-6|condo| +14729|AAAAAAAAJIJDAAAA|165|4th |Ct.|Suite 90|Oak Ridge|Ashland County|OH|48371|United States|-5|apartment| +14730|AAAAAAAAKIJDAAAA|31|East 12th|Court|Suite N|Oak Hill|Lowndes County|AL|37838|United States|-6|single family| +14731|AAAAAAAALIJDAAAA|819|Williams Mill|Cir.|Suite P|Midway|Shannon County|SD|51904|United States|-7|apartment| +14732|AAAAAAAAMIJDAAAA|469|8th |Drive|Suite W|Crossroads|Lawrence County|TN|30534|United States|-6|condo| +14733|AAAAAAAANIJDAAAA|755|Main |Ct.|Suite V|Concord|Red Willow County|NE|64107|United States|-7|single family| +14734|AAAAAAAAOIJDAAAA|569|Jackson Cedar|ST|Suite O|Five Forks|Howard County|NE|62293|United States|-7|condo| +14735|AAAAAAAAPIJDAAAA|160|Adams Franklin|Boulevard|Suite B|Florence|Atascosa County|TX|73394|United States|-6|single family| +14736|AAAAAAAAAJJDAAAA|848|7th |Avenue|Suite 120|Red Hill|Pike County|AR|74338|United States|-6|apartment| +14737|AAAAAAAABJJDAAAA|15|11th Elm|Way|Suite R|Plainview|Whiteside County|IL|63683|United States|-6|single family| +14738|AAAAAAAACJJDAAAA|31|Hillcrest Broadway|Street|Suite C|Oakdale|Panola County|MS|59584|United States|-6|apartment| +14739|AAAAAAAADJJDAAAA|851|Oak Cedar|Way|Suite 110|Shady Grove|Brown County|IN|42812|United States|-5|single family| +14740|AAAAAAAAEJJDAAAA||Walnut Jackson|Parkway|||La Plata County|CO||United States||apartment| +14741|AAAAAAAAFJJDAAAA|73|Ash 5th|Ave|Suite 120|Jamestown|Bandera County|TX|76867|United States|-6|single family| +14742|AAAAAAAAGJJDAAAA|296|Locust Willow|Circle|Suite 380|Five Points|Gates County|NC|26098|United States|-5|single family| +14743|AAAAAAAAHJJDAAAA|92|4th |Ct.|Suite 410|Five Forks|Levy County|FL|32293|United States|-5|single family| +14744|AAAAAAAAIJJDAAAA|231|Jefferson 1st|Blvd|Suite K|Bridgeport|Bronx County|NY|15817|United States|-5|single family| +14745|AAAAAAAAJJJDAAAA|912|9th Elm|Boulevard|Suite N|Belmont|Pittsylvania County|VA|20191|United States|-5|condo| +14746|AAAAAAAAKJJDAAAA|413|Washington Lincoln|Drive|Suite C|Oakland|Lafayette County|MS|59843|United States|-6|single family| +14747|AAAAAAAALJJDAAAA|198|Walnut |Ave|Suite N|Lakewood|Kittitas County|WA|98877|United States|-8|single family| +14748|AAAAAAAAMJJDAAAA|551|Lake Spring|Street|Suite L|Salem|Republic County|KS|68048|United States|-6|condo| +14749|AAAAAAAANJJDAAAA|863|Ridge |Road|Suite B|Newport|Carson County|TX|71521|United States|-6|condo| +14750|AAAAAAAAOJJDAAAA|137|Spring |Blvd|Suite 430|Georgetown|Runnels County|TX|77057|United States|-6|single family| +14751|AAAAAAAAPJJDAAAA|385|Park Laurel|Ct.|Suite 180|Friendship|Terrell County|TX|74536|United States|-6|condo| +14752|AAAAAAAAAKJDAAAA|570|Johnson Lincoln|Wy|Suite A|Fairfield|King County|WA|96192|United States|-8|condo| +14753|AAAAAAAABKJDAAAA||10th Meadow|Street|||Kittson County||58883|||condo| +14754|AAAAAAAACKJDAAAA|920|Jackson |Circle|Suite D|Stringtown|Hamilton County|IN|40162|United States|-5|single family| +14755|AAAAAAAADKJDAAAA|170|3rd |Blvd|Suite R|Greenville|Talladega County|AL|31387|United States|-6|single family| +14756|AAAAAAAAEKJDAAAA|55|Maple Hickory|Wy|Suite Y|Glenville|Greeley County|KS|63445|United States|-6|apartment| +14757|AAAAAAAAFKJDAAAA|||||Georgetown|Kane County|UT|87057|||| +14758|AAAAAAAAGKJDAAAA|374|Williams |Circle|Suite F|Freeport|Doddridge County|WV|21844|United States|-5|single family| +14759|AAAAAAAAHKJDAAAA|739|12th Spring|Ave|Suite 320|Hamilton|Santa Cruz County|CA|92808|United States|-8|single family| +14760|AAAAAAAAIKJDAAAA|556|6th |Court|Suite V|Union Hill|North Slope Borough|AK|97746|United States|-9|apartment| +14761|AAAAAAAAJKJDAAAA|345|Main Pine|Road|Suite G|Buena Vista|Guadalupe County|NM|85752|United States|-7|single family| +14762|AAAAAAAAKKJDAAAA|243|Chestnut |Boulevard|Suite 20|Walnut Grove|Hampton County|SC|27752|United States|-5|apartment| +14763|AAAAAAAALKJDAAAA|62|Maple |Avenue|Suite H|White Oak|McMinn County|TN|36668|United States|-6|single family| +14764|AAAAAAAAMKJDAAAA|773|Spring |Dr.|Suite 170|Glendale|Franklin County|VA|23951|United States|-5|single family| +14765|AAAAAAAANKJDAAAA|702|Oak Elm|Ave|Suite U|Deerfield|Crisp County|GA|39840|United States|-5|apartment| +14766|AAAAAAAAOKJDAAAA|322|Dogwood |Ave|Suite 420|Deerfield|Maries County|MO|69840|United States|-6|condo| +14767|AAAAAAAAPKJDAAAA|94|15th |Circle|Suite N|Sullivan|Sherburne County|MN|50451|United States|-6|apartment| +14768|AAAAAAAAALJDAAAA|356|Third Franklin|Ln|Suite G|Bunker Hill|Doddridge County|WV|20150|United States|-5|condo| +14769|AAAAAAAABLJDAAAA|638|Lake |Wy|Suite 170|Red Hill|Brown County|KS|64338|United States|-6|apartment| +14770|AAAAAAAACLJDAAAA|||||Liberty|Hutchinson County||73451|United States|-6|single family| +14771|AAAAAAAADLJDAAAA|826|Lakeview Woodland|Lane|Suite 50|Glenwood|Sheridan County|WY|83511|United States|-7|single family| +14772|AAAAAAAAELJDAAAA|843|Birch |Pkwy|Suite 400|Sunnyside|DeWitt County|TX|71952|United States|-6|condo| +14773|AAAAAAAAFLJDAAAA|261|Cherry |Blvd|Suite S|Bunker Hill|Travis County|TX|70150|United States|-6|apartment| +14774|AAAAAAAAGLJDAAAA||Poplar Center|Wy|Suite B||Dallas County||70499||-6|| +14775|AAAAAAAAHLJDAAAA|473|Walnut Elm|Ave|Suite C|Riverside|McCulloch County|TX|79231|United States|-6|single family| +14776|AAAAAAAAILJDAAAA|283|Wilson Elm|Dr.|Suite 10|Five Forks|Terry County|TX|72293|United States|-6|condo| +14777|AAAAAAAAJLJDAAAA|103|Franklin |Ln|Suite G|Pine Grove|Lincoln County|WA|94593|United States|-8|single family| +14778|AAAAAAAAKLJDAAAA|703|Park Green|Parkway|Suite 380|Shady Grove|Minidoka County|ID|82812|United States|-7|apartment| +14779|AAAAAAAALLJDAAAA|975|Pine Second|Court|Suite 70|Brownsville|Piatt County|IL|69310|United States|-6|single family| +14780|AAAAAAAAMLJDAAAA|794|Ninth Sycamore|Dr.|Suite W|Woodland|Knox County|IN|44854|United States|-5|condo| +14781|AAAAAAAANLJDAAAA|880|Lake Lincoln|Pkwy|Suite P|Galena|Summers County|WV|24369|United States|-5|condo| +14782|AAAAAAAAOLJDAAAA||Lakeview |||||MO|||-6|| +14783|AAAAAAAAPLJDAAAA|129|Miller |Street|Suite T|Jackson|Kent County|MI|49583|United States|-5|apartment| +14784|AAAAAAAAAMJDAAAA|153|4th Hill|ST|Suite Y|Union Hill|Rush County|IN|47746|United States|-5|condo| +14785|AAAAAAAABMJDAAAA|115|East |ST|Suite 260|Greenwood|Posey County|IN|48828|United States|-5|single family| +14786|AAAAAAAACMJDAAAA|331|Fourth |Boulevard|Suite 30|Fairview|Lee County|VA|25709|United States|-5|single family| +14787|AAAAAAAADMJDAAAA|621|Birch |RD|Suite 260|Oak Hill|Magoffin County|KY|47838|United States|-5|condo| +14788|AAAAAAAAEMJDAAAA|660|Lake Franklin|ST|Suite P|Union Hill|Sonoma County|CA|97746|United States|-8|condo| +14789|AAAAAAAAFMJDAAAA|531|12th Fourth|ST|Suite M|Paxton|Saguache County|CO|85669|United States|-7|apartment| +14790|AAAAAAAAGMJDAAAA|267|First Sunset|Way|Suite F|Highland|Nelson County|VA|29454|United States|-5|condo| +14791|AAAAAAAAHMJDAAAA|698|Cherry |Wy|Suite 330|Florence|Lynchburg city|VA|23394|United States|-5|condo| +14792|AAAAAAAAIMJDAAAA||Hickory |Court|Suite W|Mount Pleasant|Calhoun County||21933|United States||| +14793|AAAAAAAAJMJDAAAA|199|9th Poplar||Suite 190|Mount Zion||||United States||apartment| +14794|AAAAAAAAKMJDAAAA|314|Lakeview |RD|Suite E|Sulphur Springs|Franklin County|NY|18354|United States|-5|single family| +14795|AAAAAAAALMJDAAAA|926|Hickory |ST|Suite A|Oak Hill|Pasquotank County|NC|27838|United States|-5|apartment| +14796|AAAAAAAAMMJDAAAA|417|Lincoln |Drive|Suite H|Fairfield|Jim Wells County|TX|76192|United States|-6|single family| +14797|AAAAAAAANMJDAAAA|164|View 6th|Way|Suite 150|Franklin|Orleans County|VT|09701|United States|-5|condo| +14798|AAAAAAAAOMJDAAAA|785|Laurel North|Wy|Suite R|Sulphur Springs|Washington County|NC|28354|United States|-5|single family| +14799|AAAAAAAAPMJDAAAA|372|Sunset |Dr.|Suite A|Oak Ridge|Laurens County|SC|28371|United States|-5|condo| +14800|AAAAAAAAANJDAAAA|635|Elm |Street|Suite 480|Edgewood|Atoka County|OK|70069|United States|-6|apartment| +14801|AAAAAAAABNJDAAAA|437|Mill |Court|Suite R|Clinton|Cass County|ND|58222|United States|-6|single family| +14802|AAAAAAAACNJDAAAA|976|8th |Parkway|Suite 220|Mount Pleasant|Fulton County|IL|61933|United States|-6|condo| +14803|AAAAAAAADNJDAAAA|535|Franklin |Court|Suite 10|Wildwood|Roscommon County|MI|46871|United States|-5|condo| +14804|AAAAAAAAENJDAAAA|370|7th Second|Avenue|Suite X|Sulphur Springs|Benton County|MO|68354|United States|-6|single family| +14805|AAAAAAAAFNJDAAAA|708|7th |RD|Suite 370|Forest Hills|Whatcom County|WA|99237|United States|-8|single family| +14806|AAAAAAAAGNJDAAAA|666|Poplar Maple|Avenue|Suite D|Clinton|Loudoun County|VA|28222|United States|-5|condo| +14807|AAAAAAAAHNJDAAAA|414|Spruce |Blvd|Suite A|Blanchard|York County|ME|06585|United States|-5|apartment| +14808|AAAAAAAAINJDAAAA|628|Lake 3rd|Ln|Suite 440|Pleasant Valley|Ottawa County|KS|62477|United States|-6|apartment| +14809|AAAAAAAAJNJDAAAA|762|River Maple|ST|Suite 140|Glendale|Somervell County|TX|73951|United States|-6|single family| +14810|AAAAAAAAKNJDAAAA|731|Woodland |Ct.|Suite 430|New Hope|Hancock County|TN|39431|United States|-5|single family| +14811|AAAAAAAALNJDAAAA|497|West Forest|RD|Suite T|Oak Hill|Goodhue County|MN|57838|United States|-6|single family| +14812|AAAAAAAAMNJDAAAA|310|1st |Ave|Suite 420|Lebanon|Lewis and Clark County|MT|62898|United States|-7|condo| +14813|AAAAAAAANNJDAAAA|57|Park 5th|ST|Suite C|Springtown|Kearney County|NE|69858|United States|-7|condo| +14814|AAAAAAAAONJDAAAA||Franklin |Wy|Suite 180|||||United States|-5|| +14815|AAAAAAAAPNJDAAAA|763|First Jackson|Lane|Suite C|Springdale|Jackson County|TX|78883|United States|-6|condo| +14816|AAAAAAAAAOJDAAAA|366|1st |Blvd|Suite 470|Mount Olive|Hocking County|OH|48059|United States|-5|single family| +14817|AAAAAAAABOJDAAAA|881|8th Lee|Road|Suite 150|Brownsville|Park County|WY|89310|United States|-7|apartment| +14818|AAAAAAAACOJDAAAA|728|Jackson Spring|Blvd|Suite 80|Woodlawn|Clay County|IA|54098|United States|-6|condo| +14819|AAAAAAAADOJDAAAA|590|West |Cir.|Suite D|Cedar Grove|Snohomish County|WA|90411|United States|-8|apartment| +14820|AAAAAAAAEOJDAAAA|679|3rd |Dr.|Suite 430|Greenfield|Jones County|NC|25038|United States|-5|apartment| +14821|AAAAAAAAFOJDAAAA|977|13th Church|Lane|Suite L|Morgantown|Mahnomen County|MN|59193|United States|-6|single family| +14822|AAAAAAAAGOJDAAAA|954|Main |Parkway|Suite 0|Midway|York County|SC|21904|United States|-5|single family| +14823|AAAAAAAAHOJDAAAA|862|East River|Wy|Suite 400|Pine Grove|Barry County|MI|44593|United States|-5|condo| +14824|AAAAAAAAIOJDAAAA|687|Forest Main|Pkwy|Suite 250|Maple Grove|Orleans County|NY|18252|United States|-5|single family| +14825|AAAAAAAAJOJDAAAA|257|14th Spruce|Avenue|Suite 210|Glendale|Warren County|IA|53951|United States|-6|single family| +14826|AAAAAAAAKOJDAAAA|18|Sunset |Pkwy|Suite T|New Hope|Jay County|IN|49431|United States|-5|single family| +14827|AAAAAAAALOJDAAAA|255|Park |Boulevard|Suite 450|Newtown|Wilkes County|NC|21749|United States|-5|single family| +14828|AAAAAAAAMOJDAAAA|6|Birch |Cir.|Suite N|Colfax|Portage County|WI|52565|United States|-6|single family| +14829|AAAAAAAANOJDAAAA|4|Maple 11th|Ct.|Suite 50|Oak Grove|Menard County|TX|78370|United States|-6|single family| +14830|AAAAAAAAOOJDAAAA|738|2nd |Pkwy|Suite 420|Unionville|Carbon County|UT|81711|United States|-7|condo| +14831|AAAAAAAAPOJDAAAA|127|2nd Birch|Ct.|Suite 490|Glendale|Somerset County|PA|13951|United States|-5|single family| +14832|AAAAAAAAAPJDAAAA|88|Pine Lake|||Antioch|Kenedy County|||United States|-6|| +14833|AAAAAAAABPJDAAAA|599|10th 13th|Street|Suite 160|Enterprise|San Juan County|NM|81757|United States|-7|apartment| +14834|AAAAAAAACPJDAAAA|728|10th |Avenue|Suite J|Pleasant Grove|Gregg County|TX|74136|United States|-6|apartment| +14835|AAAAAAAADPJDAAAA|390|College |Wy|Suite V|Wildwood|Lewis County|WV|26871|United States|-5|single family| +14836|AAAAAAAAEPJDAAAA|252|Washington Park|Circle|Suite H|New Hope|Dougherty County|GA|39431|United States|-5|condo| +14837|AAAAAAAAFPJDAAAA|||Drive||Pine Grove|Hamilton County|||United States|-5|single family| +14838|AAAAAAAAGPJDAAAA|705|Ninth |Pkwy|Suite 90|Florence|Merced County|CA|93394|United States|-8|apartment| +14839|AAAAAAAAHPJDAAAA|724|Park Main|Way|Suite F|Summit|Ashtabula County|OH|40499|United States|-5|condo| +14840|AAAAAAAAIPJDAAAA|323|Maple |Dr.|Suite 90|Ashland|Tift County|GA|34244|United States|-5|condo| +14841|AAAAAAAAJPJDAAAA|366||Circle|Suite J|||IL|64107|United States|-6|condo| +14842|AAAAAAAAKPJDAAAA|717|Church Third|Boulevard|Suite 200|Lebanon|Polk County|AR|72898|United States|-6|condo| +14843|AAAAAAAALPJDAAAA|975|Ridge |Lane|Suite 420|Oakland|Potter County|SD|59843|United States|-7|condo| +14844|AAAAAAAAMPJDAAAA|849|Smith |Wy|Suite Y|Mountain View|Montgomery County|IA|54466|United States|-6|apartment| +14845|AAAAAAAANPJDAAAA|495|6th Church|Pkwy|Suite K|Greenfield|Jefferson County|WA|95038|United States|-8|condo| +14846|AAAAAAAAOPJDAAAA|983|West 8th|Road|Suite 170|Unionville|Fayette County|OH|41711|United States|-5|single family| +14847|AAAAAAAAPPJDAAAA|512|Jefferson |Boulevard|Suite 320|Lakeview|Delaware County|IA|58579|United States|-6|apartment| +14848|AAAAAAAAAAKDAAAA|828|13th Valley|RD|Suite B|Denmark|Nowata County|OK|75576|United States|-6|condo| +14849|AAAAAAAABAKDAAAA|657|Elm 3rd|Ct.|Suite P|Oakwood|Fall River County|SD|50169|United States|-6|single family| +14850|AAAAAAAACAKDAAAA|150|Locust Lee|Cir.|Suite X|Newtown|Oneida County|WI|51749|United States|-6|single family| +14851|AAAAAAAADAKDAAAA|762|6th Williams|Circle|Suite 400|Pleasant Grove|Braxton County|WV|24136|United States|-5|condo| +14852|AAAAAAAAEAKDAAAA|461|2nd |Avenue|Suite 400|Bunker Hill|DeKalb County|MO|60150|United States|-6|apartment| +14853|AAAAAAAAFAKDAAAA|812|7th Fifth|Pkwy|Suite N|Sulphur Springs|Haakon County|SD|58354|United States|-7|single family| +14854|AAAAAAAAGAKDAAAA|656|8th |Dr.|Suite E|Franklin|Heard County|GA|39101|United States|-5|apartment| +14855|AAAAAAAAHAKDAAAA|702|4th |RD|Suite C|Florence|Scott County|IL|63394|United States|-6|single family| +14856|AAAAAAAAIAKDAAAA|414|Madison |Pkwy|Suite 350|Greenville|Morton County|ND|51387|United States|-6|apartment| +14857|AAAAAAAAJAKDAAAA|566|1st |Road|Suite A|Friendship|Cheyenne County|KS|64536|United States|-6|condo| +14858|AAAAAAAAKAKDAAAA|793|Cedar Johnson|Parkway|Suite 270|Green Acres|Doddridge County|WV|27683|United States|-5|condo| +14859|AAAAAAAALAKDAAAA|700|East |Ave|Suite A|Arlington|Taylor County|WI|56557|United States|-6|apartment| +14860|AAAAAAAAMAKDAAAA|954|Central |Street|Suite E|Plainview|Freeborn County|MN|53683|United States|-6|apartment| +14861|AAAAAAAANAKDAAAA|427|Main 7th|Cir.|Suite Q|Youngstown|Louisa County|VA|20001|United States|-5|single family| +14862|AAAAAAAAOAKDAAAA|985|Mill Jackson|Lane|Suite 220|Glenwood|Washington County|FL|33511|United States|-5|condo| +14863|AAAAAAAAPAKDAAAA|57|7th Washington|RD|Suite 220|Crossroads|Lexington city|VA|20534|United States|-5|condo| +14864|AAAAAAAAABKDAAAA||2nd ||Suite 10|Sulphur Springs|Muskingum County||48354|United States|-5|single family| +14865|AAAAAAAABBKDAAAA|218|Third |Ct.|Suite 30|Lakewood|Hood County|TX|78877|United States|-6|apartment| +14866|AAAAAAAACBKDAAAA|558|Main |Parkway|Suite N|Spring Hill|Geary County|KS|66787|United States|-6|single family| +14867|AAAAAAAADBKDAAAA|467|4th Pine|Street|Suite 210|Valley View|Lewis and Clark County|MT|65124|United States|-7|condo| +14868|AAAAAAAAEBKDAAAA|419|Smith Fourth|Avenue|Suite Y|Green Acres|Henderson County|IL|67683|United States|-6|single family| +14869|AAAAAAAAFBKDAAAA|524|6th South|Street|Suite O|Kingston|Wilson County|NC|24975|United States|-5|apartment| +14870|AAAAAAAAGBKDAAAA|109|Lake |Ct.|Suite 290|Walnut Grove|Emanuel County|GA|37752|United States|-5|apartment| +14871|AAAAAAAAHBKDAAAA||||Suite A||Bay County|||United States|-5|| +14872|AAAAAAAAIBKDAAAA|981|11th Hill|Cir.|Suite 360|Five Points|Kleberg County|TX|76098|United States|-6|condo| +14873|AAAAAAAAJBKDAAAA|47|Poplar |Ct.|Suite 280|Salem|Virginia Beach city|VA|28048|United States|-5|single family| +14874|AAAAAAAAKBKDAAAA|604|College Woodland|Avenue|Suite 490|Delmar|Franklin County|NC|23957|United States|-5|single family| +14875|AAAAAAAALBKDAAAA|226|North Fourth|Ct.|Suite 30|Salem|Marshall County|OK|78048|United States|-6|single family| +14876|AAAAAAAAMBKDAAAA|393|Railroad Center|Blvd|Suite 140|Lakeview|Mississippi County|MO|68579|United States|-6|condo| +14877|AAAAAAAANBKDAAAA||1st ||Suite 110|Riverview|Wheeler County||||-7|single family| +14878|AAAAAAAAOBKDAAAA|875|Park Smith|Wy|Suite 170|Hopewell|Bureau County|IL|60587|United States|-6|apartment| +14879|AAAAAAAAPBKDAAAA|446|4th |Parkway|Suite G|Woodville|Morgan County|OH|44289|United States|-5|condo| +14880|AAAAAAAAACKDAAAA|865|7th |Street|Suite C|Kingston|Charlotte County|FL|34975|United States|-5|apartment| +14881|AAAAAAAABCKDAAAA|663|1st North|Court|Suite I|Marion|Herkimer County|NY|10399|United States|-5|single family| +14882|AAAAAAAACCKDAAAA|743|Lake |Cir.|Suite 190|Woodland|Addison County|VT|05454|United States|-5|condo| +14883|AAAAAAAADCKDAAAA|771|Cherry 4th|Drive|Suite L|Bunker Hill|Clinton County|PA|10150|United States|-5|single family| +14884|AAAAAAAAECKDAAAA|883|1st |Avenue|Suite 390|Clinton|Stanly County|NC|28222|United States|-5|single family| +14885|AAAAAAAAFCKDAAAA|282|Dogwood |Parkway|Suite W|Liberty|Jefferson County|GA|33451|United States|-5|condo| +14886|AAAAAAAAGCKDAAAA|474|Hill |Circle|Suite G|Franklin|Oscoda County|MI|49101|United States|-5|apartment| +14887|AAAAAAAAHCKDAAAA|783|Maple Church|Ave|Suite F|Longwood|Ulster County|NY|17021|United States|-5|condo| +14888|AAAAAAAAICKDAAAA|34|15th |Street|Suite 30|Five Forks|Union County|NC|22293|United States|-5|condo| +14889|AAAAAAAAJCKDAAAA|87|Ash |Dr.|Suite N|Oak Grove|Sutter County|CA|98370|United States|-8|single family| +14890|AAAAAAAAKCKDAAAA|296|Church View|ST|Suite 70|Springdale|Wood County|WI|58883|United States|-6|apartment| +14891|AAAAAAAALCKDAAAA|706|Elm |Pkwy|Suite E|Marion|Wheatland County|MT|60399|United States|-7|apartment| +14892|AAAAAAAAMCKDAAAA|146|Church 6th|ST|Suite 270|Centerville|Berkeley County|SC|20059|United States|-5|condo| +14893|AAAAAAAANCKDAAAA|299|Birch Valley|Ave|Suite W|Summit|Johnson County|KY|40499|United States|-6|single family| +14894|AAAAAAAAOCKDAAAA|253|Hill Cherry|Parkway|Suite 290|Friendship|Page County|VA|24536|United States|-5|single family| +14895|AAAAAAAAPCKDAAAA|538|7th |Street|Suite 80|Carthage|Monroe County|IA|51529|United States|-6|apartment| +14896|AAAAAAAAADKDAAAA|814|Railroad Fifteenth|Street|Suite Y|Pleasant Grove|Grayson County|VA|24136|United States|-5|apartment| +14897|AAAAAAAABDKDAAAA|161|4th |Boulevard|Suite H|Ashland|Cedar County|NE|64244|United States|-6|condo| +14898|AAAAAAAACDKDAAAA|868|Sixth |Blvd|Suite P|Glenwood|Pierce County|WA|93511|United States|-8|condo| +14899|AAAAAAAADDKDAAAA|815|5th |ST|Suite O|Unionville|Delta County|CO|81711|United States|-7|single family| +14900|AAAAAAAAEDKDAAAA|682|Mill |Dr.|Suite O|Liberty|Pitkin County|CO|83451|United States|-7|condo| +14901|AAAAAAAAFDKDAAAA||Lake Central|Avenue|||||26614|||| +14902|AAAAAAAAGDKDAAAA|467|Jefferson |ST|Suite 10|Bayside|Carroll County|IA|59550|United States|-6|apartment| +14903|AAAAAAAAHDKDAAAA|460|Ridge Cedar|Ln|Suite B|Lincoln|Barbour County|WV|21289|United States|-5|single family| +14904|AAAAAAAAIDKDAAAA|806|Broadway |RD|Suite Y|Riverside|Danville city|VA|29231|United States|-5|single family| +14905|AAAAAAAAJDKDAAAA|773|Lincoln |Drive|Suite 100|Oakwood|Walla Walla County|WA|90169|United States|-8|single family| +14906|AAAAAAAAKDKDAAAA|618|Mill River||Suite C|Bethel|||45281|United States|-5|single family| +14907|AAAAAAAALDKDAAAA|544|Adams Walnut|Wy|Suite 210|Bethel|Brunswick County|VA|25281|United States|-5|single family| +14908|AAAAAAAAMDKDAAAA|318|Second |Ave|Suite 140|Summit|Washington County|KY|40499|United States|-5|condo| +14909|AAAAAAAANDKDAAAA|70|Sunset Jackson|Street|Suite 110|Lakeside|Oldham County|TX|79532|United States|-6|apartment| +14910|AAAAAAAAODKDAAAA|934|13th |Boulevard|Suite 350|Brownsville|Emmons County|ND|59310|United States|-6|single family| +14911|AAAAAAAAPDKDAAAA|903|15th |Cir.|Suite S|Providence|Lake County|MN|56614|United States|-6|apartment| +14912|AAAAAAAAAEKDAAAA|403|Washington View|Avenue|Suite W|Whispering Pines|Iron County|MI|47609|United States|-5|single family| +14913|AAAAAAAABEKDAAAA|829|Fourth College||Suite 380|||ID|84098|United States|-7|single family| +14914|AAAAAAAACEKDAAAA|779|West Smith|Pkwy|Suite 350|Woodlawn|Clinton County|MI|44098|United States|-5|condo| +14915|AAAAAAAADEKDAAAA|170|Maple |Road|Suite I|Riverdale|Pierce County|ND|59391|United States|-6|single family| +14916|AAAAAAAAEEKDAAAA||12th View||Suite M|||GA||||apartment| +14917|AAAAAAAAFEKDAAAA|49|Jackson |Lane|Suite 450|Plainview|Grundy County|IA|53683|United States|-6|apartment| +14918|AAAAAAAAGEKDAAAA|295|Woodland Oak|Lane|Suite 190|Brownsville|Shelby County|KY|49310|United States|-5|apartment| +14919|AAAAAAAAHEKDAAAA|992|Spring |Dr.|Suite 460|Springfield|Knox County|KY|49303|United States|-5|apartment| +14920|AAAAAAAAIEKDAAAA|449|Mill |Wy|Suite L|Liberty|Baxter County|AR|73451|United States|-6|single family| +14921|AAAAAAAAJEKDAAAA|136|11th |Wy|Suite I|Oak Ridge|Tama County|IA|58371|United States|-6|single family| +14922|AAAAAAAAKEKDAAAA|581|Lincoln |Circle|Suite 350|Edgewood|Seward County|KS|60069|United States|-6|single family| +14923|AAAAAAAALEKDAAAA|566|8th |Dr.|Suite L|Springdale|Essex County|NJ|09483|United States|-5|condo| +14924|AAAAAAAAMEKDAAAA|38|15th 12th|Pkwy|Suite 120|Jamestown|Saunders County|NE|66867|United States|-7|single family| +14925|AAAAAAAANEKDAAAA|453|Birch Wilson|Drive|Suite H|Forest Hills|Hardin County|KY|49237|United States|-6|condo| +14926|AAAAAAAAOEKDAAAA|207|10th Pine|Wy|Suite 170|Stringtown|East Baton Rouge Parish|LA|70162|United States|-6|apartment| +14927|AAAAAAAAPEKDAAAA|1000|Eigth |Way|Suite 0|Sheffield|Day County|SD|56896|United States|-6|condo| +14928|AAAAAAAAAFKDAAAA|75|Fifth 2nd|ST|Suite R|Woodville|Pulaski County|KY|44289|United States|-5|single family| +14929|AAAAAAAABFKDAAAA||||Suite G|Howell|Hampton city|VA||||| +14930|AAAAAAAACFKDAAAA|868|Spring Cedar|Ave|Suite 440|Stringtown|Skagit County|WA|90162|United States|-8|condo| +14931|AAAAAAAADFKDAAAA|596|Birch 2nd|Lane|Suite 440|Woodlawn|Hidalgo County|NM|84098|United States|-7|condo| +14932|AAAAAAAAEFKDAAAA|951|Thirteenth Cedar|ST|Suite 390|Roxbury|Oneida County|NY|15508|United States|-5|apartment| +14933|AAAAAAAAFFKDAAAA|929|Cherry |Road|Suite N|Ashland|Colusa County|CA|94244|United States|-8|condo| +14934|AAAAAAAAGFKDAAAA|672|Lincoln Madison||Suite V||San Francisco County|CA|97066|United States||| +14935|AAAAAAAAHFKDAAAA|741|Locust |Dr.|Suite 370|Riverside|Henry County|GA|39231|United States|-5|apartment| +14936|AAAAAAAAIFKDAAAA|929|Sixth |Lane|Suite 140|Centerville|Greenup County|KY|40059|United States|-6|single family| +14937|AAAAAAAAJFKDAAAA|420|Fourteenth |Ln|Suite E|Wildwood|Macon County|MO|66871|United States|-6|single family| +14938|AAAAAAAAKFKDAAAA|91|2nd Davis|Dr.|Suite 380|Five Points|Hunt County|TX|76098|United States|-6|single family| +14939|AAAAAAAALFKDAAAA|558|1st |Court|Suite G|Camden|Gaines County|TX|77119|United States|-6|condo| +14940|AAAAAAAAMFKDAAAA|101|2nd |Road|Suite D|Fox|Stewart County|GA|30631|United States|-5|condo| +14941|AAAAAAAANFKDAAAA|90|First Jefferson|Circle|Suite 260|Jamestown|Kingsbury County|SD|56867|United States|-7|single family| +14942|AAAAAAAAOFKDAAAA|500|14th |RD|Suite M|Macedonia|Dawes County||61087|United States|-6|| +14943|AAAAAAAAPFKDAAAA|860|9th |Circle|Suite 50|Mount Zion|Perry County|MO|68054|United States|-6|single family| +14944|AAAAAAAAAGKDAAAA|241|4th |Ave|Suite 240|Hamilton|Carroll County|IL|62808|United States|-6|apartment| +14945|AAAAAAAABGKDAAAA|630|Church Oak|Way|Suite A|Concord|Clermont County|OH|44107|United States|-5|condo| +14946|AAAAAAAACGKDAAAA|65|Lakeview |Drive|Suite G|Shaw|Murray County|MN|50618|United States|-6|condo| +14947|AAAAAAAADGKDAAAA|853|Central |Wy|Suite 280|Ridgeville|Elbert County|GA|39306|United States|-5|condo| +14948|AAAAAAAAEGKDAAAA|127|Locust Chestnut|Street|Suite 220|Highland Park|Decatur County|IA|56534|United States|-6|single family| +14949|AAAAAAAAFGKDAAAA|578|Mill Lincoln|Court|Suite 90|Woodville|Lincoln County|MS|54289|United States|-6|single family| +14950|AAAAAAAAGGKDAAAA|579|Walnut Spring|Drive|Suite 430|Sunnyside|Webster County|IA|51952|United States|-6|apartment| +14951|AAAAAAAAHGKDAAAA|569|Seventh Walnut|Dr.|Suite 410|Marion|Calhoun County|IA|50399|United States|-6|single family| +14952|AAAAAAAAIGKDAAAA|492|Ridge |Pkwy|Suite 290|Enterprise|Kingfisher County|OK|71757|United States|-6|condo| +14953|AAAAAAAAJGKDAAAA|172|Spruce |Wy|Suite 490|Lebanon|McCone County|MT|62898|United States|-7|condo| +14954|AAAAAAAAKGKDAAAA|740|Mill Smith|Dr.|Suite Y|Sulphur Springs|Ward County|ND|58354|United States|-6|apartment| +14955|AAAAAAAALGKDAAAA|952|Meadow |Blvd|Suite 70|Glenwood|Kitsap County|WA|93511|United States|-8|apartment| +14956|AAAAAAAAMGKDAAAA|440|Valley |Ct.|Suite I|Edgewood|Furnas County|NE|60069|United States|-6|apartment| +14957|AAAAAAAANGKDAAAA|505|10th |Pkwy|Suite Y|Ashland|York County|SC|24244|United States|-5|apartment| +14958|AAAAAAAAOGKDAAAA|875|Forest |Dr.|Suite 130|Riverside|Escambia County|AL|39231|United States|-6|single family| +14959|AAAAAAAAPGKDAAAA|901|3rd |Parkway|Suite O|Belmont|Las Animas County|CO|80191|United States|-7|apartment| +14960|AAAAAAAAAHKDAAAA|605|Maple Main|Boulevard|Suite 350|Summit|Gates County|NC|20499|United States|-5|single family| +14961|AAAAAAAABHKDAAAA|724|Cedar |Lane|Suite J|Lakeside|Bremer County|IA|59532|United States|-6|apartment| +14962|AAAAAAAACHKDAAAA|531|Broadway Church|ST|Suite Q|Harmony|Pulaski County|VA|25804|United States|-5|single family| +14963|AAAAAAAADHKDAAAA|402|Sunset |Circle|Suite 50|Pleasant Grove|Walsh County|ND|54136|United States|-6|apartment| +14964|AAAAAAAAEHKDAAAA|53|Maple Williams|ST|Suite 450|Unionville|Calhoun County|MI|41711|United States|-5|condo| +14965|AAAAAAAAFHKDAAAA|945|9th ||||Trinity County||93683||-8|| +14966|AAAAAAAAGHKDAAAA|95|Walnut Adams|Drive|Suite 170|Fairview|Logan County|ND|55709|United States|-6|apartment| +14967|AAAAAAAAHHKDAAAA|527|Park 7th|Circle|Suite 180|Providence|Walsh County|ND|56614|United States|-6|single family| +14968|AAAAAAAAIHKDAAAA|848|Miller 3rd|Cir.|Suite 380|Woodland|Clark County|NV|84854|United States|-8|condo| +14969|AAAAAAAAJHKDAAAA|423|View |Court|Suite B|Fairfield|Stafford County|KS|66192|United States|-6|single family| +14970|AAAAAAAAKHKDAAAA|702|Maple |Dr.|Suite 360|Woodland|Bedford County|VA|24854|United States|-5|single family| +14971|AAAAAAAALHKDAAAA|258|2nd |Circle|Suite 10|Roscoe|McLeod County|MN|51854|United States|-6|apartment| +14972|AAAAAAAAMHKDAAAA|604|Main Smith|ST|Suite N|Newport|Staunton city|VA|21521|United States|-5|condo| +14973|AAAAAAAANHKDAAAA|441|Hickory |Ave|Suite R|Belmont|Jefferson County|WA|90191|United States|-8|apartment| +14974|AAAAAAAAOHKDAAAA|775|||Suite 460|Belmont|Tuscarawas County|OH|40191||-5|apartment| +14975|AAAAAAAAPHKDAAAA|310|Maple Tenth|Ln|Suite H|Oakwood|Hunterdon County|NJ|00769|United States|-5|single family| +14976|AAAAAAAAAIKDAAAA|429|Jackson Third|Road|Suite 150|Antioch|Orange County|NC|28605|United States|-5|condo| +14977|AAAAAAAABIKDAAAA|555|Washington |Dr.|Suite 420|Wilson|Salem city|VA|26971|United States|-5|condo| +14978|AAAAAAAACIKDAAAA|652|South 2nd|Road|Suite 240|Mount Pleasant|Perry County|MO|61933|United States|-6|apartment| +14979|AAAAAAAADIKDAAAA|251|View Third|Dr.|Suite 300|Newtown|Bertie County|NC|21749|United States|-5|single family| +14980|AAAAAAAAEIKDAAAA|547|Miller |Ct.|Suite 390|Marion|Taylor County|WI|50399|United States|-6|condo| +14981|AAAAAAAAFIKDAAAA|804|Sycamore |Pkwy|Suite Q|Blanchard|Jackson County|OH|45985|United States|-5|single family| +14982|AAAAAAAAGIKDAAAA|261|10th Wilson|Dr.|Suite 480|Walnut Grove|Big Horn County|WY|87752|United States|-7|single family| +14983|AAAAAAAAHIKDAAAA|154|Miller |Road|Suite 290|Union Hill|Lincoln County|GA|37746|United States|-5|condo| +14984|AAAAAAAAIIKDAAAA|656|Main |Ave|Suite V|Five Points|Breckinridge County|KY|46098|United States|-6|apartment| +14985|AAAAAAAAJIKDAAAA|398|Maple Spring|Road|Suite P|Lebanon|Hopkins County|TX|72898|United States|-6|single family| +14986|AAAAAAAAKIKDAAAA|764|Main |Way|Suite F|Crossroads|Chatham County|GA|30534|United States|-5|apartment| +14987|AAAAAAAALIKDAAAA|700|Laurel |Blvd|Suite T|Enterprise|Leake County|MS|51757|United States|-6|condo| +14988|AAAAAAAAMIKDAAAA|978|Cedar |Parkway|Suite 400|Antioch|Newton County|MS|58605|United States|-6|condo| +14989|AAAAAAAANIKDAAAA|177|Spring Williams|Lane|Suite 430|Hillcrest|Fond du Lac County|WI|53003|United States|-6|apartment| +14990|AAAAAAAAOIKDAAAA|972|Smith 5th|Circle|Suite Y|Kingston|Tift County|GA|34975|United States|-5|condo| +14991|AAAAAAAAPIKDAAAA|686|1st Valley|Court|Suite G|Bridgeport|Kerr County|TX|75817|United States|-6|apartment| +14992|AAAAAAAAAJKDAAAA|164|Cherry Jefferson|Ave|Suite 360|Lakewood|Sacramento County|CA|98877|United States|-8|condo| +14993|AAAAAAAABJKDAAAA|731|6th ||Suite T|Pleasant Hill|Fentress County|TN||United States||apartment| +14994|AAAAAAAACJKDAAAA|763||Court||Macedonia||MT|61087|United States|-7|single family| +14995|AAAAAAAADJKDAAAA|368|Hickory Spring|Court|Suite 190|Unionville|McDowell County|WV|21711|United States|-5|condo| +14996|AAAAAAAAEJKDAAAA|652|Mill North|Court|Suite 100|Brownsville|Isanti County|MN|59310|United States|-6|apartment| +14997|AAAAAAAAFJKDAAAA|341|Washington |Ave|Suite 450|Friendship|Gallatin County|IL|64536|United States|-6|apartment| +14998|AAAAAAAAGJKDAAAA|120|Madison Park|Ct.|Suite 460|Spring Hill|Ida County|IA|56787|United States|-6|apartment| +14999|AAAAAAAAHJKDAAAA|407|Miller 7th|Ave|Suite I|Oak Grove|Livingston County|IL|68370|United States|-6|condo| +15000|AAAAAAAAIJKDAAAA|548|East |Court|Suite 400|Macon|Cheyenne County|KS|60369|United States|-6|single family| +15001|AAAAAAAAJJKDAAAA|329|Ninth Sunset|RD|Suite W|Clinton|Brunswick County|VA|28222|United States|-5|apartment| +15002|AAAAAAAAKJKDAAAA|401|Maple Third|Ct.|Suite T|Church Hill|Geneva County|AL|33790|United States|-6|condo| +15003|AAAAAAAALJKDAAAA|126|10th Poplar|Wy|Suite 390|Bethel|Converse County|WY|85281|United States|-7|apartment| +15004|AAAAAAAAMJKDAAAA|256|13th Oak|Ave|Suite 120|Five Points|Campbell County|KY|46098|United States|-6|single family| +15005|AAAAAAAANJKDAAAA|935|Ninth Jefferson|Street|Suite I|Green Acres|Appling County|GA|37683|United States|-5|apartment| +15006|AAAAAAAAOJKDAAAA|623|Locust 13th|Court|Suite P|Mount Zion|Rockwall County|TX|78054|United States|-6|condo| +15007|AAAAAAAAPJKDAAAA|509|Wilson Pine|Circle|Suite K|Lebanon|Stewart County|GA|32898|United States|-5|condo| +15008|AAAAAAAAAKKDAAAA|322|Birch |Lane|Suite X|Lebanon|Humphreys County|MS|52898|United States|-6|condo| +15009|AAAAAAAABKKDAAAA|295|Sunset |Wy|Suite H|Union Hill|Terrell County|GA|37746|United States|-5|apartment| +15010|AAAAAAAACKKDAAAA|876|Lake South|Wy|Suite I|Providence|Highlands County|FL|36614|United States|-5|condo| +15011|AAAAAAAADKKDAAAA|681|Miller Locust|Pkwy|Suite 60|Belmont|Archer County|TX|70191|United States|-6|condo| +15012|AAAAAAAAEKKDAAAA|41|Woodland |Boulevard|Suite S|Pleasant Hill|Manassas city|VA|23604|United States|-5|apartment| +15013|AAAAAAAAFKKDAAAA|874|Birch Main|Pkwy|Suite 470|Friendship|Floyd County|KY|44536|United States|-6|condo| +15014|AAAAAAAAGKKDAAAA|732|Jefferson |Avenue|Suite L|Shiloh|Shenandoah County|VA|29275|United States|-5|condo| +15015|AAAAAAAAHKKDAAAA|94|Spring |Blvd|Suite 330|Marion|Daviess County|IN|40399|United States|-5|condo| +15016|AAAAAAAAIKKDAAAA|389|Williams |RD|Suite G|Brownsville|Douglas County|GA|39310|United States|-5|apartment| +15017|AAAAAAAAJKKDAAAA|585|Franklin |Ave|Suite 410|Caledonia|Scott County|TN|37411|United States|-6|single family| +15018|AAAAAAAAKKKDAAAA|469|8th 5th|Pkwy|Suite B|Oakland|Jefferson County|OH|49843|United States|-5|apartment| +15019|AAAAAAAALKKDAAAA|796|5th |RD|Suite J|Cedar|Amelia County|VA|21229|United States|-5|apartment| +15020|AAAAAAAAMKKDAAAA|97|Hill Center|Parkway|Suite 320|Riverview|Bay County|MI|49003|United States|-5|apartment| +15021|AAAAAAAANKKDAAAA|971|Lincoln View|Way|Suite G|Green Acres|Assumption Parish|LA|77683|United States|-6|condo| +15022|AAAAAAAAOKKDAAAA|496|First Mill|Wy|Suite 450|Elizabeth|Monroe County|NY|12935|United States|-5|apartment| +15023|AAAAAAAAPKKDAAAA|388|Cedar 2nd|Avenue|Suite 440|Scottsville|Stevens County|MN|54190|United States|-6|apartment| +15024|AAAAAAAAALKDAAAA|||Court||Lebanon||||United States||single family| +15025|AAAAAAAABLKDAAAA|294|Sycamore |Court|Suite I|Friendship|Washita County|OK|74536|United States|-6|condo| +15026|AAAAAAAACLKDAAAA|535|Dogwood |RD|Suite J|Crossroads|Hillsborough County|NH|01134|United States|-5|apartment| +15027|AAAAAAAADLKDAAAA|910|Maple Meadow|Blvd|Suite W|Elkton|Hancock County|KY|43481|United States|-6|condo| +15028|AAAAAAAAELKDAAAA|743|Poplar 10th|Ct.|Suite E|Bridgeport|Dewey County|OK|75817|United States|-6|apartment| +15029|AAAAAAAAFLKDAAAA|822|Valley |Ln|Suite 120|Woodville|Jackson County|FL|34289|United States|-5|single family| +15030|AAAAAAAAGLKDAAAA|612|11th 14th|Circle|Suite J|Pleasant Grove|Henry County|VA|24136|United States|-5|single family| +15031|AAAAAAAAHLKDAAAA|970|10th Pine|Ct.|Suite W|Mountain View|Park County|WY|84466|United States|-7|apartment| +15032|AAAAAAAAILKDAAAA|130|Maple |Boulevard|Suite W|Pleasant Grove|Stewart County|TN|34136|United States|-6|single family| +15033|AAAAAAAAJLKDAAAA|678|15th Spring|Avenue|Suite V|Harmony|Franklin County|WA|95804|United States|-8|single family| +15034|AAAAAAAAKLKDAAAA|232|Locust |Parkway|Suite U|Spring Hill|Loudon County|TN|36787|United States|-6|single family| +15035|AAAAAAAALLKDAAAA|77|Sixth Johnson|ST|Suite V|Bunker Hill|Orange County|TX|70150|United States|-6|single family| +15036|AAAAAAAAMLKDAAAA|826|2nd |Lane|Suite 350|Clinton|Cleveland County|NC|28222|United States|-5|condo| +15037|AAAAAAAANLKDAAAA|118|10th |Wy|Suite O|Harmony|Kenedy County|TX|75804|United States|-6|single family| +15038|AAAAAAAAOLKDAAAA|446|Broadway 12th|Cir.|Suite F|Centerville|Ballard County|KY|40059|United States|-6|apartment| +15039|AAAAAAAAPLKDAAAA|916|Spruce |Boulevard|Suite U|Maple Grove|Garza County|TX|78252|United States|-6|condo| +15040|AAAAAAAAAMKDAAAA|432|Smith |Parkway|Suite G|Oakland|Lincoln County|AR|79843|United States|-6|condo| +15041|AAAAAAAABMKDAAAA|334|Chestnut Park|Ave|Suite J|Pleasant Grove|Cortland County|NY|14136|United States|-5|condo| +15042|AAAAAAAACMKDAAAA|501|View River|Ln|Suite 120|Fairfield|Scotland County|NC|26192|United States|-5|apartment| +15043|AAAAAAAADMKDAAAA|896|Highland Railroad|ST|Suite 170|Liberty|Martinsville city|VA|23451|United States|-5|apartment| +15044|AAAAAAAAEMKDAAAA|911|Mill Hill|Road|Suite Y|Walnut Grove|Jackson County|CO|87752|United States|-7|condo| +15045|AAAAAAAAFMKDAAAA|304|2nd Meadow|Blvd|Suite R|Hamilton|Douglas County|IL|62808|United States|-6|single family| +15046|AAAAAAAAGMKDAAAA|957|Locust |Drive|Suite F|Summit|Alameda County|CA|90499|United States|-8|apartment| +15047|AAAAAAAAHMKDAAAA|593|9th Park|Cir.|Suite 0|Lakewood|Boyd County|KY|48877|United States|-6|single family| +15048|AAAAAAAAIMKDAAAA|1|Pine Mill|RD|Suite 290|Lakeview|Olmsted County|MN|58579|United States|-6|condo| +15049|AAAAAAAAJMKDAAAA|463|Locust 10th|Boulevard|Suite 400|Belmont|Shelby County|TX|70191|United States|-6|single family| +15050|AAAAAAAAKMKDAAAA|640|5th Lake|Drive|Suite 80|Riverview|Clay County|TN|39003|United States|-5|single family| +15051|AAAAAAAALMKDAAAA|144|2nd Elm|Wy|Suite 370|Rolling Hills|Elbert County|CO|87272|United States|-7|single family| +15052|AAAAAAAAMMKDAAAA|6|Sunset Lincoln|Lane|Suite J|Oakwood|Wakulla County|FL|30169|United States|-5|apartment| +15053|AAAAAAAANMKDAAAA|182|Lincoln Johnson|Cir.|Suite 250|Summit|Cumberland County|NC|20499|United States|-5|single family| +15054|AAAAAAAAOMKDAAAA|4|15th |Drive|Suite V|Perkins|Mason County|IL|61852|United States|-6|apartment| +15055|AAAAAAAAPMKDAAAA|250|Walnut |Circle|Suite 370|Franklin|Jeff Davis County|TX|79101|United States|-6|single family| +15056|AAAAAAAAANKDAAAA|327|Pine |Avenue|Suite 170|Wilson|Price County|WI|56971|United States|-6|single family| +15057|AAAAAAAABNKDAAAA|691|Pine 12th|Wy|Suite V|Farmington|Fulton County|KY|49145|United States|-6|condo| +15058|AAAAAAAACNKDAAAA|585|North |Road|Suite N|Ashland|Steuben County|NY|14244|United States|-5|single family| +15059|AAAAAAAADNKDAAAA|783|4th Hickory|Parkway|Suite M|Forest Hills|Baldwin County|GA|39237|United States|-5|single family| +15060|AAAAAAAAENKDAAAA|90|Sunset |Way|Suite 360|Lakewood|Pottawatomie County|KS|68877|United States|-6|condo| +15061|AAAAAAAAFNKDAAAA|460|Railroad |Wy|Suite 350|Guthrie|Rolette County|ND|51423|United States|-6|single family| +15062|AAAAAAAAGNKDAAAA|907|4th Fifth|Avenue|Suite N|Bridgeport|Portage County|WI|55817|United States|-6|apartment| +15063|AAAAAAAAHNKDAAAA|720|5th |Boulevard|Suite T|Pleasant Valley|Clinton County|MI|42477|United States|-5|apartment| +15064|AAAAAAAAINKDAAAA|530|Adams Second|Avenue|Suite J|Wilson|Pittsylvania County|VA|26971|United States|-5|condo| +15065|AAAAAAAAJNKDAAAA|539|Birch North|Ln|Suite 170|Woodville|Comal County|TX|74289|United States|-6|single family| +15066|AAAAAAAAKNKDAAAA|550|South Elm|Wy|Suite 340|Red Hill|Hartford County|CT|04938|United States|-5|apartment| +15067|AAAAAAAALNKDAAAA|454|Church |Cir.|Suite Y|Georgetown|Cooper County|MO|67057|United States|-6|condo| +15068|AAAAAAAAMNKDAAAA|46|2nd View|Dr.|Suite 60|Morris|Falls Church city|VA|26696|United States|-5|condo| +15069|AAAAAAAANNKDAAAA|67|Second |Circle|Suite 290|Providence|Lincoln County|WY|86614|United States|-7|single family| +15070|AAAAAAAAONKDAAAA|662|Highland |Drive|Suite 90|Highland Park|Clay County|IA|56534|United States|-6|apartment| +15071|AAAAAAAAPNKDAAAA|830|Broadway River|Street|Suite 180|Jamestown|Colonial Heights city|VA|26867|United States|-5|apartment| +15072|AAAAAAAAAOKDAAAA|354|Lakeview |Cir.|Suite K|Ashland|Sierra County|CA|94244|United States|-8|single family| +15073|AAAAAAAABOKDAAAA|286|Center Sycamore|Court|Suite S|Lakeside|Dallas County|AR|79532|United States|-6|single family| +15074|AAAAAAAACOKDAAAA|372|Oak |Boulevard|Suite 90|Lakewood|Johnson County|MO|68877|United States|-6|single family| +15075|AAAAAAAADOKDAAAA|327|4th |Drive|Suite R|Riverview|Phelps County|MO|69003|United States|-6|condo| +15076|AAAAAAAAEOKDAAAA|500|Park Washington|Street|Suite 360|Wilson|Shannon County|SD|56971|United States|-7|condo| +15077|AAAAAAAAFOKDAAAA|558|Cedar |Street|Suite 200|Mount Olive|Thurston County|WA|98059|United States|-8|apartment| +15078|AAAAAAAAGOKDAAAA|570|Pine Washington|Ct.|Suite S|Unionville|Norfolk city|VA|21711|United States|-5|apartment| +15079|AAAAAAAAHOKDAAAA|588|Woodland |Avenue|Suite 370|Enterprise|Livingston County|MO|61757|United States|-6|single family| +15080|AAAAAAAAIOKDAAAA|967|Laurel |Parkway|Suite 290|Walnut Grove|Judith Basin County|MT|67752|United States|-7|apartment| +15081|AAAAAAAAJOKDAAAA|271|10th View|RD|Suite U|Florence|Baltimore city|MD|23394|United States|-5|apartment| +15082|AAAAAAAAKOKDAAAA|361|Dogwood |Cir.|Suite 390|Philadelphia|Blount County|AL|35591|United States|-6|apartment| +15083|AAAAAAAALOKDAAAA|750|East |Ln|Suite P|Fairfield|Payette County|ID|86192|United States|-7|apartment| +15084|AAAAAAAAMOKDAAAA|506|Main |Lane|Suite D|Woodlawn|Philadelphia County|PA|14098|United States|-5|condo| +15085|AAAAAAAANOKDAAAA|627|Sunset |ST|Suite 240|Midway|Darlington County|SC|21904|United States|-5|apartment| +15086|AAAAAAAAOOKDAAAA|408|Oak |Cir.|Suite R|Brownsville|Alfalfa County|OK|79310|United States|-6|single family| +15087|AAAAAAAAPOKDAAAA|818|Davis 5th|Drive|Suite 190|Harmony|Montgomery County|IA|55804|United States|-6|apartment| +15088|AAAAAAAAAPKDAAAA|934|Mill 1st|Avenue|Suite 460|Wildwood|Wade Hampton Census Area|AK|96871|United States|-9|condo| +15089|AAAAAAAABPKDAAAA|481|River East|Parkway|Suite N|Greenfield|Hopewell city|VA|25038|United States|-5|single family| +15090|AAAAAAAACPKDAAAA|936|Jefferson |Avenue|Suite 110|Franklin|Clarke County|GA|39101|United States|-5|single family| +15091|AAAAAAAADPKDAAAA|650|15th |RD|Suite 250|Union Hill|White County|AR|77746|United States|-6|single family| +15092|AAAAAAAAEPKDAAAA|586|Cherry Woodland|Ln|Suite I|Oak Ridge|Clay County|AR|78371|United States|-6|condo| +15093|AAAAAAAAFPKDAAAA|795|Cedar Second|Road|Suite 310|Cedar Grove|Ray County|MO|60411|United States|-6|apartment| +15094|AAAAAAAAGPKDAAAA|430|Park 4th|Ln|Suite 450|Hamilton|Hoke County|NC|22808|United States|-5|condo| +15095|AAAAAAAAHPKDAAAA|912|Central 6th|Ln|Suite M|Pleasant Grove|Ottawa County|OK|74136|United States|-6|apartment| +15096|AAAAAAAAIPKDAAAA|422|Park Madison|Lane|Suite 290|Glendale|Ritchie County|WV|23951|United States|-5|apartment| +15097|AAAAAAAAJPKDAAAA|244|Ash |Avenue|Suite I|Five Forks|San Miguel County|CO|82293|United States|-7|condo| +15098|AAAAAAAAKPKDAAAA|18||||Summit|||70499|||condo| +15099|AAAAAAAALPKDAAAA|507|3rd Fourth|Ln|Suite 170|Centerville|Saline County|IL|60059|United States|-6|condo| +15100|AAAAAAAAMPKDAAAA|951|Broadway 14th|Dr.|Suite F|Summit|Galveston County|TX|70499|United States|-6|condo| +15101|AAAAAAAANPKDAAAA|846|Cherry |Dr.|Suite P|Unionville|Nueces County|TX|71711|United States|-6|condo| +15102|AAAAAAAAOPKDAAAA|960|Wilson |Parkway|Suite 0|Riverview|Koochiching County|MN|59003|United States|-6|apartment| +15103|AAAAAAAAPPKDAAAA|614|Laurel |Court|Suite 250|Friendship|Washington County|AL|34536|United States|-6|apartment| +15104|AAAAAAAAAALDAAAA|438|Park |Street|Suite F|Mount Olive|Houston County|GA|38059|United States|-5|single family| +15105|AAAAAAAABALDAAAA|994|Willow North|Drive|Suite M|Red Hill|Muskegon County|MI|44338|United States|-5|condo| +15106|AAAAAAAACALDAAAA|307|Center 14th|Court|Suite 170|Jamestown|Nevada County|AR|76867|United States|-6|single family| +15107|AAAAAAAADALDAAAA|519|Oak Davis|Court|Suite W|Hopewell|Anson County|NC|20587|United States|-5|condo| +15108|AAAAAAAAEALDAAAA|531|5th 1st|Lane|Suite 280|Fairview|Madison Parish|LA|75709|United States|-6|single family| +15109|AAAAAAAAFALDAAAA|642|Spring |Way|Suite F|Midway|Texas County|OK|71904|United States|-6|single family| +15110|AAAAAAAAGALDAAAA|611|11th |Parkway|Suite K|Shady Grove|Lucas County|IA|52812|United States|-6|condo| +15111|AAAAAAAAHALDAAAA|65|Woodland |Dr.|Suite K|Fairfield|Wilkinson County|MS|56192|United States|-6|single family| +15112|AAAAAAAAIALDAAAA|853|7th |RD|Suite Y|Gilmore|Pope County|MN|55464|United States|-6|condo| +15113|AAAAAAAAJALDAAAA|258|Second |Ct.|Suite Y|Stringtown|Faulkner County|AR|70162|United States|-6|condo| +15114|AAAAAAAAKALDAAAA|860|Madison |Wy|Suite V|Deerfield|Ingham County|MI|49840|United States|-5|single family| +15115|AAAAAAAALALDAAAA|907|6th |Drive|Suite U|Providence|Armstrong County|PA|16614|United States|-5|condo| +15116|AAAAAAAAMALDAAAA|644|Pine Oak|Lane|Suite 0|Green Acres|Anderson County|TN|37683|United States|-5|single family| +15117|AAAAAAAANALDAAAA|793|5th 3rd|Drive|Suite W|Greenville|Golden Valley County|ND|51387|United States|-6|single family| +15118|AAAAAAAAOALDAAAA|232|13th |Pkwy|Suite 460|Farmington|Waynesboro city|VA|29145|United States|-5|condo| +15119|AAAAAAAAPALDAAAA|508|1st Second|Street|Suite N|Pleasant Hill|Weber County|UT|83604|United States|-7|condo| +15120|AAAAAAAAABLDAAAA|668|Adams |Pkwy|Suite O|Ellsworth|Baltimore County|MD|25079|United States|-5|single family| +15121|AAAAAAAABBLDAAAA|557|Washington |Lane|Suite 40|Liberty|Carroll County|IL|63451|United States|-6|apartment| +15122|AAAAAAAACBLDAAAA|929|Forest Central|Ave|Suite 400|Echo|Pembina County|ND|50348|United States|-6|apartment| +15123|AAAAAAAADBLDAAAA|879|Railroad Oak|RD|Suite L|Franklin|Benton County|WA|99101|United States|-8|condo| +15124|AAAAAAAAEBLDAAAA|11|Lincoln |Lane|Suite 410|White Oak|Spencer County|KY|46668|United States|-5|single family| +15125|AAAAAAAAFBLDAAAA|173|Green Williams|Road|Suite 450|Springfield|Scott County|MS|59303|United States|-6|single family| +15126|AAAAAAAAGBLDAAAA|561|Highland Ridge|Parkway|Suite A|Silver Creek|Clearwater County|MN|54546|United States|-6|apartment| +15127|AAAAAAAAHBLDAAAA|52|Lake Sixth|Ct.|Suite H|Cedar Grove|Skagit County|WA|90411|United States|-8|apartment| +15128|AAAAAAAAIBLDAAAA|632|Twelfth |Cir.|Suite U|Lakeview|Leon County|FL|38579|United States|-5|single family| +15129|AAAAAAAAJBLDAAAA|154|Railroad |Dr.|Suite 410|Bridgeport|Trigg County|KY|45817|United States|-5|condo| +15130|AAAAAAAAKBLDAAAA|183|14th |Street|Suite S|Unionville|Dukes County|MA|02311|United States|-5|apartment| +15131|AAAAAAAALBLDAAAA|800|First Jackson|Parkway|Suite C|Willow|Atoka County|OK|76798|United States|-6|single family| +15132|AAAAAAAAMBLDAAAA|187|Hickory 3rd|Dr.|Suite R|Summit|Tulare County|CA|90499|United States|-8|apartment| +15133|AAAAAAAANBLDAAAA|704|Forest Third|Blvd|Suite 450|Waterloo|Grundy County|IL|61675|United States|-6|apartment| +15134|AAAAAAAAOBLDAAAA|530|Williams Fifth|Lane|Suite O|Oakwood|Bowman County|ND|50169|United States|-6|single family| +15135|AAAAAAAAPBLDAAAA|636|Jackson Central|Dr.|Suite Y|Lakeview|Morgan County|MO|68579|United States|-6|condo| +15136|AAAAAAAAACLDAAAA|189|Broadway Main|Pkwy|Suite 60|Woodville|Glasscock County|TX|74289|United States|-6|condo| +15137|AAAAAAAABCLDAAAA|146|Forest |Parkway|Suite 40|Evansville|Columbus County|NC|25274|United States|-5|condo| +15138|AAAAAAAACCLDAAAA|792|2nd 2nd|Ave|Suite Y|Valley View|Jefferson County|OR|95124|United States|-8|single family| +15139|AAAAAAAADCLDAAAA|717|Williams Jefferson|Ct.|Suite S|Plainview|Montgomery County|IN|43683|United States|-5|apartment| +15140|AAAAAAAAECLDAAAA|970|Hill Second|Parkway|Suite E|Springdale|Pontotoc County|MS|58883|United States|-6|apartment| +15141|AAAAAAAAFCLDAAAA|592|Broadway |Wy|Suite K|Jamestown|Johnson County|TX|76867|United States|-6|single family| +15142|AAAAAAAAGCLDAAAA|112|Smith Ridge|Parkway|Suite Q|Friendship|Glenn County|CA|94536|United States|-8|condo| +15143|AAAAAAAAHCLDAAAA|708|14th Poplar|Cir.|Suite S|Greenville|Spencer County|KY|41387|United States|-5|apartment| +15144|AAAAAAAAICLDAAAA|733|Oak |Drive|Suite K|Buena Vista|Maverick County|TX|75752|United States|-6|single family| +15145|AAAAAAAAJCLDAAAA|630|13th 4th|ST|Suite 210|Pleasant Hill|Essex County|VT|04204|United States|-5|condo| +15146|AAAAAAAAKCLDAAAA|610|Ridge |Street|Suite K|Green Acres|Lafayette Parish|LA|77683|United States|-6|apartment| +15147|AAAAAAAALCLDAAAA|553|15th |Blvd|Suite 80|Five Forks|Habersham County|GA|32293|United States|-5|single family| +15148|AAAAAAAAMCLDAAAA|5|Oak |Drive|Suite 210|Riverview|Pittsburg County|OK|79003|United States|-6|apartment| +15149|AAAAAAAANCLDAAAA|121|Eigth |Ct.|Suite B|Kingston|Dickey County|ND|54975|United States|-6|condo| +15150|AAAAAAAAOCLDAAAA|658|Fifth 2nd|ST|Suite D|Fisher|Beaver County|PA|12819|United States|-5|condo| +15151|AAAAAAAAPCLDAAAA|540|Green |Boulevard|Suite P|Oak Grove|Custer County|NE|68370|United States|-6|condo| +15152|AAAAAAAAADLDAAAA|647|1st Highland|Parkway|Suite 10|Forest Hills|Beaufort County|SC|29237|United States|-5|apartment| +15153|AAAAAAAABDLDAAAA|||RD|Suite W|Shady Grove|Lincoln County|NV|82812|United States||single family| +15154|AAAAAAAACDLDAAAA|693|North |Ct.|Suite Q|White Oak|Greenwood County|KS|66668|United States|-6|apartment| +15155|AAAAAAAADDLDAAAA|282|Cedar Center|Way|Suite R|Union Hill|Dallas County|AL|37746|United States|-6|single family| +15156|AAAAAAAAEDLDAAAA|43|8th |Court|Suite J|Georgetown|Lincoln County|MS|57057|United States|-6|condo| +15157|AAAAAAAAFDLDAAAA|552|Cedar 6th|Way|Suite 380|Green Acres|Gove County|KS|67683|United States|-6|apartment| +15158|AAAAAAAAGDLDAAAA|441|Main |RD|Suite W|Green Acres|Greenwood County|SC|27683|United States|-5|single family| +15159|AAAAAAAAHDLDAAAA|961|Hill View|RD|Suite 30|Oak Grove|Marshall County|MS|58370|United States|-6|apartment| +15160|AAAAAAAAIDLDAAAA|396|Fifth |Blvd|Suite 450|Hartland|San Benito County|CA|96594|United States|-8|apartment| +15161|AAAAAAAAJDLDAAAA|257|15th |Road|Suite 420|Ashland|Carteret County|NC|24244|United States|-5|condo| +15162|AAAAAAAAKDLDAAAA|646|Cherry |Drive|Suite N|Greenfield|Rock County|NE|65038|United States|-7|condo| +15163|AAAAAAAALDLDAAAA|321|1st Pine|Ct.|Suite 110|Enterprise|Allen County|KS|61757|United States|-6|apartment| +15164|AAAAAAAAMDLDAAAA||||Suite 280||Macomb County||49583|United States|-5|| +15165|AAAAAAAANDLDAAAA|956|Birch Second|Blvd|Suite P|Oak Hill|Park County|CO|87838|United States|-7|condo| +15166|AAAAAAAAODLDAAAA|235|Adams |Court|Suite N|Green Acres|Acadia Parish|LA|77683|United States|-6|apartment| +15167|AAAAAAAAPDLDAAAA|482|Cherry |Road|||Little River County|||United States|-6|| +15168|AAAAAAAAAELDAAAA|239|Mill |Ct.|Suite E|Pleasant Hill|Effingham County|IL|63604|United States|-6|single family| +15169|AAAAAAAABELDAAAA|813|1st Spring|Boulevard|Suite L|Five Forks|Starke County|IN|42293|United States|-5|single family| +15170|AAAAAAAACELDAAAA|66|Main Jefferson|ST|Suite 470|Lakeview|Crisp County|GA|38579|United States|-5|apartment| +15171|AAAAAAAADELDAAAA|284|Park |Pkwy|Suite P|Hopewell|Klamath County|OR|90587|United States|-8|apartment| +15172|AAAAAAAAEELDAAAA|530|Main Lincoln|Pkwy|Suite N|Spring Hill|McDowell County|WV|26787|United States|-5|condo| +15173|AAAAAAAAFELDAAAA|484|Second Washington|Avenue|Suite G|Oak Ridge|Metcalfe County|KY|48371|United States|-5|single family| +15174|AAAAAAAAGELDAAAA|501|Hill |Boulevard|Suite 160|Lincoln|Mariposa County|CA|91289|United States|-8|condo| +15175|AAAAAAAAHELDAAAA|444|Church |Lane|Suite I|Unionville|Sherman County|TX|71711|United States|-6|single family| +15176|AAAAAAAAIELDAAAA|643|Valley Davis|Dr.|Suite 220|Salem|Richmond County|NY|18048|United States|-5|condo| +15177|AAAAAAAAJELDAAAA|916|Locust Willow|Road||Riverdale|Pend Oreille County|WA||||condo| +15178|AAAAAAAAKELDAAAA|788|6th |Ln|Suite Q|Florence|Sanilac County|MI|43394|United States|-5|single family| +15179|AAAAAAAALELDAAAA|484|South 1st|Blvd|Suite 80|White Oak|Decatur County|IA|56668|United States|-6|single family| +15180|AAAAAAAAMELDAAAA|535|Spruce 2nd|ST|Suite C|Roxbury|Worth County|IA|55508|United States|-6|condo| +15181|AAAAAAAANELDAAAA|284|7th Oak|Ave|Suite 450|Lincoln|Woodford County|IL|61289|United States|-6|apartment| +15182|AAAAAAAAOELDAAAA|936|Sunset Spring|Road|Suite 310|Lincoln|Schenectady County|NY|11289|United States|-5|condo| +15183|AAAAAAAAPELDAAAA|686|Oak Railroad|Ct.|Suite 440|Stringtown|Roosevelt County|NM|80162|United States|-7|condo| +15184|AAAAAAAAAFLDAAAA|679|Birch |Lane|Suite 490|Shiloh|Cherokee County|NC|29275|United States|-5|condo| +15185|AAAAAAAABFLDAAAA|908|Wilson Center|Parkway|Suite 30|Mount Zion|Fremont County|CO|88054|United States|-7|single family| +15186|AAAAAAAACFLDAAAA|576|First Mill|Way|Suite 290|Oak Ridge|Wheatland County|MT|68371|United States|-7|apartment| +15187|AAAAAAAADFLDAAAA|364|Franklin Lake|Ave|Suite X|Bethel|Williamson County|IL|65281|United States|-6|apartment| +15188|AAAAAAAAEFLDAAAA|856|Maple |Ln|Suite 70|Walnut Grove|Austin County|TX|77752|United States|-6|single family| +15189|AAAAAAAAFFLDAAAA|219|8th Chestnut|Wy|Suite 320|Sunnyside|Wabaunsee County|KS|61952|United States|-6|condo| +15190|AAAAAAAAGFLDAAAA|291|Second |Ave|Suite V|Greenville|Wheeler County|GA|31387|United States|-5|apartment| +15191|AAAAAAAAHFLDAAAA|572|Poplar 10th|ST|Suite 350|Macedonia|Colbert County|AL|31087|United States|-6|single family| +15192|AAAAAAAAIFLDAAAA|600|Birch |Ln|Suite 180|Pleasant Grove|Warren County|MO|64136|United States|-6|condo| +15193|AAAAAAAAJFLDAAAA|489|Park Spring|Wy|Suite 100|Mount Vernon|Yakima County|WA|98482|United States|-8|condo| +15194|AAAAAAAAKFLDAAAA|610|Willow 3rd|Blvd|Suite 50|Wilson|Red Willow County|NE|66971|United States|-7|apartment| +15195|AAAAAAAALFLDAAAA|932|9th |RD|Suite 400|Arlington|Mitchell County|TX|76557|United States|-6|apartment| +15196|AAAAAAAAMFLDAAAA|909|Cedar |Cir.|Suite I|Elgin|Knox County|ME|01045|United States|-5|condo| +15197|AAAAAAAANFLDAAAA|750|College Jefferson|Circle|Suite 410|Kingston|Oliver County|ND|54975|United States|-6|single family| +15198|AAAAAAAAOFLDAAAA|890|1st East|Circle|Suite 400|Forest Hills|Lander County|NV|89237|United States|-8|apartment| +15199|AAAAAAAAPFLDAAAA|608|Walnut Broadway|Road|Suite 60|Lakeside|Orange County|VA|29532|United States|-5|condo| +15200|AAAAAAAAAGLDAAAA|597|11th Mill|Ave|Suite F|Belmont|Washtenaw County|MI|40191|United States|-5|apartment| +15201|AAAAAAAABGLDAAAA|703|Jackson |Ct.|Suite 270|Unionville|Mecklenburg County|NC|21711|United States|-5|single family| +15202|AAAAAAAACGLDAAAA|951|Maple |Way|Suite Y|Lebanon|Fairfield County|OH|42898|United States|-5|single family| +15203|AAAAAAAADGLDAAAA|480|7th East|Circle|Suite C|Riverside|Phillips County|MT|69231|United States|-7|condo| +15204|AAAAAAAAEGLDAAAA|431|Jackson |Lane|Suite W|Greenville|Archuleta County|CO|81387|United States|-7|single family| +15205|AAAAAAAAFGLDAAAA|738|Main |Road|Suite S|Newport|Searcy County|AR|71521|United States|-6|single family| +15206|AAAAAAAAGGLDAAAA|79|South |Avenue|Suite 180|Bunker Hill|Grayson County|TX|70150|United States|-6|single family| +15207|AAAAAAAAHGLDAAAA|423|2nd |Circle|Suite 330|Antioch|Ocean County|NJ|09205|United States|-5|apartment| +15208|AAAAAAAAIGLDAAAA|52|Twelfth Lincoln|Ln|Suite H|Westminster|Clay County|GA|36549|United States|-5|apartment| +15209|AAAAAAAAJGLDAAAA|766|Center |Pkwy|Suite R|Highland|Madison County|VA|29454|United States|-5|apartment| +15210|AAAAAAAAKGLDAAAA|308|15th |Parkway|Suite 280|Springfield|Santa Cruz County|AZ|89303|United States|-7|single family| +15211|AAAAAAAALGLDAAAA|240|7th Third|Blvd|Suite 480|Highland Park|Jack County|TX|76534|United States|-6|single family| +15212|AAAAAAAAMGLDAAAA|947|3rd Park|Way|Suite 80|Bennett|Lincoln County|OK|71715|United States|-6|single family| +15213|AAAAAAAANGLDAAAA|987|15th Smith|Ct.|Suite 410|Lakeview|Grainger County|TN|38579|United States|-5|condo| +15214|AAAAAAAAOGLDAAAA|29|4th |RD|Suite P|Macedonia|Haines Borough|AK|91087|United States|-9|single family| +15215|AAAAAAAAPGLDAAAA|895|Willow 14th|Dr.|Suite 100|Sunnyside|Henderson County|IL|61952|United States|-6|apartment| +15216|AAAAAAAAAHLDAAAA|331|Sycamore |Boulevard|Suite 150|Crossroads|Marshall County|AL|30534|United States|-6|single family| +15217|AAAAAAAABHLDAAAA|315|9th Smith|Pkwy|Suite 480|Valley View|Franklin County|TN|35124|United States|-5|condo| +15218|AAAAAAAACHLDAAAA|746|9th |Wy|Suite 420|Mount Zion|Carbon County|WY|88054|United States|-7|condo| +15219|AAAAAAAADHLDAAAA|726|Park |ST|Suite 300|Wilson|Perkins County|NE|66971|United States|-7|apartment| +15220|AAAAAAAAEHLDAAAA|821|Woodland Pine|Cir.|Suite K|Wilson|Cloud County|KS|66971|United States|-6|condo| +15221|AAAAAAAAFHLDAAAA|848|1st |Road|Suite H|Glenwood|Douglas County|MO|63511|United States|-6|single family| +15222|AAAAAAAAGHLDAAAA||Washington 6th|||Oakwood||NC||United States||apartment| +15223|AAAAAAAAHHLDAAAA|68|9th Maple|Blvd|Suite 220|Wildwood|Pepin County|WI|56871|United States|-6|condo| +15224|AAAAAAAAIHLDAAAA|516|1st |Court|Suite F|Pleasant Grove|Benton County|IN|44136|United States|-5|condo| +15225|AAAAAAAAJHLDAAAA|334|11th Cedar|Street|Suite J|Frenchtown|Runnels County|TX|72629|United States|-6|single family| +15226|AAAAAAAAKHLDAAAA|461|Sycamore |ST|Suite 310|Liberty|McPherson County|SD|53451|United States|-7|single family| +15227|AAAAAAAALHLDAAAA|445|Chestnut |Dr.|Suite 30|Cedar Grove|Lafourche Parish|LA|70411|United States|-6|single family| +15228|AAAAAAAAMHLDAAAA|142|2nd Green|Road|Suite K|Woodlawn|Huerfano County|CO|84098|United States|-7|condo| +15229|AAAAAAAANHLDAAAA|310|Cedar |RD|Suite 0|Marion|Barry County|MO|60399|United States|-6|condo| +15230|AAAAAAAAOHLDAAAA|168|Spring Jefferson|Ln|Suite 470|Oak Grove|Bottineau County|ND|58370|United States|-6|apartment| +15231|AAAAAAAAPHLDAAAA|119|Laurel |Wy||Woodland|Rooks County|KS||United States|-6|apartment| +15232|AAAAAAAAAILDAAAA|378|11th Cedar||||Williams County|OH|45804|United States|-5|| +15233|AAAAAAAABILDAAAA|733|Chestnut Main|RD|Suite 120|Providence|Gladwin County|MI|46614|United States|-5|apartment| +15234|AAAAAAAACILDAAAA|447|West |Ct.|Suite 150|Mount Pleasant|Gray County|TX|71933|United States|-6|condo| +15235|AAAAAAAADILDAAAA|282|Mill Railroad|Court|Suite Q|Hopewell|Burke County|NC|20587|United States|-5|condo| +15236|AAAAAAAAEILDAAAA|223|1st |Ave|Suite 470|Clifton|Alpine County|CA|98014|United States|-8|single family| +15237|AAAAAAAAFILDAAAA|544|First |Boulevard|Suite A|Shady Grove|Atkinson County|GA|32812|United States|-5|single family| +15238|AAAAAAAAGILDAAAA|8|4th |Lane|Suite D|Waterloo|Garden County|NE|61675|United States|-6|condo| +15239|AAAAAAAAHILDAAAA|277|Sixth |Pkwy|Suite 280|Forest Hills|Marion County|AL|39237|United States|-6|apartment| +15240|AAAAAAAAIILDAAAA|667|Fourteenth |Wy|Suite P|Unionville|Red River County|TX|71711|United States|-6|condo| +15241|AAAAAAAAJILDAAAA|258|Elm |Drive|Suite 80|Oakdale|Craig County|OK|79584|United States|-6|condo| +15242|AAAAAAAAKILDAAAA|30|Railroad Williams|Avenue|Suite 460|Shady Grove|Sioux County|ND|52812|United States|-6|apartment| +15243|AAAAAAAALILDAAAA|321|Elm Wilson|ST|Suite 210|Arlington|Denali Borough|AK|96557|United States|-9|condo| +15244|AAAAAAAAMILDAAAA|302|Tenth |Street|Suite 200|Fairfield|Tulare County|CA|96192|United States|-8|apartment| +15245|AAAAAAAANILDAAAA|421|Madison 11th|Avenue|Suite 230|Glenwood|Steuben County|IN|43511|United States|-5|condo| +15246|AAAAAAAAOILDAAAA|99|Highland |Way|Suite 460|Mountain View|McDowell County|NC|24466|United States|-5|single family| +15247|AAAAAAAAPILDAAAA|581|4th Green|Dr.|Suite 400|Midway|Mitchell County|TX|71904|United States|-6|apartment| +15248|AAAAAAAAAJLDAAAA|245|Jefferson 3rd|Boulevard|Suite 20|Shady Grove|Mills County|TX|72812|United States|-6|apartment| +15249|AAAAAAAABJLDAAAA|444|Oak Willow|Ct.|Suite 330|Lakeside|Grant County|OR|99532|United States|-8|apartment| +15250|AAAAAAAACJLDAAAA|852|7th Sixth|Blvd|Suite 220|Manchester|Clatsop County|OR|92884|United States|-8|apartment| +15251|AAAAAAAADJLDAAAA|669|Elm |Cir.|Suite 190|Oak Hill|Lincoln County|NV|87838|United States|-8|apartment| +15252|AAAAAAAAEJLDAAAA|387|10th Walnut|Ct.|Suite C|Union Hill|Virginia Beach city|VA|27746|United States|-5|condo| +15253|AAAAAAAAFJLDAAAA|20|3rd |Circle|Suite 260|Pleasant Grove|Texas County|OK|74136|United States|-6|apartment| +15254|AAAAAAAAGJLDAAAA|15|Park |Road|Suite H|Lakeview|Newport County|RI|09179|United States|-5|single family| +15255|AAAAAAAAHJLDAAAA|146|11th Park|Blvd|Suite U|Harmony|Otero County|CO|85804|United States|-7|condo| +15256|AAAAAAAAIJLDAAAA|113|Church Washington|Pkwy|Suite K|Jamestown|McCone County|MT|66867|United States|-7|apartment| +15257|AAAAAAAAJJLDAAAA|136|9th Sunset|Pkwy|Suite 90|Enterprise|Gaston County|NC|21757|United States|-5|apartment| +15258|AAAAAAAAKJLDAAAA|44|||||Crane County||74975||-6|condo| +15259|AAAAAAAALJLDAAAA|924|Lake |Boulevard|Suite 340|Sulphur Springs|Roscommon County|MI|48354|United States|-5|apartment| +15260|AAAAAAAAMJLDAAAA|355|4th Church|Wy|Suite 470|Salem|Wood County|OH|48048|United States|-5|single family| +15261|AAAAAAAANJLDAAAA|320|Spruce |Wy|Suite E|Fairview|Richardson County|NE|65709|United States|-7|condo| +15262|AAAAAAAAOJLDAAAA|731|Lake |Wy|Suite 200|Greenwood|Baker County|FL|38828|United States|-5|condo| +15263|AAAAAAAAPJLDAAAA|166|Center |Street|Suite O|Mount Zion|Windsor County|VT|08654|United States|-5|single family| +15264|AAAAAAAAAKLDAAAA|453|Cedar 8th|Ln|Suite 320|Salem|Chariton County|MO|68048|United States|-6|single family| +15265|AAAAAAAABKLDAAAA|537|Locust Park|Circle|Suite A|Bethel|McCone County|MT|65281|United States|-7|condo| +15266|AAAAAAAACKLDAAAA|51|Walnut |Circle|Suite N|Oakwood|Washita County|OK|70169|United States|-6|condo| +15267|AAAAAAAADKLDAAAA|54|11th Jackson|Cir.|Suite N|Shiloh|Hughes County|SD|59275|United States|-7|apartment| +15268|AAAAAAAAEKLDAAAA|806|Wilson Main|Circle|Suite J|Jackson|Mahaska County|IA|59583|United States|-6|single family| +15269|AAAAAAAAFKLDAAAA|717|View |Parkway|Suite Y|Newport|Clay County|MN|51521|United States|-6|single family| +15270|AAAAAAAAGKLDAAAA|301|Franklin Washington|Boulevard|Suite 450|Walnut Grove|Mercer County|KY|47752|United States|-5|condo| +15271|AAAAAAAAHKLDAAAA|547|Birch Church|Ln|Suite H|Summit|Searcy County|AR|70499|United States|-6|condo| +15272|AAAAAAAAIKLDAAAA|64|Sixth |Ave|Suite 340|Ellisville|Comanche County|OK|76820|United States|-6|apartment| +15273|AAAAAAAAJKLDAAAA|207|Tenth |Ct.|Suite 30|Highland Park|Lawrence County|MO|66534|United States|-6|apartment| +15274|AAAAAAAAKKLDAAAA|427|Hickory |Road|Suite 230|Valley View|Kenai Peninsula Borough|AK|95124|United States|-9|single family| +15275|AAAAAAAALKLDAAAA|35|5th 7th|Wy|Suite C|Brownsville|Haskell County|KS|69310|United States|-6|single family| +15276|AAAAAAAAMKLDAAAA|107|Johnson Mill|Circle|Suite F|Riverdale|Hidalgo County|TX|79391|United States|-6|apartment| +15277|AAAAAAAANKLDAAAA|188|Third |Pkwy|Suite 100|Elba|Lawrence County|PA|10262|United States|-5|single family| +15278|AAAAAAAAOKLDAAAA|289|Lake |Street|Suite U|Woodlawn|Lake County|IN|44098|United States|-5|apartment| +15279|AAAAAAAAPKLDAAAA|866|Tenth |Pkwy|Suite C|Riverside|Stanton County|NE|69231|United States|-7|single family| +15280|AAAAAAAAALLDAAAA|901|Spring Seventh|Street|Suite 60|Shiloh|Mercer County|WV|29275|United States|-5|apartment| +15281|AAAAAAAABLLDAAAA|66|Woodland Lee|Dr.|Suite T|Centerville|Franklin County|ID|80059|United States|-7|apartment| +15282|AAAAAAAACLLDAAAA|822|Madison Forest|Dr.|Suite 140|Saint Johns|Cuyahoga County|OH|45717|United States|-5|apartment| +15283|AAAAAAAADLLDAAAA|711|Mill |Way|Suite K|Parkwood|Poweshiek County|IA|51669|United States|-6|condo| +15284|AAAAAAAAELLDAAAA|622|Woodland Laurel|Boulevard|Suite F|Lakewood|Madison County|NC|28877|United States|-5|single family| +15285|AAAAAAAAFLLDAAAA|377|Highland |Blvd|Suite 0|Shiloh|Tipton County|TN|39275|United States|-6|single family| +15286|AAAAAAAAGLLDAAAA|501|Valley Willow|Circle|Suite G|Greenfield|Wood County|WV|25038|United States|-5|condo| +15287|AAAAAAAAHLLDAAAA|229|View Elm|Road|Suite H|Sawyer|Mono County|CA|96045|United States|-8|single family| +15288|AAAAAAAAILLDAAAA|480||||Highland Park|Berks County|||||| +15289|AAAAAAAAJLLDAAAA|596|Willow |Wy|Suite P|Midway|Bosque County|TX|71904|United States|-6|apartment| +15290|AAAAAAAAKLLDAAAA|443|West East|Ln|Suite 360|Lakewood|Delaware County|PA|18877|United States|-5|single family| +15291|AAAAAAAALLLDAAAA|293|Spring |RD|Suite V|Arlington|Colquitt County|GA|36557|United States|-5|condo| +15292|AAAAAAAAMLLDAAAA|482|Oak |Ct.|Suite R|Wilson|Geneva County|AL|36971|United States|-6|apartment| +15293|AAAAAAAANLLDAAAA|81|Lee |Street|Suite 400|White Oak|Collin County|TX|76668|United States|-6|single family| +15294|AAAAAAAAOLLDAAAA|799|Pine Central|Circle|Suite Q|Spring Hill|Richland County|ND|56787|United States|-6|condo| +15295|AAAAAAAAPLLDAAAA||||||Caldwell County||68059|United States||condo| +15296|AAAAAAAAAMLDAAAA|752|8th |Parkway|Suite 280|Crossroads|McPherson County|KS|60534|United States|-6|condo| +15297|AAAAAAAABMLDAAAA|123|View |RD|Suite R|Salem|Bear Lake County|ID|88048|United States|-7|condo| +15298|AAAAAAAACMLDAAAA||Davis Jefferson||Suite 60|Jackson|||39583|United States||| +15299|AAAAAAAADMLDAAAA|184|Maple Jefferson|Way|Suite 340|Harmony|Saunders County|NE|65804|United States|-7|apartment| +15300|AAAAAAAAEMLDAAAA|174|Green Jefferson|Ave|Suite 200|Edgewood|Pondera County|MT|60069|United States|-7|condo| +15301|AAAAAAAAFMLDAAAA|357|Maple Ash|Ln|Suite F|Owens|Washington County|TN|32324|United States|-6|condo| +15302|AAAAAAAAGMLDAAAA|595|Central Hickory|RD|Suite A|Wildwood|Golden Valley County|MT|66871|United States|-7|apartment| +15303|AAAAAAAAHMLDAAAA|449|Park |Court|Suite Q|Five Forks|Los Alamos County|NM|82293|United States|-7|apartment| +15304|AAAAAAAAIMLDAAAA|375|Main West|Court|Suite 490|Mountain View|Cass County|IL|64466|United States|-6|apartment| +15305|AAAAAAAAJMLDAAAA|414|Railroad Walnut|Dr.|Suite E|Crossroads|Howard County|NE|60534|United States|-7|condo| +15306|AAAAAAAAKMLDAAAA|69|Johnson |ST|Suite V|Wildwood|Natchitoches Parish|LA|76871|United States|-6|single family| +15307|AAAAAAAALMLDAAAA|442|Green Elm|Court|Suite 220|Glenwood|Calvert County|MD|23511|United States|-5|condo| +15308|AAAAAAAAMMLDAAAA|752|Poplar |ST|Suite T|Mount Olive|Greene County|NY|18059|United States|-5|condo| +15309|AAAAAAAANMLDAAAA|930|Jackson |Avenue|Suite 470|Lakeside|Huron County|MI|49532|United States|-5|apartment| +15310|AAAAAAAAOMLDAAAA|456|Third Pine|Avenue|Suite 180|Salem|Edgar County|IL|68048|United States|-6|condo| +15311|AAAAAAAAPMLDAAAA|37|Center |Ct.|Suite M|Oak Ridge|Pemiscot County|MO|68371|United States|-6|apartment| +15312|AAAAAAAAANLDAAAA|110|Williams |Avenue|Suite B|Belmont|Leon County|FL|30191|United States|-5|condo| +15313|AAAAAAAABNLDAAAA|279|Lake |Circle|Suite 310|Macedonia|Greene County|PA|11087|United States|-5|condo| +15314|AAAAAAAACNLDAAAA|235|Lincoln Oak|Circle|Suite G|Providence|Summit County|UT|86614|United States|-7|condo| +15315|AAAAAAAADNLDAAAA|564|5th North|Street|Suite K|Belmont|Greeley County|NE|60191|United States|-6|condo| +15316|AAAAAAAAENLDAAAA|704|Cedar Pine|Ln|Suite 460|Antioch|Adair County|KY|48605|United States|-6|condo| +15317|AAAAAAAAFNLDAAAA|619|South |Blvd|Suite Q|Deerfield|Emanuel County|GA|39840|United States|-5|condo| +15318|AAAAAAAAGNLDAAAA|189|Elm |Lane|Suite Y|Farmington|De Kalb County|IN|49145|United States|-5|condo| +15319|AAAAAAAAHNLDAAAA|567|Highland |Avenue|Suite P|Georgetown|Madison County|TN|37057|United States|-6|single family| +15320|AAAAAAAAINLDAAAA|585|Mill |Avenue|Suite 380|Oakwood|Worcester County|MA|00769|United States|-5|single family| +15321|AAAAAAAAJNLDAAAA|357|6th Ridge|Pkwy|Suite B|Plainview|Lyon County|KS|63683|United States|-6|single family| +15322|AAAAAAAAKNLDAAAA|402|Smith |Circle|Suite 130|Bethel|Caddo Parish|LA|75281|United States|-6|single family| +15323|AAAAAAAALNLDAAAA|537|15th |Parkway|Suite 40|Edgewood|Thomas County|NE|60069|United States|-7|single family| +15324|AAAAAAAAMNLDAAAA|237|Spruce Forest|Road|Suite Q|Post Oak|Lander County|NV|88567|United States|-8|condo| +15325|AAAAAAAANNLDAAAA|344|First Center|Pkwy|Suite H|Marion|Ray County|MO|60399|United States|-6|single family| +15326|AAAAAAAAONLDAAAA|100|Third |Pkwy|Suite 170|Wesley|Howard County|AR|71218|United States|-6|single family| +15327|AAAAAAAAPNLDAAAA|42|Fourth |Parkway|Suite K|New Hope|Hansford County|TX|79431|United States|-6|single family| +15328|AAAAAAAAAOLDAAAA|||||Midway|Somerset County|NJ||United States||apartment| +15329|AAAAAAAABOLDAAAA|747|Sunset |Road|Suite 260|Liberty|Upshur County|WV|23451|United States|-5|condo| +15330|AAAAAAAACOLDAAAA|172|Seventh |Wy|Suite 280|Oakdale|Randolph County|WV|29584|United States|-5|apartment| +15331|AAAAAAAADOLDAAAA|852|Willow |Cir.|Suite W|Bridgeport|Monroe County|IL|65817|United States|-6|condo| +15332|AAAAAAAAEOLDAAAA|408|Ninth |Court|Suite S|Bridgeport|Santa Barbara County|CA|95817|United States|-8|single family| +15333|AAAAAAAAFOLDAAAA||Forest |Ave|Suite 10|Oakwood|Columbia County|WA|90169|United States||| +15334|AAAAAAAAGOLDAAAA|222|Second |Lane|Suite 50|Leland|Putnam County|GA|39452|United States|-5|condo| +15335|AAAAAAAAHOLDAAAA|158|Forest Mill|Wy|Suite A|Riverview|Jefferson County|AL|39003|United States|-6|single family| +15336|AAAAAAAAIOLDAAAA|343|Birch |Drive|Suite P|Greenfield|Sandusky County|OH|45038|United States|-5|single family| +15337|AAAAAAAAJOLDAAAA|475|Laurel |Court|Suite 10|Oak Ridge|White County|TN|38371|United States|-6|single family| +15338|AAAAAAAAKOLDAAAA|885|Willow |Drive|Suite X|Springfield|Westmoreland County|VA|29303|United States|-5|single family| +15339|AAAAAAAALOLDAAAA|434|Franklin |Court|Suite U|Bethel|Webster County|MS|55281|United States|-6|condo| +15340|AAAAAAAAMOLDAAAA|435|Williams |Ln|Suite 0|New Hope|Bailey County|TX|79431|United States|-6|condo| +15341|AAAAAAAANOLDAAAA||River Fourteenth|RD|Suite 270|Waterloo|Montgomery County||||-5|| +15342|AAAAAAAAOOLDAAAA|745|4th Lincoln|Ct.|Suite B|Riverside|Crawford County|AR|79231|United States|-6|condo| +15343|AAAAAAAAPOLDAAAA|826|Spruce |Lane|Suite B|Post Oak|Stephenson County|IL|68567|United States|-6|apartment| +15344|AAAAAAAAAPLDAAAA|325|Willow 2nd|Circle|Suite Q|Antioch|Newton County|TX|78605|United States|-6|apartment| +15345|AAAAAAAABPLDAAAA|696|Hill |Wy|Suite 390|Kimball|Coke County|TX|73595|United States|-6|single family| +15346|AAAAAAAACPLDAAAA|425|Center |Circle|Suite 190|Jenkins|Wayne County|MI|47292|United States|-5|single family| +15347|AAAAAAAADPLDAAAA|816|Spruce Hill|Road|Suite 190|Antioch|Arlington County|VA|28605|United States|-5|apartment| +15348|AAAAAAAAEPLDAAAA|753|8th |Road|Suite T|Wilson|Dougherty County|GA|36971|United States|-5|condo| +15349|AAAAAAAAFPLDAAAA|276|Pine |Circle|Suite 90|Lakeview|Atchison County|KS|68579|United States|-6|condo| +15350|AAAAAAAAGPLDAAAA|865|Willow First|Dr.|Suite A|Arthur|Bristol County|RI|06565|United States|-5|single family| +15351|AAAAAAAAHPLDAAAA|463|Elm |Ave|Suite E|Midway|Fillmore County|MN|51904|United States|-6|apartment| +15352|AAAAAAAAIPLDAAAA|749|Lincoln |Pkwy|Suite 70|Warwick|Pocahontas County|IA|51398|United States|-6|apartment| +15353|AAAAAAAAJPLDAAAA|151|Wilson |Road|Suite 250|Fowler|Lyon County|MN|51083|United States|-6|apartment| +15354|AAAAAAAAKPLDAAAA|695|Walnut |Parkway|Suite 190|Pleasant Valley|Stone County|MO|62477|United States|-6|single family| +15355|AAAAAAAALPLDAAAA|473|Mill |ST|Suite K|Five Forks|Limestone County|TX|72293|United States|-6|apartment| +15356|AAAAAAAAMPLDAAAA|923|2nd Washington|Avenue|Suite 90|Pine Grove|McPherson County|SD|54593|United States|-7|single family| +15357|AAAAAAAANPLDAAAA|597|Lincoln |Circle|Suite K|Wilson|Beaver County|OK|76971|United States|-6|apartment| +15358|AAAAAAAAOPLDAAAA|171|13th |Road|Suite R|Riverview|Heard County|GA|39003|United States|-5|condo| +15359|AAAAAAAAPPLDAAAA|185|Mill |Way|Suite G|Pleasant Valley|Lincoln County|ID|82477|United States|-7|apartment| +15360|AAAAAAAAAAMDAAAA|9|Wilson 6th|Street|Suite 280||Mississippi County||||-6|condo| +15361|AAAAAAAABAMDAAAA|831|4th South|Dr.|Suite 90|Edgewood|Fayette County|PA|10069|United States|-5|single family| +15362|AAAAAAAACAMDAAAA|94|Hillcrest Sunset|Avenue|Suite P|Friendship|Rankin County|MS|54536|United States|-6|apartment| +15363|AAAAAAAADAMDAAAA|715|Spring |Way|Suite 320|Macedonia|Vermilion Parish|LA|71087|United States|-6|single family| +15364|AAAAAAAAEAMDAAAA|648|Fifth Valley|ST|Suite E|Brookwood|Pickens County|SC|20965|United States|-5|apartment| +15365|AAAAAAAAFAMDAAAA|253|Wilson Lincoln|Way|Suite 360|Valley View|Watauga County|NC|25124|United States|-5|condo| +15366|AAAAAAAAGAMDAAAA|435|Washington Oak|Lane|Suite P|Clifton|Audrain County|MO|68014|United States|-6|condo| +15367|AAAAAAAAHAMDAAAA|340|Smith Maple|Ct.|Suite V|Spring Valley|Escambia County|FL|36060|United States|-5|condo| +15368|AAAAAAAAIAMDAAAA|602|4th |Pkwy|Suite 100|Greenwood|Calhoun County|AR|78828|United States|-6|single family| +15369|AAAAAAAAJAMDAAAA|103|Franklin Park|Pkwy|Suite H|Wilson|Maui County|HI|96971|United States|-10|condo| +15370|AAAAAAAAKAMDAAAA|884|Davis |Ave|Suite R|Liberty|Ashland County|OH|43451|United States|-5|condo| +15371|AAAAAAAALAMDAAAA|761|Railroad Birch|Drive|Suite 200|Antioch|Beaverhead County|MT|68605|United States|-7|condo| +15372|AAAAAAAAMAMDAAAA|331|Pine |Circle|Suite O|Glendale|Chippewa County|MI|43951|United States|-5|single family| +15373|AAAAAAAANAMDAAAA|450|11th Sycamore|Cir.|Suite U|Pleasant Valley|Wood County|TX|72477|United States|-6|single family| +15374|AAAAAAAAOAMDAAAA|761|5th |Circle|Suite 410|Fairfield|Summit County|CO|86192|United States|-7|condo| +15375|AAAAAAAAPAMDAAAA|901|Broadway Jackson|Parkway|Suite X|Wildwood|Clallam County|WA|96871|United States|-8|condo| +15376|AAAAAAAAABMDAAAA|179|Main West|Blvd|Suite D|Five Forks|Lyon County|KY|42293|United States|-5|single family| +15377|AAAAAAAABBMDAAAA|701|Elm |RD|Suite O|Richville|Pierce County|WA|95945|United States|-8|single family| +15378|AAAAAAAACBMDAAAA|106|Elm |Ave|Suite 0|Kingston|Loup County|NE|64975|United States|-7|apartment| +15379|AAAAAAAADBMDAAAA|225|Green Lincoln|Dr.|Suite Q|Crossroads|Adair County|OK|70534|United States|-6|condo| +15380|AAAAAAAAEBMDAAAA|320|Mill Meadow|Boulevard|Suite Y|Lebanon|Garden County|NE|62898|United States|-6|apartment| +15381|AAAAAAAAFBMDAAAA|340|Locust Lincoln|Wy|Suite 450|Summit|Haines Borough|AK|90499|United States|-9|single family| +15382|AAAAAAAAGBMDAAAA|16|Center Ridge|Street|Suite 480|Greenwood|Calhoun County|MS|58828|United States|-6|apartment| +15383|AAAAAAAAHBMDAAAA|795|Wilson Laurel|Ln|Suite B|Riverdale|Hamilton County|IA|59391|United States|-6|single family| +15384|AAAAAAAAIBMDAAAA|964|Maple |Road|Suite H|Greenwood|McHenry County|ND|58828|United States|-6|apartment| +15385|AAAAAAAAJBMDAAAA|887|9th |Lane|Suite 230|Lakewood|Albany County|NY|18877|United States|-5|apartment| +15386|AAAAAAAAKBMDAAAA|365|East 3rd|Wy|Suite O|Springdale|York County|PA|18883|United States|-5|apartment| +15387|AAAAAAAALBMDAAAA|581|2nd |Way|Suite W|Harmon|Grayson County|KY|45623|United States|-6|single family| +15388|AAAAAAAAMBMDAAAA|925|Cherry |Road|Suite 50|Buena Vista|Quay County|NM|85752|United States|-7|apartment| +15389|AAAAAAAANBMDAAAA|439|Pine Highland|Ct.|Suite M|Red Hill|Staunton city|VA|24338|United States|-5|condo| +15390|AAAAAAAAOBMDAAAA|720|5th Davis|Avenue|Suite 430|Valley View|Essex County|NJ|05724|United States|-5|single family| +15391|AAAAAAAAPBMDAAAA|590|Wilson |Drive|Suite I|Centerville|San Bernardino County|CA|90059|United States|-8|apartment| +15392|AAAAAAAAACMDAAAA|651|1st Center|Ln|Suite 370|Fairview|Greene County|AL|35709|United States|-6|condo| +15393|AAAAAAAABCMDAAAA|641|Highland |Way|Suite A|Lakewood|Park County|WY|88877|United States|-7|condo| +15394|AAAAAAAACCMDAAAA|799|Center Sycamore|Road|Suite I|Highland Park|Bristol Bay Borough|AK|96534|United States|-9|condo| +15395|AAAAAAAADCMDAAAA|415|Locust |Boulevard|Suite 80|Georgetown|Garden County|NE|67057|United States|-6|condo| +15396|AAAAAAAAECMDAAAA|586|Meadow Ridge|Boulevard|Suite 290|Pleasant Valley|Livingston County|NY|12477|United States|-5|single family| +15397|AAAAAAAAFCMDAAAA|546|Oak |Court|Suite H|Salem|Wyoming County|WV|28048|United States|-5|apartment| +15398|AAAAAAAAGCMDAAAA|938|Pine |Road|Suite T|Harmony|Lapeer County|MI|45804|United States|-5|apartment| +15399|AAAAAAAAHCMDAAAA|179|Fourteenth Woodland|Parkway|Suite 130|Oakdale|Roger Mills County|OK|79584|United States|-6|apartment| +15400|AAAAAAAAICMDAAAA|871|Lincoln |Blvd|Suite B|Union|Prairie County|AR|78721|United States|-6|condo| +15401|AAAAAAAAJCMDAAAA|169|Meadow |Circle|Suite S|Shiloh|Marshall County|IL|69275|United States|-6|condo| +15402|AAAAAAAAKCMDAAAA|854|15th |Court|Suite 120|Pleasant Grove|Pike County|IL|64136|United States|-6|condo| +15403|AAAAAAAALCMDAAAA|633|Pine Fourteenth|Ln|Suite 330|Sulphur Springs|Long County|GA|38354|United States|-5|single family| +15404|AAAAAAAAMCMDAAAA|381|Eigth Green|Court|Suite E|Lakeview|Menominee County|MI|48579|United States|-5|condo| +15405|AAAAAAAANCMDAAAA|860|8th |Street|Suite Q|Pleasant Grove|Madison County|MS|54136|United States|-6|condo| +15406|AAAAAAAAOCMDAAAA|466|Maple Lakeview|Court|Suite W|Stringtown|McCone County|MT|60162|United States|-7|single family| +15407|AAAAAAAAPCMDAAAA|358|Miller 6th|Boulevard|Suite N|Mountain View|Dolores County|CO|84466|United States|-7|single family| +15408|AAAAAAAAADMDAAAA|62|Lake |Street|Suite O|Oakdale|Barry County|MO|69584|United States|-6|condo| +15409|AAAAAAAABDMDAAAA|360|Fourth 4th|Road|Suite 20|Farmington|Dickinson County|MI|49145|United States|-5|condo| +15410|AAAAAAAACDMDAAAA|510|First Maple|Dr.|Suite 10|Sunnyside|Independence County|AR|71952|United States|-6|single family| +15411|AAAAAAAADDMDAAAA|24|Meadow |Avenue|Suite 280|Richville|Sumner County|TN|35945|United States|-6|apartment| +15412|AAAAAAAAEDMDAAAA|9|14th |Pkwy|Suite G|Highland|Garfield County|MT|69454|United States|-7|single family| +15413|AAAAAAAAFDMDAAAA||||Suite E|Fairview|Washington County||15709|United States||condo| +15414|AAAAAAAAGDMDAAAA|447|Eigth Hill|Parkway|Suite 10|Arlington|Montgomery County|KY|46557|United States|-5|condo| +15415|AAAAAAAAHDMDAAAA|911|5th |Drive|Suite O|New Hope|Hill County|MT|69431|United States|-7|condo| +15416|AAAAAAAAIDMDAAAA|281|Smith Fourth|Wy|Suite L|Arlington|Washington County|OK|76557|United States|-6|single family| +15417|AAAAAAAAJDMDAAAA|23|Davis |Road|Suite F|Highland Park|Hot Springs County|WY|86534|United States|-7|apartment| +15418|AAAAAAAAKDMDAAAA|495|Highland Second|ST|Suite E|Greenfield|Hemphill County|TX|75038|United States|-6|apartment| +15419|AAAAAAAALDMDAAAA|94|Ridge Cedar|Ave|Suite 310|Kingston|Stanton County|KS|64975|United States|-6|condo| +15420|AAAAAAAAMDMDAAAA|672|Forest Walnut|Pkwy|Suite 210|Buena Vista|Henry County|IN|45752|United States|-5|single family| +15421|AAAAAAAANDMDAAAA|||Court|Suite 320|Riverview|Le Flore County|OK|79003||-6|single family| +15422|AAAAAAAAODMDAAAA|70|Highland Mill|Cir.|Suite 120|Kingston|Houston County|MN|54975|United States|-6|apartment| +15423|AAAAAAAAPDMDAAAA|707|View |Blvd|Suite W|Greenwood|Ballard County|KY|48828|United States|-6|single family| +15424|AAAAAAAAAEMDAAAA|478|Sunset |Lane|Suite 270|Marion|Cullman County|AL|30399|United States|-6|apartment| +15425|AAAAAAAABEMDAAAA|307|Ninth Elm|Ct.|Suite R|Spring Valley|Caroline County|MD|26060|United States|-5|apartment| +15426|AAAAAAAACEMDAAAA|448|9th 7th|Ln|Suite A|Unionville|Saginaw County|MI|41711|United States|-5|condo| +15427|AAAAAAAADEMDAAAA|709|4th |Road|Suite 450|Cedar Grove|Osage County|MO|60411|United States|-6|single family| +15428|AAAAAAAAEEMDAAAA|999|Adams 8th|Street|Suite Y|Flatwoods|Grand Traverse County|MI|44212|United States|-5|condo| +15429|AAAAAAAAFEMDAAAA|528|2nd |Wy|Suite 240|Hazelwood|Gilpin County|CO|81206|United States|-7|apartment| +15430|AAAAAAAAGEMDAAAA|187|Elm |Pkwy|Suite 320|Sullivan|Marion County|IA|50451|United States|-6|apartment| +15431|AAAAAAAAHEMDAAAA|236|Lakeview Railroad|Blvd|Suite Q|Hubbard|Highlands County|FL|36291|United States|-5|condo| +15432|AAAAAAAAIEMDAAAA|788|Elm Main|Circle|Suite 460|Pleasant Valley|Gosper County|NE|62477|United States|-6|single family| +15433|AAAAAAAAJEMDAAAA|75|Fourth Highland|Boulevard|Suite 350|Greenfield|Taylor County|WI|55038|United States|-6|condo| +15434|AAAAAAAAKEMDAAAA|988|2nd Meadow|RD|Suite 90|Antioch|Todd County|KY|48605|United States|-5|condo| +15435|AAAAAAAALEMDAAAA|973|Second Cedar|RD|Suite 200|Pine Grove|Lawrence County|MO|64593|United States|-6|single family| +15436|AAAAAAAAMEMDAAAA|586|4th |Ct.|Suite 30|Enterprise|Washington County|OH|41757|United States|-5|condo| +15437|AAAAAAAANEMDAAAA|244|Cherry View|Pkwy|Suite R|New Hope|Perquimans County|NC|29431|United States|-5|condo| +15438|AAAAAAAAOEMDAAAA|510|Lee 6th|Court|Suite 280|Plainview|Calhoun County|AR|73683|United States|-6|apartment| +15439|AAAAAAAAPEMDAAAA|547|East |Dr.|Suite H|Shady Grove|Dauphin County|PA|12812|United States|-5|apartment| +15440|AAAAAAAAAFMDAAAA|134|Spruce |Way|Suite 40|Ellsworth|Ector County|TX|75079|United States|-6|apartment| +15441|AAAAAAAABFMDAAAA|810|Chestnut Pine|Avenue|Suite Q|Greenwood|Edgecombe County|NC|28828|United States|-5|apartment| +15442|AAAAAAAACFMDAAAA|950|Walnut Laurel|Ct.|Suite 490|Saint Johns|Muhlenberg County|KY|45717|United States|-5|single family| +15443|AAAAAAAADFMDAAAA|823|Chestnut Valley|Ave|Suite U|Unionville|Scott County|MN|51711|United States|-6|single family| +15444|AAAAAAAAEFMDAAAA|818|Miller Miller|Ct.|Suite E|Sunnyside|San Luis Obispo County|CA|91952|United States|-8|single family| +15445|AAAAAAAAFFMDAAAA|514|Elm Park|Wy|Suite F|Arlington|Morgan County|GA|36557|United States|-5|condo| +15446|AAAAAAAAGFMDAAAA|267|Park 5th|Court|Suite Q|Mount Zion|Jackson Parish|LA|78054|United States|-6|single family| +15447|AAAAAAAAHFMDAAAA|821|Washington |Cir.|Suite 370|Greenville|Wright County|MN|51387|United States|-6|apartment| +15448|AAAAAAAAIFMDAAAA|310|2nd Third|Lane|Suite 150|Concord|Sanilac County|MI|44107|United States|-5|single family| +15449|AAAAAAAAJFMDAAAA|708|Park |Street|Suite 190|Union|Schuyler County|MO|68721|United States|-6|condo| +15450|AAAAAAAAKFMDAAAA|748|Main 7th|Street|Suite 270|Oakwood|Whiteside County|IL|60169|United States|-6|condo| +15451|AAAAAAAALFMDAAAA|29|10th Cherry|Ave|Suite F|Shiloh|Swift County|MN|59275|United States|-6|single family| +15452|AAAAAAAAMFMDAAAA|224|Davis |Blvd|Suite Q|Riverside|Lapeer County|MI|49231|United States|-5|condo| +15453|AAAAAAAANFMDAAAA|603|View |Pkwy|Suite 490|Milo|Morrill County|NE|60116|United States|-7|condo| +15454|AAAAAAAAOFMDAAAA|81|Pine Birch|RD|Suite 70|Lakeview|Peach County|GA|38579|United States|-5|condo| +15455|AAAAAAAAPFMDAAAA|678|East |Pkwy|Suite P|White Oak|Phillips County|CO|86668|United States|-7|single family| +15456|AAAAAAAAAGMDAAAA|975|Sunset |ST|Suite M|Pine Grove|Roanoke city|VA|24593|United States|-5|single family| +15457|AAAAAAAABGMDAAAA|529|Park Chestnut|Avenue|Suite W|Wildwood|Wayne County|IA|56871|United States|-6|apartment| +15458|AAAAAAAACGMDAAAA|568|11th |Blvd|Suite W|Green Acres|Cumberland County|TN|37683|United States|-5|apartment| +15459|AAAAAAAADGMDAAAA|628|Cherry |ST|Suite T|Stringtown|Hickory County|MO|60162|United States|-6|single family| +15460|AAAAAAAAEGMDAAAA|407|Oak 4th|Ln|Suite R|Waterloo|Jefferson Davis Parish|LA|71675|United States|-6|apartment| +15461|AAAAAAAAFGMDAAAA|628|Smith Third|Lane|Suite C|Union Hill|Storey County|NV|87746|United States|-8|single family| +15462|AAAAAAAAGGMDAAAA|10|5th Forest|Boulevard|Suite P|Plainview|Glasscock County|TX|73683|United States|-6|condo| +15463|AAAAAAAAHGMDAAAA|855||Avenue|Suite O|Lebanon|Hopewell city|VA|22898|United States|-5|| +15464|AAAAAAAAIGMDAAAA||Pine 1st|||||MI|||-5|| +15465|AAAAAAAAJGMDAAAA|404|Elm Main|Avenue|Suite H|Brownsville|Vilas County|WI|59310|United States|-6|condo| +15466|AAAAAAAAKGMDAAAA|861|Church Hill|Lane|Suite 400|Pleasant Grove|Anoka County|MN|54136|United States|-6|single family| +15467|AAAAAAAALGMDAAAA|843|Hill Railroad|Lane|Suite F|Macedonia|Houston County|TX|71087|United States|-6|apartment| +15468|AAAAAAAAMGMDAAAA|74|Central |Boulevard|Suite 100|Fairview|Washington County|OK|75709|United States|-6|single family| +15469|AAAAAAAANGMDAAAA|88||Dr.|Suite X|||MI||United States|-5|| +15470|AAAAAAAAOGMDAAAA|768|Broadway |Drive|Suite C|Bunker Hill|Washington County|IL|60150|United States|-6|apartment| +15471|AAAAAAAAPGMDAAAA|347|Ash |Wy|Suite C|Hardy|Taylor County|WV|25354|United States|-5|single family| +15472|AAAAAAAAAHMDAAAA|204|Adams 10th|Ln|Suite 280|Springdale|Ramsey County|ND|58883|United States|-6|apartment| +15473|AAAAAAAABHMDAAAA|553|Railroad 5th|Dr.|Suite D|Sunnyside|Isabella County|MI|41952|United States|-5|single family| +15474|AAAAAAAACHMDAAAA|197|Central 9th|Ct.||Buena Vista||TX||United States||| +15475|AAAAAAAADHMDAAAA|533|8th |Ct.|Suite 360|Riverview|Waushara County|WI|59003|United States|-6|apartment| +15476|AAAAAAAAEHMDAAAA|954|4th Birch|Ave|Suite 190|Gravel Hill|Chautauqua County|KS|61944|United States|-6|condo| +15477|AAAAAAAAFHMDAAAA|180|15th Church|Dr.|Suite H|Forest Hills|Seward County|KS|69237|United States|-6|apartment| +15478|AAAAAAAAGHMDAAAA|425|8th |Ct.|Suite P|White Oak|Gates County|NC|26668|United States|-5|single family| +15479|AAAAAAAAHHMDAAAA|389|First |Ct.|Suite 320|Mountain View|Gladwin County|MI|44466|United States|-5|apartment| +15480|AAAAAAAAIHMDAAAA|30|Maple Jefferson|Street|Suite 190|Bunker Hill|Huron County|OH|40150|United States|-5|single family| +15481|AAAAAAAAJHMDAAAA|342|10th Church|Circle|Suite W|Marion|Lee County|AL|30399|United States|-6|apartment| +15482|AAAAAAAAKHMDAAAA|709|Williams 1st|ST|Suite 0|Buena Vista|Yakima County|WA|95752|United States|-8|apartment| +15483|AAAAAAAALHMDAAAA|456|14th |Blvd|Suite I|Riverdale|Johnson County|GA|39391|United States|-5|single family| +15484|AAAAAAAAMHMDAAAA|842|Pine 1st|Cir.|Suite W|Pleasant Grove|Luna County|NM|84136|United States|-7|condo| +15485|AAAAAAAANHMDAAAA|789|First |Road|Suite K|Mount Pleasant|Lee County|KY|41933|United States|-5|condo| +15486|AAAAAAAAOHMDAAAA|818|Maple Jefferson|Circle|Suite W|Pleasant Hill|Mitchell County|IA|53604|United States|-6|condo| +15487|AAAAAAAAPHMDAAAA|438|Elm |Ln|Suite 30|Fairview|Clay County|AL|35709|United States|-6|apartment| +15488|AAAAAAAAAIMDAAAA|731|North |Pkwy|Suite 260|Wildwood|Sumner County|KS|66871|United States|-6|condo| +15489|AAAAAAAABIMDAAAA|79|Elm |Cir.|Suite 300|Woodlawn|Traverse County|MN|54098|United States|-6|condo| +15490|AAAAAAAACIMDAAAA|186|4th |Dr.|Suite C|Bethesda|Cuming County|NE|65980|United States|-6|apartment| +15491|AAAAAAAADIMDAAAA|824|2nd Washington|Ave|Suite J|Macedonia|Amelia County|VA|21087|United States|-5|condo| +15492|AAAAAAAAEIMDAAAA|856|Lake Poplar|Boulevard|Suite R|Crossroads|Nodaway County|MO|60534|United States|-6|single family| +15493|AAAAAAAAFIMDAAAA|207|Eigth |Drive|Suite 300|Pleasant Valley|Webster County|KY|42477|United States|-5|condo| +15494|AAAAAAAAGIMDAAAA|305|Ash |Road|Suite M|Fairfield|Marion County|TN|36192|United States|-6|single family| +15495|AAAAAAAAHIMDAAAA|113|Smith Main|Ave|Suite Y|Oak Grove|Jackson County|MN|58370|United States|-6|single family| +15496|AAAAAAAAIIMDAAAA|113|Central Hill|Drive|Suite 430|Greenfield|Smith County|TX|75038|United States|-6|single family| +15497|AAAAAAAAJIMDAAAA|522|1st Pine|Ct.|Suite J|Buena Vista|Catahoula Parish|LA|75752|United States|-6|apartment| +15498|AAAAAAAAKIMDAAAA|397|Fourth East|Ct.|Suite K|Mount Pleasant|Kanabec County|MN|51933|United States|-6|single family| +15499|AAAAAAAALIMDAAAA|57|Broadway |Cir.|Suite P|Newport|Isle of Wight County|VA|21521|United States|-5|apartment| +15500|AAAAAAAAMIMDAAAA|535|Mill |RD|Suite G|Oakland|Noble County|OH|49843|United States|-5|single family| +15501|AAAAAAAANIMDAAAA|828|Oak Jefferson|Parkway|Suite 250|Wildwood|Harford County|MD|26871|United States|-5|apartment| +15502|AAAAAAAAOIMDAAAA|748|South |Way|Suite 230|Summit|Jefferson County|AL|30499|United States|-6|single family| +15503|AAAAAAAAPIMDAAAA|65|Spruce 7th|Cir.|Suite 40|Highland|Texas County|OK|79454|United States|-6|condo| +15504|AAAAAAAAAJMDAAAA|573|View |Blvd|Suite 110|Brownsville|Sheridan County|MT|69310|United States|-7|condo| +15505|AAAAAAAABJMDAAAA|728|Smith |Avenue|Suite Q|White Oak|Roberts County|TX|76668|United States|-6|condo| +15506|AAAAAAAACJMDAAAA|221|10th 1st|Blvd|Suite 380|Crossroads|Greene County|VA|20534|United States|-5|single family| +15507|AAAAAAAADJMDAAAA|421|4th Laurel|Ct.|Suite 370|Lakeside|Dickinson County|IA|59532|United States|-6|apartment| +15508|AAAAAAAAEJMDAAAA|703|Maple |Blvd|Suite 180|Wilson|Tazewell County|VA|26971|United States|-5|condo| +15509|AAAAAAAAFJMDAAAA|174|Park View|Ct.|Suite L|Shady Grove|Jeff Davis County|GA|32812|United States|-5|single family| +15510|AAAAAAAAGJMDAAAA|11|Park 3rd|Parkway|Suite D|Sulphur Springs|Juneau County|WI|58354|United States|-6|single family| +15511|AAAAAAAAHJMDAAAA|741|Washington |Street|Suite O|Union Hill|Nantucket County|MA|08346|United States|-5|single family| +15512|AAAAAAAAIJMDAAAA|805|Hickory 4th|RD|Suite Q|Riverview|Montgomery County|TX|79003|United States|-6|condo| +15513|AAAAAAAAJJMDAAAA|316|First |Dr.|Suite H|Highland|Gallatin County|KY|49454|United States|-6|single family| +15514|AAAAAAAAKJMDAAAA|846|3rd Walnut|Ct.|Suite 180|Friendship|Covington city|VA|24536|United States|-5|apartment| +15515|AAAAAAAALJMDAAAA||Broadway Hill|Street|||Rhea County|||United States|-6|| +15516|AAAAAAAAMJMDAAAA|256|North View|Wy|Suite A|Greenville|Jackson County|TN|31387|United States|-5|single family| +15517|AAAAAAAANJMDAAAA|232|College Hillcrest|Dr.|Suite 60|Rocky Point|Jasper County|IN|41209|United States|-5|single family| +15518|AAAAAAAAOJMDAAAA|715|Smith View|Circle|Suite G|Kimball|Emmet County|IA|53595|United States|-6|single family| +15519|AAAAAAAAPJMDAAAA|805|Johnson Second|Drive|Suite 170|Oak Hill|Logan County|NE|67838|United States|-7|condo| +15520|AAAAAAAAAKMDAAAA|65|Sycamore |Drive|Suite A|Marion|Erie County|PA|10399|United States|-5|apartment| +15521|AAAAAAAABKMDAAAA|327|Smith |Ln|Suite 330|Enterprise|Webster County|KY|41757|United States|-5|single family| +15522|AAAAAAAACKMDAAAA|||Road|Suite 210|||||United States||| +15523|AAAAAAAADKMDAAAA|998|5th |Way|Suite 240|Ashley|Moody County|SD|54324|United States|-7|apartment| +15524|AAAAAAAAEKMDAAAA|202|Oak |Avenue|Suite 60|Pleasant Hill|Bryan County|OK|73604|United States|-6|single family| +15525|AAAAAAAAFKMDAAAA|868|Second |Avenue|Suite H|Bennett|Onslow County|NC|21715|United States|-5|single family| +15526|AAAAAAAAGKMDAAAA|84|Ninth Railroad|Dr.|Suite D|Belmont|Tillman County|OK|70191|United States|-6|apartment| +15527|AAAAAAAAHKMDAAAA|147|4th Church|Way|Suite J|Georgetown|Harrison County|IN|47057|United States|-5|condo| +15528|AAAAAAAAIKMDAAAA|717|Jefferson Mill|Avenue|Suite 280|Hillcrest|Hot Spring County|AR|73003|United States|-6|apartment| +15529|AAAAAAAAJKMDAAAA|40|Highland |Road|Suite K|Calhoun|Hillsborough County|FL|36909|United States|-5|condo| +15530|AAAAAAAAKKMDAAAA|875|10th Lake|ST|Suite 40|Lakeview|Washington County|AR|78579|United States|-6|single family| +15531|AAAAAAAALKMDAAAA|160|Main |Lane|Suite W|Sugar Hill|San Augustine County|TX|75114|United States|-6|apartment| +15532|AAAAAAAAMKMDAAAA|805|East Chestnut|ST|Suite 160|Maple Grove|Halifax County|NC|28252|United States|-5|apartment| +15533|AAAAAAAANKMDAAAA|225||||||KY|42293||-6|| +15534|AAAAAAAAOKMDAAAA|518|Lincoln Green|Way|Suite 150|Mount Vernon|Buckingham County|VA|28482|United States|-5|apartment| +15535|AAAAAAAAPKMDAAAA|836|Hickory |Court|Suite 450|Riverdale|Union County|SC|29391|United States|-5|apartment| +15536|AAAAAAAAALMDAAAA|302|3rd |Pkwy|Suite 330|Riverside|Weber County|UT|89231|United States|-7|single family| +15537|AAAAAAAABLMDAAAA|448|Seventh Washington|Parkway|Suite 490|Fairfield|Summit County|OH|46192|United States|-5|apartment| +15538|AAAAAAAACLMDAAAA|129|Laurel Washington|Court|Suite 220|Pleasant Grove|Gaston County|NC|24136|United States|-5|single family| +15539|AAAAAAAADLMDAAAA|120|Smith Broadway|Wy|Suite 70|Midway|Hardin County|KY|41904|United States|-6|condo| +15540|AAAAAAAAELMDAAAA|513|Maple 11th|ST|Suite 430|Shady Grove|Clackamas County|OR|92812|United States|-8|single family| +15541|AAAAAAAAFLMDAAAA|122|Birch |ST|Suite Y|Riverview|Wayne County|MS|59003|United States|-6|single family| +15542|AAAAAAAAGLMDAAAA|297|Johnson 12th|Ln|Suite U|Arcola|Gwinnett County|GA|31654|United States|-5|condo| +15543|AAAAAAAAHLMDAAAA|225|14th Jackson|Circle|Suite J|Bethel|Newton County|AR|75281|United States|-6|single family| +15544|AAAAAAAAILMDAAAA|876|Poplar Pine|Wy|Suite 190|Crystal|Delaware County|NY|15258|United States|-5|apartment| +15545|AAAAAAAAJLMDAAAA|227|College 10th|Cir.|Suite 340|Oak Grove|Glades County|FL|38370|United States|-5|condo| +15546|AAAAAAAAKLMDAAAA|728|Mill Chestnut|RD|Suite 390|Glendale|Orange County|FL|33951|United States|-5|condo| +15547|AAAAAAAALLMDAAAA|40|Main Miller|Court|Suite B|Shiloh|Wayne County|WV|29275|United States|-5|apartment| +15548|AAAAAAAAMLMDAAAA|545|Cherry Center|Pkwy|Suite D|Hamilton|Jenkins County|GA|32808|United States|-5|condo| +15549|AAAAAAAANLMDAAAA|65|Chestnut |Court|Suite 50|Forest Hills|Kenai Peninsula Borough|AK|99237|United States|-9|apartment| +15550|AAAAAAAAOLMDAAAA|314|Locust |Road|Suite T|Bethel|Tunica County|MS|55281|United States|-6|condo| +15551|AAAAAAAAPLMDAAAA|452|Willow 1st|Wy|Suite 490|Pleasant Hill|Caldwell County|MO|63604|United States|-6|condo| +15552|AAAAAAAAAMMDAAAA|528|Fourth Second|Boulevard|Suite 460|Liberty|Garden County|NE|63451|United States|-6|condo| +15553|AAAAAAAABMMDAAAA|704|9th Cedar|Parkway|Suite 430|Lakeview|Clarion County|PA|18579|United States|-5|apartment| +15554|AAAAAAAACMMDAAAA|636|2nd |Way|Suite X|Union Hill|Crawford County|OH|47746|United States|-5|condo| +15555|AAAAAAAADMMDAAAA|576|Valley 2nd|Boulevard|Suite 20|Marion|Callahan County|TX|70399|United States|-6|single family| +15556|AAAAAAAAEMMDAAAA|109|3rd |Way|Suite 100|Plainview|Lawrence County|TN|33683|United States|-6|single family| +15557|AAAAAAAAFMMDAAAA|514|5th North|Avenue|Suite 100|Springfield|Grant County|AR|79303|United States|-6|condo| +15558|AAAAAAAAGMMDAAAA|22|Cedar Park|Boulevard|Suite 200|Bridgeport|Randolph County|AR|75817|United States|-6|condo| +15559|AAAAAAAAHMMDAAAA|460|Woodland |Cir.|Suite 240|Fairview|Fountain County|IN|45709|United States|-5|single family| +15560|AAAAAAAAIMMDAAAA|999|Center Highland|Ln|Suite 100|Green Acres|Wayne County|KY|47683|United States|-5|single family| +15561|AAAAAAAAJMMDAAAA|421|Cherry Elevnth|Cir.|Suite 350|Riverview|Muhlenberg County|KY|49003|United States|-5|single family| +15562|AAAAAAAAKMMDAAAA|304|View |Ave|Suite X|Greenfield|Macoupin County|IL|65038|United States|-6|single family| +15563|AAAAAAAALMMDAAAA|863|Williams |Ln|Suite H|Hamilton|Fauquier County|VA|22808|United States|-5|single family| +15564|AAAAAAAAMMMDAAAA|291|3rd Elm|Cir.|Suite D|Waterloo|Caribou County|ID|81675|United States|-7|single family| +15565|AAAAAAAANMMDAAAA|649|Lakeview |Dr.|Suite X|Cedar Grove|Maries County|MO|60411|United States|-6|condo| +15566|AAAAAAAAOMMDAAAA|199|Twelfth Spruce|Lane|Suite T|Sunnyside|Pemiscot County|MO|61952|United States|-6|apartment| +15567|AAAAAAAAPMMDAAAA|441|Hill |Ln|Suite 350|Franklin|Stevens County|KS|69101|United States|-6|apartment| +15568|AAAAAAAAANMDAAAA|30|Oak |Avenue|Suite 160|Proctor|Pitt County|NC|28140|United States|-5|apartment| +15569|AAAAAAAABNMDAAAA|22|West Ridge|Wy|Suite H|Fairview|La Salle County|IL|65709|United States|-6|single family| +15570|AAAAAAAACNMDAAAA|||||Lincoln|Lake County|||United States|-6|apartment| +15571|AAAAAAAADNMDAAAA|418|College |Blvd|Suite R|New Hope|Charles Mix County|SD|59431|United States|-6|single family| +15572|AAAAAAAAENMDAAAA|264|11th |Ct.|Suite T|Mount Zion|Chippewa County|MN|58054|United States|-6|single family| +15573|AAAAAAAAFNMDAAAA|270|Jackson 7th|Court|Suite H|Newtown|Webster County|MO|61749|United States|-6|apartment| +15574|AAAAAAAAGNMDAAAA|412|Main Walnut|Pkwy|Suite B|Midway|Marshall County|IN|41904|United States|-5|single family| +15575|AAAAAAAAHNMDAAAA|232|Dogwood |Circle|Suite A|White Oak|Clark County|AR|76668|United States|-6|condo| +15576|AAAAAAAAINMDAAAA|409|3rd Cherry|Cir.|Suite X|Woodville|Wadena County|MN|54289|United States|-6|single family| +15577|AAAAAAAAJNMDAAAA|73|Spring |Circle|Suite 120|Oakwood|Taos County|NM|80169|United States|-7|condo| +15578|AAAAAAAAKNMDAAAA|697|Main |Ct.|Suite P|Bunker Hill|San Miguel County|CO|80150|United States|-7|apartment| +15579|AAAAAAAALNMDAAAA|710|Cedar View|Ave|Suite R|La Grange|Bossier Parish|LA|77941|United States|-6|condo| +15580|AAAAAAAAMNMDAAAA|935|Elm Maple|Boulevard|Suite 340|Highland Park|Grant County|NE|66534|United States|-6|condo| +15581|AAAAAAAANNMDAAAA|472|View Smith|Blvd|Suite J|White Oak|Juniata County|PA|16668|United States|-5|condo| +15582|AAAAAAAAONMDAAAA|360|Church Johnson|Drive|Suite 230|Concord|Colonial Heights city|VA|24107|United States|-5|condo| +15583|AAAAAAAAPNMDAAAA|672|Twelfth Maple|Wy|Suite N|Mountain View|Ray County|MO|64466|United States|-6|single family| +15584|AAAAAAAAAOMDAAAA|366|Wilson Church|Pkwy|Suite P|Waterloo|Lawrence County|PA|11675|United States|-5|apartment| +15585|AAAAAAAABOMDAAAA|592|Park 1st|Drive|Suite B|Greenwood|Furnas County|NE|68828|United States|-6|apartment| +15586|AAAAAAAACOMDAAAA|775|Lakeview Dogwood|Blvd|Suite V|Edgewood|DeKalb County|IL|60069|United States|-6|condo| +15587|AAAAAAAADOMDAAAA|677|Second |Street|Suite 470|Macedonia|Larue County|KY|41087|United States|-5|apartment| +15588|AAAAAAAAEOMDAAAA|45|Madison |Cir.|Suite E|Hopewell|Alamosa County|CO|80587|United States|-7|single family| +15589|AAAAAAAAFOMDAAAA|234|Forest |Ln|Suite 10|Farmington|Prince George County|MD|29145|United States|-5|single family| +15590|AAAAAAAAGOMDAAAA|18|First |Street|Suite Y|Clinton|Ouachita County|AR|78222|United States|-6|condo| +15591|AAAAAAAAHOMDAAAA|756|Forest |Ct.|Suite E|Providence|Mineral County|CO|86614|United States|-7|condo| +15592|AAAAAAAAIOMDAAAA|699|5th |Avenue|Suite D|Longwood|Polk County|FL|37021|United States|-5|condo| +15593|AAAAAAAAJOMDAAAA|217|Church |Court|Suite 300|Bridgeport|Ohio County|KY|45817|United States|-5|condo| +15594|AAAAAAAAKOMDAAAA|567|Fourteenth |Ct.|Suite D|Enterprise|Greene County|AR|71757|United States|-6|condo| +15595|AAAAAAAALOMDAAAA|824|4th |Drive|Suite 190|Enterprise|Worcester County|MD|21757|United States|-5|condo| +15596|AAAAAAAAMOMDAAAA|548|Lincoln |Cir.|Suite H|Pleasant Hill|Putnam County|GA|33604|United States|-5|condo| +15597|AAAAAAAANOMDAAAA|773|Fifth Fifth|Way|Suite S|Pleasant Valley|Scott County|IL|62477|United States|-6|apartment| +15598|AAAAAAAAOOMDAAAA|733|Woodland |Wy|Suite 450|Brandon|Portsmouth city|VA|21278|United States|-5|apartment| +15599|AAAAAAAAPOMDAAAA|777|2nd |Street|Suite U|Bridgeport|Haskell County|OK|75817|United States|-6|condo| +15600|AAAAAAAAAPMDAAAA|163|Oak |Pkwy|Suite H|Greenville|Mills County|TX|71387|United States|-6|apartment| +15601|AAAAAAAABPMDAAAA|247|Wilson |Lane|Suite 0|Vienna|Baltimore city|MD|25119|United States|-5|single family| +15602|AAAAAAAACPMDAAAA|378|Third View|Way|Suite S|Mount Vernon|Emanuel County|GA|38482|United States|-5|single family| +15603|AAAAAAAADPMDAAAA|680|Forest Park|Lane|Suite 200|Oak Grove|Sevier County|UT|88370|United States|-7|condo| +15604|AAAAAAAAEPMDAAAA|869|12th Jackson|Lane|Suite A|Pleasant Hill|Kearny County|KS|63604|United States|-6|apartment| +15605|AAAAAAAAFPMDAAAA|38|Smith |Lane|Suite 160|Stringtown|Roberts County|SD|50162|United States|-7|single family| +15606|AAAAAAAAGPMDAAAA|862|Hillcrest |Street|Suite Q|Lebanon|Meade County|SD|52898|United States|-7|condo| +15607|AAAAAAAAHPMDAAAA|428|Dogwood Oak|Avenue|Suite D|Franklin|Warren County|KY|49101|United States|-5|single family| +15608|AAAAAAAAIPMDAAAA|150|Wilson Washington|Dr.|Suite 280|Union Hill|Waller County|TX|77746|United States|-6|condo| +15609|AAAAAAAAJPMDAAAA|780|Maple 8th|Boulevard|Suite 440|Red Hill|Logan County|OH|44338|United States|-5|apartment| +15610|AAAAAAAAKPMDAAAA|786|Ridge |Drive|Suite J|Bunker Hill|Stevens County|WA|90150|United States|-8|condo| +15611|AAAAAAAALPMDAAAA|75|Lakeview Park|Drive|Suite K|Antioch|Simpson County|KY|48605|United States|-5|apartment| +15612|AAAAAAAAMPMDAAAA|393|13th Maple|Pkwy|Suite F|Plainview|Eastland County|TX|73683|United States|-6|single family| +15613|AAAAAAAANPMDAAAA|121|Oak North|Way|Suite F|Somerville|Falls County|TX|77783|United States|-6|single family| +15614|AAAAAAAAOPMDAAAA|301|Main Lakeview|Dr.|Suite N|Marion|Conway County|AR|70399|United States|-6|apartment| +15615|AAAAAAAAPPMDAAAA|449|15th 5th|Cir.|Suite 280|Frogtown|Stark County|OH|48784|United States|-5|single family| +15616|AAAAAAAAAANDAAAA|585|First Pine|Pkwy|Suite 340|Maple Grove|La Salle County|IL|68252|United States|-6|apartment| +15617|AAAAAAAABANDAAAA|33|Hillcrest Green|Way|Suite L|Oak Ridge|Yoakum County|TX|78371|United States|-6|condo| +15618|AAAAAAAACANDAAAA|248|Railroad |Circle|Suite F|Pine Grove|Champaign County|OH|44593|United States|-5|condo| +15619|AAAAAAAADANDAAAA|159|11th |Way|Suite K|Kimball|Rockwall County|TX|73595|United States|-6|apartment| +15620|AAAAAAAAEANDAAAA|349|Green |Avenue|Suite 0|Bunker Hill|Logan County|KY|40150|United States|-5|apartment| +15621|AAAAAAAAFANDAAAA|481|Park Sunset|Way|Suite 80|Walnut Grove|Isle of Wight County|VA|27752|United States|-5|single family| +15622|AAAAAAAAGANDAAAA||3rd ||Suite 210|Walnut Grove|Garza County|TX||United States|-6|| +15623|AAAAAAAAHANDAAAA|819|Williams College|RD|Suite C|Wildwood|Russell County|AL|36871|United States|-6|condo| +15624|AAAAAAAAIANDAAAA|39|Church Spring|Road|Suite B|Shady Grove|Montague County|TX|72812|United States|-6|apartment| +15625|AAAAAAAAJANDAAAA|277|Broadway Fourth|Pkwy|Suite 180|Pleasant Hill|Grant County|KY|43604|United States|-6|single family| +15626|AAAAAAAAKANDAAAA|388|9th 15th|Lane|Suite 200|Crossroads|Brown County|TX|70534|United States|-6|condo| +15627|AAAAAAAALANDAAAA|307|Lake 8th|Way|Suite 70|Carpenter|Stonewall County|TX|71147|United States|-6|condo| +15628|AAAAAAAAMANDAAAA|539|Lakeview Johnson|Lane|Suite 120|Bethel|Christian County|KY|45281|United States|-6|apartment| +15629|AAAAAAAANANDAAAA|984|Spruce 8th|Boulevard|Suite 400|Five Forks|Benton County|MO|62293|United States|-6|condo| +15630|AAAAAAAAOANDAAAA|806|Church 3rd|Boulevard|Suite 60|Edgewood|Kay County|OK|70069|United States|-6|apartment| +15631|AAAAAAAAPANDAAAA|681|Jackson Valley|Court|Suite 310|Georgetown|Ballard County|KY|47057|United States|-6|apartment| +15632|AAAAAAAAABNDAAAA|378|Pine |Wy|Suite C|Pleasant Valley|DeWitt County|TX|72477|United States|-6|apartment| +15633|AAAAAAAABBNDAAAA|389|Fourteenth |Lane|Suite 260|Woodland|Kimball County|NE|64854|United States|-7|apartment| +15634|AAAAAAAACBNDAAAA|948|West Eigth|||Vienna|||25119||-5|| +15635|AAAAAAAADBNDAAAA|932|Wilson |Wy|Suite D|Amity|Love County|OK|70766|United States|-6|apartment| +15636|AAAAAAAAEBNDAAAA|||Road||||KY||||apartment| +15637|AAAAAAAAFBNDAAAA|285|Broadway Fifth|Road|Suite R|Pleasant Valley|Martin County|MN|52477|United States|-6|apartment| +15638|AAAAAAAAGBNDAAAA|12|14th |Circle|Suite W|Springfield|Beadle County|SD|59303|United States|-6|single family| +15639|AAAAAAAAHBNDAAAA|613|Elm |Wy|Suite R|Highland Park|Madison County|MS|56534|United States|-6|condo| +15640|AAAAAAAAIBNDAAAA|850|8th |Way|Suite J|Wilson|Durham County|NC|26971|United States|-5|apartment| +15641|AAAAAAAAJBNDAAAA|851|Davis |Avenue|Suite U|Five Points|Wabash County|IL|66098|United States|-6|single family| +15642|AAAAAAAAKBNDAAAA|573|Birch |Way|Suite 150|White Oak|Douglas County|WI|56668|United States|-6|single family| +15643|AAAAAAAALBNDAAAA|869|Center |Wy|Suite L|Macedonia|Hancock County|IL|61087|United States|-6|single family| +15644|AAAAAAAAMBNDAAAA|||||||MN|||-6|| +15645|AAAAAAAANBNDAAAA|844|Third Fourteenth|Drive|Suite C|Liberty|Warren County|VA|23451|United States|-5|apartment| +15646|AAAAAAAAOBNDAAAA||Main Willow|Dr.||Mount Olive||TX|||-6|apartment| +15647|AAAAAAAAPBNDAAAA|257|Seventh Locust|Road|Suite 150|Centerville|Iredell County|NC|20059|United States|-5|condo| +15648|AAAAAAAAACNDAAAA|834|13th |Blvd|Suite A|Deerfield|Rolette County|ND|59840|United States|-6|condo| +15649|AAAAAAAABCNDAAAA|992|Ash Lake|Dr.|Suite L|Arlington|Routt County|CO|86557|United States|-7|condo| +15650|AAAAAAAACCNDAAAA|257|Jackson |Drive|Suite 340|Crossroads|Wells County|ND|50534|United States|-6|apartment| +15651|AAAAAAAADCNDAAAA|996|Eigth |Road|Suite 390|Jackson|Williamsburg County|SC|29583|United States|-5|apartment| +15652|AAAAAAAAECNDAAAA|310|Second |Circle|Suite 100|Fairview|Vinton County|OH|45709|United States|-5|condo| +15653|AAAAAAAAFCNDAAAA|547|Jefferson Davis|Blvd|Suite 110|Dewey|Baraga County|MI|41160|United States|-5|condo| +15654|AAAAAAAAGCNDAAAA|353|First 13th|Ln|Suite U|Hillcrest|Lee County|FL|33003|United States|-5|apartment| +15655|AAAAAAAAHCNDAAAA|160|Chestnut |Ave|Suite 390|Florence|Union County|AR|73394|United States|-6|condo| +15656|AAAAAAAAICNDAAAA|150|Willow |Avenue|Suite N|Walnut Grove|Ziebach County|SD|57752|United States|-6|condo| +15657|AAAAAAAAJCNDAAAA|172|Spring Woodland|Lane|Suite Q|Belleville|Falls Church city|VA|22924|United States|-5|apartment| +15658|AAAAAAAAKCNDAAAA|918|Spring 3rd|Drive|Suite Y|Macedonia|Irion County|TX|71087|United States|-6|single family| +15659|AAAAAAAALCNDAAAA|862|Jefferson |Way|Suite L|Springfield|Gooding County|ID|89303|United States|-7|apartment| +15660|AAAAAAAAMCNDAAAA|449|Meadow South|Way|Suite 200|Phoenix|Houston County|TX|72276|United States|-6|single family| +15661|AAAAAAAANCNDAAAA|755|Sunset |Circle|Suite W|Macedonia|Cedar County|MO|61087|United States|-6|condo| +15662|AAAAAAAAOCNDAAAA|52|Forest |Parkway|Suite 170|Waterloo|Jennings County|IN|41675|United States|-5|condo| +15663|AAAAAAAAPCNDAAAA|312|Washington |ST|Suite 400|Wilson|Warren County|NJ|07571|United States|-5|condo| +15664|AAAAAAAAADNDAAAA|304|College Elm|Dr.|Suite 290|Oakwood|Laurens County|GA|30169|United States|-5|condo| +15665|AAAAAAAABDNDAAAA|950|Second Ninth|Road|Suite 160|Pine Grove|Lincoln County|ME|05193|United States|-5|apartment| +15666|AAAAAAAACDNDAAAA|762|Washington |Road|Suite 300|Macedonia|Edmonson County|KY|41087|United States|-6|single family| +15667|AAAAAAAADDNDAAAA|935|Fourth |Dr.|Suite M|Lakewood|Alpine County|CA|98877|United States|-8|single family| +15668|AAAAAAAAEDNDAAAA||||Suite G|Harmony|Catoosa County|||United States|-5|apartment| +15669|AAAAAAAAFDNDAAAA|923|Spring |Ct.|Suite 180|Oakdale|Cherokee County|IA|59584|United States|-6|single family| +15670|AAAAAAAAGDNDAAAA|405|Lee |Ct.|Suite I|Friendship|Bladen County|NC|24536|United States|-5|condo| +15671|AAAAAAAAHDNDAAAA|148|Chestnut Maple|ST|Suite U|Jackson|Gilliam County|OR|99583|United States|-8|single family| +15672|AAAAAAAAIDNDAAAA|207|North |Blvd|Suite 270|Spring Hill|Stewart County|GA|36787|United States|-5|condo| +15673|AAAAAAAAJDNDAAAA|257||Ct.||Brownsville|Winston County|AL|39310|United States|-6|| +15674|AAAAAAAAKDNDAAAA|476|Johnson Locust|Road|Suite 210|Woodland|Skamania County|WA|94854|United States|-8|single family| +15675|AAAAAAAALDNDAAAA|118|Johnson 5th|Court|Suite T|Clinton|Webb County|TX|78222|United States|-6|single family| +15676|AAAAAAAAMDNDAAAA|723|Laurel Forest|Ln|Suite 100|Georgetown|Upshur County|TX|77057|United States|-6|single family| +15677|AAAAAAAANDNDAAAA|184|5th Center|RD|Suite Q|Cedar Grove|Foard County|TX|70411|United States|-6|single family| +15678|AAAAAAAAODNDAAAA|70|Jackson Laurel|Pkwy|Suite U|Hillcrest|Bartow County|GA|33003|United States|-5|apartment| +15679|AAAAAAAAPDNDAAAA|639|Locust |Circle|Suite O|Deerfield|Hamblen County|TN|39840|United States|-5|apartment| +15680|AAAAAAAAAENDAAAA|583|||||Brown County|||United States|-6|| +15681|AAAAAAAABENDAAAA|191|Sunset |Court|Suite B|Farmington|Clinton County|IN|49145|United States|-5|apartment| +15682|AAAAAAAACENDAAAA|166|Main Center|Cir.|Suite 100|Crossroads|Simpson County|MS|50534|United States|-6|condo| +15683|AAAAAAAADENDAAAA|255|11th Park|Dr.|Suite L|Plainview|Teton County|MT|63683|United States|-7|apartment| +15684|AAAAAAAAEENDAAAA|122|Third Jefferson|Wy|Suite Q|Mount Olive|Kent County|MD|28059|United States|-5|apartment| +15685|AAAAAAAAFENDAAAA|576|Spring |Ave|Suite M|Birmingham|Mason County|KY|43372|United States|-5|single family| +15686|AAAAAAAAGENDAAAA|157|Poplar Forest|Boulevard|Suite O|Shiloh|Morgan County|KY|49275|United States|-5|condo| +15687|AAAAAAAAHENDAAAA|999|10th |Cir.|Suite K|Enterprise|Terrell County|TX|71757|United States|-6|condo| +15688|AAAAAAAAIENDAAAA|476|First |Lane|Suite 420|Hopewell|Winnebago County|IL|60587|United States|-6|condo| +15689|AAAAAAAAJENDAAAA|743|Valley River|Ave|Suite 380|Summit|Greer County|OK|70499|United States|-6|apartment| +15690|AAAAAAAAKENDAAAA|932||||Lakeside||MT||United States||single family| +15691|AAAAAAAALENDAAAA|||Drive|||Quitman County|||United States||| +15692|AAAAAAAAMENDAAAA|381|First |Dr.|Suite D|Oakwood|Mississippi County|MO|60169|United States|-6|single family| +15693|AAAAAAAANENDAAAA|565|Meadow Jackson|Circle|Suite 460|Bethel|Granite County|MT|65281|United States|-7|single family| +15694|AAAAAAAAOENDAAAA|551|Sixth |Avenue|Suite 0|Crossroads|Greene County|PA|10534|United States|-5|single family| +15695|AAAAAAAAPENDAAAA|424|Highland |Avenue|Suite 460|Deerfield|Lafayette Parish|LA|79840|United States|-6|condo| +15696|AAAAAAAAAFNDAAAA|386|Third |Dr.|Suite 390|Mechanicsburg|Franklin County|IA|52219|United States|-6|single family| +15697|AAAAAAAABFNDAAAA|531|Cherry Locust|Wy|Suite 190|Mount Olive|Moore County|TN|38059|United States|-6|apartment| +15698|AAAAAAAACFNDAAAA|557|Williams |Court|Suite N|Woodville|Butler County|MO|64289|United States|-6|apartment| +15699|AAAAAAAADFNDAAAA|23|Valley 5th|Lane|Suite K|Belmont|Menominee County|WI|50191|United States|-6|condo| +15700|AAAAAAAAEFNDAAAA|330|Poplar |Lane|Suite 370|Antioch|Morton County|ND|58605|United States|-6|apartment| +15701|AAAAAAAAFFNDAAAA|629|Center |Cir.|Suite I|Oak Grove|Cameron County|TX|78370|United States|-6|condo| +15702|AAAAAAAAGFNDAAAA|430|Lake Hickory|Dr.|Suite W|Hillcrest|Carbon County|MT|63003|United States|-7|condo| +15703|AAAAAAAAHFNDAAAA|811|2nd Laurel|Street|Suite 10|West Liberty|Palo Alto County|IA|54752|United States|-6|single family| +15704|AAAAAAAAIFNDAAAA|320|Cherry Maple|Court|Suite D|Pine Valley|Pacific County|WA|98209|United States|-8|single family| +15705|AAAAAAAAJFNDAAAA|380|Sunset Central|ST|Suite O|Farmington|Bibb County|AL|39145|United States|-6|condo| +15706|AAAAAAAAKFNDAAAA|429|Park Laurel|Wy|Suite 400|Highland|Washington County|AL|39454|United States|-6|condo| +15707|AAAAAAAALFNDAAAA|198|Church |Blvd|Suite H|Five Forks|Putnam County|IN|42293|United States|-5|single family| +15708|AAAAAAAAMFNDAAAA|536|Elm Park|Drive|Suite 370|Union|Cheboygan County|MI|48721|United States|-5|single family| +15709|AAAAAAAANFNDAAAA|158|Hill |Circle|Suite D|Lakeside|Lowndes County|MS|59532|United States|-6|single family| +15710|AAAAAAAAOFNDAAAA|687|Park |Cir.|Suite 400|Arlington|Mason County|WA|96557|United States|-8|condo| +15711|AAAAAAAAPFNDAAAA|63|7th |||Greenwood|||48828|United States|-5|single family| +15712|AAAAAAAAAGNDAAAA|486|7th |Pkwy|Suite 110|Concord|Clark County|WA|94107|United States|-8|single family| +15713|AAAAAAAABGNDAAAA|64|Cedar 5th|Circle|Suite C|Enterprise|Alcona County|MI|41757|United States|-5|single family| +15714|AAAAAAAACGNDAAAA|463|3rd Lincoln|Boulevard|Suite F|Bunker Hill|Caswell County|NC|20150|United States|-5|condo| +15715|AAAAAAAADGNDAAAA|522|Washington |Road|Suite K|Oak Grove|Prince Edward County|VA|28370|United States|-5|apartment| +15716|AAAAAAAAEGNDAAAA|161|9th |Avenue|Suite 250|Midway|Chester County|PA|11904|United States|-5|single family| +15717|AAAAAAAAFGNDAAAA|750|Central |Ln|Suite 210|Concord|Newaygo County|MI|44107|United States|-5|condo| +15718|AAAAAAAAGGNDAAAA|485|Lake Elm|Street|Suite C|Forest Hills|Wilkinson County|MS|59237|United States|-6|single family| +15719|AAAAAAAAHGNDAAAA|709|Railroad First|Drive|Suite D|Salem|Atlantic County|NJ|08648|United States|-5|apartment| +15720|AAAAAAAAIGNDAAAA||7th |Road|Suite 350|Mountain View||||||| +15721|AAAAAAAAJGNDAAAA||Church ||Suite 40||Carbon County|||United States||| +15722|AAAAAAAAKGNDAAAA|915|Laurel |Avenue|Suite H|Oakdale|Vermilion Parish|LA|79584|United States|-6|single family| +15723|AAAAAAAALGNDAAAA|227|Franklin |Ave|Suite 40|Saint James|Josephine County|OR|95799|United States|-8|condo| +15724|AAAAAAAAMGNDAAAA|55|Lee Locust|Wy|Suite X|Pleasant Grove|Appanoose County|IA|54136|United States|-6|apartment| +15725|AAAAAAAANGNDAAAA|260|||Suite I|||NC||||apartment| +15726|AAAAAAAAOGNDAAAA|522|North |Blvd|Suite M|Highland Park|Taliaferro County|GA|36534|United States|-5|apartment| +15727|AAAAAAAAPGNDAAAA|187|Chestnut |Cir.|Suite 30|Westminster|Seminole County|OK|76549|United States|-6|condo| +15728|AAAAAAAAAHNDAAAA|268|14th |Parkway|Suite T|Shiloh|Chautauqua County|KS|69275|United States|-6|apartment| +15729|AAAAAAAABHNDAAAA|321|Ridge Fourth|Lane|Suite E|Lakeview|Whitman County|WA|98579|United States|-8|apartment| +15730|AAAAAAAACHNDAAAA|521|Main 2nd|RD|Suite D|Pinhook|Pottawatomie County|KS|69398|United States|-6|condo| +15731|AAAAAAAADHNDAAAA|881|North |Ave|Suite 300|Marion|Cherokee County|KS|60399|United States|-6|single family| +15732|AAAAAAAAEHNDAAAA|215|2nd 1st|Pkwy|Suite 120|Oak Ridge|Faulk County|SD|58371|United States|-6|single family| +15733|AAAAAAAAFHNDAAAA|537|Sixth Spring|Parkway|Suite 40|Providence|Avery County|NC|26614|United States|-5|condo| +15734|AAAAAAAAGHNDAAAA|265|Willow |Pkwy|Suite Q|Bethel|Wahkiakum County|WA|95281|United States|-8|condo| +15735|AAAAAAAAHHNDAAAA|718|Pine Elm|Ave|Suite X|Union|Chenango County|NY|18721|United States|-5|condo| +15736|AAAAAAAAIHNDAAAA|865|Sunset |Street|Suite Q|Woodlawn|Henry County|KY|44098|United States|-6|single family| +15737|AAAAAAAAJHNDAAAA|||||Sulphur Springs|||78354|||| +15738|AAAAAAAAKHNDAAAA|210|Park Madison|Way|Suite 380|Oakwood|Gilchrist County|FL|30169|United States|-5|condo| +15739|AAAAAAAALHNDAAAA|781|Lake |Circle|Suite 430|Centerville|Dutchess County|NY|10059|United States|-5|condo| +15740|AAAAAAAAMHNDAAAA|842|Jefferson |Boulevard|Suite V|Oakwood|Vermilion County|IL|60169|United States|-6|apartment| +15741|AAAAAAAANHNDAAAA|89|South Willow|Boulevard|Suite 280|Oakwood|Middlesex County|VA|20169|United States|-5|apartment| +15742|AAAAAAAAOHNDAAAA|895|Church |Ln|Suite 210|Jackson|San Luis Obispo County|CA|99583|United States|-8|apartment| +15743|AAAAAAAAPHNDAAAA|818|Church 10th|Avenue|Suite 80|Farmersville|Sublette County|WY|89305|United States|-7|apartment| +15744|AAAAAAAAAINDAAAA|215|Pine |Ln|Suite 300|Wilson|Golden Valley County|MT|66971|United States|-7|condo| +15745|AAAAAAAABINDAAAA|892|Franklin |Dr.|Suite E|Jamestown|Graham County|AZ|86867|United States|-7|apartment| +15746|AAAAAAAACINDAAAA|588|East |Cir.|Suite P|Maple Grove|Shelby County|IL|68252|United States|-6|apartment| +15747|AAAAAAAADINDAAAA|308|Walnut Eigth|Parkway|Suite W|Glenwood|Door County|WI|53511|United States|-6|condo| +15748|AAAAAAAAEINDAAAA|665|Lincoln Lincoln|Blvd|Suite K|Arlington|Woodbury County|IA|56557|United States|-6|condo| +15749|AAAAAAAAFINDAAAA|520|Lee |Drive|Suite W|Enterprise|Bernalillo County|NM|81757|United States|-7|condo| +15750|AAAAAAAAGINDAAAA|964|Jackson Second|RD|Suite 0|Summit|Assumption Parish|LA|70499|United States|-6|apartment| +15751|AAAAAAAAHINDAAAA|146|Park 6th|Ct.|Suite 290|Avery|Logan County|KY|40194|United States|-5|apartment| +15752|AAAAAAAAIINDAAAA|880|Meadow 5th|Court|Suite 150|Marion|Geary County|KS|60399|United States|-6|apartment| +15753|AAAAAAAAJINDAAAA|179|Second |Drive|Suite S|Fairview|Warren County|IA|55709|United States|-6|single family| +15754|AAAAAAAAKINDAAAA|402|Laurel |Boulevard|Suite X|Walnut Grove|Washington County|RI|08352|United States|-5|condo| +15755|AAAAAAAALINDAAAA|484|3rd |Parkway|Suite 50|Wildwood|Hamilton County|IN|46871|United States|-5|condo| +15756|AAAAAAAAMINDAAAA|84|9th 7th|Drive|Suite U|Wildwood|Crook County|OR|96871|United States|-8|apartment| +15757|AAAAAAAANINDAAAA|133|West |Court|Suite 270|Union|Baxter County|AR|78721|United States|-6|condo| +15758|AAAAAAAAOINDAAAA|301|Spring Highland|Lane|Suite N|Fairfield|Price County|WI|56192|United States|-6|apartment| +15759|AAAAAAAAPINDAAAA|132|4th Third|Drive|Suite Q|Union|Washakie County|WY|88721|United States|-7|condo| +15760|AAAAAAAAAJNDAAAA|292|2nd |Dr.|Suite M|Lebanon|Lincoln County|AR|72898|United States|-6|condo| +15761|AAAAAAAABJNDAAAA|810|Hillcrest 6th|Street|Suite 30|Union Hill|Jasper County|IN|47746|United States|-5|apartment| +15762|AAAAAAAACJNDAAAA|771|Adams Meadow|Ave|Suite X|Pleasant Hill|Fulton County|GA|33604|United States|-5|apartment| +15763|AAAAAAAADJNDAAAA|188|Park View|Boulevard|Suite 350|Mount Vernon|Monroe County|IA|58482|United States|-6|condo| +15764|AAAAAAAAEJNDAAAA|299|2nd |Street|Suite K|Greenfield|Routt County|CO|85038|United States|-7|single family| +15765|AAAAAAAAFJNDAAAA|149|Center 10th|Wy|Suite L|Montezuma|Bryan County|OK|72150|United States|-6|apartment| +15766|AAAAAAAAGJNDAAAA|902|Washington |Ln|Suite S|Marion|Lincoln County|NV|80399|United States|-8|apartment| +15767|AAAAAAAAHJNDAAAA|166|Park |Ave|Suite 30|Pleasant Valley|Kearney County|NE|62477|United States|-7|condo| +15768|AAAAAAAAIJNDAAAA|53|7th Poplar|Parkway|Suite 160|Sulphur Springs|Hardin County|IL|68354|United States|-6|apartment| +15769|AAAAAAAAJJNDAAAA|4|Johnson 2nd|Lane|Suite 190|Clinton|Victoria County|TX|78222|United States|-6|apartment| +15770|AAAAAAAAKJNDAAAA|910|Adams Park|Pkwy|Suite Q|Belmont|Taylor County|GA|30191|United States|-5|apartment| +15771|AAAAAAAALJNDAAAA|232|2nd |Wy|Suite F|Riverside|Montgomery County|KY|49231|United States|-5|condo| +15772|AAAAAAAAMJNDAAAA|75|Lake |Ln|Suite J|Concord|Androscoggin County|ME|04707|United States|-5|apartment| +15773|AAAAAAAANJNDAAAA|815|10th |Blvd|Suite P|Forest Hills|Galax city|VA|29237|United States|-5|apartment| +15774|AAAAAAAAOJNDAAAA|510|Eigth |Pkwy|Suite G|Unionville|Jackson County|IA|51711|United States|-6|condo| +15775|AAAAAAAAPJNDAAAA|27|||Suite U||||51933||-6|condo| +15776|AAAAAAAAAKNDAAAA|352|Lake Madison|Parkway|Suite M|Cedar Grove|Monroe County|MO|60411|United States|-6|apartment| +15777|AAAAAAAABKNDAAAA|100|College Broadway|Cir.|Suite 20|Wildwood|Franklin County|IL|66871|United States|-6|single family| +15778|AAAAAAAACKNDAAAA|592|Park Highland|Ave|Suite 50|Maple Grove|Osage County|KS|68252|United States|-6|condo| +15779|AAAAAAAADKNDAAAA|232|Lee Locust|Wy|Suite J|Marion|Chattahoochee County|GA|30399|United States|-5|apartment| +15780|AAAAAAAAEKNDAAAA|357|Green 12th|||||NE||United States||single family| +15781|AAAAAAAAFKNDAAAA|||Court||||||United States|-7|single family| +15782|AAAAAAAAGKNDAAAA|479|14th 1st|Cir.|Suite B|Summit|Pointe Coupee Parish|LA|70499|United States|-6|condo| +15783|AAAAAAAAHKNDAAAA|723|Johnson Adams|Circle|Suite C|Deerfield|Merced County|CA|99840|United States|-8|apartment| +15784|AAAAAAAAIKNDAAAA|637|Third Madison|Circle|Suite Q|Oakwood|Trousdale County|TN|30169|United States|-6|apartment| +15785|AAAAAAAAJKNDAAAA|526|15th |Court|Suite K|Maple Grove|Callahan County|TX|78252|United States|-6|condo| +15786|AAAAAAAAKKNDAAAA||4th ||Suite 160|||||United States||apartment| +15787|AAAAAAAALKNDAAAA|864|Highland |Boulevard|Suite 180|Mount Pleasant|Lake County|MN|51933|United States|-6|condo| +15788|AAAAAAAAMKNDAAAA|84|13th Jefferson|Avenue|Suite O|Oakdale|New London County|CT|09584|United States|-5|condo| +15789|AAAAAAAANKNDAAAA|865|Tenth |Ct.|Suite 300|Bridgeport|Howard County|IN|45817|United States|-5|apartment| +15790|AAAAAAAAOKNDAAAA|944|Jackson |Parkway|Suite M|Oakland|Howell County|MO|69843|United States|-6|single family| +15791|AAAAAAAAPKNDAAAA|584|4th |Parkway|Suite 360|Franklin|Bell County|KY|49101|United States|-6|single family| +15792|AAAAAAAAALNDAAAA|649|1st 12th|Way|Suite T|Green Acres|Douglas County|WI|57683|United States|-6|apartment| +15793|AAAAAAAABLNDAAAA|758|Main |Pkwy|Suite D|Edgewood|Washoe County|NV|80069|United States|-8|condo| +15794|AAAAAAAACLNDAAAA|||Dr.|Suite 300|Sunnyside|||||-6|| +15795|AAAAAAAADLNDAAAA|747|10th |ST|Suite G|Wildwood|Adams County|WA|96871|United States|-8|apartment| +15796|AAAAAAAAELNDAAAA|412|Davis |RD|Suite L|Poplar Grove|Galveston County|TX|73664|United States|-6|apartment| +15797|AAAAAAAAFLNDAAAA|205|2nd Lincoln|Wy|Suite R|Shelby|Clark County|KY|46575|United States|-6|apartment| +15798|AAAAAAAAGLNDAAAA|672|4th |Court|Suite 390|Green Acres|Crawford County|MI|47683|United States|-5|single family| +15799|AAAAAAAAHLNDAAAA|391|Franklin |Road|Suite U|Woodbine|Taney County|MO|64253|United States|-6|condo| +15800|AAAAAAAAILNDAAAA|772|Jefferson Ridge|Dr.|Suite I|Shady Grove|Lincoln County|MS|52812|United States|-6|apartment| +15801|AAAAAAAAJLNDAAAA|291|Sunset 15th|Court|Suite L|Deerfield|Colfax County|NE|69840|United States|-6|apartment| +15802|AAAAAAAAKLNDAAAA|815|Railroad Oak|Parkway|Suite 270|Glenwood|James City County|VA|23511|United States|-5|single family| +15803|AAAAAAAALLNDAAAA|316|Maple Ridge|Way|Suite F|Clifton|Gilpin County|CO|88014|United States|-7|condo| +15804|AAAAAAAAMLNDAAAA|119|15th |Avenue|Suite 370|Elkton|Menard County|IL|63481|United States|-6|single family| +15805|AAAAAAAANLNDAAAA|959|Mill |Court|Suite L|Five Forks|Lake County|IN|42293|United States|-5|apartment| +15806|AAAAAAAAOLNDAAAA|189|Valley Hillcrest|Parkway|Suite 120|White Oak|Hodgeman County|KS|66668|United States|-6|apartment| +15807|AAAAAAAAPLNDAAAA|69|Park Main|Avenue|Suite 350|Pleasant Hill|Tillamook County|OR|93604|United States|-8|single family| +15808|AAAAAAAAAMNDAAAA|366||RD|Suite 380||Wayne County|NY|14466|United States|-5|single family| +15809|AAAAAAAABMNDAAAA|648|Spring |RD|Suite T|Five Points|Calvert County|MD|26098|United States|-5|single family| +15810|AAAAAAAACMNDAAAA|197|2nd |Wy|Suite 360|Macedonia|Calloway County|KY|41087|United States|-6|apartment| +15811|AAAAAAAADMNDAAAA|315|Franklin |Way|Suite 220|Hopewell|Jefferson County|MO|60587|United States|-6|condo| +15812|AAAAAAAAEMNDAAAA|113|Ridge 11th|Blvd|Suite Q|Waterloo|Carroll County|VA|21675|United States|-5|condo| +15813|AAAAAAAAFMNDAAAA|706|Sycamore 7th|Boulevard|Suite 140|Green Acres|Reno County|KS|67683|United States|-6|condo| +15814|AAAAAAAAGMNDAAAA|781|Locust 2nd|Drive|Suite 400|Oak Grove|Webster County|IA|58370|United States|-6|single family| +15815|AAAAAAAAHMNDAAAA|167|Green Broadway|RD|Suite R|Newport|Appanoose County|IA|51521|United States|-6|apartment| +15816|AAAAAAAAIMNDAAAA|726|Cedar Birch|Parkway|Suite 330|Leland|Montgomery County|NC|29452|United States|-5|condo| +15817|AAAAAAAAJMNDAAAA|85|Spruce Lincoln|Lane|Suite L|Valley View|Kimble County|TX|75124|United States|-6|condo| +15818|AAAAAAAAKMNDAAAA|104|Pine |Boulevard|Suite X|New Hope|Clear Creek County|CO|89431|United States|-7|single family| +15819|AAAAAAAALMNDAAAA|583|Locust |Parkway|Suite B|New Hope|Beaver County|UT|89431|United States|-7|condo| +15820|AAAAAAAAMMNDAAAA|582|1st 5th|Pkwy|Suite 330|Jamestown|Kendall County|TX|76867|United States|-6|condo| +15821|AAAAAAAANMNDAAAA||Lakeview Elevnth||Suite G||San Juan County|||United States||| +15822|AAAAAAAAOMNDAAAA|366|Park |Lane|Suite H|Spring Hill|Tama County|IA|56787|United States|-6|condo| +15823|AAAAAAAAPMNDAAAA||||Suite D|Glendale|||23951|||single family| +15824|AAAAAAAAANNDAAAA|573|Chestnut Main|Boulevard|Suite 20|Union Hill|Miller County|AR|77746|United States|-6|condo| +15825|AAAAAAAABNNDAAAA|379|Second 4th|Ave|Suite D|Unionville|Dickinson County|IA|51711|United States|-6|single family| +15826|AAAAAAAACNNDAAAA|950|Oak |Ct.|Suite N|Woodlawn|Murray County|OK|74098|United States|-6|apartment| +15827|AAAAAAAADNNDAAAA|547|Maple |Ave|Suite L|Salem|Floyd County|TX|78048|United States|-6|condo| +15828|AAAAAAAAENNDAAAA|329|7th |Street|Suite Y|Sunnyside|Utah County|UT|81952|United States|-7|apartment| +15829|AAAAAAAAFNNDAAAA|526|1st |Court|Suite D|Galena|Hardin County|TN|34369|United States|-5|condo| +15830|AAAAAAAAGNNDAAAA|579|Elm |Boulevard|Suite A|Shiloh|Hawaii County|HI|99275|United States|-10|single family| +15831|AAAAAAAAHNNDAAAA|379|Birch 6th|Court|Suite Q|Pleasant Grove|Bamberg County|SC|24136|United States|-5|condo| +15832|AAAAAAAAINNDAAAA|296|Third |Pkwy|Suite 340|Greenwood|Nemaha County|NE|68828|United States|-7|single family| +15833|AAAAAAAAJNNDAAAA|694|Highland |Ln|Suite 210|Wilson|Montgomery County|KY|46971|United States|-5|single family| +15834|AAAAAAAAKNNDAAAA|987|9th |Ct.|Suite O|Union|Lexington city|VA|28721|United States|-5|apartment| +15835|AAAAAAAALNNDAAAA|337|2nd Franklin|Ln|Suite A|Johnsonville|Cameron County|TX|77745|United States|-6|condo| +15836|AAAAAAAAMNNDAAAA|420|Broadway |Pkwy|Suite M|Rocky Point|Tazewell County|IL|61209|United States|-6|apartment| +15837|AAAAAAAANNNDAAAA|905|First |Boulevard|Suite U|Hillcrest|Slope County|ND|53003|United States|-6|single family| +15838|AAAAAAAAONNDAAAA|609|Sycamore |Blvd|Suite 170|Fairview|Clark County|OH|45709|United States|-5|apartment| +15839|AAAAAAAAPNNDAAAA|325|Davis Second|Blvd|Suite N|Riverview|Fentress County|TN|39003|United States|-5|single family| +15840|AAAAAAAAAONDAAAA|966|Lake |RD|Suite 230|Hamilton|Shackelford County|TX|72808|United States|-6|condo| +15841|AAAAAAAABONDAAAA|325|13th 1st|ST|Suite B|Valley View|Green County|WI|55124|United States|-6|condo| +15842|AAAAAAAACONDAAAA|951|Locust |Lane|Suite 180|Hillcrest|Lampasas County|TX|73003|United States|-6|condo| +15843|AAAAAAAADONDAAAA|69|Woodland Center|Road|Suite G|Mountain View|Horry County|SC|24466|United States|-5|apartment| +15844|AAAAAAAAEONDAAAA|561|Park |Drive|Suite 120|Salem|Lucas County|OH|48048|United States|-5|condo| +15845|AAAAAAAAFONDAAAA|700|10th |Lane|Suite L|Fairmont|Titus County|TX|75987|United States|-6|single family| +15846|AAAAAAAAGONDAAAA|338|2nd |Street|Suite 150|Antioch|Belmont County|OH|48605|United States|-5|condo| +15847|AAAAAAAAHONDAAAA|863|Third 3rd|Avenue|Suite 320|Oak Grove|Suffolk city|VA|28370|United States|-5|condo| +15848|AAAAAAAAIONDAAAA|3|Poplar |Wy|Suite 170|Crossroads|Hartford County|CT|01134|United States|-5|condo| +15849|AAAAAAAAJONDAAAA|61|5th |Street|Suite 90|Lincoln|McLeod County|MN|51289|United States|-6|condo| +15850|AAAAAAAAKONDAAAA|44|2nd Davis|Ct.|Suite 90|Lakeside|Cerro Gordo County|IA|59532|United States|-6|condo| +15851|AAAAAAAALONDAAAA|513|Church ||Suite O|Shady Grove|Callahan County||||-6|| +15852|AAAAAAAAMONDAAAA|276|Sixth 1st|Road|Suite T|Greenwood|Huntingdon County|PA|18828|United States|-5|apartment| +15853|AAAAAAAANONDAAAA||5th ||Suite 370||Black Hawk County|IA||||| +15854|AAAAAAAAOONDAAAA|||Lane|||Clay County||68721|||| +15855|AAAAAAAAPONDAAAA|803|Pine Center|Circle|Suite V|Mount Olive|Greenbrier County|WV|28059|United States|-5|apartment| +15856|AAAAAAAAAPNDAAAA|648|Green |Pkwy|Suite 490|Clifton|Yukon-Koyukuk Census Area|AK|98014|United States|-9|condo| +15857|AAAAAAAABPNDAAAA|577|Pine |Wy|Suite J|Midway|Brown County|SD|51904|United States|-6|single family| +15858|AAAAAAAACPNDAAAA|959|Main Walnut|Avenue|Suite 10|Carthage|Fulton County|NY|11529|United States|-5|condo| +15859|AAAAAAAADPNDAAAA||Chestnut Mill||Suite O|||||United States|-5|condo| +15860|AAAAAAAAEPNDAAAA|522|4th |Way|Suite E|Mount Olive|Iberia Parish|LA|78059|United States|-6|condo| +15861|AAAAAAAAFPNDAAAA|358|Jackson Fifth|Boulevard|Suite A|Sherwood Forest|Kimball County|NE|66702|United States|-7|condo| +15862|AAAAAAAAGPNDAAAA|876|First |Street|Suite 60|Kingston|Van Buren County|TN|34975|United States|-6|apartment| +15863|AAAAAAAAHPNDAAAA|664|Main |ST|Suite 270|Green Acres|Santa Cruz County|AZ|87683|United States|-7|condo| +15864|AAAAAAAAIPNDAAAA|981|Oak |Boulevard|Suite 300|Shiloh|Jay County|IN|49275|United States|-5|single family| +15865|AAAAAAAAJPNDAAAA|427|Spring |Cir.|Suite P|Bridgeport|Belmont County|OH|45817|United States|-5|single family| +15866|AAAAAAAAKPNDAAAA|204|4th |Parkway|Suite Y|Mountain View|Pope County|AR|74466|United States|-6|apartment| +15867|AAAAAAAALPNDAAAA|936|Hickory |Court|Suite 70|Brownsville|Mohave County|AZ|89310|United States|-7|single family| +15868|AAAAAAAAMPNDAAAA|558|Dogwood Johnson|ST|Suite 20|Pleasant Grove|Montgomery County|PA|14136|United States|-5|single family| +15869|AAAAAAAANPNDAAAA|669|Church |Pkwy|Suite Q|Fairfield|Hardin County|OH|46192|United States|-5|apartment| +15870|AAAAAAAAOPNDAAAA|366|Washington |Lane|Suite P|Summerfield|Gogebic County|MI|40634|United States|-5|apartment| +15871|AAAAAAAAPPNDAAAA|684|Spring |Street|Suite 420|Springfield|Roscommon County|MI|49303|United States|-5|single family| +15872|AAAAAAAAAAODAAAA|247|Green Birch|Parkway|Suite 490|Wilson|Benewah County|ID|86971|United States|-7|apartment| +15873|AAAAAAAABAODAAAA|604|Main View|Dr.|Suite C|Lakeside|Wythe County|VA|29532|United States|-5|single family| +15874|AAAAAAAACAODAAAA|698|Second |Boulevard|Suite Q|Stringtown|Caribou County|ID|80162|United States|-7|condo| +15875|AAAAAAAADAODAAAA|229|Davis Elm|Pkwy|Suite J|Oakdale|Brooks County|GA|39584|United States|-5|single family| +15876|AAAAAAAAEAODAAAA|98|1st Spring|Ln|Suite 170|Buena Vista|San Patricio County|TX|75752|United States|-6|condo| +15877|AAAAAAAAFAODAAAA|877|Adams Fourth|Cir.|Suite L|Franklin|Wayne County|TN|39101|United States|-6|condo| +15878|AAAAAAAAGAODAAAA|632|Ash |Road|Suite H|Harmony|Caldwell County|NC|25804|United States|-5|single family| +15879|AAAAAAAAHAODAAAA|738|10th Jackson|Circle|Suite T|Fairview|Ashland County|WI|55709|United States|-6|condo| +15880|AAAAAAAAIAODAAAA|217|5th |Parkway|Suite L|Sunnyside|Duchesne County|UT|81952|United States|-7|condo| +15881|AAAAAAAAJAODAAAA|264|Broadway |Way|Suite Y|Lakeview|Union County|NJ|09179|United States|-5|single family| +15882|AAAAAAAAKAODAAAA|907|4th |Ave|Suite 170|Clinton|Iroquois County|IL|68222|United States|-6|single family| +15883|AAAAAAAALAODAAAA|282|3rd Birch|ST|Suite 370|Springdale|Ritchie County|WV|28883|United States|-5|apartment| +15884|AAAAAAAAMAODAAAA|566|Fifth |ST|Suite 420|Valley View|Bailey County|TX|75124|United States|-6|apartment| +15885|AAAAAAAANAODAAAA|691|Central Lakeview|Boulevard|Suite A|Riverview|Clay County|IL|69003|United States|-6|condo| +15886|AAAAAAAAOAODAAAA|579|Smith |Road|Suite Q|Bridgeport|Kearny County|KS|65817|United States|-6|apartment| +15887|AAAAAAAAPAODAAAA|558|Eigth |Drive|Suite 490|Jackson|Okmulgee County|OK|79583|United States|-6|single family| +15888|AAAAAAAAABODAAAA|627|Lake 5th|Way|Suite E|Fairview|Amherst County|VA|25709|United States|-5|single family| +15889|AAAAAAAABBODAAAA|211|3rd College|Lane|Suite M|Concord|Chase County|NE|64107|United States|-6|single family| +15890|AAAAAAAACBODAAAA|174|9th Spruce|Dr.|Suite L|The Meadows|Robeson County|NC|20026|United States|-5|single family| +15891|AAAAAAAADBODAAAA|882|Meadow |Court|Suite V|Providence|Prince George County|VA|26614|United States|-5|single family| +15892|AAAAAAAAEBODAAAA|117|Spring |Circle|Suite I|Greenfield|Washington County|RI|05638|United States|-5|apartment| +15893|AAAAAAAAFBODAAAA|843|West 2nd|Drive|Suite O|White Oak|Fergus County|MT|66668|United States|-7|apartment| +15894|AAAAAAAAGBODAAAA|377|Adams Oak|Circle|Suite 180|Georgetown|Anderson County|KY|47057|United States|-6|apartment| +15895|AAAAAAAAHBODAAAA|712|Williams 5th|Drive|Suite R|Greenville|Ray County|MO|61387|United States|-6|condo| +15896|AAAAAAAAIBODAAAA|510||RD||Hamilton||||||apartment| +15897|AAAAAAAAJBODAAAA|393|8th |Ave|Suite 50|Georgetown|Quitman County|GA|37057|United States|-5|apartment| +15898|AAAAAAAAKBODAAAA|312|Birch |Blvd|Suite H|Pine Grove|Moore County|TX|74593|United States|-6|single family| +15899|AAAAAAAALBODAAAA|678|Highland |Street|Suite G|Vance|Worth County|GA|30268|United States|-5|condo| +15900|AAAAAAAAMBODAAAA|7|Smith |Way|Suite M|Shiloh|Newport News city|VA|29275|United States|-5|condo| +15901|AAAAAAAANBODAAAA|179|River |Street|Suite 250|Jamestown|Hawkins County|TN|36867|United States|-5|apartment| +15902|AAAAAAAAOBODAAAA|440|Miller |Boulevard|Suite T|Valley View|Forsyth County|GA|35124|United States|-5|single family| +15903|AAAAAAAAPBODAAAA|901|Main |Parkway|Suite 460|Glenwood|Atkinson County|GA|33511|United States|-5|single family| +15904|AAAAAAAAACODAAAA|665|12th |RD|Suite Q|Oak Grove|Tom Green County|TX|78370|United States|-6|apartment| +15905|AAAAAAAABCODAAAA|924|Sixth Pine|Avenue|Suite 170|Sulphur Springs|Laurens County|SC|28354|United States|-5|apartment| +15906|AAAAAAAACCODAAAA||Main |||Hopewell|Lancaster County||20587|||apartment| +15907|AAAAAAAADCODAAAA|539|Twelfth |Lane|Suite 300|Pleasant Hill|Emery County|UT|83604|United States|-7|single family| +15908|AAAAAAAAECODAAAA|383|Hill |Blvd|Suite 220|Marion|York County|SC|20399|United States|-5|condo| +15909|AAAAAAAAFCODAAAA|590|Spring Highland|ST|Suite X|Hopewell|Baltimore city|MD|20587|United States|-5|condo| +15910|AAAAAAAAGCODAAAA|888|River |RD|Suite 280|Providence|Marlboro County|SC|26614|United States|-5|apartment| +15911|AAAAAAAAHCODAAAA|580|West |Road|Suite E|Kingston|Pipestone County|MN|54975|United States|-6|apartment| +15912|AAAAAAAAICODAAAA|980|Park Davis|Pkwy|Suite 10|Wilton|Randolph County|WV|26997|United States|-5|single family| +15913|AAAAAAAAJCODAAAA|478|Main |Road|Suite G|Shiloh|Talladega County|AL|39275|United States|-6|single family| +15914|AAAAAAAAKCODAAAA|826|Smith Franklin|Drive|Suite Y|Riverview|Defiance County|OH|49003|United States|-5|apartment| +15915|AAAAAAAALCODAAAA|305|Dogwood |Way|Suite 80|Kingston|Appanoose County|IA|54975|United States|-6|condo| +15916|AAAAAAAAMCODAAAA|54|Lee |Lane|Suite 370|Oakdale|Amador County|CA|99584|United States|-8|single family| +15917|AAAAAAAANCODAAAA|68|Highland |Circle|Suite I|Maple Grove|Hutchinson County|SD|58252|United States|-7|single family| +15918|AAAAAAAAOCODAAAA|736|Sycamore Chestnut|Ln|Suite 60|Green Acres|Smyth County|VA|27683|United States|-5|single family| +15919|AAAAAAAAPCODAAAA|296|First |Ct.|Suite 230|Arlington|Hardin County|TN|36557|United States|-5|condo| +15920|AAAAAAAAADODAAAA|603|Locust 13th|Cir.|||Dawes County|NE|68048|United States|-6|| +15921|AAAAAAAABDODAAAA|836|8th |Ct.|Suite 480|Unionville|Marathon County|WI|51711|United States|-6|apartment| +15922|AAAAAAAACDODAAAA|306|Second |Parkway|Suite Y|Jamestown|Polk County|NC|26867|United States|-5|single family| +15923|AAAAAAAADDODAAAA|653|Davis Valley|Ct.|Suite 260|Pleasant Hill|Kane County|IL|63604|United States|-6|condo| +15924|AAAAAAAAEDODAAAA|993|Birch Hill|Court|Suite 10|Providence|Seward County|KS|66614|United States|-6|single family| +15925|AAAAAAAAFDODAAAA|436|Davis |Way|Suite 270|Wilson|Goochland County|VA|26971|United States|-5|single family| +15926|AAAAAAAAGDODAAAA|398|Smith |Ln|Suite W|Lakeview|Holt County|MO|68579|United States|-6|apartment| +15927|AAAAAAAAHDODAAAA|917|4th |Drive|Suite 270|Salem|Bethel Census Area|AK|98048|United States|-9|apartment| +15928|AAAAAAAAIDODAAAA|333|Pine Jefferson|Pkwy|Suite N|Springfield|Barnes County|ND|59303|United States|-6|apartment| +15929|AAAAAAAAJDODAAAA|297|Walnut |Ct.|Suite 130|Friendship|Newaygo County|MI|44536|United States|-5|apartment| +15930|AAAAAAAAKDODAAAA|911|Third Elm|Court|Suite 440|Midway|Lafourche Parish|LA|71904|United States|-6|condo| +15931|AAAAAAAALDODAAAA|184|Washington Madison|RD|Suite F|Mount Pleasant|Perkins County|NE|61933|United States|-7|condo| +15932|AAAAAAAAMDODAAAA|342|Fifth ||Suite 370|Arlington||ID|86557|United States|-7|| +15933|AAAAAAAANDODAAAA|339|Sixth Cedar||Suite 150|||WI|||-6|single family| +15934|AAAAAAAAODODAAAA|973|11th |Way|Suite 260|Bethel|Shannon County|MO|65281|United States|-6|apartment| +15935|AAAAAAAAPDODAAAA|623|Spruce Main|Lane|Suite V|Deerfield|Kidder County|ND|59840|United States|-6|single family| +15936|AAAAAAAAAEODAAAA|259|Broadway |Ct.|Suite 450|Hopewell|Kanawha County|WV|20587|United States|-5|apartment| +15937|AAAAAAAABEODAAAA|681|Locust |Wy|Suite V|Macedonia|Crawford County|MI|41087|United States|-5|condo| +15938|AAAAAAAACEODAAAA|432|5th |Pkwy|Suite W|Greenwood|Saline County|KS|68828|United States|-6|apartment| +15939|AAAAAAAADEODAAAA|656|First 7th|Drive|Suite 490|Waterloo|Huntingdon County|PA|11675|United States|-5|condo| +15940|AAAAAAAAEEODAAAA|331|Broadway Cherry|RD|Suite W|Florence|Hillsborough County|NH|03994|United States|-5|single family| +15941|AAAAAAAAFEODAAAA|967|Sycamore |Ct.|Suite F|Westminster|Hall County|NE|66549|United States|-6|condo| +15942|AAAAAAAAGEODAAAA|688|Franklin |Parkway|Suite 120|Greenfield|Cortland County|NY|15038|United States|-5|apartment| +15943|AAAAAAAAHEODAAAA|205|Jackson |Court|Suite 130|Greenfield|Delaware County|PA|15038|United States|-5|single family| +15944|AAAAAAAAIEODAAAA|108|View |Ln|Suite U|Oakwood|Valley County|MT|60169|United States|-7|apartment| +15945|AAAAAAAAJEODAAAA|767|Broadway |RD|Suite T|Oak Hill|Washington County|VT|08438|United States|-5|single family| +15946|AAAAAAAAKEODAAAA|704|Lee Green|Ct.|Suite L|Newport|Iredell County|NC|21521|United States|-5|apartment| +15947|AAAAAAAALEODAAAA|344|Twelfth |Street|Suite 480|Georgetown|Marion County|IL|67057|United States|-6|apartment| +15948|AAAAAAAAMEODAAAA|386|5th |Ln|Suite 440|Waterloo|Washington County|NC|21675|United States|-5|apartment| +15949|AAAAAAAANEODAAAA|686|Jefferson 11th|Drive|Suite 300|Clinton|Jones County|TX|78222|United States|-6|single family| +15950|AAAAAAAAOEODAAAA|73|College |Way|Suite J|Red Hill|Nelson County|VA|24338|United States|-5|apartment| +15951|AAAAAAAAPEODAAAA|673|12th Forest|Ln|Suite 320|Mount Zion|Los Angeles County|CA|98054|United States|-8|single family| +15952|AAAAAAAAAFODAAAA|621|Maple 9th|Drive|Suite 20|Pine Grove|Sumner County|TN|34593|United States|-6|single family| +15953|AAAAAAAABFODAAAA|181|Willow Davis|Dr.|Suite Y|White Hall|Clarke County|GA|36955|United States|-5|single family| +15954|AAAAAAAACFODAAAA|87|Williams |Cir.|Suite U|Oakwood|Bernalillo County|NM|80169|United States|-7|condo| +15955|AAAAAAAADFODAAAA|811|Walnut Elm|Cir.|Suite 30|Arlington|Blaine County|MT|66557|United States|-7|apartment| +15956|AAAAAAAAEFODAAAA|314|Park Maple|RD|Suite 490|Lakeview|Hardy County|WV|28579|United States|-5|single family| +15957|AAAAAAAAFFODAAAA|605|Smith |Road|Suite W|Doyle|Scotland County|NC|28434|United States|-5|apartment| +15958|AAAAAAAAGFODAAAA|87|Park |Circle|Suite X|Pleasant Hill|Rockingham County|NH|04204|United States|-5|single family| +15959|AAAAAAAAHFODAAAA|68|Spring |Pkwy|Suite 40|Macedonia|Sherman County|NE|61087|United States|-7|condo| +15960|AAAAAAAAIFODAAAA|904|Adams |Cir.|Suite D|Highland Park|Jefferson County|MO|66534|United States|-6|apartment| +15961|AAAAAAAAJFODAAAA|54|10th Mill|Ln|Suite V|Mount Olive|Rogers County|OK|78059|United States|-6|single family| +15962|AAAAAAAAKFODAAAA|890|Oak Wilson|Ct.|Suite 280|Springdale|Wilson County|KS|68883|United States|-6|condo| +15963|AAAAAAAALFODAAAA|309|Elm |Street|Suite M|Valley View|Golden Valley County|MT|65124|United States|-7|apartment| +15964|AAAAAAAAMFODAAAA|508|Ridge Hill|Boulevard|Suite 40|Highland Park|Perry County|OH|46534|United States|-5|condo| +15965|AAAAAAAANFODAAAA|868|Elm |Ln|Suite D|Newport|Taylor County|TX|71521|United States|-6|condo| +15966|AAAAAAAAOFODAAAA|847|4th Forest|Court|Suite B|Oakdale|Pawnee County|OK|79584|United States|-6|condo| +15967|AAAAAAAAPFODAAAA||Williams ||Suite D||||||-6|condo| +15968|AAAAAAAAAGODAAAA|167|Fourteenth Madison|Ln|Suite R|Glendale|Hancock County|MS|53951|United States|-6|apartment| +15969|AAAAAAAABGODAAAA|439|5th Oak|||Woodlawn||IN||||| +15970|AAAAAAAACGODAAAA|14|Franklin |Circle|Suite B|Woodville|Sabine County|TX|74289|United States|-6|apartment| +15971|AAAAAAAADGODAAAA|408|Forest Church|Court|Suite 140|Newport|Walworth County|SD|51521|United States|-7|apartment| +15972|AAAAAAAAEGODAAAA|81|Spring |Lane|Suite F|Forest Hills|Brown County|MN|59237|United States|-6|single family| +15973|AAAAAAAAFGODAAAA|346|Hill Locust|Court|Suite C|Maple Hill|Sequoyah County|OK|78095|United States|-6|single family| +15974|AAAAAAAAGGODAAAA|762|Madison Park|RD|Suite Q|Riverdale|Northumberland County|VA|29391|United States|-5|apartment| +15975|AAAAAAAAHGODAAAA|918|Maple Meadow|Cir.|Suite R|Antioch|Allegan County|MI|48605|United States|-5|condo| +15976|AAAAAAAAIGODAAAA|11|Oak |Court|Suite M|Enterprise|Steuben County|NY|11757|United States|-5|single family| +15977|AAAAAAAAJGODAAAA|832|Central |Boulevard|Suite 260|Oakwood|Boone County|AR|70169|United States|-6|condo| +15978|AAAAAAAAKGODAAAA|821|Poplar |Avenue|Suite 60|Springfield|Lafayette County|MO|69303|United States|-6|apartment| +15979|AAAAAAAALGODAAAA|130|Third 6th|Lane|Suite 270|Whitney|Wilson County|NC|28339|United States|-5|apartment| +15980|AAAAAAAAMGODAAAA|129|Walnut 6th|Ct.|Suite 220|Greenwood|Palm Beach County|FL|38828|United States|-5|apartment| +15981|AAAAAAAANGODAAAA|766|Ridge |Court|Suite L|Belmont|Lynn County|TX|70191|United States|-6|single family| +15982|AAAAAAAAOGODAAAA|344|Locust |Street|Suite 0|Waterloo|White County|GA|31675|United States|-5|single family| +15983|AAAAAAAAPGODAAAA|314|Walnut First|Dr.|Suite 120|Pleasant Valley|Camden County|GA|32477|United States|-5|apartment| +15984|AAAAAAAAAHODAAAA|902|Washington 9th|Wy|Suite 160|Greenfield|Guadalupe County|NM|85038|United States|-7|apartment| +15985|AAAAAAAABHODAAAA|360|First |Avenue|Suite 170|Plainview|Rogers County|OK|73683|United States|-6|condo| +15986|AAAAAAAACHODAAAA|254|Fourth Spruce|Way|Suite J|Spring Valley|Orange County|VA|26060|United States|-5|single family| +15987|AAAAAAAADHODAAAA|491|Green Sunset|Cir.|Suite N|Highland|Oliver County|ND|59454|United States|-6|apartment| +15988|AAAAAAAAEHODAAAA|864|East |Parkway|Suite H|Pleasant Grove|Augusta County|VA|24136|United States|-5|apartment| +15989|AAAAAAAAFHODAAAA||||Suite 320|||LA|70191|||| +15990|AAAAAAAAGHODAAAA|135|Lakeview |Ln|Suite 10|Cedar Grove|Sublette County|WY|80411|United States|-7|condo| +15991|AAAAAAAAHHODAAAA|63|Center |Blvd|Suite 470|Lebanon|Winston County|MS|52898|United States|-6|apartment| +15992|AAAAAAAAIHODAAAA|233|Franklin 5th|Court|Suite 340|Newtown|Musselshell County|MT|61749|United States|-7|condo| +15993|AAAAAAAAJHODAAAA|378|Sunset 2nd|Dr.|Suite 190|Blanchard|Poquoson city|VA|25985|United States|-5|apartment| +15994|AAAAAAAAKHODAAAA|557|2nd |RD|Suite H|Pleasant Valley|Warren County|NJ|03077|United States|-5|apartment| +15995|AAAAAAAALHODAAAA|992|Elm |Blvd|Suite D|Concord|Clarke County|MS|54107|United States|-6|apartment| +15996|AAAAAAAAMHODAAAA|513|Valley |Avenue|Suite W|Macedonia|Davie County|NC|21087|United States|-5|single family| +15997|AAAAAAAANHODAAAA|243|East |Road|Suite 360|Brentwood|Jasper County|SC|24188|United States|-5|single family| +15998|AAAAAAAAOHODAAAA|704|Johnson ||||Fremont County|WY|||-7|single family| +15999|AAAAAAAAPHODAAAA|997|Laurel Hickory|RD|Suite M|Harmony|Wexford County|MI|45804|United States|-5|apartment| +16000|AAAAAAAAAIODAAAA|733|Jackson 4th|Way|Suite T|Forest Hills|Hancock County|WV|29237|United States|-5|apartment| +16001|AAAAAAAABIODAAAA|500|Ridge Seventh|Avenue|Suite U|Lebanon|Hockley County|TX|72898|United States|-6|condo| +16002|AAAAAAAACIODAAAA|878|6th Miller|Blvd|Suite B|Harmony|Snyder County|PA|15804|United States|-5|condo| +16003|AAAAAAAADIODAAAA|615|Forest |Ln|Suite U|Walnut Grove|Woods County|OK|77752|United States|-6|single family| +16004|AAAAAAAAEIODAAAA|487|Second Second|Ct.|Suite 410|Belmont|Denver County|CO|80191|United States|-7|condo| +16005|AAAAAAAAFIODAAAA|649|Birch |Circle|Suite H|Shiloh|Pierce County|GA|39275|United States|-5|single family| +16006|AAAAAAAAGIODAAAA|620|Center 7th|Road|Suite 380|Lakeside|Saline County|MO|69532|United States|-6|condo| +16007|AAAAAAAAHIODAAAA|998|12th |Blvd|Suite V|Forest Hills|Montgomery County|GA|39237|United States|-5|condo| +16008|AAAAAAAAIIODAAAA|903|Park |Road|Suite F|Spring Valley|Ford County|IL|66060|United States|-6|condo| +16009|AAAAAAAAJIODAAAA|626|2nd Washington|Cir.||Union|Montgomery County|VA||||condo| +16010|AAAAAAAAKIODAAAA|807|Wilson |Dr.|Suite 340|Silver Springs|Stoddard County|MO|64843|United States|-6|condo| +16011|AAAAAAAALIODAAAA|422|Hickory Mill|Cir.|Suite 220|Crossroads|Douglas County|MO|60534|United States|-6|condo| +16012|AAAAAAAAMIODAAAA|688|2nd Jackson|Court|Suite 240|Oakland|Christian County|IL|69843|United States|-6|condo| +16013|AAAAAAAANIODAAAA|871|7th First|Avenue|Suite Q|Providence|Hardeman County|TX|76614|United States|-6|single family| +16014|AAAAAAAAOIODAAAA|548|Dogwood |Way|Suite Y|Newtown|Real County|TX|71749|United States|-6|apartment| +16015|AAAAAAAAPIODAAAA|462|Ash Hickory|Ct.|Suite 220|Pleasant Hill|Jefferson County|AL|33604|United States|-6|condo| +16016|AAAAAAAAAJODAAAA|107|2nd Mill|Ave|Suite O|Glenwood|Cleveland County|OK|73511|United States|-6|apartment| +16017|AAAAAAAABJODAAAA|193|Walnut Sixth|Pkwy|Suite 120|Enterprise|Lincoln County|NC|21757|United States|-5|condo| +16018|AAAAAAAACJODAAAA|530|Adams |Drive|Suite H|Farmington|Renville County|MN|59145|United States|-6|apartment| +16019|AAAAAAAADJODAAAA|912|Davis Davis|Parkway|Suite 200|Bridgeport|Fountain County|IN|45817|United States|-5|condo| +16020|AAAAAAAAEJODAAAA|428|Pine Main|Pkwy|Suite G|Buena Vista|Inyo County|CA|95752|United States|-8|condo| +16021|AAAAAAAAFJODAAAA|245|2nd |Ave|Suite O|Georgetown|Indiana County|PA|17057|United States|-5|single family| +16022|AAAAAAAAGJODAAAA|419|Franklin Park|Boulevard|Suite J|Lincoln|White Pine County|NV|81289|United States|-8|apartment| +16023|AAAAAAAAHJODAAAA|287|Smith |Lane|Suite 180|Waterloo|Burleson County|TX|71675|United States|-6|apartment| +16024|AAAAAAAAIJODAAAA|557|5th 6th|Ave|Suite A|Red Hill|Clay County|MN|54338|United States|-6|single family| +16025|AAAAAAAAJJODAAAA|113|6th |Wy|Suite 30|Greenfield|Big Stone County|MN|55038|United States|-6|condo| +16026|AAAAAAAAKJODAAAA|297|Hill |Road|Suite 390|Pleasant Hill|Butte County|CA|93604|United States|-8|single family| +16027|AAAAAAAALJODAAAA|845||Ct.||Pleasant Valley||||United States||| +16028|AAAAAAAAMJODAAAA|241|Adams First|Street|Suite U|Plainview|Addison County|VT|04283|United States|-5|single family| +16029|AAAAAAAANJODAAAA|105|4th Lakeview|Way|Suite 480|Providence|Johnston County|OK|76614|United States|-6|condo| +16030|AAAAAAAAOJODAAAA|||Wy||Green Acres||SC|27683|United States||apartment| +16031|AAAAAAAAPJODAAAA|229|Park |Drive|Suite 130|Hopewell|Keokuk County|IA|50587|United States|-6|condo| +16032|AAAAAAAAAKODAAAA|461|10th Adams|Street|Suite 420|Newtown|Lowndes County|GA|31749|United States|-5|apartment| +16033|AAAAAAAABKODAAAA|296|Church |Dr.|Suite L|Plainview|Noble County|OK|73683|United States|-6|single family| +16034|AAAAAAAACKODAAAA|264|Center Hickory|Ct.|Suite 150|Providence|Steele County|ND|56614|United States|-6|apartment| +16035|AAAAAAAADKODAAAA|841|Laurel |Street|Suite 340|Lincoln|Holmes County|MS|51289|United States|-6|condo| +16036|AAAAAAAAEKODAAAA|870|Franklin 1st|Ct.|Suite V|Riley|Lincoln County|WA|91692|United States|-8|single family| +16037|AAAAAAAAFKODAAAA|347|Willow |Ct.|Suite H|Pine Grove|Henry County|IA|54593|United States|-6|apartment| +16038|AAAAAAAAGKODAAAA|181|College Dogwood|Circle|Suite 270|Summit|Ocean County|NJ|01099|United States|-5|condo| +16039|AAAAAAAAHKODAAAA|95|7th 7th|Boulevard|Suite H|Enterprise|Grant County|NM|81757|United States|-7|single family| +16040|AAAAAAAAIKODAAAA|837|Park |ST|Suite 250|Brownsville|Menifee County|KY|49310|United States|-5|apartment| +16041|AAAAAAAAJKODAAAA|669|River |Drive|Suite 290|Arlington|Martin County|MN|56557|United States|-6|single family| +16042|AAAAAAAAKKODAAAA|49|Pine Dogwood|Court|Suite 100|Salem|Summers County|WV|28048|United States|-5|single family| +16043|AAAAAAAALKODAAAA|302|Central |Circle|Suite O|Forest Park|Eagle County|CO|84317|United States|-7|single family| +16044|AAAAAAAAMKODAAAA|396|Locust Second|Boulevard|Suite 70|Murphy|Linn County|OR|92105|United States|-8|apartment| +16045|AAAAAAAANKODAAAA|671|Hickory |Court|Suite I|Pleasant Valley|Twin Falls County|ID|82477|United States|-7|apartment| +16046|AAAAAAAAOKODAAAA|983|Miller 8th|Ln|Suite H|Pinecrest|Brooks County|TX|79981|United States|-6|single family| +16047|AAAAAAAAPKODAAAA|461|Lake Birch|Street|Suite S|Florence|Norton city|VA|23394|United States|-5|condo| +16048|AAAAAAAAALODAAAA|926|Central |Cir.|Suite 350|Valley View|Hamilton County|KS|65124|United States|-6|condo| +16049|AAAAAAAABLODAAAA|326|1st Oak|Cir.|Suite P|Friendship|Sierra County|CA|94536|United States|-8|single family| +16050|AAAAAAAACLODAAAA|552|6th Hill|RD|Suite 80|Red Oak|Mason County|TX|75018|United States|-6|apartment| +16051|AAAAAAAADLODAAAA|396|Second Thirteenth|Drive|Suite Y|Cedar Grove|Smith County|MS|50411|United States|-6|apartment| +16052|AAAAAAAAELODAAAA|294|East |Avenue|Suite W|Pleasant Valley|Dyer County|TN|32477|United States|-5|condo| +16053|AAAAAAAAFLODAAAA|912|Railroad Willow|Ln|Suite G|Franklin|Cortland County|NY|19101|United States|-5|single family| +16054|AAAAAAAAGLODAAAA|318|Pine View|Dr.|Suite 200|Harmony|Valley County|NE|65804|United States|-7|condo| +16055|AAAAAAAAHLODAAAA|35|Main |Court|Suite 100|Sulphur Springs|Gogebic County|MI|48354|United States|-5|single family| +16056|AAAAAAAAILODAAAA|431|Main Spruce|Way|Suite 330|Sunnyside|Sheridan County|WY|81952|United States|-7|condo| +16057|AAAAAAAAJLODAAAA|946|Fourth Elm|Dr.|Suite R|Clifton|Carroll County|NH|08614|United States|-5|single family| +16058|AAAAAAAAKLODAAAA|650|Lincoln Washington|Ln|Suite X|Unionville|Wilson County|TX|71711|United States|-6|single family| +16059|AAAAAAAALLODAAAA|338|Poplar Jackson|Ct.|Suite 280|Fairview|Wyandot County|OH|45709|United States|-5|condo| +16060|AAAAAAAAMLODAAAA|731|Hill |Way|Suite 170|Cedar Grove|Hardin County|KY|40411|United States|-6|condo| +16061|AAAAAAAANLODAAAA|282|Church 13th|Road|Suite F|Oak Grove|Fulton County|IN|48370|United States|-5|condo| +16062|AAAAAAAAOLODAAAA|864|Ridge |Lane|Suite 470|Georgetown|Falls County|TX|77057|United States|-6|condo| +16063|AAAAAAAAPLODAAAA|424|Forest |Lane|Suite 230|Bunker Hill|Phillips County|MT|60150|United States|-7|single family| +16064|AAAAAAAAAMODAAAA|893|Highland Main|RD|Suite W|Unionville|Preble County|OH|41711|United States|-5|single family| +16065|AAAAAAAABMODAAAA|231|South Washington|Road|Suite V|Bath|Henderson County|IL|60573|United States|-6|apartment| +16066|AAAAAAAACMODAAAA|314|West Green|Street|Suite H|Florence|Inyo County|CA|93394|United States|-8|condo| +16067|AAAAAAAADMODAAAA|||Dr.|Suite 290||||84338|United States|-7|apartment| +16068|AAAAAAAAEMODAAAA|790|Smith Chestnut|Pkwy|Suite H|Lakeside|Sterling County|TX|79532|United States|-6|single family| +16069|AAAAAAAAFMODAAAA|780|Washington |Dr.|Suite D|Concord|Chattahoochee County|GA|34107|United States|-5|apartment| +16070|AAAAAAAAGMODAAAA|606|Main Pine|Wy|Suite C|Maple Grove|Saline County|IL|68252|United States|-6|single family| +16071|AAAAAAAAHMODAAAA|825|Valley |Ave|Suite 360|Stony Point|Monroe County|IL|64255|United States|-6|apartment| +16072|AAAAAAAAIMODAAAA|823|Meadow Willow|Dr.|Suite 450|Spring Valley|Barbour County|AL|36060|United States|-6|condo| +16073|AAAAAAAAJMODAAAA|661|2nd |Street|Suite 320|Bethel|Lewis County|WA|95281|United States|-8|apartment| +16074|AAAAAAAAKMODAAAA|890|Maple Miller|Parkway|Suite 420|Waterloo|Hickman County|KY|41675|United States|-6|apartment| +16075|AAAAAAAALMODAAAA|635|Sunset |ST|Suite H|Hamilton|Androscoggin County|ME|03408|United States|-5|condo| +16076|AAAAAAAAMMODAAAA|395|Dogwood |Ln|Suite 270|Hillcrest|Bon Homme County|SD|53003|United States|-6|apartment| +16077|AAAAAAAANMODAAAA|277|Johnson |Way|Suite L|Hopewell|Throckmorton County|TX|70587|United States|-6|condo| +16078|AAAAAAAAOMODAAAA|434|Tenth Second|Boulevard|Suite 30|Salem|Dubois County|IN|48048|United States|-5|apartment| +16079|AAAAAAAAPMODAAAA|197|13th |Drive|Suite Y|Lone Pine|Tioga County|PA|17441|United States|-5|apartment| +16080|AAAAAAAAANODAAAA|511|Mill Cedar|Boulevard|Suite B|Sulphur Springs|Delaware County|PA|18354|United States|-5|condo| +16081|AAAAAAAABNODAAAA|228|4th |Circle|Suite 330|Union Hill|Smith County|KS|67746|United States|-6|single family| +16082|AAAAAAAACNODAAAA|490|Center |Lane|Suite K|Five Forks|Saluda County|SC|22293|United States|-5|single family| +16083|AAAAAAAADNODAAAA|185|Lake Cedar|Dr.|Suite U|Pleasant Grove|Stewart County|TN|34136|United States|-6|apartment| +16084|AAAAAAAAENODAAAA|650|Oak |Circle|Suite 470|Oakwood|Wicomico County|MD|20169|United States|-5|apartment| +16085|AAAAAAAAFNODAAAA|122|Railroad Spruce|Drive|Suite O|Concord|Kearny County|KS|64107|United States|-6|condo| +16086|AAAAAAAAGNODAAAA|513|Elm |Pkwy|Suite W|Riverview|Clinton County|MI|49003|United States|-5|condo| +16087|AAAAAAAAHNODAAAA|55|Lake |ST|Suite 150|Concord|Moody County|SD|54107|United States|-7|single family| +16088|AAAAAAAAINODAAAA|742|Adams Williams|Wy|Suite E|Midway|Salt Lake County|UT|81904|United States|-7|apartment| +16089|AAAAAAAAJNODAAAA|99|Smith Smith|Parkway|Suite Y|Concord|Edgecombe County|NC|24107|United States|-5|condo| +16090|AAAAAAAAKNODAAAA|739|Fourth Church|Lane|Suite 200|Union Hill|Marion County|GA|37746|United States|-5|apartment| +16091|AAAAAAAALNODAAAA|365|Hillcrest Valley|Way|Suite 80|Walnut Grove|Kingfisher County|OK|77752|United States|-6|apartment| +16092|AAAAAAAAMNODAAAA|60|15th |RD|Suite J|Lakeside|Morgan County|KY|49532|United States|-5|condo| +16093|AAAAAAAANNODAAAA|754|Mill Spring|Pkwy|Suite 40|Lakeview|Itasca County|MN|58579|United States|-6|condo| +16094|AAAAAAAAONODAAAA|1|Wilson |Boulevard|Suite 330|Greenfield|Wilkes County|NC|25038|United States|-5|condo| +16095|AAAAAAAAPNODAAAA|296|Park |Pkwy|Suite 410|Spring Valley|Morgan County|KY|46060|United States|-5|condo| +16096|AAAAAAAAAOODAAAA|248|Mill Highland|Court|Suite L|Deerfield|Fayette County|PA|19840|United States|-5|condo| +16097|AAAAAAAABOODAAAA|922|1st View|Parkway|Suite H|Deerfield|Roseau County|MN|59840|United States|-6|single family| +16098|AAAAAAAACOODAAAA|872|9th |Road|Suite O|Woodland|Benton County|MO|64854|United States|-6|apartment| +16099|AAAAAAAADOODAAAA|259|Park |Street|Suite 330|Oak Hill|Walton County|FL|37838|United States|-5|condo| +16100|AAAAAAAAEOODAAAA|123|Center 4th|Lane|Suite O|Riverdale|Fluvanna County|VA|29391|United States|-5|apartment| +16101|AAAAAAAAFOODAAAA|974|Railroad |Parkway|Suite 340|Cedar Grove|Washington County|MS|50411|United States|-6|condo| +16102|AAAAAAAAGOODAAAA|239|8th Franklin|Cir.|Suite U|Riverside|Jackson County|CO|89231|United States|-7|single family| +16103|AAAAAAAAHOODAAAA|199|Center |Pkwy|Suite O|Lakeview|Centre County|PA|18579|United States|-5|apartment| +16104|AAAAAAAAIOODAAAA|442|Washington |Boulevard|Suite X|Riverdale|Moffat County|CO|89391|United States|-7|condo| +16105|AAAAAAAAJOODAAAA|80|View Hillcrest|Dr.|Suite N|Harmony|Patrick County|VA|25804|United States|-5|condo| +16106|AAAAAAAAKOODAAAA|437|Lake Laurel|Boulevard|Suite L|Shiloh|Northwest Arctic Borough|AK|99275|United States|-9|apartment| +16107|AAAAAAAALOODAAAA|111|Maple |Street|Suite 10|New Hope|Douglas County|MN|59431|United States|-6|apartment| +16108|AAAAAAAAMOODAAAA|99|North |Ct.|Suite 480|Greenwood|Union County|OH|48828|United States|-5|single family| +16109|AAAAAAAANOODAAAA|221|9th |Parkway|Suite 100|Forest Hills|Calhoun County|MS|59237|United States|-6|apartment| +16110|AAAAAAAAOOODAAAA|901|River River|Dr.|Suite I|Salem|Weber County|UT|88048|United States|-7|single family| +16111|AAAAAAAAPOODAAAA|318|Locust |Lane|Suite K|Macedonia|Armstrong County|PA|11087|United States|-5|apartment| +16112|AAAAAAAAAPODAAAA|812|Church |Dr.|Suite 490|Pleasant Valley|Ballard County|KY|42477|United States|-6|condo| +16113|AAAAAAAABPODAAAA|292|Maple |Street|Suite U|Riverside|Ventura County|CA|99231|United States|-8|condo| +16114|AAAAAAAACPODAAAA|95|Railroad Oak|ST|Suite F|Spring Valley|Kendall County|IL|66060|United States|-6|apartment| +16115|AAAAAAAADPODAAAA|139|First West|Drive|Suite 80|Newport|Jack County|TX|71521|United States|-6|single family| +16116|AAAAAAAAEPODAAAA|943|Second North|Wy|Suite 170|Sleepy Hollow|Tangipahoa Parish|LA|73592|United States|-6|condo| +16117|AAAAAAAAFPODAAAA|306|Sixth Broadway|Dr.|Suite V|Clinton|Unicoi County|TN|38222|United States|-6|apartment| +16118|AAAAAAAAGPODAAAA|429|Birch |Pkwy|Suite V|Edgewood|Ellis County|KS|60069|United States|-6|apartment| +16119|AAAAAAAAHPODAAAA|640|Davis Lake|Blvd|Suite J|Forest Hills|De Kalb County|IN|49237|United States|-5|single family| +16120|AAAAAAAAIPODAAAA|358|Miller 8th|Drive|Suite G|Ashland|Mason County|WA|94244|United States|-8|condo| +16121|AAAAAAAAJPODAAAA|153|Third Washington|Blvd|Suite 270|Glenwood|Monroe County|MO|63511|United States|-6|apartment| +16122|AAAAAAAAKPODAAAA|758|Sunset Smith|Wy|Suite S|Fairfield|Fresno County|CA|96192|United States|-8|single family| +16123|AAAAAAAALPODAAAA|159|Lee |Court|Suite 240|New Hope|Trigg County|KY|49431|United States|-5|apartment| +16124|AAAAAAAAMPODAAAA|101|Washington |Ave|Suite C|Plainview|Beaver County|OK|73683|United States|-6|apartment| +16125|AAAAAAAANPODAAAA|217|Spring 2nd|Wy|Suite G|Oakdale|Faulk County|SD|59584|United States|-6|condo| +16126|AAAAAAAAOPODAAAA|375|7th |Ave|Suite 340|Spring Hill|Lawrence County|AL|36787|United States|-6|condo| +16127|AAAAAAAAPPODAAAA|505|Third Williams|RD|Suite W|Pleasant Grove|Clinton County|MO|64136|United States|-6|apartment| +16128|AAAAAAAAAAPDAAAA|61|5th Pine|Road|Suite E|The Meadows|Berkeley County|SC|20026|United States|-5|condo| +16129|AAAAAAAABAPDAAAA|506|Woodland |Court|Suite 170|Oakwood|Powhatan County|VA|20169|United States|-5|single family| +16130|AAAAAAAACAPDAAAA|562|Pine |Pkwy|Suite M|Oakwood|Otoe County|NE|60169|United States|-7|apartment| +16131|AAAAAAAADAPDAAAA|859|Main 6th|Court|Suite 290|Liberty|Calhoun County|GA|33451|United States|-5|single family| +16132|AAAAAAAAEAPDAAAA|628|7th |Blvd|Suite 310|Concord|Richmond County|VA|24107|United States|-5|apartment| +16133|AAAAAAAAFAPDAAAA|284|Second Park|ST|Suite I|Richville|McIntosh County|ND|55945|United States|-6|single family| +16134|AAAAAAAAGAPDAAAA|159|10th |Pkwy|Suite 290|Liberty|Cass County|MI|43451|United States|-5|apartment| +16135|AAAAAAAAHAPDAAAA|821|West South|Way|Suite 340|California|Windsor County|VT|00741|United States|-5|condo| +16136|AAAAAAAAIAPDAAAA|948|Ridge 10th|Cir.|Suite Q|Lakewood|Knox County|OH|48877|United States|-5|apartment| +16137|AAAAAAAAJAPDAAAA|616|Spring Lee|Lane|Suite 390|Glenwood|Baltimore city|MD|23511|United States|-5|apartment| +16138|AAAAAAAAKAPDAAAA|805|Wilson Cherry|Dr.|Suite 420|Salem|Nacogdoches County|TX|78048|United States|-6|single family| +16139|AAAAAAAALAPDAAAA|5|Third |Boulevard|Suite Y|Shiloh|Presque Isle County|MI|49275|United States|-5|condo| +16140|AAAAAAAAMAPDAAAA|630|4th |Ln|Suite 30|Arlington|Hampshire County|MA|07157|United States|-5|single family| +16141|AAAAAAAANAPDAAAA|702|Miller |Boulevard|Suite X|Kelly|Chenango County|NY|12738|United States|-5|apartment| +16142|AAAAAAAAOAPDAAAA|636|Pine Valley|Pkwy|Suite H|Arlington|Sullivan County|TN|36557|United States|-6|apartment| +16143|AAAAAAAAPAPDAAAA|964|15th Third|ST|Suite E|Riverview|Pettis County|MO|69003|United States|-6|apartment| +16144|AAAAAAAAABPDAAAA|503|Ash Cherry|Ave|Suite G|Unionville|Brown County|IN|41711|United States|-5|condo| +16145|AAAAAAAABBPDAAAA|348|Twelfth |Wy|Suite 430|Marion|Cowlitz County|WA|90399|United States|-8|single family| +16146|AAAAAAAACBPDAAAA|914|Elm |RD|Suite U|Antioch|Billings County|ND|58605|United States|-6|condo| +16147|AAAAAAAADBPDAAAA|7|Main Pine|Court|Suite 460|Deerfield|Boone County|IN|49840|United States|-5|single family| +16148|AAAAAAAAEBPDAAAA||||||||41087|||| +16149|AAAAAAAAFBPDAAAA|788|Lake |Dr.|Suite 470|Hillcrest|Sanborn County|SD|53003|United States|-7|single family| +16150|AAAAAAAAGBPDAAAA|666|View Miller|Circle|Suite J|Shiloh|Rosebud County|MT|69275|United States|-7|condo| +16151|AAAAAAAAHBPDAAAA|150|6th |Way|Suite 200|Richville|Kerr County|TX|75945|United States|-6|apartment| +16152|AAAAAAAAIBPDAAAA|805|6th Woodland|Boulevard|Suite 200|Mount Pleasant|West Carroll Parish|LA|71933|United States|-6|single family| +16153|AAAAAAAAJBPDAAAA|457|Third 1st|Ct.|Suite 210|Antioch|El Dorado County|CA|98605|United States|-8|single family| +16154|AAAAAAAAKBPDAAAA|625|14th |ST|Suite W|Whitesville|Jackson County|AL|35903|United States|-6|condo| +16155|AAAAAAAALBPDAAAA|268|Tenth Forest|Lane|Suite 140|Scottsville|Gwinnett County|GA|34190|United States|-5|single family| +16156|AAAAAAAAMBPDAAAA|313|8th |Wy|Suite 230|Plainview|Logan County|OH|43683|United States|-5|single family| +16157|AAAAAAAANBPDAAAA|704|Mill Lakeview|Avenue|Suite 400|Hillcrest|Uvalde County|TX|73003|United States|-6|apartment| +16158|AAAAAAAAOBPDAAAA|454|Valley Park|Boulevard|Suite R|Fairfield|Fort Bend County|TX|76192|United States|-6|condo| +16159|AAAAAAAAPBPDAAAA||South ||Suite 210|Centerville||KS|||-6|condo| +16160|AAAAAAAAACPDAAAA|655|Sycamore |Dr.|Suite Y|Maple Hill|Wilbarger County|TX|78095|United States|-6|single family| +16161|AAAAAAAABCPDAAAA|20|Cedar Third|Boulevard|Suite N|Marion|Washburn County|WI|50399|United States|-6|condo| +16162|AAAAAAAACCPDAAAA|727|1st 2nd|Wy|Suite L|Pleasant Grove|Lewis County|NY|14136|United States|-5|apartment| +16163|AAAAAAAADCPDAAAA|268|East Hillcrest|Boulevard|Suite N|Wildwood|Custer County|ID|86871|United States|-7|condo| +16164|AAAAAAAAECPDAAAA|275|5th |Circle|Suite 170|Buena Vista|Atlantic County|NJ|06352|United States|-5|apartment| +16165|AAAAAAAAFCPDAAAA|596|Sunset Main|Court|Suite 130|Riverside|Athens County|OH|49231|United States|-5|condo| +16166|AAAAAAAAGCPDAAAA|11|Central Wilson|Cir.|Suite C|Georgetown|Wilcox County|AL|37057|United States|-6|condo| +16167|AAAAAAAAHCPDAAAA|927|Railroad |Cir.|Suite 410|Montezuma|Big Stone County|MN|52150|United States|-6|single family| +16168|AAAAAAAAICPDAAAA|668|Fourth Oak|Street|Suite 150|Fisher|Clay County|SD|52819|United States|-6|apartment| +16169|AAAAAAAAJCPDAAAA|846|Mill |Boulevard|Suite U|Oak Ridge|Tom Green County|TX|78371|United States|-6|condo| +16170|AAAAAAAAKCPDAAAA|276|6th |ST|Suite 390|Highland Park|Prairie County|AR|76534|United States|-6|condo| +16171|AAAAAAAALCPDAAAA|705|5th Spring|Pkwy|Suite 230|Springdale|Barnwell County|SC|28883|United States|-5|single family| +16172|AAAAAAAAMCPDAAAA|616|12th |Circle|Suite 390|West Liberty|Rockingham County|NH|05352|United States|-5|condo| +16173|AAAAAAAANCPDAAAA|89|Woodland Ridge|Drive|Suite F|Greenwood|Butler County|KS|68828|United States|-6|single family| +16174|AAAAAAAAOCPDAAAA|634|Third Madison|Wy|Suite 250|Fairview|Hancock County|ME|06309|United States|-5|single family| +16175|AAAAAAAAPCPDAAAA|312|Hill Wilson|Boulevard|Suite G|Mount Olive|Madison County|IN|48059|United States|-5|single family| +16176|AAAAAAAAADPDAAAA|551|8th |Wy|Suite J|Mountain View|Potter County|PA|14466|United States|-5|single family| +16177|AAAAAAAABDPDAAAA|604|Davis 3rd|Parkway|Suite E|Lebanon|Loudon County|TN|32898|United States|-6|apartment| +16178|AAAAAAAACDPDAAAA|64|2nd Railroad|Court|Suite X|Green Acres|Teller County|CO|87683|United States|-7|single family| +16179|AAAAAAAADDPDAAAA|445|Smith |Road|Suite A|Cedar Grove|Iroquois County|IL|60411|United States|-6|apartment| +16180|AAAAAAAAEDPDAAAA|901|Park |Way|Suite 480|Shady Grove|Clinton County|OH|42812|United States|-5|single family| +16181|AAAAAAAAFDPDAAAA|959|Sixth |RD|Suite 100|Highland Park|Yellowstone County|MT|66534|United States|-7|single family| +16182|AAAAAAAAGDPDAAAA|649|Church |Pkwy|Suite 100|Green Acres|Lincoln County|AR|77683|United States|-6|apartment| +16183|AAAAAAAAHDPDAAAA|329|Washington 9th|Parkway|Suite 370|Oak Grove|Lenoir County|NC|28370|United States|-5|single family| +16184|AAAAAAAAIDPDAAAA|192|1st Walnut|Court|Suite O|Arlington|Lyman County|SD|56557|United States|-7|single family| +16185|AAAAAAAAJDPDAAAA|529|Willow North|ST|Suite V|Franklin|Beaver County|UT|89101|United States|-7|condo| +16186|AAAAAAAAKDPDAAAA|483|3rd |Dr.|Suite 380|Five Forks|Johnson County|TX|72293|United States|-6|single family| +16187|AAAAAAAALDPDAAAA|713|7th |ST|Suite D|Mount Pleasant|Miller County|AR|71933|United States|-6|condo| +16188|AAAAAAAAMDPDAAAA|770|Highland Dogwood|ST|Suite E|Lewisburg|Macomb County|MI|47538|United States|-5|apartment| +16189|AAAAAAAANDPDAAAA|152|Williams Poplar|Court|Suite D|Red Hill|Wayne County|GA|34338|United States|-5|condo| +16190|AAAAAAAAODPDAAAA|426|14th |Wy|Suite 150|Pleasant Valley|McPherson County|NE|62477|United States|-7|single family| +16191|AAAAAAAAPDPDAAAA|45||||||TN|30191||-6|| +16192|AAAAAAAAAEPDAAAA|925|Locust |Boulevard|Suite 210|Shannon|Perkins County|NE|64120|United States|-7|single family| +16193|AAAAAAAABEPDAAAA|242|River East|RD|Suite G|Ashland|Crittenden County|AR|74244|United States|-6|single family| +16194|AAAAAAAACEPDAAAA|789|Cedar Laurel|Ave|Suite 400|Oakwood|Jackson County|WV|20169|United States|-5|apartment| +16195|AAAAAAAADEPDAAAA|290|Adams |Court|Suite S|Highland Park|Martin County|NC|26534|United States|-5|condo| +16196|AAAAAAAAEEPDAAAA|343|Maple |Avenue|Suite 350|Pleasant Grove|Polk County|GA|34136|United States|-5|condo| +16197|AAAAAAAAFEPDAAAA|41|Main |Way|Suite L|Bridgeport|Crook County|OR|95817|United States|-8|single family| +16198|AAAAAAAAGEPDAAAA|743|Second Woodland|Circle|Suite A|Bridgeport|Bond County|IL|65817|United States|-6|condo| +16199|AAAAAAAAHEPDAAAA|898|Main |Ct.|Suite 450|Riverview|Alcorn County|MS|59003|United States|-6|single family| +16200|AAAAAAAAIEPDAAAA|53|Meadow |Lane|Suite 280|Fairfield|Baldwin County|AL|36192|United States|-6|condo| +16201|AAAAAAAAJEPDAAAA|45|Davis Lee|Ln|Suite 390|Concord|Prince Edward County|VA|24107|United States|-5|condo| +16202|AAAAAAAAKEPDAAAA|844|Maple Main|Blvd|Suite Q|Newport|Linn County|KS|61521|United States|-6|single family| +16203|AAAAAAAALEPDAAAA|145|15th |Wy|Suite 110|Five Forks|Bibb County|GA|32293|United States|-5|condo| +16204|AAAAAAAAMEPDAAAA|176|7th 2nd|Cir.|Suite 10|Salem|Hardeman County|TX|78048|United States|-6|condo| +16205|AAAAAAAANEPDAAAA|600|Poplar |RD|Suite M|Pine Grove|Lewis County|MO|64593|United States|-6|apartment| +16206|AAAAAAAAOEPDAAAA|174|College |Ln|Suite X|Bridgeport|Anderson County|TN|35817|United States|-5|single family| +16207|AAAAAAAAPEPDAAAA|965|Washington View|Cir.|Suite P|Cumberland|Crenshaw County|AL|38971|United States|-6|condo| +16208|AAAAAAAAAFPDAAAA|563|South |Court|Suite 350|Lakeside|Fayette County|IL|69532|United States|-6|single family| +16209|AAAAAAAABFPDAAAA|981|Sixth |Road|Suite E|Macedonia|Grant County|NM|81087|United States|-7|condo| +16210|AAAAAAAACFPDAAAA|112||Ave|Suite O|Sunnyside||TN|31952|||condo| +16211|AAAAAAAADFPDAAAA|836|College |ST|Suite 80|Woodlawn|Pearl River County|MS|54098|United States|-6|condo| +16212|AAAAAAAAEFPDAAAA|488|Central |||||||United States||| +16213|AAAAAAAAFFPDAAAA|901|Hill 13th|Parkway|Suite 320|Maple Grove|Monroe County|PA|18252|United States|-5|apartment| +16214|AAAAAAAAGFPDAAAA|762|11th Valley|Circle|Suite 290|Golden|Young County|TX|70411|United States|-6|single family| +16215|AAAAAAAAHFPDAAAA|890|12th Spruce|Court|Suite 210|Cedar Grove|Monroe County|OH|40411|United States|-5|apartment| +16216|AAAAAAAAIFPDAAAA|409|East |Circle|Suite 160|Kingston|Traill County|ND|54975|United States|-6|apartment| +16217|AAAAAAAAJFPDAAAA|744|Poplar Lakeview|Circle|Suite D|Concord|Todd County|MN|54107|United States|-6|condo| +16218|AAAAAAAAKFPDAAAA|529|Cherry Miller|Wy|Suite I|Garrison|Dimmit County|TX|78767|United States|-6|apartment| +16219|AAAAAAAALFPDAAAA||||Suite 420|Fairfield|Major County||||-6|| +16220|AAAAAAAAMFPDAAAA|952|River North|Pkwy|Suite W|Stringtown|Tazewell County|VA|20162|United States|-5|single family| +16221|AAAAAAAANFPDAAAA|739|Highland Lincoln|Cir.|Suite W|Pleasant Hill|Izard County|AR|73604|United States|-6|single family| +16222|AAAAAAAAOFPDAAAA|900|6th Ninth|Ln|Suite O|White Oak|Nash County|NC|26668|United States|-5|single family| +16223|AAAAAAAAPFPDAAAA|353|Pine |Wy|Suite L|Lone Oak|Carver County|MN|56893|United States|-6|single family| +16224|AAAAAAAAAGPDAAAA|422|13th |Drive|Suite J|Buena Vista|Cheatham County|TN|35752|United States|-5|condo| +16225|AAAAAAAABGPDAAAA|851|Thirteenth |Way|Suite 420|Oak Grove|Carroll County|OH|48370|United States|-5|condo| +16226|AAAAAAAACGPDAAAA|125|Cedar |Dr.|Suite 480|Springfield|Lawrence County|IL|69303|United States|-6|apartment| +16227|AAAAAAAADGPDAAAA|54|Lake 15th|Way|Suite 10|Riverdale|Grundy County|IA|59391|United States|-6|condo| +16228|AAAAAAAAEGPDAAAA|237|Poplar |Circle|Suite E|Jackson|Reeves County|TX|79583|United States|-6|apartment| +16229|AAAAAAAAFGPDAAAA|901|Meadow |Parkway|Suite 140|Wildwood|Mason County|WA|96871|United States|-8|condo| +16230|AAAAAAAAGGPDAAAA|428|Walnut Lee|Way|Suite 240|Fairfield|Jefferson County|AL|36192|United States|-6|condo| +16231|AAAAAAAAHGPDAAAA|567|Woodland Mill|Ave|Suite 180|Hopewell|Sarasota County|FL|30587|United States|-5|single family| +16232|AAAAAAAAIGPDAAAA|451|Smith |Circle|Suite 380|Mount Olive|Livingston County|NY|18059|United States|-5|apartment| +16233|AAAAAAAAJGPDAAAA|964|8th |Wy|Suite E|Leesville|Gilchrist County|FL|35423|United States|-5|condo| +16234|AAAAAAAAKGPDAAAA|600|Park |Dr.|Suite F|New Hope|Essex County|MA|09431|United States|-5|apartment| +16235|AAAAAAAALGPDAAAA|775|Williams |Road|Suite 470|Pleasant Valley|Lake County|OR|92477|United States|-8|single family| +16236|AAAAAAAAMGPDAAAA|2|Jefferson |Blvd|Suite V|Five Points|Clear Creek County|CO|86098|United States|-7|single family| +16237|AAAAAAAANGPDAAAA|787|Cherry Fifth|Street|Suite J|Riverview|Jim Wells County|TX|79003|United States|-6|single family| +16238|AAAAAAAAOGPDAAAA|178|Spruce |RD|Suite 490|Owens|Holt County|NE|62324|United States|-7|single family| +16239|AAAAAAAAPGPDAAAA|35|River Pine|RD|Suite B|Hamilton|Union County|OR|92808|United States|-8|single family| +16240|AAAAAAAAAHPDAAAA||12th Hickory|Road||Jamestown|||||-5|| +16241|AAAAAAAABHPDAAAA|116|Woodland |Road|Suite 310|Franklin|Westchester County|NY|19101|United States|-5|condo| +16242|AAAAAAAACHPDAAAA|214|Walnut |Ave|Suite T|Somerville|Tioga County|NY|17783|United States|-5|single family| +16243|AAAAAAAADHPDAAAA|931|1st Spring|Drive|Suite 380|Crossroads|Wilson County|KS|60534|United States|-6|apartment| +16244|AAAAAAAAEHPDAAAA|702|Lincoln |Pkwy|Suite S|Spring Hill|Washington County|MD|26787|United States|-5|single family| +16245|AAAAAAAAFHPDAAAA|784|Park Jackson|RD|Suite 430|Marion|Hamilton County|IL|60399|United States|-6|single family| +16246|AAAAAAAAGHPDAAAA|717|7th Johnson|Circle|Suite 100|California|Putnam County|WV|20141|United States|-5|single family| +16247|AAAAAAAAHHPDAAAA|619|13th Johnson|Boulevard|Suite E|Oak Ridge|Champaign County|IL|68371|United States|-6|condo| +16248|AAAAAAAAIHPDAAAA|129|Twelfth |Avenue|Suite 10|Fairfield|Grimes County|TX|76192|United States|-6|single family| +16249|AAAAAAAAJHPDAAAA|363|Third |Way|Suite 0|Green Acres|Carroll County|MS|57683|United States|-6|apartment| +16250|AAAAAAAAKHPDAAAA|422|View 2nd|RD|Suite D|Buena Vista|Sherburne County|MN|55752|United States|-6|single family| +16251|AAAAAAAALHPDAAAA|121|Fourth Mill|Court|Suite 460|Wildwood|Norfolk city|VA|26871|United States|-5|single family| +16252|AAAAAAAAMHPDAAAA|284|Cedar 4th|Street|Suite 250|Pleasant Valley|Meeker County|MN|52477|United States|-6|single family| +16253|AAAAAAAANHPDAAAA|262|3rd |Boulevard|Suite 120|Sulphur Springs|DeSoto County|FL|38354|United States|-5|single family| +16254|AAAAAAAAOHPDAAAA|277|Locust |Lane|Suite T|Mount Pleasant|Oceana County|MI|41933|United States|-5|single family| +16255|AAAAAAAAPHPDAAAA|446|Franklin |Boulevard|Suite W|Clifton|Jackson County|MS|58014|United States|-6|apartment| +16256|AAAAAAAAAIPDAAAA|867|11th Cherry|Blvd|Suite W|Union|Humboldt County|NV|88721|United States|-8|single family| +16257|AAAAAAAABIPDAAAA|486|Willow |Ave|Suite E|Woodlawn|Callahan County|TX|74098|United States|-6|condo| +16258|AAAAAAAACIPDAAAA|738|Pine |Street|Suite 190|Shady Grove|Pope County|IL|62812|United States|-6|condo| +16259|AAAAAAAADIPDAAAA|816|Main |Ln|Suite 300|Sunnyside|Emanuel County|GA|31952|United States|-5|apartment| +16260|AAAAAAAAEIPDAAAA|731|College |Ave|Suite 390|Riverview|White County|IL|69003|United States|-6|single family| +16261|AAAAAAAAFIPDAAAA|789|Ridge Mill|Wy|Suite R|Mountain View|Grays Harbor County|WA|94466|United States|-8|condo| +16262|AAAAAAAAGIPDAAAA|136|Oak Seventh|Way|Suite X|Greenville|Holt County|NE|61387|United States|-7|single family| +16263|AAAAAAAAHIPDAAAA|247|Maple |RD|Suite F|Lincoln|Tulare County|CA|91289|United States|-8|single family| +16264|AAAAAAAAIIPDAAAA|753|10th Williams|Circle|Suite 80|Mount Vernon|Morehouse Parish|LA|78482|United States|-6|apartment| +16265|AAAAAAAAJIPDAAAA|349|Third |Circle|Suite O|Pleasant Grove|Lake County|MN|54136|United States|-6|apartment| +16266|AAAAAAAAKIPDAAAA|738|View |Road|Suite Q|Mount Olive|Fairfield County|CT|08659|United States|-5|apartment| +16267|AAAAAAAALIPDAAAA|994|9th View|Lane|Suite 160|Deerfield|Christian County|IL|69840|United States|-6|condo| +16268|AAAAAAAAMIPDAAAA|621|Elm 15th|RD|Suite Q|Hillcrest|White County|IL|63003|United States|-6|apartment| +16269|AAAAAAAANIPDAAAA|39|11th |Court|Suite 120|Mount Zion|Wayne County|MS|58054|United States|-6|condo| +16270|AAAAAAAAOIPDAAAA|980|Franklin |Ave|Suite O|Shady Grove|Clay County|MO|62812|United States|-6|apartment| +16271|AAAAAAAAPIPDAAAA|260|12th |Circle|Suite O|Chestnut Ridge|Sheridan County|MT|67334|United States|-7|apartment| +16272|AAAAAAAAAJPDAAAA|19|4th Jefferson|Boulevard|Suite 450|Riverdale|Chaffee County|CO|89391|United States|-7|single family| +16273|AAAAAAAABJPDAAAA|229|Third |Lane|Suite E|Franklin|Towner County|ND|59101|United States|-6|apartment| +16274|AAAAAAAACJPDAAAA|319|Hickory Hill|Court|Suite 350|Providence|Windham County|VT|07214|United States|-5|single family| +16275|AAAAAAAADJPDAAAA|990|Smith 11th|Drive|Suite H|Shiloh|Essex County|NY|19275|United States|-5|apartment| +16276|AAAAAAAAEJPDAAAA|868|Broadway First|Ct.|Suite 30|Riverdale|Wood County|TX|79391|United States|-6|condo| +16277|AAAAAAAAFJPDAAAA|269|3rd Johnson|Cir.|Suite 140|Farmington|Guilford County|NC|29145|United States|-5|single family| +16278|AAAAAAAAGJPDAAAA|295|Hickory Lake|Lane|Suite 320|Glenwood|San Miguel County|CO|83511|United States|-7|single family| +16279|AAAAAAAAHJPDAAAA|824|Davis Cherry|Dr.|Suite E|Bethel|Jefferson County|FL|35281|United States|-5|condo| +16280|AAAAAAAAIJPDAAAA|676|Lake Hickory|Lane|Suite D|Salem|Walla Walla County|WA|98048|United States|-8|condo| +16281|AAAAAAAAJJPDAAAA|790|Mill |Ct.|Suite I|Greenville|Ness County|KS|61387|United States|-6|condo| +16282|AAAAAAAAKJPDAAAA|408|9th |Ct.|Suite C|Hamilton|Asotin County|WA|92808|United States|-8|single family| +16283|AAAAAAAALJPDAAAA|499|14th Broadway|RD|Suite V|Ellisville|Fulton County|AR|76820|United States|-6|condo| +16284|AAAAAAAAMJPDAAAA|754|Ridge Locust|Avenue|Suite 320|Spring Valley|Maricopa County|AZ|86060|United States|-7|apartment| +16285|AAAAAAAANJPDAAAA|719|12th |Lane|Suite 140|Belmont|Lowndes County|AL|30191|United States|-6|single family| +16286|AAAAAAAAOJPDAAAA|37|Woodland View||||||52106|||single family| +16287|AAAAAAAAPJPDAAAA|976|4th |Pkwy|Suite 0|Harmony|Lincoln County|WV|25804|United States|-5|single family| +16288|AAAAAAAAAKPDAAAA|99|Center Elm|Court|Suite U|Spring Hill|Brown County|IN|46787|United States|-5|condo| +16289|AAAAAAAABKPDAAAA|677|Mill |Wy|Suite O|Belmont|Charles Mix County|SD|50191|United States|-6|apartment| +16290|AAAAAAAACKPDAAAA|499|East |Drive|Suite P|Maple Grove|Ransom County|ND|58252|United States|-6|condo| +16291|AAAAAAAADKPDAAAA|||Circle||||IN||United States|-5|| +16292|AAAAAAAAEKPDAAAA|579|Meadow Fourth|Boulevard|Suite X|Belmont|Spalding County|GA|30191|United States|-5|single family| +16293|AAAAAAAAFKPDAAAA|256|Locust |Pkwy|Suite 310|Five Forks|Cumberland County|IL|62293|United States|-6|condo| +16294|AAAAAAAAGKPDAAAA|762|6th West|Ln|Suite 400|Crossroads|Pope County|IL|60534|United States|-6|single family| +16295|AAAAAAAAHKPDAAAA|304|2nd Third|Drive|Suite F|Cedar Grove|Okeechobee County|FL|30411|United States|-5|condo| +16296|AAAAAAAAIKPDAAAA|781|Broadway |Lane|Suite R|Lewisburg|Juab County|UT|87538|United States|-7|apartment| +16297|AAAAAAAAJKPDAAAA|485|Sunset |Cir.|Suite 150|Bethel|Lavaca County|TX|75281|United States|-6|condo| +16298|AAAAAAAAKKPDAAAA|82|Pine |Circle|Suite 400|Oak Hill|Perry County|TN|37838|United States|-6|condo| +16299|AAAAAAAALKPDAAAA|165|West |Ct.|Suite N|Spring Valley|Esmeralda County|NV|86060|United States|-8|condo| +16300|AAAAAAAAMKPDAAAA|524|Maple East|Avenue|Suite 270|Farmersville|Cochran County|TX|79305|United States|-6|single family| +16301|AAAAAAAANKPDAAAA|407|Lake Spring|Parkway|Suite I|Glendale|Comanche County|KS|63951|United States|-6|single family| +16302|AAAAAAAAOKPDAAAA|134|||Suite I|Greenville||NE|61387|United States|-7|apartment| +16303|AAAAAAAAPKPDAAAA|69|Twelfth Franklin|Circle|Suite D|Woodville|Pottawatomie County|OK|74289|United States|-6|single family| +16304|AAAAAAAAALPDAAAA|514|Ash |RD|Suite K|New Hope|Lee County|VA|29431|United States|-5|apartment| +16305|AAAAAAAABLPDAAAA|424|Mill |Way|Suite 10|Oak Hill|Benzie County|MI|47838|United States|-5|single family| +16306|AAAAAAAACLPDAAAA|406|Second 1st|Road|Suite T|Roscoe|Clarke County|IA|51854|United States|-6|apartment| +16307|AAAAAAAADLPDAAAA|25|Pine |ST|Suite R|New Hope|Bell County|TX|79431|United States|-6|apartment| +16308|AAAAAAAAELPDAAAA|955|Pine |Lane|Suite 140|Green Acres|Siskiyou County|CA|97683|United States|-8|apartment| +16309|AAAAAAAAFLPDAAAA|390|4th View|Parkway|Suite 180|Bridgeport|Perry County|KY|45817|United States|-5|condo| +16310|AAAAAAAAGLPDAAAA|673|Broadway |Boulevard|Suite 290|Newport|Hendry County|FL|31521|United States|-5|apartment| +16311|AAAAAAAAHLPDAAAA|161|Dogwood |Parkway|Suite 320|Macedonia|Carroll County|MD|21087|United States|-5|single family| +16312|AAAAAAAAILPDAAAA|719|Park Walnut|Circle|Suite B|Shiloh|Washington County|NE|69275|United States|-7|condo| +16313|AAAAAAAAJLPDAAAA|519|Walnut |Wy|Suite 50|Cordova|Allendale County|SC|26938|United States|-5|single family| +16314|AAAAAAAAKLPDAAAA|193|3rd |Parkway|Suite O|Deerfield|Darke County|OH|49840|United States|-5|condo| +16315|AAAAAAAALLPDAAAA|409|2nd |Dr.|Suite R|Green Acres|Culberson County|TX|77683|United States|-6|condo| +16316|AAAAAAAAMLPDAAAA|774|Railroad West|Ln|Suite 440|Forest Hills|Modoc County|CA|99237|United States|-8|single family| +16317|AAAAAAAANLPDAAAA|406|Johnson |ST|Suite C|Ludlow|Smith County|TX|75566|United States|-6|single family| +16318|AAAAAAAAOLPDAAAA|486|Hill |Boulevard|Suite 270|Silver Creek|Snyder County|PA|14546|United States|-5|single family| +16319|AAAAAAAAPLPDAAAA|363|Spring |Lane|Suite 490|Macedonia|Ziebach County|SD|51087|United States|-6|apartment| +16320|AAAAAAAAAMPDAAAA|742|Hill |Pkwy|Suite 460|Westgate|Washington County|MD|22366|United States|-5|single family| +16321|AAAAAAAABMPDAAAA|207|North Center|RD|Suite K|Sunnyside|Stutsman County|ND|51952|United States|-6|apartment| +16322|AAAAAAAACMPDAAAA|903|Maple |Parkway|Suite O|Marion|Pennington County|MN|50399|United States|-6|condo| +16323|AAAAAAAADMPDAAAA|707|Cherry |Ln|Suite 460|Fairfield|Jackson County|AR|76192|United States|-6|single family| +16324|AAAAAAAAEMPDAAAA|342|Laurel |Parkway|Suite S|Plainview|DeKalb County|TN|33683|United States|-5|condo| +16325|AAAAAAAAFMPDAAAA|754|Lake View|Ln|Suite U|Red Hill|Chouteau County|MT|64338|United States|-7|apartment| +16326|AAAAAAAAGMPDAAAA|905|Hillcrest Hickory|Ave|Suite I|Deerfield|Chatham County|NC|29840|United States|-5|apartment| +16327|AAAAAAAAHMPDAAAA|354|Ninth Sycamore|Avenue|Suite 410|Five Points|Lenoir County|NC|26098|United States|-5|condo| +16328|AAAAAAAAIMPDAAAA|188|Sunset |Cir.|Suite U|Glenwood|Salem County|NJ|04111|United States|-5|condo| +16329|AAAAAAAAJMPDAAAA|866|Smith |Wy|Suite 420|Lebanon|Douglas County|NE|62898|United States|-6|single family| +16330|AAAAAAAAKMPDAAAA|45|Forest Washington|Drive|Suite 470|Oak Hill|Braxton County|WV|27838|United States|-5|condo| +16331|AAAAAAAALMPDAAAA|29|Hickory Meadow|Pkwy|Suite 260|Forest Hills|Gulf County|FL|39237|United States|-5|apartment| +16332|AAAAAAAAMMPDAAAA|89|Jefferson 4th|Blvd|Suite 390|Spring Valley|Geauga County|OH|46060|United States|-5|condo| +16333|AAAAAAAANMPDAAAA|573|Main Railroad|Ct.|Suite 450|Berea|Steuben County|IN|43804|United States|-5|condo| +16334|AAAAAAAAOMPDAAAA|903|Park Third|Court|Suite J|Oak Grove|Cass County|IN|48370|United States|-5|single family| +16335|AAAAAAAAPMPDAAAA|531|Franklin Seventh|Parkway|Suite 240|Georgetown|Nevada County|AR|77057|United States|-6|apartment| +16336|AAAAAAAAANPDAAAA|22|9th 13th|Ct.|Suite H|Oakland|Columbia County|OR|99843|United States|-8|condo| +16337|AAAAAAAABNPDAAAA|849|4th |Street|Suite F|Sulphur Springs|Castro County|TX|78354|United States|-6|condo| +16338|AAAAAAAACNPDAAAA|762|12th |Lane|Suite 30|Spring Valley|Converse County|WY|86060|United States|-7|apartment| +16339|AAAAAAAADNPDAAAA|755|Johnson |Street|Suite L|Deerfield|Fairfield County|SC|29840|United States|-5|single family| +16340|AAAAAAAAENPDAAAA|556|Smith |Drive|Suite R|Maple Grove|Martin County|FL|38252|United States|-5|apartment| +16341|AAAAAAAAFNPDAAAA|814|Adams |Street|Suite S|Tipton|McLean County|ND|58773|United States|-6|apartment| +16342|AAAAAAAAGNPDAAAA|387|Cedar ||Suite 140|Unionville||TX|||-6|apartment| +16343|AAAAAAAAHNPDAAAA|86|Forest North|Way|Suite H|Lawrenceville|Jackson County|SD|54462|United States|-7|condo| +16344|AAAAAAAAINPDAAAA|786|Oak |Ct.|Suite 100|Riverview|Renville County|MN|59003|United States|-6|single family| +16345|AAAAAAAAJNPDAAAA|380|Jefferson Main|RD|Suite Y|Valley View|Reeves County|TX|75124|United States|-6|single family| +16346|AAAAAAAAKNPDAAAA|550|Adams |Cir.|Suite Q|Brownsville|Hillsborough County|NH|09910|United States|-5|apartment| +16347|AAAAAAAALNPDAAAA|255|8th Spring|Avenue|Suite I|Red Hill|Fayette County|GA|34338|United States|-5|single family| +16348|AAAAAAAAMNPDAAAA|987|Lee |Ave|Suite K|Marion|Owyhee County|ID|80399|United States|-7|condo| +16349|AAAAAAAANNPDAAAA|866|13th |Pkwy|Suite K|Lakewood|Levy County|FL|38877|United States|-5|single family| +16350|AAAAAAAAONPDAAAA|21|River |Way|Suite 350|Macon|Craighead County|AR|70369|United States|-6|condo| +16351|AAAAAAAAPNPDAAAA|689|North |Blvd|Suite 150|Lebanon|Dickinson County|KS|62898|United States|-6|single family| +16352|AAAAAAAAAOPDAAAA|649|6th |Blvd|Suite C|Stratford|Sanborn County|SD|56668|United States|-7|single family| +16353|AAAAAAAABOPDAAAA|962|Pine |Boulevard|Suite 200|Red Hill|Clay County|KS|64338|United States|-6|apartment| +16354|AAAAAAAACOPDAAAA|206|9th View|Boulevard|Suite H|Glenwood|Taylor County|WI|53511|United States|-6|apartment| +16355|AAAAAAAADOPDAAAA|313|Lakeview Church|Boulevard|Suite 50|Pleasant Valley|Sullivan County|IN|42477|United States|-5|single family| +16356|AAAAAAAAEOPDAAAA|||Pkwy|Suite S|||TX||United States||| +16357|AAAAAAAAFOPDAAAA|37|Park Lake|Lane|Suite L|Oak Ridge|Preble County|OH|48371|United States|-5|condo| +16358|AAAAAAAAGOPDAAAA|211|Railroad |Road|Suite B|Riverdale|Roberts County|SD|59391|United States|-7|single family| +16359|AAAAAAAAHOPDAAAA|454|Ninth |Parkway|Suite U|Deerfield|Keokuk County|IA|59840|United States|-6|condo| +16360|AAAAAAAAIOPDAAAA|824|Maple 10th|Ave|Suite 270|Riverside|Litchfield County|CT|09831|United States|-5|condo| +16361|AAAAAAAAJOPDAAAA|541|Smith |Ct.|Suite O|Hillcrest|Sullivan County|PA|13003|United States|-5|condo| +16362|AAAAAAAAKOPDAAAA|84|Sycamore Lake|Court|Suite M|Poplar Grove|Drew County|AR|73664|United States|-6|condo| +16363|AAAAAAAALOPDAAAA|584|2nd View|Cir.|Suite A|Highland Park|Wayne County|GA|36534|United States|-5|apartment| +16364|AAAAAAAAMOPDAAAA|935|Eigth |Road|Suite G|Greenville|Edmunds County|SD|51387|United States|-6|single family| +16365|AAAAAAAANOPDAAAA|715|Hill |ST|Suite D|Maple Grove|Castro County|TX|78252|United States|-6|apartment| +16366|AAAAAAAAOOPDAAAA|568|Dogwood |Lane|Suite Q|Allison|Grand Forks County|ND|54167|United States|-6|condo| +16367|AAAAAAAAPOPDAAAA|478|9th Second|Wy|Suite W|Greenville|Amelia County|VA|21387|United States|-5|single family| +16368|AAAAAAAAAPPDAAAA|298|College |Circle|Suite F|Woodlawn|Ralls County|MO|64098|United States|-6|apartment| +16369|AAAAAAAABPPDAAAA|878|Fourth |Ln|Suite Q|Pine Grove|Fall River County|SD|54593|United States|-6|apartment| +16370|AAAAAAAACPPDAAAA|650|Franklin |Way|Suite 290|Pleasant Valley|Fairfield County|SC|22477|United States|-5|single family| +16371|AAAAAAAADPPDAAAA|952|Church Ash|Lane|Suite 400|Liberty|Martin County|MN|53451|United States|-6|single family| +16372|AAAAAAAAEPPDAAAA|510|Laurel 15th|Street|Suite T|Mountain View|Pocahontas County|IA|54466|United States|-6|apartment| +16373|AAAAAAAAFPPDAAAA|361|Maple |RD|Suite C|Oakland|Tama County|IA|59843|United States|-6|condo| +16374|AAAAAAAAGPPDAAAA|349|Sixth |Ave|Suite P|Highland Park|Fairfax County|VA|26534|United States|-5|single family| +16375|AAAAAAAAHPPDAAAA|600|5th |Cir.|Suite 240|Oakland|Augusta County|VA|29843|United States|-5|single family| +16376|AAAAAAAAIPPDAAAA|991|Railroad |Ct.|Suite 260|Macedonia|Kalamazoo County|MI|41087|United States|-5|apartment| +16377|AAAAAAAAJPPDAAAA|681|First |Ct.|Suite Y|Marion|Prince Edward County|VA|20399|United States|-5|single family| +16378|AAAAAAAAKPPDAAAA|452|Washington College|Drive|Suite 400|Mountain View|Routt County|CO|84466|United States|-7|condo| +16379|AAAAAAAALPPDAAAA|171|Pine Park|Parkway|Suite J|Maple Grove|Carbon County|UT|88252|United States|-7|condo| +16380|AAAAAAAAMPPDAAAA|911|Hill |Dr.|Suite 470|Summit|Keith County|NE|60499|United States|-7|condo| +16381|AAAAAAAANPPDAAAA|320|12th |Ln|Suite 240|Woodruff|Charles City County|VA|24174|United States|-5|single family| +16382|AAAAAAAAOPPDAAAA|315|West Smith|Way|Suite 290|Green Acres|Leslie County|KY|47683|United States|-5|single family| +16383|AAAAAAAAPPPDAAAA|673|Cedar Church|Blvd|Suite K|Woodlawn|Crockett County|TX|74098|United States|-6|single family| +16384|AAAAAAAAAAAEAAAA|807|Broadway |Drive|Suite C|Lee|Pike County|AR|70408|United States|-6|apartment| +16385|AAAAAAAABAAEAAAA|537|Railroad |Boulevard|Suite 280|Shady Grove|Deuel County|NE|62812|United States|-6|condo| +16386|AAAAAAAACAAEAAAA|325|Hillcrest Maple|Ave|Suite 470|Pleasant Hill|Barry County|MI|43604|United States|-5|condo| +16387|AAAAAAAADAAEAAAA|927|West Lee|Pkwy|Suite L|Gum Springs|Anderson County|SC|22106|United States|-5|condo| +16388|AAAAAAAAEAAEAAAA|137|Ninth |Ln|Suite L|Springdale|Swift County|MN|58883|United States|-6|single family| +16389|AAAAAAAAFAAEAAAA|947|Third Lake|Circle|Suite W|Proctor|Pike County|PA|18140|United States|-5|apartment| +16390|AAAAAAAAGAAEAAAA|745|Spring Washington|Court|Suite 480|Clearwater|Wyoming County|NY|19534|United States|-5|apartment| +16391|AAAAAAAAHAAEAAAA|472|Sixth |Dr.|Suite 250|Jamestown|Gates County|NC|26867|United States|-5|condo| +16392|AAAAAAAAIAAEAAAA|972|Mill 1st|Dr.|Suite W|Newport|Houghton County|MI|41521|United States|-5|apartment| +16393|AAAAAAAAJAAEAAAA|563|Maple |Ct.|Suite 370|Jamestown|Greene County|PA|16867|United States|-5|condo| +16394|AAAAAAAAKAAEAAAA|812|Lincoln 1st|Pkwy|||Beauregard Parish|LA|79310|||| +16395|AAAAAAAALAAEAAAA|48|Hill |Ln|Suite 160|Mount Zion|Comanche County|KS|68054|United States|-6|condo| +16396|AAAAAAAAMAAEAAAA|443|Chestnut Central|Boulevard|Suite W|Green Acres|Wetzel County|WV|27683|United States|-5|condo| +16397|AAAAAAAANAAEAAAA|892|Park |RD|Suite N|Deerfield|Pinellas County|FL|39840|United States|-5|apartment| +16398|AAAAAAAAOAAEAAAA|937|Oak |Way|Suite M|Walnut Grove|Logan County|ND|57752|United States|-6|apartment| +16399|AAAAAAAAPAAEAAAA|508|Walnut Walnut|Road|Suite D|Oak Grove|Mariposa County|CA|98370|United States|-8|single family| +16400|AAAAAAAAABAEAAAA|886|Ash |Blvd|Suite 360|Silver Springs|Sitka Borough|AK|94843|United States|-9|apartment| +16401|AAAAAAAABBAEAAAA|336|Railroad |Boulevard|Suite C|Union|Blaine County|MT|68721|United States|-7|condo| +16402|AAAAAAAACBAEAAAA|317|Smith 5th|Wy|Suite Q|Oakland|Ohio County|KY|49843|United States|-5|apartment| +16403|AAAAAAAADBAEAAAA|333|Sycamore Park|Lane|Suite 450|Spring Hill|Kent County|RI|07387|United States|-5|apartment| +16404|AAAAAAAAEBAEAAAA|362|Spring |Wy|Suite L|Red Hill|Freeborn County|MN|54338|United States|-6|apartment| +16405|AAAAAAAAFBAEAAAA|784|Elm Lakeview|Way|Suite 40|Bunker Hill|Van Zandt County|TX|70150|United States|-6|apartment| +16406|AAAAAAAAGBAEAAAA|351|Washington Sycamore|Pkwy|Suite H|Five Forks|Buffalo County|SD|52293|United States|-6|apartment| +16407|AAAAAAAAHBAEAAAA|||||Bridgeport|Bland County|||United States|-5|| +16408|AAAAAAAAIBAEAAAA||||Suite N|Union|Morrow County|OH|||-5|| +16409|AAAAAAAAJBAEAAAA|418|West Main|RD|Suite X|Mount Vernon|Jersey County|IL|68482|United States|-6|condo| +16410|AAAAAAAAKBAEAAAA|190|Sunset |Drive|Suite 140|Lakewood|Monroe County|AL|38877|United States|-6|apartment| +16411|AAAAAAAALBAEAAAA|673|Franklin |Boulevard|Suite 10|Highland Park|Mariposa County|CA|96534|United States|-8|single family| +16412|AAAAAAAAMBAEAAAA|274|5th |Lane|Suite F|Valley View|Turner County|SD|55124|United States|-7|apartment| +16413|AAAAAAAANBAEAAAA|844|11th Spring|Cir.|Suite S|Oakwood|Saline County|NE|60169|United States|-7|single family| +16414|AAAAAAAAOBAEAAAA|438|Park |ST|Suite 20|Buena Vista|Lincoln County|KS|65752|United States|-6|condo| +16415|AAAAAAAAPBAEAAAA|499|Jackson |Drive|Suite H|Florence|Eagle County|CO|83394|United States|-7|single family| +16416|AAAAAAAAACAEAAAA|490|Ridge Spruce|RD|Suite V|Oak Ridge|Whiteside County|IL|68371|United States|-6|condo| +16417|AAAAAAAABCAEAAAA|274|Birch |Court|Suite 310|Bunker Hill|Tippah County|MS|50150|United States|-6|apartment| +16418|AAAAAAAACCAEAAAA||Green |||Pleasant Valley|Carroll County|VA||United States|-5|| +16419|AAAAAAAADCAEAAAA|570|2nd |Boulevard|Suite T|Green Acres|La Salle County|TX|77683|United States|-6|apartment| +16420|AAAAAAAAECAEAAAA|85|Second 14th|Parkway|||||57057||-6|| +16421|AAAAAAAAFCAEAAAA|387|Hillcrest |Lane|Suite 380|Sulphur Springs|Yakutat Borough|AK|98354|United States|-9|single family| +16422|AAAAAAAAGCAEAAAA|278|Ridge |Blvd|Suite 370|Sunnyside|Simpson County|KY|41952|United States|-5|condo| +16423|AAAAAAAAHCAEAAAA|285|View Sunset|Court|Suite D|Riverdale|Gilmer County|WV|29391|United States|-5|single family| +16424|AAAAAAAAICAEAAAA|233|Church Forest|Way|Suite Q|Ferguson|Oakland County|MI|41821|United States|-5|condo| +16425|AAAAAAAAJCAEAAAA|289|Ridge Valley|Ln|Suite Y|Spring Hill|Granville County|NC|26787|United States|-5|condo| +16426|AAAAAAAAKCAEAAAA|835|Walnut 7th|Court|Suite 150|New Hope|Crook County|OR|99431|United States|-8|single family| +16427|AAAAAAAALCAEAAAA|316|East 6th|Street|Suite T|Forest Hills|Sagadahoc County|ME|09837|United States|-5|apartment| +16428|AAAAAAAAMCAEAAAA|588|Johnson 13th|Drive|Suite X|Farmington|Oldham County|KY|49145|United States|-5|condo| +16429|AAAAAAAANCAEAAAA|850|Laurel North|Pkwy|Suite 350|Valley View|Shannon County|SD|55124|United States|-7|single family| +16430|AAAAAAAAOCAEAAAA|259|Railroad Meadow|Ave|Suite E|Florence|McDuffie County|GA|33394|United States|-5|apartment| +16431|AAAAAAAAPCAEAAAA|166|North Ash|Drive|Suite I|Bridgeport|Lincoln Parish|LA|75817|United States|-6|apartment| +16432|AAAAAAAAADAEAAAA|903|Lake 11th|ST|Suite U|Providence|Claiborne Parish|LA|76614|United States|-6|apartment| +16433|AAAAAAAABDAEAAAA|394|Laurel |Dr.|Suite 370|Green Acres|Monroe County|AL|37683|United States|-6|single family| +16434|AAAAAAAACDAEAAAA|445|Lee 12th|Court|Suite 220|Woodland|Saline County|NE|64854|United States|-7|single family| +16435|AAAAAAAADDAEAAAA|989|Oak 12th|ST|Suite 430|Waterloo|Logan County|KS|61675|United States|-6|apartment| +16436|AAAAAAAAEDAEAAAA|994|Hill Ash|Parkway|Suite 420|Franklin|Clay County|TN|39101|United States|-5|single family| +16437|AAAAAAAAFDAEAAAA|507|Valley Spruce|Circle|Suite 10|Rolling Hills|Knox County|TN|37272|United States|-6|condo| +16438|AAAAAAAAGDAEAAAA|176|Johnson Hillcrest|Road|Suite 210|Waterloo|Granite County|MT|61675|United States|-7|single family| +16439|AAAAAAAAHDAEAAAA|705|9th |Ln|Suite V|Hillcrest|Sullivan County|PA|13003|United States|-5|condo| +16440|AAAAAAAAIDAEAAAA|376|View |Court|Suite 10|Shady Grove|Macon County|IL|62812|United States|-6|condo| +16441|AAAAAAAAJDAEAAAA|746|Madison |Way|Suite H|Georgetown|Hart County|KY|47057|United States|-6|apartment| +16442|AAAAAAAAKDAEAAAA|882|Maple |Cir.|Suite B|Harmony|Sheridan County|ND|55804|United States|-6|single family| +16443|AAAAAAAALDAEAAAA|514|Smith |Ln|Suite 310|Oak Ridge|Pope County|MN|58371|United States|-6|condo| +16444|AAAAAAAAMDAEAAAA|129|Lakeview Smith|Avenue|Suite 0|Oak Hill|Noxubee County|MS|57838|United States|-6|apartment| +16445|AAAAAAAANDAEAAAA|303|Sunset South|Way|Suite X|Forest Hills|Dickey County|ND|59237|United States|-6|apartment| +16446|AAAAAAAAODAEAAAA|285|Oak First|Dr.|Suite 180|Lakeview|Douglas County|WI|58579|United States|-6|single family| +16447|AAAAAAAAPDAEAAAA|403|Lakeview |RD|Suite V|Florence|Jefferson County|ID|83394|United States|-7|single family| +16448|AAAAAAAAAEAEAAAA|741|Elm |Way|Suite F|Jamestown|Toole County|MT|66867|United States|-7|condo| +16449|AAAAAAAABEAEAAAA|||Street|Suite 230||Kent County|TX|71409|||| +16450|AAAAAAAACEAEAAAA|164|Cedar |Circle|Suite I|Little River|Whitfield County|GA|30319|United States|-5|single family| +16451|AAAAAAAADEAEAAAA|699|Cherry |Ave|Suite H|Wildwood|Clinton County|MI|46871|United States|-5|single family| +16452|AAAAAAAAEEAEAAAA|740|Chestnut Jackson|Ln|Suite 420|Bunker Hill|Young County|TX|70150|United States|-6|condo| +16453|AAAAAAAAFEAEAAAA|628|Williams |Blvd|Suite 310|Richfield|Miller County|MO|66196|United States|-6|condo| +16454|AAAAAAAAGEAEAAAA|156|Oak |Street|Suite G|Summit|Bolivar County|MS|50499|United States|-6|apartment| +16455|AAAAAAAAHEAEAAAA|561|Franklin |RD|Suite 100|Belmont|Hardy County|WV|20191|United States|-5|apartment| +16456|AAAAAAAAIEAEAAAA|497|Adams |Circle|Suite 440|Forest Hills|Clatsop County|OR|99237|United States|-8|single family| +16457|AAAAAAAAJEAEAAAA|665|Adams |Dr.|Suite S|Edgewood|Hickman County|TN|30069|United States|-5|single family| +16458|AAAAAAAAKEAEAAAA|505|Ridge |Drive|Suite 130|Spring Valley|Butte County|SD|56060|United States|-6|condo| +16459|AAAAAAAALEAEAAAA|||||Union|Edmunds County||58721|United States||| +16460|AAAAAAAAMEAEAAAA|140|Franklin |Lane|Suite R|Sunnyside|Putnam County|TN|31952|United States|-6|condo| +16461|AAAAAAAANEAEAAAA|126|5th |Blvd|Suite Y|Pine Grove|Augusta County|VA|24593|United States|-5|single family| +16462|AAAAAAAAOEAEAAAA|968|4th Jefferson|Ln|Suite 140|Wilson|Gilchrist County|FL|36971|United States|-5|condo| +16463|AAAAAAAAPEAEAAAA|287|River 15th|Parkway|Suite F|Little River|Lamar County|GA|30319|United States|-5|condo| +16464|AAAAAAAAAFAEAAAA|76|6th |Lane|Suite P|Gladstone|Phillips County|CO|80894|United States|-7|apartment| +16465|AAAAAAAABFAEAAAA|511|Church 7th|Dr.|Suite U|Woodlawn|Williamson County|TN|34098|United States|-5|condo| +16466|AAAAAAAACFAEAAAA|311|Maple Valley|Street|Suite 300|Arlington|Bonneville County|ID|86557|United States|-7|single family| +16467|AAAAAAAADFAEAAAA|552|Main |Blvd|Suite P|Bethel|Franklin County|PA|15281|United States|-5|condo| +16468|AAAAAAAAEFAEAAAA|577|Spring Ridge|Ln|Suite X|Lincoln|Chittenden County|VT|01889|United States|-5|single family| +16469|AAAAAAAAFFAEAAAA|379|Third |Boulevard|Suite 250|Stringtown|Custer County|ID|80162|United States|-7|condo| +16470|AAAAAAAAGFAEAAAA|722|Highland |Street|Suite 330|Franklin|Oliver County|ND|59101|United States|-6|condo| +16471|AAAAAAAAHFAEAAAA|128|Davis Woodland|Dr.|Suite P|Edgewood|Vernon County|WI|50069|United States|-6|single family| +16472|AAAAAAAAIFAEAAAA|569|Johnson |Lane|Suite K|Union Hill|DeSoto County|MS|57746|United States|-6|single family| +16473|AAAAAAAAJFAEAAAA|604|10th |Street|Suite E|Oak Ridge|Warren County|KY|48371|United States|-5|condo| +16474|AAAAAAAAKFAEAAAA|625|Chestnut |Parkway|Suite C|Carter|Vernon County|WI|50919|United States|-6|condo| +16475|AAAAAAAALFAEAAAA|301|5th Main|Ave|Suite 420|Florence|Bourbon County|KY|43394|United States|-6|apartment| +16476|AAAAAAAAMFAEAAAA|497|Central |Blvd|Suite 80|Pleasant Hill|McMinn County|TN|33604|United States|-6|single family| +16477|AAAAAAAANFAEAAAA|872|15th 8th|Ct.|Suite 360|Cedar Grove|Hopewell city|VA|20411|United States|-5|apartment| +16478|AAAAAAAAOFAEAAAA|330|4th |Court|Suite V|Highland Park|Waukesha County|WI|56534|United States|-6|single family| +16479|AAAAAAAAPFAEAAAA|602|6th Third|Street|Suite 360|Four Points|Rusk County|WI|51216|United States|-6|condo| +16480|AAAAAAAAAGAEAAAA|458|1st Park|Blvd|Suite 280|Pleasant Hill|Sheridan County|NE|63604|United States|-7|single family| +16481|AAAAAAAABGAEAAAA|966|10th Mill|Pkwy|Suite 230|Ludlow|Wayne County|PA|15566|United States|-5|apartment| +16482|AAAAAAAACGAEAAAA|583|North Mill|Avenue|Suite 470|Riverdale|Madison County|MS|59391|United States|-6|single family| +16483|AAAAAAAADGAEAAAA|45|Third Elevnth|Drive|Suite M|White Oak|Washington County|MD|26668|United States|-5|single family| +16484|AAAAAAAAEGAEAAAA|508|2nd |Wy|Suite 0|Shady Grove|Calhoun County|WV|22812|United States|-5|single family| +16485|AAAAAAAAFGAEAAAA|270|Chestnut Miller|Cir.|Suite V|Jamestown|Orange County|IN|46867|United States|-5|single family| +16486|AAAAAAAAGGAEAAAA|77|Fourth |Way|Suite 330|Union Hill|Washington County|UT|87746|United States|-7|apartment| +16487|AAAAAAAAHGAEAAAA|318|Hill |Road|Suite J|Shady Grove|Stafford County|KS|62812|United States|-6|apartment| +16488|AAAAAAAAIGAEAAAA|||Road||Lincoln|Dawson County||31289|United States|-5|single family| +16489|AAAAAAAAJGAEAAAA|439|Willow Pine|Blvd|Suite Y|Salem|San Diego County|CA|98048|United States|-8|single family| +16490|AAAAAAAAKGAEAAAA|556|8th |Dr.|Suite 100|Sulphur Springs|Neosho County|KS|68354|United States|-6|condo| +16491|AAAAAAAALGAEAAAA|269|11th |Blvd|Suite 420|Hillcrest|Crisp County|GA|33003|United States|-5|apartment| +16492|AAAAAAAAMGAEAAAA|807|Second Main|Pkwy|Suite 220|Wildwood|Stafford County|KS|66871|United States|-6|apartment| +16493|AAAAAAAANGAEAAAA|912|12th Williams|Wy|Suite O|Stringtown|Cape Girardeau County|MO|60162|United States|-6|apartment| +16494|AAAAAAAAOGAEAAAA|235|Birch Park||Suite J|||OK|70162|||| +16495|AAAAAAAAPGAEAAAA|428|6th Ash|Ln|Suite I|Collinsville|Harvey County|KS|62459|United States|-6|condo| +16496|AAAAAAAAAHAEAAAA|708|Highland Chestnut|Road|Suite 370|Summit|King and Queen County|VA|20499|United States|-5|single family| +16497|AAAAAAAABHAEAAAA|922|Hickory |Ct.|Suite V|Wildwood|Fulton County|IN|46871|United States|-5|condo| +16498|AAAAAAAACHAEAAAA|16|Meadow |Ct.|Suite 110|Clinton|Charlton County|GA|38222|United States|-5|apartment| +16499|AAAAAAAADHAEAAAA|588|Tenth Lake|Court|Suite E|Greenfield|Macomb County|MI|45038|United States|-5|apartment| +16500|AAAAAAAAEHAEAAAA|449|Lake Franklin|Dr.|Suite N|Newport|Jefferson Davis County|MS|51521|United States|-6|condo| +16501|AAAAAAAAFHAEAAAA|619|North Pine|Circle|Suite K|Bridgeport|Calloway County|KY|45817|United States|-6|single family| +16502|AAAAAAAAGHAEAAAA|303|Valley Smith|Street|Suite P|Brownsville|Rappahannock County|VA|29310|United States|-5|single family| +16503|AAAAAAAAHHAEAAAA|523|Railroad Forest|Pkwy|Suite 450|Shady Grove|San Juan County|WA|92812|United States|-8|single family| +16504|AAAAAAAAIHAEAAAA|316|Church Valley|ST|Suite 430|Spring Hill|Douglas County|MO|66787|United States|-6|condo| +16505|AAAAAAAAJHAEAAAA|614|Meadow |Avenue|Suite F|Mountain View|Logan County|KY|44466|United States|-5|single family| +16506|AAAAAAAAKHAEAAAA|282|10th Second|Way|Suite 80|New Hope|Hughes County|OK|79431|United States|-6|single family| +16507|AAAAAAAALHAEAAAA|635|College Third|Road|Suite 300|Oakwood|Middlesex County|VA|20169|United States|-5|apartment| +16508|AAAAAAAAMHAEAAAA|491|1st Third|Ln|Suite 240|Farmington|Woodbury County|IA|59145|United States|-6|condo| +16509|AAAAAAAANHAEAAAA|655|Washington |Cir.|Suite B|Union|Warrick County|IN|48721|United States|-5|apartment| +16510|AAAAAAAAOHAEAAAA|112|Miller |Dr.|Suite B|Valley View|Cass County|MI|45124|United States|-5|single family| +16511|AAAAAAAAPHAEAAAA|71|10th Railroad|Ct.|Suite N|Plainview|Highland County|OH|43683|United States|-5|condo| +16512|AAAAAAAAAIAEAAAA|27|View |Cir.|Suite I|New Hope|Choctaw County|MS|59431|United States|-6|condo| +16513|AAAAAAAABIAEAAAA|236|Hill |Avenue|Suite 30|Woodville|Routt County|CO|84289|United States|-7|apartment| +16514|AAAAAAAACIAEAAAA|431|Highland Church|Lane|Suite L|Pleasant Hill|Garfield County|OK|73604|United States|-6|condo| +16515|AAAAAAAADIAEAAAA|236|Sixth Smith|Street|Suite E|Pleasant Valley|Washington County|MS|52477|United States|-6|apartment| +16516|AAAAAAAAEIAEAAAA|329|Second 1st|Cir.|Suite A|Jamestown|Will County|IL|66867|United States|-6|single family| +16517|AAAAAAAAFIAEAAAA|445|3rd |Lane|Suite 270|Red Hill|Louisa County|IA|54338|United States|-6|condo| +16518|AAAAAAAAGIAEAAAA|56|Main |Way|Suite D|Spring Hill|Tioga County|NY|16787|United States|-5|single family| +16519|AAAAAAAAHIAEAAAA|||||Springfield|||69303|United States|-7|condo| +16520|AAAAAAAAIIAEAAAA|903|East |Way|Suite 410|Midway|Clarke County|MS|51904|United States|-6|single family| +16521|AAAAAAAAJIAEAAAA|200|Maple Chestnut|RD|Suite 320|Hillcrest|Koochiching County|MN|53003|United States|-6|single family| +16522|AAAAAAAAKIAEAAAA|998|8th |Wy|Suite P|Sulphur Springs|Sullivan County|PA|18354|United States|-5|condo| +16523|AAAAAAAALIAEAAAA|111|3rd |Ct.|Suite 260|Hillcrest|Halifax County|VA|23003|United States|-5|single family| +16524|AAAAAAAAMIAEAAAA|803|Spring |Blvd|Suite O|Shady Grove|Lafayette County|MS|52812|United States|-6|single family| +16525|AAAAAAAANIAEAAAA|809|13th |Court|Suite V|Pine Grove|Madison County|IA|54593|United States|-6|single family| +16526|AAAAAAAAOIAEAAAA|483|Ash |Parkway|Suite 240|Pine Grove|Red Willow County|NE|64593|United States|-7|single family| +16527|AAAAAAAAPIAEAAAA|31|Oak |ST|Suite 370|Cedar Grove|Fond du Lac County|WI|50411|United States|-6|apartment| +16528|AAAAAAAAAJAEAAAA|643|Birch |Avenue|Suite 340|Franklin|Sully County|SD|59101|United States|-7|condo| +16529|AAAAAAAABJAEAAAA|931|Mill 1st|Cir.|Suite K|Hopewell|Hill County|TX|70587|United States|-6|apartment| +16530|AAAAAAAACJAEAAAA|124|Third |RD|Suite 440|Arlington|Walker County|GA|36557|United States|-5|single family| +16531|AAAAAAAADJAEAAAA|941|Willow |Ct.|Suite 480|Highland|Bradford County|PA|19454|United States|-5|condo| +16532|AAAAAAAAEJAEAAAA|48|Valley |ST|Suite O|Riverdale|Adams County|PA|19391|United States|-5|condo| +16533|AAAAAAAAFJAEAAAA|888|Sunset |Road|Suite E|Lincoln|Lyon County|NV|81289|United States|-8|apartment| +16534|AAAAAAAAGJAEAAAA|115|Highland |Ave|Suite K|Greenville|Monroe County|IN|41387|United States|-5|single family| +16535|AAAAAAAAHJAEAAAA|222|Seventh Lake|RD|Suite 90|Bethel|Windham County|VT|05881|United States|-5|condo| +16536|AAAAAAAAIJAEAAAA|991|View |Parkway|Suite Y|Oak Grove|Taylor County|KY|48370|United States|-5|apartment| +16537|AAAAAAAAJJAEAAAA|528|Ash 3rd|Ln|Suite G|Page|Kennebec County|ME|00896|United States|-5|apartment| +16538|AAAAAAAAKJAEAAAA|858|Spring Lincoln|RD|Suite K|Marion|McCormick County|SC|20399|United States|-5|single family| +16539|AAAAAAAALJAEAAAA|339|Lakeview |ST|Suite A|Sulphur Springs|Travis County|TX|78354|United States|-6|single family| +16540|AAAAAAAAMJAEAAAA|154|15th |Blvd|Suite 140|Guthrie|Crawford County|IL|61423|United States|-6|apartment| +16541|AAAAAAAANJAEAAAA|931|Jefferson Locust|Cir.|Suite 390|Liberty|Benton County|AR|73451|United States|-6|single family| +16542|AAAAAAAAOJAEAAAA|307|Green River|Ave|Suite V|Bethel|Piatt County|IL|65281|United States|-6|single family| +16543|AAAAAAAAPJAEAAAA|763|11th |Ct.|Suite G|Oak Ridge|Valley County|MT|68371|United States|-7|single family| +16544|AAAAAAAAAKAEAAAA|607|Elm |Ct.|Suite Q|Enterprise|Kidder County|ND|51757|United States|-6|condo| +16545|AAAAAAAABKAEAAAA|||Ct.||Belmont|Hamilton County|IL|60191|||condo| +16546|AAAAAAAACKAEAAAA|932|Jackson Second|Court|Suite 470|Oak Grove|Cleveland County|AR|78370|United States|-6|condo| +16547|AAAAAAAADKAEAAAA|706|Wilson Sixth||Suite 300|Red Hill||||United States|-7|condo| +16548|AAAAAAAAEKAEAAAA|17|Mill |ST|Suite 290|Belmont|Monroe County|MI|40191|United States|-5|condo| +16549|AAAAAAAAFKAEAAAA|140|4th |Drive|Suite G|Harmony|Carroll County|IL|65804|United States|-6|condo| +16550|AAAAAAAAGKAEAAAA|878|Center 12th|Ave|Suite 380|Plainview|Pembina County|ND|53683|United States|-6|single family| +16551|AAAAAAAAHKAEAAAA|66|Third Railroad|Ct.|Suite 170|Farmington|Floyd County|IN|49145|United States|-5|apartment| +16552|AAAAAAAAIKAEAAAA|304|Park Seventh|Avenue|Suite 440|Bridgeport|Dawson County|TX|75817|United States|-6|single family| +16553|AAAAAAAAJKAEAAAA|659|Birch |Boulevard|Suite 250|White Oak|Blair County|PA|16668|United States|-5|condo| +16554|AAAAAAAAKKAEAAAA|451|Miller Elm|Avenue|Suite 40|Buena Vista|Montgomery County|MS|55752|United States|-6|single family| +16555|AAAAAAAALKAEAAAA|44|Oak |Ln|Suite 90|Springdale|Atchison County|KS|68883|United States|-6|single family| +16556|AAAAAAAAMKAEAAAA|151|Spruce |Circle|Suite P|Riverview|Ellis County|TX|79003|United States|-6|condo| +16557|AAAAAAAANKAEAAAA|877|Highland Locust|Pkwy|Suite A|Red Hill|Santa Rosa County|FL|34338|United States|-5|single family| +16558|AAAAAAAAOKAEAAAA|27|Poplar |Dr.|Suite 50|Sulphur Springs|Carroll County|MS|58354|United States|-6|condo| +16559|AAAAAAAAPKAEAAAA|175|Third 5th|Lane|Suite 90|Enterprise|Divide County|ND|51757|United States|-6|apartment| +16560|AAAAAAAAALAEAAAA|970|7th |Circle|Suite 40|Littleton|Holmes County|FL|36074|United States|-5|apartment| +16561|AAAAAAAABLAEAAAA|484|3rd |Way|Suite 150|Concord|Winnebago County|IL|64107|United States|-6|apartment| +16562|AAAAAAAACLAEAAAA|615|Madison |Cir.|Suite 260|White Oak|Summit County|OH|46668|United States|-5|condo| +16563|AAAAAAAADLAEAAAA|345|Lakeview Chestnut|Blvd|Suite O|Woodbine|Haywood County|TN|34253|United States|-5|single family| +16564|AAAAAAAAELAEAAAA|391|Ash |Drive|Suite 380|Farmington|Oscoda County|MI|49145|United States|-5|single family| +16565|AAAAAAAAFLAEAAAA|108|Main ||Suite W|Crossroads|Marion County|KY|40534||-5|| +16566|AAAAAAAAGLAEAAAA|176|Spring Willow|Blvd|Suite H|New Hope|Cooper County|MO|69431|United States|-6|single family| +16567|AAAAAAAAHLAEAAAA|28|North Madison|Ct.|Suite 70|Ashland|Worth County|MO|64244|United States|-6|single family| +16568|AAAAAAAAILAEAAAA|951|Elm |Way|Suite 260|Newport|Perry County|PA|11521|United States|-5|apartment| +16569|AAAAAAAAJLAEAAAA|169|Sixth |Wy|Suite C|Mount Zion|Forsyth County|GA|38054|United States|-5|single family| +16570|AAAAAAAAKLAEAAAA|476|Wilson Thirteenth|Lane|Suite 430|Highland Park|Niagara County|NY|16534|United States|-5|condo| +16571|AAAAAAAALLAEAAAA|389|Chestnut Johnson|Pkwy|Suite F|Pleasant Hill|Wilkes County|NC|23604|United States|-5|condo| +16572|AAAAAAAAMLAEAAAA|868|Adams |Drive|Suite 110|Stringtown|Webster County|KY|40162|United States|-5|apartment| +16573|AAAAAAAANLAEAAAA|589|Oak |Road|Suite W|Oak Grove|Moore County|TX|78370|United States|-6|single family| +16574|AAAAAAAAOLAEAAAA|215|Maple |Blvd|Suite H|Greenwood|Waldo County|ME|09428|United States|-5|condo| +16575|AAAAAAAAPLAEAAAA|800|Johnson Adams|Way|Suite B|Bethel|Vermilion County|IL|65281|United States|-6|apartment| +16576|AAAAAAAAAMAEAAAA|190|Cherry |Ln|Suite F|Cedar|Porter County|IN|41229|United States|-5|condo| +16577|AAAAAAAABMAEAAAA|819|6th College|Lane|Suite 40|Franklin|Suffolk city|VA|29101|United States|-5|apartment| +16578|AAAAAAAACMAEAAAA|586|Third |Way|Suite 130|Bridgeport|Meigs County|TN|35817|United States|-6|condo| +16579|AAAAAAAADMAEAAAA|126|Oak Walnut|Pkwy|Suite C|Newport|Whitfield County|GA|31521|United States|-5|apartment| +16580|AAAAAAAAEMAEAAAA|936|14th |Drive|Suite H|Edgewood|Dyer County|TN|30069|United States|-5|condo| +16581|AAAAAAAAFMAEAAAA|74|River Central|Avenue|Suite 240|Mountain View|Worth County|GA|34466|United States|-5|condo| +16582|AAAAAAAAGMAEAAAA|25|Broadway |Cir.|Suite 420|Jamestown|Mesa County|CO|86867|United States|-7|apartment| +16583|AAAAAAAAHMAEAAAA|102|Fourth 3rd|ST|Suite 180|Enterprise|Bartow County|GA|31757|United States|-5|single family| +16584|AAAAAAAAIMAEAAAA|548|North |Avenue|Suite D|Union Hill|Sully County|SD|57746|United States|-7|condo| +16585|AAAAAAAAJMAEAAAA|500|5th |Ct.|Suite N|Oak Ridge|Hansford County|TX|78371|United States|-6|apartment| +16586|AAAAAAAAKMAEAAAA|989|Fifth Forest|Wy|Suite U|Salem|Watonwan County|MN|58048|United States|-6|apartment| +16587|AAAAAAAALMAEAAAA|521|Church |Drive|Suite 200|Farmington|Jackson County|AL|39145|United States|-6|apartment| +16588|AAAAAAAAMMAEAAAA|933|Second |RD|Suite 140|Riverview|Muskingum County|OH|49003|United States|-5|condo| +16589|AAAAAAAANMAEAAAA|513|Lake ||Suite K|Antioch|Ozark County|||United States|-6|| +16590|AAAAAAAAOMAEAAAA|||ST||Riverdale||CT|09991||-5|| +16591|AAAAAAAAPMAEAAAA|219||||||OK|78370|United States||| +16592|AAAAAAAAANAEAAAA|459|5th |RD|Suite D|Ashland|Newport County|RI|04844|United States|-5|single family| +16593|AAAAAAAABNAEAAAA|64|Jackson |Circle|Suite W|Oak Hill|Potter County|SD|57838|United States|-7|apartment| +16594|AAAAAAAACNAEAAAA|954|Smith Washington|Dr.|Suite R|Crossroads|Gallatin County|IL|60534|United States|-6|apartment| +16595|AAAAAAAADNAEAAAA|470|2nd |Cir.|Suite 320|Crossroads|Sherman County|TX|70534|United States|-6|condo| +16596|AAAAAAAAENAEAAAA|628|East 6th|Circle|Suite W|Union Hill|Haywood County|NC|27746|United States|-5|condo| +16597|AAAAAAAAFNAEAAAA|681|Main |Ct.|Suite 70|Jamestown|Terrell County|TX|76867|United States|-6|apartment| +16598|AAAAAAAAGNAEAAAA|463|10th |Cir.|Suite 180|Plainview|Grenada County|MS|53683|United States|-6|condo| +16599|AAAAAAAAHNAEAAAA|378|Jefferson |Blvd|Suite L|Shiloh|Avery County|NC|29275|United States|-5|apartment| +16600|AAAAAAAAINAEAAAA|833|Lincoln |Parkway|Suite V|Belmont|Columbia County|WI|50191|United States|-6|condo| +16601|AAAAAAAAJNAEAAAA|485|Hill 2nd|Ln|Suite 150|Arlington|Columbia County|AR|76557|United States|-6|condo| +16602|AAAAAAAAKNAEAAAA|6|Franklin |RD|Suite 240|Newport|Hickman County|TN|31521|United States|-5|single family| +16603|AAAAAAAALNAEAAAA|532|4th |Pkwy|Suite 100|Harvey|Flathead County|MT|65858|United States|-7|apartment| +16604|AAAAAAAAMNAEAAAA|38|Broadway Ridge|Lane|Suite 140|Riverside|Fairfax County|VA|29231|United States|-5|single family| +16605|AAAAAAAANNAEAAAA|107|Central Lincoln|Court|Suite 460|Shiloh|Fremont County|ID|89275|United States|-7|apartment| +16606|AAAAAAAAONAEAAAA|477|Lakeview Mill|Blvd|Suite 440|Shady Grove|Fayette County|TN|32812|United States|-5|condo| +16607|AAAAAAAAPNAEAAAA|995|3rd Park|Blvd|Suite 0|Pleasant Grove|Oconto County|WI|54136|United States|-6|apartment| +16608|AAAAAAAAAOAEAAAA|458|Park 11th|Pkwy|Suite S|Brownsville|Grady County|GA|39310|United States|-5|apartment| +16609|AAAAAAAABOAEAAAA|314|East |Boulevard|Suite 100|Unionville|Ochiltree County|TX|71711|United States|-6|condo| +16610|AAAAAAAACOAEAAAA|748|2nd 2nd|Street|Suite 100|Cedar Grove|Essex County|MA|01011|United States|-5|condo| +16611|AAAAAAAADOAEAAAA|311|7th |Cir.|Suite 460|Elizabeth|King and Queen County|VA|22935|United States|-5|single family| +16612|AAAAAAAAEOAEAAAA|102|Maple |Ave|Suite 250|Spring Hill|Jackson County|OK|76787|United States|-6|single family| +16613|AAAAAAAAFOAEAAAA|757|Sixth Cherry|Court|Suite 400|Union Hill|Stevens County|MN|57746|United States|-6|apartment| +16614|AAAAAAAAGOAEAAAA|774|Forest West|Dr.|Suite 310|Fairview|Vanderburgh County|IN|45709|United States|-5|single family| +16615|AAAAAAAAHOAEAAAA||Hill |Ave|Suite 390|Sulphur Springs||CA||United States|-8|| +16616|AAAAAAAAIOAEAAAA|496|4th 3rd|Ave|Suite 440|Bunker Hill|Mississippi County|AR|70150|United States|-6|condo| +16617|AAAAAAAAJOAEAAAA|117|10th |Lane|Suite R|Silver City|Jackson County|IN|43405|United States|-5|condo| +16618|AAAAAAAAKOAEAAAA|228|13th Miller|Road|Suite G|Union|Sheboygan County|WI|58721|United States|-6|apartment| +16619|AAAAAAAALOAEAAAA|146|2nd |Lane|Suite 360|Oak Grove|Otero County|CO|88370|United States|-7|single family| +16620|AAAAAAAAMOAEAAAA|378|10th |Street|Suite 360|Lebanon|Marion County|IL|62898|United States|-6|apartment| +16621|AAAAAAAANOAEAAAA|256|4th |Street|Suite 450|Greenfield|Cottle County|TX|75038|United States|-6|apartment| +16622|AAAAAAAAOOAEAAAA|785|Madison |Parkway|Suite 400|Enterprise|Seward County|KS|61757|United States|-6|apartment| +16623|AAAAAAAAPOAEAAAA|671|1st |Lane|Suite N|Mount Vernon|Hartley County|TX|78482|United States|-6|condo| +16624|AAAAAAAAAPAEAAAA|272|11th |Ave|Suite G|Oak Ridge|Cannon County|TN|38371|United States|-5|apartment| +16625|AAAAAAAABPAEAAAA|504|Second Lake|Street|Suite C|Louisville|Ramsey County|ND|54464|United States|-6|apartment| +16626|AAAAAAAACPAEAAAA|397|Washington |Ln|Suite 440|Concord|Rio Arriba County|NM|84107|United States|-7|single family| +16627|AAAAAAAADPAEAAAA|753|Willow |RD|Suite Q|Friendship|Canyon County|ID|84536|United States|-7|apartment| +16628|AAAAAAAAEPAEAAAA|1|College |Street|Suite Q|Sutton|Pontotoc County|MS|55413|United States|-6|single family| +16629|AAAAAAAAFPAEAAAA||Maple ||Suite 140|Salem|Louisa County|||United States||| +16630|AAAAAAAAGPAEAAAA|882|Washington Lee|RD|Suite Q|Wesley|Reno County|KS|61218|United States|-6|condo| +16631|AAAAAAAAHPAEAAAA|542|South 7th|Blvd|Suite V|Midway|Alamosa County|CO|81904|United States|-7|single family| +16632|AAAAAAAAIPAEAAAA|964|Maple College||||Bureau County|||||condo| +16633|AAAAAAAAJPAEAAAA|663|Lee |Way|Suite F|Oak Ridge|San Jacinto County|TX|78371|United States|-6|condo| +16634|AAAAAAAAKPAEAAAA|995|Spring 12th|Pkwy|Suite 270|Spring Valley|Jackson County|GA|36060|United States|-5|condo| +16635|AAAAAAAALPAEAAAA|586|3rd |Cir.|Suite B|Belmont|Todd County|MN|50191|United States|-6|apartment| +16636|AAAAAAAAMPAEAAAA|943|Fourth |Cir.|Suite 360|Mount Pleasant|Grand Forks County|ND|51933|United States|-6|condo| +16637|AAAAAAAANPAEAAAA|252|Williams |Pkwy|Suite G|Marion|Pike County|IL|60399|United States|-6|condo| +16638|AAAAAAAAOPAEAAAA|280|Willow 2nd|Ave|Suite 450|Belmont|Greene County|TN|30191|United States|-5|single family| +16639|AAAAAAAAPPAEAAAA|||Blvd||Mountain View||CT|05066||-5|single family| +16640|AAAAAAAAAABEAAAA|309|Center |Dr.|Suite 110|Clinton|Marlboro County|SC|28222|United States|-5|apartment| +16641|AAAAAAAABABEAAAA|28|Jefferson 1st|Drive|Suite 260|Oak Ridge|Dickens County|TX|78371|United States|-6|apartment| +16642|AAAAAAAACABEAAAA|702|West Wilson|Ave|Suite 210|Newport|Currituck County|NC|21521|United States|-5|single family| +16643|AAAAAAAADABEAAAA|589|1st |Cir.|Suite J|Mountain View|Schuyler County|MO|64466|United States|-6|single family| +16644|AAAAAAAAEABEAAAA|16|Miller |Ct.|Suite 270|Shady Grove|Rolette County|ND|52812|United States|-6|condo| +16645|AAAAAAAAFABEAAAA|403|Second 5th|Street|Suite 170|Union Hill|Saluda County|SC|27746|United States|-5|single family| +16646|AAAAAAAAGABEAAAA|393|Jefferson |Road|Suite 350|Berea|Starke County|IN|43804|United States|-5|condo| +16647|AAAAAAAAHABEAAAA|584|7th Center|Drive|Suite 290|Springdale|Racine County|WI|58883|United States|-6|single family| +16648|AAAAAAAAIABEAAAA|911|Willow Park|Ln|Suite T|Union|Oceana County|MI|48721|United States|-5|single family| +16649|AAAAAAAAJABEAAAA|357|Woodland |ST|Suite G|Harmony|Chase County|KS|65804|United States|-6|single family| +16650|AAAAAAAAKABEAAAA|824|Main River|Drive|Suite 50|Lincoln|Delta County|MI|41289|United States|-5|apartment| +16651|AAAAAAAALABEAAAA|996|Lake |Dr.|Suite A|Belmont|Cheboygan County|MI|40191|United States|-5|condo| +16652|AAAAAAAAMABEAAAA|249|8th |Drive|Suite H|Oak Ridge|Highland County|OH|48371|United States|-5|apartment| +16653|AAAAAAAANABEAAAA|842|Pine |Boulevard|Suite W|Clinton|Claiborne County|TN|38222|United States|-5|single family| +16654|AAAAAAAAOABEAAAA|261|Center Walnut|Cir.|Suite 340|Nichols|Camp County|TX|77940|United States|-6|single family| +16655|AAAAAAAAPABEAAAA|748|View Washington|Wy|Suite 360|Jackson|Galax city|VA|29583|United States|-5|condo| +16656|AAAAAAAAABBEAAAA|865|Park 3rd|Ct.|Suite G|Fairbanks|Fulton County|IL|66653|United States|-6|single family| +16657|AAAAAAAABBBEAAAA|184|Williams |Boulevard|Suite 130|Glenwood|Rockcastle County|KY|43511|United States|-5|apartment| +16658|AAAAAAAACBBEAAAA|888|14th |Street|Suite W|Oakwood|Delaware County|PA|10169|United States|-5|apartment| +16659|AAAAAAAADBBEAAAA|488|1st |Lane|Suite L|Green Acres|Decatur County|GA|37683|United States|-5|single family| +16660|AAAAAAAAEBBEAAAA|502|Smith South|Street|Suite Y|Pine Grove|Madison County|VA|24593|United States|-5|condo| +16661|AAAAAAAAFBBEAAAA|2|8th Park|Circle||||||United States||single family| +16662|AAAAAAAAGBBEAAAA|163|13th Park|Lane|Suite J|Mountain View|Issaquena County|MS|54466|United States|-6|condo| +16663|AAAAAAAAHBBEAAAA|270|Pine |Way|Suite A|Macedonia|Haakon County|SD|51087|United States|-7|apartment| +16664|AAAAAAAAIBBEAAAA|223|Lake |Parkway|Suite 450|Mount Pleasant|Wise County|VA|21933|United States|-5|apartment| +16665|AAAAAAAAJBBEAAAA|467|Park |Blvd|Suite 0|Unionville|Marlboro County|SC|21711|United States|-5|condo| +16666|AAAAAAAAKBBEAAAA|874|4th |Road|Suite D|Highland|Champaign County|IL|69454|United States|-6|single family| +16667|AAAAAAAALBBEAAAA|995|Maple |Way|Suite 230|Fairview|Montague County|TX|75709|United States|-6|single family| +16668|AAAAAAAAMBBEAAAA|657|Highland 13th|Boulevard|Suite B|Jackson|Tucker County|WV|29583|United States|-5|apartment| +16669|AAAAAAAANBBEAAAA|120|Cherry Mill|Pkwy|Suite C|Antioch|Alfalfa County|OK|78605|United States|-6|apartment| +16670|AAAAAAAAOBBEAAAA|565|Johnson Third|Wy|Suite D|Red Hill|Stephenson County|IL|64338|United States|-6|apartment| +16671|AAAAAAAAPBBEAAAA|834|Lincoln Oak|Street|Suite 260|Mount Zion|Colbert County|AL|38054|United States|-6|condo| +16672|AAAAAAAAACBEAAAA|181|Spring |ST|Suite 130|Arlington|Greene County|MO|66557|United States|-6|condo| +16673|AAAAAAAABCBEAAAA|939|Mill 4th|Ln|Suite A|Salem|Logan County|CO|88048|United States|-7|single family| +16674|AAAAAAAACCBEAAAA|836|Ridge |Parkway|Suite 340|Plainville|Newton County|GA|36115|United States|-5|single family| +16675|AAAAAAAADCBEAAAA|721|Green Mill|Road|Suite 290|Mount Olive|Logan County|WV|28059|United States|-5|single family| +16676|AAAAAAAAECBEAAAA|977|10th Spring|Pkwy|Suite T|Empire|Baldwin County|GA|34145|United States|-5|apartment| +16677|AAAAAAAAFCBEAAAA|830|2nd Seventh|Lane|Suite 280|Five Forks|Gogebic County|MI|42293|United States|-5|apartment| +16678|AAAAAAAAGCBEAAAA|921|Locust Fourth|Avenue|Suite 100|Buena Vista|Dawes County|NE|65752|United States|-6|single family| +16679|AAAAAAAAHCBEAAAA|778|East |Street|Suite V|Spring Valley|Somervell County|TX|76060|United States|-6|single family| +16680|AAAAAAAAICBEAAAA|528|Madison |Way|Suite 410|Thompsonville|Lexington city|VA|29651|United States|-5|single family| +16681|AAAAAAAAJCBEAAAA|543|Broadway Lakeview|Drive|Suite 230|Oak Ridge|Roger Mills County|OK|78371|United States|-6|condo| +16682|AAAAAAAAKCBEAAAA|615|Seventh Williams|Cir.|Suite C|Unionville|Tolland County|CT|02311|United States|-5|apartment| +16683|AAAAAAAALCBEAAAA|22|Meadow 2nd|Road|Suite 450|Springdale|Livingston County|NY|18883|United States|-5|condo| +16684|AAAAAAAAMCBEAAAA|134|Oak Ash|Boulevard|Suite Q|Union Hill|Jessamine County|KY|47746|United States|-6|apartment| +16685|AAAAAAAANCBEAAAA|406|Laurel 1st|Drive|Suite 290|Red Hill|Blaine County|OK|74338|United States|-6|single family| +16686|AAAAAAAAOCBEAAAA|823|1st |Avenue|Suite 30|Pleasant Valley|Coke County|TX|72477|United States|-6|single family| +16687|AAAAAAAAPCBEAAAA|340|Cedar |Ave|Suite 30|Woodville|York County|NE|64289|United States|-6|condo| +16688|AAAAAAAAADBEAAAA|1000|Adams |Parkway|Suite W|Shady Grove|Treasure County|MT|62812|United States|-7|apartment| +16689|AAAAAAAABDBEAAAA|932|Meadow |Cir.|Suite W|Wilson|Lafayette County|MO|66971|United States|-6|single family| +16690|AAAAAAAACDBEAAAA|990|Meadow |Parkway|Suite 210|Five Forks|Weston County|WY|82293|United States|-7|apartment| +16691|AAAAAAAADDBEAAAA|212|3rd |Boulevard|Suite P|Wildwood|Rappahannock County|VA|26871|United States|-5|condo| +16692|AAAAAAAAEDBEAAAA|11|Eigth |Circle|Suite B|Bridgeport|Monroe County|GA|35817|United States|-5|apartment| +16693|AAAAAAAAFDBEAAAA|971|Walnut |Ln|Suite 260|Oak Hill|Outagamie County|WI|57838|United States|-6|single family| +16694|AAAAAAAAGDBEAAAA|323|Williams Davis|Blvd|Suite 210|Chestnut Ridge|Collier County|FL|37334|United States|-5|apartment| +16695|AAAAAAAAHDBEAAAA|765|Madison Willow|Pkwy|Suite 370|Macedonia|McNairy County|TN|31087|United States|-6|apartment| +16696|AAAAAAAAIDBEAAAA|11|4th |Court|Suite W|Clinton|Parmer County|TX|78222|United States|-6|condo| +16697|AAAAAAAAJDBEAAAA|78|1st Adams|Parkway|Suite K|Oak Hill|Palo Pinto County|TX|77838|United States|-6|single family| +16698|AAAAAAAAKDBEAAAA|298|1st |Way|Suite 440|Newport|Orangeburg County|SC|21521|United States|-5|single family| +16699|AAAAAAAALDBEAAAA|480|9th |Lane|Suite G|Arlington|Graham County|AZ|86557|United States|-7|condo| +16700|AAAAAAAAMDBEAAAA|456|Fourth 3rd|Drive|Suite K|Bridgeport|Barnwell County|SC|25817|United States|-5|apartment| +16701|AAAAAAAANDBEAAAA|278|13th |Boulevard|Suite W|Belmont|Cloud County|KS|60191|United States|-6|condo| +16702|AAAAAAAAODBEAAAA|644|Willow 6th|Drive|Suite D|Edwards|Tolland County|CT|02009|United States|-5|single family| +16703|AAAAAAAAPDBEAAAA|353|Broadway Center|Blvd|Suite 420|Red Hill|Jim Hogg County|TX|74338|United States|-6|apartment| +16704|AAAAAAAAAEBEAAAA|404|Cherry |Road|Suite F|Walnut Grove|Trinity County|TX|77752|United States|-6|condo| +16705|AAAAAAAABEBEAAAA|881|Second |ST|Suite X|Woodville|Marshall County|OK|74289|United States|-6|apartment| +16706|AAAAAAAACEBEAAAA||Center |||||FL||United States|-5|single family| +16707|AAAAAAAADEBEAAAA|379|Fifth |Court|Suite C|Wilson|Fayette County|TX|76971|United States|-6|condo| +16708|AAAAAAAAEEBEAAAA|443|South South|Avenue|Suite 480|Georgetown|Salem city|VA|27057|United States|-5|condo| +16709|AAAAAAAAFEBEAAAA|800|Ash 1st|Boulevard|Suite 470|Sunnyside|Brevard County|FL|31952|United States|-5|condo| +16710|AAAAAAAAGEBEAAAA|862|1st |ST|Suite 260|Greenwood|Isanti County|MN|58828|United States|-6|apartment| +16711|AAAAAAAAHEBEAAAA|146|Forest |Street|Suite H|Lenox|Cavalier County|ND|51143|United States|-6|apartment| +16712|AAAAAAAAIEBEAAAA|670|Meadow |Way|Suite 380|Edgewood|Roane County|TN|30069|United States|-6|apartment| +16713|AAAAAAAAJEBEAAAA|365|Lee Laurel|Wy|Suite E|Union Hill|Vermilion Parish|LA|77746|United States|-6|condo| +16714|AAAAAAAAKEBEAAAA|344|Johnson 1st|Ln|Suite 30|Newport|Swift County|MN|51521|United States|-6|condo| +16715|AAAAAAAALEBEAAAA|803|Jefferson Main|Road|Suite 420|Spring Valley|Summit County|UT|86060|United States|-7|condo| +16716|AAAAAAAAMEBEAAAA|964|Highland |Cir.|Suite 460|Spring Valley|Hamilton County|TN|36060|United States|-5|apartment| +16717|AAAAAAAANEBEAAAA|215|Elm |Wy|Suite F|Woodlawn|Sumter County|GA|34098|United States|-5|single family| +16718|AAAAAAAAOEBEAAAA|658|Laurel |Drive|Suite V|Hamilton|Person County|NC|22808|United States|-5|single family| +16719|AAAAAAAAPEBEAAAA|199|Meadow Birch|Street|Suite E|Highland|Hayes County|NE|69454|United States|-6|apartment| +16720|AAAAAAAAAFBEAAAA|897|3rd Tenth|Ct.|Suite 360|Enterprise|Van Buren County|TN|31757|United States|-6|condo| +16721|AAAAAAAABFBEAAAA|447|First Chestnut|Road||||||United States||single family| +16722|AAAAAAAACFBEAAAA|490|7th |Ave|Suite 60|Deerfield|Carroll County|KY|49840|United States|-6|single family| +16723|AAAAAAAADFBEAAAA|80|||Suite P||Lee County||21675|United States||single family| +16724|AAAAAAAAEFBEAAAA|344|Smith Davis|Ave|Suite 180|Valley View|Wyoming County|NY|15124|United States|-5|apartment| +16725|AAAAAAAAFFBEAAAA|87|Forest Mill|Blvd|Suite O|Farmington|Charlottesville city|VA|29145|United States|-5|condo| +16726|AAAAAAAAGFBEAAAA|245|Davis 9th|Pkwy|Suite 460|Jamestown|Val Verde County|TX|76867|United States|-6|condo| +16727|AAAAAAAAHFBEAAAA|6|Cedar |Street|Suite 180|Belmont|Shiawassee County|MI|40191|United States|-5|single family| +16728|AAAAAAAAIFBEAAAA|124|Hill Sunset|RD|Suite 410|Valley View|Brantley County|GA|35124|United States|-5|single family| +16729|AAAAAAAAJFBEAAAA|231|Park 10th|Road|Suite X|Cedar Grove|Boone County|AR|70411|United States|-6|condo| +16730|AAAAAAAAKFBEAAAA|413|Forest |Boulevard|Suite 200|Birmingham|Claiborne County|MS|53372|United States|-6|condo| +16731|AAAAAAAALFBEAAAA|265|6th |ST|Suite J|Shiloh|Garfield County|MT|69275|United States|-7|apartment| +16732|AAAAAAAAMFBEAAAA|619|Oak |Drive|Suite 350|Pleasant Valley|Henderson County|TN|32477|United States|-5|single family| +16733|AAAAAAAANFBEAAAA|809|Pine Valley|Road|Suite Q|Wolf Creek|Caswell County|NC|22455|United States|-5|apartment| +16734|AAAAAAAAOFBEAAAA|13|Smith |Boulevard|Suite 90|Crossroads|Jackson County|IL|60534|United States|-6|apartment| +16735|AAAAAAAAPFBEAAAA|271|15th Sycamore|Dr.|Suite C|Woodville|Manistee County|MI|44289|United States|-5|single family| +16736|AAAAAAAAAGBEAAAA|365|Main Jefferson|Ct.|Suite 20|Shady Grove|Cass County|MI|42812|United States|-5|condo| +16737|AAAAAAAABGBEAAAA|130|River Park|Circle|Suite 130|Georgetown|Queens County|NY|17057|United States|-5|single family| +16738|AAAAAAAACGBEAAAA|241|8th Oak|Boulevard|Suite M|Highland|Marion County|AL|39454|United States|-6|condo| +16739|AAAAAAAADGBEAAAA|735|12th |Avenue|Suite M|Providence|DeBaca County|NM|86614|United States|-7|condo| +16740|AAAAAAAAEGBEAAAA|430|Washington |Avenue|Suite 80|Mount Zion|Armstrong County|TX|78054|United States|-6|single family| +16741|AAAAAAAAFGBEAAAA|676|Washington 10th|Street|Suite 330|Fairview|Volusia County|FL|35709|United States|-5|condo| +16742|AAAAAAAAGGBEAAAA|155|Madison |Street|Suite P|Centerville|Jefferson County|MO|60059|United States|-6|condo| +16743|AAAAAAAAHGBEAAAA|418|Pine |Ave|Suite 360|Fairview|Scott County|VA|25709|United States|-5|condo| +16744|AAAAAAAAIGBEAAAA|5|Miller |Avenue|Suite R|Deerfield|Hot Spring County|AR|79840|United States|-6|apartment| +16745|AAAAAAAAJGBEAAAA|97|Ninth 8th|Lane|Suite S|Parkwood|Mason County|IL|61669|United States|-6|apartment| +16746|AAAAAAAAKGBEAAAA|663|5th Hickory||Suite J||||75281||-6|single family| +16747|AAAAAAAALGBEAAAA|179|2nd 11th|Way|Suite O|Union Hill|Keith County|NE|67746|United States|-7|single family| +16748|AAAAAAAAMGBEAAAA|102|4th Elm|Court|Suite P|Sulphur Springs|Upson County|GA|38354|United States|-5|condo| +16749|AAAAAAAANGBEAAAA|212|5th |Cir.|Suite I|Green Acres|Marquette County|MI|47683|United States|-5|single family| +16750|AAAAAAAAOGBEAAAA|202|1st 6th|Boulevard|Suite M|Brownsville|Lake County|CA|99310|United States|-8|condo| +16751|AAAAAAAAPGBEAAAA|628|6th Hillcrest|Blvd|Suite 250|Lakewood|Logan County|OK|78877|United States|-6|apartment| +16752|AAAAAAAAAHBEAAAA|940|First Johnson|ST|Suite 350|Crossroads|Grand Isle County|VT|01134|United States|-5|single family| +16753|AAAAAAAABHBEAAAA||||||Davis County|||United States||| +16754|AAAAAAAACHBEAAAA|686|Sunset |Avenue|Suite I|Walnut Grove|Huntingdon County|PA|17752|United States|-5|condo| +16755|AAAAAAAADHBEAAAA|954|||Suite P|White Oak|Graves County||46668|United States||| +16756|AAAAAAAAEHBEAAAA|372|Locust Fourth|Blvd|Suite 50|Belleville|Richland County|MT|62924|United States|-7|single family| +16757|AAAAAAAAFHBEAAAA|317||Drive|Suite 440||Gregg County|TX||United States|-6|condo| +16758|AAAAAAAAGHBEAAAA|655|Cherry 12th|Way|Suite 320|Bunker Hill|Pulaski County|AR|70150|United States|-6|apartment| +16759|AAAAAAAAHHBEAAAA|252|Park Walnut|Street|Suite K|Farmington|Butte County|CA|99145|United States|-8|single family| +16760|AAAAAAAAIHBEAAAA|54|Main |Wy|Suite 80|Mount Olive|Stillwater County|MT|68059|United States|-7|single family| +16761|AAAAAAAAJHBEAAAA|49|Lincoln |Way|Suite 350|Newtown|Warren County|TN|31749|United States|-6|condo| +16762|AAAAAAAAKHBEAAAA|39|Green 1st|Pkwy|Suite Y|Woodville|Franklin County|TN|34289|United States|-5|condo| +16763|AAAAAAAALHBEAAAA|825|Cherry |Dr.|Suite 160|Concord|Chase County|NE|64107|United States|-6|condo| +16764|AAAAAAAAMHBEAAAA|423||Ave|Suite 0|Springfield||WY||||condo| +16765|AAAAAAAANHBEAAAA|44|11th |Avenue|Suite M|Gilmore|Erie County|PA|15464|United States|-5|apartment| +16766|AAAAAAAAOHBEAAAA|117|Central Cedar|Road|Suite U|Belmont|Charlevoix County|MI|40191|United States|-5|apartment| +16767|AAAAAAAAPHBEAAAA|326|Miller |Lane|Suite P|Mountain View|Forsyth County|GA|34466|United States|-5|single family| +16768|AAAAAAAAAIBEAAAA|973|14th Hickory|Road|Suite U|Barnes|Northampton County|VA|23788|United States|-5|apartment| +16769|AAAAAAAABIBEAAAA|499|Broadway Jefferson|Street|Suite J|Enterprise|De Witt County|IL|61757|United States|-6|condo| +16770|AAAAAAAACIBEAAAA|729|Poplar |Circle|Suite 280|Mount Zion|Natrona County|WY|88054|United States|-7|apartment| +16771|AAAAAAAADIBEAAAA|362|Lake Lake|Boulevard|Suite W|Wilson|Watauga County|NC|26971|United States|-5|apartment| +16772|AAAAAAAAEIBEAAAA|||||Oakwood|Surry County||20169|United States|-5|| +16773|AAAAAAAAFIBEAAAA|393|Maple Elm|Court|Suite 230|Clifton|Fillmore County|MN|58014|United States|-6|condo| +16774|AAAAAAAAGIBEAAAA|21|8th 3rd|Pkwy|Suite 470|Franklin|Sheridan County|WY|89101|United States|-7|apartment| +16775|AAAAAAAAHIBEAAAA|817|College |Wy|Suite Y|Bethel|Ashland County|WI|55281|United States|-6|condo| +16776|AAAAAAAAIIBEAAAA|783|Birch |Road|Suite 480|Jamestown|Chippewa County|WI|56867|United States|-6|apartment| +16777|AAAAAAAAJIBEAAAA|437|10th |Avenue|Suite J|Oak Grove|Parke County|IN|48370|United States|-5|single family| +16778|AAAAAAAAKIBEAAAA|893|First Cedar|RD|Suite 370|Shady Grove|Mathews County|VA|22812|United States|-5|apartment| +16779|AAAAAAAALIBEAAAA|370|Fourth Sycamore|Ave|Suite 200|Saint George|Houghton County|MI|45281|United States|-5|apartment| +16780|AAAAAAAAMIBEAAAA|224|Oak Railroad|Wy|Suite 370|Oak Grove|Somerset County|ME|08970|United States|-5|apartment| +16781|AAAAAAAANIBEAAAA|165|14th |Lane|Suite C|Woodville|Tillamook County|OR|94289|United States|-8|single family| +16782|AAAAAAAAOIBEAAAA||Willow Hickory|Ln|Suite K||||98371|United States|-8|single family| +16783|AAAAAAAAPIBEAAAA|||Drive|Suite E|Green Acres|||97683|United States||| +16784|AAAAAAAAAJBEAAAA|300|Highland |Cir.|Suite X|Red Hill|Custer County|NE|64338|United States|-6|single family| +16785|AAAAAAAABJBEAAAA|501|Poplar |ST|Suite F|Mount Olive|Thayer County|NE|68059|United States|-7|condo| +16786|AAAAAAAACJBEAAAA|953|Main |ST|Suite Y|Bridgeport|Dinwiddie County|VA|25817|United States|-5|single family| +16787|AAAAAAAADJBEAAAA||2nd ||||||78354|United States||apartment| +16788|AAAAAAAAEJBEAAAA|601|12th |ST|Suite 120|Edgewood|Anderson County|TX|70069|United States|-6|condo| +16789|AAAAAAAAFJBEAAAA|508|Main |Parkway|Suite 90|Gum Springs|Bossier Parish|LA|72106|United States|-6|condo| +16790|AAAAAAAAGJBEAAAA|642|Tenth Second|Court|Suite U|Midway|Kane County|UT|81904|United States|-7|apartment| +16791|AAAAAAAAHJBEAAAA|221|Valley |Circle|Suite S|Newtown|Berkeley County|WV|21749|United States|-5|condo| +16792|AAAAAAAAIJBEAAAA|531|Main |Street|Suite C|Mountain View|Taylor County|IA|54466|United States|-6|single family| +16793|AAAAAAAAJJBEAAAA|16|3rd Smith|Drive|Suite X|Fisher|Carroll County|KY|42819|United States|-6|apartment| +16794|AAAAAAAAKJBEAAAA|619|2nd Willow|Wy|Suite 370|Salem|Lincoln County|WY|88048|United States|-7|single family| +16795|AAAAAAAALJBEAAAA|161|Meadow |Court|Suite M|Marion|Greenwood County|SC|20399|United States|-5|single family| +16796|AAAAAAAAMJBEAAAA|304|12th |Ave|Suite 320|Concord|Harrison County|IA|54107|United States|-6|apartment| +16797|AAAAAAAANJBEAAAA|204|7th Birch|Ave|Suite 340|Franklin|Cecil County|MD|29101|United States|-5|apartment| +16798|AAAAAAAAOJBEAAAA|726|Maple 5th|Parkway|Suite Y|Mount Zion|Clay County|MN|58054|United States|-6|condo| +16799|AAAAAAAAPJBEAAAA|665|View |Way|Suite 210|Riverdale|Montezuma County|CO|89391|United States|-7|apartment| +16800|AAAAAAAAAKBEAAAA|767|Washington View|Ave|Suite O|Newport|Hardin County|OH|41521|United States|-5|condo| +16801|AAAAAAAABKBEAAAA|136|River |Court|Suite C|Green Acres|Hickman County|TN|37683|United States|-5|condo| +16802|AAAAAAAACKBEAAAA|80|Davis |Street|Suite 300|Pleasant Grove|Garza County|TX|74136|United States|-6|single family| +16803|AAAAAAAADKBEAAAA|10||||Oakland|Hood County||||-6|apartment| +16804|AAAAAAAAEKBEAAAA|419|Church |Lane|Suite 10|Wilson|Carter County|MO|66971|United States|-6|single family| +16805|AAAAAAAAFKBEAAAA|321|Thirteenth |Parkway|Suite G|Forest Hills|Mecklenburg County|VA|29237|United States|-5|apartment| +16806|AAAAAAAAGKBEAAAA|455|Elevnth 7th|Ave|Suite L|Liberty|McKinley County|NM|83451|United States|-7|apartment| +16807|AAAAAAAAHKBEAAAA|370|Second |Circle|Suite 200|Mountain View|Jackson County|OR|94466|United States|-8|condo| +16808|AAAAAAAAIKBEAAAA|8|Spring View|ST|Suite W|Oakdale|Archer County|TX|79584|United States|-6|condo| +16809|AAAAAAAAJKBEAAAA|457|Ash Walnut|Ct.|Suite Y|Plainview|Moore County|TN|33683|United States|-6|single family| +16810|AAAAAAAAKKBEAAAA|990|East |Drive|Suite A|Newtown|Lenoir County|NC|21749|United States|-5|apartment| +16811|AAAAAAAALKBEAAAA|39|Lake East|Ave|Suite F|Tabor|Arenac County|MI|48529|United States|-5|single family| +16812|AAAAAAAAMKBEAAAA|344|Poplar |Road|Suite 70|Woodville|Franklin County|PA|14289|United States|-5|apartment| +16813|AAAAAAAANKBEAAAA|489|Lee |Ln|Suite B|Bunker Hill|Harrison County|KY|40150|United States|-6|apartment| +16814|AAAAAAAAOKBEAAAA|||Avenue|Suite D|||MN|57057||-6|single family| +16815|AAAAAAAAPKBEAAAA|955|Hickory |Court|Suite 20|Jackson|Monroe County|AL|39583|United States|-6|single family| +16816|AAAAAAAAALBEAAAA|842|Forest |Drive|Suite W|Glenwood|Benton County|WA|93511|United States|-8|apartment| +16817|AAAAAAAABLBEAAAA|70|Lake |Pkwy|Suite 330|Salem|Baker County|FL|38048|United States|-5|apartment| +16818|AAAAAAAACLBEAAAA|377|Woodland |Drive|Suite 90|Spring Valley|Tippecanoe County|IN|46060|United States|-5|single family| +16819|AAAAAAAADLBEAAAA|860|Green |Blvd|Suite P|Riverview|Muskegon County|MI|49003|United States|-5|condo| +16820|AAAAAAAAELBEAAAA|725|Spring |Dr.|Suite 100|Riverside|East Carroll Parish|LA|79231|United States|-6|apartment| +16821|AAAAAAAAFLBEAAAA|821|Fifth 12th|Ln|Suite W|Clifton|Fremont County|IA|58014|United States|-6|apartment| +16822|AAAAAAAAGLBEAAAA|684|Cherry |Drive|Suite F|Oakland|Banner County|NE|69843|United States|-6|condo| +16823|AAAAAAAAHLBEAAAA|875|North |Drive|Suite 130|Forest Hills|Smyth County|VA|29237|United States|-5|apartment| +16824|AAAAAAAAILBEAAAA|381|Maple Walnut|Street|Suite 400|Georgetown|Henry County|KY|47057|United States|-6|apartment| +16825|AAAAAAAAJLBEAAAA|||||Kingston|||34975||-6|apartment| +16826|AAAAAAAAKLBEAAAA|232|Oak |Blvd|Suite 200|Shiloh|Lunenburg County|VA|29275|United States|-5|apartment| +16827|AAAAAAAALLBEAAAA|135|||Suite L|||IL|61904|||| +16828|AAAAAAAAMLBEAAAA|817|Highland |Dr.|Suite R|Clinton|North Slope Borough|AK|98222|United States|-9|single family| +16829|AAAAAAAANLBEAAAA|441|2nd Wilson|Cir.|Suite S|Maple Grove|Hendry County|FL|38252|United States|-5|condo| +16830|AAAAAAAAOLBEAAAA|288|Jackson |Pkwy|Suite G|Five Forks|Sebastian County|AR|72293|United States|-6|condo| +16831|AAAAAAAAPLBEAAAA|865|Oak |Wy|Suite K|Belmont|Calhoun County|GA|30191|United States|-5|condo| +16832|AAAAAAAAAMBEAAAA|169|Maple 2nd|Ave|Suite 420|Jamestown|Marion County|GA|36867|United States|-5|apartment| +16833|AAAAAAAABMBEAAAA|196|Cherry |Circle|Suite 440|Cedar|Collin County|TX|71229|United States|-6|single family| +16834|AAAAAAAACMBEAAAA|857|4th |Ave|Suite 290|Union|Glenn County|CA|98721|United States|-8|condo| +16835|AAAAAAAADMBEAAAA|611|14th South|Blvd|Suite E|Wildwood|Culpeper County|VA|26871|United States|-5|single family| +16836|AAAAAAAAEMBEAAAA|418|Hickory |Way|Suite 60|Macedonia|Middlesex County|MA|01687|United States|-5|apartment| +16837|AAAAAAAAFMBEAAAA|747|Maple Ninth|Cir.|Suite P|Lakeside|Cedar County|IA|59532|United States|-6|condo| +16838|AAAAAAAAGMBEAAAA|763|Washington |Ave|Suite 230|Valley View|Boone County|NE|65124|United States|-6|apartment| +16839|AAAAAAAAHMBEAAAA|522|Center Mill|Ln|Suite 20|Valley View|Brookings County|SD|55124|United States|-6|single family| +16840|AAAAAAAAIMBEAAAA|839|2nd Lake|Parkway|Suite 280|Forest Hills|Monterey County|CA|99237|United States|-8|condo| +16841|AAAAAAAAJMBEAAAA|824|1st Hillcrest|Dr.|Suite 80|Green Acres|Yukon-Koyukuk Census Area|AK|97683|United States|-9|apartment| +16842|AAAAAAAAKMBEAAAA|327|Cherry |Drive|Suite 490|Oak Grove|Poweshiek County|IA|58370|United States|-6|condo| +16843|AAAAAAAALMBEAAAA|947|Oak |Parkway|Suite Q|Oakland|Warren County|IN|49843|United States|-5|single family| +16844|AAAAAAAAMMBEAAAA|38|Meadow |Street|Suite V|Wilson|Wood County|TX|76971|United States|-6|apartment| +16845|AAAAAAAANMBEAAAA|801|10th |Boulevard|Suite 120|Oakdale|Apache County|AZ|89584|United States|-7|apartment| +16846|AAAAAAAAOMBEAAAA|821|Washington |Road|Suite 330|White Oak|Lee County|TX|76668|United States|-6|apartment| +16847|AAAAAAAAPMBEAAAA|326|15th Second|Road|Suite H|Lakeview|Webb County|TX|78579|United States|-6|condo| +16848|AAAAAAAAANBEAAAA|622|Oak |Road|Suite P|Cedar Grove|Renville County|MN|50411|United States|-6|condo| +16849|AAAAAAAABNBEAAAA|937|Jefferson |Circle|Suite 400|Sunnyside|Cedar County|MO|61952|United States|-6|apartment| +16850|AAAAAAAACNBEAAAA|241|Main |Blvd|Suite X|Lenox|Pike County|MO|61143|United States|-6|apartment| +16851|AAAAAAAADNBEAAAA|272|5th |RD|Suite 260|Oakwood|Stephens County|GA|30169|United States|-5|apartment| +16852|AAAAAAAAENBEAAAA|819|Ridge 3rd|Avenue|Suite O|Shady Grove|Blaine County|MT|62812|United States|-7|condo| +16853|AAAAAAAAFNBEAAAA|542|Fifteenth Dogwood|Circle|Suite 210|Harmony|Kossuth County|IA|55804|United States|-6|condo| +16854|AAAAAAAAGNBEAAAA|963|3rd 11th|Avenue|Suite C|Providence|Cumberland County|KY|46614|United States|-6|apartment| +16855|AAAAAAAAHNBEAAAA|739|Fifth Locust|Court|Suite 450|Oakdale|Onondaga County|NY|19584|United States|-5|apartment| +16856|AAAAAAAAINBEAAAA|777|4th Main|Boulevard|Suite 310|Buena Vista|McLean County|ND|55752|United States|-6|single family| +16857|AAAAAAAAJNBEAAAA|432|Spruce |Parkway|Suite 430|Providence|Gray County|TX|76614|United States|-6|single family| +16858|AAAAAAAAKNBEAAAA|248|Spruce |Cir.|Suite F|Five Forks|Jasper County|IN|42293|United States|-5|single family| +16859|AAAAAAAALNBEAAAA|379|Chestnut |Blvd|Suite 140|Belmont|Monroe County|IL|60191|United States|-6|apartment| +16860|AAAAAAAAMNBEAAAA|921|Twelfth |Drive|Suite 100|Newtown|Stark County|IL|61749|United States|-6|single family| +16861|AAAAAAAANNBEAAAA|348|View |Boulevard|Suite 20|Greenfield|Martinsville city|VA|25038|United States|-5|condo| +16862|AAAAAAAAONBEAAAA|788|5th |ST|Suite 100|Clinton|Sully County|SD|58222|United States|-7|single family| +16863|AAAAAAAAPNBEAAAA|741|Lee |Lane|Suite Y|Perkins|Brazoria County|TX|71852|United States|-6|condo| +16864|AAAAAAAAAOBEAAAA|16|Park Eigth|Dr.|Suite X|Oakland|Perkins County|SD|59843|United States|-7|condo| +16865|AAAAAAAABOBEAAAA|317|Park |Lane|Suite Q|Newport|West Baton Rouge Parish|LA|71521|United States|-6|single family| +16866|AAAAAAAACOBEAAAA|267|Maple |Wy|Suite X|Forest Hills|Hancock County|MS|59237|United States|-6|apartment| +16867|AAAAAAAADOBEAAAA|48|Spring |Court|Suite G|Highland|Huerfano County|CO|89454|United States|-7|condo| +16868|AAAAAAAAEOBEAAAA|130|Sunset Davis|Ave|Suite R|Oakwood|Penobscot County|ME|00769|United States|-5|apartment| +16869|AAAAAAAAFOBEAAAA|831|Green 10th|Ln|Suite J|Brownsville|Douglas County|WI|59310|United States|-6|condo| +16870|AAAAAAAAGOBEAAAA|||Wy|Suite U|Deerfield|Christian County|MO||United States||| +16871|AAAAAAAAHOBEAAAA|752|Lakeview |Boulevard|Suite 0|Woodville|Chaffee County|CO|84289|United States|-7|condo| +16872|AAAAAAAAIOBEAAAA|79|3rd |Court|Suite 50|Clifton|Southampton County|VA|28014|United States|-5|apartment| +16873|AAAAAAAAJOBEAAAA|348|Broadway 8th|Dr.|Suite B|Mount Zion|Kern County|CA|98054|United States|-8|single family| +16874|AAAAAAAAKOBEAAAA|932|Maple |Avenue|Suite 410|Kingston|Washington County|OR|94975|United States|-8|single family| +16875|AAAAAAAALOBEAAAA|213|Jackson |Circle|Suite C|Lakewood|Crane County|TX|78877|United States|-6|condo| +16876|AAAAAAAAMOBEAAAA|189|River |Circle|Suite 360|Texas|Fremont County|ID|83342|United States|-7|single family| +16877|AAAAAAAANOBEAAAA|700|Johnson Maple|Ln|Suite N|Springdale|Loving County|TX|78883|United States|-6|condo| +16878|AAAAAAAAOOBEAAAA|209|First |RD|Suite 170|Forest Hills|Giles County|TN|39237|United States|-5|apartment| +16879|AAAAAAAAPOBEAAAA|246|College River|Boulevard|Suite Y|Marion|Kosciusko County|IN|40399|United States|-5|apartment| +16880|AAAAAAAAAPBEAAAA|604|2nd |Ct.|Suite C|Oakwood|Washington County|AR|70169|United States|-6|single family| +16881|AAAAAAAABPBEAAAA|801|Mill 3rd|Dr.|Suite H|Friendship|Essex County|NY|14536|United States|-5|apartment| +16882|AAAAAAAACPBEAAAA|599|Poplar Seventh|Ln|Suite 270|Bridgeport|Guadalupe County|NM|85817|United States|-7|single family| +16883|AAAAAAAADPBEAAAA|483|7th Church|Ln|Suite G|Winslow|Coffey County|KS|68525|United States|-6|apartment| +16884|AAAAAAAAEPBEAAAA|900|1st |Way|Suite 70|Farmington|El Dorado County|CA|99145|United States|-8|condo| +16885|AAAAAAAAFPBEAAAA|787|9th |ST|Suite J|Pleasant Valley|Fleming County|KY|42477|United States|-6|apartment| +16886|AAAAAAAAGPBEAAAA|950|West |Street|Suite N|Springdale|Bibb County|AL|38883|United States|-6|apartment| +16887|AAAAAAAAHPBEAAAA|374|Second West|Ave|Suite 210|Sunnyside|Lincoln County|OK|71952|United States|-6|condo| +16888|AAAAAAAAIPBEAAAA|255|Maple 5th|Ln|Suite 210|Oak Grove|Bienville Parish|LA|78370|United States|-6|single family| +16889|AAAAAAAAJPBEAAAA|92|Elm 1st|Cir.|Suite 80|Concord|Beadle County|SD|54107|United States|-6|single family| +16890|AAAAAAAAKPBEAAAA|367|Walnut |Boulevard|Suite V|Wildwood|Benton County|MO|66871|United States|-6|single family| +16891|AAAAAAAALPBEAAAA|288|Poplar |Boulevard|Suite V|Newtown|Dodge County|WI|51749|United States|-6|single family| +16892|AAAAAAAAMPBEAAAA|921|Maple |Avenue|Suite P|Oak Hill|Fremont County|CO|87838|United States|-7|single family| +16893|AAAAAAAANPBEAAAA|221|Hickory |Cir.|Suite 140|Texas|Clay County|TX|73342|United States|-6|condo| +16894|AAAAAAAAOPBEAAAA|594|8th |RD|Suite V|Newtown|McCormick County|SC|21749|United States|-5|apartment| +16895|AAAAAAAAPPBEAAAA|578|||||Vernon County|WI||United States|-6|| +16896|AAAAAAAAAACEAAAA|437|Valley Adams|Street|Suite 20|Sulphur Springs|Lake County|IL|68354|United States|-6|apartment| +16897|AAAAAAAABACEAAAA|978|8th |Cir.|Suite S|Stringtown|Sanilac County|MI|40162|United States|-5|condo| +16898|AAAAAAAACACEAAAA|843|4th 9th|Ln|Suite 130|Highland|Jackson County|IA|59454|United States|-6|condo| +16899|AAAAAAAADACEAAAA|284|Cedar |RD|Suite 410|Andover|Kearny County|KS|61639|United States|-6|condo| +16900|AAAAAAAAEACEAAAA|160|5th |Boulevard|Suite 150|Enterprise|Idaho County|ID|81757|United States|-7|condo| +16901|AAAAAAAAFACEAAAA|969|Center Tenth|Boulevard|Suite 170|Mount Vernon|Prairie County|AR|78482|United States|-6|single family| +16902|AAAAAAAAGACEAAAA|||Blvd|Suite 430|Jamestown|Franklin County|NY|16867|United States||condo| +16903|AAAAAAAAHACEAAAA|269|Oak |Drive|Suite S|Pine Grove|Dawson County|MT|64593|United States|-7|single family| +16904|AAAAAAAAIACEAAAA|261|Highland |Wy|Suite W|Fairfield|Sevier County|AR|76192|United States|-6|apartment| +16905|AAAAAAAAJACEAAAA|595|Elevnth |Lane|Suite 120|Cedar Grove|Franklin County|AR|70411|United States|-6|single family| +16906|AAAAAAAAKACEAAAA|149|Spring |Court|Suite D|Woodlawn|Wibaux County|MT|64098|United States|-7|single family| +16907|AAAAAAAALACEAAAA|817|Ash |Ct.|Suite 400|Friendship|Butler County|MO|64536|United States|-6|condo| +16908|AAAAAAAAMACEAAAA|487|Ninth Center|Dr.|Suite X|White Oak|Stevens County|KS|66668|United States|-6|apartment| +16909|AAAAAAAANACEAAAA|666|Wilson |Drive|Suite W|Belmont|Ferry County|WA|90191|United States|-8|single family| +16910|AAAAAAAAOACEAAAA|733||||||||United States|-6|condo| +16911|AAAAAAAAPACEAAAA|218|Center 3rd|Circle|Suite 410|Newtown|Pottawattamie County|IA|51749|United States|-6|apartment| +16912|AAAAAAAAABCEAAAA|911|15th Elm|Boulevard|Suite C|Lebanon|Crawford County|PA|12898|United States|-5|condo| +16913|AAAAAAAABBCEAAAA|234|3rd |Parkway|Suite 340|Greenwood|Peach County|GA|38828|United States|-5|condo| +16914|AAAAAAAACBCEAAAA||Johnson |Road|Suite G|Murray|Comanche County|TX|72150|United States||| +16915|AAAAAAAADBCEAAAA|661|Church Jefferson|Ave|Suite Y|Highland Park|Jefferson County|NE|66534|United States|-7|condo| +16916|AAAAAAAAEBCEAAAA|132|Highland Johnson|Way|Suite 30|Guilford|Tooele County|UT|84408|United States|-7|apartment| +16917|AAAAAAAAFBCEAAAA|393|East Jackson|Wy|Suite L|Stringtown|Pike County|MO|60162|United States|-6|condo| +16918|AAAAAAAAGBCEAAAA|240|Pine Smith|RD|Suite F|Cedar Grove|Dodge County|MN|50411|United States|-6|condo| +16919|AAAAAAAAHBCEAAAA|88|Miller |Pkwy|Suite H|Plainview|Stanly County|NC|23683|United States|-5|single family| +16920|AAAAAAAAIBCEAAAA|786|Madison Washington|Lane|Suite G|Lakeview|Dukes County|MA|09179|United States|-5|condo| +16921|AAAAAAAAJBCEAAAA|237|Elm |Dr.|Suite F|Lakewood|Perry County|PA|18877|United States|-5|condo| +16922|AAAAAAAAKBCEAAAA|156|7th River|Ln|Suite T|Oak Grove|Richland County|ND|58370|United States|-6|condo| +16923|AAAAAAAALBCEAAAA|399|Lake College|Circle|Suite C|Edwards|Roger Mills County|OK|71409|United States|-6|single family| +16924|AAAAAAAAMBCEAAAA|217|Fourth |Way|Suite H|New Hope|Garza County|TX|79431|United States|-6|apartment| +16925|AAAAAAAANBCEAAAA|655|Forest |Boulevard|Suite P|Kingston|Lubbock County|TX|74975|United States|-6|single family| +16926|AAAAAAAAOBCEAAAA|954|Third Davis|Boulevard|Suite 140|Oak Grove|Milam County|TX|78370|United States|-6|condo| +16927|AAAAAAAAPBCEAAAA||||Suite 200|||||United States|-5|| +16928|AAAAAAAAACCEAAAA|621|2nd |RD|Suite 270|Bunker Hill|Titus County|TX|70150|United States|-6|single family| +16929|AAAAAAAABCCEAAAA|2||Ave||||MO|61087|United States|-6|single family| +16930|AAAAAAAACCCEAAAA|345|Hill Washington|Ln|Suite L|Jamestown|Stevens County|MN|56867|United States|-6|single family| +16931|AAAAAAAADCCEAAAA|57|Cherry Lee|Street|Suite 320|Arlington|Lincoln County|WA|96557|United States|-8|condo| +16932|AAAAAAAAECCEAAAA|124|Park Williams|Lane|Suite 390|Springfield|Bosque County|TX|79303|United States|-6|condo| +16933|AAAAAAAAFCCEAAAA|848|Center Oak|Avenue|Suite 60|Thompsonville|Hardeman County|TX|79651|United States|-6|apartment| +16934|AAAAAAAAGCCEAAAA|60|Park Lake|Court|Suite 60|Franklin|Warren County|MO|69101|United States|-6|apartment| +16935|AAAAAAAAHCCEAAAA|649|Cedar |Street|Suite Y|Mount Vernon|Hampden County|MA|09082|United States|-5|apartment| +16936|AAAAAAAAICCEAAAA|171|Williams |Dr.|Suite A|Buena Vista|Queen Anne County|MD|25752|United States|-5|apartment| +16937|AAAAAAAAJCCEAAAA|750|Adams |Lane|Suite 330|Union|Monroe County|NY|18721|United States|-5|single family| +16938|AAAAAAAAKCCEAAAA|257||Ct.||Jackson|Hinsdale County|CO|89583|United States||| +16939|AAAAAAAALCCEAAAA|537|Main South|Circle|Suite 400|Hamilton|Union County|NJ|03408|United States|-5|apartment| +16940|AAAAAAAAMCCEAAAA|28|2nd |Wy|Suite 210|Sullivan|Montrose County|CO|80451|United States|-7|apartment| +16941|AAAAAAAANCCEAAAA|289|Valley |RD|Suite 120|Arlington|Calhoun County|WV|26557|United States|-5|condo| +16942|AAAAAAAAOCCEAAAA||11th 4th|Boulevard|Suite 360|||SC|23003||-5|apartment| +16943|AAAAAAAAPCCEAAAA|981|Adams Spruce|RD|Suite 490|Tanglewood|Buchanan County|MO|68994|United States|-6|single family| +16944|AAAAAAAAADCEAAAA|25|Willow Adams|Ln|Suite H|Centerville|Macon County|IL|60059|United States|-6|condo| +16945|AAAAAAAABDCEAAAA|192|Cedar |Cir.|Suite J|Buena Vista|Hancock County|OH|45752|United States|-5|apartment| +16946|AAAAAAAACDCEAAAA|619|Railroad Park|Boulevard|Suite Y|Murphy|Benton County|AR|72105|United States|-6|single family| +16947|AAAAAAAADDCEAAAA|327|||||Divide County|ND|55817|United States|-6|condo| +16948|AAAAAAAAEDCEAAAA|874|Main Sixth|RD|Suite N|Enterprise|La Porte County|IN|41757|United States|-5|apartment| +16949|AAAAAAAAFDCEAAAA|984|Sixth |Parkway|Suite T|Five Forks|Daviess County|MO|62293|United States|-6|apartment| +16950|AAAAAAAAGDCEAAAA|650|Main 2nd|Wy|Suite T|Riverside|Osceola County|FL|39231|United States|-5|apartment| +16951|AAAAAAAAHDCEAAAA|809|1st |Ct.|Suite Q|New Hope|Guilford County|NC|29431|United States|-5|single family| +16952|AAAAAAAAIDCEAAAA|38|2nd |Boulevard|Suite V|Jamestown|Pike County|PA|16867|United States|-5|apartment| +16953|AAAAAAAAJDCEAAAA|755|Davis |Circle|Suite G|Valley View|Clay County|KY|45124|United States|-6|condo| +16954|AAAAAAAAKDCEAAAA|906|Seventh Cedar|Circle|Suite 400|Riverside|Wilkes County|NC|29231|United States|-5|condo| +16955|AAAAAAAALDCEAAAA|647|4th |Court|Suite 320|Mount Zion|Dale County|AL|38054|United States|-6|single family| +16956|AAAAAAAAMDCEAAAA|889|Fifth |Drive|Suite L|Macedonia|Grand County|CO|81087|United States|-7|condo| +16957|AAAAAAAANDCEAAAA|339|Hillcrest |Boulevard|Suite 420|Kimball|Jeff Davis County|TX|73595|United States|-6|apartment| +16958|AAAAAAAAODCEAAAA|148|5th Park|Street|Suite 350|Walnut|Wilson County|TX|76245|United States|-6|condo| +16959|AAAAAAAAPDCEAAAA|372|Davis |Road|Suite 220|Wilson|Martin County|IN|46971|United States|-5|single family| +16960|AAAAAAAAAECEAAAA|753|Third Green|Avenue|||Holt County|||United States|-6|| +16961|AAAAAAAABECEAAAA|521|Center |Wy|Suite 140|Glenwood|Boise County|ID|83511|United States|-7|condo| +16962|AAAAAAAACECEAAAA|232|Poplar Sunset|Way|Suite 10|Wilson|Monroe County|AL|36971|United States|-6|condo| +16963|AAAAAAAADECEAAAA|596|Washington |Road|Suite 30|Johnsonville|Champaign County|OH|47745|United States|-5|apartment| +16964|AAAAAAAAEECEAAAA|461|Main |Way|Suite U|Deerfield|Sargent County|ND|59840|United States|-6|single family| +16965|AAAAAAAAFECEAAAA|126|Maple |Dr.|Suite 230|Parkwood|Jackson County|IN|41669|United States|-5|apartment| +16966|AAAAAAAAGECEAAAA|970|Pine Chestnut|Circle|Suite 10|Sunnyside|Garfield County|UT|81952|United States|-7|single family| +16967|AAAAAAAAHECEAAAA|474|River |Circle|Suite G|Maple Hill|Wirt County|WV|28095|United States|-5|apartment| +16968|AAAAAAAAIECEAAAA|492|Chestnut |Ave|Suite C|Woodland|Floyd County|IN|44854|United States|-5|condo| +16969|AAAAAAAAJECEAAAA|189|Davis Williams|Blvd|Suite 330|Deerfield|Andrews County|TX|79840|United States|-6|condo| +16970|AAAAAAAAKECEAAAA|93|Pine Seventh|Way|Suite 50|Midway|Morgan County|AL|31904|United States|-6|single family| +16971|AAAAAAAALECEAAAA|710|View Sunset|RD|Suite 150|Oakwood|Itawamba County|MS|50169|United States|-6|single family| +16972|AAAAAAAAMECEAAAA|456|6th |Ln|Suite 250|Edgewood|Greer County|OK|70069|United States|-6|condo| +16973|AAAAAAAANECEAAAA|818|View |Blvd|Suite P|Stringtown|Wyoming County|NY|10162|United States|-5|single family| +16974|AAAAAAAAOECEAAAA|565|2nd |Ave|Suite 200|Pleasant Hill|Columbia County|NY|13604|United States|-5|apartment| +16975|AAAAAAAAPECEAAAA|654|Cherry 2nd|Pkwy|Suite H|Five Forks|Mellette County|SD|52293|United States|-7|single family| +16976|AAAAAAAAAFCEAAAA|67|Hill Jefferson|Ln|Suite G|Waterloo|Marion County|GA|31675|United States|-5|condo| +16977|AAAAAAAABFCEAAAA|837|Chestnut |Way|Suite 30|Shady Grove|Hinds County|MS|52812|United States|-6|apartment| +16978|AAAAAAAACFCEAAAA|876|Mill Jackson|RD|Suite 180|Riverdale|Perry County|IN|49391|United States|-5|apartment| +16979|AAAAAAAADFCEAAAA|17|Park |Blvd|Suite G|Oak Grove|Wilkinson County|GA|38370|United States|-5|condo| +16980|AAAAAAAAEFCEAAAA|568|Main |Wy|Suite 480|Bennett|Meigs County|OH|41715|United States|-5|condo| +16981|AAAAAAAAFFCEAAAA|572||Drive||Greenfield|||85038|United States||single family| +16982|AAAAAAAAGFCEAAAA|222|4th |Parkway|Suite D|Mount Olive|Clark County|KS|68059|United States|-6|condo| +16983|AAAAAAAAHFCEAAAA|197|Woodland |Avenue|Suite 290|Mount Pleasant|Whitfield County|GA|31933|United States|-5|condo| +16984|AAAAAAAAIFCEAAAA|810|Miller Madison|Lane|Suite 180|Wayland|Garrard County|KY|45115|United States|-6|apartment| +16985|AAAAAAAAJFCEAAAA|94|Jackson |Pkwy|Suite 450|Farmington|Logan County|NE|69145|United States|-7|condo| +16986|AAAAAAAAKFCEAAAA|541|North Adams|Cir.|Suite 120|Edgewood|Adams County|PA|10069|United States|-5|single family| +16987|AAAAAAAALFCEAAAA|868|Madison |Ln|Suite Y|Harmony|Mitchell County|NC|25804|United States|-5|apartment| +16988|AAAAAAAAMFCEAAAA|600|5th Park|RD|Suite 320|Walnut Grove|Richmond County|NY|17752|United States|-5|single family| +16989|AAAAAAAANFCEAAAA|347|West Sunset|Pkwy|Suite 420|Newport|Richmond city|VA|21521|United States|-5|apartment| +16990|AAAAAAAAOFCEAAAA|327|4th Seventh|Drive|Suite D|New Hope|Arkansas County|AR|79431|United States|-6|apartment| +16991|AAAAAAAAPFCEAAAA|496|Franklin 5th|Circle|Suite F|Marion|Randolph County|AL|30399|United States|-6|condo| +16992|AAAAAAAAAGCEAAAA|800|Lakeview |Ct.|Suite F|Lakewood|Jackson County|GA|38877|United States|-5|single family| +16993|AAAAAAAABGCEAAAA|||Ave|Suite R|||AR|79310|United States||apartment| +16994|AAAAAAAACGCEAAAA|961|Dogwood Washington|Boulevard|Suite 150|Pleasant Grove|Cleburne County|AL|34136|United States|-6|single family| +16995|AAAAAAAADGCEAAAA|44|College |Boulevard|Suite B|Florence|McCreary County|KY|43394|United States|-5|apartment| +16996|AAAAAAAAEGCEAAAA|678|Lake 11th|Ave|Suite 410|Plainview|Marion County|IN|43683|United States|-5|condo| +16997|AAAAAAAAFGCEAAAA|781|Williams |Drive|Suite H|Valley View|Caldwell Parish|LA|75124|United States|-6|condo| +16998|AAAAAAAAGGCEAAAA|146|Sunset Walnut|Cir.|Suite 440|Spring Hill|McIntosh County|ND|56787|United States|-6|apartment| +16999|AAAAAAAAHGCEAAAA|21|Miller |Lane|Suite N|Wildwood|Dubois County|IN|46871|United States|-5|condo| +17000|AAAAAAAAIGCEAAAA|141|Cedar 13th|Dr.|Suite Y|Macedonia|Garfield County|MT|61087|United States|-7|condo| +17001|AAAAAAAAJGCEAAAA|432|6th Madison|Avenue|Suite 110|Point Pleasant|Todd County|KY|44749|United States|-5|condo| +17002|AAAAAAAAKGCEAAAA|298|1st |Pkwy|Suite 360|Barnes|Hunterdon County|NJ|04388|United States|-5|single family| +17003|AAAAAAAALGCEAAAA|208|10th |Pkwy|Suite 160|Crossroads|McKinley County|NM|80534|United States|-7|single family| +17004|AAAAAAAAMGCEAAAA|281|6th Woodland|Ln|Suite 300|Walnut Grove|West Carroll Parish|LA|77752|United States|-6|apartment| +17005|AAAAAAAANGCEAAAA|650|Oak |Road|Suite 270|San Jose|Morgan County|KY|48003|United States|-5|condo| +17006|AAAAAAAAOGCEAAAA|951|Park |Drive|Suite V|Sulphur Springs|Ashland County|OH|48354|United States|-5|condo| +17007|AAAAAAAAPGCEAAAA|768|Willow |Way|Suite 180|Woodville|Richland County|MT|64289|United States|-7|single family| +17008|AAAAAAAAAHCEAAAA|877|Meadow |Pkwy|Suite N|Valley View|Choctaw County|AL|35124|United States|-6|apartment| +17009|AAAAAAAABHCEAAAA|113|Eigth Cherry|Circle|Suite M|Crossroads|Wilbarger County|TX|70534|United States|-6|apartment| +17010|AAAAAAAACHCEAAAA|491|Laurel |Avenue|Suite 220|Pleasant Hill|Greenwood County|KS|63604|United States|-6|condo| +17011|AAAAAAAADHCEAAAA|894|Maple |Wy|Suite 290|Sulphur Springs|Peoria County|IL|68354|United States|-6|apartment| +17012|AAAAAAAAEHCEAAAA|27|Hill |Dr.|Suite K|Hamilton|Wise County|TX|72808|United States|-6|apartment| +17013|AAAAAAAAFHCEAAAA|692|3rd |Cir.|Suite 80|Shiloh|Aransas County|TX|79275|United States|-6|apartment| +17014|AAAAAAAAGHCEAAAA|||Ct.|Suite 410|||MO|60418||-6|apartment| +17015|AAAAAAAAHHCEAAAA|830|13th Cedar|Avenue|Suite K|Scottsville|Assumption Parish|LA|74190|United States|-6|condo| +17016|AAAAAAAAIHCEAAAA|453|Main |ST|Suite G|Lincoln|DeKalb County|AL|31289|United States|-6|single family| +17017|AAAAAAAAJHCEAAAA|12|Fourth 12th|Pkwy|Suite 120|Mountain View|Maricopa County|AZ|84466|United States|-7|condo| +17018|AAAAAAAAKHCEAAAA|811|View Jackson|Circle|Suite S|Woodville|Montgomery County|AL|34289|United States|-6|condo| +17019|AAAAAAAALHCEAAAA|963|Poplar Maple|Avenue|Suite 190|Bunker Hill|Okanogan County|WA|90150|United States|-8|condo| +17020|AAAAAAAAMHCEAAAA|401|3rd Spring|Ln|Suite J|Lebanon|Robeson County|NC|22898|United States|-5|apartment| +17021|AAAAAAAANHCEAAAA|411|Park |RD|Suite S|Yorktown|Kleberg County|TX|70732|United States|-6|condo| +17022|AAAAAAAAOHCEAAAA|859|Spruce Cedar|Ave|Suite 170|Union Hill|Vernon Parish|LA|77746|United States|-6|condo| +17023|AAAAAAAAPHCEAAAA|846|9th |Ct.|Suite T|Oak Ridge|Greenbrier County|WV|28371|United States|-5|apartment| +17024|AAAAAAAAAICEAAAA|959|5th |Parkway|Suite 110|Jamestown|Reagan County|TX|76867|United States|-6|condo| +17025|AAAAAAAABICEAAAA|586|Main 2nd|Ct.|Suite F|Forestville|Linn County|KS|63027|United States|-6|condo| +17026|AAAAAAAACICEAAAA|206|4th Maple|Boulevard|Suite C|Mount Olive|Habersham County|GA|38059|United States|-5|condo| +17027|AAAAAAAADICEAAAA|596|2nd North|Wy|Suite 390|Mount Olive|Monroe County|PA|18059|United States|-5|apartment| +17028|AAAAAAAAEICEAAAA|732|Park Second|ST|Suite U|Pleasant Valley|Wood County|OH|42477|United States|-5|single family| +17029|AAAAAAAAFICEAAAA|118|Spring Hill|RD|Suite W|Crossroads|Bracken County|KY|40534|United States|-6|apartment| +17030|AAAAAAAAGICEAAAA|174|Laurel |Avenue|Suite F|Liberty|Clark County|KY|43451|United States|-6|single family| +17031|AAAAAAAAHICEAAAA|470|Lincoln 1st|ST|Suite I|Five Points|Chattooga County|GA|36098|United States|-5|single family| +17032|AAAAAAAAIICEAAAA|789|10th 12th|Boulevard|Suite L|Edgewood|Logan County|ND|50069|United States|-6|condo| +17033|AAAAAAAAJICEAAAA|564|2nd Fourth|RD|Suite G|Union|Marquette County|MI|48721|United States|-5|condo| +17034|AAAAAAAAKICEAAAA|172|Main Lake|Ln|Suite E|Ashland|Muscogee County|GA|34244|United States|-5|condo| +17035|AAAAAAAALICEAAAA|993|2nd |Ct.|Suite V|Wildwood|El Dorado County|CA|96871|United States|-8|single family| +17036|AAAAAAAAMICEAAAA|177|Main 5th|Ln|Suite A|Concord|Cottonwood County|MN|54107|United States|-6|condo| +17037|AAAAAAAANICEAAAA|534|10th |Parkway|Suite N|Liberty|Newport News city|VA|23451|United States|-5|apartment| +17038|AAAAAAAAOICEAAAA|579|Poplar Hickory|Pkwy|Suite 400|Bethel|Real County|TX|75281|United States|-6|single family| +17039|AAAAAAAAPICEAAAA|816|Elm |Ave|Suite 120|Somerville|Attala County|MS|57783|United States|-6|condo| +17040|AAAAAAAAAJCEAAAA|575|Oak 4th|Circle|Suite 10|Jamestown|Adams County|IN|46867|United States|-5|condo| +17041|AAAAAAAABJCEAAAA|935|Washington Oak|Drive|Suite W|New Hope|Madison County|MS|59431|United States|-6|single family| +17042|AAAAAAAACJCEAAAA|917|Hill |Boulevard|Suite 480|Lebanon|Saline County|KS|62898|United States|-6|apartment| +17043|AAAAAAAADJCEAAAA|163|East View|ST|Suite 170|Macedonia|Sarasota County|FL|31087|United States|-5|single family| +17044|AAAAAAAAEJCEAAAA|379|Spruce |Way|Suite 170|Oak Hill|Tallapoosa County|AL|37838|United States|-6|condo| +17045|AAAAAAAAFJCEAAAA|43|West |Ct.|Suite G|Five Forks|Lincoln County|KS|62293|United States|-6|condo| +17046|AAAAAAAAGJCEAAAA|910|Sycamore |Boulevard|Suite T|Lakewood|Red Willow County|NE|68877|United States|-7|single family| +17047|AAAAAAAAHJCEAAAA|460|Ash |Circle|Suite C|Siloam|Idaho County|ID|88948|United States|-7|apartment| +17048|AAAAAAAAIJCEAAAA|553|3rd Locust|Parkway|Suite 440|Greenfield|Radford city|VA|25038|United States|-5|single family| +17049|AAAAAAAAJJCEAAAA|983|Lincoln Church|Wy|Suite 180|Hopewell|Belmont County|OH|40587|United States|-5|apartment| +17050|AAAAAAAAKJCEAAAA|538|Church 4th|ST|Suite V|Buena Vista|Waukesha County|WI|55752|United States|-6|single family| +17051|AAAAAAAALJCEAAAA|272|Seventh Smith|Avenue|Suite 40|Lakeside|Itawamba County|MS|59532|United States|-6|condo| +17052|AAAAAAAAMJCEAAAA|541|Highland Fourth|Circle|Suite N|Newport|Carroll County|MO|61521|United States|-6|single family| +17053|AAAAAAAANJCEAAAA|501|Elm Adams|RD|Suite C|Mount Vernon|Swift County|MN|58482|United States|-6|single family| +17054|AAAAAAAAOJCEAAAA|71|9th |Drive|Suite C|Jamestown|Washington County|KS|66867|United States|-6|single family| +17055|AAAAAAAAPJCEAAAA|165|Sunset |Wy|Suite S|Mountain View|Wilkes County|NC|24466|United States|-5|condo| +17056|AAAAAAAAAKCEAAAA|318|2nd Lincoln|Avenue|Suite M|Beech Grove|Delta County|CO|84153|United States|-7|condo| +17057|AAAAAAAABKCEAAAA|375|Eigth 3rd|Parkway|Suite K|Enterprise|Perry County|MS|51757|United States|-6|apartment| +17058|AAAAAAAACKCEAAAA|517|10th First|Cir.|Suite Y|Brownsville|Virginia Beach city|VA|29310|United States|-5|condo| +17059|AAAAAAAADKCEAAAA|871|Sixth Cherry|Blvd|Suite Y|Ashland|Fauquier County|VA|24244|United States|-5|condo| +17060|AAAAAAAAEKCEAAAA|647|Second 9th|Street|Suite I|Shady Grove|Trinity County|TX|72812|United States|-6|single family| +17061|AAAAAAAAFKCEAAAA|28|Forest View|Road|Suite Q|Greenfield|Bristol city|VA|25038|United States|-5|single family| +17062|AAAAAAAAGKCEAAAA|594|1st Johnson|Cir.|Suite A|Arlington|Sublette County|WY|86557|United States|-7|condo| +17063|AAAAAAAAHKCEAAAA|788|Spring Hickory|Street|Suite N|Hopewell|Clallam County|WA|90587|United States|-8|single family| +17064|AAAAAAAAIKCEAAAA|1|Elm |Avenue|Suite 340|Franklin|Floyd County|IN|49101|United States|-5|apartment| +17065|AAAAAAAAJKCEAAAA|794|10th |Boulevard|Suite 460|Greenfield|Pike County|AR|75038|United States|-6|single family| +17066|AAAAAAAAKKCEAAAA|169|Sunset 13th|Road|Suite T|Riverside|Morgan County|IL|69231|United States|-6|condo| +17067|AAAAAAAALKCEAAAA|532|Cherry |Boulevard|Suite T|Edgewood|Ohio County|IN|40069|United States|-5|condo| +17068|AAAAAAAAMKCEAAAA|298|4th |Road|Suite 40|Forest Hills|Lincoln County|MN|59237|United States|-6|apartment| +17069|AAAAAAAANKCEAAAA|119|Park |Court|Suite 310|Spring Hill|Steuben County|NY|16787|United States|-5|condo| +17070|AAAAAAAAOKCEAAAA|478|Washington |RD|Suite 150|Valley View|Jim Wells County|TX|75124|United States|-6|condo| +17071|AAAAAAAAPKCEAAAA|403|Second |Boulevard|Suite 350|Lakewood|Koochiching County|MN|58877|United States|-6|single family| +17072|AAAAAAAAALCEAAAA|975|Oak 4th|Boulevard|Suite T|Concord|Lorain County|OH|44107|United States|-5|apartment| +17073|AAAAAAAABLCEAAAA|653|Main |Street|Suite D|Jamestown|Troup County|GA|36867|United States|-5|single family| +17074|AAAAAAAACLCEAAAA|513|Green South|Dr.|Suite S|Waterloo|Jackson County|WI|51675|United States|-6|apartment| +17075|AAAAAAAADLCEAAAA|95|Johnson Main|Road|Suite K|Riverside|Meeker County|MN|59231|United States|-6|apartment| +17076|AAAAAAAAELCEAAAA|996|8th |RD|Suite 270|Springdale|Forest County|WI|58883|United States|-6|apartment| +17077|AAAAAAAAFLCEAAAA|294|West |Road|Suite 70|Jamestown|Beaverhead County|MT|66867|United States|-7|single family| +17078|AAAAAAAAGLCEAAAA|882|Miller Elevnth|Road|Suite 310|Spring Grove|Greenville County|SC|26719|United States|-5|condo| +17079|AAAAAAAAHLCEAAAA|183|Maple First|Ave|Suite X|Pleasant Hill|Howard County|NE|63604|United States|-7|condo| +17080|AAAAAAAAILCEAAAA|61|Elevnth Ninth|Street|Suite 130|Deerfield|Merced County|CA|99840|United States|-8|single family| +17081|AAAAAAAAJLCEAAAA|430|Lake |Blvd|Suite 0|Roxbury|Fremont County|WY|85508|United States|-7|condo| +17082|AAAAAAAAKLCEAAAA|713|9th |Street|Suite L|Unionville|Bristol Bay Borough|AK|91711|United States|-9|single family| +17083|AAAAAAAALLCEAAAA|938|Elm |Lane|Suite G|Salem|Alachua County|FL|38048|United States|-5|single family| +17084|AAAAAAAAMLCEAAAA|87|Oak Franklin|RD|Suite 350|Shiloh|Johnston County|OK|79275|United States|-6|condo| +17085|AAAAAAAANLCEAAAA|664|2nd |Ave|Suite X|Forest Hills|Johnson County|IA|59237|United States|-6|condo| +17086|AAAAAAAAOLCEAAAA|434|Meadow |Ave|Suite E|Oak Hill|Walker County|GA|37838|United States|-5|apartment| +17087|AAAAAAAAPLCEAAAA|||Boulevard|Suite I|||||United States||single family| +17088|AAAAAAAAAMCEAAAA|311|7th Central|Wy|Suite 150|Edgewood|Dent County|MO|60069|United States|-6|single family| +17089|AAAAAAAABMCEAAAA|674|East |Ct.|Suite C|Lincoln|Rock County|NE|61289|United States|-7|apartment| +17090|AAAAAAAACMCEAAAA|914|Main Smith|Cir.|Suite X|Greenwood|Delta County|MI|48828|United States|-5|apartment| +17091|AAAAAAAADMCEAAAA|922|North Washington|Drive|Suite T|Antioch|Moultrie County|IL|68605|United States|-6|condo| +17092|AAAAAAAAEMCEAAAA|137|Railroad Oak|Lane|Suite 270|Belmont|Florence County|SC|20191|United States|-5|single family| +17093|AAAAAAAAFMCEAAAA|373|Hillcrest 5th|RD|Suite 50|Oak Hill|Emery County|UT|87838|United States|-7|single family| +17094|AAAAAAAAGMCEAAAA|985|10th |RD|Suite J|Selma|Meigs County|TN|38548|United States|-6|condo| +17095|AAAAAAAAHMCEAAAA|265|Hickory |Pkwy|Suite C|Belmont|Panola County|TX|70191|United States|-6|single family| +17096|AAAAAAAAIMCEAAAA|26|Church |Ct.|Suite W|Florence|Traverse County|MN|53394|United States|-6|condo| +17097|AAAAAAAAJMCEAAAA|40|Johnson |Ln|Suite 490|Mount Vernon|Perry County|PA|18482|United States|-5|single family| +17098|AAAAAAAAKMCEAAAA|380|River Lake|Boulevard|Suite F|Lakeside|Saratoga County|NY|19532|United States|-5|apartment| +17099|AAAAAAAALMCEAAAA|65|Oak Fourth|Avenue|Suite 450|Lincoln|La Salle County|TX|71289|United States|-6|apartment| +17100|AAAAAAAAMMCEAAAA|982|Dogwood |Blvd|Suite 150|Stringtown|Warren County|NJ|00762|United States|-5|single family| +17101|AAAAAAAANMCEAAAA|490|Broadway |Court|Suite 100|Edgewood|Orange County|FL|30069|United States|-5|single family| +17102|AAAAAAAAOMCEAAAA|13|Spring First|ST|Suite 250|Maywood|Hanover County|VA|25681|United States|-5|apartment| +17103|AAAAAAAAPMCEAAAA|379|College Johnson|Blvd|Suite 20|Highland Park|Grundy County|IL|66534|United States|-6|apartment| +17104|AAAAAAAAANCEAAAA|171|1st |Wy|Suite 140|Pleasant Grove|New London County|CT|04736|United States|-5|single family| +17105|AAAAAAAABNCEAAAA|334|Railroad Pine|Ave|Suite Q|Pine Grove|Marion County|IN|44593|United States|-5|condo| +17106|AAAAAAAACNCEAAAA|674|Jackson West|Ave|Suite 390|Evans|Palo Alto County|IA|52284|United States|-6|condo| +17107|AAAAAAAADNCEAAAA|116|Oak |Way|Suite 230|Macedonia|Southampton County|VA|21087|United States|-5|condo| +17108|AAAAAAAAENCEAAAA|437|12th Oak|Cir.|Suite M|Woodville|Lewis and Clark County|MT|64289|United States|-7|single family| +17109|AAAAAAAAFNCEAAAA|778|Miller |Lane|Suite 410|Lakeview|Broadwater County|MT|68579|United States|-7|condo| +17110|AAAAAAAAGNCEAAAA|119|||Suite L||||71952|||single family| +17111|AAAAAAAAHNCEAAAA|885|Jackson |Avenue|Suite G|Greenfield|Concordia Parish|LA|75038|United States|-6|single family| +17112|AAAAAAAAINCEAAAA|620|5th |Wy|Suite 400|Oakdale|Pierce County|WI|59584|United States|-6|single family| +17113|AAAAAAAAJNCEAAAA|510|15th |Pkwy|Suite 420|Riverside|Summers County|WV|29231|United States|-5|condo| +17114|AAAAAAAAKNCEAAAA|820|Maple Madison|Avenue|Suite C|Lakeside|Aiken County|SC|29532|United States|-5|condo| +17115|AAAAAAAALNCEAAAA|109|Ridge |Street|Suite A|Clifford|Muskingum County|OH|48164|United States|-5|apartment| +17116|AAAAAAAAMNCEAAAA|507|Park |Cir.|Suite Q|Greenwood|Tipton County|IN|48828|United States|-5|apartment| +17117|AAAAAAAANNCEAAAA|240|3rd |Way|Suite 260|Fairview|Clark County|NV|85709|United States|-8|apartment| +17118|AAAAAAAAONCEAAAA|855|15th Hill|Road|Suite 180|Wildwood|Smith County|MS|56871|United States|-6|single family| +17119|AAAAAAAAPNCEAAAA|137|Hillcrest |Dr.|Suite F|Pleasant Valley|Columbiana County|OH|42477|United States|-5|apartment| +17120|AAAAAAAAAOCEAAAA|366|Wilson |Blvd|Suite 10|Edgewood|Sequatchie County|TN|30069|United States|-6|apartment| +17121|AAAAAAAABOCEAAAA|874|14th |Road|Suite R|Harmony|Henry County|KY|45804|United States|-6|apartment| +17122|AAAAAAAACOCEAAAA|514|Pine Hill|Parkway|Suite 310|Union Hill|Wilbarger County|TX|77746|United States|-6|condo| +17123|AAAAAAAADOCEAAAA|165|West Main|Boulevard|Suite A|Hillcrest|Mercer County|KY|43003|United States|-5|condo| +17124|AAAAAAAAEOCEAAAA|211|Fifth Meadow|Drive|Suite 120|Crossroads|Decatur County|KS|60534|United States|-6|apartment| +17125|AAAAAAAAFOCEAAAA|413|Hickory Chestnut|Parkway|Suite A|Oak Hill|Platte County|NE|67838|United States|-7|single family| +17126|AAAAAAAAGOCEAAAA|51|11th |Road|Suite T|Forest Hills|Putnam County|NY|19237|United States|-5|single family| +17127|AAAAAAAAHOCEAAAA|706|14th Center|Circle|Suite D|Red Hill|James City County|VA|24338|United States|-5|condo| +17128|AAAAAAAAIOCEAAAA|515|10th Ash|Lane|Suite B|Riverview|Lackawanna County|PA|19003|United States|-5|apartment| +17129|AAAAAAAAJOCEAAAA|114|Lakeview Jefferson|ST|Suite 390|Forestville|Tattnall County|GA|33027|United States|-5|apartment| +17130|AAAAAAAAKOCEAAAA|20|Green |RD|Suite 450|Providence|Walworth County|SD|56614|United States|-7|single family| +17131|AAAAAAAALOCEAAAA|500|Jackson North|Ave|Suite N|Deerfield|Monroe County|OH|49840|United States|-5|single family| +17132|AAAAAAAAMOCEAAAA|497|East Railroad|Boulevard|Suite 290|Providence|Perry County|TN|36614|United States|-6|apartment| +17133|AAAAAAAANOCEAAAA|986|Main 8th|Boulevard|Suite 480|Spring Valley|Grant County|OK|76060|United States|-6|condo| +17134|AAAAAAAAOOCEAAAA|371|Seventh 1st|Cir.|Suite Q|Walnut Grove|Lincoln County|OK|77752|United States|-6|condo| +17135|AAAAAAAAPOCEAAAA|625|Laurel Railroad|Road|Suite R|Glenwood|Jackson County|CO|83511|United States|-7|condo| +17136|AAAAAAAAAPCEAAAA|129|Park Central|ST|Suite O|Oak Grove|Berkeley County|SC|28370|United States|-5|apartment| +17137|AAAAAAAABPCEAAAA|121|Woodland 1st|Ln|Suite E|Springfield|Allendale County|SC|29303|United States|-5|condo| +17138|AAAAAAAACPCEAAAA|570|Poplar |Wy|Suite 300|Pleasant Valley|Nome Census Area|AK|92477|United States|-9|apartment| +17139|AAAAAAAADPCEAAAA|590|Hillcrest Sycamore|Dr.|Suite 220|Wilson|Trimble County|KY|46971|United States|-5|condo| +17140|AAAAAAAAEPCEAAAA|375|Center |Way|Suite L|Lincoln|Cumberland County|KY|41289|United States|-6|apartment| +17141|AAAAAAAAFPCEAAAA|287|9th |Drive|Suite 440|Mount Pleasant|Polk County|TX|71933|United States|-6|apartment| +17142|AAAAAAAAGPCEAAAA|808|2nd |Road|Suite 440|Five Points|Scott County|MN|56098|United States|-6|single family| +17143|AAAAAAAAHPCEAAAA|677|Green Washington|Wy|Suite R|Shiloh|Kingfisher County|OK|79275|United States|-6|single family| +17144|AAAAAAAAIPCEAAAA|71|12th |Ln|Suite T|Shiloh|Cherokee County|TX|79275|United States|-6|apartment| +17145|AAAAAAAAJPCEAAAA|939|Broadway 7th|Lane|Suite 330|Cedar Grove|Nevada County|CA|90411|United States|-8|condo| +17146|AAAAAAAAKPCEAAAA|349|1st |Boulevard|Suite 340|Georgetown|Broome County|NY|17057|United States|-5|apartment| +17147|AAAAAAAALPCEAAAA|39|Spring Elm|Cir.|Suite 370|Centerville|Adair County|KY|40059|United States|-6|condo| +17148|AAAAAAAAMPCEAAAA|506|15th |Parkway|Suite U|Wilson|Jack County|TX|76971|United States|-6|single family| +17149|AAAAAAAANPCEAAAA|125|Park |Pkwy|Suite F|Lewisburg|Macon County|MO|67538|United States|-6|condo| +17150|AAAAAAAAOPCEAAAA|115|3rd 1st|Avenue|Suite 150|Lucas|Grant County|WV|24554|United States|-5|apartment| +17151|AAAAAAAAPPCEAAAA|584|Church |Avenue|Suite 40|Woodland|Meade County|KY|44854|United States|-5|single family| +17152|AAAAAAAAAADEAAAA|478|First |Avenue|Suite H|Springdale|Taylor County|IA|58883|United States|-6|single family| +17153|AAAAAAAABADEAAAA|695|8th 12th|Cir.|Suite X|Blair|Westmoreland County|VA|25465|United States|-5|apartment| +17154|AAAAAAAACADEAAAA|905|Central |Circle|Suite 80|Belmont|Nemaha County|KS|60191|United States|-6|apartment| +17155|AAAAAAAADADEAAAA|696|Pine |Boulevard|Suite V|Walnut Grove|Elkhart County|IN|47752|United States|-5|condo| +17156|AAAAAAAAEADEAAAA|303|Church Elm|Parkway|Suite 170|Glendale|Tooele County|UT|83951|United States|-7|apartment| +17157|AAAAAAAAFADEAAAA|600|Hickory 5th|Way|Suite 150|Oak Ridge|Oconto County|WI|58371|United States|-6|single family| +17158|AAAAAAAAGADEAAAA|571|Mill 13th|RD|Suite Y|Salem|Bibb County|GA|38048|United States|-5|apartment| +17159|AAAAAAAAHADEAAAA|691|Main Second|Road|Suite 250|Mount Vernon|Camp County|TX|78482|United States|-6|condo| +17160|AAAAAAAAIADEAAAA|802|North |Drive|Suite Q|Centerville|Yuba County|CA|90059|United States|-8|condo| +17161|AAAAAAAAJADEAAAA|823||Blvd||Spring Hill|||66787|United States||| +17162|AAAAAAAAKADEAAAA|871|Johnson Davis|Ct.|Suite H|Oak Hill|Mineral County|CO|87838|United States|-7|condo| +17163|AAAAAAAALADEAAAA|320|South Chestnut|Pkwy|Suite 0|Lincoln|Bexar County|TX|71289|United States|-6|apartment| +17164|AAAAAAAAMADEAAAA|541|View Maple|Circle|Suite 60|Barnes|Braxton County|WV|23788|United States|-5|single family| +17165|AAAAAAAANADEAAAA|164|Dogwood 14th|Wy|Suite 250|Belmont|Charlevoix County|MI|40191|United States|-5|single family| +17166|AAAAAAAAOADEAAAA|574|12th |Boulevard|Suite I|Clifton|Tompkins County|NY|18014|United States|-5|single family| +17167|AAAAAAAAPADEAAAA|500|3rd |Wy|Suite R|Marion|Lincoln County|ID|80399|United States|-7|single family| +17168|AAAAAAAAABDEAAAA|232|8th Chestnut|Road|Suite V|Peru|Chelan County|WA|90302|United States|-8|apartment| +17169|AAAAAAAABBDEAAAA|247|Jackson |Avenue|Suite L|Georgetown|Greene County|VA|27057|United States|-5|condo| +17170|AAAAAAAACBDEAAAA|131|||Suite 40|Forest Hills|||59237|United States||| +17171|AAAAAAAADBDEAAAA|360|Cedar 12th|Boulevard|Suite 50|Lakeview|Loup County|NE|68579|United States|-7|apartment| +17172|AAAAAAAAEBDEAAAA|602|Johnson Lakeview|Blvd|Suite Q|Farmington|Jefferson County|KY|49145|United States|-6|apartment| +17173|AAAAAAAAFBDEAAAA|904|Lee |Road|Suite Y|Mount Olive|Clay County|KY|48059|United States|-6|condo| +17174|AAAAAAAAGBDEAAAA|352|Oak Smith|Ct.|Suite 240|Hillcrest|Livingston Parish|LA|73003|United States|-6|condo| +17175|AAAAAAAAHBDEAAAA|373|Davis Walnut|RD|Suite 440|Edgewood|Montgomery County|OH|40069|United States|-5|apartment| +17176|AAAAAAAAIBDEAAAA|674|River |Cir.|Suite 340|Lakeview|McLean County|KY|48579|United States|-5|single family| +17177|AAAAAAAAJBDEAAAA||Poplar 4th|Ave|Suite 90||Belknap County|NH|02121|||| +17178|AAAAAAAAKBDEAAAA|265|5th 4th|Pkwy|Suite S|Unionville|Johnson County|GA|31711|United States|-5|single family| +17179|AAAAAAAALBDEAAAA|532|Hill |Ln|Suite 420|Pleasant Valley|Butler County|MO|62477|United States|-6|apartment| +17180|AAAAAAAAMBDEAAAA|278|Fifth |Ave|Suite 310|Arlington|Acadia Parish|LA|76557|United States|-6|apartment| +17181|AAAAAAAANBDEAAAA|65|Smith 2nd|Lane|Suite O|Five Forks|Brunswick County|VA|22293|United States|-5|apartment| +17182|AAAAAAAAOBDEAAAA|123|3rd ||Suite 470|||IA|54854|United States|-6|| +17183|AAAAAAAAPBDEAAAA|248|5th |Pkwy|Suite 370|Shady Grove|Sitka Borough|AK|92812|United States|-9|apartment| +17184|AAAAAAAAACDEAAAA|473|Twelfth Elm|Ln|Suite 480|Oak Grove|Curry County|OR|98370|United States|-8|apartment| +17185|AAAAAAAABCDEAAAA|456|Spruce East|Wy|Suite U|Franklin|Ellis County|OK|79101|United States|-6|condo| +17186|AAAAAAAACCDEAAAA||Main 7th||||||44854|United States|-6|condo| +17187|AAAAAAAADCDEAAAA|268||Road|||Oscoda County|||United States||condo| +17188|AAAAAAAAECDEAAAA|917|11th |Road|Suite 490|Spring Valley|Madison Parish|LA|76060|United States|-6|single family| +17189|AAAAAAAAFCDEAAAA|467|Spring Oak|Blvd|Suite 90|Oak Ridge|Mower County|MN|58371|United States|-6|apartment| +17190|AAAAAAAAGCDEAAAA|42|Washington |Avenue|Suite B|Antioch|Bexar County|TX|78605|United States|-6|apartment| +17191|AAAAAAAAHCDEAAAA|105|Willow Ridge|Circle|Suite T|Maple Grove|Nobles County|MN|58252|United States|-6|apartment| +17192|AAAAAAAAICDEAAAA|452|North |Wy|Suite C|Fox|Box Butte County|NE|60631|United States|-6|condo| +17193|AAAAAAAAJCDEAAAA|105|15th Oak|Ln|Suite J|Highland|Henderson County|IL|69454|United States|-6|single family| +17194|AAAAAAAAKCDEAAAA|478|Green 3rd|Pkwy|Suite 130|Concord|Howard County|IN|44107|United States|-5|condo| +17195|AAAAAAAALCDEAAAA|409|Chestnut |Lane|Suite X|Edgewood|Box Elder County|UT|80069|United States|-7|condo| +17196|AAAAAAAAMCDEAAAA|763|1st |Boulevard|Suite Y|Berea|Shawnee County|KS|63804|United States|-6|single family| +17197|AAAAAAAANCDEAAAA|244|1st |Parkway|Suite 100|Providence|Dakota County|NE|66614|United States|-6|condo| +17198|AAAAAAAAOCDEAAAA|276|Oak Ridge|Blvd|Suite O|Oakwood|Gilmer County|GA|30169|United States|-5|single family| +17199|AAAAAAAAPCDEAAAA|148|2nd |Parkway|Suite 240|Brentwood|Prince George County|MD|24188|United States|-5|condo| +17200|AAAAAAAAADDEAAAA|532||ST||Fairfield|Knox County|TX|76192|United States||| +17201|AAAAAAAABDDEAAAA|390|Washington |Ave|Suite 10|Brunswick|Harrison County|TX|74642|United States|-6|apartment| +17202|AAAAAAAACDDEAAAA|649|Lake Walnut|Ct.|Suite S|Providence|Stillwater County|MT|66614|United States|-7|single family| +17203|AAAAAAAADDDEAAAA|613|Elm |RD|Suite B|Crossroads|Wake County|NC|20534|United States|-5|condo| +17204|AAAAAAAAEDDEAAAA|474|Adams 9th|Pkwy|Suite P|Wildwood|Oceana County|MI|46871|United States|-5|single family| +17205|AAAAAAAAFDDEAAAA|627|Pine |Dr.|Suite L|Highland Park|Galveston County|TX|76534|United States|-6|condo| +17206|AAAAAAAAGDDEAAAA|229|Maple |Ave|Suite 90|Riverview|Turner County|GA|39003|United States|-5|apartment| +17207|AAAAAAAAHDDEAAAA|682|East |Drive|Suite 480|Five Points|Powell County|KY|46098|United States|-5|apartment| +17208|AAAAAAAAIDDEAAAA|946|Green Park|Ln|Suite 250|Piney Grove|Amite County|MS|58274|United States|-6|condo| +17209|AAAAAAAAJDDEAAAA|641|Oak Dogwood|Parkway|Suite 170|Cedar Grove|Thomas County|NE|60411|United States|-7|apartment| +17210|AAAAAAAAKDDEAAAA|796|Forest Highland|ST|Suite 190|Waterloo|Queen Anne County|MD|21675|United States|-5|apartment| +17211|AAAAAAAALDDEAAAA|980|Jefferson |RD|Suite 380|Hubbard|Randolph County|IN|46291|United States|-5|condo| +17212|AAAAAAAAMDDEAAAA|215|Railroad |Blvd|Suite K|Mount Olive|Jackson County|WV|28059|United States|-5|single family| +17213|AAAAAAAANDDEAAAA|779|Pine Second|Dr.|Suite 310|Riverside|Seward County|NE|69231|United States|-7|apartment| +17214|AAAAAAAAODDEAAAA|452|10th |Dr.|Suite C|Greenfield|Stephens County|GA|35038|United States|-5|single family| +17215|AAAAAAAAPDDEAAAA|891|Main Cedar|Cir.|Suite V|Page|Alamance County|NC|20296|United States|-5|apartment| +17216|AAAAAAAAAEDEAAAA|877|Eigth |Ct.|Suite 270|Valley View|Buchanan County|IA|55124|United States|-6|apartment| +17217|AAAAAAAABEDEAAAA|584|First Madison|Cir.|Suite Y|Mount Olive|Washington County|VT|08659|United States|-5|single family| +17218|AAAAAAAACEDEAAAA|831|8th |Avenue|Suite Q|Harmony|Brown County|KS|65804|United States|-6|single family| +17219|AAAAAAAADEDEAAAA|81|North |ST|Suite 120|Walnut Grove|Macon County|IL|67752|United States|-6|condo| +17220|AAAAAAAAEEDEAAAA|755|7th |Ct.|Suite F|Valley View|Bell County|TX|75124|United States|-6|apartment| +17221|AAAAAAAAFEDEAAAA|500|Green Birch|Way|Suite 170|Oakdale|Defiance County|OH|49584|United States|-5|condo| +17222|AAAAAAAAGEDEAAAA|40|Meadow |Wy|Suite C|Waterloo|La Salle Parish|LA|71675|United States|-6|condo| +17223|AAAAAAAAHEDEAAAA|56|10th River|Circle|Suite Q|Mount Olive|Tangipahoa Parish|LA|78059|United States|-6|single family| +17224|AAAAAAAAIEDEAAAA|945|13th Willow|Pkwy|Suite T|Oak Grove|Blue Earth County|MN|58370|United States|-6|condo| +17225|AAAAAAAAJEDEAAAA|251|Walnut Meadow|Ln|Suite 250|Stringtown|Cherokee County|KS|60162|United States|-6|single family| +17226|AAAAAAAAKEDEAAAA|277|3rd Walnut|Road|Suite 210|Oakland|Barry County|MI|49843|United States|-5|single family| +17227|AAAAAAAALEDEAAAA|516|Second |Avenue|Suite V|Highland Park|Carter County|MO|66534|United States|-6|apartment| +17228|AAAAAAAAMEDEAAAA|822|Sycamore |Boulevard|Suite 270|Buena Vista|Charlotte County|FL|35752|United States|-5|apartment| +17229|AAAAAAAANEDEAAAA|325|North Lee|Lane|Suite 440|Concord|Kenedy County|TX|74107|United States|-6|single family| +17230|AAAAAAAAOEDEAAAA|656|Lee |Wy|Suite L|Ashland|Gibson County|TN|34244|United States|-5|apartment| +17231|AAAAAAAAPEDEAAAA|968|Third |Drive|Suite 410|Brentwood|Pulaski County|KY|44188|United States|-5|condo| +17232|AAAAAAAAAFDEAAAA|198|Miller Hill|Wy|Suite V|Bunker Hill|Armstrong County|TX|70150|United States|-6|apartment| +17233|AAAAAAAABFDEAAAA|8|10th Seventh|Ave|Suite 480|Berea|Taylor County|TX|73804|United States|-6|apartment| +17234|AAAAAAAACFDEAAAA|746|Hickory |Road|Suite H|Woodville|Adair County|MO|64289|United States|-6|single family| +17235|AAAAAAAADFDEAAAA|776|River |Street|Suite 210|Brownsville|Wood County|WI|59310|United States|-6|condo| +17236|AAAAAAAAEFDEAAAA|621|Center |Ct.|Suite 360|Oakland|Crawford County|AR|79843|United States|-6|single family| +17237|AAAAAAAAFFDEAAAA|52|Central |Drive|Suite I|Lakewood|Livingston County|IL|68877|United States|-6|single family| +17238|AAAAAAAAGFDEAAAA|984|View Poplar|Parkway|Suite N|Bunker Hill|Montezuma County|CO|80150|United States|-7|condo| +17239|AAAAAAAAHFDEAAAA|190|Lake |Avenue|Suite S|Forest Hills|Sweet Grass County|MT|69237|United States|-7|condo| +17240|AAAAAAAAIFDEAAAA|198|Ninth |Drive|Suite X|Springdale|Mercer County|NJ|09483|United States|-5|condo| +17241|AAAAAAAAJFDEAAAA|774|Locust |Circle|Suite 290|Burns|Jackson County|IN|45272|United States|-5|condo| +17242|AAAAAAAAKFDEAAAA|295|Mill |RD|Suite Q|Highland|Montague County|TX|79454|United States|-6|condo| +17243|AAAAAAAALFDEAAAA|29|Park Poplar|Pkwy|Suite 130|Glenwood|Etowah County|AL|33511|United States|-6|apartment| +17244|AAAAAAAAMFDEAAAA|680|North Maple|Street|Suite 140|Fairfield|Carroll County|VA|26192|United States|-5|apartment| +17245|AAAAAAAANFDEAAAA|95|Park |Street|Suite O|Rocky Point|Lawrence County|AL|31209|United States|-6|single family| +17246|AAAAAAAAOFDEAAAA|354|2nd 1st|Court|Suite 360|Florence|Grant County|OR|93394|United States|-8|condo| +17247|AAAAAAAAPFDEAAAA|698|Main Lee|Lane|Suite B|Clearview|Amador County|CA|95495|United States|-8|single family| +17248|AAAAAAAAAGDEAAAA|857||Court|||Callaway County||65124||-6|apartment| +17249|AAAAAAAABGDEAAAA|502|3rd |Street|Suite 450|Friendship|Geauga County|OH|44536|United States|-5|single family| +17250|AAAAAAAACGDEAAAA|599|Washington Willow|ST|Suite 440|Sunnyside|Borden County|TX|71952|United States|-6|condo| +17251|AAAAAAAADGDEAAAA|671|Willow |Street|Suite 120|Greenville|Gallatin County|KY|41387|United States|-6|apartment| +17252|AAAAAAAAEGDEAAAA|846|Pine Hillcrest|Road|Suite G|Mount Zion|Lincoln County|NM|88054|United States|-7|apartment| +17253|AAAAAAAAFGDEAAAA|319|Lincoln 1st|Pkwy|Suite A|Providence|Bonneville County|ID|86614|United States|-7|apartment| +17254|AAAAAAAAGGDEAAAA|964|River |Court|Suite Q|Golden|Brown County|MN|50411|United States|-6|single family| +17255|AAAAAAAAHGDEAAAA|60|Tenth |Street|Suite W|Unionville|Douglas County|SD|51711|United States|-6|single family| +17256|AAAAAAAAIGDEAAAA|638|5th |Cir.|Suite V|Walnut Grove|Davis County|UT|87752|United States|-7|condo| +17257|AAAAAAAAJGDEAAAA|343|Sixth |Ct.|Suite N|Ferguson|Scott County|MO|61821|United States|-6|condo| +17258|AAAAAAAAKGDEAAAA|438|Jackson |Pkwy|Suite T|Liberty|Hopkins County|TX|73451|United States|-6|apartment| +17259|AAAAAAAALGDEAAAA|180|Elm Lakeview|Boulevard|Suite 160|Springdale|Otero County|CO|88883|United States|-7|single family| +17260|AAAAAAAAMGDEAAAA|865|Oak 1st|Blvd|Suite C|Farmington|Phelps County|MO|69145|United States|-6|condo| +17261|AAAAAAAANGDEAAAA|713|Lee Ash|Pkwy|Suite 380|New Hope|Yell County|AR|79431|United States|-6|condo| +17262|AAAAAAAAOGDEAAAA|177|Lee Dogwood|ST|Suite 260|Cedar Grove|Reynolds County|MO|60411|United States|-6|condo| +17263|AAAAAAAAPGDEAAAA|991|15th |Court|Suite Y|Fisher|Butler County|KY|42819|United States|-6|condo| +17264|AAAAAAAAAHDEAAAA|847|Hickory Mill|Boulevard|Suite E|Waterloo|Lowndes County|MS|51675|United States|-6|single family| +17265|AAAAAAAABHDEAAAA|428|Hickory |Circle|Suite E|Spring Hill|Rooks County|KS|66787|United States|-6|apartment| +17266|AAAAAAAACHDEAAAA|243|Pine |Road|Suite 120|Lakewood|Preston County|WV|28877|United States|-5|single family| +17267|AAAAAAAADHDEAAAA|138|Meadow View|Court|Suite E|Midway|Kittitas County|WA|91904|United States|-8|condo| +17268|AAAAAAAAEHDEAAAA|161|Spring |Ln|Suite 270|Springfield|Spotsylvania County|VA|29303|United States|-5|apartment| +17269|AAAAAAAAFHDEAAAA|485|Hickory |ST|Suite 430|Clinton|Anoka County|MN|58222|United States|-6|single family| +17270|AAAAAAAAGHDEAAAA|733|11th North|Pkwy|Suite 260|Green Acres|Pierce County|NE|67683|United States|-7|single family| +17271|AAAAAAAAHHDEAAAA|217|Ash Ridge|Drive|Suite 210|Riverdale|Worcester County|MD|29391|United States|-5|single family| +17272|AAAAAAAAIHDEAAAA|107|Spring |Circle|Suite E|Waterloo|Lenoir County|NC|21675|United States|-5|apartment| +17273|AAAAAAAAJHDEAAAA|720|First |Lane|Suite 440|Spring Valley|Wyoming County|WV|26060|United States|-5|single family| +17274|AAAAAAAAKHDEAAAA|415|Seventh Pine|Avenue|Suite M|Buena Vista|Woodward County|OK|75752|United States|-6|single family| +17275|AAAAAAAALHDEAAAA|93|1st Tenth|Avenue|Suite L|Enterprise|Upshur County|TX|71757|United States|-6|single family| +17276|AAAAAAAAMHDEAAAA|35|Williams Church|Parkway|Suite H|Stringtown|Cobb County|GA|30162|United States|-5|apartment| +17277|AAAAAAAANHDEAAAA|856|Walnut |Ave|Suite 370|Clifton|Comanche County|KS|68014|United States|-6|apartment| +17278|AAAAAAAAOHDEAAAA|854|Elevnth 5th|Blvd|Suite K|Glenwood|Whiteside County|IL|63511|United States|-6|apartment| +17279|AAAAAAAAPHDEAAAA|889|Wilson ||Suite R|Milltown||SC||||apartment| +17280|AAAAAAAAAIDEAAAA|904|College |ST|Suite 400|Springfield|Oscoda County|MI|49303|United States|-5|apartment| +17281|AAAAAAAABIDEAAAA|437|Madison Poplar|Boulevard|Suite C|Union Hill|Elmore County|ID|87746|United States|-7|condo| +17282|AAAAAAAACIDEAAAA|232|Wilson |ST|Suite 280|Springfield|Wharton County|TX|79303|United States|-6|single family| +17283|AAAAAAAADIDEAAAA|777|Poplar View|Court|Suite 70|Jamestown|Alexandria city|VA|26867|United States|-5|apartment| +17284|AAAAAAAAEIDEAAAA|385|Ash Lake|Boulevard|Suite X|Shiloh|Lincoln County|ME|09875|United States|-5|apartment| +17285|AAAAAAAAFIDEAAAA|484|Lakeview Lincoln|Street|Suite 290|Union|Darke County|OH|48721|United States|-5|apartment| +17286|AAAAAAAAGIDEAAAA|957|Lincoln Highland|Dr.|Suite 160|Harmony|Garden County|NE|65804|United States|-6|single family| +17287|AAAAAAAAHIDEAAAA|184|Ridge |Ct.|Suite N|Springdale|Lamar County|MS|58883|United States|-6|condo| +17288|AAAAAAAAIIDEAAAA|568|Jackson |Road|Suite A|Summit|Warren County|VA|20499|United States|-5|apartment| +17289|AAAAAAAAJIDEAAAA|380|5th 1st|ST|Suite 440|Globe|Harrisonburg city|VA|29614|United States|-5|condo| +17290|AAAAAAAAKIDEAAAA|643|13th |Lane|Suite 260|Clifton|Louisa County|IA|58014|United States|-6|condo| +17291|AAAAAAAALIDEAAAA|410|2nd River|Lane|Suite N|Kingston|McKenzie County|ND|54975|United States|-6|condo| +17292|AAAAAAAAMIDEAAAA|393|Lakeview Cherry|Avenue|Suite 390|Woodland|Wabash County|IL|64854|United States|-6|single family| +17293|AAAAAAAANIDEAAAA|819|Adams Lincoln|Court|Suite E|Plainview|Magoffin County|KY|43683|United States|-5|condo| +17294|AAAAAAAAOIDEAAAA|898|North 5th|RD|Suite X|Bunker Hill|Giles County|TN|30150|United States|-5|apartment| +17295|AAAAAAAAPIDEAAAA|231|Sycamore Forest|Parkway|Suite 470|Kingston|Linn County|IA|54975|United States|-6|single family| +17296|AAAAAAAAAJDEAAAA|726|Laurel Franklin|RD|Suite 310|Jackson|Baldwin County|GA|39583|United States|-5|apartment| +17297|AAAAAAAABJDEAAAA|359|Hill |Parkway|Suite T|Oakdale|Tucker County|WV|29584|United States|-5|apartment| +17298|AAAAAAAACJDEAAAA|371|4th Sunset|Court|Suite H|Mount Olive|Stutsman County|ND|58059|United States|-6|apartment| +17299|AAAAAAAADJDEAAAA|960|Railroad Davis|Court|Suite 190|Glenwood|Lander County|NV|83511|United States|-8|condo| +17300|AAAAAAAAEJDEAAAA|543|Maple Park|Road|Suite 90|Florence|Midland County|TX|73394|United States|-6|condo| +17301|AAAAAAAAFJDEAAAA|194|Central Green|Road|Suite 360|Bethel|Madera County|CA|95281|United States|-8|apartment| +17302|AAAAAAAAGJDEAAAA|103|2nd |Ct.|Suite 200|Shiloh|Sedgwick County|KS|69275|United States|-6|single family| +17303|AAAAAAAAHJDEAAAA|594|Sunset 14th|Avenue||Forest Hills|Wilbarger County||79237|United States||condo| +17304|AAAAAAAAIJDEAAAA|972|||Suite C||||||-5|single family| +17305|AAAAAAAAJJDEAAAA|64|Walnut |Wy|Suite 140|Greenwood|Pike County|IN|48828|United States|-5|apartment| +17306|AAAAAAAAKJDEAAAA|688|8th Hill|Drive|Suite 360|Maple Grove|Page County|VA|28252|United States|-5|single family| +17307|AAAAAAAALJDEAAAA|709|Park |Blvd|Suite 470|Deerfield|Wyoming County|NY|19840|United States|-5|condo| +17308|AAAAAAAAMJDEAAAA|412|Cedar 2nd|Drive|Suite Q|Greenwood|Lafayette County|FL|38828|United States|-5|apartment| +17309|AAAAAAAANJDEAAAA|123|Maple 4th|Ct.|Suite 350|Fairfield|Buena Vista County|IA|56192|United States|-6|apartment| +17310|AAAAAAAAOJDEAAAA|466|Jackson |Avenue|Suite Q|Bridgeport|Johnson County|TX|75817|United States|-6|apartment| +17311|AAAAAAAAPJDEAAAA|655|9th 8th|Street|Suite 200|Sulphur Springs|Carson County|TX|78354|United States|-6|single family| +17312|AAAAAAAAAKDEAAAA|721|7th |Road|Suite 60|Farmington|Rio Grande County|CO|89145|United States|-7|single family| +17313|AAAAAAAABKDEAAAA|449|14th Elm|Pkwy|Suite 360|Bath|Carter County|MT|60573|United States|-7|single family| +17314|AAAAAAAACKDEAAAA|513|Second |Avenue|Suite E|Hurricane|Metcalfe County|KY|47644|United States|-5|apartment| +17315|AAAAAAAADKDEAAAA|517|Ash South|Wy|Suite F|Mount Zion|Young County|TX|78054|United States|-6|condo| +17316|AAAAAAAAEKDEAAAA|248|2nd |Court|Suite 30|Rocky Point|Franklin County|VA|21209|United States|-5|condo| +17317|AAAAAAAAFKDEAAAA|615|Madison 6th|Cir.|Suite B|Jamestown|Fallon County|MT|66867|United States|-7|apartment| +17318|AAAAAAAAGKDEAAAA|171|South Church|RD|Suite 470|Plainview|Lincoln County|NV|83683|United States|-8|apartment| +17319|AAAAAAAAHKDEAAAA|557|Park Valley|Avenue|Suite S|Springdale|Pike County|OH|48883|United States|-5|single family| +17320|AAAAAAAAIKDEAAAA|805|West |Blvd|Suite W|Five Forks|Adair County|MO|62293|United States|-6|apartment| +17321|AAAAAAAAJKDEAAAA|530|1st Mill|Road|Suite V|Fairfield|Franklin County|OH|46192|United States|-5|condo| +17322|AAAAAAAAKKDEAAAA|549|Birch South|Parkway|Suite P|Woodville|Carroll County|MO|64289|United States|-6|single family| +17323|AAAAAAAALKDEAAAA|934|Walnut Maple|Cir.|Suite G|Oak Grove|Oceana County|MI|48370|United States|-5|single family| +17324|AAAAAAAAMKDEAAAA|515|Hillcrest |Ln|Suite S|Greenfield|Washington County|AL|35038|United States|-6|apartment| +17325|AAAAAAAANKDEAAAA|546|North |Blvd|Suite 280|Bethel|Newton County|TX|75281|United States|-6|condo| +17326|AAAAAAAAOKDEAAAA|159|South |Avenue|Suite 270|Cumberland|Indiana County|PA|18971|United States|-5|single family| +17327|AAAAAAAAPKDEAAAA|218|Chestnut |Pkwy|Suite Q|Five Points|Clarke County|IA|56098|United States|-6|condo| +17328|AAAAAAAAALDEAAAA|859|East |Ave|Suite 260|Blair|Chesterfield County|VA|25465|United States|-5|apartment| +17329|AAAAAAAABLDEAAAA|623|Central |RD|Suite 10|Summit|Clark County|WI|50499|United States|-6|condo| +17330|AAAAAAAACLDEAAAA|616|Chestnut |Court|Suite M|Owens|Audrain County|MO|62324|United States|-6|condo| +17331|AAAAAAAADLDEAAAA|756|Hill |Blvd|Suite N|Bennett|Boise County|ID|81715|United States|-7|apartment| +17332|AAAAAAAAELDEAAAA|47|Valley |ST|Suite A|Franklin|Bedford County|TN|39101|United States|-5|condo| +17333|AAAAAAAAFLDEAAAA|36|Church |Road|Suite I|Cedar Grove|Fallon County|MT|60411|United States|-7|condo| +17334|AAAAAAAAGLDEAAAA|753|Central Cedar|Circle|Suite Y|Waterloo|Rock County|NE|61675|United States|-7|apartment| +17335|AAAAAAAAHLDEAAAA|575|Forest |Avenue|Suite 400|Union Hill|Mercer County|OH|47746|United States|-5|condo| +17336|AAAAAAAAILDEAAAA|730|11th Lake|Ct.|Suite D|Gary|Morton County|ND|50418|United States|-6|apartment| +17337|AAAAAAAAJLDEAAAA|123|River Elevnth|RD|Suite 60|Marion|Van Buren County|MI|40399|United States|-5|apartment| +17338|AAAAAAAAKLDEAAAA|820|Lincoln |Way|Suite E|Enterprise|Hancock County|IN|41757|United States|-5|apartment| +17339|AAAAAAAALLDEAAAA|771|Green Pine|Boulevard|Suite E|Longwood|Anoka County|MN|57021|United States|-6|condo| +17340|AAAAAAAAMLDEAAAA|270||Court||White Oak|||76668|United States||| +17341|AAAAAAAANLDEAAAA|898|Pine 15th|Street|Suite 100|Springdale|Rockingham County|NC|28883|United States|-5|apartment| +17342|AAAAAAAAOLDEAAAA|242|Johnson 9th|Drive|Suite 20|Antioch|Coleman County|TX|78605|United States|-6|single family| +17343|AAAAAAAAPLDEAAAA|525|First |Ct.|Suite 10|Centerville|Pitkin County|CO|80059|United States|-7|apartment| +17344|AAAAAAAAAMDEAAAA|375|Washington |Boulevard|Suite W|Lincoln|Lassen County|CA|91289|United States|-8|single family| +17345|AAAAAAAABMDEAAAA|26|Adams Park|Drive|Suite 130|Mount Zion|Logan County|OK|78054|United States|-6|apartment| +17346|AAAAAAAACMDEAAAA|80|2nd |ST|Suite D|Forest Hills|Hancock County|MS|59237|United States|-6|apartment| +17347|AAAAAAAADMDEAAAA|183|South Dogwood|Blvd|Suite A|Sunnyside|Borden County|TX|71952|United States|-6|condo| +17348|AAAAAAAAEMDEAAAA|583|Chestnut |Ct.|Suite Y|Lincoln|Lincoln Parish|LA|71289|United States|-6|condo| +17349|AAAAAAAAFMDEAAAA|960|North Hickory|Wy|Suite 50|Oak Grove|Linn County|MO|68370|United States|-6|condo| +17350|AAAAAAAAGMDEAAAA|107|1st |Boulevard|Suite A|Fowler|Escambia County|FL|31083|United States|-5|apartment| +17351|AAAAAAAAHMDEAAAA|167|5th |Cir.|Suite M|Newtown|Washington County|VA|21749|United States|-5|apartment| +17352|AAAAAAAAIMDEAAAA|562|Cedar |Lane|Suite 50|Arlington|Belmont County|OH|46557|United States|-5|apartment| +17353|AAAAAAAAJMDEAAAA|59|Elm Spruce|Circle|Suite 130|Summerfield|Putnam County|MO|60634|United States|-6|apartment| +17354|AAAAAAAAKMDEAAAA|378|Lake ||||||20169|United States||| +17355|AAAAAAAALMDEAAAA|404|Maple |Ln|Suite 450|Harmony|Lafayette County|AR|75804|United States|-6|apartment| +17356|AAAAAAAAMMDEAAAA|504|Elm |Boulevard|Suite W|Friendship|Obion County|TN|34536|United States|-6|apartment| +17357|AAAAAAAANMDEAAAA|549|Dogwood Forest|Dr.|Suite 470|Willow|Ida County|IA|56798|United States|-6|apartment| +17358|AAAAAAAAOMDEAAAA|642|Laurel Park|Blvd|Suite 360|Concord|Ringgold County|IA|54107|United States|-6|condo| +17359|AAAAAAAAPMDEAAAA|705||Drive|Suite A|Brownsville|||39310||-5|condo| +17360|AAAAAAAAANDEAAAA|557|1st 7th|RD|Suite 320|Oakdale|Stark County|OH|49584|United States|-5|apartment| +17361|AAAAAAAABNDEAAAA|625|Fifth North|Dr.|Suite 150|Salem|Montgomery County|PA|18048|United States|-5|apartment| +17362|AAAAAAAACNDEAAAA|958|Pine |Dr.|Suite O|Bennett|Burleigh County|ND|51715|United States|-6|condo| +17363|AAAAAAAADNDEAAAA|400|Cherry Lake|Drive|Suite B|Unionville|Thomas County|NE|61711|United States|-7|apartment| +17364|AAAAAAAAENDEAAAA|632|Park |Court|Suite 110|Lakewood|Guernsey County|OH|48877|United States|-5|apartment| +17365|AAAAAAAAFNDEAAAA|25|Park |Court|Suite V|Hillcrest|Cheyenne County|KS|63003|United States|-6|apartment| +17366|AAAAAAAAGNDEAAAA|848|15th Cherry|Blvd|Suite 480|Belmont|Charles Mix County|SD|50191|United States|-6|apartment| +17367|AAAAAAAAHNDEAAAA|837|4th Church|Wy|Suite Y|Walnut Grove|San Juan County|CO|87752|United States|-7|apartment| +17368|AAAAAAAAINDEAAAA|990|Lee |Lane|Suite 40|Pleasant Hill|Fallon County|MT|63604|United States|-7|apartment| +17369|AAAAAAAAJNDEAAAA|937|Valley Willow|Way|Suite X|Riverview|San Miguel County|NM|89003|United States|-7|single family| +17370|AAAAAAAAKNDEAAAA|174|9th Main|Cir.|Suite 140|Franklin|Shawnee County|KS|69101|United States|-6|condo| +17371|AAAAAAAALNDEAAAA|796|15th |Ln|Suite L|Somerville|Erie County|OH|47783|United States|-5|condo| +17372|AAAAAAAAMNDEAAAA|783|10th |ST|Suite R|Hamilton|Lauderdale County|MS|52808|United States|-6|single family| +17373|AAAAAAAANNDEAAAA|68|River |Ln|Suite 230|Nichols|Kodiak Island Borough|AK|97940|United States|-9|apartment| +17374|AAAAAAAAONDEAAAA|635|Cedar |Way|Suite P|Deerfield|Jackson County|KY|49840|United States|-6|apartment| +17375|AAAAAAAAPNDEAAAA|992|Hillcrest |Circle|Suite 410|Kingston|Hamilton County|TN|34975|United States|-5|apartment| +17376|AAAAAAAAAODEAAAA|563|2nd Oak|Blvd|Suite 250|Clinton|Stephens County|OK|78222|United States|-6|single family| +17377|AAAAAAAABODEAAAA||2nd Forest|Lane||Riverview|Moore County|TX|79003|United States|-6|| +17378|AAAAAAAACODEAAAA|804|2nd |Circle|Suite K|Arlington|New Castle County|DE|16557|United States|-5|condo| +17379|AAAAAAAADODEAAAA|372|Jackson |Pkwy|Suite 340|Pleasant Hill|Vernon Parish|LA|73604|United States|-6|single family| +17380|AAAAAAAAEODEAAAA|886|Spring |Cir.|Suite B|Crossroads|Pamlico County|NC|20534|United States|-5|apartment| +17381|AAAAAAAAFODEAAAA|642|12th |Wy|Suite Y|Hamilton|Montezuma County|CO|82808|United States|-7|apartment| +17382|AAAAAAAAGODEAAAA|278|Mill First|ST|Suite 480|Clifton|Noble County|OH|48014|United States|-5|apartment| +17383|AAAAAAAAHODEAAAA|821|Railroad |Pkwy|Suite 370|Colonial Heights|Beauregard Parish|LA|73425|United States|-6|condo| +17384|AAAAAAAAIODEAAAA|712|Mill |Court|Suite E|Salem|Ashe County|NC|28048|United States|-5|apartment| +17385|AAAAAAAAJODEAAAA|226|6th Pine|Road|Suite U|Shady Grove|Screven County|GA|32812|United States|-5|condo| +17386|AAAAAAAAKODEAAAA|703|11th |Boulevard|Suite 130|Woodville|Cass County|ND|54289|United States|-6|apartment| +17387|AAAAAAAALODEAAAA|741|Chestnut 2nd|Boulevard|Suite U|Macedonia|White County|AR|71087|United States|-6|single family| +17388|AAAAAAAAMODEAAAA|629|Lee |Road|Suite 250|Marion|Minidoka County|ID|80399|United States|-7|apartment| +17389|AAAAAAAANODEAAAA|560|Oak |Ct.|Suite 290|Belmont|Petersburg city|VA|20191|United States|-5|condo| +17390|AAAAAAAAOODEAAAA|676|Pine First|Lane|Suite 450|Riverdale|Johnson County|KY|49391|United States|-6|single family| +17391|AAAAAAAAPODEAAAA|428|View Park|Lane|Suite T|Unionville|Tyler County|WV|21711|United States|-5|apartment| +17392|AAAAAAAAAPDEAAAA|968|3rd |Blvd|Suite E|Maple Grove|Dade County|MO|68252|United States|-6|condo| +17393|AAAAAAAABPDEAAAA|822|Third Lake|Dr.|Suite 400|Reno|Yalobusha County|MS|50344|United States|-6|condo| +17394|AAAAAAAACPDEAAAA|756|6th Highland|RD|Suite F|Deerfield|Rowan County|KY|49840|United States|-5|apartment| +17395|AAAAAAAADPDEAAAA|850|6th |Way|Suite 140|Deerfield|Chaffee County|CO|89840|United States|-7|condo| +17396|AAAAAAAAEPDEAAAA|779|Pine Smith|Ave|Suite O|Riverdale|Lee County|AR|79391|United States|-6|single family| +17397|AAAAAAAAFPDEAAAA|722|Sunset |ST|Suite E|White Oak|Norfolk city|VA|26668|United States|-5|condo| +17398|AAAAAAAAGPDEAAAA|824|Third |Court|Suite 80|Georgetown|Hancock County|WV|27057|United States|-5|apartment| +17399|AAAAAAAAHPDEAAAA|806|Valley West|Circle|Suite H|Antioch|Pike County|OH|48605|United States|-5|apartment| +17400|AAAAAAAAIPDEAAAA|440|Church Meadow|Blvd|Suite K|Indian Village|Macon County|AL|31075|United States|-6|apartment| +17401|AAAAAAAAJPDEAAAA|88|Park Ash|Circle|Suite R|Johnsonville|San Patricio County|TX|77745|United States|-6|apartment| +17402|AAAAAAAAKPDEAAAA|32|Church |ST|Suite 460|Providence|Middlesex County|NJ|07214|United States|-5|condo| +17403|AAAAAAAALPDEAAAA|420|3rd Davis|Ave|Suite R|Bunker Hill|Haakon County|SD|50150|United States|-7|single family| +17404|AAAAAAAAMPDEAAAA|775|Hillcrest 2nd|RD|Suite 460|Riverside|Vermilion County|IL|69231|United States|-6|single family| +17405|AAAAAAAANPDEAAAA|663|First Pine|Lane|Suite 60|Mount Vernon|Brown County|IL|68482|United States|-6|apartment| +17406|AAAAAAAAOPDEAAAA|920|Maple |Cir.|Suite F|Farmersville|Mills County|TX|79305|United States|-6|apartment| +17407|AAAAAAAAPPDEAAAA|509|Hill Elm|Ct.|Suite 140|Salem|Oneida County|NY|18048|United States|-5|apartment| +17408|AAAAAAAAAAEEAAAA|182|Jackson Maple|Way|Suite H|Shady Grove|Hettinger County|ND|52812|United States|-6|condo| +17409|AAAAAAAABAEEAAAA|726|Spring |Blvd|Suite A|Pine Grove|Clayton County|IA|54593|United States|-6|apartment| +17410|AAAAAAAACAEEAAAA|879|Spring Cedar|Road|Suite 290|Woodville|Jackson County|SD|54289|United States|-7|condo| +17411|AAAAAAAADAEEAAAA|902|10th |RD|Suite D|Pleasant Valley|Alameda County|CA|92477|United States|-8|condo| +17412|AAAAAAAAEAEEAAAA|984|6th |Pkwy|Suite G|Woodlawn|Halifax County|NC|24098|United States|-5|apartment| +17413|AAAAAAAAFAEEAAAA|322|Broadway |Ln|Suite Q|Glenville|Monroe County|WV|23445|United States|-5|condo| +17414|AAAAAAAAGAEEAAAA|637|13th |Circle|Suite O|Kingston|Boyd County|NE|64975|United States|-6|condo| +17415|AAAAAAAAHAEEAAAA|351|7th 3rd|Avenue|Suite 200|Glendale|Power County|ID|83951|United States|-7|apartment| +17416|AAAAAAAAIAEEAAAA|411|1st 13th|ST|Suite 140|Friendship|Sanpete County|UT|84536|United States|-7|apartment| +17417|AAAAAAAAJAEEAAAA||1st |Ct.|Suite 20|Mountain View|Forest County|||United States||single family| +17418|AAAAAAAAKAEEAAAA|318|Hickory |Blvd|Suite O|Wilson|Logan County|CO|86971|United States|-7|single family| +17419|AAAAAAAALAEEAAAA|186|Elm Sixth|RD|Suite 140|Hamilton|Tolland County|CT|03408|United States|-5|apartment| +17420|AAAAAAAAMAEEAAAA|503|Broadway Jefferson|Lane|Suite C|Hopewell|Bullock County|AL|30587|United States|-6|single family| +17421|AAAAAAAANAEEAAAA|454|4th Miller|Blvd|Suite V|Payne|Pawnee County|KS|66134|United States|-6|single family| +17422|AAAAAAAAOAEEAAAA|47|First Mill|ST|Suite 310|Unionville|Bullock County|AL|31711|United States|-6|apartment| +17423|AAAAAAAAPAEEAAAA|352|Park 15th|Court|Suite 100|Union Hill|Washington County|IL|67746|United States|-6|condo| +17424|AAAAAAAAABEEAAAA|726|Park Cedar|Avenue|Suite H|Clinton|Frederick County|VA|28222|United States|-5|condo| +17425|AAAAAAAABBEEAAAA|916|First |Drive|Suite J|Highland|Pondera County|MT|69454|United States|-7|apartment| +17426|AAAAAAAACBEEAAAA|637|Miller Willow|Pkwy|Suite 410|Spencer|Hamlin County|SD|58165|United States|-7|single family| +17427|AAAAAAAADBEEAAAA|719|8th 7th|Ln|Suite M|Hopewell|Haralson County|GA|30587|United States|-5|single family| +17428|AAAAAAAAEBEEAAAA|61|Washington |RD|Suite V|Riverdale|Saline County|KS|69391|United States|-6|single family| +17429|AAAAAAAAFBEEAAAA|563|Maple |Circle|Suite S|Marion|Ashe County|NC|20399|United States|-5|condo| +17430|AAAAAAAAGBEEAAAA|450|Johnson Miller|Cir.|Suite O|Providence|Greenbrier County|WV|26614|United States|-5|apartment| +17431|AAAAAAAAHBEEAAAA|125|Meadow Spring|Blvd|Suite R|Wildwood|Trimble County|KY|46871|United States|-5|single family| +17432|AAAAAAAAIBEEAAAA|658|7th |Dr.|Suite W|Fairfield|Lamar County|TX|76192|United States|-6|apartment| +17433|AAAAAAAAJBEEAAAA|27|Maple Fifth|Ave|Suite 380|Belmont|Stark County|ND|50191|United States|-6|apartment| +17434|AAAAAAAAKBEEAAAA|244|Locust |Way|Suite 0|Bunker Hill|Carter County|KY|40150|United States|-6|apartment| +17435|AAAAAAAALBEEAAAA|581|Third |Dr.|Suite 100|Liberty|Ziebach County|SD|53451|United States|-6|single family| +17436|AAAAAAAAMBEEAAAA|310|Dogwood |ST|Suite J|Antioch|Beckham County|OK|78605|United States|-6|apartment| +17437|AAAAAAAANBEEAAAA|775|View |Cir.|Suite E|Stringtown|Santa Clara County|CA|90162|United States|-8|condo| +17438|AAAAAAAAOBEEAAAA|693|Spruce |Circle|Suite 60|Wildwood|Floyd County|IA|56871|United States|-6|condo| +17439|AAAAAAAAPBEEAAAA|483|Lakeview |Street|Suite 210|Riverdale|Garfield County|MT|69391|United States|-7|condo| +17440|AAAAAAAAACEEAAAA|818|Cedar Poplar|Pkwy|Suite R|Sunnyside|Jones County|IA|51952|United States|-6|single family| +17441|AAAAAAAABCEEAAAA|653|5th |Pkwy|Suite W|Five Forks|Madison County|NC|22293|United States|-5|condo| +17442|AAAAAAAACCEEAAAA|604|Spring 2nd|RD|Suite H|Clinton|Fayette County|GA|38222|United States|-5|single family| +17443|AAAAAAAADCEEAAAA|716|Hill |Drive|Suite 50|Glenwood|Stevens County|KS|63511|United States|-6|apartment| +17444|AAAAAAAAECEEAAAA|566|View |Ln|Suite H|Lakewood|Taylor County|IA|58877|United States|-6|apartment| +17445|AAAAAAAAFCEEAAAA|736|Ninth |Drive|Suite D|Riverside|Bandera County|TX|79231|United States|-6|single family| +17446|AAAAAAAAGCEEAAAA|174|Hillcrest |RD|Suite 30|Point Pleasant|Bay County|FL|34749|United States|-5|single family| +17447|AAAAAAAAHCEEAAAA|275|Park |Way|Suite 360|Spring Hill|Craig County|VA|26787|United States|-5|apartment| +17448|AAAAAAAAICEEAAAA|583|Lakeview Seventh|Parkway|Suite Q|Lakeview|Sanders County|MT|68579|United States|-7|condo| +17449|AAAAAAAAJCEEAAAA|341|Oak |Lane|Suite 410|Green Acres|Vermilion County|IL|67683|United States|-6|condo| +17450|AAAAAAAAKCEEAAAA|272|Ridge |Lane|Suite W|Jackson|Smith County|TN|39583|United States|-6|apartment| +17451|AAAAAAAALCEEAAAA|570|Maple |Boulevard|Suite C|Five Forks|Scott County|MN|52293|United States|-6|apartment| +17452|AAAAAAAAMCEEAAAA|814|College 14th|Ln|Suite Y|Lakeview|Stephens County|TX|78579|United States|-6|single family| +17453|AAAAAAAANCEEAAAA|707|Main Woodland|Ln|Suite K|Lewisburg|Newport News city|VA|27538|United States|-5|apartment| +17454|AAAAAAAAOCEEAAAA|190|Park Cedar|Dr.|Suite A|Spring Grove|Lake County|CA|96719|United States|-8|single family| +17455|AAAAAAAAPCEEAAAA|903|Miller |Drive|Suite N|Evans|Crook County|WY|82284|United States|-7|single family| +17456|AAAAAAAAADEEAAAA|671|Cedar |Parkway|Suite 400|Franklin|Latah County|ID|89101|United States|-7|single family| +17457|AAAAAAAABDEEAAAA|258|North Chestnut|Blvd|Suite H|Forest Hills|Sevier County|TN|39237|United States|-6|condo| +17458|AAAAAAAACDEEAAAA|741|Cedar |Ave|Suite 170|Omega|Lincoln County|KY|46364|United States|-5|single family| +17459|AAAAAAAADDEEAAAA|||Wy|Suite L||||||-6|condo| +17460|AAAAAAAAEDEEAAAA|897|Mill |Way|Suite 100|Woodville|Laurens County|GA|34289|United States|-5|single family| +17461|AAAAAAAAFDEEAAAA|941|River |Drive|Suite F|Spring Hill|Kootenai County|ID|86787|United States|-7|single family| +17462|AAAAAAAAGDEEAAAA|77|View Maple|Ct.|Suite D|Unionville|Washington County|IN|41711|United States|-5|apartment| +17463|AAAAAAAAHDEEAAAA|488|Hill 1st|Wy|Suite U|Ashland|Madison County|NY|14244|United States|-5|single family| +17464|AAAAAAAAIDEEAAAA|779|4th Valley|Blvd|Suite R|Kingston|Grimes County|TX|74975|United States|-6|apartment| +17465|AAAAAAAAJDEEAAAA||||Suite I||Cocke County||31952|||single family| +17466|AAAAAAAAKDEEAAAA|769|Cherry |Wy|Suite 200|Hamilton|Lyon County|KY|42808|United States|-5|apartment| +17467|AAAAAAAALDEEAAAA|282|Broadway River|Cir.|Suite H|Union|Robeson County|NC|28721|United States|-5|apartment| +17468|AAAAAAAAMDEEAAAA|528|First First|Ln|Suite R|Wildwood|Montgomery County|PA|16871|United States|-5|apartment| +17469|AAAAAAAANDEEAAAA|216|North |Ln|Suite D|Florence|Kings County|CA|93394|United States|-8|single family| +17470|AAAAAAAAODEEAAAA|657|Second Highland||||||||-7|| +17471|AAAAAAAAPDEEAAAA|236|Church Elevnth|RD|Suite Y|Woodville|Tioga County|NY|14289|United States|-5|apartment| +17472|AAAAAAAAAEEEAAAA|556|Elm Walnut|RD|Suite 60|Pleasant Hill|Oglethorpe County|GA|33604|United States|-5|condo| +17473|AAAAAAAABEEEAAAA|280|Walnut |Street|Suite 160|Fairfield|Jasper County|SC|26192|United States|-5|single family| +17474|AAAAAAAACEEEAAAA||13th |Drive|Suite 480|||||||| +17475|AAAAAAAADEEEAAAA|778|Hill 10th|RD|Suite 140|Clinton|Wayne County|UT|88222|United States|-7|apartment| +17476|AAAAAAAAEEEEAAAA|||Wy|Suite 380|Waterloo||IL|||-6|single family| +17477|AAAAAAAAFEEEAAAA|598|Forest Center|Blvd|Suite 210|Hamilton|Winnebago County|IL|62808|United States|-6|apartment| +17478|AAAAAAAAGEEEAAAA|73|View |Ave|Suite 250|Spring Hill|Isabella County|MI|46787|United States|-5|single family| +17479|AAAAAAAAHEEEAAAA|881|Maple Ridge|Dr.|Suite 280|Pleasant Hill|Lucas County|IA|53604|United States|-6|apartment| +17480|AAAAAAAAIEEEAAAA|99|Wilson |Drive|Suite 200|Flint|Hancock County|IL|68909|United States|-6|condo| +17481|AAAAAAAAJEEEAAAA|287|Hickory |Blvd|Suite H|Summit|Nelson County|KY|40499|United States|-5|apartment| +17482|AAAAAAAAKEEEAAAA|393|Hill South|Court|Suite 400|Plainview|Hennepin County|MN|53683|United States|-6|condo| +17483|AAAAAAAALEEEAAAA|720|Miller |Ave|Suite K|Edgewood|Arenac County|MI|40069|United States|-5|single family| +17484|AAAAAAAAMEEEAAAA|119|Birch |Blvd|Suite 200|Georgetown|Logan County|NE|67057|United States|-7|apartment| +17485|AAAAAAAANEEEAAAA|340|Cedar |RD|Suite 170|Mount Olive|Morehouse Parish|LA|78059|United States|-6|single family| +17486|AAAAAAAAOEEEAAAA|936|Wilson |Cir.|Suite V|Mount Olive|Montgomery County|IL|68059|United States|-6|condo| +17487|AAAAAAAAPEEEAAAA|865|Cherry |Circle|Suite 320|Riverdale|Claiborne Parish|LA|79391|United States|-6|single family| +17488|AAAAAAAAAFEEAAAA|791|Smith Spring|Road|Suite 110|Woodlawn|Fairfield County|OH|44098|United States|-5|apartment| +17489|AAAAAAAABFEEAAAA|146|Maple 9th|Way|Suite Y|Sunnyside|Livingston Parish|LA|71952|United States|-6|condo| +17490|AAAAAAAACFEEAAAA|943|First |Court|Suite 400|Forestville|Pocahontas County|WV|23027|United States|-5|condo| +17491|AAAAAAAADFEEAAAA|409|West |Avenue|Suite M|Sulphur Springs|Newton County|AR|78354|United States|-6|single family| +17492|AAAAAAAAEFEEAAAA|610|Pine |Boulevard|Suite H|Edgewood|Suffolk County|NY|10069|United States|-5|condo| +17493|AAAAAAAAFFEEAAAA|402|Sycamore |Blvd|Suite W|Red Hill|Lafayette County|MS|54338|United States|-6|apartment| +17494|AAAAAAAAGFEEAAAA|169|Willow 14th|Dr.|Suite O|Lakeside|Emmet County|MI|49532|United States|-5|single family| +17495|AAAAAAAAHFEEAAAA|890|Pine Johnson|Wy|Suite H|Maple Grove|Richland County|MT|68252|United States|-7|apartment| +17496|AAAAAAAAIFEEAAAA|145|Johnson |Pkwy|Suite M|Bennett|Washington County|ID|81715|United States|-7|single family| +17497|AAAAAAAAJFEEAAAA|275||Road|Suite L|Oakland||KY||United States||| +17498|AAAAAAAAKFEEAAAA|294|Railroad Spruce|Road|Suite 480|Oakdale|Upton County|TX|79584|United States|-6|condo| +17499|AAAAAAAALFEEAAAA|319|Third |RD|Suite 10|Pleasant Valley|Lane County|KS|62477|United States|-6|condo| +17500|AAAAAAAAMFEEAAAA|843|Railroad 11th|Boulevard|Suite V|Friendship|Jefferson County|MS|54536|United States|-6|single family| +17501|AAAAAAAANFEEAAAA|533|Oak |Circle|Suite 320|Stringtown|Williams County|OH|40162|United States|-5|condo| +17502|AAAAAAAAOFEEAAAA|491|Cherry West|Drive|Suite 10|Lebanon|Marquette County|MI|42898|United States|-5|single family| +17503|AAAAAAAAPFEEAAAA|551|Washington Elm|Drive|Suite 0|Spring Hill|Saguache County|CO|86787|United States|-7|single family| +17504|AAAAAAAAAGEEAAAA|416|Maple |Wy|Suite F|Shiloh|Heard County|GA|39275|United States|-5|apartment| +17505|AAAAAAAABGEEAAAA|408|Jefferson Williams|ST|Suite L|Highland Park|Calhoun County|GA|36534|United States|-5|single family| +17506|AAAAAAAACGEEAAAA|86|Jackson |Drive|Suite 230|Bridgeport|Dickenson County|VA|25817|United States|-5|condo| +17507|AAAAAAAADGEEAAAA|90|Maple 7th|Blvd|Suite 490|Five Forks|Hopewell city|VA|22293|United States|-5|apartment| +17508|AAAAAAAAEGEEAAAA|244|Meadow Lincoln|Wy|Suite N|Waterloo|Winnebago County|IL|61675|United States|-6|apartment| +17509|AAAAAAAAFGEEAAAA|127|South |Parkway|Suite Q|Wyoming|Chelan County|WA|90216|United States|-8|condo| +17510|AAAAAAAAGGEEAAAA|192|South |Court|Suite E|Clifton|Saline County|KS|68014|United States|-6|apartment| +17511|AAAAAAAAHGEEAAAA|117|2nd |Court|Suite C|Highland Park|Wood County|TX|76534|United States|-6|apartment| +17512|AAAAAAAAIGEEAAAA|559|Elm Twelfth|Cir.|Suite A|Hopewell|Harrison County|IN|40587|United States|-5|apartment| +17513|AAAAAAAAJGEEAAAA|212||Dr.|||Vilas County||51933|||| +17514|AAAAAAAAKGEEAAAA|770|Mill Elm|Dr.|Suite 150|Lebanon|Trigg County|KY|42898|United States|-5|apartment| +17515|AAAAAAAALGEEAAAA|87|Oak |Street|Suite 240|New Hope|Wright County|IA|59431|United States|-6|apartment| +17516|AAAAAAAAMGEEAAAA|62|2nd |Blvd|Suite F|Lakewood|Fulton County|IN|48877|United States|-5|condo| +17517|AAAAAAAANGEEAAAA|680|Hill Ridge|Ave|Suite Y|White Oak|Marquette County|MI|46668|United States|-5|condo| +17518|AAAAAAAAOGEEAAAA|275|Church Sunset|Ln|Suite 220|Hillcrest|Burlington County|NJ|03603|United States|-5|apartment| +17519|AAAAAAAAPGEEAAAA|511|Cedar Smith|Court|Suite F|Oak Grove|Hettinger County|ND|58370|United States|-6|condo| +17520|AAAAAAAAAHEEAAAA|488|1st Walnut|Parkway|Suite X|Union Hill|Cumberland County|TN|37746|United States|-5|single family| +17521|AAAAAAAABHEEAAAA|529|4th Broadway|Blvd|Suite 440|Pleasant Valley|Henry County|GA|32477|United States|-5|condo| +17522|AAAAAAAACHEEAAAA|861|Birch Miller|Pkwy|Suite 190|Mount Olive|Clay County|WV|28059|United States|-5|single family| +17523|AAAAAAAADHEEAAAA|143|Pine |Ave|Suite Y|Greenwood|Johnson County|TX|78828|United States|-6|condo| +17524|AAAAAAAAEHEEAAAA|845|15th East|RD|Suite 40|Hillcrest|Bryan County|OK|73003|United States|-6|single family| +17525|AAAAAAAAFHEEAAAA|552|Meadow |Lane|Suite G|Woodland|Dickinson County|KS|64854|United States|-6|single family| +17526|AAAAAAAAGHEEAAAA|479|Church Laurel|Road|Suite H|Edgewood|Morgan County|UT|80069|United States|-7|condo| +17527|AAAAAAAAHHEEAAAA|765|Ash |ST|Suite A|Mountain View|Leavenworth County|KS|64466|United States|-6|single family| +17528|AAAAAAAAIHEEAAAA|464|East |Ct.|Suite 210|Mount Zion|Pratt County|KS|68054|United States|-6|single family| +17529|AAAAAAAAJHEEAAAA|578|Main Cherry|Cir.|Suite 90|Macedonia|Carroll County|MD|21087|United States|-5|single family| +17530|AAAAAAAAKHEEAAAA|844|1st Walnut|Ave|Suite 130|Arlington|Mineral County|MT|66557|United States|-7|condo| +17531|AAAAAAAALHEEAAAA|789|Lake |Parkway|Suite G|Deerfield|Queens County|NY|19840|United States|-5|single family| +17532|AAAAAAAAMHEEAAAA|712|Fourth Miller|Circle|Suite R|Mount Zion|Howell County|MO|68054|United States|-6|single family| +17533|AAAAAAAANHEEAAAA|210|Pine 13th|Pkwy|Suite 40|Mount Vernon|Scotts Bluff County|NE|68482|United States|-7|apartment| +17534|AAAAAAAAOHEEAAAA|256||Avenue|||Carroll County|KY|||-6|single family| +17535|AAAAAAAAPHEEAAAA|840|South Highland|Court|Suite C|Greenville|Steuben County|NY|11387|United States|-5|apartment| +17536|AAAAAAAAAIEEAAAA|512|River |Ln|Suite 120|Bethel|Osage County|OK|75281|United States|-6|apartment| +17537|AAAAAAAABIEEAAAA|638|Lake |Ln|Suite 120|Buena Vista|Knox County|MO|65752|United States|-6|condo| +17538|AAAAAAAACIEEAAAA|160|Walnut Oak|Street|Suite W|Gladstone|Bradley County|AR|70894|United States|-6|apartment| +17539|AAAAAAAADIEEAAAA|242|West Walnut|Court|Suite S|Forest Hills|Zavala County|TX|79237|United States|-6|condo| +17540|AAAAAAAAEIEEAAAA|443|Railroad |Dr.|Suite 150|Oakland|Houston County|AL|39843|United States|-6|apartment| +17541|AAAAAAAAFIEEAAAA|490|14th Elm|Blvd|Suite Q|Newport|Dakota County|NE|61521|United States|-6|single family| +17542|AAAAAAAAGIEEAAAA|265|Cedar |Ave|Suite I|Farmington|Ottawa County|OK|79145|United States|-6|apartment| +17543|AAAAAAAAHIEEAAAA||||Suite W||Briscoe County|TX||United States|-6|| +17544|AAAAAAAAIIEEAAAA|601|Railroad |RD|Suite 100|Woodlawn|Henry County|KY|44098|United States|-6|apartment| +17545|AAAAAAAAJIEEAAAA|845|Center Maple|Avenue|Suite 0|Union Hill|Monroe County|MI|47746|United States|-5|single family| +17546|AAAAAAAAKIEEAAAA|853|Pine Cherry|Boulevard|Suite C|Franklin|Cape May County|NJ|09701|United States|-5|apartment| +17547|AAAAAAAALIEEAAAA|572|Park |Drive|Suite W|Farmington|Shelby County|KY|49145|United States|-5|single family| +17548|AAAAAAAAMIEEAAAA|135|9th Green|Boulevard|Suite E|Riverview|Alcona County|MI|49003|United States|-5|single family| +17549|AAAAAAAANIEEAAAA|899|River |Way|Suite 490|Green Acres|Phillips County|AR|77683|United States|-6|condo| +17550|AAAAAAAAOIEEAAAA|134|Hill |Blvd|Suite I|Granite|Reynolds County|MO|66284|United States|-6|condo| +17551|AAAAAAAAPIEEAAAA|11|Washington |Drive|Suite 10|Oak Grove|Adams County|CO|88370|United States|-7|single family| +17552|AAAAAAAAAJEEAAAA|322|Broadway |Circle|Suite 140|Midway|Harney County|OR|91904|United States|-8|apartment| +17553|AAAAAAAABJEEAAAA|236|Sycamore |Cir.|Suite 410|Mount Vernon|Lincoln County|SD|58482|United States|-7|single family| +17554|AAAAAAAACJEEAAAA|258|Dogwood |Street|Suite J|Florence|Renville County|ND|53394|United States|-6|apartment| +17555|AAAAAAAADJEEAAAA|473|Jackson Pine|Street|Suite H|Riverdale|Boone County|WV|29391|United States|-5|apartment| +17556|AAAAAAAAEJEEAAAA|463|Broadway |Avenue|Suite 40|Wildwood|Erie County|NY|16871|United States|-5|single family| +17557|AAAAAAAAFJEEAAAA|179|Fifth |Boulevard|Suite 70|Riverdale|Campbell County|SD|59391|United States|-6|condo| +17558|AAAAAAAAGJEEAAAA|709|Lincoln |Ct.|Suite V|Five Points|Sebastian County|AR|76098|United States|-6|single family| +17559|AAAAAAAAHJEEAAAA|570|11th Franklin|Parkway|Suite N|Union Hill|Miami County|IN|47746|United States|-5|single family| +17560|AAAAAAAAIJEEAAAA|652|First 4th|Drive|Suite G|Longwood|Clarke County|AL|37021|United States|-6|condo| +17561|AAAAAAAAJJEEAAAA|291|Jackson Lake|Ln|Suite P|Walnut Grove|Columbia County|GA|37752|United States|-5|single family| +17562|AAAAAAAAKJEEAAAA|222|Johnson Meadow|Road|Suite 490|Wilson|McCracken County|KY|46971|United States|-5|apartment| +17563|AAAAAAAALJEEAAAA|908|Maple Park|Court|Suite 260|Pleasant Hill|Lincoln County|WV|23604|United States|-5|single family| +17564|AAAAAAAAMJEEAAAA|786|7th |Lane|Suite 110|Pleasant Valley|Zavala County|TX|72477|United States|-6|single family| +17565|AAAAAAAANJEEAAAA|561|11th |Avenue|Suite K|Sulphur Springs|Riley County|KS|68354|United States|-6|condo| +17566|AAAAAAAAOJEEAAAA|706|Walnut |Boulevard|Suite O|Vance|Fayette County|PA|10268|United States|-5|apartment| +17567|AAAAAAAAPJEEAAAA|332|Hill Hill|Way|Suite O|Lakeside|Lake County|CO|89532|United States|-7|single family| +17568|AAAAAAAAAKEEAAAA|351|Washington Park|Lane|Suite 380|Shady Grove|Adams County|IN|42812|United States|-5|single family| +17569|AAAAAAAABKEEAAAA|840|Park Davis|Cir.|Suite K|Harmony|Lincoln County|TN|35804|United States|-6|condo| +17570|AAAAAAAACKEEAAAA|860|3rd 13th|Ave|Suite A|Clinton|Mercer County|PA|18222|United States|-5|single family| +17571|AAAAAAAADKEEAAAA|416|Maple |Cir.|Suite U|Kingston|Lawrence County|MS|54975|United States|-6|condo| +17572|AAAAAAAAEKEEAAAA|132|Second Spring|Road|Suite C|Farmington|Perry County|TN|39145|United States|-6|single family| +17573|AAAAAAAAFKEEAAAA|161|Meadow |Blvd|Suite S|Westgate|Garza County|TX|72366|United States|-6|condo| +17574|AAAAAAAAGKEEAAAA|917|Pine Jackson|Lane|Suite 390|Five Forks|Cass County|IA|52293|United States|-6|single family| +17575|AAAAAAAAHKEEAAAA|945|Ninth |Avenue|Suite S|Hartland|Warren County|NY|16594|United States|-5|condo| +17576|AAAAAAAAIKEEAAAA|89|5th Lincoln|ST|Suite 30|Oakwood|Jefferson County|OK|70169|United States|-6|condo| +17577|AAAAAAAAJKEEAAAA|939|Maple |Blvd|Suite K|Sullivan|Fannin County|TX|70451|United States|-6|apartment| +17578|AAAAAAAAKKEEAAAA|515|14th |Drive|Suite 430|Union Hill|Nottoway County|VA|27746|United States|-5|apartment| +17579|AAAAAAAALKEEAAAA|298|Dogwood |Drive|Suite O|Riley|Sampson County|NC|21692|United States|-5|condo| +17580|AAAAAAAAMKEEAAAA|261|Ash |Pkwy|Suite U|Newtown|Winchester city|VA|21749|United States|-5|apartment| +17581|AAAAAAAANKEEAAAA|953|Sunset 5th|Boulevard|Suite 160|Enterprise|Bedford County|VA|21757|United States|-5|apartment| +17582|AAAAAAAAOKEEAAAA|447|Center |Pkwy|Suite O|Glenwood|Bay County|MI|43511|United States|-5|apartment| +17583|AAAAAAAAPKEEAAAA|297|Fifth |Cir.|Suite A|Newtown|Clay County|FL|31749|United States|-5|apartment| +17584|AAAAAAAAALEEAAAA|468|Adams |Ave|Suite 280|Clifton|Yoakum County|TX|78014|United States|-6|apartment| +17585|AAAAAAAABLEEAAAA|136|Elm 8th|Avenue|Suite 360|Summit|Liberty County|TX|70499|United States|-6|condo| +17586|AAAAAAAACLEEAAAA|801|Main Main|Dr.|Suite 290|Springfield|Nome Census Area|AK|99303|United States|-9|condo| +17587|AAAAAAAADLEEAAAA|||Dr.||Amity||TX|||-6|single family| +17588|AAAAAAAAELEEAAAA|7|Church |Ave|Suite 150|Riverside|Rock County|WI|59231|United States|-6|condo| +17589|AAAAAAAAFLEEAAAA|641|Fourth Willow|Drive|Suite X|Pleasant Hill|Toole County|MT|63604|United States|-7|single family| +17590|AAAAAAAAGLEEAAAA|753|Jackson Laurel|Blvd|Suite 30|Hillcrest|Foster County|ND|53003|United States|-6|single family| +17591|AAAAAAAAHLEEAAAA|129|12th |Blvd|Suite U|Oak Grove|Miller County|AR|78370|United States|-6|condo| +17592|AAAAAAAAILEEAAAA|786|College Forest|Parkway|Suite A|Arlington|Talladega County|AL|36557|United States|-6|condo| +17593|AAAAAAAAJLEEAAAA|263|8th Lincoln|Parkway|Suite 180|Jamestown|Sac County|IA|56867|United States|-6|single family| +17594|AAAAAAAAKLEEAAAA|672|Tenth Sycamore|Ave|Suite R|Lakeside|Platte County|WY|89532|United States|-7|single family| +17595|AAAAAAAALLEEAAAA|107|Lee |Court|Suite 380|Lincoln|Mason County|WV|21289|United States|-5|apartment| +17596|AAAAAAAAMLEEAAAA|935|11th Elm|Drive|Suite 30|Woodville|Rio Blanco County|CO|84289|United States|-7|single family| +17597|AAAAAAAANLEEAAAA|44|Smith |Circle|Suite 340|Allison|Escambia County|FL|34167|United States|-5|condo| +17598|AAAAAAAAOLEEAAAA|37|Oak |Street|Suite O|Plainview|Osage County|KS|63683|United States|-6|condo| +17599|AAAAAAAAPLEEAAAA|717|Oak Cherry|Street|Suite 70|Liberty|Waynesboro city|VA|23451|United States|-5|single family| +17600|AAAAAAAAAMEEAAAA|583|Ridge Hillcrest|Street|Suite 110|Harmony|Pulaski County|AR|75804|United States|-6|apartment| +17601|AAAAAAAABMEEAAAA|849|Washington |RD|Suite K|Georgetown|Greene County|NY|17057|United States|-5|single family| +17602|AAAAAAAACMEEAAAA|297|Lee Sixth|Street|Suite 450|Greenfield|Nevada County|CA|95038|United States|-8|single family| +17603|AAAAAAAADMEEAAAA|535|Jackson |RD|Suite E|Antioch|Bristol County|RI|09205|United States|-5|condo| +17604|AAAAAAAAEMEEAAAA|890|Lee |Parkway|Suite B|Woodlawn|Schuyler County|NY|14098|United States|-5|single family| +17605|AAAAAAAAFMEEAAAA|38|Davis |Road|Suite D|Wildwood|Colfax County|NE|66871|United States|-6|apartment| +17606|AAAAAAAAGMEEAAAA|231|4th |Ct.|Suite S|Brownsville|Musselshell County|MT|69310|United States|-7|single family| +17607|AAAAAAAAHMEEAAAA|440|Johnson Elevnth|Dr.|Suite 180|Glendale|Cherokee County|TX|73951|United States|-6|apartment| +17608|AAAAAAAAIMEEAAAA|767|Center Mill|Dr.|Suite 60|Riverdale|Sheridan County|KS|69391|United States|-6|single family| +17609|AAAAAAAAJMEEAAAA|391|Walnut Cherry|RD|Suite 320|Providence|Smith County|TN|36614|United States|-6|apartment| +17610|AAAAAAAAKMEEAAAA|166|2nd Cedar|Court|Suite 130|Freeport|Lee County|VA|21844|United States|-5|condo| +17611|AAAAAAAALMEEAAAA|332|Ridge |Blvd|Suite 290|Red Hill|Washington County|VA|24338|United States|-5|condo| +17612|AAAAAAAAMMEEAAAA|847|Park |Pkwy|Suite 20|Marion|Miller County|MO|60399|United States|-6|apartment| +17613|AAAAAAAANMEEAAAA|942|North |RD|Suite 300|Hamilton|Dixie County|FL|32808|United States|-5|condo| +17614|AAAAAAAAOMEEAAAA|466|Church |Street|Suite W|Lakeview|Otter Tail County|MN|58579|United States|-6|condo| +17615|AAAAAAAAPMEEAAAA|217|Washington Fourth|Way|Suite 430|Riverdale|Putnam County|GA|39391|United States|-5|apartment| +17616|AAAAAAAAANEEAAAA|722|4th |Boulevard|Suite I|Friendship|Fulton County|IL|64536|United States|-6|single family| +17617|AAAAAAAABNEEAAAA|133|Walnut Cherry|Way|Suite V|Macedonia|Kemper County|MS|51087|United States|-6|condo| +17618|AAAAAAAACNEEAAAA||Park Meadow|ST|Suite G|Marion||||United States|-5|| +17619|AAAAAAAADNEEAAAA|710|Madison |Drive|Suite 120|Waterloo|Rock County|NE|61675|United States|-7|condo| +17620|AAAAAAAAENEEAAAA|955|Williams Church|Way|Suite 350|Newtown|Hamilton County|NE|61749|United States|-6|condo| +17621|AAAAAAAAFNEEAAAA|938|Spruce Jefferson|Lane|Suite 320|Highland|Boulder County|CO|89454|United States|-7|single family| +17622|AAAAAAAAGNEEAAAA|616|Maple |Avenue|Suite W|Mount Olive|Murray County|OK|78059|United States|-6|apartment| +17623|AAAAAAAAHNEEAAAA|882|9th |Boulevard|Suite I|Union|Towner County|ND|58721|United States|-6|condo| +17624|AAAAAAAAINEEAAAA|670|Broadway |Street|Suite M|Springdale|Sullivan County|NH|09483|United States|-5|single family| +17625|AAAAAAAAJNEEAAAA|230|First Park|Cir.|Suite E|Red Oak|Rolette County|ND|55018|United States|-6|condo| +17626|AAAAAAAAKNEEAAAA|297|First Hickory|Ave|Suite N|Edgewood|Chowan County|NC|20069|United States|-5|single family| +17627|AAAAAAAALNEEAAAA|88|West 5th|Road|Suite N|Antioch|Gates County|NC|28605|United States|-5|apartment| +17628|AAAAAAAAMNEEAAAA|734|Spring |Parkway|Suite J|Waterloo|Palm Beach County|FL|31675|United States|-5|apartment| +17629|AAAAAAAANNEEAAAA|28|Dogwood |Ct.|Suite W|Deerfield|Greeley County|NE|69840|United States|-6|single family| +17630|AAAAAAAAONEEAAAA|115|6th Lee|Ln|Suite 260|Jackson|Greene County|NC|29583|United States|-5|single family| +17631|AAAAAAAAPNEEAAAA|662|Sunset Sunset|Street|Suite O|Crossroads|Jackson County|KS|60534|United States|-6|condo| +17632|AAAAAAAAAOEEAAAA|270|Washington Church|Ave|Suite 100|Jackson|Jerome County|ID|89583|United States|-7|condo| +17633|AAAAAAAABOEEAAAA|776|Walnut |Boulevard|Suite K|Spring Valley|Dodge County|NE|66060|United States|-6|condo| +17634|AAAAAAAACOEEAAAA|111|Church |Dr.|Suite 450|Mount Pleasant|Greene County|TN|31933|United States|-5|apartment| +17635|AAAAAAAADOEEAAAA|306|Smith |Boulevard|Suite Q|Oak Hill|McIntosh County|GA|37838|United States|-5|condo| +17636|AAAAAAAAEOEEAAAA|147|East Hill|Wy|Suite 450|Greenwood|Gibson County|IN|48828|United States|-5|condo| +17637|AAAAAAAAFOEEAAAA|566|4th |Avenue|Suite 260|Five Forks|Pulaski County|GA|32293|United States|-5|single family| +17638|AAAAAAAAGOEEAAAA|528|Lincoln Ash|Boulevard|Suite U|Wildwood|Slope County|ND|56871|United States|-6|single family| +17639|AAAAAAAAHOEEAAAA|978|Central |Ave|Suite 240|New Hope|Pope County|AR|79431|United States|-6|apartment| +17640|AAAAAAAAIOEEAAAA|765|15th |Boulevard|Suite R|Newtown|Douglas County|IL|61749|United States|-6|apartment| +17641|AAAAAAAAJOEEAAAA|485|Green Birch|Cir.|Suite 80|Deerfield|Monroe County|GA|39840|United States|-5|single family| +17642|AAAAAAAAKOEEAAAA|653|Main River|ST|Suite A|Florence|Bladen County|NC|23394|United States|-5|single family| +17643|AAAAAAAALOEEAAAA|675|Forest Cedar|Way|Suite Y|Springdale|Dearborn County|IN|48883|United States|-5|single family| +17644|AAAAAAAAMOEEAAAA|808|Mill |Ct.|Suite 380|Bunker Hill|Swisher County|TX|70150|United States|-6|single family| +17645|AAAAAAAANOEEAAAA|643|Ridge Johnson|Drive|Suite V|Wildwood|Gogebic County|MI|46871|United States|-5|condo| +17646|AAAAAAAAOOEEAAAA|787|Park |Lane|Suite 260|Bethesda|Ben Hill County|GA|35980|United States|-5|apartment| +17647|AAAAAAAAPOEEAAAA|181|Hickory |Dr.|Suite 460|Fairfield|Swisher County|TX|76192|United States|-6|apartment| +17648|AAAAAAAAAPEEAAAA|709|Jefferson 12th|Way|Suite 450|Enterprise|Hardeman County|TN|31757|United States|-5|apartment| +17649|AAAAAAAABPEEAAAA|37|Lee |Pkwy|Suite P|Leon|Cattaraugus County|NY|10913|United States|-5|apartment| +17650|AAAAAAAACPEEAAAA|850|Main |Dr.|Suite 270|Denmark|Cumberland County|NC|25576|United States|-5|single family| +17651|AAAAAAAADPEEAAAA|418|Jefferson |Wy|Suite 80|Buena Vista|Greene County|VA|25752|United States|-5|single family| +17652|AAAAAAAAEPEEAAAA|524|Green 5th|Ln|Suite C|Saratoga|Peoria County|IL|62123|United States|-6|single family| +17653|AAAAAAAAFPEEAAAA|656|Ridge |Lane|Suite 340|Ashland|Shelby County|OH|44244|United States|-5|apartment| +17654|AAAAAAAAGPEEAAAA|648|Pine 5th|Lane|Suite 50|Oakdale|Brown County|KS|69584|United States|-6|apartment| +17655|AAAAAAAAHPEEAAAA|885|5th Lake|ST|Suite 180|Highland Park|Mineral County|WV|26534|United States|-5|apartment| +17656|AAAAAAAAIPEEAAAA|803|Main |Road|Suite 280|Glenwood|Fulton County|IN|43511|United States|-5|condo| +17657|AAAAAAAAJPEEAAAA|837|3rd Poplar|Ct.|Suite U|Oakwood|Sanborn County|SD|50169|United States|-7|single family| +17658|AAAAAAAAKPEEAAAA|486|Central Fifth|Ct.|Suite N|Highland|Tripp County|SD|59454|United States|-7|apartment| +17659|AAAAAAAALPEEAAAA|165|Wilson 5th|Street|Suite 120|Mount Vernon|Williams County|ND|58482|United States|-6|condo| +17660|AAAAAAAAMPEEAAAA|30|Oak Pine|Ln|Suite E|Pine Grove|Dodge County|MN|54593|United States|-6|condo| +17661|AAAAAAAANPEEAAAA|510|1st |Drive|Suite 230|Union|Clinton County|IA|58721|United States|-6|apartment| +17662|AAAAAAAAOPEEAAAA|305|North Railroad|Parkway|Suite E|Midway|Cochran County|TX|71904|United States|-6|condo| +17663|AAAAAAAAPPEEAAAA|||Pkwy||||KY|47066|United States||| +17664|AAAAAAAAAAFEAAAA|10|Park |Drive|Suite K|Five Points|Adams County|WA|96098|United States|-8|apartment| +17665|AAAAAAAABAFEAAAA|662|Railroad |Ln|Suite E|Midway|Vinton County|OH|41904|United States|-5|single family| +17666|AAAAAAAACAFEAAAA|941|View |Lane|Suite 210|Pine Grove|Choctaw County|AL|34593|United States|-6|single family| +17667|AAAAAAAADAFEAAAA|78|5th |Blvd|Suite 250|Macedonia|Jim Hogg County|TX|71087|United States|-6|apartment| +17668|AAAAAAAAEAFEAAAA|25|Laurel Park|Ave|Suite 0|Watkins|Marengo County|AL|31732|United States|-6|condo| +17669|AAAAAAAAFAFEAAAA|350|Williams Jackson|Street|Suite 330|Sunnyside|Waseca County|MN|51952|United States|-6|single family| +17670|AAAAAAAAGAFEAAAA|430|Oak Wilson|Pkwy|Suite 400|Jamestown|Garfield County|UT|86867|United States|-7|condo| +17671|AAAAAAAAHAFEAAAA|98|4th Johnson|Wy|Suite I|Spring Grove|Pottawatomie County|KS|66719|United States|-6|apartment| +17672|AAAAAAAAIAFEAAAA|560|Oak |Dr.|Suite B|Acme|Santa Cruz County|AZ|80164|United States|-7|single family| +17673|AAAAAAAAJAFEAAAA|157|Sunset Pine|Avenue|Suite 150|Lincoln|Fleming County|KY|41289|United States|-6|single family| +17674|AAAAAAAAKAFEAAAA||||Suite V|Belmont||AR|70191|United States|-6|apartment| +17675|AAAAAAAALAFEAAAA|414|Walnut |Way|Suite W|Red Hill|Amherst County|VA|24338|United States|-5|condo| +17676|AAAAAAAAMAFEAAAA|657|5th |RD|Suite 190|Clinton|Franklin County|WA|98222|United States|-8|single family| +17677|AAAAAAAANAFEAAAA|126|Smith |Boulevard|Suite E|Pleasant Valley|Power County|ID|82477|United States|-7|apartment| +17678|AAAAAAAAOAFEAAAA|715|Ninth |RD|Suite 130|Springdale|Iberville Parish|LA|78883|United States|-6|apartment| +17679|AAAAAAAAPAFEAAAA|172|1st |Ave|Suite F|Enterprise|Greeley County|NE|61757|United States|-6|condo| +17680|AAAAAAAAABFEAAAA|275|Green College|Circle|Suite 460|Elizabeth|Marion County|IL|62935|United States|-6|condo| +17681|AAAAAAAABBFEAAAA|987|Pine Spring|Street|Suite V|Bridgeport|Chattooga County|GA|35817|United States|-5|apartment| +17682|AAAAAAAACBFEAAAA|912|12th |Street|Suite K|Barnes|Cherry County|NE|63788|United States|-6|single family| +17683|AAAAAAAADBFEAAAA|356|Park Maple|RD|Suite 330|Buena Vista|Mower County|MN|55752|United States|-6|condo| +17684|AAAAAAAAEBFEAAAA|813|Franklin Green|Way|Suite 230|Valley View|Sierra County|NM|85124|United States|-7|single family| +17685|AAAAAAAAFBFEAAAA|75|Birch Highland|Street|Suite P|Mount Zion|Scott County|AR|78054|United States|-6|apartment| +17686|AAAAAAAAGBFEAAAA|551|Broadway Park|Boulevard|Suite 170|Shady Grove|Utah County|UT|82812|United States|-7|apartment| +17687|AAAAAAAAHBFEAAAA|461|Center |Wy|Suite 250|Point Pleasant|Adams County|MS|54749|United States|-6|single family| +17688|AAAAAAAAIBFEAAAA|73|Miller |Circle|Suite S|Arcola|Dixon County|NE|61654|United States|-6|condo| +17689|AAAAAAAAJBFEAAAA|249|13th |Road|Suite W|Stringtown|Randall County|TX|70162|United States|-6|single family| +17690|AAAAAAAAKBFEAAAA|433|1st |Blvd|Suite Q|Union|Braxton County|WV|28721|United States|-5|single family| +17691|AAAAAAAALBFEAAAA|563|Elm Oak|Ln|Suite 450|Midway|Culberson County|TX|71904|United States|-6|apartment| +17692|AAAAAAAAMBFEAAAA|906|||||Ida County|||United States|-6|| +17693|AAAAAAAANBFEAAAA|190|View Williams|Ln|Suite 380|Macedonia|Itasca County|MN|51087|United States|-6|single family| +17694|AAAAAAAAOBFEAAAA|164|14th Mill|Wy|Suite 470|Fairfield|Berrien County|GA|36192|United States|-5|single family| +17695|AAAAAAAAPBFEAAAA|147|Davis Elm|Court|Suite L|Concord|District of Columbia|DC|24107|United States|-5|condo| +17696|AAAAAAAAACFEAAAA|916|Locust Church|Circle|Suite 410|Pleasant Hill|Union County|TN|33604|United States|-6|apartment| +17697|AAAAAAAABCFEAAAA|510|8th View|Street|Suite 220|Mount Vernon|Benson County|ND|58482|United States|-6|single family| +17698|AAAAAAAACCFEAAAA|120|Hickory |Road|Suite 50|Glenwood|Butler County|KS|63511|United States|-6|apartment| +17699|AAAAAAAADCFEAAAA|596|3rd |Cir.|Suite H|Mount Zion|Benton County|TN|38054|United States|-5|single family| +17700|AAAAAAAAECFEAAAA|54|Seventh 4th|Court|Suite S|Springfield|Cleveland County|OK|79303|United States|-6|single family| +17701|AAAAAAAAFCFEAAAA|794|Walnut Pine|ST|Suite N|Ryan|Cherokee County|KS|60525|United States|-6|single family| +17702|AAAAAAAAGCFEAAAA|760|Park Fourth|Parkway|Suite I|Pleasant Valley|Powell County|KY|42477|United States|-5|condo| +17703|AAAAAAAAHCFEAAAA|725|Chestnut |Cir.|Suite 40|Riverside|Tunica County|MS|59231|United States|-6|apartment| +17704|AAAAAAAAICFEAAAA|93|Fourth |Way|Suite 0|Stringtown|Carson City|NV|80162|United States|-8|apartment| +17705|AAAAAAAAJCFEAAAA|878|Adams 1st|Blvd|Suite V|Enterprise|Bristol County|MA|02357|United States|-5|condo| +17706|AAAAAAAAKCFEAAAA|878|Lake First|Ln|Suite I|Georgetown|Multnomah County|OR|97057|United States|-8|single family| +17707|AAAAAAAALCFEAAAA|645|Washington |Pkwy|Suite 80|Friendship|Butler County|KS|64536|United States|-6|apartment| +17708|AAAAAAAAMCFEAAAA|479|West Spruce|Wy|Suite 70|Arthur|Upson County|GA|35965|United States|-5|single family| +17709|AAAAAAAANCFEAAAA|909|College Broadway|Avenue|Suite 220|Clinton|Rusk County|WI|58222|United States|-6|single family| +17710|AAAAAAAAOCFEAAAA|349|Adams Pine|RD|Suite 440|Thompsonville|Carson City|NV|89651|United States|-8|apartment| +17711|AAAAAAAAPCFEAAAA|392|Spruce |Lane|Suite F|Harmony|Cleburne County|AR|75804|United States|-6|condo| +17712|AAAAAAAAADFEAAAA|769|Cedar |Dr.|Suite F|Pomona|Wilkinson County|MS|54153|United States|-6|condo| +17713|AAAAAAAABDFEAAAA|563|Miller |Circle|Suite 260|Brookwood|Calcasieu Parish|LA|70965|United States|-6|single family| +17714|AAAAAAAACDFEAAAA|197|Park 11th|Avenue|Suite 200|Enterprise|Indiana County|PA|11757|United States|-5|apartment| +17715|AAAAAAAADDFEAAAA|578|9th |Blvd|Suite 240|Rutland|Jackson County|OR|98375|United States|-8|condo| +17716|AAAAAAAAEDFEAAAA|201|Laurel River|Cir.|Suite G|Mount Olive|McCreary County|KY|48059|United States|-5|condo| +17717|AAAAAAAAFDFEAAAA|520|Woodland First|Road|Suite 450|Friendship|Upton County|TX|74536|United States|-6|condo| +17718|AAAAAAAAGDFEAAAA|532|8th |RD|Suite 150|Lincoln|Marion County|OH|41289|United States|-5|condo| +17719|AAAAAAAAHDFEAAAA|897|Fifth |Way|Suite 310|Oakwood|Noble County|OH|40169|United States|-5|condo| +17720|AAAAAAAAIDFEAAAA|869|Highland |Drive|Suite B|Kingston|Schuyler County|IL|64975|United States|-6|single family| +17721|AAAAAAAAJDFEAAAA|267|13th |Road|Suite 30|Fairfield|Ashley County|AR|76192|United States|-6|single family| +17722|AAAAAAAAKDFEAAAA|870|Miller Wilson|Boulevard|Suite 410|Blair|Broadwater County|MT|65465|United States|-7|single family| +17723|AAAAAAAALDFEAAAA|114|Sunset 6th|Boulevard|Suite 280|Deerfield|Scotts Bluff County|NE|69840|United States|-7|single family| +17724|AAAAAAAAMDFEAAAA|34|Poplar |Blvd|Suite M|Union Hill|Boone County|KY|47746|United States|-6|single family| +17725|AAAAAAAANDFEAAAA|326|3rd Park|ST|Suite 370|Unionville|Charlevoix County|MI|41711|United States|-5|apartment| +17726|AAAAAAAAODFEAAAA|925|Jackson |Ave|Suite W|Pleasant Hill|Montgomery County|AR|73604|United States|-6|single family| +17727|AAAAAAAAPDFEAAAA|286|Green Jackson|Cir.|Suite A|Oak Ridge|Hughes County|SD|58371|United States|-7|condo| +17728|AAAAAAAAAEFEAAAA|813|6th Franklin|Cir.|Suite Y|Buena Vista|McIntosh County|ND|55752|United States|-6|apartment| +17729|AAAAAAAABEFEAAAA|845|10th |Boulevard|Suite 230|Jackson|Shelby County|OH|49583|United States|-5|condo| +17730|AAAAAAAACEFEAAAA|409|12th Main|Lane|Suite 120|Jackson|Hickory County|MO|69583|United States|-6|condo| +17731|AAAAAAAADEFEAAAA|241|6th Lincoln|Drive|Suite 200|Salem|Perry County|PA|18048|United States|-5|condo| +17732|AAAAAAAAEEFEAAAA|308|South Dogwood|Street|Suite 480|Macon|Burt County|NE|60369|United States|-6|condo| +17733|AAAAAAAAFEFEAAAA|811|West Ridge|Pkwy|Suite L|Five Forks|Fillmore County|NE|62293|United States|-6|single family| +17734|AAAAAAAAGEFEAAAA|539|Ridge Dogwood|Blvd|Suite U|Wilson|Cedar County|NE|66971|United States|-6|single family| +17735|AAAAAAAAHEFEAAAA|445|Fourth |Ln|Suite B|Greendale|Miller County|AR|79343|United States|-6|single family| +17736|AAAAAAAAIEFEAAAA|212|3rd |Pkwy|Suite Y|Wilson|Hill County|TX|76971|United States|-6|apartment| +17737|AAAAAAAAJEFEAAAA|608|5th |Street|Suite 80|Crossroads|Ashley County|AR|70534|United States|-6|condo| +17738|AAAAAAAAKEFEAAAA|82|5th |Wy|Suite 10|Spring Valley|Winchester city|VA|26060|United States|-5|condo| +17739|AAAAAAAALEFEAAAA|61|Maple 12th|Boulevard|Suite T|Springfield|Carroll County|IL|69303|United States|-6|condo| +17740|AAAAAAAAMEFEAAAA|631|Hillcrest |Blvd|Suite A|Spring Valley|Adair County|KY|46060|United States|-6|single family| +17741|AAAAAAAANEFEAAAA|577|1st Jefferson|Dr.|Suite 490|Florence|Douglas County|SD|53394|United States|-6|single family| +17742|AAAAAAAAOEFEAAAA|343|Sunset Jefferson|Pkwy|Suite 490|Franklin|Marshall County|OK|79101|United States|-6|apartment| +17743|AAAAAAAAPEFEAAAA|669|Main Valley|Cir.|Suite E|Shiloh|Lucas County|OH|49275|United States|-5|apartment| +17744|AAAAAAAAAFFEAAAA|781|Sycamore Park|Ct.|Suite 490|Florence|Carbon County|WY|83394|United States|-7|single family| +17745|AAAAAAAABFFEAAAA|259|Williams |ST|Suite 330|Glendale|Burt County|NE|63951|United States|-6|apartment| +17746|AAAAAAAACFFEAAAA|||||Highland Park|Montcalm County|||United States||| +17747|AAAAAAAADFFEAAAA|202|9th Tenth|Road|Suite 10|Riverview|Horry County|SC|29003|United States|-5|single family| +17748|AAAAAAAAEFFEAAAA||||Suite S||||98280||-8|| +17749|AAAAAAAAFFFEAAAA||||||King County|TX|78054||-6|single family| +17750|AAAAAAAAGFFEAAAA|518|Ridge |Circle|Suite 200|Kingston|McLean County|ND|54975|United States|-6|single family| +17751|AAAAAAAAHFFEAAAA|390|9th |Parkway|Suite P|Enterprise|Bowie County|TX|71757|United States|-6|single family| +17752|AAAAAAAAIFFEAAAA|50|6th Willow|RD|Suite I|Hopewell|Staunton city|VA|20587|United States|-5|single family| +17753|AAAAAAAAJFFEAAAA|568|Lincoln |Ct.|Suite F|Greenwood|Powder River County|MT|68828|United States|-7|single family| +17754|AAAAAAAAKFFEAAAA|57|Johnson 7th|Way|Suite 320|Shiloh|Apache County|AZ|89275|United States|-7|condo| +17755|AAAAAAAALFFEAAAA|848|Oak Locust|Avenue|Suite 270|Friendship|Robertson County|TN|34536|United States|-6|apartment| +17756|AAAAAAAAMFFEAAAA|943|Main Oak|Avenue|Suite 450|Harmony|Greer County|OK|75804|United States|-6|condo| +17757|AAAAAAAANFFEAAAA|429|View |RD|Suite 50|Jackson|Bureau County|IL|69583|United States|-6|single family| +17758|AAAAAAAAOFFEAAAA|958|Jackson Main|Boulevard|Suite I|Centerville|Schuyler County|NY|10059|United States|-5|condo| +17759|AAAAAAAAPFFEAAAA|218|1st |Parkway|Suite 480|Liberty|Gasconade County|MO|63451|United States|-6|single family| +17760|AAAAAAAAAGFEAAAA|683|Main |Boulevard|Suite V|Mountain View|Sussex County|NJ|05066|United States|-5|condo| +17761|AAAAAAAABGFEAAAA|176|4th |Boulevard|Suite 30|New Hope|Oneida County|NY|19431|United States|-5|condo| +17762|AAAAAAAACGFEAAAA|987|Sixth Jefferson|Wy|Suite 450|Florence|Jefferson Davis County|MS|53394|United States|-6|condo| +17763|AAAAAAAADGFEAAAA|439|Elm River|Lane|Suite 120|Greenfield|Martin County|KY|45038|United States|-5|single family| +17764|AAAAAAAAEGFEAAAA|211|Elm |Pkwy|Suite W|Newport|Bosque County|TX|71521|United States|-6|condo| +17765|AAAAAAAAFGFEAAAA|41|College Pine|Dr.|Suite J|Fairview|Northampton County|VA|25709|United States|-5|condo| +17766|AAAAAAAAGGFEAAAA|338|Eigth |Way|Suite 170|Stringtown|Blaine County|NE|60162|United States|-6|apartment| +17767|AAAAAAAAHGFEAAAA|492|Spruce Maple|Dr.|Suite P|Murphy|Waldo County|ME|02705|United States|-5|single family| +17768|AAAAAAAAIGFEAAAA|765|13th 2nd|Lane|Suite G|Fairfield|Dunn County|ND|56192|United States|-6|apartment| +17769|AAAAAAAAJGFEAAAA|898|1st 6th|Boulevard|Suite E|Stringtown|Box Elder County|UT|80162|United States|-7|single family| +17770|AAAAAAAAKGFEAAAA|903|1st Oak|Avenue|Suite 260|Lebanon|Madison County|MT|62898|United States|-7|condo| +17771|AAAAAAAALGFEAAAA|181|6th |Wy|Suite T|Lakewood|Franklin County|KS|68877|United States|-6|single family| +17772|AAAAAAAAMGFEAAAA|311|Elm |RD|Suite N|Oakland|Ferry County|WA|99843|United States|-8|condo| +17773|AAAAAAAANGFEAAAA|208|Cherry Main|Way|Suite 320|Riverside|Terrebonne Parish|LA|79231|United States|-6|apartment| +17774|AAAAAAAAOGFEAAAA|474|||||Columbia County||||-8|| +17775|AAAAAAAAPGFEAAAA|910|Walnut Green|Wy|Suite 130|Union|Josephine County|OR|98721|United States|-8|single family| +17776|AAAAAAAAAHFEAAAA|466|North |Pkwy|Suite D|Tipton|Evangeline Parish|LA|78773|United States|-6|condo| +17777|AAAAAAAABHFEAAAA|334|Lincoln Meadow|Ln|Suite C|Union Hill|Webster County|WV|27746|United States|-5|single family| +17778|AAAAAAAACHFEAAAA|821|14th |Way|Suite W|Shiloh|Surry County|VA|29275|United States|-5|condo| +17779|AAAAAAAADHFEAAAA|772|14th |Avenue|Suite E|Midway|Lyon County|MN|51904|United States|-6|single family| +17780|AAAAAAAAEHFEAAAA|351|Maple Main|Blvd|Suite 100|Mount Olive|McCreary County|KY|48059|United States|-5|single family| +17781|AAAAAAAAFHFEAAAA|613|Seventh Mill|Street|Suite 250|Green Acres|Red Lake County|MN|57683|United States|-6|single family| +17782|AAAAAAAAGHFEAAAA|517|8th |Ave|Suite 0|Pleasant Valley|Rush County|KS|62477|United States|-6|apartment| +17783|AAAAAAAAHHFEAAAA|145|Williams |Parkway|Suite 380|Macedonia|Screven County|GA|31087|United States|-5|condo| +17784|AAAAAAAAIHFEAAAA|823|Madison Seventh|ST|Suite 440|Green Acres|Rhea County|TN|37683|United States|-6|apartment| +17785|AAAAAAAAJHFEAAAA|157|Dogwood 6th|Blvd|Suite 390|Brookville|Iron County|MO|63524|United States|-6|condo| +17786|AAAAAAAAKHFEAAAA|483|West Meadow|Way|Suite 110|Glenwood|Daviess County|KY|43511|United States|-6|condo| +17787|AAAAAAAALHFEAAAA|59|Valley |Way|Suite J|Arthur|Harney County|OR|95965|United States|-8|apartment| +17788|AAAAAAAAMHFEAAAA|171|Walnut |Parkway|Suite K|Wyoming|Franklin County|IL|60216|United States|-6|condo| +17789|AAAAAAAANHFEAAAA|127|Walnut |Road|Suite 450|Oakland|Luna County|NM|89843|United States|-7|apartment| +17790|AAAAAAAAOHFEAAAA|697|View |Boulevard|Suite 290|White Oak|Henry County|KY|46668|United States|-6|condo| +17791|AAAAAAAAPHFEAAAA|729|Highland |Way|Suite R|Pleasant Valley|Madison County|OH|42477|United States|-5|single family| +17792|AAAAAAAAAIFEAAAA|639|Hill Valley|ST|Suite 230|Pleasant Hill|Wayne County|NY|13604|United States|-5|condo| +17793|AAAAAAAABIFEAAAA|429|Woodland |Street|Suite J|Springdale|McDowell County|NC|28883|United States|-5|apartment| +17794|AAAAAAAACIFEAAAA|17|4th 4th|ST|Suite 370|Wildwood|Kaufman County|TX|76871|United States|-6|condo| +17795|AAAAAAAADIFEAAAA|232|Church |Ct.|Suite 260|Macedonia|Cheyenne County|KS|61087|United States|-6|apartment| +17796|AAAAAAAAEIFEAAAA|422|North Willow|RD|Suite E|Wilson|Cobb County|GA|36971|United States|-5|single family| +17797|AAAAAAAAFIFEAAAA|704|Central 11th|Pkwy|Suite K|Five Forks|New Castle County|DE|12293|United States|-5|condo| +17798|AAAAAAAAGIFEAAAA|29|Willow 3rd|Lane|Suite T|Riverview|San Juan County|NM|89003|United States|-7|apartment| +17799|AAAAAAAAHIFEAAAA|370|1st |Parkway|Suite S|Riverview|Teton County|ID|89003|United States|-7|apartment| +17800|AAAAAAAAIIFEAAAA|630|Williams |Wy|Suite 370|||TX|70059|||apartment| +17801|AAAAAAAAJIFEAAAA|922|Lincoln Park|Drive|Suite 20|Oak Grove|Franklin County|IL|68370|United States|-6|condo| +17802|AAAAAAAAKIFEAAAA|294|2nd |Ave|Suite Y|Mount Zion|Alamosa County|CO|88054|United States|-7|condo| +17803|AAAAAAAALIFEAAAA|374|Johnson |Blvd|Suite 100|Greenfield|Aleutians West Census Area|AK|95038|United States|-9|single family| +17804|AAAAAAAAMIFEAAAA|420|Lake |Wy|Suite D|Oakwood|Morrow County|OR|90169|United States|-8|apartment| +17805|AAAAAAAANIFEAAAA|717|Franklin Elm|Parkway|Suite 170|Murphy|Jefferson Parish|LA|72105|United States|-6|apartment| +17806|AAAAAAAAOIFEAAAA|100|Park |Avenue|Suite 350|Bath|Tift County|GA|30573|United States|-5|single family| +17807|AAAAAAAAPIFEAAAA||Mill Walnut||||||75817|United States|-6|| +17808|AAAAAAAAAJFEAAAA|86|1st |Ln|Suite D|Antioch|Bremer County|IA|58605|United States|-6|single family| +17809|AAAAAAAABJFEAAAA|683|Oak |Wy|Suite 90|Union Hill|Rensselaer County|NY|17746|United States|-5|single family| +17810|AAAAAAAACJFEAAAA|444|Seventh |Blvd|Suite M|Red Hill|Porter County|IN|44338|United States|-5|single family| +17811|AAAAAAAADJFEAAAA|732|Sunset 5th|Ln|Suite 100|Woodville|Raleigh County|WV|24289|United States|-5|apartment| +17812|AAAAAAAAEJFEAAAA|158|North Park|Ct.|Suite 290|Fairfield|Quitman County|MS|56192|United States|-6|condo| +17813|AAAAAAAAFJFEAAAA|254|8th 12th|Court|Suite 400|Gladstone|Goliad County|TX|70894|United States|-6|single family| +17814|AAAAAAAAGJFEAAAA|475|Main |Avenue|Suite O|Greenfield|Baylor County|TX|75038|United States|-6|apartment| +17815|AAAAAAAAHJFEAAAA|5|Forest |Drive|Suite D|Enterprise|Cottle County|TX|71757|United States|-6|condo| +17816|AAAAAAAAIJFEAAAA|338|Birch |Drive|Suite 130|Rossville|Warren County|MS|52503|United States|-6|condo| +17817|AAAAAAAAJJFEAAAA|118|Williams Dogwood|Avenue|Suite 10|Maple Grove|Crosby County|TX|78252|United States|-6|apartment| +17818|AAAAAAAAKJFEAAAA|979|1st 1st|Circle|Suite C|Greenfield|Monroe County|IA|55038|United States|-6|condo| +17819|AAAAAAAALJFEAAAA|873|Poplar |Dr.|Suite R|Clinton|Crawford County|AR|78222|United States|-6|condo| +17820|AAAAAAAAMJFEAAAA|771|Sycamore |Road|Suite P|Springfield|King County|TX|79303|United States|-6|apartment| +17821|AAAAAAAANJFEAAAA|513|2nd 5th|Wy|Suite 490|Hopewell|Scotts Bluff County|NE|60587|United States|-7|condo| +17822|AAAAAAAAOJFEAAAA|144|Pine |RD|Suite 420|Plainview|Essex County|NJ|04283|United States|-5|single family| +17823|AAAAAAAAPJFEAAAA|390|Ash |Circle|Suite J|King|Henderson County|TX|70008|United States|-6|single family| +17824|AAAAAAAAAKFEAAAA|583|Ridge |Wy|Suite D|Pine Grove|Denton County|TX|74593|United States|-6|single family| +17825|AAAAAAAABKFEAAAA|851|Spruce |Parkway|Suite C|Georgetown|Carroll County|GA|37057|United States|-5|single family| +17826|AAAAAAAACKFEAAAA|854|Cedar |ST|Suite 220|Oak Grove|Merrimack County|NH|08970|United States|-5|single family| +17827|AAAAAAAADKFEAAAA|408|Ninth |Ln|Suite V|Green Acres|Hood County|TX|77683|United States|-6|apartment| +17828|AAAAAAAAEKFEAAAA|66|Main Washington|Cir.|Suite 130|Harmon|Ransom County|ND|55623|United States|-6|single family| +17829|AAAAAAAAFKFEAAAA|251|South |Cir.|Suite A|Riverdale|Laramie County|WY|89391|United States|-7|condo| +17830|AAAAAAAAGKFEAAAA|257|8th 14th|Lane|Suite 310|Five Forks|Douglas County|MN|52293|United States|-6|condo| +17831|AAAAAAAAHKFEAAAA|932|Birch |ST|Suite A|Midway|Butte County|CA|91904|United States|-8|single family| +17832|AAAAAAAAIKFEAAAA|352|North 13th|Street|Suite 50|Lakeside|Tillamook County|OR|99532|United States|-8|single family| +17833|AAAAAAAAJKFEAAAA|631|Valley Hillcrest|Circle|Suite 200|Thompsonville|Esmeralda County|NV|89651|United States|-8|condo| +17834|AAAAAAAAKKFEAAAA|646|Fourth |Blvd|Suite 480|Jamestown|Lewis County|NY|16867|United States|-5|single family| +17835|AAAAAAAALKFEAAAA|932|Hill Church|Way|Suite 220|Lakeside|Richmond city|VA|29532|United States|-5|condo| +17836|AAAAAAAAMKFEAAAA|713|9th |Way|Suite 140|Clinton|Charlevoix County|MI|48222|United States|-5|condo| +17837|AAAAAAAANKFEAAAA|64|14th Sunset|Street|Suite 490|Mount Pleasant|Hinds County|MS|51933|United States|-6|apartment| +17838|AAAAAAAAOKFEAAAA|157|Forest Spruce|Ave|Suite S|Springfield|Rhea County|TN|39303|United States|-6|single family| +17839|AAAAAAAAPKFEAAAA|865|Davis Meadow|Wy|Suite H|Greenwood|Garfield County|MT|68828|United States|-7|single family| +17840|AAAAAAAAALFEAAAA|108|Main |Ct.|Suite 380|Snug Harbor|Adams County|IN|47936|United States|-5|apartment| +17841|AAAAAAAABLFEAAAA|432|Locust |Wy|Suite 220|Hamilton|McCulloch County|TX|72808|United States|-6|condo| +17842|AAAAAAAACLFEAAAA|250|Lee |Court|Suite 360|Florence|Wyoming County|PA|13394|United States|-5|single family| +17843|AAAAAAAADLFEAAAA|854|Forest |Parkway|Suite 90|Salem|Winnebago County|IL|68048|United States|-6|apartment| +17844|AAAAAAAAELFEAAAA|539|Lakeview Main|Blvd|Suite M|Glendale|Crawford County|OH|43951|United States|-5|condo| +17845|AAAAAAAAFLFEAAAA|250|Walnut Hillcrest|Ct.|Suite M|Shady Grove|Polk County|GA|32812|United States|-5|single family| +17846|AAAAAAAAGLFEAAAA|14|15th |Dr.|Suite 280|Lebanon|Frontier County|NE|62898|United States|-6|single family| +17847|AAAAAAAAHLFEAAAA|||ST|Suite 140|Oakland|Stephens County|OK|79843|United States|-6|| +17848|AAAAAAAAILFEAAAA||Chestnut Ridge|Parkway|||||24536||-5|condo| +17849|AAAAAAAAJLFEAAAA|892|Valley Ridge|Street|Suite Q|Oak Ridge|Jefferson County|ID|88371|United States|-7|apartment| +17850|AAAAAAAAKLFEAAAA|427|5th |Ct.|Suite 20|Jerome|Granite County|MT|69920|United States|-7|condo| +17851|AAAAAAAALLFEAAAA|689|Lake 6th|Ave|Suite R|Red Hill|Williamsburg County|SC|24338|United States|-5|condo| +17852|AAAAAAAAMLFEAAAA|271|Sixth Walnut|Ave|Suite 300|Caledonia|Milam County|TX|77411|United States|-6|single family| +17853|AAAAAAAANLFEAAAA|65|||||Garfield County|MT|68054|||| +17854|AAAAAAAAOLFEAAAA|630|Franklin Walnut|Road|Suite C|Union Hill|Bent County|CO|87746|United States|-7|condo| +17855|AAAAAAAAPLFEAAAA|145|2nd Miller|RD|Suite D|Mountain View|Madison County|MT|64466|United States|-7|condo| +17856|AAAAAAAAAMFEAAAA|807|Walnut Woodland|Ln|Suite 290|New Hope|Cass County|MN|59431|United States|-6|single family| +17857|AAAAAAAABMFEAAAA|403|Green 11th|Way|Suite U|Peoria|McDowell County|NC|29818|United States|-5|condo| +17858|AAAAAAAACMFEAAAA|792|Highland Park|Ave|Suite U|Liberty|Hutchinson County|TX|73451|United States|-6|condo| +17859|AAAAAAAADMFEAAAA|743|Pine Walnut|RD|Suite 210|Greenfield|Houston County|MN|55038|United States|-6|apartment| +17860|AAAAAAAAEMFEAAAA|10|Wilson |Circle|Suite 150|Sulphur Springs|Traill County|ND|58354|United States|-6|apartment| +17861|AAAAAAAAFMFEAAAA|995|Second |Lane|Suite N|Providence|Clayton County|GA|36614|United States|-5|condo| +17862|AAAAAAAAGMFEAAAA|347|Smith North|Parkway|Suite U|Lakewood|Pitt County|NC|28877|United States|-5|apartment| +17863|AAAAAAAAHMFEAAAA|304||Pkwy||Enterprise|Hot Spring County|AR||||condo| +17864|AAAAAAAAIMFEAAAA|128|East Locust|Ave|Suite G|Five Points|Woodford County|IL|66098|United States|-6|condo| +17865|AAAAAAAAJMFEAAAA|58|Main Hill|Pkwy|Suite 330|Lakeside|Newton County|IN|49532|United States|-5|single family| +17866|AAAAAAAAKMFEAAAA|494|Johnson |Ct.|Suite E|Maple Grove|Chesterfield County|SC|28252|United States|-5|apartment| +17867|AAAAAAAALMFEAAAA|827|Green |Blvd|Suite 460|Green Acres|Marion County|MO|67683|United States|-6|single family| +17868|AAAAAAAAMMFEAAAA|136|Ridge |Ct.|Suite 360|Florence|Hampden County|MA|03994|United States|-5|single family| +17869|AAAAAAAANMFEAAAA|761|2nd First|Blvd|Suite 40|Bridgeport|Sarpy County|NE|65817|United States|-7|single family| +17870|AAAAAAAAOMFEAAAA|835|Sycamore View|Blvd|Suite U|Cordova|Lincoln County|OR|96938|United States|-8|single family| +17871|AAAAAAAAPMFEAAAA|602|Chestnut |Dr.|Suite N|Oakwood|Marin County|CA|90169|United States|-8|condo| +17872|AAAAAAAAANFEAAAA|416|5th Park|Dr.|Suite 40|Glenwood|Louisa County|IA|53511|United States|-6|single family| +17873|AAAAAAAABNFEAAAA|649|Main |Boulevard|Suite 200|Five Forks|Faulk County|SD|52293|United States|-6|single family| +17874|AAAAAAAACNFEAAAA|667|Chestnut Railroad|Lane|Suite 440||Knox County|||United States|-6|apartment| +17875|AAAAAAAADNFEAAAA|930|View |Road|Suite F|Red Hill|Idaho County|ID|84338|United States|-7|single family| +17876|AAAAAAAAENFEAAAA|260|3rd |Circle|Suite 120|Union|Hartford County|CT|09321|United States|-5|single family| +17877|AAAAAAAAFNFEAAAA|658|Mill |Drive|Suite 320|Rockwood|Charles County|MD|21545|United States|-5|single family| +17878|AAAAAAAAGNFEAAAA|360|2nd Second|ST|Suite B|Jamestown|Vigo County|IN|46867|United States|-5|apartment| +17879|AAAAAAAAHNFEAAAA|590|First Cedar|Dr.|Suite K|Buena Vista|Chattahoochee County|GA|35752|United States|-5|apartment| +17880|AAAAAAAAINFEAAAA|89|Park Jackson|Ave|Suite N|Woodlawn|Fremont County|IA|54098|United States|-6|single family| +17881|AAAAAAAAJNFEAAAA|22|Mill Broadway|Court|Suite X|Mount Vernon|Chippewa County|WI|58482|United States|-6|single family| +17882|AAAAAAAAKNFEAAAA|730|Sunset North|RD|Suite 100||Shelby County||68014|||apartment| +17883|AAAAAAAALNFEAAAA|135|Park Spring|Ln|Suite T|Marion|Spencer County|KY|40399|United States|-5|single family| +17884|AAAAAAAAMNFEAAAA|263|9th |Lane|Suite N|Oak Hill|Whiteside County|IL|67838|United States|-6|single family| +17885|AAAAAAAANNFEAAAA|706|Highland First|Lane|Suite X|Clinton|Camp County|TX|78222|United States|-6|single family| +17886|AAAAAAAAONFEAAAA|171|West |Way|Suite 50|Glendale|Bremer County|IA|53951|United States|-6|single family| +17887|AAAAAAAAPNFEAAAA|456|Washington Wilson|Dr.|Suite T|Springfield|Orange County|VA|29303|United States|-5|apartment| +17888|AAAAAAAAAOFEAAAA|459|9th Valley|ST|Suite 30|Richfield|Carter County|MT|66196|United States|-7|single family| +17889|AAAAAAAABOFEAAAA|949|East |Lane|Suite V|Greenwood|Franklin County|FL|38828|United States|-5|apartment| +17890|AAAAAAAACOFEAAAA|319|Third |Circle|Suite B|Lincoln|Calhoun County|MS|51289|United States|-6|apartment| +17891|AAAAAAAADOFEAAAA|465|4th Hickory|Road|Suite 120|Clifton|Benzie County|MI|48014|United States|-5|single family| +17892|AAAAAAAAEOFEAAAA|931|Center |Ct.|Suite P|Buena Vista|Lowndes County|GA|35752|United States|-5|apartment| +17893|AAAAAAAAFOFEAAAA|110|Birch |Wy|Suite Q|Highland Park|Preston County|WV|26534|United States|-5|single family| +17894|AAAAAAAAGOFEAAAA|488|7th View|Boulevard|Suite 250|Friendship|Palm Beach County|FL|34536|United States|-5|condo| +17895|AAAAAAAAHOFEAAAA|865|Johnson Sunset|Dr.|Suite 460|Spring Valley|San Mateo County|CA|96060|United States|-8|single family| +17896|AAAAAAAAIOFEAAAA|754|Main Forest|Court|Suite E|Woodlawn|Bay County|FL|34098|United States|-5|condo| +17897|AAAAAAAAJOFEAAAA|523|Hill |Circle|Suite 240|Spring Valley|Randolph County|MO|66060|United States|-6|condo| +17898|AAAAAAAAKOFEAAAA|175|View |Wy|Suite M|New Hope|Forest County|PA|19431|United States|-5|apartment| +17899|AAAAAAAALOFEAAAA|816|Spring 10th|Ct.|Suite 60|Patterson|Choctaw County|OK|73175|United States|-6|condo| +17900|AAAAAAAAMOFEAAAA|518|Jackson College|Lane|Suite D|Lakeside|Buena Vista city|VA|29532|United States|-5|single family| +17901|AAAAAAAANOFEAAAA|892|Forest |Circle|Suite C|Crossroads|Scurry County|TX|70534|United States|-6|condo| +17902|AAAAAAAAOOFEAAAA|699|13th |Wy|Suite Y|Friendship|Hickman County|KY|44536|United States|-6|apartment| +17903|AAAAAAAAPOFEAAAA|186|Main East|Lane|Suite 140|Murphy|Yadkin County|NC|22105|United States|-5|condo| +17904|AAAAAAAAAPFEAAAA|559|Central 5th|Boulevard|Suite 210|Forest Hills|Forsyth County|NC|29237|United States|-5|single family| +17905|AAAAAAAABPFEAAAA|574|Elm Fifth|Ct.|Suite F|Mount Olive|Marshall County|WV|28059|United States|-5|single family| +17906|AAAAAAAACPFEAAAA|221|Third |Court|Suite C|Pleasant Valley|Breckinridge County|KY|42477|United States|-6|condo| +17907|AAAAAAAADPFEAAAA|342|1st 9th|RD|Suite 350|Brunswick|Wilkinson County|GA|34642|United States|-5|apartment| +17908|AAAAAAAAEPFEAAAA|102|Meadow |Ln|Suite T|Sunset Beach|Stearns County|MN|59611|United States|-6|condo| +17909|AAAAAAAAFPFEAAAA|144|8th Johnson|Dr.|Suite 140|Riverview|McMullen County|TX|79003|United States|-6|condo| +17910|AAAAAAAAGPFEAAAA|58|Cherry Cedar|Dr.|Suite 140|Marion|Columbia County|PA|10399|United States|-5|condo| +17911|AAAAAAAAHPFEAAAA|356|Davis |Parkway|Suite M|Wilson|Lafayette County|MO|66971|United States|-6|apartment| +17912|AAAAAAAAIPFEAAAA|954|Sixth Park|Ave|Suite 240|Cedar Grove|Allen County|IN|40411|United States|-5|single family| +17913|AAAAAAAAJPFEAAAA|441||Boulevard||||||United States|-8|condo| +17914|AAAAAAAAKPFEAAAA|632|Birch Forest|Blvd|Suite J|Mountain View|Daniels County|MT|64466|United States|-7|condo| +17915|AAAAAAAALPFEAAAA|189|Lee |Lane|Suite I|Spring Valley|Oregon County|MO|66060|United States|-6|apartment| +17916|AAAAAAAAMPFEAAAA|695|Center |ST|Suite 20|Sunnyside|Delta County|TX|71952|United States|-6|apartment| +17917|AAAAAAAANPFEAAAA|371|2nd |Wy|Suite Q|Hopewell|Lafayette County|MO|60587|United States|-6|condo| +17918|AAAAAAAAOPFEAAAA|511|Williams Oak|Cir.|Suite H|Antioch|Steuben County|IN|48605|United States|-5|condo| +17919|AAAAAAAAPPFEAAAA|504|Main |Ln|Suite 310|Red Hill|Buchanan County|MO|64338|United States|-6|single family| +17920|AAAAAAAAAAGEAAAA|90|Park Valley|Circle|Suite Y|Newtown|Milam County|TX|71749|United States|-6|condo| +17921|AAAAAAAABAGEAAAA|777|Miller Sycamore|Boulevard|Suite 80|Pinhook|Carroll County|MS|59398|United States|-6|apartment| +17922|AAAAAAAACAGEAAAA|293|7th |Way|Suite 410|Valley View|Wheeler County|GA|35124|United States|-5|apartment| +17923|AAAAAAAADAGEAAAA|696|Tenth Elm|Circle|Suite 220|Kingston|Loudoun County|VA|24975|United States|-5|apartment| +17924|AAAAAAAAEAGEAAAA|763|Dogwood |Street|Suite K|Sulphur Springs|Callahan County|TX|78354|United States|-6|apartment| +17925|AAAAAAAAFAGEAAAA|705|Highland Third|Ave|Suite 380|Argyle|Lincoln County|GA|38722|United States|-5|apartment| +17926|AAAAAAAAGAGEAAAA|11|Church Madison|Lane|Suite W|Franklin|Wirt County|WV|29101|United States|-5|apartment| +17927|AAAAAAAAHAGEAAAA|651|14th |Lane|Suite 310|Lakeside|Rock County|WI|59532|United States|-6|single family| +17928|AAAAAAAAIAGEAAAA|437|South 5th|Street|Suite 290|Fairfield|Comanche County|TX|76192|United States|-6|condo| +17929|AAAAAAAAJAGEAAAA|757|Meadow |Dr.|Suite N|Gum Springs|Cass County|IL|62106|United States|-6|single family| +17930|AAAAAAAAKAGEAAAA|204|Jackson Locust|Court|Suite W|Five Forks|Grayson County|KY|42293|United States|-6|condo| +17931|AAAAAAAALAGEAAAA|272|Washington Seventh|Circle|Suite J|Mount Olive|Roberts County|TX|78059|United States|-6|single family| +17932|AAAAAAAAMAGEAAAA|221|5th Woodland|Circle|Suite 250|Hopewell|Franklin County|AR|70587|United States|-6|single family| +17933|AAAAAAAANAGEAAAA|381|Third |Ave|Suite P|Midway|Mecklenburg County|NC|21904|United States|-5|single family| +17934|AAAAAAAAOAGEAAAA|564|First |Boulevard|Suite R|Green Acres|Shoshone County|ID|87683|United States|-7|condo| +17935|AAAAAAAAPAGEAAAA|951|Smith |Avenue|Suite 60|Texas|Shelby County|KY|43342|United States|-5|single family| +17936|AAAAAAAAABGEAAAA|||Blvd||Union|||18721||-5|| +17937|AAAAAAAABBGEAAAA|608|South |Boulevard|Suite J|Springdale|Bell County|TX|78883|United States|-6|apartment| +17938|AAAAAAAACBGEAAAA|363|Central Spruce|Court|Suite 150|Arlington|Washington County|NE|66557|United States|-7|condo| +17939|AAAAAAAADBGEAAAA|674|4th 4th|Ct.|Suite 380|Oakwood|Wabash County|IL|60169|United States|-6|single family| +17940|AAAAAAAAEBGEAAAA|592|5th |ST|Suite Q|Georgetown|Cherokee County|AL|37057|United States|-6|condo| +17941|AAAAAAAAFBGEAAAA|765|College Pine|Blvd|Suite 110|Salem|Washington County|PA|18048|United States|-5|apartment| +17942|AAAAAAAAGBGEAAAA|301|13th |Cir.|Suite J|Waterloo|Lincoln County|MT|61675|United States|-7|apartment| +17943|AAAAAAAAHBGEAAAA|935|Park |Pkwy|Suite 380|Lebanon|Osceola County|FL|32898|United States|-5|single family| +17944|AAAAAAAAIBGEAAAA|20|9th |Dr.|Suite 210|Mount Olive|Calaveras County|CA|98059|United States|-8|condo| +17945|AAAAAAAAJBGEAAAA|479|Park Smith|Blvd|Suite U|Amity|Major County|OK|70766|United States|-6|apartment| +17946|AAAAAAAAKBGEAAAA|848|Lincoln Chestnut|Blvd|Suite 170|Marion|Cass County|TX|70399|United States|-6|single family| +17947|AAAAAAAALBGEAAAA|929|Johnson |RD|Suite 260|Weldon|Butler County|MO|66277|United States|-6|single family| +17948|AAAAAAAAMBGEAAAA|874|Mill |Way|Suite P|Mechanicsburg|Osceola County|IA|52219|United States|-6|condo| +17949|AAAAAAAANBGEAAAA|705|3rd West|Avenue|Suite R|Greenfield|Crosby County|TX|75038|United States|-6|condo| +17950|AAAAAAAAOBGEAAAA|493|Valley Lee|Dr.|Suite Q|Mountain View|Furnas County|NE|64466|United States|-6|single family| +17951|AAAAAAAAPBGEAAAA|175|4th |Parkway|Suite 260|Whispering Pines|Hendricks County|IN|47609|United States|-5|single family| +17952|AAAAAAAAACGEAAAA|513|Lake Valley|Road|Suite Q|Riverview|Eau Claire County|WI|59003|United States|-6|apartment| +17953|AAAAAAAABCGEAAAA|110|First |ST|Suite 180|Jamestown|Lake County|CO|86867|United States|-7|apartment| +17954|AAAAAAAACCGEAAAA|375|Park Franklin|Circle|Suite A|Glenwood|Saline County|NE|63511|United States|-7|apartment| +17955|AAAAAAAADCGEAAAA|226|||Suite P|||NC||United States|-5|| +17956|AAAAAAAAECGEAAAA|81|Mill ||Suite L|Warwick||OK|71398|United States||condo| +17957|AAAAAAAAFCGEAAAA|369|Elm |Ln|Suite F|Highland Park|Lumpkin County|GA|36534|United States|-5|single family| +17958|AAAAAAAAGCGEAAAA|808|Park Park|Drive|Suite F|Woodlawn|Hartford County|CT|04698|United States|-5|condo| +17959|AAAAAAAAHCGEAAAA|630|Davis |Ave|Suite 170|Enterprise|Miami County|IN|41757|United States|-5|apartment| +17960|AAAAAAAAICGEAAAA|417|Park Lake|Lane|Suite I|Hopewell|Cherokee County|TX|70587|United States|-6|single family| +17961|AAAAAAAAJCGEAAAA|810|Second |Wy|Suite 200|Bridgeport|Fairbanks North Star Borough|AK|95817|United States|-9|apartment| +17962|AAAAAAAAKCGEAAAA|515|Davis Green||Suite 40|Highland||MO||United States||apartment| +17963|AAAAAAAALCGEAAAA|801|Church |Circle|Suite P|Bridgeport|Franklin County|IA|55817|United States|-6|condo| +17964|AAAAAAAAMCGEAAAA|576|4th |Pkwy|Suite K|Hopewell|Cass County|IL|60587|United States|-6|single family| +17965|AAAAAAAANCGEAAAA|240|11th Main|Boulevard|Suite S|Walnut|Floyd County|KY|46245|United States|-6|single family| +17966|AAAAAAAAOCGEAAAA|751|Ridge Walnut|Avenue|Suite L|Wildwood|Klickitat County|WA|96871|United States|-8|apartment| +17967|AAAAAAAAPCGEAAAA|474|First Hickory|Drive|Suite M|Centerville|Cocke County|TN|30059|United States|-5|single family| +17968|AAAAAAAAADGEAAAA|693|7th 3rd|Ave|Suite 350|Greenwood|Allegan County|MI|48828|United States|-5|condo| +17969|AAAAAAAABDGEAAAA|686|11th |Street|Suite K|Sunnyside|Stonewall County|TX|71952|United States|-6|single family| +17970|AAAAAAAACDGEAAAA|616|3rd 7th|Street|Suite C|New Town|Tensas Parish|LA|79634|United States|-6|condo| +17971|AAAAAAAADDGEAAAA|411|Church |Road|Suite 180|Caledonia|Garfield County|UT|87411|United States|-7|condo| +17972|AAAAAAAAEDGEAAAA|425|Cherry |Road|Suite W|Buena Vista|Harrison County|MS|55752|United States|-6|apartment| +17973|AAAAAAAAFDGEAAAA|763|Adams |Boulevard|Suite U|Ashland|Carroll County|IA|54244|United States|-6|apartment| +17974|AAAAAAAAGDGEAAAA|885|Poplar 1st|Parkway|Suite 360|Pleasant Grove|Stone County|MS|54136|United States|-6|apartment| +17975|AAAAAAAAHDGEAAAA|||Circle|||||23003|United States|-5|| +17976|AAAAAAAAIDGEAAAA|892|View 12th|Circle|Suite 200|Newport|Hill County|MT|61521|United States|-7|single family| +17977|AAAAAAAAJDGEAAAA|68|8th 7th|Blvd|Suite 130|Lakeside|Rankin County|MS|59532|United States|-6|condo| +17978|AAAAAAAAKDGEAAAA|803|Poplar |RD|Suite 90|Greenville|Summers County|WV|21387|United States|-5|apartment| +17979|AAAAAAAALDGEAAAA|519|Main |Lane|Suite 420|Springhill|Slope County|ND|54602|United States|-6|condo| +17980|AAAAAAAAMDGEAAAA||Green Church|||Spring Valley||NY|16060|||single family| +17981|AAAAAAAANDGEAAAA|451|First Elevnth|Parkway|Suite A|Lebanon|Abbeville County|SC|22898|United States|-5|apartment| +17982|AAAAAAAAODGEAAAA|165|7th Hillcrest|Court|Suite C|Belmont|Craighead County|AR|70191|United States|-6|apartment| +17983|AAAAAAAAPDGEAAAA|13||Circle|Suite A|Florence||UT|83394|United States|-7|single family| +17984|AAAAAAAAAEGEAAAA|912|Wilson Adams|Court|Suite C|Stringtown|Multnomah County|OR|90162|United States|-8|single family| +17985|AAAAAAAABEGEAAAA|330|4th |Drive|Suite Q|Riverside|Turner County|SD|59231|United States|-7|condo| +17986|AAAAAAAACEGEAAAA|468|Broadway Sixth|Court|Suite 480|Lakeside|Storey County|NV|89532|United States|-8|single family| +17987|AAAAAAAADEGEAAAA|496|Pine Main|Dr.|Suite P|Shady Grove|Ulster County|NY|12812|United States|-5|apartment| +17988|AAAAAAAAEEGEAAAA|324|2nd |Circle|Suite M|Clinton|Chester County|SC|28222|United States|-5|single family| +17989|AAAAAAAAFEGEAAAA|500|11th |Cir.|Suite R|Ashland|Grundy County|MO|64244|United States|-6|condo| +17990|AAAAAAAAGEGEAAAA|88|1st |Boulevard|Suite 420|Oakwood|Herkimer County|NY|10169|United States|-5|apartment| +17991|AAAAAAAAHEGEAAAA|411|Walnut |Ave|Suite O|Wilson|Powder River County|MT|66971|United States|-7|single family| +17992|AAAAAAAAIEGEAAAA|131|7th 7th|Road|Suite 450|Bridgeport|Barron County|WI|55817|United States|-6|condo| +17993|AAAAAAAAJEGEAAAA|169|Second Jefferson|Lane|Suite 440|Riverside|Baker County|FL|39231|United States|-5|single family| +17994|AAAAAAAAKEGEAAAA|843|9th 10th|Cir.|Suite A|Pleasant Grove|Belknap County|NH|04736|United States|-5|apartment| +17995|AAAAAAAALEGEAAAA|728|Twelfth Sixth|Dr.|Suite 200|Riverside|Eagle County|CO|89231|United States|-7|apartment| +17996|AAAAAAAAMEGEAAAA|493|Wilson 2nd|Avenue|Suite S|Arlington|Gilpin County|CO|86557|United States|-7|single family| +17997|AAAAAAAANEGEAAAA|924|9th Cedar|Dr.|Suite 160|Fairfield|Lycoming County|PA|16192|United States|-5|apartment| +17998|AAAAAAAAOEGEAAAA|303|Center |Road|Suite 350|Bunker Hill|Wabasha County|MN|50150|United States|-6|condo| +17999|AAAAAAAAPEGEAAAA|699|6th 2nd|Ct.|Suite F|Waterloo|Plumas County|CA|91675|United States|-8|apartment| +18000|AAAAAAAAAFGEAAAA|472|Valley Hickory|Avenue|Suite G|Mount Zion|Allen Parish|LA|78054|United States|-6|single family| +18001|AAAAAAAABFGEAAAA|820|Johnson 3rd|Court|Suite M|Crossroads|Missoula County|MT|60534|United States|-7|apartment| +18002|AAAAAAAACFGEAAAA|147|South Miller|Road|Suite 130|Five Forks|Edmonson County|KY|42293|United States|-6|single family| +18003|AAAAAAAADFGEAAAA|162|Birch Ridge|Ave|Suite R|Oak Grove|Beaverhead County|MT|68370|United States|-7|condo| +18004|AAAAAAAAEFGEAAAA|552|Lake |Way|Suite 90|Shiloh|Fairbanks North Star Borough|AK|99275|United States|-9|apartment| +18005|AAAAAAAAFFGEAAAA|592|Ridge 5th|Road|Suite B|Spring Hill|Benton County|WA|96787|United States|-8|condo| +18006|AAAAAAAAGFGEAAAA|592|Green Spring|Cir.|Suite Q|Spring Hill|Kalamazoo County|MI|46787|United States|-5|condo| +18007|AAAAAAAAHFGEAAAA|907|East |Ct.|Suite 390|Rolling Hills|Lincoln County|KY|47272|United States|-5|condo| +18008|AAAAAAAAIFGEAAAA|515|North Jefferson|Boulevard|Suite 410|Oak Hill|Patrick County|VA|27838|United States|-5|apartment| +18009|AAAAAAAAJFGEAAAA|235|Elm |Blvd|Suite 450|Spring Hill|Minnehaha County|SD|56787|United States|-7|apartment| +18010|AAAAAAAAKFGEAAAA|674|5th |Lane|Suite 450|Riverdale|Big Horn County|WY|89391|United States|-7|single family| +18011|AAAAAAAALFGEAAAA|914|Fifth Elm|Pkwy|Suite A|Glenwood|Los Angeles County|CA|93511|United States|-8|single family| +18012|AAAAAAAAMFGEAAAA|||Ct.|Suite 160||Coshocton County||||-5|single family| +18013|AAAAAAAANFGEAAAA|177|Green |Ct.|Suite C|Shady Grove|Butler County|KS|62812|United States|-6|apartment| +18014|AAAAAAAAOFGEAAAA|474||||Lakeview|Pike County|GA|38579|||| +18015|AAAAAAAAPFGEAAAA|713|East Valley|RD|Suite 130|River Oaks|Sumter County|FL|38075|United States|-5|condo| +18016|AAAAAAAAAGGEAAAA|276|6th |Blvd|Suite 210|Clifton|Cleburne County|AL|38014|United States|-6|single family| +18017|AAAAAAAABGGEAAAA|965|Main 5th|Circle|Suite A|Greenfield|Clayton County|GA|35038|United States|-5|single family| +18018|AAAAAAAACGGEAAAA|561|9th |Ct.|Suite 280|Antioch|Cherokee County|AL|38605|United States|-6|single family| +18019|AAAAAAAADGGEAAAA|565|Lake |Street|Suite Q|Andover|Russell County|VA|21639|United States|-5|single family| +18020|AAAAAAAAEGGEAAAA|142|Oak |Street|Suite 40|Five Points|Archer County|TX|76098|United States|-6|single family| +18021|AAAAAAAAFGGEAAAA|745|Wilson |Cir.|Suite K|White Oak|Des Moines County|IA|56668|United States|-6|condo| +18022|AAAAAAAAGGGEAAAA|189|Second |Parkway|Suite T|Greenfield|Morgan County|AL|35038|United States|-6|single family| +18023|AAAAAAAAHGGEAAAA|484|Johnson |Ct.|Suite 130|Lakeview|Northumberland County|VA|28579|United States|-5|apartment| +18024|AAAAAAAAIGGEAAAA|221|Cedar |Lane|Suite K|Pleasant Grove|Meade County|KS|64136|United States|-6|condo| +18025|AAAAAAAAJGGEAAAA|233|Adams 4th|Blvd|Suite K|Calhoun|Larimer County|CO|86909|United States|-7|apartment| +18026|AAAAAAAAKGGEAAAA|469|3rd |Ave|Suite 420|Summit|Hancock County|IA|50499|United States|-6|apartment| +18027|AAAAAAAALGGEAAAA|577|2nd Walnut|Circle|Suite 130|Marion|Union County|FL|30399|United States|-5|single family| +18028|AAAAAAAAMGGEAAAA|364|Woodland |Boulevard|Suite 330|Jackson|Pine County|MN|59583|United States|-6|apartment| +18029|AAAAAAAANGGEAAAA|303|Oak |Street|Suite F|Unionville|Ashland County|OH|41711|United States|-5|apartment| +18030|AAAAAAAAOGGEAAAA|105|11th |Ln|Suite I|Indian Village|Okeechobee County|FL|31075|United States|-5|condo| +18031|AAAAAAAAPGGEAAAA|377|2nd |Ave|Suite L|Liberty|Muhlenberg County|KY|43451|United States|-5|single family| +18032|AAAAAAAAAHGEAAAA|776|Pine Chestnut|Ct.|Suite 340|Mount Olive|Scott County|MO|68059|United States|-6|single family| +18033|AAAAAAAABHGEAAAA|910|Lake |Lane|Suite 490|Scottsville|Wasco County|OR|94190|United States|-8|single family| +18034|AAAAAAAACHGEAAAA|396|View 14th|Court|Suite Y|Arcola|Madison Parish|LA|71654|United States|-6|condo| +18035|AAAAAAAADHGEAAAA|262|River Jackson|Ave|Suite I|Antioch|Hardin County|OH|48605|United States|-5|apartment| +18036|AAAAAAAAEHGEAAAA|126|River Spruce|Street|Suite 330|Whitesville|Colorado County|TX|75903|United States|-6|apartment| +18037|AAAAAAAAFHGEAAAA|978|Eigth Park|Drive|Suite J|Oakwood|Oklahoma County|OK|70169|United States|-6|condo| +18038|AAAAAAAAGHGEAAAA|615|Spruce |Cir.|Suite 10|Forest Hills|Lewis County|KY|49237|United States|-5|apartment| +18039|AAAAAAAAHHGEAAAA|466|6th Green|Lane|Suite L|Brookwood|Warren County|MO|60965|United States|-6|condo| +18040|AAAAAAAAIHGEAAAA|598|Davis West|Wy|Suite 0|Harmony|Carlisle County|KY|45804|United States|-6|single family| +18041|AAAAAAAAJHGEAAAA|699|4th |RD|Suite 40|Harmony|Pickens County|GA|35804|United States|-5|condo| +18042|AAAAAAAAKHGEAAAA|597|Maple 5th|Ct.|Suite V|Lakeside|Caswell County|NC|29532|United States|-5|single family| +18043|AAAAAAAALHGEAAAA|610|North |Avenue|Suite 90|West Liberty|Sangamon County|IL|64752|United States|-6|single family| +18044|AAAAAAAAMHGEAAAA|852|Sixth |Lane|Suite B|Newport|Rock County|MN|51521|United States|-6|single family| +18045|AAAAAAAANHGEAAAA|801|Spring |Ave|Suite 110|Greenville|Greeley County|KS|61387|United States|-6|apartment| +18046|AAAAAAAAOHGEAAAA|155|14th |Ln|Suite R|Five Forks|Hooker County|NE|62293|United States|-7|single family| +18047|AAAAAAAAPHGEAAAA|683|Johnson Adams|Drive|Suite Q|Maple Grove|Love County|OK|78252|United States|-6|condo| +18048|AAAAAAAAAIGEAAAA|741|Fourth Fifth|Pkwy|Suite 140|Pleasant Grove|Santa Rosa County|FL|34136|United States|-5|single family| +18049|AAAAAAAABIGEAAAA|279|Laurel |Court|Suite U|Sunnyside|Chenango County|NY|11952|United States|-5|condo| +18050|AAAAAAAACIGEAAAA|850|Wilson |RD|Suite T|Gary|Granville County|NC|20418|United States|-5|condo| +18051|AAAAAAAADIGEAAAA|909|College |Drive|Suite X|Maple Grove|Shawnee County|KS|68252|United States|-6|condo| +18052|AAAAAAAAEIGEAAAA|213|Green Church|Drive|Suite 210|Plainview|Jenkins County|GA|33683|United States|-5|condo| +18053|AAAAAAAAFIGEAAAA|320|Maple Chestnut|RD|Suite K|Highland|Corson County|SD|59454|United States|-6|condo| +18054|AAAAAAAAGIGEAAAA|206||Boulevard|||||57745|United States|-6|condo| +18055|AAAAAAAAHIGEAAAA|981|12th |Road|Suite 170|California|Jasper County|GA|30141|United States|-5|apartment| +18056|AAAAAAAAIIGEAAAA|22|Fifth Tenth|Ln|Suite B|Kelly|Pinellas County|FL|32738|United States|-5|apartment| +18057|AAAAAAAAJIGEAAAA|234|Oak |RD|Suite K|Buena Vista|Okanogan County|WA|95752|United States|-8|apartment| +18058|AAAAAAAAKIGEAAAA|913|Ninth |Boulevard|Suite 440|Colonial Heights|Stevens County|MN|53425|United States|-6|condo| +18059|AAAAAAAALIGEAAAA|229|Mill |Drive|Suite S|Rankin|Minidoka County|ID|82621|United States|-7|apartment| +18060|AAAAAAAAMIGEAAAA||West Pine|||||||United States||apartment| +18061|AAAAAAAANIGEAAAA|180|Smith River|Road|Suite 380|Hopewell|Scott County|TN|30587|United States|-6|condo| +18062|AAAAAAAAOIGEAAAA|726|Cherry Hill|ST|Suite E|Woodville|Worth County|GA|34289|United States|-5|apartment| +18063|AAAAAAAAPIGEAAAA|152|Jackson Main|Avenue|Suite 410|Pleasant Grove|Mora County|NM|84136|United States|-7|condo| +18064|AAAAAAAAAJGEAAAA|894|4th |Court|Suite B|Union|Boyd County|NE|68721|United States|-6|apartment| +18065|AAAAAAAABJGEAAAA|298|Railroad |Ave|Suite 130|Antioch|Mifflin County|PA|18605|United States|-5|condo| +18066|AAAAAAAACJGEAAAA|255|Meadow |Lane|Suite P|White Oak|Sedgwick County|CO|86668|United States|-7|apartment| +18067|AAAAAAAADJGEAAAA|290|Mill |Parkway|Suite P|Murray|Accomack County|VA|22150|United States|-5|condo| +18068|AAAAAAAAEJGEAAAA|719|Washington Church|Road|Suite 470|Pleasant Grove|Adams County|ND|54136|United States|-6|condo| +18069|AAAAAAAAFJGEAAAA|313|River |Ct.|Suite W|Summit|Haywood County|TN|30499|United States|-5|single family| +18070|AAAAAAAAGJGEAAAA|317|College Park|Court|Suite 170|Brownsville|Franklin County|FL|39310|United States|-5|apartment| +18071|AAAAAAAAHJGEAAAA|621|Oak |Circle|Suite D|Valley View|Fillmore County|MN|55124|United States|-6|apartment| +18072|AAAAAAAAIJGEAAAA|248|Smith Seventh|Ct.|Suite 60|Lakewood|Walker County|GA|38877|United States|-5|apartment| +18073|AAAAAAAAJJGEAAAA|985|Spring West|Ave|Suite 490|Summit|Socorro County|NM|80499|United States|-7|apartment| +18074|AAAAAAAAKJGEAAAA|446|Maple 12th|Court|Suite 170|Greenville|Boundary County|ID|81387|United States|-7|condo| +18075|AAAAAAAALJGEAAAA|802|Johnson Wilson|Court|Suite E|Midway|Jim Hogg County|TX|71904|United States|-6|apartment| +18076|AAAAAAAAMJGEAAAA|495|Oak |Parkway|Suite 360|Newtown|Harvey County|KS|61749|United States|-6|apartment| +18077|AAAAAAAANJGEAAAA|918|Elm |Lane|Suite 360|Arlington|King and Queen County|VA|26557|United States|-5|single family| +18078|AAAAAAAAOJGEAAAA|426|Sycamore 11th|Ln|Suite M|Oakland|Boone County|IA|59843|United States|-6|condo| +18079|AAAAAAAAPJGEAAAA|257|1st Tenth|Lane|Suite 60|Friendship|Stanley County|SD|54536|United States|-7|apartment| +18080|AAAAAAAAAKGEAAAA|525|Meadow Pine|Lane|Suite R|Salem|Clark County|WA|98048|United States|-8|condo| +18081|AAAAAAAABKGEAAAA|513|Maple |Way|Suite 410|Walnut|Lowndes County|AL|36245|United States|-6|apartment| +18082|AAAAAAAACKGEAAAA|266|Elm Cedar|Circle|Suite X|Five Points|Seminole County|FL|36098|United States|-5|single family| +18083|AAAAAAAADKGEAAAA|174|Lincoln |Parkway|Suite 390|Oakdale|Van Buren County|AR|79584|United States|-6|apartment| +18084|AAAAAAAAEKGEAAAA|732|7th |Ln|Suite 410|Greenville|Park County|CO|81387|United States|-7|single family| +18085|AAAAAAAAFKGEAAAA|157|Walnut |Dr.|Suite 170|Murphy|Williamson County|TX|72105|United States|-6|apartment| +18086|AAAAAAAAGKGEAAAA|629|Hill Wilson|ST|Suite 50|Mount Vernon|Genesee County|NY|18482|United States|-5|apartment| +18087|AAAAAAAAHKGEAAAA|714|3rd Pine|RD|Suite 380|Valley View|Ford County|IL|65124|United States|-6|condo| +18088|AAAAAAAAIKGEAAAA|885|Forest 7th|Court|Suite 70|Marion|Calhoun County|GA|30399|United States|-5|apartment| +18089|AAAAAAAAJKGEAAAA|774|Fourth |Court|Suite 80|Newport|Terry County|TX|71521|United States|-6|condo| +18090|AAAAAAAAKKGEAAAA|||||Unionville|Rutherford County||21711|United States||apartment| +18091|AAAAAAAALKGEAAAA|92|Valley Railroad|Avenue|Suite 290|Lakeside|Oceana County|MI|49532|United States|-5|single family| +18092|AAAAAAAAMKGEAAAA|190|3rd |Blvd|Suite L|Sunnyside|Campbell County|TN|31952|United States|-5|single family| +18093|AAAAAAAANKGEAAAA|521|Willow Park|Drive|Suite 180|Maple Grove|Colfax County|NM|88252|United States|-7|single family| +18094|AAAAAAAAOKGEAAAA|816|Hillcrest |Ct.|Suite T|Valley View|Northampton County|NC|25124|United States|-5|condo| +18095|AAAAAAAAPKGEAAAA|593|Fifth |Circle|Suite P|Glendale|Burnet County|TX|73951|United States|-6|single family| +18096|AAAAAAAAALGEAAAA|976|Ridge |ST|Suite C|Ashland|Roseau County|MN|54244|United States|-6|single family| +18097|AAAAAAAABLGEAAAA|263|Smith 3rd|Circle|Suite Q|Stringtown|Northumberland County|PA|10162|United States|-5|apartment| +18098|AAAAAAAACLGEAAAA|96|Green Woodland|Circle|Suite A|Macedonia|Autauga County|AL|31087|United States|-6|single family| +18099|AAAAAAAADLGEAAAA|638|Wilson |Drive|Suite 350|Maple Grove|Grays Harbor County|WA|98252|United States|-8|apartment| +18100|AAAAAAAAELGEAAAA|611|West |Way|Suite S|Jamestown|Creek County|OK|76867|United States|-6|single family| +18101|AAAAAAAAFLGEAAAA|789|Oak |Ave|Suite 380|Five Points|Page County|VA|26098|United States|-5|apartment| +18102|AAAAAAAAGLGEAAAA|609|Pine Pine|Way|Suite 420|Walnut Grove|Josephine County|OR|97752|United States|-8|apartment| +18103|AAAAAAAAHLGEAAAA|347|Hill |Avenue|Suite 120|Arlington|Miller County|AR|76557|United States|-6|single family| +18104|AAAAAAAAILGEAAAA|999|Johnson Jackson|Ct.|Suite 300|Lewis|Lucas County|OH|47066|United States|-5|apartment| +18105|AAAAAAAAJLGEAAAA|508|Lake Laurel|Drive|Suite 120|Harmony|Davis County|UT|85804|United States|-7|condo| +18106|AAAAAAAAKLGEAAAA|301|Lincoln |Ct.|Suite M|Maple Grove|Randolph County|IN|48252|United States|-5|condo| +18107|AAAAAAAALLGEAAAA|165|Hill Maple|Drive|Suite 60|Tipton|Stephens County|TX|78773|United States|-6|condo| +18108|AAAAAAAAMLGEAAAA|72|Woodland |ST|Suite 40|White Oak|Morehouse Parish|LA|76668|United States|-6|condo| +18109|AAAAAAAANLGEAAAA|630|Jackson |Circle|Suite 370|Stewart|Garden County|NE|68041|United States|-6|apartment| +18110|AAAAAAAAOLGEAAAA|644|14th View|Ave|Suite 210|Belmont|Union County|KY|40191|United States|-5|single family| +18111|AAAAAAAAPLGEAAAA|166|Ridge |Dr.|Suite Y|Centerville|Union County|OR|90059|United States|-8|single family| +18112|AAAAAAAAAMGEAAAA|468|South Dogwood|Dr.|Suite 490|Murray|Leake County|MS|52150|United States|-6|condo| +18113|AAAAAAAABMGEAAAA|597|Elevnth East|Ln|Suite N|Mount Zion|Montague County|TX|78054|United States|-6|single family| +18114|AAAAAAAACMGEAAAA|388|Center |Cir.|Suite S|Macedonia|Aransas County|TX|71087|United States|-6|single family| +18115|AAAAAAAADMGEAAAA|645|Railroad |Ln|Suite 290|Summit|Tillamook County|OR|90499|United States|-8|single family| +18116|AAAAAAAAEMGEAAAA|255|Lincoln 4th|Parkway|Suite S|Jamestown|Sullivan County|MO|66867|United States|-6|single family| +18117|AAAAAAAAFMGEAAAA|202|Poplar Ninth|Court|Suite 210|Union|Stafford County|KS|68721|United States|-6|apartment| +18118|AAAAAAAAGMGEAAAA|41|Washington Oak|Street|Suite P|Jamestown|Ross County|OH|46867|United States|-5|single family| +18119|AAAAAAAAHMGEAAAA|716|Wilson 5th|Way|Suite H|Five Forks|Comal County|TX|72293|United States|-6|condo| +18120|AAAAAAAAIMGEAAAA|594|Hill 2nd|ST|Suite 360|Summit|East Feliciana Parish|LA|70499|United States|-6|apartment| +18121|AAAAAAAAJMGEAAAA|64|Cherry Maple|Boulevard|Suite 420|Rolling Hills|Sunflower County|MS|57272|United States|-6|single family| +18122|AAAAAAAAKMGEAAAA|326|Spruce Adams|Pkwy|Suite A|Cedar Grove|Hamilton County|FL|30411|United States|-5|apartment| +18123|AAAAAAAALMGEAAAA|3|Lee 4th|Road|Suite 400|Fairview|Richmond County|GA|35709|United States|-5|condo| +18124|AAAAAAAAMMGEAAAA|19|Washington Lakeview|RD|Suite M|Pine Grove|Cambria County|PA|14593|United States|-5|apartment| +18125|AAAAAAAANMGEAAAA|809|Elm |Avenue|Suite W|Brownsville|Santa Cruz County|CA|99310|United States|-8|single family| +18126|AAAAAAAAOMGEAAAA|555|7th 1st|Ct.|Suite 170|Marion|Nodaway County|MO|60399|United States|-6|single family| +18127|AAAAAAAAPMGEAAAA|699|Center Fifth|Parkway|Suite 120|Highland|Skagit County|WA|99454|United States|-8|single family| +18128|AAAAAAAAANGEAAAA|264|Chestnut Main|Way|Suite 380|Woodbine|Shenandoah County|VA|24253|United States|-5|single family| +18129|AAAAAAAABNGEAAAA|904|Park Cherry|Ct.|Suite 30|Friendship|Kent County|TX|74536|United States|-6|condo| +18130|AAAAAAAACNGEAAAA|969|First |Dr.|Suite W|Woodland|Muskogee County|OK|74854|United States|-6|apartment| +18131|AAAAAAAADNGEAAAA|976|Cedar Park|Road|Suite 450|Bunker Hill|Searcy County|AR|70150|United States|-6|single family| +18132|AAAAAAAAENGEAAAA|329|Sunset |RD|Suite P|Pine Grove|Antrim County|MI|44593|United States|-5|condo| +18133|AAAAAAAAFNGEAAAA|223|Cherry Sixth|Court|Suite 30|Bethel|Monroe County|NY|15281|United States|-5|apartment| +18134|AAAAAAAAGNGEAAAA|986|Main 15th|Court|||Dougherty County|||United States||| +18135|AAAAAAAAHNGEAAAA|37|4th |Cir.|Suite 240|Mountain View|Grays Harbor County|WA|94466|United States|-8|condo| +18136|AAAAAAAAINGEAAAA|823|Green |Drive|Suite L|Lakeview|Jerome County|ID|88579|United States|-7|single family| +18137|AAAAAAAAJNGEAAAA|998|Williams Church|Ln|Suite 350|Spring Valley|Allegan County|MI|46060|United States|-5|condo| +18138|AAAAAAAAKNGEAAAA|630|River Third|Cir.|Suite 360|Lakeside|Oneida County|WI|59532|United States|-6|apartment| +18139|AAAAAAAALNGEAAAA|||Boulevard|Suite O|Lebanon|Rock County|||United States|-6|| +18140|AAAAAAAAMNGEAAAA|988|Miller |Street|Suite A|Shiloh|Meade County|KS|69275|United States|-6|apartment| +18141|AAAAAAAANNGEAAAA|535|Poplar View|RD|Suite R|New Hope|Schuyler County|NY|19431|United States|-5|condo| +18142|AAAAAAAAONGEAAAA||||Suite S|||TX|70744|United States|-6|| +18143|AAAAAAAAPNGEAAAA|407|Railroad Lake|RD|Suite J|Franklin|Creek County|OK|79101|United States|-6|single family| +18144|AAAAAAAAAOGEAAAA|916|North |Street|Suite O|Springdale|Rock County|MN|58883|United States|-6|condo| +18145|AAAAAAAABOGEAAAA|864|6th View|Court|Suite O|Cedar Grove|Oswego County|NY|10411|United States|-5|single family| +18146|AAAAAAAACOGEAAAA|269|Williams |Ave|Suite N|Glendale|Kinney County|TX|73951|United States|-6|apartment| +18147|AAAAAAAADOGEAAAA|236|Wilson |Road|Suite 100|Five Points|Schley County|GA|36098|United States|-5|apartment| +18148|AAAAAAAAEOGEAAAA|132|First |Pkwy|Suite 430|Oakdale|Bowman County|ND|59584|United States|-6|apartment| +18149|AAAAAAAAFOGEAAAA|83|First |Ave|Suite X|Bridgeport|Warren County|KY|45817|United States|-5|condo| +18150|AAAAAAAAGOGEAAAA|544|Mill Mill|Ct.|Suite A|Newtown|Stevens County|WA|91749|United States|-8|apartment| +18151|AAAAAAAAHOGEAAAA|705|Woodland |RD|Suite 50|Hamilton|Chesterfield County|SC|22808|United States|-5|apartment| +18152|AAAAAAAAIOGEAAAA|913|Jackson |Boulevard|Suite 470|Greenfield|Kleberg County|TX|75038|United States|-6|single family| +18153|AAAAAAAAJOGEAAAA|795|Center Seventh|Ct.|Suite Y|New Hope|Lee County|MS|59431|United States|-6|apartment| +18154|AAAAAAAAKOGEAAAA|160|5th 2nd|Circle|Suite 110|Centerville|Island County|WA|90059|United States|-8|condo| +18155|AAAAAAAALOGEAAAA|105|2nd |Pkwy|Suite K|Riverdale|McDonald County|MO|69391|United States|-6|apartment| +18156|AAAAAAAAMOGEAAAA||Third |||Florence|Bedford County||33394||-5|single family| +18157|AAAAAAAANOGEAAAA|892|Adams |Pkwy|Suite R|Oakwood|Lincoln County|MO|60169|United States|-6|apartment| +18158|AAAAAAAAOOGEAAAA|326|Davis Johnson|Ln|Suite 130|Ashland|Dunn County|WI|54244|United States|-6|apartment| +18159|AAAAAAAAPOGEAAAA|549|East |Avenue|Suite 90|Riverdale|Douglas County|KS|69391|United States|-6|single family| +18160|AAAAAAAAAPGEAAAA|504|Oak 15th|Cir.|Suite 210|Five Forks|Chilton County|AL|32293|United States|-6|condo| +18161|AAAAAAAABPGEAAAA|966|8th |Wy|Suite 30|Wilson|Bath County|VA|26971|United States|-5|condo| +18162|AAAAAAAACPGEAAAA|932|Ridge Hillcrest|Road|Suite X|Oak Grove|Dickinson County|IA|58370|United States|-6|condo| +18163|AAAAAAAADPGEAAAA|198|Park |ST|Suite Y|Antioch|Graves County|KY|48605|United States|-6|single family| +18164|AAAAAAAAEPGEAAAA|159|Maple |Ct.|Suite 140|Clifton|Ben Hill County|GA|38014|United States|-5|condo| +18165|AAAAAAAAFPGEAAAA|99|Oak |Way|Suite 300|New Hope|Osceola County|FL|39431|United States|-5|apartment| +18166|AAAAAAAAGPGEAAAA|786|Green Lakeview|Ct.|Suite W|Glendale|Morris County|NJ|04551|United States|-5|apartment| +18167|AAAAAAAAHPGEAAAA|619|5th Birch|Boulevard|Suite 0|Cedar Creek|Atascosa County|TX|70311|United States|-6|condo| +18168|AAAAAAAAIPGEAAAA|129|Woodland 2nd|Pkwy|Suite E|Lincoln|Polk County|IA|51289|United States|-6|condo| +18169|AAAAAAAAJPGEAAAA|741|Spring |Boulevard|Suite J|Ashland|Esmeralda County|NV|84244|United States|-8|condo| +18170|AAAAAAAAKPGEAAAA|25|Lakeview Walnut|Avenue|Suite E|Wilson|Penobscot County|ME|07571|United States|-5|single family| +18171|AAAAAAAALPGEAAAA|23|Pine Third|Drive|Suite H|Union|Waushara County|WI|58721|United States|-6|apartment| +18172|AAAAAAAAMPGEAAAA|420|Cedar 7th|Parkway|Suite B|Oakwood|Hickory County|MO|60169|United States|-6|apartment| +18173|AAAAAAAANPGEAAAA|657|Park |Dr.|Suite 240|Fairfield|Hubbard County|MN|56192|United States|-6|condo| +18174|AAAAAAAAOPGEAAAA|405|Oak |Dr.|Suite 380|Concord|Pemiscot County|MO|64107|United States|-6|single family| +18175|AAAAAAAAPPGEAAAA|714|First Wilson|Ln|Suite 490|Shiloh|Stewart County|TN|39275|United States|-6|condo| +18176|AAAAAAAAAAHEAAAA|873|Elm |Court|Suite 380|Oakland|Stark County|IL|69843|United States|-6|single family| +18177|AAAAAAAABAHEAAAA|683|Second |RD|Suite 310|Highland Park|Albemarle County|VA|26534|United States|-5|single family| +18178|AAAAAAAACAHEAAAA|326|Wilson |Pkwy|Suite X|Bear Creek|Lincoln County|MN|53075|United States|-6|single family| +18179|AAAAAAAADAHEAAAA|288|Hill Jackson|Pkwy|Suite O|Newport|Reeves County|TX|71521|United States|-6|condo| +18180|AAAAAAAAEAHEAAAA|793|Park |Blvd|Suite P|Deerfield|Lincoln County|WY|89840|United States|-7|apartment| +18181|AAAAAAAAFAHEAAAA|12|Johnson West|Drive|Suite 0|Greenwood|Rapides Parish|LA|78828|United States|-6|single family| +18182|AAAAAAAAGAHEAAAA|238|Madison 7th|Parkway|Suite D|Harmony|Pecos County|TX|75804|United States|-6|single family| +18183|AAAAAAAAHAHEAAAA|919|7th Main|Dr.|Suite 40|Riverside|Douglas County|OR|99231|United States|-8|single family| +18184|AAAAAAAAIAHEAAAA|631|5th Smith|Boulevard|Suite 150|Union|Parker County|TX|78721|United States|-6|condo| +18185|AAAAAAAAJAHEAAAA|575|7th Sunset|Lane|Suite 300|Spring Valley|Pulaski County|IN|46060|United States|-5|apartment| +18186|AAAAAAAAKAHEAAAA|386|Mill Seventh|Avenue|Suite U|Highland|Carter County|KY|49454|United States|-6|single family| +18187|AAAAAAAALAHEAAAA|330|Forest |Wy|Suite 340|Providence|Douglas County|OR|96614|United States|-8|condo| +18188|AAAAAAAAMAHEAAAA|499|Main View|Drive|Suite R|Deerfield|Maverick County|TX|79840|United States|-6|single family| +18189|AAAAAAAANAHEAAAA|886|Laurel Cedar|Circle|Suite F|Oakland|Dawson County|GA|39843|United States|-5|single family| +18190|AAAAAAAAOAHEAAAA|566|Maple River|Road|Suite V|Guthrie|Floyd County|TX|71423|United States|-6|condo| +18191|AAAAAAAAPAHEAAAA|30|Sunset |Lane|Suite M|Point Pleasant|Beltrami County|MN|54749|United States|-6|single family| +18192|AAAAAAAAABHEAAAA|364|Pine |Drive|Suite E|Jackson|Door County|WI|59583|United States|-6|single family| +18193|AAAAAAAABBHEAAAA|755|Park |Way|Suite L|Willow|Ada County|ID|86798|United States|-7|apartment| +18194|AAAAAAAACBHEAAAA|408|North |Ave|Suite R|Clifton|Bartholomew County||48014||-5|single family| +18195|AAAAAAAADBHEAAAA|100|Meadow |Circle|Suite R|Red Hill|Peach County|GA|34338|United States|-5|apartment| +18196|AAAAAAAAEBHEAAAA|321|Oak |Road|Suite H|Lincoln|Mercer County|WV|21289|United States|-5|apartment| +18197|AAAAAAAAFBHEAAAA||||||Gonzales County|||||condo| +18198|AAAAAAAAGBHEAAAA|414|Sunset 2nd|Way|Suite V|Lincoln|Wilson County|KS|61289|United States|-6|condo| +18199|AAAAAAAAHBHEAAAA|976|Davis Hickory|Ct.|Suite C|Marion|Butler County|NE|60399|United States|-6|apartment| +18200|AAAAAAAAIBHEAAAA|138|4th Smith|Lane|Suite M|Georgetown|Bonner County|ID|87057|United States|-7|single family| +18201|AAAAAAAAJBHEAAAA|273|13th Davis|Ct.|Suite 30|Westminster|Faulkner County|||United States|-6|condo| +18202|AAAAAAAAKBHEAAAA|839|Broadway Seventh|RD|Suite 250|Greenfield|Douglas County|SD|55038|United States|-6|single family| +18203|AAAAAAAALBHEAAAA|||Pkwy|Suite W|Riverdale|||69391||-7|apartment| +18204|AAAAAAAAMBHEAAAA|160|Highland |Wy|Suite O|Ashland|Pawnee County|NE|64244|United States|-7|apartment| +18205|AAAAAAAANBHEAAAA|185|3rd North|Lane|Suite Y|Bethel|Crawford County|AR|75281|United States|-6|single family| +18206|AAAAAAAAOBHEAAAA|728|Chestnut Cedar|RD|Suite 270|Shady Grove|Davison County|SD|52812|United States|-6|apartment| +18207|AAAAAAAAPBHEAAAA|781|Sycamore Meadow|Ave|Suite 220|Pleasant Hill|Sunflower County|MS|53604|United States|-6|condo| +18208|AAAAAAAAACHEAAAA|561|Washington Broadway|Boulevard|Suite 140|Jackson|Bourbon County|KS|69583|United States|-6|apartment| +18209|AAAAAAAABCHEAAAA|429|Forest |Ct.|Suite W|Harmony|Washington County|NE|65804|United States|-7|apartment| +18210|AAAAAAAACCHEAAAA|417|Pine |Road|Suite 270|Fairview|Emery County|UT|85709|United States|-7|condo| +18211|AAAAAAAADCHEAAAA|549|7th Hickory|Avenue|Suite 480|Jackson|Oxford County|ME|09583|United States|-5|apartment| +18212|AAAAAAAAECHEAAAA|265|Pine View|Blvd|Suite 480|Oak Grove|Monona County|IA|58370|United States|-6|condo| +18213|AAAAAAAAFCHEAAAA|229|College North|Pkwy|Suite N|Newport|Daviess County|KY|41521|United States|-6|single family| +18214|AAAAAAAAGCHEAAAA|136|First First|Wy|Suite H|Ashley|Logan County|KY|44324|United States|-5|condo| +18215|AAAAAAAAHCHEAAAA|747|4th |Blvd|Suite M|Arlington|Juab County|UT|86557|United States|-7|single family| +18216|AAAAAAAAICHEAAAA|721|North |RD|Suite 420|Pleasant Hill|Fentress County|TN|33604|United States|-5|apartment| +18217|AAAAAAAAJCHEAAAA|52|11th Forest|Court|Suite G|Price|Orange County|CA|93824|United States|-8|single family| +18218|AAAAAAAAKCHEAAAA|452|Hill 7th|ST|Suite 300|Lebanon|Lee County|FL|32898|United States|-5|condo| +18219|AAAAAAAALCHEAAAA|856|Third |Ave|Suite Q|Greenfield|Berkshire County|MA|05638|United States|-5|single family| +18220|AAAAAAAAMCHEAAAA|190|Highland Maple|Blvd|Suite X|Hartland|Pulaski County|KY|46594|United States|-5|single family| +18221|AAAAAAAANCHEAAAA|302|4th 2nd|Avenue|Suite 320|Macedonia|Caldwell County|KY|41087|United States|-6|single family| +18222|AAAAAAAAOCHEAAAA|860|Center Lakeview|Court|Suite 60|Cedar Grove|McLean County|KY|40411|United States|-5|condo| +18223|AAAAAAAAPCHEAAAA|687|West |Way|Suite Q|Concord|Smith County|TN|34107|United States|-6|apartment| +18224|AAAAAAAAADHEAAAA|666|Green Second|Ct.|Suite 0|Mountain View|Montgomery County|KS|64466|United States|-6|single family| +18225|AAAAAAAABDHEAAAA|964|Cedar |Lane|Suite 390|Frenchtown|Haskell County|TX|72629|United States|-6|single family| +18226|AAAAAAAACDHEAAAA|182|Johnson Third|Court|Suite 50|Valley View|Lucas County|IA|55124|United States|-6|single family| +18227|AAAAAAAADDHEAAAA|685|Oak |RD|Suite 110|Bridgeport|North Slope Borough|AK|95817|United States|-9|single family| +18228|AAAAAAAAEDHEAAAA|737|Ash Second|Parkway|Suite 0|Pleasant Hill|Pendleton County|WV|23604|United States|-5|condo| +18229|AAAAAAAAFDHEAAAA|324|3rd |Circle|Suite L|Union Hill|Ward County|TX|77746|United States|-6|apartment| +18230|AAAAAAAAGDHEAAAA|845|12th Spring|Ct.|Suite 230|Enterprise|East Feliciana Parish|LA|71757|United States|-6|condo| +18231|AAAAAAAAHDHEAAAA|75|Birch Walnut|Ave|Suite I|Newtown|Buffalo County|NE|61749|United States|-6|apartment| +18232|AAAAAAAAIDHEAAAA|441|1st |Drive|Suite 380|Forest Hills|Hopkins County|KY|49237|United States|-6|apartment| +18233|AAAAAAAAJDHEAAAA|791|Poplar |Cir.|Suite X|Friendship|Allegany County|MD|24536|United States|-5|condo| +18234|AAAAAAAAKDHEAAAA|527|Johnson Eigth|Boulevard|Suite S|Clearview|Wayne County|MI|45495|United States|-5|single family| +18235|AAAAAAAALDHEAAAA|473|2nd |Parkway|Suite I|Brookwood|Lincoln County|WV|20965|United States|-5|condo| +18236|AAAAAAAAMDHEAAAA|163|Main View|Ct.|Suite 60|Salem|Otsego County|NY|18048|United States|-5|apartment| +18237|AAAAAAAANDHEAAAA|126|Laurel 5th|Avenue|Suite 170|Fairbanks|Esmeralda County|NV|86653|United States|-8|single family| +18238|AAAAAAAAODHEAAAA|946|Broadway |Ln|Suite 330|Jackson|Pittsburg County|OK|79583|United States|-6|condo| +18239|AAAAAAAAPDHEAAAA|740|Main |Circle|Suite Y|Shiloh|Nolan County|TX|79275|United States|-6|condo| +18240|AAAAAAAAAEHEAAAA|248|Locust |Street|Suite 60|Howell|Wabasha County|MN|54854|United States|-6|apartment| +18241|AAAAAAAABEHEAAAA|483|Highland |Pkwy|Suite U|Georgetown|Yuma County|AZ|87057|United States|-7|apartment| +18242|AAAAAAAACEHEAAAA|287|South View|Dr.|Suite S|Oakdale|Eagle County|CO|89584|United States|-7|single family| +18243|AAAAAAAADEHEAAAA|831|15th |Boulevard|Suite 410|Lincoln|LaMoure County|ND|51289|United States|-6|single family| +18244|AAAAAAAAEEHEAAAA|243|Park Ash|Lane|Suite 40|Highland Park|Windham County|VT|07134|United States|-5|single family| +18245|AAAAAAAAFEHEAAAA|83|Chestnut |Ct.|Suite C|Spring Hill|Fayette County|TN|36787|United States|-5|condo| +18246|AAAAAAAAGEHEAAAA|467|Main |Boulevard|Suite I|Mount Vernon|Johnson County|IL|68482|United States|-6|condo| +18247|AAAAAAAAHEHEAAAA|418|12th |Road|Suite T|Dewey|Flathead County|MT|61160|United States|-7|condo| +18248|AAAAAAAAIEHEAAAA|863|Ash Lake|Boulevard|Suite 380|Forest Hills|Dickinson County|KS|69237|United States|-6|apartment| +18249|AAAAAAAAJEHEAAAA|196|Spruce |Court|Suite 480|Woodlawn|Upshur County|WV|24098|United States|-5|single family| +18250|AAAAAAAAKEHEAAAA|898|College |Ave|Suite 10|Clifton|Brookings County|SD|58014|United States|-6|apartment| +18251|AAAAAAAALEHEAAAA|594|Jackson 14th|Avenue|Suite Y|Kingston|Nassau County|NY|14975|United States|-5|single family| +18252|AAAAAAAAMEHEAAAA|869|9th |Lane|Suite 160|Greenville|Craven County|NC|21387|United States|-5|apartment| +18253|AAAAAAAANEHEAAAA|137|Third |Lane|Suite 330|Shiloh|Caldwell County|TX|79275|United States|-6|single family| +18254|AAAAAAAAOEHEAAAA|131|2nd |Cir.|Suite 420|Concord|Marinette County|WI|54107|United States|-6|single family| +18255|AAAAAAAAPEHEAAAA|183|1st 9th|Drive|Suite 360|Bridgeport|Crisp County|GA|35817|United States|-5|condo| +18256|AAAAAAAAAFHEAAAA|988|Highland Locust|Ct.|Suite Y|Oak Ridge|Pine County|MN|58371|United States|-6|single family| +18257|AAAAAAAABFHEAAAA|891|Elm |Wy|Suite 60|Mount Zion|Fayette County|IA|58054|United States|-6|condo| +18258|AAAAAAAACFHEAAAA|350|Cedar Oak|Cir.|Suite P|Sulphur Springs|Fulton County|IN|48354|United States|-5|apartment| +18259|AAAAAAAADFHEAAAA|908|11th |Ct.|Suite 370|Fisher|Midland County|MI|42819|United States|-5|apartment| +18260|AAAAAAAAEFHEAAAA|115|Elm Walnut|Pkwy|Suite C|Pine Valley|San Juan County|NM|88209|United States|-7|single family| +18261|AAAAAAAAFFHEAAAA|35|First Johnson|Street|Suite 310|Clinton|Flagler County|FL|38222|United States|-5|apartment| +18262|AAAAAAAAGFHEAAAA|701|Jackson |Cir.|Suite 100|Five Forks|Addison County|VT|02893|United States|-5|single family| +18263|AAAAAAAAHFHEAAAA|234|1st 3rd|Ln|Suite 30|Oak Hill|Comal County|TX|77838|United States|-6|single family| +18264|AAAAAAAAIFHEAAAA|386|Spring |Dr.|Suite D|Red Hill|Iosco County|MI|44338|United States|-5|single family| +18265|AAAAAAAAJFHEAAAA|903|Park |Dr.|Suite 390|Highland|DeKalb County|IL|69454|United States|-6|condo| +18266|AAAAAAAAKFHEAAAA|966|Railroad |Dr.|Suite Q|Centerville|Garrard County|KY|40059|United States|-6|single family| +18267|AAAAAAAALFHEAAAA|372|7th Lincoln|Road|Suite B|Newtown|El Paso County|CO|81749|United States|-7|condo| +18268|AAAAAAAAMFHEAAAA|846|Locust |RD|Suite 380|Oak Ridge|Whitman County|WA|98371|United States|-8|apartment| +18269|AAAAAAAANFHEAAAA|717|Oak 12th|Wy|Suite 490|Plainview|East Baton Rouge Parish|LA|73683|United States|-6|apartment| +18270|AAAAAAAAOFHEAAAA|715|Fifth Railroad|Avenue|Suite 120|Plainview|Lincoln County|MT|63683|United States|-7|condo| +18271|AAAAAAAAPFHEAAAA|790|Center Cherry|ST|Suite 120|Friendship|Alpena County|MI|44536|United States|-5|single family| +18272|AAAAAAAAAGHEAAAA|707|Oak |Lane|Suite U|Franklin|Champaign County|OH|49101|United States|-5|condo| +18273|AAAAAAAABGHEAAAA|437|Railroad |Blvd|Suite J|Woodbury|Daviess County|KY|44489|United States|-6|condo| +18274|AAAAAAAACGHEAAAA|625|Lee |ST|Suite 200|Brownsville|Muscogee County|GA|39310|United States|-5|condo| +18275|AAAAAAAADGHEAAAA|291|Woodland |Boulevard|Suite B|Friendship|Warren County|PA|14536|United States|-5|single family| +18276|AAAAAAAAEGHEAAAA|767|Laurel |Circle|Suite V|Wildwood|Bristol County|MA|07471|United States|-5|apartment| +18277|AAAAAAAAFGHEAAAA|129|Spruce |Circle|Suite O|Glendale|Franklin County|MA|04551|United States|-5|condo| +18278|AAAAAAAAGGHEAAAA|868|Pine Main|ST|Suite T|Barnes|Rhea County|TN|33788|United States|-6|apartment| +18279|AAAAAAAAHGHEAAAA|336|Church |Ave|Suite 320|Cedar Grove|Kit Carson County|CO|80411|United States|-7|condo| +18280|AAAAAAAAIGHEAAAA|749|Church |Lane|Suite X|Union|Jefferson County|NE|68721|United States|-7|apartment| +18281|AAAAAAAAJGHEAAAA|617|6th |Pkwy|Suite 70|Mountain View|Warren County|GA|34466|United States|-5|condo| +18282|AAAAAAAAKGHEAAAA|351|Cedar |Circle|Suite O|Oakwood|Pulaski County|GA|30169|United States|-5|single family| +18283|AAAAAAAALGHEAAAA|594|Church Fourteenth|Avenue|Suite T|Kingston|Whitley County|IN|44975|United States|-5|single family| +18284|AAAAAAAAMGHEAAAA|808|Davis Spruce|Ct.|Suite W|Greenfield|Cass County|NE|65038|United States|-6|single family| +18285|AAAAAAAANGHEAAAA|809|Tenth Walnut|ST|Suite 110|Salem|Callaway County|MO|68048|United States|-6|condo| +18286|AAAAAAAAOGHEAAAA|492|Broadway |Lane|Suite 230|Oak Hill|Accomack County|VA|27838|United States|-5|single family| +18287|AAAAAAAAPGHEAAAA|155|Jackson |Parkway|Suite R|Hillcrest|Gibson County|IN|43003|United States|-5|apartment| +18288|AAAAAAAAAHHEAAAA|563|Main 12th|RD|Suite 310|Hamilton|Denver County|CO|82808|United States|-7|apartment| +18289|AAAAAAAABHHEAAAA|512|4th |Ct.|Suite 200|Greenwood|Jackson County|OK|78828|United States|-6|single family| +18290|AAAAAAAACHHEAAAA|445|||||Hamilton County||68014|||| +18291|AAAAAAAADHHEAAAA||||Suite 150|||KY|||-5|| +18292|AAAAAAAAEHHEAAAA|238|Oak Second|Street|Suite 450|Liberty|Costilla County|CO|83451|United States|-7|single family| +18293|AAAAAAAAFHHEAAAA|953|Cedar |Blvd|Suite 430|Crossroads|Warrick County|IN|40534|United States|-5|apartment| +18294|AAAAAAAAGHHEAAAA|882|11th Maple|Wy|Suite 10|Antioch|Manitowoc County|WI|58605|United States|-6|single family| +18295|AAAAAAAAHHHEAAAA|832|Forest |Road|Suite V|Salem|Hart County|GA|38048|United States|-5|single family| +18296|AAAAAAAAIHHEAAAA|97|Sycamore |Ln|Suite 20|Fairview|Gogebic County|MI|45709|United States|-5|single family| +18297|AAAAAAAAJHHEAAAA|589|Hillcrest |Court|Suite 180|Wilson|Butte County|SD|56971|United States|-6|apartment| +18298|AAAAAAAAKHHEAAAA|248|2nd Park|Boulevard|Suite 200|Hopewell|Rutherford County|TN|30587|United States|-6|condo| +18299|AAAAAAAALHHEAAAA|173|2nd Spring|Circle|Suite 360|Jamestown|Lake County|CO|86867|United States|-7|single family| +18300|AAAAAAAAMHHEAAAA|925|2nd Mill|ST|Suite 350|Greenwood|Perry County|MS|58828|United States|-6|single family| +18301|AAAAAAAANHHEAAAA|643|Chestnut River|ST|Suite 240|Springdale|Harrison County|OH|48883|United States|-5|single family| +18302|AAAAAAAAOHHEAAAA|597|Elm |Circle|Suite 140|Maple Grove|Tillman County|OK|78252|United States|-6|condo| +18303|AAAAAAAAPHHEAAAA|414|East |Avenue|Suite 20|Morgantown|Garland County|AR|79193|United States|-6|condo| +18304|AAAAAAAAAIHEAAAA|986|Railroad Lincoln|Pkwy|Suite 230|Woodville|Putnam County|FL|34289|United States|-5|single family| +18305|AAAAAAAABIHEAAAA|329|Wilson South|Lane||||||||| +18306|AAAAAAAACIHEAAAA|828|View |Ct.|Suite 200|Lakeview|Dillingham Census Area|AK|98579|United States|-9|single family| +18307|AAAAAAAADIHEAAAA|110|Walnut |Blvd|Suite 160|Pleasant Hill|Suffolk County|MA|04204|United States|-5|condo| +18308|AAAAAAAAEIHEAAAA|958|Washington Hill|Pkwy|Suite E|Waterloo|Louisa County|VA|21675|United States|-5|condo| +18309|AAAAAAAAFIHEAAAA|702|||||Andrews County||||-6|| +18310|AAAAAAAAGIHEAAAA|985|Church Church|RD|Suite 160|Five Forks|Cherokee County|GA|32293|United States|-5|apartment| +18311|AAAAAAAAHIHEAAAA|479|Cherry East|Ct.|Suite O|Lewisburg|Limestone County|TX|77538|United States|-6|single family| +18312|AAAAAAAAIIHEAAAA|524|Madison |Pkwy|Suite H|Mountain View|Monroe County|KY|44466|United States|-5|single family| +18313|AAAAAAAAJIHEAAAA|565|Washington 4th|Cir.|Suite 400|Mount Zion|Pennington County|SD|58054|United States|-7|condo| +18314|AAAAAAAAKIHEAAAA|643|Green 9th|Drive|Suite C|Acme|Atlantic County|NJ|00764|United States|-5|apartment| +18315|AAAAAAAALIHEAAAA|649|Ridge 1st|Wy|Suite C|Clinton|Rowan County|NC|28222|United States|-5|condo| +18316|AAAAAAAAMIHEAAAA|562|Elm |Way|Suite 370|Bunker Hill|Craighead County|AR|70150|United States|-6|apartment| +18317|AAAAAAAANIHEAAAA|761|6th |Court|Suite Y|Woodville|Yuma County|CO|84289|United States|-7|single family| +18318|AAAAAAAAOIHEAAAA|923|Franklin |Wy|Suite K|Red Hill|Greenup County|KY|44338|United States|-6|condo| +18319|AAAAAAAAPIHEAAAA|968|Valley |Cir.|Suite 180|Forest Hills|Ontonagon County|MI|49237|United States|-5|condo| +18320|AAAAAAAAAJHEAAAA|34|14th Lakeview|Avenue|Suite 110|Maple Grove|Lucas County|OH|48252|United States|-5|apartment| +18321|AAAAAAAABJHEAAAA|349||Boulevard|Suite T|||NC|23951||-5|| +18322|AAAAAAAACJHEAAAA|828|Third |Road|Suite G|Fairview|Pope County|MN|55709|United States|-6|condo| +18323|AAAAAAAADJHEAAAA|281|14th Jackson|Ln|Suite 370|Red Hill|Malheur County|OR|94338|United States|-8|condo| +18324|AAAAAAAAEJHEAAAA|236|Woodland Lake|Parkway|Suite S|Spring Hill|Jackson Parish|LA|76787|United States|-6|single family| +18325|AAAAAAAAFJHEAAAA|459|Valley Smith|Cir.|Suite P|Springfield|Foard County|TX|79303|United States|-6|single family| +18326|AAAAAAAAGJHEAAAA|53|Church |Street|Suite J|Georgetown|Wayne County|MI|47057|United States|-5|apartment| +18327|AAAAAAAAHJHEAAAA|628|Spring Pine|Avenue|Suite 410|Lebanon|Marshall County|AL|32898|United States|-6|condo| +18328|AAAAAAAAIJHEAAAA|641|Broadway Maple|Drive|Suite L|Clinton|Hanson County|SD|58222|United States|-7|apartment| +18329|AAAAAAAAJJHEAAAA|342|Cedar |Ln|Suite V|Unionville|Cleburne County|AR|71711|United States|-6|single family| +18330|AAAAAAAAKJHEAAAA|139|4th |Street|Suite F|Kingston|Shelby County|KY|44975|United States|-5|condo| +18331|AAAAAAAALJHEAAAA|967|Elm 5th|Wy|Suite 430|Highland|Mellette County|SD|59454|United States|-7|single family| +18332|AAAAAAAAMJHEAAAA|681|14th Spruce|Avenue|Suite F|Wilson|Eau Claire County|WI|56971|United States|-6|apartment| +18333|AAAAAAAANJHEAAAA|417|Pine |Blvd|Suite F|Enterprise|Carroll County|IL|61757|United States|-6|apartment| +18334|AAAAAAAAOJHEAAAA|947|Main |Court|Suite M|Hillcrest|Huron County|OH|43003|United States|-5|apartment| +18335|AAAAAAAAPJHEAAAA|59|Fifth Hillcrest|Pkwy|Suite Y|Salem|Kingman County|KS|68048|United States|-6|apartment| +18336|AAAAAAAAAKHEAAAA|490|Cedar |Boulevard|Suite 380|Salem|Washington County|FL|38048|United States|-5|condo| +18337|AAAAAAAABKHEAAAA|155|Fourth Maple|Parkway|Suite Y|Oak Hill|Suffolk County|NY|17838|United States|-5|condo| +18338|AAAAAAAACKHEAAAA|188|Green |Boulevard|Suite S|Arlington|Eddy County|ND|56557|United States|-6|single family| +18339|AAAAAAAADKHEAAAA|816|Ridge 3rd|Parkway|Suite 340|Riverside|Mendocino County|CA|99231|United States|-8|single family| +18340|AAAAAAAAEKHEAAAA|||Way||Liberty|||33451|United States||single family| +18341|AAAAAAAAFKHEAAAA|145|Jefferson |Circle|Suite S|Ridgeville|Putnam County|TN|39306|United States|-6|single family| +18342|AAAAAAAAGKHEAAAA|581|West |Parkway|Suite 460|Oakwood|Henderson County|KY|40169|United States|-6|condo| +18343|AAAAAAAAHKHEAAAA|869|Mill Hillcrest|Street|Suite 90|Macedonia|Lauderdale County|TN|31087|United States|-6|single family| +18344|AAAAAAAAIKHEAAAA|414|Mill |Street|Suite 300|Hillcrest|Woodward County|OK|73003|United States|-6|condo| +18345|AAAAAAAAJKHEAAAA|623|Lincoln 3rd|Cir.|Suite S|Clifton|Washington County|OR|98014|United States|-8|single family| +18346|AAAAAAAAKKHEAAAA||5th Jackson|Ln|||||46557|United States||| +18347|AAAAAAAALKHEAAAA|228|Madison Pine|Ln|Suite V|Pleasant Grove|Crawford County|IN|44136|United States|-5|apartment| +18348|AAAAAAAAMKHEAAAA|786|Hickory Johnson|Blvd|Suite 400|Maywood|Otsego County|NY|15681|United States|-5|condo| +18349|AAAAAAAANKHEAAAA|244||Road|Suite 20||Starke County||45709||-5|| +18350|AAAAAAAAOKHEAAAA|172|3rd 4th|Ln|Suite Q|Mountain View|Santa Fe County|NM|84466|United States|-7|apartment| +18351|AAAAAAAAPKHEAAAA|643|East Pine|Boulevard|Suite 350|Greenville|Trempealeau County|WI|51387|United States|-6|single family| +18352|AAAAAAAAALHEAAAA|813|4th |Way|Suite G|Ludlow|Churchill County|NV|85566|United States|-8|apartment| +18353|AAAAAAAABLHEAAAA|610|View Park|Street|Suite 120|Mount Pleasant|Hamilton County|FL|31933|United States|-5|single family| +18354|AAAAAAAACLHEAAAA|860|Meadow Pine|Dr.|Suite F|Jamestown|Ada County|ID|86867|United States|-7|single family| +18355|AAAAAAAADLHEAAAA|9|Main |Court|Suite 200|Walnut Grove|Franklin County|OH|47752|United States|-5|single family| +18356|AAAAAAAAELHEAAAA|437|Lake Woodland|Court|Suite D|Star|Leelanau County|MI|40725|United States|-5|single family| +18357|AAAAAAAAFLHEAAAA|194|South 5th|Avenue|Suite 120|Greenville|Stokes County|NC|21387|United States|-5|apartment| +18358|AAAAAAAAGLHEAAAA|293|Third |Wy|Suite 290|Glenwood|Prince George County|MD|23511|United States|-5|condo| +18359|AAAAAAAAHLHEAAAA|849|5th |Circle|Suite A|Mount Olive|Mineral County|MT|68059|United States|-7|apartment| +18360|AAAAAAAAILHEAAAA|238|Cherry |Blvd|Suite 360|Jamestown|Stark County|ND|56867|United States|-6|apartment| +18361|AAAAAAAAJLHEAAAA|577|West |Blvd|Suite 160|Greenfield|O-Brien County|IA|55038|United States|-6|apartment| +18362|AAAAAAAAKLHEAAAA|741|Johnson |Boulevard|Suite Y|Bunker Hill|Snohomish County|WA|90150|United States|-8|apartment| +18363|AAAAAAAALLHEAAAA|243|Second 2nd|ST|Suite G|Glendale|Mitchell County|TX|73951|United States|-6|single family| +18364|AAAAAAAAMLHEAAAA|319|3rd 7th|Blvd|Suite 450|Sunnyside|Polk County|NC|21952|United States|-5|condo| +18365|AAAAAAAANLHEAAAA|587|Central Davis|RD|Suite 90|Jamestown|Rosebud County|MT|66867|United States|-7|condo| +18366|AAAAAAAAOLHEAAAA|949|Valley |Parkway|Suite H|Union|Beadle County|SD|58721|United States|-6|condo| +18367|AAAAAAAAPLHEAAAA|215|7th Center|RD|Suite A|Walnut Grove|Alleghany County|VA|27752|United States|-5|single family| +18368|AAAAAAAAAMHEAAAA|82|Cherry Main|Cir.|Suite 260|Riverdale|Nacogdoches County|TX|79391|United States|-6|apartment| +18369|AAAAAAAABMHEAAAA|450|9th 5th|Drive|Suite 370|Globe|Henderson County|TX|79614|United States|-6|condo| +18370|AAAAAAAACMHEAAAA|591|7th Johnson|Way|Suite 150|Red Bank|Lumpkin County|GA|34975|United States|-5|condo| +18371|AAAAAAAADMHEAAAA|544||||Highland Park|Tishomingo County|MS|||-6|apartment| +18372|AAAAAAAAEMHEAAAA|362|View Woodland|Way|Suite X|Friendship|Tyler County|WV|24536|United States|-5|single family| +18373|AAAAAAAAFMHEAAAA|822|Lake 3rd|Lane|Suite 480|Mount Zion|Terrell County|TX|78054|United States|-6|apartment| +18374|AAAAAAAAGMHEAAAA|541|Third Willow|Pkwy|Suite S|Woodville|Prince George County|VA|24289|United States|-5|condo| +18375|AAAAAAAAHMHEAAAA|550|Mill |Lane|Suite X|Stringtown|McIntosh County|OK|70162|United States|-6|condo| +18376|AAAAAAAAIMHEAAAA|431|Lee |Road|Suite 10|Ryan|Northampton County|VA|20525|United States|-5|single family| +18377|AAAAAAAAJMHEAAAA|353|13th |ST|Suite I|Union Hill|Kane County|IL|67746|United States|-6|condo| +18378|AAAAAAAAKMHEAAAA|114|Dogwood 4th|Blvd|Suite L|New Hope|Menominee County|MI|49431|United States|-5|apartment| +18379|AAAAAAAALMHEAAAA|||RD|Suite K||Hamilton County||79971|United States|-6|| +18380|AAAAAAAAMMHEAAAA|735|Pine |Wy|Suite 440|Freedom|Windham County|VT|02251|United States|-5|apartment| +18381|AAAAAAAANMHEAAAA|693|Main Pine|Street|Suite R|Oakwood|Catoosa County|GA|30169|United States|-5|apartment| +18382|AAAAAAAAOMHEAAAA|136|Sunset |Ave|Suite 160|Greenwood|Williamson County|IL|68828|United States|-6|apartment| +18383|AAAAAAAAPMHEAAAA|723|4th |Cir.|Suite L|Fairview|Essex County|VA|25709|United States|-5|condo| +18384|AAAAAAAAANHEAAAA||||Suite S||Salt Lake County|UT|82808|United States|-7|| +18385|AAAAAAAABNHEAAAA|130|Fourth Hill|RD|Suite 100|Glenwood|Monroe County|NY|13511|United States|-5|apartment| +18386|AAAAAAAACNHEAAAA|667|Sunset |Street|Suite R|Marion|Nelson County|VA|20399|United States|-5|single family| +18387|AAAAAAAADNHEAAAA|107|Spring Lincoln|Parkway|Suite I|Springdale|Linn County|IA|58883|United States|-6|single family| +18388|AAAAAAAAENHEAAAA|599|Highland |Ave|Suite 130|Bethel|Wheeler County|TX|75281|United States|-6|apartment| +18389|AAAAAAAAFNHEAAAA|665|East |Avenue|Suite Q|Five Points|Pike County|GA|36098|United States|-5|single family| +18390|AAAAAAAAGNHEAAAA|865|River |Pkwy|Suite 100|Mount Vernon|Tyler County|WV|28482|United States|-5|apartment| +18391|AAAAAAAAHNHEAAAA|733|Laurel 8th|Lane|Suite 240|Peru|El Paso County|TX|70302|United States|-6|apartment| +18392|AAAAAAAAINHEAAAA|854|Hill Park|Circle|Suite S|Antioch|Lake County|TN|38605|United States|-6|single family| +18393|AAAAAAAAJNHEAAAA||6th 7th|Street|Suite H|||||United States||condo| +18394|AAAAAAAAKNHEAAAA|209|Railroad Elm|Circle|Suite 240|Highland Park|Routt County|CO|86534|United States|-7|single family| +18395|AAAAAAAALNHEAAAA|409|Lincoln |Lane|Suite V|Gary|Benson County|ND|50418|United States|-6|condo| +18396|AAAAAAAAMNHEAAAA|604|Church Mill|Dr.|Suite M|Wildwood|Maricopa County|AZ|86871|United States|-7|single family| +18397|AAAAAAAANNHEAAAA|669|West |Street||Hillcrest|Chisago County|MN|53003|United States||single family| +18398|AAAAAAAAONHEAAAA|235|View Birch|Avenue|Suite 380|Kingston|Lynchburg city|VA|24975|United States|-5|single family| +18399|AAAAAAAAPNHEAAAA|224|7th 5th|Cir.|Suite 440|Newport|Haskell County|OK|71521|United States|-6|condo| +18400|AAAAAAAAAOHEAAAA|441|Sunset Johnson|Wy|Suite D|Green Acres|Klamath County|OR|97683|United States|-8|condo| +18401|AAAAAAAABOHEAAAA|799|15th Pine|Court|Suite 200|Belmont|Sonoma County|CA|90191|United States|-8|condo| +18402|AAAAAAAACOHEAAAA|||||||MO|64136||-6|single family| +18403|AAAAAAAADOHEAAAA|330|Spruce |Street|Suite 90|Greenfield|Sampson County|NC|25038|United States|-5|condo| +18404|AAAAAAAAEOHEAAAA|855|Valley 14th|Drive|Suite 250|Oak Grove|Mercer County|KY|48370|United States|-5|condo| +18405|AAAAAAAAFOHEAAAA|404|Lee |RD|Suite 470|Crossroads|Tyler County|WV|20534|United States|-5|condo| +18406|AAAAAAAAGOHEAAAA|389|Spruce Tenth|Pkwy|Suite 160|Thompsonville|Garfield County|UT|89651|United States|-7|apartment| +18407|AAAAAAAAHOHEAAAA|61|Park |Boulevard|Suite L|Five Forks|Shelby County|AL|32293|United States|-6|apartment| +18408|AAAAAAAAIOHEAAAA|229|Hill |RD|Suite J|Buena Vista|Johnson County|IL|65752|United States|-6|single family| +18409|AAAAAAAAJOHEAAAA|405|Franklin Central|Lane|Suite 420|Waterloo|Gillespie County|TX|71675|United States|-6|condo| +18410|AAAAAAAAKOHEAAAA|98|1st |Circle|Suite B|Macedonia|Madison County|MS|51087|United States|-6|condo| +18411|AAAAAAAALOHEAAAA|340|College First|Wy|Suite F|Lakeview|Philadelphia County|PA|18579|United States|-5|condo| +18412|AAAAAAAAMOHEAAAA|178|5th |Cir.|Suite 140|Spring Hill|Butler County|MO|66787|United States|-6|single family| +18413|AAAAAAAANOHEAAAA|105|Hill |Blvd|Suite M|Mount Olive|Brevard County|FL|38059|United States|-5|single family| +18414|AAAAAAAAOOHEAAAA|437|Central Cedar|Ln|Suite H|Riverside|Madera County|CA|99231|United States|-8|condo| +18415|AAAAAAAAPOHEAAAA|544|Smith 14th|Pkwy|Suite S|Pleasant Hill|Burnett County|WI|53604|United States|-6|apartment| +18416|AAAAAAAAAPHEAAAA|182|Hickory Chestnut|Ct.|Suite 260|Deerfield|Dallas County|AL|39840|United States|-6|single family| +18417|AAAAAAAABPHEAAAA|105|Lake |Ln|Suite C|Oak Ridge|Bingham County|ID|88371|United States|-7|condo| +18418|AAAAAAAACPHEAAAA|847|Seventh Ash|Court|Suite N|Concord|Raleigh County|WV|24107|United States|-5|apartment| +18419|AAAAAAAADPHEAAAA|618|Church Williams|Ct.|Suite C|Greendale|Irwin County|GA|39343|United States|-5|apartment| +18420|AAAAAAAAEPHEAAAA|443|Lee North|Pkwy|Suite W|Pleasant Hill|Lamb County|TX|73604|United States|-6|condo| +18421|AAAAAAAAFPHEAAAA|177|Cedar |Boulevard|Suite A|Forest Hills|Madison County|MO|69237|United States|-6|condo| +18422|AAAAAAAAGPHEAAAA|779|Green 12th|Ave|Suite 440|Greenfield|Belknap County|NH|05638|United States|-5|condo| +18423|AAAAAAAAHPHEAAAA|757|Sunset |Lane|Suite 410|Oakwood|Liberty County|FL|30169|United States|-5|apartment| +18424|AAAAAAAAIPHEAAAA|189|North |Boulevard|Suite 30|Lakewood|Hettinger County|ND|58877|United States|-6|single family| +18425|AAAAAAAAJPHEAAAA|131|Spruce 7th|Boulevard|Suite V|Green Acres|Kendall County|TX|77683|United States|-6|condo| +18426|AAAAAAAAKPHEAAAA||||Suite 390|Marion||PA|10399||-5|| +18427|AAAAAAAALPHEAAAA|594|Lincoln |Way|Suite Y|Plainview|Payette County|ID|83683|United States|-7|apartment| +18428|AAAAAAAAMPHEAAAA|144|6th Maple|Court|Suite 140|Greenwood|Bay County|MI|48828|United States|-5|condo| +18429|AAAAAAAANPHEAAAA|366|Spring Meadow|Ct.|Suite I|Walnut Grove|Norfolk city|VA|27752|United States|-5|apartment| +18430|AAAAAAAAOPHEAAAA|469|Elm |Road|Suite 470|Buena Vista|Champaign County|OH|45752|United States|-5|single family| +18431|AAAAAAAAPPHEAAAA||Sixth |RD|||Osage County|OK||United States||single family| +18432|AAAAAAAAAAIEAAAA|126|Poplar Cedar|Drive|Suite 0|Arthur|Clackamas County|OR|95965|United States|-8|apartment| +18433|AAAAAAAABAIEAAAA|205|Green Woodland|Parkway|Suite 330|Pleasant Grove|Clarke County|IA|54136|United States|-6|condo| +18434|AAAAAAAACAIEAAAA||||||Pike County|IL|66787|United States|-6|| +18435|AAAAAAAADAIEAAAA|190|Walnut |ST|Suite 70|Oakland|Lamar County|TX|79843|United States|-6|single family| +18436|AAAAAAAAEAIEAAAA|916|Main |Drive|Suite 180|Enterprise|Sanilac County|MI|41757|United States|-5|apartment| +18437|AAAAAAAAFAIEAAAA|332|Jefferson Second|Cir.|Suite T|Farmington|Walton County|GA|39145|United States|-5|single family| +18438|AAAAAAAAGAIEAAAA|976|Jackson |Way|Suite 390|Maple Grove|Prince George County|MD|28252|United States|-5|apartment| +18439|AAAAAAAAHAIEAAAA|305|11th 8th|Drive|Suite R|Roscoe|Bandera County|TX|71854|United States|-6|condo| +18440|AAAAAAAAIAIEAAAA|901|Jackson 5th|Ave|Suite 360|Hillcrest|Taylor County|TX|73003|United States|-6|condo| +18441|AAAAAAAAJAIEAAAA|17|Sycamore Spring|Ave|Suite B|Five Forks|Marshall County|WV|22293|United States|-5|condo| +18442|AAAAAAAAKAIEAAAA|971|Mill Smith|Avenue|Suite G|Cedar Grove|Brevard County|FL|30411|United States|-5|condo| +18443|AAAAAAAALAIEAAAA|975|Woodland 12th|Boulevard|Suite 70|Mount Vernon|Jackson County|KY|48482|United States|-6|condo| +18444|AAAAAAAAMAIEAAAA|202|Maple |Road|Suite V|Union Hill|Wyandot County|OH|47746|United States|-5|single family| +18445|AAAAAAAANAIEAAAA|86|Cedar Central|Pkwy|Suite 350|Summit|Marshall County|WV|20499|United States|-5|single family| +18446|AAAAAAAAOAIEAAAA|440|Valley Jefferson|Dr.|Suite 130|Plainview|Meeker County|MN|53683|United States|-6|single family| +18447|AAAAAAAAPAIEAAAA|276|2nd |Dr.|Suite Q|Saint Clair|Pike County|MS|55294|United States|-6|apartment| +18448|AAAAAAAAABIEAAAA|230|Ridge Elm|Parkway|Suite J|Red Bank|Gibson County|TN|34975|United States|-5|single family| +18449|AAAAAAAABBIEAAAA|539|East Park|Dr.|Suite 250|Lakeview|Ohio County|IN|48579|United States|-5|condo| +18450|AAAAAAAACBIEAAAA|389|South |Ave|Suite 130|Mount Zion|Graham County|KS|68054|United States|-6|single family| +18451|AAAAAAAADBIEAAAA|782|Fourteenth |Dr.|Suite L|Woodville|Washita County|OK|74289|United States|-6|single family| +18452|AAAAAAAAEBIEAAAA|393|Park Woodland|Pkwy|Suite G|Clearwater|Kiowa County|KS|69534|United States|-6|single family| +18453|AAAAAAAAFBIEAAAA|666|Walnut |Blvd|Suite W|White Oak|Grady County|GA|36668|United States|-5|apartment| +18454|AAAAAAAAGBIEAAAA|192|3rd |Pkwy|Suite S|Valley View|Brooke County|WV|25124|United States|-5|single family| +18455|AAAAAAAAHBIEAAAA|215|Fifth |Parkway|Suite D|Friendship|Dade County|GA|34536|United States|-5|condo| +18456|AAAAAAAAIBIEAAAA|672|Washington 4th|RD|Suite V|Providence|Manassas Park city|VA|26614|United States|-5|apartment| +18457|AAAAAAAAJBIEAAAA|704|2nd |ST|Suite 260|Milan|Pine County|MN|56697|United States|-6|apartment| +18458|AAAAAAAAKBIEAAAA|109|Ninth |Circle|Suite R|Golden|Lincoln County|NV|80411|United States|-8|single family| +18459|AAAAAAAALBIEAAAA|802|River |Cir.|Suite P|Tipton|Bowie County|TX|78773|United States|-6|condo| +18460|AAAAAAAAMBIEAAAA|202|11th Seventh|Ave|Suite B|Woodlawn|Hot Springs County|WY|84098|United States|-7|condo| +18461|AAAAAAAANBIEAAAA|769|Hillcrest 5th|Parkway|Suite D|Sunnyside|Coke County|TX|71952|United States|-6|apartment| +18462|AAAAAAAAOBIEAAAA|453|9th Central|Pkwy|Suite F|Riverside|Polk County|NE|69231|United States|-7|apartment| +18463|AAAAAAAAPBIEAAAA|22|3rd |Avenue|Suite 380|Chestnut Ridge|Elliott County|KY|47334|United States|-6|condo| +18464|AAAAAAAAACIEAAAA|138|Cedar |RD|Suite W|Bunker Hill|Crenshaw County|AL|30150|United States|-6|condo| +18465|AAAAAAAABCIEAAAA|909|Hickory Lake|Way|Suite G|Macedonia|Tioga County|PA|11087|United States|-5|single family| +18466|AAAAAAAACCIEAAAA|970|Forest |Ave|Suite L|Gravel Hill|Fayette County|TN|31944|United States|-5|apartment| +18467|AAAAAAAADCIEAAAA|523|Jackson Spring|Ln|Suite 320|Rocky Point|Red River County|TX|71209|United States|-6|apartment| +18468|AAAAAAAAECIEAAAA|118|Maple Williams|Parkway|Suite D|Salem|Vilas County|WI|58048|United States|-6|condo| +18469|AAAAAAAAFCIEAAAA|876|4th |RD|Suite 370|Wayland|Polk County|AR|75115|United States|-6|condo| +18470|AAAAAAAAGCIEAAAA|407|Walnut |Way|Suite L|Farmington|Armstrong County|PA|19145|United States|-5|condo| +18471|AAAAAAAAHCIEAAAA|82|First 3rd|Ln|Suite 60|Oak Grove|Russell County|KS|68370|United States|-6|single family| +18472|AAAAAAAAICIEAAAA|354|Lee |Avenue|Suite 260|Harmony|Douglas County|SD|55804|United States|-6|condo| +18473|AAAAAAAAJCIEAAAA|192|3rd ||Suite P||Union County|||United States||single family| +18474|AAAAAAAAKCIEAAAA|684|Maple 2nd|Blvd|Suite F|Enterprise|Geary County|KS|61757|United States|-6|apartment| +18475|AAAAAAAALCIEAAAA|502|Hickory First|Way|Suite X|Jackson|Starke County|IN|49583|United States|-5|apartment| +18476|AAAAAAAAMCIEAAAA|22||Court|Suite S|Buena Vista|Washington County|VT|06352|United States||| +18477|AAAAAAAANCIEAAAA|597|Hill |Circle|Suite B|Highland Park|Washakie County|WY|86534|United States|-7|condo| +18478|AAAAAAAAOCIEAAAA|846|1st |Pkwy|Suite T|Lakeview|Fayette County|KY|48579|United States|-6|single family| +18479|AAAAAAAAPCIEAAAA|965|Lincoln |Street|Suite 380|Antioch|Pepin County|WI|58605|United States|-6|apartment| +18480|AAAAAAAAADIEAAAA|279|Ridge |Road|Suite 60|Highland Park|Hudson County|NJ|07134|United States|-5|single family| +18481|AAAAAAAABDIEAAAA|237|9th West|Wy|Suite 360|Farmington|Kay County|OK|79145|United States|-6|single family| +18482|AAAAAAAACDIEAAAA|188|1st |Drive|Suite 160|Woodlawn|Jefferson County|WI|54098|United States|-6|condo| +18483|AAAAAAAADDIEAAAA|965|Third College|Drive|Suite R|Pleasant Grove|Calhoun County|IA|54136|United States|-6|condo| +18484|AAAAAAAAEDIEAAAA|457|Ash |Court|Suite F|Belmont|Gladwin County|MI|40191|United States|-5|apartment| +18485|AAAAAAAAFDIEAAAA|441|Adams 2nd|Way|Suite 330|Deerfield|Union County|OR|99840|United States|-8|apartment| +18486|AAAAAAAAGDIEAAAA|168|First |RD|Suite 290|Fairfield|Carson City|NV|86192|United States|-8|apartment| +18487|AAAAAAAAHDIEAAAA|630|Elm 2nd|Ave|Suite S|Caledonia|Rolette County|ND|57411|United States|-6|single family| +18488|AAAAAAAAIDIEAAAA|79|5th |Circle|Suite 250|Enterprise|Burlington County|NJ|02357|United States|-5|single family| +18489|AAAAAAAAJDIEAAAA|371|10th 3rd|Drive|Suite 300|Fairview|Rock Island County|IL|65709|United States|-6|condo| +18490|AAAAAAAAKDIEAAAA|81|Main College|Wy|Suite J|Mountain View|Tolland County|CT|05066|United States|-5|condo| +18491|AAAAAAAALDIEAAAA|809|12th Main|Circle|Suite 110|Highland Park|Elbert County|GA|36534|United States|-5|condo| +18492|AAAAAAAAMDIEAAAA|517|Sunset |Ave|Suite I|Midway|Santa Fe County|NM|81904|United States|-7|condo| +18493|AAAAAAAANDIEAAAA|20|Park |Way|Suite 20|Midway|Crawford County|AR|71904|United States|-6|single family| +18494|AAAAAAAAODIEAAAA|849|Jackson Willow|Dr.|Suite 280|Five Points|Grundy County|TN|36098|United States|-5|single family| +18495|AAAAAAAAPDIEAAAA|694|East |RD|Suite 360|Somerville|Adams County|IN|47783|United States|-5|single family| +18496|AAAAAAAAAEIEAAAA|419|Lee |Lane|Suite 60|Murphy|Kittitas County|WA|92105|United States|-8|single family| +18497|AAAAAAAABEIEAAAA|166|10th |Drive|Suite 180|Florence|Butte County|SD|53394|United States|-6|single family| +18498|AAAAAAAACEIEAAAA|853|Railroad Laurel|Wy|Suite H|Riverview|Ottawa County|OK|79003|United States|-6|single family| +18499|AAAAAAAADEIEAAAA|658|View |Court|Suite 70|Sunnyside|Washtenaw County|MI|41952|United States|-5|single family| +18500|AAAAAAAAEEIEAAAA|464||Drive||Woodville||KY|44289|United States||single family| +18501|AAAAAAAAFEIEAAAA|858|Church |Parkway|Suite 140|Lee|Washington County|ME|01008|United States|-5|apartment| +18502|AAAAAAAAGEIEAAAA|517||||||AR||United States|-6|single family| +18503|AAAAAAAAHEIEAAAA|896|Main |Court|Suite 110|Freeport|Essex County|VT|02444|United States|-5|condo| +18504|AAAAAAAAIEIEAAAA|415|Ash Mill|Avenue|Suite 0|Burns|Washington County|IA|55272|United States|-6|condo| +18505|AAAAAAAAJEIEAAAA|293|4th Franklin|Drive|Suite 340|Fairfield|Clay County|KS|66192|United States|-6|apartment| +18506|AAAAAAAAKEIEAAAA|413|Elm |Avenue|Suite J|Walnut Grove|Carroll County|NH|08352|United States|-5|single family| +18507|AAAAAAAALEIEAAAA|845|Locust |Avenue|Suite B|Hamilton|Koochiching County|MN|52808|United States|-6|apartment| +18508|AAAAAAAAMEIEAAAA|458|12th Spruce|Ct.|Suite X|Bridgeport|Fayette County|AL|35817|United States|-6|condo| +18509|AAAAAAAANEIEAAAA|247|6th |Cir.|Suite N|Waterloo|Cleveland County|AR|71675|United States|-6|condo| +18510|AAAAAAAAOEIEAAAA|619|Sixth |Parkway|Suite 300|Florence|Alleghany County|NC|23394|United States|-5|condo| +18511|AAAAAAAAPEIEAAAA|882|Walnut |Court|Suite I|Roxbury|Hampshire County|WV|25508|United States|-5|single family| +18512|AAAAAAAAAFIEAAAA|932|7th Second|Dr.|Suite L|Jackson|Merced County|CA|99583|United States|-8|condo| +18513|AAAAAAAABFIEAAAA|338|9th |Circle|Suite 240|Lakewood|Butler County|NE|68877|United States|-6|single family| +18514|AAAAAAAACFIEAAAA|359|North |RD|Suite 340|Cedar Grove|Storey County|NV|80411|United States|-8|apartment| +18515|AAAAAAAADFIEAAAA|216|Adams Elm|Wy|Suite U|Liberty|Guernsey County|OH|43451|United States|-5|single family| +18516|AAAAAAAAEFIEAAAA|404|8th Walnut|Pkwy|Suite B|Oakland|San Juan County|UT|89843|United States|-7|apartment| +18517|AAAAAAAAFFIEAAAA|560|Cedar |RD|Suite E|Stringtown|DuPage County|IL|60162|United States|-6|apartment| +18518|AAAAAAAAGFIEAAAA|274|5th |RD|Suite L|Oak Grove|Atlantic County|NJ|08970|United States|-5|apartment| +18519|AAAAAAAAHFIEAAAA|512|4th |Circle|Suite 480|Globe|Labette County|KS|69614|United States|-6|condo| +18520|AAAAAAAAIFIEAAAA|||Court|Suite 320|Riverdale||MD|29391|United States|-5|| +18521|AAAAAAAAJFIEAAAA|783|Hillcrest |Dr.|Suite 140|Cedar Grove|Prince George County|MD|20411|United States|-5|condo| +18522|AAAAAAAAKFIEAAAA|66|2nd |Ct.|Suite S|Peoria|Latimer County|OK|79818|United States|-6|condo| +18523|AAAAAAAALFIEAAAA|836|Franklin Hickory|RD|Suite 200|Lakeview|Alameda County|CA|98579|United States|-8|condo| +18524|AAAAAAAAMFIEAAAA|337|Elm Sycamore|Wy|Suite N|Plainview|Barber County|KS|63683|United States|-6|single family| +18525|AAAAAAAANFIEAAAA|632|Laurel Cedar|Dr.|Suite 320|Crossroads|Clinton County|IA|50534|United States|-6|condo| +18526|AAAAAAAAOFIEAAAA|894|2nd Adams|Ave|Suite 210|Belmont|Glacier County|MT|60191|United States|-7|condo| +18527|AAAAAAAAPFIEAAAA|435|Oak Miller|Ln|Suite 350|Mount Pleasant|Pemiscot County|MO|61933|United States|-6|single family| +18528|AAAAAAAAAGIEAAAA|684|Park |Drive|Suite U|Sulphur Springs|Beckham County|OK|78354|United States|-6|apartment| +18529|AAAAAAAABGIEAAAA|737|Meadow |Circle|Suite R|Valley View|Dunklin County|MO|65124|United States|-6|apartment| +18530|AAAAAAAACGIEAAAA|629|Valley North|Ln|Suite 70|Mount Olive|Surry County|NC|28059|United States|-5|condo| +18531|AAAAAAAADGIEAAAA|551|Forest Williams|Circle|Suite Y|Jerome|Pawnee County|KS|69920|United States|-6|condo| +18532|AAAAAAAAEGIEAAAA|22|Cherry |Ave|Suite 120|Colonial Heights|Santa Barbara County|CA|93425|United States|-8|apartment| +18533|AAAAAAAAFGIEAAAA|553|Valley |Parkway|Suite T|Mount Olive|Eastland County|TX|78059|United States|-6|apartment| +18534|AAAAAAAAGGIEAAAA|346|2nd |Circle|Suite 150|Marion|Cumberland County|ME|00999|United States|-5|single family| +18535|AAAAAAAAHGIEAAAA|242|Main |Boulevard|Suite J|Woodville|Scott County|MN|54289|United States|-6|condo| +18536|AAAAAAAAIGIEAAAA|968|Mill Hillcrest|Street|Suite S|Georgetown|Chippewa County|MI|47057|United States|-5|single family| +18537|AAAAAAAAJGIEAAAA|230|15th Seventh|Blvd|Suite 240|Oakwood|Grundy County|IL|60169|United States|-6|apartment| +18538|AAAAAAAAKGIEAAAA|708|Ash |Ct.|Suite 440|Pinhook|Greenup County|KY|49398|United States|-6|single family| +18539|AAAAAAAALGIEAAAA|952|Park |Way|Suite A|Mount Vernon|Carroll County|AR|78482|United States|-6|condo| +18540|AAAAAAAAMGIEAAAA|383|Johnson 2nd|Ln|Suite 460|Clifton|Tama County|IA|58014|United States|-6|single family| +18541|AAAAAAAANGIEAAAA|904|Railroad Washington|Street|Suite 430|Shady Grove|Campbell County|VA|22812|United States|-5|single family| +18542|AAAAAAAAOGIEAAAA|531|4th |Dr.|Suite H|Mount Olive|Sumner County|TN|38059|United States|-6|apartment| +18543|AAAAAAAAPGIEAAAA|141|Smith |Ave|Suite V|Buena Vista|Caldwell Parish|LA|75752|United States|-6|apartment| +18544|AAAAAAAAAHIEAAAA|579|1st |Cir.|Suite 490|Mount Vernon|Boone County|IN|48482|United States|-5|apartment| +18545|AAAAAAAABHIEAAAA|575|Woodland Adams|Ln|Suite J|Hamilton|Grady County|GA|32808|United States|-5|single family| +18546|AAAAAAAACHIEAAAA|212|Third |Parkway|Suite O|Montpelier|Caldwell Parish|LA|78930|United States|-6|apartment| +18547|AAAAAAAADHIEAAAA|934|Seventh 11th|Blvd|Suite 30|Woodland|Erie County|PA|14854|United States|-5|condo| +18548|AAAAAAAAEHIEAAAA|328|Poplar |Circle|Suite P|Newtown|Carlton County|MN|51749|United States|-6|apartment| +18549|AAAAAAAAFHIEAAAA|509|7th 1st|Blvd|Suite E|Jamestown|Minnehaha County|SD|56867|United States|-7|condo| +18550|AAAAAAAAGHIEAAAA|819|Maple West|Cir.|Suite K|Edgewood|Lawrence County|AR|70069|United States|-6|single family| +18551|AAAAAAAAHHIEAAAA|15|||Suite G|Fairfield||AL||United States||single family| +18552|AAAAAAAAIHIEAAAA|564|North Park|Blvd|Suite 380|Hopewell|Campbell County|TN|30587|United States|-5|apartment| +18553|AAAAAAAAJHIEAAAA|270|West |Parkway|Suite I|Georgetown|Benson County|ND|57057|United States|-6|apartment| +18554|AAAAAAAAKHIEAAAA|970|View 8th|Way|Suite J|Buena Vista|Montgomery County|KS|65752|United States|-6|single family| +18555|AAAAAAAALHIEAAAA|252|Pine |Ln|Suite P|Woodland|Washington County|KS|64854|United States|-6|single family| +18556|AAAAAAAAMHIEAAAA|701|Green 6th|ST|Suite 340|Crossroads|Scotland County|NC|20534|United States|-5|apartment| +18557|AAAAAAAANHIEAAAA|419|3rd Wilson|Avenue|Suite W|Martinsville|Refugio County|TX|70419|United States|-6|apartment| +18558|AAAAAAAAOHIEAAAA|609|Valley |Boulevard|Suite 290|Hillcrest|Russell County|VA|23003|United States|-5|condo| +18559|AAAAAAAAPHIEAAAA|618|Davis 2nd|Road|Suite 450|Hamilton|Clinton County|OH|42808|United States|-5|apartment| +18560|AAAAAAAAAIIEAAAA|865|Cherry Elm|Avenue|Suite 420|Marion|Hughes County|OK|70399|United States|-6|single family| +18561|AAAAAAAABIIEAAAA|839|Spring |Ct.|Suite 70|Oakdale|Cumberland County|IL|69584|United States|-6|single family| +18562|AAAAAAAACIIEAAAA|327|Birch |Court|Suite 240|Milltown|Greene County|NC|27534|United States|-5|single family| +18563|AAAAAAAADIIEAAAA|505|12th |Avenue|Suite Y|Fairview|Branch County|MI|45709|United States|-5|single family| +18564|AAAAAAAAEIIEAAAA|426|Smith |Drive|Suite F|Greenfield|Shelby County|IL|65038|United States|-6|condo| +18565|AAAAAAAAFIIEAAAA|205|Wilson |Avenue|Suite 300|Woodlawn|Suffolk city|VA|24098|United States|-5|single family| +18566|AAAAAAAAGIIEAAAA|126|Ninth |Ln|Suite 350|Walnut Grove|Decatur County|TN|37752|United States|-5|condo| +18567|AAAAAAAAHIIEAAAA|402|Walnut 5th|Pkwy|Suite J|Leon|Lafayette Parish|LA|70913|United States|-6|single family| +18568|AAAAAAAAIIIEAAAA|88|2nd |Street|Suite 160|Stringtown|Chippewa County|MN|50162|United States|-6|condo| +18569|AAAAAAAAJIIEAAAA|67|South Broadway|Parkway|Suite 20|Pleasant Valley|Montgomery County|AR|72477|United States|-6|condo| +18570|AAAAAAAAKIIEAAAA|391|Jefferson |Parkway|Suite I|Clifford|Osceola County|FL|38164|United States|-5|single family| +18571|AAAAAAAALIIEAAAA|300|Hill Johnson|Lane|Suite 20|Jackson|Washington County|ME|09583|United States|-5|condo| +18572|AAAAAAAAMIIEAAAA|406|1st |RD|Suite 70|Salem|Wasatch County|UT|88048|United States|-7|apartment| +18573|AAAAAAAANIIEAAAA|617|Fourth Lincoln|Wy|Suite 330|Greenwood|Harlan County|KY|48828|United States|-6|single family| +18574|AAAAAAAAOIIEAAAA|58|South |Way|Suite Q|Arlington|New Kent County|VA|26557|United States|-5|apartment| +18575|AAAAAAAAPIIEAAAA|571|Cedar |Boulevard|Suite 370|Midway|Eddy County|NM|81904|United States|-7|apartment| +18576|AAAAAAAAAJIEAAAA|875|Cedar Hillcrest|Parkway|Suite 70|Summit|Fulton County|IL|60499|United States|-6|condo| +18577|AAAAAAAABJIEAAAA|172|11th |Cir.|Suite 420|Mountain View|Sedgwick County|KS|64466|United States|-6|single family| +18578|AAAAAAAACJIEAAAA|77|Meadow |RD|Suite 110|Wildwood|Las Animas County|CO|86871|United States|-7|apartment| +18579|AAAAAAAADJIEAAAA|408|Green |Wy|Suite 390|Woodville|Dakota County|MN|54289|United States|-6|condo| +18580|AAAAAAAAEJIEAAAA|871|Lake |Avenue|Suite J|Sulphur Springs|Lycoming County|PA|18354|United States|-5|condo| +18581|AAAAAAAAFJIEAAAA|294|Main Cedar|Pkwy|Suite 290|Woodville|Pierce County|GA|34289|United States|-5|single family| +18582|AAAAAAAAGJIEAAAA|620|Fourth 1st|Avenue|Suite 340|Midway|Larimer County|CO|81904|United States|-7|condo| +18583|AAAAAAAAHJIEAAAA|938|Green Jackson|Ln||||WA||United States||condo| +18584|AAAAAAAAIJIEAAAA||||Suite N||Sangamon County|IL|61933|||| +18585|AAAAAAAAJJIEAAAA|133|Park Hill|RD|Suite E|Highland Park|Boone County|IL|66534|United States|-6|single family| +18586|AAAAAAAAKJIEAAAA|628|First Laurel|Blvd|Suite C|Jackson|Northampton County|VA|29583|United States|-5|condo| +18587|AAAAAAAALJIEAAAA|903|2nd 5th|Cir.|Suite 190|Union|Hamilton County|OH|48721|United States|-5|single family| +18588|AAAAAAAAMJIEAAAA|342|Hickory |Way|Suite H|Newport|Polk County|NE|61521|United States|-7|condo| +18589|AAAAAAAANJIEAAAA|||Pkwy|Suite 490||Freeborn County||50069|||| +18590|AAAAAAAAOJIEAAAA|550|Madison |Court|Suite 350|Green Hill|Manatee County|FL|37422|United States|-5|apartment| +18591|AAAAAAAAPJIEAAAA|409|Sunset |Pkwy|Suite E|Kingston|Los Alamos County|NM|84975|United States|-7|condo| +18592|AAAAAAAAAKIEAAAA|456|Madison Church|Cir.|Suite 430|Jamestown|Prince George County|VA|26867|United States|-5|apartment| +18593|AAAAAAAABKIEAAAA|592|1st Railroad|Blvd|Suite 110|Spring Hill|Kent County|MI|46787|United States|-5|condo| +18594|AAAAAAAACKIEAAAA|789|1st 13th|Dr.|Suite 10|Oakwood|Butler County|PA|10169|United States|-5|apartment| +18595|AAAAAAAADKIEAAAA|803|5th Forest|Way|Suite M|Green Acres|Jefferson County|CO|87683|United States|-7|condo| +18596|AAAAAAAAEKIEAAAA|347|Wilson Lake|Ct.|Suite 80|Oak Hill|Frontier County|NE|67838|United States|-6|single family| +18597|AAAAAAAAFKIEAAAA|54|Hillcrest 6th|Ln|Suite S|Salem|Carroll County|MD|28048|United States|-5|apartment| +18598|AAAAAAAAGKIEAAAA|10|Elm Washington|Circle|Suite 160|Edgewood|Warren County|TN|30069|United States|-6|apartment| +18599|AAAAAAAAHKIEAAAA|390|Wilson South|Lane|Suite 420|Oakland|Lincoln County|TN|39843|United States|-6|condo| +18600|AAAAAAAAIKIEAAAA|656|||||||74854||-6|| +18601|AAAAAAAAJKIEAAAA|290|Birch |Circle|Suite 270|Deerfield|Montgomery County|NC|29840|United States|-5|condo| +18602|AAAAAAAAKKIEAAAA|248|5th |Street|Suite 200|Highland|Calhoun County|AR|79454|United States|-6|single family| +18603|AAAAAAAALKIEAAAA|556|Washington |Wy|Suite H|Midway|Kinney County|TX|71904|United States|-6|condo| +18604|AAAAAAAAMKIEAAAA|453|Broadway |Ln|Suite 350|Shady Grove|Haskell County|KS|62812|United States|-6|single family| +18605|AAAAAAAANKIEAAAA|403|First Second|Pkwy|Suite 380|Greenfield|Lafourche Parish|LA|75038|United States|-6|condo| +18606|AAAAAAAAOKIEAAAA|252|Mill |Way|Suite S|Harmony|Colorado County|TX|75804|United States|-6|condo| +18607|AAAAAAAAPKIEAAAA|454|River |Blvd|Suite R|Lakewood|Santa Fe County|NM|88877|United States|-7|single family| +18608|AAAAAAAAALIEAAAA|457|First Adams|Dr.|Suite F|Greenwood|Elk County|KS|68828|United States|-6|apartment| +18609|AAAAAAAABLIEAAAA|130|Church 1st|Wy|Suite 130|Lebanon|Columbia County|FL|32898|United States|-5|condo| +18610|AAAAAAAACLIEAAAA|422|East 8th|Parkway|Suite 100|Forest Hills|Stephenson County|IL|69237|United States|-6|single family| +18611|AAAAAAAADLIEAAAA|209|Maple |Drive|Suite 460|Waterloo|Clarke County|AL|31675|United States|-6|single family| +18612|AAAAAAAAELIEAAAA|341|Poplar 5th|Ave|Suite 170|Arlington|Klamath County|OR|96557|United States|-8|apartment| +18613|AAAAAAAAFLIEAAAA|223|Smith Lakeview|Pkwy|Suite 290|Jackson|Grant County|SD|59583|United States|-6|apartment| +18614|AAAAAAAAGLIEAAAA|680|Cedar 6th|Pkwy|Suite 320|Plainview|Greene County|IL|63683|United States|-6|apartment| +18615|AAAAAAAAHLIEAAAA|619|Spruce |Road|Suite P|Hopewell|Linn County|IA|50587|United States|-6|single family| +18616|AAAAAAAAILIEAAAA|378|Spring |Ave|Suite T|Woodville|Monroe County|IL|64289|United States|-6|single family| +18617|AAAAAAAAJLIEAAAA|704|Wilson |ST|Suite 320|Dallas|Skagit County|WA|93628|United States|-8|apartment| +18618|AAAAAAAAKLIEAAAA|491|Church Cedar|Ave|Suite H|Salem|Ford County|KS|68048|United States|-6|condo| +18619|AAAAAAAALLIEAAAA|234|Hill Sycamore|Pkwy|Suite Q|Four Points|Livingston Parish|LA|71216|United States|-6|condo| +18620|AAAAAAAAMLIEAAAA|95|Maple |Pkwy|Suite X|Enterprise|Decatur County|GA|31757|United States|-5|apartment| +18621|AAAAAAAANLIEAAAA|826|Forest Mill|Ln|Suite Q|Red Hill|Anderson County|TX|74338|United States|-6|apartment| +18622|AAAAAAAAOLIEAAAA|7|Maple Elm|Ave|Suite A|Clifton|Pennington County|MN|58014|United States|-6|condo| +18623|AAAAAAAAPLIEAAAA|1|Ridge Seventh|Lane|Suite V|Pleasant Hill|Grayson County|KY|43604|United States|-6|condo| +18624|AAAAAAAAAMIEAAAA|282|Oak Second|Boulevard|Suite 160|Deerfield|Bronx County|NY|19840|United States|-5|apartment| +18625|AAAAAAAABMIEAAAA|377|Wilson |Boulevard|Suite 180|Spring Grove|Edmunds County|SD|56719|United States|-6|condo| +18626|AAAAAAAACMIEAAAA|562|Locust Woodland|Wy|Suite 130|Wilson|Osceola County|MI|46971|United States|-5|condo| +18627|AAAAAAAADMIEAAAA|113|First 11th|Boulevard|Suite R|Midway|Clark County|KY|41904|United States|-6|single family| +18628|AAAAAAAAEMIEAAAA|791|East |Wy|Suite 470|Macedonia|Highlands County|FL|31087|United States|-5|condo| +18629|AAAAAAAAFMIEAAAA|992||Wy|Suite X|Plainview|||53683||-6|apartment| +18630|AAAAAAAAGMIEAAAA|449|Broadway |ST|Suite 490|Belmont|Tazewell County|VA|20191|United States|-5|apartment| +18631|AAAAAAAAHMIEAAAA|566|Oak Cedar|Blvd|Suite 420|Saint George|Sumter County|AL|35281|United States|-6|single family| +18632|AAAAAAAAIMIEAAAA|53|1st |Court|Suite 480|Concord|Anson County|NC|24107|United States|-5|apartment| +18633|AAAAAAAAJMIEAAAA|191|Oak Madison|Parkway|Suite 20|New Hope|Charles County|MD|29431|United States|-5|single family| +18634|AAAAAAAAKMIEAAAA|751|4th |Pkwy|Suite 160|Five Forks|Concordia Parish|LA|72293|United States|-6|condo| +18635|AAAAAAAALMIEAAAA|956|Hill Fourth|RD|Suite 430|Waterloo|Wayne County|IA|51675|United States|-6|apartment| +18636|AAAAAAAAMMIEAAAA|185|Park |Avenue|Suite Y|Lakeside|Glades County|FL|39532|United States|-5|condo| +18637|AAAAAAAANMIEAAAA|82|Lakeview Elm|Cir.|Suite 180|White Oak|Ellis County|OK|76668|United States|-6|single family| +18638|AAAAAAAAOMIEAAAA|148|6th Sunset|Street|Suite 300|Harmony|Burt County|NE|65804|United States|-6|apartment| +18639|AAAAAAAAPMIEAAAA|443|Jefferson 1st|Drive|Suite 50|Woodlawn|Kings County|NY|14098|United States|-5|single family| +18640|AAAAAAAAANIEAAAA|457|15th |Avenue|Suite 330|Florence|Knox County|KY|43394|United States|-5|condo| +18641|AAAAAAAABNIEAAAA|294|Green |Street|Suite 60|Mount Vernon|Beaufort County|NC|28482|United States|-5|condo| +18642|AAAAAAAACNIEAAAA|770|Valley |Ln|Suite C|Bunker Hill|Orleans County|NY|10150|United States|-5|apartment| +18643|AAAAAAAADNIEAAAA|888|Hickory |Court|Suite N|Ashland|Eddy County|ND|54244|United States|-6|condo| +18644|AAAAAAAAENIEAAAA|947|Poplar Mill|Ct.|Suite S|Liberty|Pittsburg County|OK|73451|United States|-6|condo| +18645|AAAAAAAAFNIEAAAA|889|Wilson Adams|Ct.|Suite 230|Fox|Callaway County|MO|60631|United States|-6|apartment| +18646|AAAAAAAAGNIEAAAA|901|Sycamore Hickory|Cir.|Suite 400|Woodland|Bolivar County|MS|54854|United States|-6|condo| +18647|AAAAAAAAHNIEAAAA|240|Church Sycamore|Drive|Suite B|New Hope|Ohio County|KY|49431|United States|-5|single family| +18648|AAAAAAAAINIEAAAA|58|Sunset Walnut|Ct.|Suite 430|Bridgeport|Pendleton County|WV|25817|United States|-5|apartment| +18649|AAAAAAAAJNIEAAAA|604|1st |Circle|Suite G|Florence|King George County|VA|23394|United States|-5|apartment| +18650|AAAAAAAAKNIEAAAA|990|Eigth Hillcrest|Way|Suite G|Cedar Grove|O-Brien County|IA|50411|United States|-6|apartment| +18651|AAAAAAAALNIEAAAA|904|Railroad |Drive|Suite S|Lakeview|Boyd County|KY|48579|United States|-6|apartment| +18652|AAAAAAAAMNIEAAAA|757|Locust |Boulevard|Suite J|Antioch|Columbia County|NY|18605|United States|-5|single family| +18653|AAAAAAAANNIEAAAA|632|9th Center|Lane|Suite N|Greenville|Tompkins County|NY|11387|United States|-5|apartment| +18654|AAAAAAAAONIEAAAA||||Suite 50|Pleasant Grove|Rice County||54136|United States||condo| +18655|AAAAAAAAPNIEAAAA|538|First |Boulevard|Suite 470|Franklin|Clark County|OH|49101|United States|-5|condo| +18656|AAAAAAAAAOIEAAAA|472|Dogwood |Court|Suite 120|Glendale|Madison County|MO|63951|United States|-6|condo| +18657|AAAAAAAABOIEAAAA|607|Park |Blvd|Suite 30|Mount Pleasant|Wayne County|IN|41933|United States|-5|condo| +18658|AAAAAAAACOIEAAAA|322|Ridge |Ct.|Suite 330|Five Forks|Leflore County|MS|52293|United States|-6|condo| +18659|AAAAAAAADOIEAAAA|922|Locust Cedar|Parkway|Suite O|Caledonia|Harrison County|KY|47411|United States|-6|condo| +18660|AAAAAAAAEOIEAAAA|29|Walnut |Pkwy|Suite 20|Farmington|Washington County|TN|39145|United States|-6|apartment| +18661|AAAAAAAAFOIEAAAA|447|Second |Boulevard|Suite 40|Concord|Dooly County|GA|34107|United States|-5|single family| +18662|AAAAAAAAGOIEAAAA|||Wy|Suite Q||Stanislaus County||97057||-8|condo| +18663|AAAAAAAAHOIEAAAA|80|Forest |Lane|Suite P|Hollywood|Jones County|TX|78001|United States|-6|condo| +18664|AAAAAAAAIOIEAAAA||||Suite 140|Jackson|Kearney County||69583||-7|single family| +18665|AAAAAAAAJOIEAAAA|582|East 6th|Ave|Suite I|Marion|Wharton County|TX|70399|United States|-6|condo| +18666|AAAAAAAAKOIEAAAA|523|Walnut 10th|Road|Suite H|Sunnyside|Coos County|OR|91952|United States|-8|single family| +18667|AAAAAAAALOIEAAAA|412|View |Drive|Suite 10|Oak Hill|Essex County|VT|08438|United States|-5|single family| +18668|AAAAAAAAMOIEAAAA|950|Wilson Spruce|Way|Suite Y|Jamestown|Gasconade County|MO|66867|United States|-6|condo| +18669|AAAAAAAANOIEAAAA|175|Chestnut |Lane|Suite 60|Antioch|Washington County|OH|48605|United States|-5|single family| +18670|AAAAAAAAOOIEAAAA|102|Park Johnson|ST|Suite F|Fairview|Yellow Medicine County|MN|55709|United States|-6|apartment| +18671|AAAAAAAAPOIEAAAA|751|First |Boulevard|Suite S|Liberty|Franklin city|VA|23451|United States|-5|condo| +18672|AAAAAAAAAPIEAAAA|||||Valley View|||45124||-5|condo| +18673|AAAAAAAABPIEAAAA|628|Hill |Ln|Suite 230|Liberty|Beadle County|SD|53451|United States|-6|apartment| +18674|AAAAAAAACPIEAAAA|606|Lakeview Elm|Parkway|Suite 470|Harvey|Dundy County|NE|65858|United States|-6|single family| +18675|AAAAAAAADPIEAAAA|336|Third Lake|Wy|Suite X|Greenville|Rusk County|TX|71387|United States|-6|condo| +18676|AAAAAAAAEPIEAAAA||Woodland ||Suite D||Judith Basin County|MT|||-7|apartment| +18677|AAAAAAAAFPIEAAAA|237|Park |Ln|Suite 410|Salem|Otsego County|NY|18048|United States|-5|single family| +18678|AAAAAAAAGPIEAAAA|177|10th Maple|Court|Suite 40|Walnut Grove|Cottle County|TX|77752|United States|-6|apartment| +18679|AAAAAAAAHPIEAAAA|754|Cherry Laurel|Parkway|Suite 480|Forest Hills|Berrien County|GA|39237|United States|-5|apartment| +18680|AAAAAAAAIPIEAAAA|653|2nd 15th|Street|Suite 200|Summit|Santa Rosa County|FL|30499|United States|-5|apartment| +18681|AAAAAAAAJPIEAAAA|379|1st Mill|Lane|Suite K|Greenwood|Geary County|KS|68828|United States|-6|single family| +18682|AAAAAAAAKPIEAAAA|610|Davis |RD|Suite Q|Cedar Grove|Lincoln County|CO|80411|United States|-7|single family| +18683|AAAAAAAALPIEAAAA|829|9th 8th|Blvd|Suite P|Plainview|Bonneville County|ID|83683|United States|-7|single family| +18684|AAAAAAAAMPIEAAAA|30|11th |Dr.|Suite 120|Spring Hill|McLean County|KY|46787|United States|-5|single family| +18685|AAAAAAAANPIEAAAA|669|Washington |Street|Suite Q|Macedonia|Schoolcraft County|MI|41087|United States|-5|condo| +18686|AAAAAAAAOPIEAAAA|661|2nd |Road|Suite Y|Oak Ridge|Franklin County|OH|48371|United States|-5|single family| +18687|AAAAAAAAPPIEAAAA|46|Meadow |Blvd|Suite 350|Macedonia|Coleman County|TX|71087|United States|-6|condo| +18688|AAAAAAAAAAJEAAAA|57|Birch Fourteenth||Suite 280|Glenwood|Decatur County|TN||United States||| +18689|AAAAAAAABAJEAAAA|228|5th |Way|Suite W|Waterloo|Garrett County|MD|21675|United States|-5|apartment| +18690|AAAAAAAACAJEAAAA|435|Park |Dr.|Suite F|Spring Hill|Montgomery County|KY|46787|United States|-5|condo| +18691|AAAAAAAADAJEAAAA|520|West 7th|Dr.|Suite R|Providence|Morgan County|WV|26614|United States|-5|condo| +18692|AAAAAAAAEAJEAAAA|169|Broadway Davis|Ct.|Suite 350|Brownsville|Freeborn County|MN|59310|United States|-6|condo| +18693|AAAAAAAAFAJEAAAA|280|Spring |Drive|Suite S|Walnut Grove|Pocahontas County|WV|27752|United States|-5|condo| +18694|AAAAAAAAGAJEAAAA|807|Jackson 4th|Pkwy|Suite 310|Oakdale|Kleberg County|TX|79584|United States|-6|condo| +18695|AAAAAAAAHAJEAAAA|879|15th |Pkwy|Suite F|Liberty|Franklin County|ID|83451|United States|-7|condo| +18696|AAAAAAAAIAJEAAAA|739|4th Pine|Boulevard|Suite M|Five Forks|Hays County|TX|72293|United States|-6|apartment| +18697|AAAAAAAAJAJEAAAA|581|Center Hill|Ave|Suite 190|Oak Ridge|Jim Wells County|TX|78371|United States|-6|single family| +18698|AAAAAAAAKAJEAAAA|903|Wilson |Ct.|Suite 30|White Oak|Cassia County|ID|86668|United States|-7|single family| +18699|AAAAAAAALAJEAAAA|222|View |Street|Suite 20|Enterprise|Elbert County|GA|31757|United States|-5|single family| +18700|AAAAAAAAMAJEAAAA|506|Sunset Ridge|Dr.|Suite 160|Oakland|Clark County|WA|99843|United States|-8|condo| +18701|AAAAAAAANAJEAAAA|22|Lake Thirteenth|Way|Suite 270|Harmony|Thomas County|KS|65804|United States|-6|condo| +18702|AAAAAAAAOAJEAAAA|5|Jackson |RD|Suite 250|Pleasant Hill|Monroe County|NY|13604|United States|-5|condo| +18703|AAAAAAAAPAJEAAAA|89|Madison Tenth|RD|Suite U|Spring Valley|Screven County|GA|36060|United States|-5|single family| +18704|AAAAAAAAABJEAAAA|221|West |Circle|Suite I|Newtown|Lincoln County|NC|21749|United States|-5|condo| +18705|AAAAAAAABBJEAAAA|398|1st |Ln|Suite T|Riverdale|Prince George County|MD|29391|United States|-5|single family| +18706|AAAAAAAACBJEAAAA|184|Third |Road|Suite H|Kingston|Fentress County|TN|34975|United States|-5|apartment| +18707|AAAAAAAADBJEAAAA|275|River |Boulevard|Suite E|Georgetown|Guadalupe County|NM|87057|United States|-7|apartment| +18708|AAAAAAAAEBJEAAAA|589|Sunset Hill|Street|Suite A|Arlington|Ottawa County|MI|46557|United States|-5|condo| +18709|AAAAAAAAFBJEAAAA|185|Cedar |Lane|Suite O|Riverside|Lander County|NV|89231|United States|-8|condo| +18710|AAAAAAAAGBJEAAAA|673|Johnson Hickory|Lane|Suite B|Woodlawn|Lyon County|KS|64098|United States|-6|condo| +18711|AAAAAAAAHBJEAAAA|953|Eigth 8th|Court|Suite 360|Union|Ouray County|CO|88721|United States|-7|apartment| +18712|AAAAAAAAIBJEAAAA|311|East Spruce|RD|Suite M|Lincoln|Pike County|MS|51289|United States|-6|single family| +18713|AAAAAAAAJBJEAAAA|293|Hillcrest 11th|Wy|Suite B|Springdale|Jones County|NC|28883|United States|-5|single family| +18714|AAAAAAAAKBJEAAAA|954|2nd |ST|Suite F|Liberty|Hopewell city|VA|23451|United States|-5|apartment| +18715|AAAAAAAALBJEAAAA|616|Sixth Pine|Circle|Suite 110|La Grange|Snyder County|PA|17941|United States|-5|single family| +18716|AAAAAAAAMBJEAAAA|436|Smith |ST|Suite C|Spring Valley|Vermilion Parish|LA|76060|United States|-6|single family| +18717|AAAAAAAANBJEAAAA||||Suite 150||Laclede County||64338|United States||| +18718|AAAAAAAAOBJEAAAA|372|Main Cedar|Circle|Suite L|Glendale|Lea County|NM|83951|United States|-7|apartment| +18719|AAAAAAAAPBJEAAAA|963|View |Lane|Suite F|Woodville|Kearney County|NE|64289|United States|-7|apartment| +18720|AAAAAAAAACJEAAAA|456|15th |Blvd|Suite A|Mount Zion|Sherman County|TX|78054|United States|-6|single family| +18721|AAAAAAAABCJEAAAA|518|3rd Oak|Blvd|Suite 360|Springfield|Madison County|TN|39303|United States|-6|single family| +18722|AAAAAAAACCJEAAAA|347|Elm Willow|Blvd|Suite H|Fairfield|Clinton County|MO|66192|United States|-6|apartment| +18723|AAAAAAAADCJEAAAA|371|12th South|Ln|Suite V|Arlington|Columbus County|NC|26557|United States|-5|condo| +18724|AAAAAAAAECJEAAAA|145|Central Pine|ST|Suite N|Appleton|Walworth County|SD|54240|United States|-7|condo| +18725|AAAAAAAAFCJEAAAA|110|Third |Boulevard|Suite 120|Glenwood|Teton County|WY|83511|United States|-7|single family| +18726|AAAAAAAAGCJEAAAA|865|Ridge |Ln|Suite 370|Northwood|Knox County|IL|64104|United States|-6|apartment| +18727|AAAAAAAAHCJEAAAA|229|Oak Jackson|Way|Suite J|Maple Hill|Jasper County|SC|28095|United States|-5|apartment| +18728|AAAAAAAAICJEAAAA|831|Sunset |Ave|Suite S|Hopewell|Franklin County|MA|01187|United States|-5|condo| +18729|AAAAAAAAJCJEAAAA|606|Hill ||Suite E|Green Acres||NE|67683|United States|-6|| +18730|AAAAAAAAKCJEAAAA|250|Locust Lincoln|Dr.|Suite W|Maple Grove|Walla Walla County|WA|98252|United States|-8|single family| +18731|AAAAAAAALCJEAAAA|729|Johnson |ST|Suite 300|Plainview|Henry County|AL|33683|United States|-6|apartment| +18732|AAAAAAAAMCJEAAAA|719|Central Ash|Dr.|Suite 480|Newtown|Crisp County|GA|31749|United States|-5|apartment| +18733|AAAAAAAANCJEAAAA|809|9th |Parkway|Suite 30|Westgate|Hidalgo County|TX|72366|United States|-6|apartment| +18734|AAAAAAAAOCJEAAAA|207|Hickory |Road|Suite 360|Valley View|Portage County|WI|55124|United States|-6|apartment| +18735|AAAAAAAAPCJEAAAA|211|Birch Poplar|Circle|Suite 240|Hillcrest|Davis County|IA|53003|United States|-6|condo| +18736|AAAAAAAAADJEAAAA|461|4th Smith|Pkwy|Suite B|Jamestown|Independence County|AR|76867|United States|-6|apartment| +18737|AAAAAAAABDJEAAAA|63|Cherry Pine|Drive|Suite 220|Oak Hill|Washington County|NC|27838|United States|-5|condo| +18738|AAAAAAAACDJEAAAA|813|5th |Cir.|Suite T|Maple Grove|Dawes County|NE|68252|United States|-6|condo| +18739|AAAAAAAADDJEAAAA|284|Sycamore 13th|RD|Suite K|Langdon|Lincoln County|CO|80852|United States|-7|apartment| +18740|AAAAAAAAEDJEAAAA|573|Meadow Birch|RD|Suite W|Newtown|Dinwiddie County|VA|21749|United States|-5|single family| +18741|AAAAAAAAFDJEAAAA|757|East |ST|Suite 20|Ashland|Calhoun County|MS|54244|United States|-6|apartment| +18742|AAAAAAAAGDJEAAAA|448|Oak 13th|Street|Suite 290|Salem|Hale County|TX|78048|United States|-6|single family| +18743|AAAAAAAAHDJEAAAA|647|Franklin |Parkway|Suite 450|Milo|Franklin County|KY|40116|United States|-6|apartment| +18744|AAAAAAAAIDJEAAAA|764|Church Lincoln|Lane|Suite 350|Mount Pleasant|Marengo County|AL|31933|United States|-6|apartment| +18745|AAAAAAAAJDJEAAAA|998|Spring |Way|Suite 400|Bridgeport|Greene County|MO|65817|United States|-6|apartment| +18746|AAAAAAAAKDJEAAAA|126|Forest Washington|Ct.|Suite L|Centerville|Person County|NC|20059|United States|-5|apartment| +18747|AAAAAAAALDJEAAAA|390|Birch Jefferson|Street|Suite 480|Newport|Union County|NM|81521|United States|-7|condo| +18748|AAAAAAAAMDJEAAAA|36|Davis |RD|Suite 110|Union Hill|King County|WA|97746|United States|-8|condo| +18749|AAAAAAAANDJEAAAA|554|Adams |ST|Suite G|Hillcrest|Wells County|IN|43003|United States|-5|apartment| +18750|AAAAAAAAODJEAAAA|823|Pine Fourth|Court|Suite 310|Woodlawn|Lucas County|OH|44098|United States|-5|apartment| +18751|AAAAAAAAPDJEAAAA|672|Ridge Elm|Avenue|Suite H|Highland Park|El Dorado County|CA|96534|United States|-8|single family| +18752|AAAAAAAAAEJEAAAA|681|Broadway |Ct.|Suite H|Oak Ridge|Laclede County|MO|68371|United States|-6|apartment| +18753|AAAAAAAABEJEAAAA|918|Pine 2nd|RD|Suite 310|Greenfield|Wheeler County|GA|35038|United States|-5|single family| +18754|AAAAAAAACEJEAAAA|682|Center East|ST|Suite F|Cedar Grove|Meigs County|OH|40411|United States|-5|condo| +18755|AAAAAAAADEJEAAAA|757|Center |Avenue|Suite I|Stringtown|Marathon County|WI|50162|United States|-6|condo| +18756|AAAAAAAAEEJEAAAA|985|Maple 7th|Avenue|Suite L|Franklin|Pawnee County|NE|69101|United States|-7|condo| +18757|AAAAAAAAFEJEAAAA|54|View Pine|RD|Suite 350|White Oak|Knox County|MO|66668|United States|-6|condo| +18758|AAAAAAAAGEJEAAAA|317|Jefferson 2nd|Cir.|Suite N|Buena Vista|Kodiak Island Borough|AK|95752|United States|-9|apartment| +18759|AAAAAAAAHEJEAAAA|382|Second |Circle|Suite 30|Lakeview|Clarke County|VA|28579|United States|-5|condo| +18760|AAAAAAAAIEJEAAAA|541|10th |Boulevard|Suite 490|Martinsville|Hamilton County|KS|60419|United States|-6|single family| +18761|AAAAAAAAJEJEAAAA|355|Forest |Lane|Suite 210|Greenwood|Bedford city|VA|28828|United States|-5|condo| +18762|AAAAAAAAKEJEAAAA|365|Center South|Ln|Suite 480|New Hope|Chambers County|AL|39431|United States|-6|condo| +18763|AAAAAAAALEJEAAAA|813|Locust |Street|Suite 330|Canaan|Scott County|KS|67713|United States|-6|condo| +18764|AAAAAAAAMEJEAAAA|793|Wilson |ST|Suite 30|Pleasant Grove|Jasper County|MO|64136|United States|-6|condo| +18765|AAAAAAAANEJEAAAA|966|Oak |Ave|Suite C|Five Points|Spink County|SD|56098|United States|-7|apartment| +18766|AAAAAAAAOEJEAAAA|618|Washington 1st|Way|Suite K|Saint James|Sarasota County|FL|35799|United States|-5|single family| +18767|AAAAAAAAPEJEAAAA|221|Smith Park|Blvd|Suite 490|Crossroads|Nobles County|MN|50534|United States|-6|condo| +18768|AAAAAAAAAFJEAAAA|210|Maple |ST|Suite 120|Hopewell|Bay County|MI|40587|United States|-5|condo| +18769|AAAAAAAABFJEAAAA|771|Jackson 9th|Circle|Suite M|Greenfield|Phillips County|CO|85038|United States|-7|apartment| +18770|AAAAAAAACFJEAAAA|812|Main Lee|Way|Suite S|Pleasant Grove|Motley County|TX|74136|United States|-6|single family| +18771|AAAAAAAADFJEAAAA|605|2nd |Cir.|Suite C|Harmony|Missaukee County|MI|45804|United States|-5|condo| +18772|AAAAAAAAEFJEAAAA|379|Hillcrest |Cir.|Suite Y|Clinton|Columbiana County|OH|48222|United States|-5|condo| +18773|AAAAAAAAFFJEAAAA|688|Park |RD|Suite Y|Greenfield|Powell County|KY|45038|United States|-5|condo| +18774|AAAAAAAAGFJEAAAA|120|10th Mill|Parkway|Suite O|Concord|Orange County|VT|04707|United States|-5|single family| +18775|AAAAAAAAHFJEAAAA|505|Sunset |Street|Suite Q|Plainview|Tuolumne County|CA|93683|United States|-8|condo| +18776|AAAAAAAAIFJEAAAA|809|Hillcrest |Avenue|Suite 230|Pine Grove|Beaufort County|NC|24593|United States|-5|single family| +18777|AAAAAAAAJFJEAAAA|157|Ash Third|Circle|Suite R|Arlington|Oconto County|WI|56557|United States|-6|apartment| +18778|AAAAAAAAKFJEAAAA|102|Williams |Pkwy|Suite B|Unionville|Sharkey County|MS|51711|United States|-6|single family| +18779|AAAAAAAALFJEAAAA|283|Broadway |Way|Suite 430|Shannon|Taliaferro County|GA|34120|United States|-5|condo| +18780|AAAAAAAAMFJEAAAA|383|Wilson |Way|Suite R|Post Oak|Somerset County|PA|18567|United States|-5|single family| +18781|AAAAAAAANFJEAAAA||Miller |||||KY|49431|United States|-6|| +18782|AAAAAAAAOFJEAAAA|354|Walnut South|Ave|Suite W|Fairbanks|Dallas County|MO|66653|United States|-6|apartment| +18783|AAAAAAAAPFJEAAAA|270|Sycamore Elm|Way|Suite Q|Friendship|Madison County|MT|64536|United States|-7|apartment| +18784|AAAAAAAAAGJEAAAA|901|East |Ct.|Suite S|Lincoln|Bear Lake County|ID|81289|United States|-7|condo| +18785|AAAAAAAABGJEAAAA|106|Pine 1st|Ct.|Suite N|Enterprise|Yazoo County|MS|51757|United States|-6|apartment| +18786|AAAAAAAACGJEAAAA|381|West |Boulevard|Suite K|Fairview|Coweta County|GA|35709|United States|-5|single family| +18787|AAAAAAAADGJEAAAA|399|Center |Ct.|Suite 330|Saratoga|Lapeer County|MI|42123|United States|-5|apartment| +18788|AAAAAAAAEGJEAAAA|664|Mill Cherry|Circle|Suite 90|Rutland|Lenoir County|NC|28375|United States|-5|apartment| +18789|AAAAAAAAFGJEAAAA|470|15th |RD|Suite M|Highland|Ashe County|NC|29454|United States|-5|condo| +18790|AAAAAAAAGGJEAAAA|853|9th |Lane|Suite B|Greenfield|Isle of Wight County|VA|25038|United States|-5|condo| +18791|AAAAAAAAHGJEAAAA|773|Maple Miller|Wy|Suite J|Lakeview|Benton County|MS|58579|United States|-6|condo| +18792|AAAAAAAAIGJEAAAA|584|View 6th|Blvd|Suite 40|Walnut Grove|Clay County|FL|37752|United States|-5|apartment| +18793|AAAAAAAAJGJEAAAA|782|4th |Ln|Suite 10|Newport|Ramsey County|MN|51521|United States|-6|single family| +18794|AAAAAAAAKGJEAAAA|915|Williams Park|Dr.|Suite K|Empire|Fayette County|GA|34145|United States|-5|apartment| +18795|AAAAAAAALGJEAAAA|569|13th North|RD|Suite 420|Spring Hill|Nome Census Area|AK|96787|United States|-9|apartment| +18796|AAAAAAAAMGJEAAAA|590|Dogwood |Way|Suite 150|Waterloo|Bryan County|OK|71675|United States|-6|apartment| +18797|AAAAAAAANGJEAAAA|138|Main |Ave|Suite H|Waterloo|Lee County|AR|71675|United States|-6|single family| +18798|AAAAAAAAOGJEAAAA|880|Jackson Fifth|Court|Suite K|Macedonia|Dallas County|AR|71087|United States|-6|apartment| +18799|AAAAAAAAPGJEAAAA|952|Second |Cir.|Suite G|Oakwood|Perry County|IL|60169|United States|-6|single family| +18800|AAAAAAAAAHJEAAAA|32|Elm |Drive|Suite R|Glenwood|Sherburne County|MN|53511|United States|-6|condo| +18801|AAAAAAAABHJEAAAA|675|Walnut |Ave|Suite F|Bethel|Slope County|ND|55281|United States|-6|condo| +18802|AAAAAAAACHJEAAAA|634|Green |Street|Suite 250|Shiloh|Union County|PA|19275|United States|-5|condo| +18803|AAAAAAAADHJEAAAA|262|10th |Ln|Suite T|Kingston|Pacific County|WA|94975|United States|-8|single family| +18804|AAAAAAAAEHJEAAAA|293|2nd Lake|Boulevard|Suite 480|Mountain View|Lewis County|KY|44466|United States|-5|condo| +18805|AAAAAAAAFHJEAAAA|919|Chestnut |Ln|Suite U|Clifton|Greenbrier County|WV|28014|United States|-5|single family| +18806|AAAAAAAAGHJEAAAA|919|Ninth 7th|Ave|Suite J|Glenwood|Dillingham Census Area|AK|93511|United States|-9|single family| +18807|AAAAAAAAHHJEAAAA|610|7th |Parkway|Suite 60|Highland Park|Franklin County|NE|66534|United States|-6|condo| +18808|AAAAAAAAIHJEAAAA|850|Cedar |Ln|Suite 150|Stringtown|Yuba County|CA|90162|United States|-8|apartment| +18809|AAAAAAAAJHJEAAAA|60|North Railroad|Drive|Suite A|Valley View|Codington County|SD|55124|United States|-6|apartment| +18810|AAAAAAAAKHJEAAAA|901|North 4th|Lane|Suite R|Pleasant Grove|Tipton County|TN|34136|United States|-6|apartment| +18811|AAAAAAAALHJEAAAA|25|Highland West|Way|Suite T|Glendale|Aransas County|TX|73951|United States|-6|apartment| +18812|AAAAAAAAMHJEAAAA|213|2nd 7th|ST|Suite V|Mount Zion|Johnson County|KY|48054|United States|-6|condo| +18813|AAAAAAAANHJEAAAA|872|Lee |Road|Suite Q|Lebanon|Wilkin County|MN|52898|United States|-6|apartment| +18814|AAAAAAAAOHJEAAAA|773|Seventh |Way|Suite 440|Pleasant Grove|Randolph County|GA|34136|United States|-5|apartment| +18815|AAAAAAAAPHJEAAAA|214|Chestnut Second|RD|Suite 470|Mount Pleasant|Pierce County|WI|51933|United States|-6|apartment| +18816|AAAAAAAAAIJEAAAA|464|Washington 10th|Wy|Suite L|Amity|Choctaw County|AL|30766|United States|-6|single family| +18817|AAAAAAAABIJEAAAA|192|Second |Street|Suite A|Macedonia|Tunica County|MS|51087|United States|-6|single family| +18818|AAAAAAAACIJEAAAA||||Suite 70||Refugio County|TX|79431|United States||| +18819|AAAAAAAADIJEAAAA|801|West |Pkwy|Suite V|Pleasant Hill|Lawrence County|OH|43604|United States|-5|single family| +18820|AAAAAAAAEIJEAAAA|489|Pine 6th|Circle|Suite O|Bunker Hill|Langlade County|WI|50150|United States|-6|apartment| +18821|AAAAAAAAFIJEAAAA|923|Pine Spring|Blvd|Suite C|Hopewell|Todd County|KY|40587|United States|-5|single family| +18822|AAAAAAAAGIJEAAAA|407|2nd |Ct.|Suite 160|Sunnyside|Mayes County|OK|71952|United States|-6|single family| +18823|AAAAAAAAHIJEAAAA|209|Cedar |Avenue|Suite Q|Stringtown|Breckinridge County|KY|40162|United States|-6|apartment| +18824|AAAAAAAAIIJEAAAA|233|First Oak|Court|Suite F|Woodland|Crittenden County|AR|74854|United States|-6|apartment| +18825|AAAAAAAAJIJEAAAA||Park ||Suite 80|Stringtown|Montgomery County|TX||United States|-6|| +18826|AAAAAAAAKIJEAAAA|105|Laurel Jackson|Ln|Suite 420|Green Acres|Stone County|MO|67683|United States|-6|single family| +18827|AAAAAAAALIJEAAAA|623|Hillcrest Lincoln|Pkwy|Suite V|Paxton|Perry County|MS|55669|United States|-6|condo| +18828|AAAAAAAAMIJEAAAA|615|Adams |Road|Suite 20|Woodland|Cowlitz County|WA|94854|United States|-8|condo| +18829|AAAAAAAANIJEAAAA|333|Locust Main|Street|Suite G|Mount Zion|Clay County|KY|48054|United States|-6|condo| +18830|AAAAAAAAOIJEAAAA|728|14th |Lane|Suite 210|Franklin|Tattnall County|GA|39101|United States|-5|condo| +18831|AAAAAAAAPIJEAAAA|559|3rd Pine|Circle|Suite 220|Greenfield|Iron County|UT|85038|United States|-7|apartment| +18832|AAAAAAAAAJJEAAAA|754|5th Green|Circle|Suite 20|Wyoming|Sheridan County|MT|60216|United States|-7|condo| +18833|AAAAAAAABJJEAAAA|52|Elm |Boulevard|Suite 450|Ashland|Butte County|ID|84244|United States|-7|condo| +18834|AAAAAAAACJJEAAAA|834|Smith Oak|Way|Suite 20|Oakland|Itasca County|MN|59843|United States|-6|apartment| +18835|AAAAAAAADJJEAAAA|185|Ridge Elevnth|Ln|Suite 400|Pine Grove|Quay County|NM|84593|United States|-7|apartment| +18836|AAAAAAAAEJJEAAAA|272|Cedar |Wy|Suite 40|Montezuma|Benson County|ND|52150|United States|-6|condo| +18837|AAAAAAAAFJJEAAAA|161|Maple River|Cir.|Suite H|Oakwood|Columbia County|NY|10169|United States|-5|single family| +18838|AAAAAAAAGJJEAAAA|987|Maple |Ln|Suite S|Clinton|Sumter County|FL|38222|United States|-5|apartment| +18839|AAAAAAAAHJJEAAAA|604|Washington 6th|RD|Suite 150|Sleepy Hollow|Fisher County|TX|73592|United States|-6|condo| +18840|AAAAAAAAIJJEAAAA|689|Church |Dr.|Suite J|Riverside|Steuben County|NY|19231|United States|-5|apartment| +18841|AAAAAAAAJJJEAAAA|399|4th |Dr.|Suite O|Clifton|McPherson County|SD|58014|United States|-7|single family| +18842|AAAAAAAAKJJEAAAA|876|Forest Jackson|Street|Suite 470|Marion|Grand Isle County|VT|00999|United States|-5|condo| +18843|AAAAAAAALJJEAAAA|624|Lincoln |Ct.|Suite 30|Spring Hill|Kodiak Island Borough|AK|96787|United States|-9|condo| +18844|AAAAAAAAMJJEAAAA|849|River |Blvd|Suite 260|Grandview|Jackson County|CO|86475|United States|-7|single family| +18845|AAAAAAAANJJEAAAA|313|Main Oak|Ln|Suite V|Glenwood|Henderson County|IL|63511|United States|-6|apartment| +18846|AAAAAAAAOJJEAAAA|560|Madison |Ln|Suite G|Bunker Hill|Hyde County|NC|20150|United States|-5|single family| +18847|AAAAAAAAPJJEAAAA|302|1st Wilson|Parkway|Suite 410|Shannon|Chautauqua County|NY|14120|United States|-5|single family| +18848|AAAAAAAAAKJEAAAA|784|Park First|Drive|Suite N|Avery|Green County|KY|40194|United States|-6|condo| +18849|AAAAAAAABKJEAAAA|783|Lincoln |Wy|Suite 230|Five Forks|Philadelphia County|PA|12293|United States|-5|single family| +18850|AAAAAAAACKJEAAAA|595|4th View|Parkway|Suite E|Ruth|Sangamon County|IL|60309|United States|-6|apartment| +18851|AAAAAAAADKJEAAAA|83|4th |Court|Suite E|Marion|Allen County|KY|40399|United States|-6|single family| +18852|AAAAAAAAEKJEAAAA|55|Lake |Way|Suite 480|Spring Valley|Douglas County|MN|56060|United States|-6|single family| +18853|AAAAAAAAFKJEAAAA|570|Pine |Blvd|Suite X|Clifton|Saline County|IL|68014|United States|-6|single family| +18854|AAAAAAAAGKJEAAAA|64|Church |Ct.|Suite G|Deerfield|Custer County|ID|89840|United States|-7|single family| +18855|AAAAAAAAHKJEAAAA|335|Hill Park|Avenue|Suite A|Shiloh|Northampton County|VA|29275|United States|-5|apartment| +18856|AAAAAAAAIKJEAAAA|127|Walnut |Boulevard|Suite T|Woodville|Hamilton County|KS|64289|United States|-6|apartment| +18857|AAAAAAAAJKJEAAAA|806|South |Blvd|Suite B|Wildwood|Early County|GA|36871|United States|-5|condo| +18858|AAAAAAAAKKJEAAAA|277|4th |Cir.|Suite 110|Enterprise|Wyandotte County|KS|61757|United States|-6|single family| +18859|AAAAAAAALKJEAAAA|837|Lake Lakeview|Drive|Suite 330|Vienna|Kershaw County|SC|25119|United States|-5|apartment| +18860|AAAAAAAAMKJEAAAA|507|Spring 2nd|Court|Suite K|Springdale|Greene County|OH|48883|United States|-5|condo| +18861|AAAAAAAANKJEAAAA|168|4th |Ave|Suite P|Lakewood|Westmoreland County|VA|28877|United States|-5|condo| +18862|AAAAAAAAOKJEAAAA|210|1st 6th|Avenue|Suite T|Hopewell|Warren County|IN|40587|United States|-5|condo| +18863|AAAAAAAAPKJEAAAA|846|7th |RD|Suite B|Bethel|Latimer County|OK|75281|United States|-6|condo| +18864|AAAAAAAAALJEAAAA|291|Lee |Road|Suite 380|Fairview|Seneca County|OH|45709|United States|-5|single family| +18865|AAAAAAAABLJEAAAA|823|Center Cherry|Ave|Suite 190|Salem|Marion County|IA|58048|United States|-6|single family| +18866|AAAAAAAACLJEAAAA|161|3rd |Cir.|Suite 0|Lakewood|Avoyelles Parish|LA|78877|United States|-6|apartment| +18867|AAAAAAAADLJEAAAA|98|2nd |Ln|Suite 120|Garrison|Ziebach County|SD|58767|United States|-6|single family| +18868|AAAAAAAAELJEAAAA|740|Johnson |Cir.|Suite 450|Kimball|Fannin County|GA|33595|United States|-5|single family| +18869|AAAAAAAAFLJEAAAA|293|South |RD|Suite P|Lebanon|Kent County|MI|42898|United States|-5|apartment| +18870|AAAAAAAAGLJEAAAA|339|Poplar Spruce|Ln|Suite C|Oakland|Camden County|GA|39843|United States|-5|single family| +18871|AAAAAAAAHLJEAAAA|691|6th Ridge|Parkway|Suite R|Mount Olive|Gilchrist County|FL|38059|United States|-5|apartment| +18872|AAAAAAAAILJEAAAA|71|Maple |Parkway|Suite D|Woodlawn|Hamilton County|IN|44098|United States|-5|apartment| +18873|AAAAAAAAJLJEAAAA|507|Second |Ln|Suite 190|Greenville|Kodiak Island Borough|AK|91387|United States|-9|condo| +18874|AAAAAAAAKLJEAAAA|458|Third Railroad|Pkwy|Suite 20|Farmington|Rice County|KS|69145|United States|-6|apartment| +18875|AAAAAAAALLJEAAAA|953|Elevnth Willow|Pkwy|Suite 250|Greenville|Union County|OH|41387|United States|-5|single family| +18876|AAAAAAAAMLJEAAAA|770|Meadow 5th|Wy|Suite Q|Woodville|Pierce County|WI|54289|United States|-6|single family| +18877|AAAAAAAANLJEAAAA|786|Birch |Ct.|Suite C|Greenfield|Le Sueur County|MN|55038|United States|-6|condo| +18878|AAAAAAAAOLJEAAAA|428|Hill Ninth|Parkway|Suite P|Fairview|Charlottesville city|VA|25709|United States|-5|condo| +18879|AAAAAAAAPLJEAAAA|238|2nd |Road|Suite T|Pine Grove|Pierce County|WI|54593|United States|-6|single family| +18880|AAAAAAAAAMJEAAAA|32|6th Franklin|Cir.|Suite K|Lebanon|Manassas city|VA|22898|United States|-5|single family| +18881|AAAAAAAABMJEAAAA|116|North |Dr.|Suite V|Mount Vernon|Morton County|KS|68482|United States|-6|single family| +18882|AAAAAAAACMJEAAAA|680|Ridge West|Ct.|Suite 30|Bunker Hill|Macon County|IL|60150|United States|-6|condo| +18883|AAAAAAAADMJEAAAA|428|Park Ridge|Drive|Suite X|Blanchard|Prentiss County|MS|55985|United States|-6|apartment| +18884|AAAAAAAAEMJEAAAA|125|College |Court|Suite 210|Highland|Nuckolls County|NE|69454|United States|-7|apartment| +18885|AAAAAAAAFMJEAAAA|2|2nd Maple|Court|Suite 50|Franklin|Lagrange County|IN|49101|United States|-5|condo| +18886|AAAAAAAAGMJEAAAA|137|Lincoln Highland|Ln|Suite 370|Shady Grove|Washington County|GA|32812|United States|-5|condo| +18887|AAAAAAAAHMJEAAAA||Valley |Drive|Suite H|Arlington|Washington Parish||||-6|| +18888|AAAAAAAAIMJEAAAA|314|Park Valley|Parkway|Suite F|Greenville|Pottawatomie County|KS|61387|United States|-6|single family| +18889|AAAAAAAAJMJEAAAA|225|Johnson |Parkway|Suite Y|Bunker Hill|Pitt County|NC|20150|United States|-5|condo| +18890|AAAAAAAAKMJEAAAA|390|View |Boulevard|Suite C|Farmington|Henderson County|TX|79145|United States|-6|apartment| +18891|AAAAAAAALMJEAAAA|663|South |Road|Suite E|Pleasant Grove|Baxter County|AR|74136|United States|-6|apartment| +18892|AAAAAAAAMMJEAAAA|344|Dogwood |Avenue|Suite 40|Wyoming|Cheyenne County|CO|80216|United States|-7|apartment| +18893|AAAAAAAANMJEAAAA|262|Ridge |Way|Suite L|Friendship|Yankton County|SD|54536|United States|-6|apartment| +18894|AAAAAAAAOMJEAAAA|549|Third |Wy|Suite 180|Fairbanks|Upson County|GA|36653|United States|-5|apartment| +18895|AAAAAAAAPMJEAAAA|111|Highland Second|Court|Suite 0|Lenox|Goodhue County|MN|51143|United States|-6|condo| +18896|AAAAAAAAANJEAAAA|166||Ln|Suite 470|Clinton|Clarke County|MS|||-6|| +18897|AAAAAAAABNJEAAAA|725|Woodland |Ln|Suite T|Greenwood|Anne Arundel County|MD|28828|United States|-5|apartment| +18898|AAAAAAAACNJEAAAA|542|Spruce Park|RD|Suite X|Union Hill|Butler County|AL|37746|United States|-6|condo| +18899|AAAAAAAADNJEAAAA|861|Center |Dr.|Suite Y|Riverview|Isanti County|MN|59003|United States|-6|single family| +18900|AAAAAAAAENJEAAAA|308|Oak |Court|Suite Q|Cedar Grove|Fallon County|MT|60411|United States|-7|single family| +18901|AAAAAAAAFNJEAAAA|553|7th |Ave|Suite E|Kingston|Tarrant County|TX|74975|United States|-6|apartment| +18902|AAAAAAAAGNJEAAAA|257|Spruce Railroad|Avenue|Suite 190|Highland Park|Mills County|IA|56534|United States|-6|condo| +18903|AAAAAAAAHNJEAAAA|823|11th Seventh|Pkwy|Suite R|Highland Park|Lassen County|CA|96534|United States|-8|single family| +18904|AAAAAAAAINJEAAAA|684|3rd |Blvd|Suite W|Woodland|Young County|TX|74854|United States|-6|single family| +18905|AAAAAAAAJNJEAAAA|789|East |Court|Suite T|Hamilton|Kossuth County|IA|52808|United States|-6|condo| +18906|AAAAAAAAKNJEAAAA|965|North |Street|Suite 160|Georgetown|Alexander County|IL|67057|United States|-6|condo| +18907|AAAAAAAALNJEAAAA|719|River Highland|ST|Suite R|Glendale|Lee County|AL|33951|United States|-6|apartment| +18908|AAAAAAAAMNJEAAAA|103|Church |Wy|Suite S|Highland Park|McKenzie County|ND|56534|United States|-6|condo| +18909|AAAAAAAANNJEAAAA|95|Park |Avenue|Suite 50|Five Points|Lincoln County|OR|96098|United States|-8|condo| +18910|AAAAAAAAONJEAAAA|417|View Williams|Street|Suite 100|Mount Vernon|Multnomah County|OR|98482|United States|-8|apartment| +18911|AAAAAAAAPNJEAAAA|999|Poplar |Pkwy|Suite O|Woodville|Vermilion Parish|LA|74289|United States|-6|condo| +18912|AAAAAAAAAOJEAAAA|11|11th Central|Ct.|Suite L|Woodville|San Miguel County|NM|84289|United States|-7|condo| +18913|AAAAAAAABOJEAAAA|589|Williams Oak|Circle|Suite T|Appleton|Guthrie County|IA|54240|United States|-6|single family| +18914|AAAAAAAACOJEAAAA|914|Cherry |Circle|Suite O|Hopewell|Washington County|RI|01187|United States|-5|condo| +18915|AAAAAAAADOJEAAAA|235|Locust Adams|RD|Suite 390|Edgewood|Marshall County|KS|60069|United States|-6|single family| +18916|AAAAAAAAEOJEAAAA|587|Church Walnut|Ln|Suite V|Lone Oak|Washington County|GA|36893|United States|-5|apartment| +18917|AAAAAAAAFOJEAAAA|942|6th Elm|Way|Suite 430|Post Oak|Grundy County|MO|68567|United States|-6|apartment| +18918|AAAAAAAAGOJEAAAA|389|North Smith|Parkway|Suite 100|Red Hill|Manatee County|FL|34338|United States|-5|apartment| +18919|AAAAAAAAHOJEAAAA|828|1st |Lane|Suite L|Proctor|Baltimore city|MD|28140|United States|-5|condo| +18920|AAAAAAAAIOJEAAAA|735|Mill |Ln|Suite 80|Plainview|Trinity County|TX|73683|United States|-6|apartment| +18921|AAAAAAAAJOJEAAAA|56|South |Street|Suite 70|Springtown|Will County|IL|69858|United States|-6|condo| +18922|AAAAAAAAKOJEAAAA|208|6th Main|Drive|Suite 390|Mountain View|Whiteside County|IL|64466|United States|-6|apartment| +18923|AAAAAAAALOJEAAAA|611|Oak ||Suite G||Brevard County|FL|37838|United States||condo| +18924|AAAAAAAAMOJEAAAA|822|12th |Ave|Suite T|Spring Valley|Columbus County|NC|26060|United States|-5|condo| +18925|AAAAAAAANOJEAAAA|888|View |Court|Suite P|Floyd|Meriwether County|GA|33235|United States|-5|condo| +18926|AAAAAAAAOOJEAAAA|292|Cherry |Road|Suite N|Bridgeport|Covington County|MS|55817|United States|-6|condo| +18927|AAAAAAAAPOJEAAAA|844|6th |Lane|Suite 60|Woodlawn|Harris County|TX|74098|United States|-6|apartment| +18928|AAAAAAAAAPJEAAAA|118|East |Road|Suite 270|Kingston|Bergen County|NJ|05575|United States|-5|single family| +18929|AAAAAAAABPJEAAAA|966|Lakeview |Avenue|Suite 50|Wildwood|Riley County|KS|66871|United States|-6|apartment| +18930|AAAAAAAACPJEAAAA|238|Hill |ST|Suite W|Bayside|Jennings County|IN|49550|United States|-5|condo| +18931|AAAAAAAADPJEAAAA|637|Washington |Way|Suite 260|Riverside|Banks County|GA|39231|United States|-5|single family| +18932|AAAAAAAAEPJEAAAA|163|Fourth River|Street|Suite N|Midway|Monroe County|PA|11904|United States|-5|single family| +18933|AAAAAAAAFPJEAAAA|322|Hill |Boulevard|Suite 450|Valley View|Bethel Census Area|AK|95124|United States|-9|condo| +18934|AAAAAAAAGPJEAAAA|184|5th Mill|Way|Suite N|Macedonia|Caldwell County|KY|41087|United States|-6|single family| +18935|AAAAAAAAHPJEAAAA|503|Valley |Lane|Suite S|Wesley|Jefferson County|KY|41218|United States|-6|apartment| +18936|AAAAAAAAIPJEAAAA|167|Oak |Drive|Suite 50|Lebanon|Dawes County|NE|62898|United States|-6|apartment| +18937|AAAAAAAAJPJEAAAA|969|Sycamore Eigth|Cir.|Suite 150|Enterprise|Troup County|GA|31757|United States|-5|single family| +18938|AAAAAAAAKPJEAAAA|583|5th West|Wy|Suite 230|Woodland|Hamilton County|NE|64854|United States|-6|single family| +18939|AAAAAAAALPJEAAAA|136|Pine |Cir.|Suite G|Kingston|Passaic County|NJ|05575|United States|-5|condo| +18940|AAAAAAAAMPJEAAAA||Maple First|||Hidden Valley|Kalkaska County|MI|45521||-5|apartment| +18941|AAAAAAAANPJEAAAA|432|Eigth Railroad|Boulevard|Suite G|Riverview|Park County|MT|69003|United States|-7|condo| +18942|AAAAAAAAOPJEAAAA|116|Ridge |Dr.|Suite 120|Florence|Wright County|MN|53394|United States|-6|single family| +18943|AAAAAAAAPPJEAAAA|16|Ridge |Ct.|Suite 330|Newport|Divide County|ND|51521|United States|-6|single family| +18944|AAAAAAAAAAKEAAAA|669|Park 4th|Way|Suite 80|Hillcrest|Grenada County|MS|53003|United States|-6|apartment| +18945|AAAAAAAABAKEAAAA|75|Locust 8th|Lane|Suite 440|Friendship|Screven County|GA|34536|United States|-5|apartment| +18946|AAAAAAAACAKEAAAA|683|Wilson |Blvd|Suite D|Wolf Creek|Bulloch County|GA|32455|United States|-5|apartment| +18947|AAAAAAAADAKEAAAA|127|Poplar |Way|Suite N|Sunnyside|Bedford County|TN|31952|United States|-5|apartment| +18948|AAAAAAAAEAKEAAAA|610|7th |Blvd|Suite S|Stringtown|Lewis County|KY|40162|United States|-5|apartment| +18949|AAAAAAAAFAKEAAAA|120|Spruce |Ct.|Suite X|Green Acres|Lincoln County|OR|97683|United States|-8|condo| +18950|AAAAAAAAGAKEAAAA|238|15th |Avenue|Suite 200|Farmington|Niobrara County|WY|89145|United States|-7|condo| +18951|AAAAAAAAHAKEAAAA|344|Jackson North|Drive|Suite 100|Woodville|Fayette County|GA|34289|United States|-5|single family| +18952|AAAAAAAAIAKEAAAA|393|Cherry Washington|Ave|Suite 370|Fairview|Washburn County|WI|55709|United States|-6|single family| +18953|AAAAAAAAJAKEAAAA|906|6th Main|Circle|Suite 390|Midway|Griggs County|ND|51904|United States|-6|single family| +18954|AAAAAAAAKAKEAAAA||Spring |Drive||||||United States||| +18955|AAAAAAAALAKEAAAA|836|Franklin |Street|Suite A|Bunker Hill|Kent County|TX|70150|United States|-6|apartment| +18956|AAAAAAAAMAKEAAAA|618|Mill |Drive|Suite 440|Cedar Grove|Madison County|IN|40411|United States|-5|condo| +18957|AAAAAAAANAKEAAAA|825|Meadow Valley|Avenue|Suite Q|Crossroads|Atkinson County|GA|30534|United States|-5|single family| +18958|AAAAAAAAOAKEAAAA|441|North |ST|Suite 450|Newport|Santa Clara County|CA|91521|United States|-8|single family| +18959|AAAAAAAAPAKEAAAA|87|Davis Lee|Way|Suite 250|Brownsville|Rutherford County|NC|29310|United States|-5|condo| +18960|AAAAAAAAABKEAAAA|514|Main Pine|RD|Suite 370|Brownsville|Hampshire County|MA|09910|United States|-5|condo| +18961|AAAAAAAABBKEAAAA|798|Poplar |RD|Suite E|Valley View|Jim Wells County|TX|75124|United States|-6|apartment| +18962|AAAAAAAACBKEAAAA|146|Spring |Parkway|Suite V|Waterloo|Glacier County|MT|61675|United States|-7|condo| +18963|AAAAAAAADBKEAAAA|24|7th |Cir.|Suite 370|Oakland|Anoka County|MN|59843|United States|-6|single family| +18964|AAAAAAAAEBKEAAAA|374|Hill |Blvd|Suite 210|Lebanon|Genesee County|NY|12898|United States|-5|apartment| +18965|AAAAAAAAFBKEAAAA|521|Willow East|Wy|Suite J|Spring Hill|Wharton County|TX|76787|United States|-6|apartment| +18966|AAAAAAAAGBKEAAAA|836|Second Davis|Pkwy|Suite 220|Red Hill|Refugio County|TX|74338|United States|-6|condo| +18967|AAAAAAAAHBKEAAAA|962|River |Ave|Suite 190|Morgantown|Cape May County|NJ|09793|United States|-5|apartment| +18968|AAAAAAAAIBKEAAAA|78|5th 1st|Ln|Suite 370|Concord|Hardin County|IL|64107|United States|-6|condo| +18969|AAAAAAAAJBKEAAAA|644|Park Church|Parkway|Suite 90|Lakewood|Shackelford County|TX|78877|United States|-6|single family| +18970|AAAAAAAAKBKEAAAA|444|Park Jackson|Boulevard|Suite 150|Kingston|Mitchell County|GA|34975|United States|-5|single family| +18971|AAAAAAAALBKEAAAA|610|View |Avenue|Suite D|Ashland|Iowa County|WI|54244|United States|-6|apartment| +18972|AAAAAAAAMBKEAAAA|685|12th Highland|Cir.|Suite 360|Wildwood|Barron County|WI|56871|United States|-6|condo| +18973|AAAAAAAANBKEAAAA|833|Mill First|Pkwy|Suite B|Friendship|Marion County|KY|44536|United States|-5|single family| +18974|AAAAAAAAOBKEAAAA|823|Center Second|Avenue|Suite 40|Oakdale|Greensville County|VA|29584|United States|-5|condo| +18975|AAAAAAAAPBKEAAAA|807|Sycamore Broadway|Ave|Suite E|Altamont|Marquette County|MI|49387|United States|-5|single family| +18976|AAAAAAAAACKEAAAA|283|Cedar Center|Ln|Suite K|Johnsonville|Jackson County|CO|87745|United States|-7|apartment| +18977|AAAAAAAABCKEAAAA|865|Fifth Pine|Pkwy|Suite I|Riverside|Langlade County|WI|59231|United States|-6|condo| +18978|AAAAAAAACCKEAAAA|240|Wilson Walnut|Court|Suite L|Mount Vernon|Hays County|TX|78482|United States|-6|single family| +18979|AAAAAAAADCKEAAAA|902|3rd |Blvd|Suite 40|Pleasant Grove|Baker County|OR|94136|United States|-8|apartment| +18980|AAAAAAAAECKEAAAA|734|Church Washington|Street|Suite G|Webb|Holmes County|OH|40899|United States|-5|single family| +18981|AAAAAAAAFCKEAAAA|947|College Poplar|Dr.|Suite H|Pleasant Grove|Lunenburg County|VA|24136|United States|-5|single family| +18982|AAAAAAAAGCKEAAAA|570|Lakeview Washington|Boulevard|Suite 30|Salem|Bledsoe County|TN|38048|United States|-5|single family| +18983|AAAAAAAAHCKEAAAA|32|Hill |Ln|Suite V|Bridgeport|Union County|MS|55817|United States|-6|apartment| +18984|AAAAAAAAICKEAAAA|906|Park |Circle|Suite L|Newport|Lake County|OH|41521|United States|-5|apartment| +18985|AAAAAAAAJCKEAAAA|425|Lincoln Johnson|Pkwy|Suite 420|Greenwood|Lexington County|SC|28828|United States|-5|single family| +18986|AAAAAAAAKCKEAAAA|100|1st 6th|Avenue|Suite S|Clinton|Elko County|NV|88222|United States|-8|apartment| +18987|AAAAAAAALCKEAAAA|890|4th |Parkway|Suite 480|Macedonia|Pointe Coupee Parish|LA|71087|United States|-6|condo| +18988|AAAAAAAAMCKEAAAA|613|Broadway |Street|Suite 160|Friendship|Anson County|NC|24536|United States|-5|single family| +18989|AAAAAAAANCKEAAAA|572||Dr.||||LA|||-6|| +18990|AAAAAAAAOCKEAAAA|273|5th Spring|Parkway|Suite A|Riverside|Wayne County|NY|19231|United States|-5|condo| +18991|AAAAAAAAPCKEAAAA|220|Hickory |Ct.|Suite J|Mount Vernon|Missoula County|MT|68482|United States|-7|single family| +18992|AAAAAAAAADKEAAAA|606|10th |RD|Suite C|Oak Hill|Holmes County|FL|37838|United States|-5|condo| +18993|AAAAAAAABDKEAAAA|756|Chestnut Lee|Cir.|Suite T|Oakwood|Marion County|TX|70169|United States|-6|apartment| +18994|AAAAAAAACDKEAAAA|867|Elm |Street|Suite D|Liberty|Cape May County|NJ|04051|United States|-5|condo| +18995|AAAAAAAADDKEAAAA|5|5th Second|Court|Suite 420|Lakeview|Real County|TX|78579|United States|-6|apartment| +18996|AAAAAAAAEDKEAAAA|783|Laurel Park|Road|Suite 20|Sunnyside|Branch County|MI|41952|United States|-5|condo| +18997|AAAAAAAAFDKEAAAA|967|West |Street|Suite G|Somerville|Navajo County|AZ|87783|United States|-7|single family| +18998|AAAAAAAAGDKEAAAA|29|Elm |Ct.|Suite 170|Franklin|Houston County|TN|39101|United States|-5|apartment| +18999|AAAAAAAAHDKEAAAA|464|Highland |Street|Suite 290|Willis|Covington County|MS|56788|United States|-6|single family| +19000|AAAAAAAAIDKEAAAA|183|15th |Avenue|Suite 60|Pleasant Hill|Mecosta County|MI|43604|United States|-5|single family| +19001|AAAAAAAAJDKEAAAA|610|Pine |Court|Suite 170|Oak Grove|Blue Earth County|MN|58370|United States|-6|condo| +19002|AAAAAAAAKDKEAAAA|142|4th |Way|Suite 340|Crossroads|Sullivan County|MO|60534|United States|-6|condo| +19003|AAAAAAAALDKEAAAA|540|Dogwood Center|Ln|Suite 190|Fairfield|Koochiching County|MN|56192|United States|-6|single family| +19004|AAAAAAAAMDKEAAAA|951|Williams |Ct.|Suite O|Lebanon|Washington County|MO|62898|United States|-6|apartment| +19005|AAAAAAAANDKEAAAA|520|Tenth |Cir.|Suite V|Florence|Barton County|KS|63394|United States|-6|condo| +19006|AAAAAAAAODKEAAAA|76|Hillcrest |Parkway|Suite 170|Wolf Creek|Mecosta County|MI|42455|United States|-5|single family| +19007|AAAAAAAAPDKEAAAA|106|Central |ST|Suite J|Highland|Maverick County|TX|79454|United States|-6|apartment| +19008|AAAAAAAAAEKEAAAA|789|Hickory View|Wy||Five Points|Alcorn County|||United States||| +19009|AAAAAAAABEKEAAAA|655|Hickory Church|Ave|Suite 60|Rutland|Chaves County|NM|88375|United States|-7|condo| +19010|AAAAAAAACEKEAAAA||View Hill||Suite 250|Mount Tabor||AL|||-6|apartment| +19011|AAAAAAAADEKEAAAA|20|Park Chestnut|Road|Suite V|Spring Hill|Madison County|OH|46787|United States|-5|single family| +19012|AAAAAAAAEEKEAAAA|947|Washington Johnson|Pkwy|Suite 320|Brownsville|Lynn County|TX|79310|United States|-6|condo| +19013|AAAAAAAAFEKEAAAA|957|Locust Woodland|Cir.|Suite 110|Riverside|Traill County|ND|59231|United States|-6|apartment| +19014|AAAAAAAAGEKEAAAA|805|Main Walnut|Pkwy|Suite M|Wilson|Moffat County|CO|86971|United States|-7|condo| +19015|AAAAAAAAHEKEAAAA|810|Lake Chestnut|Ave|Suite 270|Concord|Del Norte County|CA|94107|United States|-8|apartment| +19016|AAAAAAAAIEKEAAAA|766|11th |Wy|Suite E|White Oak|Ness County|KS|66668|United States|-6|apartment| +19017|AAAAAAAAJEKEAAAA|471|8th 2nd|Road|Suite 120|Salem|Clay County|MN|58048|United States|-6|single family| +19018|AAAAAAAAKEKEAAAA|492|College |Ave|Suite P|Macon|Washakie County|WY|80369|United States|-7|condo| +19019|AAAAAAAALEKEAAAA|25|Cedar Main|Parkway|Suite 30|Centerville|Lackawanna County|PA|10059|United States|-5|apartment| +19020|AAAAAAAAMEKEAAAA|927|Lee |Way|Suite 150|Farmington|Jefferson County|NY|19145|United States|-5|apartment| +19021|AAAAAAAANEKEAAAA|581|Lakeview |Street|Suite 190|Enterprise|Adams County|PA|11757|United States|-5|condo| +19022|AAAAAAAAOEKEAAAA|974|Maple Park|Street|Suite 330|Macedonia|Sullivan County|NH|01687|United States|-5|single family| +19023|AAAAAAAAPEKEAAAA|867|12th |Court|Suite H|Bridgeport|Fairbanks North Star Borough|AK|95817|United States|-9|condo| +19024|AAAAAAAAAFKEAAAA||Oak |||||VA|28041|United States|-5|| +19025|AAAAAAAABFKEAAAA|919|West |Street|Suite M|Lakeside|Hale County|AL|39532|United States|-6|single family| +19026|AAAAAAAACFKEAAAA|375|Broadway Fifteenth|Drive|Suite L|Thompsonville|Gray County|TX|79651|United States|-6|apartment| +19027|AAAAAAAADFKEAAAA|220|10th Park|Lane|Suite R|Antioch|Queen Anne County|MD|28605|United States|-5|condo| +19028|AAAAAAAAEFKEAAAA||Walnut Hillcrest|Boulevard|Suite 430|Mountain View|Aurora County|SD|54466|||apartment| +19029|AAAAAAAAFFKEAAAA|66|13th |ST|Suite 180|Mount Vernon|Carroll County|MD|28482|United States|-5|condo| +19030|AAAAAAAAGFKEAAAA|593|Central |Pkwy|Suite I|Liberty|Breckinridge County|KY|43451|United States|-6|single family| +19031|AAAAAAAAHFKEAAAA|862|Park Elm|Blvd|Suite T|Newport|Clark County|OH|41521|United States|-5|condo| +19032|AAAAAAAAIFKEAAAA|770|Valley |Ln|Suite B|Glendale|Clifton Forge city|VA|23951|United States|-5|apartment| +19033|AAAAAAAAJFKEAAAA|631|Chestnut |Wy|Suite 130|Clifton|Zapata County|TX|78014|United States|-6|condo| +19034|AAAAAAAAKFKEAAAA|899|Ash |Ln|Suite W|Oakland|Monroe County|IA|59843|United States|-6|apartment| +19035|AAAAAAAALFKEAAAA|416|Railroad |Parkway|Suite A|Highland|Cameron Parish|LA|79454|United States|-6|apartment| +19036|AAAAAAAAMFKEAAAA|310|Birch 2nd|Boulevard|Suite W|Lakeside|Rapides Parish|LA|79532|United States|-6|single family| +19037|AAAAAAAANFKEAAAA|417|Williams |Way|Suite Q|Jamestown|Geauga County|OH|46867|United States|-5|single family| +19038|AAAAAAAAOFKEAAAA|926|Adams |Court|Suite R|Salem|Florence County|SC|28048|United States|-5|single family| +19039|AAAAAAAAPFKEAAAA|831|Lake Jefferson|Circle|Suite X|Valley View|Toole County|MT|65124|United States|-7|condo| +19040|AAAAAAAAAGKEAAAA|||||||WV|24120|||apartment| +19041|AAAAAAAABGKEAAAA|511|Green Franklin|Circle|Suite V|Macedonia|Kalkaska County|MI|41087|United States|-5|condo| +19042|AAAAAAAACGKEAAAA|715|3rd |Dr.|Suite 430|Stringtown|Posey County|IN|40162|United States|-5|condo| +19043|AAAAAAAADGKEAAAA|||Ct.|Suite E||||43951|United States||| +19044|AAAAAAAAEGKEAAAA|626|Walnut 2nd|Way|Suite U|Sulphur Springs|Kenton County|KY|48354|United States|-5|apartment| +19045|AAAAAAAAFGKEAAAA|865|9th |Avenue|Suite 10|Lakeside|Cullman County|AL|39532|United States|-6|single family| +19046|AAAAAAAAGGKEAAAA|93|Adams Lakeview|Blvd|Suite D|New Hope|Hancock County|IA|59431|United States|-6|apartment| +19047|AAAAAAAAHGKEAAAA|102|Third Smith|ST|Suite S|Glenwood|Chambers County|TX|73511|United States|-6|apartment| +19048|AAAAAAAAIGKEAAAA|342|5th Lake|Ave|Suite R|Silver City|Grayson County|VA|23405|United States|-5|single family| +19049|AAAAAAAAJGKEAAAA|704|Second |Avenue|Suite 250|Cumberland|Johnson County|IA|58971|United States|-6|single family| +19050|AAAAAAAAKGKEAAAA|975|Main Oak|Ln|Suite 290|Spring Valley|Adair County|KY|46060|United States|-6|apartment| +19051|AAAAAAAALGKEAAAA|175|East |||Centerville|Clarke County||||-6|| +19052|AAAAAAAAMGKEAAAA|958|Oak Third|Blvd|Suite O|Oak Hill|Cleburne County|AL|37838|United States|-6|condo| +19053|AAAAAAAANGKEAAAA|606|East 3rd|Lane|Suite 390|Glenwood|Monroe County|FL|33511|United States|-5|single family| +19054|AAAAAAAAOGKEAAAA|75|15th View|Road|Suite 240|Marion|Union County|SD|50399|United States|-7|apartment| +19055|AAAAAAAAPGKEAAAA|906|Lincoln |Blvd|Suite 280|Pine Grove|McDuffie County|GA|34593|United States|-5|single family| +19056|AAAAAAAAAHKEAAAA|728|Adams Wilson|Street|Suite 420|Lakeview|Cameron County|PA|18579|United States|-5|apartment| +19057|AAAAAAAABHKEAAAA|126|Jackson 2nd|Ln|Suite 450|Winchester|Pottawatomie County|OK|73252|United States|-6|single family| +19058|AAAAAAAACHKEAAAA|869|Adams |RD|Suite 300|Highland|Power County|ID|89454|United States|-7|apartment| +19059|AAAAAAAADHKEAAAA|28|Green 2nd|Ct.|Suite 330|Farmington|Forest County|PA|19145|United States|-5|apartment| +19060|AAAAAAAAEHKEAAAA|375|Johnson |Ln|Suite 270|Walnut Grove|Cleburne County|AR|77752|United States|-6|apartment| +19061|AAAAAAAAFHKEAAAA|637|Ridge Jackson|Way|Suite 100|Mount Pleasant|George County|MS|51933|United States|-6|single family| +19062|AAAAAAAAGHKEAAAA|779|Fourth Third|Ct.|Suite A|Bethel|Jefferson County|MT|65281|United States|-7|condo| +19063|AAAAAAAAHHKEAAAA|835|Willow |Boulevard|Suite L|Bethel|Uinta County|WY|85281|United States|-7|single family| +19064|AAAAAAAAIHKEAAAA|954|Second |Avenue|Suite 230|Springfield|Marshall County|MN|59303|United States|-6|condo| +19065|AAAAAAAAJHKEAAAA|967|North |Street||California|Cumberland County|ME||United States||apartment| +19066|AAAAAAAAKHKEAAAA|829|7th |ST|Suite 50|Woodlawn|Baker County|FL|34098|United States|-5|condo| +19067|AAAAAAAALHKEAAAA|919|Lake South|Wy|Suite 200|Fairview|Polk County|NC|25709|United States|-5|apartment| +19068|AAAAAAAAMHKEAAAA|623|Park Lake|Avenue|Suite 410|Woodville|Amador County|CA|94289|United States|-8|apartment| +19069|AAAAAAAANHKEAAAA|375|Jackson Chestnut|Ln|Suite 200|Nichols|Kalamazoo County|MI|47940|United States|-5|single family| +19070|AAAAAAAAOHKEAAAA|194||Dr.|Suite 370|Sunnyside|Cherokee County||||-5|| +19071|AAAAAAAAPHKEAAAA|312|Sunset |Boulevard|Suite 30|Springdale|Hidalgo County|TX|78883|United States|-6|apartment| +19072|AAAAAAAAAIKEAAAA|887|Davis |Boulevard|Suite T|Stringtown|Yoakum County|TX|70162|United States|-6|apartment| +19073|AAAAAAAABIKEAAAA|150|Central Meadow|Wy|Suite U|Summit|Ouachita County|AR|70499|United States|-6|single family| +19074|AAAAAAAACIKEAAAA|756|Second |Avenue|Suite B|Deerfield|Carroll County|IA|59840|United States|-6|condo| +19075|AAAAAAAADIKEAAAA|438|Dogwood |Lane|Suite R|Centerville|Goodhue County|MN|50059|United States|-6|single family| +19076|AAAAAAAAEIKEAAAA|388|2nd Cedar|Avenue|Suite C|Macedonia|Meade County|KY|41087|United States|-5|apartment| +19077|AAAAAAAAFIKEAAAA|880|Adams Willow|||||ID|83683|United States|-7|apartment| +19078|AAAAAAAAGIKEAAAA|823|Maple Fourth|ST|Suite T|Spring Hill|Genesee County|NY|16787|United States|-5|apartment| +19079|AAAAAAAAHIKEAAAA|669|Seventh Valley|RD|Suite 50|Oakland|Anderson County|TX|79843|United States|-6|single family| +19080|AAAAAAAAIIKEAAAA|240|Lincoln |RD|Suite 400|Plainview|Warren County|IN|43683|United States|-5|single family| +19081|AAAAAAAAJIKEAAAA|834|Pine |RD|Suite 160|New Hope|Florence County|WI|59431|United States|-6|condo| +19082|AAAAAAAAKIKEAAAA|13|Cherry |Ct.|Suite V|Florence|Fredericksburg city|VA|23394|United States|-5|condo| +19083|AAAAAAAALIKEAAAA|451|||Suite M|New Hope||||||| +19084|AAAAAAAAMIKEAAAA|417|Third |Blvd|Suite F|Belmont|DeWitt County|TX|70191|United States|-6|single family| +19085|AAAAAAAANIKEAAAA|543|Johnson Jefferson|Road|Suite L|Wilson|Napa County|CA|96971|United States|-8|single family| +19086|AAAAAAAAOIKEAAAA|125|Spring 14th|Blvd|Suite C|Walnut Grove|Phillips County|KS|67752|United States|-6|apartment| +19087|AAAAAAAAPIKEAAAA|512|Twelfth |Street|Suite H|Fairview|Manistee County|MI|45709|United States|-5|apartment| +19088|AAAAAAAAAJKEAAAA|799|Sixth |Ct.|Suite N|Sunnyside|Lake County|MN|51952|United States|-6|condo| +19089|AAAAAAAABJKEAAAA|891|West Central|RD|Suite 240|Oak Grove|Washington County|FL|38370|United States|-5|apartment| +19090|AAAAAAAACJKEAAAA|306|College Woodland|RD|Suite Q|Buena Vista|Buffalo County|WI|55752|United States|-6|single family| +19091|AAAAAAAADJKEAAAA|308|Hill |Ave|Suite O|Five Points|Wayne County|NY|16098|United States|-5|condo| +19092|AAAAAAAAEJKEAAAA|776|Dogwood Ridge|RD|Suite 240|Mount Vernon|Louisa County|VA|28482|United States|-5|single family| +19093|AAAAAAAAFJKEAAAA|195|Elm Church|Wy|Suite 100|Hillcrest|Imperial County|CA|93003|United States|-8|apartment| +19094|AAAAAAAAGJKEAAAA|756|2nd 13th|Dr.|Suite 130|Pleasant Hill|Pike County|MS|53604|United States|-6|single family| +19095|AAAAAAAAHJKEAAAA|630|Cherry |Avenue|Suite F|Five Forks|Pulaski County|IN|42293|United States|-5|apartment| +19096|AAAAAAAAIJKEAAAA|635|Lincoln |Way|Suite 300|Siloam|Newton County|MO|68948|United States|-6|condo| +19097|AAAAAAAAJJKEAAAA|146|5th |Pkwy|Suite U|Willow|Stone County|MO|66798|United States|-6|single family| +19098|AAAAAAAAKJKEAAAA|609|Miller College|Court|Suite N|Mountain View|Martin County|KY|44466|United States|-5|apartment| +19099|AAAAAAAALJKEAAAA|463|Ash Spruce|Lane|Suite J|Hamilton|Pickaway County|OH|42808|United States|-5|single family| +19100|AAAAAAAAMJKEAAAA|832|View |Parkway|Suite X|Williamsville|Ziebach County|SD|58754|United States|-6|single family| +19101|AAAAAAAANJKEAAAA|297|Spruce Meadow|Parkway|Suite 290|Concord|Knox County|IL|64107|United States|-6|single family| +19102|AAAAAAAAOJKEAAAA|903|Main Spring|Boulevard|Suite 310|Ashland|Sacramento County|CA|94244|United States|-8|apartment| +19103|AAAAAAAAPJKEAAAA|777|Third Main|Court|Suite S|Mount Zion|Hempstead County|AR|78054|United States|-6|apartment| +19104|AAAAAAAAAKKEAAAA|732|Fourteenth |ST|Suite 60|Hopewell|Zavala County|TX|70587|United States|-6|single family| +19105|AAAAAAAABKKEAAAA|174|East |Ave|Suite 340|Highland|Goodhue County|MN|59454|United States|-6|condo| +19106|AAAAAAAACKKEAAAA|418|Davis Lee||Suite C|Springdale|||58883||-6|| +19107|AAAAAAAADKKEAAAA|930|Valley North|Boulevard|Suite 280|Five Points|Pointe Coupee Parish|LA|76098|United States|-6|apartment| +19108|AAAAAAAAEKKEAAAA|221|Lake |Road|Suite D|Riverdale|Jim Hogg County|TX|79391|United States|-6|single family| +19109|AAAAAAAAFKKEAAAA|880|Laurel College|Ct.|Suite I|Colfax|San Miguel County|CO|82565|United States|-7|single family| +19110|AAAAAAAAGKKEAAAA|960|5th Maple|Lane|Suite A|Shelby|Leon County|FL|36575|United States|-5|apartment| +19111|AAAAAAAAHKKEAAAA|668|Sycamore |Pkwy|Suite R|Riverview|Salem County|NJ|09603|United States|-5|single family| +19112|AAAAAAAAIKKEAAAA|485|Jefferson Mill|Court|Suite 450|Riverview|De Kalb County|IN|49003|United States|-5|apartment| +19113|AAAAAAAAJKKEAAAA|470|Cedar |Boulevard|Suite J|Shady Grove|Jefferson Parish|LA|72812|United States|-6|apartment| +19114|AAAAAAAAKKKEAAAA|210|Cedar |Avenue|Suite G|Pine Grove|Logan County|CO|84593|United States|-7|single family| +19115|AAAAAAAALKKEAAAA||West Miller|||Clifton||OH||||| +19116|AAAAAAAAMKKEAAAA||||Suite 410|||KS||United States|-6|| +19117|AAAAAAAANKKEAAAA|161|15th Second|Circle|Suite Y|Peru|Houston County|GA|30302|United States|-5|apartment| +19118|AAAAAAAAOKKEAAAA|282|4th |Cir.|Suite W|Clinton|Madison County|IA|58222|United States|-6|apartment| +19119|AAAAAAAAPKKEAAAA|380|Church |Pkwy|Suite F|Antioch|Coshocton County|OH|48605|United States|-5|apartment| +19120|AAAAAAAAALKEAAAA|415|Madison |Ave|Suite 110|Woodland|Parmer County|TX|74854|United States|-6|single family| +19121|AAAAAAAABLKEAAAA|524|Ridge Laurel|Ave|Suite D|Pleasant Valley|Chowan County|NC|22477|United States|-5|single family| +19122|AAAAAAAACLKEAAAA|239|Mill |Street|Suite 150|Woodville|Payne County|OK|74289|United States|-6|condo| +19123|AAAAAAAADLKEAAAA|721|Fourth |Blvd|Suite G|Gladstone|Mercer County|WV|20894|United States|-5|condo| +19124|AAAAAAAAELKEAAAA|342|Cedar Maple|Court|Suite E|Lakeview|Howard County|NE|68579|United States|-7|apartment| +19125|AAAAAAAAFLKEAAAA|284|View |Street|Suite K|Lakeside|Kimble County|TX|79532|United States|-6|apartment| +19126|AAAAAAAAGLKEAAAA|164|Franklin |Court|Suite 170|Pleasant Hill|Brown County|IL|63604|United States|-6|apartment| +19127|AAAAAAAAHLKEAAAA|664|Mill 2nd|ST|Suite 430|Greenwood|Teton County|MT|68828|United States|-7|condo| +19128|AAAAAAAAILKEAAAA|9|River Cedar|Drive|Suite C|Bryant|Martin County|KY|46984|United States|-5|apartment| +19129|AAAAAAAAJLKEAAAA|131|11th |Blvd|Suite V|Kingston|Wayne County|TN|34975|United States|-6|apartment| +19130|AAAAAAAAKLKEAAAA|12|Center 7th|Ct.|Suite F|Franklin|Lake County|OR|99101|United States|-8|single family| +19131|AAAAAAAALLKEAAAA|908|Elm Wilson|Dr.|Suite Q|Providence|Ripley County|IN|46614|United States|-5|apartment| +19132|AAAAAAAAMLKEAAAA|994|Forest Hickory|Court|Suite 110|Green Acres|New York County|NY|17683|United States|-5|single family| +19133|AAAAAAAANLKEAAAA|122|2nd 2nd|ST|Suite E|Providence|Windham County|VT|07214|United States|-5|single family| +19134|AAAAAAAAOLKEAAAA|511|6th |Blvd|Suite W|Oak Grove|Osage County|OK|78370|United States|-6|apartment| +19135|AAAAAAAAPLKEAAAA|783|Ash |Ln|Suite Q|Sunnyside|King County|TX|71952|United States|-6|condo| +19136|AAAAAAAAAMKEAAAA|170|Lake Spruce|Court|Suite K|Kingston|Riley County|KS|64975|United States|-6|apartment| +19137|AAAAAAAABMKEAAAA|209|Williams |RD|Suite B|Kingston|Valley County|MT|64975|United States|-7|single family| +19138|AAAAAAAACMKEAAAA|897||||Springdale|Daviess County||68883||-6|| +19139|AAAAAAAADMKEAAAA|653|Sycamore Adams|Avenue|Suite S|White Plains|Guadalupe County|NM|86622|United States|-7|apartment| +19140|AAAAAAAAEMKEAAAA|945|Lakeview Washington|Cir.|Suite P|Salem|Lancaster County|SC|28048|United States|-5|single family| +19141|AAAAAAAAFMKEAAAA|766|Broadway Railroad|Boulevard|Suite D|Unionville|New York County|NY|11711|United States|-5|condo| +19142|AAAAAAAAGMKEAAAA|339|Locust Fourth|Cir.|Suite B|Morgantown|Arlington County|VA|29193|United States|-5|apartment| +19143|AAAAAAAAHMKEAAAA|168|Locust |Ct.|Suite G|Macedonia|Atascosa County|TX|71087|United States|-6|single family| +19144|AAAAAAAAIMKEAAAA|123|Williams |Parkway|Suite J|Riverside|Newton County|AR|79231|United States|-6|condo| +19145|AAAAAAAAJMKEAAAA|40|2nd Miller|Road|Suite 190|Arlington|Morgan County|GA|36557|United States|-5|apartment| +19146|AAAAAAAAKMKEAAAA|960|Smith |Pkwy|Suite 480|Greenfield|Warren County|VA|25038|United States|-5|condo| +19147|AAAAAAAALMKEAAAA|882|Mill |Dr.|Suite 150|Riverdale|Rice County|MN|59391|United States|-6|single family| +19148|AAAAAAAAMMKEAAAA|747|Sunset |Parkway|Suite 30|Pleasant Valley|Marlboro County|SC|22477|United States|-5|condo| +19149|AAAAAAAANMKEAAAA|95|Locust |Avenue|Suite 470|Union Hill|Crane County|TX|77746|United States|-6|condo| +19150|AAAAAAAAOMKEAAAA|||Way||Mount Vernon|Payne County||78482|||| +19151|AAAAAAAAPMKEAAAA|264|Elm Seventh|Way|Suite 360|Lakeview|Treasure County|MT|68579|United States|-7|single family| +19152|AAAAAAAAANKEAAAA|773|Washington |Parkway|Suite 240|Greenville|Harrison County|IN|41387|United States|-5|condo| +19153|AAAAAAAABNKEAAAA|581|9th |Road|Suite C|Union Hill|Muskingum County|OH|47746|United States|-5|condo| +19154|AAAAAAAACNKEAAAA|667|Third Third|Lane|Suite 350|Pleasant Grove|Benton County|IN|44136|United States|-5|condo| +19155|AAAAAAAADNKEAAAA|803|5th |Road|Suite S|Marion|Bell County|KY|40399|United States|-6|condo| +19156|AAAAAAAAENKEAAAA|168|Main Oak|Parkway|Suite R|Mount Olive|Florence County|WI|58059|United States|-6|apartment| +19157|AAAAAAAAFNKEAAAA|78|10th 6th|Court|Suite F|Glenwood|Story County|IA|53511|United States|-6|apartment| +19158|AAAAAAAAGNKEAAAA|870|1st |Cir.|Suite 90|Riverdale|Quay County|NM|89391|United States|-7|single family| +19159|AAAAAAAAHNKEAAAA|710|Tenth |Court|Suite A|Hamilton|Lake and Peninsula Borough|AK|92808|United States|-9|apartment| +19160|AAAAAAAAINKEAAAA|168|12th |Road|Suite L|Galena|Mifflin County|PA|14369|United States|-5|single family| +19161|AAAAAAAAJNKEAAAA|44|8th |Boulevard|Suite A|Florence|Glades County|FL|33394|United States|-5|condo| +19162|AAAAAAAAKNKEAAAA|925|Broadway Hill|RD|Suite 210|Wilson|Washoe County|NV|86971|United States|-8|single family| +19163|AAAAAAAALNKEAAAA|799|Center Ninth|Wy|Suite 230|Oakland|Adams County|WI|59843|United States|-6|condo| +19164|AAAAAAAAMNKEAAAA|785|9th Lincoln|Dr.|Suite 240|Frenchtown|Charles County|MD|22629|United States|-5|condo| +19165|AAAAAAAANNKEAAAA|101|Main |Ct.|Suite 100|Oakland|Lowndes County|MS|59843|United States|-6|apartment| +19166|AAAAAAAAONKEAAAA|6|Ridge Green|RD|Suite F|Spring Valley|Shelby County|IA|56060|United States|-6|single family| +19167|AAAAAAAAPNKEAAAA|320|14th |Drive|Suite 320|Harmony|McNairy County|TN|35804|United States|-6|condo| +19168|AAAAAAAAAOKEAAAA|301|Johnson |Boulevard|Suite M|Forestville|Nueces County|TX|73027|United States|-6|apartment| +19169|AAAAAAAABOKEAAAA|731|Walnut |Drive|Suite 390|Ridgeville|Jackson County|MI|49306|United States|-5|condo| +19170|AAAAAAAACOKEAAAA|127|Woodland |Road|Suite W|Woodville|Cloud County|KS|64289|United States|-6|condo| +19171|AAAAAAAADOKEAAAA|671|3rd Hill|Dr.|Suite 260|Fairview|Douglas County|NV|85709|United States|-8|apartment| +19172|AAAAAAAAEOKEAAAA|118|13th |Boulevard|Suite 60|Concord|San Benito County|CA|94107|United States|-8|apartment| +19173|AAAAAAAAFOKEAAAA|994|Park Maple|Drive|Suite I|Lincoln|Polk County|AR|71289|United States|-6|apartment| +19174|AAAAAAAAGOKEAAAA|244|Broadway Birch|Cir.|Suite T|Pleasant Hill|Trigg County|KY|43604|United States|-5|single family| +19175|AAAAAAAAHOKEAAAA|833|Sunset |Cir.|Suite L|Clinton|Broome County|NY|18222|United States|-5|single family| +19176|AAAAAAAAIOKEAAAA|753|9th |Ln|Suite W||Worcester County||26938|United States||| +19177|AAAAAAAAJOKEAAAA|351|River |Wy|Suite 280|Walnut Grove|Waller County|TX|77752|United States|-6|condo| +19178|AAAAAAAAKOKEAAAA|922|Hill 10th|Circle|Suite 340|Georgetown|Gage County|NE|67057|United States|-6|single family| +19179|AAAAAAAALOKEAAAA|919|Hickory 6th|Avenue|Suite 420|Belmont|Henry County|AL|30191|United States|-6|apartment| +19180|AAAAAAAAMOKEAAAA|||Parkway|Suite D|Deerfield|Hutchinson County|SD|59840||-7|| +19181|AAAAAAAANOKEAAAA|931|Main River|RD|Suite 0|Shiloh|Bladen County|NC|29275|United States|-5|apartment| +19182|AAAAAAAAOOKEAAAA|140|Main Main|Wy|Suite 140|Ellisville|Dickson County|TN|36820|United States|-5|apartment| +19183|AAAAAAAAPOKEAAAA|63|East Smith|Parkway|Suite 210|Sulphur Springs|Carroll County|IN|48354|United States|-5|apartment| +19184|AAAAAAAAAPKEAAAA|95|Poplar Railroad|Way|Suite 110|Midway|Clinton County|NY|11904|United States|-5|condo| +19185|AAAAAAAABPKEAAAA|287|Hickory Madison|Dr.|Suite O|Liberty|Kent County|DE|13451|United States|-5|condo| +19186|AAAAAAAACPKEAAAA|376|Park |Court|Suite E|Mount Pleasant|Marengo County|AL|31933|United States|-6|single family| +19187|AAAAAAAADPKEAAAA|436|3rd |Drive|Suite 410|King|Dawson County|MT|60008|United States|-7|condo| +19188|AAAAAAAAEPKEAAAA|637|East |Drive|Suite H|Franklin|Mountrail County|ND|59101|United States|-6|apartment| +19189|AAAAAAAAFPKEAAAA|375|Central Railroad|Road|Suite 110|Lakeview|Contra Costa County|CA|98579|United States|-8|apartment| +19190|AAAAAAAAGPKEAAAA|214|Johnson |Ct.|Suite Q|Enterprise|Addison County|VT|02357|United States|-5|condo| +19191|AAAAAAAAHPKEAAAA|322|6th River|Lane|Suite 170|Bunker Hill|Floyd County|KY|40150|United States|-6|apartment| +19192|AAAAAAAAIPKEAAAA|950|Mill Walnut|Ct.|Suite 270|Union|Camden County|GA|38721|United States|-5|apartment| +19193|AAAAAAAAJPKEAAAA|930|||Suite G|Plainview||ME|04283||-5|condo| +19194|AAAAAAAAKPKEAAAA|78|Williams Franklin|Drive|Suite G|Edgewood|Weston County|WY|80069|United States|-7|single family| +19195|AAAAAAAALPKEAAAA||Pine ||Suite 290|||MT||United States|-7|condo| +19196|AAAAAAAAMPKEAAAA|650|Washington |Drive|Suite 210|Highland Park|Amite County|MS|56534|United States|-6|condo| +19197|AAAAAAAANPKEAAAA|211|South Cherry|Court|Suite J|Green Acres|Collin County|TX|77683|United States|-6|single family| +19198|AAAAAAAAOPKEAAAA|174|Ridge 12th|Drive|Suite 60|Franklin|Dillon County|SC|29101|United States|-5|single family| +19199|AAAAAAAAPPKEAAAA|928|Jefferson Main|Street|Suite Y|Woodland|Menifee County|KY|44854|United States|-5|apartment| +19200|AAAAAAAAAALEAAAA|730|Willow |Ave|Suite K|Bridgeport|Weakley County|TN|35817|United States|-6|single family| +19201|AAAAAAAABALEAAAA|814|Hill |Way|Suite I|Walnut Grove|Butler County|AL|37752|United States|-6|condo| +19202|AAAAAAAACALEAAAA|496|Ash Center|Dr.|Suite 50|Greenfield|Atoka County|OK|75038|United States|-6|condo| +19203|AAAAAAAADALEAAAA|925|5th Birch|ST|Suite 280|Bunker Hill|Rush County|KS|60150|United States|-6|single family| +19204|AAAAAAAAEALEAAAA|897|Chestnut |Ln|Suite 360|Greenwood|Union County|MS|58828|United States|-6|condo| +19205|AAAAAAAAFALEAAAA|501|First East|Parkway|Suite 260|Crossroads|Fairfax city|VA|20534|United States|-5|apartment| +19206|AAAAAAAAGALEAAAA|349|Davis |Boulevard|Suite 470|Sulphur Springs|Wabaunsee County|KS|68354|United States|-6|condo| +19207|AAAAAAAAHALEAAAA|940|Railroad Oak|Way|Suite 190|Liberty|Sullivan County|NY|13451|United States|-5|condo| +19208|AAAAAAAAIALEAAAA|221|Green Davis|Wy|Suite P|Newport|Giles County|TN|31521|United States|-5|apartment| +19209|AAAAAAAAJALEAAAA|193|College 15th|Parkway|Suite 480|Five Points|Wilson County|NC|26098|United States|-5|apartment| +19210|AAAAAAAAKALEAAAA|27|Chestnut Second|ST|Suite S|Winslow|Lycoming County|PA|18525|United States|-5|apartment| +19211|AAAAAAAALALEAAAA|122|2nd Chestnut|Drive|Suite 450|Greenville|Fentress County|TN|31387|United States|-5|single family| +19212|AAAAAAAAMALEAAAA|847|1st Davis|Blvd|Suite H|New Town|Somervell County|TX|79634|United States|-6|condo| +19213|AAAAAAAANALEAAAA|769|Lee North|Blvd|Suite G|Ashland|Chesterfield County|VA|24244|United States|-5|apartment| +19214|AAAAAAAAOALEAAAA|632|Lake |Circle|Suite 210|Lakeview|Colusa County|CA|98579|United States|-8|apartment| +19215|AAAAAAAAPALEAAAA|621|Hill |Ave|Suite C|Maple Grove|Sevier County|TN|38252|United States|-6|apartment| +19216|AAAAAAAAABLEAAAA|612|4th South|ST|Suite C|Maxwell|Callaway County|MO|64925|United States|-6|apartment| +19217|AAAAAAAABBLEAAAA|801|15th |Boulevard|Suite 470|Stringtown|Pasco County|FL|30162|United States|-5|single family| +19218|AAAAAAAACBLEAAAA|756|Meadow 10th|Cir.|Suite 270|Macedonia|Webster Parish|LA|71087|United States|-6|condo| +19219|AAAAAAAADBLEAAAA|137|8th Pine|Way|Suite 340|Mount Olive|Lapeer County|MI|48059|United States|-5|single family| +19220|AAAAAAAAEBLEAAAA|618|West |Blvd|Suite 290|Jackson|Sevier County|TN|39583|United States|-6|single family| +19221|AAAAAAAAFBLEAAAA|670|Dogwood 6th|Ct.|Suite M|Greenfield|Marion County|TN|35038|United States|-6|single family| +19222|AAAAAAAAGBLEAAAA|986|Railroad Hill|Road|Suite V|Jackson|Nelson County|ND|59583|United States|-6|single family| +19223|AAAAAAAAHBLEAAAA|160|Jackson |RD|Suite 20|Buena Vista|Humphreys County|TN|35752|United States|-5|condo| +19224|AAAAAAAAIBLEAAAA|869|Hickory 7th|Boulevard|Suite 390|Arlington|Jackson County|SD|56557|United States|-7|condo| +19225|AAAAAAAAJBLEAAAA|768|Jefferson Twelfth|Ct.|Suite 220|Lebanon|Newton County|MO|62898|United States|-6|single family| +19226|AAAAAAAAKBLEAAAA|829|Second |Pkwy|Suite 80|Crystal|La Porte County|IN|45258|United States|-5|condo| +19227|AAAAAAAALBLEAAAA|883|Elm |Circle|Suite 0|Lakewood|Benewah County|ID|88877|United States|-7|condo| +19228|AAAAAAAAMBLEAAAA|412|Lee |Street|Suite 360|Greenwood|Pittsburg County|OK|78828|United States|-6|apartment| +19229|AAAAAAAANBLEAAAA|232|Elm |Lane|Suite 330|Perkins|Stanton County|KS|61852|United States|-6|single family| +19230|AAAAAAAAOBLEAAAA|5|Ninth Second|Way|Suite W|Lakewood|Harper County|KS|68877|United States|-6|condo| +19231|AAAAAAAAPBLEAAAA|672|2nd 12th|RD|Suite 170|Friendship|Cherry County|NE|64536|United States|-6|condo| +19232|AAAAAAAAACLEAAAA|788|Twelfth Maple|Pkwy|Suite 130|Summit|Gooding County|ID|80499|United States|-7|condo| +19233|AAAAAAAABCLEAAAA|384||Street|Suite L||Kootenai County|ID|||-7|| +19234|AAAAAAAACCLEAAAA|165|Willow Cedar|Ave|Suite 290|Lincoln|Sacramento County|CA|91289|United States|-8|condo| +19235|AAAAAAAADCLEAAAA|713|Spring Church|Dr.|Suite 360|Clifton|Coosa County|AL|38014|United States|-6|single family| +19236|AAAAAAAAECLEAAAA|28|6th |Drive|Suite J|Prosperity|Bradley County|TN|39089|United States|-5|apartment| +19237|AAAAAAAAFCLEAAAA|568|Hickory Church|Cir.|Suite 460|Jamestown|Gilmer County|GA|36867|United States|-5|single family| +19238|AAAAAAAAGCLEAAAA|388|Franklin |Ave|Suite 200|Sunnyside|Jackson County|MI|41952|United States|-5|condo| +19239|AAAAAAAAHCLEAAAA|332|Railroad View|ST|Suite 270|Hamilton|Flagler County|FL|32808|United States|-5|condo| +19240|AAAAAAAAICLEAAAA|616|Lincoln Willow|Lane|Suite J|Newtown|Putnam County|WV|21749|United States|-5|apartment| +19241|AAAAAAAAJCLEAAAA|766|Poplar Jackson|Court|Suite H|Fairview|Henderson County|NC|25709|United States|-5|single family| +19242|AAAAAAAAKCLEAAAA|595|Railroad Oak|Wy|Suite 180|Spring Valley|Calhoun County|IL|66060|United States|-6|single family| +19243|AAAAAAAALCLEAAAA|152|Sunset Eigth|Court|Suite 410|Glenwood|Cumberland County|VA|23511|United States|-5|condo| +19244|AAAAAAAAMCLEAAAA|889|14th Chestnut|Boulevard|Suite 200|Springfield|Walthall County|MS|59303|United States|-6|apartment| +19245|AAAAAAAANCLEAAAA|558|Spruce 7th|Avenue|Suite 390|Hillcrest|Union County|AR|73003|United States|-6|single family| +19246|AAAAAAAAOCLEAAAA|61|11th Lee|Court|Suite M|Wildwood|Jackson County|NC|26871|United States|-5|single family| +19247|AAAAAAAAPCLEAAAA|237||Blvd|Suite V|||VA|29310|||condo| +19248|AAAAAAAAADLEAAAA|704|Laurel Smith|Wy|Suite F|Jones|Renville County|MN|52686|United States|-6|condo| +19249|AAAAAAAABDLEAAAA|812|Sunset |Parkway|Suite 400|Midway|Prince George County|MD|21904|United States|-5|single family| +19250|AAAAAAAACDLEAAAA|364|12th |Cir.|Suite M|Fairview|Shawano County|WI|55709|United States|-6|apartment| +19251|AAAAAAAADDLEAAAA|633|Oak |Parkway|Suite G|Unionville|Defiance County|OH|41711|United States|-5|apartment| +19252|AAAAAAAAEDLEAAAA|328|First 4th|Ave|Suite U|Oak Grove|Washita County|OK|78370|United States|-6|condo| +19253|AAAAAAAAFDLEAAAA|659|||||Tooele County||84107||-7|condo| +19254|AAAAAAAAGDLEAAAA|||Avenue||Pleasant Hill||WA|93604||-8|single family| +19255|AAAAAAAAHDLEAAAA|438|Dogwood Park|Ln|Suite 400|Deerfield|Rockingham County|VA|29840|United States|-5|single family| +19256|AAAAAAAAIDLEAAAA|831|Third Willow|Ln|Suite 180|Unionville|Sharp County|AR|71711|United States|-6|single family| +19257|AAAAAAAAJDLEAAAA|912|Highland Cedar|RD|Suite T|Wilson|McHenry County|ND|56971|United States|-6|condo| +19258|AAAAAAAAKDLEAAAA|50|Green Miller|Ave|Suite V|Riverside|Henry County|KY|49231|United States|-6|condo| +19259|AAAAAAAALDLEAAAA|929|Jefferson West|Wy|Suite Y|Mount Olive|Wheeler County|TX|78059|United States|-6|single family| +19260|AAAAAAAAMDLEAAAA|869|Green Lake|Blvd|Suite U|Spring Valley|Washington County|KY|46060|United States|-5|single family| +19261|AAAAAAAANDLEAAAA|976||Road|Suite O|||TX|78054|United States|-6|| +19262|AAAAAAAAODLEAAAA|447|South North|ST|Suite A|Mount Vernon|Huntington County|IN|48482|United States|-5|apartment| +19263|AAAAAAAAPDLEAAAA|962|Maple |Circle|Suite 230|Riverside|Barry County|MO|69231|United States|-6|condo| +19264|AAAAAAAAAELEAAAA|172|Forest |ST|Suite 210|Arlington|Paulding County|GA|36557|United States|-5|condo| +19265|AAAAAAAABELEAAAA|446|Cherry 4th|Ct.|Suite T|Clifton|Humphreys County|MS|58014|United States|-6|single family| +19266|AAAAAAAACELEAAAA|958|Hickory |Cir.|Suite V|Kingston|Lincoln County|TN|34975|United States|-6|single family| +19267|AAAAAAAADELEAAAA|69|Washington |Parkway|Suite 280|Spring Hill|Franklin County|OH|46787|United States|-5|apartment| +19268|AAAAAAAAEELEAAAA|75|Railroad Wilson|Blvd|Suite 250|Highland|Montgomery County|MS|59454|United States|-6|single family| +19269|AAAAAAAAFELEAAAA|302|1st Adams|Cir.|Suite 370|Five Points|Pulaski County|GA|36098|United States|-5|single family| +19270|AAAAAAAAGELEAAAA|191|Church 15th|Lane|Suite 370|Mount Zion|Dawson County|NE|68054|United States|-6|apartment| +19271|AAAAAAAAHELEAAAA|757|Madison Main|Way|Suite C|Woodland Park|Benewah County|ID|81934|United States|-7|condo| +19272|AAAAAAAAIELEAAAA|803|Eigth |Ct.|Suite 290|Woodlawn|Little River County|AR|74098|United States|-6|apartment| +19273|AAAAAAAAJELEAAAA|101|Central Main|Street|Suite 460|White Oak|Stillwater County|MT|66668|United States|-7|apartment| +19274|AAAAAAAAKELEAAAA|340|Park |Road|Suite E|Deerfield|Somervell County|TX|79840|United States|-6|apartment| +19275|AAAAAAAALELEAAAA|510|Main |Way|Suite 180|Oak Hill|Stonewall County|TX|77838|United States|-6|apartment| +19276|AAAAAAAAMELEAAAA|226|3rd Spring|Ave|Suite T|Hopewell|Avery County|NC|20587|United States|-5|apartment| +19277|AAAAAAAANELEAAAA|679|Elm |ST|Suite I|Maple Grove|Bergen County|NJ|08852|United States|-5|apartment| +19278|AAAAAAAAOELEAAAA|121|6th |Ct.|Suite 280|Enterprise|Delta County|TX|71757|United States|-6|apartment| +19279|AAAAAAAAPELEAAAA|994|Park |Parkway|Suite S|Woodville|Lafayette Parish|LA|74289|United States|-6|single family| +19280|AAAAAAAAAFLEAAAA|275|15th |Court|Suite X|Ashley|Marion County|TN|34324|United States|-6|condo| +19281|AAAAAAAABFLEAAAA|206|Spruce Hill|Pkwy|Suite 40|Marion|Jasper County|MS|50399|United States|-6|single family| +19282|AAAAAAAACFLEAAAA|755|Center |Drive|Suite 320|Mountain View|Irion County|TX|74466|United States|-6|single family| +19283|AAAAAAAADFLEAAAA|534|First 11th|Lane|Suite B|Liberty|Onondaga County|NY|13451|United States|-5|apartment| +19284|AAAAAAAAEFLEAAAA|552|Third |Way||Oakwood|||||-5|apartment| +19285|AAAAAAAAFFLEAAAA|707|7th Wilson|Pkwy|Suite L|Summit|Larue County|KY|40499|United States|-5|single family| +19286|AAAAAAAAGFLEAAAA|329||Drive|Suite K|Newport|McKean County||11521|||apartment| +19287|AAAAAAAAHFLEAAAA|709|Fifth |Ave|Suite J|Concord|Polk County|WI|54107|United States|-6|apartment| +19288|AAAAAAAAIFLEAAAA|741|Hillcrest |Wy|Suite E|Clinton|Clarke County|MS|58222|United States|-6|single family| +19289|AAAAAAAAJFLEAAAA|525|Wilson 3rd|Blvd|Suite C|Rossville|Allen County|KY|42503|United States|-6|condo| +19290|AAAAAAAAKFLEAAAA|260|9th |RD|Suite S|Lakeview|Bandera County|TX|78579|United States|-6|single family| +19291|AAAAAAAALFLEAAAA|247|Main |Blvd|Suite 410|Centerville|Reno County|KS|60059|United States|-6|single family| +19292|AAAAAAAAMFLEAAAA|54|Cedar 15th|Ct.|Suite A|Deerfield|Stanton County|KS|69840|United States|-6|single family| +19293|AAAAAAAANFLEAAAA|934|River |Way|Suite U|Mountain View|Caldwell County|TX|74466|United States|-6|apartment| +19294|AAAAAAAAOFLEAAAA|883|Tenth |Boulevard|Suite J|Adrian|Slope County|ND|53301|United States|-6|apartment| +19295|AAAAAAAAPFLEAAAA|126|Green Fifth|Avenue|Suite 310|Lincoln|Upshur County|WV|21289|United States|-5|condo| +19296|AAAAAAAAAGLEAAAA|903|4th |Wy|Suite 310|Marion|Issaquena County|MS|50399|United States|-6|apartment| +19297|AAAAAAAABGLEAAAA|170|East Adams|Ct.|Suite 110|Florence|Thurston County|WA|93394|United States|-8|apartment| +19298|AAAAAAAACGLEAAAA|702|Spruce |Lane|Suite 250|Deerfield|Tuscarawas County|OH|49840|United States|-5|single family| +19299|AAAAAAAADGLEAAAA|636|Fifth |Lane|Suite 480|Glenwood|Sanpete County|UT|83511|United States|-7|condo| +19300|AAAAAAAAEGLEAAAA|198|Valley |Lane|Suite 150|Fairview|Warren County|IN|45709|United States|-5|condo| +19301|AAAAAAAAFGLEAAAA|106|2nd |Dr.|Suite A|Valley View|Okanogan County|WA|95124|United States|-8|single family| +19302|AAAAAAAAGGLEAAAA|636|Meadow Spruce|Ln|Suite 50|Buena Vista|Henry County|GA|35752|United States|-5|single family| +19303|AAAAAAAAHGLEAAAA|724|11th 10th|Dr.|Suite D|Wildwood|Noble County|IN|46871|United States|-5|apartment| +19304|AAAAAAAAIGLEAAAA|891|Church |RD|Suite 400|Deerfield|Vinton County|OH|49840|United States|-5|single family| +19305|AAAAAAAAJGLEAAAA|658|1st Mill|Street|Suite 360|Unionville|Columbia County|PA|11711|United States|-5|condo| +19306|AAAAAAAAKGLEAAAA|805|Meadow |Pkwy|Suite 480|Nichols|Hancock County|TN|37940|United States|-5|apartment| +19307|AAAAAAAALGLEAAAA|197|Church |Ave|Suite 140|Woodlawn|Middlesex County|VA|24098|United States|-5|single family| +19308|AAAAAAAAMGLEAAAA|366|Willow |Pkwy|Suite G|White Oak|Bosque County|TX|76668|United States|-6|condo| +19309|AAAAAAAANGLEAAAA|625|Lee |ST|Suite L|Springhill|Rio Grande County|CO|84602|United States|-7|apartment| +19310|AAAAAAAAOGLEAAAA|299|South Poplar|Lane|Suite B|Farmington|Kimble County|TX|79145|United States|-6|condo| +19311|AAAAAAAAPGLEAAAA|269|11th |Drive|Suite 210|Bay View|Essex County|VA|26457|United States|-5|apartment| +19312|AAAAAAAAAHLEAAAA|162|Cherry |Street|Suite K|Sunnyside|Kemper County|MS|51952|United States|-6|condo| +19313|AAAAAAAABHLEAAAA|260|Pine Johnson|Parkway|Suite 390|Pleasant Grove|Faulkner County|AR|74136|United States|-6|apartment| +19314|AAAAAAAACHLEAAAA|19|Valley |Parkway|Suite S|Summit||GA||United States|-5|single family| +19315|AAAAAAAADHLEAAAA|866|11th Hickory|Dr.|Suite 410|Lebanon|Green County|KY|42898|United States|-6|apartment| +19316|AAAAAAAAEHLEAAAA|505|Central Walnut|Cir.|Suite 400|Woodlawn|Prince George County|MD|24098|United States|-5|apartment| +19317|AAAAAAAAFHLEAAAA|731|Highland Hickory|Wy|Suite Q|Clinton|Cerro Gordo County|IA|58222|United States|-6|single family| +19318|AAAAAAAAGHLEAAAA|925|First |Road|Suite 250|Stringtown|Lafayette County|FL|30162|United States|-5|condo| +19319|AAAAAAAAHHLEAAAA|142|West Lakeview|Court|Suite F|Greenfield|Menard County|IL|65038|United States|-6|apartment| +19320|AAAAAAAAIHLEAAAA|488|River Sixth|Street|Suite 270|Northwood|Page County|IA|54104|United States|-6|apartment| +19321|AAAAAAAAJHLEAAAA|481|First |Boulevard|Suite 110|Hamilton|Comal County|TX|72808|United States|-6|condo| +19322|AAAAAAAAKHLEAAAA|952|Main Spring|Lane|Suite 170|Pleasant Valley|Oldham County|KY|42477|United States|-5|single family| +19323|AAAAAAAALHLEAAAA|200|Willow |Cir.|Suite A|Oakdale|Bertie County|NC|29584|United States|-5|single family| +19324|AAAAAAAAMHLEAAAA|274|Poplar |Drive|Suite J|Concord|Northumberland County|VA|24107|United States|-5|condo| +19325|AAAAAAAANHLEAAAA||Oak Oak|Drive|Suite 330|Buena Vista||ID|||-7|| +19326|AAAAAAAAOHLEAAAA|451|5th North|Pkwy|Suite 340|Woodville|Iberia Parish|LA|74289|United States|-6|condo| +19327|AAAAAAAAPHLEAAAA|291|12th |RD|Suite 440|Farmington|Forrest County|MS|59145|United States|-6|single family| +19328|AAAAAAAAAILEAAAA|347|Sycamore |Ct.|Suite R|Marion|Columbia County|PA|10399|United States|-5|condo| +19329|AAAAAAAABILEAAAA|698|2nd |Lane|Suite I|Lakeside|Eau Claire County|WI|59532|United States|-6|condo| +19330|AAAAAAAACILEAAAA|868|Laurel ||Suite I|Plainview|Carter County||33683||-5|single family| +19331|AAAAAAAADILEAAAA|496|Dogwood |ST|Suite 50|Oak Ridge|Las Animas County|CO|88371|United States|-7|single family| +19332|AAAAAAAAEILEAAAA|931|Walnut 11th|Lane|Suite I|Oak Ridge|Brown County|MN|58371|United States|-6|apartment| +19333|AAAAAAAAFILEAAAA|622|Railroad Pine|Ct.|Suite D|Hopewell|McIntosh County|OK|70587|United States|-6|single family| +19334|AAAAAAAAGILEAAAA|882|Elm Madison|Cir.|Suite 330|Lakeville|Obion County|TN|38811|United States|-6|apartment| +19335|AAAAAAAAHILEAAAA|963|Spring Johnson|Lane|Suite R|Fairview|Ashtabula County|OH|45709|United States|-5|condo| +19336|AAAAAAAAIILEAAAA|222|Davis 4th|Court|Suite 280|Concord|Centre County|PA|14107|United States|-5|apartment| +19337|AAAAAAAAJILEAAAA|243|West |Ln|Suite Y|Pleasant Valley|Lafayette County|AR|72477|United States|-6|single family| +19338|AAAAAAAAKILEAAAA|541|Birch West|Dr.|Suite 340|Bridgeport|Rooks County|KS|65817|United States|-6|condo| +19339|AAAAAAAALILEAAAA|155|Ash |ST|Suite T|Lakewood|Beadle County|SD|58877|United States|-6|single family| +19340|AAAAAAAAMILEAAAA|445|Valley Central|Road|Suite 380|Mount Olive|Saline County|MO|68059|United States|-6|single family| +19341|AAAAAAAANILEAAAA|334|3rd Walnut|Cir.|Suite 270|Oakdale|Butler County|NE|69584|United States|-6|single family| +19342|AAAAAAAAOILEAAAA|443|3rd West|Boulevard|Suite X|Pleasant Grove|Hutchinson County|TX|74136|United States|-6|single family| +19343|AAAAAAAAPILEAAAA|876|Jackson North|RD|Suite L|Walnut Grove|Carroll County|VA|27752|United States|-5|apartment| +19344|AAAAAAAAAJLEAAAA|106|North Meadow|RD|Suite Q|Oak Grove|Lake County|FL|38370|United States|-5|condo| +19345|AAAAAAAABJLEAAAA|21|Smith |Ln|Suite X|Springdale|Darke County|OH|48883|United States|-5|apartment| +19346|AAAAAAAACJLEAAAA|824|Poplar 2nd|ST|Suite 260|Macedonia|Taylor County|TX|71087|United States|-6|condo| +19347|AAAAAAAADJLEAAAA|370|Lincoln |Lane|Suite 100|Jackson|Day County|SD|59583|United States|-6|condo| +19348|AAAAAAAAEJLEAAAA||Jackson |Boulevard||||||United States|-5|| +19349|AAAAAAAAFJLEAAAA|791|Thirteenth 15th|Circle|Suite T|Pleasant Hill|Armstrong County|PA|13604|United States|-5|condo| +19350|AAAAAAAAGJLEAAAA|902|5th West|Boulevard|Suite 150|Franklin|Middlesex County|VA|29101|United States|-5|single family| +19351|AAAAAAAAHJLEAAAA|640|1st |Avenue|Suite 300|Shiloh|Berks County|PA|19275|United States|-5|condo| +19352|AAAAAAAAIJLEAAAA|363|View Park|Circle|Suite 380|Plainview|Adams County|ND|53683|United States|-6|apartment| +19353|AAAAAAAAJJLEAAAA|919|Lake |Wy|Suite 420|Red Hill|Dyer County|TN|34338|United States|-5|single family| +19354|AAAAAAAAKJLEAAAA|296|5th 1st|Lane|Suite O|Maywood|Montmorency County|MI|45681|United States|-5|condo| +19355|AAAAAAAALJLEAAAA||Fifth Park|Boulevard|Suite N|Spring Valley|Madison County|FL||||| +19356|AAAAAAAAMJLEAAAA|410|Park Madison|Circle|Suite 190|Oakland|Franklin County|IN|49843|United States|-5|apartment| +19357|AAAAAAAANJLEAAAA|912|Maple Pine|Dr.|Suite Q|Buena Vista|Uinta County|WY|85752|United States|-7|single family| +19358|AAAAAAAAOJLEAAAA|228|3rd |Blvd|Suite I|Cedar Grove|Mower County|MN|50411|United States|-6|condo| +19359|AAAAAAAAPJLEAAAA|740|Railroad Davis|Cir.|Suite 10|Springdale|Union County|IL|68883|United States|-6|apartment| +19360|AAAAAAAAAKLEAAAA|866|Lakeview Oak|Dr.|Suite 180|Sunnyside|Kit Carson County|CO|81952|United States|-7|condo| +19361|AAAAAAAABKLEAAAA|360|Jackson Smith|RD|Suite G|Mechanicsburg|Pickens County|AL|32219|United States|-6|condo| +19362|AAAAAAAACKLEAAAA|858|Washington Lee|Lane|Suite 100|Lincoln|Shelby County|IA|51289|United States|-6|condo| +19363|AAAAAAAADKLEAAAA||Birch ||Suite 330|Clinton|||98222|||condo| +19364|AAAAAAAAEKLEAAAA|573|8th Locust|Pkwy|Suite K|Lakewood|Finney County|KS|68877|United States|-6|single family| +19365|AAAAAAAAFKLEAAAA|831|Wilson |Street|Suite V|Calhoun|McPherson County|KS|66909|United States|-6|apartment| +19366|AAAAAAAAGKLEAAAA|518|First Park|Ct.|Suite 220|Providence|Buchanan County|MO|66614|United States|-6|single family| +19367|AAAAAAAAHKLEAAAA|109|Third Maple|Ave|Suite G|Jackson|Dillingham Census Area|AK|99583|United States|-9|apartment| +19368|AAAAAAAAIKLEAAAA|759|View Pine|Cir.|Suite 300|Five Points|Brevard County|FL|36098|United States|-5|apartment| +19369|AAAAAAAAJKLEAAAA|886|Lake |Street|Suite 260|Pleasant Valley|Richland County|IL|62477|United States|-6|apartment| +19370|AAAAAAAAKKLEAAAA|617|Railroad Lincoln|RD|Suite P|Maple Grove|Carlton County|MN|58252|United States|-6|apartment| +19371|AAAAAAAALKLEAAAA|621|Fourth |Blvd|Suite Y|Highland|Robeson County|NC|29454|United States|-5|condo| +19372|AAAAAAAAMKLEAAAA|697|Green 2nd|Lane|Suite O|Payne|San Luis Obispo County|CA|96134|United States|-8|condo| +19373|AAAAAAAANKLEAAAA|289|Cedar |Blvd|Suite T|Franklin|Warren County|MO|69101|United States|-6|condo| +19374|AAAAAAAAOKLEAAAA|172|Second |Dr.|Suite 120|Pleasant Hill|Hancock County|GA|33604|United States|-5|condo| +19375|AAAAAAAAPKLEAAAA|568|Lake |Pkwy|Suite W|Kingston|Arapahoe County|CO|84975|United States|-7|condo| +19376|AAAAAAAAALLEAAAA|743|11th |Dr.|Suite K|Pine Grove|Iron County|WI|54593|United States|-6|apartment| +19377|AAAAAAAABLLEAAAA|902|Lake |Circle|Suite B|Marion|Newaygo County|MI|40399|United States|-5|condo| +19378|AAAAAAAACLLEAAAA|||Boulevard|||Davie County|NC|29310|||condo| +19379|AAAAAAAADLLEAAAA|989|Eigth |Blvd|Suite 80|Belmont|Lexington city|VA|20191|United States|-5|condo| +19380|AAAAAAAAELLEAAAA|356|Sycamore |Dr.|Suite 160|Riverdale|Winneshiek County|IA|59391|United States|-6|condo| +19381|AAAAAAAAFLLEAAAA|335|Spruce Pine|Road|Suite 130|Spring Valley|Lamar County|GA|36060|United States|-5|single family| +19382|AAAAAAAAGLLEAAAA|671|North 4th|Ct.|Suite Q|White Oak|Washington County|IA|56668|United States|-6|apartment| +19383|AAAAAAAAHLLEAAAA|456|3rd 12th|Wy|Suite 0|Oakwood|Armstrong County|PA|10169|United States|-5|condo| +19384|AAAAAAAAILLEAAAA||||Suite G||Pasco County|FL|36971|United States||| +19385|AAAAAAAAJLLEAAAA|924|4th Park|Ln|Suite S|Arlington|Bastrop County|TX|76557|United States|-6|apartment| +19386|AAAAAAAAKLLEAAAA|541|Chestnut |Ct.|Suite 410|Oak Hill|Rogers County|OK|77838|United States|-6|condo| +19387|AAAAAAAALLLEAAAA|608|Second Hill|Way|Suite N|Woodville|Fremont County|CO|84289|United States|-7|single family| +19388|AAAAAAAAMLLEAAAA|137|Smith |Avenue|Suite N|Woodcrest|Assumption Parish|LA|74919|United States|-6|condo| +19389|AAAAAAAANLLEAAAA|102|Maple 3rd|Pkwy|Suite P|Woodville|Teton County|ID|84289|United States|-7|single family| +19390|AAAAAAAAOLLEAAAA|608|Hickory |Wy|Suite 200|Summit|Gratiot County|MI|40499|United States|-5|condo| +19391|AAAAAAAAPLLEAAAA|518|Meadow Smith|Ct.|Suite C|Riverdale|Bleckley County|GA|39391|United States|-5|single family| +19392|AAAAAAAAAMLEAAAA|800|13th Walnut|ST|Suite 220|Enterprise|Marion County|AR|71757|United States|-6|single family| +19393|AAAAAAAABMLEAAAA|943|Dogwood |Court|Suite 320|Enterprise|Oneida County|WI|51757|United States|-6|apartment| +19394|AAAAAAAACMLEAAAA|209|Broadway Lincoln||Suite V|||WA|93394|United States|-8|| +19395|AAAAAAAADMLEAAAA|636|Valley |Parkway|Suite 210|Spring Valley|Haskell County|KS|66060|United States|-6|apartment| +19396|AAAAAAAAEMLEAAAA|380|Chestnut Chestnut|Way|Suite 110|Georgetown|Greenville County|SC|27057|United States|-5|single family| +19397|AAAAAAAAFMLEAAAA|||Avenue|Suite 450|Mountain View|Juab County|||||single family| +19398|AAAAAAAAGMLEAAAA|284|3rd Washington|Parkway|Suite 170|Greenwood|Fayette County|IA|58828|United States|-6|single family| +19399|AAAAAAAAHMLEAAAA|767|11th |Drive|Suite 490|Bridgeport|Fairfield County|SC|25817|United States|-5|single family| +19400|AAAAAAAAIMLEAAAA|194|Meadow Washington||Suite W||Pike County||42503|United States||condo| +19401|AAAAAAAAJMLEAAAA|188|3rd |Road|Suite 0|Enterprise|Sherman County|NE|61757|United States|-7|apartment| +19402|AAAAAAAAKMLEAAAA|189|Tenth Broadway|Street|Suite 40|Shelby|Lincoln County|ID|86575|United States|-7|apartment| +19403|AAAAAAAALMLEAAAA|490|Hill Washington|Pkwy|Suite 40|Bradford|Buffalo County|NE|61184|United States|-6|apartment| +19404|AAAAAAAAMMLEAAAA|854|6th 3rd|Ln|Suite 200|Marion|Judith Basin County|MT|60399|United States|-7|condo| +19405|AAAAAAAANMLEAAAA|872|4th Madison|RD|Suite E|Oakwood|Rice County|MN|50169|United States|-6|apartment| +19406|AAAAAAAAOMLEAAAA|285|Eigth |Blvd|Suite 240|Five Points|Hamilton County|KS|66098|United States|-6|condo| +19407|AAAAAAAAPMLEAAAA|553|11th |Drive|Suite O|Forest Hills|Rutland County|VT|09837|United States|-5|apartment| +19408|AAAAAAAAANLEAAAA|869|Church |Ct.|Suite T|Riverview|Porter County|IN|49003|United States|-5|condo| +19409|AAAAAAAABNLEAAAA|823|Meadow |Circle|Suite I|Rockwood|Saginaw County|MI|41545|United States|-5|apartment| +19410|AAAAAAAACNLEAAAA|763|Sunset Lincoln|Pkwy|Suite V|Brownsville|Clallam County|WA|99310|United States|-8|single family| +19411|AAAAAAAADNLEAAAA|868|River |Cir.|Suite 290|Newport|Arthur County|NE|61521|United States|-6|condo| +19412|AAAAAAAAENLEAAAA|591|Johnson College|Boulevard|Suite V|Globe|Jackson County|KS|69614|United States|-6|condo| +19413|AAAAAAAAFNLEAAAA|752|Third |Blvd|Suite 490|Greenfield|Kendall County|TX|75038|United States|-6|condo| +19414|AAAAAAAAGNLEAAAA|184|2nd |Ct.|Suite 180|Five Points|Hyde County|SD|56098|United States|-7|apartment| +19415|AAAAAAAAHNLEAAAA|17|Hillcrest |Street|Suite 340|Oak Ridge|Allegheny County|PA|18371|United States|-5|condo| +19416|AAAAAAAAINLEAAAA|540|First |Blvd|Suite 430|Shady Grove|Watonwan County|MN|52812|United States|-6|apartment| +19417|AAAAAAAAJNLEAAAA|915|Davis Jefferson|Parkway|Suite 380|Oak Hill|Northampton County|PA|17838|United States|-5|condo| +19418|AAAAAAAAKNLEAAAA|64|South |Ct.|Suite I|Summit|Broward County|FL|30499|United States|-5|condo| +19419|AAAAAAAALNLEAAAA|772|Fourth Main|Road|Suite P|Bunker Hill|Franklin County|IL|60150|United States|-6|apartment| +19420|AAAAAAAAMNLEAAAA|758|Central Park|Parkway|Suite L|Franklin|Willacy County|TX|79101|United States|-6|single family| +19421|AAAAAAAANNLEAAAA|66|South |Street|Suite H|Deerfield|Wapello County|IA|59840|United States|-6|single family| +19422|AAAAAAAAONLEAAAA|895|Cherry |Avenue|Suite U|Oakwood|Floyd County|KY|40169|United States|-6|apartment| +19423|AAAAAAAAPNLEAAAA|869|Main Ash|Cir.|Suite X|Greenwood|Menominee County|MI|48828|United States|-5|condo| +19424|AAAAAAAAAOLEAAAA|947|Railroad 6th|Ct.|Suite 60|Spring Hill|Beadle County|SD|56787|United States|-6|single family| +19425|AAAAAAAABOLEAAAA|131|Spruce |Street|Suite D|Greenfield|Saline County|MO|65038|United States|-6|single family| +19426|AAAAAAAACOLEAAAA|565|Center |Blvd|Suite 380|Enterprise|Boise County|ID|81757|United States|-7|condo| +19427|AAAAAAAADOLEAAAA|158|Central First|Dr.|Suite 400|Valley View|Jackson County|OR|95124|United States|-8|apartment| +19428|AAAAAAAAEOLEAAAA|946|View Railroad|Dr.|Suite 420|Mount Vernon|Barnwell County|SC|28482|United States|-5|apartment| +19429|AAAAAAAAFOLEAAAA|216|Hill |Boulevard|Suite 140|Glendale|Pike County|AR|73951|United States|-6|apartment| +19430|AAAAAAAAGOLEAAAA|987||Ave||Jamestown|Moniteau County|||United States||| +19431|AAAAAAAAHOLEAAAA|240|Jackson |ST|Suite K|Union|Bates County|MO|68721|United States|-6|apartment| +19432|AAAAAAAAIOLEAAAA|748|Ash 1st|Way|Suite 330|Enterprise|Shelby County|KY|41757|United States|-5|single family| +19433|AAAAAAAAJOLEAAAA|359|Second |Wy|Suite V|Sulphur Springs|Clay County|IN|48354|United States|-5|apartment| +19434|AAAAAAAAKOLEAAAA|581|College |Lane|Suite 180|Mount Zion|Henry County|OH|48054|United States|-5|apartment| +19435|AAAAAAAALOLEAAAA|285|Spring 2nd|ST|Suite 340|Greenville|Douglas County|OR|91387|United States|-8|single family| +19436|AAAAAAAAMOLEAAAA|751|Park |Parkway|Suite 160|Hillcrest|Lincoln County|NE|63003|United States|-7|single family| +19437|AAAAAAAANOLEAAAA|143|Railroad |Blvd|Suite 440|Sulphur Springs|Des Moines County|IA|58354|United States|-6|condo| +19438|AAAAAAAAOOLEAAAA|608|Spring |Road|Suite M|Spring Valley|Henderson County|TN|36060|United States|-5|condo| +19439|AAAAAAAAPOLEAAAA|626|13th |Cir.|Suite A|Bethel|Summers County|WV|25281|United States|-5|apartment| +19440|AAAAAAAAAPLEAAAA|250|Green |RD|Suite 440|Arlington|Mahnomen County|MN|56557|United States|-6|condo| +19441|AAAAAAAABPLEAAAA|246|Oak Jackson|Court|Suite 180|Antioch|Lee County|IL|68605|United States|-6|condo| +19442|AAAAAAAACPLEAAAA|57|Willow |Blvd|Suite R|Union|Nowata County|OK|78721|United States|-6|single family| +19443|AAAAAAAADPLEAAAA|944|Pine Washington|Circle|Suite 420|Oakland|Sanilac County|MI|49843|United States|-5|condo| +19444|AAAAAAAAEPLEAAAA|611|Center |Avenue|Suite 210|Glenwood|Ware County|GA|33511|United States|-5|condo| +19445|AAAAAAAAFPLEAAAA|959|Lake |Road|Suite 270|Lakeside|San Juan County|WA|99532|United States|-8|condo| +19446|AAAAAAAAGPLEAAAA|356|View Chestnut|Cir.|Suite 250|Clifton|Renville County|ND|58014|United States|-6|single family| +19447|AAAAAAAAHPLEAAAA|142|Willow Spring|Boulevard|Suite 300|Buena Vista|Stafford County|KS|65752|United States|-6|apartment| +19448|AAAAAAAAIPLEAAAA|693|9th Laurel|Street|Suite 490|Oak Hill|Jenkins County|GA|37838|United States|-5|condo| +19449|AAAAAAAAJPLEAAAA|80|Meadow Adams|Parkway|Suite W|Mount Pleasant|Pawnee County|KS|61933|United States|-6|apartment| +19450|AAAAAAAAKPLEAAAA|682|Spring Central|Parkway|Suite 110|Green Acres|Pickett County|TN|37683|United States|-6|condo| +19451|AAAAAAAALPLEAAAA|423|Lake |Road|Suite 280|Midway|Kootenai County|ID|81904|United States|-7|single family| +19452|AAAAAAAAMPLEAAAA|896|Johnson |Pkwy|Suite 350|Clinton|Carroll County|KY|48222|United States|-6|condo| +19453|AAAAAAAANPLEAAAA|868|Elm |Avenue|Suite S|Wilson|Victoria County|TX|76971|United States|-6|apartment| +19454|AAAAAAAAOPLEAAAA|737|Elm Maple|Ct.|Suite 30|Wildwood|Hardin County|IL|66871|United States|-6|single family| +19455|AAAAAAAAPPLEAAAA|39|2nd |Blvd|Suite P|Five Forks|Whitley County|IN|42293|United States|-5|condo| +19456|AAAAAAAAAAMEAAAA|622|Sunset |Boulevard|Suite K|Church Hill|Wood County|WV|23790|United States|-5|single family| +19457|AAAAAAAABAMEAAAA|788|Main Davis|Way|Suite 230|Union|Washington County|MO|68721|United States|-6|single family| +19458|AAAAAAAACAMEAAAA|862|First 8th|Court|Suite H|Freeman|Gibson County|IN|42297|United States|-5|apartment| +19459|AAAAAAAADAMEAAAA|735|North |Cir.|Suite D|Bunker Hill|Union County|IN|40150|United States|-5|single family| +19460|AAAAAAAAEAMEAAAA|285|Franklin Birch|Parkway|Suite 360|Greenville|Nacogdoches County|TX|71387|United States|-6|apartment| +19461|AAAAAAAAFAMEAAAA|||Ct.|||Ripley County|||United States||apartment| +19462|AAAAAAAAGAMEAAAA|796|Spruce 12th|Wy|Suite 210|Oak Grove|Reynolds County|MO|68370|United States|-6|condo| +19463|AAAAAAAAHAMEAAAA|810|Hillcrest |ST|Suite 350|Woodland|Racine County|WI|54854|United States|-6|single family| +19464|AAAAAAAAIAMEAAAA|357|Johnson Church|ST|Suite 440|Cedar Grove|Jasper County|IN|40411|United States|-5|condo| +19465|AAAAAAAAJAMEAAAA|733|6th 4th|Dr.|Suite C|Oak Hill|Hamilton County|KS|67838|United States|-6|condo| +19466|AAAAAAAAKAMEAAAA|803|Sunset |Ln|Suite F|Macedonia|Monroe County|MO|61087|United States|-6|condo| +19467|AAAAAAAALAMEAAAA|187|Ninth |Circle|Suite X|Murray|Kittitas County|WA|92150|United States|-8|condo| +19468|AAAAAAAAMAMEAAAA|907|13th |Road|Suite 220|Riverside|Carroll County|MS|59231|United States|-6|apartment| +19469|AAAAAAAANAMEAAAA|43|Cherry Cedar|Avenue|Suite C|Buena Vista|Weld County|CO|85752|United States|-7|condo| +19470|AAAAAAAAOAMEAAAA|146|East |Way|Suite 50|Franklin|Ozark County|MO|69101|United States|-6|condo| +19471|AAAAAAAAPAMEAAAA|983|Cedar 8th|Wy|Suite 40|Lakeview|Kenosha County|WI|58579|United States|-6|condo| +19472|AAAAAAAAABMEAAAA|768|Main |Cir.|Suite 440|Lakeview|Grant County|AR|78579|United States|-6|single family| +19473|AAAAAAAABBMEAAAA|933|Pine |RD|Suite A|Deerfield|Gates County|NC|29840|United States|-5|single family| +19474|AAAAAAAACBMEAAAA|567|Jackson Cedar|Court|Suite 180|Summit|Amite County|MS|50499|United States|-6|single family| +19475|AAAAAAAADBMEAAAA|700|Park |Street|Suite 260|Greenville|Grand County|UT|81387|United States|-7|single family| +19476|AAAAAAAAEBMEAAAA|613|Birch Meadow|Ave|Suite A|Oakdale|Escambia County|FL|39584|United States|-5|single family| +19477|AAAAAAAAFBMEAAAA|280|Spruce |Road|Suite 240|Springdale|Adams County|WA|98883|United States|-8|condo| +19478|AAAAAAAAGBMEAAAA|901|12th 2nd|Pkwy|Suite 470|Enterprise|Bristol County|MA|02357|United States|-5|apartment| +19479|AAAAAAAAHBMEAAAA|303|West 5th|||||KS||||apartment| +19480|AAAAAAAAIBMEAAAA|952|1st |Drive|Suite Q|Newtown|Hopkins County|KY|41749|United States|-6|single family| +19481|AAAAAAAAJBMEAAAA|810|Second |Circle|Suite F|Belmont|Holmes County|MS|50191|United States|-6|condo| +19482|AAAAAAAAKBMEAAAA|972|13th |Road|Suite 20|Sulphur Springs|Laramie County|WY|88354|United States|-7|condo| +19483|AAAAAAAALBMEAAAA|238|2nd 4th|Blvd|Suite I|||MT|68354|United States||condo| +19484|AAAAAAAAMBMEAAAA|213|Washington Tenth|Avenue|Suite C|Marion|Essex County|NY|10399|United States|-5|condo| +19485|AAAAAAAANBMEAAAA|954|Lakeview |RD|Suite 170|Antioch|Danville city|VA|28605|United States|-5|single family| +19486|AAAAAAAAOBMEAAAA|522|7th |Dr.|Suite 410|Spring Hill|Franklin city|VA|26787|United States|-5|single family| +19487|AAAAAAAAPBMEAAAA|17||Blvd|||||69583|||condo| +19488|AAAAAAAAACMEAAAA|598|||Suite 40|Bunker Hill|Fayette County|KY|40150|United States||| +19489|AAAAAAAABCMEAAAA|728|Madison 2nd|Avenue|Suite I|Royal|Brown County|OH|45819|United States|-5|single family| +19490|AAAAAAAACCMEAAAA|748|Oak South|Street|Suite A|Riverview|Custer County|NE|69003|United States|-6|single family| +19491|AAAAAAAADCMEAAAA|997|Cedar |Drive|Suite N|Fairfield|Scurry County|TX|76192|United States|-6|single family| +19492|AAAAAAAAECMEAAAA|183|2nd Smith|Blvd|Suite 230|Hartland|Emmet County|IA|56594|United States|-6|condo| +19493|AAAAAAAAFCMEAAAA|639|River Lakeview|Ave|Suite 350|Glenwood|Travis County|TX|73511|United States|-6|apartment| +19494|AAAAAAAAGCMEAAAA|921|Sixth |Parkway|Suite 290|New Hope|Bay County|MI|49431|United States|-5|condo| +19495|AAAAAAAAHCMEAAAA|580|Woodland |Road|Suite H|Riverview|Franklin County|NC|29003|United States|-5|apartment| +19496|AAAAAAAAICMEAAAA|779|Hickory Willow|Ln|Suite 110|Hopewell|Mesa County|CO|80587|United States|-7|single family| +19497|AAAAAAAAJCMEAAAA|680|East Walnut|Street|Suite O|Antioch|Lincoln County|MN|58605|United States|-6|single family| +19498|AAAAAAAAKCMEAAAA|989|Green |Circle|Suite 70|Shiloh|Clinton County|KY|49275|United States|-6|condo| +19499|AAAAAAAALCMEAAAA|691|West |Avenue|Suite 450|Marion|Monroe County|TN|30399|United States|-6|single family| +19500|AAAAAAAAMCMEAAAA|||||Deerfield|Tyler County|WV|29840|United States|-5|condo| +19501|AAAAAAAANCMEAAAA|80|Johnson 5th|Road|Suite 370|Woodland|Hood County|TX|74854|United States|-6|single family| +19502|AAAAAAAAOCMEAAAA|76|Second |Blvd|Suite 180|Bethel|Camden County|MO|65281|United States|-6|single family| +19503|AAAAAAAAPCMEAAAA|648|Pine Lincoln|Pkwy|Suite 470|New Hope|Hubbard County|MN|59431|United States|-6|single family| +19504|AAAAAAAAADMEAAAA|739|Second Cedar|Cir.|Suite 350|Glendale|Cass County|IA|53951|United States|-6|single family| +19505|AAAAAAAABDMEAAAA|825|Oak |Avenue|Suite A|Marion|Rhea County|TN|30399|United States|-6|single family| +19506|AAAAAAAACDMEAAAA|320|Second Tenth|Way|Suite C|Harmony|Harrison County|TX|75804|United States|-6|condo| +19507|AAAAAAAADDMEAAAA|13|Ash |Way|Suite F|Pine Grove|Pope County|MN|54593|United States|-6|condo| +19508|AAAAAAAAEDMEAAAA|442|Lincoln |Street|Suite J|Highland|Clinton County|KY|49454|United States|-6|condo| +19509|AAAAAAAAFDMEAAAA|322|Eigth Main|Blvd|Suite 160|Lakeview|Habersham County|GA|38579|United States|-5|single family| +19510|AAAAAAAAGDMEAAAA|538|Lake Mill|Road|Suite S|Springfield|Warren County|PA|19303|United States|-5|condo| +19511|AAAAAAAAHDMEAAAA|898|7th Main|Boulevard|Suite I|Westgate|Tangipahoa Parish|LA|72366|United States|-6|single family| +19512|AAAAAAAAIDMEAAAA|468|Oak |Pkwy|Suite 480|Oak Ridge|Solano County|CA|98371|United States|-8|condo| +19513|AAAAAAAAJDMEAAAA|341|Lakeview |Court|Suite B|Riverside|Henderson County|IL|69231|United States|-6|single family| +19514|AAAAAAAAKDMEAAAA|643|Lee East|Dr.|Suite A|Ashland|Culberson County|TX|74244|United States|-6|single family| +19515|AAAAAAAALDMEAAAA|354|Highland |ST|Suite 30|Crossroads|Dyer County|TN|30534|United States|-5|apartment| +19516|AAAAAAAAMDMEAAAA|648|Fifth |Wy|Suite X|Highland Park|Branch County|MI|46534|United States|-5|condo| +19517|AAAAAAAANDMEAAAA|511|West Washington|Lane|Suite 160|Glendale|Cleburne County|AR|73951|United States|-6|single family| +19518|AAAAAAAAODMEAAAA|117|2nd 12th|Blvd|Suite 350|Crossroads|Burnett County|WI|50534|United States|-6|condo| +19519|AAAAAAAAPDMEAAAA|51|Fourth |Ln|Suite 230|Brownsville|Hoke County|NC|29310|United States|-5|apartment| +19520|AAAAAAAAAEMEAAAA|293|Walnut Pine|Circle|Suite 90|Lakewood|Washington County|MD|28877|United States|-5|single family| +19521|AAAAAAAABEMEAAAA|460|3rd |Boulevard|Suite P|Highland Park|Brunswick County|VA|26534|United States|-5|apartment| +19522|AAAAAAAACEMEAAAA|141|Park |Lane|Suite O|Mount Olive|Bedford County|TN|38059|United States|-5|apartment| +19523|AAAAAAAADEMEAAAA|695|Lee Twelfth|Ct.|Suite 70|Unionville|McCurtain County|OK|71711|United States|-6|condo| +19524|AAAAAAAAEEMEAAAA|346|2nd |Parkway|Suite 120|Riverside|Berkshire County|MA|09831|United States|-5|single family| +19525|AAAAAAAAFEMEAAAA|935|Walnut Hickory|Drive|Suite X|Bethel|Crawford County|IA|55281|United States|-6|single family| +19526|AAAAAAAAGEMEAAAA|517|Pine |Blvd|Suite Y|Lakewood|Bowie County|TX|78877|United States|-6|condo| +19527|AAAAAAAAHEMEAAAA|38|Lake |Blvd|Suite K|Oak Hill|Tippah County|MS|57838|United States|-6|single family| +19528|AAAAAAAAIEMEAAAA|224|Spruce Park|ST|Suite 390|Deerfield|Caroline County|VA|29840|United States|-5|single family| +19529|AAAAAAAAJEMEAAAA|938|Willow |Pkwy|Suite 40|Harmony|Johnston County|NC|25804|United States|-5|single family| +19530|AAAAAAAAKEMEAAAA|490|Twelfth 6th|Avenue|Suite 280|Rutland|Bienville Parish|LA|78375|United States|-6|single family| +19531|AAAAAAAALEMEAAAA|963|1st Pine|Dr.|Suite 290|Waterloo|Horry County|SC|21675|United States|-5|single family| +19532|AAAAAAAAMEMEAAAA|923|View |Ln|Suite 440|Jacksonville|Union County|IA|58223|United States|-6|condo| +19533|AAAAAAAANEMEAAAA|679|Fourth Franklin|Street|Suite 100|Sulphur Springs|Huron County|MI|48354|United States|-5|single family| +19534|AAAAAAAAOEMEAAAA|608|Hickory |Drive|Suite 190|New Hope|Alfalfa County|OK|79431|United States|-6|single family| +19535|AAAAAAAAPEMEAAAA|386|6th |Wy|Suite P|Highland Park|Middlesex County|NJ|07134|United States|-5|apartment| +19536|AAAAAAAAAFMEAAAA|63|North Valley|Parkway|Suite 400|Edgewood|Otoe County|NE|60069|United States|-7|apartment| +19537|AAAAAAAABFMEAAAA|946|Main Hickory|Street|Suite F|Fairview|Tuolumne County|CA|95709|United States|-8|condo| +19538|AAAAAAAACFMEAAAA|635|South |Ave|Suite 320|Riverside|Fremont County|CO|89231|United States|-7|single family| +19539|AAAAAAAADFMEAAAA|640|Maple |Cir.|Suite 450|Ashland|Okfuskee County|OK|74244|United States|-6|condo| +19540|AAAAAAAAEFMEAAAA|467|Madison |Dr.|Suite R|Hamilton|Bernalillo County|NM|82808|United States|-7|condo| +19541|AAAAAAAAFFMEAAAA|696|Hill |Lane|Suite 480|Five Forks|Prowers County|CO|82293|United States|-7|apartment| +19542|AAAAAAAAGFMEAAAA|200|Oak |RD|Suite R|Freeman|Jeff Davis County|TX|72297|United States|-6|apartment| +19543|AAAAAAAAHFMEAAAA|267|11th Ridge|Parkway|Suite 420|Glendale|Howard County|IA|53951|United States|-6|condo| +19544|AAAAAAAAIFMEAAAA|188|Sunset |Wy|Suite J|Pleasant Hill|Banks County|GA|33604|United States|-5|single family| +19545|AAAAAAAAJFMEAAAA|426|Cherry |Dr.|Suite 330|Glenwood|Blanco County|TX|73511|United States|-6|apartment| +19546|AAAAAAAAKFMEAAAA|||Boulevard|||Columbia County|||United States||| +19547|AAAAAAAALFMEAAAA|516|Highland |Circle|Suite A|Four Points|Faribault County|MN|51216|United States|-6|single family| +19548|AAAAAAAAMFMEAAAA|818|Lincoln |Court|Suite 420|Brownsville|Johnson County|IL|69310|United States|-6|single family| +19549|AAAAAAAANFMEAAAA|270|Laurel |Avenue|Suite 210|Mount Olive|Holt County|MO|68059|United States|-6|apartment| +19550|AAAAAAAAOFMEAAAA|72|Walnut 2nd|RD|Suite I|Harmony|Colquitt County|GA|35804|United States|-5|apartment| +19551|AAAAAAAAPFMEAAAA|961|Pine |Ave|Suite I|Union Hill|Llano County|TX|77746|United States|-6|apartment| +19552|AAAAAAAAAGMEAAAA|719|Ridge |Avenue|Suite 390|Nottingham|Fauquier County|VA|24074|United States|-5|single family| +19553|AAAAAAAABGMEAAAA|112|Hill Oak|Ct.|Suite 480|Union Hill|Montgomery County|NC|27746|United States|-5|apartment| +19554|AAAAAAAACGMEAAAA||Cedar College||Suite M||Marion County|WV|25124|United States|-5|| +19555|AAAAAAAADGMEAAAA|765|8th |Circle|Suite N|Pine Grove|Humboldt County|NV|84593|United States|-8|single family| +19556|AAAAAAAAEGMEAAAA|471|Cedar Davis|Parkway|Suite 60|Newport|Raleigh County|WV|21521|United States|-5|condo| +19557|AAAAAAAAFGMEAAAA|271|6th Fourth|Boulevard|Suite 140|Mount Olive|Owsley County|KY|48059|United States|-5|single family| +19558|AAAAAAAAGGMEAAAA|795|Walnut |Blvd|Suite 430|Lakewood|Lamb County|TX|78877|United States|-6|apartment| +19559|AAAAAAAAHGMEAAAA|321|Elm Ridge|ST|Suite 50|Deerfield|Windham County|CT|09840|United States|-5|single family| +19560|AAAAAAAAIGMEAAAA|287|Lake Third|Wy|Suite P|Mount Zion|Kanawha County|WV|28054|United States|-5|condo| +19561|AAAAAAAAJGMEAAAA|645|Main Park|Dr.|Suite P|Sand Hill|Jefferson County|OR|99530|United States|-8|single family| +19562|AAAAAAAAKGMEAAAA|949|Spruce Chestnut|Blvd|Suite 130|Riverview|Lincoln County|WI|59003|United States|-6|apartment| +19563|AAAAAAAALGMEAAAA|204|Third Church|Cir.|Suite 160|Clifton|Buchanan County|MO|68014|United States|-6|condo| +19564|AAAAAAAAMGMEAAAA|334|Jefferson |Street|Suite 400|Green Acres|Colquitt County|GA|37683|United States|-5|apartment| +19565|AAAAAAAANGMEAAAA|79|10th |Road|Suite 300|Woodland|Tulare County|CA|94854|United States|-8|single family| +19566|AAAAAAAAOGMEAAAA|675|5th |RD|Suite 400|Harmony|Kidder County|ND|55804|United States|-6|apartment| +19567|AAAAAAAAPGMEAAAA|686|Lincoln |Court|Suite 340|Shaw|Roane County|TN|30618|United States|-6|single family| +19568|AAAAAAAAAHMEAAAA|16|5th Hillcrest|Lane|Suite R|Spring Hill|Douglas County|MN|56787|United States|-6|apartment| +19569|AAAAAAAABHMEAAAA|641|Meadow Davis|RD|Suite A|Mount Vernon|Mora County|NM|88482|United States|-7|single family| +19570|AAAAAAAACHMEAAAA|8|10th 11th|Ave|Suite A|Clinton|Walker County|GA|38222|United States|-5|single family| +19571|AAAAAAAADHMEAAAA|925|Elm |ST|Suite L|Union|Crawford County|MI|48721|United States|-5|condo| +19572|AAAAAAAAEHMEAAAA|425|11th |Pkwy|Suite T|Stringtown|Lincoln County|GA|30162|United States|-5|apartment| +19573|AAAAAAAAFHMEAAAA|||Drive|Suite 460|||KY|||-6|| +19574|AAAAAAAAGHMEAAAA|14|||Suite X|Clifton|Polk County|WI|||-6|single family| +19575|AAAAAAAAHHMEAAAA|30|North 5th|Pkwy|Suite B|Jackson|Brown County|SD|59583|United States|-6|condo| +19576|AAAAAAAAIHMEAAAA|647|Washington |Ave|Suite 270|Griffin|Forest County|PA|11204|United States|-5|single family| +19577|AAAAAAAAJHMEAAAA|899|View 13th|Circle|Suite J|Georgetown|Charlottesville city|VA|27057|United States|-5|condo| +19578|AAAAAAAAKHMEAAAA|874|Lakeview |Court|Suite 240|Springdale|Kit Carson County|CO|88883|United States|-7|single family| +19579|AAAAAAAALHMEAAAA||Ash ||Suite 470|Jamestown||||United States||| +19580|AAAAAAAAMHMEAAAA|870|Laurel Laurel|Road|Suite Y|Centerville|Hot Spring County|AR|70059|United States|-6|condo| +19581|AAAAAAAANHMEAAAA|849|Walnut Park|Way|Suite E|Riverview|Greene County|NC|29003|United States|-5|apartment| +19582|AAAAAAAAOHMEAAAA|863|Lincoln |Parkway|Suite H|Spring Hill|San Luis Obispo County|CA|96787|United States|-8|single family| +19583|AAAAAAAAPHMEAAAA|430|Church |Ct.|Suite 280|Mount Olive|Haskell County|OK|78059|United States|-6|single family| +19584|AAAAAAAAAIMEAAAA|30|Lake Church|Ln|Suite 410|Greenfield|Rio Grande County|CO|85038|United States|-7|condo| +19585|AAAAAAAABIMEAAAA|446|Locust Forest|Ave|Suite 370|Harmony|Clay County|TN|35804|United States|-5|apartment| +19586|AAAAAAAACIMEAAAA|108|Chestnut |Ct.|Suite 460|Riverside|Sawyer County|WI|59231|United States|-6|apartment| +19587|AAAAAAAADIMEAAAA|303|South 11th|Ave|Suite J|Summit|Gladwin County|MI|40499|United States|-5|apartment| +19588|AAAAAAAAEIMEAAAA|988|South |Drive|Suite S|Clearview|Perry County|AR|75495|United States|-6|apartment| +19589|AAAAAAAAFIMEAAAA|535|Third Center|Ave|Suite 180|Woodrow|Harnett County|NC|24273|United States|-5|apartment| +19590|AAAAAAAAGIMEAAAA|403|Davis |Ln|Suite N|Oakdale|Lee County|AL|39584|United States|-6|condo| +19591|AAAAAAAAHIMEAAAA|980|View Fourth|Ave|Suite 460|Lebanon|Osage County|OK|72898|United States|-6|single family| +19592|AAAAAAAAIIMEAAAA|775|Sycamore |Ln|Suite S|Allison|Pine County|MN|54167|United States|-6|apartment| +19593|AAAAAAAAJIMEAAAA|165|Church |Avenue|Suite Q|Lake Forest|Tulare County|CA|96000|United States|-8|condo| +19594|AAAAAAAAKIMEAAAA|999|Cedar 14th|Wy|Suite 220|Greenfield|Barron County|WI|55038|United States|-6|condo| +19595|AAAAAAAALIMEAAAA|220|North |Circle|Suite 450|Liberty|Prince George County|MD|23451|United States|-5|single family| +19596|AAAAAAAAMIMEAAAA|435|Park Fourth|Drive|Suite 180|Mountain View|Roberts County|SD|54466|United States|-7|condo| +19597|AAAAAAAANIMEAAAA|76|Meadow 5th|Ave|Suite 30|Oak Grove|Anne Arundel County|MD|28370|United States|-5|single family| +19598|AAAAAAAAOIMEAAAA|30|West Ash|Boulevard|Suite 490|Hillcrest|Swift County|MN|53003|United States|-6|single family| +19599|AAAAAAAAPIMEAAAA|624|Laurel |Parkway|Suite 50|Harmony|Mississippi County|MO|65804|United States|-6|apartment| +19600|AAAAAAAAAJMEAAAA|953|Central 4th|Cir.|Suite O|Hopewell|Missaukee County|MI|40587|United States|-5|apartment| +19601|AAAAAAAABJMEAAAA|838|Spring Church|Boulevard|Suite N|Blue Springs|Boyle County|KY|44686|United States|-6|condo| +19602|AAAAAAAACJMEAAAA|877|Ash Railroad|Ct.|Suite 160|Bunker Hill|Grenada County|MS|50150|United States|-6|apartment| +19603|AAAAAAAADJMEAAAA|248|Church |RD|Suite H|Green Acres|Daviess County|KY|47683|United States|-6|single family| +19604|AAAAAAAAEJMEAAAA|423|3rd East|Avenue|Suite 480|Ellisville|Delaware County|OH|46820|United States|-5|single family| +19605|AAAAAAAAFJMEAAAA|560|Chestnut River|Avenue|Suite 140|Brownsville|Lauderdale County|TN|39310|United States|-6|apartment| +19606|AAAAAAAAGJMEAAAA|90|Hillcrest Elm|Blvd|Suite W|Spring Hill|Marion County|OH|46787|United States|-5|apartment| +19607|AAAAAAAAHJMEAAAA|273|Smith Second|Ave|Suite 140|New Town|Fremont County|WY|89634|United States|-7|apartment| +19608|AAAAAAAAIJMEAAAA|763|9th |Blvd|Suite 130|Liberty|Scott County|IL|63451|United States|-6|apartment| +19609|AAAAAAAAJJMEAAAA|631|1st |Avenue|Suite J|Denmark|Wabash County|IL|65576|United States|-6|condo| +19610|AAAAAAAAKJMEAAAA|968|5th |Way|Suite E|Riverdale|Palo Pinto County|TX|79391|United States|-6|single family| +19611|AAAAAAAALJMEAAAA|185|Valley |Drive|Suite 170|Hopewell|Fairfax city|VA|20587|United States|-5|apartment| +19612|AAAAAAAAMJMEAAAA|853|View 6th|RD|Suite 40|Mount Vernon|Archuleta County|CO|88482|United States|-7|apartment| +19613|AAAAAAAANJMEAAAA|285|Miller Walnut|Drive|Suite S|Woodland|Twin Falls County|ID|84854|United States|-7|apartment| +19614|AAAAAAAAOJMEAAAA|272|Sunset |Way|Suite 290|Fairview|Ontonagon County|MI|45709|United States|-5|apartment| +19615|AAAAAAAAPJMEAAAA||Sunset Elm|Wy||Ashland||KY|44244|United States|-6|| +19616|AAAAAAAAAKMEAAAA|504|Main |Ct.|Suite O|Lebanon|La Plata County|CO|82898|United States|-7|single family| +19617|AAAAAAAABKMEAAAA|734|4th |Wy|Suite 330|Shady Grove|Lapeer County|MI|42812|United States|-5|single family| +19618|AAAAAAAACKMEAAAA|828|9th |Circle|Suite H|Glenwood|Clay County|TN|33511|United States|-5|condo| +19619|AAAAAAAADKMEAAAA|691|Wilson Meadow|Street|Suite 60|Pleasant Hill|Rogers County|OK|73604|United States|-6|condo| +19620|AAAAAAAAEKMEAAAA|760|6th 1st|Pkwy|Suite 60|Calhoun|Orleans County|NY|16909|United States|-5|single family| +19621|AAAAAAAAFKMEAAAA|954|2nd 2nd|RD|Suite 490|Riverdale|Henry County|IA|59391|United States|-6|apartment| +19622|AAAAAAAAGKMEAAAA|892|1st |Wy|Suite I|Oakland|Gage County|NE|69843|United States|-6|condo| +19623|AAAAAAAAHKMEAAAA|319|9th College|Road|Suite M|Sulphur Springs|Clearwater County|MN|58354|United States|-6|apartment| +19624|AAAAAAAAIKMEAAAA|88|Lincoln |Dr.|Suite 250|Woodland|Grafton County|NH|05454|United States|-5|condo| +19625|AAAAAAAAJKMEAAAA|948|Fifth |Circle|Suite 430|Macedonia|Hood River County|OR|91087|United States|-8|condo| +19626|AAAAAAAAKKMEAAAA|774|College |Pkwy|Suite 370|Union|Harris County|TX|78721|United States|-6|apartment| +19627|AAAAAAAALKMEAAAA|239|3rd East|RD|Suite 90|Williamsville|Aransas County|TX|78754|United States|-6|single family| +19628|AAAAAAAAMKMEAAAA|463|Main 2nd|Street|Suite 0|Hillcrest|Newton County|IN|43003|United States|-5|single family| +19629|AAAAAAAANKMEAAAA|881|Seventh |Drive|Suite 150|Ashland|Yates County|NY|14244|United States|-5|single family| +19630|AAAAAAAAOKMEAAAA|526|6th |Street|Suite U|Jackson|Washoe County|NV|89583|United States|-8|apartment| +19631|AAAAAAAAPKMEAAAA|242|Green Oak|Blvd|Suite K|White Oak|Blaine County|OK|76668|United States|-6|condo| +19632|AAAAAAAAALMEAAAA|99|View |Ave|Suite 120|Clifton|Sherburne County|MN|58014|United States|-6|condo| +19633|AAAAAAAABLMEAAAA|159|Third 3rd|Blvd|Suite 450|Cedar Grove|Reeves County|TX|70411|United States|-6|single family| +19634|AAAAAAAACLMEAAAA|983|Lake Lake|Road|Suite O|Woodlawn|Pondera County|MT|64098|United States|-7|apartment| +19635|AAAAAAAADLMEAAAA|499|Lee Ridge|Street|Suite 120|Summit|Whatcom County|WA|90499|United States|-8|apartment| +19636|AAAAAAAAELMEAAAA|801|Ash 11th|Pkwy|Suite 420|Bridgeport|Charlevoix County|MI|45817|United States|-5|single family| +19637|AAAAAAAAFLMEAAAA|284|Walnut |Ln|Suite 350|Riverview|Winkler County|TX|79003|United States|-6|single family| +19638|AAAAAAAAGLMEAAAA|778|10th |Boulevard|Suite K|Brownsville|Coffee County|AL|39310|United States|-6|single family| +19639|AAAAAAAAHLMEAAAA|59|5th Highland|Ct.|Suite 130|Summit|San Benito County|CA|90499|United States|-8|single family| +19640|AAAAAAAAILMEAAAA|229|East Johnson|Wy|Suite N|Pleasant Valley|Beaufort County|NC|22477|United States|-5|apartment| +19641|AAAAAAAAJLMEAAAA|962|River Lincoln|Cir.|Suite Q|Deerfield|Greenlee County|AZ|89840|United States|-7|apartment| +19642|AAAAAAAAKLMEAAAA|300|Forest |Way|Suite O|Lakewood|San Patricio County|TX|78877|United States|-6|single family| +19643|AAAAAAAALLMEAAAA|69|West |Drive|Suite W|Macedonia|Lewis County|WV|21087|United States|-5|condo| +19644|AAAAAAAAMLMEAAAA|206|Willow |Ln|Suite F|Bethel|Rockingham County|VA|25281|United States|-5|single family| +19645|AAAAAAAANLMEAAAA|286|Third Pine|Dr.|Suite 150|Liberty|Schley County|GA|33451|United States|-5|single family| +19646|AAAAAAAAOLMEAAAA|179|Main Hillcrest|Street|Suite 280|Lebanon|Bedford city|VA|22898|United States|-5|condo| +19647|AAAAAAAAPLMEAAAA|312|Lake Johnson|Road|Suite 450|Cedar Grove|Sagadahoc County|ME|01011|United States|-5|single family| +19648|AAAAAAAAAMMEAAAA|335|5th Sixth|Boulevard|Suite 420|Glenwood|Williams County|OH|43511|United States|-5|apartment| +19649|AAAAAAAABMMEAAAA|881|2nd Johnson|Ln|Suite 140|Marion|Liberty County|FL|30399|United States|-5|apartment| +19650|AAAAAAAACMMEAAAA|513|Hillcrest Railroad|Boulevard|Suite O|White Oak|Shelby County|TN|36668|United States|-6|single family| +19651|AAAAAAAADMMEAAAA|605|Sycamore Cedar|Road|Suite 480|Pleasant Grove|Lackawanna County|PA|14136|United States|-5|condo| +19652|AAAAAAAAEMMEAAAA|728|Sunset |Court|Suite 60|Granite|Power County|ID|86284|United States|-7|single family| +19653|AAAAAAAAFMMEAAAA|545|7th |Pkwy|Suite 350|Walnut Grove|Juneau Borough|AK|97752|United States|-9|condo| +19654|AAAAAAAAGMMEAAAA|262|Birch |Drive|Suite C|Greenwood|Barnwell County|SC|28828|United States|-5|apartment| +19655|AAAAAAAAHMMEAAAA|819|15th 8th|Boulevard|Suite 370|Woodland|Forsyth County|GA|34854|United States|-5|condo| +19656|AAAAAAAAIMMEAAAA|55|4th |Lane|Suite S|Sunnyside|Franklin County|TN|31952|United States|-5|apartment| +19657|AAAAAAAAJMMEAAAA|516|6th Jackson|Circle|Suite O|Greenwood|Surry County|NC|28828|United States|-5|apartment| +19658|AAAAAAAAKMMEAAAA|861|Miller |ST|Suite V|Fairfield|Foard County|TX|76192|United States|-6|single family| +19659|AAAAAAAALMMEAAAA|86|Elm |Cir.|Suite Y|Providence|Mower County|MN|56614|United States|-6|single family| +19660|AAAAAAAAMMMEAAAA|463|Lake |Drive|Suite 130|Mount Vernon|Carbon County|WY|88482|United States|-7|single family| +19661|AAAAAAAANMMEAAAA|190|3rd Oak|Dr.|Suite 380|Mount Olive|Nueces County|TX|78059|United States|-6|condo| +19662|AAAAAAAAOMMEAAAA|803|Church Center|Way|Suite Y|Union Hill|Logan County|CO|87746|United States|-7|apartment| +19663|AAAAAAAAPMMEAAAA|4|10th |Ave|Suite 70|Deerfield|Suffolk County|MA|09840|United States|-5|apartment| +19664|AAAAAAAAANMEAAAA|951|Walnut 9th|Blvd|Suite W|Five Points|Coffey County|KS|66098|United States|-6|condo| +19665|AAAAAAAABNMEAAAA|951|Park |Court|Suite 310|Summit|Lanier County|GA|30499|United States|-5|apartment| +19666|AAAAAAAACNMEAAAA|18|Main Jefferson|Circle|Suite O|Sugar Hill|Parmer County|TX|75114|United States|-6|single family| +19667|AAAAAAAADNMEAAAA|190|1st West|Drive|Suite 240|Waterloo|Boone County|NE|61675|United States|-6|single family| +19668|AAAAAAAAENMEAAAA|764|Lee |Drive|Suite 230|Nottingham|Geauga County|OH|44074|United States|-5|single family| +19669|AAAAAAAAFNMEAAAA|174|5th |Ct.|Suite Y|Salem|Jones County|MS|58048|United States|-6|condo| +19670|AAAAAAAAGNMEAAAA|85|Maple South|Circle|Suite I|Greenwood|Green Lake County|WI|58828|United States|-6|apartment| +19671|AAAAAAAAHNMEAAAA|476|13th Sycamore|Drive|Suite 270|Weldon|Carroll County|MS|56277|United States|-6|single family| +19672|AAAAAAAAINMEAAAA|548|Willow |Parkway|Suite J|Silver Creek|Emmet County|MI|44546|United States|-5|single family| +19673|AAAAAAAAJNMEAAAA|359|Seventh |Parkway|Suite 60|Newtown|Fulton County|GA|31749|United States|-5|condo| +19674|AAAAAAAAKNMEAAAA|298|3rd |Parkway|Suite O|Greenville|Hardin County|IL|61387|United States|-6|single family| +19675|AAAAAAAALNMEAAAA|746|Willow |ST|Suite H|Green Acres|Wells County|ND|57683|United States|-6|single family| +19676|AAAAAAAAMNMEAAAA|254|College Cedar|Boulevard|Suite B|Shiloh|Pulaski County|AR|79275|United States|-6|apartment| +19677|AAAAAAAANNMEAAAA|416|Madison Fourth|Dr.|Suite A|Allentown|Goodhue County|MN|51838|United States|-6|apartment| +19678|AAAAAAAAONMEAAAA|753|Woodland Dogwood|Drive|Suite G|Greenville|Greene County|IN|41387|United States|-5|condo| +19679|AAAAAAAAPNMEAAAA|945|View |Court|Suite M|Ashland|Covington County|AL|34244|United States|-6|condo| +19680|AAAAAAAAAOMEAAAA|493|Main Miller|Pkwy|Suite 240|Kingston|Adams County|WI|54975|United States|-6|apartment| +19681|AAAAAAAABOMEAAAA|778|2nd |Boulevard|Suite T|Brookwood|Lincoln County|ID|80965|United States|-7|apartment| +19682|AAAAAAAACOMEAAAA|496|Main Sunset|Ave|Suite 30|Red Hill|Hart County|GA|34338|United States|-5|single family| +19683|AAAAAAAADOMEAAAA|||||||OH|40411||-5|apartment| +19684|AAAAAAAAEOMEAAAA|698|Fourth |Court|Suite 140|Macedonia|Middlesex County|CT|01687|United States|-5|apartment| +19685|AAAAAAAAFOMEAAAA|276|3rd |Road|Suite 150|Hillcrest|Hancock County|KY|43003|United States|-6|condo| +19686|AAAAAAAAGOMEAAAA|650|Main College|Blvd|Suite J|Providence|Cannon County|TN|36614|United States|-5|condo| +19687|AAAAAAAAHOMEAAAA|69|First Poplar|RD|Suite H|Newport|Stone County|AR|71521|United States|-6|single family| +19688|AAAAAAAAIOMEAAAA|341|West |Pkwy|Suite Y|Lakeview|Cape May County|NJ|09179|United States|-5|apartment| +19689|AAAAAAAAJOMEAAAA||||Suite 360||Crow Wing County|MN|51229|United States||| +19690|AAAAAAAAKOMEAAAA|559|Dogwood Ash|Ln|Suite Y|Clinton|Brewster County|TX|78222|United States|-6|apartment| +19691|AAAAAAAALOMEAAAA|27|1st ||Suite 30|Jackson|Ottawa County|||United States|-6|| +19692|AAAAAAAAMOMEAAAA|976|Woodland |Wy|Suite 140|Fairfield|Solano County|CA|96192|United States|-8|condo| +19693|AAAAAAAANOMEAAAA|263|11th Walnut|Way|Suite 400|Florence|Surry County|VA|23394|United States|-5|single family| +19694|AAAAAAAAOOMEAAAA|32|||Suite M||Pratt County||66534|||condo| +19695|AAAAAAAAPOMEAAAA|311|Pine Madison|Circle|Suite B|Fairview|Warren County|IN|45709|United States|-5|single family| +19696|AAAAAAAAAPMEAAAA|32|Johnson |Circle|Suite 190|Highland Park|Kane County|IL|66534|United States|-6|single family| +19697|AAAAAAAABPMEAAAA|366|||Suite 450|Woodlawn||MI|44098||-5|apartment| +19698|AAAAAAAACPMEAAAA|344|Willow |Wy|Suite 240|Centerville|Wyoming County|NY|10059|United States|-5|apartment| +19699|AAAAAAAADPMEAAAA|521|Jackson Fourth|Boulevard|Suite Y|Bayview|Acadia Parish|LA|79672|United States|-6|single family| +19700|AAAAAAAAEPMEAAAA|218|South Johnson|RD|Suite 220|Franklin|Washoe County|NV|89101|United States|-8|apartment| +19701|AAAAAAAAFPMEAAAA|84|Wilson 4th|RD|Suite 170|Spring Valley|Wells County|IN|46060|United States|-5|apartment| +19702|AAAAAAAAGPMEAAAA|253|Elm |Cir.|Suite D|Bridgeport|Cabell County|WV|25817|United States|-5|single family| +19703|AAAAAAAAHPMEAAAA|885|Birch Dogwood|Ln|Suite N|Greenville|Decatur County|IA|51387|United States|-6|condo| +19704|AAAAAAAAIPMEAAAA|123|Maple |Cir.|Suite 140|Highland Park|Chesterfield County|SC|26534|United States|-5|apartment| +19705|AAAAAAAAJPMEAAAA|691|View Eigth|Ct.|Suite 140|Stafford|Piatt County|IL|64980|United States|-6|apartment| +19706|AAAAAAAAKPMEAAAA|577|Main Park||Suite 410||Elbert County|CO||||| +19707|AAAAAAAALPMEAAAA|157|Madison Cedar|Wy|Suite 400|Concord|Graves County|KY|44107|United States|-6|single family| +19708|AAAAAAAAMPMEAAAA|527|7th 2nd|Cir.|Suite Q|Bayside|Beaver County|UT|89550|United States|-7|condo| +19709|AAAAAAAANPMEAAAA|443|Miller Williams|Way|Suite 160|Glenwood|Aransas County|TX|73511|United States|-6|condo| +19710|AAAAAAAAOPMEAAAA|642|Pine |Boulevard|Suite 280|Oak Hill|Jefferson County|IL|67838|United States|-6|apartment| +19711|AAAAAAAAPPMEAAAA|872|East |Blvd|Suite H|Belmont|Wade Hampton Census Area|AK|90191|United States|-9|condo| +19712|AAAAAAAAAANEAAAA|939|Mill 15th|Wy|Suite 80|Woodland|Marshall County|KS|64854|United States|-6|condo| +19713|AAAAAAAABANEAAAA|496|Sunset |Wy|Suite Q|Highland Park|Washington County|ID|86534|United States|-7|condo| +19714|AAAAAAAACANEAAAA|392|9th |Ave|Suite W|Hillcrest|Windham County|VT|03603|United States|-5|single family| +19715|AAAAAAAADANEAAAA|509|Johnson |Ln|Suite X|Antioch|Richmond County|NY|18605|United States|-5|single family| +19716|AAAAAAAAEANEAAAA|653|Ash Railroad|Road|Suite T|Oak Ridge|Lamoille County|VT|08971|United States|-5|apartment| +19717|AAAAAAAAFANEAAAA|322|Forest |RD|Suite O|Enterprise|Sioux County|NE|61757|United States|-7|single family| +19718|AAAAAAAAGANEAAAA|25|Davis Railroad|Ave|Suite 470|Sunnyside|Franklin County|VT|02552|United States|-5|apartment| +19719|AAAAAAAAHANEAAAA|365|Chestnut Oak|Pkwy|Suite Y|Summit|Fall River County|SD|50499|United States|-6|condo| +19720|AAAAAAAAIANEAAAA|137|Lake |Road|Suite 200|Hopewell|Lincoln County|NC|20587|United States|-5|apartment| +19721|AAAAAAAAJANEAAAA|406|Elm Mill|Way|Suite M|Oakwood|Pike County|IL|60169|United States|-6|single family| +19722|AAAAAAAAKANEAAAA|919|Birch |Circle|Suite W|Riverside|Wilson County|TN|39231|United States|-5|single family| +19723|AAAAAAAALANEAAAA|51|Dogwood |Way|Suite Q|Brownsville|Pennington County|SD|59310|United States|-7|apartment| +19724|AAAAAAAAMANEAAAA|892|Highland Forest|Lane|Suite 120|Macedonia|McCone County|MT|61087|United States|-7|single family| +19725|AAAAAAAANANEAAAA|721|Franklin |Parkway|Suite C|Ashland|Belmont County|OH|44244|United States|-5|apartment| +19726|AAAAAAAAOANEAAAA|779|Green 15th|Parkway|Suite H|Shiloh|Marion County|KY|49275|United States|-5|single family| +19727|AAAAAAAAPANEAAAA|216|Poplar Smith|Dr.|Suite Y|Kingston|Butte County|CA|94975|United States|-8|single family| +19728|AAAAAAAAABNEAAAA|335|Jackson Lake|Wy|Suite 440|Fairfield|Scott County|IA|56192|United States|-6|single family| +19729|AAAAAAAABBNEAAAA|25|Jackson |Court|Suite 400|Johnsonville|Passaic County|NJ|08345|United States|-5|apartment| +19730|AAAAAAAACBNEAAAA|150|Cedar Tenth|Ct.|Suite 400|Riverview|Robertson County|TX|79003|United States|-6|apartment| +19731|AAAAAAAADBNEAAAA|491|Birch |Ct.|Suite L|Paxton|Jessamine County|KY|45669|United States|-6|apartment| +19732|AAAAAAAAEBNEAAAA|58|Lee Park|Parkway|Suite 90|Florence|Adams County|OH|43394|United States|-5|single family| +19733|AAAAAAAAFBNEAAAA|914|5th Park|RD|Suite T|Newtown|Douglas County|MN|51749|United States|-6|apartment| +19734|AAAAAAAAGBNEAAAA|776|11th |Drive|Suite P|Buena Vista|Dickinson County|IA|55752|United States|-6|condo| +19735|AAAAAAAAHBNEAAAA|62|Seventh |Dr.|Suite 290|Antioch|Falls County|TX|78605|United States|-6|apartment| +19736|AAAAAAAAIBNEAAAA|868|Hillcrest |Avenue|Suite 50|Highland|Grand County|UT|89454|United States|-7|apartment| +19737|AAAAAAAAJBNEAAAA|506|Railroad |Way|Suite 240|Georgetown|Pierce County|NE|67057|United States|-7|apartment| +19738|AAAAAAAAKBNEAAAA|||||||OK|77746||-6|condo| +19739|AAAAAAAALBNEAAAA|259|14th |Ln|Suite 110|Five Points|Anderson County|KY|46098|United States|-6|condo| +19740|AAAAAAAAMBNEAAAA|738|Hillcrest |Drive|Suite H|Newtown|Sedgwick County|CO|81749|United States|-7|single family| +19741|AAAAAAAANBNEAAAA|143|Woodland Main|RD|Suite 440|New Hope|Giles County|VA|29431|United States|-5|apartment| +19742|AAAAAAAAOBNEAAAA|674|Hickory Seventh|Wy|Suite A|Green Acres|Howard County|NE|67683|United States|-7|apartment| +19743|AAAAAAAAPBNEAAAA|62|View |Ln|Suite 150|Midway|Roanoke County|VA|21904|United States|-5|single family| +19744|AAAAAAAAACNEAAAA|516|Hickory |Dr.|Suite 140|Springtown|Monroe County|MI|49858|United States|-5|single family| +19745|AAAAAAAABCNEAAAA|612|Washington |Ln|Suite 40|Mount Olive|Carroll County|MS|58059|United States|-6|condo| +19746|AAAAAAAACCNEAAAA|518|3rd |Street|Suite 200|Crossroads|Starr County|TX|70534|United States|-6|apartment| +19747|AAAAAAAADCNEAAAA|594|Maple Fifth|Wy|Suite D|Bridgeport|Hillsborough County|FL|35817|United States|-5|condo| +19748|AAAAAAAAECNEAAAA|820|Walnut |Way|Suite N|Fairfield|Elk County|PA|16192|United States|-5|condo| +19749|AAAAAAAAFCNEAAAA|213|West First|Dr.|Suite 160|Glendale|Wells County|IN|43951|United States|-5|condo| +19750|AAAAAAAAGCNEAAAA|831|Lakeview Lake|Circle|Suite D|Centerville|Des Moines County|IA|50059|United States|-6|apartment| +19751|AAAAAAAAHCNEAAAA|909|3rd Sycamore|RD|Suite O|Glenwood|Terry County|TX|73511|United States|-6|condo| +19752|AAAAAAAAICNEAAAA|940|4th |Blvd|Suite 260|Centerville|McClain County|OK|70059|United States|-6|apartment| +19753|AAAAAAAAJCNEAAAA|252|Valley |Avenue|Suite 260|Price|Marlboro County|SC|23824|United States|-5|single family| +19754|AAAAAAAAKCNEAAAA|53|Second 1st|Court|Suite 250|Caledonia|Osage County|MO|67411|United States|-6|condo| +19755|AAAAAAAALCNEAAAA|318|Hillcrest Spring|Blvd|Suite Y|Winona|Huntingdon County|PA|18205|United States|-5|single family| +19756|AAAAAAAAMCNEAAAA|736|West |Blvd|Suite V|Five Points|Ford County|IL|66098|United States|-6|condo| +19757|AAAAAAAANCNEAAAA|164|Cherry |Ave|Suite 120|Concord|Lawrence County|PA|14107|United States|-5|single family| +19758|AAAAAAAAOCNEAAAA|259|1st 13th|Street|Suite U|Deerfield|Clay County|AL|39840|United States|-6|single family| +19759|AAAAAAAAPCNEAAAA|525|9th 7th|Parkway|Suite D|Plainview|Adams County|IN|43683|United States|-5|apartment| +19760|AAAAAAAAADNEAAAA|277|3rd Pine|Cir.|Suite 430|Shaw|Hall County|NE|60618|United States|-6|apartment| +19761|AAAAAAAABDNEAAAA|280|10th Hill|Pkwy|Suite W|Mount Vernon|Dooly County|GA|38482|United States|-5|condo| +19762|AAAAAAAACDNEAAAA||||Suite U|||SC||United States||| +19763|AAAAAAAADDNEAAAA|434|Main |Cir.|Suite 140|Pleasant Hill|Caledonia County|VT|04204|United States|-5|apartment| +19764|AAAAAAAAEDNEAAAA|860|View 4th|Dr.|Suite 320|Hopewell|Napa County|CA|90587|United States|-8|apartment| +19765|AAAAAAAAFDNEAAAA|58|11th |RD|Suite 190|Georgetown|Loup County|NE|67057|United States|-7|apartment| +19766|AAAAAAAAGDNEAAAA|778|7th Cedar|Parkway|Suite A|Sunnyside|Winnebago County|IA|51952|United States|-6|condo| +19767|AAAAAAAAHDNEAAAA|577|Main Wilson|Wy|Suite 350|Glendale|Shannon County|SD|53951|United States|-7|apartment| +19768|AAAAAAAAIDNEAAAA|316||Boulevard|Suite 450|Mount Olive|||78059|United States|-6|single family| +19769|AAAAAAAAJDNEAAAA|31|Hillcrest |Ln|Suite 60|Marion|Monroe County|OH|40399|United States|-5|single family| +19770|AAAAAAAAKDNEAAAA|971|Park |Pkwy|Suite 60|Union Hill|Washington County|ME|08346|United States|-5|single family| +19771|AAAAAAAALDNEAAAA|604|Railroad |Parkway|Suite A|Arlington|DeWitt County|TX|76557|United States|-6|single family| +19772|AAAAAAAAMDNEAAAA|473|10th Lake|Ln|Suite 440|Providence|Gilmer County|WV|26614|United States|-5|single family| +19773|AAAAAAAANDNEAAAA|262|Mill |Drive|Suite 180|Lebanon|Alleghany County|VA|22898|United States|-5|single family| +19774|AAAAAAAAODNEAAAA|380|Jefferson Oak|Way|Suite 150|Oakdale|Bryan County|OK|79584|United States|-6|apartment| +19775|AAAAAAAAPDNEAAAA|698|Sunset |Drive|Suite L|Oak Ridge|Okanogan County|WA|98371|United States|-8|condo| +19776|AAAAAAAAAENEAAAA|495|Woodland |Pkwy|Suite W|Midway|Franklin city|VA|21904|United States|-5|apartment| +19777|AAAAAAAABENEAAAA|259|Center 10th|Road|Suite 450|Riverview|Cook County|IL|69003|United States|-6|condo| +19778|AAAAAAAACENEAAAA|965|Williams |Ln|Suite A|Marion|Perry County|MO|60399|United States|-6|condo| +19779|AAAAAAAADENEAAAA||Park Franklin|Drive|Suite W|||||United States||| +19780|AAAAAAAAEENEAAAA|778|Sunset |Lane|Suite 240|Newtown|Fresno County|CA|91749|United States|-8|apartment| +19781|AAAAAAAAFENEAAAA|312|Hillcrest |Drive|Suite 420|Liberty|Chase County|NE|63451|United States|-6|apartment| +19782|AAAAAAAAGENEAAAA|224|Main |Ln|Suite J|Cherry Valley|Ontario County|NY|10854|United States|-5|apartment| +19783|AAAAAAAAHENEAAAA|332|Laurel Seventh|Circle|Suite 280|Springfield|Portage County|OH|49303|United States|-5|apartment| +19784|AAAAAAAAIENEAAAA|625|Main |Blvd|Suite B|Woodland|Liberty County|MT|64854|United States|-7|single family| +19785|AAAAAAAAJENEAAAA|248|6th |Ct.|Suite B|Antioch|Saluda County|SC|28605|United States|-5|single family| +19786|AAAAAAAAKENEAAAA|87|College 15th|Ave|Suite 410|Salem|Lawrence County|IL|68048|United States|-6|condo| +19787|AAAAAAAALENEAAAA|909|Wilson 1st|Boulevard|Suite 120|Mount Pleasant|Dimmit County|TX|71933|United States|-6|single family| +19788|AAAAAAAAMENEAAAA|403|Madison |Ave|Suite T|Oakland|Brevard County|FL|39843|United States|-5|condo| +19789|AAAAAAAANENEAAAA|724|8th Walnut|ST|Suite 310|Antioch|Fluvanna County|VA|28605|United States|-5|condo| +19790|AAAAAAAAOENEAAAA||||Suite Q|Lone Oak|Lincoln County||07493|United States|-5|| +19791|AAAAAAAAPENEAAAA|663|Jefferson Lake|ST|Suite 280|Marion|Warren County|IA|50399|United States|-6|apartment| +19792|AAAAAAAAAFNEAAAA||Adams |||Greenfield||TN|35038||-5|single family| +19793|AAAAAAAABFNEAAAA|690|Willow Church|Pkwy|Suite 390|Oakland|Utah County|UT|89843|United States|-7|single family| +19794|AAAAAAAACFNEAAAA|681|15th Lake|Way|Suite 300|Hardy|Whitley County|IN|45354|United States|-5|condo| +19795|AAAAAAAADFNEAAAA|834|Thirteenth |Avenue|Suite B|Valley View|Adams County|MS|55124|United States|-6|apartment| +19796|AAAAAAAAEFNEAAAA|482|Sunset |Parkway|Suite G|Five Forks|Daviess County|IN|42293|United States|-5|single family| +19797|AAAAAAAAFFNEAAAA|6|Lincoln Jackson|Wy|Suite Y|Providence|Jefferson County|WA|96614|United States|-8|apartment| +19798|AAAAAAAAGFNEAAAA|862|View |Boulevard|Suite T|Lakeside|Alcorn County|MS|59532|United States|-6|condo| +19799|AAAAAAAAHFNEAAAA|647|7th Miller|ST|Suite 140|Highland Park|Quitman County|GA|36534|United States|-5|apartment| +19800|AAAAAAAAIFNEAAAA|886|Pine |Ave|Suite 420|Clearview|Bureau County|IL|65495|United States|-6|condo| +19801|AAAAAAAAJFNEAAAA|156|Third 6th|ST|Suite N|Brownsville|Holt County|NE|69310|United States|-7|condo| +19802|AAAAAAAAKFNEAAAA|260|Oak |Pkwy|Suite 240|Sunnyside|Union Parish|LA|71952|United States|-6|condo| +19803|AAAAAAAALFNEAAAA|702|Park Oak|Drive|Suite N|Pleasant Valley|Mountrail County|ND|52477|United States|-6|condo| +19804|AAAAAAAAMFNEAAAA|338|Walnut Hillcrest|Way|Suite 410|Mount Zion|Amherst County|VA|28054|United States|-5|condo| +19805|AAAAAAAANFNEAAAA|232|Jackson 10th|Circle|Suite X|Cedar Grove|Murray County|MN|50411|United States|-6|condo| +19806|AAAAAAAAOFNEAAAA|443|Pine |Parkway|Suite R|Belleville|Kitsap County|WA|92924|United States|-8|single family| +19807|AAAAAAAAPFNEAAAA|696|Main |Ave|Suite A|Oakwood|Wright County|MO|60169|United States|-6|apartment| +19808|AAAAAAAAAGNEAAAA|218|Walnut |Dr.|Suite O|Roy|Pettis County|MO|60744|United States|-6|apartment| +19809|AAAAAAAABGNEAAAA|628|Fifth Church|Way|Suite A|Newtown|Andrew County|MO|61749|United States|-6|single family| +19810|AAAAAAAACGNEAAAA|250|4th |Street|Suite N|Hopewell|Kenedy County|TX|70587|United States|-6|single family| +19811|AAAAAAAADGNEAAAA|875|Wilson Spruce|Ct.|Suite 120|Springdale|Ross County|OH|48883|United States|-5|single family| +19812|AAAAAAAAEGNEAAAA|577|||Suite 390|Liberty||||||| +19813|AAAAAAAAFGNEAAAA|842|Mill |Way|Suite 70|Forest Hills|Tulsa County|OK|79237|United States|-6|single family| +19814|AAAAAAAAGGNEAAAA|928|Spring |ST|Suite N|Forest Hills|Platte County|MO|69237|United States|-6|single family| +19815|AAAAAAAAHGNEAAAA|474|River Thirteenth|Ct.|Suite H|Clifford|Hernando County|FL|38164|United States|-5|condo| +19816|AAAAAAAAIGNEAAAA|365|Locust Cherry|Street|Suite 250|Stringtown|Stephens County|TX|70162|United States|-6|single family| +19817|AAAAAAAAJGNEAAAA|927|3rd 3rd|Lane|Suite 390|Bethel|Yadkin County|NC|25281|United States|-5|single family| +19818|AAAAAAAAKGNEAAAA|531|Railroad |Wy|Suite R|Crossroads|Sullivan County|IN|40534|United States|-5|apartment| +19819|AAAAAAAALGNEAAAA|158|Hickory Fourth|Wy|Suite 100|Hopewell|Sullivan County|MO|60587|United States|-6|apartment| +19820|AAAAAAAAMGNEAAAA|189|Main Hickory|ST|Suite 0|Fairfield|Union County|SC|26192|United States|-5|condo| +19821|AAAAAAAANGNEAAAA|||Court|||Sumter County|||United States|-6|| +19822|AAAAAAAAOGNEAAAA|211|Park 2nd|Cir.|Suite 40|Cedar Grove|Wayne County|IN|40411|United States|-5|apartment| +19823|AAAAAAAAPGNEAAAA|319|Meadow |Blvd|Suite A|Mount Olive|Volusia County|FL|38059|United States|-5|single family| +19824|AAAAAAAAAHNEAAAA|624|Smith |RD|Suite 10|Woodland|Weakley County|TN|34854|United States|-6|condo| +19825|AAAAAAAABHNEAAAA|191|6th |Drive|Suite 130|Carpenter|Chisago County|MN|51147|United States|-6|single family| +19826|AAAAAAAACHNEAAAA|731|Lake 12th|Parkway|Suite 70|Newtown|Avoyelles Parish|LA|71749|United States|-6|apartment| +19827|AAAAAAAADHNEAAAA|959|Adams College|Court|Suite Q|Pierce|Marquette County|MI|43360|United States|-5|apartment| +19828|AAAAAAAAEHNEAAAA|374|River Sunset|Circle|Suite M|Buena Vista|Linn County|MO|65752|United States|-6|condo| +19829|AAAAAAAAFHNEAAAA|940|Dogwood |Avenue|Suite E|Hardy|Andrew County|MO|65354|United States|-6|condo| +19830|AAAAAAAAGHNEAAAA|713|Maple Sunset|Pkwy|Suite L|Lakeview|Lake County|FL|38579|United States|-5|apartment| +19831|AAAAAAAAHHNEAAAA|984|View |Dr.|Suite I|Centerville|Skamania County|WA|90059|United States|-8|condo| +19832|AAAAAAAAIHNEAAAA|111|River Park|Parkway|Suite D|Oak Grove|Fulton County|PA|18370|United States|-5|single family| +19833|AAAAAAAAJHNEAAAA|766|Second Fifth|Street|Suite G|Mount Vernon|Telfair County|GA|38482|United States|-5|single family| +19834|AAAAAAAAKHNEAAAA|122|Locust |Ln|Suite K|Centerville|Davie County|NC|20059|United States|-5|single family| +19835|AAAAAAAALHNEAAAA|35|View 4th|Ave|Suite N|Forest Hills|Pontotoc County|MS|59237|United States|-6|apartment| +19836|AAAAAAAAMHNEAAAA|212|5th West|Avenue|Suite 80|Centerville|Callaway County|MO|60059|United States|-6|single family| +19837|AAAAAAAANHNEAAAA|128|Smith Fifteenth|Pkwy|Suite 120|Kingston|White County|TN|34975|United States|-6|condo| +19838|AAAAAAAAOHNEAAAA|156|Sixth |Drive|Suite W|Woodland|Elkhart County|IN|44854|United States|-5|single family| +19839|AAAAAAAAPHNEAAAA|828|3rd |RD|Suite 390|Pleasant Valley|Atlantic County|NJ|03077|United States|-5|apartment| +19840|AAAAAAAAAINEAAAA|456|Spruce |Ct.|Suite 210|Red Bank|Morrow County|OR|94975|United States|-8|apartment| +19841|AAAAAAAABINEAAAA|685|Poplar |Wy|Suite 30|Concord|Wayne County|GA|34107|United States|-5|condo| +19842|AAAAAAAACINEAAAA|139|Third |Parkway|Suite O|Maple Grove|Randall County|TX|78252|United States|-6|condo| +19843|AAAAAAAADINEAAAA|762|Oak Sycamore|Street|Suite U|Ashley|Sequoyah County|OK|74324|United States|-6|apartment| +19844|AAAAAAAAEINEAAAA|296|3rd |Circle|Suite 30|Sulphur Springs|Colquitt County|GA|38354|United States|-5|apartment| +19845|AAAAAAAAFINEAAAA|311|Birch |Dr.|Suite 220|Franklin|Jefferson County|PA|19101|United States|-5|condo| +19846|AAAAAAAAGINEAAAA|425|Dogwood Highland|Blvd|Suite I|Pleasant Grove|Harrison County|KY|44136|United States|-6|apartment| +19847|AAAAAAAAHINEAAAA|416|Walnut Lakeview|Road|Suite 140|Brownsville|Bingham County|ID|89310|United States|-7|apartment| +19848|AAAAAAAAIINEAAAA|203|Fourteenth |Ct.|Suite D|Jamestown|Hopkins County|TX|76867|United States|-6|apartment| +19849|AAAAAAAAJINEAAAA|538|Sycamore |Way|Suite O|Sawyer|Gooding County|ID|86045|United States|-7|apartment| +19850|AAAAAAAAKINEAAAA|218|1st |RD|Suite 90|Deerfield|Jefferson County|WI|59840|United States|-6|apartment| +19851|AAAAAAAALINEAAAA|80|College 3rd|Boulevard|Suite J|Hopewell|Worcester County|MD|20587|United States|-5|condo| +19852|AAAAAAAAMINEAAAA|193|Main |Street|Suite 60|Lincoln|Walker County|GA|31289|United States|-5|single family| +19853|AAAAAAAANINEAAAA|352|Meadow Forest|Pkwy|Suite A|Mount Olive|Washakie County|WY|88059|United States|-7|apartment| +19854|AAAAAAAAOINEAAAA|997|1st |Court|Suite S|Hamilton|White County|IN|42808|United States|-5|apartment| +19855|AAAAAAAAPINEAAAA|691|Maple |Avenue|Suite 160|Lakeview|Lee County|TX|78579|United States|-6|condo| +19856|AAAAAAAAAJNEAAAA|152|Lake |Cir.|Suite 170|Union Hill|Holt County|NE|67746|United States|-7|condo| +19857|AAAAAAAABJNEAAAA|806|10th River|Ave|Suite 10|Oak Hill|Fountain County|IN|47838|United States|-5|apartment| +19858|AAAAAAAACJNEAAAA|757|Jefferson Walnut|Ln|Suite N|Stringtown|Mineral County|CO|80162|United States|-7|single family| +19859|AAAAAAAADJNEAAAA|116|Sixth |Ave|Suite 460|Mountain View|Dewey County|OK|74466|United States|-6|single family| +19860|AAAAAAAAEJNEAAAA|440|9th |Ct.|Suite 470|Florence|Pulaski County|AR|73394|United States|-6|condo| +19861|AAAAAAAAFJNEAAAA|602|Railroad Center|Court|Suite Q|Maple Grove|Lincoln Parish|LA|78252|United States|-6|apartment| +19862|AAAAAAAAGJNEAAAA|48|River 1st|Road|Suite 440|Union|Jefferson County|WI|58721|United States|-6|single family| +19863|AAAAAAAAHJNEAAAA|539|Williams |Way|Suite G|Arlington|Yankton County|SD|56557|United States|-6|single family| +19864|AAAAAAAAIJNEAAAA|812|Church Oak|ST|Suite K|Wilson|Rutland County|VT|07571|United States|-5|single family| +19865|AAAAAAAAJJNEAAAA|917|Williams |Cir.|Suite 180|Deerfield|Cherokee County|TX|79840|United States|-6|condo| +19866|AAAAAAAAKJNEAAAA|915|Ridge River|Blvd|Suite 30|Belmont|Whitfield County|GA|30191|United States|-5|single family| +19867|AAAAAAAALJNEAAAA|257|Main |Pkwy|Suite N|Pleasant Valley|Wells County|IN|42477|United States|-5|apartment| +19868|AAAAAAAAMJNEAAAA|310|Sixth Sunset|Boulevard|Suite 250|Fairbanks|Valdez-Cordova Census Area|AK|96653|United States|-9|apartment| +19869|AAAAAAAANJNEAAAA|501|Hill |Parkway|Suite 120|Woodville|Calhoun County|GA|34289|United States|-5|condo| +19870|AAAAAAAAOJNEAAAA|830|4th Williams|Blvd|Suite 480|Wildwood|Tillman County|OK|76871|United States|-6|single family| +19871|AAAAAAAAPJNEAAAA|507|Pine |Ln|Suite X|Shiloh|Adams County|NE|69275|United States|-6|single family| +19872|AAAAAAAAAKNEAAAA|766|Ninth |Avenue|Suite 170|Lewisburg|Richland County|SC|27538|United States|-5|single family| +19873|AAAAAAAABKNEAAAA|676|Chestnut |Cir.|Suite 470|Georgetown|Outagamie County|WI|57057|United States|-6|condo| +19874|AAAAAAAACKNEAAAA|||Court||||OH|49145|||apartment| +19875|AAAAAAAADKNEAAAA|127|3rd South|Avenue|Suite 240|Unionville|Philadelphia County|PA|11711|United States|-5|condo| +19876|AAAAAAAAEKNEAAAA|146|Wilson Center|Drive|Suite 450|Edgewood|Dona Ana County|NM|80069|United States|-7|single family| +19877|AAAAAAAAFKNEAAAA|802|North |Street|Suite I|Glenwood|El Paso County|TX|73511|United States|-6|apartment| +19878|AAAAAAAAGKNEAAAA|414|Oak 5th|Street|Suite 490|Riverview|Clare County|MI|49003|United States|-5|condo| +19879|AAAAAAAAHKNEAAAA|373|Walnut |Ct.|Suite W|Lakeside|Cedar County|NE|69532|United States|-6|condo| +19880|AAAAAAAAIKNEAAAA|659|West |Blvd|Suite L|Woodlawn|Wells County|IN|44098|United States|-5|single family| +19881|AAAAAAAAJKNEAAAA|302|Maple |Way|Suite B|Forest Hills|Fountain County|IN|49237|United States|-5|condo| +19882|AAAAAAAAKKNEAAAA|835|Oak |Avenue|Suite 160|Pine Valley|Middlesex County|NJ|08809|United States|-5|apartment| +19883|AAAAAAAALKNEAAAA|729|Railroad Pine|ST|Suite J|Lakeside|Bryan County|OK|79532|United States|-6|condo| +19884|AAAAAAAAMKNEAAAA|518|5th Hill|Lane|Suite 50|Woodville|Moffat County|CO|84289|United States|-7|single family| +19885|AAAAAAAANKNEAAAA|150|Park Willow|Cir.|Suite 90|Ashland|Wetzel County|WV|24244|United States|-5|single family| +19886|AAAAAAAAOKNEAAAA|||Circle||White Oak|Cherokee County||26668|||| +19887|AAAAAAAAPKNEAAAA|703|Dogwood Second|Wy|Suite 420|Hopewell|Pecos County|TX|70587|United States|-6|apartment| +19888|AAAAAAAAALNEAAAA|706|River |Circle|Suite W|Shiloh|Oneida County|NY|19275|United States|-5|apartment| +19889|AAAAAAAABLNEAAAA|913|5th |Court|Suite F|Riverside|Sumter County|SC|29231|United States|-5|condo| +19890|AAAAAAAACLNEAAAA|363|3rd |ST|Suite J|Greenwood|Bland County|VA|28828|United States|-5|condo| +19891|AAAAAAAADLNEAAAA|704|Ridge South|Lane|Suite 190|Macedonia|Gem County|ID|81087|United States|-7|apartment| +19892|AAAAAAAAELNEAAAA|463|4th Hillcrest|RD|Suite 370|Winslow|Keweenaw County|MI|48525|United States|-5|apartment| +19893|AAAAAAAAFLNEAAAA|854|Walnut Ash|Ln|Suite 440|Jamestown|Leake County|MS|56867|United States|-6|single family| +19894|AAAAAAAAGLNEAAAA|344|West Green|||Florence|Bernalillo County||83394|United States||| +19895|AAAAAAAAHLNEAAAA|432|Pine Spring|Parkway|Suite 60|Lakewood|Mecklenburg County|NC|28877|United States|-5|single family| +19896|AAAAAAAAILNEAAAA|19|Elm Adams|Street|Suite L|Bunker Hill|Hardy County|WV|20150|United States|-5|condo| +19897|AAAAAAAAJLNEAAAA|226|Locust Railroad|Circle|Suite R|Five Forks|Alfalfa County|OK|72293|United States|-6|single family| +19898|AAAAAAAAKLNEAAAA|767|5th |Pkwy|Suite L|Harmon|Clay County|NC|25623|United States|-5|apartment| +19899|AAAAAAAALLNEAAAA|683|Lincoln |Dr.|Suite 140|Five Forks|San Juan County|NM|82293|United States|-7|apartment| +19900|AAAAAAAAMLNEAAAA|665|North View|Dr.|Suite 270|Midway|Lander County|NV|81904|United States|-8|apartment| +19901|AAAAAAAANLNEAAAA|106|South |Road|Suite A|Greenfield|Hawkins County|TN|35038|United States|-5|single family| +19902|AAAAAAAAOLNEAAAA|788|Miller South|Ln|Suite 50|Riverside|San Juan County|UT|89231|United States|-7|apartment| +19903|AAAAAAAAPLNEAAAA|458|Seventh 3rd|Circle|Suite V|Walnut Grove|Johnson County|AR|77752|United States|-6|condo| +19904|AAAAAAAAAMNEAAAA|752|Second |Ave|Suite 110|Valley View|Corson County|SD|55124|United States|-6|single family| +19905|AAAAAAAABMNEAAAA|||||||OK|74098|||single family| +19906|AAAAAAAACMNEAAAA|182|Mill |Cir.|Suite 230|Red Hill|Columbia County|FL|34338|United States|-5|condo| +19907|AAAAAAAADMNEAAAA|411|Davis West|Parkway|Suite X|Oak Ridge|Kiowa County|CO|88371|United States|-7|condo| +19908|AAAAAAAAEMNEAAAA|130|Maple 2nd|Pkwy|Suite 220|Nichols|Hayes County|NE|67940|United States|-6|apartment| +19909|AAAAAAAAFMNEAAAA|846|15th West|Pkwy|Suite P|Maple Grove|Kittson County|MN|58252|United States|-6|single family| +19910|AAAAAAAAGMNEAAAA|253|Main Valley|Cir.|Suite 430|Kingston|Rockcastle County|KY|44975|United States|-5|apartment| +19911|AAAAAAAAHMNEAAAA|428|Hickory |Avenue|Suite 220|Greenville|Jefferson County|GA|31387|United States|-5|single family| +19912|AAAAAAAAIMNEAAAA|443|Willow |Drive|Suite 290|Highland|Nicholas County|WV|29454|United States|-5|condo| +19913|AAAAAAAAJMNEAAAA|158|Walnut |Ct.|Suite 330|Valley View|Stafford County|KS|65124|United States|-6|condo| +19914|AAAAAAAAKMNEAAAA|36|Wilson |Avenue|Suite 420|Oakdale|Todd County|MN|59584|United States|-6|apartment| +19915|AAAAAAAALMNEAAAA|830|Cherry View|Drive|Suite B|Macedonia|Gaston County|NC|21087|United States|-5|apartment| +19916|AAAAAAAAMMNEAAAA|447|Elm Sunset|Ln|Suite F|Riverdale|Iberia Parish|LA|79391|United States|-6|apartment| +19917|AAAAAAAANMNEAAAA|881|Ash Miller|Lane|Suite N|Five Forks|Cascade County|MT|62293|United States|-7|condo| +19918|AAAAAAAAOMNEAAAA|331|2nd |Pkwy|Suite 390|Springdale|Whiteside County|IL|68883|United States|-6|single family| +19919|AAAAAAAAPMNEAAAA|901|Railroad |Parkway|Suite 150|Glenwood|Hutchinson County|TX|73511|United States|-6|condo| +19920|AAAAAAAAANNEAAAA|526|Adams ||Suite 120||Shelby County|IA|58222|United States|-6|| +19921|AAAAAAAABNNEAAAA|309|Woodland |Lane|Suite 370|Sulphur Springs|Hooker County|NE|68354|United States|-7|apartment| +19922|AAAAAAAACNNEAAAA|633|Elm |Circle|Suite 350|Salem|Guernsey County|OH|48048|United States|-5|single family| +19923|AAAAAAAADNNEAAAA|376|Sunset Elm|Cir.|Suite 300|Arcola|Sierra County|NM|81654|United States|-7|apartment| +19924|AAAAAAAAENNEAAAA|196|Cedar |Ln|Suite B|Bunker Hill|San Juan County|NM|80150|United States|-7|condo| +19925|AAAAAAAAFNNEAAAA|921|Fourth |Dr.|Suite 320||Robertson County||75709||-6|| +19926|AAAAAAAAGNNEAAAA|185|7th Wilson|Ln|Suite 360|Lincoln|Orleans County|VT|01889|United States|-5|single family| +19927|AAAAAAAAHNNEAAAA|395|11th |Pkwy|Suite 360|Greenville|Tucker County|WV|21387|United States|-5|apartment| +19928|AAAAAAAAINNEAAAA|946|Locust Jefferson|Parkway|Suite 150|Spring Hill|Evangeline Parish|LA|76787|United States|-6|condo| +19929|AAAAAAAAJNNEAAAA|10|15th Birch|Circle|Suite 0|Hillcrest|Crisp County|GA|33003|United States|-5|condo| +19930|AAAAAAAAKNNEAAAA|483|Ridge |Road|Suite 270|Mount Zion|Aleutians West Census Area|AK|98054|United States|-9|apartment| +19931|AAAAAAAALNNEAAAA|390|Hillcrest Birch|Way|Suite 440|Greenville|Sedgwick County|KS|61387|United States|-6|single family| +19932|AAAAAAAAMNNEAAAA|461|Broadway Highland|Dr.|Suite 210|Wilton|Stephens County|GA|36997|United States|-5|condo| +19933|AAAAAAAANNNEAAAA|498|River Fifth|Drive|Suite 390|Georgetown|Marshall County|TN|37057|United States|-6|condo| +19934|AAAAAAAAONNEAAAA|54|Franklin Third|Parkway|Suite F|Bridgeport|Hutchinson County|SD|55817|United States|-7|condo| +19935|AAAAAAAAPNNEAAAA|805|Adams West|ST|Suite 250|Bunker Hill|Lee County|SC|20150|United States|-5|single family| +19936|AAAAAAAAAONEAAAA|667|Mill 2nd|Dr.|Suite 180|Lakeside|Barber County|KS|69532|United States|-6|single family| +19937|AAAAAAAABONEAAAA|||Ct.|||Mineral County|||United States|-5|condo| +19938|AAAAAAAACONEAAAA|91|Birch |ST|Suite 10|Bunker Hill|Knox County|TX|70150|United States|-6|single family| +19939|AAAAAAAADONEAAAA|19|Cedar |Cir.|Suite 310|Franklin|Kanawha County|WV|29101|United States|-5|single family| +19940|AAAAAAAAEONEAAAA|676|Hickory |Avenue|Suite D|Shady Grove|Gaines County|TX|72812|United States|-6|apartment| +19941|AAAAAAAAFONEAAAA|199|Jefferson |RD|Suite W|Lebanon|Reynolds County|MO|62898|United States|-6|condo| +19942|AAAAAAAAGONEAAAA|743|Central |Boulevard|Suite U|Florence|Baker County|GA|33394|United States|-5|apartment| +19943|AAAAAAAAHONEAAAA|471|View |Road|Suite U|Washington Heights|Lincoln County|MN|58167|United States|-6|apartment| +19944|AAAAAAAAIONEAAAA|553|Valley |Street|Suite K|Midway|Hillsdale County|MI|41904|United States|-5|single family| +19945|AAAAAAAAJONEAAAA|325|Miller Highland|Circle|Suite 420|Florence|Manistee County|MI|43394|United States|-5|apartment| +19946|AAAAAAAAKONEAAAA|664|Cedar |Pkwy|Suite G|Waterloo|Kanawha County|WV|21675|United States|-5|single family| +19947|AAAAAAAALONEAAAA|804|River Church|Street|Suite W|Springfield|Florence County|WI|59303|United States|-6|single family| +19948|AAAAAAAAMONEAAAA|996|9th |Cir.|Suite 340|Macedonia|Morton County|KS|61087|United States|-6|condo| +19949|AAAAAAAANONEAAAA|425|Green Pine|Blvd|Suite O|Stringtown|Mississippi County|AR|70162|United States|-6|single family| +19950|AAAAAAAAOONEAAAA|634|Sixth |Circle|Suite F|Enterprise|Chicot County|AR|71757|United States|-6|condo| +19951|AAAAAAAAPONEAAAA|720|Chestnut Green|Dr.|Suite V|Brentwood|Jones County|NC|24188|United States|-5|apartment| +19952|AAAAAAAAAPNEAAAA||Washington |Avenue|Suite B|Franklin||MN||||apartment| +19953|AAAAAAAABPNEAAAA|315|6th |Ave|Suite T|Frankfort|Bennett County|SD|59681|United States|-6|single family| +19954|AAAAAAAACPNEAAAA|453|Spring |RD|Suite Q|Hubbard|Leavenworth County|KS|66291|United States|-6|apartment| +19955|AAAAAAAADPNEAAAA|759|Jackson Seventh|Ave|Suite P|Belmont|Chittenden County|VT|00791|United States|-5|single family| +19956|AAAAAAAAEPNEAAAA|992|||Suite R||Tippah County||59843||-6|| +19957|AAAAAAAAFPNEAAAA|831|North Woodland|Avenue|Suite 260|Jackson|Knox County|TN|39583|United States|-6|condo| +19958|AAAAAAAAGPNEAAAA|537|River |Cir.|Suite V|Hillcrest|Perry County|MO|63003|United States|-6|single family| +19959|AAAAAAAAHPNEAAAA|371|Spruce |ST|Suite W|Franklin|Beltrami County|MN|59101|United States|-6|apartment| +19960|AAAAAAAAIPNEAAAA|139|Woodland Valley|Boulevard|Suite U|Spring Valley|Hunterdon County|NJ|06660|United States|-5|apartment| +19961|AAAAAAAAJPNEAAAA|62|Walnut |Dr.|Suite F|Arthur|Wayne County|IA|55965|United States|-6|single family| +19962|AAAAAAAAKPNEAAAA|971|Seventh 9th|Pkwy|Suite K|Mount Pleasant|Rockingham County|VA|21933|United States|-5|condo| +19963|AAAAAAAALPNEAAAA|664|West |Avenue|Suite I|Clifton|Rowan County|KY|48014|United States|-5|apartment| +19964|AAAAAAAAMPNEAAAA|855|Park |Blvd|Suite 380|Five Forks|Jack County|TX|72293|United States|-6|condo| +19965|AAAAAAAANPNEAAAA|141|Cedar Washington|Wy|Suite I|Fairview|Morrill County|NE|65709|United States|-7|apartment| +19966|AAAAAAAAOPNEAAAA|82|3rd |Way|Suite U|Riverdale|Sweetwater County|WY|89391|United States|-7|single family| +19967|AAAAAAAAPPNEAAAA|261|12th |Way|Suite 170|Oakdale|Henrico County|VA|29584|United States|-5|condo| +19968|AAAAAAAAAAOEAAAA|916|Spring |Ln|Suite O|Hopewell|Moultrie County|IL|60587|United States|-6|apartment| +19969|AAAAAAAABAOEAAAA|757|Miller Walnut|Ct.|Suite 270|Pine Grove|Smyth County|VA|24593|United States|-5|single family| +19970|AAAAAAAACAOEAAAA|574|Cedar South|Dr.|Suite V|Ruth|Lincoln County|MS|50309|United States|-6|condo| +19971|AAAAAAAADAOEAAAA|773|7th |Pkwy|Suite 430|Post Oak|Edgar County|IL|68567|United States|-6|condo| +19972|AAAAAAAAEAOEAAAA|745|North First|Lane|Suite S|Crossroads|Polk County|WI|50534|United States|-6|apartment| +19973|AAAAAAAAFAOEAAAA|255|North |Blvd|Suite K|Spring Valley|Fayette County|GA|36060|United States|-5|single family| +19974|AAAAAAAAGAOEAAAA|887|College View|Wy|Suite 150|Hidden Valley|Rutland County|VT|06121|United States|-5|single family| +19975|AAAAAAAAHAOEAAAA|669|Walnut Elm|Dr.|Suite 150|Bethel|Prairie County|MT|65281|United States|-7|single family| +19976|AAAAAAAAIAOEAAAA|763|Main |Cir.|Suite V|Midway|Buchanan County|MO|61904|United States|-6|apartment| +19977|AAAAAAAAJAOEAAAA|983|Pine Lee|Drive|Suite 80|Texas|Tippecanoe County|IN|43342|United States|-5|single family| +19978|AAAAAAAAKAOEAAAA|912|Lincoln |Boulevard|Suite 250|Belmont|Delta County|TX|70191|United States|-6|condo| +19979|AAAAAAAALAOEAAAA|401|Birch |Street|Suite D|Franklin|San Jacinto County|TX|79101|United States|-6|condo| +19980|AAAAAAAAMAOEAAAA|998|Lake |Parkway|Suite M|Springdale|Griggs County|ND|58883|United States|-6|apartment| +19981|AAAAAAAANAOEAAAA|135|West |Wy|Suite W|Riley|Henry County|TN|31692|United States|-5|condo| +19982|AAAAAAAAOAOEAAAA|846|Second |Avenue|Suite 420|Providence|Monroe County|AR|76614|United States|-6|single family| +19983|AAAAAAAAPAOEAAAA|160|Elevnth |Parkway|Suite 70|Lakeview|Lawrence County|KY|48579|United States|-5|apartment| +19984|AAAAAAAAABOEAAAA|859|Johnson Lakeview|Drive|Suite L|Superior|Lowndes County|AL|32562|United States|-6|condo| +19985|AAAAAAAABBOEAAAA|68|Third |Pkwy|Suite R|Wildwood|Juniata County|PA|16871|United States|-5|condo| +19986|AAAAAAAACBOEAAAA|798|Adams |ST|Suite 460|Wildwood|East Baton Rouge Parish|LA|76871|United States|-6|single family| +19987|AAAAAAAADBOEAAAA|940|9th |Drive|Suite U|Oakdale|Benton County|AR|79584|United States|-6|apartment| +19988|AAAAAAAAEBOEAAAA|346|3rd Johnson|Blvd|Suite F|Hopewell|El Paso County|TX|70587|United States|-6|condo| +19989|AAAAAAAAFBOEAAAA|334|Maple Elm||Suite F|||||United States|-7|| +19990|AAAAAAAAGBOEAAAA|307|Walnut |Drive|Suite 40|Riverview|Clark County|AR|79003|United States|-6|apartment| +19991|AAAAAAAAHBOEAAAA|284|River |Avenue|Suite 100|Riverside|Lake County|OR|99231|United States|-8|apartment| +19992|AAAAAAAAIBOEAAAA|380|Wilson Fifth|RD|Suite Y|Willow|Shoshone County|ID|86798|United States|-7|single family| +19993|AAAAAAAAJBOEAAAA|555|6th Washington|Pkwy|Suite P|Woodrow|Bradford County|FL|34273|United States|-5|single family| +19994|AAAAAAAAKBOEAAAA|897|Second Broadway|Dr.|Suite E|Fairview|Bourbon County|KS|65709|United States|-6|apartment| +19995|AAAAAAAALBOEAAAA|669|Hillcrest |Lane|Suite 490|Maple Grove|Monterey County|CA|98252|United States|-8|condo| +19996|AAAAAAAAMBOEAAAA|188|Wilson |Ave|Suite 250|Pleasant Hill|Rosebud County|MT|63604|United States|-7|condo| +19997|AAAAAAAANBOEAAAA|664|4th |Wy|Suite 50|Creston|Bexar County|TX|71104|United States|-6|apartment| +19998|AAAAAAAAOBOEAAAA|512|River |RD|Suite C|Franklin|Calhoun County|MI|49101|United States|-5|apartment| +19999|AAAAAAAAPBOEAAAA|597|3rd |Ct.|Suite 140|Shiloh|Polk County|NC|29275|United States|-5|condo| +20000|AAAAAAAAACOEAAAA|886|Lake |Wy|Suite J|Pleasant Grove|Sussex County|NJ|04736|United States|-5|condo| +20001|AAAAAAAABCOEAAAA|801|Highland Spruce|Court|Suite 230|Enterprise|Erie County|PA|11757|United States|-5|condo| +20002|AAAAAAAACCOEAAAA|514|Sunset |Circle|Suite W|Antioch|Latah County|ID|88605|United States|-7|condo| +20003|AAAAAAAADCOEAAAA|906|First Washington|Ct.|Suite C|Pine Grove|Garfield County|UT|84593|United States|-7|condo| +20004|AAAAAAAAECOEAAAA|856|Lee 10th|Court|Suite N|Green Acres|Indian River County|FL|37683|United States|-5|single family| +20005|AAAAAAAAFCOEAAAA|900|Spring |Wy|Suite 90|Clinton|Buchanan County|IA|58222|United States|-6|condo| +20006|AAAAAAAAGCOEAAAA|81|Washington Church|Court|Suite X|Fairfield|Kingfisher County|OK|76192|United States|-6|condo| +20007|AAAAAAAAHCOEAAAA|777|10th |Parkway|Suite 330|Ashland|Sedgwick County|CO|84244|United States|-7|single family| +20008|AAAAAAAAICOEAAAA|797|Pine |Lane|Suite R|Glenwood|Perkins County|SD|53511|United States|-7|condo| +20009|AAAAAAAAJCOEAAAA|758|Park Eigth|Cir.|Suite K|Springdale|Harrison County|MO|68883|United States|-6|single family| +20010|AAAAAAAAKCOEAAAA|453|Cherry Jefferson|RD|Suite 200|Riverdale|Schenectady County|NY|19391|United States|-5|condo| +20011|AAAAAAAALCOEAAAA|365|Sixth |Street|Suite V|Oak Hill|Holmes County|MS|57838|United States|-6|condo| +20012|AAAAAAAAMCOEAAAA|782|Lake Jackson|Ct.|Suite T|Mountain View|Montgomery County|IA|54466|United States|-6|apartment| +20013|AAAAAAAANCOEAAAA|774|Walnut |Drive|Suite 240|Farmington|Franklin County|IN|49145|United States|-5|condo| +20014|AAAAAAAAOCOEAAAA|218|3rd |Circle|Suite 330|Woodland|Sanborn County|SD|54854|United States|-7|single family| +20015|AAAAAAAAPCOEAAAA|809|Adams Adams|Court|Suite O|Oak Hill|Jackson County|IL|67838|United States|-6|condo| +20016|AAAAAAAAADOEAAAA|582|Pine |Ct.|Suite 240|Gary|Potter County|TX|70418|United States|-6|condo| +20017|AAAAAAAABDOEAAAA|593|6th |Lane|Suite P|Glendale|Giles County|TN|33951|United States|-5|condo| +20018|AAAAAAAACDOEAAAA|572|Pine Franklin|Court|Suite 100|Concord|Summit County|OH|44107|United States|-5|single family| +20019|AAAAAAAADDOEAAAA|19|6th |Parkway|Suite 310|Lakewood|Stanislaus County|CA|98877|United States|-8|apartment| +20020|AAAAAAAAEDOEAAAA|757|Maple |Boulevard|Suite M|Birmingham|Carter County|TN|33372|United States|-5|single family| +20021|AAAAAAAAFDOEAAAA|238|2nd |Avenue|Suite 10|Shiloh|Umatilla County|OR|99275|United States|-8|apartment| +20022|AAAAAAAAGDOEAAAA|92|14th |Ct.|Suite 350|Lakewood|Wood County|TX|78877|United States|-6|single family| +20023|AAAAAAAAHDOEAAAA|832|Hickory Park|Boulevard|Suite L|Friendship|Lincoln County|WV|24536|United States|-5|single family| +20024|AAAAAAAAIDOEAAAA|300|12th River|Ct.|Suite 10||Huerfano County||86614|United States||single family| +20025|AAAAAAAAJDOEAAAA|212|Chestnut 1st|Avenue|Suite X|Riley|Midland County|TX|71692|United States|-6|apartment| +20026|AAAAAAAAKDOEAAAA|565|Miller 5th|Circle|Suite E|Georgetown|Gage County|NE|67057|United States|-6|apartment| +20027|AAAAAAAALDOEAAAA|404|Cedar West|Ct.|Suite G|Greenwood|Essex County|NY|18828|United States|-5|single family| +20028|AAAAAAAAMDOEAAAA|403|Park Seventh|Pkwy|Suite D|Pleasant Valley|Cayuga County|NY|12477|United States|-5|single family| +20029|AAAAAAAANDOEAAAA|234|9th 5th|RD|Suite 100|Greenville|Graham County|NC|21387|United States|-5|condo| +20030|AAAAAAAAODOEAAAA|374|East |Parkway|Suite 230|Bridgeport|Buncombe County|NC|25817|United States|-5|single family| +20031|AAAAAAAAPDOEAAAA|274|College |Cir.|Suite F|Silver Springs|Butler County|MO|64843|United States|-6|apartment| +20032|AAAAAAAAAEOEAAAA|225|Oak |Court|Suite 390|Summit|Moultrie County|IL|60499|United States|-6|apartment| +20033|AAAAAAAABEOEAAAA|512|Center |Lane|Suite C|Valley View|Prairie County|MT|65124|United States|-7|single family| +20034|AAAAAAAACEOEAAAA|197|Highland |Lane|Suite 190|Buena Vista|Parke County|IN|45752|United States|-5|condo| +20035|AAAAAAAADEOEAAAA|||Drive|Suite C|Woodland|||94854|United States|-8|| +20036|AAAAAAAAEEOEAAAA|175|Hill |ST|Suite 350||Essex County|NJ|||-5|condo| +20037|AAAAAAAAFEOEAAAA|412|Dogwood |Circle|Suite 0|Belmont|Christian County|MO|60191|United States|-6|condo| +20038|AAAAAAAAGEOEAAAA|1000|Poplar |Circle|Suite S|Green Acres|Cascade County|MT|67683|United States|-7|apartment| +20039|AAAAAAAAHEOEAAAA|127|Seventh |Circle|Suite 270|Oakwood|Hardee County|FL|30169|United States|-5|apartment| +20040|AAAAAAAAIEOEAAAA|144|Cedar Highland|Pkwy|Suite Q|Murphy|Robertson County|TN|32105|United States|-6|condo| +20041|AAAAAAAAJEOEAAAA||||Suite 80|||KS|69145||-6|apartment| +20042|AAAAAAAAKEOEAAAA|767|2nd |Lane|Suite 170|Enterprise|Clermont County|OH|41757|United States|-5|single family| +20043|AAAAAAAALEOEAAAA|542|Locust |Ct.|Suite 200|Friendship|Guadalupe County|TX|74536|United States|-6|condo| +20044|AAAAAAAAMEOEAAAA|281|||Suite 100|||TN|37683|||| +20045|AAAAAAAANEOEAAAA|314|West |Drive|Suite J|Highland|Martin County|FL|39454|United States|-5|condo| +20046|AAAAAAAAOEOEAAAA|810|Chestnut |Parkway|Suite U|Royal|Jennings County|IN|45819|United States|-5|condo| +20047|AAAAAAAAPEOEAAAA|57|Davis |ST|Suite L|White Oak|Scurry County|TX|76668|United States|-6|condo| +20048|AAAAAAAAAFOEAAAA|782|Second Third|Dr.|Suite W|Mountain View|Madera County|CA|94466|United States|-8|single family| +20049|AAAAAAAABFOEAAAA|86|Hill 12th|Drive|Suite 330|Brownsville|Gove County|KS|69310|United States|-6|apartment| +20050|AAAAAAAACFOEAAAA|5|8th 10th|Cir.|Suite R|Macedonia|Kerr County|TX|71087|United States|-6|condo| +20051|AAAAAAAADFOEAAAA|91|||Suite 450|Sunnyside|Carroll County||71952|||apartment| +20052|AAAAAAAAEFOEAAAA|299|Hillcrest Cherry|Street|Suite 490|Rosebud|Susquehanna County|PA|12244|United States|-5|condo| +20053|AAAAAAAAFFOEAAAA|531|First Pine|Street|Suite 350|Sulphur Springs|Apache County|AZ|88354|United States|-7|condo| +20054|AAAAAAAAGFOEAAAA|856|College 12th|Lane|Suite 490|Sunnyside|La Salle County|TX|71952|United States|-6|single family| +20055|AAAAAAAAHFOEAAAA|570|Lincoln |Court|Suite 100|Plainview|Noxubee County|MS|53683|United States|-6|single family| +20056|AAAAAAAAIFOEAAAA|269|Laurel |Ct.|Suite L|Union Hill|Baldwin County|AL|37746|United States|-6|apartment| +20057|AAAAAAAAJFOEAAAA|459|Sycamore Walnut|Court|Suite 150|Jackson|Cotton County|OK|79583|United States|-6|single family| +20058|AAAAAAAAKFOEAAAA|863|Park |Cir.|Suite G|Highland Park|San Miguel County|CO|86534|United States|-7|single family| +20059|AAAAAAAALFOEAAAA|598|Tenth Hillcrest|Avenue|Suite V|Buena Vista|Cherokee County|GA|35752|United States|-5|single family| +20060|AAAAAAAAMFOEAAAA|343|Church |Ln|Suite 90|Riverview|Valencia County|NM|89003|United States|-7|apartment| +20061|AAAAAAAANFOEAAAA|608|Washington |RD|Suite O|Brunswick|Roane County|WV|24642|United States|-5|condo| +20062|AAAAAAAAOFOEAAAA|784|Sycamore |Boulevard|Suite R|Clearview|Franklin Parish|LA|75495|United States|-6|apartment| +20063|AAAAAAAAPFOEAAAA|810|Hillcrest |RD|Suite 470|Oakdale|Houston County|MN|59584|United States|-6|condo| +20064|AAAAAAAAAGOEAAAA|311|Central |Circle|Suite L|White Oak|Harding County|NM|86668|United States|-7|condo| +20065|AAAAAAAABGOEAAAA|346|Ridge 4th|Ct.|Suite 290|Greenville|Columbia County|FL|31387|United States|-5|condo| +20066|AAAAAAAACGOEAAAA|647|Walnut Second|Avenue|Suite B|Enterprise|Troup County|GA|31757|United States|-5|single family| +20067|AAAAAAAADGOEAAAA|535|Cedar |Boulevard|Suite G|Georgetown|Waldo County|ME|07657|United States|-5|apartment| +20068|AAAAAAAAEGOEAAAA|26|Park 12th|Parkway|Suite L|Concord|Oregon County|MO|64107|United States|-6|single family| +20069|AAAAAAAAFGOEAAAA|295|Madison |Pkwy|Suite H|Clifton|Yancey County|NC|28014|United States|-5|single family| +20070|AAAAAAAAGGOEAAAA|129|4th |Parkway|Suite R|Franklin|Kalamazoo County|MI|49101|United States|-5|condo| +20071|AAAAAAAAHGOEAAAA|431|Lake Park|Court|Suite 370|Bunker Hill|Sherman County|NE|60150|United States|-7|apartment| +20072|AAAAAAAAIGOEAAAA|194|Ash |Pkwy|Suite Y|Franklin|Lawrence County|KY|49101|United States|-5|single family| +20073|AAAAAAAAJGOEAAAA|776|3rd Central|Boulevard|Suite D|White Oak|Harrison County|WV|26668|United States|-5|single family| +20074|AAAAAAAAKGOEAAAA|63||Ct.|||Hardin County||33683|United States||apartment| +20075|AAAAAAAALGOEAAAA|327|Jefferson |Court|Suite K|Newport|Ottawa County|OK|71521|United States|-6|apartment| +20076|AAAAAAAAMGOEAAAA|677|South |Wy|Suite S|Oak Grove|King William County|VA|28370|United States|-5|condo| +20077|AAAAAAAANGOEAAAA|928|Central 3rd|Wy|Suite V|Mount Olive|Waupaca County|WI|58059|United States|-6|single family| +20078|AAAAAAAAOGOEAAAA|380|Pine |Wy|Suite U|Vienna|Barbour County|WV|25119|United States|-5|condo| +20079|AAAAAAAAPGOEAAAA|483|Chestnut Twelfth|Ln|Suite W|Lakewood|Ottawa County|OK|78877|United States|-6|condo| +20080|AAAAAAAAAHOEAAAA|586|Ash |Ave|Suite B|Arlington|Durham County|NC|26557|United States|-5|single family| +20081|AAAAAAAABHOEAAAA|633|Washington |Ln|Suite V|Clinton|Renville County|ND|58222|United States|-6|apartment| +20082|AAAAAAAACHOEAAAA|376|First Oak|Way|Suite P|Summit|Jackson County|GA|30499|United States|-5|apartment| +20083|AAAAAAAADHOEAAAA|451|Eigth |Way|Suite 310|Stringtown|Rusk County|WI|50162|United States|-6|single family| +20084|AAAAAAAAEHOEAAAA|112|Cherry |Parkway|Suite K|Wesley|Umatilla County|OR|91218|United States|-8|apartment| +20085|AAAAAAAAFHOEAAAA|365|Chestnut Elm|Court|Suite H|Belmont|Little River County|AR|70191|United States|-6|condo| +20086|AAAAAAAAGHOEAAAA|435|4th Cedar|||||MN||United States||| +20087|AAAAAAAAHHOEAAAA|255|12th West|Dr.|Suite 490|Lakeview|Upson County|GA|38579|United States|-5|condo| +20088|AAAAAAAAIHOEAAAA|589|Third West|Street|Suite Y|Walnut Grove|Santa Barbara County|CA|97752|United States|-8|apartment| +20089|AAAAAAAAJHOEAAAA|171|Elm 11th|ST|Suite M|Newtown|Baxter County|AR|71749|United States|-6|condo| +20090|AAAAAAAAKHOEAAAA|475|Center |Cir.|Suite 170|White Oak|Warren County|MS|56668|United States|-6|apartment| +20091|AAAAAAAALHOEAAAA|531|13th Dogwood|Wy|Suite D|Parkwood|Madison County|GA|31669|United States|-5|condo| +20092|AAAAAAAAMHOEAAAA|155|14th |Ct.|Suite 350|Oakdale|Clark County|IL|69584|United States|-6|single family| +20093|AAAAAAAANHOEAAAA|173|Hickory |Ave|Suite 350|Riverside|Aleutians West Census Area|AK|99231|United States|-9|apartment| +20094|AAAAAAAAOHOEAAAA|528|1st |Court|Suite B|Ashland|Washington County|MD|24244|United States|-5|single family| +20095|AAAAAAAAPHOEAAAA|318|Lincoln Fourth|Dr.|Suite H|Mountain View|Logan County|KY|44466|United States|-5|condo| +20096|AAAAAAAAAIOEAAAA|754|Fourth First|Dr.|Suite E|Macedonia|McCormick County|SC|21087|United States|-5|condo| +20097|AAAAAAAABIOEAAAA|662|Broadway |Ct.|Suite L|Harvey|Emmons County|ND|55858|United States|-6|apartment| +20098|AAAAAAAACIOEAAAA|349|Park |Court|Suite 280|Wilson|Butler County|AL|36971|United States|-6|single family| +20099|AAAAAAAADIOEAAAA|645|Jackson Washington|Circle|Suite A|Red Hill|Polk County|OR|94338|United States|-8|condo| +20100|AAAAAAAAEIOEAAAA|391|Broadway |Ct.|Suite 400|Highland|Moore County|TN|39454|United States|-6|condo| +20101|AAAAAAAAFIOEAAAA|||Way||Marion|Cimarron County|OK||||single family| +20102|AAAAAAAAGIOEAAAA|708|Lakeview 3rd|Lane|Suite B|Kingston|Mason County|WA|94975|United States|-8|single family| +20103|AAAAAAAAHIOEAAAA|543|Maple 11th|ST|Suite 340|White Oak|Madison County|ID|86668|United States|-7|single family| +20104|AAAAAAAAIIOEAAAA|982|Washington |Wy|Suite 490|Woodlawn|Brooke County|WV|24098|United States|-5|apartment| +20105|AAAAAAAAJIOEAAAA|441|13th Miller|RD|Suite L|Ashland|Middlesex County|CT|04844|United States|-5|apartment| +20106|AAAAAAAAKIOEAAAA|613|Hill Church|Lane|Suite 440|Georgetown|Blaine County|MT|67057|United States|-7|condo| +20107|AAAAAAAALIOEAAAA|481|Main |Street|Suite W|Kingston|White County|IN|44975|United States|-5|apartment| +20108|AAAAAAAAMIOEAAAA|450|Laurel College|Dr.|Suite 80|Maple Grove|Wayne County|MI|48252|United States|-5|single family| +20109|AAAAAAAANIOEAAAA|558|Dogwood |Blvd|Suite 230|Stringtown|Pope County|AR|70162|United States|-6|single family| +20110|AAAAAAAAOIOEAAAA|785|Sycamore Jackson|Avenue|Suite 450|Green Acres|Clay County|TN|37683|United States|-5|single family| +20111|AAAAAAAAPIOEAAAA|17|Highland Fifth|Lane|Suite W|Woodland|Dallas County|TX|74854|United States|-6|single family| +20112|AAAAAAAAAJOEAAAA|546|Lake Spring|Street|Suite C|Philadelphia|Cheyenne County|KS|65591|United States|-6|condo| +20113|AAAAAAAABJOEAAAA|429|Smith Railroad|Cir.|Suite L|Harmony|Stephens County|TX|75804|United States|-6|condo| +20114|AAAAAAAACJOEAAAA|879|8th 7th|Boulevard|Suite 390|New Hope|Moultrie County|IL|69431|United States|-6|condo| +20115|AAAAAAAADJOEAAAA|763|2nd |Lane|Suite 390|Plainview|Chester County|PA|13683|United States|-5|apartment| +20116|AAAAAAAAEJOEAAAA|505|Park Park|Lane|Suite R|Friendship|Haakon County|SD|54536|United States|-7|apartment| +20117|AAAAAAAAFJOEAAAA|||||Mount Olive|Bell County|TX|78059|||| +20118|AAAAAAAAGJOEAAAA|684|Sycamore |Avenue|Suite 20|Unionville|Quitman County|MS|51711|United States|-6|apartment| +20119|AAAAAAAAHJOEAAAA|432|Park Hickory|Street|Suite 110|Midway|Wheeler County|TX|71904|United States|-6|condo| +20120|AAAAAAAAIJOEAAAA|726|Ash Lincoln|Road|Suite S|Waterloo|Bernalillo County|NM|81675|United States|-7|apartment| +20121|AAAAAAAAJJOEAAAA|867|2nd |Blvd|Suite 60|Summit|Berrien County|MI|40499|United States|-5|apartment| +20122|AAAAAAAAKJOEAAAA|262|9th Dogwood|Blvd|Suite F|Shiloh|Dickinson County|IA|59275|United States|-6|single family| +20123|AAAAAAAALJOEAAAA|665|5th |Circle|Suite Y|Riverside|Windham County|CT|09831|United States|-5|condo| +20124|AAAAAAAAMJOEAAAA|420|Mill Lake|Ave|Suite E|Liberty|Washington County|ID|83451|United States|-7|condo| +20125|AAAAAAAANJOEAAAA|393|Elm 3rd|Road|Suite 40|Cedar Grove|Marshall County|MS|50411|United States|-6|apartment| +20126|AAAAAAAAOJOEAAAA|276|Third |Cir.|Suite Y|Concord|Colbert County|AL|34107|United States|-6|single family| +20127|AAAAAAAAPJOEAAAA|986|Poplar |Court|Suite 250|Pleasant Hill|Whitley County|KY|43604|United States|-5|single family| +20128|AAAAAAAAAKOEAAAA|237|Oak |Ln|Suite 250|Stringtown|Greene County|TN|30162|United States|-5|single family| +20129|AAAAAAAABKOEAAAA|217|Elm |ST|Suite 160|Marion|Yolo County|CA|90399|United States|-8|single family| +20130|AAAAAAAACKOEAAAA|164|View |RD|Suite 230|Brownsville|Davis County|IA|59310|United States|-6|condo| +20131|AAAAAAAADKOEAAAA|131|Park |Wy|Suite 260|Greenwood|Grant County|KY|48828|United States|-6|single family| +20132|AAAAAAAAEKOEAAAA|339|Willow |Way|Suite 330|New Hope|Moultrie County|IL|69431|United States|-6|apartment| +20133|AAAAAAAAFKOEAAAA|417|Pine 2nd|Circle|Suite J|Hillcrest|Lowndes County|MS|53003|United States|-6|single family| +20134|AAAAAAAAGKOEAAAA|720|Oak |Boulevard|Suite L|Kingston|Iowa County|WI|54975|United States|-6|apartment| +20135|AAAAAAAAHKOEAAAA|394|1st |Blvd|Suite R|Arlington|Cherokee County|KS|66557|United States|-6|apartment| +20136|AAAAAAAAIKOEAAAA|129|Spring North|RD|Suite U|New Town|Spokane County|WA|99634|United States|-8|condo| +20137|AAAAAAAAJKOEAAAA|828|Hillcrest |ST|Suite R|Shiloh|Davis County|UT|89275|United States|-7|apartment| +20138|AAAAAAAAKKOEAAAA|610|West |Dr.|Suite V|Crossroads|Austin County|TX|70534|United States|-6|single family| +20139|AAAAAAAALKOEAAAA|377|Main Willow|RD|Suite 250|Waterloo|Winston County|MS|51675|United States|-6|condo| +20140|AAAAAAAAMKOEAAAA|296|Jefferson |RD|Suite 370|Springdale|Coryell County|TX|78883|United States|-6|condo| +20141|AAAAAAAANKOEAAAA|304|Fourth Adams|Wy|Suite F|Greenwood|Bethel Census Area|AK|98828|United States|-9|single family| +20142|AAAAAAAAOKOEAAAA|802|Pine |Ct.|Suite O|Woodland|Grant County|KS|64854|United States|-6|single family| +20143|AAAAAAAAPKOEAAAA|401|Mill |Pkwy|Suite E|Mountain View|Tuolumne County|CA|94466|United States|-8|apartment| +20144|AAAAAAAAALOEAAAA|2|Sixth 9th|Court|Suite 370|Pleasant Hill|Meigs County|TN|33604|United States|-6|apartment| +20145|AAAAAAAABLOEAAAA|844|Park |Wy|||||||-5|single family| +20146|AAAAAAAACLOEAAAA|590|Hickory |Road|Suite 60|Wildwood|Holt County|MO|66871|United States|-6|single family| +20147|AAAAAAAADLOEAAAA|676|Hill 8th|Way|Suite 50|Forest Hills|Madison County|AL|39237|United States|-6|single family| +20148|AAAAAAAAELOEAAAA|669|Oak |Cir.|Suite V|Bloomingdale|Kearney County|NE|61824|United States|-7|single family| +20149|AAAAAAAAFLOEAAAA|337|Pine 11th|Blvd|Suite 480|Bethel|Clay County|GA|35281|United States|-5|condo| +20150|AAAAAAAAGLOEAAAA|999|View |Ln|Suite 170|Shady Grove|Miller County|AR|72812|United States|-6|condo| +20151|AAAAAAAAHLOEAAAA|537|Lincoln Poplar|Blvd|Suite 0|Bunker Hill|Lyon County|KS|60150|United States|-6|condo| +20152|AAAAAAAAILOEAAAA|869|1st |Blvd|Suite X|Deerfield|Stephens County|OK|79840|United States|-6|single family| +20153|AAAAAAAAJLOEAAAA|342|West |Blvd|Suite I|Friendship|Bond County|IL|64536|United States|-6|single family| +20154|AAAAAAAAKLOEAAAA|541|Miller Main|RD|Suite D|Friendship|Frio County|TX|74536|United States|-6|apartment| +20155|AAAAAAAALLOEAAAA|20|13th |ST|Suite O|Texas|Kenton County|KY|43342|United States|-5|single family| +20156|AAAAAAAAMLOEAAAA|589|Mill Highland|Cir.|Suite 130|Riverview|Edgecombe County|NC|29003|United States|-5|condo| +20157|AAAAAAAANLOEAAAA|423|Oak |Boulevard|Suite 480|Wilson|Kingman County|KS|66971|United States|-6|condo| +20158|AAAAAAAAOLOEAAAA||Hill |Dr.|Suite 480||Vanderburgh County||48222|United States||| +20159|AAAAAAAAPLOEAAAA|888|Smith Walnut|Street|Suite 300|Five Points|Pitkin County|CO|86098|United States|-7|single family| +20160|AAAAAAAAAMOEAAAA|169|Williams River|Blvd|Suite X|Jamestown|Anderson County|KY|46867|United States|-6|condo| +20161|AAAAAAAABMOEAAAA||Forest |RD|Suite F|||||United States||| +20162|AAAAAAAACMOEAAAA|974|Madison |Pkwy|Suite 50|Newport|Martin County|TX|71521|United States|-6|condo| +20163|AAAAAAAADMOEAAAA|404|Spruce |Street|Suite 170|Kingston|Caroline County|VA|24975|United States|-5|apartment| +20164|AAAAAAAAEMOEAAAA|707|Adams |Dr.|Suite C|Glendale|Roosevelt County|NM|83951|United States|-7|condo| +20165|AAAAAAAAFMOEAAAA|589|2nd |Avenue|Suite 50|White Oak|Morrow County|OH|46668|United States|-5|condo| +20166|AAAAAAAAGMOEAAAA|692|First Central|Ct.|Suite 110|Ashland|Fairfax County|VA|24244|United States|-5|condo| +20167|AAAAAAAAHMOEAAAA|690|4th Fifth|Ct.|Suite 390|Hillcrest|Spalding County|GA|33003|United States|-5|condo| +20168|AAAAAAAAIMOEAAAA|341|9th Wilson|Wy|Suite Q|Hopewell|Colusa County|CA|90587|United States|-8|condo| +20169|AAAAAAAAJMOEAAAA|161|First Center|Drive|Suite 240|Friendship|Sumner County|TN|34536|United States|-6|condo| +20170|AAAAAAAAKMOEAAAA|435|Spring |Parkway|Suite 470|Greenwood|McLean County|ND|58828|United States|-6|single family| +20171|AAAAAAAALMOEAAAA|545|4th Chestnut|Court|Suite C|Harmony|Cheshire County|NH|06404|United States|-5|condo| +20172|AAAAAAAAMMOEAAAA|115|Park |Road|Suite 50|Louisville|Navarro County|TX|74464|United States|-6|condo| +20173|AAAAAAAANMOEAAAA|940|Fourth |Wy|Suite Y|Woodlawn|Iron County|WI|54098|United States|-6|apartment| +20174|AAAAAAAAOMOEAAAA|413|2nd North|Lane|Suite 130|Greenville|Wasco County|OR|91387|United States|-8|condo| +20175|AAAAAAAAPMOEAAAA|818|View 6th|Blvd|Suite K|Macedonia|Haywood County|NC|21087|United States|-5|condo| +20176|AAAAAAAAANOEAAAA|483|Forest |Avenue|Suite E|Woodlawn|Coles County|IL|64098|United States|-6|single family| +20177|AAAAAAAABNOEAAAA|421|Washington 1st|RD|Suite 420|Fairview|Cass County|MI|45709|United States|-5|single family| +20178|AAAAAAAACNOEAAAA|359|3rd 5th|Ln|Suite 450|Hopewell|Wood County|TX|70587|United States|-6|single family| +20179|AAAAAAAADNOEAAAA|318|Main |Pkwy|Suite 90|Pleasant Grove|Scott County|KS|64136|United States|-6|single family| +20180|AAAAAAAAENOEAAAA|240|4th |Circle|Suite 390|Hopewell|Oneida County|WI|50587|United States|-6|single family| +20181|AAAAAAAAFNOEAAAA|355|Davis |Cir.|Suite 180|Greenville|Jennings County|IN|41387|United States|-5|apartment| +20182|AAAAAAAAGNOEAAAA|47|East |Cir.|Suite G|Sugar Hill|Martin County|IN|45114|United States|-5|single family| +20183|AAAAAAAAHNOEAAAA|861|Park |Pkwy|Suite L|Shady Grove|Doniphan County|KS|62812|United States|-6|apartment| +20184|AAAAAAAAINOEAAAA|817|5th |Blvd|Suite O|Bridgeport|Marion County|AL|35817|United States|-6|apartment| +20185|AAAAAAAAJNOEAAAA|719|13th Wilson|Court|Suite 210|Walnut|Rankin County|MS|56245|United States|-6|condo| +20186|AAAAAAAAKNOEAAAA|885|6th |Pkwy|Suite T|Hamilton|Boone County|MO|62808|United States|-6|condo| +20187|AAAAAAAALNOEAAAA|779|Adams |Street|Suite 70|Proctor|Renville County|ND|58140|United States|-6|condo| +20188|AAAAAAAAMNOEAAAA|375|9th |Lane|Suite 290|Mount Zion|Roosevelt County|NM|88054|United States|-7|condo| +20189|AAAAAAAANNOEAAAA|354|Walnut |Way|Suite S|Thompsonville|Ouachita County|AR|79651|United States|-6|single family| +20190|AAAAAAAAONOEAAAA|501|Birch |Lane|Suite 20|Mount Zion|Oliver County|ND|58054|United States|-6|apartment| +20191|AAAAAAAAPNOEAAAA|945|Cedar |Street|Suite 320|Five Points|Lafourche Parish|LA|76098|United States|-6|single family| +20192|AAAAAAAAAOOEAAAA|304|Lee |Circle|Suite 310|Jackson|Hidalgo County|TX|79583|United States|-6|single family| +20193|AAAAAAAABOOEAAAA|153|Willow Valley|Dr.|Suite D|Snug Harbor|Chilton County|AL|37936|United States|-6|condo| +20194|AAAAAAAACOOEAAAA|533|Meadow |ST|Suite H|Oakdale|Columbus County|NC|29584|United States|-5|single family| +20195|AAAAAAAADOOEAAAA|288|Williams Hickory|Ct.|Suite V|Shady Grove|Grant County|KY|42812|United States|-6|single family| +20196|AAAAAAAAEOOEAAAA|13|First |ST|Suite 30|Bunker Hill|Washington County|ME|00750|United States|-5|condo| +20197|AAAAAAAAFOOEAAAA|789|Woodland Lee|Dr.|Suite 50|Evans|Jackson County|IN|42284|United States|-5|single family| +20198|AAAAAAAAGOOEAAAA|825||Lane|Suite W|Oak Grove||CO|88370|United States||| +20199|AAAAAAAAHOOEAAAA|14|Tenth |Road|Suite 100|Providence|Rice County|MN|56614|United States|-6|condo| +20200|AAAAAAAAIOOEAAAA|229|Smith |Wy|Suite J|Clifton|Clermont County|OH|48014|United States|-5|single family| +20201|AAAAAAAAJOOEAAAA|552|Hillcrest |Parkway|Suite 210|Enterprise|Effingham County|GA|31757|United States|-5|apartment| +20202|AAAAAAAAKOOEAAAA|888|13th |Ct.|Suite 160|Friendship|Edwards County|KS|64536|United States|-6|condo| +20203|AAAAAAAALOOEAAAA|55|Sixth |Circle|Suite H|Mount Vernon|Kemper County|MS|58482|United States|-6|apartment| +20204|AAAAAAAAMOOEAAAA|400|Main 11th|Cir.|Suite 90|Highland Park|Marion County|IA|56534|United States|-6|single family| +20205|AAAAAAAANOOEAAAA|743|Laurel |Blvd|Suite 170|Spring Grove|Wilcox County|GA|36719|United States|-5|single family| +20206|AAAAAAAAOOOEAAAA|293|Willow |Avenue|Suite S|Mount Pleasant|Catahoula Parish|LA|71933|United States|-6|apartment| +20207|AAAAAAAAPOOEAAAA|664|7th View|Wy|Suite H|Riverside|Aransas County|TX|79231|United States|-6|apartment| +20208|AAAAAAAAAPOEAAAA|442|Lakeview Park|Blvd|Suite N|Highland|Phillips County|KS|69454|United States|-6|single family| +20209|AAAAAAAABPOEAAAA|489|Mill North|RD|Suite X|Franklin|Foster County|ND|59101|United States|-6|apartment| +20210|AAAAAAAACPOEAAAA|146|Hickory 14th|Ave|Suite 60|Simpson|Missaukee County|MI|49865|United States|-5|apartment| +20211|AAAAAAAADPOEAAAA|181|3rd |Cir.|Suite B|Buena Vista|San Juan County|UT|85752|United States|-7|single family| +20212|AAAAAAAAEPOEAAAA|552|8th West|Avenue|Suite F|Forest|Pulaski County|IN|47537|United States|-5|single family| +20213|AAAAAAAAFPOEAAAA||Pine Wilson||Suite H|Antioch||||||| +20214|AAAAAAAAGPOEAAAA|827|North 2nd|Cir.|Suite O|Eureka|Pike County|IN|45990|United States|-5|single family| +20215|AAAAAAAAHPOEAAAA|544|9th Williams|Blvd|Suite A|Union Hill|Richmond County|NY|17746|United States|-5|condo| +20216|AAAAAAAAIPOEAAAA|596||||Woodbine|Gratiot County||44253||-5|condo| +20217|AAAAAAAAJPOEAAAA|511|Railroad |Lane|Suite 490|Midway|Houston County|GA|31904|United States|-5|apartment| +20218|AAAAAAAAKPOEAAAA|179|Chestnut 3rd|Lane|Suite O|Arlington|Elkhart County|IN|46557|United States|-5|condo| +20219|AAAAAAAALPOEAAAA|935|Lee Meadow|Boulevard|Suite M|Buena Vista|Richland County|SC|25752|United States|-5|single family| +20220|AAAAAAAAMPOEAAAA|11|Maple |Avenue|Suite D|Deerfield|Mayes County|OK|79840|United States|-6|single family| +20221|AAAAAAAANPOEAAAA|651|Chestnut Birch|Lane|Suite Y|Plainview|Morgan County|IL|63683|United States|-6|apartment| +20222|AAAAAAAAOPOEAAAA|593|3rd |Street|Suite 470|Somerville|Erath County|TX|77783|United States|-6|condo| +20223|AAAAAAAAPPOEAAAA|530|11th 2nd|Parkway|Suite R|Pleasant Valley|Humphreys County|MS|52477|United States|-6|single family| +20224|AAAAAAAAAAPEAAAA|612|Cedar |Ct.|Suite O|Lebanon|Wayne County|WV|22898|United States|-5|single family| +20225|AAAAAAAABAPEAAAA|600|15th Spruce|RD|Suite W|Clifton|Union County|NM|88014|United States|-7|single family| +20226|AAAAAAAACAPEAAAA|237|Williams Madison|Blvd|Suite A|Shady Grove|Crawford County|IN|42812|United States|-5|apartment| +20227|AAAAAAAADAPEAAAA|48|Railroad 5th|Cir.|Suite W|New Hope|Pamlico County|NC|29431|United States|-5|single family| +20228|AAAAAAAAEAPEAAAA|802|Railroad |Way|Suite S|Green Acres|Wilkinson County|MS|57683|United States|-6|condo| +20229|AAAAAAAAFAPEAAAA|559|First Green|Wy|Suite 460|Union Hill|Mason County|TX|77746|United States|-6|apartment| +20230|AAAAAAAAGAPEAAAA|110|2nd |Ln|Suite N|Summit|Cuyahoga County|OH|40499|United States|-5|single family| +20231|AAAAAAAAHAPEAAAA|351|7th |Avenue|Suite 310|Oak Grove|Smith County|TN|38370|United States|-6|apartment| +20232|AAAAAAAAIAPEAAAA|511|Central Ridge|Lane|Suite 90|Sunnyside|Richmond County|NC|21952|United States|-5|single family| +20233|AAAAAAAAJAPEAAAA|509|South |Way|Suite 240|Pleasant Hill|Pend Oreille County|WA|93604|United States|-8|condo| +20234|AAAAAAAAKAPEAAAA|970|Railroad |Dr.|Suite F|Perkins|Stone County|AR|71852|United States|-6|single family| +20235|AAAAAAAALAPEAAAA|543|Main |Pkwy|Suite 330|Concord|Floyd County|IN|44107|United States|-5|apartment| +20236|AAAAAAAAMAPEAAAA|561|Ash |Street|Suite B|Spring Valley|Ellis County|OK|76060|United States|-6|apartment| +20237|AAAAAAAANAPEAAAA|66|Church 11th|RD|Suite 440|New Hope|Gray County|TX|79431|United States|-6|single family| +20238|AAAAAAAAOAPEAAAA|367|Seventh |Drive|Suite 210|Shady Grove|Kane County|UT|82812|United States|-7|single family| +20239|AAAAAAAAPAPEAAAA|203|Pine Hillcrest|Ave|Suite 380|Sullivan|Cobb County|GA|30451|United States|-5|apartment| +20240|AAAAAAAAABPEAAAA|582|Oak 4th|Road|Suite F|Bridgeport|Denton County|TX|75817|United States|-6|single family| +20241|AAAAAAAABBPEAAAA|638|Lake |Blvd|Suite 390|Midway|Monroe County|IL|61904|United States|-6|single family| +20242|AAAAAAAACBPEAAAA|596|Central Johnson|Blvd|Suite P|Crystal|Sublette County|WY|85258|United States|-7|single family| +20243|AAAAAAAADBPEAAAA|764|11th |Blvd|Suite W|Fairview|Stephenson County|IL|65709|United States|-6|single family| +20244|AAAAAAAAEBPEAAAA|820|2nd |Court|Suite M|Rutland|Pulaski County|KY|48375|United States|-5|condo| +20245|AAAAAAAAFBPEAAAA|805|Ninth Willow|Court|Suite Y|Sulphur Springs|Davison County|SD|58354|United States|-6|apartment| +20246|AAAAAAAAGBPEAAAA|554|6th |Cir.|Suite 370|Oak Ridge|Daviess County|MO|68371|United States|-6|apartment| +20247|AAAAAAAAHBPEAAAA|142|East Pine|Ln|Suite H|Mount Vernon|Clinton County|PA|18482|United States|-5|apartment| +20248|AAAAAAAAIBPEAAAA|348|Elevnth |Way|Suite L|New Town|Manitowoc County|WI|59634|United States|-6|apartment| +20249|AAAAAAAAJBPEAAAA|213|First |Circle|Suite 40|Stewart|Ottawa County|OK|78041|United States|-6|condo| +20250|AAAAAAAAKBPEAAAA|706|14th Hill|Court|Suite J|Georgetown|Butler County|KS|67057|United States|-6|condo| +20251|AAAAAAAALBPEAAAA|666|Ash Williams|Ave|Suite 380|Riverview|Jennings County|IN|49003|United States|-5|single family| +20252|AAAAAAAAMBPEAAAA|783|1st View|Avenue|Suite 150|Spring Valley|Dawes County|NE|66060|United States|-6|single family| +20253|AAAAAAAANBPEAAAA|354|2nd 8th|Court|Suite G|Farmington|Buffalo County|NE|69145|United States|-6|condo| +20254|AAAAAAAAOBPEAAAA|61|Hill Park|Ln|Suite 450|Lebanon|Kiowa County|CO|82898|United States|-7|single family| +20255|AAAAAAAAPBPEAAAA|735|Main Third|Wy|Suite J|Arlington|Bond County|IL|66557|United States|-6|single family| +20256|AAAAAAAAACPEAAAA|332|Spring Maple|Pkwy|Suite 200|Highland Park|Henderson County|NC|26534|United States|-5|single family| +20257|AAAAAAAABCPEAAAA|951|Second Mill|Wy|Suite B|Bridgeport|Lincoln County|ID|85817|United States|-7|apartment| +20258|AAAAAAAACCPEAAAA|157|10th Park|Wy|Suite 0|Glenwood|Phillips County|KS|63511|United States|-6|condo| +20259|AAAAAAAADCPEAAAA|442|13th Oak|Blvd|Suite 80|Clifford|Furnas County|NE|68164|United States|-6|condo| +20260|AAAAAAAAECPEAAAA|290|River |Parkway|Suite Y|Vienna|Nacogdoches County|TX|75119|United States|-6|condo| +20261|AAAAAAAAFCPEAAAA|814|6th |Pkwy|Suite 350|Concord|Union County|FL|34107|United States|-5|single family| +20262|AAAAAAAAGCPEAAAA|211|Maple 6th|RD|Suite P|Riverdale|King George County|VA|29391|United States|-5|apartment| +20263|AAAAAAAAHCPEAAAA|678|Ridge Main|Pkwy|Suite G|Clifton|Glacier County|MT|68014|United States|-7|apartment| +20264|AAAAAAAAICPEAAAA|507|Park West|RD|Suite 260|Lewisburg|Cibola County|NM|87538|United States|-7|single family| +20265|AAAAAAAAJCPEAAAA|449|Main |Avenue|Suite J|New Hope|Bradley County|TN|39431|United States|-5|condo| +20266|AAAAAAAAKCPEAAAA|198|1st |Cir.|Suite X|Buena Vista|Madison County|IL|65752|United States|-6|apartment| +20267|AAAAAAAALCPEAAAA|782|Walnut Lake|ST|Suite 250|Fairview|Mono County|CA|95709|United States|-8|single family| +20268|AAAAAAAAMCPEAAAA|505|Ridge |Ln|Suite F|Walnut Grove|Madison County|IL|67752|United States|-6|single family| +20269|AAAAAAAANCPEAAAA|||||Mount Vernon|Lyman County|SD|||-7|| +20270|AAAAAAAAOCPEAAAA|51|Center |Road|Suite O|Salem|Stevens County|KS|68048|United States|-6|condo| +20271|AAAAAAAAPCPEAAAA|57|Cherry River|Ct.|Suite J|Pleasant Valley|Hamilton County|TN|32477|United States|-5|apartment| +20272|AAAAAAAAADPEAAAA|375|Seventh |Pkwy|Suite 360|Red Hill|Clay County|AR|74338|United States|-6|condo| +20273|AAAAAAAABDPEAAAA|701|Oak 8th|Ave|Suite 380|Wilson|Galveston County|TX|76971|United States|-6|apartment| +20274|AAAAAAAACDPEAAAA|536|Madison Willow|Street|Suite B|Bethel|Highland County|OH|45281|United States|-5|apartment| +20275|AAAAAAAADDPEAAAA|749|4th Park|Cir.|Suite 390|Pine Grove|Coweta County|GA|34593|United States|-5|apartment| +20276|AAAAAAAAEDPEAAAA|||Court|Suite 280||||40534|United States||apartment| +20277|AAAAAAAAFDPEAAAA|253|Lake |Street|Suite 100|Spring Hill|Randolph County|MO|66787|United States|-6|condo| +20278|AAAAAAAAGDPEAAAA|32|13th |Lane|Suite 250|Roseville|Sequoyah County|OK|72139|United States|-6|apartment| +20279|AAAAAAAAHDPEAAAA|699|Fourth View|Court|Suite 110|Georgetown|Jefferson County|AR|77057|United States|-6|single family| +20280|AAAAAAAAIDPEAAAA|992|Central |Ln|Suite 60|Spring Valley|Glenn County|CA|96060|United States|-8|condo| +20281|AAAAAAAAJDPEAAAA|766|Center Center|Road|Suite F|Mount Zion|Wayne County|NC|28054|United States|-5|single family| +20282|AAAAAAAAKDPEAAAA|423|Tenth Mill|Street|Suite 0|Belleville|Phillips County|MT|62924|United States|-7|condo| +20283|AAAAAAAALDPEAAAA|332|Locust |Road|Suite Q|Green Acres|Johnston County|OK|77683|United States|-6|condo| +20284|AAAAAAAAMDPEAAAA|882|5th |Wy|Suite M|Woodville|Neosho County|KS|64289|United States|-6|condo| +20285|AAAAAAAANDPEAAAA|596|View 6th|Circle|Suite 280|Highland Park|Stark County|ND|56534|United States|-6|apartment| +20286|AAAAAAAAODPEAAAA|87|10th |Cir.|Suite 170|Denmark|Kenosha County|WI|55576|United States|-6|single family| +20287|AAAAAAAAPDPEAAAA|408|6th |Cir.|Suite 230|Walnut Grove|Worth County|IA|57752|United States|-6|condo| +20288|AAAAAAAAAEPEAAAA|830|Highland |Way|Suite J|Oakdale|Lake County|IL|69584|United States|-6|single family| +20289|AAAAAAAABEPEAAAA|428|Valley |Drive|Suite 480|Tabor|White County|AR|78529|United States|-6|condo| +20290|AAAAAAAACEPEAAAA|85|Maple 5th|Circle|Suite 100|Enterprise|Candler County|GA|31757|United States|-5|single family| +20291|AAAAAAAADEPEAAAA|519|2nd |Ln|Suite 20|Oak Grove|Webster County|MS|58370|United States|-6|condo| +20292|AAAAAAAAEEPEAAAA|151|Pine |Boulevard|Suite 100|Arlington|Mercer County|NJ|07157|United States|-5|apartment| +20293|AAAAAAAAFEPEAAAA|943|Washington |Circle|Suite 420|Spring Hill|Chautauqua County|KS|66787|United States|-6|apartment| +20294|AAAAAAAAGEPEAAAA|162|Seventh |Blvd|Suite 390|Salem|Talbot County|GA|38048|United States|-5|single family| +20295|AAAAAAAAHEPEAAAA|96|Valley |Parkway|Suite S|Riverview|Nacogdoches County|TX|79003|United States|-6|apartment| +20296|AAAAAAAAIEPEAAAA|266|Lee Davis|Ave|Suite 430|Buena Vista|San Bernardino County|CA|95752|United States|-8|single family| +20297|AAAAAAAAJEPEAAAA||Hickory Willow||||Silver Bow County|MT|69840||-7|| +20298|AAAAAAAAKEPEAAAA|912|Williams Park|Parkway|Suite 110|Valley View|Slope County|ND|55124|United States|-6|single family| +20299|AAAAAAAALEPEAAAA|723|Maple |Street|Suite H|Macedonia|Buncombe County|NC|21087|United States|-5|single family| +20300|AAAAAAAAMEPEAAAA|256|Willow Forest|Street|Suite O|Leesville|Webb County|TX|75423|United States|-6|single family| +20301|AAAAAAAANEPEAAAA|87|Park |Ave|Suite O|Westminster|Coconino County|AZ|86549|United States|-7|apartment| +20302|AAAAAAAAOEPEAAAA|948|Hill 6th|Blvd|Suite 410|Greenwood|Franklin County|NC|28828|United States|-5|apartment| +20303|AAAAAAAAPEPEAAAA|457|Davis Washington|Wy|Suite B|Maple Grove|McCreary County|KY|48252|United States|-5|apartment| +20304|AAAAAAAAAFPEAAAA|248|Park |Circle|Suite T|Belmont|Custer County|NE|60191|United States|-6|apartment| +20305|AAAAAAAABFPEAAAA|571|Meadow Walnut|Lane|Suite N|Newport|Castro County|TX|71521|United States|-6|condo| +20306|AAAAAAAACFPEAAAA|722|Hickory Poplar|Wy|Suite B||Ottawa County|KS|69003|United States||single family| +20307|AAAAAAAADFPEAAAA|612|1st 4th|Way|Suite Q|Oak Grove|Hamlin County|SD|58370|United States|-7|single family| +20308|AAAAAAAAEFPEAAAA|699|Second Oak|Parkway|Suite 340|Woodlawn|Grundy County|IL|64098|United States|-6|single family| +20309|AAAAAAAAFFPEAAAA|824|Willow |Cir.|Suite D|Union Hill|Nye County|NV|87746|United States|-8|condo| +20310|AAAAAAAAGFPEAAAA|391|Mill Tenth|Ct.|Suite 480|Clinton|Marion County|OH|48222|United States|-5|single family| +20311|AAAAAAAAHFPEAAAA|62|4th Highland|Ave|Suite I|Centerville|Richardson County|NE|60059|United States|-7|apartment| +20312|AAAAAAAAIFPEAAAA|524|1st |Court|Suite 0|Collinsville|Morgan County|AL|32459|United States|-6|apartment| +20313|AAAAAAAAJFPEAAAA|787|Woodland South|Lane|Suite 40|Fairview|Shelby County|TN|35709|United States|-6|apartment| +20314|AAAAAAAAKFPEAAAA|653|View |||Stringtown||NC||United States|-5|| +20315|AAAAAAAALFPEAAAA|212|Lake |Wy|Suite 0|Woodland|San Augustine County|TX|74854|United States|-6|single family| +20316|AAAAAAAAMFPEAAAA|753|Lincoln Mill|Wy|Suite 280|Antioch|Runnels County|TX|78605|United States|-6|condo| +20317|AAAAAAAANFPEAAAA|667|3rd Valley|Drive|Suite 230|Woodlawn|Barnwell County|SC|24098|United States|-5|single family| +20318|AAAAAAAAOFPEAAAA|782|6th Maple|Blvd|Suite U|Bayview|Greene County|AR|79672|United States|-6|apartment| +20319|AAAAAAAAPFPEAAAA|769|Valley |Lane|Suite 0|Lewis|Limestone County|AL|37066|United States|-6|condo| +20320|AAAAAAAAAGPEAAAA|50|Park |Ln|Suite P|Providence|Wabaunsee County|KS|66614|United States|-6|apartment| +20321|AAAAAAAABGPEAAAA|677|Hill |Circle|Suite W|Spring Hill|Magoffin County|KY|46787|United States|-5|apartment| +20322|AAAAAAAACGPEAAAA|820|Highland Main|Street|Suite L|Spring Hill|Gage County|NE|66787|United States|-6|condo| +20323|AAAAAAAADGPEAAAA|583|North 10th|Road|Suite O|Mount Pleasant|Sioux County|ND|51933|United States|-6|apartment| +20324|AAAAAAAAEGPEAAAA|893|West |Avenue|Suite G|Green Acres|Jackson County|OH|47683|United States|-5|condo| +20325|AAAAAAAAFGPEAAAA|724|Johnson |Circle|Suite 140|Harmony|Lowndes County|GA|35804|United States|-5|condo| +20326|AAAAAAAAGGPEAAAA|561|Miller Valley|Circle|Suite M|Summit|Hampden County|MA|01099|United States|-5|apartment| +20327|AAAAAAAAHGPEAAAA|881|12th Lake|Pkwy|Suite F|Summit|Shelby County|IA|50499|United States|-6|condo| +20328|AAAAAAAAIGPEAAAA|602|15th 3rd|Court|Suite F|Waterloo|Logan County|KY|41675|United States|-5|apartment| +20329|AAAAAAAAJGPEAAAA|105|Second |Blvd|Suite I|Union Hill|Richmond city|VA|27746|United States|-5|condo| +20330|AAAAAAAAKGPEAAAA|823|North 10th|Blvd|Suite K|Summit|Grundy County|IL|60499|United States|-6|single family| +20331|AAAAAAAALGPEAAAA|742||Lane||||FL|30150|United States||condo| +20332|AAAAAAAAMGPEAAAA|363|6th Walnut|Boulevard|Suite 210|Macedonia|Gove County|KS|61087|United States|-6|condo| +20333|AAAAAAAANGPEAAAA|691|5th |Ave|Suite 210|Woodlawn|Cherokee County|SC|24098|United States|-5|single family| +20334|AAAAAAAAOGPEAAAA|278|Church Laurel|Wy|Suite T|Hillcrest|Jefferson County|FL|33003|United States|-5|condo| +20335|AAAAAAAAPGPEAAAA|251|Church |Drive|Suite 260|Enterprise|Bucks County|PA|11757|United States|-5|apartment| +20336|AAAAAAAAAHPEAAAA|766|Washington Central|ST|Suite X|Five Forks|Worth County|MO|62293|United States|-6|single family| +20337|AAAAAAAABHPEAAAA|188|Mill River|Court|Suite E|Mount Zion|San Diego County|CA|98054|United States|-8|single family| +20338|AAAAAAAACHPEAAAA|325|Cedar |Drive|Suite 490|Arlington|Gilpin County|CO|86557|United States|-7|single family| +20339|AAAAAAAADHPEAAAA|329|Highland |Way|Suite 280|Bunker Hill|Fayette County|AL|30150|United States|-6|single family| +20340|AAAAAAAAEHPEAAAA|848|10th Meadow|Pkwy|Suite T|Buena Vista|Massac County|IL|65752|United States|-6|single family| +20341|AAAAAAAAFHPEAAAA|653|Maple |Dr.|Suite 10|Glenville|Wayne County|UT|83445|United States|-7|apartment| +20342|AAAAAAAAGHPEAAAA|327|Hill 3rd|Ave|Suite 100|Belmont|Sitka Borough|AK|90191|United States|-9|apartment| +20343|AAAAAAAAHHPEAAAA|212|11th 7th|Pkwy|Suite A|Richfield|Cabarrus County|NC|26196|United States|-5|single family| +20344|AAAAAAAAIHPEAAAA||South ||Suite 310|Salem|Jessamine County|KY|48048|||apartment| +20345|AAAAAAAAJHPEAAAA|194|4th Ninth|Pkwy|Suite 50|Springfield|Dorchester County|MD|29303|United States|-5|condo| +20346|AAAAAAAAKHPEAAAA|990|Lake Hickory|Street|Suite B|Woodbury|Dawson County|TX|74489|United States|-6|condo| +20347|AAAAAAAALHPEAAAA|985|3rd |Way|Suite P|Fairview|Wayne County|IN|45709|United States|-5|condo| +20348|AAAAAAAAMHPEAAAA|448|Hickory |Dr.|Suite V|Floyd|Tippah County|MS|53235|United States|-6|condo| +20349|AAAAAAAANHPEAAAA|114|Oak |Way|Suite 480|Clearwater|Morgan County|GA|39534|United States|-5|single family| +20350|AAAAAAAAOHPEAAAA|914|Hickory Third|Drive|Suite 300|Jamestown|Iosco County|MI|46867|United States|-5|condo| +20351|AAAAAAAAPHPEAAAA|324|13th |Avenue|Suite Y|Oakwood|Sheboygan County|WI|50169|United States|-6|condo| +20352|AAAAAAAAAIPEAAAA|974|Jefferson Twelfth|Way|Suite H|Shaw|Castro County|TX|70618|United States|-6|apartment| +20353|AAAAAAAABIPEAAAA|413|3rd 14th|RD|Suite C|Liberty|Hitchcock County|NE|63451|United States|-6|condo| +20354|AAAAAAAACIPEAAAA|249|11th |Ave|Suite 190|Marion|Newton County|MO|60399|United States|-6|apartment| +20355|AAAAAAAADIPEAAAA|393|South Poplar|Avenue|Suite T|Hamilton|Forsyth County|GA|32808|United States|-5|single family| +20356|AAAAAAAAEIPEAAAA|665|Poplar |Pkwy|Suite 270|Fairfield|Caroline County|MD|26192|United States|-5|apartment| +20357|AAAAAAAAFIPEAAAA|721|Jefferson |Ln|Suite K|Stringtown|Vernon County|WI|50162|United States|-6|condo| +20358|AAAAAAAAGIPEAAAA|802|Elm |Pkwy|Suite 50|Empire|Treutlen County|GA|34145|United States|-5|single family| +20359|AAAAAAAAHIPEAAAA|130|River 6th|Boulevard|Suite N|Florence|Bacon County|GA|33394|United States|-5|apartment| +20360|AAAAAAAAIIPEAAAA|398|Broadway First|Avenue|Suite T|Gladstone|Wheeler County|OR|90894|United States|-8|apartment| +20361|AAAAAAAAJIPEAAAA|81|Elm Lincoln|Court|Suite C|Fairview|Clay County|AR|75709|United States|-6|condo| +20362|AAAAAAAAKIPEAAAA|51|First |Road|Suite H|Lakeview|Rogers County|OK|78579|United States|-6|single family| +20363|AAAAAAAALIPEAAAA|433|Church |Cir.|Suite N|Lakeside|Beaver County|UT|89532|United States|-7|condo| +20364|AAAAAAAAMIPEAAAA|15|Third |Pkwy|Suite 420|Macedonia|Warren County|PA|11087|United States|-5|condo| +20365|AAAAAAAANIPEAAAA|575|10th North|Pkwy|Suite V|Oak Hill|Morgan County|KY|47838|United States|-5|condo| +20366|AAAAAAAAOIPEAAAA|431|Spring |Lane|Suite 330|Sunnyside|Cobb County|GA|31952|United States|-5|apartment| +20367|AAAAAAAAPIPEAAAA|926|Spring Spruce|Ave|Suite 300|Mount Pleasant|Cherokee County|SC|21933|United States|-5|apartment| +20368|AAAAAAAAAJPEAAAA|93|Main |Drive|Suite 100|Springdale|Jasper County|MO|68883|United States|-6|single family| +20369|AAAAAAAABJPEAAAA|920|South 10th|Road|Suite T|White Oak|Attala County|MS|56668|United States|-6|single family| +20370|AAAAAAAACJPEAAAA|712|Hickory ||Suite L|Liberty|Columbia County||||-5|apartment| +20371|AAAAAAAADJPEAAAA|453|||Suite U||Hancock County|MS|54136||-6|apartment| +20372|AAAAAAAAEJPEAAAA|727|1st |Parkway|Suite 420|Spring Valley|Seneca County|NY|16060|United States|-5|apartment| +20373|AAAAAAAAFJPEAAAA|333|Highland |Blvd|Suite Y|Valley View|Jefferson Parish|LA|75124|United States|-6|condo| +20374|AAAAAAAAGJPEAAAA|292|Jackson Railroad|Drive|Suite 330|Bridgeport|Sweetwater County|WY|85817|United States|-7|condo| +20375|AAAAAAAAHJPEAAAA|787|Center |Avenue|Suite F|Fairfield|Sumter County|SC|26192|United States|-5|single family| +20376|AAAAAAAAIJPEAAAA|211|Washington Main|Boulevard|Suite Q|Highland|Dillon County|SC|29454|United States|-5|apartment| +20377|AAAAAAAAJJPEAAAA|842|3rd Central|Ln|Suite S|Longwood|Lake County|OR|97021|United States|-8|apartment| +20378|AAAAAAAAKJPEAAAA|823|Elm Washington|RD|Suite 20|Macedonia|Lyon County|KY|41087|United States|-5|apartment| +20379|AAAAAAAALJPEAAAA|721|Second Laurel|Lane|Suite X|Glendale|Humphreys County|TN|33951|United States|-5|single family| +20380|AAAAAAAAMJPEAAAA|751|Fourth 13th|Blvd|Suite W|Springdale|Ness County|KS|68883|United States|-6|single family| +20381|AAAAAAAANJPEAAAA|800|West Sixth|Lane|Suite V|Stringtown|Forest County|WI|50162|United States|-6|apartment| +20382|AAAAAAAAOJPEAAAA|809|7th |RD|Suite 50|Clinton|Monona County|IA|58222|United States|-6|single family| +20383|AAAAAAAAPJPEAAAA|862|Franklin 1st|Boulevard|Suite 270|Woodcrest|Lucas County|OH|44919|United States|-5|apartment| +20384|AAAAAAAAAKPEAAAA|314|Central |Avenue|Suite H|Crossroads|Panola County|MS|50534|United States|-6|apartment| +20385|AAAAAAAABKPEAAAA|829|3rd Pine|Cir.|Suite 270|Hamilton|White Pine County|NV|82808|United States|-8|apartment| +20386|AAAAAAAACKPEAAAA|698|Central |Ln|Suite K|Wilson|Travis County|TX|76971|United States|-6|condo| +20387|AAAAAAAADKPEAAAA|912|College Jackson|Avenue|Suite N|Crossroads|Starke County|IN|40534|United States|-5|apartment| +20388|AAAAAAAAEKPEAAAA|299|Oak |RD|Suite 260|Providence|Kenton County|KY|46614|United States|-5|condo| +20389|AAAAAAAAFKPEAAAA|183|Sixth Main|Wy|Suite K|Walnut Grove|Bureau County|IL|67752|United States|-6|apartment| +20390|AAAAAAAAGKPEAAAA|8|Second |Court|Suite 390|Liberty|Lincoln County|MN|53451|United States|-6|apartment| +20391|AAAAAAAAHKPEAAAA|525|South Cherry|Wy|Suite 40|Spring Valley|Sioux County|ND|56060|United States|-6|condo| +20392|AAAAAAAAIKPEAAAA|781|Forest |Lane|Suite 470|Oak Ridge|Palo Pinto County|TX|78371|United States|-6|apartment| +20393|AAAAAAAAJKPEAAAA|118|Franklin |Court|Suite 190|Crossroads|Duval County|FL|30534|United States|-5|apartment| +20394|AAAAAAAAKKPEAAAA|828|Pine Railroad|Boulevard|Suite 410|Clifton|Henry County|IL|68014|United States|-6|condo| +20395|AAAAAAAALKPEAAAA|506|4th |Dr.|Suite 480|Marion|Pike County|IN|40399|United States|-5|single family| +20396|AAAAAAAAMKPEAAAA|282|6th 2nd|RD|Suite S|Spring Valley|Mercer County|IL|66060|United States|-6|condo| +20397|AAAAAAAANKPEAAAA|804|14th |Pkwy|Suite 160|Oakland|Henderson County|TX|79843|United States|-6|condo| +20398|AAAAAAAAOKPEAAAA|983|5th |Street|Suite G|Greenfield|Livingston County|MI|45038|United States|-5|single family| +20399|AAAAAAAAPKPEAAAA|523|Railroad Spring|Avenue|Suite J|Sulphur Springs|Colorado County|TX|78354|United States|-6|condo| +20400|AAAAAAAAALPEAAAA|774|Park |Street|Suite B|Spring Hill|Cerro Gordo County|IA|56787|United States|-6|single family| +20401|AAAAAAAABLPEAAAA|132|1st 2nd|Drive|Suite B|Valley View|Park County|MT|65124|United States|-7|single family| +20402|AAAAAAAACLPEAAAA|383|8th |Wy|Suite 90|Union Hill|Sevier County|UT|87746|United States|-7|apartment| +20403|AAAAAAAADLPEAAAA|308|Chestnut |Ave|Suite F|Maple Grove|Polk County|NE|68252|United States|-7|apartment| +20404|AAAAAAAAELPEAAAA|116|Wilson Railroad|Wy|Suite 450|Farmington|Evans County|GA|39145|United States|-5|condo| +20405|AAAAAAAAFLPEAAAA|1|Eigth Green|Ave|Suite M|Maple Grove|Madison County|TX|78252|United States|-6|condo| +20406|AAAAAAAAGLPEAAAA|185|Willow |Drive|Suite S|Farmington|Union County|OR|99145|United States|-8|apartment| +20407|AAAAAAAAHLPEAAAA|92|View |Way|Suite V|Glenwood|Pinellas County|FL|33511|United States|-5|apartment| +20408|AAAAAAAAILPEAAAA|17|3rd |Way|Suite 330|Glendale|Platte County|WY|83951|United States|-7|condo| +20409|AAAAAAAAJLPEAAAA|149|Chestnut Forest|Pkwy|Suite 0|Waterloo|Drew County|AR|71675|United States|-6|single family| +20410|AAAAAAAAKLPEAAAA|829|4th |Drive|Suite 280|Lakewood|Marshall County|TN|38877|United States|-6|condo| +20411|AAAAAAAALLPEAAAA|163|Mill Hill|Court|Suite 10|Brownsville|Wheeler County|TX|79310|United States|-6|condo| +20412|AAAAAAAAMLPEAAAA|627|Mill Cherry|Cir.|Suite T|Woodville|Bristol County|RI|04889|United States|-5|single family| +20413|AAAAAAAANLPEAAAA|938|Main |Avenue|Suite H|Highland Park|Dodge County|MN|56534|United States|-6|condo| +20414|AAAAAAAAOLPEAAAA|977|7th |Street|Suite S|Newport|Manassas city|VA|21521|United States|-5|apartment| +20415|AAAAAAAAPLPEAAAA|198|Chestnut Main|Circle|Suite H|Jamestown|Navarro County|TX|76867|United States|-6|condo| +20416|AAAAAAAAAMPEAAAA|571|West |Circle|Suite R|Wayne|King and Queen County|VA|24966|United States|-5|apartment| +20417|AAAAAAAABMPEAAAA|695|15th |Road|Suite 370|Mount Olive|Mille Lacs County|MN|58059|United States|-6|single family| +20418|AAAAAAAACMPEAAAA|805|Spruce |Street|Suite N|Barnes|Jefferson County|FL|33788|United States|-5|condo| +20419|AAAAAAAADMPEAAAA|378|5th |Avenue|Suite 60|Georgetown|Cheyenne County|KS|67057|United States|-6|condo| +20420|AAAAAAAAEMPEAAAA|657|Fourth Jefferson|Street|Suite N|Bethel|Price County|WI|55281|United States|-6|apartment| +20421|AAAAAAAAFMPEAAAA|679|Highland Tenth||Suite 460|Shiloh|Hinsdale County|CO||United States|-7|| +20422|AAAAAAAAGMPEAAAA|795|Smith |Ln|Suite Q|Woodland|Zavala County|TX|74854|United States|-6|condo| +20423|AAAAAAAAHMPEAAAA|495|1st |Road|Suite A|White Hall|Franklin County|MS|56955|United States|-6|apartment| +20424|AAAAAAAAIMPEAAAA|660|Lincoln Center|RD|Suite U|Howell|Jones County|NC|24854|United States|-5|condo| +20425|AAAAAAAAJMPEAAAA|76|Cedar Pine|Way|Suite G|Highland Park|Fayette County|TN|36534|United States|-5|condo| +20426|AAAAAAAAKMPEAAAA|750|Sunset 2nd|Circle|Suite D|Pleasant Grove|Cumberland County|PA|14136|United States|-5|condo| +20427|AAAAAAAALMPEAAAA|558|Center Hill|Court|Suite 370|Five Forks|Jefferson County|ID|82293|United States|-7|apartment| +20428|AAAAAAAAMMPEAAAA|80|Main Franklin|Way|Suite Y|Centerville|Lee County|NC|20059|United States|-5|single family| +20429|AAAAAAAANMPEAAAA|652|Maple |Court|Suite W|Riverside|Dickinson County|MI|49231|United States|-5|condo| +20430|AAAAAAAAOMPEAAAA||Adams Park||||||||-5|single family| +20431|AAAAAAAAPMPEAAAA|727|Jackson |Avenue|Suite 30|Lakewood|Leslie County|KY|48877|United States|-5|apartment| +20432|AAAAAAAAANPEAAAA|557|Cedar Lincoln|Boulevard|Suite X|Lebanon|Barry County|MI|42898|United States|-5|apartment| +20433|AAAAAAAABNPEAAAA|460|Pine |ST|Suite I|New Hope|Lafayette County|MS|59431|United States|-6|apartment| +20434|AAAAAAAACNPEAAAA|417|Poplar |Parkway|Suite V|Union|Calcasieu Parish|LA|78721|United States|-6|apartment| +20435|AAAAAAAADNPEAAAA|752|Washington |Circle|Suite W|Crossroads|Belknap County|NH|01134|United States|-5|single family| +20436|AAAAAAAAENPEAAAA|385|Dogwood |Street|Suite I|Lebanon|Summit County|CO|82898|United States|-7|single family| +20437|AAAAAAAAFNPEAAAA|382|Main |Wy|Suite E|Edgewood|Oconee County|GA|30069|United States|-5|condo| +20438|AAAAAAAAGNPEAAAA|711|Jackson |Court|Suite 210|Oak Ridge|Tama County|IA|58371|United States|-6|single family| +20439|AAAAAAAAHNPEAAAA|355|Park |Dr.|Suite 90|Macedonia|Hardin County|OH|41087|United States|-5|apartment| +20440|AAAAAAAAINPEAAAA|191|Ash |Road|Suite 180|Shiloh|Emanuel County|GA|39275|United States|-5|condo| +20441|AAAAAAAAJNPEAAAA|625|Hill Ridge|Wy|Suite 460|Pleasant Valley|Jim Wells County|TX|72477|United States|-6|apartment| +20442|AAAAAAAAKNPEAAAA|427|Lee |Parkway|Suite D|Springdale|Clay County|NE|68883|United States|-6|apartment| +20443|AAAAAAAALNPEAAAA|870|Elm |Way|Suite W|Plainview|Ventura County|CA|93683|United States|-8|single family| +20444|AAAAAAAAMNPEAAAA|491|Franklin Sunset|Way|Suite 10|Crossroads|New Hanover County|NC|20534|United States|-5|condo| +20445|AAAAAAAANNPEAAAA|362|Sixth |Way|Suite S|Oak Grove|Montgomery County|NY|18370|United States|-5|apartment| +20446|AAAAAAAAONPEAAAA|981|Locust Locust|Boulevard|Suite 30|Red Hill|Lincoln County|OK|74338|United States|-6|condo| +20447|AAAAAAAAPNPEAAAA|592|East |Circle|Suite X|Plainview|Pittsburg County|OK|73683|United States|-6|condo| +20448|AAAAAAAAAOPEAAAA|513|Pine Green|Way|Suite O|Wildwood|Wasatch County|UT|86871|United States|-7|apartment| +20449|AAAAAAAABOPEAAAA|839|Fifth Main|Cir.|Suite 180|Oakland|Woods County|OK|79843|United States|-6|apartment| +20450|AAAAAAAACOPEAAAA|554|Park Oak|Drive|Suite A|Antioch|Noble County|OK|78605|United States|-6|single family| +20451|AAAAAAAADOPEAAAA|665|Ridge Center|Ave|Suite M|Oakwood|Monroe County|IL|60169|United States|-6|condo| +20452|AAAAAAAAEOPEAAAA|463|Cedar West|Street|Suite E|Point Pleasant|Marion County|IN|44749|United States|-5|apartment| +20453|AAAAAAAAFOPEAAAA|814|Pine Eigth|Circle|Suite 40|Salem|Ouray County|CO|88048|United States|-7|condo| +20454|AAAAAAAAGOPEAAAA|503|North |Boulevard|Suite 380|Summit|Rosebud County|MT|60499|United States|-7|condo| +20455|AAAAAAAAHOPEAAAA|581|West Cedar||Suite 440|Florence|Cumberland County|PA|||-5|apartment| +20456|AAAAAAAAIOPEAAAA|329|2nd |Avenue|Suite 180|Providence|Polk County|OR|96614|United States|-8|single family| +20457|AAAAAAAAJOPEAAAA|299|Park |Street|Suite 130|Hillsdale|Martinsville city|VA|23054|United States|-5|single family| +20458|AAAAAAAAKOPEAAAA|393|Forest |Blvd||Florence|Yolo County|CA|||-8|condo| +20459|AAAAAAAALOPEAAAA|191|Pine Highland|Court|Suite 210|Oakwood|Hayes County|NE|60169|United States|-6|apartment| +20460|AAAAAAAAMOPEAAAA|33|Sunset |ST|Suite V|Kingston|Jackson County|OH|44975|United States|-5|single family| +20461|AAAAAAAANOPEAAAA|774|Chestnut Locust|Cir.|Suite N|Macedonia|Yellowstone County|MT|61087|United States|-7|condo| +20462|AAAAAAAAOOPEAAAA|433|Spring Franklin|Street|Suite 320|Cedar Grove|Jeff Davis County|GA|30411|United States|-5|single family| +20463|AAAAAAAAPOPEAAAA|465|Sycamore |Dr.|Suite U|Liberty|Mora County|NM|83451|United States|-7|apartment| +20464|AAAAAAAAAPPEAAAA|124|Lake |Ln|Suite 370|Crossroads|Chilton County|AL|30534|United States|-6|apartment| +20465|AAAAAAAABPPEAAAA|158|Highland Elm|Ln|Suite K|Woodland|Pickens County|GA|34854|United States|-5|apartment| +20466|AAAAAAAACPPEAAAA|584|North Cedar|ST|Suite 60|Oakdale|Berks County|PA|19584|United States|-5|single family| +20467|AAAAAAAADPPEAAAA|26|8th River|Ave|Suite G|Forest Hills|Macoupin County|IL|69237|United States|-6|condo| +20468|AAAAAAAAEPPEAAAA|122|Spruce Church|Ct.|Suite 400|Pleasant Valley|Webster County|GA|32477|United States|-5|apartment| +20469|AAAAAAAAFPPEAAAA|101|Birch 11th|Court|Suite 440|Riverdale|Barnstable County|MA|09991|United States|-5|condo| +20470|AAAAAAAAGPPEAAAA|830|10th Valley|Way|Suite 350|New Salem|Crane County|TX|79568|United States|-6|single family| +20471|AAAAAAAAHPPEAAAA|840|Madison Fifth|Lane|Suite 440|Macedonia|Mellette County|SD|51087|United States|-7|apartment| +20472|AAAAAAAAIPPEAAAA|118|Elm |Street|Suite Q|Newtown|Burke County|ND|51749|United States|-6|condo| +20473|AAAAAAAAJPPEAAAA|873|River |Cir.|Suite K|Fairview|Sandusky County|OH|45709|United States|-5|apartment| +20474|AAAAAAAAKPPEAAAA|955|3rd |Dr.|Suite X|Oakland|Carroll County|AR|79843|United States|-6|single family| +20475|AAAAAAAALPPEAAAA|280|Lincoln |Parkway|Suite 450|Highland|San Saba County|TX|79454|United States|-6|apartment| +20476|AAAAAAAAMPPEAAAA|316|3rd |Pkwy|Suite 220|Woodlawn|Sevier County|TN|34098|United States|-6|apartment| +20477|AAAAAAAANPPEAAAA|717|Second 13th|Dr.|Suite 50|Riverdale|Randolph County|AR|79391|United States|-6|condo| +20478|AAAAAAAAOPPEAAAA|805|South Miller|Ave|Suite 470|Waterloo|Lewis County|KY|41675|United States|-5|single family| +20479|AAAAAAAAPPPEAAAA|358|Second Main|Road|Suite 40|Unionville|Orange County|TX|71711|United States|-6|single family| +20480|AAAAAAAAAAAFAAAA|254|Cedar Pine|Court|Suite 100|Sunnyside|Etowah County|AL|31952|United States|-6|condo| +20481|AAAAAAAABAAFAAAA|13|6th Forest|Wy|Suite O|Friendship|Alcorn County|MS|54536|United States|-6|apartment| +20482|AAAAAAAACAAFAAAA|682|Fifth 2nd|Way|Suite W|Salem|Uintah County|UT|88048|United States|-7|single family| +20483|AAAAAAAADAAFAAAA|756|Pine 2nd|ST|Suite C|Crossroads|Beaufort County|NC|20534|United States|-5|apartment| +20484|AAAAAAAAEAAFAAAA|939|Cedar |RD|Suite E|Hillcrest|Prince William County|VA|23003|United States|-5|apartment| +20485|AAAAAAAAFAAFAAAA|654|Birch |Boulevard|Suite Y|Mount Zion|Marshall County|MS|58054|United States|-6|apartment| +20486|AAAAAAAAGAAFAAAA|519|Smith |Lane|Suite Q|Clinton|McLean County|ND|58222|United States|-6|condo| +20487|AAAAAAAAHAAFAAAA|607|Wilson |Ct.|Suite S|Riverview|Sweetwater County|WY|89003|United States|-7|condo| +20488|AAAAAAAAIAAFAAAA|907|Green |Parkway|Suite 280|Pinecrest|Davie County|NC|29981|United States|-5|single family| +20489|AAAAAAAAJAAFAAAA|475|7th 11th|Wy|Suite 490|Jackson|Smith County|MS|59583|United States|-6|condo| +20490|AAAAAAAAKAAFAAAA|703|Sixth |Ln|Suite 80|Lakeside|Ramsey County|ND|59532|United States|-6|single family| +20491|AAAAAAAALAAFAAAA|38|Oak Maple|Wy|Suite 340|Woodland|Clark County|WI|54854|United States|-6|single family| +20492|AAAAAAAAMAAFAAAA|900|2nd |Parkway|Suite 110|Belmont|Forsyth County|NC|20191|United States|-5|condo| +20493|AAAAAAAANAAFAAAA|695|First Fourth|Avenue|Suite 320|Summit|Pocahontas County|WV|20499|United States|-5|apartment| +20494|AAAAAAAAOAAFAAAA|724|5th 14th|Wy|Suite Y|Floyd|Laurens County|SC|23235|United States|-5|apartment| +20495|AAAAAAAAPAAFAAAA|971|Poplar |Street|Suite 230|Liberty|Lincoln County|WI|53451|United States|-6|single family| +20496|AAAAAAAAABAFAAAA|952|Fifth |Way|Suite 160|Valley View|Union Parish|LA|75124|United States|-6|single family| +20497|AAAAAAAABBAFAAAA|235|Lakeview Oak|Court|Suite 60|Lenox||MO|61143|United States||| +20498|AAAAAAAACBAFAAAA|948|College |Boulevard|Suite A|Colfax|Putnam County|OH|42565|United States|-5|single family| +20499|AAAAAAAADBAFAAAA|879|Spruce |Ave|Suite 430|Summit|Mecosta County|MI|40499|United States|-5|condo| +20500|AAAAAAAAEBAFAAAA|110|Elm Mill|ST|Suite 480|Union|Ben Hill County|GA|38721|United States|-5|apartment| +20501|AAAAAAAAFBAFAAAA|54|3rd Fifth|Way|Suite I|Murray|Ouachita County|AR|72150|United States|-6|single family| +20502|AAAAAAAAGBAFAAAA|145|5th |Road|Suite 190|Glenwood|Barbour County|WV|23511|United States|-5|apartment| +20503|AAAAAAAAHBAFAAAA|137|Cedar |Ct.|Suite Y|Waterloo|Ravalli County|MT|61675|United States|-7|single family| +20504|AAAAAAAAIBAFAAAA|714|Valley |Boulevard|Suite 270|Buena Vista|Randolph County|IL|65752|United States|-6|condo| +20505|AAAAAAAAJBAFAAAA|151|Lake |Wy|Suite D|Mount Pleasant|Jasper County|TX|71933|United States|-6|condo| +20506|AAAAAAAAKBAFAAAA|964|Broadway Woodland|Wy|Suite 10|Pine Grove|King George County|VA|24593|United States|-5|condo| +20507|AAAAAAAALBAFAAAA|703|Chestnut |Cir.|Suite 20|Springfield|Iroquois County|IL|69303|United States|-6|condo| +20508|AAAAAAAAMBAFAAAA|964|Jackson Sixth||Suite 140|||GA|||-5|| +20509|AAAAAAAANBAFAAAA|364|Elm |Parkway|Suite 40|Hopewell|Weston County|WY|80587|United States|-7|condo| +20510|AAAAAAAAOBAFAAAA|500|Hillcrest Spring|Street|Suite D|Welcome|Sabine County|TX|76386|United States|-6|single family| +20511|AAAAAAAAPBAFAAAA|869|Hickory |ST|Suite N|Wildwood|Treutlen County|GA|36871|United States|-5|apartment| +20512|AAAAAAAAACAFAAAA|836|Center Meadow|Street|Suite 330|Georgetown|Potter County|SD|57057|United States|-7|single family| +20513|AAAAAAAABCAFAAAA|396|East 3rd||||Brown County||66871|United States||| +20514|AAAAAAAACCAFAAAA|248|West 3rd|RD|Suite 0|Franklin|Richland County|OH|49101|United States|-5|apartment| +20515|AAAAAAAADCAFAAAA|138|Railroad |Street|Suite C|Ashland|Sunflower County|MS|54244|United States|-6|single family| +20516|AAAAAAAAECAFAAAA|542|Third |Drive|Suite 280|Red Oak|Colleton County|SC|25018|United States|-5|apartment| +20517|AAAAAAAAFCAFAAAA|740|Second |Blvd|Suite 200|Marion|Parke County|IN|40399|United States|-5|apartment| +20518|AAAAAAAAGCAFAAAA|747|Main 10th|Parkway|Suite D|Maple Grove|Oneida County|NY|18252|United States|-5|apartment| +20519|AAAAAAAAHCAFAAAA|815|3rd Jackson|Lane|Suite 220|Brentwood|Greenwood County|KS|64188|United States|-6|single family| +20520|AAAAAAAAICAFAAAA|54|Fourteenth Hill|Road|Suite D|Marion|Pickaway County|OH|40399|United States|-5|single family| +20521|AAAAAAAAJCAFAAAA|200|Oak |Street|Suite 420|Sunnyside|Warren County|NJ|02552|United States|-5|single family| +20522|AAAAAAAAKCAFAAAA|657|Second Fourth|Ave|Suite X|Riverview|Laurel County|KY|49003|United States|-5|condo| +20523|AAAAAAAALCAFAAAA|217|Third |Circle|Suite 480|Florence|Orleans County|VT|03994|United States|-5|single family| +20524|AAAAAAAAMCAFAAAA|156|South Franklin|Pkwy|Suite Y|Five Points|Beaverhead County|MT|66098|United States|-7|apartment| +20525|AAAAAAAANCAFAAAA|432|Sunset |Ct.|Suite 110|Waterloo|Sevier County|TN|31675|United States|-6|apartment| +20526|AAAAAAAAOCAFAAAA|862|Meadow Main|Street|Suite 80|Antioch|Gem County|ID|88605|United States|-7|condo| +20527|AAAAAAAAPCAFAAAA|282|West Locust|Lane|Suite E|Maple Grove|Wharton County|TX|78252|United States|-6|single family| +20528|AAAAAAAAADAFAAAA|933|Sycamore Park|Ct.|Suite 70|Shiloh|Albany County|NY|19275|United States|-5|apartment| +20529|AAAAAAAABDAFAAAA|628|3rd Lincoln|Boulevard|Suite 130|Antioch|Yolo County|CA|98605|United States|-8|apartment| +20530|AAAAAAAACDAFAAAA|257|Poplar |Boulevard|Suite D|Liberty|Abbeville County|SC|23451|United States|-5|apartment| +20531|AAAAAAAADDAFAAAA|46|First 2nd|Road|Suite T|Oak Ridge|Evans County|GA|38371|United States|-5|condo| +20532|AAAAAAAAEDAFAAAA|720|Fifth |Drive|Suite 250|Newport|Racine County|WI|51521|United States|-6|single family| +20533|AAAAAAAAFDAFAAAA|126|9th Smith|Wy|Suite X|Centerville|Scotland County|MO|60059|United States|-6|single family| +20534|AAAAAAAAGDAFAAAA|14|Washington Lincoln|Ct.|Suite U|Oakdale|Howard County|MO|69584|United States|-6|apartment| +20535|AAAAAAAAHDAFAAAA|275|Valley Jefferson|Court|Suite 340|Oak Grove|Pleasants County|WV|28370|United States|-5|condo| +20536|AAAAAAAAIDAFAAAA|126|Park Johnson|Boulevard|Suite 20|Elkton|Vermilion Parish|LA|73481|United States|-6|condo| +20537|AAAAAAAAJDAFAAAA|243|Hill |ST|Suite M|Riverview|Rockingham County|NH|09603|United States|-5|single family| +20538|AAAAAAAAKDAFAAAA|562|10th Adams|Street|Suite 410|Franklin|Cameron County|TX|79101|United States|-6|single family| +20539|AAAAAAAALDAFAAAA||||Suite 290|Freeman|Jefferson County|WV|22297|United States||| +20540|AAAAAAAAMDAFAAAA|905|Lake |Court|Suite J|Sulphur Springs|Effingham County|GA|38354|United States|-5|apartment| +20541|AAAAAAAANDAFAAAA|543|Sixth Madison|Ln|Suite 300|Waterloo|Pulaski County|GA|31675|United States|-5|condo| +20542|AAAAAAAAODAFAAAA|925|Railroad |ST|Suite 20|Georgetown|Musselshell County|MT|67057|United States|-7|apartment| +20543|AAAAAAAAPDAFAAAA|929|6th |ST|Suite 20|Franklin|Marion County|OH|49101|United States|-5|condo| +20544|AAAAAAAAAEAFAAAA|520|10th |Street|Suite 160|Five Forks|Broward County|FL|32293|United States|-5|single family| +20545|AAAAAAAABEAFAAAA|587|10th |RD|Suite 160|New Hope|Fillmore County|MN|59431|United States|-6|single family| +20546|AAAAAAAACEAFAAAA|840|Elm |Dr.|Suite 420|Newtown|Hill County|MT|61749|United States|-7|single family| +20547|AAAAAAAADEAFAAAA|281|Washington Sycamore|Lane|Suite 480|Enterprise|Franklin County|IL|61757|United States|-6|apartment| +20548|AAAAAAAAEEAFAAAA|571|Washington Park|Circle|Suite F|Hopewell|Lee County|FL|30587|United States|-5|single family| +20549|AAAAAAAAFEAFAAAA|656|Lakeview Williams|Street|Suite L|Mount Zion|Nicollet County|MN|58054|United States|-6|condo| +20550|AAAAAAAAGEAFAAAA|221|Sunset Center|Lane|Suite 200|Jackson|Mercer County|MO|69583|United States|-6|apartment| +20551|AAAAAAAAHEAFAAAA|843|1st |Dr.|Suite A|Hamilton|Blount County|TN|32808|United States|-5|apartment| +20552|AAAAAAAAIEAFAAAA|383|5th |Pkwy|Suite 370|Macedonia|Montgomery County|IL|61087|United States|-6|apartment| +20553|AAAAAAAAJEAFAAAA|243|1st Mill|ST|Suite B|Highland|Hamblen County|TN|39454|United States|-5|condo| +20554|AAAAAAAAKEAFAAAA|||||Fairview||||United States|-6|single family| +20555|AAAAAAAALEAFAAAA|825|Wilson |Street|Suite 30|Ruth|Columbus County|NC|20309|United States|-5|condo| +20556|AAAAAAAAMEAFAAAA|112|Third Sixth|Way|Suite 210|New Hope|Washington County|CO|89431|United States|-7|apartment| +20557|AAAAAAAANEAFAAAA|459|East |Ave|Suite 10|Buena Vista|Washington County|WI|55752|United States|-6|apartment| +20558|AAAAAAAAOEAFAAAA|144|Mill |Lane|Suite 460|Riverside|Grant County|OR|99231|United States|-8|single family| +20559|AAAAAAAAPEAFAAAA|286|Lincoln |Street|Suite 10|Wilson|Tulare County|CA|96971|United States|-8|condo| +20560|AAAAAAAAAFAFAAAA|996|14th 7th|Street|Suite G|Florence|Jefferson County|OR|93394|United States|-8|single family| +20561|AAAAAAAABFAFAAAA|744|Spring Park|Road|Suite S|Oak Ridge|Austin County|TX|78371|United States|-6|condo| +20562|AAAAAAAACFAFAAAA|334|Highland 1st|Avenue|Suite L|Jamestown|Fulton County|PA|16867|United States|-5|single family| +20563|AAAAAAAADFAFAAAA|299|Williams Hill|Blvd|Suite D|Enterprise|Cape Girardeau County|MO|61757|United States|-6|apartment| +20564|AAAAAAAAEFAFAAAA|9|Sycamore |Avenue|Suite O|River Oaks|Darlington County|SC|28075|United States|-5|condo| +20565|AAAAAAAAFFAFAAAA|629|Valley |Blvd|Suite T|White Oak|Kemper County|MS|56668|United States|-6|apartment| +20566|AAAAAAAAGFAFAAAA|55|Franklin |Road|Suite K|Hamilton|Rhea County|TN|32808|United States|-6|condo| +20567|AAAAAAAAHFAFAAAA|379|Dogwood Third|Lane|Suite 0|Riverdale|Dubois County|IN|49391|United States|-5|apartment| +20568|AAAAAAAAIFAFAAAA|385|Sunset Adams|Parkway|Suite 140|Mount Zion|Wabash County|IN|48054|United States|-5|single family| +20569|AAAAAAAAJFAFAAAA|852|Walnut |Court|Suite D|Franklin|Bulloch County|GA|39101|United States|-5|condo| +20570|AAAAAAAAKFAFAAAA|121|Madison Third|Ct.|Suite 310|Wilson|Banks County|GA|36971|United States|-5|condo| +20571|AAAAAAAALFAFAAAA|935|Miller |Ln|Suite 0|Greenwood|Grenada County|MS|58828|United States|-6|apartment| +20572|AAAAAAAAMFAFAAAA|900|Main Second|Wy|Suite X|Friendship|Swift County|MN|54536|United States|-6|apartment| +20573|AAAAAAAANFAFAAAA|594|Spruce |Ln|Suite 280|Unionville|Big Stone County|MN|51711|United States|-6|single family| +20574|AAAAAAAAOFAFAAAA|263|Cedar |Road|Suite Q|Sunnyside|Ontario County|NY|11952|United States|-5|single family| +20575|AAAAAAAAPFAFAAAA|341|Locust Hill|ST|Suite B|Farmington|Jefferson County|TN|39145|United States|-6|single family| +20576|AAAAAAAAAGAFAAAA|555|Adams |Circle|Suite 250|Kingston|Cass County|IA|54975|United States|-6|condo| +20577|AAAAAAAABGAFAAAA|807|Lincoln 4th|Court|Suite 370|Springdale|Union County|TN|38883|United States|-6|single family| +20578|AAAAAAAACGAFAAAA|470|South |Ln|Suite 270|Clifton|Hodgeman County|KS|68014|United States|-6|condo| +20579|AAAAAAAADGAFAAAA|707|10th |Pkwy|Suite 410|Brownsville|Jackson County|SD|59310|United States|-7|condo| +20580|AAAAAAAAEGAFAAAA|974|8th 3rd|Way|Suite R|Summit|West Carroll Parish|LA|70499|United States|-6|single family| +20581|AAAAAAAAFGAFAAAA|203|View Sunset|Parkway|Suite Q|Pleasant Valley|Pontotoc County|OK|72477|United States|-6|single family| +20582|AAAAAAAAGGAFAAAA|703|Lake 13th|Boulevard|Suite 470|Liberty|Edwards County|IL|63451|United States|-6|condo| +20583|AAAAAAAAHGAFAAAA|585|Lake 6th|Avenue|||||||-5|| +20584|AAAAAAAAIGAFAAAA|947||||||||United States||condo| +20585|AAAAAAAAJGAFAAAA|777|River View|Road|Suite P|Sulphur Springs|Weakley County|TN|38354|United States|-6|apartment| +20586|AAAAAAAAKGAFAAAA|363|Eigth |Boulevard|Suite 210|Fairview|Cumberland County|PA|15709|United States|-5|single family| +20587|AAAAAAAALGAFAAAA|419|Elm |Wy|Suite 150|Forest Hills|Foster County|ND|59237|United States|-6|single family| +20588|AAAAAAAAMGAFAAAA|67|Highland |Street|Suite J|New Hope|Onslow County|NC|29431|United States|-5|apartment| +20589|AAAAAAAANGAFAAAA|334|5th Walnut|Ct.|Suite 150|Lakewood|Cumberland County|KY|48877|United States|-6|apartment| +20590|AAAAAAAAOGAFAAAA|123|Park Center|RD|Suite H|Newtown|Lancaster County|SC|21749|United States|-5|condo| +20591|AAAAAAAAPGAFAAAA|88|5th 3rd|Blvd|Suite N|Mount Olive|Guthrie County|IA|58059|United States|-6|apartment| +20592|AAAAAAAAAHAFAAAA|586|10th Oak|Drive|Suite 470|Woodlawn|Bailey County|TX|74098|United States|-6|apartment| +20593|AAAAAAAABHAFAAAA|649|Fourth |Street|Suite 0|Bridgeport|Angelina County|TX|75817|United States|-6|condo| +20594|AAAAAAAACHAFAAAA|121|4th |Lane|Suite 110|Enterprise|Fisher County|TX|71757|United States|-6|single family| +20595|AAAAAAAADHAFAAAA|125|Ridge Elevnth|Road|Suite L|Oak Hill|Keweenaw County|MI|47838|United States|-5|apartment| +20596|AAAAAAAAEHAFAAAA|687|South 5th|Avenue|Suite K|Macedonia|Marion County|IN|41087|United States|-5|condo| +20597|AAAAAAAAFHAFAAAA|859|Forest |Lane|Suite 90|Shady Grove|Comal County|TX|72812|United States|-6|single family| +20598|AAAAAAAAGHAFAAAA|226|Johnson |Ct.|Suite 420|Friendship|Hudspeth County|TX|74536|United States|-6|condo| +20599|AAAAAAAAHHAFAAAA|801|Tenth Washington|Way|Suite 200|Riverdale|Walsh County|ND|59391|United States|-6|condo| +20600|AAAAAAAAIHAFAAAA|49|4th |Cir.|Suite Y|Forest Hills|Jefferson County|WV|29237|United States|-5|apartment| +20601|AAAAAAAAJHAFAAAA|541|Main |Boulevard|Suite 230|Macedonia|Catahoula Parish|LA|71087|United States|-6|apartment| +20602|AAAAAAAAKHAFAAAA|782|Center Hill|Cir.|Suite 120|Jackson|Jeff Davis County|GA|39583|United States|-5|single family| +20603|AAAAAAAALHAFAAAA||||||Owsley County|KY||United States|-5|apartment| +20604|AAAAAAAAMHAFAAAA|766|Center |Ave|Suite 30|Five Forks|Franklin Parish|LA|72293|United States|-6|condo| +20605|AAAAAAAANHAFAAAA|73|Oak |Blvd|Suite 480|Macedonia|Barry County|MI|41087|United States|-5|apartment| +20606|AAAAAAAAOHAFAAAA|744|West |ST|Suite A|Spring Hill|Motley County|TX|76787|United States|-6|condo| +20607|AAAAAAAAPHAFAAAA|107|Woodland |Drive|Suite O|Highland Park|Mercer County|KY|46534|United States|-5|single family| +20608|AAAAAAAAAIAFAAAA|175|Third 9th|Ave|Suite U|Spring Valley|Williamson County|IL|66060|United States|-6|single family| +20609|AAAAAAAABIAFAAAA|234|Spruce Highland|Court|Suite M|Mount Pleasant|Logan County|WV|21933|United States|-5|condo| +20610|AAAAAAAACIAFAAAA|291|Broadway |Wy|Suite C|Springhill|Baraga County|MI|44602|United States|-5|single family| +20611|AAAAAAAADIAFAAAA|983|3rd Second|ST|Suite V|Oak Hill|Oconee County|GA|37838|United States|-5|apartment| +20612|AAAAAAAAEIAFAAAA|906|Locust 13th|Ave|Suite 280|Sulphur Springs|Jefferson County|GA|38354|United States|-5|single family| +20613|AAAAAAAAFIAFAAAA|905|3rd 2nd|Road|Suite 490|Cedar Grove|Dallas County|AR|70411|United States|-6|apartment| +20614|AAAAAAAAGIAFAAAA|464|11th Sixth|Avenue|Suite 140|Gum Springs|Navarro County|TX|72106|United States|-6|condo| +20615|AAAAAAAAHIAFAAAA|192|Walnut |Way|Suite 280|Concord|Columbia County|FL|34107|United States|-5|single family| +20616|AAAAAAAAIIAFAAAA|310|Cherry Oak|Road|Suite 190|Maple Grove|Mineral County|WV|28252|United States|-5|single family| +20617|AAAAAAAAJIAFAAAA|959|Cedar |Ave|Suite 270|Garrison|Wetzel County|WV|28767|United States|-5|condo| +20618|AAAAAAAAKIAFAAAA|190|Mill |Circle|Suite E|Shiloh|Adams County|ND|59275|United States|-6|apartment| +20619|AAAAAAAALIAFAAAA|28|Williams 15th|Cir.|Suite S|Summit|New Kent County|VA|20499|United States|-5|apartment| +20620|AAAAAAAAMIAFAAAA|322|Sycamore North|ST|Suite 230|Oakwood|Nemaha County|NE|60169|United States|-7|single family| +20621|AAAAAAAANIAFAAAA|995|Sunset Fourth|Court|Suite 60|Mount Zion|Bledsoe County|TN|38054|United States|-5|condo| +20622|AAAAAAAAOIAFAAAA|181|1st Spring|Cir.|Suite 240|Harmony|Covington County|MS|55804|United States|-6|apartment| +20623|AAAAAAAAPIAFAAAA|216|Locust |Pkwy|Suite E|Summit|Coleman County|TX|70499|United States|-6|apartment| +20624|AAAAAAAAAJAFAAAA|558|5th |Drive|Suite 50|Plainview|Traverse County|MN|53683|United States|-6|apartment| +20625|AAAAAAAABJAFAAAA|985|5th |Pkwy|Suite J|Oakland|Montgomery County|OH|49843|United States|-5|single family| +20626|AAAAAAAACJAFAAAA|355|Washington 1st|Parkway|Suite D|Jackson|Northwest Arctic Borough|AK|99583|United States|-9|apartment| +20627|AAAAAAAADJAFAAAA|552|Third |Blvd|Suite B|Lincoln|Blaine County|OK|71289|United States|-6|condo| +20628|AAAAAAAAEJAFAAAA|46|Fourth Second|Ct.|Suite 380|Forest Hills|Leslie County|KY|49237|United States|-5|condo| +20629|AAAAAAAAFJAFAAAA||Poplar Jefferson||Suite Y||||30059|United States||| +20630|AAAAAAAAGJAFAAAA|802|7th Sunset|||||||United States||| +20631|AAAAAAAAHJAFAAAA|791|Fourth Cedar|Circle|Suite H|Oak Grove|Hot Spring County|AR|78370|United States|-6|single family| +20632|AAAAAAAAIJAFAAAA|39|Adams |RD|Suite I|Shady Grove|Upton County|TX|72812|United States|-6|condo| +20633|AAAAAAAAJJAFAAAA|14|3rd Maple|Ct.|Suite M|Antioch|Bergen County|NJ|09205|United States|-5|condo| +20634|AAAAAAAAKJAFAAAA|797|First |Dr.|Suite C|Five Forks|Judith Basin County|MT|62293|United States|-7|single family| +20635|AAAAAAAALJAFAAAA|727|View |Blvd|Suite 170|Arlington|Blount County|TN|36557|United States|-5|single family| +20636|AAAAAAAAMJAFAAAA|660|Mill View|Street|Suite F|Glendale|Jefferson County|NY|13951|United States|-5|single family| +20637|AAAAAAAANJAFAAAA|162|6th Sunset|Wy|Suite A|Pleasant Hill|Sagadahoc County|ME|04204|United States|-5|single family| +20638|AAAAAAAAOJAFAAAA|383|Cedar |ST|Suite Q|Sulphur Springs|Bath County|VA|28354|United States|-5|condo| +20639|AAAAAAAAPJAFAAAA|442|Highland Broadway|Cir.|Suite O|Hillcrest|Greene County|IL|63003|United States|-6|single family| +20640|AAAAAAAAAKAFAAAA|439|Ash Church|Avenue|Suite 380|Montezuma|Bay County|FL|32150|United States|-5|apartment| +20641|AAAAAAAABKAFAAAA|83|Johnson |Way|Suite S|Oakland|Hendry County|FL|39843|United States|-5|apartment| +20642|AAAAAAAACKAFAAAA|25|Lincoln Mill|Court|Suite 110|Oakdale|Eddy County|NM|89584|United States|-7|condo| +20643|AAAAAAAADKAFAAAA|107|Park |ST|Suite 170|Plainview|Macoupin County|IL|63683|United States|-6|condo| +20644|AAAAAAAAEKAFAAAA|641|Williams |Blvd|Suite T|Oakwood|Choctaw County|AL|30169|United States|-6|single family| +20645|AAAAAAAAFKAFAAAA|450|Park Center|RD|Suite 160|Pleasant Valley|Jackson County|WV|22477|United States|-5|condo| +20646|AAAAAAAAGKAFAAAA|212|Willow |Dr.|Suite 100|Mountain View|Buffalo County|SD|54466|United States|-6|single family| +20647|AAAAAAAAHKAFAAAA|247|Church |Wy|Suite E|Buckingham|Peach County|GA|34092|United States|-5|condo| +20648|AAAAAAAAIKAFAAAA|263|South |ST|Suite 160|Shady Grove|Miller County|AR|72812|United States|-6|condo| +20649|AAAAAAAAJKAFAAAA|370|Smith |Way|Suite V|Spring Hill|Lipscomb County|TX|76787|United States|-6|apartment| +20650|AAAAAAAAKKAFAAAA|95|Church |ST|Suite L|Summit|San Augustine County|TX|70499|United States|-6|condo| +20651|AAAAAAAALKAFAAAA|2|Sycamore Church|ST|Suite M|King|Choctaw County|AL|30008|United States|-6|apartment| +20652|AAAAAAAAMKAFAAAA|927|Lake |Boulevard|Suite 100|Harmony|Shawano County|||||| +20653|AAAAAAAANKAFAAAA|307|6th |Blvd|Suite 0|Mount Olive|Calhoun County|AR|78059|United States|-6|single family| +20654|AAAAAAAAOKAFAAAA|559|Meadow Main|Street|Suite 280|Flint|Metcalfe County|KY|48909|United States|-5|condo| +20655|AAAAAAAAPKAFAAAA|777|Johnson |Drive|Suite 380|Midway|Angelina County|TX|71904|United States|-6|apartment| +20656|AAAAAAAAALAFAAAA|827|Railroad |Ln|Suite L|Oak Grove|Williamson County|IL|68370|United States|-6|single family| +20657|AAAAAAAABLAFAAAA|552|Elm |RD|Suite F|Cedar Grove|Stephens County|TX|70411|United States|-6|single family| +20658|AAAAAAAACLAFAAAA|99||Ln|Suite 380|||||United States|-6|single family| +20659|AAAAAAAADLAFAAAA|662|Williams North|Court|Suite 210|Centerville|Clay County|KY|40059|United States|-6|single family| +20660|AAAAAAAAELAFAAAA|474|Jefferson College|Wy|Suite P|Dallas|Jackson County|OK|73628|United States|-6|condo| +20661|AAAAAAAAFLAFAAAA|104|Poplar |Pkwy|Suite 330|Oakdale|Lander County|NV|89584|United States|-8|apartment| +20662|AAAAAAAAGLAFAAAA|773|Pine 5th|Ln|Suite M|Riverside|El Paso County|CO|89231|United States|-7|apartment| +20663|AAAAAAAAHLAFAAAA|693|Hill 5th|Lane|Suite F|Pleasant Hill|Yolo County|CA|93604|United States|-8|single family| +20664|AAAAAAAAILAFAAAA|703|Center |Cir.|Suite 320|Gilmore|Luce County|MI|45464|United States|-5|single family| +20665|AAAAAAAAJLAFAAAA|767|Spring |Cir.|Suite L|Woodlawn|Mitchell County|IA|54098|United States|-6|single family| +20666|AAAAAAAAKLAFAAAA|430|Green Center|Wy|Suite T|Green Acres|De Soto Parish|LA|77683|United States|-6|apartment| +20667|AAAAAAAALLAFAAAA|709|14th Park|Blvd|Suite 30|Franklin|Waseca County|MN|59101|United States|-6|single family| +20668|AAAAAAAAMLAFAAAA|276|Cherry Jackson|Cir.|Suite L|Union|Floyd County|KY|48721|United States|-6|single family| +20669|AAAAAAAANLAFAAAA|346|10th |Drive|Suite 70|Shady Grove|Chase County|KS|62812|United States|-6|apartment| +20670|AAAAAAAAOLAFAAAA|695|Center Davis|Dr.|Suite 170|Mount Vernon|Pottawatomie County|KS|68482|United States|-6|apartment| +20671|AAAAAAAAPLAFAAAA|655|5th |Pkwy|Suite T|Highland Park|Weston County|WY|86534|United States|-7|apartment| +20672|AAAAAAAAAMAFAAAA|242|North Ash|Boulevard|Suite 270|Shiloh|Cherokee County|GA|39275|United States|-5|single family| +20673|AAAAAAAABMAFAAAA|256|Main |Court|Suite J|Ridgeville|Mayes County|OK|79306|United States|-6|apartment| +20674|AAAAAAAACMAFAAAA|530|11th |Parkway|Suite T|Five Points|Butte County|ID|86098|United States|-7|apartment| +20675|AAAAAAAADMAFAAAA|661|15th Washington|Way|Suite N|Lincoln|Calhoun County|MS|51289|United States|-6|apartment| +20676|AAAAAAAAEMAFAAAA|656|6th Oak|Pkwy|Suite 250|Riverside|Lynn County|TX|79231|United States|-6|condo| +20677|AAAAAAAAFMAFAAAA|693|Davis |Court|Suite B|Valley View|Floyd County|TX|75124|United States|-6|single family| +20678|AAAAAAAAGMAFAAAA|723|Smith |Street|Suite X|Marion|Victoria County|TX|70399|United States|-6|apartment| +20679|AAAAAAAAHMAFAAAA|416|Adams |Wy|Suite D|Mount Olive|Philadelphia County|PA|18059|United States|-5|condo| +20680|AAAAAAAAIMAFAAAA|620|East |Ln|Suite I|Fairview|Henderson County|TX|75709|United States|-6|apartment| +20681|AAAAAAAAJMAFAAAA|525|Williams Walnut|RD|Suite 10|Waterloo|Henderson County|TN|31675|United States|-5|single family| +20682|AAAAAAAAKMAFAAAA|153|Spruce |Cir.|Suite 390|Maple Hill|Cumberland County|TN|38095|United States|-5|apartment| +20683|AAAAAAAALMAFAAAA|375|6th Cedar|Lane|Suite V|Waterloo|Jeff Davis County|TX|71675|United States|-6|condo| +20684|AAAAAAAAMMAFAAAA|||ST||Bridgeport|Yamhill County|OR||||| +20685|AAAAAAAANMAFAAAA|8|Oak |Way|Suite 430|Wilson|Dodge County|MN|56971|United States|-6|single family| +20686|AAAAAAAAOMAFAAAA|547|Johnson Cherry|RD|Suite E|Springfield|Jones County|MS|59303|United States|-6|single family| +20687|AAAAAAAAPMAFAAAA|983|Meadow |Parkway|Suite 340|Wilson|McMinn County|TN|36971|United States|-6|apartment| +20688|AAAAAAAAANAFAAAA|353|7th 10th|Avenue|Suite 490|Five Points|Marion County|IA|56098|United States|-6|condo| +20689|AAAAAAAABNAFAAAA|903|Maple |RD|Suite Y|Lakeview|Defiance County|OH|48579|United States|-5|condo| +20690|AAAAAAAACNAFAAAA|27|Lake |RD|Suite Y|Waterloo|Somerset County|ME|02275|United States|-5|condo| +20691|AAAAAAAADNAFAAAA|304|Park |Blvd|Suite V|New Hope|Tippecanoe County|IN|49431|United States|-5|single family| +20692|AAAAAAAAENAFAAAA|468|Maple Ash|Court|Suite Q|Mount Zion|Jefferson County|WI|58054|United States|-6|single family| +20693|AAAAAAAAFNAFAAAA|329|Jackson |ST|Suite I|Liberty|Delaware County|NY|13451|United States|-5|single family| +20694|AAAAAAAAGNAFAAAA|107|Church 13th|Court|Suite T|White Oak|Franklin County|IA|56668|United States|-6|single family| +20695|AAAAAAAAHNAFAAAA|586|Washington |Drive|Suite 130|Northwood|Scott County|IN|44104|United States|-5|apartment| +20696|AAAAAAAAINAFAAAA|431|15th |Ln|Suite W|Sulphur Springs|Rabun County|GA|38354|United States|-5|single family| +20697|AAAAAAAAJNAFAAAA|18|Sunset Railroad|Pkwy|Suite P|Fairfield|Warren County|MS|56192|United States|-6|condo| +20698|AAAAAAAAKNAFAAAA|950|1st Lakeview|Ave|Suite O|Riverdale|Frio County|TX|79391|United States|-6|single family| +20699|AAAAAAAALNAFAAAA|443|9th |Blvd|Suite D|Harmony|Watauga County|NC|25804|United States|-5|apartment| +20700|AAAAAAAAMNAFAAAA|501|Miller |Road|Suite 160|Lakeview|Hale County|AL|38579|United States|-6|condo| +20701|AAAAAAAANNAFAAAA|547|Lake Sunset|Ave|Suite 380|Belmont|Elliott County|KY|40191|United States|-6|condo| +20702|AAAAAAAAONAFAAAA|92|Washington Park|Circle|Suite R|Clinton|La Salle County|TX|78222|United States|-6|single family| +20703|AAAAAAAAPNAFAAAA|418|3rd Hillcrest|Cir.|Suite T|Florence|Pickett County|TN|33394|United States|-6|apartment| +20704|AAAAAAAAAOAFAAAA|153|South Maple|Ct.|Suite 400|Union Hill|Camas County|ID|87746|United States|-7|condo| +20705|AAAAAAAABOAFAAAA|383|2nd Maple|Pkwy|Suite B|Mount Vernon|Humphreys County|MS|58482|United States|-6|apartment| +20706|AAAAAAAACOAFAAAA|527|3rd |Drive|Suite T|Mount Zion|West Feliciana Parish|LA|78054|United States|-6|condo| +20707|AAAAAAAADOAFAAAA|664|Washington Main|Wy|Suite A|Texas|Telfair County|GA|33342|United States|-5|condo| +20708|AAAAAAAAEOAFAAAA|868|3rd |ST|Suite G|Sunnyside|Franklin County|IN|41952|United States|-5|condo| +20709|AAAAAAAAFOAFAAAA|519|Hill Pine|Boulevard|Suite 150|Fairfield|Rice County|MN|56192|United States|-6|single family| +20710|AAAAAAAAGOAFAAAA|487|5th |Wy|Suite H|Antioch|Real County|TX|78605|United States|-6|single family| +20711|AAAAAAAAHOAFAAAA|684|Meadow South|Parkway|Suite A|Riverview|Bartow County|GA|39003|United States|-5|condo| +20712|AAAAAAAAIOAFAAAA|327|Oak Walnut|Ln|Suite V|Georgetown|Logan County|OH|47057|United States|-5|condo| +20713|AAAAAAAAJOAFAAAA|878|Highland |Pkwy|Suite 120|Kelly|Clinton County|IL|62738|United States|-6|condo| +20714|AAAAAAAAKOAFAAAA|393|Washington |Road|Suite Q|Forest Hills|Rush County|IN|49237|United States|-5|single family| +20715|AAAAAAAALOAFAAAA|870|East Spring|Way|Suite 130|Cedar Grove|Irwin County|GA|30411|United States|-5|single family| +20716|AAAAAAAAMOAFAAAA|69|Main West|Parkway|Suite X|Five Forks|Clay County|NC|22293|United States|-5|apartment| +20717|AAAAAAAANOAFAAAA|416|5th |Cir.|Suite 60|Greendale|Brown County|MN|59343|United States|-6|condo| +20718|AAAAAAAAOOAFAAAA|303|Cherry |Blvd|Suite I|Farmington|Hand County|SD|59145|United States|-7|apartment| +20719|AAAAAAAAPOAFAAAA|367|Oak |Drive|Suite 300|New Hope|Lawrence County|MS|59431|United States|-6|single family| +20720|AAAAAAAAAPAFAAAA|337|Miller Park|Ln|Suite C|Cedar Grove|McLean County|KY|40411|United States|-5|single family| +20721|AAAAAAAABPAFAAAA|232|Mill |ST|Suite V|New Hope|Sanilac County|MI|49431|United States|-5|single family| +20722|AAAAAAAACPAFAAAA|325|Cedar 1st|Ln|Suite C|Mount Olive|Cook County|GA|38059|United States|-5|single family| +20723|AAAAAAAADPAFAAAA|952|First Chestnut|RD|Suite O|Highland|Mecklenburg County|VA|29454|United States|-5|single family| +20724|AAAAAAAAEPAFAAAA|653|Forest Lincoln|Blvd|Suite 50|Salem|Garfield County|UT|88048|United States|-7|single family| +20725|AAAAAAAAFPAFAAAA|545|Lake |Ln|Suite W|Five Forks|Mason County|MI|42293|United States|-5|apartment| +20726|AAAAAAAAGPAFAAAA|26|Madison |Avenue|Suite T|Red Hill|Sampson County|NC|24338|United States|-5|single family| +20727|AAAAAAAAHPAFAAAA|234|Main |Ave|Suite 280|Clifton|Rockingham County|NH|08614|United States|-5|single family| +20728|AAAAAAAAIPAFAAAA|697|First Third|Lane|Suite 440|Providence|Williamson County|TN|36614|United States|-5|apartment| +20729|AAAAAAAAJPAFAAAA|345|1st Elm|Court|Suite 360|Mount Pleasant|Stone County|MS|51933|United States|-6|apartment| +20730|AAAAAAAAKPAFAAAA|362|Hillcrest Spring|Avenue|Suite Q|Superior|Johnson County|IA|52562|United States|-6|condo| +20731|AAAAAAAALPAFAAAA|378|Miller East|Parkway|Suite U|Andover|Terrebonne Parish|LA|71639|United States|-6|single family| +20732|AAAAAAAAMPAFAAAA|617|South 1st|Way|Suite P|Mountain View|Jefferson County|OH|44466|United States|-5|condo| +20733|AAAAAAAANPAFAAAA|752|Wilson Lincoln|Pkwy|Suite F|Bunker Hill|Dutchess County|NY|10150|United States|-5|apartment| +20734|AAAAAAAAOPAFAAAA|128|Walnut |Ave|Suite 200|Mount Zion|Forrest County|MS|58054|United States|-6|apartment| +20735|AAAAAAAAPPAFAAAA|313|Pine Main|Boulevard|Suite 470|Milo|Sedgwick County|CO|80116|United States|-7|condo| +20736|AAAAAAAAAABFAAAA|89|Fourth Eigth|Ave|Suite 480|Lakeside|Chicot County|AR|79532|United States|-6|single family| +20737|AAAAAAAABABFAAAA|841|Second |Lane|Suite 350|Kingston|Tallahatchie County|MS|54975|United States|-6|single family| +20738|AAAAAAAACABFAAAA|292|10th |Road|Suite Y|Oak Ridge|Jones County|GA|38371|United States|-5|condo| +20739|AAAAAAAADABFAAAA|136|Miller North|RD|Suite 170|Enterprise|Clackamas County|OR|91757|United States|-8|single family| +20740|AAAAAAAAEABFAAAA|76|Green Locust|Avenue|Suite 10|Union|Dixon County|NE|68721|United States|-6|single family| +20741|AAAAAAAAFABFAAAA|721|Central |Street|Suite 90|Georgetown|Crowley County|CO|87057|United States|-7|apartment| +20742|AAAAAAAAGABFAAAA|983|2nd Franklin|RD|Suite N|Freeport|Tioga County|NY|11844|United States|-5|single family| +20743|AAAAAAAAHABFAAAA|907|Laurel Sycamore|Ln|Suite L|Farmington|Gray County|KS|69145|United States|-6|single family| +20744|AAAAAAAAIABFAAAA|297|Ridge 3rd|Blvd|Suite 150|Crossroads|Calhoun County|MI|40534|United States|-5|condo| +20745|AAAAAAAAJABFAAAA|137|Franklin Highland|Avenue|Suite P|Springfield|Putnam County|IL|69303|United States|-6|apartment| +20746|AAAAAAAAKABFAAAA|688|Mill Washington|Blvd|Suite 90|Highland|Elliott County|KY|49454|United States|-6|apartment| +20747|AAAAAAAALABFAAAA|494|Madison 8th|Parkway|Suite 240|Pine Grove|Logan County|KS|64593|United States|-6|apartment| +20748|AAAAAAAAMABFAAAA|406|2nd South|Ln|Suite 180|Glendale|Crockett County|TN|33951|United States|-5|apartment| +20749|AAAAAAAANABFAAAA|453|3rd |ST|Suite 140|Clifton|Dawson County|NE|68014|United States|-6|apartment| +20750|AAAAAAAAOABFAAAA|607|Church Spruce|Ln|Suite H|Buena Vista|Okanogan County|WA|95752|United States|-8|condo| +20751|AAAAAAAAPABFAAAA|808|Jackson |Road|Suite F|Hillcrest|Windham County|VT|03603|United States|-5|condo| +20752|AAAAAAAAABBFAAAA|656|Ridge Hill|Circle|Suite 160|Woodville|Harford County|MD|24289|United States|-5|apartment| +20753|AAAAAAAABBBFAAAA|588|Third Williams|ST|Suite V|Mount Vernon|Tipton County|TN|38482|United States|-6|single family| +20754|AAAAAAAACBBFAAAA|142|Jefferson |Cir.|Suite K|Boyd|Mitchell County|NC|20163|United States|-5|apartment| +20755|AAAAAAAADBBFAAAA|431|River |Court|Suite K|Bear Creek|Outagamie County|WI|53075|United States|-6|apartment| +20756|AAAAAAAAEBBFAAAA|119|Pine |Boulevard|Suite 60|Green Acres|Johnson County|KS|67683|United States|-6|condo| +20757|AAAAAAAAFBBFAAAA|272|Lake |Ave|Suite 480|Five Points|Garfield County|CO|86098|United States|-7|single family| +20758|AAAAAAAAGBBFAAAA|728|Davis |Ave|Suite C|Bunker Hill|Robertson County|TN|30150|United States|-6|condo| +20759|AAAAAAAAHBBFAAAA|756|Adams Washington|Cir.|Suite 170|Oakland|Grimes County|TX|79843|United States|-6|apartment| +20760|AAAAAAAAIBBFAAAA|817||||Plainview||MN|||-6|single family| +20761|AAAAAAAAJBBFAAAA|941|Cherry |RD|Suite W|Oak Hill|Highlands County|FL|37838|United States|-5|apartment| +20762|AAAAAAAAKBBFAAAA|750|College |RD|Suite F|Greenwood|Richland County|OH|48828|United States|-5|condo| +20763|AAAAAAAALBBFAAAA|797|Adams Ninth|Blvd|Suite 160|Unionville|East Feliciana Parish|LA|71711|United States|-6|apartment| +20764|AAAAAAAAMBBFAAAA|936|8th Park|Way|Suite T|Concord|Johnson County|TN|34107|United States|-6|single family| +20765|AAAAAAAANBBFAAAA|762|3rd |Road|Suite K|Kingston|Onondaga County|NY|14975|United States|-5|apartment| +20766|AAAAAAAAOBBFAAAA|529|Eigth Oak|Dr.|Suite 340|Kingston|Pepin County|WI|54975|United States|-6|apartment| +20767|AAAAAAAAPBBFAAAA|380|1st |Ave|Suite 40|Summit|Bristol County|RI|01099|United States|-5|single family| +20768|AAAAAAAAACBFAAAA|521|6th |Wy|Suite 290|Plainview|Sequoyah County|OK|73683|United States|-6|apartment| +20769|AAAAAAAABCBFAAAA|857|Spruce |Avenue|Suite U|Summit|Lewis County|NY|10499|United States|-5|single family| +20770|AAAAAAAACCBFAAAA|765|Cherry |Wy|Suite 10|Green Acres|Rice County|KS|67683|United States|-6|single family| +20771|AAAAAAAADCBFAAAA|427|12th |Boulevard|Suite 210|Arlington|Edgecombe County|NC|26557|United States|-5|condo| +20772|AAAAAAAAECBFAAAA|808|Lake Third|ST|Suite Y|Springdale|Perkins County|NE|68883|United States|-7|single family| +20773|AAAAAAAAFCBFAAAA|504|Ridge |RD|Suite D|Kingston|Fremont County|IA|54975|United States|-6|condo| +20774|AAAAAAAAGCBFAAAA|970|Hickory 8th|Blvd|Suite R|Deerfield|Kay County|OK|79840|United States|-6|single family| +20775|AAAAAAAAHCBFAAAA|183|Lincoln Mill|Road|Suite 100|Shady Grove|Weakley County|TN|32812|United States|-6|apartment| +20776|AAAAAAAAICBFAAAA|405|Third Smith|RD|Suite O|Five Points|Vilas County|WI|56098|United States|-6|condo| +20777|AAAAAAAAJCBFAAAA|742|Center |Cir.|Suite 40|Union Hill|Washington County|IA|57746|United States|-6|apartment| +20778|AAAAAAAAKCBFAAAA|431|Maple |Ave|Suite 320|Maple Grove|Hardeman County|TN|38252|United States|-5|single family| +20779|AAAAAAAALCBFAAAA|993|Hill |Cir.|Suite 120|Crossroads|Caldwell County|MO|60534|United States|-6|single family| +20780|AAAAAAAAMCBFAAAA|130|River |Boulevard|Suite 130|Fairview|Albemarle County|VA|25709|United States|-5|condo| +20781|AAAAAAAANCBFAAAA|707|Railroad |Parkway|Suite G|Gilmore|Ashland County|OH|45464|United States|-5|condo| +20782|AAAAAAAAOCBFAAAA|840|2nd |Cir.|Suite 0|Farmington|Lawrence County|MS|59145|United States|-6|condo| +20783|AAAAAAAAPCBFAAAA|44|Pine 10th|Road|Suite 140|Providence|Roscommon County|MI|46614|United States|-5|condo| +20784|AAAAAAAAADBFAAAA|74|6th View|Drive|Suite Q|Riverview|Union County|GA|39003|United States|-5|single family| +20785|AAAAAAAABDBFAAAA|615|Birch |Court|Suite 290|Kingston|Lipscomb County|TX|74975|United States|-6|apartment| +20786|AAAAAAAACDBFAAAA|954|7th Ash|Pkwy|Suite 230|Kingston|Judith Basin County|MT|64975|United States|-7|apartment| +20787|AAAAAAAADDBFAAAA|769|7th Poplar|Circle|Suite 200|Deerfield|Maries County|MO|69840|United States|-6|single family| +20788|AAAAAAAAEDBFAAAA|647|Maple |Lane|Suite B|Belmont|Monroe County|MS|50191|United States|-6|condo| +20789|AAAAAAAAFDBFAAAA|363|7th |Way|Suite 290|Hopewell|Bennett County|SD|50587|United States|-6|apartment| +20790|AAAAAAAAGDBFAAAA|144|East Second|Lane|Suite 40|Wilson|Madison County|MS|56971|United States|-6|single family| +20791|AAAAAAAAHDBFAAAA|466|Walnut |RD|Suite 160|Jamestown|Coles County|IL|66867|United States|-6|apartment| +20792|AAAAAAAAIDBFAAAA|625|Birch South|Lane|Suite M|Riley|Niagara County|NY|11692|United States|-5|condo| +20793|AAAAAAAAJDBFAAAA|197|South |Pkwy|Suite M|Greenfield|Jefferson County|MT|65038|United States|-7|single family| +20794|AAAAAAAAKDBFAAAA|944|Spring Walnut|Drive|Suite Q|Union Hill|Butte County|CA|97746|United States|-8|single family| +20795|AAAAAAAALDBFAAAA|573|Woodland |Street|Suite Y|Lakeside|Genesee County|NY|19532|United States|-5|single family| +20796|AAAAAAAAMDBFAAAA|19|Cherry Sixth|Street|Suite 130|Mount Vernon|Coconino County|AZ|88482|United States|-7|apartment| +20797|AAAAAAAANDBFAAAA|820|Birch Spruce|Drive|Suite K|Providence|Ouray County|CO|86614|United States|-7|condo| +20798|AAAAAAAAODBFAAAA|321|First |Ct.||Mount Pleasant|Cotton County||71933|United States|-6|condo| +20799|AAAAAAAAPDBFAAAA|788|Highland 7th|Blvd|Suite C|Oakdale|Portsmouth city|VA|29584|United States|-5|condo| +20800|AAAAAAAAAEBFAAAA|560|Park View|Way|Suite D|Newtown|Mitchell County|NC|21749|United States|-5|condo| +20801|AAAAAAAABEBFAAAA|13|Hill |Drive|||||24289|||single family| +20802|AAAAAAAACEBFAAAA|465|North |Lane|Suite 210|Mountain View|Pike County|IN|44466|United States|-5|single family| +20803|AAAAAAAADEBFAAAA|642|Valley |Ct.|Suite E|Franklin|Rhea County|TN|39101|United States|-6|apartment| +20804|AAAAAAAAEEBFAAAA|266|Third Smith|Ct.|Suite 450|Centerville|Ada County|ID|80059|United States|-7|condo| +20805|AAAAAAAAFEBFAAAA|75|4th |Ct.|Suite W|Riverview|Seminole County|FL|39003|United States|-5|apartment| +20806|AAAAAAAAGEBFAAAA|35|Jefferson Locust|Ave|Suite G|Pleasant Grove|Peoria County|IL|64136|United States|-6|apartment| +20807|AAAAAAAAHEBFAAAA|577|Ash Birch|Cir.|Suite 450|Woodville|Pushmataha County|OK|74289|United States|-6|condo| +20808|AAAAAAAAIEBFAAAA|730|9th |Boulevard|Suite W|Pleasant Valley|Taliaferro County|GA|32477|United States|-5|single family| +20809|AAAAAAAAJEBFAAAA|972|Elm 2nd|Ct.|Suite 140|Arlington|Lucas County|IA|56557|United States|-6|single family| +20810|AAAAAAAAKEBFAAAA|70|Jefferson 6th|Blvd|Suite 370|Jamestown|Morrow County|OR|96867|United States|-8|single family| +20811|AAAAAAAALEBFAAAA|843|View 10th|Wy|Suite U|Enterprise|Meriwether County|GA|31757|United States|-5|single family| +20812|AAAAAAAAMEBFAAAA|835|Jefferson South|Road|Suite T|Enterprise|Donley County|TX|71757|United States|-6|apartment| +20813|AAAAAAAANEBFAAAA|444|Johnson |Boulevard|Suite F|Florence|Manatee County|FL|33394|United States|-5|condo| +20814|AAAAAAAAOEBFAAAA|25|Johnson |Dr.|Suite 250|Unionville|Hutchinson County|TX|71711|United States|-6|single family| +20815|AAAAAAAAPEBFAAAA|99|Fifth 9th|Road|Suite 110|Walnut Grove|La Salle County|TX|77752|United States|-6|condo| +20816|AAAAAAAAAFBFAAAA|662|Jackson |Lane|Suite 230|Thompsonville|Loudoun County|VA|29651|United States|-5|single family| +20817|AAAAAAAABFBFAAAA|428|5th |Cir.|Suite 290|Deerfield|Fulton County|GA|39840|United States|-5|apartment| +20818|AAAAAAAACFBFAAAA|761|Miller |Way|Suite Y|Mountain View|McKenzie County|ND|54466|United States|-6|single family| +20819|AAAAAAAADFBFAAAA|506|South |Street|Suite R|Springdale|Stephens County|OK|78883|United States|-6|condo| +20820|AAAAAAAAEFBFAAAA|599|7th |Drive|Suite F|Mount Pleasant|Bernalillo County|NM|81933|United States|-7|condo| +20821|AAAAAAAAFFBFAAAA|291|8th Chestnut|Ave|Suite 200|Springdale|Doniphan County|KS|68883|United States|-6|apartment| +20822|AAAAAAAAGFBFAAAA|645|Jefferson 3rd|RD|Suite Q|Jenkins|Cameron County|PA|17292|United States|-5|apartment| +20823|AAAAAAAAHFBFAAAA|571|Jackson |Ln|Suite 210|Woodland|Grenada County|MS|54854|United States|-6|condo| +20824|AAAAAAAAIFBFAAAA|645|Elm |Circle|Suite 100|Spring Valley|Cook County|IL|66060|United States|-6|apartment| +20825|AAAAAAAAJFBFAAAA|861|Second |Way|Suite 220|Lakeview|Ballard County|KY|48579|United States|-6|condo| +20826|AAAAAAAAKFBFAAAA|909|Hickory |Street|Suite X|Buena Vista|Stanly County|NC|25752|United States|-5|condo| +20827|AAAAAAAALFBFAAAA|799|12th |Circle|Suite 30|Springfield|Union County|TN|39303|United States|-6|apartment| +20828|AAAAAAAAMFBFAAAA|139|Cherry Lake|RD|Suite R|Lincoln|Howard County|IN|41289|United States|-5|condo| +20829|AAAAAAAANFBFAAAA|436|Mill 4th|ST|Suite L|Plainview|Baker County|GA|33683|United States|-5|single family| +20830|AAAAAAAAOFBFAAAA|268|Forest Birch|Ct.|Suite 320|Newport|Buchanan County|IA|51521|United States|-6|condo| +20831|AAAAAAAAPFBFAAAA|269|Church Walnut|Lane|Suite 240|Wildwood|Harrison County|IA|56871|United States|-6|condo| +20832|AAAAAAAAAGBFAAAA|457|Washington 15th|Road|Suite R|Arlington|Colfax County|NE|66557|United States|-6|single family| +20833|AAAAAAAABGBFAAAA|115|Pine |Avenue|Suite N|Plainview|Matanuska-Susitna Borough|AK|93683|United States|-9|condo| +20834|AAAAAAAACGBFAAAA|51|Franklin Poplar|Circle|Suite C|Denmark|Copiah County|MS|55576|United States|-6|apartment| +20835|AAAAAAAADGBFAAAA|710|2nd |Wy|Suite C|Fairfield|Peoria County|IL|66192|United States|-6|condo| +20836|AAAAAAAAEGBFAAAA|416|1st 1st|Pkwy|Suite N|Hopewell|Radford city|VA|20587|United States|-5|single family| +20837|AAAAAAAAFGBFAAAA|976|Hickory |Parkway|Suite 380|Crossroads|Marshall County|IN|40534|United States|-5|apartment| +20838|AAAAAAAAGGBFAAAA|511|6th |Street|Suite 430|Oakland|Los Alamos County|NM|89843|United States|-7|condo| +20839|AAAAAAAAHGBFAAAA|918|Sixth |Ct.|Suite 140|Oak Ridge|Warren County|IA|58371|United States|-6|condo| +20840|AAAAAAAAIGBFAAAA|334|7th ||Suite 460|Greenwood|Madison County||||-5|apartment| +20841|AAAAAAAAJGBFAAAA|350|Woodland 1st|Ave|Suite 110|Lakeside|Jefferson County|NY|19532|United States|-5|apartment| +20842|AAAAAAAAKGBFAAAA|507|Third |Ct.|Suite 210|Springfield|Burnett County|WI|59303|United States|-6|single family| +20843|AAAAAAAALGBFAAAA|856|Hickory Park|Avenue|Suite I|Hillcrest|Blue Earth County|MN|53003|United States|-6|condo| +20844|AAAAAAAAMGBFAAAA|833|Cherry Lake|Circle|Suite Y|Tipton|Corson County|SD|58773|United States|-6|condo| +20845|AAAAAAAANGBFAAAA|654|East |Dr.|Suite B|Glendale|Choctaw County|OK|73951|United States|-6|single family| +20846|AAAAAAAAOGBFAAAA|442|Woodland North|Street|Suite Y|Howell|Crosby County|TX|74854|United States|-6|single family| +20847|AAAAAAAAPGBFAAAA|56|Church Seventh|Ave|Suite 440|Maple Grove|Shelby County|AL|38252|United States|-6|condo| +20848|AAAAAAAAAHBFAAAA|728|13th |Drive|Suite G|Macedonia|Lincoln County|AR|71087|United States|-6|single family| +20849|AAAAAAAABHBFAAAA|88|3rd |Ave|Suite J|Riverside|Irion County|TX|79231|United States|-6|single family| +20850|AAAAAAAACHBFAAAA|63|Walnut 3rd|Lane|Suite 260|Oakland|Bucks County|PA|19843|United States|-5|apartment| +20851|AAAAAAAADHBFAAAA|643|Oak South|Dr.|Suite P|Forest Hills|Archer County|TX|79237|United States|-6|condo| +20852|AAAAAAAAEHBFAAAA|936|8th |Circle|Suite 380|Green Acres|Sullivan County|IN|47683|United States|-5|apartment| +20853|AAAAAAAAFHBFAAAA|800|Third |Street|Suite W|Clifton|Greene County|VA|28014|United States|-5|condo| +20854|AAAAAAAAGHBFAAAA|298|East |Parkway|Suite A|Marion|Wright County|MN|50399|United States|-6|apartment| +20855|AAAAAAAAHHBFAAAA|113|Wilson |Blvd|Suite 150|Woodland|Greenup County|KY|44854|United States|-6|single family| +20856|AAAAAAAAIHBFAAAA|819|5th Pine|Cir.|Suite N|Oakwood|Union County|TN|30169|United States|-6|apartment| +20857|AAAAAAAAJHBFAAAA|477|Pine Dogwood|Street|Suite F|Pine Grove|San Bernardino County|CA|94593|United States|-8|single family| +20858|AAAAAAAAKHBFAAAA|681|Green |ST|Suite A|Newport|Midland County|MI|41521|United States|-5|apartment| +20859|AAAAAAAALHBFAAAA|982|East Lake|Ln|Suite 120|Spring Valley|Otoe County|NE|66060|United States|-7|single family| +20860|AAAAAAAAMHBFAAAA|754|Jackson 9th|Avenue|Suite 350|Mount Olive|Dinwiddie County|VA|28059|United States|-5|apartment| +20861|AAAAAAAANHBFAAAA|688|Hill |Pkwy|Suite U|Fairview|Clark County|AR|75709|United States|-6|single family| +20862|AAAAAAAAOHBFAAAA|725|South 15th|Drive|Suite M|Oak Grove|Minidoka County|ID|88370|United States|-7|condo| +20863|AAAAAAAAPHBFAAAA|564|Spruce |Blvd|Suite F|Bridgeport|Watonwan County|MN|55817|United States|-6|single family| +20864|AAAAAAAAAIBFAAAA|449|Jackson 2nd|Cir.|Suite 430|Salem|Greensville County|VA|28048|United States|-5|single family| +20865|AAAAAAAABIBFAAAA|530|Birch |Pkwy|Suite Y|Peoria|Fairfield County|CT|09818|United States|-5|condo| +20866|AAAAAAAACIBFAAAA|376|Fourth |Lane|Suite 370|Lakeside|Monroe County|NY|19532|United States|-5|apartment| +20867|AAAAAAAADIBFAAAA|207|5th 10th|Wy|Suite O|Glenwood|Decatur County|IA|53511|United States|-6|single family| +20868|AAAAAAAAEIBFAAAA|74|12th |Avenue|Suite C|Valley View|Daniels County|MT|65124|United States|-7|apartment| +20869|AAAAAAAAFIBFAAAA|701|North Highland|Court|Suite 380|Spring Valley|Butte County|CA|96060|United States|-8|apartment| +20870|AAAAAAAAGIBFAAAA|832|Elm Ridge|Blvd|Suite 200|Woodville|Orange County|VA|24289|United States|-5|condo| +20871|AAAAAAAAHIBFAAAA|107|7th Hill|Ln|Suite U|Pomona|Portsmouth city|VA|24153|United States|-5|apartment| +20872|AAAAAAAAIIBFAAAA|776|Locust Second|Pkwy|Suite 150|Riverview|Gilliam County|OR|99003|United States|-8|apartment| +20873|AAAAAAAAJIBFAAAA|812|Forest Jefferson|Wy|Suite B|Cedar Grove|Jackson County|IA|50411|United States|-6|condo| +20874|AAAAAAAAKIBFAAAA|24|Jackson |Drive|Suite A|Lakeview|Union County|KY|48579|United States|-5|single family| +20875|AAAAAAAALIBFAAAA|967|Forest |RD|Suite 90|Clinton|Rock County|NE|68222|United States|-7|condo| +20876|AAAAAAAAMIBFAAAA|414||||Belmont|Riley County||60191||-6|condo| +20877|AAAAAAAANIBFAAAA|28|6th Johnson|RD|Suite 270|Fairview|Jackson County|AR|75709|United States|-6|condo| +20878|AAAAAAAAOIBFAAAA|672|East Oak|Avenue|Suite U|Hardy|Fayette County|PA|15354|United States|-5|condo| +20879|AAAAAAAAPIBFAAAA|598|South Cedar|Way|Suite K|Maple Grove|Hancock County|ME|08852|United States|-5|apartment| +20880|AAAAAAAAAJBFAAAA|496|12th 12th|Boulevard|Suite M|Mount Pleasant|Hennepin County|MN|51933|United States|-6|apartment| +20881|AAAAAAAABJBFAAAA|161|5th Laurel|Way|Suite J|Washington Heights|Wichita County|TX|78167|United States|-6|apartment| +20882|AAAAAAAACJBFAAAA|29|Sycamore Davis|Circle|Suite B|Pleasant Valley|Apache County|AZ|82477|United States|-7|single family| +20883|AAAAAAAADJBFAAAA|917|Spring Main|Road|Suite U|Oakdale|Monroe County|AR|79584|United States|-6|apartment| +20884|AAAAAAAAEJBFAAAA|670|Jefferson Lake|Road|Suite 190|White Hall|Muskogee County|OK|76955|United States|-6|apartment| +20885|AAAAAAAAFJBFAAAA|711|Pine |Avenue|Suite Q|Mount Olive|Crook County|WY|88059|United States|-7|apartment| +20886|AAAAAAAAGJBFAAAA|311|Cedar First|Drive|Suite 370|Forest Hills|Levy County|FL|39237|United States|-5|apartment| +20887|AAAAAAAAHJBFAAAA|452|Hill |Road|Suite U|Galena|Menifee County|KY|44369|United States|-5|condo| +20888|AAAAAAAAIJBFAAAA|134|Spring |Way|Suite V|Jamestown|San Bernardino County|CA|96867|United States|-8|condo| +20889|AAAAAAAAJJBFAAAA|947|3rd Willow|Drive|Suite 450|Antioch|Dillon County|SC|28605|United States|-5|single family| +20890|AAAAAAAAKJBFAAAA|224|Pine Elm|Avenue|Suite 430|Friendship|Sumner County|KS|64536|United States|-6|condo| +20891|AAAAAAAALJBFAAAA|371|East |Dr.|Suite S|Bridgeport|Bennett County|SD|55817|United States|-6|apartment| +20892|AAAAAAAAMJBFAAAA|40|West |Pkwy|Suite V|Alma|Cameron County|TX|70244|United States|-6|apartment| +20893|AAAAAAAANJBFAAAA|268|Sunset |Ln|Suite 470|Brunswick|Lander County|NV|84642|United States|-8|condo| +20894|AAAAAAAAOJBFAAAA|77|Spruce Jefferson|Road|Suite 250|Marion|Stevens County|MN|50399|United States|-6|condo| +20895|AAAAAAAAPJBFAAAA|791||||Walnut Grove||FL|37752||-5|apartment| +20896|AAAAAAAAAKBFAAAA|261|Lake |Ln|Suite 20|Sunnyside|Hamilton County|TN|31952|United States|-5|single family| +20897|AAAAAAAABKBFAAAA|479|10th Valley|Boulevard|Suite 110|Brownsville|Elk County|KS|69310|United States|-6|condo| +20898|AAAAAAAACKBFAAAA|972|Sycamore Ridge|Circle|Suite B|Vance|Coahoma County|MS|50268|United States|-6|single family| +20899|AAAAAAAADKBFAAAA|239|Laurel Ridge|Circle|Suite 340|Spring Hill|Pulaski County|GA|36787|United States|-5|single family| +20900|AAAAAAAAEKBFAAAA|286|View Second|Road|Suite B|Dallas|Albemarle County|VA|23628|United States|-5|single family| +20901|AAAAAAAAFKBFAAAA|652|Pine |RD|Suite 230|White Oak|Coryell County|TX|76668|United States|-6|condo| +20902|AAAAAAAAGKBFAAAA|756|View |Ln|Suite H|Brownsville|Faribault County|MN|59310|United States|-6|apartment| +20903|AAAAAAAAHKBFAAAA|23|Smith |Parkway|Suite T|Wilson|Powhatan County|VA|26971|United States|-5|condo| +20904|AAAAAAAAIKBFAAAA||Hill Wilson|Parkway|Suite S||||||-6|single family| +20905|AAAAAAAAJKBFAAAA|532|North 3rd|ST|Suite 170|Kingston|Allen County|KS|64975|United States|-6|single family| +20906|AAAAAAAAKKBFAAAA|251|View |Ln|Suite 470|Berea|Price County|WI|53804|United States|-6|single family| +20907|AAAAAAAALKBFAAAA|510|Sunset 6th|Lane|Suite 310|Fairfield|Kent County|MI|46192|United States|-5|apartment| +20908|AAAAAAAAMKBFAAAA|578|Lee 14th|RD|Suite L|Woodland|Clinton County|NY|14854|United States|-5|single family| +20909|AAAAAAAANKBFAAAA|411|Mill |Dr.|Suite 30|Dewey|Duchesne County|UT|81160|United States|-7|single family| +20910|AAAAAAAAOKBFAAAA|351|Fourth |Way|Suite N|Hurricane|Yuba County|CA|97644|United States|-8|single family| +20911|AAAAAAAAPKBFAAAA|941|Park Smith|RD|Suite 10|Walnut Grove|Maury County|TN|37752|United States|-6|condo| +20912|AAAAAAAAALBFAAAA|995|5th Willow|Blvd|Suite B|Lakewood|Moore County|TN|38877|United States|-6|apartment| +20913|AAAAAAAABLBFAAAA|238|||Suite T|Spring Valley|Claiborne County||56060|||single family| +20914|AAAAAAAACLBFAAAA|884|Sixth |Blvd|Suite Q|Lebanon|Garland County|AR|72898|United States|-6|condo| +20915|AAAAAAAADLBFAAAA|344|Lakeview Railroad|Dr.|Suite 120|Maple Grove|Warren County|TN|38252|United States|-6|single family| +20916|AAAAAAAAELBFAAAA|80|Mill |ST|Suite 60|Oakdale|Colfax County|NM|89584|United States|-7|single family| +20917|AAAAAAAAFLBFAAAA|553|13th |RD|Suite J|Maple Grove|Scott County|TN|38252|United States|-6|condo| +20918|AAAAAAAAGLBFAAAA|659|Lake |Parkway|Suite Y|Lakeview|Hunterdon County|NJ|09179|United States|-5|single family| +20919|AAAAAAAAHLBFAAAA|608|Main |Road|Suite 180|Bunker Hill|Genesee County|MI|40150|United States|-5|condo| +20920|AAAAAAAAILBFAAAA|828|Cedar Willow|Blvd|Suite P|Shiloh|Perry County|MS|59275|United States|-6|condo| +20921|AAAAAAAAJLBFAAAA|298||Ln||White Oak|Pinellas County|FL||||| +20922|AAAAAAAAKLBFAAAA|659|Smith Pine|Blvd|Suite 30|Hurricane|Cedar County|NE|67644|United States|-6|condo| +20923|AAAAAAAALLBFAAAA|97|4th |Pkwy|Suite 260|Florence|Yellowstone County|MT|63394|United States|-7|apartment| +20924|AAAAAAAAMLBFAAAA|783|Third |RD|Suite 360|Walnut Grove|Daviess County|IN|47752|United States|-5|condo| +20925|AAAAAAAANLBFAAAA|704|College |RD|Suite 80|Oakdale|Hardin County|IA|59584|United States|-6|apartment| +20926|AAAAAAAAOLBFAAAA|798|Willow |Wy|Suite X|Belmont|Laurel County|KY|40191|United States|-5|apartment| +20927|AAAAAAAAPLBFAAAA|800|Seventh |Blvd|Suite 490|Rochester|Henry County|TN|31383|United States|-5|condo| +20928|AAAAAAAAAMBFAAAA|336|Jackson Cherry|Wy|Suite K|Hubbard|Columbia County|NY|16291|United States|-5|single family| +20929|AAAAAAAABMBFAAAA|830|View |Way|Suite X|Mount Olive|Simpson County|MS|58059|United States|-6|condo| +20930|AAAAAAAACMBFAAAA|627|Adams |Parkway|Suite V|Jackson|Cherry County|NE|69583|United States|-6|condo| +20931|AAAAAAAADMBFAAAA|788|Hillcrest |Street|Suite 80|Mount Olive|Ashe County|NC|28059|United States|-5|single family| +20932|AAAAAAAAEMBFAAAA|959|Railroad Franklin|Court|Suite U|Valley View|Kingman County|KS|65124|United States|-6|condo| +20933|AAAAAAAAFMBFAAAA|450|Sixth Park|RD|Suite J|Sunnyside|Colorado County|TX|71952|United States|-6|condo| +20934|AAAAAAAAGMBFAAAA|522|Jefferson Miller|Dr.|Suite 210|Lakewood|Scott County|KY|48877|United States|-5|condo| +20935|AAAAAAAAHMBFAAAA|421||Wy|Suite U|Sulphur Springs||IA|58354|United States||| +20936|AAAAAAAAIMBFAAAA|432|Fifth |Circle|Suite R|Hopewell|Natchitoches Parish|LA|70587|United States|-6|single family| +20937|AAAAAAAAJMBFAAAA|467|North |Ln|Suite Y|Pine Grove|Valley County|NE|64593|United States|-7|condo| +20938|AAAAAAAAKMBFAAAA|871|Main 5th|Blvd|Suite 40|Bunker Hill|Sabine Parish|LA|70150|United States|-6|apartment| +20939|AAAAAAAALMBFAAAA|331|First Main|Court||Arlington|Newton County|MO|66557|United States|-6|| +20940|AAAAAAAAMMBFAAAA|62|South 1st|RD|Suite N|Crossroads|Ritchie County|WV|20534|United States|-5|single family| +20941|AAAAAAAANMBFAAAA|386|Pine Oak|RD|Suite 490|Riverdale|Douglas County|MN|59391|United States|-6|condo| +20942|AAAAAAAAOMBFAAAA|358|Poplar |Circle|Suite 130|Stringtown|Pulaski County|GA|30162|United States|-5|apartment| +20943|AAAAAAAAPMBFAAAA|879|West |Ave|Suite G|Buena Vista|Queens County|NY|15752|United States|-5|apartment| +20944|AAAAAAAAANBFAAAA|67|Franklin |Way|Suite 390|Arlington|Bandera County|TX|76557|United States|-6|condo| +20945|AAAAAAAABNBFAAAA|162|Smith |Way|Suite 20|Riverside|Union County|KY|49231|United States|-5|single family| +20946|AAAAAAAACNBFAAAA|939|Cedar |Road|Suite R|Tremont|Greensville County|VA|29515|United States|-5|condo| +20947|AAAAAAAADNBFAAAA|474|Spring |Ct.|Suite A|Oakwood|Butte County|SD|50169|United States|-6|condo| +20948|AAAAAAAAENBFAAAA|619|Forest |Way|Suite 250|Wilson|Tom Green County|TX|76971|United States|-6|single family| +20949|AAAAAAAAFNBFAAAA|109|Fifth |Ave|Suite Y|Fairview|King County|WA|95709|United States|-8|apartment| +20950|AAAAAAAAGNBFAAAA|830|Church |Ct.|Suite 470|Concord|Marathon County|WI|54107|United States|-6|apartment| +20951|AAAAAAAAHNBFAAAA|781|Birch |Parkway|Suite M|Highland|Martin County|NC|29454|United States|-5|condo| +20952|AAAAAAAAINBFAAAA|254|14th Washington|Dr.|Suite A|Springfield|Warren County|NY|19303|United States|-5|single family| +20953|AAAAAAAAJNBFAAAA|407|Broadway 10th|Ln|Suite 210|Green Acres|Hyde County|SD|57683|United States|-7|condo| +20954|AAAAAAAAKNBFAAAA|253|Center |Blvd|Suite S|Sunnyside|Lexington city|VA|21952|United States|-5|single family| +20955|AAAAAAAALNBFAAAA|172|Pine Lake|Street|Suite I|Georgetown|Sevier County|AR|77057|United States|-6|apartment| +20956|AAAAAAAAMNBFAAAA|688|Meadow |RD|Suite 470|Cedar Grove|Dickinson County|KS|60411|United States|-6|single family| +20957|AAAAAAAANNBFAAAA|53|Birch Cedar|Street|Suite 120|Crestview|Deuel County|NE|61421|United States|-6|single family| +20958|AAAAAAAAONBFAAAA|170|Park |Boulevard|Suite 60|Bridgeport|Crawford County|MI|45817|United States|-5|single family| +20959|AAAAAAAAPNBFAAAA||6th |Boulevard|Suite S|||||United States||| +20960|AAAAAAAAAOBFAAAA|396|Ridge 6th|Court|Suite Q|Centerville|Licking County|OH|40059|United States|-5|condo| +20961|AAAAAAAABOBFAAAA|56|Dogwood View|RD|Suite 490|Lake Forest|Blount County|TN|36000|United States|-5|apartment| +20962|AAAAAAAACOBFAAAA|981|Franklin |Dr.|Suite 210|Bethel|Clarke County|GA|35281|United States|-5|condo| +20963|AAAAAAAADOBFAAAA|220|Laurel 7th|Cir.|Suite E|Lebanon|King George County|VA|22898|United States|-5|condo| +20964|AAAAAAAAEOBFAAAA|104|1st |Avenue|Suite 430|Marion|Burke County|NC|20399|United States|-5|condo| +20965|AAAAAAAAFOBFAAAA|867|West |Avenue|Suite 360|Green Acres|Grays Harbor County|WA|97683|United States|-8|condo| +20966|AAAAAAAAGOBFAAAA|981|Cherry |Dr.|Suite 230|Lincoln|Salem County|NJ|01889|United States|-5|condo| +20967|AAAAAAAAHOBFAAAA|552|Smith |Drive|Suite 80|Salem|Chariton County|MO|68048|United States|-6|single family| +20968|AAAAAAAAIOBFAAAA|377|Lake |Circle|Suite 120|Forest Hills|Hancock County|MS|59237|United States|-6|condo| +20969|AAAAAAAAJOBFAAAA|570|Sycamore |Wy|Suite P|Green Acres|Flagler County|FL|37683|United States|-5|single family| +20970|AAAAAAAAKOBFAAAA|220|Elm |Cir.|Suite 210|Clifton|Douglas County|MO|68014|United States|-6|apartment| +20971|AAAAAAAALOBFAAAA|551|Johnson Poplar|Boulevard|Suite 250|Walnut Grove|Brooks County|TX|77752|United States|-6|condo| +20972|AAAAAAAAMOBFAAAA|933|Adams Second|RD|Suite 380|Green Acres|Teller County|CO|87683|United States|-7|condo| +20973|AAAAAAAANOBFAAAA|429|Walnut |Drive|Suite 270|Mount Vernon|Cowley County|KS|68482|United States|-6|single family| +20974|AAAAAAAAOOBFAAAA|750|Oak South|Circle|Suite 280|Plainview|Madison County|NY|13683|United States|-5|single family| +20975|AAAAAAAAPOBFAAAA|149|6th |Way|Suite 130|Forest Hills|Russell County|KS|69237|United States|-6|apartment| +20976|AAAAAAAAAPBFAAAA|790|Sycamore |Circle|Suite K|Liberty|Sharp County|AR|73451|United States|-6|apartment| +20977|AAAAAAAABPBFAAAA|660|Tenth College|Pkwy|Suite S|Centerville|Newton County|GA|30059|United States|-5|apartment| +20978|AAAAAAAACPBFAAAA|233|9th |Avenue|Suite C|Hillcrest|Barbour County|AL|33003|United States|-6|condo| +20979|AAAAAAAADPBFAAAA|513|North |Circle|Suite O|Oakdale|Adams County|IN|49584|United States|-5|apartment| +20980|AAAAAAAAEPBFAAAA|342|9th West|RD|Suite 440|Sunnyside|Meigs County|OH|41952|United States|-5|condo| +20981|AAAAAAAAFPBFAAAA|146|Fourth |Avenue|Suite 200|Shiloh|Chambers County|TX|79275|United States|-6|single family| +20982|AAAAAAAAGPBFAAAA|386|Lee Fourth|Ave|Suite I|Lincoln|Sutton County|TX|71289|United States|-6|apartment| +20983|AAAAAAAAHPBFAAAA|452|Highland Lake|Road|Suite H|River Oaks|Grant County|SD|58075|United States|-6|apartment| +20984|AAAAAAAAIPBFAAAA|202|Second First|Cir.|Suite F|Lebanon|Ray County|MO|62898|United States|-6|single family| +20985|AAAAAAAAJPBFAAAA|31|Hickory Park|Dr.|Suite M|Bear Creek|Manassas Park city|VA|23075|United States|-5|condo| +20986|AAAAAAAAKPBFAAAA|614|Lake 8th|Blvd|Suite 330|Friendship|Union County|SC|24536|United States|-5|single family| +20987|AAAAAAAALPBFAAAA|343|3rd Sycamore|Cir.|Suite G|Macedonia|Decatur County|GA|31087|United States|-5|single family| +20988|AAAAAAAAMPBFAAAA||||Suite 330|Elba|De Soto Parish|LA||United States||| +20989|AAAAAAAANPBFAAAA|715|Franklin Park|Street|Suite 170|Jones|Chambers County|TX|72686|United States|-6|single family| +20990|AAAAAAAAOPBFAAAA|116|Jefferson |Boulevard||Mount Pleasant|Goodhue County||51933|United States||condo| +20991|AAAAAAAAPPBFAAAA|291|Washington |RD|Suite U|Farmington|Marshall County|SD|59145|United States|-7|condo| +20992|AAAAAAAAAACFAAAA|421|Pine |Ln|Suite 40|Arlington|Pontotoc County|OK|76557|United States|-6|apartment| +20993|AAAAAAAABACFAAAA|371|3rd Church|||Forest|La Porte County||47537||-5|single family| +20994|AAAAAAAACACFAAAA|198|Hickory Cherry|Parkway|Suite 330|Glendale|Pittsylvania County|VA|23951|United States|-5|single family| +20995|AAAAAAAADACFAAAA|614|1st Washington|Cir.|Suite 330|Antioch|Arapahoe County|CO|88605|United States|-7|condo| +20996|AAAAAAAAEACFAAAA|888|Eigth |Parkway|Suite 70|White Oak|Humboldt County|IA|56668|United States|-6|apartment| +20997|AAAAAAAAFACFAAAA|545|Woodland Meadow|ST|Suite 20|Oak Hill|San Benito County|CA|97838|United States|-8|apartment| +20998|AAAAAAAAGACFAAAA|488|Seventh 9th|Street|Suite 190|Lebanon|Luna County|NM|82898|United States|-7|single family| +20999|AAAAAAAAHACFAAAA|239|West |Dr.|Suite B|Riverside|Noxubee County|MS|59231|United States|-6|condo| +21000|AAAAAAAAIACFAAAA|442|Center |Court|Suite 460|Hopewell|Sheridan County|WY|80587|United States|-7|condo| +21001|AAAAAAAAJACFAAAA|983|Main Seventh|Way|Suite 130|Waterloo|Ward County|TX|71675|United States|-6|condo| +21002|AAAAAAAAKACFAAAA|317|North Central|Pkwy|Suite 220|Riverside|Mitchell County|GA|39231|United States|-5|single family| +21003|AAAAAAAALACFAAAA|813|River |Way|Suite 470|Mount Zion|Madison County|VA|28054|United States|-5|condo| +21004|AAAAAAAAMACFAAAA|165|8th Mill|Court|Suite M|Mount Pleasant|Saline County|MO|61933|United States|-6|apartment| +21005|AAAAAAAANACFAAAA|738|Laurel Park|Parkway|Suite 180|Oak Hill|Washington County|ME|08438|United States|-5|single family| +21006|AAAAAAAAOACFAAAA|270|Adams |Dr.|Suite 380|Bethel|Ashley County|AR|75281|United States|-6|apartment| +21007|AAAAAAAAPACFAAAA|467|Miller |Avenue|Suite 120|Freeman|Howard County|IA|52297|United States|-6|apartment| +21008|AAAAAAAAABCFAAAA|452|Hickory |Road|Suite 160|Pleasant Grove|Isle of Wight County|VA|24136|United States|-5|apartment| +21009|AAAAAAAABBCFAAAA|62|Woodland |Boulevard|Suite Q|Clinton|Grand Isle County|VT|08822|United States|-5|apartment| +21010|AAAAAAAACBCFAAAA|854|Laurel Railroad|Cir.|Suite S|Newport|Wood County|OH|41521|United States|-5|single family| +21011|AAAAAAAADBCFAAAA|590|Dogwood |Blvd|Suite 280|Valley View|Todd County|MN|55124|United States|-6|single family| +21012|AAAAAAAAEBCFAAAA|654|5th |Blvd|Suite P|Smith|DeKalb County|AL|37317|United States|-6|condo| +21013|AAAAAAAAFBCFAAAA|664|Walnut |Drive|Suite 460|Summit|Lemhi County|ID|80499|United States|-7|apartment| +21014|AAAAAAAAGBCFAAAA|839|4th |Ln|Suite 60|Hillcrest|Lampasas County|TX|73003|United States|-6|condo| +21015|AAAAAAAAHBCFAAAA|925|Ridge Eigth|Lane|Suite 380|Mount Zion|Lampasas County|TX|78054|United States|-6|condo| +21016|AAAAAAAAIBCFAAAA|349|First |Ct.|Suite 20|Valley View|Staunton city|VA|25124|United States|-5|condo| +21017|AAAAAAAAJBCFAAAA|411|Church View|Drive|Suite 370|Oak Hill|Adair County|MO|67838|United States|-6|single family| +21018|AAAAAAAAKBCFAAAA|32|Fourth |Avenue|Suite 40|Centerville|Wayne County|WV|20059|United States|-5|condo| +21019|AAAAAAAALBCFAAAA|820|Maple View|Blvd|Suite 250|White Oak|Franklin Parish|LA|76668|United States|-6|apartment| +21020|AAAAAAAAMBCFAAAA|98|Meadow Second|Avenue|Suite B|Greenville|Mississippi County|AR|71387|United States|-6|apartment| +21021|AAAAAAAANBCFAAAA|948|Davis 6th|Blvd|Suite F|Valley View|Quitman County|GA|35124|United States|-5|apartment| +21022|AAAAAAAAOBCFAAAA|569|Williams |Court|Suite M|Edgewood|Fayette County|IA|50069|United States|-6|condo| +21023|AAAAAAAAPBCFAAAA|353|Spring |Blvd|Suite E|Jerome|Imperial County|CA|99920|United States|-8|single family| +21024|AAAAAAAAACCFAAAA|618|Fourth Park|Drive|Suite A|New Hope|Tehama County|CA|99431|United States|-8|condo| +21025|AAAAAAAABCCFAAAA|180|1st |Wy|Suite L|Newport|Benton County|IA|51521|United States|-6|apartment| +21026|AAAAAAAACCCFAAAA|83|13th |Avenue|Suite 40|Mount Olive|Koochiching County|MN|58059|United States|-6|single family| +21027|AAAAAAAADCCFAAAA|786|Oak 5th|||||||||condo| +21028|AAAAAAAAECCFAAAA|456|Pine |Ct.|Suite D|Oakwood|Johnston County|NC|20169|United States|-5|single family| +21029|AAAAAAAAFCCFAAAA|275|Lincoln |ST|Suite 130|Five Forks|Ellis County|OK|72293|United States|-6|apartment| +21030|AAAAAAAAGCCFAAAA|134|First Chestnut|Parkway|Suite 70|Concord|Wichita County|TX|74107|United States|-6|condo| +21031|AAAAAAAAHCCFAAAA|210|Eigth Ash|Avenue|Suite 420|Belleview|Pulaski County|MO|60492|United States|-6|single family| +21032|AAAAAAAAICCFAAAA|482|Park Ash|Ct.|Suite O|Lone Oak|Poinsett County|AR|76893|United States|-6|apartment| +21033|AAAAAAAAJCCFAAAA|63|Maple Johnson|Avenue|Suite A|Woodlawn|Macon County|NC|24098|United States|-5|condo| +21034|AAAAAAAAKCCFAAAA|619|Center |Ln|Suite 100|Crossroads|Langlade County|WI|50534|United States|-6|apartment| +21035|AAAAAAAALCCFAAAA|254|Williams Locust|Court|Suite I|Franklin|Putnam County|WV|29101|United States|-5|single family| +21036|AAAAAAAAMCCFAAAA|645|Central |Ct.|Suite 300|Sunnyside|Allen County|KY|41952|United States|-6|single family| +21037|AAAAAAAANCCFAAAA|849|9th |Cir.|Suite 210|Deerfield|Otoe County|NE|69840|United States|-7|condo| +21038|AAAAAAAAOCCFAAAA|198|1st |Boulevard|Suite 300|Arlington|Crawford County|IN|46557|United States|-5|single family| +21039|AAAAAAAAPCCFAAAA|667|Miller |Ct.|Suite T|Buena Vista|Jackson County|AL|35752|United States|-6|single family| +21040|AAAAAAAAADCFAAAA|181|College Laurel|Street|Suite T|Five Forks|Garvin County|OK|72293|United States|-6|apartment| +21041|AAAAAAAABDCFAAAA|295|Ninth |Boulevard|Suite F|Marion|Avery County|NC|20399|United States|-5|apartment| +21042|AAAAAAAACDCFAAAA|61|Cedar |Ln|Suite 340|Calhoun|Hockley County|TX|76909|United States|-6|apartment| +21043|AAAAAAAADDCFAAAA|880|Davis |Circle|Suite 490|Arlington|Las Animas County|CO|86557|United States|-7|condo| +21044|AAAAAAAAEDCFAAAA|520|Cherry |Dr.|Suite O|Edgewood|Lee County|SC|20069|United States|-5|apartment| +21045|AAAAAAAAFDCFAAAA|187|Miller Lincoln|Cir.|Suite 430|Hamilton|Shelby County|OH|42808|United States|-5|condo| +21046|AAAAAAAAGDCFAAAA|65|Miller |Ln|Suite 100|Franklin|Rockwall County|TX|79101|United States|-6|apartment| +21047|AAAAAAAAHDCFAAAA|631|Maple |Boulevard|Suite 120|Waterloo|Bee County|TX|71675|United States|-6|single family| +21048|AAAAAAAAIDCFAAAA|56|Smith Washington|Street|Suite 180|Union Hill|Fremont County|WY|87746|United States|-7|single family| +21049|AAAAAAAAJDCFAAAA|693|8th Walnut|Circle|Suite D|Oakwood|Limestone County|AL|30169|United States|-6|single family| +21050|AAAAAAAAKDCFAAAA|628|Meadow |Avenue|Suite 340|Maple Grove|Winnebago County|WI|58252|United States|-6|single family| +21051|AAAAAAAALDCFAAAA|444|Spruce Oak|Road|Suite 330|New Hope|Laurel County|KY|49431|United States|-5|apartment| +21052|AAAAAAAAMDCFAAAA|338|Oak Maple|Wy|Suite H|Cedar Grove|Nacogdoches County|TX|70411|United States|-6|condo| +21053|AAAAAAAANDCFAAAA|335|Poplar |Ct.|Suite T|Green Acres|San Juan County|NM|87683|United States|-7|single family| +21054|AAAAAAAAODCFAAAA|194|Oak |Court|Suite 350|Cedar Grove|Jones County|SD|50411|United States|-7|apartment| +21055|AAAAAAAAPDCFAAAA|742|North |Pkwy|Suite Y|Springfield|Portage County|WI|59303|United States|-6|apartment| +21056|AAAAAAAAAECFAAAA|574|Lincoln |Court|Suite Y|Cedar Grove|Gila County|AZ|80411|United States|-7|condo| +21057|AAAAAAAABECFAAAA|363|1st |RD|Suite 410|Sulphur Springs|Iredell County|NC|28354|United States|-5|apartment| +21058|AAAAAAAACECFAAAA|963|College |Ct.|Suite Q|Salem|Russell County|VA|28048|United States|-5|apartment| +21059|AAAAAAAADECFAAAA|917|Broadway Elm|Boulevard|Suite X|Kingston|Kennebec County|ME|05575|United States|-5|condo| +21060|AAAAAAAAEECFAAAA|49|4th 2nd|Drive|Suite 400|Oak Grove|Weakley County|TN|38370|United States|-6|apartment| +21061|AAAAAAAAFECFAAAA|734|Lincoln |Pkwy|Suite 490|Springfield|Gulf County|FL|39303|United States|-5|single family| +21062|AAAAAAAAGECFAAAA|684|River Meadow|ST|Suite 180|Georgetown|Thurston County|NE|67057|United States|-7|apartment| +21063|AAAAAAAAHECFAAAA|655|Cedar |Avenue|Suite 440|Pleasant Valley|Livingston County|MO|62477|United States|-6|condo| +21064|AAAAAAAAIECFAAAA|985|Broadway |Cir.|Suite 100|Highland Park|Beaver County|PA|16534|United States|-5|condo| +21065|AAAAAAAAJECFAAAA|502|Lake Valley|RD|Suite X|Oakland|Sawyer County|WI|59843|United States|-6|single family| +21066|AAAAAAAAKECFAAAA|619|View Railroad|Parkway|Suite 60|Rankin|Alleghany County|VA|22621|United States|-5|condo| +21067|AAAAAAAALECFAAAA|574|Fourth Third|Avenue|Suite 260|Valley View|Oakland County|MI|45124|United States|-5|single family| +21068|AAAAAAAAMECFAAAA|160|Highland |Ct.|Suite 450|Glenwood|Cherokee County|KS|63511|United States|-6|condo| +21069|AAAAAAAANECFAAAA|740|12th Park|Drive|Suite 390|Arlington|Butler County|KS|66557|United States|-6|apartment| +21070|AAAAAAAAOECFAAAA|580|1st Jackson|Circle|Suite 320|Tipton|White County|TN|38773|United States|-6|apartment| +21071|AAAAAAAAPECFAAAA|458|Jefferson |Lane|Suite 210|Highland Park|Robeson County|NC|26534|United States|-5|apartment| +21072|AAAAAAAAAFCFAAAA|730|Smith Sunset|Way|Suite L|Lakeside|Kalkaska County|MI|49532|United States|-5|single family| +21073|AAAAAAAABFCFAAAA|757|Walnut Spring|Circle|Suite 470|Cedar Grove|Lane County|OR|90411|United States|-8|apartment| +21074|AAAAAAAACFCFAAAA|919|Elm |Ln|Suite 410|Enterprise|Sequoyah County|OK|71757|United States|-6|apartment| +21075|AAAAAAAADFCFAAAA|251|Cedar 2nd|ST|Suite Y|Kingston|Erath County|TX|74975|United States|-6|single family| +21076|AAAAAAAAEFCFAAAA|50|4th Oak|Avenue|Suite G|Antioch|Montgomery County|NC|28605|United States|-5|condo| +21077|AAAAAAAAFFCFAAAA|477|2nd |RD|Suite 200|Lebanon|Bucks County|PA|12898|United States|-5|single family| +21078|AAAAAAAAGFCFAAAA|930|Williams |Circle|Suite 330|Bayside|Hawaii County|HI|99550|United States|-10|single family| +21079|AAAAAAAAHFCFAAAA|490|Mill View|Avenue|Suite 360|Woodland|Bibb County|GA|34854|United States|-5|apartment| +21080|AAAAAAAAIFCFAAAA|94|Lee |Pkwy|Suite 220|Belmont|Columbia County|WI|50191|United States|-6|single family| +21081|AAAAAAAAJFCFAAAA|691|Miller |Lane|Suite 220|Greenwood|Moultrie County|IL|68828|United States|-6|single family| +21082|AAAAAAAAKFCFAAAA|719|2nd 4th|Ln|Suite P|Concord|Coos County|NH|04707|United States|-5|condo| +21083|AAAAAAAALFCFAAAA|130|North 2nd|Street|Suite 300|Oakdale|Henry County|IN|49584|United States|-5|single family| +21084|AAAAAAAAMFCFAAAA|412|4th |Ave|Suite 20|Cedar Grove|Juab County|UT|80411|United States|-7|condo| +21085|AAAAAAAANFCFAAAA|364|Hickory |Road|Suite S|Shady Grove|Cherokee County|AL|32812|United States|-6|apartment| +21086|AAAAAAAAOFCFAAAA|356|5th |RD|Suite 480|Macedonia|Gage County|NE|61087|United States|-6|condo| +21087|AAAAAAAAPFCFAAAA|739|Seventh |Way|Suite 480|Maple Grove|Real County|TX|78252|United States|-6|apartment| +21088|AAAAAAAAAGCFAAAA|998|1st |RD|Suite 330|Red Hill|Sagadahoc County|ME|04938|United States|-5|apartment| +21089|AAAAAAAABGCFAAAA|550|Lake |Lane|Suite G|Antioch|Dubuque County|IA|58605|United States|-6|single family| +21090|AAAAAAAACGCFAAAA|730|Church |Ct.|Suite 450|Friendship|Monroe County|PA|14536|United States|-5|single family| +21091|AAAAAAAADGCFAAAA|118|Ridge |Boulevard|Suite 0|Woodland|Carroll County|TN|34854|United States|-5|apartment| +21092|AAAAAAAAEGCFAAAA|402|Ninth |Road|Suite 130|Mount Pleasant|Dallas County|MO|61933|United States|-6|apartment| +21093|AAAAAAAAFGCFAAAA|803|Railroad |Road|Suite E|Mount Zion|Charleston County|SC|28054|United States|-5|condo| +21094|AAAAAAAAGGCFAAAA|59|Willow |Avenue|Suite W|Glendale|Ravalli County|MT|63951|United States|-7|apartment| +21095|AAAAAAAAHGCFAAAA|962|Park Ridge|Drive|Suite 440|Georgetown|Covington city|VA|27057|United States|-5|single family| +21096|AAAAAAAAIGCFAAAA|272|Adams |Street|Suite G|Sanford|Orangeburg County|SC|29737|United States|-5|single family| +21097|AAAAAAAAJGCFAAAA|351|13th Third|Parkway|Suite T|Oakwood|Middlesex County|NJ|00769|United States|-5|apartment| +21098|AAAAAAAAKGCFAAAA|212|East Woodland|Wy|Suite 110|Franklin|Starke County|IN|49101|United States|-5|condo| +21099|AAAAAAAALGCFAAAA|567|2nd |Drive|Suite E|Woodlawn|Waseca County|MN|54098|United States|-6|single family| +21100|AAAAAAAAMGCFAAAA|241|Fourth |Cir.|Suite 380|Brownsville|Walker County|AL|39310|United States|-6|single family| +21101|AAAAAAAANGCFAAAA|17|Pine |Drive|Suite O|Jamestown|Carteret County|NC|26867|United States|-5|apartment| +21102|AAAAAAAAOGCFAAAA|939|Center |Dr.|Suite I|New Hope|Harding County|SD|59431|United States|-7|apartment| +21103|AAAAAAAAPGCFAAAA|817|Meadow |Court|Suite 270|Four Points|Beckham County|OK|71216|United States|-6|single family| +21104|AAAAAAAAAHCFAAAA|868|Forest |Ln|Suite U|Hillcrest|Jefferson County|AL|33003|United States|-6|apartment| +21105|AAAAAAAABHCFAAAA|810|Maple |Pkwy|Suite A|Lakeview|Schoolcraft County|MI|48579|United States|-5|apartment| +21106|AAAAAAAACHCFAAAA|444|Lake |Wy|Suite K|Woodlawn|Franklin County|NY|14098|United States|-5|condo| +21107|AAAAAAAADHCFAAAA|75|10th |Ct.|Suite C|Hamilton|Chatham County|NC|22808|United States|-5|apartment| +21108|AAAAAAAAEHCFAAAA|261|5th |Drive|Suite 90|Blanchard|Oceana County|MI|45985|United States|-5|apartment| +21109|AAAAAAAAFHCFAAAA|236|Pine |Dr.|Suite K|Five Forks|Howard County|MD|22293|United States|-5|apartment| +21110|AAAAAAAAGHCFAAAA|645|Forest 5th|Lane|Suite T|Hamilton|Dubuque County|IA|52808|United States|-6|condo| +21111|AAAAAAAAHHCFAAAA|903|Spring |Ave|Suite 110|Salem|Major County|OK|78048|United States|-6|single family| +21112|AAAAAAAAIHCFAAAA|176|Main |Avenue|Suite 0|Nichols|Fayette County|IN|47940|United States|-5|condo| +21113|AAAAAAAAJHCFAAAA|490|Sunset Hillcrest|Blvd|Suite 370|Ashland|Boone County|IN|44244|United States|-5|condo| +21114|AAAAAAAAKHCFAAAA|90|Park |Avenue|Suite X|Harmony|Anderson County|KY|45804|United States|-6|condo| +21115|AAAAAAAALHCFAAAA|17|Pine |Court|Suite H|Woodland|Greeley County|KS|64854|United States|-6|condo| +21116|AAAAAAAAMHCFAAAA|173|12th |Lane|Suite 60|Clifton|Miami County|OH|48014|United States|-5|apartment| +21117|AAAAAAAANHCFAAAA|46|Fourth Church|Dr.|Suite 360|Waterloo|Menard County|TX|71675|United States|-6|apartment| +21118|AAAAAAAAOHCFAAAA|841|Sixth 14th|Circle|Suite 230|Allentown|Evans County|GA|31838|United States|-5|single family| +21119|AAAAAAAAPHCFAAAA|521|North Lake|Wy|Suite 270|Edgewood|Renville County|MN|50069|United States|-6|apartment| +21120|AAAAAAAAAICFAAAA|358|South |Way|Suite 260|Spring Hill|Southampton County|VA|26787|United States|-5|single family| +21121|AAAAAAAABICFAAAA|313|9th |Blvd|Suite U|Pleasant Grove|Limestone County|TX|74136|United States|-6|single family| +21122|AAAAAAAACICFAAAA|229|1st |Parkway|Suite 300|Edgewood|San Juan County|CO|80069|United States|-7|condo| +21123|AAAAAAAADICFAAAA|593|West |RD|Suite L|Spring Hill|Ouachita Parish|LA|76787|United States|-6|apartment| +21124|AAAAAAAAEICFAAAA|223|South |RD|Suite S|Shady Grove|Yell County|AR|72812|United States|-6|condo| +21125|AAAAAAAAFICFAAAA|521|Walnut Broadway|Wy|Suite 0|Macedonia|Appling County|GA|31087|United States|-5|apartment| +21126|AAAAAAAAGICFAAAA|353|Jefferson Jackson|Ct.|Suite 430|Bunker Hill|Saginaw County|MI|40150|United States|-5|apartment| +21127|AAAAAAAAHICFAAAA|664|Sixth Jackson|Road|Suite 260|Farmington|Lorain County|OH|49145|United States|-5|condo| +21128|AAAAAAAAIICFAAAA|984|7th Main|Blvd|Suite 40|Greenfield|Martin County|IN|45038|United States|-5|single family| +21129|AAAAAAAAJICFAAAA|592|3rd |Parkway|Suite 340|Shiloh|Red Willow County|NE|69275|United States|-7|apartment| +21130|AAAAAAAAKICFAAAA|891|Dogwood |ST|Suite 30|Georgetown|Blaine County|NE|67057|United States|-6|condo| +21131|AAAAAAAALICFAAAA|496|Smith Washington|ST|Suite 350|Farmington|Williamsburg city|VA|29145|United States|-5|apartment| +21132|AAAAAAAAMICFAAAA|565|6th North|Road|Suite 490|Mount Zion|Wabasha County|MN|58054|United States|-6|apartment| +21133|AAAAAAAANICFAAAA|793|Pine |Wy|Suite R|Woodbine|Polk County|TX|74253|United States|-6|single family| +21134|AAAAAAAAOICFAAAA|153|Locust Jackson|Road|Suite 160|Oak Ridge|Stevens County|WA|98371|United States|-8|single family| +21135|AAAAAAAAPICFAAAA|22|Ridge Hillcrest|RD|Suite 130|Edgewood|Boone County|AR|70069|United States|-6|apartment| +21136|AAAAAAAAAJCFAAAA|257|Davis |Wy|Suite X|White Oak|Doddridge County|WV|26668|United States|-5|condo| +21137|AAAAAAAABJCFAAAA|922|Fifth |Parkway|Suite V|Forest Hills|Barnes County|ND|59237|United States|-6|apartment| +21138|AAAAAAAACJCFAAAA|229|Park |Parkway|Suite 460|Greenwood|Cimarron County|OK|78828|United States|-6|single family| +21139|AAAAAAAADJCFAAAA|234|Mill Sixth|RD|Suite W|Oak Grove|Grainger County|TN|38370|United States|-5|apartment| +21140|AAAAAAAAEJCFAAAA|523|Park |Lane|Suite M|Parkwood|Carroll County|MS|51669|United States|-6|condo| +21141|AAAAAAAAFJCFAAAA|64|Spruce Jackson|Ave|Suite L|Tabor|Itawamba County|MS|58529|United States|-6|apartment| +21142|AAAAAAAAGJCFAAAA|673|Pine Main|Ave|Suite 110|Highland Park|Morrow County|OR|96534|United States|-8|single family| +21143|AAAAAAAAHJCFAAAA|375|5th Lincoln|Pkwy|Suite 440|Clifton|Logan County|CO|88014|United States|-7|single family| +21144|AAAAAAAAIJCFAAAA|569|12th 6th|Ln|Suite 270|Midway|Columbia County|GA|31904|United States|-5|single family| +21145|AAAAAAAAJJCFAAAA|170|3rd Johnson|Ave|Suite 460|Riverside|Frederick County|MD|29231|United States|-5|single family| +21146|AAAAAAAAKJCFAAAA|736|Lake Sunset|Pkwy|Suite 100|Mount Vernon|Webster County|WV|28482|United States|-5|condo| +21147|AAAAAAAALJCFAAAA|149|Ninth 15th|Wy|Suite U|Wildwood|Deuel County|SD|56871|United States|-6|single family| +21148|AAAAAAAAMJCFAAAA|586|Forest |Parkway|Suite E|Harmony|Mecklenburg County|NC|25804|United States|-5|apartment| +21149|AAAAAAAANJCFAAAA|||Road|Suite 130||Webb County|||United States|-6|| +21150|AAAAAAAAOJCFAAAA|952|Walnut |Cir.|Suite I|Cedar Grove|Worth County|MO|60411|United States|-6|single family| +21151|AAAAAAAAPJCFAAAA|183|View Main|Circle|Suite 160|Clifton|Perry County|AL|38014|United States|-6|apartment| +21152|AAAAAAAAAKCFAAAA|979|7th |Ave|Suite 260|Bethel|Hanson County|SD|55281|United States|-7|apartment| +21153|AAAAAAAABKCFAAAA|50|Lake |Street|Suite 310|Spring Hill|La Salle Parish|LA|76787|United States|-6|apartment| +21154|AAAAAAAACKCFAAAA|413|Forest |Pkwy|Suite 60|Shady Grove|Hughes County|SD|52812|United States|-7|apartment| +21155|AAAAAAAADKCFAAAA|22|Highland |Pkwy|Suite 120|Enterprise|Richmond County|GA|31757|United States|-5|apartment| +21156|AAAAAAAAEKCFAAAA|692|Sunset |Drive|Suite X|Waterloo|Jackson County|IN|41675|United States|-5|apartment| +21157|AAAAAAAAFKCFAAAA|959|Seventh Cedar|Boulevard|Suite D|Flatwoods|Alexander County|IL|64212|United States|-6|single family| +21158|AAAAAAAAGKCFAAAA|827|Cedar |Dr.|Suite 460|Fairfield|Lake County|CA|96192|United States|-8|apartment| +21159|AAAAAAAAHKCFAAAA|944|Sycamore North|Court|Suite 310|Ridgeville|Emanuel County|GA|39306|United States|-5|single family| +21160|AAAAAAAAIKCFAAAA|139|View West|Road|Suite 160|Mount Pleasant|Jasper County|MO|61933|United States|-6|apartment| +21161|AAAAAAAAJKCFAAAA|140|12th River|Parkway|Suite L|Springdale|Union County|GA|38883|United States|-5|apartment| +21162|AAAAAAAAKKCFAAAA|292|Poplar |Ln|Suite 410|Providence|Harvey County|KS|66614|United States|-6|condo| +21163|AAAAAAAALKCFAAAA|819|Lincoln |Ln|Suite 160|Brownsville|Berkeley County|SC|29310|United States|-5|single family| +21164|AAAAAAAAMKCFAAAA|899|Fifth |Wy|Suite S|Belmont|Jefferson County|KY|40191|United States|-6|condo| +21165|AAAAAAAANKCFAAAA|47|14th Pine|Ln|Suite 230|Jamestown|Portage County|WI|56867|United States|-6|single family| +21166|AAAAAAAAOKCFAAAA|808|Main |Dr.|Suite J|Fairview|Sedgwick County|KS|65709|United States|-6|condo| +21167|AAAAAAAAPKCFAAAA|583|Sycamore |ST|Suite 90|Clinton|Warren County|MO|68222|United States|-6|single family| +21168|AAAAAAAAALCFAAAA|267|Spring |ST|Suite 170|Edgewood|Lee County|KY|40069|United States|-5|apartment| +21169|AAAAAAAABLCFAAAA|481|South Maple|Cir.|Suite S|Hillcrest|Colfax County|NE|63003|United States|-6|condo| +21170|AAAAAAAACLCFAAAA|237|3rd |Boulevard|Suite 290|Mount Zion|Highland County|OH|48054|United States|-5|apartment| +21171|AAAAAAAADLCFAAAA|685|Franklin |Avenue|Suite 160|Lincoln|Natrona County|WY|81289|United States|-7|condo| +21172|AAAAAAAAELCFAAAA|125|Railroad Lake|Ln|Suite 130|Greenville|Rice County|MN|51387|United States|-6|apartment| +21173|AAAAAAAAFLCFAAAA|461|Lee 5th|Road|Suite S|Arlington|Sampson County|NC|26557|United States|-5|single family| +21174|AAAAAAAAGLCFAAAA|304|3rd |Cir.|Suite 220|Jamestown|Macon County|NC|26867|United States|-5|condo| +21175|AAAAAAAAHLCFAAAA|782|Valley East|Pkwy|Suite L|Mechanicsburg|Stutsman County|ND|52219|United States|-6|single family| +21176|AAAAAAAAILCFAAAA|995|Hillcrest |Boulevard|Suite 370|Farmersville|Belmont County|OH|49305|United States|-5|single family| +21177|AAAAAAAAJLCFAAAA|601|Birch |Court|Suite V|Springdale|De Kalb County|IN|48883|United States|-5|single family| +21178|AAAAAAAAKLCFAAAA|616|9th |Court|Suite D|Union Hill|Mellette County|SD|57746|United States|-7|condo| +21179|AAAAAAAALLCFAAAA|672|Jackson |Ln|Suite 370|Glenwood|Lucas County|OH|43511|United States|-5|single family| +21180|AAAAAAAAMLCFAAAA|139|7th |Blvd|Suite 360|Highland|Newton County|MO|69454|United States|-6|apartment| +21181|AAAAAAAANLCFAAAA||4th Washington||||||25817||-5|| +21182|AAAAAAAAOLCFAAAA|751|6th Sunset|Dr.|Suite M|Centerville|Albemarle County|VA|20059|United States|-5|apartment| +21183|AAAAAAAAPLCFAAAA|126|View |Circle|Suite G|Lakewood|Mason County|WV|28877|United States|-5|condo| +21184|AAAAAAAAAMCFAAAA|37|Walnut |Boulevard|Suite 480|Greenfield|Billings County|ND|55038|United States|-6|condo| +21185|AAAAAAAABMCFAAAA|382|Main Maple|Ct.|Suite F|Bridgeport|San Jacinto County|TX|75817|United States|-6|condo| +21186|AAAAAAAACMCFAAAA|807|Sunset |Lane|Suite L|Spring Valley|Manatee County|FL|36060|United States|-5|apartment| +21187|AAAAAAAADMCFAAAA|703|Birch |ST|Suite 240|Five Forks|Cass County|NE|62293|United States|-6|single family| +21188|AAAAAAAAEMCFAAAA|759|11th |Street|Suite 450|Farmington|Sumner County|KS|69145|United States|-6|condo| +21189|AAAAAAAAFMCFAAAA|746|Broadway Lincoln|Dr.|Suite 160|Five Forks|Edwards County|IL|62293|United States|-6|condo| +21190|AAAAAAAAGMCFAAAA|612|River Lee|RD|Suite 430|Florence|Potter County|TX|73394|United States|-6|condo| +21191|AAAAAAAAHMCFAAAA|873|Davis Walnut|Street|Suite D|Woodlawn|Owyhee County|ID|84098|United States|-7|condo| +21192|AAAAAAAAIMCFAAAA|634|4th Washington|Dr.|Suite 380|Redland|Giles County|VA|26343|United States|-5|condo| +21193|AAAAAAAAJMCFAAAA|857|Cherry |Ln|Suite 150|Shiloh|Saline County|IL|69275|United States|-6|apartment| +21194|AAAAAAAAKMCFAAAA|167|Franklin 3rd|Drive|Suite 230|Shady Grove|Wise County|TX|72812|United States|-6|apartment| +21195|AAAAAAAALMCFAAAA|633|Eigth |Ct.|Suite U|Kingston|Montgomery County|MO|64975|United States|-6|single family| +21196|AAAAAAAAMMCFAAAA|591|Green Fifth|Lane|Suite D|Fairview|Clare County|MI|45709|United States|-5|condo| +21197|AAAAAAAANMCFAAAA|338|College 2nd|Boulevard|Suite 80|Green Acres|Dane County|WI|57683|United States|-6|apartment| +21198|AAAAAAAAOMCFAAAA|875|Jefferson Tenth|Court|Suite I|Jamestown|Hancock County|GA|36867|United States|-5|single family| +21199|AAAAAAAAPMCFAAAA|456|Johnson |Ct.|Suite 70|Friendship|Bon Homme County|SD|54536|United States|-6|single family| +21200|AAAAAAAAANCFAAAA|431|Hickory 3rd|Pkwy|Suite H|Fairview|Nantucket County|MA|06309|United States|-5|single family| +21201|AAAAAAAABNCFAAAA|977|Highland 7th|Blvd|Suite P|Five Points|Lincoln County|KY|46098|United States|-5|apartment| +21202|AAAAAAAACNCFAAAA|984|Second |Pkwy|Suite 220|Springdale|Benton County|MS|58883|United States|-6|condo| +21203|AAAAAAAADNCFAAAA|990|Main |Ct.|Suite I|Lone Oak|Clifton Forge city|VA|26893|United States|-5|apartment| +21204|AAAAAAAAENCFAAAA|175|Oak |Court|Suite D|Bethel|Wilkinson County|GA|35281|United States|-5|condo| +21205|AAAAAAAAFNCFAAAA|428|Central 12th|Ave|Suite 400|Kimball|Upson County|GA|33595|United States|-5|apartment| +21206|AAAAAAAAGNCFAAAA|130|Ridge |Boulevard|Suite X|Providence|Clark County|WI|56614|United States|-6|condo| +21207|AAAAAAAAHNCFAAAA|100|Center |Pkwy|Suite 270|Belmont|Strafford County|NH|00791|United States|-5|condo| +21208|AAAAAAAAINCFAAAA|766|Sycamore Jackson|Dr.|Suite 80|Marion|Kankakee County|IL|60399|United States|-6|condo| +21209|AAAAAAAAJNCFAAAA|837|River Railroad|RD|Suite 80|Hillcrest|Greenville County|SC|23003|United States|-5|apartment| +21210|AAAAAAAAKNCFAAAA|183|Oak |Circle|Suite S|Florence|Wirt County|WV|23394|United States|-5|single family| +21211|AAAAAAAALNCFAAAA|636|8th Ridge|Court|Suite Y|Bethel|Nueces County|TX|75281|United States|-6|condo| +21212|AAAAAAAAMNCFAAAA|715|1st |Lane|Suite 310|Plainview|Putnam County|OH|43683|United States|-5|condo| +21213|AAAAAAAANNCFAAAA|522|1st Madison|Street|Suite 440|Cedar Grove|La Crosse County|WI|50411|United States|-6|apartment| +21214|AAAAAAAAONCFAAAA|978|Ridge Walnut|Cir.|Suite R|Shiloh|Lamoille County|VT|09875|United States|-5|condo| +21215|AAAAAAAAPNCFAAAA|951|Meadow Miller|Road|Suite G|Pleasant Valley|Muskegon County|MI|42477|United States|-5|apartment| +21216|AAAAAAAAAOCFAAAA|96|4th Spring|Ln|Suite 10|Maple Grove|Morgan County|AL|38252|United States|-6|apartment| +21217|AAAAAAAABOCFAAAA|973|North Lakeview|RD|Suite 290|Glenwood|Wood County|TX|73511|United States|-6|apartment| +21218|AAAAAAAACOCFAAAA|801|6th Laurel|Road|Suite H|Pinhook|Mitchell County|TX|79398|United States|-6|apartment| +21219|AAAAAAAADOCFAAAA|607|9th Fifth|Dr.|Suite X|Lincoln|Lake County|FL|31289|United States|-5|single family| +21220|AAAAAAAAEOCFAAAA|818|Johnson |Avenue|Suite I|Kingston|Butler County|IA|54975|United States|-6|apartment| +21221|AAAAAAAAFOCFAAAA|473|4th |Wy|Suite U|Riverside|Jefferson County|GA|39231|United States|-5|single family| +21222|AAAAAAAAGOCFAAAA|912|1st |Wy|Suite D|Sunnyside|Carbon County|PA|11952|United States|-5|apartment| +21223|AAAAAAAAHOCFAAAA|646|Woodland Locust|Boulevard|Suite 270|New Town|Matagorda County|TX|79634|United States|-6|apartment| +21224|AAAAAAAAIOCFAAAA|827|Oak |Drive|Suite V|Oak Ridge|Johnson County|AR|78371|United States|-6|condo| +21225|AAAAAAAAJOCFAAAA|687|Willow |Parkway|Suite 60|Greenville|Dent County|MO|61387|United States|-6|apartment| +21226|AAAAAAAAKOCFAAAA|388|Chestnut |Ln|Suite W|Greenwood|La Salle County|TX|78828|United States|-6|single family| +21227|AAAAAAAALOCFAAAA|833|Main Spring|Parkway|Suite H|Highland|Belknap County|NH|09454|United States|-5|single family| +21228|AAAAAAAAMOCFAAAA|138|Valley |Blvd|Suite K|Midway|Clay County|NC|21904|United States|-5|condo| +21229|AAAAAAAANOCFAAAA||Cherry Park||||Gogebic County|||||single family| +21230|AAAAAAAAOOCFAAAA|518|Railroad |Circle|Suite O|Franklin|Columbia County|AR|79101|United States|-6|single family| +21231|AAAAAAAAPOCFAAAA|224|Main |Cir.|Suite 60|Fairfield|Arapahoe County|CO|86192|United States|-7|condo| +21232|AAAAAAAAAPCFAAAA|599|Davis Mill|Dr.|Suite Y|Farmington|Clearfield County|PA|19145|United States|-5|condo| +21233|AAAAAAAABPCFAAAA|181|First Sixth|Pkwy|Suite D|Woodbine|Putnam County|IL|64253|United States|-6|apartment| +21234|AAAAAAAACPCFAAAA|367|6th |Cir.|Suite R|Roxbury|Morton County|ND|55508|United States|-6|apartment| +21235|AAAAAAAADPCFAAAA|594|Sixth Ridge|Wy|Suite S|Deerfield|Brunswick County|NC|29840|United States|-5|single family| +21236|AAAAAAAAEPCFAAAA|337|Sycamore |Street|Suite 70|Clinton|Reno County|KS|68222|United States|-6|single family| +21237|AAAAAAAAFPCFAAAA|562|College |Lane|Suite W|Woodruff|Houston County|GA|34174|United States|-5|single family| +21238|AAAAAAAAGPCFAAAA|90|South Locust|Avenue|Suite U|Sulphur Springs|Floyd County|KY|48354|United States|-6|apartment| +21239|AAAAAAAAHPCFAAAA|857|College River|RD|Suite O|Lakewood|Livingston County|IL|68877|United States|-6|condo| +21240|AAAAAAAAIPCFAAAA||Lee |Avenue|Suite Q||Kossuth County|IA|50534|United States|-6|single family| +21241|AAAAAAAAJPCFAAAA|138|Second |Circle|Suite J|Springfield|Hot Springs County|WY|89303|United States|-7|condo| +21242|AAAAAAAAKPCFAAAA|933|7th 15th|Pkwy|Suite 380|Washington Heights|Gallia County|OH|48167|United States|-5|apartment| +21243|AAAAAAAALPCFAAAA|278|Cedar 8th|Circle|Suite P|Newtown|Hill County|TX|71749|United States|-6|apartment| +21244|AAAAAAAAMPCFAAAA|66|9th |Lane|Suite 60|Providence|Heard County|GA|36614|United States|-5|apartment| +21245|AAAAAAAANPCFAAAA|732|Woodland 5th|RD|Suite W|Kingston|Finney County|KS|64975|United States|-6|single family| +21246|AAAAAAAAOPCFAAAA|586|5th Hill|Circle|Suite 60|Mount Zion|O-Brien County|IA|58054|United States|-6|apartment| +21247|AAAAAAAAPPCFAAAA|582|9th |RD|Suite Q|Jamestown|Cerro Gordo County|IA|56867|United States|-6|single family| +21248|AAAAAAAAAADFAAAA|266|3rd Pine|Blvd|Suite E|Ashland|Greene County|AL|34244|United States|-6|single family| +21249|AAAAAAAABADFAAAA|965|Chestnut |Boulevard|Suite N|Friendship|Johnson County|TN|34536|United States|-6|condo| +21250|AAAAAAAACADFAAAA|885|Valley 15th|Court|Suite B|Pine Grove|Morris County|NJ|05193|United States|-5|single family| +21251|AAAAAAAADADFAAAA|551|Railroad |Blvd|Suite 430|Cedar Grove|Putnam County|TN|30411|United States|-6|single family| +21252|AAAAAAAAEADFAAAA|963|11th 10th|Drive|Suite 400|Jackson|Shelby County|TN|39583|United States|-6|single family| +21253|AAAAAAAAFADFAAAA|401|Eigth |Ct.|Suite 410|Highland|Alamosa County|CO|89454|United States|-7|condo| +21254|AAAAAAAAGADFAAAA|88|Fourth |Wy|Suite S|Waterloo|Gove County|KS|61675|United States|-6|single family| +21255|AAAAAAAAHADFAAAA|322|Green |Blvd|Suite 390|Salem|White County|TN|38048|United States|-6|apartment| +21256|AAAAAAAAIADFAAAA|952|Washington |Wy|Suite N|Shiloh|Concordia Parish|LA|79275|United States|-6|single family| +21257|AAAAAAAAJADFAAAA||Fourth |Street|Suite D|||||||condo| +21258|AAAAAAAAKADFAAAA|944|Chestnut |Way|Suite T|Glenwood|Medina County|TX|73511|United States|-6|condo| +21259|AAAAAAAALADFAAAA|4|Lake |ST|Suite 390|Glenwood|Baltimore city|MD|23511|United States|-5|condo| +21260|AAAAAAAAMADFAAAA|505|First Center|Wy|Suite 270|Hillcrest|Alcona County|MI|43003|United States|-5|apartment| +21261|AAAAAAAANADFAAAA|225|Main |RD|Suite B|Oak Grove|Casey County|KY|48370|United States|-6|single family| +21262|AAAAAAAAOADFAAAA|400|||||Cheshire County||09903|United States||| +21263|AAAAAAAAPADFAAAA|992|2nd Ridge|Ln|Suite 480|Wilson|Lawrence County|OH|46971|United States|-5|apartment| +21264|AAAAAAAAABDFAAAA|688|Smith 8th|Blvd|Suite 250|Providence|Cherokee County|IA|56614|United States|-6|single family| +21265|AAAAAAAABBDFAAAA|276|Birch |Way|Suite 150|Green Acres|Lagrange County|IN|47683|United States|-5|condo| +21266|AAAAAAAACBDFAAAA|874|Broadway Second|Ave|Suite 200|Union Hill|Humboldt County|IA|57746|United States|-6|condo| +21267|AAAAAAAADBDFAAAA|816|Locust |Ave|Suite 450|Red Hill|Pershing County|NV|84338|United States|-8|single family| +21268|AAAAAAAAEBDFAAAA|576|Walnut Johnson|Way|Suite 30|Unionville|McHenry County|ND|51711|United States|-6|single family| +21269|AAAAAAAAFBDFAAAA|439|Poplar |Way|Suite 440|Union|Zapata County|TX|78721|United States|-6|single family| +21270|AAAAAAAAGBDFAAAA|827|Twelfth |ST|Suite 170|Franklin|Marshall County|AL|39101|United States|-6|apartment| +21271|AAAAAAAAHBDFAAAA|862|1st |Ln|Suite W|Oak Ridge|Walker County|GA|38371|United States|-5|condo| +21272|AAAAAAAAIBDFAAAA|775|11th Second|Drive|Suite I|Lincoln|Sutton County|TX|71289|United States|-6|apartment| +21273|AAAAAAAAJBDFAAAA|880|East 5th|Drive|Suite 460|Newtown|Clay County|IL|61749|United States|-6|apartment| +21274|AAAAAAAAKBDFAAAA|35|Lee 15th|Cir.|Suite 200|Birmingham|Kenosha County|WI|53372|United States|-6|single family| +21275|AAAAAAAALBDFAAAA|436|4th |Dr.|Suite P|Springdale|Oneida County|ID|88883|United States|-7|condo| +21276|AAAAAAAAMBDFAAAA|135|1st Smith|Parkway|Suite 100|Forest Hills|Jefferson County|FL|39237|United States|-5|apartment| +21277|AAAAAAAANBDFAAAA|201|Main 6th|ST|Suite H|Mountain View|Tipton County|TN|34466|United States|-6|single family| +21278|AAAAAAAAOBDFAAAA|80|6th |Drive|Suite X|Mount Olive|Mayes County|OK|78059|United States|-6|condo| +21279|AAAAAAAAPBDFAAAA|489|Woodland 2nd|Dr.|Suite P|Riverview|Baltimore city|MD|29003|United States|-5|single family| +21280|AAAAAAAAACDFAAAA|119|Johnson Birch|Avenue|Suite N|Mount Vernon|Crockett County|TX|78482|United States|-6|condo| +21281|AAAAAAAABCDFAAAA|355|West Maple|Court|Suite K|Oakwood|Morton County|KS|60169|United States|-6|single family| +21282|AAAAAAAACCDFAAAA|506|5th Smith|Blvd|Suite 110|Clifton|Hillsborough County|NH|08614|United States|-5|single family| +21283|AAAAAAAADCDFAAAA|338|Franklin |Dr.|Suite 320|Buena Vista|Taylor County|GA|35752|United States|-5|condo| +21284|AAAAAAAAECDFAAAA|981|Woodland Highland|Circle|Suite 160|Ashland|Shelby County|IN|44244|United States|-5|condo| +21285|AAAAAAAAFCDFAAAA|978|8th Railroad|Boulevard|Suite 320|Cedar Grove|Bernalillo County|NM|80411|United States|-7|apartment| +21286|AAAAAAAAGCDFAAAA|444|Chestnut |Ln|Suite 20|Jackson|Glenn County|CA|99583|United States|-8|single family| +21287|AAAAAAAAHCDFAAAA|130|Elm |Dr.|Suite X|Enterprise|Hemphill County|TX|71757|United States|-6|single family| +21288|AAAAAAAAICDFAAAA|10|Second |Court|Suite E|Highland Park|Norton County|KS|66534|United States|-6|condo| +21289|AAAAAAAAJCDFAAAA|803|Elm |Way|Suite W|Sulphur Springs|Okmulgee County|OK|78354|United States|-6|condo| +21290|AAAAAAAAKCDFAAAA|119|2nd Main|Wy|Suite T|Avon|Dundy County|NE|60069|United States|-6|single family| +21291|AAAAAAAALCDFAAAA|843|Washington |Pkwy|Suite 160|Jerome|Charles City County|VA|29920|United States|-5|condo| +21292|AAAAAAAAMCDFAAAA|468|Walnut |Lane|Suite U|Scottsville|Humphreys County|TN|34190|United States|-5|single family| +21293|AAAAAAAANCDFAAAA|491|Oak Railroad|Ave|Suite W|Caledonia|East Baton Rouge Parish|LA|77411|United States|-6|single family| +21294|AAAAAAAAOCDFAAAA|740|Davis |Way|Suite 480|Oak Ridge|Perry County|OH|48371|United States|-5|apartment| +21295|AAAAAAAAPCDFAAAA|159|Hill Hill|Pkwy|Suite S|Brownsville|Cass County|TX|79310|United States|-6|condo| +21296|AAAAAAAAADDFAAAA|650|9th |Ln|Suite O|Farmington|Randall County|TX|79145|United States|-6|condo| +21297|AAAAAAAABDDFAAAA|111|4th 2nd|Cir.|Suite 70|Leesville|Lawrence County|MS|55423|United States|-6|single family| +21298|AAAAAAAACDDFAAAA|832|7th |Lane|Suite P|Providence|Wilkes County|NC|26614|United States|-5|condo| +21299|AAAAAAAADDDFAAAA|202|Lee North|Wy|Suite 100|Brownsville|Stanly County|NC|29310|United States|-5|apartment| +21300|AAAAAAAAEDDFAAAA|596|Park Eigth|Wy|Suite 480|Deerfield|Jackson County|IL|69840|United States|-6|apartment| +21301|AAAAAAAAFDDFAAAA|500|Oak |Blvd|Suite 410|Glenwood|Kent County|MI|43511|United States|-5|apartment| +21302|AAAAAAAAGDDFAAAA|792|Maple |Ln|Suite 0|Tyrone|Madison County|MO|61201|United States|-6|condo| +21303|AAAAAAAAHDDFAAAA|700|Locust Mill|Blvd|Suite 70|Belmont|Elko County|NV|80191|United States|-8|apartment| +21304|AAAAAAAAIDDFAAAA|633|Maple |Parkway|Suite 0|Highland|Amherst County|VA|29454|United States|-5|condo| +21305|AAAAAAAAJDDFAAAA|497|Spring |RD|Suite I|Georgetown|Carter County|TN|37057|United States|-5|condo| +21306|AAAAAAAAKDDFAAAA|812|Johnson |Ave|Suite 210|Valley View|Tompkins County|NY|15124|United States|-5|apartment| +21307|AAAAAAAALDDFAAAA|432|Elm Ash|Blvd|Suite D|Spring Valley|Uvalde County|TX|76060|United States|-6|apartment| +21308|AAAAAAAAMDDFAAAA|138|West Main|ST|Suite 160|Spring Hill|Pembina County|ND|56787|United States|-6|condo| +21309|AAAAAAAANDDFAAAA|550|Walnut |||Fairview|Carroll County||65709|United States|-6|condo| +21310|AAAAAAAAODDFAAAA|811|Laurel |Ave|Suite 280|Riverview|Stephens County|TX|79003|United States|-6|condo| +21311|AAAAAAAAPDDFAAAA|138|9th |Lane|Suite 420|Cordova|Sargent County|ND|56938|United States|-6|apartment| +21312|AAAAAAAAAEDFAAAA|893|Main |Ave|Suite 300|Hardy|Big Horn County|MT|65354|United States|-7|condo| +21313|AAAAAAAABEDFAAAA|779|7th Franklin|Blvd|Suite 170|Oakwood|Lafayette Parish|LA|70169|United States|-6|apartment| +21314|AAAAAAAACEDFAAAA|280|Sixth 2nd|Pkwy|Suite R|Page|Ohio County|KY|40296|United States|-5|apartment| +21315|AAAAAAAADEDFAAAA|875|Second 2nd|Ct.|Suite F|Shiloh|Dickson County|TN|39275|United States|-5|apartment| +21316|AAAAAAAAEEDFAAAA|764|Elm |Dr.|Suite H|Jamestown|Jackson County|KY|46867|United States|-6|single family| +21317|AAAAAAAAFEDFAAAA|585|Adams Mill|Way|Suite 130|Bethel|Seminole County|OK|75281|United States|-6|condo| +21318|AAAAAAAAGEDFAAAA|490|12th |Ct.|Suite E|Oak Hill|Woodbury County|IA|57838|United States|-6|apartment| +21319|AAAAAAAAHEDFAAAA|764|Lincoln |Court|||Rogers County|OK|||-6|| +21320|AAAAAAAAIEDFAAAA|672|6th |Blvd|Suite 160|Crossroads|Kearney County|NE|60534|United States|-7|apartment| +21321|AAAAAAAAJEDFAAAA|227|1st |Drive|Suite X|Lakeside|Kane County|IL|69532|United States|-6|apartment| +21322|AAAAAAAAKEDFAAAA|||ST|Suite B|Mount Vernon|Adair County|KY||||| +21323|AAAAAAAALEDFAAAA|951|4th |RD|Suite S|Midway|Humboldt County|NV|81904|United States|-8|single family| +21324|AAAAAAAAMEDFAAAA|115|Second |Wy|Suite 270|Walnut Grove|Delaware County|IN|47752|United States|-5|single family| +21325|AAAAAAAANEDFAAAA|173|Central 4th|Way|Suite P|Fairfield|Citrus County|FL|36192|United States|-5|apartment| +21326|AAAAAAAAOEDFAAAA|524|||Suite G||Loving County|TX|75752|United States|-6|apartment| +21327|AAAAAAAAPEDFAAAA|675|Cherry Woodland|Ln|Suite 230|Highland Park|Broadwater County|MT|66534|United States|-7|condo| +21328|AAAAAAAAAFDFAAAA|388|5th Cherry|Blvd|Suite 240|Clinton|Gallia County|OH|48222|United States|-5|single family| +21329|AAAAAAAABFDFAAAA|264|Adams Washington|Blvd|Suite 160|Springdale|Winnebago County|IA|58883|United States|-6|condo| +21330|AAAAAAAACFDFAAAA|229|Lakeview |Street|Suite 190|Pine Grove|Adams County|IL|64593|United States|-6|single family| +21331|AAAAAAAADFDFAAAA|951|Walnut |ST|Suite 40|Edwards|Falls County|TX|71409|United States|-6|condo| +21332|AAAAAAAAEFDFAAAA|782|2nd |Dr.|Suite 60|Rockland|Donley County|TX|72044|United States|-6|condo| +21333|AAAAAAAAFFDFAAAA|122|Lake |Dr.|Suite 150|Glendale|Glynn County|GA|33951|United States|-5|single family| +21334|AAAAAAAAGFDFAAAA|945|Second |Cir.|Suite 300|Woodville|Antelope County|NE|64289|United States|-6|apartment| +21335|AAAAAAAAHFDFAAAA|136|15th 14th|Circle|Suite X|Springtown|Ozark County|MO|69858|United States|-6|single family| +21336|AAAAAAAAIFDFAAAA|762|Cedar |Parkway|Suite S|Mount Olive|Lee County|FL|38059|United States|-5|condo| +21337|AAAAAAAAJFDFAAAA|452|View Maple|RD|Suite Y|Washington Heights|Dodge County|WI|58167|United States|-6|apartment| +21338|AAAAAAAAKFDFAAAA|733|Oak |Way|Suite D|Antioch|Colfax County|NE|68605|United States|-6|single family| +21339|AAAAAAAALFDFAAAA|635|Forest |Avenue|Suite K|Macedonia|Dutchess County|NY|11087|United States|-5|single family| +21340|AAAAAAAAMFDFAAAA|870|View |ST|Suite I|Kingston|East Baton Rouge Parish|LA|74975|United States|-6|condo| +21341|AAAAAAAANFDFAAAA|617|North Pine|Street|Suite D|Buena Vista|Elmore County|AL|35752|United States|-6|apartment| +21342|AAAAAAAAOFDFAAAA|709|1st |RD|Suite X|Brownsville|Knox County|MO|69310|United States|-6|single family| +21343|AAAAAAAAPFDFAAAA|722|View Birch|Dr.|Suite 490|Mountain View|Lincoln County|MT|64466|United States|-7|apartment| +21344|AAAAAAAAAGDFAAAA|763|Church Fourth|ST|Suite H|Woodland|Goshen County|WY|84854|United States|-7|condo| +21345|AAAAAAAABGDFAAAA|962|Main |Court|Suite 20|Highland Park|Kiowa County|OK|76534|United States|-6|single family| +21346|AAAAAAAACGDFAAAA|706|Sycamore 1st|Cir.|Suite R|Franklin|Ralls County|MO|69101|United States|-6|single family| +21347|AAAAAAAADGDFAAAA|798|Sycamore |Court|Suite X|Five Points|Davie County|NC|26098|United States|-5|apartment| +21348|AAAAAAAAEGDFAAAA|707|Wilson Fifth|Street|Suite 230|Spring Hill|Franklin County|NY|16787|United States|-5|condo| +21349|AAAAAAAAFGDFAAAA|790|Cedar Cherry|Wy|Suite 240|Jenkins|Greene County|IN|47292|United States|-5|condo| +21350|AAAAAAAAGGDFAAAA|842|Fourth Spring|ST|Suite U|Woodland|Chautauqua County|KS|64854|United States|-6|single family| +21351|AAAAAAAAHGDFAAAA|843|Main Walnut|Ln|Suite W|Kingston|Dickinson County|KS|64975|United States|-6|single family| +21352|AAAAAAAAIGDFAAAA|732|Cedar 1st|Wy|Suite 20|Oakwood|Yuma County|CO|80169|United States|-7|condo| +21353|AAAAAAAAJGDFAAAA|744|Hickory View|Ct.|Suite 210|Union Hill|Kankakee County|IL|67746|United States|-6|condo| +21354|AAAAAAAAKGDFAAAA|564|Main Walnut|Pkwy|Suite F|Centerville|Edgar County|IL|60059|United States|-6|apartment| +21355|AAAAAAAALGDFAAAA|93|Highland |Ln|Suite Y|Providence|Fountain County|IN|46614|United States|-5|apartment| +21356|AAAAAAAAMGDFAAAA|771|Railroad 1st|Dr.|Suite F|Spring Hill|Cass County|MN|56787|United States|-6|condo| +21357|AAAAAAAANGDFAAAA|218|5th |Cir.|Suite 460|Lakewood|Scott County|MO|68877|United States|-6|condo| +21358|AAAAAAAAOGDFAAAA|297|South |Wy|Suite 140|Buckingham|Livingston County|KY|44092|United States|-5|apartment| +21359|AAAAAAAAPGDFAAAA|228|Sycamore Wilson|Circle|Suite 360|Oakdale|King and Queen County|VA|29584|United States|-5|single family| +21360|AAAAAAAAAHDFAAAA|345|West |Dr.|Suite B|Maple Grove|Roanoke County|VA|28252|United States|-5|apartment| +21361|AAAAAAAABHDFAAAA|919|Hill |Ln|Suite 0|Lakeside|Dewey County|SD|59532|United States|-6|apartment| +21362|AAAAAAAACHDFAAAA|471|Lincoln Pine|Ave|Suite Q|Greenfield|Lonoke County|AR|75038|United States|-6|single family| +21363|AAAAAAAADHDFAAAA|686|||Suite 50||Richmond County||10534|United States||apartment| +21364|AAAAAAAAEHDFAAAA|478|13th |Way|Suite A|Owens|Keweenaw County|MI|42324|United States|-5|apartment| +21365|AAAAAAAAFHDFAAAA|494|14th |Lane|Suite 420|Macedonia|Transylvania County|NC|21087|United States|-5|single family| +21366|AAAAAAAAGHDFAAAA|731|Jefferson |Parkway|Suite H|Mountain View|Taylor County|GA|34466|United States|-5|apartment| +21367|AAAAAAAAHHDFAAAA|870|Dogwood |Way|Suite 280|Liberty|York County|NE|63451|United States|-6|condo| +21368|AAAAAAAAIHDFAAAA|641|Chestnut |Ln|Suite R|Bunker Hill|Montgomery County|MD|20150|United States|-5|apartment| +21369|AAAAAAAAJHDFAAAA|843|5th |Parkway|Suite 60|Arlington|Minidoka County|ID|86557|United States|-7|apartment| +21370|AAAAAAAAKHDFAAAA|508|Cherry East|Blvd|Suite V|Brookville|San Francisco County|CA|93524|United States|-8|condo| +21371|AAAAAAAALHDFAAAA|657|9th First|Street|Suite 50|Deerfield|McCook County|SD|59840|United States|-7|single family| +21372|AAAAAAAAMHDFAAAA|368|Green 11th|Wy|Suite O|Oak Hill|Dutchess County|NY|17838|United States|-5|single family| +21373|AAAAAAAANHDFAAAA|365|Main |ST|Suite 160|Jackson|Orangeburg County|SC|29583|United States|-5|single family| +21374|AAAAAAAAOHDFAAAA|611|Church |Wy|Suite 460|Brownsville|Leavenworth County|KS|69310|United States|-6|single family| +21375|AAAAAAAAPHDFAAAA|805|2nd Locust|Court|Suite 230|Brownsville|Bourbon County|KS|69310|United States|-6|single family| +21376|AAAAAAAAAIDFAAAA|515|Washington |RD|Suite 150|Blanchard|Franklin County|ME|06585|United States|-5|apartment| +21377|AAAAAAAABIDFAAAA|799|Spruce |Ln|Suite W|Taft|Guadalupe County|TX|70589|United States|-6|apartment| +21378|AAAAAAAACIDFAAAA|718|Jefferson |Blvd|Suite I|Sunnyside|Cape Girardeau County|MO|61952|United States|-6|apartment| +21379|AAAAAAAADIDFAAAA|263|Highland Valley|RD|Suite G|Valley View|Stoddard County|MO|65124|United States|-6|single family| +21380|AAAAAAAAEIDFAAAA|330|Jackson Main|Drive|Suite 310|Tracy|Wayne County|MS|56340|United States|-6|apartment| +21381|AAAAAAAAFIDFAAAA|48|Cherry |Blvd|Suite X|Crossroads|Todd County|KY|40534|United States|-5|single family| +21382|AAAAAAAAGIDFAAAA|281|Lincoln |ST|Suite 190|Macedonia|Orange County|IN|41087|United States|-5|single family| +21383|AAAAAAAAHIDFAAAA|842|Valley Second|Lane|Suite 200|Sulphur Springs|Douglas County|MN|58354|United States|-6|apartment| +21384|AAAAAAAAIIDFAAAA||||Suite 0|||TN|||-6|| +21385|AAAAAAAAJIDFAAAA|855|Sycamore |Ln|Suite 350|Greenville|Kodiak Island Borough|AK|91387|United States|-9|apartment| +21386|AAAAAAAAKIDFAAAA|913|8th Mill|Street|Suite F|Little River|Bossier Parish|LA|70319|United States|-6|condo| +21387|AAAAAAAALIDFAAAA|76|Park |Street|Suite 220|Shady Grove|Bristol city|VA|22812|United States|-5|single family| +21388|AAAAAAAAMIDFAAAA|463|11th |Ave|Suite C|Plainview|Concho County|TX|73683|United States|-6|condo| +21389|AAAAAAAANIDFAAAA|568|Poplar South|Way|Suite K|Fowler|Assumption Parish|LA|71083|United States|-6|condo| +21390|AAAAAAAAOIDFAAAA|492|4th Center|Lane|Suite H|Hamilton|Anderson County|TX|72808|United States|-6|condo| +21391|AAAAAAAAPIDFAAAA|642|View |Pkwy|Suite D|Springdale|Brunswick County|NC|28883|United States|-5|condo| +21392|AAAAAAAAAJDFAAAA|19|Cedar Cedar|Parkway|Suite 70|Jackson|Linn County|IA|59583|United States|-6|single family| +21393|AAAAAAAABJDFAAAA|250|Highland 11th|Ave|Suite 130|Brownsville|Grundy County|IA|59310|United States|-6|condo| +21394|AAAAAAAACJDFAAAA|468|1st |Boulevard|Suite L|Riverdale|Upshur County|WV|29391|United States|-5|condo| +21395|AAAAAAAADJDFAAAA|752|Lee 6th|Drive|Suite W|Unionville|Screven County|GA|31711|United States|-5|apartment| +21396|AAAAAAAAEJDFAAAA|410|East |Parkway|Suite 70|Enterprise|Sweet Grass County|MT|61757|United States|-7|condo| +21397|AAAAAAAAFJDFAAAA|312|Maple Third|ST|Suite 190|Mount Vernon|York County|VA|28482|United States|-5|condo| +21398|AAAAAAAAGJDFAAAA|901|Second South|ST|Suite 200|Unionville|Mitchell County|GA|31711|United States|-5|condo| +21399|AAAAAAAAHJDFAAAA|100|Madison |RD|Suite W|Highland Park|Lincoln County|OK|76534|United States|-6|single family| +21400|AAAAAAAAIJDFAAAA|550|Hillcrest |Boulevard|Suite 180|Deerfield|Stafford County|VA|29840|United States|-5|single family| +21401|AAAAAAAAJJDFAAAA|93|Spring |Avenue|Suite J|Lakewood|Washington County|FL|38877|United States|-5|single family| +21402|AAAAAAAAKJDFAAAA|510|Lincoln |Street|Suite 240|Riverdale|Greene County|AL|39391|United States|-6|condo| +21403|AAAAAAAALJDFAAAA|199|Forest |ST|Suite T|Union|Winchester city|VA|28721|United States|-5|apartment| +21404|AAAAAAAAMJDFAAAA|676|Willow Fifth|Avenue|Suite 60|Friendship|Knox County|NE|64536|United States|-7|single family| +21405|AAAAAAAANJDFAAAA|610|Elm |Parkway|Suite 130|Centerville|Klickitat County|WA|90059|United States|-8|condo| +21406|AAAAAAAAOJDFAAAA|811|Church North|Cir.|Suite 180|New Hope|West Carroll Parish|LA|79431|United States|-6|apartment| +21407|AAAAAAAAPJDFAAAA|803|Main |Pkwy|Suite W|Enon|Henry County|IA|50965|United States|-6|apartment| +21408|AAAAAAAAAKDFAAAA|461|Locust 7th|ST|Suite A|Forest Hills|Craig County|VA|29237|United States|-5|single family| +21409|AAAAAAAABKDFAAAA|505|10th 4th|Blvd|Suite 280|Crossroads|Mahoning County|OH|40534|United States|-5|apartment| +21410|AAAAAAAACKDFAAAA|284|Lincoln 4th|Way|Suite T|Deerfield|Esmeralda County|NV|89840|United States|-8|apartment| +21411|AAAAAAAADKDFAAAA|463|Main |Lane|Suite T|Clifton|Monroe County|WV|28014|United States|-5|apartment| +21412|AAAAAAAAEKDFAAAA|649|1st Lincoln|Ct.|Suite 250|Georgetown|Clearfield County|PA|17057|United States|-5|condo| +21413|AAAAAAAAFKDFAAAA|453|Main Walnut|Ct.|Suite 100|Spring Hill|Wayne County|IN|46787|United States|-5|apartment| +21414|AAAAAAAAGKDFAAAA|48|Main |Ln|Suite 40|Buena Vista|Tippecanoe County|IN|45752|United States|-5|single family| +21415|AAAAAAAAHKDFAAAA|499|Spruce |Way|Suite 10|Five Points|Knott County|KY|46098|United States|-5|condo| +21416|AAAAAAAAIKDFAAAA|798|Laurel Pine|Avenue|Suite 380|Newtown|Benton County|IA|51749|United States|-6|condo| +21417|AAAAAAAAJKDFAAAA|560|Williams Wilson|Road|Suite 390|Lakeside|McIntosh County|OK|79532|United States|-6|condo| +21418|AAAAAAAAKKDFAAAA|521|15th Railroad|Ct.|Suite 400|Five Forks|Cabell County|WV|22293|United States|-5|apartment| +21419|AAAAAAAALKDFAAAA|198|Lincoln |Boulevard|Suite 180|Providence|Kendall County|IL|66614|United States|-6|apartment| +21420|AAAAAAAAMKDFAAAA|740|Central Wilson|Road|Suite 130|Plainview|Baker County|GA|33683|United States|-5|condo| +21421|AAAAAAAANKDFAAAA|372|Valley Central|Blvd|Suite 450|Royal|Meade County|SD|55819|United States|-7|apartment| +21422|AAAAAAAAOKDFAAAA|527|6th |Ave|Suite L|Bethel|Bullock County|AL|35281|United States|-6|apartment| +21423|AAAAAAAAPKDFAAAA|3|Main |Street|Suite Q|Fairfield|Albany County|WY|86192|United States|-7|condo| +21424|AAAAAAAAALDFAAAA|280|2nd |Dr.|Suite A|Georgetown|Lexington County|SC|27057|United States|-5|apartment| +21425|AAAAAAAABLDFAAAA|492|View Seventh|Drive|Suite D|Clearview|Oldham County|TX|75495|United States|-6|single family| +21426|AAAAAAAACLDFAAAA|334|11th Seventh|Road|Suite 410|Centerville|Mitchell County|TX|70059|United States|-6|apartment| +21427|AAAAAAAADLDFAAAA|879|7th |RD|Suite 380|Pleasant Hill|Kalamazoo County|MI|43604|United States|-5|single family| +21428|AAAAAAAAELDFAAAA|296|Highland |Road|Suite V|Marion|Crook County|OR|90399|United States|-8|single family| +21429|AAAAAAAAFLDFAAAA|253|Wilson |RD|Suite 110|Woodville|Benton County|MN|54289|United States|-6|condo| +21430|AAAAAAAAGLDFAAAA|894|Oak |Ave|Suite 0|Providence|Vigo County|IN|46614|United States|-5|condo| +21431|AAAAAAAAHLDFAAAA|654|Meadow Meadow|RD|Suite S|Greenville|Linn County|IA|51387|United States|-6|single family| +21432|AAAAAAAAILDFAAAA|507|3rd Adams|RD|Suite 140|Springtown|Wasatch County|UT|89858|United States|-7|single family| +21433|AAAAAAAAJLDFAAAA|477|Forest |Way|Suite M|Ashland|Henderson County|TX|74244|United States|-6|single family| +21434|AAAAAAAAKLDFAAAA|994|Highland 2nd|RD|Suite J|Lakewood|Real County|TX|78877|United States|-6|apartment| +21435|AAAAAAAALLDFAAAA|229|College |Wy|Suite 10|White Oak|Banks County|GA|36668|United States|-5|condo| +21436|AAAAAAAAMLDFAAAA|514|Cherry Sixth|Ln|Suite 350|Riverdale|Lincoln County|WY|89391|United States|-7|condo| +21437|AAAAAAAANLDFAAAA|780|Wilson Hickory|Way|Suite F|Morris|Benton County|MO|66696|United States|-6|single family| +21438|AAAAAAAAOLDFAAAA|368|Lake Sycamore|Cir.|Suite 420|Walnut Grove|Scott County|MO|67752|United States|-6|condo| +21439|AAAAAAAAPLDFAAAA|606|River |Ct.|Suite 240|Providence|Lincoln County|WV|26614|United States|-5|single family| +21440|AAAAAAAAAMDFAAAA|467|Poplar |Pkwy|Suite 230|Bayview|Dunklin County|MO|69672|United States|-6|condo| +21441|AAAAAAAABMDFAAAA|595|Lincoln |Court|Suite W|Cedar Grove|Coffee County|TN|30411|United States|-5|single family| +21442|AAAAAAAACMDFAAAA|117|6th |Lane|Suite Q|Antioch|Gove County|KS|68605|United States|-6|condo| +21443|AAAAAAAADMDFAAAA|887|Sixth Cherry|Boulevard|Suite V|Fairview|Kossuth County|IA|55709|United States|-6|condo| +21444|AAAAAAAAEMDFAAAA|693|Wilson 3rd|Pkwy|Suite F|Farmington|Wayne County|KY|49145|United States|-5|condo| +21445|AAAAAAAAFMDFAAAA|134|Poplar |ST|Suite P|Mount Pleasant|Jefferson County|ID|81933|United States|-7|condo| +21446|AAAAAAAAGMDFAAAA|546|Fifth Elm|Road|Suite R|Midway|Linn County|IA|51904|United States|-6|apartment| +21447|AAAAAAAAHMDFAAAA|825|Sunset |Ln|Suite T|Florence|Brunswick County|VA|23394|United States|-5|condo| +21448|AAAAAAAAIMDFAAAA|950|7th 4th|Street|Suite 380|Mount Zion|Sweetwater County|WY|88054|United States|-7|single family| +21449|AAAAAAAAJMDFAAAA|778|Laurel |Dr.|Suite V|Sulphur Springs|Independence County|AR|78354|United States|-6|apartment| +21450|AAAAAAAAKMDFAAAA|474|Main Cedar|Pkwy|Suite P|Reno|Hand County|SD|50344|United States|-7|condo| +21451|AAAAAAAALMDFAAAA|997|7th |Cir.|Suite 100|Plainview|Dimmit County|TX|73683|United States|-6|apartment| +21452|AAAAAAAAMMDFAAAA|424|Second Laurel|ST|Suite W|Highland Park|Marshall County|IA|56534|United States|-6|condo| +21453|AAAAAAAANMDFAAAA|800|Elm Sunset|Ave|Suite L|Georgetown|Edgefield County|SC|27057|United States|-5|condo| +21454|AAAAAAAAOMDFAAAA|750|Smith |ST|Suite 400|Hopewell|Wilkes County|GA|30587|United States|-5|apartment| +21455|AAAAAAAAPMDFAAAA||Johnson |Cir.|Suite G|Lakewood|Dixon County|NE|||-6|| +21456|AAAAAAAAANDFAAAA|246|East |Circle|Suite C|Mountain View|Fort Bend County|TX|74466|United States|-6|condo| +21457|AAAAAAAABNDFAAAA|119|Lake |Blvd|Suite J|Antioch|Dolores County|CO|88605|United States|-7|condo| +21458|AAAAAAAACNDFAAAA|291|Hickory Pine|ST|Suite 370|Florence|Carroll County|NH|03994|United States|-5|condo| +21459|AAAAAAAADNDFAAAA|188|Poplar |ST|Suite I|Enterprise|Isle of Wight County|VA|21757|United States|-5|condo| +21460|AAAAAAAAENDFAAAA|373|Sunset Mill|Court|Suite W|Riverdale|Burleigh County|ND|59391|United States|-6|condo| +21461|AAAAAAAAFNDFAAAA|776|10th Jefferson|Road|Suite 140|Highland Park|Jasper County|SC|26534|United States|-5|single family| +21462|AAAAAAAAGNDFAAAA|669|Lake |Way|Suite R|Fairfield|Lyon County|KS|66192|United States|-6|single family| +21463|AAAAAAAAHNDFAAAA|130|2nd |Wy|Suite 320|Hidden Valley|Monroe County|MO|65521|United States|-6|single family| +21464|AAAAAAAAINDFAAAA|249|Highland 2nd|Drive|Suite 210|Wilson|Morton County|KS|66971|United States|-6|apartment| +21465|AAAAAAAAJNDFAAAA|543|Church 4th|Blvd|Suite D|Mount Vernon|Sequoyah County|OK|78482|United States|-6|condo| +21466|AAAAAAAAKNDFAAAA|680|2nd Cherry|Drive|Suite 100|Spring Valley|Sanpete County|UT|86060|United States|-7|condo| +21467|AAAAAAAALNDFAAAA|761|2nd |Ln|Suite 150|Jackson|Delaware County|OH|49583|United States|-5|condo| +21468|AAAAAAAAMNDFAAAA|121|Mill Dogwood|Street|Suite 210|Pleasant Valley|Worth County|GA|32477|United States|-5|condo| +21469|AAAAAAAANNDFAAAA|435|10th |Blvd|Suite 340|Forest Hills|Cass County|MI|49237|United States|-5|condo| +21470|AAAAAAAAONDFAAAA|461|Main Spring|Ct.|Suite 80|Jamestown|Butler County|MO|66867|United States|-6|apartment| +21471|AAAAAAAAPNDFAAAA|265|11th Second|Cir.|Suite 230|Shiloh|Hutchinson County|SD|59275|United States|-7|single family| +21472|AAAAAAAAAODFAAAA|834|Second Seventh|||Stafford||||United States||condo| +21473|AAAAAAAABODFAAAA|829|Hill |Court|Suite 420|Centerville|Grant County|AR|70059|United States|-6|single family| +21474|AAAAAAAACODFAAAA|379|Pine |Circle|Suite K|Lakewood|Kingman County|KS|68877|United States|-6|apartment| +21475|AAAAAAAADODFAAAA|286|Jackson Lakeview|Street|Suite L|Wilson|Grant County|IN|46971|United States|-5|condo| +21476|AAAAAAAAEODFAAAA|375|Ridge 11th|Avenue|Suite Q|Bethel|Lewis County|WV|25281|United States|-5|single family| +21477|AAAAAAAAFODFAAAA|264|7th Main|Ln|Suite O|Mount Vernon|Surry County|VA|28482|United States|-5|apartment| +21478|AAAAAAAAGODFAAAA|727|Smith |Court|Suite 60|Jamestown|Pulaski County|IL|66867|United States|-6|condo| +21479|AAAAAAAAHODFAAAA|934|View |Pkwy|Suite 180|Lone Pine|Randolph County|AR|77441|United States|-6|apartment| +21480|AAAAAAAAIODFAAAA|861|Spring 9th|Wy|Suite C|Bayside|Hamlin County|SD|59550|United States|-7|single family| +21481|AAAAAAAAJODFAAAA|443|Poplar 5th|Boulevard|Suite A|Highland|Logan County|NE|69454|United States|-7|apartment| +21482|AAAAAAAAKODFAAAA|418|Lake |Ct.|Suite 90|Macedonia|Scott County|MN|51087|United States|-6|single family| +21483|AAAAAAAALODFAAAA|265|Smith Main|Blvd|Suite K|Florence|Brule County|SD|53394|United States|-6|condo| +21484|AAAAAAAAMODFAAAA|167|8th Sunset|RD|Suite C|Cedar Grove|Columbia County|PA|10411|United States|-5|apartment| +21485|AAAAAAAANODFAAAA|495|Tenth View|Road|Suite 280|Green Acres|Honolulu County|HI|97683|United States|-10|single family| +21486|AAAAAAAAOODFAAAA|62|Hill Park|Road|Suite 150|Wildwood|Cheboygan County|MI|46871|United States|-5|single family| +21487|AAAAAAAAPODFAAAA|948|Cherry Ninth|Drive|Suite O|Center Point|Lyon County|KY|41883|United States|-5|apartment| +21488|AAAAAAAAAPDFAAAA|848|Oak Jefferson|ST|Suite 140|Green Acres|Washakie County|WY|87683|United States|-7|apartment| +21489|AAAAAAAABPDFAAAA|927|Maple Franklin|Cir.|Suite U|Pleasant Hill|Bourbon County|KS|63604|United States|-6|condo| +21490|AAAAAAAACPDFAAAA|226|Willow West|ST|Suite Y|Five Points|Thurston County|NE|66098|United States|-7|condo| +21491|AAAAAAAADPDFAAAA|284|Valley Railroad|Court|Suite N|Shiloh|Ramsey County|MN|59275|United States|-6|condo| +21492|AAAAAAAAEPDFAAAA|151|Ninth |Street|Suite 110|Hopewell|Luce County|MI|40587|United States|-5|apartment| +21493|AAAAAAAAFPDFAAAA|582|Highland |RD|Suite 270|Woodville|Sawyer County|WI|54289|United States|-6|condo| +21494|AAAAAAAAGPDFAAAA|56|3rd |RD|Suite J|Fairfield|Grant County|WI|56192|United States|-6|condo| +21495|AAAAAAAAHPDFAAAA|217||Ct.||||IA|50534||-6|apartment| +21496|AAAAAAAAIPDFAAAA|528|11th 4th|RD|Suite P|Mount Vernon|Armstrong County|PA|18482|United States|-5|condo| +21497|AAAAAAAAJPDFAAAA|951|Jackson |ST||Woodland||||||| +21498|AAAAAAAAKPDFAAAA|504|First |Way|Suite L|Oak Hill|Fremont County|IA|57838|United States|-6|apartment| +21499|AAAAAAAALPDFAAAA|348|Hill |Circle|Suite 220|Newport|Perry County|OH|41521|United States|-5|apartment| +21500|AAAAAAAAMPDFAAAA|916|Hickory |Drive|Suite E|Spring Hill|Charles City County|VA|26787|United States|-5|condo| +21501|AAAAAAAANPDFAAAA|391|2nd Highland|Circle|Suite K|Willow|Cotton County|OK|76798|United States|-6|single family| +21502|AAAAAAAAOPDFAAAA|449|Laurel |Dr.|Suite B|Wright|Suffolk County|NY|12814|United States|-5|condo| +21503|AAAAAAAAPPDFAAAA|675|7th |Circle|Suite 160|Lakeside|Loup County|NE|69532|United States|-7|single family| +21504|AAAAAAAAAAEFAAAA|238|Hill Fifth|Street|Suite G|Oak Ridge|Gooding County|ID|88371|United States|-7|apartment| +21505|AAAAAAAABAEFAAAA|985|Main Walnut|Cir.|Suite 0|Pine Grove|Dickey County|ND|54593|United States|-6|condo| +21506|AAAAAAAACAEFAAAA|601|9th River|Circle|Suite J|Spring Hill|Benson County|ND|56787|United States|-6|condo| +21507|AAAAAAAADAEFAAAA|410|Washington Adams|ST|Suite 40|Highland Park|Onslow County|NC|26534|United States|-5|apartment| +21508|AAAAAAAAEAEFAAAA|270|Center Mill|Pkwy|Suite M|Sunnyside|Big Horn County|MT|61952|United States|-7|condo| +21509|AAAAAAAAFAEFAAAA|359|Sycamore |Parkway|Suite 190|Florence|Grayson County|VA|23394|United States|-5|single family| +21510|AAAAAAAAGAEFAAAA|26||RD|Suite U||Garden County|NE|61749||-6|| +21511|AAAAAAAAHAEFAAAA|806|4th Lake|ST|Suite N|Shady Grove|Emmet County|IA|52812|United States|-6|condo| +21512|AAAAAAAAIAEFAAAA||Sycamore Park|Parkway|Suite 70|Mountain View||TX||||apartment| +21513|AAAAAAAAJAEFAAAA|612|Mill Church|Blvd|Suite 0|Sunnyside|Fannin County|GA|31952|United States|-5|single family| +21514|AAAAAAAAKAEFAAAA|352|6th |Drive|Suite P|Green Acres|Medina County|TX|77683|United States|-6|apartment| +21515|AAAAAAAALAEFAAAA|103|Maple |Blvd|Suite 310|New Hope|Nantucket County|MA|09431|United States|-5|apartment| +21516|AAAAAAAAMAEFAAAA|153|Valley Maple|Ct.|Suite L|New Hope|Jenkins County|GA|39431|United States|-5|single family| +21517|AAAAAAAANAEFAAAA|153|Adams Madison|Wy|Suite N|Macedonia|Bennington County|VT|01687|United States|-5|single family| +21518|AAAAAAAAOAEFAAAA|484|Cedar Sunset|Court|Suite R|Concord|Wicomico County|MD|24107|United States|-5|apartment| +21519|AAAAAAAAPAEFAAAA|320|Sunset |Drive|Suite S|Glenwood|Lewis County|KY|43511|United States|-5|condo| +21520|AAAAAAAAABEFAAAA|498|2nd |Dr.|Suite 460|Crossroads|Walla Walla County|WA|90534|United States|-8|single family| +21521|AAAAAAAABBEFAAAA|342|Jackson Park|Boulevard|Suite 410|Plainview|Scotts Bluff County|NE|63683|United States|-7|apartment| +21522|AAAAAAAACBEFAAAA|518|Fourth View|Ct.|Suite A|Wilson|Mariposa County|CA|96971|United States|-8|apartment| +21523|AAAAAAAADBEFAAAA|608|Locust |Parkway|Suite 280|Plainview|Palo Pinto County|TX|73683|United States|-6|apartment| +21524|AAAAAAAAEBEFAAAA||Sunset 15th|||Friendship|Dimmit County||74536|||| +21525|AAAAAAAAFBEFAAAA|826|View 11th|RD|Suite 110|Hamilton|Greenville County|SC|22808|United States|-5|apartment| +21526|AAAAAAAAGBEFAAAA|614|1st Walnut|Ct.|Suite N|Oak Ridge|Grant County|IN|48371|United States|-5|single family| +21527|AAAAAAAAHBEFAAAA|775|Highland Church|Dr.|Suite X|Greenville|Greene County|OH|41387|United States|-5|apartment| +21528|AAAAAAAAIBEFAAAA|615|Davis |Court|Suite 350|Wilson|Scotland County|NC|26971|United States|-5|single family| +21529|AAAAAAAAJBEFAAAA|748|Forest |Dr.|Suite 10|Sulphur Springs|Essex County|VA|28354|United States|-5|single family| +21530|AAAAAAAAKBEFAAAA|638|6th |Ln|Suite L|Shady Grove|Montgomery County|KS|62812|United States|-6|apartment| +21531|AAAAAAAALBEFAAAA|938|5th Birch|Court|Suite C|Lebanon|Walthall County|MS|52898|United States|-6|apartment| +21532|AAAAAAAAMBEFAAAA|361|Park |Ct.|Suite 460|Mountain View|Jackson County|IL|64466|United States|-6|apartment| +21533|AAAAAAAANBEFAAAA|355|Woodland Park|Wy|Suite 260|Fairview|Greene County|AR|75709|United States|-6|condo| +21534|AAAAAAAAOBEFAAAA|271|Oak |Street|Suite A|Belmont|Hopkins County|TX|70191|United States|-6|single family| +21535|AAAAAAAAPBEFAAAA|103|First |ST|Suite U|Oak Ridge|Howell County|MO|68371|United States|-6|single family| +21536|AAAAAAAAACEFAAAA|158|Green Center|Boulevard|Suite 170|Jackson|Union County|NJ|09583|United States|-5|apartment| +21537|AAAAAAAABCEFAAAA|896|Cedar |Boulevard|Suite U|Newport|Orange County|FL|31521|United States|-5|single family| +21538|AAAAAAAACCEFAAAA|226|Williams Oak|Lane|Suite O|New Hope|Lonoke County|AR|79431|United States|-6|single family| +21539|AAAAAAAADCEFAAAA|42|North |Parkway|Suite 190|Waterford|Shoshone County|ID|82082|United States|-7|condo| +21540|AAAAAAAAECEFAAAA|2|6th 9th|RD|Suite 120|Blair|Jasper County|MO|65465|United States|-6|apartment| +21541|AAAAAAAAFCEFAAAA|75|Franklin |Way|Suite 10|Friendship|Somerset County|PA|14536|United States|-5|apartment| +21542|AAAAAAAAGCEFAAAA|560|Laurel Chestnut|Wy|Suite 340|Maple Grove|Mohave County|AZ|88252|United States|-7|condo| +21543|AAAAAAAAHCEFAAAA|946|Church |Parkway|Suite 20|Fairview|Phelps County|NE|65709|United States|-7|apartment| +21544|AAAAAAAAICEFAAAA|923|Pine |Street|Suite 170|Mount Olive|Wayne County|KY|48059|United States|-5|apartment| +21545|AAAAAAAAJCEFAAAA|825|Franklin Forest|Ct.|Suite 480|Shiloh|Hancock County|KY|49275|United States|-6|apartment| +21546|AAAAAAAAKCEFAAAA|151|Highland Ash|Pkwy|Suite 220|Five Forks|Clay County|NE|62293|United States|-6|single family| +21547|AAAAAAAALCEFAAAA|699|East |Wy|Suite S|Riverdale|Washington County|ID|89391|United States|-7|condo| +21548|AAAAAAAAMCEFAAAA|237|Hickory |Dr.|Suite W|Silver Creek|DeKalb County|TN|34546|United States|-5|single family| +21549|AAAAAAAANCEFAAAA|220|Hill South|Court|Suite 450|Ashland|Grant County|SD|54244|United States|-6|single family| +21550|AAAAAAAAOCEFAAAA|623|Lincoln |Cir.|Suite 420|Greenwood|Wythe County|VA|28828|United States|-5|single family| +21551|AAAAAAAAPCEFAAAA||3rd 4th|Ave|Suite 380|Waterloo||VA|||-5|| +21552|AAAAAAAAADEFAAAA|320|Park |Wy|Suite V|Frankfort|Jackson County|FL|39681|United States|-5|single family| +21553|AAAAAAAABDEFAAAA||South Park|RD||||||||condo| +21554|AAAAAAAACDEFAAAA|141|Madison Forest|Parkway|Suite 240|Glendale|Leavenworth County|KS|63951|United States|-6|apartment| +21555|AAAAAAAADDEFAAAA|962|Mill 7th|Lane|Suite K|Mechanicsburg|Valley County|ID|82219|United States|-7|single family| +21556|AAAAAAAAEDEFAAAA||||Suite 160|||||United States||apartment| +21557|AAAAAAAAFDEFAAAA|58|6th North|Lane|Suite R|Clifton|Mercer County|NJ|08614|United States|-5|single family| +21558|AAAAAAAAGDEFAAAA|338|Tenth |Ln|Suite D|Wilson|Schoolcraft County|MI|46971|United States|-5|apartment| +21559|AAAAAAAAHDEFAAAA|684|12th Pine|Ct.|Suite 310|White Oak|Covington County|MS|56668|United States|-6|single family| +21560|AAAAAAAAIDEFAAAA|55|Eigth 9th|Boulevard|Suite V|Salem|Pulaski County|MO|68048|United States|-6|apartment| +21561|AAAAAAAAJDEFAAAA|526|6th |ST|Suite W|Harmony|Kinney County|TX|75804|United States|-6|single family| +21562|AAAAAAAAKDEFAAAA|80|Fourth Smith|ST|Suite I|Riverview|Greene County|NY|19003|United States|-5|condo| +21563|AAAAAAAALDEFAAAA|193|East |Boulevard|Suite B|Oak Hill|Aurora County|SD|57838|United States|-6|apartment| +21564|AAAAAAAAMDEFAAAA|544|Birch Twelfth|Parkway|Suite 240|Springfield|Freestone County|TX|79303|United States|-6|apartment| +21565|AAAAAAAANDEFAAAA|738|Oak Highland|Boulevard|Suite 200|Fairview|Henry County|AL|35709|United States|-6|apartment| +21566|AAAAAAAAODEFAAAA|723|Hill Pine|ST|Suite 400|Newtown|Linn County|KS|61749|United States|-6|condo| +21567|AAAAAAAAPDEFAAAA|785|Hill Cedar|Ave|Suite 190|Jamestown|Venango County|PA|16867|United States|-5|condo| +21568|AAAAAAAAAEEFAAAA|175|Maple Sycamore|Parkway|Suite 390|Five Forks|Mitchell County|TX|72293|United States|-6|single family| +21569|AAAAAAAABEEFAAAA|832|Third |Cir.|Suite 50|Red Hill|Pottawatomie County|KS|64338|United States|-6|apartment| +21570|AAAAAAAACEEFAAAA|551|Birch Fourth|Drive|Suite U|Belmont|Williamson County|IL|60191|United States|-6|apartment| +21571|AAAAAAAADEEFAAAA|121|1st |Ave|Suite V|Cedar Grove|Weld County|CO|80411|United States|-7|condo| +21572|AAAAAAAAEEEFAAAA|145|1st 4th|Parkway|Suite 210|Lakeside|Elk County|KS|69532|United States|-6|apartment| +21573|AAAAAAAAFEEFAAAA|771|Laurel |Lane|Suite K|Riverview|Greene County|NC|29003|United States|-5|condo| +21574|AAAAAAAAGEEFAAAA|440|Third |Pkwy|Suite F|Hopewell|Holmes County|FL|30587|United States|-5|apartment| +21575|AAAAAAAAHEEFAAAA|787|15th Main|Wy|Suite Q|Lone Pine|Duplin County|NC|27441|United States|-5|apartment| +21576|AAAAAAAAIEEFAAAA|817|Madison 10th|Ave|||Stanislaus County|CA|91521|||condo| +21577|AAAAAAAAJEEFAAAA|382|Walnut |Cir.|Suite N|Fairview|Hooker County|NE|65709|United States|-7|single family| +21578|AAAAAAAAKEEFAAAA|877|Highland East|Road|Suite I|Valley View|Bartow County|GA|35124|United States|-5|single family| +21579|AAAAAAAALEEFAAAA|341|Cedar |Cir.|Suite D|Concord|Eaton County|MI|44107|United States|-5|single family| +21580|AAAAAAAAMEEFAAAA|317|Hillcrest |Avenue|Suite S|Highland Park|Placer County|CA|96534|United States|-8|apartment| +21581|AAAAAAAANEEFAAAA|816|Third Park|Road|Suite O|Springfield|Clark County|WA|99303|United States|-8|apartment| +21582|AAAAAAAAOEEFAAAA|847|Fifth |RD|Suite 370|Greenville|Jefferson County|OR|91387|United States|-8|condo| +21583|AAAAAAAAPEEFAAAA|887|First |Ave|Suite 110|Hillcrest|Owsley County|KY|43003|United States|-5|single family| +21584|AAAAAAAAAFEFAAAA|761|5th Forest|Ct.|Suite V|Deerfield|Rankin County|MS|59840|United States|-6|single family| +21585|AAAAAAAABFEFAAAA|729|Cherry |Dr.|Suite 300|Enterprise|Defiance County|OH|41757|United States|-5|apartment| +21586|AAAAAAAACFEFAAAA|572|15th 4th|Blvd|Suite 20|Lakeview|Lincoln County|KY|48579|United States|-5|apartment| +21587|AAAAAAAADFEFAAAA|178|Main |Street|Suite X|Woodlawn|Lander County|NV|84098|United States|-8|apartment| +21588|AAAAAAAAEFEFAAAA|193|Fourth 4th|Wy|Suite X|Lincoln|McDuffie County|GA|31289|United States|-5|condo| +21589|AAAAAAAAFFEFAAAA|367|9th |Circle|Suite F|Franklin|Edwards County|KS|69101|United States|-6|single family| +21590|AAAAAAAAGFEFAAAA|81|Pine |RD|Suite 250|Greenfield|Howard County|MO|65038|United States|-6|apartment| +21591|AAAAAAAAHFEFAAAA|858|4th |ST|Suite X|Bunker Hill|Jefferson Davis County|MS|50150|United States|-6|condo| +21592|AAAAAAAAIFEFAAAA|448|Broadway |RD|Suite 410|Hillcrest|Sheridan County|NE|63003|United States|-7|apartment| +21593|AAAAAAAAJFEFAAAA|506|Highland |Ln|Suite V|Concord|Hunterdon County|NJ|04707|United States|-5|condo| +21594|AAAAAAAAKFEFAAAA|506|East |Court|Suite 370|Oakwood|Quitman County|MS|50169|United States|-6|condo| +21595|AAAAAAAALFEFAAAA|809|Elm Jefferson|Circle|Suite R|Stringtown|Wirt County|WV|20162|United States|-5|single family| +21596|AAAAAAAAMFEFAAAA|414|Railroad Maple|Way|Suite 390|Tanglewood|Guernsey County|OH|48994|United States|-5|single family| +21597|AAAAAAAANFEFAAAA|568|Williams Forest|Ave|Suite S|Mount Olive|Ralls County|MO|68059|United States|-6|condo| +21598|AAAAAAAAOFEFAAAA|286|Oak |Boulevard|Suite 410|Kingston|Nueces County|TX|74975|United States|-6|condo| +21599|AAAAAAAAPFEFAAAA|246|East |Cir.|Suite J|Mount Zion|Wallowa County|OR|98054|United States|-8|single family| +21600|AAAAAAAAAGEFAAAA||Green Eigth|Court||Liberty||AL|33451|United States||| +21601|AAAAAAAABGEFAAAA|217|Center Washington|Ct.|Suite 180|Franklin|Towner County|ND|59101|United States|-6|condo| +21602|AAAAAAAACGEFAAAA|746|Elm |ST|Suite S|Gum Springs|Okfuskee County|OK|72106|United States|-6|apartment| +21603|AAAAAAAADGEFAAAA|723|6th |Boulevard|Suite O|Bethesda|Edmonson County|KY|45980|United States|-6|condo| +21604|AAAAAAAAEGEFAAAA|32|Lee Poplar|Parkway|Suite 420|Clifton|Mellette County|SD|58014|United States|-7|condo| +21605|AAAAAAAAFGEFAAAA|34|Valley |ST|Suite U|Spring Hill|Hardin County|OH|46787|United States|-5|condo| +21606|AAAAAAAAGGEFAAAA|990|Fourth |Wy|Suite O|Kingston|Franklin County|NC|24975|United States|-5|condo| +21607|AAAAAAAAHGEFAAAA|140|4th |Ln|||||90499|||condo| +21608|AAAAAAAAIGEFAAAA|689|Meadow Main|Way|Suite 440|Fairfield|Jackson County|KY|46192|United States|-6|condo| +21609|AAAAAAAAJGEFAAAA|60|Sycamore |Court|Suite K|Bethesda|Renville County|ND|55980|United States|-6|single family| +21610|AAAAAAAAKGEFAAAA|733|Cedar Oak|Wy|Suite 200|Franklin|Sampson County|NC|29101|United States|-5|apartment| +21611|AAAAAAAALGEFAAAA|722|Valley Park|Street|Suite I|Woodville|Tillamook County|OR|94289|United States|-8|condo| +21612|AAAAAAAAMGEFAAAA|426|Meadow |Drive|Suite 330|Foster|Wyandotte County|KS|68587|United States|-6|apartment| +21613|AAAAAAAANGEFAAAA|192|Central Maple|Street|Suite 80|Pine Grove|Mora County|NM|84593|United States|-7|apartment| +21614|AAAAAAAAOGEFAAAA|55|Railroad |Parkway|Suite 400|Springfield|Wharton County|TX|79303|United States|-6|apartment| +21615|AAAAAAAAPGEFAAAA|950|View Eigth|Ave|Suite D|Highland Park|Harvey County|KS|66534|United States|-6|condo| +21616|AAAAAAAAAHEFAAAA|14|Park 1st|Way|Suite 310|Ellisville|Custer County|ID|86820|United States|-7|single family| +21617|AAAAAAAABHEFAAAA|610|Valley 6th|Road|Suite 100|Lakeside|Benton County|TN|39532|United States|-5|condo| +21618|AAAAAAAACHEFAAAA|271|Hill Washington|Blvd|Suite G|Morris|Cass County|NE|66696|United States|-6|condo| +21619|AAAAAAAADHEFAAAA|346|Hill |Drive|Suite 280|Stringtown|Jackson County|OK|70162|United States|-6|condo| +21620|AAAAAAAAEHEFAAAA|463|Johnson |Dr.|Suite N|Riverview|Hancock County|IN|49003|United States|-5|single family| +21621|AAAAAAAAFHEFAAAA|936|Maple 5th|Road|Suite 410|Mount Olive|Midland County|TX|78059|United States|-6|single family| +21622|AAAAAAAAGHEFAAAA|925|Washington |Blvd|Suite F|Wilson|Roanoke city|VA|26971|United States|-5|single family| +21623|AAAAAAAAHHEFAAAA|734|Oak |Ct.|Suite 480|Hopewell|Spencer County|KY|40587|United States|-5|condo| +21624|AAAAAAAAIHEFAAAA|185|Lincoln Walnut|Ave|Suite V|Quincy|Bristol Bay Borough|AK|93868|United States|-9|condo| +21625|AAAAAAAAJHEFAAAA|210|Johnson Lake|Avenue|Suite 100|Newport|Decatur County|IN|41521|United States|-5|condo| +21626|AAAAAAAAKHEFAAAA|353|2nd |Court|Suite L|Greenville|Carroll County|MS|51387|United States|-6|condo| +21627|AAAAAAAALHEFAAAA|971|5th Pine|Drive|Suite 220|Lakeside|Wallace County|KS|69532|United States|-6|single family| +21628|AAAAAAAAMHEFAAAA|552|Elm |Street|Suite 460|Riverdale|Rowan County|NC|29391|United States|-5|apartment| +21629|AAAAAAAANHEFAAAA|884|1st |Parkway|Suite U|Parkwood|Liberty County|MT|61669|United States|-7|apartment| +21630|AAAAAAAAOHEFAAAA|787|Adams South|Ct.|Suite R|King|Grady County|OK|70008|United States|-6|single family| +21631|AAAAAAAAPHEFAAAA|383|Madison |Road|Suite 390|Oakdale|Grafton County|NH|09584|United States|-5|single family| +21632|AAAAAAAAAIEFAAAA|662|Jackson Cedar|ST|Suite 280|Plainview|Moody County|SD|53683|United States|-7|single family| +21633|AAAAAAAABIEFAAAA|883|Oak |Ln|Suite 450|Glenwood|Traverse County|MN|53511|United States|-6|apartment| +21634|AAAAAAAACIEFAAAA|589|Pine |Dr.|Suite F|Mount Zion|Scott County|MN|58054|United States|-6|condo| +21635|AAAAAAAADIEFAAAA|318|Railroad Washington|Way|Suite V|Stringtown|Otsego County|MI|40162|United States|-5|apartment| +21636|AAAAAAAAEIEFAAAA|765|Franklin |Cir.|Suite 360|Antioch|Tunica County|MS|58605|United States|-6|single family| +21637|AAAAAAAAFIEFAAAA|650|Oak |Parkway|Suite O|Centerville|Boone County|KY|40059|United States|-6|single family| +21638|AAAAAAAAGIEFAAAA|855|Spring |Street|Suite 220|Spring Hill|Pittsburg County|OK|76787|United States|-6|single family| +21639|AAAAAAAAHIEFAAAA|399|Fourth 5th|Wy|Suite 250|Liberty|Sierra County|NM|83451|United States|-7|single family| +21640|AAAAAAAAIIEFAAAA|890|Ninth Hickory|Cir.|Suite 170|Antioch|Pickens County|GA|38605|United States|-5|apartment| +21641|AAAAAAAAJIEFAAAA|659|Second Oak|Court|Suite 370|Red Hill|Kalkaska County|MI|44338|United States|-5|single family| +21642|AAAAAAAAKIEFAAAA|205|6th |Way|Suite 350|Valley View|Putnam County|TN|35124|United States|-6|single family| +21643|AAAAAAAALIEFAAAA|86|Williams Hill|RD|Suite U|Georgetown|Newport County|RI|07657|United States|-5|single family| +21644|AAAAAAAAMIEFAAAA|688|1st |Avenue|Suite B|Woodland|Meigs County|OH|44854|United States|-5|apartment| +21645|AAAAAAAANIEFAAAA|958|South |Pkwy|Suite X|Belmont|Pulaski County|IN|40191|United States|-5|single family| +21646|AAAAAAAAOIEFAAAA|840|Smith |Wy|Suite G|Spring Hill|Carteret County|NC|26787|United States|-5|apartment| +21647|AAAAAAAAPIEFAAAA|529|Park 4th|Way|Suite X|Mount Vernon|Hart County|GA|38482|United States|-5|apartment| +21648|AAAAAAAAAJEFAAAA|791|13th Sunset|Ln|Suite V|Parkwood|Clark County|NV|81669|United States|-8|single family| +21649|AAAAAAAABJEFAAAA|307|||Suite Q|Bethel|Anderson County|||United States||single family| +21650|AAAAAAAACJEFAAAA|620|River Park|Way|Suite 10|Saint Johns|Granite County|MT|65717|United States|-7|apartment| +21651|AAAAAAAADJEFAAAA|497|Franklin 6th|Wy|Suite H|Woodville|Nueces County|TX|74289|United States|-6|apartment| +21652|AAAAAAAAEJEFAAAA|596|Fifth |Court|Suite 200|Hillcrest|Orleans County|NY|13003|United States|-5|single family| +21653|AAAAAAAAFJEFAAAA|79|Sunset Sycamore|Ct.|Suite 70|Midway|Wilcox County|AL|31904|United States|-6|apartment| +21654|AAAAAAAAGJEFAAAA|341|Franklin |Ln|Suite 460|Lincoln|Morris County|KS|61289|United States|-6|condo| +21655|AAAAAAAAHJEFAAAA|944|Ninth Jefferson|ST|Suite U|Newport|Dickinson County|IA|51521|United States|-6|apartment| +21656|AAAAAAAAIJEFAAAA|951|Hickory Fourteenth|Circle|Suite W|Harmony|Red River Parish|LA|75804|United States|-6|single family| +21657|AAAAAAAAJJEFAAAA|252|Davis 5th|Dr.|Suite D|Pleasant Valley|Jackson County|OR|92477|United States|-8|apartment| +21658|AAAAAAAAKJEFAAAA|535|View 2nd|Ct.|Suite 310|Shady Grove|Nicholas County|KY|42812|United States|-5|apartment| +21659|AAAAAAAALJEFAAAA|663|River Third|Street|Suite 320|Oakdale|Chautauqua County|KS|69584|United States|-6|single family| +21660|AAAAAAAAMJEFAAAA|130|Woodland |Ln|Suite E|Mount Zion|Flathead County|MT|68054|United States|-7|condo| +21661|AAAAAAAANJEFAAAA|621|West |Cir.|Suite 350|Red Hill|Huerfano County|CO|84338|United States|-7|apartment| +21662|AAAAAAAAOJEFAAAA|933|Willow |Ct.|Suite M|Price|Live Oak County|TX|73824|United States|-6|apartment| +21663|AAAAAAAAPJEFAAAA|280|Willow Jackson|Circle|Suite C|Newtown|Jones County|IA|51749|United States|-6|condo| +21664|AAAAAAAAAKEFAAAA|415|Second 2nd|ST|Suite 320|Glenwood|Limestone County|TX|73511|United States|-6|condo| +21665|AAAAAAAABKEFAAAA|747|West |Avenue|Suite G|Glendale|Morgan County|MO|63951|United States|-6|apartment| +21666|AAAAAAAACKEFAAAA|222|First |Pkwy|Suite C|Brownsville|Fannin County|GA|39310|United States|-5|single family| +21667|AAAAAAAADKEFAAAA|229|Pine |Ave|Suite 40|Martinsville|Snyder County|PA|10419|United States|-5|single family| +21668|AAAAAAAAEKEFAAAA|647|Lincoln |Way|Suite 230|Five Forks|Taylor County|FL|32293|United States|-5|apartment| +21669|AAAAAAAAFKEFAAAA|272|Dogwood Sunset|Pkwy|Suite 0|Jackson|Madison County|MT|69583|United States|-7|condo| +21670|AAAAAAAAGKEFAAAA|908|Dogwood East|Pkwy|Suite 310|Hamilton|Grant County|NM|82808|United States|-7|single family| +21671|AAAAAAAAHKEFAAAA|199|13th |Boulevard|Suite 320|Brownsville|Tulsa County|OK|79310|United States|-6|single family| +21672|AAAAAAAAIKEFAAAA|708|Spring |Road|Suite 420|Fowler|Washtenaw County|MI|41083|United States|-5|condo| +21673|AAAAAAAAJKEFAAAA|422|Park |Street|Suite S|Georgetown|Okeechobee County|FL|37057|United States|-5|single family| +21674|AAAAAAAAKKEFAAAA|575|Hill |Street|Suite X|Pleasant Valley|Coos County|OR|92477|United States|-8|apartment| +21675|AAAAAAAALKEFAAAA|645|Walnut |Way|Suite 220|Glenwood|Cheyenne County|NE|63511|United States|-6|single family| +21676|AAAAAAAAMKEFAAAA|813|Poplar Lee|Drive|Suite 450|La Grange|Lamar County|GA|37941|United States|-5|condo| +21677|AAAAAAAANKEFAAAA|233|View Adams|ST|Suite A|Bethel|Perry County|OH|45281|United States|-5|apartment| +21678|AAAAAAAAOKEFAAAA|454|Third Twelfth|Drive|Suite X|Lebanon|Kenai Peninsula Borough|AK|92898|United States|-9|single family| +21679|AAAAAAAAPKEFAAAA|954|Fifth |ST|Suite 60|Valley View|Evans County|GA|35124|United States|-5|apartment| +21680|AAAAAAAAALEFAAAA|739|Cedar Locust|Ln|Suite 300|Georgetown|Pleasants County|WV|27057|United States|-5|apartment| +21681|AAAAAAAABLEFAAAA|759|Woodland |ST|Suite 440|Woodlawn|Knox County|ME|04698|United States|-5|single family| +21682|AAAAAAAACLEFAAAA|87|Smith Adams|Court|Suite G|Kingston|San Patricio County|TX|74975|United States|-6|condo| +21683|AAAAAAAADLEFAAAA|307|4th Lincoln|Ln|Suite I|Harmony|Platte County|NE|65804|United States|-7|condo| +21684|AAAAAAAAELEFAAAA|269|1st Laurel|Wy|Suite P|Mount Zion|Marshall County|TN|38054|United States|-6|single family| +21685|AAAAAAAAFLEFAAAA|559|11th 5th|Boulevard|Suite M|Ashland|Newton County|MS|54244|United States|-6|single family| +21686|AAAAAAAAGLEFAAAA|749|Woodland Oak|Circle|Suite 390|Glenwood|Sunflower County|MS|53511|United States|-6|apartment| +21687|AAAAAAAAHLEFAAAA|481|Birch |Dr.|Suite 110|Unionville|Hamilton County|IN|41711|United States|-5|condo| +21688|AAAAAAAAILEFAAAA|42|Pine |Blvd|Suite J|Maple Grove|Murray County|GA|38252|United States|-5|condo| +21689|AAAAAAAAJLEFAAAA|121|4th Lincoln|Ln|Suite I|Spring Valley|Potter County|SD|56060|United States|-7|apartment| +21690|AAAAAAAAKLEFAAAA|94|6th Sunset|Dr.|Suite 480|Friendship|Briscoe County|TX|74536|United States|-6|apartment| +21691|AAAAAAAALLEFAAAA|805|Maple |Street|Suite O|Greenville|Cocke County|TN|31387|United States|-5|single family| +21692|AAAAAAAAMLEFAAAA|352|Ninth |Dr.|Suite 30|Lakeview|Monroe County|NY|18579|United States|-5|condo| +21693|AAAAAAAANLEFAAAA|761|Meadow |Dr.|Suite G|Glendale|Essex County|MA|04551|United States|-5|condo| +21694|AAAAAAAAOLEFAAAA|107|Hickory Forest|Avenue|Suite J|Summit|Linn County|KS|60499|United States|-6|apartment| +21695|AAAAAAAAPLEFAAAA|615|Railroad Third|Boulevard|Suite 90|Morris|Columbia County|FL|36696|United States|-5|apartment| +21696|AAAAAAAAAMEFAAAA|449|Miller 8th|Blvd|Suite C|Wildwood|Jackson County|OR|96871|United States|-8|condo| +21697|AAAAAAAABMEFAAAA|509|Park Eigth|Circle|Suite 300|Lenox|Chesterfield County|SC|21143|United States|-5|apartment| +21698|AAAAAAAACMEFAAAA|393|Jackson |ST|Suite 350|Oakwood|Delta County|CO|80169|United States|-7|single family| +21699|AAAAAAAADMEFAAAA|332|2nd Madison|Drive|Suite P|Bethel|Washington County|NY|15281|United States|-5|apartment| +21700|AAAAAAAAEMEFAAAA|646|Broadway |Road|Suite 80|Pleasant Grove|Wayne County|KY|44136|United States|-5|condo| +21701|AAAAAAAAFMEFAAAA|336|Dogwood |Lane|Suite U|Edwards|Lincoln County|ME|02009|United States|-5|condo| +21702|AAAAAAAAGMEFAAAA|527|Madison Lake|Ct.|Suite 230|Arlington|Montgomery County|MS|56557|United States|-6|single family| +21703|AAAAAAAAHMEFAAAA|691|Ninth 5th|Wy|Suite A|Shiloh|Gallatin County|KY|49275|United States|-6|condo| +21704|AAAAAAAAIMEFAAAA|874|1st Willow|Lane|Suite 400|Enterprise|Donley County|TX|71757|United States|-6|single family| +21705|AAAAAAAAJMEFAAAA|35|Walnut Jackson|Pkwy|Suite H|Bunker Hill|Yamhill County|OR|90150|United States|-8|single family| +21706|AAAAAAAAKMEFAAAA|489|Third Williams|Court|Suite M|Clinton|Holmes County|MS|58222|United States|-6|apartment| +21707|AAAAAAAALMEFAAAA|360|Cedar |Pkwy|Suite J|Bethel|Garvin County|OK|75281|United States|-6|single family| +21708|AAAAAAAAMMEFAAAA|42|Park Park|Dr.|Suite G|Hopewell|Kendall County|TX|70587|United States|-6|single family| +21709|AAAAAAAANMEFAAAA|271|Smith Dogwood|Ave|Suite I|Oak Hill|Coffee County|GA|37838|United States|-5|apartment| +21710|AAAAAAAAOMEFAAAA|894|Park |Wy|Suite 220|Marion|Redwood County|MN|50399|United States|-6|single family| +21711|AAAAAAAAPMEFAAAA|501|Elm |Pkwy|Suite R|Glendale|Harrison County|IN|43951|United States|-5|apartment| +21712|AAAAAAAAANEFAAAA|301|4th West|Cir.|Suite 380|Greenwood|Davidson County|TN|38828|United States|-5|single family| +21713|AAAAAAAABNEFAAAA|379|Hill |Way|Suite 430|Mount Zion|Rowan County|KY|48054|United States|-5|condo| +21714|AAAAAAAACNEFAAAA|658|Oak |Boulevard|Suite V|Kingston|Bleckley County|GA|34975|United States|-5|condo| +21715|AAAAAAAADNEFAAAA|476|Washington |Way|Suite 390|Red Hill|Yuba County|CA|94338|United States|-8|apartment| +21716|AAAAAAAAENEFAAAA|83|Cedar |Dr.|Suite 60|Harmon|Buffalo County|SD|55623|United States|-6|condo| +21717|AAAAAAAAFNEFAAAA||||Suite 100|||||||| +21718|AAAAAAAAGNEFAAAA|106|Ridge Sixth|RD|Suite 60|Deerfield|Shelby County|IN|49840|United States|-5|apartment| +21719|AAAAAAAAHNEFAAAA|331|12th Park|Blvd|Suite S|Florence|New Madrid County|MO|63394|United States|-6|single family| +21720|AAAAAAAAINEFAAAA|562|Railroad First|Road|Suite U|Greenville|Arthur County|NE|61387|United States|-6|single family| +21721|AAAAAAAAJNEFAAAA|808|Walnut |Boulevard|Suite E|Union Hill|Harrison County|TX|77746|United States|-6|single family| +21722|AAAAAAAAKNEFAAAA|500|3rd |Lane|Suite 10|Pine Grove|Brown County|NE|64593|United States|-6|condo| +21723|AAAAAAAALNEFAAAA|867|Hill Railroad|Street|Suite 0|Bunker Hill|Curry County|NM|80150|United States|-7|apartment| +21724|AAAAAAAAMNEFAAAA|879|1st |Lane|Suite N|Glendale|Wilcox County|GA|33951|United States|-5|apartment| +21725|AAAAAAAANNEFAAAA|624|8th |Blvd|Suite G|Marion|Sarasota County|FL|30399|United States|-5|condo| +21726|AAAAAAAAONEFAAAA|759|Poplar 7th|Blvd|Suite 190|Greenwood|Rains County|TX|78828|United States|-6|condo| +21727|AAAAAAAAPNEFAAAA|25|Maple Washington|Ct.|Suite 250|Plainview|Frederick County|VA|23683|United States|-5|condo| +21728|AAAAAAAAAOEFAAAA|638|11th Fifth|Dr.|Suite 20|Richardson|Wells County|IN|47687|United States|-5|apartment| +21729|AAAAAAAABOEFAAAA|249|Pine West|Avenue|Suite P|Mount Olive|Perry County|IL|68059|United States|-6|single family| +21730|AAAAAAAACOEFAAAA|774|Laurel |Way|Suite 410|Deerfield|Saline County|NE|69840|United States|-7|single family| +21731|AAAAAAAADOEFAAAA|755|13th ||Suite H||Kenton County|KY|47746|United States||apartment| +21732|AAAAAAAAEOEFAAAA|596|2nd Williams|Drive|Suite T|Wildwood|Moffat County|CO|86871|United States|-7|condo| +21733|AAAAAAAAFOEFAAAA|729|2nd West|Lane|Suite M|Plainview|Eau Claire County|WI|53683|United States|-6|apartment| +21734|AAAAAAAAGOEFAAAA|382|Jefferson |Way|Suite O|Greenfield|Putnam County|FL|35038|United States|-5|single family| +21735|AAAAAAAAHOEFAAAA|619|Jefferson Railroad|Road|Suite 150|Sulphur Springs|Dallas County|TX|78354|United States|-6|condo| +21736|AAAAAAAAIOEFAAAA|898|Center |Wy|Suite R|Belmont|Lycoming County|PA|10191|United States|-5|apartment| +21737|AAAAAAAAJOEFAAAA|997|Lincoln Davis|Blvd|Suite 20|Woodland|Lackawanna County|PA|14854|United States|-5|condo| +21738|AAAAAAAAKOEFAAAA|351|Davis 3rd|Drive|Suite I|Summit|Monroe County|PA|10499|United States|-5|single family| +21739|AAAAAAAALOEFAAAA|507|Highland |Avenue|Suite 60|Woodville|Vermilion County|IL|64289|United States|-6|condo| +21740|AAAAAAAAMOEFAAAA|71|5th West|Wy|Suite C|Union Hill|Pawnee County|NE|67746|United States|-7|condo| +21741|AAAAAAAANOEFAAAA|754|East Jefferson|Dr.|Suite Q|Friendship|Chelan County|WA|94536|United States|-8|apartment| +21742|AAAAAAAAOOEFAAAA|955|River 12th|Ave|Suite 290|Buena Vista|Broadwater County|MT|65752|United States|-7|apartment| +21743|AAAAAAAAPOEFAAAA|685|Oak |Circle|Suite 40|Oak Ridge|Brown County|KS|68371|United States|-6|condo| +21744|AAAAAAAAAPEFAAAA|707|Oak |Boulevard|Suite H|Valley View|Grant County|OK|75124|United States|-6|condo| +21745|AAAAAAAABPEFAAAA|569|Lincoln Pine|ST|Suite W|Oakdale|Hale County|AL|39584|United States|-6|condo| +21746|AAAAAAAACPEFAAAA|309|View Spring|Ave|Suite M|Bunker Hill|Wayne County|NC|20150|United States|-5|condo| +21747|AAAAAAAADPEFAAAA|812|First Spruce|RD|Suite X|Forest Hills|Bronx County|NY|19237|United States|-5|apartment| +21748|AAAAAAAAEPEFAAAA|597|1st |Road|Suite 250|Oak Ridge|Jefferson County|KY|48371|United States|-6|condo| +21749|AAAAAAAAFPEFAAAA|557|Hill |Blvd|Suite 140|Riley|Centre County|PA|11692|United States|-5|single family| +21750|AAAAAAAAGPEFAAAA|502|Highland |Way|Suite 410|Shady Grove|Oneida County|NY|12812|United States|-5|single family| +21751|AAAAAAAAHPEFAAAA|527|Ridge 11th|Street|Suite 130|Greenfield|Greenville County|SC|25038|United States|-5|single family| +21752|AAAAAAAAIPEFAAAA|740|4th Oak|Wy|Suite 90|Mount Vernon|Taney County|MO|68482|United States|-6|single family| +21753|AAAAAAAAJPEFAAAA|297|Jackson |Boulevard|Suite V|Buckingham|Stevens County|WA|94092|United States|-8|single family| +21754|AAAAAAAAKPEFAAAA|673|Spring Elevnth|Parkway|Suite W|Five Forks|Tehama County|CA|92293|United States|-8|single family| +21755|AAAAAAAALPEFAAAA|786|Lee |Ave|Suite 320|Lakewood|Bracken County|KY|48877|United States|-6|condo| +21756|AAAAAAAAMPEFAAAA|176|Railroad 3rd|Cir.|Suite 440|Walnut Grove|Rankin County|MS|57752|United States|-6|single family| +21757|AAAAAAAANPEFAAAA|200|Seventh |Pkwy|Suite Y|Macedonia|Ozark County|MO|61087|United States|-6|apartment| +21758|AAAAAAAAOPEFAAAA|244|Hill |Ct.|Suite H|Lakewood|Republic County|KS|68877|United States|-6|single family| +21759|AAAAAAAAPPEFAAAA|283|Spruce |RD|Suite R|Jamestown|Williamson County|TN|36867|United States|-5|single family| +21760|AAAAAAAAAAFFAAAA|247|4th 8th|Ct.|Suite N|Georgetown|Grafton County|NH|07657|United States|-5|apartment| +21761|AAAAAAAABAFFAAAA|201|Oak Main|Avenue|Suite 70|Woodland|Knox County|ME|05454|United States|-5|condo| +21762|AAAAAAAACAFFAAAA|302|Second |Street|Suite X|Rosewood|Cheshire County|NH|02805|United States|-5|single family| +21763|AAAAAAAADAFFAAAA|189|Poplar Jefferson|Wy|Suite K|Kingston|Muhlenberg County|KY|44975|United States|-5|single family| +21764|AAAAAAAAEAFFAAAA|184|7th |Circle|Suite W|New Hope|Franklin County|NE|69431|United States|-6|single family| +21765|AAAAAAAAFAFFAAAA|264|Chestnut |Cir.|Suite 480|Hillcrest|Montgomery County|AL|33003|United States|-6|apartment| +21766|AAAAAAAAGAFFAAAA|204|Cedar Church|Dr.|Suite L|Gilmore|Grant County|OK|75464|United States|-6|condo| +21767|AAAAAAAAHAFFAAAA|624|5th 1st|Wy|Suite C|Liberty|Payette County|ID|83451|United States|-7|condo| +21768|AAAAAAAAIAFFAAAA|891|Mill Elm|Cir.|Suite H|Fairfield|Karnes County|TX|76192|United States|-6|single family| +21769|AAAAAAAAJAFFAAAA|738|6th |Boulevard|Suite 150|Summit|Weld County|CO|80499|United States|-7|single family| +21770|AAAAAAAAKAFFAAAA|641|Madison |Wy|Suite 280|Wildwood|Russell County|VA|26871|United States|-5|single family| +21771|AAAAAAAALAFFAAAA|226|Hill |Cir.|Suite O|Pleasant Hill|Washington County|UT|83604|United States|-7|apartment| +21772|AAAAAAAAMAFFAAAA|739|Jackson |Blvd|Suite 50|Mountain View|Presidio County|TX|74466|United States|-6|condo| +21773|AAAAAAAANAFFAAAA|875|West Maple|Ct.|Suite I|Hillcrest|Otsego County|NY|13003|United States|-5|single family| +21774|AAAAAAAAOAFFAAAA|410|10th 9th|ST|Suite 140|Spring Valley|Crockett County|TX|76060|United States|-6|apartment| +21775|AAAAAAAAPAFFAAAA|330|Cherry North|Cir.|Suite 290|Jackson|Carter County|TN|39583|United States|-5|condo| +21776|AAAAAAAAABFFAAAA|732|3rd 2nd|Pkwy|Suite 70|Marion|Dawson County|TX|70399|United States|-6|single family| +21777|AAAAAAAABBFFAAAA|644|Miller Oak|Blvd|Suite X|Farmington|Franklin County|WA|99145|United States|-8|apartment| +21778|AAAAAAAACBFFAAAA|290|Maple |Wy|Suite 260|Westgate|Sevier County|TN|32366|United States|-6|condo| +21779|AAAAAAAADBFFAAAA|469|Main Eigth|ST|Suite 360|Spring Hill|White County|AR|76787|United States|-6|apartment| +21780|AAAAAAAAEBFFAAAA|521|8th 3rd|Circle|Suite 0|Clinton|Marion County|KY|48222|United States|-5|apartment| +21781|AAAAAAAAFBFFAAAA|597|14th |Circle|Suite 400|Woodlawn|Anchorage Borough|AK|94098|United States|-9|single family| +21782|AAAAAAAAGBFFAAAA|755|Railroad |Avenue|Suite 330|Friendship|Grimes County|TX|74536|United States|-6|apartment| +21783|AAAAAAAAHBFFAAAA|530|Laurel |Way|Suite K|Wildwood|Edwards County|TX|76871|United States|-6|condo| +21784|AAAAAAAAIBFFAAAA|198|Hill Woodland|Ave|Suite 160|Union Hill|Nobles County|MN|57746|United States|-6|apartment| +21785|AAAAAAAAJBFFAAAA|553|Green North|RD|Suite 390|Union|Blount County|TN|38721|United States|-5|apartment| +21786|AAAAAAAAKBFFAAAA||Fourth |Avenue|Suite J|Spring Valley||||United States||apartment| +21787|AAAAAAAALBFFAAAA|798|2nd |Blvd|Suite 150|Centerville|Pulaski County|MO|60059|United States|-6|single family| +21788|AAAAAAAAMBFFAAAA|946|Ash Forest|RD|Suite 10|Greenfield|Tompkins County|NY|15038|United States|-5|condo| +21789|AAAAAAAANBFFAAAA|990|Main |Circle|Suite W|Oak Grove|Chenango County|NY|18370|United States|-5|condo| +21790|AAAAAAAAOBFFAAAA|747|Spruce |Parkway|Suite 120|Providence|Crawford County|WI|56614|United States|-6|apartment| +21791|AAAAAAAAPBFFAAAA|756|Meadow |Parkway|Suite 50|Shady Grove|Walthall County|MS|52812|United States|-6|condo| +21792|AAAAAAAAACFFAAAA|194|Oak |Lane|Suite L|Plainview|Josephine County|OR|93683|United States|-8|single family| +21793|AAAAAAAABCFFAAAA|||ST|Suite 380|Georgetown|Middlesex County||27057|United States||apartment| +21794|AAAAAAAACCFFAAAA||Main Second|||Franklin|Adams County||89101|United States||condo| +21795|AAAAAAAADCFFAAAA|236|Oak Fifth|Cir.|Suite F|Pleasant Hill|Monongalia County|WV|23604|United States|-5|single family| +21796|AAAAAAAAECFFAAAA|284|East 3rd|Boulevard|Suite A|Wildwood|Maverick County|TX|76871|United States|-6|condo| +21797|AAAAAAAAFCFFAAAA|914|Mill Cherry|Ave|Suite D|Pleasant Valley|Mille Lacs County|MN|52477|United States|-6|apartment| +21798|AAAAAAAAGCFFAAAA|856|Lee |Lane|Suite S|Oakwood|Stafford County|KS|60169|United States|-6|apartment| +21799|AAAAAAAAHCFFAAAA|245|Pine |Wy|Suite K|Shiloh|Monroe County|PA|19275|United States|-5|condo| +21800|AAAAAAAAICFFAAAA|481|Smith |Pkwy|Suite T|Glendale|Fulton County|NY|13951|United States|-5|apartment| +21801|AAAAAAAAJCFFAAAA|629|Park Oak|Ave|Suite 150|Sulphur Springs|Bradley County|AR|78354|United States|-6|condo| +21802|AAAAAAAAKCFFAAAA|82|Ninth 3rd|Drive|Suite 330|Woodville|Pendleton County|WV|24289|United States|-5|apartment| +21803|AAAAAAAALCFFAAAA|294|11th Oak|Ln|Suite V|Stringtown|Castro County|TX|70162|United States|-6|apartment| +21804|AAAAAAAAMCFFAAAA|252|Williams Sixth|Parkway|Suite S|Mount Zion|Robertson County|TN|38054|United States|-6|apartment| +21805|AAAAAAAANCFFAAAA|896|Birch |Blvd|Suite S|Crystal|Ware County|GA|35258|United States|-5|single family| +21806|AAAAAAAAOCFFAAAA|815|Pine 9th|Avenue|Suite I|Ashland|Dickens County|TX|74244|United States|-6|single family| +21807|AAAAAAAAPCFFAAAA|332|River |Way|Suite B|Jackson|Bollinger County|MO|69583|United States|-6|condo| +21808|AAAAAAAAADFFAAAA|808|2nd |Cir.|Suite 440|Macedonia|Phillips County|AR|71087|United States|-6|condo| +21809|AAAAAAAABDFFAAAA|369|Smith |Wy|Suite 410|Arlington|Van Zandt County|TX|76557|United States|-6|condo| +21810|AAAAAAAACDFFAAAA|999|Park |Dr.|Suite E|Woodland|Norton County|KS|64854|United States|-6|condo| +21811|AAAAAAAADDFFAAAA|223|Cherry Pine|Dr.|Suite 350|Pleasant Valley|Potter County|SD|52477|United States|-7|apartment| +21812|AAAAAAAAEDFFAAAA|220|Chestnut |Circle|Suite 370|Pleasant Grove|Greene County|VA|24136|United States|-5|condo| +21813|AAAAAAAAFDFFAAAA|795|Lincoln Willow|Ln|Suite 200|Leon|Jackson County|KS|60913|United States|-6|apartment| +21814|AAAAAAAAGDFFAAAA|247|Park Center|Ct.|Suite 180|Union Hill|Independence County|AR|77746|United States|-6|apartment| +21815|AAAAAAAAHDFFAAAA|843|Green 2nd|Blvd|Suite 280|Salem|Laclede County|MO|68048|United States|-6|apartment| +21816|AAAAAAAAIDFFAAAA||Lincoln |||Concord|||64107|United States|-7|| +21817|AAAAAAAAJDFFAAAA|596|Cedar Walnut|Dr.|Suite 400|Oak Ridge|Beaufort County|NC|28371|United States|-5|apartment| +21818|AAAAAAAAKDFFAAAA|47|Fourth |Street|Suite 230|Greenwood|Crockett County|TX|78828|United States|-6|apartment| +21819|AAAAAAAALDFFAAAA|390|Adams |Way|Suite C|Edgewood|Montgomery County|TX|70069|United States|-6|single family| +21820|AAAAAAAAMDFFAAAA|538|Railroad |Parkway|Suite 340|Woodland|Somerset County|PA|14854|United States|-5|single family| +21821|AAAAAAAANDFFAAAA|231|Smith 9th|Lane|Suite 300|Sunnyside|Kent County|MI|41952|United States|-5|single family| +21822|AAAAAAAAODFFAAAA|860|Maple |Boulevard|Suite S|Sulphur Springs|Evans County|GA|38354|United States|-5|apartment| +21823|AAAAAAAAPDFFAAAA|518|Church |Ave|Suite 210|Hillcrest|Rolette County|ND|53003|United States|-6|condo| +21824|AAAAAAAAAEFFAAAA|943|Valley Poplar|Wy|Suite 50|Unionville|Clarion County|PA|11711|United States|-5|apartment| +21825|AAAAAAAABEFFAAAA|899|Sycamore |RD|Suite H|Point Pleasant|Gillespie County|TX|74749|United States|-6|single family| +21826|AAAAAAAACEFFAAAA|290|Cedar Elm|Way|Suite B|Five Points|Houston County|TX|76098|United States|-6|condo| +21827|AAAAAAAADEFFAAAA|615|Third |Ln|Suite 240|Mount Zion|Brule County|SD|58054|United States|-6|apartment| +21828|AAAAAAAAEEFFAAAA|512|11th |Wy|Suite H|Hopewell|Waynesboro city|VA|20587|United States|-5|condo| +21829|AAAAAAAAFEFFAAAA|663|||Suite O|||TN||United States|-6|apartment| +21830|AAAAAAAAGEFFAAAA|602|Jackson 1st|Court|Suite O|Wilson|Roane County|WV|26971|United States|-5|condo| +21831|AAAAAAAAHEFFAAAA|375|Main |Street|Suite O|Brownsville|Spencer County|KY|49310|United States|-5|single family| +21832|AAAAAAAAIEFFAAAA|849|8th Madison||Suite 190|Four Points|Rowan County|NC|21216|||| +21833|AAAAAAAAJEFFAAAA|857|North Maple|ST|Suite 390|Oakland|Warrick County|IN|49843|United States|-5|condo| +21834|AAAAAAAAKEFFAAAA|535|Lake Park|Lane|Suite W|Highland|Scurry County|TX|79454|United States|-6|single family| +21835|AAAAAAAALEFFAAAA|530|Broadway 3rd|Pkwy|Suite 150|Mount Olive|Dickens County|TX|78059|United States|-6|single family| +21836|AAAAAAAAMEFFAAAA|179|Walnut Park|Boulevard|Suite A|Ashland|Ottawa County|OH|44244|United States|-5|apartment| +21837|AAAAAAAANEFFAAAA|453|Church |Wy|Suite N|Greenfield|Emmet County|MI|45038|United States|-5|single family| +21838|AAAAAAAAOEFFAAAA|536|West |Pkwy|Suite 420|Riverview|Vernon County|WI|59003|United States|-6|apartment| +21839|AAAAAAAAPEFFAAAA|932|Spruce Fourth|Ln|Suite O|Woodville|Howard County|MD|24289|United States|-5|condo| +21840|AAAAAAAAAFFFAAAA|740|Second 3rd|Ln|Suite 230|Plainview|Ross County|OH|43683|United States|-5|condo| +21841|AAAAAAAABFFFAAAA|890|Highland |Court|Suite X|Hamilton|Jones County|TX|72808|United States|-6|apartment| +21842|AAAAAAAACFFFAAAA|420|West Woodland|Ct.|Suite V|Summit|Gray County|KS|60499|United States|-6|apartment| +21843|AAAAAAAADFFFAAAA|396|Maple |Road|Suite L|Buena Vista|Pike County|OH|45752|United States|-5|condo| +21844|AAAAAAAAEFFFAAAA|386|Adams |Dr.|Suite 430|Mountain View|Lee County|GA|34466|United States|-5|condo| +21845|AAAAAAAAFFFFAAAA|663|Washington Second|RD|Suite X|Oak Hill|Walsh County|ND|57838|United States|-6|condo| +21846|AAAAAAAAGFFFAAAA|53|Miller Pine|Boulevard|Suite U|Waterloo|Durham County|NC|21675|United States|-5|condo| +21847|AAAAAAAAHFFFAAAA|875|7th |Avenue|Suite 60|Oakdale|Burke County|ND|59584|United States|-6|condo| +21848|AAAAAAAAIFFFAAAA|875|Washington |Lane|Suite 20|Walnut Grove|Cowley County|KS|67752|United States|-6|condo| +21849|AAAAAAAAJFFFAAAA|929|Spring |Avenue|Suite 370|Midway|Kearny County|KS|61904|United States|-6|single family| +21850|AAAAAAAAKFFFAAAA|252|Woodland Forest||Suite K||Hampton County||||-5|| +21851|AAAAAAAALFFFAAAA|338|North |Blvd|Suite A|Sawyer|Chester County|TN|36045|United States|-5|condo| +21852|AAAAAAAAMFFFAAAA||Maple Ridge|Boulevard|Suite P|Clinton|Sierra County|CA|98222|United States||| +21853|AAAAAAAANFFFAAAA|183|Highland Locust|Boulevard|Suite 480|Newport|Brooke County|WV|21521|United States|-5|apartment| +21854|AAAAAAAAOFFFAAAA||13th |Way|||Josephine County|||||| +21855|AAAAAAAAPFFFAAAA|49|North 2nd|Parkway|Suite 380|Pine Grove|Cape May County|NJ|05193|United States|-5|condo| +21856|AAAAAAAAAGFFAAAA|294|Pine Adams|Lane|Suite B|Buena Vista|Montgomery County|AR|75752|United States|-6|apartment| +21857|AAAAAAAABGFFAAAA|76|5th Smith|Ln|Suite I|Kingston|Zavala County|TX|74975|United States|-6|single family| +21858|AAAAAAAACGFFAAAA|321|Smith Cedar|Court|Suite R|Morgantown|Troup County|GA|39193|United States|-5|apartment| +21859|AAAAAAAADGFFAAAA|828|Railroad |RD|Suite Q|Oakland|Worth County|MO|69843|United States|-6|condo| +21860|AAAAAAAAEGFFAAAA|826|14th |ST|Suite 470|Greenfield|East Baton Rouge Parish|LA|75038|United States|-6|condo| +21861|AAAAAAAAFGFFAAAA|880|Wilson River|Avenue|Suite Q|Antioch|Nassau County|NY|18605|United States|-5|single family| +21862|AAAAAAAAGGFFAAAA|257|Hickory |Ct.|Suite 260|Sutton|Runnels County|TX|75413|United States|-6|condo| +21863|AAAAAAAAHGFFAAAA|919|11th Highland|Boulevard|Suite K|Mount Vernon|Union County|PA|18482|United States|-5|condo| +21864|AAAAAAAAIGFFAAAA|238|Ash |ST|Suite 400|Waterloo|Madison County|KY|41675|United States|-5|single family| +21865|AAAAAAAAJGFFAAAA|408|Broadway Railroad|Road|Suite F|Hillcrest|Jefferson County|KS|63003|United States|-6|single family| +21866|AAAAAAAAKGFFAAAA|564|2nd 8th|Way|Suite 280|Crossroads|Archer County|TX|70534|United States|-6|single family| +21867|AAAAAAAALGFFAAAA|492|Central Sixth|Avenue|Suite R|Pleasant Grove|Live Oak County|TX|74136|United States|-6|apartment| +21868|AAAAAAAAMGFFAAAA|612|Hillcrest |Street|Suite 40|Clifton|Allen County|IN|48014|United States|-5|single family| +21869|AAAAAAAANGFFAAAA|459|Broadway West|Cir.|Suite 70|Summit|Guernsey County|OH|40499|United States|-5|condo| +21870|AAAAAAAAOGFFAAAA|456|View |Court|Suite 40|Macedonia|Colfax County|NM|81087|United States|-7|condo| +21871|AAAAAAAAPGFFAAAA|225|First |Cir.|Suite W|Lakeview|Canadian County|OK|78579|United States|-6|single family| +21872|AAAAAAAAAHFFAAAA|739|Park Park|Ave|Suite 380|Concord|Vermilion County|IL|64107|United States|-6|condo| +21873|AAAAAAAABHFFAAAA|417|Mill Central|Ct.|Suite 300|Woodville|Angelina County|TX|74289|United States|-6|apartment| +21874|AAAAAAAACHFFAAAA|459|Pine |Pkwy|Suite 80|Arlington|Levy County|FL|36557|United States|-5|condo| +21875|AAAAAAAADHFFAAAA|969|Center |Cir.|Suite W|Deerfield|Hickman County|TN|39840|United States|-5|single family| +21876|AAAAAAAAEHFFAAAA|60|West |Avenue|Suite O|Liberty|Nevada County|AR|73451|United States|-6|single family| +21877|AAAAAAAAFHFFAAAA|534|Park 7th|Ct.|Suite V|Forest Hills|Cheatham County|TN|39237|United States|-5|apartment| +21878|AAAAAAAAGHFFAAAA|136|1st River|Parkway|Suite 60|Glenwood|Tucker County|WV|23511|United States|-5|condo| +21879|AAAAAAAAHHFFAAAA|632|1st |Dr.|Suite L|Shady Grove|Lamar County|TX|72812|United States|-6|apartment| +21880|AAAAAAAAIHFFAAAA|454|Ash Washington|Ln|Suite W|Glenwood|Conway County|AR|73511|United States|-6|condo| +21881|AAAAAAAAJHFFAAAA|806|Willow Dogwood|ST|Suite Q|Spring Hill|Ransom County|ND|56787|United States|-6|single family| +21882|AAAAAAAAKHFFAAAA|361|11th Fourth|Avenue|Suite 420|Marion|Horry County|SC|20399|United States|-5|condo| +21883|AAAAAAAALHFFAAAA|96|4th West|Street|Suite 380|White Oak|Monterey County|CA|96668|United States|-8|single family| +21884|AAAAAAAAMHFFAAAA|773|4th Davis|Blvd|Suite 260|Kingston|Ashe County|NC|24975|United States|-5|condo| +21885|AAAAAAAANHFFAAAA|636|Lake Sycamore|Road|Suite W|Forest Hills|Greensville County|VA|29237|United States|-5|single family| +21886|AAAAAAAAOHFFAAAA|211|South |Way|Suite B|Forestville|Montgomery County|IA|53027|United States|-6|condo| +21887|AAAAAAAAPHFFAAAA|786|Green Jackson|Ave|Suite 230|Carthage|Pawnee County|KS|61529|United States|-6|single family| +21888|AAAAAAAAAIFFAAAA|236|Fourth |Ln|Suite 450|Walnut|Newport County|RI|06845|United States|-5|condo| +21889|AAAAAAAABIFFAAAA|263|4th Wilson|Dr.|Suite 320|Cedar|Union County|GA|31229|United States|-5|condo| +21890|AAAAAAAACIFFAAAA|144|Williams |Pkwy|Suite C|Spring Valley|Kiowa County|CO|86060|United States|-7|apartment| +21891|AAAAAAAADIFFAAAA|951|Broadway |Ln|Suite V|Woodville|Mifflin County|PA|14289|United States|-5|apartment| +21892|AAAAAAAAEIFFAAAA|979|Lakeview Lake|Avenue|Suite V|Deerfield|Fairfield County|OH|49840|United States|-5|single family| +21893|AAAAAAAAFIFFAAAA|212|Smith |RD|Suite 360|White Oak|Jasper County|TX|76668|United States|-6|single family| +21894|AAAAAAAAGIFFAAAA|942|View |Avenue|Suite J|Summit|Langlade County|WI|50499|United States|-6|single family| +21895|AAAAAAAAHIFFAAAA|344|Second |Ave|Suite 150|White Oak|Emery County|UT|86668|United States|-7|condo| +21896|AAAAAAAAIIFFAAAA|320|7th West|Ave|Suite F|Mountain View|Mower County|MN|54466|United States|-6|apartment| +21897|AAAAAAAAJIFFAAAA|708|River Spring|Circle|Suite 60|Bloomingdale|Warren County|MS|51824|United States|-6|single family| +21898|AAAAAAAAKIFFAAAA|312|Tenth |Road|Suite G|Clinton|Johnson County|IN|48222|United States|-5|single family| +21899|AAAAAAAALIFFAAAA|919|North |Street|Suite I|Fairview|Moultrie County|IL|65709|United States|-6|condo| +21900|AAAAAAAAMIFFAAAA|848|2nd |Ave|Suite K|Unionville|Emporia city|VA|21711|United States|-5|apartment| +21901|AAAAAAAANIFFAAAA|99|College Green|Drive|Suite B|Harmon|Dyer County|TN|35623|United States|-5|single family| +21902|AAAAAAAAOIFFAAAA|72|Park |Dr.|Suite 160|Hamilton|Kern County|CA|92808|United States|-8|apartment| +21903|AAAAAAAAPIFFAAAA|575|1st Walnut|Way|Suite A|Vista|McDonough County|IL|64694|United States|-6|single family| +21904|AAAAAAAAAJFFAAAA|233||Cir.||Deerfield|Camden County|NJ|09840|||| +21905|AAAAAAAABJFFAAAA|432|Elm 4th|ST|Suite F|Georgetown|Onondaga County|NY|17057|United States|-5|single family| +21906|AAAAAAAACJFFAAAA|935|Jefferson |Parkway|Suite 200|Bunker Hill|Coryell County|TX|70150|United States|-6|apartment| +21907|AAAAAAAADJFFAAAA|8|Elm Ash|Cir.|Suite Y|Youngstown|Jasper County|MO|60001|United States|-6|single family| +21908|AAAAAAAAEJFFAAAA|735|1st View|Ct.|Suite 440|Colfax|Newport News city|VA|22565|United States|-5|condo| +21909|AAAAAAAAFJFFAAAA|865|Main Meadow|Drive|Suite 190|Lincoln|Presque Isle County|MI|41289|United States|-5|single family| +21910|AAAAAAAAGJFFAAAA|694|Maple |Cir.|Suite W|Mountain View|Chambers County|TX|74466|United States|-6|single family| +21911|AAAAAAAAHJFFAAAA|162|Third |Circle|Suite 490|Centerville|Los Alamos County|NM|80059|United States|-7|apartment| +21912|AAAAAAAAIJFFAAAA|442|5th Hill|Pkwy|Suite 140|Harmony|Putnam County|NY|15804|United States|-5|apartment| +21913|AAAAAAAAJJFFAAAA|301|Pine Pine|Cir.|Suite F|Sulphur Springs|Bay County|FL|38354|United States|-5|single family| +21914|AAAAAAAAKJFFAAAA|796|10th Maple|Blvd|Suite Q|Wilson|Pitkin County|CO|86971|United States|-7|single family| +21915|AAAAAAAALJFFAAAA|168|View |Ln|Suite P|Green Acres|Finney County|KS|67683|United States|-6|condo| +21916|AAAAAAAAMJFFAAAA|69|Pine Park|Drive|Suite 280|Union|Ford County|KS|68721|United States|-6|apartment| +21917|AAAAAAAANJFFAAAA|319|Third Twelfth|Avenue||Summit|Trinity County||70499||-6|apartment| +21918|AAAAAAAAOJFFAAAA|782|Meadow |Road|Suite I|Avoca|Macon County|IL|60540|United States|-6|apartment| +21919|AAAAAAAAPJFFAAAA|215|3rd |Lane|Suite H|Oakdale|Powell County|KY|49584|United States|-5|condo| +21920|AAAAAAAAAKFFAAAA|43|||||Jackson County|KY||United States|-6|apartment| +21921|AAAAAAAABKFFAAAA|78|6th |Parkway|Suite U|Mount Pleasant|De Kalb County|IN|41933|United States|-5|single family| +21922|AAAAAAAACKFFAAAA|909|Maple Park|Boulevard|Suite C|Harmony|Charles Mix County|SD|55804|United States|-6|single family| +21923|AAAAAAAADKFFAAAA|2|College Fourth|Pkwy|Suite 80|Harmony|Menifee County|KY|45804|United States|-5|condo| +21924|AAAAAAAAEKFFAAAA|723|Main |RD|Suite 490|Glendale|Burnett County|WI|53951|United States|-6|apartment| +21925|AAAAAAAAFKFFAAAA|53|Main Sixth|Drive|Suite U|Hazelwood|Terry County|TX|71206|United States|-6|condo| +21926|AAAAAAAAGKFFAAAA|118|Elm Johnson|Circle|Suite 410||Sarpy County|NE||United States||condo| +21927|AAAAAAAAHKFFAAAA|435|Smith Pine|Ct.|Suite L|Cedar Grove|Saratoga County|NY|10411|United States|-5|apartment| +21928|AAAAAAAAIKFFAAAA|377|2nd 14th|Blvd|Suite 190|Glenville|Wicomico County|MD|23445|United States|-5|apartment| +21929|AAAAAAAAJKFFAAAA|929|Hill Smith|Dr.|Suite 490|Springfield|Colbert County|AL|39303|United States|-6|single family| +21930|AAAAAAAAKKFFAAAA|502|Cherry Hill|Ln|Suite Y|Friendship|Brooke County|WV|24536|United States|-5|single family| +21931|AAAAAAAALKFFAAAA|911|10th |Road|Suite 400|Woodville|Buena Vista city|VA|24289|United States|-5|single family| +21932|AAAAAAAAMKFFAAAA|548|Broadway Williams|Boulevard|Suite 200|Wilson|Queen Anne County|MD|26971|United States|-5|apartment| +21933|AAAAAAAANKFFAAAA|876|Lake |Way|Suite H|Stringtown|Hancock County|TN|30162|United States|-5|condo| +21934|AAAAAAAAOKFFAAAA|973|View Cherry|Way|Suite M|Georgetown|Madison County|TN|37057|United States|-6|apartment| +21935|AAAAAAAAPKFFAAAA|540|Third |Way|Suite 480|Mount Pleasant|Oconee County|SC|21933|United States|-5|condo| +21936|AAAAAAAAALFFAAAA|785|Dogwood |Road|Suite B|Spring Hill|Butler County|KS|66787|United States|-6|apartment| +21937|AAAAAAAABLFFAAAA|264|Hickory Wilson|Street|Suite U|Kingston|Marshall County|MS|54975|United States|-6|apartment| +21938|AAAAAAAACLFFAAAA|949|Pine Pine|Boulevard|Suite 330|Bridgeport|Lancaster County|PA|15817|United States|-5|condo| +21939|AAAAAAAADLFFAAAA||Seventh Main|ST|Suite 280|Farmington||VA|||-5|condo| +21940|AAAAAAAAELFFAAAA|333|Ash Cherry|Ln|Suite K|Glendale|Harney County|OR|93951|United States|-8|condo| +21941|AAAAAAAAFLFFAAAA|713|Wilson Woodland|Avenue|Suite Q|Yorktown|Brooke County|WV|20732|United States|-5|condo| +21942|AAAAAAAAGLFFAAAA|804|Maple Third|Road|Suite Y|Red Hill|East Baton Rouge Parish|LA|74338|United States|-6|single family| +21943|AAAAAAAAHLFFAAAA|79|Meadow |Pkwy|Suite E|Woodland|Wright County|MN|54854|United States|-6|single family| +21944|AAAAAAAAILFFAAAA|919|Cedar |Way|Suite 10|Hamilton|Jackson County|SD|52808|United States|-7|apartment| +21945|AAAAAAAAJLFFAAAA|436|Maple |ST|Suite 230|Shady Grove|Lincoln Parish|LA|72812|United States|-6|single family| +21946|AAAAAAAAKLFFAAAA|318|Willow |Blvd|Suite 430|Walnut Grove|Sabine County|TX|77752|United States|-6|condo| +21947|AAAAAAAALLFFAAAA|875|Ninth |Lane|Suite L|Oak Ridge|Grant County|SD|58371|United States|-6|apartment| +21948|AAAAAAAAMLFFAAAA|816|Central Ninth|Court|Suite G|Hamilton|Amherst County|VA|22808|United States|-5|single family| +21949|AAAAAAAANLFFAAAA|114|Maple College|Circle|Suite P|Crossroads|Itawamba County|MS|50534|United States|-6|condo| +21950|AAAAAAAAOLFFAAAA|327|Oak Railroad|Parkway|Suite 310|Hopewell|Koochiching County|MN|50587|United States|-6|condo| +21951|AAAAAAAAPLFFAAAA|224|Hillcrest |RD|Suite Y|Providence|Tarrant County|TX|76614|United States|-6|single family| +21952|AAAAAAAAAMFFAAAA|48|Main |Dr.|Suite 280|Riverdale|Curry County|NM|89391|United States|-7|single family| +21953|AAAAAAAABMFFAAAA|50|Seventh |Circle|Suite 480|Cedar Grove|Chester County|TN|30411|United States|-5|condo| +21954|AAAAAAAACMFFAAAA|272|3rd Lincoln|Pkwy|Suite 350|Highland Park|Langlade County|WI|56534|United States|-6|condo| +21955|AAAAAAAADMFFAAAA|413|3rd |Boulevard|Suite P|Edgewood|Charlottesville city|VA|20069|United States|-5|single family| +21956|AAAAAAAAEMFFAAAA|525|Valley Maple|Court|Suite 10|Jackson|Ionia County|MI|49583|United States|-5|single family| +21957|AAAAAAAAFMFFAAAA|907|Jackson |Dr.|Suite T|Fairfield|Hart County|KY|46192|United States|-6|single family| +21958|AAAAAAAAGMFFAAAA|364|6th |Ln|Suite O|Centerville|Harding County|NM|80059|United States|-7|single family| +21959|AAAAAAAAHMFFAAAA|670|Ash |ST|Suite W|Edgewood|Lancaster County|NE|60069|United States|-7|apartment| +21960|AAAAAAAAIMFFAAAA|811|Green Maple|Circle|Suite C|Oak Ridge|DuPage County|IL|68371|United States|-6|apartment| +21961|AAAAAAAAJMFFAAAA|831|Railroad |Road|Suite 60|Shiloh|Webster County|IA|59275|United States|-6|condo| +21962|AAAAAAAAKMFFAAAA|733|North West|Road|Suite W|Lebanon|Woodford County|IL|62898|United States|-6|single family| +21963|AAAAAAAALMFFAAAA|200|Broadway |Avenue|Suite C|Franklin|Crawford County|OH|49101|United States|-5|single family| +21964|AAAAAAAAMMFFAAAA|392|Dogwood |Ln|Suite 90|Five Forks|Pulaski County|KY|42293|United States|-5|condo| +21965|AAAAAAAANMFFAAAA|455|Oak |RD|Suite S|Summit|Alleghany County|NC|20499|United States|-5|single family| +21966|AAAAAAAAOMFFAAAA|67|||Suite 20|Fairfield||NC|||-5|apartment| +21967|AAAAAAAAPMFFAAAA|421|Laurel South|Wy|Suite 430|Mount Pleasant|Bryan County|OK|71933|United States|-6|single family| +21968|AAAAAAAAANFFAAAA|231|Second |Blvd|Suite D|Shady Grove|Clark County|WA|92812|United States|-8|single family| +21969|AAAAAAAABNFFAAAA|320|Main |Circle|Suite G|Belmont|Platte County|NE|60191|United States|-7|condo| +21970|AAAAAAAACNFFAAAA|290|Mill |Boulevard|Suite W|Caledonia|Churchill County|NV|87411|United States|-8|condo| +21971|AAAAAAAADNFFAAAA|552|14th |Ave|Suite C|Harmony|Harrison County|KY|45804|United States|-6|apartment| +21972|AAAAAAAAENFFAAAA|943|1st |Wy|Suite B|White Hall|Marion County|SC|26955|United States|-5|condo| +21973|AAAAAAAAFNFFAAAA|688|Central |Wy|Suite V|Lebanon|Miller County|MO|62898|United States|-6|single family| +21974|AAAAAAAAGNFFAAAA|281||Pkwy||Riverside|Foard County|TX||United States|-6|condo| +21975|AAAAAAAAHNFFAAAA|324|9th 8th|Ct.|Suite 80|Greenfield|White Pine County|NV|85038|United States|-8|condo| +21976|AAAAAAAAINFFAAAA|247|Lake First|Road|Suite 150|Fairfield|Dallas County|AL|36192|United States|-6|condo| +21977|AAAAAAAAJNFFAAAA|810|View Green|Avenue|Suite B|Woodland|Saline County|KS|64854|United States|-6|condo| +21978|AAAAAAAAKNFFAAAA|788|Walnut Laurel|Avenue|Suite 450|Mount Pleasant|Alfalfa County|OK|71933|United States|-6|apartment| +21979|AAAAAAAALNFFAAAA|54|6th |Lane|Suite Y|Fairview|Dade County|MO|65709|United States|-6|condo| +21980|AAAAAAAAMNFFAAAA|714|Park 14th|Avenue|Suite O|Mountain View|McDonough County|IL|64466|United States|-6|single family| +21981|AAAAAAAANNFFAAAA|401|Sunset |Ct.|Suite 220|Oak Grove|Delaware County|PA|18370|United States|-5|apartment| +21982|AAAAAAAAONFFAAAA|524|Fourteenth |Ave|Suite E|Collinsville|Pulaski County|IN|42459|United States|-5|single family| +21983|AAAAAAAAPNFFAAAA|245|North |Parkway|Suite 110|Fairfield|Henry County|VA|26192|United States|-5|apartment| +21984|AAAAAAAAAOFFAAAA|19|2nd |Lane|Suite D|Derby|Jefferson County|TX|77702|United States|-6|apartment| +21985|AAAAAAAABOFFAAAA|342|Hillcrest |Lane|Suite H|Oakwood|Sagadahoc County|ME|00769|United States|-5|apartment| +21986|AAAAAAAACOFFAAAA|708|Forest |Street|Suite U|Newtown|Jackson County|AL|31749|United States|-6|single family| +21987|AAAAAAAADOFFAAAA|91|Williams Adams|Avenue|Suite 250|Green Acres|Hamilton County|IL|67683|United States|-6|single family| +21988|AAAAAAAAEOFFAAAA|221|4th |Road|Suite 70|Antioch|Monroe County|MS|58605|United States|-6|condo| +21989|AAAAAAAAFOFFAAAA|321|Ash |RD|Suite 460|Oakdale|White Pine County|NV|89584|United States|-8|apartment| +21990|AAAAAAAAGOFFAAAA|815|6th |Ln|Suite 440|Springdale|Orangeburg County|SC|28883|United States|-5|single family| +21991|AAAAAAAAHOFFAAAA|890|Poplar 14th|Ln|Suite D|Hillcrest|Iowa County|WI|53003|United States|-6|condo| +21992|AAAAAAAAIOFFAAAA|156|Ninth 10th|Drive|Suite X|Liberty|Warrick County|IN|43451|United States|-5|condo| +21993|AAAAAAAAJOFFAAAA|196|3rd 1st|ST|Suite C|Quincy|Page County|VA|23868|United States|-5|apartment| +21994|AAAAAAAAKOFFAAAA|845|4th |RD|Suite R|Glenwood|Story County|IA|53511|United States|-6|apartment| +21995|AAAAAAAALOFFAAAA|618|View |Street|Suite 470|Brownsville|Dixon County|NE|69310|United States|-6|condo| +21996|AAAAAAAAMOFFAAAA|43|5th Third|Avenue|Suite 330|Florence|Knox County|TN|33394|United States|-6|single family| +21997|AAAAAAAANOFFAAAA|934|Davis |Court|Suite E|Woodlawn|Petroleum County|MT|64098|United States|-7|apartment| +21998|AAAAAAAAOOFFAAAA|754|5th |Road|Suite 20|Waterloo|Clinton County|MI|41675|United States|-5|apartment| +21999|AAAAAAAAPOFFAAAA|107|Cedar |Ave|Suite F|Hillcrest|Banks County|GA|33003|United States|-5|apartment| +22000|AAAAAAAAAPFFAAAA|830|Meadow Ridge|Way|Suite 460|Stringtown|Covington County|MS|50162|United States|-6|single family| +22001|AAAAAAAABPFFAAAA|856|Walnut Main|Ln|Suite 350|Green Acres|Cumberland County|IL|67683|United States|-6|apartment| +22002|AAAAAAAACPFFAAAA|169|Park Adams|ST|Suite M|Shore Acres|Sebastian County|AR|72724|United States|-6|condo| +22003|AAAAAAAADPFFAAAA|83|15th Railroad|Street|Suite 440|Bethel|Ramsey County|ND|55281|United States|-6|apartment| +22004|AAAAAAAAEPFFAAAA|708|15th Broadway|Avenue|Suite V|Lakeview|Hunterdon County|NJ|09179|United States|-5|condo| +22005|AAAAAAAAFPFFAAAA|22|Ash Main|Cir.|Suite 60|Springdale|Pendleton County|WV|28883|United States|-5|single family| +22006|AAAAAAAAGPFFAAAA||Willow 2nd|||Highland Park|Marshall County||||-6|| +22007|AAAAAAAAHPFFAAAA|664|Eigth |Street|Suite 40|Five Forks|Geary County|KS|62293|United States|-6|apartment| +22008|AAAAAAAAIPFFAAAA|533|South |Road|Suite 0|Maple Grove|Columbiana County|OH|48252|United States|-5|single family| +22009|AAAAAAAAJPFFAAAA|507|View |Lane|Suite 370|Five Forks|Jackson County|FL|32293|United States|-5|condo| +22010|AAAAAAAAKPFFAAAA|767|Wilson |Circle|Suite 270|Mount Vernon|Jackson County|WV|28482|United States|-5|condo| +22011|AAAAAAAALPFFAAAA|526|Fourth |Ct.|Suite F|Sulphur Springs|Obion County|TN|38354|United States|-6|condo| +22012|AAAAAAAAMPFFAAAA|554|15th |Wy|Suite 150|Antioch|Lynchburg city|VA|28605|United States|-5|condo| +22013|AAAAAAAANPFFAAAA|945|Chestnut |Drive|Suite V|Ashland|Mineral County|CO|84244|United States|-7|condo| +22014|AAAAAAAAOPFFAAAA|579|Spring |Avenue|Suite Y|White Oak|Massac County|IL|66668|United States|-6|single family| +22015|AAAAAAAAPPFFAAAA|102|Central |Street|Suite 40|Oak Hill|Pawnee County|NE|67838|United States|-7|condo| +22016|AAAAAAAAAAGFAAAA|703|2nd Franklin|Circle|Suite 70|Oakwood|Robeson County|NC|20169|United States|-5|condo| +22017|AAAAAAAABAGFAAAA|395|4th 1st|Parkway|Suite 90|Buena Vista|Pottawatomie County|KS|65752|United States|-6|apartment| +22018|AAAAAAAACAGFAAAA|819|Forest |Dr.|Suite L|Unionville|Gates County|NC|21711|United States|-5|condo| +22019|AAAAAAAADAGFAAAA|51|Cedar Park|Blvd|Suite W|Hopewell|Monroe County|IN|40587|United States|-5|condo| +22020|AAAAAAAAEAGFAAAA|583|Poplar Second|Dr.|Suite 440|Riverview|Franklin County|AL|39003|United States|-6|apartment| +22021|AAAAAAAAFAGFAAAA|379|Laurel Ash|Court|Suite 80|Valley View|Laurel County|KY|45124|United States|-5|apartment| +22022|AAAAAAAAGAGFAAAA|869|Williams |Ave|Suite I|Clinton|Riley County|KS|68222|United States|-6|condo| +22023|AAAAAAAAHAGFAAAA||||Suite N|||||United States||apartment| +22024|AAAAAAAAIAGFAAAA|869|11th Seventh|Boulevard|Suite S|Florence|Prince George County|VA|23394|United States|-5|single family| +22025|AAAAAAAAJAGFAAAA|740|6th |Road|Suite 260|Arcola|Clatsop County|OR|91654|United States|-8|apartment| +22026|AAAAAAAAKAGFAAAA|542|Fourth |RD|Suite Y|Walnut Grove|Grant County|NE|67752|United States|-6|apartment| +22027|AAAAAAAALAGFAAAA|912|Hill Park|Drive|Suite V|Oakdale|Gratiot County|MI|49584|United States|-5|apartment| +22028|AAAAAAAAMAGFAAAA|428|Willow 5th|Boulevard|Suite W|Lebanon|Union County|MS|52898|United States|-6|condo| +22029|AAAAAAAANAGFAAAA|233||Blvd|Suite 340|Globe|Baltimore city|||||| +22030|AAAAAAAAOAGFAAAA|44|Franklin Hill|Ct.|Suite 200|Greenwood|Wasatch County|UT|88828|United States|-7|apartment| +22031|AAAAAAAAPAGFAAAA|727|South Miller|Street|Suite C|Pleasant Hill|Calhoun County|MI|43604|United States|-5|apartment| +22032|AAAAAAAAABGFAAAA|333|Highland 7th|Avenue|Suite 230|Greenfield|Johnson County|AR|75038|United States|-6|condo| +22033|AAAAAAAABBGFAAAA|155|East Third|ST|Suite 100|Pinhook|Baldwin County|GA|39398|United States|-5|apartment| +22034|AAAAAAAACBGFAAAA|56|Seventh Tenth|RD|Suite 90|Cedar Grove|Muskegon County|MI|40411|United States|-5|condo| +22035|AAAAAAAADBGFAAAA|276|15th Cedar|ST|Suite 0|Greenwood|Pope County|AR|78828|United States|-6|single family| +22036|AAAAAAAAEBGFAAAA|911|Tenth |Dr.|Suite 460|Sulphur Springs|Real County|TX|78354|United States|-6|single family| +22037|AAAAAAAAFBGFAAAA|867|Highland 15th|Parkway|Suite N|Spring Valley|Elkhart County|IN|46060|United States|-5|apartment| +22038|AAAAAAAAGBGFAAAA|282|Willow Maple|Ln|Suite 440|Stringtown|Hopkins County|TX|70162|United States|-6|apartment| +22039|AAAAAAAAHBGFAAAA|741|Jackson 7th|Pkwy|Suite S|Oakland|Brown County|TX|79843|United States|-6|condo| +22040|AAAAAAAAIBGFAAAA|561|Lakeview Locust|Boulevard|Suite F|Greenwood|Pickens County|SC|28828|United States|-5|condo| +22041|AAAAAAAAJBGFAAAA|550|Third Railroad|Ct.|Suite P|Midway|Yellow Medicine County|MN|51904|United States|-6|single family| +22042|AAAAAAAAKBGFAAAA|234|Cedar Park|Drive|Suite L|Bethel|Casey County|KY|45281|United States|-6|condo| +22043|AAAAAAAALBGFAAAA|792|Miller |Wy|Suite 400|Riverdale|Chickasaw County|MS|59391|United States|-6|condo| +22044|AAAAAAAAMBGFAAAA|58|Cedar Hillcrest|Way|Suite 190|Glendale|Bacon County|GA|33951|United States|-5|single family| +22045|AAAAAAAANBGFAAAA|301|Adams |ST|Suite O|Unionville|Cannon County|TN|31711|United States|-5|single family| +22046|AAAAAAAAOBGFAAAA|434|North |Road|Suite 0|Harmony|Payette County|ID|85804|United States|-7|condo| +22047|AAAAAAAAPBGFAAAA|291|College |Circle|Suite B|Globe|Chesterfield County|SC|29614|United States|-5|apartment| +22048|AAAAAAAAACGFAAAA|980|Main |RD|Suite 390|White Oak|Champaign County|IL|66668|United States|-6|apartment| +22049|AAAAAAAABCGFAAAA|721|Hillcrest |Cir.|Suite X|Greenfield|Nolan County|TX|75038|United States|-6|single family| +22050|AAAAAAAACCGFAAAA|536|Forest |Ave|Suite 460|Lone Pine|Garfield County|OK|77441|United States|-6|apartment| +22051|AAAAAAAADCGFAAAA|700|Church College|Cir.|Suite 260|Bridgeport|Shoshone County|ID|85817|United States|-7|single family| +22052|AAAAAAAAECGFAAAA|942|Fourth |Boulevard|Suite E|Woodville|Ballard County|KY|44289|United States|-6|apartment| +22053|AAAAAAAAFCGFAAAA|376|2nd |Dr.|Suite 190|Hamilton|Newton County|MO|62808|United States|-6|apartment| +22054|AAAAAAAAGCGFAAAA|702|Hill View|Wy|Suite J|Macedonia|Dunklin County|MO|61087|United States|-6|apartment| +22055|AAAAAAAAHCGFAAAA|567|Willow |Court|Suite X|Bethel|Halifax County|NC|25281|United States|-5|condo| +22056|AAAAAAAAICGFAAAA|616|Williams Miller|Way|Suite 160|Farmington|Walton County|GA|39145|United States|-5|condo| +22057|AAAAAAAAJCGFAAAA|818|Second |Way|Suite 10|Mount Pleasant|Lyon County|IA|51933|United States|-6|single family| +22058|AAAAAAAAKCGFAAAA|887|Central Lincoln|Circle|Suite 430|Five Points|Clay County|NE|66098|United States|-6|single family| +22059|AAAAAAAALCGFAAAA|325|11th 4th|Way|Suite 320|Ashland|Wayne County|WV|24244|United States|-5|apartment| +22060|AAAAAAAAMCGFAAAA|978|Madison |Boulevard|Suite P|Friendship|Benton County|WA|94536|United States|-8|single family| +22061|AAAAAAAANCGFAAAA|155|Main |Way|Suite Y|Georgetown|Manistee County|MI|47057|United States|-5|single family| +22062|AAAAAAAAOCGFAAAA|690|Fifth |Lane|Suite 100|Tanglewood|Caldwell County|TX|78994|United States|-6|apartment| +22063|AAAAAAAAPCGFAAAA|701|Oak |Road|Suite 360|Kingston|Estill County|KY|44975|United States|-6|single family| +22064|AAAAAAAAADGFAAAA|146|Poplar |Dr.|Suite 300|Shady Grove|Lonoke County|AR|72812|United States|-6|apartment| +22065|AAAAAAAABDGFAAAA|386|Meadow |Street|Suite 260|Franklin|Loudon County|TN|39101|United States|-6|condo| +22066|AAAAAAAACDGFAAAA|160|Oak View|Circle|Suite 210|Saratoga|Pulaski County|GA|32123|United States|-5|single family| +22067|AAAAAAAADDGFAAAA|608|Mill Spring|Wy|Suite 360|Franklin|Bennett County|SD|59101|United States|-6|condo| +22068|AAAAAAAAEDGFAAAA|123|5th Spring|Ave|Suite M|Highland Park|Meeker County|MN|56534|United States|-6|single family| +22069|AAAAAAAAFDGFAAAA|579|College |Parkway|Suite 480|Fairview|Orange County|CA|95709|United States|-8|single family| +22070|AAAAAAAAGDGFAAAA|275|West |Parkway|Suite 80|Riverview|Jerauld County|SD|59003|United States|-7|condo| +22071|AAAAAAAAHDGFAAAA|613|Pine |Circle|Suite 410|Little River|Lincoln County|NM|80319|United States|-7|single family| +22072|AAAAAAAAIDGFAAAA|773|Birch |Parkway|Suite Y|Macedonia|Erie County|PA|11087|United States|-5|single family| +22073|AAAAAAAAJDGFAAAA|275|South Birch|Lane|Suite 300|Macedonia|Switzerland County|IN|41087|United States|-5|condo| +22074|AAAAAAAAKDGFAAAA|417|Elm |Wy|Suite Q|Shiloh|Mercer County|NJ|09875|United States|-5|condo| +22075|AAAAAAAALDGFAAAA|289|3rd Cherry|Wy|Suite 100|Johnsonville|Cheyenne County|NE|67745|United States|-6|single family| +22076|AAAAAAAAMDGFAAAA|44|Ash 3rd|Ln|Suite 440|Arlington|Harlan County|NE|66557|United States|-6|condo| +22077|AAAAAAAANDGFAAAA|59|Williams |RD|Suite J|Oak Ridge|Turner County|GA|38371|United States|-5|condo| +22078|AAAAAAAAODGFAAAA|783|Washington |Ct.|Suite V|Mount Olive|Pocahontas County|WV|28059|United States|-5|single family| +22079|AAAAAAAAPDGFAAAA|780|Meadow Woodland|Pkwy|Suite T|Midway|Clay County|NE|61904|United States|-6|single family| +22080|AAAAAAAAAEGFAAAA||Oak |||Lebanon||||||condo| +22081|AAAAAAAABEGFAAAA|679|Cedar |Boulevard|Suite P|Hazelwood|Nacogdoches County|TX|71206|United States|-6|condo| +22082|AAAAAAAACEGFAAAA|487|Franklin Cedar|RD|Suite 490|Deerfield|Vilas County|WI|59840|United States|-6|apartment| +22083|AAAAAAAADEGFAAAA|971|Miller |RD|Suite 490|Woodlawn|Ness County|KS|64098|United States|-6|apartment| +22084|AAAAAAAAEEGFAAAA|40|Park |Way|Suite 150|Greenville|Lincoln County|OR|91387|United States|-8|condo| +22085|AAAAAAAAFEGFAAAA|709|Maple |Lane|Suite L|Highland Park|Chester County|SC|26534|United States|-5|single family| +22086|AAAAAAAAGEGFAAAA|108|Cherry |Blvd|Suite 320|Woodlawn|Lincoln County|NE|64098|United States|-7|apartment| +22087|AAAAAAAAHEGFAAAA|331|Fourth East|Lane|Suite H|Shiloh|Lafayette County|FL|39275|United States|-5|single family| +22088|AAAAAAAAIEGFAAAA|953|Center Smith|Road|Suite 270|Crossroads|Lincoln County|ME|01134|United States|-5|single family| +22089|AAAAAAAAJEGFAAAA|790|Pine |Lane|Suite M|Woodlawn|DeSoto County|FL|34098|United States|-5|apartment| +22090|AAAAAAAAKEGFAAAA|307|Jackson Center|Lane|Suite B|Elba|Crawford County|IL|60262|United States|-6|condo| +22091|AAAAAAAALEGFAAAA|208|Main |Cir.|Suite 90|Mount Zion|Jones County|SD|58054|United States|-7|single family| +22092|AAAAAAAAMEGFAAAA|726|6th |Boulevard|Suite D|Woodlawn|Rawlins County|KS|64098|United States|-6|condo| +22093|AAAAAAAANEGFAAAA|999|Walnut Seventh|ST|Suite K|Shiloh|Ford County|KS|69275|United States|-6|apartment| +22094|AAAAAAAAOEGFAAAA|139|Elm |Way|Suite 260|Providence|Tulsa County|OK|76614|United States|-6|apartment| +22095|AAAAAAAAPEGFAAAA|258|Cedar Dogwood|Ave|Suite 70|Farmersville|Davidson County|TN|39305|United States|-5|single family| +22096|AAAAAAAAAFGFAAAA|505|Cedar Dogwood|Cir.|Suite 70|Spring Valley|Taylor County|IA|56060|United States|-6|condo| +22097|AAAAAAAABFGFAAAA|719|Dogwood |Avenue|Suite 460|Weldon|Schuylkill County|PA|16277|United States|-5|condo| +22098|AAAAAAAACFGFAAAA|138|Mill Third|Lane|Suite 220|Colfax|Brown County|OH|42565|United States|-5|condo| +22099|AAAAAAAADFGFAAAA|364|Hickory |Road|Suite 130|Arlington|Walton County|GA|36557|United States|-5|single family| +22100|AAAAAAAAEFGFAAAA|655|Hill |Wy|Suite M|Highland Park|San Juan County|WA|96534|United States|-8|condo| +22101|AAAAAAAAFFGFAAAA|811|Main |Ave|Suite V|Arlington|Northumberland County|VA|26557|United States|-5|condo| +22102|AAAAAAAAGFGFAAAA|509|Park Lincoln|Road|Suite 90|Lone Pine|Macon County|MO|67441|United States|-6|apartment| +22103|AAAAAAAAHFGFAAAA|975|Green Park|Wy|Suite 490|Enterprise|Kimball County|NE|61757|United States|-7|single family| +22104|AAAAAAAAIFGFAAAA||Maple 8th|Lane|||Wexford County||48059|United States||condo| +22105|AAAAAAAAJFGFAAAA|542|9th |Street|Suite M|Pleasant Grove|Trimble County|KY|44136|United States|-5|apartment| +22106|AAAAAAAAKFGFAAAA|374|Madison 12th|RD|Suite 20|Arlington|Oconee County|SC|26557|United States|-5|condo| +22107|AAAAAAAALFGFAAAA|834|Jackson 5th|Boulevard|Suite L|Oakwood|Crawford County|IA|50169|United States|-6|apartment| +22108|AAAAAAAAMFGFAAAA|567|Sunset Center|Cir.|Suite E|Lakeview|Orange County|IN|48579|United States|-5|apartment| +22109|AAAAAAAANFGFAAAA|274|6th View|Circle|Suite W|Kingston|Hamilton County|IA|54975|United States|-6|single family| +22110|AAAAAAAAOFGFAAAA|555|Ridge |Road|Suite 490|Woodville|Gilpin County|CO|84289|United States|-7|condo| +22111|AAAAAAAAPFGFAAAA|387|Mill |Avenue|Suite L|Spring Hill|Tompkins County|NY|16787|United States|-5|apartment| +22112|AAAAAAAAAGGFAAAA|588|7th |Court|Suite 30|Pleasant Hill|Johnston County|OK|73604|United States|-6|single family| +22113|AAAAAAAABGGFAAAA|306|Railroad Walnut|Lane|Suite 80|Cedar Grove|Licking County|OH|40411|United States|-5|condo| +22114|AAAAAAAACGGFAAAA|767|Oak |Pkwy|Suite W|Highland Park|Posey County|IN|46534|United States|-5|apartment| +22115|AAAAAAAADGGFAAAA|263|Jefferson Sunset|Circle|Suite K|Springdale|Wayne County|MI|48883|United States|-5|single family| +22116|AAAAAAAAEGGFAAAA|173|5th Johnson|Way|Suite I|Ashland|Kewaunee County|WI|54244|United States|-6|condo| +22117|AAAAAAAAFGGFAAAA|627|Maple 12th|Street|Suite 150|Oakwood|Brooks County|GA|30169|United States|-5|condo| +22118|AAAAAAAAGGGFAAAA|132|Mill View|Ave|Suite 160|Kirkland|Kimble County|TX|77896|United States|-6|apartment| +22119|AAAAAAAAHGGFAAAA|760|Miller |Street|Suite 50|Jamestown|Peoria County|IL|66867|United States|-6|apartment| +22120|AAAAAAAAIGGFAAAA|588|9th 5th|Circle|Suite B|Jamestown|Boone County|IN|46867|United States|-5|single family| +22121|AAAAAAAAJGGFAAAA|107|River Oak|Circle|Suite I|Clifton|McDowell County|NC|28014|United States|-5|single family| +22122|AAAAAAAAKGGFAAAA|805|Lee |Boulevard|Suite 120|Valley View|Jefferson County|WI|55124|United States|-6|apartment| +22123|AAAAAAAALGGFAAAA|673|Walnut Lee|Parkway|Suite 380|Forest Hills|Prince George County|VA|29237|United States|-5|apartment| +22124|AAAAAAAAMGGFAAAA|242|Pine Lincoln|Ln|Suite L|Salem|DuPage County|IL|68048|United States|-6|apartment| +22125|AAAAAAAANGGFAAAA|220|Church |Lane|Suite N|Pinhook|Redwood County|MN|59398|United States|-6|condo| +22126|AAAAAAAAOGGFAAAA|287|Sycamore 2nd|Street|Suite 70|Newtown|Charlottesville city|VA|21749|United States|-5|single family| +22127|AAAAAAAAPGGFAAAA|31|Adams Woodland|Street|Suite W|Edgewood|Jones County|TX|70069|United States|-6|apartment| +22128|AAAAAAAAAHGFAAAA|854|11th |Court|Suite K|Hopewell|Box Elder County|UT|80587|United States|-7|apartment| +22129|AAAAAAAABHGFAAAA|18|Forest |Drive|Suite 430|Marion|Webster County|NE|60399|United States|-7|single family| +22130|AAAAAAAACHGFAAAA|815|Cherry Highland|Ct.|Suite 80|Bunker Hill|Cheboygan County|MI|40150|United States|-5|condo| +22131|AAAAAAAADHGFAAAA|884|Madison |ST|Suite 360|Shady Grove|Dunn County|WI|52812|United States|-6|single family| +22132|AAAAAAAAEHGFAAAA|462|Fourth Laurel|Avenue|Suite G|New Hope|Okeechobee County|FL|39431|United States|-5|single family| +22133|AAAAAAAAFHGFAAAA|235|South |Parkway|Suite A|Spring Hill|Franklin County|NE|66787|United States|-6|condo| +22134|AAAAAAAAGHGFAAAA|269|Walnut |Lane|Suite R|Avoca|Hillsdale County|MI|40540|United States|-5|single family| +22135|AAAAAAAAHHGFAAAA|423|Walnut Pine|Pkwy|Suite M|River Oaks|Liberty County|GA|38075|United States|-5|condo| +22136|AAAAAAAAIHGFAAAA|243|Oak Ash|Ave|Suite 350|Mount Zion|Waynesboro city|VA|28054|United States|-5|condo| +22137|AAAAAAAAJHGFAAAA|25|Green Elm|Dr.|Suite E|White Oak|Burt County|NE|66668|United States|-6|single family| +22138|AAAAAAAAKHGFAAAA|566|Eigth |ST|Suite 70|Hillcrest|Big Stone County|MN|53003|United States|-6|single family| +22139|AAAAAAAALHGFAAAA|87|4th Center|Court|Suite J|Harmony|Polk County|IA|55804|United States|-6|apartment| +22140|AAAAAAAAMHGFAAAA|141|Cherry 5th|Court|Suite G|Woodville|Henry County|IL|64289|United States|-6|apartment| +22141|AAAAAAAANHGFAAAA|527|11th 1st|Cir.|Suite 40|Jamestown|Ohio County|WV|26867|United States|-5|apartment| +22142|AAAAAAAAOHGFAAAA||||Suite 340|Newport||MS||United States|-6|| +22143|AAAAAAAAPHGFAAAA|256|Second Maple|Blvd|Suite 10|Shady Grove|Ogle County|IL|62812|United States|-6|condo| +22144|AAAAAAAAAIGFAAAA|246|Hill 11th|Ct.|Suite 450|Mountain View|Lagrange County|IN|44466|United States|-5|single family| +22145|AAAAAAAABIGFAAAA|348|East Church|Ct.|Suite G|Woodland|Peach County|GA|34854|United States|-5|condo| +22146|AAAAAAAACIGFAAAA|511|Willow |Blvd|Suite 390|Hopewell|Franklin County|NY|10587|United States|-5|single family| +22147|AAAAAAAADIGFAAAA|727|West |Dr.|Suite 10|Liberty|Crane County|TX|73451|United States|-6|single family| +22148|AAAAAAAAEIGFAAAA|342|Franklin |ST|Suite K|Owens|Franklin County|AL|32324|United States|-6|apartment| +22149|AAAAAAAAFIGFAAAA|472|3rd Lincoln|Ct.|Suite 40|Riverdale|Alexandria city|VA|29391|United States|-5|single family| +22150|AAAAAAAAGIGFAAAA|28|14th |Pkwy|Suite 320|Concord|Philadelphia County|PA|14107|United States|-5|condo| +22151|AAAAAAAAHIGFAAAA|941|East |Blvd|Suite 420|Springdale|Lincoln County|WA|98883|United States|-8|single family| +22152|AAAAAAAAIIGFAAAA|||||Woodland||AR|74854||-6|condo| +22153|AAAAAAAAJIGFAAAA|830|Lincoln Lakeview|Drive|Suite T|Marion|Carlisle County|KY|40399|United States|-6|apartment| +22154|AAAAAAAAKIGFAAAA|970|2nd Pine|Blvd|Suite 60|Centerville|Kidder County|ND|50059|United States|-6|apartment| +22155|AAAAAAAALIGFAAAA|860|Madison |Boulevard|Suite 310|Lakeview|Dimmit County|TX|78579|United States|-6|apartment| +22156|AAAAAAAAMIGFAAAA|408|Davis Meadow|RD|Suite 350|Deerfield|Henrico County|VA|29840|United States|-5|single family| +22157|AAAAAAAANIGFAAAA|934|12th |Cir.|Suite 390|Rockwood|Carteret County|NC|21545|United States|-5|single family| +22158|AAAAAAAAOIGFAAAA|616|Church |Dr.|Suite K|Ashland|Genesee County|MI|44244|United States|-5|single family| +22159|AAAAAAAAPIGFAAAA|463|11th |Road|Suite M|Liberty|Brown County|WI|53451|United States|-6|apartment| +22160|AAAAAAAAAJGFAAAA|415|Ridge Forest|Dr.|Suite P|Franklin|Polk County|NC|29101|United States|-5|apartment| +22161|AAAAAAAABJGFAAAA|605|Chestnut |Way|Suite Q||||89193|United States|-7|single family| +22162|AAAAAAAACJGFAAAA|9|College 15th|Court|Suite 100|Waterloo|Geauga County|OH|41675|United States|-5|single family| +22163|AAAAAAAADJGFAAAA|183|Wilson |Road|Suite 390|Woodlawn|Hertford County|NC|24098|United States|-5|apartment| +22164|AAAAAAAAEJGFAAAA|14|Second Central|Dr.|Suite 270|Kingston|Hanson County|SD|54975|United States|-7|condo| +22165|AAAAAAAAFJGFAAAA|828|First Jefferson|Parkway|Suite 420|Montague|Cambria County|PA|14062|United States|-5|single family| +22166|AAAAAAAAGJGFAAAA|320|East |Way|Suite C|Lincoln|Pulaski County|KY|41289|United States|-5|condo| +22167|AAAAAAAAHJGFAAAA|361|Locust |RD|Suite 370|Roscoe|Mason County|MI|41854|United States|-5|condo| +22168|AAAAAAAAIJGFAAAA|740|Highland 8th|Avenue|Suite O|Hillcrest|Morrison County|MN|53003|United States|-6|single family| +22169|AAAAAAAAJJGFAAAA|814|Ridge Railroad|Wy|Suite 190|Oakwood|Marshall County|AL|30169|United States|-6|condo| +22170|AAAAAAAAKJGFAAAA|466|View Chestnut|Ct.|Suite 0|Green Acres|Coosa County|AL|37683|United States|-6|apartment| +22171|AAAAAAAALJGFAAAA|424|Ridge Chestnut|Dr.|Suite U|Shady Grove|Miller County|AR|72812|United States|-6|apartment| +22172|AAAAAAAAMJGFAAAA|378|Oak |Street|Suite J|Union|Phillips County|AR|78721|United States|-6|condo| +22173|AAAAAAAANJGFAAAA|251|6th |Avenue|Suite 320|Bunker Hill|Aransas County|TX|70150|United States|-6|single family| +22174|AAAAAAAAOJGFAAAA|294|Locust |Boulevard|Suite X|Union Hill|Ashe County|NC|27746|United States|-5|condo| +22175|AAAAAAAAPJGFAAAA|383|Woodland |Road|Suite 0|Pleasant Grove|San Juan County|UT|84136|United States|-7|single family| +22176|AAAAAAAAAKGFAAAA|684|Woodland |ST|Suite 470|Stringtown|Roane County|TN|30162|United States|-6|apartment| +22177|AAAAAAAABKGFAAAA|453|14th Railroad|Way|Suite L|Maple Grove|Polk County|TX|78252|United States|-6|apartment| +22178|AAAAAAAACKGFAAAA|373|View Hickory|ST|Suite P|Salem|Henderson County|NC|28048|United States|-5|apartment| +22179|AAAAAAAADKGFAAAA|684|Cedar |Boulevard|Suite 40|Clifton|Nobles County|MN|58014|United States|-6|single family| +22180|AAAAAAAAEKGFAAAA|490|First Third|Wy|Suite G|Barnes|Kearney County|NE|63788|United States|-7|apartment| +22181|AAAAAAAAFKGFAAAA|514|Church 3rd|Road|Suite 280|Red Hill|Shawnee County|KS|64338|United States|-6|apartment| +22182|AAAAAAAAGKGFAAAA|84|Sunset 7th|Circle|Suite 380|Mount Olive|Grant County|IN|48059|United States|-5|single family| +22183|AAAAAAAAHKGFAAAA|703|Poplar Third|Pkwy|Suite 100|Oak Ridge|Clarendon County|SC|28371|United States|-5|condo| +22184|AAAAAAAAIKGFAAAA|459|1st |Road|Suite 200|Hillcrest|Beaver County|OK|73003|United States|-6|condo| +22185|AAAAAAAAJKGFAAAA|477|||Suite B|Maple Grove|Cocke County|||United States|-5|condo| +22186|AAAAAAAAKKGFAAAA|547|View Park|Lane|Suite 280|Unionville|Ogle County|IL|61711|United States|-6|apartment| +22187|AAAAAAAALKGFAAAA|268|13th Elm|Lane|Suite Y|Wilson|Hutchinson County|TX|76971|United States|-6|single family| +22188|AAAAAAAAMKGFAAAA|953|13th |Lane|Suite X|Greenwood|Dorchester County|SC|28828|United States|-5|single family| +22189|AAAAAAAANKGFAAAA|426|West |Boulevard|Suite R|Wildwood|Trinity County|TX|76871|United States|-6|apartment| +22190|AAAAAAAAOKGFAAAA|192|Pine Oak|Boulevard|Suite L|Riverside|Luna County|NM|89231|United States|-7|single family| +22191|AAAAAAAAPKGFAAAA|788|Lincoln Sunset|Ave|Suite X|Salem|Cassia County|ID|88048|United States|-7|single family| +22192|AAAAAAAAALGFAAAA|7|2nd Birch|Ct.|Suite 40|Mount Pleasant|Sullivan County|TN|31933|United States|-6|single family| +22193|AAAAAAAABLGFAAAA|978|Highland Maple|ST|Suite U|Mount Olive|Hopkins County|KY|48059|United States|-6|single family| +22194|AAAAAAAACLGFAAAA|724|Sunset Mill|Blvd|Suite K|Midway|Sussex County|DE|11904|United States|-5|single family| +22195|AAAAAAAADLGFAAAA|452|Green Oak|ST|Suite 280|Cedar Grove|Dallas County|AR|70411|United States|-6|apartment| +22196|AAAAAAAAELGFAAAA|791|Birch |Avenue|Suite C|Welcome|Lowndes County|AL|36386|United States|-6|condo| +22197|AAAAAAAAFLGFAAAA|102|Broadway Lakeview|Wy|Suite L|Oakland|Plymouth County|MA|09843|United States|-5|single family| +22198|AAAAAAAAGLGFAAAA|494|Oak |Pkwy|Suite 200|Mount Zion|Pennington County|SD|58054|United States|-7|single family| +22199|AAAAAAAAHLGFAAAA|||Way|Suite 110|Cedar Grove|Chariton County|||United States|-6|single family| +22200|AAAAAAAAILGFAAAA|151|8th |RD|Suite 440|Lebanon|Phillips County|CO|82898|United States|-7|apartment| +22201|AAAAAAAAJLGFAAAA|696|View 7th|RD|Suite 270|Pierce|Ellis County|OK|73360|United States|-6|apartment| +22202|AAAAAAAAKLGFAAAA|11|College Main|Boulevard|Suite 220|Mount Vernon|Rice County|MN|58482|United States|-6|single family| +22203|AAAAAAAALLGFAAAA|371|Willow 15th|Ct.|Suite J|Marion|Bandera County|TX|70399|United States|-6|apartment| +22204|AAAAAAAAMLGFAAAA|801|Valley Chestnut|Way|Suite 190|Pleasant Valley|Lewis County|WA|92477|United States|-8|apartment| +22205|AAAAAAAANLGFAAAA|516|3rd Lincoln|Ln|Suite 380|Riverside|Rockingham County|NC|29231|United States|-5|single family| +22206|AAAAAAAAOLGFAAAA|788|Spruce Locust|Street|Suite W|Woodland|Chatham County|NC|24854|United States|-5|condo| +22207|AAAAAAAAPLGFAAAA|313|7th |Avenue|Suite 90|Crossroads|Weld County|CO|80534|United States|-7|condo| +22208|AAAAAAAAAMGFAAAA|737|Wilson Ridge|Drive|Suite 240|Edgewood|Rockingham County|VA|20069|United States|-5|single family| +22209|AAAAAAAABMGFAAAA|577|Maple Park|Ct.|Suite 40|Franklin|Cook County|MN|59101|United States|-6|apartment| +22210|AAAAAAAACMGFAAAA|657|Miller |Road|Suite L|Red Hill|Northumberland County|VA|24338|United States|-5|condo| +22211|AAAAAAAADMGFAAAA|564|5th |Ave|Suite N|Lakeview|Franklin County|PA|18579|United States|-5|condo| +22212|AAAAAAAAEMGFAAAA|419|6th 2nd|RD|Suite U|Friendship|Greene County|NC|24536|United States|-5|single family| +22213|AAAAAAAAFMGFAAAA|696|Willow Park|Avenue|Suite A|Oakdale|Titus County|TX|79584|United States|-6|condo| +22214|AAAAAAAAGMGFAAAA|506|Miller Park|Court|Suite M|Lebanon|Meeker County|MN|52898|United States|-6|condo| +22215|AAAAAAAAHMGFAAAA|451|Lakeview Lake|Lane|Suite X|Lakeville|Lowndes County|MS|58811|United States|-6|single family| +22216|AAAAAAAAIMGFAAAA|54|Smith 3rd|Circle|Suite Q|Arlington|Pondera County|MT|66557|United States|-7|single family| +22217|AAAAAAAAJMGFAAAA|375|Ridge |Avenue|Suite 0|Wildwood|Howard County|MD|26871|United States|-5|condo| +22218|AAAAAAAAKMGFAAAA|933|Forest 14th|Pkwy|Suite M|Five Forks|Evangeline Parish|LA|72293|United States|-6|apartment| +22219|AAAAAAAALMGFAAAA|227|Pine |Way|||Oconto County||51933|||apartment| +22220|AAAAAAAAMMGFAAAA|891|East Walnut|Avenue|Suite I|Greenville|Whitfield County|GA|31387|United States|-5|apartment| +22221|AAAAAAAANMGFAAAA|419|Lake |Court|Suite U|Lakeview|Ascension Parish|LA|78579|United States|-6|condo| +22222|AAAAAAAAOMGFAAAA|351|Franklin Lake|Road|Suite X|Salem|Tulare County|CA|98048|United States|-8|condo| +22223|AAAAAAAAPMGFAAAA|304|South |Ln|Suite 380|Glendale|New Haven County|CT|04551|United States|-5|condo| +22224|AAAAAAAAANGFAAAA|403|Willow |Blvd|Suite 40|Shady Grove|Clarke County|MS|52812|United States|-6|condo| +22225|AAAAAAAABNGFAAAA|345|Main |||Green Acres|Jenkins County|GA||United States||| +22226|AAAAAAAACNGFAAAA|473|Lakeview Lee|RD|Suite P|Pleasant Hill|Rockwall County|TX|73604|United States|-6|apartment| +22227|AAAAAAAADNGFAAAA||Hill Miller|||Wilson||MN||United States||apartment| +22228|AAAAAAAAENGFAAAA|671|River Chestnut|Way|Suite W|Newtown|King County|WA|91749|United States|-8|condo| +22229|AAAAAAAAFNGFAAAA|757|5th |Ct.|Suite S|Newport|Pinal County|AZ|81521|United States|-7|apartment| +22230|AAAAAAAAGNGFAAAA|432|15th East|Ln|Suite 170|Jackson|Colonial Heights city|VA|29583|United States|-5|condo| +22231|AAAAAAAAHNGFAAAA|952|4th |Court|Suite 400|Greenville|Cochran County|TX|71387|United States|-6|single family| +22232|AAAAAAAAINGFAAAA|131|Johnson |Street|Suite 410|Oak Hill|Hickman County|KY|47838|United States|-6|single family| +22233|AAAAAAAAJNGFAAAA|146|Park |Cir.|Suite T|Shady Grove|Pike County|IN|42812|United States|-5|apartment| +22234|AAAAAAAAKNGFAAAA|428|Second Main||||Pottawatomie County|KS|68137|United States|-6|single family| +22235|AAAAAAAALNGFAAAA|573|Seventh |Dr.|Suite 40|Greenwood|Daggett County|UT|88828|United States|-7|single family| +22236|AAAAAAAAMNGFAAAA|258|South |Circle|Suite 290|Harmony|Nevada County|CA|95804|United States|-8|apartment| +22237|AAAAAAAANNGFAAAA|576|Hill |Wy|Suite T|Oakley|Fairfield County|OH|49698|United States|-5|single family| +22238|AAAAAAAAONGFAAAA|545|4th Hill|Blvd|Suite D|Buena Vista|Jefferson County|WA|95752|United States|-8|condo| +22239|AAAAAAAAPNGFAAAA|69|Third Valley|Ct.|Suite 220|Pleasant Grove|Jeff Davis County|TX|74136|United States|-6|condo| +22240|AAAAAAAAAOGFAAAA|250|Park |Street|Suite E|Five Forks|Humboldt County|IA|52293|United States|-6|single family| +22241|AAAAAAAABOGFAAAA|699|Elm |Dr.|Suite 370|Springdale|Neshoba County|MS|58883|United States|-6|condo| +22242|AAAAAAAACOGFAAAA|212|Cedar First|ST|Suite 490|Salem|Greensville County|VA|28048|United States|-5|single family| +22243|AAAAAAAADOGFAAAA||Eigth |Parkway|Suite A||||||-6|| +22244|AAAAAAAAEOGFAAAA|854|3rd Oak|Pkwy|Suite I|Waterloo|Howard County|TX|71675|United States|-6|single family| +22245|AAAAAAAAFOGFAAAA|26|Lake College|Ct.|Suite 180|Tracy|Carter County|MO|66340|United States|-6|single family| +22246|AAAAAAAAGOGFAAAA|208|10th |Wy|Suite S|Stringtown|Hart County|KY|40162|United States|-6|condo| +22247|AAAAAAAAHOGFAAAA|798|Sunset Pine|Lane|Suite N|Kingston|Todd County|SD|54975|United States|-7|single family| +22248|AAAAAAAAIOGFAAAA|471|Miller Johnson|Ct.|Suite 380|Hillcrest|Columbia County|WI|53003|United States|-6|single family| +22249|AAAAAAAAJOGFAAAA|627|Oak |Blvd|Suite D|Pleasant Grove|Hettinger County|ND|54136|United States|-6|condo| +22250|AAAAAAAAKOGFAAAA|893|East Sixth|Ln|Suite Y|Spring Hill|Tooele County|UT|86787|United States|-7|apartment| +22251|AAAAAAAALOGFAAAA|836|Maple Willow|Pkwy|Suite Y|Five Points|Quitman County|MS|56098|United States|-6|apartment| +22252|AAAAAAAAMOGFAAAA|863|3rd Walnut|Parkway|Suite L|Clinton|Bossier Parish|LA|78222|United States|-6|single family| +22253|AAAAAAAANOGFAAAA|858|Fifth |Ave|Suite E|Liberty|Iron County|MO|63451|United States|-6|apartment| +22254|AAAAAAAAOOGFAAAA|916|Johnson Main|RD|Suite 200|Mount Vernon|Walworth County|SD|58482|United States|-7|apartment| +22255|AAAAAAAAPOGFAAAA|13|Cedar Spring|Dr.|Suite R|Stringtown|Day County|SD|50162|United States|-6|single family| +22256|AAAAAAAAAPGFAAAA|587|Lake |Parkway|Suite 220|Arlington|Rich County|UT|86557|United States|-7|single family| +22257|AAAAAAAABPGFAAAA|644|Highland Fourth|Ave|Suite 10|Sulphur Springs|Grant County|WA|98354|United States|-8|single family| +22258|AAAAAAAACPGFAAAA|761|Sixth |Circle|Suite F|Elkton|Faribault County|MN|53481|United States|-6|apartment| +22259|AAAAAAAADPGFAAAA|565|Elm |Road|||||13604|||single family| +22260|AAAAAAAAEPGFAAAA|529|Franklin Williams|Court|Suite 250|Oakdale|York County|PA|19584|United States|-5|single family| +22261|AAAAAAAAFPGFAAAA|161|11th Lakeview|Way|Suite K|Sunnyside|Campbell County|WY|81952|United States|-7|single family| +22262|AAAAAAAAGPGFAAAA|692|South Ridge|Lane|Suite U|Sunnyside|Lowndes County|MS|51952|United States|-6|apartment| +22263|AAAAAAAAHPGFAAAA|105|Oak 7th|Pkwy|Suite 390|Brownsville|Turner County|SD|59310|United States|-7|apartment| +22264|AAAAAAAAIPGFAAAA|478|3rd |Ct.|Suite S|Brownsville|Butler County|IA|59310|United States|-6|single family| +22265|AAAAAAAAJPGFAAAA|933|Lakeview 15th|Ave|Suite 360|Pleasant Valley|Perry County|TN|32477|United States|-6|single family| +22266|AAAAAAAAKPGFAAAA|298|Ash Pine|RD|Suite Q|Brunswick|Washington County|VT|05242|United States|-5|single family| +22267|AAAAAAAALPGFAAAA|616|Maple |Road|Suite 460|Wilson|Washburn County|WI|56971|United States|-6|condo| +22268|AAAAAAAAMPGFAAAA|234|Highland |Way|Suite 210|Providence|Lycoming County|PA|16614|United States|-5|single family| +22269|AAAAAAAANPGFAAAA|573|Hill |Cir.|Suite 280|Bunker Hill|Macon County|MO|60150|United States|-6|apartment| +22270|AAAAAAAAOPGFAAAA||Hickory Poplar|Ln|Suite 60|||||||| +22271|AAAAAAAAPPGFAAAA|201|West Laurel|Court|Suite 50|Hamilton|Okaloosa County|FL|32808|United States|-5|condo| +22272|AAAAAAAAAAHFAAAA|569|Cedar |Lane|Suite 40|Georgetown|Clark County|NV|87057|United States|-8|single family| +22273|AAAAAAAABAHFAAAA|539|Birch Mill|Blvd|Suite R|Ashley|Traverse County|MN|54324|United States|-6|single family| +22274|AAAAAAAACAHFAAAA|759|North |Boulevard|Suite 140|Valley View|Sabine County|TX|75124|United States|-6|apartment| +22275|AAAAAAAADAHFAAAA|697|Willow |Boulevard|Suite C|Newport|Mason County|TX|71521|United States|-6|condo| +22276|AAAAAAAAEAHFAAAA|886|Fourth 9th|Parkway|Suite F|Pleasant Grove|Cherokee County|AL|34136|United States|-6|apartment| +22277|AAAAAAAAFAHFAAAA|947|13th Jackson|Pkwy|Suite 330|Walnut Grove|Box Butte County|NE|67752|United States|-6|single family| +22278|AAAAAAAAGAHFAAAA|87|Church Forest|Pkwy|Suite A|Macedonia|Washakie County|WY|81087|United States|-7|apartment| +22279|AAAAAAAAHAHFAAAA|875|Park River|Lane|Suite B|Lincoln|San Miguel County|NM|81289|United States|-7|condo| +22280|AAAAAAAAIAHFAAAA|638|River |Cir.|Suite R|Cedar Grove|Strafford County|NH|01011|United States|-5|condo| +22281|AAAAAAAAJAHFAAAA|256|Pine |Pkwy|Suite L|Maple Grove|Carter County|KY|48252|United States|-6|apartment| +22282|AAAAAAAAKAHFAAAA|540|Maple |Dr.|Suite 320|Sunnyside|Summit County|CO|81952|United States|-7|single family| +22283|AAAAAAAALAHFAAAA|50|Maple 5th|Way|Suite H|Springdale|Washington County|WI|58883|United States|-6|apartment| +22284|AAAAAAAAMAHFAAAA|95|Davis North|Way|Suite D|Springfield|Renville County|ND|59303|United States|-6|condo| +22285|AAAAAAAANAHFAAAA||Willow |RD|Suite X||Fluvanna County||23511||-5|| +22286|AAAAAAAAOAHFAAAA|423|8th |ST|Suite F|Glenwood|White County|IN|43511|United States|-5|condo| +22287|AAAAAAAAPAHFAAAA|295|Wilson |||Brunswick||AR|74642||-6|| +22288|AAAAAAAAABHFAAAA|361|Wilson |Road|Suite G|Pine Grove|Grant County|WI|54593|United States|-6|single family| +22289|AAAAAAAABBHFAAAA|31|Walnut |Parkway|Suite I|Union Hill|Warren County|IA|57746|United States|-6|apartment| +22290|AAAAAAAACBHFAAAA|551|Pine Lake|Ave|Suite 400|Cedar Grove|Pipestone County|MN|50411|United States|-6|condo| +22291|AAAAAAAADBHFAAAA|918|Pine |Way|Suite G|Kingston|Laurel County|KY|44975|United States|-5|condo| +22292|AAAAAAAAEBHFAAAA|258|7th 7th|RD|Suite 480|Five Forks|Furnas County|NE|62293|United States|-6|condo| +22293|AAAAAAAAFBHFAAAA|946|North |Ln|Suite 460|Plainview|Dubuque County|IA|53683|United States|-6|condo| +22294|AAAAAAAAGBHFAAAA|547|Maple |Way|Suite A|Shiloh|Outagamie County|WI|59275|United States|-6|single family| +22295|AAAAAAAAHBHFAAAA|195|Main |ST|Suite 480|Forest Hills|Boone County|KY|49237|United States|-6|single family| +22296|AAAAAAAAIBHFAAAA|712|3rd |Road|Suite U|Springdale|Warren County|MS|58883|United States|-6|condo| +22297|AAAAAAAAJBHFAAAA|836|8th |Lane|Suite G|Tyrone|Hamilton County|IA|51201|United States|-6|condo| +22298|AAAAAAAAKBHFAAAA|620|East |Ct.|Suite 320|Sulphur Springs|Menominee County|MI|48354|United States|-5|apartment| +22299|AAAAAAAALBHFAAAA|35|Church 5th|Parkway|Suite 260|Ashland|Hardin County|IA|54244|United States|-6|single family| +22300|AAAAAAAAMBHFAAAA|300|Fourth |Avenue|Suite 310|Glendale|Clark County|IL|63951|United States|-6|single family| +22301|AAAAAAAANBHFAAAA|958|Railroad |Drive|Suite X|Mount Pleasant|Schuyler County|MO|61933|United States|-6|apartment| +22302|AAAAAAAAOBHFAAAA|454|Hillcrest |Circle|Suite 400|Five Points|Danville city|VA|26098|United States|-5|single family| +22303|AAAAAAAAPBHFAAAA|565||Court||||MO|63451|||| +22304|AAAAAAAAACHFAAAA|836|9th South|Blvd|Suite O|Summit|Pratt County|KS|60499|United States|-6|single family| +22305|AAAAAAAABCHFAAAA|66|Fourteenth Second|Wy|Suite G|Sleepy Hollow|Howard County|IN|43592|United States|-5|apartment| +22306|AAAAAAAACCHFAAAA|24|Cherry Valley|Wy|Suite 150|Hillcrest|Washington County|MN|53003|United States|-6|single family| +22307|AAAAAAAADCHFAAAA|549|4th Mill|Ln|Suite E|Shady Grove|Lawrence County|IN|42812|United States|-5|apartment| +22308|AAAAAAAAECHFAAAA|93|Forest 1st|RD|Suite G|Clifton|Garfield County|UT|88014|United States|-7|apartment| +22309|AAAAAAAAFCHFAAAA|628|Twelfth Madison|Ct.|Suite D|Pine Grove|Volusia County|FL|34593|United States|-5|apartment| +22310|AAAAAAAAGCHFAAAA|652|4th Oak|Blvd|Suite 190|Highland|Montgomery County|NC|29454|United States|-5|condo| +22311|AAAAAAAAHCHFAAAA|788|South Elm|Boulevard|Suite R|Peru|Marion County|OH|40302|United States|-5|single family| +22312|AAAAAAAAICHFAAAA|889|13th |Avenue|Suite 150|Pleasant Hill|Clinton County|MO|63604|United States|-6|single family| +22313|AAAAAAAAJCHFAAAA|690|Johnson |Way|Suite L|Greenwood|Carroll County|MD|28828|United States|-5|apartment| +22314|AAAAAAAAKCHFAAAA|429|Lincoln 12th|Dr.|Suite 80|Unionville|Fall River County|SD|51711|United States|-6|single family| +22315|AAAAAAAALCHFAAAA|354|1st |Road|Suite I|Midway|Gaines County|TX|71904|United States|-6|apartment| +22316|AAAAAAAAMCHFAAAA|117|Johnson |Ct.|Suite 170|Cedar Grove|Carver County|MN|50411|United States|-6|condo| +22317|AAAAAAAANCHFAAAA|692|9th |Avenue|Suite 90|Woodland|Morrow County|OR|94854|United States|-8|condo| +22318|AAAAAAAAOCHFAAAA|870|6th |Court|Suite 80|Macedonia|Stafford County|KS|61087|United States|-6|single family| +22319|AAAAAAAAPCHFAAAA|760|Willow |Ave|Suite 450|Mount Pleasant|Brantley County|GA|31933|United States|-5|condo| +22320|AAAAAAAAADHFAAAA|109|Church Second|Ln|Suite 340|Summit|Marshall County|IA|50499|United States|-6|single family| +22321|AAAAAAAABDHFAAAA|588|Broadway 3rd|Pkwy|Suite H|Pleasant Grove|Garfield County|OK|74136|United States|-6|apartment| +22322|AAAAAAAACDHFAAAA|524|Ash |Parkway|Suite 470|Spring Hill|Ellis County|OK|76787|United States|-6|single family| +22323|AAAAAAAADDHFAAAA|228|Madison Willow||||||34854||-6|single family| +22324|AAAAAAAAEDHFAAAA|66|3rd |Wy|Suite Y|Sulphur Springs|Putnam County|FL|38354|United States|-5|single family| +22325|AAAAAAAAFDHFAAAA|857|Smith Williams|Ct.|Suite 270|Nichols|Pike County|GA|37940|United States|-5|single family| +22326|AAAAAAAAGDHFAAAA|943|3rd Maple|Avenue|Suite W|Pleasant Hill|Jackson County|SD|53604|United States|-7|condo| +22327|AAAAAAAAHDHFAAAA|||||Rankin|||82621|United States||| +22328|AAAAAAAAIDHFAAAA|404|Main Washington|Avenue|Suite B|Riverside|Florence County|WI|59231|United States|-6|apartment| +22329|AAAAAAAAJDHFAAAA|979|Adams Central|Wy|Suite 180|Wildwood|Collin County|TX|76871|United States|-6|single family| +22330|AAAAAAAAKDHFAAAA|55|Cedar Park|Ln|Suite 210|Providence|Cimarron County|OK|76614|United States|-6|apartment| +22331|AAAAAAAALDHFAAAA|667|5th Elm|Street|Suite O|Liberty|Powell County|MT|63451|United States|-7|condo| +22332|AAAAAAAAMDHFAAAA|400|Lake |Lane|Suite F|Plainview|Faulkner County|AR|73683|United States|-6|condo| +22333|AAAAAAAANDHFAAAA|148|Franklin 4th|Lane|Suite R|Enterprise|Clark County|IL|61757|United States|-6|single family| +22334|AAAAAAAAODHFAAAA|534|Tenth North|Court|Suite Q|Bridgeport|Perry County|MS|55817|United States|-6|single family| +22335|AAAAAAAAPDHFAAAA|55|Elm Hillcrest|Wy|Suite 450|Shady Grove|Ralls County|MO|62812|United States|-6|condo| +22336|AAAAAAAAAEHFAAAA|201||||Lone Oak|Randolph County|NC|||-5|| +22337|AAAAAAAABEHFAAAA||Main West|Boulevard|Suite I|Franklin|Ouray County||89101|United States|-7|| +22338|AAAAAAAACEHFAAAA|355|North |Court|Suite 120|Riverdale|Galax city|VA|29391|United States|-5|single family| +22339|AAAAAAAADEHFAAAA|458|Lincoln |Ave|Suite B|Glenwood|Collin County|TX|73511|United States|-6|apartment| +22340|AAAAAAAAEEHFAAAA|886|Main |Ave|Suite O|Lakewood|Lincoln County|MT|68877|United States|-7|single family| +22341|AAAAAAAAFEHFAAAA|674|Hill Main|RD|Suite A|Arlington|Loudon County|TN|36557|United States|-6|apartment| +22342|AAAAAAAAGEHFAAAA|588|Williams |Ct.|Suite 420|Clifton|Laclede County|MO|68014|United States|-6|single family| +22343|AAAAAAAAHEHFAAAA|836|Oak |Pkwy|Suite I|Redland|Grant County|KS|66343|United States|-6|condo| +22344|AAAAAAAAIEHFAAAA|184|Washington |Road|Suite 70|Friendship|Calumet County|WI|54536|United States|-6|apartment| +22345|AAAAAAAAJEHFAAAA|723|Hill |Cir.|Suite 490|Greenfield|Kemper County|MS|55038|United States|-6|single family| +22346|AAAAAAAAKEHFAAAA|262|9th Park|Ln|Suite N|Maple Grove|Hood County|TX|78252|United States|-6|single family| +22347|AAAAAAAALEHFAAAA|277|Jefferson |Boulevard|Suite M|Plainview|Cumberland County|ME|04283|United States|-5|condo| +22348|AAAAAAAAMEHFAAAA|566|Walnut |Pkwy|Suite 430|Pomona|Pinellas County|FL|34153|United States|-5|condo| +22349|AAAAAAAANEHFAAAA|921|Meadow |Court|Suite T|Mountain View|Barnes County|ND|54466|United States|-6|apartment| +22350|AAAAAAAAOEHFAAAA|389|Oak |Avenue|Suite P|Mount Vernon|Franklin County|MO|68482|United States|-6|condo| +22351|AAAAAAAAPEHFAAAA|630|Adams Mill|ST|Suite X|Greenville|Marion County|MO|61387|United States|-6|apartment| +22352|AAAAAAAAAFHFAAAA|932|Fifth |Dr.|Suite D|Unionville|Clinton County|MI|41711|United States|-5|apartment| +22353|AAAAAAAABFHFAAAA|490|||Suite 170||Van Zandt County|TX|||-6|condo| +22354|AAAAAAAACFHFAAAA|118|13th Adams|Ln|Suite 100|Oak Hill|Clark County|NV|87838|United States|-8|single family| +22355|AAAAAAAADFHFAAAA|515|Maple Valley|Ct.|Suite 0|Deerfield|Parke County|IN|49840|United States|-5|condo| +22356|AAAAAAAAEFHFAAAA|847|Eigth ||||Donley County||73451|||apartment| +22357|AAAAAAAAFFHFAAAA|597|9th |ST|Suite 30|Ferguson|Hart County|KY|41821|United States|-6|single family| +22358|AAAAAAAAGFHFAAAA|714|Franklin |Wy|Suite K|Five Points|Ripley County|MO|66098|United States|-6|condo| +22359|AAAAAAAAHFHFAAAA|563|Park Broadway|ST|Suite L|Greenfield|Bennington County|VT|05638|United States|-5|single family| +22360|AAAAAAAAIFHFAAAA|547|2nd Poplar|Wy|Suite W|Lakeside|Rockingham County|VA|29532|United States|-5|single family| +22361|AAAAAAAAJFHFAAAA|470|3rd |Pkwy|Suite W|Edgewood|Tippecanoe County|IN|40069|United States|-5|condo| +22362|AAAAAAAAKFHFAAAA|85|13th |Parkway|Suite 0|Mount Olive|Calhoun County|IL|68059|United States|-6|single family| +22363|AAAAAAAALFHFAAAA|5|Jefferson 9th|RD|Suite D|Riverview|Webster County|IA|59003|United States|-6|apartment| +22364|AAAAAAAAMFHFAAAA|802|Third |ST|Suite Y|Liberty|Taylor County|GA|33451|United States|-5|apartment| +22365|AAAAAAAANFHFAAAA|370|Elevnth Lake|RD|Suite 120|Oak Ridge|McKean County|PA|18371|United States|-5|apartment| +22366|AAAAAAAAOFHFAAAA|||Ln|Suite P||Ben Hill County|GA||United States||single family| +22367|AAAAAAAAPFHFAAAA|580|Third Birch|RD|Suite 250|Concord|Cavalier County|ND|54107|United States|-6|single family| +22368|AAAAAAAAAGHFAAAA|908|Johnson Third|Street|Suite P|Newport|Elliott County|KY|41521|United States|-6|condo| +22369|AAAAAAAABGHFAAAA|728|15th |Avenue|Suite F|Highland Park|Midland County|MI|46534|United States|-5|single family| +22370|AAAAAAAACGHFAAAA|520|Cedar |Dr.|Suite L|Plainview|Knox County|MO|63683|United States|-6|single family| +22371|AAAAAAAADGHFAAAA|856|Davis 8th|Road|Suite P|Green Acres|Navarro County|TX|77683|United States|-6|apartment| +22372|AAAAAAAAEGHFAAAA|654|Main |Road|Suite 130|Lebanon|Litchfield County|CT|03498|United States|-5|condo| +22373|AAAAAAAAFGHFAAAA|920|Meadow Lakeview|Way|Suite V|Ellisville|Fannin County|TX|76820|United States|-6|single family| +22374|AAAAAAAAGGHFAAAA|454|Center |Street|Suite G|Richfield|Clarke County|GA|36196|United States|-5|apartment| +22375|AAAAAAAAHGHFAAAA||||Suite 450|Spring Hill||||||condo| +22376|AAAAAAAAIGHFAAAA|819|11th West|ST|Suite 170|Spring Valley|Trousdale County|TN|36060|United States|-6|single family| +22377|AAAAAAAAJGHFAAAA|82|5th |Court|Suite 210|Richardson|Butler County|OH|47687|United States|-5|condo| +22378|AAAAAAAAKGHFAAAA|655|15th |Ct.|Suite 60|Saint Johns|Rooks County|KS|65717|United States|-6|single family| +22379|AAAAAAAALGHFAAAA|386|Valley |Ave|Suite 490|Pleasant Valley|Wilkes County|NC|22477|United States|-5|apartment| +22380|AAAAAAAAMGHFAAAA|624|Oak |Ln|Suite 330|Red Hill|Van Buren County|AR|74338|United States|-6|apartment| +22381|AAAAAAAANGHFAAAA|99|Ridge |Road|Suite 320|Deerfield|Berkeley County|WV|29840|United States|-5|condo| +22382|AAAAAAAAOGHFAAAA|100|East |Pkwy|Suite D|Kingston|Levy County|FL|34975|United States|-5|condo| +22383|AAAAAAAAPGHFAAAA|901|View |Pkwy|Suite G|Union|Gentry County|MO|68721|United States|-6|condo| +22384|AAAAAAAAAHHFAAAA|494|Railroad |Boulevard|Suite 270|Five Forks|Robeson County|NC|22293|United States|-5|single family| +22385|AAAAAAAABHHFAAAA|413|Valley Spruce|Drive|Suite 250|Walnut Grove|Hamilton County|NY|17752|United States|-5|apartment| +22386|AAAAAAAACHHFAAAA|1000|Madison |Road|Suite O|Argyle|Kenai Peninsula Borough|AK|98722|United States|-9|apartment| +22387|AAAAAAAADHHFAAAA|53|View Maple|Blvd|Suite 270|Glendale|Boone County|NE|63951|United States|-6|single family| +22388|AAAAAAAAEHHFAAAA|336|Walnut |Road|Suite A|Barnes|Pontotoc County|OK|73788|United States|-6|single family| +22389|AAAAAAAAFHHFAAAA|95|East |Ave|Suite 110|Five Forks|Tulsa County|OK|72293|United States|-6|apartment| +22390|AAAAAAAAGHHFAAAA|668|Hillcrest |Road|Suite G|Oak Hill|Hamilton County|FL|37838|United States|-5|apartment| +22391|AAAAAAAAHHHFAAAA|765|North 13th|Drive|Suite 480|Clinton|Oldham County|KY|48222|United States|-5|condo| +22392|AAAAAAAAIHHFAAAA|364|Eigth |Lane|Suite W|Doyle|Oceana County|MI|48434|United States|-5|apartment| +22393|AAAAAAAAJHHFAAAA|270|Mill Walnut|Way|Suite 250|Sunnyside|Woodson County|KS|61952|United States|-6|apartment| +22394|AAAAAAAAKHHFAAAA||Oak ||Suite 210||Ulster County|NY|||-5|single family| +22395|AAAAAAAALHHFAAAA|864|First 1st|Ln|Suite A|Oakwood|Nelson County|KY|40169|United States|-5|condo| +22396|AAAAAAAAMHHFAAAA|725|Park |Dr.|Suite R|Appleton|Carter County|KY|44240|United States|-6|apartment| +22397|AAAAAAAANHHFAAAA|720|Chestnut River|Circle|Suite 20|Jackson|DeKalb County|MO|69583|United States|-6|condo| +22398|AAAAAAAAOHHFAAAA|475|2nd Park|Ct.|Suite U|Jackson|Winkler County|TX|79583|United States|-6|apartment| +22399|AAAAAAAAPHHFAAAA||Oak |Circle|Suite S|Sunnyside|Lac qui Parle County|MN||United States|-6|| +22400|AAAAAAAAAIHFAAAA|29|5th |ST|Suite S|Waterloo|Dubois County|IN|41675|United States|-5|apartment| +22401|AAAAAAAABIHFAAAA|25|Lincoln Lee|Street|Suite A|Mount Pleasant|Robertson County|KY|41933|United States|-5|single family| +22402|AAAAAAAACIHFAAAA|803|Elevnth Davis|Avenue|Suite G|Cedar Grove|Gage County|NE|60411|United States|-6|single family| +22403|AAAAAAAADIHFAAAA|873|East |Ave|Suite A|Riverdale|Decatur County|IA|59391|United States|-6|single family| +22404|AAAAAAAAEIHFAAAA|907|Broadway |RD|Suite 200|Oak Ridge|Pershing County|NV|88371|United States|-8|apartment| +22405|AAAAAAAAFIHFAAAA|1|Laurel |Ave|Suite 400|Farmington|Rutland County|VT|09745|United States|-5|single family| +22406|AAAAAAAAGIHFAAAA|242|Church Green|RD|Suite Y|Salem|Stone County|MS|58048|United States|-6|condo| +22407|AAAAAAAAHIHFAAAA|969|Second |Circle|Suite J|Summit|Liberty County|MT|60499|United States|-7|apartment| +22408|AAAAAAAAIIHFAAAA|485|2nd Lincoln|Wy|Suite A|Oakdale|Elk County|PA|19584|United States|-5|single family| +22409|AAAAAAAAJIHFAAAA|956|Maple Williams|Court|Suite 220|Oak Hill|Cherokee County|AL|37838|United States|-6|condo| +22410|AAAAAAAAKIHFAAAA|663|Franklin Hickory|Dr.|Suite 470|Jamestown|Glascock County|GA|36867|United States|-5|single family| +22411|AAAAAAAALIHFAAAA|751|Elevnth |Way|Suite V|Union|Mackinac County|MI|48721|United States|-5|single family| +22412|AAAAAAAAMIHFAAAA|435|14th Willow|Pkwy|Suite 360|Lincoln|Cambria County|PA|11289|United States|-5|condo| +22413|AAAAAAAANIHFAAAA|868|Maple |Lane|Suite 70|Red Hill|Maui County|HI|94338|United States|-10|condo| +22414|AAAAAAAAOIHFAAAA|632|3rd |Wy|Suite 490|Ashland|Latimer County|OK|74244|United States|-6|apartment| +22415|AAAAAAAAPIHFAAAA|22|Lee |Court|Suite F|Summit|Clinton County|OH|40499|United States|-5|condo| +22416|AAAAAAAAAJHFAAAA|123|Birch |Ave|Suite T|Elm Grove|Baldwin County|GA|33298|United States|-5|apartment| +22417|AAAAAAAABJHFAAAA|703|Third |Boulevard|Suite 210|Centerville|Douglas County|CO|80059|United States|-7|condo| +22418|AAAAAAAACJHFAAAA|51|River Oak|Street|Suite 70|Edgewood|Chickasaw County|IA|50069|United States|-6|single family| +22419|AAAAAAAADJHFAAAA|79|Spruce |Circle|Suite 140|Bridgeport|Roseau County|MN|55817|United States|-6|condo| +22420|AAAAAAAAEJHFAAAA|259|8th |Road|Suite W|Forestville|Pickens County|AL|33027|United States|-6|apartment| +22421|AAAAAAAAFJHFAAAA|348|Davis |Dr.|Suite 0|Lincoln|Powell County|MT|61289|United States|-7|condo| +22422|AAAAAAAAGJHFAAAA|189|Cedar Lee|Ave|Suite 150|Pleasant Valley|Onslow County|NC|22477|United States|-5|condo| +22423|AAAAAAAAHJHFAAAA|769|10th |Pkwy|Suite X|Elkton|Accomack County|VA|23481|United States|-5|condo| +22424|AAAAAAAAIJHFAAAA|320|3rd |Street|Suite V|Arlington|Caledonia County|VT|07157|United States|-5|condo| +22425|AAAAAAAAJJHFAAAA|801|Pine 15th|Cir.|Suite S|Woodville|Pike County|IN|44289|United States|-5|condo| +22426|AAAAAAAAKJHFAAAA|46|Williams Franklin|Pkwy|Suite 80|Waterloo|Ellsworth County|KS|61675|United States|-6|apartment| +22427|AAAAAAAALJHFAAAA|600|5th Green|Ave|Suite T|Bunker Hill|Windsor County|VT|00750|United States|-5|single family| +22428|AAAAAAAAMJHFAAAA|507|Hill |Blvd|Suite 240|Providence|Palo Alto County|IA|56614|United States|-6|condo| +22429|AAAAAAAANJHFAAAA|258|Madison Lincoln|Way|Suite Q|Twin Oaks|Shelby County|TX|70663|United States|-6|single family| +22430|AAAAAAAAOJHFAAAA|298|College |Ct.|Suite A|Springfield|Winston County|MS|59303|United States|-6|condo| +22431|AAAAAAAAPJHFAAAA|213|Poplar Highland|Drive|Suite I|Waterloo|Hamilton County|TX|71675|United States|-6|condo| +22432|AAAAAAAAAKHFAAAA|617|Cedar |Pkwy|Suite F|Mount Vernon|Lowndes County|AL|38482|United States|-6|single family| +22433|AAAAAAAABKHFAAAA|986|Birch |Ave|Suite 260|Midway|Lowndes County|MS|51904|United States|-6|single family| +22434|AAAAAAAACKHFAAAA|945|4th |Boulevard|Suite 150|Waterloo|Orleans County|VT|02275|United States|-5|condo| +22435|AAAAAAAADKHFAAAA|742|Jefferson |Lane|Suite 450|Griffin|Steele County|MN|51204|United States|-6|condo| +22436|AAAAAAAAEKHFAAAA|870|3rd Hill|Dr.|Suite 370|Mount Olive|Upshur County|WV|28059|United States|-5|single family| +22437|AAAAAAAAFKHFAAAA|253|1st River|RD|Suite 320|Oakdale|Strafford County|NH|09584|United States|-5|apartment| +22438|AAAAAAAAGKHFAAAA|893|Spring |Way|Suite J|Franklin|Fairfield County|CT|09701|United States|-5|condo| +22439|AAAAAAAAHKHFAAAA|599|Church |RD|Suite G|Mountain View|Pike County|GA|34466|United States|-5|apartment| +22440|AAAAAAAAIKHFAAAA|874|Pine Ridge||Suite Y|Cedar Grove|Edgefield County|SC|||-5|| +22441|AAAAAAAAJKHFAAAA|529|3rd Lincoln|Drive|Suite 80|Arlington|Telfair County|GA|36557|United States|-5|apartment| +22442|AAAAAAAAKKHFAAAA|538|14th |Parkway|Suite Q|Jackson|McLeod County|MN|59583|United States|-6|apartment| +22443|AAAAAAAALKHFAAAA||15th |||Oakwood|Patrick County||||-5|| +22444|AAAAAAAAMKHFAAAA|809|Lakeview |Pkwy|Suite A|Siloam|Brown County|KS|68948|United States|-6|condo| +22445|AAAAAAAANKHFAAAA|937|Lake Park|Ct.|Suite X|Pleasant Valley|Sherburne County|MN|52477|United States|-6|condo| +22446|AAAAAAAAOKHFAAAA|888|Sunset |Drive|Suite 220|Mount Zion|Van Buren County|MI|48054|United States|-5|apartment| +22447|AAAAAAAAPKHFAAAA|554|Second |Circle|Suite H|Plainview|Grayson County|VA|23683|United States|-5|apartment| +22448|AAAAAAAAALHFAAAA|745|7th |Ct.|Suite I|Clifton|Yates County|NY|18014|United States|-5|single family| +22449|AAAAAAAABLHFAAAA|196|6th |Circle|Suite Q|Lakeview|Lyon County|IA|58579|United States|-6|single family| +22450|AAAAAAAACLHFAAAA|627|Washington |Cir.|Suite V|Pleasant Valley|Nowata County|OK|72477|United States|-6|apartment| +22451|AAAAAAAADLHFAAAA|539|Lake |Street|Suite 100|Bunker Hill|Alexander County|NC|20150|United States|-5|single family| +22452|AAAAAAAAELHFAAAA|645|East |Road|Suite F|Flint|Lyon County|KS|68909|United States|-6|apartment| +22453|AAAAAAAAFLHFAAAA|489|Third |Cir.|Suite 80|Fairview|Somerset County|PA|15709|United States|-5|condo| +22454|AAAAAAAAGLHFAAAA|348|Wilson |ST|Suite 150|Spring Valley|Norfolk city|VA|26060|United States|-5|condo| +22455|AAAAAAAAHLHFAAAA|556|Williams |Pkwy|Suite K|Mount Pleasant||TX||||apartment| +22456|AAAAAAAAILHFAAAA|107|Jackson Meadow|Avenue|Suite F|Jamestown|Vinton County|OH|46867|United States|-5|apartment| +22457|AAAAAAAAJLHFAAAA|715|14th |Cir.|Suite S|Enterprise|Buffalo County|WI|51757|United States|-6|apartment| +22458|AAAAAAAAKLHFAAAA|58|9th |Parkway|Suite 350|Oakland|Calhoun County|AR|79843|United States|-6|condo| +22459|AAAAAAAALLHFAAAA|471|Dogwood Pine|Blvd|Suite 260|Dallas|Worth County|GA|33628|United States|-5|condo| +22460|AAAAAAAAMLHFAAAA|174|Oak |Dr.|Suite B|Woodland|Lake County|IN|44854|United States|-5|apartment| +22461|AAAAAAAANLHFAAAA|467|9th |Dr.|Suite 410|Valley View|Campbell County|WY|85124|United States|-7|condo| +22462|AAAAAAAAOLHFAAAA|378|Broadway |Dr.|Suite A|Clinton|Stark County|OH|48222|United States|-5|single family| +22463|AAAAAAAAPLHFAAAA|||Boulevard|Suite D|Oak Hill|||97838|United States||| +22464|AAAAAAAAAMHFAAAA|375|2nd |Way|Suite P|Belleville|Crawford County|IA|52924|United States|-6|condo| +22465|AAAAAAAABMHFAAAA|887|Franklin |Avenue|Suite 470|Valley View|Coffey County|KS|65124|United States|-6|single family| +22466|AAAAAAAACMHFAAAA|420|South |Ln|Suite 440|Pleasant Hill|Millard County|UT|83604|United States|-7|single family| +22467|AAAAAAAADMHFAAAA|321|View |Pkwy|Suite 350|Green Acres|Park County|MT|67683|United States|-7|apartment| +22468|AAAAAAAAEMHFAAAA|231|Woodland |Avenue|Suite G|Shannon|McPherson County|KS|64120|United States|-6|apartment| +22469|AAAAAAAAFMHFAAAA|342|Walnut View|Ln|Suite 190|White Oak|East Baton Rouge Parish|LA|76668|United States|-6|single family| +22470|AAAAAAAAGMHFAAAA|351|Seventh |Ave||Pleasant Hill|Kandiyohi County||53604|||| +22471|AAAAAAAAHMHFAAAA|17|5th |RD|Suite J|Springdale|Rush County|KS|68883|United States|-6|single family| +22472|AAAAAAAAIMHFAAAA|399|Second |Boulevard|Suite O|White Oak|Wayne County|GA|36668|United States|-5|condo| +22473|AAAAAAAAJMHFAAAA|981|Second Franklin|Dr.|Suite 60|Crossroads|Clackamas County|OR|90534|United States|-8|apartment| +22474|AAAAAAAAKMHFAAAA|16|Pine |Wy|Suite X|Mount Vernon|Oktibbeha County|MS|58482|United States|-6|apartment| +22475|AAAAAAAALMHFAAAA|970|Adams |Boulevard|Suite B|Mount Olive|Liberty County|FL|38059|United States|-5|apartment| +22476|AAAAAAAAMMHFAAAA|777|Spring |Avenue|Suite 50|Vienna|Calcasieu Parish|LA|75119|United States|-6|apartment| +22477|AAAAAAAANMHFAAAA|631|4th |Lane|Suite 90|Five Points|Washington County|IA|56098|United States|-6|apartment| +22478|AAAAAAAAOMHFAAAA|582|Main |Road|Suite J|Guilford|Rio Grande County|CO|84408|United States|-7|condo| +22479|AAAAAAAAPMHFAAAA|883|9th Cherry|Avenue|Suite C|Lebanon|Atascosa County|TX|72898|United States|-6|apartment| +22480|AAAAAAAAANHFAAAA|795|Lee 6th|RD|Suite 100|Oakwood|Transylvania County|NC|20169|United States|-5|apartment| +22481|AAAAAAAABNHFAAAA|46|9th Church|Avenue|Suite N|Salem|Ward County|TX|78048|United States|-6|condo| +22482|AAAAAAAACNHFAAAA|300|12th |Ct.|Suite R|Sawyer|Cochise County|AZ|86045|United States|-7|apartment| +22483|AAAAAAAADNHFAAAA|816||Pkwy|Suite 470||||70069|United States||single family| +22484|AAAAAAAAENHFAAAA|515|Poplar Central|Dr.|Suite X|Tremont|Blaine County|OK|79515|United States|-6|condo| +22485|AAAAAAAAFNHFAAAA|945|Maple |Wy|Suite 490|Valley View|Cibola County|NM|85124|United States|-7|apartment| +22486|AAAAAAAAGNHFAAAA|711|Hill |Street|Suite 150|Pine Grove|Tazewell County|VA|24593|United States|-5|apartment| +22487|AAAAAAAAHNHFAAAA|457|Valley |Avenue|Suite 460|Rankin|Socorro County|NM|82621|United States|-7|apartment| +22488|AAAAAAAAINHFAAAA|271|Fourteenth Cherry|ST|Suite O|Springdale|Johnson County|GA|38883|United States|-5|apartment| +22489|AAAAAAAAJNHFAAAA|81|Ash |Pkwy|Suite B|White Oak|Danville city|VA|26668|United States|-5|single family| +22490|AAAAAAAAKNHFAAAA|930|Elm Second|Boulevard|Suite 290|Hillcrest|Marion County|OR|93003|United States|-8|apartment| +22491|AAAAAAAALNHFAAAA|614|Spring |Street|Suite G|Wildwood|Laclede County|MO|66871|United States|-6|condo| +22492|AAAAAAAAMNHFAAAA|25|Willow Mill|Street|Suite Q|Bunker Hill|Haralson County|GA|30150|United States|-5|single family| +22493|AAAAAAAANNHFAAAA|109|5th |Avenue|Suite 320|Green Acres|Crittenden County|AR|77683|United States|-6|condo| +22494|AAAAAAAAONHFAAAA|152|Broadway Chestnut|Blvd|Suite C|Riverview|Clark County|NV|89003|United States|-8|apartment| +22495|AAAAAAAAPNHFAAAA|280|Pine |Circle|Suite 390|Union Hill|Effingham County|IL|67746|United States|-6|condo| +22496|AAAAAAAAAOHFAAAA|215|Franklin |Circle|Suite 300|Bunker Hill|Kay County|OK|70150|United States|-6|single family| +22497|AAAAAAAABOHFAAAA|67|Cedar 12th|Circle|Suite 460|Salem|Glynn County|GA|38048|United States|-5|condo| +22498|AAAAAAAACOHFAAAA|741|River Railroad|Wy|Suite A|Blanchard|Carteret County|NC|25985|United States|-5|condo| +22499|AAAAAAAADOHFAAAA|126|College College|Ln|Suite 430|Highland|Merced County|CA|99454|United States|-8|apartment| +22500|AAAAAAAAEOHFAAAA|212|Third |Circle|Suite I|Union City|Pulaski County|VA|28087|United States|-5|single family| +22501|AAAAAAAAFOHFAAAA|725|Jackson |Avenue|Suite P|Hopewell|Berrien County|MI|40587|United States|-5|single family| +22502|AAAAAAAAGOHFAAAA|692|Williams 5th|Way|Suite 420|Forest Hills|Nueces County|TX|79237|United States|-6|apartment| +22503|AAAAAAAAHOHFAAAA|333|Adams Sunset|Lane|Suite Q|Summit|Lake County|MI|40499|United States|-5|apartment| +22504|AAAAAAAAIOHFAAAA|305|9th |Way|Suite 200|Clifton|Manassas city|VA|28014|United States|-5|condo| +22505|AAAAAAAAJOHFAAAA|732|12th Spring|Street|Suite E|Evans|Muscogee County|GA|32284|United States|-5|single family| +22506|AAAAAAAAKOHFAAAA|196|Washington Church|Cir.|Suite 200|Arlington|Woodruff County|AR|76557|United States|-6|condo| +22507|AAAAAAAALOHFAAAA|474|Railroad North|Drive|Suite 330|Spring Hill|Morris County|TX|76787|United States|-6|condo| +22508|AAAAAAAAMOHFAAAA|206|12th Fifth|Lane|Suite X|Liberty|Switzerland County|IN|43451|United States|-5|single family| +22509|AAAAAAAANOHFAAAA|500|View 14th|Lane|Suite 90|Pleasant Valley|Hardin County|TX|72477|United States|-6|condo| +22510|AAAAAAAAOOHFAAAA|369|View Cherry|Street|Suite A|Mount Olive|Nelson County|ND|58059|United States|-6|single family| +22511|AAAAAAAAPOHFAAAA|740|11th |Street|Suite N|Dallas|Lincoln County|ID|83628|United States|-7|apartment| +22512|AAAAAAAAAPHFAAAA|472|2nd Ridge|Pkwy|Suite Q|Riverview|Hertford County|NC|29003|United States|-5|apartment| +22513|AAAAAAAABPHFAAAA|289|Hill |Avenue|Suite O|Summit|Dodge County|MN|50499|United States|-6|apartment| +22514|AAAAAAAACPHFAAAA|170|6th |Boulevard|Suite N|Antioch|Grant County|MN|58605|United States|-6|condo| +22515|AAAAAAAADPHFAAAA|335|Oak Birch|Dr.|Suite 100|Midway|Lafayette County|FL|31904|United States|-5|apartment| +22516|AAAAAAAAEPHFAAAA|144|Forest Valley|Court|Suite N|Oakwood|Grant County|SD|50169|United States|-6|single family| +22517|AAAAAAAAFPHFAAAA|102|Pine Davis|Blvd|Suite 150|Bunker Hill|Jackson County|MN|50150|United States|-6|single family| +22518|AAAAAAAAGPHFAAAA|271|||Suite 210||Alcona County|MI|42219|United States||| +22519|AAAAAAAAHPHFAAAA|306|Cherry Locust|Road|Suite M|Friendship|Columbia County|AR|74536|United States|-6|single family| +22520|AAAAAAAAIPHFAAAA|439|2nd |Circle|Suite 270|Redland|Henry County|OH|46343|United States|-5|single family| +22521|AAAAAAAAJPHFAAAA||Pine Eigth||Suite 70||||||-5|| +22522|AAAAAAAAKPHFAAAA|624|7th |Ln|Suite 310|Valley View|Hancock County|KY|45124|United States|-6|apartment| +22523|AAAAAAAALPHFAAAA|121|2nd |Lane|Suite 90|Hopewell|Lawrence County|MO|60587|United States|-6|single family| +22524|AAAAAAAAMPHFAAAA|216|6th |Dr.|Suite 340|Stringtown|Greene County|AR|70162|United States|-6|single family| +22525|AAAAAAAANPHFAAAA|136|Center |Pkwy|Suite 420|White Oak|Pierce County|GA|36668|United States|-5|condo| +22526|AAAAAAAAOPHFAAAA|989|2nd |Court|Suite E|Mountain View|Caroline County|MD|24466|United States|-5|apartment| +22527|AAAAAAAAPPHFAAAA|419|3rd Wilson|Drive|Suite 140|Florence|Monroe County|IA|53394|United States|-6|single family| +22528|AAAAAAAAAAIFAAAA|75|Spruce |Pkwy|Suite 490|Jackson|Chambers County|AL|39583|United States|-6|condo| +22529|AAAAAAAABAIFAAAA|285|Tenth |Dr.|Suite L|Fairfield|Osceola County|IA|56192|United States|-6|condo| +22530|AAAAAAAACAIFAAAA|677|10th Highland|Cir.|Suite G|New Town|Presque Isle County|MI|49634|United States|-5|single family| +22531|AAAAAAAADAIFAAAA|414|Woodland |Court|Suite N|Pleasant Valley|Colorado County|TX|72477|United States|-6|condo| +22532|AAAAAAAAEAIFAAAA|498|Sunset |Lane|Suite 390|Georgetown|Monroe County|AL|37057|United States|-6|condo| +22533|AAAAAAAAFAIFAAAA|278|Railroad Sunset|Cir.|Suite 100|Macedonia|Corson County|SD|51087|United States|-6|single family| +22534|AAAAAAAAGAIFAAAA|878|||Suite F|Oak Grove|Rock Island County|||||condo| +22535|AAAAAAAAHAIFAAAA|983|Spruce |Ln|Suite 460|Riverview|Wolfe County|KY|49003|United States|-5|condo| +22536|AAAAAAAAIAIFAAAA|904|Locust First|Circle||Oakland|||49843|United States||condo| +22537|AAAAAAAAJAIFAAAA|864|Third |Lane|Suite 460|Spring Valley|Meeker County|MN|56060|United States|-6|apartment| +22538|AAAAAAAAKAIFAAAA|630|4th Williams|Way|Suite 20|Maple Grove|Windham County|VT|08852|United States|-5|condo| +22539|AAAAAAAALAIFAAAA|367|Pine |Ct.|Suite 160|Edgewood|Schuyler County|NY|10069|United States|-5|apartment| +22540|AAAAAAAAMAIFAAAA|585|Oak |Boulevard|Suite 160|Lakeside|Motley County|TX|79532|United States|-6|single family| +22541|AAAAAAAANAIFAAAA||Spruce ||Suite S||Coke County|||||| +22542|AAAAAAAAOAIFAAAA|436|Maple |Court|Suite E|Harmony|Upshur County|WV|25804|United States|-5|condo| +22543|AAAAAAAAPAIFAAAA|27|Highland Mill|Pkwy|Suite M|Mount Olive|Adair County|KY|48059|United States|-6|condo| +22544|AAAAAAAAABIFAAAA|630|||Suite 130||Lincoln County|KS||||single family| +22545|AAAAAAAABBIFAAAA|356|Forest Willow|Parkway|Suite 470|Riverdale|Cecil County|MD|29391|United States|-5|apartment| +22546|AAAAAAAACBIFAAAA|422|Railroad |RD|Suite A|Union|La Porte County|IN|48721|United States|-5|single family| +22547|AAAAAAAADBIFAAAA|917|Maple Maple|Court|Suite U|Milo|Roane County|WV|20116|United States|-5|single family| +22548|AAAAAAAAEBIFAAAA|910|Chestnut |Dr.|Suite Y|Greenwood|Monroe County|IN|48828|United States|-5|apartment| +22549|AAAAAAAAFBIFAAAA|634|Birch Eigth|Court|Suite T|Unionville|Todd County|SD|51711|United States|-7|single family| +22550|AAAAAAAAGBIFAAAA|566|Walnut East|Road|Suite L|Franklin|Kingsbury County|SD|59101|United States|-7|single family| +22551|AAAAAAAAHBIFAAAA|418|Laurel Sunset|Cir.|Suite 240|Springfield|Hall County|TX|79303|United States|-6|apartment| +22552|AAAAAAAAIBIFAAAA|453|Poplar South|Ave|Suite B|Sunnyside|Pine County|MN|51952|United States|-6|condo| +22553|AAAAAAAAJBIFAAAA|197|10th |Road|Suite 310|Bunker Hill|Miller County|GA|30150|United States|-5|apartment| +22554|AAAAAAAAKBIFAAAA|899|5th |Road|Suite 190|Lakewood|Jim Wells County|TX|78877|United States|-6|single family| +22555|AAAAAAAALBIFAAAA|268|Wilson Lincoln|RD|Suite V|Bridgeport|Hays County|TX|75817|United States|-6|condo| +22556|AAAAAAAAMBIFAAAA|961|4th 1st|Lane|Suite C|Oakwood|Roger Mills County|OK|70169|United States|-6|apartment| +22557|AAAAAAAANBIFAAAA|533|Lee |Wy|Suite X|Oak Grove|Dade County|MO|68370|United States|-6|apartment| +22558|AAAAAAAAOBIFAAAA|495|8th Sycamore|Ave|Suite 310|Glendale|Polk County|NC|23951|United States|-5|single family| +22559|AAAAAAAAPBIFAAAA||||Suite 400||Rockingham County|||||| +22560|AAAAAAAAACIFAAAA|272|7th |Boulevard|Suite 460|Union|Richland County|WI|58721|United States|-6|apartment| +22561|AAAAAAAABCIFAAAA|424|Wilson |Ln|Suite 50|Oakdale|Leake County|MS|59584|United States|-6|condo| +22562|AAAAAAAACCIFAAAA|413|Lake |Road|Suite L|Liberty|Sweet Grass County|MT|63451|United States|-7|apartment| +22563|AAAAAAAADCIFAAAA|797|Third Hill|Pkwy|Suite P|Bridgeport|Isanti County|MN|55817|United States|-6|condo| +22564|AAAAAAAAECIFAAAA|236|Second Hillcrest|Wy|Suite 330|Wilson|Knox County|TX|76971|United States|-6|single family| +22565|AAAAAAAAFCIFAAAA|584|7th Franklin|ST|Suite 370|Pleasant Valley|Clarke County|IA|52477|United States|-6|condo| +22566|AAAAAAAAGCIFAAAA|771|11th |Street|Suite 210|Hartland|Starr County|TX|76594|United States|-6|condo| +22567|AAAAAAAAHCIFAAAA|943|River |Road|Suite 100|Unionville|Wayne County|KY|41711|United States|-5|single family| +22568|AAAAAAAAICIFAAAA|593|Lake 1st|Pkwy|Suite 460|Marion|Sheridan County|MT|60399|United States|-7|apartment| +22569|AAAAAAAAJCIFAAAA|439|Hickory |Parkway|Suite R|Ellsworth|Chambers County|AL|35079|United States|-6|single family| +22570|AAAAAAAAKCIFAAAA|637|9th |Way|Suite C|Mount Vernon|Dane County|WI|58482|United States|-6|condo| +22571|AAAAAAAALCIFAAAA|157|First |Way|Suite 340|Springdale|Dane County|WI|58883|United States|-6|apartment| +22572|AAAAAAAAMCIFAAAA|195|Maple |Wy|Suite L|Sulphur Springs|White County|IN|48354|United States|-5|condo| +22573|AAAAAAAANCIFAAAA||Adams |Ct.|Suite 460||Washington County|PA|16668|||| +22574|AAAAAAAAOCIFAAAA|483|3rd Chestnut|Ave||Buena Vista|Fairfax city||25752|United States||condo| +22575|AAAAAAAAPCIFAAAA|55|Maple |Lane|Suite 60|Pleasant Hill|Stone County|MO|63604|United States|-6|condo| +22576|AAAAAAAAADIFAAAA|696|Broadway Willow|Road|Suite 0|Woodlawn|Lee County|SC|24098|United States|-5|single family| +22577|AAAAAAAABDIFAAAA|259|Second 9th|Cir.|Suite Y|Fairview|George County|MS|55709|United States|-6|condo| +22578|AAAAAAAACDIFAAAA||11th ||Suite 430|Greenfield||||United States|-7|| +22579|AAAAAAAADDIFAAAA|636|Lake |Ct.|Suite R|Red Hill|Waller County|TX|74338|United States|-6|condo| +22580|AAAAAAAAEDIFAAAA|715|Jefferson North|Ln|Suite L|Crossroads|Ashe County|NC|20534|United States|-5|apartment| +22581|AAAAAAAAFDIFAAAA|758|2nd Walnut|Way|Suite F|Bayview|Dinwiddie County|VA|29672|United States|-5|apartment| +22582|AAAAAAAAGDIFAAAA|934|Mill |Parkway|Suite M|Jackson|Somerset County|PA|19583|United States|-5|single family| +22583|AAAAAAAAHDIFAAAA|156|Cedar Main|Pkwy|Suite 460|Midway|Chester County|TN|31904|United States|-5|single family| +22584|AAAAAAAAIDIFAAAA|988|7th 10th|Cir.|Suite D|Edgewood|Humphreys County|TN|30069|United States|-5|apartment| +22585|AAAAAAAAJDIFAAAA|109|Park Hickory|ST|Suite 190|Sunnyside|Santa Fe County|NM|81952|United States|-7|apartment| +22586|AAAAAAAAKDIFAAAA|236|Maple Smith|Avenue|Suite M|Plainville|Converse County|WY|86115|United States|-7|condo| +22587|AAAAAAAALDIFAAAA|531|First Main|Lane|Suite P|Five Points|Macon County|GA|36098|United States|-5|single family| +22588|AAAAAAAAMDIFAAAA|41|Forest Ash|Pkwy|Suite 270|Liberty|Andrew County|MO|63451|United States|-6|apartment| +22589|AAAAAAAANDIFAAAA|131|Walnut Railroad|Court|Suite 150|Crossroads|Richland County|WI|50534|United States|-6|condo| +22590|AAAAAAAAODIFAAAA|684|Cedar |Ave|Suite 30|Antioch|El Paso County|TX|78605|United States|-6|single family| +22591|AAAAAAAAPDIFAAAA|186|5th Locust|Road|Suite R|Marion|Laurel County|KY|40399|United States|-5|single family| +22592|AAAAAAAAAEIFAAAA|995|2nd |Pkwy|Suite 240|Riverdale|Concho County|TX|79391|United States|-6|apartment| +22593|AAAAAAAABEIFAAAA|690|9th 13th|Blvd|Suite K|Crossroads|Madison County|FL|30534|United States|-5|condo| +22594|AAAAAAAACEIFAAAA|66|Sixth Lake|Cir.|Suite C|Belmont|Albany County|NY|10191|United States|-5|condo| +22595|AAAAAAAADEIFAAAA|296|Johnson |Ave|Suite M|Salem|Clay County|AL|38048|United States|-6|single family| +22596|AAAAAAAAEEIFAAAA|411|12th 8th|Blvd|Suite N|Shady Grove|Pratt County|KS|62812|United States|-6|apartment| +22597|AAAAAAAAFEIFAAAA|812|Lincoln |Road|Suite 260|Derby|Robeson County|NC|27702|United States|-5|single family| +22598|AAAAAAAAGEIFAAAA|961|Lee |Drive|Suite 40|Providence|Amherst County|VA|26614|United States|-5|apartment| +22599|AAAAAAAAHEIFAAAA|879|8th |RD|Suite G|Silver Creek|Sequoyah County|OK|74546|United States|-6|single family| +22600|AAAAAAAAIEIFAAAA|283|Third Spruce|ST|Suite U|Maple Grove|Harlan County|KY|48252|United States|-6|apartment| +22601|AAAAAAAAJEIFAAAA|362|Spring |Pkwy|Suite 40|Highland Park|Orange County|CA|96534|United States|-8|condo| +22602|AAAAAAAAKEIFAAAA|752|Laurel |Court|Suite Q|California|Lawrence County|MS|50141|United States|-6|apartment| +22603|AAAAAAAALEIFAAAA|435|Elm |Ln|Suite S|Antioch|Gooding County|ID|88605|United States|-7|single family| +22604|AAAAAAAAMEIFAAAA|694|11th |Way|Suite N|Mountain View|Mendocino County|CA|94466|United States|-8|single family| +22605|AAAAAAAANEIFAAAA|75|Washington River|Cir.|Suite B|Centerville|Campbell County|TN|30059|United States|-5|single family| +22606|AAAAAAAAOEIFAAAA|290|3rd Maple|RD|Suite G|Newtown|Bertie County|NC|21749|United States|-5|condo| +22607|AAAAAAAAPEIFAAAA|400|River Adams|Circle|Suite 430|Harmony|Ashland County|WI|55804|United States|-6|condo| +22608|AAAAAAAAAFIFAAAA|421|11th |Pkwy|Suite C|Woodville|Hodgeman County|KS|64289|United States|-6|apartment| +22609|AAAAAAAABFIFAAAA|110|Spruce |Drive|Suite K|Arlington|Ohio County|IN|46557|United States|-5|apartment| +22610|AAAAAAAACFIFAAAA|593|Elm 4th|Dr.|Suite 440|Shady Grove|Black Hawk County|IA|52812|United States|-6|single family| +22611|AAAAAAAADFIFAAAA|665|Main |Ave|Suite 240|Macedonia|Bexar County|TX|71087|United States|-6|apartment| +22612|AAAAAAAAEFIFAAAA|588|12th |Circle|Suite A|Deerfield|Tyler County|TX|79840|United States|-6|single family| +22613|AAAAAAAAFFIFAAAA|995|Main |Court|Suite P|Williamsville|Montcalm County|MI|48754|United States|-5|apartment| +22614|AAAAAAAAGFIFAAAA|7|Woodland |Ave|Suite 320|Mount Pleasant|Morrow County|OH|41933|United States|-5|single family| +22615|AAAAAAAAHFIFAAAA|849|3rd |Ln|Suite 230|Highland Park|Lenoir County|NC|26534|United States|-5|apartment| +22616|AAAAAAAAIFIFAAAA|190|Park |Street|Suite 0|Newport|Guadalupe County|NM|81521|United States|-7|single family| +22617|AAAAAAAAJFIFAAAA|81|North |Way|Suite D|Providence|Chemung County|NY|16614|United States|-5|single family| +22618|AAAAAAAAKFIFAAAA|649|Eigth Meadow|Boulevard|Suite H|Oakdale|Trinity County|CA|99584|United States|-8|single family| +22619|AAAAAAAALFIFAAAA||13th |RD||||TX||United States||single family| +22620|AAAAAAAAMFIFAAAA|20|Spruce Locust|Circle|Suite T|Antioch|Preble County|OH|48605|United States|-5|single family| +22621|AAAAAAAANFIFAAAA|209|Sunset |Pkwy|Suite 320|Lakeside|Holmes County|OH|49532|United States|-5|condo| +22622|AAAAAAAAOFIFAAAA|4|Walnut Walnut|Court|Suite 0|Salem|Milwaukee County|WI|58048|United States|-6|apartment| +22623|AAAAAAAAPFIFAAAA|898|6th |Pkwy|Suite Y|Greenwood|Butts County|GA|38828|United States|-5|apartment| +22624|AAAAAAAAAGIFAAAA|531|2nd |Drive|Suite 410|Burns|Marshall County|IN|45272|United States|-5|apartment| +22625|AAAAAAAABGIFAAAA|864|Lake |Ct.|Suite 370|Blaine|Caldwell County|MO|65497|United States|-6|apartment| +22626|AAAAAAAACGIFAAAA|708|Laurel |Way|Suite H|Buena Vista|La Salle County|IL|65752|United States|-6|single family| +22627|AAAAAAAADGIFAAAA|615|College |Cir.|Suite 270|Five Forks|Sonoma County|CA|92293|United States|-8|single family| +22628|AAAAAAAAEGIFAAAA|992|Center Adams|Pkwy|Suite D|Enterprise|Kleberg County|TX|71757|United States|-6|apartment| +22629|AAAAAAAAFGIFAAAA|240|6th |Drive|Suite K|Mount Vernon|Oneida County|WI|58482|United States|-6|condo| +22630|AAAAAAAAGGIFAAAA|714|Maple |Avenue|Suite U|Woodland|Buncombe County|NC|24854|United States|-5|condo| +22631|AAAAAAAAHGIFAAAA|442|Second 8th|Dr.|Suite 230|Jackson|Colfax County|NM|89583|United States|-7|single family| +22632|AAAAAAAAIGIFAAAA|789|Wilson |Parkway|Suite 30|Oakwood|Black Hawk County|IA|50169|United States|-6|single family| +22633|AAAAAAAAJGIFAAAA|505|1st 3rd|Boulevard|Suite S|Perkins|Panola County|TX|71852|United States|-6|apartment| +22634|AAAAAAAAKGIFAAAA|937|Hillcrest Cherry|Dr.|Suite 20|Enterprise|Cass County|IA|51757|United States|-6|apartment| +22635|AAAAAAAALGIFAAAA|710|Sycamore |Wy|Suite 160|Plainview|Arkansas County|AR|73683|United States|-6|single family| +22636|AAAAAAAAMGIFAAAA|||Ave|Suite 490|Newtown||MS|51749|||apartment| +22637|AAAAAAAANGIFAAAA|465|Oak Lee|Parkway|Suite V|Five Points|Clinch County|GA|36098|United States|-5|condo| +22638|AAAAAAAAOGIFAAAA|934|Mill |Drive|Suite W|Wilson|Johnson County|MO|66971|United States|-6|apartment| +22639|AAAAAAAAPGIFAAAA|548|Main Green|Circle|Suite K|Newport|Robertson County|TX|71521|United States|-6|apartment| +22640|AAAAAAAAAHIFAAAA|944|Sycamore |Circle|Suite X|Allentown|Winchester city|VA|21838|United States|-5|condo| +22641|AAAAAAAABHIFAAAA|11|First |Street|Suite B|Hopewell|Montgomery County|TN|30587|United States|-6|condo| +22642|AAAAAAAACHIFAAAA|831|Mill |Ln|Suite 30|Stringtown|Camp County|TX|70162|United States|-6|apartment| +22643|AAAAAAAADHIFAAAA|320|North View|Drive|Suite Y|Forest Hills|Avoyelles Parish|LA|79237|United States|-6|single family| +22644|AAAAAAAAEHIFAAAA|278|Ridge Mill|Way|Suite S|Hopewell|Montgomery County|NC|20587|United States|-5|condo| +22645|AAAAAAAAFHIFAAAA|155|2nd |Ct.|Suite V|Bunker Hill|Antrim County|MI|40150|United States|-5|single family| +22646|AAAAAAAAGHIFAAAA|520|Walnut |Ave|Suite I|Oak Hill|Schoolcraft County|MI|47838|United States|-5|single family| +22647|AAAAAAAAHHIFAAAA|613|11th |Boulevard|Suite 330|Crossroads|Prince George County|VA|20534|United States|-5|single family| +22648|AAAAAAAAIHIFAAAA|946|Park |Pkwy|Suite 160|Red Hill|McKean County|PA|14338|United States|-5|single family| +22649|AAAAAAAAJHIFAAAA|786|Cedar |Boulevard|Suite 40|Woodside|Clallam County|WA|95010|United States|-8|condo| +22650|AAAAAAAAKHIFAAAA|126|Cherry Jackson|Lane|Suite L|Spring Hill|George County|MS|56787|United States|-6|condo| +22651|AAAAAAAALHIFAAAA|510|Pine 14th|Road|Suite M|White Oak|Terry County|TX|76668|United States|-6|condo| +22652|AAAAAAAAMHIFAAAA|||Cir.|Suite 160||Blount County||32812||-6|| +22653|AAAAAAAANHIFAAAA|212|Hickory |Pkwy|Suite 90|Oakland|Adams County|IN|49843|United States|-5|condo| +22654|AAAAAAAAOHIFAAAA|124|Pine |Dr.|Suite R|Oak Ridge|Gunnison County|CO|88371|United States|-7|condo| +22655|AAAAAAAAPHIFAAAA|463|Jackson Forest|Circle|Suite 290|Wildwood|Kanabec County|MN|56871|United States|-6|condo| +22656|AAAAAAAAAIIFAAAA|991|Laurel 2nd|Wy|Suite 70|Cedar Grove|Wabash County|IN|40411|United States|-5|single family| +22657|AAAAAAAABIIFAAAA|133|Pine |Wy|Suite A|Richardson|Uvalde County|TX|77687|United States|-6|single family| +22658|AAAAAAAACIIFAAAA|201|Main 5th|Ave|Suite 30|Greenwood|Richland County|MT|68828|United States|-7|apartment| +22659|AAAAAAAADIIFAAAA|783||Avenue||White Oak|Gosper County|||United States|-6|| +22660|AAAAAAAAEIIFAAAA|683|8th |Blvd|Suite K|Sulphur Springs|Nicollet County|MN|58354|United States|-6|condo| +22661|AAAAAAAAFIIFAAAA|354|Meadow Fourth|Ave|Suite N|Belmont|Gasconade County|MO|60191|United States|-6|condo| +22662|AAAAAAAAGIIFAAAA|623|Lake Davis|Avenue|Suite B|Wilson|Wapello County|IA|56971|United States|-6|single family| +22663|AAAAAAAAHIIFAAAA|508|Maple |Ln|Suite X|Midway|Boone County|KY|41904|United States|-6|single family| +22664|AAAAAAAAIIIFAAAA|66|9th Sycamore|Road|Suite 150|Five Forks|Anoka County|MN|52293|United States|-6|single family| +22665|AAAAAAAAJIIFAAAA|716|Washington |Ct.|Suite 430|Antioch|Jefferson County|WI|58605|United States|-6|single family| +22666|AAAAAAAAKIIFAAAA|942|13th |RD|Suite M|Marion|Marion County|GA|30399|United States|-5|condo| +22667|AAAAAAAALIIFAAAA|401|2nd 11th|Road|Suite T|Indian Village|Scott County|VA|21075|United States|-5|single family| +22668|AAAAAAAAMIIFAAAA|651|Birch View|Parkway|Suite M|Oakdale|Adams County|IA|59584|United States|-6|apartment| +22669|AAAAAAAANIIFAAAA|425|Oak |Street|Suite 290|Liberty|Bradley County|AR|73451|United States|-6|condo| +22670|AAAAAAAAOIIFAAAA|896|8th West|Boulevard|Suite 70|Oakwood|Spalding County|GA|30169|United States|-5|single family| +22671|AAAAAAAAPIIFAAAA|418|Hill |Ct.|Suite 350|Wright|Pender County|NC|22814|United States|-5|single family| +22672|AAAAAAAAAJIFAAAA|67|Walnut |Ct.|Suite A|Antioch|Fisher County|TX|78605|United States|-6|apartment| +22673|AAAAAAAABJIFAAAA|871|West |Blvd|Suite P|Riverview|Bailey County|TX|79003|United States|-6|condo| +22674|AAAAAAAACJIFAAAA|574|Hill Valley|Ave|Suite Y|Oak Hill|Whitley County|KY|47838|United States|-5|condo| +22675|AAAAAAAADJIFAAAA|286|Hill |Ct.|Suite A|Sullivan|Chester County|PA|10451|United States|-5|apartment| +22676|AAAAAAAAEJIFAAAA|345|Third Birch|Parkway|Suite 470|Lone Pine|Atkinson County|GA|37441|United States|-5|apartment| +22677|AAAAAAAAFJIFAAAA|597|Elm 1st|Pkwy|Suite G|Crossroads|Candler County|GA|30534|United States|-5|single family| +22678|AAAAAAAAGJIFAAAA|188|Willow |Dr.|Suite F|Mount Olive|Marquette County|MI|48059|United States|-5|apartment| +22679|AAAAAAAAHJIFAAAA|810|Seventh |Ct.|Suite 190|Wilson|White County|IL|66971|United States|-6|single family| +22680|AAAAAAAAIJIFAAAA|442|Ninth |Wy|Suite M|Brownsville|Cache County|UT|89310|United States|-7|condo| +22681|AAAAAAAAJJIFAAAA|204|Park Lee|Pkwy|Suite W|Sutton|Poinsett County|AR|75413|United States|-6|apartment| +22682|AAAAAAAAKJIFAAAA|735|3rd East|Parkway|Suite 150|Cedar Grove|Morton County|KS|60411|United States|-6|apartment| +22683|AAAAAAAALJIFAAAA||Spring Lee|Lane|Suite 90|Jamestown|Pickens County||36867||-6|apartment| +22684|AAAAAAAAMJIFAAAA|97|Maple Jackson|Ct.|Suite 40|Marion|Scott County|VA|20399|United States|-5|condo| +22685|AAAAAAAANJIFAAAA|548|Forest South|Road|Suite I|Cedar Grove|Hanover County|VA|20411|United States|-5|condo| +22686|AAAAAAAAOJIFAAAA|1000|5th |Ln|Suite 410|Pleasant Grove|Morrow County|OR|94136|United States|-8|condo| +22687|AAAAAAAAPJIFAAAA|667|1st |Parkway|Suite T|Green Acres|New London County|CT|08283|United States|-5|apartment| +22688|AAAAAAAAAKIFAAAA|241|East |Avenue|Suite 120|San Jose|Slope County|ND|58003|United States|-6|condo| +22689|AAAAAAAABKIFAAAA|588|East |Pkwy|Suite 0|Wildwood|Lafayette County|MO|66871|United States|-6|single family| +22690|AAAAAAAACKIFAAAA|42|1st |Ct.|Suite 450|Shannon|Gray County|TX|74120|United States|-6|apartment| +22691|AAAAAAAADKIFAAAA|69|6th |Avenue|Suite 160|Edgewood|Eau Claire County|WI|50069|United States|-6|condo| +22692|AAAAAAAAEKIFAAAA|314|Tenth Sunset|Avenue|Suite Y|Belmont|Cass County|MI|40191|United States|-5|apartment| +22693|AAAAAAAAFKIFAAAA|89|6th 10th|Road|Suite 100|Woodland|Stephens County|OK|74854|United States|-6|apartment| +22694|AAAAAAAAGKIFAAAA|458|Smith |Drive|Suite G|Bethel|Forsyth County|GA|35281|United States|-5|condo| +22695|AAAAAAAAHKIFAAAA|133|Poplar Cherry|Street|Suite 170|Valley View|Liberty County|FL|35124|United States|-5|condo| +22696|AAAAAAAAIKIFAAAA|252|7th |ST|Suite 420|Lincoln|Radford city|VA|21289|United States|-5|single family| +22697|AAAAAAAAJKIFAAAA|457|Pine 8th|Blvd|Suite 120|Sheffield|Cameron County|TX|76896|United States|-6|single family| +22698|AAAAAAAAKKIFAAAA|144|Main |Road|Suite 210|Bayview|Caddo County|OK|79672|United States|-6|single family| +22699|AAAAAAAALKIFAAAA|115|North Park|Blvd|Suite 420|Wilson|Douglas County|MO|66971|United States|-6|condo| +22700|AAAAAAAAMKIFAAAA|90|3rd Poplar|Boulevard|Suite 20|Lakeside|Cotton County|OK|79532|United States|-6|apartment| +22701|AAAAAAAANKIFAAAA|628|8th |Boulevard|Suite 90|Walnut Grove|Avery County|NC|27752|United States|-5|apartment| +22702|AAAAAAAAOKIFAAAA|416|Wilson |Street|Suite U|Red Hill|Crittenden County|KY|44338|United States|-6|apartment| +22703|AAAAAAAAPKIFAAAA|181|9th Hickory|Way|Suite 310|Greenwood|Audubon County|IA|58828|United States|-6|apartment| +22704|AAAAAAAAALIFAAAA|745|North |Boulevard|Suite 70|Pine Grove|Wake County|NC|24593|United States|-5|single family| +22705|AAAAAAAABLIFAAAA|870|Park |Pkwy|Suite 190|Five Points|Tyler County|WV|26098|United States|-5|apartment| +22706|AAAAAAAACLIFAAAA|827|Ninth |Wy|Suite M|Mount Olive|Washington County|MO|68059|United States|-6|condo| +22707|AAAAAAAADLIFAAAA|37|Twelfth Lincoln|Parkway|Suite 450|Woodlawn|Claiborne County|MS|54098|United States|-6|single family| +22708|AAAAAAAAELIFAAAA|68|Sunset |Ave|Suite Q|Mount Olive|Weston County|WY|88059|United States|-7|condo| +22709|AAAAAAAAFLIFAAAA|997|Oak Smith|Blvd|Suite 160|Plainview|Greenlee County|AZ|83683|United States|-7|single family| +22710|AAAAAAAAGLIFAAAA|4|Pine |Dr.|Suite G|Norton|Kendall County|TX|73725|United States|-6|apartment| +22711|AAAAAAAAHLIFAAAA|736|South 5th|Avenue|Suite J|Forest Hills|Broadwater County|MT|69237|United States|-7|apartment| +22712|AAAAAAAAILIFAAAA|152|5th Fourth|Avenue|Suite 350|Cedar Grove|Blue Earth County|MN|50411|United States|-6|single family| +22713|AAAAAAAAJLIFAAAA|252|Cherry |Cir.|Suite 20|Marion|Washington County|UT|80399|United States|-7|apartment| +22714|AAAAAAAAKLIFAAAA|591|Spring |Ln|Suite 80|Bridgeport|Brown County|IL|65817|United States|-6|condo| +22715|AAAAAAAALLIFAAAA|546|View |Road|Suite P|Lakeside|Macon County|MO|69532|United States|-6|apartment| +22716|AAAAAAAAMLIFAAAA|447|Walnut |Cir.|Suite 370|Saint James|Kern County|CA|95799|United States|-8|single family| +22717|AAAAAAAANLIFAAAA|385|Jackson |Cir.|Suite 180|Providence|Lincoln County|OR|96614|United States|-8|condo| +22718|AAAAAAAAOLIFAAAA|426|Valley Walnut|Dr.|Suite 0|Mountain View|Clayton County|GA|34466|United States|-5|single family| +22719|AAAAAAAAPLIFAAAA|||Drive|Suite 210|||||United States||apartment| +22720|AAAAAAAAAMIFAAAA|991|Cherry Hill|Circle|Suite U|Harmony|Westmoreland County|VA|25804|United States|-5|single family| +22721|AAAAAAAABMIFAAAA|181|Lincoln Adams|Pkwy|Suite O|Morris|Foard County|TX|76696|United States|-6|single family| +22722|AAAAAAAACMIFAAAA|492|West Green|Pkwy|Suite N|Watkins|Harrison County|MS|51732|United States|-6|apartment| +22723|AAAAAAAADMIFAAAA|775|Main Sunset|Cir.|Suite 280|Highland|Harlan County|NE|69454|United States|-6|single family| +22724|AAAAAAAAEMIFAAAA|841|West 8th|Road|Suite 10|Oakdale|Waller County|TX|79584|United States|-6|single family| +22725|AAAAAAAAFMIFAAAA|830|1st |Ln|Suite B|Plainview|Baker County|OR|93683|United States|-8|condo| +22726|AAAAAAAAGMIFAAAA|951|Wilson |Street|Suite C|Walnut Grove|Shiawassee County|MI|47752|United States|-5|single family| +22727|AAAAAAAAHMIFAAAA|639|Pine Pine|Way|Suite C|Oak Hill|Elk County|KS|67838|United States|-6|single family| +22728|AAAAAAAAIMIFAAAA|643|Johnson |Circle|Suite 280|Oak Grove|McKean County|PA|18370|United States|-5|condo| +22729|AAAAAAAAJMIFAAAA|829|Park Williams|Lane|Suite 320|Friendship|Pocahontas County|WV|24536|United States|-5|condo| +22730|AAAAAAAAKMIFAAAA|920|3rd Laurel|Pkwy|Suite 360|Franklin|Lake County|OH|49101|United States|-5|condo| +22731|AAAAAAAALMIFAAAA|713|Ridge 5th|Parkway|Suite 340|Mount Vernon|Schoharie County|NY|18482|United States|-5|condo| +22732|AAAAAAAAMMIFAAAA|637|View |Circle|Suite 240|Silver Creek|Chariton County|MO|64546|United States|-6|single family| +22733|AAAAAAAANMIFAAAA|165|Valley Lincoln|Lane|Suite B|Florence|Frontier County|NE|63394|United States|-6|condo| +22734|AAAAAAAAOMIFAAAA|993|Tenth |Lane|Suite M|Friendship|Tulare County|CA|94536|United States|-8|apartment| +22735|AAAAAAAAPMIFAAAA|288|||Suite 180||Clark County||||-7|| +22736|AAAAAAAAANIFAAAA|390|2nd |ST|Suite 40|Hamilton|Comanche County|TX|72808|United States|-6|single family| +22737|AAAAAAAABNIFAAAA|554|Highland Sixth|Ave|Suite L|Weldon|Rutherford County|TN|36277|United States|-6|single family| +22738|AAAAAAAACNIFAAAA|108|Ridge Oak|ST|Suite 380|Oak Ridge|Heard County|GA|38371|United States|-5|condo| +22739|AAAAAAAADNIFAAAA|690|View |Ct.|Suite J|Centerville|Frio County|TX|70059|United States|-6|apartment| +22740|AAAAAAAAENIFAAAA|494||Ln|Suite A|||KY||||apartment| +22741|AAAAAAAAFNIFAAAA|3|10th |Ave|Suite 200|White Oak|Socorro County|NM|86668|United States|-7|apartment| +22742|AAAAAAAAGNIFAAAA|112|Railroad |Road|Suite E|Woodland|Presidio County|TX|74854|United States|-6|condo| +22743|AAAAAAAAHNIFAAAA|601|9th |RD|Suite 220|Sulphur Springs|Levy County|FL|38354|United States|-5|condo| +22744|AAAAAAAAINIFAAAA|581|Oak |Wy|Suite 420|Oak Ridge|Buena Vista County|IA|58371|United States|-6|apartment| +22745|AAAAAAAAJNIFAAAA|720|Main |Ave|Suite 220|Huntsville|Tillman County|OK|77570|United States|-6|condo| +22746|AAAAAAAAKNIFAAAA|419|Lincoln |Boulevard|Suite V|Hopewell|Buena Vista County|IA|50587|United States|-6|single family| +22747|AAAAAAAALNIFAAAA|752|West |Lane|Suite 310|Sullivan|Benton County|TN|30451|United States|-5|condo| +22748|AAAAAAAAMNIFAAAA|281|3rd |Street|Suite C|Hamilton|Beauregard Parish|LA|72808|United States|-6|single family| +22749|AAAAAAAANNIFAAAA|415|Elm Maple|Avenue|Suite 310|Newtown|Clearwater County|MN|51749|United States|-6|condo| +22750|AAAAAAAAONIFAAAA|443|Main |Cir.|Suite B|Four Points|Hyde County|SD|51216|United States|-7|apartment| +22751|AAAAAAAAPNIFAAAA|990|Spring Smith|Road|Suite 100|Wilson|Jackson County|FL|36971|United States|-5|condo| +22752|AAAAAAAAAOIFAAAA|190|Lakeview |Blvd|Suite H|Glendale|Clermont County|OH|43951|United States|-5|condo| +22753|AAAAAAAABOIFAAAA|840|Third |Parkway|Suite 120|Bloomingdale|Jackson County|CO|81824|United States|-7|single family| +22754|AAAAAAAACOIFAAAA|213|Third |Dr.|Suite 340|Wildwood|Garza County|TX|76871|United States|-6|apartment| +22755|AAAAAAAADOIFAAAA|317|Third |Court|Suite 170|Mount Pleasant|Forsyth County|NC|21933|United States|-5|condo| +22756|AAAAAAAAEOIFAAAA|577|12th |Ln|Suite 450|Pine Grove|Irwin County|GA|34593|United States|-5|apartment| +22757|AAAAAAAAFOIFAAAA|798|East North|Cir.|Suite 360|Spring Hill|Martin County|IN|46787|United States|-5|apartment| +22758|AAAAAAAAGOIFAAAA|866|Oak |Avenue|Suite 50|Lincoln|Yazoo County|MS|51289|United States|-6|condo| +22759|AAAAAAAAHOIFAAAA|526|Park |Way|Suite C|Florence|Allen County|KS|63394|United States|-6|condo| +22760|AAAAAAAAIOIFAAAA|198|Davis |Court|Suite 350|Hopewell|Crawford County|IL|60587|United States|-6|single family| +22761|AAAAAAAAJOIFAAAA|352|Oak |Circle|Suite 0|Fairview|Covington County|MS|55709|United States|-6|single family| +22762|AAAAAAAAKOIFAAAA|308|Cherry |Avenue|Suite Y|Five Forks|Park County|WY|82293|United States|-7|single family| +22763|AAAAAAAALOIFAAAA|953|1st |Dr.|Suite 180|Franklin|Emmons County|ND|59101|United States|-6|apartment| +22764|AAAAAAAAMOIFAAAA|535|Meadow |Way|Suite 280|Walnut Grove|Douglas County|NV|87752|United States|-8|condo| +22765|AAAAAAAANOIFAAAA|693|Maple 1st|Avenue|Suite N|Guilford|Perkins County|NE|64408|United States|-7|apartment| +22766|AAAAAAAAOOIFAAAA|119|View 5th|Dr.|Suite O|Point Pleasant|Jefferson County|KY|44749|United States|-6|condo| +22767|AAAAAAAAPOIFAAAA|||Road|||Cottonwood County|||United States|-6|single family| +22768|AAAAAAAAAPIFAAAA|19|Seventh |Parkway|Suite 160|Unionville|Coleman County|TX|71711|United States|-6|apartment| +22769|AAAAAAAABPIFAAAA|612|4th 12th|Ln|Suite 430|Hillcrest|Decatur County|GA|33003|United States|-5|condo| +22770|AAAAAAAACPIFAAAA|432|Fifth |Pkwy|Suite X|Sullivan|Marion County|SC|20451|United States|-5|apartment| +22771|AAAAAAAADPIFAAAA|285|10th Washington|Dr.|Suite 370|Colonial Heights|Cumberland County|VA|23425|United States|-5|single family| +22772|AAAAAAAAEPIFAAAA|459|Jefferson Williams|Way|Suite X|Deerfield|Sevier County|AR|79840|United States|-6|apartment| +22773|AAAAAAAAFPIFAAAA|349|Laurel Park|Avenue|Suite F|Mount Olive|Saunders County|NE|68059|United States|-7|apartment| +22774|AAAAAAAAGPIFAAAA|796|Ash Highland|Ct.|Suite 400|Centerville|Piscataquis County|ME|00659|United States|-5|single family| +22775|AAAAAAAAHPIFAAAA|342|Franklin |ST|Suite 130|Bayview|Sanilac County|MI|49672|United States|-5|apartment| +22776|AAAAAAAAIPIFAAAA|988|Washington |Circle|Suite 390|Midway|Columbia County|WI|51904|United States|-6|apartment| +22777|AAAAAAAAJPIFAAAA|501|1st Walnut|Circle|Suite 130|Highland Park|Camp County|TX|76534|United States|-6|single family| +22778|AAAAAAAAKPIFAAAA|453|Lincoln |Ct.|Suite S|Concord|Logan County|AR|74107|United States|-6|condo| +22779|AAAAAAAALPIFAAAA|904|East |ST|Suite 410|Kingston|Howard County|TX|74975|United States|-6|condo| +22780|AAAAAAAAMPIFAAAA|864|4th |Dr.|Suite 490|Oakwood|Renville County|ND|50169|United States|-6|apartment| +22781|AAAAAAAANPIFAAAA|40|Ash Jefferson|ST|Suite S|Riverside|Lake County|IL|69231|United States|-6|apartment| +22782|AAAAAAAAOPIFAAAA|971|Main Valley|Pkwy|Suite 130|Greenville|Chambers County|TX|71387|United States|-6|apartment| +22783|AAAAAAAAPPIFAAAA|987|4th 2nd|Cir.|Suite 320|Five Forks|Columbus County|NC|22293|United States|-5|apartment| +22784|AAAAAAAAAAJFAAAA|976|Pine West|Dr.|Suite L|White Oak|Skagit County|WA|96668|United States|-8|single family| +22785|AAAAAAAABAJFAAAA|612|South |Circle|Suite 300|Union|Eastland County|TX|78721|United States|-6|condo| +22786|AAAAAAAACAJFAAAA|282|River |Drive|Suite 100|Riverdale|Rice County|KS|69391|United States|-6|condo| +22787|AAAAAAAADAJFAAAA|706|Fourth |Cir.|Suite V|Lakeside|Shiawassee County|MI|49532|United States|-5|condo| +22788|AAAAAAAAEAJFAAAA|37|North Jackson|Lane|Suite 250|Newport|Rolette County|ND|51521|United States|-6|single family| +22789|AAAAAAAAFAJFAAAA|848|Birch |ST|Suite 380|Riverview|Coffee County|AL|39003|United States|-6|condo| +22790|AAAAAAAAGAJFAAAA|827|8th Ridge|Avenue|Suite I|Georgetown|Traverse County|MN|57057|United States|-6|condo| +22791|AAAAAAAAHAJFAAAA|57|Hill |||||MT|67838|United States||single family| +22792|AAAAAAAAIAJFAAAA|711|Adams |Dr.|Suite 440|Lakeview|Sherman County|TX|78579|United States|-6|condo| +22793|AAAAAAAAJAJFAAAA|368|Oak |Lane|Suite T|Mount Vernon|Los Alamos County|NM|88482|United States|-7|condo| +22794|AAAAAAAAKAJFAAAA|753|Walnut |Dr.|Suite C|Newtown|Mathews County|VA|21749|United States|-5|single family| +22795|AAAAAAAALAJFAAAA|548|Lincoln Lee|Drive|Suite 240|Bunker Hill|Cheyenne County|NE|60150|United States|-6|single family| +22796|AAAAAAAAMAJFAAAA|923|11th |Court|Suite 490|Shiloh|Rockingham County|VA|29275|United States|-5|condo| +22797|AAAAAAAANAJFAAAA|288|Railroad 9th|Parkway|Suite K|Lebanon|Logan County|OK|72898|United States|-6|condo| +22798|AAAAAAAAOAJFAAAA|119|South |Road|Suite 330|Oak Grove|Martin County|TX|78370|United States|-6|condo| +22799|AAAAAAAAPAJFAAAA|98|Jackson |ST|Suite 290|Wilson|McCurtain County|OK|76971|United States|-6|single family| +22800|AAAAAAAAABJFAAAA|678|Lee Birch|Court|Suite X|Mount Olive|Hancock County|IA|58059|United States|-6|condo| +22801|AAAAAAAABBJFAAAA|155|10th Meadow|ST|Suite P|Five Points|Sabine County|TX|76098|United States|-6|single family| +22802|AAAAAAAACBJFAAAA|874|Hill Spruce|Road|Suite 150|Howell|Hall County|TX|74854|United States|-6|apartment| +22803|AAAAAAAADBJFAAAA|838|Hill Oak|Circle|Suite T|Florence|Louisa County|IA|53394|United States|-6|apartment| +22804|AAAAAAAAEBJFAAAA|855|6th |Road|Suite S|Highland|Santa Clara County|CA|99454|United States|-8|apartment| +22805|AAAAAAAAFBJFAAAA|796|Second |Drive|Suite 460|Shiloh|DeSoto County|FL|39275|United States|-5|single family| +22806|AAAAAAAAGBJFAAAA|969|1st Wilson|Lane|Suite T|Granite|Auglaize County|OH|46284|United States|-5|apartment| +22807|AAAAAAAAHBJFAAAA||14th Lakeview||Suite 160|Georgetown||IN|47057|United States||apartment| +22808|AAAAAAAAIBJFAAAA|254|Ash Chestnut|Wy|Suite 460|Midway|Lake County|OH|41904|United States|-5|single family| +22809|AAAAAAAAJBJFAAAA|585|Seventh |Ln|Suite L|Nottingham|Adams County|NE|64074|United States|-6|apartment| +22810|AAAAAAAAKBJFAAAA|711|1st Railroad|Street|Suite 190|Riverside|Jefferson County|CO|89231|United States|-7|single family| +22811|AAAAAAAALBJFAAAA|683|Smith Madison|Circle|Suite W|Fairview|Saginaw County|MI|45709|United States|-5|single family| +22812|AAAAAAAAMBJFAAAA|606|Pine Broadway|Way|Suite U|Plainview|Caddo Parish|LA|73683|United States|-6|single family| +22813|AAAAAAAANBJFAAAA|778|River North|Ct.|Suite T|Concord|Hardee County|FL|34107|United States|-5|condo| +22814|AAAAAAAAOBJFAAAA|396|Highland |Dr.|Suite 480|Dewey|Spencer County|KY|41160|United States|-5|apartment| +22815|AAAAAAAAPBJFAAAA|958|13th |Ct.|Suite 260|Sullivan|Benton County|WA|90451|United States|-8|condo| +22816|AAAAAAAAACJFAAAA|22|Valley Wilson|Ave|Suite T|Bunker Hill|Issaquena County|MS|50150|United States|-6|condo| +22817|AAAAAAAABCJFAAAA|373|Cherry |Ln|Suite 130|Kingston|Schoolcraft County|MI|44975|United States|-5|condo| +22818|AAAAAAAACCJFAAAA|726|Valley Birch|Pkwy|Suite 50|Lakeside|Mitchell County|IA|59532|United States|-6|single family| +22819|AAAAAAAADCJFAAAA|292|Johnson Church|Ct.|Suite K|Glenwood|Martin County|KY|43511|United States|-5|condo| +22820|AAAAAAAAECJFAAAA|212|North |Way|Suite R|Edgewood|Goshen County|WY|80069|United States|-7|condo| +22821|AAAAAAAAFCJFAAAA|15|Maple |Avenue|Suite 240|Union|Monroe County|OH|48721|United States|-5|single family| +22822|AAAAAAAAGCJFAAAA|714|2nd Laurel|Parkway|Suite 280|Spring Hill|Lincoln County|TN|36787|United States|-6|condo| +22823|AAAAAAAAHCJFAAAA|498|Birch |Parkway|Suite 490|Fairfield|Hillsborough County|FL|36192|United States|-5|condo| +22824|AAAAAAAAICJFAAAA|194|Second Birch|Dr.|Suite 410|Frogtown|Allen Parish|LA|78784|United States|-6|condo| +22825|AAAAAAAAJCJFAAAA|173|Railroad |Dr.|Suite 350|Farmington|Becker County|MN|59145|United States|-6|single family| +22826|AAAAAAAAKCJFAAAA|715|West 2nd|Cir.|Suite T|Liberty|Fentress County|TN|33451|United States|-5|condo| +22827|AAAAAAAALCJFAAAA|208|Lake Ninth|Circle|Suite C|Clifton|Hardeman County|TN|38014|United States|-5|single family| +22828|AAAAAAAAMCJFAAAA|152|Lakeview |Blvd|Suite M|Spring Hill|Hughes County|OK|76787|United States|-6|apartment| +22829|AAAAAAAANCJFAAAA|826|Spruce Lincoln|Drive|Suite 240|Lebanon|Baker County|FL|32898|United States|-5|single family| +22830|AAAAAAAAOCJFAAAA|704|Birch Johnson|Circle|Suite 380|Concord|Fergus County|MT|64107|United States|-7|apartment| +22831|AAAAAAAAPCJFAAAA|159|8th Miller|Lane|Suite 450|Lebanon|Chesapeake city|VA|22898|United States|-5|single family| +22832|AAAAAAAAADJFAAAA|921|Ash Poplar|Parkway|Suite J|Highland|Polk County|TN|39454|United States|-6|single family| +22833|AAAAAAAABDJFAAAA|438|Miller |Cir.|Suite D|Perkins|Graves County|KY|41852|United States|-6|single family| +22834|AAAAAAAACDJFAAAA|538|West |Ct.|Suite 40|Ashland|Nicholas County|WV|24244|United States|-5|single family| +22835|AAAAAAAADDJFAAAA|352|Tenth Madison|Way|Suite 280|Riverview|Whitley County|KY|49003|United States|-5|apartment| +22836|AAAAAAAAEDJFAAAA|250|12th |Circle|Suite 150|Hamilton|Grant County|IN|42808|United States|-5|apartment| +22837|AAAAAAAAFDJFAAAA|50|Johnson |Drive|Suite B|Red Hill|Ector County|TX|74338|United States|-6|condo| +22838|AAAAAAAAGDJFAAAA|100|7th Locust|RD|Suite J|Franklin|Shelby County|KY|49101|United States|-5|single family| +22839|AAAAAAAAHDJFAAAA|||Cir.|||Thomas County|||United States||| +22840|AAAAAAAAIDJFAAAA|884|North Hill|ST|Suite 70|Walnut Grove|Slope County|ND|57752|United States|-6|single family| +22841|AAAAAAAAJDJFAAAA|522|1st |Way|Suite 50|White Oak|Rawlins County|KS|66668|United States|-6|single family| +22842|AAAAAAAAKDJFAAAA|301|Walnut Main|Drive|Suite I|Pine Grove|Tishomingo County|MS|54593|United States|-6|apartment| +22843|AAAAAAAALDJFAAAA|17|Willow 9th|Lane|Suite 210|Jamestown|Ross County|OH|46867|United States|-5|condo| +22844|AAAAAAAAMDJFAAAA|501|3rd |Lane|Suite B|Sunnyside|Cheyenne County|NE|61952|United States|-6|single family| +22845|AAAAAAAANDJFAAAA|165|10th Ninth|Way|Suite 370|Sulphur Springs|Northumberland County|VA|28354|United States|-5|condo| +22846|AAAAAAAAODJFAAAA|132|8th Birch|Court|Suite 430|Green Acres|Hunt County|TX|77683|United States|-6|single family| +22847|AAAAAAAAPDJFAAAA|975|1st |Drive|Suite 80|Plainview|Livingston County|IL|63683|United States|-6|apartment| +22848|AAAAAAAAAEJFAAAA|983|North Wilson|Ln|Suite M|Sumner|Cortland County|NY|10519|United States|-5|condo| +22849|AAAAAAAABEJFAAAA|902|8th |RD|Suite S|Lebanon|Osceola County|MI|42898|United States|-5|condo| +22850|AAAAAAAACEJFAAAA|344|Broadway |Ln|Suite Y|Wilson|Allen County|KY|46971|United States|-6|single family| +22851|AAAAAAAADEJFAAAA|486|Hickory Hill|Court|Suite 80|Whispering Pines|Clay County|MS|57609|United States|-6|single family| +22852|AAAAAAAAEEJFAAAA|887|10th |Drive|Suite I||Montgomery County|||United States|-6|| +22853|AAAAAAAAFEJFAAAA|510|Adams |ST|Suite R|Mount Zion|Tooele County|UT|88054|United States|-7|single family| +22854|AAAAAAAAGEJFAAAA|365|7th Locust|ST|Suite X|Shiloh|Webster County|MO|69275|United States|-6|apartment| +22855|AAAAAAAAHEJFAAAA|813|Dogwood |RD|Suite 120|Lakewood|Somerset County|ME|09477|United States|-5|single family| +22856|AAAAAAAAIEJFAAAA|697|Madison |Ln|Suite 160|Bethel|Buchanan County|IA|55281|United States|-6|apartment| +22857|AAAAAAAAJEJFAAAA||Walnut |Street|Suite V||Garza County||76534|United States||| +22858|AAAAAAAAKEJFAAAA|722|Center Center|Blvd|Suite 120|Plainview|Comal County|TX|73683|United States|-6|condo| +22859|AAAAAAAALEJFAAAA|385|Ridge |Ave|Suite 210|Jackson|Massac County|IL|69583|United States|-6|apartment| +22860|AAAAAAAAMEJFAAAA|688|Willow |Ct.|Suite I|Granite|Flathead County|MT|66284|United States|-7|condo| +22861|AAAAAAAANEJFAAAA|267|Ridge |Avenue|Suite 120|Concord|Lincoln County|MS|54107|United States|-6|apartment| +22862|AAAAAAAAOEJFAAAA|215|River Spring|Wy|Suite G|Unionville|Lexington city|VA|21711|United States|-5|apartment| +22863|AAAAAAAAPEJFAAAA|643|Railroad Hillcrest|Ct.|Suite 350|Farmington|Park County|WY|89145|United States|-7|condo| +22864|AAAAAAAAAFJFAAAA|607|Cedar Highland|Boulevard|Suite M|Farmington|Wexford County|MI|49145|United States|-5|single family| +22865|AAAAAAAABFJFAAAA|991|11th 4th|Pkwy|Suite 30|Easton|Garrett County|MD|25553|United States|-5|condo| +22866|AAAAAAAACFJFAAAA|608|2nd Spring|Dr.|Suite 430|Deerfield|Scott County|IL|69840|United States|-6|condo| +22867|AAAAAAAADFJFAAAA|202|Hickory |Road|Suite 310|Watkins|Fayette County|TX|71732|United States|-6|condo| +22868|AAAAAAAAEFJFAAAA|970|4th |Cir.|Suite 440|Spring Valley|Baraga County|MI|46060|United States|-5|condo| +22869|AAAAAAAAFFJFAAAA|126|6th Sunset|Blvd|Suite D|Green Acres|Morrow County|OH|47683|United States|-5|condo| +22870|AAAAAAAAGFJFAAAA|430|Park |Boulevard|Suite G|Jerome|Kewaunee County|WI|59920|United States|-6|apartment| +22871|AAAAAAAAHFJFAAAA|300|Pine Maple|Ct.|Suite Y|Salem|Smith County|KS|68048|United States|-6|condo| +22872|AAAAAAAAIFJFAAAA|370|14th First|Street|Suite M|Fairfield|Gentry County|MO|66192|United States|-6|condo| +22873|AAAAAAAAJFJFAAAA|723|Smith Lee|Lane|Suite F|Georgetown|Faribault County|MN|57057|United States|-6|apartment| +22874|AAAAAAAAKFJFAAAA|682|Willow |RD|Suite F|Kingston|Jefferson Parish|LA|74975|United States|-6|apartment| +22875|AAAAAAAALFJFAAAA|343|Main West|Ln|Suite M|Crossroads|Fayette County|KY|40534|United States|-6|apartment| +22876|AAAAAAAAMFJFAAAA|838|9th Forest|Avenue|Suite Q|Mount Olive|Lancaster County|SC|28059|United States|-5|single family| +22877|AAAAAAAANFJFAAAA|201|6th Sycamore|Boulevard|Suite 40|Glendale|Montgomery County|MS|53951|United States|-6|single family| +22878|AAAAAAAAOFJFAAAA|336|Washington Hill|ST|Suite 30|Saint John|Fairbanks North Star Borough|AK|95392|United States|-9|single family| +22879|AAAAAAAAPFJFAAAA|667|Fourth 3rd|ST|Suite I|Richardson|Bradford County|FL|37687|United States|-5|single family| +22880|AAAAAAAAAGJFAAAA|160|Willow 11th|Dr.|Suite 280|Enterprise|Aiken County|SC|21757|United States|-5|condo| +22881|AAAAAAAABGJFAAAA|552|Lake Park|Cir.|Suite F|Jackson|Wharton County|TX|79583|United States|-6|apartment| +22882|AAAAAAAACGJFAAAA|21|Hill |Court|Suite H|Riverview|Culpeper County|VA|29003|United States|-5|condo| +22883|AAAAAAAADGJFAAAA|843|2nd |Parkway|Suite 270|Newport|Izard County|AR|71521|United States|-6|single family| +22884|AAAAAAAAEGJFAAAA|742|6th Ridge|Blvd|Suite 330|Fowler|Madison County|IL|61083|United States|-6|apartment| +22885|AAAAAAAAFGJFAAAA|462|Seventh Adams|Drive|Suite 40|Pleasant Hill|Brown County|IL|63604|United States|-6|apartment| +22886|AAAAAAAAGGJFAAAA|579|10th |Pkwy|Suite 400|Marion|Somerset County|ME|00999|United States|-5|apartment| +22887|AAAAAAAAHGJFAAAA|995|7th Highland|Court|Suite I|New Hope|Emmet County|MI|49431|United States|-5|single family| +22888|AAAAAAAAIGJFAAAA|166|Madison 9th|Pkwy|Suite 70|Antioch|Douglas County|MO|68605|United States|-6|condo| +22889|AAAAAAAAJGJFAAAA|969|13th |ST|Suite 70|Mount Pleasant|Cabell County|WV|21933|United States|-5|single family| +22890|AAAAAAAAKGJFAAAA|52|Johnson 3rd|Drive|Suite C|Bunker Hill|Cass County|MO|60150|United States|-6|apartment| +22891|AAAAAAAALGJFAAAA|244|Third |Ln|Suite 300|Farmington|Echols County|GA|39145|United States|-5|condo| +22892|AAAAAAAAMGJFAAAA|466|3rd |Lane|Suite 450|Fairfield|Franklin County|IL|66192|United States|-6|condo| +22893|AAAAAAAANGJFAAAA|28|West |RD|Suite R|Woodlawn|Nelson County|ND|54098|United States|-6|condo| +22894|AAAAAAAAOGJFAAAA|229|2nd |Lane|Suite 0|Greenfield|Allen County|KY|45038|United States|-6|single family| +22895|AAAAAAAAPGJFAAAA|430|College |Circle|Suite 180|Deerfield|Moore County|NC|29840|United States|-5|single family| +22896|AAAAAAAAAHJFAAAA|741|Cedar Lake|Blvd|Suite G|Hopewell|Robeson County|NC|20587|United States|-5|single family| +22897|AAAAAAAABHJFAAAA|659|Lincoln Maple|Parkway|Suite 480|Greenwood|El Dorado County|CA|98828|United States|-8|single family| +22898|AAAAAAAACHJFAAAA|275|Fifth Fifth|RD|Suite G|Plainview|Carroll County|TN|33683|United States|-5|condo| +22899|AAAAAAAADHJFAAAA|932|Main Jackson|Court|Suite 330|Rosewood|Keya Paha County|NE|62205|United States|-7|condo| +22900|AAAAAAAAEHJFAAAA|674|River |Blvd|Suite Q|Friendship|Uinta County|WY|84536|United States|-7|single family| +22901|AAAAAAAAFHJFAAAA|719|Hill |Road|Suite R|Providence|Pike County|IL|66614|United States|-6|condo| +22902|AAAAAAAAGHJFAAAA|||Way|Suite Q|Oak Grove|Parke County||48370|United States|-5|apartment| +22903|AAAAAAAAHHJFAAAA|206|Mill 3rd|Street|Suite H|Shiloh|Marion County|AR|79275|United States|-6|single family| +22904|AAAAAAAAIHJFAAAA|754|Church Park|Cir.|Suite M|White Oak|Dunklin County|MO|66668|United States|-6|condo| +22905|AAAAAAAAJHJFAAAA|28|Jackson 2nd|Dr.|Suite U|Woodville|DeKalb County|AL|34289|United States|-6|condo| +22906|AAAAAAAAKHJFAAAA|784|Second View|Ln|Suite K|Mountain View|Grant County|MN|54466|United States|-6|condo| +22907|AAAAAAAALHJFAAAA|971|6th |ST|Suite 280|Greenville|Gilmer County|GA|31387|United States|-5|single family| +22908|AAAAAAAAMHJFAAAA|123|Adams |Wy|Suite G|Pine Grove|Putnam County|NY|14593|United States|-5|single family| +22909|AAAAAAAANHJFAAAA|28|Forest |Wy|Suite 120|Welcome|Mississippi County|AR|76386|United States|-6|apartment| +22910|AAAAAAAAOHJFAAAA|766|Hickory |Drive|Suite W|Amherst|Jackson County|GA|38119|United States|-5|single family| +22911|AAAAAAAAPHJFAAAA|75|Second Cherry|Wy|Suite 360|Thompsonville|Adams County|ND|59651|United States|-6|single family| +22912|AAAAAAAAAIJFAAAA|482|Second College|Circle|Suite 50|Brownsville|Page County|VA|29310|United States|-5|apartment| +22913|AAAAAAAABIJFAAAA|223|3rd |Blvd|Suite D|Montague|Union County|NC|24062|United States|-5|apartment| +22914|AAAAAAAACIJFAAAA|548|14th 5th|RD|Suite 400|Marion|Iron County|WI|50399|United States|-6|apartment| +22915|AAAAAAAADIJFAAAA|872|Park North|Blvd|Suite 180|Woodland|Newton County|AR|74854|United States|-6|single family| +22916|AAAAAAAAEIJFAAAA|700||||Guilford|Lucas County||54408|||single family| +22917|AAAAAAAAFIJFAAAA|984|Ridge Dogwood|Court|Suite Y|Fowler|Elbert County|GA|31083|United States|-5|condo| +22918|AAAAAAAAGIJFAAAA|885|Sixth |Dr.|Suite 300|Martinsville|Clay County|NE|60419|United States|-6|condo| +22919|AAAAAAAAHIJFAAAA|869|Sixth |Circle|Suite 310|Union|Morton County|ND|58721|United States|-6|apartment| +22920|AAAAAAAAIIJFAAAA|123|Park |Dr.|Suite 490|Springfield|White County|TN|39303|United States|-6|condo| +22921|AAAAAAAAJIJFAAAA|158|9th 12th|Ave|Suite W|Maple Grove|Saline County|MO|68252|United States|-6|condo| +22922|AAAAAAAAKIJFAAAA|673|Ash |Street|Suite 430|Shady Grove|Schley County|GA|32812|United States|-5|single family| +22923|AAAAAAAALIJFAAAA|||Blvd|Suite U|Brunswick|Wichita County|KS|||-6|| +22924|AAAAAAAAMIJFAAAA|351|Hill |Way|Suite 50|Providence|Lewis and Clark County|MT|66614|United States|-7|condo| +22925|AAAAAAAANIJFAAAA|961|8th Ridge|Ln|Suite 440|Mountain View|Hidalgo County|NM|84466|United States|-7|condo| +22926|AAAAAAAAOIJFAAAA|563|3rd |ST|Suite 0|Edgewood|Lynn County|TX|70069|United States|-6|apartment| +22927|AAAAAAAAPIJFAAAA|427|Walnut |Pkwy|Suite I|Midway|Bartow County|GA|31904|United States|-5|condo| +22928|AAAAAAAAAJJFAAAA|447|North |Parkway|Suite O|Liberty|Wyandot County|OH|43451|United States|-5|apartment| +22929|AAAAAAAABJJFAAAA|872|Sixth |Road|Suite J|Shiloh|Jackson County|MI|49275|United States|-5|single family| +22930|AAAAAAAACJJFAAAA|855|Jackson |Cir.|Suite 190|Marion|Mitchell County|KS|60399|United States|-6|apartment| +22931|AAAAAAAADJJFAAAA|798|Sycamore 4th|Ln|Suite M|Farmington|Natrona County|WY|89145|United States|-7|condo| +22932|AAAAAAAAEJJFAAAA|634|Hillcrest |RD|Suite 210|Harmon|San Diego County|CA|95623|United States|-8|apartment| +22933|AAAAAAAAFJJFAAAA|586|Lake Willow|Dr.|Suite 90|Woodland|Scott County|MN|54854|United States|-6|apartment| +22934|AAAAAAAAGJJFAAAA|37|Main 15th|Wy|Suite B|Riverside|Warren County|NJ|09831|United States|-5|apartment| +22935|AAAAAAAAHJJFAAAA|231|Hillcrest Park|Cir.|Suite 430|Cedar Grove|Hamilton County|FL|30411|United States|-5|apartment| +22936|AAAAAAAAIJJFAAAA|235|Franklin 2nd|Boulevard|||Knox County||61933|United States||single family| +22937|AAAAAAAAJJJFAAAA|277|View Walnut|Avenue|Suite V|Fairview|Spalding County|GA|35709|United States|-5|single family| +22938|AAAAAAAAKJJFAAAA|997|Sixth |Drive|Suite Q|Green Acres|Granite County|MT|67683|United States|-7|single family| +22939|AAAAAAAALJJFAAAA|477|5th Walnut|Cir.|Suite 270|Lakeside|Stafford County|KS|69532|United States|-6|single family| +22940|AAAAAAAAMJJFAAAA|433|Spruce 2nd|Avenue|||Kalkaska County||44593||-5|| +22941|AAAAAAAANJJFAAAA|304|Ridge Pine|Way|Suite 370|Macedonia|Stanley County|SD|51087|United States|-7|apartment| +22942|AAAAAAAAOJJFAAAA|782|6th |Blvd|Suite 230|Riverside|Story County|IA|59231|United States|-6|apartment| +22943|AAAAAAAAPJJFAAAA|765|Spring |Drive|Suite A|Georgetown|Nevada County|CA|97057|United States|-8|condo| +22944|AAAAAAAAAKJFAAAA|254|13th |Drive|Suite 120|Five Forks|Gaston County|NC|22293|United States|-5|apartment| +22945|AAAAAAAABKJFAAAA|163|Cedar |Blvd|Suite X|Florence|Cottle County|TX|73394|United States|-6|condo| +22946|AAAAAAAACKJFAAAA|238|Sunset Valley|Ct.|Suite E|Wildwood|Pickens County|SC|26871|United States|-5|single family| +22947|AAAAAAAADKJFAAAA|545|Park |||Newtown|Phillips County|||||condo| +22948|AAAAAAAAEKJFAAAA|750|15th |Blvd|Suite I|Midway|Perry County|KY|41904|United States|-5|apartment| +22949|AAAAAAAAFKJFAAAA|533|11th East|Way|Suite 390|Five Forks|McLean County|IL|62293|United States|-6|condo| +22950|AAAAAAAAGKJFAAAA|933|South |ST|Suite 160|Maple Grove|Franklin County|WA|98252|United States|-8|condo| +22951|AAAAAAAAHKJFAAAA|127|Pine Chestnut|ST|Suite 360|Enterprise|Oneida County|ID|81757|United States|-7|condo| +22952|AAAAAAAAIKJFAAAA|397|Highland |Lane|Suite P|Bethel|Johnson County|NE|65281|United States|-7|condo| +22953|AAAAAAAAJKJFAAAA|832|8th Meadow|Blvd|Suite 30|Colfax|Harris County|GA|32565|United States|-5|single family| +22954|AAAAAAAAKKJFAAAA|705|Park Park|RD|Suite 320|Oakland|Custer County|OK|79843|United States|-6|condo| +22955|AAAAAAAALKJFAAAA|918|7th |Ln|Suite 90|Redland|Lake County|IL|66343|United States|-6|single family| +22956|AAAAAAAAMKJFAAAA|706|5th |Wy|Suite 120|Summit|Washington County|AL|30499|United States|-6|single family| +22957|AAAAAAAANKJFAAAA|780|Hill 5th|Pkwy|Suite 80|Dunlap|Scioto County|OH|46497|United States|-5|apartment| +22958|AAAAAAAAOKJFAAAA|91|Hill |Circle|Suite V|Newport|Beltrami County|MN|51521|United States|-6|apartment| +22959|AAAAAAAAPKJFAAAA||||Suite T|||TN|35752|United States|-6|single family| +22960|AAAAAAAAALJFAAAA|151|6th |Ct.|Suite U|Lincoln|Wheatland County|MT|61289|United States|-7|condo| +22961|AAAAAAAABLJFAAAA|364|Lake 5th|Court|Suite 310|Guilford|Hyde County|SD|54408|United States|-7|apartment| +22962|AAAAAAAACLJFAAAA|445|Walnut College|Cir.|Suite C|Pleasant Valley|Trego County|KS|62477|United States|-6|condo| +22963|AAAAAAAADLJFAAAA|467|8th |Street|Suite U|Marion|Montgomery County|AR|70399|United States|-6|apartment| +22964|AAAAAAAAELJFAAAA|986|1st |Drive|Suite 430|Lincoln|McClain County|OK|71289|United States|-6|apartment| +22965|AAAAAAAAFLJFAAAA|138|Elm Cherry|Drive|Suite N|Oak Grove|Door County|WI|58370|United States|-6|single family| +22966|AAAAAAAAGLJFAAAA|993|Oak 3rd|Court|Suite B|Pleasant Hill|Kingsbury County|SD|53604|United States|-7|single family| +22967|AAAAAAAAHLJFAAAA|798|Seventh 5th|Dr.|Suite 490|Shannon|Escambia County|AL|34120|United States|-6|single family| +22968|AAAAAAAAILJFAAAA|280|Washington Hill|Street|Suite 250|Riverview|Meade County|KY|49003|United States|-5|condo| +22969|AAAAAAAAJLJFAAAA|385|Church View|Avenue|Suite 430|Plainview|Albemarle County|VA|23683|United States|-5|apartment| +22970|AAAAAAAAKLJFAAAA|605|Eigth |Blvd|Suite B|Star|Cherokee County|IA|50725|United States|-6|single family| +22971|AAAAAAAALLJFAAAA|632|Lake Center|Boulevard|Suite 100|White Oak|Seneca County|NY|16668|United States|-5|apartment| +22972|AAAAAAAAMLJFAAAA|290|Meadow |Boulevard|Suite Q|Concord|Fairfax city|VA|24107|United States|-5|apartment| +22973|AAAAAAAANLJFAAAA|73|Cedar View|Ln|Suite 430|Sulphur Springs|Texas County|OK|78354|United States|-6|apartment| +22974|AAAAAAAAOLJFAAAA|869|Second |Wy|Suite M|Wilson|Highland County|VA|26971|United States|-5|single family| +22975|AAAAAAAAPLJFAAAA|556|Ridge |Blvd|Suite 160|Brownsville|Wyoming County|WV|29310|United States|-5|apartment| +22976|AAAAAAAAAMJFAAAA|929|Meadow |Avenue|Suite 370|Hopewell|Washington County|NY|10587|United States|-5|condo| +22977|AAAAAAAABMJFAAAA|818|4th |Way|Suite 260|Sulphur Springs|Vermillion County|IN|48354|United States|-5|apartment| +22978|AAAAAAAACMJFAAAA|975|11th Johnson|Cir.|Suite 370|Shiloh|Dutchess County|NY|19275|United States|-5|single family| +22979|AAAAAAAADMJFAAAA|891|1st |Road|Suite 250|Arlington|McCook County|SD|56557|United States|-7|condo| +22980|AAAAAAAAEMJFAAAA|94|View |Dr.|Suite J|Harmony|Defiance County|OH|45804|United States|-5|apartment| +22981|AAAAAAAAFMJFAAAA|198|1st |Court|Suite 10|Union|Montgomery County|MS|58721|United States|-6|single family| +22982|AAAAAAAAGMJFAAAA|99|Main Mill|Way|Suite A|Newtown|Scotland County|MO|61749|United States|-6|single family| +22983|AAAAAAAAHMJFAAAA|62|Lincoln Seventh|Road|Suite O|Ashland|Jefferson County|FL|34244|United States|-5|single family| +22984|AAAAAAAAIMJFAAAA|817|Forest Birch||Suite E|||IN|48222||-5|| +22985|AAAAAAAAJMJFAAAA|424|Woodland |Parkway|Suite 300|Lakeside|Peoria County|IL|69532|United States|-6|condo| +22986|AAAAAAAAKMJFAAAA|273|Chestnut |Boulevard|Suite E|Pleasant Hill|DeSoto County|FL|33604|United States|-5|single family| +22987|AAAAAAAALMJFAAAA|133|Spring |Ave|Suite 200|Mount Olive|Marshall County|IN|48059|United States|-5|single family| +22988|AAAAAAAAMMJFAAAA|949|15th |Parkway|Suite 240|Providence|Windham County|CT|07214|United States|-5|condo| +22989|AAAAAAAANMJFAAAA|691|Hillcrest Spruce|Ave|Suite O|Green Acres|Putnam County|FL|37683|United States|-5|single family| +22990|AAAAAAAAOMJFAAAA|919|Valley 1st|Pkwy|Suite 20|Providence|Washington County|WI|56614|United States|-6|apartment| +22991|AAAAAAAAPMJFAAAA|995|Meadow Park|Parkway|Suite Y|New Hope|Harney County|OR|99431|United States|-8|single family| +22992|AAAAAAAAANJFAAAA|934|Ash |ST|Suite L|Summit|Moultrie County|IL|60499|United States|-6|apartment| +22993|AAAAAAAABNJFAAAA|876|Park |Lane|Suite 370|Franklin|McCreary County|KY|49101|United States|-5|apartment| +22994|AAAAAAAACNJFAAAA|43|Ash 5th|Circle|Suite 130|Union Hill|Edmonson County|KY|47746|United States|-6|single family| +22995|AAAAAAAADNJFAAAA|327|Valley Hill|Pkwy|Suite 380|Greenville|Dauphin County|PA|11387|United States|-5|apartment| +22996|AAAAAAAAENJFAAAA|176|North Lincoln|Lane|Suite 330|Belmont|Buffalo County||50191|United States||| +22997|AAAAAAAAFNJFAAAA|120|Center |Way|Suite Y|Sunnyside|Spalding County|GA|31952|United States|-5|apartment| +22998|AAAAAAAAGNJFAAAA|495|Valley |RD|Suite I|Lakeside|Logan County|AR|79532|United States|-6|condo| +22999|AAAAAAAAHNJFAAAA|430|8th |Boulevard|Suite 100|Newport|Madison County|OH|41521|United States|-5|apartment| +23000|AAAAAAAAINJFAAAA|905|12th |Court|Suite 180|Providence|O-Brien County|IA|56614|United States|-6|single family| +23001|AAAAAAAAJNJFAAAA|125|South |Circle|Suite L|Centerville|Dunn County|ND|50059|United States|-6|single family| +23002|AAAAAAAAKNJFAAAA|201|Maple Hickory|Court|Suite 450|Union Hill|Gratiot County|MI|47746|United States|-5|condo| +23003|AAAAAAAALNJFAAAA|732|1st Smith|Lane|Suite 180|Hopewell|Carroll County|MO|60587|United States|-6|condo| +23004|AAAAAAAAMNJFAAAA|254|Spring Forest|Drive|Suite Q|Crossroads|Buffalo County|SD|50534|United States|-6|apartment| +23005|AAAAAAAANNJFAAAA|270|Third Elm|ST|Suite J|Newport|Hamilton County|KS|61521|United States|-6|condo| +23006|AAAAAAAAONJFAAAA|783|Hillcrest |Street|Suite B|Mount Pleasant|Osborne County|KS|61933|United States|-6|apartment| +23007|AAAAAAAAPNJFAAAA|491|Laurel |Boulevard|Suite 40|Bethel|Milam County|TX|75281|United States|-6|apartment| +23008|AAAAAAAAAOJFAAAA|481|9th Park|RD|Suite 30|Mount Tabor|Potter County|TX|77702|United States|-6|condo| +23009|AAAAAAAABOJFAAAA|785|5th Smith|Cir.|Suite 70|Oak Grove|Perry County|KY|48370|United States|-5|single family| +23010|AAAAAAAACOJFAAAA|469|Dogwood Franklin|Cir.|Suite Q|Clinton|Cuming County|NE|68222|United States|-6|apartment| +23011|AAAAAAAADOJFAAAA|959|14th Johnson|Circle|Suite 340|Lincoln|Tipton County|IN|41289|United States|-5|single family| +23012|AAAAAAAAEOJFAAAA|662|12th Meadow|Drive|Suite K|Brownsville|Charles County|MD|29310|United States|-5|condo| +23013|AAAAAAAAFOJFAAAA|772|Broadway Third|Way|Suite 70|Oak Ridge|Lawrence County|AR|78371|United States|-6|single family| +23014|AAAAAAAAGOJFAAAA|71|East |Pkwy|Suite 340|Riverdale|Yalobusha County|MS|59391|United States|-6|condo| +23015|AAAAAAAAHOJFAAAA|666|Maple |Ln|Suite D|Pleasant Grove|Johnson County|AR|74136|United States|-6|condo| +23016|AAAAAAAAIOJFAAAA|711|Ash |Ln|Suite L|Sulphur Springs|Wahkiakum County|WA|98354|United States|-8|condo| +23017|AAAAAAAAJOJFAAAA|225|9th 10th|Road|Suite 50|Unionville|DeKalb County|MO|61711|United States|-6|apartment| +23018|AAAAAAAAKOJFAAAA|355|Sycamore |Wy|Suite 230|Bethel|Le Sueur County|MN|55281|United States|-6|single family| +23019|AAAAAAAALOJFAAAA|513|5th 4th|ST|Suite A|Buckingham|Alachua County|FL|34092|United States|-5|single family| +23020|AAAAAAAAMOJFAAAA|520|Main 1st|Circle|Suite 110|Shady Grove|Ottawa County|MI|42812|United States|-5|single family| +23021|AAAAAAAANOJFAAAA|622|Cherry Ash|Lane|Suite I|Newport|Fluvanna County|VA|21521|United States|-5|condo| +23022|AAAAAAAAOOJFAAAA|744|Hill Washington|Street|Suite 120|Macedonia|Lee County|IA|51087|United States|-6|apartment| +23023|AAAAAAAAPOJFAAAA|738|1st 3rd|Court|Suite 30|Franklin|Carson City|NV|89101|United States|-8|single family| +23024|AAAAAAAAAPJFAAAA|564|Forest |Lane|Suite X|Oak Ridge|Furnas County|NE|68371|United States|-6|condo| +23025|AAAAAAAABPJFAAAA|337|Pine |Wy|Suite 30|Glenwood|Newberry County|SC|23511|United States|-5|single family| +23026|AAAAAAAACPJFAAAA|807|9th Lincoln|Circle|Suite 130|Pine Grove|Cotton County|OK|74593|United States|-6|apartment| +23027|AAAAAAAADPJFAAAA|754|Hillcrest Madison|Ave|Suite 210|New Hope|Denver County|CO|89431|United States|-7|apartment| +23028|AAAAAAAAEPJFAAAA|159|Highland Cherry|Ln|Suite 140|Forestville|Graham County|KS|63027|United States|-6|apartment| +23029|AAAAAAAAFPJFAAAA|541|7th |Pkwy|Suite 150|Antioch|Queen Anne County|MD|28605|United States|-5|single family| +23030|AAAAAAAAGPJFAAAA|513|Franklin |Parkway|Suite 270|Five Forks|Brewster County|TX|72293|United States|-6|condo| +23031|AAAAAAAAHPJFAAAA|810|Lee |Ln|Suite 270|Clinton|Marion County|OR|98222|United States|-8|apartment| +23032|AAAAAAAAIPJFAAAA|199|First |Dr.|Suite G|Hillcrest|Davie County|NC|23003|United States|-5|apartment| +23033|AAAAAAAAJPJFAAAA|728|Jackson Sycamore|Road|Suite 280|Riverdale|Roberts County|TX|79391|United States|-6|apartment| +23034|AAAAAAAAKPJFAAAA|566|4th 8th|Pkwy|Suite J|Green Acres|Union Parish|LA|77683|United States|-6|single family| +23035|AAAAAAAALPJFAAAA|545|Ash Fourteenth|Cir.|Suite 20|Glendale|Lavaca County|TX|73951|United States|-6|condo| +23036|AAAAAAAAMPJFAAAA|624|Ash River|Circle|Suite 390|Jackson|Teton County|MT|69583|United States|-7|condo| +23037|AAAAAAAANPJFAAAA||Fourth North|Wy||||||||apartment| +23038|AAAAAAAAOPJFAAAA|549|Park |Ct.|Suite 30|Walnut Grove|Chippewa County|MN|57752|United States|-6|single family| +23039|AAAAAAAAPPJFAAAA|214|Green |Street|Suite E|Stringtown|Lewis County|MO|60162|United States|-6|apartment| +23040|AAAAAAAAAAKFAAAA|763|Main |Cir.|Suite O|Bunker Hill|Morton County|KS|60150|United States|-6|condo| +23041|AAAAAAAABAKFAAAA|70|Center Madison|Street|Suite P|Marion|Monroe County|MS|50399|United States|-6|apartment| +23042|AAAAAAAACAKFAAAA|967|3rd |Circle|Suite W|Summit|Switzerland County|IN|40499|United States|-5|apartment| +23043|AAAAAAAADAKFAAAA|44|Washington |Avenue|Suite 410|Louisville|Acadia Parish|LA|74464|United States|-6|apartment| +23044|AAAAAAAAEAKFAAAA|310|View Madison|Parkway|Suite 0|Union City|Anderson County|KS|68087|United States|-6|single family| +23045|AAAAAAAAFAKFAAAA|106|Chestnut First|Blvd|Suite A|Farmington|Lamar County|AL|39145|United States|-6|condo| +23046|AAAAAAAAGAKFAAAA|815|Forest ||Suite 270||||68883|||condo| +23047|AAAAAAAAHAKFAAAA|219|Elm |Dr.|Suite X|Freeport|Tucker County|WV|21844|United States|-5|single family| +23048|AAAAAAAAIAKFAAAA|974|Smith |Dr.|Suite 40|Pine Grove|DeKalb County|MO|64593|United States|-6|condo| +23049|AAAAAAAAJAKFAAAA|10|Ash Dogwood|ST|Suite M|Georgetown|Divide County|ND|57057|United States|-6|condo| +23050|AAAAAAAAKAKFAAAA|736|13th 8th|Ct.|Suite 370|Wildwood|Van Buren County|IA|56871|United States|-6|condo| +23051|AAAAAAAALAKFAAAA|437|Meadow |Road|Suite J|Crossroads|Morehouse Parish|LA|70534|United States|-6|single family| +23052|AAAAAAAAMAKFAAAA|133|Jackson Pine|Court|Suite V|Lone Pine|Mercer County|NJ|08041|United States|-5|single family| +23053|AAAAAAAANAKFAAAA|106|Hill |Way|Suite G|Jackson|Henry County|IL|69583|United States|-6|single family| +23054|AAAAAAAAOAKFAAAA|903||ST||Marion||AR|70399|United States|-6|single family| +23055|AAAAAAAAPAKFAAAA|333|Franklin |Way|Suite N|Lebanon|Bear Lake County|ID|82898|United States|-7|single family| +23056|AAAAAAAAABKFAAAA|459|Broadway Pine|Court|Suite U|Friendship|Cass County|MI|44536|United States|-5|condo| +23057|AAAAAAAABBKFAAAA|608|11th Washington|Lane|Suite 440|Clifford|Boyle County|KY|48164|United States|-6|condo| +23058|AAAAAAAACBKFAAAA|654|2nd |ST|Suite 250|Hamilton|Union County|MS|52808|United States|-6|single family| +23059|AAAAAAAADBKFAAAA|938|Hill 2nd|Ave|Suite 40|Cedar Grove|Hancock County|GA|30411|United States|-5|apartment| +23060|AAAAAAAAEBKFAAAA|287|Dogwood |Road|Suite 330|Clifton|Red Lake County|MN|58014|United States|-6|condo| +23061|AAAAAAAAFBKFAAAA|432|6th |Way|Suite F|Five Points|Monroe County|MI|46098|United States|-5|single family| +23062|AAAAAAAAGBKFAAAA|212|Washington |Avenue|Suite 60|Mount Zion|Madison County|AR|78054|United States|-6|apartment| +23063|AAAAAAAAHBKFAAAA|775|River |RD|Suite G|Mount Pleasant|Amelia County|VA|21933|United States|-5|condo| +23064|AAAAAAAAIBKFAAAA|578|Wilson |Wy|Suite B|Stringtown|Montgomery County|GA|30162|United States|-5|single family| +23065|AAAAAAAAJBKFAAAA|80|5th |Drive|Suite A|Stringtown|Monroe County|NY|10162|United States|-5|single family| +23066|AAAAAAAAKBKFAAAA|211|Oak |Boulevard|Suite H|Lebanon|Benton County|IN|42898|United States|-5|single family| +23067|AAAAAAAALBKFAAAA|301|Cedar |Drive|Suite 10|Summit|Idaho County|ID|80499|United States|-7|condo| +23068|AAAAAAAAMBKFAAAA|87|Adams 11th|Ln|Suite H|Maple Grove|Hancock County|OH|48252|United States|-5|apartment| +23069|AAAAAAAANBKFAAAA|292|Center |Pkwy|Suite 320|Georgetown|Essex County|VA|27057|United States|-5|single family| +23070|AAAAAAAAOBKFAAAA|438|Oak |Lane|Suite G|Oakwood|Anderson County|SC|20169|United States|-5|condo| +23071|AAAAAAAAPBKFAAAA|885|3rd Elm|Pkwy|Suite 310|Farmington|Brule County|SD|59145|United States|-6|single family| +23072|AAAAAAAAACKFAAAA|741|Fourth |Court|Suite D|Greenville|Valencia County|NM|81387|United States|-7|condo| +23073|AAAAAAAABCKFAAAA|554|3rd |Circle|Suite 460|Lakeview|Carter County|KY|48579|United States|-6|apartment| +23074|AAAAAAAACCKFAAAA|292|Broadway |Court|Suite C|Florence|Hill County|TX|73394|United States|-6|condo| +23075|AAAAAAAADCKFAAAA|241|Lake |Street|Suite O|Bethel|Marion County|AR|75281|United States|-6|condo| +23076|AAAAAAAAECKFAAAA|798|Wilson Sixth|Dr.|Suite S|Unionville|Ingham County|MI|41711|United States|-5|single family| +23077|AAAAAAAAFCKFAAAA|235|Jefferson |Drive|Suite J|Red Hill|Walker County|GA|34338|United States|-5|single family| +23078|AAAAAAAAGCKFAAAA|631|8th North|Ave|Suite E|Fairview|Brooke County|WV|25709|United States|-5|single family| +23079|AAAAAAAAHCKFAAAA|271|Walnut |Boulevard|Suite 250|New Hope|Anderson County|TX|79431|United States|-6|single family| +23080|AAAAAAAAICKFAAAA|983|5th |RD|Suite F|Valley View|Johnson County|GA|35124|United States|-5|apartment| +23081|AAAAAAAAJCKFAAAA|295|Park River|Way|Suite K|Clifton|Iberia Parish|LA|78014|United States|-6|single family| +23082|AAAAAAAAKCKFAAAA|314|Chestnut |Pkwy|Suite 340|Clifton|Kiowa County|KS|68014|United States|-6|single family| +23083|AAAAAAAALCKFAAAA|438|Broadway 1st|RD|Suite E|Pleasant Grove|San Benito County|CA|94136|United States|-8|single family| +23084|AAAAAAAAMCKFAAAA|258|Green Meadow|Parkway|Suite 270|Plainview|Hocking County|OH|43683|United States|-5|condo| +23085|AAAAAAAANCKFAAAA|378|Fourth |Avenue|Suite 490|Oakland|Crockett County|TX|79843|United States|-6|condo| +23086|AAAAAAAAOCKFAAAA|906|Ridge Church|Way|Suite H|Bethel|Daviess County|MO|65281|United States|-6|single family| +23087|AAAAAAAAPCKFAAAA|636|Park Spring|Ave|Suite G|Lakewood|Rockingham County|VA|28877|United States|-5|single family| +23088|AAAAAAAAADKFAAAA|765|Lee |Ave|Suite I|Ashland|Scott County|IN|44244|United States|-5|condo| +23089|AAAAAAAABDKFAAAA|602|Fourth Cedar|Blvd|Suite C|Mount Zion|Darlington County|SC|28054|United States|-5|condo| +23090|AAAAAAAACDKFAAAA|594|10th Main|Avenue|Suite W|Kingston|Ouachita County|AR|74975|United States|-6|apartment| +23091|AAAAAAAADDKFAAAA|630|||Suite U|||||||apartment| +23092|AAAAAAAAEDKFAAAA|486|Cherry |Wy|Suite 10|Jamestown|Pulaski County|AR|76867|United States|-6|apartment| +23093|AAAAAAAAFDKFAAAA|878|Park Main|Way|Suite 120|Yorktown|Wheeler County|TX|70732|United States|-6|single family| +23094|AAAAAAAAGDKFAAAA|162|Poplar |Court|Suite 70|Midway|Lincoln County|ID|81904|United States|-7|single family| +23095|AAAAAAAAHDKFAAAA|350|Cherry |Avenue|Suite E|Greenfield|Allen Parish|LA|75038|United States|-6|single family| +23096|AAAAAAAAIDKFAAAA|53|Fourteenth |RD|Suite Y|Pleasant Valley|Anoka County|MN|52477|United States|-6|condo| +23097|AAAAAAAAJDKFAAAA|394|Birch Lake|RD|Suite 0|Pleasant Hill|Beaver County|UT|83604|United States|-7|apartment| +23098|AAAAAAAAKDKFAAAA|896|7th Cherry|Drive|Suite 260|Bethel|Franklin County|OH|45281|United States|-5|condo| +23099|AAAAAAAALDKFAAAA|977|7th Laurel|Cir.|Suite L|Woodville|Ohio County|KY|44289|United States|-5|single family| +23100|AAAAAAAAMDKFAAAA|430|Elm Williams|Avenue|Suite 30|Mount Zion|Bartholomew County|IN|48054|United States|-5|condo| +23101|AAAAAAAANDKFAAAA|918|Meadow |Drive|Suite I|Kingston|Jackson County|MN|54975|United States|-6|condo| +23102|AAAAAAAAODKFAAAA|532|Park 1st|Way|Suite 90|Delmar|Marshall County|SD|53957|United States|-7|condo| +23103|AAAAAAAAPDKFAAAA|358|Oak |Ln|Suite 270|Bunker Hill|Henry County|VA|20150|United States|-5|apartment| +23104|AAAAAAAAAEKFAAAA|6|Fourteenth Maple|Ave|Suite 440|Pleasant Valley|Anderson County|KS|62477|United States|-6|condo| +23105|AAAAAAAABEKFAAAA|227|Lake |Boulevard|Suite 480|||||||| +23106|AAAAAAAACEKFAAAA|31|Fifth |Ct.|Suite A|Cedar Grove|Morgan County|MO|60411|United States|-6|condo| +23107|AAAAAAAADEKFAAAA|490|Maple |Circle|Suite O|Centerville|Pocahontas County|IA|50059|United States|-6|condo| +23108|AAAAAAAAEEKFAAAA||3rd Ridge|Street||Cedar Grove|La Crosse County|WI||||single family| +23109|AAAAAAAAFEKFAAAA|448|Lake 4th|Wy|Suite O|Oak Hill|Pottawatomie County|KS|67838|United States|-6|apartment| +23110|AAAAAAAAGEKFAAAA|143|River |Cir.|Suite C|Valley View|Upshur County|TX|75124|United States|-6|apartment| +23111|AAAAAAAAHEKFAAAA|862|Pine |Ln|Suite X|Flatwoods|Carson City|NV|84212|United States|-8|apartment| +23112|AAAAAAAAIEKFAAAA|232|1st |Avenue|Suite U|Springfield|Ontonagon County|MI|49303|United States|-5|apartment| +23113|AAAAAAAAJEKFAAAA|86|Church Lincoln|Lane|Suite V|Lakewood|Calhoun County|TX|78877|United States|-6|condo| +23114|AAAAAAAAKEKFAAAA|774|North Miller|Lane|Suite D|Riverview|Wilson County|TX|79003|United States|-6|apartment| +23115|AAAAAAAALEKFAAAA|587|Miller |Street|Suite 300|Summit|Warren County|IL|60499|United States|-6|condo| +23116|AAAAAAAAMEKFAAAA|893|Lake |Street|Suite 230|Hartland|Genesee County|NY|16594|United States|-5|single family| +23117|AAAAAAAANEKFAAAA|966|Tenth |Avenue|Suite F|Tipton|Glasscock County|TX|78773|United States|-6|condo| +23118|AAAAAAAAOEKFAAAA|187|Williams |Circle|Suite 390|Spring Hill|Fairfield County|OH|46787|United States|-5|condo| +23119|AAAAAAAAPEKFAAAA|80|3rd Davis|Court|Suite N|Farmington|Cumberland County|VA|29145|United States|-5|single family| +23120|AAAAAAAAAFKFAAAA|68|Miller |Street|Suite S|Sulphur Springs|Dickinson County|IA|58354|United States|-6|apartment| +23121|AAAAAAAABFKFAAAA|199|View Park|Wy|Suite 130|Bridgeport|Wasco County|OR|95817|United States|-8|single family| +23122|AAAAAAAACFKFAAAA|668|Church |Wy|Suite 10|Prosperity|Shelby County|AL|39089|United States|-6|single family| +23123|AAAAAAAADFKFAAAA|290||Street|Suite I|||IA|56192||-6|apartment| +23124|AAAAAAAAEFKFAAAA|538|9th Maple|Lane|Suite V|Rosebud|Upton County|TX|72244|United States|-6|single family| +23125|AAAAAAAAFFKFAAAA|850|Maple Birch|RD|Suite 260|Wildwood|Moore County|TX|76871|United States|-6|single family| +23126|AAAAAAAAGFKFAAAA|747|Main |Pkwy|Suite W|Woodland|Caddo County|OK|74854|United States|-6|condo| +23127|AAAAAAAAHFKFAAAA|147|Oak Wilson|Ln|Suite B|Pomona|Butts County|GA|34153|United States|-5|single family| +23128|AAAAAAAAIFKFAAAA|917|Ridge 2nd|Blvd|Suite Q|Greenwood|Boyle County|KY|48828|United States|-6|single family| +23129|AAAAAAAAJFKFAAAA|543|Sycamore |Avenue|Suite 480|Pine Grove|Howard County|IA|54593|United States|-6|apartment| +23130|AAAAAAAAKFKFAAAA|971|Dogwood Hill|Cir.|Suite 110|Plainview|Bedford city|VA|23683|United States|-5|single family| +23131|AAAAAAAALFKFAAAA|20|Cherry Maple|Circle|Suite I|Pleasant Hill|Clay County|IN|43604|United States|-5|single family| +23132|AAAAAAAAMFKFAAAA|707|Mill |Way|Suite D|Tanglewood|Isle of Wight County|VA|28994|United States|-5|condo| +23133|AAAAAAAANFKFAAAA|485|Fourteenth Lake|Avenue|Suite U|Maple Hill|Hutchinson County|SD|58095|United States|-7|single family| +23134|AAAAAAAAOFKFAAAA|308|Washington Lee|Drive|Suite J|Brookville|Kern County|CA|93524|United States|-8|condo| +23135|AAAAAAAAPFKFAAAA|689|Central |Drive|Suite 270|Florence|Mercer County|ND|53394|United States|-6|apartment| +23136|AAAAAAAAAGKFAAAA|222|Sycamore |Cir.|Suite W|Greenwood|Greensville County|VA|28828|United States|-5|apartment| +23137|AAAAAAAABGKFAAAA|854|3rd |Cir.|Suite 480|Mount Zion|Eagle County|CO|88054|United States|-7|single family| +23138|AAAAAAAACGKFAAAA|705|6th Park|Court|Suite O|Mount Pleasant|Jackson County|IN|41933|United States|-5|apartment| +23139|AAAAAAAADGKFAAAA|984|Highland |Avenue|Suite J|Wildwood|Dodge County|MN|56871|United States|-6|condo| +23140|AAAAAAAAEGKFAAAA|630|6th Church|Parkway|Suite 270|Valley View|Murray County|OK|75124|United States|-6|condo| +23141|AAAAAAAAFGKFAAAA|135|Second Davis|Ct.|Suite P|Woodland|Hart County|KY|44854|United States|-6|condo| +23142|AAAAAAAAGGKFAAAA|764|Madison 3rd|ST|Suite 380|Riverdale|Putnam County|WV|29391|United States|-5|single family| +23143|AAAAAAAAHGKFAAAA|507|7th |Pkwy|Suite V|||KY|48828|||apartment| +23144|AAAAAAAAIGKFAAAA|493|Cedar |Wy|Suite G|Andover|Adams County|ND|51639|United States|-6|condo| +23145|AAAAAAAAJGKFAAAA|830|Sixth |ST|Suite V|Farmington|Tyler County|TX|79145|United States|-6|condo| +23146|AAAAAAAAKGKFAAAA|98|Park West|ST|Suite 410|Crossroads|Montgomery County|MS|50534|United States|-6|condo| +23147|AAAAAAAALGKFAAAA|902|Forest |Boulevard|Suite 50|Belmont|Madison County|MO|60191|United States|-6|condo| +23148|AAAAAAAAMGKFAAAA|7|Spring Meadow|RD|Suite L|Springdale|Assumption Parish|LA|78883|United States|-6|single family| +23149|AAAAAAAANGKFAAAA|590|Dogwood |Blvd|Suite 40|Crossroads|Cumberland County|TN|30534|United States|-5|single family| +23150|AAAAAAAAOGKFAAAA|200|Central Forest|Way|Suite 370|Waterloo|Fairbanks North Star Borough|AK|91675|United States|-9|condo| +23151|AAAAAAAAPGKFAAAA|189|14th |Boulevard|Suite 340|Wilson|Covington County|MS|56971|United States|-6|single family| +23152|AAAAAAAAAHKFAAAA|1000|Park |ST|Suite 30|Spring Hill|Gosper County|NE|66787|United States|-6|apartment| +23153|AAAAAAAABHKFAAAA|874|Jackson |Court|Suite 20|Walnut Grove|Mineral County|CO|87752|United States|-7|condo| +23154|AAAAAAAACHKFAAAA|215|8th |Wy|Suite 310|Fairfield|Blount County|TN|36192|United States|-5|condo| +23155|AAAAAAAADHKFAAAA|624|Sixth |Road|Suite 130|Georgetown|Caldwell County|TX|77057|United States|-6|apartment| +23156|AAAAAAAAEHKFAAAA|151|14th |Ct.|Suite 130|Rocky Point|Beaverhead County|MT|61209|United States|-7|single family| +23157|AAAAAAAAFHKFAAAA|39|Laurel |Lane|Suite C|Mount Olive|Jasper County|IN|48059|United States|-5|condo| +23158|AAAAAAAAGHKFAAAA|748|Cedar Williams|Drive|Suite P|Shiloh|Brooks County|GA|39275|United States|-5|condo| +23159|AAAAAAAAHHKFAAAA|615|4th |Dr.|Suite 470|Riverside|Wayne County|NY|19231|United States|-5|apartment| +23160|AAAAAAAAIHKFAAAA|220|Ridge |Ave|Suite 360|Union|Colorado County|TX|78721|United States|-6|single family| +23161|AAAAAAAAJHKFAAAA|488|3rd |Dr.|Suite J|Salem|Somerset County|NJ|08648|United States|-5|single family| +23162|AAAAAAAAKHKFAAAA|668|Davis Seventh|Pkwy|Suite X|White Oak|Walton County|GA|36668|United States|-5|apartment| +23163|AAAAAAAALHKFAAAA|483|Smith First|Ave|Suite G|Five Forks|Calhoun County|AL|32293|United States|-6|apartment| +23164|AAAAAAAAMHKFAAAA|286|Sunset |Street|Suite H|Riverview|Jersey County|IL|69003|United States|-6|single family| +23165|AAAAAAAANHKFAAAA|775|Ridge |Drive|Suite N|Lakeside|Madison County|NY|19532|United States|-5|apartment| +23166|AAAAAAAAOHKFAAAA|652|Oak |Road|Suite O|Lebanon|Glades County|FL|32898|United States|-5|single family| +23167|AAAAAAAAPHKFAAAA|730|View |Street|Suite N|Mount Vernon|Pierce County|WI|58482|United States|-6|single family| +23168|AAAAAAAAAIKFAAAA|774|North |Boulevard|Suite 310|Jordan|Craig County|VA|25391|United States|-5|condo| +23169|AAAAAAAABIKFAAAA|458|South Maple|Ln|Suite 140|Stringtown|Echols County|GA|30162|United States|-5|condo| +23170|AAAAAAAACIKFAAAA|88|Elm First|Court|Suite P|Belmont|Hawaii County|HI|90191|United States|-10|apartment| +23171|AAAAAAAADIKFAAAA|87|12th Sunset|RD|Suite 140|Pleasant Grove|Wyoming County|PA|14136|United States|-5|condo| +23172|AAAAAAAAEIKFAAAA|372|4th Washington|Pkwy|Suite 390|Maple Grove|Olmsted County|MN|58252|United States|-6|single family| +23173|AAAAAAAAFIKFAAAA|311|Williams |Way|Suite Y|Riverview|Washington County|AR|79003|United States|-6|apartment| +23174|AAAAAAAAGIKFAAAA|594|Seventh |Boulevard|Suite V|Providence|Hamilton County|TX|76614|United States|-6|apartment| +23175|AAAAAAAAHIKFAAAA|676|River Lake|Boulevard|Suite 170|Pleasant Valley|Talbot County|GA|32477|United States|-5|apartment| +23176|AAAAAAAAIIKFAAAA|2|Park |Blvd|Suite R|Deerfield|Trinity County|TX|79840|United States|-6|apartment| +23177|AAAAAAAAJIKFAAAA|912|Willow |Dr.|Suite 10|Hamilton|Jefferson County|AR|72808|United States|-6|single family| +23178|AAAAAAAAKIKFAAAA|603|Fourth |Ave|Suite 190|Springfield|Hughes County|OK|79303|United States|-6|apartment| +23179|AAAAAAAALIKFAAAA|39|9th |Lane|Suite S|New Hope|Macoupin County|IL|69431|United States|-6|apartment| +23180|AAAAAAAAMIKFAAAA|792|Williams |Ct.|Suite U|Florence|Adams County|CO|83394|United States|-7|apartment| +23181|AAAAAAAANIKFAAAA|705|4th |Parkway|Suite 120|Riley|Cottonwood County|MN|51692|United States|-6|apartment| +23182|AAAAAAAAOIKFAAAA|853|College Sunset||||Norfolk city|VA||||| +23183|AAAAAAAAPIKFAAAA|931|Smith First|Ln|Suite M|Valley View|Wexford County|MI|45124|United States|-5|single family| +23184|AAAAAAAAAJKFAAAA|504|14th Sunset|Court|Suite N|Edgewood|Owen County|IN|40069|United States|-5|single family| +23185|AAAAAAAABJKFAAAA|759|11th |ST|Suite H|West End|Anchorage Borough|AK|92210|United States|-9|apartment| +23186|AAAAAAAACJKFAAAA|111|Park Green|Dr.|Suite W|Oak Ridge|Union Parish|LA|78371|United States|-6|apartment| +23187|AAAAAAAADJKFAAAA|14|Poplar 5th|Ln|Suite 130|Summerville|Cloud County|KS|62033|United States|-6|condo| +23188|AAAAAAAAEJKFAAAA|971|Williams 2nd|Circle|Suite P|Kelly|Nash County|NC|22738|United States|-5|apartment| +23189|AAAAAAAAFJKFAAAA|999|3rd |ST|Suite 30|Georgetown|Talbot County|MD|27057|United States|-5|apartment| +23190|AAAAAAAAGJKFAAAA|35|Lake |Circle|Suite 380|Florence|Dickinson County|KS|63394|United States|-6|single family| +23191|AAAAAAAAHJKFAAAA||South |Boulevard||Buckingham||||United States|-6|single family| +23192|AAAAAAAAIJKFAAAA|529|Third |RD|Suite 210|Greenfield|Berks County|PA|15038|United States|-5|condo| +23193|AAAAAAAAJJKFAAAA|376|Main |Street|Suite L|Greenfield|Early County|GA|35038|United States|-5|single family| +23194|AAAAAAAAKJKFAAAA|964|Sunset |Ct.|Suite D|Oakdale|Spink County|SD|59584|United States|-7|single family| +23195|AAAAAAAALJKFAAAA|681|Williams Spring|Boulevard|Suite A|Kingston|Montgomery County|AR|74975|United States|-6|single family| +23196|AAAAAAAAMJKFAAAA|150|4th |Drive|Suite P|Concord|Keweenaw County|MI|44107|United States|-5|condo| +23197|AAAAAAAANJKFAAAA|107|Park Railroad|ST|Suite 390|Five Forks|Linn County|KS|62293|United States|-6|single family| +23198|AAAAAAAAOJKFAAAA|323|Railroad |Ave|Suite 200|Clifton|Nash County|NC|28014|United States|-5|apartment| +23199|AAAAAAAAPJKFAAAA|273|1st |Boulevard|Suite V|Clyde|Grant Parish|LA|78522|United States|-6|single family| +23200|AAAAAAAAAKKFAAAA|268|5th Walnut|Pkwy|Suite 140|Wilson|Logan County|AR|76971|United States|-6|condo| +23201|AAAAAAAABKKFAAAA|221|Lake Franklin|Avenue|Suite 30|Mount Olive|Barton County|MO|68059|United States|-6|condo| +23202|AAAAAAAACKKFAAAA|185|Pine Miller|Ln|Suite B|Highland|Benson County|ND|59454|United States|-6|single family| +23203|AAAAAAAADKKFAAAA|551|||Suite S|||IA|58877|United States||condo| +23204|AAAAAAAAEKKFAAAA|76|4th |Court|Suite 420|Clifton|Nemaha County|NE|68014|United States|-7|single family| +23205|AAAAAAAAFKKFAAAA|486|North |Avenue|Suite 190|Oak Ridge|Miller County|MO|68371|United States|-6|single family| +23206|AAAAAAAAGKKFAAAA|727|Second |Street|Suite P|Centerville|Barber County|KS|60059|United States|-6|single family| +23207|AAAAAAAAHKKFAAAA|258|Railroad |Cir.|Suite 190|Enterprise|Douglas County|GA|31757|United States|-5|apartment| +23208|AAAAAAAAIKKFAAAA|234|Sunset |Dr.|Suite R|Jamestown|Carter County|MO|66867|United States|-6|apartment| +23209|AAAAAAAAJKKFAAAA|454|2nd 3rd|ST|Suite H|Lebanon|Warren County|IA|52898|United States|-6|apartment| +23210|AAAAAAAAKKKFAAAA|515|Williams |Wy|Suite 430|Salem|Dewey County|SD|58048|United States|-6|apartment| +23211|AAAAAAAALKKFAAAA|||Blvd|||Calhoun County|IL||United States|-6|| +23212|AAAAAAAAMKKFAAAA|53|Spring |Blvd|Suite 350|Perkins|Iron County|MO|61852|United States|-6|single family| +23213|AAAAAAAANKKFAAAA|103|Jackson |Ct.|Suite E|Fairfield|Macon County|MO|66192|United States|-6|single family| +23214|AAAAAAAAOKKFAAAA|345|Broadway |Court|Suite 240|Riverview|Richland County|MT|69003|United States|-7|condo| +23215|AAAAAAAAPKKFAAAA|737|Dogwood Willow|Dr.|Suite U|White Oak|Brown County|TX|76668|United States|-6|condo| +23216|AAAAAAAAALKFAAAA|83|Hill 2nd|Road|Suite 100|Green Acres|Van Wert County|OH|47683|United States|-5|condo| +23217|AAAAAAAABLKFAAAA|265|Jefferson Elevnth|Ln|Suite E|Rankin|Archer County|TX|72621|United States|-6|apartment| +23218|AAAAAAAACLKFAAAA||Spruce Locust||Suite 430||Rice County|||United States|-6|| +23219|AAAAAAAADLKFAAAA||Madison Ridge|RD|||Emery County||86971||-7|| +23220|AAAAAAAAELKFAAAA|262|6th |Parkway|Suite 230|Sulphur Springs|Motley County|TX|78354|United States|-6|apartment| +23221|AAAAAAAAFLKFAAAA|810|Lakeview |Court|Suite 490|Jackson|Allen County|OH|49583|United States|-5|apartment| +23222|AAAAAAAAGLKFAAAA|7|Ridge |Ct.|Suite 70|Greenfield|Scott County|IA|55038|United States|-6|condo| +23223|AAAAAAAAHLKFAAAA|298|Oak West|Drive|Suite P|Greenwood|Dyer County|TN|38828|United States|-5|apartment| +23224|AAAAAAAAILKFAAAA|419|North |Avenue|Suite C|Mountain View|Nicholas County|KY|44466|United States|-5|condo| +23225|AAAAAAAAJLKFAAAA|412|Meadow |Blvd|Suite 260|Deerfield|McMinn County|TN|39840|United States|-6|apartment| +23226|AAAAAAAAKLKFAAAA|556|Second Forest|Blvd|Suite J|Walnut Grove|Clare County|MI|47752|United States|-5|condo| +23227|AAAAAAAALLKFAAAA|420|Third |Wy|Suite 250|Farmington|Clay County|TX|79145|United States|-6|apartment| +23228|AAAAAAAAMLKFAAAA|441|Lee |ST|Suite K||Bowman County|||||| +23229|AAAAAAAANLKFAAAA|991|2nd Woodland|Circle|Suite L|Union|Willacy County|TX|78721|United States|-6|single family| +23230|AAAAAAAAOLKFAAAA|627|Valley Main|Blvd|Suite F|Five Forks|Orange County|TX|72293|United States|-6|apartment| +23231|AAAAAAAAPLKFAAAA|828|Hickory |Blvd|Suite U|Lincoln|Hickman County|KY|41289|United States|-6|apartment| +23232|AAAAAAAAAMKFAAAA|41|Sycamore |Pkwy|Suite 380|Sulphur Springs|Okanogan County|WA|98354|United States|-8|single family| +23233|AAAAAAAABMKFAAAA|639|11th 10th|Dr.|Suite 240|Brownsville|Jefferson County|TN|39310|United States|-6|condo| +23234|AAAAAAAACMKFAAAA|96|Elevnth |Ln|Suite 390|Calhoun|Carroll County|MD|26909|United States|-5|condo| +23235|AAAAAAAADMKFAAAA|795|Lake |Drive|Suite 140|Shiloh|Waukesha County|WI|59275|United States|-6|single family| +23236|AAAAAAAAEMKFAAAA|10|8th Sunset|Avenue|Suite Y|Plainview|Fremont County|IA|53683|United States|-6|single family| +23237|AAAAAAAAFMKFAAAA|152|Birch Hill|Way|Suite 80|Summit|Gibson County|TN|30499|United States|-5|single family| +23238|AAAAAAAAGMKFAAAA|689|Jefferson |ST|Suite U|Walnut Grove|Miami County|IN|47752|United States|-5|apartment| +23239|AAAAAAAAHMKFAAAA|147|Ridge Green|Court|Suite W|Pine Grove|Hamilton County|IN|44593|United States|-5|apartment| +23240|AAAAAAAAIMKFAAAA|491|Fifth Main|Court|Suite 490|Pine Grove|Nicholas County|KY|44593|United States|-5|apartment| +23241|AAAAAAAAJMKFAAAA|766|Hillcrest |Circle|Suite 30|Macedonia|Stephenson County|IL|61087|United States|-6|single family| +23242|AAAAAAAAKMKFAAAA|447|Highland Chestnut|Wy|Suite 360|Enterprise|Shelby County|IN|41757|United States|-5|condo| +23243|AAAAAAAALMKFAAAA|842|14th Third|Street|Suite A|Georgetown|Miller County|GA|37057|United States|-5|apartment| +23244|AAAAAAAAMMKFAAAA|734|3rd |Dr.|Suite A|Riverdale|Hickory County|MO|69391|United States|-6|apartment| +23245|AAAAAAAANMKFAAAA|387|2nd 6th|Boulevard|Suite N|Hopewell|Madison County|AL|30587|United States|-6|single family| +23246|AAAAAAAAOMKFAAAA|676|Hill |Street|Suite T|Hardy|Lorain County|OH|45354|United States|-5|apartment| +23247|AAAAAAAAPMKFAAAA|830|Sunset |Street|Suite 220|Mount Vernon|Sequatchie County|TN|38482|United States|-6|condo| +23248|AAAAAAAAANKFAAAA|719|Central Spring|Avenue|Suite A|Greenville|Hidalgo County|NM|81387|United States|-7|single family| +23249|AAAAAAAABNKFAAAA|672|Elm Pine|Ave|Suite 20|Greenfield|Tensas Parish|LA|75038|United States|-6|single family| +23250|AAAAAAAACNKFAAAA|237|Poplar |ST|Suite 320|Bethel|Casey County|KY|45281|United States|-6|single family| +23251|AAAAAAAADNKFAAAA|178|Green Hillcrest|Parkway|Suite G|Lakeside|Sublette County|WY|89532|United States|-7|condo| +23252|AAAAAAAAENKFAAAA|562|View Main|Blvd|Suite 20|Hopewell|Dundy County|NE|60587|United States|-6|condo| +23253|AAAAAAAAFNKFAAAA|672|Lee Park|Court|Suite 370|Unionville|Blue Earth County|MN|51711|United States|-6|condo| +23254|AAAAAAAAGNKFAAAA|495|Lincoln Valley|Pkwy|Suite M|Belmont|Benton County|OR|90191|United States|-8|condo| +23255|AAAAAAAAHNKFAAAA|70|Seventh |Ct.|Suite L|Longwood|Guadalupe County|NM|87021|United States|-7|apartment| +23256|AAAAAAAAINKFAAAA|22|Pine South|Court|Suite 380|Unionville|Dallas County|TX|71711|United States|-6|single family| +23257|AAAAAAAAJNKFAAAA|297|13th 4th|Street|Suite 130|Floyd|Blair County|PA|13235|United States|-5|single family| +23258|AAAAAAAAKNKFAAAA|436|11th Dogwood|Court|Suite G|Five Points|Washington County|IL|66098|United States|-6|condo| +23259|AAAAAAAALNKFAAAA|748|Hillcrest Meadow|RD|Suite B|Red Hill|Ray County|MO|64338|United States|-6|apartment| +23260|AAAAAAAAMNKFAAAA|823|Poplar |Wy|Suite L|Antioch|Charles County|MD|28605|United States|-5|single family| +23261|AAAAAAAANNKFAAAA|773|Jackson |Lane|Suite 470|Edgewood|Belknap County|NH|00669|United States|-5|apartment| +23262|AAAAAAAAONKFAAAA|342|Lake Laurel|Blvd|Suite M|Bethel|Montrose County|CO|85281|United States|-7|apartment| +23263|AAAAAAAAPNKFAAAA|833|Green |Avenue|Suite I|Five Forks|Shelby County|TX|72293|United States|-6|condo| +23264|AAAAAAAAAOKFAAAA|749|Pine 13th|Drive|Suite H|Amity|Weber County|UT|80766|United States|-7|apartment| +23265|AAAAAAAABOKFAAAA|7|Park 2nd|Lane|Suite D|Franklin|Morgan County|WV|29101|United States|-5|apartment| +23266|AAAAAAAACOKFAAAA|906|8th |Cir.|Suite 320|Mount Vernon|Menard County|IL|68482|United States|-6|apartment| +23267|AAAAAAAADOKFAAAA|963|South Second|Lane|Suite L|Wyoming|Montgomery County|IL|60216|United States|-6|apartment| +23268|AAAAAAAAEOKFAAAA|428|Cedar |Road|Suite 440|Ashland|Windham County|VT|04844|United States|-5|condo| +23269|AAAAAAAAFOKFAAAA|973|Elevnth |Circle|Suite 130|Midway|Cumberland County|TN|31904|United States|-5|single family| +23270|AAAAAAAAGOKFAAAA|533|Hill |Circle|Suite 230|Valley View|Niagara County|NY|15124|United States|-5|single family| +23271|AAAAAAAAHOKFAAAA|388|Pine Lincoln|Dr.|Suite 300|Plainview|Pasquotank County|NC|23683|United States|-5|condo| +23272|AAAAAAAAIOKFAAAA|902|Elm |Way|Suite R|Forest Hills|Jackson County|KS|69237|United States|-6|apartment| +23273|AAAAAAAAJOKFAAAA|870|Mill |Street|Suite 460|Webb|Franklin County|MA|01499|United States|-5|apartment| +23274|AAAAAAAAKOKFAAAA|198|Adams 6th|Wy|Suite 300|Jamestown|Barren County|KY|46867|United States|-6|condo| +23275|AAAAAAAALOKFAAAA|835|Park Birch|Pkwy|Suite 80|Sunnyside|Carroll County|MD|21952|United States|-5|condo| +23276|AAAAAAAAMOKFAAAA|328|Williams Poplar|Street|Suite 30|Greenville|Louisa County|IA|51387|United States|-6|condo| +23277|AAAAAAAANOKFAAAA|253|Jackson |Blvd|Suite 470|Mount Olive|Teller County|CO|88059|United States|-7|single family| +23278|AAAAAAAAOOKFAAAA|69|3rd |Road|Suite B|Shady Grove|Kemper County|MS|52812|United States|-6|apartment| +23279|AAAAAAAAPOKFAAAA|152|Cedar 11th|ST|Suite E|Greenfield|Quitman County|MS|55038|United States|-6|apartment| +23280|AAAAAAAAAPKFAAAA|428|Main College|Road|Suite I|Jackson|Van Buren County|IA|59583|United States|-6|condo| +23281|AAAAAAAABPKFAAAA|33|2nd |Ave|Suite 240|Newtown|Crittenden County|KY|41749|United States|-6|condo| +23282|AAAAAAAACPKFAAAA|363|Park Poplar|Court|Suite D|Salem|Matagorda County|TX|78048|United States|-6|condo| +23283|AAAAAAAADPKFAAAA|883|13th |Dr.|Suite 70|Mount Pleasant|Jasper County|TX|71933|United States|-6|single family| +23284|AAAAAAAAEPKFAAAA|39|9th Woodland|Boulevard|Suite 390|Walnut Grove|Refugio County|TX|77752|United States|-6|condo| +23285|AAAAAAAAFPKFAAAA|18|6th |Wy|Suite H|Maple Grove|Goshen County|WY|88252|United States|-7|condo| +23286|AAAAAAAAGPKFAAAA|509|Sycamore Maple|Avenue|Suite 320|Summit|Houston County|MN|50499|United States|-6|condo| +23287|AAAAAAAAHPKFAAAA|99|Park |Ct.|Suite Y|Jamestown|Jefferson County|KS|66867|United States|-6|apartment| +23288|AAAAAAAAIPKFAAAA|52|Franklin |Way|Suite J|Liberty|Alexander County|IL|63451|United States|-6|condo| +23289|AAAAAAAAJPKFAAAA|848|Smith Ridge|Circle|Suite 60|Glendale|Harding County|NM|83951|United States|-7|condo| +23290|AAAAAAAAKPKFAAAA|776|West |Avenue|Suite E|Pleasant Hill|Pontotoc County|OK|73604|United States|-6|condo| +23291|AAAAAAAALPKFAAAA|51|Washington |Dr.|Suite 30|Centerville|Chatham County|GA|30059|United States|-5|condo| +23292|AAAAAAAAMPKFAAAA|536|3rd |Blvd|Suite R|Jamestown|Jefferson County|MO|66867|United States|-6|condo| +23293|AAAAAAAANPKFAAAA|474|Main 4th|Ln|Suite B|Green Acres|Choctaw County|MS|57683|United States|-6|apartment| +23294|AAAAAAAAOPKFAAAA|701|Jackson |Cir.|Suite 190|Arlington|Steele County|MN|56557|United States|-6|single family| +23295|AAAAAAAAPPKFAAAA|558|Jackson |Street|Suite W|Newport|Pulaski County|IN|41521|United States|-5|apartment| +23296|AAAAAAAAAALFAAAA|836|Broadway Lincoln|Road|Suite M|Brookville|Thurston County|NE|63524|United States|-7|condo| +23297|AAAAAAAABALFAAAA|597|Locust |Street|Suite 120|Rosebud|Victoria County|TX|72244|United States|-6|apartment| +23298|AAAAAAAACALFAAAA|161|Cedar |Parkway|Suite Q|Valley View|Raleigh County|WV|25124|United States|-5|single family| +23299|AAAAAAAADALFAAAA|594|Park 14th|Road|Suite Y|Enterprise|Gordon County|GA|31757|United States|-5|single family| +23300|AAAAAAAAEALFAAAA|474|Cherry Williams|RD|Suite U|Greenwood|Washington County|PA|18828|United States|-5|condo| +23301|AAAAAAAAFALFAAAA|920|5th |Ct.|Suite 160|Georgetown|Teton County|WY|87057|United States|-7|apartment| +23302|AAAAAAAAGALFAAAA|294|Hickory |Street|Suite 20|Edgewood|Graham County|KS|60069|United States|-6|condo| +23303|AAAAAAAAHALFAAAA|156|Lee |Way|Suite 390|Lawrenceville|Barren County|KY|44462|United States|-6|single family| +23304|AAAAAAAAIALFAAAA|807|Hill |Circle|Suite E|Union|Campbell County|TN|38721|United States|-5|apartment| +23305|AAAAAAAAJALFAAAA|376|Cedar Lake|Ct.|Suite 320|Glendale|Henry County|VA|23951|United States|-5|single family| +23306|AAAAAAAAKALFAAAA|552|13th |Dr.|Suite I|Bethel|Williamson County|TX|75281|United States|-6|single family| +23307|AAAAAAAALALFAAAA|648|Central |Lane|Suite U|Highland|Mitchell County|TX|79454|United States|-6|condo| +23308|AAAAAAAAMALFAAAA|663|Dogwood Church|RD|Suite 240|Green Acres|Palm Beach County|FL|37683|United States|-5|single family| +23309|AAAAAAAANALFAAAA|484|Washington Third|ST|Suite 70|Greenfield|Baxter County|AR|75038|United States|-6|single family| +23310|AAAAAAAAOALFAAAA|300|Tenth 3rd|Road|Suite 130|Edgewood|Meriwether County|GA|30069|United States|-5|single family| +23311|AAAAAAAAPALFAAAA|444|Second Forest|ST|Suite U|Bethel|Douglas County|IL|65281|United States|-6|single family| +23312|AAAAAAAAABLFAAAA|714|Main |Wy|Suite 480|Five Points|Mineral County|MT|66098|United States|-7|condo| +23313|AAAAAAAABBLFAAAA|570|North 5th|Road|Suite 0|Oakwood|Moultrie County|IL|60169|United States|-6|single family| +23314|AAAAAAAACBLFAAAA|846|North Davis|Cir.|Suite 40|Pleasant Valley|Lanier County|GA|32477|United States|-5|single family| +23315|AAAAAAAADBLFAAAA|914|Central |Boulevard|Suite 0|Bridgeport|Warren County|MS|55817|United States|-6|apartment| +23316|AAAAAAAAEBLFAAAA|174|Center Park|Circle|Suite 110|Red Hill|Kit Carson County|CO|84338|United States|-7|apartment| +23317|AAAAAAAAFBLFAAAA|290|Pine Franklin|Parkway|Suite 470|Freeport|Madison County|AR|71844|United States|-6|condo| +23318|AAAAAAAAGBLFAAAA|780|Seventh |Wy|Suite 120|Crossroads|Chaves County|NM|80534|United States|-7|condo| +23319|AAAAAAAAHBLFAAAA|136|West Birch|RD|Suite 160|Woodland Park|Allegan County|MI|41934|United States|-5|apartment| +23320|AAAAAAAAIBLFAAAA|||Ct.||Kimball|Lake and Peninsula Borough|AK|93595|||| +23321|AAAAAAAAJBLFAAAA|459|Spring |Dr.|Suite P|Deerfield|Grayson County|VA|29840|United States|-5|single family| +23322|AAAAAAAAKBLFAAAA|603|Center |Way|Suite C|Sulphur Springs|Shasta County|CA|98354|United States|-8|apartment| +23323|AAAAAAAALBLFAAAA|651|Third Elm|Avenue|Suite 250|Hamilton|Lampasas County|TX|72808|United States|-6|single family| +23324|AAAAAAAAMBLFAAAA|683|Fifth Adams|Boulevard|Suite 120|Florence|Osborne County|KS|63394|United States|-6|condo| +23325|AAAAAAAANBLFAAAA|95|Railroad Valley|Cir.|Suite 350|Providence|Pend Oreille County|WA|96614|United States|-8|apartment| +23326|AAAAAAAAOBLFAAAA|45|Dogwood Oak|Ln|Suite E|Newtown|New Haven County|CT|02349|United States|-5|single family| +23327|AAAAAAAAPBLFAAAA|834|Cedar Jefferson|RD|Suite 10|Newtown|Marion County|GA|31749|United States|-5|condo| +23328|AAAAAAAAACLFAAAA|379|Davis Broadway|Court|Suite B|Spring Hill|Robertson County|TN|36787|United States|-6|condo| +23329|AAAAAAAABCLFAAAA|67|West Woodland|ST|Suite W|Mount Vernon|Otero County|NM|88482|United States|-7|condo| +23330|AAAAAAAACCLFAAAA|633|3rd 8th|Cir.|Suite E|Hamilton|Johnston County|NC|22808|United States|-5|single family| +23331|AAAAAAAADCLFAAAA|472|8th |Cir.|Suite 90|Plainview|Livingston County|KY|43683|United States|-5|apartment| +23332|AAAAAAAAECLFAAAA|356|Central Spring|Wy|Suite J|Kingston|Berrien County|MI|44975|United States|-5|condo| +23333|AAAAAAAAFCLFAAAA|306|Valley |Ln|Suite 180|Newtown|Renville County|MN|51749|United States|-6|single family| +23334|AAAAAAAAGCLFAAAA|487|Hillcrest Fifth|Court|Suite 110|Wolf Creek|Walton County|FL|32455|United States|-5|single family| +23335|AAAAAAAAHCLFAAAA|785|Third Maple|Street|Suite K|Salem|Albany County|NY|18048|United States|-5|single family| +23336|AAAAAAAAICLFAAAA|459|2nd Hillcrest|Dr.|Suite F|Bunker Hill|Billings County|ND|50150|United States|-6|single family| +23337|AAAAAAAAJCLFAAAA|775|Ridge Lee|Wy|Suite 360|Shaw|Stoddard County|MO|60618|United States|-6|condo| +23338|AAAAAAAAKCLFAAAA|533|Spruce Washington|Dr.|Suite X|Springdale|Clifton Forge city|VA|28883|United States|-5|apartment| +23339|AAAAAAAALCLFAAAA|589|Willow Chestnut|Street|Suite 210|Greenfield|Lincoln County|NV|85038|United States|-8|condo| +23340|AAAAAAAAMCLFAAAA|801|Hill 1st|Cir.|Suite 370|Spring Hill|Nash County|NC|26787|United States|-5|condo| +23341|AAAAAAAANCLFAAAA|200|Johnson |Dr.|Suite 450|Tyler|Renville County|ND|56853|United States|-6|condo| +23342|AAAAAAAAOCLFAAAA|419|College |Pkwy|Suite 10|Forest Hills|Warren County|TN|39237|United States|-6|single family| +23343|AAAAAAAAPCLFAAAA|931|Lakeview Second|Street|Suite 150|Union Hill|Fulton County|GA|37746|United States|-5|condo| +23344|AAAAAAAAADLFAAAA|118|Lincoln Maple|Lane|Suite Y|Deerfield|Bibb County|GA|39840|United States|-5|condo| +23345|AAAAAAAABDLFAAAA|973|Main North|Parkway|Suite 480|Shiloh|Toole County|MT|69275|United States|-7|condo| +23346|AAAAAAAACDLFAAAA|281|Laurel Fourth|Court|Suite K|Newtown|Allegany County|NY|11749|United States|-5|single family| +23347|AAAAAAAADDLFAAAA|503|6th |Way|Suite E|Riverside|Minidoka County|ID|89231|United States|-7|condo| +23348|AAAAAAAAEDLFAAAA|585|Cherry Park|Parkway|Suite J|Five Forks|Yalobusha County|MS|52293|United States|-6|condo| +23349|AAAAAAAAFDLFAAAA|890|4th Maple|Wy|Suite D|Woodville|Hickman County|TN|34289|United States|-5|apartment| +23350|AAAAAAAAGDLFAAAA|654|Railroad 2nd|ST|Suite 30|Woodville|Inyo County|CA|94289|United States|-8|single family| +23351|AAAAAAAAHDLFAAAA|994|||||Wharton County|TX|71387|United States||| +23352|AAAAAAAAIDLFAAAA|613|Woodland Railroad|Avenue|Suite M|Stringtown|Henry County|KY|40162|United States|-6|condo| +23353|AAAAAAAAJDLFAAAA|473|Madison Sycamore|Road|Suite 240|Sullivan|Elmore County|AL|30451|United States|-6|single family| +23354|AAAAAAAAKDLFAAAA|239|||Suite B|||KS|||-6|| +23355|AAAAAAAALDLFAAAA|868|Railroad First|Street|Suite 100|Lakeview|Franklin County|FL|38579|United States|-5|condo| +23356|AAAAAAAAMDLFAAAA|932|Jefferson Lake|Dr.|Suite 480|Centerville|Sanpete County|UT|80059|United States|-7|single family| +23357|AAAAAAAANDLFAAAA|550|8th |Blvd|Suite I|Moore|Colonial Heights city|VA|24614|United States|-5|single family| +23358|AAAAAAAAODLFAAAA|824|Pine |Dr.|Suite V|Spring Valley|Owen County|KY|46060|United States|-5|apartment| +23359|AAAAAAAAPDLFAAAA|700|Highland Willow|Pkwy|Suite U|Greenville|Anoka County|MN|51387|United States|-6|apartment| +23360|AAAAAAAAAELFAAAA|246|Madison Valley|Court|Suite Y|Fairview|Emmet County|MI|45709|United States|-5|single family| +23361|AAAAAAAABELFAAAA|477|Willow |Ave|Suite L|Sunnyside|Hopkins County|TX|71952|United States|-6|single family| +23362|AAAAAAAACELFAAAA|266|Oak Oak|ST|Suite 470|Mount Zion|Stafford County|KS|68054|United States|-6|condo| +23363|AAAAAAAADELFAAAA|838|Ash Hickory|Street|Suite 50|Five Points|Hinsdale County|CO|86098|United States|-7|single family| +23364|AAAAAAAAEELFAAAA|496|Cherry 14th|Avenue|Suite E|Woodlawn|Ripley County|IN|44098|United States|-5|single family| +23365|AAAAAAAAFELFAAAA|995|Cedar |Dr.|Suite 120|Spring Hill|Jewell County|KS|66787|United States|-6|apartment| +23366|AAAAAAAAGELFAAAA|533|2nd |Road|Suite 400|Hillcrest|La Crosse County|WI|53003|United States|-6|apartment| +23367|AAAAAAAAHELFAAAA|578|1st |Blvd|Suite 50|Wilson|Lauderdale County|MS|56971|United States|-6|condo| +23368|AAAAAAAAIELFAAAA|678|4th Franklin|Ln|Suite 410|Mount Pleasant|Dallas County|TX|71933|United States|-6|single family| +23369|AAAAAAAAJELFAAAA|340|Lake |Pkwy|Suite 140|Forest Hills|Buncombe County|NC|29237|United States|-5|condo| +23370|AAAAAAAAKELFAAAA|590|Oak 14th|Blvd|Suite P|Highland|Calhoun County|FL|39454|United States|-5|apartment| +23371|AAAAAAAALELFAAAA|852|Lee |Boulevard|Suite W|Wilson|Scott County|IA|56971|United States|-6|apartment| +23372|AAAAAAAAMELFAAAA|176|Willow Pine|Boulevard|Suite H|Oak Ridge|Mesa County|CO|88371|United States|-7|single family| +23373|AAAAAAAANELFAAAA|5|2nd |Way|Suite E|Shady Grove|Umatilla County|OR|92812|United States|-8|condo| +23374|AAAAAAAAOELFAAAA|92|5th |Blvd|Suite N|Hillcrest|Bond County|IL|63003|United States|-6|apartment| +23375|AAAAAAAAPELFAAAA|630|Adams |Way|Suite A|Five Points|Logan County|KY|46098|United States|-5|apartment| +23376|AAAAAAAAAFLFAAAA|698|Jefferson 2nd|ST|Suite 190|Frenchtown|Worth County|GA|32629|United States|-5|single family| +23377|AAAAAAAABFLFAAAA|504||Way|||Wilkes County|NC|27057|United States||| +23378|AAAAAAAACFLFAAAA|664|Central Sycamore|Ave|Suite 380|Springdale|Pemiscot County|MO|68883|United States|-6|apartment| +23379|AAAAAAAADFLFAAAA|41|7th |Court|Suite 240|Riverside|Tulsa County|OK|79231|United States|-6|single family| +23380|AAAAAAAAEFLFAAAA|867|Franklin |Dr.|Suite Y|Sutton|Polk County|MO|65413|United States|-6|single family| +23381|AAAAAAAAFFLFAAAA|697|Johnson 8th|Boulevard|Suite E|Friendship|San Francisco County|CA|94536|United States|-8|single family| +23382|AAAAAAAAGFLFAAAA|315|Lincoln 2nd|Way|Suite 0|Antioch|Limestone County|AL|38605|United States|-6|single family| +23383|AAAAAAAAHFLFAAAA||||Suite 240|||||United States||apartment| +23384|AAAAAAAAIFLFAAAA|428|Washington |Drive|Suite O|Mountain View|Crockett County|TN|34466|United States|-5|apartment| +23385|AAAAAAAAJFLFAAAA|526|Johnson Main|Blvd|Suite 340|Glenwood|Jewell County|KS|63511|United States|-6|single family| +23386|AAAAAAAAKFLFAAAA|467|River Main|Way|Suite D|Fairfield|Amite County|MS|56192|United States|-6|condo| +23387|AAAAAAAALFLFAAAA|126|Laurel Washington|Drive|Suite 170|Woodlawn|Buncombe County|NC|24098|United States|-5|apartment| +23388|AAAAAAAAMFLFAAAA|76|15th 13th|Parkway|Suite O|Hillcrest|Mathews County|VA|23003|United States|-5|condo| +23389|AAAAAAAANFLFAAAA|730|Green Center|Cir.|Suite E|Greenwood|Garvin County|OK|78828|United States|-6|apartment| +23390|AAAAAAAAOFLFAAAA|142|Pine Locust|Pkwy|Suite X|Bridgeport|Mercer County|WV|25817|United States|-5|apartment| +23391|AAAAAAAAPFLFAAAA|295|Dogwood |ST|Suite 290|Walnut Grove|Bullock County|AL|37752|United States|-6|condo| +23392|AAAAAAAAAGLFAAAA|14|Washington Railroad|Blvd|Suite 300|Ashland|Burleson County|TX|74244|United States|-6|apartment| +23393|AAAAAAAABGLFAAAA|722|1st 3rd|RD|Suite 260|Woodville|Greene County|AR|74289|United States|-6|single family| +23394|AAAAAAAACGLFAAAA|751|Jackson Jefferson|Drive|Suite R|Salem|Plaquemines Parish|LA|78048|United States|-6|apartment| +23395|AAAAAAAADGLFAAAA|505|4th |Cir.|Suite 50|Georgetown|Contra Costa County|CA|97057|United States|-8|apartment| +23396|AAAAAAAAEGLFAAAA|891|Lincoln 3rd|Avenue|Suite 330|Pleasant Hill|Upshur County|TX|73604|United States|-6|apartment| +23397|AAAAAAAAFGLFAAAA|300|Johnson 8th|Blvd|Suite R|Springfield|Cass County|IL|69303|United States|-6|apartment| +23398|AAAAAAAAGGLFAAAA|762|View Park|Street|Suite 10|Shady Grove|Iron County|UT|82812|United States|-7|condo| +23399|AAAAAAAAHGLFAAAA|771|Oak Sycamore|Street|Suite X|Pleasant Grove|La Plata County|CO|84136|United States|-7|apartment| +23400|AAAAAAAAIGLFAAAA|327|Eigth |Way|Suite P|Wilson|Wagoner County|OK|76971|United States|-6|apartment| +23401|AAAAAAAAJGLFAAAA|856|Lake |RD|Suite 490|Pleasant Hill|Coles County|IL|63604|United States|-6|single family| +23402|AAAAAAAAKGLFAAAA|767|Cedar Elm|Boulevard|Suite 80|Centerville|Knox County|KY|40059|United States|-5|single family| +23403|AAAAAAAALGLFAAAA|849|14th |Ave|Suite 220|Sulphur Springs|Howard County|MO|68354|United States|-6|apartment| +23404|AAAAAAAAMGLFAAAA|617|Maple View|ST|Suite 310|Indian Village|Burleigh County|ND|51075|United States|-6|apartment| +23405|AAAAAAAANGLFAAAA|208|Park |Parkway|Suite 370|Newtown|Jefferson County|MS|51749|United States|-6|condo| +23406|AAAAAAAAOGLFAAAA|361|Davis Jackson|ST|Suite 230|Springtown|Santa Fe County|NM|89858|United States|-7|single family| +23407|AAAAAAAAPGLFAAAA|890|Birch |Parkway|Suite 480|Woodland|Los Alamos County|NM|84854|United States|-7|condo| +23408|AAAAAAAAAHLFAAAA|975|Lakeview |RD|Suite 70|Mountain View|Brown County|KS|64466|United States|-6|apartment| +23409|AAAAAAAABHLFAAAA|438|Spring Fourth|RD|Suite 480|Pleasant Valley|Beaverhead County|MT|62477|United States|-7|apartment| +23410|AAAAAAAACHLFAAAA|860|Jefferson |Cir.|Suite K|Greenville|Massac County|IL|61387|United States|-6|single family| +23411|AAAAAAAADHLFAAAA||Spruce |Court||||KY|47057|||| +23412|AAAAAAAAEHLFAAAA|344|Johnson Pine|Boulevard|Suite 400|Wilson|Houston County|MN|56971|United States|-6|apartment| +23413|AAAAAAAAFHLFAAAA|187|Dogwood West|Dr.|Suite P|Vienna|Kearney County|NE|65119|United States|-7|single family| +23414|AAAAAAAAGHLFAAAA|709|Hickory |Wy|Suite S|Kingston|Jasper County|MO|64975|United States|-6|single family| +23415|AAAAAAAAHHLFAAAA|994|North 12th|Parkway|Suite 230|Fairfield|Marion County|KS|66192|United States|-6|apartment| +23416|AAAAAAAAIHLFAAAA|546|Railroad |Pkwy|Suite R|Jamestown|Grundy County|IL|66867|United States|-6|single family| +23417|AAAAAAAAJHLFAAAA|428||Court|Suite 240|Greenwood|Dade County||68828|United States||single family| +23418|AAAAAAAAKHLFAAAA|226|Elm Main|Lane|Suite 440|Valley View|Wharton County|TX|75124|United States|-6|apartment| +23419|AAAAAAAALHLFAAAA|776|Meadow Church|ST|Suite 340|Marion|Barbour County|WV|20399|United States|-5|condo| +23420|AAAAAAAAMHLFAAAA|543|West 13th|Cir.|Suite B|Mountain View|Centre County|PA|14466|United States|-5|condo| +23421|AAAAAAAANHLFAAAA|137|First |Boulevard|Suite J|Lincoln|Gooding County|ID|81289|United States|-7|condo| +23422|AAAAAAAAOHLFAAAA|61|6th Center|Blvd|Suite J|Shady Grove|Fulton County|AR|72812|United States|-6|condo| +23423|AAAAAAAAPHLFAAAA|60|Hickory |Pkwy|Suite 170|Mount Vernon|Independence County|AR|78482|United States|-6|apartment| +23424|AAAAAAAAAILFAAAA|218|Valley Walnut|Blvd|Suite Y|Antioch|Cass County|NE|68605|United States|-6|apartment| +23425|AAAAAAAABILFAAAA|156|4th Sunset|Cir.|Suite I|Red Hill|Appling County|GA|34338|United States|-5|single family| +23426|AAAAAAAACILFAAAA|431|Spruce |Road|Suite W|Woodland|Monroe County|PA|14854|United States|-5|single family| +23427|AAAAAAAADILFAAAA|132|Meadow Meadow|Blvd|Suite 210|Buena Vista|Kearney County|NE|65752|United States|-7|single family| +23428|AAAAAAAAEILFAAAA|286|Mill |Cir.|Suite B|Clifton|Grafton County|NH|08614|United States|-5|single family| +23429|AAAAAAAAFILFAAAA|905|Main |Wy|Suite B|Kingston|Williamsburg city|VA|24975|United States|-5|condo| +23430|AAAAAAAAGILFAAAA|754|Poplar Park|Ct.|Suite L|Watkins|King and Queen County|VA|21732|United States|-5|single family| +23431|AAAAAAAAHILFAAAA|398|2nd |Circle|Suite I|New Hope|Champaign County|IL|69431|United States|-6|apartment| +23432|AAAAAAAAIILFAAAA|230|Laurel Woodland|Way|Suite M|Harmon|Luce County|MI|45623|United States|-5|apartment| +23433|AAAAAAAAJILFAAAA|915|Green Adams|Dr.|Suite G|Mount Zion|Washington County|NE|68054|United States|-7|single family| +23434|AAAAAAAAKILFAAAA|455|Willow |Court|Suite 140|Maple Grove|Geneva County|AL|38252|United States|-6|single family| +23435|AAAAAAAALILFAAAA|923||||Bridgeport||MS|55817|United States||| +23436|AAAAAAAAMILFAAAA|406|Elm Fifth|Cir.|Suite B|Lakeview|Scott County|TN|38579|United States|-6|apartment| +23437|AAAAAAAANILFAAAA|365|Highland |Street|Suite H|Walnut Grove|McKenzie County|ND|57752|United States|-6|apartment| +23438|AAAAAAAAOILFAAAA|439|Oak |Court|Suite P|Arcola|Scott County|KS|61654|United States|-6|condo| +23439|AAAAAAAAPILFAAAA|810|Railroad First|Dr.|Suite 10|Buena Vista|Jones County|NC|25752|United States|-5|condo| +23440|AAAAAAAAAJLFAAAA|444|River |Boulevard|Suite D|Brownsville|West Feliciana Parish|LA|79310|United States|-6|single family| +23441|AAAAAAAABJLFAAAA|190|Elm |Circle|Suite 410|Highland Park|Franklin County|MS|56534|United States|-6|condo| +23442|AAAAAAAACJLFAAAA|503|Park |Ave|Suite 40|Sulphur Springs|San Saba County|TX|78354|United States|-6|apartment| +23443|AAAAAAAADJLFAAAA|258|Ridge |Parkway|Suite O|Lakeview|Monroe County|PA|18579|United States|-5|condo| +23444|AAAAAAAAEJLFAAAA|708|Forest |Parkway|Suite 400|Liberty|Texas County|OK|73451|United States|-6|apartment| +23445|AAAAAAAAFJLFAAAA|263|Hill |Ln|Suite 160|White Oak|Gulf County|FL|36668|United States|-5|single family| +23446|AAAAAAAAGJLFAAAA|828|Davis 4th|Road|Suite I|Tremont|McClain County|OK|79515|United States|-6|single family| +23447|AAAAAAAAHJLFAAAA|844|2nd Laurel|Circle|Suite 370|Pleasant Hill|Cherry County|NE|63604|United States|-6|single family| +23448|AAAAAAAAIJLFAAAA|728|Lee |Avenue|Suite 230|Unionville|Russell County|VA|21711|United States|-5|condo| +23449|AAAAAAAAJJLFAAAA|748|Ridge |Pkwy|Suite 400|Wolf Creek|Taylor County|WV|22455|United States|-5|condo| +23450|AAAAAAAAKJLFAAAA|395|Elevnth |Lane|Suite X|Summit|Olmsted County|MN|50499|United States|-6|apartment| +23451|AAAAAAAALJLFAAAA|31|Lee 6th|Wy|Suite 240|Clifton|Slope County|ND|58014|United States|-6|condo| +23452|AAAAAAAAMJLFAAAA|413|Smith 1st|Dr.|Suite Y|Riverside|Lander County|NV|89231|United States|-8|apartment| +23453|AAAAAAAANJLFAAAA|907|Ridge |Street|Suite 20|Pleasant Hill|Goochland County|VA|23604|United States|-5|condo| +23454|AAAAAAAAOJLFAAAA|501|Meadow Center|Ln|Suite D|Mount Zion|Osage County|KS|68054|United States|-6|single family| +23455|AAAAAAAAPJLFAAAA|12|Washington Maple|Street|Suite M|Union Hill|Walker County|TX|77746|United States|-6|single family| +23456|AAAAAAAAAKLFAAAA|705|Lincoln |RD|Suite 280|Buena Vista|Appomattox County|VA|25752|United States|-5|apartment| +23457|AAAAAAAABKLFAAAA|120|Railroad Second|Way|Suite J|Tabor|Hooker County|NE|68529|United States|-7|condo| +23458|AAAAAAAACKLFAAAA|116|Poplar |Drive|Suite 100|Green Acres|Upshur County|TX|77683|United States|-6|apartment| +23459|AAAAAAAADKLFAAAA|696|Lee Central|ST|Suite 130|Mount Vernon|Liberty County|GA|38482|United States|-5|single family| +23460|AAAAAAAAEKLFAAAA|338|North Lake|Blvd|Suite L|Clifton|Prairie County|MT|68014|United States|-7|condo| +23461|AAAAAAAAFKLFAAAA|809|3rd Cherry|Dr.|Suite J|Jackson|Yolo County|CA|99583|United States|-8|apartment| +23462|AAAAAAAAGKLFAAAA|613|Washington |Wy|Suite 150|Franklin|Thurston County|NE|69101|United States|-7|single family| +23463|AAAAAAAAHKLFAAAA|940|Adams Oak|Parkway|Suite 170|Oakwood|Centre County|PA|10169|United States|-5|single family| +23464|AAAAAAAAIKLFAAAA|966|3rd 4th|Circle|Suite S|Jamestown|Franklin city|VA|26867|United States|-5|condo| +23465|AAAAAAAAJKLFAAAA|692|Cherry |Ave|Suite 160|Springdale|Coahoma County|MS|58883|United States|-6|apartment| +23466|AAAAAAAAKKLFAAAA|129|3rd Fourth|Ct.|Suite 420|Pleasant Valley|Beauregard Parish|LA|72477|United States|-6|condo| +23467|AAAAAAAALKLFAAAA|266|Sunset |Ln|Suite 480|Centerville|Barry County|MI|40059|United States|-5|single family| +23468|AAAAAAAAMKLFAAAA|16|Oak Third|Ln|Suite C|Milan|Greene County|AL|36697|United States|-6|apartment| +23469|AAAAAAAANKLFAAAA|645|Broadway |Pkwy|Suite K|Lakeview|Benton County|OR|98579|United States|-8|apartment| +23470|AAAAAAAAOKLFAAAA|339|4th Jefferson|Drive|Suite V|Ferguson|Anderson County|KS|61821|United States|-6|apartment| +23471|AAAAAAAAPKLFAAAA|745|East Hillcrest|Road|Suite 320|Spring Hill|Arthur County|NE|66787|United States|-6|apartment| +23472|AAAAAAAAALLFAAAA|654|River Spring|Parkway|Suite D|Green Acres|Manatee County|FL|37683|United States|-5|condo| +23473|AAAAAAAABLLFAAAA|190|Pine 11th|Boulevard|Suite 290|Riverside|Hampton County|SC|29231|United States|-5|single family| +23474|AAAAAAAACLLFAAAA|29|Lee ||Suite V|Florence|Hamblen County|TN|||-5|single family| +23475|AAAAAAAADLLFAAAA|305|1st Poplar|Blvd|Suite Y|Pleasant Valley|Sabine County|TX|72477|United States|-6|apartment| +23476|AAAAAAAAELLFAAAA|624|13th Mill|Parkway|Suite A|Lakewood|Douglas County|NE|68877|United States|-6|condo| +23477|AAAAAAAAFLLFAAAA|799|South |Ave|Suite 240|Woodland|Izard County|AR|74854|United States|-6|apartment| +23478|AAAAAAAAGLLFAAAA|374|Green |Parkway|Suite E|Woodland|Steuben County|NY|14854|United States|-5|apartment| +23479|AAAAAAAAHLLFAAAA|393|College |Ct.|Suite J|Forest Hills|Sabine County|TX|79237|United States|-6|apartment| +23480|AAAAAAAAILLFAAAA|563|9th |Ave|Suite U|Macedonia|McKean County|PA|11087|United States|-5|condo| +23481|AAAAAAAAJLLFAAAA|412|Main Woodland|RD|Suite A|Riverview|Cheboygan County|MI|49003|United States|-5|apartment| +23482|AAAAAAAAKLLFAAAA|953|Sunset 2nd|Cir.|Suite 420|Union Hill|Washington County|TX|77746|United States|-6|apartment| +23483|AAAAAAAALLLFAAAA|454|Thirteenth |Way|Suite H|Union Hill|Broadwater County|MT|67746|United States|-7|condo| +23484|AAAAAAAAMLLFAAAA|804|Ridge |Drive|Suite D|Belmont|Cherokee County|NC|20191|United States|-5|condo| +23485|AAAAAAAANLLFAAAA|613|Spring |Avenue|Suite 490|Clinton|Tillman County|OK|78222|United States|-6|condo| +23486|AAAAAAAAOLLFAAAA||Forest College|Way|Suite 200||||32022|United States|-5|single family| +23487|AAAAAAAAPLLFAAAA|638|Fourth |Ct.|Suite R|Granite|Grant Parish|LA|76284|United States|-6|condo| +23488|AAAAAAAAAMLFAAAA|340|12th |RD|Suite 10|Wildwood|Tucker County|WV|26871|United States|-5|single family| +23489|AAAAAAAABMLFAAAA|937|Washington College|Avenue|Suite 130|Shiloh|Anderson County|KS|69275|United States|-6|apartment| +23490|AAAAAAAACMLFAAAA|571|Lake Broadway|Drive|Suite D|Shady Grove|Crow Wing County|MN|52812|United States|-6|apartment| +23491|AAAAAAAADMLFAAAA|307|15th 2nd|Dr.|Suite 30|Rosewood|Hardy County|WV|22205|United States|-5|condo| +23492|AAAAAAAAEMLFAAAA|111|8th |Circle|Suite U|Oak Hill|Humphreys County|MS|57838|United States|-6|condo| +23493|AAAAAAAAFMLFAAAA|100|2nd |Pkwy|Suite C|Spring Hill|Worcester County|MA|07387|United States|-5|apartment| +23494|AAAAAAAAGMLFAAAA|800|Central Main|Lane|Suite T|New Hope|Fresno County|CA|99431|United States|-8|single family| +23495|AAAAAAAAHMLFAAAA|503|Hill |Ln|Suite 320|Springdale|Ellis County|TX|78883|United States|-6|apartment| +23496|AAAAAAAAIMLFAAAA|965|Willow 2nd|Ct.|Suite I|Oak Grove|Clark County|WA|98370|United States|-8|condo| +23497|AAAAAAAAJMLFAAAA|800|Sunset Cedar|RD|Suite 240|Unionville|Sutton County|TX|71711|United States|-6|condo| +23498|AAAAAAAAKMLFAAAA|705|Poplar Jefferson|Boulevard|Suite 60|Crossroads|Walton County|FL|30534|United States|-5|apartment| +23499|AAAAAAAALMLFAAAA|679|South |Ct.|Suite 120|Peoria|Wibaux County|MT|69818|United States|-7|condo| +23500|AAAAAAAAMMLFAAAA|178|Pine 10th|Road|Suite 240|Providence|Berrien County|MI|46614|United States|-5|condo| +23501|AAAAAAAANMLFAAAA|16|Sunset Franklin|Court|Suite 450|Oakland|Rio Blanco County|CO|89843|United States|-7|apartment| +23502|AAAAAAAAOMLFAAAA|994|Lee 2nd|ST|Suite 300|Macedonia|Pleasants County|WV|21087|United States|-5|single family| +23503|AAAAAAAAPMLFAAAA|643|Highland |Court|Suite 40|Verona|Fairfield County|SC|23629|United States|-5|single family| +23504|AAAAAAAAANLFAAAA|734|Hill Poplar|Wy|Suite 320|Oak Ridge|Covington city|VA|28371|United States|-5|condo| +23505|AAAAAAAABNLFAAAA|953|2nd 6th|Drive|Suite 230|Hamilton|Washington County|MD|22808|United States|-5|single family| +23506|AAAAAAAACNLFAAAA|373|Forest |Cir.|Suite M|Fairview|Nevada County|CA|95709|United States|-8|apartment| +23507|AAAAAAAADNLFAAAA|436|Washington |ST|Suite R|Bridgeport|McIntosh County|OK|75817|United States|-6|condo| +23508|AAAAAAAAENLFAAAA|255|Valley |Wy|Suite 270|Maple Grove|Pushmataha County|OK|78252|United States|-6|apartment| +23509|AAAAAAAAFNLFAAAA|293|West |Court|Suite 420|Red Hill|Elk County|PA|14338|United States|-5|apartment| +23510|AAAAAAAAGNLFAAAA|456|Cedar North|Wy|Suite H|Jamestown|Corson County|SD|56867|United States|-6|single family| +23511|AAAAAAAAHNLFAAAA|42|Center |Boulevard|Suite 280|Shiloh|Cassia County|ID|89275|United States|-7|apartment| +23512|AAAAAAAAINLFAAAA|703|12th |Drive|Suite N|Shore Acres|Blaine County|MT|62724|United States|-7|apartment| +23513|AAAAAAAAJNLFAAAA|311|7th 9th|Cir.|Suite 100|Union Hill|Faulkner County|AR|77746|United States|-6|apartment| +23514|AAAAAAAAKNLFAAAA|93|Center Pine|Drive|Suite Y|Maple Hill|Huerfano County|CO|88095|United States|-7|single family| +23515|AAAAAAAALNLFAAAA|741|Third |Street|Suite 110|Pine Grove|White County|TN|34593|United States|-6|apartment| +23516|AAAAAAAAMNLFAAAA|850|1st Adams|Court|Suite 120|Pine Grove|Barber County|KS|64593|United States|-6|apartment| +23517|AAAAAAAANNLFAAAA|541|15th |Ct.|Suite R|Centerville|Riley County|KS|60059|United States|-6|apartment| +23518|AAAAAAAAONLFAAAA|753|Oak |ST|Suite Y|Kirkland|Morgan County|GA|37896|United States|-5|apartment| +23519|AAAAAAAAPNLFAAAA|200|Fifth |Ave|Suite L|Brookwood|Jefferson County|OH|40965|United States|-5|single family| +23520|AAAAAAAAAOLFAAAA|308|Green |Ct.|Suite J|New Hope|Wabasha County|MN|59431|United States|-6|apartment| +23521|AAAAAAAABOLFAAAA|462|Park Fourth|Circle|Suite 40|Franklin|Chickasaw County|MS|59101|United States|-6|apartment| +23522|AAAAAAAACOLFAAAA|437|View Chestnut|Ct.|Suite 90|Friendship|Lycoming County|PA|14536|United States|-5|single family| +23523|AAAAAAAADOLFAAAA|709|2nd Smith|Cir.|Suite Y|Lebanon|Greene County|GA|32898|United States|-5|apartment| +23524|AAAAAAAAEOLFAAAA|179|North Valley|Dr.|Suite 460|Enterprise|Inyo County|CA|91757|United States|-8|apartment| +23525|AAAAAAAAFOLFAAAA|539|Fourth Lincoln|Court|Suite W|Clifton|Trego County|KS|68014|United States|-6|apartment| +23526|AAAAAAAAGOLFAAAA|876|College |Parkway|Suite W|Oak Ridge|Limestone County|TX|78371|United States|-6|single family| +23527|AAAAAAAAHOLFAAAA|879|Woodland Hickory|RD|Suite Q|Elm Grove|Cameron Parish|LA|73298|United States|-6|condo| +23528|AAAAAAAAIOLFAAAA|196|2nd |Way|Suite 50|Bridgeport|Muscatine County|IA|55817|United States|-6|single family| +23529|AAAAAAAAJOLFAAAA|||Road|Suite 110|Hillcrest||||United States||single family| +23530|AAAAAAAAKOLFAAAA|272|Main Jackson|Pkwy|Suite 340|Clifton|Hempstead County|AR|78014|United States|-6|condo| +23531|AAAAAAAALOLFAAAA|704|11th |Parkway|Suite A|Springdale|Giles County|TN|38883|United States|-5|condo| +23532|AAAAAAAAMOLFAAAA|719|West Cherry|Pkwy|Suite 130|Woodland|Hyde County|SD|54854|United States|-7|single family| +23533|AAAAAAAANOLFAAAA|722|Williams Railroad|Dr.|Suite 60|Crossroads|Henderson County|KY|40534|United States|-6|condo| +23534|AAAAAAAAOOLFAAAA|317|Dogwood Hillcrest|Road|Suite 220|Wildwood|Cloud County|KS|66871|United States|-6|apartment| +23535|AAAAAAAAPOLFAAAA|818|Pine Meadow|Ct.|Suite 180|Oak Ridge|Wabasha County|MN|58371|United States|-6|condo| +23536|AAAAAAAAAPLFAAAA|184|West |Ave|Suite Q|Red Hill|Sherburne County|MN|54338|United States|-6|apartment| +23537|AAAAAAAABPLFAAAA|37|East |Boulevard|Suite 0|Deerfield|Washington County|MD|29840|United States|-5|condo| +23538|AAAAAAAACPLFAAAA|246|Jefferson |Parkway|Suite T|Oak Grove|Hall County|TX|78370|United States|-6|single family| +23539|AAAAAAAADPLFAAAA|166|Maple First|Pkwy|Suite I|Glendale|Vernon County|WI|53951|United States|-6|single family| +23540|AAAAAAAAEPLFAAAA|516|2nd |Pkwy|Suite 470|Shiloh|Lincoln County|ME|09875|United States|-5|single family| +23541|AAAAAAAAFPLFAAAA|912|South |Way|Suite H|Clinton|Leake County|MS|58222|United States|-6|apartment| +23542|AAAAAAAAGPLFAAAA|157|Oak Forest|Pkwy|Suite N|Green Acres|Pickaway County|OH|47683|United States|-5|condo| +23543|AAAAAAAAHPLFAAAA|602|1st |Drive|Suite 70|Elba|Pickens County|SC|20262|United States|-5|condo| +23544|AAAAAAAAIPLFAAAA|36|Lincoln |Street|Suite 100|Five Forks|Pulaski County|GA|32293|United States|-5|apartment| +23545|AAAAAAAAJPLFAAAA|305|Sycamore Madison|Court|Suite 310|Ashland|Jones County|TX|74244|United States|-6|apartment| +23546|AAAAAAAAKPLFAAAA|158|River Williams|Parkway|Suite 70|Centerville|Barren County|KY|40059|United States|-6|apartment| +23547|AAAAAAAALPLFAAAA|136|Birch |Pkwy|Suite 320|Clinton|Kenton County|KY|48222|United States|-5|apartment| +23548|AAAAAAAAMPLFAAAA|337|Railroad |Ln|Suite H|Lebanon|Douglas County|SD|52898|United States|-6|apartment| +23549|AAAAAAAANPLFAAAA|476|Oak |Way|Suite 100|Cedar Grove|Ouachita County|AR|70411|United States|-6|single family| +23550|AAAAAAAAOPLFAAAA|662|Maple Hillcrest|Wy|Suite 250|Buena Vista|Pike County|PA|15752|United States|-5|apartment| +23551|AAAAAAAAPPLFAAAA|536|Cherry 7th|Avenue|Suite S|Newtown|Meagher County|MT|61749|United States|-7|single family| +23552|AAAAAAAAAAMFAAAA|496|Dogwood Birch|Wy|Suite 230|Mountain View|Pittsylvania County|VA|24466|United States|-5|condo| +23553|AAAAAAAABAMFAAAA|306|Laurel 5th|Cir.|Suite 90|Frenchtown|Madison County|NE|62629|United States|-7|condo| +23554|AAAAAAAACAMFAAAA|887|Walnut Poplar|Parkway|Suite 450|Red Oak|West Carroll Parish|LA|75018|United States|-6|single family| +23555|AAAAAAAADAMFAAAA|893|12th |Boulevard|Suite W|Pleasant Hill|Cedar County|IA|53604|United States|-6|apartment| +23556|AAAAAAAAEAMFAAAA|507|Maple West|Drive|Suite 420|Wilson|Green Lake County|WI|56971|United States|-6|single family| +23557|AAAAAAAAFAMFAAAA|293|Lake North|Lane|Suite 360|Sulphur Springs|Henderson County|NC|28354|United States|-5|condo| +23558|AAAAAAAAGAMFAAAA|993|Franklin |Drive|Suite 10|Glendale|Edmunds County|SD|53951|United States|-6|single family| +23559|AAAAAAAAHAMFAAAA|558|Hill |Dr.|Suite K|Rockford|Humphreys County|MS|51606|United States|-6|condo| +23560|AAAAAAAAIAMFAAAA|402|Cedar |Dr.|Suite 10|Woodlawn|Sibley County|MN|54098|United States|-6|condo| +23561|AAAAAAAAJAMFAAAA|463|Third |Court|Suite 380|Buena Vista|Charles Mix County|SD|55752|United States|-6|apartment| +23562|AAAAAAAAKAMFAAAA|||||Rankin|||22621||-5|apartment| +23563|AAAAAAAALAMFAAAA|58|8th College|ST|Suite 420|Providence|Marion County|IL|66614|United States|-6|apartment| +23564|AAAAAAAAMAMFAAAA|805|Adams |Ln|Suite 250|Spring Hill|Clay County|IA|56787|United States|-6|apartment| +23565|AAAAAAAANAMFAAAA|559|Franklin |Court|Suite D|Enterprise|Pike County|KY|41757|United States|-5|single family| +23566|AAAAAAAAOAMFAAAA|838|Ridge College|Pkwy|Suite 100|Oakdale|McCracken County|KY|49584|United States|-5|apartment| +23567|AAAAAAAAPAMFAAAA|436|College |Boulevard|Suite W|Five Points|Russell County|VA|26098|United States|-5|single family| +23568|AAAAAAAAABMFAAAA|263|Mill Park|Drive|Suite C|Union|Hayes County|NE|68721|United States|-6|condo| +23569|AAAAAAAABBMFAAAA|526|Smith 4th|Circle|Suite S|Edgewood|Edwards County|KS|60069|United States|-6|apartment| +23570|AAAAAAAACBMFAAAA|229|First |ST|Suite M|Union Hill|Nevada County|AR|77746|United States|-6|single family| +23571|AAAAAAAADBMFAAAA|280|Lee |Cir.|Suite 150|Providence|Pickaway County|OH|46614|United States|-5|apartment| +23572|AAAAAAAAEBMFAAAA|861|Fifth |Boulevard|Suite H|Belmont|Pine County|MN|50191|United States|-6|apartment| +23573|AAAAAAAAFBMFAAAA|949|Johnson |Ct.|Suite V|Walnut Grove|Cedar County|MO|67752|United States|-6|condo| +23574|AAAAAAAAGBMFAAAA|88|13th |Boulevard|Suite A|Union Hill|Camden County|NC|27746|United States|-5|condo| +23575|AAAAAAAAHBMFAAAA|879|Oak Sixth|Blvd|Suite 150|Wilson|Westmoreland County|VA|26971|United States|-5|apartment| +23576|AAAAAAAAIBMFAAAA|270|Second |Avenue|Suite 480|Hillcrest|Lenoir County|NC|23003|United States|-5|condo| +23577|AAAAAAAAJBMFAAAA|271|North Fifth|Parkway|Suite 320|Fairview|Lenawee County|MI|45709|United States|-5|single family| +23578|AAAAAAAAKBMFAAAA|185|Meadow Ridge|Drive|Suite 360|Tipton|Guernsey County|OH|48773|United States|-5|single family| +23579|AAAAAAAALBMFAAAA|950|Elm |ST|Suite B|Collinsville|Polk County|MN|52459|United States|-6|apartment| +23580|AAAAAAAAMBMFAAAA|254|Second Oak|Dr.|Suite 450|Lakeside|Nottoway County|VA|29532|United States|-5|single family| +23581|AAAAAAAANBMFAAAA|606|Washington Forest|Boulevard|Suite 230|Bridgeport|Christian County|KY|45817|United States|-6|single family| +23582|AAAAAAAAOBMFAAAA|900|8th 4th|Cir.|Suite 250|Spring Hill|Kings County|NY|16787|United States|-5|apartment| +23583|AAAAAAAAPBMFAAAA|669|Eigth |Avenue|Suite 190|Hillcrest|Woodbury County|IA|53003|United States|-6|condo| +23584|AAAAAAAAACMFAAAA|254|Lakeview North|Way|Suite 360|Jenkins|East Carroll Parish|LA|77292|United States|-6|apartment| +23585|AAAAAAAABCMFAAAA|149|Cherry 5th|Parkway|Suite 400|Murray|Doddridge County|WV|22150|United States|-5|apartment| +23586|AAAAAAAACCMFAAAA|998|Ridge |Wy|Suite R|Mountain View|Darke County|OH|44466|United States|-5|single family| +23587|AAAAAAAADCMFAAAA|225|Hill |Road|Suite 240|Milan|Tolland County|CT|07297|United States|-5|single family| +23588|AAAAAAAAECMFAAAA|892|Pine 6th|Street|Suite M|Welcome|Delaware County|NY|16386|United States|-5|single family| +23589|AAAAAAAAFCMFAAAA|440|Eigth Smith|Boulevard|Suite O|Philadelphia|Jackson County|IN|45591|United States|-5|condo| +23590|AAAAAAAAGCMFAAAA|753|Ninth |Boulevard|Suite 390|Centerville|Wayne County|OH|40059|United States|-5|apartment| +23591|AAAAAAAAHCMFAAAA|160|Ridge |Road|Suite 260|Riverview|Robertson County|TX|79003|United States|-6|condo| +23592|AAAAAAAAICMFAAAA|460|Lake Church|Way|Suite S|Walnut Grove|Sterling County|TX|77752|United States|-6|single family| +23593|AAAAAAAAJCMFAAAA|45|Hillcrest |Ave|Suite 240|Jackson|Hughes County|SD|59583|United States|-7|apartment| +23594|AAAAAAAAKCMFAAAA|63|View |Cir.|Suite 40|Clifton|Union County|SC|28014|United States|-5|condo| +23595|AAAAAAAALCMFAAAA|921|Davis Jackson|Pkwy|Suite 450|Jackson|Clay County|NE|69583|United States|-6|apartment| +23596|AAAAAAAAMCMFAAAA|752|3rd Sycamore|Lane|Suite 430|Pleasant Valley|Frederick County|VA|22477|United States|-5|single family| +23597|AAAAAAAANCMFAAAA|237|Spruce Laurel|Lane|Suite 400|Woodland|Washington County|FL|34854|United States|-5|single family| +23598|AAAAAAAAOCMFAAAA|507|Cedar Fourth|Parkway|Suite D|Wildwood|Bibb County|GA|36871|United States|-5|condo| +23599|AAAAAAAAPCMFAAAA|60|11th 4th|Ct.|Suite I|Nottingham|Albemarle County|VA|24074|United States|-5|single family| +23600|AAAAAAAAADMFAAAA|577|2nd 9th|Way|Suite M|Five Points|Sanborn County|SD|56098|United States|-7|apartment| +23601|AAAAAAAABDMFAAAA|662|2nd Fifth|Parkway|Suite 70|Greenwood|Fayette County|AL|38828|United States|-6|condo| +23602|AAAAAAAACDMFAAAA|96|Lincoln Oak|Ln|Suite J|Vance|Franklin County|IN|40268|United States|-5|apartment| +23603|AAAAAAAADDMFAAAA|241|Cedar |Ln|Suite 460|Oakwood|Sebastian County|AR|70169|United States|-6|condo| +23604|AAAAAAAAEDMFAAAA|675|Oak Third|Boulevard|Suite X|Marion|Nicholas County|WV|20399|United States|-5|single family| +23605|AAAAAAAAFDMFAAAA|994|Oak Chestnut|Ave|Suite A|Georgetown|Lamoille County|VT|07657|United States|-5|apartment| +23606|AAAAAAAAGDMFAAAA||Jefferson Meadow|RD|Suite 310||Fayette County|||United States|-5|| +23607|AAAAAAAAHDMFAAAA|108|Wilson Willow|Street|Suite G|Midway|Wayne County|MI|41904|United States|-5|single family| +23608|AAAAAAAAIDMFAAAA|560|College |Parkway|Suite 290|Oakwood|Colonial Heights city|VA|20169|United States|-5|apartment| +23609|AAAAAAAAJDMFAAAA|504|First East|Wy|Suite M|Woodruff|Lexington County|SC|24174|United States|-5|apartment| +23610|AAAAAAAAKDMFAAAA|113|14th |Pkwy|Suite E|Woodland|Washington County|NC|24854|United States|-5|condo| +23611|AAAAAAAALDMFAAAA|72|Lincoln North|Dr.|Suite 290|Highland Park|Tolland County|CT|07134|United States|-5|single family| +23612|AAAAAAAAMDMFAAAA|527||Court|Suite R||Hughes County|OK||United States||condo| +23613|AAAAAAAANDMFAAAA|141|Cedar Spruce|Circle|Suite 190|Woodlawn|Sevier County|AR|74098|United States|-6|apartment| +23614|AAAAAAAAODMFAAAA|258|Hill |Way|Suite 310|Edgewood|Camp County|TX|70069|United States|-6|condo| +23615|AAAAAAAAPDMFAAAA|375|Main 1st|RD|Suite S|Clinton|Vernon Parish|LA|78222|United States|-6|single family| +23616|AAAAAAAAAEMFAAAA|521|College Locust|Court|Suite F|Oakwood|Arthur County|NE|60169|United States|-6|apartment| +23617|AAAAAAAABEMFAAAA|656|Woodland |Cir.|Suite O|Franklin|Putnam County|TN|39101|United States|-6|apartment| +23618|AAAAAAAACEMFAAAA|716|Second |Way|Suite L|Glenwood|Washita County|OK|73511|United States|-6|apartment| +23619|AAAAAAAADEMFAAAA|387|Pine |ST||||WI|53394||-6|condo| +23620|AAAAAAAAEEMFAAAA|76|Cedar Davis|Blvd|Suite S|Woodcrest|Huntington County|IN|44919|United States|-5|condo| +23621|AAAAAAAAFEMFAAAA|81|Lee |Cir.|Suite 310|Summit|Montgomery County|VA|20499|United States|-5|single family| +23622|AAAAAAAAGEMFAAAA|240|Fourth Dogwood|Ln|Suite B|Murphy|Lincoln Parish|LA|72105|United States|-6|single family| +23623|AAAAAAAAHEMFAAAA|343|Railroad |Blvd|Suite 240|Franklin|Douglas County|KS|69101|United States|-6|single family| +23624|AAAAAAAAIEMFAAAA|772|4th View|Dr.|Suite 40|Stringtown|Scott County|IA|50162|United States|-6|condo| +23625|AAAAAAAAJEMFAAAA|673|Walnut Main|Lane|Suite 410|Stringtown|Allendale County|SC|20162|United States|-5|apartment| +23626|AAAAAAAAKEMFAAAA|645|Locust |Ct.|Suite 130|Midway|Wayne County|IA|51904|United States|-6|condo| +23627|AAAAAAAALEMFAAAA|57|Hill Williams|Avenue|Suite D|Union|Perry County|TN|38721|United States|-6|condo| +23628|AAAAAAAAMEMFAAAA|597|Sixth Forest|Parkway|Suite O|Gary|Solano County|CA|90418|United States|-8|condo| +23629|AAAAAAAANEMFAAAA|146|Wilson |Boulevard|Suite E|Shiloh|Jasper County|MO|69275|United States|-6|single family| +23630|AAAAAAAAOEMFAAAA|736|Elm Park|Road|Suite Q|Greenfield|Platte County|NE|65038|United States|-7|single family| +23631|AAAAAAAAPEMFAAAA|446|Davis Woodland|Avenue|Suite 50|Salem|Starr County|TX|78048|United States|-6|apartment| +23632|AAAAAAAAAFMFAAAA|212|Meadow |Cir.|Suite O|New Town|Chambers County|AL|39634|United States|-6|single family| +23633|AAAAAAAABFMFAAAA|371|Elm |Wy|Suite 240|Highland Park|Miami County|KS|66534|United States|-6|apartment| +23634|AAAAAAAACFMFAAAA|565|Third Smith|Parkway|Suite L|Summit|Routt County|CO|80499|United States|-7|single family| +23635|AAAAAAAADFMFAAAA||||||Marion County|SC|||-5|| +23636|AAAAAAAAEFMFAAAA|487|Sixth Spring|Parkway|Suite A|Oakwood|Fremont County|IA|50169|United States|-6|apartment| +23637|AAAAAAAAFFMFAAAA|481|4th |Ave|Suite R|Harmony|Cullman County|AL|35804|United States|-6|condo| +23638|AAAAAAAAGFMFAAAA|230|Second 3rd|Avenue|Suite V|Mountain View|Union County|NM|84466|United States|-7|condo| +23639|AAAAAAAAHFMFAAAA|67|Lincoln Oak|Ave|Suite 470|Lakeside|Marshall County|MS|59532|United States|-6|condo| +23640|AAAAAAAAIFMFAAAA|866|Franklin 4th|Pkwy|Suite 400|Newport|Lebanon County|PA|11521|United States|-5|single family| +23641|AAAAAAAAJFMFAAAA|524|4th Main|Dr.|Suite M|Riverdale|Nemaha County|KS|69391|United States|-6|apartment| +23642|AAAAAAAAKFMFAAAA|321|Main Maple|Ave|Suite G|Summit|Winona County|MN|50499|United States|-6|single family| +23643|AAAAAAAALFMFAAAA|217|6th |Lane|Suite X|Gum Springs|San Juan County|NM|82106|United States|-7|apartment| +23644|AAAAAAAAMFMFAAAA|761|Meadow 3rd|Avenue|Suite 110|Five Forks|Randolph County|GA|32293|United States|-5|condo| +23645|AAAAAAAANFMFAAAA|300|Maple North|Lane|Suite 340|Springfield|Trinity County|TX|79303|United States|-6|single family| +23646|AAAAAAAAOFMFAAAA|285|7th Franklin|Pkwy|Suite 350|Glenwood|Okfuskee County|OK|73511|United States|-6|single family| +23647|AAAAAAAAPFMFAAAA|||Road|||Barbour County|AL|36557|||single family| +23648|AAAAAAAAAGMFAAAA|943|Railroad |Lane|Suite 490|Union|Grant County|OR|98721|United States|-8|condo| +23649|AAAAAAAABGMFAAAA|508|South Fifth|Ct.|Suite S|Bethel|Kern County|CA|95281|United States|-8|apartment| +23650|AAAAAAAACGMFAAAA|195|First East|Cir.|Suite D|Farmersville|Baxter County|AR|79305|United States|-6|condo| +23651|AAAAAAAADGMFAAAA|694|Main Lincoln|Court|Suite 230|Deerfield|Coconino County|AZ|89840|United States|-7|single family| +23652|AAAAAAAAEGMFAAAA|890|8th |Circle|Suite V|Oak Grove|Rooks County|KS|68370|United States|-6|single family| +23653|AAAAAAAAFGMFAAAA|588|Cherry |Wy|Suite D|Oak Hill|Scott County|KY|47838|United States|-5|condo| +23654|AAAAAAAAGGMFAAAA|512|13th Fourth|Ln|Suite 130|Union Hill|Lyon County|KY|47746|United States|-5|single family| +23655|AAAAAAAAHGMFAAAA|811|Williams Sunset|Parkway|Suite H|Lakeview|Sevier County|AR|78579|United States|-6|single family| +23656|AAAAAAAAIGMFAAAA|985|Jackson Oak|Way|Suite 410|Lakeview|Rutland County|VT|09179|United States|-5|condo| +23657|AAAAAAAAJGMFAAAA|227|4th |Street|Suite 290|Oak Grove|Bay County|FL|38370|United States|-5|single family| +23658|AAAAAAAAKGMFAAAA|340|Hill 14th|Road|Suite 50|Five Points|Dade County|GA|36098|United States|-5|condo| +23659|AAAAAAAALGMFAAAA|685|5th |Dr.|Suite T|Hillcrest|Sarpy County|NE|63003|United States|-7|single family| +23660|AAAAAAAAMGMFAAAA|548|Lake |Cir.|Suite 270|Waterloo|Knox County|IL|61675|United States|-6|condo| +23661|AAAAAAAANGMFAAAA|145|Dogwood |Cir.|Suite 220|Scottsville|Richland County|SC|24190|United States|-5|apartment| +23662|AAAAAAAAOGMFAAAA|771|Central 2nd|Street|Suite O|Riverside|Washington County|MD|29231|United States|-5|apartment| +23663|AAAAAAAAPGMFAAAA|792|Fifth Maple|Blvd|Suite 460|Jamestown|Clallam County|WA|96867|United States|-8|condo| +23664|AAAAAAAAAHMFAAAA|103|6th Pine|Court|Suite H|Union Hill|Waukesha County|WI|57746|United States|-6|single family| +23665|AAAAAAAABHMFAAAA|594|Mill |Avenue|Suite L|Lakeview|Floyd County|IN|48579|United States|-5|condo| +23666|AAAAAAAACHMFAAAA|470|Eigth Jackson|Blvd|Suite A|Frenchtown|Montague County|TX|72629|United States|-6|condo| +23667|AAAAAAAADHMFAAAA|18|Elm |Ct.|Suite 210|Jamestown|Pitt County|NC|26867|United States|-5|single family| +23668|AAAAAAAAEHMFAAAA|582|Spruce Oak|Avenue|Suite M|Newtown|Brown County|SD|51749|United States|-6|apartment| +23669|AAAAAAAAFHMFAAAA|972|West 2nd|Drive|Suite 410|Shady Grove|Schuyler County|MO|62812|United States|-6|condo| +23670|AAAAAAAAGHMFAAAA|583|View 6th|Ave|Suite X|Brunswick|Jo Daviess County|IL|64642|United States|-6|apartment| +23671|AAAAAAAAHHMFAAAA|73||Blvd|Suite P|Pine Grove|||34593|||| +23672|AAAAAAAAIHMFAAAA|311|Laurel Center|Circle|Suite 170|Montezuma|Albany County|WY|82150|United States|-7|single family| +23673|AAAAAAAAJHMFAAAA|336|Main Railroad|Parkway|Suite X|Union|Rhea County|TN|38721|United States|-6|condo| +23674|AAAAAAAAKHMFAAAA|323|Fifth |Court|Suite V|Gary|Lake County|MT|60418|United States|-7|single family| +23675|AAAAAAAALHMFAAAA|570|Pine 12th|Road|Suite M|Shady Grove|Umatilla County|OR|92812|United States|-8|single family| +23676|AAAAAAAAMHMFAAAA|333|Hickory |Way|Suite P|Pleasant Valley|Le Sueur County|MN|52477|United States|-6|apartment| +23677|AAAAAAAANHMFAAAA|59|Hill Park|Drive|Suite 150|Lakewood|Norton city|VA|28877|United States|-5|apartment| +23678|AAAAAAAAOHMFAAAA|641|13th Fourth|Cir.|Suite 140|Red Hill|Golden Valley County|ND|54338|United States|-6|single family| +23679|AAAAAAAAPHMFAAAA|456|Maple Sunset|Drive|Suite 370|Friendship|Jackson County|NC|24536|United States|-5|condo| +23680|AAAAAAAAAIMFAAAA|599|7th |Way|Suite 470|Philadelphia|Morgan County|CO|85591|United States|-7|condo| +23681|AAAAAAAABIMFAAAA|148|View 6th|Dr.|Suite S|Georgetown|Colfax County|NE|67057|United States|-6|condo| +23682|AAAAAAAACIMFAAAA|936|Mill |Way|Suite 410|Stringtown|Medina County|TX|70162|United States|-6|apartment| +23683|AAAAAAAADIMFAAAA|948|Lake Oak|Circle|Suite 400|Franklin|Carroll County|VA|29101|United States|-5|single family| +23684|AAAAAAAAEIMFAAAA|831|Woodland Sycamore|Pkwy|Suite 220|Spring Hill|Matanuska-Susitna Borough|AK|96787|United States|-9|condo| +23685|AAAAAAAAFIMFAAAA|482|9th 8th|Dr.|Suite 260|Macedonia|Cape May County|NJ|01687|United States|-5|apartment| +23686|AAAAAAAAGIMFAAAA|966|Cherry Sixth|Pkwy|Suite 300|Buena Vista|Clarke County|IA|55752|United States|-6|condo| +23687|AAAAAAAAHIMFAAAA|175|2nd |Drive|Suite 40|Deerfield|Shelby County|MO|69840|United States|-6|single family| +23688|AAAAAAAAIIMFAAAA|815|Madison Central|Street|Suite 400|Enterprise|Clay County|IN|41757|United States|-5|single family| +23689|AAAAAAAAJIMFAAAA|801|4th Ash|Wy|Suite 110|Belmont|Woodford County|KY|40191|United States|-6|apartment| +23690|AAAAAAAAKIMFAAAA|525|Sycamore East|Boulevard|Suite 40|Dewey|Bolivar County|MS|51160|United States|-6|apartment| +23691|AAAAAAAALIMFAAAA|252|Second |Court|Suite 380|Marion|Roane County|WV|20399|United States|-5|apartment| +23692|AAAAAAAAMIMFAAAA|174|Birch |Avenue|Suite J|Saint Clair|Grundy County|IL|65294|United States|-6|apartment| +23693|AAAAAAAANIMFAAAA|182|Meadow 5th|Road|Suite R|Salem|Montmorency County|MI|48048|United States|-5|condo| +23694|AAAAAAAAOIMFAAAA|762|Oak River|Ln|Suite H|Salem|Itawamba County|MS|58048|United States|-6|apartment| +23695|AAAAAAAAPIMFAAAA|970|Hill |Court|Suite W|Hillcrest|Geary County|KS|63003|United States|-6|single family| +23696|AAAAAAAAAJMFAAAA|401|Forest |Way|Suite 480|Riverside|Quitman County|MS|59231|United States|-6|single family| +23697|AAAAAAAABJMFAAAA|923|Ash Wilson|Court|Suite 400|Argyle|Newberry County|SC|28722|United States|-5|single family| +23698|AAAAAAAACJMFAAAA|900|Park 6th|Street|Suite 0|Lebanon|Clinton County|OH|42898|United States|-5|single family| +23699|AAAAAAAADJMFAAAA|751|Adams |Street|Suite O|Hillcrest|Gillespie County|TX|73003|United States|-6|apartment| +23700|AAAAAAAAEJMFAAAA|278|Maple Forest|Parkway|Suite 320|Freeport|Pinellas County|FL|31844|United States|-5|single family| +23701|AAAAAAAAFJMFAAAA|202|5th Church|ST|Suite 80|Roy|Phillips County|CO|80744|United States|-7|single family| +23702|AAAAAAAAGJMFAAAA|459|Washington |ST|Suite O|Glendale|Washington County|AR|73951|United States|-6|condo| +23703|AAAAAAAAHJMFAAAA|19|Dogwood Washington|Pkwy|Suite 350|Five Points|Greenbrier County|WV|26098|United States|-5|condo| +23704|AAAAAAAAIJMFAAAA|123|Church 13th|Avenue|Suite K|Summit|Polk County|OR|90499|United States|-8|apartment| +23705|AAAAAAAAJJMFAAAA|420|7th Lee|Street|Suite 440|Friendship|Brooke County|WV|24536|United States|-5|condo| +23706|AAAAAAAAKJMFAAAA|171|River Meadow|ST|Suite U|White Oak|Kittitas County|WA|96668|United States|-8|condo| +23707|AAAAAAAALJMFAAAA|684|Hillcrest Cherry|Lane|Suite 130|Georgetown|Modoc County|CA|97057|United States|-8|apartment| +23708|AAAAAAAAMJMFAAAA|30|5th |Cir.|Suite 430|Pleasant Grove|Lawrence County|IL|64136|United States|-6|condo| +23709|AAAAAAAANJMFAAAA|873|Fifth |Court|Suite 280|Farmington|Cass County|MI|49145|United States|-5|condo| +23710|AAAAAAAAOJMFAAAA|461|Willow 14th|RD|Suite 360|Springfield|Union County|IN|49303|United States|-5|condo| +23711|AAAAAAAAPJMFAAAA|371|Forest |Street|Suite Y|Shiloh|Jefferson County|KS|69275|United States|-6|apartment| +23712|AAAAAAAAAKMFAAAA|735|5th Pine|Road|Suite V|Longwood|Gates County|NC|27021|United States|-5|single family| +23713|AAAAAAAABKMFAAAA|346|Chestnut |Street|Suite G|Mount Vernon|El Dorado County|CA|98482|United States|-8|apartment| +23714|AAAAAAAACKMFAAAA|455|West |Avenue|Suite I|Pine Grove|Fluvanna County|VA|24593|United States|-5|single family| +23715|AAAAAAAADKMFAAAA|439|5th Hill|RD|Suite P|Lakewood|Newton County|AR|78877|United States|-6|condo| +23716|AAAAAAAAEKMFAAAA|460|4th Park|Wy|Suite 110|Centerville|Pulaski County|KY|40059|United States|-5|apartment| +23717|AAAAAAAAFKMFAAAA|393|Ridge |Cir.|Suite F|Clifton|Fulton County|IL|68014|United States|-6|apartment| +23718|AAAAAAAAGKMFAAAA|42|Washington 2nd|Court|Suite 80|Howell|Jessamine County|KY|44854|United States|-6|single family| +23719|AAAAAAAAHKMFAAAA|678|Madison Central|Ave|Suite F|Fairfield|DuPage County|IL|66192|United States|-6|apartment| +23720|AAAAAAAAIKMFAAAA|770|6th |RD|Suite H|Milan|Monroe County|KY|46697|United States|-5|condo| +23721|AAAAAAAAJKMFAAAA|221|9th First|Road|Suite N|Riverview|Muskogee County|OK|79003|United States|-6|single family| +23722|AAAAAAAAKKMFAAAA|531|Lakeview |RD|Suite C|Greenwood|Wayne County|WV|28828|United States|-5|condo| +23723|AAAAAAAALKMFAAAA|914|Park Fifth|Boulevard|Suite N|Summit|Howard County|AR|70499|United States|-6|condo| +23724|AAAAAAAAMKMFAAAA|766|Forest 5th|Lane|Suite 200|Jamestown|Cass County|IA|56867|United States|-6|condo| +23725|AAAAAAAANKMFAAAA|15|South Jefferson|Way|Suite G|Antioch|Manistee County|MI|48605|United States|-5|apartment| +23726|AAAAAAAAOKMFAAAA|467|Locust |Avenue|Suite G|Mount Vernon|Beauregard Parish|LA|78482|United States|-6|apartment| +23727|AAAAAAAAPKMFAAAA|253|Fifth 1st|Pkwy|Suite 270|Friendship|Accomack County|VA|24536|United States|-5|single family| +23728|AAAAAAAAALMFAAAA|675|Jefferson |Pkwy|Suite 20|Crossroads|Russell County|VA|20534|United States|-5|condo| +23729|AAAAAAAABLMFAAAA|251|4th |Street|Suite 350|Harmony|Carbon County|MT|65804|United States|-7|single family| +23730|AAAAAAAACLMFAAAA|915|Park |Parkway|Suite 200|Red Hill|Prentiss County|MS|54338|United States|-6|condo| +23731|AAAAAAAADLMFAAAA|716|Oak View|Ln|Suite R|Woodlawn|Douglas County|GA|34098|United States|-5|single family| +23732|AAAAAAAAELMFAAAA|72|Ridge Hill|Lane|Suite R|Highland Park|Beaufort County|NC|26534|United States|-5|apartment| +23733|AAAAAAAAFLMFAAAA|239|View Third|Dr.|Suite M|Fairview|Taylor County|FL|35709|United States|-5|single family| +23734|AAAAAAAAGLMFAAAA|84|Sunset |Drive|Suite B|Ashland|Saunders County|NE|64244|United States|-7|condo| +23735|AAAAAAAAHLMFAAAA|281|8th |Street|Suite D|Lakeside|Sagadahoc County|ME|09532|United States|-5|apartment| +23736|AAAAAAAAILMFAAAA|436|Hill |Dr.|Suite 410|Walnut Grove|Salem city|VA|27752|United States|-5|condo| +23737|AAAAAAAAJLMFAAAA|923|2nd 12th||||||71387||-6|| +23738|AAAAAAAAKLMFAAAA|722|11th Elm|Court|Suite 410|Salem|Matagorda County|TX|78048|United States|-6|condo| +23739|AAAAAAAALLMFAAAA|900|College Fifteenth|Cir.|Suite R|Oak Grove|Stevens County|MN|58370|United States|-6|single family| +23740|AAAAAAAAMLMFAAAA|491|Meadow 4th|Cir.|Suite 180|Maple Grove|Carver County|MN|58252|United States|-6|single family| +23741|AAAAAAAANLMFAAAA|39|South |Way|Suite 340|Riverside|Barnwell County|SC|29231|United States|-5|single family| +23742|AAAAAAAAOLMFAAAA|294|13th Ash|Dr.|Suite N|Farmington|Gladwin County|MI|49145|United States|-5|condo| +23743|AAAAAAAAPLMFAAAA|555|Park |Ct.|Suite Y|Oakwood|Thomas County|NE|60169|United States|-7|single family| +23744|AAAAAAAAAMMFAAAA|773|East |Avenue|Suite 480|Highland Park|Cimarron County|OK|76534|United States|-6|single family| +23745|AAAAAAAABMMFAAAA|777|10th |Wy|Suite U|Fairfield|Butte County|CA|96192|United States|-8|condo| +23746|AAAAAAAACMMFAAAA|873|Railroad |Dr.|Suite 150|Bethel|Highland County|OH|45281|United States|-5|single family| +23747|AAAAAAAADMMFAAAA|782||Blvd||Belleview|Calhoun County|MS|50492||-6|| +23748|AAAAAAAAEMMFAAAA|385|Jefferson |Avenue|Suite P|Spring Hill|Valencia County|NM|86787|United States|-7|single family| +23749|AAAAAAAAFMMFAAAA|294|First |Boulevard|Suite O|Hamilton|Yuba County|CA|92808|United States|-8|condo| +23750|AAAAAAAAGMMFAAAA|697|Green |Circle|Suite 330|Pleasant Grove|Dooly County|GA|34136|United States|-5|apartment| +23751|AAAAAAAAHMMFAAAA|539|Walnut Pine|Court|Suite 10|Sunnyside|Nicollet County|MN|51952|United States|-6|single family| +23752|AAAAAAAAIMMFAAAA|381|Ridge 6th|Ave|Suite 140|Greenwood|Floyd County|KY|48828|United States|-6|condo| +23753|AAAAAAAAJMMFAAAA|238|Jackson East|RD|Suite X|Pine Grove|Madison County|NE|64593|United States|-7|single family| +23754|AAAAAAAAKMMFAAAA|892|Lake |Cir.|Suite C|Buena Vista|Ballard County|KY|45752|United States|-6|apartment| +23755|AAAAAAAALMMFAAAA|68|4th |Lane|Suite E|Riverview|Erie County|PA|19003|United States|-5|apartment| +23756|AAAAAAAAMMMFAAAA|457|Hillcrest Hickory|Dr.|Suite 410|Highland Park|McNairy County|TN|36534|United States|-6|apartment| +23757|AAAAAAAANMMFAAAA|679|Maple Jackson|Pkwy|Suite 280|Edgewood|Early County|GA|30069|United States|-5|single family| +23758|AAAAAAAAOMMFAAAA|464|Dogwood River|Drive|Suite S|Deerfield|Logan County|KY|49840|United States|-5|apartment| +23759|AAAAAAAAPMMFAAAA|429|Oak |Avenue|Suite F|Five Points|Kent County|MD|26098|United States|-5|condo| +23760|AAAAAAAAANMFAAAA|891|Woodland |Way|Suite 80|Kingston|Los Angeles County|CA|94975|United States|-8|condo| +23761|AAAAAAAABNMFAAAA|949|Washington |Court|Suite 270|Clifton|Montgomery County|VA|28014|United States|-5|apartment| +23762|AAAAAAAACNMFAAAA|498|Hill |Blvd|Suite 250|White Oak|Greene County|VA|26668|United States|-5|apartment| +23763|AAAAAAAADNMFAAAA|703|10th 5th|Parkway|Suite 20|Pleasant Hill|Trinity County|CA|93604|United States|-8|apartment| +23764|AAAAAAAAENMFAAAA|342|Lake |Court|Suite 90|Bethel|Bastrop County|TX|75281|United States|-6|condo| +23765|AAAAAAAAFNMFAAAA|541|Lee 9th|Court|Suite 200|Shady Grove|Rogers County|OK|72812|United States|-6|condo| +23766|AAAAAAAAGNMFAAAA|191|Second Main||Suite 10||Grant County|||United States|-6|single family| +23767|AAAAAAAAHNMFAAAA|151|7th |Avenue|Suite 230|Frogtown|Hinds County|MS|58784|United States|-6|condo| +23768|AAAAAAAAINMFAAAA|474|Lee First|Wy|Suite 360|Centerville|Suffolk County|NY|10059|United States|-5|apartment| +23769|AAAAAAAAJNMFAAAA|593|Church Cedar|ST|Suite 70|Ashland|Power County|ID|84244|United States|-7|apartment| +23770|AAAAAAAAKNMFAAAA|172|Ridge Smith|Lane|Suite G|Ashland|York County|SC|24244|United States|-5|condo| +23771|AAAAAAAALNMFAAAA|804|3rd Hill|Wy|Suite S|Mount Olive|Ocean County|NJ|08659|United States|-5|single family| +23772|AAAAAAAAMNMFAAAA|417|Highland |Ln|Suite Q|Oak Ridge|Big Stone County|MN|58371|United States|-6|condo| +23773|AAAAAAAANNMFAAAA|864|Locust Cedar|Wy|Suite D|Riverview|Manatee County|FL|39003|United States|-5|single family| +23774|AAAAAAAAONMFAAAA|288|4th |Drive|Suite 290|Springfield|DeSoto County|MS|59303|United States|-6|condo| +23775|AAAAAAAAPNMFAAAA|904|Sycamore |Road|Suite K|Springhill|Wythe County|VA|24602|United States|-5|apartment| +23776|AAAAAAAAAOMFAAAA|230|First |Ln|Suite W|Jackson|Morris County|KS|69583|United States|-6|single family| +23777|AAAAAAAABOMFAAAA|286|Lake |Road|Suite A|Woodlawn|Camden County|GA|34098|United States|-5|condo| +23778|AAAAAAAACOMFAAAA|863|Forest Jefferson|Ave|Suite O|Bethlehem|Luce County|MI|45325|United States|-5|condo| +23779|AAAAAAAADOMFAAAA|884|13th |RD|Suite D|Pleasant Valley|Goliad County|TX|72477|United States|-6|condo| +23780|AAAAAAAAEOMFAAAA|815|Meadow Maple|Ave|Suite A|Leland|Antelope County|NE|69452|United States|-6|condo| +23781|AAAAAAAAFOMFAAAA|213|Woodland View|Parkway|Suite 480|Marion|Alfalfa County|OK|70399|United States|-6|apartment| +23782|AAAAAAAAGOMFAAAA|643|1st North|Lane|Suite 330|Macedonia|Kiowa County|KS|61087|United States|-6|apartment| +23783|AAAAAAAAHOMFAAAA|919|15th Ridge|Boulevard|Suite V|Glenwood|Walker County|TX|73511|United States|-6|condo| +23784|AAAAAAAAIOMFAAAA|683|Highland |Ct.|Suite B|Union Hill|Porter County|IN|47746|United States|-5|apartment| +23785|AAAAAAAAJOMFAAAA|303|Cedar Sunset|Ln|Suite 110|Mount Vernon|Day County|SD|58482|United States|-6|condo| +23786|AAAAAAAAKOMFAAAA|800|5th |Way|Suite 360|Farmington|Webster County|WV|29145|United States|-5|apartment| +23787|AAAAAAAALOMFAAAA|941|1st |Lane|Suite E|Greenville|Stone County|AR|71387|United States|-6|condo| +23788|AAAAAAAAMOMFAAAA||Jefferson Hickory|Drive||||||||| +23789|AAAAAAAANOMFAAAA|94|Cedar Main|Ct.|Suite T|Lawrenceville|Alcona County|MI|44462|United States|-5|condo| +23790|AAAAAAAAOOMFAAAA|555|3rd |Court|Suite 410|Union|Caroline County|MD|28721|United States|-5|apartment| +23791|AAAAAAAAPOMFAAAA|371|Willow Lincoln|RD|Suite G|Pleasant Valley|Kankakee County|IL|62477|United States|-6|condo| +23792|AAAAAAAAAPMFAAAA|254|5th |Lane|Suite E|Wildwood|Oklahoma County|OK|76871|United States|-6|apartment| +23793|AAAAAAAABPMFAAAA|406|River 14th|Ln|Suite S|Crossroads|Harvey County|KS|60534|United States|-6|condo| +23794|AAAAAAAACPMFAAAA|283|Hillcrest Smith|Wy|Suite G|Plainview|Greeley County|NE|63683|United States|-6|condo| +23795|AAAAAAAADPMFAAAA|205|Oak Maple|Ave|Suite 170|Centerville|Elbert County|GA|30059|United States|-5|condo| +23796|AAAAAAAAEPMFAAAA|92|Lincoln |Ln|Suite 230|Newtown|Lauderdale County|TN|31749|United States|-6|condo| +23797|AAAAAAAAFPMFAAAA|381|First Hill|Court|Suite L|Antioch|Harris County|TX|78605|United States|-6|single family| +23798|AAAAAAAAGPMFAAAA|852|East Hickory|RD|Suite B|Woodbury|Tuolumne County|CA|94489|United States|-8|apartment| +23799|AAAAAAAAHPMFAAAA|841|Second Jefferson|Avenue|Suite C|Woodland|Yolo County|CA|94854|United States|-8|condo| +23800|AAAAAAAAIPMFAAAA|776|Second Miller|Pkwy|Suite 100|Greenville|Hardin County|IL|61387|United States|-6|apartment| +23801|AAAAAAAAJPMFAAAA|4|Railroad |Way|Suite V|Leesville|Los Alamos County|NM|85423|United States|-7|apartment| +23802|AAAAAAAAKPMFAAAA|242|4th |Lane|Suite V|Oakdale|Marshall County|WV|29584|United States|-5|apartment| +23803|AAAAAAAALPMFAAAA|181|Hill |Circle|Suite 230|Belmont|Washington County|CO|80191|United States|-7|condo| +23804|AAAAAAAAMPMFAAAA|608|||Suite B||||87057|United States||| +23805|AAAAAAAANPMFAAAA|559|First Sycamore|Ct.|Suite 360|Maple Grove|Coffey County|KS|68252|United States|-6|single family| +23806|AAAAAAAAOPMFAAAA|113|Dogwood |Boulevard|Suite Q|Crossroads|Camp County|TX|70534|United States|-6|single family| +23807|AAAAAAAAPPMFAAAA|787|Second 2nd|Ln|Suite 420|Maple Grove|Daviess County|MO|68252|United States|-6|single family| +23808|AAAAAAAAAANFAAAA|783|Spruce |ST|Suite 220|Spring Hill|Harris County|GA|36787|United States|-5|condo| +23809|AAAAAAAABANFAAAA|650|Ash |Parkway|Suite 260|Glenwood|Saratoga County|NY|13511|United States|-5|condo| +23810|AAAAAAAACANFAAAA|447|Miller Laurel|Pkwy|Suite 170|Springfield|Dallam County|TX|79303|United States|-6|condo| +23811|AAAAAAAADANFAAAA|355|Davis Park|Boulevard|Suite 480|Oak Ridge|Simpson County|MS|58371|United States|-6|condo| +23812|AAAAAAAAEANFAAAA|965|Third 8th|ST|Suite 30|Salem|Campbell County|WY|88048|United States|-7|condo| +23813|AAAAAAAAFANFAAAA|283|Third |RD|Suite 100|Clinton|Camp County|TX|78222|United States|-6|single family| +23814|AAAAAAAAGANFAAAA|||Boulevard|||Monterey County|CA||||condo| +23815|AAAAAAAAHANFAAAA|429|3rd |Ln|Suite 420|Five Points|Newport News city|VA|26098|United States|-5|condo| +23816|AAAAAAAAIANFAAAA|777|14th Johnson|ST|Suite 230|Fairview|Lincoln County|WY|85709|United States|-7|condo| +23817|AAAAAAAAJANFAAAA|964|Twelfth |Lane|Suite O|Jackson|Madison County|AR|79583|United States|-6|condo| +23818|AAAAAAAAKANFAAAA|495|Elm Locust|Ct.|Suite 410|Oakwood|Jones County|IA|50169|United States|-6|condo| +23819|AAAAAAAALANFAAAA|967|South 13th|Dr.|Suite W|Enterprise|Northampton County|PA|11757|United States|-5|apartment| +23820|AAAAAAAAMANFAAAA|238|Birch Chestnut|Parkway|Suite 470|Morris|Grays Harbor County|WA|96696|United States|-8|condo| +23821|AAAAAAAANANFAAAA|320|Lake |Street|Suite B|Salem|Anson County|NC|28048|United States|-5|condo| +23822|AAAAAAAAOANFAAAA|596|Ridge |Dr.|Suite 170|Antioch|Bracken County|KY|48605|United States|-6|condo| +23823|AAAAAAAAPANFAAAA|349|Central Chestnut|Ln|Suite C|Jackson|Charlotte County|VA|29583|United States|-5|apartment| +23824|AAAAAAAAABNFAAAA|515|Sunset |Boulevard|Suite 30|Plainview|Hawaii County|HI|93683|United States|-10|apartment| +23825|AAAAAAAABBNFAAAA|962|Spring 7th|Parkway|Suite N|Midway|Wheatland County|MT|61904|United States|-7|condo| +23826|AAAAAAAACBNFAAAA|||RD|||||||-7|condo| +23827|AAAAAAAADBNFAAAA|793|Park |Avenue|Suite 150|Glenwood|Crow Wing County|MN|53511|United States|-6|single family| +23828|AAAAAAAAEBNFAAAA|590|1st |Boulevard|Suite V|Centerville|Gila County|AZ|80059|United States|-7|apartment| +23829|AAAAAAAAFBNFAAAA|668|Adams Hillcrest|Boulevard|Suite A|Greenville|Carbon County|WY|81387|United States|-7|apartment| +23830|AAAAAAAAGBNFAAAA|303|8th Central|Avenue|Suite C|Hillcrest|Franklin County|NC|23003|United States|-5|apartment| +23831|AAAAAAAAHBNFAAAA||Birch |Circle|Suite 470|Mount Vernon|||68482|United States|-6|| +23832|AAAAAAAAIBNFAAAA|282|Oak Willow|Drive|Suite 260|Mount Zion|Wells County|ND|58054|United States|-6|single family| +23833|AAAAAAAAJBNFAAAA|444|Oak |Cir.|Suite K|Lakewood|Bon Homme County|SD|58877|United States|-6|condo| +23834|AAAAAAAAKBNFAAAA|484|Hickory |Drive|Suite J|Greenfield|Jackson County|SD|55038|United States|-7|single family| +23835|AAAAAAAALBNFAAAA|554|Hill 2nd|Way|Suite 220|Five Points|Columbia County|PA|16098|United States|-5|single family| +23836|AAAAAAAAMBNFAAAA|663|Willow Poplar|Ln|Suite J|Highland Park|Greene County|NC|26534|United States|-5|apartment| +23837|AAAAAAAANBNFAAAA|742|Church Railroad|Boulevard|Suite C|Hillcrest|Berkeley County|WV|23003|United States|-5|condo| +23838|AAAAAAAAOBNFAAAA|889|2nd Park|Ave|Suite 40|Friendship|Hamilton County|TN|34536|United States|-5|apartment| +23839|AAAAAAAAPBNFAAAA|49|Lake Madison|Road|Suite 260|Hillcrest|Wade Hampton Census Area|AK|93003|United States|-9|apartment| +23840|AAAAAAAAACNFAAAA|296|Mill |Pkwy|Suite F|Liberty|Umatilla County|OR|93451|United States|-8|single family| +23841|AAAAAAAABCNFAAAA|85|Poplar |ST|Suite 130|Oak Ridge|Colbert County|AL|38371|United States|-6|single family| +23842|AAAAAAAACCNFAAAA|692|South College|Blvd|Suite 320|Greenwood|Mecklenburg County|NC|28828|United States|-5|condo| +23843|AAAAAAAADCNFAAAA|983|Pine |Pkwy||Brookville||||United States|-6|| +23844|AAAAAAAAECNFAAAA|491|5th Meadow|Ct.|Suite 150|Edgewood|Kennebec County|ME|00669|United States|-5|condo| +23845|AAAAAAAAFCNFAAAA|738|Fifth Cherry|Way|Suite 400|Glenwood|Cedar County|NE|63511|United States|-6|single family| +23846|AAAAAAAAGCNFAAAA|333|Main |ST|Suite 400|Five Forks|Grayson County|KY|42293|United States|-6|condo| +23847|AAAAAAAAHCNFAAAA|104|Spring Forest|Ave|Suite 380|Glendale|Coos County|NH|04551|United States|-5|single family| +23848|AAAAAAAAICNFAAAA|392|Cherry Johnson|Ave|Suite E|Brownsville|Huron County|OH|49310|United States|-5|apartment| +23849|AAAAAAAAJCNFAAAA|676|Davis 2nd|Pkwy|Suite 120|Centerville|Breckinridge County|KY|40059|United States|-6|condo| +23850|AAAAAAAAKCNFAAAA|906|Sycamore |Parkway|Suite B|San Jose|Fulton County|AR|78003|United States|-6|single family| +23851|AAAAAAAALCNFAAAA|||||New Hope|Grant County|KY|49431||-6|single family| +23852|AAAAAAAAMCNFAAAA|896|Main |Street|Suite 490|Lakeside|Jerauld County|SD|59532|United States|-7|apartment| +23853|AAAAAAAANCNFAAAA|38|3rd 8th|Way|Suite A|Alpha|Nelson County|VA|20586|United States|-5|condo| +23854|AAAAAAAAOCNFAAAA|870|First |Street|Suite Q|Salem|Cochran County|TX|78048|United States|-6|condo| +23855|AAAAAAAAPCNFAAAA|628|Green Green|Drive|Suite 380|Newport||IN|||-5|| +23856|AAAAAAAAADNFAAAA|209|Third |Parkway|Suite D|Lakeside|Cibola County|NM|89532|United States|-7|single family| +23857|AAAAAAAABDNFAAAA|685|11th Valley|RD|Suite 70|Red Hill|Drew County|AR|74338|United States|-6|single family| +23858|AAAAAAAACDNFAAAA|712|13th Cedar|Boulevard|Suite D|Oakdale|Vermilion County|IL|69584|United States|-6|single family| +23859|AAAAAAAADDNFAAAA|783|Highland River|Lane|Suite Y|Pleasant Grove|Thomas County|GA|34136|United States|-5|single family| +23860|AAAAAAAAEDNFAAAA|270|9th |Street|Suite 210|Oak Grove|Suffolk County|NY|18370|United States|-5|condo| +23861|AAAAAAAAFDNFAAAA|954|Lincoln Ridge|RD|Suite B|Oak Grove|Marion County|TX|78370|United States|-6|single family| +23862|AAAAAAAAGDNFAAAA|387|Sunset 4th|Lane|Suite E|Arlington|Carlisle County|KY|46557|United States|-6|condo| +23863|AAAAAAAAHDNFAAAA|676|Highland 2nd|Drive|Suite Y|Five Forks|Powder River County|MT|62293|United States|-7|condo| +23864|AAAAAAAAIDNFAAAA|205|Forest |Blvd|Suite H|Shiloh|Bladen County|NC|29275|United States|-5|condo| +23865|AAAAAAAAJDNFAAAA|341|Cherry |Street|Suite 80|Glenville|Kennebec County|ME|04045|United States|-5|single family| +23866|AAAAAAAAKDNFAAAA|51|Railroad |Avenue|Suite 80|Lakewood|Morgan County|OH|48877|United States|-5|apartment| +23867|AAAAAAAALDNFAAAA|144|Forest |Pkwy|Suite K|Clearview|Dickens County|TX|75495|United States|-6|apartment| +23868|AAAAAAAAMDNFAAAA|802|10th |Blvd|Suite K|Union Hill|Edgecombe County|NC|27746|United States|-5|apartment| +23869|AAAAAAAANDNFAAAA|282|Madison |Dr.|Suite C|Hillcrest|Lincoln County|NE|63003|United States|-7|single family| +23870|AAAAAAAAODNFAAAA|440|Oak Adams|ST|Suite R|Riverdale|Harnett County|NC|29391|United States|-5|apartment| +23871|AAAAAAAAPDNFAAAA|986|First Adams|Parkway|Suite 380|Oak Ridge|West Carroll Parish|LA|78371|United States|-6|single family| +23872|AAAAAAAAAENFAAAA|893|Center 5th|Parkway|Suite K|Hopewell|Thayer County|NE|60587|United States|-7|apartment| +23873|AAAAAAAABENFAAAA|393|Dogwood 5th|Ave|Suite O|Five Forks|Noble County|OK|72293|United States|-6|condo| +23874|AAAAAAAACENFAAAA|663|Locust |Ave|Suite G|Stringtown|Orangeburg County|SC|20162|United States|-5|apartment| +23875|AAAAAAAADENFAAAA|560|Jackson |Cir.|Suite 10|Oakland|Fremont County|CO|89843|United States|-7|single family| +23876|AAAAAAAAEENFAAAA|537|View North|Circle|Suite 120|Macedonia|Warren County|IL|61087|United States|-6|condo| +23877|AAAAAAAAFENFAAAA|106|Spring 9th|Parkway|Suite 400|Shannon|Trumbull County|OH|44120|United States|-5|apartment| +23878|AAAAAAAAGENFAAAA|806|5th |Street|Suite 490|Bunker Hill|Amador County|CA|90150|United States|-8|single family| +23879|AAAAAAAAHENFAAAA|298|15th |Parkway|Suite 220|Wilson|Maries County|MO|66971|United States|-6|single family| +23880|AAAAAAAAIENFAAAA|803|Lincoln |ST|Suite 70|Enterprise|Tipton County|TN|31757|United States|-6|single family| +23881|AAAAAAAAJENFAAAA|352|Laurel |ST|Suite M|Centerville|Washtenaw County|MI|40059|United States|-5|condo| +23882|AAAAAAAAKENFAAAA|917|Johnson |Avenue|Suite E|Brownsville|Maries County|MO|69310|United States|-6|apartment| +23883|AAAAAAAALENFAAAA|183|11th Fourth|Parkway|Suite 440|Riverside|Coahoma County|MS|59231|United States|-6|condo| +23884|AAAAAAAAMENFAAAA|958|14th Fourth|Way|Suite G|Stringtown|Crook County|OR|90162|United States|-8|condo| +23885|AAAAAAAANENFAAAA|27|Lincoln |Ln|Suite 90|Hillcrest|Mingo County|WV|23003|United States|-5|apartment| +23886|AAAAAAAAOENFAAAA|691|South South|Road|Suite J|Golden|Brown County|KS|60411|United States|-6|single family| +23887|AAAAAAAAPENFAAAA|431|Poplar Jefferson|ST|Suite G|Hopewell|Gasconade County|MO|60587|United States|-6|apartment| +23888|AAAAAAAAAFNFAAAA|974|9th Oak|Ct.|Suite B|Jamestown|Teton County|ID|86867|United States|-7|condo| +23889|AAAAAAAABFNFAAAA|210|Dogwood 6th|Cir.|Suite 240|Mountain View|Hudspeth County|TX|74466|United States|-6|apartment| +23890|AAAAAAAACFNFAAAA|155|First |Drive|Suite S|Bunker Hill|Ford County|KS|60150|United States|-6|single family| +23891|AAAAAAAADFNFAAAA|150|Birch |Court|Suite 300|Riley|La Plata County|CO|81692|United States|-7|single family| +23892|AAAAAAAAEFNFAAAA|625|Cedar Ridge|Court|Suite Y|Lakeview|Penobscot County|ME|09179|United States|-5|condo| +23893|AAAAAAAAFFNFAAAA|186|3rd Valley||||||69840|United States|-6|apartment| +23894|AAAAAAAAGFNFAAAA|740|1st |Way|Suite 300|Newtown|Atkinson County|GA|31749|United States|-5|single family| +23895|AAAAAAAAHFNFAAAA|709|6th |Pkwy|Suite K|Newport|DeKalb County|TN|31521|United States|-5|apartment| +23896|AAAAAAAAIFNFAAAA|932|Smith Second|Lane|Suite 250|Midway|Nobles County|MN|51904|United States|-6|condo| +23897|AAAAAAAAJFNFAAAA|279|6th |Cir.|Suite E|Edgewood|Monroe County|WV|20069|United States|-5|single family| +23898|AAAAAAAAKFNFAAAA|921|Center |Drive|Suite 280|Shiloh|Madison County|MT|69275|United States|-7|apartment| +23899|AAAAAAAALFNFAAAA|314|Wilson |ST|Suite 180|Oak Grove|Perry County|OH|48370|United States|-5|apartment| +23900|AAAAAAAAMFNFAAAA|556|3rd |Parkway|Suite L|Plainview|Fairbanks North Star Borough|AK|93683|United States|-9|single family| +23901|AAAAAAAANFNFAAAA|209|Pine |Ln|Suite Y|Franklin|Clay County|AR|79101|United States|-6|apartment| +23902|AAAAAAAAOFNFAAAA|224|Maple |Road|Suite A|Lakeside|Oconee County|GA|39532|United States|-5|apartment| +23903|AAAAAAAAPFNFAAAA|192|9th Sunset|Blvd|Suite E|Woodland|Glynn County|GA|34854|United States|-5|single family| +23904|AAAAAAAAAGNFAAAA|464|12th |ST|Suite 180|Harmony|Alameda County|CA|95804|United States|-8|condo| +23905|AAAAAAAABGNFAAAA|237|Maple |Way|Suite S|Concord|Cabell County|WV|24107|United States|-5|single family| +23906|AAAAAAAACGNFAAAA|378|Spruce |Avenue|Suite 440|Buckingham|Norman County|MN|54092|United States|-6|apartment| +23907|AAAAAAAADGNFAAAA|810|13th Hillcrest|Boulevard|Suite 360|Lincoln|Rutherford County|TN|31289|United States|-6|condo| +23908|AAAAAAAAEGNFAAAA|256|4th |Dr.|Suite Y|Five Forks|Virginia Beach city|VA|22293|United States|-5|condo| +23909|AAAAAAAAFGNFAAAA|143|Smith 10th|Ln|Suite 250|Mount Vernon|Bryan County|OK|78482|United States|-6|condo| +23910|AAAAAAAAGGNFAAAA|722|Cherry Davis|Ln|Suite H|Highland|Maui County|HI|99454|United States|-10|apartment| +23911|AAAAAAAAHGNFAAAA|761|Elm |ST|Suite I|Hopewell|McCreary County|KY|40587|United States|-5|single family| +23912|AAAAAAAAIGNFAAAA|710|Sunset |Road|Suite F|Glendale|Putnam County|MO|63951|United States|-6|apartment| +23913|AAAAAAAAJGNFAAAA|249|Main View|Avenue|Suite 210|Roy|Georgetown County|SC|20744|United States|-5|condo| +23914|AAAAAAAAKGNFAAAA|966|13th |Ave|Suite 80|Cedar Grove|Saginaw County|MI|40411|United States|-5|single family| +23915|AAAAAAAALGNFAAAA|201|Pine Third|Dr.|Suite 70|Pine Grove|Greene County|NC|24593|United States|-5|single family| +23916|AAAAAAAAMGNFAAAA|780|College 1st|Court|Suite 340|Collinsville|Ritchie County|WV|22459|United States|-5|condo| +23917|AAAAAAAANGNFAAAA|516|||||Williams County|OH|43788|United States||apartment| +23918|AAAAAAAAOGNFAAAA|643|3rd |Ave|Suite A|Concord|Cottle County|TX|74107|United States|-6|single family| +23919|AAAAAAAAPGNFAAAA|203|3rd |Ave|Suite 180|Enterprise|Leavenworth County|KS|61757|United States|-6|single family| +23920|AAAAAAAAAHNFAAAA|444|1st 2nd|Parkway|Suite V|Union|Allen County|KS|68721|United States|-6|single family| +23921|AAAAAAAABHNFAAAA|844|Locust Elm|Lane|Suite S|Pleasant Grove|Woodruff County|AR|74136|United States|-6|apartment| +23922|AAAAAAAACHNFAAAA|961|Church |Street|Suite 480|Centerville|Phillips County|KS|60059|United States|-6|apartment| +23923|AAAAAAAADHNFAAAA|946|7th Second|Boulevard|Suite 480|Five Forks|Platte County|WY|82293|United States|-7|apartment| +23924|AAAAAAAAEHNFAAAA|672|Chestnut |Street|Suite 300|Westgate|Union County|IL|62366|United States|-6|condo| +23925|AAAAAAAAFHNFAAAA|91|Hill |Ave|Suite F|Montague|Kingman County|KS|64062|United States|-6|single family| +23926|AAAAAAAAGHNFAAAA|719|4th Maple|Ln|Suite J|Lebanon|Fulton County|IN|42898|United States|-5|condo| +23927|AAAAAAAAHHNFAAAA|311|7th |Way|Suite R|Lakeview|Herkimer County|NY|18579|United States|-5|apartment| +23928|AAAAAAAAIHNFAAAA|187|7th |Avenue|Suite J|Unionville|Charles County|MD|21711|United States|-5|single family| +23929|AAAAAAAAJHNFAAAA|833|10th |Blvd|Suite X|Milo|Clay County|KY|40116|United States|-6|apartment| +23930|AAAAAAAAKHNFAAAA|646|7th Poplar|Ct.|Suite 90|Sulphur Springs|Mahaska County|IA|58354|United States|-6|apartment| +23931|AAAAAAAALHNFAAAA|726|Elm |Ave|Suite P|Enterprise|Kearny County|KS|61757|United States|-6|single family| +23932|AAAAAAAAMHNFAAAA|472|Jackson |Lane|Suite 300|Newtown|Glynn County|GA|31749|United States|-5|single family| +23933|AAAAAAAANHNFAAAA|359|Mill |Pkwy|Suite X|Highland Park|Iosco County|MI|46534|United States|-5|condo| +23934|AAAAAAAAOHNFAAAA|988|Williams |Cir.|Suite 430|Wilton|Woodward County|OK|76997|United States|-6|condo| +23935|AAAAAAAAPHNFAAAA|769|Woodland |RD|Suite O|Arlington|Prentiss County|MS|56557|United States|-6|apartment| +23936|AAAAAAAAAINFAAAA|742|Hill |Ln|Suite 130|Pine Grove|Appling County|GA|34593|United States|-5|single family| +23937|AAAAAAAABINFAAAA|760|1st Smith|Dr.|Suite C|Buena Vista|Blaine County|MT|65752|United States|-7|condo| +23938|AAAAAAAACINFAAAA|570|Mill East|Pkwy|Suite R|Friendship|Bailey County|TX|74536|United States|-6|condo| +23939|AAAAAAAADINFAAAA|935|6th |Avenue|Suite K|Riverdale|Irwin County|GA|39391|United States|-5|condo| +23940|AAAAAAAAEINFAAAA|40|Park Franklin|Ct.|Suite 250|Maple Grove|Teton County|ID|88252|United States|-7|single family| +23941|AAAAAAAAFINFAAAA|468|5th Hill|Blvd|Suite 230|Stringtown|Gadsden County|FL|30162|United States|-5|single family| +23942|AAAAAAAAGINFAAAA|922|Hillcrest |RD|Suite 490|Five Forks|Washington County|VT|02893|United States|-5|apartment| +23943|AAAAAAAAHINFAAAA|578|Ash |Boulevard|Suite C|Pleasant Grove|Colfax County|NE|64136|United States|-6|single family| +23944|AAAAAAAAIINFAAAA|16|Cedar |Circle|Suite Q|Mount Vernon|Harrison County|KY|48482|United States|-6|apartment| +23945|AAAAAAAAJINFAAAA|977|Lake 7th|Parkway|Suite S|Oak Grove|Douglas County|OR|98370|United States|-8|condo| +23946|AAAAAAAAKINFAAAA|261|Ridge Davis|Dr.|Suite Y|Pleasant Grove|Nance County|NE|64136|United States|-7|apartment| +23947|AAAAAAAALINFAAAA|980|Fourth Ridge|Parkway|Suite F|Lakeside|Lucas County|OH|49532|United States|-5|apartment| +23948|AAAAAAAAMINFAAAA|609|Park Meadow|Boulevard|Suite 320|Stringtown|Campbell County|KY|40162|United States|-6|condo| +23949|AAAAAAAANINFAAAA|562|Hillcrest 6th|Wy|Suite 110|Oakland|Summit County|OH|49843|United States|-5|single family| +23950|AAAAAAAAOINFAAAA|288|Jefferson 4th|Drive|Suite 240|Oak Hill|Callahan County|TX|77838|United States|-6|apartment| +23951|AAAAAAAAPINFAAAA|165|11th Fourth|Boulevard|Suite W|Unionville|Lexington County|SC|21711|United States|-5|apartment| +23952|AAAAAAAAAJNFAAAA|980|Ridge |Ln|Suite 160|Glendale|Wells County|IN|43951|United States|-5|single family| +23953|AAAAAAAABJNFAAAA|563|Sunset |Circle|Suite R|Fairview|Bon Homme County|SD|55709|United States|-6|condo| +23954|AAAAAAAACJNFAAAA|642|Third |Drive|Suite 460|Oakwood|Herkimer County|NY|10169|United States|-5|single family| +23955|AAAAAAAADJNFAAAA|500|Church |Cir.|Suite 400|Pleasant Valley|Crawford County|IN|42477|United States|-5|single family| +23956|AAAAAAAAEJNFAAAA|815|4th |Avenue|Suite 330|Oakland|Big Horn County|WY|89843|United States|-7|single family| +23957|AAAAAAAAFJNFAAAA|965|||Suite 460|Mountain View|Noxubee County|MS||||| +23958|AAAAAAAAGJNFAAAA|701|Hillcrest 3rd|ST|Suite Q|Summit|Howard County|AR|70499|United States|-6|condo| +23959|AAAAAAAAHJNFAAAA|572|10th 1st|Drive|Suite M|Marion|Lancaster County|PA|10399|United States|-5|apartment| +23960|AAAAAAAAIJNFAAAA|211|9th |Ct.|Suite 410|Concord|Montgomery County|AR|74107|United States|-6|condo| +23961|AAAAAAAAJJNFAAAA|404|Birch |Wy|Suite Q|Oakland|Rockwall County|TX|79843|United States|-6|single family| +23962|AAAAAAAAKJNFAAAA|913|First Lincoln|Court|Suite T|Springdale|Alexander County|NC|28883|United States|-5|condo| +23963|AAAAAAAALJNFAAAA|600|4th |ST|Suite U|Brownsville|Knox County|IN|49310|United States|-5|single family| +23964|AAAAAAAAMJNFAAAA|187|Birch Railroad|Dr.|Suite 370|Mount Vernon|Kenton County|KY|48482|United States|-5|single family| +23965|AAAAAAAANJNFAAAA|644|Cherry |Boulevard|Suite 380|Macedonia|Rock County|NE|61087|United States|-7|single family| +23966|AAAAAAAAOJNFAAAA|342|Pine 7th|Pkwy|Suite K|Greenfield|Walthall County|MS|55038|United States|-6|apartment| +23967|AAAAAAAAPJNFAAAA|706|River |Cir.|Suite E|Woodland|Lyman County|SD|54854|United States|-7|condo| +23968|AAAAAAAAAKNFAAAA|114|Wilson Jefferson|Ave|Suite D|Oakdale|Lewis County|WV|29584|United States|-5|apartment| +23969|AAAAAAAABKNFAAAA|144|Washington Maple|Road|Suite 230|Oak Hill|Saluda County|SC|27838|United States|-5|condo| +23970|AAAAAAAACKNFAAAA|924|View |Dr.|Suite F|Deerfield|Caldwell County|TX|79840|United States|-6|single family| +23971|AAAAAAAADKNFAAAA|434|3rd Elm|Street|Suite 280|Oak Grove|Stone County|MO|68370|United States|-6|apartment| +23972|AAAAAAAAEKNFAAAA|22|Ridge View|Way|Suite 30|Acme|Decatur County|IN|40164|United States|-5|apartment| +23973|AAAAAAAAFKNFAAAA||Ridge |Drive|Suite 480|Oak Hill|Delaware County|IA|||-6|apartment| +23974|AAAAAAAAGKNFAAAA|689|North |Pkwy|Suite 80|Sunnyside|Webster County|WV|21952|United States|-5|apartment| +23975|AAAAAAAAHKNFAAAA|763|Laurel Hickory|Dr.|Suite X|Five Points|Shackelford County|TX|76098|United States|-6|single family| +23976|AAAAAAAAIKNFAAAA|480|Hickory |Blvd|Suite 100|Green Acres|Wayne County|NC|27683|United States|-5|apartment| +23977|AAAAAAAAJKNFAAAA|218|Thirteenth 11th|Pkwy|Suite 280|Glenwood|Carter County|KY|43511|United States|-6|apartment| +23978|AAAAAAAAKKNFAAAA|690|11th |Lane|Suite 330|Yorktown|Moore County|TN|30732|United States|-6|condo| +23979|AAAAAAAALKNFAAAA|625|4th West|Way|Suite A|Brownsville|Crawford County|KS|69310|United States|-6|single family| +23980|AAAAAAAAMKNFAAAA|571|West |Way|Suite 350|Wilton|Fayette County|GA|36997|United States|-5|single family| +23981|AAAAAAAANKNFAAAA|594|15th Eigth|ST|Suite E|Mount Vernon|McIntosh County|GA|38482|United States|-5|condo| +23982|AAAAAAAAOKNFAAAA|744|Sunset |Court|Suite 230|Riverside|Franklin County|MA|09831|United States|-5|single family| +23983|AAAAAAAAPKNFAAAA|294|9th |RD|Suite K|Oak Hill|Platte County|WY|87838|United States|-7|apartment| +23984|AAAAAAAAALNFAAAA|573|Sycamore Poplar|Dr.|Suite R|Lebanon|Van Buren County|IA|52898|United States|-6|condo| +23985|AAAAAAAABLNFAAAA|504|Willow 6th|Road|Suite 80|Clinton|Kalkaska County|MI|48222|United States|-5|condo| +23986|AAAAAAAACLNFAAAA|842|Oak |Drive|Suite 290|Clinton|Morton County|ND|58222|United States|-6|condo| +23987|AAAAAAAADLNFAAAA|322|6th |RD|Suite W|Glendale|Creek County|OK|73951|United States|-6|single family| +23988|AAAAAAAAELNFAAAA|559|Dogwood Cherry|RD|Suite 20|Bridgeport|Adams County|ID|85817|United States|-7|single family| +23989|AAAAAAAAFLNFAAAA|692|Woodland |Drive|Suite S|Royal|Muskingum County|OH|45819|United States|-5|condo| +23990|AAAAAAAAGLNFAAAA|933|Lee |Drive|Suite F|Crossroads|Sangamon County|IL|60534|United States|-6|condo| +23991|AAAAAAAAHLNFAAAA|877|Chestnut 1st|Dr.|Suite Y|Five Points|Oscoda County|MI|46098|United States|-5|apartment| +23992|AAAAAAAAILNFAAAA|722|10th North|Lane|Suite X|Five Forks|Randolph County|IN|42293|United States|-5|apartment| +23993|AAAAAAAAJLNFAAAA|401|Park |Parkway|Suite X|New Hope|Howell County|MO|69431|United States|-6|single family| +23994|AAAAAAAAKLNFAAAA|628|Sunset |Lane|Suite 380|Hamilton|Wilcox County|GA|32808|United States|-5|condo| +23995|AAAAAAAALLNFAAAA|877|Highland |Boulevard|Suite M|Leon|Kearney County|NE|60913|United States|-7|single family| +23996|AAAAAAAAMLNFAAAA|138|4th Mill|Ln|Suite O|Newtown|Multnomah County|OR|91749|United States|-8|apartment| +23997|AAAAAAAANLNFAAAA|906|Railroad Forest|Court|Suite P|Mountain View|Shackelford County|TX|74466|United States|-6|condo| +23998|AAAAAAAAOLNFAAAA|352|5th Park|Ct.|Suite 480|Concord|Mason County|WA|94107|United States|-8|apartment| +23999|AAAAAAAAPLNFAAAA|701|2nd |Dr.|Suite 470|Mount Pleasant|Parmer County|TX|71933|United States|-6|single family| +24000|AAAAAAAAAMNFAAAA|||Wy||Buena Vista|Hendricks County|||||| +24001|AAAAAAAABMNFAAAA|684|Spring |ST|Suite O|Cedar Grove|Martin County|KY|40411|United States|-5|condo| +24002|AAAAAAAACMNFAAAA|199|Lake |Way|Suite S|Blanchard|Marshall County||55985|||| +24003|AAAAAAAADMNFAAAA|792|7th Madison|Drive|Suite 110|Union Hill|Crow Wing County|MN|57746|United States|-6|apartment| +24004|AAAAAAAAEMNFAAAA|149|Park |Wy|Suite 290|Hamilton|Washington County|KS|62808|United States|-6|condo| +24005|AAAAAAAAFMNFAAAA|192|Spring |Parkway|Suite K|Union Hill|Dallas County|MO|67746|United States|-6|apartment| +24006|AAAAAAAAGMNFAAAA|305|View Sunset|RD|Suite F|Greenwood|Durham County|NC|28828|United States|-5|condo| +24007|AAAAAAAAHMNFAAAA|588|13th Maple|Lane|Suite P|Liberty|Lee County|SC|23451|United States|-5|apartment| +24008|AAAAAAAAIMNFAAAA|9|6th Davis|Circle|Suite X|Forest Hills|Trousdale County|TN|39237|United States|-6|single family| +24009|AAAAAAAAJMNFAAAA|602|Lake Maple|Parkway|Suite 190|Red Oak|Dodge County|WI|55018|United States|-6|condo| +24010|AAAAAAAAKMNFAAAA|756|First |ST|Suite 150|Yorktown|Harding County|SD|50732|United States|-7|condo| +24011|AAAAAAAALMNFAAAA|576|4th 2nd|Street|Suite 420|Highland Park|Iowa County|WI|56534|United States|-6|condo| +24012|AAAAAAAAMMNFAAAA|838|Lake |Lane|Suite F|Franklin|Wayne County|IA|59101|United States|-6|single family| +24013|AAAAAAAANMNFAAAA|643|Oak |Street|Suite 150|Springdale|Morgan County|OH|48883|United States|-5|single family| +24014|AAAAAAAAOMNFAAAA|99|Walnut |Parkway|Suite O|Riverside|Columbia County|FL|39231|United States|-5|apartment| +24015|AAAAAAAAPMNFAAAA|87|Ridge |ST|Suite 100|Woodville|Hampden County|MA|04889|United States|-5|apartment| +24016|AAAAAAAAANNFAAAA|791|Washington |Cir.|Suite H|Stratford|Faribault County|MN|56668|United States|-6|condo| +24017|AAAAAAAABNNFAAAA|636|Second |Ave|Suite X|Oakwood|Noble County|OH|40169|United States|-5|single family| +24018|AAAAAAAACNNFAAAA|689|North First|Cir.|Suite G|Valley View|Robertson County|TN|35124|United States|-6|single family| +24019|AAAAAAAADNNFAAAA|647|8th |Way|Suite Y|Five Forks|Carbon County|PA|12293|United States|-5|single family| +24020|AAAAAAAAENNFAAAA|71|North |Lane|Suite F|Lakeside|Massac County|IL|69532|United States|-6|single family| +24021|AAAAAAAAFNNFAAAA|865|Lake Main|Lane|Suite F|Midway|Traverse County|MN|51904|United States|-6|condo| +24022|AAAAAAAAGNNFAAAA|848|9th Lake|Blvd||Hubbard||||||| +24023|AAAAAAAAHNNFAAAA|582|Lakeview Main|Cir.|Suite W|Springfield|Onslow County|NC|29303|United States|-5|apartment| +24024|AAAAAAAAINNFAAAA|197|Adams Spruce|Avenue|Suite O|Ashland|Forest County|WI|54244|United States|-6|single family| +24025|AAAAAAAAJNNFAAAA|926|Mill Main|Road|Suite 210|Stringtown|Dubuque County|IA|50162|United States|-6|single family| +24026|AAAAAAAAKNNFAAAA|234|Chestnut Dogwood|Road|Suite 420|Woodlawn|Harper County|OK|74098|United States|-6|single family| +24027|AAAAAAAALNNFAAAA|683|Willow Jefferson|RD|Suite 10|Peru|Madison County|MO|60302|United States|-6|apartment| +24028|AAAAAAAAMNNFAAAA|984|Lincoln 6th|Cir.|Suite L|Garrison|Jasper County|IA|58767|United States|-6|condo| +24029|AAAAAAAANNNFAAAA|411|3rd |Parkway|Suite 190|Forest Hills|Okanogan County|WA|99237|United States|-8|apartment| +24030|AAAAAAAAONNFAAAA|633|Main |RD|Suite M|Lakeview|Huron County|OH|48579|United States|-5|apartment| +24031|AAAAAAAAPNNFAAAA|670|Maple |Ave|Suite 220|Marion|Hudspeth County|TX|70399|United States|-6|condo| +24032|AAAAAAAAAONFAAAA|642|3rd |Avenue|Suite O|Fairfield|Oneida County|WI|56192|United States|-6|apartment| +24033|AAAAAAAABONFAAAA|113|Poplar 10th|ST|Suite 340|Stringtown|Butler County|MO|60162|United States|-6|single family| +24034|AAAAAAAACONFAAAA|910|Jackson |Dr.|Suite 330|Midway|Oconee County|GA|31904|United States|-5|apartment| +24035|AAAAAAAADONFAAAA|391|Locust |Road|Suite D|Pleasant Hill|White Pine County|NV|83604|United States|-8|apartment| +24036|AAAAAAAAEONFAAAA|425|5th Green|Way|Suite W|Summerville|Daviess County|MO|62033|United States|-6|apartment| +24037|AAAAAAAAFONFAAAA|321|Ridge Third|Ln|Suite J|Brownsville|Cumberland County|PA|19310|United States|-5|apartment| +24038|AAAAAAAAGONFAAAA|862|5th |Dr.|Suite N|Newtown|Sumter County|FL|31749|United States|-5|apartment| +24039|AAAAAAAAHONFAAAA|513|Lakeview |Pkwy|Suite H|Oakwood|Lee County|AL|30169|United States|-6|apartment| +24040|AAAAAAAAIONFAAAA|105|4th Lake|Wy|Suite H|Florence|Dauphin County|PA|13394|United States|-5|single family| +24041|AAAAAAAAJONFAAAA|697|13th |Circle|Suite 320|White Oak|Hamilton County|FL|36668|United States|-5|single family| +24042|AAAAAAAAKONFAAAA|632|1st |Ln|Suite 440|Highland|Cooper County|MO|69454|United States|-6|apartment| +24043|AAAAAAAALONFAAAA|752|First Sycamore|Ln|Suite L|Walnut Grove|Yates County|NY|17752|United States|-5|condo| +24044|AAAAAAAAMONFAAAA|539|Church |Ave|Suite G|Five Points|Missaukee County|MI|46098|United States|-5|apartment| +24045|AAAAAAAANONFAAAA|972|Maple |Ave|Suite 70|Crossroads|Edgefield County|SC|20534|United States|-5|single family| +24046|AAAAAAAAOONFAAAA|505|Cherry |Wy|Suite 370|Lakeview|Powhatan County|VA|28579|United States|-5|single family| +24047|AAAAAAAAPONFAAAA|695|Park |Dr.|Suite 440|Concord|Morgan County|WV|24107|United States|-5|single family| +24048|AAAAAAAAAPNFAAAA|552|Ridge Center|Pkwy|Suite M|Mount Vernon|Knox County|ME|09082|United States|-5|apartment| +24049|AAAAAAAABPNFAAAA|890|7th Cedar|Court|Suite E|Woodville|Independence County|AR|74289|United States|-6|single family| +24050|AAAAAAAACPNFAAAA|932|Valley |Wy|Suite L|Friendship|Millard County|UT|84536|United States|-7|apartment| +24051|AAAAAAAADPNFAAAA|668|10th West|Street|Suite X|Marion|Cache County|UT|80399|United States|-7|single family| +24052|AAAAAAAAEPNFAAAA|403|Second Chestnut|Lane|Suite 490|Shiloh|Muscogee County|GA|39275|United States|-5|condo| +24053|AAAAAAAAFPNFAAAA|632|9th Ash|Cir.|Suite 40|Kingston|Grant County|WV|24975|United States|-5|single family| +24054|AAAAAAAAGPNFAAAA|589|Ash Main|Wy|Suite 440|Harmony|Buncombe County|NC|25804|United States|-5|apartment| +24055|AAAAAAAAHPNFAAAA|367|2nd |RD|Suite O|Mountain View|Hopkins County|KY|44466|United States|-6|condo| +24056|AAAAAAAAIPNFAAAA|424|Park |Street|Suite 110|Ruth|Chattooga County|GA|30309|United States|-5|single family| +24057|AAAAAAAAJPNFAAAA|268|Sixth |Wy|Suite I|White Oak|Gregg County|TX|76668|United States|-6|apartment| +24058|AAAAAAAAKPNFAAAA|300|Central First|RD|Suite I|Florence|Cass County|IN|43394|United States|-5|apartment| +24059|AAAAAAAALPNFAAAA||||Suite 250|Unionville|||61711|United States||| +24060|AAAAAAAAMPNFAAAA||Second Spring||Suite 490||Cass County||74975|||single family| +24061|AAAAAAAANPNFAAAA|802|Third 7th|Cir.|Suite N|Midway|Hennepin County|MN|51904|United States|-6|single family| +24062|AAAAAAAAOPNFAAAA|269|Adams River|Boulevard|Suite A|Brownsville|Chittenden County|VT|09910|United States|-5|condo| +24063|AAAAAAAAPPNFAAAA|82|8th Main|Lane|Suite 110|Armstrong|Knox County|OH|40647|United States|-5|single family| +24064|AAAAAAAAAAOFAAAA|46|Mill 2nd|Boulevard|Suite 180|Newtown|Hyde County|NC|21749|United States|-5|condo| +24065|AAAAAAAABAOFAAAA|523|10th |Avenue|Suite 90|New Hope|Palm Beach County|FL|39431|United States|-5|single family| +24066|AAAAAAAACAOFAAAA|146|Hill |Wy|Suite C|Plainview|Martin County|IN|43683|United States|-5|condo| +24067|AAAAAAAADAOFAAAA|40|Washington Lakeview|Lane|Suite 100|Five Points|Marshall County|AL|36098|United States|-6|single family| +24068|AAAAAAAAEAOFAAAA|560|Oak Madison|Cir.|Suite J|Cedar Grove|Conejos County|CO|80411|United States|-7|condo| +24069|AAAAAAAAFAOFAAAA|85|Maple |Wy|Suite H|Forest Hills|Kanawha County|WV|29237|United States|-5|condo| +24070|AAAAAAAAGAOFAAAA|664|2nd Hillcrest|Court|Suite I|Pleasant Valley|Pickens County|AL|32477|United States|-6|apartment| +24071|AAAAAAAAHAOFAAAA|931|Lincoln Central|Ln|Suite 200|Greenville|Webster County|KY|41387|United States|-5|apartment| +24072|AAAAAAAAIAOFAAAA|466|Jackson 4th|ST|Suite 380|Pleasant Valley|Jackson County|KS|62477|United States|-6|single family| +24073|AAAAAAAAJAOFAAAA|415|Elm |Drive|Suite 430|Lincoln|Prairie County|AR|71289|United States|-6|single family| +24074|AAAAAAAAKAOFAAAA|171|Sixth Forest|Circle|Suite F|Lawrenceville|Independence County|AR|74462|United States|-6|single family| +24075|AAAAAAAALAOFAAAA|279|Pine |Parkway|Suite M|Clinton|Chesterfield County|VA|28222|United States|-5|single family| +24076|AAAAAAAAMAOFAAAA|568|First |Lane|Suite 290|Pine Grove|Macoupin County|IL|64593|United States|-6|apartment| +24077|AAAAAAAANAOFAAAA|505|Walnut |Way|Suite 430|Centerville|New Kent County|VA|20059|United States|-5|single family| +24078|AAAAAAAAOAOFAAAA|990|Hill Second|Wy|Suite M|Pleasant Hill|Missoula County|MT|63604|United States|-7|condo| +24079|AAAAAAAAPAOFAAAA|961|Cedar |Circle|Suite Y|Providence|Jackson County|KS|66614|United States|-6|apartment| +24080|AAAAAAAAABOFAAAA||7th |Pkwy|Suite 250|||AR|76557||-6|apartment| +24081|AAAAAAAABBOFAAAA|901|Locust Washington|Ct.|Suite D|Oakdale|Gonzales County|TX|79584|United States|-6|single family| +24082|AAAAAAAACBOFAAAA|868|Birch Washington|Boulevard|Suite S|Clearwater|Oswego County|NY|19534|United States|-5|single family| +24083|AAAAAAAADBOFAAAA|431||Wy||||KS||||| +24084|AAAAAAAAEBOFAAAA|853|8th College|Cir.|Suite 10|Bethel|Scott County|MO|65281|United States|-6|single family| +24085|AAAAAAAAFBOFAAAA|732|Elm |RD|Suite 190|Greenfield|Grant County|IN|45038|United States|-5|condo| +24086|AAAAAAAAGBOFAAAA|979|3rd |Blvd|Suite C|Mount Pleasant|Union County|IA|51933|United States|-6|apartment| +24087|AAAAAAAAHBOFAAAA|501|Oak |Lane|Suite 170|Edgewood|Wolfe County|KY|40069|United States|-5|apartment| +24088|AAAAAAAAIBOFAAAA|573|Woodland Seventh|Boulevard|Suite F|Freeport|Stark County|OH|41844|United States|-5|apartment| +24089|AAAAAAAAJBOFAAAA|597|7th Adams|Pkwy|Suite B|Plainview|Camp County|TX|73683|United States|-6|apartment| +24090|AAAAAAAAKBOFAAAA|138|Walnut |RD|Suite R|Enterprise|Shelby County|MO|61757|United States|-6|condo| +24091|AAAAAAAALBOFAAAA|798|Church Fourth|Pkwy|Suite 160|Highland|Coweta County|GA|39454|United States|-5|single family| +24092|AAAAAAAAMBOFAAAA|578|Main 2nd|Boulevard|Suite 430|Hopewell|Hocking County|OH|40587|United States|-5|condo| +24093|AAAAAAAANBOFAAAA|851|Second |Ct.|Suite Q|Maple Grove|Rooks County|KS|68252|United States|-6|apartment| +24094|AAAAAAAAOBOFAAAA|920|Walnut |Pkwy|Suite 420|Buena Vista|Seminole County|FL|35752|United States|-5|condo| +24095|AAAAAAAAPBOFAAAA|5|Main 4th|Road|Suite 460|Sulphur Springs|Clarke County|AL|38354|United States|-6|condo| +24096|AAAAAAAAACOFAAAA|973|Lake Highland|Wy|Suite 300|Glendale|West Carroll Parish|LA|73951|United States|-6|condo| +24097|AAAAAAAABCOFAAAA|170|10th |Pkwy|Suite G|Woodland|Pike County|IN|44854|United States|-5|single family| +24098|AAAAAAAACCOFAAAA|640|Valley |Pkwy|Suite 380|Greenfield|Tulsa County|OK|75038|United States|-6|condo| +24099|AAAAAAAADCOFAAAA|694|3rd |Wy|Suite C|Mountain View|Gallatin County|KY|44466|United States|-6|condo| +24100|AAAAAAAAECOFAAAA||Ridge ||Suite 170||||||-5|| +24101|AAAAAAAAFCOFAAAA|497|Maple |Wy|Suite A|Arlington|Glacier County|MT|66557|United States|-7|apartment| +24102|AAAAAAAAGCOFAAAA|880|Twelfth |Pkwy|Suite L|Clifton|Chippewa County|WI|58014|United States|-6|condo| +24103|AAAAAAAAHCOFAAAA|40|10th |Boulevard|Suite H|Harmony|Holmes County|MS|55804|United States|-6|condo| +24104|AAAAAAAAICOFAAAA|930|View Third|Dr.|Suite 150|Florence|Watauga County|NC|23394|United States|-5|condo| +24105|AAAAAAAAJCOFAAAA|428|Sunset Wilson|Blvd|Suite 490|Lakeview|Cuyahoga County|OH|48579|United States|-5|apartment| +24106|AAAAAAAAKCOFAAAA|608|Central |Ln|Suite V|Lakeview|Saline County|NE|68579|United States|-7|apartment| +24107|AAAAAAAALCOFAAAA|335|Meadow |RD|Suite W|Hopewell|Dale County|AL|30587|United States|-6|condo| +24108|AAAAAAAAMCOFAAAA|836|College |Boulevard|Suite A|Wildwood|Calhoun County|WV|26871|United States|-5|condo| +24109|AAAAAAAANCOFAAAA|978|6th College|Drive|Suite Q|Kingston|Prince George County|VA|24975|United States|-5|single family| +24110|AAAAAAAAOCOFAAAA|117|3rd |Ct.|Suite 190|Red Hill|Clinch County|GA|34338|United States|-5|condo| +24111|AAAAAAAAPCOFAAAA|45|Church |Drive|Suite P|Union Hill|Drew County|AR|77746|United States|-6|single family| +24112|AAAAAAAAADOFAAAA|957|North Sixth|Ave|Suite P|Springfield|Cibola County|NM|89303|United States|-7|single family| +24113|AAAAAAAABDOFAAAA|907|View Valley|Blvd|Suite N|Glendale|Glades County|FL|33951|United States|-5|condo| +24114|AAAAAAAACDOFAAAA|471|4th 9th|Parkway|Suite S|Greenfield|Manassas Park city|VA|25038|United States|-5|single family| +24115|AAAAAAAADDOFAAAA|144|1st |Lane|Suite 140|Ashland|Venango County|PA|14244|United States|-5|apartment| +24116|AAAAAAAAEDOFAAAA|244|Franklin West|Drive|Suite N|Hillcrest|Whatcom County|WA|93003|United States|-8|single family| +24117|AAAAAAAAFDOFAAAA|600|East Elm|Road|Suite R|Mount Pleasant|Lee County|VA|21933|United States|-5|condo| +24118|AAAAAAAAGDOFAAAA|848|Hickory Sixth|Dr.|Suite 150|Rockland|Schuylkill County|PA|12044|United States|-5|condo| +24119|AAAAAAAAHDOFAAAA|468|Cherry |Ct.|Suite 50|Union Hill|Delaware County|NY|17746|United States|-5|condo| +24120|AAAAAAAAIDOFAAAA|646|Maple |Court|Suite J|Edwards|Alleghany County|VA|21409|United States|-5|condo| +24121|AAAAAAAAJDOFAAAA|743|Dogwood Meadow|Ct.|Suite 350|Harmony|Mecosta County|MI|45804|United States|-5|single family| +24122|AAAAAAAAKDOFAAAA|21|2nd |Drive|Suite 170|Mount Zion|Lincoln County|WI|58054|United States|-6|single family| +24123|AAAAAAAALDOFAAAA|170|Park |RD|Suite K|Marion|Baxter County|AR|70399|United States|-6|apartment| +24124|AAAAAAAAMDOFAAAA|782|Ridge Lee|ST|Suite 140|Spring Valley|Leavenworth County|KS|66060|United States|-6|condo| +24125|AAAAAAAANDOFAAAA|503|River ||Suite S|Newport|Polk County|IA|51521|United States||| +24126|AAAAAAAAODOFAAAA|431|Forest 10th|Wy|Suite D|Mount Pleasant|Fulton County|PA|11933|United States|-5|condo| +24127|AAAAAAAAPDOFAAAA|108|2nd 9th|RD|Suite A|Lakewood|Montague County|TX|78877|United States|-6|condo| +24128|AAAAAAAAAEOFAAAA|312|5th |Boulevard|Suite 350|Woodland|Spink County|SD|54854|United States|-7|apartment| +24129|AAAAAAAABEOFAAAA|830|Ridge |Ave|Suite 350|Centerville|Henry County|IL|60059|United States|-6|single family| +24130|AAAAAAAACEOFAAAA|76|1st Smith|Drive|Suite 90|Newport|Costilla County|CO|81521|United States|-7|apartment| +24131|AAAAAAAADEOFAAAA|305|Church |Road|Suite 390|Clifton|Haskell County|OK|78014|United States|-6|apartment| +24132|AAAAAAAAEEOFAAAA|869|2nd Williams|||Oak Hill||GA|37838|||apartment| +24133|AAAAAAAAFEOFAAAA|171|First |Way|Suite 310|Kingston|Shasta County|CA|94975|United States|-8|apartment| +24134|AAAAAAAAGEOFAAAA|479|Oak Central|Boulevard|Suite 60|Green Acres|Jasper County|MO|67683|United States|-6|condo| +24135|AAAAAAAAHEOFAAAA|131|Lincoln |Boulevard|Suite A|Harmony|Sussex County|NJ|06404|United States|-5|single family| +24136|AAAAAAAAIEOFAAAA|616|Washington Chestnut|Blvd|Suite H|Riverdale|Camden County|GA|39391|United States|-5|apartment| +24137|AAAAAAAAJEOFAAAA|977|1st |Lane|Suite O|Lincoln|Emmons County|ND|51289|United States|-6|condo| +24138|AAAAAAAAKEOFAAAA|145|Lakeview |Wy|Suite 170|Forest Hills|Twin Falls County|ID|89237|United States|-7|apartment| +24139|AAAAAAAALEOFAAAA|462|Laurel Willow|Ct.|Suite Y|Mount Olive|Talbot County|MD|28059|United States|-5|apartment| +24140|AAAAAAAAMEOFAAAA|||Boulevard|Suite 0|Maple Grove||||United States||| +24141|AAAAAAAANEOFAAAA|657|Birch Spring|Way|Suite 80|Ashland|Kankakee County|IL|64244|United States|-6|apartment| +24142|AAAAAAAAOEOFAAAA|27|Pine Hill|Ct.|Suite 170|Dallas|Cedar County|MO|63628|United States|-6|apartment| +24143|AAAAAAAAPEOFAAAA|159|Spring Davis|Lane|Suite 330|Valley View|Copiah County|MS|55124|United States|-6|single family| +24144|AAAAAAAAAFOFAAAA|795|Johnson Maple|Avenue|Suite G|Unionville|Decatur County|IA|51711|United States|-6|single family| +24145|AAAAAAAABFOFAAAA|908|4th |Pkwy|Suite 40|Union|Westmoreland County|PA|18721|United States|-5|condo| +24146|AAAAAAAACFOFAAAA|213|9th |Court|Suite K|Hazelwood|Chittenden County|VT|01806|United States|-5|single family| +24147|AAAAAAAADFOFAAAA|570|Wilson Lakeview|Dr.|Suite 380|Liberty|Ashley County|AR|73451|United States|-6|condo| +24148|AAAAAAAAEFOFAAAA|762|Franklin Hill|Way|Suite 410|Watkins|Williamsburg city|VA|21732|United States|-5|single family| +24149|AAAAAAAAFFOFAAAA|235|Forest Spruce|Road|Suite 320|Texas|Morgan County|AL|33342|United States|-6|single family| +24150|AAAAAAAAGFOFAAAA|274|3rd Walnut|Ave|Suite L|Arlington|Lee County|GA|36557|United States|-5|condo| +24151|AAAAAAAAHFOFAAAA|399|3rd |Way|Suite N|Springfield|Knox County|TX|79303|United States|-6|condo| +24152|AAAAAAAAIFOFAAAA|690|College Maple|ST|Suite 170|Bethel|Wyoming County|NY|15281|United States|-5|apartment| +24153|AAAAAAAAJFOFAAAA|19|Elm Main|Lane|Suite Q|Bethel|Chickasaw County|MS|55281|United States|-6|condo| +24154|AAAAAAAAKFOFAAAA|708|Meadow Sycamore|Blvd|Suite 160|Bethel|Tyler County|TX|75281|United States|-6|apartment| +24155|AAAAAAAALFOFAAAA|384|Hillcrest |Ave|Suite D|Antioch|Mercer County|NJ|09205|United States|-5|condo| +24156|AAAAAAAAMFOFAAAA|945|9th |Pkwy|Suite 230|Clifton|Johnston County|NC|28014|United States|-5|single family| +24157|AAAAAAAANFOFAAAA|334|Sunset Park|Way|Suite 0|Fairfield|Forest County|PA|16192|United States|-5|condo| +24158|AAAAAAAAOFOFAAAA|859|Park 7th|Lane|Suite Y|Pleasant Hill|Park County|MT|63604|United States|-7|condo| +24159|AAAAAAAAPFOFAAAA|692|Railroad 2nd|Boulevard|Suite H|Lakeview|Clark County|WA|98579|United States|-8|single family| +24160|AAAAAAAAAGOFAAAA||Wilson College|Circle|||||61933||-6|condo| +24161|AAAAAAAABGOFAAAA|588|9th Washington|Parkway|Suite R|Newport|Ray County|MO|61521|United States|-6|apartment| +24162|AAAAAAAACGOFAAAA|351|Madison 1st|Ln|Suite 270|Newport|Sullivan County|PA|11521|United States|-5|condo| +24163|AAAAAAAADGOFAAAA|706|11th Main|RD|Suite 440|Shady Grove|McLeod County|MN|52812|United States|-6|apartment| +24164|AAAAAAAAEGOFAAAA|992|Park Center|Way|Suite I|Red Hill|Garfield County|MT|64338|United States|-7|condo| +24165|AAAAAAAAFGOFAAAA|620|Fourteenth 7th|Dr.|Suite 40|Jackson|Cleburne County|AR|79583|United States|-6|apartment| +24166|AAAAAAAAGGOFAAAA|292|North |Way|Suite K|Florence|Renville County|MN|53394|United States|-6|condo| +24167|AAAAAAAAHGOFAAAA||Meadow |Lane||Shiloh|Franklin County|MO|69275|United States|-6|apartment| +24168|AAAAAAAAIGOFAAAA|430|Elevnth |Blvd|Suite O|Tracy|Keokuk County|IA|56340|United States|-6|single family| +24169|AAAAAAAAJGOFAAAA|367|3rd 10th|Lane|Suite 140|White Oak|Paulding County|GA|36668|United States|-5|apartment| +24170|AAAAAAAAKGOFAAAA|623|Third |Dr.|Suite F|Fairfield|Wayne County|TN|36192|United States|-6|single family| +24171|AAAAAAAALGOFAAAA|791|Ash Oak|Ct.|Suite U|Enterprise|Ventura County|CA|91757|United States|-8|single family| +24172|AAAAAAAAMGOFAAAA|294|Poplar Washington|Way|Suite N|White Oak|Faribault County|MN|56668|United States|-6|condo| +24173|AAAAAAAANGOFAAAA|55|Cedar |Blvd|Suite S|Greenville|Newport County|RI|01987|United States|-5|condo| +24174|AAAAAAAAOGOFAAAA|460|Washington Mill|Road|Suite 30|Wilson|Warren County|IA|56971|United States|-6|condo| +24175|AAAAAAAAPGOFAAAA|111|Sixth |Avenue|Suite 230|Farmington|Montmorency County|MI|49145|United States|-5|single family| +24176|AAAAAAAAAHOFAAAA|99|Sycamore |Blvd|Suite T|Riverdale|Geneva County|AL|39391|United States|-6|apartment| +24177|AAAAAAAABHOFAAAA|267|Maple |Way|Suite 130|Deerfield|Buchanan County|IA|59840|United States|-6|condo| +24178|AAAAAAAACHOFAAAA|867|Valley North|Cir.|Suite K|Lakeview|Cortland County|NY|18579|United States|-5|condo| +24179|AAAAAAAADHOFAAAA|642|Cedar North|Circle|Suite 300|Mount Zion|Owsley County|KY|48054|United States|-5|single family| +24180|AAAAAAAAEHOFAAAA|284|Hickory |Blvd|Suite K|Farmington|Lewis County|WA|99145|United States|-8|apartment| +24181|AAAAAAAAFHOFAAAA|858|Second |Avenue|Suite I|Oakland|Overton County|TN|39843|United States|-6|condo| +24182|AAAAAAAAGHOFAAAA|482|Highland View|Boulevard|Suite 30|Point Pleasant|Graves County|KY|44749|United States|-6|condo| +24183|AAAAAAAAHHOFAAAA|247|Fifteenth Railroad|Ln|Suite 290|Maple Grove|Wheeler County|TX|78252|United States|-6|apartment| +24184|AAAAAAAAIHOFAAAA|980|Central Hillcrest|Way|Suite I|Pleasant Grove|Prairie County|MT|64136|United States|-7|single family| +24185|AAAAAAAAJHOFAAAA||River |Ave|Suite 340|Enterprise|Dunn County|ND|51757|United States||single family| +24186|AAAAAAAAKHOFAAAA|643|Center Railroad|Way|Suite 220|Harmon|Wright County|MN|55623|United States|-6|apartment| +24187|AAAAAAAALHOFAAAA|301|6th Pine|Dr.|Suite H|Rocky Point|Bryan County|GA|31209|United States|-5|single family| +24188|AAAAAAAAMHOFAAAA|945|Maple |Pkwy|Suite F|Bridgeport|Henry County|TN|35817|United States|-5|single family| +24189|AAAAAAAANHOFAAAA|374|Railroad |Avenue|Suite A|Greenville|San Bernardino County|CA|91387|United States|-8|condo| +24190|AAAAAAAAOHOFAAAA|940|Church |Pkwy|Suite J|Buena Vista|Brown County|OH|45752|United States|-5|condo| +24191|AAAAAAAAPHOFAAAA|4|2nd Madison|ST|Suite N|Wilson|Shelby County|IN|46971|United States|-5|apartment| +24192|AAAAAAAAAIOFAAAA|351|Cherry |Wy|Suite A|Greenfield|Monroe County|FL|35038|United States|-5|apartment| +24193|AAAAAAAABIOFAAAA|299|Main Main|Court|Suite B|Greenwood|Tooele County|UT|88828|United States|-7|single family| +24194|AAAAAAAACIOFAAAA|957|Willow |Road|Suite O|Harmony|Wyandot County|OH|45804|United States|-5|apartment| +24195|AAAAAAAADIOFAAAA|716|Park Main|Way|Suite U|Concord|Mitchell County|TX|74107|United States|-6|condo| +24196|AAAAAAAAEIOFAAAA|929|Pine Main|Circle|Suite D|Greenwood|Jackson County|MO|68828|United States|-6|condo| +24197|AAAAAAAAFIOFAAAA|63|Oak |Avenue|Suite X|Bunker Hill|Bremer County|IA|50150|United States|-6|single family| +24198|AAAAAAAAGIOFAAAA|200|Jefferson |Blvd|Suite 410|Belmont|Magoffin County|KY|40191|United States|-5|condo| +24199|AAAAAAAAHIOFAAAA|51|Dogwood |Court|Suite M|Brownsville|Kenai Peninsula Borough|AK|99310|United States|-9|apartment| +24200|AAAAAAAAIIOFAAAA|788|Central |Drive|Suite 280|Union Hill|Tillamook County|OR|97746|United States|-8|single family| +24201|AAAAAAAAJIOFAAAA|139|Wilson Second|Court|Suite 280|Mountain View|McPherson County|SD|54466|United States|-7|condo| +24202|AAAAAAAAKIOFAAAA|621|Park |Ln|Suite 310|Stringtown|Collin County|TX|70162|United States|-6|single family| +24203|AAAAAAAALIOFAAAA|168|Lakeview Lake|Cir.|Suite Y|Wakefield|Bonner County|ID|85798|United States|-7|condo| +24204|AAAAAAAAMIOFAAAA|394|Jackson 1st|RD|Suite S|New Hope|Mariposa County|CA|99431|United States|-8|single family| +24205|AAAAAAAANIOFAAAA|850|First 1st|Wy|Suite 160|Riverside|Crawford County|AR|79231|United States|-6|single family| +24206|AAAAAAAAOIOFAAAA|967|5th |Court|Suite F|Greenwood|Jasper County|IA|58828|United States|-6|apartment| +24207|AAAAAAAAPIOFAAAA|754|3rd |ST|Suite J|Crossroads|Grant County|AR|70534|United States|-6|condo| +24208|AAAAAAAAAJOFAAAA|682|Sunset Hillcrest|Ct.|Suite 10|Lakeview|Pinal County|AZ|88579|United States|-7|single family| +24209|AAAAAAAABJOFAAAA|74|8th Spruce|Ct.|Suite O|Salem|Lincoln County|WA|98048|United States|-8|apartment| +24210|AAAAAAAACJOFAAAA|78|Locust 1st|Street|Suite 200|Green Acres|Ashe County|NC|27683|United States|-5|single family| +24211|AAAAAAAADJOFAAAA|912|8th |Wy|Suite 370|Fairview|Fairfield County|OH|45709|United States|-5|single family| +24212|AAAAAAAAEJOFAAAA|917|Smith |Pkwy|Suite 130|Liberty|Bucks County|PA|13451|United States|-5|apartment| +24213|AAAAAAAAFJOFAAAA|144|Elm |Drive|Suite E|Glenwood|Duplin County|NC|23511|United States|-5|single family| +24214|AAAAAAAAGJOFAAAA|962|6th West|Ave|Suite Q|Edgewood|Franklin County|IA|50069|United States|-6|condo| +24215|AAAAAAAAHJOFAAAA|278|Miller |Circle|Suite 290|Friendship|Wake County|NC|24536|United States|-5|condo| +24216|AAAAAAAAIJOFAAAA|933|Seventh |Road|Suite 290|Plainview|Litchfield County|CT|04283|United States|-5|apartment| +24217|AAAAAAAAJJOFAAAA|936|Broadway Elm|RD|Suite U|Centerville|Somerset County|PA|10059|United States|-5|apartment| +24218|AAAAAAAAKJOFAAAA|940|3rd 6th|Cir.|Suite E|Maple Grove|Porter County|IN|48252|United States|-5|apartment| +24219|AAAAAAAALJOFAAAA|577|Ridge Fifth|Road|Suite M|Clifton|Starke County|IN|48014|United States|-5|apartment| +24220|AAAAAAAAMJOFAAAA|427|Meadow Park|Blvd|Suite 120|Brownsville|Vermillion County|IN|49310|United States|-5|condo| +24221|AAAAAAAANJOFAAAA|986|Smith |Circle|Suite R|Union|Madison County|NC|28721|United States|-5|single family| +24222|AAAAAAAAOJOFAAAA|255|Fourth Cedar|Road|Suite 480|Oak Grove|Vance County|NC|28370|United States|-5|condo| +24223|AAAAAAAAPJOFAAAA|145|Smith Hillcrest|RD|Suite I|Greenfield|Boulder County|CO|85038|United States|-7|single family| +24224|AAAAAAAAAKOFAAAA|589|Church |Street|Suite M|Pleasant Grove|Henderson County|KY|44136|United States|-6|apartment| +24225|AAAAAAAABKOFAAAA|638|2nd |Cir.|Suite 350|Greenville|Sully County|SD|51387|United States|-7|condo| +24226|AAAAAAAACKOFAAAA|827|Church |Cir.|Suite A|Greenville|Pitt County|NC|21387|United States|-5|condo| +24227|AAAAAAAADKOFAAAA|736|Cherry Franklin|Drive|Suite 80|Ashland|Mille Lacs County|MN|54244|United States|-6|condo| +24228|AAAAAAAAEKOFAAAA|978|8th |Street|Suite A|Pleasant Grove|Chariton County|MO|64136|United States|-6|condo| +24229|AAAAAAAAFKOFAAAA|801|Ash |Circle|Suite E|Clinton|Wise County|TX|78222|United States|-6|apartment| +24230|AAAAAAAAGKOFAAAA|594|Dogwood Hillcrest|Way|Suite L|Arlington|Grant County|ND|56557|United States|-6|condo| +24231|AAAAAAAAHKOFAAAA|513|Maple 9th|Dr.|Suite D|Green Acres|Lenoir County|NC|27683|United States|-5|single family| +24232|AAAAAAAAIKOFAAAA|869|Lincoln |Ln|Suite N|Riverdale|Hopkins County|KY|49391|United States|-6|apartment| +24233|AAAAAAAAJKOFAAAA|690|Washington Walnut|Parkway|Suite E|Lebanon|Clark County|KS|62898|United States|-6|apartment| +24234|AAAAAAAAKKOFAAAA|809|Madison |Parkway|Suite B|Riverside|Menard County|IL|69231|United States|-6|apartment| +24235|AAAAAAAALKOFAAAA|153|15th |Way|Suite F|Maple Grove|McClain County|OK|78252|United States|-6|apartment| +24236|AAAAAAAAMKOFAAAA|645|Jackson Lake|Dr.|Suite 270|Highland|Sully County|SD|59454|United States|-7|condo| +24237|AAAAAAAANKOFAAAA|102|Church Third|Court|Suite N|Kingston|Houston County|GA|34975|United States|-5|single family| +24238|AAAAAAAAOKOFAAAA|524|8th Birch|Circle|Suite H|Macedonia|Garfield County|OK|71087|United States|-6|condo| +24239|AAAAAAAAPKOFAAAA|500|Meadow |Cir.|Suite 130|Centerville|Millard County|UT|80059|United States|-7|single family| +24240|AAAAAAAAALOFAAAA|623|River 8th|Dr.|Suite 190|Shady Grove|Swain County|NC|22812|United States|-5|condo| +24241|AAAAAAAABLOFAAAA|240|Spruce |Wy|Suite A|Friendship|Clearwater County|ID|84536|United States|-7|apartment| +24242|AAAAAAAACLOFAAAA|139|College Adams|Court|Suite Y|Woodland|Glacier County|MT|64854|United States|-7|condo| +24243|AAAAAAAADLOFAAAA|800|Hickory |Ln|Suite D|Greenwood|Bland County|VA|28828|United States|-5|condo| +24244|AAAAAAAAELOFAAAA|422|Cedar Hickory|Drive|Suite 80|Lenox|Seward County|NE|61143|United States|-7|single family| +24245|AAAAAAAAFLOFAAAA|605|Cedar |Street|Suite D|Spring Valley|Winnebago County|IL|66060|United States|-6|apartment| +24246|AAAAAAAAGLOFAAAA|235|Main |ST|Suite W|Greenfield|Union County|NC|25038|United States|-5|apartment| +24247|AAAAAAAAHLOFAAAA|46|Ash |Boulevard|Suite P|Kingston|Van Buren County|TN|34975|United States|-6|single family| +24248|AAAAAAAAILOFAAAA|555|11th |RD|Suite 70|Centerville|Pueblo County|CO|80059|United States|-7|apartment| +24249|AAAAAAAAJLOFAAAA|407|Davis Mill|Ln|Suite 250|Mount Olive|Saguache County|CO|88059|United States|-7|apartment| +24250|AAAAAAAAKLOFAAAA|944|Fifteenth |Boulevard|Suite 350|Marion|Iowa County|IA|50399|United States|-6|apartment| +24251|AAAAAAAALLOFAAAA|562|Miller Elm|Wy|Suite 40|Farmington|Brunswick County|NC|29145|United States|-5|single family| +24252|AAAAAAAAMLOFAAAA|38|4th 2nd|Road|Suite R|Walnut Grove|Miami County|KS|67752|United States|-6|single family| +24253|AAAAAAAANLOFAAAA|607|14th Davis|Way|Suite K|Lakewood|Kosciusko County|IN|48877|United States|-5|apartment| +24254|AAAAAAAAOLOFAAAA|542|Lee |Ct.|Suite R|Acme|Walthall County|MS|50164|United States|-6|single family| +24255|AAAAAAAAPLOFAAAA|953|View |Ct.|Suite G|Riverdale|Madera County|CA|99391|United States|-8|single family| +24256|AAAAAAAAAMOFAAAA|877|12th Johnson|Road|Suite 60|Midway|Schuyler County|NY|11904|United States|-5|condo| +24257|AAAAAAAABMOFAAAA|91|Poplar Davis|Cir.|Suite 60|Woodland|Beaver County|OK|74854|United States|-6|condo| +24258|AAAAAAAACMOFAAAA|973|Williams |Street|Suite H|Newtown|Kandiyohi County|MN|51749|United States|-6|apartment| +24259|AAAAAAAADMOFAAAA|552|Central |Blvd|Suite A|Oakdale|Summit County|CO|89584|United States|-7|single family| +24260|AAAAAAAAEMOFAAAA|271|Miller |Pkwy|Suite N|Cumberland|Custer County|CO|88971|United States|-7|single family| +24261|AAAAAAAAFMOFAAAA|858|West Twelfth|Cir.|Suite P|Spring Hill|Davison County|SD|56787|United States|-6|apartment| +24262|AAAAAAAAGMOFAAAA|977|Hickory |Parkway|Suite 80|Pleasant Hill|Christian County|MO|63604|United States|-6|apartment| +24263|AAAAAAAAHMOFAAAA|165|Jackson 11th|Lane|Suite 20|Providence|Rosebud County|MT|66614|United States|-7|condo| +24264|AAAAAAAAIMOFAAAA|498|Dogwood |Drive|Suite O|Farmington|Bienville Parish|LA|79145|United States|-6|single family| +24265|AAAAAAAAJMOFAAAA|267|13th Broadway|Ln|Suite 170|Plainview|Woodford County|IL|63683|United States|-6|single family| +24266|AAAAAAAAKMOFAAAA|355|Fifth 4th|Road|Suite 60|Fairview|Sabine Parish|LA|75709|United States|-6|condo| +24267|AAAAAAAALMOFAAAA|657|Birch Spruce|Dr.|Suite N|Green Acres|Chelan County|WA|97683|United States|-8|single family| +24268|AAAAAAAAMMOFAAAA|969|1st |Ave|Suite 190|Florence|Mitchell County|GA|33394|United States|-5|single family| +24269|AAAAAAAANMOFAAAA|625|3rd |Avenue|Suite U|Greenwood|Fergus County|MT|68828|United States|-7|condo| +24270|AAAAAAAAOMOFAAAA|349|Valley View|Dr.|Suite B|Oakwood|Clarke County|MS|50169|United States|-6|condo| +24271|AAAAAAAAPMOFAAAA|719|Central Valley|Road|Suite O|Lakewood|Washington County|OK|78877|United States|-6|condo| +24272|AAAAAAAAANOFAAAA|158|Cherry |Pkwy|Suite D|Farmington|Casey County|KY|49145|United States|-6|condo| +24273|AAAAAAAABNOFAAAA|235|Railroad Lakeview|Circle|Suite 300|Forest Hills|Tyrrell County|NC|29237|United States|-5|single family| +24274|AAAAAAAACNOFAAAA|566|Poplar Cedar|RD|Suite V|Friendship|Nash County|NC|24536|United States|-5|single family| +24275|AAAAAAAADNOFAAAA|951|Center |Road|Suite T|Jamestown|Boulder County|CO|86867|United States|-7|apartment| +24276|AAAAAAAAENOFAAAA|130|Lee 5th|Way|Suite 90|Fairview|Oswego County|NY|15709|United States|-5|single family| +24277|AAAAAAAAFNOFAAAA|554|Spruce 4th|Ln|Suite 280|Friendship|Rockingham County|NH|05136|United States|-5|condo| +24278|AAAAAAAAGNOFAAAA|77|Chestnut Hill|ST|Suite 210|Mount Vernon|King William County|VA|28482|United States|-5|single family| +24279|AAAAAAAAHNOFAAAA|564|Hickory Pine|Circle|Suite O|Pleasant Grove|Price County|WI|54136|United States|-6|single family| +24280|AAAAAAAAINOFAAAA|567|5th |Drive|Suite W|Hopewell|Yadkin County|NC|20587|United States|-5|condo| +24281|AAAAAAAAJNOFAAAA|633|Church First|Lane|Suite 70|Pleasant Hill|Falls County|TX|73604|United States|-6|single family| +24282|AAAAAAAAKNOFAAAA|493|Spruce |Drive|Suite 240|Freeport|Dallam County|TX|71844|United States|-6|apartment| +24283|AAAAAAAALNOFAAAA|305|Mill 1st|Drive|Suite 210|Oakwood|Nantucket County|MA|00769|United States|-5|condo| +24284|AAAAAAAAMNOFAAAA|157|Poplar |Avenue|Suite 20|Oakland|Menifee County|KY|49843|United States|-5|single family| +24285|AAAAAAAANNOFAAAA|297|4th Meadow|Ln|Suite E|Hopewell|Natchitoches Parish|LA|70587|United States|-6|apartment| +24286|AAAAAAAAONOFAAAA|515|Ash Hill|Way|Suite 210|Riverview|Meeker County|MN|59003|United States|-6|condo| +24287|AAAAAAAAPNOFAAAA|970|4th |Ave|Suite 330|Union Hill|Lee County|NC|27746|United States|-5|apartment| +24288|AAAAAAAAAOOFAAAA|726||Street||Riverside|Caldwell Parish|||United States||condo| +24289|AAAAAAAABOOFAAAA|746|Willow Mill|Way|Suite W|Antioch|Klamath County|OR|98605|United States|-8|condo| +24290|AAAAAAAACOOFAAAA|575|3rd |Court|Suite G|Highland Park|Hancock County|OH|46534|United States|-5|single family| +24291|AAAAAAAADOOFAAAA|317|Park Green|Court|Suite R|Deerfield|Sweetwater County|WY|89840|United States|-7|apartment| +24292|AAAAAAAAEOOFAAAA|821|Lee |Circle|Suite 130|Five Forks|Hinds County|MS|52293|United States|-6|single family| +24293|AAAAAAAAFOOFAAAA|719|Cedar 4th|Court|Suite N|White Oak|Chase County|KS|66668|United States|-6|single family| +24294|AAAAAAAAGOOFAAAA|727|5th Green|Avenue|Suite J|Glendale|Evans County|GA|33951|United States|-5|condo| +24295|AAAAAAAAHOOFAAAA|159|Franklin |Ct.|Suite 240|Fairview|Apache County|AZ|85709|United States|-7|apartment| +24296|AAAAAAAAIOOFAAAA|355|Hill |RD|Suite U|Mountain View|Lorain County|OH|44466|United States|-5|apartment| +24297|AAAAAAAAJOOFAAAA|621|2nd |Court|Suite 200|Jackson|Wetzel County|WV|29583|United States|-5|condo| +24298|AAAAAAAAKOOFAAAA|228|Lincoln 3rd|Drive|Suite E|Bear Creek|Jo Daviess County|IL|63075|United States|-6|single family| +24299|AAAAAAAALOOFAAAA|824|Pine |Court|Suite O|Marion|Butler County|NE|60399|United States|-6|single family| +24300|AAAAAAAAMOOFAAAA|502|West Park|Street|Suite 80|Shore Acres|Fairfield County|SC|22724|United States|-5|apartment| +24301|AAAAAAAANOOFAAAA|55|Franklin Birch|Street|Suite 310|Stringtown|Morrow County|OH|40162|United States|-5|condo| +24302|AAAAAAAAOOOFAAAA|414|Cherry |Avenue|Suite A|Edgewater|Oconto County|WI|50635|United States|-6|single family| +24303|AAAAAAAAPOOFAAAA|102|Valley |Pkwy|Suite 180|Lebanon|Adams County|ID|82898|United States|-7|single family| +24304|AAAAAAAAAPOFAAAA|923|11th Birch|Avenue|Suite 480|Tracy|Franklin County|OH|46340|United States|-5|apartment| +24305|AAAAAAAABPOFAAAA|813|Main Main|Ct.|Suite I|Marion|Colleton County|SC|20399|United States|-5|apartment| +24306|AAAAAAAACPOFAAAA|97|2nd Dogwood|Way|Suite 100|Belmont|Fremont County|IA|50191|United States|-6|condo| +24307|AAAAAAAADPOFAAAA|488|7th Second|Dr.|Suite U|Oak Hill|Butte County|ID|87838|United States|-7|condo| +24308|AAAAAAAAEPOFAAAA|130|Third |Court|Suite O|Maple Grove|Murray County|MN|58252|United States|-6|condo| +24309|AAAAAAAAFPOFAAAA|325|1st |Boulevard|Suite 180|Mount Vernon|Anson County|NC|28482|United States|-5|condo| +24310|AAAAAAAAGPOFAAAA|941|3rd |Lane|Suite C|Jackson|Pepin County|WI|59583|United States|-6|condo| +24311|AAAAAAAAHPOFAAAA|87|Willow |Street|Suite N|Yorktown|Stevens County|MN|50732|United States|-6|single family| +24312|AAAAAAAAIPOFAAAA|934|Williams Valley|Court|Suite M|Concord|Mahnomen County|MN|54107|United States|-6|apartment| +24313|AAAAAAAAJPOFAAAA|902|11th |Ln|Suite T|Ashland|Washington County|IA|54244|United States|-6|apartment| +24314|AAAAAAAAKPOFAAAA|698|Seventh |Drive|Suite T|Harmony|Liberty County|GA|35804|United States|-5|apartment| +24315|AAAAAAAALPOFAAAA|757|Wilson Green|Wy|Suite 290|Pleasant Valley|Atkinson County|GA|32477|United States|-5|single family| +24316|AAAAAAAAMPOFAAAA|332|Cedar |Ln|Suite S|Oakland|Sedgwick County|CO|89843|United States|-7|single family| +24317|AAAAAAAANPOFAAAA||Walnut |Ct.|Suite 320||||||-7|| +24318|AAAAAAAAOPOFAAAA|406|4th View|Dr.|Suite 0|Georgetown|Ohio County|KY|47057|United States|-5|apartment| +24319|AAAAAAAAPPOFAAAA|159|1st 5th|Street|Suite 360|Oak Grove|Cochran County|TX|78370|United States|-6|apartment| +24320|AAAAAAAAAAPFAAAA|463|Washington Broadway|Blvd|Suite I|Plainview|Valley County|NE|63683|United States|-7|single family| +24321|AAAAAAAABAPFAAAA|297|Pine 14th|Boulevard|Suite O|Spring Valley|Baltimore city|MD|26060|United States|-5|single family| +24322|AAAAAAAACAPFAAAA|513|Highland Ash|Boulevard|Suite 370|Georgetown|Baker County|GA|37057|United States|-5|apartment| +24323|AAAAAAAADAPFAAAA|767|2nd |Avenue|Suite N|Oak Grove|Wright County|MO|68370|United States|-6|single family| +24324|AAAAAAAAEAPFAAAA|510|Ridge |Ct.|Suite 470|Five Points|Fulton County|GA|36098|United States|-5|single family| +24325|AAAAAAAAFAPFAAAA|761|Hickory |Parkway|Suite 320|Maple Grove|Carroll County|MO|68252|United States|-6|condo| +24326|AAAAAAAAGAPFAAAA|417|3rd Sycamore|Ave|Suite W|Wildwood|Camp County|TX|76871|United States|-6|single family| +24327|AAAAAAAAHAPFAAAA|291|Washington Jackson|Ct.|Suite U|Lincoln|Searcy County|AR|71289|United States|-6|condo| +24328|AAAAAAAAIAPFAAAA|937|Mill |Avenue|Suite 440|Shiloh|Champaign County|OH|49275|United States|-5|single family| +24329|AAAAAAAAJAPFAAAA|127|Church |Road|Suite K|Marion|Warren County|IL|60399|United States|-6|condo| +24330|AAAAAAAAKAPFAAAA|750|1st Spruce|Street|Suite 300|Summit|King George County|VA|20499|United States|-5|apartment| +24331|AAAAAAAALAPFAAAA|495|Center Birch|Street|Suite 140|Cedar Grove|Elko County|NV|80411|United States|-8|condo| +24332|AAAAAAAAMAPFAAAA|66|Church Meadow|Dr.|Suite O|Highland|Richland County|SC|29454|United States|-5|single family| +24333|AAAAAAAANAPFAAAA|609|Mill |Wy|Suite J|Lincoln|Crisp County|GA|31289|United States|-5|apartment| +24334|AAAAAAAAOAPFAAAA|575|11th Second|Court|Suite 260|Jackson|Scotland County|MO|69583|United States|-6|condo| +24335|AAAAAAAAPAPFAAAA|580|Sycamore Lake|RD|Suite 170|Omega|Nueces County|TX|76364|United States|-6|condo| +24336|AAAAAAAAABPFAAAA|650|Pine |Blvd|Suite 400|Greenville|Wright County|IA|51387|United States|-6|condo| +24337|AAAAAAAABBPFAAAA|30|Ninth Valley|ST|Suite P|Ashland|Lincoln County|MN|54244|United States|-6|apartment| +24338|AAAAAAAACBPFAAAA|199|Second 8th|Circle|Suite 350|Stringtown|Placer County|CA|90162|United States|-8|condo| +24339|AAAAAAAADBPFAAAA|708|Church Lee|Boulevard|Suite 400|Mountain View|Grant County|WV|24466|United States|-5|single family| +24340|AAAAAAAAEBPFAAAA|206|Maple |RD|Suite 350|Carthage|Gillespie County|TX|71529|United States|-6|condo| +24341|AAAAAAAAFBPFAAAA|410|View |Street|Suite 30|Springfield|Louisa County|IA|59303|United States|-6|single family| +24342|AAAAAAAAGBPFAAAA|461|Elm Railroad|Way|Suite M|Liberty|Crittenden County|AR|73451|United States|-6|condo| +24343|AAAAAAAAHBPFAAAA|875|Maple Smith|Ct.|Suite 370|Walnut Grove|Crawford County|IN|47752|United States|-5|apartment| +24344|AAAAAAAAIBPFAAAA|736|1st South|Circle|Suite D|Riverview|Garden County|NE|69003|United States|-6|apartment| +24345|AAAAAAAAJBPFAAAA|393|Sixth Mill|Ct.|Suite R|Greenwood|Dallas County|AR|78828|United States|-6|condo| +24346|AAAAAAAAKBPFAAAA|676|3rd |ST|Suite 450|White Oak|Sussex County|DE|16668|United States|-5|condo| +24347|AAAAAAAALBPFAAAA|428|Park 1st|Circle|Suite 250|Green Acres|Hopkins County|KY|47683|United States|-6|single family| +24348|AAAAAAAAMBPFAAAA|429|Green Church|Circle|Suite 200|Farmington|Addison County|VT|09745|United States|-5|single family| +24349|AAAAAAAANBPFAAAA|987|Hillcrest |Parkway|Suite 330|Fairbanks|Nelson County|KY|46653|United States|-5|single family| +24350|AAAAAAAAOBPFAAAA|212|Meadow Highland|Parkway|Suite T|Riverside|Panola County|MS|59231|United States|-6|single family| +24351|AAAAAAAAPBPFAAAA|557|2nd |Road|Suite T|Highland Park|Clarke County|GA|36534|United States|-5|single family| +24352|AAAAAAAAACPFAAAA|612||||Unionville|||21711|United States|-5|apartment| +24353|AAAAAAAABCPFAAAA|820|River Seventh|Ct.|Suite H|Hamilton|Platte County|MO|62808|United States|-6|single family| +24354|AAAAAAAACCPFAAAA|990|Meadow 12th|Avenue|Suite 360|Sulphur Springs|Vinton County|OH|48354|United States|-5|single family| +24355|AAAAAAAADCPFAAAA|98|6th Davis|Way|Suite 230|Allison|Danville city|VA|24167|United States|-5|condo| +24356|AAAAAAAAECPFAAAA|97|Walnut |Wy|Suite 320|Bunker Hill|Smyth County|VA|20150|United States|-5|apartment| +24357|AAAAAAAAFCPFAAAA|841|12th |Dr.|Suite 370|Liberty|Washington County|MN|53451|United States|-6|apartment| +24358|AAAAAAAAGCPFAAAA|32|College |Drive|Suite L|Mountain View|Washburn County|WI|54466|United States|-6|single family| +24359|AAAAAAAAHCPFAAAA|270|Eigth |Street|Suite 370|Antioch|Morgan County|IL|68605|United States|-6|apartment| +24360|AAAAAAAAICPFAAAA|360|8th |Parkway|Suite 460|Springfield|Sargent County|ND|59303|United States|-6|apartment| +24361|AAAAAAAAJCPFAAAA|77|Highland Maple|Drive|Suite A|Belmont|Brown County|NE|60191|United States|-6|single family| +24362|AAAAAAAAKCPFAAAA|641|View Cherry|Avenue|Suite T|Riverdale|Keokuk County|IA|59391|United States|-6|single family| +24363|AAAAAAAALCPFAAAA|951|Fourth |Road|Suite M|Macedonia|Madison Parish|LA|71087|United States|-6|condo| +24364|AAAAAAAAMCPFAAAA|241|Hill |Wy|Suite F|Highland|Hocking County|OH|49454|United States|-5|condo| +24365|AAAAAAAANCPFAAAA|896|Pine 4th|Dr.|Suite 40|Woodland|Warren County|MO|64854|United States|-6|apartment| +24366|AAAAAAAAOCPFAAAA|708|South |Drive|Suite 20|Belmont|Comanche County|OK|70191|United States|-6|condo| +24367|AAAAAAAAPCPFAAAA|39|Park |Circle|Suite S|Farmington|Pipestone County|MN|59145|United States|-6|single family| +24368|AAAAAAAAADPFAAAA|642|Birch |Way|Suite 140|Fairfield|Brooks County|GA|36192|United States|-5|apartment| +24369|AAAAAAAABDPFAAAA||Pine ||Suite P|Edgewood|Pittsburg County||||-6|apartment| +24370|AAAAAAAACDPFAAAA|637||||Spring Hill||||United States||| +24371|AAAAAAAADDPFAAAA|890|Mill Poplar|Dr.|Suite X|Shiloh|Cherokee County|KS|69275|United States|-6|condo| +24372|AAAAAAAAEDPFAAAA|563|Sixth |RD|Suite 320|Griffin|Linn County|KS|61204|United States|-6|apartment| +24373|AAAAAAAAFDPFAAAA|5|View |Ln|Suite 70|Woodland|Fairfield County|OH|44854|United States|-5|apartment| +24374|AAAAAAAAGDPFAAAA|44|Lakeview |Blvd|Suite 260|Pleasant Valley|Albany County|WY|82477|United States|-7|condo| +24375|AAAAAAAAHDPFAAAA|73|Fifth |Cir.|Suite P|Providence|Madison County|KY|46614|United States|-5|single family| +24376|AAAAAAAAIDPFAAAA|500|Fifth Church|Parkway|Suite 110|Springfield|Knox County|OH|49303|United States|-5|single family| +24377|AAAAAAAAJDPFAAAA|792|Meadow 4th|Way|Suite 90|Highland|Camas County|ID|89454|United States|-7|single family| +24378|AAAAAAAAKDPFAAAA|807|Oak |Street|Suite 420|Pleasant Grove|Stone County|MS|54136|United States|-6|apartment| +24379|AAAAAAAALDPFAAAA|242|15th Ridge|Blvd|Suite 280|Bethel|Menifee County|KY|45281|United States|-5|apartment| +24380|AAAAAAAAMDPFAAAA|907|Walnut |Ave|Suite W|Buena Vista|Carbon County|UT|85752|United States|-7|condo| +24381|AAAAAAAANDPFAAAA|713|First |Wy|Suite U|Lakeside|Chickasaw County|MS|59532|United States|-6|single family| +24382|AAAAAAAAODPFAAAA|799|Laurel Oak|RD|Suite A|Plainview|Camp County|TX|73683|United States|-6|single family| +24383|AAAAAAAAPDPFAAAA|359|3rd 5th|Drive|Suite 400|Forest Hills|Charlotte County|VA|29237|United States|-5|condo| +24384|AAAAAAAAAEPFAAAA|540|Ridge Ash|Lane|Suite 310|Greenfield|Jasper County|MS|55038|United States|-6|apartment| +24385|AAAAAAAABEPFAAAA|700|North |Dr.|Suite 320|Springfield|Pierce County|GA|39303|United States|-5|condo| +24386|AAAAAAAACEPFAAAA|232|Walnut Maple|RD|Suite G|Mount Olive|Grady County|OK|78059|United States|-6|condo| +24387|AAAAAAAADEPFAAAA|212|Main Forest|Cir.|Suite X|Sunnyside|Neosho County|KS|61952|United States|-6|single family| +24388|AAAAAAAAEEPFAAAA|619|Center Willow|Ave|Suite V|Union Hill|Montgomery County|PA|17746|United States|-5|apartment| +24389|AAAAAAAAFEPFAAAA|523|Jackson View|Street|Suite 0|Dewey|Marinette County|WI|51160|United States|-6|apartment| +24390|AAAAAAAAGEPFAAAA|699|West Sycamore|Parkway|Suite 350|Morris|Archuleta County|CO|86696|United States|-7|condo| +24391|AAAAAAAAHEPFAAAA|732|Washington Walnut|Avenue|Suite 60|Bridgeport|Kemper County|MS|55817|United States|-6|condo| +24392|AAAAAAAAIEPFAAAA|||Ave|Suite 250|Sulphur Springs|||78354|United States|-6|apartment| +24393|AAAAAAAAJEPFAAAA|660|2nd |Ln|Suite D|Hamilton|Hot Springs County|WY|82808|United States|-7|single family| +24394|AAAAAAAAKEPFAAAA|699|5th Sunset|Avenue|Suite 10|Bunker Hill|King County|WA|90150|United States|-8|apartment| +24395|AAAAAAAALEPFAAAA|48|9th Adams|Avenue|Suite 190|Highland|Wheatland County|MT|69454|United States|-7|condo| +24396|AAAAAAAAMEPFAAAA|351|Pine |ST|Suite U|White Oak|Westmoreland County|PA|16668|United States|-5|single family| +24397|AAAAAAAANEPFAAAA|967|Locust |ST|Suite 220|Summit|Carroll County|AR|70499|United States|-6|apartment| +24398|AAAAAAAAOEPFAAAA|638|3rd 15th|Circle|Suite J|Waterloo|Bath County|VA|21675|United States|-5|condo| +24399|AAAAAAAAPEPFAAAA|803|Spring |Avenue|Suite P|Sulphur Springs|Mills County|IA|58354|United States|-6|single family| +24400|AAAAAAAAAFPFAAAA|889|Park 2nd|Dr.|Suite D|Cedar Grove|Carroll County|IN|40411|United States|-5|condo| +24401|AAAAAAAABFPFAAAA|303|Oak Spring|Avenue|Suite K|Red Hill|Teton County|WY|84338|United States|-7|condo| +24402|AAAAAAAACFPFAAAA|907|2nd Cedar|Road|Suite X|Franklin|Delta County|MI|49101|United States|-5|condo| +24403|AAAAAAAADFPFAAAA|356|Lake |Cir.|Suite 310|Oak Grove|Osage County|KS|68370|United States|-6|single family| +24404|AAAAAAAAEFPFAAAA|158|Lakeview Jackson|Cir.|Suite 490|Edgewood|Evangeline Parish|LA|70069|United States|-6|apartment| +24405|AAAAAAAAFFPFAAAA|720|Washington Ridge|Cir.|Suite 360|Spring Hill|Charlton County|GA|36787|United States|-5|single family| +24406|AAAAAAAAGFPFAAAA|997|Laurel Oak|Ln|Suite P|Fairview|Washington County|CO|85709|United States|-7|single family| +24407|AAAAAAAAHFPFAAAA|465|Birch |Ct.|Suite 490|Lincoln|San Saba County|TX|71289|United States|-6|single family| +24408|AAAAAAAAIFPFAAAA|785|East Park|Boulevard|Suite V|Riverside|Weld County|CO|89231|United States|-7|single family| +24409|AAAAAAAAJFPFAAAA|248|Sunset |Ave|Suite C|Little River|Mayes County|OK|70319|United States|-6|single family| +24410|AAAAAAAAKFPFAAAA|387|Spring 5th|Cir.|Suite V|Newport|Rawlins County|KS|61521|United States|-6|single family| +24411|AAAAAAAALFPFAAAA|228|Chestnut |Way|Suite W|Macedonia|Coryell County|TX|71087|United States|-6|single family| +24412|AAAAAAAAMFPFAAAA|992|West |Drive|Suite V|Hillcrest|Reagan County|TX|73003|United States|-6|condo| +24413|AAAAAAAANFPFAAAA|160|Jackson 8th|ST|Suite 110|Centerville|Dunn County|WI|50059|United States|-6|apartment| +24414|AAAAAAAAOFPFAAAA|593|Oak Park||||Knox County|OH|47746|United States||| +24415|AAAAAAAAPFPFAAAA|892|Washington Washington|Pkwy|Suite K|Liberty|Harris County|GA|33451|United States|-5|single family| +24416|AAAAAAAAAGPFAAAA|680|College Maple|Street|Suite K|Union Hill|Columbus County|NC|27746|United States|-5|single family| +24417|AAAAAAAABGPFAAAA|633|Birch |Road|Suite E|Sunnyside|Chilton County|AL|31952|United States|-6|condo| +24418|AAAAAAAACGPFAAAA|996|8th |Drive|Suite 480|Harmony|Rawlins County|KS|65804|United States|-6|apartment| +24419|AAAAAAAADGPFAAAA|651|2nd Maple|Avenue|Suite B|Lewisburg|Union County|MS|57538|United States|-6|single family| +24420|AAAAAAAAEGPFAAAA|547|View |Circle|Suite Q|Leon|Drew County|AR|70913|United States|-6|condo| +24421|AAAAAAAAFGPFAAAA|85|Church |Parkway|Suite F|Greenwood|Ada County|ID|88828|United States|-7|single family| +24422|AAAAAAAAGGPFAAAA|869|Madison |Wy|Suite L|Wilson|Harney County|OR|96971|United States|-8|apartment| +24423|AAAAAAAAHGPFAAAA|656|Twelfth |RD|Suite 430|Union|New Kent County|VA|28721|United States|-5|condo| +24424|AAAAAAAAIGPFAAAA|963|Willow 12th|Boulevard|Suite 240|Hillcrest|Wakulla County|FL|33003|United States|-5|apartment| +24425|AAAAAAAAJGPFAAAA|174|West Locust|Pkwy|Suite 170|Richfield|Crittenden County|AR|76196|United States|-6|condo| +24426|AAAAAAAAKGPFAAAA|274|4th |Pkwy|Suite 480|Bethel|San Luis Obispo County|CA|95281|United States|-8|apartment| +24427|AAAAAAAALGPFAAAA|832|South |Drive|Suite T|Ashley|Halifax County|VA|24324|United States|-5|apartment| +24428|AAAAAAAAMGPFAAAA|223|North ||Suite H|Red Hill|Park County|WY||||single family| +24429|AAAAAAAANGPFAAAA|647|1st Church|Boulevard|Suite M|Newport|Muscogee County|GA|31521|United States|-5|apartment| +24430|AAAAAAAAOGPFAAAA|830|1st Third|Ln|Suite 160|Springtown|Warren County|MS|59858|United States|-6|apartment| +24431|AAAAAAAAPGPFAAAA|457|Dogwood Laurel|Road|Suite 490|Woodville|Warren County|OH|44289|United States|-5|condo| +24432|AAAAAAAAAHPFAAAA|758|8th East|Court|Suite 390|Clifton|Buchanan County|IA|58014|United States|-6|single family| +24433|AAAAAAAABHPFAAAA|747|Pine Davis|ST|Suite 210|Mountain View|Summit County|CO|84466|United States|-7|apartment| +24434|AAAAAAAACHPFAAAA|734|10th |Blvd|Suite Y|Lincoln|Greeley County|KS|61289|United States|-6|apartment| +24435|AAAAAAAADHPFAAAA|478|River |Parkway|Suite X|Bethel|Ramsey County|ND|55281|United States|-6|condo| +24436|AAAAAAAAEHPFAAAA|494|View |Court|Suite D|Valley View|Laurens County|GA|35124|United States|-5|single family| +24437|AAAAAAAAFHPFAAAA|463|10th Green|Avenue|Suite W|Fairview|Hood County|TX|75709|United States|-6|apartment| +24438|AAAAAAAAGHPFAAAA|130|Locust Seventh|ST|Suite W|Wilson|Plymouth County|IA|56971|United States|-6|single family| +24439|AAAAAAAAHHPFAAAA|87|Williams |Lane|Suite 250|Ridgeville|Braxton County|WV|29306|United States|-5|single family| +24440|AAAAAAAAIHPFAAAA|681|Davis |Way|Suite 180|Oakwood|Polk County|MO|60169|United States|-6|condo| +24441|AAAAAAAAJHPFAAAA|903|Green Adams|Drive|Suite X|Sawyer|Houston County|GA|36045|United States|-5|condo| +24442|AAAAAAAAKHPFAAAA|746|11th |Dr.|Suite 370|Franklin|Ashe County|NC|29101|United States|-5|apartment| +24443|AAAAAAAALHPFAAAA|629|Hillcrest |Road|Suite L|Marion|Jackson County|GA|30399|United States|-5|single family| +24444|AAAAAAAAMHPFAAAA|987|Oak Washington|Drive|Suite Q|Valley View|Knox County|NE|65124|United States|-7|single family| +24445|AAAAAAAANHPFAAAA|624|Poplar Church|Way|Suite 250|Hamilton|Cass County|ND|52808|United States|-6|apartment| +24446|AAAAAAAAOHPFAAAA|704|1st |Blvd|Suite O|Jamestown|Madison County|NY|16867|United States|-5|condo| +24447|AAAAAAAAPHPFAAAA|841|Center Ninth|Road|Suite X|Red Hill|Dutchess County|NY|14338|United States|-5|apartment| +24448|AAAAAAAAAIPFAAAA|330|Willow Sixth|Drive|Suite R|Deerfield|Cheyenne County|KS|69840|United States|-6|condo| +24449|AAAAAAAABIPFAAAA|35|Fourth Locust|Wy|Suite 170|Woodland|Montgomery County|MO|64854|United States|-6|apartment| +24450|AAAAAAAACIPFAAAA|161|Center 7th|Wy|Suite 60|Centerville|Liberty County|TX|70059|United States|-6|single family| +24451|AAAAAAAADIPFAAAA|363|2nd Green|Circle|Suite U|Riverdale|Logan County|CO|89391|United States|-7|single family| +24452|AAAAAAAAEIPFAAAA|363|10th |Parkway|Suite 180|Franklin|Clay County|IL|69101|United States|-6|single family| +24453|AAAAAAAAFIPFAAAA|310|Park |Avenue|Suite 130|Clifton|Routt County|CO|88014|United States|-7|apartment| +24454|AAAAAAAAGIPFAAAA|270|Oak |Dr.|Suite H|Leon|Wise County|TX|70913|United States|-6|apartment| +24455|AAAAAAAAHIPFAAAA|361|Woodland Hill|Wy|Suite 400|Springfield|San Juan County|UT|89303|United States|-7|condo| +24456|AAAAAAAAIIPFAAAA|409|Fifth |Blvd|Suite T|Woodrow|Dubuque County|IA|54273|United States|-6|condo| +24457|AAAAAAAAJIPFAAAA|545|Walnut Jefferson|RD|Suite U|Shady Grove|Eastland County|TX|72812|United States|-6|apartment| +24458|AAAAAAAAKIPFAAAA|728|Hillcrest |Ave|Suite A|Springdale|Prince George County|MD|28883|United States|-5|single family| +24459|AAAAAAAALIPFAAAA|193|View Fifteenth|Parkway|Suite 250|Vance|Hamlin County|SD|50268|United States|-7|apartment| +24460|AAAAAAAAMIPFAAAA|663|Park Central|Parkway|Suite Y|Marion|Bottineau County|ND|50399|United States|-6|apartment| +24461|AAAAAAAANIPFAAAA|842|North |Street|Suite 390|White Oak|Walker County|TX|76668|United States|-6|condo| +24462|AAAAAAAAOIPFAAAA|474|Sunset Pine|Street|Suite 450|Five Points|La Plata County|CO|86098|United States|-7|apartment| +24463|AAAAAAAAPIPFAAAA|398|Center 5th|Lane|Suite F|Carter|Russell County|KY|40919|United States|-5|condo| +24464|AAAAAAAAAJPFAAAA|548|Ridge |Ct.|Suite 400|Silver Springs|Cleveland County|NC|24843|United States|-5|single family| +24465|AAAAAAAABJPFAAAA|464|Lincoln |Ct.|Suite 210|Greenville|Warren County|OH|41387|United States|-5|apartment| +24466|AAAAAAAACJPFAAAA|634|Cherry Ridge|Boulevard|Suite S|Oakland|Spalding County|GA|39843|United States|-5|apartment| +24467|AAAAAAAADJPFAAAA|277|Hickory |ST|Suite C|Macedonia|Stutsman County|ND|51087|United States|-6|single family| +24468|AAAAAAAAEJPFAAAA|985|Fourth Cherry|Lane|Suite B|Spring Valley|Hawkins County|TN|36060|United States|-5|condo| +24469|AAAAAAAAFJPFAAAA|4|Hill |||||MI|40534|United States||| +24470|AAAAAAAAGJPFAAAA|955|Main |Wy|Suite C|Wilson|Adair County|IA|56971|United States|-6|condo| +24471|AAAAAAAAHJPFAAAA|443|Oak Chestnut|Wy|Suite 110|Harmony|Tate County|MS|55804|United States|-6|apartment| +24472|AAAAAAAAIJPFAAAA|800||Road||Valley View|||55124|||single family| +24473|AAAAAAAAJJPFAAAA|4|Second Lee|Cir.|Suite 80|Pleasant Valley|Lucas County|IA|52477|United States|-6|single family| +24474|AAAAAAAAKJPFAAAA|713|North Center|Blvd|Suite 140|Friendship|Guilford County|NC|24536|United States|-5|apartment| +24475|AAAAAAAALJPFAAAA|160|Highland Davis|Ct.|Suite 360|Greenwood|Defiance County|OH|48828|United States|-5|single family| +24476|AAAAAAAAMJPFAAAA|869|5th |Street|Suite F|Springdale|Ellis County|OK|78883|United States|-6|single family| +24477|AAAAAAAANJPFAAAA|64|Willow West|Cir.|Suite 220|Mount Pleasant|Marion County|SC|21933|United States|-5|apartment| +24478|AAAAAAAAOJPFAAAA|177|North |RD|Suite 420|Lakeview|Cayuga County|NY|18579|United States|-5|single family| +24479|AAAAAAAAPJPFAAAA|511|Main |Lane|Suite 90|Highland Park|Monroe County|IL|66534|United States|-6|condo| +24480|AAAAAAAAAKPFAAAA|914|Cherry |Court|Suite 340|Springfield|Linn County|IA|59303|United States|-6|apartment| +24481|AAAAAAAABKPFAAAA|202||Dr.|Suite 220|Riverview|||||-5|single family| +24482|AAAAAAAACKPFAAAA|187|Ash Lincoln|Circle|Suite 20|Spring Valley|Archer County|TX|76060|United States|-6|apartment| +24483|AAAAAAAADKPFAAAA|636|Third |Boulevard|Suite V|Greenwood|Carroll County|TN|38828|United States|-5|condo| +24484|AAAAAAAAEKPFAAAA|714|Ridge |Ave|Suite X|Forest Hills|Texas County|OK|79237|United States|-6|single family| +24485|AAAAAAAAFKPFAAAA|742|Main Hillcrest|Pkwy|Suite Q|Farmington|Stark County|ND|59145|United States|-6|condo| +24486|AAAAAAAAGKPFAAAA|941|3rd |Circle|Suite F|Riverside|Jefferson County|TX|79231|United States|-6|apartment| +24487|AAAAAAAAHKPFAAAA|847|Second |Drive|Suite 190|Florence|Red Lake County|MN|53394|United States|-6|apartment| +24488|AAAAAAAAIKPFAAAA|426|Park |Lane|Suite Y|Ferguson|Roger Mills County|OK|71821|United States|-6|condo| +24489|AAAAAAAAJKPFAAAA|140|East College|Pkwy|Suite 110|Liberty|Marion County|TX|73451|United States|-6|single family| +24490|AAAAAAAAKKPFAAAA|217|Spring |Road|Suite 340|Brookwood|Smith County|KS|60965|United States|-6|condo| +24491|AAAAAAAALKPFAAAA|26|Poplar 8th|Pkwy|Suite 430|Fairfield|Meigs County|OH|46192|United States|-5|condo| +24492|AAAAAAAAMKPFAAAA|702|Spring |Road|Suite S|Shiloh|Howard County|AR|79275|United States|-6|condo| +24493|AAAAAAAANKPFAAAA|678|Park Johnson|Lane|Suite 120|Greenfield|Northumberland County|VA|25038|United States|-5|apartment| +24494|AAAAAAAAOKPFAAAA|920|Madison |Wy|Suite D|Mount Pleasant|Winona County|MN|51933|United States|-6|single family| +24495|AAAAAAAAPKPFAAAA|846|Second |Way|Suite 60|Bridgeport|Rutland County|VT|06417|United States|-5|condo| +24496|AAAAAAAAALPFAAAA|593|Maple Ridge|RD|Suite 190|Hurricane|Caledonia County|VT|08244|United States|-5|single family| +24497|AAAAAAAABLPFAAAA|65|11th |Blvd|Suite 0|Kingston|Hampton city|VA|24975|United States|-5|condo| +24498|AAAAAAAACLPFAAAA|64|Pine |Cir.|Suite P|Ashland|Lamar County|MS|54244|United States|-6|apartment| +24499|AAAAAAAADLPFAAAA|725|4th Madison|Cir.|Suite 150|Liberty|Gaston County|NC|23451|United States|-5|single family| +24500|AAAAAAAAELPFAAAA|282|2nd |Ave|Suite 40|Bridgeport|Benton County|IN|45817|United States|-5|apartment| +24501|AAAAAAAAFLPFAAAA|199|4th Fifth|Circle|Suite 230|Glendale|Lawrence County|AL|33951|United States|-6|condo| +24502|AAAAAAAAGLPFAAAA|382|Oak 12th|Drive|Suite 180|Midway|Troup County|GA|31904|United States|-5|condo| +24503|AAAAAAAAHLPFAAAA|180|Jefferson Park|Ct.|Suite 130|Summit|East Feliciana Parish|LA|70499|United States|-6|condo| +24504|AAAAAAAAILPFAAAA|982|Jefferson |Pkwy|Suite H|Union Hill|Brooks County|GA|37746|United States|-5|apartment| +24505|AAAAAAAAJLPFAAAA|700|Mill |Cir.|Suite 200|Greenwood|Iowa County|WI|58828|United States|-6|single family| +24506|AAAAAAAAKLPFAAAA|448|Oak Oak|Blvd|Suite O|Highland|Scotland County|MO|69454|United States|-6|single family| +24507|AAAAAAAALLPFAAAA|904|7th |Wy|Suite F|Enterprise|Polk County|MO|61757|United States|-6|apartment| +24508|AAAAAAAAMLPFAAAA|966|6th Oak|Ave|Suite 420|Edgewood|Dewey County|SD|50069|United States|-6|condo| +24509|AAAAAAAANLPFAAAA|356|Maple 6th|Ct.|Suite 0|Centerville|Graves County|KY|40059|United States|-6|apartment| +24510|AAAAAAAAOLPFAAAA|383|Elm Cherry|Street|Suite 360|Deerfield|Appling County|GA|39840|United States|-5|apartment| +24511|AAAAAAAAPLPFAAAA|342|Johnson Maple|Circle|Suite 490|Greenville|Louisa County|IA|51387|United States|-6|single family| +24512|AAAAAAAAAMPFAAAA|259|10th |ST|Suite 210|Clifford|Salem County|NJ|08764|United States|-5|single family| +24513|AAAAAAAABMPFAAAA|635|Williams |Court|||Manistee County|||United States||| +24514|AAAAAAAACMPFAAAA|464|Maple |Street|Suite 30|Jamestown|Madison County|FL|36867|United States|-5|condo| +24515|AAAAAAAADMPFAAAA|448|Mill |Drive|Suite N|Riverside|Smith County|MS|59231|United States|-6|single family| +24516|AAAAAAAAEMPFAAAA|948|Cherry |Ave|Suite N|Clifton|Crawford County|GA|38014|United States|-5|apartment| +24517|AAAAAAAAFMPFAAAA|855|Hickory Smith|Drive|Suite C|Salem|Teton County|MT|68048|United States|-7|condo| +24518|AAAAAAAAGMPFAAAA|180|West Mill|Dr.|Suite K|Shiloh|Payette County|ID|89275|United States|-7|single family| +24519|AAAAAAAAHMPFAAAA|431|Johnson |Parkway|Suite D|Wildwood|Montour County|PA|16871|United States|-5|condo| +24520|AAAAAAAAIMPFAAAA|340|Maple |Wy|Suite 40|Lebanon|Douglas County|NV|82898|United States|-8|apartment| +24521|AAAAAAAAJMPFAAAA|120|11th |RD|Suite 260|Pleasant Hill|Wilcox County|AL|33604|United States|-6|single family| +24522|AAAAAAAAKMPFAAAA|342|Second |Road|Suite T|Hamilton|Mobile County|AL|32808|United States|-6|apartment| +24523|AAAAAAAALMPFAAAA|847|Franklin 9th|Street|Suite 80|Cedar Grove|Heard County|GA|30411|United States|-5|condo| +24524|AAAAAAAAMMPFAAAA|301|3rd College|Wy|Suite 220|Oakland|Milam County|TX|79843|United States|-6|condo| +24525|AAAAAAAANMPFAAAA|146|River |RD|Suite 180|Brookville|Graham County|AZ|83524|United States|-7|apartment| +24526|AAAAAAAAOMPFAAAA|642|Spruce |Boulevard|Suite 390|Wilson|Dent County|MO|66971|United States|-6|apartment| +24527|AAAAAAAAPMPFAAAA|556|13th |Ave|Suite 310|Bunker Hill|Russell County|KS|60150|United States|-6|apartment| +24528|AAAAAAAAANPFAAAA|645|Franklin |Street|Suite G|Newtown|Belmont County|OH|41749|United States|-5|condo| +24529|AAAAAAAABNPFAAAA|509|Elm 5th|Lane|Suite 430|Lakeside|Chautauqua County|NY|19532|United States|-5|single family| +24530|AAAAAAAACNPFAAAA|632|Jefferson River|Way|Suite 370|Greenwood|Wayne County|UT|88828|United States|-7|condo| +24531|AAAAAAAADNPFAAAA|615|Willow |Street|Suite K|Green Acres|Burke County|ND|57683|United States|-6|single family| +24532|AAAAAAAAENPFAAAA|840|Pine Smith|Lane|Suite S|Waterloo|Wayne County|NE|61675|United States|-7|apartment| +24533|AAAAAAAAFNPFAAAA|657|West |Ave|Suite E|Ashland|Montrose County|CO|84244|United States|-7|condo| +24534|AAAAAAAAGNPFAAAA|300|Cherry |Road|Suite J|Farmington|Wyoming County|NY|19145|United States|-5|apartment| +24535|AAAAAAAAHNPFAAAA|460|Elm |Road|Suite E|Lebanon|Macomb County|MI|42898|United States|-5|apartment| +24536|AAAAAAAAINPFAAAA|597|Sunset Maple|Cir.|Suite M|Harmony|Shackelford County|TX|75804|United States|-6|condo| +24537|AAAAAAAAJNPFAAAA|579|6th Spruce|RD|Suite 340|Sulphur Springs|Missoula County|MT|68354|United States|-7|condo| +24538|AAAAAAAAKNPFAAAA|372|4th |Wy|Suite X|Hamilton|Sedgwick County|CO|82808|United States|-7|single family| +24539|AAAAAAAALNPFAAAA|511|Willow |Avenue|Suite 170|Oakland|Chippewa County|MN|59843|United States|-6|single family| +24540|AAAAAAAAMNPFAAAA|594|Railroad |Blvd|Suite G|Springfield|Washington County|TN|39303|United States|-6|apartment| +24541|AAAAAAAANNPFAAAA|362|Pine |Wy|Suite 230|Oak Ridge|Louisa County|IA|58371|United States|-6|condo| +24542|AAAAAAAAONPFAAAA|92|10th |Lane|Suite 200|Jamestown|Montmorency County|MI|46867|United States|-5|condo| +24543|AAAAAAAAPNPFAAAA|541|1st |Road|Suite T|Hopewell|Cuming County|NE|60587|United States|-6|apartment| +24544|AAAAAAAAAOPFAAAA|434|Railroad Cherry|Ct.|Suite 290|Concord|Montague County|TX|74107|United States|-6|single family| +24545|AAAAAAAABOPFAAAA|389|Cedar Second|Street|Suite 400|Brentwood|Hall County|NE|64188|United States|-6|apartment| +24546|AAAAAAAACOPFAAAA|615|Maple Dogwood|Wy|Suite 480|Summit|Musselshell County|MT|60499|United States|-7|apartment| +24547|AAAAAAAADOPFAAAA|454|Hillcrest |Pkwy|Suite 440|Green Acres|Lea County|NM|87683|United States|-7|apartment| +24548|AAAAAAAAEOPFAAAA|683|Cedar Dogwood|Court|Suite 50|Hopewell|Sherman County|NE|60587|United States|-7|single family| +24549|AAAAAAAAFOPFAAAA|644|Locust Willow|Ave|Suite 40|Highland Park|Jackson County|SD|56534|United States|-7|apartment| +24550|AAAAAAAAGOPFAAAA|255|Cherry Highland|Ln|Suite I|Maple Grove|Montmorency County|MI|48252|United States|-5|single family| +24551|AAAAAAAAHOPFAAAA|422|Washington |Way|Suite 190|Five Forks|Hamilton County|IN|42293|United States|-5|condo| +24552|AAAAAAAAIOPFAAAA|431|Lincoln |Wy|Suite G|Glenwood|Butte County|CA|93511|United States|-8|apartment| +24553|AAAAAAAAJOPFAAAA|306|Lee Birch|Parkway|Suite 300|Arlington|Stafford County|VA|26557|United States|-5|single family| +24554|AAAAAAAAKOPFAAAA|750|Park |Road|Suite 180|Concord|Taylor County|IA|54107|United States|-6|single family| +24555|AAAAAAAALOPFAAAA|531|Chestnut Hill|Avenue||||TX||||single family| +24556|AAAAAAAAMOPFAAAA|368|Jefferson |ST|Suite 170|Mount Pleasant|Bienville Parish|LA|71933|United States|-6|single family| +24557|AAAAAAAANOPFAAAA|838|Birch Maple|Lane|Suite 320|Walnut|Jefferson County|WV|26245|United States|-5|apartment| +24558|AAAAAAAAOOPFAAAA|407|Cherry 6th|Avenue|Suite 400|Friendship|Early County|GA|34536|United States|-5|condo| +24559|AAAAAAAAPOPFAAAA|889|Franklin Park|Drive|Suite Q|Mount Zion|Barrow County|GA|38054|United States|-5|condo| +24560|AAAAAAAAAPPFAAAA|351|Lake |Ct.|Suite T|Harmony|Jefferson County|IN|45804|United States|-5|condo| +24561|AAAAAAAABPPFAAAA|999|Green |Court|Suite 100|Shannon|Burleigh County|ND|54120|United States|-6|single family| +24562|AAAAAAAACPPFAAAA|397|Valley |Ct.|Suite W|Arlington|Eddy County|NM|86557|United States|-7|apartment| +24563|AAAAAAAADPPFAAAA|605|Center |Blvd|Suite G|Roxbury|Faribault County|MN|55508|United States|-6|condo| +24564|AAAAAAAAEPPFAAAA|521|1st |Pkwy|Suite 10|Highland|San Miguel County|CO|89454|United States|-7|single family| +24565|AAAAAAAAFPPFAAAA|401|View Park|Avenue|Suite B|Oak Grove|Sullivan County|MO|68370|United States|-6|condo| +24566|AAAAAAAAGPPFAAAA|681|Elm |Cir.|Suite D|Spring Valley|Marshall County|IN|46060|United States|-5|condo| +24567|AAAAAAAAHPPFAAAA|333|Madison |Cir.|Suite 390|Springfield|Montgomery County|GA|39303|United States|-5|condo| +24568|AAAAAAAAIPPFAAAA|444||Wy||Edgewood|||||-5|| +24569|AAAAAAAAJPPFAAAA|100|Ridge |Way|Suite 100|Spring Hill|Clinton County|OH|46787|United States|-5|condo| +24570|AAAAAAAAKPPFAAAA|677|Oak Chestnut|Avenue|Suite C|Lakewood|Clark County|OH|48877|United States|-5|condo| +24571|AAAAAAAALPPFAAAA|322|Sycamore West|Cir.|Suite W|Springfield|Clay County|TN|39303|United States|-5|condo| +24572|AAAAAAAAMPPFAAAA|278|Ash 6th|Circle|Suite 320|Marion|Polk County|TX|70399|United States|-6|apartment| +24573|AAAAAAAANPPFAAAA|858|11th Lake|RD|Suite A|Mechanicsburg|Floyd County|IA|52219|United States|-6|condo| +24574|AAAAAAAAOPPFAAAA|618|Cherry Cedar|Boulevard|Suite 80|Franklin|Pulaski County|MO|69101|United States|-6|single family| +24575|AAAAAAAAPPPFAAAA|34|Cedar |Court|||||05454||-5|| +24576|AAAAAAAAAAAGAAAA|22|Lee |Lane|Suite J|Oakdale|Isle of Wight County|VA|29584|United States|-5|single family| +24577|AAAAAAAABAAGAAAA|330|2nd |Pkwy|Suite A|Riverdale|Marion County|OR|99391|United States|-8|single family| +24578|AAAAAAAACAAGAAAA||North ||Suite 150|Mountain View||NY|14466||-5|apartment| +24579|AAAAAAAADAAGAAAA||Birch |ST|Suite S|Oakland|Franklin County|NC||United States||| +24580|AAAAAAAAEAAGAAAA|789|Highland |Ln|Suite J|Hillcrest|Calhoun County|MI|43003|United States|-5|apartment| +24581|AAAAAAAAFAAGAAAA|180|Spring Park|Ln|Suite H|Friendship|Pulaski County|MO|64536|United States|-6|condo| +24582|AAAAAAAAGAAGAAAA|73|Park Hillcrest|Avenue|Suite Q|Ashland|Floyd County|IA|54244|United States|-6|condo| +24583|AAAAAAAAHAAGAAAA|311|Lincoln |RD|Suite D|Valley View|Vigo County|IN|45124|United States|-5|single family| +24584|AAAAAAAAIAAGAAAA|337|Eigth Oak|Ct.|Suite 60|Ashland|Meagher County|MT|64244|United States|-7|single family| +24585|AAAAAAAAJAAGAAAA|200|Elm Miller|Street|Suite 50|Shiloh|Lincoln County|SD|59275|United States|-7|apartment| +24586|AAAAAAAAKAAGAAAA|648|Thirteenth |Street|Suite O|Fairfield|Sandoval County|NM|86192|United States|-7|single family| +24587|AAAAAAAALAAGAAAA|349|Laurel |Parkway|Suite Y|Tremont|Lampasas County|TX|79515|United States|-6|condo| +24588|AAAAAAAAMAAGAAAA|384|Miller Birch|Wy|Suite 370|Centerville|Gilliam County|OR|90059|United States|-8|condo| +24589|AAAAAAAANAAGAAAA|316|Jackson Oak|Blvd|Suite W|Centerville|Carroll County|IN|40059|United States|-5|apartment| +24590|AAAAAAAAOAAGAAAA|337|First Fourth|Wy|Suite 90|Red Hill|Los Angeles County|CA|94338|United States|-8|condo| +24591|AAAAAAAAPAAGAAAA|972|Park Ash|ST|Suite S|Five Points|Crawford County|IN|46098|United States|-5|condo| +24592|AAAAAAAAABAGAAAA|669|Elm |Pkwy|Suite 490|Friendship|Osage County|MO|64536|United States|-6|condo| +24593|AAAAAAAABBAGAAAA|282|Hillcrest Lincoln|Parkway|Suite D|Green Acres|Monroe County|PA|17683|United States|-5|apartment| +24594|AAAAAAAACBAGAAAA|32|Woodland |Wy|Suite Y|Riverside|Falls County|TX|79231|United States|-6|condo| +24595|AAAAAAAADBAGAAAA|375||||Shiloh||TX||||| +24596|AAAAAAAAEBAGAAAA|798|Park |Drive|Suite 100|Oakwood|Lamoille County|VT|00769|United States|-5|single family| +24597|AAAAAAAAFBAGAAAA|461|Oak |RD|Suite 100|Farmington|Glascock County|GA|39145|United States|-5|condo| +24598|AAAAAAAAGBAGAAAA||Third |||||MO|66534|United States|-6|| +24599|AAAAAAAAHBAGAAAA|376|Meadow |Cir.|Suite Q|Montague|Lee County|AL|34062|United States|-6|condo| +24600|AAAAAAAAIBAGAAAA|394|7th |Boulevard|Suite I|Valley View|Motley County|TX|75124|United States|-6|apartment| +24601|AAAAAAAAJBAGAAAA|445|8th |Drive|Suite Y|Omega|Greenwood County|SC|26364|United States|-5|single family| +24602|AAAAAAAAKBAGAAAA|429|Madison 6th|Road|Suite Q|Hamilton|Hale County|AL|32808|United States|-6|condo| +24603|AAAAAAAALBAGAAAA|764|Walnut |Street|Suite B|Brownsville|Lumpkin County|GA|39310|United States|-5|condo| +24604|AAAAAAAAMBAGAAAA|279|Hillcrest Washington|Dr.|Suite J|Pleasant Grove|Towner County|ND|54136|United States|-6|apartment| +24605|AAAAAAAANBAGAAAA|757|Park Central|Parkway|Suite Q|Highland Park|Martin County|KY|46534|United States|-5|apartment| +24606|AAAAAAAAOBAGAAAA|579|Oak |Court|Suite O|Riverview|Henry County|AL|39003|United States|-6|single family| +24607|AAAAAAAAPBAGAAAA|13|10th Second|Road|Suite 50|Shiloh|Mercer County|OH|49275|United States|-5|single family| +24608|AAAAAAAAACAGAAAA|777|Davis Lakeview|Ln|Suite 200|Kirkland|Harper County|KS|67896|United States|-6|condo| +24609|AAAAAAAABCAGAAAA|113|South |Ave|Suite D|Sunnyside|Conway County|AR|71952|United States|-6|condo| +24610|AAAAAAAACCAGAAAA|64||Court|Suite R|Spring Hill|Coleman County|||United States|-6|condo| +24611|AAAAAAAADCAGAAAA|815|Oak Lake|RD|Suite 310|Fairview|Lane County|OR|95709|United States|-8|apartment| +24612|AAAAAAAAECAGAAAA|||Pkwy|||||28605|United States|-5|| +24613|AAAAAAAAFCAGAAAA|547|Park Jackson|Ln|Suite H|Union|Nome Census Area|AK|98721|United States|-9|condo| +24614|AAAAAAAAGCAGAAAA|545|Hillcrest |Blvd|Suite 210|Concord|Webster County|MS|54107|United States|-6|single family| +24615|AAAAAAAAHCAGAAAA|124|Elm 7th|Ln|Suite W|Antioch|Runnels County|TX|78605|United States|-6|single family| +24616|AAAAAAAAICAGAAAA|103|East 4th|Dr.|Suite R|Union Hill|Smith County|TX|77746|United States|-6|condo| +24617|AAAAAAAAJCAGAAAA|170|First Poplar|Pkwy|Suite 410|Walnut Grove|Livingston County|IL|67752|United States|-6|condo| +24618|AAAAAAAAKCAGAAAA|371|Fifth 2nd|Pkwy|Suite 100|Forest Hills|Lawrence County|AR|79237|United States|-6|condo| +24619|AAAAAAAALCAGAAAA|455|2nd Lincoln|Cir.|Suite N|Five Forks|Washington County|IN|42293|United States|-5|apartment| +24620|AAAAAAAAMCAGAAAA|701|Center North|ST|Suite 490|Kingston|Washington County|TN|34975|United States|-6|condo| +24621|AAAAAAAANCAGAAAA|||Cir.||Leesburg||||United States|-5|| +24622|AAAAAAAAOCAGAAAA|635|1st |Street|Suite 270|Ludlow|Clinton County|IA|55566|United States|-6|single family| +24623|AAAAAAAAPCAGAAAA|481|1st |Ave|Suite U|Macedonia|Hansford County|TX|71087|United States|-6|single family| +24624|AAAAAAAAADAGAAAA|568|11th North|Court|Suite P|Unionville|Newton County|GA|31711|United States|-5|condo| +24625|AAAAAAAABDAGAAAA|434|Cedar Oak|Cir.|Suite 340|Dallas|Crawford County|KS|63628|United States|-6|apartment| +24626|AAAAAAAACDAGAAAA|806|Poplar |Blvd|Suite 320|Stringtown|Jasper County|MO|60162|United States|-6|apartment| +24627|AAAAAAAADDAGAAAA|595||Dr.|Suite G|Unionville|Bacon County|GA|||-5|| +24628|AAAAAAAAEDAGAAAA|96|Lincoln |Way|Suite E|Yorktown|Toombs County|GA|30732|United States|-5|single family| +24629|AAAAAAAAFDAGAAAA|3|11th Miller|Road|Suite H|Oakdale|Jefferson County|PA|19584|United States|-5|apartment| +24630|AAAAAAAAGDAGAAAA|910|6th |Circle|Suite V|Marion|Washington County|VA|20399|United States|-5|single family| +24631|AAAAAAAAHDAGAAAA|157|4th |RD|Suite 250|Edgewood|Douglas County|GA|30069|United States|-5|apartment| +24632|AAAAAAAAIDAGAAAA|750|Miller |Pkwy|Suite 240|Salem|District of Columbia|DC|28048|United States|-5|single family| +24633|AAAAAAAAJDAGAAAA|672|Hill Sunset|Circle|Suite 260|Arlington|Butler County|OH|46557|United States|-5|condo| +24634|AAAAAAAAKDAGAAAA|244|Park |Court|Suite P|Riverdale|Accomack County|VA|29391|United States|-5|apartment| +24635|AAAAAAAALDAGAAAA|678|Eigth |Boulevard|Suite 110|Sunnyside|Wilkes County|GA|31952|United States|-5|single family| +24636|AAAAAAAAMDAGAAAA|791|Meadow 13th|Circle|Suite M|Midway|Jo Daviess County|IL|61904|United States|-6|single family| +24637|AAAAAAAANDAGAAAA|213|Park |ST|Suite M|Pleasant Grove|Park County|MT|64136|United States|-7|apartment| +24638|AAAAAAAAODAGAAAA|290|Pine 6th|Way|Suite F|Green Acres|Alameda County|CA|97683|United States|-8|condo| +24639|AAAAAAAAPDAGAAAA|617|3rd |Court|Suite X|Mountain View|Colbert County|AL|34466|United States|-6|condo| +24640|AAAAAAAAAEAGAAAA|465|Cherry |Court|Suite F|Five Forks|Irion County|TX|72293|United States|-6|condo| +24641|AAAAAAAABEAGAAAA|633|3rd Second|Blvd|Suite Y|Walnut Grove|Grand Isle County|VT|08352|United States|-5|condo| +24642|AAAAAAAACEAGAAAA|401|Birch |ST|Suite L|Mount Olive|Mineral County|WV|28059|United States|-5|single family| +24643|AAAAAAAADEAGAAAA|361|2nd |Drive|Suite 210|Shelby|Choctaw County|AL|36575|United States|-6|condo| +24644|AAAAAAAAEEAGAAAA|997|Oak |Drive|Suite S|Lakeside|Wirt County|WV|29532|United States|-5|single family| +24645|AAAAAAAAFEAGAAAA|555|Fifteenth |Street|Suite 100|Maywood|Wayne County|NY|15681|United States|-5|apartment| +24646|AAAAAAAAGEAGAAAA|526|6th 4th|Road|Suite 90|Mount Pleasant|Sanilac County|MI|41933|United States|-5|apartment| +24647|AAAAAAAAHEAGAAAA|112|5th |Avenue|Suite M|Plainview|Howard County|MO|63683|United States|-6|single family| +24648|AAAAAAAAIEAGAAAA|937|South |Cir.|Suite D|Belleville|Steuben County|IN|42924|United States|-5|apartment| +24649|AAAAAAAAJEAGAAAA|966|Lee 1st|Drive|Suite 0|Wilson|Hinsdale County|CO|86971|United States|-7|apartment| +24650|AAAAAAAAKEAGAAAA|638|First 2nd|Ln|Suite T|Lakeview|Coleman County|TX|78579|United States|-6|apartment| +24651|AAAAAAAALEAGAAAA|746|Jackson |Parkway|Suite W|Newport|Sharkey County|MS|51521|United States|-6|single family| +24652|AAAAAAAAMEAGAAAA|278|1st Ash|Circle|Suite B|Plainview|Robertson County|TN|33683|United States|-6|apartment| +24653|AAAAAAAANEAGAAAA|415|Oak |Pkwy|Suite 240|Shady Grove|Cochran County|TX|72812|United States|-6|single family| +24654|AAAAAAAAOEAGAAAA|945|Fifth Lake|ST|Suite Q|Harmony|Pratt County|KS|65804|United States|-6|single family| +24655|AAAAAAAAPEAGAAAA|59|Spruce |Parkway|Suite 40|Riverdale|Henry County|TN|39391|United States|-5|apartment| +24656|AAAAAAAAAFAGAAAA|36|Wilson Jefferson|Pkwy|Suite Q|Woodville|Baltimore city|MD|24289|United States|-5|condo| +24657|AAAAAAAABFAGAAAA|506|Main |ST|Suite 310|Crestview|Hodgeman County|KS|61421|United States|-6|condo| +24658|AAAAAAAACFAGAAAA|435||||Oak Ridge|Jefferson County|KS||United States||| +24659|AAAAAAAADFAGAAAA|636|13th |Street|Suite 70|Forest Hills|Wyoming County|WV|29237|United States|-5|single family| +24660|AAAAAAAAEFAGAAAA|606|Wilson |RD|Suite 160|Fairfield|Marion County|SC|26192|United States|-5|single family| +24661|AAAAAAAAFFAGAAAA|968|Lake Locust|Parkway|Suite 180|Spring Hill|Wilkes County|NC|26787|United States|-5|condo| +24662|AAAAAAAAGFAGAAAA|661|Walnut |Ct.|Suite F|Crossroads|Orangeburg County|SC|20534|United States|-5|single family| +24663|AAAAAAAAHFAGAAAA|712|Church Washington|Avenue|Suite Y|Union|Winston County|AL|38721|United States|-6|single family| +24664|AAAAAAAAIFAGAAAA|826|3rd 6th|Circle|Suite O|Lakeview|Ziebach County|SD|58579|United States|-6|apartment| +24665|AAAAAAAAJFAGAAAA|579|Center |Court|Suite H|Oak Grove|Shelby County|IN|48370|United States|-5|single family| +24666|AAAAAAAAKFAGAAAA|246|3rd Third|Court|Suite 180|Oakdale|Crawford County|PA|19584|United States|-5|apartment| +24667|AAAAAAAALFAGAAAA|804|Ash |Court|Suite X|Pine Grove|Wahkiakum County|WA|94593|United States|-8|single family| +24668|AAAAAAAAMFAGAAAA|724|14th |Avenue|Suite 150|Stringtown|Crook County|OR|90162|United States|-8|condo| +24669|AAAAAAAANFAGAAAA|839|Sixth |Road|Suite 100|Mount Pleasant|Wayne County|MO|61933|United States|-6|condo| +24670|AAAAAAAAOFAGAAAA||Oak ||Suite M|Woodlawn||||||| +24671|AAAAAAAAPFAGAAAA|905|Hickory |Avenue|Suite 220|Red Hill|Menard County|TX|74338|United States|-6|condo| +24672|AAAAAAAAAGAGAAAA|98|Hillcrest |Cir.|Suite E|Westminster|Iberia Parish|LA|76549|United States|-6|single family| +24673|AAAAAAAABGAGAAAA|491|Franklin |Ave|Suite 360|Stringtown|Alleghany County|NC|20162|United States|-5|single family| +24674|AAAAAAAACGAGAAAA|725|View Lakeview|Avenue|Suite D|Hazelwood|Box Butte County|NE|61206|United States|-6|condo| +24675|AAAAAAAADGAGAAAA|414|Dogwood |Lane|Suite Y|Lakeview|Casey County|KY|48579|United States|-6|single family| +24676|AAAAAAAAEGAGAAAA|410|4th |Avenue|Suite T|Bunker Hill|Harford County|MD|20150|United States|-5|single family| +24677|AAAAAAAAFGAGAAAA|363|Central |Avenue|Suite J|Marion|Baca County|CO|80399|United States|-7|condo| +24678|AAAAAAAAGGAGAAAA|994|Hickory |ST|Suite L|Riverside|Nicholas County|WV|29231|United States|-5|apartment| +24679|AAAAAAAAHGAGAAAA|832|Fourth 6th|Ave|Suite A|Five Points|Lamoille County|VT|06698|United States|-5|single family| +24680|AAAAAAAAIGAGAAAA|543|Johnson |Pkwy|Suite B|New Hope|Lawrence County|MO|69431|United States|-6|condo| +24681|AAAAAAAAJGAGAAAA|947|Main |Blvd|Suite M|Highland Park|Madison County|TX|76534|United States|-6|condo| +24682|AAAAAAAAKGAGAAAA|807|First |Boulevard|Suite Y|Mountain View|Brewster County|TX|74466|United States|-6|single family| +24683|AAAAAAAALGAGAAAA|671|||Suite Q||Blount County|||||| +24684|AAAAAAAAMGAGAAAA|783|Forest Chestnut|Blvd|Suite Y|Kingston|Kiowa County|KS|64975|United States|-6|single family| +24685|AAAAAAAANGAGAAAA|119|College |Circle|Suite 290|Five Points|Greer County|OK|76098|United States|-6|apartment| +24686|AAAAAAAAOGAGAAAA|222|Second |Pkwy|Suite 350|Green Acres|Hancock County|IN|47683|United States|-5|apartment| +24687|AAAAAAAAPGAGAAAA|900|Fifth South|Pkwy|Suite B|Jamestown|Llano County|TX|76867|United States|-6|condo| +24688|AAAAAAAAAHAGAAAA|703|Spring 4th|Dr.|Suite Y|Providence|Oliver County|ND|56614|United States|-6|condo| +24689|AAAAAAAABHAGAAAA|795|Lee Spring|Pkwy|Suite H|Oak Grove|Catron County|NM|88370|United States|-7|single family| +24690|AAAAAAAACHAGAAAA|715|West North|Parkway|Suite N|Bunker Hill|Attala County|MS|50150|United States|-6|condo| +24691|AAAAAAAADHAGAAAA|383|Spruce Third|Way|Suite 240|Marion|Clarke County|IA|50399|United States|-6|apartment| +24692|AAAAAAAAEHAGAAAA|960|Willow Hickory|Ln|Suite 190|Jamestown|Lauderdale County|AL|36867|United States|-6|condo| +24693|AAAAAAAAFHAGAAAA|381|Madison |Dr.|Suite 240|Arlington|Sharkey County|MS|56557|United States|-6|single family| +24694|AAAAAAAAGHAGAAAA||Maple |||||||United States||apartment| +24695|AAAAAAAAHHAGAAAA|657|12th |RD|Suite H|Sunnyside|Scott County|IA|51952|United States|-6|single family| +24696|AAAAAAAAIHAGAAAA|836|Ridge Laurel|Wy|Suite 220|Springfield|Saluda County|SC|29303|United States|-5|apartment| +24697|AAAAAAAAJHAGAAAA|203|7th |Road|Suite M|Jackson|Red River Parish|LA|79583|United States|-6|condo| +24698|AAAAAAAAKHAGAAAA|392|Madison Walnut|Boulevard|Suite 60|Oak Ridge|Carter County|OK|78371|United States|-6|single family| +24699|AAAAAAAALHAGAAAA|104|Davis |Dr.|Suite P|Midway|Bergen County|NJ|02504|United States|-5|apartment| +24700|AAAAAAAAMHAGAAAA|445|Railroad Cedar|Ln|Suite U|Unionville|McHenry County|IL|61711|United States|-6|condo| +24701|AAAAAAAANHAGAAAA|215|Franklin North|Parkway|Suite O|Jamestown|Leon County|FL|36867|United States|-5|condo| +24702|AAAAAAAAOHAGAAAA|397|12th Adams|Court|Suite 420|Cedar Grove|Lowndes County|AL|30411|United States|-6|condo| +24703|AAAAAAAAPHAGAAAA|33|West |Blvd|Suite 210|Woodlawn|Lee County|SC|24098|United States|-5|apartment| +24704|AAAAAAAAAIAGAAAA|817|Main 7th|Road|Suite T|Oakland|Amador County|CA|99843|United States|-8|condo| +24705|AAAAAAAABIAGAAAA|898|Fifth Park|Blvd|Suite L|Shady Grove|Apache County|AZ|82812|United States|-7|single family| +24706|AAAAAAAACIAGAAAA|530|Walnut |Ct.|Suite S|Clifton|Madison County|MS|58014|United States|-6|apartment| +24707|AAAAAAAADIAGAAAA|949|Second |Street|Suite 230|Pine Grove|Amherst County|VA|24593|United States|-5|apartment| +24708|AAAAAAAAEIAGAAAA|534|10th Davis|Street|Suite 140|Union Hill|Jackson County|OR|97746|United States|-8|apartment| +24709|AAAAAAAAFIAGAAAA|587|Fifth College|Road|Suite 90|Georgetown|Grundy County|IL|67057|United States|-6|single family| +24710|AAAAAAAAGIAGAAAA|527|North Oak|ST|Suite 160|Glenwood|Dawson County|MT|63511|United States|-7|single family| +24711|AAAAAAAAHIAGAAAA|214|Ridge |Blvd|Suite 50|Hillcrest|Wood County|WI|53003|United States|-6|condo| +24712|AAAAAAAAIIAGAAAA|957|1st |Street|Suite C|Glenwood|Culberson County|TX|73511|United States|-6|apartment| +24713|AAAAAAAAJIAGAAAA|149|Church First|Dr.|Suite D|Fairview|Breckinridge County|KY|45709|United States|-6|condo| +24714|AAAAAAAAKIAGAAAA|26|South |Lane|Suite S|Edgewood|Jackson County|TX|70069|United States|-6|single family| +24715|AAAAAAAALIAGAAAA|887|Lincoln 3rd|Dr.|Suite 180|Amherst|Isle of Wight County|VA|28119|United States|-5|single family| +24716|AAAAAAAAMIAGAAAA|132|Park Miller|RD|Suite A|Granite|Chautauqua County|NY|16284|United States|-5|single family| +24717|AAAAAAAANIAGAAAA|803|Spring 2nd|Ct.|Suite 90|Kingston|Grundy County|IL|64975|United States|-6|apartment| +24718|AAAAAAAAOIAGAAAA|659|View |Ave|Suite 420|Florence|Washington County|UT|83394|United States|-7|apartment| +24719|AAAAAAAAPIAGAAAA|745|Madison |Circle|Suite H|Woodland|Putnam County|WV|24854|United States|-5|condo| +24720|AAAAAAAAAJAGAAAA|478|First |Blvd|Suite 340|Marion|Bibb County|AL|30399|United States|-6|single family| +24721|AAAAAAAABJAGAAAA|915|Franklin Wilson|Street|Suite 450|Woodland Park|Iberville Parish|LA|71934|United States|-6|apartment| +24722|AAAAAAAACJAGAAAA|783|Hill |Street|Suite 60|Riverdale|Rockingham County|NC|29391|United States|-5|apartment| +24723|AAAAAAAADJAGAAAA|853|Lincoln |Circle|Suite 390|Sunnyside|Crane County|TX|71952|United States|-6|condo| +24724|AAAAAAAAEJAGAAAA|207|1st Green|Dr.|Suite F|Unionville|Anderson County|TX|71711|United States|-6|condo| +24725|AAAAAAAAFJAGAAAA|720|8th Maple||Suite 450|Wildwood||SD||||single family| +24726|AAAAAAAAGJAGAAAA|685|Hill |Drive|Suite H|Kingston|Schenectady County|NY|14975|United States|-5|apartment| +24727|AAAAAAAAHJAGAAAA|602|Fifth |Drive|Suite 450|Deerfield|Hunterdon County|NJ|09840|United States|-5|single family| +24728|AAAAAAAAIJAGAAAA|451|11th College|Ct.|Suite 240|Perkins|Morrison County|MN|51852|United States|-6|apartment| +24729|AAAAAAAAJJAGAAAA|750|Park |Boulevard|Suite 380|Plainview|Mercer County|NJ|04283|United States|-5|apartment| +24730|AAAAAAAAKJAGAAAA|974|Walnut |Ct.|Suite K|Walnut Grove|Clinton County|KY|47752|United States|-6|single family| +24731|AAAAAAAALJAGAAAA|572|Jackson Oak|Cir.|Suite 30|Mount Vernon|Union County|FL|38482|United States|-5|single family| +24732|AAAAAAAAMJAGAAAA|638|Sixth |Street|Suite 20|Kingston|Harlan County|KY|44975|United States|-6|condo| +24733|AAAAAAAANJAGAAAA|382|River |Ave|Suite 30|Bath|Middlesex County|CT|01173|United States|-5|single family| +24734|AAAAAAAAOJAGAAAA|869|Franklin Ridge|Parkway|Suite T|Lebanon|Keya Paha County|NE|62898|United States|-7|condo| +24735|AAAAAAAAPJAGAAAA|377|Madison Cherry|Road|Suite L|Macedonia|Payne County|OK|71087|United States|-6|single family| +24736|AAAAAAAAAKAGAAAA|105|Walnut View|Way|Suite N|Five Points|La Paz County|AZ|86098|United States|-7|condo| +24737|AAAAAAAABKAGAAAA|946|9th |RD|Suite X|Edgewood|Jewell County|KS|60069|United States|-6|condo| +24738|AAAAAAAACKAGAAAA|102|North Lincoln|Pkwy|Suite E|Enterprise|Custer County|MT|61757|United States|-7|apartment| +24739|AAAAAAAADKAGAAAA|486|Mill |Boulevard|Suite R|Ridgewood|Middlesex County|NJ|07347|United States|-5|apartment| +24740|AAAAAAAAEKAGAAAA|768|Eigth |Lane|Suite E|Five Points|Colonial Heights city|VA|26098|United States|-5|apartment| +24741|AAAAAAAAFKAGAAAA|812|Lake Oak|Court|Suite 300|Sulphur Springs|Union County|NM|88354|United States|-7|single family| +24742|AAAAAAAAGKAGAAAA|846|Walnut Central|RD|Suite 220|Lebanon|Eaton County|MI|42898|United States|-5|single family| +24743|AAAAAAAAHKAGAAAA|114|3rd |Street|Suite G|Mount Pleasant|Barron County|WI|51933|United States|-6|condo| +24744|AAAAAAAAIKAGAAAA|250|6th |Drive|Suite H|Woodland|Washington County|NC|24854|United States|-5|apartment| +24745|AAAAAAAAJKAGAAAA|5|Spruce |Street|Suite Y|Farmington|Bullock County|AL|39145|United States|-6|apartment| +24746|AAAAAAAAKKAGAAAA|450|Smith Ridge|Way|Suite O|Newport|Caldwell County|NC|21521|United States|-5|condo| +24747|AAAAAAAALKAGAAAA|683|Broadway |Blvd|Suite 30|Enterprise|Pepin County|WI|51757|United States|-6|condo| +24748|AAAAAAAAMKAGAAAA|531|11th Walnut|Cir.|Suite H|Greenwood|Wayne County|MO|68828|United States|-6|apartment| +24749|AAAAAAAANKAGAAAA|860|East 7th|Drive|Suite R|Walnut Grove|Deuel County|NE|67752|United States|-6|single family| +24750|AAAAAAAAOKAGAAAA|12|Railroad |Ave|Suite 310|Union|Gray County|KS|68721|United States|-6|apartment| +24751|AAAAAAAAPKAGAAAA|773|Wilson |Drive|Suite T|Ferndale|Richland County|OH|42731|United States|-5|condo| +24752|AAAAAAAAALAGAAAA|203|Third Mill|Circle|Suite B|Crossroads|Gallia County|OH|40534|United States|-5|apartment| +24753|AAAAAAAABLAGAAAA|552|River |Ct.|Suite 250|Clifton|McMinn County|TN|38014|United States|-6|single family| +24754|AAAAAAAACLAGAAAA|741|Park Jefferson|ST|Suite 270|Oak Ridge|McIntosh County|GA|38371|United States|-5|single family| +24755|AAAAAAAADLAGAAAA|573|Smith |Court|Suite W|Jamestown|Perry County|KY|46867|United States|-5|apartment| +24756|AAAAAAAAELAGAAAA|596|1st |Blvd|Suite 380|Summit|Letcher County|KY|40499|United States|-5|single family| +24757|AAAAAAAAFLAGAAAA|275|Broadway |ST|Suite E|Antioch|Del Norte County|CA|98605|United States|-8|apartment| +24758|AAAAAAAAGLAGAAAA|199|Third |Parkway|Suite L|Crossroads|Macon County|MO|60534|United States|-6|apartment| +24759|AAAAAAAAHLAGAAAA|920|Main |Lane|Suite F|Valley View|Maury County|TN|35124|United States|-6|condo| +24760|AAAAAAAAILAGAAAA|610|Wilson 4th|Ave|Suite I|Indian Village|Washington County|CO|81075|United States|-7|single family| +24761|AAAAAAAAJLAGAAAA|2|Church |Cir.|Suite 120|Lone Pine|Mitchell County|TX|77441|United States|-6|condo| +24762|AAAAAAAAKLAGAAAA|98|Sunset |Court|Suite 400|Unionville|Scurry County|TX|71711|United States|-6|condo| +24763|AAAAAAAALLAGAAAA|141|Sixth |Lane|Suite C|Five Forks|Franklin County|MS|52293|United States|-6|condo| +24764|AAAAAAAAMLAGAAAA|121|College 1st|Ave|Suite 420|Mount Zion|Coffee County|GA|38054|United States|-5|condo| +24765|AAAAAAAANLAGAAAA|32|2nd Spruce|Drive|Suite L|Spring Hill|Oxford County|ME|07387|United States|-5|apartment| +24766|AAAAAAAAOLAGAAAA|890|Sunset Valley|ST|Suite 110|Lenox|Horry County|SC|21143|United States|-5|condo| +24767|AAAAAAAAPLAGAAAA|591|Main |Parkway|Suite O|Edgewater|Sarpy County|NE|60635|United States|-7|single family| +24768|AAAAAAAAAMAGAAAA|475|Valley ||Suite 280|Gilmore|Caledonia County||06064|||condo| +24769|AAAAAAAABMAGAAAA|896|Fourteenth |Road|Suite 380|Valley View|Athens County|OH|45124|United States|-5|apartment| +24770|AAAAAAAACMAGAAAA|757|5th Lincoln|Wy|Suite N|Clinton|Vernon County|MO|68222|United States|-6|apartment| +24771|AAAAAAAADMAGAAAA|773|Fourth |Ct.|Suite L|New Hope|Esmeralda County|NV|89431|United States|-8|single family| +24772|AAAAAAAAEMAGAAAA|8|Lake |Dr.|Suite 30|Lebanon|Upson County|GA|32898|United States|-5|condo| +24773|AAAAAAAAFMAGAAAA|397|Madison |Ln|Suite P|Hopewell|Cattaraugus County|NY|10587|United States|-5|apartment| +24774|AAAAAAAAGMAGAAAA|677|5th |Dr.|Suite B|Georgetown|Phillips County|CO|87057|United States|-7|apartment| +24775|AAAAAAAAHMAGAAAA|946|Wilson |Boulevard|Suite C|Maple Grove|Stone County|AR|78252|United States|-6|condo| +24776|AAAAAAAAIMAGAAAA|749|Church |Road|Suite 330|Sumner|Edgar County|IL|60519|United States|-6|single family| +24777|AAAAAAAAJMAGAAAA|531|6th |Street|Suite V|Salem|Real County|TX|78048|United States|-6|apartment| +24778|AAAAAAAAKMAGAAAA|294|Adams Sunset|Parkway|Suite 460|Hopewell|Mecklenburg County|NC|20587|United States|-5|condo| +24779|AAAAAAAALMAGAAAA|743|Park 5th|Ave|Suite G|Fairview|Madison County|VA|25709|United States|-5|apartment| +24780|AAAAAAAAMMAGAAAA|342|Jackson |Blvd|Suite E|Oakwood|Converse County|WY|80169|United States|-7|single family| +24781|AAAAAAAANMAGAAAA|639|Ninth First|Ct.|Suite J|Greenfield|Hamlin County|SD|55038|United States|-7|single family| +24782|AAAAAAAAOMAGAAAA|327|Green |ST|Suite 70|Springfield|Essex County|VT|09903|United States|-5|apartment| +24783|AAAAAAAAPMAGAAAA|760|North |ST|Suite 190|Greenfield|Coryell County|TX|75038|United States|-6|apartment| +24784|AAAAAAAAANAGAAAA|38|First 9th|Boulevard|Suite C|Jamestown|Clarke County|AL|36867|United States|-6|condo| +24785|AAAAAAAABNAGAAAA|||||Mount Olive|||98059|||| +24786|AAAAAAAACNAGAAAA|780|Elm Lakeview|Drive|Suite W|Centerville|Johnson County|KY|40059|United States|-6|condo| +24787|AAAAAAAADNAGAAAA|855|Willow |Lane|Suite 420|Lebanon|Cumberland County|KY|42898|United States|-6|apartment| +24788|AAAAAAAAENAGAAAA|||Ct.|||||78605||-6|| +24789|AAAAAAAAFNAGAAAA|799|8th Hillcrest|Lane|Suite 290|Avery|Houston County|TX|70194|United States|-6|apartment| +24790|AAAAAAAAGNAGAAAA|398|Park River|Cir.|Suite B|Union Hill|Woodford County|KY|47746|United States|-6|single family| +24791|AAAAAAAAHNAGAAAA|308|Washington 4th|Road|Suite 420|Oakland|Neshoba County|MS|59843|United States|-6|apartment| +24792|AAAAAAAAINAGAAAA|990|7th |Circle|Suite 460|Monroe|Bradford County|PA|13984|United States|-5|apartment| +24793|AAAAAAAAJNAGAAAA|279|East |Court|Suite 400|Pleasant Grove|Greene County|MS|54136|United States|-6|single family| +24794|AAAAAAAAKNAGAAAA|119|Fifth Lake|Blvd|Suite 470|Glendale|San Francisco County|CA|93951|United States|-8|condo| +24795|AAAAAAAALNAGAAAA|278|Center |ST|Suite H|Hillcrest|Grant County|MN|53003|United States|-6|single family| +24796|AAAAAAAAMNAGAAAA|896|Washington |Court|Suite 70|Oakland|Teton County|MT|69843|United States|-7|single family| +24797|AAAAAAAANNAGAAAA|318|Spring |Court|Suite 200|Red Hill|Anchorage Borough|AK|94338|United States|-9|apartment| +24798|AAAAAAAAONAGAAAA|928|Twelfth Oak|Ave|Suite N|Lakeview|Bath County|VA|28579|United States|-5|condo| +24799|AAAAAAAAPNAGAAAA|954|Highland |Boulevard|Suite Q|Farmington|Cowley County|KS|69145|United States|-6|apartment| +24800|AAAAAAAAAOAGAAAA|752|Elm Chestnut|Boulevard|Suite E|Leesville|Dunn County|ND|55423|United States|-6|apartment| +24801|AAAAAAAABOAGAAAA|892|Adams |Court|Suite M|Springfield|White County|IN|49303|United States|-5|single family| +24802|AAAAAAAACOAGAAAA|308|10th |Lane|Suite L|Newtown|Cape Girardeau County|MO|61749|United States|-6|apartment| +24803|AAAAAAAADOAGAAAA|17|South Spruce|Avenue|Suite 140|Macedonia|Rutherford County|NC|21087|United States|-5|single family| +24804|AAAAAAAAEOAGAAAA|40|Davis |Way|Suite H|Greenfield|Howell County|MO|65038|United States|-6|apartment| +24805|AAAAAAAAFOAGAAAA|611|Mill |Court|Suite 60|Florence|Camp County|TX|73394|United States|-6|apartment| +24806|AAAAAAAAGOAGAAAA|65|5th Highland|Parkway|Suite 320|Woodville|Fountain County|IN|44289|United States|-5|condo| +24807|AAAAAAAAHOAGAAAA|475|Fifth Walnut|Ln|Suite H|Nottingham|Franklin County|FL|34074|United States|-5|single family| +24808|AAAAAAAAIOAGAAAA|498|8th |Dr.|Suite G|Siloam|Ionia County|MI|48948|United States|-5|condo| +24809|AAAAAAAAJOAGAAAA|549|Jackson Ash|Circle|Suite 380|Lakewood|Steele County|ND|58877|United States|-6|condo| +24810|AAAAAAAAKOAGAAAA|463|7th |Ave|Suite E|Harvey|Clarke County|IA|55858|United States|-6|apartment| +24811|AAAAAAAALOAGAAAA|375||||Cedar Grove|Atascosa County|TX||United States|-6|single family| +24812|AAAAAAAAMOAGAAAA|644|Lincoln |Wy|Suite 270|Woodland|Dickinson County|KS|64854|United States|-6|apartment| +24813|AAAAAAAANOAGAAAA|101|Lincoln Railroad|Wy|Suite U|Crystal|Lauderdale County|MS|55258|United States|-6|apartment| +24814|AAAAAAAAOOAGAAAA|232|Railroad |Parkway|Suite I|Jackson|Jerauld County|SD|59583|United States|-7|condo| +24815|AAAAAAAAPOAGAAAA|376|Spring |Boulevard|Suite P|Antioch|Woodbury County|IA|58605|United States|-6|condo| +24816|AAAAAAAAAPAGAAAA|134|Pine Mill|Way|Suite T|Marion|Winkler County|TX|70399|United States|-6|apartment| +24817|AAAAAAAABPAGAAAA|99|View |Ave|Suite 0|Arlington|White County|TN|36557|United States|-6|condo| +24818|AAAAAAAACPAGAAAA|610|Oak |Avenue|Suite 440|Georgetown|Stone County|MO|67057|United States|-6|single family| +24819|AAAAAAAADPAGAAAA|425|Forest |Way|Suite A|Valley View|Charlton County|GA|35124|United States|-5|condo| +24820|AAAAAAAAEPAGAAAA|317|4th Williams|Road|Suite 140|Plainview|Bay County|MI|43683|United States|-5|condo| +24821|AAAAAAAAFPAGAAAA|102|Elm Ash|Circle|Suite 490|Waterloo|Edgecombe County|NC|21675|United States|-5|single family| +24822|AAAAAAAAGPAGAAAA|344|7th Mill|Parkway|Suite 370|Centerville|Union County|IA|50059|United States|-6|apartment| +24823|AAAAAAAAHPAGAAAA|737|Adams Valley|RD|Suite 310|Springfield|Richland County|IL|69303|United States|-6|apartment| +24824|AAAAAAAAIPAGAAAA|586|Ridge Park|Ave|Suite 30|Oak Hill|Marshall County|MS|57838|United States|-6|apartment| +24825|AAAAAAAAJPAGAAAA|394|Meadow |Ln|Suite J|Mount Vernon|Maries County|MO|68482|United States|-6|single family| +24826|AAAAAAAAKPAGAAAA|57|Spring View|Boulevard|Suite V|Lee|Jones County|NC|20408|United States|-5|single family| +24827|AAAAAAAALPAGAAAA|859|Mill |Cir.|Suite 90|Macedonia|Stanton County|NE|61087|United States|-7|single family| +24828|AAAAAAAAMPAGAAAA|965|10th Hillcrest|Cir.|Suite W|Glendale|Hardeman County|TN|33951|United States|-5|single family| +24829|AAAAAAAANPAGAAAA|719|Hillcrest Valley|Cir.|Suite 130|Glenwood|Sagadahoc County|ME|04111|United States|-5|apartment| +24830|AAAAAAAAOPAGAAAA|978|9th Birch|Ave|Suite 390|Red Hill|Leon County|TX|74338|United States|-6|condo| +24831|AAAAAAAAPPAGAAAA|422|Park |Ave|Suite 50|Pine Grove|Calhoun County|AR|74593|United States|-6|single family| +24832|AAAAAAAAAABGAAAA|488|Willow Tenth|Way|Suite V|Washington Heights|Chesterfield County|SC|28167|United States|-5|single family| +24833|AAAAAAAABABGAAAA|382|12th Hill|Dr.|Suite X|Forest Hills|Bergen County|NJ|09837|United States|-5|condo| +24834|AAAAAAAACABGAAAA|252|10th Highland|Dr.|Suite 60|Sulphur Springs|Boyle County|KY|48354|United States|-6|single family| +24835|AAAAAAAADABGAAAA|992|Spruce Poplar|RD|Suite 350|Spring Valley|Jefferson County|OH|46060|United States|-5|single family| +24836|AAAAAAAAEABGAAAA|499|10th |Street|Suite 190|Walnut Grove|Norton County|KS|67752|United States|-6|condo| +24837|AAAAAAAAFABGAAAA|549|5th |Boulevard|Suite N|Texas|Richmond County|NY|13342|United States|-5|condo| +24838|AAAAAAAAGABGAAAA|865|First Broadway|Boulevard|Suite 310|Brownsville|Wayne County|MS|59310|United States|-6|apartment| +24839|AAAAAAAAHABGAAAA|305|South |Court|Suite 370|Lakeside|Bartow County|GA|39532|United States|-5|condo| +24840|AAAAAAAAIABGAAAA|950|College Wilson|Wy|Suite U|Lincoln|Wicomico County|MD|21289|United States|-5|condo| +24841|AAAAAAAAJABGAAAA|888|Lake |Wy|Suite 430|Union|DeBaca County|NM|88721|United States|-7|single family| +24842|AAAAAAAAKABGAAAA|682|West Laurel|Lane|Suite 420|Springdale|Lafayette County|AR|78883|United States|-6|single family| +24843|AAAAAAAALABGAAAA|857|Lake Chestnut|Boulevard|Suite T|Jamestown|Lawrence County|MS|56867|United States|-6|condo| +24844|AAAAAAAAMABGAAAA|671|Hill Lakeview|Wy|Suite 460|Union|Jefferson County|OH|48721|United States|-5|single family| +24845|AAAAAAAANABGAAAA|519|Lincoln Ash|Cir.|Suite 370|Mount Pleasant|Jones County|NC|21933|United States|-5|condo| +24846|AAAAAAAAOABGAAAA|233|2nd Willow|Ln|Suite 80|New Hope|Jefferson County|OK|79431|United States|-6|single family| +24847|AAAAAAAAPABGAAAA|387|Dogwood |ST|Suite 170|Lakeside|Wheeler County|TX|79532|United States|-6|condo| +24848|AAAAAAAAABBGAAAA|86|10th Johnson|ST|Suite L|Nottingham|Charles Mix County|SD|54074|United States|-6|apartment| +24849|AAAAAAAABBBGAAAA|327|Center 3rd|ST|Suite L|Bunker Hill|Edgecombe County|NC|20150|United States|-5|single family| +24850|AAAAAAAACBBGAAAA|319|Laurel |RD|Suite 150|Franklin|Wagoner County|OK|79101|United States|-6|condo| +24851|AAAAAAAADBBGAAAA|892|7th |Cir.|Suite 290|Concord|Lamar County|TX|74107|United States|-6|condo| +24852|AAAAAAAAEBBGAAAA|761|Sycamore Franklin|ST|Suite Q|Sunnyside|Clarke County|GA|31952|United States|-5|single family| +24853|AAAAAAAAFBBGAAAA|600|3rd |Boulevard|Suite W|Buena Vista|Cook County|MN|55752|United States|-6|single family| +24854|AAAAAAAAGBBGAAAA|148|Center Birch|Ave|Suite S|Clinton|Cerro Gordo County|IA|58222|United States|-6|single family| +24855|AAAAAAAAHBBGAAAA|474|Franklin |Ln|Suite K|Stringtown|Cedar County|MO|60162|United States|-6|condo| +24856|AAAAAAAAIBBGAAAA|892|Park |Blvd|Suite 180|Newtown|Grafton County|NH|02349|United States|-5|single family| +24857|AAAAAAAAJBBGAAAA|492|Pine Meadow|Blvd|Suite 260|Antioch|Knox County|TX|78605|United States|-6|apartment| +24858|AAAAAAAAKBBGAAAA|80|First Pine|Wy|Suite 440|Enterprise|Bosque County|TX|71757|United States|-6|condo| +24859|AAAAAAAALBBGAAAA|934|Pine Lakeview|Court|Suite 480|Mount Zion|Cottle County|TX|78054|United States|-6|apartment| +24860|AAAAAAAAMBBGAAAA|504|Hickory South|Ln|Suite 360|Jamestown|Beltrami County|MN|56867|United States|-6|single family| +24861|AAAAAAAANBBGAAAA|592|Jackson Main|Ave|Suite G|Concord|Carroll County|NH|04707|United States|-5|single family| +24862|AAAAAAAAOBBGAAAA|||Ave|Suite N||Lafayette County|MO|68252||-6|| +24863|AAAAAAAAPBBGAAAA|492|Hickory |Ln|Suite R|Antioch|Alachua County|FL|38605|United States|-5|apartment| +24864|AAAAAAAAACBGAAAA|373|West |Street|Suite U|Georgetown|Livingston County|IL|67057|United States|-6|single family| +24865|AAAAAAAABCBGAAAA|781|North |Blvd|Suite A|Enterprise|Wayne County|MO|61757|United States|-6|apartment| +24866|AAAAAAAACCBGAAAA|849|4th |Ave|Suite O|Jamestown|Norman County|MN|56867|United States|-6|single family| +24867|AAAAAAAADCBGAAAA|504|Elm |Dr.|Suite C|Glenwood|Lawrence County|PA|13511|United States|-5|condo| +24868|AAAAAAAAECBGAAAA|105|Hillcrest |Pkwy|Suite 300|Plainville|Jerome County|ID|86115|United States|-7|condo| +24869|AAAAAAAAFCBGAAAA|738|15th Spring|Street|Suite D|Clifton|Glynn County|GA|38014|United States|-5|apartment| +24870|AAAAAAAAGCBGAAAA|642|2nd Franklin|Drive|Suite 380|Lakeside|Lyon County|NV|89532|United States|-8|condo| +24871|AAAAAAAAHCBGAAAA|364|Sixth Green|Boulevard|Suite 320|Riceville|Latah County|ID|85867|United States|-7|apartment| +24872|AAAAAAAAICBGAAAA|774|Mill Hillcrest|Ct.|Suite M|Georgetown|Oceana County|MI|47057|United States|-5|single family| +24873|AAAAAAAAJCBGAAAA|232|15th Woodland|Boulevard|Suite 430|Ashland|Bibb County|AL|34244|United States|-6|condo| +24874|AAAAAAAAKCBGAAAA|894|Oak Cherry|Lane|Suite 70|Stringtown|Monroe County|FL|30162|United States|-5|apartment| +24875|AAAAAAAALCBGAAAA|83|4th Hillcrest|Pkwy|Suite 270|Bridgeport|Madison County|NY|15817|United States|-5|condo| +24876|AAAAAAAAMCBGAAAA|323|Ninth Oak|Avenue|Suite S|Plainview|Fillmore County|NE|63683|United States|-6|apartment| +24877|AAAAAAAANCBGAAAA|921|Poplar Fifth|Pkwy|Suite 490|Woodland|Custer County|ID|84854|United States|-7|single family| +24878|AAAAAAAAOCBGAAAA|196|Hillcrest 7th|Blvd|Suite 240|Marion|Nassau County|NY|10399|United States|-5|single family| +24879|AAAAAAAAPCBGAAAA|81|Walnut |Avenue|Suite 340|Five Forks|Guadalupe County|NM|82293|United States|-7|single family| +24880|AAAAAAAAADBGAAAA|178|Sixth |Ave|Suite 470|Lakeview|Twin Falls County|ID|88579|United States|-7|single family| +24881|AAAAAAAABDBGAAAA|830|Church |Ave|Suite 140|Jackson|Otoe County|NE|69583|United States|-7|condo| +24882|AAAAAAAACDBGAAAA|||Wy||||GA|||-5|| +24883|AAAAAAAADDBGAAAA|907|Green |Circle|Suite H|Wilson|Adams County|NE|66971|United States|-6|apartment| +24884|AAAAAAAAEDBGAAAA|561|5th |Drive|Suite C|Oak Hill|Wilson County|KS|67838|United States|-6|condo| +24885|AAAAAAAAFDBGAAAA|750|Third Smith|Drive|Suite 200|Ashland|Elbert County|GA|34244|United States|-5|condo| +24886|AAAAAAAAGDBGAAAA|321|Oak Sixth|Dr.|Suite 340|Springtown|Chattahoochee County|GA|39858|United States|-5|single family| +24887|AAAAAAAAHDBGAAAA|868|Cherry Broadway|Ln|Suite 140|Marion|Grundy County|IA|50399|United States|-6|condo| +24888|AAAAAAAAIDBGAAAA|619|Adams |ST|Suite 290|Franklin|Guernsey County|OH|49101|United States|-5|apartment| +24889|AAAAAAAAJDBGAAAA|717|7th |Way|Suite 70|Oak Hill|Summit County|UT|87838|United States|-7|apartment| +24890|AAAAAAAAKDBGAAAA|89|Mill Park|Parkway|Suite U|Spring Valley|Monroe County|IA|56060|United States|-6|condo| +24891|AAAAAAAALDBGAAAA|926|Pine |Dr.|Suite 0|Salem|Fort Bend County|TX|78048|United States|-6|apartment| +24892|AAAAAAAAMDBGAAAA|940|9th |Ln|Suite 380|Greenville|Aleutians West Census Area|AK|91387|United States|-9|condo| +24893|AAAAAAAANDBGAAAA|70|Ridge |RD|Suite 60|Glendale|Treasure County|MT|63951|United States|-7|condo| +24894|AAAAAAAAODBGAAAA|868|Lee Pine|Dr.|Suite 320|Edgewood|Meriwether County|GA|30069|United States|-5|apartment| +24895|AAAAAAAAPDBGAAAA|702|1st |Blvd|Suite M|Flint|Darlington County|SC|28909|United States|-5|single family| +24896|AAAAAAAAAEBGAAAA|47|Jackson Jackson|Pkwy|Suite V|Lakewood|Yell County|AR|78877|United States|-6|single family| +24897|AAAAAAAABEBGAAAA|678|14th |Cir.|Suite M|Antioch|Gilchrist County|FL|38605|United States|-5|apartment| +24898|AAAAAAAACEBGAAAA|712|College |Blvd|Suite S|Acme|Leelanau County|MI|40164|United States|-5|condo| +24899|AAAAAAAADEBGAAAA|417|Green Third|Pkwy|Suite H|Plainview|Young County|TX|73683|United States|-6|single family| +24900|AAAAAAAAEEBGAAAA||Sunset Washington||Suite X|Valley View|||45124|||condo| +24901|AAAAAAAAFEBGAAAA|700|First |Road|Suite J|Cedar Grove|Staunton city|VA|20411|United States|-5|single family| +24902|AAAAAAAAGEBGAAAA|557|Cedar |Ln|Suite L|Waterloo|Stevens County|KS|61675|United States|-6|single family| +24903|AAAAAAAAHEBGAAAA|415|Central 2nd|Pkwy|Suite 190|Arlington|Tehama County|CA|96557|United States|-8|condo| +24904|AAAAAAAAIEBGAAAA|375|First |ST|Suite M|Shiloh|Audubon County|IA|59275|United States|-6|condo| +24905|AAAAAAAAJEBGAAAA|349|Jefferson |ST|Suite P|Maple Grove|Ellsworth County|KS|68252|United States|-6|apartment| +24906|AAAAAAAAKEBGAAAA|331|Hill Park|Court|Suite 20|Clifton|Red Willow County|NE|68014|United States|-7|single family| +24907|AAAAAAAALEBGAAAA|629|Walnut 6th|Wy|Suite 70|Providence|Phillips County|MT|66614|United States|-7|single family| +24908|AAAAAAAAMEBGAAAA|537|3rd 1st|Road|Suite R|Friendship|Dewey County|OK|74536|United States|-6|condo| +24909|AAAAAAAANEBGAAAA|543|Railroad Maple|RD|Suite 160|Fairbanks|Polk County|OR|96653|United States|-8|condo| +24910|AAAAAAAAOEBGAAAA|516|South |Avenue|Suite 160|Pleasant Grove|Jones County|SD|54136|United States|-7|condo| +24911|AAAAAAAAPEBGAAAA|941|Laurel |Street|Suite C|Oakdale|Tazewell County|VA|29584|United States|-5|apartment| +24912|AAAAAAAAAFBGAAAA|695|Park |Boulevard|Suite 310|Florence|Pike County|MS|53394|United States|-6|apartment| +24913|AAAAAAAABFBGAAAA|853|East 15th|Blvd|Suite W|Jamestown|Lucas County|OH|46867|United States|-5|condo| +24914|AAAAAAAACFBGAAAA|840|Pine View|Road|Suite 120|Hazelwood|Dakota County|MN|51206|United States|-6|single family| +24915|AAAAAAAADFBGAAAA|588|Johnson 6th|Boulevard|Suite 80|Harmony|Oconee County|SC|25804|United States|-5|condo| +24916|AAAAAAAAEFBGAAAA|831|Pine 5th|Street|Suite 0|Oakland|Tishomingo County|MS|59843|United States|-6|condo| +24917|AAAAAAAAFFBGAAAA||Hill Cedar|||Hopewell|||30587|||condo| +24918|AAAAAAAAGFBGAAAA|953|Second 5th|Way|Suite 0|Lakeview|Clay County|TN|38579|United States|-5|single family| +24919|AAAAAAAAHFBGAAAA|808|9th 11th|Boulevard|Suite 430|Wilson|Forsyth County|GA|36971|United States|-5|apartment| +24920|AAAAAAAAIFBGAAAA|567|2nd |Way|Suite F|Arlington|Lawrence County|MS|56557|United States|-6|apartment| +24921|AAAAAAAAJFBGAAAA|780|6th |Blvd|Suite 150|Brownsville|Clinton County|IA|59310|United States|-6|condo| +24922|AAAAAAAAKFBGAAAA|573|Main Maple|RD|Suite Y|Glenwood|Richland County|IL|63511|United States|-6|apartment| +24923|AAAAAAAALFBGAAAA|||||Walnut Grove||MN||United States||| +24924|AAAAAAAAMFBGAAAA|514|12th |Way|Suite 180|Hopewell|Union County|IN|40587|United States|-5|condo| +24925|AAAAAAAANFBGAAAA|925|Madison Railroad|Way|Suite T|Oak Hill|Limestone County|AL|37838|United States|-6|condo| +24926|AAAAAAAAOFBGAAAA|582|4th Williams|Ln|Suite 260|Bridgeport|Cleveland County|AR|75817|United States|-6|apartment| +24927|AAAAAAAAPFBGAAAA|776|Central |Court|Suite C|Shore Acres|Washington County|IA|52724|United States|-6|single family| +24928|AAAAAAAAAGBGAAAA|853|Oak Pine|Ave|Suite T|Salem|Summers County|WV|28048|United States|-5|condo| +24929|AAAAAAAABGBGAAAA|832|Madison East|Cir.|Suite 140|Georgetown|Llano County|TX|77057|United States|-6|apartment| +24930|AAAAAAAACGBGAAAA||||Suite J||||23683||-5|apartment| +24931|AAAAAAAADGBGAAAA|6|15th |Ct.|Suite 0|Greenwood|Bossier Parish|LA|78828|United States|-6|single family| +24932|AAAAAAAAEGBGAAAA|76|Lincoln |Circle|Suite 30|Union Hill|Atchison County|MO|67746|United States|-6|condo| +24933|AAAAAAAAFGBGAAAA|492|Spring Center|RD|Suite 10|Plainview|Loving County|TX|73683|United States|-6|apartment| +24934|AAAAAAAAGGBGAAAA|348|Hill |Lane|Suite 470|Bridgeport|Huron County|OH|45817|United States|-5|condo| +24935|AAAAAAAAHGBGAAAA|522|Williams |Ct.|Suite 10|Plainview|Muskingum County|OH|43683|United States|-5|condo| +24936|AAAAAAAAIGBGAAAA|843|Mill 2nd|Road|Suite 70|Clifton|Wapello County|IA|58014|United States|-6|single family| +24937|AAAAAAAAJGBGAAAA|206|Williams Center|Court|Suite G|Edgewood|Atchison County|MO|60069|United States|-6|condo| +24938|AAAAAAAAKGBGAAAA|62|Lakeview |Ave|Suite I|Crossroads|DeKalb County|TN|30534|United States|-5|apartment| +24939|AAAAAAAALGBGAAAA|972|Park 7th|Dr.|Suite 80|Arlington|Clare County|MI|46557|United States|-5|condo| +24940|AAAAAAAAMGBGAAAA|142|Cedar |Ct.|Suite G|Valley View|Saratoga County|NY|15124|United States|-5|condo| +24941|AAAAAAAANGBGAAAA|911|Sixth |Boulevard|Suite 220|Summit|Uinta County|WY|80499|United States|-7|single family| +24942|AAAAAAAAOGBGAAAA|922||Parkway||||MO||||single family| +24943|AAAAAAAAPGBGAAAA|170|Spring Smith|Parkway|Suite U|Stewart|Steuben County|IN|48041|United States|-5|condo| +24944|AAAAAAAAAHBGAAAA|383|Madison |Boulevard|Suite L|Plainview|Spencer County|IN|43683|United States|-5|apartment| +24945|AAAAAAAABHBGAAAA|473|Jackson |Wy|Suite 440|Liberty|Brookings County|SD|53451|United States|-6|apartment| +24946|AAAAAAAACHBGAAAA|623|Franklin |Wy|Suite 320|Sunnyside|Solano County|CA|91952|United States|-8|condo| +24947|AAAAAAAADHBGAAAA||Chestnut Spring|Lane|Suite 450||Perry County|||United States|-5|| +24948|AAAAAAAAEHBGAAAA|15|Johnson Chestnut|Lane|Suite O|Thompsonville|Botetourt County|VA|29651|United States|-5|apartment| +24949|AAAAAAAAFHBGAAAA|304|Walnut |Ct.|Suite 250|Clifton|Knox County|TN|38014|United States|-6|condo| +24950|AAAAAAAAGHBGAAAA|121|12th Cedar|Court|Suite A|Tyrone|Kleberg County|TX|71201|United States|-6|single family| +24951|AAAAAAAAHHBGAAAA||Jackson ||Suite 150||Dyer County||32898|United States|-5|single family| +24952|AAAAAAAAIHBGAAAA|913|West Smith|Cir.|Suite U|Buena Vista|Chemung County|NY|15752|United States|-5|single family| +24953|AAAAAAAAJHBGAAAA|461|Mill |Boulevard|Suite B|Shiloh|Newport News city|VA|29275|United States|-5|apartment| +24954|AAAAAAAAKHBGAAAA|703|Sixth |Wy|Suite I|Brownsville|Columbia County|GA|39310|United States|-5|single family| +24955|AAAAAAAALHBGAAAA|237|Elm |Road|Suite D|Roxbury|Nelson County|ND|55508|United States|-6|single family| +24956|AAAAAAAAMHBGAAAA|249|Jefferson |Way|Suite T|Argyle|Roger Mills County|OK|78722|United States|-6|condo| +24957|AAAAAAAANHBGAAAA|12|Smith 5th|Cir.|Suite 330|Pleasant Valley|Orangeburg County|SC|22477|United States|-5|condo| +24958|AAAAAAAAOHBGAAAA|537|Laurel Sunset|Boulevard|Suite 190|Mount Vernon|Hardin County|IL|68482|United States|-6|condo| +24959|AAAAAAAAPHBGAAAA|484|College |Court|Suite 410|Clifton|Gray County|KS|68014|United States|-6|single family| +24960|AAAAAAAAAIBGAAAA|284|Madison Birch|Lane|Suite X|Edgewood|Sullivan County|TN|30069|United States|-6|single family| +24961|AAAAAAAABIBGAAAA|13|Oak 11th|Way|Suite 420|Summit|Ellis County|KS|60499|United States|-6|condo| +24962|AAAAAAAACIBGAAAA|123|Hillcrest |Way|Suite 110|Red Hill|Greene County|IN|44338|United States|-5|single family| +24963|AAAAAAAADIBGAAAA|816|Jackson |Ave|Suite 100|Five Points|Nelson County|ND|56098|United States|-6|condo| +24964|AAAAAAAAEIBGAAAA|961|South Maple|Ct.|Suite 380|Plainview|Genesee County|MI|43683|United States|-5|apartment| +24965|AAAAAAAAFIBGAAAA|140|Ash |RD|Suite R|Lakewood|Wabasha County|MN|58877|United States|-6|single family| +24966|AAAAAAAAGIBGAAAA|661|Chestnut Ridge|Drive|Suite 30|Spring Hill|Harrison County|IA|56787|United States|-6|single family| +24967|AAAAAAAAHIBGAAAA|428|3rd South|Drive|Suite X|Marion|||20399|United States||condo| +24968|AAAAAAAAIIBGAAAA|852|View Washington|Parkway|Suite S|Providence|Washington County|WI|56614|United States|-6|condo| +24969|AAAAAAAAJIBGAAAA|878|South 6th|Street|Suite 320|Hopewell|Brule County|SD|50587|United States|-6|apartment| +24970|AAAAAAAAKIBGAAAA|||Parkway|Suite 260|||||United States||| +24971|AAAAAAAALIBGAAAA|418|12th 10th|Avenue|Suite 280|Union|Daggett County|UT|88721|United States|-7|apartment| +24972|AAAAAAAAMIBGAAAA|335|Cherry 9th|Avenue|Suite 220|Stringtown|Dundy County|NE|60162|United States|-6|apartment| +24973|AAAAAAAANIBGAAAA|571|Meadow |Pkwy|Suite 80|Lebanon|Jackson County|IA|52898|United States|-6|apartment| +24974|AAAAAAAAOIBGAAAA|337|Chestnut |Dr.|Suite 60|Hopewell|Steuben County|NY|10587|United States|-5|single family| +24975|AAAAAAAAPIBGAAAA|549|Lee |Cir.|Suite 470|Clinton|Cecil County|MD|28222|United States|-5|single family| +24976|AAAAAAAAAJBGAAAA|9|Maple |Ct.|Suite 280|Hamilton|Hart County|KY|42808|United States|-6|single family| +24977|AAAAAAAABJBGAAAA|51|Wilson 5th|Street|Suite U|Georgetown|Gogebic County|MI|47057|United States|-5|apartment| +24978|AAAAAAAACJBGAAAA|255|Williams 2nd|Court|Suite 420|Farmington|Madison County|IL|69145|United States|-6|apartment| +24979|AAAAAAAADJBGAAAA|48|Oak East|Parkway|Suite P|Valley View|Jasper County|IL|65124|United States|-6|condo| +24980|AAAAAAAAEJBGAAAA|102|Highland |Dr.|Suite 270|Union Hill|Lincoln County|SD|57746|United States|-7|condo| +24981|AAAAAAAAFJBGAAAA|728|7th Wilson|Court|Suite 0|New Hope|De Witt County|IL|69431|United States|-6|apartment| +24982|AAAAAAAAGJBGAAAA|422|Washington Center|Road|Suite W|Bethel|Oconto County|WI|55281|United States|-6|apartment| +24983|AAAAAAAAHJBGAAAA|483|Washington |Road|Suite 230|Mount Zion|Stanly County|NC|28054|United States|-5|single family| +24984|AAAAAAAAIJBGAAAA|915|Maple |Blvd|Suite H|Oak Ridge|Shoshone County|ID|88371|United States|-7|single family| +24985|AAAAAAAAJJBGAAAA|690|Franklin |Ave|Suite M|Mount Zion|Island County|WA|98054|United States|-8|apartment| +24986|AAAAAAAAKJBGAAAA|581|Walnut 10th|Court|Suite M|Sulphur Springs|Northumberland County|PA|18354|United States|-5|single family| +24987|AAAAAAAALJBGAAAA|568|10th Hill|Street|Suite U|Lakeview|Henderson County|KY|48579|United States|-6|apartment| +24988|AAAAAAAAMJBGAAAA|595|Lakeview |Wy|Suite 480|Brownsville|Columbia County|OR|99310|United States|-8|apartment| +24989|AAAAAAAANJBGAAAA|249|Cherry Fifth|Parkway|Suite H|Oak Hill|Honolulu County|HI|97838|United States|-10|condo| +24990|AAAAAAAAOJBGAAAA|534|4th Sunset|Way|Suite I|Wilson|Porter County|IN|46971|United States|-5|single family| +24991|AAAAAAAAPJBGAAAA|939|5th 2nd|Blvd|Suite 30|Lebanon|Jones County|NC|22898|United States|-5|condo| +24992|AAAAAAAAAKBGAAAA|413||Dr.|Suite 130||||47838|||| +24993|AAAAAAAABKBGAAAA|85|Eigth |Boulevard|Suite 0|Plainview|Rabun County|GA|33683|United States|-5|single family| +24994|AAAAAAAACKBGAAAA|93|Pine Lincoln|ST|Suite O|Jackson|Placer County|CA|99583|United States|-8|single family| +24995|AAAAAAAADKBGAAAA|356|Adams Main|Lane|Suite N|Mount Olive|Pottawatomie County|KS|68059|United States|-6|apartment| +24996|AAAAAAAAEKBGAAAA|978|Maple |Boulevard|Suite K|Sunnyside|Irion County|TX|71952|United States|-6|condo| +24997|AAAAAAAAFKBGAAAA|875|Cedar |Court|Suite F|Lakewood|Pennington County|MN|58877|United States|-6|condo| +24998|AAAAAAAAGKBGAAAA|432|Woodland Highland|Wy|Suite 340|Hartland|Brown County|IN|46594|United States|-5|single family| +24999|AAAAAAAAHKBGAAAA|930|4th Fourth|Court|Suite 30|Liberty|Jefferson County|IN|43451|United States|-5|condo| +25000|AAAAAAAAIKBGAAAA|872|10th |Court|Suite 210|Riverside|Graham County|NC|29231|United States|-5|apartment| +25001|AAAAAAAAJKBGAAAA|877|Second Main|Street|Suite A|Jackson|Lawrence County|SD|59583|United States|-7|condo| +25002|AAAAAAAAKKBGAAAA|396|2nd |Cir.|Suite 110|Lakeside|Oswego County|NY|19532|United States|-5|apartment| +25003|AAAAAAAALKBGAAAA|646|Eigth |Boulevard|Suite R|Bridgeport|Lee County|NC|25817|United States|-5|single family| +25004|AAAAAAAAMKBGAAAA|78|Church |Cir.|Suite I|Newport|Miami County|OH|41521|United States|-5|condo| +25005|AAAAAAAANKBGAAAA|955||Parkway|Suite 100||Island County|WA|||-8|condo| +25006|AAAAAAAAOKBGAAAA|763|6th Oak|Avenue|Suite 400|Bridgeport|Campbell County|TN|35817|United States|-5|apartment| +25007|AAAAAAAAPKBGAAAA|324|Ridge Woodland|Cir.|Suite 380|Oak Hill|Jasper County|MO|67838|United States|-6|condo| +25008|AAAAAAAAALBGAAAA|912|Johnson Second|Ave|Suite 380|Sunnyside|Owen County|IN|41952|United States|-5|condo| +25009|AAAAAAAABLBGAAAA|836|7th 3rd|Boulevard|Suite 20|Waterloo|Custer County|OK|71675|United States|-6|apartment| +25010|AAAAAAAACLBGAAAA|971|||Suite 70|Union Hill|Haines Borough|||||condo| +25011|AAAAAAAADLBGAAAA|406|7th Adams|Court|Suite 370|Mount Vernon|Hillsborough County|FL|38482|United States|-5|single family| +25012|AAAAAAAAELBGAAAA|977|Valley |Boulevard|Suite 280|Bethel|Marshall County|MS|55281|United States|-6|apartment| +25013|AAAAAAAAFLBGAAAA|1000|Washington Sunset|Street|Suite 170|Five Forks|Hancock County|ME|02893|United States|-5|apartment| +25014|AAAAAAAAGLBGAAAA|670|Poplar Miller|Court|Suite 160|Shiloh|Union County|OH|49275|United States|-5|condo| +25015|AAAAAAAAHLBGAAAA|508|Walnut |Blvd|Suite 330|Green Acres|Casey County|KY|47683|United States|-6|apartment| +25016|AAAAAAAAILBGAAAA|108|Willow |Ln|Suite 100|Clinton|Hickman County|KY|48222|United States|-6|apartment| +25017|AAAAAAAAJLBGAAAA|846|Oak Central|ST|Suite 370|Little River|Richland County|MT|60319|United States|-7|single family| +25018|AAAAAAAAKLBGAAAA|175|Main |Parkway|Suite 490|Sunnyside|Chattooga County|GA|31952|United States|-5|condo| +25019|AAAAAAAALLBGAAAA||Park |Dr.|Suite 330|Florence||||||| +25020|AAAAAAAAMLBGAAAA|145|Lake |Lane|Suite E|Sunnyside|Jones County|SD|51952|United States|-7|condo| +25021|AAAAAAAANLBGAAAA|||RD|Suite 240||Floyd County|KY|43511|United States|-6|condo| +25022|AAAAAAAAOLBGAAAA|822|Lakeview |Drive|Suite 480|Farmington|Le Sueur County|MN|59145|United States|-6|condo| +25023|AAAAAAAAPLBGAAAA||||||Antelope County|NE|64273|||single family| +25024|AAAAAAAAAMBGAAAA|458|Main |Lane|Suite T|Pleasant Grove|Stoddard County|MO|64136|United States|-6|single family| +25025|AAAAAAAABMBGAAAA|308|East Center|Way|Suite 410|Liberty|Osceola County|MI|43451|United States|-5|condo| +25026|AAAAAAAACMBGAAAA|500|Spring Broadway|Court|Suite Y|Ellisville|Leelanau County|MI|46820|United States|-5|apartment| +25027|AAAAAAAADMBGAAAA|782|10th |Court|Suite 220|Cedar Grove|Walsh County|ND|50411|United States|-6|single family| +25028|AAAAAAAAEMBGAAAA|603|Birch Wilson|Ln|Suite Q|Sulphur Springs|La Crosse County|WI|58354|United States|-6|apartment| +25029|AAAAAAAAFMBGAAAA|194|Valley Fifth|Boulevard|Suite 230|Red Hill|Ripley County|IN|44338|United States|-5|condo| +25030|AAAAAAAAGMBGAAAA||Willow 1st|ST||Shiloh|Laramie County|WY||United States||| +25031|AAAAAAAAHMBGAAAA|30|3rd |Way|Suite V|Jamestown|Atkinson County|GA|36867|United States|-5|single family| +25032|AAAAAAAAIMBGAAAA|301|1st River|Blvd|Suite M|Glenwood|Dodge County|MN|53511|United States|-6|condo| +25033|AAAAAAAAJMBGAAAA|164|Jefferson Fifth|Parkway|Suite U|Jackson|Madison County|TX|79583|United States|-6|condo| +25034|AAAAAAAAKMBGAAAA|137|West |Parkway|Suite 470|Friendship|Sawyer County|WI|54536|United States|-6|condo| +25035|AAAAAAAALMBGAAAA|721|Jackson Locust|Ln|Suite 380|Wright|Williamsburg city|VA|22814|United States|-5|condo| +25036|AAAAAAAAMMBGAAAA|153|8th Valley|Circle|Suite V|Woodlawn|McCulloch County|TX|74098|United States|-6|apartment| +25037|AAAAAAAANMBGAAAA|512|Center |Avenue|Suite E|Fisher|Pima County|AZ|82819|United States|-7|condo| +25038|AAAAAAAAOMBGAAAA|143|Main |Dr.|Suite D|Wilson|Cowlitz County|WA|96971|United States|-8|condo| +25039|AAAAAAAAPMBGAAAA|756|Laurel |Street|Suite 400|Jamestown|Barbour County|AL|36867|United States|-6|apartment| +25040|AAAAAAAAANBGAAAA|201|Smith |Road|Suite 90|New Hope|Leslie County|KY|49431|United States|-5|apartment| +25041|AAAAAAAABNBGAAAA|674|North Spring|ST|Suite V|Pleasant Grove|Scott County|MO|64136|United States|-6|single family| +25042|AAAAAAAACNBGAAAA|504|Main |Drive|Suite B|Farmington|Millard County|UT|89145|United States|-7|apartment| +25043|AAAAAAAADNBGAAAA|36|3rd |Lane|Suite R|Walnut Grove|Coal County|OK|77752|United States|-6|single family| +25044|AAAAAAAAENBGAAAA|59|Main Chestnut|Pkwy|Suite 490|Red Hill|Polk County|OR|94338|United States|-8|single family| +25045|AAAAAAAAFNBGAAAA|473|Lee 5th|Circle|Suite B|Woodville|White County|GA|34289|United States|-5|single family| +25046|AAAAAAAAGNBGAAAA|848|North |Boulevard|Suite V|Fisher|Trinity County|TX|72819|United States|-6|single family| +25047|AAAAAAAAHNBGAAAA|263|Fourth |Ln|Suite 20|Salem|Phelps County|NE|68048|United States|-7|apartment| +25048|AAAAAAAAINBGAAAA|658|Birch |Ln|Suite S|Summit|Sharp County|AR|70499|United States|-6|single family| +25049|AAAAAAAAJNBGAAAA|178|Jackson |Way|Suite E|Belmont|Prowers County|CO|80191|United States|-7|condo| +25050|AAAAAAAAKNBGAAAA|993|Pine North|RD|Suite D|Newport|Grand County|UT|81521|United States|-7|apartment| +25051|AAAAAAAALNBGAAAA|929|Central |Wy|Suite 200|Clifton|Caldwell Parish|LA|78014|United States|-6|apartment| +25052|AAAAAAAAMNBGAAAA|403|Hill Central|Lane|Suite L|Harmony|Gladwin County|MI|45804|United States|-5|apartment| +25053|AAAAAAAANNBGAAAA|95|First Ridge|Wy|Suite 400|Riverside|Bladen County|NC|29231|United States|-5|apartment| +25054|AAAAAAAAONBGAAAA|245|4th |Street|Suite 100|Oak Grove|Ohio County|KY|48370|United States|-5|apartment| +25055|AAAAAAAAPNBGAAAA|297|Cherry |Parkway|Suite L|Oak Grove|Pipestone County|MN|58370|United States|-6|apartment| +25056|AAAAAAAAAOBGAAAA|251|Cherry 3rd|Wy|Suite Y|Bunker Hill|Hickman County|TN|30150|United States|-5|condo| +25057|AAAAAAAABOBGAAAA|468|Birch |Cir.|Suite 170|Summit|Henry County|TN|30499|United States|-5|single family| +25058|AAAAAAAACOBGAAAA|207|Thirteenth |ST|Suite 250|Oakland|Sweetwater County|WY|89843|United States|-7|apartment| +25059|AAAAAAAADOBGAAAA|736|Green |Road|Suite 20|Maple Grove|Wyandotte County|KS|68252|United States|-6|condo| +25060|AAAAAAAAEOBGAAAA|155|Dogwood 2nd|Street|Suite 470|Five Forks|Pierce County|ND|52293|United States|-6|condo| +25061|AAAAAAAAFOBGAAAA|516|6th |Avenue|Suite X|Barnes|Lenawee County|MI|43788|United States|-5|condo| +25062|AAAAAAAAGOBGAAAA|290|11th Lakeview|Road|Suite C|Oak Ridge|Indiana County|PA|18371|United States|-5|single family| +25063|AAAAAAAAHOBGAAAA|972|Spring 2nd|Street|Suite Y|Jamestown|McCurtain County|OK|76867|United States|-6|single family| +25064|AAAAAAAAIOBGAAAA|220|Walnut 11th|Street|Suite 40|Red Hill|Howard County|TX|74338|United States|-6|condo| +25065|AAAAAAAAJOBGAAAA|313|Cedar |Avenue|Suite L|Bethel|Sauk County|WI|55281|United States|-6|condo| +25066|AAAAAAAAKOBGAAAA|511|Washington North|Boulevard|Suite G|Jamestown|Charles Mix County|SD|56867|United States|-6|condo| +25067|AAAAAAAALOBGAAAA|27||Parkway||Five Points||OH|46098|United States|-5|| +25068|AAAAAAAAMOBGAAAA|672|14th |Wy|Suite 310|Mount Vernon|Cotton County|OK|78482|United States|-6|single family| +25069|AAAAAAAANOBGAAAA|992|Williams Johnson|Parkway|Suite 350|Enterprise|Raleigh County|WV|21757|United States|-5|single family| +25070|AAAAAAAAOOBGAAAA|153|Hill Center|Dr.|Suite Y|Centerville|Phelps County|MO|60059|United States|-6|condo| +25071|AAAAAAAAPOBGAAAA|527|Jefferson |Wy|Suite W|Mount Zion|Person County|NC|28054|United States|-5|apartment| +25072|AAAAAAAAAPBGAAAA|503|2nd |Circle|Suite 320|Red Hill|Beaver County|OK|74338|United States|-6|condo| +25073|AAAAAAAABPBGAAAA|127|Johnson |Boulevard|Suite 50|Centerville|Ozark County|MO|60059|United States|-6|condo| +25074|AAAAAAAACPBGAAAA|104|Pine Spring|Ln|Suite 370|Clifton|Lewis County|MO|68014|United States|-6|apartment| +25075|AAAAAAAADPBGAAAA|569|Chestnut |ST|Suite U|Oak Grove|Candler County|GA|38370|United States|-5|apartment| +25076|AAAAAAAAEPBGAAAA|937|Dogwood |Lane|Suite 460|Hopewell|Cass County|IA|50587|United States|-6|single family| +25077|AAAAAAAAFPBGAAAA|890|2nd 7th|Circle|Suite T|Mountain View|Greene County|TN|34466|United States|-5|condo| +25078|AAAAAAAAGPBGAAAA|998|Park |ST|Suite L|Marion|La Plata County|CO|80399|United States|-7|apartment| +25079|AAAAAAAAHPBGAAAA|453|Oak |Pkwy|Suite X|Clinton|Woodson County|KS|68222|United States|-6|apartment| +25080|AAAAAAAAIPBGAAAA|||Court|Suite A|||ND||United States||apartment| +25081|AAAAAAAAJPBGAAAA|427|Oak South|Court|Suite G|Hopewell|Waupaca County|WI|50587|United States|-6|single family| +25082|AAAAAAAAKPBGAAAA|792|Johnson Jackson|RD|Suite H|Pleasant Grove|Wise County|TX|74136|United States|-6|condo| +25083|AAAAAAAALPBGAAAA|67|Elm |Ct.|Suite K|Buena Vista|Harlan County|NE|65752|United States|-6|condo| +25084|AAAAAAAAMPBGAAAA|340|8th Jackson|Lane|Suite D|Wildwood|Pitt County|NC|26871|United States|-5|condo| +25085|AAAAAAAANPBGAAAA|828|Broadway Willow|Ct.|Suite 110|Green Acres|Clarke County|AL|37683|United States|-6|condo| +25086|AAAAAAAAOPBGAAAA|847|Spruce |Ln|Suite L|Welcome|Terrebonne Parish|LA|76386|United States|-6|apartment| +25087|AAAAAAAAPPBGAAAA|738|Hickory Ridge|Pkwy|Suite 390|Edgewood|Crosby County|TX|70069|United States|-6|condo| +25088|AAAAAAAAAACGAAAA|106|7th |Ct.|Suite Y|Fairfield|Martin County|NC|26192|United States|-5|single family| +25089|AAAAAAAABACGAAAA|418|Park |Way|Suite 120|Forest Hills|Effingham County|GA|39237|United States|-5|single family| +25090|AAAAAAAACACGAAAA|986|Johnson Forest|RD|Suite 190|Mount Zion|Humboldt County|IA|58054|United States|-6|apartment| +25091|AAAAAAAADACGAAAA|954|Elm Mill|Road|Suite 220|Bridgeport|Gage County|NE|65817|United States|-6|single family| +25092|AAAAAAAAEACGAAAA|383|Poplar |Avenue|Suite P|Westminster|Nassau County|NY|16549|United States|-5|apartment| +25093|AAAAAAAAFACGAAAA|979|Pine Walnut|ST|Suite Y|Glendale|Larue County|KY|43951|United States|-5|condo| +25094|AAAAAAAAGACGAAAA|703|Lee 3rd|Ct.|Suite W|Saint James|Duval County|FL|35799|United States|-5|apartment| +25095|AAAAAAAAHACGAAAA|272|Sunset |Ave|Suite G|Centerville|Chesterfield County|VA|20059|United States|-5|apartment| +25096|AAAAAAAAIACGAAAA|426|Mill |Boulevard|Suite 470|Riverdale|Red River County|TX|79391|United States|-6|condo| +25097|AAAAAAAAJACGAAAA|511|Hickory Maple|Ln|Suite F|Oak Hill|Lapeer County|MI|47838|United States|-5|single family| +25098|AAAAAAAAKACGAAAA|556|Hill View|Avenue|Suite 150|Cedar Grove|Effingham County|IL|60411|United States|-6|apartment| +25099|AAAAAAAALACGAAAA|495|1st Central|Avenue|Suite 130|Green Acres|Bonner County|ID|87683|United States|-7|apartment| +25100|AAAAAAAAMACGAAAA|4|Adams Center|RD|Suite 110|Macedonia|West Baton Rouge Parish|LA|71087|United States|-6|single family| +25101|AAAAAAAANACGAAAA|921|Forest 8th|Ave|Suite 270|Woodlawn|Union County|SD|54098|United States|-7|single family| +25102|AAAAAAAAOACGAAAA|||Cir.|Suite L||Seminole County|GA|36871|||single family| +25103|AAAAAAAAPACGAAAA|669|Lakeview Mill|Ave|Suite 150|Forest Hills|Jones County|NC|29237|United States|-5|single family| +25104|AAAAAAAAABCGAAAA|76|Washington Cedar|Street|Suite 140|Kimball|Armstrong County|TX|73595|United States|-6|condo| +25105|AAAAAAAABBCGAAAA|660|9th |ST|Suite W|Plainview|Haskell County|KS|63683|United States|-6|condo| +25106|AAAAAAAACBCGAAAA|444|8th Madison|Street|Suite 50|Lakeville|Hickory County|MO|68811|United States|-6|apartment| +25107|AAAAAAAADBCGAAAA|212|10th Maple|RD|Suite 330|Shady Grove|Broward County|FL|32812|United States|-5|condo| +25108|AAAAAAAAEBCGAAAA|55|Hillcrest |Pkwy|Suite 430|Red Hill|Jasper County|SC|24338|United States|-5|condo| +25109|AAAAAAAAFBCGAAAA|191|Fourth Park|Wy|Suite 330|Pleasant Hill|Platte County|NE|63604|United States|-7|apartment| +25110|AAAAAAAAGBCGAAAA|228|First |Circle|Suite P|Arlington|Simpson County|MS|56557|United States|-6|apartment| +25111|AAAAAAAAHBCGAAAA|345|Hillcrest Church|Cir.|Suite T|Hopewell|Platte County|NE|60587|United States|-7|condo| +25112|AAAAAAAAIBCGAAAA|867|View |Lane|Suite 90|Greenwood|Jackson County|TN|38828|United States|-5|condo| +25113|AAAAAAAAJBCGAAAA|302|Smith |Wy|Suite M|Oak Hill|Genesee County|NY|17838|United States|-5|single family| +25114|AAAAAAAAKBCGAAAA|869|10th |Court|Suite 200|Sunnyside|Southampton County|VA|21952|United States|-5|apartment| +25115|AAAAAAAALBCGAAAA|847|Birch |Street|Suite B|Sulphur Springs|Polk County|MN|58354|United States|-6|condo| +25116|AAAAAAAAMBCGAAAA|875|1st Birch|Drive|Suite L|Franklin|Bibb County|AL|39101|United States|-6|condo| +25117|AAAAAAAANBCGAAAA|495|Spring Poplar|RD|Suite 320|Bradley|Macon County|NC|21777|United States|-5|single family| +25118|AAAAAAAAOBCGAAAA|18|Green Highland|Blvd|Suite Q|Forest Hills|Dooly County|GA|39237|United States|-5|condo| +25119|AAAAAAAAPBCGAAAA|380|Lake |Road|Suite P|Forest Hills|Chouteau County|MT|69237|United States|-7|condo| +25120|AAAAAAAAACCGAAAA|486|River |Way|Suite 220|Shady Grove|Cheyenne County|NE|62812|United States|-6|apartment| +25121|AAAAAAAABCCGAAAA|291|Mill |Wy|Suite 30|Oak Ridge|Franklin County|KY|48371|United States|-6|condo| +25122|AAAAAAAACCCGAAAA|655|Spring |Street|Suite I|Hillcrest|Quay County|NM|83003|United States|-7|apartment| +25123|AAAAAAAADCCGAAAA|262|Lake 7th|Circle|Suite F|Highland Park|Fentress County|TN|36534|United States|-5|condo| +25124|AAAAAAAAECCGAAAA|792|Walnut 11th|Court|Suite U|Oakwood|Teton County|ID|80169|United States|-7|apartment| +25125|AAAAAAAAFCCGAAAA|176|1st |Circle|Suite 360|Marion|Panola County|MS|50399|United States|-6|single family| +25126|AAAAAAAAGCCGAAAA|994|First |Avenue|Suite P|Bethel|Worcester County|MD|25281|United States|-5|single family| +25127|AAAAAAAAHCCGAAAA|384|Cedar First|Boulevard|Suite O|Riceville|Northwest Arctic Borough|AK|95867|United States|-9|single family| +25128|AAAAAAAAICCGAAAA|319|Lincoln |Drive|Suite B|Gilmore|Hamilton County|KS|65464|United States|-6|condo| +25129|AAAAAAAAJCCGAAAA|618|14th |Ave|Suite 0|White Oak|Swift County|MN|56668|United States|-6|condo| +25130|AAAAAAAAKCCGAAAA|778|College Smith|RD|Suite C|Leland|San Juan County|NM|89452|United States|-7|apartment| +25131|AAAAAAAALCCGAAAA|329|12th |ST|Suite 330|New Hope|Grand County|CO|89431|United States|-7|apartment| +25132|AAAAAAAAMCCGAAAA|368|Hillcrest Spring|Ave|Suite E|Five Points|Harlan County|NE|66098|United States|-6|condo| +25133|AAAAAAAANCCGAAAA|488|Meadow Washington|Road|Suite C|Belmont|Leavenworth County|KS|60191|United States|-6|condo| +25134|AAAAAAAAOCCGAAAA|136|Park River|Parkway|Suite T|Shady Grove|Orange County|TX|72812|United States|-6|single family| +25135|AAAAAAAAPCCGAAAA|18|First Hill|RD|Suite C|Oak Hill|Benzie County|MI|47838|United States|-5|apartment| +25136|AAAAAAAAADCGAAAA|482|14th |Circle|Suite F|Glendale|Dakota County|NE|63951|United States|-6|single family| +25137|AAAAAAAABDCGAAAA|88|13th Cherry|Wy|Suite J|Woodland|Jackson County|IL|64854|United States|-6|single family| +25138|AAAAAAAACDCGAAAA|181|Laurel |Ln|Suite 240|Spring Hill|Uinta County|WY|86787|United States|-7|apartment| +25139|AAAAAAAADDCGAAAA|115|Main College|Ln|Suite T|Hamilton|Forsyth County|GA|32808|United States|-5|apartment| +25140|AAAAAAAAEDCGAAAA|986|3rd |Wy|Suite 470|Spring Hill|McLean County|KY|46787|United States|-5|apartment| +25141|AAAAAAAAFDCGAAAA|318|Davis |Cir.|Suite 10|Lebanon|Lander County|NV|82898|United States|-8|condo| +25142|AAAAAAAAGDCGAAAA|443|2nd 6th|Avenue|Suite 130|Union Hill|Union County|SD|57746|United States|-7|condo| +25143|AAAAAAAAHDCGAAAA|252|Maple East|Street|Suite W|Shiloh|Yadkin County|NC|29275|United States|-5|single family| +25144|AAAAAAAAIDCGAAAA|440|2nd Franklin|Wy|Suite 320|Hamilton|Larimer County|CO|82808|United States|-7|condo| +25145|AAAAAAAAJDCGAAAA|605|Woodland Hillcrest|Street|Suite G|Pleasant Valley|Lyon County|NV|82477|United States|-8|condo| +25146|AAAAAAAAKDCGAAAA|655|View |Blvd|Suite 350|Edgewood|Worth County|MO|60069|United States|-6|apartment| +25147|AAAAAAAALDCGAAAA|996|Second |Court|Suite 180|Belmont|Jersey County|IL|60191|United States|-6|single family| +25148|AAAAAAAAMDCGAAAA|708|Oak |RD|Suite C|Lakeside|Hill County|MT|69532|United States|-7|apartment| +25149|AAAAAAAANDCGAAAA|587|First |Circle|Suite 280|Harmony|McCreary County|KY|45804|United States|-5|single family| +25150|AAAAAAAAODCGAAAA|812|Washington 10th|Street|Suite 180|Valley View|Okmulgee County|OK|75124|United States|-6|condo| +25151|AAAAAAAAPDCGAAAA|914|Oak |Wy|Suite O|Blanchard|DeKalb County|IL|65985|United States|-6|single family| +25152|AAAAAAAAAECGAAAA|586|12th |Drive|Suite B|New Hope|Arapahoe County|CO|89431|United States|-7|apartment| +25153|AAAAAAAABECGAAAA|434|Hillcrest Lake|Way|Suite C|Oak Ridge|Putnam County|FL|38371|United States|-5|single family| +25154|AAAAAAAACECGAAAA|787|Washington Spruce|Pkwy|Suite 140|Red Hill|Hardin County|TX|74338|United States|-6|apartment| +25155|AAAAAAAADECGAAAA|676|Sunset |ST|Suite 410|Friendship|Highland County|OH|44536|United States|-5|condo| +25156|AAAAAAAAEECGAAAA|498|Spring |Ln|Suite Q|Farmersville|McIntosh County|GA|39305|United States|-5|single family| +25157|AAAAAAAAFECGAAAA|240|Green |Wy|Suite L|Georgetown|Montgomery County|IL|67057|United States|-6|apartment| +25158|AAAAAAAAGECGAAAA|75|Oak |Wy|Suite H|Farmington|Yakima County|WA|99145|United States|-8|single family| +25159|AAAAAAAAHECGAAAA|282|View |Circle|Suite 360|Hopewell|Decatur County|IA|50587|United States|-6|single family| +25160|AAAAAAAAIECGAAAA|252|Miller Chestnut|Pkwy|Suite 430|Macedonia|Craighead County|AR|71087|United States|-6|condo| +25161|AAAAAAAAJECGAAAA|21|Hickory Hill|Avenue|Suite 390|Mount Pleasant|Morris County|TX|71933|United States|-6|apartment| +25162|AAAAAAAAKECGAAAA|613|Lee |Court|Suite 110|Forest Hills|White County|IL|69237|United States|-6|condo| +25163|AAAAAAAALECGAAAA|284|Laurel |Way|Suite 100|Deerfield|Pierce County|WI|59840|United States|-6|condo| +25164|AAAAAAAAMECGAAAA|349|6th |ST|Suite 480|Buena Vista|Faulk County|SD|55752|United States|-6|single family| +25165|AAAAAAAANECGAAAA|926|Laurel Second|Road|Suite 310|Mount Olive|Nemaha County|NE|68059|United States|-7|single family| +25166|AAAAAAAAOECGAAAA|504|Willow |ST|Suite 70|Hopewell|Lee County|IA|50587|United States|-6|single family| +25167|AAAAAAAAPECGAAAA|265|Meadow |Ct.|Suite I|Spring Valley|DeKalb County|TN|36060|United States|-5|apartment| +25168|AAAAAAAAAFCGAAAA||||Suite 400||Marshall County|AL||United States|-6|| +25169|AAAAAAAABFCGAAAA|599|Central |Parkway|Suite M|Mount Pleasant|Texas County|OK|71933|United States|-6|single family| +25170|AAAAAAAACFCGAAAA|417|5th Forest|Blvd|Suite X|Clinton|Cheshire County|NH|08822|United States|-5|single family| +25171|AAAAAAAADFCGAAAA|745|Meadow Mill|Ln|Suite 460|Thompson|Benton County|WA|90336|United States|-8|condo| +25172|AAAAAAAAEFCGAAAA|401|8th |Drive|Suite K|Fairview|Switzerland County|IN|45709|United States|-5|apartment| +25173|AAAAAAAAFFCGAAAA|248|Eigth |Road|Suite R|Riverside|Schuyler County|NY|19231|United States|-5|apartment| +25174|AAAAAAAAGFCGAAAA|432|Cedar College|RD|Suite 210|Ashland|Cheyenne County|NE|64244|United States|-6|single family| +25175|AAAAAAAAHFCGAAAA|460|North Poplar|ST|Suite 200|Fox|Franklin County|AL|30631|United States|-6|apartment| +25176|AAAAAAAAIFCGAAAA|||RD|||Greer County|OK|78579||-6|single family| +25177|AAAAAAAAJFCGAAAA|504|Locust |Court|Suite G|Bethel|Montgomery County|AR|75281|United States|-6|condo| +25178|AAAAAAAAKFCGAAAA|427|Cherry Washington|Boulevard|Suite F|Liberty|Aiken County|SC|23451|United States|-5|condo| +25179|AAAAAAAALFCGAAAA|863|Laurel Williams|Cir.|Suite 110|Jamestown|Robertson County|KY|46867|United States|-5|condo| +25180|AAAAAAAAMFCGAAAA|561|Park |Ave|Suite L|New Hope|Campbell County|VA|29431|United States|-5|single family| +25181|AAAAAAAANFCGAAAA|987|Pine |Road|Suite 270|Arlington|Grundy County|MO|66557|United States|-6|condo| +25182|AAAAAAAAOFCGAAAA|783|7th 7th|Ln|Suite 100|Fairview|Missaukee County|MI|45709|United States|-5|single family| +25183|AAAAAAAAPFCGAAAA|227|North Church|Way|Suite 240|Unionville|Ashland County|OH|41711|United States|-5|single family| +25184|AAAAAAAAAGCGAAAA|604|Railroad |Street|Suite 450|Wilton|New Castle County|DE|16997|United States|-5|single family| +25185|AAAAAAAABGCGAAAA|620|Highland |Circle|Suite R|Shady Grove|Rice County|KS|62812|United States|-6|condo| +25186|AAAAAAAACGCGAAAA|137|View |Street|Suite Q|Riverside|Pickens County|GA|39231|United States|-5|single family| +25187|AAAAAAAADGCGAAAA|304|Ash |Lane|Suite 100|Liberty|Garrard County|KY|43451|United States|-6|apartment| +25188|AAAAAAAAEGCGAAAA|391|Oak |Lane|Suite N|Greenfield|Kendall County|TX|75038|United States|-6|condo| +25189|AAAAAAAAFGCGAAAA|786|Main |Lane|Suite 230|Mount Olive|Monroe County|GA|38059|United States|-5|condo| +25190|AAAAAAAAGGCGAAAA|805|Sycamore 3rd|Ct.|Suite 140|Spring Hill|Lancaster County|SC|26787|United States|-5|apartment| +25191|AAAAAAAAHGCGAAAA|641|Cedar View|Boulevard|Suite 190|Belmont|Tattnall County|GA|30191|United States|-5|condo| +25192|AAAAAAAAIGCGAAAA|242|River |Lane|Suite 480|Clinton|Summit County|CO|88222|United States|-7|apartment| +25193|AAAAAAAAJGCGAAAA|787|3rd Johnson|RD|Suite 90|Red Hill|Waseca County|MN|54338|United States|-6|apartment| +25194|AAAAAAAAKGCGAAAA|||||Five Forks||||United States||| +25195|AAAAAAAALGCGAAAA|551|4th South|Court|Suite 300|Wilson|Daniels County|MT|66971|United States|-7|condo| +25196|AAAAAAAAMGCGAAAA|961|Spring Sunset|Wy|Suite E|Jackson|Montgomery County|KS|69583|United States|-6|apartment| +25197|AAAAAAAANGCGAAAA|184|Locust |Court|Suite K|Union Hill|Miami County|KS|67746|United States|-6|single family| +25198|AAAAAAAAOGCGAAAA|473|6th Pine|Court|Suite 460|Macedonia|Wichita County|TX|71087|United States|-6|apartment| +25199|AAAAAAAAPGCGAAAA|917|||||Camas County||87838|United States||| +25200|AAAAAAAAAHCGAAAA|513|View Laurel|Court|Suite V|Mountain View|Greeley County|KS|64466|United States|-6|condo| +25201|AAAAAAAABHCGAAAA|855|2nd Pine|Lane|Suite 160|Oak Hill|Langlade County|WI|57838|United States|-6|condo| +25202|AAAAAAAACHCGAAAA|851|Mill |Ln|Suite 260|Antioch|Gates County|NC|28605|United States|-5|condo| +25203|AAAAAAAADHCGAAAA|832|Second Elm|Cir.|Suite D|Waterloo|Gordon County|GA|31675|United States|-5|apartment| +25204|AAAAAAAAEHCGAAAA|126|13th |Ct.|Suite 70|Delmar|Douglas County|GA|33957|United States|-5|condo| +25205|AAAAAAAAFHCGAAAA|81|Davis 8th|Ave|Suite T|Concord|Delaware County|IA|54107|United States|-6|apartment| +25206|AAAAAAAAGHCGAAAA|233|Woodland |Blvd|Suite 420|Lakeside|Lee County|TX|79532|United States|-6|condo| +25207|AAAAAAAAHHCGAAAA|912|Cedar Locust|Wy|Suite 150|Frankfort|Nobles County|MN|59681|United States|-6|apartment| +25208|AAAAAAAAIHCGAAAA|682||Ln|Suite F|Buena Vista|Haywood County|||||condo| +25209|AAAAAAAAJHCGAAAA||||Suite 220|||||United States||single family| +25210|AAAAAAAAKHCGAAAA|102|Sycamore 15th|Ct.|Suite W|Cedar Grove|Ventura County|CA|90411|United States|-8|apartment| +25211|AAAAAAAALHCGAAAA|531|Washington Valley|Ln|Suite 10|Mount Zion|Coffee County|AL|38054|United States|-6|condo| +25212|AAAAAAAAMHCGAAAA|87|||Suite 340|Centerville|Evans County||||-5|| +25213|AAAAAAAANHCGAAAA|272|Third |ST|Suite 460|Newtown|Livingston County|KY|41749|United States|-5|condo| +25214|AAAAAAAAOHCGAAAA|289|Sycamore |Way|Suite I|Ashland|Sangamon County|IL|64244|United States|-6|condo| +25215|AAAAAAAAPHCGAAAA|750|Franklin Oak|Drive|Suite 10|Five Forks|Washington County|CO|82293|United States|-7|apartment| +25216|AAAAAAAAAICGAAAA|881|10th |Ave|Suite E|Highland Park|Calhoun County|WV|26534|United States|-5|single family| +25217|AAAAAAAABICGAAAA|271|Spruce |Drive|Suite 90|Glendale|Texas County|OK|73951|United States|-6|apartment| +25218|AAAAAAAACICGAAAA|220|Wilson |Blvd|Suite G|Glendale|Macon County|IL|63951|United States|-6|single family| +25219|AAAAAAAADICGAAAA|374|Ridge |Blvd|Suite L|Shady Grove|Newton County|IN|42812|United States|-5|single family| +25220|AAAAAAAAEICGAAAA|14|Jackson Broadway|Pkwy|Suite 150|Greenwood|Sioux County|ND|58828|United States|-6|apartment| +25221|AAAAAAAAFICGAAAA|810|Jackson |RD|Suite V|Forest Hills|Hill County|MT|69237|United States|-7|condo| +25222|AAAAAAAAGICGAAAA|554|7th 8th|Ct.|Suite 200|Pine Grove|Decatur County|TN|34593|United States|-5|single family| +25223|AAAAAAAAHICGAAAA|398|Smith Spruce|Road|Suite N|Riceville|Sutton County|TX|75867|United States|-6|single family| +25224|AAAAAAAAIICGAAAA|600|Forest |Boulevard|Suite G|White Oak|Jefferson County|IL|66668|United States|-6|single family| +25225|AAAAAAAAJICGAAAA|852|6th Adams|Dr.|Suite H|Oakland|Petersburg city|VA|29843|United States|-5|apartment| +25226|AAAAAAAAKICGAAAA|244|Hill North|Road|Suite P|Oak Hill|Lehigh County|PA|17838|United States|-5|apartment| +25227|AAAAAAAALICGAAAA|615|2nd |Road|Suite Q|Clifton|Buchanan County|MO|68014|United States|-6|condo| +25228|AAAAAAAAMICGAAAA|418|7th Valley|Road|Suite 130|Jackson|Crawford County|KS|69583|United States|-6|condo| +25229|AAAAAAAANICGAAAA|508|Hill |ST|Suite 300|Lakewood|Latimer County|OK|78877|United States|-6|single family| +25230|AAAAAAAAOICGAAAA|276|Pine Hill|Boulevard|Suite 150|Shiloh|Lincoln County|GA|39275|United States|-5|single family| +25231|AAAAAAAAPICGAAAA|482|Pine Jefferson|Ct.|Suite 470|Liberty|Daviess County|IN|43451|United States|-5|condo| +25232|AAAAAAAAAJCGAAAA|983|Thirteenth Thirteenth|RD|Suite 290|Sulphur Springs|Page County|IA|58354|United States|-6|single family| +25233|AAAAAAAABJCGAAAA|572|Ash Park|Blvd|Suite F|Bridgeport|Spencer County|KY|45817|United States|-5|single family| +25234|AAAAAAAACJCGAAAA|32|Elm |Road|Suite 400|Indian Village|Barnwell County|SC|21075|United States|-5|single family| +25235|AAAAAAAADJCGAAAA|820|Cherry |Circle|Suite Y|Kingston|Charlton County|GA|34975|United States|-5|single family| +25236|AAAAAAAAEJCGAAAA|386|Cherry |Road|Suite 20|Stringtown|Early County|GA|30162|United States|-5|condo| +25237|AAAAAAAAFJCGAAAA|880|Park 3rd|Cir.|Suite 280|Woodland|Lafayette County|FL|34854|United States|-5|single family| +25238|AAAAAAAAGJCGAAAA|736|Ash |Dr.|Suite J|Valley View|Yuma County|CO|85124|United States|-7|apartment| +25239|AAAAAAAAHJCGAAAA|772|4th Adams|Drive|Suite K|Blanchard|Ellis County|OK|75985|United States|-6|single family| +25240|AAAAAAAAIJCGAAAA|907|5th |Blvd|Suite O|Shady Grove|Jefferson County|OR|92812|United States|-8|condo| +25241|AAAAAAAAJJCGAAAA|582|River |Boulevard|Suite 470|Sulphur Springs|Lincoln County|CO|88354|United States|-7|condo| +25242|AAAAAAAAKJCGAAAA|970|Hickory |Circle|Suite P|Shiloh|Morgan County|UT|89275|United States|-7|condo| +25243|AAAAAAAALJCGAAAA|483|Highland |Blvd|Suite 370|Liberty|Lancaster County|VA|23451|United States|-5|condo| +25244|AAAAAAAAMJCGAAAA|724|2nd |Way|Suite W|Jamestown|Inyo County|CA|96867|United States|-8|condo| +25245|AAAAAAAANJCGAAAA|788|Lee |Wy|Suite O|Pleasant Hill|Cavalier County|ND|53604|United States|-6|single family| +25246|AAAAAAAAOJCGAAAA|17|2nd Lee|Boulevard|Suite J|Florence|Glasscock County|TX|73394|United States|-6|single family| +25247|AAAAAAAAPJCGAAAA|699|Birch |Way|Suite L|Liberty|Daviess County|IN|43451|United States|-5|apartment| +25248|AAAAAAAAAKCGAAAA|744|Franklin First|Ct.|Suite 450|Hopewell|Lauderdale County|MS|50587|United States|-6|apartment| +25249|AAAAAAAABKCGAAAA|173|Sunset |Pkwy|Suite 190|Bethel|Gray County|TX|75281|United States|-6|apartment| +25250|AAAAAAAACKCGAAAA|37|Hickory |Way|Suite 120|Springdale|Glascock County|GA|38883|United States|-5|apartment| +25251|AAAAAAAADKCGAAAA|823|2nd Mill|Drive|Suite 20|Summit|Huron County|MI|40499|United States|-5|single family| +25252|AAAAAAAAEKCGAAAA|39|East Jackson|Way|Suite X|Concord|Marathon County|WI|54107|United States|-6|condo| +25253|AAAAAAAAFKCGAAAA|731|Oak |Blvd|Suite 280|Five Forks|Cowlitz County|WA|92293|United States|-8|apartment| +25254|AAAAAAAAGKCGAAAA|814|12th 1st|Avenue|Suite F|Hillcrest|Jefferson County|IA|53003|United States|-6|single family| +25255|AAAAAAAAHKCGAAAA|332|8th |Circle|Suite Y|Lakeside|Putnam County|WV|29532|United States|-5|single family| +25256|AAAAAAAAIKCGAAAA|856|Cherry Franklin|Drive|Suite 190|Red Hill|Montgomery County|IN|44338|United States|-5|apartment| +25257|AAAAAAAAJKCGAAAA|456|Valley Johnson|Lane|Suite 50|Providence|Cook County|IL|66614|United States|-6|condo| +25258|AAAAAAAAKKCGAAAA|931|Smith 6th|Cir.|Suite H|Sulphur Springs|Sweet Grass County|MT|68354|United States|-7|condo| +25259|AAAAAAAALKCGAAAA|53|Oak Davis|Cir.|Suite J|Pine Valley|Chickasaw County|MS|58209|United States|-6|condo| +25260|AAAAAAAAMKCGAAAA|310|River |ST|Suite H|Oakland|Robertson County|KY|49843|United States|-5|condo| +25261|AAAAAAAANKCGAAAA|430|Mill |ST|Suite S|Newtown|Tulare County|CA|91749|United States|-8|condo| +25262|AAAAAAAAOKCGAAAA|293|Valley |Ave|Suite 190|Centerville|Allen Parish|LA|70059|United States|-6|apartment| +25263|AAAAAAAAPKCGAAAA|815|Maple First|Street|Suite X|Providence|Mercer County|PA|16614|United States|-5|condo| +25264|AAAAAAAAALCGAAAA|974|Central |Drive|Suite X|Brownsville|Gadsden County|FL|39310|United States|-5|apartment| +25265|AAAAAAAABLCGAAAA|988|Wilson 8th|Court|Suite D|Greenfield|San Luis Obispo County|CA|95038|United States|-8|apartment| +25266|AAAAAAAACLCGAAAA|185|Third |Boulevard|Suite X|Mount Vernon|Wright County|IA|58482|United States|-6|single family| +25267|AAAAAAAADLCGAAAA|4|Sixth |Parkway|Suite 380|Riverside|Clinton County|IL|69231|United States|-6|apartment| +25268|AAAAAAAAELCGAAAA|395|6th |Circle|Suite Y|Clifton|Houston County|TX|78014|United States|-6|single family| +25269|AAAAAAAAFLCGAAAA|712|Sunset |Court|Suite 240|Newtown|Nash County|NC|21749|United States|-5|condo| +25270|AAAAAAAAGLCGAAAA|990|Center Oak|RD|Suite K|Bethel|Gratiot County|MI|45281|United States|-5|single family| +25271|AAAAAAAAHLCGAAAA|815|Third |Blvd|||Cherokee County|||United States|-6|| +25272|AAAAAAAAILCGAAAA|95|5th |Ct.|Suite R|Mountain View|Boone County|IA|54466|United States|-6|apartment| +25273|AAAAAAAAJLCGAAAA|855|Cherry |Boulevard|Suite 30|Fayetteville|Goochland County|VA|21732|United States|-5|condo| +25274|AAAAAAAAKLCGAAAA|895|Railroad Main|Ct.|Suite T|Oakwood|Lamar County|MS|50169|United States|-6|apartment| +25275|AAAAAAAALLCGAAAA|597|Hill |ST|Suite 260|Glenwood|San Miguel County|CO|83511|United States|-7|single family| +25276|AAAAAAAAMLCGAAAA|531|3rd 2nd|Dr.|Suite 100|Oakwood|Jewell County|KS|60169|United States|-6|condo| +25277|AAAAAAAANLCGAAAA|221|Hickory 6th|Circle|Suite V|Arlington|Salem city|VA|26557|United States|-5|single family| +25278|AAAAAAAAOLCGAAAA|605|South Oak|Avenue|Suite 280|Mount Pleasant|Peoria County|IL|61933|United States|-6|single family| +25279|AAAAAAAAPLCGAAAA|185|View |Street|Suite 350|Shady Grove|Harlan County|NE|62812|United States|-6|apartment| +25280|AAAAAAAAAMCGAAAA|202|Ash |Road|Suite L|Wildwood|Dolores County|CO|86871|United States|-7|apartment| +25281|AAAAAAAABMCGAAAA|192|Second |Street|Suite 60|Springhill|Seward County|NE|64602|United States|-7|single family| +25282|AAAAAAAACMCGAAAA|22|East |Boulevard|Suite 290|Oakland|Paulding County|OH|49843|United States|-5|single family| +25283|AAAAAAAADMCGAAAA|681|Cherry Sunset|Ct.|Suite C|Franklin|Dyer County|TN|39101|United States|-5|single family| +25284|AAAAAAAAEMCGAAAA|453|Sunset |Cir.|Suite 100|Hillcrest|Winona County|MN|53003|United States|-6|single family| +25285|AAAAAAAAFMCGAAAA|175|Chestnut |Avenue|Suite 280|Lakeside|Ulster County|NY|19532|United States|-5|single family| +25286|AAAAAAAAGMCGAAAA|36|Johnson |Ln|Suite 260|New Hope|Lawrence County|MS|59431|United States|-6|condo| +25287|AAAAAAAAHMCGAAAA|120|6th |Circle|Suite 270|Oakwood|Audubon County|IA|50169|United States|-6|condo| +25288|AAAAAAAAIMCGAAAA|495|4th |Avenue|Suite R|Wildwood|Stanley County|SD|56871|United States|-7|apartment| +25289|AAAAAAAAJMCGAAAA|354|Hickory |Boulevard|Suite J|Youngstown|Turner County|SD|50001|United States|-7|condo| +25290|AAAAAAAAKMCGAAAA|283|Forest |Blvd|Suite 200|Buena Vista|McDonald County|MO|65752|United States|-6|apartment| +25291|AAAAAAAALMCGAAAA|546|1st |Wy|Suite 440|Antioch|Knox County|TX|78605|United States|-6|condo| +25292|AAAAAAAAMMCGAAAA|647||||Bath|||70573|||| +25293|AAAAAAAANMCGAAAA|695|Davis Davis|Wy|Suite 70|Riverside|Limestone County|TX|79231|United States|-6|single family| +25294|AAAAAAAAOMCGAAAA|153|Washington |ST|Suite 380|Macedonia|Highland County|OH|41087|United States|-5|condo| +25295|AAAAAAAAPMCGAAAA|717||||||MT|68579|United States|-7|| +25296|AAAAAAAAANCGAAAA|113|Birch Birch|RD|Suite A|Jackson|Chase County|NE|69583|United States|-6|condo| +25297|AAAAAAAABNCGAAAA|200|Birch |ST|Suite 150|Oakdale|Hopewell city|VA|29584|United States|-5|apartment| +25298|AAAAAAAACNCGAAAA|679|5th |Cir.|Suite Q|Newport|Clackamas County|OR|91521|United States|-8|condo| +25299|AAAAAAAADNCGAAAA|600|Elm Oak|Boulevard|Suite 60|Summit|Cocke County|TN|30499|United States|-5|condo| +25300|AAAAAAAAENCGAAAA|904|Eigth Spring|Drive|Suite R|Greenfield|Morton County|KS|65038|United States|-6|apartment| +25301|AAAAAAAAFNCGAAAA|761|Twelfth Davis|RD|Suite 20|Belmont|Columbia County|NY|10191|United States|-5|single family| +25302|AAAAAAAAGNCGAAAA|833|South 2nd|RD|Suite H|Valley View|Lincoln County|WA|95124|United States|-8|condo| +25303|AAAAAAAAHNCGAAAA|488|4th 3rd|Court|Suite 230|Riverdale|Gage County|NE|69391|United States|-6|apartment| +25304|AAAAAAAAINCGAAAA|371|||Suite H|||TN|38048|United States||| +25305|AAAAAAAAJNCGAAAA|970|Oak Pine|Ct.|Suite 430|Mount Pleasant|Del Norte County|CA|91933|United States|-8|apartment| +25306|AAAAAAAAKNCGAAAA|185|Chestnut |Lane|Suite 320|Carpenter|Carroll County|AR|71147|United States|-6|single family| +25307|AAAAAAAALNCGAAAA|884|Washington Sycamore|Street|Suite I|Riverview|Bamberg County|SC|29003|United States|-5|condo| +25308|AAAAAAAAMNCGAAAA|162|11th Johnson|Circle|Suite 400|Belmont|Edgecombe County|NC|20191|United States|-5|condo| +25309|AAAAAAAANNCGAAAA|670|Jackson |Boulevard|Suite B|Oak Grove|Macon County|IL|68370|United States|-6|single family| +25310|AAAAAAAAONCGAAAA|699|Central North|Lane|Suite 240|Maywood|Labette County|KS|65681|United States|-6|single family| +25311|AAAAAAAAPNCGAAAA|134|4th |Dr.|Suite 280|Pleasant Grove|Pointe Coupee Parish|LA|74136|United States|-6|condo| +25312|AAAAAAAAAOCGAAAA|752|Washington Washington|RD|Suite 310|Oakdale|Charles Mix County|SD|59584|United States|-6|single family| +25313|AAAAAAAABOCGAAAA|750|4th |ST|Suite 490|Concord|Morrill County|NE|64107|United States|-7|single family| +25314|AAAAAAAACOCGAAAA|388|East |Court|Suite N|Highland|Uinta County|WY|89454|United States|-7|single family| +25315|AAAAAAAADOCGAAAA|768|Pine Meadow|RD|Suite I|Deerfield|Spokane County|WA|99840|United States|-8|single family| +25316|AAAAAAAAEOCGAAAA|8|Spring |Parkway|Suite L|Milan|Pulaski County|KY|46697|United States|-5|apartment| +25317|AAAAAAAAFOCGAAAA|361|Thirteenth Cherry|Pkwy|Suite 170|New Hope|Macon County|GA|39431|United States|-5|condo| +25318|AAAAAAAAGOCGAAAA|11|Washington 1st|Ct.|Suite 270|Golden|Washita County|OK|70411|United States|-6|single family| +25319|AAAAAAAAHOCGAAAA|407|4th |Dr.|Suite I||Napa County|CA||United States||| +25320|AAAAAAAAIOCGAAAA|1|Woodland Oak|Drive|Suite L|Oak Hill|Panola County|MS|57838|United States|-6|single family| +25321|AAAAAAAAJOCGAAAA|185|Davis Lincoln|Ct.|Suite E|Glendale|McLean County|ND|53951|United States|-6|condo| +25322|AAAAAAAAKOCGAAAA|438|West Meadow|Street|Suite 220|Webb|Venango County|PA|10899|United States|-5|apartment| +25323|AAAAAAAALOCGAAAA|720|4th |ST|Suite 390|Mount Olive|Ward County|ND|58059|United States|-6|single family| +25324|AAAAAAAAMOCGAAAA|49|Highland Meadow|Road|Suite F|Hopewell|Lee County|TX|70587|United States|-6|single family| +25325|AAAAAAAANOCGAAAA|512|Sycamore River|Road|Suite 150|Franklin|Albany County|NY|19101|United States|-5|single family| +25326|AAAAAAAAOOCGAAAA|311|North Mill|Parkway|Suite P|Belfast|Little River County|AR|70125|United States|-6|apartment| +25327|AAAAAAAAPOCGAAAA|712|Meadow |Avenue|Suite S|Lakeside|Cleburne County|AR|79532|United States|-6|apartment| +25328|AAAAAAAAAPCGAAAA|862|Jackson 5th|Dr.|Suite J|Riverview|Clark County|WI|59003|United States|-6|condo| +25329|AAAAAAAABPCGAAAA|521|North Third|Ave|Suite U|Enterprise|Henderson County|KY|41757|United States|-6|condo| +25330|AAAAAAAACPCGAAAA|419||Parkway||||SD||||single family| +25331|AAAAAAAADPCGAAAA|361|Center Hickory|Road|Suite U|Spring Valley|Wabasha County|MN|56060|United States|-6|condo| +25332|AAAAAAAAEPCGAAAA|186|4th College|Lane|Suite 290|Clifton|Thomas County|NE|68014|United States|-7|single family| +25333|AAAAAAAAFPCGAAAA|361|Oak |Pkwy|Suite V|King|Taylor County|IA|50008|United States|-6|condo| +25334|AAAAAAAAGPCGAAAA|355|2nd Main|RD|Suite 450|Deerfield|Walsh County|ND|59840|United States|-6|single family| +25335|AAAAAAAAHPCGAAAA|229|Lincoln Fourth|Ln|Suite 450|Centerville|Clay County|NC|20059|United States|-5|single family| +25336|AAAAAAAAIPCGAAAA|72|Park |Blvd|Suite M|Bunker Hill|Anderson County|TN|30150|United States|-5|single family| +25337|AAAAAAAAJPCGAAAA|990|Third |Ave|Suite C|Liberty|Jones County|SD|53451|United States|-7|single family| +25338|AAAAAAAAKPCGAAAA|237|Elevnth 10th|Ln|Suite G|Colonial Heights|Washington County|AL|33425|United States|-6|condo| +25339|AAAAAAAALPCGAAAA|444|Willow |Ave|Suite 210|Shady Grove|Franklin County|IA|52812|United States|-6|single family| +25340|AAAAAAAAMPCGAAAA|734|Oak |Parkway|Suite 350|Flint|Garrard County|KY|48909|United States|-6|condo| +25341|AAAAAAAANPCGAAAA|988|Pine |Street|Suite 390|Friendship|Wirt County|WV|24536|United States|-5|condo| +25342|AAAAAAAAOPCGAAAA|112|Pine Main|Pkwy|Suite 170|Oak Grove|Columbus County|NC|28370|United States|-5|single family| +25343|AAAAAAAAPPCGAAAA|293|13th |Ave|Suite W|Oak Hill|Rockland County|NY|17838|United States|-5|apartment| +25344|AAAAAAAAAADGAAAA|949|Washington Third|Lane|Suite 160|Greenfield|Sauk County|WI|55038|United States|-6|single family| +25345|AAAAAAAABADGAAAA|911|||Suite 420||Wilkinson County||59101|United States|-6|| +25346|AAAAAAAACADGAAAA|300|1st |Ln|Suite 370|Bethel|Franklin Parish|LA|75281|United States|-6|apartment| +25347|AAAAAAAADADGAAAA|428|3rd |Ct.|Suite 180|Unionville|Grant County|ND|51711|United States|-6|single family| +25348|AAAAAAAAEADGAAAA|22|Second Franklin|Way|Suite A|Oak Ridge|Allen County|KS|68371|United States|-6|single family| +25349|AAAAAAAAFADGAAAA|102|Main 4th|Cir.|Suite J|Macedonia|San Juan County|NM|81087|United States|-7|apartment| +25350|AAAAAAAAGADGAAAA|311|Sunset Main|RD|Suite 360|Franklin|Nassau County|FL|39101|United States|-5|apartment| +25351|AAAAAAAAHADGAAAA|316|Third |Road|Suite 360|Unionville|Taylor County|GA|31711|United States|-5|condo| +25352|AAAAAAAAIADGAAAA|861|Center |Way|Suite O|Buena Vista|Eaton County|MI|45752|United States|-5|single family| +25353|AAAAAAAAJADGAAAA|386|Second Ridge|Court|Suite 150|Lee|Upshur County|WV|20408|United States|-5|condo| +25354|AAAAAAAAKADGAAAA|517|4th 7th|Boulevard|Suite D|Ashland|Calhoun County|IA|54244|United States|-6|single family| +25355|AAAAAAAALADGAAAA|907|Lake |Court|Suite B|Shiloh|Fleming County|KY|49275|United States|-6|condo| +25356|AAAAAAAAMADGAAAA|913|Railroad |Road|Suite 130|Mount Olive|Rawlins County|KS|68059|United States|-6|apartment| +25357|AAAAAAAANADGAAAA|345|1st Sycamore|Street|Suite H|Hillcrest|Faribault County|MN|53003|United States|-6|single family| +25358|AAAAAAAAOADGAAAA|419|Jefferson |Ln|Suite 290|Reno|Dunn County|ND|50344|United States|-6|single family| +25359|AAAAAAAAPADGAAAA|910|10th |Ct.|Suite G|Providence|Carver County|MN|56614|United States|-6|single family| +25360|AAAAAAAAABDGAAAA|689|8th |Lane|Suite C|Lucas|Green Lake County|WI|54554|United States|-6|single family| +25361|AAAAAAAABBDGAAAA|586|Broadway Lake|Ave|Suite 210|Valley View|Martinsville city|VA|25124|United States|-5|condo| +25362|AAAAAAAACBDGAAAA|216|Fifth 7th|Avenue|Suite 270|Plainview|Spencer County|KY|43683|United States|-5|single family| +25363|AAAAAAAADBDGAAAA|766|2nd Birch|Ln|Suite 170|Kingston|LaMoure County|ND|54975|United States|-6|apartment| +25364|AAAAAAAAEBDGAAAA|189|Hill |Court|Suite 110|Glenwood|Burke County|NC|23511|United States|-5|apartment| +25365|AAAAAAAAFBDGAAAA|88|Main Williams|RD|Suite 100|Belmont|Upton County|TX|70191|United States|-6|single family| +25366|AAAAAAAAGBDGAAAA|256|Hill River|Blvd|Suite 180|Mount Vernon|Cheyenne County|KS|68482|United States|-6|condo| +25367|AAAAAAAAHBDGAAAA|713|View South|Boulevard|Suite 270|Lakeview|Putnam County|MO|68579|United States|-6|apartment| +25368|AAAAAAAAIBDGAAAA|838|Locust |Boulevard|Suite N|Collinsville|Clear Creek County|CO|82459|United States|-7|condo| +25369|AAAAAAAAJBDGAAAA|428|Maple 6th|ST|Suite 80|Woodlawn|Grand Isle County|VT|04698|United States|-5|condo| +25370|AAAAAAAAKBDGAAAA|797|Ash Wilson|Parkway|Suite 320|Providence|Jones County|SD|56614|United States|-7|single family| +25371|AAAAAAAALBDGAAAA|48|East Fourth|Ln|Suite 290|Plainview|Carbon County|UT|83683|United States|-7|single family| +25372|AAAAAAAAMBDGAAAA|366|View |Way|Suite Q|Fairfield|Dodge County|GA|36192|United States|-5|single family| +25373|AAAAAAAANBDGAAAA|484|Hillcrest |Street|Suite F|Oak Grove|Trego County|KS|68370|United States|-6|condo| +25374|AAAAAAAAOBDGAAAA|659|Chestnut |Parkway|Suite 120|Edgewood|Union County|SD|50069|United States|-7|single family| +25375|AAAAAAAAPBDGAAAA|192|Oak |Cir.|Suite M|Lakewood|Clark County|WI|58877|United States|-6|single family| +25376|AAAAAAAAACDGAAAA|128|Maple 8th|Boulevard|Suite Y|Green Acres|Mississippi County|AR|77683|United States|-6|apartment| +25377|AAAAAAAABCDGAAAA|724|Washington |Lane|Suite Y|Deerfield|Glenn County|CA|99840|United States|-8|apartment| +25378|AAAAAAAACCDGAAAA|103|Fifth Lakeview|Road|Suite 390|Highland|Lamar County|TX|79454|United States|-6|single family| +25379|AAAAAAAADCDGAAAA|767|12th Main|Blvd|Suite Q|Lakewood|Georgetown County|SC|28877|United States|-5|single family| +25380|AAAAAAAAECDGAAAA|119|Hickory |Way|Suite H|Mountain View|Somerset County|ME|05066|United States|-5|condo| +25381|AAAAAAAAFCDGAAAA|139|Locust Smith|Way|Suite N|Howell|Muscatine County|IA|54854|United States|-6|condo| +25382|AAAAAAAAGCDGAAAA|603|5th Ridge|RD|Suite 270|Greenfield|Orange County|TX|75038|United States|-6|apartment| +25383|AAAAAAAAHCDGAAAA|478|Fifth |Ln|Suite 170|Deerfield|San Francisco County|CA|99840|United States|-8|condo| +25384|AAAAAAAAICDGAAAA|700|Cedar |Ct.|Suite 410|Spring Hill|Pasco County|FL|36787|United States|-5|condo| +25385|AAAAAAAAJCDGAAAA|642|Second |Ave|Suite A|Ferguson|Gem County|ID|81821|United States|-7|apartment| +25386|AAAAAAAAKCDGAAAA|559|Cedar |Blvd|Suite 150|Waterloo|Story County|IA|51675|United States|-6|single family| +25387|AAAAAAAALCDGAAAA|216|Green |Dr.|Suite I|Union Hill|Lincoln County|MN|57746|United States|-6|apartment| +25388|AAAAAAAAMCDGAAAA|621|Main 3rd|Lane|Suite 10|Highland Park|Hidalgo County|TX|76534|United States|-6|apartment| +25389|AAAAAAAANCDGAAAA|314|College |Ct.|Suite 210|Oak Ridge|Geauga County|OH|48371|United States|-5|condo| +25390|AAAAAAAAOCDGAAAA|5|Maple Franklin|Drive|Suite S|Belmont|Butler County|OH|40191|United States|-5|condo| +25391|AAAAAAAAPCDGAAAA|185|3rd |Circle|Suite 200|Georgetown|Clinch County|GA|37057|United States|-5|condo| +25392|AAAAAAAAADDGAAAA|818|Oak Woodland|Circle|Suite 460|Valley View|Jasper County|MO|65124|United States|-6|condo| +25393|AAAAAAAABDDGAAAA|628|Willow |Pkwy|Suite M|Shady Grove|Craig County|VA|22812|United States|-5|single family| +25394|AAAAAAAACDDGAAAA|293|Ash Railroad|ST|Suite J|Springdale|Washtenaw County|MI|48883|United States|-5|condo| +25395|AAAAAAAADDDGAAAA|135|13th |Ct.|Suite M|Red Hill|Powder River County|MT|64338|United States|-7|condo| +25396|AAAAAAAAEDDGAAAA|545|Sunset |ST|Suite 30|Saint George|Warren County|KY|45281|United States|-5|apartment| +25397|AAAAAAAAFDDGAAAA|116|Oak |Circle|Suite C|Five Points|Jefferson County|MT|66098|United States|-7|apartment| +25398|AAAAAAAAGDDGAAAA|653|Valley Fifth|Circle|Suite I|Woodville|George County|MS|54289|United States|-6|condo| +25399|AAAAAAAAHDDGAAAA|925|Mill 13th|Ave|Suite R|Centerville|McKinley County|NM|80059|United States|-7|single family| +25400|AAAAAAAAIDDGAAAA|22|5th ||Suite O|Enterprise|Grayson County|||United States||apartment| +25401|AAAAAAAAJDDGAAAA|422|8th |Drive|Suite I|Union|Uinta County|WY|88721|United States|-7|condo| +25402|AAAAAAAAKDDGAAAA|906|Maple |Road|Suite D|Philadelphia|Pittsburg County|OK|75591|United States|-6|apartment| +25403|AAAAAAAALDDGAAAA|411|8th 2nd|Avenue|Suite U|Springfield|Plumas County|CA|99303|United States|-8|single family| +25404|AAAAAAAAMDDGAAAA|978|College Second|RD|Suite 370|Jamestown|Anne Arundel County|MD|26867|United States|-5|single family| +25405|AAAAAAAANDDGAAAA|897|7th 7th|Drive|Suite O|Riverview|Moniteau County|MO|69003|United States|-6|single family| +25406|AAAAAAAAODDGAAAA|534|Madison River|Cir.|Suite D|Cedar Grove|Caldwell County|MO|60411|United States|-6|apartment| +25407|AAAAAAAAPDDGAAAA|549|Lake |Pkwy|Suite 350|Union Hill|Halifax County|NC|27746|United States|-5|single family| +25408|AAAAAAAAAEDGAAAA|418|Johnson |Blvd|Suite 210|Georgetown|Rio Arriba County|NM|87057|United States|-7|apartment| +25409|AAAAAAAABEDGAAAA|460|5th Church|Court|Suite 400|Highland|Greene County|TN|39454|United States|-5|single family| +25410|AAAAAAAACEDGAAAA|588|4th |Boulevard|Suite N|Hillcrest|Chippewa County|MN|53003|United States|-6|condo| +25411|AAAAAAAADEDGAAAA|572|Park Second|Drive|Suite L|Highland Park|Wilson County|NC|26534|United States|-5|single family| +25412|AAAAAAAAEEDGAAAA|741|Maple |Road|Suite Y|Sunnyside|Pickens County|AL|31952|United States|-6|condo| +25413|AAAAAAAAFEDGAAAA|719|View Wilson|Ct.|Suite 320|Springfield|Pecos County|TX|79303|United States|-6|condo| +25414|AAAAAAAAGEDGAAAA|950|Johnson |Blvd|Suite K|Lakewood|Highland County|VA|28877|United States|-5|single family| +25415|AAAAAAAAHEDGAAAA|98|Main 8th|Parkway|Suite 390|Sullivan|Decatur County|GA|30451|United States|-5|condo| +25416|AAAAAAAAIEDGAAAA|274|Davis |Way|Suite A|Pleasant Hill|Carson County|TX|73604|United States|-6|condo| +25417|AAAAAAAAJEDGAAAA|815|Jefferson Forest|ST|Suite 250|Woodland|Bibb County|AL|34854|United States|-6|single family| +25418|AAAAAAAAKEDGAAAA|801|View 8th|Road|Suite O|Jamestown|Marion County|WV|26867|United States|-5|condo| +25419|AAAAAAAALEDGAAAA|806|Spring Madison|Avenue|Suite K|Lakewood|Marshall County|MN|58877|United States|-6|condo| +25420|AAAAAAAAMEDGAAAA|618|Lincoln |Avenue|Suite 170|Lebanon|Lunenburg County|VA|22898|United States|-5|single family| +25421|AAAAAAAANEDGAAAA|304|Oak |Court|Suite 430|Blanchard|Tucker County|WV|25985|United States|-5|single family| +25422|AAAAAAAAOEDGAAAA|740|Elevnth |ST|Suite X|Midway|Cheshire County|NH|02504|United States|-5|apartment| +25423|AAAAAAAAPEDGAAAA|645|Lakeview 14th|Way|Suite L|Five Points|Newton County|IN|46098|United States|-5|condo| +25424|AAAAAAAAAFDGAAAA|915|View |Boulevard|Suite U|Union|Buncombe County|NC|28721|United States|-5|apartment| +25425|AAAAAAAABFDGAAAA|551|Wilson Pine||Suite 460||||49454||-5|| +25426|AAAAAAAACFDGAAAA|427|Forest Oak|Court|Suite 390|Arlington|Door County|WI|56557|United States|-6|condo| +25427|AAAAAAAADFDGAAAA|257|4th Center|Cir.|Suite Y|Pleasant Grove|Lyman County|SD|54136|United States|-7|single family| +25428|AAAAAAAAEFDGAAAA|303|Meadow |Road|Suite N|Five Forks|Christian County|KY|42293|United States|-6|condo| +25429|AAAAAAAAFFDGAAAA|613|River |Court|Suite J|Springdale|Solano County|CA|98883|United States|-8|condo| +25430|AAAAAAAAGFDGAAAA|703|Lincoln Valley|Pkwy|Suite 0|Hamilton|Beckham County|OK|72808|United States|-6|apartment| +25431|AAAAAAAAHFDGAAAA|275|Maple Green|Ave|Suite W|Riley|Callahan County|TX|71692|United States|-6|condo| +25432|AAAAAAAAIFDGAAAA|786|Broadway |Court|Suite 150|Enterprise|Orange County|NY|11757|United States|-5|condo| +25433|AAAAAAAAJFDGAAAA|214|Maple |Road|Suite D|Glendale|Kankakee County|IL|63951|United States|-6|apartment| +25434|AAAAAAAAKFDGAAAA|24|West |Ct.|Suite W|Walnut Grove|Story County|IA|57752|United States|-6|single family| +25435|AAAAAAAALFDGAAAA|364|Meadow |Avenue|Suite J|Franklin|Forrest County|MS|59101|United States|-6|condo| +25436|AAAAAAAAMFDGAAAA|268|Dogwood Ash|Road|Suite 330|Montague|Clarke County|GA|34062|United States|-5|condo| +25437|AAAAAAAANFDGAAAA|191|Lake |Dr.|Suite 160|Belmont|Lonoke County|AR|70191|United States|-6|condo| +25438|AAAAAAAAOFDGAAAA|376|6th |ST|Suite 30|Red Hill|Passaic County|NJ|04938|United States|-5|single family| +25439|AAAAAAAAPFDGAAAA|228|Walnut |Ln|Suite B|Pleasant Hill|Crook County|OR|93604|United States|-8|apartment| +25440|AAAAAAAAAGDGAAAA|311|Pine |Street|Suite 420|Glenwood|Val Verde County|TX|73511|United States|-6|apartment| +25441|AAAAAAAABGDGAAAA|837|Ridge West|Dr.|Suite 220|Springfield|Clinton County|IA|59303|United States|-6|condo| +25442|AAAAAAAACGDGAAAA|456|Third Franklin|Ct.|Suite K|Union|Carter County|OK|78721|United States|-6|apartment| +25443|AAAAAAAADGDGAAAA|903|10th |ST|Suite 120|Oak Ridge|Montgomery County|AL|38371|United States|-6|single family| +25444|AAAAAAAAEGDGAAAA|576|Railroad Oak|Drive|Suite 60|Frankfort|Carroll County|TN|39681|United States|-5|condo| +25445|AAAAAAAAFGDGAAAA|518|Main |Dr.|Suite 240|Belmont|Montrose County|CO|80191|United States|-7|condo| +25446|AAAAAAAAGGDGAAAA|124|11th |ST|Suite 90|Summit|Montgomery County|MS|50499|United States|-6|condo| +25447|AAAAAAAAHGDGAAAA|650|Third |Avenue|Suite 370|Spring Hill|Armstrong County|TX|76787|United States|-6|single family| +25448|AAAAAAAAIGDGAAAA|198|6th |Road|Suite 290|Pine Grove|Franklin County|NE|64593|United States|-6|single family| +25449|AAAAAAAAJGDGAAAA|162|Madison Valley|Court|Suite 440|Bridgeport|Macon County|TN|35817|United States|-6|single family| +25450|AAAAAAAAKGDGAAAA|855|Ash |RD|Suite I|Summit|Liberty County|MT|60499|United States|-7|apartment| +25451|AAAAAAAALGDGAAAA|883|Pine 5th|Dr.|Suite 200|Greenville|Sheboygan County|WI|51387|United States|-6|condo| +25452|AAAAAAAAMGDGAAAA|446|2nd Meadow|RD|Suite T|Louisville|Guadalupe County|TX|74464|United States|-6|apartment| +25453|AAAAAAAANGDGAAAA|6|Church 12th|Ave|Suite A|Westgate|Lincoln County|ME|02966|United States|-5|condo| +25454|AAAAAAAAOGDGAAAA|787|13th |Street|Suite 380|Oak Grove|Duval County|TX|78370|United States|-6|single family| +25455|AAAAAAAAPGDGAAAA|211|Washington |Cir.|Suite 390|Pioneer|Rusk County|WI|50967|United States|-6|apartment| +25456|AAAAAAAAAHDGAAAA|303|2nd Church|Street|Suite 180|Lebanon|Hernando County|FL|32898|United States|-5|condo| +25457|AAAAAAAABHDGAAAA|558|Cherry |ST|Suite 280|Forest Hills|Cleveland County|NC|29237|United States|-5|condo| +25458|AAAAAAAACHDGAAAA|975|Willow East|Pkwy|Suite P|Pleasant Hill|King George County|VA|23604|United States|-5|condo| +25459|AAAAAAAADHDGAAAA|537|Meadow |ST|Suite J|Waterloo|Screven County|GA|31675|United States|-5|condo| +25460|AAAAAAAAEHDGAAAA|298|3rd |Lane|Suite 10|Mount Olive|Cherokee County|KS|68059|United States|-6|apartment| +25461|AAAAAAAAFHDGAAAA|558|11th |RD|Suite X|Riverdale|Jefferson County|AL|39391|United States|-6|single family| +25462|AAAAAAAAGHDGAAAA|21|Elm Church|Ct.|Suite 490|Union|Erath County|TX|78721|United States|-6|apartment| +25463|AAAAAAAAHHDGAAAA|773|6th |Wy|Suite 20|Oak Ridge|Van Wert County|OH|48371|United States|-5|condo| +25464|AAAAAAAAIHDGAAAA|33|Elm |Ln|Suite M|Shiloh|Osage County|OK|79275|United States|-6|apartment| +25465|AAAAAAAAJHDGAAAA|444|Chestnut |Lane|Suite 290|Waterloo|Crockett County|TN|31675|United States|-5|apartment| +25466|AAAAAAAAKHDGAAAA|453|Washington Walnut|Way|Suite 50|Mountain View|Loudon County|TN|34466|United States|-6|apartment| +25467|AAAAAAAALHDGAAAA|216|Wilson South|Pkwy|Suite 170|New Hope|Cook County|IL|69431|United States|-6|single family| +25468|AAAAAAAAMHDGAAAA|20|Park 2nd|Wy|Suite V|Cedar Grove|DeKalb County|AL|30411|United States|-6|apartment| +25469|AAAAAAAANHDGAAAA|878|East Laurel|Lane|Suite O|Roy|Kane County|IL|60744|United States|-6|condo| +25470|AAAAAAAAOHDGAAAA|812|3rd Park|Drive|Suite L|Mountain View|Dougherty County|GA|34466|United States|-5|apartment| +25471|AAAAAAAAPHDGAAAA|659|Lincoln |ST|Suite D|Shady Grove|Nacogdoches County|TX|72812|United States|-6|single family| +25472|AAAAAAAAAIDGAAAA|604|Madison Park|Wy|Suite 430|Deerfield|Stephenson County|IL|69840|United States|-6|condo| +25473|AAAAAAAABIDGAAAA|366|8th |Way|Suite I|Greenwood|Edgefield County|SC|28828|United States|-5|single family| +25474|AAAAAAAACIDGAAAA|308|Ash North|Drive|Suite W|Adrian|Castro County|TX|73301|United States|-6|apartment| +25475|AAAAAAAADIDGAAAA|924|Ridge Elm|Drive|Suite N|Springfield|Elmore County|AL|39303|United States|-6|apartment| +25476|AAAAAAAAEIDGAAAA|88|Hickory |Drive|Suite G|Oak Hill|Jefferson County|PA|17838|United States|-5|single family| +25477|AAAAAAAAFIDGAAAA|147|Smith Davis|Circle|Suite B|Glenwood|Washburn County|WI|53511|United States|-6|apartment| +25478|AAAAAAAAGIDGAAAA|537|Davis |Ave|Suite B|Kingston|Guadalupe County|TX|74975|United States|-6|apartment| +25479|AAAAAAAAHIDGAAAA|204|Cherry Oak|Street|Suite 250|Clifton|Santa Rosa County|FL|38014|United States|-5|single family| +25480|AAAAAAAAIIDGAAAA|651|15th |Road|Suite M|Point Pleasant|Matagorda County|TX|74749|United States|-6|condo| +25481|AAAAAAAAJIDGAAAA|403|South Cedar|RD|Suite 430|Woodland|Williamson County|TN|34854|United States|-5|condo| +25482|AAAAAAAAKIDGAAAA|941|Walnut |Circle|Suite 170|Salem|Lehigh County|PA|18048|United States|-5|condo| +25483|AAAAAAAALIDGAAAA|905|Hillcrest |Court|Suite N|Walnut|Chippewa County|MI|46245|United States|-5|single family| +25484|AAAAAAAAMIDGAAAA|452|Dogwood |RD|Suite 30|Deerfield|Jefferson County|WA|99840|United States|-8|apartment| +25485|AAAAAAAANIDGAAAA|361|Adams Lincoln|Dr.|Suite 300|Sunnyside|Putnam County|MO|61952|United States|-6|condo| +25486|AAAAAAAAOIDGAAAA|750|Fourth Walnut|Ln|||Fulton County|IL||United States||| +25487|AAAAAAAAPIDGAAAA|160|Cedar Maple|Drive|Suite T|Saint James|Otoe County|NE|65799|United States|-7|single family| +25488|AAAAAAAAAJDGAAAA|354|Main Park|Boulevard|Suite M|Brownsville|Lackawanna County|PA|19310|United States|-5|condo| +25489|AAAAAAAABJDGAAAA|710|15th Wilson|Road|Suite I|Clifford|Baca County|CO|88164|United States|-7|apartment| +25490|AAAAAAAACJDGAAAA|56|8th |Drive|Suite 260|Plainview|Haralson County|GA|33683|United States|-5|apartment| +25491|AAAAAAAADJDGAAAA|121|6th |Boulevard|Suite 260|Pine Grove|Floyd County|VA|24593|United States|-5|apartment| +25492|AAAAAAAAEJDGAAAA|98|Spring |Road|Suite 490|Hamilton|Shackelford County|TX|72808|United States|-6|condo| +25493|AAAAAAAAFJDGAAAA|372|Laurel |Street|Suite O|Fairfield|Mason County|MI|46192|United States|-5|single family| +25494|AAAAAAAAGJDGAAAA|586|Main Sunset|Boulevard|Suite S|Murray|Alcorn County|MS|52150|United States|-6|condo| +25495|AAAAAAAAHJDGAAAA|818|Meadow Sunset|Circle|Suite 380|Antioch|Darke County|OH|48605|United States|-5|apartment| +25496|AAAAAAAAIJDGAAAA|12|Poplar |Circle|Suite 290|Warwick|Kingsbury County|SD|51398|United States|-7|apartment| +25497|AAAAAAAAJJDGAAAA|398|5th 7th|Way|Suite C|Florence|Kearney County|NE|63394|United States|-7|condo| +25498|AAAAAAAAKJDGAAAA|936|Green Elm|Lane|Suite H|Green Acres|Clay County|KY|47683|United States|-6|apartment| +25499|AAAAAAAALJDGAAAA|546|First Hill|Blvd|Suite D|Midway|Grant County|KS|61904|United States|-6|single family| +25500|AAAAAAAAMJDGAAAA|216|Spring |Ave|Suite S|Hopewell|San Patricio County|TX|70587|United States|-6|condo| +25501|AAAAAAAANJDGAAAA|353|Fifth Elm|Pkwy|Suite Q|Mount Vernon|Putnam County|IL|68482|United States|-6|condo| +25502|AAAAAAAAOJDGAAAA|876|Wilson Oak|Dr.|Suite W|Maple Grove|Lee County|AR|78252|United States|-6|single family| +25503|AAAAAAAAPJDGAAAA|429|Walnut |Ln|Suite X|Mount Pleasant|Williamson County|IL|61933|United States|-6|apartment| +25504|AAAAAAAAAKDGAAAA|341|8th |Ct.|Suite 260|Greenwood|Huntington County|IN|48828|United States|-5|condo| +25505|AAAAAAAABKDGAAAA|976|Railroad Pine|Way|Suite 200|Jamestown|Chicot County|AR|76867|United States|-6|single family| +25506|AAAAAAAACKDGAAAA|224|Johnson Cedar|ST|Suite 350|Bunker Hill|O-Brien County|IA|50150|United States|-6|single family| +25507|AAAAAAAADKDGAAAA|611|6th Main|RD|Suite 170|Highland|Cerro Gordo County|IA|59454|United States|-6|single family| +25508|AAAAAAAAEKDGAAAA|35|Second |Pkwy|Suite S|Edgewood|Cumberland County|NC|20069|United States|-5|apartment| +25509|AAAAAAAAFKDGAAAA|643|Lee Madison|ST|Suite A|Whitney|Jackson County|KY|48339|United States|-6|single family| +25510|AAAAAAAAGKDGAAAA|379|8th Lincoln|Way|Suite 380|Pleasant Valley|Clay County|MN|52477|United States|-6|apartment| +25511|AAAAAAAAHKDGAAAA|591|Laurel |Dr.|Suite D|Pine Grove|Tattnall County|GA|34593|United States|-5|apartment| +25512|AAAAAAAAIKDGAAAA|303|Second |Ln|Suite 300|White Oak|Val Verde County|TX|76668|United States|-6|condo| +25513|AAAAAAAAJKDGAAAA|495|Lincoln River|Cir.|Suite 410|Brownsville|Cumberland County|IL|69310|United States|-6|condo| +25514|AAAAAAAAKKDGAAAA|478|6th First|Street|Suite 280|Newport|Preston County|WV|21521|United States|-5|apartment| +25515|AAAAAAAALKDGAAAA|193|5th Park|Cir.|Suite E|Fairfield|Caldwell Parish|LA|76192|United States|-6|apartment| +25516|AAAAAAAAMKDGAAAA|489|South |Ln|Suite U|Pine Grove|Musselshell County|MT|64593|United States|-7|single family| +25517|AAAAAAAANKDGAAAA|483|Railroad |Ln|Suite G|Deerfield|Wythe County|VA|29840|United States|-5|condo| +25518|AAAAAAAAOKDGAAAA|979|13th View|Way|Suite K|Hartland|Suffolk County|MA|07194|United States|-5|apartment| +25519|AAAAAAAAPKDGAAAA|458|4th |Court|Suite 100|Mount Vernon|Calhoun County|IL|68482|United States|-6|apartment| +25520|AAAAAAAAALDGAAAA|993|Hill 11th|Blvd|Suite 440|Oak Ridge|Hooker County|NE|68371|United States|-7|apartment| +25521|AAAAAAAABLDGAAAA|507|Wilson |Court|Suite Q|Leon|Russell County|KY|40913|United States|-5|apartment| +25522|AAAAAAAACLDGAAAA|89|Willow Elm|Circle|Suite 30|Glenwood|Gratiot County|MI|43511|United States|-5|single family| +25523|AAAAAAAADLDGAAAA|438|Lake |RD|Suite X|Farmington|Gooding County|ID|89145|United States|-7|single family| +25524|AAAAAAAAELDGAAAA|880|Maple |Road|Suite U|Lebanon|Jackson County|MN|52898|United States|-6|condo| +25525|AAAAAAAAFLDGAAAA|349|Walnut Park|Ct.|Suite 370|Enterprise|Braxton County|WV|21757|United States|-5|single family| +25526|AAAAAAAAGLDGAAAA|114|6th Main|Blvd|Suite 240|Summit|Yukon-Koyukuk Census Area|AK|90499|United States|-9|apartment| +25527|AAAAAAAAHLDGAAAA|678|Elm |Ln|Suite 420|Walnut Grove|Anderson County|SC|27752|United States|-5|apartment| +25528|AAAAAAAAILDGAAAA|823|Sunset |Ct.|Suite P|Pleasant Valley|Todd County|KY|42477|United States|-5|condo| +25529|AAAAAAAAJLDGAAAA|755|Meadow |Lane|Suite 160|Oakwood|Greene County|IN|40169|United States|-5|single family| +25530|AAAAAAAAKLDGAAAA|780|4th |Parkway|Suite B|Greenfield|Burke County|ND|55038|United States|-6|single family| +25531|AAAAAAAALLDGAAAA|283|Lee |Ct.|Suite H|Shady Grove|Mason County|WA|92812|United States|-8|condo| +25532|AAAAAAAAMLDGAAAA|937|East |Ct.|Suite R|Leland|Sussex County|VA|29452|United States|-5|apartment| +25533|AAAAAAAANLDGAAAA|431|Lincoln |Court|Suite J|Spring Hill|Jones County|TX|76787|United States|-6|apartment| +25534|AAAAAAAAOLDGAAAA|24|4th |Ave|Suite 250|Church Hill|Morrow County|OH|43790|United States|-5|single family| +25535|AAAAAAAAPLDGAAAA|239|Forest |Court|Suite N|Shiloh|Huron County|OH|49275|United States|-5|condo| +25536|AAAAAAAAAMDGAAAA|73|Seventh 2nd|Drive|Suite 170|Friendship|Gunnison County|CO|84536|United States|-7|single family| +25537|AAAAAAAABMDGAAAA|831|Davis |Ln|Suite 150|Empire|Dickinson County|IA|54145|United States|-6|single family| +25538|AAAAAAAACMDGAAAA|807|Fourth |Wy|Suite 290|New Hope|Jackson County|WI|59431|United States|-6|apartment| +25539|AAAAAAAADMDGAAAA|308|Lakeview South|Boulevard|Suite R|The Meadows|Prowers County|CO|80026|United States|-7|single family| +25540|AAAAAAAAEMDGAAAA|926|Main |Court|Suite 260|Enterprise|Grady County|OK|71757|United States|-6|single family| +25541|AAAAAAAAFMDGAAAA|519|Lake |Blvd|Suite 220|Riverdale|Fairfax County|VA|29391|United States|-5|single family| +25542|AAAAAAAAGMDGAAAA|473|3rd 3rd|Ct.|Suite 160|Marion|Chatham County|NC|20399|United States|-5|single family| +25543|AAAAAAAAHMDGAAAA|64|Walnut Sixth|Dr.|Suite 450|Cedar Grove|Coweta County|GA|30411|United States|-5|single family| +25544|AAAAAAAAIMDGAAAA|250|Ridge Hickory|Parkway|Suite D|Union Hill|Calhoun County|WV|27746|United States|-5|single family| +25545|AAAAAAAAJMDGAAAA|797|Main First|Ln|Suite H|Summit|Beaver County|PA|10499|United States|-5|single family| +25546|AAAAAAAAKMDGAAAA|593|Second Center|Road||Delmar||OK|||-6|condo| +25547|AAAAAAAALMDGAAAA|20|Maple |Avenue|Suite S|Summit|Burke County|ND|50499|United States|-6|condo| +25548|AAAAAAAAMMDGAAAA|43|7th 10th|Ave|Suite 250|Sunnyside|Dane County|WI|51952|United States|-6|apartment| +25549|AAAAAAAANMDGAAAA|878|Main |Wy|Suite 330|Hopewell|Hickman County|TN|30587|United States|-5|condo| +25550|AAAAAAAAOMDGAAAA|799|Lake 4th|Blvd|Suite 90|Buena Vista|Knox County|IL|65752|United States|-6|single family| +25551|AAAAAAAAPMDGAAAA|886|Hickory |Road|Suite 90|Brunswick|Woodruff County|AR|74642|United States|-6|single family| +25552|AAAAAAAAANDGAAAA|492|Willow College|Blvd|Suite Y|Concord|Preston County|WV|24107|United States|-5|single family| +25553|AAAAAAAABNDGAAAA|543|Miller |Way|Suite 140|Greenwood|Lawrence County|IN|48828|United States|-5|condo| +25554|AAAAAAAACNDGAAAA|683|Jackson 12th|Ct.|Suite T|Sutton|Chattooga County|GA|35413|United States|-5|condo| +25555|AAAAAAAADNDGAAAA|540|Adams |Court|Suite D|Ashland|Lapeer County|MI|44244|United States|-5|apartment| +25556|AAAAAAAAENDGAAAA|528|13th |Circle|Suite 120|Cedar Grove|Hamilton County|FL|30411|United States|-5|apartment| +25557|AAAAAAAAFNDGAAAA|989|6th 2nd|RD|Suite T|Riverdale|Clinton County|PA|19391|United States|-5|condo| +25558|AAAAAAAAGNDGAAAA|810|Jackson Green|Circle|Suite 280|Plainview|Macon County|TN|33683|United States|-6|single family| +25559|AAAAAAAAHNDGAAAA|892|Smith Laurel|ST|Suite 80|Oakdale|Hinds County|MS|59584|United States|-6|single family| +25560|AAAAAAAAINDGAAAA|545|Williams |Ct.|Suite 420|Farmington|Calhoun County|MS|59145|United States|-6|single family| +25561|AAAAAAAAJNDGAAAA|247|Smith |Blvd|Suite 100|Bethel|Emmons County|ND|55281|United States|-6|condo| +25562|AAAAAAAAKNDGAAAA|81|7th Pine|Lane|Suite 330|Cedar Grove|Macoupin County|IL|60411|United States|-6|apartment| +25563|AAAAAAAALNDGAAAA|811|Highland |Circle|Suite 340|Greenfield|Eagle County|CO|85038|United States|-7|apartment| +25564|AAAAAAAAMNDGAAAA||Woodland ||||Cheatham County||32293|United States|-5|condo| +25565|AAAAAAAANNDGAAAA|805|Valley |Court|Suite L|Maple Grove|Breathitt County|KY|48252|United States|-6|single family| +25566|AAAAAAAAONDGAAAA|457|Jackson 5th|Blvd|Suite R|Georgetown|Alpena County|MI|47057|United States|-5|single family| +25567|AAAAAAAAPNDGAAAA|569|Lake |Cir.|Suite P|Centerville|Queens County|NY|10059|United States|-5|condo| +25568|AAAAAAAAAODGAAAA|734|Madison Highland|Way|Suite 240|Forest Hills|Monmouth County|NJ|09837|United States|-5|condo| +25569|AAAAAAAABODGAAAA|491|15th |Circle|Suite N|Bethel|Prowers County|CO|85281|United States|-7|single family| +25570|AAAAAAAACODGAAAA|339|Willow Hill|Street|Suite B|Hopewell|Daggett County|UT|80587|United States|-7|apartment| +25571|AAAAAAAADODGAAAA|69|Woodland Sycamore|Ct.|Suite 210|Lebanon|Hill County|MT|62898|United States|-7|apartment| +25572|AAAAAAAAEODGAAAA|227|Madison Meadow|Circle|Suite A|Five Forks|Teton County|MT|62293|United States|-7|single family| +25573|AAAAAAAAFODGAAAA|349|Fourth |Parkway|Suite J|Riceville|Cheshire County|NH|06467|United States|-5|apartment| +25574|AAAAAAAAGODGAAAA|205|12th Lincoln|Parkway|Suite K|Oakland|Perkins County|SD|59843|United States|-7|apartment| +25575|AAAAAAAAHODGAAAA|569|Meadow |Street|Suite 140|Valley View|Union Parish|LA|75124|United States|-6|apartment| +25576|AAAAAAAAIODGAAAA|827||Circle|||Brantley County||39101|United States||condo| +25577|AAAAAAAAJODGAAAA|165|Davis |Avenue|Suite A|Edgewood|Hooker County|NE|60069|United States|-7|apartment| +25578|AAAAAAAAKODGAAAA|495|Meadow First|Wy|Suite M|Wildwood|Weston County|WY|86871|United States|-7|apartment| +25579|AAAAAAAALODGAAAA|192|Madison Walnut|Boulevard|Suite 210|Greenville|Buckingham County|VA|21387|United States|-5|apartment| +25580|AAAAAAAAMODGAAAA|443|Walnut |Ave|Suite 390|Shiloh|Washoe County|NV|89275|United States|-8|apartment| +25581|AAAAAAAANODGAAAA|66|Lincoln Miller|Ct.|Suite Y|Deerfield|Fisher County|TX|79840|United States|-6|apartment| +25582|AAAAAAAAOODGAAAA|810|Valley |Pkwy|Suite 370|Wilson|Curry County|NM|86971|United States|-7|apartment| +25583|AAAAAAAAPODGAAAA|154|2nd |Ln|Suite E|Lakewood|Somerset County|NJ|09477|United States|-5|single family| +25584|AAAAAAAAAPDGAAAA|435|Tenth South|Parkway|Suite X|Union|Sierra County|NM|88721|United States|-7|apartment| +25585|AAAAAAAABPDGAAAA|827|Maple |Ln|Suite 430|Pleasant Grove|Henry County|IL|64136|United States|-6|single family| +25586|AAAAAAAACPDGAAAA|907|Hill |RD|Suite I|Oak Grove|Montgomery County|MO|68370|United States|-6|apartment| +25587|AAAAAAAADPDGAAAA|904|Jefferson |Ln|Suite 300|Fairview|Alleghany County|VA|25709|United States|-5|condo| +25588|AAAAAAAAEPDGAAAA|676|Center Williams|Street|Suite 290|Riverdale|Plumas County|CA|99391|United States|-8|single family| +25589|AAAAAAAAFPDGAAAA|565|Pine Pine|Boulevard|Suite J|Spring Valley|Hawaii County|HI|96060|United States|-10|condo| +25590|AAAAAAAAGPDGAAAA||||Suite 190||Atoka County||70587|||condo| +25591|AAAAAAAAHPDGAAAA|984|Miller Sunset|Court|Suite J|Georgetown|Buena Vista County|IA|57057|United States|-6|single family| +25592|AAAAAAAAIPDGAAAA|268|Fifth Johnson|Ln|Suite 320|Milan|Sabine County|TX|76697|United States|-6|condo| +25593|AAAAAAAAJPDGAAAA|35|Birch Center|ST|Suite M|Mount Zion|Martin County|MN|58054|United States|-6|single family| +25594|AAAAAAAAKPDGAAAA|377|Cedar Jackson|Street|Suite 350|Glendale|Carson County|TX|73951|United States|-6|condo| +25595|AAAAAAAALPDGAAAA|601|Lakeview |Blvd|Suite 430|Bethel|Forrest County|MS|55281|United States|-6|single family| +25596|AAAAAAAAMPDGAAAA|188|6th |Boulevard|Suite I|Centerville|Jones County|SD|50059|United States|-7|condo| +25597|AAAAAAAANPDGAAAA|304|Valley |||Forest Hills||OH||United States||single family| +25598|AAAAAAAAOPDGAAAA||Hill Dogwood|||New Hope||||United States|-6|single family| +25599|AAAAAAAAPPDGAAAA|688|Miller |Ln|Suite B|Harmony|Brantley County|GA|35804|United States|-5|condo| +25600|AAAAAAAAAAEGAAAA||||||Judith Basin County||69583||-7|| +25601|AAAAAAAABAEGAAAA|556|Seventh Spruce|Circle|Suite F|Newtown|Scott County|IA|51749|United States|-6|apartment| +25602|AAAAAAAACAEGAAAA||Elevnth View|Ave||Highland||||United States||apartment| +25603|AAAAAAAADAEGAAAA|210|River |Ct.|Suite K|Brentwood|Lynn County|TX|74188|United States|-6|single family| +25604|AAAAAAAAEAEGAAAA|343|11th |Court|Suite 440|Mount Pleasant|Keith County|NE|61933|United States|-7|single family| +25605|AAAAAAAAFAEGAAAA|633|8th Cherry|Street|Suite W|Shady Grove|Independence County|AR|72812|United States|-6|single family| +25606|AAAAAAAAGAEGAAAA|401||Circle||||AR||United States|-6|single family| +25607|AAAAAAAAHAEGAAAA|91|View |Wy|Suite V|Antioch|Pierce County|WI|58605|United States|-6|apartment| +25608|AAAAAAAAIAEGAAAA|49|Lake |Circle|Suite S|Springfield|Elko County|NV|89303|United States|-8|single family| +25609|AAAAAAAAJAEGAAAA|700|Second Fourteenth|Boulevard|Suite R|Shady Grove|Martin County|IN|42812|United States|-5|apartment| +25610|AAAAAAAAKAEGAAAA|436|First 7th|Dr.|Suite X|Mount Pleasant|Greene County|VA|21933|United States|-5|single family| +25611|AAAAAAAALAEGAAAA|323|7th 7th|Ave|Suite 50|Highland|Johnson County|AR|79454|United States|-6|apartment| +25612|AAAAAAAAMAEGAAAA|239|Church |Blvd|Suite K|Summit|Woods County|OK|70499|United States|-6|apartment| +25613|AAAAAAAANAEGAAAA|1|Forest |Court|Suite 250|Hopewell|Boone County|NE|60587|United States|-6|single family| +25614|AAAAAAAAOAEGAAAA|654|Madison |ST|Suite 300|Woodlawn|Haywood County|NC|24098|United States|-5|condo| +25615|AAAAAAAAPAEGAAAA|111|Hill Main|Lane|Suite W|Pleasant Valley|Lincoln County|OK|72477|United States|-6|apartment| +25616|AAAAAAAAABEGAAAA|255|Poplar |RD|Suite 90|Pleasant Hill|Lake of the Woods County|MN|53604|United States|-6|single family| +25617|AAAAAAAABBEGAAAA|585|Ridge 14th|Street|Suite 250|Highland Park|Surry County|VA|26534|United States|-5|single family| +25618|AAAAAAAACBEGAAAA|526|Fifth |Road|Suite U|Springdale|Hardin County|TX|78883|United States|-6|apartment| +25619|AAAAAAAADBEGAAAA|58|Elm |Lane|Suite 110|Cumberland|Lemhi County|ID|88971|United States|-7|condo| +25620|AAAAAAAAEBEGAAAA|594|10th |Blvd|Suite C|Summit|Washington County|MO|60499|United States|-6|single family| +25621|AAAAAAAAFBEGAAAA|768|11th |Circle|Suite U|Springdale|Berkeley County|SC|28883|United States|-5|apartment| +25622|AAAAAAAAGBEGAAAA|816|Franklin 3rd|Road|Suite 230|Mount Zion|Phillips County|CO|88054|United States|-7|apartment| +25623|AAAAAAAAHBEGAAAA|800|Green Elm|Dr.|Suite 430|Valley View|Pickens County|GA|35124|United States|-5|apartment| +25624|AAAAAAAAIBEGAAAA|855|Mill |Drive|Suite 340|Arlington|Brown County|OH|46557|United States|-5|single family| +25625|AAAAAAAAJBEGAAAA|655|Elm |Wy|Suite 70|Newport|Schuyler County|IL|61521|United States|-6|condo| +25626|AAAAAAAAKBEGAAAA|702|Locust |Avenue|Suite 150|Blanchard|Wabasha County|MN|55985|United States|-6|condo| +25627|AAAAAAAALBEGAAAA|395|Lakeview |Court|Suite W|Centerville|Martin County|NC|20059|United States|-5|condo| +25628|AAAAAAAAMBEGAAAA|858|2nd |ST|Suite N|Woodlawn|O-Brien County|IA|54098|United States|-6|single family| +25629|AAAAAAAANBEGAAAA|768|Lee Oak|Road|Suite Y|Buena Vista|Franklin County|NC|25752|United States|-5|condo| +25630|AAAAAAAAOBEGAAAA|774|Wilson 7th|Road|Suite N|Ashland|Pike County|PA|14244|United States|-5|apartment| +25631|AAAAAAAAPBEGAAAA|189|River |Wy|Suite 130|Spring Valley|Kewaunee County|WI|56060|United States|-6|apartment| +25632|AAAAAAAAACEGAAAA|265|Lake Davis|Cir.|Suite M|White Oak|Huntington County|IN|46668|United States|-5|apartment| +25633|AAAAAAAABCEGAAAA|508|Center Park|Lane|Suite W|Greenville|Lamb County|TX|71387|United States|-6|single family| +25634|AAAAAAAACCEGAAAA|597|3rd 9th|Avenue|Suite B|Highland|Broadwater County|MT|69454|United States|-7|apartment| +25635|AAAAAAAADCEGAAAA|924|Maple |Road|Suite 130|White Oak|Sweet Grass County|MT|66668|United States|-7|single family| +25636|AAAAAAAAECEGAAAA|990|Locust |Pkwy|Suite C|Mount Pleasant|Chouteau County|MT|61933|United States|-7|apartment| +25637|AAAAAAAAFCEGAAAA|335|8th |Court|Suite 190|Jackson|Citrus County|FL|39583|United States|-5|apartment| +25638|AAAAAAAAGCEGAAAA|772|Locust |Court|Suite 340|Midway|Cherokee County|TX|71904|United States|-6|single family| +25639|AAAAAAAAHCEGAAAA|814|Lincoln 6th|Way|Suite P|Riverview|Worth County|GA|39003|United States|-5|single family| +25640|AAAAAAAAICEGAAAA|527|Willow |Wy|Suite 30|Cedar Grove|Mitchell County|KS|60411|United States|-6|single family| +25641|AAAAAAAAJCEGAAAA||||||Hutchinson County|TX|74244|||| +25642|AAAAAAAAKCEGAAAA|469|Sycamore Sunset|Street|Suite A|Lakewood|Wyandotte County|KS|68877|United States|-6|single family| +25643|AAAAAAAALCEGAAAA|163|Spring Highland|Boulevard|Suite 310|Highland Park|Calhoun County|FL|36534|United States|-5|single family| +25644|AAAAAAAAMCEGAAAA|877|Adams |RD|Suite L|Concord|Lake County|IN|44107|United States|-5|single family| +25645|AAAAAAAANCEGAAAA|943|Maple |Court|Suite 380|Mountain View|Prairie County|AR|74466|United States|-6|apartment| +25646|AAAAAAAAOCEGAAAA|234|Chestnut Seventh|RD|Suite W|Fairfield|McDonough County|IL|66192|United States|-6|condo| +25647|AAAAAAAAPCEGAAAA|372|Fourth Oak|Wy|Suite G|Spring Hill|Haywood County|NC|26787|United States|-5|apartment| +25648|AAAAAAAAADEGAAAA|998|East |Pkwy|Suite A|Church Hill|Cabarrus County|NC|23790|United States|-5|single family| +25649|AAAAAAAABDEGAAAA|208|4th |ST|Suite 90|Mount Olive|Henry County|TN|38059|United States|-5|apartment| +25650|AAAAAAAACDEGAAAA|57|Hill Sycamore|Cir.|Suite F|Lakewood|Aroostook County|ME|09477|United States|-5|single family| +25651|AAAAAAAADDEGAAAA|881|6th |Parkway|Suite N|Woodland|Becker County|MN|54854|United States|-6|single family| +25652|AAAAAAAAEDEGAAAA|150|14th Park|Ct.|Suite Y|Antioch|Burt County|NE|68605|United States|-6|condo| +25653|AAAAAAAAFDEGAAAA|607|4th |ST|Suite C|Jerome|Bourbon County|KS|69920|United States|-6|condo| +25654|AAAAAAAAGDEGAAAA|467|Miller |Road|Suite 0|Summit|Graham County|NC|20499|United States|-5|single family| +25655|AAAAAAAAHDEGAAAA|994|Birch |ST|Suite R|Greenville|Cherokee County|KS|61387|United States|-6|apartment| +25656|AAAAAAAAIDEGAAAA|556|Walnut Park|Parkway|Suite 470|Bridgeport|New Haven County|CT|06417|United States|-5|single family| +25657|AAAAAAAAJDEGAAAA|261|Miller |Dr.|||Bradford County|PA||United States||single family| +25658|AAAAAAAAKDEGAAAA|590|Davis Wilson|Parkway|Suite S|Shiloh|Carter County|OK|79275|United States|-6|single family| +25659|AAAAAAAALDEGAAAA|939|Smith Walnut|Cir.|Suite 490|Plainview|Emmet County|MI|43683|United States|-5|condo| +25660|AAAAAAAAMDEGAAAA|951|Johnson Lincoln|Dr.|Suite 240|Highland|Seneca County|NY|19454|United States|-5|apartment| +25661|AAAAAAAANDEGAAAA|293|Lakeview |Way|Suite X|Red Hill|Vernon County|WI|54338|United States|-6|single family| +25662|AAAAAAAAODEGAAAA|585|10th West|Road|Suite W|Stringtown|Placer County|CA|90162|United States|-8|single family| +25663|AAAAAAAAPDEGAAAA|932|3rd |Drive|Suite 110|Antioch|Miami County|OH|48605|United States|-5|condo| +25664|AAAAAAAAAEEGAAAA|31|Jackson |Drive|Suite V|Clifton|Habersham County|GA|38014|United States|-5|apartment| +25665|AAAAAAAABEEGAAAA|302|Locust ||||Virginia Beach city||26192||-5|| +25666|AAAAAAAACEEGAAAA|504|Highland Spruce|RD|Suite O|Glenwood|Santa Clara County|CA|93511|United States|-8|apartment| +25667|AAAAAAAADEEGAAAA|978|Spring Main|Lane|Suite 70|Stringtown|Loving County|TX|70162|United States|-6|condo| +25668|AAAAAAAAEEEGAAAA|332|View Main|Ct.|Suite M|Fairfield|Mercer County|WV|26192|United States|-5|apartment| +25669|AAAAAAAAFEEGAAAA|979|2nd Hillcrest|Ave|Suite 180|Brownsville|Oregon County|MO|69310|United States|-6|single family| +25670|AAAAAAAAGEEGAAAA|53|Hickory |Street|Suite G|Pleasant Valley|Decatur County|IA|52477|United States|-6|apartment| +25671|AAAAAAAAHEEGAAAA|638|Franklin |Cir.|Suite 400|Pine Grove|Marshall County|OK|74593|United States|-6|single family| +25672|AAAAAAAAIEEGAAAA|294|Cherry Poplar|Pkwy|Suite 60|Spring Hill|Pike County|AL|36787|United States|-6|condo| +25673|AAAAAAAAJEEGAAAA|883|Ridge |Circle|Suite 460|Florence|Traill County|ND|53394|United States|-6|apartment| +25674|AAAAAAAAKEEGAAAA|680|Oak Forest|Ln|Suite Q|Forestville|Childress County|TX|73027|United States|-6|condo| +25675|AAAAAAAALEEGAAAA||||Suite 300|Pleasant Grove|Dare County|NC|24136|||single family| +25676|AAAAAAAAMEEGAAAA|41|Davis |Dr.|Suite A|Riverdale|El Paso County|CO|89391|United States|-7|condo| +25677|AAAAAAAANEEGAAAA|53|Sunset 6th|Lane|Suite 410|Union|Dodge County|MN|58721|United States|-6|apartment| +25678|AAAAAAAAOEEGAAAA|699|Lincoln 5th|Ct.|Suite B|Buena Vista|Dearborn County|IN|45752|United States|-5|apartment| +25679|AAAAAAAAPEEGAAAA|57|Hickory |Parkway|Suite 270|Wildwood|Carroll County|MS|56871|United States|-6|apartment| +25680|AAAAAAAAAFEGAAAA|183|Washington |Lane|Suite 140|Unionville|Stokes County|NC|21711|United States|-5|apartment| +25681|AAAAAAAABFEGAAAA|510|Adams |Drive|Suite K|Jamestown|King George County|VA|26867|United States|-5|condo| +25682|AAAAAAAACFEGAAAA|840|Washington |Lane|Suite 480|Spring Valley|Wabash County|IN|46060|United States|-5|apartment| +25683|AAAAAAAADFEGAAAA|878|Woodland Franklin|Court|Suite O|Riverview|Barton County|KS|69003|United States|-6|apartment| +25684|AAAAAAAAEFEGAAAA|778|Hickory |Wy|Suite 180|Belmont|Lincoln County|KY|40191|United States|-5|condo| +25685|AAAAAAAAFFEGAAAA|638|Woodland 5th|RD|Suite O|Five Forks|Clinton County|KY|42293|United States|-6|single family| +25686|AAAAAAAAGFEGAAAA|481|Hickory |Way|Suite C|Pine Grove|Pawnee County|NE|64593|United States|-7|condo| +25687|AAAAAAAAHFEGAAAA|788|River |Avenue|Suite L|Shiloh|Fremont County|WY|89275|United States|-7|apartment| +25688|AAAAAAAAIFEGAAAA|57|Lincoln |RD|Suite V|Farmington|Jackson County|TN|39145|United States|-5|apartment| +25689|AAAAAAAAJFEGAAAA|555|Locust |ST|Suite 150|Pleasant Grove|Polk County|MN|54136|United States|-6|apartment| +25690|AAAAAAAAKFEGAAAA|521|Miller Sunset|Blvd|Suite U|Edgewood|Clark County|IL|60069|United States|-6|condo| +25691|AAAAAAAALFEGAAAA|97|8th 3rd|Ln|Suite O|Oak Grove|Madison County|MO|68370|United States|-6|apartment| +25692|AAAAAAAAMFEGAAAA|143|Cherry 3rd|Court|Suite 210|Providence|Worth County|MO|66614|United States|-6|condo| +25693|AAAAAAAANFEGAAAA|591|Center |Ct.|Suite D|Northwood|Siskiyou County|CA|94104|United States|-8|single family| +25694|AAAAAAAAOFEGAAAA|831|Lake |Parkway|Suite 50|Oak Grove|Fayette County|IA|58370|United States|-6|single family| +25695|AAAAAAAAPFEGAAAA|425|Eigth |Ave|Suite 190|Summit|Tucker County|WV|20499|United States|-5|single family| +25696|AAAAAAAAAGEGAAAA|158|Miller Johnson|Street|Suite 370|Marion|Clay County|IA|50399|United States|-6|apartment| +25697|AAAAAAAABGEGAAAA|846|Oak |||Plainview||TX|73683|||condo| +25698|AAAAAAAACGEGAAAA|468|2nd |Court|Suite Y|Midway|Cass County|MN|51904|United States|-6|single family| +25699|AAAAAAAADGEGAAAA|572|Birch Cedar|Boulevard|Suite T|Dallas|Frederick County|MD|23628|United States|-5|single family| +25700|AAAAAAAAEGEGAAAA||Ninth ||Suite W||Callahan County||||-6|| +25701|AAAAAAAAFGEGAAAA|377|Pine Twelfth|Pkwy|Suite F|Florence|Saline County|IL|63394|United States|-6|condo| +25702|AAAAAAAAGGEGAAAA|1000|12th Park|Ln|Suite K|Hidden Valley|Nolan County|TX|75521|United States|-6|apartment| +25703|AAAAAAAAHGEGAAAA|398|Sunset |Avenue|Suite J|Friendship|Blaine County|ID|84536|United States|-7|single family| +25704|AAAAAAAAIGEGAAAA|855|2nd 4th|Cir.|Suite 0|Lakewood|Caribou County|ID|88877|United States|-7|apartment| +25705|AAAAAAAAJGEGAAAA|614|West |Court|Suite 400|Edgewood|Toole County|MT|60069|United States|-7|apartment| +25706|AAAAAAAAKGEGAAAA|722|Center |Boulevard|Suite 120|Clinton|Beaver County|OK|78222|United States|-6|single family| +25707|AAAAAAAALGEGAAAA|516|Johnson |Lane|Suite 130|Kingston|Clay County|KS|64975|United States|-6|single family| +25708|AAAAAAAAMGEGAAAA|80|9th |Blvd|Suite 10|Wright|Whitfield County|GA|32814|United States|-5|condo| +25709|AAAAAAAANGEGAAAA|511|Maple 4th|Blvd||Bethel||||||condo| +25710|AAAAAAAAOGEGAAAA|544|1st |Pkwy|Suite I|Woodland|Prince George County|MD|24854|United States|-5|single family| +25711|AAAAAAAAPGEGAAAA|864|Wilson Railroad|Circle|Suite U|Pleasant Hill|Logan County|AR|73604|United States|-6|single family| +25712|AAAAAAAAAHEGAAAA|299|Walnut Pine|Street|Suite 170|Providence|Menard County|TX|76614|United States|-6|condo| +25713|AAAAAAAABHEGAAAA|590|Lee |Ct.|Suite 60|Pleasant Grove|Brooks County|TX|74136|United States|-6|apartment| +25714|AAAAAAAACHEGAAAA|815|Walnut Fourth|Drive|Suite E|Woodland|Nelson County|KY|44854|United States|-5|single family| +25715|AAAAAAAADHEGAAAA|106|Sycamore Ridge|Cir.|Suite O|Franklin|Franklin County|KY|49101|United States|-6|condo| +25716|AAAAAAAAEHEGAAAA|551|Lincoln |Parkway|Suite 50|Centerville|Athens County|OH|40059|United States|-5|single family| +25717|AAAAAAAAFHEGAAAA|48|Sixth |Wy|Suite O|Greenfield|Pawnee County|NE|65038|United States|-7|condo| +25718|AAAAAAAAGHEGAAAA|915|Birch Sixth|Avenue|Suite 300|Spring Hill|O-Brien County|IA|56787|United States|-6|apartment| +25719|AAAAAAAAHHEGAAAA|112|9th Walnut|Parkway|Suite 290|Oak Grove|Redwood County|MN|58370|United States|-6|apartment| +25720|AAAAAAAAIHEGAAAA|910|Central |ST|Suite 140|Sheffield|Jones County|TX|76896|United States|-6|single family| +25721|AAAAAAAAJHEGAAAA|982||Street|||Burke County|NC|24098|United States|-5|condo| +25722|AAAAAAAAKHEGAAAA|635|Ninth |Street|Suite M|Summit|Hart County|GA|30499|United States|-5|single family| +25723|AAAAAAAALHEGAAAA|||||Farmersville|Austin County|||||| +25724|AAAAAAAAMHEGAAAA|798|Maple |Ln|Suite F|Friendship|Otter Tail County|MN|54536|United States|-6|apartment| +25725|AAAAAAAANHEGAAAA|201|South |RD|Suite X|Antioch|Taney County|MO|68605|United States|-6|single family| +25726|AAAAAAAAOHEGAAAA|74|Washington Maple|Circle|Suite 390|Harmony|Haakon County|SD|55804|United States|-7|condo| +25727|AAAAAAAAPHEGAAAA|965|Third Park|Drive|Suite F|Mount Olive|Kenosha County|WI|58059|United States|-6|apartment| +25728|AAAAAAAAAIEGAAAA|33|Ridge Forest|RD|Suite E|Georgetown|Solano County|CA|97057|United States|-8|apartment| +25729|AAAAAAAABIEGAAAA|602|East Johnson|Road|Suite X|Oak Hill|Highlands County|FL|37838|United States|-5|condo| +25730|AAAAAAAACIEGAAAA|748|Miller |Blvd|Suite P|Enterprise|Northwest Arctic Borough|AK|91757|United States|-9|condo| +25731|AAAAAAAADIEGAAAA|759|11th East|Court|Suite O|Bunker Hill|Douglas County|IL|60150|United States|-6|single family| +25732|AAAAAAAAEIEGAAAA|978|7th |RD|Suite 160|Fairview|Champaign County|OH|45709|United States|-5|apartment| +25733|AAAAAAAAFIEGAAAA|465|Willow |Drive|Suite 270|Unionville|Lee County|IA|51711|United States|-6|apartment| +25734|AAAAAAAAGIEGAAAA|477|Church |Ct.|Suite M|Waterloo|Jackson County|SD|51675|United States|-7|apartment| +25735|AAAAAAAAHIEGAAAA|656|Pine |Ct.|Suite G|Pleasant Grove|Mackinac County|MI|44136|United States|-5|single family| +25736|AAAAAAAAIIEGAAAA|128|Maple 1st|Way|Suite 300|Caledonia|Washington County|AR|77411|United States|-6|condo| +25737|AAAAAAAAJIEGAAAA|536|Davis |Court|Suite Y|Jackson|Glasscock County|TX|79583|United States|-6|single family| +25738|AAAAAAAAKIEGAAAA|786|Washington 3rd|Parkway|Suite 210|Spring Hill|Bottineau County|ND|56787|United States|-6|single family| +25739|AAAAAAAALIEGAAAA|685|Ash Main|Parkway|Suite 180|Green Acres|Wilson County|TX|77683|United States|-6|single family| +25740|AAAAAAAAMIEGAAAA|120|First Main|Street|Suite 70|Shiloh|Elbert County|CO|89275|United States|-7|apartment| +25741|AAAAAAAANIEGAAAA|51|Pine |Avenue|Suite 170|Walnut Grove|Douglas County|MO|67752|United States|-6|apartment| +25742|AAAAAAAAOIEGAAAA|153|Cherry Oak|Road|Suite B|Highland|Defiance County|OH|49454|United States|-5|apartment| +25743|AAAAAAAAPIEGAAAA|918|1st |Wy|Suite A|Union City|Hancock County|WV|28087|United States|-5|apartment| +25744|AAAAAAAAAJEGAAAA|445|7th Hickory|Boulevard|Suite 360|Greenwood|Leavenworth County|KS|68828|United States|-6|single family| +25745|AAAAAAAABJEGAAAA|710|4th |Lane|Suite 220|Marion|Champaign County|OH|40399|United States|-5|condo| +25746|AAAAAAAACJEGAAAA|339|3rd |ST|Suite J|Lebanon|Kings County|CA|92898|United States|-8|apartment| +25747|AAAAAAAADJEGAAAA|441|14th |Street|Suite U|Greenville|Piatt County|IL|61387|United States|-6|single family| +25748|AAAAAAAAEJEGAAAA|316|Twelfth 14th|Road|Suite U|Georgetown|McNairy County|TN|37057|United States|-6|condo| +25749|AAAAAAAAFJEGAAAA|279|North 1st|Road|Suite 470|Pleasant Hill|Vinton County|OH|43604|United States|-5|single family| +25750|AAAAAAAAGJEGAAAA|941|Pine Franklin|Lane|Suite 150|Hamilton|Clark County|SD|52808|United States|-6|single family| +25751|AAAAAAAAHJEGAAAA|556|5th |Dr.|Suite 30|Deerfield|Franklin County|NC|29840|United States|-5|apartment| +25752|AAAAAAAAIJEGAAAA|14|Ridge |Boulevard|Suite U|Saint Clair|Washita County|OK|75294|United States|-6|apartment| +25753|AAAAAAAAJJEGAAAA|976|Davis |Wy|Suite 180|Belmont|San Juan County|NM|80191|United States|-7|condo| +25754|AAAAAAAAKJEGAAAA|428|Railroad Maple|Circle|Suite 390|Brownsville|Wallace County|KS|69310|United States|-6|apartment| +25755|AAAAAAAALJEGAAAA|987|Oak Johnson|Avenue|Suite C|Enterprise|McCreary County|KY|41757|United States|-5|apartment| +25756|AAAAAAAAMJEGAAAA|847|6th |Street|Suite V|Lakeside|Hancock County|WV|29532|United States|-5|single family| +25757|AAAAAAAANJEGAAAA|6|3rd |Circle|Suite J|Salem|Grand County|UT|88048|United States|-7|single family| +25758|AAAAAAAAOJEGAAAA|7|Hillcrest Meadow|Avenue|Suite X|Bethel|Newton County|TX|75281|United States|-6|apartment| +25759|AAAAAAAAPJEGAAAA|990|Dogwood Oak|Court|Suite K|White Oak|Lee County|AL|36668|United States|-6|apartment| +25760|AAAAAAAAAKEGAAAA|644|Lakeview |Drive|Suite 280|Oak Hill|Marshall County|AL|37838|United States|-6|condo| +25761|AAAAAAAABKEGAAAA|456|6th Hill|Pkwy|Suite 380|Lakeside|Butler County|IA|59532|United States|-6|apartment| +25762|AAAAAAAACKEGAAAA|605|Willow Chestnut|Drive|Suite 260|Appleton|Beadle County|SD|54240|United States|-6|condo| +25763|AAAAAAAADKEGAAAA|811|1st |Wy|Suite T|Buena Vista|Marion County|MS|55752|United States|-6|apartment| +25764|AAAAAAAAEKEGAAAA|797|Davis |Street|Suite 390|Newtown|Morris County|KS|61749|United States|-6|single family| +25765|AAAAAAAAFKEGAAAA|45|Maple Elm|ST|Suite N|Shiloh|Clinton County|MI|49275|United States|-5|single family| +25766|AAAAAAAAGKEGAAAA|668|Main Mill|Way|Suite L|Glendale|Addison County|VT|04551|United States|-5|condo| +25767|AAAAAAAAHKEGAAAA|25|4th Green|Street|Suite 40|Oak Grove|Benzie County|MI|48370|United States|-5|single family| +25768|AAAAAAAAIKEGAAAA|805|First |Lane|Suite H|Arthur|Montgomery County|TN|35965|United States|-6|apartment| +25769|AAAAAAAAJKEGAAAA|540|Pine 15th|Blvd|Suite S|Farmersville|Van Buren County|IA|59305|United States|-6|apartment| +25770|AAAAAAAAKKEGAAAA|266|Forest |Ct.|Suite X|Jackson|Tishomingo County|MS|59583|United States|-6|apartment| +25771|AAAAAAAALKEGAAAA|582|Cherry |Boulevard|Suite Y|Walnut Grove|Baca County|CO|87752|United States|-7|single family| +25772|AAAAAAAAMKEGAAAA|101|Ridge |RD|Suite 450|Oak Hill|Whitman County|WA|97838|United States|-8|apartment| +25773|AAAAAAAANKEGAAAA|586|Smith |Wy|Suite L|Hopewell|Oklahoma County|OK|70587|United States|-6|condo| +25774|AAAAAAAAOKEGAAAA|459|4th |Wy|Suite K|Cedar Grove|Northampton County|NC|20411|United States|-5|condo| +25775|AAAAAAAAPKEGAAAA|319|Locust |Wy|Suite V|Walnut Grove|Clinton County|NY|17752|United States|-5|single family| +25776|AAAAAAAAALEGAAAA|857|Mill 14th|Drive|Suite Y|Walnut Grove|Gilliam County|OR|97752|United States|-8|single family| +25777|AAAAAAAABLEGAAAA|288|Johnson |Blvd|Suite L|Crystal|Roger Mills County|OK|75258|United States|-6|condo| +25778|AAAAAAAACLEGAAAA|299|6th |Road|Suite 400|Howell|Morgan County|GA|34854|United States|-5|single family| +25779|AAAAAAAADLEGAAAA|3|6th Fourth|Ave|Suite S|Riverview|Young County|TX|79003|United States|-6|apartment| +25780|AAAAAAAAELEGAAAA|65|Pine Oak|Ave|Suite 370|Pleasant Hill|Stewart County|TN|33604|United States|-6|condo| +25781|AAAAAAAAFLEGAAAA|315|South Park|Pkwy|Suite A|Concord|Spink County|SD|54107|United States|-7|single family| +25782|AAAAAAAAGLEGAAAA|318|3rd |Drive|Suite 20|Riverdale|Lee County|AL|39391|United States|-6|single family| +25783|AAAAAAAAHLEGAAAA|378|North |Court|Suite 100|Oak Hill|Crow Wing County|MN|57838|United States|-6|single family| +25784|AAAAAAAAILEGAAAA|942|11th |Drive|Suite M|Concord|Navarro County|TX|74107|United States|-6|apartment| +25785|AAAAAAAAJLEGAAAA|778|Elm |Circle|Suite 390|Oak Ridge|Perry County|MS|58371|United States|-6|condo| +25786|AAAAAAAAKLEGAAAA|419|Fifth |RD|Suite P|Newtown|Elko County|NV|81749|United States|-8|apartment| +25787|AAAAAAAALLEGAAAA|336|Pine Hickory|Cir.|Suite 130|White Rock|Cook County|MN|56944|United States|-6|condo| +25788|AAAAAAAAMLEGAAAA|448|5th 8th|Blvd|Suite M|Fairfield|Okeechobee County|FL|36192|United States|-5|single family| +25789|AAAAAAAANLEGAAAA|559|Chestnut |Blvd|Suite X|Marion|Cook County|IL|60399|United States|-6|apartment| +25790|AAAAAAAAOLEGAAAA|289|Locust |Ct.|Suite 300|Mount Zion|Frederick County|VA|28054|United States|-5|apartment| +25791|AAAAAAAAPLEGAAAA|150|Poplar |RD|Suite Y|Mount Vernon|Mohave County|AZ|88482|United States|-7|condo| +25792|AAAAAAAAAMEGAAAA|973|Highland |Road|Suite 370|Union|Carson City|NV|88721|United States|-8|single family| +25793|AAAAAAAABMEGAAAA|335|Highland Pine|Pkwy|Suite X|Greenville|Brown County|IL|61387|United States|-6|single family| +25794|AAAAAAAACMEGAAAA|343|7th |Parkway|Suite 320|Five Forks|Brunswick County|VA|22293|United States|-5|single family| +25795|AAAAAAAADMEGAAAA|940|Church |Avenue|Suite 130|Woodland|Morgan County|OH|44854|United States|-5|apartment| +25796|AAAAAAAAEMEGAAAA|382|Main |Street|Suite T|Mountain View|Weakley County|TN|34466|United States|-6|apartment| +25797|AAAAAAAAFMEGAAAA|234|Davis Pine|Ct.|Suite 380|Woodville|Humphreys County|TN|34289|United States|-5|apartment| +25798|AAAAAAAAGMEGAAAA|449|Madison Lincoln|Dr.|Suite 170|Four Points|Carroll County|MS|51216|United States|-6|condo| +25799|AAAAAAAAHMEGAAAA|320|Cherry |Avenue|Suite R|Five Forks|Park County|WY|82293|United States|-7|condo| +25800|AAAAAAAAIMEGAAAA|183|Main |Circle|Suite Y|Oakland|San Joaquin County|CA|99843|United States|-8|single family| +25801|AAAAAAAAJMEGAAAA|295|||Suite I|Hopewell||||United States||| +25802|AAAAAAAAKMEGAAAA|122|Oak 8th|Circle|Suite C|Red Hill|Wayne County|NE|64338|United States|-7|single family| +25803|AAAAAAAALMEGAAAA|626|Spruce 5th|Boulevard|Suite 490|Arlington|Crawford County|PA|16557|United States|-5|single family| +25804|AAAAAAAAMMEGAAAA|199|East Center|Drive|Suite 240|Sulphur Springs|Newton County|MO|68354|United States|-6|condo| +25805|AAAAAAAANMEGAAAA|896|Ridge Miller|Ave|Suite 330|Enterprise|Worth County|IA|51757|United States|-6|single family| +25806|AAAAAAAAOMEGAAAA|733|4th |Way|Suite 200|Centerville|Hopewell city|VA|20059|United States|-5|condo| +25807|AAAAAAAAPMEGAAAA|615|Third Fifth|Street|Suite 100|River Oaks|Lassen County|CA|98075|United States|-8|apartment| +25808|AAAAAAAAANEGAAAA|685|Oak |Cir.|Suite N|Green Acres|Beaver County|PA|17683|United States|-5|apartment| +25809|AAAAAAAABNEGAAAA|703|Lake Sycamore|Ct.|Suite A|Shady Grove|Magoffin County|KY|42812|United States|-5|condo| +25810|AAAAAAAACNEGAAAA|955|Washington |Parkway|Suite M|Belmont|Orange County|CA|90191|United States|-8|condo| +25811|AAAAAAAADNEGAAAA||Adams Smith||Suite L|Hopewell|Lee County|VA|||-5|| +25812|AAAAAAAAENEGAAAA|706|Seventh |RD|Suite 190|Jackson|Morgan County|MO|69583|United States|-6|apartment| +25813|AAAAAAAAFNEGAAAA|204|Lincoln |Wy|Suite J|Liberty|Cass County|IA|53451|United States|-6|apartment| +25814|AAAAAAAAGNEGAAAA|236|Franklin |Court|Suite M|Greenwood|Dakota County|MN|58828|United States|-6|apartment| +25815|AAAAAAAAHNEGAAAA|600|Adams |Drive|Suite 300|Cedar Grove|Macomb County|MI|40411|United States|-5|condo| +25816|AAAAAAAAINEGAAAA|501|Maple |Court|Suite X|Ashland|Nash County|NC|24244|United States|-5|condo| +25817|AAAAAAAAJNEGAAAA|507|Maple |Road|Suite Q|Payne|Macon County|GA|36134|United States|-5|apartment| +25818|AAAAAAAAKNEGAAAA|493|11th |Parkway|Suite A|Newport|Bernalillo County|NM|81521|United States|-7|single family| +25819|AAAAAAAALNEGAAAA|530|12th |Cir.|Suite W|Brownsville|Cross County|AR|79310|United States|-6|single family| +25820|AAAAAAAAMNEGAAAA|301|Spring Washington|Boulevard|Suite 210|Union|Gilchrist County|FL|38721|United States|-5|condo| +25821|AAAAAAAANNEGAAAA|584|First |Ave|Suite 120|Shiloh|Douglas County|OR|99275|United States|-8|single family| +25822|AAAAAAAAONEGAAAA|378|Sunset |RD|Suite T|Riverside|Bolivar County|MS|59231|United States|-6|single family| +25823|AAAAAAAAPNEGAAAA|864|Central Valley|Wy|Suite Q|Pine Grove|Warren County|NY|14593|United States|-5|single family| +25824|AAAAAAAAAOEGAAAA|225|||Suite 470|Stringtown|Monroe County|OH||||| +25825|AAAAAAAABOEGAAAA|243|Adams Center|Drive|Suite H|Oakdale|Denali Borough|AK|99584|United States|-9|single family| +25826|AAAAAAAACOEGAAAA|243|Seventh |Ave|Suite A|Bloomingdale|Spotsylvania County|VA|21824|United States|-5|condo| +25827|AAAAAAAADOEGAAAA|463|Dogwood |Ct.|Suite 160|Smith|Sandoval County|NM|87317|United States|-7|condo| +25828|AAAAAAAAEOEGAAAA|481|Franklin Washington|Ln|Suite 140|Greendale|Greene County|TN|39343|United States|-5|apartment| +25829|AAAAAAAAFOEGAAAA|964|8th Maple|Cir.|Suite X|Greenville|Fairfield County|CT|01987|United States|-5|apartment| +25830|AAAAAAAAGOEGAAAA|322|2nd |Court|Suite A|Jamestown|Sauk County|WI|56867|United States|-6|condo| +25831|AAAAAAAAHOEGAAAA|747|Lake 9th|ST|Suite J|Mount Olive|Fresno County|CA|98059|United States|-8|condo| +25832|AAAAAAAAIOEGAAAA|9|Adams |Dr.|Suite 180|Prosperity|Anderson County|KS|69089|United States|-6|single family| +25833|AAAAAAAAJOEGAAAA|582|Church Hillcrest|Road|Suite 20|Riverside|Crook County|OR|99231|United States|-8|apartment| +25834|AAAAAAAAKOEGAAAA|669|Highland 15th|ST|Suite A|Concord|Clark County|WI|54107|United States|-6|apartment| +25835|AAAAAAAALOEGAAAA|70|Dogwood |Way|Suite 450|Jamestown|McCreary County|KY|46867|United States|-5|single family| +25836|AAAAAAAAMOEGAAAA|312|14th |Drive|Suite X|Unionville|Tioga County|NY|11711|United States|-5|single family| +25837|AAAAAAAANOEGAAAA|516|South |Lane|Suite P|Hamilton|Hickman County|TN|32808|United States|-5|condo| +25838|AAAAAAAAOOEGAAAA|982|Fifth |Way|Suite V|Jackson|Middlesex County|MA|09583|United States|-5|single family| +25839|AAAAAAAAPOEGAAAA|633|Third |Blvd|Suite Y|Milo|Harrison County|IA|50116|United States|-6|condo| +25840|AAAAAAAAAPEGAAAA|778|Lincoln Spring|Drive|Suite 390|Riverview|Cuyahoga County|OH|49003|United States|-5|apartment| +25841|AAAAAAAABPEGAAAA|299|Washington |Avenue|Suite 50|Mount Olive|Webb County|TX|78059|United States|-6|single family| +25842|AAAAAAAACPEGAAAA|460|Maple 3rd|Blvd|Suite 190|Byron|Titus County|TX|75597|United States|-6|single family| +25843|AAAAAAAADPEGAAAA|614|2nd Hill|Ave|Suite F|Greenwood|McLennan County|TX|78828|United States|-6|apartment| +25844|AAAAAAAAEPEGAAAA|||||Caledonia||||United States||apartment| +25845|AAAAAAAAFPEGAAAA|618|1st Adams|RD|Suite D|Brownsville|Callahan County|TX|79310|United States|-6|condo| +25846|AAAAAAAAGPEGAAAA|243|8th |Street|Suite 190|Midway|Blaine County|NE|61904|United States|-6|apartment| +25847|AAAAAAAAHPEGAAAA|535|Cedar |Way|Suite E|Payne|Lamar County|TX|76134|United States|-6|apartment| +25848|AAAAAAAAIPEGAAAA|961|11th Ridge|Boulevard|Suite 310|Edgewood|Fisher County|TX|70069|United States|-6|condo| +25849|AAAAAAAAJPEGAAAA|812|Chestnut |Ct.|Suite T|Hamilton|Muskogee County|OK|72808|United States|-6|condo| +25850|AAAAAAAAKPEGAAAA|507|Locust 15th|Lane|Suite G|Riverview|Clarendon County|SC|29003|United States|-5|apartment| +25851|AAAAAAAALPEGAAAA|396|Center 4th|Ln|Suite 290|Greenville|Kidder County|ND|51387|United States|-6|single family| +25852|AAAAAAAAMPEGAAAA|494|Johnson |Pkwy|Suite 300|Fairfield|Scott County|KY|46192|United States|-5|single family| +25853|AAAAAAAANPEGAAAA|145|2nd |Avenue|Suite K|Pine Grove|Somerset County|ME|05193|United States|-5|single family| +25854|AAAAAAAAOPEGAAAA|899|Pine |Road|Suite 20|Riverside|McHenry County|IL|69231|United States|-6|single family| +25855|AAAAAAAAPPEGAAAA|247|Maple |Avenue|Suite 260|Hamilton|Elbert County|GA|32808|United States|-5|single family| +25856|AAAAAAAAAAFGAAAA|983|15th |RD|Suite 310|Newtown|Coahoma County|MS|51749|United States|-6|apartment| +25857|AAAAAAAABAFGAAAA|956|Hickory College|ST|Suite J|Pleasant Valley|Edgecombe County|NC|22477|United States|-5|apartment| +25858|AAAAAAAACAFGAAAA|61||Avenue||White Oak|||36668|United States|-6|apartment| +25859|AAAAAAAADAFGAAAA|376|Third Adams|Cir.|Suite 310|Union Hill|Lehigh County|PA|17746|United States|-5|condo| +25860|AAAAAAAAEAFGAAAA|751|First 15th|Ct.|Suite K|Newtown|Seminole County|OK|71749|United States|-6|single family| +25861|AAAAAAAAFAFGAAAA|401|Cherry Lincoln|Boulevard|Suite L|Springfield|Taylor County|WV|29303|United States|-5|apartment| +25862|AAAAAAAAGAFGAAAA|598|3rd |Road|Suite 30|Jamestown|Stanton County|KS|66867|United States|-6|single family| +25863|AAAAAAAAHAFGAAAA|4|3rd |Ave|Suite J|Union|Teton County|WY|88721|United States|-7|single family| +25864|AAAAAAAAIAFGAAAA|865|View |RD|Suite 460|Franklin|Marshall County|MN|59101|United States|-6|condo| +25865|AAAAAAAAJAFGAAAA|858|2nd |Ave|Suite Q|Edgewood|Lenoir County|NC|20069|United States|-5|condo| +25866|AAAAAAAAKAFGAAAA|902|9th Fourth|Boulevard|Suite U|Oakdale|San Patricio County|TX|79584|United States|-6|apartment| +25867|AAAAAAAALAFGAAAA|364|7th Second|Lane|Suite W|Wildwood|Curry County|OR|96871|United States|-8|apartment| +25868|AAAAAAAAMAFGAAAA|407|Third |Boulevard|Suite 430|Springfield|Knox County|KY|49303|United States|-5|condo| +25869|AAAAAAAANAFGAAAA|996|West First|Way|Suite W|Smith|Tuscarawas County|OH|47317|United States|-5|single family| +25870|AAAAAAAAOAFGAAAA|513|Cedar |Pkwy|Suite S|Red Hill|Wilkes County|NC|24338|United States|-5|apartment| +25871|AAAAAAAAPAFGAAAA|506|7th Walnut|Street|Suite 210|Waterloo|Moultrie County|IL|61675|United States|-6|single family| +25872|AAAAAAAAABFGAAAA|49|Lincoln Jackson|Boulevard|Suite 220|Waterloo|Washtenaw County|MI|41675|United States|-5|single family| +25873|AAAAAAAABBFGAAAA|270|Main |Circle|Suite F|Pleasant Hill|Coryell County|TX|73604|United States|-6|condo| +25874|AAAAAAAACBFGAAAA|354|West Oak|Ave|Suite 220|Concord|Renville County|ND|54107|United States|-6|apartment| +25875|AAAAAAAADBFGAAAA|263|Oak Park|Dr.|Suite L|Glendale|Muskegon County|MI|43951|United States|-5|condo| +25876|AAAAAAAAEBFGAAAA|952|First Forest|Road|Suite 0|Waterloo|Colbert County|AL|31675|United States|-6|single family| +25877|AAAAAAAAFBFGAAAA|440|South |Street|Suite Y|Shannon|Rockland County|NY|14120|United States|-5|single family| +25878|AAAAAAAAGBFGAAAA|944|Lakeview |Lane|Suite 390|Phoenix|Buchanan County|IA|52276|United States|-6|single family| +25879|AAAAAAAAHBFGAAAA|121|Railroad |Lane|Suite O|Stratford|Wallace County|KS|66668|United States|-6|apartment| +25880|AAAAAAAAIBFGAAAA|242|Cedar |Dr.|Suite 190|Ashland|Washington County|MS|54244|United States|-6|condo| +25881|AAAAAAAAJBFGAAAA|170|Hickory |Parkway|Suite 220|Brownsville|Pope County|AR|79310|United States|-6|apartment| +25882|AAAAAAAAKBFGAAAA|273|2nd Washington|Court|Suite F|Providence|White County|AR|76614|United States|-6|single family| +25883|AAAAAAAALBFGAAAA|846|8th |Ct.|Suite L|Antioch|Scott County|IA|58605|United States|-6|single family| +25884|AAAAAAAAMBFGAAAA|552|Lakeview |Avenue|Suite 480||Whitley County|KY|42812|United States|-5|| +25885|AAAAAAAANBFGAAAA|911|5th |Court|Suite 250|Providence|Custer County|CO|86614|United States|-7|single family| +25886|AAAAAAAAOBFGAAAA|974|West Davis|ST|Suite D|Spring Hill|Allendale County|SC|26787|United States|-5|condo| +25887|AAAAAAAAPBFGAAAA|82|Center |Ct.|Suite M|Highland Park|Orange County|NC|26534|United States|-5|apartment| +25888|AAAAAAAAACFGAAAA|825|Seventh |Lane|Suite J|Walnut Grove|Solano County|CA|97752|United States|-8|single family| +25889|AAAAAAAABCFGAAAA|566|Smith |Wy|Suite 230|New Hope|Bryan County|OK|79431|United States|-6|apartment| +25890|AAAAAAAACCFGAAAA|249|Forest Park|Ln|Suite 170|Woodland|King George County|VA|24854|United States|-5|condo| +25891|AAAAAAAADCFGAAAA|590|Jackson |Avenue|Suite A|Pierce|Waupaca County|WI|53360|United States|-6|condo| +25892|AAAAAAAAECFGAAAA|236|Central |Way|Suite B|Pleasant Grove|Rusk County|WI|54136|United States|-6|single family| +25893|AAAAAAAAFCFGAAAA|755|Maple |Parkway|Suite V|Union|Ripley County|IN|48721|United States|-5|single family| +25894|AAAAAAAAGCFGAAAA|167|Pine |Lane|Suite Y|Richfield|Floyd County|GA|36196|United States|-5|apartment| +25895|AAAAAAAAHCFGAAAA|923|Seventh Ash|Ln|Suite 470|Oak Hill|Pershing County|NV|87838|United States|-8|condo| +25896|AAAAAAAAICFGAAAA|559|College 1st|Drive|Suite 220|Arlington|Blount County|TN|36557|United States|-5|apartment| +25897|AAAAAAAAJCFGAAAA|527|8th |Lane|Suite O|Brownsville|Cherokee County|SC|29310|United States|-5|single family| +25898|AAAAAAAAKCFGAAAA|885|Locust |Ct.|Suite 420|Lakewood|Blanco County|TX|78877|United States|-6|condo| +25899|AAAAAAAALCFGAAAA|641|North Washington|Wy|Suite Q|Friendship|Kittson County|MN|54536|United States|-6|apartment| +25900|AAAAAAAAMCFGAAAA|545|Hickory |Court|Suite M|Liberty|Winchester city|VA|23451|United States|-5|condo| +25901|AAAAAAAANCFGAAAA|864|2nd |Dr.|Suite 30|Stringtown|Crawford County|AR|70162|United States|-6|single family| +25902|AAAAAAAAOCFGAAAA|354|Johnson 8th|Cir.|Suite C|Hillcrest|Pocahontas County|IA|53003|United States|-6|single family| +25903|AAAAAAAAPCFGAAAA|503|Third |RD|Suite 330|Georgetown|Burt County|NE|67057|United States|-6|single family| +25904|AAAAAAAAADFGAAAA|811|Main |Street|Suite 310|Lakewood|Sterling County|TX|78877|United States|-6|single family| +25905|AAAAAAAABDFGAAAA|154|Third |Dr.|Suite N|Shiloh|Coleman County|TX|79275|United States|-6|condo| +25906|AAAAAAAACDFGAAAA|538|Cedar Cedar|Wy|Suite 420|Unionville|Russell County|VA|21711|United States|-5|apartment| +25907|AAAAAAAADDFGAAAA|757|Cedar |Blvd|Suite 220|Sunnyside|Green County|WI|51952|United States|-6|apartment| +25908|AAAAAAAAEDFGAAAA|766|Oak |Blvd|Suite 370|Franklin|Meigs County|OH|49101|United States|-5|single family| +25909|AAAAAAAAFDFGAAAA|338|Jefferson View|RD|Suite 490|Midway|Worth County|GA|31904|United States|-5|single family| +25910|AAAAAAAAGDFGAAAA|539|Birch Lincoln|Lane|Suite V|Liberty|Mecklenburg County|VA|23451|United States|-5|condo| +25911|AAAAAAAAHDFGAAAA|427|Main Fourth|Street|Suite 390|Riverdale|Lawrence County|SD|59391|United States|-7|single family| +25912|AAAAAAAAIDFGAAAA|764|Hillcrest |Cir.|Suite 450|Wright|San Augustine County|TX|72814|United States|-6|condo| +25913|AAAAAAAAJDFGAAAA|583|4th |Ln|Suite 450|Red Hill|Emery County|UT|84338|United States|-7|single family| +25914|AAAAAAAAKDFGAAAA|712|7th |Ct.|Suite 110|Florence|Sharp County|AR|73394|United States|-6|apartment| +25915|AAAAAAAALDFGAAAA|428|Main |Ct.|Suite S|Mount Zion|Elk County|KS|68054|United States|-6|apartment| +25916|AAAAAAAAMDFGAAAA|318|East 6th||Suite R|Riverview|||49003|United States||| +25917|AAAAAAAANDFGAAAA|914|8th Park|Court|Suite 80|Calhoun|Simpson County|MS|56909|United States|-6|apartment| +25918|AAAAAAAAODFGAAAA|572|Elm |Lane|Suite F|Edgewood|Scott County|KS|60069|United States|-6|single family| +25919|AAAAAAAAPDFGAAAA|652|Oak Second|Dr.|Suite 310|Farmington|Monroe County|NY|19145|United States|-5|condo| +25920|AAAAAAAAAEFGAAAA|518|Miller Park|Wy|Suite 480|Oak Hill|Crawford County|IA|57838|United States|-6|single family| +25921|AAAAAAAABEFGAAAA|285|Elm Park|Wy|Suite J|Oakwood|Fountain County|IN|40169|United States|-5|condo| +25922|AAAAAAAACEFGAAAA|550|Washington |Court|Suite V|Salem|Haskell County|OK|78048|United States|-6|apartment| +25923|AAAAAAAADEFGAAAA|94|Fifth Sycamore|Cir.|Suite E|Somerville|Tripp County|SD|57783|United States|-7|single family| +25924|AAAAAAAAEEFGAAAA|218|||Suite D||Richmond County|||United States||apartment| +25925|AAAAAAAAFEFGAAAA|899|View |Road|Suite 200|Midway|Buena Vista city|VA|21904|United States|-5|apartment| +25926|AAAAAAAAGEFGAAAA|646|Main |Avenue|Suite 30|Fairview|Pickens County|AL|35709|United States|-6|apartment| +25927|AAAAAAAAHEFGAAAA|367|Cherry |Cir.|Suite M|Globe|Burleson County|TX|79614|United States|-6|single family| +25928|AAAAAAAAIEFGAAAA|187|Lakeview 14th|Drive|Suite N|Oak Grove|Jefferson County|WV|28370|United States|-5|single family| +25929|AAAAAAAAJEFGAAAA|101|Maple |RD|Suite B|Glendale|Henderson County|IL|63951|United States|-6|condo| +25930|AAAAAAAAKEFGAAAA|146|3rd |Avenue|Suite I|Lincoln|Kitsap County|WA|91289|United States|-8|single family| +25931|AAAAAAAALEFGAAAA|68|4th |Circle|Suite B|New Hope|Bedford city|VA|29431|United States|-5|apartment| +25932|AAAAAAAAMEFGAAAA|594|Hickory |Drive|Suite V|Walnut Grove|Golden Valley County|ND|57752|United States|-6|condo| +25933|AAAAAAAANEFGAAAA|49|Adams Broadway|Ave|Suite 430|Shiloh|Christian County|MO|69275|United States|-6|apartment| +25934|AAAAAAAAOEFGAAAA|826|Park River|Lane|Suite G|Spring Hill|Franklin city|VA|26787|United States|-5|single family| +25935|AAAAAAAAPEFGAAAA|843|Lee |Pkwy|Suite T|Mount Olive|Randolph County|AR|78059|United States|-6|apartment| +25936|AAAAAAAAAFFGAAAA|103|10th |Ln|Suite R|Marion|Taylor County|FL|30399|United States|-5|condo| +25937|AAAAAAAABFFGAAAA|833|Valley 10th|Court|Suite 120|Brownsville|Levy County|FL|39310|United States|-5|single family| +25938|AAAAAAAACFFGAAAA|60|Broadway Lake|Road|Suite J|New Hope|Breckinridge County|KY|49431|United States|-6|condo| +25939|AAAAAAAADFFGAAAA|2|Franklin |Boulevard|Suite 260|Maple Grove|White County|GA|38252|United States|-5|single family| +25940|AAAAAAAAEFFGAAAA|102|Park |Ave|Suite 200|Mount Pleasant|Clinton County|MI|41933|United States|-5|apartment| +25941|AAAAAAAAFFFGAAAA|337|North West|Court|Suite G|Harmony|Lavaca County|TX|75804|United States|-6|condo| +25942|AAAAAAAAGFFGAAAA|615|River First|Blvd|Suite 90|Kimball|Hardeman County|TX|73595|United States|-6|apartment| +25943|AAAAAAAAHFFGAAAA|607|Elm Williams|Avenue|Suite U|Glendale|Stephenson County|IL|63951|United States|-6|condo| +25944|AAAAAAAAIFFGAAAA|755|Church |Parkway|Suite 460|Crossroads|Bullock County|AL|30534|United States|-6|apartment| +25945|AAAAAAAAJFFGAAAA|45|South Lincoln|Wy|Suite 280|Jamestown|Meigs County|OH|46867|United States|-5|condo| +25946|AAAAAAAAKFFGAAAA|707|Park Hickory|Court|Suite I|Oak Grove|Antelope County|NE|68370|United States|-6|single family| +25947|AAAAAAAALFFGAAAA|650|Hillcrest |Drive|Suite 260|Fairfield|Somerset County|NJ|06792|United States|-5|apartment| +25948|AAAAAAAAMFFGAAAA|4|College Lincoln|Ave|Suite 300|Midway|Sevier County|UT|81904|United States|-7|condo| +25949|AAAAAAAANFFGAAAA|666|Seventh 13th|Dr.|Suite B|Glendale|Meigs County|TN|33951|United States|-6|apartment| +25950|AAAAAAAAOFFGAAAA|6|5th Johnson|Avenue|Suite O|Jackson|Russell County|KS|69583|United States|-6|single family| +25951|AAAAAAAAPFFGAAAA|549|Highland |Blvd|Suite 420|Marion|Yalobusha County|MS|50399|United States|-6|single family| +25952|AAAAAAAAAGFGAAAA|72|7th 1st|Circle|Suite U|Wilson|Sanpete County|UT|86971|United States|-7|apartment| +25953|AAAAAAAABGFGAAAA||Park |ST|Suite L|Edgewood|Carroll County|MS||||| +25954|AAAAAAAACGFGAAAA|936|5th West|ST|Suite P|Kingston|Tippah County|MS|54975|United States|-6|apartment| +25955|AAAAAAAADGFGAAAA|766|Davis |Avenue|Suite C|Brownsville|Middlesex County|CT|09910|United States|-5|condo| +25956|AAAAAAAAEGFGAAAA|262|North Forest|Circle|Suite P|New Hope|Benton County|MN|59431|United States|-6|condo| +25957|AAAAAAAAFGFGAAAA|11|Johnson |Ct.|Suite C|Red Hill|Seneca County|OH|44338|United States|-5|single family| +25958|AAAAAAAAGGFGAAAA|748|6th Jackson|Cir.|Suite S|Midway|Midland County|TX|71904|United States|-6|condo| +25959|AAAAAAAAHGFGAAAA|101|Eigth Johnson|Ct.|Suite 170|Wilson|Knox County|MO|66971|United States|-6|condo| +25960|AAAAAAAAIGFGAAAA||||Suite C||Kay County|||United States|-6|| +25961|AAAAAAAAJGFGAAAA|148|11th 3rd|RD|Suite B|Farmington|Morrison County|MN|59145|United States|-6|apartment| +25962|AAAAAAAAKGFGAAAA|33|4th |Drive|Suite L|Highland|Mitchell County|NC|29454|United States|-5|single family| +25963|AAAAAAAALGFGAAAA|123|Lincoln |Avenue|Suite 290|Plainview|Republic County|KS|63683|United States|-6|condo| +25964|AAAAAAAAMGFGAAAA|653|Meadow Ash|Lane|Suite Y|New Hope|Carroll County|NH|09431|United States|-5|single family| +25965|AAAAAAAANGFGAAAA|694|Eigth College|Ct.|Suite C|Spring Hill|Mitchell County|GA|36787|United States|-5|condo| +25966|AAAAAAAAOGFGAAAA|759|Franklin |Boulevard|Suite 250|Brownsville|Christian County|IL|69310|United States|-6|single family| +25967|AAAAAAAAPGFGAAAA|544|Lincoln Hickory|Ct.|Suite D|Antioch|Columbia County|WA|98605|United States|-8|apartment| +25968|AAAAAAAAAHFGAAAA|722|Park East|Street|Suite A|Belmont|Tom Green County|TX|70191|United States|-6|condo| +25969|AAAAAAAABHFGAAAA|180|Valley Jefferson|ST|Suite 380|Westport|Knox County|OH|41802|United States|-5|single family| +25970|AAAAAAAACHFGAAAA|660|8th |Court|Suite F|Bethel|Johnson County|KS|65281|United States|-6|condo| +25971|AAAAAAAADHFGAAAA|482|Poplar 4th|RD|Suite W|Pine Grove|Hertford County|NC|24593|United States|-5|apartment| +25972|AAAAAAAAEHFGAAAA|206|Fourth South|Parkway|Suite 370|Unionville|Montgomery County|IL|61711|United States|-6|condo| +25973|AAAAAAAAFHFGAAAA|727|View |Ave|Suite N|Maple Grove|Sarasota County|FL|38252|United States|-5|apartment| +25974|AAAAAAAAGHFGAAAA|683|Poplar |Drive|Suite 160|Woodland|Evans County|GA|34854|United States|-5|single family| +25975|AAAAAAAAHHFGAAAA|285|Laurel Wilson|Avenue|Suite Q|Marion|Cassia County|ID|80399|United States|-7|single family| +25976|AAAAAAAAIHFGAAAA|157|Williams |Drive|Suite Y|Green Acres|Kent County|DE|17683|United States|-5|apartment| +25977|AAAAAAAAJHFGAAAA|820|Cherry 3rd|Pkwy|Suite U|Franklin|Meade County|KY|49101|United States|-5|single family| +25978|AAAAAAAAKHFGAAAA|594|View |Cir.|Suite 340|Lebanon|Adams County|CO|82898|United States|-7|condo| +25979|AAAAAAAALHFGAAAA|149|Maple |ST|Suite 350|Fairview|Cherokee County|GA|35709|United States|-5|apartment| +25980|AAAAAAAAMHFGAAAA|107|Adams 7th|ST|Suite 490|Springdale|Ballard County|KY|48883|United States|-6|condo| +25981|AAAAAAAANHFGAAAA|460|8th |Pkwy|Suite 330|Ashland|Jackson County|MS|54244|United States|-6|single family| +25982|AAAAAAAAOHFGAAAA|234|Oak |Pkwy|Suite 290|Bridgeport|Grant County|NE|65817|United States|-6|single family| +25983|AAAAAAAAPHFGAAAA|417|Lincoln |Boulevard|Suite 300|Antioch|Pulaski County|KY|48605|United States|-5|single family| +25984|AAAAAAAAAIFGAAAA|492|Sixth |Parkway|Suite J|Sawyer|Santa Cruz County|AZ|86045|United States|-7|apartment| +25985|AAAAAAAABIFGAAAA|242|11th |Court|Suite 460|Woodland|Laclede County|MO|64854|United States|-6|condo| +25986|AAAAAAAACIFGAAAA|556|Cedar |Way|Suite 300|Providence|Fayette County|TX|76614|United States|-6|apartment| +25987|AAAAAAAADIFGAAAA|772|East |Street|Suite 90|Highland Park|Livingston Parish|LA|76534|United States|-6|apartment| +25988|AAAAAAAAEIFGAAAA|184|Maple |ST|Suite J|Unionville|Taylor County|TX|71711|United States|-6|single family| +25989|AAAAAAAAFIFGAAAA|289|Park Oak|RD|Suite 320|Lakeside|Cedar County|NE|69532|United States|-6|single family| +25990|AAAAAAAAGIFGAAAA|854|Franklin Oak|Court|Suite G|Green Acres|Irwin County|GA|37683|United States|-5|single family| +25991|AAAAAAAAHIFGAAAA|129||ST|||Liberty County|||United States|-5|| +25992|AAAAAAAAIIFGAAAA|371|5th 6th|Street|Suite 330|Riverview|Green County|KY|49003|United States|-6|condo| +25993|AAAAAAAAJIFGAAAA|985|Park 10th|Way|Suite 180|Farmington|Potter County|PA|19145|United States|-5|condo| +25994|AAAAAAAAKIFGAAAA|234|Hickory |Boulevard|Suite 150|Shiloh|Lewis County|MO|69275|United States|-6|condo| +25995|AAAAAAAALIFGAAAA|153|Lee |Road|Suite X|Spring Hill|Ringgold County|IA|56787|United States|-6|condo| +25996|AAAAAAAAMIFGAAAA|120|Davis |Cir.|Suite N|Newport|Schleicher County|TX|71521|United States|-6|apartment| +25997|AAAAAAAANIFGAAAA|185|Hickory Miller|Avenue|Suite 330|Greenwood|Putnam County|NY|18828|United States|-5|apartment| +25998|AAAAAAAAOIFGAAAA|533|Railroad |RD|Suite S|Kingston|Morgan County|WV|24975|United States|-5|condo| +25999|AAAAAAAAPIFGAAAA|657|1st Hickory|Blvd|Suite 370|Antioch|Cass County|TX|78605|United States|-6|single family| +26000|AAAAAAAAAJFGAAAA|593|Laurel |Way|Suite R|Pleasant Valley|Amite County|MS|52477|United States|-6|single family| +26001|AAAAAAAABJFGAAAA|156|Park Pine|Ave|Suite 190|Edgewood|Madison County|MO|60069|United States|-6|condo| +26002|AAAAAAAACJFGAAAA|628|10th |ST|Suite 370|Mount Olive|De Witt County|IL|68059|United States|-6|apartment| +26003|AAAAAAAADJFGAAAA|611|Madison Eigth|RD|Suite W|Greenwood|Pleasants County|WV|28828|United States|-5|condo| +26004|AAAAAAAAEJFGAAAA|384|Franklin |Street|Suite L|Deerfield|Stanley County|SD|59840|United States|-7|condo| +26005|AAAAAAAAFJFGAAAA|850|Johnson |Pkwy|Suite 460|California|Cooke County|TX|70141|United States|-6|condo| +26006|AAAAAAAAGJFGAAAA|303|East |Blvd|Suite Q|Lakeside|Smyth County|VA|29532|United States|-5|single family| +26007|AAAAAAAAHJFGAAAA|682|8th Park|Circle|Suite A|Waterloo|Ashland County|WI|51675|United States|-6|condo| +26008|AAAAAAAAIJFGAAAA|856|Maple |Blvd|Suite H|Newport|Clay County|WV|21521|United States|-5|single family| +26009|AAAAAAAAJJFGAAAA|861|Hill |Ave|Suite 250|Spring Hill|Castro County|TX|76787|United States|-6|condo| +26010|AAAAAAAAKJFGAAAA|749|Cherry |Pkwy|Suite L|Red Hill|Emmet County|IA|54338|United States|-6|condo| +26011|AAAAAAAALJFGAAAA|319|Maple |Ave|Suite 270|Union|Crawford County|IA|58721|United States|-6|apartment| +26012|AAAAAAAAMJFGAAAA|714|10th Walnut|Street|Suite 110|Camelot|Cherokee County|SC|27354|United States|-5|condo| +26013|AAAAAAAANJFGAAAA|447|Church Fifth|Cir.|Suite D|White Oak|Richland County|MT|66668|United States|-7|condo| +26014|AAAAAAAAOJFGAAAA|680|13th |Pkwy|Suite 220|Oak Ridge|Boyd County|NE|68371|United States|-6|single family| +26015|AAAAAAAAPJFGAAAA|580|South |Court|Suite 240|Ridgeville|Butler County|PA|19306|United States|-5|condo| +26016|AAAAAAAAAKFGAAAA|310|3rd Church|Wy|Suite I|Valley View|Orange County|NY|15124|United States|-5|single family| +26017|AAAAAAAABKFGAAAA|484|Tenth |Pkwy|Suite R|Shore Acres|Horry County|SC|22724|United States|-5|apartment| +26018|AAAAAAAACKFGAAAA|13|Oak Washington|Dr.|Suite 110|Antioch|Pike County|IL|68605|United States|-6|apartment| +26019|AAAAAAAADKFGAAAA|659|Madison |Ave|Suite 200|Shady Grove|Dillon County|SC|22812|United States|-5|apartment| +26020|AAAAAAAAEKFGAAAA|682|6th |Court|Suite E|Centerville|Wheeler County|OR|90059|United States|-8|single family| +26021|AAAAAAAAFKFGAAAA|545|Washington Lake|Blvd|Suite 150|Highland|Catoosa County|GA|39454|United States|-5|single family| +26022|AAAAAAAAGKFGAAAA|683|Second |Court|Suite P|Belmont|Caldwell Parish|LA|70191|United States|-6|single family| +26023|AAAAAAAAHKFGAAAA|||Lane|Suite A|Mountain View|Lincoln County|KY|44466|United States||condo| +26024|AAAAAAAAIKFGAAAA|64|8th Spring|Drive|Suite U|Green Acres|Washington County|AL|37683|United States|-6|apartment| +26025|AAAAAAAAJKFGAAAA|939|14th |Way|Suite 480|Pine Grove|Knox County|ME|05193|United States|-5|apartment| +26026|AAAAAAAAKKFGAAAA|233|Tenth |Circle|Suite 230|Berea|Hutchinson County|SD|53804|United States|-7|single family| +26027|AAAAAAAALKFGAAAA|746|12th |Wy|Suite V|Greenville|Gentry County|MO|61387|United States|-6|condo| +26028|AAAAAAAAMKFGAAAA|702|Walnut |Pkwy|Suite S|Clifton|Taylor County|KY|48014|United States|-5|condo| +26029|AAAAAAAANKFGAAAA|585|Meadow North|Avenue|Suite A|Greenfield|Fannin County|GA|35038|United States|-5|single family| +26030|AAAAAAAAOKFGAAAA|719|Lakeview Jefferson|Ln|Suite M|Sulphur Springs|Cabarrus County|NC|28354|United States|-5|apartment| +26031|AAAAAAAAPKFGAAAA|877|Park Park|Ct.|Suite 0|Maple Grove|Meigs County|TN|38252|United States|-6|condo| +26032|AAAAAAAAALFGAAAA|839|Lincoln |Drive|Suite A|Oakland|Chaves County|NM|89843|United States|-7|single family| +26033|AAAAAAAABLFGAAAA|133|Church |Street|Suite R|Bunker Hill|Hardin County|KY|40150|United States|-6|condo| +26034|AAAAAAAACLFGAAAA|138|Main 10th|Drive|Suite R|Jackson|Graham County|KS|69583|United States|-6|condo| +26035|AAAAAAAADLFGAAAA||||Suite 440|Clifton|Okeechobee County|FL|38014||-5|| +26036|AAAAAAAAELFGAAAA|138|Davis |Parkway|Suite 70|Belmont|Rock County|NE|60191|United States|-7|apartment| +26037|AAAAAAAAFLFGAAAA|213|6th Park|Ave|Suite S|Highland Park|Asotin County|WA|96534|United States|-8|condo| +26038|AAAAAAAAGLFGAAAA|615|Third Maple|Blvd|Suite 90|Woodland|Cherokee County|OK|74854|United States|-6|condo| +26039|AAAAAAAAHLFGAAAA|577|Church |Ave|Suite 110|Bethel|White County|GA|35281|United States|-5|single family| +26040|AAAAAAAAILFGAAAA||Forest ||||||56534|||| +26041|AAAAAAAAJLFGAAAA|217|Ridge 3rd|Wy|Suite M|Edgewood|Nodaway County|MO|60069|United States|-6|apartment| +26042|AAAAAAAAKLFGAAAA|280|7th Washington|ST|Suite A|Plainview|Oxford County|ME|04283|United States|-5|condo| +26043|AAAAAAAALLFGAAAA|716|4th |RD|Suite 440|Maple Grove|Tazewell County|VA|28252|United States|-5|single family| +26044|AAAAAAAAMLFGAAAA|553|Park |Court|Suite 420|Pleasant Hill|Candler County|GA|33604|United States|-5|apartment| +26045|AAAAAAAANLFGAAAA|777|14th 6th|Ct.|Suite 410|Sulphur Springs|Calhoun County|FL|38354|United States|-5|condo| +26046|AAAAAAAAOLFGAAAA|280|Smith North|Cir.|Suite J|Springfield|Johnston County|OK|79303|United States|-6|condo| +26047|AAAAAAAAPLFGAAAA|436|3rd Meadow|Ln|Suite 280|Pine Grove|Noble County|OH|44593|United States|-5|condo| +26048|AAAAAAAAAMFGAAAA|548|Maple |Pkwy|Suite O|Springdale|Conecuh County|AL|38883|United States|-6|apartment| +26049|AAAAAAAABMFGAAAA|470|Birch Hickory|Parkway|Suite 350|Hopewell|Bear Lake County|ID|80587|United States|-7|apartment| +26050|AAAAAAAACMFGAAAA|331|Wilson East|Ct.|Suite 170|Highland Park|Calhoun County|WV|26534|United States|-5|apartment| +26051|AAAAAAAADMFGAAAA|729|West Sunset|Cir.|Suite 130|Fairfield|Dale County|AL|36192|United States|-6|condo| +26052|AAAAAAAAEMFGAAAA|493|7th Elm|Blvd|Suite B|Lakeside|Dawson County|TX|79532|United States|-6|single family| +26053|AAAAAAAAFMFGAAAA|89|Park 3rd|Street|Suite A|Harper|Logan County|ND|55695|United States|-6|apartment| +26054|AAAAAAAAGMFGAAAA|753|Lake Lake|Boulevard|Suite 230|Georgetown|Ada County|ID|87057|United States|-7|single family| +26055|AAAAAAAAHMFGAAAA|75|6th |Pkwy|Suite 60|Macedonia|Nassau County|NY|11087|United States|-5|apartment| +26056|AAAAAAAAIMFGAAAA|351|Washington |Road|Suite 200|Brentwood|Mills County|TX|74188|United States|-6|single family| +26057|AAAAAAAAJMFGAAAA|809|Smith |Road|Suite I|Lakeview|Lincoln County|CO|88579|United States|-7|single family| +26058|AAAAAAAAKMFGAAAA|518|Meadow |Dr.|Suite 30|Spring Grove|Flagler County|FL|36719|United States|-5|condo| +26059|AAAAAAAALMFGAAAA|894|Lee Sixth|Pkwy|Suite E|Five Forks|Carbon County|WY|82293|United States|-7|apartment| +26060|AAAAAAAAMMFGAAAA|376|12th |Court|Suite 310|Forest Hills|Elbert County|CO|89237|United States|-7|apartment| +26061|AAAAAAAANMFGAAAA|36|College Walnut|Court|Suite 110|Sulphur Springs|Gallatin County|MT|68354|United States|-7|condo| +26062|AAAAAAAAOMFGAAAA|436|2nd |Pkwy|Suite 490|Brownsville|Sampson County|NC|29310|United States|-5|condo| +26063|AAAAAAAAPMFGAAAA|758|Main Elm|Avenue|Suite N|Helena|Racine County|WI|51649|United States|-6|single family| +26064|AAAAAAAAANFGAAAA|28|Johnson Johnson|Road|Suite 190|Midway|Schley County|GA|31904|United States|-5|single family| +26065|AAAAAAAABNFGAAAA|891|Maple Park|ST|Suite 90|Shiloh|Grundy County|MO|69275|United States|-6|apartment| +26066|AAAAAAAACNFGAAAA|303|Dogwood |ST|Suite H|Five Points|Natchitoches Parish|LA|76098|United States|-6|apartment| +26067|AAAAAAAADNFGAAAA|636|Washington |Avenue|Suite 350|Shiloh|Elkhart County|IN|49275|United States|-5|single family| +26068|AAAAAAAAENFGAAAA|685|Lakeview |Court|Suite B|Summerville|Northampton County|NC|22033|United States|-5|single family| +26069|AAAAAAAAFNFGAAAA|793|4th Cherry|Circle|Suite 470|Summit|Coffee County|AL|30499|United States|-6|single family| +26070|AAAAAAAAGNFGAAAA|726|Green |Wy|Suite 320|Plainview|Twin Falls County|ID|83683|United States|-7|apartment| +26071|AAAAAAAAHNFGAAAA|455|Woodland 5th|Boulevard|Suite 310|White Oak|Ottawa County|MI|46668|United States|-5|single family| +26072|AAAAAAAAINFGAAAA|775|Cedar Ridge|Avenue|Suite 210|Clifton|Cattaraugus County|NY|18014|United States|-5|apartment| +26073|AAAAAAAAJNFGAAAA|142|Hill |Parkway|Suite K|Mountain View|Barnwell County|SC|24466|United States|-5|apartment| +26074|AAAAAAAAKNFGAAAA|693|10th East|Street|Suite M|Spring Hill|Clark County|KS|66787|United States|-6|single family| +26075|AAAAAAAALNFGAAAA|484|Johnson |Parkway|Suite 140|Mount Zion|Stephens County|OK|78054|United States|-6|apartment| +26076|AAAAAAAAMNFGAAAA|30|Spring 14th|Wy|Suite W|Highland|Maverick County|TX|79454|United States|-6|apartment| +26077|AAAAAAAANNFGAAAA|73|View |RD|Suite 320|Newport|Nemaha County|KS|61521|United States|-6|condo| +26078|AAAAAAAAONFGAAAA|276|Ridge Spruce|Way|Suite 100|Marion|Big Horn County|MT|60399|United States|-7|single family| +26079|AAAAAAAAPNFGAAAA|502|15th View|Circle|Suite F|Mountain View|Lampasas County|TX|74466|United States|-6|apartment| +26080|AAAAAAAAAOFGAAAA|529|12th Main|Wy|Suite M|Riverside|Fluvanna County|VA|29231|United States|-5|apartment| +26081|AAAAAAAABOFGAAAA|762|Church Ridge|Court|Suite 260|Cedar Grove|Morris County|KS|60411|United States|-6|apartment| +26082|AAAAAAAACOFGAAAA|920|Hillcrest |Court|Suite I|Forest Hills|Caldwell Parish|LA|79237|United States|-6|apartment| +26083|AAAAAAAADOFGAAAA|819|Madison Adams|RD|Suite F|Fayetteville|Milwaukee County|WI|51732|United States|-6|single family| +26084|AAAAAAAAEOFGAAAA|691|1st Lake|Street|Suite L|Farmington|Ector County|TX|79145|United States|-6|single family| +26085|AAAAAAAAFOFGAAAA|670|South |Drive|Suite J|Pine Grove|Hardin County|TN|34593|United States|-5|condo| +26086|AAAAAAAAGOFGAAAA|192|Hickory |Street|Suite 0|Greenwood|Richmond County|NC|28828|United States|-5|apartment| +26087|AAAAAAAAHOFGAAAA|879|5th 10th|Parkway|Suite W|Crossroads|Millard County|UT|80534|United States|-7|single family| +26088|AAAAAAAAIOFGAAAA|493|Chestnut |Street|Suite 240|Rocky Point|Elbert County|CO|81209|United States|-7|condo| +26089|AAAAAAAAJOFGAAAA|274|Hillcrest Highland|Dr.|Suite 120|Lakeland|Coshocton County|OH|49376|United States|-5|single family| +26090|AAAAAAAAKOFGAAAA|945|Laurel Main|Ct.|Suite 160|Cedar Grove|Wayne County|NY|10411|United States|-5|single family| +26091|AAAAAAAALOFGAAAA|257|View Cedar|Pkwy|Suite 330|Franklin|Ketchikan Gateway Borough|AK|99101|United States|-9|condo| +26092|AAAAAAAAMOFGAAAA|564|Cedar |Pkwy|Suite 470|Wildwood|Stillwater County|MT|66871|United States|-7|condo| +26093|AAAAAAAANOFGAAAA|825|3rd |Parkway|Suite 190|Pleasant Valley|Hall County|TX|72477|United States|-6|single family| +26094|AAAAAAAAOOFGAAAA|556|West |Avenue|Suite 190|New Hope|Wabash County|IL|69431|United States|-6|apartment| +26095|AAAAAAAAPOFGAAAA|293|Railroad Locust|Pkwy|Suite 400|Spring Hill|Isle of Wight County|VA|26787|United States|-5|condo| +26096|AAAAAAAAAPFGAAAA|761|Davis |Street|Suite U|Salem|Roane County|TN|38048|United States|-6|single family| +26097|AAAAAAAABPFGAAAA|445|1st Jefferson|ST|Suite W|Newtown|Hand County|SD|51749|United States|-7|single family| +26098|AAAAAAAACPFGAAAA|305|3rd Ash|Parkway|Suite 60|Pine Grove|Assumption Parish|LA|74593|United States|-6|single family| +26099|AAAAAAAADPFGAAAA|436|2nd Forest|Way|Suite O|Harmony|Craven County|NC|25804|United States|-5|condo| +26100|AAAAAAAAEPFGAAAA|883|5th 1st|Blvd|Suite B|Antioch|Logan County|KY|48605|United States|-5|single family| +26101|AAAAAAAAFPFGAAAA|498|Maple 1st|Boulevard|Suite 320|Greenfield|Tripp County|SD|55038|United States|-7|single family| +26102|AAAAAAAAGPFGAAAA|634|Lincoln |Dr.|Suite 280|Mount Pleasant|Boone County|KY|41933|United States|-6|condo| +26103|AAAAAAAAHPFGAAAA|665|Main |Ct.|Suite H|Hamilton|DeWitt County|TX|72808|United States|-6|apartment| +26104|AAAAAAAAIPFGAAAA|500|East |Blvd|Suite A|Pleasant Hill|Childress County|TX|73604|United States|-6|condo| +26105|AAAAAAAAJPFGAAAA|416|Fourth Park|Blvd|Suite R|Summit|Flathead County|MT|60499|United States|-7|single family| +26106|AAAAAAAAKPFGAAAA|828|2nd Madison|Circle|Suite 390|Mount Pleasant|Alleghany County|NC|21933|United States|-5|condo| +26107|AAAAAAAALPFGAAAA|898|Oak 15th|Dr.|Suite 440|Sunnyside|Wagoner County|OK|71952|United States|-6|apartment| +26108|AAAAAAAAMPFGAAAA|345|Broadway |Blvd|Suite N|Union Hill|Marin County|CA|97746|United States|-8|single family| +26109|AAAAAAAANPFGAAAA|805|9th Willow|Circle|Suite 320|Pleasant Hill|Amador County|CA|93604|United States|-8|apartment| +26110|AAAAAAAAOPFGAAAA|932|Sycamore |ST|Suite Q|Stringtown|Lynchburg city|VA|20162|United States|-5|apartment| +26111|AAAAAAAAPPFGAAAA|552|Laurel |Blvd|Suite 240|Ashland|Berkeley County|SC|24244|United States|-5|condo| +26112|AAAAAAAAAAGGAAAA|504|8th Madison|Circle|Suite U|Sunnyside|Oceana County|MI|41952|United States|-5|apartment| +26113|AAAAAAAABAGGAAAA|415|11th Pine|Dr.|Suite E|Lakeview|Box Elder County|UT|88579|United States|-7|single family| +26114|AAAAAAAACAGGAAAA|699|Railroad Adams|Road|Suite 260|Georgetown|McLean County|ND|57057|United States|-6|single family| +26115|AAAAAAAADAGGAAAA|354|13th |Drive|Suite M|Green Acres|Floyd County|VA|27683|United States|-5|apartment| +26116|AAAAAAAAEAGGAAAA|713|Central ||Suite 290|Lincoln|Cottle County||71289|United States|-6|apartment| +26117|AAAAAAAAFAGGAAAA|581|Lincoln |Boulevard|Suite R|Paxton|Fayette County|AL|35669|United States|-6|condo| +26118|AAAAAAAAGAGGAAAA|374|Green 5th|Lane|Suite Y|Superior|Andrew County|MO|62562|United States|-6|apartment| +26119|AAAAAAAAHAGGAAAA|342|6th |Parkway|Suite 380|Pleasant Grove|Polk County|TX|74136|United States|-6|apartment| +26120|AAAAAAAAIAGGAAAA|240|12th |Ln|Suite 340|Bunker Hill|Livingston County|MO|60150|United States|-6|condo| +26121|AAAAAAAAJAGGAAAA|116|2nd Central|Ct.|Suite H|Red Hill|Cass County|NE|64338|United States|-6|condo| +26122|AAAAAAAAKAGGAAAA|709|Davis |Court|Suite C|Midway|Marathon County|WI|51904|United States|-6|single family| +26123|AAAAAAAALAGGAAAA|734|Elm |Pkwy|Suite N|Granite|Blair County|PA|16284|United States|-5|apartment| +26124|AAAAAAAAMAGGAAAA|601|Hill 12th|Ave|Suite 430|Jamestown|La Paz County|AZ|86867|United States|-7|single family| +26125|AAAAAAAANAGGAAAA|518|Maple |RD|Suite 460|Woodville|Grafton County|NH|04889|United States|-5|single family| +26126|AAAAAAAAOAGGAAAA|391|View |Dr.|Suite K|Cedar Grove|Presque Isle County|MI|40411|United States|-5|apartment| +26127|AAAAAAAAPAGGAAAA|769|Second River|Street|Suite 320|Cordova|Powell County|MT|66938|United States|-7|apartment| +26128|AAAAAAAAABGGAAAA|119|Mill |Circle|Suite M|Walnut Grove|Baylor County|TX|77752|United States|-6|condo| +26129|AAAAAAAABBGGAAAA|509|Broadway |Drive|Suite 450|Maple Grove|Bureau County|IL|68252|United States|-6|apartment| +26130|AAAAAAAACBGGAAAA|166|Cedar |Ct.|Suite 180|Belmont|Kanabec County|MN|50191|United States|-6|condo| +26131|AAAAAAAADBGGAAAA|952|15th |Way|Suite L|Friendship|Jackson County|AR|74536|United States|-6|single family| +26132|AAAAAAAAEBGGAAAA|293|View |Circle|Suite E|Frankfort|Lewis County|WA|99681|United States|-8|apartment| +26133|AAAAAAAAFBGGAAAA|331|Lake Lake|Boulevard|Suite 110|Plainview|Somerset County|PA|13683|United States|-5|single family| +26134|AAAAAAAAGBGGAAAA|738|Dogwood |Dr.|Suite 470|Mount Pleasant|Seminole County|GA|31933|United States|-5|single family| +26135|AAAAAAAAHBGGAAAA|357|Railroad Park|Ct.|Suite 30|Riverside|Calhoun County|AR|79231|United States|-6|condo| +26136|AAAAAAAAIBGGAAAA|119|Willow |Street|Suite S|Five Forks|Butts County|GA|32293|United States|-5|single family| +26137|AAAAAAAAJBGGAAAA|568|Chestnut |Circle|Suite 110|Clifton|Livingston County|MO|68014|United States|-6|condo| +26138|AAAAAAAAKBGGAAAA|538|Park |Circle|Suite 30|Deerfield|Wells County|IN|49840|United States|-5|apartment| +26139|AAAAAAAALBGGAAAA|696|Smith |Way|Suite U|Deerfield|Washington County|TX|79840|United States|-6|apartment| +26140|AAAAAAAAMBGGAAAA|557|Poplar |Way|Suite V|Fairview|Perry County|KY|45709|United States|-5|apartment| +26141|AAAAAAAANBGGAAAA|659|Maple Locust|Dr.|Suite 360|Newport|Livingston Parish|LA|71521|United States|-6|condo| +26142|AAAAAAAAOBGGAAAA|538|Second 4th|Drive|Suite G|Oak Hill|Cuming County|NE|67838|United States|-6|single family| +26143|AAAAAAAAPBGGAAAA|799|Hillcrest Spruce|Blvd|Suite 70|Oak Ridge|Wayne County|PA|18371|United States|-5|single family| +26144|AAAAAAAAACGGAAAA|511|North |Blvd|Suite 0|Wilson|Kay County|OK|76971|United States|-6|condo| +26145|AAAAAAAABCGGAAAA|758|Church 11th|Way|Suite 240|Providence|Lafayette County|FL|36614|United States|-5|apartment| +26146|AAAAAAAACCGGAAAA|683|7th |Wy|Suite O|Union Hill|Grant County|ND|57746|United States|-6|apartment| +26147|AAAAAAAADCGGAAAA|743|2nd Green|Cir.|Suite 360|Harmony|Upshur County|WV|25804|United States|-5|single family| +26148|AAAAAAAAECGGAAAA|415|Second |Blvd|Suite 260|Lakewood|Greene County|PA|18877|United States|-5|condo| +26149|AAAAAAAAFCGGAAAA|894|Sycamore |Ave|Suite J|Newport|Wharton County|TX|71521|United States|-6|single family| +26150|AAAAAAAAGCGGAAAA|910|Central 6th|Cir.|Suite 300|Greenville|Green County|KY|41387|United States|-6|apartment| +26151|AAAAAAAAHCGGAAAA|577|4th |Drive|Suite Y|Woodville|Wayne County|WV|24289|United States|-5|condo| +26152|AAAAAAAAICGGAAAA|189|Main |||Riverview|Louisa County|VA|||-5|| +26153|AAAAAAAAJCGGAAAA|112|Main Wilson|Ln|Suite H|Pleasant Valley|Montcalm County|MI|42477|United States|-5|condo| +26154|AAAAAAAAKCGGAAAA|488|Hillcrest |Road|Suite F|Greenwood|Ramsey County|ND|58828|United States|-6|condo| +26155|AAAAAAAALCGGAAAA|554|Washington Davis|Street|Suite F|Oakdale|Faulk County|SD|59584|United States|-6|condo| +26156|AAAAAAAAMCGGAAAA|313|West West|Wy|Suite 150|Arlington|Smith County|KS|66557|United States|-6|apartment| +26157|AAAAAAAANCGGAAAA|844|Hill |Pkwy|Suite I|Cedar|Barbour County|WV|21229|United States|-5|apartment| +26158|AAAAAAAAOCGGAAAA|749|Hickory Ridge|Ave|Suite A|Newtown|Bradley County|AR|71749|United States|-6|apartment| +26159|AAAAAAAAPCGGAAAA|165|Wilson Fifth|Cir.|Suite 330|Highland Park|Starr County|TX|76534|United States|-6|condo| +26160|AAAAAAAAADGGAAAA|274|College Pine|Avenue|Suite 30|Green Acres|Tuscarawas County|OH|47683|United States|-5|apartment| +26161|AAAAAAAABDGGAAAA|70|Woodland Oak|Street|Suite 190|Kingston|Davidson County|TN|34975|United States|-5|single family| +26162|AAAAAAAACDGGAAAA|593|Maple Spring|Pkwy|Suite F|Crossroads|Fallon County|MT|60534|United States|-7|single family| +26163|AAAAAAAADDGGAAAA|317|Dogwood 5th|ST|Suite L|Hopewell|Tom Green County|TX|70587|United States|-6|condo| +26164|AAAAAAAAEDGGAAAA|423|Hill Elm|Lane|Suite M|Wilson|Hockley County|TX|76971|United States|-6|condo| +26165|AAAAAAAAFDGGAAAA|729|Center Jackson|Cir.|Suite 300|Oakwood|Navajo County|AZ|80169|United States|-7|apartment| +26166|AAAAAAAAGDGGAAAA|413|||||DeKalb County||30919|||| +26167|AAAAAAAAHDGGAAAA|362|6th |RD|Suite E|Jamestown|Langlade County|WI|56867|United States|-6|apartment| +26168|AAAAAAAAIDGGAAAA|493|Seventh |Way|Suite B|Shiloh|Mercer County|MO|69275|United States|-6|single family| +26169|AAAAAAAAJDGGAAAA|527|Poplar |Dr.|Suite 220|Guilford|Conejos County|CO|84408|United States|-7|apartment| +26170|AAAAAAAAKDGGAAAA|627|Central |RD|Suite S|Glendale|Poinsett County|AR|73951|United States|-6|condo| +26171|AAAAAAAALDGGAAAA|851|Spruce 3rd|Lane|Suite 30|Pleasant Grove|Zapata County|TX|74136|United States|-6|single family| +26172|AAAAAAAAMDGGAAAA|348|9th |Parkway|Suite 480|Greenfield|Franklin County|MO|65038|United States|-6|single family| +26173|AAAAAAAANDGGAAAA|685|View |Court|Suite 380|Greenfield|Gwinnett County|GA|35038|United States|-5|apartment| +26174|AAAAAAAAODGGAAAA|433|Church |Circle|Suite 360|Walnut|Mendocino County|CA|96245|United States|-8|condo| +26175|AAAAAAAAPDGGAAAA|554|Forest |Street|Suite 420|Clinton|Dewey County|OK|78222|United States|-6|apartment| +26176|AAAAAAAAAEGGAAAA|628|Railroad |Blvd|Suite 360|Bunker Hill|Kent County|DE|10150|United States|-5|condo| +26177|AAAAAAAABEGGAAAA|268|Willow First|RD|Suite 320|Florence|Navajo County|AZ|83394|United States|-7|apartment| +26178|AAAAAAAACEGGAAAA|144|Green |Dr.|Suite 380|Concord|Chowan County|NC|24107|United States|-5|condo| +26179|AAAAAAAADEGGAAAA|33|5th Meadow|RD|Suite 460|Pleasant Valley|Muskegon County|MI|42477|United States|-5|apartment| +26180|AAAAAAAAEEGGAAAA|964|Spring |Wy|Suite U|Greenfield|Wyandot County|OH|45038|United States|-5|apartment| +26181|AAAAAAAAFEGGAAAA|722|Park Hill|Ave|Suite X|Shannon|Jasper County|TX|74120|United States|-6|apartment| +26182|AAAAAAAAGEGGAAAA|162|Johnson Lake|Cir.|Suite R|Harmony|White County|IN|45804|United States|-5|condo| +26183|AAAAAAAAHEGGAAAA|550|Valley Railroad|Dr.|Suite 450|Bunker Hill|Shawnee County|KS|60150|United States|-6|single family| +26184|AAAAAAAAIEGGAAAA|563|Center |Parkway|Suite 270|Highland Park|Menard County|IL|66534|United States|-6|apartment| +26185|AAAAAAAAJEGGAAAA|486|Second |ST|Suite T|Farmington|Luce County|MI|49145|United States|-5|single family| +26186|AAAAAAAAKEGGAAAA|642|Spruce River|Ln|Suite 450|Oakland|Radford city|VA|29843|United States|-5|apartment| +26187|AAAAAAAALEGGAAAA|661|East |Ave|Suite P|Macedonia|Issaquena County|MS|51087|United States|-6|apartment| +26188|AAAAAAAAMEGGAAAA|78|Pine |Ln|Suite 300|Friendship|Bowman County|ND|54536|United States|-6|apartment| +26189|AAAAAAAANEGGAAAA|514|1st Ash|Ct.|Suite C|Pleasant Valley|Charleston County|SC|22477|United States|-5|apartment| +26190|AAAAAAAAOEGGAAAA|482|Ash Green|ST|Suite 80|Greenwood|Switzerland County|IN|48828|United States|-5|single family| +26191|AAAAAAAAPEGGAAAA|951|4th |RD|Suite F|Oak Ridge|Monroe County|NY|18371|United States|-5|condo| +26192|AAAAAAAAAFGGAAAA|776|1st South|Boulevard|Suite 280|Glendale|Wagoner County|OK|73951|United States|-6|condo| +26193|AAAAAAAABFGGAAAA|771|10th Dogwood|ST|Suite 420|Bridgeport|Pend Oreille County|WA|95817|United States|-8|condo| +26194|AAAAAAAACFGGAAAA|549|5th |Ct.|Suite 230|Oakdale|Wyoming County|NY|19584|United States|-5|single family| +26195|AAAAAAAADFGGAAAA|528|Wilson |Avenue|Suite 270|Spring Valley|Banner County|NE|66060|United States|-6|apartment| +26196|AAAAAAAAEFGGAAAA|||Ave|Suite G|Oakland|Plumas County||99843|||apartment| +26197|AAAAAAAAFFGGAAAA|155|Park |Boulevard|Suite 390|Wildwood|Newton County|GA|36871|United States|-5|condo| +26198|AAAAAAAAGFGGAAAA|833|Ridge |Ct.|Suite 50|Sulphur Springs|Hubbard County|MN|58354|United States|-6|condo| +26199|AAAAAAAAHFGGAAAA|8|West Washington|Dr.|Suite 430|Walnut Grove|Gogebic County|MI|47752|United States|-5|condo| +26200|AAAAAAAAIFGGAAAA|609|Cedar |Dr.|Suite J|White Oak|Mason County|IL|66668|United States|-6|apartment| +26201|AAAAAAAAJFGGAAAA|680|Second |Road|Suite X|Woodville|Knox County|KY|44289|United States|-5|single family| +26202|AAAAAAAAKFGGAAAA|118|Elm |Avenue|Suite 330|Plainview|Lipscomb County|TX|73683|United States|-6|condo| +26203|AAAAAAAALFGGAAAA|353|2nd |Ct.|Suite P|Lakeside|Grant County|NM|89532|United States|-7|apartment| +26204|AAAAAAAAMFGGAAAA|347|8th Hill|Circle|Suite B|Greenwood|Washington County|OK|78828|United States|-6|single family| +26205|AAAAAAAANFGGAAAA|327|Lincoln |Parkway|Suite L|Union|Bledsoe County|TN|38721|United States|-5|apartment| +26206|AAAAAAAAOFGGAAAA|678|Elm |Road|Suite 220|Shady Grove|Mayes County|OK|72812|United States|-6|apartment| +26207|AAAAAAAAPFGGAAAA|430|Lakeview |Cir.|Suite 280|River Oaks|Matagorda County|TX|78075|United States|-6|single family| +26208|AAAAAAAAAGGGAAAA|213|Pine Sycamore|Boulevard|Suite 470|Spring Valley|Weber County|UT|86060|United States|-7|single family| +26209|AAAAAAAABGGGAAAA|416|Park |Parkway|Suite B|Union Hill|Lee County|IL|67746|United States|-6|single family| +26210|AAAAAAAACGGGAAAA|626|Dogwood Johnson|Road|Suite H|Edgewood|Gaston County|NC|20069|United States|-5|single family| +26211|AAAAAAAADGGGAAAA|883|South 15th|Dr.|Suite 480|Providence|Roane County|WV|26614|United States|-5|single family| +26212|AAAAAAAAEGGGAAAA|919|2nd |Ct.|Suite 70|New Hope|Marion County|IL|69431|United States|-6|condo| +26213|AAAAAAAAFGGGAAAA|160|12th |Dr.|Suite Y|Franklin|Carroll County|KY|49101|United States|-6|single family| +26214|AAAAAAAAGGGGAAAA|169|Laurel |Ave|Suite Q|Crossroads|McIntosh County|ND|50534|United States|-6|apartment| +26215|AAAAAAAAHGGGAAAA|394|Lincoln |Pkwy|Suite 190|Belmont|Isanti County|MN|50191|United States|-6|apartment| +26216|AAAAAAAAIGGGAAAA|18|4th |Street|Suite 160|Wilson|Suffolk County|NY|16971|United States|-5|single family| +26217|AAAAAAAAJGGGAAAA|668|14th |Pkwy|Suite D|Farmington|Broome County|NY|19145|United States|-5|condo| +26218|AAAAAAAAKGGGAAAA|710|Smith Church|RD|Suite 400|Jackson|Frontier County|NE|69583|United States|-6|single family| +26219|AAAAAAAALGGGAAAA|462|Birch Madison|Wy|Suite 240|Hidden Valley|Warren County|PA|15521|United States|-5|apartment| +26220|AAAAAAAAMGGGAAAA|635|Fourth |Lane|Suite T|Carpenter|Stanislaus County|CA|91147|United States|-8|single family| +26221|AAAAAAAANGGGAAAA|122|12th |Wy|Suite 70|Valley View|Osceola County|IA|55124|United States|-6|apartment| +26222|AAAAAAAAOGGGAAAA|393|6th Second|Blvd|Suite V|Brownsville|Beadle County|SD|59310|United States|-6|condo| +26223|AAAAAAAAPGGGAAAA|229|Hillcrest |Drive|Suite X|Mount Zion|Marlboro County|SC|28054|United States|-5|condo| +26224|AAAAAAAAAHGGAAAA|809|Valley |Boulevard|Suite S|Clifton|Jasper County|IL|68014|United States|-6|condo| +26225|AAAAAAAABHGGAAAA|800|2nd 5th|Boulevard|Suite 440|Harmony|Jackson County|IN|45804|United States|-5|single family| +26226|AAAAAAAACHGGAAAA|353|Maple |ST|Suite 260|Stringtown|DeKalb County|MO|60162|United States|-6|apartment| +26227|AAAAAAAADHGGAAAA|749|Sunset |ST|Suite 120|Unionville|Wetzel County|WV|21711|United States|-5|apartment| +26228|AAAAAAAAEHGGAAAA|441|Park Miller|Wy|Suite M|Westminster|Canadian County|OK|76549|United States|-6|apartment| +26229|AAAAAAAAFHGGAAAA|582|1st ||Suite H|Whitesville|Galax city|||United States||| +26230|AAAAAAAAGHGGAAAA|875|Second |Blvd|Suite 150|Oak Grove|Gaston County|NC|28370|United States|-5|condo| +26231|AAAAAAAAHHGGAAAA|473|Adams |Cir.|Suite 450|Oakwood|Camden County|NJ|00769|United States|-5|apartment| +26232|AAAAAAAAIHGGAAAA||Dogwood Hickory|Drive||||||United States|-6|apartment| +26233|AAAAAAAAJHGGAAAA|492|Jackson |Parkway|Suite 430|Spring Hill|Athens County|OH|46787|United States|-5|apartment| +26234|AAAAAAAAKHGGAAAA|864|Franklin Hillcrest|Street|Suite U|Kingston|Lackawanna County|PA|14975|United States|-5|apartment| +26235|AAAAAAAALHGGAAAA|557|6th |Cir.|Suite K|Maple Grove|LaMoure County|ND|58252|United States|-6|condo| +26236|AAAAAAAAMHGGAAAA|580|Central |Road|Suite H|Bridgeport|Tuscaloosa County|AL|35817|United States|-6|apartment| +26237|AAAAAAAANHGGAAAA|570|Fourth Dogwood|Ln|Suite 450|Oakwood|Richmond County|NC|20169|United States|-5|condo| +26238|AAAAAAAAOHGGAAAA|||RD||Valley View|Vanderburgh County||45124|||| +26239|AAAAAAAAPHGGAAAA|138|Washington |Court|Suite Q|Crossroads|Greenwood County|KS|60534|United States|-6|apartment| +26240|AAAAAAAAAIGGAAAA|445|Church 15th|Dr.|Suite 140|Wildwood|New London County|CT|07471|United States|-5|single family| +26241|AAAAAAAABIGGAAAA|423|Highland Hillcrest|Avenue|Suite I|Bridgeport|Dodge County|WI|55817|United States|-6|single family| +26242|AAAAAAAACIGGAAAA|659|Sycamore Hill|Blvd|Suite D|Brownsville|Duval County|TX|79310|United States|-6|single family| +26243|AAAAAAAADIGGAAAA|974|Wilson |Ln|Suite K|Harmony|Attala County|MS|55804|United States|-6|single family| +26244|AAAAAAAAEIGGAAAA|292|Forest |Pkwy|Suite E|Mount Olive|Essex County|MA|08659|United States|-5|apartment| +26245|AAAAAAAAFIGGAAAA|869|Hickory 2nd|Street|Suite 90|Hopewell|Worth County|GA|30587|United States|-5|apartment| +26246|AAAAAAAAGIGGAAAA|838|Lee |Way|Suite R|Friendship|Wayne County|MI|44536|United States|-5|condo| +26247|AAAAAAAAHIGGAAAA|270||||Centerville|||20059||-5|| +26248|AAAAAAAAIIGGAAAA|458|Spruce Laurel|ST|Suite D|Bear Creek|Russell County|AL|33075|United States|-6|apartment| +26249|AAAAAAAAJIGGAAAA|73|Hickory Elm|Cir.|Suite 320|Brownsville|Trego County|KS|69310|United States|-6|condo| +26250|AAAAAAAAKIGGAAAA|84|Hill East|Court|Suite B|Waterloo|Reno County|KS|61675|United States|-6|single family| +26251|AAAAAAAALIGGAAAA|559|Jackson |Parkway|Suite 340|Ashland|Clinton County|IL|64244|United States|-6|single family| +26252|AAAAAAAAMIGGAAAA|915|4th |Way|Suite A|Cedar Grove|Kalkaska County|MI|40411|United States|-5|condo| +26253|AAAAAAAANIGGAAAA||6th |Ln|||Marion County||39840||-5|condo| +26254|AAAAAAAAOIGGAAAA|828|Washington Williams|Street|Suite U|Mount Vernon|Logan County|NE|68482|United States|-7|single family| +26255|AAAAAAAAPIGGAAAA|845|8th 6th|Ct.|Suite 310|Lincoln|Coffey County|KS|61289|United States|-6|condo| +26256|AAAAAAAAAJGGAAAA|963|7th |Boulevard|Suite N|Adrian|Buena Vista city|VA|23301|United States|-5|apartment| +26257|AAAAAAAABJGGAAAA|964|Smith |ST|Suite 10|Wilson|Langlade County|WI|56971|United States|-6|apartment| +26258|AAAAAAAACJGGAAAA|613|Ninth |Pkwy|Suite M|Lakeside|Jennings County|IN|49532|United States|-5|condo| +26259|AAAAAAAADJGGAAAA|975|River |Way|Suite C|Lebanon|Wagoner County|OK|72898|United States|-6|apartment| +26260|AAAAAAAAEJGGAAAA|985|Miller |Pkwy|Suite E|Highland|Fredericksburg city|VA|29454|United States|-5|apartment| +26261|AAAAAAAAFJGGAAAA|744|Cedar |Pkwy|Suite 160|Florence|Saline County|AR|73394|United States|-6|apartment| +26262|AAAAAAAAGJGGAAAA|497|College Wilson|Ave|Suite N|Spring Hill|Chouteau County|MT|66787|United States|-7|apartment| +26263|AAAAAAAAHJGGAAAA|10|Meadow |Dr.|Suite L|Wildwood|Lauderdale County|AL|36871|United States|-6|condo| +26264|AAAAAAAAIJGGAAAA|962|Second |Lane|Suite A|Farmington|Wyandotte County|KS|69145|United States|-6|apartment| +26265|AAAAAAAAJJGGAAAA|855|Railroad |Lane|Suite O|Red Hill|Alcorn County|MS|54338|United States|-6|condo| +26266|AAAAAAAAKJGGAAAA|469|River 1st|Avenue|Suite X|Woodville|Allegany County|NY|14289|United States|-5|apartment| +26267|AAAAAAAALJGGAAAA|681|Church |Circle|Suite Y|Green Acres|McCormick County|SC|27683|United States|-5|apartment| +26268|AAAAAAAAMJGGAAAA|92|Sunset |Avenue|Suite 300|Five Forks|Costilla County|CO|82293|United States|-7|apartment| +26269|AAAAAAAANJGGAAAA|103|Ridge View|ST|Suite U|Bethel|Belmont County|OH|45281|United States|-5|single family| +26270|AAAAAAAAOJGGAAAA|395|Wilson |Blvd|Suite 180|Maple Grove|Macon County|MO|68252|United States|-6|condo| +26271|AAAAAAAAPJGGAAAA|146|Tenth East|Street|Suite 260|Oakdale|Lexington County|SC|29584|United States|-5|apartment| +26272|AAAAAAAAAKGGAAAA|432|Madison Jackson|Ave|Suite O|Greenwood|Franklin County|IN|48828|United States|-5|single family| +26273|AAAAAAAABKGGAAAA|201|Maple |Blvd|Suite 250|Jamestown|Warrick County|IN|46867|United States|-5|condo| +26274|AAAAAAAACKGGAAAA|701|Third |Ln|Suite 40|Friendship|Rabun County|GA|34536|United States|-5|single family| +26275|AAAAAAAADKGGAAAA|392|Third 7th|Ave|Suite N|Macedonia|Prowers County|CO|81087|United States|-7|condo| +26276|AAAAAAAAEKGGAAAA|305|Ash |Lane|Suite M|Fairview|Polk County|NC|25709|United States|-5|condo| +26277|AAAAAAAAFKGGAAAA|573|5th |Circle|Suite 180|Bridgeport|Caribou County|ID|85817|United States|-7|single family| +26278|AAAAAAAAGKGGAAAA|882|5th |Wy|Suite 300|Buena Vista|Brown County|TX|75752|United States|-6|condo| +26279|AAAAAAAAHKGGAAAA|880|Laurel 7th|Wy|Suite 450|Salem|Swift County|MN|58048|United States|-6|single family| +26280|AAAAAAAAIKGGAAAA|214|5th |RD|Suite Y|Concord|Garvin County|OK|74107|United States|-6|condo| +26281|AAAAAAAAJKGGAAAA|554|Locust |ST|Suite 10|Forestville|Jones County|MS|53027|United States|-6|single family| +26282|AAAAAAAAKKGGAAAA|629|Wilson |Ln|Suite Q|Lebanon|Franklin County|WA|92898|United States|-8|apartment| +26283|AAAAAAAALKGGAAAA|320|Sixth |Ct.|Suite 280|Waterloo|Washtenaw County|MI|41675|United States|-5|condo| +26284|AAAAAAAAMKGGAAAA|230|Cherry |Boulevard|Suite 280|Amherst|Grenada County|MS|58119|United States|-6|condo| +26285|AAAAAAAANKGGAAAA|785|Johnson 2nd|Boulevard|Suite 20|Unionville|Lexington County|SC|21711|United States|-5|apartment| +26286|AAAAAAAAOKGGAAAA|765|5th 8th|Street|Suite Q|Concord|Candler County|GA|34107|United States|-5|apartment| +26287|AAAAAAAAPKGGAAAA|586|Woodland East|ST|Suite Y|Riceville|Cowlitz County|WA|95867|United States|-8|condo| +26288|AAAAAAAAALGGAAAA|854|Elm |Avenue|Suite N|Woodlawn|Jackson County|AL|34098|United States|-6|single family| +26289|AAAAAAAABLGGAAAA|278|1st |Avenue|Suite 340|Wildwood|Jefferson County|IN|46871|United States|-5|condo| +26290|AAAAAAAACLGGAAAA|201|Walnut |Wy|Suite 270|Reno|Owsley County|KY|40344|United States|-5|single family| +26291|AAAAAAAADLGGAAAA|227|Spruce 3rd|Pkwy|Suite 70|Belmont|Hickman County|TN|30191|United States|-5|apartment| +26292|AAAAAAAAELGGAAAA|||Street|Suite A||Marshall County|MS||United States||condo| +26293|AAAAAAAAFLGGAAAA|579|3rd |Lane|Suite F|Providence|Cook County|MN|56614|United States|-6|single family| +26294|AAAAAAAAGLGGAAAA|431|North 6th|RD|Suite V|Glendale|Ballard County|KY|43951|United States|-6|single family| +26295|AAAAAAAAHLGGAAAA|35|Valley Park|Drive|Suite 290|Franklin|Dakota County|NE|69101|United States|-6|apartment| +26296|AAAAAAAAILGGAAAA|553|Hill |Boulevard|Suite J|Enterprise|Newport News city|VA|21757|United States|-5|apartment| +26297|AAAAAAAAJLGGAAAA|174|Lake 4th|Ave|Suite C|Peoria|Barry County|MI|49818|United States|-5|apartment| +26298|AAAAAAAAKLGGAAAA|557|1st |Blvd|Suite A|Friendship|Vilas County|WI|54536|United States|-6|condo| +26299|AAAAAAAALLGGAAAA|125|2nd |Lane|Suite E|Salem|Botetourt County|VA|28048|United States|-5|condo| +26300|AAAAAAAAMLGGAAAA|846|Meadow Walnut||||Eagle County||88354|United States|-7|| +26301|AAAAAAAANLGGAAAA|29|Elm |Cir.|Suite 20|Saint George|Escambia County|FL|35281|United States|-5|single family| +26302|AAAAAAAAOLGGAAAA|366|Woodland |Way|Suite 470|Summit|Pottawatomie County|KS|60499|United States|-6|single family| +26303|AAAAAAAAPLGGAAAA|34|Williams Woodland|Wy|Suite X|Belmont|Clinton County|NY|10191|United States|-5|condo| +26304|AAAAAAAAAMGGAAAA|773|Hill Spring|Pkwy|Suite J|Antioch|Schuylkill County|PA|18605|United States|-5|condo| +26305|AAAAAAAABMGGAAAA|236|Sycamore Fifth|Ct.|Suite A|Owens|Orange County|TX|72324|United States|-6|single family| +26306|AAAAAAAACMGGAAAA|545|West |Cir.|Suite 30|Spring Valley|Muscogee County|GA|36060|United States|-5|single family| +26307|AAAAAAAADMGGAAAA|259|Washington Spruce|Boulevard|Suite 50|Pleasant Grove|Hunterdon County|NJ|04736|United States|-5|apartment| +26308|AAAAAAAAEMGGAAAA|935|10th |Drive|Suite H|Elba|Elko County|NV|80262|United States|-8|single family| +26309|AAAAAAAAFMGGAAAA|597|Adams Elm|Wy|Suite L|Red Hill|Mason County|KY|44338|United States|-5|condo| +26310|AAAAAAAAGMGGAAAA|495|View |Way|Suite Q|Oak Ridge|Trempealeau County|WI|58371|United States|-6|single family| +26311|AAAAAAAAHMGGAAAA|720|Third 10th|Drive|Suite W|Fairview|Hampshire County|WV|25709|United States|-5|single family| +26312|AAAAAAAAIMGGAAAA|72|1st |Parkway|Suite T|Union Hill|Goshen County|WY|87746|United States|-7|apartment| +26313|AAAAAAAAJMGGAAAA|995|Forest |Ct.|Suite N|Buena Vista|Richmond city|VA|25752|United States|-5|single family| +26314|AAAAAAAAKMGGAAAA|||Parkway||Midway|Noble County|IN||||| +26315|AAAAAAAALMGGAAAA|567|Fourth |Dr.|Suite N|Cedar Grove|Camden County|NJ|01011|United States|-5|single family| +26316|AAAAAAAAMMGGAAAA|138|Main 1st|Ln|Suite H|Green Acres|Sevier County|UT|87683|United States|-7|single family| +26317|AAAAAAAANMGGAAAA|454|Davis |Ln|Suite 470|Marion|Madison County|KY|40399|United States|-5|condo| +26318|AAAAAAAAOMGGAAAA|639|1st View|Blvd|Suite A|Hamilton|Miami County|OH|42808|United States|-5|apartment| +26319|AAAAAAAAPMGGAAAA|948|Highland |Dr.|Suite 270|Georgetown|Rowan County|NC|27057|United States|-5|single family| +26320|AAAAAAAAANGGAAAA|182|4th |Court|Suite R|Unionville|Hot Spring County|AR|71711|United States|-6|apartment| +26321|AAAAAAAABNGGAAAA|244|Fifth View|Road|Suite A|Hillcrest|Pinellas County|FL|33003|United States|-5|condo| +26322|AAAAAAAACNGGAAAA|768|Ash South|Lane|Suite Q|Wolf Creek|Washington County|AL|32455|United States|-6|apartment| +26323|AAAAAAAADNGGAAAA|167|Miller |Blvd|Suite F|Star|Willacy County|TX|70725|United States|-6|single family| +26324|AAAAAAAAENGGAAAA|79|Cedar |Ct.|Suite Q|Hidden Valley|Richland County|OH|45521|United States|-5|condo| +26325|AAAAAAAAFNGGAAAA|223|Center Park|Road|Suite 430|Belmont|Gratiot County|MI|40191|United States|-5|apartment| +26326|AAAAAAAAGNGGAAAA|459|3rd Railroad|Boulevard|Suite N|Jamestown|Cook County|GA|36867|United States|-5|single family| +26327|AAAAAAAAHNGGAAAA|352|Hickory Franklin|Ave|Suite 70|Forest Hills|Russell County|VA|29237|United States|-5|apartment| +26328|AAAAAAAAINGGAAAA|360|Fifth |Wy|Suite 270|Jamestown|Hamilton County|IL|66867|United States|-6|single family| +26329|AAAAAAAAJNGGAAAA|268|View |Cir.|Suite T|Mount Zion|Lawrence County|TN|38054|United States|-6|apartment| +26330|AAAAAAAAKNGGAAAA|581|Johnson East|Ln|Suite 380|Clifford|Johnson County|IL|68164|United States|-6|condo| +26331|AAAAAAAALNGGAAAA|824|Lake |Dr.|Suite 130|Providence|Waldo County|ME|07214|United States|-5|apartment| +26332|AAAAAAAAMNGGAAAA|795|Cedar Spring|Blvd|Suite 170|Pleasant Valley|Kidder County|ND|52477|United States|-6|apartment| +26333|AAAAAAAANNGGAAAA||Main Main||||Camden County||31711|United States||| +26334|AAAAAAAAONGGAAAA|996|Washington |Ln|Suite I|Bridgeport|Hopkins County|TX|75817|United States|-6|single family| +26335|AAAAAAAAPNGGAAAA|664|2nd 5th|Ave|Suite 20|Jerome|Polk County|AR|79920|United States|-6|condo| +26336|AAAAAAAAAOGGAAAA|293|9th Park|Wy|Suite 350|Ashland|Valley County|MT|64244|United States|-7|single family| +26337|AAAAAAAABOGGAAAA|389|5th Lake|Dr.|Suite N|New Hope|Skamania County|WA|99431|United States|-8|single family| +26338|AAAAAAAACOGGAAAA||Cedar |||Salem||||United States|-5|| +26339|AAAAAAAADOGGAAAA|36|Third 6th|Wy|Suite 40|Harmony|Weber County|UT|85804|United States|-7|single family| +26340|AAAAAAAAEOGGAAAA|972|Hillcrest |Cir.|Suite 230|Oak Hill|Huron County|OH|47838|United States|-5|single family| +26341|AAAAAAAAFOGGAAAA|142|Mill Elm|Avenue|Suite W|Greenville|Manassas Park city|VA|21387|United States|-5|condo| +26342|AAAAAAAAGOGGAAAA|81|Jackson Williams|Circle|Suite V|Waterloo|Norman County|MN|51675|United States|-6|condo| +26343|AAAAAAAAHOGGAAAA|551|15th |Dr.|Suite K|Newport|Archuleta County|CO|81521|United States|-7|apartment| +26344|AAAAAAAAIOGGAAAA|232|River Pine|Wy|Suite 330|Bethel|Spencer County|IN|45281|United States|-5|condo| +26345|AAAAAAAAJOGGAAAA|770|Maple Cedar|Blvd|Suite B|Lakeside|Cannon County|TN|39532|United States|-5|single family| +26346|AAAAAAAAKOGGAAAA|445|Sycamore |ST|Suite 200|Woodland|Tioga County|NY|14854|United States|-5|apartment| +26347|AAAAAAAALOGGAAAA|999|Jackson Cedar|Blvd|Suite C|Newtown|Dickinson County|IA|51749|United States|-6|single family| +26348|AAAAAAAAMOGGAAAA|531|Church Fifth|Dr.|Suite P|Lakeside|Banks County|GA|39532|United States|-5|single family| +26349|AAAAAAAANOGGAAAA|122|Church Hill|Street|Suite 210|Hopewell|Coshocton County|OH|40587|United States|-5|apartment| +26350|AAAAAAAAOOGGAAAA|472|Church Park|RD|Suite C|Woodville|Laurens County|SC|24289|United States|-5|single family| +26351|AAAAAAAAPOGGAAAA|14|6th |Street|Suite W|Lakeside|White Pine County|NV|89532|United States|-8|apartment| +26352|AAAAAAAAAPGGAAAA|517|6th |ST|Suite 280|Church Hill|Yazoo County|MS|53790|United States|-6|condo| +26353|AAAAAAAABPGGAAAA|971|Miller First|Wy|Suite U|Morris|Montgomery County|PA|16696|United States|-5|single family| +26354|AAAAAAAACPGGAAAA|288|Woodland |Ln|Suite P|Oak Grove|Delaware County|PA|18370|United States|-5|apartment| +26355|AAAAAAAADPGGAAAA|641|Park |Road|Suite R|Greenfield|Freestone County|TX|75038|United States|-6|apartment| +26356|AAAAAAAAEPGGAAAA|964|Cedar |Circle|Suite W|Concord|Monroe County|AR|74107|United States|-6|condo| +26357|AAAAAAAAFPGGAAAA|507|Poplar Eigth|Drive|Suite 480|Greenwood|Webster County|MS|58828|United States|-6|condo| +26358|AAAAAAAAGPGGAAAA|527|Mill |Pkwy|Suite N|Franklin|Johnson County|WY|89101|United States|-7|single family| +26359|AAAAAAAAHPGGAAAA|840|First |Way|Suite O|Friendship|Sussex County|NJ|05136|United States|-5|condo| +26360|AAAAAAAAIPGGAAAA|720|Willow |Way|Suite V|Sumner|Taliaferro County|GA|30519|United States|-5|condo| +26361|AAAAAAAAJPGGAAAA|993|North Sycamore|Ave|Suite K|Shady Grove|Noxubee County|MS|52812|United States|-6|single family| +26362|AAAAAAAAKPGGAAAA|231|5th Park|Pkwy|Suite O|Unionville|Kanabec County|MN|51711|United States|-6|condo| +26363|AAAAAAAALPGGAAAA|769|Cedar Main|Street|Suite U|Murphy|Mineral County|WV|22105|United States|-5|condo| +26364|AAAAAAAAMPGGAAAA|231|8th |Dr.|Suite W|Middletown|Warren County|OH|41422|United States|-5|condo| +26365|AAAAAAAANPGGAAAA|408|Lakeview |Avenue|Suite 340|Spring Valley|Garfield County|CO|86060|United States|-7|condo| +26366|AAAAAAAAOPGGAAAA|186|Birch |Wy|Suite 300|Burns|Highland County|VA|25272|United States|-5|single family| +26367|AAAAAAAAPPGGAAAA|875|Oak 3rd|Circle|Suite F|Clifton|Grant County|AR|78014|United States|-6|single family| +26368|AAAAAAAAAAHGAAAA|151|10th |Ave|Suite 440|Oak Ridge|Alamance County|NC|28371|United States|-5|apartment| +26369|AAAAAAAABAHGAAAA|758|Hickory River|Cir.|Suite 140|Woodville|Kane County|UT|84289|United States|-7|condo| +26370|AAAAAAAACAHGAAAA|884|Green Oak|Drive|Suite 300|Pleasant Grove|Brown County|SD|54136|United States|-6|apartment| +26371|AAAAAAAADAHGAAAA|154|Poplar |Drive|Suite 480|Brownsville|Ellis County|KS|69310|United States|-6|apartment| +26372|AAAAAAAAEAHGAAAA|825|5th First|Ct.|Suite 50|Providence|Isanti County|MN|56614|United States|-6|single family| +26373|AAAAAAAAFAHGAAAA|367|Ash 11th|ST|Suite 490|Lee|Carroll County|TN|30408|United States|-5|single family| +26374|AAAAAAAAGAHGAAAA|353|Mill |Lane|Suite 200|Waterloo|Stillwater County|MT|61675|United States|-7|apartment| +26375|AAAAAAAAHAHGAAAA|648|West |Parkway|Suite E|Hopewell|Twiggs County|GA|30587|United States|-5|condo| +26376|AAAAAAAAIAHGAAAA|833|Williams Main|Road|Suite I|Waterloo|Carroll County|MS|51675|United States|-6|condo| +26377|AAAAAAAAJAHGAAAA|63|Third |Ln|Suite 90|Waterloo|Yakutat Borough|AK|91675|United States|-9|apartment| +26378|AAAAAAAAKAHGAAAA|345|Walnut Cedar|Dr.|Suite Y|Forest Hills|Shackelford County|TX|79237|United States|-6|single family| +26379|AAAAAAAALAHGAAAA|51|Sunset |Avenue|Suite 290|Centerville|Galveston County|TX|70059|United States|-6|single family| +26380|AAAAAAAAMAHGAAAA|218|North Pine|Way|Suite 370|Arlington|Dawes County|NE|66557|United States|-6|apartment| +26381|AAAAAAAANAHGAAAA|171|Maple Park|Cir.|Suite 190|Harmony|Canadian County|OK|75804|United States|-6|single family| +26382|AAAAAAAAOAHGAAAA|343|Locust Church|Dr.|Suite 310|Wildwood|Plymouth County|MA|07471|United States|-5|single family| +26383|AAAAAAAAPAHGAAAA|996|Broadway |Blvd|Suite 490|Oakdale|Dare County|NC|29584|United States|-5|single family| +26384|AAAAAAAAABHGAAAA|277|Mill Main|Wy|Suite G|Farmington|Northumberland County|VA|29145|United States|-5|apartment| +26385|AAAAAAAABBHGAAAA|926|Franklin |Avenue|Suite 100|Farmington|Valley County|MT|69145|United States|-7|apartment| +26386|AAAAAAAACBHGAAAA|773|Sixth Fourth|Avenue|Suite R|Riverdale|Polk County|NE|69391|United States|-7|condo| +26387|AAAAAAAADBHGAAAA|37|10th Park|Ln|Suite 80|Langdon|Hyde County|SD|50852|United States|-7|single family| +26388|AAAAAAAAEBHGAAAA|62|Jackson Dogwood|Cir.|Suite N|Antioch|Washtenaw County|MI|48605|United States|-5|condo| +26389|AAAAAAAAFBHGAAAA|465|Lincoln |Court|Suite 250|Jamestown|Oconee County|SC|26867|United States|-5|single family| +26390|AAAAAAAAGBHGAAAA|125|Jefferson |Road|Suite 270|Buena Vista|Johnson County|WY|85752|United States|-7|single family| +26391|AAAAAAAAHBHGAAAA|226|4th Center|Wy|Suite J|Hopewell|Woodson County|KS|60587|United States|-6|apartment| +26392|AAAAAAAAIBHGAAAA|983|Sixth Oak|Road|Suite 100|Mount Olive|Lincoln County|MT|68059|United States|-7|apartment| +26393|AAAAAAAAJBHGAAAA|146|5th 10th|Dr.|Suite K|Farmington|Hampshire County|MA|09745|United States|-5|condo| +26394|AAAAAAAAKBHGAAAA|779|Ash |ST|Suite 160|Antioch|Miller County|AR|78605|United States|-6|single family| +26395|AAAAAAAALBHGAAAA|558|Oak |Circle|Suite 430|Pine Grove|Gage County|NE|64593|United States|-6|apartment| +26396|AAAAAAAAMBHGAAAA|518|Wilson Poplar|Parkway|Suite F|Brownsville|Lexington County|SC|29310|United States|-5|single family| +26397|AAAAAAAANBHGAAAA|21|Franklin Wilson|ST|Suite 150|Johnsonville|Webster County|NE|67745|United States|-7|condo| +26398|AAAAAAAAOBHGAAAA|411|Park Valley|Dr.|Suite T|Brownsville|Hempstead County|AR|79310|United States|-6|condo| +26399|AAAAAAAAPBHGAAAA|742|College |Ct.|Suite B|Maple Grove|Gloucester County|NJ|08852|United States|-5|single family| +26400|AAAAAAAAACHGAAAA|467|Poplar Oak|Ave|Suite 490|Woodruff|Essex County|VT|04774|United States|-5|single family| +26401|AAAAAAAABCHGAAAA|775|West Smith|Parkway|Suite 410|Hillcrest|Fallon County|MT|63003|United States|-7|single family| +26402|AAAAAAAACCHGAAAA|972|Spruce Cedar|Ave|Suite 450|Midway|Montgomery County|NC|21904|United States|-5|single family| +26403|AAAAAAAADCHGAAAA|996|Fourth |Way|Suite 300|Longwood|Madison County|TN|37021|United States|-6|condo| +26404|AAAAAAAAECHGAAAA|783|Lincoln |Way|Suite P|Bridgeport|Pike County|AL|35817|United States|-6|single family| +26405|AAAAAAAAFCHGAAAA|318|6th |Blvd|Suite E|Woodville|Highland County|OH|44289|United States|-5|condo| +26406|AAAAAAAAGCHGAAAA|890|3rd |Pkwy|Suite 20|Greenville|Kingman County|KS|61387|United States|-6|apartment| +26407|AAAAAAAAHCHGAAAA|5|South |Court|Suite F|Pleasant Hill|Merrick County|NE|63604|United States|-7|single family| +26408|AAAAAAAAICHGAAAA|979|Wilson |Boulevard|Suite 120|Riverview|Trousdale County|TN|39003|United States|-6|single family| +26409|AAAAAAAAJCHGAAAA|443||||Providence|||36614||-5|single family| +26410|AAAAAAAAKCHGAAAA|69|13th |Parkway|Suite 220|Lakewood|Erath County|TX|78877|United States|-6|single family| +26411|AAAAAAAALCHGAAAA|184|Pine 15th|Parkway|Suite 490|Five Forks|Wayne County|MS|52293|United States|-6|apartment| +26412|AAAAAAAAMCHGAAAA|853|Oak South|Parkway|Suite 480|Carthage|Bibb County|AL|31529|United States|-6|single family| +26413|AAAAAAAANCHGAAAA|800|Locust |Court|Suite H|Rockwood|Hoke County|NC|21545|United States|-5|apartment| +26414|AAAAAAAAOCHGAAAA|913||||||MI|||-5|| +26415|AAAAAAAAPCHGAAAA|104|Spring |Parkway|Suite 220|Deerfield|Texas County|MO|69840|United States|-6|condo| +26416|AAAAAAAAADHGAAAA|832|Washington Hickory|Street|Suite 370|Oakwood|Raleigh County|WV|20169|United States|-5|single family| +26417|AAAAAAAABDHGAAAA|521|4th |Road|Suite 310|Five Forks|San Juan County|NM|82293|United States|-7|single family| +26418|AAAAAAAACDHGAAAA|310|Forest 7th|Road|Suite 490|Glendale|Grundy County|MO|63951|United States|-6|apartment| +26419|AAAAAAAADDHGAAAA|15|5th |Boulevard|Suite E|Antioch|Meade County|SD|58605|United States|-7|single family| +26420|AAAAAAAAEDHGAAAA|944|Park Lake|Parkway|Suite 330|Florence|Pasco County|FL|33394|United States|-5|condo| +26421|AAAAAAAAFDHGAAAA||Washington 7th|Pkwy|Suite J|Oakwood||LA|70169|United States|-6|single family| +26422|AAAAAAAAGDHGAAAA|730|Cherry Washington|Ave|Suite 220|Jackson|Monroe County|MI|49583|United States|-5|apartment| +26423|AAAAAAAAHDHGAAAA|145|5th Maple|Avenue|Suite 230|Providence|Delaware County|IN|46614|United States|-5|apartment| +26424|AAAAAAAAIDHGAAAA|353|9th 2nd|Pkwy|Suite H|Edgewood|Eureka County|NV|80069|United States|-8|apartment| +26425|AAAAAAAAJDHGAAAA|389|Sunset |Cir.|Suite Q|Macedonia|Butler County|KY|41087|United States|-6|condo| +26426|AAAAAAAAKDHGAAAA|255|Hill 1st|Wy|Suite S|Ellisville|Adair County|OK|76820|United States|-6|condo| +26427|AAAAAAAALDHGAAAA|92|3rd 1st|Street|Suite E|Riverside|Traverse County|MN|59231|United States|-6|condo| +26428|AAAAAAAAMDHGAAAA|569|Pine |ST|Suite 80|Ashland|Taney County|MO|64244|United States|-6|single family| +26429|AAAAAAAANDHGAAAA|167|Cedar Davis|Dr.|Suite S|Woodville|Gladwin County|MI|44289|United States|-5|condo| +26430|AAAAAAAAODHGAAAA|249|Smith 1st|Blvd|Suite H|Union Hill|La Crosse County|WI|57746|United States|-6|single family| +26431|AAAAAAAAPDHGAAAA|179|Oak Oak|Road|Suite D|Antioch|Swain County|NC|28605|United States|-5|condo| +26432|AAAAAAAAAEHGAAAA|800|Willow |Ave|Suite X|Lakewood|Orange County|VT|09477|United States|-5|single family| +26433|AAAAAAAABEHGAAAA|934|South |Pkwy|Suite F|Guilford|Bleckley County|GA|34408|United States|-5|single family| +26434|AAAAAAAACEHGAAAA|710|Davis |Dr.|Suite 150|Glendale|Teller County|CO|83951|United States|-7|single family| +26435|AAAAAAAADEHGAAAA|604|Locust |Pkwy|Suite 340|Union|Collier County|FL|38721|United States|-5|apartment| +26436|AAAAAAAAEEHGAAAA|182|Meadow |Ln|Suite 150|Valley View|Bolivar County|MS|55124|United States|-6|apartment| +26437|AAAAAAAAFEHGAAAA|933|Spruce |Road|Suite 380|Union Hill|Pulaski County|MO|67746|United States|-6|single family| +26438|AAAAAAAAGEHGAAAA|51|Park Johnson|Boulevard|Suite 380|Jamestown|York County|NE|66867|United States|-6|condo| +26439|AAAAAAAAHEHGAAAA|525|Oak 9th|Pkwy|Suite U|Garrison|Power County|ID|88767|United States|-7|single family| +26440|AAAAAAAAIEHGAAAA|224|Cherry |Ct.|Suite 280|Newtown|Harlan County|KY|41749|United States|-6|single family| +26441|AAAAAAAAJEHGAAAA|494|Lake |Court|Suite V|Bunker Hill|Decatur County|KS|60150|United States|-6|single family| +26442|AAAAAAAAKEHGAAAA|204|Broadway |Road|Suite O|Springfield|Nantucket County|MA|09903|United States|-5|single family| +26443|AAAAAAAALEHGAAAA|848|Valley Elm|Drive|Suite 70|Liberty|Stevens County|WA|93451|United States|-8|condo| +26444|AAAAAAAAMEHGAAAA|785|5th |ST|Suite 480|Macedonia|Lavaca County|TX|71087|United States|-6|apartment| +26445|AAAAAAAANEHGAAAA|565|Forest |Lane|Suite 120|Crystal|Washington Parish|LA|75258|United States|-6|condo| +26446|AAAAAAAAOEHGAAAA|731|4th |Ln|Suite L|Spring Hill|Golden Valley County|ND|56787|United States|-6|single family| +26447|AAAAAAAAPEHGAAAA|253|Willow |||||NY||United States||| +26448|AAAAAAAAAFHGAAAA|264|South 7th|Boulevard|Suite F|Shady Grove|Santa Fe County|NM|82812|United States|-7|condo| +26449|AAAAAAAABFHGAAAA|407|South Cedar|Circle|Suite 160|Newport|Fulton County|IL|61521|United States|-6|single family| +26450|AAAAAAAACFHGAAAA|811|View |Dr.|Suite 480|Greenville|Fairfax city|VA|21387|United States|-5|apartment| +26451|AAAAAAAADFHGAAAA|453|Washington |Ave|Suite 240|Woodlawn|Keith County|NE|64098|United States|-7|single family| +26452|AAAAAAAAEFHGAAAA|560|Sunset View|Avenue|Suite G|Lincoln|Ada County|ID|81289|United States|-7|apartment| +26453|AAAAAAAAFFHGAAAA|444|Ash 12th|Ave|Suite V|Prosperity|Stevens County|KS|69089|United States|-6|apartment| +26454|AAAAAAAAGFHGAAAA|266|10th Sixth|Dr.|Suite E|Lakeview|Phillips County|AR|78579|United States|-6|single family| +26455|AAAAAAAAHFHGAAAA|630|River |ST|Suite N|Belmont|Hancock County|ME|00791|United States|-5|single family| +26456|AAAAAAAAIFHGAAAA|301|Maple |Ct.|Suite Q|Hopewell|Bond County|IL|60587|United States|-6|apartment| +26457|AAAAAAAAJFHGAAAA|576|14th Seventh|Ct.|Suite 20|Freeport|Currituck County|NC|21844|United States|-5|condo| +26458|AAAAAAAAKFHGAAAA|306|Elm Park|Drive|Suite 110|Kingston|Washtenaw County|MI|44975|United States|-5|apartment| +26459|AAAAAAAALFHGAAAA|166|First Hill|RD|Suite A|Clearview|Appling County|GA|35495|United States|-5|apartment| +26460|AAAAAAAAMFHGAAAA|867|Cherry Railroad|Street|Suite 400|Unionville|Searcy County|AR|71711|United States|-6|single family| +26461|AAAAAAAANFHGAAAA|581|Washington Chestnut|Parkway|Suite 420|Five Forks|Bath County|KY|42293|United States|-6|condo| +26462|AAAAAAAAOFHGAAAA|503|West |Ave|Suite 360|Providence|Coahoma County|MS|56614|United States|-6|single family| +26463|AAAAAAAAPFHGAAAA|221|Valley Maple|Cir.|Suite 160|Mount Olive|Abbeville County|SC|28059|United States|-5|condo| +26464|AAAAAAAAAGHGAAAA|693|Chestnut Meadow|Way|Suite 290|Fairfield|Maury County|TN|36192|United States|-6|condo| +26465|AAAAAAAABGHGAAAA|329|Park |Ave|Suite R|Green Acres|Cullman County|AL|37683|United States|-6|apartment| +26466|AAAAAAAACGHGAAAA|||||Waterloo|Harlan County|KY||United States||single family| +26467|AAAAAAAADGHGAAAA|139|Poplar |Court|Suite O|Riverside|Bent County|CO|89231|United States|-7|apartment| +26468|AAAAAAAAEGHGAAAA|429|10th |Ln|Suite Y|Bridgeport|Sac County|IA|55817|United States|-6|single family| +26469|AAAAAAAAFGHGAAAA|164|Ash |Way|Suite E|Friendship|Columbia County|AR|74536|United States|-6|condo| +26470|AAAAAAAAGGHGAAAA|354|5th Central|Boulevard|Suite 390|Clinton|Marin County|CA|98222|United States|-8|condo| +26471|AAAAAAAAHGHGAAAA|337|Hickory Birch|Wy|Suite 60|Riverside|Owen County|KY|49231|United States|-5|condo| +26472|AAAAAAAAIGHGAAAA|525|8th |Ln|Suite 140|New Hope|Bedford County|VA|29431|United States|-5|condo| +26473|AAAAAAAAJGHGAAAA|291|Central |Parkway|Suite 90|Mountain View|Grant County|KY|44466|United States|-6|condo| +26474|AAAAAAAAKGHGAAAA|771|River 6th|Boulevard|Suite Q|Crossroads|Ulster County|NY|10534|United States|-5|single family| +26475|AAAAAAAALGHGAAAA|126|Elm 3rd|Boulevard|Suite N|Newtown|Wichita County|TX|71749|United States|-6|apartment| +26476|AAAAAAAAMGHGAAAA|814|Maple Pine|Boulevard|Suite Y|Shiloh|Clarke County|AL|39275|United States|-6|single family| +26477|AAAAAAAANGHGAAAA|723|4th |ST|Suite D|Glenwood|Brown County|TX|73511|United States|-6|single family| +26478|AAAAAAAAOGHGAAAA|3|Second 8th|Circle|Suite B|Lincoln|Newton County|TX|71289|United States|-6|apartment| +26479|AAAAAAAAPGHGAAAA|976|West |Parkway|Suite E|Oakwood|Moore County|NC|20169|United States|-5|condo| +26480|AAAAAAAAAHHGAAAA|219|Forest |Blvd|Suite 230|Riverdale|Lyon County|MN|59391|United States|-6|apartment| +26481|AAAAAAAABHHGAAAA|327|North |Blvd|Suite 110|Concord|Matanuska-Susitna Borough|AK|94107|United States|-9|apartment| +26482|AAAAAAAACHHGAAAA|354|Pine |Wy|Suite G|Salem|Juab County|UT|88048|United States|-7|apartment| +26483|AAAAAAAADHHGAAAA|579|Willow College|Drive|Suite 80|Morgantown|Orange County|IN|49193|United States|-5|apartment| +26484|AAAAAAAAEHHGAAAA|369|Madison Hill|Way|Suite 140|Walnut Grove|Pawnee County|OK|77752|United States|-6|single family| +26485|AAAAAAAAFHHGAAAA|892|Ninth Highland|Ct.|Suite 40|Spring Hill|Page County|IA|56787|United States|-6|apartment| +26486|AAAAAAAAGHHGAAAA|332|Third |Wy|Suite 360|Friendship|Craven County|NC|24536|United States|-5|condo| +26487|AAAAAAAAHHHGAAAA|704|13th |Parkway|Suite S|Maple Grove|Morris County|KS|68252|United States|-6|condo| +26488|AAAAAAAAIHHGAAAA|630|Main |Lane|Suite 160|Wildwood|Nantucket County|MA|07471|United States|-5|apartment| +26489|AAAAAAAAJHHGAAAA|998|Forest |Ave|Suite 170|Concord|New Madrid County|MO|64107|United States|-6|apartment| +26490|AAAAAAAAKHHGAAAA|834|Franklin |Street|Suite N|Hopewell|Door County|WI|50587|United States|-6|apartment| +26491|AAAAAAAALHHGAAAA|831|Spruce 4th|Blvd|Suite 190|Ashland|Polk County|OR|94244|United States|-8|single family| +26492|AAAAAAAAMHHGAAAA|166|Lake |Avenue|Suite 170|Jenkins|Manassas city|VA|27292|United States|-5|single family| +26493|AAAAAAAANHHGAAAA|806|Park |Ct.|Suite 30|Bunker Hill|Morgan County|OH|40150|United States|-5|condo| +26494|AAAAAAAAOHHGAAAA|98|Church |Street|Suite 90|Oakland|Robertson County|KY|49843|United States|-5|single family| +26495|AAAAAAAAPHHGAAAA|965|2nd North|Avenue|Suite 130|Wildwood|Pulaski County|KY|46871|United States|-5|condo| +26496|AAAAAAAAAIHGAAAA|858|Oak View|Way|Suite E|Buena Vista|Roane County|TN|35752|United States|-6|single family| +26497|AAAAAAAABIHGAAAA|784|Fourth |Pkwy|Suite 20|Highland Park|Worcester County|MD|26534|United States|-5|condo| +26498|AAAAAAAACIHGAAAA|55|Fourth |Circle|Suite T|Valley View|Kenosha County|WI|55124|United States|-6|single family| +26499|AAAAAAAADIHGAAAA|70|Oak |Avenue|Suite 320|Oak Grove|Muskegon County|MI|48370|United States|-5|condo| +26500|AAAAAAAAEIHGAAAA|508|Lincoln Sycamore|Street|Suite 430|Mount Olive|Nicollet County|MN|58059|United States|-6|single family| +26501|AAAAAAAAFIHGAAAA|||||Mount Vernon||MO|68482|United States|-6|| +26502|AAAAAAAAGIHGAAAA|784|East |Street|Suite O|Oakland|Bleckley County|GA|39843|United States|-5|single family| +26503|AAAAAAAAHIHGAAAA|964|Oak 2nd|Way|Suite N|Green Acres|Putnam County|NY|17683|United States|-5|apartment| +26504|AAAAAAAAIIHGAAAA|431|Lincoln Church|Way|Suite B|Kingston|Cape May County|NJ|05575|United States|-5|apartment| +26505|AAAAAAAAJIHGAAAA|251|College |Court|Suite B|Woodlawn|Richmond city|VA|24098|United States|-5|apartment| +26506|AAAAAAAAKIHGAAAA|48|Davis |Ave|Suite C|Jamestown|Sharp County|AR|76867|United States|-6|condo| +26507|AAAAAAAALIHGAAAA|751|10th |Ave|Suite L|Oak Grove|Leflore County|MS|58370|United States|-6|condo| +26508|AAAAAAAAMIHGAAAA|679|Wilson Pine||||Choctaw County|AL|||-6|| +26509|AAAAAAAANIHGAAAA|268|Park |Boulevard|Suite M|Union|Mineral County|NV|88721|United States|-8|condo| +26510|AAAAAAAAOIHGAAAA|666|6th |Avenue|Suite 280|Oakwood|Wolfe County|KY|40169|United States|-5|single family| +26511|AAAAAAAAPIHGAAAA|366|Forest |Road|Suite 440|Salem|Chesapeake city|VA|28048|United States|-5|condo| +26512|AAAAAAAAAJHGAAAA|291|Railroad Laurel|Parkway|Suite 400|Oakdale|Newport News city|VA|29584|United States|-5|single family| +26513|AAAAAAAABJHGAAAA|857|5th Cherry|Ave|Suite T|Sumner|Dearborn County|IN|40519|United States|-5|condo| +26514|AAAAAAAACJHGAAAA|861|1st |Pkwy|Suite 300|Five Forks|Walker County|TX|72293|United States|-6|apartment| +26515|AAAAAAAADJHGAAAA|495|Chestnut Highland|Ave|Suite A|Greenville|Marshall County|WV|21387|United States|-5|apartment| +26516|AAAAAAAAEJHGAAAA|611|Maple 1st|Court|Suite X|Buena Vista|Monroe County|IL|65752|United States|-6|condo| +26517|AAAAAAAAFJHGAAAA|506|Eigth |Court|Suite M|Springdale|Valley County|NE|68883|United States|-7|condo| +26518|AAAAAAAAGJHGAAAA|950|1st Chestnut|Court|Suite 370|Providence|Oklahoma County|OK|76614|United States|-6|apartment| +26519|AAAAAAAAHJHGAAAA|419|River |||Clinton|Lafayette Parish|||United States|-6|| +26520|AAAAAAAAIJHGAAAA|426|North Green|Pkwy|Suite Y|Plainview|Harney County|OR|93683|United States|-8|condo| +26521|AAAAAAAAJJHGAAAA|786|Lake Washington|Blvd|Suite P|Clifton|Marshall County|KY|48014|United States|-5|condo| +26522|AAAAAAAAKJHGAAAA|952|Forest |Ave|Suite E|Ashland|Worcester County|MD|24244|United States|-5|single family| +26523|AAAAAAAALJHGAAAA|267|5th |Blvd|Suite 350|Mountain View|Westmoreland County|PA|14466|United States|-5|condo| +26524|AAAAAAAAMJHGAAAA|877|10th |Ct.|Suite 360|Cedar Grove|Person County|NC|20411|United States|-5|condo| +26525|AAAAAAAANJHGAAAA|310|River |Court|Suite 450|Sunnyside|Shelby County|MO|61952|United States|-6|apartment| +26526|AAAAAAAAOJHGAAAA|451|Church |RD|Suite J|Fairfield|Gladwin County|MI|46192|United States|-5|condo| +26527|AAAAAAAAPJHGAAAA|539|Sycamore Pine|Wy|Suite 150|Wilson|Bandera County|TX|76971|United States|-6|single family| +26528|AAAAAAAAAKHGAAAA|538|Dogwood |Blvd|Suite 0|Woodland|Karnes County|TX|74854|United States|-6|single family| +26529|AAAAAAAABKHGAAAA|490|Forest |RD|Suite 260|Buena Vista|Osceola County|MI|45752|United States|-5|condo| +26530|AAAAAAAACKHGAAAA|530|Laurel First|Boulevard|Suite 210|Red Hill|Greene County|OH|44338|United States|-5|apartment| +26531|AAAAAAAADKHGAAAA|991|North Lee|Way|Suite 120|Woodlawn|Custer County|OK|74098|United States|-6|single family| +26532|AAAAAAAAEKHGAAAA|812|Church Pine|Wy|Suite 130|Brownsville|Switzerland County|IN|49310|United States|-5|single family| +26533|AAAAAAAAFKHGAAAA|100|Jefferson Lake|Street|Suite W|Woodville|Chase County|NE|64289|United States|-6|apartment| +26534|AAAAAAAAGKHGAAAA|350|Poplar 10th|Circle|Suite O|Maple Grove|Cherokee County|SC|28252|United States|-5|apartment| +26535|AAAAAAAAHKHGAAAA|32|View Elm|Parkway|Suite B|Sugar Hill|Washington County|KS|65114|United States|-6|apartment| +26536|AAAAAAAAIKHGAAAA|558|2nd Park|Avenue|Suite D|Superior|Benton County|IN|42562|United States|-5|condo| +26537|AAAAAAAAJKHGAAAA|613|Fifth |Boulevard|Suite U|Farmington|Nottoway County|VA|29145|United States|-5|apartment| +26538|AAAAAAAAKKHGAAAA|593|Lee River|Blvd|Suite R|Riverside|Stark County|IL|69231|United States|-6|apartment| +26539|AAAAAAAALKHGAAAA|471|Smith |Avenue|Suite 180|Shady Grove|Lassen County|CA|92812|United States|-8|condo| +26540|AAAAAAAAMKHGAAAA|656|Walnut 9th|Parkway|Suite R|Providence|Grayson County|VA|26614|United States|-5|single family| +26541|AAAAAAAANKHGAAAA|202|Oak Lincoln|Wy|Suite 40|Richville|Okeechobee County|FL|35945|United States|-5|apartment| +26542|AAAAAAAAOKHGAAAA|240|West 5th|Circle|Suite 300|Glendale|Pulaski County|GA|33951|United States|-5|apartment| +26543|AAAAAAAAPKHGAAAA|996|Highland Sycamore||Suite 100||||76192||-6|| +26544|AAAAAAAAALHGAAAA|395|Sycamore 12th|Ln|Suite 430|Concord|Richland Parish|LA|74107|United States|-6|apartment| +26545|AAAAAAAABLHGAAAA|121|Pine 6th|Court|Suite I|Liberty|Loudoun County|VA|23451|United States|-5|single family| +26546|AAAAAAAACLHGAAAA|259|Johnson |ST|Suite 310|Mountain View|Garland County|AR|74466|United States|-6|single family| +26547|AAAAAAAADLHGAAAA|762|Green Ridge|Cir.|Suite 120|Lakeview|Clay County|NE|68579|United States|-6|condo| +26548|AAAAAAAAELHGAAAA|129|Franklin |Ave|Suite 140|Lynn|Fauquier County|VA|20407|United States|-5|single family| +26549|AAAAAAAAFLHGAAAA||Davis Lincoln||||Greenup County|KY|||-6|| +26550|AAAAAAAAGLHGAAAA|947|Spruce |Street|Suite 80|Union|Tompkins County|NY|18721|United States|-5|condo| +26551|AAAAAAAAHLHGAAAA|833|Maple |||Oakland||||United States||condo| +26552|AAAAAAAAILHGAAAA|931|Highland |Circle|Suite 170|Midway|Yates County|NY|11904|United States|-5|apartment| +26553|AAAAAAAAJLHGAAAA|856|College |Avenue|Suite 480|Arlington|Essex County|VA|26557|United States|-5|apartment| +26554|AAAAAAAAKLHGAAAA|139|Third |Ave|Suite C|Oakland|Lawrence County|KY|49843|United States|-5|single family| +26555|AAAAAAAALLHGAAAA|295|Green Second|RD|Suite 60|Salem|Muscogee County|GA|38048|United States|-5|apartment| +26556|AAAAAAAAMLHGAAAA|888|Broadway |Court|Suite V|Riverview|Yakima County|WA|99003|United States|-8|single family| +26557|AAAAAAAANLHGAAAA|275|Main Fifth|Ct.|Suite 440|Rankin|Clayton County|IA|52621|United States|-6|apartment| +26558|AAAAAAAAOLHGAAAA|691|Ash |Ln|Suite 300|Mountain View|Marion County|FL|34466|United States|-5|condo| +26559|AAAAAAAAPLHGAAAA|541|Meadow |Cir.|Suite 410|Mount Vernon|Reno County|KS|68482|United States|-6|apartment| +26560|AAAAAAAAAMHGAAAA|739|Elm |Pkwy|Suite A|Enterprise|Polk County|TN|31757|United States|-6|single family| +26561|AAAAAAAABMHGAAAA|616|Maple |Pkwy|Suite N|Antioch|Trempealeau County|WI|58605|United States|-6|apartment| +26562|AAAAAAAACMHGAAAA|843|Maple |Blvd|Suite F|Oak Ridge|Harrison County|IA|58371|United States|-6|single family| +26563|AAAAAAAADMHGAAAA|647|Hillcrest 13th|Dr.|Suite 40|Belmont|Terry County|TX|70191|United States|-6|condo| +26564|AAAAAAAAEMHGAAAA|123||RD||Glendale|Webster County|||United States||single family| +26565|AAAAAAAAFMHGAAAA|361|Sunset |Lane|Suite J|Vista|Kimble County|TX|74694|United States|-6|apartment| +26566|AAAAAAAAGMHGAAAA|944|Church |Road|Suite H|White Oak|Haakon County|SD|56668|United States|-7|apartment| +26567|AAAAAAAAHMHGAAAA|408||Way|Suite 280|||AR||United States||single family| +26568|AAAAAAAAIMHGAAAA|194|Main |Court|Suite 70|Union City|Barbour County|WV|28087|United States|-5|apartment| +26569|AAAAAAAAJMHGAAAA|769|||||New Hanover County||29003|||single family| +26570|AAAAAAAAKMHGAAAA|157|2nd North|RD|Suite O|Woodland|Dade County|GA|34854|United States|-5|apartment| +26571|AAAAAAAALMHGAAAA|946|Davis View|RD|Suite V|Unionville|Poweshiek County|IA|51711|United States|-6|apartment| +26572|AAAAAAAAMMHGAAAA|429|Pine 8th|Ln|Suite 140|Griffin|Geary County|KS|61204|United States|-6|condo| +26573|AAAAAAAANMHGAAAA|947|Broadway |Court|Suite 90|Lakeview|Orange County|NC|28579|United States|-5|condo| +26574|AAAAAAAAOMHGAAAA|264|Center |Blvd|Suite 180|Lincoln|Adams County|WA|91289|United States|-8|single family| +26575|AAAAAAAAPMHGAAAA|901|First |Road|Suite 480|Buena Vista|Kalamazoo County|MI|45752|United States|-5|single family| +26576|AAAAAAAAANHGAAAA|299|2nd Birch|Street|Suite A|Hamilton|Little River County|AR|72808|United States|-6|condo| +26577|AAAAAAAABNHGAAAA|422|Walnut Park|Avenue|Suite 60|Waterloo|Ketchikan Gateway Borough|AK|91675|United States|-9|condo| +26578|AAAAAAAACNHGAAAA|285|Lee Willow|Road|Suite D|Florence|Mercer County|PA|13394|United States|-5|condo| +26579|AAAAAAAADNHGAAAA|793|Hickory North|Drive|Suite I|Greenville|Fentress County|TN|31387|United States|-5|single family| +26580|AAAAAAAAENHGAAAA|149|Highland |Way|Suite 20|Hillcrest|Lake County|MT|63003|United States|-7|single family| +26581|AAAAAAAAFNHGAAAA|229|Ash |Drive|Suite 170|Oak Grove|Sully County|SD|58370|United States|-7|condo| +26582|AAAAAAAAGNHGAAAA|294|Elm |Court|Suite X|Cedar Grove|Whitley County|KY|40411|United States|-5|single family| +26583|AAAAAAAAHNHGAAAA|661|South |Circle|Suite X|Bunker Hill|Jackson County|MO|60150|United States|-6|apartment| +26584|AAAAAAAAINHGAAAA|146|Davis 10th|Lane|Suite 440|Clifton|Gibson County|IN|48014|United States|-5|condo| +26585|AAAAAAAAJNHGAAAA|527|Franklin Church|Way|Suite 130|Clinton|Shelby County|AL|38222|United States|-6|apartment| +26586|AAAAAAAAKNHGAAAA|38|Adams Park|Lane|Suite U|Unionville|Warren County|NY|11711|United States|-5|condo| +26587|AAAAAAAALNHGAAAA|929|Hill Valley|Wy|Suite J|Highland Park|Russell County|KS|66534|United States|-6|single family| +26588|AAAAAAAAMNHGAAAA|601|Fifth West|Cir.|Suite 380|Providence|Cobb County|GA|36614|United States|-5|apartment| +26589|AAAAAAAANNHGAAAA|91|Cherry |Wy|Suite 430|Stratford|Ouray County|CO|86668|United States|-7|condo| +26590|AAAAAAAAONHGAAAA|859|8th |Cir.|Suite 260|Oakland|Highland County|VA|29843|United States|-5|apartment| +26591|AAAAAAAAPNHGAAAA|699|||Suite N||Colorado County||79454|United States||| +26592|AAAAAAAAAOHGAAAA|564|South South|RD|Suite I|Empire|Cumberland County|NJ|04745|United States|-5|apartment| +26593|AAAAAAAABOHGAAAA|472|13th |Wy|Suite T|Valley View|Campbell County|VA|25124|United States|-5|apartment| +26594|AAAAAAAACOHGAAAA|950|Washington |Boulevard|Suite Y|Lincoln|Navajo County|AZ|81289|United States|-7|single family| +26595|AAAAAAAADOHGAAAA|343|3rd Cedar|Court|Suite 50|Woodville|Washington County|ID|84289|United States|-7|single family| +26596|AAAAAAAAEOHGAAAA|970|15th Wilson|Way|Suite I|Centerville|Carter County|TN|30059|United States|-5|apartment| +26597|AAAAAAAAFOHGAAAA|604|Second Elm|RD|Suite 60|Salem|Guadalupe County|NM|88048|United States|-7|condo| +26598|AAAAAAAAGOHGAAAA|568|Washington 4th|ST|Suite 130|Glendale|Adams County|IN|43951|United States|-5|single family| +26599|AAAAAAAAHOHGAAAA|569|College |Drive|Suite 10|Arlington|Linn County|MO|66557|United States|-6|single family| +26600|AAAAAAAAIOHGAAAA|304|Sycamore |Lane|Suite R|Liberty|Robertson County|TN|33451|United States|-6|single family| +26601|AAAAAAAAJOHGAAAA|638|Miller Dogwood|Lane|Suite R|Centerville|Mineral County|MT|60059|United States|-7|apartment| +26602|AAAAAAAAKOHGAAAA|969|Meadow Main|RD|Suite 220|Bunker Hill|Matagorda County|TX|70150|United States|-6|apartment| +26603|AAAAAAAALOHGAAAA|44|Second 4th|Parkway|Suite 200|Newport|Delaware County|OH|41521|United States|-5|apartment| +26604|AAAAAAAAMOHGAAAA|751|Main 8th|RD|Suite T|Clifton|Teton County|ID|88014|United States|-7|single family| +26605|AAAAAAAANOHGAAAA|233|Locust |Avenue|Suite 280|Riverside|Webster County|MO|69231|United States|-6|condo| +26606|AAAAAAAAOOHGAAAA|226|Willow Maple|Ln|Suite G|Centerville|Crenshaw County|AL|30059|United States|-6|single family| +26607|AAAAAAAAPOHGAAAA|134|Johnson 9th|Circle|Suite B|Pleasant Grove|Suffolk County|MA|04736|United States|-5|condo| +26608|AAAAAAAAAPHGAAAA|989|7th |Ln|Suite 320|Clinton|Itawamba County|MS|58222|United States|-6|apartment| +26609|AAAAAAAABPHGAAAA|653|Fourth |Pkwy|Suite 450|Springfield|Midland County|MI|49303|United States|-5|condo| +26610|AAAAAAAACPHGAAAA|957|4th Park|Dr.|Suite Q|Midway|Bibb County|AL|31904|United States|-6|single family| +26611|AAAAAAAADPHGAAAA|565|Adams Park|Blvd|Suite 10|Mount Pleasant|Lake of the Woods County|MN|51933|United States|-6|apartment| +26612|AAAAAAAAEPHGAAAA|115|View |Parkway|Suite 200|Five Forks|Houston County|AL|32293|United States|-6|condo| +26613|AAAAAAAAFPHGAAAA|618|2nd |Blvd|Suite 380|Frenchtown|Newton County|IN|42629|United States|-5|single family| +26614|AAAAAAAAGPHGAAAA|689|Hill |Court|Suite N|Oakwood|Boone County|IN|40169|United States|-5|condo| +26615|AAAAAAAAHPHGAAAA|242|1st |Ave|Suite X|Cedar Grove|Rockingham County|NC|20411|United States|-5|single family| +26616|AAAAAAAAIPHGAAAA|44|Main |Circle|Suite 320|Pleasant Hill|Bergen County|NJ|04204|United States|-5|apartment| +26617|AAAAAAAAJPHGAAAA|941|Church |Blvd|Suite L|Valley View|Thomas County|NE|65124|United States|-7|condo| +26618|AAAAAAAAKPHGAAAA|28|Eigth Washington|Cir.|Suite 220|Clifton|Knox County|MO|68014|United States|-6|single family| +26619|AAAAAAAALPHGAAAA|610|Meadow |Boulevard|Suite 380|Springfield|Denton County|TX|79303|United States|-6|apartment| +26620|AAAAAAAAMPHGAAAA||11th ||Suite 130|Bridgeport|Pike County|OH||United States||| +26621|AAAAAAAANPHGAAAA|297|Oak Broadway|Ct.|Suite E|Spring Hill|Concho County|TX|76787|United States|-6|single family| +26622|AAAAAAAAOPHGAAAA|168|Park Woodland|ST|Suite 460|Burns|Atascosa County|TX|75272|United States|-6|condo| +26623|AAAAAAAAPPHGAAAA|||Court|Suite O||Hancock County|||United States|-5|| +26624|AAAAAAAAAAIGAAAA|353|Smith |Parkway|Suite Y|Salem|Jefferson County|ID|88048|United States|-7|single family| +26625|AAAAAAAABAIGAAAA|485|Spruce 6th|Boulevard|Suite 100|Hopewell|Tioga County|PA|10587|United States|-5|condo| +26626|AAAAAAAACAIGAAAA|342|Oak 1st|Boulevard|Suite P|Enterprise|Vermilion Parish|LA|71757|United States|-6|single family| +26627|AAAAAAAADAIGAAAA|780|Willow |Cir.|||Walworth County||||-6|single family| +26628|AAAAAAAAEAIGAAAA||5th |Boulevard||||TX|74593|United States||| +26629|AAAAAAAAFAIGAAAA|467|7th |Way|Suite F|Pleasant Valley|Lancaster County|SC|22477|United States|-5|single family| +26630|AAAAAAAAGAIGAAAA|509|4th |Ave|Suite 140|Antioch|Wells County|ND|58605|United States|-6|single family| +26631|AAAAAAAAHAIGAAAA|961|2nd Elm|Road|Suite M|Mount Vernon|Jefferson County|KS|68482|United States|-6|apartment| +26632|AAAAAAAAIAIGAAAA|751|9th Birch|Drive|Suite T|Denmark|Adams County|WA|95576|United States|-8|apartment| +26633|AAAAAAAAJAIGAAAA|568|11th Sycamore|Road|Suite 290|Farmington|Hidalgo County|NM|89145|United States|-7|apartment| +26634|AAAAAAAAKAIGAAAA|291|9th 4th|Dr.|Suite R|Plainview|Mitchell County|GA|33683|United States|-5|apartment| +26635|AAAAAAAALAIGAAAA|763|Sycamore |Lane|Suite 90|Woodville|Skagit County|WA|94289|United States|-8|apartment| +26636|AAAAAAAAMAIGAAAA|451|Adams |Avenue|Suite H|Lakewood|Bryan County|GA|38877|United States|-5|condo| +26637|AAAAAAAANAIGAAAA|||Boulevard||Providence|||||-7|single family| +26638|AAAAAAAAOAIGAAAA|708|Ridge Oak|Cir.|Suite G|Pleasant Grove|Reno County|KS|64136|United States|-6|condo| +26639|AAAAAAAAPAIGAAAA|311|2nd |Wy|Suite 250|Lebanon|Eddy County|NM|82898|United States|-7|condo| +26640|AAAAAAAAABIGAAAA|479|Lake Madison|Street|Suite 140|Greenville|Jefferson County|NE|61387|United States|-7|single family| +26641|AAAAAAAABBIGAAAA|535|Lee |RD|Suite 160|Sutton|Bacon County|GA|35413|United States|-5|apartment| +26642|AAAAAAAACBIGAAAA|36|River |Dr.|Suite 450|Glenwood|Scott County|MN|53511|United States|-6|condo| +26643|AAAAAAAADBIGAAAA||Maple |Boulevard|Suite G|||||United States||apartment| +26644|AAAAAAAAEBIGAAAA|804|7th Ridge|Road|Suite F|Redland|Jasper County|IL|66343|United States|-6|condo| +26645|AAAAAAAAFBIGAAAA|552|Second Mill|Boulevard|Suite W|Jamestown|Borden County|TX|76867|United States|-6|condo| +26646|AAAAAAAAGBIGAAAA|486|Fourth |Drive|Suite A|Fairfield|Bath County|VA|26192|United States|-5|condo| +26647|AAAAAAAAHBIGAAAA|103|Franklin |Parkway|Suite 230|Lakewood|Greenwood County|KS|68877|United States|-6|apartment| +26648|AAAAAAAAIBIGAAAA|38|5th Spring|RD|Suite 160|Wilson|Prince William County|VA|26971|United States|-5|condo| +26649|AAAAAAAAJBIGAAAA||Dogwood |Ct.|Suite 290|Pleasant Grove|||||-6|| +26650|AAAAAAAAKBIGAAAA|362|Lee Lakeview|Ave|Suite 310|Woodville|Stone County|AR|74289|United States|-6|apartment| +26651|AAAAAAAALBIGAAAA|126|Miller |Way|Suite U|Belmont|Lancaster County|NE|60191|United States|-7|single family| +26652|AAAAAAAAMBIGAAAA|790|Walnut |Parkway|Suite E|Centerville|Allen County|IN|40059|United States|-5|single family| +26653|AAAAAAAANBIGAAAA|563|8th Walnut|Ln|Suite D|Harmony|Dare County|NC|25804|United States|-5|apartment| +26654|AAAAAAAAOBIGAAAA|6|West |Avenue|Suite 190|Lakewood|Cherokee County|KS|68877|United States|-6|apartment| +26655|AAAAAAAAPBIGAAAA|686|Center Elm|Boulevard|Suite H|Cedar Grove|Clayton County|IA|50411|United States|-6|apartment| +26656|AAAAAAAAACIGAAAA|205|Hillcrest Walnut|Blvd|Suite Q|Sunnyside|Moore County|TX|71952|United States|-6|apartment| +26657|AAAAAAAABCIGAAAA|502|Maple |ST|Suite D|Oak Grove|Lassen County|CA|98370|United States|-8|single family| +26658|AAAAAAAACCIGAAAA|927|Poplar |Blvd|Suite 160|Plainview|Bernalillo County|NM|83683|United States|-7|apartment| +26659|AAAAAAAADCIGAAAA|346|Church |Circle|Suite 260|Mount Pleasant|Clare County|MI|41933|United States|-5|condo| +26660|AAAAAAAAECIGAAAA|2|First Spring|Boulevard|Suite 60|Greenfield|Nicholas County|WV|25038|United States|-5|apartment| +26661|AAAAAAAAFCIGAAAA|349|Smith |Road|Suite 460|Clinton|Iron County|UT|88222|United States|-7|single family| +26662|AAAAAAAAGCIGAAAA|274|9th |Court|Suite P|Centerville|Plymouth County|MA|00659|United States|-5|apartment| +26663|AAAAAAAAHCIGAAAA|656|Walnut Willow|Road|Suite Q|Warwick|Petroleum County|MT|61398|United States|-7|single family| +26664|AAAAAAAAICIGAAAA|335|Sixth Woodland|Parkway|Suite M|Crossroads|Fisher County|TX|70534|United States|-6|condo| +26665|AAAAAAAAJCIGAAAA|751|Church |Ave|Suite 190|Armstrong|Atchison County|MO|60647|United States|-6|single family| +26666|AAAAAAAAKCIGAAAA|317|14th |ST|Suite G|Bunker Hill|Russell County|KS|60150|United States|-6|condo| +26667|AAAAAAAALCIGAAAA|171|3rd |Street|Suite 330|Little River|Pontotoc County|MS|50319|United States|-6|apartment| +26668|AAAAAAAAMCIGAAAA||Cherry |||Florence||||||| +26669|AAAAAAAANCIGAAAA|372|Sycamore |Ct.|Suite 50|Lakewood|Klamath County|OR|98877|United States|-8|apartment| +26670|AAAAAAAAOCIGAAAA|633|10th Main|Dr.|Suite G|Mountain View|Branch County|MI|44466|United States|-5|single family| +26671|AAAAAAAAPCIGAAAA|187|Mill |Ln|Suite 20|Cedar Grove|Hopewell city|VA|20411|United States|-5|single family| +26672|AAAAAAAAADIGAAAA|893|5th |Road|Suite 20|Little River|Coles County|IL|60319|United States|-6|apartment| +26673|AAAAAAAABDIGAAAA|558|River |Drive|Suite P|Sunnyside|Putnam County|TN|31952|United States|-6|apartment| +26674|AAAAAAAACDIGAAAA|749|North |Drive|Suite 310|Sulphur Springs|Crisp County|GA|38354|United States|-5|condo| +26675|AAAAAAAADDIGAAAA||12th River||Suite I|||||United States|-5|single family| +26676|AAAAAAAAEDIGAAAA|338|Locust Smith|Drive|Suite F|Franklin|Caldwell County|NC|29101|United States|-5|condo| +26677|AAAAAAAAFDIGAAAA|729|Cedar Cedar|RD|Suite E|Plainview|Lincoln County|NE|63683|United States|-7|single family| +26678|AAAAAAAAGDIGAAAA|214|Walnut |Street|Suite B|Union|Lumpkin County|GA|38721|United States|-5|condo| +26679|AAAAAAAAHDIGAAAA|162|Ash Cherry|Parkway|Suite 340|Crossroads|Minnehaha County|SD|50534|United States|-7|apartment| +26680|AAAAAAAAIDIGAAAA|659|9th Oak|ST|Suite 160|Coldwater|Flagler County|FL|33373|United States|-5|condo| +26681|AAAAAAAAJDIGAAAA|505|Main |Boulevard|Suite 80|Summit|Beltrami County|MN|50499|United States|-6|condo| +26682|AAAAAAAAKDIGAAAA|597|Cedar Main|Drive|Suite L|New Hope|Albany County|WY|89431|United States|-7|condo| +26683|AAAAAAAALDIGAAAA|338|Main 6th|Cir.|Suite F|Summit|Hardin County|TX|70499|United States|-6|condo| +26684|AAAAAAAAMDIGAAAA|985|Oak |Drive|Suite T|Plainview|Butte County|CA|93683|United States|-8|apartment| +26685|AAAAAAAANDIGAAAA|840|6th Adams|Cir.|Suite 480|Derby|Union County|IN|47702|United States|-5|condo| +26686|AAAAAAAAODIGAAAA|739|8th Davis|Blvd|Suite 360|White Oak|Iroquois County|IL|66668|United States|-6|single family| +26687|AAAAAAAAPDIGAAAA|353||RD|Suite F||Vance County||27746|||apartment| +26688|AAAAAAAAAEIGAAAA|802|Green |Drive|Suite C|Centerville|Kent County|TX|70059|United States|-6|condo| +26689|AAAAAAAABEIGAAAA|997|Hickory Dogwood|Road|Suite A|Pleasant Hill|Oconto County|WI|53604|United States|-6|condo| +26690|AAAAAAAACEIGAAAA|453|Main Miller|Dr.|Suite N|Kingston|Lyon County|KY|44975|United States|-5|condo| +26691|AAAAAAAADEIGAAAA|976|Sunset Birch|Ct.|Suite R|Taft|San Augustine County|TX|70589|United States|-6|condo| +26692|AAAAAAAAEEIGAAAA|551|Jefferson Johnson|Road|Suite 190|Lincoln|Harney County|OR|91289|United States|-8|condo| +26693|AAAAAAAAFEIGAAAA|439|9th College|Ln|Suite W|Williamsville|Albany County|WY|88754|United States|-7|single family| +26694|AAAAAAAAGEIGAAAA|416|9th |Ln|Suite 120|Wildwood|Madison County|GA|36871|United States|-5|apartment| +26695|AAAAAAAAHEIGAAAA|265|Church Church|Drive|Suite K|Sunnyside|Newport County|RI|02552|United States|-5|condo| +26696|AAAAAAAAIEIGAAAA|446|11th 11th|Boulevard|Suite D|Florence|Ringgold County|IA|53394|United States|-6|condo| +26697|AAAAAAAAJEIGAAAA|336|2nd North|Drive|Suite 120|Pleasant Valley|Harrisonburg city|VA|22477|United States|-5|single family| +26698|AAAAAAAAKEIGAAAA|480|Pine Maple|Blvd|Suite 30|Phoenix|Butler County|NE|62276|United States|-6|single family| +26699|AAAAAAAALEIGAAAA|622|Oak Elm|Ln|Suite 340|Jackson|Bucks County|PA|19583|United States|-5|single family| +26700|AAAAAAAAMEIGAAAA|116|First Birch|Boulevard|Suite 440|Harmony|Duval County|FL|35804|United States|-5|apartment| +26701|AAAAAAAANEIGAAAA|514|Poplar |Ct.|Suite K|Birmingham|Cobb County|GA|33372|United States|-5|apartment| +26702|AAAAAAAAOEIGAAAA|95|Fifth |Ct.|Suite D|Gravel Hill|Bell County|KY|41944|United States|-6|apartment| +26703|AAAAAAAAPEIGAAAA|475|14th |Avenue|Suite 470|Plainview|Garfield County|UT|83683|United States|-7|single family| +26704|AAAAAAAAAFIGAAAA|97|10th |Avenue|Suite 50|Highland Park|Chelan County|WA|96534|United States|-8|single family| +26705|AAAAAAAABFIGAAAA|136|7th |Avenue|Suite R|Mount Olive|Morgan County|CO|88059|United States|-7|condo| +26706|AAAAAAAACFIGAAAA|696|Hill Oak|Boulevard|Suite L|Mountain View|Boulder County|CO|84466|United States|-7|condo| +26707|AAAAAAAADFIGAAAA|868|3rd Jefferson|Way|Suite 200|Sunnyside|Washington County|CO|81952|United States|-7|apartment| +26708|AAAAAAAAEFIGAAAA|452|Spring |Pkwy|Suite H|Red Oak|Atlantic County|NJ|05618|United States|-5|condo| +26709|AAAAAAAAFFIGAAAA|292|Valley 15th|Ave|Suite W|Belmont|Curry County|NM|80191|United States|-7|condo| +26710|AAAAAAAAGFIGAAAA|177|Walnut 4th|Blvd|Suite 170|Riverside|Vermilion Parish|LA|79231|United States|-6|apartment| +26711|AAAAAAAAHFIGAAAA|133|10th |Way|Suite 50|Waterloo|Brown County|IN|41675|United States|-5|condo| +26712|AAAAAAAAIFIGAAAA|823|Fourteenth North|Court|Suite 170|Centerville|Manassas city|VA|20059|United States|-5|condo| +26713|AAAAAAAAJFIGAAAA|267|9th |Cir.|Suite 160|Jackson|Hampshire County|MA|09583|United States|-5|condo| +26714|AAAAAAAAKFIGAAAA|655|Washington |Parkway|Suite S|Bethel|Scott County|VA|25281|United States|-5|apartment| +26715|AAAAAAAALFIGAAAA|871|Third |Court|Suite 200|Walnut Grove|Union County|AR|77752|United States|-6|single family| +26716|AAAAAAAAMFIGAAAA|940|2nd |Blvd|Suite E|Lakeview|Lexington city|VA|28579|United States|-5|single family| +26717|AAAAAAAANFIGAAAA|801|Sunset |Pkwy|Suite 80|Maple Grove|Arthur County|NE|68252|United States|-6|single family| +26718|AAAAAAAAOFIGAAAA|935|Center 1st|Pkwy|Suite G|Brunswick|Wadena County|MN|54642|United States|-6|condo| +26719|AAAAAAAAPFIGAAAA|453|14th |Way|Suite 250|Farmersville|Chippewa County|WI|59305|United States|-6|single family| +26720|AAAAAAAAAGIGAAAA|779|Adams |Pkwy|Suite O|Lakewood|Harvey County|KS|68877|United States|-6|apartment| +26721|AAAAAAAABGIGAAAA|774|Main |Pkwy|Suite 360|Mount Olive|Overton County|TN|38059|United States|-6|condo| +26722|AAAAAAAACGIGAAAA|784|Seventh |Road|Suite T|Shiloh|Midland County|TX|79275|United States|-6|single family| +26723|AAAAAAAADGIGAAAA|775|Ridge |Ln|Suite Y|Oak Grove|Sheridan County|KS|68370|United States|-6|apartment| +26724|AAAAAAAAEGIGAAAA|805|Oak Broadway|Wy|Suite 330|Tanglewood|Gasconade County|MO|68994|United States|-6|condo| +26725|AAAAAAAAFGIGAAAA|114|Washington Ridge|RD|Suite L|Clifton|Monmouth County|NJ|08614|United States|-5|apartment| +26726|AAAAAAAAGGIGAAAA|371|Railroad |Avenue|Suite 220|Millbrook|Los Angeles County|CA|97529|United States|-8|single family| +26727|AAAAAAAAHGIGAAAA|815|Fourteenth |Ln|Suite B|Friendship|Hendry County|FL|34536|United States|-5|single family| +26728|AAAAAAAAIGIGAAAA|207|||Suite 110|||||||| +26729|AAAAAAAAJGIGAAAA|999|Church Highland|Street|Suite 100|Woodville|Alachua County|FL|34289|United States|-5|condo| +26730|AAAAAAAAKGIGAAAA|482|Center |Lane|Suite 290|Walton|Brown County|IN|46389|United States|-5|apartment| +26731|AAAAAAAALGIGAAAA|291|Hill |Wy|Suite 250|Green Acres|Baxter County|AR|77683|United States|-6|single family| +26732|AAAAAAAAMGIGAAAA|418|Pine |Court|Suite X|Oakwood|Clarendon County|SC|20169|United States|-5|apartment| +26733|AAAAAAAANGIGAAAA|445||Dr.||||GA||United States||| +26734|AAAAAAAAOGIGAAAA|469|Hill River|Dr.|Suite 170|Centerville|Logan County|ND|50059|United States|-6|condo| +26735|AAAAAAAAPGIGAAAA|720|Pine |Parkway|Suite A|Bridgeport|Red Willow County|NE|65817|United States|-7|single family| +26736|AAAAAAAAAHIGAAAA|805|Broadway 10th|Ct.|Suite 170|Riverdale|Worth County|GA|39391|United States|-5|apartment| +26737|AAAAAAAABHIGAAAA|676|Cedar |Ave|Suite 370|Edgewood|Mason County|WA|90069|United States|-8|single family| +26738|AAAAAAAACHIGAAAA|956|Main |Court|Suite 350|Lakewood|Henrico County|VA|28877|United States|-5|apartment| +26739|AAAAAAAADHIGAAAA|286|Lee |Circle|Suite 280|Springfield|Lincoln County|ME|09903|United States|-5|single family| +26740|AAAAAAAAEHIGAAAA|593|4th Washington|Avenue|Suite M|Pine Valley|New Haven County|CT|08809|United States|-5|single family| +26741|AAAAAAAAFHIGAAAA|602|Second |Street|Suite 300|Woodville|Burleigh County|ND|54289|United States|-6|apartment| +26742|AAAAAAAAGHIGAAAA|118|Hill |Cir.|Suite 480|Edgewood|Sevier County|TN|30069|United States|-6|apartment| +26743|AAAAAAAAHHIGAAAA|249|Oak |Blvd|Suite U|Ferguson|Clayton County|IA|51821|United States|-6|condo| +26744|AAAAAAAAIHIGAAAA|434|Williams |Street|Suite 450|Concord|Lee County|TX|74107|United States|-6|apartment| +26745|AAAAAAAAJHIGAAAA|59|Park Railroad|Dr.|Suite F|Spring Hill|Story County|IA|56787|United States|-6|apartment| +26746|AAAAAAAAKHIGAAAA|866|Lake Laurel|Way|Suite J|Jackson|Stanton County|NE|69583|United States|-7|single family| +26747|AAAAAAAALHIGAAAA|723|Seventh |Wy|Suite L|Lakewood|Cass County|IN|48877|United States|-5|single family| +26748|AAAAAAAAMHIGAAAA|841|Main Miller|Pkwy|Suite T|Pleasant Hill|Crawford County|AR|73604|United States|-6|condo| +26749|AAAAAAAANHIGAAAA|230|Washington |Pkwy|Suite J|Hidden Valley|Spencer County|KY|45521|United States|-5|single family| +26750|AAAAAAAAOHIGAAAA|858|Second |Blvd|Suite 490|Antioch|Menard County|TX|78605|United States|-6|condo| +26751|AAAAAAAAPHIGAAAA|148|Oak |Circle|Suite R|Centerville|Stutsman County|ND|50059|United States|-6|single family| +26752|AAAAAAAAAIIGAAAA|823|13th |Wy|Suite Q|Lincoln|Franklin County|MO|61289|United States|-6|apartment| +26753|AAAAAAAABIIGAAAA|476|Ninth |Court|Suite 140|Greenfield|Scurry County|TX|75038|United States|-6|condo| +26754|AAAAAAAACIIGAAAA|275|Spring 13th|RD|Suite F|Belmont|Durham County|NC|20191|United States|-5|single family| +26755|AAAAAAAADIIGAAAA|814|Johnson |Ave|Suite 190|The Meadows|Pennington County|SD|50026|United States|-7|single family| +26756|AAAAAAAAEIIGAAAA|940|Main Main|Drive|Suite T|Midway|Winnebago County|WI|51904|United States|-6|condo| +26757|AAAAAAAAFIIGAAAA|968|Main Sunset|ST|Suite 280|Woodlawn|Saunders County|NE|64098|United States|-7|single family| +26758|AAAAAAAAGIIGAAAA|249|Highland |Blvd|Suite 20|Edgewood|Iron County|UT|80069|United States|-7|single family| +26759|AAAAAAAAHIIGAAAA|925|7th Fourth|Cir.|Suite 420|Walnut Grove|Worcester County|MD|27752|United States|-5|condo| +26760|AAAAAAAAIIIGAAAA|240|Valley 4th|Lane|Suite O|Spring Hill|Union County|KY|46787|United States|-5|condo| +26761|AAAAAAAAJIIGAAAA|955|3rd |Drive|Suite 60|Enterprise|Lincoln County|NC|21757|United States|-5|condo| +26762|AAAAAAAAKIIGAAAA|983|Mill Johnson|ST|Suite 310|Pleasant Valley|Wirt County|WV|22477|United States|-5|condo| +26763|AAAAAAAALIIGAAAA|835|Miller Lakeview|Avenue|Suite R|Mineral Springs|Mecosta County|MI|41686|United States|-5|single family| +26764|AAAAAAAAMIIGAAAA|617|Park |Boulevard|Suite C|Oak Ridge|Obion County|TN|38371|United States|-6|apartment| +26765|AAAAAAAANIIGAAAA|414|River South|Parkway|Suite 460|Spring Hill|Kimble County|TX|76787|United States|-6|condo| +26766|AAAAAAAAOIIGAAAA|217|East River|Street|Suite A|Fairfield|Curry County|NM|86192|United States|-7|single family| +26767|AAAAAAAAPIIGAAAA|622|Wilson Jefferson|Lane|Suite 360|Five Points|Morgan County|CO|86098|United States|-7|single family| +26768|AAAAAAAAAJIGAAAA|65|1st |Lane|Suite 120|Concord|Lee County|NC|24107|United States|-5|single family| +26769|AAAAAAAABJIGAAAA|813|Railroad Dogwood|Road|Suite Q|New Town|Beadle County|SD|59634|United States|-6|apartment| +26770|AAAAAAAACJIGAAAA|389|13th |Blvd|Suite 310|Fairfield|Kenedy County|TX|76192|United States|-6|apartment| +26771|AAAAAAAADJIGAAAA|831|Forest |Wy|Suite A|Ashland|Wabash County|IL|64244|United States|-6|single family| +26772|AAAAAAAAEJIGAAAA|698|3rd 1st|ST|Suite H|Mountain View|Reno County|KS|64466|United States|-6|single family| +26773|AAAAAAAAFJIGAAAA|651|South Fourth|Pkwy|Suite G|Arlington|Fulton County|KY|46557|United States|-6|condo| +26774|AAAAAAAAGJIGAAAA|389|Wilson 12th|Lane|Suite 60|Crestview|Coffee County|GA|31421|United States|-5|condo| +26775|AAAAAAAAHJIGAAAA|171|East |Avenue|Suite D|Stringtown|Park County|WY|80162|United States|-7|condo| +26776|AAAAAAAAIJIGAAAA|364|Fifth Hillcrest|Blvd|Suite J|Summit|Iroquois County|IL|60499|United States|-6|apartment| +26777|AAAAAAAAJJIGAAAA|621|Willow College|Dr.|Suite 120|Saratoga|Yellowstone County|MT|62123|United States|-7|single family| +26778|AAAAAAAAKJIGAAAA|284|Ninth North|Avenue|Suite M|Newtown|Eureka County|NV|81749|United States|-8|condo| +26779|AAAAAAAALJIGAAAA|562|Franklin |Wy|Suite 490|Maple Grove|Yellowstone County|MT|68252|United States|-7|single family| +26780|AAAAAAAAMJIGAAAA|544|South |Road|Suite R|Riverview|Stewart County|GA|39003|United States|-5|single family| +26781|AAAAAAAANJIGAAAA|893|Oak |Circle|Suite 100|Hillcrest|Nicollet County|MN|53003|United States|-6|apartment| +26782|AAAAAAAAOJIGAAAA|787|1st |Road|Suite S|Newtown|Waupaca County|WI|51749|United States|-6|condo| +26783|AAAAAAAAPJIGAAAA|993|Smith Fourth|RD|Suite L|Union|Wayne County|IN|48721|United States|-5|single family| +26784|AAAAAAAAAKIGAAAA|751|Cherry River|Ct.|Suite 260|Marion|Lincoln County|KS|60399|United States|-6|condo| +26785|AAAAAAAABKIGAAAA|50|Lincoln Mill|Road|Suite 350|Pine Grove|Red Willow County|NE|64593|United States|-7|condo| +26786|AAAAAAAACKIGAAAA|716|3rd Hill|Boulevard|Suite 340|Oak Hill|Bland County|VA|27838|United States|-5|single family| +26787|AAAAAAAADKIGAAAA|474|Willow |Ct.|Suite 80|Franklin|Bell County|KY|49101|United States|-6|single family| +26788|AAAAAAAAEKIGAAAA|291|Park |Ct.|Suite I|New Hope|Putnam County|OH|49431|United States|-5|condo| +26789|AAAAAAAAFKIGAAAA|492|Spring Meadow|Street|Suite J|Riverdale|Colbert County|AL|39391|United States|-6|single family| +26790|AAAAAAAAGKIGAAAA|505|1st Washington|Ln|Suite T|Jerome|Gallatin County|IL|69920|United States|-6|condo| +26791|AAAAAAAAHKIGAAAA|873|Washington Cedar|Avenue|Suite H|Pleasant Valley|Sevier County|TN|32477|United States|-6|condo| +26792|AAAAAAAAIKIGAAAA|626|Willow |Blvd|Suite Y|Oakwood|DeKalb County|TN|30169|United States|-5|condo| +26793|AAAAAAAAJKIGAAAA|108|Jefferson 1st|Circle|Suite P|Riverdale|Merrimack County|NH|09991|United States|-5|apartment| +26794|AAAAAAAAKKIGAAAA||Washington ||Suite E||Platte County|||United States|-6|| +26795|AAAAAAAALKIGAAAA||Lake |Parkway||Harmony|Rutherford County|||United States|-5|| +26796|AAAAAAAAMKIGAAAA|259|Mill |Parkway|Suite 370|Bridgeport|Berkeley County|WV|25817|United States|-5|single family| +26797|AAAAAAAANKIGAAAA|467|Franklin West|Cir.|Suite 490|Florence|Covington city|VA|23394|United States|-5|apartment| +26798|AAAAAAAAOKIGAAAA|503|5th |Parkway|Suite 200|Ashland|Perry County|IL|64244|United States|-6|single family| +26799|AAAAAAAAPKIGAAAA|622|Sunset Cedar|Ave|Suite 360|Wildwood|Davis County|UT|86871|United States|-7|condo| +26800|AAAAAAAAALIGAAAA|220|Tenth |Way|Suite 80|Riverside|Lehigh County|PA|19231|United States|-5|condo| +26801|AAAAAAAABLIGAAAA|111|Park 11th|Street|Suite F|Fairview|Sandoval County|NM|85709|United States|-7|condo| +26802|AAAAAAAACLIGAAAA|746|Highland Willow|ST|Suite Q|Brookwood|Fulton County|IL|60965|United States|-6|condo| +26803|AAAAAAAADLIGAAAA|550|Johnson |ST|Suite B|Centerville|Faribault County|MN|50059|United States|-6|condo| +26804|AAAAAAAAELIGAAAA|477|8th 1st|Ct.|Suite 390|Clifton|Allen Parish|LA|78014|United States|-6|condo| +26805|AAAAAAAAFLIGAAAA|166|Mill Third|Cir.|Suite 480|Ashley|Brown County|IN|44324|United States|-5|single family| +26806|AAAAAAAAGLIGAAAA|860|Green Washington|Parkway|Suite 430|Mountain View|Caroline County|VA|24466|United States|-5|single family| +26807|AAAAAAAAHLIGAAAA|164|Jackson |Court|Suite L|Sulphur Springs|Wayne County|NE|68354|United States|-7|single family| +26808|AAAAAAAAILIGAAAA|109|Highland |Road|Suite S|Shiloh|Parmer County|TX|79275|United States|-6|single family| +26809|AAAAAAAAJLIGAAAA|296|Main Mill|Circle|Suite V|Calhoun|Gaines County|TX|76909|United States|-6|condo| +26810|AAAAAAAAKLIGAAAA|313|6th East|Ct.|Suite L|Hurricane|Washington County|KS|67644|United States|-6|condo| +26811|AAAAAAAALLIGAAAA|872|Elm |Pkwy|Suite 20|Lenox|Douglas County|GA|31143|United States|-5|single family| +26812|AAAAAAAAMLIGAAAA|405|Washington Franklin|Pkwy|Suite 370|Oakwood|Defiance County|OH|40169|United States|-5|condo| +26813|AAAAAAAANLIGAAAA|93|Main Oak|Cir.|Suite 200|Macedonia|Galveston County|TX|71087|United States|-6|apartment| +26814|AAAAAAAAOLIGAAAA|621|Spruce |Wy|Suite Q|Salem|Washington Parish|LA|78048|United States|-6|single family| +26815|AAAAAAAAPLIGAAAA|106|Oak Cedar|Ln|Suite 190|Brookwood|Allen County|IN|40965|United States|-5|condo| +26816|AAAAAAAAAMIGAAAA|296|3rd |Ct.|Suite 460|Georgetown|Uvalde County|TX|77057|United States|-6|apartment| +26817|AAAAAAAABMIGAAAA|288|South |Boulevard|Suite V|Shiloh|Gaines County|TX|79275|United States|-6|apartment| +26818|AAAAAAAACMIGAAAA|541|Elm |Ct.|Suite 420|Springfield|Merrimack County|NH|09903|United States|-5|condo| +26819|AAAAAAAADMIGAAAA|9|Maple |Ct.|Suite L|White Oak|Glenn County|CA|96668|United States|-8|apartment| +26820|AAAAAAAAEMIGAAAA|108|Cherry |Boulevard|Suite 120|Granite|De Soto Parish|LA|76284|United States|-6|apartment| +26821|AAAAAAAAFMIGAAAA|425|13th Main|Pkwy|Suite H|Wilson|Ashland County|OH|46971|United States|-5|apartment| +26822|AAAAAAAAGMIGAAAA|747|Ash Central|Circle|Suite H|Forest Hills|Fayette County|PA|19237|United States|-5|condo| +26823|AAAAAAAAHMIGAAAA|190|Tenth Center|Boulevard|Suite 310|Sunnyside|Pulaski County|IL|61952|United States|-6|single family| +26824|AAAAAAAAIMIGAAAA|977|Lee Park|RD|Suite K|Woodland|Etowah County|AL|34854|United States|-6|apartment| +26825|AAAAAAAAJMIGAAAA|330|North East|Ct.|Suite 430|Hopewell|Terry County|TX|70587|United States|-6|single family| +26826|AAAAAAAAKMIGAAAA|162|Lincoln |Circle|Suite T|Mount Pleasant|Coffey County|KS|61933|United States|-6|condo| +26827|AAAAAAAALMIGAAAA|54|Central |Lane|Suite 470|Mount Pleasant|Gilpin County|CO|81933|United States|-7|condo| +26828|AAAAAAAAMMIGAAAA|262|Chestnut |Avenue|Suite 270|Buena Vista|Bergen County|NJ|06352|United States|-5|single family| +26829|AAAAAAAANMIGAAAA|199|Cedar Oak|Court|Suite 0|Kingston|Richmond city|VA|24975|United States|-5|single family| +26830|AAAAAAAAOMIGAAAA|540|Main 13th|Ct.|Suite 370|Woodlawn|Hancock County|MS|54098|United States|-6|apartment| +26831|AAAAAAAAPMIGAAAA|421|Church |Circle|Suite X|Farmington|Izard County|AR|79145|United States|-6|condo| +26832|AAAAAAAAANIGAAAA|689|Davis Railroad|Ave|Suite 380|Fairfield|Gray County|KS|66192|United States|-6|condo| +26833|AAAAAAAABNIGAAAA|81|Mill Tenth|Road|Suite M|Mount Olive|Smith County|TN|38059|United States|-6|single family| +26834|AAAAAAAACNIGAAAA|937|Wilson |Drive|Suite V|Lebanon|Livingston County|IL|62898|United States|-6|apartment| +26835|AAAAAAAADNIGAAAA|141|Fifth |Boulevard|Suite 30|Sunnyside|Saluda County|SC|21952|United States|-5|apartment| +26836|AAAAAAAAENIGAAAA|712|7th 11th|Boulevard|Suite 190|Red Hill|Tangipahoa Parish|LA|74338|United States|-6|single family| +26837|AAAAAAAAFNIGAAAA|584|Center Wilson||||Bureau County|||United States||condo| +26838|AAAAAAAAGNIGAAAA|74|12th Oak|Road|Suite 0|Mount Pleasant|Hidalgo County|TX|71933|United States|-6|condo| +26839|AAAAAAAAHNIGAAAA|215|Valley Washington|Pkwy|Suite K|Edwards|Evangeline Parish|LA|71409|United States|-6|apartment| +26840|AAAAAAAAINIGAAAA|666|Elm |Boulevard|Suite 410|Spring Hill|Story County|IA|56787|United States|-6|apartment| +26841|AAAAAAAAJNIGAAAA|355|Lincoln |Circle|Suite T|Providence|Nolan County|TX|76614|United States|-6|single family| +26842|AAAAAAAAKNIGAAAA|657|Lincoln |Parkway|Suite 430|Crossroads|Le Sueur County|MN|50534|United States|-6|single family| +26843|AAAAAAAALNIGAAAA|||Boulevard|Suite Y|||||United States|-6|condo| +26844|AAAAAAAAMNIGAAAA|764|Ninth |Parkway|Suite 220|Shiloh|Greene County|PA|19275|United States|-5|single family| +26845|AAAAAAAANNIGAAAA|384|Park Oak|Way|Suite M|Bethel|DeKalb County|IL|65281|United States|-6|condo| +26846|AAAAAAAAONIGAAAA|849|8th Poplar|RD|Suite 360|Mount Olive|Campbell County|KY|48059|United States|-6|apartment| +26847|AAAAAAAAPNIGAAAA|903|3rd ||||Briscoe County|TX|74593|United States|-6|apartment| +26848|AAAAAAAAAOIGAAAA|512||Ave|Suite W||Crosby County||78482||-6|single family| +26849|AAAAAAAABOIGAAAA|219|Third 10th|Lane|Suite H|Indian Village|Lane County|OR|91075|United States|-8|condo| +26850|AAAAAAAACOIGAAAA|20|7th |RD|Suite 50|Lakewood|Whitman County|WA|98877|United States|-8|single family| +26851|AAAAAAAADOIGAAAA|835|Franklin Oak|RD|Suite Q|Forest Hills|Pope County|MN|59237|United States|-6|single family| +26852|AAAAAAAAEOIGAAAA|234|Park River|Wy|Suite 170|Hopewell|Jefferson County|ID|80587|United States|-7|apartment| +26853|AAAAAAAAFOIGAAAA|673|North |Blvd|Suite 20|Five Points|Albemarle County|VA|26098|United States|-5|apartment| +26854|AAAAAAAAGOIGAAAA|353|Mill Pine|Ct.|Suite J|Greenfield|Washington County|CO|85038|United States|-7|apartment| +26855|AAAAAAAAHOIGAAAA|219|5th Sycamore|Ln|Suite C|Riverview|Jasper County|GA|39003|United States|-5|condo| +26856|AAAAAAAAIOIGAAAA|56|West |Circle|Suite I|Oakdale|Spalding County|GA|39584|United States|-5|condo| +26857|AAAAAAAAJOIGAAAA|802|Wilson |Boulevard|Suite 200|Union Hill|Warren County|GA|37746|United States|-5|condo| +26858|AAAAAAAAKOIGAAAA|254|Main Elm|Ct.|Suite O|Spring Valley|Gaston County|NC|26060|United States|-5|single family| +26859|AAAAAAAALOIGAAAA||||Suite 160|Sulphur Springs||PA||United States||single family| +26860|AAAAAAAAMOIGAAAA|887|13th 14th|Ln|Suite G|Summit|Daniels County|MT|60499|United States|-7|apartment| +26861|AAAAAAAANOIGAAAA|307|Jackson Spring|Dr.|Suite 20|Mount Pleasant|Montgomery County|TX|71933|United States|-6|apartment| +26862|AAAAAAAAOOIGAAAA|813|Cedar Ridge|Ln|Suite E|Pleasant Hill|Bartholomew County|IN|43604|United States|-5|single family| +26863|AAAAAAAAPOIGAAAA|386|Third |Blvd|Suite 480|Lakewood|Garfield County|OK|78877|United States|-6|condo| +26864|AAAAAAAAAPIGAAAA|724|||Suite F||Centre County|PA|10069|||| +26865|AAAAAAAABPIGAAAA|163|East |Cir.|Suite 170|Oak Grove|Madison County|MO|68370|United States|-6|apartment| +26866|AAAAAAAACPIGAAAA|598|Maple |Road|Suite 460|Providence|Salt Lake County|UT|86614|United States|-7|apartment| +26867|AAAAAAAADPIGAAAA|921||Ct.|Suite J|Birmingham|Terrell County|||United States||| +26868|AAAAAAAAEPIGAAAA|556|1st |Ct.|Suite 90|Mount Zion|Perry County|IL|68054|United States|-6|single family| +26869|AAAAAAAAFPIGAAAA|688|Railroad |Ave|Suite 140|Providence|Chambers County|TX|76614|United States|-6|single family| +26870|AAAAAAAAGPIGAAAA||Park Hickory|||Stafford|Washington County||||-5|| +26871|AAAAAAAAHPIGAAAA|341|Sunset |Ln|Suite A|Phoenix|Clinton County|MO|62276|United States|-6|single family| +26872|AAAAAAAAIPIGAAAA|607|Cedar |Circle|Suite H|Lakeside|Austin County|TX|79532|United States|-6|apartment| +26873|AAAAAAAAJPIGAAAA|360|Pine |Blvd|Suite Q|Oakland|Brewster County|TX|79843|United States|-6|single family| +26874|AAAAAAAAKPIGAAAA|897|Woodland |Lane|Suite H|Oakdale|Cherokee County|AL|39584|United States|-6|condo| +26875|AAAAAAAALPIGAAAA|873|Woodland 2nd|Dr.|Suite 130|Unionville|Sheridan County|KS|61711|United States|-6|single family| +26876|AAAAAAAAMPIGAAAA|612|Valley Fifteenth|Blvd|Suite 120|Oakdale|Southampton County|VA|29584|United States|-5|apartment| +26877|AAAAAAAANPIGAAAA|570|Lake |Lane|Suite 20|Woodville|Clay County|FL|34289|United States|-5|apartment| +26878|AAAAAAAAOPIGAAAA|293|South |Boulevard|Suite K|Liberty|Stokes County|NC|23451|United States|-5|apartment| +26879|AAAAAAAAPPIGAAAA|885|Hillcrest Ash|Ln|Suite 330|Colonial Heights|Grant County|OR|93425|United States|-8|single family| +26880|AAAAAAAAAAJGAAAA|496|Forest |ST|Suite L|Saratoga|Lumpkin County|GA|32123|United States|-5|apartment| +26881|AAAAAAAABAJGAAAA|644|Washington |Dr.|Suite 50|Summerfield|McMullen County|TX|70634|United States|-6|apartment| +26882|AAAAAAAACAJGAAAA|306|Ridge |Dr.|Suite 420|Green Acres|Cass County|MN|57683|United States|-6|condo| +26883|AAAAAAAADAJGAAAA|239|Johnson |Wy|Suite L|Oakland|Fremont County|IA|59843|United States|-6|apartment| +26884|AAAAAAAAEAJGAAAA|742|Hickory |Drive|Suite 380|Sunnyside|Lapeer County|MI|41952|United States|-5|single family| +26885|AAAAAAAAFAJGAAAA|163|Park Highland|Ct.|Suite 60|Union|Donley County|TX|78721|United States|-6|apartment| +26886|AAAAAAAAGAJGAAAA|267|4th West|Road|Suite 10|Edgewood|Watauga County|NC|20069|United States|-5|single family| +26887|AAAAAAAAHAJGAAAA|376|Eigth Lee|RD|Suite 110|Bridgeport|Searcy County|AR|75817|United States|-6|single family| +26888|AAAAAAAAIAJGAAAA|935|9th Meadow|Pkwy|Suite 150|Walnut Grove|Barry County|MO|67752|United States|-6|condo| +26889|AAAAAAAAJAJGAAAA|529|Cherry Park|Parkway|Suite 440|Riverview|Lauderdale County|TN|39003|United States|-6|condo| +26890|AAAAAAAAKAJGAAAA|749|Pine Jefferson|Wy|Suite 450|Walnut Grove|Kimball County|NE|67752|United States|-7|single family| +26891|AAAAAAAALAJGAAAA|107|Hillcrest 5th|ST|Suite J|Shiloh|Bradford County|PA|19275|United States|-5|single family| +26892|AAAAAAAAMAJGAAAA|130|College 13th|Dr.|Suite D|Stringtown|Hennepin County|MN|50162|United States|-6|condo| +26893|AAAAAAAANAJGAAAA|56|8th Lincoln|Lane|Suite F|Florence|Marin County|CA|93394|United States|-8|single family| +26894|AAAAAAAAOAJGAAAA|364|Hillcrest |Lane|Suite Q|Mount Zion|Orange County|IN|48054|United States|-5|apartment| +26895|AAAAAAAAPAJGAAAA|812|West Pine|Street|Suite K|Bunker Hill|Pottawattamie County|IA|50150|United States|-6|single family| +26896|AAAAAAAAABJGAAAA||Pine Pine|Ln|||Ionia County||43451|||| +26897|AAAAAAAABBJGAAAA|339|View |Boulevard|Suite S|Centerville|Tehama County|CA|90059|United States|-8|apartment| +26898|AAAAAAAACBJGAAAA|783|8th Lee|Dr.|Suite R|Woodville|Val Verde County|TX|74289|United States|-6|single family| +26899|AAAAAAAADBJGAAAA|414|2nd |Dr.|Suite 140|Centerville|Benton County|MO|60059|United States|-6|single family| +26900|AAAAAAAAEBJGAAAA|||Pkwy|Suite W|Woodlawn|Jackson County||||-5|condo| +26901|AAAAAAAAFBJGAAAA|435|West |RD|Suite O|Mount Olive|Union County|SD|58059|United States|-7|single family| +26902|AAAAAAAAGBJGAAAA|826|West |ST|Suite I|Farmington|Buffalo County|WI|59145|United States|-6|single family| +26903|AAAAAAAAHBJGAAAA|534|Ash |Parkway|Suite D|Greenfield|Oktibbeha County|MS|55038|United States|-6|apartment| +26904|AAAAAAAAIBJGAAAA|899|South Lee|Avenue|Suite O|Greenwood|Cortland County|NY|18828|United States|-5|apartment| +26905|AAAAAAAAJBJGAAAA|715|Fourth 9th|Blvd|Suite C|Kingston|Salem city|VA|24975|United States|-5|apartment| +26906|AAAAAAAAKBJGAAAA|269|2nd |Dr.|Suite N|Marion|Lincoln County|NC|20399|United States|-5|condo| +26907|AAAAAAAALBJGAAAA|707|Central Walnut|Lane|Suite 220|Frogtown|Madison County|AL|38784|United States|-6|single family| +26908|AAAAAAAAMBJGAAAA|184|Adams |Street|Suite I|Clinton|Newton County|MO|68222|United States|-6|apartment| +26909|AAAAAAAANBJGAAAA|309|3rd Meadow|Ave|Suite H|Sheffield|Liberty County|MT|66896|United States|-7|condo| +26910|AAAAAAAAOBJGAAAA|142|Walnut Spring|Dr.|Suite 10|Crossroads|Buchanan County|VA|20534|United States|-5|single family| +26911|AAAAAAAAPBJGAAAA|123|2nd |Blvd|Suite R|Summit|Clay County|MO|60499|United States|-6|single family| +26912|AAAAAAAAACJGAAAA|260|2nd |Boulevard|Suite A|Newtown|McDuffie County|GA|31749|United States|-5|apartment| +26913|AAAAAAAABCJGAAAA|740|Wilson |Ave|Suite 20|Greenville|Hawaii County|HI|91387|United States|-10|apartment| +26914|AAAAAAAACCJGAAAA|907|1st |Blvd|Suite L|Cordova|Wayne County|NE|66938|United States|-7|condo| +26915|AAAAAAAADCJGAAAA|221|River |Way|Suite 380|Glendale|Runnels County|TX|73951|United States|-6|apartment| +26916|AAAAAAAAECJGAAAA|637|Dogwood |Cir.|Suite W|Pleasant Hill|Inyo County|CA|93604|United States|-8|condo| +26917|AAAAAAAAFCJGAAAA|18|Ash Oak|Circle|Suite I|Five Points|Dent County|MO|66098|United States|-6|condo| +26918|AAAAAAAAGCJGAAAA|875|Sunset Valley|RD|Suite 150|Riverview|Sanders County|MT|69003|United States|-7|single family| +26919|AAAAAAAAHCJGAAAA|605|2nd |Drive|Suite 170|Harmony|Wayne County|WV|25804|United States|-5|single family| +26920|AAAAAAAAICJGAAAA|461|Center |Street|Suite G|Kingston|Ramsey County|MN|54975|United States|-6|condo| +26921|AAAAAAAAJCJGAAAA|693|2nd |RD|Suite 330|Jamestown|Pierce County|GA|36867|United States|-5|single family| +26922|AAAAAAAAKCJGAAAA|798|5th Lakeview|Circle|Suite 80|Greenwood|Atchison County|||United States||| +26923|AAAAAAAALCJGAAAA|42|First |Parkway|Suite 280|Oak Ridge|Clayton County|GA|38371|United States|-5|apartment| +26924|AAAAAAAAMCJGAAAA|297|Highland |Pkwy|Suite 80|Waterloo|Socorro County|NM|81675|United States|-7|apartment| +26925|AAAAAAAANCJGAAAA|238|Pine |Pkwy|Suite 310|Forest Hills|Union County|KY|49237|United States|-5|apartment| +26926|AAAAAAAAOCJGAAAA|636|6th East|Wy|Suite J|Forest Hills|Honolulu County|HI|99237|United States|-10|apartment| +26927|AAAAAAAAPCJGAAAA|657|Oak |Dr.|Suite 400|Westminster|Lampasas County|TX|76549|United States|-6|condo| +26928|AAAAAAAAADJGAAAA|417|Spring |Pkwy|Suite Q|Peoria|Scotland County|MO|69818|United States|-6|single family| +26929|AAAAAAAABDJGAAAA|857|Railroad 2nd|Drive|Suite B|Riverview|Lamoille County|VT|09603|United States|-5|condo| +26930|AAAAAAAACDJGAAAA|958|Lincoln |Lane|Suite 290|Summit|Columbia County|AR|70499|United States|-6|condo| +26931|AAAAAAAADDJGAAAA|360|14th North|Avenue|Suite B|Franklin|Kenosha County|WI|59101|United States|-6|single family| +26932|AAAAAAAAEDJGAAAA|662|Central Ridge|RD|Suite 330|Arlington|Bedford County|VA|26557|United States|-5|condo| +26933|AAAAAAAAFDJGAAAA|643|South |Cir.|Suite H|Stringtown|Madison County|OH|40162|United States|-5|condo| +26934|AAAAAAAAGDJGAAAA|275|Cedar |ST|Suite H|Oak Hill|Russell County|KS|67838|United States|-6|apartment| +26935|AAAAAAAAHDJGAAAA|401|Mill |Lane|Suite Y|Belmont|Golden Valley County|ND|50191|United States|-6|apartment| +26936|AAAAAAAAIDJGAAAA|114|Church Green|Pkwy|Suite F|Plainview|Divide County|ND|53683|United States|-6|condo| +26937|AAAAAAAAJDJGAAAA|412|Broadway Center|Lane|Suite T|New Hope|Genesee County|MI|49431|United States|-5|apartment| +26938|AAAAAAAAKDJGAAAA|393|Dogwood |RD|Suite U|Clifton|Harris County|GA|38014|United States|-5|condo| +26939|AAAAAAAALDJGAAAA|243|Washington River|Blvd|Suite 100|Plainview|Phillips County|CO|83683|United States|-7|condo| +26940|AAAAAAAAMDJGAAAA|811|Center |Street|Suite 440|Maple Hill|Chautauqua County|KS|68095|United States|-6|single family| +26941|AAAAAAAANDJGAAAA|169|Madison |Cir.|Suite Y|New Hope|Cotton County|OK|79431|United States|-6|condo| +26942|AAAAAAAAODJGAAAA|378|Third |Parkway|Suite 490|Providence|Clark County|IN|46614|United States|-5|condo| +26943|AAAAAAAAPDJGAAAA|801|Broadway 9th|Wy|Suite P|Oak Grove|Litchfield County|CT|08970|United States|-5|apartment| +26944|AAAAAAAAAEJGAAAA|515|15th Lake|Way|Suite K|Mount Olive|Mountrail County|ND|58059|United States|-6|condo| +26945|AAAAAAAABEJGAAAA|285|Thirteenth Maple|Avenue|Suite 310|Macedonia|Wells County|ND|51087|United States|-6|apartment| +26946|AAAAAAAACEJGAAAA|277|Church |Wy|Suite R|Wildwood|Hocking County|OH|46871|United States|-5|apartment| +26947|AAAAAAAADEJGAAAA|758|Meadow |Lane|Suite 200|Hillcrest|Broadwater County|MT|63003|United States|-7|apartment| +26948|AAAAAAAAEEJGAAAA|378|Railroad |ST|Suite Q|Wildwood|Oktibbeha County|MS|56871|United States|-6|single family| +26949|AAAAAAAAFEJGAAAA|941|2nd |Dr.|Suite I|Kingston|Logan County|IL|64975|United States|-6|single family| +26950|AAAAAAAAGEJGAAAA|144|12th |Circle|Suite P|Elba|Cascade County|MT|60262|United States|-7|apartment| +26951|AAAAAAAAHEJGAAAA|534|Ash Hill|Street|Suite L|Deerfield|Mecosta County|MI|49840|United States|-5|single family| +26952|AAAAAAAAIEJGAAAA|236|4th Franklin|ST|Suite I|Pleasant Valley|Bandera County|TX|72477|United States|-6|single family| +26953|AAAAAAAAJEJGAAAA|267|3rd Miller|Blvd|Suite 120|Highland|Amherst County|VA|29454|United States|-5|apartment| +26954|AAAAAAAAKEJGAAAA|434|4th |Dr.|Suite E|Guilford|Butler County|OH|44408|United States|-5|single family| +26955|AAAAAAAALEJGAAAA|481|13th |Blvd|Suite 10|Oakland|Meagher County|MT|69843|United States|-7|single family| +26956|AAAAAAAAMEJGAAAA|647|Willow 15th|Pkwy|Suite W|Plainview|Grundy County|IA|53683|United States|-6|condo| +26957|AAAAAAAANEJGAAAA|881|River |Street|Suite 310|Mount Zion|Kingman County|KS|68054|United States|-6|condo| +26958|AAAAAAAAOEJGAAAA|621|East |Avenue|Suite J|Crossroads|Curry County|NM|80534|United States|-7|single family| +26959|AAAAAAAAPEJGAAAA|83|4th |Parkway|Suite D|Mount Pleasant|Teller County|CO|81933|United States|-7|single family| +26960|AAAAAAAAAFJGAAAA|262|Main |Way|Suite L|Montague|Leon County|FL|34062|United States|-5|apartment| +26961|AAAAAAAABFJGAAAA|818|Pine Laurel|RD|Suite 240|Florence|Lafayette County|WI|53394|United States|-6|condo| +26962|AAAAAAAACFJGAAAA|582|Wilson Mill|Road|Suite U|Macedonia|Meriwether County|GA|31087|United States|-5|condo| +26963|AAAAAAAADFJGAAAA|216|Walnut Walnut|Ln|Suite Q|Mount Vernon|Sangamon County|IL|68482|United States|-6|apartment| +26964|AAAAAAAAEFJGAAAA|50|Laurel |Drive|Suite 390|Maple Grove|Fisher County|TX|78252|United States|-6|apartment| +26965|AAAAAAAAFFJGAAAA|62|Birch |Boulevard|Suite V|Mount Zion|Putnam County|IN|48054|United States|-5|apartment| +26966|AAAAAAAAGFJGAAAA|110|Willow Willow|Road|Suite K|Sunnyside|Yankton County|SD|51952|United States|-6|condo| +26967|AAAAAAAAHFJGAAAA|803|Lincoln Meadow|Wy|Suite 100|Clearview|Ouachita Parish|LA|75495|United States|-6|apartment| +26968|AAAAAAAAIFJGAAAA|513|Hickory |Parkway|Suite 440|Shiloh|Calcasieu Parish|LA|79275|United States|-6|condo| +26969|AAAAAAAAJFJGAAAA|614|Pine Madison|Cir.|Suite 430|Mount Zion|Inyo County|CA|98054|United States|-8|single family| +26970|AAAAAAAAKFJGAAAA|383|Main Elm|ST|Suite F|Pine Grove|Garland County|AR|74593|United States|-6|apartment| +26971|AAAAAAAALFJGAAAA|350|Pine |Ave|Suite 260|Springfield|Amador County|CA|99303|United States|-8|condo| +26972|AAAAAAAAMFJGAAAA|868|Park |Road|Suite P|Lincoln|Venango County|PA|11289|United States|-5|apartment| +26973|AAAAAAAANFJGAAAA|714|Hill |Dr.|Suite 70|Plainview|Union County|NM|83683|United States|-7|apartment| +26974|AAAAAAAAOFJGAAAA|304|Oak |Boulevard|Suite 210|Hamilton|Jay County|IN|42808|United States|-5|condo| +26975|AAAAAAAAPFJGAAAA|425|Lake |Avenue|Suite Y|Plainview|Marion County|FL|33683|United States|-5|condo| +26976|AAAAAAAAAGJGAAAA|69|2nd Forest|Wy|Suite 150|Plainview|Carter County|OK|73683|United States|-6|condo| +26977|AAAAAAAABGJGAAAA|682|Green Broadway|Blvd|Suite U|Brownsville|Spotsylvania County|VA|29310|United States|-5|single family| +26978|AAAAAAAACGJGAAAA|78|Miller |Court|Suite 210|Macedonia|Broadwater County|MT|61087|United States|-7|apartment| +26979|AAAAAAAADGJGAAAA|575|Mill Laurel|ST|Suite 310|Fairfield|Peach County|GA|36192|United States|-5|single family| +26980|AAAAAAAAEGJGAAAA|20|Third |Boulevard|Suite 410|Hopewell|Greeley County|NE|60587|United States|-6|single family| +26981|AAAAAAAAFGJGAAAA|146|Walnut |Drive|Suite 330|Proctor|Butte County|ID|88140|United States|-7|apartment| +26982|AAAAAAAAGGJGAAAA|236|9th Maple|Blvd|Suite X|White Hall|Cumberland County|IL|66955|United States|-6|single family| +26983|AAAAAAAAHGJGAAAA|676|9th Jefferson|Road|Suite 260|Highland|Greene County|PA|19454|United States|-5|condo| +26984|AAAAAAAAIGJGAAAA|883|Washington |Circle|Suite 180|Union Hill|Turner County|SD|57746|United States|-7|apartment| +26985|AAAAAAAAJGJGAAAA|737|River 6th|Way|Suite 450|Springdale|Goliad County|TX|78883|United States|-6|condo| +26986|AAAAAAAAKGJGAAAA|416|15th |Circle|Suite 440|Hurricane|Tooele County|UT|87644|United States|-7|single family| +26987|AAAAAAAALGJGAAAA|313|Fourth Forest|Avenue|Suite V|Cedar Grove|Talladega County|AL|30411|United States|-6|condo| +26988|AAAAAAAAMGJGAAAA|||||||CO|84136|||apartment| +26989|AAAAAAAANGJGAAAA|493|Main Williams|Drive|Suite T|Shiloh|Shiawassee County|MI|49275|United States|-5|apartment| +26990|AAAAAAAAOGJGAAAA|894|Wilson Woodland|Pkwy|Suite 480|Plainview|Ralls County|MO|63683|United States|-6|condo| +26991|AAAAAAAAPGJGAAAA|511|Locust |Way|Suite I|Walnut Grove|Minidoka County|ID|87752|United States|-7|single family| +26992|AAAAAAAAAHJGAAAA|864|Davis |Drive|Suite 120|Green Acres|Polk County|NC|27683|United States|-5|apartment| +26993|AAAAAAAABHJGAAAA|362|Fourth West|Circle|Suite 400|Blue Springs|Grant County|AR|74686|United States|-6|apartment| +26994|AAAAAAAACHJGAAAA|406|1st Madison|Street|Suite T|Riverdale|Fulton County|PA|19391|United States|-5|apartment| +26995|AAAAAAAADHJGAAAA|701|Second |Parkway|Suite 280|Red Hill|Lavaca County|TX|74338|United States|-6|apartment| +26996|AAAAAAAAEHJGAAAA|264|Maple |Ave|Suite 120|Oakdale|Alleghany County|NC|29584|United States|-5|single family| +26997|AAAAAAAAFHJGAAAA|64|Mill |Lane|Suite Q|Amherst|Coahoma County|MS|58119|United States|-6|single family| +26998|AAAAAAAAGHJGAAAA|510|Forest 3rd|Street|Suite 290|Edgewood|West Feliciana Parish|LA|70069|United States|-6|condo| +26999|AAAAAAAAHHJGAAAA|723|Main Hill|Ln|Suite J|Summit|Monroe County|IA|50499|United States|-6|single family| +27000|AAAAAAAAIHJGAAAA|78|Hill |Avenue|Suite Q|White Oak|Okanogan County|WA|96668|United States|-8|single family| +27001|AAAAAAAAJHJGAAAA|30|Poplar 12th|ST|Suite W|Georgetown|Tulsa County|OK|77057|United States|-6|single family| +27002|AAAAAAAAKHJGAAAA|617|Hillcrest 1st|Dr.|Suite E|Fairview|Baker County|GA|35709|United States|-5|condo| +27003|AAAAAAAALHJGAAAA|867|Franklin Second|Parkway|Suite E|Hopewell|Clay County|IN|40587|United States|-5|single family| +27004|AAAAAAAAMHJGAAAA|337|Fifth Elm|Blvd|Suite R|Peoria|Early County|GA|39818|United States|-5|single family| +27005|AAAAAAAANHJGAAAA|918|College Center|Wy|Suite F|Belmont|Grant County|KS|60191|United States|-6|apartment| +27006|AAAAAAAAOHJGAAAA|555|Hickory East|Ln|Suite 450|Glenwood|Douglas County|NV|83511|United States|-8|apartment| +27007|AAAAAAAAPHJGAAAA|851|1st Hickory|Parkway|Suite E|Belmont|Ford County|IL|60191|United States|-6|apartment| +27008|AAAAAAAAAIJGAAAA|674|Elm Locust|Lane|Suite G|Sulphur Springs|Hardeman County|TX|78354|United States|-6|apartment| +27009|AAAAAAAABIJGAAAA|435|Cedar Madison|RD|Suite M|Liberty|Burt County|NE|63451|United States|-6|single family| +27010|AAAAAAAACIJGAAAA|304|Walnut |RD|Suite X|Lebanon|Chippewa County|MN|52898|United States|-6|single family| +27011|AAAAAAAADIJGAAAA|61|Johnson 8th|Lane|Suite 310|Crossroads|Oneida County|NY|10534|United States|-5|condo| +27012|AAAAAAAAEIJGAAAA|860|Dogwood Park|Street|Suite 300|Farmington|Fayette County|AL|39145|United States|-6|condo| +27013|AAAAAAAAFIJGAAAA|769||RD||Hillcrest|Rooks County||63003|United States|-6|| +27014|AAAAAAAAGIJGAAAA|410|3rd North|Avenue|Suite 60|Riverside|Randolph County|AL|39231|United States|-6|single family| +27015|AAAAAAAAHIJGAAAA|35|5th |Avenue|Suite W|Wilson|Perry County|IL|66971|United States|-6|apartment| +27016|AAAAAAAAIIJGAAAA|642|Chestnut Park|Avenue|Suite O|Jackson|Bradford County|FL|39583|United States|-5|single family| +27017|AAAAAAAAJIJGAAAA|779|7th |RD|Suite M|Wildwood|Schuyler County|NY|16871|United States|-5|single family| +27018|AAAAAAAAKIJGAAAA|80|Willow 13th|Dr.|Suite 30|Cumberland|Franklin County|KY|48971|United States|-6|condo| +27019|AAAAAAAALIJGAAAA|85|River |Way|Suite W|Jamestown|Solano County|CA|96867|United States|-8|single family| +27020|AAAAAAAAMIJGAAAA|67|Sixth Cedar|Pkwy|Suite 100|Highland Park|Otero County|CO|86534|United States|-7|apartment| +27021|AAAAAAAANIJGAAAA|134|Green |Wy|Suite H|Five Points|Gilmer County|GA|36098|United States|-5|apartment| +27022|AAAAAAAAOIJGAAAA|81|Highland Chestnut|Dr.|Suite B|Bethel|Venango County|PA|15281|United States|-5|apartment| +27023|AAAAAAAAPIJGAAAA|427|Johnson 8th|Avenue|Suite P|Elba|Campbell County|TN|30262|United States|-5|apartment| +27024|AAAAAAAAAJJGAAAA|239|Meadow |Lane|Suite B|Enterprise|Hillsborough County|NH|02357|United States|-5|apartment| +27025|AAAAAAAABJJGAAAA|479|Adams |Pkwy|Suite H|Belmont|Martin County|NC|20191|United States|-5|single family| +27026|AAAAAAAACJJGAAAA|495|Green |ST|Suite N|Wilson|Gaines County|TX|76971|United States|-6|single family| +27027|AAAAAAAADJJGAAAA|51|South 3rd|Ln|Suite V|Shady Grove|Black Hawk County|IA|52812|United States|-6|condo| +27028|AAAAAAAAEJJGAAAA|249|2nd |Way|Suite 40|Shannon|Teton County|MT|64120|United States|-7|apartment| +27029|AAAAAAAAFJJGAAAA|204|East Fifth|ST|Suite K|Glenwood|DeKalb County|TN|33511|United States|-5|single family| +27030|AAAAAAAAGJJGAAAA|324|Railroad |Lane|Suite 390|Sulphur Springs|Emmet County|MI|48354|United States|-5|apartment| +27031|AAAAAAAAHJJGAAAA|402|Spring Smith|Blvd|Suite 160|Guilford|Stanly County|NC|24408|United States|-5|condo| +27032|AAAAAAAAIJJGAAAA|950|4th |Boulevard|Suite V|Lakeview|Murray County|MN|58579|United States|-6|single family| +27033|AAAAAAAAJJJGAAAA|698|Third |Pkwy|Suite 330|Marion|Lyon County|KY|40399|United States|-5|condo| +27034|AAAAAAAAKJJGAAAA|769|Third |Drive|Suite 270|Harmony|Shelby County|MO|65804|United States|-6|single family| +27035|AAAAAAAALJJGAAAA|226|Woodland |Drive|Suite 130|Scottsville|Chase County|NE|64190|United States|-6|single family| +27036|AAAAAAAAMJJGAAAA|789|15th Poplar|Dr.|Suite X|Bunker Hill|Piscataquis County|ME|00750|United States|-5|condo| +27037|AAAAAAAANJJGAAAA|297|6th Williams|Lane|Suite K|Highland Park|Grays Harbor County|WA|96534|United States|-8|single family| +27038|AAAAAAAAOJJGAAAA|741|Central Miller|Pkwy|Suite W|Spring Hill|Jefferson County|OH|46787|United States|-5|single family| +27039|AAAAAAAAPJJGAAAA|847|Pine Cedar|Drive|Suite W|Salem|Brown County|SD|58048|United States|-6|single family| +27040|AAAAAAAAAKJGAAAA|693|6th |Way|Suite R|Deerfield|Aiken County|SC|29840|United States|-5|apartment| +27041|AAAAAAAABKJGAAAA|777|4th |Dr.|Suite 40|Spring Valley|Flagler County|FL|36060|United States|-5|condo| +27042|AAAAAAAACKJGAAAA|337|Main |Dr.|Suite Y|Georgetown|Lee County|FL|37057|United States|-5|single family| +27043|AAAAAAAADKJGAAAA|786|Hill Church|Lane|Suite U|Maple Grove|Raleigh County|WV|28252|United States|-5|condo| +27044|AAAAAAAAEKJGAAAA|126|Walnut |Pkwy|Suite 430|Forest Hills|San Juan County|WA|99237|United States|-8|apartment| +27045|AAAAAAAAFKJGAAAA|91|Second |Road|Suite 420|Concord|Lumpkin County|GA|34107|United States|-5|apartment| +27046|AAAAAAAAGKJGAAAA|822|Broadway Wilson|Way|Suite A|Oak Ridge|Burlington County|NJ|08971|United States|-5|condo| +27047|AAAAAAAAHKJGAAAA|126|South Maple|Ln|Suite 310|Mount Vernon|Bee County|TX|78482|United States|-6|single family| +27048|AAAAAAAAIKJGAAAA|644|15th |Wy|Suite 420|Kingston|Charlottesville city|VA|24975|United States|-5|single family| +27049|AAAAAAAAJKJGAAAA|747|Lake Park|RD|Suite P|Glendale|Big Horn County|MT|63951|United States|-7|single family| +27050|AAAAAAAAKKJGAAAA|197|Oak View|Wy|Suite Y|Newtown|Tippah County|MS|51749|United States|-6|single family| +27051|AAAAAAAALKJGAAAA|341|1st |Way|Suite 490|New Hope|Itasca County|MN|59431|United States|-6|condo| +27052|AAAAAAAAMKJGAAAA|705|1st Pine|Cir.|Suite 400|Enterprise|Bennington County|VT|02357|United States|-5|single family| +27053|AAAAAAAANKJGAAAA|493|8th Lincoln|Parkway|Suite F|Riceville|Union County|IN|45867|United States|-5|single family| +27054|AAAAAAAAOKJGAAAA|934|Lake Main|Court|Suite 260|Woodville|Decatur County|GA|34289|United States|-5|apartment| +27055|AAAAAAAAPKJGAAAA|178|West 7th|ST|Suite 450|Bridgeport|Mercer County|NJ|06417|United States|-5|condo| +27056|AAAAAAAAALJGAAAA|231|Hickory Hickory|Pkwy|Suite 440|Ruth|Winona County|MN|50309|United States|-6|apartment| +27057|AAAAAAAABLJGAAAA|394|Oak |Circle|Suite 140|Concord|Union County|OH|44107|United States|-5|single family| +27058|AAAAAAAACLJGAAAA|873|Green East|Avenue|Suite 320|Crossroads|Allegany County|NY|10534|United States|-5|single family| +27059|AAAAAAAADLJGAAAA|211|Oak |Lane|Suite 10||Jackson County|TN|38883|United States|-5|single family| +27060|AAAAAAAAELJGAAAA||9th |Dr.||Deerfield|Clearwater County||89840|||| +27061|AAAAAAAAFLJGAAAA|575|East Pine|Avenue|Suite 270|New Hope|Athens County|OH|49431|United States|-5|single family| +27062|AAAAAAAAGLJGAAAA|98|Ridge Fifth|Parkway|Suite U|Lakeside|Grant County|KS|69532|United States|-6|single family| +27063|AAAAAAAAHLJGAAAA|557|Sycamore |RD|Suite 360|Tremont|McClain County|OK|79515|United States|-6|single family| +27064|AAAAAAAAILJGAAAA|789|Sunset Main|Avenue|Suite J|Summit|Matagorda County|TX|70499|United States|-6|apartment| +27065|AAAAAAAAJLJGAAAA|15|Hickory North|Road|Suite 190|Friendship|Clarendon County|SC|24536|United States|-5|condo| +27066|AAAAAAAAKLJGAAAA|599|Pine Hill|Cir.|Suite D|Green Acres|Lawrence County|SD|57683|United States|-7|apartment| +27067|AAAAAAAALLJGAAAA|755|8th |Dr.|Suite 320|Sunnyside|Ben Hill County|GA|31952|United States|-5|condo| +27068|AAAAAAAAMLJGAAAA|136|Park Railroad|Avenue|Suite 100|New Hope|Beauregard Parish|LA|79431|United States|-6|condo| +27069|AAAAAAAANLJGAAAA|548|Cherry Smith|Ave|Suite 120|Cedar Grove|Franklin County|MA|01011|United States|-5|condo| +27070|AAAAAAAAOLJGAAAA|320|3rd Williams|Cir.|Suite 440|Belmont|Audubon County|IA|50191|United States|-6|condo| +27071|AAAAAAAAPLJGAAAA|80|South |Ct.|Suite 410|Marion|Osborne County|KS|60399|United States|-6|condo| +27072|AAAAAAAAAMJGAAAA|709|Fifth Maple||Suite 300|Pleasant Valley||SC|22477|||condo| +27073|AAAAAAAABMJGAAAA|379|Fourth |Drive|Suite U|Howell|Perry County|MO|64854|United States|-6|single family| +27074|AAAAAAAACMJGAAAA|917|Adams |Pkwy|Suite 150|Woodlawn|Oceana County|MI|44098|United States|-5|single family| +27075|AAAAAAAADMJGAAAA|355|South Third|Ave|Suite U|Edgewood|Hancock County|GA|30069|United States|-5|condo| +27076|AAAAAAAAEMJGAAAA|721|Pine |Court|Suite 330|Salem|Northampton County|PA|18048|United States|-5|apartment| +27077|AAAAAAAAFMJGAAAA|246||Blvd||||SD||||single family| +27078|AAAAAAAAGMJGAAAA|810|Hill River|Wy|Suite 310|Wildwood|Montezuma County|CO|86871|United States|-7|single family| +27079|AAAAAAAAHMJGAAAA|696|6th 2nd|Ct.|Suite O|Woodlawn|Benton County|OR|94098|United States|-8|condo| +27080|AAAAAAAAIMJGAAAA|541|First Hill|Avenue|Suite H|Shady Grove|Cibola County|NM|82812|United States|-7|condo| +27081|AAAAAAAAJMJGAAAA|10|2nd |Ct.|Suite R|Shady Grove|Magoffin County|KY|42812|United States|-5|condo| +27082|AAAAAAAAKMJGAAAA|248|Jackson |Cir.|Suite X|Shiloh|Reno County|KS|69275|United States|-6|condo| +27083|AAAAAAAALMJGAAAA|214|Ash Jefferson|RD|Suite G|Sheffield|Shannon County|MO|66896|United States|-6|condo| +27084|AAAAAAAAMMJGAAAA|907|Ash |RD|Suite 460|Pierce|Hampshire County|MA|03960|United States|-5|condo| +27085|AAAAAAAANMJGAAAA|353|5th 6th|Parkway|Suite 420|Belmont|Park County|MT|60191|United States|-7|apartment| +27086|AAAAAAAAOMJGAAAA|380|Main |Road|Suite C|Bethel|Flathead County|MT|65281|United States|-7|apartment| +27087|AAAAAAAAPMJGAAAA|271|Forest |Ct.|Suite 200|Oak Ridge|Valley County|NE|68371|United States|-7|single family| +27088|AAAAAAAAANJGAAAA|201|Dogwood |Lane|Suite 390|Oak Hill|Zapata County|TX|77838|United States|-6|apartment| +27089|AAAAAAAABNJGAAAA|706|3rd Thirteenth|Blvd|Suite 360|Georgetown|King County|TX|77057|United States|-6|condo| +27090|AAAAAAAACNJGAAAA|916|Jackson |ST|Suite M|Liberty|Florence County|WI|53451|United States|-6|condo| +27091|AAAAAAAADNJGAAAA|||Ln||||MN||United States|-6|| +27092|AAAAAAAAENJGAAAA|235|3rd |Cir.|Suite F|Lakeview|Rockdale County|GA|38579|United States|-5|apartment| +27093|AAAAAAAAFNJGAAAA|572|Dogwood Poplar|Road|Suite 150|White Oak|Greenbrier County|WV|26668|United States|-5|condo| +27094|AAAAAAAAGNJGAAAA|698|Dogwood |Blvd|Suite 30|Pleasant Hill|||33604|United States||single family| +27095|AAAAAAAAHNJGAAAA|434|Elm Valley|Boulevard|Suite 70|Lakeside|Walworth County|WI|59532|United States|-6|single family| +27096|AAAAAAAAINJGAAAA|||||Clifton|||28014|United States||| +27097|AAAAAAAAJNJGAAAA|801|5th Maple|Wy|Suite F|Kingston|Traill County|ND|54975|United States|-6|condo| +27098|AAAAAAAAKNJGAAAA|968|Spruce Hill|Avenue|Suite N|Little River|Bedford County|VA|20319|United States|-5|condo| +27099|AAAAAAAALNJGAAAA|97|Walnut |Dr.|Suite L|Hillcrest|Lewis County|MO|63003|United States|-6|single family| +27100|AAAAAAAAMNJGAAAA|155||||||IN||||| +27101|AAAAAAAANNJGAAAA|90|Walnut 8th|Blvd|Suite D|Shady Grove|Grayson County|KY|42812|United States|-6|single family| +27102|AAAAAAAAONJGAAAA|30|Hill |Way|Suite W|Sunnyside|Douglas County|NE|61952|United States|-6|apartment| +27103|AAAAAAAAPNJGAAAA|91|Sunset |Ln|Suite 140|Union|Heard County|GA|38721|United States|-5|condo| +27104|AAAAAAAAAOJGAAAA|409|Third |Circle|Suite W|Deerfield|Hooker County|NE|69840|United States|-7|apartment| +27105|AAAAAAAABOJGAAAA|647|Park Cedar|ST|Suite 180|White Oak|Napa County|CA|96668|United States|-8|condo| +27106|AAAAAAAACOJGAAAA|291|College Church|Pkwy|Suite N|Newport|Greene County|PA|11521|United States|-5|single family| +27107|AAAAAAAADOJGAAAA|573|Hickory |ST|Suite 140|Midway|Martin County|TX|71904|United States|-6|apartment| +27108|AAAAAAAAEOJGAAAA|239|Walnut Spruce|ST|Suite A|Pine Grove|Webster County|GA|34593|United States|-5|condo| +27109|AAAAAAAAFOJGAAAA|186|Park Tenth|Dr.|Suite 410|Greenville|Cumberland County|TN|31387|United States|-5|single family| +27110|AAAAAAAAGOJGAAAA|591|River |Ln|Suite 190|Springfield|Caddo Parish|LA|79303|United States|-6|apartment| +27111|AAAAAAAAHOJGAAAA|295|15th |Ave|Suite 200|Lakeview|Cheboygan County|MI|48579|United States|-5|apartment| +27112|AAAAAAAAIOJGAAAA|790|Poplar Poplar|Pkwy|Suite 440|Woodland|Alfalfa County|OK|74854|United States|-6|apartment| +27113|AAAAAAAAJOJGAAAA|665|East |Cir.|Suite 420|Mount Olive|Broadwater County|MT|68059|United States|-7|condo| +27114|AAAAAAAAKOJGAAAA|368|Franklin Lakeview|Pkwy|Suite 120|Oak Ridge|Peoria County|IL|68371|United States|-6|apartment| +27115|AAAAAAAALOJGAAAA|794|Second Laurel|Circle|Suite Q|Riverdale|Rockingham County|NC|29391|United States|-5|apartment| +27116|AAAAAAAAMOJGAAAA||Park 10th||Suite 260||Tyler County|||United States|-6|| +27117|AAAAAAAANOJGAAAA|232|Forest 12th|Dr.|Suite M|Greenville|Wise County|VA|21387|United States|-5|single family| +27118|AAAAAAAAOOJGAAAA|362|Lakeview |ST|Suite I|Newport|Chase County|KS|61521|United States|-6|apartment| +27119|AAAAAAAAPOJGAAAA|297|Ridge |Ln|Suite Q|Woodville|Scott County|KY|44289|United States|-5|condo| +27120|AAAAAAAAAPJGAAAA|250|Spring |Cir.|Suite V|Newtown|Denver County|CO|81749|United States|-7|condo| +27121|AAAAAAAABPJGAAAA|324|Walnut Railroad|Boulevard|Suite 0|Enterprise|Nevada County|AR|71757|United States|-6|condo| +27122|AAAAAAAACPJGAAAA|49|Broadway Valley|Circle|Suite 150|Maple Grove|Grays Harbor County|WA|98252|United States|-8|single family| +27123|AAAAAAAADPJGAAAA|||Pkwy||Lincoln|Brown County|NE|||-6|| +27124|AAAAAAAAEPJGAAAA|605|Railroad |ST|Suite 150|Belmont|Clarke County|VA|20191|United States|-5|single family| +27125|AAAAAAAAFPJGAAAA|896|River |Ave|Suite 310|Post Oak|Oglethorpe County|GA|38567|United States|-5|single family| +27126|AAAAAAAAGPJGAAAA|875|College Oak|Cir.|Suite F|Hopewell|Newton County|AR|70587|United States|-6|single family| +27127|AAAAAAAAHPJGAAAA|101|Hill |Drive|Suite 250|White Oak|Madison County|TN|36668|United States|-6|apartment| +27128|AAAAAAAAIPJGAAAA|527|Seventh Davis|Road|Suite 120|Valley View|Matagorda County|TX|75124|United States|-6|condo| +27129|AAAAAAAAJPJGAAAA|954|Smith 1st|Lane|Suite B|Riverview|Spencer County|IN|49003|United States|-5|condo| +27130|AAAAAAAAKPJGAAAA|32|Church Oak|Ct.|Suite 250|Sleepy Hollow|Logan County|IL|63592|United States|-6|apartment| +27131|AAAAAAAALPJGAAAA|352|Chestnut 13th|Ct.|Suite J|Forest Hills|Ector County|TX|79237|United States|-6|condo| +27132|AAAAAAAAMPJGAAAA|364|1st |Way|Suite Y|Belmont|Smith County|TN|30191|United States|-6|condo| +27133|AAAAAAAANPJGAAAA|878|Maple 15th|Lane|Suite N|Hopewell|Kane County|UT|80587|United States|-7|single family| +27134|AAAAAAAAOPJGAAAA|100|Lincoln |Lane|Suite 40|Newtown|Branch County|MI|41749|United States|-5|single family| +27135|AAAAAAAAPPJGAAAA|423|Mill Madison|Cir.|Suite N|Clifton|Dixie County|FL|38014|United States|-5|condo| +27136|AAAAAAAAAAKGAAAA|997|Johnson |RD|Suite 220|Hillcrest|Montezuma County|CO|83003|United States|-7|condo| +27137|AAAAAAAABAKGAAAA|755|Davis |Blvd|Suite 280|Antioch|Pike County|GA|38605|United States|-5|condo| +27138|AAAAAAAACAKGAAAA|201|Madison |Lane|Suite 270|Concord|Meade County|KY|44107|United States|-5|single family| +27139|AAAAAAAADAKGAAAA|347|Railroad 3rd|Pkwy|Suite U|Wildwood|Hood County|TX|76871|United States|-6|single family| +27140|AAAAAAAAEAKGAAAA|832|Cedar |Way|Suite A|Georgetown|Greene County|MS|57057|United States|-6|apartment| +27141|AAAAAAAAFAKGAAAA|998|9th |Avenue|Suite L|Plainview|Real County|TX|73683|United States|-6|apartment| +27142|AAAAAAAAGAKGAAAA|643|Pine Oak|Wy|Suite 340|Jamestown|||46867||-5|| +27143|AAAAAAAAHAKGAAAA|460|Railroad |Road|Suite 80|Forest Hills|Sumter County|FL|39237|United States|-5|condo| +27144|AAAAAAAAIAKGAAAA|431|7th |Boulevard|Suite 170|White Oak|Hancock County|IL|66668|United States|-6|condo| +27145|AAAAAAAAJAKGAAAA|51|View |Lane|Suite 240|Turner|Warren County|NC|20875|United States|-5|apartment| +27146|AAAAAAAAKAKGAAAA|56|Hillcrest |Drive|Suite X|Bunker Hill|McDonough County|IL|60150|United States|-6|apartment| +27147|AAAAAAAALAKGAAAA|315|Lakeview |Cir.|Suite 80|Spring Valley|Summers County|WV|26060|United States|-5|condo| +27148|AAAAAAAAMAKGAAAA|496|Hickory |Boulevard|Suite W|Spring Valley|Coosa County|AL|36060|United States|-6|apartment| +27149|AAAAAAAANAKGAAAA|412|Willow |Street|Suite 350|Pine Grove|Coweta County|GA|34593|United States|-5|apartment| +27150|AAAAAAAAOAKGAAAA|100|9th |Dr.|Suite 410|Milo|New Kent County|VA|20116|United States|-5|single family| +27151|AAAAAAAAPAKGAAAA|879|5th |Avenue|Suite 10|Plainview|Jefferson County|KY|43683|United States|-6|apartment| +27152|AAAAAAAAABKGAAAA|197|Poplar |Circle|Suite 20|Stratford|Wichita County|KS|66668|United States|-6|single family| +27153|AAAAAAAABBKGAAAA|181|10th Park|Cir.|Suite Y|Sulphur Springs|Adair County|OK|78354|United States|-6|single family| +27154|AAAAAAAACBKGAAAA|69|Maple Jefferson|Boulevard|Suite E|Ashland|Lewis County|ID|84244|United States|-7|single family| +27155|AAAAAAAADBKGAAAA|672|Walnut |Ave|Suite D|Woodland|Coffee County|TN|34854|United States|-5|single family| +27156|AAAAAAAAEBKGAAAA|480|First Park|Ave|Suite B|Spring Hill|Perquimans County|NC|26787|United States|-5|apartment| +27157|AAAAAAAAFBKGAAAA|237|3rd |Ave|Suite 350|Sunnyside|Reeves County|TX|71952|United States|-6|apartment| +27158|AAAAAAAAGBKGAAAA|78|Hill |Way|Suite T|Oakwood|Perry County|AL|30169|United States|-6|condo| +27159|AAAAAAAAHBKGAAAA|||Drive|||Chemung County||18579||-5|| +27160|AAAAAAAAIBKGAAAA||Cedar Main|Drive|Suite 100|New Hope||||United States|-5|apartment| +27161|AAAAAAAAJBKGAAAA|945|8th Main|Wy|Suite W|Jackson|Wells County|ND|59583|United States|-6|condo| +27162|AAAAAAAAKBKGAAAA|445|Sycamore 11th|Parkway|Suite S|Kensington|Granite County|MT|61781|United States|-7|apartment| +27163|AAAAAAAALBKGAAAA|321|Main Davis|Wy|Suite 270|Oakwood|Rabun County|GA|30169|United States|-5|condo| +27164|AAAAAAAAMBKGAAAA|107|Spruce |Court|Suite T|White Oak|Fulton County|GA|36668|United States|-5|condo| +27165|AAAAAAAANBKGAAAA|176|Hill Fifth|Ct.|Suite A|Buena Vista|Nez Perce County|ID|85752|United States|-7|apartment| +27166|AAAAAAAAOBKGAAAA|883|Lakeview |Road|Suite 490|Peru|Martin County|TX|70302|United States|-6|apartment| +27167|AAAAAAAAPBKGAAAA|773|Church Railroad|RD|Suite F|Walnut Grove|Midland County|MI|47752|United States|-5|single family| +27168|AAAAAAAAACKGAAAA|344|Hickory 15th|Dr.|Suite E|Pleasant Grove|Pulaski County|MO|64136|United States|-6|condo| +27169|AAAAAAAABCKGAAAA|657|1st Seventh|Ct.|Suite 290|Highland Park|Brown County|OH|46534|United States|-5|single family| +27170|AAAAAAAACCKGAAAA|432|Valley |Drive|Suite 140|Belmont|Renville County|ND|50191|United States|-6|condo| +27171|AAAAAAAADCKGAAAA|753|4th |ST|Suite G|Forest Hills|Palo Alto County|IA|59237|United States|-6|single family| +27172|AAAAAAAAECKGAAAA|319|Franklin Meadow|Pkwy|Suite 20|Liberty|La Salle County|IL|63451|United States|-6|apartment| +27173|AAAAAAAAFCKGAAAA|780|Woodland Eigth|Blvd|Suite G|Five Forks|Lawrence County|OH|42293|United States|-5|apartment| +27174|AAAAAAAAGCKGAAAA|895|Broadway |Wy|Suite Y|Woodruff|Menominee County|MI|44174|United States|-5|condo| +27175|AAAAAAAAHCKGAAAA|807|4th Hillcrest|RD|Suite 150|Springfield|Comanche County|TX|79303|United States|-6|single family| +27176|AAAAAAAAICKGAAAA|325|14th |Boulevard|Suite 460|Bunker Hill|Iredell County|NC|20150|United States|-5|condo| +27177|AAAAAAAAJCKGAAAA|681|9th 8th|Ave|Suite 400|Gravel Hill|Summit County|OH|41944|United States|-5|condo| +27178|AAAAAAAAKCKGAAAA|120|2nd |Parkway|Suite P|Greenville|Walton County|FL|31387|United States|-5|condo| +27179|AAAAAAAALCKGAAAA|455|Lee Mill|Dr.|Suite K|Highland Park|Racine County|WI|56534|United States|-6|condo| +27180|AAAAAAAAMCKGAAAA|594|2nd |Wy|Suite 490|Oak Ridge|Charles County|MD|28371|United States|-5|condo| +27181|AAAAAAAANCKGAAAA|730|Oak |Blvd|Suite 420|Spring Valley|Lowndes County|AL|36060|United States|-6|single family| +27182|AAAAAAAAOCKGAAAA|845|Williams |Avenue|Suite D|Harmony|Gasconade County|MO|65804|United States|-6|apartment| +27183|AAAAAAAAPCKGAAAA|951|5th |Ln|Suite Q|Montague|Carroll County|IL|64062|United States|-6|apartment| +27184|AAAAAAAAADKGAAAA||||Suite 400|Antioch||MS|||-6|apartment| +27185|AAAAAAAABDKGAAAA|260|5th |Ave|Suite L|Salem|Wood County|WI|58048|United States|-6|condo| +27186|AAAAAAAACDKGAAAA|711|Miller Lee|Lane|Suite 50|Fairview|Jefferson County|GA|35709|United States|-5|condo| +27187|AAAAAAAADDKGAAAA|51|2nd 2nd|Blvd|Suite 300|Valley View|Ashland County|OH|45124|United States|-5|single family| +27188|AAAAAAAAEDKGAAAA|815|Park Madison|RD|Suite D|Pleasant Hill|Ashland County|OH|43604|United States|-5|condo| +27189|AAAAAAAAFDKGAAAA|265|1st South|Boulevard|Suite 250|Lebanon|Taylor County|FL|32898|United States|-5|condo| +27190|AAAAAAAAGDKGAAAA|974|2nd Park|Avenue|Suite 130|Pleasant Grove|Garfield County|WA|94136|United States|-8|apartment| +27191|AAAAAAAAHDKGAAAA|628|Spruce |Pkwy|Suite M|Clifton|Peoria County|IL|68014|United States|-6|single family| +27192|AAAAAAAAIDKGAAAA|530|Lincoln |Ave|Suite 240|Enterprise|Hickman County|KY|41757|United States|-6|condo| +27193|AAAAAAAAJDKGAAAA|16|Wilson Johnson|Cir.|Suite Q|Union|Maverick County|TX|78721|United States|-6|single family| +27194|AAAAAAAAKDKGAAAA|115|Madison Meadow|ST|Suite 400|Riverview|Manassas city|VA|29003|United States|-5|apartment| +27195|AAAAAAAALDKGAAAA|944|Seventh Woodland|Circle|Suite 380|Union|Napa County|CA|98721|United States|-8|condo| +27196|AAAAAAAAMDKGAAAA|525|Madison Oak|Circle|Suite C|Mount Olive|Shenandoah County|VA|28059|United States|-5|condo| +27197|AAAAAAAANDKGAAAA|359|View |Ct.|Suite K|Bunker Hill|Douglas County|WA|90150|United States|-8|single family| +27198|AAAAAAAAODKGAAAA|892|Adams 2nd|Avenue|Suite C|Kingston|Weld County|CO|84975|United States|-7|apartment| +27199|AAAAAAAAPDKGAAAA|738|Eigth |Ln|Suite R|Lincoln|Grady County|OK|71289|United States|-6|condo| +27200|AAAAAAAAAEKGAAAA|78|Walnut 15th|Ln|Suite 450|Lincoln|Reynolds County|MO|61289|United States|-6|single family| +27201|AAAAAAAABEKGAAAA|191|Fifth |Boulevard|Suite 260|Mount Pleasant|Howell County|MO|61933|United States|-6|condo| +27202|AAAAAAAACEKGAAAA|327|3rd 3rd|Dr.|Suite 300|Freeman|Mercer County|PA|12297|United States|-5|apartment| +27203|AAAAAAAADEKGAAAA|186|Ridge Elm|Circle|Suite D|Centerville|Kankakee County|IL|60059|United States|-6|single family| +27204|AAAAAAAAEEKGAAAA|901|Sunset Hillcrest|Way|Suite N|Newtown|Berkshire County|MA|02349|United States|-5|apartment| +27205|AAAAAAAAFEKGAAAA|960|Fourth 2nd|Lane|Suite 180|Woodlawn|Beaufort County|NC|24098|United States|-5|condo| +27206|AAAAAAAAGEKGAAAA|833|Washington Sixth|RD|Suite Q|Spring Hill|McDonald County|MO|66787|United States|-6|single family| +27207|AAAAAAAAHEKGAAAA|707|Sixth West|ST|Suite 280|Antioch|Jackson County|AR|78605|United States|-6|apartment| +27208|AAAAAAAAIEKGAAAA|816|8th Lee|Street|Suite S|Spring Valley|Churchill County|NV|86060|United States|-8|single family| +27209|AAAAAAAAJEKGAAAA|107|North |Street|Suite 120|Antioch|Crane County|TX|78605|United States|-6|apartment| +27210|AAAAAAAAKEKGAAAA|648|Ridge 4th|Ct.|Suite 480|Ashland|Pendleton County|KY|44244|United States|-5|single family| +27211|AAAAAAAALEKGAAAA|426|Maple |Pkwy|Suite 330|Plainview|Dukes County|MA|04283|United States|-5|single family| +27212|AAAAAAAAMEKGAAAA|906|Hill 7th|Ct.|Suite 10|Oak Grove|Hardy County|WV|28370|United States|-5|condo| +27213|AAAAAAAANEKGAAAA|769|3rd |Ave|Suite 170|Spring Valley|Ware County|GA|36060|United States|-5|condo| +27214|AAAAAAAAOEKGAAAA|100|Ash Washington|Ln|Suite 0|Highland|LaMoure County|ND|59454|United States|-6|apartment| +27215|AAAAAAAAPEKGAAAA|139|Adams West|Avenue|Suite 270|Woodrow|Scott County|MO|64273|United States|-6|condo| +27216|AAAAAAAAAFKGAAAA||||Suite 90||||23451||-5|| +27217|AAAAAAAABFKGAAAA|527|Main Walnut|Court|||York County||22477|United States||| +27218|AAAAAAAACFKGAAAA|972|14th |Ave|Suite 20|Forestville|McDowell County|NC|23027|United States|-5|condo| +27219|AAAAAAAADFKGAAAA|994|Pine |Ct.|Suite F|Hardy|Nye County|NV|85354|United States|-8|condo| +27220|AAAAAAAAEFKGAAAA|801|Walnut |Ct.|Suite L|Franklin|Story County|IA|59101|United States|-6|apartment| +27221|AAAAAAAAFFKGAAAA|489|Poplar |Blvd|Suite J|Oakland|Leavenworth County|KS|69843|United States|-6|single family| +27222|AAAAAAAAGFKGAAAA|354|Mill |Dr.|Suite 360|Concord|Douglas County|MO|64107|United States|-6|single family| +27223|AAAAAAAAHFKGAAAA||||||De Soto Parish|LA|||-6|| +27224|AAAAAAAAIFKGAAAA|146|Davis Johnson|Parkway|Suite K|Bayside|Clearwater County|MN|59550|United States|-6|single family| +27225|AAAAAAAAJFKGAAAA|740|First |Ct.|Suite M|Lakeside|Valley County|NE|69532|United States|-7|condo| +27226|AAAAAAAAKFKGAAAA|364|Lake Spruce|Ct.|Suite E|Oak Grove|Fayette County|AL|38370|United States|-6|condo| +27227|AAAAAAAALFKGAAAA|258|View Park|Ct.|Suite W|Wilson|Phelps County|MO|66971|United States|-6|condo| +27228|AAAAAAAAMFKGAAAA|945|Davis Elm|Way|Suite V|Buena Vista|DeKalb County|TN|35752|United States|-5|apartment| +27229|AAAAAAAANFKGAAAA|864|4th |Circle|Suite 400|Lakeside|Juneau Borough|AK|99532|United States|-9|single family| +27230|AAAAAAAAOFKGAAAA|472|Willow |Drive|Suite Y|Jamestown|Washington County|NY|16867|United States|-5|apartment| +27231|AAAAAAAAPFKGAAAA|582|Oak |Court|Suite I|Unionville|Meade County|KS|61711|United States|-6|apartment| +27232|AAAAAAAAAGKGAAAA|18|Maple Woodland|Cir.|Suite 400|Harmony|Breathitt County|KY|45804|United States|-6|single family| +27233|AAAAAAAABGKGAAAA|634|11th |Avenue|Suite 140|Newport|Paulding County|GA|31521|United States|-5|apartment| +27234|AAAAAAAACGKGAAAA|983|Center Mill|Way|Suite 90|Mount Vernon|Alleghany County|VA|28482|United States|-5|condo| +27235|AAAAAAAADGKGAAAA|197|Center |Lane|Suite 340|Shiloh|Atchison County|KS|69275|United States|-6|single family| +27236|AAAAAAAAEGKGAAAA|177|Elm First|Pkwy|Suite 210|Shiloh|Orleans Parish|LA|79275|United States|-6|single family| +27237|AAAAAAAAFGKGAAAA|599|Main Cedar|Road|Suite A|Waterloo|Kings County|CA|91675|United States|-8|condo| +27238|AAAAAAAAGGKGAAAA|616|Oak |ST|Suite K|Midway|Noxubee County|MS|51904|United States|-6|condo| +27239|AAAAAAAAHGKGAAAA|279|Walnut |Pkwy|Suite W|Ruth|Paulding County|GA|30309|United States|-5|apartment| +27240|AAAAAAAAIGKGAAAA|326|10th |Cir.|Suite T|Shaw|Dickey County|ND|50618|United States|-6|single family| +27241|AAAAAAAAJGKGAAAA|910|Johnson |ST|Suite Y|Shannon|Grant County|MN|54120|United States|-6|single family| +27242|AAAAAAAAKGKGAAAA|104|Park Broadway|Avenue|Suite Q|Proctor|Olmsted County|MN|58140|United States|-6|apartment| +27243|AAAAAAAALGKGAAAA|294|8th |Pkwy|Suite F|Unionville|Beckham County|OK|71711|United States|-6|condo| +27244|AAAAAAAAMGKGAAAA|610|Lee 13th|Ave|Suite 170|Franklin|Dallam County|TX|79101|United States|-6|condo| +27245|AAAAAAAANGKGAAAA|880|Wilson 3rd|RD|Suite 490|Springdale|Johnson County|NE|68883|United States|-7|single family| +27246|AAAAAAAAOGKGAAAA|642|7th Park|Ln|Suite R|Union City|Garfield County|UT|88087|United States|-7|condo| +27247|AAAAAAAAPGKGAAAA|694|12th North|Ave|Suite R|Macedonia|Bulloch County|GA|31087|United States|-5|apartment| +27248|AAAAAAAAAHKGAAAA|590|1st 13th|Way|Suite P|Harmony|Jasper County|SC|25804|United States|-5|single family| +27249|AAAAAAAABHKGAAAA|108|15th Oak|Way|Suite U|Mount Pleasant|Tyler County|TX|71933|United States|-6|apartment| +27250|AAAAAAAACHKGAAAA|335|First Green|Avenue|Suite 470|Hillcrest|Spotsylvania County|VA|23003|United States|-5|single family| +27251|AAAAAAAADHKGAAAA|81|Wilson Lincoln|Avenue|Suite D|Walnut Grove|Russell County|AL|37752|United States|-6|single family| +27252|AAAAAAAAEHKGAAAA|18|Valley |Drive|Suite C|Shady Grove|Real County|TX|72812|United States|-6|apartment| +27253|AAAAAAAAFHKGAAAA|22|Green Franklin|Circle|Suite 460|Springfield|Madison County|NE|69303|United States|-7|condo| +27254|AAAAAAAAGHKGAAAA|184|Wilson 2nd|Pkwy|Suite X|Midway|Murray County|OK|71904|United States|-6|apartment| +27255|AAAAAAAAHHKGAAAA|739|Oak |Dr.|Suite R|Pleasant Valley|Colleton County|SC|22477|United States|-5|apartment| +27256|AAAAAAAAIHKGAAAA|280|Park |Way|Suite S|Oakland|Surry County|NC|29843|United States|-5|condo| +27257|AAAAAAAAJHKGAAAA|640|Walnut 3rd|Street|Suite I|Springdale|Pickaway County|OH|48883|United States|-5|apartment| +27258|AAAAAAAAKHKGAAAA|167|Oak |Drive|Suite X|Green Acres|Garfield County|MT|67683|United States|-7|single family| +27259|AAAAAAAALHKGAAAA|745|Willow 3rd|Pkwy|Suite 20|Fairfield|Burleigh County|ND|56192|United States|-6|single family| +27260|AAAAAAAAMHKGAAAA|667|Fourth 11th|Way|Suite E|Lakeville|Montour County|PA|18811|United States|-5|condo| +27261|AAAAAAAANHKGAAAA|526|5th |ST|Suite J|Mount Vernon|Northumberland County|VA|28482|United States|-5|single family| +27262|AAAAAAAAOHKGAAAA|243|Walnut |Blvd|Suite Y|Brookville|Washington County|KY|43524|United States|-5|apartment| +27263|AAAAAAAAPHKGAAAA|415|3rd Laurel|Avenue|Suite 440|Spring Hill|Johnson County|IL|66787|United States|-6|apartment| +27264|AAAAAAAAAIKGAAAA|833|North |Cir.|Suite O|Woodlawn|Goochland County|VA|24098|United States|-5|condo| +27265|AAAAAAAABIKGAAAA|604|Forest 14th|Ct.|Suite 20|Lakewood|Van Buren County|MI|48877|United States|-5|apartment| +27266|AAAAAAAACIKGAAAA|154|College |Blvd|Suite 310|White Plains|Kit Carson County|CO|86622|United States|-7|condo| +27267|AAAAAAAADIKGAAAA|365|Madison Hill|Cir.|Suite A|Stringtown|Stillwater County|MT|60162|United States|-7|single family| +27268|AAAAAAAAEIKGAAAA|64|River Pine|Avenue|Suite 200|Clinton|Walsh County|ND|58222|United States|-6|condo| +27269|AAAAAAAAFIKGAAAA|53|East |Ave|Suite 440|Hamilton|Nash County|NC|22808|United States|-5|apartment| +27270|AAAAAAAAGIKGAAAA|848|Hillcrest Madison|Road|Suite S|Saint Johns|Brookings County|SD|55717|United States|-6|single family| +27271|AAAAAAAAHIKGAAAA|446|6th |Pkwy|Suite 320|Concord|Powhatan County|VA|24107|United States|-5|single family| +27272|AAAAAAAAIIKGAAAA|991|North South|Blvd|Suite 180|Pleasant Hill|Lake County|MN|53604|United States|-6|single family| +27273|AAAAAAAAJIKGAAAA|453|Valley 4th|ST|Suite 260|Maple Grove|Gallatin County|MT|68252|United States|-7|single family| +27274|AAAAAAAAKIKGAAAA|448|Woodland Cherry|Way|Suite B|Unionville|Hale County|AL|31711|United States|-6|single family| +27275|AAAAAAAALIKGAAAA|557|River 12th|Road|Suite D|Spring Grove|Tarrant County|TX|76719|United States|-6|single family| +27276|AAAAAAAAMIKGAAAA|568|Center Cherry|Avenue|Suite V|Shady Grove|Steele County|MN|52812|United States|-6|condo| +27277|AAAAAAAANIKGAAAA|8|Willow |Ln|Suite Q|Centerville|Franklin County|KY|40059|United States|-6|single family| +27278|AAAAAAAAOIKGAAAA|721|Jackson Walnut|Pkwy|Suite 220|Morris|Lincoln County|SD|56696|United States|-7|apartment| +27279|AAAAAAAAPIKGAAAA|971|7th |Pkwy|Suite 240|Harvey|Milwaukee County|WI|55858|United States|-6|single family| +27280|AAAAAAAAAJKGAAAA|434|Franklin Jefferson|Ct.|Suite 440|Clearview|Randolph County|IL|65495|United States|-6|apartment| +27281|AAAAAAAABJKGAAAA|682|Dogwood Fourth|Blvd|Suite S|Red Hill|Plymouth County|IA|54338|United States|-6|apartment| +27282|AAAAAAAACJKGAAAA|685|Hill Main|Dr.|Suite I|Arlington|Jeff Davis County|TX|76557|United States|-6|apartment| +27283|AAAAAAAADJKGAAAA|720|Fifth |RD|Suite C|Liberty|Washoe County|NV|83451|United States|-8|apartment| +27284|AAAAAAAAEJKGAAAA|607|Cedar |Dr.|Suite A|Altamont|Calvert County|MD|29387|United States|-5|single family| +27285|AAAAAAAAFJKGAAAA|533|First Sunset|Parkway|Suite 200|Wildwood|McKenzie County|ND|56871|United States|-6|apartment| +27286|AAAAAAAAGJKGAAAA|666|Spruce |Street|Suite C|Cedar Grove|Deuel County|NE|60411|United States|-6|single family| +27287|AAAAAAAAHJKGAAAA|155|2nd |Boulevard|Suite U|Hopewell|Bandera County|TX|70587|United States|-6|apartment| +27288|AAAAAAAAIJKGAAAA|775|2nd |RD|Suite 140|Summit|Aiken County|SC|20499|United States|-5|condo| +27289|AAAAAAAAJJKGAAAA|918|South |Road|Suite 340|Glenwood|Atascosa County|TX|73511|United States|-6|single family| +27290|AAAAAAAAKJKGAAAA|924|Walnut Main|Avenue|Suite 470|Salem|Crawford County|GA|38048|United States|-5|single family| +27291|AAAAAAAALJKGAAAA|330|East 13th|Avenue|Suite 170|Red Hill|Mitchell County|NC|24338|United States|-5|condo| +27292|AAAAAAAAMJKGAAAA|222|3rd 6th|Avenue|Suite 300|Spring Hill|Bronx County|NY|16787|United States|-5|single family| +27293|AAAAAAAANJKGAAAA|663|4th |Boulevard|Suite 210|Ashland|Chaves County|NM|84244|United States|-7|single family| +27294|AAAAAAAAOJKGAAAA|157|Woodland Lincoln|Ln|Suite 150|Shady Grove|Rockingham County|NC|22812|United States|-5|apartment| +27295|AAAAAAAAPJKGAAAA|920|Mill 1st|Street|Suite F|Oak Grove|Miami County|KS|68370|United States|-6|single family| +27296|AAAAAAAAAKKGAAAA|896|Washington Church|Dr.|Suite V|Deerfield|Columbia County|GA|39840|United States|-5|apartment| +27297|AAAAAAAABKKGAAAA|||Road|Suite I|Enterprise||||United States||| +27298|AAAAAAAACKKGAAAA|167|Highland |Court|Suite Y|Marion|Franklin County|FL|30399|United States|-5|condo| +27299|AAAAAAAADKKGAAAA|303|6th Main|ST|Suite N|Highland|Windham County|VT|09454|United States|-5|single family| +27300|AAAAAAAAEKKGAAAA|154|Birch |Parkway|Suite 0|Valley View|Vilas County|WI|55124|United States|-6|apartment| +27301|AAAAAAAAFKKGAAAA|802|Oak |Ln|Suite A|Pleasant Valley|Androscoggin County|ME|03077|United States|-5|condo| +27302|AAAAAAAAGKKGAAAA|465|First |Lane|Suite 490|Marion|Green County|KY|40399|United States|-6|apartment| +27303|AAAAAAAAHKKGAAAA|99|Ninth |Road|Suite 10|Shady Grove|Tallapoosa County|AL|32812|United States|-6|condo| +27304|AAAAAAAAIKKGAAAA|479|Maple Pine|Wy|Suite Q|Jackson|Barber County|KS|69583|United States|-6|single family| +27305|AAAAAAAAJKKGAAAA|730|Broadway |Ave|Suite Q|Bridgeport|Sharkey County|MS|55817|United States|-6|single family| +27306|AAAAAAAAKKKGAAAA|865|Park 13th|Parkway|Suite G|Edgewood|Crockett County|TX|70069|United States|-6|condo| +27307|AAAAAAAALKKGAAAA|730|14th Main|Drive|Suite D|Pleasant Hill|Deuel County|SD|53604|United States|-6|condo| +27308|AAAAAAAAMKKGAAAA|697|Cherry Willow|Dr.|Suite W|Spring Valley|Montgomery County|IN|46060|United States|-5|condo| +27309|AAAAAAAANKKGAAAA|566|Highland 7th|Street|Suite O|Newtown|Vernon Parish|LA|71749|United States|-6|single family| +27310|AAAAAAAAOKKGAAAA|255|1st |Court|Suite F|Riverview|Cheatham County|TN|39003|United States|-5|single family| +27311|AAAAAAAAPKKGAAAA|381|8th |Circle|Suite U|Ashland|Blount County|AL|34244|United States|-6|apartment| +27312|AAAAAAAAALKGAAAA|48|Park |Ave|Suite C|Lodi|Meeker County|MN|50382|United States|-6|apartment| +27313|AAAAAAAABLKGAAAA|393|Main Wilson|Way|Suite S|Riverside|Hawaii County|HI|99231|United States|-10|condo| +27314|AAAAAAAACLKGAAAA|85|Miller 6th|Boulevard|Suite 130|Plainview|Glacier County|MT|63683|United States|-7|condo| +27315|AAAAAAAADLKGAAAA|303|Hickory |Parkway|Suite J|Farmington|Roseau County|MN|59145|United States|-6|apartment| +27316|AAAAAAAAELKGAAAA|466|6th Valley|Street|Suite 90|Harmony|Dallas County|AL|35804|United States|-6|apartment| +27317|AAAAAAAAFLKGAAAA|408|Elm Lake|Ct.|Suite Y|Forest Hills|Winneshiek County|IA|59237|United States|-6|condo| +27318|AAAAAAAAGLKGAAAA|127|Broadway Madison|ST|Suite A|Walnut Grove|Grant County|ND|57752|United States|-6|condo| +27319|AAAAAAAAHLKGAAAA|676|Washington 4th|Wy|Suite 420|Bridgeport|Lewis County|WA|95817|United States|-8|single family| +27320|AAAAAAAAILKGAAAA|664|Twelfth 12th|Court|Suite O|Glendale|Crawford County|IA|53951|United States|-6|single family| +27321|AAAAAAAAJLKGAAAA|79|13th West|Boulevard|Suite R|Bunker Hill|Hyde County|SD|50150|United States|-7|single family| +27322|AAAAAAAAKLKGAAAA|172|Jefferson Wilson|Boulevard|Suite 400|Ruth|Wilson County|NC|20309|United States|-5|apartment| +27323|AAAAAAAALLKGAAAA|744|Lee Lakeview|Cir.|Suite 220|Hamilton|Ward County|ND|52808|United States|-6|apartment| +27324|AAAAAAAAMLKGAAAA|167|4th |Pkwy|Suite 390|Roy|Philadelphia County|PA|10744|United States|-5|condo| +27325|AAAAAAAANLKGAAAA|358|View |Way|Suite I|White Oak|Taos County|NM|86668|United States|-7|apartment| +27326|AAAAAAAAOLKGAAAA|785|Laurel |Wy|Suite F|Oak Grove|Dade County|MO|68370|United States|-6|single family| +27327|AAAAAAAAPLKGAAAA|692|Green Madison|Parkway|Suite 230|Oakdale|Nicollet County|MN|59584|United States|-6|condo| +27328|AAAAAAAAAMKGAAAA|896|||||Richmond city|||United States||single family| +27329|AAAAAAAABMKGAAAA|557|||||McClain County||||-6|apartment| +27330|AAAAAAAACMKGAAAA||Pine |||||TX|74289|United States|-6|| +27331|AAAAAAAADMKGAAAA|587|Smith 6th|Ct.|Suite 390|Buena Vista|Union County|NM|85752|United States|-7|condo| +27332|AAAAAAAAEMKGAAAA|197|2nd |Road|Suite 50|Lincoln|Wilcox County|AL|31289|United States|-6|apartment| +27333|AAAAAAAAFMKGAAAA|859|Main 1st|Ln|Suite 10|Greenwood|Harnett County|NC|28828|United States|-5|condo| +27334|AAAAAAAAGMKGAAAA|816|Chestnut 10th|Parkway|Suite A|Lakeside|Amherst County|VA|29532|United States|-5|condo| +27335|AAAAAAAAHMKGAAAA|962|First |RD|Suite F|Harmony|Lauderdale County|AL|35804|United States|-6|condo| +27336|AAAAAAAAIMKGAAAA|814|West |Ct.|Suite 40|New Town|Kodiak Island Borough|AK|99634|United States|-9|condo| +27337|AAAAAAAAJMKGAAAA|690|Johnson 6th|Way|Suite 410|Oak Ridge|Mercer County|PA|18371|United States|-5|single family| +27338|AAAAAAAAKMKGAAAA|780|Spring |Cir.|Suite T|Lakewood|Calumet County|WI|58877|United States|-6|condo| +27339|AAAAAAAALMKGAAAA|||Way|Suite 340|New Hope|Osceola County||59431|United States||condo| +27340|AAAAAAAAMMKGAAAA|813|Park 1st|ST|Suite 50|Leland|Tensas Parish|LA|79452|United States|-6|apartment| +27341|AAAAAAAANMKGAAAA|200|Smith |Pkwy|Suite V|Franklin|Whitley County|KY|49101|United States|-5|condo| +27342|AAAAAAAAOMKGAAAA|414|Lincoln |RD|Suite 100|Brownsville|Smith County|TX|79310|United States|-6|apartment| +27343|AAAAAAAAPMKGAAAA|107|River 6th|Street|Suite J|Liberty|East Carroll Parish|LA|73451|United States|-6|condo| +27344|AAAAAAAAANKGAAAA|862|Maple |Street|Suite 30|Spring Hill|Anson County|NC|26787|United States|-5|condo| +27345|AAAAAAAABNKGAAAA|584|Ridge Elm|Pkwy|Suite 480|Sunnyside|Cottle County|TX|71952|United States|-6|single family| +27346|AAAAAAAACNKGAAAA|301|6th |Avenue|Suite Q|Green Acres|Ottawa County|OK|77683|United States|-6|single family| +27347|AAAAAAAADNKGAAAA|438|Meadow 9th|Street|Suite 120|Oakwood|Adair County|IA|50169|United States|-6|apartment| +27348|AAAAAAAAENKGAAAA|16|Jefferson |Drive|Suite J|Ashland|Poweshiek County|IA|54244|United States|-6|condo| +27349|AAAAAAAAFNKGAAAA|874|5th Cedar|Wy|Suite K|Antioch|Plymouth County|MA|09205|United States|-5|single family| +27350|AAAAAAAAGNKGAAAA|455|5th Pine|Ave|Suite 220|Arthur|Sumter County|GA|35965|United States|-5|single family| +27351|AAAAAAAAHNKGAAAA|487|||||||78371|||| +27352|AAAAAAAAINKGAAAA|177|Lee |Avenue|Suite 50|Salem|Rappahannock County|VA|28048|United States|-5|apartment| +27353|AAAAAAAAJNKGAAAA|732|Jackson |Dr.|Suite L|Rockwood|Liberty County|GA|31545|United States|-5|condo| +27354|AAAAAAAAKNKGAAAA|172|Lincoln Elm|Wy|Suite 340|Lakewood|Williamsburg County|SC|28877|United States|-5|apartment| +27355|AAAAAAAALNKGAAAA|769|Dogwood |Wy|Suite K|Shady Grove|Sioux County|NE|62812|United States|-7|apartment| +27356|AAAAAAAAMNKGAAAA|202|Highland |Ln|Suite X|Brookville|Cherry County|NE|63524|United States|-6|single family| +27357|AAAAAAAANNKGAAAA|952|Spring 3rd|Wy|Suite 240|Edgewood|Morrison County|MN|50069|United States|-6|single family| +27358|AAAAAAAAONKGAAAA|997|Valley Valley|Street|Suite 320|Bridgeport|Hardin County|TX|75817|United States|-6|single family| +27359|AAAAAAAAPNKGAAAA|843|Second |Dr.|Suite E|Springfield|Kleberg County|TX|79303|United States|-6|condo| +27360|AAAAAAAAAOKGAAAA|398|Madison Meadow|Ct.|Suite C|Jackson|Eddy County|ND|59583|United States|-6|condo| +27361|AAAAAAAABOKGAAAA|397|Center Williams|Ave|Suite J|Oakwood|Mecklenburg County|NC|20169|United States|-5|apartment| +27362|AAAAAAAACOKGAAAA|989|Elm Chestnut|Road|Suite 340|Franklin|Lake of the Woods County|MN|59101|United States|-6|single family| +27363|AAAAAAAADOKGAAAA|659|Walnut |Road|Suite K|Clinton|Venango County|PA|18222|United States|-5|condo| +27364|AAAAAAAAEOKGAAAA|433|Walnut Johnson|Lane|Suite 10|Wildwood|Jackson County|OR|96871|United States|-8|condo| +27365|AAAAAAAAFOKGAAAA|907|13th |Way|Suite 330|Harmony|Gilmer County|GA|35804|United States|-5|condo| +27366|AAAAAAAAGOKGAAAA|798|Maple 9th|Cir.|Suite K|Forest|Grant County|KS|67537|United States|-6|apartment| +27367|AAAAAAAAHOKGAAAA|250|Park Lincoln|Ct.|Suite R|Belleville|Lamb County|TX|72924|United States|-6|condo| +27368|AAAAAAAAIOKGAAAA|853|Main Lake|Ave|Suite C|Woodlawn|Cass County|ND|54098|United States|-6|condo| +27369|AAAAAAAAJOKGAAAA|420|Railroad Central|Cir.|Suite 110|Riverview|Clearwater County|ID|89003|United States|-7|single family| +27370|AAAAAAAAKOKGAAAA|583|Dogwood |ST|Suite 220|Green Acres|Wyoming County|PA|17683|United States|-5|condo| +27371|AAAAAAAALOKGAAAA|98|Hill |Ln|Suite 420|Antioch|Muscogee County|GA|38605|United States|-5|single family| +27372|AAAAAAAAMOKGAAAA|692|Hill |Ct.|Suite 360|Sunnyside|Walton County|GA|31952|United States|-5|condo| +27373|AAAAAAAANOKGAAAA|10|Main |Ln|Suite 40|Shady Grove|Beltrami County|MN|52812|United States|-6|apartment| +27374|AAAAAAAAOOKGAAAA|731|Park |Lane|Suite 270|Jamestown|Isle of Wight County|VA|26867|United States|-5|condo| +27375|AAAAAAAAPOKGAAAA|379|8th Mill|Way|Suite Q|Pleasant Grove|Houston County|AL|34136|United States|-6|single family| +27376|AAAAAAAAAPKGAAAA|535|12th |Parkway|Suite 80|Fairfield|Lincoln County|SD|56192|United States|-7|apartment| +27377|AAAAAAAABPKGAAAA|719|8th |Ct.|Suite 130|Antioch|Teton County|MT|68605|United States|-7|condo| +27378|AAAAAAAACPKGAAAA|905|7th Poplar|Way|Suite 440|Pleasant Valley|Ray County|MO|62477|United States|-6|condo| +27379|AAAAAAAADPKGAAAA|452|1st |Circle|Suite 90|Bridgeport|Panola County|TX|75817|United States|-6|condo| +27380|AAAAAAAAEPKGAAAA|209|Ridge 10th|Pkwy|Suite 270|Providence|Independence County|AR|76614|United States|-6|condo| +27381|AAAAAAAAFPKGAAAA|856|Sycamore Birch|Avenue|Suite 90|Riverside|Brunswick County|VA|29231|United States|-5|condo| +27382|AAAAAAAAGPKGAAAA|291|Cedar |Court|Suite I|Green Acres|Dakota County|MN|57683|United States|-6|condo| +27383|AAAAAAAAHPKGAAAA|74|Oak |RD|Suite M|Mount Zion|Lenawee County|MI|48054|United States|-5|condo| +27384|AAAAAAAAIPKGAAAA|785|Third |Boulevard|Suite D|Shaw|Borden County|TX|70618|United States|-6|condo| +27385|AAAAAAAAJPKGAAAA||||Suite Q|||SC||||| +27386|AAAAAAAAKPKGAAAA|578|Lake Miller|Avenue|Suite E|Brownsville|Alger County|MI|49310|United States|-5|condo| +27387|AAAAAAAALPKGAAAA|801|Mill |ST|Suite 240|Jackson|Goshen County|WY|89583|United States|-7|single family| +27388|AAAAAAAAMPKGAAAA|248|Smith |Pkwy|Suite 430|Lamont|Marion County|IA|50204|United States|-6|condo| +27389|AAAAAAAANPKGAAAA|455|Mill 14th|Lane|Suite 390|Franklin|Beaufort County|SC|29101|United States|-5|apartment| +27390|AAAAAAAAOPKGAAAA|655|Main |Boulevard|Suite Y|Newtown|Phillips County|MT|61749|United States|-7|apartment| +27391|AAAAAAAAPPKGAAAA|894|Lakeview Jackson|Parkway|Suite X|Five Points|Bucks County|PA|16098|United States|-5|apartment| +27392|AAAAAAAAAALGAAAA|488|Forest |Dr.|Suite 310|Waterloo|Placer County|CA|91675|United States|-8|single family| +27393|AAAAAAAABALGAAAA|612|10th |Road|Suite K|Indian Village|Nemaha County|KS|61075|United States|-6|single family| +27394|AAAAAAAACALGAAAA|353|River Walnut|Road|Suite Y|Pleasant Hill|Tulare County|CA|93604|United States|-8|single family| +27395|AAAAAAAADALGAAAA|114|Washington |Drive|Suite 60|Avoca|Meriwether County|GA|30540|United States|-5|apartment| +27396|AAAAAAAAEALGAAAA|438|Jackson |Ave|Suite 490|Crossroads|Mingo County|WV|20534|United States|-5|single family| +27397|AAAAAAAAFALGAAAA|462|Sunset |Circle|Suite 50|Edgewood|Bradford County|PA|10069|United States|-5|single family| +27398|AAAAAAAAGALGAAAA|642|Cherry |Wy|Suite 90|Greenfield|Harvey County|KS|65038|United States|-6|apartment| +27399|AAAAAAAAHALGAAAA|727|Sunset |Ave|Suite W|Clifton|Cumberland County|KY|48014|United States|-6|single family| +27400|AAAAAAAAIALGAAAA|910|Walnut South|Ln|Suite Q|Enterprise|DeSoto County|MS|51757|United States|-6|single family| +27401|AAAAAAAAJALGAAAA|584|9th |Court|Suite D|Newport|Surry County|VA|21521|United States|-5|condo| +27402|AAAAAAAAKALGAAAA|213|9th |Blvd|Suite N|Providence|Coffey County|KS|66614|United States|-6|apartment| +27403|AAAAAAAALALGAAAA|||Ln||Sulphur Springs||||||apartment| +27404|AAAAAAAAMALGAAAA|339|College Sunset|Blvd|Suite 380|Riverview|Chippewa County|MI|49003|United States|-5|single family| +27405|AAAAAAAANALGAAAA|463|Hickory |Wy|Suite 110|Woodland|Tipton County|IN|44854|United States|-5|condo| +27406|AAAAAAAAOALGAAAA|633|Spruce Lincoln|Cir.|Suite V|Hilltop|Claiborne County|TN|32352|United States|-5|condo| +27407|AAAAAAAAPALGAAAA|889|11th Hickory|Parkway|Suite H|Sulphur Springs|Carter County|KY|48354|United States|-6|single family| +27408|AAAAAAAAABLGAAAA|504|8th 6th|Blvd|Suite X|Ashland|Hughes County|SD|54244|United States|-7|apartment| +27409|AAAAAAAABBLGAAAA|826|Spruce |Cir.|Suite D|Caldwell|Cavalier County|ND|56913|United States|-6|condo| +27410|AAAAAAAACBLGAAAA|718|Center 4th|Cir.|Suite 470|Clinton|Henry County|KY|48222|United States|-6|apartment| +27411|AAAAAAAADBLGAAAA|225|Center Dogwood|Way|Suite E|Riverside|Lyman County|SD|59231|United States|-7|condo| +27412|AAAAAAAAEBLGAAAA|887|View 3rd|Street|Suite O|Ashland|Queen Anne County|MD|24244|United States|-5|condo| +27413|AAAAAAAAFBLGAAAA|99|Dogwood Hillcrest|Ct.|Suite 230|Franklin|Divide County|ND|59101|United States|-6|apartment| +27414|AAAAAAAAGBLGAAAA|685|Washington |Avenue|Suite M|Greenfield|Person County|NC|25038|United States|-5|condo| +27415|AAAAAAAAHBLGAAAA|492|2nd East|Court|Suite W|Spring Valley|Hunterdon County|NJ|06660|United States|-5|condo| +27416|AAAAAAAAIBLGAAAA|397|8th |Avenue|Suite H|Five Points|Taliaferro County|GA|36098|United States|-5|condo| +27417|AAAAAAAAJBLGAAAA|482|West 9th|Parkway|Suite J|Greenwood|Granville County|NC|28828|United States|-5|single family| +27418|AAAAAAAAKBLGAAAA|135|View |Parkway|Suite D|Brownsville|Ashland County|OH|49310|United States|-5|condo| +27419|AAAAAAAALBLGAAAA|47|Third Oak|Street|Suite 110|Woodlawn|Park County|CO|84098|United States|-7|apartment| +27420|AAAAAAAAMBLGAAAA|390|Ridge |Ln|Suite 320|Shaw|Dinwiddie County|VA|20618|United States|-5|apartment| +27421|AAAAAAAANBLGAAAA|516|Williams Fifth|Ct.|Suite B|Milo|Robertson County|TN|30116|United States|-6|condo| +27422|AAAAAAAAOBLGAAAA|935|Cedar Lake|RD|Suite 50|Riverside|Mason County|KY|49231|United States|-5|single family| +27423|AAAAAAAAPBLGAAAA|802|3rd |Lane|Suite K|Bethel|Pocahontas County|WV|25281|United States|-5|condo| +27424|AAAAAAAAACLGAAAA|401|West |Drive|Suite Q|Sulphur Springs|Russell County|VA|28354|United States|-5|apartment| +27425|AAAAAAAABCLGAAAA|460|Washington Lee|Blvd|Suite F|Georgetown|Carroll County|TN|37057|United States|-5|apartment| +27426|AAAAAAAACCLGAAAA||Elm 12th|Pkwy|Suite 270||Tensas Parish|||United States|-6|| +27427|AAAAAAAADCLGAAAA|170|Main |Drive|Suite 140|Oakland|Iron County|MI|49843|United States|-5|single family| +27428|AAAAAAAAECLGAAAA|154|Poplar |Dr.|Suite 130|Waterloo|Morgan County|KY|41675|United States|-5|single family| +27429|AAAAAAAAFCLGAAAA|378|Ridge |Parkway|Suite M|Arlington|Lincoln County|OK|76557|United States|-6|single family| +27430|AAAAAAAAGCLGAAAA|380|Chestnut |Circle|Suite 60|Red Hill|Lawrence County|AL|34338|United States|-6|condo| +27431|AAAAAAAAHCLGAAAA|18|Walnut South|Wy|Suite 220|Five Points|Antelope County|NE|66098|United States|-6|apartment| +27432|AAAAAAAAICLGAAAA|314|Franklin |Circle||Bunker Hill|Nelson County||20150|United States||single family| +27433|AAAAAAAAJCLGAAAA|505|Johnson Park|Lane|Suite R|Hillcrest|Galveston County|TX|73003|United States|-6|single family| +27434|AAAAAAAAKCLGAAAA|525|4th Cherry|Dr.|Suite Y|Red Hill|Newport News city|VA|24338|United States|-5|apartment| +27435|AAAAAAAALCLGAAAA|82|Sunset Ridge|Lane|Suite D|Wilson|Grenada County|MS|56971|United States|-6|apartment| +27436|AAAAAAAAMCLGAAAA|285|Locust |Dr.|Suite 290|Kingston|Caroline County|VA|24975|United States|-5|condo| +27437|AAAAAAAANCLGAAAA|93|Willow |Avenue|Suite 160|Bear Creek|Mitchell County|GA|33075|United States|-5|apartment| +27438|AAAAAAAAOCLGAAAA|18|Ridge Lee|Drive|Suite 160|Liberty|Lake County|IL|63451|United States|-6|single family| +27439|AAAAAAAAPCLGAAAA|154|4th |Blvd|Suite 380|Walnut Grove|Trinity County|TX|77752|United States|-6|single family| +27440|AAAAAAAAADLGAAAA|761|View |Dr.|Suite F|Riverdale|Nantucket County|MA|09991|United States|-5|condo| +27441|AAAAAAAABDLGAAAA|972|River Wilson|Way|Suite E|Sulphur Springs|Schoolcraft County|MI|48354|United States|-5|condo| +27442|AAAAAAAACDLGAAAA|379|Ash North|Cir.|Suite 100|Crossroads|Pawnee County|NE|60534|United States|-7|condo| +27443|AAAAAAAADDLGAAAA|377|Pine |Dr.|Suite N|Oak Hill|Hyde County|SD|57838|United States|-7|apartment| +27444|AAAAAAAAEDLGAAAA|293|Tenth |Cir.|Suite U|Mount Vernon|Quitman County|GA|38482|United States|-5|single family| +27445|AAAAAAAAFDLGAAAA|513|Washington |Blvd|Suite 360|Willis|Morgan County|WV|26788|United States|-5|single family| +27446|AAAAAAAAGDLGAAAA|20|Woodland |Street|Suite L|Waterloo|McHenry County|ND|51675|United States|-6|apartment| +27447|AAAAAAAAHDLGAAAA|287|Woodland Cedar|Wy|Suite D|Concord|Colfax County|NE|64107|United States|-6|single family| +27448|AAAAAAAAIDLGAAAA|782|Cedar Williams|Avenue|Suite Q|Lebanon|Maui County|HI|92898|United States|-10|apartment| +27449|AAAAAAAAJDLGAAAA|553|Miller Lee|Avenue|Suite Y|Jackson|Cherokee County|GA|39583|United States|-5|condo| +27450|AAAAAAAAKDLGAAAA|932|4th 10th|Way|Suite 300|Lakeview|Cobb County|GA|38579|United States|-5|condo| +27451|AAAAAAAALDLGAAAA|336|Sunset 5th|Court|Suite X|Scottsville|Putnam County|NY|14190|United States|-5|condo| +27452|AAAAAAAAMDLGAAAA|284|10th |Road|Suite Y|Mount Olive|Kimball County|NE|68059|United States|-7|single family| +27453|AAAAAAAANDLGAAAA|574|1st Main|Pkwy|Suite N|Woodland|Jackson County|KY|44854|United States|-6|condo| +27454|AAAAAAAAODLGAAAA|17|Johnson |Ln|Suite 330|Oakland|Hopewell city|VA|29843|United States|-5|apartment| +27455|AAAAAAAAPDLGAAAA|508|Pine |Circle|Suite 90|Clifton|Ontario County|NY|18014|United States|-5|condo| +27456|AAAAAAAAAELGAAAA|239|12th West|Avenue|Suite A|Friendship|Erie County|OH|44536|United States|-5|single family| +27457|AAAAAAAABELGAAAA|961|Madison |Blvd|Suite R|Highland Park|Lee County|MS|56534|United States|-6|single family| +27458|AAAAAAAACELGAAAA|285|Meadow Sycamore|Circle|Suite B|Midway|Williamsburg County|SC|21904|United States|-5|single family| +27459|AAAAAAAADELGAAAA|570|Dogwood West|Avenue|Suite H|Granite|Lee County|AR|76284|United States|-6|apartment| +27460|AAAAAAAAEELGAAAA|856||Blvd|Suite 450||Thurston County|WA|90499|||condo| +27461|AAAAAAAAFELGAAAA|573|Ridge |Drive|Suite T|Bethel|Ogle County|IL|65281|United States|-6|single family| +27462|AAAAAAAAGELGAAAA|971|2nd |Blvd|Suite 230|Fowler|Duplin County|NC|21083|United States|-5|apartment| +27463|AAAAAAAAHELGAAAA|230|Adams |Parkway|Suite 420|Waterloo|Calhoun County|IL|61675|United States|-6|condo| +27464|AAAAAAAAIELGAAAA|169|Chestnut Lakeview|Lane|Suite 420|Pine Grove|Crawford County|IN|44593|United States|-5|condo| +27465|AAAAAAAAJELGAAAA|38|Valley Franklin|Avenue|Suite 490|Midway|Sevier County|TN|31904|United States|-6|condo| +27466|AAAAAAAAKELGAAAA|835|College Main|ST|Suite 280|Green Acres|Eddy County|NM|87683|United States|-7|condo| +27467|AAAAAAAALELGAAAA|917|East Williams|Circle|Suite A|Pierce|Knott County|KY|43360|United States|-5|single family| +27468|AAAAAAAAMELGAAAA|631|Pine 6th|Ave|Suite 350|Belmont|Benewah County|ID|80191|United States|-7|apartment| +27469|AAAAAAAANELGAAAA|386|Pine |Wy|Suite K|Wilson|Gibson County|IN|46971|United States|-5|apartment| +27470|AAAAAAAAOELGAAAA|472|Ridge Railroad|Way|Suite J|Greenfield|Kosciusko County|IN|45038|United States|-5|single family| +27471|AAAAAAAAPELGAAAA||||Suite X||Hamilton County|||United States|-6|| +27472|AAAAAAAAAFLGAAAA|630|Highland Cherry|Ln|Suite 430|New Hope|Ford County|KS|69431|United States|-6|apartment| +27473|AAAAAAAABFLGAAAA|69|Adams |Pkwy|Suite 30|Lakewood|Washington County|TN|38877|United States|-6|condo| +27474|AAAAAAAACFLGAAAA|863|Washington |RD|Suite 240|Fairbanks|Parker County|TX|76653|United States|-6|apartment| +27475|AAAAAAAADFLGAAAA|245|Fifth East|Street|Suite Y|Mount Olive|Dewey County|SD|58059|United States|-6|single family| +27476|AAAAAAAAEFLGAAAA|396|3rd |Wy|Suite 260|Lakeview|Somerset County|ME|09179|United States|-5|single family| +27477|AAAAAAAAFFLGAAAA|627|South Smith|Parkway|Suite 100|Lincoln|Steele County|MN|51289|United States|-6|condo| +27478|AAAAAAAAGFLGAAAA|25|Park |||||||United States|-5|| +27479|AAAAAAAAHFLGAAAA|310|Oak Elm|Ave|Suite M|Roscoe|Washtenaw County|MI|41854|United States|-5|single family| +27480|AAAAAAAAIFLGAAAA|852|4th |Avenue|Suite 70|Newtown|Roane County|TN|31749|United States|-6|apartment| +27481|AAAAAAAAJFLGAAAA|552|4th Seventh|Ln|Suite 90|Springdale|Currituck County|NC|28883|United States|-5|condo| +27482|AAAAAAAAKFLGAAAA|839|Third Pine|Ln|Suite 170|Mount Vernon|Letcher County|KY|48482|United States|-5|condo| +27483|AAAAAAAALFLGAAAA|538|11th |Avenue|Suite 40|Cedar Grove|Belmont County|OH|40411|United States|-5|single family| +27484|AAAAAAAAMFLGAAAA|562|Cherry |ST|Suite 70|Walnut Grove|Ware County|GA|37752|United States|-5|single family| +27485|AAAAAAAANFLGAAAA|461|Park Ninth|Street|Suite 240|Lincoln|Johnson County|IL|61289|United States|-6|condo| +27486|AAAAAAAAOFLGAAAA|540|1st |Drive|Suite V|Florence|Poweshiek County|IA|53394|United States|-6|condo| +27487|AAAAAAAAPFLGAAAA|105|Third Cedar|Drive|Suite 410|Oakwood|Muskogee County|OK|70169|United States|-6|condo| +27488|AAAAAAAAAGLGAAAA|278|North 1st|Cir.|Suite 50|Ashley|Bolivar County|MS|54324|United States|-6|condo| +27489|AAAAAAAABGLGAAAA|881|Laurel 1st|Ct.|Suite 410|Edgewood|Marion County|SC|20069|United States|-5|apartment| +27490|AAAAAAAACGLGAAAA|823|View 15th|ST|Suite 490|Enterprise|York County|SC|21757|United States|-5|single family| +27491|AAAAAAAADGLGAAAA|10|Dogwood |Pkwy|Suite 350|Riverview|Miami County|IN|49003|United States|-5|apartment| +27492|AAAAAAAAEGLGAAAA|768|Washington Meadow|Road|Suite 230|Shady Grove|Lancaster County|VA|22812|United States|-5|single family| +27493|AAAAAAAAFGLGAAAA|173|Cherry Spring|Ave|Suite 450|Enterprise|Hyde County|NC|21757|United States|-5|condo| +27494|AAAAAAAAGGLGAAAA|488|Smith |Blvd|Suite N|Oak Ridge|Menominee County|WI|58371|United States|-6|condo| +27495|AAAAAAAAHGLGAAAA|67|6th |Ln|Suite 70|Carter|Rhea County|TN|30919|United States|-6|apartment| +27496|AAAAAAAAIGLGAAAA|190|Main |ST|Suite V|Walnut Grove|Tippah County|MS|57752|United States|-6|apartment| +27497|AAAAAAAAJGLGAAAA|47|Elm |Court|Suite 320|Pleasant Hill|Hillsborough County|NH|04204|United States|-5|apartment| +27498|AAAAAAAAKGLGAAAA|579|Jackson View|Drive|Suite Q|Sulphur Springs|Nottoway County|VA|28354|United States|-5|condo| +27499|AAAAAAAALGLGAAAA|619|Lake |Court|Suite 360|Springdale|Harvey County|KS|68883|United States|-6|condo| +27500|AAAAAAAAMGLGAAAA|740|Washington |Street|Suite 430|Lakewood|Vermillion County|IN|48877|United States|-5|apartment| +27501|AAAAAAAANGLGAAAA|506|View Center|Boulevard|Suite 280|Greenwood|Columbia County|WA|98828|United States|-8|apartment| +27502|AAAAAAAAOGLGAAAA|185|First |Parkway|Suite B|Florence|Van Wert County|OH|43394|United States|-5|single family| +27503|AAAAAAAAPGLGAAAA|620|Sunset |Road|Suite 290|Pleasant Hill|Clay County|IN|43604|United States|-5|condo| +27504|AAAAAAAAAHLGAAAA||Sunset Fifth|Wy|Suite 410||Columbia County|NY|14466|United States||single family| +27505|AAAAAAAABHLGAAAA|425|Oak Sunset|RD|Suite 240|Bethel|Juneau Borough|AK|95281|United States|-9|single family| +27506|AAAAAAAACHLGAAAA|65|Park Cedar|Way|Suite H|Mount Pleasant|Allegheny County|PA|11933|United States|-5|condo| +27507|AAAAAAAADHLGAAAA|241|13th Railroad|Avenue|Suite Y|Midway|Columbus County|NC|21904|United States|-5|apartment| +27508|AAAAAAAAEHLGAAAA|941|Maple Chestnut|Lane|Suite 290|Valley View|Pocahontas County|IA|55124|United States|-6|condo| +27509|AAAAAAAAFHLGAAAA|500|Forest Main|Lane|Suite E|Fairview|Brazoria County|TX|75709|United States|-6|apartment| +27510|AAAAAAAAGHLGAAAA|686|13th |Way|Suite R|Lincoln|Clark County|MO|61289|United States|-6|apartment| +27511|AAAAAAAAHHLGAAAA|454|1st Maple|Way|Suite H|Wildwood|Galveston County|TX|76871|United States|-6|condo| +27512|AAAAAAAAIHLGAAAA|660|Sycamore |Ave|Suite S|Providence|Jackson County|SD|56614|United States|-7|apartment| +27513|AAAAAAAAJHLGAAAA|59|Walnut 11th|Avenue|Suite W|Waterloo|Knox County|OH|41675|United States|-5|apartment| +27514|AAAAAAAAKHLGAAAA|848|Birch |Avenue|Suite 410|Stringtown|Oscoda County|MI|40162|United States|-5|single family| +27515|AAAAAAAALHLGAAAA|421|Franklin Main|Boulevard|Suite 430|Greenwood|Curry County|OR|98828|United States|-8|apartment| +27516|AAAAAAAAMHLGAAAA|318|Hickory |Street|Suite 310|Lincoln|Mitchell County|IA|51289|United States|-6|condo| +27517|AAAAAAAANHLGAAAA|775|View |Street|Suite V|Spring Valley|Brown County|TX|76060|United States|-6|apartment| +27518|AAAAAAAAOHLGAAAA||South Oak||Suite N|Bridgeport||||United States||| +27519|AAAAAAAAPHLGAAAA|530||||||TX||United States|-6|apartment| +27520|AAAAAAAAAILGAAAA|167|Washington Fifteenth|Boulevard|Suite K|Glendale|McPherson County|SD|53951|United States|-7|apartment| +27521|AAAAAAAABILGAAAA|535|Lakeview |Court|Suite 180|Sunnyside|Kenedy County|TX|71952|United States|-6|apartment| +27522|AAAAAAAACILGAAAA|339|Maple Dogwood|Avenue|Suite I|Lincoln|Van Buren County|MI|41289|United States|-5|single family| +27523|AAAAAAAADILGAAAA|180|Twelfth |Court|Suite Q|Mount Zion|Jefferson County|OH|48054|United States|-5|condo| +27524|AAAAAAAAEILGAAAA|9|6th Highland|Parkway|Suite 140|Plainview|Dodge County|NE|63683|United States|-6|single family| +27525|AAAAAAAAFILGAAAA|373|Railroad |RD|Suite Q|Lakeside|Pocahontas County|WV|29532|United States|-5|single family| +27526|AAAAAAAAGILGAAAA|156|Lincoln Center|Cir.|Suite W|Springdale|Charlevoix County|MI|48883|United States|-5|condo| +27527|AAAAAAAAHILGAAAA|603|North |Parkway|Suite P|Wildwood|Orangeburg County|SC|26871|United States|-5|single family| +27528|AAAAAAAAIILGAAAA|609|Meadow |Way|Suite 20|Fairfield|Union County|NM|86192|United States|-7|condo| +27529|AAAAAAAAJILGAAAA|564|9th |Lane|Suite Q|Union Hill|Irion County|TX|77746|United States|-6|single family| +27530|AAAAAAAAKILGAAAA|360|11th |Ln|Suite C|Midway|Allegany County|NY|11904|United States|-5|condo| +27531|AAAAAAAALILGAAAA|449||Drive|Suite 270||Washington County|FL|||-5|| +27532|AAAAAAAAMILGAAAA|548|Maple Lake|Pkwy|Suite C|Oak Grove|Calhoun County|FL|38370|United States|-5|apartment| +27533|AAAAAAAANILGAAAA|764|4th |Cir.|Suite 30|Highland Park|Chesapeake city|VA|26534|United States|-5|apartment| +27534|AAAAAAAAOILGAAAA|707|Chestnut |Avenue|Suite C|Edgewood|Sherman County|TX|70069|United States|-6|condo| +27535|AAAAAAAAPILGAAAA|908|Chestnut |Boulevard|Suite 50|Colonial Heights|Cowley County|KS|63425|United States|-6|apartment| +27536|AAAAAAAAAJLGAAAA|38|Center View|Circle|Suite R|Wright|Boyd County|KY|42814|United States|-6|apartment| +27537|AAAAAAAABJLGAAAA|636|9th |Drive|Suite 110|Riverview|Choctaw County|MS|59003|United States|-6|condo| +27538|AAAAAAAACJLGAAAA|90|Locust |Court|Suite H|Providence|Harding County|NM|86614|United States|-7|single family| +27539|AAAAAAAADJLGAAAA|395|Railroad |Blvd|Suite B|Centerville|Garrett County|MD|20059|United States|-5|apartment| +27540|AAAAAAAAEJLGAAAA|883|Dogwood Madison|Drive|Suite Y|Spring Hill|Johnson County|AR|76787|United States|-6|condo| +27541|AAAAAAAAFJLGAAAA|292|Willow |Road|Suite 240|Bridgeport|Tensas Parish|LA|75817|United States|-6|apartment| +27542|AAAAAAAAGJLGAAAA|619|15th |Drive|Suite 330|Forest Hills|Elkhart County|IN|49237|United States|-5|condo| +27543|AAAAAAAAHJLGAAAA|372|Cedar |Circle|Suite U|Mount Pleasant|Granville County|NC|21933|United States|-5|apartment| +27544|AAAAAAAAIJLGAAAA|334|Jackson Pine|Street|Suite 0|Riverview|Hertford County|NC|29003|United States|-5|condo| +27545|AAAAAAAAJJLGAAAA||Walnut Park|||Highland|Montgomery County|IA|59454|United States||| +27546|AAAAAAAAKJLGAAAA|625|Railroad |Dr.|Suite 220|Shady Grove|Newton County|GA|32812|United States|-5|apartment| +27547|AAAAAAAALJLGAAAA|766|Main |Ln|Suite B|Cedar Grove|Hutchinson County|SD|50411|United States|-7|condo| +27548|AAAAAAAAMJLGAAAA|142|Locust 6th|ST|Suite S|Newtown|Forsyth County|GA|31749|United States|-5|condo| +27549|AAAAAAAANJLGAAAA|5|9th Wilson|Ave|Suite 140|Allison|Lyon County|NV|84167|United States|-8|apartment| +27550|AAAAAAAAOJLGAAAA|995|Walnut |Lane|Suite R|Oak Ridge|Floyd County|KY|48371|United States|-6|condo| +27551|AAAAAAAAPJLGAAAA|538|Adams |Blvd|Suite 350|Newtown|Allendale County|SC|21749|United States|-5|condo| +27552|AAAAAAAAAKLGAAAA|385||Ln|Suite C|||MD||United States|-5|| +27553|AAAAAAAABKLGAAAA|940|Ash Spruce|Cir.|Suite 450|Rolling Hills|Fairfax County|VA|27272|United States|-5|apartment| +27554|AAAAAAAACKLGAAAA|143|View 2nd|ST|Suite 160|Jenkins|Jennings County|IN|47292|United States|-5|apartment| +27555|AAAAAAAADKLGAAAA|621|9th |Blvd|Suite U|Kingston|Pike County|IL|64975|United States|-6|apartment| +27556|AAAAAAAAEKLGAAAA|669|Walnut Park|ST|Suite 100|Five Forks|Denali Borough|AK|92293|United States|-9|apartment| +27557|AAAAAAAAFKLGAAAA|960|Birch |Wy|Suite X|Lakewood|Turner County|SD|58877|United States|-7|single family| +27558|AAAAAAAAGKLGAAAA|13|Church Hill|Wy|Suite 60|Mount Zion|Monroe County|IN|48054|United States|-5|apartment| +27559|AAAAAAAAHKLGAAAA|654|Elm |Avenue|Suite 460|Farmington|Stark County|OH|49145|United States|-5|apartment| +27560|AAAAAAAAIKLGAAAA|777|Hill |RD|Suite 310|Silver Springs|Steele County|MN|54843|United States|-6|apartment| +27561|AAAAAAAAJKLGAAAA|285|Oak |Cir.|Suite 130|Lakeview|Hamilton County|IN|48579|United States|-5|condo| +27562|AAAAAAAAKKLGAAAA|683|Laurel |Dr.|Suite 380|Bethel|Terrell County|TX|75281|United States|-6|apartment| +27563|AAAAAAAALKLGAAAA|10|9th |Road|Suite 430|Maple Grove|Power County|ID|88252|United States|-7|condo| +27564|AAAAAAAAMKLGAAAA|894|Park 1st|RD|Suite 180|Springdale|Manassas city|VA|28883|United States|-5|apartment| +27565|AAAAAAAANKLGAAAA|864|6th |Parkway|Suite L|Griffin|Gordon County|GA|31204|United States|-5|single family| +27566|AAAAAAAAOKLGAAAA|402|3rd Railroad|Blvd|Suite M|Springfield|Kossuth County|IA|59303|United States|-6|single family| +27567|AAAAAAAAPKLGAAAA|616|2nd Jefferson|Court|Suite 60|Hillcrest|Wayne County|MS|53003|United States|-6|single family| +27568|AAAAAAAAALLGAAAA|488|Highland |Way|Suite 200|White Oak|Tate County|MS|56668|United States|-6|apartment| +27569|AAAAAAAABLLGAAAA|732|Park |Pkwy|Suite 30|Wilson|La Porte County|IN|46971|United States|-5|single family| +27570|AAAAAAAACLLGAAAA|872|Church Adams|Wy|Suite 360|Fairview|Cassia County|ID|85709|United States|-7|condo| +27571|AAAAAAAADLLGAAAA|631|Davis Church|Drive|Suite 460|Proctor|Holt County|NE|68140|United States|-7|condo| +27572|AAAAAAAAELLGAAAA|476|Park 11th|Wy|Suite 270|Florence|Grand County|UT|83394|United States|-7|condo| +27573|AAAAAAAAFLLGAAAA|412|4th Sycamore|Cir.|Suite D|Oak Ridge|Haskell County|KS|68371|United States|-6|condo| +27574|AAAAAAAAGLLGAAAA|990|5th |Boulevard|Suite L|Shiloh|Lafourche Parish|LA|79275|United States|-6|apartment| +27575|AAAAAAAAHLLGAAAA|300|Oak Ridge|Wy|Suite D|Walnut Grove|Ellis County|KS|67752|United States|-6|apartment| +27576|AAAAAAAAILLGAAAA|143|Twelfth |Blvd|Suite 200|Oak Hill|Baker County|OR|97838|United States|-8|single family| +27577|AAAAAAAAJLLGAAAA|307|Dogwood 9th|RD|Suite 40|Wilson|DuPage County|IL|66971|United States|-6|single family| +27578|AAAAAAAAKLLGAAAA|767|Forest |Parkway|Suite N|Newtown|Camp County|TX|71749|United States|-6|single family| +27579|AAAAAAAALLLGAAAA|973|West |Lane|Suite 170|Greenfield|Suwannee County|FL|35038|United States|-5|single family| +27580|AAAAAAAAMLLGAAAA|337|Wilson Ninth|Ave|Suite 50|Glenwood|Chautauqua County|NY|13511|United States|-5|condo| +27581|AAAAAAAANLLGAAAA|650|Poplar |Dr.|Suite 120|Newtown|Hooker County|NE|61749|United States|-7|condo| +27582|AAAAAAAAOLLGAAAA|434|East |Drive|Suite 310|Oak Ridge|Furnas County|NE|68371|United States|-6|condo| +27583|AAAAAAAAPLLGAAAA|259|Hickory Sunset|Ln|Suite 440|Woodlawn|Bergen County|NJ|04698|United States|-5|condo| +27584|AAAAAAAAAMLGAAAA|236|Lincoln |Pkwy|Suite F|Lebanon|Broward County|FL|32898|United States|-5|single family| +27585|AAAAAAAABMLGAAAA|922|12th |Circle|Suite N|Greenfield|Allen Parish|LA|75038|United States|-6|single family| +27586|AAAAAAAACMLGAAAA|340|Eigth |Wy|Suite 200|Mount Olive|Strafford County|NH|08659|United States|-5|condo| +27587|AAAAAAAADMLGAAAA|55|Walnut |Lane|Suite H|Newtown|Monroe County|NY|11749|United States|-5|apartment| +27588|AAAAAAAAEMLGAAAA|250|Lincoln Second|Way|Suite A|Pleasant Valley|Kootenai County|ID|82477|United States|-7|single family| +27589|AAAAAAAAFMLGAAAA|401|Lake |Circle|Suite 110|Jackson|Mahoning County|OH|49583|United States|-5|apartment| +27590|AAAAAAAAGMLGAAAA|838|4th Hickory|Ave|Suite 480|Mount Olive|Benton County|TN|38059|United States|-5|single family| +27591|AAAAAAAAHMLGAAAA|32|Sunset Spring|Ln|Suite 120|Harmony|Daniels County|MT|65804|United States|-7|apartment| +27592|AAAAAAAAIMLGAAAA|901|14th |Pkwy|Suite U|Wilson|Clark County|SD|56971|United States|-6|apartment| +27593|AAAAAAAAJMLGAAAA|587|Poplar Cedar|Road|Suite 330|Riverside|Franklin County|OH|49231|United States|-5|condo| +27594|AAAAAAAAKMLGAAAA||||Suite R|Glendale|Radford city|VA|23951|||| +27595|AAAAAAAALMLGAAAA|847|Walnut |RD|Suite 30|Unionville|Adams County|WA|91711|United States|-8|single family| +27596|AAAAAAAAMMLGAAAA|574|Park South||Suite 200|Forest Hills|Culberson County|||United States||apartment| +27597|AAAAAAAANMLGAAAA|551|7th |Boulevard|Suite G|Glendale|Barbour County|WV|23951|United States|-5|single family| +27598|AAAAAAAAOMLGAAAA|348|Green |Ave|Suite 120|Oneida|Alachua County|FL|34027|United States|-5|single family| +27599|AAAAAAAAPMLGAAAA|912|Maple |ST|Suite 220|Taft|Kit Carson County|CO|80589|United States|-7|single family| +27600|AAAAAAAAANLGAAAA|908|11th Park|RD|Suite 30|Richville|Portage County|WI|55945|United States|-6|condo| +27601|AAAAAAAABNLGAAAA|711|9th |Ave|Suite 190|Oakland|Adair County|OK|79843|United States|-6|condo| +27602|AAAAAAAACNLGAAAA|281|5th |Pkwy|Suite 200|Clifton|Covington County|MS|58014|United States|-6|condo| +27603|AAAAAAAADNLGAAAA|519|Second |Road|Suite K|Macedonia|Trempealeau County|WI|51087|United States|-6|single family| +27604|AAAAAAAAENLGAAAA|846|Hickory View|Ct.|Suite W|Red Hill|Sagadahoc County|ME|04938|United States|-5|single family| +27605|AAAAAAAAFNLGAAAA|419|6th North|Ln|Suite E|Jackson|Bristol County|RI|09583|United States|-5|single family| +27606|AAAAAAAAGNLGAAAA|139|Willow |Drive|Suite I|Shady Grove|Lassen County|CA|92812|United States|-8|single family| +27607|AAAAAAAAHNLGAAAA|399|Hill Chestnut|Ave|Suite T|New Hope|Bristol city|VA|29431|United States|-5|apartment| +27608|AAAAAAAAINLGAAAA|582|College Johnson|Drive|Suite 190|Stringtown|Colusa County|CA|90162|United States|-8|condo| +27609|AAAAAAAAJNLGAAAA|938|Hillcrest |Circle|Suite W|Collinsville|Vernon County|MO|62459|United States|-6|condo| +27610|AAAAAAAAKNLGAAAA|715|Madison |Ave|Suite K|Harmony|Erie County|OH|45804|United States|-5|condo| +27611|AAAAAAAALNLGAAAA|605|Walnut Elm|Street|Suite C|Marion|Nemaha County|NE|60399|United States|-7|apartment| +27612|AAAAAAAAMNLGAAAA|674|Cedar Valley|Avenue|Suite C|Oakland|Tripp County|SD|59843|United States|-7|condo| +27613|AAAAAAAANNLGAAAA|845|Madison Willow|Ln|Suite E|Wilson|Coal County|OK|76971|United States|-6|condo| +27614|AAAAAAAAONLGAAAA|73|Ash |Parkway|Suite K|Proctor|Jefferson County|PA|18140|United States|-5|condo| +27615|AAAAAAAAPNLGAAAA|||Dr.|Suite N|Woodville|Stanton County|NE|64289||-7|apartment| +27616|AAAAAAAAAOLGAAAA|531|Church 14th|Circle|Suite 60|Ashland|Franklin County|GA|34244|United States|-5|apartment| +27617|AAAAAAAABOLGAAAA|322|Fourth Fourth|Cir.|Suite 240|Oak Hill|Charles City County|VA|27838|United States|-5|condo| +27618|AAAAAAAACOLGAAAA|732|11th |Court|Suite X|Crossroads|Kandiyohi County|MN|50534|United States|-6|apartment| +27619|AAAAAAAADOLGAAAA|350|Walnut |Boulevard|Suite G|Sulphur Springs|Hawaii County|HI|98354|United States|-10|apartment| +27620|AAAAAAAAEOLGAAAA|668|2nd Wilson|Boulevard|Suite 90|Arlington|DeKalb County|AL|36557|United States|-6|apartment| +27621|AAAAAAAAFOLGAAAA|822|Woodland 2nd|Cir.|Suite 50|White Oak|Chattahoochee County|GA|36668|United States|-5|single family| +27622|AAAAAAAAGOLGAAAA|537|Davis |Road|Suite 260|Green Acres|Perry County|MO|67683|United States|-6|condo| +27623|AAAAAAAAHOLGAAAA|30|8th |Ave|Suite K|Richardson|Barnwell County|SC|27687|United States|-5|apartment| +27624|AAAAAAAAIOLGAAAA|97|7th Spring|Court|Suite F|Pleasant Grove|Winston County|AL|34136|United States|-6|apartment| +27625|AAAAAAAAJOLGAAAA|82|Main |RD|Suite E|Summit|Milam County|TX|70499|United States|-6|single family| +27626|AAAAAAAAKOLGAAAA||View |Parkway||Mount Olive|Ouachita Parish||78059||-6|| +27627|AAAAAAAALOLGAAAA|773|Lincoln Third|Ln|Suite 320|Buena Vista|Edwards County|IL|65752|United States|-6|single family| +27628|AAAAAAAAMOLGAAAA|970|Spruce 2nd|Lane|Suite 290|Oakwood|Midland County|TX|70169|United States|-6|condo| +27629|AAAAAAAANOLGAAAA|279|Walnut |Drive|Suite Q|Fairview|Hays County|TX|75709|United States|-6|single family| +27630|AAAAAAAAOOLGAAAA|369|Forest |Dr.|Suite 150|Greenville|Champaign County|IL|61387|United States|-6|single family| +27631|AAAAAAAAPOLGAAAA|395|Jackson 8th|RD|Suite 40|Lakewood|Lincoln County|MO|68877|United States|-6|condo| +27632|AAAAAAAAAPLGAAAA|634|Mill 4th|Drive|Suite G|Fairview|Crowley County|CO|85709|United States|-7|condo| +27633|AAAAAAAABPLGAAAA|172|West |Wy|Suite X|Mount Pleasant|Le Sueur County|MN|51933|United States|-6|apartment| +27634|AAAAAAAACPLGAAAA|503|West |Circle|Suite R||Monroe County||50534|United States|-6|| +27635|AAAAAAAADPLGAAAA|28|Green |Avenue|Suite R|Concord|Monterey County|CA|94107|United States|-8|single family| +27636|AAAAAAAAEPLGAAAA|315|Fourth Hickory|Cir.|Suite F|Providence|La Salle County|IL|66614|United States|-6|condo| +27637|AAAAAAAAFPLGAAAA|784|South Wilson|Street|Suite U|Ashland|Fulton County|PA|14244|United States|-5|apartment| +27638|AAAAAAAAGPLGAAAA|681|14th Park|ST|Suite 460|Riverdale|Camden County|MO|69391|United States|-6|apartment| +27639|AAAAAAAAHPLGAAAA|360|Park Main|Street|Suite 310|Little River|Mason County|MI|40319|United States|-5|apartment| +27640|AAAAAAAAIPLGAAAA|752|Oak Oak|Cir.|Suite 290|Murray|Decatur County|GA|32150|United States|-5|single family| +27641|AAAAAAAAJPLGAAAA|513|Sycamore |Blvd|Suite 60|Oak Ridge|Walworth County|WI|58371|United States|-6|single family| +27642|AAAAAAAAKPLGAAAA|670|Locust |Blvd|Suite 170|Salem|DeKalb County|IL|68048|United States|-6|apartment| +27643|AAAAAAAALPLGAAAA|834|Hillcrest Jefferson|Cir.|Suite 280|Lebanon|Denton County|TX|72898|United States|-6|condo| +27644|AAAAAAAAMPLGAAAA|714|West Williams|Court|Suite F|Arlington|Madison County|VA|26557|United States|-5|apartment| +27645|AAAAAAAANPLGAAAA|913|Spring |Court|Suite A|Unionville|Muskegon County|MI|41711|United States|-5|single family| +27646|AAAAAAAAOPLGAAAA|659|Poplar |Dr.|Suite 410|Jamestown|Adams County|IA|56867|United States|-6|apartment| +27647|AAAAAAAAPPLGAAAA|77|Seventh |Wy|Suite V|Maple Grove|Koochiching County|MN|58252|United States|-6|condo| +27648|AAAAAAAAAAMGAAAA|908|Elm Fifteenth|Circle|Suite 220|Pierce|Union Parish|LA|73360|United States|-6|condo| +27649|AAAAAAAABAMGAAAA|51|Lee |Street|Suite 40|Five Forks|York County|NE|62293|United States|-6|condo| +27650|AAAAAAAACAMGAAAA|357|South |Court|Suite W|Lakeview|Crawford County|MO|68579|United States|-6|apartment| +27651|AAAAAAAADAMGAAAA|470|Green |Boulevard|Suite 470|Florence|Oconee County|GA|33394|United States|-5|apartment| +27652|AAAAAAAAEAMGAAAA|439|1st |ST|Suite S|Oakwood|Wyandot County|OH|40169|United States|-5|condo| +27653|AAAAAAAAFAMGAAAA|623|Valley |Drive|Suite 60|Waterloo|Russell County|KS|61675|United States|-6|apartment| +27654|AAAAAAAAGAMGAAAA|694|Madison |Drive|Suite P|Highland Park|Gila County|AZ|86534|United States|-7|single family| +27655|AAAAAAAAHAMGAAAA|14|15th |Blvd|Suite 130|Midway|Tate County|MS|51904|United States|-6|condo| +27656|AAAAAAAAIAMGAAAA|559|Sycamore |Court|Suite I|Newtown|Crowley County|CO|81749|United States|-7|single family| +27657|AAAAAAAAJAMGAAAA|77|Main 5th|Ln|Suite O|Providence|Anderson County|TN|36614|United States|-5|condo| +27658|AAAAAAAAKAMGAAAA|148|Cedar |Drive|Suite L|Belmont|Gilmer County|WV|20191|United States|-5|single family| +27659|AAAAAAAALAMGAAAA|895|4th Miller|Street|Suite X|Rankin|Carbon County|MT|62621|United States|-7|apartment| +27660|AAAAAAAAMAMGAAAA|338|Lakeview Mill|Drive|Suite 220|Pleasant Valley|Jackson County|MN|52477|United States|-6|single family| +27661|AAAAAAAANAMGAAAA|685|3rd Central|Pkwy|Suite 30|Marion|Buncombe County|NC|20399|United States|-5|condo| +27662|AAAAAAAAOAMGAAAA|1000|Second |Ln|Suite R|Crossroads|Skamania County|WA|90534|United States|-8|apartment| +27663|AAAAAAAAPAMGAAAA|866|Wilson |Road|Suite B|Crossroads|Roosevelt County|MT|60534|United States|-7|single family| +27664|AAAAAAAAABMGAAAA|478|9th Birch|Blvd|Suite 200|Lakewood|Lafayette County|FL|38877|United States|-5|single family| +27665|AAAAAAAABBMGAAAA|754|2nd Fifth|Road|Suite 400|Fairview|Adams County|MS|55709|United States|-6|single family| +27666|AAAAAAAACBMGAAAA|915|Maple East|Dr.|Suite 450|Bunker Hill|Hertford County|NC|20150|United States|-5|condo| +27667|AAAAAAAADBMGAAAA|877|Chestnut |Wy|Suite T|Woodlawn|Lamoille County|VT|04698|United States|-5|condo| +27668|AAAAAAAAEBMGAAAA|971||Cir.|||||26557|United States||| +27669|AAAAAAAAFBMGAAAA|804|Miller |Avenue|Suite E|Oakwood|Coffey County|KS|60169|United States|-6|single family| +27670|AAAAAAAAGBMGAAAA|917|Cedar First|Ct.|Suite 90|Wilson|Ellis County|OK|76971|United States|-6|apartment| +27671|AAAAAAAAHBMGAAAA|413|4th Johnson|Lane|Suite B|Johnsonville|Jefferson County|TN|37745|United States|-6|condo| +27672|AAAAAAAAIBMGAAAA|340|Highland Broadway|Cir.|Suite N|Mountain View|Franklin County|IN|44466|United States|-5|apartment| +27673|AAAAAAAAJBMGAAAA|85|Highland |Blvd|Suite 420|Cedar Grove|Butler County|PA|10411|United States|-5|condo| +27674|AAAAAAAAKBMGAAAA|558|Lincoln Seventh|Ln|Suite 260|Edwards|Hood County|TX|71409|United States|-6|single family| +27675|AAAAAAAALBMGAAAA|628|6th Maple|Ct.|Suite M|Hopewell|Storey County|NV|80587|United States|-8|single family| +27676|AAAAAAAAMBMGAAAA|299|10th 2nd|RD|Suite 140|Highland|Kane County|IL|69454|United States|-6|apartment| +27677|AAAAAAAANBMGAAAA|228|Spruce |Parkway|Suite X|Jones|Salem County|NJ|03286|United States|-5|condo| +27678|AAAAAAAAOBMGAAAA|905|11th |Dr.|Suite J|Riverdale|Burt County|NE|69391|United States|-6|single family| +27679|AAAAAAAAPBMGAAAA|223|1st |Court|Suite 440|Granite|Floyd County|IA|56284|United States|-6|condo| +27680|AAAAAAAAACMGAAAA|60|Railroad |Wy|Suite 30|Salem|Windham County|VT|08648|United States|-5|apartment| +27681|AAAAAAAABCMGAAAA|830|Park |ST|Suite D|Riverdale|Denton County|TX|79391|United States|-6|apartment| +27682|AAAAAAAACCMGAAAA|284|Second Main|Boulevard|Suite C|Ashley|Tarrant County|TX|74324|United States|-6|condo| +27683|AAAAAAAADCMGAAAA|221|Railroad Washington|Blvd|Suite 210|Hopewell|Saline County|MO|60587|United States|-6|condo| +27684|AAAAAAAAECMGAAAA|47|Sycamore |Avenue|Suite 80|Fairview|Clackamas County|OR|95709|United States|-8|condo| +27685|AAAAAAAAFCMGAAAA|4|Fourth Park|RD|Suite D|Saint James|Kingsbury County|SD|55799|United States|-7|single family| +27686|AAAAAAAAGCMGAAAA|702|1st |RD|Suite B|Plainview|Lancaster County|VA|23683|United States|-5|condo| +27687|AAAAAAAAHCMGAAAA|500|Lake |Ct.|Suite 220|Greenwood|Marshall County|IA|58828|United States|-6|condo| +27688|AAAAAAAAICMGAAAA|222|Second |Avenue|Suite I|Green Acres|Berkshire County|MA|08283|United States|-5|condo| +27689|AAAAAAAAJCMGAAAA|847|Smith |Street|Suite 300|Pine Grove|Grady County|GA|34593|United States|-5|apartment| +27690|AAAAAAAAKCMGAAAA|837|3rd North|Ave|Suite 210|Riverdale|Essex County|VT|09991|United States|-5|single family| +27691|AAAAAAAALCMGAAAA|699||||Page|Grant County|||United States|-6|apartment| +27692|AAAAAAAAMCMGAAAA|394|Central |Blvd|Suite 90|Saint George|Lapeer County|MI|45281|United States|-5|apartment| +27693|AAAAAAAANCMGAAAA|941|Jackson Forest|Cir.|Suite 190|Providence|Winston County|AL|36614|United States|-6|apartment| +27694|AAAAAAAAOCMGAAAA|999|Hill |Court|Suite 320|Waterloo|Essex County|MA|02275|United States|-5|condo| +27695|AAAAAAAAPCMGAAAA|801|Mill |Parkway|Suite 430|Belmont|Vigo County|IN|40191|United States|-5|condo| +27696|AAAAAAAAADMGAAAA|126|Woodland |Boulevard|Suite F|Sunnyside|West Carroll Parish|LA|71952|United States|-6|single family| +27697|AAAAAAAABDMGAAAA|861|Dogwood |Pkwy|Suite J|Waterloo|Rowan County|NC|21675|United States|-5|condo| +27698|AAAAAAAACDMGAAAA|573|4th Jefferson|Ave|Suite T|Salem|Madison County|IN|48048|United States|-5|apartment| +27699|AAAAAAAADDMGAAAA|318|Sixth |Pkwy|Suite B|Lebanon|San Bernardino County|CA|92898|United States|-8|apartment| +27700|AAAAAAAAEDMGAAAA|407|Chestnut |Road|Suite H|Buckhorn|Greene County|AL|35458|United States|-6|condo| +27701|AAAAAAAAFDMGAAAA|917|8th Park|Street|Suite 390|Woodland|Rutherford County|NC|24854|United States|-5|single family| +27702|AAAAAAAAGDMGAAAA|284|Valley |RD|Suite W|Friendship|Bee County|TX|74536|United States|-6|apartment| +27703|AAAAAAAAHDMGAAAA|414|Park |Avenue|Suite A|Harmony|Sherman County|KS|65804|United States|-6|condo| +27704|AAAAAAAAIDMGAAAA|335|Hickory |Way|Suite 110|Riverview|Sedgwick County|KS|69003|United States|-6|single family| +27705|AAAAAAAAJDMGAAAA|798|Jackson |ST|Suite 220|Clinton|Pulaski County|IN|48222|United States|-5|apartment| +27706|AAAAAAAAKDMGAAAA|16|Hillcrest |Way|Suite 270|Walnut Grove|Stanly County|NC|27752|United States|-5|apartment| +27707|AAAAAAAALDMGAAAA|822|7th |Avenue|Suite O|Pleasant Hill|Manitowoc County|WI|53604|United States|-6|single family| +27708|AAAAAAAAMDMGAAAA|254|Washington |Court|Suite D|Lee|Monona County|IA|50408|United States|-6|apartment| +27709|AAAAAAAANDMGAAAA|806|North |Parkway|Suite E|Farmington|Cleveland County|OK|79145|United States|-6|apartment| +27710|AAAAAAAAODMGAAAA|990|Willow Washington|Drive|Suite H|Post Oak|Boundary County|ID|88567|United States|-7|apartment| +27711|AAAAAAAAPDMGAAAA||||Suite 430||Juniata County||16787|United States||| +27712|AAAAAAAAAEMGAAAA|634|Lake |ST|Suite 280|Edgewood|Greene County|IA|50069|United States|-6|condo| +27713|AAAAAAAABEMGAAAA|416|River Wilson|Pkwy|Suite 430|Union|Windham County|CT|09321|United States|-5|apartment| +27714|AAAAAAAACEMGAAAA|623|Lincoln |Parkway|Suite 400|Mount Vernon|Washington County|PA|18482|United States|-5|condo| +27715|AAAAAAAADEMGAAAA|3|Center 5th|Blvd|Suite R|Longwood|Sullivan County|IN|47021|United States|-5|single family| +27716|AAAAAAAAEEMGAAAA|456|Highland Williams|Avenue|Suite J|Riverdale|Wilcox County|GA|39391|United States|-5|condo| +27717|AAAAAAAAFEMGAAAA|331|10th 3rd|Cir.|Suite 130|Jackson|Lamar County|TX|79583|United States|-6|single family| +27718|AAAAAAAAGEMGAAAA|259|10th 7th|Wy|Suite 50|Pine Grove|Lunenburg County|VA|24593|United States|-5|condo| +27719|AAAAAAAAHEMGAAAA|249|View Washington|Wy|Suite 490|Reno|Storey County|NV|80344|United States|-8|single family| +27720|AAAAAAAAIEMGAAAA|259|Sunset Fifth||Suite R|Friendship||CA|||-8|condo| +27721|AAAAAAAAJEMGAAAA|214|River Woodland|Cir.|Suite A|Hamilton|Macon County|NC|22808|United States|-5|single family| +27722|AAAAAAAAKEMGAAAA|92|Lincoln Jackson|Cir.|Suite D|Willis|Clarke County|IA|56788|United States|-6|condo| +27723|AAAAAAAALEMGAAAA|396|5th |Pkwy|Suite 150|Springfield|Cabell County|WV|29303|United States|-5|single family| +27724|AAAAAAAAMEMGAAAA|866||Circle||Greenville||||United States|-7|single family| +27725|AAAAAAAANEMGAAAA|642|Willow South|Cir.|Suite 200|Collinsville|Henry County|VA|22459|United States|-5|apartment| +27726|AAAAAAAAOEMGAAAA|245|Chestnut Sunset|Circle|Suite 280|Mount Olive|Clark County|WA|98059|United States|-8|condo| +27727|AAAAAAAAPEMGAAAA|255|Ash Davis|Ct.|Suite M|Valley View|Marion County|OR|95124|United States|-8|single family| +27728|AAAAAAAAAFMGAAAA|202|3rd |Avenue|Suite 470|Hopewell|Union County|KY|40587|United States|-5|apartment| +27729|AAAAAAAABFMGAAAA|236|7th Jefferson|Blvd|Suite F|Green Acres|Clatsop County|OR|97683|United States|-8|apartment| +27730|AAAAAAAACFMGAAAA|382|12th |Dr.|Suite 90|Union City|Leflore County|MS|58087|United States|-6|condo| +27731|AAAAAAAADFMGAAAA|885|1st |Blvd|Suite 420|Woodlawn|Boyd County|NE|64098|United States|-6|apartment| +27732|AAAAAAAAEFMGAAAA|808|Park Third|Ave|Suite G|Greenfield|Wallace County|KS|65038|United States|-6|apartment| +27733|AAAAAAAAFFMGAAAA|126|Franklin |Drive|Suite R|Fairfield|East Baton Rouge Parish|LA|76192|United States|-6|apartment| +27734|AAAAAAAAGFMGAAAA|902|View 3rd|Court|Suite K|Jacksonville|Perkins County|NE|68223|United States|-7|single family| +27735|AAAAAAAAHFMGAAAA|908|3rd |Drive|Suite O|Shaw|Wayne County|IL|60618|United States|-6|single family| +27736|AAAAAAAAIFMGAAAA|948|Chestnut |Road|Suite T|||OR|98059||-8|| +27737|AAAAAAAAJFMGAAAA|630|Railroad Sunset|Ave|Suite 470|Red Hill|Tippah County|MS|54338|United States|-6|condo| +27738|AAAAAAAAKFMGAAAA|34|College 7th|Ave|Suite 340|Valley View|Cass County|ND|55124|United States|-6|single family| +27739|AAAAAAAALFMGAAAA|960|Main |Ln|Suite 360|Marion|Storey County|NV|80399|United States|-8|apartment| +27740|AAAAAAAAMFMGAAAA|472|Park |Court|Suite 40|Oakland|Thurston County|NE|69843|United States|-7|condo| +27741|AAAAAAAANFMGAAAA|273|13th |Ave|Suite 330|Flint|Dickey County|ND|58909|United States|-6|single family| +27742|AAAAAAAAOFMGAAAA|858|Walnut Lake|Circle|Suite 320|Springdale|Nassau County|FL|38883|United States|-5|apartment| +27743|AAAAAAAAPFMGAAAA|298|West |Dr.|Suite 460|Arlington|Muhlenberg County|KY|46557|United States|-5|single family| +27744|AAAAAAAAAGMGAAAA|511|Elm Park|Ct.|Suite I|Crossroads|Wabaunsee County|KS|60534|United States|-6|single family| +27745|AAAAAAAABGMGAAAA|421|Valley Hickory|Boulevard|Suite H|Oakdale|Flathead County|MT|69584|United States|-7|apartment| +27746|AAAAAAAACGMGAAAA|487|Spruce |Blvd|Suite L|Unionville|Cabarrus County|NC|21711|United States|-5|single family| +27747|AAAAAAAADGMGAAAA|259|College Fifth|Lane|Suite 220|Providence|Pasquotank County|NC|26614|United States|-5|condo| +27748|AAAAAAAAEGMGAAAA|776|12th |RD|Suite B|Oak Grove|Warren County|NJ|08970|United States|-5|single family| +27749|AAAAAAAAFGMGAAAA|253|||||Orleans County|NY||United States|-5|| +27750|AAAAAAAAGGMGAAAA|598|Eigth |Drive|Suite H|Spring Hill|Harrison County|OH|46787|United States|-5|condo| +27751|AAAAAAAAHGMGAAAA|120|Fifth Highland|Court|Suite 250|Belmont|Wadena County|MN|50191|United States|-6|condo| +27752|AAAAAAAAIGMGAAAA|691|Church |Street|Suite A|Centerville|Jefferson County|AR|70059|United States|-6|single family| +27753|AAAAAAAAJGMGAAAA|784|12th Wilson|Dr.|Suite 40|Jackson|Pembina County|ND|59583|United States|-6|condo| +27754|AAAAAAAAKGMGAAAA|731|7th |Dr.|Suite B|Glenwood|Aleutians East Borough|AK|93511|United States|-9|condo| +27755|AAAAAAAALGMGAAAA|880|Park South|Ct.|Suite 430|Shiloh|Riley County|KS|69275|United States|-6|condo| +27756|AAAAAAAAMGMGAAAA|1|Maple North|||Mount Pleasant||||United States|-5|condo| +27757|AAAAAAAANGMGAAAA|20|Walnut Oak|Street|Suite 230|Oakwood|Cumberland County|KY|40169|United States|-6|single family| +27758|AAAAAAAAOGMGAAAA|302|9th |Avenue|Suite E|Farmersville|Aroostook County|ME|09905|United States|-5|condo| +27759|AAAAAAAAPGMGAAAA|487|Elm |ST|Suite X|Payne|Monroe County|MS|56134|United States|-6|condo| +27760|AAAAAAAAAHMGAAAA|733|Meadow |Pkwy|Suite 90|Oak Hill|Logan County|IL|67838|United States|-6|apartment| +27761|AAAAAAAABHMGAAAA|197|15th South|Blvd|Suite X|Franklin|Craig County|VA|29101|United States|-5|single family| +27762|AAAAAAAACHMGAAAA|395|8th Maple|Drive|Suite X|Greenwood|Heard County|GA|38828|United States|-5|single family| +27763|AAAAAAAADHMGAAAA|643|Mill |Avenue|Suite H|Arlington|Columbia County|PA|16557|United States|-5|apartment| +27764|AAAAAAAAEHMGAAAA|413|Franklin Spruce|Ln|Suite 320|Pleasant Valley|Allegany County|NY|12477|United States|-5|single family| +27765|AAAAAAAAFHMGAAAA|769|Broadway |Court|Suite 430|Forest|Armstrong County|PA|17537|United States|-5|condo| +27766|AAAAAAAAGHMGAAAA|429|Main Ash|Cir.|Suite J|Midway|Lauderdale County|TN|31904|United States|-6|single family| +27767|AAAAAAAAHHMGAAAA|206|Lake Wilson|Court|Suite 320|Lebanon|Lincoln County|GA|32898|United States|-5|single family| +27768|AAAAAAAAIHMGAAAA|73|Central |Ave|Suite D|Deerfield|Geauga County|OH|49840|United States|-5|condo| +27769|AAAAAAAAJHMGAAAA|903|Madison |Pkwy|Suite 460|Rockwood|Trempealeau County|WI|51545|United States|-6|apartment| +27770|AAAAAAAAKHMGAAAA|751|Cedar Mill|Road|Suite 450|Ludlow|McDonough County|IL|65566|United States|-6|single family| +27771|AAAAAAAALHMGAAAA|652|2nd |Road|Suite 140|Antioch|Burke County|GA|38605|United States|-5|condo| +27772|AAAAAAAAMHMGAAAA|354|Green |Parkway|Suite A|Woodville|Hampton County|SC|24289|United States|-5|condo| +27773|AAAAAAAANHMGAAAA|869|Adams |Dr.|Suite 300|Clifton|Goodhue County|MN|58014|United States|-6|condo| +27774|AAAAAAAAOHMGAAAA|902|Oak |||Buena Vista||||United States|-6|single family| +27775|AAAAAAAAPHMGAAAA|717|Center |Ln|Suite T|Mount Vernon|Lee County|TX|78482|United States|-6|apartment| +27776|AAAAAAAAAIMGAAAA|898|Central |ST|Suite 490|Lebanon|Newton County|GA|32898|United States|-5|condo| +27777|AAAAAAAABIMGAAAA|544|Smith Davis|Ct.|Suite 90|Belmont|Alexandria city|VA|20191|United States|-5|apartment| +27778|AAAAAAAACIMGAAAA|137|2nd |Ln|Suite D|Parkwood|Bennett County|SD|51669|United States|-6|condo| +27779|AAAAAAAADIMGAAAA|35|Mill |Ave|Suite K|Salem|Noble County|IN|48048|United States|-5|condo| +27780|AAAAAAAAEIMGAAAA|892|Fourth |Pkwy|Suite 440|Arlington|Auglaize County|OH|46557|United States|-5|apartment| +27781|AAAAAAAAFIMGAAAA|754|10th College|Ct.|Suite D|Belleville|Bland County|VA|22924|United States|-5|single family| +27782|AAAAAAAAGIMGAAAA|468|Green Johnson|Blvd|Suite 280|Cedar Grove|Somervell County|TX|70411|United States|-6|condo| +27783|AAAAAAAAHIMGAAAA|625|Laurel Franklin|Avenue|Suite 300|Plainview|Harris County|TX|73683|United States|-6|apartment| +27784|AAAAAAAAIIMGAAAA|633|3rd |Parkway|Suite H|Concord|Forest County|PA|14107|United States|-5|apartment| +27785|AAAAAAAAJIMGAAAA|609|Dogwood East|RD|Suite K|Ashland|Lawrence County|IL|64244|United States|-6|apartment| +27786|AAAAAAAAKIMGAAAA|367|Hillcrest Cedar|Cir.|Suite S|Walnut Grove|Dickinson County|KS|67752|United States|-6|apartment| +27787|AAAAAAAALIMGAAAA|223|North |RD|Suite V|Shiloh|Aleutians West Census Area|AK|99275|United States|-9|condo| +27788|AAAAAAAAMIMGAAAA|419|Green |Parkway|Suite X|Cedar Grove|Beckham County|OK|70411|United States|-6|single family| +27789|AAAAAAAANIMGAAAA|441|Franklin Railroad|Cir.|Suite 360|Wilson|Benton County|MO|66971|United States|-6|condo| +27790|AAAAAAAAOIMGAAAA|433|Central |Lane|Suite 60|Richville|McCreary County|KY|45945|United States|-5|single family| +27791|AAAAAAAAPIMGAAAA|4|Lake 9th|RD|Suite K|New Hope|Emery County|UT|89431|United States|-7|apartment| +27792|AAAAAAAAAJMGAAAA|321|Central |Ct.|Suite 70|Oak Hill|Franklin County|GA|37838|United States|-5|condo| +27793|AAAAAAAABJMGAAAA|666|1st Jackson|Way|Suite A|Wilson|Holt County|NE|66971|United States|-7|single family| +27794|AAAAAAAACJMGAAAA|712|Main |Wy|Suite R|Lebanon|Clay County|TN|32898|United States|-5|single family| +27795|AAAAAAAADJMGAAAA|267|Johnson Washington|Blvd|Suite V|Glenwood|Grayson County|KY|43511|United States|-6|single family| +27796|AAAAAAAAEJMGAAAA|876|8th |Circle|Suite 80|Pleasant Valley|Fulton County|OH|42477|United States|-5|single family| +27797|AAAAAAAAFJMGAAAA|419|Franklin |Dr.|Suite Q|Woodlawn|Powell County|KY|44098|United States|-5|single family| +27798|AAAAAAAAGJMGAAAA|247|Railroad Poplar|Dr.|Suite J|Belmont|Sacramento County|CA|90191|United States|-8|apartment| +27799|AAAAAAAAHJMGAAAA|165|1st Spring|Ave|Suite L|Frogtown|Sevier County|TN|38784|United States|-6|condo| +27800|AAAAAAAAIJMGAAAA|129|2nd |Ave|Suite 390|Lakeside|San Diego County|CA|99532|United States|-8|single family| +27801|AAAAAAAAJJMGAAAA|58|Hillcrest Lee|Circle|Suite P|Belmont|Ottawa County|OK|70191|United States|-6|apartment| +27802|AAAAAAAAKJMGAAAA|675|10th Adams|Circle|Suite 330|Sutton|Phillips County|AR|75413|United States|-6|condo| +27803|AAAAAAAALJMGAAAA|77|Main Second|Way|Suite 480|Mount Vernon|Salem city|VA|28482|United States|-5|apartment| +27804|AAAAAAAAMJMGAAAA|466|Ridge |Drive|Suite O|Edgewood|Titus County|TX|70069|United States|-6|condo| +27805|AAAAAAAANJMGAAAA|16|6th |RD|Suite D|Wilson|Hillsborough County|NH|07571|United States|-5|single family| +27806|AAAAAAAAOJMGAAAA|692|View |Ln|Suite 370|Clifton|Adams County|WA|98014|United States|-8|apartment| +27807|AAAAAAAAPJMGAAAA||||||Abbeville County|SC||||| +27808|AAAAAAAAAKMGAAAA|301|4th |Way|Suite 310|Jamestown|Covington County|MS|56867|United States|-6|condo| +27809|AAAAAAAABKMGAAAA|607|9th |Wy|Suite 320|Oak Hill|Douglas County|MO|67838|United States|-6|condo| +27810|AAAAAAAACKMGAAAA|249|West Birch|Wy|Suite 460|Ashland|Wilcox County|GA|34244|United States|-5|apartment| +27811|AAAAAAAADKMGAAAA|212|Ash |Dr.|Suite W|Bunker Hill|Yukon-Koyukuk Census Area|AK|90150|United States|-9|condo| +27812|AAAAAAAAEKMGAAAA|890|Ninth |RD|Suite 210|Woodlawn|Clay County|IN|44098|United States|-5|apartment| +27813|AAAAAAAAFKMGAAAA|926|14th Laurel|Avenue|Suite 210|Macedonia|Louisa County|VA|21087|United States|-5|single family| +27814|AAAAAAAAGKMGAAAA|544|Elm 7th|Road|Suite 460|Springfield|Perry County|PA|19303|United States|-5|condo| +27815|AAAAAAAAHKMGAAAA|353|Mill 6th|Circle|Suite 240|Bunker Hill|Washington County|VT|00750|United States|-5|condo| +27816|AAAAAAAAIKMGAAAA|783|Johnson 3rd|Cir.|Suite R|New Hope|Union County|NJ|09431|United States|-5|apartment| +27817|AAAAAAAAJKMGAAAA|70|8th |Cir.|Suite 380|Shiloh|Ashley County|AR|79275|United States|-6|apartment| +27818|AAAAAAAAKKMGAAAA|792|1st Jefferson|Blvd|Suite Q|Unionville|Clarke County|MS|51711|United States|-6|apartment| +27819|AAAAAAAALKMGAAAA|752|Johnson 1st|Ln|Suite J|Guilford|Clackamas County|OR|94408|United States|-8|apartment| +27820|AAAAAAAAMKMGAAAA|633|14th |Drive|Suite P|Greenwood|Valley County|ID|88828|United States|-7|condo| +27821|AAAAAAAANKMGAAAA|883|10th |Street|Suite I|Springfield|Cherry County|NE|69303|United States|-6|single family| +27822|AAAAAAAAOKMGAAAA|491|7th |Dr.|Suite 120|Arlington|Robertson County|TX|76557|United States|-6|single family| +27823|AAAAAAAAPKMGAAAA|890|East |Avenue|Suite 310|Forest Hills|Bleckley County|GA|39237|United States|-5|condo| +27824|AAAAAAAAALMGAAAA|233|Cedar South|Street|Suite H|Jackson|McDonough County|IL|69583|United States|-6|condo| +27825|AAAAAAAABLMGAAAA|852|North Hill|Street|Suite 250|Lakewood|Macomb County|MI|48877|United States|-5|single family| +27826|AAAAAAAACLMGAAAA|775|Second |Wy|Suite P|Bethel|Union County|IN|45281|United States|-5|apartment| +27827|AAAAAAAADLMGAAAA|731|13th Smith|Ct.|Suite Y|Arlington|Rio Blanco County|CO|86557|United States|-7|apartment| +27828|AAAAAAAAELMGAAAA|421|Hill |Drive|Suite 0|Springdale|Sheridan County|ND|58883|United States|-6|apartment| +27829|AAAAAAAAFLMGAAAA|499|Central |Ct.|Suite N|Pleasant Hill|Chester County|PA|13604|United States|-5|single family| +27830|AAAAAAAAGLMGAAAA|442|South |Ave|Suite T|Harmony|Lincoln County|MS|55804|United States|-6|apartment| +27831|AAAAAAAAHLMGAAAA|518|Ridge |Ave|Suite 240|Liberty|Georgetown County|SC|23451|United States|-5|condo| +27832|AAAAAAAAILMGAAAA|243|Lincoln Broadway|Way|Suite D|Lakeview|Madison County|AR|78579|United States|-6|condo| +27833|AAAAAAAAJLMGAAAA|355|Walnut Johnson|Street|Suite 300|Riverdale|York County|PA|19391|United States|-5|condo| +27834|AAAAAAAAKLMGAAAA|657|Thirteenth Lakeview|RD|Suite Y|Arlington|Lander County|NV|86557|United States|-8|apartment| +27835|AAAAAAAALLMGAAAA|483|Seventh Sycamore|Avenue|Suite 360|Oakland|Clay County|AL|39843|United States|-6|single family| +27836|AAAAAAAAMLMGAAAA|966|Elm |Avenue|Suite 10|Hazelwood|Kings County|CA|91206|United States|-8|single family| +27837|AAAAAAAANLMGAAAA|808|Park Maple|Boulevard|Suite U|Allentown|Columbia County|GA|31838|United States|-5|condo| +27838|AAAAAAAAOLMGAAAA|940|River |Ct.|Suite A|Franklin|Tippecanoe County|IN|49101|United States|-5|single family| +27839|AAAAAAAAPLMGAAAA|406|Twelfth Main|Wy|Suite O|Pleasant Valley|McLennan County|TX|72477|United States|-6|single family| +27840|AAAAAAAAAMMGAAAA|207|Davis Oak|Circle|Suite 380|Spring Hill|Miller County|AR|76787|United States|-6|single family| +27841|AAAAAAAABMMGAAAA|140|Main Sunset|Boulevard|Suite Q|Sunnyside|Washington County|GA|31952|United States|-5|apartment| +27842|AAAAAAAACMMGAAAA|516|Jefferson |Road|Suite 470|Arlington|Pulaski County|MO|66557|United States|-6|single family| +27843|AAAAAAAADMMGAAAA|771|Elm |Way|Suite A|Georgetown|La Salle County|TX|77057|United States|-6|single family| +27844|AAAAAAAAEMMGAAAA|308|Jefferson 3rd|Boulevard|Suite W|Gladstone|Culpeper County|VA|20894|United States|-5|apartment| +27845|AAAAAAAAFMMGAAAA|493|Williams Meadow|Wy|Suite 210|Vance|Monroe County|IA|50268|United States|-6|single family| +27846|AAAAAAAAGMMGAAAA|648|Park |ST|Suite D|Greenfield|Weld County|CO|85038|United States|-7|single family| +27847|AAAAAAAAHMMGAAAA|952|5th |Boulevard|Suite 110|Lakewood|Natrona County|WY|88877|United States|-7|apartment| +27848|AAAAAAAAIMMGAAAA|259|Smith |Avenue|Suite R|Bridgeport|Seward County|KS|65817|United States|-6|apartment| +27849|AAAAAAAAJMMGAAAA|408|Hickory |Lane|Suite A|Bridgeport|Jasper County|TX|75817|United States|-6|apartment| +27850|AAAAAAAAKMMGAAAA|260|Walnut |Boulevard|Suite E|White Oak|Washington Parish|LA|76668|United States|-6|apartment| +27851|AAAAAAAALMMGAAAA|976|River Locust|Boulevard|Suite 40|New Hope|Onslow County|NC|29431|United States|-5|apartment| +27852|AAAAAAAAMMMGAAAA|279|Sunset Williams|Court|Suite R|Friendship|McDowell County|NC|24536|United States|-5|condo| +27853|AAAAAAAANMMGAAAA|905|Sunset |Court|Suite C|Ashland|Crawford County|AR|74244|United States|-6|single family| +27854|AAAAAAAAOMMGAAAA|909|Valley Sycamore|Pkwy||Riverview|Armstrong County|PA|||-5|condo| +27855|AAAAAAAAPMMGAAAA|469|First Pine|ST|Suite R|Pierce|Val Verde County|TX|73360|United States|-6|single family| +27856|AAAAAAAAANMGAAAA|378|Spring |ST|Suite 220|Mount Vernon|Anderson County|SC|28482|United States|-5|condo| +27857|AAAAAAAABNMGAAAA|466|Highland 15th|ST|Suite F|Pleasant Hill|Presque Isle County|MI|43604|United States|-5|condo| +27858|AAAAAAAACNMGAAAA|337|Spring Locust|Ct.|Suite 250|Buena Vista|Lenawee County|MI|45752|United States|-5|apartment| +27859|AAAAAAAADNMGAAAA|842|Hickory Second|Boulevard|Suite 350|Hillcrest|Hancock County|OH|43003|United States|-5|single family| +27860|AAAAAAAAENMGAAAA|628|Locust |Cir.|Suite S|Mount Olive|Lee County|GA|38059|United States|-5|apartment| +27861|AAAAAAAAFNMGAAAA|856|Forest 2nd|ST|Suite Q|Plainview|Buena Vista County|IA|53683|United States|-6|single family| +27862|AAAAAAAAGNMGAAAA|776|9th |Ct.|Suite 360|Freeport|Scott County|MO|61844|United States|-6|condo| +27863|AAAAAAAAHNMGAAAA|397|Main 2nd|RD|Suite 210|Unionville|Wood County|WI|51711|United States|-6|apartment| +27864|AAAAAAAAINMGAAAA|540|Laurel |Ct.|Suite W|Andover|Issaquena County|MS|51639|United States|-6|condo| +27865|AAAAAAAAJNMGAAAA|342|Oak 12th|Wy|Suite 90|Bunker Hill|Fulton County|PA|10150|United States|-5|apartment| +27866|AAAAAAAAKNMGAAAA|158|Woodland |RD|Suite 270|Mountain View|Fayette County|PA|14466|United States|-5|single family| +27867|AAAAAAAALNMGAAAA|767|2nd |Drive|Suite 190|Liberty|Phillips County|CO|83451|United States|-7|apartment| +27868|AAAAAAAAMNMGAAAA|75|Washington |RD|Suite L|Greenwood|Richland County|ND|58828|United States|-6|single family| +27869|AAAAAAAANNMGAAAA|970|Jefferson Jackson|Ct.|Suite P|Jamestown|Candler County|GA|36867|United States|-5|condo| +27870|AAAAAAAAONMGAAAA|343|Mill |Cir.|Suite 280|Marion|Aitkin County|MN|50399|United States|-6|single family| +27871|AAAAAAAAPNMGAAAA|637|East |Lane|Suite 420|Shiloh|Stafford County|KS|69275|United States|-6|condo| +27872|AAAAAAAAAOMGAAAA|459|Pine Fourth|Street|Suite T|Oak Hill|Brooks County|TX|77838|United States|-6|apartment| +27873|AAAAAAAABOMGAAAA|359|Cherry |Blvd|Suite 70|Valley View|Clinton County|IN|45124|United States|-5|single family| +27874|AAAAAAAACOMGAAAA|843|Jackson |ST|Suite F|Oakwood|Pawnee County|NE|60169|United States|-7|condo| +27875|AAAAAAAADOMGAAAA|173|Center |Cir.|Suite F|Kingston|Powder River County|MT|64975|United States|-7|apartment| +27876|AAAAAAAAEOMGAAAA|90|Meadow |Ave|Suite H|Franklin|Juneau Borough|AK|99101|United States|-9|apartment| +27877|AAAAAAAAFOMGAAAA|580|Sunset |Blvd|Suite M|Maple Grove|Prairie County|MT|68252|United States|-7|apartment| +27878|AAAAAAAAGOMGAAAA|149|View Sycamore|Parkway|Suite L|Pomona|Floyd County|TX|74153|United States|-6|condo| +27879|AAAAAAAAHOMGAAAA|429|College |Lane|Suite 130|Lake Forest|Pennington County|MN|56000|United States|-6|condo| +27880|AAAAAAAAIOMGAAAA|298|Williams Willow|Drive|Suite 60|Buena Vista|Polk County|TN|35752|United States|-6|apartment| +27881|AAAAAAAAJOMGAAAA|475|Highland |Lane|Suite 200|Unionville|Richmond County|NC|21711|United States|-5|apartment| +27882|AAAAAAAAKOMGAAAA||Pine |Boulevard|Suite 310|Lakeview||KS|68579|United States||condo| +27883|AAAAAAAALOMGAAAA|426|Hillcrest |Pkwy|Suite 480|Edgewood|Goshen County|WY|80069|United States|-7|condo| +27884|AAAAAAAAMOMGAAAA|119|Poplar Walnut|Dr.|Suite 340|Franklin|Accomack County|VA|29101|United States|-5|apartment| +27885|AAAAAAAANOMGAAAA|627|Mill 4th|Pkwy|Suite 120|Victoria|Cole County|MO|64088|United States|-6|condo| +27886|AAAAAAAAOOMGAAAA|54|Franklin 15th|RD|Suite J|Oakdale|Prairie County|MT|69584|United States|-7|apartment| +27887|AAAAAAAAPOMGAAAA|348|Laurel |Court|Suite 110|Lakewood|Jackson County|MS|58877|United States|-6|apartment| +27888|AAAAAAAAAPMGAAAA|78|View Smith|Circle|Suite G|Crossroads|Rush County|IN|40534|United States|-5|condo| +27889|AAAAAAAABPMGAAAA|752|Fifth |Cir.|Suite P|Hamilton|Linn County|IA|52808|United States|-6|single family| +27890|AAAAAAAACPMGAAAA|270|1st |RD|Suite 110|Lakeville|Wichita County|TX|78811|United States|-6|apartment| +27891|AAAAAAAADPMGAAAA|982|South Wilson|Cir.|Suite P|Stringtown|Montrose County|CO|80162|United States|-7|apartment| +27892|AAAAAAAAEPMGAAAA|435|1st |Lane|Suite U|Buena Vista|Ocean County|NJ|06352|United States|-5|single family| +27893|AAAAAAAAFPMGAAAA|143|4th |Circle|Suite 170|Sulphur Springs|Johnson County|TX|78354|United States|-6|single family| +27894|AAAAAAAAGPMGAAAA|668|8th |Blvd|Suite A|Lakeside|Hancock County|IA|59532|United States|-6|single family| +27895|AAAAAAAAHPMGAAAA|312|1st |RD|Suite 70|Roy|Daggett County|UT|80744|United States|-7|condo| +27896|AAAAAAAAIPMGAAAA|291|Valley |Ln|Suite 380|Ashland|Johnson County|GA|34244|United States|-5|apartment| +27897|AAAAAAAAJPMGAAAA|271|Walnut |Boulevard|Suite S|Oak Hill|Lake and Peninsula Borough|AK|97838|United States|-9|single family| +27898|AAAAAAAAKPMGAAAA|983|Central Oak|Cir.|Suite 70|Pleasant Grove|Garvin County|OK|74136|United States|-6|single family| +27899|AAAAAAAALPMGAAAA|857|Park Jefferson|Boulevard|Suite M|Union|Campbell County|WY|88721|United States|-7|apartment| +27900|AAAAAAAAMPMGAAAA|865|Wilson 8th|Court|Suite R|Greenfield|Edgar County|IL|65038|United States|-6|condo| +27901|AAAAAAAANPMGAAAA|615|First Main|Circle|Suite 30|Newtown|Clarke County|IA|51749|United States|-6|condo| +27902|AAAAAAAAOPMGAAAA|969|1st |Parkway|Suite 10|Maple Grove|Greene County|MO|68252|United States|-6|condo| +27903|AAAAAAAAPPMGAAAA|615|12th Smith|Pkwy|Suite Y|Carthage|Cumberland County|NC|21529|United States|-5|single family| +27904|AAAAAAAAAANGAAAA|947|Ridge Sunset|Pkwy|Suite 30|Stringtown|Kanawha County|WV|20162|United States|-5|condo| +27905|AAAAAAAABANGAAAA|864|Willow |Blvd|Suite O|Waterloo|Scott County|KS|61675|United States|-6|apartment| +27906|AAAAAAAACANGAAAA|843|Green 9th|Wy|Suite 290|Gladstone|Brooks County|TX|70894|United States|-6|condo| +27907|AAAAAAAADANGAAAA|99|Fifth Church|Road|Suite 0|Maple Grove|Houghton County|MI|48252|United States|-5|single family| +27908|AAAAAAAAEANGAAAA|772|6th |Ave|Suite I|Buena Vista|Thurston County|NE|65752|United States|-7|apartment| +27909|AAAAAAAAFANGAAAA|164|2nd Lincoln|Avenue|Suite 280|Antioch|Alexander County|NC|28605|United States|-5|single family| +27910|AAAAAAAAGANGAAAA|685|4th |Avenue|Suite K|Little River|Gloucester County|NJ|00919|United States|-5|apartment| +27911|AAAAAAAAHANGAAAA|400|Davis |Drive|Suite L|Greenfield|Carson City|NV|85038|United States|-8|condo| +27912|AAAAAAAAIANGAAAA||||Suite 400||Red Willow County|NE||United States|-7|apartment| +27913|AAAAAAAAJANGAAAA|303|Pine |ST|Suite I|Arlington|Hartley County|TX|76557|United States|-6|condo| +27914|AAAAAAAAKANGAAAA|844|Sycamore Second|Lane|Suite V|Brownsville|Clackamas County|OR|99310|United States|-8|condo| +27915|AAAAAAAALANGAAAA|165|15th |Lane|Suite 180|Fairview|Marion County|KY|45709|United States|-5|single family| +27916|AAAAAAAAMANGAAAA|27|Elm |Wy|Suite Y|Fairview|Roosevelt County|MT|65709|United States|-7|apartment| +27917|AAAAAAAANANGAAAA|126||RD|||Will County||||-6|| +27918|AAAAAAAAOANGAAAA|552|Jefferson Fourth|Road|Suite 130|Woodlawn|Northampton County|VA|24098|United States|-5|condo| +27919|AAAAAAAAPANGAAAA|578|Lincoln |Lane|Suite 20|Shelby|Pueblo County|CO|86575|United States|-7|apartment| +27920|AAAAAAAAABNGAAAA|992|3rd |Ct.|Suite Q|Ashland|Hanover County|VA|24244|United States|-5|condo| +27921|AAAAAAAABBNGAAAA|618|Valley |ST|Suite 120|Plainville|Nuckolls County|NE|66115|United States|-7|apartment| +27922|AAAAAAAACBNGAAAA|450|Lincoln Jefferson|ST|Suite N|Woodlawn|Fulton County|PA|14098|United States|-5|condo| +27923|AAAAAAAADBNGAAAA|570|Fifth |Way|Suite 300|Woodland|Livingston County|NY|14854|United States|-5|single family| +27924|AAAAAAAAEBNGAAAA|259|Lincoln |Way|Suite 110|Mount Pleasant|Sullivan County|PA|11933|United States|-5|single family| +27925|AAAAAAAAFBNGAAAA|939|Poplar Park|Ct.|Suite V|Union|Marquette County|WI|58721|United States|-6|condo| +27926|AAAAAAAAGBNGAAAA|278|Lake |Wy|Suite I|Concord|Hoke County|NC|24107|United States|-5|single family| +27927|AAAAAAAAHBNGAAAA|334|Cherry 12th|Court|Suite 430|Spring Hill|Jefferson County|GA|36787|United States|-5|single family| +27928|AAAAAAAAIBNGAAAA|426|Washington |Ave|Suite 260|Midway|Humboldt County|CA|91904|United States|-8|apartment| +27929|AAAAAAAAJBNGAAAA|660|Valley |Street|Suite 50|Harvey|Mineral County|CO|85858|United States|-7|apartment| +27930|AAAAAAAAKBNGAAAA|886|Smith |Blvd|Suite 290|Union|York County|VA|28721|United States|-5|apartment| +27931|AAAAAAAALBNGAAAA|438|Johnson First|Avenue|Suite 170|Waterloo|Marion County|KY|41675|United States|-5|single family| +27932|AAAAAAAAMBNGAAAA|691|Lakeview 8th|Parkway|Suite O|Fisher|Titus County|TX|72819|United States|-6|condo| +27933|AAAAAAAANBNGAAAA|523|Pine View|Road|Suite 320|Summit|Milwaukee County|WI|50499|United States|-6|apartment| +27934|AAAAAAAAOBNGAAAA|933|Ridge Seventh|Blvd|Suite W|Oakdale|Davidson County|TN|39584|United States|-5|single family| +27935|AAAAAAAAPBNGAAAA|803|Locust |Parkway|Suite H|Jackson|Union County|MS|59583|United States|-6|single family| +27936|AAAAAAAAACNGAAAA|535|12th |ST|Suite 70|Georgetown|Citrus County|FL|37057|United States|-5|single family| +27937|AAAAAAAABCNGAAAA|160|10th |Way|Suite I|Greenfield|Union County|TN|35038|United States|-6|single family| +27938|AAAAAAAACCNGAAAA|253|Williams |Ct.|Suite 380|Lakeview|Calcasieu Parish|LA|78579|United States|-6|apartment| +27939|AAAAAAAADCNGAAAA|973|Elm |Ct.|Suite T|Jamestown|Jerauld County|SD|56867|United States|-7|condo| +27940|AAAAAAAAECNGAAAA|302|Lake |Blvd|Suite T|Rosebud|Glenn County|CA|92244|United States|-8|apartment| +27941|AAAAAAAAFCNGAAAA|324|Hickory Smith|Wy|Suite F|Harvey|Schoharie County|NY|15858|United States|-5|apartment| +27942|AAAAAAAAGCNGAAAA|215|View Cedar|Circle|Suite 0|Pine Grove|McKinley County|NM|84593|United States|-7|condo| +27943|AAAAAAAAHCNGAAAA|102|4th 12th|RD|Suite W|Greenwood|Catahoula Parish|LA|78828|United States|-6|single family| +27944|AAAAAAAAICNGAAAA|278|14th 6th|Pkwy|Suite 250|Brentwood|Oconto County|WI|54188|United States|-6|apartment| +27945|AAAAAAAAJCNGAAAA|514|Cedar Main|Court|Suite 80|Bayview|Dallas County|TX|79672|United States|-6|condo| +27946|AAAAAAAAKCNGAAAA|231|6th Washington|ST|Suite 110|Yorktown|DuPage County|IL|60732|United States|-6|apartment| +27947|AAAAAAAALCNGAAAA|921|9th Ridge|Parkway|Suite C|Woodland|Doddridge County|WV|24854|United States|-5|single family| +27948|AAAAAAAAMCNGAAAA|108|Pine 7th|Street|Suite N|Clinton|Allegany County|NY|18222|United States|-5|single family| +27949|AAAAAAAANCNGAAAA||Oak |Wy|Suite E|||CO|80587|United States|-7|single family| +27950|AAAAAAAAOCNGAAAA|527|12th Smith|Parkway|Suite 100|Bethel|Fremont County|CO|85281|United States|-7|apartment| +27951|AAAAAAAAPCNGAAAA|851|First |Street|Suite V|Mount Pleasant|Monroe County|IL|61933|United States|-6|apartment| +27952|AAAAAAAAADNGAAAA|369|Sycamore Main|Dr.|Suite 280|Edgewood|Transylvania County|NC|20069|United States|-5|apartment| +27953|AAAAAAAABDNGAAAA|869|Lakeview |Cir.|Suite D|Farmington|Fallon County|MT|69145|United States|-7|apartment| +27954|AAAAAAAACDNGAAAA|173|Walnut |Boulevard|Suite P|Bunker Hill|Albemarle County|VA|20150|United States|-5|condo| +27955|AAAAAAAADDNGAAAA|994|Lake |Avenue|Suite 210|Greenfield|Aleutians East Borough|AK|95038|United States|-9|single family| +27956|AAAAAAAAEDNGAAAA|596|Spring Franklin|Parkway|Suite 260|Marion|Whitley County|KY|40399|United States|-5|condo| +27957|AAAAAAAAFDNGAAAA|328|Broadway |Drive|Suite D|Lebanon|Hempstead County|AR|72898|United States|-6|single family| +27958|AAAAAAAAGDNGAAAA|909|Lakeview |Ct.|Suite 100|Mount Olive|Siskiyou County|CA|98059|United States|-8|condo| +27959|AAAAAAAAHDNGAAAA|190|Adams 15th|Road|||Passaic County|NJ||United States||| +27960|AAAAAAAAIDNGAAAA|264|Fifth 9th|Lane|Suite 230|Pleasant Hill|Tuscaloosa County|AL|33604|United States|-6|apartment| +27961|AAAAAAAAJDNGAAAA|170|Ridge Johnson|Avenue|Suite A|Five Forks|Potter County|SD|52293|United States|-7|condo| +27962|AAAAAAAAKDNGAAAA|907|Third Sycamore|Boulevard|Suite K|Summit|Garfield County|UT|80499|United States|-7|condo| +27963|AAAAAAAALDNGAAAA|440|Smith Sycamore|Ave|Suite M|Franklin|Coles County|IL|69101|United States|-6|single family| +27964|AAAAAAAAMDNGAAAA|892|Jefferson |Drive|Suite J|Wildwood|La Paz County|AZ|86871|United States|-7|apartment| +27965|AAAAAAAANDNGAAAA|414|Miller Broadway|Ct.|Suite 250|Harmony|Franklin County|WA|95804|United States|-8|apartment| +27966|AAAAAAAAODNGAAAA|223|Ash 3rd|Parkway|Suite L|Lakeview|Morgan County|GA|38579|United States|-5|single family| +27967|AAAAAAAAPDNGAAAA|596|Davis 10th|Blvd|Suite 340|Mount Zion|Carroll County|IN|48054|United States|-5|single family| +27968|AAAAAAAAAENGAAAA|88|Lincoln |Ct.|Suite 260|Greenfield|Upson County|GA|35038|United States|-5|single family| +27969|AAAAAAAABENGAAAA|597|4th 12th|Blvd|Suite D|Union Hill|Kanabec County|MN|57746|United States|-6|single family| +27970|AAAAAAAACENGAAAA|24|Dogwood |Street|Suite C|Oakland|Knox County|MO|69843|United States|-6|condo| +27971|AAAAAAAADENGAAAA|58|Lake 11th|Boulevard|Suite 190|Harmony|Montezuma County|CO|85804|United States|-7|condo| +27972|AAAAAAAAEENGAAAA|145|Washington Maple|Drive|Suite Q|Glenwood|Burleigh County|ND|53511|United States|-6|single family| +27973|AAAAAAAAFENGAAAA|891|Second Adams|Boulevard|Suite R|Woodville|Madison County|GA|34289|United States|-5|apartment| +27974|AAAAAAAAGENGAAAA|664|Lake Willow|Avenue|Suite 90|Royal|Fall River County|SD|55819|United States|-6|condo| +27975|AAAAAAAAHENGAAAA|432|Railroad |ST|Suite M|Riverside|Candler County|GA|39231|United States|-5|condo| +27976|AAAAAAAAIENGAAAA|66|Third Maple|Drive|Suite 310|Franklin|Lee County|AL|39101|United States|-6|apartment| +27977|AAAAAAAAJENGAAAA|249|Park |Ct.|Suite W|Belmont|Allen Parish|LA|70191|United States|-6|condo| +27978|AAAAAAAAKENGAAAA||13th Elm|||Plainview|Wise County|TX||United States||condo| +27979|AAAAAAAALENGAAAA|||Road||Hamilton|Covington County|||United States||| +27980|AAAAAAAAMENGAAAA|965|Pine 7th|Parkway|Suite N|Oak Grove|Carroll County|IA|58370|United States|-6|condo| +27981|AAAAAAAANENGAAAA|579|East Davis|Parkway|Suite B|Youngstown|Martin County|TX|70001|United States|-6|single family| +27982|AAAAAAAAOENGAAAA|611|11th Cedar|Street|Suite 370|Summerfield|Utah County|UT|80634|United States|-7|condo| +27983|AAAAAAAAPENGAAAA|734|Pine |Parkway|Suite 140|Brownsville|Humphreys County|MS|59310|United States|-6|apartment| +27984|AAAAAAAAAFNGAAAA|886|9th Locust|Road|Suite 60|Pleasant Grove|Ramsey County|MN|54136|United States|-6|single family| +27985|AAAAAAAABFNGAAAA|851|Seventh Sixth|Pkwy|Suite 180|Maple Grove|Livingston County|KY|48252|United States|-5|single family| +27986|AAAAAAAACFNGAAAA|594|Lee |Pkwy|Suite 200|Pleasant Hill|Benton County|MN|53604|United States|-6|apartment| +27987|AAAAAAAADFNGAAAA|602|Sycamore |Lane|Suite H|Plainville|Bee County|TX|76115|United States|-6|apartment| +27988|AAAAAAAAEFNGAAAA|742|10th River|Road|Suite Y|Farmington|Lafayette Parish|LA|79145|United States|-6|single family| +27989|AAAAAAAAFFNGAAAA|30|10th Adams|Road|Suite 320|Harmony|San Juan County|WA|95804|United States|-8|apartment| +27990|AAAAAAAAGFNGAAAA|172|2nd Main|Wy|Suite 70|Mountain View|Canyon County|ID|84466|United States|-7|single family| +27991|AAAAAAAAHFNGAAAA|457|3rd Cherry|Road|Suite 230|Bethel|Hopewell city|VA|25281|United States|-5|single family| +27992|AAAAAAAAIFNGAAAA|690|Spring |Circle|Suite 450|Mount Vernon|Chase County|KS|68482|United States|-6|condo| +27993|AAAAAAAAJFNGAAAA|430|Locust Pine|Ct.|Suite U|Little River|Jefferson Parish|LA|70319|United States|-6|condo| +27994|AAAAAAAAKFNGAAAA|369|Cherry |Road|Suite 60|Woodland|Iberville Parish|LA|74854|United States|-6|apartment| +27995|AAAAAAAALFNGAAAA|177|View Walnut|Court|Suite 250|Clifton|Mineral County|WV|28014|United States|-5|apartment| +27996|AAAAAAAAMFNGAAAA||||||Lassen County|CA|98883|||condo| +27997|AAAAAAAANFNGAAAA|651|Spring 4th|Way|Suite 430|Montague|Yamhill County|OR|94062|United States|-8|apartment| +27998|AAAAAAAAOFNGAAAA|64|Park |Avenue|Suite 300|Marion|Aroostook County|ME|00999|United States|-5|condo| +27999|AAAAAAAAPFNGAAAA|516|Sunset |Cir.|Suite 110|Unionville|Pembina County|ND|51711|United States|-6|single family| +28000|AAAAAAAAAGNGAAAA|813|Johnson |Way|Suite M|Summit|Greeley County|KS|60499|United States|-6|single family| +28001|AAAAAAAABGNGAAAA|80|Valley Ridge|Street|Suite 180|Buena Vista|Kay County|OK|75752|United States|-6|apartment| +28002|AAAAAAAACGNGAAAA|965|Lincoln Mill|Court|Suite J|Macon|Coos County|OR|90369|United States|-8|single family| +28003|AAAAAAAADGNGAAAA|173|Ash Ash|Blvd|Suite 380|Springdale|Ripley County|MO|68883|United States|-6|single family| +28004|AAAAAAAAEGNGAAAA|802|Ash |Boulevard|Suite L|Lawrenceville|Oneida County|WI|54462|United States|-6|condo| +28005|AAAAAAAAFGNGAAAA|809|Park Mill|Road|Suite 130|Jamestown|Cook County|GA|36867|United States|-5|apartment| +28006|AAAAAAAAGGNGAAAA|113|Second |Blvd|Suite 450|Lakeview|Garvin County|OK|78579|United States|-6|apartment| +28007|AAAAAAAAHGNGAAAA|154|Valley |ST|Suite K|Midway|Mississippi County|AR|71904|United States|-6|single family| +28008|AAAAAAAAIGNGAAAA|270|Fourth |Lane|Suite 330|Lincoln|Calhoun County|AL|31289|United States|-6|apartment| +28009|AAAAAAAAJGNGAAAA|958|1st Pine|ST|Suite 80|Red Bank|Richland County|WI|54975|United States|-6|single family| +28010|AAAAAAAAKGNGAAAA|697|Park |ST|Suite P|Enterprise|Ferry County|WA|91757|United States|-8|condo| +28011|AAAAAAAALGNGAAAA|478|South 7th|Boulevard|Suite U|Clifton|Stevens County|MN|58014|United States|-6|condo| +28012|AAAAAAAAMGNGAAAA|340|Woodland |ST|Suite M|Hamilton|Ogemaw County|MI|42808|United States|-5|apartment| +28013|AAAAAAAANGNGAAAA|7|Center Center|Lane|Suite 440|Leesville|Newport County|RI|06023|United States|-5|condo| +28014|AAAAAAAAOGNGAAAA|953|8th 9th|Ct.|Suite B|Oak Hill|Burke County|ND|57838|United States|-6|condo| +28015|AAAAAAAAPGNGAAAA|46|Elm 13th|Dr.|Suite H|Walnut Grove|Watauga County|NC|27752|United States|-5|single family| +28016|AAAAAAAAAHNGAAAA|42|Thirteenth Second|Cir.|Suite 470|Highland|Platte County|NE|69454|United States|-7|apartment| +28017|AAAAAAAABHNGAAAA||Cherry Hill||Suite 50||Willacy County||76867|||condo| +28018|AAAAAAAACHNGAAAA|986|Cherry Oak|ST|Suite 90|Ashland|Garden County|NE|64244|United States|-6|single family| +28019|AAAAAAAADHNGAAAA|876|2nd |Road|Suite 430|Oak Hill|Washington County|OK|77838|United States|-6|single family| +28020|AAAAAAAAEHNGAAAA|203|Lincoln Oak|Ln|Suite I|Union Hill|Perry County|AR|77746|United States|-6|condo| +28021|AAAAAAAAFHNGAAAA||||Suite G|Edgewood|Garfield County||60069||-6|single family| +28022|AAAAAAAAGHNGAAAA|895|7th |Dr.|Suite 340|Wildwood|Union County|TN|36871|United States|-6|single family| +28023|AAAAAAAAHHNGAAAA|359|13th Hill|Court|Suite 150|Plainville|Dawson County|NE|66115|United States|-6|apartment| +28024|AAAAAAAAIHNGAAAA|974|Johnson |Boulevard|Suite 90|Saint James|Greene County|VA|25799|United States|-5|apartment| +28025|AAAAAAAAJHNGAAAA|438|North |Court|Suite N|Midway|Carroll County|IA|51904|United States|-6|apartment| +28026|AAAAAAAAKHNGAAAA|773|11th 13th|Ave|Suite 180|Mount Pleasant|Hyde County|NC|21933|United States|-5|single family| +28027|AAAAAAAALHNGAAAA|591|Laurel |Cir.|Suite 140|Wildwood|Iron County|WI|56871|United States|-6|single family| +28028|AAAAAAAAMHNGAAAA|201|West 13th|Drive|Suite J|Pleasant Hill|Marshall County|IN|43604|United States|-5|apartment| +28029|AAAAAAAANHNGAAAA|790|Chestnut Highland|Avenue|Suite W|Spring Valley|Jones County|NC|26060|United States|-5|apartment| +28030|AAAAAAAAOHNGAAAA|588|Park |Circle|Suite 160|Pleasant Grove|Umatilla County|OR|94136|United States|-8|apartment| +28031|AAAAAAAAPHNGAAAA|158|Highland |Parkway|Suite 340|Oak Grove|Calhoun County|WV|28370|United States|-5|single family| +28032|AAAAAAAAAINGAAAA|240|Ridge Lee|Dr.|Suite F|Edgewood|Angelina County|TX|70069|United States|-6|apartment| +28033|AAAAAAAABINGAAAA|880|Elm Third|Ln|Suite 190|Shiloh|Ashland County|WI|59275|United States|-6|condo| +28034|AAAAAAAACINGAAAA|721|Wilson |Court|Suite C|Mount Vernon|Sevier County|TN|38482|United States|-6|apartment| +28035|AAAAAAAADINGAAAA|656|Second |Dr.|Suite 450|Pleasant Grove|Dane County|WI|54136|United States|-6|apartment| +28036|AAAAAAAAEINGAAAA|739|Ash Laurel|Boulevard|Suite N|Marion|Shelby County|TX|70399|United States|-6|single family| +28037|AAAAAAAAFINGAAAA|786|Smith |Cir.|Suite 190|Amity|Carbon County|WY|80766|United States|-7|apartment| +28038|AAAAAAAAGINGAAAA|921|Church |RD|Suite D|Shiloh|Milwaukee County|WI|59275|United States|-6|apartment| +28039|AAAAAAAAHINGAAAA|600|Seventh |Boulevard|Suite W|Bethel|Avery County|NC|25281|United States|-5|condo| +28040|AAAAAAAAIINGAAAA|904|Main |Dr.|Suite Y|Lucas|Johnson County|MO|64554|United States|-6|condo| +28041|AAAAAAAAJINGAAAA|359|First Elm|Street|Suite 430|Wolf Creek|Lincoln County|NM|82455|United States|-7|apartment| +28042|AAAAAAAAKINGAAAA|729|River 4th|Ln|Suite 450|Marion|Lake County|FL|30399|United States|-5|condo| +28043|AAAAAAAALINGAAAA|232|11th |Avenue|Suite 290|Oak Hill|Washington County|PA|17838|United States|-5|condo| +28044|AAAAAAAAMINGAAAA|362|8th |Ct.|Suite I|Bunker Hill|Trigg County|KY|40150|United States|-5|condo| +28045|AAAAAAAANINGAAAA|759|Main |Ave|Suite H|Lincoln|Latimer County|OK|71289|United States|-6|apartment| +28046|AAAAAAAAOINGAAAA|722|Third Elm|Dr.|Suite O|Riverview|Washington County|GA|39003|United States|-5|apartment| +28047|AAAAAAAAPINGAAAA|413|Mill |Street|Suite 130|Concord|Lee County|SC|24107|United States|-5|single family| +28048|AAAAAAAAAJNGAAAA|340|Railroad |Avenue|Suite 210|Payne|Bolivar County|MS|56134|United States|-6|condo| +28049|AAAAAAAABJNGAAAA|586|Ridge Miller|Blvd|Suite O|Newtown|Seminole County|OK|71749|United States|-6|single family| +28050|AAAAAAAACJNGAAAA|80|Williams |RD|Suite 110|Red Hill|Massac County|IL|64338|United States|-6|condo| +28051|AAAAAAAADJNGAAAA|656|3rd |Cir.|Suite 340|Gray|Jackson County|MO|60468|United States|-6|condo| +28052|AAAAAAAAEJNGAAAA|20|Laurel |Avenue|Suite 360|Bridgeport|Brazos County|TX|75817|United States|-6|single family| +28053|AAAAAAAAFJNGAAAA|732|Main |Court|Suite 200|Harmony|Grady County|GA|35804|United States|-5|single family| +28054|AAAAAAAAGJNGAAAA|280|Williams Elm|Way|Suite T|Red Hill|Franklin County|AR|74338|United States|-6|single family| +28055|AAAAAAAAHJNGAAAA|144|11th |RD|Suite 470|Oak Hill|Wahkiakum County|WA|97838|United States|-8|single family| +28056|AAAAAAAAIJNGAAAA|6|Oak |Ct.|Suite E|Macedonia|Miner County|SD|51087|United States|-7|apartment| +28057|AAAAAAAAJJNGAAAA|354|Smith Elm|Dr.|Suite 300|Concord|Winneshiek County|IA|54107|United States|-6|single family| +28058|AAAAAAAAKJNGAAAA|817|Wilson Lake|Avenue|Suite P|Summit|Yuba County|CA|90499|United States|-8|condo| +28059|AAAAAAAALJNGAAAA|317|View Hill|Street|Suite C|Midway|Dimmit County|TX|71904|United States|-6|condo| +28060|AAAAAAAAMJNGAAAA|314|8th Central|Road|Suite S|Red Hill|Moore County|TN|34338|United States|-6|condo| +28061|AAAAAAAANJNGAAAA|295|Main Meadow|Boulevard|Suite I|Lakewood|Harrison County|MS|58877|United States|-6|apartment| +28062|AAAAAAAAOJNGAAAA|13|12th Jackson|Drive|Suite 90|Marion|Bureau County|IL|60399|United States|-6|apartment| +28063|AAAAAAAAPJNGAAAA|435|2nd Jackson|Drive|Suite 30|Edgewood|Madison County|MS|50069|United States|-6|apartment| +28064|AAAAAAAAAKNGAAAA|776|Valley Spring|Lane|Suite 70|Glenville|Schenectady County|NY|13445|United States|-5|single family| +28065|AAAAAAAABKNGAAAA|476|Main |Avenue|Suite 140|Green Acres|Morris County|NJ|08283|United States|-5|single family| +28066|AAAAAAAACKNGAAAA|499|3rd |Circle|Suite 350|Springdale|Albemarle County|VA|28883|United States|-5|apartment| +28067|AAAAAAAADKNGAAAA|887|7th |Blvd|Suite Y|Shady Grove|Campbell County|KY|42812|United States|-6|apartment| +28068|AAAAAAAAEKNGAAAA|256|Second |Lane|Suite 450|Bay View|Woodbury County|IA|56457|United States|-6|apartment| +28069|AAAAAAAAFKNGAAAA|99|East Birch|Pkwy|Suite P|Colonial Heights|Buffalo County|WI|53425|United States|-6|apartment| +28070|AAAAAAAAGKNGAAAA|375|Cedar Chestnut|Street|Suite 470|Maple Grove|Karnes County|TX|78252|United States|-6|condo| +28071|AAAAAAAAHKNGAAAA|536|Forest |Ave|Suite P|Georgetown|McDonough County|IL|67057|United States|-6|condo| +28072|AAAAAAAAIKNGAAAA|931|Maple Main|Dr.|Suite 20|Lincoln|Winona County|MN|51289|United States|-6|apartment| +28073|AAAAAAAAJKNGAAAA|341|Main Maple|Pkwy|Suite B|Glenwood|Preston County|WV|23511|United States|-5|condo| +28074|AAAAAAAAKKNGAAAA|731|7th Hill|Pkwy|Suite 290|Glenwood|||63511|United States|-6|| +28075|AAAAAAAALKNGAAAA|447|Mill |Circle|Suite B|Arcola|Macoupin County|IL|61654|United States|-6|condo| +28076|AAAAAAAAMKNGAAAA||Central ||||Martin County|NC|21749||-5|| +28077|AAAAAAAANKNGAAAA|719|13th 13th|Lane|Suite 170|Hamilton|Maui County|HI|92808|United States|-10|condo| +28078|AAAAAAAAOKNGAAAA|262|2nd |Ct.|Suite P|Oakland|Powder River County|MT|69843|United States|-7|single family| +28079|AAAAAAAAPKNGAAAA|240|Miller |Ave|Suite 390|Deerfield|Gilchrist County|FL|39840|United States|-5|condo| +28080|AAAAAAAAALNGAAAA|986|12th |Ct.|Suite X|Forest Hills|Mercer County|PA|19237|United States|-5|condo| +28081|AAAAAAAABLNGAAAA||Spruce |RD||Green Acres|Hanson County|||United States||| +28082|AAAAAAAACLNGAAAA|389|Lincoln 11th|Pkwy|Suite 200|Lakewood|Neosho County|KS|68877|United States|-6|single family| +28083|AAAAAAAADLNGAAAA|677|Oak |Boulevard|Suite 380|Oak Hill|Franklin County|FL|37838|United States|-5|single family| +28084|AAAAAAAAELNGAAAA|751|Forest |Circle|Suite E|Wildwood|Salem County|NJ|07471|United States|-5|single family| +28085|AAAAAAAAFLNGAAAA|789|7th 2nd|Pkwy|Suite N|Lake Forest|Gosper County|NE|66000|United States|-6|single family| +28086|AAAAAAAAGLNGAAAA|133|Park |Cir.|Suite W|Florence|Montgomery County|IN|43394|United States|-5|apartment| +28087|AAAAAAAAHLNGAAAA|||Blvd||Brownsville|Hampton city|VA||||| +28088|AAAAAAAAILNGAAAA|640|Oak |Ave|Suite N|Providence|Dade County|GA|36614|United States|-5|apartment| +28089|AAAAAAAAJLNGAAAA|398|Wilson Valley|RD|Suite 360|Shore Acres|Amelia County|VA|22724|United States|-5|single family| +28090|AAAAAAAAKLNGAAAA||Central First|Wy|||Jones County||58252|United States|-6|single family| +28091|AAAAAAAALLNGAAAA|686|Chestnut Main|Dr.|Suite J|Buena Vista|Gulf County|FL|35752|United States|-5|single family| +28092|AAAAAAAAMLNGAAAA|631|Sixth |Street|Suite 370|Newport|Coweta County|GA|31521|United States|-5|single family| +28093|AAAAAAAANLNGAAAA|750|Hickory |Drive|Suite 280|Clinton|Wyandotte County|KS|68222|United States|-6|single family| +28094|AAAAAAAAOLNGAAAA||Central Franklin||Suite 380||Grundy County|||||single family| +28095|AAAAAAAAPLNGAAAA|742|Miller |Wy|Suite 370|Mountain View|Coahoma County|MS|54466|United States|-6|condo| +28096|AAAAAAAAAMNGAAAA|340|10th |Ave|Suite D|Kingston|Freestone County|TX|74975|United States|-6|condo| +28097|AAAAAAAABMNGAAAA|792|Oak |Road|Suite L|Stringtown|Dubuque County|IA|50162|United States|-6|condo| +28098|AAAAAAAACMNGAAAA|218|Seventh |Avenue|Suite C|Maple Grove|Wayne County|NC|28252|United States|-5|apartment| +28099|AAAAAAAADMNGAAAA|984|Spring River|Pkwy|Suite L|Paxton|Dickson County|TN|35669|United States|-5|single family| +28100|AAAAAAAAEMNGAAAA|318|Hill |Parkway|Suite 40|Wildwood|Wilcox County|AL|36871|United States|-6|single family| +28101|AAAAAAAAFMNGAAAA|328|Pine |Ln|Suite D|Springdale|Deuel County|SD|58883|United States|-6|condo| +28102|AAAAAAAAGMNGAAAA|370|15th |Ct.|Suite P|Stewart|Taylor County|GA|38041|United States|-5|apartment| +28103|AAAAAAAAHMNGAAAA|90|Woodland 15th|Lane|Suite O|Hillcrest|Dukes County|MA|03603|United States|-5|condo| +28104|AAAAAAAAIMNGAAAA|391|Third 2nd|Ct.|Suite 400|Arlington|Newport News city|VA|26557|United States|-5|condo| +28105|AAAAAAAAJMNGAAAA|48|North Pine|Street|Suite Q|Oak Ridge|Butler County|AL|38371|United States|-6|single family| +28106|AAAAAAAAKMNGAAAA|375|3rd |Court|Suite K|Antioch|Langlade County|WI|58605|United States|-6|condo| +28107|AAAAAAAALMNGAAAA|392|2nd Dogwood|Court|Suite R|Bridgeport|Stearns County|MN|55817|United States|-6|apartment| +28108|AAAAAAAAMMNGAAAA|904|College |Pkwy|Suite X|Macedonia|La Crosse County|WI|51087|United States|-6|condo| +28109|AAAAAAAANMNGAAAA|180|Forest 5th|Pkwy|Suite 410|Riverview|Pocahontas County|WV|29003|United States|-5|apartment| +28110|AAAAAAAAOMNGAAAA|850|Adams |Ct.|Suite T|Springfield|Montgomery County|AL|39303|United States|-6|single family| +28111|AAAAAAAAPMNGAAAA|836|Twelfth River|Lane|Suite C|Lakewood|Jasper County|IL|68877|United States|-6|single family| +28112|AAAAAAAAANNGAAAA|211|Mill Green|Parkway|Suite M|Greenwood|Monroe County|WV|28828|United States|-5|single family| +28113|AAAAAAAABNNGAAAA|691|Chestnut |Road|Suite 200|Arlington|Dukes County|MA|07157|United States|-5|single family| +28114|AAAAAAAACNNGAAAA|658|2nd |Pkwy|Suite F|Farmington|Breathitt County|KY|49145|United States|-6|single family| +28115|AAAAAAAADNNGAAAA|971|Central |ST|Suite 300|Midway|Brown County|OH|41904|United States|-5|apartment| +28116|AAAAAAAAENNGAAAA|774|Woodland Walnut|Pkwy|Suite A|Enterprise|Perry County|IN|41757|United States|-5|single family| +28117|AAAAAAAAFNNGAAAA|86|First |ST|Suite 340|Oakland|Columbus County|NC|29843|United States|-5|condo| +28118|AAAAAAAAGNNGAAAA|171|Maple Walnut|Cir.|Suite 160|Buena Vista|Gordon County|GA|35752|United States|-5|condo| +28119|AAAAAAAAHNNGAAAA|460|Poplar 8th|Parkway|Suite D|Jamestown|Hancock County|IA|56867|United States|-6|condo| +28120|AAAAAAAAINNGAAAA|294|2nd |Ln|Suite D|Buena Vista|Early County|GA|35752|United States|-5|single family| +28121|AAAAAAAAJNNGAAAA|3|12th |Dr.|Suite U|Stringtown|Geary County|KS|60162|United States|-6|single family| +28122|AAAAAAAAKNNGAAAA|137|Park |Blvd|Suite W|Liberty|Granville County|NC|23451|United States|-5|condo| +28123|AAAAAAAALNNGAAAA|272|Spring Poplar|Way|Suite 450|Waterloo|Schuyler County|MO|61675|United States|-6|single family| +28124|AAAAAAAAMNNGAAAA|858|Franklin Church|Parkway|Suite 300|Spring Valley|Van Buren County|MI|46060|United States|-5|condo| +28125|AAAAAAAANNNGAAAA|533|Church Green|Avenue|Suite F|Shiloh|Herkimer County|NY|19275|United States|-5|single family| +28126|AAAAAAAAONNGAAAA|414|Fifth 2nd|Ave|Suite W|Hopewell|Rock County|NE|60587|United States|-7|single family| +28127|AAAAAAAAPNNGAAAA|503|Lincoln 14th|Lane|Suite W|Friendship|McPherson County|SD|54536|United States|-7|condo| +28128|AAAAAAAAAONGAAAA|404|2nd |Ct.|Suite I|Clinton|Ogemaw County|MI|48222|United States|-5|single family| +28129|AAAAAAAABONGAAAA|35|Ridge |Court|Suite 280|Mount Olive|Westmoreland County|PA|18059|United States|-5|condo| +28130|AAAAAAAACONGAAAA|488||Wy||Marion|Dubois County|||United States||condo| +28131|AAAAAAAADONGAAAA|373|Second 13th|Ct.|Suite 270|Wildwood|Tazewell County|IL|66871|United States|-6|single family| +28132|AAAAAAAAEONGAAAA|101|Willow Chestnut|Dr.|Suite L|Kingston|Randolph County|IL|64975|United States|-6|apartment| +28133|AAAAAAAAFONGAAAA|554|Adams |Ct.|Suite 280|Colonial Heights|Stephenson County|IL|63425|United States|-6|single family| +28134|AAAAAAAAGONGAAAA|595|Lee Meadow|Pkwy|Suite N|Taft|Webster County|MS|50589|United States|-6|condo| +28135|AAAAAAAAHONGAAAA|36|Fourth 5th|Ave|Suite 200|Pleasant Hill|Bradley County|AR|73604|United States|-6|condo| +28136|AAAAAAAAIONGAAAA|556|14th Church|Dr.|Suite 160|Shady Grove|Valencia County|NM|82812|United States|-7|condo| +28137|AAAAAAAAJONGAAAA|477|Highland |Wy|Suite W|Belmont|Sherman County|OR|90191|United States|-8|apartment| +28138|AAAAAAAAKONGAAAA|156|Williams Hillcrest|Wy|Suite W|Ashland|Polk County|WI|54244|United States|-6|apartment| +28139|AAAAAAAALONGAAAA|920|Pine |Avenue|Suite U|Bridgeport|Calloway County|KY|45817|United States|-6|condo| +28140|AAAAAAAAMONGAAAA|865|East |Street|Suite E|Hopewell|Iron County|WI|50587|United States|-6|single family| +28141|AAAAAAAANONGAAAA|215|1st |Drive|Suite Q|Newtown|Kane County|UT|81749|United States|-7|condo| +28142|AAAAAAAAOONGAAAA|197|Ash |Boulevard|Suite C|Hillcrest|Cook County|GA|33003|United States|-5|apartment| +28143|AAAAAAAAPONGAAAA|296|Park |Street|Suite 460|Lincoln|Menard County|IL|61289|United States|-6|apartment| +28144|AAAAAAAAAPNGAAAA|572|4th Valley|Street|Suite T|Mount Zion|Williamson County|TN|38054|United States|-5|single family| +28145|AAAAAAAABPNGAAAA|895|First |ST|Suite 370|Maple Hill|Sullivan County|PA|18095|United States|-5|single family| +28146|AAAAAAAACPNGAAAA|714|Maple Cedar|Ave|Suite 270|Carthage|Haskell County|OK|71529|United States|-6|single family| +28147|AAAAAAAADPNGAAAA|692|Pine Pine|Pkwy|Suite W|Pleasant Grove|Valdez-Cordova Census Area|AK|94136|United States|-9|single family| +28148|AAAAAAAAEPNGAAAA|979|||||Walthall County||58909|United States|-6|condo| +28149|AAAAAAAAFPNGAAAA|181|Central |Circle|Suite 290|Green Acres|Allegheny County|PA|17683|United States|-5|condo| +28150|AAAAAAAAGPNGAAAA|583|Main |Court|Suite 190|Stringtown|Decatur County|TN|30162|United States|-5|apartment| +28151|AAAAAAAAHPNGAAAA|845|Davis Fifth|Blvd|Suite 100|Liberty|Portage County|WI|53451|United States|-6|apartment| +28152|AAAAAAAAIPNGAAAA|791|14th Pine|RD|Suite 20|Murray|Cass County|MI|42150|United States|-5|apartment| +28153|AAAAAAAAJPNGAAAA|137|East 10th|Road|Suite 360|Belmont|Belmont County|OH|40191|United States|-5|apartment| +28154|AAAAAAAAKPNGAAAA|127|Third Third|Court|Suite 400|Oakwood|Cleveland County|NC|20169|United States|-5|condo| +28155|AAAAAAAALPNGAAAA|666|1st |Court|Suite 380|Hopewell|Nicholas County|KY|40587|United States|-5|condo| +28156|AAAAAAAAMPNGAAAA|362|West |Court|Suite U|Quincy|Pierce County|GA|33868|United States|-5|condo| +28157|AAAAAAAANPNGAAAA|703|Spring |RD|Suite H|Salem|Brooke County|WV|28048|United States|-5|condo| +28158|AAAAAAAAOPNGAAAA|148|4th |Drive|Suite U|Oakland|Phillips County|CO|89843|United States|-7|apartment| +28159|AAAAAAAAPPNGAAAA|207|Park Cedar|Wy|Suite 440|Sulphur Springs|Livingston County|KY|48354|United States|-5|single family| +28160|AAAAAAAAAAOGAAAA|724|Cherry Jefferson|Lane|Suite 460|Florence|Sac County|IA|53394|United States|-6|condo| +28161|AAAAAAAABAOGAAAA|977|Cedar West|Court|Suite 240|Oakwood|Richland County|IL|60169|United States|-6|apartment| +28162|AAAAAAAACAOGAAAA|533|Maple |Boulevard|Suite 430|Liberty|Drew County|AR|73451|United States|-6|condo| +28163|AAAAAAAADAOGAAAA|275|Miller |Circle|Suite G|Fairview|Crawford County|PA|15709|United States|-5|condo| +28164|AAAAAAAAEAOGAAAA|98|Church |Road|Suite 10|Pleasant Hill|Crow Wing County|MN|53604|United States|-6|condo| +28165|AAAAAAAAFAOGAAAA|303|College Third|Boulevard|Suite 20|Plainview|Nemaha County|NE|63683|United States|-7|single family| +28166|AAAAAAAAGAOGAAAA|661|2nd View|RD|Suite 320|Centerville|Keya Paha County|NE|60059|United States|-7|single family| +28167|AAAAAAAAHAOGAAAA|342|Walnut |Blvd|Suite 270|Springfield|Stafford County|VA|29303|United States|-5|apartment| +28168|AAAAAAAAIAOGAAAA|591|Johnson |Wy|Suite O|Friendship|Warren County|GA|34536|United States|-5|single family| +28169|AAAAAAAAJAOGAAAA|181|3rd |Dr.|Suite 190|Waterloo|Sanpete County|UT|81675|United States|-7|single family| +28170|AAAAAAAAKAOGAAAA|||Street|Suite R|Red Hill|||64338|||| +28171|AAAAAAAALAOGAAAA|769|Elm |Road|Suite 90|Red Hill|Concordia Parish|LA|74338|United States|-6|apartment| +28172|AAAAAAAAMAOGAAAA|291|5th |Pkwy|Suite 260|Concord|Pointe Coupee Parish|LA|74107|United States|-6|condo| +28173|AAAAAAAANAOGAAAA|772|Pine Mill|Street|Suite R|Stringtown|Pima County|AZ|80162|United States|-7|condo| +28174|AAAAAAAAOAOGAAAA|57|View Park|ST|Suite 120|Lakeview|Big Stone County|MN|58579|United States|-6|single family| +28175|AAAAAAAAPAOGAAAA|675|First Main|Lane|Suite F|Buena Vista|Cowlitz County|WA|95752|United States|-8|apartment| +28176|AAAAAAAAABOGAAAA|410|Oak Church|Cir.|Suite 430|Buena Vista|McMullen County|TX|75752|United States|-6|apartment| +28177|AAAAAAAABBOGAAAA|480|3rd East|Dr.|Suite A|Glenville|Montgomery County|AR|73445|United States|-6|apartment| +28178|AAAAAAAACBOGAAAA|943|South Dogwood|Ln|Suite V|Oakdale|Sanpete County|UT|89584|United States|-7|condo| +28179|AAAAAAAADBOGAAAA|737|South Green|ST|Suite 320|Bridgeport|Pocahontas County|IA|55817|United States|-6|apartment| +28180|AAAAAAAAEBOGAAAA|387|College Cherry|Way|Suite 470|Glenwood|Hinds County|MS|53511|United States|-6|condo| +28181|AAAAAAAAFBOGAAAA|548|Pine |Street|Suite 470|Harmony|Sevier County|UT|85804|United States|-7|condo| +28182|AAAAAAAAGBOGAAAA|583|Fourth Walnut|Cir.|Suite V|Hopewell|Burt County|NE|60587|United States|-6|condo| +28183|AAAAAAAAHBOGAAAA|23|Church 5th|Boulevard|Suite R|Midway|Hennepin County|MN|51904|United States|-6|apartment| +28184|AAAAAAAAIBOGAAAA|818|Elevnth |RD|Suite V|Fairfield|Rockwall County|TX|76192|United States|-6|apartment| +28185|AAAAAAAAJBOGAAAA|2|Laurel |RD|Suite 450|Five Forks|Davie County|NC|22293|United States|-5|single family| +28186|AAAAAAAAKBOGAAAA|253|Central |Avenue|Suite R|Sunnyside|Jackson County|MI|41952|United States|-5|single family| +28187|AAAAAAAALBOGAAAA|780|6th Locust|Cir.|Suite 430|Oakwood|Kershaw County|SC|20169|United States|-5|apartment| +28188|AAAAAAAAMBOGAAAA|806|14th Sixth|RD|Suite 40|Plainview|Canadian County|OK|73683|United States|-6|apartment| +28189|AAAAAAAANBOGAAAA|102|6th Sycamore|Pkwy|Suite 440|Oakdale|Cayuga County|NY|19584|United States|-5|apartment| +28190|AAAAAAAAOBOGAAAA|563|Main |RD|Suite 20|Highland Park|Pulaski County|GA|36534|United States|-5|condo| +28191|AAAAAAAAPBOGAAAA|135|13th Church|Court|Suite Q|Brownsville|Jeff Davis County|GA|39310|United States|-5|single family| +28192|AAAAAAAAACOGAAAA|936|Jackson River|Road|Suite 280|Salem|Madison County|FL|38048|United States|-5|single family| +28193|AAAAAAAABCOGAAAA|719|8th |Wy|Suite 10|Delmar|Elbert County|CO|83957|United States|-7|condo| +28194|AAAAAAAACCOGAAAA|229|Fourth Lincoln|Drive|Suite 130|Oakdale|Franklin Parish|LA|79584|United States|-6|single family| +28195|AAAAAAAADCOGAAAA|405|Miller Spruce|Road|Suite N|Powell|Osage County|OK|70662|United States|-6|apartment| +28196|AAAAAAAAECOGAAAA|939|Poplar |Avenue|Suite 250|Mount Pleasant|Alcorn County|MS|51933|United States|-6|apartment| +28197|AAAAAAAAFCOGAAAA|883|Fourth North|Road|Suite Q|Hopewell|Pitt County|NC|20587|United States|-5|single family| +28198|AAAAAAAAGCOGAAAA|983|Johnson Highland|Lane|Suite T|New Hope|Dakota County|NE|69431|United States|-6|single family| +28199|AAAAAAAAHCOGAAAA|783|Franklin |Way|Suite 420|Mountain View|Perry County|AL|34466|United States|-6|apartment| +28200|AAAAAAAAICOGAAAA|310|Valley |RD|Suite 480|Shady Grove|Yellow Medicine County|MN|52812|United States|-6|condo| +28201|AAAAAAAAJCOGAAAA|885|Sunset 13th|Circle|Suite X|Five Points|Fannin County|TX|76098|United States|-6|condo| +28202|AAAAAAAAKCOGAAAA|692|Lee River|Lane|Suite I|Shiloh|Allendale County|SC|29275|United States|-5|single family| +28203|AAAAAAAALCOGAAAA|925|Hill Center|RD|Suite E|White City|Marshall County|KY|46704|United States|-5|apartment| +28204|AAAAAAAAMCOGAAAA|354|Dogwood |Ave|Suite N|Newport|Talbot County|GA|31521|United States|-5|apartment| +28205|AAAAAAAANCOGAAAA|700|3rd |Boulevard|Suite C|Newport|Saline County|IL|61521|United States|-6|condo| +28206|AAAAAAAAOCOGAAAA|525|Forest |Cir.|Suite 180|Clearview|Screven County|GA|35495|United States|-5|single family| +28207|AAAAAAAAPCOGAAAA|412|View |Blvd|Suite Y|Forest Hills|Bennett County|SD|59237|United States|-6|apartment| +28208|AAAAAAAAADOGAAAA|937|12th Green|Blvd|Suite H|||||||| +28209|AAAAAAAABDOGAAAA|403|Adams |ST|Suite 190|Five Forks|Thomas County|KS|62293|United States|-6|single family| +28210|AAAAAAAACDOGAAAA|647|Spring 3rd|Wy|Suite 260|Macedonia|Oneida County|NY|11087|United States|-5|single family| +28211|AAAAAAAADDOGAAAA|766|Chestnut Oak|Pkwy|Suite V|Hopewell|Kent County|MI|40587|United States|-5|condo| +28212|AAAAAAAAEDOGAAAA|560|Ash 5th|Blvd|Suite 400|Friendship|Martin County|IN|44536|United States|-5|single family| +28213|AAAAAAAAFDOGAAAA|244|South Lakeview|ST|Suite F|Brentwood|Mason County|IL|64188|United States|-6|condo| +28214|AAAAAAAAGDOGAAAA|654|Cherry 3rd|Dr.|Suite P|Clifton|Moniteau County|MO|68014|United States|-6|condo| +28215|AAAAAAAAHDOGAAAA|818|Sixth Elm|Road|Suite 130|Farmington|Fond du Lac County|WI|59145|United States|-6|condo| +28216|AAAAAAAAIDOGAAAA|875|Poplar |Way|Suite 380|Deerfield|Posey County|IN|49840|United States|-5|condo| +28217|AAAAAAAAJDOGAAAA|94|4th Fourth|Road|Suite 490|Jackson|Jefferson Davis County|MS|59583|United States|-6|single family| +28218|AAAAAAAAKDOGAAAA|382|Hillcrest |Avenue|Suite W|Kingston|Adams County|ID|84975|United States|-7|condo| +28219|AAAAAAAALDOGAAAA|452|7th |RD|Suite 140|Lone Oak|Republic County|KS|66893|United States|-6|condo| +28220|AAAAAAAAMDOGAAAA|262|Lee Wilson|Cir.|Suite D|Five Points|Charlton County|GA|36098|United States|-5|single family| +28221|AAAAAAAANDOGAAAA|379|Twelfth Ash||||Ketchikan Gateway Borough|||United States|-9|| +28222|AAAAAAAAODOGAAAA|255|Third Fifteenth|Street|Suite 340|Lakeside|Calhoun County|IA|59532|United States|-6|apartment| +28223|AAAAAAAAPDOGAAAA|634|North 6th|Dr.|Suite 90|White Oak|Dona Ana County|NM|86668|United States|-7|condo| +28224|AAAAAAAAAEOGAAAA|527|Madison |Circle|Suite G|Walnut Grove|Finney County|KS|67752|United States|-6|condo| +28225|AAAAAAAABEOGAAAA|881|North |Avenue|Suite Q|Newport|Wheeler County|TX|71521|United States|-6|single family| +28226|AAAAAAAACEOGAAAA|468|Valley Valley|Dr.|Suite 450|Arcola|Cortland County|NY|11654|United States|-5|apartment| +28227|AAAAAAAADEOGAAAA|361|15th |Lane|Suite L|Maple Grove|Grand Isle County|VT|08852|United States|-5|single family| +28228|AAAAAAAAEEOGAAAA|16|Pine Oak|Avenue|Suite 350|Hopewell|Cottle County|TX|70587|United States|-6|condo| +28229|AAAAAAAAFEOGAAAA|961|Main Oak|Lane|Suite Q|Kingston|Mayes County|OK|74975|United States|-6|apartment| +28230|AAAAAAAAGEOGAAAA|630|Jefferson 2nd|Circle|Suite 490|Fairfield|Lincoln County|WY|86192|United States|-7|apartment| +28231|AAAAAAAAHEOGAAAA|104|6th Elm|Boulevard|Suite 120|Oakwood|Mason County|WV|20169|United States|-5|condo| +28232|AAAAAAAAIEOGAAAA|538|Ash Wilson|Road|Suite 60|Unionville|Saginaw County|MI|41711|United States|-5|single family| +28233|AAAAAAAAJEOGAAAA|208|Jefferson |ST|Suite 440|Concord|Johnson County|KY|44107|United States|-6|single family| +28234|AAAAAAAAKEOGAAAA|718|Cherry Woodland|Avenue|Suite R|Oakland|Dunn County|ND|59843|United States|-6|single family| +28235|AAAAAAAALEOGAAAA|915|Jackson |RD|Suite O|Concord|Columbia County|GA|34107|United States|-5|apartment| +28236|AAAAAAAAMEOGAAAA|496|2nd |Parkway|Suite 400|White Oak|Mills County|IA|56668|United States|-6|condo| +28237|AAAAAAAANEOGAAAA|189||Road|||Orleans Parish|LA||||condo| +28238|AAAAAAAAOEOGAAAA|540|11th |Boulevard|Suite J|Oak Grove|Kearney County|NE|68370|United States|-7|single family| +28239|AAAAAAAAPEOGAAAA|3|Ridge |Boulevard|Suite H|Jamestown|Staunton city|VA|26867|United States|-5|apartment| +28240|AAAAAAAAAFOGAAAA|4|Forest Mill|Boulevard|Suite D|Wildwood|Limestone County|AL|36871|United States|-6|condo| +28241|AAAAAAAABFOGAAAA|367|Center 7th|Way|Suite 430|Wildwood|Wahkiakum County|WA|96871|United States|-8|single family| +28242|AAAAAAAACFOGAAAA|170|Oak North|Drive|Suite C|Riverview|Loudoun County|VA|29003|United States|-5|condo| +28243|AAAAAAAADFOGAAAA|994|Second 7th|Ct.|Suite K|Oakland|Vermilion County|IL|69843|United States|-6|single family| +28244|AAAAAAAAEFOGAAAA|852|Valley Main|Avenue|Suite 150|Friendship|Perry County|PA|14536|United States|-5|single family| +28245|AAAAAAAAFFOGAAAA|366|Twelfth Hillcrest|Avenue|Suite 60|Cedar Grove|Kauai County|HI|90411|United States|-10|single family| +28246|AAAAAAAAGFOGAAAA|1|Jackson Third|Ct.|Suite U|Jerome|Gulf County|FL|39920|United States|-5|apartment| +28247|AAAAAAAAHFOGAAAA|229|8th |Ln|Suite S|Georgetown|Pulaski County|VA|27057|United States|-5|single family| +28248|AAAAAAAAIFOGAAAA|802|Valley First|Ct.|Suite 260|Harmony|Union County|NM|85804|United States|-7|single family| +28249|AAAAAAAAJFOGAAAA|726|Third Lincoln|Dr.|Suite Q|Edgewood|Ouachita County|AR|70069|United States|-6|condo| +28250|AAAAAAAAKFOGAAAA|94|Spring |Street|Suite L|Marion|Muskingum County|OH|40399|United States|-5|apartment| +28251|AAAAAAAALFOGAAAA|590|Washington Poplar|Court|Suite 490|Lincoln|Henderson County|IL|61289|United States|-6|single family| +28252|AAAAAAAAMFOGAAAA|580|Williams Madison|Ct.|Suite 490|Sulphur Springs|Renville County|MN|58354|United States|-6|apartment| +28253|AAAAAAAANFOGAAAA|293|Laurel |Boulevard|Suite 120|Ashland|Braxton County|WV|24244|United States|-5|condo| +28254|AAAAAAAAOFOGAAAA|297|5th Center|Drive|Suite K|Bunker Hill|Pike County|PA|10150|United States|-5|single family| +28255|AAAAAAAAPFOGAAAA|628|Ninth |Road|Suite 270|Cedar Grove|Greeley County|NE|60411|United States|-6|apartment| +28256|AAAAAAAAAGOGAAAA|870|Park |Pkwy|Suite 90|Clifton|Clearfield County|PA|18014|United States|-5|single family| +28257|AAAAAAAABGOGAAAA|733|Railroad Sunset|Avenue|Suite 280|Waterloo|Jackson County|OR|91675|United States|-8|condo| +28258|AAAAAAAACGOGAAAA|184|Third |Road|Suite 170|Woodland|Edmunds County|SD|54854|United States|-6|apartment| +28259|AAAAAAAADGOGAAAA|723|Jackson Ridge|Court|Suite W|Concord|King William County|VA|24107|United States|-5|apartment| +28260|AAAAAAAAEGOGAAAA|427|Broadway 7th|ST|Suite T|Franklin|Jim Hogg County|TX|79101|United States|-6|apartment| +28261|AAAAAAAAFGOGAAAA|367|Dogwood |Dr.|Suite 180|Clinton|Kauai County|HI|98222|United States|-10|apartment| +28262|AAAAAAAAGGOGAAAA|809|Adams |Ave|Suite 250|Wildwood|Cabarrus County|NC|26871|United States|-5|single family| +28263|AAAAAAAAHGOGAAAA|||Court|Suite V|||MT|69231|United States|-7|condo| +28264|AAAAAAAAIGOGAAAA|257|First |Ct.|Suite J|Fairfield|Bonneville County|ID|86192|United States|-7|condo| +28265|AAAAAAAAJGOGAAAA|487|Smith Elm|Drive|Suite D|Cedar Grove|Musselshell County|MT|60411|United States|-7|apartment| +28266|AAAAAAAAKGOGAAAA|879|2nd Walnut|Court|Suite T|Arlington|Lamar County|TX|76557|United States|-6|apartment| +28267|AAAAAAAALGOGAAAA|80|First |Pkwy|Suite 210|Valley View|Franklin County|VT|05724|United States|-5|condo| +28268|AAAAAAAAMGOGAAAA|159|Main Johnson|Wy|Suite 200|Summit|Logan County|KY|40499|United States|-5|condo| +28269|AAAAAAAANGOGAAAA|443|Maple |Street|Suite 170|Leesburg|Pawnee County|NE|65605|United States|-7|single family| +28270|AAAAAAAAOGOGAAAA|565|Franklin 6th|Drive|Suite 470|Harmony|Randolph County|IN|45804|United States|-5|single family| +28271|AAAAAAAAPGOGAAAA|461|Birch Dogwood||||||61757||-6|single family| +28272|AAAAAAAAAHOGAAAA|656|Walnut Ash|Avenue|Suite 320|Crossroads|Lancaster County|SC|20534|United States|-5|single family| +28273|AAAAAAAABHOGAAAA|361|11th ||||Spokane County||||-8|| +28274|AAAAAAAACHOGAAAA|783|Lake |Way|Suite A|Enterprise|Pulaski County|IL|61757|United States|-6|condo| +28275|AAAAAAAADHOGAAAA|927|Spring |||Glendale|Butte County|ID|83951|United States|-7|| +28276|AAAAAAAAEHOGAAAA|649|Ridge 7th|Circle|Suite 240|Maple Grove|Kodiak Island Borough|AK|98252|United States|-9|condo| +28277|AAAAAAAAFHOGAAAA|152|Lincoln |Wy|Suite 360|Clifton|Taylor County|FL|38014|United States|-5|apartment| +28278|AAAAAAAAGHOGAAAA|240|Locust |Ave|Suite B|Clifton|Kingsbury County|SD|58014|United States|-7|single family| +28279|AAAAAAAAHHOGAAAA|546|Highland |Blvd|Suite 20|Arlington|Millard County|UT|86557|United States|-7|apartment| +28280|AAAAAAAAIHOGAAAA|949|Williams |Pkwy|Suite F|Oakwood|Converse County|WY|80169|United States|-7|single family| +28281|AAAAAAAAJHOGAAAA|558|Park |Way|Suite W|Fairview|Hillsdale County|MI|45709|United States|-5|condo| +28282|AAAAAAAAKHOGAAAA|799|5th |Road|Suite 170|Montague|Warren County|MS|54062|United States|-6|condo| +28283|AAAAAAAALHOGAAAA|256|Sycamore 4th|Dr.|Suite C|Pleasant Hill|Jones County|TX|73604|United States|-6|single family| +28284|AAAAAAAAMHOGAAAA|636|Broadway |RD|Suite 270|Highland|Berrien County|MI|49454|United States|-5|apartment| +28285|AAAAAAAANHOGAAAA|209|Ridge |Dr.|Suite F|Pine Grove|Ransom County|ND|54593|United States|-6|condo| +28286|AAAAAAAAOHOGAAAA|603|Adams |Blvd|Suite G|Stringtown|Monroe County|IA|50162|United States|-6|condo| +28287|AAAAAAAAPHOGAAAA|618|Fifth |RD|Suite 440|Oakdale|Franklin County|NY|19584|United States|-5|apartment| +28288|AAAAAAAAAIOGAAAA|797|Sixth |Circle|Suite S|Antioch|Polk County|NC|28605|United States|-5|apartment| +28289|AAAAAAAABIOGAAAA|836|Wilson 4th|Blvd|Suite 400|Oakdale|Marin County|CA|99584|United States|-8|apartment| +28290|AAAAAAAACIOGAAAA|451|10th Miller|Dr.|Suite 90|Oakwood|Lake County|OR|90169|United States|-8|condo| +28291|AAAAAAAADIOGAAAA|553|Jackson College|Lane|Suite I|Oakdale|Woodbury County|IA|59584|United States|-6|apartment| +28292|AAAAAAAAEIOGAAAA|377|Birch Walnut|Pkwy|Suite 450|Mount Zion|Orleans County|NY|18054|United States|-5|single family| +28293|AAAAAAAAFIOGAAAA|||Ave|||Scott County|MO||United States||single family| +28294|AAAAAAAAGIOGAAAA|614|First Davis|Ave|Suite S|Ashland|Menifee County|KY|44244|United States|-5|single family| +28295|AAAAAAAAHIOGAAAA|580|Johnson |Parkway|Suite I|Pleasant Valley|Motley County|TX|72477|United States|-6|single family| +28296|AAAAAAAAIIOGAAAA|||||Kingston|Clay County|IN|44975|United States|-5|single family| +28297|AAAAAAAAJIOGAAAA|330|Hill Laurel|Court|Suite 180|Glenwood|Greenlee County|AZ|83511|United States|-7|condo| +28298|AAAAAAAAKIOGAAAA|417|4th |Ln|Suite 470|New Hope|Clark County|WA|99431|United States|-8|condo| +28299|AAAAAAAALIOGAAAA|389|14th First|Lane|Suite 450|Pine Grove|Wyandot County|OH|44593|United States|-5|condo| +28300|AAAAAAAAMIOGAAAA|896|Jackson Washington|Ct.|Suite 290|Georgetown|Faulkner County|AR|77057|United States|-6|condo| +28301|AAAAAAAANIOGAAAA|351|Elm |ST|Suite L|Forest Hills|Harlan County|NE|69237|United States|-6|apartment| +28302|AAAAAAAAOIOGAAAA|71|10th |Court|Suite 440|Kingston|Lincoln County|MT|64975|United States|-7|apartment| +28303|AAAAAAAAPIOGAAAA|45|Mill 1st|Parkway|Suite E|Oak Grove|Fayette County|WV|28370|United States|-5|single family| +28304|AAAAAAAAAJOGAAAA|34|3rd |Ct.|Suite S|Kingston|Conejos County|CO|84975|United States|-7|single family| +28305|AAAAAAAABJOGAAAA|881|River |Road|Suite 140|Oak Grove|Major County|OK|78370|United States|-6|single family| +28306|AAAAAAAACJOGAAAA|342|Williams |Road|Suite 100|Newport|Union Parish|LA|71521|United States|-6|apartment| +28307|AAAAAAAADJOGAAAA|679|Meadow |Avenue|Suite J|Pleasant Valley|Nueces County|TX|72477|United States|-6|condo| +28308|AAAAAAAAEJOGAAAA|123|Cherry |Cir.|Suite W|Lakewood|Livingston County|IL|68877|United States|-6|condo| +28309|AAAAAAAAFJOGAAAA|631|Cherry Ridge|Lane|Suite N|Forestville|Fulton County|IL|63027|United States|-6|condo| +28310|AAAAAAAAGJOGAAAA|5|Highland Elm|Ln|Suite 470|Franklin|Griggs County|ND|59101|United States|-6|apartment| +28311|AAAAAAAAHJOGAAAA|858|Woodland Fourth|RD|Suite 170|Lincoln|Reno County|KS|61289|United States|-6|apartment| +28312|AAAAAAAAIJOGAAAA|742|College |Avenue|Suite 190|Liberty|Porter County|IN|43451|United States|-5|apartment| +28313|AAAAAAAAJJOGAAAA|539|6th Meadow|Ct.|Suite D|Buena Vista|Ramsey County|ND|55752|United States|-6|single family| +28314|AAAAAAAAKJOGAAAA|610|Main |Ct.|Suite Q|Mountain View|Transylvania County|NC|24466|United States|-5|apartment| +28315|AAAAAAAALJOGAAAA|330|Spruce Sunset|ST|Suite I|Watkins|Coshocton County|OH|41732|United States|-5|single family| +28316|AAAAAAAAMJOGAAAA|507|4th 15th|Cir.|Suite 100|Greendale|Yavapai County|AZ|89343|United States|-7|single family| +28317|AAAAAAAANJOGAAAA|528|Miller |Road|Suite V|Shady Grove|Washington County|ID|82812|United States|-7|single family| +28318|AAAAAAAAOJOGAAAA|929|Madison |Avenue|Suite H|Green Acres|Houston County|GA|37683|United States|-5|condo| +28319|AAAAAAAAPJOGAAAA|354|Sycamore Locust|Street|Suite H|Centerville|Ripley County|IN|40059|United States|-5|condo| +28320|AAAAAAAAAKOGAAAA|5|3rd View|Cir.|Suite Y|Clinton|Bowman County|ND|58222|United States|-6|condo| +28321|AAAAAAAABKOGAAAA|118|Chestnut Park|Dr.|Suite 440|Summit|Clay County|WV|20499|United States|-5|apartment| +28322|AAAAAAAACKOGAAAA|4|Walnut Meadow|Drive|Suite J|Five Forks|Gray County|KS|62293|United States|-6|apartment| +28323|AAAAAAAADKOGAAAA|296|College |Court|Suite H|Stringtown|Poquoson city|VA|20162|United States|-5|condo| +28324|AAAAAAAAEKOGAAAA|613|Walnut Oak|Dr.|Suite 120|New Hope|Randolph County|GA|39431|United States|-5|condo| +28325|AAAAAAAAFKOGAAAA|327|College |RD|Suite B|Sulphur Springs|Guernsey County|OH|48354|United States|-5|condo| +28326|AAAAAAAAGKOGAAAA|598|1st |Ln|Suite 180|Midway|Barton County|MO|61904|United States|-6|condo| +28327|AAAAAAAAHKOGAAAA|500|1st West|Lane|Suite D|Winslow|Ellis County|KS|68525|United States|-6|single family| +28328|AAAAAAAAIKOGAAAA|177|Park Laurel|Dr.|Suite K|Kingston|Rio Grande County|CO|84975|United States|-7|condo| +28329|AAAAAAAAJKOGAAAA|266|Franklin 2nd|Drive|Suite 290|Brownsville|Greeley County|KS|69310|United States|-6|single family| +28330|AAAAAAAAKKOGAAAA|543|Franklin Fourth|Blvd|Suite 160|Gravel Hill|New Kent County|VA|21944|United States|-5|apartment| +28331|AAAAAAAALKOGAAAA|837|Hill |Ave|Suite U|Sunnyside|Calhoun County|IA|51952|United States|-6|single family| +28332|AAAAAAAAMKOGAAAA|583|North Oak|Road|Suite 70|Jackson|Johnson County|IN|49583|United States|-5|single family| +28333|AAAAAAAANKOGAAAA|260|View Jefferson|Circle|Suite K|Waterloo|Sullivan County|MO|61675|United States|-6|single family| +28334|AAAAAAAAOKOGAAAA|348|5th |Cir.|Suite 240|Bunker Hill|Tipton County|IN|40150|United States|-5|condo| +28335|AAAAAAAAPKOGAAAA|893|Pine Green|Circle|Suite A|Newport|Bonner County|ID|81521|United States|-7|condo| +28336|AAAAAAAAALOGAAAA|677|Lincoln Lincoln|Cir.|Suite J|Hopewell|Harrison County|MO|60587|United States|-6|condo| +28337|AAAAAAAABLOGAAAA|944|Oak Main|Ln|Suite 320|Bridgeport|Herkimer County|NY|15817|United States|-5|condo| +28338|AAAAAAAACLOGAAAA|998|Sunset Maple|Wy|Suite P|Arlington|Klamath County|OR|96557|United States|-8|condo| +28339|AAAAAAAADLOGAAAA|3|9th |Avenue|Suite I|Five Points|Chenango County|NY|16098|United States|-5|apartment| +28340|AAAAAAAAELOGAAAA|138|Hill Williams|Circle|Suite I|Kingston|Sheridan County|WY|84975|United States|-7|single family| +28341|AAAAAAAAFLOGAAAA|969|7th |Blvd|Suite 330|Pleasant Hill|Norton city|VA|23604|United States|-5|single family| +28342|AAAAAAAAGLOGAAAA|763|Fourth Walnut|Way|Suite T|Ashland|Schleicher County|TX|74244|United States|-6|apartment| +28343|AAAAAAAAHLOGAAAA|932|Ridge North|Way|Suite 360|Clinton|Alfalfa County|OK|78222|United States|-6|condo| +28344|AAAAAAAAILOGAAAA|434|2nd |Lane|Suite N|Friendship|Gallatin County|MT|64536|United States|-7|single family| +28345|AAAAAAAAJLOGAAAA|233|Johnson 2nd|Ave|Suite 460|Maple Grove|Geauga County|OH|48252|United States|-5|apartment| +28346|AAAAAAAAKLOGAAAA|367|Lake Fifth|Lane|Suite 70|Clinton|Fulton County|KY|48222|United States|-6|single family| +28347|AAAAAAAALLOGAAAA|78|Park Elm|Street|Suite M|Maple Grove|Barber County|KS|68252|United States|-6|condo| +28348|AAAAAAAAMLOGAAAA|151|Washington |Wy|Suite 40|Plainview|Coos County|NH|04283|United States|-5|apartment| +28349|AAAAAAAANLOGAAAA|808|Ash |Ave|Suite 290|Oak Hill|Jasper County|IA|57838|United States|-6|condo| +28350|AAAAAAAAOLOGAAAA|288|8th |Way|Suite 460|Wildwood|Scott County|MO|66871|United States|-6|condo| +28351|AAAAAAAAPLOGAAAA|991|2nd |Ln|Suite 150|Oak Hill|Lyman County|SD|57838|United States|-7|apartment| +28352|AAAAAAAAAMOGAAAA|974|Sunset |Circle|Suite Y|Lakeside|Grand Traverse County|MI|49532|United States|-5|single family| +28353|AAAAAAAABMOGAAAA|691|Franklin Main|RD|Suite 470|Pine Grove|San Miguel County|NM|84593|United States|-7|apartment| +28354|AAAAAAAACMOGAAAA|759|Hill Forest|Cir.|Suite B|Star|Lee County|GA|30725|United States|-5|condo| +28355|AAAAAAAADMOGAAAA|970|Miller Spring|Lane|Suite I|Buckingham|Montrose County|CO|84092|United States|-7|apartment| +28356|AAAAAAAAEMOGAAAA|12|Sunset |Court|Suite E|Antioch|Green Lake County|WI|58605|United States|-6|condo| +28357|AAAAAAAAFMOGAAAA||Forest ||Suite 350|Wilson|Oglethorpe County||||-5|apartment| +28358|AAAAAAAAGMOGAAAA|111|Forest Birch|Blvd|Suite 100|Belfast|Monroe County|AL|30125|United States|-6|condo| +28359|AAAAAAAAHMOGAAAA|793|2nd |Ln|Suite 280|Glenwood|Geauga County|OH|43511|United States|-5|condo| +28360|AAAAAAAAIMOGAAAA|403|9th |Cir.|Suite A|The Meadows|Rockingham County|VA|20026|United States|-5|single family| +28361|AAAAAAAAJMOGAAAA||Maple Adams|||||GA|30534|United States|-5|| +28362|AAAAAAAAKMOGAAAA|913|College |Road|Suite 100|Springfield|Cuyahoga County|OH|49303|United States|-5|apartment| +28363|AAAAAAAALMOGAAAA|694|East |Pkwy|Suite 60|Five Forks|Mahaska County|IA|52293|United States|-6|condo| +28364|AAAAAAAAMMOGAAAA|353|Hill Washington|Cir.|Suite 260|Shiloh|Martin County|FL|39275|United States|-5|single family| +28365|AAAAAAAANMOGAAAA|905|Oak Pine|Road|Suite Q|Lenox|Bourbon County|KS|61143|United States|-6|condo| +28366|AAAAAAAAOMOGAAAA|402|10th |Dr.|Suite 490|Woodville|Chelan County|WA|94289|United States|-8|condo| +28367|AAAAAAAAPMOGAAAA|173|9th |Court|Suite P|Liberty|McLeod County|MN|53451|United States|-6|condo| +28368|AAAAAAAAANOGAAAA|375|12th Elm|Ct.|Suite 230|Crossroads|Pawnee County|OK|70534|United States|-6|condo| +28369|AAAAAAAABNOGAAAA|931|Sixth |Circle|Suite M|Oakland|Giles County|VA|29843|United States|-5|apartment| +28370|AAAAAAAACNOGAAAA|203|Pine |Avenue|Suite M|Elba|Randolph County|AR|70262|United States|-6|condo| +28371|AAAAAAAADNOGAAAA|461|Valley Jefferson|Street|Suite W|Barnes|Washington County|AR|73788|United States|-6|condo| +28372|AAAAAAAAENOGAAAA|564|Miller |Avenue|Suite 350|Jamestown|Harlan County|NE|66867|United States|-6|condo| +28373|AAAAAAAAFNOGAAAA|821|Birch |Court|Suite 450|Pleasant Grove|Childress County|TX|74136|United States|-6|apartment| +28374|AAAAAAAAGNOGAAAA|24|Sycamore |Street|Suite 100|Garrison|Gilliam County|OR|98767|United States|-8|condo| +28375|AAAAAAAAHNOGAAAA|94|Lee Center|RD|Suite 470|Union|Charles Mix County|SD|58721|United States|-6|condo| +28376|AAAAAAAAINOGAAAA|986|9th Lakeview|Dr.|Suite 180|Riverview|Iroquois County|IL|69003|United States|-6|condo| +28377|AAAAAAAAJNOGAAAA|64|10th |Boulevard|Suite 210|Greenfield|Elko County|NV|85038|United States|-8|single family| +28378|AAAAAAAAKNOGAAAA|210|Locust |Way|Suite D|||NJ|02357|United States|-5|| +28379|AAAAAAAALNOGAAAA|98|9th |RD|Suite 200|Farmington|Dodge County|WI|59145|United States|-6|single family| +28380|AAAAAAAAMNOGAAAA|763|Cedar Forest|Boulevard|Suite 90|Stafford|Monroe County|FL|34980|United States|-5|condo| +28381|AAAAAAAANNOGAAAA|318|View |Road|Suite U|Shiloh|Macon County|AL|39275|United States|-6|apartment| +28382|AAAAAAAAONOGAAAA|516|Miller |Ln|Suite 30|Summit|Livingston County|NY|10499|United States|-5|condo| +28383|AAAAAAAAPNOGAAAA|111|West |Dr.|Suite C|Oak Grove|Howard County|IN|48370|United States|-5|single family| +28384|AAAAAAAAAOOGAAAA|412|4th |ST|Suite 230|Red Hill|Cass County|IA|54338|United States|-6|apartment| +28385|AAAAAAAABOOGAAAA|912|Ash Miller|Road|Suite 450|Belmont|Murray County|MN|50191|United States|-6|single family| +28386|AAAAAAAACOOGAAAA|366|Smith Lee|Dr.|Suite M|Brownsville|Faribault County|MN|59310|United States|-6|single family| +28387|AAAAAAAADOOGAAAA|985|2nd |Street|Suite J|Shiloh|Greene County|MO|69275|United States|-6|single family| +28388|AAAAAAAAEOOGAAAA|202|Pine 1st|Lane|Suite Y|Providence|Barren County|KY|46614|United States|-6|condo| +28389|AAAAAAAAFOOGAAAA|639|Ridge Spring|Wy|Suite 260|Ryan|Taylor County|IA|50525|United States|-6|condo| +28390|AAAAAAAAGOOGAAAA|182|Lake Cedar|Dr.|Suite 30|Walnut Grove|Pulaski County|MO|67752|United States|-6|condo| +28391|AAAAAAAAHOOGAAAA|329|Center Dogwood|Ct.|Suite S|Cedar Grove|Tooele County|UT|80411|United States|-7|apartment| +28392|AAAAAAAAIOOGAAAA|636|Davis Elm|Ln|Suite 40|Centerville|Cameron County|TX|70059|United States|-6|single family| +28393|AAAAAAAAJOOGAAAA|884|Elm |Pkwy|Suite Q|Dewey|Union County|NM|81160|United States|-7|apartment| +28394|AAAAAAAAKOOGAAAA|304|Park |Way|Suite 450|Springdale|Loup County|NE|68883|United States|-7|condo| +28395|AAAAAAAALOOGAAAA|751|Maple Spruce|Avenue|Suite 320|Riverdale|Williamson County|TX|79391|United States|-6|apartment| +28396|AAAAAAAAMOOGAAAA|396|Laurel ||Suite E|Lebanon||NJ||United States||single family| +28397|AAAAAAAANOOGAAAA|511|Chestnut |Way|Suite 450|Waterloo|Henry County|MO|61675|United States|-6|condo| +28398|AAAAAAAAOOOGAAAA|820|Jefferson Maple|Dr.|Suite 320|Lakewood|Brazoria County|TX|78877|United States|-6|condo| +28399|AAAAAAAAPOOGAAAA|450|Eigth Oak|Court|Suite 380|Arlington|Kankakee County|IL|66557|United States|-6|condo| +28400|AAAAAAAAAPOGAAAA|898|1st |Blvd|Suite Q|Stringtown|Sevier County|AR|70162|United States|-6|apartment| +28401|AAAAAAAABPOGAAAA|988|Main |Lane|Suite 490|Glendale|Randolph County|WV|23951|United States|-5|apartment| +28402|AAAAAAAACPOGAAAA|991|Wilson |Street|Suite B|Greenville|Somerset County|PA|11387|United States|-5|single family| +28403|AAAAAAAADPOGAAAA|627|10th 11th|Avenue|Suite 420|Riley|Pima County|AZ|81692|United States|-7|condo| +28404|AAAAAAAAEPOGAAAA|160|Fifth |Blvd|Suite 420|Brownsville|Wise County|TX|79310|United States|-6|apartment| +28405|AAAAAAAAFPOGAAAA|241|Jackson |Ave|Suite O|Liberty|Massac County|IL|63451|United States|-6|apartment| +28406|AAAAAAAAGPOGAAAA|885|Smith |ST|Suite J|Shiloh|McKenzie County|ND|59275|United States|-6|apartment| +28407|AAAAAAAAHPOGAAAA|367|Jefferson |Lane|Suite 240|Midway|Fulton County|IL|61904|United States|-6|condo| +28408|AAAAAAAAIPOGAAAA|340|Sunset |Ave|Suite A|Clifton|Lewis County|WA|98014|United States|-8|single family| +28409|AAAAAAAAJPOGAAAA|386|Forest Central|Dr.|Suite 350|Gary|Jefferson County|NE|60418|United States|-7|apartment| +28410|AAAAAAAAKPOGAAAA|137|Pine |Way|Suite R|Highland|Greene County|IL|69454|United States|-6|apartment| +28411|AAAAAAAALPOGAAAA|734|3rd Willow|Lane|Suite 260|Greenwood|Fremont County|ID|88828|United States|-7|single family| +28412|AAAAAAAAMPOGAAAA|477|Willow River|Avenue|Suite 200|Enterprise|Elbert County|CO|81757|United States|-7|single family| +28413|AAAAAAAANPOGAAAA|484|Oak 7th|Drive|Suite D|Maple Grove|Lynn County|TX|78252|United States|-6|condo| +28414|AAAAAAAAOPOGAAAA|159|Lee Adams|Blvd|Suite 260|Marion|Bamberg County|SC|20399|United States|-5|single family| +28415|AAAAAAAAPPOGAAAA|780|9th |Pkwy|Suite 300|Edgewood|Grant County|OK|70069|United States|-6|apartment| +28416|AAAAAAAAAAPGAAAA|723|Center Willow|Boulevard|Suite 120|Enterprise|Custer County|SD|51757|United States|-6|apartment| +28417|AAAAAAAABAPGAAAA|146|Center Cherry|Drive|Suite 230|Oakland|Berrien County|GA|39843|United States|-5|apartment| +28418|AAAAAAAACAPGAAAA|666|Williams 8th|Road|Suite 260|Summit|Randolph County|AR|70499|United States|-6|single family| +28419|AAAAAAAADAPGAAAA|744|Washington |Avenue|Suite 310|Mount Zion|Russell County|KS|68054|United States|-6|condo| +28420|AAAAAAAAEAPGAAAA|757|Highland South|Pkwy|Suite 0|Bunker Hill|Crawford County|KS|60150|United States|-6|condo| +28421|AAAAAAAAFAPGAAAA|924|Laurel |Way|Suite 0|Lebanon|Bond County|IL|62898|United States|-6|single family| +28422|AAAAAAAAGAPGAAAA|828|Fourth |Ln|Suite P|Lone Pine|Portsmouth city|VA|27441|United States|-5|apartment| +28423|AAAAAAAAHAPGAAAA|828|Fifteenth ||Suite U|Pleasant Hill|||83604||-7|| +28424|AAAAAAAAIAPGAAAA|883|Pine |Avenue|Suite 140|Bunker Hill|Bacon County|GA|30150|United States|-5|single family| +28425|AAAAAAAAJAPGAAAA|188|Jackson |Boulevard|Suite 450|Antioch|Marshall County|WV|28605|United States|-5|apartment| +28426|AAAAAAAAKAPGAAAA|234|Cherry |Wy|Suite P|River Oaks|Linn County|OR|98075|United States|-8|single family| +28427|AAAAAAAALAPGAAAA|574|Railroad River|Avenue|Suite 480|Rolling Hills|Garfield County|WA|97272|United States|-8|single family| +28428|AAAAAAAAMAPGAAAA|991|8th Johnson|Drive|Suite D|Oak Hill|Bradford County|PA|17838|United States|-5|condo| +28429|AAAAAAAANAPGAAAA|711|Eigth |Street|Suite 220|Green Acres|Johnson County|MO|67683|United States|-6|condo| +28430|AAAAAAAAOAPGAAAA|109|4th |ST|Suite 430|Glendale|Marion County|IA|53951|United States|-6|apartment| +28431|AAAAAAAAPAPGAAAA|849|View |Boulevard|Suite 130|Glendale|Douglas County|NV|83951|United States|-8|condo| +28432|AAAAAAAAABPGAAAA|926|Hickory Park|Dr.|Suite O|Carpenter|Monmouth County|NJ|01747|United States|-5|condo| +28433|AAAAAAAABBPGAAAA|264|Green Lee|Ct.|Suite 20|Edgewood|Ohio County|IN|40069|United States|-5|apartment| +28434|AAAAAAAACBPGAAAA|820|Second |Cir.|Suite Q|Lincoln|Baxter County|AR|71289|United States|-6|condo| +28435|AAAAAAAADBPGAAAA|709|Main |||Brunswick|Venango County|PA|14642|United States|-5|| +28436|AAAAAAAAEBPGAAAA|210|Eigth Spruce|Circle|Suite 30|Shiloh|Lackawanna County|PA|19275|United States|-5|apartment| +28437|AAAAAAAAFBPGAAAA|79|College West|Avenue|Suite 50|Riverside|Douglas County|SD|59231|United States|-6|apartment| +28438|AAAAAAAAGBPGAAAA|181|Lee |Ln|Suite Y|Woodrow|Newaygo County|MI|44273|United States|-5|single family| +28439|AAAAAAAAHBPGAAAA|920|Maple Hillcrest|Road|Suite 440|Sulphur Springs|Pulaski County|IL|68354|United States|-6|apartment| +28440|AAAAAAAAIBPGAAAA|799|Lincoln |Ct.|Suite 60|Riverdale|Benton County|WA|99391|United States|-8|single family| +28441|AAAAAAAAJBPGAAAA|253|Third Seventh|Drive|Suite E|Greenville|Charlton County|GA|31387|United States|-5|apartment| +28442|AAAAAAAAKBPGAAAA|669|4th |Boulevard|Suite N|Rosebud|Natrona County|WY|82244|United States|-7|single family| +28443|AAAAAAAALBPGAAAA|56|Maple River|ST|Suite X|Walnut Grove|Zapata County|TX|77752|United States|-6|apartment| +28444|AAAAAAAAMBPGAAAA|915|Jefferson Lake|Ave|Suite K|Greenfield|Guadalupe County|TX|75038|United States|-6|single family| +28445|AAAAAAAANBPGAAAA|220|Spruce 11th|Circle|Suite 120|Riverview|Elmore County|AL|39003|United States|-6|single family| +28446|AAAAAAAAOBPGAAAA|670|Park |Drive|Suite 310|Deerfield|Harrison County|TX|79840|United States|-6|condo| +28447|AAAAAAAAPBPGAAAA|759|Green |Court|Suite D|Pleasant Valley|Sheridan County|KS|62477|United States|-6|apartment| +28448|AAAAAAAAACPGAAAA||Adams Pine|||Highland Park|Howard County|IA||United States|-6|| +28449|AAAAAAAABCPGAAAA|788|11th 2nd|Avenue|Suite L|Valley View|Lee County|MS|55124|United States|-6|single family| +28450|AAAAAAAACCPGAAAA|756|River Wilson|Pkwy|Suite 310|Oakwood|Schuylkill County|PA|10169|United States|-5|single family| +28451|AAAAAAAADCPGAAAA|463|Sunset Ridge|Road|Suite A|Wilson|Iowa County|IA|56971|United States|-6|condo| +28452|AAAAAAAAECPGAAAA|651|Spruce |RD|Suite 410|Antioch|Delaware County|OK|78605|United States|-6|single family| +28453|AAAAAAAAFCPGAAAA|444|2nd |ST|Suite M|Concord|Sierra County|NM|84107|United States|-7|condo| +28454|AAAAAAAAGCPGAAAA|628|Railroad |Circle|Suite 450|Shiloh|Monterey County|CA|99275|United States|-8|condo| +28455|AAAAAAAAHCPGAAAA|550|Mill |Dr.|Suite N|Red Hill|Colusa County|CA|94338|United States|-8|apartment| +28456|AAAAAAAAICPGAAAA|238|5th |RD|Suite 60|Union Hill|Lee County|TX|77746|United States|-6|apartment| +28457|AAAAAAAAJCPGAAAA|31|Wilson |Drive||||UT||United States||apartment| +28458|AAAAAAAAKCPGAAAA|456|Maple |Cir.|Suite E|Ashland|Stevens County|WA|94244|United States|-8|condo| +28459|AAAAAAAALCPGAAAA|997|Third |RD|Suite 300|Midway|Henry County|VA|21904|United States|-5|apartment| +28460|AAAAAAAAMCPGAAAA|321|Church Adams|Cir.|Suite 260|Midway|Rankin County|MS|51904|United States|-6|single family| +28461|AAAAAAAANCPGAAAA|754|Adams 5th|Ave|Suite 410|Macedonia|Cameron County|TX|71087|United States|-6|condo| +28462|AAAAAAAAOCPGAAAA|149|4th Wilson|Circle|Suite R|Jackson|Desha County|AR|79583|United States|-6|condo| +28463|AAAAAAAAPCPGAAAA|345|Chestnut |ST|Suite 50|Mount Olive|Grundy County|TN|38059|United States|-5|apartment| +28464|AAAAAAAAADPGAAAA|632|6th Ridge|RD|Suite S|Glenwood|Gem County|ID|83511|United States|-7|single family| +28465|AAAAAAAABDPGAAAA|124|Chestnut 8th|Ct.|Suite Y|Spring Hill|Navarro County|TX|76787|United States|-6|condo| +28466|AAAAAAAACDPGAAAA|730|2nd |Road|Suite 440|Oak Grove|Lincoln County|NM|88370|United States|-7|apartment| +28467|AAAAAAAADDPGAAAA|105|Mill 3rd|Pkwy|Suite 270|Providence|Las Animas County|CO|86614|United States|-7|apartment| +28468|AAAAAAAAEDPGAAAA|298|Chestnut |Ln|Suite 460|Philadelphia|Grand Isle County|VT|06191|United States|-5|apartment| +28469|AAAAAAAAFDPGAAAA|931|Ridge Washington|Lane|Suite U|Sumner|Faulk County|SD|50519|United States|-6|apartment| +28470|AAAAAAAAGDPGAAAA|30|North Adams|RD|Suite O|Buena Vista|Highland County|VA|25752|United States|-5|apartment| +28471|AAAAAAAAHDPGAAAA|689|Church |Circle|Suite 100|Shiloh|Polk County|WI|59275|United States|-6|apartment| +28472|AAAAAAAAIDPGAAAA|31|Second Hill|Court|Suite 450|Pleasant Grove|Upshur County|TX|74136|United States|-6|apartment| +28473|AAAAAAAAJDPGAAAA|725|Lake Broadway|Street|Suite M|Glenwood|Garza County|TX|73511|United States|-6|single family| +28474|AAAAAAAAKDPGAAAA|422|Church |Cir.|Suite 210|Price|Waller County|TX|73824|United States|-6|condo| +28475|AAAAAAAALDPGAAAA|538|North Dogwood|RD|Suite B|Five Points|Clarion County|PA|16098|United States|-5|single family| +28476|AAAAAAAAMDPGAAAA|221|6th |Wy|Suite 370|Red Hill|Wilcox County|GA|34338|United States|-5|single family| +28477|AAAAAAAANDPGAAAA|236|Cherry Third|Cir.|Suite 300|Red Hill|Taney County|MO|64338|United States|-6|single family| +28478|AAAAAAAAODPGAAAA|703|Broadway Chestnut|Boulevard|Suite X|Guilford|Lyon County|KS|64408|United States|-6|apartment| +28479|AAAAAAAAPDPGAAAA|26|Railroad |Ave|Suite T|Bunker Hill|Bedford County|TN|30150|United States|-5|condo| +28480|AAAAAAAAAEPGAAAA|88|6th |Cir.|Suite 420|Greenfield|Macomb County|MI|45038|United States|-5|apartment| +28481|AAAAAAAABEPGAAAA|940|Hill Wilson|Circle|Suite O|Spring Valley|Shelby County|OH|46060|United States|-5|single family| +28482|AAAAAAAACEPGAAAA|51|Highland |Way|Suite W|Walnut Grove|Fillmore County|MN|57752|United States|-6|apartment| +28483|AAAAAAAADEPGAAAA|715|Washington |Dr.|Suite A|Valley View|Washington County|IN|45124|United States|-5|condo| +28484|AAAAAAAAEEPGAAAA|472|North |Dr.|Suite H|Bunker Hill|Rogers County|OK|70150|United States|-6|single family| +28485|AAAAAAAAFEPGAAAA|117|Spring Second|Circle|Suite T|Newtown|Rock Island County|IL|61749|United States|-6|apartment| +28486|AAAAAAAAGEPGAAAA|831|Elm |Dr.|Suite 310|Mount Pleasant|Gordon County|GA|31933|United States|-5|condo| +28487|AAAAAAAAHEPGAAAA|119|Maple |Drive|Suite 270|Ashland|Galveston County|TX|74244|United States|-6|single family| +28488|AAAAAAAAIEPGAAAA|578|4th |Drive|Suite 280|Crossroads|Daggett County|UT|80534|United States|-7|apartment| +28489|AAAAAAAAJEPGAAAA|666|Dogwood Lee|Ln|Suite S|Frogtown|Cameron Parish|LA|78784|United States|-6|condo| +28490|AAAAAAAAKEPGAAAA|98|4th |ST|Suite S|Walnut Grove|Gregg County|TX|77752|United States|-6|apartment| +28491|AAAAAAAALEPGAAAA|63|2nd Ridge|Ave|Suite S|Summit|Murray County|GA|30499|United States|-5|single family| +28492|AAAAAAAAMEPGAAAA|460|South River|Boulevard|Suite 490|Shady Grove|Howard County|MD|22812|United States|-5|apartment| +28493|AAAAAAAANEPGAAAA|800|2nd Center|Ct.|Suite 100|New Hope|Hamilton County|NE|69431|United States|-6|single family| +28494|AAAAAAAAOEPGAAAA|158|Second Oak|RD|Suite 270|Globe|Stone County|AR|79614|United States|-6|condo| +28495|AAAAAAAAPEPGAAAA|610|Jackson |Way|Suite F|Buena Vista|Lincoln County|MN|55752|United States|-6|single family| +28496|AAAAAAAAAFPGAAAA|40|Maple |Parkway|Suite 160|Salem|East Carroll Parish|LA|78048|United States|-6|condo| +28497|AAAAAAAABFPGAAAA|671|Washington 1st|Ct.|Suite 80|Summit|Mountrail County|ND|50499|United States|-6|condo| +28498|AAAAAAAACFPGAAAA|418|15th |Drive|Suite K|Maple Grove|Austin County|TX|78252|United States|-6|apartment| +28499|AAAAAAAADFPGAAAA|118||Street||Spring Valley||IA|56060|United States|-6|condo| +28500|AAAAAAAAEFPGAAAA|167|4th |Parkway|Suite R|Liberty|La Paz County|AZ|83451|United States|-7|condo| +28501|AAAAAAAAFFPGAAAA|978|Ridge Broadway|Wy|Suite 80|Pleasant Valley|Webster County|IA|52477|United States|-6|single family| +28502|AAAAAAAAGFPGAAAA|1000|Hillcrest |RD|Suite V|Hawthorne|Branch County|MI|49959|United States|-5|single family| +28503|AAAAAAAAHFPGAAAA|427|Hillcrest |Ave|Suite 440|Pleasant Grove|Warren County|NY|14136|United States|-5|condo| +28504|AAAAAAAAIFPGAAAA|435|River Elm|Drive|Suite Q|Utica|Crenshaw County|AL|33852|United States|-6|single family| +28505|AAAAAAAAJFPGAAAA|326|7th |Circle|Suite I|Oak Hill|Bennington County|VT|08438|United States|-5|apartment| +28506|AAAAAAAAKFPGAAAA|896|7th |Road|Suite 0|Wildwood|Sequoyah County|OK|76871|United States|-6|apartment| +28507|AAAAAAAALFPGAAAA|26|14th Willow|Ave|Suite 200|Cedar Grove|Clinton County|MI|40411|United States|-5|apartment| +28508|AAAAAAAAMFPGAAAA|223|Church East|Cir.|Suite E|Deerfield|Salem County|NJ|09840|United States|-5|single family| +28509|AAAAAAAANFPGAAAA|446|Hill Jefferson|Wy|Suite 290|Mountain View|Hempstead County|AR|74466|United States|-6|apartment| +28510|AAAAAAAAOFPGAAAA|534|Fourth Railroad|Court|Suite 270|Aberdeen|Story County|IA|57995|United States|-6|apartment| +28511|AAAAAAAAPFPGAAAA|992|Valley |Lane|Suite 60|Clinton|Schuyler County|NY|18222|United States|-5|single family| +28512|AAAAAAAAAGPGAAAA||||Suite Y||Taylor County||58054|United States||single family| +28513|AAAAAAAABGPGAAAA|485|15th Davis|Circle|Suite 390|Farmington|Houston County|TX|79145|United States|-6|single family| +28514|AAAAAAAACGPGAAAA|716|Sunset 2nd|Ln|Suite 260|Valley View|Pearl River County|MS|55124|United States|-6|apartment| +28515|AAAAAAAADGPGAAAA|70|Walnut Pine|Street|Suite 250|Jamestown|San Augustine County|TX|76867|United States|-6|single family| +28516|AAAAAAAAEGPGAAAA|649|Lake 8th|ST|Suite U|Marion|Platte County|WY|80399|United States|-7|apartment| +28517|AAAAAAAAFGPGAAAA|301|4th |Pkwy|Suite 210|Buena Vista|Deer Lodge County|MT|65752|United States|-7|apartment| +28518|AAAAAAAAGGPGAAAA|559|5th Lake|Court|Suite 360|Belmont|Madison County|OH|40191|United States|-5|single family| +28519|AAAAAAAAHGPGAAAA|68|Birch |Ave|Suite 190|Pleasant Grove|Dickinson County|IA|54136|United States|-6|apartment| +28520|AAAAAAAAIGPGAAAA|650|South |Lane|Suite 130|Maple Grove|Robertson County|TN|38252|United States|-6|apartment| +28521|AAAAAAAAJGPGAAAA|873|River North|Ln|Suite C|Glendale|Walker County|GA|33951|United States|-5|condo| +28522|AAAAAAAAKGPGAAAA|331|Dogwood |Boulevard|Suite X|Green Acres|Iberville Parish|LA|77683|United States|-6|apartment| +28523|AAAAAAAALGPGAAAA|||||Riverdale|Moore County|||||| +28524|AAAAAAAAMGPGAAAA|120|Sunset 9th|Wy|Suite 40|Highland Park|Kendall County|TX|76534|United States|-6|single family| +28525|AAAAAAAANGPGAAAA|585|Chestnut Jefferson|Road|Suite J|Glendale|Clinton County|NY|13951|United States|-5|condo| +28526|AAAAAAAAOGPGAAAA|501|Davis |Lane|Suite X|Post Oak|Nacogdoches County|TX|78567|United States|-6|condo| +28527|AAAAAAAAPGPGAAAA|29|1st Madison|Ave|Suite 420|Waterloo|Jefferson County|WA|91675|United States|-8|condo| +28528|AAAAAAAAAHPGAAAA|746|Main Hill|RD|Suite F|Pleasant Grove|Polk County|AR|74136|United States|-6|single family| +28529|AAAAAAAABHPGAAAA|546|Franklin Lincoln|Blvd|Suite J|Salem|Stafford County|VA|28048|United States|-5|apartment| +28530|AAAAAAAACHPGAAAA|109|7th |Dr.|Suite 450|Glenwood|Wilson County|TN|33511|United States|-5|condo| +28531|AAAAAAAADHPGAAAA|966|Locust |Ln|Suite E|Harmony|Saginaw County|MI|45804|United States|-5|condo| +28532|AAAAAAAAEHPGAAAA|457|12th North|Ave|Suite 450|Hopewell|Claiborne Parish|LA|70587|United States|-6|apartment| +28533|AAAAAAAAFHPGAAAA|909|Laurel Lake|Boulevard|Suite A|Mount Zion|Claiborne County|MS|58054|United States|-6|single family| +28534|AAAAAAAAGHPGAAAA|743|Meadow Sixth|Lane|Suite N|Proctor|Shackelford County|TX|78140|United States|-6|single family| +28535|AAAAAAAAHHPGAAAA|106|Central Meadow|Court|Suite 140|Rankin|Franklin County|AR|72621|United States|-6|single family| +28536|AAAAAAAAIHPGAAAA|409|Lake Seventh|Street|Suite I|Newport|Osage County|OK|71521|United States|-6|single family| +28537|AAAAAAAAJHPGAAAA|993|Oak Hill|Parkway|Suite Q|Mount Pleasant|Florence County|SC|21933|United States|-5|apartment| +28538|AAAAAAAAKHPGAAAA|449|Church Highland|ST|Suite 210|Pine Grove|Amelia County|VA|24593|United States|-5|apartment| +28539|AAAAAAAALHPGAAAA|949|Hillcrest Wilson|Pkwy|Suite C|Shiloh|Morrison County|MN|59275|United States|-6|single family| +28540|AAAAAAAAMHPGAAAA|419|College Oak|Boulevard|Suite R|Green Acres|Comanche County|KS|67683|United States|-6|apartment| +28541|AAAAAAAANHPGAAAA|196|Oak Second|Drive|Suite R|Wilson|Wythe County|VA|26971|United States|-5|apartment| +28542|AAAAAAAAOHPGAAAA|428|Hillcrest East|Parkway|Suite 260|Spring Valley|Morgan County|OH|46060|United States|-5|condo| +28543|AAAAAAAAPHPGAAAA|888|Park |Cir.|Suite V|Bunker Hill|Franklin County|MS|50150|United States|-6|apartment| +28544|AAAAAAAAAIPGAAAA|350|1st |RD|Suite 320|Lakeside|Alcorn County|MS|59532|United States|-6|condo| +28545|AAAAAAAABIPGAAAA|711|Lee |Drive|Suite T|White Oak|Ocean County|NJ|07268|United States|-5|apartment| +28546|AAAAAAAACIPGAAAA|778|Pine |Street|Suite Y|Clinton|Crawford County|MI|48222|United States|-5|condo| +28547|AAAAAAAADIPGAAAA|70|Third Elm|Boulevard|Suite 380|Fairview|Burke County|ND|55709|United States|-6|condo| +28548|AAAAAAAAEIPGAAAA|272|Highland |Ct.|Suite 320|Greenville|Wilkinson County|GA|31387|United States|-5|apartment| +28549|AAAAAAAAFIPGAAAA|615|5th Broadway|Ln|Suite A|Somerville|Osage County|OK|77783|United States|-6|single family| +28550|AAAAAAAAGIPGAAAA|||RD||||OK||United States||| +28551|AAAAAAAAHIPGAAAA|256|Hickory Park|Court|Suite O|Jackson|Jennings County|IN|49583|United States|-5|apartment| +28552|AAAAAAAAIIPGAAAA|829|Woodland Second|Lane|Suite G|Plainview|Clark County|AR|73683|United States|-6|single family| +28553|AAAAAAAAJIPGAAAA|104|6th |Parkway|Suite O|Edgewater|Fayette County|TN|30635|United States|-5|condo| +28554|AAAAAAAAKIPGAAAA|284|Hill Smith|Circle|Suite I|Union Hill|Barton County|KS|67746|United States|-6|apartment| +28555|AAAAAAAALIPGAAAA|543|Center |Ave|Suite 420|Page|Ashe County|NC|20296|United States|-5|apartment| +28556|AAAAAAAAMIPGAAAA|41|Second |Circle|Suite B|Woodland|Arkansas County|AR|74854|United States|-6|single family| +28557|AAAAAAAANIPGAAAA|368|Railroad |Dr.|Suite 160|Buena Vista|Maricopa County|AZ|85752|United States|-7|single family| +28558|AAAAAAAAOIPGAAAA||South ||Suite 170||||59843|||condo| +28559|AAAAAAAAPIPGAAAA|943|Washington Railroad|Ct.|Suite 190|Shady Grove|Clark County|IN|42812|United States|-5|condo| +28560|AAAAAAAAAJPGAAAA|264|Madison |Ave|Suite L|Highland|Russell County|AL|39454|United States|-6|condo| +28561|AAAAAAAABJPGAAAA|615|Maple |Pkwy|Suite 120|Appleton|Floyd County|GA|34240|United States|-5|apartment| +28562|AAAAAAAACJPGAAAA|214|11th Third|Lane|Suite H|Mount Vernon|Indiana County|PA|18482|United States|-5|single family| +28563|AAAAAAAADJPGAAAA|867|Cherry Church|Parkway|Suite C|Lakeside|Rooks County|KS|69532|United States|-6|apartment| +28564|AAAAAAAAEJPGAAAA|271|View 1st|Court|Suite 60|Centerville|Orange County|VT|00659|United States|-5|condo| +28565|AAAAAAAAFJPGAAAA|761|Maple |Road|Suite 190|Jamestown|Calhoun County|GA|36867|United States|-5|condo| +28566|AAAAAAAAGJPGAAAA|848|River |RD|Suite 50|Walnut Grove|Lawrence County|MS|57752|United States|-6|condo| +28567|AAAAAAAAHJPGAAAA|553|West Jackson|ST|Suite 270|Union|Navajo County|AZ|88721|United States|-7|single family| +28568|AAAAAAAAIJPGAAAA|440|Pine |Blvd|Suite 370|Summit|Vernon County|MO|60499|United States|-6|condo| +28569|AAAAAAAAJJPGAAAA|882|Third |Way|Suite 180|Glenwood|Holmes County|FL|33511|United States|-5|condo| +28570|AAAAAAAAKJPGAAAA|76|Davis |Blvd|Suite 250|Greenwood|Pierce County|NE|68828|United States|-7|apartment| +28571|AAAAAAAALJPGAAAA|586|Woodland Park|Cir.|Suite U|Antioch|Giles County|VA|28605|United States|-5|condo| +28572|AAAAAAAAMJPGAAAA|690|Hickory Hickory|Boulevard|Suite 140|New Hope|Webster County|WV|29431|United States|-5|single family| +28573|AAAAAAAANJPGAAAA|664|Jackson Washington|Blvd|Suite A|Wildwood|Polk County|TX|76871|United States|-6|apartment| +28574|AAAAAAAAOJPGAAAA|886|12th |Ln|Suite L|Walnut|Sussex County|DE|16245|United States|-5|apartment| +28575|AAAAAAAAPJPGAAAA|512|Hill |Road|Suite 440|New Hope|Josephine County|OR|99431|United States|-8|condo| +28576|AAAAAAAAAKPGAAAA|386|Mill Park|RD|Suite 300|Wildwood|Jim Hogg County|TX|76871|United States|-6|condo| +28577|AAAAAAAABKPGAAAA|552|Valley |Way|Suite K|Pleasant Grove|Ventura County|CA|94136|United States|-8|condo| +28578|AAAAAAAACKPGAAAA|279|Birch |Road|Suite D|Lincoln|Anderson County|KY|41289|United States|-6|single family| +28579|AAAAAAAADKPGAAAA|588|Williams 4th|Cir.|Suite 290|Bath|Orange County|VT|01173|United States|-5|apartment| +28580|AAAAAAAAEKPGAAAA|770|Main |Blvd|Suite 290|Walnut Grove|Charlotte County|FL|37752|United States|-5|condo| +28581|AAAAAAAAFKPGAAAA|188|Hill |Parkway|Suite 300|Woodlawn|Washington County|FL|34098|United States|-5|apartment| +28582|AAAAAAAAGKPGAAAA|657|Franklin Seventh|Cir.|Suite F|Deerfield|Coos County|NH|09840|United States|-5|condo| +28583|AAAAAAAAHKPGAAAA|517|Highland |Blvd|Suite I|New Hope|Jackson County|TX|79431|United States|-6|apartment| +28584|AAAAAAAAIKPGAAAA|965|10th |Wy|Suite W|Clinton|Banner County|NE|68222|United States|-6|apartment| +28585|AAAAAAAAJKPGAAAA|125|Mill 3rd|Blvd|||Franklin County||54338||-6|single family| +28586|AAAAAAAAKKPGAAAA|51|Laurel Maple|Lane|Suite 180|Valley View|Williamson County|TX|75124|United States|-6|single family| +28587|AAAAAAAALKPGAAAA|879|Center |Wy|Suite T|Clearview|Choctaw County|OK|75495|United States|-6|single family| +28588|AAAAAAAAMKPGAAAA|219|Chestnut Spring|Way|Suite 400|New Hope|Bacon County|GA|39431|United States|-5|apartment| +28589|AAAAAAAANKPGAAAA|168|East 1st|Cir.|Suite V|Greenfield|Colonial Heights city|VA|25038|United States|-5|apartment| +28590|AAAAAAAAOKPGAAAA|936|Walnut |Street|Suite 220|Mount Olive|Morton County|ND|58059|United States|-6|condo| +28591|AAAAAAAAPKPGAAAA|787|1st |Drive|Suite 280|Sulphur Springs|Slope County|ND|58354|United States|-6|condo| +28592|AAAAAAAAALPGAAAA|692|Main Smith|Parkway|Suite 40|Buena Vista|Caldwell County|KY|45752|United States|-6|apartment| +28593|AAAAAAAABLPGAAAA|842|College Wilson|Lane|Suite 130|Mount Vernon|Grundy County|MO|68482|United States|-6|single family| +28594|AAAAAAAACLPGAAAA|684|5th |Way|Suite E|Mount Zion|Marion County|MO|68054|United States|-6|condo| +28595|AAAAAAAADLPGAAAA|467|Cherry |Lane|Suite P|Greenville|Carroll County|IA|51387|United States|-6|condo| +28596|AAAAAAAAELPGAAAA|611|Forest |||Pine Valley||TX|78209|United States|-6|single family| +28597|AAAAAAAAFLPGAAAA|953|Valley |Avenue|Suite 110|Summit|Sitka Borough|AK|90499|United States|-9|single family| +28598|AAAAAAAAGLPGAAAA|809|Third Smith|Wy|Suite 200|Union|Lincoln County|SD|58721|United States|-7|single family| +28599|AAAAAAAAHLPGAAAA|340|Jackson View|Blvd|Suite V|Walnut Grove|Belknap County|NH|08352|United States|-5|condo| +28600|AAAAAAAAILPGAAAA|307|Railroad 11th|Street|Suite 250|Oak Hill|Teton County|ID|87838|United States|-7|single family| +28601|AAAAAAAAJLPGAAAA|904|Sunset |Blvd|Suite W|Clifton|Suffolk County|MA|08614|United States|-5|condo| +28602|AAAAAAAAKLPGAAAA|386|12th |ST|Suite 440|Hillcrest|Marion County|OH|43003|United States|-5|single family| +28603|AAAAAAAALLPGAAAA||Broadway |Avenue|Suite 20|||NE|||-7|| +28604|AAAAAAAAMLPGAAAA|989|Walnut |Wy|Suite V|Providence|Columbia County|GA|36614|United States|-5|condo| +28605|AAAAAAAANLPGAAAA|531|Third 7th|Street|Suite 140|Pine Grove|Marion County|MS|54593|United States|-6|apartment| +28606|AAAAAAAAOLPGAAAA|109|Woodland Woodland|Wy|Suite 400|Hubbard|Ozark County|MO|66291|United States|-6|apartment| +28607|AAAAAAAAPLPGAAAA|579|Johnson Ash|Way|Suite 110|Concord|Seminole County|GA|34107|United States|-5|condo| +28608|AAAAAAAAAMPGAAAA|821|Central |Boulevard|Suite 370|Macedonia|Nassau County|NY|11087|United States|-5|apartment| +28609|AAAAAAAABMPGAAAA|406|4th Miller|Road|Suite W|Plainview|Boone County|IA|53683|United States|-6|single family| +28610|AAAAAAAACMPGAAAA|244|Valley |RD|Suite 100|Lincoln|Guadalupe County|TX|71289|United States|-6|condo| +28611|AAAAAAAADMPGAAAA|929|Lincoln |Court|Suite 120|Glenwood|Jefferson County|AR|73511|United States|-6|condo| +28612|AAAAAAAAEMPGAAAA|||Circle||Edgewood|Rice County||50069|United States||| +28613|AAAAAAAAFMPGAAAA|772|Main Railroad|Road|Suite Q|Clinton|Guadalupe County|TX|78222|United States|-6|single family| +28614|AAAAAAAAGMPGAAAA|78|Twelfth |Lane|Suite 370|Oak Grove|Dakota County|MN|58370|United States|-6|condo| +28615|AAAAAAAAHMPGAAAA|657|9th |Circle|Suite 320|Woodlawn|Comal County|TX|74098|United States|-6|condo| +28616|AAAAAAAAIMPGAAAA|978|Hillcrest |Wy|Suite 420|Oakland|Madison County|NC|29843|United States|-5|condo| +28617|AAAAAAAAJMPGAAAA|981|Central Cedar|Circle|Suite 190|Buckingham|Skagit County|WA|94092|United States|-8|condo| +28618|AAAAAAAAKMPGAAAA|371|Church Adams|ST||New Hope||||United States|-5|condo| +28619|AAAAAAAALMPGAAAA|29|2nd |Ave|Suite 400|Green Acres|Marshall County|WV|27683|United States|-5|condo| +28620|AAAAAAAAMMPGAAAA|426|Pine ||Suite 490|Waterloo|Fairfax city||21675|||| +28621|AAAAAAAANMPGAAAA|856|6th Washington|Drive|Suite 470|Newport|Sedgwick County|KS|61521|United States|-6|apartment| +28622|AAAAAAAAOMPGAAAA|154|Center Franklin|Road|Suite A|Mount Vernon|Monroe County|FL|38482|United States|-5|apartment| +28623|AAAAAAAAPMPGAAAA|584|North Forest|Court|Suite 70|Oak Ridge|Stonewall County|TX|78371|United States|-6|apartment| +28624|AAAAAAAAANPGAAAA|425|First Spruce|RD|Suite J|Cedar Grove|Izard County|AR|70411|United States|-6|apartment| +28625|AAAAAAAABNPGAAAA|184|12th |Ct.|Suite M|Mount Pleasant|Sumter County|GA|31933|United States|-5|condo| +28626|AAAAAAAACNPGAAAA|886||Ave||||||United States|-6|| +28627|AAAAAAAADNPGAAAA|494|Meadow |Road|Suite S|Clinton|Hampden County|MA|08822|United States|-5|condo| +28628|AAAAAAAAENPGAAAA|864|Sycamore |Drive|Suite 480|Highland Park|Tipton County|TN|36534|United States|-6|condo| +28629|AAAAAAAAFNPGAAAA|226|Forest 9th|Parkway|Suite 410|Farmington|Burke County|ND|59145|United States|-6|condo| +28630|AAAAAAAAGNPGAAAA|573|Spruce Mill|Ln|Suite C|White Oak|King County|WA|96668|United States|-8|condo| +28631|AAAAAAAAHNPGAAAA|693|East |Street|Suite 250|Leland|Powder River County|MT|69452|United States|-7|condo| +28632|AAAAAAAAINPGAAAA|531|Sunset |Drive|Suite D|Valley View|Alpine County|CA|95124|United States|-8|single family| +28633|AAAAAAAAJNPGAAAA|455|11th |Ave|Suite 120|Spring Valley|Warren County|VA|26060|United States|-5|condo| +28634|AAAAAAAAKNPGAAAA|945|Park 5th|Way|Suite 50|Highland Park|Dixon County|NE|66534|United States|-6|apartment| +28635|AAAAAAAALNPGAAAA|139|14th |Road|Suite I|Glendale|Green County|KY|43951|United States|-6|apartment| +28636|AAAAAAAAMNPGAAAA|811|Main 12th|Pkwy|Suite 390|Hillcrest|Morgan County|WV|23003|United States|-5|apartment| +28637|AAAAAAAANNPGAAAA|213|Washington |Pkwy|Suite X|Edgewood|Republic County|KS|60069|United States|-6|condo| +28638|AAAAAAAAONPGAAAA|48|Meadow |Boulevard|Suite C|Woodville|West Baton Rouge Parish|LA|74289|United States|-6|single family| +28639|AAAAAAAAPNPGAAAA|667|View Lakeview|Blvd|Suite V|Little River|Power County|ID|80319|United States|-7|single family| +28640|AAAAAAAAAOPGAAAA|422|Pine 3rd|Cir.|Suite 170|Crossroads|Berrien County|GA|30534|United States|-5|single family| +28641|AAAAAAAABOPGAAAA|640|Madison |Street|Suite G|Arlington|Lamar County|MS|56557|United States|-6|apartment| +28642|AAAAAAAACOPGAAAA|701|2nd Ridge|Cir.|Suite T|Oakwood|Patrick County|VA|20169|United States|-5|apartment| +28643|AAAAAAAADOPGAAAA|214|Maple |Dr.|Suite 280|Springfield|DeWitt County|TX|79303|United States|-6|condo| +28644|AAAAAAAAEOPGAAAA|223|Cedar |Ave|Suite U|Greenville|Marion County|AR|71387|United States|-6|condo| +28645|AAAAAAAAFOPGAAAA|695|Second |Street|Suite V|Highland|Monmouth County|NJ|09454|United States|-5|single family| +28646|AAAAAAAAGOPGAAAA|654|Lake |Ln|Suite 200|Harmony|Lafayette County|AR|75804|United States|-6|single family| +28647|AAAAAAAAHOPGAAAA|598|First Lee|Ave|Suite G|Sherman|Scioto County|OH|47563|United States|-5|apartment| +28648|AAAAAAAAIOPGAAAA|506|Willow |Street|Suite I|Enterprise|Tompkins County|NY|11757|United States|-5|condo| +28649|AAAAAAAAJOPGAAAA|843|14th |Court|Suite 440|Brookville|King and Queen County|VA|23524|United States|-5|apartment| +28650|AAAAAAAAKOPGAAAA|999|1st Ash|Pkwy|Suite 220|Greenville|Dubuque County|IA|51387|United States|-6|apartment| +28651|AAAAAAAALOPGAAAA||Park |ST|Suite 10|Union|Benton County||58721||-6|single family| +28652|AAAAAAAAMOPGAAAA|839|Lincoln Jackson|Dr.|Suite O|Florence|Pulaski County|KY|43394|United States|-5|condo| +28653|AAAAAAAANOPGAAAA|421|South 12th|Pkwy|Suite 460|Oak Ridge|Lauderdale County|AL|38371|United States|-6|single family| +28654|AAAAAAAAOOPGAAAA|603|East |Avenue|Suite B|Morris|Stephens County|GA|36696|United States|-5|condo| +28655|AAAAAAAAPOPGAAAA|558|Park |Ln|Suite M|Friendship|Northumberland County|PA|14536|United States|-5|single family| +28656|AAAAAAAAAPPGAAAA|996|Main |Wy|Suite 370|Red Hill|Will County|IL|64338|United States|-6|condo| +28657|AAAAAAAABPPGAAAA|690|Elm |Dr.|Suite 90|Fairfield|Neshoba County|MS|56192|United States|-6|condo| +28658|AAAAAAAACPPGAAAA|642|Maple Main|Pkwy|Suite 350|Riverside|Panola County|TX|79231|United States|-6|single family| +28659|AAAAAAAADPPGAAAA|903|Hill Railroad|Dr.|Suite H|Wilton|Yakima County|WA|96997|United States|-8|condo| +28660|AAAAAAAAEPPGAAAA|877|9th |Way|Suite E|Wildwood|Clayton County|GA|36871|United States|-5|condo| +28661|AAAAAAAAFPPGAAAA|163|Willow |Ct.|Suite L|Greenwood|Kenton County|KY|48828|United States|-5|single family| +28662|AAAAAAAAGPPGAAAA|741|East Pine|ST|Suite 440|Fairview|Knott County|KY|45709|United States|-5|condo| +28663|AAAAAAAAHPPGAAAA|412|7th |Avenue|Suite S|Belmont|Dane County|WI|50191|United States|-6|condo| +28664|AAAAAAAAIPPGAAAA|477|Smith |Boulevard|Suite N|Buena Vista|Nome Census Area|AK|95752|United States|-9|condo| +28665|AAAAAAAAJPPGAAAA|666|First West|Drive|Suite Q|Valley View|Dutchess County|NY|15124|United States|-5|apartment| +28666|AAAAAAAAKPPGAAAA|643|15th Smith|Circle|Suite L|Greenfield|Caldwell County|MO|65038|United States|-6|apartment| +28667|AAAAAAAALPPGAAAA|802|West |Blvd|Suite L|Union Hill|Upton County|TX|77746|United States|-6|single family| +28668|AAAAAAAAMPPGAAAA|765|Willow Oak|Drive|Suite X|Greenfield|Somerset County|PA|15038|United States|-5|single family| +28669|AAAAAAAANPPGAAAA|931|South South|Blvd|Suite 340|Mount Zion|Cedar County|MO|68054|United States|-6|apartment| +28670|AAAAAAAAOPPGAAAA|642|Lake |Cir.|Suite 390|Clinton|Pickett County|TN|38222|United States|-6|apartment| +28671|AAAAAAAAPPPGAAAA|791|Railroad |Court|Suite 380|Riverside|Tom Green County|TX|79231|United States|-6|single family| +28672|AAAAAAAAAAAHAAAA|830|Oak North|Ave|Suite C|Newtown|Yolo County|CA|91749|United States|-8|apartment| +28673|AAAAAAAABAAHAAAA|131|Church Sunset|ST|Suite G|Newtown|Washington County|MS|51749|United States|-6|single family| +28674|AAAAAAAACAAHAAAA|846|10th |Boulevard|Suite 480|Guilford|Plymouth County|MA|05008|United States|-5|apartment| +28675|AAAAAAAADAAHAAAA|494|Main |Road|Suite V|Greenville|Cottonwood County|MN|51387|United States|-6|apartment| +28676|AAAAAAAAEAAHAAAA|420|Seventh |Blvd|Suite G|Fairview|Jackson County|CO|85709|United States|-7|apartment| +28677|AAAAAAAAFAAHAAAA|813|Wilson Lake|RD|Suite X|Valley View|Yakutat Borough|AK|95124|United States|-9|single family| +28678|AAAAAAAAGAAHAAAA|788|Fifth |Way|Suite 380|Oakland|Kenosha County|WI|59843|United States|-6|condo| +28679|AAAAAAAAHAAHAAAA|714|Railroad Madison|Boulevard|Suite K|Clinton|Sierra County|CA|98222|United States|-8|condo| +28680|AAAAAAAAIAAHAAAA|537|Miller |Street|Suite B|Edgewood|Rogers County|OK|70069|United States|-6|apartment| +28681|AAAAAAAAJAAHAAAA|771|Lake |Avenue|Suite 360|Woodland|Huerfano County|CO|84854|United States|-7|apartment| +28682|AAAAAAAAKAAHAAAA|952|Lee Green|Ln|Suite 60|Pleasant Grove|Brooks County|GA|34136|United States|-5|apartment| +28683|AAAAAAAALAAHAAAA|808|Elm 6th|ST|Suite 60|Buena Vista|Dade County|MO|65752|United States|-6|apartment| +28684|AAAAAAAAMAAHAAAA|851|West Hill|RD|Suite 330|Shiloh|Bristol County|RI|09875|United States|-5|condo| +28685|AAAAAAAANAAHAAAA|767|Franklin |Ln|Suite H|Greenville|Portsmouth city|VA|21387|United States|-5|single family| +28686|AAAAAAAAOAAHAAAA|446|Poplar Third|Pkwy|Suite 460|Pleasant Hill|Eau Claire County|WI|53604|United States|-6|single family| +28687|AAAAAAAAPAAHAAAA|416|Locust 8th|ST|Suite 200|Arlington|Berkeley County|SC|26557|United States|-5|condo| +28688|AAAAAAAAABAHAAAA|47|Hickory |Boulevard|Suite 0|Shady Grove|Avoyelles Parish|LA|72812|United States|-6|single family| +28689|AAAAAAAABBAHAAAA|411|Hill |Cir.|Suite C|Green Acres|Van Buren County|IA|57683|United States|-6|apartment| +28690|AAAAAAAACBAHAAAA|800|Jefferson |Way|Suite 260|Newport|Stone County|MS|51521|United States|-6|condo| +28691|AAAAAAAADBAHAAAA|561|Mill 3rd|Road|Suite 240|Cuba|Aurora County|SD|50194|United States|-6|condo| +28692|AAAAAAAAEBAHAAAA|828|Elm |Avenue|Suite 450|Lakewood|Grady County|GA|38877|United States|-5|condo| +28693|AAAAAAAAFBAHAAAA|332|West |Road|Suite D|Oak Grove|Craighead County|AR|78370|United States|-6|apartment| +28694|AAAAAAAAGBAHAAAA|798|View |Pkwy|Suite G|Stringtown|Ward County|ND|50162|United States|-6|single family| +28695|AAAAAAAAHBAHAAAA|443|Main |Wy|Suite 50|Spring Hill|Coleman County|TX|76787|United States|-6|apartment| +28696|AAAAAAAAIBAHAAAA|259|View Center|Blvd|Suite 180|Concord|Cumberland County|VA|24107|United States|-5|apartment| +28697|AAAAAAAAJBAHAAAA|894|Hill |Boulevard|Suite 270|Enterprise|Sussex County|NJ|02357|United States|-5|apartment| +28698|AAAAAAAAKBAHAAAA|298|Sycamore |Parkway|Suite H|Greenville|Neosho County|KS|61387|United States|-6|apartment| +28699|AAAAAAAALBAHAAAA|74|Smith North|Court|Suite 110|Midway|Huron County|OH|41904|United States|-5|condo| +28700|AAAAAAAAMBAHAAAA|593|Third |Circle|Suite R|Jackson|Carroll County|KY|49583|United States|-6|condo| +28701|AAAAAAAANBAHAAAA|377|11th East|Pkwy|Suite S|Lakeview|Alpena County|MI|48579|United States|-5|apartment| +28702|AAAAAAAAOBAHAAAA|992|Pine Oak|Avenue|Suite H|Lakeside|Kittson County|MN|59532|United States|-6|single family| +28703|AAAAAAAAPBAHAAAA|400|14th |Pkwy|Suite L|Oak Grove|Worth County|IA|58370|United States|-6|single family| +28704|AAAAAAAAACAHAAAA|675|Broadway Park|Avenue|Suite X|Harmony|Ross County|OH|45804|United States|-5|apartment| +28705|AAAAAAAABCAHAAAA|37|6th Johnson|Court|Suite 470|Union|Madison County|IA|58721|United States|-6|apartment| +28706|AAAAAAAACCAHAAAA|401|13th 5th|Ave|Suite H|Liberty|Greer County|OK|73451|United States|-6|single family| +28707|AAAAAAAADCAHAAAA|14|4th Oak|Parkway|Suite 10|Freeman|Jefferson Davis Parish|LA|72297|United States|-6|apartment| +28708|AAAAAAAAECAHAAAA|559|View Ridge|Dr.|Suite S|Kingston|Dakota County|MN|54975|United States|-6|apartment| +28709|AAAAAAAAFCAHAAAA|304|Main Church|ST|Suite K|Flatwoods|Leslie County|KY|44212|United States|-5|condo| +28710|AAAAAAAAGCAHAAAA|437|Ridge Cedar|RD|Suite 330|Kingston|Wells County|IN|44975|United States|-5|apartment| +28711|AAAAAAAAHCAHAAAA|270|Sixth |Ln|Suite A|Antioch|Steuben County|NY|18605|United States|-5|condo| +28712|AAAAAAAAICAHAAAA|975|Walnut Davis|Ave|Suite R|Pine Grove|Treasure County|MT|64593|United States|-7|apartment| +28713|AAAAAAAAJCAHAAAA|173|Spring Second|Pkwy|Suite H|Floyd|Moore County|TN|33235|United States|-6|apartment| +28714|AAAAAAAAKCAHAAAA|568|Jefferson 6th|Pkwy|Suite D|Lancaster|Clearfield County|PA|10765|United States|-5|apartment| +28715|AAAAAAAALCAHAAAA|871|Elm |Way|Suite 220||||||-5|condo| +28716|AAAAAAAAMCAHAAAA|323|Park Third|Wy|Suite Y|Wayland|Washington County|VT|05715|United States|-5|apartment| +28717|AAAAAAAANCAHAAAA|190|13th |Ave|Suite 90|Jamestown|Chisago County|MN|56867|United States|-6|apartment| +28718|AAAAAAAAOCAHAAAA|29|Sunset Cherry|Drive|Suite 310|Enterprise|Dickenson County|VA|21757|United States|-5|apartment| +28719|AAAAAAAAPCAHAAAA|581|Spruce |Way|Suite 130|Indian Village|Botetourt County|VA|21075|United States|-5|apartment| +28720|AAAAAAAAADAHAAAA|451|Valley |Pkwy|Suite 20|Sullivan|Columbia County|WA|90451|United States|-8|apartment| +28721|AAAAAAAABDAHAAAA|110|Third |Way|Suite 20|Mountain View|Pike County|MS|54466|United States|-6|apartment| +28722|AAAAAAAACDAHAAAA|594|Center Laurel|Avenue|Suite 200|Sunnyside|Austin County|TX|71952|United States|-6|apartment| +28723|AAAAAAAADDAHAAAA|525|Oak |Ct.|Suite 40|Jackson|Wyoming County|NY|19583|United States|-5|single family| +28724|AAAAAAAAEDAHAAAA|989|Ash |Drive|Suite F|Highland|Washington Parish|LA|79454|United States|-6|apartment| +28725|AAAAAAAAFDAHAAAA|979|Lake |Ct.|Suite 440|Red Hill|Linn County|MO|64338|United States|-6|apartment| +28726|AAAAAAAAGDAHAAAA|293|Walnut |Dr.|Suite K|Peoria|Merced County|CA|99818|United States|-8|single family| +28727|AAAAAAAAHDAHAAAA|331|Highland |Boulevard|Suite O|Unionville|Franklin County|GA|31711|United States|-5|single family| +28728|AAAAAAAAIDAHAAAA|391|Hickory Meadow|Ln|Suite W|Sunnyside|Yellow Medicine County|MN|51952|United States|-6|condo| +28729|AAAAAAAAJDAHAAAA|773|Central Franklin|Ln|Suite 390|Plainview|Hardin County|OH|43683|United States|-5|single family| +28730|AAAAAAAAKDAHAAAA|16|Lee |Way|Suite A|Farmington|Adams County|PA|19145|United States|-5|apartment| +28731|AAAAAAAALDAHAAAA|569|Fifth |Dr.|Suite K|Oak Hill|Adams County|WI|57838|United States|-6|apartment| +28732|AAAAAAAAMDAHAAAA|614|Green Highland|Road|Suite G|Springfield|Staunton city|VA|29303|United States|-5|condo| +28733|AAAAAAAANDAHAAAA|243|Lake |Ln|Suite R|Marion|Hillsborough County|NH|00999|United States|-5|condo| +28734|AAAAAAAAODAHAAAA|374|Park Cedar|ST|Suite 460|Lincoln|Newton County|AR|71289|United States|-6|single family| +28735|AAAAAAAAPDAHAAAA|549|Green Madison|Ct.|Suite 130|Green Acres|Washington County|VT|08283|United States|-5|condo| +28736|AAAAAAAAAEAHAAAA|539|Pine |Cir.|Suite E|Brownsville|Inyo County|CA|99310|United States|-8|apartment| +28737|AAAAAAAABEAHAAAA|110|Pine |Wy|Suite Y|Woodville|Sanpete County|UT|84289|United States|-7|apartment| +28738|AAAAAAAACEAHAAAA|770|Maple Lincoln|Way|Suite W|Sunnyside|Chautauqua County|KS|61952|United States|-6|condo| +28739|AAAAAAAADEAHAAAA|143|Elm Pine|Street|Suite 400|Houston|Collingsworth County|TX|74072|United States|-6|condo| +28740|AAAAAAAAEEAHAAAA|884|Hill |Blvd|Suite 140|Bethel|Yancey County|NC|25281|United States|-5|single family| +28741|AAAAAAAAFEAHAAAA|378|Cherry |Ave|Suite 150|Georgetown|Bergen County|NJ|07657|United States|-5|condo| +28742|AAAAAAAAGEAHAAAA|543|Washington |Wy|Suite K|Sulphur Springs|Keith County|NE|68354|United States|-7|condo| +28743|AAAAAAAAHEAHAAAA|550|Park |Parkway|Suite U|Highland Park|Sutton County|TX|76534|United States|-6|condo| +28744|AAAAAAAAIEAHAAAA|454|5th |Lane|Suite 320|Cedar|Dubois County|IN|41229|United States|-5|apartment| +28745|AAAAAAAAJEAHAAAA|653|10th |Cir.|Suite D|Concord|Sedgwick County|CO|84107|United States|-7|apartment| +28746|AAAAAAAAKEAHAAAA|517|Park View|Way|Suite 380|Pleasant Valley|Montgomery County|MD|22477|United States|-5|single family| +28747|AAAAAAAALEAHAAAA|235|3rd 11th|Boulevard|Suite 470|Unionville|Cache County|UT|81711|United States|-7|apartment| +28748|AAAAAAAAMEAHAAAA|712|Maple |Street|Suite F|Oakwood|Bureau County|IL|60169|United States|-6|single family| +28749|AAAAAAAANEAHAAAA|217|College |Blvd|Suite B|Pleasant Valley||||United States|-6|apartment| +28750|AAAAAAAAOEAHAAAA|446|12th View|Cir.|Suite 160|Glendale|Coffee County|GA|33951|United States|-5|apartment| +28751|AAAAAAAAPEAHAAAA|81|Johnson |Way|Suite O|Highland|Rawlins County|KS|69454|United States|-6|single family| +28752|AAAAAAAAAFAHAAAA|180|Franklin |Ct.|Suite 280|Greenville|McLeod County|MN|51387|United States|-6|apartment| +28753|AAAAAAAABFAHAAAA|886|Birch |Street|Suite V|Allentown|Irion County|TX|71838|United States|-6|single family| +28754|AAAAAAAACFAHAAAA|8|Birch Willow|Blvd|Suite 350|Kimball|Oktibbeha County|MS|53595|United States|-6|condo| +28755|AAAAAAAADFAHAAAA|||||Franklin|Crockett County||||-5|| +28756|AAAAAAAAEFAHAAAA|386|10th |Parkway|Suite 40|Oak Grove|Platte County|WY|88370|United States|-7|single family| +28757|AAAAAAAAFFAHAAAA|947|Johnson |Blvd|Suite G|Woodland|Mecklenburg County|VA|24854|United States|-5|single family| +28758|AAAAAAAAGFAHAAAA|724|East Railroad|Way|Suite I|Shady Grove|Floyd County|TX|72812|United States|-6|single family| +28759|AAAAAAAAHFAHAAAA|291|North |Wy|Suite 90|Cordova|Dolores County|CO|86938|United States|-7|condo| +28760|AAAAAAAAIFAHAAAA|156|7th |Circle|Suite D|Rosebud|Elk County|KS|62244|United States|-6|single family| +28761|AAAAAAAAJFAHAAAA|968|Hillcrest Maple|Ln|Suite 80|Greenwood|Sullivan County|NH|09428|United States|-5|single family| +28762|AAAAAAAAKFAHAAAA|121|West Hill|Parkway|Suite Y|Springdale|McIntosh County|OK|78883|United States|-6|single family| +28763|AAAAAAAALFAHAAAA|172|Church Third|Lane|Suite V|Plainview|Antelope County|NE|63683|United States|-6|apartment| +28764|AAAAAAAAMFAHAAAA|73|5th Wilson|Boulevard|Suite L|Springhill|Lincoln County|ID|84602|United States|-7|condo| +28765|AAAAAAAANFAHAAAA|601|West |Ct.|Suite 390|Lakeview|Northumberland County|VA|28579|United States|-5|single family| +28766|AAAAAAAAOFAHAAAA|859|Madison Green|Ln|Suite G|Spring Hill|Zapata County|TX|76787|United States|-6|apartment| +28767|AAAAAAAAPFAHAAAA|524|Ridge |Road|Suite 310|Clifford|Ellis County|OK|78164|United States|-6|apartment| +28768|AAAAAAAAAGAHAAAA|374|Lincoln |Circle|Suite 340|Oak Hill|Ashland County|OH|47838|United States|-5|condo| +28769|AAAAAAAABGAHAAAA|162|Church |Lane|Suite Q|Oakland|Wayne County|NE|69843|United States|-7|single family| +28770|AAAAAAAACGAHAAAA|54|Cedar 13th|Ave|Suite R|Unionville|Spokane County|WA|91711|United States|-8|single family| +28771|AAAAAAAADGAHAAAA|817|Green |Parkway|Suite 140|Rockwood|Mercer County|PA|11545|United States|-5|single family| +28772|AAAAAAAAEGAHAAAA|564|Cedar |Circle|Suite 50|Stratford|Pittsylvania County|VA|26668|United States|-5|single family| +28773|AAAAAAAAFGAHAAAA|981|2nd Oak|Street|Suite 250|Oakland|Putnam County|IL|69843|United States|-6|apartment| +28774|AAAAAAAAGGAHAAAA|712|1st |Circle|Suite O|Bethel|Anoka County|MN|55281|United States|-6|single family| +28775|AAAAAAAAHGAHAAAA|570|Sixth 2nd|Court|Suite 110|Jamestown|Hidalgo County|TX|76867|United States|-6|single family| +28776|AAAAAAAAIGAHAAAA|370|4th |Wy|Suite 480|Jackson|Kinney County|TX|79583|United States|-6|single family| +28777|AAAAAAAAJGAHAAAA|969|Valley Forest|Parkway|Suite I|Fairfield|Westchester County|NY|16192|United States|-5|single family| +28778|AAAAAAAAKGAHAAAA|10|Wilson Fifth|Drive|Suite G|Mountain View|Fall River County|SD|54466|United States|-6|single family| +28779|AAAAAAAALGAHAAAA|993|2nd |Lane|Suite P|Tyrone|Howard County|TX|71201|United States|-6|single family| +28780|AAAAAAAAMGAHAAAA|845|Pine |Cir.|Suite S|Lincoln|Harrison County|OH|41289|United States|-5|condo| +28781|AAAAAAAANGAHAAAA|88|||Suite O|Hamilton||FL||United States|-5|single family| +28782|AAAAAAAAOGAHAAAA||Main Lakeview|Drive|||Park County||66594|||single family| +28783|AAAAAAAAPGAHAAAA|194|Washington |Wy|Suite R|Lebanon|Roscommon County|MI|42898|United States|-5|single family| +28784|AAAAAAAAAHAHAAAA|881|1st |Parkway|Suite O|Pine Grove|Cabell County|WV|24593|United States|-5|single family| +28785|AAAAAAAABHAHAAAA|324||Pkwy||Enterprise|Montour County|PA||United States|-5|| +28786|AAAAAAAACHAHAAAA|976|South 8th|Court|Suite H|Forest Hills|Warren County|MS|59237|United States|-6|apartment| +28787|AAAAAAAADHAHAAAA|963|2nd Cherry|Way|Suite Y|Macedonia|Parke County|IN|41087|United States|-5|apartment| +28788|AAAAAAAAEHAHAAAA|992|North |Dr.|Suite A|Hillcrest|Sumner County|TN|33003|United States|-6|apartment| +28789|AAAAAAAAFHAHAAAA|97|South |Road|Suite T|Union|Orange County|CA|98721|United States|-8|apartment| +28790|AAAAAAAAGHAHAAAA|576|Park Birch|RD|Suite D|Sunnyside|Union County|TN|31952|United States|-6|apartment| +28791|AAAAAAAAHHAHAAAA|84|Laurel Franklin|Ave|Suite 210|Hopewell|Washington County|NE|60587|United States|-7|apartment| +28792|AAAAAAAAIHAHAAAA|479|Jackson 11th|Court|Suite 430|Cedar Grove|Kent County|MI|40411|United States|-5|condo| +28793|AAAAAAAAJHAHAAAA|271|4th 4th|Ct.|Suite M|Highland Park|Washtenaw County|MI|46534|United States|-5|condo| +28794|AAAAAAAAKHAHAAAA|839|Cedar |Ave|Suite D|Oakwood|Kenosha County|WI|50169|United States|-6|single family| +28795|AAAAAAAALHAHAAAA|187|Poplar |Wy|Suite N|Glendale|Plumas County|CA|93951|United States|-8|apartment| +28796|AAAAAAAAMHAHAAAA|692|Williams Miller|Parkway|Suite 340|Centerville|McDonough County|IL|60059|United States|-6|single family| +28797|AAAAAAAANHAHAAAA|443|Jackson |Way|Suite T|Highland|Alcona County|MI|49454|United States|-5|condo| +28798|AAAAAAAAOHAHAAAA|117|Birch Sunset|Ln|Suite F|Farmington|Union County|GA|39145|United States|-5|single family| +28799|AAAAAAAAPHAHAAAA|302|Second |Avenue|Suite 200|Five Forks|Union County|OH|42293|United States|-5|apartment| +28800|AAAAAAAAAIAHAAAA|965|Lincoln |Lane|Suite H|Waterloo|Jackson Parish|LA|71675|United States|-6|single family| +28801|AAAAAAAABIAHAAAA|316|Smith |Dr.|Suite F|Spring Grove|Miller County|AR|76719|United States|-6|single family| +28802|AAAAAAAACIAHAAAA|101|6th |Avenue|Suite 450|Lakeside|Otter Tail County|MN|59532|United States|-6|apartment| +28803|AAAAAAAADIAHAAAA|535|12th |Circle|Suite P|Liberty|Pipestone County|MN|53451|United States|-6|apartment| +28804|AAAAAAAAEIAHAAAA|914|5th River|Cir.|Suite 260|Wildwood|Hartford County|CT|07471|United States|-5|apartment| +28805|AAAAAAAAFIAHAAAA|819|River Maple|Avenue|Suite 280|Webb|Carroll County|MS|50899|United States|-6|single family| +28806|AAAAAAAAGIAHAAAA|828|Meadow East|Boulevard|Suite 150|Jamestown|Crosby County|TX|76867|United States|-6|apartment| +28807|AAAAAAAAHIAHAAAA|957|12th |Wy|Suite 230|White Oak|Scott County|MS|56668|United States|-6|condo| +28808|AAAAAAAAIIAHAAAA|369|4th |ST|Suite E|Unionville|San Juan County|CO|81711|United States|-7|apartment| +28809|AAAAAAAAJIAHAAAA|682|Spruce |Street|Suite K|Farmington|Stevens County|MN|59145|United States|-6|condo| +28810|AAAAAAAAKIAHAAAA|109|Maple Willow|Way|Suite 400|Barnes|Jasper County|IN|43788|United States|-5|condo| +28811|AAAAAAAALIAHAAAA|919|Park Washington|Dr.|Suite 340|Belleville|Jefferson County|AL|32924|United States|-6|single family| +28812|AAAAAAAAMIAHAAAA|919|Hill |Ln|Suite Q|Mount Pleasant|Ouray County|CO|81933|United States|-7|single family| +28813|AAAAAAAANIAHAAAA|409|Hickory |Ave|Suite 330|Bridgeport|Clarion County|PA|15817|United States|-5|single family| +28814|AAAAAAAAOIAHAAAA|418|Green |Lane|Suite L|Belmont|Kanawha County|WV|20191|United States|-5|single family| +28815|AAAAAAAAPIAHAAAA|608|Poplar Washington|Parkway|Suite 110|Spring Valley|Cannon County|TN|36060|United States|-5|condo| +28816|AAAAAAAAAJAHAAAA|38|Smith Johnson|Lane|Suite U|Pleasant Hill|Laurens County|GA|33604|United States|-5|single family| +28817|AAAAAAAABJAHAAAA|456|9th Davis|ST|Suite W|Midway|Shelby County|KY|41904|United States|-5|condo| +28818|AAAAAAAACJAHAAAA|183|9th 9th|Circle|Suite 110|Oakwood|Roosevelt County|MT|60169|United States|-7|single family| +28819|AAAAAAAADJAHAAAA|767|2nd 14th|Ave|Suite 450|Pine Grove|Lawrence County|PA|14593|United States|-5|apartment| +28820|AAAAAAAAEJAHAAAA|334|5th |Dr.|Suite 300|Fairfield|Richmond County|NC|26192|United States|-5|apartment| +28821|AAAAAAAAFJAHAAAA|296|Jefferson |Drive|Suite 150|Freeport|Harrison County|OH|41844|United States|-5|condo| +28822|AAAAAAAAGJAHAAAA|860|Third |Ln|Suite 240|Mountain View|Carteret County|NC|24466|United States|-5|apartment| +28823|AAAAAAAAHJAHAAAA|926|15th |Circle|Suite 380|Wesley|Williamsburg city|VA|21218|United States|-5|condo| +28824|AAAAAAAAIJAHAAAA|890|Highland 13th|Blvd|Suite 50|Lebanon|Rice County|KS|62898|United States|-6|condo| +28825|AAAAAAAAJJAHAAAA|266|Fifteenth Elevnth|Dr.|Suite 70|Highland|Malheur County|OR|99454|United States|-8|condo| +28826|AAAAAAAAKJAHAAAA|215|Highland Valley|Parkway|Suite W|Webb|Llano County|TX|70899|United States|-6|single family| +28827|AAAAAAAALJAHAAAA|889|Dogwood |RD|Suite 490|Greenville|Stephens County|TX|71387|United States|-6|condo| +28828|AAAAAAAAMJAHAAAA|798|Fifth |Lane|Suite F|Springdale|Crane County|TX|78883|United States|-6|condo| +28829|AAAAAAAANJAHAAAA|593|North 12th|Boulevard|Suite S|Richardson|Lincoln County|MT|67687|United States|-7|single family| +28830|AAAAAAAAOJAHAAAA|126|Laurel |RD|Suite N|Milan|Alameda County|CA|96697|United States|-8|condo| +28831|AAAAAAAAPJAHAAAA|||ST|Suite 230|||||United States|-6|| +28832|AAAAAAAAAKAHAAAA|613|Center |Ln|Suite T|Woodland|Duchesne County|UT|84854|United States|-7|condo| +28833|AAAAAAAABKAHAAAA|462|5th |Avenue|Suite A|Macedonia|Juneau County|WI|51087|United States|-6|condo| +28834|AAAAAAAACKAHAAAA|167|Center |Circle|Suite G|Pierce|Knox County|KY|43360|United States|-5|condo| +28835|AAAAAAAADKAHAAAA|56|Third |Ave|Suite 240|Concord|Lee County|GA|34107|United States|-5|single family| +28836|AAAAAAAAEKAHAAAA|365|Miller Central|Ct.|Suite 130|Bethel|Livingston County|NY|15281|United States|-5|apartment| +28837|AAAAAAAAFKAHAAAA|797|Spruce 10th|Drive|Suite T|Pleasant Valley|Windham County|CT|03077|United States|-5|single family| +28838|AAAAAAAAGKAHAAAA|652|Jefferson Hickory|Avenue|Suite 350|Buena Vista|Clearwater County|ID|85752|United States|-7|condo| +28839|AAAAAAAAHKAHAAAA|491|Lake |Way|Suite X|Harmony|Perry County|TN|35804|United States|-6|single family| +28840|AAAAAAAAIKAHAAAA||6th Johnson|||||NY||United States|-5|| +28841|AAAAAAAAJKAHAAAA|355|14th |Ct.|Suite 480|Pleasant Valley|Dimmit County|TX|72477|United States|-6|condo| +28842|AAAAAAAAKKAHAAAA|710|Park 2nd|Street|Suite 20|Edgewood|Wayne County|MO|60069|United States|-6|apartment| +28843|AAAAAAAALKAHAAAA|65|Main Maple|Way|Suite 170|Providence|Henry County|IN|46614|United States|-5|single family| +28844|AAAAAAAAMKAHAAAA|312|Forest 2nd|Dr.|Suite B|Chestnut Ridge|Alcona County|MI|47334|United States|-5|single family| +28845|AAAAAAAANKAHAAAA|600|6th |Ln|Suite N|Greenville|Republic County|KS|61387|United States|-6|single family| +28846|AAAAAAAAOKAHAAAA|237|Laurel |Cir.|Suite 360|Red Hill|Baker County|GA|34338|United States|-5|apartment| +28847|AAAAAAAAPKAHAAAA||||Suite 460|Greenfield||WV||United States|-5|condo| +28848|AAAAAAAAALAHAAAA|236|Laurel Main|Way|Suite 190|Lakeview|Sandoval County|NM|88579|United States|-7|apartment| +28849|AAAAAAAABLAHAAAA|394|6th |Wy|Suite 160|West Liberty|Chatham County|NC|24752|United States|-5|condo| +28850|AAAAAAAACLAHAAAA|77|Jefferson View|Parkway|Suite 460|Arlington|Rensselaer County|NY|16557|United States|-5|single family| +28851|AAAAAAAADLAHAAAA|808|Jackson |Boulevard|Suite I|Jackson|Dare County|NC|29583|United States|-5|apartment| +28852|AAAAAAAAELAHAAAA||||Suite 380|Freeman|||72297|United States|-6|single family| +28853|AAAAAAAAFLAHAAAA|796|Madison |Boulevard|Suite 300|Oak Ridge|Archer County|TX|78371|United States|-6|condo| +28854|AAAAAAAAGLAHAAAA|753|3rd |Cir.|Suite 210|Elkton|Carroll County|KY|43481|United States|-6|condo| +28855|AAAAAAAAHLAHAAAA|23|Maple |Boulevard|Suite 60|Mount Zion|Stephens County|TX|78054|United States|-6|condo| +28856|AAAAAAAAILAHAAAA|332|Hillcrest South|Ln|Suite 200|Deerfield|Hettinger County|ND|59840|United States|-6|condo| +28857|AAAAAAAAJLAHAAAA|79|Meadow |ST|Suite 10|Five Forks|Boyd County|KY|42293|United States|-6|single family| +28858|AAAAAAAAKLAHAAAA|694|Jackson |Road|Suite O|Shelby|Belknap County|NH|07175|United States|-5|condo| +28859|AAAAAAAALLAHAAAA|577||Wy||Wildwood|Merrimack County|NH|07471||-5|condo| +28860|AAAAAAAAMLAHAAAA|279|Wilson East|Ct.|Suite 260|Mount Vernon|Scioto County|OH|48482|United States|-5|condo| +28861|AAAAAAAANLAHAAAA|39|Third Lake|Ave|Suite 50|Greenwood|Transylvania County|NC|28828|United States|-5|single family| +28862|AAAAAAAAOLAHAAAA|347|North |Parkway|Suite X|Concord|Berrien County|GA|34107|United States|-5|single family| +28863|AAAAAAAAPLAHAAAA|793|Woodland Ridge|Ct.|Suite 440|Buena Vista|Caldwell County|NC|25752|United States|-5|condo| +28864|AAAAAAAAAMAHAAAA|593|Broadway First|Avenue|Suite U|Hamilton|Jasper County|SC|22808|United States|-5|single family| +28865|AAAAAAAABMAHAAAA|544|10th |Ln|Suite 490|Lawrenceville|Kanabec County|MN|54462|United States|-6|apartment| +28866|AAAAAAAACMAHAAAA|321|Park Park|Parkway|Suite Q|Centerville|Columbia County|AR|70059|United States|-6|apartment| +28867|AAAAAAAADMAHAAAA|795|4th |Drive|Suite J|Greenfield|Craven County|NC|25038|United States|-5|single family| +28868|AAAAAAAAEMAHAAAA|365|Laurel |Court|Suite W|Westminster|Madison County|GA|36549|United States|-5|apartment| +28869|AAAAAAAAFMAHAAAA|578|Williams 5th|Parkway|Suite 170|Woodlawn|Gonzales County|TX|74098|United States|-6|apartment| +28870|AAAAAAAAGMAHAAAA||Third Fifth|Lane|Suite T|Unionville||IN||United States||| +28871|AAAAAAAAHMAHAAAA|51|Main |Parkway|Suite 30|Newport|DuPage County|IL|61521|United States|-6|single family| +28872|AAAAAAAAIMAHAAAA|290|Green Lincoln|Blvd|Suite D|Red Hill|Davidson County|TN|34338|United States|-5|apartment| +28873|AAAAAAAAJMAHAAAA|365|Center 3rd|Ln|Suite 410|Springfield|Carter County|OK|79303|United States|-6|condo| +28874|AAAAAAAAKMAHAAAA|900|Davis Williams|Ct.|Suite V|Mount Pleasant|Clay County|NC|21933|United States|-5|condo| +28875|AAAAAAAALMAHAAAA|522|11th |Court|Suite J|Arlington|Hardin County|IA|56557|United States|-6|apartment| +28876|AAAAAAAAMMAHAAAA|528|East Johnson|Avenue|Suite 180|Enterprise|New Castle County|DE|11757|United States|-5|condo| +28877|AAAAAAAANMAHAAAA|845|8th |Cir.|Suite U|Forest Hills|Payette County|ID|89237|United States|-7|single family| +28878|AAAAAAAAOMAHAAAA|687|Spruce |Parkway|Suite B|Salem|Fairfield County|CT|08648|United States|-5|condo| +28879|AAAAAAAAPMAHAAAA|403|Willow Lake|Parkway|Suite 430|Bailey|Amherst County|VA|27658|United States|-5|apartment| +28880|AAAAAAAAANAHAAAA|938|Pine Franklin|Ave|Suite G|Lakeside|Surry County|VA|29532|United States|-5|condo| +28881|AAAAAAAABNAHAAAA|511|Broadway Elm|Lane|Suite X|Waterloo|Martin County|TX|71675|United States|-6|single family| +28882|AAAAAAAACNAHAAAA|155|4th 7th|Blvd|Suite 320|Highland Park|Highland County|OH|46534|United States|-5|apartment| +28883|AAAAAAAADNAHAAAA|477|Washington |Street|Suite U|Union|Kent County|RI|09321|United States|-5|condo| +28884|AAAAAAAAENAHAAAA|461|Main Willow|Road|Suite 250|Summit|Aleutians East Borough|AK|90499|United States|-9|condo| +28885|AAAAAAAAFNAHAAAA|790|Church Main|Pkwy|Suite T|Lincoln|Menifee County|KY|41289|United States|-5|apartment| +28886|AAAAAAAAGNAHAAAA|321|Miller Spring|Parkway|Suite Y|Lakeview|Henry County|VA|28579|United States|-5|single family| +28887|AAAAAAAAHNAHAAAA|755|2nd Jefferson|Road|Suite T|Harmony|Kankakee County|IL|65804|United States|-6|single family| +28888|AAAAAAAAINAHAAAA|539|Meadow |Circle|Suite 240|Five Forks|Grant County|SD|52293|United States|-6|condo| +28889|AAAAAAAAJNAHAAAA|320|Oak First|Way|Suite 40|Shiloh|Sutton County|TX|79275|United States|-6|single family| +28890|AAAAAAAAKNAHAAAA|236|Central |Way|Suite 170|Macedonia|Crawford County|WI|51087|United States|-6|single family| +28891|AAAAAAAALNAHAAAA|394|Sunset River|Circle|Suite V|New Hope|Waynesboro city|VA|29431|United States|-5|single family| +28892|AAAAAAAAMNAHAAAA|220|Jefferson |Lane|Suite T|Franklin|DeKalb County|MO|69101|United States|-6|condo| +28893|AAAAAAAANNAHAAAA|955|Walnut Spruce|Boulevard||Spring Valley||TX||United States|-6|| +28894|AAAAAAAAONAHAAAA|514|Park |Cir.|Suite 120|Lee|Schuyler County|NY|10408|United States|-5|condo| +28895|AAAAAAAAPNAHAAAA|81|2nd |Street|Suite 250|Mountain View|Lee County|MS|54466|United States|-6|condo| +28896|AAAAAAAAAOAHAAAA|138|3rd |Ct.|Suite P|Gum Springs|McLean County|ND|52106|United States|-6|single family| +28897|AAAAAAAABOAHAAAA|656|Ash Lake|Way|Suite H|Mount Vernon|Bennington County|VT|09082|United States|-5|single family| +28898|AAAAAAAACOAHAAAA|806|Cedar Fifth|Wy|Suite A|Longwood|Snohomish County|WA|97021|United States|-8|condo| +28899|AAAAAAAADOAHAAAA|208|Washington |Road|Suite S|Five Points|Izard County|AR|76098|United States|-6|apartment| +28900|AAAAAAAAEOAHAAAA|954|9th |Boulevard|Suite B|Salem|Petersburg city|VA|28048|United States|-5|single family| +28901|AAAAAAAAFOAHAAAA|869|First |Pkwy|Suite I|Antioch|Lexington County|SC|28605|United States|-5|single family| +28902|AAAAAAAAGOAHAAAA|413|Center 11th|Parkway|Suite 110|Pleasant Valley|Bedford County|VA|22477|United States|-5|apartment| +28903|AAAAAAAAHOAHAAAA|195|Valley |Wy|Suite 290|Glenwood|Falls County|TX|73511|United States|-6|single family| +28904|AAAAAAAAIOAHAAAA|945|Eigth Poplar|Drive|Suite 440|||TN|||-6|condo| +28905|AAAAAAAAJOAHAAAA|906|Willow Hillcrest|Lane|Suite 370|Fairfield|Klamath County|OR|96192|United States|-8|apartment| +28906|AAAAAAAAKOAHAAAA|376|Ash Spruce|ST|Suite E|Mountain View|Sabine Parish|LA|74466|United States|-6|single family| +28907|AAAAAAAALOAHAAAA|756|Fourth |Road|Suite G|Spring Hill|Crawford County|IA|56787|United States|-6|single family| +28908|AAAAAAAAMOAHAAAA|799|Hill |Ct.|Suite 310|Bridgeport|Grant County|KS|65817|United States|-6|condo| +28909|AAAAAAAANOAHAAAA|507|14th Forest|Way|Suite V|Jackson|Macon County|IL|69583|United States|-6|apartment| +28910|AAAAAAAAOOAHAAAA|421|Davis |Ave|Suite 460|Oak Grove|Nassau County|FL|38370|United States|-5|condo| +28911|AAAAAAAAPOAHAAAA|230|Lincoln |Avenue|Suite 480|Five Points|Worth County|GA|36098|United States|-5|apartment| +28912|AAAAAAAAAPAHAAAA|466|Central Second|ST|Suite 140|Five Points|Colbert County|AL|36098|United States|-6|apartment| +28913|AAAAAAAABPAHAAAA|871|3rd Center|Parkway|Suite 370|Pleasant Grove|Henrico County|VA|24136|United States|-5|apartment| +28914|AAAAAAAACPAHAAAA|138|River Cedar|Court|Suite Y|Lakeview|Morris County|TX|78579|United States|-6|apartment| +28915|AAAAAAAADPAHAAAA|790|East Laurel|Way|Suite 470|Glenwood|Tippecanoe County|IN|43511|United States|-5|apartment| +28916|AAAAAAAAEPAHAAAA|496|Elevnth Wilson|Blvd|Suite E|Oak Grove|Douglas County|SD|58370|United States|-6|single family| +28917|AAAAAAAAFPAHAAAA|232|5th |Wy|Suite M|Mountain View|Dodge County|NE|64466|United States|-6|single family| +28918|AAAAAAAAGPAHAAAA|64|Oak Ridge|Parkway|Suite T|Ashland|Gadsden County|FL|34244|United States|-5|single family| +28919|AAAAAAAAHPAHAAAA|450|Elm |Ct.|Suite D|Unionville|Martin County|NC|21711|United States|-5|apartment| +28920|AAAAAAAAIPAHAAAA|61|Fifth |Parkway|Suite L|Pine Grove|Douglas County|OR|94593|United States|-8|apartment| +28921|AAAAAAAAJPAHAAAA|651|Oak Chestnut|Ct.|Suite 310|Lakeview|Oliver County|ND|58579|United States|-6|condo| +28922|AAAAAAAAKPAHAAAA|654|Fourth Lake|Avenue|Suite L|Red Bank|Delta County|MI|44975|United States|-5|single family| +28923|AAAAAAAALPAHAAAA|281|Elm |Cir.|Suite 330|Brownsville|Greensville County|VA|29310|United States|-5|single family| +28924|AAAAAAAAMPAHAAAA|344|Chestnut |RD|Suite 190|Pinhook|Cibola County|NM|89398|United States|-7|apartment| +28925|AAAAAAAANPAHAAAA|15|Maple View|Dr.|Suite 470|Bethel|Bosque County|TX|75281|United States|-6|condo| +28926|AAAAAAAAOPAHAAAA|851|1st Johnson|Lane|Suite 200|Antioch|Greer County|OK|78605|United States|-6|condo| +28927|AAAAAAAAPPAHAAAA|461|6th |Blvd|Suite 70|Oakdale|Cullman County|AL|39584|United States|-6|condo| +28928|AAAAAAAAAABHAAAA|17|Main |Pkwy|Suite 480|Glendale|Campbell County|WY|83951|United States|-7|single family| +28929|AAAAAAAABABHAAAA|393|Meadow |Drive|Suite X|Kingston|Hudson County|NJ|05575|United States|-5|condo| +28930|AAAAAAAACABHAAAA|64|Lakeview |Boulevard|Suite 340|Riverside|Montgomery County|MO|69231|United States|-6|condo| +28931|AAAAAAAADABHAAAA|324|Park East|Ct.|Suite H|Mount Olive|Vigo County|IN|48059|United States|-5|apartment| +28932|AAAAAAAAEABHAAAA|879|Meadow Washington|Circle|Suite 290|Pine Grove|Candler County|GA|34593|United States|-5|apartment| +28933|AAAAAAAAFABHAAAA|783|Birch Park|Dr.|Suite 350|Mount Pleasant|Ketchikan Gateway Borough|AK|91933|United States|-9|apartment| +28934|AAAAAAAAGABHAAAA|575|8th |Ln|Suite 40|Franklin|Grundy County|IA|59101|United States|-6|condo| +28935|AAAAAAAAHABHAAAA|155|Birch |ST|Suite 140|Summit|Ramsey County|MN|50499|United States|-6|condo| +28936|AAAAAAAAIABHAAAA|680|South |Pkwy|Suite H|Waterloo|Rockingham County|NC|21675|United States|-5|single family| +28937|AAAAAAAAJABHAAAA|271|Elm |Pkwy|Suite P|Spring Hill|Crook County|WY|86787|United States|-7|condo| +28938|AAAAAAAAKABHAAAA|116|8th Center|Street|Suite D|Wyoming|Lavaca County|TX|70216|United States|-6|condo| +28939|AAAAAAAALABHAAAA|557|Washington |Ave|Suite 100|Newtown|Callahan County|TX|71749|United States|-6|condo| +28940|AAAAAAAAMABHAAAA|130|Laurel Elm|Street|Suite G|Florence|Sequatchie County|TN|33394|United States|-6|condo| +28941|AAAAAAAANABHAAAA|635|Park |Street|Suite 70|Five Forks|Perry County|IL|62293|United States|-6|condo| +28942|AAAAAAAAOABHAAAA|737|Lee |Cir.|Suite A|Fairbanks|Payette County|ID|86653|United States|-7|apartment| +28943|AAAAAAAAPABHAAAA|412|8th |Boulevard|Suite 190|Westminster|Green Lake County|WI|56549|United States|-6|single family| +28944|AAAAAAAAABBHAAAA|657|Wilson Center|Boulevard|Suite 490|Wilson|Llano County|TX|76971|United States|-6|single family| +28945|AAAAAAAABBBHAAAA|72|Miller |Wy|Suite 80|Lebanon|Chester County|SC|22898|United States|-5|apartment| +28946|AAAAAAAACBBHAAAA|312|Cedar |ST|Suite I|Ashland|Lincoln County|TN|34244|United States|-6|single family| +28947|AAAAAAAADBBHAAAA|638|Walnut |Street|Suite 480|Lakeview|Jewell County|KS|68579|United States|-6|apartment| +28948|AAAAAAAAEBBHAAAA|294|First |RD|Suite F|Springdale|Dickinson County|MI|48883|United States|-5|condo| +28949|AAAAAAAAFBBHAAAA|549|Laurel |Circle|Suite 70|Floyd|Miami County|OH|43235|United States|-5|apartment| +28950|AAAAAAAAGBBHAAAA|348|Hill 2nd|Pkwy|Suite 480|Oakland|New Madrid County|MO|69843|United States|-6|single family| +28951|AAAAAAAAHBBHAAAA|80|Ridge |Lane|Suite 320|Lakeview|Summit County|CO|88579|United States|-7|condo| +28952|AAAAAAAAIBBHAAAA|282|Fourth |Boulevard|Suite J|Hamilton|Essex County|VA|22808|United States|-5|single family| +28953|AAAAAAAAJBBHAAAA|765|13th Davis|Ln|Suite 350|Maple Grove|Martin County|TX|78252|United States|-6|condo| +28954|AAAAAAAAKBBHAAAA|162|North Tenth|Cir.|Suite M|Friendship|Swift County|MN|54536|United States|-6|single family| +28955|AAAAAAAALBBHAAAA|324|Pine |Lane|Suite Q|Pleasant Valley|Livingston Parish|LA|72477|United States|-6|single family| +28956|AAAAAAAAMBBHAAAA|129|Eigth Lake|Dr.|Suite 350|White Oak|Teton County|ID|86668|United States|-7|condo| +28957|AAAAAAAANBBHAAAA|270|Church |Dr.|Suite 10|Red Hill|McLennan County|TX|74338|United States|-6|apartment| +28958|AAAAAAAAOBBHAAAA|906|Lake Jackson|Parkway|Suite 10|Page|Mason County|TX|70296|United States|-6|condo| +28959|AAAAAAAAPBBHAAAA|356|12th Main|Ave|Suite 390|Cordova|Archer County|TX|76938|United States|-6|apartment| +28960|AAAAAAAAACBHAAAA|254|Sycamore |Drive|Suite 250|Gum Springs|Graham County|KS|62106|United States|-6|single family| +28961|AAAAAAAABCBHAAAA|901|Washington |Circle|Suite 190|New Hope|Loudon County|TN|39431|United States|-6|apartment| +28962|AAAAAAAACCBHAAAA|994|1st View|Way|Suite R|Greenwood|Carroll County|IN|48828|United States|-5|apartment| +28963|AAAAAAAADCBHAAAA|602|Cedar |RD|Suite 70|Pierce|Macoupin County|IL|63360|United States|-6|single family| +28964|AAAAAAAAECBHAAAA|733|9th |Pkwy|Suite 350|Pleasant Hill|Hughes County|OK|73604|United States|-6|apartment| +28965|AAAAAAAAFCBHAAAA|385|East Woodland|Street|Suite 50|Mountain View|Rains County|TX|74466|United States|-6|single family| +28966|AAAAAAAAGCBHAAAA|911|Third |Boulevard|Suite 450|Springdale|Madison County|NE|68883|United States|-7|apartment| +28967|AAAAAAAAHCBHAAAA|406|Davis |Drive|Suite B|Oakdale|Cape May County|NJ|09584|United States|-5|condo| +28968|AAAAAAAAICBHAAAA|66|Spruce Elevnth|Ave|Suite 160|Mountain View|Hardin County|IA|54466|United States|-6|single family| +28969|AAAAAAAAJCBHAAAA|154|Elm 8th|Boulevard|Suite A|Green Acres|Marengo County|AL|37683|United States|-6|apartment| +28970|AAAAAAAAKCBHAAAA|2|4th 4th|Drive|Suite 490|Oak Ridge|Toole County|MT|68371|United States|-7|single family| +28971|AAAAAAAALCBHAAAA|416|Park Lincoln|Road|Suite A|Liberty|Columbia County|FL|33451|United States|-5|condo| +28972|AAAAAAAAMCBHAAAA|814|Chestnut |Way|Suite 270|Clinton|Nevada County|AR|78222|United States|-6|single family| +28973|AAAAAAAANCBHAAAA|779|Cedar Elm|Ln|Suite U|Concord|Richland County|IL|64107|United States|-6|apartment| +28974|AAAAAAAAOCBHAAAA|36|Cedar 8th|Drive|Suite M|Mountain View|Amite County|MS|54466|United States|-6|apartment| +28975|AAAAAAAAPCBHAAAA|513|5th View|RD|Suite 220|Lincoln|Plymouth County|MA|01889|United States|-5|condo| +28976|AAAAAAAAADBHAAAA|903|6th |Ct.|Suite 350|Forest Hills|Dade County|GA|39237|United States|-5|condo| +28977|AAAAAAAABDBHAAAA|808|Williams |ST|Suite F|White Hall|Pacific County|WA|96955|United States|-8|condo| +28978|AAAAAAAACDBHAAAA|736|Pine Meadow|Way|Suite 140|Plainview|Erie County|PA|13683|United States|-5|condo| +28979|AAAAAAAADDBHAAAA|296|Highland |Circle|Suite G|Mountain View|Westmoreland County|PA|14466|United States|-5|apartment| +28980|AAAAAAAAEDBHAAAA|87|Center Oak|Ln|Suite A|Farmington|Kanawha County|WV|29145|United States|-5|condo| +28981|AAAAAAAAFDBHAAAA|444|Adams 10th|ST|Suite R|Salem|Owen County|KY|48048|United States|-5|single family| +28982|AAAAAAAAGDBHAAAA|955|Walnut Franklin|Ct.|Suite 230|Highland|Obion County|TN|39454|United States|-6|apartment| +28983|AAAAAAAAHDBHAAAA|518|Madison Elm|Ct.|Suite 320|Bethel|Blaine County|ID|85281|United States|-7|condo| +28984|AAAAAAAAIDBHAAAA||Park Elm|||Fairfield||FL||United States||condo| +28985|AAAAAAAAJDBHAAAA|698|Sixth Tenth|Court|Suite V|Glendale|Estill County|KY|43951|United States|-6|condo| +28986|AAAAAAAAKDBHAAAA|167|1st 12th|Lane|Suite 300|Bridgeport|Chippewa County|WI|55817|United States|-6|apartment| +28987|AAAAAAAALDBHAAAA|663|Main |Wy|Suite 490|Roscoe|Oxford County|ME|02454|United States|-5|single family| +28988|AAAAAAAAMDBHAAAA|492|1st Jackson|Drive|Suite 130|Wildwood|Franklin County|WA|96871|United States|-8|apartment| +28989|AAAAAAAANDBHAAAA|456|Miller 7th|Avenue|Suite 460|Pleasant Hill|Perkins County|NE|63604|United States|-7|apartment| +28990|AAAAAAAAODBHAAAA|344|Highland |Cir.|Suite S|Edgewood|Autauga County|AL|30069|United States|-6|apartment| +28991|AAAAAAAAPDBHAAAA|357|9th Meadow|Dr.|Suite 420|Woodlawn|Summit County|OH|44098|United States|-5|single family| +28992|AAAAAAAAAEBHAAAA|355|Hill |Way|Suite 310|Pleasant Hill|Appanoose County|IA|53604|United States|-6|condo| +28993|AAAAAAAABEBHAAAA|127|Main |Ln|Suite T|Shady Grove|Lyman County|SD|52812|United States|-7|apartment| +28994|AAAAAAAACEBHAAAA|721|Main |Blvd|Suite Y|Waterloo|Bowie County|TX|71675|United States|-6|apartment| +28995|AAAAAAAADEBHAAAA|917|Locust East|Wy|Suite H|Pleasant Hill|Harrison County|TX|73604|United States|-6|apartment| +28996|AAAAAAAAEEBHAAAA|891|7th |Cir.|Suite 320|Greenville|Oglethorpe County|GA|31387|United States|-5|condo| +28997|AAAAAAAAFEBHAAAA|440|Lake Ridge|Blvd|Suite 230|Enterprise|Henry County|MO|61757|United States|-6|apartment| +28998|AAAAAAAAGEBHAAAA|18|Sycamore |Circle|Suite F|Highland Park|Morton County|KS|66534|United States|-6|apartment| +28999|AAAAAAAAHEBHAAAA|422|Cedar View|RD|Suite B|Mountain View|Petersburg city|VA|24466|United States|-5|condo| +29000|AAAAAAAAIEBHAAAA|414|Jackson |Ln|Suite 290|Salem|Knox County|NE|68048|United States|-7|condo| +29001|AAAAAAAAJEBHAAAA|162|Railroad Washington|Drive|Suite 420|Union Hill|Blount County|AL|37746|United States|-6|apartment| +29002|AAAAAAAAKEBHAAAA|650|Smith View|RD|Suite L|Providence|Stevens County|KS|66614|United States|-6|condo| +29003|AAAAAAAALEBHAAAA|931|First Walnut|Avenue|Suite K|Riverside|Gallia County|OH|49231|United States|-5|condo| +29004|AAAAAAAAMEBHAAAA||Lake Second||Suite 260||||71513|||condo| +29005|AAAAAAAANEBHAAAA|783|Hickory |Drive|Suite 460|Glenwood|Comanche County|OK|73511|United States|-6|condo| +29006|AAAAAAAAOEBHAAAA|165|1st Oak|Ave|Suite M|Edgewood|Piute County|UT|80069|United States|-7|single family| +29007|AAAAAAAAPEBHAAAA|510|7th Pine|Blvd|Suite F|Lincoln|Beadle County|SD|51289|United States|-6|apartment| +29008|AAAAAAAAAFBHAAAA|370|11th |Ln|Suite 490|Rockwood|Hardin County|KY|41545|United States|-6|apartment| +29009|AAAAAAAABFBHAAAA|457|Sunset |Way|Suite I|Midway|De Witt County|IL|61904|United States|-6|condo| +29010|AAAAAAAACFBHAAAA|700|North Second|Wy|Suite W|White Oak|Lincoln County|ME|07268|United States|-5|single family| +29011|AAAAAAAADFBHAAAA|447|Johnson Third|RD|Suite U|Clifton|Alachua County|FL|38014|United States|-5|apartment| +29012|AAAAAAAAEFBHAAAA|304|View |ST|Suite 270|Franklin|Washington County|IL|69101|United States|-6|apartment| +29013|AAAAAAAAFFBHAAAA|319|Sunset |Dr.|Suite 250|Providence|Nolan County|TX|76614|United States|-6|condo| +29014|AAAAAAAAGFBHAAAA|667|9th |RD|Suite V|Mount Olive|Hill County|TX|78059|United States|-6|condo| +29015|AAAAAAAAHFBHAAAA|450|Valley |Wy|Suite 220|Glendale|Dallas County|MO|63951|United States|-6|apartment| +29016|AAAAAAAAIFBHAAAA|119|Franklin |Circle|Suite Y|Pleasant Valley|Houston County|AL|32477|United States|-6|apartment| +29017|AAAAAAAAJFBHAAAA|244|5th Lake|Wy|Suite 340|Hamilton|Braxton County|WV|22808|United States|-5|apartment| +29018|AAAAAAAAKFBHAAAA|318|First |Street|Suite 390|Riverdale|Morrill County|NE|69391|United States|-7|condo| +29019|AAAAAAAALFBHAAAA|726|Railroad East|Dr.|Suite 390|Fairview|Edgefield County|SC|25709|United States|-5|apartment| +29020|AAAAAAAAMFBHAAAA|758||Street|Suite Y|||IA||||| +29021|AAAAAAAANFBHAAAA|846|Lake |Ave|Suite K|Harmony|Northumberland County|PA|15804|United States|-5|condo| +29022|AAAAAAAAOFBHAAAA|703|South |Drive|Suite 250|Omega|Grant County|KS|66364|United States|-6|single family| +29023|AAAAAAAAPFBHAAAA|303|West |Circle|Suite 280|Glendale|Transylvania County|NC|23951|United States|-5|single family| +29024|AAAAAAAAAGBHAAAA|901|14th Washington|Drive|Suite L|Summit|Monroe County|NY|10499|United States|-5|apartment| +29025|AAAAAAAABGBHAAAA|259|Cherry |Road|Suite 390|||||||| +29026|AAAAAAAACGBHAAAA|646|Seventh |Wy|Suite 180|Kingston|Talbot County|MD|24975|United States|-5|apartment| +29027|AAAAAAAADGBHAAAA|12|Spring Lakeview|Avenue|Suite L|Arlington|Hardy County|WV|26557|United States|-5|single family| +29028|AAAAAAAAEGBHAAAA|698|2nd |Avenue|Suite O|Valley View|Val Verde County|TX|75124|United States|-6|condo| +29029|AAAAAAAAFGBHAAAA|817|Ash Davis|Way|Suite K|Ashland|Harrison County|WV|24244|United States|-5|condo| +29030|AAAAAAAAGGBHAAAA|440|Ridge 3rd|Boulevard|Suite E|Mount Zion|Moore County|TX|78054|United States|-6|condo| +29031|AAAAAAAAHGBHAAAA|715|Mill |ST|Suite U|Cedar Grove|Quay County|NM|80411|United States|-7|single family| +29032|AAAAAAAAIGBHAAAA|5|1st Fourth|Parkway|Suite F|Pine Grove|Pierce County|NE|64593|United States|-7|apartment| +29033|AAAAAAAAJGBHAAAA|387|Jefferson Hill|Circle|Suite B|Forest Hills|Wayne County|GA|39237|United States|-5|apartment| +29034|AAAAAAAAKGBHAAAA|352|Park |Lane|Suite P|Glenwood|Spalding County|GA|33511|United States|-5|condo| +29035|AAAAAAAALGBHAAAA|367|13th |Avenue|Suite 250|Fairview|Presque Isle County|MI|45709|United States|-5|apartment| +29036|AAAAAAAAMGBHAAAA|423|8th First|Ln|Suite 380|Mount Pleasant|Polk County|WI|51933|United States|-6|condo| +29037|AAAAAAAANGBHAAAA|844|Dogwood Mill|Drive|Suite J|Lucas|Summit County|UT|84554|United States|-7|single family| +29038|AAAAAAAAOGBHAAAA|947|Ash |Pkwy|Suite 40|Crossroads|Dauphin County|PA|10534|United States|-5|condo| +29039|AAAAAAAAPGBHAAAA|118|Third Cedar|Wy|Suite 420|Spring Valley|Rankin County|MS|56060|United States|-6|single family| +29040|AAAAAAAAAHBHAAAA|60|Twelfth Hill|Lane|Suite 30|Clifton|DeKalb County|AL|38014|United States|-6|apartment| +29041|AAAAAAAABHBHAAAA|433|Jackson |Ave|Suite 270|Lebanon|Wake County|NC|22898|United States|-5|condo| +29042|AAAAAAAACHBHAAAA|324|Poplar |Circle|Suite 400|Antioch|Upton County|TX|78605|United States|-6|condo| +29043|AAAAAAAADHBHAAAA|753|East |Pkwy|Suite 240|Edgewood|Stanley County|SD|50069|United States|-7|apartment| +29044|AAAAAAAAEHBHAAAA|855|Miller East|Avenue|Suite K|Riverside|Clarke County|AL|39231|United States|-6|single family| +29045|AAAAAAAAFHBHAAAA|591|Ridge |Way|Suite 340|Edgewood|San Joaquin County|CA|90069|United States|-8|condo| +29046|AAAAAAAAGHBHAAAA|329|North |Parkway|Suite 290|Concord|Roane County|TN|34107|United States|-6|condo| +29047|AAAAAAAAHHBHAAAA||5th |Court|||Miami County|||United States||| +29048|AAAAAAAAIHBHAAAA|256|Spruce |Blvd|Suite 220|Antioch|Cecil County|MD|28605|United States|-5|single family| +29049|AAAAAAAAJHBHAAAA|280|Maple Main|RD|Suite 240|Westminster|Kingman County|KS|66549|United States|-6|apartment| +29050|AAAAAAAAKHBHAAAA|916|3rd 1st|Circle|Suite B|Unionville|Nye County|NV|81711|United States|-8|condo| +29051|AAAAAAAALHBHAAAA|752|Oak |Drive|Suite 400|Denmark|Emanuel County|GA|35576|United States|-5|condo| +29052|AAAAAAAAMHBHAAAA|844|Railroad Lee|Lane|Suite 250|Florence|Lycoming County|PA|13394|United States|-5|apartment| +29053|AAAAAAAANHBHAAAA|283|Cedar Fourteenth|Blvd|Suite 190|Newport|Nemaha County|KS|61521|United States|-6|apartment| +29054|AAAAAAAAOHBHAAAA|661|Pine |Way|Suite K|Harvey|Poweshiek County|IA|55858|United States|-6|single family| +29055|AAAAAAAAPHBHAAAA|638|Park Birch|Ave|Suite 290|Midway|Pontotoc County|OK|71904|United States|-6|condo| +29056|AAAAAAAAAIBHAAAA|||RD|Suite T|Valley View|Tarrant County|TX||United States|-6|| +29057|AAAAAAAABIBHAAAA|896|South Sunset|Way|Suite 430|Farmington|Boone County|IA|59145|United States|-6|apartment| +29058|AAAAAAAACIBHAAAA|228|Second Railroad|Ct.|Suite K|Kingston|Horry County|SC|24975|United States|-5|condo| +29059|AAAAAAAADIBHAAAA|688|Laurel Seventh|Road|Suite U|Buena Vista|Montgomery County|PA|15752|United States|-5|apartment| +29060|AAAAAAAAEIBHAAAA|319|15th Oak|Ave|Suite H|Five Points|Drew County|AR|76098|United States|-6|condo| +29061|AAAAAAAAFIBHAAAA|619|Smith |Lane|Suite 30|Glendale|Dakota County|NE|63951|United States|-6|single family| +29062|AAAAAAAAGIBHAAAA|293|Davis |Ave|Suite 470|Springfield|Mille Lacs County|MN|59303|United States|-6|single family| +29063|AAAAAAAAHIBHAAAA|604|Williams |Circle|Suite A|Woodlawn|Winkler County|TX|74098|United States|-6|single family| +29064|AAAAAAAAIIBHAAAA|436|Second Elm|Way|Suite 30|Green Acres|Laurens County|GA|37683|United States|-5|single family| +29065|AAAAAAAAJIBHAAAA|53|Eigth West|Cir.|Suite J|Harmony|Hockley County|TX|75804|United States|-6|single family| +29066|AAAAAAAAKIBHAAAA|284|Jackson |Dr.|Suite L|Macedonia|Tom Green County|TX|71087|United States|-6|single family| +29067|AAAAAAAALIBHAAAA|776|Cedar Pine|Circle|Suite V|Five Forks|Bradford County|PA|12293|United States|-5|condo| +29068|AAAAAAAAMIBHAAAA|387|Spring ||||Monroe County||56557|United States||| +29069|AAAAAAAANIBHAAAA|384|Ridge |Dr.|Suite 150|Fairview|Lee County|MS|55709|United States|-6|apartment| +29070|AAAAAAAAOIBHAAAA|468|Dogwood |ST|Suite A|Red Hill|Jefferson County|GA|34338|United States|-5|single family| +29071|AAAAAAAAPIBHAAAA|49|Walnut |Wy|Suite J|Providence|Crawford County|IN|46614|United States|-5|single family| +29072|AAAAAAAAAJBHAAAA|484|Lincoln |Court|Suite G|Cumberland|Polk County|WI|58971|United States|-6|single family| +29073|AAAAAAAABJBHAAAA|327|Seventh 15th|Street|Suite 440|Fairview|Traill County|ND|55709|United States|-6|apartment| +29074|AAAAAAAACJBHAAAA|596|Tenth Lincoln|Parkway|Suite 160|Belmont|Palm Beach County|FL|30191|United States|-5|single family| +29075|AAAAAAAADJBHAAAA|921|Fifth |ST|Suite O|Union|Perry County|MS|58721|United States|-6|single family| +29076|AAAAAAAAEJBHAAAA|948|1st Lincoln|Road|Suite C|Wildwood|Love County|OK|76871|United States|-6|apartment| +29077|AAAAAAAAFJBHAAAA||||Suite 10|Newport|||21521|United States||| +29078|AAAAAAAAGJBHAAAA|148|Dogwood |RD|Suite 230|Jackson|Emery County|UT|89583|United States|-7|single family| +29079|AAAAAAAAHJBHAAAA|890|4th East|RD|Suite 160|Lakeview|Allegany County|MD|28579|United States|-5|condo| +29080|AAAAAAAAIJBHAAAA|590|Pine |Avenue|Suite 330|Brookwood|Gray County|KS|60965|United States|-6|condo| +29081|AAAAAAAAJJBHAAAA|131|13th 3rd|Ave|Suite 40|Newtown|Marengo County|AL|31749|United States|-6|condo| +29082|AAAAAAAAKJBHAAAA|397|Willow |Ln|Suite H|Acme|Bennington County|VT|00764|United States|-5|apartment| +29083|AAAAAAAALJBHAAAA|533|South 12th|Pkwy|Suite D|Antioch|Rappahannock County|VA|28605|United States|-5|condo| +29084|AAAAAAAAMJBHAAAA|636|Central South|Ln|Suite 160|Ludlow|Grimes County|TX|75566|United States|-6|apartment| +29085|AAAAAAAANJBHAAAA|958|Pine |Circle|Suite J|Glenwood|Refugio County|TX|73511|United States|-6|condo| +29086|AAAAAAAAOJBHAAAA|955|West Lee|Ave|Suite 10|Tracy|Cowlitz County|WA|96340|United States|-8|condo| +29087|AAAAAAAAPJBHAAAA|432|1st Valley|Court|Suite 310|Lincoln|Branch County|MI|41289|United States|-5|apartment| +29088|AAAAAAAAAKBHAAAA|103|Walnut |Blvd|Suite Y|Bridgeport|Lawrence County|KY|45817|United States|-5|single family| +29089|AAAAAAAABKBHAAAA|998|4th |Wy|Suite 390|Liberty|Bamberg County|SC|23451|United States|-5|single family| +29090|AAAAAAAACKBHAAAA|613|6th River|Avenue|Suite I|Jackson|Okmulgee County|OK|79583|United States|-6|apartment| +29091|AAAAAAAADKBHAAAA|538|3rd Pine|Court|Suite U|Lakeview|Stearns County|MN|58579|United States|-6|apartment| +29092|AAAAAAAAEKBHAAAA|54|2nd Second|Boulevard|Suite 130|Walnut Grove|Nelson County|ND|57752|United States|-6|apartment| +29093|AAAAAAAAFKBHAAAA|747|Maple |Blvd|Suite F|Unionville|Calhoun County|AL|31711|United States|-6|single family| +29094|AAAAAAAAGKBHAAAA|597|Valley |Ln|Suite 450|Kingston|Hardeman County|TN|34975|United States|-5|apartment| +29095|AAAAAAAAHKBHAAAA|767|Park North|Avenue|Suite 460|New Hope|Shasta County|CA|99431|United States|-8|apartment| +29096|AAAAAAAAIKBHAAAA|587|13th |RD|Suite 290|Oak Ridge|De Soto Parish|LA|78371|United States|-6|apartment| +29097|AAAAAAAAJKBHAAAA|547|Main Third|Pkwy|Suite Y|Concord|Charlotte County|FL|34107|United States|-5|apartment| +29098|AAAAAAAAKKBHAAAA|945|Chestnut Hillcrest|Street||Greenville|Wayne County|NC|21387|||apartment| +29099|AAAAAAAALKBHAAAA|509|First |Ln|Suite X|Liberty|Montgomery County|PA|13451|United States|-5|single family| +29100|AAAAAAAAMKBHAAAA|478|Third |Court|Suite G|Mount Vernon|Costilla County|CO|88482|United States|-7|apartment| +29101|AAAAAAAANKBHAAAA|134|Pine |Lane|Suite 480|Oak Ridge|Shelby County|AL|38371|United States|-6|apartment| +29102|AAAAAAAAOKBHAAAA|739|Sunset Spruce|Cir.|Suite D|Newtown|Juniata County|PA|11749|United States|-5|condo| +29103|AAAAAAAAPKBHAAAA|329|Walnut |Boulevard|Suite 460|Woodlawn|Morgan County|GA|34098|United States|-5|single family| +29104|AAAAAAAAALBHAAAA|315|2nd 6th|RD|Suite 340|Greenville|Converse County|WY|81387|United States|-7|apartment| +29105|AAAAAAAABLBHAAAA|451|Ash Miller|Avenue|Suite 40|Unionville|McIntosh County|GA|31711|United States|-5|apartment| +29106|AAAAAAAACLBHAAAA|911|Church |Lane|Suite 340|Harmony|Jefferson County|NY|15804|United States|-5|single family| +29107|AAAAAAAADLBHAAAA|206|Sixth Meadow|Ct.|Suite J|Bunker Hill|Taylor County|WV|20150|United States|-5|apartment| +29108|AAAAAAAAELBHAAAA|965|12th Railroad|Blvd|Suite 370|Springdale|Oklahoma County|OK|78883|United States|-6|single family| +29109|AAAAAAAAFLBHAAAA|29|Mill Highland|Road|Suite 460|Union|Centre County|PA|18721|United States|-5|condo| +29110|AAAAAAAAGLBHAAAA|285|Central Center|Ct.|Suite 140|Union Hill|Routt County|CO|87746|United States|-7|single family| +29111|AAAAAAAAHLBHAAAA|837|Laurel 14th|Circle|Suite 80|Oak Hill|Moffat County|CO|87838|United States|-7|apartment| +29112|AAAAAAAAILBHAAAA|90|12th Jefferson|Ln|Suite V|Liberty|Belmont County|OH|43451|United States|-5|single family| +29113|AAAAAAAAJLBHAAAA|519|Poplar |Blvd|Suite P|Highland|Tuscaloosa County|AL|39454|United States|-6|condo| +29114|AAAAAAAAKLBHAAAA|542|Laurel Cedar|Avenue|Suite 250|Shady Grove|Pontotoc County|OK|72812|United States|-6|condo| +29115|AAAAAAAALLBHAAAA|525|Locust |Ct.|Suite R|Clinton|Frederick County|VA|28222|United States|-5|single family| +29116|AAAAAAAAMLBHAAAA|933|Park Maple|Ct.|Suite 30|Lebanon|Nez Perce County|ID|82898|United States|-7|condo| +29117|AAAAAAAANLBHAAAA|886|South Woodland|Boulevard|Suite 20|White Oak|Sussex County|NJ|07268|United States|-5|condo| +29118|AAAAAAAAOLBHAAAA|874|15th 4th|Street|Suite S|Morgantown|Middlesex County|NJ|09793|United States|-5|single family| +29119|AAAAAAAAPLBHAAAA|633|Walnut |Circle|Suite E|Redland|Redwood County|MN|56343|United States|-6|apartment| +29120|AAAAAAAAAMBHAAAA|314|4th Valley|Pkwy|Suite 220|Woodville|Pima County|AZ|84289|United States|-7|apartment| +29121|AAAAAAAABMBHAAAA|784|Pine |Wy|Suite C|Georgetown|Sitka Borough|AK|97057|United States|-9|condo| +29122|AAAAAAAACMBHAAAA|573|Fourth |Wy|Suite 30|Macedonia|Early County|GA|31087|United States|-5|condo| +29123|AAAAAAAADMBHAAAA|674|Spring |Way|Suite P|Newtown|Kaufman County|TX|71749|United States|-6|apartment| +29124|AAAAAAAAEMBHAAAA|276|2nd |Lane|Suite R|Mount Pleasant|Calaveras County|CA|91933|United States|-8|condo| +29125|AAAAAAAAFMBHAAAA|15|Fourth 9th|ST|Suite 150|Fairview|Fentress County|TN|35709|United States|-5|condo| +29126|AAAAAAAAGMBHAAAA|391|Third View|ST|Suite 370|Bunker Hill|Foard County|TX|70150|United States|-6|condo| +29127|AAAAAAAAHMBHAAAA|802|Wilson South|RD|Suite 450|Shelby|Baker County|OR|96575|United States|-8|condo| +29128|AAAAAAAAIMBHAAAA|169|Hickory |Lane|Suite F|Westgate|Lake County|OH|42366|United States|-5|single family| +29129|AAAAAAAAJMBHAAAA|277|Lee West|Ln|Suite 290|Union|Kootenai County|ID|88721|United States|-7|condo| +29130|AAAAAAAAKMBHAAAA|541|6th 1st|Parkway|Suite 350|Hamilton|Weber County|UT|82808|United States|-7|apartment| +29131|AAAAAAAALMBHAAAA|373|9th Green|Blvd|Suite L|Shaw|Lawrence County|AR|70618|United States|-6|apartment| +29132|AAAAAAAAMMBHAAAA|251|Chestnut |Dr.|Suite 260|Belmont|Cascade County|MT|60191|United States|-7|apartment| +29133|AAAAAAAANMBHAAAA|830|Laurel |RD|Suite F|Oakwood|Ozark County|MO|60169|United States|-6|single family| +29134|AAAAAAAAOMBHAAAA|836|Tenth Green|Drive|Suite 470|Lebanon|Montgomery County|AR|72898|United States|-6|condo| +29135|AAAAAAAAPMBHAAAA|173|Jackson 7th|Pkwy|Suite S|Union|Bond County|IL|68721|United States|-6|single family| +29136|AAAAAAAAANBHAAAA|956|Maple |Ct.|Suite S|Harmony|Rutland County|VT|06404|United States|-5|apartment| +29137|AAAAAAAABNBHAAAA|789|8th 8th|Ave|Suite F|Springfield|Dare County|NC|29303|United States|-5|condo| +29138|AAAAAAAACNBHAAAA|||||Georgetown||ND||||| +29139|AAAAAAAADNBHAAAA|599|Lee |Boulevard|Suite S|Shiloh|Charlotte County|VA|29275|United States|-5|apartment| +29140|AAAAAAAAENBHAAAA|430|10th |Ct.|Suite P|Five Points|Stephenson County|IL|66098|United States|-6|condo| +29141|AAAAAAAAFNBHAAAA|350|6th |Lane|Suite 260|Nichols|Cook County|MN|57940|United States|-6|apartment| +29142|AAAAAAAAGNBHAAAA|92|Davis Spruce|Ct.|Suite N|Oak Ridge|McCone County|MT|68371|United States|-7|condo| +29143|AAAAAAAAHNBHAAAA|41|Spring |Circle|Suite A|Providence|Hale County|AL|36614|United States|-6|apartment| +29144|AAAAAAAAINBHAAAA||Hickory ||Suite 170||Kingsbury County||50169|United States||apartment| +29145|AAAAAAAAJNBHAAAA|218|South Main|Ct.|Suite 20|Belmont|Nowata County|OK|70191|United States|-6|single family| +29146|AAAAAAAAKNBHAAAA|959|Hill 5th|Boulevard|Suite M|Belmont|Loudon County|TN|30191|United States|-6|apartment| +29147|AAAAAAAALNBHAAAA|169|5th Wilson|Lane|Suite 290|Springfield|Dallas County|AR|79303|United States|-6|apartment| +29148|AAAAAAAAMNBHAAAA|992|Willow Lakeview|Way|Suite R|Marion|Love County|OK|70399|United States|-6|single family| +29149|AAAAAAAANNBHAAAA|918|4th |Way|Suite L|Highland|Sherman County|NE|69454|United States|-7|apartment| +29150|AAAAAAAAONBHAAAA||Jackson Dogwood||Suite 340|Unionville|||21711|||condo| +29151|AAAAAAAAPNBHAAAA|302|Spruce Center|Lane|Suite 240|Burns|Henry County|IL|65272|United States|-6|apartment| +29152|AAAAAAAAAOBHAAAA|496|9th 3rd|Dr.|Suite R|Clifton|Latimer County|OK|78014|United States|-6|single family| +29153|AAAAAAAABOBHAAAA|915|5th 6th|Ln|Suite E|Belmont|Christian County|KY|40191|United States|-6|condo| +29154|AAAAAAAACOBHAAAA|381|4th 3rd|Ave|Suite 150|Jamestown|Atchison County|MO|66867|United States|-6|apartment| +29155|AAAAAAAADOBHAAAA|305|12th |Avenue|Suite W|Glendale|Granville County|NC|23951|United States|-5|single family| +29156|AAAAAAAAEOBHAAAA|33|12th |Ln|Suite 460|Georgetown|Delaware County|IN|47057|United States|-5|single family| +29157|AAAAAAAAFOBHAAAA|85|Dogwood |Dr.|Suite P|Oak Ridge|Gaines County|TX|78371|United States|-6|apartment| +29158|AAAAAAAAGOBHAAAA|238|View |Cir.|Suite 160|Harmony|Glasscock County|TX|75804|United States|-6|condo| +29159|AAAAAAAAHOBHAAAA|811|Lee South|Cir.|Suite G|Jamestown|Pembina County|ND|56867|United States|-6|apartment| +29160|AAAAAAAAIOBHAAAA|860|Spring Sycamore|Court|Suite 180|Brownsville|Camden County|NC|29310|United States|-5|apartment| +29161|AAAAAAAAJOBHAAAA|730|Davis Poplar|Lane|Suite Y|Gladstone|Long County|GA|30894|United States|-5|condo| +29162|AAAAAAAAKOBHAAAA|625|Williams 9th|Street|Suite 250|Texas|Aransas County|TX|73342|United States|-6|apartment| +29163|AAAAAAAALOBHAAAA|502|4th 4th|Parkway|Suite 10|Brookwood|Eddy County|NM|80965|United States|-7|single family| +29164|AAAAAAAAMOBHAAAA|243|Wilson |ST|Suite 80|Marion|Monroe County|GA|30399|United States|-5|condo| +29165|AAAAAAAANOBHAAAA|54|Dogwood Highland|Wy|Suite Q|Lincoln|Aleutians West Census Area|AK|91289|United States|-9|single family| +29166|AAAAAAAAOOBHAAAA|661|Railroad 3rd|Ave|Suite 40|Shiloh|Saline County|NE|69275|United States|-7|condo| +29167|AAAAAAAAPOBHAAAA|780|Williams Ash|Wy|Suite 210|Glendale|Allen County|OH|43951|United States|-5|condo| +29168|AAAAAAAAAPBHAAAA|505|Main View|Way|Suite 210|Freeman|Coke County|TX|72297|United States|-6|condo| +29169|AAAAAAAABPBHAAAA|104|North |RD|Suite 330|Oakdale|Elbert County|GA|39584|United States|-5|condo| +29170|AAAAAAAACPBHAAAA|620||Court||Fairview|Greene County||65709|United States||condo| +29171|AAAAAAAADPBHAAAA|405|4th Maple|RD|Suite 120|Clifton|Oliver County|ND|58014|United States|-6|single family| +29172|AAAAAAAAEPBHAAAA|514|Washington College|Wy|Suite 210|Macedonia|Franklin County|OH|41087|United States|-5|apartment| +29173|AAAAAAAAFPBHAAAA|74|Church |Road|Suite A|Newport|Van Wert County|OH|41521|United States|-5|apartment| +29174|AAAAAAAAGPBHAAAA|239|6th 6th|Cir.|Suite 400|Wilson|Webster County|IA|56971|United States|-6|apartment| +29175|AAAAAAAAHPBHAAAA|838|West |Pkwy|Suite 290|Jamestown|Lenawee County|MI|46867|United States|-5|single family| +29176|AAAAAAAAIPBHAAAA|237|Center Mill|Avenue|Suite G|Lakeview|Latah County|ID|88579|United States|-7|condo| +29177|AAAAAAAAJPBHAAAA|916|Spring Main|Court|Suite 270|Salem|Swift County|MN|58048|United States|-6|apartment| +29178|AAAAAAAAKPBHAAAA|393|4th Fifth|Road|Suite 10|Hillcrest|Humphreys County|MS|53003|United States|-6|single family| +29179|AAAAAAAALPBHAAAA|774|Forest |Boulevard|Suite 420|Mount Pleasant|Greeley County|NE|61933|United States|-6|condo| +29180|AAAAAAAAMPBHAAAA|73|Cherry 4th|Dr.|Suite I|Providence|Winston County|MS|56614|United States|-6|apartment| +29181|AAAAAAAANPBHAAAA|867|Elm |ST|Suite 110|Georgetown|Lincoln County|CO|87057|United States|-7|apartment| +29182|AAAAAAAAOPBHAAAA|223|Eigth |Blvd|Suite M|Salem|Cortland County|NY|18048|United States|-5|single family| +29183|AAAAAAAAPPBHAAAA|388|South 11th|Blvd|Suite 350|Centerville|Kalamazoo County|MI|40059|United States|-5|single family| +29184|AAAAAAAAAACHAAAA|168|Oak |RD|Suite M|Plainview|Dewey County|SD|53683|United States|-6|single family| +29185|AAAAAAAABACHAAAA|914|Walnut |Street|Suite L|Hardy|Bucks County|PA|15354|United States|-5|condo| +29186|AAAAAAAACACHAAAA|237|Franklin |Circle|Suite 470|Forest Hills|Calumet County|WI|59237|United States|-6|condo| +29187|AAAAAAAADACHAAAA|943|Mill |Ave|Suite 190|Oak Hill|Daviess County|MO|67838|United States|-6|single family| +29188|AAAAAAAAEACHAAAA|680|Meadow |Dr.|Suite P|Woodlawn|Crawford County|OH|44098|United States|-5|apartment| +29189|AAAAAAAAFACHAAAA|202|Lake |Cir.|Suite M|Buena Vista|Hancock County|MS|55752|United States|-6|single family| +29190|AAAAAAAAGACHAAAA|60||||Greenville||CA|91387|United States||| +29191|AAAAAAAAHACHAAAA|852|Davis Lincoln|Court|Suite V|Cedar Grove|McPherson County|NE|60411|United States|-7|apartment| +29192|AAAAAAAAIACHAAAA|934|North |Dr.|Suite K|Pleasant Valley|Fairfax County|VA|22477|United States|-5|condo| +29193|AAAAAAAAJACHAAAA|289|Ridge |Avenue|Suite W|New Hope|Pike County|OH|49431|United States|-5|apartment| +29194|AAAAAAAAKACHAAAA|708|Sunset 9th|Court|Suite 350|Five Forks|Union County|OR|92293|United States|-8|single family| +29195|AAAAAAAALACHAAAA|189|East Maple|RD|Suite 410|Stratford|Dickson County|TN|36668|United States|-5|single family| +29196|AAAAAAAAMACHAAAA|573|Hickory |Way|Suite W|Hamilton|Crittenden County|AR|72808|United States|-6|condo| +29197|AAAAAAAANACHAAAA|581|Elm 5th|Street|Suite U|Oak Ridge|Garza County|TX|78371|United States|-6|apartment| +29198|AAAAAAAAOACHAAAA|676|Poplar |ST|Suite P|Jamestown|Humboldt County|NV|86867|United States|-8|single family| +29199|AAAAAAAAPACHAAAA|244|Second Elm|ST|Suite 330|Spring Valley|Menifee County|KY|46060|United States|-5|condo| +29200|AAAAAAAAABCHAAAA|936|4th Miller|Circle|Suite 300|Woodland|Fremont County|CO|84854|United States|-7|apartment| +29201|AAAAAAAABBCHAAAA|594|Mill |Blvd|Suite L|Johnsonville|Shelby County|IA|57745|United States|-6|apartment| +29202|AAAAAAAACBCHAAAA|662|Lakeview Center|Blvd|Suite 200|Shiloh|Richmond County|GA|39275|United States|-5|apartment| +29203|AAAAAAAADBCHAAAA|107|Smith College|Cir.|Suite P|Friendship|Boone County|IL|64536|United States|-6|single family| +29204|AAAAAAAAEBCHAAAA|778||Court|||Fulton County||45124||-5|| +29205|AAAAAAAAFBCHAAAA|50|College |Circle|Suite L|Spring Valley|Erie County|NY|16060|United States|-5|condo| +29206|AAAAAAAAGBCHAAAA|732|5th |Lane|Suite 60|Union|York County|VA|28721|United States|-5|single family| +29207|AAAAAAAAHBCHAAAA|737|Willow |Boulevard|Suite 40|Union|Jefferson County|OH|48721|United States|-5|condo| +29208|AAAAAAAAIBCHAAAA|811|Spruce West|ST|Suite 190|Clifton|Hardeman County|TN|38014|United States|-5|apartment| +29209|AAAAAAAAJBCHAAAA||Mill ||Suite 120|Lewisburg|Cape Girardeau County|||United States||single family| +29210|AAAAAAAAKBCHAAAA|759|Center |Dr.|Suite N|Oak Ridge|Gogebic County|MI|48371|United States|-5|condo| +29211|AAAAAAAALBCHAAAA|361|Sunset |Way|Suite P|Farmersville|Val Verde County|TX|79305|United States|-6|condo| +29212|AAAAAAAAMBCHAAAA|227|First Oak|Circle|Suite H|Five Forks|Concho County|TX|72293|United States|-6|single family| +29213|AAAAAAAANBCHAAAA|523|Washington |Parkway|Suite C|Riverview|Muscogee County|GA|39003|United States|-5|condo| +29214|AAAAAAAAOBCHAAAA|45||Boulevard|Suite D|Kingston|Finney County||64975|United States|-6|| +29215|AAAAAAAAPBCHAAAA|421|2nd |Ct.|Suite K|Greenfield|Walworth County|WI|55038|United States|-6|apartment| +29216|AAAAAAAAACCHAAAA||||||Benton County||||-6|| +29217|AAAAAAAABCCHAAAA|684|East |Parkway|Suite 180|Newark|Winnebago County|IL|60851|United States|-6|single family| +29218|AAAAAAAACCCHAAAA|969|Third Main|Ln|Suite G|Gladstone|Clay County|NC|20894|United States|-5|condo| +29219|AAAAAAAADCCHAAAA|326|Woodland Washington|Court|Suite 320|Hamilton|Jackson County|TX|72808|United States|-6|apartment| +29220|AAAAAAAAECCHAAAA|949|Park |Parkway|Suite E|White Oak|Gilmer County|GA|36668|United States|-5|apartment| +29221|AAAAAAAAFCCHAAAA|429|Smith Poplar|Road|Suite 390|Enterprise|Clay County|WV|21757|United States|-5|apartment| +29222|AAAAAAAAGCCHAAAA|403|Oak |Circle|Suite 240|Georgetown|Brevard County|FL|37057|United States|-5|apartment| +29223|AAAAAAAAHCCHAAAA|160|Poplar Fourth|Drive|Suite 410|Nichols|Taylor County|IA|57940|United States|-6|single family| +29224|AAAAAAAAICCHAAAA|409|Sunset |Cir.|Suite 150|Mount Pleasant|Nelson County|KY|41933|United States|-5|condo| +29225|AAAAAAAAJCCHAAAA|674|4th |Ln|Suite 110|Salem|Johnson County|IL|68048|United States|-6|condo| +29226|AAAAAAAAKCCHAAAA|320|East |Dr.|Suite Q|Bath|Oscoda County|MI|40573|United States|-5|apartment| +29227|AAAAAAAALCCHAAAA|612|Twelfth |ST|Suite 70|Wesley|Cheshire County|NH|01818|United States|-5|condo| +29228|AAAAAAAAMCCHAAAA|725|Williams Ninth|Lane|Suite 380|Unionville|Elkhart County|IN|41711|United States|-5|apartment| +29229|AAAAAAAANCCHAAAA|823|Third |Pkwy|Suite 340|Silver Springs|Willacy County|TX|74843|United States|-6|condo| +29230|AAAAAAAAOCCHAAAA|910|Maple Ash|Dr.|Suite T|Glendale|Bucks County|PA|13951|United States|-5|apartment| +29231|AAAAAAAAPCCHAAAA|911|Lakeview East||Suite Y|Sleepy Hollow|Troup County|||United States||single family| +29232|AAAAAAAAADCHAAAA|502|Thirteenth |Boulevard|Suite 410|Belmont|Kimball County|NE|60191|United States|-7|condo| +29233|AAAAAAAABDCHAAAA|526|Seventh |Ln|Suite G|Edgewood|Geauga County|OH|40069|United States|-5|condo| +29234|AAAAAAAACDCHAAAA|713|Second |Pkwy|Suite 130|Forest Hills|Grainger County|TN|39237|United States|-5|condo| +29235|AAAAAAAADDCHAAAA|565|Adams Central|Dr.|Suite R|Midway|Wood County|WI|51904|United States|-6|single family| +29236|AAAAAAAAEDCHAAAA|410|Smith |Avenue|Suite C|Clinton|Swain County|NC|28222|United States|-5|single family| +29237|AAAAAAAAFDCHAAAA|252|6th |Drive|Suite G|Bridgeport|Orange County|FL|35817|United States|-5|condo| +29238|AAAAAAAAGDCHAAAA|254|8th |Street|Suite 170|Liberty|Ritchie County|WV|23451|United States|-5|apartment| +29239|AAAAAAAAHDCHAAAA|783|8th 1st|Street|Suite 110|Shady Grove|Hidalgo County|NM|82812|United States|-7|condo| +29240|AAAAAAAAIDCHAAAA|849|Johnson |RD|Suite 300|Antioch|Webster County|MS|58605|United States|-6|apartment| +29241|AAAAAAAAJDCHAAAA|454|4th |Avenue|Suite 370|Belmont|Nueces County|TX|70191|United States|-6|condo| +29242|AAAAAAAAKDCHAAAA|114|8th West|RD|Suite 120|Pine Grove|Lee County|IL|64593|United States|-6|condo| +29243|AAAAAAAALDCHAAAA|996|Fourth Sunset|Dr.|Suite 420|Stewart|Warren County|PA|18041|United States|-5|apartment| +29244|AAAAAAAAMDCHAAAA|533|Lincoln |Road|Suite 370|Union|Winnebago County|IL|68721|United States|-6|single family| +29245|AAAAAAAANDCHAAAA|415|West 3rd|RD|Suite Q|Plainview|Del Norte County|CA|93683|United States|-8|condo| +29246|AAAAAAAAODCHAAAA|189|College Maple|Way|Suite 310|Wildwood|Douglas County|NE|66871|United States|-6|condo| +29247|AAAAAAAAPDCHAAAA|241|Main |Drive|Suite I|Oakwood|Maries County|MO|60169|United States|-6|apartment| +29248|AAAAAAAAAECHAAAA|154|Oak |Drive|Suite Q|Jackson|Cumberland County|VA|29583|United States|-5|condo| +29249|AAAAAAAABECHAAAA|221|Dogwood |Parkway|Suite I|Valley View|Baraga County|MI|45124|United States|-5|condo| +29250|AAAAAAAACECHAAAA|452|3rd West|Street|Suite U|Elba|Garfield County|CO|80262|United States|-7|single family| +29251|AAAAAAAADECHAAAA|847|Oak |Road|Suite A|Taft|Erie County|OH|40589|United States|-5|single family| +29252|AAAAAAAAEECHAAAA|622|8th Walnut|Wy|Suite 470|Fairview|Judith Basin County|MT|65709|United States|-7|condo| +29253|AAAAAAAAFECHAAAA|134|Broadway |Road|Suite R|Centerville|Coweta County|GA|30059|United States|-5|single family| +29254|AAAAAAAAGECHAAAA|207|Church |Drive|Suite 60|Texas|Madison County|IA|53342|United States|-6|single family| +29255|AAAAAAAAHECHAAAA|40|Meadow |RD|Suite 390|Oak Grove|Monroe County|AR|78370|United States|-6|condo| +29256|AAAAAAAAIECHAAAA|616|Jackson South|Road|Suite T|Spring Valley|Luce County|MI|46060|United States|-5|apartment| +29257|AAAAAAAAJECHAAAA|759|Second Maple|Ave|Suite F|Northwood|Fauquier County|VA|24104|United States|-5|single family| +29258|AAAAAAAAKECHAAAA|246|Church |Drive|Suite 250|Lakewood|Cass County|TX|78877|United States|-6|single family| +29259|AAAAAAAALECHAAAA|77|West |Boulevard|Suite I|Fairfield|Bexar County|TX|76192|United States|-6|single family| +29260|AAAAAAAAMECHAAAA|377|East Fifth|Wy|Suite 340|Bethel|Clark County|MO|65281|United States|-6|condo| +29261|AAAAAAAANECHAAAA|13|Pine |Drive|Suite 230|Bridgeport|Pike County|KY|45817|United States|-5|condo| +29262|AAAAAAAAOECHAAAA|887|Valley Cedar|Street|Suite Q|Shiloh|Fairfax County|VA|29275|United States|-5|condo| +29263|AAAAAAAAPECHAAAA|270|12th Pine|RD|Suite G|King|Sacramento County|CA|90008|United States|-8|single family| +29264|AAAAAAAAAFCHAAAA|520|2nd Sycamore|Boulevard|Suite V|Newport|Prairie County|AR|71521|United States|-6|apartment| +29265|AAAAAAAABFCHAAAA|148|Meadow First|Dr.|Suite D|Union|Lewis County|NY|18721|United States|-5|condo| +29266|AAAAAAAACFCHAAAA|906|Cedar |Wy|Suite M|Wildwood|Belknap County|NH|07471|United States|-5|single family| +29267|AAAAAAAADFCHAAAA|56|Dogwood Broadway|RD|Suite J|Riverview|Montcalm County|MI|49003|United States|-5|apartment| +29268|AAAAAAAAEFCHAAAA|620|9th |Boulevard|Suite 310|Hamilton|Iosco County|MI|42808|United States|-5|single family| +29269|AAAAAAAAFFCHAAAA|842|Park |Ln|Suite A|Glendale|Carroll County|MS|53951|United States|-6|apartment| +29270|AAAAAAAAGFCHAAAA|250|Woodland Lakeview|Parkway|Suite 290|Oakdale|Colbert County|AL|39584|United States|-6|single family| +29271|AAAAAAAAHFCHAAAA|678|Spruce |Ln|Suite J|Newtown|Summit County|UT|81749|United States|-7|apartment| +29272|AAAAAAAAIFCHAAAA|288|Central |Cir.|Suite 30|Mount Pleasant|Rockingham County|VA|21933|United States|-5|apartment| +29273|AAAAAAAAJFCHAAAA|244|East View|Circle|Suite 70|Spring Valley|San Benito County|CA|96060|United States|-8|condo| +29274|AAAAAAAAKFCHAAAA|273|North |Wy|Suite 90|Midway|Marin County|CA|91904|United States|-8|single family| +29275|AAAAAAAALFCHAAAA|6|Washington Cherry|Street|Suite H|Fairfield|Lawrence County|AL|36192|United States|-6|condo| +29276|AAAAAAAAMFCHAAAA|878|East |Circle|Suite 300|Phoenix|Callahan County|TX|72276|United States|-6|single family| +29277|AAAAAAAANFCHAAAA|370|First |Court|Suite 270|Five Forks|Bristol County|MA|02893|United States|-5|condo| +29278|AAAAAAAAOFCHAAAA|264|||Suite 450|Hopewell||||||single family| +29279|AAAAAAAAPFCHAAAA|320|11th |RD|Suite 60|Fairfield|Dixon County|NE|66192|United States|-6|condo| +29280|AAAAAAAAAGCHAAAA|284|12th Johnson|Ave|Suite 210|Riverdale|Collin County|TX|79391|United States|-6|apartment| +29281|AAAAAAAABGCHAAAA|10|Oak |ST|Suite 290|Sulphur Springs|Hawkins County|TN|38354|United States|-5|apartment| +29282|AAAAAAAACGCHAAAA|366|Jefferson |Pkwy|Suite I|Macedonia|Van Buren County|AR|71087|United States|-6|apartment| +29283|AAAAAAAADGCHAAAA|572|Willow First|Boulevard|Suite 360|Walnut Grove|Fayette County|PA|17752|United States|-5|apartment| +29284|AAAAAAAAEGCHAAAA|705|Sunset Highland|RD|Suite 420|Oakwood|Morgan County|IN|40169|United States|-5|apartment| +29285|AAAAAAAAFGCHAAAA|222|4th Davis|Street|Suite A|Springdale|Morris County|NJ|09483|United States|-5|condo| +29286|AAAAAAAAGGCHAAAA|489|Meadow |Dr.|Suite B|Bridgeport|Lincoln County|NM|85817|United States|-7|condo| +29287|AAAAAAAAHGCHAAAA|302|13th |Circle|Suite 10|Harmony|Fallon County|MT|65804|United States|-7|apartment| +29288|AAAAAAAAIGCHAAAA|442|Chestnut |Lane|Suite 370|Galena|Hancock County|IL|64369|United States|-6|apartment| +29289|AAAAAAAAJGCHAAAA|747|9th 1st|Parkway|Suite J|Forest Hills|Gallatin County|MT|69237|United States|-7|condo| +29290|AAAAAAAAKGCHAAAA|681|Meadow Wilson|Court|Suite 290|Five Points|Henry County|KY|46098|United States|-6|single family| +29291|AAAAAAAALGCHAAAA|810|Third Pine|Street|Suite D|Springdale|Breathitt County|KY|48883|United States|-6|condo| +29292|AAAAAAAAMGCHAAAA|255|Park Center|Boulevard|Suite 10|Prosperity|Calhoun County|AL|39089|United States|-6|apartment| +29293|AAAAAAAANGCHAAAA|977|Park |Ln|Suite W|Red Hill|Murray County|OK|74338|United States|-6|condo| +29294|AAAAAAAAOGCHAAAA|268|Laurel Maple|ST|Suite 230|Salem|Elkhart County|IN|48048|United States|-5|apartment| +29295|AAAAAAAAPGCHAAAA|60|Main Cherry|Lane|Suite 230|White Oak|Sampson County|NC|26668|United States|-5|single family| +29296|AAAAAAAAAHCHAAAA|690|Miller |Ave|Suite Q|Lincoln|Bladen County|NC|21289|United States|-5|condo| +29297|AAAAAAAABHCHAAAA|146|Cedar 8th|Way|Suite 40|Cedar Grove|Yankton County|SD|50411|United States|-6|apartment| +29298|AAAAAAAACHCHAAAA|327|4th Church|RD|Suite H|Oak Hill|Wyoming County|WV|27838|United States|-5|single family| +29299|AAAAAAAADHCHAAAA|210|Second |RD|Suite Y|Greenwood|Eddy County|NM|88828|United States|-7|apartment| +29300|AAAAAAAAEHCHAAAA|626|Cedar 7th|Street|Suite 310|Union|Fremont County|ID|88721|United States|-7|single family| +29301|AAAAAAAAFHCHAAAA|23|14th Central|Cir.|Suite B|Gum Springs|Bartow County|GA|32106|United States|-5|apartment| +29302|AAAAAAAAGHCHAAAA|528|Lakeview Twelfth|Cir.|Suite 350|Oak Hill|Alfalfa County|OK|77838|United States|-6|single family| +29303|AAAAAAAAHHCHAAAA|297|Smith |RD|Suite 360|Pleasant Valley|Mitchell County|NC|22477|United States|-5|apartment| +29304|AAAAAAAAIHCHAAAA|722|Madison |ST|Suite A|Lakeside|Highlands County|FL|39532|United States|-5|apartment| +29305|AAAAAAAAJHCHAAAA||Forest |Way|Suite 160|Pleasant Grove||KY||United States||| +29306|AAAAAAAAKHCHAAAA|576|Cedar Miller|Avenue|Suite 380|Walnut Grove|Upson County|GA|37752|United States|-5|single family| +29307|AAAAAAAALHCHAAAA|36|7th |ST|Suite T|Franklin|Keya Paha County|NE|69101|United States|-7|apartment| +29308|AAAAAAAAMHCHAAAA|568|Fourth |Boulevard|Suite I|Forest Hills|Schuylkill County|PA|19237|United States|-5|condo| +29309|AAAAAAAANHCHAAAA|280|6th Oak|Pkwy|Suite 190|Cedar Grove|Brooke County|WV|20411|United States|-5|single family| +29310|AAAAAAAAOHCHAAAA|734|Wilson Lakeview|Way|Suite 270|Newport|Washington County|OK|71521|United States|-6|apartment| +29311|AAAAAAAAPHCHAAAA|707|2nd |Circle|Suite N|Union|Douglas County|CO|88721|United States|-7|single family| +29312|AAAAAAAAAICHAAAA|558|1st Fourth|Drive|Suite 280|Oak Ridge|Dixie County|FL|38371|United States|-5|single family| +29313|AAAAAAAABICHAAAA|173|13th Lake|Road|Suite 200|Liberty|Greenville County|SC|23451|United States|-5|single family| +29314|AAAAAAAACICHAAAA|780|5th Elm|Dr.|Suite 450|Hamilton|Saginaw County|MI|42808|United States|-5|single family| +29315|AAAAAAAADICHAAAA|337|East |RD|Suite 420|Pine Grove|York County|PA|14593|United States|-5|single family| +29316|AAAAAAAAEICHAAAA|684|Central |Wy|Suite 330|Deerfield|Beaverhead County|MT|69840|United States|-7|apartment| +29317|AAAAAAAAFICHAAAA|463|Lakeview |Road|Suite L|Mount Zion|Rice County|KS|68054|United States|-6|single family| +29318|AAAAAAAAGICHAAAA|609|Spruce Green|Road|Suite V|Oak Hill|Cass County|NE|67838|United States|-6|single family| +29319|AAAAAAAAHICHAAAA|898|Lakeview |Way|Suite 200|Pleasant Grove|Bee County|TX|74136|United States|-6|condo| +29320|AAAAAAAAIICHAAAA|805|Pine |Drive|Suite R|Providence|Lake and Peninsula Borough|AK|96614|United States|-9|apartment| +29321|AAAAAAAAJICHAAAA|22|5th Hill|Street|Suite V|Oakdale|Neosho County|KS|69584|United States|-6|single family| +29322|AAAAAAAAKICHAAAA|179|Pine Third|Road|Suite 400|Greenfield|Kent County|TX|75038|United States|-6|condo| +29323|AAAAAAAALICHAAAA|513|Maple Miller|Avenue|Suite 80|Mount Zion|Dodge County|WI|58054|United States|-6|single family| +29324|AAAAAAAAMICHAAAA|77|Elm |Cir.|Suite D|Oak Ridge|Stanley County|SD|58371|United States|-7|apartment| +29325|AAAAAAAANICHAAAA|51|Maple |Cir.|Suite I|Lakeview|Fayette County|IA|58579|United States|-6|single family| +29326|AAAAAAAAOICHAAAA|192|Main |Boulevard|Suite 450|Riverdale|Shelby County|KY|49391|United States|-5|single family| +29327|AAAAAAAAPICHAAAA|343|Broadway |Blvd|Suite S|Jackson|Coos County|OR|99583|United States|-8|apartment| +29328|AAAAAAAAAJCHAAAA|656|Lakeview Fifth|Avenue|Suite M|Brownsville|Rains County|TX|79310|United States|-6|condo| +29329|AAAAAAAABJCHAAAA|902|Green Oak|Ct.|Suite W|Walnut Grove|LaMoure County|ND|57752|United States|-6|apartment| +29330|AAAAAAAACJCHAAAA|492|Lincoln Main|ST|Suite 370|Harmony|Sioux County|IA|55804|United States|-6|single family| +29331|AAAAAAAADJCHAAAA|625|Seventh Oak|Circle|Suite M|Marion|Pope County|MN|50399|United States|-6|single family| +29332|AAAAAAAAEJCHAAAA|613|North Pine|Ave|Suite 10|Valley View|Brooke County|WV|25124|United States|-5|apartment| +29333|AAAAAAAAFJCHAAAA|96|Elm |Lane|Suite 260|Midway|Hamilton County|IN|41904|United States|-5|single family| +29334|AAAAAAAAGJCHAAAA|268|Park View|Circle|Suite 340|Westwood|Story County|IA|51715|United States|-6|condo| +29335|AAAAAAAAHJCHAAAA||||Suite 30|Newport|Solano County|CA|91521|||| +29336|AAAAAAAAIJCHAAAA|980|Oak Church|Street|Suite F|Mount Vernon|Franklin County|AR|78482|United States|-6|apartment| +29337|AAAAAAAAJJCHAAAA|618|Walnut |Boulevard|Suite 480|Hopewell|Dixon County|NE|60587|United States|-6|single family| +29338|AAAAAAAAKJCHAAAA|849|First Sunset|Court|Suite 80|Centerville|Elmore County|AL|30059|United States|-6|single family| +29339|AAAAAAAALJCHAAAA|604|Pine |Way|Suite D|Kingston|Etowah County|AL|34975|United States|-6|condo| +29340|AAAAAAAAMJCHAAAA|869|Park |Ave|Suite 410|Vista|Sumter County|GA|34694|United States|-5|apartment| +29341|AAAAAAAANJCHAAAA|736|Spring Smith|Pkwy|Suite 90|Cedar Grove|Claiborne County|TN|30411|United States|-5|condo| +29342|AAAAAAAAOJCHAAAA|702|View |Parkway|Suite A|Clifton|DeKalb County|GA|38014|United States|-5|condo| +29343|AAAAAAAAPJCHAAAA|616|Valley |Road|Suite 40|Five Forks|Richland County|MT|62293|United States|-7|condo| +29344|AAAAAAAAAKCHAAAA|527|First Pine|ST||||FL|35817|||single family| +29345|AAAAAAAABKCHAAAA|323|Spruce |Wy|Suite C|Mount Olive|Yankton County|SD|58059|United States|-6|condo| +29346|AAAAAAAACKCHAAAA|673|Walnut South|Ave|Suite 200|Green Acres|Audubon County|IA|57683|United States|-6|single family| +29347|AAAAAAAADKCHAAAA|848|Maple |Wy|Suite 50|Clinton|Jackson County|GA|38222|United States|-5|single family| +29348|AAAAAAAAEKCHAAAA|72|Green Lincoln|Boulevard|Suite E|Mount Zion|Fairfax city|VA|28054|United States|-5|single family| +29349|AAAAAAAAFKCHAAAA|590|Main 1st|Pkwy|Suite M|Clinton|Hamilton County|TN|38222|United States|-5|condo| +29350|AAAAAAAAGKCHAAAA|731|Walnut |Ln|Suite K||Waldo County||06352|||single family| +29351|AAAAAAAAHKCHAAAA|841|Pine Main|Ave|Suite R|Fox|Conejos County|CO|80631|United States|-7|apartment| +29352|AAAAAAAAIKCHAAAA|53|4th |Boulevard|Suite N|Leon|Wilbarger County|TX|70913|United States|-6|single family| +29353|AAAAAAAAJKCHAAAA|124|10th 13th|Ln|Suite V|Bunker Hill|Cass County|MN|50150|United States|-6|single family| +29354|AAAAAAAAKKCHAAAA|784|Birch |Ct.|Suite 180|Lebanon|Otero County|NM|82898|United States|-7|single family| +29355|AAAAAAAALKCHAAAA|182|North Oak|Wy|Suite S|Greenville|Payette County|ID|81387|United States|-7|apartment| +29356|AAAAAAAAMKCHAAAA|237|5th 4th|Wy|Suite O|Blair|Pasco County|FL|35465|United States|-5|condo| +29357|AAAAAAAANKCHAAAA|940|Park |Drive|Suite 270|Mount Zion|Richland County|WI|58054|United States|-6|condo| +29358|AAAAAAAAOKCHAAAA|296|Oak |Boulevard|Suite F|Green Acres|Linn County|OR|97683|United States|-8|condo| +29359|AAAAAAAAPKCHAAAA|152|Church Broadway|Road|Suite 370|Liberty|Sheridan County|KS|63451|United States|-6|apartment| +29360|AAAAAAAAALCHAAAA|995|North |Ct.|Suite C|Oak Grove|Ohio County|WV|28370|United States|-5|apartment| +29361|AAAAAAAABLCHAAAA|952|Hillcrest |Circle|Suite M|Highland Park|Natrona County|WY|86534|United States|-7|condo| +29362|AAAAAAAACLCHAAAA|230|Hickory Tenth|Boulevard|Suite 220|Mount Vernon|Mahaska County|IA|58482|United States|-6|condo| +29363|AAAAAAAADLCHAAAA|735|Sixth |Avenue|Suite F|Riverview|Warren County|NY|19003|United States|-5|condo| +29364|AAAAAAAAELCHAAAA|952|Davis |Way|Suite K|Oak Hill|Richmond County|NY|17838|United States|-5|apartment| +29365|AAAAAAAAFLCHAAAA|833|Elm |Parkway|Suite T|Greenville|Ida County|IA|51387|United States|-6|apartment| +29366|AAAAAAAAGLCHAAAA|357|10th |Way|Suite 130|Clinton|Montgomery County|AL|38222|United States|-6|apartment| +29367|AAAAAAAAHLCHAAAA|34|Locust |Parkway|Suite M|Pleasant Grove|O-Brien County|IA|54136|United States|-6|condo| +29368|AAAAAAAAILCHAAAA|576|Forest |Avenue|Suite T|Clinton|Nicollet County|MN|58222|United States|-6|single family| +29369|AAAAAAAAJLCHAAAA|102|4th Fourth|Circle|Suite 170|Mountain View|Hansford County|TX|74466|United States|-6|condo| +29370|AAAAAAAAKLCHAAAA|469|Fourth Spring|Boulevard|Suite 230|Mount Zion|Dorchester County|MD|28054|United States|-5|condo| +29371|AAAAAAAALLCHAAAA|349|Lake 9th|Lane|Suite 170|Avery|Galax city|VA|20194|United States|-5|condo| +29372|AAAAAAAAMLCHAAAA|976|Maple Second|Way|Suite 30|Sunnyside|Roosevelt County|NM|81952|United States|-7|single family| +29373|AAAAAAAANLCHAAAA|806|Johnson |Ct.|Suite E|Ashland|Hancock County|TN|34244|United States|-5|apartment| +29374|AAAAAAAAOLCHAAAA|165|Valley |Boulevard|Suite 340|Riverside|Wakulla County|FL|39231|United States|-5|condo| +29375|AAAAAAAAPLCHAAAA|849|Williams |Way|Suite V|Plainview|Wheeler County|TX|73683|United States|-6|apartment| +29376|AAAAAAAAAMCHAAAA|693|Second Franklin|Lane|Suite F|Providence|Gaines County|TX|76614|United States|-6|condo| +29377|AAAAAAAABMCHAAAA|235|Birch |Circle|Suite 340|Unionville|Randall County|TX|71711|United States|-6|apartment| +29378|AAAAAAAACMCHAAAA|298|Third |Way|Suite S|Hillcrest|Edgecombe County|NC|23003|United States|-5|single family| +29379|AAAAAAAADMCHAAAA|638|Third Highland|Ct.|Suite 480|Liberty|Scott County|KY|43451|United States|-5|condo| +29380|AAAAAAAAEMCHAAAA|94|Walnut Hill|RD|Suite 310|Mount Zion|Los Angeles County|CA|98054|United States|-8|condo| +29381|AAAAAAAAFMCHAAAA|526|11th Madison|Parkway|Suite 40|Oakdale|Hancock County|MS|59584|United States|-6|apartment| +29382|AAAAAAAAGMCHAAAA|166|Poplar |Wy|Suite J|Union Hill|Van Buren County|TN|37746|United States|-6|single family| +29383|AAAAAAAAHMCHAAAA|891|1st |Road|Suite 220|Vienna|Beaverhead County|MT|65119|United States|-7|single family| +29384|AAAAAAAAIMCHAAAA|549|9th |Ct.|Suite 350|Highland|Charlotte County|FL|39454|United States|-5|single family| +29385|AAAAAAAAJMCHAAAA|836|3rd |Way|Suite J|Centerville|Allegheny County|PA|10059|United States|-5|condo| +29386|AAAAAAAAKMCHAAAA|34|Park 3rd|Parkway|Suite K|Riverdale|Alexander County|NC|29391|United States|-5|apartment| +29387|AAAAAAAALMCHAAAA|995|Park |Avenue|Suite E|Calhoun|Lowndes County|AL|36909|United States|-6|condo| +29388|AAAAAAAAMMCHAAAA|767|2nd Green|Circle|Suite 40|Frankfort|Bullock County|AL|39681|United States|-6|single family| +29389|AAAAAAAANMCHAAAA|638|5th 9th|Boulevard|Suite T|Buena Vista|Jackson County|IN|45752|United States|-5|condo| +29390|AAAAAAAAOMCHAAAA|650|7th |Lane|Suite 30|Mount Vernon|Virginia Beach city|VA|28482|United States|-5|single family| +29391|AAAAAAAAPMCHAAAA|948|Hill |Road|Suite 40|Unity|Calhoun County|WV|28862|United States|-5|apartment| +29392|AAAAAAAAANCHAAAA|264|Hillcrest 8th|Boulevard|Suite V|Ashland|Floyd County|IA|54244|United States|-6|condo| +29393|AAAAAAAABNCHAAAA|343|Hillcrest |Street|Suite X|Green Acres|Rankin County|MS|57683|United States|-6|apartment| +29394|AAAAAAAACNCHAAAA|943|Dogwood Lee|Ln|Suite 60|Fairview|Fayette County|IA|55709|United States|-6|apartment| +29395|AAAAAAAADNCHAAAA|425|Jefferson |Ave|Suite 170|Stringtown|Greeley County|NE|60162|United States|-6|apartment| +29396|AAAAAAAAENCHAAAA|739|Hill Washington|ST|Suite 270|Clinton|Orange County|FL|38222|United States|-5|single family| +29397|AAAAAAAAFNCHAAAA|365|15th |Boulevard|Suite 40|Highland Park|Jackson County|GA|36534|United States|-5|single family| +29398|AAAAAAAAGNCHAAAA|600|Park |Drive|Suite K|Liberty|Runnels County|TX|73451|United States|-6|apartment| +29399|AAAAAAAAHNCHAAAA|193|Sunset |Boulevard|Suite 450|Five Points|Owen County|KY|46098|United States|-5|single family| +29400|AAAAAAAAINCHAAAA|561|||Suite V|||GA||United States|-5|condo| +29401|AAAAAAAAJNCHAAAA|761|2nd Hill|Avenue|Suite C|Macedonia|Livingston County|KY|41087|United States|-5|single family| +29402|AAAAAAAAKNCHAAAA|88|Railroad Locust|Way|Suite 480|Freeport|Orangeburg County|SC|21844|United States|-5|condo| +29403|AAAAAAAALNCHAAAA|264|Lake Main|Ln|Suite E|Pleasant Hill|Accomack County|VA|23604|United States|-5|condo| +29404|AAAAAAAAMNCHAAAA|195|Maple |Lane|Suite S|Five Forks|Blount County|AL|32293|United States|-6|apartment| +29405|AAAAAAAANNCHAAAA|116|Lincoln |Parkway|Suite R|Harmony|Isle of Wight County|VA|25804|United States|-5|apartment| +29406|AAAAAAAAONCHAAAA|245|Highland |Avenue|Suite H|Kirkland|Lowndes County|GA|37896|United States|-5|apartment| +29407|AAAAAAAAPNCHAAAA|969|Birch River|ST|Suite 280|Bay View|Amador County|CA|96457|United States|-8|condo| +29408|AAAAAAAAAOCHAAAA|118|Washington |Way|Suite T|Brownsville|Red River Parish|LA|79310|United States|-6|apartment| +29409|AAAAAAAABOCHAAAA|481|Adams |Ct.|Suite 410|Sulphur Springs|Linn County|IA|58354|United States|-6|condo| +29410|AAAAAAAACOCHAAAA|244|Lincoln Laurel|Court|Suite 190|Friendship|Scott County|IA|54536|United States|-6|condo| +29411|AAAAAAAADOCHAAAA|948|4th Valley|Ave|Suite 200|Crossroads|Scott County|MO|60534|United States|-6|condo| +29412|AAAAAAAAEOCHAAAA|144|Maple 11th|Circle|Suite 120|Five Forks|Palo Alto County|IA|52293|United States|-6|apartment| +29413|AAAAAAAAFOCHAAAA|776|5th |Street|Suite 250|Highland Park|Erath County|TX|76534|United States|-6|apartment| +29414|AAAAAAAAGOCHAAAA|216|4th |Cir.|Suite 150|Jamestown|Williamson County|TX|76867|United States|-6|single family| +29415|AAAAAAAAHOCHAAAA|669|8th |Drive|Suite M|Rosebud|Blount County|AL|32244|United States|-6|apartment| +29416|AAAAAAAAIOCHAAAA|946|4th |Dr.|Suite R|Oakwood|Norfolk County|MA|00769|United States|-5|apartment| +29417|AAAAAAAAJOCHAAAA|381|Lincoln |Road|Suite 140|Patterson|Concordia Parish|LA|73175|United States|-6|condo| +29418|AAAAAAAAKOCHAAAA|958|13th |Blvd|Suite 400|Watkins|Wake County|NC|21732|United States|-5|condo| +29419|AAAAAAAALOCHAAAA|306|9th |Wy|Suite D|Jackson|Calloway County|KY|49583|United States|-6|single family| +29420|AAAAAAAAMOCHAAAA|325|Laurel Cedar|Blvd|Suite A|Belmont|McDonough County|IL|60191|United States|-6|apartment| +29421|AAAAAAAANOCHAAAA|998|Hickory Jackson|Court|Suite 450|Pierce|Bath County|VA|23360|United States|-5|condo| +29422|AAAAAAAAOOCHAAAA|294|Second |Way|Suite 490|Stringtown|Morrow County|OR|90162|United States|-8|apartment| +29423|AAAAAAAAPOCHAAAA|148|Fourteenth |Ct.|Suite 30|Georgetown|Stark County|ND|57057|United States|-6|single family| +29424|AAAAAAAAAPCHAAAA|893|Lincoln Lake|Way|Suite Y|Oak Ridge|Breathitt County|KY|48371|United States|-6|apartment| +29425|AAAAAAAABPCHAAAA|825|Chestnut |Circle|Suite 220|Bethesda|Warren County|KY|45980|United States|-5|condo| +29426|AAAAAAAACPCHAAAA|104|14th 6th|Circle|Suite 190|Kingston|Windham County|VT|05575|United States|-5|condo| +29427|AAAAAAAADPCHAAAA|199|7th 3rd|Avenue|Suite S|Marion|Somerset County|MD|20399|United States|-5|condo| +29428|AAAAAAAAEPCHAAAA|833|Hill |ST|Suite V|Newport|Monroe County|IN|41521|United States|-5|condo| +29429|AAAAAAAAFPCHAAAA|907|Sixth |Pkwy|Suite 20|Deerfield|Dewey County|SD|59840|United States|-6|single family| +29430|AAAAAAAAGPCHAAAA|618|8th |Circle|Suite 300|Greenwood|Ransom County|ND|58828|United States|-6|apartment| +29431|AAAAAAAAHPCHAAAA|179|West Elm|Pkwy|Suite V|New Hope|Huron County|MI|49431|United States|-5|apartment| +29432|AAAAAAAAIPCHAAAA|659|2nd Valley|RD|Suite X|Riverdale|Shawano County|WI|59391|United States|-6|condo| +29433|AAAAAAAAJPCHAAAA|706|Lake Davis|Lane|Suite 390|Woodland|Essex County|NJ|05454|United States|-5|apartment| +29434|AAAAAAAAKPCHAAAA|554|Main 14th|Blvd|Suite 450|Pleasant Grove|Howard County|IN|44136|United States|-5|single family| +29435|AAAAAAAALPCHAAAA|796|7th Cedar|Wy|Suite 360|Summit|Catahoula Parish|LA|70499|United States|-6|single family| +29436|AAAAAAAAMPCHAAAA|417|Birch |Court|Suite J|Shiloh|Kingsbury County|SD|59275|United States|-7|apartment| +29437|AAAAAAAANPCHAAAA|853|River |Pkwy|Suite 150|Lakewood|Delaware County|PA|18877|United States|-5|apartment| +29438|AAAAAAAAOPCHAAAA|839|Lake Spruce|Ln|Suite 430|Hamilton|Lawrence County|AR|72808|United States|-6|condo| +29439|AAAAAAAAPPCHAAAA|449|Spring |Street|Suite 130|Pleasant Grove|Jefferson County|OR|94136|United States|-8|apartment| +29440|AAAAAAAAAADHAAAA|429|13th River|Avenue|Suite 10|Mount Pleasant|Marshall County|MN|51933|United States|-6|condo| +29441|AAAAAAAABADHAAAA|741|5th |Parkway|Suite J|Fairview|Caldwell County|TX|75709|United States|-6|apartment| +29442|AAAAAAAACADHAAAA|467|Miller 2nd|Blvd|Suite O|Springtown|Schleicher County|TX|79858|United States|-6|condo| +29443|AAAAAAAADADHAAAA|957|Oak North|Dr.|Suite 400|Glendale|Marengo County|AL|33951|United States|-6|condo| +29444|AAAAAAAAEADHAAAA|969|Jackson Main|Wy|Suite M|Centerville|Tate County|MS|50059|United States|-6|single family| +29445|AAAAAAAAFADHAAAA|485|Railroad View|Street|Suite 170|Newtown|Cameron County|TX|71749|United States|-6|apartment| +29446|AAAAAAAAGADHAAAA|564|Locust |Cir.|Suite T|Arlington|Delaware County|IA|56557|United States|-6|apartment| +29447|AAAAAAAAHADHAAAA|127|Tenth |RD|Suite 220|Enterprise|Stevens County|KS|61757|United States|-6|condo| +29448|AAAAAAAAIADHAAAA|993|Chestnut 10th||Suite 150||Marion County|TX|78014|United States||single family| +29449|AAAAAAAAJADHAAAA|319|8th |Ct.|Suite S|Pleasant Hill|Butler County|PA|13604|United States|-5|condo| +29450|AAAAAAAAKADHAAAA|182|Valley |Lane|Suite C|Bunker Hill|Wilkes County|GA|30150|United States|-5|condo| +29451|AAAAAAAALADHAAAA|446|Green |RD|Suite 50|Pleasant Grove|Garfield County|WA|94136|United States|-8|single family| +29452|AAAAAAAAMADHAAAA|749|Fourth |Court|Suite G|Marion|Rusk County|WI|50399|United States|-6|condo| +29453|AAAAAAAANADHAAAA|432|Highland Fifth|Parkway|Suite O|Shelby|Colfax County|NM|86575|United States|-7|apartment| +29454|AAAAAAAAOADHAAAA|515|Second Spruce|Ct.|Suite T|Royal|Washington County|VA|25819|United States|-5|condo| +29455|AAAAAAAAPADHAAAA||2nd ||Suite C|||NH|||-5|condo| +29456|AAAAAAAAABDHAAAA|50|Elm |Dr.|Suite 410|Unionville|Mackinac County|MI|41711|United States|-5|condo| +29457|AAAAAAAABBDHAAAA|41|Walnut View|Wy|Suite D|Forest Hills|Bosque County|TX|79237|United States|-6|apartment| +29458|AAAAAAAACBDHAAAA|281|Park Pine|Ln|Suite D|Newport|Clinton County|KY|41521|United States|-6|single family| +29459|AAAAAAAADBDHAAAA|383|Hill 2nd|Pkwy|Suite 70|Antioch|Chesterfield County|SC|28605|United States|-5|condo| +29460|AAAAAAAAEBDHAAAA|902|4th |RD|Suite 370|Oakwood|White County|IL|60169|United States|-6|condo| +29461|AAAAAAAAFBDHAAAA|751|Ridge Sycamore|Wy|Suite 480|Arcola|Grant County|MN|51654|United States|-6|single family| +29462|AAAAAAAAGBDHAAAA|187|Second |Ave|Suite X|Mount Pleasant|Russell County|VA|21933|United States|-5|single family| +29463|AAAAAAAAHBDHAAAA|540|6th |Ave|Suite R|Greenfield|Del Norte County|CA|95038|United States|-8|condo| +29464|AAAAAAAAIBDHAAAA|735|11th |Ln|Suite 90|Liberty|Putnam County|OH|43451|United States|-5|apartment| +29465|AAAAAAAAJBDHAAAA|487|Washington |Lane|Suite D|Green Acres|Belmont County|OH|47683|United States|-5|single family| +29466|AAAAAAAAKBDHAAAA|541|Hillcrest Oak|Drive|Suite I|Sunnyside|Martinsville city|VA|21952|United States|-5|single family| +29467|AAAAAAAALBDHAAAA|871|Miller Johnson|Ct.|Suite 350|Glendale|Coryell County|TX|73951|United States|-6|apartment| +29468|AAAAAAAAMBDHAAAA|369|River 4th|Ln|Suite Q|Pleasant Hill|Howard County|AR|73604|United States|-6|apartment| +29469|AAAAAAAANBDHAAAA|640|Cherry |Ln|Suite 210|Five Points|Johnson County|MO|66098|United States|-6|condo| +29470|AAAAAAAAOBDHAAAA|528|Madison |Avenue|Suite T|Lakeview|New Kent County|VA|28579|United States|-5|apartment| +29471|AAAAAAAAPBDHAAAA|495|2nd 12th|Court|Suite 10|Mount Vernon|Sitka Borough|AK|98482|United States|-9|apartment| +29472|AAAAAAAAACDHAAAA||Willow |||White Oak||||United States||| +29473|AAAAAAAABCDHAAAA|400|Cherry Church|Ln|Suite W|Fairview|Sharp County|AR|75709|United States|-6|condo| +29474|AAAAAAAACCDHAAAA|645|Maple Maple|Pkwy|Suite T|Jamestown|Marion County|MO|66867|United States|-6|apartment| +29475|AAAAAAAADCDHAAAA|475|5th |ST|Suite G|Mount Zion|Gasconade County|MO|68054|United States|-6|apartment| +29476|AAAAAAAAECDHAAAA|913|Cherry View|Avenue|Suite 210|Mount Zion|Louisa County|IA|58054|United States|-6|apartment| +29477|AAAAAAAAFCDHAAAA|357|Jefferson Valley|Street|Suite Q|Martin|Poweshiek County|IA|50679|United States|-6|condo| +29478|AAAAAAAAGCDHAAAA|932|Walnut |Street|Suite 80|Ashland|Taylor County|GA|34244|United States|-5|apartment| +29479|AAAAAAAAHCDHAAAA|452|14th |Street|Suite 280|Glenwood|Page County|VA|23511|United States|-5|single family| +29480|AAAAAAAAICDHAAAA|511|Central Maple|Ct.|Suite 200|Lebanon|Taliaferro County|GA|32898|United States|-5|apartment| +29481|AAAAAAAAJCDHAAAA|353|West Church|Road|Suite E|Oakland|Columbia County|NY|19843|United States|-5|condo| +29482|AAAAAAAAKCDHAAAA|439|9th 12th|Way|Suite 80|Belfast|Dorchester County|SC|20125|United States|-5|condo| +29483|AAAAAAAALCDHAAAA|861|East Central|ST|Suite 130|Woodlawn|Hardin County|TX|74098|United States|-6|condo| +29484|AAAAAAAAMCDHAAAA|371|Woodland Johnson|Avenue|Suite 440|Hopewell|Putnam County|FL|30587|United States|-5|apartment| +29485|AAAAAAAANCDHAAAA|361|Railroad Lake|Court|Suite M|Florence|Augusta County|VA|23394|United States|-5|apartment| +29486|AAAAAAAAOCDHAAAA|703|College |Road|Suite 240|Pine Grove|Alcorn County|MS|54593|United States|-6|condo| +29487|AAAAAAAAPCDHAAAA|224|Meadow Hickory|Wy|Suite 20|Wildwood|Rosebud County|MT|66871|United States|-7|single family| +29488|AAAAAAAAADDHAAAA|139|8th |Ln|Suite E|Pine Grove|Crawford County|WI|54593|United States|-6|apartment| +29489|AAAAAAAABDDHAAAA|724|Church |Street|Suite M|Maple Grove|Mason County|IL|68252|United States|-6|condo| +29490|AAAAAAAACDDHAAAA|611|Ridge Park|Road|Suite D|Crystal|Randolph County|IL|65258|United States|-6|single family| +29491|AAAAAAAADDDHAAAA|857|West Washington|Cir.|Suite 100|Pleasant Grove|Madison County|IL|64136|United States|-6|single family| +29492|AAAAAAAAEDDHAAAA|614|North Second|Pkwy|Suite H|Mount Zion|Fayette County|GA|38054|United States|-5|single family| +29493|AAAAAAAAFDDHAAAA|840|7th |Avenue|Suite 380|Crossroads|Clackamas County|OR|90534|United States|-8|single family| +29494|AAAAAAAAGDDHAAAA|516|Highland |Lane|Suite V|Oakwood|Bond County|IL|60169|United States|-6|apartment| +29495|AAAAAAAAHDDHAAAA|250|Hickory Sunset|Drive|Suite 300|Mount Pleasant|Hayes County|NE|61933|United States|-6|single family| +29496|AAAAAAAAIDDHAAAA|940|North |Blvd|Suite 140|Pine Grove|Coweta County|GA|34593|United States|-5|condo| +29497|AAAAAAAAJDDHAAAA|48|Lee 2nd|Boulevard|Suite 400|Lakeside|Jackson County|WV|29532|United States|-5|condo| +29498|AAAAAAAAKDDHAAAA|519|Hill 10th|Court|Suite 230|Harmony|Noble County|OH|45804|United States|-5|apartment| +29499|AAAAAAAALDDHAAAA|252|Second |Circle|Suite 230|Red Hill|Long County|GA|34338|United States|-5|apartment| +29500|AAAAAAAAMDDHAAAA|871|Seventh |Blvd|Suite 160|Lakeside|Labette County|KS|69532|United States|-6|single family| +29501|AAAAAAAANDDHAAAA|373|Main Ash|Boulevard|Suite Q|Concord|Hardin County|TX|74107|United States|-6|condo| +29502|AAAAAAAAODDHAAAA|199|Mill Meadow|Parkway|Suite F|Unionville|Pearl River County|MS|51711|United States|-6|condo| +29503|AAAAAAAAPDDHAAAA|402|Elm Sunset|Wy|Suite 400|Enterprise|Marion County|OR|91757|United States|-8|condo| +29504|AAAAAAAAAEDHAAAA|2|Fourth |Avenue|Suite S|Cedar|Marshall County|SD|51229|United States|-7|condo| +29505|AAAAAAAABEDHAAAA|729|Third Maple|Street|Suite 380|Salem|Wetzel County|WV|28048|United States|-5|condo| +29506|AAAAAAAACEDHAAAA|195|Second Willow|Road|Suite 150|Florence|Charles Mix County|SD|53394|United States|-6|apartment| +29507|AAAAAAAADEDHAAAA|||Boulevard||||WV|24244|United States||condo| +29508|AAAAAAAAEEDHAAAA|187|Ridge |Parkway|Suite I|Valley View|Wahkiakum County|WA|95124|United States|-8|apartment| +29509|AAAAAAAAFEDHAAAA|564|Willow Maple|Circle|Suite 40|Mount Pleasant|Somerset County|NJ|02533|United States|-5|apartment| +29510|AAAAAAAAGEDHAAAA|130|Smith |Street|Suite N|Cedar Grove|Obion County|TN|30411|United States|-6|apartment| +29511|AAAAAAAAHEDHAAAA|857|13th |Ct.|Suite U|Hamilton|Monroe County|MI|42808|United States|-5|apartment| +29512|AAAAAAAAIEDHAAAA|958|Green |Blvd|Suite R|Hillcrest|Linn County|OR|93003|United States|-8|single family| +29513|AAAAAAAAJEDHAAAA|761|Railroad Birch|Ct.|Suite 350|Greenville|San Luis Obispo County|CA|91387|United States|-8|apartment| +29514|AAAAAAAAKEDHAAAA|5|Ridge |Lane|Suite T|Tremont|Furnas County|NE|69515|United States|-6|condo| +29515|AAAAAAAALEDHAAAA|827|5th Dogwood|Pkwy|Suite H|Five Forks|Kandiyohi County|MN|52293|United States|-6|single family| +29516|AAAAAAAAMEDHAAAA|804|Washington |Ln|Suite Y|New Hope|Jefferson County|AL|39431|United States|-6|apartment| +29517|AAAAAAAANEDHAAAA|65|3rd |Street|Suite H|Oak Ridge|Amite County|MS|58371|United States|-6|single family| +29518|AAAAAAAAOEDHAAAA|116|Elm |ST|Suite W|Newtown|Hartford County|CT|02349|United States|-5|apartment| +29519|AAAAAAAAPEDHAAAA|178|Lee Cedar|Ln|Suite 70|Redland|Candler County|GA|36343|United States|-5|apartment| +29520|AAAAAAAAAFDHAAAA|796|Pine |Pkwy|Suite C|Brownsville|Pulaski County|GA|39310|United States|-5|single family| +29521|AAAAAAAABFDHAAAA|747|Cedar |Court|Suite M|Five Forks|Putnam County|FL|32293|United States|-5|condo| +29522|AAAAAAAACFDHAAAA|200|Valley |Street|Suite 120|Jackson|Hartley County|TX|79583|United States|-6|condo| +29523|AAAAAAAADFDHAAAA|410|Spruce |Ct.|Suite X|Concord|Essex County|NY|14107|United States|-5|apartment| +29524|AAAAAAAAEFDHAAAA|771|Ridge |Way|Suite 50|Belmont|Hancock County|MS|50191|United States|-6|condo| +29525|AAAAAAAAFFDHAAAA|716|Elm Ash|Circle|Suite 390|Valley View|Pennington County|SD|55124|United States|-7|single family| +29526|AAAAAAAAGFDHAAAA|587|College |Ln|Suite C|Unionville|Calhoun County|TX|71711|United States|-6|apartment| +29527|AAAAAAAAHFDHAAAA|694|Cedar |Way|Suite F|Oak Ridge|Staunton city|VA|28371|United States|-5|single family| +29528|AAAAAAAAIFDHAAAA|342|Seventh |Road|Suite 160|Oak Grove|Coffee County|GA|38370|United States|-5|apartment| +29529|AAAAAAAAJFDHAAAA|920|Cherry |Avenue|Suite T|Midway|Spartanburg County|SC|21904|United States|-5|condo| +29530|AAAAAAAAKFDHAAAA|387|10th College|Drive|Suite P|Woodlawn|Essex County|NJ|04698|United States|-5|single family| +29531|AAAAAAAALFDHAAAA|704|||Suite Q|Ashland||KY|44244|United States||apartment| +29532|AAAAAAAAMFDHAAAA|413|Railroad South|Boulevard|Suite 330|Hopewell|Pierce County|WA|90587|United States|-8|condo| +29533|AAAAAAAANFDHAAAA|139|North Valley|Street|Suite Q|Woodville|Eagle County|CO|84289|United States|-7|single family| +29534|AAAAAAAAOFDHAAAA|782|First |ST|Suite 220|Enterprise|Tom Green County|TX|71757|United States|-6|single family| +29535|AAAAAAAAPFDHAAAA|634|5th Hickory|Avenue|Suite 330|Maple Grove|Warren County|IN|48252|United States|-5|condo| +29536|AAAAAAAAAGDHAAAA|643|Pine 4th|Boulevard|Suite L|Georgetown|Benton County|IN|47057|United States|-5|apartment| +29537|AAAAAAAABGDHAAAA|773|Sixth 7th|Drive|Suite H|Pleasant Valley|Esmeralda County|NV|82477|United States|-8|apartment| +29538|AAAAAAAACGDHAAAA|849|Second |Dr.|Suite M|Shady Grove|Logan County|OK|72812|United States|-6|condo| +29539|AAAAAAAADGDHAAAA|510|Central |Circle|Suite 420|Walnut Grove|Ringgold County|IA|57752|United States|-6|apartment| +29540|AAAAAAAAEGDHAAAA|790|3rd Pine|Ln|Suite 250|Jackson|Boyle County|KY|49583|United States|-6|apartment| +29541|AAAAAAAAFGDHAAAA|465|Second |Blvd|Suite J|Deerfield|Knox County|MO|69840|United States|-6|apartment| +29542|AAAAAAAAGGDHAAAA|948|College |Drive|Suite 230|Providence|Montgomery County|TN|36614|United States|-6|single family| +29543|AAAAAAAAHGDHAAAA|397|Main 15th|Avenue|Suite J|Kingston|Shannon County|SD|54975|United States|-7|apartment| +29544|AAAAAAAAIGDHAAAA|112|North |RD|Suite 230|Union|San Bernardino County|CA|98721|United States|-8|condo| +29545|AAAAAAAAJGDHAAAA|240|Franklin Highland|Court|Suite O|Spring Hill|Swisher County|TX|76787|United States|-6|apartment| +29546|AAAAAAAAKGDHAAAA|666|College |Blvd|Suite D|Clifton|Lafayette Parish|LA|78014|United States|-6|condo| +29547|AAAAAAAALGDHAAAA|162|Church |Blvd|Suite D|Lakeview|Tioga County|PA|18579|United States|-5|apartment| +29548|AAAAAAAAMGDHAAAA|785|Valley Sycamore|RD|Suite O|Hartland|Owen County|KY|46594|United States|-5|apartment| +29549|AAAAAAAANGDHAAAA|773|9th Maple|Avenue|Suite W|Oakland|Crane County|TX|79843|United States|-6|condo| +29550|AAAAAAAAOGDHAAAA|655|West Church|Parkway|Suite 160|The Meadows|Nicholas County|KY|40026|United States|-5|apartment| +29551|AAAAAAAAPGDHAAAA|494|Davis Center|||The Meadows|Sheridan County|NE|60026|United States|-7|| +29552|AAAAAAAAAHDHAAAA|911|Davis Cedar|Parkway|Suite P|Lakeview|Reagan County|TX|78579|United States|-6|apartment| +29553|AAAAAAAABHDHAAAA|713|7th 3rd|ST|Suite D|Acme|Hardeman County|TX|70164|United States|-6|single family| +29554|AAAAAAAACHDHAAAA|861|View |Drive|Suite 220|Hopewell|Cape May County|NJ|01187|United States|-5|apartment| +29555|AAAAAAAADHDHAAAA|839|Wilson |Boulevard|Suite 190|Red Hill|Essex County|VA|24338|United States|-5|apartment| +29556|AAAAAAAAEHDHAAAA|154|First Jefferson|Boulevard|Suite 120|Franklin|Shawano County|WI|59101|United States|-6|apartment| +29557|AAAAAAAAFHDHAAAA|718|Miller |Drive|Suite 170|Five Points|Franklin County|IA|56098|United States|-6|single family| +29558|AAAAAAAAGHDHAAAA|240|Sunset |Ct.|Suite A|Shady Grove|Minnehaha County|SD|52812|United States|-7|single family| +29559|AAAAAAAAHHDHAAAA|937|Hill Willow|Lane|Suite 480|Newport|Lamar County|TX|71521|United States|-6|condo| +29560|AAAAAAAAIHDHAAAA|189|Forest |RD|Suite H|Providence|Big Horn County|WY|86614|United States|-7|single family| +29561|AAAAAAAAJHDHAAAA|457|North |ST|Suite O|Millbrook|Jasper County|TX|77529|United States|-6|apartment| +29562|AAAAAAAAKHDHAAAA|258|Thirteenth |Blvd|Suite 280|Edgewood|Bowie County|TX|70069|United States|-6|apartment| +29563|AAAAAAAALHDHAAAA|216|Walnut |Court|Suite 430|Oakland|Rockwall County|TX|79843|United States|-6|condo| +29564|AAAAAAAAMHDHAAAA|96|First |Cir.|Suite G|Harmony|Pulaski County|IL|65804|United States|-6|single family| +29565|AAAAAAAANHDHAAAA|458|Washington |Lane|Suite 390|Fairfield|Sacramento County|CA|96192|United States|-8|single family| +29566|AAAAAAAAOHDHAAAA|416|College |Ct.|Suite 350|Vienna|Wayne County|GA|35119|United States|-5|apartment| +29567|AAAAAAAAPHDHAAAA||||Suite 90|||WA||United States||single family| +29568|AAAAAAAAAIDHAAAA|100|4th |Court|Suite C|Lakeview|Montgomery County|NY|18579|United States|-5|apartment| +29569|AAAAAAAABIDHAAAA|25|Sycamore |Avenue|Suite 200|Lebanon|Leon County|FL|32898|United States|-5|apartment| +29570|AAAAAAAACIDHAAAA|137|Park |Parkway|Suite J|Centerville|Adams County|ID|80059|United States|-7|single family| +29571|AAAAAAAADIDHAAAA|979|Ridge Church|Blvd|Suite 100|Highland|El Paso County|CO|89454|United States|-7|apartment| +29572|AAAAAAAAEIDHAAAA|473|Church Laurel|ST|Suite 270|Oak Grove|Appomattox County|VA|28370|United States|-5|apartment| +29573|AAAAAAAAFIDHAAAA|230|Forest 1st|Road|Suite 30|Spring Hill|Macon County|AL|36787|United States|-6|single family| +29574|AAAAAAAAGIDHAAAA|389|Locust |Parkway|Suite 150|Pleasant Grove|Graham County|KS|64136|United States|-6|apartment| +29575|AAAAAAAAHIDHAAAA|68|Pine |Wy|Suite S|Midway|Baker County|OR|91904|United States|-8|apartment| +29576|AAAAAAAAIIDHAAAA|316|Church Main|RD|Suite 160|Brownsville|Burlington County|NJ|09910|United States|-5|single family| +29577|AAAAAAAAJIDHAAAA|539|Birch Cherry|Circle|Suite T|Marion|Benton County|IN|40399|United States|-5|apartment| +29578|AAAAAAAAKIDHAAAA|952|Fifth |Cir.|Suite I|Providence|Grady County|GA|36614|United States|-5|apartment| +29579|AAAAAAAALIDHAAAA|760|Park Elm|Way|Suite P|Pleasant Grove|Chambers County|AL|34136|United States|-6|condo| +29580|AAAAAAAAMIDHAAAA|904|Central |Boulevard|Suite A|Walnut Grove|Holmes County|FL|37752|United States|-5|condo| +29581|AAAAAAAANIDHAAAA||North Laurel|Lane|Suite 80|||||||| +29582|AAAAAAAAOIDHAAAA|981|Franklin Valley|ST||Five Forks|Monroe County|||United States||| +29583|AAAAAAAAPIDHAAAA|749|Washington Dogwood|Court|Suite 160|Greenwood|Muskegon County|MI|48828|United States|-5|apartment| +29584|AAAAAAAAAJDHAAAA|900|6th |ST|Suite 330|Globe|Saline County|IL|69614|United States|-6|condo| +29585|AAAAAAAABJDHAAAA|964|Pine |Road|Suite I|Mount Vernon|Jefferson County|IN|48482|United States|-5|single family| +29586|AAAAAAAACJDHAAAA|665|12th |Boulevard|Suite O|Rosewood|Pepin County|WI|52205|United States|-6|condo| +29587|AAAAAAAADJDHAAAA|933|13th |Way|Suite 40|Sunnyside|Marathon County|WI|51952|United States|-6|condo| +29588|AAAAAAAAEJDHAAAA|770|Pine |RD|Suite 110|Kingston|Mitchell County|GA|34975|United States|-5|condo| +29589|AAAAAAAAFJDHAAAA|457|Franklin Madison|Road|Suite O|Mount Vernon|Lincoln County|WY|88482|United States|-7|condo| +29590|AAAAAAAAGJDHAAAA|217|Hillcrest 1st|Dr.|Suite 460|Fairfield|New Haven County|CT|06792|United States|-5|condo| +29591|AAAAAAAAHJDHAAAA|11|Main Broadway|Court|Suite 270|Glenwood|Pulaski County|VA|23511|United States|-5|apartment| +29592|AAAAAAAAIJDHAAAA|658|Elm Lincoln|Court|Suite 80|Clifton|Chisago County|MN|58014|United States|-6|condo| +29593|AAAAAAAAJJDHAAAA|518|Washington 13th|Ct.|Suite H|Salem|Fannin County|GA|38048|United States|-5|single family| +29594|AAAAAAAAKJDHAAAA|579|Spring View|Wy|Suite 380|Concord|Murray County|MN|54107|United States|-6|single family| +29595|AAAAAAAALJDHAAAA|395|Oak |Road|Suite 190|Marion|Morgan County|UT|80399|United States|-7|condo| +29596|AAAAAAAAMJDHAAAA|822|Seventh |Boulevard|Suite Q|Farmington|Garvin County|OK|79145|United States|-6|apartment| +29597|AAAAAAAANJDHAAAA|712|Washington |Drive|Suite 200|Lakeside|Preble County|OH|49532|United States|-5|single family| +29598|AAAAAAAAOJDHAAAA|446|Highland View|Cir.|Suite F|King|Pueblo County|CO|80008|United States|-7|apartment| +29599|AAAAAAAAPJDHAAAA|3|Sunset Lincoln|Parkway|Suite G|Providence|Kaufman County|TX|76614|United States|-6|condo| +29600|AAAAAAAAAKDHAAAA||||Suite H|||||||single family| +29601|AAAAAAAABKDHAAAA|961|Church |Cir.|Suite P|Providence|Worth County|MO|66614|United States|-6|apartment| +29602|AAAAAAAACKDHAAAA|75|Spring |Blvd|Suite I|Greenville|Marshall County|OK|71387|United States|-6|apartment| +29603|AAAAAAAADKDHAAAA|952|Third |Road|Suite 170|Highland Park|Thayer County|NE|66534|United States|-7|condo| +29604|AAAAAAAAEKDHAAAA|474|7th |Ln|Suite V|Stewart|Baltimore County|MD|28041|United States|-5|single family| +29605|AAAAAAAAFKDHAAAA|374|Meadow 8th|Ct.|Suite K|Mountain View|Wilson County|NC|24466|United States|-5|condo| +29606|AAAAAAAAGKDHAAAA|650|Wilson Wilson|ST|Suite 260|Shiloh|Morgan County|IL|69275|United States|-6|condo| +29607|AAAAAAAAHKDHAAAA|465|Franklin Park|Avenue|Suite C|Derby|Missoula County|MT|67702|United States|-7|single family| +29608|AAAAAAAAIKDHAAAA|33|College 5th|RD|Suite 70|Franklin|El Paso County|CO|89101|United States|-7|condo| +29609|AAAAAAAAJKDHAAAA|634|Franklin |Ln|Suite J|Plainview|Treasure County|MT|63683|United States|-7|single family| +29610|AAAAAAAAKKDHAAAA|195|Franklin Adams|Avenue|Suite I|Riverdale|Steuben County|NY|19391|United States|-5|condo| +29611|AAAAAAAALKDHAAAA|445|Hill River|Blvd|Suite O|Edgewood|Vernon Parish|LA|70069|United States|-6|apartment| +29612|AAAAAAAAMKDHAAAA|996|14th 1st|Pkwy|Suite 260|Clinton|Boyd County|NE|68222|United States|-6|apartment| +29613|AAAAAAAANKDHAAAA|828|Fourth Maple|Dr.|Suite 360|Greenfield|Bristol County|MA|05638|United States|-5|single family| +29614|AAAAAAAAOKDHAAAA|554|Lakeview Oak|Blvd|Suite 110|Rankin|Dunn County|WI|52621|United States|-6|single family| +29615|AAAAAAAAPKDHAAAA|648|3rd Maple|RD|Suite M|Oakland|Forest County|PA|19843|United States|-5|condo| +29616|AAAAAAAAALDHAAAA|298|Hill |Ave|Suite 320|Summit|Alcorn County|MS|50499|United States|-6|condo| +29617|AAAAAAAABLDHAAAA|886|Johnson |Cir.|Suite O|Springfield|Bernalillo County|NM|89303|United States|-7|apartment| +29618|AAAAAAAACLDHAAAA||Hill |||||IN||United States||apartment| +29619|AAAAAAAADLDHAAAA|710|Cedar Maple|Cir.|Suite W|Stringtown|Thomas County|GA|30162|United States|-5|condo| +29620|AAAAAAAAELDHAAAA|776|Cherry |Ln|Suite 260|Lebanon|Big Horn County|WY|82898|United States|-7|condo| +29621|AAAAAAAAFLDHAAAA|277|4th Hill|Blvd|Suite X|Kingston|Clinton County|IA|54975|United States|-6|apartment| +29622|AAAAAAAAGLDHAAAA|376|Spring Sycamore|Blvd|Suite 490|Greenwood|Wells County|IN|48828|United States|-5|single family| +29623|AAAAAAAAHLDHAAAA|712|First |Pkwy|Suite U|Edgewood|Boyd County|NE|60069|United States|-6|condo| +29624|AAAAAAAAILDHAAAA|389|East Broadway|Court|Suite 240|Oak Hill|Hill County|TX|77838|United States|-6|condo| +29625|AAAAAAAAJLDHAAAA|535|Washington Green|Lane|Suite 170|Richfield|Colusa County|CA|96196|United States|-8|apartment| +29626|AAAAAAAAKLDHAAAA|405|Park |Wy|Suite 210|Longwood|Staunton city|VA|27021|United States|-5|single family| +29627|AAAAAAAALLDHAAAA|33|Woodland Miller|Wy|Suite E|Enterprise|Gaston County|NC|21757|United States|-5|condo| +29628|AAAAAAAAMLDHAAAA|984|Laurel 15th|ST|Suite R|Sunnyside|New Kent County|VA|21952|United States|-5|apartment| +29629|AAAAAAAANLDHAAAA|550|Walnut 7th|Lane|Suite 440|Springfield|Haakon County|SD|59303|United States|-7|apartment| +29630|AAAAAAAAOLDHAAAA|595|East |Circle|Suite C|Fairfield|Simpson County|KY|46192|United States|-5|apartment| +29631|AAAAAAAAPLDHAAAA|999|Washington Hillcrest|Cir.|Suite X|Lakeside|Rockdale County|GA|39532|United States|-5|apartment| +29632|AAAAAAAAAMDHAAAA|783|College Fifth|Circle|Suite W|Rockwood|Stanislaus County|CA|91545|United States|-8|condo| +29633|AAAAAAAABMDHAAAA||Church Elm|||Marion||||United States|-6|single family| +29634|AAAAAAAACMDHAAAA|52|Hill 6th|Way|Suite I|Springfield|Mississippi County|AR|79303|United States|-6|condo| +29635|AAAAAAAADMDHAAAA|501|Cedar |Dr.|Suite 260|Edgewood|Price County|WI|50069|United States|-6|condo| +29636|AAAAAAAAEMDHAAAA|768|9th |Ln|Suite 130|Unionville|Boone County|MO|61711|United States|-6|apartment| +29637|AAAAAAAAFMDHAAAA|625|5th Laurel|Blvd|Suite 420|Farmington|Onslow County|NC|29145|United States|-5|condo| +29638|AAAAAAAAGMDHAAAA|425|7th |ST|Suite J|Price|Page County|VA|23824|United States|-5|apartment| +29639|AAAAAAAAHMDHAAAA|345|Walnut Jefferson|Drive|Suite S|Kingston|Branch County|MI|44975|United States|-5|apartment| +29640|AAAAAAAAIMDHAAAA|242|Elm Forest|Court|Suite 460|Oak Ridge|Carlisle County|KY|48371|United States|-6|condo| +29641|AAAAAAAAJMDHAAAA|743|Washington East|Parkway|Suite K|The Meadows|Morrill County|NE|60026|United States|-7|single family| +29642|AAAAAAAAKMDHAAAA|495|Meadow Hillcrest|Ave|Suite O|Georgetown|Le Flore County|OK|77057|United States|-6|condo| +29643|AAAAAAAALMDHAAAA|148|Laurel 6th|ST|Suite A|Enterprise|Benton County|TN|31757|United States|-5|apartment| +29644|AAAAAAAAMMDHAAAA|413|Railroad |Ave|Suite P|Riverview|Stewart County|GA|39003|United States|-5|condo| +29645|AAAAAAAANMDHAAAA|958|Cedar Williams|Ct.|Suite 490|Maple Grove|Oakland County|MI|48252|United States|-5|condo| +29646|AAAAAAAAOMDHAAAA|256|Davis Fourteenth|Cir.|Suite 220|Martinsville|Putnam County|MO|60419|United States|-6|condo| +29647|AAAAAAAAPMDHAAAA|614|1st |Boulevard|Suite 360|Glendale|Osage County|OK|73951|United States|-6|single family| +29648|AAAAAAAAANDHAAAA|668|Washington |Drive|Suite O|Royal|Greeley County|NE|65819|United States|-6|apartment| +29649|AAAAAAAABNDHAAAA|35|6th |Court|Suite 70|Mount Zion|Pierce County|NE|68054|United States|-7|single family| +29650|AAAAAAAACNDHAAAA|467|Locust First|Drive|Suite P|Crossroads|Floyd County|VA|20534|United States|-5|apartment| +29651|AAAAAAAADNDHAAAA|884|Johnson |Way|Suite 50|Oak Hill|Essex County|VT|08438|United States|-5|single family| +29652|AAAAAAAAENDHAAAA|755|Elm |Street|Suite 430|Jamestown|Fairfield County|SC|26867|United States|-5|condo| +29653|AAAAAAAAFNDHAAAA|626|Cherry ||||La Porte County||44107|United States|-5|apartment| +29654|AAAAAAAAGNDHAAAA|988|Spring Chestnut|Way|Suite 290|Cedar Grove|Fayette County|OH|40411|United States|-5|single family| +29655|AAAAAAAAHNDHAAAA|345|Highland First|Circle|Suite 250|Glendale|Lee County|MS|53951|United States|-6|apartment| +29656|AAAAAAAAINDHAAAA|679|Chestnut |RD|Suite N|Concord|Peoria County|IL|64107|United States|-6|condo| +29657|AAAAAAAAJNDHAAAA|542|Main |Ln||Mountain View|Whitley County|||United States|-5|apartment| +29658|AAAAAAAAKNDHAAAA|256|Poplar Maple|Blvd|Suite 370|Farmington|Pushmataha County|OK|79145|United States|-6|single family| +29659|AAAAAAAALNDHAAAA|87|Meadow |Blvd|Suite 430|Spring Valley|Shannon County|MO|66060|United States|-6|condo| +29660|AAAAAAAAMNDHAAAA|644|Second 5th|Dr.|Suite T|Unionville|Clay County|MO|61711|United States|-6|single family| +29661|AAAAAAAANNDHAAAA|372|Johnson Jackson|Circle|Suite 130|Arlington|Madison County|MT|66557|United States|-7|condo| +29662|AAAAAAAAONDHAAAA|127|Miller |Lane|Suite V|Fairview|Guadalupe County|TX|75709|United States|-6|condo| +29663|AAAAAAAAPNDHAAAA|129|Mill |Wy|Suite X|Highland Park|Louisa County|VA|26534|United States|-5|condo| +29664|AAAAAAAAAODHAAAA|565|Poplar |ST|Suite R|Springdale|Monroe County|IA|58883|United States|-6|condo| +29665|AAAAAAAABODHAAAA|205|Ridge Green|Drive|Suite N|Highland|Trimble County|KY|49454|United States|-5|apartment| +29666|AAAAAAAACODHAAAA|659|Main |Court|Suite J|Sunnyside|Walton County|FL|31952|United States|-5|single family| +29667|AAAAAAAADODHAAAA|993|Fourth Fifth|Ct.|Suite S|Highland|Lawrence County|PA|19454|United States|-5|apartment| +29668|AAAAAAAAEODHAAAA|985|Jackson 3rd|Court|Suite N|Enterprise|Lane County|OR|91757|United States|-8|condo| +29669|AAAAAAAAFODHAAAA|758|Park |Street|Suite 220|Wilson|McCracken County|KY|46971|United States|-5|single family| +29670|AAAAAAAAGODHAAAA|862|9th Forest|Blvd|Suite M|Bay View|Daviess County|MO|66457|United States|-6|apartment| +29671|AAAAAAAAHODHAAAA|896|5th Spring|Street|Suite D|Pleasant Hill|Nez Perce County|ID|83604|United States|-7|apartment| +29672|AAAAAAAAIODHAAAA|863|North 14th|RD|Suite 360|Oakwood|Floyd County|IN|40169|United States|-5|condo| +29673|AAAAAAAAJODHAAAA|615|3rd |Parkway|Suite 290|Fairview|Wilkes County|GA|35709|United States|-5|apartment| +29674|AAAAAAAAKODHAAAA|363|Second Park|Drive|Suite 340|Hopewell|Barnwell County|SC|20587|United States|-5|apartment| +29675|AAAAAAAALODHAAAA|775|10th South|Street|Suite 210|Georgetown|Wilcox County|GA|37057|United States|-5|condo| +29676|AAAAAAAAMODHAAAA|589|Third |Wy|Suite 420|Wilson|Haskell County|TX|76971|United States|-6|single family| +29677|AAAAAAAANODHAAAA|522|West North|Pkwy|Suite W|Lakeside|Dougherty County|GA|39532|United States|-5|apartment| +29678|AAAAAAAAOODHAAAA|159|Sunset Smith|Dr.|Suite 310|Greenfield|West Baton Rouge Parish|LA|75038|United States|-6|apartment| +29679|AAAAAAAAPODHAAAA|687|Lee Johnson|Blvd|Suite 330|Newtown|Duplin County|NC|21749|United States|-5|single family| +29680|AAAAAAAAAPDHAAAA|355|First 3rd|Ct.|Suite P|Macedonia|Hale County|TX|71087|United States|-6|apartment| +29681|AAAAAAAABPDHAAAA|473|River |Dr.|Suite 340|Webb|Jackson County|IA|50899|United States|-6|condo| +29682|AAAAAAAACPDHAAAA|815|North Cedar|Dr.|Suite C|Union Hill|Crawford County|MO|67746|United States|-6|condo| +29683|AAAAAAAADPDHAAAA|536|Oak Birch|Street|Suite S|Pleasant Grove|Sullivan County|PA|14136|United States|-5|condo| +29684|AAAAAAAAEPDHAAAA|290|Madison |ST|Suite E|Pleasant Hill|Wyoming County|PA|13604|United States|-5|apartment| +29685|AAAAAAAAFPDHAAAA|106|14th |Ct.|Suite M|Florence|Mitchell County|TX|73394|United States|-6|condo| +29686|AAAAAAAAGPDHAAAA|902|Washington Johnson|ST|Suite C|Kingston|Douglas County|NE|64975|United States|-6|apartment| +29687|AAAAAAAAHPDHAAAA|840|4th Main|Lane|Suite 270|Lakeview|Oconee County|GA|38579|United States|-5|single family| +29688|AAAAAAAAIPDHAAAA|480|1st Church|Circle|Suite 30|Marion|Labette County|KS|60399|United States|-6|single family| +29689|AAAAAAAAJPDHAAAA|951|South Lake|Ln|Suite D|Greenfield|Gray County|KS|65038|United States|-6|condo| +29690|AAAAAAAAKPDHAAAA|206|Cherry |Ave|Suite 0|Red Hill|Chatham County|GA|34338|United States|-5|single family| +29691|AAAAAAAALPDHAAAA|273|Madison Elm|Boulevard|Suite N|Georgetown|Pima County|AZ|87057|United States|-7|condo| +29692|AAAAAAAAMPDHAAAA|400|Green |Ln|Suite T|Oakdale|Blue Earth County|MN|59584|United States|-6|condo| +29693|AAAAAAAANPDHAAAA|294|Washington Poplar|Road|Suite H|Kelly|Webster County|IA|52738|United States|-6|apartment| +29694|AAAAAAAAOPDHAAAA|24|Eigth |Boulevard|Suite 370|Florence|Gloucester County|NJ|03994|United States|-5|condo| +29695|AAAAAAAAPPDHAAAA|916|7th |Road|Suite 30|Franklin|Oceana County|MI|49101|United States|-5|apartment| +29696|AAAAAAAAAAEHAAAA|927|Pine |Dr.|Suite 430|Wilson|Hamilton County|NY|16971|United States|-5|single family| +29697|AAAAAAAABAEHAAAA|166|Willow |Ct.|Suite 90|Bethel|Warren County|KY|45281|United States|-5|apartment| +29698|AAAAAAAACAEHAAAA|616|Seventh Hill|Blvd|Suite B|Unionville|Concordia Parish|LA|71711|United States|-6|condo| +29699|AAAAAAAADAEHAAAA|845|1st Cherry|Street|Suite 240|Woodlawn|Columbia County|PA|14098|United States|-5|condo| +29700|AAAAAAAAEAEHAAAA|397|5th View|Ln|Suite 300|Midway|Gallatin County|IL|61904|United States|-6|condo| +29701|AAAAAAAAFAEHAAAA|237|Washington Jackson|ST|Suite 330|Friendship|Breckinridge County|KY|44536|United States|-6|condo| +29702|AAAAAAAAGAEHAAAA|998|Third |Boulevard|Suite 460|Crossroads|Lewis County|NY|10534|United States|-5|single family| +29703|AAAAAAAAHAEHAAAA|831||||Glenwood|Calaveras County||||-8|| +29704|AAAAAAAAIAEHAAAA|929|Main Walnut|Parkway|Suite D|Mount Zion|Adams County|IL|68054|United States|-6|condo| +29705|AAAAAAAAJAEHAAAA|54|Ash 8th|Ave|Suite 170|Riverdale|San Saba County|TX|79391|United States|-6|apartment| +29706|AAAAAAAAKAEHAAAA|919|9th |Dr.|Suite K|Farmington|Coffee County|GA|39145|United States|-5|single family| +29707|AAAAAAAALAEHAAAA|424|Fourth |Ln|Suite 180|Five Points|Pennington County|MN|56098|United States|-6|condo| +29708|AAAAAAAAMAEHAAAA|690|Johnson 3rd|Court|Suite 160|Oakwood|Dubois County|IN|40169|United States|-5|apartment| +29709|AAAAAAAANAEHAAAA|228|Williams |Street|Suite 10|Edgewood|Rio Blanco County|CO|80069|United States|-7|condo| +29710|AAAAAAAAOAEHAAAA|563|Laurel |Ln|Suite N|Lakeside|Marshall County|MN|59532|United States|-6|condo| +29711|AAAAAAAAPAEHAAAA|976|Oak Second|Court|Suite 450|Springfield|Morris County|TX|79303|United States|-6|single family| +29712|AAAAAAAAABEHAAAA|223|North |Ln|Suite 280|Ashland|Routt County|CO|84244|United States|-7|condo| +29713|AAAAAAAABBEHAAAA|456|Pine |Cir.|Suite 450|Clifton|Fairfield County|SC|28014|United States|-5|apartment| +29714|AAAAAAAACBEHAAAA|649|Highland |Avenue|Suite 460|Hillcrest|Bon Homme County|SD|53003|United States|-6|single family| +29715|AAAAAAAADBEHAAAA|222|Laurel |Way|Suite 340|Buena Vista|Logan County|OH|45752|United States|-5|apartment| +29716|AAAAAAAAEBEHAAAA|646|Sunset Main|Ave|Suite B|Jamestown|Deuel County|NE|66867|United States|-6|condo| +29717|AAAAAAAAFBEHAAAA|139|College 3rd|Street|Suite P|Newport|Midland County|TX|71521|United States|-6|condo| +29718|AAAAAAAAGBEHAAAA|881|Williams Maple|Ct.|Suite 390|Five Points|San Juan County|UT|86098|United States|-7|single family| +29719|AAAAAAAAHBEHAAAA|832|College |Ct.|Suite Y|Five Forks|Pike County|PA|12293|United States|-5|single family| +29720|AAAAAAAAIBEHAAAA|448|Central Second|Court|Suite J|Fowler|Fairfield County|OH|41083|United States|-5|condo| +29721|AAAAAAAAJBEHAAAA|964|Lakeview |Parkway|Suite M|Springdale|Stephens County|OK|78883|United States|-6|single family| +29722|AAAAAAAAKBEHAAAA|||Blvd||||NC|28994||-5|| +29723|AAAAAAAALBEHAAAA|169|Cherry Jefferson|Boulevard|Suite O|Arlington|Pasco County|FL|36557|United States|-5|condo| +29724|AAAAAAAAMBEHAAAA|33|West |Lane|Suite 180|Oakdale|Webb County|TX|79584|United States|-6|condo| +29725|AAAAAAAANBEHAAAA|329|5th 5th|Ct.|Suite H|Walnut Grove|Carbon County|UT|87752|United States|-7|condo| +29726|AAAAAAAAOBEHAAAA|599|Lee |Blvd|Suite P|Valley View|Person County|NC|25124|United States|-5|apartment| +29727|AAAAAAAAPBEHAAAA|928|11th Locust|ST|Suite 20|Arlington|Atascosa County|TX|76557|United States|-6|apartment| +29728|AAAAAAAAACEHAAAA|25|Oak Hill|Boulevard|Suite S|Riverview|Kingman County|KS|69003|United States|-6|condo| +29729|AAAAAAAABCEHAAAA|507|Railroad |Ave|Suite 330|Hillcrest|Multnomah County|OR|93003|United States|-8|condo| +29730|AAAAAAAACCEHAAAA|443|Hill North|Boulevard|Suite I|Farmington|Orange County|IN|49145|United States|-5|apartment| +29731|AAAAAAAADCEHAAAA|698|6th |Drive|Suite O|Plainview|Screven County|GA|33683|United States|-5|apartment| +29732|AAAAAAAAECEHAAAA|257|Central |Pkwy|Suite S|Greenwood|Bolivar County|MS|58828|United States|-6|condo| +29733|AAAAAAAAFCEHAAAA|495|Laurel Jefferson|Ave|Suite J|Pleasant Hill|Bibb County|AL|33604|United States|-6|condo| +29734|AAAAAAAAGCEHAAAA|221|11th 2nd|Circle|Suite C|Wesley|Fountain County|IN|41218|United States|-5|single family| +29735|AAAAAAAAHCEHAAAA|726|North |Wy|Suite 120|Pleasant Valley|Chaves County|NM|82477|United States|-7|condo| +29736|AAAAAAAAICEHAAAA|372|Washington |Lane|Suite V|Mount Pleasant|Carbon County|PA|11933|United States|-5|single family| +29737|AAAAAAAAJCEHAAAA||||||Independence County||79584|United States|-6|| +29738|AAAAAAAAKCEHAAAA|94|Sycamore |Wy|Suite 170|Sunnyside|Shelby County|AL|31952|United States|-6|condo| +29739|AAAAAAAALCEHAAAA|401|West |Lane|Suite J|Riverdale|Nome Census Area|AK|99391|United States|-9|condo| +29740|AAAAAAAAMCEHAAAA|56|Meadow Center|Ave|Suite 420|Rocky Point|Madison County|OH|41209|United States|-5|condo| +29741|AAAAAAAANCEHAAAA|980|Maple |ST|||Skamania County|WA|91521|||| +29742|AAAAAAAAOCEHAAAA|450|Park |Avenue|Suite 380|Fairview|San Saba County|TX|75709|United States|-6|condo| +29743|AAAAAAAAPCEHAAAA|205|3rd |Boulevard|Suite B|Walnut Grove|Adams County|NE|67752|United States|-6|single family| +29744|AAAAAAAAADEHAAAA|59|Main 9th|Drive|Suite 190|Riverside|Calcasieu Parish|LA|79231|United States|-6|condo| +29745|AAAAAAAABDEHAAAA|636|3rd |Cir.|Suite G|Andover|Brule County|SD|51639|United States|-6|apartment| +29746|AAAAAAAACDEHAAAA||||Suite 40|Greenwood|||78828|||| +29747|AAAAAAAADDEHAAAA|721|Hickory 12th|Parkway|Suite E|Georgetown|Effingham County|IL|67057|United States|-6|apartment| +29748|AAAAAAAAEDEHAAAA|827|3rd 6th|Wy|Suite 10|Mount Olive|Logan County|KY|48059|United States|-5|condo| +29749|AAAAAAAAFDEHAAAA|402|Highland Railroad|Pkwy|Suite 240|Newport|Vernon County|WI|51521|United States|-6|single family| +29750|AAAAAAAAGDEHAAAA|846|South |Ct.|Suite N|Fairview|Tattnall County|GA|35709|United States|-5|condo| +29751|AAAAAAAAHDEHAAAA|514|Hillcrest |Pkwy|Suite U|Forest Hills|Mitchell County|TX|79237|United States|-6|single family| +29752|AAAAAAAAIDEHAAAA|266|Oak Ridge|Blvd|Suite 300|Lakeview|Pickaway County|OH|48579|United States|-5|apartment| +29753|AAAAAAAAJDEHAAAA|744|2nd |Wy|Suite 230|Enterprise|Otsego County|MI|41757|United States|-5|apartment| +29754|AAAAAAAAKDEHAAAA|730|12th Railroad|Street|Suite A|Riverdale|Nye County|NV|89391|United States|-8|single family| +29755|AAAAAAAALDEHAAAA|577|7th |Cir.|Suite 460|Georgetown|Howard County|IA|57057|United States|-6|single family| +29756|AAAAAAAAMDEHAAAA|875|Birch Mill|Circle|Suite W|Enterprise|Botetourt County|VA|21757|United States|-5|apartment| +29757|AAAAAAAANDEHAAAA|599|8th Center|Drive|Suite O|Edgewood|Anchorage Borough|AK|90069|United States|-9|single family| +29758|AAAAAAAAODEHAAAA|955|Park |Pkwy|Suite G|Deerfield|Duplin County|NC|29840|United States|-5|condo| +29759|AAAAAAAAPDEHAAAA|615|Seventh |ST|Suite 50|Oak Hill|Allen Parish|LA|77838|United States|-6|single family| +29760|AAAAAAAAAEEHAAAA|300|Sycamore Third|Street|Suite 240|Glendale|Jackson County|SD|53951|United States|-7|condo| +29761|AAAAAAAABEEHAAAA|235|5th |Circle|Suite Q|Edgewater|Jim Wells County|TX|70635|United States|-6|apartment| +29762|AAAAAAAACEEHAAAA|770|Spruce Hill|Boulevard|Suite U|Antioch|Elliott County|KY|48605|United States|-6|condo| +29763|AAAAAAAADEEHAAAA|27|Spring |Way|Suite 210|Lincoln|Lagrange County|IN|41289|United States|-5|apartment| +29764|AAAAAAAAEEEHAAAA|149|3rd |Ct.|Suite Q|Garfield|Murray County|MN|58903|United States|-6|condo| +29765|AAAAAAAAFEEHAAAA|430|Cedar |Parkway|Suite 40|Buena Vista|Christian County|MO|65752|United States|-6|condo| +29766|AAAAAAAAGEEHAAAA|729|Church Park|Pkwy|Suite 440|Jackson|Nash County|NC|29583|United States|-5|apartment| +29767|AAAAAAAAHEEHAAAA|688|Woodland |Boulevard|Suite 190|Ashland|Luna County|NM|84244|United States|-7|single family| +29768|AAAAAAAAIEEHAAAA|626|Lake |Avenue|Suite 10|Five Forks|Rawlins County|KS|62293|United States|-6|single family| +29769|AAAAAAAAJEEHAAAA|379|Woodland Park|Ln|Suite 20|Carpenter|Snohomish County|WA|91147|United States|-8|condo| +29770|AAAAAAAAKEEHAAAA|703|Green |Road|Suite S|Spring Hill|Macon County|NC|26787|United States|-5|condo| +29771|AAAAAAAALEEHAAAA|122|Pine Jackson|Ave|Suite M|Wilson|Lincoln County|MT|66971|United States|-7|condo| +29772|AAAAAAAAMEEHAAAA|931|Green Highland|Cir.|Suite 220|Riverdale|Livingston Parish|LA|79391|United States|-6|apartment| +29773|AAAAAAAANEEHAAAA|718|North |Court|Suite A|Wildwood|Siskiyou County|CA|96871|United States|-8|condo| +29774|AAAAAAAAOEEHAAAA|537|Pine |Cir.|Suite F|Lakewood|Buena Vista County|IA|58877|United States|-6|condo| +29775|AAAAAAAAPEEHAAAA|398|Hill Jackson|Drive|Suite 340|Summit|Caledonia County|VT|01099|United States|-5|apartment| +29776|AAAAAAAAAFEHAAAA|744|7th |Avenue|Suite U|Providence|Chilton County|AL|36614|United States|-6|single family| +29777|AAAAAAAABFEHAAAA|588|West Oak|Avenue|Suite L|Farmington|Cloud County|KS|69145|United States|-6|condo| +29778|AAAAAAAACFEHAAAA|495|1st Ninth|Avenue|Suite 160|Springdale|Hardeman County|TX|78883|United States|-6|apartment| +29779|AAAAAAAADFEHAAAA|811|Church |Street|Suite 10|Bloomingdale|Siskiyou County|CA|91824|United States|-8|condo| +29780|AAAAAAAAEFEHAAAA|553|Lincoln 3rd|Boulevard|Suite 400|Highland|Columbia County|PA|19454|United States|-5|condo| +29781|AAAAAAAAFFEHAAAA|179|4th |Dr.|Suite 110|Mountain View|Trumbull County|OH|44466|United States|-5|condo| +29782|AAAAAAAAGFEHAAAA|817|3rd |RD|Suite N|Summerfield|Fayette County|OH|40634|United States|-5|apartment| +29783|AAAAAAAAHFEHAAAA|136|7th |RD|Suite 400|Farmington|Livingston Parish|LA|79145|United States|-6|apartment| +29784|AAAAAAAAIFEHAAAA|644|Davis Third|Boulevard|Suite D|Fairfield|De Soto Parish|LA|76192|United States|-6|single family| +29785|AAAAAAAAJFEHAAAA|684|South |RD|Suite 460|Mount Zion|Tazewell County|IL|68054|United States|-6|single family| +29786|AAAAAAAAKFEHAAAA|335|2nd |Cir.|Suite 80|San Jose|Major County|OK|78003|United States|-6|apartment| +29787|AAAAAAAALFEHAAAA|489|Mill Church|Blvd|Suite X|Salem|Mathews County|VA|28048|United States|-5|single family| +29788|AAAAAAAAMFEHAAAA|223|Lincoln Ninth|Pkwy|Suite X|Greenfield|El Paso County|CO|85038|United States|-7|single family| +29789|AAAAAAAANFEHAAAA|508|Fourth |Parkway|Suite J|Shiloh|Faulkner County|AR|79275|United States|-6|apartment| +29790|AAAAAAAAOFEHAAAA|259|Wilson Center|Street|Suite X|Pleasant Grove|Custer County|ID|84136|United States|-7|condo| +29791|AAAAAAAAPFEHAAAA|983|1st |Lane|Suite U|Union Hill|Clifton Forge city|VA|27746|United States|-5|apartment| +29792|AAAAAAAAAGEHAAAA|510|Pine West|Ave|Suite M|Oak Ridge|Lapeer County|MI|48371|United States|-5|single family| +29793|AAAAAAAABGEHAAAA|237|Birch Fifth|Parkway|Suite 410|Enterprise|Pontotoc County|MS|51757|United States|-6|condo| +29794|AAAAAAAACGEHAAAA|623|Fourth 7th|Ct.|Suite 260|Enterprise|Barren County|KY|41757|United States|-6|single family| +29795|AAAAAAAADGEHAAAA|412|3rd Mill|RD|Suite 60|Five Forks|Polk County|TN|32293|United States|-6|single family| +29796|AAAAAAAAEGEHAAAA|222|Davis Laurel|Parkway|Suite C|White Oak|Pitkin County|CO|86668|United States|-7|condo| +29797|AAAAAAAAFGEHAAAA|868|Cedar Church|Wy|Suite 0|Riverside|Osage County|KS|69231|United States|-6|single family| +29798|AAAAAAAAGGEHAAAA|523|Railroad |Parkway|Suite 480|Shady Grove|McCone County|MT|62812|United States|-7|condo| +29799|AAAAAAAAHGEHAAAA|245|Main |Parkway|Suite 290|Plainview|Nez Perce County|ID|83683|United States|-7|apartment| +29800|AAAAAAAAIGEHAAAA|588|Highland Park|Cir.|Suite S|Mount Vernon|Dunn County|WI|58482|United States|-6|apartment| +29801|AAAAAAAAJGEHAAAA|363|Church Sixth|Court|Suite M|Lakeside|Okaloosa County|FL|39532|United States|-5|apartment| +29802|AAAAAAAAKGEHAAAA|||||||||||| +29803|AAAAAAAALGEHAAAA|860|West Second|Drive|Suite 360|Crossroads|Wood County|WV|20534|United States|-5|single family| +29804|AAAAAAAAMGEHAAAA|471|Fifth |Ln|Suite V|Forest Hills|Dakota County|NE|69237|United States|-6|apartment| +29805|AAAAAAAANGEHAAAA|536|Locust Main|Parkway|Suite T|Edgewood|Barber County|KS|60069|United States|-6|single family| +29806|AAAAAAAAOGEHAAAA|818|5th |ST|Suite 330|Oakdale|Waller County|TX|79584|United States|-6|condo| +29807|AAAAAAAAPGEHAAAA||College |Street|Suite 280|Marion|Giles County|VA|20399|United States|-5|| +29808|AAAAAAAAAHEHAAAA|178|10th |Avenue|Suite 160|Green Acres|Cibola County|NM|87683|United States|-7|condo| +29809|AAAAAAAABHEHAAAA|215|Dogwood |Parkway|Suite S|Macedonia|Hudspeth County|TX|71087|United States|-6|single family| +29810|AAAAAAAACHEHAAAA|518|Wilson Mill|Ln|Suite 400|Highland|Morgan County|UT|89454|United States|-7|condo| +29811|AAAAAAAADHEHAAAA|241|Ridge Washington|Street|Suite 170|Post Oak|Choctaw County|MS|58567|United States|-6|single family| +29812|AAAAAAAAEHEHAAAA|992|Jackson Woodland|Blvd|Suite C|Five Points|Montgomery County|IN|46098|United States|-5|apartment| +29813|AAAAAAAAFHEHAAAA|786|6th |Cir.|Suite 340|Midway|Adams County|IN|41904|United States|-5|single family| +29814|AAAAAAAAGHEHAAAA|546|Elm |Court|Suite V|Oakland|Tom Green County|TX|79843|United States|-6|single family| +29815|AAAAAAAAHHEHAAAA|613|River |Court|Suite L|Brownsville|Casey County|KY|49310|United States|-6|condo| +29816|AAAAAAAAIHEHAAAA|824|West Hillcrest|Street|Suite 470|Payne|Pinellas County|FL|36134|United States|-5|condo| +29817|AAAAAAAAJHEHAAAA|924|5th |Drive|Suite 230|White Oak|Rusk County|WI|56668|United States|-6|condo| +29818|AAAAAAAAKHEHAAAA|91|Lincoln Poplar|Avenue|Suite P|Buena Vista|Barry County|MO|65752|United States|-6|apartment| +29819|AAAAAAAALHEHAAAA|60|Laurel |Street|Suite 300|Chestnut Ridge|Dakota County|MN|57334|United States|-6|apartment| +29820|AAAAAAAAMHEHAAAA|151|Hill Dogwood|Street|Suite 280|Oakdale|Monongalia County|WV|29584|United States|-5|single family| +29821|AAAAAAAANHEHAAAA|698|7th Green|RD|||Wilkin County|MN||United States||| +29822|AAAAAAAAOHEHAAAA|186|Birch Sunset|Cir.|Suite B|Rosebud|Bracken County|KY|42244|United States|-6|apartment| +29823|AAAAAAAAPHEHAAAA|762|Central |Circle|Suite 230|Oak Hill|Emery County|UT|87838|United States|-7|condo| +29824|AAAAAAAAAIEHAAAA|183|5th |Lane|Suite O|Providence|Yakima County|WA|96614|United States|-8|single family| +29825|AAAAAAAABIEHAAAA|545|Wilson Oak|Avenue|Suite W|Bethel|Geary County|KS|65281|United States|-6|single family| +29826|AAAAAAAACIEHAAAA|328|Church |Road|Suite 50|Clifton|Hot Springs County|WY|88014|United States|-7|single family| +29827|AAAAAAAADIEHAAAA|535|South |Court|Suite 260|Spring Hill|Atascosa County|TX|76787|United States|-6|single family| +29828|AAAAAAAAEIEHAAAA|359|Main Cedar|Cir.|Suite 170|Oakdale|Lake County|IL|69584|United States|-6|condo| +29829|AAAAAAAAFIEHAAAA|147|Green |Avenue|Suite L|Saint George|Bullitt County|KY|45281|United States|-6|apartment| +29830|AAAAAAAAGIEHAAAA|787|Elm Woodland|Parkway|Suite 210|Enterprise|DeKalb County|TN|31757|United States|-5|condo| +29831|AAAAAAAAHIEHAAAA|964|Smith Jackson|Lane|Suite 330|Gladstone|Autauga County|AL|30894|United States|-6|condo| +29832|AAAAAAAAIIEHAAAA|181|Oak 4th|Parkway|Suite 420|Salem|McNairy County|TN|38048|United States|-6|condo| +29833|AAAAAAAAJIEHAAAA|392|East Birch|ST|Suite 400|Sunnyside|McDonough County|IL|61952|United States|-6|apartment| +29834|AAAAAAAAKIEHAAAA|506|Oak |Blvd|Suite 70|Newport|Paulding County|GA|31521|United States|-5|condo| +29835|AAAAAAAALIEHAAAA|896|1st |Court|Suite I|Macedonia|Wayne County|TN|31087|United States|-6|single family| +29836|AAAAAAAAMIEHAAAA|348|Jefferson |Wy|Suite 320|Buena Vista|Brooks County|TX|75752|United States|-6|apartment| +29837|AAAAAAAANIEHAAAA|120|Hill |Road|Suite 250|Buena Vista|Pecos County|TX|75752|United States|-6|single family| +29838|AAAAAAAAOIEHAAAA|825|Mill 3rd|Boulevard|Suite T|Bunker Hill|Limestone County|AL|30150|United States|-6|condo| +29839|AAAAAAAAPIEHAAAA|766|Spring 1st|Wy|Suite T|Newport|Pinellas County|FL|31521|United States|-5|single family| +29840|AAAAAAAAAJEHAAAA|545|Cedar |RD|Suite T|Bethel|Sully County|SD|55281|United States|-7|apartment| +29841|AAAAAAAABJEHAAAA|758|Hickory |RD|Suite P|Bethel|Cameron County|PA|15281|United States|-5|single family| +29842|AAAAAAAACJEHAAAA|540|Sixth Jackson|Road|Suite 410|Sunnyside|Sioux County|ND|51952|United States|-6|condo| +29843|AAAAAAAADJEHAAAA|53|Valley Willow|Boulevard|Suite O|Lakeview|Sierra County|CA|98579|United States|-8|single family| +29844|AAAAAAAAEJEHAAAA|243|Washington Adams|Cir.|Suite U|Rolling Hills|Iowa County|IA|57272|United States|-6|condo| +29845|AAAAAAAAFJEHAAAA|464|9th East|Ct.|Suite U|Concord|Calhoun County|IL|64107|United States|-6|single family| +29846|AAAAAAAAGJEHAAAA|340|Mill 5th|Lane|Suite L|Lakeview|Lowndes County|MS|58579|United States|-6|condo| +29847|AAAAAAAAHJEHAAAA||Willow |||||CA||||| +29848|AAAAAAAAIJEHAAAA|83|7th |Court|Suite G|Union Hill|Hancock County|MS|57746|United States|-6|apartment| +29849|AAAAAAAAJJEHAAAA|768|13th Lee|Circle|Suite W|Pine Grove|Alfalfa County|OK|74593|United States|-6|single family| +29850|AAAAAAAAKJEHAAAA|632|West |Dr.|Suite V|Shiloh|Portage County|OH|49275|United States|-5|apartment| +29851|AAAAAAAALJEHAAAA|24|3rd View|Ln|Suite I|Oak Hill|Platte County|WY|87838|United States|-7|apartment| +29852|AAAAAAAAMJEHAAAA|369|Smith Franklin|Blvd|Suite 330|Walnut Grove|Custer County|MT|67752|United States|-7|single family| +29853|AAAAAAAANJEHAAAA|851|5th |Dr.|Suite 470|The Meadows|Kendall County|TX|70026|United States|-6|single family| +29854|AAAAAAAAOJEHAAAA|225|Hickory Twelfth|Lane|Suite O|Woodville|Mercer County|ND|54289|United States|-6|single family| +29855|AAAAAAAAPJEHAAAA|887|9th |Ct.|Suite I|Georgetown|White Pine County|NV|87057|United States|-8|condo| +29856|AAAAAAAAAKEHAAAA|238|2nd |Street|Suite G|Springdale|Rutland County|VT|09483|United States|-5|apartment| +29857|AAAAAAAABKEHAAAA|950|Mill |Road|Suite J|Woodland|Hanson County|SD|54854|United States|-7|single family| +29858|AAAAAAAACKEHAAAA|780|North |Way|Suite 380|Spring Hill|Moffat County|CO|86787|United States|-7|apartment| +29859|AAAAAAAADKEHAAAA|462|North Second|Cir.|Suite K|Summit|Jackson County|KS|60499|United States|-6|apartment| +29860|AAAAAAAAEKEHAAAA|572|Spring |Court|Suite A|Lakeview|Jack County|TX|78579|United States|-6|single family| +29861|AAAAAAAAFKEHAAAA|303|Williams Jefferson|Ln|Suite D|Enterprise|Coos County|NH|02357|United States|-5|single family| +29862|AAAAAAAAGKEHAAAA|389|Dogwood |Ave|Suite V|Hamilton|Midland County|MI|42808|United States|-5|condo| +29863|AAAAAAAAHKEHAAAA|251|11th 15th|Ct.|Suite 370|Oak Ridge|Green County|WI|58371|United States|-6|condo| +29864|AAAAAAAAIKEHAAAA|728|Green |Boulevard|Suite 10|Macedonia|Clatsop County|OR|91087|United States|-8|single family| +29865|AAAAAAAAJKEHAAAA|441|Main Locust|Blvd|Suite Q|Oakdale|Lawrence County|MS|59584|United States|-6|apartment| +29866|AAAAAAAAKKEHAAAA|511|6th |Cir.|Suite 290|Sheffield|Tate County|MS|56896|United States|-6|single family| +29867|AAAAAAAALKEHAAAA|146|Spring |Cir.|Suite 30|Centerville|Tuolumne County|CA|90059|United States|-8|apartment| +29868|AAAAAAAAMKEHAAAA|723|West Cedar|Way|Suite 260|Georgetown|Fulton County|IN|47057|United States|-5|single family| +29869|AAAAAAAANKEHAAAA|21|Eigth |Dr.|Suite 40|Mount Vernon|Washington County|KY|48482|United States|-5|single family| +29870|AAAAAAAAOKEHAAAA|261|Lakeview |Ave|Suite B|Antioch|Rice County|KS|68605|United States|-6|condo| +29871|AAAAAAAAPKEHAAAA|635|9th |Boulevard|Suite W|Jackson|Wise County|TX|79583|United States|-6|condo| +29872|AAAAAAAAALEHAAAA|901|Franklin View|Pkwy|Suite X|Pine Grove|Green Lake County|WI|54593|United States|-6|condo| +29873|AAAAAAAABLEHAAAA|420|Eigth |Way|Suite H|Arlington|Coosa County|AL|36557|United States|-6|single family| +29874|AAAAAAAACLEHAAAA|22|Main Adams|Street|Suite G|Ellisville|Lexington city|VA|26820|United States|-5|apartment| +29875|AAAAAAAADLEHAAAA|996|3rd |Boulevard|Suite 90|Sulphur Springs|Crook County|WY|88354|United States|-7|single family| +29876|AAAAAAAAELEHAAAA|439||Dr.||||||||| +29877|AAAAAAAAFLEHAAAA|928|Broadway |Ct.|Suite C|Macedonia|Haines Borough|AK|91087|United States|-9|single family| +29878|AAAAAAAAGLEHAAAA|579|Jefferson |Ave|Suite J|Lincoln|Berks County|PA|11289|United States|-5|single family| +29879|AAAAAAAAHLEHAAAA|417|Jackson |Avenue|Suite A|Salem|Scott County|KS|68048|United States|-6|condo| +29880|AAAAAAAAILEHAAAA|624|8th Cedar|Dr.|Suite L|Union Hill|Wyoming County|NY|17746|United States|-5|apartment| +29881|AAAAAAAAJLEHAAAA|401|River 10th|Drive|Suite 240|White Oak|Yankton County|SD|56668|United States|-6|apartment| +29882|AAAAAAAAKLEHAAAA|561|Cedar |Ave|Suite 170|Frankfort|Bartholomew County|IN|49681|United States|-5|single family| +29883|AAAAAAAALLEHAAAA|624|Valley |Ave|Suite C|Hillcrest|Warren County|IL|63003|United States|-6|apartment| +29884|AAAAAAAAMLEHAAAA|507|Third |Road|Suite Y|Farmington|Craig County|VA|29145|United States|-5|condo| +29885|AAAAAAAANLEHAAAA|979|Railroad Hillcrest|Pkwy|Suite 150|Oakland|Decatur County|TN|39843|United States|-5|single family| +29886|AAAAAAAAOLEHAAAA|33|Elm |Parkway|Suite V|Riverview|Mahaska County|IA|59003|United States|-6|condo| +29887|AAAAAAAAPLEHAAAA|716|7th View|Court|Suite 480|Riverdale|Howard County|MO|69391|United States|-6|apartment| +29888|AAAAAAAAAMEHAAAA|308|Pine |Avenue|Suite 90|Willow|Albemarle County|VA|26798|United States|-5|condo| +29889|AAAAAAAABMEHAAAA|449|Lake |Drive|Suite U|Providence|Nelson County|KY|46614|United States|-5|single family| +29890|AAAAAAAACMEHAAAA|876|South Broadway|RD|Suite 330|Springdale|Ontonagon County|MI|48883|United States|-5|apartment| +29891|AAAAAAAADMEHAAAA|337|Third |Street|Suite 480|Bunker Hill|Lee County|VA|20150|United States|-5|apartment| +29892|AAAAAAAAEMEHAAAA|440|9th |Road|Suite E|Kingston|Carlton County|MN|54975|United States|-6|apartment| +29893|AAAAAAAAFMEHAAAA|347|Woodland |Circle|Suite S|Red Oak|Marin County|CA|95018|United States|-8|single family| +29894|AAAAAAAAGMEHAAAA|432|Laurel |RD|Suite H|Providence|Clark County|WA|96614|United States|-8|single family| +29895|AAAAAAAAHMEHAAAA||||Suite 150||Langlade County|WI|54975|United States|-6|| +29896|AAAAAAAAIMEHAAAA|732|Oak 7th|Way|Suite 380|Pleasant Valley|Casey County|KY|42477|United States|-6|condo| +29897|AAAAAAAAJMEHAAAA|749|Jefferson North|Parkway|Suite 490|Pleasant Grove|Buena Vista County|IA|54136|United States|-6|apartment| +29898|AAAAAAAAKMEHAAAA|390|Hill |Lane|Suite 100|Macedonia|Nevada County|CA|91087|United States|-8|single family| +29899|AAAAAAAALMEHAAAA|963|Laurel |Drive|Suite 440|Stringtown|Saunders County|NE|60162|United States|-7|apartment| +29900|AAAAAAAAMMEHAAAA|367|Willow Jefferson|Pkwy|Suite D|Walnut Grove|Washington County|FL|37752|United States|-5|single family| +29901|AAAAAAAANMEHAAAA|72|Dogwood West|ST|Suite 430|Georgetown|Karnes County|TX|77057|United States|-6|apartment| +29902|AAAAAAAAOMEHAAAA|35|River |Lane|Suite J|Edgewood|Robertson County|KY|40069|United States|-5|single family| +29903|AAAAAAAAPMEHAAAA|198|Pine Birch|Road|Suite Y|Sunnyside|Bent County|CO|81952|United States|-7|single family| +29904|AAAAAAAAANEHAAAA|843|Park |Pkwy|Suite 450|Buena Vista|Union County|MS|55752|United States|-6|apartment| +29905|AAAAAAAABNEHAAAA|530|Oak Oak|Boulevard|Suite N|Springdale|Grundy County|IL|68883|United States|-6|single family| +29906|AAAAAAAACNEHAAAA|926|Forest 3rd|Ct.|Suite 480|Oak Ridge|Pierce County|WA|98371|United States|-8|apartment| +29907|AAAAAAAADNEHAAAA|181|South Sixth|Road|Suite M|Fairfield|Northwest Arctic Borough|AK|96192|United States|-9|condo| +29908|AAAAAAAAENEHAAAA|952|Oak 7th|Lane|Suite B|Springdale|Perry County|KY|48883|United States|-5|condo| +29909|AAAAAAAAFNEHAAAA|809|Hill Ash|Court|Suite A|Newtown|Henrico County|VA|21749|United States|-5|single family| +29910|AAAAAAAAGNEHAAAA|311|West |Ln|Suite 30|Mount Zion|Garrett County|MD|28054|United States|-5|single family| +29911|AAAAAAAAHNEHAAAA|325|College 2nd|Wy|Suite 0|Woodville|Northampton County|NC|24289|United States|-5|apartment| +29912|AAAAAAAAINEHAAAA|28|Park |Wy|Suite 30|Clinton|Ferry County|WA|98222|United States|-8|single family| +29913|AAAAAAAAJNEHAAAA|214|12th |ST|Suite 30|Bath|Schleicher County|TX|70573|United States|-6|single family| +29914|AAAAAAAAKNEHAAAA|344|Wilson Sunset|Way|Suite G|Macedonia|Laramie County|WY|81087|United States|-7|condo| +29915|AAAAAAAALNEHAAAA|550|Broadway Hill|Drive|Suite 420|Woodruff|Houston County|MN|54174|United States|-6|apartment| +29916|AAAAAAAAMNEHAAAA|503|First 7th|Street|Suite 430|Richardson|Etowah County|AL|37687|United States|-6|condo| +29917|AAAAAAAANNEHAAAA|518|3rd |Pkwy|Suite 60|Lakeview|DuPage County|IL|68579|United States|-6|single family| +29918|AAAAAAAAONEHAAAA|379|Oak Lee|Pkwy|Suite O|Oak Ridge|Cass County|NE|68371|United States|-6|condo| +29919|AAAAAAAAPNEHAAAA|366|10th Fifth|Drive|Suite C|Macedonia|Marshall County|KY|41087|United States|-5|single family| +29920|AAAAAAAAAOEHAAAA|87|13th |Ct.|Suite J|Fairview|Calhoun County|TX|75709|United States|-6|condo| +29921|AAAAAAAABOEHAAAA|35|Ridge Washington|Avenue|Suite 220|Belmont|Beaufort County|NC|20191|United States|-5|single family| +29922|AAAAAAAACOEHAAAA|||||Shiloh||KY||United States||apartment| +29923|AAAAAAAADOEHAAAA|872|Main Seventh|Wy|Suite T|Concord|Portsmouth city|VA|24107|United States|-5|single family| +29924|AAAAAAAAEOEHAAAA|709|4th Franklin|Street|Suite L|Riverview|Wagoner County|OK|79003|United States|-6|apartment| +29925|AAAAAAAAFOEHAAAA|678|East Madison|Cir.|Suite D|Valley View|Sullivan County|TN|35124|United States|-6|condo| +29926|AAAAAAAAGOEHAAAA|93|North Spring|Ct.|Suite 380|Lincoln|Appling County|GA|31289|United States|-5|single family| +29927|AAAAAAAAHOEHAAAA|120|Adams |ST|Suite E|Pine Grove|Anchorage Borough|AK|94593|United States|-9|single family| +29928|AAAAAAAAIOEHAAAA|95|Third |Pkwy|Suite 350|Franklin|San Jacinto County|TX|79101|United States|-6|condo| +29929|AAAAAAAAJOEHAAAA|65|Franklin |Drive|Suite 360|Fairfield|Greenwood County|SC|26192|United States|-5|condo| +29930|AAAAAAAAKOEHAAAA|482|Center 3rd|Ct.|Suite T|Shiloh|Cherokee County|IA|59275|United States|-6|apartment| +29931|AAAAAAAALOEHAAAA|69|Railroad |Ct.|Suite 390|Spring Valley|Jones County|MS|56060|United States|-6|condo| +29932|AAAAAAAAMOEHAAAA|748|Ridge |ST|Suite T|Antioch|Garfield County|CO|88605|United States|-7|single family| +29933|AAAAAAAANOEHAAAA|469|6th |Avenue|Suite K|Fairview|Caldwell County|MO|65709|United States|-6|single family| +29934|AAAAAAAAOOEHAAAA|375|Railroad |Ave|Suite A|Springhill|Kodiak Island Borough|AK|94602|United States|-9|single family| +29935|AAAAAAAAPOEHAAAA|197|Highland |RD|Suite 90|Shiloh|Bradford County|FL|39275|United States|-5|single family| +29936|AAAAAAAAAPEHAAAA|529|Third |ST|Suite 220|Jackson|Baltimore city|MD|29583|United States|-5|condo| +29937|AAAAAAAABPEHAAAA|799|Pine |Pkwy|Suite 430|Union Hill|Minidoka County|ID|87746|United States|-7|single family| +29938|AAAAAAAACPEHAAAA|791|Jefferson |Lane|Suite 300|Mount Vernon|Sanpete County|UT|88482|United States|-7|condo| +29939|AAAAAAAADPEHAAAA|159|Lake |Dr.|Suite H|Florence|Nemaha County|KS|63394|United States|-6|apartment| +29940|AAAAAAAAEPEHAAAA|521|Eigth |Pkwy|Suite 30|Pleasant Valley|Cuyahoga County|OH|42477|United States|-5|apartment| +29941|AAAAAAAAFPEHAAAA|642|5th |ST|Suite 190|Pine Grove|Crawford County|GA|34593|United States|-5|condo| +29942|AAAAAAAAGPEHAAAA|469|4th |RD|Suite A|Oak Ridge|Sedgwick County|KS|68371|United States|-6|apartment| +29943|AAAAAAAAHPEHAAAA|857|Hickory Lake|Pkwy|Suite L|Fairview|Major County|OK|75709|United States|-6|single family| +29944|AAAAAAAAIPEHAAAA|186|Madison |Parkway|Suite L|Cedar Grove|Charles County|MD|20411|United States|-5|condo| +29945|AAAAAAAAJPEHAAAA|299|2nd 2nd|Wy|Suite 360|Ellisville|Hernando County|FL|36820|United States|-5|condo| +29946|AAAAAAAAKPEHAAAA|505|Sycamore 12th|Drive|Suite W|Enterprise|Adams County|PA|11757|United States|-5|single family| +29947|AAAAAAAALPEHAAAA|889|Fourth |Court|Suite I|Maple Grove|Callaway County|MO|68252|United States|-6|apartment| +29948|AAAAAAAAMPEHAAAA|241|South Mill|Road|Suite 110|Glenwood|Northampton County|NC|23511|United States|-5|condo| +29949|AAAAAAAANPEHAAAA|297|12th |Lane|Suite 420|Shady Grove|Greene County|PA|12812|United States|-5|single family| +29950|AAAAAAAAOPEHAAAA|634|Lincoln Spring|Ave|Suite 320|Woodland|Jackson Parish|LA|74854|United States|-6|apartment| +29951|AAAAAAAAPPEHAAAA|273|Pine |RD|Suite 420|Oak Grove|Clinton County|IA|58370|United States|-6|apartment| +29952|AAAAAAAAAAFHAAAA|||Pkwy|Suite 320|Harmony||||United States|-5|condo| +29953|AAAAAAAABAFHAAAA|||Court|Suite 410|Hopewell|||80587|United States|-7|single family| +29954|AAAAAAAACAFHAAAA|573|North 2nd|Ct.|Suite G|Fairview|Kalamazoo County|MI|45709|United States|-5|condo| +29955|AAAAAAAADAFHAAAA|565|5th Meadow|ST|Suite 190|Jamestown|Deschutes County|OR|96867|United States|-8|condo| +29956|AAAAAAAAEAFHAAAA|936|First |Ct.|Suite I|Jerome|Mecosta County|MI|49920|United States|-5|condo| +29957|AAAAAAAAFAFHAAAA|759|Lakeview |Ln|Suite P|Fairview|Adams County|OH|45709|United States|-5|apartment| +29958|AAAAAAAAGAFHAAAA|538|3rd |Ave|Suite M|Concord|Saguache County|CO|84107|United States|-7|single family| +29959|AAAAAAAAHAFHAAAA||Park 2nd||||Adams County||||-6|apartment| +29960|AAAAAAAAIAFHAAAA|826|11th Wilson|Parkway|Suite 180|Green Acres|Lincoln County|OK|77683|United States|-6|condo| +29961|AAAAAAAAJAFHAAAA|50|7th |Ave|Suite L|Friendship|Washington County|RI|05136|United States|-5|condo| +29962|AAAAAAAAKAFHAAAA|44|Smith |Cir.|Suite 450|Wildwood|Carroll County|IN|46871|United States|-5|condo| +29963|AAAAAAAALAFHAAAA|448|12th Elm|Blvd|Suite E|Sunnyside|Carter County|KY|41952|United States|-6|condo| +29964|AAAAAAAAMAFHAAAA|447|Railroad Williams|Pkwy|Suite T|Newport|Marshall County|IN|41521|United States|-5|apartment| +29965|AAAAAAAANAFHAAAA|322|Spring |Road|Suite U|Forest|Rio Grande County|CO|87537|United States|-7|apartment| +29966|AAAAAAAAOAFHAAAA|168|4th |Street|Suite Y|Shady Grove|Nueces County|TX|72812|United States|-6|condo| +29967|AAAAAAAAPAFHAAAA|762|College Fourth|Street|Suite 200|Riverdale|Faribault County|MN|59391|United States|-6|single family| +29968|AAAAAAAAABFHAAAA|942|11th |Cir.|Suite 100|Star|Robertson County|KY|40725|United States|-5|apartment| +29969|AAAAAAAABBFHAAAA|84|Johnson Mill|Street|Suite X|Franklin|Murray County|GA|39101|United States|-5|single family| +29970|AAAAAAAACBFHAAAA|216|Hickory Ridge|Avenue|Suite 160|Waterloo|Sherman County|NE|61675|United States|-7|single family| +29971|AAAAAAAADBFHAAAA|101|North |Ln|Suite P|Brownsville|Cross County|AR|79310|United States|-6|apartment| +29972|AAAAAAAAEBFHAAAA|577|Wilson |Avenue|Suite G|Clinton|Madison County|AR|78222|United States|-6|condo| +29973|AAAAAAAAFBFHAAAA|954|7th 9th|Boulevard|Suite 310|Pleasant Hill|Jefferson County|FL|33604|United States|-5|apartment| +29974|AAAAAAAAGBFHAAAA|224|4th 1st|RD|Suite 210|Highland Park|Webster County|KY|46534|United States|-5|single family| +29975|AAAAAAAAHBFHAAAA|762|Mill |Pkwy|Suite 190|Newport|Livingston County|MO|61521|United States|-6|apartment| +29976|AAAAAAAAIBFHAAAA|720|Davis |Ln|Suite 300|Clinton|Saratoga County|NY|18222|United States|-5|single family| +29977|AAAAAAAAJBFHAAAA|990|Second Walnut|Way|Suite J|Midway|Hernando County|FL|31904|United States|-5|single family| +29978|AAAAAAAAKBFHAAAA|214|View |Street|Suite 410|Mount Vernon|Fredericksburg city|VA|28482|United States|-5|single family| +29979|AAAAAAAALBFHAAAA|278|10th 14th|ST|Suite 430|Springfield|Hart County|GA|39303|United States|-5|single family| +29980|AAAAAAAAMBFHAAAA|731|Forest |Cir.|Suite 180|Oak Grove|Ottawa County|OH|48370|United States|-5|condo| +29981|AAAAAAAANBFHAAAA|332|Cherry |||Lakewood|Saline County|NE|68877|||| +29982|AAAAAAAAOBFHAAAA|425|11th 12th|Cir.|Suite W|Bridgeport|Cameron Parish|LA|75817|United States|-6|condo| +29983|AAAAAAAAPBFHAAAA|614|First Lake|Wy|Suite U|Pine Grove|Jeff Davis County|TX|74593|United States|-6|single family| +29984|AAAAAAAAACFHAAAA|86|Mill Spring|Dr.|Suite H|Liberty|Teton County|ID|83451|United States|-7|apartment| +29985|AAAAAAAABCFHAAAA|311|Walnut West|Blvd|Suite B|Spring Valley|Rogers County|OK|76060|United States|-6|condo| +29986|AAAAAAAACCFHAAAA|401|13th |Ave|Suite 110|Rutland|Lyon County|KS|68375|United States|-6|condo| +29987|AAAAAAAADCFHAAAA|715|Lincoln |Boulevard|Suite 390|Shiloh|Warren County|IN|49275|United States|-5|condo| +29988|AAAAAAAAECFHAAAA|487|Birch Lincoln|Street|Suite 450|Clifford|Butler County|IA|58164|United States|-6|single family| +29989|AAAAAAAAFCFHAAAA|793|Wilson Hickory|Cir.|Suite 70|Hopewell|Waynesboro city|VA|20587|United States|-5|apartment| +29990|AAAAAAAAGCFHAAAA|12|Hill |Drive|Suite P|Harmony|Caswell County|NC|25804|United States|-5|apartment| +29991|AAAAAAAAHCFHAAAA|967|Hillcrest 3rd|Street|Suite T|Oak Grove|Washington County|OR|98370|United States|-8|apartment| +29992|AAAAAAAAICFHAAAA|551|Lakeview |RD|Suite 50|Oak Ridge|Custer County|MT|68371|United States|-7|apartment| +29993|AAAAAAAAJCFHAAAA|201|Cedar 4th|Parkway|Suite 440|Mount Vernon|Martin County|TX|78482|United States|-6|apartment| +29994|AAAAAAAAKCFHAAAA|537|Wilson |Court|Suite V|Ashland|Moffat County|CO|84244|United States|-7|single family| +29995|AAAAAAAALCFHAAAA|363|Cedar Dogwood|Ln|Suite 150|Westgate|Monongalia County|WV|22366|United States|-5|condo| +29996|AAAAAAAAMCFHAAAA|11|First |Pkwy|Suite 460|Mount Vernon|Manassas Park city|VA|28482|United States|-5|apartment| +29997|AAAAAAAANCFHAAAA|907|Park 3rd|Ln|Suite Q|Freeport|Prentiss County|MS|51844|United States|-6|single family| +29998|AAAAAAAAOCFHAAAA|993|Franklin |Court|Suite 390|Midway|Musselshell County|MT|61904|United States|-7|condo| +29999|AAAAAAAAPCFHAAAA|558|Washington Lakeview|Ave|Suite 210|Glendale|Las Animas County|CO|83951|United States|-7|single family| +30000|AAAAAAAAADFHAAAA|712|Adams |Blvd|Suite X|Walnut Grove|Waushara County|WI|57752|United States|-6|single family| +30001|AAAAAAAABDFHAAAA|584|Sycamore |Lane|Suite 450|Macedonia|Vermilion Parish|LA|71087|United States|-6|single family| +30002|AAAAAAAACDFHAAAA|348|Fifth |Street|Suite 490|Oakdale|Nelson County|VA|29584|United States|-5|single family| +30003|AAAAAAAADDFHAAAA|291|3rd Birch|Avenue|Suite D|Greenwood|Quay County|NM|88828|United States|-7|condo| +30004|AAAAAAAAEDFHAAAA|179|Walnut |Way|Suite R|Oakdale|Barbour County|AL|39584|United States|-6|apartment| +30005|AAAAAAAAFDFHAAAA|715|Broadway Walnut|Dr.|Suite X|Shady Grove|Cottle County|TX|72812|United States|-6|single family| +30006|AAAAAAAAGDFHAAAA|343|Main |Cir.|Suite G|Kingston|Doniphan County|KS|64975|United States|-6|condo| +30007|AAAAAAAAHDFHAAAA|187|Second Green|ST|Suite X|Jamestown|Hemphill County|TX|76867|United States|-6|apartment| +30008|AAAAAAAAIDFHAAAA|359|Lake Main|Pkwy|Suite Q|Friendship|Kingsbury County|SD|54536|United States|-7|apartment| +30009|AAAAAAAAJDFHAAAA|229|Washington Fourteenth|Blvd|Suite 360|Lincoln|Franklin County|MA|01889|United States|-5|single family| +30010|AAAAAAAAKDFHAAAA|579|Chestnut Smith|Cir.|Suite 460|Valley View|Marshall County|MN|55124|United States|-6|condo| +30011|AAAAAAAALDFHAAAA|619|Lakeview Ridge|Drive|Suite S|Farmington|Wheatland County|MT|69145|United States|-7|condo| +30012|AAAAAAAAMDFHAAAA|297|View Second|Dr.|Suite 290|Walnut Grove|Grant County|WV|27752|United States|-5|condo| +30013|AAAAAAAANDFHAAAA|485|Sycamore Lakeview|Ct.|Suite K|Walnut|Graham County|NC|26245|United States|-5|condo| +30014|AAAAAAAAODFHAAAA|833|Pine View|Road|Suite M|Florence|Marengo County|AL|33394|United States|-6|condo| +30015|AAAAAAAAPDFHAAAA|138|Fourth Center|Ave|Suite A|Glendale|Richland County|SC|23951|United States|-5|single family| +30016|AAAAAAAAAEFHAAAA|||ST|Suite 420||Cabarrus County|NC||United States|-5|single family| +30017|AAAAAAAABEFHAAAA|115|13th 15th|Cir.|Suite 230|Mount Olive|Carver County|MN|58059|United States|-6|single family| +30018|AAAAAAAACEFHAAAA|932|Williams Adams|Avenue|Suite W|Bunker Hill|Madison County|KY|40150|United States|-5|single family| +30019|AAAAAAAADEFHAAAA|522|View Oak|Ct.|Suite I|Brownsville|Putnam County|FL|39310|United States|-5|single family| +30020|AAAAAAAAEEFHAAAA|170|Pine Lincoln|Cir.|Suite R|Waterloo|Forrest County|MS|51675|United States|-6|apartment| +30021|AAAAAAAAFEFHAAAA|467|Valley Birch|Ln|Suite 260|Five Forks|Bradford County|FL|32293|United States|-5|single family| +30022|AAAAAAAAGEFHAAAA|433|7th Main|RD|Suite S|Belmont|Jefferson County|TN|30191|United States|-6|condo| +30023|AAAAAAAAHEFHAAAA|536|Central |Road|Suite 130|Wilson|Wicomico County|MD|26971|United States|-5|condo| +30024|AAAAAAAAIEFHAAAA|669|Cedar |Way|Suite N|Lebanon|Isanti County|MN|52898|United States|-6|condo| +30025|AAAAAAAAJEFHAAAA|663|2nd Valley|ST|Suite P|Blue Springs|Cass County|MI|44686|United States|-5|single family| +30026|AAAAAAAAKEFHAAAA|951|Dogwood Cedar|Ave|Suite F|Oak Grove|Blount County|AL|38370|United States|-6|apartment| +30027|AAAAAAAALEFHAAAA|447|Sixth |Court|Suite K|Belmont|Racine County|WI|50191|United States|-6|condo| +30028|AAAAAAAAMEFHAAAA|849|Railroad |Wy|Suite C|Little River|Taylor County|TX|70319|United States|-6|single family| +30029|AAAAAAAANEFHAAAA|75|Adams Mill|Ave|Suite 220|Centerville|Decatur County|TN|30059|United States|-5|apartment| +30030|AAAAAAAAOEFHAAAA|782|Highland Franklin|Court|Suite 490|Hopewell|Cottonwood County|MN|50587|United States|-6|condo| +30031|AAAAAAAAPEFHAAAA|726|5th Lincoln|Blvd|Suite W|Midway|Garfield County|UT|81904|United States|-7|apartment| +30032|AAAAAAAAAFFHAAAA|328|Valley West|Circle|Suite I|Oakdale|Emmet County|MI|49584|United States|-5|condo| +30033|AAAAAAAABFFHAAAA|940|||||Knox County|||United States|-6|| +30034|AAAAAAAACFFHAAAA|74|3rd |Ave|Suite 270|Plainview|Foard County|TX|73683|United States|-6|single family| +30035|AAAAAAAADFFHAAAA|332|Spruce Park|ST|Suite 460|Greenfield|Dearborn County|IN|45038|United States|-5|single family| +30036|AAAAAAAAEFFHAAAA|936|Cedar |Ave|Suite M|Oakland|Kemper County|MS|59843|United States|-6|single family| +30037|AAAAAAAAFFFHAAAA|529|1st North|Street|Suite 210|Union|Little River County|AR|78721|United States|-6|single family| +30038|AAAAAAAAGFFHAAAA|58|Oak |Dr.|Suite 360|Shiloh|Barry County|MI|49275|United States|-5|condo| +30039|AAAAAAAAHFFHAAAA|771|South Lakeview|Circle|Suite E|Harmony|Crane County|TX|75804|United States|-6|single family| +30040|AAAAAAAAIFFHAAAA|913|Chestnut |Pkwy|Suite G|Bethel|Mineral County|NV|85281|United States|-8|single family| +30041|AAAAAAAAJFFHAAAA|521|Jefferson |Parkway|Suite 120|Enterprise|Wilcox County|GA|31757|United States|-5|apartment| +30042|AAAAAAAAKFFHAAAA|325|Walnut Cherry|RD|Suite 30|Sunnyside|Stark County|OH|41952|United States|-5|single family| +30043|AAAAAAAALFFHAAAA|790|Ash Hillcrest|Drive|Suite W|Riverview|Grant County|ND|59003|United States|-6|single family| +30044|AAAAAAAAMFFHAAAA|514|View |Wy|Suite Q|Oak Ridge|Sanders County|MT|68371|United States|-7|apartment| +30045|AAAAAAAANFFHAAAA|201|Sycamore Eigth|Ave|Suite H|Salem|Cheatham County|TN|38048|United States|-5|condo| +30046|AAAAAAAAOFFHAAAA|213|2nd |Circle|Suite 400|Antioch|Marshall County|WV|28605|United States|-5|apartment| +30047|AAAAAAAAPFFHAAAA|845|Lake |Pkwy|Suite 30|Silver Creek|Nicholas County|WV|24546|United States|-5|single family| +30048|AAAAAAAAAGFHAAAA|918|Mill |Cir.|Suite 130|Union Hill|Clay County|NC|27746|United States|-5|condo| +30049|AAAAAAAABGFHAAAA|739|5th |Ct.|Suite 90|Hamilton|Boise County|ID|82808|United States|-7|condo| +30050|AAAAAAAACGFHAAAA|843|Park |Ave|Suite 390|Springdale|Lorain County|OH|48883|United States|-5|single family| +30051|AAAAAAAADGFHAAAA|108|Smith |Ln|Suite U|Jamestown|Hampshire County|MA|07467|United States|-5|single family| +30052|AAAAAAAAEGFHAAAA|272|Central |Street|Suite 290|Woodville|Crosby County|TX|74289|United States|-6|single family| +30053|AAAAAAAAFGFHAAAA|640|2nd |Road|Suite A|Deerfield|Rice County|MN|59840|United States|-6|apartment| +30054|AAAAAAAAGGFHAAAA|401|Central Park|Circle|Suite 90|Mount Zion|Osceola County|IA|58054|United States|-6|apartment| +30055|AAAAAAAAHGFHAAAA|243|East Spruce|Blvd|Suite E|Sulphur Springs|Fall River County|SD|58354|United States|-6|single family| +30056|AAAAAAAAIGFHAAAA|578|Maple |Pkwy|Suite 410|Rankin|Morgan County|CO|82621|United States|-7|condo| +30057|AAAAAAAAJGFHAAAA|||ST||Highland Park|Long County|GA|36534||-5|condo| +30058|AAAAAAAAKGFHAAAA|545|Sunset Main|Ave|Suite 150|Riverview|Musselshell County|MT|69003|United States|-7|condo| +30059|AAAAAAAALGFHAAAA|442|Fourth |Pkwy|Suite B|Philadelphia|Botetourt County|VA|25591|United States|-5|single family| +30060|AAAAAAAAMGFHAAAA|538|15th Central|Wy|Suite R|Plainview|Mercer County|OH|43683|United States|-5|condo| +30061|AAAAAAAANGFHAAAA|933|Mill |Blvd|Suite 70|Oakdale|Meade County|KY|49584|United States|-5|condo| +30062|AAAAAAAAOGFHAAAA|453|Washington 1st|RD|Suite L|Woodlawn|Cherokee County|IA|54098|United States|-6|condo| +30063|AAAAAAAAPGFHAAAA|802|Elm Oak|Circle|Suite L|Concord|Grant County|AR|74107|United States|-6|condo| +30064|AAAAAAAAAHFHAAAA|600|5th |Cir.|Suite S|Valley View|Concho County|TX|75124|United States|-6|condo| +30065|AAAAAAAABHFHAAAA|850|14th 14th|Road|Suite H|||SD||United States|-6|| +30066|AAAAAAAACHFHAAAA|339|1st |Wy|Suite 20|Newport|Crawford County|PA|11521|United States|-5|apartment| +30067|AAAAAAAADHFHAAAA|928|First Oak|Ct.|Suite C|Summit|Delta County|MI|40499|United States|-5|condo| +30068|AAAAAAAAEHFHAAAA|61|8th Cedar|Parkway|Suite 320|Crossroads|Caddo Parish|LA|70534|United States|-6|apartment| +30069|AAAAAAAAFHFHAAAA|280|Green |Dr.|Suite X|Edgewater|Oglethorpe County|GA|30635|United States|-5|condo| +30070|AAAAAAAAGHFHAAAA|354|Chestnut Church|Street|Suite 350|Stafford|Erie County|PA|14980|United States|-5|apartment| +30071|AAAAAAAAHHFHAAAA|862|14th Ash|Circle|Suite 300|Georgetown|Madison County|ID|87057|United States|-7|single family| +30072|AAAAAAAAIHFHAAAA|334|Smith |Drive|Suite J|Oak Ridge|Grand County|UT|88371|United States|-7|apartment| +30073|AAAAAAAAJHFHAAAA|496|Davis Laurel|Street|Suite 480|Shiloh|Sussex County|NJ|09875|United States|-5|single family| +30074|AAAAAAAAKHFHAAAA|679|Ash |Dr.|Suite 480|Clifton|Gilmer County|WV|28014|United States|-5|condo| +30075|AAAAAAAALHFHAAAA|185|East 5th|Ave|Suite 120|Ashland|King and Queen County|VA|24244|United States|-5|condo| +30076|AAAAAAAAMHFHAAAA|695|2nd Spring||Suite 30||||81749|United States|-7|condo| +30077|AAAAAAAANHFHAAAA|599|Seventh Locust|Way|Suite 450|Friendship|Alexander County|IL|64536|United States|-6|apartment| +30078|AAAAAAAAOHFHAAAA|930|7th |Drive|Suite D|Waterloo|Granville County|NC|21675|United States|-5|single family| +30079|AAAAAAAAPHFHAAAA|911|9th |ST|Suite T|Deerfield|Benton County|TN|39840|United States|-5|condo| +30080|AAAAAAAAAIFHAAAA||Hill |Drive|Suite X||Randolph County|||United States|-5|single family| +30081|AAAAAAAABIFHAAAA|731|Second 8th|Dr.|Suite 310|Glendale|Maui County|HI|93951|United States|-10|single family| +30082|AAAAAAAACIFHAAAA|371|Sycamore |Court|Suite P|Lebanon|King and Queen County|VA|22898|United States|-5|single family| +30083|AAAAAAAADIFHAAAA|495|Lincoln |Court|Suite 480|Friendship|Powhatan County|VA|24536|United States|-5|single family| +30084|AAAAAAAAEIFHAAAA|938|11th |Wy|Suite U|Lake Forest|Hancock County|OH|46000|United States|-5|apartment| +30085|AAAAAAAAFIFHAAAA|362|13th Fourth|Ln|Suite I|Summit|Tioga County|PA|10499|United States|-5|apartment| +30086|AAAAAAAAGIFHAAAA|746|Lake |Dr.|Suite J|The Meadows|Reagan County|TX|70026|United States|-6|single family| +30087|AAAAAAAAHIFHAAAA|260|Chestnut Johnson|Street|Suite P|Edgewood|Blackford County|IN|40069|United States|-5|apartment| +30088|AAAAAAAAIIFHAAAA|376|River Franklin|Cir.|Suite B|Montezuma|Sebastian County|AR|72150|United States|-6|single family| +30089|AAAAAAAAJIFHAAAA|430|Cedar |Lane|Suite 310|Mount Pleasant|Elk County|PA|11933|United States|-5|condo| +30090|AAAAAAAAKIFHAAAA|484|View Green|Wy|Suite X|Franklin|Union County|AR|79101|United States|-6|condo| +30091|AAAAAAAALIFHAAAA|730|Jefferson Sunset|Ln|Suite 80|Stafford|Morgan County|AL|34980|United States|-6|condo| +30092|AAAAAAAAMIFHAAAA|276|Fourth |Street|Suite 50|Woodlawn|Anchorage Borough|AK|94098|United States|-9|condo| +30093|AAAAAAAANIFHAAAA|255|10th |Lane|Suite 260|Kingston|Dewey County|OK|74975|United States|-6|condo| +30094|AAAAAAAAOIFHAAAA|962|5th View|Wy|Suite 150|Oak Hill|Lincoln County|MT|67838|United States|-7|single family| +30095|AAAAAAAAPIFHAAAA|391|Central |Parkway|Suite M|Buena Vista|Bates County|MO|65752|United States|-6|single family| +30096|AAAAAAAAAJFHAAAA|94|Pine Lakeview|Lane|Suite J|Wilson|Taylor County|FL|36971|United States|-5|apartment| +30097|AAAAAAAABJFHAAAA|390|Adams College|Wy|Suite 370|Spring Valley|McKenzie County|ND|56060|United States|-6|apartment| +30098|AAAAAAAACJFHAAAA|310|6th Spring|Cir.|Suite 50|Pine Grove|Kalkaska County|MI|44593|United States|-5|condo| +30099|AAAAAAAADJFHAAAA|223|11th |Circle|Suite 330|Macedonia|Brown County|WI|51087|United States||apartment| +30100|AAAAAAAAEJFHAAAA|741|Elm 9th|Way|Suite 110|Edgewood|Lake County|CO|80069|United States|-7|single family| +30101|AAAAAAAAFJFHAAAA|574|Main |Ave|Suite 350|Lakeview|Bradley County|AR|78579|United States|-6|single family| +30102|AAAAAAAAGJFHAAAA|386|Jefferson |Dr.|Suite P|Granite|Billings County|ND|56284|United States|-6|condo| +30103|AAAAAAAAHJFHAAAA|856|Elm |Avenue|Suite 460|Cedar Grove|Dunn County|ND|50411|United States|-6|condo| +30104|AAAAAAAAIJFHAAAA|505|14th |Avenue|Suite N|Unionville|Cross County|AR|71711|United States|-6|apartment| +30105|AAAAAAAAJJFHAAAA|508|Center Main|Avenue|Suite 480|Green Acres|Pershing County|NV|87683|United States|-8|single family| +30106|AAAAAAAAKJFHAAAA|416|9th |ST|Suite 340|Crossroads|Jackson County|TX|70534|United States|-6|single family| +30107|AAAAAAAALJFHAAAA|128|Johnson |Lane|Suite N|Valley View|Scott County|KY|45124|United States|-5|condo| +30108|AAAAAAAAMJFHAAAA|479|Davis |RD|Suite O|Mount Zion|Warren County|NC|28054|United States|-5|condo| +30109|AAAAAAAANJFHAAAA|8|Second |Boulevard|Suite 280|Ellisville|Monona County|IA|56820|United States|-6|apartment| +30110|AAAAAAAAOJFHAAAA|659|Church Hickory|Boulevard|Suite 90|Shady Grove|Kendall County|TX|72812|United States|-6|apartment| +30111|AAAAAAAAPJFHAAAA|||Way|||West Feliciana Parish|LA|79431||-6|| +30112|AAAAAAAAAKFHAAAA|257|Park Lake|Parkway|Suite L|Plainville|Warren County|NJ|06715|United States|-5|apartment| +30113|AAAAAAAABKFHAAAA|57|View 12th|Wy|Suite 160|Mount Vernon|Custer County|MT|68482|United States|-7|apartment| +30114|AAAAAAAACKFHAAAA|757|8th |Lane|Suite W|Woodville|Tarrant County|TX|74289|United States|-6|apartment| +30115|AAAAAAAADKFHAAAA|363|Maple Fifth|Parkway|Suite 190|Kingston|Richland County|MT|64975|United States|-7|single family| +30116|AAAAAAAAEKFHAAAA|206||Road|Suite A||Kossuth County|||United States||condo| +30117|AAAAAAAAFKFHAAAA|461|Locust Park|Blvd|Suite D|Liberty|Carroll County|AR|73451|United States|-6|single family| +30118|AAAAAAAAGKFHAAAA|673|Dogwood |Wy|Suite 240|Sunnyside|Tulare County|CA|91952|United States|-8|condo| +30119|AAAAAAAAHKFHAAAA|161|Locust Park|Lane|Suite M|Oakland|Dickenson County|VA|29843|United States|-5|apartment| +30120|AAAAAAAAIKFHAAAA|850|Ninth |Way|Suite 200|Youngstown|Harding County|NM|80001|United States|-7|apartment| +30121|AAAAAAAAJKFHAAAA|98|Washington |Boulevard|Suite B|Fairfield|Clare County|MI|46192|United States|-5|single family| +30122|AAAAAAAAKKFHAAAA|610|Highland Sunset|Blvd|Suite S|Riverdale|Escambia County|AL|39391|United States|-6|single family| +30123|AAAAAAAALKFHAAAA|719||RD|Suite Q||Fayette County|||United States||apartment| +30124|AAAAAAAAMKFHAAAA|735|5th Lake|Road|Suite L|River Oaks|Long County|GA|38075|United States|-5|single family| +30125|AAAAAAAANKFHAAAA|882|8th 8th|Wy|Suite G|Colonial Heights|Pinellas County|FL|33425|United States|-5|apartment| +30126|AAAAAAAAOKFHAAAA|855|Meadow Davis|Boulevard|Suite 200|Montague|Sullivan County|IN|44062|United States|-5|single family| +30127|AAAAAAAAPKFHAAAA|669|Hillcrest Lakeview|Road|Suite N|Lakeside|Jackson County|MN|59532|United States|-6|apartment| +30128|AAAAAAAAALFHAAAA|370|Oak |Court|Suite 120|Fairview|Santa Rosa County|FL|35709|United States|-5|apartment| +30129|AAAAAAAABLFHAAAA|320|Hickory |Cir.|Suite C|Harrisburg|Marion County|KY|40974|United States|-5|condo| +30130|AAAAAAAACLFHAAAA|890||Cir.|Suite S||Hemphill County|||||| +30131|AAAAAAAADLFHAAAA|825|Williams Sunset||Suite O||Edwards County||63604||-6|apartment| +30132|AAAAAAAAELFHAAAA|880|5th River|Court|Suite A|Oak Grove|Guadalupe County|TX|78370|United States|-6|condo| +30133|AAAAAAAAFLFHAAAA|187|Ridge Center|Ct.|Suite N|Deerfield|Broward County|FL|39840|United States|-5|single family| +30134|AAAAAAAAGLFHAAAA|136|Seventh West|||Mount Olive|Clackamas County|||United States|-8|condo| +30135|AAAAAAAAHLFHAAAA|919|Spring |Dr.|Suite K|Saint Johns|Valley County|NE|65717|United States|-7|single family| +30136|AAAAAAAAILFHAAAA|299|Madison 12th|Ave|Suite 350|Antioch|Moffat County|CO|88605|United States|-7|condo| +30137|AAAAAAAAJLFHAAAA|523|Willow |Boulevard|Suite K|Harmony|Lamb County|TX|75804|United States|-6|apartment| +30138|AAAAAAAAKLFHAAAA|420|Maple 3rd|Dr.|Suite M|Woodrow|Brown County|NE|64273|United States|-6|single family| +30139|AAAAAAAALLFHAAAA|451|Washington |Blvd|Suite 320|Pine Grove|Jackson County|FL|34593|United States|-5|condo| +30140|AAAAAAAAMLFHAAAA|278|Railroad |Lane|Suite K|Guilford|Allen County|KY|44408|United States|-6|single family| +30141|AAAAAAAANLFHAAAA|863|10th Forest|Court|Suite B|Spring Hill|Calhoun County|FL|36787|United States|-5|single family| +30142|AAAAAAAAOLFHAAAA|825|6th |Lane|Suite 130|Oak Hill|Benton County|OR|97838|United States|-8|apartment| +30143|AAAAAAAAPLFHAAAA|139|Maple |Street|Suite J|Clinton|Ralls County|MO|68222|United States|-6|apartment| +30144|AAAAAAAAAMFHAAAA|227|Oak 13th|Dr.|Suite K|Springtown|Menard County|TX|79858|United States|-6|apartment| +30145|AAAAAAAABMFHAAAA|849|3rd Railroad|Ct.|Suite B|Walnut Grove|Orleans County|VT|08352|United States|-5|condo| +30146|AAAAAAAACMFHAAAA|488|8th |Pkwy|Suite O|Crossroads|Allen County|KS|60534|United States|-6|apartment| +30147|AAAAAAAADMFHAAAA|499|West |Parkway|Suite P|Georgetown|Hot Springs County|WY|87057|United States|-7|apartment| +30148|AAAAAAAAEMFHAAAA|786|Pine |||Brownsville|||59310|United States||apartment| +30149|AAAAAAAAFMFHAAAA|505|Lincoln Spring|Ln|Suite 260|Belmont|Ocean County|NJ|00791|United States|-5|condo| +30150|AAAAAAAAGMFHAAAA|3|4th |Court|Suite 420|Oak Ridge|Bradley County|AR|78371|United States|-6|condo| +30151|AAAAAAAAHMFHAAAA|811|Adams Elm|RD|Suite 360|Caledonia|Sequatchie County|TN|37411|United States|-6|condo| +30152|AAAAAAAAIMFHAAAA|908|Elm |RD|Suite K|Farmington|Bailey County|TX|79145|United States|-6|apartment| +30153|AAAAAAAAJMFHAAAA|677|Johnson Smith|Avenue|Suite 390|Pleasant Grove|Treasure County|MT|64136|United States|-7|condo| +30154|AAAAAAAAKMFHAAAA|38|14th |Drive|Suite Q|Murray|Butte County|ID|82150|United States|-7|apartment| +30155|AAAAAAAALMFHAAAA|376|Church 4th|Ct.|Suite H|Amherst|District of Columbia|DC|28119|United States|-5|condo| +30156|AAAAAAAAMMFHAAAA|924|Smith |Drive|Suite D|Wildwood|Fairfield County|OH|46871|United States|-5|apartment| +30157|AAAAAAAANMFHAAAA|75|11th View|Lane|Suite 260|Greenwood|Zapata County|TX|78828|United States|-6|apartment| +30158|AAAAAAAAOMFHAAAA|129|Sycamore |Ave|Suite 480|Concord|Jefferson Davis Parish|LA|74107|United States|-6|condo| +30159|AAAAAAAAPMFHAAAA|||Avenue|||||69231|United States||condo| +30160|AAAAAAAAANFHAAAA|403|Pine Ridge|Ct.|Suite 30|Oakland|Gray County|TX|79843|United States|-6|condo| +30161|AAAAAAAABNFHAAAA||||Suite L||Preble County|OH|45566||-5|| +30162|AAAAAAAACNFHAAAA|683|View Mill|Parkway|Suite I|Bunker Hill|Mitchell County|NC|20150|United States|-5|single family| +30163|AAAAAAAADNFHAAAA|991|River |Court|Suite 30|Edgewood|Stark County|OH|40069|United States|-5|condo| +30164|AAAAAAAAENFHAAAA|506|Hillcrest Hill|Circle|Suite 400|Enterprise|Clay County|SD|51757|United States|-6|apartment| +30165|AAAAAAAAFNFHAAAA|391|Seventh |Blvd|Suite 440|Bridgeport|Douglas County|WI|55817|United States|-6|condo| +30166|AAAAAAAAGNFHAAAA|722|Cedar Smith|Avenue|Suite P|Highland Park|Scott County|VA|26534|United States|-5|single family| +30167|AAAAAAAAHNFHAAAA|344|10th 12th|ST|Suite 220|Mount Zion|Imperial County|CA|98054|United States|-8|apartment| +30168|AAAAAAAAINFHAAAA|99|Walnut 5th|Parkway|Suite J|Oak Ridge|Woodruff County|AR|78371|United States|-6|single family| +30169|AAAAAAAAJNFHAAAA|3|Washington |RD|Suite 250|Forest Hills|Toombs County|GA|39237|United States|-5|single family| +30170|AAAAAAAAKNFHAAAA|978|Meadow |Ln|Suite 310|Greenfield|Webb County|TX|75038|United States|-6|single family| +30171|AAAAAAAALNFHAAAA|126|Park |Drive|Suite 100|Shady Grove|Sequoyah County|OK|72812|United States|-6|apartment| +30172|AAAAAAAAMNFHAAAA|335|Seventh |Dr.|Suite S|New Hope|Bath County|KY|49431|United States|-6|apartment| +30173|AAAAAAAANNFHAAAA|307|2nd |Avenue|Suite T|Enterprise|Broadwater County|MT|61757|United States|-7|single family| +30174|AAAAAAAAONFHAAAA|746|Seventh River|Parkway|Suite L|Somerset|Ashley County|AR|75580|United States|-6|apartment| +30175|AAAAAAAAPNFHAAAA|402|8th |Avenue|Suite 380|Spring Hill|Wharton County|TX|76787|United States|-6|single family| +30176|AAAAAAAAAOFHAAAA|543|8th |Boulevard|Suite W|Brownsville|DeBaca County|NM|89310|United States|-7|condo| +30177|AAAAAAAABOFHAAAA|240|14th Smith|Parkway|Suite 30|Sulphur Springs|Jefferson County|IA|58354|United States|-6|apartment| +30178|AAAAAAAACOFHAAAA|89|Valley Lincoln|Avenue|Suite 240|Riverview|Sauk County|WI|59003|United States|-6|single family| +30179|AAAAAAAADOFHAAAA|30|Adams Hill|Boulevard|Suite 420|Ellsworth|Val Verde County|TX|75079|United States|-6|apartment| +30180|AAAAAAAAEOFHAAAA|497|Williams Railroad|Dr.|Suite V|Unionville|Sebastian County|AR|71711|United States|-6|condo| +30181|AAAAAAAAFOFHAAAA|794|Railroad Lee|Parkway|Suite V|Walnut Grove|Weber County|UT|87752|United States|-7|condo| +30182|AAAAAAAAGOFHAAAA|573|14th |Blvd|Suite 240|Maple Grove|Meigs County|OH|48252|United States|-5|condo| +30183|AAAAAAAAHOFHAAAA|989|8th |Way|Suite V|Oak Hill|Erie County|OH|47838|United States|-5|single family| +30184|AAAAAAAAIOFHAAAA|94|Cedar |Pkwy|Suite 190|Belmont|Russell County|KY|40191|United States|-5|apartment| +30185|AAAAAAAAJOFHAAAA|267|4th |Court|Suite 210|Greenville|Page County|VA|21387|United States|-5|single family| +30186|AAAAAAAAKOFHAAAA|966|Sixth |Way|Suite 110|Belfast|Washington County|TN|30125|United States|-6|condo| +30187|AAAAAAAALOFHAAAA|900|Chestnut Adams|ST|Suite C|Belmont|Wilson County|KS|60191|United States|-6|condo| +30188|AAAAAAAAMOFHAAAA|971|Meadow 8th|Dr.|Suite 60|Deerfield|Summit County|CO|89840|United States|-7|condo| +30189|AAAAAAAANOFHAAAA|63||Circle|Suite 340|||NC||||| +30190|AAAAAAAAOOFHAAAA|659|Sunset Forest|Cir.|Suite 400|Hardy|Emery County|UT|85354|United States|-7|apartment| +30191|AAAAAAAAPOFHAAAA|513|Wilson Washington|Parkway|Suite F|Carthage|Barnes County|ND|51529|United States|-6|condo| +30192|AAAAAAAAAPFHAAAA|596|Second |Ct.|Suite A|Jamestown|Baxter County|AR|76867|United States|-6|single family| +30193|AAAAAAAABPFHAAAA|962|15th Jefferson|Dr.|Suite R|Edgewood|Dixon County|NE|60069|United States|-6|apartment| +30194|AAAAAAAACPFHAAAA|80|Maple Elm|Boulevard|Suite R|Phoenix|Wayne County|MS|52276|United States|-6|single family| +30195|AAAAAAAADPFHAAAA|437|Sycamore Main|Dr.|Suite 130|Salem|La Salle County|IL|68048|United States|-6|apartment| +30196|AAAAAAAAEPFHAAAA|27|View 2nd|Cir.|Suite T|Arthur|Worcester County|MA|06565|United States|-5|condo| +30197|AAAAAAAAFPFHAAAA|230|Walnut Broadway|Ave|Suite 310|Ruth|Hancock County|KY|40309|United States|-6|apartment| +30198|AAAAAAAAGPFHAAAA|387||Ct.|Suite 90||Van Buren County|||United States||| +30199|AAAAAAAAHPFHAAAA|393|Maple 6th|Dr.|Suite 270|Richville|Lincoln County|AR|75945|United States|-6|single family| +30200|AAAAAAAAIPFHAAAA|220|15th |Pkwy|Suite 350|Waterloo|McLennan County|TX|71675|United States|-6|single family| +30201|AAAAAAAAJPFHAAAA|465|Woodland Poplar|Ln|Suite X|Liberty|Boone County|IA|53451|United States|-6|apartment| +30202|AAAAAAAAKPFHAAAA|153|Smith |Parkway|Suite 190|Highland|Phillips County|KS|69454|United States|-6|condo| +30203|AAAAAAAALPFHAAAA|956||Street|Suite 160||Jackson County|WV|20534|United States||condo| +30204|AAAAAAAAMPFHAAAA|352|Dogwood 5th|Way|Suite 170|Brownsville|Itasca County|MN|59310|United States|-6|single family| +30205|AAAAAAAANPFHAAAA|778|10th |Ave|Suite 390|Plainview|Mason County|MI|43683|United States|-5|condo| +30206|AAAAAAAAOPFHAAAA|588|4th Mill|RD|Suite I|Mount Vernon|Renville County|MN|58482|United States|-6|apartment| +30207|AAAAAAAAPPFHAAAA|164|8th |Wy|Suite S|Mountain View|Hughes County|OK|74466|United States|-6|single family| +30208|AAAAAAAAAAGHAAAA|508|11th Walnut|Blvd|Suite 190|Newport|Madison County|NY|11521|United States|-5|single family| +30209|AAAAAAAABAGHAAAA|7|Maple 1st|Blvd|Suite 430|Kingston|Nez Perce County|ID|84975|United States|-7|single family| +30210|AAAAAAAACAGHAAAA|386|Jackson 6th|RD|Suite N|Little River|Glades County|FL|30319|United States|-5|condo| +30211|AAAAAAAADAGHAAAA|422|Washington Oak|Court|Suite K|Bethel|Greeley County|NE|65281|United States|-6|apartment| +30212|AAAAAAAAEAGHAAAA|160|Elm |Wy|Suite V|Union|Clark County|OH|48721|United States|-5|condo| +30213|AAAAAAAAFAGHAAAA|733|Madison |Circle|Suite T|Highland Park|Shelby County|OH|46534|United States|-5|apartment| +30214|AAAAAAAAGAGHAAAA|520|Lee Franklin|Ct.|Suite Y|Edgewood|Radford city|VA|20069|United States|-5|apartment| +30215|AAAAAAAAHAGHAAAA|678|Elm |Parkway|Suite E|White Oak|Franklin County|VT|07268|United States|-5|single family| +30216|AAAAAAAAIAGHAAAA|576|Smith |Ln|Suite 430|Plainview|Chickasaw County|MS|53683|United States|-6|apartment| +30217|AAAAAAAAJAGHAAAA|980|Laurel |Blvd|Suite Y|Deerfield|Geary County|KS|69840|United States|-6|apartment| +30218|AAAAAAAAKAGHAAAA|387|Ridge Franklin|Way|Suite B|Bunker Hill|Buffalo County|SD|50150|United States|-6|condo| +30219|AAAAAAAALAGHAAAA|647|Cherry Central|Ct.|Suite 340|Roxbury|Sanilac County|MI|45508|United States|-5|single family| +30220|AAAAAAAAMAGHAAAA|746|Locust |Avenue|Suite F|Enterprise|Montmorency County|MI|41757|United States|-5|apartment| +30221|AAAAAAAANAGHAAAA|773|Sunset Center|RD|Suite I|Baker|Delta County|TX|78785|United States|-6|single family| +30222|AAAAAAAAOAGHAAAA|909|East Ash|Lane|Suite E|Wildwood|Horry County|SC|26871|United States|-5|single family| +30223|AAAAAAAAPAGHAAAA|737|Maple Chestnut|Cir.|Suite C|Wildwood|Winston County|MS|56871|United States|-6|apartment| +30224|AAAAAAAAABGHAAAA|123|First |RD|Suite L|Pine Grove|Sullivan County|NH|05193|United States|-5|condo| +30225|AAAAAAAABBGHAAAA|832|Wilson Willow|Boulevard|Suite 230|Red Hill|Perkins County|SD|54338|United States|-7|condo| +30226|AAAAAAAACBGHAAAA|380|10th Maple|Parkway|Suite 200|Woodland|Wasatch County|UT|84854|United States|-7|single family| +30227|AAAAAAAADBGHAAAA|276|8th |Dr.|Suite 150|Sunnyside|Brown County|IL|61952|United States|-6|condo| +30228|AAAAAAAAEBGHAAAA|648|East Seventh|Ln|Suite 280|Stratford|Albany County|NY|16668|United States|-5|apartment| +30229|AAAAAAAAFBGHAAAA|349|Meadow Lincoln|Street|Suite 390|Bunker Hill|Hardin County|TN|30150|United States|-5|condo| +30230|AAAAAAAAGBGHAAAA|129|Cherry Walnut|Court|Suite H|Springfield|Oglethorpe County|GA|39303|United States|-5|single family| +30231|AAAAAAAAHBGHAAAA|478|Jefferson |Way|Suite 440|Pleasant Grove|White County|GA|34136|United States|-5|condo| +30232|AAAAAAAAIBGHAAAA|157|Park |Way|Suite O|Marion|Riverside County|CA|90399|United States|-8|apartment| +30233|AAAAAAAAJBGHAAAA|932|Broadway 4th|Road|Suite 50|Fernwood|Polk County|NC|22291|United States|-5|single family| +30234|AAAAAAAAKBGHAAAA|360|6th Washington|Wy|Suite 150|Summit|Porter County|IN|40499|United States|-5|apartment| +30235|AAAAAAAALBGHAAAA|703|Hill Cedar|Pkwy|Suite D|Lakeview|Hall County|GA|38579|United States|-5|apartment| +30236|AAAAAAAAMBGHAAAA|344|Second |Pkwy|Suite 390|Sheffield|Anoka County|MN|56896|United States|-6|condo| +30237|AAAAAAAANBGHAAAA|290|First |Ct.|Suite 290|Mount Pleasant|Mercer County|ND|51933|United States|-6|single family| +30238|AAAAAAAAOBGHAAAA|728|Williams Elm|RD|Suite E|Greenville|Columbia County|PA|11387|United States|-5|condo| +30239|AAAAAAAAPBGHAAAA|804|3rd Church|Street|Suite N|Springdale|Genesee County|NY|18883|United States|-5|apartment| +30240|AAAAAAAAACGHAAAA|399|Cedar 2nd|Circle|Suite S|Plainview|Muscogee County|GA|33683|United States|-5|apartment| +30241|AAAAAAAABCGHAAAA|653|Walnut Washington|Court|Suite R|Sulphur Springs|Cabell County|WV|28354|United States|-5|single family| +30242|AAAAAAAACCGHAAAA|743|Maple Lake|Wy|Suite 440|Unionville|Winston County|AL|31711|United States|-6|condo| +30243|AAAAAAAADCGHAAAA|464|Forest 12th|Court|Suite L|Oakwood|Logan County|CO|80169|United States|-7|single family| +30244|AAAAAAAAECGHAAAA|505|Spruce Williams|ST|Suite 310|Edgewood|Gadsden County|FL|30069|United States|-5|apartment| +30245|AAAAAAAAFCGHAAAA||Cherry ||||McCurtain County||72293|||| +30246|AAAAAAAAGCGHAAAA|957|Jefferson |ST|Suite M|Union Hill|Cass County|NE|67746|United States|-6|single family| +30247|AAAAAAAAHCGHAAAA|565|Maple Spring|Lane|Suite 220|Farmington|Choctaw County|AL|39145|United States|-6|single family| +30248|AAAAAAAAICGHAAAA|747|Smith |Drive|Suite 170|Littleton|Clark County|WI|56074|United States|-6|condo| +30249|AAAAAAAAJCGHAAAA|129|Main |Lane|Suite 490|Summit|Corson County|SD|50499|United States|-6|single family| +30250|AAAAAAAAKCGHAAAA|787|East |Lane|Suite V|Fairfield|Branch County|MI|46192|United States|-5|single family| +30251|AAAAAAAALCGHAAAA|456|Sixth 7th|Road|Suite V|Union|Davis County|UT|88721|United States|-7|single family| +30252|AAAAAAAAMCGHAAAA|287|Ash |Street|Suite M|Stringtown|Clark County|WA|90162|United States|-8|condo| +30253|AAAAAAAANCGHAAAA|6|South Lake|Wy|Suite J|Union|Effingham County|GA|38721|United States|-5|apartment| +30254|AAAAAAAAOCGHAAAA|847|Ninth Pine|Boulevard|Suite 90|Fairview|Kenosha County|WI|55709|United States|-6|apartment| +30255|AAAAAAAAPCGHAAAA|522|Cedar 6th|Lane|Suite O|Arlington|Titus County|TX|76557|United States|-6|condo| +30256|AAAAAAAAADGHAAAA|268|9th |Circle|Suite D|Red Hill|Mora County|NM|84338|United States|-7|apartment| +30257|AAAAAAAABDGHAAAA|59|Johnson Park|Parkway|Suite Y|Buena Vista|Macon County|AL|35752|United States|-6|single family| +30258|AAAAAAAACDGHAAAA|98|Johnson |Ct.|Suite V|Clinton|Kay County|OK|78222|United States|-6|apartment| +30259|AAAAAAAADDGHAAAA|292|Mill |Ct.|Suite 80|Providence|Grafton County|NH|07214|United States|-5|apartment| +30260|AAAAAAAAEDGHAAAA|395|Laurel |ST|Suite 400|Salem|Ottawa County|KS|68048|United States|-6|apartment| +30261|AAAAAAAAFDGHAAAA|335|Cedar Lincoln|ST|Suite 380|Bath|Potter County|SD|50573|United States|-7|apartment| +30262|AAAAAAAAGDGHAAAA|930|Johnson |||New Hope||AR||United States|-6|single family| +30263|AAAAAAAAHDGHAAAA|170|Davis |ST|Suite G|Plainview|Wayne County|GA|33683|United States|-5|single family| +30264|AAAAAAAAIDGHAAAA|39|Jackson |Boulevard|Suite 60|Garrison|Gove County|KS|68767|United States|-6|apartment| +30265|AAAAAAAAJDGHAAAA|168|Willow Mill|Pkwy|Suite 140|Hamilton|Lapeer County|MI|42808|United States|-5|single family| +30266|AAAAAAAAKDGHAAAA|275|Valley 11th|Pkwy|Suite 230|Midway|Iron County|MO|61904|United States|-6|condo| +30267|AAAAAAAALDGHAAAA|770|Park Main|Cir.|Suite 460|Green Acres|Wilson County|TN|37683|United States|-5|apartment| +30268|AAAAAAAAMDGHAAAA|910|South Green|Lane|Suite L|Lincoln|Caldwell County|TX|71289|United States|-6|condo| +30269|AAAAAAAANDGHAAAA|997|South Pine|Way|Suite 100|Pleasant Valley|Madison County|VA|22477|United States|-5|condo| +30270|AAAAAAAAODGHAAAA|286|Green |ST|Suite 430|Mount Olive|Lee County|SC|28059|United States|-5|apartment| +30271|AAAAAAAAPDGHAAAA|724|14th 11th|Circle|Suite 340|Buena Vista|Montour County|PA|15752|United States|-5|apartment| +30272|AAAAAAAAAEGHAAAA|599|Woodland 6th|Court|Suite N|Salem|DeKalb County|AL|38048|United States|-6|condo| +30273|AAAAAAAABEGHAAAA|495|Hickory Park|Avenue|Suite T|Jackson|Lamoille County|VT|09583|United States|-5|single family| +30274|AAAAAAAACEGHAAAA|680|Highland |ST|Suite N|Highland|Kidder County|ND|59454|United States|-6|condo| +30275|AAAAAAAADEGHAAAA|192|3rd |Lane|Suite F|Crossroads|Boone County|NE|60534|United States|-6|condo| +30276|AAAAAAAAEEGHAAAA|601|Fourth Washington|Way|Suite 440|Glendale|Cherokee County|GA|33951|United States|-5|apartment| +30277|AAAAAAAAFEGHAAAA|287|6th |Road|Suite 310|Mountain View|Knox County|NE|64466|United States|-7|condo| +30278|AAAAAAAAGEGHAAAA|405|Madison |Ct.|Suite 280|Highland Park|Sussex County|VA|26534|United States|-5|condo| +30279|AAAAAAAAHEGHAAAA|129|Highland Pine|Lane|Suite 380|Greenfield|Crawford County|KS|65038|United States|-6|condo| +30280|AAAAAAAAIEGHAAAA|760|Center Mill|Circle|Suite K|Springfield|Washington County|NC|29303|United States|-5|single family| +30281|AAAAAAAAJEGHAAAA|566|Pine |RD|Suite 20|Greenfield|Koochiching County|MN|55038|United States|-6|single family| +30282|AAAAAAAAKEGHAAAA|834|2nd |Cir.|Suite J|Westgate|Vance County|NC|22366|United States|-5|condo| +30283|AAAAAAAALEGHAAAA|487|Lee |Ln|Suite 120|Union Hill|Clark County|KS|67746|United States|-6|apartment| +30284|AAAAAAAAMEGHAAAA|485|14th |Boulevard|Suite 440|Waterloo|Portage County|OH|41675|United States|-5|condo| +30285|AAAAAAAANEGHAAAA|152|Ridge |Ct.|Suite C|Oak Hill|Jackson County|TN|37838|United States|-5|single family| +30286|AAAAAAAAOEGHAAAA|618|Lee |Road|Suite G|Oak Ridge|Deuel County|NE|68371|United States|-6|apartment| +30287|AAAAAAAAPEGHAAAA|394|Hill |Ln|Suite 180|Spring Hill|Greene County|AL|36787|United States|-6|apartment| +30288|AAAAAAAAAFGHAAAA|68|Hickory Jackson|Ln|Suite W|Highland Park|Stearns County|MN|56534|United States|-6|condo| +30289|AAAAAAAABFGHAAAA|117|Madison |Wy|Suite E|Woodville|Gloucester County|VA|24289|United States|-5|condo| +30290|AAAAAAAACFGHAAAA|42|Park |Court|Suite 80|Harmony|Kershaw County|SC|25804|United States|-5|single family| +30291|AAAAAAAADFGHAAAA|32|6th |Lane|Suite 310|Oakwood|Fulton County|KY|40169|United States|-6|apartment| +30292|AAAAAAAAEFGHAAAA|592|East 4th|RD|Suite P|Greenwood|Clay County|KS|68828|United States|-6|apartment| +30293|AAAAAAAAFFGHAAAA|||Circle|Suite J||Androscoggin County||04051|||single family| +30294|AAAAAAAAGFGHAAAA|948|Tenth 7th|Pkwy|Suite 250|Oak Hill|Wasco County|OR|97838|United States|-8|single family| +30295|AAAAAAAAHFGHAAAA|433|Maple Second|Wy|Suite 330|Highland Park|Jones County|MS|56534|United States|-6|condo| +30296|AAAAAAAAIFGHAAAA|915|Lake |Cir.|Suite W|Concord|Johnson County|WY|84107|United States|-7|condo| +30297|AAAAAAAAJFGHAAAA|991|Main |Road|Suite V|Ashland|Lebanon County|PA|14244|United States|-5|single family| +30298|AAAAAAAAKFGHAAAA|879|Woodland 15th|Wy|Suite 390|Pleasant Valley|Grant County|WI|52477|United States|-6|single family| +30299|AAAAAAAALFGHAAAA|192|Smith Spruce|Blvd|Suite 230|Leesville|Tallahatchie County|MS|55423|United States|-6|condo| +30300|AAAAAAAAMFGHAAAA|990|Main 4th|Court|Suite 130|Lewis|Stearns County|MN|57066|United States|-6|apartment| +30301|AAAAAAAANFGHAAAA|764|East Washington|Lane|Suite E|Providence|Cleburne County|AL|36614|United States|-6|condo| +30302|AAAAAAAAOFGHAAAA|774|Central Broadway|||White Oak||IL||United States||| +30303|AAAAAAAAPFGHAAAA|690|3rd |Wy|Suite 100|Plainview|Sarpy County|NE|63683|United States|-7|single family| +30304|AAAAAAAAAGGHAAAA|750|Washington Washington|Avenue|Suite 330|Union Hill|Tuscaloosa County|AL|37746|United States|-6|condo| +30305|AAAAAAAABGGHAAAA|237|Washington |ST|Suite 380|Rutland|Ballard County|KY|48375|United States|-6|condo| +30306|AAAAAAAACGGHAAAA|824|Valley Hillcrest|Blvd|Suite 170|Clifton|Reagan County|TX|78014|United States|-6|condo| +30307|AAAAAAAADGGHAAAA|211|Oak Pine|Wy|Suite 70|Woodland|Polk County|NE|64854|United States|-7|condo| +30308|AAAAAAAAEGGHAAAA|153|Mill Lincoln|Blvd|Suite M|Greendale|Glasscock County|TX|79343|United States|-6|condo| +30309|AAAAAAAAFGGHAAAA|89|3rd Sixth|Parkway|Suite 440|Shiloh|Hudson County|NJ|09875|United States|-5|single family| +30310|AAAAAAAAGGGHAAAA|176|Locust Spring|Dr.|Suite L|Riverview|Loup County|NE|69003|United States|-7|condo| +30311|AAAAAAAAHGGHAAAA|32|Elm Highland|Wy|Suite S|Mount Olive|Rio Grande County|CO|88059|United States|-7|apartment| +30312|AAAAAAAAIGGHAAAA|314|Central |Pkwy|Suite 290|Georgetown|Decatur County|KS|67057|United States|-6|apartment| +30313|AAAAAAAAJGGHAAAA|990|Ridge West|Boulevard|Suite 440|Red Hill|Ida County|IA|54338|United States|-6|single family| +30314|AAAAAAAAKGGHAAAA|205|First Main|Drive|Suite I|Greenville|Winnebago County|WI|51387|United States|-6|apartment| +30315|AAAAAAAALGGHAAAA|||||||NC|22033|||single family| +30316|AAAAAAAAMGGHAAAA|664|Valley Forest|Ln|Suite Y|Newtown|Colbert County|AL|31749|United States|-6|condo| +30317|AAAAAAAANGGHAAAA|344|Meadow Second|Parkway|Suite 470|Friendship|Crawford County|AR|74536|United States|-6|single family| +30318|AAAAAAAAOGGHAAAA|294|Green |Ave|Suite L|Mount Zion|Centre County|PA|18054|United States|-5|condo| +30319|AAAAAAAAPGGHAAAA||||Suite D|||MT|68877|United States|-7|apartment| +30320|AAAAAAAAAHGHAAAA|919|First |Ln|Suite T|Greenville|Norfolk city|VA|21387|United States|-5|apartment| +30321|AAAAAAAABHGHAAAA|875|Green 3rd|Blvd|Suite 250|Oak Ridge|White County|IN|48371|United States|-5|condo| +30322|AAAAAAAACHGHAAAA|307|West Spruce|Ave|Suite 400|Belmont|Sumner County|KS|60191|United States|-6|condo| +30323|AAAAAAAADHGHAAAA|57|8th Dogwood|Ct.|Suite X|Antioch|Whitman County|WA|98605|United States|-8|apartment| +30324|AAAAAAAAEHGHAAAA|605|South South|Parkway|Suite R|Riverside|Barbour County|AL|39231|United States|-6|apartment| +30325|AAAAAAAAFHGHAAAA|331|Park 11th|Road|Suite C|Marion|Morehouse Parish|LA|70399|United States|-6|condo| +30326|AAAAAAAAGHGHAAAA|222|Main |Street|Suite 260|Newtown|Louisa County|IA|51749|United States|-6|condo| +30327|AAAAAAAAHHGHAAAA|117|Sunset 6th|Avenue|Suite 410|Unionville|Hyde County|SD|51711|United States|-7|condo| +30328|AAAAAAAAIHGHAAAA|959|15th Green|Dr.|Suite 20|Hamilton|Tyrrell County|NC|22808|United States|-5|single family| +30329|AAAAAAAAJHGHAAAA|685|Second |Court|Suite G|Fairview|Clark County|MO|65709|United States|-6|apartment| +30330|AAAAAAAAKHGHAAAA|142|Cedar Cherry|Lane|Suite 390|Sulphur Springs|Middlesex County|CT|08954|United States|-5|condo| +30331|AAAAAAAALHGHAAAA|190|Railroad |Road|Suite Q|Lakeside|Rio Blanco County|CO|89532|United States|-7|condo| +30332|AAAAAAAAMHGHAAAA|561|14th |Ln|Suite 200|Oak Hill|Jefferson County|GA|37838|United States|-5|condo| +30333|AAAAAAAANHGHAAAA|||Street|Suite 110||Randolph County|GA||United States||single family| +30334|AAAAAAAAOHGHAAAA|584|8th Ridge|Road|Suite P|Florence|Redwood County|MN|53394|United States|-6|condo| +30335|AAAAAAAAPHGHAAAA|240|Madison |Wy|Suite 50|Unionville|Coosa County|AL|31711|United States|-6|condo| +30336|AAAAAAAAAIGHAAAA|117|4th |Boulevard|Suite 40|Wildwood|Carbon County|UT|86871|United States|-7|condo| +30337|AAAAAAAABIGHAAAA|430|Maple |Ct.|Suite 90|Woodland|Howell County|MO|64854|United States|-6|apartment| +30338|AAAAAAAACIGHAAAA|469|Elm |Pkwy|Suite 290|Hubbard|Lewis County|WA|96291|United States|-8|condo| +30339|AAAAAAAADIGHAAAA|411|10th |Parkway|Suite 70|Waterloo|Whitley County|IN|41675|United States|-5|apartment| +30340|AAAAAAAAEIGHAAAA||||Suite X|Mount Olive|Raleigh County|WV||||single family| +30341|AAAAAAAAFIGHAAAA|738|Hillcrest Main|Boulevard|Suite 110|Lebanon|Wabaunsee County|KS|62898|United States|-6|apartment| +30342|AAAAAAAAGIGHAAAA|332|Sunset |Drive|Suite I|Mount Zion|Sharkey County|MS|58054|United States|-6|condo| +30343|AAAAAAAAHIGHAAAA|614|Johnson |ST|Suite L|Pinecrest|Richland County|SC|29981|United States|-5|single family| +30344|AAAAAAAAIIGHAAAA|608|Ash Third|Ct.|Suite 60|Oakland|Starke County|IN|49843|United States|-5|single family| +30345|AAAAAAAAJIGHAAAA|335|1st |Ave|Suite 280|Murray|Eddy County|NM|82150|United States|-7|apartment| +30346|AAAAAAAAKIGHAAAA|947|West Fourth|Pkwy|Suite X|Jamestown|Prairie County|MT|66867|United States|-7|apartment| +30347|AAAAAAAALIGHAAAA|148|Smith |Circle|Suite 100|Deerfield|Bannock County|ID|89840|United States|-7|single family| +30348|AAAAAAAAMIGHAAAA|745|Sunset |RD|Suite O|Union Hill|Spotsylvania County|VA|27746|United States|-5|single family| +30349|AAAAAAAANIGHAAAA|603|Oak |Street|Suite E|Greenfield|Madison County|AR|75038|United States|-6|single family| +30350|AAAAAAAAOIGHAAAA|760|Fifth Eigth|Ave|Suite S|Belmont|Madison County|KY|40191|United States|-5|condo| +30351|AAAAAAAAPIGHAAAA|148|Jackson |Way|Suite J|Bridgeport|Webb County|TX|75817|United States|-6|condo| +30352|AAAAAAAAAJGHAAAA|594|10th |Boulevard|Suite T|Rolling Hills|Dodge County|NE|67272|United States|-6|single family| +30353|AAAAAAAABJGHAAAA|978|Maple Walnut|Boulevard|Suite W|Riverview|Warren County|GA|39003|United States|-5|apartment| +30354|AAAAAAAACJGHAAAA|428|Walnut |Avenue|Suite R|Deerfield|Sonoma County|CA|99840|United States|-8|single family| +30355|AAAAAAAADJGHAAAA|440|Dogwood Hillcrest|Street|Suite A|Mount Zion|Marion County|WV|28054|United States|-5|apartment| +30356|AAAAAAAAEJGHAAAA||||Suite M|Oak Hill|Coal County|OK|77838|||| +30357|AAAAAAAAFJGHAAAA|185|Wilson |Road|Suite 70|Highland Park|Stoddard County|MO|66534|United States|-6|condo| +30358|AAAAAAAAGJGHAAAA|250|3rd Spruce|Blvd|Suite 230|Newport|Wheatland County|MT|61521|United States|-7|apartment| +30359|AAAAAAAAHJGHAAAA|782|Valley |Ln|Suite P|Newport|Codington County|SD|51521|United States|-6|apartment| +30360|AAAAAAAAIJGHAAAA|444|Wilson Oak|Way|Suite K|Antioch|Harper County|OK|78605|United States|-6|single family| +30361|AAAAAAAAJJGHAAAA|135|Hillcrest |Way|Suite 30|Farmington|Barton County|MO|69145|United States|-6|apartment| +30362|AAAAAAAAKJGHAAAA|203|Forest |Drive|Suite W|Marion|Blackford County|IN|40399|United States|-5|apartment| +30363|AAAAAAAALJGHAAAA|425|Locust |Ln|Suite 390|Brownsville|Adair County|KY|49310|United States|-6|apartment| +30364|AAAAAAAAMJGHAAAA|135|Center |Drive|Suite 390|Plainview|Montgomery County|AL|33683|United States|-6|apartment| +30365|AAAAAAAANJGHAAAA|625|2nd Forest|Street|Suite 210|Red Hill|Pope County|IL|64338|United States|-6|single family| +30366|AAAAAAAAOJGHAAAA|615|Oak North|Ct.|Suite 270|Tracy|Yates County|NY|16340|United States|-5|apartment| +30367|AAAAAAAAPJGHAAAA|670|Adams |Ave|Suite 310|White Oak|Montgomery County|OH|46668|United States|-5|single family| +30368|AAAAAAAAAKGHAAAA|818|Forest |Street|Suite A|Bunker Hill|Atchison County|MO|60150|United States|-6|apartment| +30369|AAAAAAAABKGHAAAA|819|Highland |Ct.|Suite 450|Friendship|Pulaski County|VA|24536|United States|-5|single family| +30370|AAAAAAAACKGHAAAA|232|Forest 15th|Court|Suite U|Mount Vernon|Cumberland County|ME|09082|United States|-5|condo| +30371|AAAAAAAADKGHAAAA|484|Birch Lakeview|Wy|Suite E|Stewart|Switzerland County||48041|||| +30372|AAAAAAAAEKGHAAAA|374|8th |ST|Suite O|Green Acres|Lincoln County|NV|87683|United States|-8|condo| +30373|AAAAAAAAFKGHAAAA|870|Madison Birch|Circle|Suite L|Spring Valley|Roberts County|SD|56060|United States|-7|condo| +30374|AAAAAAAAGKGHAAAA|969|Ridge |Dr.|Suite 410|Carthage|Bristol city|VA|21529|United States|-5|apartment| +30375|AAAAAAAAHKGHAAAA|875|Forest |Cir.|Suite 170|Marion|Sullivan County|NY|10399|United States|-5|apartment| +30376|AAAAAAAAIKGHAAAA|439|View Ninth|Blvd|Suite 140|Lakeside|Newton County|IN|49532|United States|-5|single family| +30377|AAAAAAAAJKGHAAAA|345|Adams |RD|Suite 350|Mount Olive|Fresno County|CA|98059|United States|-8|apartment| +30378|AAAAAAAAKKGHAAAA|595|3rd Chestnut|Ln|Suite 300|Woodlawn|Carbon County|MT|64098|United States|-7|condo| +30379|AAAAAAAALKGHAAAA|211|Lee |Cir.|Suite P|Pomona|Glacier County|MT|64153|United States|-7|single family| +30380|AAAAAAAAMKGHAAAA||Seventh |Court|Suite R||Griggs County|||United States||| +30381|AAAAAAAANKGHAAAA|843|3rd |Avenue|Suite 480|Jackson|Fannin County|TX|79583|United States|-6|single family| +30382|AAAAAAAAOKGHAAAA|902|4th |Blvd|Suite M|Antioch|Madison County|TX|78605|United States|-6|single family| +30383|AAAAAAAAPKGHAAAA|532|Ash |Pkwy|Suite Y|Enterprise|Wayne County|NE|61757|United States|-7|condo| +30384|AAAAAAAAALGHAAAA|513|Johnson Maple|Court|Suite 240|Newport|Taylor County|FL|31521|United States|-5|apartment| +30385|AAAAAAAABLGHAAAA|877|Second Jackson|Blvd|Suite 310|Oak Hill|Norman County|MN|57838|United States|-6|condo| +30386|AAAAAAAACLGHAAAA|633|North Jefferson|Drive|Suite 190|Maple Grove|Choctaw County|OK|78252|United States|-6|condo| +30387|AAAAAAAADLGHAAAA|409|Locust Laurel|Wy|Suite P|Kingston|Adams County|IN|44975|United States|-5|apartment| +30388|AAAAAAAAELGHAAAA|188|Center |Cir.|Suite N|Greenwood|DeKalb County|AL|38828|United States|-6|apartment| +30389|AAAAAAAAFLGHAAAA|538|Elm Green|Ln|Suite 180|Fairfield|Sussex County|VA|26192|United States|-5|single family| +30390|AAAAAAAAGLGHAAAA|879|Cherry Lake|Court|Suite 180|Bethel|Wythe County|VA|25281|United States|-5|condo| +30391|AAAAAAAAHLGHAAAA|405|Lincoln |Drive|Suite M|Hubbard|Cameron Parish|LA|76291|United States|-6|single family| +30392|AAAAAAAAILGHAAAA|642|Valley |Avenue|Suite 60|Fairview|Winn Parish|LA|75709|United States|-6|single family| +30393|AAAAAAAAJLGHAAAA|121|11th 3rd|Way|Suite G|Mechanicsburg|Nelson County|ND|52219|United States|-6|single family| +30394|AAAAAAAAKLGHAAAA|650|Poplar View|Ln|Suite S|Mountain View|Kenedy County|TX|74466|United States|-6|apartment| +30395|AAAAAAAALLGHAAAA|375|Jackson Spring|Way|Suite 10|Lakewood|Stillwater County|MT|68877|United States|-7|condo| +30396|AAAAAAAAMLGHAAAA|650||Avenue|Suite 400||Owen County|KY|40059|United States||| +30397|AAAAAAAANLGHAAAA|398|Pine 2nd|Wy|Suite 420|Hillcrest|Assumption Parish|LA|73003|United States|-6|condo| +30398|AAAAAAAAOLGHAAAA|85|Railroad River|ST|Suite 350|New Hope|Yankton County|SD|59431|United States|-6|condo| +30399|AAAAAAAAPLGHAAAA|665|15th Maple|Avenue|Suite Y|Camden|McDuffie County|GA|37119|United States|-5|single family| +30400|AAAAAAAAAMGHAAAA|373|4th Pine|Parkway|Suite G|Greenwood|Price County|WI|58828|United States|-6|apartment| +30401|AAAAAAAABMGHAAAA|530|East |Drive|Suite 410|Hopewell|Grant County|NM|80587|United States|-7|apartment| +30402|AAAAAAAACMGHAAAA|629|College Wilson|Drive|Suite D|Winslow|Mecklenburg County|VA|28525|United States|-5|apartment| +30403|AAAAAAAADMGHAAAA|979|Hickory 6th|Blvd|Suite D|Newtown|Allegany County|NY|11749|United States|-5|condo| +30404|AAAAAAAAEMGHAAAA|849|Highland Sixth|Blvd|||Silver Bow County|MT|60169|United States|-7|apartment| +30405|AAAAAAAAFMGHAAAA|982|Elm Valley|Ct.|Suite 260|Plainview|Hardin County|KY|43683|United States|-6|condo| +30406|AAAAAAAAGMGHAAAA|70|Maple |Way|Suite 240|Oak Ridge|Carteret County|NC|28371|United States|-5|condo| +30407|AAAAAAAAHMGHAAAA|964|Lincoln Third|Circle|Suite C|Antioch|Cheshire County|NH|09205|United States|-5|condo| +30408|AAAAAAAAIMGHAAAA|784|Smith |Parkway|Suite C|Oak Hill|Lapeer County|MI|47838|United States|-5|apartment| +30409|AAAAAAAAJMGHAAAA|923|Main |Avenue|Suite K|Arlington|Woodford County|IL|66557|United States|-6|apartment| +30410|AAAAAAAAKMGHAAAA|779||||Spring Hill||IL|66787||-6|| +30411|AAAAAAAALMGHAAAA|830|Woodland Walnut|Ct.|Suite 210|Glendale|Richland County|ND|53951|United States|-6|single family| +30412|AAAAAAAAMMGHAAAA|921|Davis Cedar|Blvd|Suite W|New Hope|Burt County|NE|69431|United States|-6|apartment| +30413|AAAAAAAANMGHAAAA|56|6th 8th|Ln|Suite K|Oakwood|Essex County|NJ|00769|United States|-5|apartment| +30414|AAAAAAAAOMGHAAAA|29|14th Meadow|Blvd|Suite K|Oakland|Casey County|KY|49843|United States|-6|condo| +30415|AAAAAAAAPMGHAAAA|450|Williams |Blvd|Suite 80|Crossroads|McMullen County|TX|70534|United States|-6|apartment| +30416|AAAAAAAAANGHAAAA|127|4th |Wy|Suite 380|Summit|Lewis County|WV|20499|United States|-5|apartment| +30417|AAAAAAAABNGHAAAA|265|1st Broadway|Circle|Suite K|Walnut Grove|Athens County|OH|47752|United States|-5|single family| +30418|AAAAAAAACNGHAAAA|238|Ninth Railroad|RD|Suite 10|Hamilton|Delta County|MI|42808|United States|-5|condo| +30419|AAAAAAAADNGHAAAA|232|Meadow Green|Dr.|Suite 450|Greenfield|Shannon County|MO|65038|United States|-6|single family| +30420|AAAAAAAAENGHAAAA|472|Williams Spring|Lane|Suite V|Marion|Walker County|AL|30399|United States|-6|apartment| +30421|AAAAAAAAFNGHAAAA|617|View Woodland|Ct.|Suite S|Valley View|McCook County|SD|55124|United States|-7|single family| +30422|AAAAAAAAGNGHAAAA|451|South |Road|Suite 310|Summit|Stutsman County|ND|50499|United States|-6|condo| +30423|AAAAAAAAHNGHAAAA|8|Center South|Ct.|Suite V|Kingston|Elk County|KS|64975|United States|-6|condo| +30424|AAAAAAAAINGHAAAA|758|Center |Ave|Suite 490|Clinton|McDowell County|NC|28222|United States|-5|apartment| +30425|AAAAAAAAJNGHAAAA|160|8th Franklin|Blvd|Suite H|Burns|Lincoln County|WA|95272|United States|-8|single family| +30426|AAAAAAAAKNGHAAAA|418|Sycamore |Drive|Suite H|Cedar Grove|Muskegon County|MI|40411|United States|-5|condo| +30427|AAAAAAAALNGHAAAA|198|Fourth |Boulevard|Suite B|Sulphur Springs|Richland County|WI|58354|United States|-6|single family| +30428|AAAAAAAAMNGHAAAA|276|1st Walnut|Wy|Suite W|Pine Grove|Barron County|WI|54593|United States|-6|single family| +30429|AAAAAAAANNGHAAAA|97|Cedar 2nd|Ln|Suite 170|Mountain View|Madera County|CA|94466|United States|-8|condo| +30430|AAAAAAAAONGHAAAA|285|Center |Lane|Suite Y|Riverdale|Delaware County|NY|19391|United States|-5|condo| +30431|AAAAAAAAPNGHAAAA|310|Locust Spruce|Wy|Suite 200|Cedar Grove|Teton County|ID|80411|United States|-7|single family| +30432|AAAAAAAAAOGHAAAA|78|North |Lane|Suite X|Vienna|Red River County|TX|75119|United States|-6|apartment| +30433|AAAAAAAABOGHAAAA|854|Adams Washington|Pkwy|Suite 0|Yorktown|Guernsey County|OH|40732|United States|-5|single family| +30434|AAAAAAAACOGHAAAA|819|14th |Ave|Suite W|Woodland|Rowan County|NC|24854|United States|-5|condo| +30435|AAAAAAAADOGHAAAA|666|Jackson |Pkwy|Suite X|Macedonia|Osborne County|KS|61087|United States|-6|single family| +30436|AAAAAAAAEOGHAAAA|855|13th West|Street|Suite 220|Fairview|Wilkinson County|GA|35709|United States|-5|single family| +30437|AAAAAAAAFOGHAAAA|677|Ridge |Wy|Suite 210|Glendale|Wolfe County|KY|43951|United States|-5|apartment| +30438|AAAAAAAAGOGHAAAA|716|||Suite M|Forest Hills|Morgan County|CO|||-7|| +30439|AAAAAAAAHOGHAAAA|584|Dogwood Oak|Parkway|Suite T|Crossroads|Coleman County|TX|70534|United States|-6|single family| +30440|AAAAAAAAIOGHAAAA|705|Poplar Walnut|Street|Suite 60|Ashland|Summit County|CO|84244|United States|-7|condo| +30441|AAAAAAAAJOGHAAAA|833|Willow |Ln|Suite R|Forest Hills|San Miguel County|NM|89237|United States|-7|single family| +30442|AAAAAAAAKOGHAAAA|432|Highland |Ave|Suite G|Glendale|Pearl River County|MS|53951|United States|-6|condo| +30443|AAAAAAAALOGHAAAA|101|Laurel Cedar|Ave|Suite 350|Salem|Finney County|KS|68048|United States|-6|apartment| +30444|AAAAAAAAMOGHAAAA|656|View Smith|Court|Suite N|Marion|Gibson County|IN|40399|United States|-5|single family| +30445|AAAAAAAANOGHAAAA|482|Davis |ST|Suite C|Spring Grove|Howard County|AR|76719|United States|-6|single family| +30446|AAAAAAAAOOGHAAAA|260|6th |Ln|Suite 120|Liberty|Yellow Medicine County|MN|53451|United States|-6|apartment| +30447|AAAAAAAAPOGHAAAA|659|Hickory |Pkwy|Suite 470|Mount Pleasant|Saline County|NE|61933|United States|-7|single family| +30448|AAAAAAAAAPGHAAAA|780|Maple Davis|Boulevard|Suite J|Oakdale|Bulloch County|GA|39584|United States|-5|single family| +30449|AAAAAAAABPGHAAAA|928|Pine |ST|Suite V|Harmony|Macoupin County|IL|65804|United States|-6|apartment| +30450|AAAAAAAACPGHAAAA|43|Park |RD|Suite 420|Lakeview|Huron County|OH|48579|United States|-5|condo| +30451|AAAAAAAADPGHAAAA|792|Cherry |Wy|Suite 210|Highland|Quay County|NM|89454|United States|-7|single family| +30452|AAAAAAAAEPGHAAAA|129|Main Lee|Road|Suite 220|Glendale|Cass County|IN|43951|United States|-5|apartment| +30453|AAAAAAAAFPGHAAAA|976|Valley |Ln|Suite B|Maple Grove|Bollinger County|MO|68252|United States|-6|apartment| +30454|AAAAAAAAGPGHAAAA|701|Central Lake|Lane|Suite G|Edgewood|Danville city|VA|20069|United States|-5|single family| +30455|AAAAAAAAHPGHAAAA|739|Jefferson Hickory|Parkway|Suite 70|Riverside|Iberville Parish|LA|79231|United States|-6|condo| +30456|AAAAAAAAIPGHAAAA|760|Davis |Way|Suite Y|Maple Hill|Douglas County|IL|68095|United States|-6|apartment| +30457|AAAAAAAAJPGHAAAA|522|Washington |Way|Suite 430|Pleasant Valley|Pope County|AR|72477|United States|-6|single family| +30458|AAAAAAAAKPGHAAAA|941|Seventh |Drive|Suite J|Red Hill|Polk County|WI|54338|United States|-6|apartment| +30459|AAAAAAAALPGHAAAA|917|Oak |Way|Suite 90|Bunker Hill|Monona County|IA|50150|United States|-6|single family| +30460|AAAAAAAAMPGHAAAA|11|7th |Dr.|Suite S|Mount Zion|Cumberland County|ME|08654|United States|-5|condo| +30461|AAAAAAAANPGHAAAA|792|Jackson Johnson|Avenue|Suite 110|Parkwood|Marathon County|WI|51669|United States|-6|condo| +30462|AAAAAAAAOPGHAAAA|694|5th 3rd|Way|Suite 270|Fairfield|Clearwater County|MN|56192|United States|-6|apartment| +30463|AAAAAAAAPPGHAAAA|772|3rd |Street|Suite 190|Oak Grove|Johnston County|NC|28370|United States|-5|single family| +30464|AAAAAAAAAAHHAAAA|102|Mill Meadow|Pkwy|Suite O|Snug Harbor|Dickenson County|VA|27936|United States|-5|apartment| +30465|AAAAAAAABAHHAAAA|843|Maple |Cir.|Suite R|Greenville|Jackson County|KY|41387|United States|-6|condo| +30466|AAAAAAAACAHHAAAA|986|Hickory |Drive|Suite B|Fox|Greenville County|SC|20631|United States|-5|single family| +30467|AAAAAAAADAHHAAAA|598|Park Highland|Circle|Suite Y|Red Hill|Costilla County|CO|84338|United States|-7|single family| +30468|AAAAAAAAEAHHAAAA|354|Sycamore Jackson|Road|Suite 290|Pleasant Grove|Williamson County|TN|34136|United States|-5|apartment| +30469|AAAAAAAAFAHHAAAA|443|5th |Cir.|Suite 140|Union|Shelby County|IA|58721|United States|-6|apartment| +30470|AAAAAAAAGAHHAAAA|89|Cedar Lake|Avenue|Suite K|Spring Valley|Nevada County|CA|96060|United States|-8|single family| +30471|AAAAAAAAHAHHAAAA|415|12th |Cir.|Suite N|Union|McClain County|OK|78721|United States|-6|condo| +30472|AAAAAAAAIAHHAAAA|946|Hill |Parkway|Suite 360|Mount Pleasant|Monroe County|OH|41933|United States|-5|single family| +30473|AAAAAAAAJAHHAAAA|551|Maple Meadow|Parkway|Suite 310|Mount Olive|Marshall County|MN|58059|United States|-6|condo| +30474|AAAAAAAAKAHHAAAA|948|Chestnut Meadow|Blvd|Suite 50|Bunker Hill|Dixon County|NE|60150|United States|-6|single family| +30475|AAAAAAAALAHHAAAA|941|5th 6th|RD|Suite T|Five Points|Bremer County|IA|56098|United States|-6|single family| +30476|AAAAAAAAMAHHAAAA|582|Smith South|Ave|Suite V|Newport|Clermont County|OH|41521|United States|-5|single family| +30477|AAAAAAAANAHHAAAA|714|Sycamore Cedar|Street|Suite 270|Mount Pleasant|Twiggs County|GA|31933|United States|-5|condo| +30478|AAAAAAAAOAHHAAAA|872|2nd Madison|Ln|Suite L|Belmont|Dinwiddie County|VA|20191|United States|-5|apartment| +30479|AAAAAAAAPAHHAAAA|126|Center |Court|Suite R|Hamilton|Anchorage Borough|AK|92808|United States|-9|single family| +30480|AAAAAAAAABHHAAAA|587|Cherry 6th|Ln|Suite 230|Green Acres|Seward County|NE|67683|United States|-7|condo| +30481|AAAAAAAABBHHAAAA|845|7th |Drive|Suite C|Plainview|Kane County|IL|63683|United States|-6|apartment| +30482|AAAAAAAACBHHAAAA|182|South Sycamore|Parkway|Suite 470|Highland|Fayette County|IA|59454|United States|-6|apartment| +30483|AAAAAAAADBHHAAAA|543|Birch Sycamore|Ln|Suite 390|Concord|Macon County|MO|64107|United States|-6|apartment| +30484|AAAAAAAAEBHHAAAA|853|Wilson Washington|Cir.|Suite J|Highland|Limestone County|AL|39454|United States|-6|condo| +30485|AAAAAAAAFBHHAAAA|850|Chestnut Mill|ST|Suite 270|Concord|Cumberland County|KY|44107|United States|-6|apartment| +30486|AAAAAAAAGBHHAAAA|582|Lee Elm|Avenue|Suite G|Plainview|Amherst County|VA|23683|United States|-5|single family| +30487|AAAAAAAAHBHHAAAA|26|10th 3rd|Parkway|Suite M|Oak Hill|Harney County|OR|97838|United States|-8|apartment| +30488|AAAAAAAAIBHHAAAA|627|Church Washington|Drive|Suite N|Sulphur Springs|Rosebud County|MT|68354|United States|-7|single family| +30489|AAAAAAAAJBHHAAAA|921|Main Park|RD|Suite S|Union Hill|Woodford County|IL|67746|United States|-6|apartment| +30490|AAAAAAAAKBHHAAAA|691|2nd |Way|Suite A|Galena|Lake County|OR|94369|United States|-8|apartment| +30491|AAAAAAAALBHHAAAA|395|Smith |RD|Suite B|Spring Hill|Hancock County|IL|66787|United States|-6|apartment| +30492|AAAAAAAAMBHHAAAA|10|Green Lee|Street|Suite E|Enterprise|Winona County|MN|51757|United States|-6|condo| +30493|AAAAAAAANBHHAAAA|681|5th |Parkway|Suite 120|Georgetown|Wabash County|IL|67057|United States|-6|apartment| +30494|AAAAAAAAOBHHAAAA|151|7th |Circle|Suite E|Union Hill|Blount County|AL|37746|United States|-6|single family| +30495|AAAAAAAAPBHHAAAA|506|Lee |ST|Suite 200|Lakeside|Yellowstone County|MT|69532|United States|-7|condo| +30496|AAAAAAAAACHHAAAA|182|4th |Pkwy|Suite 230|Valley View|Coweta County|GA|35124|United States|-5|apartment| +30497|AAAAAAAABCHHAAAA|505|||||Huron County||48014|||condo| +30498|AAAAAAAACCHHAAAA|872|Maple |Boulevard|Suite B|Greenfield|Clay County|GA|35038|United States|-5|apartment| +30499|AAAAAAAADCHHAAAA|24|4th Cedar|Parkway|Suite 470|Sunnyside|Gladwin County|MI|41952|United States|-5|apartment| +30500|AAAAAAAAECHHAAAA|200|6th |Blvd|Suite X|Five Points|Baker County|OR|96098|United States|-8|condo| +30501|AAAAAAAAFCHHAAAA|973|Sixth |Circle|Suite 260|Woodland|Gooding County|ID|84854|United States|-7|condo| +30502|AAAAAAAAGCHHAAAA|771|Fifth |Lane|Suite 190|Deerfield|Defiance County|OH|49840|United States|-5|single family| +30503|AAAAAAAAHCHHAAAA|716|2nd |Street|Suite J|Woodlawn|Asotin County|WA|94098|United States|-8|apartment| +30504|AAAAAAAAICHHAAAA|51|Sixth Pine|Pkwy|Suite A|Newport|Warren County|KY|41521|United States|-5|single family| +30505|AAAAAAAAJCHHAAAA|704|Smith 1st|Circle|Suite W|Woodland|Ouachita County|AR|74854|United States|-6|apartment| +30506|AAAAAAAAKCHHAAAA|825|Jefferson Williams|Circle|Suite 390|Fairview|Terrebonne Parish|LA|75709|United States|-6|condo| +30507|AAAAAAAALCHHAAAA|935|Washington |Court|Suite 20|Springdale|Rusk County|WI|58883|United States|-6|single family| +30508|AAAAAAAAMCHHAAAA|303|Center |Boulevard|Suite C|Hopewell|Edmunds County|SD|50587|United States|-6|apartment| +30509|AAAAAAAANCHHAAAA|508|Spring Lake|Circle|Suite 280|Mountain View|Otoe County|NE|64466|United States|-7|apartment| +30510|AAAAAAAAOCHHAAAA|72|Highland Lake|Pkwy|Suite 280|Mechanicsburg|Clay County|SD|52219|United States|-6|apartment| +30511|AAAAAAAAPCHHAAAA|466|Oak 11th|Lane|Suite 260|Midway|Cassia County|ID|81904|United States|-7|condo| +30512|AAAAAAAAADHHAAAA|360|4th 11th|Blvd|Suite M|Galena|Otsego County|MI|44369|United States|-5|condo| +30513|AAAAAAAABDHHAAAA|597|3rd |Ct.|Suite 320|Kingston|Crow Wing County|MN|54975|United States|-6|apartment| +30514|AAAAAAAACDHHAAAA|290|2nd |Circle|Suite 440|Pleasant Valley|Okaloosa County|FL|32477|United States|-5|apartment| +30515|AAAAAAAADDHHAAAA|628|7th Pine|Drive|Suite 40|Antioch|Wayne County|UT|88605|United States|-7|apartment| +30516|AAAAAAAAEDHHAAAA|767|Laurel Ridge|Street|Suite U|Unionville|Pierce County|WA|91711|United States|-8|single family| +30517|AAAAAAAAFDHHAAAA|552|3rd 13th|RD|Suite 420|Valley View|Kenton County|KY|45124|United States|-5|apartment| +30518|AAAAAAAAGDHHAAAA|422|Willow |Dr.|Suite 40|Hamilton|Scott County|IL|62808|United States|-6|condo| +30519|AAAAAAAAHDHHAAAA|912|Second Green|Ln|Suite O|Oakwood|Nassau County|FL|30169|United States|-5|single family| +30520|AAAAAAAAIDHHAAAA|690|Laurel |Cir.|Suite 140|Elizabeth|Briscoe County|TX|72935|United States|-6|apartment| +30521|AAAAAAAAJDHHAAAA|980|Smith Spruce|Cir.|Suite 490|Centerville|Bowie County|TX|70059|United States|-6|single family| +30522|AAAAAAAAKDHHAAAA|340|Second |Cir.|Suite O|Mount Zion|Shelby County|IN|48054|United States|-5|condo| +30523|AAAAAAAALDHHAAAA|130|Meadow |Cir.|Suite X|Marion|Hanover County|VA|20399|United States|-5|apartment| +30524|AAAAAAAAMDHHAAAA|155|Locust Birch|Way|Suite 380|Buena Vista|Morton County|KS|65752|United States|-6|apartment| +30525|AAAAAAAANDHHAAAA|666|Elevnth Sycamore|Wy|Suite P|Glendale|Chambers County|AL|33951|United States|-6|single family| +30526|AAAAAAAAODHHAAAA|851|Locust 6th|Wy|Suite H|Mount Olive|Lauderdale County|AL|38059|United States|-6|condo| +30527|AAAAAAAAPDHHAAAA|650|Oak View|Ave|Suite 220|Riverside|Canyon County|ID|89231|United States|-7|apartment| +30528|AAAAAAAAAEHHAAAA|114|8th |Ln|Suite 370|Howell|Dickinson County|MI|44854|United States|-5|single family| +30529|AAAAAAAABEHHAAAA|942|Pine |Court|Suite B|Buena Vista|Logan County|OH|45752|United States|-5|condo| +30530|AAAAAAAACEHHAAAA|16|8th |Pkwy|Suite Q|Mount Pleasant|Randolph County|IN|41933|United States|-5|apartment| +30531|AAAAAAAADEHHAAAA|3|Wilson Spring|Cir.|Suite Y|Glenwood|Menard County|IL|63511|United States|-6|condo| +30532|AAAAAAAAEEHHAAAA|497|River Lake|Court|Suite Q|Macon|Bowie County|TX|70369|United States|-6|condo| +30533|AAAAAAAAFEHHAAAA|276|Jefferson Lakeview|Boulevard|Suite L|Farmington|Sheboygan County|WI|59145|United States|-6|condo| +30534|AAAAAAAAGEHHAAAA|||||Maple Grove|Washington County|ID||||single family| +30535|AAAAAAAAHEHHAAAA|683|Elm |Wy|Suite D|Marion|Jefferson County|ID|80399|United States|-7|condo| +30536|AAAAAAAAIEHHAAAA|148|Pine Oak|Ct.|Suite 350|Plainview|Oldham County|KY|43683|United States|-5|single family| +30537|AAAAAAAAJEHHAAAA|398|9th |Avenue|Suite E|Sheffield|Crowley County|CO|86896|United States|-7|single family| +30538|AAAAAAAAKEHHAAAA|33|West North|Circle|Suite 270|Maple Grove|Mille Lacs County|MN|58252|United States|-6|condo| +30539|AAAAAAAALEHHAAAA|406|Seventh |Pkwy|Suite O|Riverdale|Philadelphia County|PA|19391|United States|-5|apartment| +30540|AAAAAAAAMEHHAAAA|564|Oak |Street|Suite C|Oakdale|Grayson County|TX|79584|United States|-6|single family| +30541|AAAAAAAANEHHAAAA|132|9th 7th|Circle|Suite Y|Westgate|Wapello County|IA|52366|United States|-6|condo| +30542|AAAAAAAAOEHHAAAA|83|Cedar Spring|Boulevard|Suite Y|Highland Park|Charlotte County|FL|36534|United States|-5|apartment| +30543|AAAAAAAAPEHHAAAA|131|Mill |Ct.|Suite 350|Forest Hills|Otero County|CO|89237|United States|-7|condo| +30544|AAAAAAAAAFHHAAAA|473|Meadow Ridge|Street|Suite Q|Pleasant Grove|Edwards County|KS|64136|United States|-6|apartment| +30545|AAAAAAAABFHHAAAA|635|Jefferson Broadway|Drive|Suite C|Lebanon|Wood County|WV|22898|United States|-5|apartment| +30546|AAAAAAAACFHHAAAA|434|Lake 9th|Court|Suite I|Glenwood|Throckmorton County|TX|73511|United States|-6|condo| +30547|AAAAAAAADFHHAAAA|213|Washington 8th|Road|Suite F|White Hall|Washington County|FL|36955|United States|-5|single family| +30548|AAAAAAAAEFHHAAAA|795|Elm Wilson|Boulevard|Suite 160|Bridgeport|Greenup County|KY|45817|United States|-6|condo| +30549|AAAAAAAAFFHHAAAA|657|2nd |Drive|Suite 460|Union|Onondaga County|NY|18721|United States|-5|condo| +30550|AAAAAAAAGFHHAAAA|49|Dogwood Eigth|Circle|Suite Y|Greenfield|Ohio County|KY|45038|United States|-5|single family| +30551|AAAAAAAAHFHHAAAA|879|Park 7th|ST|Suite U|Perry|Rice County|MN|51830|United States|-6|apartment| +30552|AAAAAAAAIFHHAAAA|69|Woodland |RD|Suite R|Salem|Yukon-Koyukuk Census Area|AK|98048|United States|-9|single family| +30553|AAAAAAAAJFHHAAAA|898|4th |Blvd|Suite Q|Pleasant Hill|Woodford County|KY|43604|United States|-6|single family| +30554|AAAAAAAAKFHHAAAA|951|View |Boulevard|Suite X|New Hope|York County|NE|69431|United States|-6|apartment| +30555|AAAAAAAALFHHAAAA|49|Sixth Lake|Pkwy|Suite S|Sulphur Springs|Daniels County|MT|68354|United States|-7|single family| +30556|AAAAAAAAMFHHAAAA|397|Elm |Cir.|Suite 190|Spring Valley|Yellow Medicine County|MN|56060|United States|-6|condo| +30557|AAAAAAAANFHHAAAA|72|Mill |Cir.|Suite R|Highland|Knox County|NE|69454|United States|-7|condo| +30558|AAAAAAAAOFHHAAAA|451|Green South|Wy|Suite 140|Kingston|Kanabec County|MN|54975|United States|-6|apartment| +30559|AAAAAAAAPFHHAAAA|141|Poplar 2nd|RD|Suite N|Clifton|Saline County|MO|68014|United States|-6|condo| +30560|AAAAAAAAAGHHAAAA|76|Main Birch|Blvd|Suite O|Five Forks|Christian County|KY|42293|United States|-6|condo| +30561|AAAAAAAABGHHAAAA|566|Adams |Cir.|Suite 90|Five Forks|Wyandotte County|KS|62293|United States|-6|condo| +30562|AAAAAAAACGHHAAAA|701|13th Woodland|Blvd|Suite 430|Springfield|Cobb County|GA|39303|United States|-5|condo| +30563|AAAAAAAADGHHAAAA|160|Elm |Avenue|Suite 490|Spring Valley|Lyon County|KS|66060|United States|-6|single family| +30564|AAAAAAAAEGHHAAAA|35|Church Woodland|Ct.|Suite 380|Riverview|Hampton County|SC|29003|United States|-5|single family| +30565|AAAAAAAAFGHHAAAA|449|Pine |Parkway|Suite B|Mount Olive|Gregg County|TX|78059|United States|-6|single family| +30566|AAAAAAAAGGHHAAAA|426|Center Main|Lane|Suite 120|Springfield|Carbon County|WY|89303|United States|-7|single family| +30567|AAAAAAAAHGHHAAAA|82|3rd |Drive|Suite 410|Stringtown|Marion County|OH|40162|United States|-5|apartment| +30568|AAAAAAAAIGHHAAAA|114|Spring Main|Ln|Suite U|Westfield|Cuyahoga County|OH|41900|United States|-5|single family| +30569|AAAAAAAAJGHHAAAA|306|7th |Way|Suite R|Richfield|Logan County|KS|66196|United States|-6|condo| +30570|AAAAAAAAKGHHAAAA|865|Pine |Boulevard|Suite W|Oak Hill|Camp County|TX|77838|United States|-6|condo| +30571|AAAAAAAALGHHAAAA|621|Oak |Way|Suite F|Lincoln|Itasca County|MN|51289|United States|-6|condo| +30572|AAAAAAAAMGHHAAAA|259|Washington |Pkwy|Suite B|Mountain View|Henrico County|VA|24466|United States|-5|single family| +30573|AAAAAAAANGHHAAAA|685|Lee Pine|Road|Suite 240|Oak Grove|Anson County|NC|28370|United States|-5|apartment| +30574|AAAAAAAAOGHHAAAA|23|Williams |Dr.|Suite 220|Newport|Labette County|KS|61521|United States|-6|apartment| +30575|AAAAAAAAPGHHAAAA|747|Pine Forest|Circle|Suite J|Oakland|York County|SC|29843|United States|-5|single family| +30576|AAAAAAAAAHHHAAAA|747|Park 6th|Wy|Suite 390|Red Hill|Perry County|MO|64338|United States|-6|condo| +30577|AAAAAAAABHHHAAAA|917|Third 6th|Ave|Suite 40|Unionville|Mercer County|ND|51711|United States|-6|apartment| +30578|AAAAAAAACHHHAAAA|491|Meadow |Dr.|Suite O|Hamilton|Calhoun County|IL|62808|United States|-6|condo| +30579|AAAAAAAADHHHAAAA|350|Pine Tenth|Way|Suite X|Glenwood|Jackson County|AR|73511|United States|-6|single family| +30580|AAAAAAAAEHHHAAAA|138|8th |Avenue|Suite B|Hillcrest|Nolan County|TX|73003|United States|-6|single family| +30581|AAAAAAAAFHHHAAAA|304|Elm Poplar|Street|Suite 0|Fairview|Craven County|NC|25709|United States|-5|apartment| +30582|AAAAAAAAGHHHAAAA|154|Central |Ave|Suite O|Waterloo|Franklin County|TX|71675|United States|-6|single family| +30583|AAAAAAAAHHHHAAAA|779|Oak North|Boulevard|Suite A|Cedar Grove|Jefferson County|AL|30411|United States|-6|single family| +30584|AAAAAAAAIHHHAAAA|572|9th Davis|Dr.|Suite 450|Woodland|Nowata County|OK|74854|United States|-6|condo| +30585|AAAAAAAAJHHHAAAA|158|South Smith|Parkway|Suite K|Franklin|Emmet County|IA|59101|United States|-6|apartment| +30586|AAAAAAAAKHHHAAAA|20|Lake College|ST|Suite I|Hamilton|Vinton County|OH|42808|United States|-5|condo| +30587|AAAAAAAALHHHAAAA|48|3rd |Wy|Suite G|Woodlawn|Charlotte County|FL|34098|United States|-5|condo| +30588|AAAAAAAAMHHHAAAA|421|Pine |Dr.|Suite K|Lakeside|Marshall County|IA|59532|United States|-6|single family| +30589|AAAAAAAANHHHAAAA|553|Adams Park|Circle|Suite 330|Glenwood|Caldwell Parish|LA|73511|United States|-6|apartment| +30590|AAAAAAAAOHHHAAAA|674|Willow Valley|Ln|Suite 270|Valley View|Southampton County|VA|25124|United States|-5|condo| +30591|AAAAAAAAPHHHAAAA|409|Birch Elm|Ave|Suite E|Oak Hill|Lawrence County|AR|77838|United States|-6|single family| +30592|AAAAAAAAAIHHAAAA|846|Oak Cherry|Ave|Suite 490|Oak Hill|Lincoln County|WV|27838|United States|-5|apartment| +30593|AAAAAAAABIHHAAAA|698|13th Fourteenth|Parkway|Suite 350|Newport|Walker County|AL|31521|United States|-6|single family| +30594|AAAAAAAACIHHAAAA|151|Lee |Wy|Suite G|Macon|Bell County|TX|70369|United States|-6|apartment| +30595|AAAAAAAADIHHAAAA|376|Ridge Poplar|Dr.|Suite 350|Fairview|Riley County|KS|65709|United States|-6|apartment| +30596|AAAAAAAAEIHHAAAA|949|Adams Center|Ln|Suite A|Avery|Union County|TN|30194|United States|-6|single family| +30597|AAAAAAAAFIHHAAAA|555|Smith |Court|Suite H|Lakeside|Clay County|MN|59532|United States|-6|apartment| +30598|AAAAAAAAGIHHAAAA|735|Ash South|Court|Suite 420|Newtown|DeWitt County|TX|71749|United States|-6|condo| +30599|AAAAAAAAHIHHAAAA|492|Eigth Dogwood|Pkwy|Suite W|Belmont|Atlantic County|NJ|00791|United States|-5|apartment| +30600|AAAAAAAAIIHHAAAA|885|View |Avenue|Suite 40|Fairview|Strafford County|NH|06309|United States|-5|single family| +30601|AAAAAAAAJIHHAAAA|77|13th |Pkwy|Suite V|Highland Park|Graham County|NC|26534|United States|-5|single family| +30602|AAAAAAAAKIHHAAAA|969|Washington Lincoln|Lane|Suite 450|Centerville|Glascock County|GA|30059|United States|-5|apartment| +30603|AAAAAAAALIHHAAAA|946|Franklin |Road|Suite S|Bethel|Van Buren County|AR|75281|United States|-6|single family| +30604|AAAAAAAAMIHHAAAA|||Court|Suite 90|Mountain View|Nuckolls County||||-7|| +30605|AAAAAAAANIHHAAAA|499|Fifth Third|Court|Suite X|Walnut Grove|Ashland County|OH|47752|United States|-5|condo| +30606|AAAAAAAAOIHHAAAA|379|Oak |Circle|Suite G|Florence|Collier County|FL|33394|United States|-5|single family| +30607|AAAAAAAAPIHHAAAA||||Suite K|||OR||||| +30608|AAAAAAAAAJHHAAAA|835|Hill |Avenue|Suite H|Florence|Rutherford County|TN|33394|United States|-6|single family| +30609|AAAAAAAABJHHAAAA|452|Central |Way|Suite P|Oak Ridge|Refugio County|TX|78371|United States|-6|condo| +30610|AAAAAAAACJHHAAAA|635|15th Fourth|Avenue|Suite R|Greenfield|Florence County|WI|55038|United States|-6|condo| +30611|AAAAAAAADJHHAAAA|678|Hill 7th|Road|Suite I|Lakewood|Murray County|GA|38877|United States|-5|single family| +30612|AAAAAAAAEJHHAAAA|284|First Cedar|Avenue|Suite 170|Shiloh|Furnas County|NE|69275|United States|-6|apartment| +30613|AAAAAAAAFJHHAAAA|446|Oak |Wy|Suite N|Ashland|Sullivan County|PA|14244|United States|-5|condo| +30614|AAAAAAAAGJHHAAAA|8|13th |Circle|Suite B|Pleasant Valley|Montcalm County|MI|42477|United States|-5|single family| +30615|AAAAAAAAHJHHAAAA|273|Chestnut Wilson|Blvd|Suite V|Lakewood|Polk County|FL|38877|United States|-5|condo| +30616|AAAAAAAAIJHHAAAA|627|Green Jackson|Circle|Suite M|Brownsville|Orange County|CA|99310|United States|-8|condo| +30617|AAAAAAAAJJHHAAAA|781|Fourth Main|Parkway|Suite U|Plainview|Churchill County|NV|83683|United States|-8|single family| +30618|AAAAAAAAKJHHAAAA|693|North Elm|ST|Suite 340|Enterprise|Clarke County|VA|21757|United States|-5|apartment| +30619|AAAAAAAALJHHAAAA|73|8th Johnson|Ave|Suite 80|Stringtown|Jefferson County|AL|30162|United States|-6|condo| +30620|AAAAAAAAMJHHAAAA|35|West |Drive|Suite 80|Willow|Oglethorpe County|GA|36798|United States|-5|apartment| +30621|AAAAAAAANJHHAAAA|305|Elevnth Washington|Ct.|Suite 170|Oakdale|Ector County|TX|79584|United States|-6|single family| +30622|AAAAAAAAOJHHAAAA|575|Second |Ave|Suite 100|Macedonia|Grand County|UT|81087|United States|-7|condo| +30623|AAAAAAAAPJHHAAAA|79|East |ST|Suite 10|Bethel|Crawford County|IN|45281|United States|-5|apartment| +30624|AAAAAAAAAKHHAAAA|487|15th Fourth|Court|Suite 110|Wildwood|Monroe County|KY|46871|United States|-5|single family| +30625|AAAAAAAABKHHAAAA|447|Lakeview |Ct.|Suite D|Winslow|Louisa County|VA|28525|United States|-5|condo| +30626|AAAAAAAACKHHAAAA|607|Lake Lincoln|Lane|Suite 170|Five Forks|Montgomery County|MD|22293|United States|-5|apartment| +30627|AAAAAAAADKHHAAAA|113|Walnut |Drive|Suite A|Greenville|Harrisonburg city|VA|21387|United States|-5|single family| +30628|AAAAAAAAEKHHAAAA|741|Smith 11th|Street|Suite J|Farmington|Jefferson County|ID|89145|United States|-7|single family| +30629|AAAAAAAAFKHHAAAA|469|Meadow Franklin|Lane|Suite M|Jackson|Franklin County|KS|69583|United States|-6|condo| +30630|AAAAAAAAGKHHAAAA|379|9th Meadow|Ct.|Suite B|Enterprise|Isabella County|MI|41757|United States|-5|single family| +30631|AAAAAAAAHKHHAAAA|681|Elm 9th|Cir.|Suite 420|Fairview|Chemung County|NY|15709|United States|-5|single family| +30632|AAAAAAAAIKHHAAAA|88|Smith Williams|Street|Suite V|Woodlawn|Butler County|KY|44098|United States|-6|apartment| +30633|AAAAAAAAJKHHAAAA|654|Railroad |Blvd|Suite T|Enterprise|Lincoln County|SD|51757|United States|-7|condo| +30634|AAAAAAAAKKHHAAAA|761|Hillcrest 6th|Ct.|||Lincoln County|||United States||| +30635|AAAAAAAALKHHAAAA|930|Woodland Spring|ST|Suite F|Five Forks|Des Moines County|IA|52293|United States|-6|condo| +30636|AAAAAAAAMKHHAAAA|441|Hillcrest 9th|Cir.|Suite X|Summit|Marquette County|MI|40499|United States|-5|condo| +30637|AAAAAAAANKHHAAAA|395|2nd |Cir.|Suite 180|Springfield|Dougherty County|GA|39303|United States|-5|apartment| +30638|AAAAAAAAOKHHAAAA|303|Chestnut Third|Way|Suite 480|Enterprise|Foard County|TX|71757|United States|-6|single family| +30639|AAAAAAAAPKHHAAAA|313|Walnut |Dr.|Suite 130|Riverdale|Powell County|KY|49391|United States|-5|condo| +30640|AAAAAAAAALHHAAAA|662|Walnut Maple|Street|Suite 470|Kingston|Washington County|WI|54975|United States|-6|condo| +30641|AAAAAAAABLHHAAAA|457|2nd Sycamore|Drive|Suite G|Scottsville|Perry County|IL|64190|United States|-6|single family| +30642|AAAAAAAACLHHAAAA|314|Adams |Ct.|Suite V|White Oak|Edmonson County|KY|46668|United States|-6|single family| +30643|AAAAAAAADLHHAAAA|760|Davis Fifteenth|Parkway|Suite O|Wildwood|Hitchcock County|NE|66871|United States|-6|apartment| +30644|AAAAAAAAELHHAAAA|839|Spruce Walnut|Pkwy|Suite 40|Lincoln|Pawnee County|KS|61289|United States|-6|single family| +30645|AAAAAAAAFLHHAAAA|674|14th |Way|Suite 200|Newport|Crawford County|PA|11521|United States|-5|condo| +30646|AAAAAAAAGLHHAAAA|47|5th 9th|ST|Suite 330|Shiloh|Rio Arriba County|NM|89275|United States|-7|single family| +30647|AAAAAAAAHLHHAAAA|498|Ninth Hillcrest|Street|Suite 130|Waterloo|Oxford County|ME|02275|United States|-5|apartment| +30648|AAAAAAAAILHHAAAA|683|11th Park|Avenue|Suite S|Pleasant Grove|Hale County|TX|74136|United States|-6|single family| +30649|AAAAAAAAJLHHAAAA|391|7th |Court|Suite T|Enterprise|Bladen County|NC|21757|United States|-5|condo| +30650|AAAAAAAAKLHHAAAA|73|View |Lane|Suite 190|Frogtown|Missaukee County|MI|48784|United States|-5|single family| +30651|AAAAAAAALLHHAAAA|316|Park Walnut|Lane|Suite 300|Enterprise|Lake County|TN|31757|United States|-6|condo| +30652|AAAAAAAAMLHHAAAA|810|Ridge Locust|Parkway|Suite E|Forest Hills|Juneau County|WI|59237|United States|-6|single family| +30653|AAAAAAAANLHHAAAA|427|Washington |Ln|Suite B|Cedar Grove|Otter Tail County|MN|50411|United States|-6|single family| +30654|AAAAAAAAOLHHAAAA|722|Center |Avenue|Suite W|Franklin|Orleans County|NY|19101|United States|-5|apartment| +30655|AAAAAAAAPLHHAAAA|503|Central |Way|Suite 10|Jackson|Suffolk city|VA|29583|United States|-5|single family| +30656|AAAAAAAAAMHHAAAA|220|Chestnut |Cir.|Suite 420|Five Forks|Henry County|OH|42293|United States|-5|condo| +30657|AAAAAAAABMHHAAAA|979|5th Oak|Street|Suite 40|Pleasant Valley|Washington County|NE|62477|United States|-7|single family| +30658|AAAAAAAACMHHAAAA|213|Ridge Jackson|Road|Suite 40|Pleasant Grove|Conecuh County|AL|34136|United States|-6|apartment| +30659|AAAAAAAADMHHAAAA|423|13th |Parkway|Suite G|Bunker Hill|Jessamine County|KY|40150|United States|-6|condo| +30660|AAAAAAAAEMHHAAAA|381|Hill Church|Drive|Suite R|Woodville|Brown County|MN|54289|United States|-6|single family| +30661|AAAAAAAAFMHHAAAA|977|5th |Parkway|Suite C|Shiloh|Briscoe County|TX|79275|United States|-6|condo| +30662|AAAAAAAAGMHHAAAA|672|8th |Blvd|Suite D|Clearview|Dixon County|NE|65495|United States|-6|single family| +30663|AAAAAAAAHMHHAAAA|205|9th |Drive|Suite G|Kingston|Guthrie County|IA|54975|United States|-6|apartment| +30664|AAAAAAAAIMHHAAAA|884|6th |Lane|Suite 330|Oak Ridge|Meeker County|MN|58371|United States|-6|condo| +30665|AAAAAAAAJMHHAAAA|214|Dogwood Valley|Blvd|Suite N|Lakeville|Coosa County|AL|38811|United States|-6|condo| +30666|AAAAAAAAKMHHAAAA|394|Wilson |Avenue|Suite R|Shiloh|Pulaski County|VA|29275|United States|-5|condo| +30667|AAAAAAAALMHHAAAA|335|8th |Avenue|Suite 150|Lakewood|Halifax County|VA|28877|United States|-5|condo| +30668|AAAAAAAAMMHHAAAA|351|Davis Dogwood|Boulevard|Suite 380|Five Forks|Cedar County|NE|62293|United States|-6|condo| +30669|AAAAAAAANMHHAAAA|558|Lincoln Lincoln|Ln|Suite L|Spring Hill|Presidio County|TX|76787|United States|-6|single family| +30670|AAAAAAAAOMHHAAAA|401|Birch Forest|Drive|Suite 360|Providence|Columbia County|FL|36614|United States|-5|condo| +30671|AAAAAAAAPMHHAAAA|966|4th 8th|Boulevard|Suite 450|Valley View|Sunflower County|MS|55124|United States|-6|apartment| +30672|AAAAAAAAANHHAAAA|878|Lee |Blvd|Suite 40|Clifton|Freeborn County|MN|58014|United States|-6|apartment| +30673|AAAAAAAABNHHAAAA|447|Adams Johnson|Dr.|Suite S|Wilson|Ashe County|NC|26971|United States|-5|single family| +30674|AAAAAAAACNHHAAAA|112|Smith Lake|ST|Suite 430|Oakland|Marengo County|AL|39843|United States|-6|apartment| +30675|AAAAAAAADNHHAAAA|151|Park Spring|Parkway|Suite 60|Forest Hills|Whiteside County|IL|69237|United States|-6|single family| +30676|AAAAAAAAENHHAAAA|221|Cherry |Parkway|Suite 180|Oakland|Henry County|MO|69843|United States|-6|single family| +30677|AAAAAAAAFNHHAAAA|917|13th |Wy|Suite E|Sleepy Hollow|Howard County|IA|53592|United States|-6|single family| +30678|AAAAAAAAGNHHAAAA|187|Adams 2nd|Ave|Suite 440|Woodville|Claiborne Parish|LA|74289|United States|-6|apartment| +30679|AAAAAAAAHNHHAAAA|772|Locust |Cir.|Suite E|Shady Grove|Wright County|IA|52812|United States|-6|apartment| +30680|AAAAAAAAINHHAAAA|662|9th Sixth|RD|Suite P|Owens|San Juan County|WA|92324|United States|-8|apartment| +30681|AAAAAAAAJNHHAAAA|529|Church Ash|Ave|Suite Y|Woodville|Love County|OK|74289|United States|-6|apartment| +30682|AAAAAAAAKNHHAAAA|137|Davis |Blvd|Suite Q|Fairfield|Leon County|TX|76192|United States|-6|single family| +30683|AAAAAAAALNHHAAAA|743|Davis 13th|ST|Suite N|Salem|Pendleton County|WV|28048|United States|-5|condo| +30684|AAAAAAAAMNHHAAAA|437|Fourth Mill|Court|Suite S|Highland Park|Teller County|CO|86534|United States|-7|condo| +30685|AAAAAAAANNHHAAAA|195|Valley |Blvd|Suite A|Riverview|Louisa County|IA|59003|United States|-6|apartment| +30686|AAAAAAAAONHHAAAA|700|South |Parkway|Suite R|White Oak|Carroll County|IN|46668|United States|-5|apartment| +30687|AAAAAAAAPNHHAAAA|368|Forest |Blvd|Suite 140|Denmark|Brooke County|WV|25576|United States|-5|condo| +30688|AAAAAAAAAOHHAAAA|761|Ash 3rd|ST|Suite 440|Pleasant Valley|Clark County|ID|82477|United States|-7|single family| +30689|AAAAAAAABOHHAAAA|200|Walnut |Street|Suite 270|Scottsville|Pike County|MO|64190|United States|-6|condo| +30690|AAAAAAAACOHHAAAA|396|Pine |Court|Suite 470|Oak Ridge|Archuleta County|CO|88371|United States|-7|apartment| +30691|AAAAAAAADOHHAAAA|487|Meadow 7th|Blvd|Suite P|Oakwood|Richardson County|NE|60169|United States|-7|apartment| +30692|AAAAAAAAEOHHAAAA|104|Spring |Ln|Suite 210|Newtown|Stafford County|KS|61749|United States|-6|single family| +30693|AAAAAAAAFOHHAAAA|1|Cedar |Street|Suite 420|Newtown|Clinch County|GA|31749|United States|-5|apartment| +30694|AAAAAAAAGOHHAAAA|421|6th Miller|RD|Suite 370|Rolling Hills|Duval County|TX|77272|United States|-6|condo| +30695|AAAAAAAAHOHHAAAA|60|Railroad Mill|Cir.|Suite T|Antioch|Louisa County|VA|28605|United States|-5|condo| +30696|AAAAAAAAIOHHAAAA|597|1st Birch|Wy|Suite 120|Union Hill|Bennington County|VT|08346|United States|-5|single family| +30697|AAAAAAAAJOHHAAAA|601|Second West|Ave|Suite U|Springfield|Bristol Bay Borough|AK|99303|United States|-9|condo| +30698|AAAAAAAAKOHHAAAA||Jackson Main|Circle||||KS|67745||-6|| +30699|AAAAAAAALOHHAAAA|556|Railroad |Boulevard|Suite B|Forest Hills|Jefferson County|CO|89237|United States|-7|apartment| +30700|AAAAAAAAMOHHAAAA|792|Green |Ln|Suite Q|Hillcrest|Knox County|IN|43003|United States|-5|single family| +30701|AAAAAAAANOHHAAAA|730|Hill Sycamore|RD|Suite 170|Friendship|Fluvanna County|VA|24536|United States|-5|apartment| +30702|AAAAAAAAOOHHAAAA|965|Woodland Maple|Circle|Suite 380|Centerville|Madison County|AL|30059||-6|single family| +30703|AAAAAAAAPOHHAAAA|208|Lincoln Lakeview|ST|Suite S|Church Hill|Stephenson County|IL|63790|United States|-6|single family| +30704|AAAAAAAAAPHHAAAA|814|North |Way|Suite 310|Spring Hill|Kodiak Island Borough|AK|96787|United States|-9|single family| +30705|AAAAAAAABPHHAAAA|351|Railroad |Dr.|Suite 360|Caledonia|Colquitt County|GA|37411|United States|-5|apartment| +30706|AAAAAAAACPHHAAAA|708|3rd Green|Street|Suite 270|Woodland|Perry County|MS|54854|United States|-6|condo| +30707|AAAAAAAADPHHAAAA|802|4th Mill|Circle|Suite 320|Willis|Otoe County|NE|66788|United States|-7|apartment| +30708|AAAAAAAAEPHHAAAA|503|Cedar Elm|Pkwy|Suite 470|Oakwood|Richland County|MT|60169|United States|-7|condo| +30709|AAAAAAAAFPHHAAAA|1|Sixth Washington|Pkwy|Suite R|Pine Grove|Cook County|MN|54593|United States|-6|apartment| +30710|AAAAAAAAGPHHAAAA|842|Wilson Third|Ln|Suite Y|Woodlawn|Quitman County|MS|54098|United States|-6|apartment| +30711|AAAAAAAAHPHHAAAA||1st |||Newport|Calhoun County|IL|61521|||single family| +30712|AAAAAAAAIPHHAAAA|709||RD|Suite 210|||MO|60587|||apartment| +30713|AAAAAAAAJPHHAAAA|564|Park Oak|Cir.|Suite I|Maple Grove|Taylor County|IA|58252|United States|-6|condo| +30714|AAAAAAAAKPHHAAAA|141|13th North|Parkway|Suite 150|Lebanon|Siskiyou County|CA|92898|United States|-8|condo| +30715|AAAAAAAALPHHAAAA|891|Williams Pine|Blvd|Suite S|Oakdale|Auglaize County|OH|49584|United States|-5|apartment| +30716|AAAAAAAAMPHHAAAA|434|Willow |Court|Suite N|Arlington|Lincoln County|AR|76557|United States|-6|single family| +30717|AAAAAAAANPHHAAAA|611|Elm |Street|Suite 160|Omega|Canyon County|ID|86364|United States|-7|condo| +30718|AAAAAAAAOPHHAAAA|629|Dogwood |RD|Suite 380|Brownsville|Martin County|NC|29310|United States|-5|condo| +30719|AAAAAAAAPPHHAAAA|575|Johnson Hickory|Blvd|Suite D|Price|Jackson County|MS|53824|United States|-6|single family| +30720|AAAAAAAAAAIHAAAA|381|Jefferson College|Boulevard|Suite C|Brownsville|Kanawha County|WV|29310|United States|-5|single family| +30721|AAAAAAAABAIHAAAA|741|4th 6th|Ln|Suite O|Oak Grove|Suffolk County|NY|18370|United States|-5|single family| +30722|AAAAAAAACAIHAAAA|346|9th |Wy|Suite Q|Crossroads|Howard County|IN|40534|United States|-5|single family| +30723|AAAAAAAADAIHAAAA|567|Mill |Ave|Suite S|Evans|Marion County|MO|62284|United States|-6|condo| +30724|AAAAAAAAEAIHAAAA|790|15th Cedar||Suite 70||Jasper County||||-6|| +30725|AAAAAAAAFAIHAAAA|975|Cherry Elm|Boulevard|Suite O|Marion|Shenandoah County|VA|20399|United States|-5|single family| +30726|AAAAAAAAGAIHAAAA|290|South 15th|RD|Suite X|Greenwood|Scott County|MO|68828|United States|-6|apartment| +30727|AAAAAAAAHAIHAAAA|533|Maple |Lane|Suite Q|Franklin|McDuffie County|GA|39101|United States|-5|single family| +30728|AAAAAAAAIAIHAAAA|86|Smith |Dr.|Suite L|White Oak|Hudson County|NJ|07268|United States|-5|condo| +30729|AAAAAAAAJAIHAAAA|530|Dogwood 2nd|Court|Suite R|Valley View|Rockwall County|TX|75124|United States|-6|single family| +30730|AAAAAAAAKAIHAAAA|828|Pine Lee|Boulevard|Suite V|Pleasant Grove|Clinch County|GA|34136|United States|-5|apartment| +30731|AAAAAAAALAIHAAAA|547|14th |Pkwy|Suite R|Glenwood|Frio County|TX|73511|United States|-6|condo| +30732|AAAAAAAAMAIHAAAA|959|Poplar |Cir.|Suite G|Avoca|Gilmer County|GA|30540|United States|-5|condo| +30733|AAAAAAAANAIHAAAA|149|Highland Chestnut|RD|Suite 380|Pleasant Valley|Hocking County|OH|42477|United States|-5|single family| +30734|AAAAAAAAOAIHAAAA|397|Adams |ST|Suite 230|Mountain View|Webb County|TX|74466|United States|-6|condo| +30735|AAAAAAAAPAIHAAAA|564|Mill North||||Lee County||||-5|apartment| +30736|AAAAAAAAABIHAAAA|||Cir.|Suite O|||MO||United States||| +30737|AAAAAAAABBIHAAAA|229|Chestnut Ridge|Court|Suite E|Maple Grove|Fayette County|IL|68252|United States|-6|single family| +30738|AAAAAAAACBIHAAAA|895|2nd 7th|Way|Suite 70|Valley View|Stevens County|WA|95124|United States|-8|apartment| +30739|AAAAAAAADBIHAAAA|385|Jackson |Road|Suite R|Winslow|Shelby County|IA|58525|United States|-6|condo| +30740|AAAAAAAAEBIHAAAA|324|Fifth |Ave|Suite 90|Valley View|Lawrence County|KY|45124|United States|-5|single family| +30741|AAAAAAAAFBIHAAAA|126|Pine |RD|Suite Y|Newtown|Wayne County|IN|41749|United States|-5|single family| +30742|AAAAAAAAGBIHAAAA|617|Oak |ST|Suite 80|Liberty|Lake County|SD|53451|United States|-7|condo| +30743|AAAAAAAAHBIHAAAA|616|View 4th|Blvd|Suite U|Lakeside|Washington County|IA|59532|United States|-6|single family| +30744|AAAAAAAAIBIHAAAA|602|Locust 3rd|Way|Suite X|Bethel|Lorain County|OH|45281|United States|-5|single family| +30745|AAAAAAAAJBIHAAAA|798|3rd Thirteenth|Ln|Suite 320|Springdale|Renville County|ND|58883|United States|-6|apartment| +30746|AAAAAAAAKBIHAAAA|466|Washington Adams|Lane|Suite L|Summit|Amador County|CA|90499|United States|-8|condo| +30747|AAAAAAAALBIHAAAA|236|Johnson Broadway|Boulevard|Suite V|Riverside|Otoe County|NE|69231|United States|-7|single family| +30748|AAAAAAAAMBIHAAAA|977|Jackson |Ln|Suite K|Friendship|Carroll County|VA|24536|United States|-5|single family| +30749|AAAAAAAANBIHAAAA|653|Park Tenth|Avenue|Suite S|Concord|Union County|KY|44107|United States|-5|condo| +30750|AAAAAAAAOBIHAAAA|461|Ash |Blvd|Suite V|Shady Grove|Dickinson County|IA|52812|United States|-6|single family| +30751|AAAAAAAAPBIHAAAA|787|Franklin |Blvd|Suite X|Belmont|Meeker County|MN|50191|United States|-6|condo| +30752|AAAAAAAAACIHAAAA|136|14th |Dr.|Suite U|West Liberty|Mills County|IA|54752|United States|-6|single family| +30753|AAAAAAAABCIHAAAA|546|6th |Parkway||Jackson|||79583|United States|-6|| +30754|AAAAAAAACCIHAAAA|273|Main Second|RD|Suite 90|Woodville|Polk County|OR|94289|United States|-8|apartment| +30755|AAAAAAAADCIHAAAA|613|Elm Railroad|Ave|Suite M|Friendship|Rice County|MN|54536|United States|-6|apartment| +30756|AAAAAAAAECIHAAAA|189|Valley |Blvd|Suite 330|Oak Ridge|Johnston County|NC|28371|United States|-5|condo| +30757|AAAAAAAAFCIHAAAA|221|Oak |Blvd|Suite L|Friendship|Laurens County|GA|34536|United States|-5|apartment| +30758|AAAAAAAAGCIHAAAA|708|8th |Wy|Suite T|Spring Valley|Edwards County|TX|76060|United States|-6|apartment| +30759|AAAAAAAAHCIHAAAA|403|5th |Court|Suite 80|Riverview|Stoddard County|MO|69003|United States|-6|single family| +30760|AAAAAAAAICIHAAAA|616|8th Spring|Way|Suite F|Red Hill|Nome Census Area|AK|94338|United States|-9|single family| +30761|AAAAAAAAJCIHAAAA|497|Spruce Cedar|Street|Suite U|Wilson|Saunders County|NE|66971|United States|-7|single family| +30762|AAAAAAAAKCIHAAAA|976|Williams Main|Avenue|Suite 120|Fairfield|Barton County|KS|66192|United States|-6|condo| +30763|AAAAAAAALCIHAAAA|633|3rd |Court|Suite 470|Newport|Cabell County|WV|21521|United States|-5|condo| +30764|AAAAAAAAMCIHAAAA|442|Oak |Pkwy|Suite A|Lakeside|Washington County|MD|29532|United States|-5|condo| +30765|AAAAAAAANCIHAAAA|910|Park Laurel|Court|Suite Y|Enterprise|Russell County|KS|61757|United States|-6|condo| +30766|AAAAAAAAOCIHAAAA|797|15th |Boulevard|Suite 210|Lebanon|Heard County|GA|32898|United States|-5|apartment| +30767|AAAAAAAAPCIHAAAA|981|Spruce |Parkway|Suite B|Bunker Hill|Palo Pinto County|TX|70150|United States|-6|single family| +30768|AAAAAAAAADIHAAAA|79|11th |Boulevard|Suite Q|Florence|Greenwood County|KS|63394|United States|-6|apartment| +30769|AAAAAAAABDIHAAAA|315|Fifth |Avenue|Suite 90|Kingston|Iberia Parish|LA|74975|United States|-6|single family| +30770|AAAAAAAACDIHAAAA|227|8th Hill|Dr.|Suite 340|Valley View|Jefferson County|PA|15124|United States|-5|single family| +30771|AAAAAAAADDIHAAAA|571|4th |Parkway|Suite F|Crossroads|Colfax County|NM|80534|United States|-7|apartment| +30772|AAAAAAAAEDIHAAAA|377|Cedar 1st|Drive|Suite 240|Edgewood|Sierra County|CA|90069|United States|-8|condo| +30773|AAAAAAAAFDIHAAAA|1000|Jackson Ridge|Pkwy|Suite 360|Green Acres|King County|TX|77683|United States|-6|condo| +30774|AAAAAAAAGDIHAAAA|938|Cedar Main|Ln|Suite R|Wildwood|Carver County|MN|56871|United States|-6|apartment| +30775|AAAAAAAAHDIHAAAA|190|Lincoln |Blvd|Suite I|Greenville|Hand County|SD|51387|United States|-7|condo| +30776|AAAAAAAAIDIHAAAA|940|River Maple|Drive|Suite E|Star|Benton County|OR|90725|United States|-8|apartment| +30777|AAAAAAAAJDIHAAAA|56|Spruce 2nd|RD|Suite E|Unionville|Thomas County|GA|31711|United States|-5|condo| +30778|AAAAAAAAKDIHAAAA|809|Sycamore 14th|Street|Suite O|Oakland|Toole County|MT|69843|United States|-7|apartment| +30779|AAAAAAAALDIHAAAA|922|Park |Road|Suite 340|Unionville|Henry County|MO|61711|United States|-6|condo| +30780|AAAAAAAAMDIHAAAA|122|Washington Fifth|Wy|Suite 100|Springdale|Charles City County|VA|28883|United States|-5|single family| +30781|AAAAAAAANDIHAAAA|598|West Ridge|Blvd|Suite 120|Woodland|Wabash County|IN|44854|United States|-5|condo| +30782|AAAAAAAAODIHAAAA|575|Poplar |Cir.|Suite 20|Oak Grove|Laurens County|SC|28370|United States|-5|apartment| +30783|AAAAAAAAPDIHAAAA|22|Madison |Cir.|Suite 250|Buena Vista|Jackson County|WV|25752|United States|-5|single family| +30784|AAAAAAAAAEIHAAAA|439|Washington Pine|RD|Suite 340|Pleasant Hill|Pitkin County|CO|83604|United States|-7|single family| +30785|AAAAAAAABEIHAAAA|227|Church Ash|Blvd|Suite 350|Liberty|Cleburne County|AR|73451|United States|-6|condo| +30786|AAAAAAAACEIHAAAA|252|8th |Circle|Suite 300|Hillcrest|Columbia County|GA|33003|United States|-5|single family| +30787|AAAAAAAADEIHAAAA|826|Sixth |Pkwy|Suite I|Greenville|Pittsburg County|OK|71387|United States|-6|apartment| +30788|AAAAAAAAEEIHAAAA|254|Sunset |Blvd|Suite 390|Edgewood|Corson County|SD|50069|United States|-6|condo| +30789|AAAAAAAAFEIHAAAA||Walnut Sunset|Circle||Oakwood|Dallas County|AL|30169||-6|| +30790|AAAAAAAAGEIHAAAA|540|Spring |||||UT|85709|United States||| +30791|AAAAAAAAHEIHAAAA|600|3rd |Dr.|Suite 80|Springdale|Westmoreland County|PA|18883|United States|-5|condo| +30792|AAAAAAAAIEIHAAAA|881|15th |Pkwy|Suite S|Howell|Somerset County|PA|14854|United States|-5|single family| +30793|AAAAAAAAJEIHAAAA|956|Jackson Dogwood|Ln|Suite 460|Valley View|Grady County|OK|75124|United States|-6|apartment| +30794|AAAAAAAAKEIHAAAA|246|Willow |Pkwy|Suite C|Mount Pleasant|Grant County|IN|41933|United States|-5|single family| +30795|AAAAAAAALEIHAAAA|646|Main West|Street|Suite K|Bethel|Hall County|TX|75281|United States|-6|single family| +30796|AAAAAAAAMEIHAAAA|120|Adams |Pkwy|Suite C|Bethel|Adams County|ID|85281|United States|-7|apartment| +30797|AAAAAAAANEIHAAAA|619|Main Walnut|Wy|Suite 410|Woodlawn|Tate County|MS|54098|United States|-6|apartment| +30798|AAAAAAAAOEIHAAAA|749||Ln|Suite R||Johnson County|AR|74136|United States||apartment| +30799|AAAAAAAAPEIHAAAA|361|Wilson |Pkwy|Suite I|Point Pleasant|Florence County|WI|54749|United States|-6|condo| +30800|AAAAAAAAAFIHAAAA|845|4th Oak|Ln|Suite 300|Green Acres|Paulding County|OH|47683|United States|-5|apartment| +30801|AAAAAAAABFIHAAAA|47|Center |Circle|Suite 310|Midway|Jefferson County|MO|61904|United States|-6|apartment| +30802|AAAAAAAACFIHAAAA|427|Center 6th|Street|Suite 210|Amity|Lavaca County|TX|70766|United States|-6|condo| +30803|AAAAAAAADFIHAAAA|429|Main 6th|Ave|Suite K|Sleepy Hollow|Rock County|MN|53592|United States|-6|condo| +30804|AAAAAAAAEFIHAAAA|362|1st |Cir.|Suite 310|Macedonia|Jerauld County|SD|51087|United States|-7|single family| +30805|AAAAAAAAFFIHAAAA|64|Ash Main|Lane|Suite O|Lincoln|Leflore County|MS|51289|United States|-6|apartment| +30806|AAAAAAAAGFIHAAAA|797|Hickory |Boulevard|Suite 250|Mount Vernon|Fannin County|TX|78482|United States|-6|single family| +30807|AAAAAAAAHFIHAAAA|506|Elm |Blvd|Suite 240|Woodlawn|Pratt County|KS|64098|United States|-6|apartment| +30808|AAAAAAAAIFIHAAAA|545|Meadow |Court|Suite R|Antioch|Creek County|OK|78605|United States|-6|condo| +30809|AAAAAAAAJFIHAAAA|982|5th |Parkway|Suite U|Pleasant Valley|Lawrence County|AR|72477|United States|-6|single family| +30810|AAAAAAAAKFIHAAAA|552|2nd |Ct.|Suite 390|Pleasant Valley|Hinds County|MS|52477|United States|-6|apartment| +30811|AAAAAAAALFIHAAAA|47|Lake |Ln|Suite J|Concord|Lawrence County|PA|14107|United States|-5|apartment| +30812|AAAAAAAAMFIHAAAA|614|Maple Dogwood|ST|Suite 380|Kingston|Adams County|ID|84975|United States|-7|condo| +30813|AAAAAAAANFIHAAAA|18|Central |ST|Suite F|Sulphur Springs|Collin County|TX|78354|United States|-6|single family| +30814|AAAAAAAAOFIHAAAA|656|Hillcrest |RD|Suite V|Spring Valley|Transylvania County|NC|26060|United States|-5|condo| +30815|AAAAAAAAPFIHAAAA|983|Park |Ct.|Suite 100|Concord|Lincoln County|NC|24107|United States|-5|condo| +30816|AAAAAAAAAGIHAAAA|586|Hill |Cir.|Suite 150|Riverdale|Carlton County|MN|59391|United States|-6|condo| +30817|AAAAAAAABGIHAAAA|634|Mill Williams|Pkwy|Suite Q|Glenwood|Hardy County|WV|23511|United States|-5|apartment| +30818|AAAAAAAACGIHAAAA|723|Park First|Wy|Suite 20|Bethel|Lincoln County|WY|85281|United States|-7|single family| +30819|AAAAAAAADGIHAAAA|658|Maple |Dr.|Suite E|Woodlawn|Franklin County|TX|74098|United States|-6|single family| +30820|AAAAAAAAEGIHAAAA|419|Washington |Street|Suite X|Greenville|Merrick County|NE|61387|United States|-7|single family| +30821|AAAAAAAAFGIHAAAA|471|13th 7th|RD|Suite V|Farmington|Callaway County|MO|69145|United States|-6|apartment| +30822|AAAAAAAAGGIHAAAA|795|Chestnut Second|Blvd|Suite A|Lakewood|Fresno County|CA|98877|United States|-8|condo| +30823|AAAAAAAAHGIHAAAA|71|1st |Avenue|Suite Y|Fairfield|West Baton Rouge Parish|LA|76192|United States|-6|condo| +30824|AAAAAAAAIGIHAAAA|837|4th 4th|RD|Suite 120|Five Points|Latah County|ID|86098|United States|-7|apartment| +30825|AAAAAAAAJGIHAAAA|966|13th Maple|Road|Suite Y|Union Hill|Covington city|VA|27746|United States|-5|condo| +30826|AAAAAAAAKGIHAAAA|242|11th |Cir.|Suite 200|Perkins|Burke County|GA|31852|United States|-5|apartment| +30827|AAAAAAAALGIHAAAA|847|South |Ln|Suite D|Oakdale|Washington County|MO|69584|United States|-6|apartment| +30828|AAAAAAAAMGIHAAAA|269|West Jackson|Lane|Suite 120|Arlington|McLean County|KY|46557|United States|-5|condo| +30829|AAAAAAAANGIHAAAA|721|Park |Avenue|Suite 190|Fairview|Orange County|VT|06309|United States|-5|single family| +30830|AAAAAAAAOGIHAAAA|172|Hillcrest |Pkwy|Suite Q|Bethel|Screven County|GA|35281|United States|-5|apartment| +30831|AAAAAAAAPGIHAAAA|742|Hickory East|RD|Suite G|Hamilton|Grand County|CO|82808|United States|-7|single family| +30832|AAAAAAAAAHIHAAAA|659|Sunset |Pkwy|Suite G|Midway|Grady County|OK|71904|United States|-6|apartment| +30833|AAAAAAAABHIHAAAA|23|13th Pine|Road|Suite 440|Clinton|Phillips County|CO|88222|United States|-7|single family| +30834|AAAAAAAACHIHAAAA|192|12th |RD|Suite U|Buena Vista|Hancock County|OH|45752|United States|-5|single family| +30835|AAAAAAAADHIHAAAA|62|Ash 12th|Pkwy|Suite 230|Greenfield|Howell County|MO|65038|United States|-6|condo| +30836|AAAAAAAAEHIHAAAA|514||Boulevard|Suite 210|Green Acres||PA|||-5|| +30837|AAAAAAAAFHIHAAAA|570|Walnut Hill|Ln|Suite 0|Bunker Hill|Morgan County|UT|80150|United States|-7|single family| +30838|AAAAAAAAGHIHAAAA|730|Lake |Blvd|Suite 330|Antioch|Macoupin County|IL|68605|United States|-6|single family| +30839|AAAAAAAAHHIHAAAA|489|Second Green|Wy|Suite R|Highland|Larimer County|CO|89454|United States|-7|single family| +30840|AAAAAAAAIHIHAAAA|158|Center Seventh|Ave|Suite 80|Summerfield|Skagit County|WA|90634|United States|-8|single family| +30841|AAAAAAAAJHIHAAAA|85|6th |Wy|Suite S|Hamilton|Deuel County|NE|62808|United States|-6|condo| +30842|AAAAAAAAKHIHAAAA|53|Hickory West|Way|Suite K|Deerfield|Marquette County|MI|49840|United States|-5|apartment| +30843|AAAAAAAALHIHAAAA|667|15th Park|Dr.|Suite 60|Mount Olive|Kewaunee County|WI|58059|United States|-6|apartment| +30844|AAAAAAAAMHIHAAAA|395|Lincoln Woodland|Cir.|Suite 260|Shiloh|Fairfield County|SC|29275|United States|-5|apartment| +30845|AAAAAAAANHIHAAAA|639|7th |Drive|Suite U|White Oak|Butler County|IA|56668|United States|-6|apartment| +30846|AAAAAAAAOHIHAAAA|941|Franklin |Drive|Suite 130|Lakeside|Washington County|NE|69532|United States|-7|apartment| +30847|AAAAAAAAPHIHAAAA|476|Forest |Circle|Suite I|Ashland|Dallas County|TX|74244|United States|-6|single family| +30848|AAAAAAAAAIIHAAAA|964|1st Walnut|Ave|Suite 60|Liberty|Stanislaus County|CA|93451|United States|-8|apartment| +30849|AAAAAAAABIIHAAAA|749|Forest 4th|Blvd|Suite 490|Unionville|Churchill County|NV|81711|United States|-8|condo| +30850|AAAAAAAACIIHAAAA|784|Jefferson Oak|Avenue|Suite 20|Lebanon|Cooper County|MO|62898|United States|-6|apartment| +30851|AAAAAAAADIIHAAAA|673|3rd Walnut|ST|Suite 480|Springdale|Johnston County|NC|28883|United States|-5|single family| +30852|AAAAAAAAEIIHAAAA|559|Oak |Lane|Suite 380|Providence|Johnson County|WY|86614|United States|-7|single family| +30853|AAAAAAAAFIIHAAAA|285|Second |ST|Suite G|Bethel|Union County|IN|45281|United States|-5|apartment| +30854|AAAAAAAAGIIHAAAA|994|Locust Oak|Avenue|Suite 140|Providence|Hertford County|NC|26614|United States|-5|single family| +30855|AAAAAAAAHIIHAAAA|60|Miller 6th|RD|Suite D|Fairfield|Bristol city|VA|26192|United States|-5|apartment| +30856|AAAAAAAAIIIHAAAA|991|3rd 2nd|Avenue|Suite N|Riverdale|Claiborne County|TN|39391|United States|-5|apartment| +30857|AAAAAAAAJIIHAAAA|459|Smith Williams|RD|Suite 190||||||-6|| +30858|AAAAAAAAKIIHAAAA|227|Elm |Boulevard|Suite 410|Friendship|Orange County|VT|05136|United States|-5|condo| +30859|AAAAAAAALIIHAAAA|834|Valley |Ave|Suite 390|Fairview|Page County|IA|55709|United States|-6|condo| +30860|AAAAAAAAMIIHAAAA||Cedar Seventh|Parkway|Suite 280||Grundy County|IL|65752|||| +30861|AAAAAAAANIIHAAAA|806|Pine |Drive|Suite 480|Forest Hills|Ashtabula County|OH|49237|United States|-5|apartment| +30862|AAAAAAAAOIIHAAAA|256|Laurel Adams|Pkwy|Suite A|Glenwood|Powell County|KY|43511|United States|-5|condo| +30863|AAAAAAAAPIIHAAAA|200|Church 4th|Cir.|Suite 240|Unionville|Genesee County|MI|41711|United States|-5|single family| +30864|AAAAAAAAAJIHAAAA|635|2nd Church|Dr.|Suite J|Lakewood|Lake County|CA|98877|United States|-8|single family| +30865|AAAAAAAABJIHAAAA|734|Wilson |Parkway|Suite F|Providence|Anne Arundel County|MD|26614|United States|-5|apartment| +30866|AAAAAAAACJIHAAAA|876|Hickory 8th|Blvd|Suite 310|Riverview|Worth County|MO|69003|United States|-6|condo| +30867|AAAAAAAADJIHAAAA|906|3rd |Avenue|Suite X|Lincoln|Clark County|OH|41289|United States|-5|condo| +30868|AAAAAAAAEJIHAAAA|993|12th Davis|RD|Suite 30|Fairfield|Price County|WI|56192|United States|-6|apartment| +30869|AAAAAAAAFJIHAAAA|589|Fifth |ST|Suite 180|Appleton|Burleigh County|ND|54240|United States|-6|single family| +30870|AAAAAAAAGJIHAAAA|475|Main |ST|Suite 380|Crossroads|DeBaca County|NM|80534|United States|-7|condo| +30871|AAAAAAAAHJIHAAAA|484|College 1st|Circle|Suite 450|Lincoln|San Augustine County|TX|71289|United States|-6|single family| +30872|AAAAAAAAIJIHAAAA|786|Main View|Dr.|Suite 30|Friendship|Plymouth County|MA|05136|United States|-5|single family| +30873|AAAAAAAAJJIHAAAA|188|Main Jackson|Road|Suite E|Little River|Grant County|MN|50319|United States|-6|apartment| +30874|AAAAAAAAKJIHAAAA|134|Central |Ct.|Suite O|Mount Olive|Bristol County|MA|08659|United States|-5|single family| +30875|AAAAAAAALJIHAAAA|727|Cedar 2nd|Street|Suite 380|Hillcrest|Butler County|MO|63003|United States|-6|single family| +30876|AAAAAAAAMJIHAAAA|808|3rd |Street|Suite M|Lebanon|Orange County|VA|22898|United States|-5|condo| +30877|AAAAAAAANJIHAAAA|40|Railroad Pine|Avenue|Suite 330|Highland Park|Burke County|GA|36534|United States|-5|condo| +30878|AAAAAAAAOJIHAAAA||River ||||||74289|||single family| +30879|AAAAAAAAPJIHAAAA|857|Spring |Avenue|Suite R|Marion|Rio Arriba County|NM|80399|United States|-7|apartment| +30880|AAAAAAAAAKIHAAAA|247|Fourth |Way|Suite 430|Pine Grove|Lamar County|TX|74593|United States|-6|single family| +30881|AAAAAAAABKIHAAAA|985|Main |Way|Suite 150|Sheffield|Lowndes County|AL|36896|United States|-6|apartment| +30882|AAAAAAAACKIHAAAA|796|Oak Eigth|Cir.|Suite R|Oak Grove|Hampshire County|MA|08970|United States|-5|condo| +30883|AAAAAAAADKIHAAAA|300|Hillcrest Miller|Road|||Mora County||80399|United States||condo| +30884|AAAAAAAAEKIHAAAA|599|1st Mill|Cir.|Suite H|Riverside|McCormick County|SC|29231|United States|-5|condo| +30885|AAAAAAAAFKIHAAAA|898|Highland |Cir.|Suite F|Mount Pleasant|Mackinac County|MI|41933|United States|-5|apartment| +30886|AAAAAAAAGKIHAAAA|562|Davis Second|Dr.|Suite X|Macedonia|Manatee County|FL|31087|United States|-5|apartment| +30887|AAAAAAAAHKIHAAAA|268|Adams 8th|Boulevard|Suite F|Pleasant Hill|Troup County|GA|33604|United States|-5|single family| +30888|AAAAAAAAIKIHAAAA|999|Walnut |Parkway|Suite W|Ashland|Geary County|KS|64244|United States|-6|single family| +30889|AAAAAAAAJKIHAAAA|654|Park Cherry|Blvd|Suite V|Lakeside|Owsley County|KY|49532|United States|-5|single family| +30890|AAAAAAAAKKIHAAAA|784|Second |Dr.|Suite L|Jamestown|Polk County|NC|26867|United States|-5|condo| +30891|AAAAAAAALKIHAAAA|653|Fifth |Way|Suite T|Stringtown|Judith Basin County|MT|60162|United States|-7|condo| +30892|AAAAAAAAMKIHAAAA|616|Park Jefferson|Blvd|Suite 450|Edgewood|Thomas County|GA|30069|United States|-5|condo| +30893|AAAAAAAANKIHAAAA|554|Hillcrest |Court|Suite K|Florence|Lee County|MS|53394|United States|-6|condo| +30894|AAAAAAAAOKIHAAAA|612|Cedar |Avenue|Suite J|Valley View|Union County|OR|95124|United States|-8|single family| +30895|AAAAAAAAPKIHAAAA|12|View |RD|Suite 250|Ferguson|Roanoke County|VA|21821|United States|-5|apartment| +30896|AAAAAAAAALIHAAAA|131|Johnson Eigth|Parkway|Suite 220|Mount Vernon|Wichita County|KS|68482|United States|-6|single family| +30897|AAAAAAAABLIHAAAA|813|Lake Madison|Avenue|Suite 310|Riverdale|Marion County|IA|59391|United States|-6|condo| +30898|AAAAAAAACLIHAAAA|634|9th Adams|Wy|Suite 100|Newport|Sullivan County|NH|02121|United States|-5|condo| +30899|AAAAAAAADLIHAAAA|449|North Washington|Pkwy|Suite 390|Buena Vista|Hockley County|TX|75752|United States|-6|condo| +30900|AAAAAAAAELIHAAAA|464|Laurel |Ave|Suite Y|Oakdale|Eau Claire County|WI|59584|United States|-6|apartment| +30901|AAAAAAAAFLIHAAAA|683|South |Road|Suite 80|Glendale|Park County|CO|83951|United States|-7|condo| +30902|AAAAAAAAGLIHAAAA|566|Third |Blvd|Suite H|Green Acres|Amador County|CA|97683|United States|-8|apartment| +30903|AAAAAAAAHLIHAAAA|170|East Spruce|Road|Suite V|Edgewater|Catron County|NM|80635|United States|-7|apartment| +30904|AAAAAAAAILIHAAAA|247|Lincoln Main|Drive|Suite 130|Stringtown|Susquehanna County|PA|10162|United States|-5|single family| +30905|AAAAAAAAJLIHAAAA|535|Fourteenth Church|RD|Suite 160|Mount Pleasant|Childress County|TX|71933|United States|-6|single family| +30906|AAAAAAAAKLIHAAAA|267|Walnut |Blvd|Suite D|Greenwood|Pine County|MN|58828|United States|-6|single family| +30907|AAAAAAAALLIHAAAA|712|Meadow Second|Dr.|Suite U|Oakwood|Virginia Beach city|VA|20169|United States|-5|single family| +30908|AAAAAAAAMLIHAAAA|26|Fourth |Blvd|Suite P|Berea|Humphreys County|TN|33804|United States|-5|apartment| +30909|AAAAAAAANLIHAAAA|339|Center |Cir.|Suite 320|Wilson|Slope County|ND|56971|United States|-6|condo| +30910|AAAAAAAAOLIHAAAA|364|Ash |Drive|Suite 120|Woodville|New Hanover County|NC|24289|United States|-5|condo| +30911|AAAAAAAAPLIHAAAA|202|10th 6th|Dr.|Suite E|Sutton|Lauderdale County|AL|35413|United States|-6|condo| +30912|AAAAAAAAAMIHAAAA|619|Green 4th|Avenue|Suite A|Cedar Grove|Lenoir County|NC|20411|United States|-5|single family| +30913|AAAAAAAABMIHAAAA|393|7th |Boulevard|Suite E|White Oak|Hudspeth County|TX|76668|United States|-6|apartment| +30914|AAAAAAAACMIHAAAA|243|Hickory |Road|Suite 320|Hidden Valley|Marshall County|IL|65521|United States|-6|apartment| +30915|AAAAAAAADMIHAAAA|568|Jefferson 8th|Court|Suite L|Cedar Grove|Wichita County|TX|70411|United States|-6|condo| +30916|AAAAAAAAEMIHAAAA|976|Park |Parkway|Suite B|Newtown|Chariton County|MO|61749|United States|-6|single family| +30917|AAAAAAAAFMIHAAAA|815|Mill 7th|Wy|Suite V|Highland Park|De Kalb County|IN|46534|United States|-5|condo| +30918|AAAAAAAAGMIHAAAA|657|11th South|Drive|Suite Q|Summit|La Plata County|CO|80499|United States|-7|apartment| +30919|AAAAAAAAHMIHAAAA|318|Walnut Main|Ct.|Suite 210|Stringtown|Morgan County|AL|30162|United States|-6|condo| +30920|AAAAAAAAIMIHAAAA|565|3rd |Boulevard|Suite P|Springdale|Loup County|NE|68883|United States|-7|apartment| +30921|AAAAAAAAJMIHAAAA|407|1st Poplar|Drive|Suite W|Marion|Barren County|KY|40399|United States|-6|single family| +30922|AAAAAAAAKMIHAAAA|329|South |Road|Suite U|New Hope|Borden County|TX|79431|United States|-6|single family| +30923|AAAAAAAALMIHAAAA|393|Maple |Ave|Suite U|Dewey|Union County|FL|31160|United States|-5|condo| +30924|AAAAAAAAMMIHAAAA|470|Forest |Ave|Suite E|Unionville|Brewster County|TX|71711|United States|-6|apartment| +30925|AAAAAAAANMIHAAAA|278|Jefferson |Wy|Suite O|Lakeview|Escambia County|FL|38579|United States|-5|apartment| +30926|AAAAAAAAOMIHAAAA|782|3rd Davis|ST|Suite 210|Bridgeport|Shawnee County|KS|65817|United States|-6|apartment| +30927|AAAAAAAAPMIHAAAA|393|First |Lane|Suite 210|Maple Grove|Wright County|MN|58252|United States|-6|condo| +30928|AAAAAAAAANIHAAAA||||||Boundary County|ID|88059|United States||| +30929|AAAAAAAABNIHAAAA|496|Second |Parkway|Suite G|Wilson|Coos County|OR|96971|United States|-8|single family| +30930|AAAAAAAACNIHAAAA|9|Church |Cir.|Suite I|Holland|Warrick County|IN|47751|United States|-5|single family| +30931|AAAAAAAADNIHAAAA|265|Church |Parkway|Suite 250|Greenville|Pulaski County|VA|21387|United States|-5|single family| +30932|AAAAAAAAENIHAAAA|376|2nd |Avenue|Suite T|Highland|Union County|OR|99454|United States|-8|single family| +30933|AAAAAAAAFNIHAAAA|144|Green |RD|Suite W|Pleasant Grove|Noble County|IN|44136|United States|-5|condo| +30934|AAAAAAAAGNIHAAAA|924|Washington |Court|Suite R|Buena Vista|Cumberland County|TN|35752|United States|-5|single family| +30935|AAAAAAAAHNIHAAAA|427|Broadway |ST|Suite 300|Woodville|Lewis County|WA|94289|United States|-8|condo| +30936|AAAAAAAAINIHAAAA|592|Cedar |RD|Suite 10|Pleasant Grove|Delaware County|PA|14136|United States|-5|apartment| +30937|AAAAAAAAJNIHAAAA|717|Elm |Court|Suite 80|Waterloo|Clarion County|PA|11675|United States|-5|apartment| +30938|AAAAAAAAKNIHAAAA|333|Cedar |RD|Suite 410|Harmony|Dubois County|IN|45804|United States|-5|condo| +30939|AAAAAAAALNIHAAAA|236|3rd |Boulevard|Suite 400|Enterprise|Lea County|NM|81757|United States|-7|condo| +30940|AAAAAAAAMNIHAAAA|279|Pine |Ct.|Suite G|Howell|Matagorda County|TX|74854|United States|-6|apartment| +30941|AAAAAAAANNIHAAAA|383|Oak View|Ln|Suite 60|Newtown|Norman County|MN|51749|United States|-6|condo| +30942|AAAAAAAAONIHAAAA|344|Elm |Blvd|Suite X|Clifton|Carter County|MO|68014|United States|-6|condo| +30943|AAAAAAAAPNIHAAAA|551|7th |Blvd|Suite O|Green Acres|Portage County|OH|47683|United States|-5|single family| +30944|AAAAAAAAAOIHAAAA|749|Willow |Lane|Suite N|Five Points|Klickitat County|WA|96098|United States|-8|condo| +30945|AAAAAAAABOIHAAAA|609|4th |Circle|Suite 50|Colfax|Granite County|MT|62565|United States|-7|single family| +30946|AAAAAAAACOIHAAAA|375|Fourth |Lane|Suite R|Hamilton|Winnebago County|WI|52808|United States|-6|single family| +30947|AAAAAAAADOIHAAAA|493|Hickory 3rd|Ct.|Suite K|New Town|Indiana County|PA|19634|United States|-5|single family| +30948|AAAAAAAAEOIHAAAA|844|Spruce |Avenue|Suite J|Hamilton|Hamilton County|FL|32808|United States|-5|apartment| +30949|AAAAAAAAFOIHAAAA|526|Miller Woodland|Blvd|Suite E|Unionville|Merrimack County|NH|02311|United States|-5|apartment| +30950|AAAAAAAAGOIHAAAA|161|Park |Dr.|Suite E|Lincoln|Herkimer County|NY|11289|United States|-5|single family| +30951|AAAAAAAAHOIHAAAA|854|Second |Drive|Suite W|Centerville|Sherman County|OR|90059|United States|-8|apartment| +30952|AAAAAAAAIOIHAAAA|836|8th |Way|Suite N|Wildwood|Monroe County|AR|76871|United States|-6|single family| +30953|AAAAAAAAJOIHAAAA|346|Broadway |Parkway|Suite D|Wayland|Tuscarawas County|OH|45115|United States|-5|condo| +30954|AAAAAAAAKOIHAAAA|85|9th 8th|Dr.|Suite 20|Webb|Bayfield County|WI|50899|United States|-6|apartment| +30955|AAAAAAAALOIHAAAA|390|Lakeview Birch|Circle|Suite 480|Woodruff|Uinta County|WY|84174|United States|-7|apartment| +30956|AAAAAAAAMOIHAAAA|864|11th |Street|Suite 340|Lakeside|Lafayette County|MO|69532|United States|-6|condo| +30957|AAAAAAAANOIHAAAA|700|5th |Ave|Suite 70|Shiloh|Green County|WI|59275|United States|-6|single family| +30958|AAAAAAAAOOIHAAAA|92|Lincoln Jackson|ST|Suite 10|Shady Grove|Holt County|NE|62812|United States|-7|apartment| +30959|AAAAAAAAPOIHAAAA|833|Lake |Parkway|Suite Y|Shady Grove|Rich County|UT|82812|United States|-7|single family| +30960|AAAAAAAAAPIHAAAA|773|Willow Walnut|Avenue|Suite B|Union|Evans County|GA|38721|United States|-5|single family| +30961|AAAAAAAABPIHAAAA|512|Hickory |Drive|Suite V|Hamilton|Union County|NC|22808|United States|-5|apartment| +30962|AAAAAAAACPIHAAAA|376|Pine Pine|Lane|Suite M|Jackson|Clay County|SD|59583|United States|-6|single family| +30963|AAAAAAAADPIHAAAA|428|Oak College|Ave|Suite 100|Pleasant Grove|Warren County|MS|54136|United States|-6|apartment| +30964|AAAAAAAAEPIHAAAA|109|Highland |Cir.|Suite 310|Lakeview|Hamilton County|IA|58579|United States|-6|single family| +30965|AAAAAAAAFPIHAAAA|621|Jackson Sixth|Pkwy|Suite 380|Youngstown|Greenwood County|KS|60001|United States|-6|condo| +30966|AAAAAAAAGPIHAAAA|17|Park Lee|Ln|Suite D|Lebanon|Kent County|RI|03498|United States|-5|apartment| +30967|AAAAAAAAHPIHAAAA|17|5th Washington|Ave|Suite T|Highland Park|Presque Isle County|MI|46534|United States|-5|apartment| +30968|AAAAAAAAIPIHAAAA|793|Ridge Hill|Cir.|Suite F|Rose Hill|Quay County|NM|82024|United States|-7|condo| +30969|AAAAAAAAJPIHAAAA|760|Broadway |Wy|Suite 50|Belmont|Washington County|OH|40191|United States|-5|apartment| +30970|AAAAAAAAKPIHAAAA|347|Forest 4th|Boulevard|Suite U|Riverdale|Gentry County|MO|69391|United States|-6|condo| +30971|AAAAAAAALPIHAAAA|643|Miller |Boulevard|Suite T|Union Hill|Hardin County|IL|67746|United States|-6|condo| +30972|AAAAAAAAMPIHAAAA|988|South |Pkwy|Suite 370|Hopewell|Clinton County|PA|10587|United States|-5|single family| +30973|AAAAAAAANPIHAAAA|161|Valley |Boulevard|Suite Y|Mount Zion|Wayne County|MS|58054|United States|-6|condo| +30974|AAAAAAAAOPIHAAAA|663|First |Ln|Suite 350|Greenwood|Pitkin County|CO|88828|United States|-7|apartment| +30975|AAAAAAAAPPIHAAAA|568|Forest |Circle|Suite 110|Maple Grove|Lavaca County|TX|78252|United States|-6|single family| +30976|AAAAAAAAAAJHAAAA|967|Oak |Ct.|Suite C|Fairfield|Waushara County|WI|56192|United States|-6|apartment| +30977|AAAAAAAABAJHAAAA|616|Cherry ||Suite H|Bridgeport|Burnet County|TX|||-6|apartment| +30978|AAAAAAAACAJHAAAA|45|Johnson 2nd|Street|Suite P|Milo|Jones County|MS|50116|United States|-6|apartment| +30979|AAAAAAAADAJHAAAA|142|Jackson Lake|Parkway|Suite T|Greenwood|White Pine County|NV|88828|United States|-8|apartment| +30980|AAAAAAAAEAJHAAAA|652|Fifth |Ave|Suite M|Pleasant Valley|Mercer County|IL|62477|United States|-6|apartment| +30981|AAAAAAAAFAJHAAAA|701|4th Fifteenth|Ct.|Suite J|Harmony|Payne County|OK|75804|United States|-6|condo| +30982|AAAAAAAAGAJHAAAA|680|Birch |Road|Suite J|Woodville|West Carroll Parish|LA|74289|United States|-6|condo| +30983|AAAAAAAAHAJHAAAA|265|11th Jackson|Pkwy|Suite 20|Oak Ridge|Cass County|MN|58371|United States|-6|condo| +30984|AAAAAAAAIAJHAAAA|859|Cedar |Parkway|Suite 250|Fairfield|San Patricio County|TX|76192|United States|-6|condo| +30985|AAAAAAAAJAJHAAAA|551|Hill |Circle|Suite 60|Newport|Asotin County|WA|91521|United States|-8|apartment| +30986|AAAAAAAAKAJHAAAA|4|Williams Pine|Circle|Suite K|Mount Pleasant|Oldham County|TX|71933|United States|-6|single family| +30987|AAAAAAAALAJHAAAA|102|Maple |Dr.|Suite Y|Wolf Creek|Bailey County|TX|72455|United States|-6|single family| +30988|AAAAAAAAMAJHAAAA|103|Willow |Pkwy|Suite 150|Springfield|Garfield County|CO|89303|United States|-7|apartment| +30989|AAAAAAAANAJHAAAA|296|Maple Adams|Lane|Suite S|New Hope|Logan County|WV|29431|United States|-5|single family| +30990|AAAAAAAAOAJHAAAA|642|Locust Walnut|Pkwy|Suite 470|Bunker Hill|Lubbock County|TX|70150|United States|-6|apartment| +30991|AAAAAAAAPAJHAAAA|277|Green Washington|Wy|Suite C|Hillcrest|Putnam County|FL|33003|United States|-5|apartment| +30992|AAAAAAAAABJHAAAA|204|Adams |Parkway|Suite Q|Oakdale|Chesterfield County|VA|29584|United States|-5|condo| +30993|AAAAAAAABBJHAAAA|504|9th 5th|RD|Suite W|Shiloh|Richland County|WI|59275|United States|-6|single family| +30994|AAAAAAAACBJHAAAA|||Pkwy||Newtown|||41749||-5|apartment| +30995|AAAAAAAADBJHAAAA|561|Church Locust|Street|Suite 330|Oak Hill|Kittson County|MN|57838|United States|-6|condo| +30996|AAAAAAAAEBJHAAAA|456|10th Madison|Parkway|Suite 440|Mount Olive|Caldwell County|NC|28059|United States|-5|single family| +30997|AAAAAAAAFBJHAAAA|148|4th |RD|Suite 450|Greenwood|Sussex County|VA|28828|United States|-5|single family| +30998|AAAAAAAAGBJHAAAA|738|South Laurel|Boulevard|Suite N|Sulphur Springs|Dickinson County|MI|48354|United States|-5|apartment| +30999|AAAAAAAAHBJHAAAA|91|View |Street|Suite 10|Cedar Grove|Crittenden County|KY|40411|United States|-6|condo| +31000|AAAAAAAAIBJHAAAA|726|Hickory |Parkway|Suite D|Hawthorne|Clermont County|OH|49959|United States|-5|condo| +31001|AAAAAAAAJBJHAAAA|845|Railroad |Ave|Suite V|White Oak|McLean County|KY|46668|United States|-5|single family| +31002|AAAAAAAAKBJHAAAA|654|Forest View|Cir.|Suite 80|Sunnyside|Greenville County|SC|21952|United States|-5|apartment| +31003|AAAAAAAALBJHAAAA|785|Adams |Ln|Suite A|Edgewood|Logan County|ND|50069|United States|-6|apartment| +31004|AAAAAAAAMBJHAAAA|||Court|||Custer County|CO|||-7|| +31005|AAAAAAAANBJHAAAA|175|Cherry River|ST|Suite 450|Five Forks|Copiah County|MS|52293|United States|-6|single family| +31006|AAAAAAAAOBJHAAAA|61|4th |RD|Suite O|Concord|Coke County|TX|74107|United States|-6|apartment| +31007|AAAAAAAAPBJHAAAA|881|Sycamore Sycamore|Parkway|Suite 130|Bunker Hill|Halifax County|NC|20150|United States|-5|apartment| +31008|AAAAAAAAACJHAAAA|13|Washington |Way|Suite 150|Ferguson|Floyd County|IA|51821|United States|-6|single family| +31009|AAAAAAAABCJHAAAA|406|Lee Center|RD|Suite 90|Five Points|Wabash County|IL|66098|United States|-6|condo| +31010|AAAAAAAACCJHAAAA|427|Fourth Davis|Court|Suite W|Oakland|Essex County|MA|09843|United States|-5|apartment| +31011|AAAAAAAADCJHAAAA|715|Ninth |Avenue|Suite Q|Glendale|Macon County|IL|63951|United States|-6|single family| +31012|AAAAAAAAECJHAAAA|794|7th |Boulevard|Suite 270|Morris|Eaton County|MI|46696|United States|-5|single family| +31013|AAAAAAAAFCJHAAAA|336|10th Locust|Ct.|Suite 400|Highland Park|Saguache County|CO|86534|United States|-7|single family| +31014|AAAAAAAAGCJHAAAA|380|East Franklin|Dr.|Suite I|Sheffield|Ottawa County|OH|46896|United States|-5|single family| +31015|AAAAAAAAHCJHAAAA|576|Chestnut Spring|Cir.|Suite 430|Summit|Milwaukee County|WI|50499|United States|-6|single family| +31016|AAAAAAAAICJHAAAA|445|Madison |ST|Suite G|Walnut Grove|Washington County|ME|08352|United States|-5|condo| +31017|AAAAAAAAJCJHAAAA|44|1st |Court|Suite 160|Riverview|Iron County|MI|49003|United States|-5|condo| +31018|AAAAAAAAKCJHAAAA|701|Pine |Ln|Suite A|Oak Ridge|Tipton County|TN|38371|United States|-6|condo| +31019|AAAAAAAALCJHAAAA|826|Mill |Lane|Suite 470|Crossroads|Robeson County|NC|20534|United States|-5|apartment| +31020|AAAAAAAAMCJHAAAA|331|Dogwood |ST|Suite D|Summit|Kenai Peninsula Borough|AK|90499|United States|-9|apartment| +31021|AAAAAAAANCJHAAAA|260|Cedar |Ave|Suite 290|Sunnyside|Steele County|ND|51952|United States|-6|condo| +31022|AAAAAAAAOCJHAAAA|868|Seventh Second|Boulevard|Suite 200|Wilson|Benzie County|MI|46971|United States|-5|single family| +31023|AAAAAAAAPCJHAAAA|141|Elevnth |Dr.|Suite 410|Fairfield|Dickens County|TX|76192|United States|-6|single family| +31024|AAAAAAAAADJHAAAA|878|12th |Blvd|Suite I|Pine Grove|Dawson County|TX|74593|United States|-6|condo| +31025|AAAAAAAABDJHAAAA|173|4th Pine|Circle|Suite D|Griffin|Chilton County|AL|31204|United States|-6|apartment| +31026|AAAAAAAACDJHAAAA|792|Spruce |Road|Suite K|Sulphur Springs|Grant County|NE|68354|United States|-6|condo| +31027|AAAAAAAADDJHAAAA|378|Ridge Mill|Street|Suite J|Woodlawn|Floyd County|TX|74098|United States|-6|apartment| +31028|AAAAAAAAEDJHAAAA|723|Washington 8th|Drive|Suite 80|Woodville|Slope County|ND|54289|United States|-6|apartment| +31029|AAAAAAAAFDJHAAAA|139|South |Road|Suite V|Hopewell|Monroe County|PA|10587|United States|-5|apartment| +31030|AAAAAAAAGDJHAAAA|357|Lee 9th|Avenue|Suite 100|Lebanon|Burnett County|WI|52898|United States|-6|apartment| +31031|AAAAAAAAHDJHAAAA|189|Maple |Ln|Suite 220|Wilson|Charles City County|VA|26971|United States|-5|apartment| +31032|AAAAAAAAIDJHAAAA|320|Railroad |Blvd|Suite V|Hamilton|Josephine County|OR|92808|United States|-8|apartment| +31033|AAAAAAAAJDJHAAAA|461|Highland |Blvd|Suite N|Highland Park|Callahan County|TX|76534|United States|-6|single family| +31034|AAAAAAAAKDJHAAAA|63|Madison |Boulevard|Suite 190|Elkton|McHenry County|IL|63481|United States|-6|condo| +31035|AAAAAAAALDJHAAAA|839|Park |Ln|Suite 20|Cedar Grove|Lee County|IA|50411|United States|-6|condo| +31036|AAAAAAAAMDJHAAAA|708|9th Locust|Wy|Suite D|Friendship|Plymouth County|IA|54536|United States|-6|condo| +31037|AAAAAAAANDJHAAAA|642|Railroad Franklin|Blvd|Suite M|Nottingham|Catoosa County|GA|34074|United States|-5|single family| +31038|AAAAAAAAODJHAAAA|791|Adams |Court|Suite 0|Colfax|Prince George County|VA|22565|United States|-5|condo| +31039|AAAAAAAAPDJHAAAA|828|Hill Pine|Circle|Suite 90|Lakeside|Iron County|MO|69532|United States|-6|apartment| +31040|AAAAAAAAAEJHAAAA|513|Fourth 2nd|Street|Suite G|Oakland|Lincoln County|WV|29843|United States|-5|single family| +31041|AAAAAAAABEJHAAAA|514|Walnut |ST|Suite O|Riverdale|Coweta County|GA|39391|United States|-5|condo| +31042|AAAAAAAACEJHAAAA|320|1st Elm|Dr.|Suite L|Union Hill|Alexander County|NC|27746|United States|-5|apartment| +31043|AAAAAAAADEJHAAAA|649|Jackson Central|Wy|Suite Q|Woodville|Brantley County|GA|34289|United States|-5|condo| +31044|AAAAAAAAEEJHAAAA|366|View |Parkway|Suite 400|Macedonia|Washita County|OK|71087|United States|-6|single family| +31045|AAAAAAAAFEJHAAAA|575|Fourteenth 2nd|Cir.|Suite 360|Fairview|Yolo County|CA|95709|United States|-8|single family| +31046|AAAAAAAAGEJHAAAA|||||Wilson|Allen Parish|LA|76971||-6|condo| +31047|AAAAAAAAHEJHAAAA|332|Madison |Road|Suite 220|Blair|Fall River County|SD|55465|United States|-6|condo| +31048|AAAAAAAAIEJHAAAA|865|Cedar |Avenue|Suite T|Georgetown|Florence County|SC|27057|United States|-5|apartment| +31049|AAAAAAAAJEJHAAAA|700|Woodland |Pkwy|Suite 140|Hamilton|Beaufort County|SC|22808|United States|-5|apartment| +31050|AAAAAAAAKEJHAAAA|692|Thirteenth Main|Avenue|Suite S|Oak Hill|Claiborne Parish|LA|77838|United States|-6|condo| +31051|AAAAAAAALEJHAAAA|864|College Willow|Drive|Suite L|Lakeside|Smith County|TN|39532|United States|-6|apartment| +31052|AAAAAAAAMEJHAAAA|744|5th |Parkway|Suite 30|Redland|Roosevelt County|NM|86343|United States|-7|apartment| +31053|AAAAAAAANEJHAAAA|815|8th |Way|Suite 110|Riverdale|Worth County|GA|39391|United States|-5|single family| +31054|AAAAAAAAOEJHAAAA|284|Hill |RD|Suite 90|Waterloo|Mercer County|ND|51675|United States|-6|single family| +31055|AAAAAAAAPEJHAAAA|594|South |Dr.|Suite L|Deerfield|Walton County|GA|39840|United States|-5|condo| +31056|AAAAAAAAAFJHAAAA|146|5th Birch|Pkwy|Suite M|Franklin|Presidio County|TX|79101|United States|-6|apartment| +31057|AAAAAAAABFJHAAAA|555|Second 14th|Boulevard|Suite 310|Liberty|Clay County|NC|23451|United States|-5|condo| +31058|AAAAAAAACFJHAAAA|342|Jefferson South|Ln|Suite 410|Spring Hill|Luzerne County|PA|16787|United States|-5|single family| +31059|AAAAAAAADFJHAAAA|408|Sunset |Boulevard|Suite 450|Marion|Lee County|KY|40399|United States|-5|apartment| +31060|AAAAAAAAEFJHAAAA|58|Central |Street|Suite F|Pleasant Valley|Cass County|MN|52477|United States|-6|single family| +31061|AAAAAAAAFFJHAAAA|1000|Spruce Forest|Avenue|Suite E|Mount Olive|Ohio County|IN|48059|United States|-5|condo| +31062|AAAAAAAAGFJHAAAA|746|Smith Fourth|Street|Suite K|Ashland|Bladen County|NC|24244|United States|-5|condo| +31063|AAAAAAAAHFJHAAAA|143|Jefferson |Street|Suite J|Red Hill|Schoharie County|NY|14338|United States|-5|condo| +31064|AAAAAAAAIFJHAAAA|915|Second |Ln|Suite Q|New Hope|Wood County|WI|59431|United States|-6|condo| +31065|AAAAAAAAJFJHAAAA|95|Sunset |Ln|Suite 40|Elkton|Osage County|MO|63481|United States|-6|single family| +31066|AAAAAAAAKFJHAAAA|124|Washington |RD|Suite I|Enterprise|Madison County|TN|31757|United States|-6|condo| +31067|AAAAAAAALFJHAAAA|17|Spruce |Lane|Suite 280|Ellisville|Potter County|SD|56820|United States|-7|condo| +31068|AAAAAAAAMFJHAAAA|879|5th Ridge|Parkway|Suite 210|Florence|Shelby County|KY|43394|United States|-5|single family| +31069|AAAAAAAANFJHAAAA|395|6th Oak|Cir.|Suite M|Plainview|Rock County|WI|53683|United States|-6|condo| +31070|AAAAAAAAOFJHAAAA|789|Adams |Circle|Suite 50|Indian Village|Zavala County|TX|71075|United States|-6|apartment| +31071|AAAAAAAAPFJHAAAA|945|Main Birch|Ln|Suite 70|Friendship|Grant County|MN|54536|United States|-6|condo| +31072|AAAAAAAAAGJHAAAA|528|Main Hickory|Avenue|Suite A|Mount Pleasant|Adams County|CO|81933|United States|-7|single family| +31073|AAAAAAAABGJHAAAA|857|Walnut Willow|Cir.|Suite 30|Spring Valley|Woodbury County|IA|56060|United States|-6|apartment| +31074|AAAAAAAACGJHAAAA|860|Poplar First|Boulevard|Suite T|Clifton|Torrance County|NM|88014|United States|-7|single family| +31075|AAAAAAAADGJHAAAA|88|3rd |Ln|Suite N|Highland Park|Jackson Parish|LA|76534|United States|-6|condo| +31076|AAAAAAAAEGJHAAAA|959|Sixth 5th|Street|Suite 0|Marion|Warren County|IN|40399|United States|-5|apartment| +31077|AAAAAAAAFGJHAAAA|517|Hill South|Wy|Suite 90|Shady Grove|Benton County|MO|62812|United States|-6|condo| +31078|AAAAAAAAGGJHAAAA|641|Maple |RD|Suite 220|Willis|Halifax County|VA|26788|United States|-5|condo| +31079|AAAAAAAAHGJHAAAA|89|Lake Maple|Avenue|Suite X|Greenwood|Jones County|IA|58828|United States|-6|single family| +31080|AAAAAAAAIGJHAAAA|896|Park Jefferson|Drive|Suite 360|Antioch|Marin County|CA|98605|United States|-8|condo| +31081|AAAAAAAAJGJHAAAA|889|Pine Adams|Way|Suite P|Macedonia|Cass County|MN|51087|United States|-6|apartment| +31082|AAAAAAAAKGJHAAAA|648|Second |Lane|Suite K|Enterprise|Upshur County|TX|71757|United States|-6|single family| +31083|AAAAAAAALGJHAAAA|147|Lake Hill|Circle|Suite K|Brownsville|Yakima County|WA|99310|United States|-8|apartment| +31084|AAAAAAAAMGJHAAAA|420|Jackson Lake|Blvd|Suite R|Lakeside|Haakon County|SD|59532|United States|-7|single family| +31085|AAAAAAAANGJHAAAA|25|2nd River|Court|Suite 200|Fairfield|Evangeline Parish|LA|76192|United States|-6|apartment| +31086|AAAAAAAAOGJHAAAA|22|First |Court|Suite 380|Riverview|Roger Mills County|OK|79003|United States|-6|apartment| +31087|AAAAAAAAPGJHAAAA|163|4th |Circle|Suite 420|Tipton|Oconee County|SC|28773|United States|-5|condo| +31088|AAAAAAAAAHJHAAAA||Elm 5th|Pkwy|Suite M|Bunker Hill|Phillips County||60150|||| +31089|AAAAAAAABHJHAAAA|336|Poplar Highland|RD|Suite 380|Oakdale|Buffalo County|NE|69584|United States|-6|single family| +31090|AAAAAAAACHJHAAAA|393|Mill Lincoln|Parkway|Suite 140|Pine Grove|Phillips County|AR|74593|United States|-6|single family| +31091|AAAAAAAADHJHAAAA|357|College Cherry|Road|Suite 340|Fairfield|Clay County|WV|26192|United States|-5|single family| +31092|AAAAAAAAEHJHAAAA|294|Madison Walnut|Road|Suite E|Glenwood|Baxter County|AR|73511|United States|-6|apartment| +31093|AAAAAAAAFHJHAAAA|713|Valley |Circle|Suite Y|Franklin|Fulton County|OH|49101|United States|-5|apartment| +31094|AAAAAAAAGHJHAAAA|266|Walnut |Boulevard|Suite 460|Deerfield|Kendall County|IL|69840|United States|-6|condo| +31095|AAAAAAAAHHJHAAAA|750|River Railroad|Ave|Suite 20|New Hope|Trego County|KS|69431|United States|-6|apartment| +31096|AAAAAAAAIHJHAAAA|822|||Suite T|Riverdale|New York County|NY|19391|||single family| +31097|AAAAAAAAJHJHAAAA|847|Wilson Jefferson|Ave|Suite 80|Franklin|Washington County|NY|19101|United States|-5|apartment| +31098|AAAAAAAAKHJHAAAA|500|2nd |Cir.|Suite 430|Lakewood|Whitley County|KY|48877|United States|-5|single family| +31099|AAAAAAAALHJHAAAA|148|Oak |Ln|Suite 150|Walnut Grove|Emery County|UT|87752|United States|-7|condo| +31100|AAAAAAAAMHJHAAAA|512|Spring |Wy|Suite L|Newtown|Forest County|PA|11749|United States|-5|apartment| +31101|AAAAAAAANHJHAAAA|504|College 4th|Lane|Suite P|Crescent|Essex County|VT|02003|United States|-5|condo| +31102|AAAAAAAAOHJHAAAA|145|Franklin Chestnut|Pkwy|Suite 80|Unionville|Union County|SC|21711|United States|-5|condo| +31103|AAAAAAAAPHJHAAAA|690|6th |Drive|Suite 350|Farmington|Hale County|TX|79145|United States|-6|condo| +31104|AAAAAAAAAIJHAAAA|289|Church Adams|Court|Suite T|Summit|Trempealeau County|WI|50499|United States|-6|apartment| +31105|AAAAAAAABIJHAAAA|744|Church |Drive|Suite X|Clinton|Beaver County|UT|88222|United States|-7|single family| +31106|AAAAAAAACIJHAAAA|103|East |Ave|Suite W|Glendale|Chickasaw County|MS|53951|United States|-6|condo| +31107|AAAAAAAADIJHAAAA|61|Hickory Third|Way|Suite 480|Frogtown|Pembina County|ND|58784|United States|-6|apartment| +31108|AAAAAAAAEIJHAAAA|862|Green |Ct.|Suite 270|Bethel|Brown County|KS|65281|United States|-6|single family| +31109|AAAAAAAAFIJHAAAA|877|Park Lakeview||Suite 10|||MO|||-6|single family| +31110|AAAAAAAAGIJHAAAA|66|Ninth |Cir.|Suite L|Newtown|Pulaski County|GA|31749|United States|-5|single family| +31111|AAAAAAAAHIJHAAAA|849|Maple Washington|Dr.|Suite 170|Five Points|Grenada County|MS|56098|United States|-6|apartment| +31112|AAAAAAAAIIJHAAAA|333|Meadow 9th|Road|Suite 300|Florence|Pickaway County|OH|43394|United States|-5|apartment| +31113|AAAAAAAAJIJHAAAA|255|Walnut Spruce|Ave|Suite 260|Liberty|Coffee County|AL|33451|United States|-6|condo| +31114|AAAAAAAAKIJHAAAA|782|Third |Lane|Suite H|Concord|Alleghany County|NC|24107|United States|-5|apartment| +31115|AAAAAAAALIJHAAAA|961|East |Cir.|Suite K|Lakeview|Stanly County|NC|28579|United States|-5|apartment| +31116|AAAAAAAAMIJHAAAA|455|Johnson |Dr.|Suite 190|Plainview|Sitka Borough|AK|93683|United States|-9|condo| +31117|AAAAAAAANIJHAAAA|900|Willow |Cir.|Suite 110|Highland Park|Santa Cruz County|CA|96534|United States|-8|condo| +31118|AAAAAAAAOIJHAAAA|372|East Park|Circle|Suite X|Riverdale|Newberry County|SC|29391|United States|-5|condo| +31119|AAAAAAAAPIJHAAAA||||Suite 420||Johnson County|GA|37752|||| +31120|AAAAAAAAAJJHAAAA|133|Forest |Boulevard|Suite 220|Spring Valley|Northwest Arctic Borough|AK|96060|United States|-9|single family| +31121|AAAAAAAABJJHAAAA|553|Mill |Blvd|Suite 80|Sunnyside|Preston County|WV|21952|United States|-5|condo| +31122|AAAAAAAACJJHAAAA|164|Sixth |Drive|Suite W|Riverview|Deaf Smith County|TX|79003|United States|-6|apartment| +31123|AAAAAAAADJJHAAAA|41|Maple Washington|Parkway|Suite 360|Newport|Weakley County|TN|31521|United States|-6|apartment| +31124|AAAAAAAAEJJHAAAA|42|Jackson |Cir.|Suite A|Five Forks|Laclede County|MO|62293|United States|-6|single family| +31125|AAAAAAAAFJJHAAAA|178|Oak |Lane|Suite D|Five Forks|Gregory County|SD|52293|United States|-6|condo| +31126|AAAAAAAAGJJHAAAA|185|Oak |Blvd|Suite 440|Springdale|Wade Hampton Census Area|AK|98883|United States|-9|condo| +31127|AAAAAAAAHJJHAAAA|310|East Main|Pkwy|Suite I|Springdale|Muhlenberg County|KY|48883|United States|-5|condo| +31128|AAAAAAAAIJJHAAAA|249|Jefferson |Pkwy|Suite 70|Edgewood|Uintah County|UT|80069|United States|-7|apartment| +31129|AAAAAAAAJJJHAAAA|992|Tenth |Road|Suite 70|Mount Vernon|Sangamon County|IL|68482|United States|-6|condo| +31130|AAAAAAAAKJJHAAAA|547|Lincoln |Drive|Suite 400|Belmont|Bourbon County|KS|60191|United States|-6|apartment| +31131|AAAAAAAALJJHAAAA|369|Railroad |ST|Suite 0|Marion|Fairfax County|VA|20399|United States|-5|condo| +31132|AAAAAAAAMJJHAAAA|||Ct.|Suite 370|Unionville|Kendall County|||United States|-6|| +31133|AAAAAAAANJJHAAAA|306|Walnut |Dr.|Suite 460|Georgetown|Washington County|KY|47057|United States|-5|apartment| +31134|AAAAAAAAOJJHAAAA|541|Spruce Pine|Wy|Suite 100|Oak Ridge|Rains County|TX|78371|United States|-6|apartment| +31135|AAAAAAAAPJJHAAAA||Birch |||Lakewood||HI|||-10|single family| +31136|AAAAAAAAAKJHAAAA|139|Willow Maple|Ave|Suite I|Deerfield|Duval County|TX|79840|United States|-6|apartment| +31137|AAAAAAAABKJHAAAA|618|Second Jefferson|Road|Suite 70|Gravel Hill|Henderson County|NC|21944|United States|-5|condo| +31138|AAAAAAAACKJHAAAA|825|Ridge Cedar|Road|Suite 10|Valley View|Oxford County|ME|05724|United States|-5|single family| +31139|AAAAAAAADKJHAAAA|445|Seventh |Parkway|Suite 100|Bethel|San Jacinto County|TX|75281|United States|-6|condo| +31140|AAAAAAAAEKJHAAAA|969|First |Road|Suite K|Woodland|Sauk County|WI|54854|United States|-6|condo| +31141|AAAAAAAAFKJHAAAA|411|3rd |Parkway|Suite 210|Shiloh|Red Willow County|NE|69275|United States|-7|single family| +31142|AAAAAAAAGKJHAAAA|18|Sunset Second|Way|Suite 470|Sunnyside|Jefferson County|ID|81952|United States|-7|single family| +31143|AAAAAAAAHKJHAAAA|441|West |Avenue|Suite 480|Stringtown|Washington County|CO|80162|United States|-7|single family| +31144|AAAAAAAAIKJHAAAA|842|Ridge |Drive|Suite 110|Centerville|Cullman County|AL|30059|United States|-6|single family| +31145|AAAAAAAAJKJHAAAA|859|Lake Chestnut|Lane|Suite 160|Salem|Dale County|AL|38048|United States|-6|single family| +31146|AAAAAAAAKKJHAAAA|436|Johnson |Circle|Suite W|Ellsworth|Waldo County|ME|05679|United States|-5|apartment| +31147|AAAAAAAALKJHAAAA|884|Williams 4th|Cir.|Suite 350|Maple Grove|Sheridan County|MT|68252|United States|-7|apartment| +31148|AAAAAAAAMKJHAAAA|748|Hillcrest |Ave|Suite 430|Fairfield|Columbus County|NC|26192|United States|-5|single family| +31149|AAAAAAAANKJHAAAA|150|Jackson |RD|Suite 290|Ashland|Walton County|GA|34244|United States|-5|single family| +31150|AAAAAAAAOKJHAAAA|702|Lee Mill|Dr.|Suite 440|Valley View|Perry County|AL|35124|United States|-6|single family| +31151|AAAAAAAAPKJHAAAA|779|Washington |Lane|Suite O|Bethel|Sumner County|KS|65281|United States|-6|single family| +31152|AAAAAAAAALJHAAAA|730|Tenth 3rd|Pkwy|Suite 200|Bunker Hill|Bourbon County|KY|40150|United States|-6|apartment| +31153|AAAAAAAABLJHAAAA|377|Third |Lane|Suite 100|Sardis|Cass County|TX|72124|United States|-6|condo| +31154|AAAAAAAACLJHAAAA|564|View |Street|Suite Y|Bridgeport|McIntosh County|ND|55817|United States|-6|apartment| +31155|AAAAAAAADLJHAAAA|131|10th |Blvd|Suite J|Harmony|Tuscarawas County|OH|45804|United States|-5|single family| +31156|AAAAAAAAELJHAAAA|479|Park Lake|Drive|Suite 150|Silver Creek|Jefferson County|FL|34546|United States|-5|single family| +31157|AAAAAAAAFLJHAAAA|982|Church |Dr.|Suite B|Centerville|Larue County|KY|40059|United States|-5|condo| +31158|AAAAAAAAGLJHAAAA|835|Ridge 9th|Dr.|Suite F|Newtown|Tama County|IA|51749|United States|-6|apartment| +31159|AAAAAAAAHLJHAAAA|629|Tenth |Ct.|Suite N|Forest Hills|La Salle County|TX|79237|United States|-6|apartment| +31160|AAAAAAAAILJHAAAA|966|Hickory |Ct.|Suite 380|Pine Grove|Lake County|CA|94593|United States|-8|condo| +31161|AAAAAAAAJLJHAAAA|85|Fifth |Ct.|Suite 190|Harmony|Blue Earth County|MN|55804|United States|-6|condo| +31162|AAAAAAAAKLJHAAAA|887|1st |Boulevard|Suite T|Ellsworth|Bristol County|MA|05679|United States|-5|single family| +31163|AAAAAAAALLJHAAAA|635||Ct.|Suite 430||||40162||-5|single family| +31164|AAAAAAAAMLJHAAAA|359|Adams |Way|Suite Q|Woodlawn|Webb County|TX|74098|United States|-6|condo| +31165|AAAAAAAANLJHAAAA|388|College Railroad|Avenue|Suite G|Lincoln|Washington County|MD|21289|United States|-5|single family| +31166|AAAAAAAAOLJHAAAA|814|Central |Ave|Suite 280|Unionville|Kidder County|ND|51711|United States|-6|apartment| +31167|AAAAAAAAPLJHAAAA|586|Jackson |Road|Suite I|Stringtown|Douglas County|MO|60162|United States|-6|apartment| +31168|AAAAAAAAAMJHAAAA|891||Blvd|Suite 310|Spring Hill|Utah County|||United States||| +31169|AAAAAAAABMJHAAAA|673|View River|Wy|Suite M|Farmington|Polk County|NE|69145|United States|-7|apartment| +31170|AAAAAAAACMJHAAAA|566|Second |Pkwy|Suite 40|Hamilton|Garrett County|MD|22808|United States|-5|condo| +31171|AAAAAAAADMJHAAAA|667|Lincoln |Dr.|Suite 150|Mount Vernon|Plymouth County|MA|09082|United States|-5|single family| +31172|AAAAAAAAEMJHAAAA|623|Park 9th|Dr.|Suite 270|Clifton|Hidalgo County|TX|78014|United States|-6|condo| +31173|AAAAAAAAFMJHAAAA|407|Cedar |Parkway|Suite 410|New Hope|Arthur County|NE|69431|United States|-6|single family| +31174|AAAAAAAAGMJHAAAA|872|1st Church|Dr.|Suite A|Woodland|Ketchikan Gateway Borough|AK|94854|United States|-9|single family| +31175|AAAAAAAAHMJHAAAA|676|Park |Ln|Suite V|Oakland|Hockley County|TX|79843|United States|-6|apartment| +31176|AAAAAAAAIMJHAAAA|228|Jackson 7th|Street|Suite 320|Fairfield|Humphreys County|MS|56192|United States|-6|single family| +31177|AAAAAAAAJMJHAAAA|212|Franklin Park|RD|Suite C|Bear Creek|Island County|WA|93075|United States|-8|condo| +31178|AAAAAAAAKMJHAAAA|639|14th |Drive|Suite 400|Montezuma|Stanton County|NE|62150|United States|-7|condo| +31179|AAAAAAAALMJHAAAA|514|Davis Lincoln|Pkwy|Suite E|Newtown|Merrick County|NE|61749|United States|-7|apartment| +31180|AAAAAAAAMMJHAAAA|768|Lake Fifth|Parkway|Suite D|Harmony|Marion County|WV|25804|United States|-5|apartment| +31181|AAAAAAAANMJHAAAA|9|Johnson West|ST|Suite Q|Greenwood|Montgomery County|NC|28828|United States|-5|condo| +31182|AAAAAAAAOMJHAAAA|733|Lincoln West|Blvd|Suite X|Lakeview|Hardin County|OH|48579|United States|-5|apartment| +31183|AAAAAAAAPMJHAAAA|902|West Jackson|Dr.|Suite S|Unionville|Grant Parish|LA|71711|United States|-6|single family| +31184|AAAAAAAAANJHAAAA|296|Sycamore Tenth|Road|Suite P|Thompsonville|Mingo County|WV|29651|United States|-5|condo| +31185|AAAAAAAABNJHAAAA|464|8th |Ct.|Suite A|Five Points|Wetzel County|WV|26098|United States|-5|apartment| +31186|AAAAAAAACNJHAAAA|726|Willow |Street|Suite V|Providence|Ottawa County|OH|46614|United States|-5|condo| +31187|AAAAAAAADNJHAAAA|777|14th Miller|Street|Suite W|Golden|Fayette County|PA|10411|United States|-5|single family| +31188|AAAAAAAAENJHAAAA|974|Oak |Lane|Suite J|Brentwood|Coffee County|GA|34188|United States|-5|condo| +31189|AAAAAAAAFNJHAAAA|876|10th 4th|Wy|Suite F|Maple Grove|Buncombe County|NC|28252|United States|-5|single family| +31190|AAAAAAAAGNJHAAAA|186|Second |Parkway|Suite W|Forest Hills|Jones County|SD|59237|United States|-7|apartment| +31191|AAAAAAAAHNJHAAAA|69|Ridge Second|Street|Suite M|Ashland|Elliott County|KY|44244|United States|-6|apartment| +31192|AAAAAAAAINJHAAAA|||||Concord|||||-6|condo| +31193|AAAAAAAAJNJHAAAA|273|Jackson |Cir.|Suite T|Riverdale|Habersham County|GA|39391|United States|-5|condo| +31194|AAAAAAAAKNJHAAAA|642|South |Wy|Suite 70|New Hope|McIntosh County|OK|79431|United States|-6|single family| +31195|AAAAAAAALNJHAAAA|434|Williams Ash|Avenue|Suite S|Sulphur Springs|Pondera County|MT|68354|United States|-7|apartment| +31196|AAAAAAAAMNJHAAAA|861|West Highland|Street|Suite 340|Summit|Pettis County|MO|60499|United States|-6|single family| +31197|AAAAAAAANNJHAAAA|930|8th |Pkwy|Suite 360|Greenwood|Power County|ID|88828|United States|-7|apartment| +31198|AAAAAAAAONJHAAAA|124|Madison Hickory|Ln|Suite A|Springhill|Dillon County|SC|24602|United States|-5|single family| +31199|AAAAAAAAPNJHAAAA|919|Cherry |Dr.|Suite L|Stratford|Plumas County|CA|96668|United States|-8|apartment| +31200|AAAAAAAAAOJHAAAA|||ST|||Rappahannock County||23394|||apartment| +31201|AAAAAAAABOJHAAAA|489||Avenue||Arlington||||||| +31202|AAAAAAAACOJHAAAA|64|View Dogwood|Circle|Suite V|Union Hill|Banks County|GA|37746|United States|-5|single family| +31203|AAAAAAAADOJHAAAA|505|Main View|Wy|Suite X|Arlington|Allegany County|MD|26557|United States|-5|condo| +31204|AAAAAAAAEOJHAAAA|337|Second |Way|Suite S|Pine Grove|San Jacinto County|TX|74593|United States|-6|condo| +31205|AAAAAAAAFOJHAAAA|734||||Appleton|||54240|United States||| +31206|AAAAAAAAGOJHAAAA|23|6th |Ave|Suite D|Oak Ridge|Huron County|MI|48371|United States|-5|apartment| +31207|AAAAAAAAHOJHAAAA|408|Washington Elm||||||||-6|| +31208|AAAAAAAAIOJHAAAA|832|View |Dr.|Suite 490|Summit|Hardin County|KY|40499|United States|-6|condo| +31209|AAAAAAAAJOJHAAAA|908|Spring |Avenue|Suite R|Mount Vernon|Simpson County|KY|48482|United States|-5|single family| +31210|AAAAAAAAKOJHAAAA|812|Central Pine|Lane|Suite X|Clifton|Calloway County|KY|48014|United States|-6|condo| +31211|AAAAAAAALOJHAAAA|38|Lee |Street|Suite U|Forest Hills|Polk County|TN|39237|United States|-6|single family| +31212|AAAAAAAAMOJHAAAA|441|North |Way|Suite 0|Union|Wells County|IN|48721|United States|-5|single family| +31213|AAAAAAAANOJHAAAA|960|Third |RD|Suite X|Pleasant Grove|Ada County|ID|84136|United States|-7|single family| +31214|AAAAAAAAOOJHAAAA|231|Oak |Ct.|Suite 90|Paxton|Hempstead County|AR|75669|United States|-6|condo| +31215|AAAAAAAAPOJHAAAA|600|Third Main|Avenue|Suite R|Mountain View|Daniels County|MT|64466|United States|-7|apartment| +31216|AAAAAAAAAPJHAAAA|816|Main Cedar|Road|Suite 330|Jackson|Garden County|NE|69583|United States|-6|single family| +31217|AAAAAAAABPJHAAAA|930|7th Mill|Dr.|Suite 330|Antioch|Robeson County|NC|28605|United States|-5|single family| +31218|AAAAAAAACPJHAAAA||||||Quitman County||||-5|| +31219|AAAAAAAADPJHAAAA|673|West |||||WV||||apartment| +31220|AAAAAAAAEPJHAAAA|975|West 11th|Cir.|Suite 300|Wayland|Cheshire County|NH|05715|United States|-5|single family| +31221|AAAAAAAAFPJHAAAA|264|River |Street|Suite T|Deerfield|Lake County|TN|39840|United States|-6|condo| +31222|AAAAAAAAGPJHAAAA|272|Church Poplar|Lane|Suite W|Maple Grove|Pocahontas County|IA|58252|United States|-6|condo| +31223|AAAAAAAAHPJHAAAA|569|Lakeview |Boulevard|Suite C|Providence|Fillmore County|MN|56614|United States|-6|apartment| +31224|AAAAAAAAIPJHAAAA|992|Broadway 1st|Way|Suite Q|Riceville|Rio Arriba County|NM|85867|United States|-7|condo| +31225|AAAAAAAAJPJHAAAA|479|Washington Sycamore|Pkwy|Suite 120|Greenwood|Stark County|IL|68828|United States|-6|condo| +31226|AAAAAAAAKPJHAAAA|773|Oak |Wy|Suite 10|Forest Hills|McKinley County|NM|89237|United States|-7|apartment| +31227|AAAAAAAALPJHAAAA|31|Davis |Court|Suite 420|Lakeview|Searcy County|AR|78579|United States|-6|single family| +31228|AAAAAAAAMPJHAAAA|875|Third Main|Lane|Suite O|Wildwood|Rockingham County|NC|26871|United States|-5|condo| +31229|AAAAAAAANPJHAAAA|282|Main |Pkwy|Suite R|Hamilton|Washington County|AR|72808|United States|-6|single family| +31230|AAAAAAAAOPJHAAAA|682|6th |Circle|Suite 350|Shiloh|Aiken County|SC|29275|United States|-5|condo| +31231|AAAAAAAAPPJHAAAA|857|Park |Cir.|Suite P|Edgewood|Conway County|AR|70069|United States|-6|single family| +31232|AAAAAAAAAAKHAAAA|229|Washington |ST|Suite X|Lakeview|Dakota County|MN|58579|United States|-6|condo| +31233|AAAAAAAABAKHAAAA|75|2nd |Lane|Suite K|Riverdale|Taylor County|WI|59391|United States|-6|single family| +31234|AAAAAAAACAKHAAAA|218|Elevnth |Court|Suite T|Union Hill|Owsley County|KY|47746|United States|-5|single family| +31235|AAAAAAAADAKHAAAA|514|Second 2nd|Boulevard|Suite 10|Sunnyside|Dickenson County|VA|21952|United States|-5|apartment| +31236|AAAAAAAAEAKHAAAA|101|Oak |Drive|Suite 0|Mount Olive|Ottawa County|OH|48059|United States|-5|condo| +31237|AAAAAAAAFAKHAAAA|659|Ash Lake|Road|Suite 60|Lakewood|Newton County|MO|68877|United States|-6|condo| +31238|AAAAAAAAGAKHAAAA|29|Chestnut 7th|Court|Suite E|New Hope|Alexander County|NC|29431|United States|-5|single family| +31239|AAAAAAAAHAKHAAAA|999|2nd |Wy|Suite 270|Enterprise|Marshall County|KS|61757|United States|-6|apartment| +31240|AAAAAAAAIAKHAAAA|811|West 9th|Blvd|Suite 210|Lawrenceville|Licking County|OH|44462|United States|-5|apartment| +31241|AAAAAAAAJAKHAAAA|50|Laurel |Road|Suite Y|Pleasant Valley|Zapata County|TX|72477|United States|-6|single family| +31242|AAAAAAAAKAKHAAAA|850|Jefferson |Pkwy|Suite K|Wilson|Bledsoe County|TN|36971|United States|-5|apartment| +31243|AAAAAAAALAKHAAAA|104|Franklin |Wy|Suite X|Bunker Hill|Matanuska-Susitna Borough|AK|90150|United States|-9|single family| +31244|AAAAAAAAMAKHAAAA|380|Sunset Green|Dr.|Suite T|Clifford|Ocean County|NJ|08764|United States|-5|condo| +31245|AAAAAAAANAKHAAAA|329|College |Ct.|Suite 60|Shiloh|Lincoln County|MO|69275|United States|-6|condo| +31246|AAAAAAAAOAKHAAAA|191|10th South|Dr.|Suite 30|Shiloh|Frederick County|MD|29275|United States|-5|condo| +31247|AAAAAAAAPAKHAAAA|778|Poplar 2nd|Ct.|Suite I|Lakewood|Izard County|AR|78877|United States|-6|single family| +31248|AAAAAAAAABKHAAAA|742|Wilson Fifth|Lane|Suite 110|Mountain View|West Carroll Parish|LA|74466|United States|-6|condo| +31249|AAAAAAAABBKHAAAA|31|8th Hickory|Parkway|Suite T|Mount Olive|Nobles County|MN|58059|United States|-6|apartment| +31250|AAAAAAAACBKHAAAA|652|Lee |Dr.|Suite F|Sunnyside|Clermont County|OH|41952|United States|-5|single family| +31251|AAAAAAAADBKHAAAA|994|4th Meadow|Ln|Suite B|Woodland|Pima County|AZ|84854|United States|-7|single family| +31252|AAAAAAAAEBKHAAAA|565|River Green|Ln|Suite P|Kingston|Wise County|VA|24975|United States|-5|single family| +31253|AAAAAAAAFBKHAAAA|353|Laurel |Wy|Suite 200|Forest Hills|Georgetown County|SC|29237|United States|-5|single family| +31254|AAAAAAAAGBKHAAAA|783|Maple |Lane|Suite 330|Belmont|Troup County|GA|30191|United States|-5|condo| +31255|AAAAAAAAHBKHAAAA|202|3rd |Pkwy|Suite P|Clinton|Dona Ana County|NM|88222|United States|-7|apartment| +31256|AAAAAAAAIBKHAAAA|248|Maple Ninth|Street|Suite S|Harmony|Hancock County|MS|55804|United States|-6|condo| +31257|AAAAAAAAJBKHAAAA|247|Fourth Willow|Parkway|Suite 380|Oak Ridge|Madison County|NE|68371|United States|-7|single family| +31258|AAAAAAAAKBKHAAAA|959|Washington |Ln|Suite X|Georgetown|Caddo Parish|LA|77057|United States|-6|apartment| +31259|AAAAAAAALBKHAAAA|816|7th |Parkway|Suite U|Concord|Madera County|CA|94107|United States|-8|apartment| +31260|AAAAAAAAMBKHAAAA|289|7th Center|Road|Suite 0|Five Points|Owen County|IN|46098|United States|-5|apartment| +31261|AAAAAAAANBKHAAAA|787|Green 10th|Circle|Suite 460|Marion|Childress County|TX|70399|United States|-6|condo| +31262|AAAAAAAAOBKHAAAA|265|Cedar 1st|Circle|Suite T|Union|Elk County|PA|18721|United States|-5|single family| +31263|AAAAAAAAPBKHAAAA|648|Chestnut Park|Court|Suite 240|Mount Vernon|San Juan County|UT|88482|United States|-7|apartment| +31264|AAAAAAAAACKHAAAA|||Ln|Suite 430|||GA|30411|United States|-5|| +31265|AAAAAAAABCKHAAAA|802|Railroad |Ave|Suite N|Highland|De Soto Parish|LA|79454|United States|-6|single family| +31266|AAAAAAAACCKHAAAA|83|Sunset Jackson|Drive|Suite X|Centerville|San Juan County|NM|80059|United States|-7|apartment| +31267|AAAAAAAADCKHAAAA|353|Main Cedar|Parkway|Suite E|Lakeview|Pickett County|TN|38579|United States|-6|apartment| +31268|AAAAAAAAECKHAAAA|370|Hill Cherry|Ave|Suite R|Perkins|Butte County|ID|81852|United States|-7|apartment| +31269|AAAAAAAAFCKHAAAA|386|Pine Park|Blvd|Suite W|Forest Hills|Henry County|AL|39237|United States|-6|condo| +31270|AAAAAAAAGCKHAAAA|369|Seventh 11th|Lane|Suite W|Pine Valley|Grant County|MN|58209|United States|-6|condo| +31271|AAAAAAAAHCKHAAAA|59|Locust Meadow|ST|Suite 170|Highland|Sanders County|MT|69454|United States|-7|apartment| +31272|AAAAAAAAICKHAAAA|234|Eigth |Parkway|Suite L|Bunker Hill|Gaston County|NC|20150|United States|-5|single family| +31273|AAAAAAAAJCKHAAAA|388|Franklin |Wy|Suite 280|Clifton|Dallas County|TX|78014|United States|-6|single family| +31274|AAAAAAAAKCKHAAAA|331|Madison |Lane|Suite 240|Lakeside|Chattahoochee County|GA|39532|United States|-5|single family| +31275|AAAAAAAALCKHAAAA|741|Davis |Avenue|Suite 70|Kingston|Leelanau County|MI|44975|United States|-5|apartment| +31276|AAAAAAAAMCKHAAAA|225|Hill |Pkwy|Suite N|Plainview|Tyler County|TX|73683|United States|-6|single family| +31277|AAAAAAAANCKHAAAA|584|Willow |Pkwy|Suite M|Waterloo|Taney County|MO|61675|United States|-6|single family| +31278|AAAAAAAAOCKHAAAA|780|Second 8th|Cir.|Suite F|Pine Grove|Poweshiek County|IA|54593|United States|-6|condo| +31279|AAAAAAAAPCKHAAAA|199|15th |Ln|Suite Y|Brownsville|Roberts County|TX|79310|United States|-6|condo| +31280|AAAAAAAAADKHAAAA|376|Dogwood Railroad|Parkway|Suite 190|Forest Hills|Marion County|AL|39237|United States|-6|apartment| +31281|AAAAAAAABDKHAAAA|810|Oak Jackson|ST|Suite M|Centerville|Gloucester County|NJ|00659|United States|-5|single family| +31282|AAAAAAAACDKHAAAA|295|Mill |Wy|Suite A|Riverside|Marshall County|TN|39231|United States|-6|apartment| +31283|AAAAAAAADDKHAAAA|853|Davis Pine|Avenue|Suite V|Sunnyside|Pope County|MN|51952|United States|-6|condo| +31284|AAAAAAAAEDKHAAAA|780|1st |Circle|Suite G|Cedar Grove|Caldwell County|TX|70411|United States|-6|apartment| +31285|AAAAAAAAFDKHAAAA|94|Wilson Main|Ln|Suite Y|Five Forks|Dickenson County|VA|22293|United States|-5|single family| +31286|AAAAAAAAGDKHAAAA|985|Main Hickory|RD|Suite 80|Mount Zion|Pierce County|NE|68054|United States|-7|condo| +31287|AAAAAAAAHDKHAAAA|213|Seventh |Boulevard|Suite W|Pleasant Hill|Henry County|GA|33604|United States|-5|apartment| +31288|AAAAAAAAIDKHAAAA|556|Maple |Street|Suite T|Maple Grove|Logan County|IL|68252|United States|-6|condo| +31289|AAAAAAAAJDKHAAAA|798|Locust |Circle|Suite D|Macedonia|Onondaga County|NY|11087|United States|-5|apartment| +31290|AAAAAAAAKDKHAAAA||Main 12th|||Clifton|Caldwell County|||||apartment| +31291|AAAAAAAALDKHAAAA|23|East |Way|Suite 170|Riverview|White Pine County|NV|89003|United States|-8|condo| +31292|AAAAAAAAMDKHAAAA|854|2nd Elm|Ln|Suite R|Crossroads|Clay County|MO|60534|United States|-6|condo| +31293|AAAAAAAANDKHAAAA|488|Lincoln |Circle|Suite 80|Forest|Choctaw County|MS|57537|United States|-6|single family| +31294|AAAAAAAAODKHAAAA|213|Davis 2nd|Cir.|Suite O|Lebanon|Cheyenne County|KS|62898|United States|-6|condo| +31295|AAAAAAAAPDKHAAAA|89|View West|Dr.|Suite 150|Mountain View|Nodaway County|MO|64466|United States|-6|condo| +31296|AAAAAAAAAEKHAAAA|611|Forest Smith|Wy|Suite T|Liberty|Fannin County|GA|33451|United States|-5|condo| +31297|AAAAAAAABEKHAAAA|333|Elm |Dr.|Suite E|Tanglewood|Cross County|AR|78994|United States|-6|single family| +31298|AAAAAAAACEKHAAAA|794|Elm |Wy|Suite 350|Crossroads|Van Buren County|TN|30534|United States|-6|single family| +31299|AAAAAAAADEKHAAAA|300|Hill Jefferson|ST|Suite 370|Oakdale|Ector County|TX|79584|United States|-6|single family| +31300|AAAAAAAAEEKHAAAA|738|4th |Lane|Suite U|Maple Grove|Hancock County|IN|48252|United States|-5|condo| +31301|AAAAAAAAFEKHAAAA|140|7th Park|Circle|Suite 260|Guilford|Hall County|GA|34408|United States|-5|single family| +31302|AAAAAAAAGEKHAAAA|464|View |Ct.|Suite 190|Shady Grove|Lac qui Parle County|MN|52812|United States|-6|apartment| +31303|AAAAAAAAHEKHAAAA|146|Highland Oak|Ct.|Suite 20|Pleasant Hill|Sampson County|NC|23604|United States|-5|apartment| +31304|AAAAAAAAIEKHAAAA|114|10th |Cir.|Suite 380|Riverside|Nye County|NV|89231|United States|-8|single family| +31305|AAAAAAAAJEKHAAAA|991|Maple South|Drive|Suite Q|Glendale|Wheeler County|GA|33951|United States|-5|condo| +31306|AAAAAAAAKEKHAAAA|529|Forest Hill|Blvd|Suite L|Woodrow|Weld County|CO|84273|United States|-7|condo| +31307|AAAAAAAALEKHAAAA|146|7th |Ln|Suite I|New Hope|Switzerland County|IN|49431|United States|-5|single family| +31308|AAAAAAAAMEKHAAAA|328|Walnut |Parkway|Suite 250|Stewart|Maricopa County|AZ|88041|United States|-7|condo| +31309|AAAAAAAANEKHAAAA|852|Fifth |Pkwy|Suite 130|Harmony|Emmet County|IA|55804|United States|-6|condo| +31310|AAAAAAAAOEKHAAAA|678|7th |Street|Suite G|Mount Pleasant|Dawson County|MT|61933|United States|-7|apartment| +31311|AAAAAAAAPEKHAAAA|171|Main |Street|Suite T|Bethel|Franklin County|VA|25281|United States|-5|single family| +31312|AAAAAAAAAFKHAAAA|923|4th Seventh|Pkwy|Suite 190|Watkins|Rock Island County|IL|61732|United States|-6|condo| +31313|AAAAAAAABFKHAAAA|297|First |Boulevard|Suite T|New Hope|Morgan County|OH|49431|United States|-5|apartment| +31314|AAAAAAAACFKHAAAA|810|Wilson East|Circle|Suite 130|Oak Ridge|Howard County|IN|48371|United States|-5|single family| +31315|AAAAAAAADFKHAAAA|325|Madison North|Ln|Suite U|Glendale|Will County|IL|63951|United States|-6|apartment| +31316|AAAAAAAAEFKHAAAA|53|Railroad Elm|Way|Suite 390|Doyle|Sabine County|TX|78434|United States|-6|single family| +31317|AAAAAAAAFFKHAAAA|898|Second |Boulevard|Suite 290|Crossroads|Patrick County|VA|20534|United States|-5|condo| +31318|AAAAAAAAGFKHAAAA|10|8th |Ave|Suite 480|Greenwood|Colfax County|NE|68828|United States|-6|apartment| +31319|AAAAAAAAHFKHAAAA|592|14th |Wy|Suite 240|Riverview|Los Alamos County|NM|89003|United States|-7|apartment| +31320|AAAAAAAAIFKHAAAA|724||||Oak Hill||OK|77838||-6|| +31321|AAAAAAAAJFKHAAAA|678|North Madison|Lane|Suite U|Hopewell|O-Brien County|IA|50587|United States|-6|condo| +31322|AAAAAAAAKFKHAAAA|103|Fifth Eigth|Road|Suite G|Valley View|Corson County|SD|55124|United States|-6|single family| +31323|AAAAAAAALFKHAAAA|573|Ridge |Street|Suite 360|Georgetown|Tucker County|WV|27057|United States|-5|condo| +31324|AAAAAAAAMFKHAAAA|918|||Suite H|Superior|San Juan County||82562|||condo| +31325|AAAAAAAANFKHAAAA|82|Birch North|Blvd|Suite N|Oak Hill|Dickey County|ND|57838|United States|-6|single family| +31326|AAAAAAAAOFKHAAAA|204|Ninth |Lane|Suite H|Bridgeport|Scott County|MO|65817|United States|-6|condo| +31327|AAAAAAAAPFKHAAAA|800|Church First|RD|Suite 320|Oak Grove|Hunt County|TX|78370|United States|-6|apartment| +31328|AAAAAAAAAGKHAAAA|328|Walnut Fourth|Wy|Suite Q|Lakeview|Ellsworth County|KS|68579|United States|-6|apartment| +31329|AAAAAAAABGKHAAAA|743|Walnut 2nd|ST|Suite D|Five Points|Klamath County|OR|96098|United States|-8|single family| +31330|AAAAAAAACGKHAAAA|196|East |Blvd|Suite 410|Shiloh|Coshocton County|OH|49275|United States|-5|single family| +31331|AAAAAAAADGKHAAAA|698|Wilson |Parkway|Suite D|Marion|Merrimack County|NH|00999|United States|-5|condo| +31332|AAAAAAAAEGKHAAAA|222|College |Parkway|Suite 190|Riverdale|Macoupin County|IL|69391|United States|-6|condo| +31333|AAAAAAAAFGKHAAAA|429||Pkwy||Sunnyside|Aleutians West Census Area||91952|||| +31334|AAAAAAAAGGKHAAAA|896|Laurel |Lane||Oakdale||ND||United States||| +31335|AAAAAAAAHGKHAAAA|||RD|||Gage County|NE|60191|United States|-6|| +31336|AAAAAAAAIGKHAAAA|572|Madison |Lane|Suite U|Woodville|Montgomery County|TN|34289|United States|-6|condo| +31337|AAAAAAAAJGKHAAAA|23|Spring |Court|Suite U|Kingston|Clay County|TN|34975|United States|-5|condo| +31338|AAAAAAAAKGKHAAAA|120|Chestnut Park|Road|Suite 230|Five Points|Coke County|TX|76098|United States|-6|apartment| +31339|AAAAAAAALGKHAAAA|922|Lake |Road|Suite L|Oak Hill|Wyoming County|NY|17838|United States|-5|single family| +31340|AAAAAAAAMGKHAAAA|230|Forest 13th|Court|Suite 200|Page|Schuyler County|NY|10296|United States|-5|condo| +31341|AAAAAAAANGKHAAAA|659|Ash |Ln|Suite 30|Shiloh|Page County|IA|59275|United States|-6|single family| +31342|AAAAAAAAOGKHAAAA|970|College |Road|Suite 320|Spring Hill|King County|TX|76787|United States|-6|condo| +31343|AAAAAAAAPGKHAAAA|635|Fourth Cedar|Cir.|Suite A|Clifton|Floyd County|IA|58014|United States|-6|apartment| +31344|AAAAAAAAAHKHAAAA|507|5th |Way|Suite 80|Enterprise|Glacier County|MT|61757|United States|-7|apartment| +31345|AAAAAAAABHKHAAAA|190|Lake Spruce|Lane|Suite 460|Oakwood|Decatur County|GA|30169|United States|-5|apartment| +31346|AAAAAAAACHKHAAAA|580|South |Lane|Suite M|Spring Hill|Henry County|KY|46787|United States|-6|apartment| +31347|AAAAAAAADHKHAAAA|478|Oak |Road|Suite 130|Spring Valley|Brooks County|GA|36060|United States|-5|apartment| +31348|AAAAAAAAEHKHAAAA|453|Maple |Pkwy|Suite 300|Newport|Hayes County|NE|61521|United States|-6|single family| +31349|AAAAAAAAFHKHAAAA|412|14th |ST|Suite 400|Glendale|Berrien County|MI|43951|United States|-5|condo| +31350|AAAAAAAAGHKHAAAA|610|Fifth Hill|Road|Suite 320|Unionville|McKenzie County|ND|51711|United States|-6|condo| +31351|AAAAAAAAHHKHAAAA|523|Church Birch|Avenue|Suite H|Buena Vista|Hernando County|FL|35752|United States|-5|condo| +31352|AAAAAAAAIHKHAAAA|181|Third Mill|Lane|Suite D|Crossroads|Christian County|IL|60534|United States|-6|apartment| +31353|AAAAAAAAJHKHAAAA|12|Broadway 6th|RD|Suite L|Five Points|York County|ME|06698|United States|-5|single family| +31354|AAAAAAAAKHKHAAAA|255|2nd |Wy|Suite H|Red Hill|Hancock County|ME|04938|United States|-5|single family| +31355|AAAAAAAALHKHAAAA|517|Main West|RD|Suite B|Bunker Hill|Blaine County|MT|60150|United States|-7|apartment| +31356|AAAAAAAAMHKHAAAA|959|Ash North|Street|Suite 280|Mount Pleasant|Clackamas County|OR|91933|United States|-8|condo| +31357|AAAAAAAANHKHAAAA|14|3rd |RD|Suite 40|Lewisburg|Muscatine County|IA|57538|United States|-6|single family| +31358|AAAAAAAAOHKHAAAA|900|Elm |ST|Suite 130|Cedar Grove|Gosper County|NE|60411|United States|-6|single family| +31359|AAAAAAAAPHKHAAAA|668|Cherry |ST|Suite 260|Shady Grove|Washington County|GA|32812|United States|-5|apartment| +31360|AAAAAAAAAIKHAAAA|940|Sunset Highland|Street|Suite 210|Red Hill|Upton County|TX|74338|United States|-6|single family| +31361|AAAAAAAABIKHAAAA|876|10th |Avenue|Suite 110|Pleasant Grove|Geary County|KS|64136|United States|-6|single family| +31362|AAAAAAAACIKHAAAA|180|Smith |Dr.|Suite 320|Mount Vernon|Monroe County|AL|38482|United States|-6|condo| +31363|AAAAAAAADIKHAAAA|602|Wilson |Way|Suite 230|Arlington|Lincoln County|KS|66557|United States|-6|condo| +31364|AAAAAAAAEIKHAAAA|625|North |Drive|Suite 230|Oakdale|Haakon County|SD|59584|United States|-7|single family| +31365|AAAAAAAAFIKHAAAA|10|Lincoln South|Road|Suite T|Oak Ridge|Marion County|TN|38371|United States|-6|single family| +31366|AAAAAAAAGIKHAAAA|240|First |Lane|Suite 220|Antioch|Attala County|MS|58605|United States|-6|condo| +31367|AAAAAAAAHIKHAAAA|117|Willow |Circle|Suite 390|Lebanon|Montgomery County|GA|32898|United States|-5|apartment| +31368|AAAAAAAAIIKHAAAA|285|Hill |Ln|Suite 110|Greenville|Atchison County|MO|61387|United States|-6|apartment| +31369|AAAAAAAAJIKHAAAA|708|Pine |Ave|Suite 210|Jamestown|Eagle County|CO|86867|United States|-7|condo| +31370|AAAAAAAAKIKHAAAA|372|Lincoln |Court|Suite 350|Newtown|Iron County|MO|61749|United States|-6|single family| +31371|AAAAAAAALIKHAAAA|277|Locust |ST|Suite A|Mountain View|Brule County|SD|54466|United States|-6|single family| +31372|AAAAAAAAMIKHAAAA|680|10th 2nd|Blvd|Suite P|Hopewell|Hood County|TX|70587|United States|-6|single family| +31373|AAAAAAAANIKHAAAA|298|Davis |Ave|Suite U|Salem|Pike County|AR|78048|United States|-6|apartment| +31374|AAAAAAAAOIKHAAAA|305|Church |Court|Suite F|Enterprise|Buckingham County|VA|21757|United States|-5|single family| +31375|AAAAAAAAPIKHAAAA|67|Maple |Ct.|Suite H|Bunker Hill|Lane County|KS|60150|United States|-6|apartment| +31376|AAAAAAAAAJKHAAAA|480|Wilson |Blvd|Suite X|Valley View|La Salle County|TX|75124|United States|-6|condo| +31377|AAAAAAAABJKHAAAA|949|||Suite P|||||||single family| +31378|AAAAAAAACJKHAAAA|281|Elm South|Court|Suite 110|Lakewood|Oconee County|GA|38877|United States|-5|apartment| +31379|AAAAAAAADJKHAAAA|341|Lincoln 4th|Ct.|Suite 220|Stringtown|Essex County|NY|10162|United States|-5|single family| +31380|AAAAAAAAEJKHAAAA|524|3rd |Ave|Suite 360|Five Points|Anderson County|TN|36098|United States|-5|single family| +31381|AAAAAAAAFJKHAAAA|569|12th |Circle|Suite C|Sulphur Springs|Laurens County|GA|38354|United States|-5|single family| +31382|AAAAAAAAGJKHAAAA|576|Forest Walnut|Wy|Suite 200|Bridgeport|Lincoln County|WI|55817|United States|-6|condo| +31383|AAAAAAAAHJKHAAAA|191|Washington Lincoln|Avenue|Suite 300|Pleasant Grove|Lee County|MS|54136|United States|-6|single family| +31384|AAAAAAAAIJKHAAAA|521|Spring |Boulevard|Suite 370|Highland|Buckingham County|VA|29454|United States|-5|condo| +31385|AAAAAAAAJJKHAAAA|318|Center 6th|Wy|Suite 410|Allison|Perry County|MS|54167|United States|-6|condo| +31386|AAAAAAAAKJKHAAAA|682|3rd Cherry|Dr.|Suite U|Woodlawn|Emmet County|MI|44098|United States|-5|apartment| +31387|AAAAAAAALJKHAAAA|661|Chestnut 1st|Way|Suite E|Newport|Unicoi County|TN|31521|United States|-6|single family| +31388|AAAAAAAAMJKHAAAA|273|College 13th|Boulevard|Suite P|Red Hill|Deuel County|NE|64338|United States|-6|single family| +31389|AAAAAAAANJKHAAAA|285|Locust Cherry|Boulevard|Suite M|Omega|Chautauqua County|KS|66364|United States|-6|condo| +31390|AAAAAAAAOJKHAAAA|450|Willow |Cir.|Suite G|Marion|Franklin County|GA|30399|United States|-5|condo| +31391|AAAAAAAAPJKHAAAA|224|Park Seventh|Ave|Suite J|Salem|Jackson County|WV|28048|United States|-5|single family| +31392|AAAAAAAAAKKHAAAA|565|Railroad Park|Lane|Suite 180|Kingston|Cache County|UT|84975|United States|-7|single family| +31393|AAAAAAAABKKHAAAA|709|5th |Lane|Suite T|Cedar Grove|Isanti County|MN|50411|United States|-6|single family| +31394|AAAAAAAACKKHAAAA|430|Hickory 9th|Ln|Suite 120|Pierce|Washington County|RI|03960|United States|-5|apartment| +31395|AAAAAAAADKKHAAAA|260|1st |Avenue|Suite T|Lucas|Antrim County|MI|44554|United States|-5|condo| +31396|AAAAAAAAEKKHAAAA|193|Maple Highland|Pkwy|Suite W|Brownsville|Lawrence County|MS|59310|United States|-6|single family| +31397|AAAAAAAAFKKHAAAA|220|Lincoln |Parkway|Suite 10|Harmony|Rockdale County|GA|35804|United States|-5|condo| +31398|AAAAAAAAGKKHAAAA|698|4th Eigth|ST|Suite 420|Antioch|Dixie County|FL|38605|United States|-5|condo| +31399|AAAAAAAAHKKHAAAA|199|Johnson Church|Parkway|Suite Y|Pleasant Valley|Livingston County|MO|62477|United States|-6|single family| +31400|AAAAAAAAIKKHAAAA|312|4th Smith|Blvd|Suite 30|Woodville|Wahkiakum County|WA|94289|United States|-8|single family| +31401|AAAAAAAAJKKHAAAA|864|Williams Railroad|Road|Suite 110|Whitesville|Boyd County|NE|65903|United States|-6|apartment| +31402|AAAAAAAAKKKHAAAA|947|15th |RD|Suite 460|Macedonia|Burke County|GA|31087|United States|-5|condo| +31403|AAAAAAAALKKHAAAA|208|Park |Road|Suite 440|Woodland|Ohio County|WV|24854|United States|-5|condo| +31404|AAAAAAAAMKKHAAAA|527|Second Elm|Ave|Suite K|Macedonia|Putnam County|WV|21087|United States|-5|condo| +31405|AAAAAAAANKKHAAAA|434|Ridge Park|Street|Suite 20|Summit|Marshall County|MS|50499|United States|-6|apartment| +31406|AAAAAAAAOKKHAAAA|34|Jefferson Forest|Circle|Suite 380|Newtown|Lincoln County|WV|21749|United States|-5|apartment| +31407|AAAAAAAAPKKHAAAA|910|Spring Mill|Avenue|Suite B|Floyd|Sawyer County|WI|53235|United States|-6|single family| +31408|AAAAAAAAALKHAAAA|382|Central Pine|Pkwy|Suite 410|Stringtown|Clackamas County|OR|90162|United States|-8|apartment| +31409|AAAAAAAABLKHAAAA|89|Green |Drive|Suite R|Farmington|Red Lake County|MN|59145|United States|-6|single family| +31410|AAAAAAAACLKHAAAA|681|4th Adams|Lane|Suite 30|Pinecrest|Juneau Borough|AK|99981|United States|-9|single family| +31411|AAAAAAAADLKHAAAA|662|Cedar |Pkwy|Suite 420|Oakland|Prairie County|MT|69843|United States|-7|apartment| +31412|AAAAAAAAELKHAAAA|434|3rd North|Street|Suite 170|Highland Park|Hawaii County|HI|96534|United States|-10|single family| +31413|AAAAAAAAFLKHAAAA|783|Mill Fifth|Street|Suite H|Fairfield|Crawford County|KS|66192|United States|-6|apartment| +31414|AAAAAAAAGLKHAAAA|546|Fifth |Ave|Suite 90|Concord|Dawson County|TX|74107|United States|-6|single family| +31415|AAAAAAAAHLKHAAAA|246|Center Green|Circle|Suite 170|Newtown|Carroll County|IN|41749|United States|-5|apartment| +31416|AAAAAAAAILKHAAAA|312|Chestnut |Ln|Suite 340|Valley View|Johnson County|IN|45124|United States|-5|single family| +31417|AAAAAAAAJLKHAAAA|42|Davis 9th|Dr.|Suite P|Concord|Allen County|KY|44107|United States|-6|apartment| +31418|AAAAAAAAKLKHAAAA|171|12th Maple|Ln|Suite 270|Friendship|Greenlee County|AZ|84536|United States|-7|condo| +31419|AAAAAAAALLKHAAAA|3|Broadway |Court|Suite 400|Glenwood|Adair County|IA|53511|United States|-6|apartment| +31420|AAAAAAAAMLKHAAAA|862|First |Pkwy|Suite 150|Clifton|Olmsted County|MN|58014|United States|-6|apartment| +31421|AAAAAAAANLKHAAAA|985|2nd 7th|Road|Suite 40|Shady Grove|Campbell County|SD|52812|United States|-6|condo| +31422|AAAAAAAAOLKHAAAA|802|6th |Road|Suite 20|Georgetown|Orange County|VT|07657|United States|-5|single family| +31423|AAAAAAAAPLKHAAAA|579|11th 15th|Circle|Suite B|Valley View|Osceola County|MI|45124|United States|-5|apartment| +31424|AAAAAAAAAMKHAAAA|958|1st |Blvd|Suite T|Spring Hill|Nelson County|VA|26787|United States|-5|apartment| +31425|AAAAAAAABMKHAAAA|819|Jefferson |Boulevard|Suite P|Highland|Mason County|WV|29454|United States|-5|single family| +31426|AAAAAAAACMKHAAAA|458|2nd Chestnut|Pkwy|Suite 230|Greenwood|Hardee County|FL|38828|United States|-5|condo| +31427|AAAAAAAADMKHAAAA|972|Lakeview Williams|Ln|Suite 460|Georgetown|Stillwater County|MT|67057|United States|-7|condo| +31428|AAAAAAAAEMKHAAAA|293|Maple Fourth|Ln|Suite 350|Arlington|Bannock County|ID|86557|United States|-7|single family| +31429|AAAAAAAAFMKHAAAA|423|College |Ln|Suite N|Hillcrest|Franklin County|NE|63003|United States|-6|apartment| +31430|AAAAAAAAGMKHAAAA|686|North Hill|Drive|Suite 220|Antioch|Pearl River County|MS|58605|United States|-6|condo| +31431|AAAAAAAAHMKHAAAA|452|Sixth |Avenue|Suite 50|Buena Vista|Davidson County|TN|35752|United States|-5|apartment| +31432|AAAAAAAAIMKHAAAA|664|Pine |||Liberty||||United States||apartment| +31433|AAAAAAAAJMKHAAAA||Birch |Ave||Lakewood|||68877|||condo| +31434|AAAAAAAAKMKHAAAA|180|Locust |Court|Suite T|Georgetown|Lee County|AL|37057|United States|-6|single family| +31435|AAAAAAAALMKHAAAA|917|12th 8th|Drive|Suite 30|Spring Hill|Wood County|WV|26787|United States|-5|apartment| +31436|AAAAAAAAMMKHAAAA|431|10th Washington|Ct.|Suite O|Liberty|Llano County|TX|73451|United States|-6|condo| +31437|AAAAAAAANMKHAAAA|317|Park |Drive|Suite F|Carter|Collingsworth County|TX|70919|United States|-6|condo| +31438|AAAAAAAAOMKHAAAA|457|2nd 6th|Boulevard|Suite C|Fairview|Knox County|IL|65709|United States|-6|condo| +31439|AAAAAAAAPMKHAAAA|912|Oak |Circle|Suite M|Hamilton|Polk County|TX|72808|United States|-6|condo| +31440|AAAAAAAAANKHAAAA|464|Fifth Ash|Lane|Suite M|Greenfield|Walker County|GA|35038|United States|-5|apartment| +31441|AAAAAAAABNKHAAAA||||||Frio County||76060||-6|| +31442|AAAAAAAACNKHAAAA|281|1st Willow|Lane|Suite E|Springfield|Randolph County|IL|69303|United States|-6|condo| +31443|AAAAAAAADNKHAAAA|920|1st |Ln|Suite P|Spring Valley|Poweshiek County|IA|56060|United States|-6|apartment| +31444|AAAAAAAAENKHAAAA|240|East Green|RD|Suite Q|Marion|Kootenai County|ID|80399|United States|-7|condo| +31445|AAAAAAAAFNKHAAAA|381|Church |Cir.|Suite 60|Georgetown|Red Lake County|MN|57057|United States|-6|apartment| +31446|AAAAAAAAGNKHAAAA|520|Mill |Ln|Suite B|Saint Clair|Ward County|TX|75294|United States|-6|single family| +31447|AAAAAAAAHNKHAAAA|364|3rd |Cir.|Suite 190|Oak Hill|Bay County|MI|47838|United States|-5|condo| +31448|AAAAAAAAINKHAAAA|553|First Park|Parkway|Suite 400|Five Points|Madison County|ID|86098|United States|-7|condo| +31449|AAAAAAAAJNKHAAAA|551|Jackson Church|Drive|Suite 40|Farmington|Mills County|TX|79145|United States|-6|single family| +31450|AAAAAAAAKNKHAAAA|554|View Poplar|RD|Suite I|Smith|Tippah County|MS|57317|United States|-6|single family| +31451|AAAAAAAALNKHAAAA|240|5th |Circle|Suite V|White Hall|Loudoun County|VA|26955|United States|-5|apartment| +31452|AAAAAAAAMNKHAAAA|194|Franklin 7th|Ave|Suite 140|Summit|Cascade County|MT|60499|United States|-7|single family| +31453|AAAAAAAANNKHAAAA|569|Second Franklin|Lane|Suite 90|Five Forks|Lee County|TX|72293|United States|-6|apartment| +31454|AAAAAAAAONKHAAAA|507|12th First|Way|Suite 450|Lincoln|Lamar County|TX|71289|United States|-6|condo| +31455|AAAAAAAAPNKHAAAA|38|Cherry |Lane|Suite G|Greenville|Ben Hill County|GA|31387|United States|-5|condo| +31456|AAAAAAAAAOKHAAAA|638|Smith |Boulevard|Suite 10|Union Hill|Pembina County|ND|57746|United States|-6|apartment| +31457|AAAAAAAABOKHAAAA|619|Laurel Spring|Ave|Suite 180|Newtown|Brookings County|SD|51749|United States|-6|condo| +31458|AAAAAAAACOKHAAAA|342|Oak |Court|Suite H|Webb|Adams County|PA|10899|United States|-5|single family| +31459|AAAAAAAADOKHAAAA|311|Williams |Ave|Suite R|Harmony|Newberry County|SC|25804|United States|-5|apartment| +31460|AAAAAAAAEOKHAAAA|774|Davis 4th|Ct.|Suite O|Mount Olive|Grant County|WA|98059|United States|-8|apartment| +31461|AAAAAAAAFOKHAAAA|625|Woodland |Parkway|Suite 170|Wilson|Dorchester County|MD|26971|United States|-5|single family| +31462|AAAAAAAAGOKHAAAA|118|Elm |Ln|Suite 140|Sulphur Springs|Jackson County|MO|68354|United States|-6|condo| +31463|AAAAAAAAHOKHAAAA|132|12th South|Circle|Suite 80|Wildwood|Jefferson County|WA|96871|United States|-8|single family| +31464|AAAAAAAAIOKHAAAA|267|10th 4th|Way|Suite C|Midway|Middlesex County|VA|21904|United States|-5|single family| +31465|AAAAAAAAJOKHAAAA|318|Cherry |Ln|Suite 180|Belleview|Banks County|GA|30492|United States|-5|single family| +31466|AAAAAAAAKOKHAAAA|600|4th Twelfth|Ln|Suite X|Pleasant Valley|Banner County|NE|62477|United States|-6|single family| +31467|AAAAAAAALOKHAAAA|64|Main |Cir.|Suite 260|Greenville|Darlington County|SC|21387|United States|-5|apartment| +31468|AAAAAAAAMOKHAAAA|308|Center Sixth|Cir.|Suite R|Longwood|Page County|IA|57021|United States|-6|apartment| +31469|AAAAAAAANOKHAAAA|756|Mill |Cir.|Suite H|Fairview|Sheridan County|NE|65709|United States|-7|condo| +31470|AAAAAAAAOOKHAAAA|429|View 2nd|ST|Suite N|Spring Valley|Manistee County|MI|46060|United States|-5|single family| +31471|AAAAAAAAPOKHAAAA|896|Center |Parkway|Suite 200|Highland Park|Travis County|TX|76534|United States|-6|single family| +31472|AAAAAAAAAPKHAAAA|604|Fifth Miller|ST|Suite T|Springfield|Haskell County|TX|79303|United States|-6|single family| +31473|AAAAAAAABPKHAAAA|806|View Cedar|Dr.|Suite L|Pleasant Valley|Madison County|NY|12477|United States|-5|apartment| +31474|AAAAAAAACPKHAAAA|248|Meadow 9th|Ln|Suite 120|Red Hill|Houghton County|MI|44338|United States|-5|single family| +31475|AAAAAAAADPKHAAAA|28|Laurel |Lane|Suite 470|Point Pleasant|Darke County|OH|44749|United States|-5|condo| +31476|AAAAAAAAEPKHAAAA|995|Davis Railroad|Street|Suite N|Union|Cole County|MO|68721|United States|-6|apartment| +31477|AAAAAAAAFPKHAAAA|173|Forest Meadow|Blvd|Suite 420|Florence|Hancock County|IA|53394|United States|-6|single family| +31478|AAAAAAAAGPKHAAAA|486|North Center|Parkway|Suite 80|Bethel|Baltimore County|MD|25281|United States|-5|condo| +31479|AAAAAAAAHPKHAAAA|195|Laurel |Pkwy|Suite 10|Spring Valley|Glades County|FL|36060|United States|-5|single family| +31480|AAAAAAAAIPKHAAAA|164|Davis |Blvd|Suite D|Antioch|Limestone County|AL|38605|United States|-6|apartment| +31481|AAAAAAAAJPKHAAAA|470|7th Main|Ct.|Suite 40|Cordova|Jefferson Davis County|MS|56938|United States|-6|apartment| +31482|AAAAAAAAKPKHAAAA|990|Ash Mill|Avenue|Suite E|Oakdale|Cochran County|TX|79584|United States|-6|apartment| +31483|AAAAAAAALPKHAAAA|138|Birch 3rd|Drive|Suite 330|Pine Grove|Aransas County|TX|74593|United States|-6|single family| +31484|AAAAAAAAMPKHAAAA|271|8th |Lane|Suite 230|Woodville|Merrimack County|NH|04889|United States|-5|single family| +31485|AAAAAAAANPKHAAAA|906|9th |Way|Suite 140|Harmony|Essex County|VT|06404|United States|-5|single family| +31486|AAAAAAAAOPKHAAAA|637|Adams Highland|Pkwy|Suite V|Lincoln|Caroline County|MD|21289|United States|-5|apartment| +31487|AAAAAAAAPPKHAAAA|823|Locust |Ave|Suite S|Wilson|Bacon County|GA|36971|United States|-5|condo| +31488|AAAAAAAAAALHAAAA|377|South |Dr.|Suite C|Woodland|Franklin County|OH|44854|United States|-5|apartment| +31489|AAAAAAAABALHAAAA|952|Locust |Ln|Suite M|||ME|||-5|single family| +31490|AAAAAAAACALHAAAA|276|Center |Cir.|Suite H|Forest Hills|La Porte County|IN|49237|United States|-5|single family| +31491|AAAAAAAADALHAAAA|431|Dogwood Poplar|Way|Suite 150|Hillcrest|Alpena County|MI|43003|United States|-5|apartment| +31492|AAAAAAAAEALHAAAA|352|Chestnut Laurel|Circle|Suite 140|Ashland|Tulsa County|OK|74244|United States|-6|single family| +31493|AAAAAAAAFALHAAAA|393|North |Lane|Suite M|Highland Park|Bay County|FL|36534|United States|-5|condo| +31494|AAAAAAAAGALHAAAA|455|Fourth 1st|Wy|Suite D|Centerville|Marion County|MO|60059|United States|-6|apartment| +31495|AAAAAAAAHALHAAAA|184|15th Fourth|Avenue|Suite D|Crossroads|Lake County|MT|60534|United States|-7|apartment| +31496|AAAAAAAAIALHAAAA|721|Sixth |Avenue|Suite U|Page|Jessamine County|KY|40296|United States|-6|condo| +31497|AAAAAAAAJALHAAAA|278|Church |ST|Suite I|Sunnyside|Mahaska County|IA|51952|United States|-6|single family| +31498|AAAAAAAAKALHAAAA|774|Ridge Wilson|Ave|Suite X|Maple Grove|De Witt County|IL|68252|United States|-6|apartment| +31499|AAAAAAAALALHAAAA|81|Ridge |Street|Suite 200|Enterprise|Colbert County|AL|31757|United States|-6|condo| +31500|AAAAAAAAMALHAAAA|837|Jackson Park|Cir.|Suite S|Lakeside|Jefferson County|MS|59532|United States|-6|apartment| +31501|AAAAAAAANALHAAAA||Main Cherry|Blvd|Suite U|Mount Pleasant|Lake County|FL||||| +31502|AAAAAAAAOALHAAAA|896|Madison |Court|Suite O|Pine Grove|Lamb County|TX|74593|United States|-6|single family| +31503|AAAAAAAAPALHAAAA|968|2nd Lee|Cir.|Suite N|Florence|Hampton County|SC|23394|United States|-5|condo| +31504|AAAAAAAAABLHAAAA|902|College |Pkwy|Suite P|Pleasant Hill|Harrison County|KY|43604|United States|-6|apartment| +31505|AAAAAAAABBLHAAAA|810|3rd |Boulevard|Suite 250|Walnut Grove|Jefferson County|NE|67752|United States|-7|condo| +31506|AAAAAAAACBLHAAAA|68|North Cherry|Avenue|Suite R|Guthrie|Niagara County|NY|11423|United States|-5|apartment| +31507|AAAAAAAADBLHAAAA|490|1st ||Suite 280|Florence|Adams County|PA||United States||| +31508|AAAAAAAAEBLHAAAA|967|3rd Park|Pkwy|Suite 400|Forest Hills|Knox County|NE|69237|United States|-7|apartment| +31509|AAAAAAAAFBLHAAAA|883|Jackson Dogwood|Drive|Suite O|Forest Hills|Augusta County|VA|29237|United States|-5|single family| +31510|AAAAAAAAGBLHAAAA|896|8th Lee|Blvd|Suite 170|Gum Springs|San Francisco County|CA|92106|United States|-8|single family| +31511|AAAAAAAAHBLHAAAA|946|Franklin |Way|Suite L|Midway|Perry County|MO|61904|United States|-6|single family| +31512|AAAAAAAAIBLHAAAA||Mill Lincoln|Ln||Hillcrest|Tate County||53003|||single family| +31513|AAAAAAAAJBLHAAAA|875|Forest Hill|ST|Suite R|Farmington|Lamar County|GA|39145|United States|-5|apartment| +31514|AAAAAAAAKBLHAAAA|945|South Cedar|RD|Suite 20|Riverdale|Clearfield County|PA|19391|United States|-5|apartment| +31515|AAAAAAAALBLHAAAA|466|1st |Wy|Suite 210|Stringtown|Roger Mills County|OK|70162|United States|-6|apartment| +31516|AAAAAAAAMBLHAAAA|694|Second Lee|Avenue|Suite J|Bethel|Middlesex County|MA|05881|United States|-5|apartment| +31517|AAAAAAAANBLHAAAA|744|Miller |Road|Suite O|Spring Valley|Idaho County|ID|86060|United States|-7|condo| +31518|AAAAAAAAOBLHAAAA|10|Lincoln Cedar|Cir.|Suite 260|Enterprise|Concordia Parish|LA|71757|United States|-6|apartment| +31519|AAAAAAAAPBLHAAAA|594|River Pine|Pkwy|Suite G|Edgewood|Salt Lake County|UT|80069|United States|-7|single family| +31520|AAAAAAAAACLHAAAA|557|Pine 8th|Way|Suite P|Glenwood|Eureka County|NV|83511|United States|-8|single family| +31521|AAAAAAAABCLHAAAA|108|Jefferson Center|Boulevard|Suite 480|Spring Hill|Macon County|GA|36787|United States|-5|apartment| +31522|AAAAAAAACCLHAAAA|533|Tenth |Drive|Suite 470|Kingston|Jerauld County|SD|54975|United States|-7|condo| +31523|AAAAAAAADCLHAAAA|981|Cedar Miller|Parkway|Suite 360|Salem|Boone County|KY|48048|United States|-6|apartment| +31524|AAAAAAAAECLHAAAA|275|Tenth |RD|Suite 130|Mount Zion|Nome Census Area|AK|98054|United States|-9|condo| +31525|AAAAAAAAFCLHAAAA|735|West 2nd|RD|Suite B|Midway|Fairfax city|VA|21904|United States|-5|apartment| +31526|AAAAAAAAGCLHAAAA|754|Center |Boulevard|Suite 250|Oak Hill|Upshur County|TX|77838|United States|-6|single family| +31527|AAAAAAAAHCLHAAAA|407|14th |Boulevard|Suite O|Mountain View|Watonwan County|MN|54466|United States|-6|single family| +31528|AAAAAAAAICLHAAAA|838|Main Spring|Circle|Suite I|Riverside|Cumberland County|NJ|09831|United States|-5|single family| +31529|AAAAAAAAJCLHAAAA|800|Central 7th|Avenue|Suite U|Farmington|Essex County|VT|09745|United States|-5|single family| +31530|AAAAAAAAKCLHAAAA|810|East |Drive|Suite I|Deerfield|McHenry County|IL|69840|United States|-6|apartment| +31531|AAAAAAAALCLHAAAA|409|Maple |Road|Suite 290|Paxton|Grand County|CO|85669|United States|-7|condo| +31532|AAAAAAAAMCLHAAAA|429|Elm Spring|Ln|Suite 270|Jackson|Jackson County|OH|49583|United States|-5|single family| +31533|AAAAAAAANCLHAAAA|510|Willow Miller|Way|Suite 0|Springfield|Osage County|KS|69303|United States|-6|single family| +31534|AAAAAAAAOCLHAAAA|298|First Lee|Avenue|Suite 380|Cherry Valley|Henderson County|IL|60854|United States|-6|condo| +31535|AAAAAAAAPCLHAAAA|516|Franklin Seventh|Dr.|Suite 400|Antioch|Calhoun County|IA|58605|United States|-6|single family| +31536|AAAAAAAAADLHAAAA|470|Elm Adams|Dr.|Suite 10|Lakeside|Kewaunee County|WI|59532|United States|-6|single family| +31537|AAAAAAAABDLHAAAA|||Pkwy|Suite W|White Oak|Republic County|||||condo| +31538|AAAAAAAACDLHAAAA|869|Second Church|Lane|Suite 430|Wildwood|Carroll County|AR|76871|United States|-6|apartment| +31539|AAAAAAAADDLHAAAA|336|1st |Ln|Suite I|Walnut|Poquoson city|VA|26245|United States|-5|apartment| +31540|AAAAAAAAEDLHAAAA|355|West |Wy|Suite V|Hillcrest|Barbour County|WV|23003|United States|-5|apartment| +31541|AAAAAAAAFDLHAAAA|767|Jackson Spruce|Wy|Suite X|Arlington|Putnam County|NY|16557|United States|-5|single family| +31542|AAAAAAAAGDLHAAAA|130|Pine Ridge|Dr.|Suite 470|Pleasant Valley|Runnels County|TX|72477|United States|-6|apartment| +31543|AAAAAAAAHDLHAAAA|16|Elm |ST|Suite S|Oakwood|Seward County|KS|60169|United States|-6|condo| +31544|AAAAAAAAIDLHAAAA|132|Laurel |Lane|Suite B|Kingston|Osceola County|MI|44975|United States|-5|single family| +31545|AAAAAAAAJDLHAAAA|459|Jackson |Dr.|Suite C|Bunker Hill|Fulton County|GA|30150|United States|-5|condo| +31546|AAAAAAAAKDLHAAAA|98|1st |Cir.|Suite Y|Spring Valley|Upshur County|WV|26060|United States|-5|apartment| +31547|AAAAAAAALDLHAAAA|389|Smith |Ave|Suite 280|Friendship|Trigg County|KY|44536|United States|-5|condo| +31548|AAAAAAAAMDLHAAAA|378|1st |Blvd|Suite W|Sunnyside|Lewis County|MO|61952|United States|-6|single family| +31549|AAAAAAAANDLHAAAA|172|Fourth |Drive|Suite N|Highland|Traill County|ND|59454|United States|-6|condo| +31550|AAAAAAAAODLHAAAA|633|Sunset Third|Street|Suite A|Hamilton|Lee County|SC|22808|United States|-5|single family| +31551|AAAAAAAAPDLHAAAA|394|1st |Drive|Suite P|White Oak|Bowie County|TX|76668|United States|-6|apartment| +31552|AAAAAAAAAELHAAAA|515|Smith |Avenue|Suite O|Hillcrest|Schuyler County|IL|63003|United States|-6|single family| +31553|AAAAAAAABELHAAAA|198|Center |Drive|Suite 350|Mount Pleasant|Horry County|SC|21933|United States|-5|single family| +31554|AAAAAAAACELHAAAA|227|10th 5th|Wy|Suite B|Maple Grove|Bingham County|ID|88252|United States|-7|apartment| +31555|AAAAAAAADELHAAAA|87|Lakeview Dogwood|RD|Suite 330|Union Hill|Shiawassee County|MI|47746|United States|-5|condo| +31556|AAAAAAAAEELHAAAA|739|Church Birch|Drive|Suite 320|Forest Hills|Bedford County|PA|19237|United States|-5|single family| +31557|AAAAAAAAFELHAAAA|695|Oak Lakeview|ST|Suite Q|Maple Grove|Harmon County|OK|78252|United States|-6|condo| +31558|AAAAAAAAGELHAAAA|465|1st |Blvd|Suite 380|Liberty|Lee County|FL|33451|United States|-5|apartment| +31559|AAAAAAAAHELHAAAA|666|Green |Blvd|Suite 240|Five Points|Copiah County|MS|56098|United States|-6|single family| +31560|AAAAAAAAIELHAAAA|207|Adams |Parkway|Suite 370|New Hope|East Baton Rouge Parish|LA|79431|United States|-6|condo| +31561|AAAAAAAAJELHAAAA|926|Woodland North|Circle|Suite 160|Mountain View|Clinton County|MI|44466|United States|-5|single family| +31562|AAAAAAAAKELHAAAA|618|Johnson Park|Drive|Suite K|Buena Vista|Hernando County|FL|35752|United States|-5|condo| +31563|AAAAAAAALELHAAAA|777|Park |Pkwy|Suite 480|Lebanon|Jefferson County|MT|62898|United States|-7|apartment| +31564|AAAAAAAAMELHAAAA|934|6th ||Suite C||Panola County||54136||-6|| +31565|AAAAAAAANELHAAAA|577|Church |Ln|Suite D|Kingston|Seminole County|FL|34975|United States|-5|apartment| +31566|AAAAAAAAOELHAAAA||Ash First||Suite M|Ashland|Cass County|IN|44244|United States||| +31567|AAAAAAAAPELHAAAA|737|Spring Lake|Circle|Suite I|Cedar Grove|Cannon County|TN|30411|United States|-5|condo| +31568|AAAAAAAAAFLHAAAA|674|10th |Ave|Suite X|Centerville|Todd County|MN|50059|United States|-6|condo| +31569|AAAAAAAABFLHAAAA|832|Hill North|Lane|Suite 240|Fairfield|Warren County|TN|36192|United States|-6|condo| +31570|AAAAAAAACFLHAAAA|519|Second Park|Ln|Suite Y|Mount Pleasant|Putnam County|IN|41933|United States|-5|single family| +31571|AAAAAAAADFLHAAAA|97|Second |Wy|Suite P|Florence|Coke County|TX|73394|United States|-6|condo| +31572|AAAAAAAAEFLHAAAA|||Dr.|Suite Y||Henry County|IA||United States||apartment| +31573|AAAAAAAAFFLHAAAA|550|Johnson 8th|Boulevard|Suite 280|Whitesville|Richland County|MT|65903|United States|-7|condo| +31574|AAAAAAAAGFLHAAAA|644|Cedar River|Avenue|Suite L|Woodville|Wells County|IN|44289|United States|-5|single family| +31575|AAAAAAAAHFLHAAAA|192|Fifth |RD|Suite R|Greenville|Burlington County|NJ|01987|United States|-5|apartment| +31576|AAAAAAAAIFLHAAAA|400|South |Ave|Suite M|Globe|Hays County|TX|79614|United States|-6|apartment| +31577|AAAAAAAAJFLHAAAA|811|Oak |Way|Suite 400|Shady Grove|Hopewell city|VA|22812|United States|-5|single family| +31578|AAAAAAAAKFLHAAAA|224|River |Avenue|Suite H|Liberty|Terrell County|TX|73451|United States|-6|condo| +31579|AAAAAAAALFLHAAAA|206|Hill Maple|Ave|Suite 440|Providence|Jeff Davis County|TX|76614|United States|-6|condo| +31580|AAAAAAAAMFLHAAAA|618|Franklin 6th|Wy|Suite Q|Friendship|Schoolcraft County|MI|44536|United States|-5|single family| +31581|AAAAAAAANFLHAAAA|444|Birch |Ave|Suite J|Oak Grove|Ralls County|MO|68370|United States|-6|single family| +31582|AAAAAAAAOFLHAAAA|8|Cedar |Street|Suite 290|Sulphur Springs|Vermilion Parish|LA|78354|United States|-6|apartment| +31583|AAAAAAAAPFLHAAAA|24|Third Lakeview|Avenue|Suite 270|Highland Park|Madison County|KY|46534|United States|-5|single family| +31584|AAAAAAAAAGLHAAAA|||||||VA|26871|||| +31585|AAAAAAAABGLHAAAA|81|10th |Street|Suite 420|Salem|Camden County|GA|38048|United States|-5|single family| +31586|AAAAAAAACGLHAAAA|621|Mill Meadow|Wy|Suite K|Walnut Grove|Harrison County|MO|67752|United States|-6|single family| +31587|AAAAAAAADGLHAAAA|933|Jefferson |Dr.|Suite 160|Highland Park|Sussex County|NJ|07134|United States|-5|condo| +31588|AAAAAAAAEGLHAAAA|618|Dogwood Hillcrest|Drive|Suite 220|Wilson|Knox County|NE|66971|United States|-7|single family| +31589|AAAAAAAAFGLHAAAA|871|Second Fourth|Pkwy|Suite S|Belmont|Minnehaha County|SD|50191|United States|-7|single family| +31590|AAAAAAAAGGLHAAAA|745|13th |Dr.|Suite B|Brownsville|Cottonwood County|MN|59310|United States|-6|apartment| +31591|AAAAAAAAHGLHAAAA|951|Elm |Parkway|Suite 280|Lakeside|Hidalgo County|TX|79532|United States|-6|condo| +31592|AAAAAAAAIGLHAAAA|316|Cherry 8th|Court|Suite D|Georgetown|Champaign County|IL|67057|United States|-6|apartment| +31593|AAAAAAAAJGLHAAAA|133|Hickory |Blvd|Suite A|White Oak|Otoe County|NE|66668|United States|-7|condo| +31594|AAAAAAAAKGLHAAAA|610|Washington Johnson|Avenue|Suite F|Westminster|Harrison County|TX|76549|United States|-6|single family| +31595|AAAAAAAALGLHAAAA|983|Park |Wy|Suite 440|Newport|Perkins County|NE|61521|United States|-7|apartment| +31596|AAAAAAAAMGLHAAAA|277|Sunset 1st|Drive|Suite N|Red Hill|Wichita County|TX|74338|United States|-6|single family| +31597|AAAAAAAANGLHAAAA|135|Sycamore Dogwood|Avenue|Suite F|Clifton|Scioto County|OH|48014|United States|-5|single family| +31598|AAAAAAAAOGLHAAAA|380|Church |Cir.|Suite X|Oak Hill|Teton County|ID|87838|United States|-7|single family| +31599|AAAAAAAAPGLHAAAA|431|8th Forest|Parkway|Suite C|Glendale|Osborne County|KS|63951|United States|-6|condo| +31600|AAAAAAAAAHLHAAAA|688|Chestnut |Avenue|Suite 50|Macedonia|Bailey County|TX|71087|United States|-6|apartment| +31601|AAAAAAAABHLHAAAA|510|Hillcrest 4th|RD|Suite U|Belfast|Bayfield County|WI|50125|United States|-6|apartment| +31602|AAAAAAAACHLHAAAA|496|Hickory |Street|Suite K|Red Hill|Person County|NC|24338|United States|-5|condo| +31603|AAAAAAAADHLHAAAA|13|Williams 1st|Parkway|Suite 310|Oakland|Noble County|OK|79843|United States|-6|single family| +31604|AAAAAAAAEHLHAAAA|393|Cherry |Ave|Suite 180|Bloomingdale|Coles County|IL|61824|United States|-6|condo| +31605|AAAAAAAAFHLHAAAA|34|Seventh Meadow|Drive|Suite 20|Centerville|Sussex County|NJ|00659|United States|-5|apartment| +31606|AAAAAAAAGHLHAAAA|61|Williams |Dr.|Suite 390|Lakeview|Green County|KY|48579|United States|-6|single family| +31607|AAAAAAAAHHLHAAAA|821|Forest |Road|Suite 460|Pleasant Grove|Richland County|IL|64136|United States|-6|condo| +31608|AAAAAAAAIHLHAAAA|775|Wilson Ninth|Cir.|Suite K|Woodlawn|Webster Parish|LA|74098|United States|-6|condo| +31609|AAAAAAAAJHLHAAAA|575|Cherry |Circle|Suite U|Buena Vista|Sequatchie County|TN|35752|United States|-6|apartment| +31610|AAAAAAAAKHLHAAAA|578|Davis Fifth|Ln|Suite S|Duncan|Cleveland County|OK|76261|United States|-6|condo| +31611|AAAAAAAALHLHAAAA|428|Jefferson Ridge|Ave|Suite 10|Glendale|Lexington County|SC|23951|United States|-5|single family| +31612|AAAAAAAAMHLHAAAA|929|View Hillcrest|Pkwy|Suite 480|Wilson|Butte County|ID|86971|United States|-7|apartment| +31613|AAAAAAAANHLHAAAA|234|Park Johnson|Street|Suite 390|Pine Grove|Ector County|TX|74593|United States|-6|single family| +31614|AAAAAAAAOHLHAAAA|230|4th |Drive|Suite P|Bethel|Kinney County|TX|75281|United States|-6|single family| +31615|AAAAAAAAPHLHAAAA|156|Adams Fifth|Ct.|Suite 110|Union Hill|Beadle County|SD|57746|United States|-6|single family| +31616|AAAAAAAAAILHAAAA|412|Elm |Ln|Suite 100|Mountain View|Wilson County|KS|64466|United States|-6|single family| +31617|AAAAAAAABILHAAAA|962|Elevnth |Ct.|Suite 260|New Hope|Covington County|MS|59431|United States|-6|condo| +31618|AAAAAAAACILHAAAA|785|Johnson College|Street|Suite G|Lake Forest|Winona County|MN|56000|United States|-6|condo| +31619|AAAAAAAADILHAAAA||River Park|||Tabor|Transylvania County||28529|United States||| +31620|AAAAAAAAEILHAAAA|45|3rd |Road|Suite 70|Tremont|Teller County|CO|89515|United States|-7|apartment| +31621|AAAAAAAAFILHAAAA|645|West Park|Road|Suite 50|Scottsville|Livingston Parish|LA|74190|United States|-6|apartment| +31622|AAAAAAAAGILHAAAA|232|Church Hill|ST|Suite 200|Oak Grove|Ventura County|CA|98370|United States|-8|condo| +31623|AAAAAAAAHILHAAAA|670|Fifth |Ln|Suite 320|Ashland|Scott County|KS|64244|United States|-6|apartment| +31624|AAAAAAAAIILHAAAA|460|2nd Poplar|Boulevard|Suite 190|Woodlawn|Dauphin County|PA|14098|United States|-5|single family| +31625|AAAAAAAAJILHAAAA|807|Hickory |Boulevard|Suite 380|Shady Grove|Jackson County|MS|52812|United States|-6|single family| +31626|AAAAAAAAKILHAAAA|230|3rd |Wy|Suite H|Hillcrest|Judith Basin County|MT|63003|United States|-7|apartment| +31627|AAAAAAAALILHAAAA|608|Main Maple|Ct.|Suite T|Belmont|Oakland County|MI|40191|United States|-5|condo| +31628|AAAAAAAAMILHAAAA|992|Main Sunset|RD|Suite 310|Greenfield|Owyhee County|ID|85038|United States|-7|condo| +31629|AAAAAAAANILHAAAA|305|Johnson Mill|Parkway|Suite S|Oakland|Floyd County|IA|59843|United States|-6|apartment| +31630|AAAAAAAAOILHAAAA|34|First Lake|Street|Suite 260|Oak Hill|Wheeler County|NE|67838|United States|-7|single family| +31631|AAAAAAAAPILHAAAA|295|Spring |Way|Suite 450|White Oak|Trinity County|TX|76668|United States|-6|apartment| +31632|AAAAAAAAAJLHAAAA|530|Hillcrest Hillcrest|ST|Suite K|Highland|Richmond County||19454|United States|-5|condo| +31633|AAAAAAAABJLHAAAA|742|1st Washington|Drive|Suite G|Glendale|Douglas County|WA|93951|United States|-8|apartment| +31634|AAAAAAAACJLHAAAA|811|Dogwood |Road|Suite 420|Enterprise|Trigg County|KY|41757|United States|-5|apartment| +31635|AAAAAAAADJLHAAAA|401|Smith |Circle|Suite W|Walnut Grove|McIntosh County|GA|37752|United States|-5|condo| +31636|AAAAAAAAEJLHAAAA|536|Spring First|Pkwy|Suite 410|Stringtown|Bourbon County|KS|60162|United States|-6|apartment| +31637|AAAAAAAAFJLHAAAA|878|View |Road|Suite 280|Lakeview|Fleming County|KY|48579|United States|-6|single family| +31638|AAAAAAAAGJLHAAAA|265|Valley Cedar|Way|Suite 310|Georgetown|Cheyenne County|NE|67057|United States|-6|single family| +31639|AAAAAAAAHJLHAAAA|156|Park |Cir.|Suite O|Maple Grove|Lamoille County|VT|08852|United States|-5|single family| +31640|AAAAAAAAIJLHAAAA|249|Hillcrest Forest|Avenue|Suite V|Pleasant Grove|Adams County|IN|44136|United States|-5|single family| +31641|AAAAAAAAJJLHAAAA|90|11th |Drive|Suite 10|Bethel|Martin County|IN|45281|United States|-5|single family| +31642|AAAAAAAAKJLHAAAA|491|Lake 4th|Ct.|Suite 380|New Hope|Armstrong County|TX|79431|United States|-6|single family| +31643|AAAAAAAALJLHAAAA|80|Hillcrest |Ave|Suite 30|Pleasant Grove|Culpeper County|VA|24136|United States|-5|single family| +31644|AAAAAAAAMJLHAAAA|410|Jefferson Main|Way|Suite 330|Proctor|Wayne County|NE|68140|United States|-7|apartment| +31645|AAAAAAAANJLHAAAA|95|Dogwood |Ct.|Suite V|Bridgeport|Suffolk city|VA|25817|United States|-5|apartment| +31646|AAAAAAAAOJLHAAAA|553|River Ridge|Street|Suite U|Maple Grove|Montcalm County|MI|48252|United States|-5|apartment| +31647|AAAAAAAAPJLHAAAA|198|Lake |Ln|Suite 200|Springhill|Pulaski County|IL|64602|United States|-6|condo| +31648|AAAAAAAAAKLHAAAA|621|Franklin Main|Court|Suite E|Pleasant Valley|Washakie County|WY|82477|United States|-7|apartment| +31649|AAAAAAAABKLHAAAA|955|Cedar |ST|Suite W|Farmington|Seminole County|FL|39145|United States|-5|single family| +31650|AAAAAAAACKLHAAAA|944|Lakeview |Drive|Suite 470|Rosebud|Morrill County|NE|62244|United States|-7|condo| +31651|AAAAAAAADKLHAAAA|585|Main |ST|Suite 450|Wildwood|Concho County|TX|76871|United States|-6|single family| +31652|AAAAAAAAEKLHAAAA|470|Hill Washington|Pkwy|Suite E|Greenwood|Bacon County|GA|38828|United States|-5|apartment| +31653|AAAAAAAAFKLHAAAA|516|Poplar |RD|Suite 410|Antioch|Steele County|ND|58605|United States|-6|single family| +31654|AAAAAAAAGKLHAAAA|658|Oak |Ct.|Suite 300|Jackson|Bates County|MO|69583|United States|-6|single family| +31655|AAAAAAAAHKLHAAAA|614|8th Oak|Boulevard|Suite 150|Antioch|Gogebic County|MI|48605|United States|-5|condo| +31656|AAAAAAAAIKLHAAAA|163|Davis |Cir.|Suite X|Sulphur Springs|Colusa County|CA|98354|United States|-8|single family| +31657|AAAAAAAAJKLHAAAA|750||Parkway|Suite 10|Cherry Grove|Pendleton County|WV|21513|United States||| +31658|AAAAAAAAKKLHAAAA|589|East |Parkway|Suite 470|Shiloh|Montgomery County|KY|49275|United States|-5|single family| +31659|AAAAAAAALKLHAAAA|660|Central |Street|Suite G|Shady Grove|Cass County|ND|52812|United States|-6|condo| +31660|AAAAAAAAMKLHAAAA|318|Mill Wilson|Ln|Suite Q|Five Points|Briscoe County|TX|76098|United States|-6|condo| +31661|AAAAAAAANKLHAAAA|400|14th |Parkway|Suite 40|Walnut Grove|Cherokee County|TX|77752|United States|-6|single family| +31662|AAAAAAAAOKLHAAAA|350|Ridge |Road|Suite E|Friendship|Wagoner County|OK|74536|United States|-6|condo| +31663|AAAAAAAAPKLHAAAA|610|8th Main|Circle|Suite 270|Newport|San Bernardino County|CA|91521|United States|-8|apartment| +31664|AAAAAAAAALLHAAAA|827|9th Davis|Blvd|Suite 410|Riverdale|Johnson County|KS|69391|United States|-6|condo| +31665|AAAAAAAABLLHAAAA|549|Park 2nd|Drive|Suite 190|Enterprise|Richmond County|NC|21757|United States|-5|apartment| +31666|AAAAAAAACLLHAAAA|442|Spring Madison|Street|Suite 290|Antioch|Wayne County|OH|48605|United States|-5|condo| +31667|AAAAAAAADLLHAAAA|360|Church East|ST|Suite E|Farmington|Fremont County|ID|89145|United States|-7|single family| +31668|AAAAAAAAELLHAAAA|715|Spruce |Dr.|Suite X|Unionville|Shenandoah County|VA|21711|United States|-5|single family| +31669|AAAAAAAAFLLHAAAA|914|Poplar |Avenue|Suite A|Centerville|Nelson County|ND|50059|United States|-6|apartment| +31670|AAAAAAAAGLLHAAAA|219|Oak Sunset|Wy|Suite Q|Pine Grove|Blackford County|IN|44593|United States|-5|apartment| +31671|AAAAAAAAHLLHAAAA|154|Forest |Avenue|Suite 350|Salem|Seneca County|OH|48048|United States|-5|apartment| +31672|AAAAAAAAILLHAAAA|624|View Second|Wy|Suite G|Brookwood|Stanly County|NC|20965|United States|-5|condo| +31673|AAAAAAAAJLLHAAAA|133|Lincoln Railroad|Way|Suite 420|Union|Phillips County|KS|68721|United States|-6|condo| +31674|AAAAAAAAKLLHAAAA|705|Hickory |Ave|Suite 450|Forest Hills|Lamar County|TX|79237|United States|-6|apartment| +31675|AAAAAAAALLLHAAAA|361|Walnut View|Way|Suite N|Kimball|Will County|IL|63595|United States|-6|single family| +31676|AAAAAAAAMLLHAAAA|898|3rd Green|Boulevard|Suite K|Fairview|La Plata County|CO|85709|United States|-7|single family| +31677|AAAAAAAANLLHAAAA|831|Sycamore |RD|Suite 400|Lakeside|Kankakee County|IL|69532|United States|-6|apartment| +31678|AAAAAAAAOLLHAAAA|215|3rd Fourth|Parkway|Suite 440|Riverview|Essex County|NJ|09603|United States|-5|single family| +31679|AAAAAAAAPLLHAAAA|217|7th |Blvd|Suite N|Tremont|Ontario County|NY|19515|United States|-5|apartment| +31680|AAAAAAAAAMLHAAAA|871|Main |Blvd|Suite 290|Hillcrest|Douglas County|MO|63003|United States|-6|condo| +31681|AAAAAAAABMLHAAAA|111|Spring |ST|Suite 110|Golden|Orange County|TX|70411|United States|-6|apartment| +31682|AAAAAAAACMLHAAAA|464|Maple |Ave|Suite 30|Lone Oak|Woodruff County|AR|76893|United States|-6|apartment| +31683|AAAAAAAADMLHAAAA|858|Spring 11th|Boulevard|Suite S|Maple Grove|Missaukee County|MI|48252|United States|-5|condo| +31684|AAAAAAAAEMLHAAAA|455|South Cherry|Avenue|Suite 140|Longwood|Lincoln County|ME|07621|United States|-5|apartment| +31685|AAAAAAAAFMLHAAAA|753|14th |Blvd|Suite 360|Hillcrest|Beauregard Parish|LA|73003|United States|-6|apartment| +31686|AAAAAAAAGMLHAAAA|841|||Suite D|Salem||||United States||| +31687|AAAAAAAAHMLHAAAA||Church ||Suite C|Springfield|Stark County||||-6|condo| +31688|AAAAAAAAIMLHAAAA|486|Walnut 5th|Road|Suite C|Red Hill|Carter County|KY|44338|United States|-6|single family| +31689|AAAAAAAAJMLHAAAA|718|Sunset |Pkwy|Suite 140|Farmington|Calhoun County|SC|29145|United States|-5|apartment| +31690|AAAAAAAAKMLHAAAA|446|Jefferson |Circle|Suite M|Liberty|Lyon County|MN|53451|United States|-6|single family| +31691|AAAAAAAALMLHAAAA|402|Wilson North|Blvd|Suite 30|Macedonia|Edwards County|TX|71087|United States|-6|apartment| +31692|AAAAAAAAMMLHAAAA|419|1st |Street|Suite Y|Pleasant Grove|Young County|TX|74136|United States|-6|condo| +31693|AAAAAAAANMLHAAAA|654|First Meadow|Dr.|Suite 130|Star|Russell County|VA|20725|United States|-5|condo| +31694|AAAAAAAAOMLHAAAA|610|Poplar |Ln|Suite 360|Lincoln|Kingsbury County|SD|51289|United States|-7|single family| +31695|AAAAAAAAPMLHAAAA|67|Locust |Ln|Suite N|Oak Ridge|Flathead County|MT|68371|United States|-7|single family| +31696|AAAAAAAAANLHAAAA|25|River Hill|Avenue|Suite K|Salem|Leon County|FL|38048|United States|-5|condo| +31697|AAAAAAAABNLHAAAA|741|3rd |Lane|Suite R|Allison|Stewart County|TN|34167|United States|-6|apartment| +31698|AAAAAAAACNLHAAAA|171|South 2nd|Court|Suite 390|Barnes|Muskingum County|OH|43788|United States|-5|apartment| +31699|AAAAAAAADNLHAAAA|386|Hickory |Court|Suite 250|Lincoln|Marion County|MO|61289|United States|-6|apartment| +31700|AAAAAAAAENLHAAAA|287|Eigth |Way|Suite P|Lincoln|Licking County|OH|41289|United States|-5|condo| +31701|AAAAAAAAFNLHAAAA|505|Pine 14th|Dr.|Suite 10|Mechanicsburg|Albany County|NY|12219|United States|-5|condo| +31702|AAAAAAAAGNLHAAAA|177|View Hill|Way|Suite E|Stringtown|Decatur County|TN|30162|United States|-5|condo| +31703|AAAAAAAAHNLHAAAA|105|3rd |Boulevard|Suite 140|Lebanon|Elk County|PA|12898|United States|-5|apartment| +31704|AAAAAAAAINLHAAAA|632|Locust 2nd|Circle|Suite D|Union Hill|Charlton County|GA|37746|United States|-5|apartment| +31705|AAAAAAAAJNLHAAAA|520|Pine Oak|Drive|Suite H|Farmington|Smith County|KS|69145|United States|-6|single family| +31706|AAAAAAAAKNLHAAAA|247|Sunset Locust|Boulevard|Suite 60|Hamilton|Jefferson County|OK|72808|United States|-6|condo| +31707|AAAAAAAALNLHAAAA|929|View Forest|Ct.|Suite 220|Riverview|Alamance County|NC|29003|United States|-5|condo| +31708|AAAAAAAAMNLHAAAA|||Street|||Brown County|IL|63951||-6|single family| +31709|AAAAAAAANNLHAAAA|755|Elm |Cir.|Suite 460|Red Bank|Ripley County|MO|64975|United States|-6|single family| +31710|AAAAAAAAONLHAAAA|168|Smith |Lane|Suite 310|Ferguson|Bartholomew County|IN|41821|United States|-5|condo| +31711|AAAAAAAAPNLHAAAA|506|2nd Valley|Circle|Suite N|Mount Zion|Ashe County|NC|28054|United States|-5|single family| +31712|AAAAAAAAAOLHAAAA|799|7th Smith|Avenue|Suite H|Gum Springs|Lagrange County|IN|42106|United States|-5|single family| +31713|AAAAAAAABOLHAAAA|888|View |Ct.|Suite N|Newport|Saguache County|CO|81521|United States|-7|condo| +31714|AAAAAAAACOLHAAAA|804|Ninth |Street|Suite C|Oakwood|Boone County|NE|60169|United States|-6|condo| +31715|AAAAAAAADOLHAAAA|472|Sycamore |Blvd|Suite 0|Spring Valley|Calhoun County|MS|56060|United States|-6|single family| +31716|AAAAAAAAEOLHAAAA|145|East |Parkway|Suite 0|Wilson|Riley County|KS|66971|United States|-6|single family| +31717|AAAAAAAAFOLHAAAA|||Ct.|Suite 130||Dakota County|||||condo| +31718|AAAAAAAAGOLHAAAA|558|Spring |RD|Suite 10|Montpelier|Douglas County|KS|68930|United States|-6|apartment| +31719|AAAAAAAAHOLHAAAA|381|10th Center|Blvd|Suite X|Walnut Grove|Brown County|OH|47752|United States|-5|condo| +31720|AAAAAAAAIOLHAAAA|553|Cedar Jackson|Wy|Suite F|Concord|Bailey County|TX|74107|United States|-6|condo| +31721|AAAAAAAAJOLHAAAA|561|Park Lake|Cir.|Suite 10|Oneida|McDuffie County|GA|34027|United States|-5|condo| +31722|AAAAAAAAKOLHAAAA|540|Dogwood Pine|Cir.|Suite I|Harmony|Jackson County|AR|75804|United States|-6|single family| +31723|AAAAAAAALOLHAAAA|505|Sunset Meadow|ST|Suite F|Centerville|Davis County|UT|80059|United States|-7|apartment| +31724|AAAAAAAAMOLHAAAA|59|4th |Avenue|Suite 350|Union|Talladega County|AL|38721|United States|-6|condo| +31725|AAAAAAAANOLHAAAA|787|Laurel Smith|Court|Suite O|Woodville|||24289|||| +31726|AAAAAAAAOOLHAAAA|737|Walnut |Wy|Suite X|Woodlawn|Allegany County|MD|24098|United States|-5|apartment| +31727|AAAAAAAAPOLHAAAA|725|Chestnut Meadow|Cir.|Suite 360|Mount Pleasant|Little River County|AR|71933|United States|-6|apartment| +31728|AAAAAAAAAPLHAAAA|903|6th Davis|Ln|Suite 220|Fox|Clark County|OH|40631|United States|-5|single family| +31729|AAAAAAAABPLHAAAA|35|Main |ST|Suite 250|Providence|Platte County|NE|66614|United States|-7|apartment| +31730|AAAAAAAACPLHAAAA|886|Oak Hill|Way|Suite N|Jamestown|Jeff Davis County|GA|36867|United States|-5|condo| +31731|AAAAAAAADPLHAAAA|5|Meadow Main|Road|Suite 440|Newtown|Tripp County|SD|51749|United States|-7|apartment| +31732|AAAAAAAAEPLHAAAA|550|View 10th|Drive|Suite 270|Oakwood|Brunswick County|NC|20169|United States|-5|condo| +31733|AAAAAAAAFPLHAAAA|564|South Mill|Wy|Suite 190|Shiloh|Sumter County|SC|29275|United States|-5|condo| +31734|AAAAAAAAGPLHAAAA|682|Park Franklin|Parkway|Suite 450|Gilmore|Boyle County|KY|45464|United States|-6|single family| +31735|AAAAAAAAHPLHAAAA|885|2nd |Boulevard|Suite I|Mount Pleasant|Union County|PA|11933|United States|-5|single family| +31736|AAAAAAAAIPLHAAAA|436|Tenth West|Wy|Suite 410|Shiloh|Red Lake County|MN|59275|United States|-6|single family| +31737|AAAAAAAAJPLHAAAA|866|Hill 10th|Lane|Suite I|Macon|Clay County|NE|60369|United States|-6|single family| +31738|AAAAAAAAKPLHAAAA|763|Pine Oak|Road|Suite 110|Shady Grove|Lackawanna County|PA|12812|United States|-5|single family| +31739|AAAAAAAALPLHAAAA|121|13th |Ln|Suite T|Florence|Grafton County|NH|03994|United States|-5|condo| +31740|AAAAAAAAMPLHAAAA||3rd 11th|Court||Friendship|Carter County|MO||||single family| +31741|AAAAAAAANPLHAAAA|806|Birch |Way|Suite D|Stringtown|Ottawa County|MI|40162|United States|-5|apartment| +31742|AAAAAAAAOPLHAAAA|588|Williams |Cir.|Suite 440|Mount Zion|Laramie County|WY|88054|United States|-7|single family| +31743|AAAAAAAAPPLHAAAA|864|10th |Boulevard|Suite X|Green Acres|Queen Anne County|MD|27683|United States|-5|apartment| +31744|AAAAAAAAAAMHAAAA|434|North Spruce|Court|Suite D|Oak Grove|Brunswick County|NC|28370|United States|-5|single family| +31745|AAAAAAAABAMHAAAA|985|4th Washington|Circle|Suite P|Harmony|Fulton County|OH|45804|United States|-5|apartment| +31746|AAAAAAAACAMHAAAA|721|Railroad |Road|Suite 390|Wilson|Cache County|UT|86971|United States|-7|apartment| +31747|AAAAAAAADAMHAAAA|460|Park Lake|Blvd|Suite V|Walnut Grove|Edwards County|KS|67752|United States|-6|single family| +31748|AAAAAAAAEAMHAAAA|728|5th 8th|Wy|Suite M|Granite|Spink County|SD|56284|United States|-7|apartment| +31749|AAAAAAAAFAMHAAAA|39|1st |Lane|Suite Y|Fairfield|Salem city|VA|26192|United States|-5|condo| +31750|AAAAAAAAGAMHAAAA|315|Mill Valley|Ln|Suite E|Springdale|Erie County|NY|18883|United States|-5|single family| +31751|AAAAAAAAHAMHAAAA|907|Walnut |RD|Suite 280|Mount Zion|Spencer County|IN|48054|United States|-5|single family| +31752|AAAAAAAAIAMHAAAA|91|Cherry Sycamore|Dr.|Suite 360|Five Points|Emporia city|VA|26098|United States|-5|single family| +31753|AAAAAAAAJAMHAAAA|387|Main |Wy|Suite 40|Kingston|Sherman County|KS|64975|United States|-6|apartment| +31754|AAAAAAAAKAMHAAAA|314|||Suite 120||||63604|United States||| +31755|AAAAAAAALAMHAAAA|504|2nd |Lane|Suite H|Springdale|Quitman County|MS|58883|United States|-6|apartment| +31756|AAAAAAAAMAMHAAAA|760|5th |Way|Suite 310|Buena Vista|Tama County|IA|55752|United States|-6|single family| +31757|AAAAAAAANAMHAAAA|618|River |Lane|Suite 280|Summit|Aroostook County|ME|01099|United States|-5|condo| +31758|AAAAAAAAOAMHAAAA|415|Laurel |Avenue|Suite 430|Oakdale|Tioga County|NY|19584|United States|-5|apartment| +31759|AAAAAAAAPAMHAAAA|602|Center Dogwood|Cir.|Suite 190|Friendship|Dewey County|OK|74536|United States|-6|single family| +31760|AAAAAAAAABMHAAAA|932|3rd |Boulevard|Suite G|Five Points|Pennington County|SD|56098|United States|-7|condo| +31761|AAAAAAAABBMHAAAA|870|Pine |Parkway|Suite E|Salem|Roanoke County|VA|28048|United States|-5|condo| +31762|AAAAAAAACBMHAAAA|889|Mill Elm|Parkway|Suite K|Pleasant Hill|Elbert County|GA|33604|United States|-5|apartment| +31763|AAAAAAAADBMHAAAA|221|Cedar Railroad|Circle|Suite I|Huntsville|Norton city|VA|27570|United States|-5|apartment| +31764|AAAAAAAAEBMHAAAA|743|Franklin 15th|Road|Suite E|Belmont|Haralson County|GA|30191|United States|-5|condo| +31765|AAAAAAAAFBMHAAAA|651|Birch Spring|Blvd|Suite P|Sulphur Springs|Barrow County|GA|38354|United States|-5|single family| +31766|AAAAAAAAGBMHAAAA|628|Madison |Lane|Suite M|Bunker Hill|Traill County|ND|50150|United States|-6|apartment| +31767|AAAAAAAAHBMHAAAA|831|Lakeview Wilson|Street|Suite X|Jamestown|Surry County|NC|26867|United States|-5|apartment| +31768|AAAAAAAAIBMHAAAA|477|Lake Hill|Ln|Suite J|Shady Grove|Sheboygan County|WI|52812|United States|-6|single family| +31769|AAAAAAAAJBMHAAAA|974|7th North|Dr.|Suite 330|Franklin|Benton County|MN|59101|United States|-6|condo| +31770|AAAAAAAAKBMHAAAA|281|8th 3rd|RD|Suite 80|Jamestown|Christian County|IL|66867|United States|-6|apartment| +31771|AAAAAAAALBMHAAAA|867|Second 10th|ST|Suite 20|Sunnyside|Penobscot County|ME|02552|United States|-5|apartment| +31772|AAAAAAAAMBMHAAAA|283|Cedar |Avenue|Suite 180|Fairfield|Rockingham County|NH|06792|United States|-5|single family| +31773|AAAAAAAANBMHAAAA|360|Sunset Hill|Avenue|Suite O|Owens|Madera County|CA|92324|United States|-8|condo| +31774|AAAAAAAAOBMHAAAA|443|First Elm|Pkwy|Suite 290|Woodville|Guilford County|NC|24289|United States|-5|condo| +31775|AAAAAAAAPBMHAAAA|214|Lincoln 11th|Pkwy|Suite 490|Bunker Hill|Bossier Parish|LA|70150|United States|-6|condo| +31776|AAAAAAAAACMHAAAA|315|Hillcrest |Wy|Suite L|Farmington|Iredell County|NC|29145|United States|-5|condo| +31777|AAAAAAAABCMHAAAA|188|Wilson 3rd|Cir.|Suite K|Sunnyside|Crook County|OR|91952|United States|-8|condo| +31778|AAAAAAAACCMHAAAA|570|Central Spring|Ave|Suite A|Jackson|Harris County|GA|39583|United States|-5|single family| +31779|AAAAAAAADCMHAAAA|382|Main |Wy|Suite C|Shannon|Grand County|UT|84120|United States|-7|single family| +31780|AAAAAAAAECMHAAAA|681|Main |Court|Suite S|Liberty|Warren County|KY|43451|United States|-5|condo| +31781|AAAAAAAAFCMHAAAA|393|15th |Ln|Suite 240|Birmingham|Graham County|KS|63372|United States|-6|condo| +31782|AAAAAAAAGCMHAAAA|967|Elm Ridge|Avenue|Suite 140|Shady Grove|Armstrong County|PA|12812|United States|-5|single family| +31783|AAAAAAAAHCMHAAAA|805|Oak 5th|Cir.|Suite O|Phoenix|Acadia Parish|LA|72276|United States|-6|apartment| +31784|AAAAAAAAICMHAAAA|615|2nd Lake|Cir.|Suite 480|Oak Ridge|Davie County|NC|28371|United States|-5|single family| +31785|AAAAAAAAJCMHAAAA|878|Tenth |RD|Suite 30|Crossroads|DeWitt County|TX|70534|United States|-6|condo| +31786|AAAAAAAAKCMHAAAA|973|4th Park|Court|Suite 250|Five Points|Chowan County|NC|26098|United States|-5|apartment| +31787|AAAAAAAALCMHAAAA|958|4th 1st|Ct.|Suite H|Mountain View|Johnson County|KY|44466|United States|-6|condo| +31788|AAAAAAAAMCMHAAAA|442|South West|Blvd|Suite 120|Woodville|Santa Barbara County|CA|94289|United States|-8|apartment| +31789|AAAAAAAANCMHAAAA|803|3rd |Ln|Suite L|Lakeside|Montague County|TX|79532|United States|-6|apartment| +31790|AAAAAAAAOCMHAAAA|849|11th |Blvd|Suite H|Springfield|Mercer County|IL|69303|United States|-6|single family| +31791|AAAAAAAAPCMHAAAA|54|Valley |RD|Suite K|Newtown|Keweenaw County|MI|41749|United States|-5|apartment| +31792|AAAAAAAAADMHAAAA|14|10th 4th|Drive|Suite 240|Oak Ridge|Pike County|MO|68371|United States|-6|apartment| +31793|AAAAAAAABDMHAAAA|685|Davis |Road|Suite 330|Hopewell|Grant County|KS|60587|United States|-6|single family| +31794|AAAAAAAACDMHAAAA|508|14th |Ln|Suite 260|Valley View|Arthur County|NE|65124|United States|-6|apartment| +31795|AAAAAAAADDMHAAAA|248|5th |Cir.|Suite G|Jackson|East Carroll Parish|LA|79583|United States|-6|single family| +31796|AAAAAAAAEDMHAAAA|407|Park 2nd|Ln|Suite N|Mount Olive|Henderson County|NC|28059|United States|-5|condo| +31797|AAAAAAAAFDMHAAAA|218|Birch Cedar|Cir.|Suite 410|Union Hill|Grundy County|IA|57746|United States|-6|apartment| +31798|AAAAAAAAGDMHAAAA|320|2nd |Street|Suite U|Oakwood|Seward County|KS|60169|United States|-6|condo| +31799|AAAAAAAAHDMHAAAA|830|Maple Park|Wy|Suite P|Shady Grove|Franklin County|IL|62812|United States|-6|condo| +31800|AAAAAAAAIDMHAAAA|726|13th |Wy|Suite 260|Cumberland|Sumter County|AL|38971|United States|-6|apartment| +31801|AAAAAAAAJDMHAAAA|688|4th |Ln|Suite H|Hillcrest|Comanche County|KS|63003|United States|-6|single family| +31802|AAAAAAAAKDMHAAAA|211|7th |Parkway|Suite 480|Union Hill|Cabell County|WV|27746|United States|-5|condo| +31803|AAAAAAAALDMHAAAA|70|8th |RD|Suite 290|Saint Johns|Wilkin County|MN|55717|United States|-6|single family| +31804|AAAAAAAAMDMHAAAA|492|Lakeview Third|Dr.|Suite N|Concord|Pepin County|WI|54107|United States|-6|single family| +31805|AAAAAAAANDMHAAAA|777|Mill |Ave|Suite C|Hopewell|Scott County|TN|30587|United States|-6|single family| +31806|AAAAAAAAODMHAAAA|676|Laurel |ST|Suite K|Mount Vernon|Palo Alto County|IA|58482|United States|-6|apartment| +31807|AAAAAAAAPDMHAAAA|990|5th |Drive|Suite N|Hillcrest|Union County|SC|23003|United States|-5|condo| +31808|AAAAAAAAAEMHAAAA|241|2nd |Avenue|Suite 460|Glendale|Atchison County|KS|63951|United States|-6|apartment| +31809|AAAAAAAABEMHAAAA|751|Broadway Cherry|Avenue|Suite 80|Mount Pleasant|Wilson County|KS|61933|United States|-6|condo| +31810|AAAAAAAACEMHAAAA|451|Park |Lane|Suite F|Glenwood|Pike County|AL|33511|United States|-6|condo| +31811|AAAAAAAADEMHAAAA|97|Center Cedar|RD|Suite 400|Highland|Yazoo County|MS|59454|United States|-6|condo| +31812|AAAAAAAAEEMHAAAA|834|14th 8th|Road|Suite 470|Spring Hill|Holmes County|OH|46787|United States|-5|single family| +31813|AAAAAAAAFEMHAAAA|39|Highland |Parkway|Suite 70|Ashland|Hamlin County|SD|54244|United States|-7|condo| +31814|AAAAAAAAGEMHAAAA|389|Hickory College|RD|Suite G|Greenwood|Clay County|FL|38828|United States|-5|apartment| +31815|AAAAAAAAHEMHAAAA|397|Walnut Hill|Lane|Suite 340|Centerville|Williamsburg city|VA|20059|United States|-5|single family| +31816|AAAAAAAAIEMHAAAA||Madison |Ct.||Summit|Trimble County|||||| +31817|AAAAAAAAJEMHAAAA|805|Washington |Drive|Suite 0|Oakland|Panola County|MS|59843|United States|-6|apartment| +31818|AAAAAAAAKEMHAAAA|644|Meadow Pine|Pkwy|Suite 110|Harmony|Bledsoe County|TN|35804|United States|-5|single family| +31819|AAAAAAAALEMHAAAA|172|Lincoln North|Road|Suite O|Cedar Grove|Saratoga County|NY|10411|United States|-5|apartment| +31820|AAAAAAAAMEMHAAAA|74|Sunset |Blvd|Suite 350|Pine Grove|Scotland County|MO|64593|United States|-6|apartment| +31821|AAAAAAAANEMHAAAA|914|Fourth 10th|Drive|Suite F|Ashland|Harrison County|OH|44244|United States|-5|apartment| +31822|AAAAAAAAOEMHAAAA|472|7th |Street|Suite 80|Springdale|Kendall County|IL|68883|United States|-6|apartment| +31823|AAAAAAAAPEMHAAAA|415|Center |Avenue|Suite 150|Mountain View|Lenoir County|NC|24466|United States|-5|single family| +31824|AAAAAAAAAFMHAAAA|60|3rd |Drive|Suite J|Walnut Grove|Midland County|TX|77752|United States|-6|condo| +31825|AAAAAAAABFMHAAAA|416|College |Dr.|Suite O|Warwick|Robertson County|TX|71398|United States|-6|single family| +31826|AAAAAAAACFMHAAAA|460|Willow |Dr.|Suite 170|Riverside|Colleton County|SC|29231|United States|-5|single family| +31827|AAAAAAAADFMHAAAA|736|Cedar |Parkway|Suite F|Franklin|Keweenaw County|MI|49101|United States|-5|apartment| +31828|AAAAAAAAEFMHAAAA|401|Miller 11th|Cir.|Suite B|Sunnyside|Polk County|NC|21952|United States|-5|apartment| +31829|AAAAAAAAFFMHAAAA|182|Cherry |Dr.|Suite L|Unionville|O-Brien County|IA|51711|United States|-6|single family| +31830|AAAAAAAAGFMHAAAA|696|3rd Valley|Lane|Suite G|Deerfield|Androscoggin County|ME|09840|United States|-5|single family| +31831|AAAAAAAAHFMHAAAA||View |Avenue||Franklin|Blanco County|||||| +31832|AAAAAAAAIFMHAAAA|890|4th |Lane|Suite L|Plainview|Jackson County|TN|33683|United States|-5|single family| +31833|AAAAAAAAJFMHAAAA|624|Pine Wilson|Ct.|Suite 380|Marion|Boyd County|NE|60399|United States|-6|apartment| +31834|AAAAAAAAKFMHAAAA|910|Forest |Lane|Suite 20|Hillcrest|Pemiscot County|MO|63003|United States|-6|condo| +31835|AAAAAAAALFMHAAAA|477|7th Smith|Lane|Suite K|Liberty|Box Elder County|UT|83451|United States|-7|apartment| +31836|AAAAAAAAMFMHAAAA|66|Elm |Drive|Suite 10|Greenwood|Grand Traverse County|MI|48828|United States|-5|condo| +31837|AAAAAAAANFMHAAAA|372|Jefferson Miller|RD|Suite G|Bunker Hill|Faribault County|MN|50150|United States|-6|condo| +31838|AAAAAAAAOFMHAAAA|563|Adams |Pkwy|Suite 130|Bunker Hill|Chenango County|NY|10150|United States|-5|single family| +31839|AAAAAAAAPFMHAAAA|158|Pine |Avenue|Suite 120|Sunnyside|Custer County|SD|51952|United States|-6|apartment| +31840|AAAAAAAAAGMHAAAA|343|Mill Elm|Road|Suite 490|Granite|Caldwell County|TX|76284|United States|-6|apartment| +31841|AAAAAAAABGMHAAAA|4|Hillcrest |Blvd|Suite 290|Bridgeport|Burke County|NC|25817|United States|-5|condo| +31842|AAAAAAAACGMHAAAA|152|Park |Dr.|Suite R|Fairfield|Dallas County|AL|36192|United States|-6|apartment| +31843|AAAAAAAADGMHAAAA||Walnut |Blvd|||||76534||-6|| +31844|AAAAAAAAEGMHAAAA|835|3rd Jackson|Boulevard|Suite R|Crystal|Wakulla County|FL|35258|United States|-5|apartment| +31845|AAAAAAAAFGMHAAAA|2|Hillcrest Dogwood|Ct.|Suite Y|Five Points|Pointe Coupee Parish|LA|76098|United States|-6|condo| +31846|AAAAAAAAGGMHAAAA|17|Locust |Pkwy|Suite 480|Buckingham|Tom Green County|TX|74092|United States|-6|condo| +31847|AAAAAAAAHGMHAAAA|109|Oak 3rd|Ct.|Suite E|Lakeview|Perry County|TN|38579|United States|-6|condo| +31848|AAAAAAAAIGMHAAAA|255|Main |ST|Suite 130|Murphy|Walworth County|WI|52105|United States|-6|condo| +31849|AAAAAAAAJGMHAAAA|496|First 5th|Boulevard|Suite 70|Walnut Grove|Washington County|IN|47752|United States|-5|condo| +31850|AAAAAAAAKGMHAAAA|445|3rd Hill|Dr.|Suite 160|Union Hill|Frederick County|VA|27746|United States|-5|apartment| +31851|AAAAAAAALGMHAAAA|367|Walnut View|Wy|Suite J||Randolph County||48371||-5|| +31852|AAAAAAAAMGMHAAAA|742|2nd |Road|Suite 470|Clifton|Colfax County|NM|88014|United States|-7|single family| +31853|AAAAAAAANGMHAAAA|740|Main |Road|Suite D|Stringtown|Lake County|SD|50162|United States|-7|single family| +31854|AAAAAAAAOGMHAAAA|149|11th |Ave|Suite W|Midway|Park County|MT|61904|United States|-7|condo| +31855|AAAAAAAAPGMHAAAA|338|Pine Twelfth|Court|Suite 380|Marion|Ransom County|ND|50399|United States|-6|condo| +31856|AAAAAAAAAHMHAAAA|874|Church |Avenue|Suite 250|Greenville|Halifax County|VA|21387|United States|-5|condo| +31857|AAAAAAAABHMHAAAA|699|Lincoln |ST|Suite M|Walnut Grove|Gage County|NE|67752|United States|-6|single family| +31858|AAAAAAAACHMHAAAA|190|Sunset |Avenue|Suite 200|Newport|Lincoln County|WA|91521|United States|-8|condo| +31859|AAAAAAAADHMHAAAA|66|Meadow |ST|Suite O|Greendale|Barnwell County|SC|29343|United States|-5|apartment| +31860|AAAAAAAAEHMHAAAA|472|Spruce |Blvd|Suite 210|Arlington|Greenup County|KY|46557|United States|-6|condo| +31861|AAAAAAAAFHMHAAAA|766|3rd |Ave|Suite 380|Pleasant Valley|Madison County|AL|32477|United States|-6|apartment| +31862|AAAAAAAAGHMHAAAA|402|Second |ST|Suite V|Red Hill|Meade County|KS|64338|United States|-6|condo| +31863|AAAAAAAAHHMHAAAA|744|Ash Wilson|Circle|Suite 410|Shady Grove|Neosho County|KS|62812|United States|-6|single family| +31864|AAAAAAAAIHMHAAAA|174|Hill |Wy|Suite 380|Lakeview|McKean County|PA|18579|United States|-5|apartment| +31865|AAAAAAAAJHMHAAAA|454|3rd |Boulevard|Suite 210|Salem|Texas County|MO|68048|United States|-6|single family| +31866|AAAAAAAAKHMHAAAA|533|12th South|Ave|Suite V|Superior|Liberty County|GA|32562|United States|-5|single family| +31867|AAAAAAAALHMHAAAA|117|8th |Ct.|Suite 210|Springdale|Lamar County|AL|38883|United States|-6|single family| +31868|AAAAAAAAMHMHAAAA|625|West 3rd|Blvd|Suite 300|Lewisburg|Rockcastle County|KY|47538|United States|-5|condo| +31869|AAAAAAAANHMHAAAA|442|5th |Lane|Suite M|Woodville|Gunnison County|CO|84289|United States|-7|single family| +31870|AAAAAAAAOHMHAAAA|118|Park |Ct.|Suite L|Waterloo|Henderson County|KY|41675|United States|-6|apartment| +31871|AAAAAAAAPHMHAAAA|441|Oak |Avenue|Suite N|Oak Ridge|Holmes County|FL|38371|United States|-5|apartment| +31872|AAAAAAAAAIMHAAAA|974|4th |ST|Suite A|Union|Patrick County|VA|28721|United States|-5|apartment| +31873|AAAAAAAABIMHAAAA|146|Pine |Street|Suite W|Farmington|Giles County|VA|29145|United States|-5|single family| +31874|AAAAAAAACIMHAAAA|549|Mill |ST|Suite 20|Phoenix|Doniphan County|KS|62276|United States|-6|condo| +31875|AAAAAAAADIMHAAAA||3rd ||||DeKalb County|MO|||-6|| +31876|AAAAAAAAEIMHAAAA|813|Laurel |Way|Suite A|Marion|Maury County|TN|30399|United States|-6|apartment| +31877|AAAAAAAAFIMHAAAA|904|Center |Parkway|Suite 140|Pleasant Grove|Knox County|MO|64136|United States|-6|apartment| +31878|AAAAAAAAGIMHAAAA|323||Parkway||Glendale||||United States||condo| +31879|AAAAAAAAHIMHAAAA|745|Lee Second|Parkway|Suite 360|Centerville|Marshall County|MN|50059|United States|-6|single family| +31880|AAAAAAAAIIMHAAAA|728|Cedar |Street|Suite 330|Oak Hill|Hopewell city|VA|27838|United States|-5|single family| +31881|AAAAAAAAJIMHAAAA|896|River Elm|Ct.|Suite S|Clinton|Long County|GA|38222|United States|-5|condo| +31882|AAAAAAAAKIMHAAAA|290|10th 10th|Way|Suite K|Springfield|Lowndes County|GA|39303|United States|-5|apartment| +31883|AAAAAAAALIMHAAAA|554|Washington Miller|Ct.|Suite E|Bethel|Van Buren County|TN|35281|United States|-6|single family| +31884|AAAAAAAAMIMHAAAA|539|1st Maple|Road|Suite 80|Woodland|Chatham County|NC|24854|United States|-5|condo| +31885|AAAAAAAANIMHAAAA|964|3rd Franklin|Street|Suite F|Deerfield|Yamhill County|OR|99840|United States|-8|condo| +31886|AAAAAAAAOIMHAAAA|102|12th 5th|Pkwy|Suite O|Greenwood|Marshall County|IA|58828|United States|-6|single family| +31887|AAAAAAAAPIMHAAAA|614|Highland |Cir.|Suite 210|Florence|Monroe County|AR|73394|United States|-6|condo| +31888|AAAAAAAAAJMHAAAA|931|Hill |Boulevard|Suite P|Greenville|Benzie County|MI|41387|United States|-5|apartment| +31889|AAAAAAAABJMHAAAA|202|Park Church|Parkway|Suite 460|Lakeview|Rice County|MN|58579|United States|-6|apartment| +31890|AAAAAAAACJMHAAAA|534|Oak |Dr.|Suite 440|Sleepy Hollow|Carroll County|IL|63592|United States|-6|apartment| +31891|AAAAAAAADJMHAAAA|95|11th |Boulevard|Suite G|Sulphur Springs|Harrison County|MO|68354|United States|-6|apartment| +31892|AAAAAAAAEJMHAAAA|87|Madison 4th|Avenue|Suite 60|Macedonia|Upshur County|WV|21087|United States|-5|condo| +31893|AAAAAAAAFJMHAAAA|75|Poplar |Blvd|Suite 200|Glendale|Coconino County|AZ|83951|United States|-7|single family| +31894|AAAAAAAAGJMHAAAA|649|Park College|Circle|Suite 480|Mount Vernon|Webster County|MO|68482|United States|-6|single family| +31895|AAAAAAAAHJMHAAAA|307|Jackson Lee|Blvd|Suite 360|Providence|Iosco County|MI|46614|United States|-5|single family| +31896|AAAAAAAAIJMHAAAA|352|Maple Sixth|Cir.|Suite G|Woodville|Merrimack County|NH|04889|United States|-5|condo| +31897|AAAAAAAAJJMHAAAA||Walnut ||Suite N|Spring Valley|San Benito County|CA||United States|-8|| +31898|AAAAAAAAKJMHAAAA|237|Chestnut Main|Wy|Suite Q|Woodville|Sequatchie County|TN|34289|United States|-6|condo| +31899|AAAAAAAALJMHAAAA|83|Valley 4th|RD|Suite N|Buena Vista|Woodford County|IL|65752|United States|-6|condo| +31900|AAAAAAAAMJMHAAAA|822|3rd |Lane|Suite 30|Collinsville|Posey County|IN|42459|United States|-5|single family| +31901|AAAAAAAANJMHAAAA|111|Hickory |Pkwy|Suite 200|Walnut Grove|Wayne County|MI|47752|United States|-5|single family| +31902|AAAAAAAAOJMHAAAA|260|Wilson Wilson|Ave|Suite L|Midway|Taylor County|KY|41904|United States|-5|apartment| +31903|AAAAAAAAPJMHAAAA|136|5th 5th|Boulevard|Suite P|Blair|Wythe County|VA|25465|United States|-5|condo| +31904|AAAAAAAAAKMHAAAA|583|Third ||Suite H||||||-6|single family| +31905|AAAAAAAABKMHAAAA|12|Miller |Lane|Suite N|Unionville|Hughes County|SD|51711|United States|-7|apartment| +31906|AAAAAAAACKMHAAAA|230|Park |RD|Suite 20|Shady Grove|Oregon County|MO|62812|United States|-6|apartment| +31907|AAAAAAAADKMHAAAA|825|Lee |Lane|Suite V|Oak Ridge|Baldwin County|AL|38371|United States|-6|single family| +31908|AAAAAAAAEKMHAAAA|367|6th First|Parkway|Suite 330|Mount Zion|Gosper County|NE|68054|United States|-6|condo| +31909|AAAAAAAAFKMHAAAA|||Court|Suite P||Peoria County|IL|68371|||| +31910|AAAAAAAAGKMHAAAA|582|Lincoln |Ct.|Suite 110|Hillcrest|Smyth County|VA|23003|United States|-5|condo| +31911|AAAAAAAAHKMHAAAA|||Parkway|Suite 440||Morgan County|||||apartment| +31912|AAAAAAAAIKMHAAAA|121|Third Maple|ST|Suite J|Pleasant Valley|Sargent County|ND|52477|United States|-6|apartment| +31913|AAAAAAAAJKMHAAAA|82|Pine |Wy|Suite 110|Union|Jackson County|KS|68721|United States|-6|single family| +31914|AAAAAAAAKKMHAAAA|497|Hillcrest 5th|Avenue|Suite F|Greenville|Swisher County|TX|71387|United States|-6|apartment| +31915|AAAAAAAALKMHAAAA|370|Elm |Parkway|Suite 280|Greenwood|Navajo County|AZ|88828|United States|-7|condo| +31916|AAAAAAAAMKMHAAAA|696|Forest |Ln|Suite I|Edgewood|Lewis and Clark County|MT|60069|United States|-7|single family| +31917|AAAAAAAANKMHAAAA|473|Washington |Drive|Suite 170|Valley View|Ellis County|KS|65124|United States|-6|apartment| +31918|AAAAAAAAOKMHAAAA|858|13th |Road|Suite 310|Deerfield|Indian River County|FL|39840|United States|-5|apartment| +31919|AAAAAAAAPKMHAAAA|26|Oak River|Avenue|Suite X|Concord|Guthrie County|IA|54107|United States|-6|single family| +31920|AAAAAAAAALMHAAAA|62|Meadow |Street|Suite 360|Kingston|Langlade County|WI|54975|United States|-6|single family| +31921|AAAAAAAABLMHAAAA|420|Chestnut River|Road|Suite J|Mount Pleasant|Cedar County|MO|61933|United States|-6|single family| +31922|AAAAAAAACLMHAAAA|545|View |Avenue|Suite B|Fairview|Nicholas County|KY|45709|United States|-5|condo| +31923|AAAAAAAADLMHAAAA|432|Central |Court|Suite 40|Salem|Meriwether County|GA|38048|United States|-5|single family| +31924|AAAAAAAAELMHAAAA|256|West |Ln|Suite Y|Harmony|Cass County|NE|65804|United States|-6|condo| +31925|AAAAAAAAFLMHAAAA||East Park||Suite 300|Farmington||KY|49145|United States||apartment| +31926|AAAAAAAAGLMHAAAA|224|Spring Hickory|Lane|Suite F|Somerville|York County|PA|17783|United States|-5|condo| +31927|AAAAAAAAHLMHAAAA|200|Church |Lane|Suite U|Bethel|Perry County|MS|55281|United States|-6|apartment| +31928|AAAAAAAAILMHAAAA|740|Main Birch|Wy|Suite 300|Highland|Cassia County|ID|89454|United States|-7|single family| +31929|AAAAAAAAJLMHAAAA|918|River 7th|Court|Suite O|Highland Park|Dickinson County|MI|46534|United States|-5|apartment| +31930|AAAAAAAAKLMHAAAA|862|Mill Spruce|Cir.|Suite Y|Maple Grove|Livingston County|NY|18252|United States|-5|condo| +31931|AAAAAAAALLMHAAAA|643|10th |ST|Suite C|Lakewood|Aransas County|TX|78877|United States|-6|single family| +31932|AAAAAAAAMLMHAAAA|1|Sycamore 12th|Avenue|Suite 250|Highland Park|Reagan County|TX|76534|United States|-6|condo| +31933|AAAAAAAANLMHAAAA|768|Locust |Drive|Suite 440|Wildwood|Callaway County|MO|66871|United States|-6|single family| +31934|AAAAAAAAOLMHAAAA|755|Woodland |Parkway|Suite G|Summit|Union County|NM|80499|United States|-7|apartment| +31935|AAAAAAAAPLMHAAAA|870|Pine |Way|Suite 230|Spring Hill|Barbour County|AL|36787|United States|-6|apartment| +31936|AAAAAAAAAMMHAAAA|228|South 1st|Ct.|Suite Q|Oak Hill|Boise County|ID|87838|United States|-7|apartment| +31937|AAAAAAAABMMHAAAA|671|Green Third|Way|Suite 60|Clifton|Levy County|FL|38014|United States|-5|apartment| +31938|AAAAAAAACMMHAAAA|812|First 10th|Ln|Suite H|Woodland|Spotsylvania County|VA|24854|United States|-5|apartment| +31939|AAAAAAAADMMHAAAA|216|Lake |Boulevard|Suite K|Liberty|Mitchell County|TX|73451|United States|-6|condo| +31940|AAAAAAAAEMMHAAAA|963|Locust 3rd|Ln|Suite 410|Belmont|Washington County|AR|70191|United States|-6|single family| +31941|AAAAAAAAFMMHAAAA|798|Green 3rd|Drive|Suite 460|Brownsville|Washington County|KY|49310|United States|-5|single family| +31942|AAAAAAAAGMMHAAAA|728|Maple Chestnut|Blvd|Suite E|Bunker Hill|Menifee County|KY|40150|United States|-5|single family| +31943|AAAAAAAAHMMHAAAA|395|Cedar |Road|Suite S|Mountain View|Suwannee County|FL|34466|United States|-5|apartment| +31944|AAAAAAAAIMMHAAAA|307|10th Church|RD|Suite R|Shady Grove|Seward County|NE|62812|United States|-7|apartment| +31945|AAAAAAAAJMMHAAAA|224|5th |Road|Suite 50|Midway|Stanton County|KS|61904|United States|-6|apartment| +31946|AAAAAAAAKMMHAAAA|418|Mill |Dr.|Suite 160|Lakewood|Auglaize County|OH|48877|United States|-5|condo| +31947|AAAAAAAALMMHAAAA|414|Spruce North|Parkway|Suite 430|Woodlawn|Johnson County|WY|84098|United States|-7|apartment| +31948|AAAAAAAAMMMHAAAA|768|2nd |Avenue|Suite 230|Belmont|Prince William County|VA|20191|United States|-5|single family| +31949|AAAAAAAANMMHAAAA|263|Hill |Ct.|Suite R|Lincoln|Jersey County|IL|61289|United States|-6|apartment| +31950|AAAAAAAAOMMHAAAA|329|South Elm|Ln|Suite M|Woodland|Cayuga County|NY|14854|United States|-5|single family| +31951|AAAAAAAAPMMHAAAA|303|Tenth |Ct.|Suite 140|Hamilton|Yellowstone County|MT|62808|United States|-7|single family| +31952|AAAAAAAAANMHAAAA|792|Main Church|Lane|Suite E|Mount Pleasant|Ross County|OH|41933|United States|-5|single family| +31953|AAAAAAAABNMHAAAA|434|Locust Main|Street|Suite M|Clinton|Jefferson County|OR|98222|United States|-8|apartment| +31954|AAAAAAAACNMHAAAA|178|Maple Green|Cir.|Suite O|Woodland Park|Middlesex County|VA|21934|United States|-5|apartment| +31955|AAAAAAAADNMHAAAA|141|14th Church|Circle|Suite 90|Green Acres|White County|AR|77683|United States|-6|condo| +31956|AAAAAAAAENMHAAAA|116|3rd Second|RD|Suite 240|Stringtown|Calhoun County|IA|50162|United States|-6|apartment| +31957|AAAAAAAAFNMHAAAA|712|Chestnut Sunset|Wy|Suite Q|Union|Owyhee County|ID|88721|United States|-7|single family| +31958|AAAAAAAAGNMHAAAA|864|Davis Mill|Avenue|Suite 180|Mountain View|Lincoln County|CO|84466|United States|-7|single family| +31959|AAAAAAAAHNMHAAAA|313|Highland |Avenue|Suite Q|Lebanon|Amherst County|VA|22898|United States|-5|single family| +31960|AAAAAAAAINMHAAAA|185|Madison |Court|Suite B|Crossroads|Shelby County|TX|70534|United States|-6|single family| +31961|AAAAAAAAJNMHAAAA|292|Main 6th|Cir.|Suite 270|Oak Grove|Bonner County|ID|88370|United States|-7|apartment| +31962|AAAAAAAAKNMHAAAA|290|Hill |RD|Suite J|Kingston|Metcalfe County|KY|44975|United States|-5|single family| +31963|AAAAAAAALNMHAAAA|34|Lakeview Sycamore|RD|Suite P|Oakdale|Houston County|TX|79584|United States|-6|condo| +31964|AAAAAAAAMNMHAAAA|707|Ridge |Avenue|Suite 50|Highland|Campbell County|TN|39454|United States|-5|condo| +31965|AAAAAAAANNMHAAAA|783|Cherry Oak|Blvd|Suite O|Cedar Grove|Lunenburg County|VA|20411|United States|-5|condo| +31966|AAAAAAAAONMHAAAA|819|Poplar |Ct.|Suite S|Providence|Hutchinson County|SD|56614|United States|-7|condo| +31967|AAAAAAAAPNMHAAAA|659|South Third|Street|Suite 440|Woodlawn|Winnebago County|IL|64098|United States|-6|condo| +31968|AAAAAAAAAOMHAAAA|161|Elm 5th|Circle|Suite S|Mount Vernon|Daviess County|IN|48482|United States|-5|condo| +31969|AAAAAAAABOMHAAAA|||Dr.||Walnut Grove|Franklin County|MO||United States|-6|| +31970|AAAAAAAACOMHAAAA|904|5th |Pkwy|Suite 380|Spring Grove|Clay County|MO|66719|United States|-6|single family| +31971|AAAAAAAADOMHAAAA|425|Jefferson Cedar|Road|Suite S|Pleasant Hill|Spencer County|IN|43604|United States|-5|single family| +31972|AAAAAAAAEOMHAAAA|257|Main Second|Drive|Suite 330|Crossroads|Brown County|OH|40534|United States|-5|single family| +31973|AAAAAAAAFOMHAAAA|763|Chestnut Main|Road|Suite R|Saint Clair|Marion County|FL|35294|United States|-5|single family| +31974|AAAAAAAAGOMHAAAA|827|4th |Blvd|Suite 160|Red Hill|Hays County|TX|74338|United States|-6|condo| +31975|AAAAAAAAHOMHAAAA|426|3rd |Pkwy|Suite Q|Shiloh|Lycoming County|PA|19275|United States|-5|apartment| +31976|AAAAAAAAIOMHAAAA|241|7th Railroad|Pkwy|Suite Y|Lewisburg|Stevens County|MN|57538|United States|-6|apartment| +31977|AAAAAAAAJOMHAAAA|614|Davis |Pkwy|Suite V|Pine Grove|Alcorn County|MS|54593|United States|-6|single family| +31978|AAAAAAAAKOMHAAAA|626|2nd |Ln|Suite O|Webb|Benson County|ND|50899|United States|-6|condo| +31979|AAAAAAAALOMHAAAA|609|South |Lane|Suite 360|Riverview|Carter County|KY|49003|United States|-6|single family| +31980|AAAAAAAAMOMHAAAA||||Suite C|Pleasant Valley||MN|52477|United States|-6|condo| +31981|AAAAAAAANOMHAAAA|146|First |Drive|Suite F|Lincoln|Jefferson County|GA|31289|United States|-5|condo| +31982|AAAAAAAAOOMHAAAA|902|Dogwood View|Wy|Suite C|Riverside|Desha County|AR|79231|United States|-6|apartment| +31983|AAAAAAAAPOMHAAAA|196|Laurel Second|Road|Suite W|Valley View|Ziebach County|SD|55124|United States|-6|single family| +31984|AAAAAAAAAPMHAAAA|82|South |Parkway|Suite 290|Jamestown|Teller County|CO|86867|United States|-7|single family| +31985|AAAAAAAABPMHAAAA|616|5th First|Blvd|Suite Y|Riverview|Willacy County|TX|79003|United States|-6|single family| +31986|AAAAAAAACPMHAAAA|39|4th Park|Wy|Suite A|Willow|Washington County|TX|76798|United States|-6|single family| +31987|AAAAAAAADPMHAAAA|436|4th Meadow|Lane|Suite W|Lee|Hays County|TX|70408|United States|-6|apartment| +31988|AAAAAAAAEPMHAAAA|642|Wilson |ST|Suite 30|Glendale|Goochland County|VA|23951|United States|-5|condo| +31989|AAAAAAAAFPMHAAAA|886|Church Maple|Ave|Suite O|Glendale|Fluvanna County|VA|23951|United States|-5|single family| +31990|AAAAAAAAGPMHAAAA|51||Court||Pleasant Grove||OK|74136||-6|apartment| +31991|AAAAAAAAHPMHAAAA|295|River |Wy|Suite P|Greenfield|Cass County|NE|65038|United States|-6|single family| +31992|AAAAAAAAIPMHAAAA|558|Fifth |Circle|Suite R|Marion|Fayette County|WV|20399|United States|-5|apartment| +31993|AAAAAAAAJPMHAAAA|442|Highland Broadway|Lane|Suite 130|Antioch|DeSoto County|FL|38605|United States|-5|single family| +31994|AAAAAAAAKPMHAAAA|331|Oak Dogwood|Boulevard|Suite G|Lakeside|Lehigh County|PA|19532|United States|-5|single family| +31995|AAAAAAAALPMHAAAA|217|14th |Ct.|Suite 0|Superior|Tunica County|MS|52562|United States|-6|single family| +31996|AAAAAAAAMPMHAAAA|806|East Park|Parkway|Suite B|Red Hill|Hendricks County|IN|44338|United States|-5|apartment| +31997|AAAAAAAANPMHAAAA|547|Sunset Seventh|Cir.|Suite V|Waterloo|Lee County|IL|61675|United States|-6|single family| +31998|AAAAAAAAOPMHAAAA|978|6th Willow|Avenue|Suite Y|Union|Breathitt County|KY|48721|United States|-6|apartment| +31999|AAAAAAAAPPMHAAAA|161|Spring Willow|Circle|Suite 390|Waterloo|Mitchell County|NC|21675|United States|-5|condo| +32000|AAAAAAAAAANHAAAA|451|College Second|RD|Suite 480|Salem|East Baton Rouge Parish|LA|78048|United States|-6|condo| +32001|AAAAAAAABANHAAAA|347|14th 7th|Boulevard|Suite 490|Rolling Hills|Madison County|FL|37272|United States|-5|single family| +32002|AAAAAAAACANHAAAA|120|11th |Street|Suite 90|Forest Hills|Chambers County|AL|39237|United States|-6|apartment| +32003|AAAAAAAADANHAAAA|855|View Hillcrest|ST|Suite Q|Walnut Grove|Torrance County|NM|87752|United States|-7|condo| +32004|AAAAAAAAEANHAAAA|195|Willow |Street|Suite 160|Lakeside|Benton County|MO|69532|United States|-6|apartment| +32005|AAAAAAAAFANHAAAA|133|River Hill|Circle|Suite S|Bethel|Modoc County|CA|95281|United States|-8|condo| +32006|AAAAAAAAGANHAAAA|49|View 14th|Drive|Suite Q|Glendale|Brown County|WI|53951|United States|-6|apartment| +32007|AAAAAAAAHANHAAAA|918|Forest |Ln|Suite H|Newtown|Lamar County|MS|51749|United States|-6|single family| +32008|AAAAAAAAIANHAAAA|833|Forest |Parkway|Suite N|Buena Vista|Berrien County|MI|45752|United States|-5|single family| +32009|AAAAAAAAJANHAAAA|309|Ridge Locust|Road|Suite 280|Maywood|Humphreys County|TN|35681|United States|-5|single family| +32010|AAAAAAAAKANHAAAA|106|Mill First|Parkway|Suite A|Wilson|Williamson County|TN|36971|United States|-5|condo| +32011|AAAAAAAALANHAAAA|908|2nd View|Ave|Suite 40|Ashland|Albany County|NY|14244|United States|-5|condo| +32012|AAAAAAAAMANHAAAA|965|Sycamore |Boulevard|Suite 120|Riverview|Jackson County|MI|49003|United States|-5|condo| +32013|AAAAAAAANANHAAAA|597|2nd |Wy|Suite E|Highland Park|Carroll County|GA|36534|United States|-5|apartment| +32014|AAAAAAAAOANHAAAA|375|13th Madison|Ln|Suite 440|Mountain View|Marengo County|AL|34466|United States|-6|single family| +32015|AAAAAAAAPANHAAAA|26|Meadow Johnson|Boulevard|Suite 40|Spring Valley|Beauregard Parish|LA|76060|United States|-6|single family| +32016|AAAAAAAAABNHAAAA|435|First First|Blvd|Suite J|Clearwater|Franklin County|NE|69534|United States|-6|single family| +32017|AAAAAAAABBNHAAAA|693||Ct.|Suite 410|||TX||United States|-6|condo| +32018|AAAAAAAACBNHAAAA|909|Green |Road|Suite X|Pleasant Hill|Victoria County|TX|73604|United States|-6|apartment| +32019|AAAAAAAADBNHAAAA|703|Adams |Boulevard|Suite N|Post Oak|Lac qui Parle County|MN|58567|United States|-6|apartment| +32020|AAAAAAAAEBNHAAAA|233|Park Park|Court|Suite 140|Franklin|Fayette County|OH|49101|United States|-5|apartment| +32021|AAAAAAAAFBNHAAAA|68|Center Johnson|Way|Suite 100|New Hope|Indian River County|FL|39431|United States|-5|condo| +32022|AAAAAAAAGBNHAAAA|||Parkway|Suite W||Harnett County|NC|27746|||| +32023|AAAAAAAAHBNHAAAA|668|College |Wy|Suite Q|Monticello|Albemarle County|VA|24146|United States|-5|condo| +32024|AAAAAAAAIBNHAAAA|915|Park West|Circle|Suite R|Valley View|Pawnee County|NE|65124|United States|-7|single family| +32025|AAAAAAAAJBNHAAAA|191|North |Avenue|Suite L|Bunker Hill|Livingston County|MI|40150|United States|-5|condo| +32026|AAAAAAAAKBNHAAAA|207||Ln|||Gilliam County|OR|96867|United States||condo| +32027|AAAAAAAALBNHAAAA|280|Oak Oak|Pkwy|Suite 420|Fairview|Pope County|IL|65709|United States|-6|single family| +32028|AAAAAAAAMBNHAAAA|467|Lake |Ln|Suite 20|Riverside|Turner County|GA|39231|United States|-5|single family| +32029|AAAAAAAANBNHAAAA|190|Washington Broadway|Cir.|Suite 160|Richville|Seminole County|FL|35945|United States|-5|single family| +32030|AAAAAAAAOBNHAAAA|845|Hillcrest |Wy|Suite L|Cordova|Fergus County|MT|66938|United States|-7|condo| +32031|AAAAAAAAPBNHAAAA|316|Forest Johnson|Court|Suite F|Woodlawn|Carter County|MT|64098|United States|-7|single family| +32032|AAAAAAAAACNHAAAA|248|East |Court|Suite 480|Leesville|Stonewall County|TX|75423|United States|-6|condo| +32033|AAAAAAAABCNHAAAA|40|View West|Street|Suite K|Salem|Bourbon County|KY|48048|United States|-6|single family| +32034|AAAAAAAACCNHAAAA|188|Cedar |Lane|Suite 390|Hillcrest|Wahkiakum County|WA|93003|United States|-8|condo| +32035|AAAAAAAADCNHAAAA|159|8th 3rd|Blvd|Suite 40|Lakewood|Graham County|KS|68877|United States|-6|condo| +32036|AAAAAAAAECNHAAAA|290|Park |Wy|Suite C|New Hope|Catron County|NM|89431|United States|-7|single family| +32037|AAAAAAAAFCNHAAAA||Jefferson Main|Boulevard||||TX|||-6|apartment| +32038|AAAAAAAAGCNHAAAA|163|7th |Ave|Suite D|White Oak|Fluvanna County|VA|26668|United States|-5|single family| +32039|AAAAAAAAHCNHAAAA|771|Laurel View|Cir.|Suite 310|Tracy|Kiowa County|OK|76340|United States|-6|single family| +32040|AAAAAAAAICNHAAAA|581|Sycamore Meadow|Lane|Suite 210|Silver Creek|Dimmit County|TX|74546|United States|-6|apartment| +32041|AAAAAAAAJCNHAAAA|419|Washington Laurel|Ave|Suite 320|Plainview|White Pine County|NV|83683|United States|-8|condo| +32042|AAAAAAAAKCNHAAAA|374|Mill |Court|Suite B|Sheffield|Big Horn County|MT|66896|United States|-7|condo| +32043|AAAAAAAALCNHAAAA|452|Valley Highland|Way|Suite 120|Sulphur Springs|Davidson County|NC|28354|United States|-5|apartment| +32044|AAAAAAAAMCNHAAAA|615|Jackson Walnut|Parkway|Suite 200|Oak Ridge|Salt Lake County|UT|88371|United States|-7|condo| +32045|AAAAAAAANCNHAAAA|544|Franklin |Dr.|Suite 70|Edgewood|Allegheny County|PA|10069|United States|-5|apartment| +32046|AAAAAAAAOCNHAAAA|935|Meadow |ST|Suite 350|Cedar Grove|Washington County|IA|50411|United States|-6|single family| +32047|AAAAAAAAPCNHAAAA|224|14th Washington|Cir.|Suite K|Lakewood|Washington County|VT|09477|United States|-5|apartment| +32048|AAAAAAAAADNHAAAA|393|7th |Street|Suite 40|Green Acres|Douglas County|OR|97683|United States|-8|single family| +32049|AAAAAAAABDNHAAAA|971|Forest |Ct.|Suite S|Shady Grove|Lamb County|TX|72812|United States|-6|apartment| +32050|AAAAAAAACDNHAAAA|758|Maple Willow|Court|Suite 440|New Hope|Prince George County|VA|29431|United States|-5|single family| +32051|AAAAAAAADDNHAAAA|779|South Maple|ST|Suite L|Unionville|Twin Falls County|ID|81711|United States|-7|condo| +32052|AAAAAAAAEDNHAAAA|946|Smith Third|Way|Suite 40|Stringtown|Lincoln County|AR|70162|United States|-6|single family| +32053|AAAAAAAAFDNHAAAA|708|7th |RD|Suite 30|Friendship|Louisa County|IA|54536|United States|-6|condo| +32054|AAAAAAAAGDNHAAAA|89|South |Ct.|Suite A|Sulphur Springs|Armstrong County|PA|18354|United States|-5|condo| +32055|AAAAAAAAHDNHAAAA|975|River |Wy|Suite 0|Thompson|Washington County|NY|10336|United States|-5|condo| +32056|AAAAAAAAIDNHAAAA|550|Franklin |Avenue|Suite D|Arlington|Butler County|NE|66557|United States|-6|apartment| +32057|AAAAAAAAJDNHAAAA|584|Highland |Dr.|Suite 120|Greenwood|Warren County|PA|18828|United States|-5|single family| +32058|AAAAAAAAKDNHAAAA|57|Walnut Second|Ln|Suite 470|Florence|Lawrence County|SD|53394|United States|-7|condo| +32059|AAAAAAAALDNHAAAA|376|Mill |ST|Suite N|Blair|Lincoln County|GA|35465|United States|-5|condo| +32060|AAAAAAAAMDNHAAAA|682|Pine Sunset|Ct.|Suite P|Salem|Kenton County|KY|48048|United States|-5|apartment| +32061|AAAAAAAANDNHAAAA|358|Fifth |Avenue|Suite T|Midway|Washington County|IA|51904|United States|-6|condo| +32062|AAAAAAAAODNHAAAA|305|14th |Road|Suite N|Kingston|Burt County|NE|64975|United States|-6|apartment| +32063|AAAAAAAAPDNHAAAA|944|View 1st|Road|Suite 290|Glenwood|Dawson County|GA|33511|United States|-5|apartment| +32064|AAAAAAAAAENHAAAA|655|Eigth Williams|Street|Suite C|Oak Ridge|Stafford County|KS|68371|United States|-6|condo| +32065|AAAAAAAABENHAAAA|646|Elm |Ave|Suite 70|Concord|Cowley County|KS|64107|United States|-6|condo| +32066|AAAAAAAACENHAAAA|385|Walnut |Drive|Suite N|Oakdale|Kenosha County|WI|59584|United States|-6|single family| +32067|AAAAAAAADENHAAAA|194|Cedar |Way|Suite M|Providence|Lawrence County|MS|56614|United States|-6|single family| +32068|AAAAAAAAEENHAAAA|479|Wilson Adams|Ave|Suite V|Wilson|Dixon County|NE|66971|United States|-6|single family| +32069|AAAAAAAAFENHAAAA|372|7th Johnson|Circle|Suite M|Bridgeport|Gray County|TX|75817|United States|-6|single family| +32070|AAAAAAAAGENHAAAA|||Road||Mountain View|Chippewa County||54466||-6|| +32071|AAAAAAAAHENHAAAA|61|Lake Poplar|ST|Suite 150|Bunker Hill|Mecosta County|MI|40150|United States|-5|apartment| +32072|AAAAAAAAIENHAAAA|400|View Sunset|Dr.|Suite X|Smith|Mille Lacs County|MN|57317|United States|-6|condo| +32073|AAAAAAAAJENHAAAA|332|3rd Pine|Ave|Suite 130|Unionville|Montgomery County|MS|51711|United States|-6|condo| +32074|AAAAAAAAKENHAAAA|256|1st |Road|Suite T|Jackson|Wythe County|VA|29583|United States|-5|apartment| +32075|AAAAAAAALENHAAAA|295|Wilson |Parkway|Suite B|Shaw|Essex County|VA|20618|United States|-5|single family| +32076|AAAAAAAAMENHAAAA|422|Broadway 2nd|Wy|Suite 440|Oakdale|Hardin County|OH|49584|United States|-5|single family| +32077|AAAAAAAANENHAAAA|328|Eigth |RD|Suite 490|Macon|De Soto Parish|LA|70369|United States|-6|apartment| +32078|AAAAAAAAOENHAAAA|657|Adams 7th|Lane|Suite 300|Jamestown|Colorado County|TX|76867|United States|-6|condo| +32079|AAAAAAAAPENHAAAA||Laurel |Parkway|Suite 20|Midway||VA||United States|-5|condo| +32080|AAAAAAAAAFNHAAAA|585|7th |Cir.|Suite V|Valley View|Juneau Borough|AK|95124|United States|-9|condo| +32081|AAAAAAAABFNHAAAA|882|Sunset Chestnut|RD|Suite 120|White Oak|Buckingham County|VA|26668|United States|-5|condo| +32082|AAAAAAAACFNHAAAA|416|Johnson |Blvd|Suite 370|Highland Park|Manassas city|VA|26534|United States|-5|single family| +32083|AAAAAAAADFNHAAAA|353|Railroad 11th|Boulevard|Suite S|Little River|Payette County|ID|80319|United States|-7|condo| +32084|AAAAAAAAEFNHAAAA|230|Fourth First|Ln|Suite 450|Belmont|Douglas County|GA|30191|United States|-5|single family| +32085|AAAAAAAAFFNHAAAA|368|Hickory |Pkwy|Suite M|Greenfield|Sacramento County|CA|95038|United States|-8|apartment| +32086|AAAAAAAAGFNHAAAA|152|5th |RD|Suite 200|Riverdale|Stearns County|MN|59391|United States|-6|apartment| +32087|AAAAAAAAHFNHAAAA|829|Sunset Hill|Dr.|Suite 440|Liberty|Henry County|OH|43451|United States|-5|condo| +32088|AAAAAAAAIFNHAAAA|420|Mill 2nd|Ln|Suite 280|Unionville|Sumner County|KS|61711|United States|-6|single family| +32089|AAAAAAAAJFNHAAAA|979|Madison Jackson|Blvd|Suite 280|Ashland|Washington County|OR|94244|United States|-8|apartment| +32090|AAAAAAAAKFNHAAAA|383|Meadow Lee|Court|Suite T|Greenville|Crawford County|IN|41387|United States|-5|apartment| +32091|AAAAAAAALFNHAAAA|41|Park First|Wy|Suite 40|Weldon|Garza County|TX|76277|United States|-6|single family| +32092|AAAAAAAAMFNHAAAA|452|Main |Circle|Suite 100|Sulphur Springs|Dent County|MO|68354|United States|-6|condo| +32093|AAAAAAAANFNHAAAA|933|Mill Madison|Way|Suite 460|Pinhook|Phillips County|MT|69398|United States|-7|apartment| +32094|AAAAAAAAOFNHAAAA|278|2nd |Road|Suite 460|Brownsville|Richmond County|GA|39310|United States|-5|apartment| +32095|AAAAAAAAPFNHAAAA|618|Park |Ct.|Suite M|Mount Vernon|Bourbon County|KY|48482|United States|-6|condo| +32096|AAAAAAAAAGNHAAAA|546|Willow |RD|Suite 10|Pine Grove|Aleutians West Census Area|AK|94593|United States|-9|single family| +32097|AAAAAAAABGNHAAAA|706|Wilson Main|ST|Suite M|Oakdale|Wilkes County|GA|39584|United States|-5|condo| +32098|AAAAAAAACGNHAAAA|918|Cherry |Parkway|Suite 40|Summit|Chittenden County|VT|01099|United States|-5|condo| +32099|AAAAAAAADGNHAAAA|64|2nd |Street|Suite L|Five Forks|Jackson County|GA|32293|United States|-5|single family| +32100|AAAAAAAAEGNHAAAA|548|Sixth Madison|Drive|Suite P|Martinsville|Walworth County|SD|50419|United States|-7|apartment| +32101|AAAAAAAAFGNHAAAA|130|7th Forest|Dr.|Suite 440|Unionville|Union County|AR|71711|United States|-6|single family| +32102|AAAAAAAAGGNHAAAA|738|Wilson |Street|Suite 250|Plainview|Henry County|MO|63683|United States|-6|single family| +32103|AAAAAAAAHGNHAAAA|323|4th |Dr.|Suite 200|Lone Oak|Bourbon County|KS|66893|United States|-6|condo| +32104|AAAAAAAAIGNHAAAA|452|Spring |Dr.|Suite 350|Edgewood|Albany County|NY|10069|United States|-5|condo| +32105|AAAAAAAAJGNHAAAA|102|Maple Forest|Pkwy|Suite 120|Oakwood|Attala County|MS|50169|United States|-6|single family| +32106|AAAAAAAAKGNHAAAA|151|College North|Wy|Suite D|Oak Grove|Greer County|OK|78370|United States|-6|condo| +32107|AAAAAAAALGNHAAAA|147|Dogwood Spruce|Drive|Suite 300|Ashland|Howard County|TX|74244|United States|-6|single family| +32108|AAAAAAAAMGNHAAAA|541|Maple |Ln|Suite P|Glendale|Grant County|OR|93951|United States|-8|single family| +32109|AAAAAAAANGNHAAAA|700|11th |Pkwy|Suite N|Woodville|Alamance County|NC|24289|United States|-5|condo| +32110|AAAAAAAAOGNHAAAA|481|View |Ave|Suite P|Berea|Lake County|IN|43804|United States|-5|single family| +32111|AAAAAAAAPGNHAAAA|668|Green View|Blvd|||McIntosh County|GA|30344|United States||| +32112|AAAAAAAAAHNHAAAA|67|6th |Ct.|Suite D|Glenwood|Pope County|AR|73511|United States|-6|apartment| +32113|AAAAAAAABHNHAAAA|249|Fourth 5th|Lane|Suite F|Pleasant Valley|Hood River County|OR|92477|United States|-8|single family| +32114|AAAAAAAACHNHAAAA|912|South |ST|Suite 150|Page|Union County|PA|10296|United States|-5|condo| +32115|AAAAAAAADHNHAAAA|429|6th Laurel|Circle|Suite X|Doyle|Red River Parish|LA|78434|United States|-6|apartment| +32116|AAAAAAAAEHNHAAAA|450|Park |Ave|Suite 120|Brownsville|Wayne County|MI|49310|United States|-5|condo| +32117|AAAAAAAAFHNHAAAA|||Dr.|Suite 10|||MI||United States||| +32118|AAAAAAAAGHNHAAAA|482|6th Elevnth|Avenue|Suite 90|Mount Vernon|Mesa County|CO|88482|United States|-7|apartment| +32119|AAAAAAAAHHNHAAAA|222|Mill Lee|RD|Suite 220|Fairfield|Washington County|KS|66192|United States|-6|condo| +32120|AAAAAAAAIHNHAAAA|562|5th |Avenue|Suite A|Clifton|Dimmit County|TX|78014|United States|-6|condo| +32121|AAAAAAAAJHNHAAAA|538|Mill |Wy|Suite 400|Stringtown|Galveston County|TX|70162|United States|-6|condo| +32122|AAAAAAAAKHNHAAAA|357|Main |Court|Suite D|Mountain View|DeWitt County|TX|74466|United States|-6|single family| +32123|AAAAAAAALHNHAAAA|29|Tenth |Ct.|Suite 360|Lawrenceville|Queens County|NY|14462|United States|-5|apartment| +32124|AAAAAAAAMHNHAAAA|591|Highland Maple|Pkwy|Suite 30|Mountain View|Warren County|MO|64466|United States|-6|condo| +32125|AAAAAAAANHNHAAAA|534|Birch |Street|Suite P|Riverdale|Dixon County|NE|69391|United States|-6|single family| +32126|AAAAAAAAOHNHAAAA|500|Franklin |Cir.|Suite 260|Greenwood|Willacy County|TX|78828|United States|-6|apartment| +32127|AAAAAAAAPHNHAAAA|56|Ridge 3rd|Road|Suite 470|Summit|Schley County|GA|30499|United States|-5|condo| +32128|AAAAAAAAAINHAAAA|791|Walnut |Lane|Suite 80|Newtown|Roberts County|SD|51749|United States|-7|single family| +32129|AAAAAAAABINHAAAA|698|Miller |Parkway|Suite 100|Pierce|Wheatland County|MT|63360|United States|-7|single family| +32130|AAAAAAAACINHAAAA|975|Center Chestnut|Ct.|Suite 100|Fairview|Huron County|OH|45709|United States|-5|single family| +32131|AAAAAAAADINHAAAA|525|Lakeview |Circle|Suite U|Oak Hill|Baldwin County|AL|37838|United States|-6|condo| +32132|AAAAAAAAEINHAAAA|782|3rd Spruce|Circle|Suite H|Farmington|Wheatland County|MT|69145|United States|-7|condo| +32133|AAAAAAAAFINHAAAA||||Suite Y||Issaquena County|||||condo| +32134|AAAAAAAAGINHAAAA|234|Sunset |Wy|Suite Y|Mount Vernon|Cowley County|KS|68482|United States|-6|apartment| +32135|AAAAAAAAHINHAAAA|248|2nd Franklin|Ln|Suite K|Newtown|Childress County|TX|71749|United States|-6|single family| +32136|AAAAAAAAIINHAAAA||Forest Hillcrest||Suite C|||KY|45980|United States|-5|| +32137|AAAAAAAAJINHAAAA|525|Church |Blvd|Suite I|Clifton|Marquette County|WI|58014|United States|-6|condo| +32138|AAAAAAAAKINHAAAA|170|Wilson Second|Ave|Suite 450|Plainview|Franklin County|MS|53683|United States|-6|apartment| +32139|AAAAAAAALINHAAAA|485|Meadow |Court|Suite 420|Mount Vernon|Worcester County|MD|28482|United States|-5|condo| +32140|AAAAAAAAMINHAAAA|394|Lee |Lane|Suite 400|Cedar Grove|Campbell County|WY|80411|United States|-7|apartment| +32141|AAAAAAAANINHAAAA|108|12th 5th|Wy|Suite 320|Woodland|Linn County|OR|94854|United States|-8|condo| +32142|AAAAAAAAOINHAAAA||Jackson |Boulevard|Suite V|Wildwood|||||-6|| +32143|AAAAAAAAPINHAAAA|123|Johnson |Street|Suite 40|Highland|Dillingham Census Area|AK|99454|United States|-9|single family| +32144|AAAAAAAAAJNHAAAA|739|North Cedar|Court|Suite 270|Enterprise|Nance County|NE|61757|United States|-7|apartment| +32145|AAAAAAAABJNHAAAA|495|Williams Laurel|ST|Suite X|Lebanon|Arenac County|MI|42898|United States|-5|single family| +32146|AAAAAAAACJNHAAAA|711|5th 15th|Boulevard|Suite 440|Shady Grove|Benton County|WA|92812|United States|-8|single family| +32147|AAAAAAAADJNHAAAA|311|Ash Central|Road|Suite 20|Enterprise|Portage County|WI|51757|United States|-6|condo| +32148|AAAAAAAAEJNHAAAA|965|11th Adams|Drive|Suite 120|Belmont|Menominee County|MI|40191|United States|-5|apartment| +32149|AAAAAAAAFJNHAAAA|978|Miller Woodland|Parkway|Suite K|Franklin|Muscogee County|GA|39101|United States|-5|condo| +32150|AAAAAAAAGJNHAAAA|629|Sycamore |Parkway|Suite 380|Richland|Terrebonne Parish|LA|76510|United States|-6|single family| +32151|AAAAAAAAHJNHAAAA|223||Lane|Suite O|Walnut Grove|San Mateo County|CA|97752|United States|-8|| +32152|AAAAAAAAIJNHAAAA|697|View Adams|Boulevard|Suite R|Jackson|Snohomish County|WA|99583|United States|-8|single family| +32153|AAAAAAAAJJNHAAAA|42|Dogwood Church|Drive|Suite A|Concord|Washakie County|WY|84107|United States|-7|condo| +32154|AAAAAAAAKJNHAAAA|502|West Chestnut|Boulevard|Suite 380|Glendale|Dickens County|TX|73951|United States|-6|apartment| +32155|AAAAAAAALJNHAAAA|900|First Spring|Road|Suite G|Union|Mahoning County|OH|48721|United States|-5|single family| +32156|AAAAAAAAMJNHAAAA|253|7th |Boulevard|Suite 10|Marion|Cape Girardeau County|MO|60399|United States|-6|condo| +32157|AAAAAAAANJNHAAAA|66|Miller Mill|RD|Suite 220|Fairfield|Beaufort County|SC|26192|United States|-5|condo| +32158|AAAAAAAAOJNHAAAA|839|Lake Jackson|Ave|Suite 310|Friendship|Clinton County|MI|44536|United States|-5|single family| +32159|AAAAAAAAPJNHAAAA|135|Walnut 15th|Wy|Suite U|Shady Grove|Somerset County|ME|03412|United States|-5|condo| +32160|AAAAAAAAAKNHAAAA|760|4th 1st|Lane|Suite 150|Friendship|Sutton County|TX|74536|United States|-6|single family| +32161|AAAAAAAABKNHAAAA|889|East Elm|Circle|Suite 300|Hopewell|Salt Lake County|UT|80587|United States|-7|apartment| +32162|AAAAAAAACKNHAAAA|618|1st Maple|Blvd|Suite T|Unionville|Pendleton County|WV|21711|United States|-5|condo| +32163|AAAAAAAADKNHAAAA|512|Mill |ST|Suite R|Floyd|Aiken County|SC|23235|United States|-5|condo| +32164|AAAAAAAAEKNHAAAA|878|Meadow |Street|Suite 360|Fairfield|Bourbon County|KY|46192|United States|-6|single family| +32165|AAAAAAAAFKNHAAAA|923|Sycamore |Ct.|Suite 400|Union Hill|Madison County|TN|37746|United States|-6|apartment| +32166|AAAAAAAAGKNHAAAA|652|Ridge |Cir.|Suite 100|Salem|Aitkin County|MN|58048|United States|-6|single family| +32167|AAAAAAAAHKNHAAAA|607|Ridge |RD|Suite R|Shiloh|Knox County|TX|79275|United States|-6|condo| +32168|AAAAAAAAIKNHAAAA|329|North Green|Parkway|Suite 250|Glendale|Doddridge County|WV|23951|United States|-5|condo| +32169|AAAAAAAAJKNHAAAA|942|First Thirteenth|Drive|Suite 420|Lakeview|Wheeler County|GA|38579|United States|-5|single family| +32170|AAAAAAAAKKNHAAAA||Park 6th||||Pulaski County|||||apartment| +32171|AAAAAAAALKNHAAAA|29|1st |Avenue|Suite 130|Carter|Dyer County|TN|30919|United States|-5|apartment| +32172|AAAAAAAAMKNHAAAA|354|6th |Parkway|Suite W|Oak Hill|Lowndes County|AL|37838|United States|-6|condo| +32173|AAAAAAAANKNHAAAA|941|Laurel Hillcrest|Court|Suite 300|Bethel|Howell County|MO|65281|United States|-6|apartment| +32174|AAAAAAAAOKNHAAAA|159|Fifth Elm|Drive|Suite W|Midway|Woodford County|IL|61904|United States|-6|apartment| +32175|AAAAAAAAPKNHAAAA|484|River Adams|Wy|Suite 70|Macedonia|Curry County|OR|91087|United States|-8|apartment| +32176|AAAAAAAAALNHAAAA|220|Smith |Circle|Suite J|Fairfield|Ingham County|MI|46192|United States|-5|apartment| +32177|AAAAAAAABLNHAAAA|566|4th Franklin|||Williamsville||CA||||condo| +32178|AAAAAAAACLNHAAAA|923|Meadow |Court|Suite Q|Woodland|Jackson County|GA|34854|United States|-5|apartment| +32179|AAAAAAAADLNHAAAA||||||Greene County|IA|55717|||condo| +32180|AAAAAAAAELNHAAAA|897|Laurel Fourth|Road|Suite P|Jamestown|Middlesex County|MA|07467|United States|-5|apartment| +32181|AAAAAAAAFLNHAAAA|384|Center Sycamore|Way|Suite 70|Jackson|Glasscock County|TX|79583|United States|-6|apartment| +32182|AAAAAAAAGLNHAAAA|440|Hill |Pkwy|Suite H|Lakeside|Staunton city|VA|29532|United States|-5|condo| +32183|AAAAAAAAHLNHAAAA|178|12th 1st|Ln|Suite E|Deerfield|La Plata County|CO|89840|United States|-7|apartment| +32184|AAAAAAAAILNHAAAA|509|4th |RD|Suite 0|White Oak|Hancock County|IL|66668|United States|-6|condo| +32185|AAAAAAAAJLNHAAAA|346|Oak |Ct.|Suite 280|Friendship|Eastland County|TX|74536|United States|-6|condo| +32186|AAAAAAAAKLNHAAAA|108|10th |Blvd|Suite 220|Mount Olive|Mariposa County|CA|98059|United States|-8|condo| +32187|AAAAAAAALLNHAAAA|366|5th River|Avenue|Suite T|Cedar Grove|Pepin County|WI|50411|United States|-6|single family| +32188|AAAAAAAAMLNHAAAA|70|11th 5th|Boulevard|Suite 70|Woodlawn|Valley County|NE|64098|United States|-7|condo| +32189|AAAAAAAANLNHAAAA|977|Maple |Ave|Suite 410|Five Points|Jefferson County|GA|36098|United States|-5|condo| +32190|AAAAAAAAOLNHAAAA|110|Spring Forest|Way|Suite 0|Pine Grove|San Francisco County|CA|94593|United States|-8|single family| +32191|AAAAAAAAPLNHAAAA|702|Poplar Willow|Dr.|Suite I|Pleasant Hill|Johnston County|OK|73604|United States|-6|apartment| +32192|AAAAAAAAAMNHAAAA|857|Church |Lane|Suite 200|Mount Zion|Ozark County|MO|68054|United States|-6|single family| +32193|AAAAAAAABMNHAAAA|832|Maple |Ct.|Suite 100|Enterprise|Adair County|MO|61757|United States|-6|condo| +32194|AAAAAAAACMNHAAAA|229|Park |Way|Suite 380|Roy|Kankakee County|IL|60744|United States|-6|condo| +32195|AAAAAAAADMNHAAAA|||Parkway||Five Points||||United States||apartment| +32196|AAAAAAAAEMNHAAAA|562|College |Ln|Suite G|Lakeside|Logan County|WV|29532|United States|-5|condo| +32197|AAAAAAAAFMNHAAAA|||||Greenville|Clay County|KS|61387|United States|-6|| +32198|AAAAAAAAGMNHAAAA|167|12th 3rd|Blvd|Suite 150|Red Hill|Houston County|TN|34338|United States|-5|single family| +32199|AAAAAAAAHMNHAAAA|545|Franklin |Pkwy|Suite 0|Laurel|Morgan County|AL|35673|United States|-6|apartment| +32200|AAAAAAAAIMNHAAAA|257|Chestnut 3rd|Road|Suite U|Newport|Florence County|WI|51521|United States|-6|single family| +32201|AAAAAAAAJMNHAAAA|183|10th Third|RD|Suite X|Union Hill|Lincoln County|NC|27746|United States|-5|apartment| +32202|AAAAAAAAKMNHAAAA|492|College Miller|Circle|Suite H|Shiloh|Wichita County|TX|79275|United States|-6|single family| +32203|AAAAAAAALMNHAAAA|355|Jefferson South|Road|Suite 40|Glenwood|Kendall County|IL|63511|United States|-6|condo| +32204|AAAAAAAAMMNHAAAA|337|First |ST|Suite L|Highland Park|Crawford County|IL|66534|United States|-6|single family| +32205|AAAAAAAANMNHAAAA|679|Green |Drive|Suite 170|Pleasant Valley|Monroe County|WI|52477|United States|-6|apartment| +32206|AAAAAAAAOMNHAAAA|646|Chestnut Fourth|Dr.|Suite 170|Liberty|Cuyahoga County|OH|43451|United States|-5|single family| +32207|AAAAAAAAPMNHAAAA|482|Central |Boulevard|Suite 210|Union Hill|Dickinson County|MI|47746|United States|-5|apartment| +32208|AAAAAAAAANNHAAAA|861|Cherry Fourth|Pkwy|Suite 490|Mount Vernon|Hampton County|SC|28482|United States|-5|single family| +32209|AAAAAAAABNNHAAAA|646|Spring |Boulevard|Suite 440|Oakland|Darke County|OH|49843|United States|-5|apartment| +32210|AAAAAAAACNNHAAAA|726|Chestnut 1st|RD|Suite 270|Stringtown|Scott County|IN|40162|United States|-5|single family| +32211|AAAAAAAADNNHAAAA|55|Lakeview |Cir.|Suite S|Woodlawn|Harvey County|KS|64098|United States|-6|condo| +32212|AAAAAAAAENNHAAAA|706|Sycamore Mill|Cir.|Suite 310|Spring Hill|Hunterdon County|NJ|07387|United States|-5|condo| +32213|AAAAAAAAFNNHAAAA|794|Woodland |Parkway|Suite 480|Farmington|Walworth County|WI|59145|United States|-6|single family| +32214|AAAAAAAAGNNHAAAA|203|North Valley|Blvd|Suite 220|Mountain View|Doniphan County|KS|64466|United States|-6|apartment| +32215|AAAAAAAAHNNHAAAA|336|Mill |Circle|Suite R|Macedonia|Keokuk County|IA|51087|United States|-6|condo| +32216|AAAAAAAAINNHAAAA|732|13th |Blvd|Suite R|Deerfield|Douglas County|WA|99840|United States|-8|single family| +32217|AAAAAAAAJNNHAAAA|788|5th Sunset|Drive|Suite 390|Enterprise|Ashley County|AR|71757|United States|-6|condo| +32218|AAAAAAAAKNNHAAAA|128|Sycamore |Cir.|Suite X|Vance|Grant County|WV|20268|United States|-5|apartment| +32219|AAAAAAAALNNHAAAA|950|13th |Avenue|Suite O|Bunker Hill|Orleans Parish|LA|70150|United States|-6|apartment| +32220|AAAAAAAAMNNHAAAA|453|Elm |Cir.|Suite P|Pleasant Valley|Snohomish County|WA|92477|United States|-8|condo| +32221|AAAAAAAANNNHAAAA|544|Valley 7th|Court|Suite H|New Hope|Jasper County|MO|69431|United States|-6|single family| +32222|AAAAAAAAONNHAAAA|576|Valley Seventh|Wy|Suite 200|Arlington|Smith County|TN|36557|United States|-6|apartment| +32223|AAAAAAAAPNNHAAAA|504|South |Ln|Suite 250|Centerville|Clay County|TN|30059|United States|-5|apartment| +32224|AAAAAAAAAONHAAAA|||Ln|Suite 480|Shelby||AK|96575|United States|-9|apartment| +32225|AAAAAAAABONHAAAA|93|8th |Wy|Suite 320|Wildwood|Hamilton County|NE|66871|United States|-6|single family| +32226|AAAAAAAACONHAAAA|762|1st |RD|Suite M|Oakwood|Itawamba County|MS|50169|United States|-6|single family| +32227|AAAAAAAADONHAAAA|332|13th |Pkwy|Suite 280|Doyle|Scott County|KY|48434|United States|-5|apartment| +32228|AAAAAAAAEONHAAAA|268|9th Elm|Pkwy|Suite H|Caledonia|Comanche County|TX|77411|United States|-6|condo| +32229|AAAAAAAAFONHAAAA|91|Highland Second|Road|Suite S|Maple Grove|Elmore County|ID|88252|United States|-7|condo| +32230|AAAAAAAAGONHAAAA|1000|Spring 5th|Cir.|Suite 380|Stringtown|Washington County|OR|90162|United States|-8|apartment| +32231|AAAAAAAAHONHAAAA|104|College 7th|Dr.|Suite X|Walnut Grove|Andrews County|TX|77752|United States|-6|single family| +32232|AAAAAAAAIONHAAAA|915|10th Maple|Dr.|Suite H|Pleasant Grove|La Salle County|TX|74136|United States|-6|condo| +32233|AAAAAAAAJONHAAAA|379|Mill |Court|Suite 410|Concord|McPherson County|NE|64107|United States|-7|condo| +32234|AAAAAAAAKONHAAAA|963|Green Main|Lane|Suite 450|Hamilton|Vance County|NC|22808|United States|-5|single family| +32235|AAAAAAAALONHAAAA|448|Center |Court|Suite Q|Antioch|Fulton County|GA|38605|United States|-5|single family| +32236|AAAAAAAAMONHAAAA|662||||Edgewood|Powder River County|MT|60069|United States|-7|apartment| +32237|AAAAAAAANONHAAAA|947|Church Lake|Parkway|Suite U|Highland|Darlington County|SC|29454|United States|-5|apartment| +32238|AAAAAAAAOONHAAAA|310|Franklin |Road|Suite 420|Oakland|McIntosh County|ND|59843|United States|-6|single family| +32239|AAAAAAAAPONHAAAA|754|Oak |Road|Suite 100|Mount Pleasant|Fayette County|AL|31933|United States|-6|apartment| +32240|AAAAAAAAAPNHAAAA|542|Valley Spring|Way|Suite 400|Derby|Tama County|IA|57702|United States|-6|apartment| +32241|AAAAAAAABPNHAAAA|970|Pine Poplar|Drive|Suite W|Hopewell|Pope County|MN|50587|United States|-6|single family| +32242|AAAAAAAACPNHAAAA|831|1st |Ct.|Suite 100|Brownsville|El Paso County|TX|79310|United States|-6|condo| +32243|AAAAAAAADPNHAAAA|397|Williams Wilson|ST|Suite D|Greenfield|Butts County|GA|35038|United States|-5|single family| +32244|AAAAAAAAEPNHAAAA|199|Central |Pkwy|Suite 90|Jackson|Knott County|KY|49583|United States|-5|apartment| +32245|AAAAAAAAFPNHAAAA|60|Laurel 7th|Dr.|Suite A|Morgantown|Jackson County|CO|89193|United States|-7|single family| +32246|AAAAAAAAGPNHAAAA|720|Central Hill|Court|Suite O|Jamestown|Lincoln County|SD|56867|United States|-7|apartment| +32247|AAAAAAAAHPNHAAAA|938|Oak |Dr.|Suite 320|Clinton|Floyd County|KY|48222|United States|-6|apartment| +32248|AAAAAAAAIPNHAAAA|219|Johnson |Dr.|Suite J|New Hope|Cameron County|PA|19431|United States|-5|single family| +32249|AAAAAAAAJPNHAAAA|704|4th |Way|Suite 200|Newtown|Polk County|NE|61749|United States|-7|single family| +32250|AAAAAAAAKPNHAAAA|286|First Hill|Lane|Suite 340|Newtown|Wetzel County|WV|21749|United States|-5|single family| +32251|AAAAAAAALPNHAAAA|56|Jefferson 3rd|Circle|Suite 170|Willow|Slope County|ND|56798|United States|-6|condo| +32252|AAAAAAAAMPNHAAAA|86|Seventh |Wy|Suite Y|Hurricane|Florence County|SC|27644|United States|-5|single family| +32253|AAAAAAAANPNHAAAA|419|River Adams|Circle|Suite K|Quincy|Dauphin County|PA|13868|United States|-5|apartment| +32254|AAAAAAAAOPNHAAAA|97|Ridge |Pkwy|Suite 150|Sulphur Springs|Trousdale County|TN|38354|United States|-6|apartment| +32255|AAAAAAAAPPNHAAAA|165|View |Boulevard|Suite X|Shady Grove|Tuolumne County|CA|92812|United States|-8|apartment| +32256|AAAAAAAAAAOHAAAA|742|Woodland Church|Drive|Suite Y|Midway|Keokuk County|IA|51904|United States|-6|apartment| +32257|AAAAAAAABAOHAAAA|911|River |Boulevard|Suite 220|Franklin|Miami County|OH|49101|United States|-5|single family| +32258|AAAAAAAACAOHAAAA|705|Poplar |Blvd|Suite A|Lakewood|Appanoose County|IA|58877|United States|-6|condo| +32259|AAAAAAAADAOHAAAA|403|Central |Circle|Suite 70|Macedonia|DeWitt County|TX|71087|United States|-6|apartment| +32260|AAAAAAAAEAOHAAAA|347|Madison Walnut|Parkway|Suite 490|Shady Grove|Cherokee County|AL|32812|United States|-6|condo| +32261|AAAAAAAAFAOHAAAA|903|2nd |Drive|Suite 310|Cedar Grove|Kossuth County|IA|50411|United States|-6|single family| +32262|AAAAAAAAGAOHAAAA|84|Third |Circle|Suite 30|Crossroads|Marion County|AL|30534|United States|-6|apartment| +32263|AAAAAAAAHAOHAAAA|31|Sycamore Fifth|Pkwy|Suite 380|Spring Hill|Union County|NC|26787|United States|-5|single family| +32264|AAAAAAAAIAOHAAAA|905|3rd Washington|Court|Suite 210|Liberty|Washington County|GA|33451|United States|-5|condo| +32265|AAAAAAAAJAOHAAAA|593|Park |Lane|Suite 440|Enterprise|Texas County|OK|71757|United States|-6|apartment| +32266|AAAAAAAAKAOHAAAA|404|Jefferson Railroad|Avenue|Suite 200|Friendship|Bear Lake County|ID|84536|United States|-7|condo| +32267|AAAAAAAALAOHAAAA|728|Mill |Lane|Suite Q|Woodland|Shoshone County|ID|84854|United States|-7|condo| +32268|AAAAAAAAMAOHAAAA|968|Madison |Ave|Suite 20|Walnut Grove|Carbon County|UT|87752|United States|-7|condo| +32269|AAAAAAAANAOHAAAA|175|Second Hill|Dr.|Suite L|Buena Vista|Pushmataha County|OK|75752|United States|-6|apartment| +32270|AAAAAAAAOAOHAAAA|623|Main |Road|Suite U|Bunker Hill|Benton County|OR|90150|United States|-8|apartment| +32271|AAAAAAAAPAOHAAAA|327|6th Meadow|Street|Suite M|Cherry Valley|Jefferson County|PA|10854|United States|-5|apartment| +32272|AAAAAAAAABOHAAAA|709|Cedar |Drive|Suite E|Unionville|Lamar County|GA|31711|United States|-5|apartment| +32273|AAAAAAAABBOHAAAA|95|Green |Boulevard|Suite 490|Florence|Perry County|IN|43394|United States|-5|condo| +32274|AAAAAAAACBOHAAAA|951|1st |Boulevard|Suite A|Wildwood|Kiowa County|KS|66871|United States|-6|condo| +32275|AAAAAAAADBOHAAAA|34|Main |Avenue|Suite Y|Owens|Kendall County|TX|72324|United States|-6|apartment| +32276|AAAAAAAAEBOHAAAA|563|Meadow Fourth|Blvd|Suite 450|Macedonia|Morris County|KS|61087|United States|-6|apartment| +32277|AAAAAAAAFBOHAAAA|147|7th Dogwood|Street|Suite 290|Bethel|Comanche County|TX|75281|United States|-6|single family| +32278|AAAAAAAAGBOHAAAA|510|Mill |Drive|Suite X|Shore Acres|Salem city|VA|22724|United States|-5|single family| +32279|AAAAAAAAHBOHAAAA|313|Hill Park|Dr.|Suite 140|Five Points|Republic County|KS|66098|United States|-6|apartment| +32280|AAAAAAAAIBOHAAAA|754|Hill |Lane|Suite 240|Spring Hill|Dyer County|TN|36787|United States|-5|apartment| +32281|AAAAAAAAJBOHAAAA|366|3rd |Circle|Suite R|Summit|Mitchell County|KS|60499|United States|-6|apartment| +32282|AAAAAAAAKBOHAAAA||Washington 14th||||Marshall County||33683||-6|condo| +32283|AAAAAAAALBOHAAAA|102|Railroad Railroad|Ct.|Suite 300|Spring Hill|Shenandoah County|VA|26787|United States|-5|condo| +32284|AAAAAAAAMBOHAAAA|446|6th Cedar|Avenue|Suite M|Glendale|Crawford County|MI|43951|United States|-5|condo| +32285|AAAAAAAANBOHAAAA|965|Main Hickory|Road|Suite Y|Lincoln|Nez Perce County|ID|81289|United States|-7|apartment| +32286|AAAAAAAAOBOHAAAA|431|Pine Meadow|Way|Suite K|Blue Springs|Marshall County|OK|74686|United States|-6|condo| +32287|AAAAAAAAPBOHAAAA|459|Lakeview |Court|Suite 380|Pleasant Hill|Baker County|GA|33604|United States|-5|condo| +32288|AAAAAAAAACOHAAAA|836|Mill |Dr.|Suite 490|Lebanon|Eaton County|MI|42898|United States|-5|apartment| +32289|AAAAAAAABCOHAAAA|222|Franklin |Lane|Suite 0|Pine Grove|Iosco County|MI|44593|United States|-5|apartment| +32290|AAAAAAAACCOHAAAA|801|River 8th|Court|Suite 190|Hopewell|Jasper County|IA|50587|United States|-6|condo| +32291|AAAAAAAADCOHAAAA|699|2nd Washington|Pkwy|Suite 140|Mountain View|North Slope Borough|AK|94466|United States|-9|single family| +32292|AAAAAAAAECOHAAAA|369|Pine |Ln|Suite 200|Hillcrest|Corson County|SD|53003|United States|-6|apartment| +32293|AAAAAAAAFCOHAAAA|353|Woodland |RD|Suite G|Riverside|La Crosse County|WI|59231|United States|-6|apartment| +32294|AAAAAAAAGCOHAAAA|284|Jackson 4th|ST|Suite 360|Cedar Grove|Pike County|MO|60411|United States|-6|single family| +32295|AAAAAAAAHCOHAAAA|542|Mill Walnut|Ave|Suite 390|Brownsville|East Baton Rouge Parish|LA|79310|United States|-6|apartment| +32296|AAAAAAAAICOHAAAA|702|4th Elm|Street|Suite 90|Summit|Waushara County|WI|50499|United States|-6|apartment| +32297|AAAAAAAAJCOHAAAA|548|Park |Blvd|Suite E|Marion|Humboldt County|CA|90399|United States|-8|apartment| +32298|AAAAAAAAKCOHAAAA|773|Johnson |Drive|Suite 460|Marion|Trego County|KS|60399|United States|-6|single family| +32299|AAAAAAAALCOHAAAA|949|View Maple|Boulevard|Suite D|Pine Grove|Noble County|OK|74593|United States|-6|apartment| +32300|AAAAAAAAMCOHAAAA|295|1st |Street|Suite 390|Mount Vernon|Clarke County|IA|58482|United States|-6|condo| +32301|AAAAAAAANCOHAAAA|994|8th |Pkwy|Suite Q|Brownsville|Comal County|TX|79310|United States|-6|single family| +32302|AAAAAAAAOCOHAAAA|185|View |Avenue|Suite 80|Greenville|Blaine County|ID|81387|United States|-7|condo| +32303|AAAAAAAAPCOHAAAA|875|Seventh 3rd|Ln|Suite P|Oakland|Iowa County|WI|59843|United States|-6|single family| +32304|AAAAAAAAADOHAAAA|94|Hillcrest Seventh|Avenue|Suite D|Five Forks|Rutherford County|NC|22293|United States|-5|condo| +32305|AAAAAAAABDOHAAAA|147||Lane|Suite L|Arlington|Santa Cruz County|||United States||condo| +32306|AAAAAAAACDOHAAAA|349|North |Drive|Suite 210|Union Hill|Webster County|KY|47746|United States|-5|single family| +32307|AAAAAAAADDOHAAAA|192|Miller Maple|Circle|Suite P|Franklin|Sullivan County|NH|09701|United States|-5|condo| +32308|AAAAAAAAEDOHAAAA|528|Hickory 7th|Drive|Suite C|Lewis|Rawlins County|KS|67066|United States|-6|condo| +32309|AAAAAAAAFDOHAAAA|245|12th ||Suite 170|Springfield|Mills County|TX||United States||| +32310|AAAAAAAAGDOHAAAA|492|14th Madison|Avenue|Suite 150|Farmington|Wayne County|NY|19145|United States|-5|condo| +32311|AAAAAAAAHDOHAAAA|733|Walnut |ST|Suite I|Smith|Orange County|TX|77317|United States|-6|condo| +32312|AAAAAAAAIDOHAAAA|4|Franklin 4th|Circle|Suite Q|Clearwater|Ballard County|KY|49534|United States|-6|single family| +32313|AAAAAAAAJDOHAAAA|884|Willow Elm|Street|Suite F|Woodland|Carroll County|MD|24854|United States|-5|condo| +32314|AAAAAAAAKDOHAAAA|65|13th |Blvd|Suite 150|Marion|Nicholas County|KY|40399|United States|-5|condo| +32315|AAAAAAAALDOHAAAA|325|Woodland Birch|Road|Suite 130|Glenwood|Dallas County|AL|33511|United States|-6|condo| +32316|AAAAAAAAMDOHAAAA|702|6th |Cir.|Suite 390|Salem|Ottawa County|OK|78048|United States|-6|condo| +32317|AAAAAAAANDOHAAAA|358|Washington |||Lincoln||||United States||apartment| +32318|AAAAAAAAODOHAAAA|115|Railroad |Ave|Suite C|Walnut Grove|Hale County|AL|37752|United States|-6|apartment| +32319|AAAAAAAAPDOHAAAA|649|Walnut 6th|Ct.|Suite Q|Oakland|Bland County|VA|29843|United States|-5|apartment| +32320|AAAAAAAAAEOHAAAA|189|2nd |Street|Suite T|Maple Grove|Washita County|OK|78252|United States|-6|single family| +32321|AAAAAAAABEOHAAAA|970|Second Lake|Boulevard|Suite 170|Highland|Treasure County|MT|69454|United States|-7|apartment| +32322|AAAAAAAACEOHAAAA|979|College Jefferson|Pkwy|Suite W|Harmony|Arkansas County|AR|75804|United States|-6|single family| +32323|AAAAAAAADEOHAAAA|216|Spruce Maple|Court|Suite W|Fairview|Gilmer County|WV|25709|United States|-5|single family| +32324|AAAAAAAAEEOHAAAA|316|2nd Hill|Cir.|Suite 340|Silver Springs|Union County|IA|54843|United States|-6|single family| +32325|AAAAAAAAFEOHAAAA|16|Park Valley|Ct.|Suite 380|Woodland|Sequoyah County|OK|74854|United States|-6|apartment| +32326|AAAAAAAAGEOHAAAA|884|Lake |ST|Suite J|Hillcrest|Tangipahoa Parish|LA|73003|United States|-6|condo| +32327|AAAAAAAAHEOHAAAA|729|Lincoln |Avenue|Suite B|Centerville|Iroquois County|IL|60059|United States|-6|condo| +32328|AAAAAAAAIEOHAAAA|94|Second |Cir.|Suite Q|Webb|Citrus County|FL|30899|United States|-5|condo| +32329|AAAAAAAAJEOHAAAA|593|Lake |Wy|Suite 40|Woodville|Saginaw County|MI|44289|United States|-5|apartment| +32330|AAAAAAAAKEOHAAAA|968|Railroad View|RD|Suite I|Pleasant Grove|West Baton Rouge Parish|LA|74136|United States|-6|single family| +32331|AAAAAAAALEOHAAAA|278|Smith Chestnut|Road|Suite G|Belmont|Colonial Heights city|VA|20191|United States|-5|condo| +32332|AAAAAAAAMEOHAAAA|518|View |Drive|Suite L|Mount Zion|Elkhart County|IN|48054|United States|-5|single family| +32333|AAAAAAAANEOHAAAA|341|Miller Lake|Cir.|Suite 90|Mount Zion|Muscogee County|GA|38054|United States|-5|condo| +32334|AAAAAAAAOEOHAAAA|358|7th |Circle|Suite T|Oakwood|Jenkins County|GA|30169|United States|-5|apartment| +32335|AAAAAAAAPEOHAAAA|719|South 4th|Ct.|Suite 80|Oak Ridge|Douglas County|NV|88371|United States|-8|condo| +32336|AAAAAAAAAFOHAAAA|34|1st River|Drive|Suite Q|Greenwood|Deuel County|SD|58828|United States|-6|condo| +32337|AAAAAAAABFOHAAAA|800|Maple |Cir.|Suite B|Edgewood|Dundy County|NE|60069|United States|-6|condo| +32338|AAAAAAAACFOHAAAA|344|1st |Ave|Suite C|Mountain View|Dearborn County|IN|44466|United States|-5|apartment| +32339|AAAAAAAADFOHAAAA|484|Adams |Circle|Suite 180|Oakdale|Calhoun County|GA|39584|United States|-5|single family| +32340|AAAAAAAAEFOHAAAA|558|Birch |Wy|Suite S|Waterloo|Grand County|CO|81675|United States|-7|apartment| +32341|AAAAAAAAFFOHAAAA|174|Franklin |Street|Suite A|Springfield|Lafourche Parish|LA|79303|United States|-6|apartment| +32342|AAAAAAAAGFOHAAAA|575|3rd |Lane|Suite 120|Maple Grove|Bradley County|AR|78252|United States|-6|apartment| +32343|AAAAAAAAHFOHAAAA|590|5th Willow|Blvd|Suite V|Spring Valley|Petersburg city|VA|26060|United States|-5|apartment| +32344|AAAAAAAAIFOHAAAA|275|1st North|Blvd|Suite K|Greenwood|Starke County|IN|48828|United States|-5|condo| +32345|AAAAAAAAJFOHAAAA|722|Third Main|Avenue|Suite 460|Springdale|Kingfisher County|OK|78883|United States|-6|apartment| +32346|AAAAAAAAKFOHAAAA|194|Walnut View|Lane|Suite Y|Lakewood|Shelby County|KY|48877|United States|-5|condo| +32347|AAAAAAAALFOHAAAA|182|South |Circle|Suite O|Liberty|Wakulla County|FL|33451|United States|-5|single family| +32348|AAAAAAAAMFOHAAAA|881|Spruce Hickory|Avenue|Suite S|Highland Park|Nueces County|TX|76534|United States|-6|apartment| +32349|AAAAAAAANFOHAAAA|649|Fourth |Ln|Suite P|Oakdale|Androscoggin County|ME|09584|United States|-5|apartment| +32350|AAAAAAAAOFOHAAAA|492|7th |RD|Suite 420|Flatwoods|Pratt County|KS|64212|United States|-6|condo| +32351|AAAAAAAAPFOHAAAA|792|Elm Chestnut|RD|Suite P|Clinton|Clay County|IA|58222|United States|-6|condo| +32352|AAAAAAAAAGOHAAAA|95|Meadow Sycamore|Circle|Suite 290|Stringtown|Linn County|IA|50162|United States|-6|apartment| +32353|AAAAAAAABGOHAAAA|636|Forest 4th|Boulevard|Suite 340|Highland Park|Loup County|NE|66534|United States|-7|condo| +32354|AAAAAAAACGOHAAAA|490|Chestnut Oak|Ave|Suite V|Greenwood|Caledonia County|VT|09428|United States|-5|apartment| +32355|AAAAAAAADGOHAAAA|604|Wilson |Wy|Suite P|Mount Vernon|Sarasota County|FL|38482|United States|-5|apartment| +32356|AAAAAAAAEGOHAAAA|967|Fifth |Drive|Suite 80|Riverside|Queens County|NY|19231|United States|-5|condo| +32357|AAAAAAAAFGOHAAAA|267|Jefferson |RD|Suite L|Highland|Marshall County|KS|69454|United States|-6|condo| +32358|AAAAAAAAGGOHAAAA|318|Lakeview |Road|Suite 460|Stringtown|Rockwall County|TX|70162|United States|-6|single family| +32359|AAAAAAAAHGOHAAAA|405|Washington 4th|Parkway|Suite I|Georgetown|Ulster County|NY|17057|United States|-5|condo| +32360|AAAAAAAAIGOHAAAA|941|Sunset Cedar|Court|Suite P|Salem|Elbert County|CO|88048|United States|-7|single family| +32361|AAAAAAAAJGOHAAAA|514|Pine Jefferson|Way|Suite N|Woodlawn|Hunt County|TX|74098|United States|-6|condo| +32362|AAAAAAAAKGOHAAAA|944|Park |Way|Suite 70|Oak Grove|Tishomingo County|MS|58370|United States|-6|condo| +32363|AAAAAAAALGOHAAAA|891|West Williams|Court|Suite W|Brentwood|Wabash County|IN|44188|United States|-5|apartment| +32364|AAAAAAAAMGOHAAAA|943|Davis 2nd|Blvd|Suite M|Lakeview|Lewis County|TN|38579|United States|-6|apartment| +32365|AAAAAAAANGOHAAAA|814|Pine |Court|Suite V|Oak Hill|Washington County|UT|87838|United States|-7|apartment| +32366|AAAAAAAAOGOHAAAA|556|Hill |Blvd|Suite 450|Greenwood|Linn County|OR|98828|United States|-8|apartment| +32367|AAAAAAAAPGOHAAAA|378|Cedar |Cir.|Suite 0|Oak Ridge|Eddy County|NM|88371|United States|-7|single family| +32368|AAAAAAAAAHOHAAAA|289|Ninth |Street|Suite N|Clinton|Warren County|NJ|08822|United States|-5|single family| +32369|AAAAAAAABHOHAAAA|165|East |Way|Suite 390|Shannon|Fremont County|IA|54120|United States|-6|condo| +32370|AAAAAAAACHOHAAAA|402|Hill |Dr.|Suite 30|Five Forks|Sedgwick County|CO|82293|United States|-7|single family| +32371|AAAAAAAADHOHAAAA|982|Elm 9th|Cir.|Suite 320|Plainview|Fleming County|KY|43683|United States|-6|apartment| +32372|AAAAAAAAEHOHAAAA|297|2nd Walnut|Ave|Suite 200|Elkton|Upton County|TX|73481|United States|-6|condo| +32373|AAAAAAAAFHOHAAAA|426|East 9th|Cir.|Suite W|Oakland|Jefferson County|GA|39843|United States|-5|condo| +32374|AAAAAAAAGHOHAAAA|609|Fourth 15th|Lane|Suite 60|Walnut Grove|Clark County|AR|77752|United States|-6|single family| +32375|AAAAAAAAHHOHAAAA|353|North River|Court|Suite 240|Weldon|Throckmorton County|TX|76277|United States|-6|condo| +32376|AAAAAAAAIHOHAAAA|634|Valley Sycamore|Pkwy|Suite 60|Springdale|Windsor County|VT|09483|United States|-5|condo| +32377|AAAAAAAAJHOHAAAA|254|South |Blvd|Suite Y|Lebanon|Grayson County|VA|22898|United States|-5|apartment| +32378|AAAAAAAAKHOHAAAA|273|Seventh Broadway|Pkwy|Suite 240|Woodville|Washington County|MD|24289|United States|-5|condo| +32379|AAAAAAAALHOHAAAA|820|Fifth |Ct.|Suite A|Woodville|Wabash County|IL|64289|United States|-6|single family| +32380|AAAAAAAAMHOHAAAA|227|Cedar Williams|Circle|Suite 70|Stringtown|Noble County|OH|40162|United States|-5|condo| +32381|AAAAAAAANHOHAAAA|500|Poplar Washington|Parkway|Suite 80|Ashland|Orangeburg County|SC|24244|United States|-5|apartment| +32382|AAAAAAAAOHOHAAAA|64|Green 1st|Road|Suite C|Oak Grove|Douglas County|MO|68370|United States|-6|apartment| +32383|AAAAAAAAPHOHAAAA|403|North |Dr.|Suite 80|Hillcrest|Anchorage Borough|AK|93003|United States|-9|condo| +32384|AAAAAAAAAIOHAAAA|325|Laurel |Ave|Suite 0|Oak Hill|Pontotoc County|MS|57838|United States|-6|single family| +32385|AAAAAAAABIOHAAAA|484|Cedar Maple|Lane|Suite L|Fairview|Vermilion Parish|LA|75709|United States|-6|apartment| +32386|AAAAAAAACIOHAAAA|991|Cherry Elm|Way|Suite G|Valley View|Morgan County|MO|65124|United States|-6|condo| +32387|AAAAAAAADIOHAAAA|488|Smith |Ln|Suite Q|Harmony|Knox County|TX|75804|United States|-6|apartment| +32388|AAAAAAAAEIOHAAAA|301|Ninth |Boulevard|Suite V|Pleasant Grove|Sequoyah County|OK|74136|United States|-6|single family| +32389|AAAAAAAAFIOHAAAA|835|Sunset |Way|Suite L|Greenwood|Tulare County|CA|98828|United States|-8|apartment| +32390|AAAAAAAAGIOHAAAA|470|Highland Mill|Wy|Suite L|Liberty|Wayne County|MI|43451|United States|-5|apartment| +32391|AAAAAAAAHIOHAAAA|438|Broadway |Lane|Suite 380|Pine Grove|Columbia County|FL|34593|United States|-5|single family| +32392|AAAAAAAAIIOHAAAA|940|Valley |Boulevard|Suite 130|Red Hill|Marion County|OH|44338|United States|-5|apartment| +32393|AAAAAAAAJIOHAAAA|918|||||||36622||-5|| +32394|AAAAAAAAKIOHAAAA|978|North ||Suite 300|Fairfield|Washington County|CO||||condo| +32395|AAAAAAAALIOHAAAA|795|Hickory Elm|Parkway|Suite N|Lebanon|Jefferson County|FL|32898|United States|-5|apartment| +32396|AAAAAAAAMIOHAAAA|955|River Main|Wy|Suite D|Springfield|Benton County|MN|59303|United States|-6|single family| +32397|AAAAAAAANIOHAAAA|129|Johnson Mill|Circle|Suite 60|Pine Grove|Baker County|FL|34593|United States|-5|condo| +32398|AAAAAAAAOIOHAAAA|830|Fourth |RD|Suite X|Highland|Harrison County|IN|49454|United States|-5|single family| +32399|AAAAAAAAPIOHAAAA|751|College 14th|||||KY||United States||single family| +32400|AAAAAAAAAJOHAAAA|530|Meadow 6th|Blvd|Suite 260|Macedonia|Fulton County|PA|11087|United States|-5|apartment| +32401|AAAAAAAABJOHAAAA|||Wy|Suite 290|Five Points||MO||||single family| +32402|AAAAAAAACJOHAAAA|732|Valley Fourth|Court|Suite 290|Oak Grove|Snohomish County|WA|98370|United States|-8|single family| +32403|AAAAAAAADJOHAAAA|87|Hickory Lincoln|Road|Suite 300|Mount Pleasant|Red Willow County|NE|61933|United States|-7|single family| +32404|AAAAAAAAEJOHAAAA|477|North Lincoln|Circle|Suite 20|Highland|Treutlen County|GA|39454|United States|-5|apartment| +32405|AAAAAAAAFJOHAAAA|85|West |ST|Suite 0|Oak Hill|Baker County|FL|37838|United States|-5|condo| +32406|AAAAAAAAGJOHAAAA|60|Valley 8th|Pkwy|Suite 410|Centerville|Kittson County|MN|50059|United States|-6|apartment| +32407|AAAAAAAAHJOHAAAA|343|Johnson |Street|Suite 150|Plainview|Knox County|TX|73683|United States|-6|single family| +32408|AAAAAAAAIJOHAAAA|565|Spring 3rd|Lane|Suite X|Liberty|Ramsey County|ND|53451|United States|-6|condo| +32409|AAAAAAAAJJOHAAAA|905|Center |Wy|Suite Q|Wilson|Rutherford County|TN|36971|United States|-6|apartment| +32410|AAAAAAAAKJOHAAAA|993|Fifth 2nd|Ave|Suite 240|Dewey|Nicollet County|MN|51160|United States|-6|single family| +32411|AAAAAAAALJOHAAAA|877|Wilson Sycamore|Circle|Suite K|Lakeside|Socorro County|NM|89532|United States|-7|condo| +32412|AAAAAAAAMJOHAAAA|830|Valley |Lane|Suite Y|Blair|Bowman County|ND|55465|United States|-6|single family| +32413|AAAAAAAANJOHAAAA||3rd |Parkway||Farmington|Lincoln County|||United States||apartment| +32414|AAAAAAAAOJOHAAAA|649|Eigth 14th|Way|Suite 390|Edgewood|Lee County|KY|40069|United States|-5|condo| +32415|AAAAAAAAPJOHAAAA|691|5th Lincoln|Dr.|Suite 60|Forest|Itawamba County|MS|57537|United States|-6|condo| +32416|AAAAAAAAAKOHAAAA|371|Oak |RD|Suite B|Harmony|Cascade County|MT|65804|United States|-7|apartment| +32417|AAAAAAAABKOHAAAA|836|Highland |RD|Suite 320|Woodville|Mitchell County|TX|74289|United States|-6|condo| +32418|AAAAAAAACKOHAAAA|751|Willow Spring|Blvd|Suite H|Oak Grove|Crawford County|IA|58370|United States|-6|condo| +32419|AAAAAAAADKOHAAAA|5|Park |Dr.|Suite 160|Lewis|Crawford County|IN|47066|United States|-5|single family| +32420|AAAAAAAAEKOHAAAA||Meadow Church|Road|Suite R|Spring Grove|||||-5|condo| +32421|AAAAAAAAFKOHAAAA|66|Sycamore |ST|Suite U|Forest Hills|Potter County|SD|59237|United States|-7|apartment| +32422|AAAAAAAAGKOHAAAA|20|Pine Oak|Blvd|Suite 240|Bridgeport|Pike County|PA|15817|United States|-5|single family| +32423|AAAAAAAAHKOHAAAA|281|Hill |Road|Suite V|Hopewell|Rice County|KS|60587|United States|-6|single family| +32424|AAAAAAAAIKOHAAAA|972|7th Hillcrest|Street|Suite B|New Hope|Kerr County|TX|79431|United States|-6|apartment| +32425|AAAAAAAAJKOHAAAA|666|4th |Blvd|Suite P|Concord|Oregon County|MO|64107|United States|-6|condo| +32426|AAAAAAAAKKOHAAAA|464|6th Fifth|Way|Suite 10|Mount Olive|Fairfield County|OH|48059|United States|-5|apartment| +32427|AAAAAAAALKOHAAAA|166|Lincoln |Blvd|Suite 10|Clinton|Steuben County|NY|18222|United States|-5|single family| +32428|AAAAAAAAMKOHAAAA|265|Smith River|Ct.|Suite 90|Newport|Coal County|OK|71521|United States|-6|condo| +32429|AAAAAAAANKOHAAAA|254|1st Third|Circle|Suite 140|Ellisville|Cattaraugus County|NY|16820|United States|-5|condo| +32430|AAAAAAAAOKOHAAAA|483|4th 2nd|Dr.|Suite M|Franklin|Grimes County|TX|79101|United States|-6|single family| +32431|AAAAAAAAPKOHAAAA|617|Church |Road|Suite T|Lakeview|Morgan County|IN|48579|United States|-5|condo| +32432|AAAAAAAAALOHAAAA|45|Wilson |Ln|Suite 280|Crossroads|Dukes County|MA|01134|United States|-5|single family| +32433|AAAAAAAABLOHAAAA|834|Laurel |Circle|Suite Y|Highland|Newton County|GA|39454|United States|-5|condo| +32434|AAAAAAAACLOHAAAA|185|Valley |Boulevard|Suite L|Deerfield|Webster County|MS|59840|United States|-6|apartment| +32435|AAAAAAAADLOHAAAA|765|Cherry South|Dr.|Suite 20|Shiloh|Sumner County|TN|39275|United States|-6|apartment| +32436|AAAAAAAAELOHAAAA|653|Madison West|ST|Suite D|Kingston|Chattahoochee County|GA|34975|United States|-5|apartment| +32437|AAAAAAAAFLOHAAAA|911|Highland 5th|Ct.|Suite J|Friendship|Gilliam County|OR|94536|United States|-8|condo| +32438|AAAAAAAAGLOHAAAA|803|Washington |Street|Suite L|Marion|Bennett County|SD|50399|United States|-6|apartment| +32439|AAAAAAAAHLOHAAAA|444|Johnson |Lane|Suite B|Shiloh|Cass County|IL|69275|United States|-6|condo| +32440|AAAAAAAAILOHAAAA|826|Johnson 7th|RD|Suite Q|Kingston|Larimer County|CO|84975|United States|-7|apartment| +32441|AAAAAAAAJLOHAAAA|851|Johnson |Avenue|Suite 480|Springfield|Chattooga County|GA|39303|United States|-5|apartment| +32442|AAAAAAAAKLOHAAAA|268|Lincoln 6th|Drive|Suite 450|Arlington|Sullivan County|NY|16557|United States|-5|single family| +32443|AAAAAAAALLOHAAAA|777|Sunset |RD|Suite O|Pleasant Hill|Saratoga County|NY|13604|United States|-5|condo| +32444|AAAAAAAAMLOHAAAA|123|Church Broadway|Road|Suite Q|Walnut Grove|Maui County|HI|97752|United States|-10|single family| +32445|AAAAAAAANLOHAAAA|664|Green Lincoln|Street|Suite 270|Five Points|Oconee County|GA|36098|United States|-5|single family| +32446|AAAAAAAAOLOHAAAA|583|Pine |Pkwy|Suite T|Stewart|Jefferson County|WA|98041|United States|-8|single family| +32447|AAAAAAAAPLOHAAAA|393|View |Parkway|Suite 210|Mount Pleasant|Tippecanoe County|IN|41933|United States|-5|single family| +32448|AAAAAAAAAMOHAAAA|28|9th 6th|Ave|Suite 30|Golden|Todd County|SD|50411|United States|-7|single family| +32449|AAAAAAAABMOHAAAA|966|Third |Avenue|Suite W|Green Acres|Washington County|MS|57683|United States|-6|apartment| +32450|AAAAAAAACMOHAAAA|437|7th |Circle|Suite J|Williamsville|Harrison County|IA|58754|United States|-6|single family| +32451|AAAAAAAADMOHAAAA|808||Cir.|Suite N|Woodlawn|Lucas County|IA|54098|United States|-6|| +32452|AAAAAAAAEMOHAAAA|111|Laurel Lincoln|Dr.|Suite 190|Lakeview|Richardson County|NE|68579|United States|-7|apartment| +32453|AAAAAAAAFMOHAAAA|222|Walnut 8th|Boulevard|Suite M|Cedar Grove|Macon County|IL|60411|United States|-6|apartment| +32454|AAAAAAAAGMOHAAAA|198|11th Sunset|Dr.|Suite 430|Farmington|Edwards County|KS|69145|United States|-6|single family| +32455|AAAAAAAAHMOHAAAA|472|Jefferson Center|RD|Suite T|Walnut Grove|Jackson County|FL|37752|United States|-5|apartment| +32456|AAAAAAAAIMOHAAAA|21|First Highland|ST|Suite T|Shady Grove|Stephens County|GA|32812|United States|-5|condo| +32457|AAAAAAAAJMOHAAAA|395|13th Green|Ln|Suite 420|Liberty|Fillmore County|NE|63451|United States|-6|condo| +32458|AAAAAAAAKMOHAAAA|610|5th |Dr.|Suite 310|Doyle|Wheeler County|OR|98434|United States|-8|apartment| +32459|AAAAAAAALMOHAAAA|229|East |Circle|Suite X|King|Pipestone County|MN|50008|United States|-6|single family| +32460|AAAAAAAAMMOHAAAA|468|Adams 2nd|Circle|Suite 390|Cedar Grove|Buchanan County|IA|50411|United States|-6|apartment| +32461|AAAAAAAANMOHAAAA|440|||Suite 100||Mason County|TX|||-6|condo| +32462|AAAAAAAAOMOHAAAA|162|Main |||||IN|48371|||| +32463|AAAAAAAAPMOHAAAA|242|Smith Second|Drive|Suite E|Shiloh|Mahaska County|IA|59275|United States|-6|apartment| +32464|AAAAAAAAANOHAAAA|24|Park Elm|Court|Suite G|Mountain View|Washington County|AL|34466|United States|-6|condo| +32465|AAAAAAAABNOHAAAA|783|5th |Drive|Suite 330|Oak Ridge|Winona County|MN|58371|United States|-6|single family| +32466|AAAAAAAACNOHAAAA|632|Hill |Ln|Suite 40|Five Points|Gentry County|MO|66098|United States|-6|apartment| +32467|AAAAAAAADNOHAAAA|179|River Madison|Ct.|Suite 230|White Oak|Rapides Parish|LA|76668|United States|-6|apartment| +32468|AAAAAAAAENOHAAAA|230|North Oak|Way|Suite Y|Five Points|Bedford County|VA|26098|United States|-5|single family| +32469|AAAAAAAAFNOHAAAA|66|Sixth |Circle|Suite 390|Greenwood|Mineral County|NV|88828|United States|-8|apartment| +32470|AAAAAAAAGNOHAAAA|48|Lake |Cir.|Suite 130|Mount Vernon|Webster County|NE|68482|United States|-7|apartment| +32471|AAAAAAAAHNOHAAAA|961|7th Meadow|Avenue|Suite 420|Oak Grove|Orange County|FL|38370|United States|-5|condo| +32472|AAAAAAAAINOHAAAA|3|6th Cedar|Ln|Suite 130|Greenville|Radford city|VA|21387|United States|-5|single family| +32473|AAAAAAAAJNOHAAAA|486|Lee Park|Ct.|Suite M|Elba|Columbia County|NY|10262|United States|-5|single family| +32474|AAAAAAAAKNOHAAAA|595|Maple Park|Pkwy|Suite 40|Bridgeport|Perry County|AR|75817|United States|-6|condo| +32475|AAAAAAAALNOHAAAA|491|Main 4th|Parkway|Suite 50|Tracy|Kleberg County|TX|76340|United States|-6|apartment| +32476|AAAAAAAAMNOHAAAA|71|Main Woodland|Street|Suite D|Providence|Adair County|MO|66614|United States|-6|apartment| +32477|AAAAAAAANNOHAAAA|863|Pine Walnut|Street|Suite 250|The Meadows|Fayette County|IA|50026|United States|-6|apartment| +32478|AAAAAAAAONOHAAAA|423|Smith Maple|Boulevard|Suite 140|Four Points|Uintah County|UT|81216|United States|-7|apartment| +32479|AAAAAAAAPNOHAAAA|435|10th |Lane|Suite X|Five Forks|Wayne County|UT|82293|United States|-7|apartment| +32480|AAAAAAAAAOOHAAAA|220|5th 2nd|Circle|Suite 310|Fairview|Wilkes County|NC|25709|United States|-5|single family| +32481|AAAAAAAABOOHAAAA|64|14th |Boulevard|Suite 440|Pierce|Jackson County|MO|63360|United States|-6|condo| +32482|AAAAAAAACOOHAAAA|295|Sycamore Jefferson|Circle|Suite 430|Spring Valley|Monroe County|MO|66060|United States|-6|single family| +32483|AAAAAAAADOOHAAAA|254|Laurel |Circle|Suite 360|Riverview|Jefferson County|MT|69003|United States|-7|single family| +32484|AAAAAAAAEOOHAAAA|545|2nd |Blvd|Suite C|Spring Hill|Culberson County|TX|76787|United States|-6|apartment| +32485|AAAAAAAAFOOHAAAA|823|7th |Way|Suite Q|Concord|Lamar County|MS|54107|United States|-6|apartment| +32486|AAAAAAAAGOOHAAAA|796|Miller Fourth|Lane|Suite B|Ashland|Tolland County|CT|04844|United States|-5|condo| +32487|AAAAAAAAHOOHAAAA|12|Miller |Dr.|Suite 10|Centerville|Tulsa County|OK|70059|United States|-6|apartment| +32488|AAAAAAAAIOOHAAAA|612|Willow 15th|Circle|Suite 10|Summit|Lincoln County|NV|80499|United States|-8|condo| +32489|AAAAAAAAJOOHAAAA|371|4th |Ave|Suite 110|Spring Hill|Franklin County|ME|07387|United States|-5|apartment| +32490|AAAAAAAAKOOHAAAA|488|Mill |RD|Suite L|Plainview|Leelanau County|MI|43683|United States|-5|condo| +32491|AAAAAAAALOOHAAAA|865|Pine Walnut|Court|Suite 0|Hillcrest|Columbia County|OR|93003|United States|-8|single family| +32492|AAAAAAAAMOOHAAAA|844|Main |Parkway|Suite 340|Bethel|Waupaca County|WI|55281|United States|-6|condo| +32493|AAAAAAAANOOHAAAA|841|6th |Ct.|Suite C|Five Forks|Yadkin County|NC|22293|United States|-5|apartment| +32494|AAAAAAAAOOOHAAAA|910|Birch Davis|Ave|Suite 280|Oakwood|Andrew County|MO|60169|United States|-6|single family| +32495|AAAAAAAAPOOHAAAA|533|Church |Street|Suite B|Unionville|Oneida County|ID|81711|United States|-7|apartment| +32496|AAAAAAAAAPOHAAAA|624|Cherry |Drive|Suite B|Vista|Canyon County|ID|84694|United States|-7|apartment| +32497|AAAAAAAABPOHAAAA|618|View Ridge|Court|Suite K|Centerville|Jerome County|ID|80059|United States|-7|condo| +32498|AAAAAAAACPOHAAAA|80|Cedar |Drive|Suite 390|Fairview|Grand Isle County|VT|06309|United States|-5|condo| +32499|AAAAAAAADPOHAAAA|848|1st |Drive|Suite 120|Springdale|Lamar County|TX|78883|United States|-6|condo| +32500|AAAAAAAAEPOHAAAA|135|9th Jefferson|Ln|Suite 60|Providence|Todd County|MN|56614|United States|-6|single family| +32501|AAAAAAAAFPOHAAAA|649|13th 10th|Blvd|Suite 340|Lebanon|Iroquois County|IL|62898|United States|-6|apartment| +32502|AAAAAAAAGPOHAAAA|810|Elm 9th|RD|Suite 320|Lincoln|Bienville Parish|LA|71289|United States|-6|single family| +32503|AAAAAAAAHPOHAAAA|915|6th |Parkway|Suite T|Brownsville|Smyth County|VA|29310|United States|-5|apartment| +32504|AAAAAAAAIPOHAAAA|710|Fifth Central|Court|Suite V|Greenwood|Lawrence County|IL|68828|United States|-6|single family| +32505|AAAAAAAAJPOHAAAA|136|Lee |Boulevard|Suite A|Mount Olive|Columbia County|WA|98059|United States|-8|single family| +32506|AAAAAAAAKPOHAAAA|312|Hill 15th|Ct.|Suite 380|Greenwood|Hickman County|KY|48828|United States|-6|single family| +32507|AAAAAAAALPOHAAAA|665|Locust Hillcrest|Wy|Suite 240|Macedonia|Eastland County|TX|71087|United States|-6|apartment| +32508|AAAAAAAAMPOHAAAA|786|Park 13th|ST|Suite W|Forest Hills|Flagler County|FL|39237|United States|-5|apartment| +32509|AAAAAAAANPOHAAAA|207|5th South|Way|Suite T|Stringtown|Dougherty County|GA|30162|United States|-5|single family| +32510|AAAAAAAAOPOHAAAA|352|Sixth |Avenue|Suite 20|Five Forks|Marshall County|OK|72293|United States|-6|single family| +32511|AAAAAAAAPPOHAAAA|402|11th Ash|Road|Suite E|Brownsville|Brule County|SD|59310|United States|-6|single family| +32512|AAAAAAAAAAPHAAAA|867|11th Oak|Drive|Suite S|Pierce|Cooper County|MO|63360|United States|-6|apartment| +32513|AAAAAAAABAPHAAAA|796|5th Park|Avenue|Suite V|Crossroads|Montgomery County|AR|70534|United States|-6|apartment| +32514|AAAAAAAACAPHAAAA|411|Lake |Ct.|Suite G|Oak Ridge|Lee County|IL|68371|United States|-6|apartment| +32515|AAAAAAAADAPHAAAA|29|Lincoln |Boulevard|Suite 50|White Plains|Beauregard Parish|LA|76622|United States|-6|condo| +32516|AAAAAAAAEAPHAAAA|473|Hillcrest |Street|Suite N|Union Hill|Traill County|ND|57746|United States|-6|apartment| +32517|AAAAAAAAFAPHAAAA|270|Laurel Park|Ave|Suite 150|Richville|Poweshiek County|IA|55945|United States|-6|condo| +32518|AAAAAAAAGAPHAAAA|175|Second |Road|Suite 170|Marion|Licking County|OH|40399|United States|-5|condo| +32519|AAAAAAAAHAPHAAAA|965|Jefferson |Drive|Suite 150|Highland Park|Pierce County|NE|66534|United States|-7|single family| +32520|AAAAAAAAIAPHAAAA|526|Spring |Drive|Suite 490|Green Acres|Marshall County|OK|77683|United States|-6|single family| +32521|AAAAAAAAJAPHAAAA|882|Madison Franklin|Dr.|Suite C|Arlington|Mathews County|VA|26557|United States|-5|condo| +32522|AAAAAAAAKAPHAAAA|606|Wilson 4th|Street|Suite F|Bear Creek|Pope County|MN|53075|United States|-6|condo| +32523|AAAAAAAALAPHAAAA|200|Valley Locust|Circle|Suite M|Green Acres|Clallam County|WA|97683|United States|-8|single family| +32524|AAAAAAAAMAPHAAAA|510|8th Jackson|Ln|Suite I|Marion|Arenac County|MI|40399|United States|-5|apartment| +32525|AAAAAAAANAPHAAAA|811|Hill 1st|Avenue|Suite G|Stringtown|Churchill County|NV|80162|United States|-8|condo| +32526|AAAAAAAAOAPHAAAA|247|5th |Wy|Suite 480|Five Forks|Lake County|IL|62293|United States|-6|single family| +32527|AAAAAAAAPAPHAAAA|958|9th Ninth|Street||Lincoln||GA||United States||apartment| +32528|AAAAAAAAABPHAAAA|166|Sycamore Railroad|Road|Suite R|Bunker Hill|Catoosa County|GA|30150|United States|-5|apartment| +32529|AAAAAAAABBPHAAAA|227|2nd Tenth|Cir.|Suite W|Ellisville|Uinta County|WY|86820|United States|-7|apartment| +32530|AAAAAAAACBPHAAAA|437|15th |Blvd|Suite 370|Hillcrest|Christian County|MO|63003|United States|-6|condo| +32531|AAAAAAAADBPHAAAA|733|Valley |Wy|Suite 470|Concord|Rockland County|NY|14107|United States|-5|single family| +32532|AAAAAAAAEBPHAAAA||Highland |Ct.|Suite 360|Five Points|Calloway County||46098||-6|| +32533|AAAAAAAAFBPHAAAA|39|Washington Second|||Mount Zion|Page County|IA|58054|United States|-6|single family| +32534|AAAAAAAAGBPHAAAA|395|Spring |Court|Suite V|Woodville|Lafayette County|MO|64289|United States|-6|single family| +32535|AAAAAAAAHBPHAAAA|830|Park Lee|Avenue|Suite X|Spring Valley|Dorchester County|SC|26060|United States|-5|apartment| +32536|AAAAAAAAIBPHAAAA|701|Maple Cedar|Ave|Suite W|Summit|Paulding County|OH|40499|United States|-5|condo| +32537|AAAAAAAAJBPHAAAA|449|5th Williams|Ct.|Suite 190|Oakland|Tipton County|TN|39843|United States|-6|apartment| +32538|AAAAAAAAKBPHAAAA|694|North |Drive|Suite 160|Vance|Colusa County|CA|90268|United States|-8|apartment| +32539|AAAAAAAALBPHAAAA|121|Fourth Franklin|Lane|Suite P|Murray|Porter County|IN|42150|United States|-5|single family| +32540|AAAAAAAAMBPHAAAA|941|Dogwood Main|Street|Suite W|Fisher|Scott County|TN|32819|United States|-6|apartment| +32541|AAAAAAAANBPHAAAA|61|12th River|Avenue|Suite V||Jefferson County||49584||-6|apartment| +32542|AAAAAAAAOBPHAAAA|730|12th 5th|Road|Suite 260|Union|Tippah County|MS|58721|United States|-6|condo| +32543|AAAAAAAAPBPHAAAA|286|Cedar |Drive|Suite Q|Lakeside|Hanson County|SD|59532|United States|-7|condo| +32544|AAAAAAAAACPHAAAA|298|10th Jefferson|Blvd|Suite K|Antioch|Shelby County|OH|48605|United States|-5|single family| +32545|AAAAAAAABCPHAAAA|128|15th |Parkway|Suite 50|Pleasant Grove|Lake of the Woods County|MN|54136|United States|-6|condo| +32546|AAAAAAAACCPHAAAA|394|Mill Adams|Parkway|Suite A|Plainview|Wabaunsee County|KS|63683|United States|-6|apartment| +32547|AAAAAAAADCPHAAAA|385|Maple |Ln|Suite X|Oakwood|Beaverhead County|MT|60169|United States|-7|apartment| +32548|AAAAAAAAECPHAAAA|456|Lake |ST|Suite 120|Sunnyside|Wyoming County|PA|11952|United States|-5|condo| +32549|AAAAAAAAFCPHAAAA|668|9th |Avenue|Suite 160|Greenfield|Wilson County|KS|65038|United States|-6|single family| +32550|AAAAAAAAGCPHAAAA|812|6th 14th|Road|Suite Q|Millbrook|Calloway County|KY|47529|United States|-6|apartment| +32551|AAAAAAAAHCPHAAAA|428|Ash |Avenue|Suite F|Liberty|Sherman County|TX|73451|United States|-6|apartment| +32552|AAAAAAAAICPHAAAA|349|Locust College|ST|Suite 270|Sulphur Springs|Allen Parish|LA|78354|United States|-6|condo| +32553|AAAAAAAAJCPHAAAA|32|11th |RD|Suite G|Harmony|Forsyth County|GA|35804|United States|-5|single family| +32554|AAAAAAAAKCPHAAAA|115|West |Pkwy|Suite 40|Fairfield|Vigo County|IN|46192|United States|-5|condo| +32555|AAAAAAAALCPHAAAA|529|Hill Park|Ln|Suite 50|Oak Hill|Haywood County|NC|27838|United States|-5|apartment| +32556|AAAAAAAAMCPHAAAA|844|Center |Wy|Suite L|Adrian|Monroe County|WV|23301|United States|-5|apartment| +32557|AAAAAAAANCPHAAAA|394|Oak Maple|Road|Suite P|Red Hill|Benton County|MO|64338|United States|-6|condo| +32558|AAAAAAAAOCPHAAAA|749|Highland |Dr.|Suite 380|Newtown|Saguache County|CO|81749|United States|-7|apartment| +32559|AAAAAAAAPCPHAAAA|441|Elm Lincoln|Parkway|Suite 250|Providence|Keya Paha County|NE|66614|United States|-7|single family| +32560|AAAAAAAAADPHAAAA|570|Cedar 4th|Court|Suite C|Riverside|Bandera County|TX|79231|United States|-6|condo| +32561|AAAAAAAABDPHAAAA|633|Pine |Circle|Suite 110|Deerfield|Montgomery County|MD|29840|United States|-5|apartment| +32562|AAAAAAAACDPHAAAA|26|Miller |Way|Suite 480|Woodland|McPherson County|KS|64854|United States|-6|condo| +32563|AAAAAAAADDPHAAAA|214|Maple |RD|Suite Q|Mount Olive|Montgomery County|NY|18059|United States|-5|single family| +32564|AAAAAAAAEDPHAAAA|209|8th Tenth|Boulevard|Suite 180|Oak Grove|Sevier County|UT|88370|United States|-7|single family| +32565|AAAAAAAAFDPHAAAA|556|Smith |Circle|Suite U|Macedonia|Taney County|MO|61087|United States|-6|single family| +32566|AAAAAAAAGDPHAAAA|747|West |Dr.|Suite 190|Unionville|Duchesne County|UT|81711|United States|-7|apartment| +32567|AAAAAAAAHDPHAAAA|140|Willow |Dr.|Suite 190|Riverdale|Grayson County|KY|49391|United States|-6|single family| +32568|AAAAAAAAIDPHAAAA|627|Walnut Smith|Circle|Suite 280|Liberty|Orange County|TX|73451|United States|-6|apartment| +32569|AAAAAAAAJDPHAAAA|573|6th Mill|Ct.|Suite 140|Red Hill|Hillsborough County|NH|04938|United States|-5|single family| +32570|AAAAAAAAKDPHAAAA|773|1st |Drive|Suite L|Five Points|Carter County|MO|66098|United States|-6|apartment| +32571|AAAAAAAALDPHAAAA|470|Green |Parkway|Suite 410|Wildwood|Clark County|AR|76871|United States|-6|condo| +32572|AAAAAAAAMDPHAAAA|455|4th Madison|Parkway|Suite 20|Arlington|Cherokee County|SC|26557|United States|-5|apartment| +32573|AAAAAAAANDPHAAAA|577||Ct.||Bethel||SD|||-6|apartment| +32574|AAAAAAAAODPHAAAA|296|Ridge |Circle|Suite L|Pine Grove|Morrow County|OH|44593|United States|-5|single family| +32575|AAAAAAAAPDPHAAAA|709|South Ridge|Blvd|Suite 370|Brownsville|Cumberland County|KY|49310|United States|-6|apartment| +32576|AAAAAAAAAEPHAAAA|386|Lakeview Lake|ST|Suite Y|Antioch|Gosper County|NE|68605|United States|-6|condo| +32577|AAAAAAAABEPHAAAA|304|Lincoln Maple|Cir.|Suite G|Cedar Grove|Steele County|ND|50411|United States|-6|single family| +32578|AAAAAAAACEPHAAAA|470|Green Oak|Dr.|Suite 260|Spring Hill|Orange County|NY|16787|United States|-5|single family| +32579|AAAAAAAADEPHAAAA|334|Hickory First|Ct.|Suite G|Mount Vernon|Franklin County|AL|38482|United States|-6|single family| +32580|AAAAAAAAEEPHAAAA|166|Willow |Parkway|Suite I|Mount Zion|Dawson County|MT|68054|United States|-7|single family| +32581|AAAAAAAAFEPHAAAA|995|Park Walnut|Blvd|Suite W|Montague|Swift County|MN|54062|United States|-6|single family| +32582|AAAAAAAAGEPHAAAA|955|Central 10th|Blvd|Suite 300|Liberty|Marshall County|SD|53451|United States|-7|apartment| +32583|AAAAAAAAHEPHAAAA|246|Hickory |Court|Suite U|Riverside|Richmond County|NC|29231|United States|-5|apartment| +32584|AAAAAAAAIEPHAAAA|859|||Suite R|||PA|16557|||single family| +32585|AAAAAAAAJEPHAAAA||Pine |Ln|||Montgomery County|MS|59310|United States||| +32586|AAAAAAAAKEPHAAAA|907|3rd Walnut|Court|Suite V|Oakland|Washington County|IN|49843|United States|-5|single family| +32587|AAAAAAAALEPHAAAA|667|Thirteenth |Pkwy|Suite 430|Greenwood|Finney County|KS|68828|United States|-6|apartment| +32588|AAAAAAAAMEPHAAAA|641|Oak Hickory|Drive|Suite Q|Pleasant Valley|Lee County|IA|52477|United States|-6|condo| +32589|AAAAAAAANEPHAAAA|270|Park Main|Wy|Suite 110|Oak Hill|Menifee County|KY|47838|United States|-5|apartment| +32590|AAAAAAAAOEPHAAAA|446|Lake |Road|Suite V|Marion|Arthur County|NE|60399|United States|-6|apartment| +32591|AAAAAAAAPEPHAAAA|879|Highland |Wy|Suite 40|Bunker Hill|Columbia County|WA|90150|United States|-8|condo| +32592|AAAAAAAAAFPHAAAA|277|15th Forest|Pkwy|Suite W|Cedar Grove|Nolan County|TX|70411|United States|-6|condo| +32593|AAAAAAAABFPHAAAA|941|3rd Sunset|RD|Suite 360|Mount Vernon|Tehama County|CA|98482|United States|-8|apartment| +32594|AAAAAAAACFPHAAAA|43|8th Lincoln|Dr.|Suite H|Liberty|Shoshone County|ID|83451|United States|-7|condo| +32595|AAAAAAAADFPHAAAA|553|Eigth East|Circle|Suite 370|Union Hill|Grays Harbor County|WA|97746|United States|-8|condo| +32596|AAAAAAAAEFPHAAAA|789|River |Avenue|Suite C|Hopewell|Seminole County|FL|30587|United States|-5|apartment| +32597|AAAAAAAAFFPHAAAA|772|Pine First|Dr.|Suite 110|Greenwood|Jackson County|NC|28828|United States|-5|single family| +32598|AAAAAAAAGFPHAAAA|719|View Fourth|Lane|Suite K|White Oak|Adams County|IA|56668|United States|-6|condo| +32599|AAAAAAAAHFPHAAAA|535|9th Franklin|Wy|Suite X|Springfield|Marion County|TN|39303|United States|-6|single family| +32600|AAAAAAAAIFPHAAAA|679|Locust |Ln|Suite 250|Clinton|Howard County|IA|58222|United States|-6|apartment| +32601|AAAAAAAAJFPHAAAA|930|Park |Parkway|Suite 90|Bridgeport|Calhoun County|MI|45817|United States|-5|apartment| +32602|AAAAAAAAKFPHAAAA|65|Madison |Circle|Suite V|Ashland|Bayfield County|WI|54244|United States|-6|single family| +32603|AAAAAAAALFPHAAAA|900|1st |Circle|Suite 0|Wilson|Williamsburg city|VA|26971|United States|-5|condo| +32604|AAAAAAAAMFPHAAAA|134|3rd Cedar|Way|Suite I|Bridgeport|Shelby County|KY|45817|United States|-5|single family| +32605|AAAAAAAANFPHAAAA|193|12th |Boulevard|Suite 300|Cherry Valley|Fayette County|IN|40854|United States|-5|single family| +32606|AAAAAAAAOFPHAAAA|879|Jefferson |Avenue|Suite S|Fairfield|Schoharie County|NY|16192|United States|-5|condo| +32607|AAAAAAAAPFPHAAAA|493|Fourth |Way|Suite S|Brownsville|Furnas County|NE|69310|United States|-6|condo| +32608|AAAAAAAAAGPHAAAA|97|West Sunset|RD|Suite J|Mountain View|Blount County|TN|34466|United States|-5|single family| +32609|AAAAAAAABGPHAAAA|990|Jefferson 7th|Lane|Suite 130|Fairview|Bath County|VA|25709|United States|-5|condo| +32610|AAAAAAAACGPHAAAA||||Suite E|Woodville||GA|34289|United States||single family| +32611|AAAAAAAADGPHAAAA|952|Sycamore 12th|Pkwy|Suite 90|Newport|Washington County|CO|81521|United States|-7|single family| +32612|AAAAAAAAEGPHAAAA|873|Spring 14th|Way|Suite 240|Deerfield|Iberia Parish|LA|79840|United States|-6|single family| +32613|AAAAAAAAFGPHAAAA|479|Lake 8th|Ave|Suite L|Fairfield|Chase County|NE|66192|United States|-6|apartment| +32614|AAAAAAAAGGPHAAAA|640|Lincoln |Boulevard|Suite 350|White Oak|Madison County|ID|86668|United States|-7|single family| +32615|AAAAAAAAHGPHAAAA||Lakeview Sixth||Suite 290||||30059||-5|| +32616|AAAAAAAAIGPHAAAA|261|Railroad |Ct.|Suite 210|Greenville|Fayette County|WV|21387|United States|-5|apartment| +32617|AAAAAAAAJGPHAAAA|509|3rd Thirteenth|Ave|Suite 210|Liberty|Meeker County|MN|53451|United States|-6|single family| +32618|AAAAAAAAKGPHAAAA|941|Jefferson Johnson|Road|Suite 480|Franklin|Rowan County|NC|29101|United States|-5|condo| +32619|AAAAAAAALGPHAAAA|326|Ridge |Wy|Suite 150|Buena Vista|Pierce County|WI|55752|United States|-6|apartment| +32620|AAAAAAAAMGPHAAAA|526|Walnut 8th|Court|Suite W|Farmington|Hill County|MT|69145|United States|-7|single family| +32621|AAAAAAAANGPHAAAA|560|Williams Fourth|Lane|Suite 410|Fairfield|Yancey County|NC|26192|United States|-5|single family| +32622|AAAAAAAAOGPHAAAA|167|Ridge Elevnth|Ct.|Suite 90|Woodlawn|Cherokee County|NC|24098|United States|-5|condo| +32623|AAAAAAAAPGPHAAAA|61|Maple |Drive|Suite O|Kingston|Kent County|RI|05575|United States|-5|condo| +32624|AAAAAAAAAHPHAAAA|475|Ridge Williams|RD|Suite 230|Providence|Iredell County|NC|26614|United States|-5|condo| +32625|AAAAAAAABHPHAAAA|207|Forest Second|Ave|Suite N|Wilson|Young County|TX|76971|United States|-6|condo| +32626|AAAAAAAACHPHAAAA|109|College Eigth|Avenue|Suite 270|Red Hill|Pope County|IL|64338|United States|-6|condo| +32627|AAAAAAAADHPHAAAA|530|Lake Broadway|Drive|Suite R|Brownsville|Gaines County|TX|79310|United States|-6|condo| +32628|AAAAAAAAEHPHAAAA|894|Johnson Forest|RD|Suite E|Centerville|Dallas County|TX|70059|United States|-6|single family| +32629|AAAAAAAAFHPHAAAA|241|Jackson Railroad|Ave|Suite 200|Redland|Rhea County|TN|36343|United States|-6|apartment| +32630|AAAAAAAAGHPHAAAA|747|6th |Ct.|Suite 130|Florence|Neshoba County|MS|53394|United States|-6|apartment| +32631|AAAAAAAAHHPHAAAA|835|Jackson 6th|Court|Suite W|Oak Hill|Orange County|CA|97838|United States|-8|condo| +32632|AAAAAAAAIHPHAAAA|892|Center |Street|Suite V|Pleasant Valley|Cortland County|NY|12477|United States|-5|single family| +32633|AAAAAAAAJHPHAAAA|634|Hickory |Dr.|Suite O|Lakeview|Wilcox County|AL|38579|United States|-6|apartment| +32634|AAAAAAAAKHPHAAAA|811|Washington Sunset|Ct.|Suite R|Aberdeen|Ogemaw County|MI|47995|United States|-5|apartment| +32635|AAAAAAAALHPHAAAA|70|||||||36871||-5|apartment| +32636|AAAAAAAAMHPHAAAA||||Suite 140|Lakeside|||||-5|| +32637|AAAAAAAANHPHAAAA|438|11th |Boulevard|Suite 140|Peoria|Pierce County|WI|59818|United States|-6|condo| +32638|AAAAAAAAOHPHAAAA|403|Cedar |Ct.|Suite 360|Valley View|Dukes County|MA|05724|United States|-5|apartment| +32639|AAAAAAAAPHPHAAAA|187|Davis |Blvd|Suite R|Wildwood|Walla Walla County|WA|96871|United States|-8|single family| +32640|AAAAAAAAAIPHAAAA|288|View |Pkwy|Suite N|La Grange|Webster County|MS|57941|United States|-6|single family| +32641|AAAAAAAABIPHAAAA|626|Lincoln |Ave|Suite 20|Wilson|Armstrong County|PA|16971|United States|-5|single family| +32642|AAAAAAAACIPHAAAA|999|South |Court|Suite U|Waterloo|Edgefield County|SC|21675|United States|-5|condo| +32643|AAAAAAAADIPHAAAA|300|View |Wy|Suite X|Union Hill|Henry County|IL|67746|United States|-6|apartment| +32644|AAAAAAAAEIPHAAAA|905|1st Green|Ln|Suite U|Brownsville|Daggett County|UT|89310|United States|-7|apartment| +32645|AAAAAAAAFIPHAAAA|937|Center Maple|Street|Suite 60|Woodville|Roger Mills County|OK|74289|United States|-6|apartment| +32646|AAAAAAAAGIPHAAAA|97|14th |Wy|Suite 370|Florence|Moffat County|CO|83394|United States|-7|apartment| +32647|AAAAAAAAHIPHAAAA|380|Maple |Boulevard|Suite W|Georgetown|Deuel County|SD|57057|United States|-6|condo| +32648|AAAAAAAAIIPHAAAA|65|West Main|Circle|Suite 240|Greenfield|Baltimore County|MD|25038|United States|-5|single family| +32649|AAAAAAAAJIPHAAAA|995|Spring |RD|Suite 440|Harmony|Allen County|IN|45804|United States|-5|single family| +32650|AAAAAAAAKIPHAAAA|172|Cedar |Road|Suite Y|Stringtown|Carroll County|NH|00762|United States|-5|condo| +32651|AAAAAAAALIPHAAAA|853|East Cedar|Ct.|Suite Q|Glendale|Neosho County|KS|63951|United States|-6|single family| +32652|AAAAAAAAMIPHAAAA|187|Meadow Spring|Wy|Suite 300|Hopewell|Carroll County|OH|40587|United States|-5|condo| +32653|AAAAAAAANIPHAAAA|789|Hill |Blvd|Suite X|Lincoln|McCook County|SD|51289|United States|-7|apartment| +32654|AAAAAAAAOIPHAAAA|952|Lake |Ln|Suite O|Highland|Edmunds County|SD|59454|United States|-6|condo| +32655|AAAAAAAAPIPHAAAA|837|Spring Park|Pkwy|Suite H|Friendship|Mason County|TX|74536|United States|-6|apartment| +32656|AAAAAAAAAJPHAAAA|652|Locust |Parkway|Suite M|Harmony|Issaquena County|MS|55804|United States|-6|condo| +32657|AAAAAAAABJPHAAAA|649|Park Tenth|Way|Suite 430|Belmont|DeKalb County|AL|30191|United States|-6|apartment| +32658|AAAAAAAACJPHAAAA|187|Lincoln Lincoln|Cir.|Suite E|Oak Ridge|Yell County|AR|78371|United States|-6|single family| +32659|AAAAAAAADJPHAAAA|797|Railroad 8th|Court|Suite 130|Maple Grove|Gregory County|SD|58252|United States|-6|condo| +32660|AAAAAAAAEJPHAAAA|909|Davis |Parkway|Suite A|Belmont|Polk County|TN|30191|United States|-6|single family| +32661|AAAAAAAAFJPHAAAA|430|8th |Court|Suite 90|Clinton|Benton County|IA|58222|United States|-6|single family| +32662|AAAAAAAAGJPHAAAA|579|13th 8th|Drive|Suite V|Oak Ridge|Taylor County|TX|78371|United States|-6|single family| +32663|AAAAAAAAHJPHAAAA|587|Park Lee|ST|Suite V|Wildwood|Montgomery County|OH|46871|United States|-5|single family| +32664|AAAAAAAAIJPHAAAA|693|12th |Ct.|Suite T|Galena|Calhoun County|AR|74369|United States|-6|condo| +32665|AAAAAAAAJJPHAAAA|719|Forest Jackson|ST|Suite 40|Maple Grove|Madison County|KY|48252|United States|-5|single family| +32666|AAAAAAAAKJPHAAAA|480|View |ST|Suite 340|Wilson|Orange County|NY|16971|United States|-5|condo| +32667|AAAAAAAALJPHAAAA|510|View |Boulevard|Suite 10|Enterprise|Lawrence County|KY|41757|United States|-5|apartment| +32668|AAAAAAAAMJPHAAAA|432|Lincoln |RD|Suite 40|Maple Grove|Brunswick County|NC|28252|United States|-5|apartment| +32669|AAAAAAAANJPHAAAA|432|North |RD|Suite 420|Lincoln|Macon County|NC|21289|United States|-5|condo| +32670|AAAAAAAAOJPHAAAA|283|Third Dogwood|Cir.|Suite T|Valley View|Ingham County|MI|45124|United States|-5|single family| +32671|AAAAAAAAPJPHAAAA|358|Sunset 1st|Cir.|Suite N|Carlisle|Bear Lake County|ID|81620|United States|-7|single family| +32672|AAAAAAAAAKPHAAAA|693|Sunset Seventh|Boulevard|Suite 350|Kingston|Power County|ID|84975|United States|-7|single family| +32673|AAAAAAAABKPHAAAA|346|Poplar |Circle|Suite N|Newtown|Marion County|IA|51749|United States|-6|single family| +32674|AAAAAAAACKPHAAAA|86|Jefferson |Drive|Suite 210|Summerfield|Rappahannock County|VA|20634|United States|-5|condo| +32675|AAAAAAAADKPHAAAA|351|7th 8th|Cir.|Suite O|Lakeside|Crawford County|IA|59532|United States|-6|apartment| +32676|AAAAAAAAEKPHAAAA|651|Sunset Park|Ct.|Suite 20|Ashland|Dallas County|AR|74244|United States|-6|condo| +32677|AAAAAAAAFKPHAAAA|52|Dogwood Ninth|Way|Suite 220|Providence|Montgomery County|TX|76614|United States|-6|apartment| +32678|AAAAAAAAGKPHAAAA|29|Valley |ST|Suite 480|Mount Olive|Henry County|TN|38059|United States|-5|apartment| +32679|AAAAAAAAHKPHAAAA|719|Woodland |Way|Suite 410|Waterloo|Love County|OK|71675|United States|-6|apartment| +32680|AAAAAAAAIKPHAAAA|917|Fifth Hill|Circle|Suite Q|Unionville|Fauquier County|VA|21711|United States|-5|apartment| +32681|AAAAAAAAJKPHAAAA|913|Madison |Boulevard|Suite 370|Enterprise|Morgan County|AL|31757|United States|-6|condo| +32682|AAAAAAAAKKPHAAAA|529|Hill Jackson|Circle|Suite N|Arthur|East Carroll Parish|LA|75965|United States|-6|condo| +32683|AAAAAAAALKPHAAAA|955|Walnut Ridge|Boulevard|Suite W|Cedar Grove|Troup County|GA|30411|United States|-5|condo| +32684|AAAAAAAAMKPHAAAA|481|4th Cedar|Pkwy|Suite 100|Sulphur Springs|Fremont County|ID|88354|United States|-7|single family| +32685|AAAAAAAANKPHAAAA|189|Willow 2nd|Pkwy|Suite R|Waterloo|Bartholomew County|IN|41675|United States|-5|condo| +32686|AAAAAAAAOKPHAAAA|383|9th |RD|Suite 100|Antioch|Pueblo County|CO|88605|United States|-7|condo| +32687|AAAAAAAAPKPHAAAA|958|Maple |ST|Suite H|Union Hill|King County|WA|97746|United States|-8|condo| +32688|AAAAAAAAALPHAAAA|784|Poplar Sunset|RD|Suite 330|Mount Olive|Bowie County|TX|78059|United States|-6|condo| +32689|AAAAAAAABLPHAAAA|591|Third Wilson|Parkway|Suite 330|Pine Grove|Adams County|CO|84593|United States|-7|condo| +32690|AAAAAAAACLPHAAAA|758|10th Wilson|Road|Suite I|Lakewood|Pottawatomie County|KS|68877|United States|-6|apartment| +32691|AAAAAAAADLPHAAAA|530|Madison Maple|Ln|Suite C|Valley View|Henry County|KY|45124|United States|-6|single family| +32692|AAAAAAAAELPHAAAA|841|Johnson 3rd|Pkwy|Suite 150|Sutton|Pierce County|WA|95413|United States|-8|single family| +32693|AAAAAAAAFLPHAAAA|972|Madison |Parkway|Suite C|Cedar Grove|Lee County|SC|20411|United States|-5|condo| +32694|AAAAAAAAGLPHAAAA|477|River |Way|Suite 450|Ashland|Gila County|AZ|84244|United States|-7|condo| +32695|AAAAAAAAHLPHAAAA|31|Cedar |Blvd|Suite 170|Mount Olive|Vinton County|OH|48059|United States|-5|single family| +32696|AAAAAAAAILPHAAAA|968|Hillcrest |Road|Suite 130|Oakwood|Simpson County|MS|50169|United States|-6|single family| +32697|AAAAAAAAJLPHAAAA|24|Church Maple|RD|Suite 90|Plainview|Fulton County|IL|63683|United States|-6|single family| +32698|AAAAAAAAKLPHAAAA|177|3rd |Way|Suite R|Red Hill|Dallas County|TX|74338|United States|-6|apartment| +32699|AAAAAAAALLPHAAAA|211|View Third|ST|Suite O|Concord|Chautauqua County|NY|14107|United States|-5|condo| +32700|AAAAAAAAMLPHAAAA|602|Smith |Street|Suite 220|Greenville|Monroe County|NY|11387|United States|-5|condo| +32701|AAAAAAAANLPHAAAA|176|Valley 3rd|RD|Suite H|Kingston|Ripley County|IN|44975|United States|-5|condo| +32702|AAAAAAAAOLPHAAAA|430|Laurel |Drive|Suite 330|Friendship|Murray County|GA|34536|United States|-5|apartment| +32703|AAAAAAAAPLPHAAAA|687|Williams 14th|Street|Suite D|Highland|Greene County|AR|79454|United States|-6|condo| +32704|AAAAAAAAAMPHAAAA|40|Wilson |Lane|Suite R|Greenwood|Charlotte County|VA|28828|United States|-5|apartment| +32705|AAAAAAAABMPHAAAA|817|15th Third|Court|Suite X|Highland Park|Divide County|ND|56534|United States|-6|condo| +32706|AAAAAAAACMPHAAAA|592|2nd |Court|Suite K|Pine Grove|Buchanan County|VA|24593|United States|-5|condo| +32707|AAAAAAAADMPHAAAA|121|Ninth |Cir.|Suite X|Georgetown|Franklin County|ID|87057|United States|-7|apartment| +32708|AAAAAAAAEMPHAAAA|978|Johnson |Street|Suite 280|Four Points|Deer Lodge County|MT|61216|United States|-7|single family| +32709|AAAAAAAAFMPHAAAA|215|River |Way|Suite X|Hamilton|Anchorage Borough|AK|92808|United States|-9|condo| +32710|AAAAAAAAGMPHAAAA|246|Second |Dr.|Suite 170|Highland|Pulaski County|IL|69454|United States|-6|condo| +32711|AAAAAAAAHMPHAAAA|313|Franklin Lee|Cir.|Suite 190|Mount Zion|Jefferson County|OR|98054|United States|-8|condo| +32712|AAAAAAAAIMPHAAAA|625|Green 3rd|ST|Suite G|Shady Grove|Greene County|GA|32812|United States|-5|apartment| +32713|AAAAAAAAJMPHAAAA|278|First |Street|Suite E|Newport|Rusk County|TX|71521|United States|-6|single family| +32714|AAAAAAAAKMPHAAAA|251|Church |Way|Suite 460|Hopewell|Whiteside County|IL|60587|United States|-6|apartment| +32715|AAAAAAAALMPHAAAA|742|12th |ST|Suite V|Springfield|Buchanan County|VA|29303|United States|-5|single family| +32716|AAAAAAAAMMPHAAAA|141|Willow |ST|Suite 440|Waterloo|Arenac County|MI|41675|United States|-5|condo| +32717|AAAAAAAANMPHAAAA|469|5th |Ave|Suite 370|Highland Park|La Salle County|IL|66534|United States|-6|condo| +32718|AAAAAAAAOMPHAAAA|999|Mill Green|Ln|Suite 350|Jamestown|Smith County|KS|66867|United States|-6|single family| +32719|AAAAAAAAPMPHAAAA|919|Johnson |Way|Suite J|Riverview|Danville city|VA|29003|United States|-5|apartment| +32720|AAAAAAAAANPHAAAA|49|First Lake|Ave|Suite 340|Springhill|Adams County|IL|64602|United States|-6|apartment| +32721|AAAAAAAABNPHAAAA|588|Highland |Circle|Suite X|Union Hill|Waynesboro city|VA|27746|United States|-5|condo| +32722|AAAAAAAACNPHAAAA|609|Park |Avenue|Suite 190|Jamestown|Quitman County|GA|36867|United States|-5|condo| +32723|AAAAAAAADNPHAAAA|17|Park Madison|Way|Suite N|Lodi|Putnam County|GA|30382|United States|-5|condo| +32724|AAAAAAAAENPHAAAA|393|1st |Street|Suite F|Woodland|Ohio County|KY|44854|United States|-5|apartment| +32725|AAAAAAAAFNPHAAAA|31|Davis Poplar|Parkway|Suite 280|Stringtown|Miner County|SD|50162|United States|-7|single family| +32726|AAAAAAAAGNPHAAAA|530|Lee Washington|RD|Suite 10|Hopewell|Greensville County|VA|20587|United States|-5|condo| +32727|AAAAAAAAHNPHAAAA|986|Ninth Seventh|Boulevard|Suite 50|Lebanon|Mono County|CA|92898|United States|-8|apartment| +32728|AAAAAAAAINPHAAAA|458|Spring |Ct.|Suite 120|Riverview|Williamsburg County|SC|29003|United States|-5|condo| +32729|AAAAAAAAJNPHAAAA|459|3rd |Street|Suite 60|Saint George|Yolo County|CA|95281|United States|-8|apartment| +32730|AAAAAAAAKNPHAAAA|365|6th 7th|Circle|Suite O|Oak Ridge|Gila County|AZ|88371|United States|-7|apartment| +32731|AAAAAAAALNPHAAAA|872|First |Blvd|Suite D|Friendship|Lincoln County|WV|24536|United States|-5|apartment| +32732|AAAAAAAAMNPHAAAA|853|Park |Avenue|Suite B|Edgewood|Greene County|PA|10069|United States|-5|single family| +32733|AAAAAAAANNPHAAAA|618|Valley |Wy|Suite 270|Lawrenceville|Buckingham County|VA|24462|United States|-5|apartment| +32734|AAAAAAAAONPHAAAA|564|North Adams|Boulevard|Suite N|Arlington|Walker County|GA|36557|United States|-5|condo| +32735|AAAAAAAAPNPHAAAA|789|Sixth Spring|RD|Suite E|Highland|Lincoln County|SD|59454|United States|-7|apartment| +32736|AAAAAAAAAOPHAAAA|865|Highland |Road|Suite N|Deerfield|Hemphill County|TX|79840|United States|-6|apartment| +32737|AAAAAAAABOPHAAAA|963|Willow |Dr.|Suite M|Oakwood|Clay County|WV|20169|United States|-5|condo| +32738|AAAAAAAACOPHAAAA|192|12th South|Circle|Suite 40|Concord|Stewart County|GA|34107|United States|-5|condo| +32739|AAAAAAAADOPHAAAA|792|Mill Miller|Blvd|Suite H|Stringtown|Hall County|NE|60162|United States|-6|apartment| +32740|AAAAAAAAEOPHAAAA|223|1st Jefferson|ST|Suite 430|Hopewell|Tulare County|CA|90587|United States|-8|condo| +32741|AAAAAAAAFOPHAAAA|773|Elm Hickory|Circle|Suite D|Unionville|Victoria County|TX|71711|United States|-6|condo| +32742|AAAAAAAAGOPHAAAA||Lincoln Oak|ST||Oak Hill|||||-5|| +32743|AAAAAAAAHOPHAAAA|285|Sunset |Circle|Suite 220|Pleasant Hill|Henderson County|TN|33604|United States|-5|apartment| +32744|AAAAAAAAIOPHAAAA|758|10th 6th|Circle|Suite D|Franklin|Lake County|FL|39101|United States|-5|condo| +32745|AAAAAAAAJOPHAAAA|729|2nd |Blvd|Suite Q|Georgetown|Dickinson County|IA|57057|United States|-6|condo| +32746|AAAAAAAAKOPHAAAA|404|Center 1st|Pkwy|Suite P|Mount Pleasant|Russell County|KY|41933|United States|-5|apartment| +32747|AAAAAAAALOPHAAAA|893|5th Lee|Drive|Suite H|Lakeside|Sherman County|KS|69532|United States|-6|condo| +32748|AAAAAAAAMOPHAAAA|820|10th |Cir.|Suite B|Gravel Hill|Erie County|OH|41944|United States|-5|apartment| +32749|AAAAAAAANOPHAAAA|201|Cedar |Parkway|Suite 210|Clinton|McIntosh County|OK|78222|United States|-6|condo| +32750|AAAAAAAAOOPHAAAA|613|12th West|Road|Suite 80|Friendship|Franklin County|WA|94536|United States|-8|condo| +32751|AAAAAAAAPOPHAAAA|429|Mill |Pkwy|Suite 260|Oak Grove|Houston County|AL|38370|United States|-6|condo| +32752|AAAAAAAAAPPHAAAA|404|14th |RD|Suite Q|Woodville|Lincoln County|MS|54289|United States|-6|single family| +32753|AAAAAAAABPPHAAAA|206|Williams |||||PA|10534|United States|-5|| +32754|AAAAAAAACPPHAAAA|135|Ridge |Circle|Suite J|Oakland|Prince George County|MD|29843|United States|-5|apartment| +32755|AAAAAAAADPPHAAAA|440|Sunset 3rd|RD|Suite I|Fairview|Beckham County|OK|75709|United States|-6|apartment| +32756|AAAAAAAAEPPHAAAA|885|8th 11th|Drive|Suite 480|Springfield|Cass County|MO|69303|United States|-6|apartment| +32757|AAAAAAAAFPPHAAAA|740|1st |Parkway|Suite H|Georgetown|Calhoun County|SC|27057|United States|-5|apartment| +32758|AAAAAAAAGPPHAAAA|807|Meadow |Wy|Suite H|Pleasant Hill|Van Buren County|AR|73604|United States|-6|apartment| +32759|AAAAAAAAHPPHAAAA|133|Park |Circle|Suite T|Arlington|Page County|IA|56557|United States|-6|apartment| +32760|AAAAAAAAIPPHAAAA|715|Jackson |Ln|Suite 330|Unionville|Bourbon County|KS|61711|United States|-6|apartment| +32761|AAAAAAAAJPPHAAAA|846|Adams |Ave|Suite 180|Woodville|Allegany County|MD|24289|United States|-5|apartment| +32762|AAAAAAAAKPPHAAAA|508|Pine Dogwood|RD|Suite 270|San Jose|Cottonwood County|MN|58003|United States|-6|single family| +32763|AAAAAAAALPPHAAAA|202|11th |Ct.|Suite S|Gilmore|Jones County|IA|55464|United States|-6|condo| +32764|AAAAAAAAMPPHAAAA|26|Sycamore |Drive|Suite X|Belmont|Aransas County|TX|70191|United States|-6|condo| +32765|AAAAAAAANPPHAAAA|915|Madison Maple|Ln|Suite 230|Springfield|Lake County|OR|99303|United States|-8|condo| +32766|AAAAAAAAOPPHAAAA|897|Poplar |Circle|Suite U|Lincoln|Lee County|TX|71289|United States|-6|single family| +32767|AAAAAAAAPPPHAAAA|343|Sixth |Avenue|Suite 300|Bridgeport|Duval County|TX|75817|United States|-6|condo| +32768|AAAAAAAAAAAIAAAA|44|2nd Pine|Ct.|Suite Y|Deerfield|Simpson County|MS|59840|United States|-6|apartment| +32769|AAAAAAAABAAIAAAA|583|Highland West|Court|Suite D|Edgewood|Blanco County|TX|70069|United States|-6|apartment| +32770|AAAAAAAACAAIAAAA|651|Park Forest|||Deerfield||VA|29840|United States||condo| +32771|AAAAAAAADAAIAAAA|310|Willow Fourth|Road|Suite 50|Maple Grove|West Feliciana Parish|LA|78252|United States|-6|apartment| +32772|AAAAAAAAEAAIAAAA|588|Oak North|Way|Suite 320|Oak Ridge|Henry County|GA|38371|United States|-5|condo| +32773|AAAAAAAAFAAIAAAA|318|7th Maple|Wy|Suite 120|Sulphur Springs|Zapata County|TX|78354|United States|-6|apartment| +32774|AAAAAAAAGAAIAAAA|627|West |Ln|Suite 20|Pine Grove|Belknap County|NH|05193|United States|-5|apartment| +32775|AAAAAAAAHAAIAAAA|228|Forest Sycamore|||Bethel|Milam County|TX||||apartment| +32776|AAAAAAAAIAAIAAAA|564|Madison |Circle|Suite O|Sunnyside|Grafton County|NH|02552|United States|-5|single family| +32777|AAAAAAAAJAAIAAAA|726|Poplar |Street|Suite C|Maple Grove|Knox County|KY|48252|United States|-5|single family| +32778|AAAAAAAAKAAIAAAA|954|Johnson 3rd|Court|Suite N|Lakeview|Loving County|TX|78579|United States|-6|single family| +32779|AAAAAAAALAAIAAAA||Pine 6th|Ave||Greenwood||MI|48828||-5|condo| +32780|AAAAAAAAMAAIAAAA|65||Ave|Suite 70||Cumberland County||28222|||single family| +32781|AAAAAAAANAAIAAAA|380|7th Main|Dr.|Suite U|Oak Hill|Salt Lake County|UT|87838|United States|-7|condo| +32782|AAAAAAAAOAAIAAAA|||||Friendship|Laclede County|MO|64536|||| +32783|AAAAAAAAPAAIAAAA|838|Park |Cir.|Suite V|Stringtown|Madera County|CA|90162|United States|-8|condo| +32784|AAAAAAAAABAIAAAA|760|6th Jackson|Pkwy|Suite V|Sand Hill|Lee County|VA|29530|United States|-5|apartment| +32785|AAAAAAAABBAIAAAA|409|Elm Locust|Lane|Suite 130|Waterloo|Jackson County|AL|31675|United States|-6|single family| +32786|AAAAAAAACBAIAAAA|678|Woodland Cedar|Way|Suite 190|Arlington|Barnes County|ND|56557|United States|-6|condo| +32787|AAAAAAAADBAIAAAA|153|5th Main|Boulevard|Suite Q|Riverside|Iron County|UT|89231|United States|-7|apartment| +32788|AAAAAAAAEBAIAAAA|345|Green Lakeview|Boulevard|Suite 120|Macedonia|Walla Walla County|WA|91087|United States|-8|apartment| +32789|AAAAAAAAFBAIAAAA|947|1st |Circle|Suite E|Liberty|Carroll County|IN|43451|United States|-5|condo| +32790|AAAAAAAAGBAIAAAA|691|3rd South|Court|Suite V|Green Acres|Cherokee County|AL|37683|United States|-6|apartment| +32791|AAAAAAAAHBAIAAAA|726|Franklin 13th|ST|Suite R|Sulphur Springs|Sutter County|CA|98354|United States|-8|apartment| +32792|AAAAAAAAIBAIAAAA|303|Meadow |Circle|Suite 370|Newtown|Tehama County|CA|91749|United States|-8|condo| +32793|AAAAAAAAJBAIAAAA|169|Miller 7th|ST|Suite P|Arlington|Randolph County|AL|36557|United States|-6|apartment| +32794|AAAAAAAAKBAIAAAA|855|East Franklin|Wy|Suite I|Kirkland|Madison County|AR|77896|United States|-6|single family| +32795|AAAAAAAALBAIAAAA|316|Meadow South|Cir.|Suite K|Unionville|Foard County|TX|71711|United States|-6|apartment| +32796|AAAAAAAAMBAIAAAA|860|8th Park|Boulevard|Suite E|Sulphur Springs|Haywood County|TN|38354|United States|-5|single family| +32797|AAAAAAAANBAIAAAA|160|Valley Oak|RD|Suite X|Springfield|Nevada County|CA|99303|United States|-8|single family| +32798|AAAAAAAAOBAIAAAA|348|Elm Franklin|Way|Suite 30|Macedonia|Ocean County|NJ|01687|United States|-5|single family| +32799|AAAAAAAAPBAIAAAA|820|West |Ln|Suite 10|Lakewood|Kankakee County|IL|68877|United States|-6|single family| +32800|AAAAAAAAACAIAAAA|665|Smith |Road|Suite 110|Woodland|Jefferson County|MO|64854|United States|-6|single family| +32801|AAAAAAAABCAIAAAA|824|Sunset |Dr.|Suite W|Oakland|Washakie County|WY|89843|United States|-7|single family| +32802|AAAAAAAACCAIAAAA|154|Second |Blvd|Suite N|Marion|Washington County|AL|30399|United States|-6|apartment| +32803|AAAAAAAADCAIAAAA|21|7th |Circle|Suite L|Enterprise|Atascosa County|TX|71757|United States|-6|apartment| +32804|AAAAAAAAECAIAAAA|53|Ridge South|Parkway|Suite S|Granite|Douglas County|IL|66284|United States|-6|single family| +32805|AAAAAAAAFCAIAAAA|246|Center Elm|Court|Suite T|New Hope|Miller County|MO|69431|United States|-6|single family| +32806|AAAAAAAAGCAIAAAA|183|Lakeview |Way|Suite 260|Argyle|Shawano County|WI|58722|United States|-6|condo| +32807|AAAAAAAAHCAIAAAA|218|West Davis|Lane|Suite 280|Avon|Pacific County|WA|90069|United States|-8|single family| +32808|AAAAAAAAICAIAAAA|868|Pine |Dr.|Suite E|Fairview|Claiborne County|MS|55709|United States|-6|condo| +32809|AAAAAAAAJCAIAAAA|483|Sycamore |ST|Suite U|Paxton|Wayne County|NE|65669|United States|-7|condo| +32810|AAAAAAAAKCAIAAAA|788|Smith Miller|Road|Suite D|Greenfield|Dubois County|IN|45038|United States|-5|single family| +32811|AAAAAAAALCAIAAAA|970|7th Park|Dr.|Suite S|Parkwood|Livingston County|MO|61669|United States|-6|condo| +32812|AAAAAAAAMCAIAAAA|265|Park Center|Parkway|Suite 410|Beechwood|Hot Springs County|WY|85402|United States|-7|apartment| +32813|AAAAAAAANCAIAAAA|285||||Johnsonville|Warren County|||United States|-5|| +32814|AAAAAAAAOCAIAAAA|771|Pine Cedar|Ct.|Suite I|Clearview|Highland County|VA|25495|United States|-5|condo| +32815|AAAAAAAAPCAIAAAA|982|Spring |Dr.|Suite 280|Harmony|Coryell County|TX|75804|United States|-6|condo| +32816|AAAAAAAAADAIAAAA|486|Forest 4th|Wy|Suite S|Franklin|Northwest Arctic Borough|AK|99101|United States|-9|condo| +32817|AAAAAAAABDAIAAAA|651|Washington |Ave|Suite V|Lebanon|Jennings County|IN|42898|United States|-5|single family| +32818|AAAAAAAACDAIAAAA|379|West 9th|Circle|Suite C|Sunnyside|Westchester County|NY|11952|United States|-5|apartment| +32819|AAAAAAAADDAIAAAA|903|13th 1st|Road|Suite G|Oakdale|Howard County|NE|69584|United States|-7|apartment| +32820|AAAAAAAAEDAIAAAA|419|Chestnut |RD|Suite N|Mount Vernon|Navajo County|AZ|88482|United States|-7|apartment| +32821|AAAAAAAAFDAIAAAA|863|River |Ave|Suite 460|Greenfield|Poinsett County|AR|75038|United States|-6|single family| +32822|AAAAAAAAGDAIAAAA|452|Elm |Way|Suite 120|Providence|Jackson County|GA|36614|United States|-5|single family| +32823|AAAAAAAAHDAIAAAA|487|Dogwood |Avenue|Suite J|Harmony|Lamar County|GA|35804|United States|-5|single family| +32824|AAAAAAAAIDAIAAAA|872|Sixth 5th|Avenue|Suite 180|Kingston|Caribou County|ID|84975|United States|-7|apartment| +32825|AAAAAAAAJDAIAAAA|716|Seventh |Ave|Suite X|Glenwood|Muskogee County|OK|73511|United States|-6|single family| +32826|AAAAAAAAKDAIAAAA|864|10th |Blvd|Suite W|Glenwood|Klamath County|OR|93511|United States|-8|apartment| +32827|AAAAAAAALDAIAAAA|966|2nd Lincoln|ST|Suite 310|Concord|Jasper County|MO|64107|United States|-6|single family| +32828|AAAAAAAAMDAIAAAA|||Wy|Suite 410|||VA||United States|-5|| +32829|AAAAAAAANDAIAAAA||First Locust|Dr.|||Boone County|||United States|-6|single family| +32830|AAAAAAAAODAIAAAA|943|Mill Hillcrest|Boulevard|Suite D|Highland|Benzie County|MI|49454|United States|-5|apartment| +32831|AAAAAAAAPDAIAAAA|657|10th |Circle|Suite 130|Providence|Johnston County|OK|76614|United States|-6|condo| +32832|AAAAAAAAAEAIAAAA||Center Meadow||Suite I||Miami County|OH|||-5|single family| +32833|AAAAAAAABEAIAAAA|47|Pine |Boulevard|Suite 330|Lakeside|Oktibbeha County|MS|59532|United States|-6|single family| +32834|AAAAAAAACEAIAAAA|938|3rd |Boulevard|Suite 280|Lincoln|Orangeburg County|SC|21289|United States|-5|apartment| +32835|AAAAAAAADEAIAAAA|60|14th |Road|Suite 140|Unionville|Wayne County|GA|31711|United States|-5|single family| +32836|AAAAAAAAEEAIAAAA|557|1st |RD|Suite V|Crossroads|Telfair County|GA|30534|United States|-5|condo| +32837|AAAAAAAAFEAIAAAA|316|1st 5th|Street|Suite G|Summit|Beauregard Parish|LA|70499|United States|-6|condo| +32838|AAAAAAAAGEAIAAAA|954|Fifth Ridge|Parkway|Suite A|Pleasant Hill|Dakota County|NE|63604|United States|-6|condo| +32839|AAAAAAAAHEAIAAAA|323|Spring ||Suite 210|||CA|99003||-8|condo| +32840|AAAAAAAAIEAIAAAA|681|Central |Road|Suite 210|Centerville|Marion County|MS|50059|United States|-6|single family| +32841|AAAAAAAAJEAIAAAA|910|Lee Park|Avenue|Suite 70|Jerome|Dent County|MO|69920|United States|-6|single family| +32842|AAAAAAAAKEAIAAAA|849|10th |Wy|Suite H|Pleasant Hill|Randolph County|WV|23604|United States|-5|single family| +32843|AAAAAAAALEAIAAAA|450|Miller |Ct.|Suite E|Walnut Grove|Sharp County|AR|77752|United States|-6|apartment| +32844|AAAAAAAAMEAIAAAA|500|11th 5th|Wy|Suite 200|Pine Grove|Orange County|NC|24593|United States|-5|single family| +32845|AAAAAAAANEAIAAAA|409|Central 6th|Street|Suite 240|Maple Grove|Franklin County|MA|08852|United States|-5|single family| +32846|AAAAAAAAOEAIAAAA|18|13th |Lane|Suite 380|Greenville|Clay County|SD|51387|United States|-6|single family| +32847|AAAAAAAAPEAIAAAA|714|Hill 9th|Boulevard|Suite 350|Union|Washington County|ID|88721|United States|-7|condo| +32848|AAAAAAAAAFAIAAAA|641|Adams First|ST|Suite 350|Kingston|Franklin Parish|LA|74975|United States|-6|single family| +32849|AAAAAAAABFAIAAAA|318|Forest |Way|Suite W|Mount Pleasant|Merced County|CA|91933|United States|-8|condo| +32850|AAAAAAAACFAIAAAA|265|Church |Blvd|Suite A|Wilson|Jackson County|AL|36971|United States|-6|apartment| +32851|AAAAAAAADFAIAAAA|71|Wilson |Boulevard|Suite B|White Oak|Madison County|KY|46668|United States|-5|condo| +32852|AAAAAAAAEFAIAAAA|553|Elm Spruce|Way|Suite E|Greenfield|Calhoun County|WV|25038|United States|-5|apartment| +32853|AAAAAAAAFFAIAAAA|516|Main |Lane|Suite 410|Star|Burlington County|NJ|01325|United States|-5|single family| +32854|AAAAAAAAGFAIAAAA|426|Park 3rd|Boulevard|Suite 60|Green Acres|Whitman County|WA|97683|United States|-8|condo| +32855|AAAAAAAAHFAIAAAA|335|Sycamore Third|Wy|Suite T|Antioch|Brunswick County|VA|28605|United States|-5|apartment| +32856|AAAAAAAAIFAIAAAA|669|1st Davis|Blvd|Suite U|Florence|Assumption Parish|LA|73394|United States|-6|apartment| +32857|AAAAAAAAJFAIAAAA|906|10th 10th|Boulevard|Suite V|Buena Vista|Ogemaw County|MI|45752|United States|-5|condo| +32858|AAAAAAAAKFAIAAAA|771|2nd North|Dr.|Suite A|Pine Grove|Newberry County|SC|24593|United States|-5|condo| +32859|AAAAAAAALFAIAAAA|572|9th |Avenue|Suite N|Valley View|Pickett County|TN|35124|United States|-6|single family| +32860|AAAAAAAAMFAIAAAA|619|Hillcrest Walnut|Circle|Suite 190|Brownsville|Richland County|SC|29310|United States|-5|condo| +32861|AAAAAAAANFAIAAAA|318|Willow |Pkwy|Suite 170|Hardy|Jefferson Davis Parish|LA|75354|United States|-6|condo| +32862|AAAAAAAAOFAIAAAA|863|Meadow |Pkwy|Suite X|Forest Hills|Escambia County|AL|39237|United States|-6|apartment| +32863|AAAAAAAAPFAIAAAA|575|Ridge 6th|ST|Suite S|Oak Hill|Yoakum County|TX|77838|United States|-6|condo| +32864|AAAAAAAAAGAIAAAA|464|Sixth Broadway|ST|Suite B|Vance|Mercer County|MO|60268|United States|-6|condo| +32865|AAAAAAAABGAIAAAA||Woodland ||Suite 370||Alexander County|NC|28371||-5|single family| +32866|AAAAAAAACGAIAAAA|217|8th |Street|Suite K|Antioch|Pendleton County|KY|48605|United States|-5|condo| +32867|AAAAAAAADGAIAAAA|526|View Cedar|Pkwy|Suite Y|Oakland|Boundary County|ID|89843|United States|-7|apartment| +32868|AAAAAAAAEGAIAAAA|596|Cedar |Pkwy|Suite 240|Shady Grove|Park County|CO|82812|United States|-7|condo| +32869|AAAAAAAAFGAIAAAA|563|Tenth 1st|Road|Suite Y|Jackson|Nassau County|FL|39583|United States|-5|single family| +32870|AAAAAAAAGGAIAAAA|406|Oak View|Road|Suite 490|Greenwood|DeSoto County|MS|58828|United States|-6|apartment| +32871|AAAAAAAAHGAIAAAA|960|Lake |Lane|Suite 260|Greenville|Livingston County|MI|41387|United States|-5|apartment| +32872|AAAAAAAAIGAIAAAA|60|View |Cir.|Suite S|Kingston|Eaton County|MI|44975|United States|-5|apartment| +32873|AAAAAAAAJGAIAAAA|574|7th Elm|Ln|Suite 410|Greenfield|Delaware County|OH|45038|United States|-5|apartment| +32874|AAAAAAAAKGAIAAAA|972|Ridge Adams|Circle|Suite 330|Riverview|Adams County|MS|59003|United States|-6|apartment| +32875|AAAAAAAALGAIAAAA|430|Church Adams|Pkwy|Suite 40|Glenwood|Pottawattamie County|IA|53511|United States|-6|single family| +32876|AAAAAAAAMGAIAAAA|791|7th Third|Ln|Suite E|Crossroads|Chester County|SC|20534|United States|-5|condo| +32877|AAAAAAAANGAIAAAA|434|Highland Jefferson|Ln|Suite 320|Franklin|Whiteside County|IL|69101|United States|-6|single family| +32878|AAAAAAAAOGAIAAAA|659|Park |Drive|Suite X|Florence|Morrison County|MN|53394|United States|-6|single family| +32879|AAAAAAAAPGAIAAAA|||||Siloam||TX|78948||-6|single family| +32880|AAAAAAAAAHAIAAAA|581|Forest |Parkway|Suite 280|Belmont|Marion County|TN|30191|United States|-6|apartment| +32881|AAAAAAAABHAIAAAA|341|Pine |Lane|Suite 160|Farmington|DeKalb County|GA|39145|United States|-5|condo| +32882|AAAAAAAACHAIAAAA|734|Pine Railroad|Drive|Suite B|Forest Hills|Alexander County|IL|69237|United States|-6|single family| +32883|AAAAAAAADHAIAAAA|779|2nd 2nd|Avenue|Suite 430|Rocky Point|Wilson County|TN|31209|United States|-5|single family| +32884|AAAAAAAAEHAIAAAA|165|West |Drive|Suite 480|Maple Grove|Lee County|IA|58252|United States|-6|apartment| +32885|AAAAAAAAFHAIAAAA|175|Franklin |Circle|Suite K|Wildwood|Sonoma County|CA|96871|United States|-8|single family| +32886|AAAAAAAAGHAIAAAA|635|Main |ST|Suite 440|Woodland|Culpeper County|VA|24854|United States|-5|single family| +32887|AAAAAAAAHHAIAAAA|144|Hill |Street|Suite G|Farmington|Panola County|MS|59145|United States|-6|condo| +32888|AAAAAAAAIHAIAAAA|573|Elm |Avenue|Suite 80|Salem|Otero County|CO|88048|United States|-7|condo| +32889|AAAAAAAAJHAIAAAA|738|Washington |Road|Suite 340|Marion|Catoosa County|GA|30399|United States|-5|single family| +32890|AAAAAAAAKHAIAAAA|||Court|||Crockett County|TN|30411|United States|-5|apartment| +32891|AAAAAAAALHAIAAAA||Spring Sunset||Suite W|Bridgeport||GA|35817|United States|-5|apartment| +32892|AAAAAAAAMHAIAAAA|851|Main Green|||||NV|||-8|condo| +32893|AAAAAAAANHAIAAAA|193|13th |Circle|Suite W|Bunker Hill|Jackson County|IN|40150|United States|-5|condo| +32894|AAAAAAAAOHAIAAAA|307|Lake Fifteenth|Court|Suite 490|Blanchard|Clay County|KY|45985|United States|-6|single family| +32895|AAAAAAAAPHAIAAAA|872|Main |Drive|Suite 110|Ashland|Aleutians West Census Area|AK|94244|United States|-9|condo| +32896|AAAAAAAAAIAIAAAA|453|Fourth Jackson|Way|Suite 430|Riverview|Schuyler County|IL|69003|United States|-6|single family| +32897|AAAAAAAABIAIAAAA||6th Elm|||Buena Vista|Windham County||06352|||| +32898|AAAAAAAACIAIAAAA|638|View Washington|Dr.|Suite 80|Maple Grove|Harper County|OK|78252|United States|-6|apartment| +32899|AAAAAAAADIAIAAAA|140|Sixth Washington|Street|Suite L|Walnut Grove|Fayette County|IN|47752|United States|-5|condo| +32900|AAAAAAAAEIAIAAAA|393|Laurel |Ct.|Suite 370|Mount Zion|Hill County|MT|68054|United States|-7|single family| +32901|AAAAAAAAFIAIAAAA|828|Walnut |Ct.|Suite 310|Mount Pleasant|Madison County|NY|11933|United States|-5|condo| +32902|AAAAAAAAGIAIAAAA|408|2nd Jackson|Ln|Suite B|Mount Pleasant|Little River County|AR|71933|United States|-6|condo| +32903|AAAAAAAAHIAIAAAA|761|Johnson |Way|Suite P|Liberty|Imperial County|CA|93451|United States|-8|apartment| +32904|AAAAAAAAIIAIAAAA|406|Johnson Washington|Avenue|Suite R|Centerville|Searcy County|AR|70059|United States|-6|condo| +32905|AAAAAAAAJIAIAAAA|781|First Second|RD|Suite U|Newtown|Somerset County|NJ|02349|United States|-5|condo| +32906|AAAAAAAAKIAIAAAA|590|Laurel 2nd|Road|Suite 80|Stringtown|Waukesha County|WI|50162|United States|-6|apartment| +32907|AAAAAAAALIAIAAAA|768|6th Center|Way|Suite U|Lakeside|Portage County|OH|49532|United States|-5|condo| +32908|AAAAAAAAMIAIAAAA|554|Park |Lane|Suite W|Mountain View|Randolph County|IL|64466|United States|-6|condo| +32909|AAAAAAAANIAIAAAA|492|Maple 3rd|Way|Suite 420|Hidden Valley|Logan County|WV|25521|United States|-5|single family| +32910|AAAAAAAAOIAIAAAA|805|Church Oak|Ct.|Suite 160|Shady Grove|Stoddard County|MO|62812|United States|-6|condo| +32911|AAAAAAAAPIAIAAAA|593|Spruce |Cir.|Suite R|Buena Vista|Dakota County|NE|65752|United States|-6|condo| +32912|AAAAAAAAAJAIAAAA|798|Williams |Wy|Suite L|Oakwood|Dodge County|MN|50169|United States|-6|single family| +32913|AAAAAAAABJAIAAAA|80|2nd North|Way|Suite 380|Edgewood|Power County|ID|80069|United States|-7|apartment| +32914|AAAAAAAACJAIAAAA|672|5th Spring|Parkway|Suite M|Glenwood|Coffee County|GA|33511|United States|-5|condo| +32915|AAAAAAAADJAIAAAA|544|East |Blvd|Suite U|Oak Ridge|Union County|SD|58371|United States|-7|condo| +32916|AAAAAAAAEJAIAAAA|429|1st West|Road|Suite 280|Jamestown|Grant County|MN|56867|United States|-6|single family| +32917|AAAAAAAAFJAIAAAA|143|Valley |Dr.|Suite D|Macedonia|Dearborn County|IN|41087|United States|-5|apartment| +32918|AAAAAAAAGJAIAAAA|871|River 3rd|Ave|Suite 80|Union|Jefferson County|CO|88721|United States|-7|condo| +32919|AAAAAAAAHJAIAAAA|544|Miller |Drive|Suite I|Springdale|Quitman County|GA|38883|United States|-5|single family| +32920|AAAAAAAAIJAIAAAA|379|Park |Ln|Suite A|Richardson|Adams County|ID|87687|United States|-7|single family| +32921|AAAAAAAAJJAIAAAA|24|Willow Spring|Street|Suite J|Cedar Grove|Goochland County|VA|20411|United States|-5|single family| +32922|AAAAAAAAKJAIAAAA|282|Park Adams|Court|Suite 310|Forest Hills|Sherman County|TX|79237|United States|-6|single family| +32923|AAAAAAAALJAIAAAA|775|Railroad |Circle|Suite 60|Five Points|Boyle County|KY|46098|United States|-6|condo| +32924|AAAAAAAAMJAIAAAA|833|First |Ave|Suite 10|Hamilton|Kenedy County|TX|72808|United States|-6|condo| +32925|AAAAAAAANJAIAAAA|796|Cherry Main|Cir.|Suite H|Greenville|Harding County|SD|51387|United States|-7|condo| +32926|AAAAAAAAOJAIAAAA|723|Lake |Dr.|Suite 270|Providence|Scott County|IN|46614|United States|-5|single family| +32927|AAAAAAAAPJAIAAAA|231|Walnut Chestnut|Ln|Suite 30|Washington Heights|Montgomery County|NY|18167|United States|-5|single family| +32928|AAAAAAAAAKAIAAAA|630|Pine |Boulevard|Suite 90|Harmon|Owen County|IN|45623|United States|-5|apartment| +32929|AAAAAAAABKAIAAAA|82|Third Smith|Boulevard|Suite H|Caledonia|Minidoka County|ID|87411|United States|-7|apartment| +32930|AAAAAAAACKAIAAAA|319|Miller Jefferson|Wy|Suite 230|Midway|Carroll County|IA|51904|United States|-6|condo| +32931|AAAAAAAADKAIAAAA|806|4th |Drive|Suite 340|Riverside|Erie County|NY|19231|United States|-5|apartment| +32932|AAAAAAAAEKAIAAAA|20|Pine |Blvd|Suite 250|Franklin|Fresno County|CA|99101|United States|-8|apartment| +32933|AAAAAAAAFKAIAAAA|663|Forest |Blvd|Suite L|Farmington|Willacy County|TX|79145|United States|-6|condo| +32934|AAAAAAAAGKAIAAAA|508|Ash |Ave|Suite 120|Highland Park|Morgan County|TN|36534|United States|-6|apartment| +32935|AAAAAAAAHKAIAAAA|866|Ridge 13th|Street|Suite G|Five Forks|Giles County|VA|22293|United States|-5|single family| +32936|AAAAAAAAIKAIAAAA|376|Ninth Fifth|Wy|Suite W|Hopewell|Wheeler County|OR|90587|United States|-8|single family| +32937|AAAAAAAAJKAIAAAA|182|13th |Wy|Suite 200|Clinton|Virginia Beach city|VA|28222|United States|-5|condo| +32938|AAAAAAAAKKAIAAAA|628|Second |Circle|Suite L|Macedonia|Camp County|TX|71087|United States|-6|condo| +32939|AAAAAAAALKAIAAAA|119|Davis |Ln|Suite I|Mount Zion|Klickitat County|WA|98054|United States|-8|condo| +32940|AAAAAAAAMKAIAAAA|736|Williams |Avenue|Suite 320|Bridgeport|Wayne County|GA|35817|United States|-5|apartment| +32941|AAAAAAAANKAIAAAA|613|View 6th|Ct.|Suite B|Five Forks|Edgar County|IL|62293|United States|-6|apartment| +32942|AAAAAAAAOKAIAAAA|397|Main 4th|Ct.|Suite 250|Oakland|Osceola County|IA|59843|United States|-6|apartment| +32943|AAAAAAAAPKAIAAAA|||RD|||Oceana County||48811|United States|-5|| +32944|AAAAAAAAALAIAAAA|111|3rd Sixth|Street|Suite 360|Red Hill|Warren County|GA|34338|United States|-5|single family| +32945|AAAAAAAABLAIAAAA|881|Jackson |Circle|Suite 260|Summit|Culberson County|TX|70499|United States|-6|apartment| +32946|AAAAAAAACLAIAAAA|325|Chestnut Main|Ln|Suite I|Spring Hill|Leflore County|MS|56787|United States|-6|apartment| +32947|AAAAAAAADLAIAAAA|710|Wilson 12th|Drive|Suite T|Franklin|Charlton County|GA|39101|United States|-5|single family| +32948|AAAAAAAAELAIAAAA|894|Oak |Boulevard|Suite 440|Bunker Hill|Pike County|IN|40150|United States|-5|condo| +32949|AAAAAAAAFLAIAAAA|823|Lake |Blvd|Suite 110|Summit|Richmond County|NC|20499|United States|-5|single family| +32950|AAAAAAAAGLAIAAAA|137|Cherry Thirteenth|Pkwy|Suite V|Jackson|Lincoln County|NC|29583|United States|-5|single family| +32951|AAAAAAAAHLAIAAAA|866|Hill Oak|Dr.|Suite R|Woodland|Tioga County|PA|14854|United States|-5|single family| +32952|AAAAAAAAILAIAAAA|979|Lincoln |Lane|Suite S|Buena Vista|Pondera County|MT|65752|United States|-7|condo| +32953|AAAAAAAAJLAIAAAA|25|Fifth 14th|Cir.|Suite 340|Lakewood|Norton city|VA|28877|United States|-5|apartment| +32954|AAAAAAAAKLAIAAAA|663|11th |Way|Suite C|Flatwoods|Crittenden County|KY|44212|United States|-6|single family| +32955|AAAAAAAALLAIAAAA|278|Mill College|Way|Suite B|Winslow|Meade County|KS|68525|United States|-6|condo| +32956|AAAAAAAAMLAIAAAA|626|Franklin Wilson|Ave|Suite 130|Green Acres|Polk County|MN|57683|United States|-6|single family| +32957|AAAAAAAANLAIAAAA|102|13th |Circle|Suite D|Verona|Boone County|IL|63629|United States|-6|apartment| +32958|AAAAAAAAOLAIAAAA|956|5th 3rd|ST|Suite E|Greenville|Polk County|TX|71387|United States|-6|single family| +32959|AAAAAAAAPLAIAAAA|14|Sixth Mill|Court|Suite 380|Plainview|Tyrrell County|NC|23683|United States|-5|condo| +32960|AAAAAAAAAMAIAAAA|826|West Smith|Cir.|Suite X|Pinhook|Allen County|KY|49398|United States|-6|single family| +32961|AAAAAAAABMAIAAAA|390|Cedar |Drive|Suite 230|Liberty|Newton County|GA|33451|United States|-5|condo| +32962|AAAAAAAACMAIAAAA|697|Forest |Ln|Suite U|Dale|Brazoria County|TX|70514|United States|-6|condo| +32963|AAAAAAAADMAIAAAA|543|Elm |Ct.|Suite 460|Pine Valley|Delaware County|NY|18209|United States|-5|single family| +32964|AAAAAAAAEMAIAAAA|647|Adams Madison|Ln|Suite 300|Lakewood|Cowlitz County|WA|98877|United States|-8|apartment| +32965|AAAAAAAAFMAIAAAA|801|Jefferson |Parkway|Suite T|Georgetown|Washita County|OK|77057|United States|-6|single family| +32966|AAAAAAAAGMAIAAAA|911|Spring |Cir.|Suite Q|Five Forks|Monroe County|IN|42293|United States|-5|apartment| +32967|AAAAAAAAHMAIAAAA|500|Fifth |Way|Suite E|Greenville|Navajo County|AZ|81387|United States|-7|apartment| +32968|AAAAAAAAIMAIAAAA|746|Hillcrest 5th|Wy|Suite 60|Walnut Grove|Columbia County|WA|97752|United States|-8|single family| +32969|AAAAAAAAJMAIAAAA|486|First |Ln|Suite 410|Pleasant Grove|Lincoln County|NM|84136|United States|-7|single family| +32970|AAAAAAAAKMAIAAAA|464|Lakeview Forest|Ct.|Suite 320|Hamilton|Portage County|OH|42808|United States|-5|single family| +32971|AAAAAAAALMAIAAAA|726|Woodland Spring|Circle|Suite K|Vienna|Custer County|MT|65119|United States|-7|condo| +32972|AAAAAAAAMMAIAAAA|522|Church |Wy|Suite I|Waterloo|Cleveland County|OK|71675|United States|-6|condo| +32973|AAAAAAAANMAIAAAA||Cedar |Way|Suite 140||Marquette County|WI||||| +32974|AAAAAAAAOMAIAAAA|287|Third |Street|Suite 400|Providence|Codington County|SD|56614|United States|-6|single family| +32975|AAAAAAAAPMAIAAAA|132|10th |Drive|Suite T|Buena Vista|Norton County|KS|65752|United States|-6|apartment| +32976|AAAAAAAAANAIAAAA|886|Smith Railroad|Street|Suite 90|Antioch|Deer Lodge County|MT|68605|United States|-7|single family| +32977|AAAAAAAABNAIAAAA|323|1st |Blvd|Suite C|Oak Grove|Hickman County|KY|48370|United States|-6|single family| +32978|AAAAAAAACNAIAAAA|722|First Maple|Cir.|Suite J|Crossroads|Monongalia County|WV|20534|United States|-5|single family| +32979|AAAAAAAADNAIAAAA|993|3rd Meadow|RD|Suite 220|Union|Herkimer County|NY|18721|United States|-5|apartment| +32980|AAAAAAAAENAIAAAA|916|View |Ave|Suite P|Wyoming|Guadalupe County|TX|70216|United States|-6|condo| +32981|AAAAAAAAFNAIAAAA|369|Wilson |Ave|Suite F|Sunnyside|Coles County|IL|61952|United States|-6|condo| +32982|AAAAAAAAGNAIAAAA|806|Ninth |Road|Suite 240|Red Hill|Powell County|KY|44338|United States|-5|apartment| +32983|AAAAAAAAHNAIAAAA|194|Broadway Valley|Boulevard|Suite 360|Riverdale|Todd County|SD|59391|United States|-7|single family| +32984|AAAAAAAAINAIAAAA|552|8th Church|Parkway|Suite 390|Newtown|Harlan County|KY|41749|United States|-6|single family| +32985|AAAAAAAAJNAIAAAA|817|5th Locust|Blvd|Suite 190|Hopewell|Mills County|TX|70587|United States|-6|single family| +32986|AAAAAAAAKNAIAAAA|131|Oak |Pkwy|Suite O|Harmony|Tangipahoa Parish|LA|75804|United States|-6|single family| +32987|AAAAAAAALNAIAAAA|904|5th Washington|Way|Suite N|Kingston|Mecklenburg County|VA|24975|United States|-5|single family| +32988|AAAAAAAAMNAIAAAA|867|Main Davis|Dr.|Suite B|Crossroads|Siskiyou County|CA|90534|United States|-8|apartment| +32989|AAAAAAAANNAIAAAA|225|Valley Central|Boulevard|Suite O|Spring Valley|Oneida County|ID|86060|United States|-7|apartment| +32990|AAAAAAAAONAIAAAA|96|Willow |Avenue|Suite R|Roy|Anderson County|KY|40744|United States|-6|apartment| +32991|AAAAAAAAPNAIAAAA|568|River |Circle|Suite 150|Forest Hills|Wyoming County|NY|19237|United States|-5|condo| +32992|AAAAAAAAAOAIAAAA|290|Willow |Way|Suite A|Woodland|Boone County|MO|64854|United States|-6|single family| +32993|AAAAAAAABOAIAAAA|22|Maple |Ave|Suite B|Mount Zion|Northampton County|PA|18054|United States|-5|condo| +32994|AAAAAAAACOAIAAAA|90|West 3rd|Drive|Suite 90|Concord|Merrick County|NE|64107|United States|-7|apartment| +32995|AAAAAAAADOAIAAAA|337|Washington |Blvd|Suite 490|Wildwood|Jackson County|MI|46871|United States|-5|single family| +32996|AAAAAAAAEOAIAAAA|312|Pine |Ave|Suite M|Wilson|Fisher County|TX|76971|United States|-6|condo| +32997|AAAAAAAAFOAIAAAA|494|Forest |Lane|Suite E|Chestnut Ridge|Harrison County|WV|27334|United States|-5|single family| +32998|AAAAAAAAGOAIAAAA|30|Fourteenth Willow|ST|Suite P|Crossroads|Grundy County|TN|30534|United States|-5|condo| +32999|AAAAAAAAHOAIAAAA|154|Oak Spruce|Court|Suite C|Enterprise|Zavala County|TX|71757|United States|-6|single family| +33000|AAAAAAAAIOAIAAAA|945|Hickory Johnson|Wy|Suite 210|Bridgeport|Riley County|KS|65817|United States|-6|condo| +33001|AAAAAAAAJOAIAAAA|611|East |Drive|Suite W|Woodland Park|Jefferson County|WA|91934|United States|-8|single family| +33002|AAAAAAAAKOAIAAAA|190|Adams Seventh|Dr.|Suite H|Lakeview|Jim Hogg County|TX|78579|United States|-6|apartment| +33003|AAAAAAAALOAIAAAA|157|Lee 11th|Wy|Suite F|Oakdale|Stillwater County|MT|69584|United States|-7|single family| +33004|AAAAAAAAMOAIAAAA|604|Jefferson Sycamore|Ct.|Suite K|Saratoga|Keokuk County|IA|52123|United States|-6|single family| +33005|AAAAAAAANOAIAAAA|52|Pine Chestnut|Road|Suite 130|Harmony|Ripley County|IN|45804|United States|-5|apartment| +33006|AAAAAAAAOOAIAAAA|785|Sycamore Cherry|Way|Suite 320|Union Hill|Eureka County|NV|87746|United States|-8|apartment| +33007|AAAAAAAAPOAIAAAA|847|Jefferson 4th|Cir.|Suite 80|Salem|Northumberland County|PA|18048|United States|-5|single family| +33008|AAAAAAAAAPAIAAAA|947|Cherry 10th|Dr.|Suite Q|Woodland|Deer Lodge County|MT|64854|United States|-7|condo| +33009|AAAAAAAABPAIAAAA|216|Spruce 5th|Court|Suite 170|Walnut Grove|Hardee County|FL|37752|United States|-5|condo| +33010|AAAAAAAACPAIAAAA|955|2nd |Parkway|Suite 250|Wildwood|Jefferson Parish|LA|76871|United States|-6|single family| +33011|AAAAAAAADPAIAAAA|330|Green Mill|Street|Suite T|Clinton|Washington County|OR|98222|United States|-8|condo| +33012|AAAAAAAAEPAIAAAA|419|Pine 5th|Way|Suite G|Oakdale|Harlan County|KY|49584|United States|-6|apartment| +33013|AAAAAAAAFPAIAAAA|527|Washington |Drive|Suite 440|Brownsville|Williams County|ND|59310|United States|-6|apartment| +33014|AAAAAAAAGPAIAAAA|558|Main 10th|Lane|Suite 190|Shiloh|Cedar County|IA|59275|United States|-6|apartment| +33015|AAAAAAAAHPAIAAAA|37|Washington Tenth|RD|Suite 260|Macedonia|Vermilion Parish|LA|71087|United States|-6|condo| +33016|AAAAAAAAIPAIAAAA|609|Main River|Boulevard|Suite 50|Lebanon|Monmouth County|NJ|03498|United States|-5|single family| +33017|AAAAAAAAJPAIAAAA||1st ||Suite T|Fairfield||SD|56192||-6|apartment| +33018|AAAAAAAAKPAIAAAA|34|Mill |Wy|Suite U|Oakland|Brown County|SD|59843|United States|-6|single family| +33019|AAAAAAAALPAIAAAA|601|Locust Center|Circle|Suite E|Clifton|Washington County|GA|38014|United States|-5|condo| +33020|AAAAAAAAMPAIAAAA|773|North |Wy|Suite X|Wildwood|Carroll County|AR|76871|United States|-6|single family| +33021|AAAAAAAANPAIAAAA|316|11th Franklin|Street|Suite 440|Oakland|McPherson County|KS|69843|United States|-6|apartment| +33022|AAAAAAAAOPAIAAAA|960|1st |Street|Suite 290|White Oak|Carroll County|KY|46668|United States|-6|single family| +33023|AAAAAAAAPPAIAAAA||||Suite M|Fairview|||||-5|| +33024|AAAAAAAAAABIAAAA|869|Eigth Seventh|Circle|Suite E|Pleasant Valley|Oneida County|ID|82477|United States|-7|single family| +33025|AAAAAAAABABIAAAA|586|Smith Fourth|Court|Suite F|Mount Zion|Jackson County|MO|68054|United States|-6|single family| +33026|AAAAAAAACABIAAAA|109|Sunset 4th|Way|Suite 380|Highland Park|Henry County|MO|66534|United States|-6|single family| +33027|AAAAAAAADABIAAAA|258|First |Blvd|Suite 260|Cedar Grove|Polk County|TX|70411|United States|-6|condo| +33028|AAAAAAAAEABIAAAA|165|Mill Wilson|Lane|Suite M|Shady Grove|Calhoun County|TX|72812|United States|-6|single family| +33029|AAAAAAAAFABIAAAA|555|Cedar 7th|Blvd|Suite V|Greenfield|Kenedy County|TX|75038|United States|-6|single family| +33030|AAAAAAAAGABIAAAA|893|1st |Street|Suite Q|Walnut Grove|Nye County|NV|87752|United States|-8|apartment| +33031|AAAAAAAAHABIAAAA|543|Ridge |Parkway|Suite H|Lakewood|Calhoun County|TX|78877|United States|-6|apartment| +33032|AAAAAAAAIABIAAAA|791|1st |Drive|Suite U|Elba|Allen County|IN|40262|United States|-5|apartment| +33033|AAAAAAAAJABIAAAA|601|Walnut |Pkwy|Suite L|Glenwood|Sheridan County|MT|63511|United States|-7|single family| +33034|AAAAAAAAKABIAAAA|489|Broadway |Pkwy|Suite 250|Lincoln|Issaquena County|MS|51289|United States|-6|apartment| +33035|AAAAAAAALABIAAAA|922|Lakeview |Street|Suite 110|Murphy|Ralls County|MO|62105|United States|-6|condo| +33036|AAAAAAAAMABIAAAA|657|Jackson |RD|Suite D|Globe|Burt County|NE|69614|United States|-6|apartment| +33037|AAAAAAAANABIAAAA|422|River |Parkway|Suite 400|Glenwood|Coles County|IL|63511|United States|-6|apartment| +33038|AAAAAAAAOABIAAAA|831|Seventh Maple|Street|Suite U|Woodland|Caroline County|VA|24854|United States|-5|condo| +33039|AAAAAAAAPABIAAAA|161|College |Circle|Suite L|Lebanon|Gadsden County|FL|32898|United States|-5|condo| +33040|AAAAAAAAABBIAAAA|607|Willow |Dr.|Suite D|Lincoln|Wright County|MO|61289|United States|-6|condo| +33041|AAAAAAAABBBIAAAA|786|College Dogwood|Ave|Suite P|Stringtown|Lee County|NC|20162|United States|-5|condo| +33042|AAAAAAAACBBIAAAA|||Boulevard||Forest Hills|New Castle County||19237|United States||condo| +33043|AAAAAAAADBBIAAAA|750|2nd Spring|Lane|Suite I|Antioch|Adams County|ID|88605|United States|-7|apartment| +33044|AAAAAAAAEBBIAAAA|309||Court|Suite D||Garfield County||62898|||| +33045|AAAAAAAAFBBIAAAA|371|13th |Pkwy|Suite O|Harmon|Essex County|VA|25623|United States|-5|apartment| +33046|AAAAAAAAGBBIAAAA|901|13th |Boulevard|Suite F|Enterprise|Lyon County|KY|41757|United States|-5|condo| +33047|AAAAAAAAHBBIAAAA|145|Lake |Pkwy|Suite Q|Riverside|Van Zandt County|TX|79231|United States|-6|apartment| +33048|AAAAAAAAIBBIAAAA|26|Williams South|Cir.|Suite 340|Concord|Calhoun County|MS|54107|United States|-6|condo| +33049|AAAAAAAAJBBIAAAA|770|4th |Blvd|Suite 130|Franklin|Columbia County|NY|19101|United States|-5|single family| +33050|AAAAAAAAKBBIAAAA|232|Second |Ave|Suite H|Sulphur Springs|Marion County|MS|58354|United States|-6|apartment| +33051|AAAAAAAALBBIAAAA|792|Mill Railroad|Ave|Suite 130|Bridgeport|Martin County|FL|35817|United States|-5|single family| +33052|AAAAAAAAMBBIAAAA|109||Road|Suite A|Glendale|Treasure County|MT||United States|-7|apartment| +33053|AAAAAAAANBBIAAAA|546|Church ||Suite Y|Kingston|||54975|United States|-6|| +33054|AAAAAAAAOBBIAAAA|197|6th |Road|Suite 370|Buena Vista|Roseau County|MN|55752|United States|-6|single family| +33055|AAAAAAAAPBBIAAAA|391|Davis |Ave|Suite Y|Waterloo|Mercer County|OH|41675|United States|-5|condo| +33056|AAAAAAAAACBIAAAA|100|North |Road|Suite 0|Raymond|Rowan County|NC|20275|United States|-5|single family| +33057|AAAAAAAABCBIAAAA|716|10th |Drive|Suite 350|Stringtown|Dent County|MO|60162|United States|-6|apartment| +33058|AAAAAAAACCBIAAAA|159|View Forest|Lane|Suite 20|Buena Vista|Wheeler County|OR|95752|United States|-8|condo| +33059|AAAAAAAADCBIAAAA|840|Center Railroad|Cir.|Suite Q|Hillcrest|Bucks County|PA|13003|United States|-5|condo| +33060|AAAAAAAAECBIAAAA|478|Hill |Pkwy|Suite I|Sunnyside|Love County|OK|71952|United States|-6|apartment| +33061|AAAAAAAAFCBIAAAA|||Wy|Suite M||Berrien County||48605|United States||| +33062|AAAAAAAAGCBIAAAA|742|Adams Washington|Pkwy|Suite L|Lakeside|Jackson County|OK|79532|United States|-6|single family| +33063|AAAAAAAAHCBIAAAA|614|Laurel Pine|Blvd|Suite F|Stringtown|Blaine County|ID|80162|United States|-7|condo| +33064|AAAAAAAAICBIAAAA|817|Jefferson Center|Boulevard|Suite M|Deerfield|Lanier County|GA|39840|United States|-5|apartment| +33065|AAAAAAAAJCBIAAAA|443|Central Lakeview|Dr.|Suite 310|Vance|Anderson County|TX|70268|United States|-6|condo| +33066|AAAAAAAAKCBIAAAA|730|Fifth |Lane|Suite 180|Marion|Dukes County|MA|00999|United States|-5|single family| +33067|AAAAAAAALCBIAAAA|689|Mill 7th|Ct.|Suite G|Howell|Dubuque County|IA|54854|United States|-6|apartment| +33068|AAAAAAAAMCBIAAAA|178|15th Highland|Parkway|Suite D|White Oak|Holt County|MO|66668|United States|-6|condo| +33069|AAAAAAAANCBIAAAA|450|9th ||Suite 390|Owens|Sullivan County||||-5|| +33070|AAAAAAAAOCBIAAAA|989|Wilson |Wy|Suite W|Shawnee|Worcester County|MD|25085|United States|-5|single family| +33071|AAAAAAAAPCBIAAAA|720|View |Street|Suite 190|Bethel|Fall River County|SD|55281|United States|-6|apartment| +33072|AAAAAAAAADBIAAAA|185|Franklin |Ln|Suite B|Franklin|Douglas County|WA|99101|United States|-8|condo| +33073|AAAAAAAABDBIAAAA|842|Cherry 4th|Lane|Suite 50|Liberty|Douglas County|WA|93451|United States|-8|single family| +33074|AAAAAAAACDBIAAAA|906|Hickory Second|Road|Suite J|Pleasant Valley|Beadle County|SD|52477|United States|-6|single family| +33075|AAAAAAAADDBIAAAA|670|8th |Blvd|Suite U|Marion|White County|GA|30399|United States|-5|apartment| +33076|AAAAAAAAEDBIAAAA|732|Forest |Ave|Suite N|Mount Vernon|Fayette County|KY|48482|United States|-6|single family| +33077|AAAAAAAAFDBIAAAA|512|Hill 4th|Circle|Suite V|Whitney|Fentress County|TN|38339|United States|-5|condo| +33078|AAAAAAAAGDBIAAAA|946|Highland Main|Road|Suite O|Ludlow|Sullivan County|TN|35566|United States|-6|apartment| +33079|AAAAAAAAHDBIAAAA|455|8th |Ln|Suite A|Oakdale|Stoddard County|MO|69584|United States|-6|condo| +33080|AAAAAAAAIDBIAAAA|228|West |Court|Suite V|Waterloo|Chesterfield County|VA|21675|United States|-5|condo| +33081|AAAAAAAAJDBIAAAA|812|Cedar East|Cir.|Suite B|Hopewell|Shoshone County|ID|80587|United States|-7|apartment| +33082|AAAAAAAAKDBIAAAA|165|5th |Drive|Suite 490|Harmony|Haskell County|TX|75804|United States|-6|single family| +33083|AAAAAAAALDBIAAAA|238|Willow Railroad|Street|Suite E|Forest|Frio County|TX|77537|United States|-6|apartment| +33084|AAAAAAAAMDBIAAAA||2nd Laurel|||Woodville||MO|64289|United States||condo| +33085|AAAAAAAANDBIAAAA|284|8th |Cir.|Suite 130|Jackson|McCracken County|KY|49583|United States|-5|condo| +33086|AAAAAAAAODBIAAAA|680|Lake Third|Dr.|Suite R|Carpenter|Alcorn County|MS|51147|United States|-6|single family| +33087|AAAAAAAAPDBIAAAA|120|Dogwood Adams|Boulevard|Suite N|Blue Springs|Union County|AR|74686|United States|-6|condo| +33088|AAAAAAAAAEBIAAAA|687|6th |Ct.|Suite M|Globe|Mercer County|KY|49614|United States|-5|condo| +33089|AAAAAAAABEBIAAAA|845|14th |ST|Suite M|Brownsville|Cross County|AR|79310|United States|-6|apartment| +33090|AAAAAAAACEBIAAAA|993|2nd |Way|Suite C|Mount Pleasant|Lafayette County|WI|51933|United States|-6|apartment| +33091|AAAAAAAADEBIAAAA|911|Maple 12th|Boulevard|Suite 400|Clifton|Litchfield County|CT|08614|United States|-5|single family| +33092|AAAAAAAAEEBIAAAA|404|3rd Madison|Ln|Suite 410|Lakeview|Benton County|IN|48579|United States|-5|condo| +33093|AAAAAAAAFEBIAAAA|157|Jefferson |Circle|Suite P|Plainview|Canyon County|ID|83683|United States|-7|condo| +33094|AAAAAAAAGEBIAAAA|776|Forest Eigth|RD|Suite 160|Ashland|Brown County|SD|54244|United States|-6|apartment| +33095|AAAAAAAAHEBIAAAA|672|Broadway Mill|Blvd|Suite 180|Riverside|Albemarle County|VA|29231|United States|-5|condo| +33096|AAAAAAAAIEBIAAAA|664|Lee South|Pkwy|Suite 30|Brownsville|Atascosa County|TX|79310|United States|-6|apartment| +33097|AAAAAAAAJEBIAAAA|632|3rd Lincoln|Circle|Suite F|Farmington|Lincoln County|TN|39145|United States|-6|condo| +33098|AAAAAAAAKEBIAAAA|176|Third |Circle|Suite I|Ashland|Bland County|VA|24244|United States|-5|apartment| +33099|AAAAAAAALEBIAAAA|932|4th 2nd|Blvd|Suite V|Red Hill|Cortland County|NY|14338|United States|-5|apartment| +33100|AAAAAAAAMEBIAAAA|206|Cedar Elm|Avenue|Suite 490|Clifton|Keokuk County|IA|58014|United States|-6|apartment| +33101|AAAAAAAANEBIAAAA|60|River |RD|Suite Y|Union Hill|Aleutians West Census Area|AK|97746|United States|-9|condo| +33102|AAAAAAAAOEBIAAAA|861|1st |Road|Suite J|Green Acres|Shiawassee County|MI|47683|United States|-5|condo| +33103|AAAAAAAAPEBIAAAA|155|Seventh Third|Lane|Suite S|Belmont|Marlboro County|SC|20191|United States|-5|condo| +33104|AAAAAAAAAFBIAAAA|310|8th Laurel|Cir.|Suite 480|Riverside|Kewaunee County|WI|59231|United States|-6|single family| +33105|AAAAAAAABFBIAAAA|590|Hill 3rd|Cir.|Suite 20|Ashland|Watonwan County|MN|54244|United States|-6|apartment| +33106|AAAAAAAACFBIAAAA|583|4th |Ct.|Suite 460|Stringtown|Vernon Parish|LA|70162|United States|-6|condo| +33107|AAAAAAAADFBIAAAA|689|Mill Locust|Boulevard|Suite 190|Franklin|Tuscaloosa County|AL|39101|United States|-6|apartment| +33108|AAAAAAAAEFBIAAAA|698|Church |Pkwy|Suite O|Deerfield|Randolph County|AL|39840|United States|-6|apartment| +33109|AAAAAAAAFFBIAAAA|957||||Hamilton||TX|72808|||condo| +33110|AAAAAAAAGFBIAAAA|19|Park |Pkwy|Suite 330|Highland|Limestone County|AL|39454|United States|-6|condo| +33111|AAAAAAAAHFBIAAAA|549|Sixth |Wy|Suite N|Unionville|Lincoln County|TN|31711|United States|-6|apartment| +33112|AAAAAAAAIFBIAAAA|950||Circle|Suite 160|Enterprise|||71757|||condo| +33113|AAAAAAAAJFBIAAAA|297|Highland |Avenue|Suite M|Waterloo|Wapello County|IA|51675|United States|-6|apartment| +33114|AAAAAAAAKFBIAAAA|439|Washington |Cir.|Suite O|Plainview|Jackson County|SD|53683|United States|-7|condo| +33115|AAAAAAAALFBIAAAA|21|6th 12th|Dr.|Suite 370|Glendale|Carroll County|AR|73951|United States|-6|condo| +33116|AAAAAAAAMFBIAAAA|189|Tenth Eigth|Blvd|Suite 0|Oakland|Platte County|WY|89843|United States|-7|single family| +33117|AAAAAAAANFBIAAAA|676|First |Pkwy|Suite U|Newport|Henry County|KY|41521|United States|-6|single family| +33118|AAAAAAAAOFBIAAAA|370|Elevnth |Ln|Suite L|Marion|Alleghany County|NC|20399|United States|-5|condo| +33119|AAAAAAAAPFBIAAAA|762|2nd Center|Way|Suite 130|Lebanon|Custer County|OK|72898|United States|-6|condo| +33120|AAAAAAAAAGBIAAAA|701|Locust View|RD|Suite 60|Enterprise|Atkinson County|GA|31757|United States|-5|single family| +33121|AAAAAAAABGBIAAAA|752|Locust |Wy|Suite 10|Franklin|Ballard County|KY|49101|United States|-6|apartment| +33122|AAAAAAAACGBIAAAA|318|Locust |Street|Suite K|Lincoln|Wayne County|OH|41289|United States|-5|apartment| +33123|AAAAAAAADGBIAAAA|996|Broadway 9th|Blvd|Suite J|Centerville|Cleburne County|AL|30059|United States|-6|apartment| +33124|AAAAAAAAEGBIAAAA|100|Wilson |Ave|Suite U|Pleasant Hill|Poweshiek County|IA|53604|United States|-6|single family| +33125|AAAAAAAAFGBIAAAA|855|Johnson 5th|Road|Suite C|Spring Valley|Blue Earth County|MN|56060|United States|-6|apartment| +33126|AAAAAAAAGGBIAAAA|883|14th Walnut|Blvd|Suite Y|Spring Valley|Nance County|NE|66060|United States|-7|apartment| +33127|AAAAAAAAHGBIAAAA|29|Forest Jefferson|Pkwy|Suite P|Wilson|Madison County|MS|56971|United States|-6|single family| +33128|AAAAAAAAIGBIAAAA|56|2nd |Pkwy|Suite 240|Midway|Rhea County|TN|31904|United States|-6|apartment| +33129|AAAAAAAAJGBIAAAA|504|6th 7th|Ave|Suite Q|Fairfield|Warrick County|IN|46192|United States|-5|apartment| +33130|AAAAAAAAKGBIAAAA|763|11th East|Avenue|Suite I|Lakeview|Union County|FL|38579|United States|-5|single family| +33131|AAAAAAAALGBIAAAA|237|Valley |Wy|Suite X|Centerville|Poweshiek County|IA|50059|United States|-6|apartment| +33132|AAAAAAAAMGBIAAAA|114|Dogwood |Parkway|Suite 360|Union|Emmet County|IA|58721|United States|-6|single family| +33133|AAAAAAAANGBIAAAA|945|Sycamore Birch|Ct.|Suite 330|Amherst|Ohio County|IN|48119|United States|-5|single family| +33134|AAAAAAAAOGBIAAAA|787|Mill 1st|Parkway|Suite P|Red Hill|Mitchell County|NC|24338|United States|-5|condo| +33135|AAAAAAAAPGBIAAAA|50|Poplar 6th|RD|Suite T|Jamestown|Benson County|ND|56867|United States|-6|condo| +33136|AAAAAAAAAHBIAAAA|501|First |Ave|Suite L|Pleasant Grove|Grant County|NE|64136|United States|-6|condo| +33137|AAAAAAAABHBIAAAA|307|8th |Boulevard|Suite 0|Antioch|Coffee County|GA|38605|United States|-5|single family| +33138|AAAAAAAACHBIAAAA||Willow Williams|ST|Suite G|Harmony|||25804||-5|apartment| +33139|AAAAAAAADHBIAAAA|904|Elm |Blvd|Suite 490|Buena Vista|Thomas County|NE|65752|United States|-7|condo| +33140|AAAAAAAAEHBIAAAA|241|Adams Johnson|Cir.|Suite 60|New Hope|Alger County|MI|49431|United States|-5|apartment| +33141|AAAAAAAAFHBIAAAA|928|Elm |Circle|Suite D|Kingston|Augusta County|VA|24975|United States|-5|condo| +33142|AAAAAAAAGHBIAAAA|663|Park Maple|Court|Suite 200|Riverdale|Ashland County|OH|49391|United States|-5|apartment| +33143|AAAAAAAAHHBIAAAA|380|11th |Street|Suite 220|Springfield|Hampton city|VA|29303|United States|-5|apartment| +33144|AAAAAAAAIHBIAAAA|165|Elm |Blvd|Suite 340|Gladstone|Anne Arundel County|MD|20894|United States|-5|apartment| +33145|AAAAAAAAJHBIAAAA||Park |ST||Glenwood||OR|||-8|apartment| +33146|AAAAAAAAKHBIAAAA|85|Third Cherry|Parkway|Suite P|Greenville|Walton County|GA|31387|United States|-5|single family| +33147|AAAAAAAALHBIAAAA|42|Elm 12th|Ct.|Suite 110|White Oak|Berkeley County|WV|26668|United States|-5|apartment| +33148|AAAAAAAAMHBIAAAA|888|Miller |Parkway|Suite E|Shady Grove|Caddo Parish|LA|72812|United States|-6|condo| +33149|AAAAAAAANHBIAAAA|57|Willow Cedar|Street|Suite A|Woodville|Alameda County|CA|94289|United States|-8|condo| +33150|AAAAAAAAOHBIAAAA|710|7th View|Lane|Suite 380|Maple Grove|Phillips County|AR|78252|United States|-6|condo| +33151|AAAAAAAAPHBIAAAA|411|Hickory |Circle|Suite C|Union City|Saline County|KS|68087|United States|-6|single family| +33152|AAAAAAAAAIBIAAAA||Hillcrest ||||Cortland County|NY||United States|-5|| +33153|AAAAAAAABIBIAAAA|999|Elm |Avenue|Suite 140|Valley View|Gaston County|NC|25124|United States|-5|single family| +33154|AAAAAAAACIBIAAAA|152|2nd |Ave|Suite Y|Antioch|Hidalgo County|TX|78605|United States|-6|apartment| +33155|AAAAAAAADIBIAAAA|480|Maple Maple|Avenue|Suite N|Hopewell|Baraga County|MI|40587|United States|-5|apartment| +33156|AAAAAAAAEIBIAAAA|54|Meadow Wilson|Boulevard|Suite 310|Valley View|Pontotoc County|OK|75124|United States|-6|apartment| +33157|AAAAAAAAFIBIAAAA|791|Fifteenth Maple|Circle|Suite L|Bethel|Putnam County|FL|35281|United States|-5|condo| +33158|AAAAAAAAGIBIAAAA|599|Wilson Fourth|Court|Suite 280|Unionville|Polk County|TN|31711|United States|-6|apartment| +33159|AAAAAAAAHIBIAAAA|270|3rd |Pkwy|Suite D|Wildwood|Brooks County|GA|36871|United States|-5|single family| +33160|AAAAAAAAIIBIAAAA|425|View |ST|Suite 30|Edgewood|Union County|IA|50069|United States|-6|condo| +33161|AAAAAAAAJIBIAAAA|919|Wilson 2nd|Lane|Suite M|Woodlawn|Codington County|SD|54098|United States|-6|condo| +33162|AAAAAAAAKIBIAAAA|198|12th |Boulevard|Suite 480|Springdale|Washington County|VA|28883|United States|-5|condo| +33163|AAAAAAAALIBIAAAA|353|10th Walnut|ST|Suite 30|Whispering Pines|Salt Lake County|UT|87609|United States|-7|single family| +33164|AAAAAAAAMIBIAAAA|836|Madison Church|Parkway|Suite P|Oakwood|Armstrong County|TX|70169|United States|-6|condo| +33165|AAAAAAAANIBIAAAA|210|Lakeview |Way|Suite O|Shady Grove|Dewey County|OK|72812|United States|-6|apartment| +33166|AAAAAAAAOIBIAAAA|184|Hillcrest |Road|Suite 220|Green Acres|Emmet County|IA|57683|United States|-6|single family| +33167|AAAAAAAAPIBIAAAA|120|6th |Cir.|Suite R|Spring Valley|Sublette County|WY|86060|United States|-7|condo| +33168|AAAAAAAAAJBIAAAA|396|Fifth Jackson|Way|Suite 210|Spring Hill|Decatur County|TN|36787|United States|-5|apartment| +33169|AAAAAAAABJBIAAAA|502|South |RD|Suite 360|Forestville|Franklin County|IA|53027|United States|-6|single family| +33170|AAAAAAAACJBIAAAA|478|Elm |Pkwy|Suite 370|Waterloo|Jackson Parish|LA|71675|United States|-6|apartment| +33171|AAAAAAAADJBIAAAA|991||Avenue|Suite 460|Oak Ridge|Pike County||||-5|| +33172|AAAAAAAAEJBIAAAA|631|East Lakeview|Ct.|Suite R|Belmont|Rosebud County|MT|60191|United States|-7|apartment| +33173|AAAAAAAAFJBIAAAA|197|Davis Cherry|Ln|Suite W|Bunker Hill|Galax city|VA|20150|United States|-5|single family| +33174|AAAAAAAAGJBIAAAA|411|Poplar |ST|Suite 10|Sulphur Springs|Lake County|OH|48354|United States|-5|single family| +33175|AAAAAAAAHJBIAAAA|378|Franklin |Ave|Suite R|Lakeview|Paulding County|GA|38579|United States|-5|condo| +33176|AAAAAAAAIJBIAAAA|949|Fifth |Ln|Suite 170|Harmony|Jefferson County|KS|65804|United States|-6|apartment| +33177|AAAAAAAAJJBIAAAA|695|Oak River|Parkway|Suite 250|Liberty|Ketchikan Gateway Borough|AK|93451|United States|-9|condo| +33178|AAAAAAAAKJBIAAAA|598|Sunset |Ln|Suite A|Hamilton|Decatur County|KS|62808|United States|-6|single family| +33179|AAAAAAAALJBIAAAA|160|South |Avenue|Suite 100|Pleasant Hill|Hamilton County|KS|63604|United States|-6|single family| +33180|AAAAAAAAMJBIAAAA|691|Spruce |Ct.|Suite K|Enterprise|Terry County|TX|71757|United States|-6|condo| +33181|AAAAAAAANJBIAAAA|172|College |Dr.|Suite E|Bethel|Schuyler County|NY|15281|United States|-5|apartment| +33182|AAAAAAAAOJBIAAAA|176|8th East|Drive|Suite 120|Marion|Leslie County|KY|40399|United States|-5|condo| +33183|AAAAAAAAPJBIAAAA|516|Meadow Wilson|Pkwy|Suite 390|Kimball|Pine County|MN|53595|United States|-6|condo| +33184|AAAAAAAAAKBIAAAA|688|Jefferson |Pkwy|Suite 360|Harmony|Stonewall County|TX|75804|United States|-6|condo| +33185|AAAAAAAABKBIAAAA|843|Chestnut |Ave|Suite B|Newtown|Karnes County|TX|71749|United States|-6|apartment| +33186|AAAAAAAACKBIAAAA|927|Railroad |Dr.|Suite A|Ashland|Gogebic County|MI|44244|United States|-5|condo| +33187|AAAAAAAADKBIAAAA|83|Central |Circle|Suite 250|Lakewood|Wood County|TX|78877|United States|-6|condo| +33188|AAAAAAAAEKBIAAAA|576|Elm Elm|Boulevard|Suite N|Fayetteville|Johnson County|TN|31732|United States|-6|apartment| +33189|AAAAAAAAFKBIAAAA|642|5th |Wy|Suite F|Centerville|Walworth County|SD|50059|United States|-7|condo| +33190|AAAAAAAAGKBIAAAA|840|Pine |Boulevard|Suite K|Forest Hills|Duval County|FL|39237|United States|-5|condo| +33191|AAAAAAAAHKBIAAAA|157|Maple |RD|Suite U|Pleasant Valley|Cheboygan County|MI|42477|United States|-5|apartment| +33192|AAAAAAAAIKBIAAAA|468|West Ridge|Street|Suite 320|Denmark|Greene County|AL|35576|United States|-6|apartment| +33193|AAAAAAAAJKBIAAAA|718|Woodland |Pkwy|Suite 420|Kingston|Owsley County|KY|44975|United States|-5|condo| +33194|AAAAAAAAKKBIAAAA|332|Lincoln North|Dr.|Suite 110|Union Hill|Linn County|IA|57746|United States|-6|single family| +33195|AAAAAAAALKBIAAAA|935|Jackson Hill|Ave|Suite 480|Empire|Adair County|KY|44145|United States|-6|single family| +33196|AAAAAAAAMKBIAAAA|185|10th Hill|Drive|Suite 60|Edgewood|Pope County|IL|60069|United States|-6|condo| +33197|AAAAAAAANKBIAAAA|129|Park |Ave|Suite N|Riverside|Neshoba County|MS|59231|United States|-6|condo| +33198|AAAAAAAAOKBIAAAA|345|13th Lakeview|Wy|Suite 230|Newport|Marshall County|IN|41521|United States|-5|condo| +33199|AAAAAAAAPKBIAAAA|855|View |Street|Suite T|Hamilton|Laclede County|MO|62808|United States|-6|condo| +33200|AAAAAAAAALBIAAAA|136|14th |Ln|Suite B|Greenville|Dare County|NC|21387|United States|-5|single family| +33201|AAAAAAAABLBIAAAA|534|12th Laurel|Cir.|Suite D|Brownsville|Williamson County|TN|39310|United States|-5|condo| +33202|AAAAAAAACLBIAAAA|832|Lake Willow|Cir.|Suite 490|Mount Olive|Wayne County|UT|88059|United States|-7|single family| +33203|AAAAAAAADLBIAAAA|485|6th |Blvd|Suite 180|Union|Midland County|MI|48721|United States|-5|apartment| +33204|AAAAAAAAELBIAAAA|997|West |Boulevard|Suite A|Pierce|Denver County|CO|83360|United States|-7|single family| +33205|AAAAAAAAFLBIAAAA|288|Spring Spring|Ave|Suite 10|Pleasant Hill|Tift County|GA|33604|United States|-5|apartment| +33206|AAAAAAAAGLBIAAAA|630|Mill |Pkwy|Suite F|White Oak|Sevier County|TN|36668|United States|-6|apartment| +33207|AAAAAAAAHLBIAAAA|296|River Washington|ST|Suite 190|Enterprise|Emmet County|MI|41757|United States|-5|condo| +33208|AAAAAAAAILBIAAAA|43|South |Court|Suite X|Lakewood|Otsego County|NY|18877|United States|-5|condo| +33209|AAAAAAAAJLBIAAAA|610|Pine |Lane|Suite H|Cedar Grove|Coffee County|TN|30411|United States|-5|apartment| +33210|AAAAAAAAKLBIAAAA|467|Locust View|Court|Suite 10|Lincoln|Pearl River County|MS|51289|United States|-6|single family| +33211|AAAAAAAALLBIAAAA|182|Ridge Oak|Wy|Suite Y|Concord|Baylor County|TX|74107|United States|-6|condo| +33212|AAAAAAAAMLBIAAAA|576|Woodland Davis|Ave|Suite 210|Friendship|Decatur County|TN|34536|United States|-5|condo| +33213|AAAAAAAANLBIAAAA|391|Washington |Pkwy|Suite S|Riverview|Newton County|TX|79003|United States|-6|apartment| +33214|AAAAAAAAOLBIAAAA|298|Smith Church|Way|Suite H|Lakeside|Goliad County|TX|79532|United States|-6|apartment| +33215|AAAAAAAAPLBIAAAA|706|Lincoln Washington|Pkwy|Suite H|Frankfort|Carlisle County|KY|49681|United States|-6|apartment| +33216|AAAAAAAAAMBIAAAA|464|Maple Lincoln|RD|Suite 0|Newtown|Harrison County|OH|41749|United States|-5|condo| +33217|AAAAAAAABMBIAAAA|584|Cherry Ninth|Ct.|Suite K|Jackson|Flathead County|MT|69583|United States|-7|single family| +33218|AAAAAAAACMBIAAAA|740|Broadway Main|Way|Suite S|Georgetown|Meade County|KY|47057|United States|-5|apartment| +33219|AAAAAAAADMBIAAAA|789|North Oak|ST|Suite G|Red Hill|Montgomery County|IN|44338|United States|-5|single family| +33220|AAAAAAAAEMBIAAAA|761|Spruce 7th|Parkway|Suite H|Georgetown|Somervell County|TX|77057|United States|-6|condo| +33221|AAAAAAAAFMBIAAAA|906|1st Green|Way|Suite J|Mount Pleasant|Kent County|MD|21933|United States|-5|condo| +33222|AAAAAAAAGMBIAAAA|796|Third Ash|Drive|Suite 220|Georgetown|Hampton city|VA|27057|United States|-5|apartment| +33223|AAAAAAAAHMBIAAAA|836|Main |Blvd|Suite 430|Bunker Hill|Tunica County|MS|50150|United States|-6|single family| +33224|AAAAAAAAIMBIAAAA|437|Walnut 6th|Dr.|Suite D|New Hope|Wahkiakum County|WA|99431|United States|-8|condo| +33225|AAAAAAAAJMBIAAAA|497|First |Cir.|Suite W|Kingston|Hamilton County|KS|64975|United States|-6|single family| +33226|AAAAAAAAKMBIAAAA|735|Park |Court|Suite 270|Cordova|Boulder County|CO|86938|United States|-7|apartment| +33227|AAAAAAAALMBIAAAA|97|Sixth Miller|Ave|Suite F|Marion|Schenectady County|NY|10399|United States|-5|condo| +33228|AAAAAAAAMMBIAAAA|71|3rd Davis|ST|Suite E|Five Forks|Lee County|MS|52293|United States|-6|single family| +33229|AAAAAAAANMBIAAAA|88|4th |Drive|Suite 170|Florence|Harnett County|NC|23394|United States|-5|condo| +33230|AAAAAAAAOMBIAAAA|163|Adams Center|Street|Suite A|Oakdale|Fort Bend County|TX|79584|United States|-6|single family| +33231|AAAAAAAAPMBIAAAA|104|7th Wilson|Ave|Suite 280|Riverdale|Minnehaha County|SD|59391|United States|-7|single family| +33232|AAAAAAAAANBIAAAA|376|3rd |Avenue|Suite B|Montezuma|Victoria County|TX|72150|United States|-6|apartment| +33233|AAAAAAAABNBIAAAA|553|Poplar Central|Lane|Suite 130|Glendale|Crawford County|IN|43951|United States|-5|apartment| +33234|AAAAAAAACNBIAAAA|359|Johnson 2nd|RD|Suite 40|Buena Vista|Baker County|OR|95752|United States|-8|condo| +33235|AAAAAAAADNBIAAAA|694|7th |Avenue|Suite 90|Marion|Snyder County|PA|10399|United States|-5|apartment| +33236|AAAAAAAAENBIAAAA|594|10th Eigth|Ct.|Suite 340|Clinton|Lane County|OR|98222|United States|-8|apartment| +33237|AAAAAAAAFNBIAAAA|51|11th Walnut|Pkwy|Suite 180|Bridgeport|Union County|SC|25817|United States|-5|condo| +33238|AAAAAAAAGNBIAAAA|931|Wilson 1st|Boulevard|Suite W|Jamestown|Waldo County|ME|07467|United States|-5|single family| +33239|AAAAAAAAHNBIAAAA|651|4th Elm|Road|Suite O|Highland Park|Utah County|UT|86534|United States|-7|apartment| +33240|AAAAAAAAINBIAAAA|507|Ash |Boulevard|Suite B|Woodland|Union County|AR|74854|United States|-6|single family| +33241|AAAAAAAAJNBIAAAA|981|Spruce |Court|Suite 80|Concord|Catawba County|NC|24107|United States|-5|apartment| +33242|AAAAAAAAKNBIAAAA|440|View |Lane|Suite 110|Friendship|Platte County|WY|84536|United States|-7|apartment| +33243|AAAAAAAALNBIAAAA|935|Lake |Court|Suite P|Five Points|Spotsylvania County|VA|26098|United States|-5|condo| +33244|AAAAAAAAMNBIAAAA|237|2nd |ST|Suite L|Mount Olive|Archuleta County|CO|88059|United States|-7|single family| +33245|AAAAAAAANNBIAAAA|100|Broadway |Pkwy|Suite L|Springdale|Fulton County|IN|48883|United States|-5|condo| +33246|AAAAAAAAONBIAAAA|302|Smith |Ct.|Suite 270|Greenwood|Marshall County|OK|78828|United States|-6|single family| +33247|AAAAAAAAPNBIAAAA|646|Spruce |Blvd|Suite N|Elkton|Jackson County|KY|43481|United States|-6|single family| +33248|AAAAAAAAAOBIAAAA|3||Drive||Waterloo||WI|51675|United States|-6|| +33249|AAAAAAAABOBIAAAA|632|Jefferson |Blvd|Suite 90|Harmony|Carter County|MT|65804|United States|-7|single family| +33250|AAAAAAAACOBIAAAA|125|3rd Ridge|Street|Suite 40|Fairview|Wallace County|KS|65709|United States|-6|apartment| +33251|AAAAAAAADOBIAAAA|95|Ash Railroad|Lane|Suite C|Farmington|Fulton County|PA|19145|United States|-5|single family| +33252|AAAAAAAAEOBIAAAA|726|2nd Madison|Court|Suite R|Belfast|Kodiak Island Borough|AK|90125|United States|-9|apartment| +33253|AAAAAAAAFOBIAAAA|766|North |Lane|Suite K|Cedar Grove|Traill County|ND|50411|United States|-6|apartment| +33254|AAAAAAAAGOBIAAAA|934|11th |Dr.|Suite V|Mount Pleasant|Crawford County|PA|11933|United States|-5|single family| +33255|AAAAAAAAHOBIAAAA|578|Lake Maple|RD|Suite 140|Mountain View|Conway County|AR|74466|United States|-6|apartment| +33256|AAAAAAAAIOBIAAAA|911|Fourth |Wy|Suite O|Bethesda|Decatur County|TN|35980|United States|-5|apartment| +33257|AAAAAAAAJOBIAAAA|914|Oak Fifth|Ct.|Suite 470|Arlington|Hot Springs County|WY|86557|United States|-7|apartment| +33258|AAAAAAAAKOBIAAAA|66|Lake Pine|Road|Suite W|Mountain View|Crittenden County|KY|44466|United States|-6|single family| +33259|AAAAAAAALOBIAAAA|275|East Hill|Blvd|Suite I|Jackson|Coos County|NH|09583|United States|-5|condo| +33260|AAAAAAAAMOBIAAAA|351|3rd |Pkwy|Suite 90|Riverview|Leon County|FL|39003|United States|-5|single family| +33261|AAAAAAAANOBIAAAA||Sixth 15th|||Glenwood||AR||||single family| +33262|AAAAAAAAOOBIAAAA|787|Smith |Cir.|Suite P|Friendship|Deer Lodge County|MT|64536|United States|-7|condo| +33263|AAAAAAAAPOBIAAAA|709|Tenth Maple|Wy|Suite R|Wildwood|Sequoyah County|OK|76871|United States|-6|apartment| +33264|AAAAAAAAAPBIAAAA|309|Spruce |Court|Suite 170|Lakeside|Bibb County|GA|39532|United States|-5|condo| +33265|AAAAAAAABPBIAAAA|754|Walnut |Ln|Suite Q|Buena Vista|Clarke County|MS|55752|United States|-6|condo| +33266|AAAAAAAACPBIAAAA||Davis First|Ct.||Oakland|Barton County|MO|||-6|apartment| +33267|AAAAAAAADPBIAAAA|25|South 13th|Court|Suite 420|Deerfield|San Luis Obispo County|CA|99840|United States|-8|condo| +33268|AAAAAAAAEPBIAAAA|542|Williams Elm|Blvd|Suite 330|Hamilton|Phelps County|MO|62808|United States|-6|condo| +33269|AAAAAAAAFPBIAAAA|925|7th |Road|Suite B|Shady Grove|Lee County|MS|52812|United States|-6|condo| +33270|AAAAAAAAGPBIAAAA|313|Elevnth Washington|Parkway|Suite P|Blue Springs|Iron County|WI|54686|United States|-6|single family| +33271|AAAAAAAAHPBIAAAA|879|Spring Franklin|Parkway|Suite 310|Jackson|Cook County|IL|69583|United States|-6|single family| +33272|AAAAAAAAIPBIAAAA|159|Highland |Boulevard|Suite P|Five Points|Piscataquis County|ME|06698|United States|-5|single family| +33273|AAAAAAAAJPBIAAAA|446|11th |Wy|Suite H|Greenville|Wayne County|OH|41387|United States|-5|condo| +33274|AAAAAAAAKPBIAAAA|120|Fourth |Parkway|Suite 300|Greenville|Candler County|GA|31387|United States|-5|apartment| +33275|AAAAAAAALPBIAAAA|993|Hickory |ST|Suite 250|Liberty|Union County|IL|63451|United States|-6|condo| +33276|AAAAAAAAMPBIAAAA|565|Williams Eigth|Ln|Suite 190|Summerville|Montgomery County|VA|22033|United States|-5|apartment| +33277|AAAAAAAANPBIAAAA|689|Hill Broadway|Wy|Suite 140|Newtown|Stokes County|NC|21749|United States|-5|single family| +33278|AAAAAAAAOPBIAAAA||View |Blvd||Antioch||VA|28605|United States|-5|| +33279|AAAAAAAAPPBIAAAA|150|Davis |RD|Suite M|Crossroads|Coshocton County|OH|40534|United States|-5|single family| +33280|AAAAAAAAAACIAAAA|65|Second |Ave|Suite P|Mount Pleasant|Story County|IA|51933|United States|-6|apartment| +33281|AAAAAAAABACIAAAA|961|7th |Dr.|Suite O|Kingston|Putnam County|WV|24975|United States|-5|apartment| +33282|AAAAAAAACACIAAAA|345|View Woodland|Drive|Suite Q|Springfield|Will County|IL|69303|United States|-6|condo| +33283|AAAAAAAADACIAAAA|109|Main Madison|Street|Suite L|Highland|White County|IL|69454|United States|-6|apartment| +33284|AAAAAAAAEACIAAAA|791|Spruce 3rd|Court|Suite 60|Bartlett|Perkins County|NE|64614|United States|-7|condo| +33285|AAAAAAAAFACIAAAA|20|10th |Lane|Suite P|Friendship|Stanton County|NE|64536|United States|-7|condo| +33286|AAAAAAAAGACIAAAA|824|Cedar Main|Pkwy|Suite 60|Mount Olive|Marin County|CA|98059|United States|-8|condo| +33287|AAAAAAAAHACIAAAA|481|14th Madison|ST|Suite Y|Stringtown|Scott County|MO|60162|United States|-6|apartment| +33288|AAAAAAAAIACIAAAA|623|11th |Wy|Suite 10|Crossroads|Carroll County|MS|50534|United States|-6|single family| +33289|AAAAAAAAJACIAAAA|22|6th |Ave|Suite 460|Union Hill|Lee County|IL|67746|United States|-6|apartment| +33290|AAAAAAAAKACIAAAA|218|Oak 9th|Road|Suite 50|Woodland|Knox County|IN|44854|United States|-5|single family| +33291|AAAAAAAALACIAAAA|982|Church |Boulevard|Suite E|Riverside|Routt County|CO|89231|United States|-7|apartment| +33292|AAAAAAAAMACIAAAA|800|Third |Circle|Suite F|Bridgeport|Currituck County|NC|25817|United States|-5|single family| +33293|AAAAAAAANACIAAAA|505|Wilson |Ct.|Suite 260|Green Acres|Custer County|NE|67683|United States|-6|single family| +33294|AAAAAAAAOACIAAAA|486|Ridge Oak|Blvd|Suite 380|Newport|Greene County|NY|11521|United States|-5|apartment| +33295|AAAAAAAAPACIAAAA|307|8th West|Parkway|Suite 340|Centerville|Henry County|OH|40059|United States|-5|apartment| +33296|AAAAAAAAABCIAAAA|376|Cedar |Pkwy|Suite N|Garrison|Jasper County|MS|58767|United States|-6|apartment| +33297|AAAAAAAABBCIAAAA|456|Jefferson |Ave|Suite A|Highland Park|Gordon County|GA|36534|United States|-5|apartment| +33298|AAAAAAAACBCIAAAA|304|Fourth 14th|Cir.|Suite L|Enterprise|Clay County|WV|21757|United States|-5|apartment| +33299|AAAAAAAADBCIAAAA|600|Main |Drive|Suite X|Newport|Worth County|MO|61521|United States|-6|apartment| +33300|AAAAAAAAEBCIAAAA|317|Seventh |Parkway|Suite W|Union|Brunswick County|NC|28721|United States|-5|condo| +33301|AAAAAAAAFBCIAAAA|7|Church 5th|Court|Suite F|Lakeside|Pettis County|MO|69532|United States|-6|condo| +33302|AAAAAAAAGBCIAAAA|537|Main View|Avenue|Suite 420|Fairfield|Washington County|OK|76192|United States|-6|single family| +33303|AAAAAAAAHBCIAAAA|301|View |Street|Suite 120|Springdale|Charleston County|SC|28883|United States|-5|apartment| +33304|AAAAAAAAIBCIAAAA|809|Lincoln |Parkway|Suite X|Pleasant Hill|Nottoway County|VA|23604|United States|-5|single family| +33305|AAAAAAAAJBCIAAAA|548|5th |Court|Suite 460|Greenfield|Forest County|PA|15038|United States|-5|single family| +33306|AAAAAAAAKBCIAAAA|121|7th |Way|Suite P|Summit|Sullivan County|TN|30499|United States|-6|single family| +33307|AAAAAAAALBCIAAAA|659|1st Church|Boulevard|Suite P|Highland Park|Sullivan County|NH|07134|United States|-5|single family| +33308|AAAAAAAAMBCIAAAA|628|Lincoln |Court|Suite 80|Walnut Grove|Webster County|MO|67752|United States|-6|single family| +33309|AAAAAAAANBCIAAAA|588|Third 2nd|Court|Suite 140|Pleasant Hill|Pepin County|WI|53604|United States|-6|apartment| +33310|AAAAAAAAOBCIAAAA|95|Ridge |Ln|Suite 280|Marion|Nance County|NE|60399|United States|-7|single family| +33311|AAAAAAAAPBCIAAAA|435|4th Lincoln|Lane|Suite G|Waterloo|Lee County|FL|31675|United States|-5|apartment| +33312|AAAAAAAAACCIAAAA|925|9th Second|Circle|Suite 220|Oak Hill|Phillips County|MT|67838|United States|-7|apartment| +33313|AAAAAAAABCCIAAAA|252|East 3rd|RD|Suite 160|Franklin|Door County|WI|59101|United States|-6|single family| +33314|AAAAAAAACCCIAAAA|91|Third |Avenue|Suite C|Oakdale|Clay County|IL|69584|United States|-6|apartment| +33315|AAAAAAAADCCIAAAA|449|14th |Lane|Suite U|Fairview|Rolette County|ND|55709|United States|-6|condo| +33316|AAAAAAAAECCIAAAA|216|Elm Chestnut|Avenue|Suite D|Buena Vista|Yankton County|SD|55752|United States|-6|apartment| +33317|AAAAAAAAFCCIAAAA|146|Oak 7th|Parkway|Suite 310|Wildwood|Dixie County|FL|36871|United States|-5|single family| +33318|AAAAAAAAGCCIAAAA|874|Dogwood |Ct.|Suite K|Milan|Graham County|KS|66697|United States|-6|single family| +33319|AAAAAAAAHCCIAAAA|628|4th Adams|Pkwy|Suite Q|Saratoga|Pawnee County|KS|62123|United States|-6|apartment| +33320|AAAAAAAAICCIAAAA|665|Lake |Drive|Suite 440|Sulphur Springs|Wilson County|TN|38354|United States|-5|condo| +33321|AAAAAAAAJCCIAAAA|295|Hickory |Court|Suite 260|Jamestown|Henry County|MO|66867|United States|-6|single family| +33322|AAAAAAAAKCCIAAAA|742|Elm 11th|Boulevard|Suite W|Farmington|Grainger County|TN|39145|United States|-5|apartment| +33323|AAAAAAAALCCIAAAA|53|Cedar Spring|Cir.|Suite 410|Sunnyside|Saline County|NE|61952|United States|-7|condo| +33324|AAAAAAAAMCCIAAAA|80|1st 3rd|Avenue|Suite Y|Waterloo|Madison County|MS|51675|United States|-6|apartment| +33325|AAAAAAAANCCIAAAA||South Lincoln|Ave||Clearview||||United States||| +33326|AAAAAAAAOCCIAAAA|881|Washington Pine|Street|Suite C|Mount Zion|Douglas County|WA|98054|United States|-8|single family| +33327|AAAAAAAAPCCIAAAA|187|Meadow |Ln|Suite Q|Kimball|Davidson County|NC|23595|United States|-5|apartment| +33328|AAAAAAAAADCIAAAA|741|Ninth |Ct.|Suite B|Springdale|Jefferson County|MS|58883|United States|-6|single family| +33329|AAAAAAAABDCIAAAA|385|Locust 6th|Avenue|Suite P|Union Hill|Roberts County|SD|57746|United States|-7|apartment| +33330|AAAAAAAACDCIAAAA|855|Locust Washington|ST|Suite T|Plainview|Vernon Parish|LA|73683|United States|-6|apartment| +33331|AAAAAAAADDCIAAAA|33|Fifth Jefferson|Dr.|Suite 440|Allentown|Deaf Smith County|TX|71838|United States|-6|condo| +33332|AAAAAAAAEDCIAAAA|885|Willow Valley|Lane|Suite 290|Kingston|Lauderdale County|MS|54975|United States|-6|condo| +33333|AAAAAAAAFDCIAAAA||Center |||||||United States|-5|| +33334|AAAAAAAAGDCIAAAA|539|Main |Blvd|Suite 100|California|Hyde County|NC|20141|United States|-5|single family| +33335|AAAAAAAAHDCIAAAA|864|Birch Sunset|Way|Suite U|Mount Pleasant|Oakland County|MI|41933|United States|-5|apartment| +33336|AAAAAAAAIDCIAAAA|866|Second |Drive|Suite I|Greenwood|Todd County|SD|58828|United States|-7|single family| +33337|AAAAAAAAJDCIAAAA|576|11th |Wy|Suite 320|Spring Hill|Cooper County|MO|66787|United States|-6|apartment| +33338|AAAAAAAAKDCIAAAA|637|West 8th|Road|Suite Y|Providence|Logan County|ND|56614|United States|-6|single family| +33339|AAAAAAAALDCIAAAA|60|Elm |Parkway|Suite Q|Kingston|Hardin County|IA|54975|United States|-6|condo| +33340|AAAAAAAAMDCIAAAA|763|Johnson |Drive|Suite R|Kingston|Elbert County|CO|84975|United States|-7|condo| +33341|AAAAAAAANDCIAAAA|252|Main 3rd|Drive|Suite Y|Bridgeport|Anderson County|KS|65817|United States|-6|single family| +33342|AAAAAAAAODCIAAAA|997|Laurel Ridge|Ave|Suite 460|Bloomingdale|Onslow County|NC|21824|United States|-5|condo| +33343|AAAAAAAAPDCIAAAA|34|Seventh Smith|Ct.|Suite 480|Hamilton|Okanogan County|WA|92808|United States|-8|condo| +33344|AAAAAAAAAECIAAAA|373|6th Ninth|Blvd|Suite 340|Bethel|Washington County|IN|45281|United States|-5|single family| +33345|AAAAAAAABECIAAAA|658|Johnson |Road|Suite 330|Sullivan|Perkins County|SD|50451|United States|-7|apartment| +33346|AAAAAAAACECIAAAA|825|Park |ST|Suite D|Harmony|Moody County|SD|55804|United States|-7|single family| +33347|AAAAAAAADECIAAAA|953|14th First|Ct.|Suite 490|Fayetteville|Botetourt County|VA|21732|United States|-5|single family| +33348|AAAAAAAAEECIAAAA|762|Oak Sixth|Avenue|Suite V|Red Hill|Burleson County|TX|74338|United States|-6|apartment| +33349|AAAAAAAAFECIAAAA|681|Williams |RD|Suite 40|Stringtown|Comanche County|KS|60162|United States|-6|apartment| +33350|AAAAAAAAGECIAAAA|13|Hill Park|Dr.|Suite M|Riverdale|Adams County|IL|69391|United States|-6|single family| +33351|AAAAAAAAHECIAAAA|189|3rd |Avenue|Suite K|Valley View|Stevens County|MN|55124|United States|-6|apartment| +33352|AAAAAAAAIECIAAAA|966|4th Valley|Boulevard|Suite P|Ashland|Bandera County|TX|74244|United States|-6|single family| +33353|AAAAAAAAJECIAAAA|595|Park Park|Parkway|Suite 260|Greenville|Winona County|MN|51387|United States|-6|apartment| +33354|AAAAAAAAKECIAAAA|350|First 1st|Ct.|Suite 130|Five Points|Dodge County|MN|56098|United States|-6|condo| +33355|AAAAAAAALECIAAAA|192|Oak Lakeview|Way|Suite 130|Stringtown|Washington County|IL|60162|United States|-6|condo| +33356|AAAAAAAAMECIAAAA|905|4th Forest|Road|Suite V|Pleasant Valley|Crosby County|TX|72477|United States|-6|single family| +33357|AAAAAAAANECIAAAA|925|Hill Broadway|Street||||AR||United States|-6|apartment| +33358|AAAAAAAAOECIAAAA|543|East View|Cir.|Suite 470|Jamestown|Lyon County|NV|86867|United States|-8|condo| +33359|AAAAAAAAPECIAAAA|||Dr.||Union Hill|Terry County|TX||United States||| +33360|AAAAAAAAAFCIAAAA|59|Maple ||Suite P||Colquitt County||34104|||single family| +33361|AAAAAAAABFCIAAAA|237|First Lee|Ave|Suite N|Oakdale|Langlade County|WI|59584|United States|-6|single family| +33362|AAAAAAAACFCIAAAA|981|2nd |Dr.|||Menominee County|WI|55413|United States||| +33363|AAAAAAAADFCIAAAA|739|8th |Drive|Suite 70|Green Acres|Caswell County|NC|27683|United States|-5|condo| +33364|AAAAAAAAEFCIAAAA|481|Woodland |Ct.|Suite 490|Pleasant Hill|Loving County|TX|73604|United States|-6|condo| +33365|AAAAAAAAFFCIAAAA|732|Oak Sixth|Drive|Suite 260|Leesburg|Kearney County|NE|65605|United States|-7|single family| +33366|AAAAAAAAGFCIAAAA|412|Forest |Dr.|Suite P|Sunnyside|Presidio County|TX|71952|United States|-6|single family| +33367|AAAAAAAAHFCIAAAA|812|Cherry Forest|Boulevard|Suite 250|Union Hill|Abbeville County|SC|27746|United States|-5|single family| +33368|AAAAAAAAIFCIAAAA|647|2nd Main|Court|Suite M|Mount Zion|Iron County|MO|68054|United States|-6|single family| +33369|AAAAAAAAJFCIAAAA|23|6th |RD|Suite 350|Fairview|Logan County|WV|25709|United States|-5|condo| +33370|AAAAAAAAKFCIAAAA|129|Johnson |Parkway|Suite 330|Oak Grove|Person County|NC|28370|United States|-5|apartment| +33371|AAAAAAAALFCIAAAA|544|Jackson East|Parkway|Suite D|Five Forks|Sitka Borough|AK|92293|United States|-9|condo| +33372|AAAAAAAAMFCIAAAA|181|1st |Drive|Suite 410|Brownsville|Davis County|IA|59310|United States|-6|single family| +33373|AAAAAAAANFCIAAAA|983|Laurel Park|Drive|Suite K|Five Forks|Fulton County|PA|12293|United States|-5|single family| +33374|AAAAAAAAOFCIAAAA|122|Lake |Pkwy|Suite 370|Vienna|Douglas County|IL|65119|United States|-6|apartment| +33375|AAAAAAAAPFCIAAAA|977|Walnut Meadow|Street|Suite 390|Forest Hills|Beaverhead County|MT|69237|United States|-7|single family| +33376|AAAAAAAAAGCIAAAA|349|15th 7th|Street|Suite F|Highland|Anderson County|TX|79454|United States|-6|condo| +33377|AAAAAAAABGCIAAAA|166|Hillcrest |Lane|Suite 400|New Hope|Cassia County|ID|89431|United States|-7|condo| +33378|AAAAAAAACGCIAAAA|861|Main North|Ct.|Suite 0|Bridgeport|Boone County|IL|65817|United States|-6|apartment| +33379|AAAAAAAADGCIAAAA|143|2nd Jackson|Parkway|Suite E|Pleasant Grove|Chase County|KS|64136|United States|-6|apartment| +33380|AAAAAAAAEGCIAAAA|687|First |Cir.|Suite 100|Bear Creek|Putnam County|GA|33075|United States|-5|apartment| +33381|AAAAAAAAFGCIAAAA|105|Highland |Street|Suite 340|Midway|Seward County|KS|61904|United States|-6|single family| +33382|AAAAAAAAGGCIAAAA|122|Ash West|Drive|Suite I|White Oak|Gogebic County|MI|46668|United States|-5|single family| +33383|AAAAAAAAHGCIAAAA|498|Forest |Dr.|Suite R|Valley View|Sumter County|FL|35124|United States|-5|single family| +33384|AAAAAAAAIGCIAAAA|337|Pine Spring|RD|Suite 40|Red Oak|Crook County|OR|95018|United States|-8|condo| +33385|AAAAAAAAJGCIAAAA|233|Cedar |Lane|Suite 280|Lakewood|Seminole County|OK|78877|United States|-6|apartment| +33386|AAAAAAAAKGCIAAAA|347|Church South|Cir.|Suite Q|Bethel|Rice County|KS|65281|United States|-6|condo| +33387|AAAAAAAALGCIAAAA|872|4th Hickory|Ave|Suite Q|White Oak|Marion County|MS|56668|United States|-6|single family| +33388|AAAAAAAAMGCIAAAA|35|Wilson |RD|Suite 430|Lebanon|Frederick County|VA|22898|United States|-5|condo| +33389|AAAAAAAANGCIAAAA|250|Hillcrest Ridge|Drive|Suite P|Belmont|Indian River County|FL|30191|United States|-5|single family| +33390|AAAAAAAAOGCIAAAA|192|Fourth 3rd|Avenue|Suite 280|Hamilton|Cook County|IL|62808|United States|-6|apartment| +33391|AAAAAAAAPGCIAAAA|544|West 4th|Blvd|Suite 200|Waterloo|Houghton County|MI|41675|United States|-5|condo| +33392|AAAAAAAAAHCIAAAA|117|Park |Wy|Suite B|Pleasant Valley|Crockett County|TX|72477|United States|-6|apartment| +33393|AAAAAAAABHCIAAAA|80|Willow |ST|Suite 120|Maple Grove|Taylor County|TX|78252|United States|-6|single family| +33394|AAAAAAAACHCIAAAA|979|15th |Ave|Suite J|Greenville|Garland County|AR|71387|United States|-6|apartment| +33395|AAAAAAAADHCIAAAA|314|1st Lincoln|Lane|Suite 310|Greenville|Obion County|TN|31387|United States|-6|apartment| +33396|AAAAAAAAEHCIAAAA|653|Maple Park|Drive|Suite T|Mount Vernon|Mason County|WV|28482|United States|-5|single family| +33397|AAAAAAAAFHCIAAAA|920|Maple |Way|Suite G|Winslow|Wood County|OH|48525|United States|-5|single family| +33398|AAAAAAAAGHCIAAAA|309|Birch |ST|Suite H|Franklin|Madison County|MO|69101|United States|-6|single family| +33399|AAAAAAAAHHCIAAAA|983|6th Seventh|Blvd|Suite 330|Macedonia|Greensville County|VA|21087|United States|-5|condo| +33400|AAAAAAAAIHCIAAAA|792|Williams |Avenue|Suite Y|Glenwood|Johnson County|NE|63511|United States|-7|apartment| +33401|AAAAAAAAJHCIAAAA|342|Spruce Smith|Circle|Suite P|Clifton|Hunt County|TX|78014|United States|-6|condo| +33402|AAAAAAAAKHCIAAAA|363|Oak Highland|Ln|Suite Y|Mount Vernon|Greenwood County|KS|68482|United States|-6|condo| +33403|AAAAAAAALHCIAAAA|569|Third Lake|Circle|Suite 80|Valley View|McCurtain County|OK|75124|United States|-6|apartment| +33404|AAAAAAAAMHCIAAAA|641|Pine Smith|Wy|Suite L|Clifton|Newton County|GA|38014|United States|-5|condo| +33405|AAAAAAAANHCIAAAA|184|6th Central|Ct.|Suite M|Cedar Grove|Iron County|UT|80411|United States|-7|condo| +33406|AAAAAAAAOHCIAAAA|95|First Broadway|Ct.|Suite K|Lawrenceville|Brown County|NE|64462|United States|-6|single family| +33407|AAAAAAAAPHCIAAAA|864|Cedar |Ct.|Suite 400|Bridgeport|Aleutians West Census Area|AK|95817|United States|-9|condo| +33408|AAAAAAAAAICIAAAA|188|Dogwood |Pkwy|Suite 220|Oakdale|Newton County|MS|59584|United States|-6|apartment| +33409|AAAAAAAABICIAAAA|575|Valley |Way|Suite X|Wilson|Emery County|UT|86971|United States|-7|apartment| +33410|AAAAAAAACICIAAAA|893|Ash |Boulevard|Suite N|Lakeside|Umatilla County|OR|99532|United States|-8|single family| +33411|AAAAAAAADICIAAAA|28|2nd |Parkway|Suite N|Buena Vista|Cumberland County|NC|25752|United States|-5|condo| +33412|AAAAAAAAEICIAAAA|775|||||Pulaski County|GA|31387|United States|-5|condo| +33413|AAAAAAAAFICIAAAA|236|Central |Court|Suite V|Harmon|Burke County|NC|25623|United States|-5|single family| +33414|AAAAAAAAGICIAAAA|191|Railroad |Drive|Suite 160|Buena Vista|Pitkin County|CO|85752|United States|-7|apartment| +33415|AAAAAAAAHICIAAAA|889|Hill 4th|Court|Suite A|Waterloo|Campbell County|KY|41675|United States|-6|condo| +33416|AAAAAAAAIICIAAAA|425|North |RD|Suite 230|Enterprise|Del Norte County|CA|91757|United States|-8|condo| +33417|AAAAAAAAJICIAAAA|46|Hillcrest Pine|Street|Suite 180|Forest Hills|Hillsborough County|FL|39237|United States|-5|condo| +33418|AAAAAAAAKICIAAAA|742|North Williams|Ave|Suite O|Shady Grove|Grant County|SD|52812|United States|-6|apartment| +33419|AAAAAAAALICIAAAA|728|Railroad Highland|Drive|Suite M|Macedonia|Edwards County|TX|71087|United States|-6|condo| +33420|AAAAAAAAMICIAAAA||South |Road||||IN||United States|-5|| +33421|AAAAAAAANICIAAAA|424|Oak Seventh|Drive|Suite 110|Salem|Yoakum County|TX|78048|United States|-6|single family| +33422|AAAAAAAAOICIAAAA|950|Lee |Dr.|Suite 90|Buena Vista|Douglas County|WA|95752|United States|-8|apartment| +33423|AAAAAAAAPICIAAAA|392|Green |ST|Suite 290|Fairfield|Butler County|KS|66192|United States|-6|single family| +33424|AAAAAAAAAJCIAAAA|402|Franklin Sunset|ST|Suite T|Five Points|Jefferson County|OK|76098|United States|-6|apartment| +33425|AAAAAAAABJCIAAAA|989|Washington |ST|Suite 420|Fairfield|Karnes County|TX|76192|United States|-6|single family| +33426|AAAAAAAACJCIAAAA|549|Spruce Franklin|RD|Suite 440|Centerville|Jefferson Davis Parish|LA|70059|United States|-6|apartment| +33427|AAAAAAAADJCIAAAA|380|Third |Ct.|Suite 150|Pleasant Grove|Goliad County|TX|74136|United States|-6|condo| +33428|AAAAAAAAEJCIAAAA|649|Spring |Ave|Suite U|Florence|Emmons County|ND|53394|United States|-6|single family| +33429|AAAAAAAAFJCIAAAA|150|6th Central|Pkwy|Suite 240|New Hope|Washington County|ME|09431|United States|-5|single family| +33430|AAAAAAAAGJCIAAAA|427|Ridge Lakeview|Wy|Suite 360|Fairfield|Ohio County|WV|26192|United States|-5|apartment| +33431|AAAAAAAAHJCIAAAA|605|Sixth |Court|Suite 230|Blair|Defiance County|OH|45465|United States|-5|single family| +33432|AAAAAAAAIJCIAAAA|515|South 10th|RD|Suite U|Morris|Somervell County|TX|76696|United States|-6|apartment| +33433|AAAAAAAAJJCIAAAA|642|Third 7th|Avenue|Suite F|Gravel Hill|San Benito County|CA|91944|United States|-8|apartment| +33434|AAAAAAAAKJCIAAAA|139|Spruce Wilson|Blvd|Suite 240|Clifton|Pierce County|GA|38014|United States|-5|apartment| +33435|AAAAAAAALJCIAAAA|157|View Williams|Pkwy|Suite 140|Lakeview|Coffee County|GA|38579|United States|-5|apartment| +33436|AAAAAAAAMJCIAAAA|835|Lake |Ct.|Suite W|Harmony|Granville County|NC|25804|United States|-5|condo| +33437|AAAAAAAANJCIAAAA|511|Fourth |Ln|Suite A|Lakeside|Neshoba County|MS|59532|United States|-6|single family| +33438|AAAAAAAAOJCIAAAA|532|Railroad |Boulevard|Suite 60|Woodlawn|Macon County|MO|64098|United States|-6|condo| +33439|AAAAAAAAPJCIAAAA|368|Second |Way|Suite T|Clinton|Randolph County|AL|38222|United States|-6|single family| +33440|AAAAAAAAAKCIAAAA|81|Second |Parkway|Suite R|Antioch|Amador County|CA|98605|United States|-8|single family| +33441|AAAAAAAABKCIAAAA|800|Oak |Street|Suite 170|Lakeview|Macon County|IL|68579|United States|-6|apartment| +33442|AAAAAAAACKCIAAAA||Second Cedar|Court|Suite 140||||66343|||| +33443|AAAAAAAADKCIAAAA|76|2nd |Ave|Suite A|Pinhook|Champaign County|IL|69398|United States|-6|apartment| +33444|AAAAAAAAEKCIAAAA|783|First Jackson|Ct.|Suite 110|Salem|Middlesex County|VA|28048|United States|-5|apartment| +33445|AAAAAAAAFKCIAAAA|547|Sunset |Boulevard|Suite 410|Pleasant Hill|Forrest County|MS|53604|United States|-6|single family| +33446|AAAAAAAAGKCIAAAA|149|15th |Circle|Suite 420|Woodville|Galveston County|TX|74289|United States|-6|apartment| +33447|AAAAAAAAHKCIAAAA|324|Chestnut Park|Road|Suite X|Liberty|Lee County|NC|23451|United States|-5|single family| +33448|AAAAAAAAIKCIAAAA|536|Fifth Chestnut|Drive|Suite 90|Langdon|Clarke County|VA|20852|United States|-5|condo| +33449|AAAAAAAAJKCIAAAA|191|Park Poplar|Circle|Suite 140|Arlington|Wirt County|WV|26557|United States|-5|single family| +33450|AAAAAAAAKKCIAAAA|285|12th |Wy|Suite X|Clinton|Comanche County|KS|68222|United States|-6|apartment| +33451|AAAAAAAALKCIAAAA|630|Main 2nd|Court|Suite X|Kingston|Wapello County|IA|54975|United States|-6|apartment| +33452|AAAAAAAAMKCIAAAA|971|Lee South|Street|Suite 50|Brookwood|Lauderdale County|AL|30965|United States|-6|condo| +33453|AAAAAAAANKCIAAAA|915|Cedar |Wy|Suite 440|Unionville|Ashtabula County|OH|41711|United States|-5|single family| +33454|AAAAAAAAOKCIAAAA|896|Meadow |Blvd|Suite 120|Bridgeport|Jones County|IA|55817|United States|-6|single family| +33455|AAAAAAAAPKCIAAAA|853|Williams Sunset|Street|Suite D|Glendale|Polk County|IA|53951|United States|-6|condo| +33456|AAAAAAAAALCIAAAA|260|West Pine|Cir.|Suite 260|Glendale|Wyoming County|NY|13951|United States|-5|condo| +33457|AAAAAAAABLCIAAAA|242|View |Blvd|Suite T|Valley View|Swift County|MN|55124|United States|-6|single family| +33458|AAAAAAAACLCIAAAA|692|5th 5th|Cir.|Suite 0|Lakewood|Macon County|IL|68877|United States|-6|condo| +33459|AAAAAAAADLCIAAAA|877|First |Avenue|Suite 420|White Oak|Sanilac County|MI|||-5|single family| +33460|AAAAAAAAELCIAAAA|437|Jackson |Ln|Suite W|Greenville|Madison County|IA|51387|United States|-6|apartment| +33461|AAAAAAAAFLCIAAAA|530|Ridge Ninth||||Saunders County|NE|61409|United States||single family| +33462|AAAAAAAAGLCIAAAA|870|Johnson Lake|RD|Suite D|Shiloh|Lowndes County|MS|59275|United States|-6|condo| +33463|AAAAAAAAHLCIAAAA|748|Main |Avenue|Suite 230|Pleasant Valley|Tolland County|CT|03077|United States|-5|single family| +33464|AAAAAAAAILCIAAAA|949|Maple |Way|Suite F|Watkins|Clay County|MS|51732|United States|-6|apartment| +33465|AAAAAAAAJLCIAAAA|372|Sixth Spring|Boulevard|Suite 160|Oakland|Ida County|IA|59843|United States|-6|single family| +33466|AAAAAAAAKLCIAAAA|317|Main 10th|Lane|Suite C|Oak Grove|Wilson County|TX|78370|United States|-6|single family| +33467|AAAAAAAALLCIAAAA|604|3rd Miller|Circle|Suite 320|Greenville|Sumter County|AL|31387|United States|-6|apartment| +33468|AAAAAAAAMLCIAAAA|984|Cherry South|RD|Suite X|Oak Ridge|Goliad County|TX|78371|United States|-6|single family| +33469|AAAAAAAANLCIAAAA|611|Madison |Drive|Suite D|Greenwood|Door County|WI|58828|United States|-6|apartment| +33470|AAAAAAAAOLCIAAAA|409|Pine Poplar|Ave|Suite 330|Pine Grove|Creek County|OK|74593|United States|-6|condo| +33471|AAAAAAAAPLCIAAAA|944|Elm |RD|Suite M|Lakeview|Campbell County|WY|88579|United States|-7|apartment| +33472|AAAAAAAAAMCIAAAA|999|Oak 4th|RD|Suite Y|Longwood|Washington County|KY|47021|United States|-5|condo| +33473|AAAAAAAABMCIAAAA|203|9th Hillcrest|Road|Suite K|Watkins|Moody County|SD|51732|United States|-7|condo| +33474|AAAAAAAACMCIAAAA|183|Cherry 14th|RD|Suite 310|Springdale|Ellis County|KS|68883|United States|-6|single family| +33475|AAAAAAAADMCIAAAA|434|Jackson 7th|Drive|Suite T|Bayview|Madison County|TN|39672|United States|-6|single family| +33476|AAAAAAAAEMCIAAAA|31|Second Main|Wy|Suite R|Enterprise|Greene County|MS|51757|United States|-6|condo| +33477|AAAAAAAAFMCIAAAA|937|Oak Second|Circle|Suite W|Edgewood|Marquette County|MI|40069|United States|-5|apartment| +33478|AAAAAAAAGMCIAAAA|||Lane|Suite 310|Omega|Baldwin County||||-5|| +33479|AAAAAAAAHMCIAAAA|410|Center 11th|Lane|Suite K|Hillcrest|Waseca County|MN|53003|United States|-6|condo| +33480|AAAAAAAAIMCIAAAA|228|Meadow Church|Road|Suite 20|Georgetown|Whatcom County|WA|97057|United States|-8|single family| +33481|AAAAAAAAJMCIAAAA|948|5th |Ln|Suite 460|Pierce|Tangipahoa Parish|LA|73360|United States|-6|single family| +33482|AAAAAAAAKMCIAAAA|424|Lake |Road|Suite 200|Richville|Adair County|IA|55945|United States|-6|condo| +33483|AAAAAAAALMCIAAAA|240|Second |Boulevard|Suite 370|Spring Valley|Allen County|OH|46060|United States|-5|apartment| +33484|AAAAAAAAMMCIAAAA|697|Spruce 5th|Court|Suite E|Macedonia|Mackinac County|MI|41087|United States|-5|condo| +33485|AAAAAAAANMCIAAAA|924|Maple 7th|Dr.|Suite 430|Oak Ridge|Randolph County|WV|28371|United States|-5|apartment| +33486|AAAAAAAAOMCIAAAA|511|Church |Ave|Suite 450|Clinton|Waushara County|WI|58222|United States|-6|condo| +33487|AAAAAAAAPMCIAAAA|397|Sixth Washington|Boulevard|Suite X|Red Hill|Holt County|NE|64338|United States|-7|condo| +33488|AAAAAAAAANCIAAAA|301|1st |Way|Suite 180|Woodland|Cameron County|PA|14854|United States|-5|single family| +33489|AAAAAAAABNCIAAAA|817|Hill Third|Ct.|Suite W|Newport|Cumberland County|ME|02121|United States|-5|condo| +33490|AAAAAAAACNCIAAAA|832|Lakeview |Street|Suite 390|Wildwood|Brunswick County|NC|26871|United States|-5|condo| +33491|AAAAAAAADNCIAAAA|49|Meadow Hill|ST|Suite 60|Oakwood|Edmonson County|KY|40169|United States|-6|apartment| +33492|AAAAAAAAENCIAAAA|398|9th |Lane|Suite E|Greenwood|Webster County|KY|48828|United States|-5|condo| +33493|AAAAAAAAFNCIAAAA|62|Chestnut 2nd|Ave|Suite I|Mount Zion|Richland County|OH|48054|United States|-5|single family| +33494|AAAAAAAAGNCIAAAA|394|4th |Avenue|Suite 290|Oakland|Duval County|FL|39843|United States|-5|apartment| +33495|AAAAAAAAHNCIAAAA|319|Central |Boulevard|Suite X|Franklin|Thurston County|NE|69101|United States|-7|apartment| +33496|AAAAAAAAINCIAAAA|407|Lincoln |Drive|Suite 270|Bunker Hill|Berks County|PA|10150|United States|-5|single family| +33497|AAAAAAAAJNCIAAAA|150|Park |Boulevard|Suite C|Antioch|Union County|OR|98605|United States|-8|condo| +33498|AAAAAAAAKNCIAAAA|266|12th |Ct.|Suite D|Glenwood|Moore County|TN|33511|United States|-6|single family| +33499|AAAAAAAALNCIAAAA|876|Elm |Road|Suite 270|Oakwood|Hardy County|WV|20169|United States|-5|single family| +33500|AAAAAAAAMNCIAAAA|828|Green |Drive|Suite 50|Kingston|Sevier County|AR|74975|United States|-6|apartment| +33501|AAAAAAAANNCIAAAA|637|Locust |ST|Suite 90|Riverdale|Lubbock County|TX|79391|United States|-6|apartment| +33502|AAAAAAAAONCIAAAA|910|Hickory Railroad|RD|Suite 470|Walnut Grove|Clearwater County|MN|57752|United States|-6|condo| +33503|AAAAAAAAPNCIAAAA|678|2nd Meadow|Lane|Suite H|Sulphur Springs|Buchanan County|IA|58354|United States|-6|condo| +33504|AAAAAAAAAOCIAAAA|196|Third Maple|Way|Suite Y|Franklin|Pinellas County|FL|39101|United States|-5|condo| +33505|AAAAAAAABOCIAAAA|493|1st 4th|Wy|Suite X|Bethel|Dodge County|MN|55281|United States|-6|condo| +33506|AAAAAAAACOCIAAAA|108|Ash |Ct.|Suite 410|Sawyer|Taylor County|KY|46045|United States|-5|single family| +33507|AAAAAAAADOCIAAAA|964|Fifth |Cir.|Suite O|Stringtown|Philadelphia County|PA|10162|United States|-5|apartment| +33508|AAAAAAAAEOCIAAAA|513|Pine |ST|Suite 170|Arlington|District of Columbia|DC|26557|United States|-5|apartment| +33509|AAAAAAAAFOCIAAAA|52|Jefferson Sycamore|Boulevard|Suite P|Granite|Fayette County|TX|76284|United States|-6|condo| +33510|AAAAAAAAGOCIAAAA|313|West |Court|Suite 490|Maple Grove|Tippecanoe County|IN|48252|United States|-5|single family| +33511|AAAAAAAAHOCIAAAA|960|South 1st|Ct.|Suite 300|Mount Zion|Huerfano County|CO|88054|United States|-7|condo| +33512|AAAAAAAAIOCIAAAA|428|Adams |Avenue|Suite G|Summit|Suwannee County|FL|30499|United States|-5|condo| +33513|AAAAAAAAJOCIAAAA|9|Park 3rd|Blvd|Suite X|Oak Ridge|Johnson County|TX|78371|United States|-6|apartment| +33514|AAAAAAAAKOCIAAAA|115|First Lincoln|Cir.|Suite 300|Franklin|Fulton County|PA|19101|United States|-5|apartment| +33515|AAAAAAAALOCIAAAA|135|Willow |Pkwy|Suite X|Mount Pleasant|Buchanan County|MO|61933|United States|-6|single family| +33516|AAAAAAAAMOCIAAAA|18|West |Ave|Suite J|Shady Grove|Winn Parish|LA|72812|United States|-6|single family| +33517|AAAAAAAANOCIAAAA|276|Main Pine|Cir.|Suite Y|Lakewood|Montgomery County|MO|68877|United States|-6|condo| +33518|AAAAAAAAOOCIAAAA|291|Oak |Boulevard|Suite 200|Oak Ridge|Roosevelt County|NM|88371|United States|-7|condo| +33519|AAAAAAAAPOCIAAAA|860|Lakeview |Street|Suite J|Oakdale|Monroe County|MO|69584|United States|-6|apartment| +33520|AAAAAAAAAPCIAAAA|842|Pine |Pkwy|Suite F|Pleasant Hill|Lamar County|TX|73604|United States|-6|single family| +33521|AAAAAAAABPCIAAAA|74|Chestnut 11th|Wy|Suite W|Mount Pleasant|Brown County|IN|41933|United States|-5|apartment| +33522|AAAAAAAACPCIAAAA|751|Lee 4th|Parkway|Suite R|Pleasant Valley|Fresno County|CA|92477|United States|-8|single family| +33523|AAAAAAAADPCIAAAA|42|Highland |Drive|Suite F|Union Hill|Collier County|FL|37746|United States|-5|single family| +33524|AAAAAAAAEPCIAAAA|345|Hill 1st|Circle|Suite K|Louisville|Alexander County|IL|64464|United States|-6|single family| +33525|AAAAAAAAFPCIAAAA|856|Dogwood |Dr.|Suite H|Ruth|Johnson County|WY|80309|United States|-7|apartment| +33526|AAAAAAAAGPCIAAAA|340|Lee 10th|Ave|Suite 330|Four Points|Union County|OR|91216|United States|-8|apartment| +33527|AAAAAAAAHPCIAAAA|217|Main 12th|Circle|Suite 110|Farmington|Norton County|KS|69145|United States|-6|condo| +33528|AAAAAAAAIPCIAAAA|364|Fourth |Avenue|Suite O|Saint Clair|Gaston County|NC|25294|United States|-5|condo| +33529|AAAAAAAAJPCIAAAA|331|4th 9th|Street|Suite 490|Cedar Grove|Gem County|ID|80411|United States|-7|apartment| +33530|AAAAAAAAKPCIAAAA|126|Main |Court|Suite 330|Springfield|Pike County|GA|39303|United States|-5|apartment| +33531|AAAAAAAALPCIAAAA|89|Railroad |Parkway|Suite 270|Clifford|Charlevoix County|MI|48164|United States|-5|single family| +33532|AAAAAAAAMPCIAAAA|300|First |RD|Suite L|Redland|Minidoka County|ID|86343|United States|-7|apartment| +33533|AAAAAAAANPCIAAAA|935|Lincoln |Lane|Suite L|Red Hill|Mitchell County|TX|74338|United States|-6|apartment| +33534|AAAAAAAAOPCIAAAA|750|Fifth 3rd|Blvd|Suite L|Five Points|Boone County|IN|46098|United States|-5|apartment| +33535|AAAAAAAAPPCIAAAA|411|Smith |Dr.|Suite 140|Clifton|Aurora County|SD|58014|United States|-6|single family| +33536|AAAAAAAAAADIAAAA|212|Forest 6th|Cir.|Suite B|Green Acres|Garfield County|MT|67683|United States|-7|single family| +33537|AAAAAAAABADIAAAA|998|Cherry Lake|Way|Suite 290|Bethel|Fairfield County|SC|25281|United States|-5|apartment| +33538|AAAAAAAACADIAAAA|37|Elm Broadway|Court|Suite 470|Midway|Unicoi County|TN|31904|United States|-6|single family| +33539|AAAAAAAADADIAAAA|543|Hillcrest |Blvd|Suite 380|Springdale|Pitkin County|CO|88883|United States|-7|apartment| +33540|AAAAAAAAEADIAAAA|357|Poplar Jefferson|Blvd|Suite T|Woodland|Calaveras County|CA|94854|United States|-8|apartment| +33541|AAAAAAAAFADIAAAA|254|Cherry Mill|Boulevard|Suite L|Greenwood|Wadena County|MN|58828|United States|-6|apartment| +33542|AAAAAAAAGADIAAAA|370|View East|Blvd|Suite 350|Kingston|Monroe County|NY|14975|United States|-5|apartment| +33543|AAAAAAAAHADIAAAA|876|College |Avenue|Suite C|Shiloh|Daviess County|IN|49275|United States|-5|apartment| +33544|AAAAAAAAIADIAAAA|5|Park |Boulevard|Suite D|Birmingham|Williams County|ND|53372|United States|-6|apartment| +33545|AAAAAAAAJADIAAAA|1000|Johnson 3rd|Street|Suite 190|Oakwood|Boyle County|KY|40169|United States|-6|apartment| +33546|AAAAAAAAKADIAAAA|20|Ridge |RD|Suite 310|Buena Vista|Douglas County|OR|95752|United States|-8|apartment| +33547|AAAAAAAALADIAAAA|303|4th |Pkwy|Suite B|Marion|La Plata County|CO|80399|United States|-7|apartment| +33548|AAAAAAAAMADIAAAA|434|First |Cir.|Suite 160|Little River|Park County|MT|60319|United States|-7|apartment| +33549|AAAAAAAANADIAAAA|527|Meadow |Avenue|Suite 150|Arcola|Osage County|MO|61654|United States|-6|single family| +33550|AAAAAAAAOADIAAAA|657|Railroad River|Dr.|Suite 100|Salem|Taylor County|WV|28048|United States|-5|single family| +33551|AAAAAAAAPADIAAAA|934|View |Blvd|Suite K|Mountain View|Cleveland County|AR|74466|United States|-6|condo| +33552|AAAAAAAAABDIAAAA|800|9th |RD|Suite 160|Enterprise|Steele County|MN|51757|United States|-6|single family| +33553|AAAAAAAABBDIAAAA|869|South Eigth|Wy|Suite 360|Stringtown|Caldwell Parish|LA|70162|United States|-6|condo| +33554|AAAAAAAACBDIAAAA|388|Ridge Cedar|Ln|Suite R|Buena Vista|Carroll County|NH|06352|United States|-5|apartment| +33555|AAAAAAAADBDIAAAA|647|3rd Hickory|Boulevard|Suite D|Wilton|Henry County|IA|56997|United States|-6|single family| +33556|AAAAAAAAEBDIAAAA|643|Willow |Ave|Suite F|Lincoln|Cowley County|KS|61289|United States|-6|condo| +33557|AAAAAAAAFBDIAAAA|483|Chestnut South|Lane|Suite N|Arlington|Pulaski County|KY|46557|United States|-5|apartment| +33558|AAAAAAAAGBDIAAAA|731|Lake |Court|Suite 370|Mount Pleasant|Colleton County|SC|21933|United States|-5|condo| +33559|AAAAAAAAHBDIAAAA|454|2nd |ST|Suite C|Bridgeport|Harvey County|KS|65817|United States|-6|single family| +33560|AAAAAAAAIBDIAAAA|860|Oak |Avenue|Suite 350|Sutton|Yolo County|CA|95413|United States|-8|apartment| +33561|AAAAAAAAJBDIAAAA|939|South Willow|Boulevard|Suite T|Liberty|Loup County|NE|63451|United States|-7|condo| +33562|AAAAAAAAKBDIAAAA|373|Locust Valley|Pkwy|Suite G|Red Bank|Hansford County|TX|74975|United States|-6|apartment| +33563|AAAAAAAALBDIAAAA|722|Cherry Lake|Blvd|Suite 350|Hopewell|Wayne County|NC|20587|United States|-5|apartment| +33564|AAAAAAAAMBDIAAAA|105|Eigth |ST|Suite S|Glendale|||||-5|apartment| +33565|AAAAAAAANBDIAAAA|92|Ninth |Circle|Suite M|Edgewood|Dallam County|TX|70069|United States|-6|apartment| +33566|AAAAAAAAOBDIAAAA|306|Valley Chestnut|Street|Suite N|Mount Olive|Dearborn County|IN|48059|United States|-5|condo| +33567|AAAAAAAAPBDIAAAA|169|9th |Street|Suite 420|Plainview|Kearney County|NE|63683|United States|-7|single family| +33568|AAAAAAAAACDIAAAA|130|First |Ct.|Suite L|Antioch|Gila County|AZ|88605|United States|-7|single family| +33569|AAAAAAAABCDIAAAA|459|Valley |Cir.|Suite T|Lebanon|Lincoln County|WV|22898|United States|-5|condo| +33570|AAAAAAAACCDIAAAA|762|Woodland Poplar|Road|Suite Q|Woodville|Kingfisher County|OK|74289|United States|-6|single family| +33571|AAAAAAAADCDIAAAA|96|Wilson Maple|Wy|Suite I|Riverside|Apache County|AZ|89231|United States|-7|apartment| +33572|AAAAAAAAECDIAAAA|636|Willow |Blvd|Suite E|Riverdale|Knox County|MO|69391|United States|-6|apartment| +33573|AAAAAAAAFCDIAAAA|970|Cedar Jefferson|Drive|Suite I|Texas|Webster County|KY|43342|United States|-5|apartment| +33574|AAAAAAAAGCDIAAAA|987|First |Avenue|Suite M|Highland|Hillsborough County|NH|09454|United States|-5|condo| +33575|AAAAAAAAHCDIAAAA|210|Laurel |Blvd|Suite 210|Highland Park|New Kent County|VA|26534|United States|-5|apartment| +33576|AAAAAAAAICDIAAAA|561|First 3rd|Cir.|Suite S|Enterprise|Kane County|IL|61757|United States|-6|apartment| +33577|AAAAAAAAJCDIAAAA|437|View Highland|Avenue|Suite 260|Liberty|Walton County|FL|33451|United States|-5|single family| +33578|AAAAAAAAKCDIAAAA|39|Spruce |Ln|Suite 410|Harmony|Casey County|KY|45804|United States|-6|condo| +33579|AAAAAAAALCDIAAAA|800|Ash Valley|RD|Suite 420|Kingston|Grant County|KS|64975|United States|-6|single family| +33580|AAAAAAAAMCDIAAAA|183|Maple |Ln|Suite 360|Greenville|Buena Vista County|IA|51387|United States|-6|single family| +33581|AAAAAAAANCDIAAAA|706|West |Ln|Suite N|Salem|Gove County|KS|68048|United States|-6|condo| +33582|AAAAAAAAOCDIAAAA|354|Oak |Wy|Suite B|Salem|Carter County|OK|78048|United States|-6|apartment| +33583|AAAAAAAAPCDIAAAA|524|Washington Hill|Circle|Suite 10|Greenfield|Morgan County|AL|35038|United States|-6|condo| +33584|AAAAAAAAADDIAAAA|919|8th South|Street|Suite 190|Calhoun|Wichita County|TX|76909|United States|-6|apartment| +33585|AAAAAAAABDDIAAAA|462|Meadow |Ave|Suite L|Oakdale|Mercer County|MO|69584|United States|-6|apartment| +33586|AAAAAAAACDDIAAAA|896|South |Avenue|Suite 300|Brownsville|Stephens County|TX|79310|United States|-6|single family| +33587|AAAAAAAADDDIAAAA|456|Seventh |Ln|Suite F|Lebanon|Deaf Smith County|TX|72898|United States|-6|apartment| +33588|AAAAAAAAEDDIAAAA|404|Highland Smith|Drive|Suite X|Lebanon|Vernon County|WI|52898|United States|-6|single family| +33589|AAAAAAAAFDDIAAAA|948|Madison |Court|Suite F|Valley View|Otero County|CO|85124|United States|-7|apartment| +33590|AAAAAAAAGDDIAAAA|889|7th |Lane|Suite 380|Wilson|Edgecombe County|NC|26971|United States|-5|condo| +33591|AAAAAAAAHDDIAAAA|593|Washington |ST|Suite 200|Longwood|Calhoun County|AR|77021|United States|-6|apartment| +33592|AAAAAAAAIDDIAAAA|773|3rd |Dr.|Suite A|Oak Hill|Northumberland County|VA|27838|United States|-5|condo| +33593|AAAAAAAAJDDIAAAA|122|Spring Lincoln|Dr.|Suite 130|Valley View|Clay County|MS|55124|United States|-6|condo| +33594|AAAAAAAAKDDIAAAA|930|6th |Avenue|Suite 260|Summit|Linn County|OR|90499|United States|-8|apartment| +33595|AAAAAAAALDDIAAAA||||Suite 370|||GA|36534|United States|-5|condo| +33596|AAAAAAAAMDDIAAAA|848|Railroad |Ln|Suite 280|Riverview|Evangeline Parish|LA|79003|United States|-6|condo| +33597|AAAAAAAANDDIAAAA|297|Hickory |Ct.|Suite 280|Glendale|Vilas County|WI|53951|United States|-6|condo| +33598|AAAAAAAAODDIAAAA|454|Church 12th|Avenue|Suite 420|Shiloh|Parke County|IN|49275|United States|-5|apartment| +33599|AAAAAAAAPDDIAAAA|742|Mill |Ave|Suite F|Hillcrest|Estill County|KY|43003|United States|-6|single family| +33600|AAAAAAAAAEDIAAAA|703|College South|Circle|Suite 270|Hamilton|Sequoyah County|OK|72808|United States|-6|condo| +33601|AAAAAAAABEDIAAAA|600|Forest West|Pkwy|Suite 330|Concord|Polk County|IA|54107|United States|-6|apartment| +33602|AAAAAAAACEDIAAAA|88|Washington West|Avenue|Suite 230|Sunnyside|Elkhart County|IN|41952|United States|-5|single family| +33603|AAAAAAAADEDIAAAA|62|Poplar |Parkway|Suite J|Shady Grove|Evans County|GA|32812|United States|-5|single family| +33604|AAAAAAAAEEDIAAAA|301|Lee Willow|Pkwy|Suite R|Wildwood|Genesee County|MI|46871|United States|-5|apartment| +33605|AAAAAAAAFEDIAAAA|918|North Pine|Way|Suite 0|Arlington|Franklin County|FL|36557|United States|-5|condo| +33606|AAAAAAAAGEDIAAAA|317|7th |Court|Suite 400|Hopewell|Wayne County|GA|30587|United States|-5|apartment| +33607|AAAAAAAAHEDIAAAA|349|Maple Woodland|Street|Suite 40|Plainview|Stanly County|NC|23683|United States|-5|apartment| +33608|AAAAAAAAIEDIAAAA|708|Willow 2nd|Cir.|Suite G|Lakeside|Providence County|RI|09532|United States|-5|condo| +33609|AAAAAAAAJEDIAAAA|492|Madison Cedar|Boulevard|Suite H|Fairview|Washington County|OH|45709|United States|-5|single family| +33610|AAAAAAAAKEDIAAAA|||Blvd|Suite N|Woodlawn|James City County||||-5|| +33611|AAAAAAAALEDIAAAA|7|Walnut |Way|Suite 420|Sunnyside|Cavalier County|ND|51952|United States|-6|single family| +33612|AAAAAAAAMEDIAAAA|884|Wilson |Way|Suite 180|Union Hill|Storey County|NV|87746|United States|-8|condo| +33613|AAAAAAAANEDIAAAA|577|Elm Elm|ST|Suite W|Newport|Delaware County|OH|41521|United States|-5|apartment| +33614|AAAAAAAAOEDIAAAA|189|6th |Cir.|Suite R|Salem|Robertson County|TN|38048|United States|-6|single family| +33615|AAAAAAAAPEDIAAAA|359|Main |Parkway|Suite 220|Springdale|Albany County|WY|88883|United States|-7|condo| +33616|AAAAAAAAAFDIAAAA|319|Pine Sixth|Ct.|Suite N|Washington Heights|Clay County|KY|48167|United States|-6|single family| +33617|AAAAAAAABFDIAAAA|184|Spring Johnson|Ln|Suite 340|Jackson|Butler County|IA|59583|United States|-6|condo| +33618|AAAAAAAACFDIAAAA|388|South |Boulevard|Suite 470|Pleasant Valley|Lee County|IA|52477|United States|-6|single family| +33619|AAAAAAAADFDIAAAA|165|Walnut |Ln|Suite 440|Antioch|Maricopa County|AZ|88605|United States|-7|condo| +33620|AAAAAAAAEFDIAAAA|485|Woodland |Road|Suite 220|Jones|Southampton County|VA|22686|United States|-5|condo| +33621|AAAAAAAAFFDIAAAA|528|Cedar |Ct.|Suite 470|Lakeside|Grant County|NE|69532|United States|-6|single family| +33622|AAAAAAAAGFDIAAAA|481|Fourth |Ln|Suite C|Forest Hills|Lincoln County|MO|69237|United States|-6|condo| +33623|AAAAAAAAHFDIAAAA|245|Oak River|Cir.|Suite W|Nichols|Golden Valley County|ND|57940|United States|-6|single family| +33624|AAAAAAAAIFDIAAAA|592|4th |ST|Suite 250|Arlington|Tattnall County|GA|36557|United States|-5|single family| +33625|AAAAAAAAJFDIAAAA|143|Elm Broadway|RD|Suite 30|Bunker Hill|Sac County||50150|United States|-6|| +33626|AAAAAAAAKFDIAAAA|229|Oak Lee|Road|Suite B|Proctor|Culpeper County|VA|28140|United States|-5|apartment| +33627|AAAAAAAALFDIAAAA|747|Eigth River|Road|Suite 130|Mount Pleasant|Caswell County|NC|21933|United States|-5|apartment| +33628|AAAAAAAAMFDIAAAA|432|7th Pine|Ct.|Suite 470|Oak Ridge|Carson City|NV|88371|United States|-8|apartment| +33629|AAAAAAAANFDIAAAA||First Franklin|Way|Suite 380|||||||apartment| +33630|AAAAAAAAOFDIAAAA|971|||Suite G|Glendale|Columbia County|GA|33951|United States||| +33631|AAAAAAAAPFDIAAAA|321|Willow |Pkwy|Suite 380|Hopewell|Barbour County|AL|30587|United States|-6|condo| +33632|AAAAAAAAAGDIAAAA|957|2nd Hill|Dr.|Suite M|Green Acres|Brule County|SD|57683|United States|-6|single family| +33633|AAAAAAAABGDIAAAA|935|Maple 3rd|Ave|Suite 40|Providence|Clay County|KY|46614|United States|-6|apartment| +33634|AAAAAAAACGDIAAAA|431|Fourth |Road|Suite 290|Sulphur Springs|Ransom County|ND|58354|United States|-6|apartment| +33635|AAAAAAAADGDIAAAA|157|North 1st|Dr.|Suite O|Spring Lake|Bernalillo County|NM|89275|United States|-7|single family| +33636|AAAAAAAAEGDIAAAA|65|6th Highland|Ct.|Suite W|Glendale|Pueblo County|CO|83951|United States|-7|single family| +33637|AAAAAAAAFGDIAAAA|431|Chestnut |Parkway|Suite H|Franklin|Terrell County|TX|79101|United States|-6|condo| +33638|AAAAAAAAGGDIAAAA|708|Pine |Street|Suite Q|Fairfield|Branch County|MI|46192|United States|-5|single family| +33639|AAAAAAAAHGDIAAAA|846|Spruce |Dr.|Suite R|Murphy|Trego County|KS|62105|United States|-6|apartment| +33640|AAAAAAAAIGDIAAAA|451|4th |Wy|Suite P|Bunker Hill|Crawford County|IN|40150|United States|-5|single family| +33641|AAAAAAAAJGDIAAAA|854|Broadway |Drive|Suite B|Jamestown|Wasco County|OR|96867|United States|-8|apartment| +33642|AAAAAAAAKGDIAAAA|769|Smith Second|Road|Suite 340|Altamont|DuPage County|IL|69387|United States|-6|single family| +33643|AAAAAAAALGDIAAAA|537|Main |Drive|Suite W|Edgewood|Atlantic County|NJ|00669|United States|-5|single family| +33644|AAAAAAAAMGDIAAAA|698|8th First|Wy|Suite 320|Kingston|Hanover County|VA|24975|United States|-5|condo| +33645|AAAAAAAANGDIAAAA|57|Railroad |Parkway|Suite F|Deerfield|Hancock County|MS|59840|United States|-6|condo| +33646|AAAAAAAAOGDIAAAA|571|Eigth Fourth|Street|Suite 90|Jamestown|Whitley County|KY|46867|United States|-5|condo| +33647|AAAAAAAAPGDIAAAA|184|11th Central|Dr.|Suite 340|Union|Carroll County|TN|38721|United States|-5|condo| +33648|AAAAAAAAAHDIAAAA|516|2nd 3rd|Lane|Suite 10|Berea|Boone County|IA|53804|United States|-6|apartment| +33649|AAAAAAAABHDIAAAA|91|Franklin |Street|Suite U|Pleasant Valley|Floyd County|VA|22477|United States|-5|apartment| +33650|AAAAAAAACHDIAAAA|98|Broadway Sunset|Circle|Suite D|Highland|Ringgold County|IA|59454|United States|-6|apartment| +33651|AAAAAAAADHDIAAAA|197|Locust Pine|Wy|Suite 240|Mount Vernon|Letcher County|KY|48482|United States|-5|condo| +33652|AAAAAAAAEHDIAAAA|25|North |Circle|Suite 10|Woodville|Stanly County|NC|24289|United States|-5|apartment| +33653|AAAAAAAAFHDIAAAA|961|Meadow 9th|Ct.|Suite M|Farmington|Lake County|OH|49145|United States|-5|single family| +33654|AAAAAAAAGHDIAAAA|489|1st Walnut|Parkway|Suite 270|Lebanon|Kiowa County|OK|72898|United States|-6|condo| +33655|AAAAAAAAHHDIAAAA|289|Madison |Lane|Suite U|Maple Grove|Jefferson County|KS|68252|United States|-6|condo| +33656|AAAAAAAAIHDIAAAA|667|Pine Second|Avenue|Suite 350|Georgetown|Fulton County|GA|37057|United States|-5|single family| +33657|AAAAAAAAJHDIAAAA|231|11th |Wy|Suite 390|Georgetown|Napa County|CA|97057|United States|-8|apartment| +33658|AAAAAAAAKHDIAAAA|966|Fifth Center|Road|Suite 290|Mount Olive|Jackson County|GA|38059|United States|-5|apartment| +33659|AAAAAAAALHDIAAAA|918|Walnut River|Street|Suite R|Milan|Lee County|VA|26697|United States|-5|apartment| +33660|AAAAAAAAMHDIAAAA|561|8th |Road|Suite 260|Crossroads|Bayfield County|WI|50534|United States|-6|single family| +33661|AAAAAAAANHDIAAAA|651|Broadway |RD|Suite 60|Crossroads|Lynchburg city|VA|20534|United States|-5|single family| +33662|AAAAAAAAOHDIAAAA|747|12th |Way|Suite 430|Five Forks|Washington County|IN|42293|United States|-5|apartment| +33663|AAAAAAAAPHDIAAAA|644|Railroad |Road|Suite A|Hamilton|Kossuth County|IA|52808|United States|-6|single family| +33664|AAAAAAAAAIDIAAAA|531|West College|RD|Suite P|Maple Grove|Forrest County|MS|58252|United States|-6|single family| +33665|AAAAAAAABIDIAAAA|306|1st Walnut|Road|Suite 360|Springdale|Camden County|MO|68883|United States|-6|condo| +33666|AAAAAAAACIDIAAAA|20|Williams Cherry|Ct.|Suite Y|Farmington|Surry County|NC|29145|United States|-5|apartment| +33667|AAAAAAAADIDIAAAA|960|10th Wilson|Avenue|Suite P|Oak Hill|Keith County|NE|67838|United States|-7|single family| +33668|AAAAAAAAEIDIAAAA|577|Second |Way|Suite 110|Wilson|Accomack County|VA|26971|United States|-5|condo| +33669|AAAAAAAAFIDIAAAA|639|South Walnut|Ave|Suite 100|Five Forks|Buffalo County|WI|52293|United States|-6|apartment| +33670|AAAAAAAAGIDIAAAA|988|Ash Lake|Road|Suite O|Sulphur Springs|Dale County|AL|38354|United States|-6|apartment| +33671|AAAAAAAAHIDIAAAA|253|4th Jackson|Cir.|Suite N|Tremont|Upshur County|TX|79515|United States|-6|apartment| +33672|AAAAAAAAIIDIAAAA|935|3rd 7th|Pkwy|Suite K|Fox|Boone County|IN|40631|United States|-5|apartment| +33673|AAAAAAAAJIDIAAAA|507|Johnson |Blvd|Suite 190|Liberty|Waller County|TX|73451|United States|-6|condo| +33674|AAAAAAAAKIDIAAAA|686|1st Twelfth|Ave|Suite H|Pleasant Valley|Roger Mills County|OK|72477|United States|-6|condo| +33675|AAAAAAAALIDIAAAA|24|3rd |Lane|Suite C|Pine Grove|Northampton County|NC|24593|United States|-5|apartment| +33676|AAAAAAAAMIDIAAAA|342|Johnson Washington|Way|Suite 300|Glendale|Manatee County|FL|33951|United States|-5|condo| +33677|AAAAAAAANIDIAAAA|589|Sixth 4th|Parkway|Suite 90|Murphy|Washington County|NC|22105|United States|-5|condo| +33678|AAAAAAAAOIDIAAAA||Sunset Ridge||||Crawford County||33451|United States||condo| +33679|AAAAAAAAPIDIAAAA|72|||Suite K|Fairfield||MO|66192|United States|-6|| +33680|AAAAAAAAAJDIAAAA|938|Davis |Road|Suite I|Riverdale|Pike County|IN|49391|United States|-5|apartment| +33681|AAAAAAAABJDIAAAA|266|Hickory Lincoln|ST|Suite B|Jamestown|Wilkinson County|GA|36867|United States|-5|apartment| +33682|AAAAAAAACJDIAAAA|784|Broadway Ridge|Avenue|Suite A|New Hope|Texas County|OK|79431|United States|-6|apartment| +33683|AAAAAAAADJDIAAAA|32|Lake 8th|RD|Suite 420|Oak Hill|Bristol County|MA|08438|United States|-5|apartment| +33684|AAAAAAAAEJDIAAAA|3|Railroad |Lane|Suite 330|Riverdale|Cleveland County|NC|29391|United States|-5|single family| +33685|AAAAAAAAFJDIAAAA|674|First Chestnut|Drive|Suite K|Cedar Grove|De Soto Parish|LA|70411|United States|-6|apartment| +33686|AAAAAAAAGJDIAAAA|416|14th Washington|Ave|Suite Y|Hidden Valley|Floyd County|IA|55521|United States|-6|apartment| +33687|AAAAAAAAHJDIAAAA|940|Meadow |Court|Suite 380|Oak Ridge|Pontotoc County|MS|58371|United States|-6|condo| +33688|AAAAAAAAIJDIAAAA|125|Church Mill|Road|Suite J|White Oak|Lincoln Parish|LA|76668|United States|-6|condo| +33689|AAAAAAAAJJDIAAAA|627|Laurel |Lane|Suite A|Liberty|Bryan County|OK|73451|United States|-6|single family| +33690|AAAAAAAAKJDIAAAA|166|4th |Avenue|Suite D|Fox|Bullitt County|KY|40631|United States|-6|apartment| +33691|AAAAAAAALJDIAAAA|76|2nd 4th|Ln|Suite 360|Woodlawn|Ramsey County|MN|54098|United States|-6|single family| +33692|AAAAAAAAMJDIAAAA|889|1st |Ct.|Suite I|Spring Hill|Childress County|TX|76787|United States|-6|condo| +33693|AAAAAAAANJDIAAAA|977|Pine |Drive|Suite 440|Newport|Issaquena County|MS|51521|United States|-6|condo| +33694|AAAAAAAAOJDIAAAA|960|Cherry |Parkway|Suite S|Fairfield|Hampshire County|MA|06792|United States|-5|condo| +33695|AAAAAAAAPJDIAAAA|125|Jefferson 4th|Ave|Suite Q|Woodland|Montgomery County|TX|74854|United States|-6|condo| +33696|AAAAAAAAAKDIAAAA|946|Green Cherry|Pkwy|Suite 60|Unionville|Wolfe County|KY|41711|United States|-5|condo| +33697|AAAAAAAABKDIAAAA|777|Second |Boulevard|Suite 10|Lakewood|Jackson County|TX|78877|United States|-6|single family| +33698|AAAAAAAACKDIAAAA|837|Church Oak|Avenue|Suite 160|Oakdale|Karnes County|TX|79584|United States|-6|condo| +33699|AAAAAAAADKDIAAAA|580|Willow |Pkwy|Suite 330|Jamestown|Cole County|MO|66867|United States|-6|apartment| +33700|AAAAAAAAEKDIAAAA|262|View |Ct.|Suite 470|Forest Hills|Currituck County|NC|29237|United States|-5|apartment| +33701|AAAAAAAAFKDIAAAA|582|2nd |RD|Suite M|Buena Vista|Hampton city|VA|25752|United States|-5|condo| +33702|AAAAAAAAGKDIAAAA|312|Chestnut |Boulevard|Suite G|Unionville|Walthall County|MS|51711|United States|-6|apartment| +33703|AAAAAAAAHKDIAAAA|88|Jackson |Boulevard|Suite J|Centerville|Coffee County|TN|30059|United States|-5|single family| +33704|AAAAAAAAIKDIAAAA|482|Highland Smith|Lane|Suite 240|Jamestown|Keokuk County|IA|56867|United States|-6|condo| +33705|AAAAAAAAJKDIAAAA|213|Broadway Spruce|Street|Suite 230|Pleasant Grove|Bath County|KY|44136|United States|-6|condo| +33706|AAAAAAAAKKDIAAAA|545|4th |Dr.|Suite 370|Oakwood|Douglas County|MN|50169|United States|-6|single family| +33707|AAAAAAAALKDIAAAA|930|Maple |Lane|Suite L|Shady Grove|Logan County|IL|62812|United States|-6|single family| +33708|AAAAAAAAMKDIAAAA|803|Franklin |Dr.|Suite 50|Spring Hill|Fulton County|OH|46787|United States|-5|condo| +33709|AAAAAAAANKDIAAAA|374|Ash Jefferson|Street|Suite 260|San Jose|Saguache County|CO|88003|United States|-7|apartment| +33710|AAAAAAAAOKDIAAAA|815|Third Elm|Pkwy|Suite L|Wildwood|Fredericksburg city|VA|26871|United States|-5|apartment| +33711|AAAAAAAAPKDIAAAA|502|Forest Washington|Street|Suite C|Deerfield|Nottoway County|VA|29840|United States|-5|single family| +33712|AAAAAAAAALDIAAAA|824|9th |Ave|Suite S|Springfield|King William County|VA|29303|United States|-5|condo| +33713|AAAAAAAABLDIAAAA|235|College |Avenue|Suite 10|Cedar Grove|Treasure County|MT|60411|United States|-7|condo| +33714|AAAAAAAACLDIAAAA|470|Pine Chestnut|Lane|Suite N|Concord|Beltrami County|MN|54107|United States|-6|single family| +33715|AAAAAAAADLDIAAAA||||Suite 220||||66098|United States||| +33716|AAAAAAAAELDIAAAA|65|2nd Johnson|Way|Suite 460|Glendale|Crockett County|TX|73951|United States|-6|apartment| +33717|AAAAAAAAFLDIAAAA|76|Seventh |Wy|Suite D|Mount Zion|Wagoner County|OK|78054|United States|-6|single family| +33718|AAAAAAAAGLDIAAAA|155|7th 3rd|Avenue|Suite E|Clinton|Tunica County|MS|58222|United States|-6|apartment| +33719|AAAAAAAAHLDIAAAA|45|Cherry |Blvd|Suite 0|Farmington|Osage County|OK|79145|United States|-6|apartment| +33720|AAAAAAAAILDIAAAA|669|Fourth Park|Wy|Suite M|Riverdale|Yavapai County|AZ|89391|United States|-7|condo| +33721|AAAAAAAAJLDIAAAA|146|Forest Spring|RD|Suite N|Scottsville|Sullivan County|PA|14190|United States|-5|apartment| +33722|AAAAAAAAKLDIAAAA|300|9th |Ave|Suite 490|Lincoln|Howard County|IA|51289|United States|-6|condo| +33723|AAAAAAAALLDIAAAA|456|Laurel |Ave|Suite 280|Greenwood|Shenandoah County|VA|28828|United States|-5|apartment| +33724|AAAAAAAAMLDIAAAA|697|View Miller|Ln|Suite 320|Five Points|Morris County|NJ|06698|United States|-5|condo| +33725|AAAAAAAANLDIAAAA|268|Oak |Dr.|Suite 360|Walnut Grove|Grainger County|TN|37752|United States|-5|apartment| +33726|AAAAAAAAOLDIAAAA|991|Hill |ST|Suite 50|Woodlawn|Graham County|KS|64098|United States|-6|apartment| +33727|AAAAAAAAPLDIAAAA|319|Cedar |Street|Suite D|Harmony|Park County|CO|85804|United States|-7|condo| +33728|AAAAAAAAAMDIAAAA|588|South Second|Pkwy|Suite G|Union|Logan County|OH|48721|United States|-5|condo| +33729|AAAAAAAABMDIAAAA|824|Maple 10th|Ct.|Suite 240|Shiloh|Parke County|IN|49275|United States|-5|condo| +33730|AAAAAAAACMDIAAAA|542|Park |Road|Suite 420|Oak Hill|Jackson County|MS|57838|United States|-6|single family| +33731|AAAAAAAADMDIAAAA|300|13th |Drive|Suite I|Union Hill|Jackson County|TX|77746|United States|-6|apartment| +33732|AAAAAAAAEMDIAAAA|562|Hill |Pkwy|Suite W|Summit|Stone County|AR|70499|United States|-6|condo| +33733|AAAAAAAAFMDIAAAA|820|12th |Circle|Suite 380|Hillcrest|Wilkes County|GA|33003|United States|-5|single family| +33734|AAAAAAAAGMDIAAAA|45|Seventh River|Lane|Suite 370|Friendship|Piatt County|IL|64536|United States|-6|apartment| +33735|AAAAAAAAHMDIAAAA|116|Johnson |Ave|Suite T|Newport|Maui County|HI|91521|United States|-10|apartment| +33736|AAAAAAAAIMDIAAAA|201|5th 6th|Pkwy|Suite 100|Gum Springs|Camden County|NC|22106|United States|-5|single family| +33737|AAAAAAAAJMDIAAAA|387|7th |ST|Suite 70|Concord|Lyon County|NV|84107|United States|-8|apartment| +33738|AAAAAAAAKMDIAAAA|763|Elm |Street|Suite P|Providence|Dorchester County|SC|26614|United States|-5|condo| +33739|AAAAAAAALMDIAAAA|600|Cedar Davis|Pkwy|Suite Y|Riverview|Terrell County|TX|79003|United States|-6|apartment| +33740|AAAAAAAAMMDIAAAA|628|Meadow |Way|Suite R|Valley View|Pawnee County|NE|65124|United States|-7|single family| +33741|AAAAAAAANMDIAAAA|844|Third Main|Cir.|Suite D|Jackson|Gilliam County|OR|99583|United States|-8|single family| +33742|AAAAAAAAOMDIAAAA|450|Washington |Circle|Suite 330|Bloomingdale|Comanche County|KS|61824|United States|-6|condo| +33743|AAAAAAAAPMDIAAAA|941|13th |Circle|Suite 410|Crossroads|Bamberg County|SC|20534|United States|-5|apartment| +33744|AAAAAAAAANDIAAAA|572|Hill |ST|Suite Q|Centerville|Conway County|AR|70059|United States|-6|single family| +33745|AAAAAAAABNDIAAAA|393|Main Ash|ST|Suite O|Highland|Clark County|AR|79454|United States|-6|apartment| +33746|AAAAAAAACNDIAAAA|775|Cedar Green|Boulevard|Suite 100|Spring Valley|Madison County|NY|16060|United States|-5|condo| +33747|AAAAAAAADNDIAAAA|661|Hickory Lincoln|Ln|Suite W|Bethel|Carteret County|NC|25281|United States|-5|single family| +33748|AAAAAAAAENDIAAAA|216|East Oak|Street|Suite H|Oakland|Williamsburg County|SC|29843|United States|-5|single family| +33749|AAAAAAAAFNDIAAAA|201|Highland |Lane|Suite W|Valley View|Washington County|KS|65124|United States|-6|single family| +33750|AAAAAAAAGNDIAAAA|371|Elm Highland|Street|Suite 400|Oakland|Sarasota County|FL|39843|United States|-5|apartment| +33751|AAAAAAAAHNDIAAAA|768|Pine |Avenue|Suite K|Clifton|Milwaukee County|WI|58014|United States|-6|single family| +33752|AAAAAAAAINDIAAAA|37|Hill Center|Avenue|Suite C|Five Points|Mineral County|CO|86098|United States|-7|single family| +33753|AAAAAAAAJNDIAAAA|345|Railroad View|Ave|Suite I|Jackson|Irwin County|GA|39583|United States|-5|single family| +33754|AAAAAAAAKNDIAAAA|831|Jefferson |Ln|Suite 290|Waterloo|Montgomery County|AR|71675|United States|-6|condo| +33755|AAAAAAAALNDIAAAA|232|Hickory |Circle|Suite 310|Summit|Warren County|MO|60499|United States|-6|condo| +33756|AAAAAAAAMNDIAAAA|902|Miller 2nd|RD|Suite Q|Lakeside|Coosa County|AL|39532|United States|-6|apartment| +33757|AAAAAAAANNDIAAAA|889|Second Maple|Lane|Suite 270|Glenville|Woodward County|OK|73445|United States|-6|condo| +33758|AAAAAAAAONDIAAAA|762|7th Johnson|Pkwy|Suite 190|Greenville|Franklin County|WA|91387|United States|-8|condo| +33759|AAAAAAAAPNDIAAAA|819|2nd |Ave|Suite U|White Oak|Union Parish|LA|76668|United States|-6|apartment| +33760|AAAAAAAAAODIAAAA|531|7th |Wy|Suite 260|Spring Valley|Stanley County|SD|56060|United States|-7|condo| +33761|AAAAAAAABODIAAAA|214|Willow |Court|Suite 80|Fairfield|Cole County|MO|66192|United States|-6|condo| +33762|AAAAAAAACODIAAAA|624|Maple Wilson|Ln|Suite M|Midway|Chenango County|NY|11904|United States|-5|single family| +33763|AAAAAAAADODIAAAA|602|Sunset Smith|Court|Suite Y|Highland Park|Bradford County|PA|16534|United States|-5|apartment| +33764|AAAAAAAAEODIAAAA|597|Main 10th|Avenue|Suite 430|Antioch|Dodge County|WI|58605|United States|-6|condo| +33765|AAAAAAAAFODIAAAA|605|Smith |ST|Suite 460|Glenwood|Kiowa County|CO|83511|United States|-7|apartment| +33766|AAAAAAAAGODIAAAA|456|Cherry |Drive|Suite M|Macedonia|Uinta County|WY|81087|United States|-7|single family| +33767|AAAAAAAAHODIAAAA|266|Birch |Ave|Suite V|Clifford|Nelson County|ND|58164|United States|-6|single family| +33768|AAAAAAAAIODIAAAA|593|1st Church|Ct.|Suite G|Granite|Oconee County|GA|36284|United States|-5|single family| +33769|AAAAAAAAJODIAAAA|203|Church Hickory|Circle|Suite 130|Florence|Jim Hogg County|TX|73394|United States|-6|condo| +33770|AAAAAAAAKODIAAAA|27|4th Lake|Parkway|Suite 260|Howell|Crawford County|MO|64854|United States|-6|condo| +33771|AAAAAAAALODIAAAA|955|8th Center|ST|Suite 400|Union Hill|Union County|GA|37746|United States|-5|condo| +33772|AAAAAAAAMODIAAAA|944|Cedar |Circle|Suite S|Jamestown|Boundary County|ID|86867|United States|-7|single family| +33773|AAAAAAAANODIAAAA|641|3rd Broadway|Way|Suite L|Mountain View|Rowan County|KY|44466|United States|-5|single family| +33774|AAAAAAAAOODIAAAA|60|7th Williams|Parkway|Suite K|Lakeville|Brown County|OH|48811|United States|-5|single family| +33775|AAAAAAAAPODIAAAA|602|Central Poplar|Street|Suite 490|Greenwood|Frederick County|MD|28828|United States|-5|condo| +33776|AAAAAAAAAPDIAAAA|788|Poplar |Way|Suite F|Mount Olive|Johnson County|TN|38059|United States|-6|apartment| +33777|AAAAAAAABPDIAAAA|692|Jefferson 3rd|Ave|Suite M|Shady Grove|Cherokee County|AL|32812|United States|-6|condo| +33778|AAAAAAAACPDIAAAA|764|Hickory |Ct.|Suite O|Florence|Page County|VA|23394|United States|-5|condo| +33779|AAAAAAAADPDIAAAA|808|First View|RD|Suite L|Antioch|Isle of Wight County|VA|28605|United States|-5|single family| +33780|AAAAAAAAEPDIAAAA|461|Madison South|Wy|Suite 180|Friendship|Yamhill County|OR|94536|United States|-8|condo| +33781|AAAAAAAAFPDIAAAA|660|Cedar |Ct.|Suite I|Pinhook|Baraga County|MI|49398|United States|-5|apartment| +33782|AAAAAAAAGPDIAAAA|717|4th |Dr.|Suite 490|Highland|Golden Valley County|ND|59454|United States|-6|apartment| +33783|AAAAAAAAHPDIAAAA|174|Lincoln |RD|Suite 400|Waterloo|Mora County|NM|81675|United States|-7|single family| +33784|AAAAAAAAIPDIAAAA|443|Cedar Park|Lane|Suite O|Birmingham|Waynesboro city|VA|23372|United States|-5|condo| +33785|AAAAAAAAJPDIAAAA|459|Fourth Twelfth|Ln|Suite 340|Lebanon|Lake County|MT|62898|United States|-7|apartment| +33786|AAAAAAAAKPDIAAAA|421|3rd Hill|Street|Suite S|Woodville|Gunnison County|CO|84289|United States|-7|condo| +33787|AAAAAAAALPDIAAAA|441|Cherry Second|Ct.|Suite 230|Enterprise|Coos County|OR|91757|United States|-8|condo| +33788|AAAAAAAAMPDIAAAA|143|Park Wilson|Court|Suite I|Bethel|Madison County|AR|75281|United States|-6|condo| +33789|AAAAAAAANPDIAAAA|472|10th |Lane|Suite 460|Shiloh|Elmore County|AL|39275|United States|-6|condo| +33790|AAAAAAAAOPDIAAAA|566|Lake Fifth|Court|Suite 260|Oakwood|Emmons County|ND|50169|United States|-6|apartment| +33791|AAAAAAAAPPDIAAAA|33|Pine |Circle|Suite 170|Newport|Gloucester County|VA|21521|United States|-5|condo| +33792|AAAAAAAAAAEIAAAA|992|3rd Highland|Drive|Suite 50|Bethesda|Hardin County|IL|65980|United States|-6|single family| +33793|AAAAAAAABAEIAAAA|198|7th |Cir.|Suite R|Jamestown|Stark County|ND|56867|United States|-6|apartment| +33794|AAAAAAAACAEIAAAA|77|Jackson Maple|ST|Suite P|Ashland|Hartford County|CT|04844|United States|-5|apartment| +33795|AAAAAAAADAEIAAAA|131|Forest |Drive|Suite 0|White Oak|Dooly County|GA|36668|United States|-5|single family| +33796|AAAAAAAAEAEIAAAA|190|North |Wy|Suite 250|Summit|Cerro Gordo County|IA|50499|United States|-6|apartment| +33797|AAAAAAAAFAEIAAAA|778|15th |Lane|Suite 150|Oak Hill|Monroe County|MS|57838|United States|-6|apartment| +33798|AAAAAAAAGAEIAAAA|779|Fourth Miller|ST|Suite P|Lakeview|Penobscot County|ME|09179|United States|-5|apartment| +33799|AAAAAAAAHAEIAAAA|913|5th |Avenue|Suite N|Lakeside|Woodford County|KY|49532|United States|-6|single family| +33800|AAAAAAAAIAEIAAAA|360|5th |Boulevard|Suite L|Oak Grove|Middlesex County|CT|08970|United States|-5|single family| +33801|AAAAAAAAJAEIAAAA|198|Pine |Wy|Suite 130|Mountain View|Madison County|TN|34466|United States|-6|single family| +33802|AAAAAAAAKAEIAAAA|878|Johnson Fourth|RD|Suite 470|Harmony|Box Butte County|NE|65804|United States|-6|apartment| +33803|AAAAAAAALAEIAAAA|792|East |Parkway|Suite 70|Unionville|Richmond city|VA|21711|United States|-5|condo| +33804|AAAAAAAAMAEIAAAA|364|Central Green|Parkway|Suite T|Sugar Hill|Switzerland County|IN|45114|United States|-5|apartment| +33805|AAAAAAAANAEIAAAA|614|Fifth |RD|Suite N|Antioch|Clay County|KY|48605|United States|-6|condo| +33806|AAAAAAAAOAEIAAAA|591|4th |Ct.|Suite H|Phoenix|Shawnee County|KS|62276|United States|-6|apartment| +33807|AAAAAAAAPAEIAAAA|771|Park |RD|Suite 200|Florence|Grant County|ND|53394|United States|-6|condo| +33808|AAAAAAAAABEIAAAA|510|Broadway Center|Dr.|Suite 400|Enterprise|Spartanburg County|SC|21757|United States|-5|apartment| +33809|AAAAAAAABBEIAAAA|214|Washington Mill|Lane|Suite S|Liberty|Carroll County|GA|33451|United States|-5|condo| +33810|AAAAAAAACBEIAAAA|886|Cedar Birch|Ln|Suite A|Lakeside|Noble County|OH|49532|United States|-5|apartment| +33811|AAAAAAAADBEIAAAA|31|Mill |RD|Suite 370|Ashland|Mingo County|WV|24244|United States|-5|condo| +33812|AAAAAAAAEBEIAAAA|498|2nd |Dr.|Suite S|Providence|Andrew County|MO|66614|United States|-6|condo| +33813|AAAAAAAAFBEIAAAA|88|9th Hickory|Avenue|Suite D|Spring Hill|Brown County|WI|56787|United States|-6|apartment| +33814|AAAAAAAAGBEIAAAA|881|4th Tenth|ST|Suite G|Buena Vista|Clallam County|WA|95752|United States|-8|single family| +33815|AAAAAAAAHBEIAAAA|483|Walnut |Lane|Suite Y|Franklin|Ripley County|MO|69101|United States|-6|apartment| +33816|AAAAAAAAIBEIAAAA|541|Johnson 11th|Street|Suite 40|Farmington|Johnson County|KY|49145|United States|-6|condo| +33817|AAAAAAAAJBEIAAAA|134|Lee |Ln|Suite M|Bridgeport|Robeson County|NC|25817|United States|-5|apartment| +33818|AAAAAAAAKBEIAAAA|40|Park Center|Avenue|Suite 160|Greenwood|Washington County|OR|98828|United States|-8|single family| +33819|AAAAAAAALBEIAAAA|480|Church |Circle|Suite S|Sulphur Springs|Webster County|WV|28354|United States|-5|single family| +33820|AAAAAAAAMBEIAAAA|700|Willow 5th|Parkway|Suite W|Newtown|Stonewall County|TX|71749|United States|-6|single family| +33821|AAAAAAAANBEIAAAA|802|Willow |Ave|Suite 460|Brownsville|Grayson County|VA|29310|United States|-5|apartment| +33822|AAAAAAAAOBEIAAAA|667|Green Cedar|Avenue|Suite X|Fairfield|Roberts County|TX|76192|United States|-6|apartment| +33823|AAAAAAAAPBEIAAAA|756|Johnson |Pkwy|Suite 240|Sheffield|Hunterdon County|NJ|07496|United States|-5|condo| +33824|AAAAAAAAACEIAAAA|277|Railroad Chestnut|Ln|Suite 290|White Oak|Ferry County|WA|96668|United States|-8|condo| +33825|AAAAAAAABCEIAAAA|582|Jackson |Drive|Suite 380|Stringtown|Huron County|OH|40162|United States|-5|single family| +33826|AAAAAAAACCEIAAAA|317|Fourth |Circle|Suite Q|Antioch|Hamilton County|IN|48605|United States|-5|apartment| +33827|AAAAAAAADCEIAAAA|865|Pine |Boulevard|Suite E|Pleasant Hill|Adair County|MO|63604|United States|-6|condo| +33828|AAAAAAAAECEIAAAA|895|Church |ST|Suite 80|Tanglewood|Nolan County|TX|78994|United States|-6|single family| +33829|AAAAAAAAFCEIAAAA|553|Spruce |Street|Suite B|Leland|Terrell County|TX|79452|United States|-6|condo| +33830|AAAAAAAAGCEIAAAA|37|Poplar Main|Road|Suite M|Cedar Grove|Pulaski County|MO|60411|United States|-6|condo| +33831|AAAAAAAAHCEIAAAA|810|10th Lincoln|Court|Suite 420|Greenville|Burnet County|TX|71387|United States|-6|condo| +33832|AAAAAAAAICEIAAAA|957|Williams 2nd|Pkwy|Suite 490|Forest Hills|Scioto County|OH|49237|United States|-5|condo| +33833|AAAAAAAAJCEIAAAA|401|Maple 3rd|Ave|Suite 260|Riverview|Taylor County|GA|39003|United States|-5|apartment| +33834|AAAAAAAAKCEIAAAA|957|Lincoln |Boulevard|Suite 290|Green Acres|Carroll County|MD|27683|United States|-5|single family| +33835|AAAAAAAALCEIAAAA|873|Meadow 1st|Ln|Suite V|Farmington|Hampton County|SC|29145|United States|-5|single family| +33836|AAAAAAAAMCEIAAAA|700|7th Hillcrest|Ln|Suite 360|Five Forks|Sherman County|TX|72293|United States|-6|condo| +33837|AAAAAAAANCEIAAAA|231|Cherry Pine|Way|Suite 330|Wilson|Watauga County|NC|26971|United States|-5|apartment| +33838|AAAAAAAAOCEIAAAA|406|Hill Elm|Dr.|Suite 0|Highland|Price County|WI|59454|United States|-6|condo| +33839|AAAAAAAAPCEIAAAA|431|Hickory 7th|Ln|Suite P|Spring Valley|Catahoula Parish|LA|76060|United States|-6|apartment| +33840|AAAAAAAAADEIAAAA|765|Laurel |Ave|Suite 240|Centerville|Colusa County|CA|90059|United States|-8|apartment| +33841|AAAAAAAABDEIAAAA|839|14th |Drive|Suite 270|Shiloh|Clay County|MO|69275|United States|-6|single family| +33842|AAAAAAAACDEIAAAA|524|Birch Dogwood|Drive|Suite T|Riverview|Sumter County|GA|39003|United States|-5|condo| +33843|AAAAAAAADDEIAAAA|874|1st |ST|Suite A|Florence|Colfax County|NM|83394|United States|-7|apartment| +33844|AAAAAAAAEDEIAAAA|368|1st Second|Road|Suite 360|Riverview|Clatsop County|OR|99003|United States|-8|apartment| +33845|AAAAAAAAFDEIAAAA|||||Oakdale|Lee County||79584||-6|single family| +33846|AAAAAAAAGDEIAAAA|119|Fifth |Blvd|Suite U|Oakwood|Overton County|TN|30169|United States|-6|single family| +33847|AAAAAAAAHDEIAAAA|334|9th |Road|Suite A|Five Forks|Lee County|AR|72293|United States|-6|condo| +33848|AAAAAAAAIDEIAAAA|480|3rd Center|Ave|Suite X|Jackson|Campbell County|VA|29583|United States|-5|single family| +33849|AAAAAAAAJDEIAAAA|555|Forest |Ave|Suite 170|Hopewell|Ouray County|CO|80587|United States|-7|single family| +33850|AAAAAAAAKDEIAAAA|954|Second |Pkwy|Suite H|Forest Hills|Park County|MT|69237|United States|-7|apartment| +33851|AAAAAAAALDEIAAAA|938|East Second|Way|Suite Q|Union Hill|Sussex County|VA|27746|United States|-5|single family| +33852|AAAAAAAAMDEIAAAA|385|9th Third|Ct.|Suite Q|Oakdale|Hudson County|NJ|09584|United States|-5|single family| +33853|AAAAAAAANDEIAAAA|731|Woodland |Ln|Suite K|Buena Vista|Vanderburgh County|IN|45752|United States|-5|single family| +33854|AAAAAAAAODEIAAAA|213|Ash |Ct.|Suite J|Union|Taylor County|KY|48721|United States|-5|apartment| +33855|AAAAAAAAPDEIAAAA|690|Johnson Washington|Lane|Suite W|Edgewood|Tippah County|MS|50069|United States|-6|apartment| +33856|AAAAAAAAAEEIAAAA|796|First Johnson|Dr.|Suite M|Lakewood|Baca County|CO|88877|United States|-7|condo| +33857|AAAAAAAABEEIAAAA|893|Maple Oak|Circle|Suite U|Midway|Collier County|FL|31904|United States|-5|apartment| +33858|AAAAAAAACEEIAAAA|1000|Oak |ST|Suite N|Lakeview|Chenango County|NY|18579|United States|-5|apartment| +33859|AAAAAAAADEEIAAAA|665|8th |Cir.|Suite N|Oakland|Blaine County|OK|79843|United States|-6|single family| +33860|AAAAAAAAEEEIAAAA|923|First Ash|Way|Suite 110|Buena Vista|Olmsted County|MN|55752|United States|-6|single family| +33861|AAAAAAAAFEEIAAAA|243|Main Church|Pkwy|Suite 280|Providence|Herkimer County|NY|16614|United States|-5|condo| +33862|AAAAAAAAGEEIAAAA|330|11th |Street|Suite 300|Freeport|Wirt County|WV|21844|United States|-5|apartment| +33863|AAAAAAAAHEEIAAAA|212|Pine |Pkwy|Suite 320|Cedar Grove|Brown County|TX|70411|United States|-6|apartment| +33864|AAAAAAAAIEEIAAAA|208|Cherry Williams|Ln|Suite G|Springfield|Aitkin County|MN|59303|United States|-6|condo| +33865|AAAAAAAAJEEIAAAA|150|Mill Dogwood|ST|Suite L|Wilson|Howard County|AR|76971|United States|-6|apartment| +33866|AAAAAAAAKEEIAAAA|898|Cedar Madison|Boulevard|Suite 20|Glenwood|Coffee County|GA|33511|United States|-5|condo| +33867|AAAAAAAALEEIAAAA|574|Eigth Ash|Dr.|Suite 470|Cordova|Coles County|IL|66938|United States|-6|single family| +33868|AAAAAAAAMEEIAAAA|609|South |Cir.|Suite E|Cedar Grove|Dixie County|FL|30411|United States|-5|single family| +33869|AAAAAAAANEEIAAAA|888|East |Way|Suite A|Mount Pleasant|Marshall County|OK|71933|United States|-6|apartment| +33870|AAAAAAAAOEEIAAAA|907|10th |Avenue|Suite X|Deerfield|Luna County|NM|89840|United States|-7|condo| +33871|AAAAAAAAPEEIAAAA|754|Williams West|Wy|Suite W|Bunker Hill|Lincoln County|CO|80150|United States|-7|apartment| +33872|AAAAAAAAAFEIAAAA|806|West |Street|Suite E|Oakwood|Stevens County|KS|60169|United States|-6|single family| +33873|AAAAAAAABFEIAAAA|720|Hill Walnut|Road|Suite C|Glendale|Clermont County|OH|43951|United States|-5|condo| +33874|AAAAAAAACFEIAAAA|195|Main Main|Pkwy|Suite G|Hillcrest|Chase County|KS|63003|United States|-6|apartment| +33875|AAAAAAAADFEIAAAA|59|Cedar Maple|Road|Suite K|Nichols|Polk County|WI|57940|United States|-6|condo| +33876|AAAAAAAAEFEIAAAA|696|Fifteenth |Ave|Suite 180|Shady Grove|Decatur County|GA|32812|United States|-5|apartment| +33877|AAAAAAAAFFEIAAAA|280|Smith |Circle|Suite J|West Liberty|Houston County|AL|34752|United States|-6|apartment| +33878|AAAAAAAAGFEIAAAA|641|Cherry |Cir.|Suite H|Sulphur Springs|Kandiyohi County|MN|58354|United States|-6|apartment| +33879|AAAAAAAAHFEIAAAA|419|Central Hickory|Lane|Suite H|Pleasant Hill|Calhoun County|AL|33604|United States|-6|single family| +33880|AAAAAAAAIFEIAAAA|742|Jackson |Cir.|Suite 450|Union Hill|Barnes County|ND|57746|United States|-6|single family| +33881|AAAAAAAAJFEIAAAA|798|Fourth Main|Ln|Suite 210|Oakdale|Caldwell County|KY|49584|United States|-6|condo| +33882|AAAAAAAAKFEIAAAA|993|Hill Forest|Road|Suite 230|White Oak|Hyde County|NC|26668|United States|-5|apartment| +33883|AAAAAAAALFEIAAAA|202|3rd |Blvd|Suite 340|Centerville|Atoka County|OK|70059|United States|-6|apartment| +33884|AAAAAAAAMFEIAAAA|432|Franklin Elm|Circle|Suite X|Shiloh|Cumberland County|VA|29275|United States|-5|condo| +33885|AAAAAAAANFEIAAAA|575|Walnut |Cir.|Suite V|Highland Park|Prentiss County|MS|56534|United States|-6|condo| +33886|AAAAAAAAOFEIAAAA|559|7th 9th|Street|Suite T|Edgewood|Decatur County|GA|30069|United States|-5|condo| +33887|AAAAAAAAPFEIAAAA|481|1st |Wy|Suite N|Morris|Laclede County|MO|66696|United States|-6|single family| +33888|AAAAAAAAAGEIAAAA|509|Maple |Boulevard|Suite 90|Jackson|Falls County|TX|79583|United States|-6|condo| +33889|AAAAAAAABGEIAAAA|957|Elm |Road|Suite 240|Harmony|Wayne County|WV|25804|United States|-5|apartment| +33890|AAAAAAAACGEIAAAA|501|South |Ave|Suite O|Clifton|Gilpin County|CO|88014|United States|-7|apartment| +33891|AAAAAAAADGEIAAAA|102|9th |Ct.|Suite 310|Greenwood|Smith County|MS|58828|United States|-6|condo| +33892|AAAAAAAAEGEIAAAA|315|Lincoln |Dr.|Suite 310|Golden|Columbia County|FL|30411|United States|-5|apartment| +33893|AAAAAAAAFGEIAAAA|589|Oak Oak|Wy|Suite 80|Georgetown|Alger County|MI|47057|United States|-5|single family| +33894|AAAAAAAAGGEIAAAA|690|Ridge |Circle|Suite L|Red Hill|Cedar County|NE|64338|United States|-6|single family| +33895|AAAAAAAAHGEIAAAA|348|Cherry |Dr.|Suite 110|Lakeside|Allen County|IN|49532|United States|-5|single family| +33896|AAAAAAAAIGEIAAAA|756|Green Jefferson|Drive|Suite J|Red Hill|Campbell County|VA|24338|United States|-5|apartment| +33897|AAAAAAAAJGEIAAAA|91|Lake |Wy|Suite 200|Enterprise|Granite County|MT|61757|United States|-7|single family| +33898|AAAAAAAAKGEIAAAA|144|4th |ST|Suite B|Shiloh|Cumberland County|PA|19275|United States|-5|single family| +33899|AAAAAAAALGEIAAAA|44|Railroad |Avenue|Suite 380|Providence|Graham County|AZ|86614|United States|-7|condo| +33900|AAAAAAAAMGEIAAAA|9|Madison Pine|Circle|Suite Y|Sunnyside|Colfax County|NM|81952|United States|-7|condo| +33901|AAAAAAAANGEIAAAA|752|Sycamore |Blvd|Suite 400|Clifton|Scott County|VA|28014|United States|-5|single family| +33902|AAAAAAAAOGEIAAAA|111|Mill |ST|Suite K|Shady Grove|Harris County|GA|32812|United States|-5|single family| +33903|AAAAAAAAPGEIAAAA|147||Lane||Oakdale|Palm Beach County|FL||United States|-5|| +33904|AAAAAAAAAHEIAAAA|637|View Sixth|Avenue|Suite 230|Highland|DeWitt County|TX|79454|United States|-6|condo| +33905|AAAAAAAABHEIAAAA|919|Locust |Blvd|Suite L|Oak Ridge|Wabash County|IL|68371|United States|-6|condo| +33906|AAAAAAAACHEIAAAA|405|Madison 14th|RD|Suite Q|Fairfield|Adair County|OK|76192|United States|-6|single family| +33907|AAAAAAAADHEIAAAA|258|8th |Ave|Suite 30|Cedar Grove|Iowa County|IA|50411|United States|-6|condo| +33908|AAAAAAAAEHEIAAAA|26|5th Main|Ln|Suite 0|Lenox|Shelby County|TN|31143|United States|-6|apartment| +33909|AAAAAAAAFHEIAAAA|823|Miller |ST|Suite T|Salem|Grant County|KY|48048|United States|-6|apartment| +33910|AAAAAAAAGHEIAAAA|131|Cherry 10th|RD|Suite 140|Greenville|Kankakee County|IL|61387|United States|-6|apartment| +33911|AAAAAAAAHHEIAAAA|742|4th West|Court|Suite I|Lincoln|Orange County|FL|31289|United States|-5|single family| +33912|AAAAAAAAIHEIAAAA|359|Sunset 2nd|Parkway|Suite K|Harmony|Clayton County|GA|35804|United States|-5|single family| +33913|AAAAAAAAJHEIAAAA|378|Mill |Ave|Suite T|Riverview|Craven County|NC|29003|United States|-5|condo| +33914|AAAAAAAAKHEIAAAA|65|Main |Parkway|Suite 380|Walnut Grove|Caddo County|OK|77752|United States|-6|apartment| +33915|AAAAAAAALHEIAAAA|38|6th |Dr.|Suite 430|Sawyer|Bland County|VA|26045|United States|-5|condo| +33916|AAAAAAAAMHEIAAAA|303|Jefferson View|Court|Suite 460|Glenwood|Harrison County|TX|73511|United States|-6|single family| +33917|AAAAAAAANHEIAAAA|827|North |Boulevard|Suite 140|Plainview|Aurora County|SD|53683|United States|-6|condo| +33918|AAAAAAAAOHEIAAAA|19|Davis Ridge|ST|Suite 230|Stafford|Pecos County|TX|74980|United States|-6|condo| +33919|AAAAAAAAPHEIAAAA|876|Forest ||Suite P||||||-8|| +33920|AAAAAAAAAIEIAAAA|634|Hill |Avenue|Suite G|Saint James|Simpson County|KY|45799|United States|-5|condo| +33921|AAAAAAAABIEIAAAA|280|First |RD|Suite 140|Arlington|Santa Cruz County|CA|96557|United States|-8|single family| +33922|AAAAAAAACIEIAAAA|213|Poplar Lake|ST|Suite A|Oakland|Wayne County|IA|59843|United States|-6|condo| +33923|AAAAAAAADIEIAAAA|984|Locust Mill|Parkway|Suite O|Oakwood|Mineral County|MT|60169|United States|-7|condo| +33924|AAAAAAAAEIEIAAAA|876|Walnut Second|Ave|Suite F|Harmony|Guilford County|NC|25804|United States|-5|apartment| +33925|AAAAAAAAFIEIAAAA|780|Park |Wy|Suite 420|Oak Grove|Cleburne County|AL|38370|United States|-6|condo| +33926|AAAAAAAAGIEIAAAA|828|Cherry Locust|Court|Suite 170|Bunker Hill|Sangamon County|IL|60150|United States|-6|condo| +33927|AAAAAAAAHIEIAAAA|475|1st Locust|Blvd|Suite N|Florence|Delaware County|NY|13394|United States|-5|condo| +33928|AAAAAAAAIIEIAAAA|630|11th Railroad|ST|Suite 80|Walnut Grove|Washington Parish|LA|77752|United States|-6|single family| +33929|AAAAAAAAJIEIAAAA|318|Third |Lane|Suite 350|Green Acres|Claiborne Parish|LA|77683|United States|-6|condo| +33930|AAAAAAAAKIEIAAAA|650||Ave|Suite 260|Salem|Lassen County|CA||||| +33931|AAAAAAAALIEIAAAA|826|Center |Avenue|Suite G|Midway|Lamb County|TX|71904|United States|-6|condo| +33932|AAAAAAAAMIEIAAAA|793|Hillcrest |Avenue|Suite 110|Cedar Grove|Lagrange County|IN|40411|United States|-5|apartment| +33933|AAAAAAAANIEIAAAA|810|Main West|||Greenville||MN|51387|United States|-6|single family| +33934|AAAAAAAAOIEIAAAA|33|10th |ST|Suite 300|Little River|Bedford city|VA|20319|United States|-5|single family| +33935|AAAAAAAAPIEIAAAA|426|First 9th|Parkway|Suite 400|Clinton|Jefferson County|TN|38222|United States|-6|single family| +33936|AAAAAAAAAJEIAAAA|903|2nd View|Ln|Suite P|Red Hill|Roberts County|SD|54338|United States|-7|apartment| +33937|AAAAAAAABJEIAAAA|471|Cedar Hill|Blvd|Suite W|Greenwood|Summers County|WV|28828|United States|-5|condo| +33938|AAAAAAAACJEIAAAA|617|Madison Park|Avenue|Suite 440|Woodville|Franklin County|IN|44289|United States|-5|apartment| +33939|AAAAAAAADJEIAAAA|447|Jackson |Dr.|Suite O|Brookwood|Franklin County|KS|60965|United States|-6|apartment| +33940|AAAAAAAAEJEIAAAA|576|College Wilson|Parkway|Suite O|Sunnyside|Breckinridge County|KY|41952|United States|-6|condo| +33941|AAAAAAAAFJEIAAAA|878|Fifth |Street|Suite 450|Springdale|Guthrie County|IA|58883|United States|-6|condo| +33942|AAAAAAAAGJEIAAAA|531|Hillcrest Sunset|Boulevard|Suite 100|Highland Park|Escambia County|FL|36534|United States|-5|condo| +33943|AAAAAAAAHJEIAAAA|740|Ash Park|RD|Suite D|Sunnyside|Woods County|OK|71952|United States|-6|single family| +33944|AAAAAAAAIJEIAAAA|920|Spring |Avenue|Suite D|Liberty|Navarro County|TX|73451|United States|-6|apartment| +33945|AAAAAAAAJJEIAAAA|907|Park Sixth|Ln|Suite 0|Marion|Barnes County|ND|50399|United States|-6|apartment| +33946|AAAAAAAAKJEIAAAA|59|Spruce |Road|Suite 310|White Oak|Pierce County|NE|66668|United States|-7|condo| +33947|AAAAAAAALJEIAAAA|982|Sixth West|Way|Suite Q|New Hope|Herkimer County|NY|19431|United States|-5|single family| +33948|AAAAAAAAMJEIAAAA|659|South East|Ct.|Suite 450|Antioch|Hodgeman County|KS|68605|United States|-6|apartment| +33949|AAAAAAAANJEIAAAA|654|Maple 3rd|Avenue|Suite L|Unionville|Ottawa County|OH|41711|United States|-5|condo| +33950|AAAAAAAAOJEIAAAA|703|Washington 13th|Ave|Suite K|Glenwood|Rosebud County|MT|63511|United States|-7|condo| +33951|AAAAAAAAPJEIAAAA|323|Meadow Ash|Ave|Suite 460|Lakeside|Haywood County|TN|39532|United States|-5|condo| +33952|AAAAAAAAAKEIAAAA|731|Pine |RD|Suite 460|Stringtown|DeSoto County|FL|30162|United States|-5|condo| +33953|AAAAAAAABKEIAAAA|446|Lee |Pkwy|Suite R|Plainview|Boyd County|NE|63683|United States|-6|condo| +33954|AAAAAAAACKEIAAAA|545|Mill |Parkway|Suite 260|Walnut Grove|Kent County|MI|47752|United States|-5|single family| +33955|AAAAAAAADKEIAAAA|911|9th View|Ln|Suite 230|Belmont|Burt County|NE|60191|United States|-6|condo| +33956|AAAAAAAAEKEIAAAA|508|Davis |Road|Suite Q|Oakland|Cuyahoga County|OH|49843|United States|-5|apartment| +33957|AAAAAAAAFKEIAAAA|577|Jackson |Way|Suite O|Mount Olive|Kearney County|NE|68059|United States|-7|single family| +33958|AAAAAAAAGKEIAAAA|42|2nd Lake|Drive|Suite 340|Bridgeport|Osborne County|KS|65817|United States|-6|single family| +33959|AAAAAAAAHKEIAAAA|151|Williams Dogwood|Lane|Suite 450|Cedar Grove|Wright County|MO|60411|United States|-6|apartment| +33960|AAAAAAAAIKEIAAAA|511|Park |Drive|Suite 80|Mountain View|Henry County|GA|34466|United States|-5|apartment| +33961|AAAAAAAAJKEIAAAA|22|Sunset |Boulevard|Suite 240|Pleasant Hill|Wayne County|MS|53604|United States|-6|apartment| +33962|AAAAAAAAKKEIAAAA|561|Highland |Parkway|Suite 460|Greenfield|Franklin County|IN|45038|United States|-5|condo| +33963|AAAAAAAALKEIAAAA|568|South |Ct.|Suite C|Hamilton|Pawnee County|OK|72808|United States|-6|condo| +33964|AAAAAAAAMKEIAAAA|27|Oak Sunset|Avenue|Suite 250|Crossroads|Armstrong County|PA|10534|United States|-5|condo| +33965|AAAAAAAANKEIAAAA||Spring |||||KY|46668|United States|-6|| +33966|AAAAAAAAOKEIAAAA|194|Valley |Cir.|Suite 410|Oakland|Mason County|TX|79843|United States|-6|condo| +33967|AAAAAAAAPKEIAAAA|587|2nd |Circle|Suite 20|Woodville|Jefferson County|ID|84289|United States|-7|condo| +33968|AAAAAAAAALEIAAAA|49|Hill |Boulevard|Suite 60|Springfield|Snyder County|PA|19303|United States|-5|apartment| +33969|AAAAAAAABLEIAAAA|343|North Lakeview|Parkway|Suite H|Woodcrest|Natchitoches Parish|LA|74919|United States|-6|condo| +33970|AAAAAAAACLEIAAAA|931|Elm |RD|Suite 380|Oak Grove|Harding County|SD|58370|United States|-7|single family| +33971|AAAAAAAADLEIAAAA|992|Cedar Ash|Ln|Suite 130|Valley View|Henry County|MO|65124|United States|-6|apartment| +33972|AAAAAAAAELEIAAAA|51|River Walnut|Cir.|Suite 30|Bunker Hill|Jefferson County|OR|90150|United States|-8|single family| +33973|AAAAAAAAFLEIAAAA|283|10th |Cir.|Suite J|Five Points|Gordon County|GA|36098|United States|-5|condo| +33974|AAAAAAAAGLEIAAAA|372|3rd Lincoln|Ct.|Suite E|Franklin|Spotsylvania County|VA|29101|United States|-5|apartment| +33975|AAAAAAAAHLEIAAAA|642|13th |Pkwy|Suite 300|Georgetown|Trumbull County|OH|47057|United States|-5|condo| +33976|AAAAAAAAILEIAAAA|263|Meadow Cherry|Circle|Suite 470|Lakeside|Callaway County|MO|69532|United States|-6|single family| +33977|AAAAAAAAJLEIAAAA|663|First |Ln|Suite N|Bunker Hill|Waller County|TX|70150|United States|-6|single family| +33978|AAAAAAAAKLEIAAAA|810|Lake |Cir.|Suite 270|Jackson|Vermillion County|IN|49583|United States|-5|condo| +33979|AAAAAAAALLEIAAAA|998|Main Oak|Dr.|Suite E|Newtown|Rowan County|KY|41749|United States|-5|condo| +33980|AAAAAAAAMLEIAAAA|230|North |Circle|Suite B|Riverview|Augusta County|VA|29003|United States|-5|apartment| +33981|AAAAAAAANLEIAAAA|89|Ash Maple|Ave|Suite L|Oak Grove|Buffalo County|SD|58370|United States|-6|condo| +33982|AAAAAAAAOLEIAAAA|917|Third |Circle|Suite Y|Antioch|Meriwether County|GA|38605|United States|-5|single family| +33983|AAAAAAAAPLEIAAAA|718|First Hickory|RD|Suite P|Ashley|Gila County|AZ|84324|United States|-7|single family| +33984|AAAAAAAAAMEIAAAA|15|Lake |ST|Suite F|Fisher|Greene County|AL|32819|United States|-6|single family| +33985|AAAAAAAABMEIAAAA|852|5th |Ct.|Suite 190|Clinton|Putnam County|MO|68222|United States|-6|single family| +33986|AAAAAAAACMEIAAAA|984|Ash First|RD|Suite 70|Union Hill|Anderson County|TN|37746|United States|-5|condo| +33987|AAAAAAAADMEIAAAA|732|Woodland Mill|RD|Suite 340|Rockwood|Washoe County|NV|81545|United States|-8|single family| +33988|AAAAAAAAEMEIAAAA|218|Ash |Blvd|Suite 350|Greenwood|Klamath County|OR|98828|United States|-8|condo| +33989|AAAAAAAAFMEIAAAA|687|Ridge |Lane|Suite 180|Cedar|Benton County|IA|51229|United States|-6|condo| +33990|AAAAAAAAGMEIAAAA|999|South |Wy|Suite X|Bridgeport|Lake County|SD|55817|United States|-7|apartment| +33991|AAAAAAAAHMEIAAAA|799|Jefferson Walnut|Way|Suite O|Green Acres|Iron County|UT|87683|United States|-7|apartment| +33992|AAAAAAAAIMEIAAAA|163|Hickory Dogwood|Avenue|Suite 360|Oakdale|McCurtain County|OK|79584|United States|-6|condo| +33993|AAAAAAAAJMEIAAAA|833|Ash |Parkway|Suite 290|Mount Olive|Washington Parish|LA|78059|United States|-6|single family| +33994|AAAAAAAAKMEIAAAA|685|Johnson |Court|Suite B|Lakeview|Jefferson County|PA|18579|United States|-5|apartment| +33995|AAAAAAAALMEIAAAA|100|Poplar |Parkway|||Carroll County|OH||United States|-5|condo| +33996|AAAAAAAAMMEIAAAA|608|Miller |Way|Suite 190|Maple Grove|Hood River County|OR|98252|United States|-8|condo| +33997|AAAAAAAANMEIAAAA|963|Third |Dr.|Suite T|Bridgeport|DeKalb County|MO|65817|United States|-6|single family| +33998|AAAAAAAAOMEIAAAA|958|Chestnut Lincoln|Pkwy|Suite 390|Liberty|Stokes County|NC|23451|United States|-5|apartment| +33999|AAAAAAAAPMEIAAAA|324|Forest Davis|Wy|Suite D|Sunnyside|Spokane County|WA|91952|United States|-8|single family| +34000|AAAAAAAAANEIAAAA|482|Seventh |Drive|Suite T|Edgewood|Wheatland County|MT|60069|United States|-7|apartment| +34001|AAAAAAAABNEIAAAA|893|Spruce |Cir.|Suite 90|Mount Pleasant|Montrose County|CO|81933|United States|-7|apartment| +34002|AAAAAAAACNEIAAAA|176|Twelfth |Street|Suite K|Concord|Montgomery County|OH|44107|United States|-5|condo| +34003|AAAAAAAADNEIAAAA|54|Main |Boulevard|Suite 440|Clifford|Jackson County|TN|38164|United States|-5|condo| +34004|AAAAAAAAENEIAAAA|481|Sixth |Road|Suite 30|Mount Vernon|Buena Vista County|IA|58482|United States|-6|apartment| +34005|AAAAAAAAFNEIAAAA|733|13th Fourth|Blvd|Suite 170|Riverview|Cannon County|TN|39003|United States|-5|single family| +34006|AAAAAAAAGNEIAAAA|848|Johnson Pine|RD|Suite K|Harmon|Mackinac County|MI|45623|United States|-5|apartment| +34007|AAAAAAAAHNEIAAAA|837|First 4th|RD|Suite 260|Shiloh|Powell County|MT|69275|United States|-7|condo| +34008|AAAAAAAAINEIAAAA|907|Smith Williams|RD|Suite O|Marion|Nacogdoches County|TX|70399|United States|-6|apartment| +34009|AAAAAAAAJNEIAAAA|805|Oak |Wy|Suite X|Mountain View|Beaver County|PA|14466|United States|-5|single family| +34010|AAAAAAAAKNEIAAAA|469|Third Cherry|Lane|Suite 90|Sumner|York County|VA|20519|United States|-5|single family| +34011|AAAAAAAALNEIAAAA|146|Ninth |Pkwy|Suite 490|Georgetown|Butler County|KY|47057|United States|-6|single family| +34012|AAAAAAAAMNEIAAAA|33|Locust Walnut|Way|Suite 230|Oakwood|Wayne County|PA|10169|United States|-5|single family| +34013|AAAAAAAANNEIAAAA|94|Oak |Circle|Suite 0|Nashville|Andrews County|TX|78249|United States|-6|single family| +34014|AAAAAAAAONEIAAAA|97|Johnson |Parkway|Suite T|Glendale|Lincoln County|KS|63951|United States|-6|apartment| +34015|AAAAAAAAPNEIAAAA|892||||Shiloh||||||condo| +34016|AAAAAAAAAOEIAAAA|203|Railroad |Circle|Suite 430|Waterloo|Lyon County|KY|41675|United States|-5|condo| +34017|AAAAAAAABOEIAAAA|25|13th |Road|Suite 290|Providence|Utah County|UT|86614|United States|-7|condo| +34018|AAAAAAAACOEIAAAA|735|Sunset Railroad|Wy|Suite B|Wildwood|Los Alamos County|NM|86871|United States|-7|apartment| +34019|AAAAAAAADOEIAAAA|517|First |Lane|Suite 420|Longwood|Floyd County|VA|27021|United States|-5|condo| +34020|AAAAAAAAEOEIAAAA|272|Church |Lane|Suite W|Riverview|Cleveland County|NC|29003|United States|-5|single family| +34021|AAAAAAAAFOEIAAAA|874|Third Park|Wy|Suite 40|Riverside|Catoosa County|GA|39231|United States|-5|apartment| +34022|AAAAAAAAGOEIAAAA|||Dr.||White Oak|Camas County||||-7|apartment| +34023|AAAAAAAAHOEIAAAA|821|Ridge |Circle|Suite O|Ruth|Santa Rosa County|FL|30309|United States|-5|condo| +34024|AAAAAAAAIOEIAAAA|452|Sycamore ||||Indian River County||35709||-5|condo| +34025|AAAAAAAAJOEIAAAA|983|Elm |Lane|Suite 70|Oakwood|Butte County|CA|90169|United States|-8|condo| +34026|AAAAAAAAKOEIAAAA|237|Hill |Dr.|Suite I|Riverside|Jefferson County|TN|39231|United States|-6|condo| +34027|AAAAAAAALOEIAAAA|330|Birch |Wy|Suite B|Centerville|Collingsworth County|TX|70059|United States|-6|condo| +34028|AAAAAAAAMOEIAAAA|545|Green |Ct.|Suite C|Greenfield|Florence County|WI|55038|United States|-6|apartment| +34029|AAAAAAAANOEIAAAA|982|Park |Cir.|Suite 160|Brownsville|Morgan County|WV|29310|United States|-5|single family| +34030|AAAAAAAAOOEIAAAA|432|Birch Park|Boulevard|Suite 10|Oak Grove|Worcester County|MA|08970|United States|-5|apartment| +34031|AAAAAAAAPOEIAAAA|696|4th Spring|Boulevard|Suite 260|Deerfield|Whatcom County|WA|99840|United States|-8|apartment| +34032|AAAAAAAAAPEIAAAA|||Court|||Lenoir County|||||| +34033|AAAAAAAABPEIAAAA|891|Jefferson |Avenue|Suite 360|Plainview|Grant County|KS|63683|United States|-6|condo| +34034|AAAAAAAACPEIAAAA|880|Church |Blvd|Suite B|Harmony|Davison County|SD|55804|United States|-6|single family| +34035|AAAAAAAADPEIAAAA|161|Central 3rd|Ave|Suite Y|Macon|Bethel Census Area|AK|90369|United States|-9|condo| +34036|AAAAAAAAEPEIAAAA|9|Spring |Ave|Suite S|Unionville|Barbour County|WV|21711|United States|-5|single family| +34037|AAAAAAAAFPEIAAAA|312|Walnut |RD|Suite F|Whitesville|Terrebonne Parish|LA|75903|United States|-6|apartment| +34038|AAAAAAAAGPEIAAAA|965|9th Poplar|Way|Suite 150|Shady Grove|Iron County|MO|62812|United States|-6|apartment| +34039|AAAAAAAAHPEIAAAA|125|Adams Seventh|Blvd|Suite 330|Oakwood|Nevada County|CA|90169|United States|-8|single family| +34040|AAAAAAAAIPEIAAAA|718|Jackson Second|Ln|Suite 450|Springdale|Canadian County|OK|78883|United States|-6|apartment| +34041|AAAAAAAAJPEIAAAA|991|Sunset |Cir.|Suite L|Walnut Grove|Chickasaw County|MS|57752|United States|-6|condo| +34042|AAAAAAAAKPEIAAAA|625|13th |Avenue|Suite 470|Georgetown|Forsyth County|NC|27057|United States|-5|condo| +34043|AAAAAAAALPEIAAAA|332|College 8th|Way|Suite Q|Hamilton|Muskingum County|OH|42808|United States|-5|condo| +34044|AAAAAAAAMPEIAAAA|434|Lee Franklin|Ave|Suite M|Buena Vista|Lewis County|NY|15752|United States|-5|apartment| +34045|AAAAAAAANPEIAAAA|606|Walnut Fourth||Suite G|||IL||United States||apartment| +34046|AAAAAAAAOPEIAAAA|337|6th |Ct.|Suite 420|Belmont|Gordon County|GA|30191|United States|-5|apartment| +34047|AAAAAAAAPPEIAAAA|724|Hillcrest |Blvd|Suite 120|Buena Vista|Vernon County|WI|55752|United States|-6|single family| +34048|AAAAAAAAAAFIAAAA|970|Main |Way|Suite X|Bridgeport|Jeff Davis County|TX|75817|United States|-6|condo| +34049|AAAAAAAABAFIAAAA|698|Cherry |Pkwy|Suite 480|Florence|Marion County|SC|23394|United States|-5|apartment| +34050|AAAAAAAACAFIAAAA|921|Birch 7th|Street|Suite 190|Welcome|Danville city|VA|26386|United States|-5|condo| +34051|AAAAAAAADAFIAAAA|699|Madison |Way|Suite M|Mount Pleasant|Jefferson County|PA|11933|United States|-5|single family| +34052|AAAAAAAAEAFIAAAA|773|Spring |Road|Suite Q|Cedar Grove|Berkeley County|WV|20411|United States|-5|single family| +34053|AAAAAAAAFAFIAAAA|138|Lakeview Williams|Ct.|Suite 70|Marion|Spotsylvania County|VA|20399|United States|-5|single family| +34054|AAAAAAAAGAFIAAAA|52|Eigth |Street|Suite Y|Springfield|Schuylkill County|PA|19303|United States|-5|condo| +34055|AAAAAAAAHAFIAAAA|522||||||||United States|-6|| +34056|AAAAAAAAIAFIAAAA|713|2nd |Avenue|Suite 340|Unionville|Amherst County|VA|21711|United States|-5|condo| +34057|AAAAAAAAJAFIAAAA||Cedar ||||||||-6|single family| +34058|AAAAAAAAKAFIAAAA|343|Fifth Woodland|Dr.|Suite T|Red Hill|Union Parish|LA|74338|United States|-6|apartment| +34059|AAAAAAAALAFIAAAA|674|2nd |Dr.|Suite 410|Pleasant Hill|Whitley County|IN|43604|United States|-5|single family| +34060|AAAAAAAAMAFIAAAA|662|11th |Road|Suite 50|Lincoln|Switzerland County|IN|41289|United States|-5|condo| +34061|AAAAAAAANAFIAAAA|846|Meadow |Parkway|Suite C|Forest Hills|Tooele County|UT|89237|United States|-7|condo| +34062|AAAAAAAAOAFIAAAA|13|Franklin |Lane|Suite 340|Union Hill|Pendleton County|KY|47746|United States|-5|condo| +34063|AAAAAAAAPAFIAAAA|823|Second |Parkway|Suite V|Buena Vista|Marion County|AL|35752|United States|-6|single family| +34064|AAAAAAAAABFIAAAA|423|Wilson Meadow|Drive|Suite Y|Climax|Barron County|WI|57991|United States|-6|single family| +34065|AAAAAAAABBFIAAAA|614|First |Ln|Suite 230|Forest Hills|Fayette County|KY|49237|United States|-6|single family| +34066|AAAAAAAACBFIAAAA|571|First Park|Pkwy|Suite 40|Springfield|Daviess County|IN|49303|United States|-5|single family| +34067|AAAAAAAADBFIAAAA|693|13th Lee|Circle|Suite C|Pleasant Grove|Gentry County|MO|64136|United States|-6|single family| +34068|AAAAAAAAEBFIAAAA|888|Ash |Street|Suite V|Walnut Grove|Kandiyohi County|MN|57752|United States|-6|apartment| +34069|AAAAAAAAFBFIAAAA|316|Maple |Way|Suite 20|Red Hill|Cass County|MO|64338|United States|-6|apartment| +34070|AAAAAAAAGBFIAAAA|740|Railroad |Dr.|Suite O|Fairview|Fallon County|MT|65709|United States|-7|apartment| +34071|AAAAAAAAHBFIAAAA|580|Poplar Jackson|Court|Suite 120|Cedar|Jefferson County|AL|31229|United States|-6|single family| +34072|AAAAAAAAIBFIAAAA|562|13th |Way|Suite P|Brownsville|Manassas Park city|VA|29310|United States|-5|condo| +34073|AAAAAAAAJBFIAAAA|281|Hillcrest |Pkwy|Suite D|Shady Grove|Westmoreland County|VA|22812|United States|-5|apartment| +34074|AAAAAAAAKBFIAAAA|89|8th Lake|Circle|Suite K|Spring Valley|Montgomery County|OH|46060|United States|-5|apartment| +34075|AAAAAAAALBFIAAAA|670|Oak |Court|Suite B|Union|Wilson County|TN|38721|United States|-5|condo| +34076|AAAAAAAAMBFIAAAA|159|3rd Jefferson|Road|Suite 470|Springdale|Daggett County|UT|88883|United States|-7|condo| +34077|AAAAAAAANBFIAAAA|220|Church |Dr.|Suite B|Woodlawn|Mineral County|WV|24098|United States|-5|| +34078|AAAAAAAAOBFIAAAA|901|3rd Sunset|ST|Suite 40|Antioch|Halifax County|VA|28605|United States|-5|single family| +34079|AAAAAAAAPBFIAAAA|471|Sunset |Ln|Suite D|Kingston|Bay County|MI|44975|United States|-5|single family| +34080|AAAAAAAAACFIAAAA|110|Jefferson |Street|Suite 300|Bunker Hill|Liberty County|MT|60150|United States|-7|condo| +34081|AAAAAAAABCFIAAAA|836|Walnut |Ln|Suite 90|Jackson|Walworth County|WI|59583|United States|-6|single family| +34082|AAAAAAAACCFIAAAA|613|Park Lakeview|Pkwy|Suite N|Pleasant Hill|Ouray County|CO|83604|United States|-7|apartment| +34083|AAAAAAAADCFIAAAA|797|Fourth |Parkway|Suite O|Red Hill|Burnett County|WI|54338|United States|-6|single family| +34084|AAAAAAAAECFIAAAA|772|Main Mill|Circle|Suite 10|New Hope|Simpson County|KY|49431|United States|-5|apartment| +34085|AAAAAAAAFCFIAAAA|17|3rd 14th|Road|Suite 70|Lakewood|Crittenden County|KY|48877|United States|-6|apartment| +34086|AAAAAAAAGCFIAAAA|795|Highland |Wy|Suite A|Lakewood|Baxter County|AR|78877|United States|-6|single family| +34087|AAAAAAAAHCFIAAAA|393|1st West|Ave|Suite 350|Fairview|Ramsey County|ND|55709|United States|-6|single family| +34088|AAAAAAAAICFIAAAA|845|3rd |Ave||Salem|||28048|United States|-5|apartment| +34089|AAAAAAAAJCFIAAAA|756|Spring Broadway|Cir.|Suite 240|Birmingham|Little River County|AR|73372|United States|-6|condo| +34090|AAAAAAAAKCFIAAAA|574|Davis 8th|Ave|Suite 360|Forest Hills|San Patricio County|TX|79237|United States|-6|condo| +34091|AAAAAAAALCFIAAAA|75|Lincoln |Blvd|Suite I|Oak Ridge|Cumberland County|KY|48371|United States|-6|apartment| +34092|AAAAAAAAMCFIAAAA|226|Jackson |Boulevard|Suite A|Empire|Johnson County|WY|84145|United States|-7|condo| +34093|AAAAAAAANCFIAAAA|496|Park |Pkwy|Suite F|Kingston|Evans County|GA|34975|United States|-5|single family| +34094|AAAAAAAAOCFIAAAA|481|View Cedar|Wy|Suite O|Lynn|Alachua County|FL|30407|United States|-5|single family| +34095|AAAAAAAAPCFIAAAA|929|6th |Lane|Suite 230||Tioga County|PA||United States||| +34096|AAAAAAAAADFIAAAA|621|Johnson |ST|Suite Y|Shady Grove|Woodward County|OK|72812|United States|-6|apartment| +34097|AAAAAAAABDFIAAAA|986|Washington |Cir.|Suite 440|Oakwood|Seneca County|NY|10169|United States|-5|apartment| +34098|AAAAAAAACDFIAAAA|680|Poplar Forest|Ct.|Suite 130|Mountain View|Lee County|KY|44466|United States|-5|single family| +34099|AAAAAAAADDFIAAAA|457|Fifteenth |Drive|Suite 430|Sugar Hill|Allen County|IN|45114|United States|-5|single family| +34100|AAAAAAAAEDFIAAAA|176|Washington |Blvd|Suite 160|White Oak|Kinney County|TX|76668|United States|-6|single family| +34101|AAAAAAAAFDFIAAAA|207|Park 11th|Court|Suite 10|Monroe|Rich County|UT|83984|United States|-7|condo| +34102|AAAAAAAAGDFIAAAA|913|||||Scott County|MN||United States|-6|| +34103|AAAAAAAAHDFIAAAA|320|Oak |Ct.|Suite 200|Union Hill|Hanover County|VA|27746|United States|-5|apartment| +34104|AAAAAAAAIDFIAAAA|36|Ash Madison|ST|Suite U|Concord|Stafford County|KS|64107|United States|-6|apartment| +34105|AAAAAAAAJDFIAAAA|132|Laurel |Avenue|Suite 410|Oakwood|Union County|MS|50169|United States|-6|condo| +34106|AAAAAAAAKDFIAAAA|483|Walnut |Ln|Suite 160|Woodville|Pontotoc County|OK|74289|United States|-6|condo| +34107|AAAAAAAALDFIAAAA|917|10th |Way|Suite F|Midway|Wayne County|NE|61904|United States|-7|condo| +34108|AAAAAAAAMDFIAAAA|568|9th |Court|Suite 490|Thompsonville|Cass County|IA|59651|United States|-6|apartment| +34109|AAAAAAAANDFIAAAA|726|Oak |Blvd|Suite H|Fairfield|Thurston County|NE|66192|United States|-7|apartment| +34110|AAAAAAAAODFIAAAA|306|Meadow Ridge|Drive|Suite E|Midway|Lynn County|TX|71904|United States|-6|condo| +34111|AAAAAAAAPDFIAAAA|950|5th |Avenue|Suite T|Tracy|Onondaga County|NY|16340|United States|-5|single family| +34112|AAAAAAAAAEFIAAAA|499|Elm 7th|RD|Suite 410|Antioch|Scott County|IN|48605|United States|-5|single family| +34113|AAAAAAAABEFIAAAA|134|Lake |Pkwy|Suite S|Greenwood|Pearl River County|MS|58828|United States|-6|single family| +34114|AAAAAAAACEFIAAAA|108|6th |Boulevard|Suite O|Jamestown|Newport County|RI|07467|United States|-5|single family| +34115|AAAAAAAADEFIAAAA|261|Park Smith|Blvd|Suite M|Midway|Middlesex County|MA|02504|United States|-5|condo| +34116|AAAAAAAAEEFIAAAA|725|Oak |Blvd|Suite 150|Walnut Grove|Terrebonne Parish|LA|77752|United States|-6|apartment| +34117|AAAAAAAAFEFIAAAA|969|Lee Lakeview|Wy|Suite 190|Arlington|Bremer County|IA|56557|United States|-6|condo| +34118|AAAAAAAAGEFIAAAA|752|Pine |Blvd|Suite O|Enterprise|Atchison County|KS|61757|United States|-6|single family| +34119|AAAAAAAAHEFIAAAA|144|1st |Drive|Suite 370|Stringtown|Thomas County|GA|30162|United States|-5|condo| +34120|AAAAAAAAIEFIAAAA|916|15th Wilson|Ave|Suite O|Saint Clair|Buchanan County|VA|25294|United States|-5|single family| +34121|AAAAAAAAJEFIAAAA|413|3rd |Boulevard|Suite 40|Woodville|Milam County|TX|74289|United States|-6|condo| +34122|AAAAAAAAKEFIAAAA|102|Main Green|Ave|Suite 410|Kingston|Dewey County|SD|54975|United States|-6|single family| +34123|AAAAAAAALEFIAAAA|424|6th Ridge|Drive|Suite 420|Fairfield|Rusk County|WI|56192|United States|-6|single family| +34124|AAAAAAAAMEFIAAAA|328|3rd Forest|Street|Suite 360|Edgewood|Tishomingo County|MS|50069|United States|-6|single family| +34125|AAAAAAAANEFIAAAA|992|Williams 10th|Ln|Suite P|Macedonia|Macon County|GA|31087|United States|-5|condo| +34126|AAAAAAAAOEFIAAAA|732|Park |Wy|Suite 40|Liberty|Marion County|FL|33451|United States|-5|single family| +34127|AAAAAAAAPEFIAAAA|513|Main |Ln|Suite 340|Green Acres|Caledonia County|VT|08283|United States|-5|condo| +34128|AAAAAAAAAFFIAAAA|833|Chestnut 9th|Blvd|Suite U|Paxton|Lancaster County|NE|65669|United States|-7|condo| +34129|AAAAAAAABFFIAAAA||9th |Wy|||||40191|United States|-6|| +34130|AAAAAAAACFFIAAAA|799|Pine |Pkwy|Suite W|Oakland|Douglas County|MO|69843|United States|-6|condo| +34131|AAAAAAAADFFIAAAA|55|Pine 6th|Ct.|Suite O|Lakewood|Middlesex County|VA|28877|United States|-5|condo| +34132|AAAAAAAAEFFIAAAA|166|Poplar Walnut|Avenue|Suite R|Hamilton|DeWitt County|TX|72808|United States|-6|apartment| +34133|AAAAAAAAFFFIAAAA|14|Miller |Street|Suite 20|Oakdale|Tulare County|CA|99584|United States|-8|single family| +34134|AAAAAAAAGFFIAAAA|728|Lincoln Ninth|Circle|Suite 230|Woodville|Johnson County|TX|74289|United States|-6|condo| +34135|AAAAAAAAHFFIAAAA|909|15th West|Lane|Suite I|Clinton|New Madrid County|MO|68222|United States|-6|single family| +34136|AAAAAAAAIFFIAAAA|769|North 1st|Ln|Suite 110|Brentwood|Logan County|CO|84188|United States|-7|apartment| +34137|AAAAAAAAJFFIAAAA|208|12th |Drive|Suite Y|Deerfield|Buckingham County|VA|29840|United States|-5|condo| +34138|AAAAAAAAKFFIAAAA|92|Seventh Washington|Street|Suite 200|Shiloh|Jefferson County|IL|69275|United States|-6|apartment| +34139|AAAAAAAALFFIAAAA|393|Oak Washington|Court|Suite W|Highland|Claiborne County|MS|59454|United States|-6|apartment| +34140|AAAAAAAAMFFIAAAA|854|Green |Lane|Suite V|Newport|Cass County|NE|61521|United States|-6|single family| +34141|AAAAAAAANFFIAAAA|397|Franklin |Circle|Suite 360|Valley View|Halifax County|NC|25124|United States|-5|single family| +34142|AAAAAAAAOFFIAAAA|953|Johnson Dogwood|Circle|Suite O|Forest Hills|Russell County|AL|39237|United States|-6|single family| +34143|AAAAAAAAPFFIAAAA|582|Washington 1st|Court|Suite 460|Fairview|Caledonia County|VT|06309|United States|-5|single family| +34144|AAAAAAAAAGFIAAAA|983|Third |Drive|Suite 350|Edgewood|Storey County|NV|80069|United States|-8|condo| +34145|AAAAAAAABGFIAAAA|124|First |Ave|Suite 220|Mount Pleasant|Jefferson Davis Parish|LA|71933|United States|-6|apartment| +34146|AAAAAAAACGFIAAAA|30|Cherry |Parkway|Suite W|White Oak|Beadle County|SD|56668|United States|-6|apartment| +34147|AAAAAAAADGFIAAAA|205|View |Ln|Suite E|Woodville|Brazoria County|TX|74289|United States|-6|apartment| +34148|AAAAAAAAEGFIAAAA|551|Hill 5th|Ave|Suite T|Spring Valley|Jefferson County|PA|16060|United States|-5|apartment| +34149|AAAAAAAAFGFIAAAA|810|Park Highland|Avenue|Suite F|Salem|Platte County|MO|68048|United States|-6|single family| +34150|AAAAAAAAGGFIAAAA|922|Sycamore |Avenue|Suite I|Valley View|Philadelphia County|PA|15124|United States|-5|apartment| +34151|AAAAAAAAHGFIAAAA|35|Lincoln |Way|Suite O|Hopewell|Hampton city|VA|20587|United States|-5|condo| +34152|AAAAAAAAIGFIAAAA|28|North Lincoln|Wy|Suite I|Green Acres|Briscoe County|TX|77683|United States|-6|condo| +34153|AAAAAAAAJGFIAAAA|144|Elm |Blvd|Suite 410|Lakeside|Conejos County|CO|89532|United States|-7|condo| +34154|AAAAAAAAKGFIAAAA|375|Center Adams|Court|Suite 140|Centerville|Fremont County|CO|80059|United States|-7|apartment| +34155|AAAAAAAALGFIAAAA|152|Highland Railroad|Pkwy|Suite E|Sunnyside|Washington County|RI|02552|United States|-5|condo| +34156|AAAAAAAAMGFIAAAA|20|First Sunset|Ln|Suite I|Pleasant Valley|Chester County|PA|12477|United States|-5|condo| +34157|AAAAAAAANGFIAAAA|542|Laurel |Ln|Suite 330|Shiloh|Warren County|IN|49275|United States|-5|single family| +34158|AAAAAAAAOGFIAAAA|969|Green |Parkway|Suite C|Sterling|Guadalupe County|NM|87845|United States|-7|condo| +34159|AAAAAAAAPGFIAAAA|854|Williams |Lane|Suite P|Deerfield|Ward County|ND|59840|United States|-6|single family| +34160|AAAAAAAAAHFIAAAA|433|Highland 11th|Court|Suite M|Woodland|Douglas County|OR|94854|United States|-8|apartment| +34161|AAAAAAAABHFIAAAA|597|Dogwood |Street|Suite T|Elba|Ashe County|NC|20262|United States|-5|apartment| +34162|AAAAAAAACHFIAAAA|772|Fourth Cedar|Ave|Suite G|Springdale|San Augustine County|TX|78883|United States|-6|condo| +34163|AAAAAAAADHFIAAAA|414|Washington |Pkwy||Guthrie|Charles Mix County|||United States|-6|condo| +34164|AAAAAAAAEHFIAAAA|407|1st 6th|Court|Suite D|Hamilton|Hawaii County|HI|92808|United States|-10|condo| +34165|AAAAAAAAFHFIAAAA|267|3rd Washington|Parkway|Suite 20|Bunker Hill|Cheyenne County|CO|80150|United States|-7|single family| +34166|AAAAAAAAGHFIAAAA|390|Miller Poplar|Parkway|Suite C|Mount Zion|Lake County|OR|98054|United States|-8|apartment| +34167|AAAAAAAAHHFIAAAA|468|Cedar |Ln|Suite F|Summerfield|Emanuel County|GA|30634|United States|-5|single family| +34168|AAAAAAAAIHFIAAAA|23|West Oak|Lane|Suite 450|Edgewood|Orange County|NY|10069|United States|-5|single family| +34169|AAAAAAAAJHFIAAAA|504|Spruce |Ave|Suite 170|Mount Olive|Fredericksburg city|VA|28059|United States|-5|single family| +34170|AAAAAAAAKHFIAAAA|369|Maple |Boulevard|Suite P|Fairview|Grainger County|TN|35709|United States|-5|apartment| +34171|AAAAAAAALHFIAAAA|43|Lincoln |Ave|Suite 280|Highland|Howard County|AR|79454|United States|-6|single family| +34172|AAAAAAAAMHFIAAAA|234|Meadow Forest|Avenue|Suite 320|Mount Vernon|De Kalb County|IN|48482|United States|-5|single family| +34173|AAAAAAAANHFIAAAA|876|13th Meadow|Dr.|Suite X|Georgetown|Luce County|MI|47057|United States|-5|apartment| +34174|AAAAAAAAOHFIAAAA|576|Poplar Maple|Parkway|Suite E|Owens|Sterling County|TX|72324|United States|-6|single family| +34175|AAAAAAAAPHFIAAAA|560|Birch |Avenue|Suite 420|Arlington|Murray County|OK|76557|United States|-6|condo| +34176|AAAAAAAAAIFIAAAA|772|View |Pkwy|Suite M|Glendale|Harford County|MD|23951|United States|-5|apartment| +34177|AAAAAAAABIFIAAAA|459|North |Road|Suite Q|Mountain View|Montgomery County|TX|74466|United States|-6|condo| +34178|AAAAAAAACIFIAAAA|632|4th Woodland|Blvd|Suite W|Birmingham|Todd County|KY|43372|United States|-5|single family| +34179|AAAAAAAADIFIAAAA|744|South Valley|Pkwy|Suite 390|Sunnyside|Taney County|MO|61952|United States|-6|condo| +34180|AAAAAAAAEIFIAAAA|999|Jefferson 6th|Way|Suite S|Pine Grove|Big Horn County|WY|84593|United States|-7|apartment| +34181|AAAAAAAAFIFIAAAA|599|Hillcrest Ninth|Court|Suite U|Springfield|Franklin County|ME|09903|United States|-5|single family| +34182|AAAAAAAAGIFIAAAA|918|Ridge |Drive|Suite 260|Lebanon|Searcy County|AR|72898|United States|-6|condo| +34183|AAAAAAAAHIFIAAAA|877||Way|||||96971|United States||apartment| +34184|AAAAAAAAIIFIAAAA|560|Seventh Maple|Court|Suite 430|Mountain View|Wetzel County|WV|24466|United States|-5|apartment| +34185|AAAAAAAAJIFIAAAA|677|Madison First|Ln|Suite 50|Maple Grove|Franklin County|VA|28252|United States|-5|condo| +34186|AAAAAAAAKIFIAAAA|640|Twelfth Railroad|Ct.|Suite 60|Forest Hills|Ozaukee County|WI|59237|United States|-6|condo| +34187|AAAAAAAALIFIAAAA|958|12th Fourth|Avenue|Suite 470|Oak Ridge|Gove County|KS|68371|United States|-6|apartment| +34188|AAAAAAAAMIFIAAAA|645|Church |Cir.|Suite W||||77683||-6|| +34189|AAAAAAAANIFIAAAA|621|Mill Jefferson|Avenue|Suite M|Woodland|Hall County|NE|64854|United States|-6|single family| +34190|AAAAAAAAOIFIAAAA|478|Maple |Drive|Suite 270|Buena Vista|Escambia County|FL|35752|United States|-5|condo| +34191|AAAAAAAAPIFIAAAA|456|Jackson |Wy|Suite C|Centerville|Boone County|MO|60059|United States|-6|apartment| +34192|AAAAAAAAAJFIAAAA|108|Woodland |Ave|Suite E|Plainview|Polk County|TX|73683|United States|-6|single family| +34193|AAAAAAAABJFIAAAA|182|Jackson |Parkway||Plainview|Renville County|MN|||-6|single family| +34194|AAAAAAAACJFIAAAA|108|Main |Cir.|Suite O|Burns|Merced County|CA|95272|United States|-8|single family| +34195|AAAAAAAADJFIAAAA|762|Fourth Poplar|ST|Suite A|Providence|Morgan County|GA|36614|United States|-5|single family| +34196|AAAAAAAAEJFIAAAA|549|5th |Parkway|Suite 230|Friendship|Randolph County|IN|44536|United States|-5|apartment| +34197|AAAAAAAAFJFIAAAA|408|Pine |Way|Suite M|Arlington|Eureka County|NV|86557|United States|-8|condo| +34198|AAAAAAAAGJFIAAAA|998|Mill |Drive|Suite 280|Crossroads|Codington County|SD|50534|United States|-6|single family| +34199|AAAAAAAAHJFIAAAA|151|Cherry |Drive|Suite F|Franklin|Santa Rosa County|FL|39101|United States|-5|single family| +34200|AAAAAAAAIJFIAAAA|337|Adams Second|Pkwy|Suite 430|Friendship|Monroe County|IN|44536|United States|-5|apartment| +34201|AAAAAAAAJJFIAAAA||Park View||Suite K||Otsego County||41904|||| +34202|AAAAAAAAKJFIAAAA|829|1st Johnson|Drive|Suite V|Greenfield|Wood County|OH|45038|United States|-5|condo| +34203|AAAAAAAALJFIAAAA|470|View |Wy|Suite 60|Red Hill|Shelby County|TN|34338|United States|-6|single family| +34204|AAAAAAAAMJFIAAAA|867|Lee Second|Boulevard|Suite 270|Edgewood|Shawnee County|KS|60069|United States|-6|single family| +34205|AAAAAAAANJFIAAAA|528|Jefferson Birch|Pkwy|Suite 370|Calhoun|Adams County|ND|56909|United States|-6|apartment| +34206|AAAAAAAAOJFIAAAA|745|West Eigth|Ct.|Suite 430|Woodville|Carroll County|IL|64289|United States|-6|condo| +34207|AAAAAAAAPJFIAAAA|234|Cherry Green|Cir.|Suite 450|Cedar Grove|Kanabec County|MN|50411|United States|-6|apartment| +34208|AAAAAAAAAKFIAAAA|917|Poplar 2nd|Court|Suite 420|Oak Ridge|Jefferson County|ID|88371|United States|-7|single family| +34209|AAAAAAAABKFIAAAA|476|1st |Avenue|Suite 310|Crossroads|Macomb County|MI|40534|United States|-5|single family| +34210|AAAAAAAACKFIAAAA|822|Sycamore |Ln|Suite 440|Enterprise|Worcester County|MA|02357|United States|-5|apartment| +34211|AAAAAAAADKFIAAAA|426|Dogwood |Lane|Suite S|Woodlawn|Wakulla County|FL|34098|United States|-5|single family| +34212|AAAAAAAAEKFIAAAA|578|Franklin |Drive|Suite 100|Midway|Montour County|PA|11904|United States|-5|single family| +34213|AAAAAAAAFKFIAAAA|603|9th |Parkway|Suite 290|Wildwood|Boundary County|ID|86871|United States|-7|apartment| +34214|AAAAAAAAGKFIAAAA|259|Willow East|ST|Suite 40|Lakewood|Hubbard County|MN|58877|United States|-6|single family| +34215|AAAAAAAAHKFIAAAA|894|Cherry Maple|Street|Suite 410|Sunnyside|Lincoln County|WY|81952|United States|-7|apartment| +34216|AAAAAAAAIKFIAAAA|||Ave|Suite 120||Scott County|IA|||-6|| +34217|AAAAAAAAJKFIAAAA|609|Park |Pkwy|Suite S|Newtown|Lewis County|WV|21749|United States|-5|condo| +34218|AAAAAAAAKKFIAAAA|165|Hillcrest |Ave|Suite P|Friendship|Pitkin County|CO|84536|United States|-7|condo| +34219|AAAAAAAALKFIAAAA|335|Madison |Wy|Suite S|California|Sullivan County|TN|30141|United States|-6|condo| +34220|AAAAAAAAMKFIAAAA|830|Spruce View|Ln|Suite S|Deerfield|Otero County|CO|89840|United States|-7|condo| +34221|AAAAAAAANKFIAAAA|265|Ridge |Circle|Suite 300|Antioch|Grimes County|TX|78605|United States|-6|apartment| +34222|AAAAAAAAOKFIAAAA|271|South |Cir.|Suite 250|Walnut Grove|Moody County|SD|57752|United States|-7|apartment| +34223|AAAAAAAAPKFIAAAA|136|Fourth |Pkwy|Suite 80|Wilson|McDonough County|IL|66971|United States|-6|apartment| +34224|AAAAAAAAALFIAAAA|795|Church Ridge|Cir.|Suite N|Fayetteville|Broadwater County|MT|61732|United States|-7|apartment| +34225|AAAAAAAABLFIAAAA|83|Cedar Maple|Boulevard|Suite 140|Waterloo|Lee County|AL|31675|United States|-6|single family| +34226|AAAAAAAACLFIAAAA|801|Fifth Wilson|Blvd|Suite T|Riverview|Hendricks County|IN|49003|United States|-5|condo| +34227|AAAAAAAADLFIAAAA|49|Park |Blvd|Suite 100|Shady Grove|Union County|NC|22812|United States|-5|single family| +34228|AAAAAAAAELFIAAAA|111|Mill |Ln|Suite O|Kingston|Montgomery County|NC|24975|United States|-5|single family| +34229|AAAAAAAAFLFIAAAA|957|North |Boulevard|Suite 490|Georgetown|Hinds County|MS|57057|United States|-6|apartment| +34230|AAAAAAAAGLFIAAAA|460|Willow 9th|Ct.|Suite G|Fairview|Beaufort County|SC|25709|United States|-5|condo| +34231|AAAAAAAAHLFIAAAA|53|Williams Hickory|Ave|Suite 490|Jamestown|Hamilton County|IL|66867|United States|-6|condo| +34232|AAAAAAAAILFIAAAA|923|Valley Oak|Way|Suite M|Woodlawn|Pawnee County|OK|74098|United States|-6|condo| +34233|AAAAAAAAJLFIAAAA|766|Pine 14th|Way|Suite 430|Georgetown|Power County|ID|87057|United States|-7|apartment| +34234|AAAAAAAAKLFIAAAA|25|6th |Circle|Suite V|Sulphur Springs|Nassau County|FL|38354|United States|-5|condo| +34235|AAAAAAAALLFIAAAA|102|Center |Boulevard|Suite 310|Fairfield|Plumas County|CA|96192|United States|-8|single family| +34236|AAAAAAAAMLFIAAAA|733|5th |Street|Suite 370|Summit|Leon County|TX|70499|United States|-6|condo| +34237|AAAAAAAANLFIAAAA|702|Walnut Meadow|Drive|Suite N|Liberty|Monroe County|MI|43451|United States|-5|condo| +34238|AAAAAAAAOLFIAAAA|285|Central Cedar|Ln|Suite J||Lemhi County|ID|87683|||single family| +34239|AAAAAAAAPLFIAAAA|87|Locust |RD|Suite Q|Liberty|Pickaway County|OH|43451|United States|-5|single family| +34240|AAAAAAAAAMFIAAAA|299|Ash |RD|Suite 490|Georgetown|Linn County|KS|67057|United States|-6|condo| +34241|AAAAAAAABMFIAAAA|189|Highland 6th|Ave|Suite 170|Lincoln|New Haven County|CT|01889|United States|-5|single family| +34242|AAAAAAAACMFIAAAA|44|Park Franklin|Boulevard|Suite 420|Harmony|Webster County|MS|55804|United States|-6|single family| +34243|AAAAAAAADMFIAAAA|691|Oak |Circle|Suite 430|Union|Adams County|ND|58721|United States|-6|condo| +34244|AAAAAAAAEMFIAAAA|786|Forest |Parkway|Suite 50|Riverview|Polk County|TN|39003|United States|-6|apartment| +34245|AAAAAAAAFMFIAAAA|223|Jackson South|Parkway|Suite 220|Hamilton|Jefferson County|OR|92808|United States|-8|condo| +34246|AAAAAAAAGMFIAAAA|982|Williams |Road|Suite 190|Clifton|Plaquemines Parish|LA|78014|United States|-6|single family| +34247|AAAAAAAAHMFIAAAA|80|Washington |Ln|Suite 200|Woodville|Bullitt County|KY|44289|United States|-6|condo| +34248|AAAAAAAAIMFIAAAA|181|Park 13th|RD|Suite P|Antioch|Dallas County|AR|78605|United States|-6|single family| +34249|AAAAAAAAJMFIAAAA|277|Jackson Cherry|Court|Suite 20|Jamestown|Tulare County|CA|96867|United States|-8|condo| +34250|AAAAAAAAKMFIAAAA|797|11th |Circle|Suite 460|Springfield|Hood River County|OR|99303|United States|-8|condo| +34251|AAAAAAAALMFIAAAA|453|Fifth |ST|Suite 240|Midway|Sagadahoc County|ME|02504|United States|-5|single family| +34252|AAAAAAAAMMFIAAAA|77|5th |Cir.|Suite K|Morris|Kane County|IL|66696|United States|-6|condo| +34253|AAAAAAAANMFIAAAA|948|Mill View|Pkwy|Suite X|Mountain View|Highland County|OH|44466|United States|-5|single family| +34254|AAAAAAAAOMFIAAAA|798|Poplar |Wy|Suite B|Woodville|Salt Lake County|UT|84289|United States|-7|apartment| +34255|AAAAAAAAPMFIAAAA|851|4th Sunset|Street|Suite E|Stringtown|Leavenworth County|KS|60162|United States|-6|single family| +34256|AAAAAAAAANFIAAAA|35|Willow 13th|Lane|Suite V|Walnut Grove|Southampton County|VA|27752|United States|-5|single family| +34257|AAAAAAAABNFIAAAA|352|7th Wilson|Ct.|Suite K|Highland Park|Toombs County|GA|36534|United States|-5|single family| +34258|AAAAAAAACNFIAAAA|913|Lake |Ave|Suite T|Union Hill|Addison County|VT|08346|United States|-5|apartment| +34259|AAAAAAAADNFIAAAA|234|Center |Court|Suite 460|Union Hill|Gladwin County|MI|47746|United States|-5|single family| +34260|AAAAAAAAENFIAAAA|106|View |ST|Suite 460|Walnut Grove|Greene County|VA|27752|United States|-5|condo| +34261|AAAAAAAAFNFIAAAA|705|Jackson Ninth|Way|Suite 360|Crossroads|Addison County|VT|01134|United States|-5|single family| +34262|AAAAAAAAGNFIAAAA|97|Maple |Circle|Suite R|Pleasant Grove|Atkinson County|GA|34136|United States|-5|apartment| +34263|AAAAAAAAHNFIAAAA|930|9th Maple|Dr.|Suite G|Newtown|McHenry County|ND|51749|United States|-6|single family| +34264|AAAAAAAAINFIAAAA|255|Laurel 2nd|Parkway|Suite 470|Thompsonville|Queen Anne County|MD|29651|United States|-5|single family| +34265|AAAAAAAAJNFIAAAA|308|Wilson |Ln|Suite 50|New Hope|Johnson County|IN|49431|United States|-5|apartment| +34266|AAAAAAAAKNFIAAAA|168|Willow Poplar|RD|Suite U|Centerville|Nicholas County|WV|20059|United States|-5|single family| +34267|AAAAAAAALNFIAAAA|295|2nd |Drive|Suite T|Oakwood|Mille Lacs County|MN|50169|United States|-6|single family| +34268|AAAAAAAAMNFIAAAA|56|Pine |Avenue|Suite 350|Oakdale|Powell County|KY|49584|United States|-5|single family| +34269|AAAAAAAANNFIAAAA|73|Valley |Cir.|Suite 200|Lenox|Van Wert County|OH|41143|United States|-5|condo| +34270|AAAAAAAAONFIAAAA|881|Maple |Court|Suite B|Cambridge|Broadwater County|MT|67759|United States|-7|condo| +34271|AAAAAAAAPNFIAAAA|88|River |Wy|Suite 150|Hopewell|Hardin County|TX|70587|United States|-6|single family| +34272|AAAAAAAAAOFIAAAA|59|Spring |Blvd|Suite T|Deerfield|Cass County|MI|49840|United States|-5|apartment| +34273|AAAAAAAABOFIAAAA|968|View |Boulevard|Suite A|Riverview|Custer County|CO|89003|United States|-7|single family| +34274|AAAAAAAACOFIAAAA|537|Central |Cir.|Suite 40|Peoria|Buffalo County|SD|59818|United States|-6|single family| +34275|AAAAAAAADOFIAAAA|287|Park Park|Court|Suite 0|Marion|Clark County|KS|60399|United States|-6|single family| +34276|AAAAAAAAEOFIAAAA|686|Center Oak|Street|Suite 480|Oakland|Tuscola County|MI|49843|United States|-5|apartment| +34277|AAAAAAAAFOFIAAAA|497|7th |Circle|Suite 0|Waterloo|Adams County|MS|51675|United States|-6|single family| +34278|AAAAAAAAGOFIAAAA|881|Broadway First|Road|Suite 30|Ashland|Grant County|WA|94244|United States|-8|condo| +34279|AAAAAAAAHOFIAAAA|274|Main Oak|Blvd|Suite G|Edgewood|Mason County|WA|90069|United States|-8|apartment| +34280|AAAAAAAAIOFIAAAA|566|Cherry Hillcrest|Ave|Suite 270|Union|Deaf Smith County|TX|78721|United States|-6|apartment| +34281|AAAAAAAAJOFIAAAA|304|Lincoln |Circle|Suite 120|||IL|||-6|| +34282|AAAAAAAAKOFIAAAA|771|Miller |Pkwy|Suite D|Bath|Pope County|MN|50573|United States|-6|single family| +34283|AAAAAAAALOFIAAAA|595|Dogwood Elevnth|Ln|Suite 20|Crossroads|Middlesex County|CT|01134|United States|-5|condo| +34284|AAAAAAAAMOFIAAAA|61|Woodland Hill|Cir.|Suite M|Oakland|Jones County|NC|29843|United States|-5|condo| +34285|AAAAAAAANOFIAAAA|694|Walnut Main|Drive|Suite 340|Belmont|Montrose County|CO|80191|United States|-7|apartment| +34286|AAAAAAAAOOFIAAAA|685|Lakeview |Circle|Suite 60|Curtis|Marion County|KS|62444|United States|-6|apartment| +34287|AAAAAAAAPOFIAAAA|586|Ridge Lincoln|Ln|Suite 240|White Oak|Kimble County|TX|76668|United States|-6|single family| +34288|AAAAAAAAAPFIAAAA|388|Mill Pine|Blvd|Suite J|Unionville|Meade County|KY|41711|United States|-5|apartment| +34289|AAAAAAAABPFIAAAA|865|4th |Wy|Suite 230|Enterprise|Kent County|TX|71757|United States|-6|single family| +34290|AAAAAAAACPFIAAAA|920|College Woodland|Ln|Suite P|Hamilton|Livingston County|NY|12808|United States|-5|condo| +34291|AAAAAAAADPFIAAAA|552|5th Broadway|Avenue|Suite W|Macedonia|Cass County|IN|41087|United States|-5|condo| +34292|AAAAAAAAEPFIAAAA|286|Meadow Davis|Drive|Suite V|Oneida|Union County|TN|34027|United States|-6|condo| +34293|AAAAAAAAFPFIAAAA|455|Mill |Ln|Suite 380|Lebanon|Howard County|NE|62898|United States|-7|single family| +34294|AAAAAAAAGPFIAAAA||||||Fayette County|KY|44289||-6|| +34295|AAAAAAAAHPFIAAAA|351|Jefferson |Boulevard|Suite X|Maple Grove|Gilmer County|WV|28252|United States|-5|condo| +34296|AAAAAAAAIPFIAAAA|301|Railroad |Parkway|Suite W|White Oak|Shannon County|MO|66668|United States|-6|apartment| +34297|AAAAAAAAJPFIAAAA|161|Hill Broadway|Circle|Suite A|Union Hill|Henry County|IL|67746|United States|-6|single family| +34298|AAAAAAAAKPFIAAAA|319|Ridge |Ln|Suite V|Macedonia|Hall County|GA|31087|United States|-5|single family| +34299|AAAAAAAALPFIAAAA|401|Ridge 5th|Street|Suite P|Spring Valley|Teller County|CO|86060|United States|-7|apartment| +34300|AAAAAAAAMPFIAAAA|898|Lincoln Pine|Lane|Suite 60|Springfield|Powder River County|MT|69303|United States|-7|apartment| +34301|AAAAAAAANPFIAAAA|72|Seventh |Drive|Suite 270|Sulphur Springs|Beaverhead County|MT|68354|United States|-7|condo| +34302|AAAAAAAAOPFIAAAA|631|Cherry |Boulevard|Suite P|Jackson|Cherokee County|AL|39583|United States|-6|single family| +34303|AAAAAAAAPPFIAAAA|124|2nd 3rd|ST|Suite 270|Plainville|Sheridan County|WY|86115|United States|-7|condo| +34304|AAAAAAAAAAGIAAAA|759|Cherry |Way|Suite 420|Green Acres|Kendall County|TX|77683|United States|-6|single family| +34305|AAAAAAAABAGIAAAA|254|Wilson |Ct.|Suite S|Deerfield|Tyrrell County|NC|29840|United States|-5|single family| +34306|AAAAAAAACAGIAAAA|638|12th |Court|Suite 480|Providence|Minnehaha County|SD|56614|United States|-7|single family| +34307|AAAAAAAADAGIAAAA|495|Green |Ct.|Suite V|Oakwood|Pushmataha County|OK|70169|United States|-6|condo| +34308|AAAAAAAAEAGIAAAA|254|Hillcrest Church|Lane|Suite M|Springfield|Pickens County|GA|39303|United States|-5|apartment| +34309|AAAAAAAAFAGIAAAA|243|Davis |Street|Suite U|Cedar Grove|Jones County|SD|50411|United States|-7|condo| +34310|AAAAAAAAGAGIAAAA|208|1st |Lane|Suite I|Highland|Appling County|GA|39454|United States|-5|condo| +34311|AAAAAAAAHAGIAAAA|131|Valley Eigth|Ave|Suite 110|Wilson|Lewis County|MO|66971|United States|-6|apartment| +34312|AAAAAAAAIAGIAAAA|327|Birch Ridge|Lane|Suite P|Highland Park|Warren County|TN|36534|United States|-6|apartment| +34313|AAAAAAAAJAGIAAAA|734|Laurel 6th|Boulevard|Suite R|Riverview|Wright County|MN|59003|United States|-6|single family| +34314|AAAAAAAAKAGIAAAA|116|Smith |ST|Suite E|Macedonia|Howard County|IA|51087|United States|-6|condo| +34315|AAAAAAAALAGIAAAA|303|North |Ln|Suite 110|Riverdale|Stoddard County|MO|69391|United States|-6|condo| +34316|AAAAAAAAMAGIAAAA|588|North |Avenue|Suite 490|Sunnyside|York County|PA|11952|United States|-5|condo| +34317|AAAAAAAANAGIAAAA|447|East |Dr.|Suite 80|Ashland|Winnebago County|WI|54244|United States|-6|single family| +34318|AAAAAAAAOAGIAAAA|467|4th 4th|Avenue|Suite Q|Clinton|Tensas Parish|LA|78222|United States|-6|condo| +34319|AAAAAAAAPAGIAAAA|162|Spring |Ln|Suite C|New Hope|Jones County|SD|59431|United States|-7|condo| +34320|AAAAAAAAABGIAAAA|479|Maple 1st|Road|Suite 340|Brownsville|Weber County|UT|89310|United States|-7|single family| +34321|AAAAAAAABBGIAAAA|694|2nd |Drive|Suite R|Valley View|Nolan County|TX|75124|United States|-6|apartment| +34322|AAAAAAAACBGIAAAA|217|Park |Road|Suite 340|Providence|Lea County|NM|86614|United States|-7|single family| +34323|AAAAAAAADBGIAAAA|308|Ridge |RD|Suite 400|Hillcrest|Union County|NM|83003|United States|-7|condo| +34324|AAAAAAAAEBGIAAAA|374|Railroad College|Avenue|Suite Y|Mount Pleasant|Gallatin County|KY|41933|United States|-6|single family| +34325|AAAAAAAAFBGIAAAA|235|Woodland |Pkwy|Suite 450|Pleasant Hill|Cavalier County|ND|53604|United States|-6|condo| +34326|AAAAAAAAGBGIAAAA|280|Park |Lane|Suite I|Pleasant Valley|Whitman County|WA|92477|United States|-8|condo| +34327|AAAAAAAAHBGIAAAA|348|Willow |ST|Suite C|Ashland|Wolfe County|KY|44244|United States|-5|condo| +34328|AAAAAAAAIBGIAAAA|789|6th Valley|Drive|Suite 370|Green Acres|Monmouth County|NJ|08283|United States|-5|condo| +34329|AAAAAAAAJBGIAAAA|869|10th Wilson|Blvd|Suite O|River Oaks|Saunders County|NE|68075|United States|-7|condo| +34330|AAAAAAAAKBGIAAAA|997|South 12th|Pkwy|Suite 80|Valley View|Decatur County|TN|35124|United States|-5|condo| +34331|AAAAAAAALBGIAAAA|357|Cedar |Wy|Suite 150|Clinton|Texas County|MO|68222|United States|-6|single family| +34332|AAAAAAAAMBGIAAAA|466|Meadow River|Street|Suite 340|Pleasant Grove|Douglas County|SD|54136|United States|-6|apartment| +34333|AAAAAAAANBGIAAAA|601|Lake Hickory|Way|Suite 30|Friendship|Brown County|IN|44536|United States|-5|apartment| +34334|AAAAAAAAOBGIAAAA|158|12th |Wy|Suite 240|Summit|Calvert County|MD|20499|United States|-5|condo| +34335|AAAAAAAAPBGIAAAA|151|Ninth |Blvd|Suite Q|Fairview|Emery County|UT|85709|United States|-7|condo| +34336|AAAAAAAAACGIAAAA|57|3rd |Circle|Suite S|Mount Pleasant|Dolores County|CO|81933|United States|-7|apartment| +34337|AAAAAAAABCGIAAAA|173|Sunset |Blvd|Suite T|Maple Grove|Garfield County|MT|68252|United States|-7|single family| +34338|AAAAAAAACCGIAAAA|897|9th North|Road|Suite 440|Mount Olive|Alleghany County|VA|28059|United States|-5|apartment| +34339|AAAAAAAADCGIAAAA|656|Cedar Jefferson|Circle|Suite 130|Cedar Grove|Harford County|MD|20411|United States|-5|apartment| +34340|AAAAAAAAECGIAAAA|909|Adams 4th|Ave|Suite N|Harmony|Harper County|OK|75804|United States|-6|condo| +34341|AAAAAAAAFCGIAAAA|861|6th Poplar|Court|Suite 250|Ashland|Owsley County|KY|44244|United States|-5|apartment| +34342|AAAAAAAAGCGIAAAA|949|Center |ST|Suite T|Shiloh|Clarion County|PA|19275|United States|-5|single family| +34343|AAAAAAAAHCGIAAAA|769|Wilson |Blvd|Suite Y|Jackson|Pinellas County|FL|39583|United States|-5|apartment| +34344|AAAAAAAAICGIAAAA|791|Sixth Birch|Dr.|Suite W|Brownsville|Terrell County|TX|79310|United States|-6|condo| +34345|AAAAAAAAJCGIAAAA|713|Valley |Dr.|Suite D|Union|Greene County|OH|48721|United States|-5|condo| +34346|AAAAAAAAKCGIAAAA|729|East |Avenue|Suite 190|Walnut Grove|Lawrence County|AR|77752|United States|-6|single family| +34347|AAAAAAAALCGIAAAA|946|Sycamore |Street|Suite 10|Buena Vista|Twiggs County|GA|35752|United States|-5|single family| +34348|AAAAAAAAMCGIAAAA|894|Smith Ridge|Avenue|Suite X|Red Oak|Marion County|OR|95018|United States|-8|condo| +34349|AAAAAAAANCGIAAAA|145|Forest |Parkway|Suite 320|Riverside|Box Butte County|NE|69231|United States|-6|single family| +34350|AAAAAAAAOCGIAAAA|930|Green |ST|Suite A|Northwood|McKenzie County|ND|54104|United States|-6|single family| +34351|AAAAAAAAPCGIAAAA|882|Mill |Ln|Suite X|Highland|Sierra County|CA|99454|United States|-8|apartment| +34352|AAAAAAAAADGIAAAA|92|Spruce 1st|RD|Suite 40|Nichols|Okanogan County|WA|97940|United States|-8|single family| +34353|AAAAAAAABDGIAAAA|998|Jefferson |Drive|Suite N|Stewart|Yuma County|AZ|88041|United States|-7|apartment| +34354|AAAAAAAACDGIAAAA|773|Maple |Wy|Suite U|Riverside|Bond County|IL|69231|United States|-6|single family| +34355|AAAAAAAADDGIAAAA|381|5th 15th|Parkway|Suite 90|Buena Vista|Kauai County|HI|95752|United States|-10|single family| +34356|AAAAAAAAEDGIAAAA||Hillcrest ||Suite 230|Church Hill||||United States||| +34357|AAAAAAAAFDGIAAAA|461|Maple Hickory|Way|Suite A|Franklin|Sherburne County|MN|59101|United States|-6|single family| +34358|AAAAAAAAGDGIAAAA||Poplar |Lane|Suite L|Waterloo||IN|41675|United States|-5|apartment| +34359|AAAAAAAAHDGIAAAA|258|Willow |Pkwy|Suite C|Woodlawn|Cape Girardeau County|MO|64098|United States|-6|condo| +34360|AAAAAAAAIDGIAAAA|474|1st |Wy|Suite W|Woodville|Logan County|OH|44289|United States|-5|apartment| +34361|AAAAAAAAJDGIAAAA|339|Main |Circle|Suite N|Canaan|Archer County|TX|77713|United States|-6|condo| +34362|AAAAAAAAKDGIAAAA|597|Lake Spruce|Cir.|Suite 340|Five Forks|Santa Cruz County|CA|92293|United States|-8|apartment| +34363|AAAAAAAALDGIAAAA|445|Church Wilson|Parkway|Suite G|Brookville|Howard County|IA|53524|United States|-6|single family| +34364|AAAAAAAAMDGIAAAA|958|Third 4th|Avenue|Suite 330|Fairfield|Powder River County|MT|66192|United States|-7|single family| +34365|AAAAAAAANDGIAAAA|667|College |Court|Suite 490|Lakewood|Polk County|TN|38877|United States|-6|apartment| +34366|AAAAAAAAODGIAAAA|795|Park |ST|Suite 170|Liberty|Mesa County|CO|83451|United States|-7|apartment| +34367|AAAAAAAAPDGIAAAA|354|Main |Avenue|Suite U|Frenchtown|Livingston County|IL|62629|United States|-6|apartment| +34368|AAAAAAAAAEGIAAAA|576|Mill |Ave|Suite S|Elmwood|Oswego County|NY|15097|United States|-5|single family| +34369|AAAAAAAABEGIAAAA|511|Oak Highland|Circle|Suite C|Greenfield|Newport News city|VA|25038|United States|-5|apartment| +34370|AAAAAAAACEGIAAAA|292|Hill |Circle|Suite 320|Hamilton|Gonzales County|TX|72808|United States|-6|single family| +34371|AAAAAAAADEGIAAAA|354|Washington Elm|Ct.|Suite M|Maple Grove|Porter County|IN|48252|United States|-5|apartment| +34372|AAAAAAAAEEGIAAAA|524|Main |Road|Suite 410|Union|Labette County|KS|68721|United States|-6|apartment| +34373|AAAAAAAAFEGIAAAA|429|Broadway Washington|Circle|Suite 200|Oak Ridge|Merced County|CA|98371|United States|-8|condo| +34374|AAAAAAAAGEGIAAAA|438||Lane||Union||AK||United States|-9|| +34375|AAAAAAAAHEGIAAAA|350|Maple |Parkway|Suite Y|Five Forks|Niobrara County|WY|82293|United States|-7|condo| +34376|AAAAAAAAIEGIAAAA|590||Court|Suite Y|Plainview||||||apartment| +34377|AAAAAAAAJEGIAAAA|886|Cherry |Road|Suite X|Jamestown|Cascade County|MT|66867|United States|-7|apartment| +34378|AAAAAAAAKEGIAAAA|492|11th |Ave|Suite 390|Oak Grove|Robertson County|TN|38370|United States|-6|apartment| +34379|AAAAAAAALEGIAAAA|656|Jefferson |Road|Suite 40|Marion|Webster County|MS|50399|United States|-6|apartment| +34380|AAAAAAAAMEGIAAAA|604|4th |Lane|Suite X|Belmont|Blackford County|IN|40191|United States|-5|condo| +34381|AAAAAAAANEGIAAAA|613|Center Lincoln|Street|Suite 260|Mount Olive|Jackson County|NC|28059|United States|-5|single family| +34382|AAAAAAAAOEGIAAAA|299|Second |Wy|Suite O|Wildwood|Knox County|MO|66871|United States|-6|apartment| +34383|AAAAAAAAPEGIAAAA|238|10th Woodland|Street|Suite 310|Clearwater|Pennington County|SD|59534|United States|-7|single family| +34384|AAAAAAAAAFGIAAAA|431|Park |Boulevard|Suite J|Edgewood|Tripp County|SD|50069|United States|-7|apartment| +34385|AAAAAAAABFGIAAAA|987|Hickory 1st|Parkway|Suite 100|Plainview|La Salle Parish|LA|73683|United States|-6|apartment| +34386|AAAAAAAACFGIAAAA|874|Maple Center|Way|Suite 440|Springdale|Schuyler County|NY|18883|United States|-5|single family| +34387|AAAAAAAADFGIAAAA|931|Third |ST|Suite 360|Bethel|Chelan County|WA|95281|United States|-8|apartment| +34388|AAAAAAAAEFGIAAAA|792|4th |Ave|Suite I|Jackson|Denver County|CO|89583|United States|-7|apartment| +34389|AAAAAAAAFFGIAAAA|527|1st |Ln|Suite 380|Lakewood|Jasper County|GA|38877|United States|-5|single family| +34390|AAAAAAAAGFGIAAAA|765|North Adams|Blvd|Suite 280|Wilson|Brooks County|GA|36971|United States|-5|single family| +34391|AAAAAAAAHFGIAAAA|527|Highland |Street|Suite M|Whitesville|Defiance County|OH|45903|United States|-5|condo| +34392|AAAAAAAAIFGIAAAA|820|9th |Street|Suite R|Forest Hills|Morgan County|KY|49237|United States|-5|single family| +34393|AAAAAAAAJFGIAAAA|347|First |Court|Suite 340|Pleasant Valley|Red Willow County|NE|62477|United States|-7|condo| +34394|AAAAAAAAKFGIAAAA|60|Oak Fifth|Drive|Suite R|Bridgeport|Big Stone County|MN|55817|United States|-6|single family| +34395|AAAAAAAALFGIAAAA|928|Fifth |Cir.|Suite Q|Lebanon|Tulare County|CA|92898|United States|-8|single family| +34396|AAAAAAAAMFGIAAAA|386|Willow |Cir.|Suite C|Hopewell|Lac qui Parle County|MN|50587|United States|-6|condo| +34397|AAAAAAAANFGIAAAA|22|View Locust|ST|Suite V|Marion|Orleans Parish|LA|70399|United States|-6|condo| +34398|AAAAAAAAOFGIAAAA|402|Jefferson |ST|Suite R|Georgetown|Dade County|GA|37057|United States|-5|condo| +34399|AAAAAAAAPFGIAAAA|760|Cherry |Court|Suite F|Oakland|Blue Earth County|MN|59843|United States|-6|condo| +34400|AAAAAAAAAGGIAAAA|390|Hill |Pkwy|Suite 400|Georgetown|Ottawa County|OH|47057|United States|-5|condo| +34401|AAAAAAAABGGIAAAA||Oak Third|Court|Suite O|The Meadows||||United States|-7|| +34402|AAAAAAAACGGIAAAA|176|Cedar Locust|Road|Suite B|Red Hill|Gibson County|TN|34338|United States|-5|single family| +34403|AAAAAAAADGGIAAAA|261|Dogwood Ash|Street|Suite I|Stratford|Garfield County|UT|86668|United States|-7|condo| +34404|AAAAAAAAEGGIAAAA|23|Pine |Avenue|Suite G|Five Forks|Colfax County|NE|62293|United States|-6|single family| +34405|AAAAAAAAFGGIAAAA|246|Maple |RD|Suite Q|Maple Grove|Otoe County|NE|68252|United States|-7|condo| +34406|AAAAAAAAGGGIAAAA|440|1st 9th|RD|Suite F|Oak Ridge|Jackson County|IL|68371|United States|-6|apartment| +34407|AAAAAAAAHGGIAAAA|180|Maple |ST|Suite A|Mountain View|Lake County|SD|54466|United States|-7|apartment| +34408|AAAAAAAAIGGIAAAA|881|Ridge South|Wy|Suite C|Hamilton|Caldwell County|TX|72808|United States|-6|condo| +34409|AAAAAAAAJGGIAAAA|587|Fifth |Avenue|Suite F|Hopewell|Edmunds County|SD|50587|United States|-6|condo| +34410|AAAAAAAAKGGIAAAA|139|6th Highland|Court|Suite Y|Sleepy Hollow|Clay County|IL|63592|United States|-6|condo| +34411|AAAAAAAALGGIAAAA|739|1st 10th|Avenue|Suite 10|Hartland|Alleghany County|NC|26594|United States|-5|condo| +34412|AAAAAAAAMGGIAAAA|395|Church |Ave|Suite T|Midway|Morehouse Parish|LA|71904|United States|-6|condo| +34413|AAAAAAAANGGIAAAA|797|Broadway Hill|Lane|Suite F|Hillcrest|Charlton County|GA|33003|United States|-5|single family| +34414|AAAAAAAAOGGIAAAA|743|Mill |Drive|Suite 420|Springdale|Clinch County|GA|38883|United States|-5|single family| +34415|AAAAAAAAPGGIAAAA|990|5th Hillcrest|Avenue|Suite W|Walnut Grove|Sarpy County|NE|67752|United States|-7|single family| +34416|AAAAAAAAAHGIAAAA|557|1st Mill|Wy|Suite 130|Georgetown|Dorchester County|SC|27057|United States|-5|condo| +34417|AAAAAAAABHGIAAAA|266|Center Sunset|Ln|Suite 250|Oak Ridge|Campbell County|SD|58371|United States|-6|apartment| +34418|AAAAAAAACHGIAAAA|608|Cherry |Cir.|Suite 490|Jackson|Cumberland County|IL|69583|United States|-6|apartment| +34419|AAAAAAAADHGIAAAA|71|South |Boulevard|Suite 190|Oakwood|Limestone County|TX|70169|United States|-6|condo| +34420|AAAAAAAAEHGIAAAA|599|14th |Dr.|Suite U|Jackson|Jefferson County|OH|||-5|| +34421|AAAAAAAAFHGIAAAA|571|Pine |Blvd|Suite 80|Kingston|Ray County|MO|64975|United States|-6|apartment| +34422|AAAAAAAAGHGIAAAA|138|Sycamore Laurel|Parkway|Suite 100|Newport|Trego County|KS|61521|United States|-6|condo| +34423|AAAAAAAAHHGIAAAA|558|Birch Washington|Ct.|Suite X|Lebanon|Jefferson County|FL|32898|United States|-5|condo| +34424|AAAAAAAAIHGIAAAA|990|Church |Cir.|Suite 370|Five Forks|Fulton County|IN|42293|United States|-5|condo| +34425|AAAAAAAAJHGIAAAA|585|Johnson 3rd|Pkwy|Suite B|Oak Grove|District of Columbia|DC|28370|United States|-5|condo| +34426|AAAAAAAAKHGIAAAA|963|7th First|Drive|Suite J|Ridgeville|Williamson County|TN|39306|United States|-5|single family| +34427|AAAAAAAALHGIAAAA|500|Johnson Hill|Lane|Suite T|Maple Grove|Slope County|ND|58252|United States|-6|single family| +34428|AAAAAAAAMHGIAAAA|364|Forest |Circle|Suite R|Roy|Hamilton County|OH|40744|United States|-5|apartment| +34429|AAAAAAAANHGIAAAA|640|9th |Blvd|Suite I|Springdale|Caddo County|OK|78883|United States|-6|condo| +34430|AAAAAAAAOHGIAAAA|469|Walnut |Avenue|Suite 440|Thompsonville|Bacon County|GA|39651|United States|-5|apartment| +34431|AAAAAAAAPHGIAAAA|796|Franklin 13th|Blvd|Suite 340|Riverdale|Rolette County|ND|59391|United States|-6|apartment| +34432|AAAAAAAAAIGIAAAA|63|Second |Parkway|Suite C|Bethel|Taylor County|KY|45281|United States|-5|condo| +34433|AAAAAAAABIGIAAAA|677|Sunset Maple|Street|Suite A|Cedar Grove|Cook County|MN|50411|United States|-6|condo| +34434|AAAAAAAACIGIAAAA|821|3rd Walnut|Ct.|Suite 450|Springfield|Jefferson Parish|LA|79303|United States|-6|condo| +34435|AAAAAAAADIGIAAAA|375|1st Lee|Avenue|Suite Y|Lincoln|Linn County|KS|61289|United States|-6|single family| +34436|AAAAAAAAEIGIAAAA|169|Willow Hill|Blvd|Suite 400|Georgetown|Kiowa County|KS|67057|United States|-6|single family| +34437|AAAAAAAAFIGIAAAA|564|Walnut |Ct.|Suite 210|Newtown|Carroll County|IN|41749|United States|-5|apartment| +34438|AAAAAAAAGIGIAAAA|597|Adams 2nd|Dr.|Suite I|Fairfield|Magoffin County|KY|46192|United States|-5|single family| +34439|AAAAAAAAHIGIAAAA|146|Wilson 1st|Road|Suite 380|Forest Hills|Pawnee County|KS|69237|United States|-6|single family| +34440|AAAAAAAAIIGIAAAA|524|Elm Thirteenth|Ave|Suite W|Highland Park|Carter County|OK|76534|United States|-6|apartment| +34441|AAAAAAAAJIGIAAAA|706|Maple |Wy|Suite 360|Riverview|Clinch County|GA|39003|United States|-5|single family| +34442|AAAAAAAAKIGIAAAA|775|Hillcrest |Avenue|Suite 140|Hopewell|Charles City County|VA|20587|United States|-5|single family| +34443|AAAAAAAALIGIAAAA|938|Hillcrest Meadow|Cir.|Suite Y|Bethel|Louisa County|IA|55281|United States|-6|condo| +34444|AAAAAAAAMIGIAAAA|625|Park |Cir.|Suite Q|Wildwood|Kennebec County|ME|07471|United States|-5|condo| +34445|AAAAAAAANIGIAAAA|769|Lincoln |ST|Suite K|Riverview|Carroll County|IL|69003|United States|-6|condo| +34446|AAAAAAAAOIGIAAAA|793|Sunset Wilson|ST|Suite N|Waterloo|McCurtain County|OK|71675|United States|-6|condo| +34447|AAAAAAAAPIGIAAAA|974|12th Lakeview|Drive|Suite F|Hamilton|Grays Harbor County|WA|92808|United States|-8|single family| +34448|AAAAAAAAAJGIAAAA|||Lane|Suite 10|||VT||United States|-5|| +34449|AAAAAAAABJGIAAAA|349|Lincoln West|Blvd|Suite 330|Providence|Jefferson County|OK|76614|United States|-6|apartment| +34450|AAAAAAAACJGIAAAA|709|Cherry |Circle|Suite S|Springdale|Hamlin County|SD|58883|United States|-7|condo| +34451|AAAAAAAADJGIAAAA|667|15th |RD|Suite Q|Bunker Hill|Carlton County|MN|50150|United States|-6|apartment| +34452|AAAAAAAAEJGIAAAA|156|North Central|Pkwy|Suite T|New Hope|Gilmer County|GA|39431|United States|-5|condo| +34453|AAAAAAAAFJGIAAAA|392|East Mill|Lane|Suite O|Wilson|Orange County|VA|26971|United States|-5|condo| +34454|AAAAAAAAGJGIAAAA|546|Railroad |ST|Suite 370|Waterloo|Union County|NM|81675|United States|-7|apartment| +34455|AAAAAAAAHJGIAAAA|867|3rd Spring|Boulevard|Suite C|Stringtown|Clark County|IN|40162|United States|-5|condo| +34456|AAAAAAAAIJGIAAAA|367|Washington |Parkway|Suite 30|Brownsville|Boone County|IA|59310|United States|-6|condo| +34457|AAAAAAAAJJGIAAAA|419|Ridge Hill|Avenue|Suite Y|Lewis|Garrett County|MD|27066|United States|-5|condo| +34458|AAAAAAAAKJGIAAAA|641|View Davis|Ln|Suite 490|Cedar Grove|Aitkin County|MN|50411|United States|-6|condo| +34459|AAAAAAAALJGIAAAA|894|Cedar View|Ct.|Suite 240|Evans|Grundy County|IA|52284|United States|-6|single family| +34460|AAAAAAAAMJGIAAAA|989|Church |Avenue|Suite 320|Colonial Heights|Livingston County|KY|43425|United States|-5|condo| +34461|AAAAAAAANJGIAAAA|453|14th Spruce|Circle|Suite 290|New Hope|Lincoln County|CO|89431|United States|-7|apartment| +34462|AAAAAAAAOJGIAAAA|522|Lake |Wy|Suite 240|Edgewood|Palm Beach County|FL|30069|United States|-5|apartment| +34463|AAAAAAAAPJGIAAAA|298|Fourteenth |Blvd|Suite X|Springdale|Benewah County|ID|88883|United States|-7|condo| +34464|AAAAAAAAAKGIAAAA|430|Davis Franklin|ST|Suite Q|Lebanon|Pike County|IN|42898|United States|-5|condo| +34465|AAAAAAAABKGIAAAA|||||Buena Vista|Yavapai County|||United States||| +34466|AAAAAAAACKGIAAAA|586|First Smith|Ct.|Suite D|New Hope|Jasper County|IL|69431|United States|-6|condo| +34467|AAAAAAAADKGIAAAA|391|11th View|Lane|Suite P|Jamestown|Grayson County|KY|46867|United States|-6|condo| +34468|AAAAAAAAEKGIAAAA|916|6th North|Circle|Suite 100|Hamilton|Elbert County|GA|32808|United States|-5|single family| +34469|AAAAAAAAFKGIAAAA|82|View |Ave||Ashland|||64244|United States||| +34470|AAAAAAAAGKGIAAAA|23|Pine 15th|Boulevard|Suite A|Woodland|Williamsburg city|VA|24854|United States|-5|apartment| +34471|AAAAAAAAHKGIAAAA|350|Adams Lee|Wy|Suite A|Sulphur Springs|Fort Bend County|TX|78354|United States|-6|apartment| +34472|AAAAAAAAIKGIAAAA|657|5th Birch|Pkwy|Suite 400|Edwards|Gray County|TX|71409|United States|-6|condo| +34473|AAAAAAAAJKGIAAAA|907|Forest 2nd|Circle|Suite V|Pleasant Valley|Real County|TX|72477|United States|-6|condo| +34474|AAAAAAAAKKGIAAAA|105|Meadow Oak|Way|Suite 360|Franklin|Monroe County|MS|59101|United States|-6|condo| +34475|AAAAAAAALKGIAAAA|539|Adams Franklin|Lane|Suite U|Thompsonville|Midland County|MI|49651|United States|-5|single family| +34476|AAAAAAAAMKGIAAAA|663|Johnson |ST|Suite B|Crossroads|Wallace County|KS|60534|United States|-6|apartment| +34477|AAAAAAAANKGIAAAA|97|Lake |Avenue|Suite O|Spring Hill|Ford County|KS|66787|United States|-6|condo| +34478|AAAAAAAAOKGIAAAA|811|Willow West|Pkwy|Suite 100|Providence|Ford County|KS|66614|United States|-6|condo| +34479|AAAAAAAAPKGIAAAA|433|Sycamore South|Boulevard|Suite X|Antioch|Juab County|UT|88605|United States|-7|apartment| +34480|AAAAAAAAALGIAAAA|503|Highland 9th|Ln|Suite F|Oakland|Jackson County|KY|49843|United States|-6|apartment| +34481|AAAAAAAABLGIAAAA|674|2nd |RD|Suite U|Lakeview|Portage County|OH|48579|United States|-5|apartment| +34482|AAAAAAAACLGIAAAA|908|Park |Lane|Suite S|Clifton|Harris County|GA|38014|United States|-5|single family| +34483|AAAAAAAADLGIAAAA|325|10th Hickory|Boulevard|Suite 210|Mountain View|Donley County|TX|74466|United States|-6|single family| +34484|AAAAAAAAELGIAAAA|547||||Farmington||ID|89145|||| +34485|AAAAAAAAFLGIAAAA|155|Sunset 10th|Way|Suite L|Washington Heights|Seward County|NE|68167|United States|-7|single family| +34486|AAAAAAAAGLGIAAAA|888|Pine |Circle|Suite W|Liberty|Hampden County|MA|04051|United States|-5|condo| +34487|AAAAAAAAHLGIAAAA|899|Highland |Drive|Suite 350|Jamestown|Douglas County|GA|36867|United States|-5|single family| +34488|AAAAAAAAILGIAAAA|309|Johnson 2nd|Ct.|Suite 360|Antioch|San Patricio County|TX|78605|United States|-6|apartment| +34489|AAAAAAAAJLGIAAAA|317|3rd |Pkwy|Suite O|Riverdale|Colleton County|SC|29391|United States|-5|apartment| +34490|AAAAAAAAKLGIAAAA|988|Jefferson Woodland|ST|Suite P|Fox|Gwinnett County|GA|30631|United States|-5|apartment| +34491|AAAAAAAALLGIAAAA|226|13th Johnson|Street|Suite 240|Waterloo|Sutter County|CA|91675|United States|-8|condo| +34492|AAAAAAAAMLGIAAAA|327|Sunset Laurel|Ct.|Suite 280|Ashland|Benton County|MO|64244|United States|-6|single family| +34493|AAAAAAAANLGIAAAA|786|1st First|Pkwy|Suite 190|Shiloh|Crawford County|IL|69275|United States|-6|condo| +34494|AAAAAAAAOLGIAAAA|846|Miller 9th|Ln|Suite S|Hamilton|McCone County|MT|62808|United States|-7|apartment| +34495|AAAAAAAAPLGIAAAA|741|15th Cedar|Drive|Suite 360|Riverdale|Dubuque County|IA|59391|United States|-6|apartment| +34496|AAAAAAAAAMGIAAAA|59|Wilson Elm|Lane|Suite 480|Mount Vernon|Benton County|MO|68482|United States|-6|apartment| +34497|AAAAAAAABMGIAAAA|815|Hickory Fifth|||Needmore||LA||||| +34498|AAAAAAAACMGIAAAA|126|North View|Avenue|Suite A|Clifton|Shasta County|CA|98014|United States|-8|single family| +34499|AAAAAAAADMGIAAAA|766|Sunset |Wy|Suite Q|Hillcrest|Shawano County|WI|53003|United States|-6|single family| +34500|AAAAAAAAEMGIAAAA|831|Maple 8th|Street|Suite 440|Edgewood|Bear Lake County|ID|80069|United States|-7|single family| +34501|AAAAAAAAFMGIAAAA|950||Ct.||Sleepy Hollow|Potter County|TX|73592|United States||| +34502|AAAAAAAAGMGIAAAA|110|Sixth Johnson|Ave|Suite 40|Newtown|Burleigh County|ND|51749|United States|-6|condo| +34503|AAAAAAAAHMGIAAAA|5|Hillcrest Davis|RD|Suite V|Shady Grove|Bullock County|AL|32812|United States|-6|single family| +34504|AAAAAAAAIMGIAAAA|706|Cherry Park|RD|Suite C|Macedonia|Madison County|MT|61087|United States|-7|apartment| +34505|AAAAAAAAJMGIAAAA|812|East Lincoln|Court|Suite T|Mountain View|Deaf Smith County|TX|74466|United States|-6|condo| +34506|AAAAAAAAKMGIAAAA|758|Chestnut |Circle|Suite L|Oakland|Allen Parish|LA|79843|United States|-6|condo| +34507|AAAAAAAALMGIAAAA|323|6th |RD|Suite 310|Oak Hill|Washington County|MD|27838|United States|-5|single family| +34508|AAAAAAAAMMGIAAAA|865|Main Broadway|Dr.|Suite 470|Forest Hills|Crawford County|IL|69237|United States|-6|condo| +34509|AAAAAAAANMGIAAAA|||Blvd|Suite O|Oak Hill|Lee County|GA|37838||-5|single family| +34510|AAAAAAAAOMGIAAAA|808|Maple Valley|Road|Suite 290|Riverdale|Stanley County|SD|59391|United States|-7|apartment| +34511|AAAAAAAAPMGIAAAA|187|1st 9th|Parkway|Suite E|Providence|Chelan County|WA|96614|United States|-8|condo| +34512|AAAAAAAAANGIAAAA|214|Mill Cedar|ST|Suite V|Midway|Newton County|IN|41904|United States|-5|condo| +34513|AAAAAAAABNGIAAAA|450|Pine North|Ln|Suite K|Riverdale|Hampden County|MA|09991|United States|-5|condo| +34514|AAAAAAAACNGIAAAA|933|Smith Franklin|Way|Suite K|Crossroads|Ward County|TX|70534|United States|-6|condo| +34515|AAAAAAAADNGIAAAA|444|South North|Ct.|Suite Q|Oak Grove|Greensville County|VA|28370|United States|-5|apartment| +34516|AAAAAAAAENGIAAAA|238|Lakeview |Blvd|Suite 260|Clinton|Dane County|WI|58222|United States|-6|single family| +34517|AAAAAAAAFNGIAAAA|797|13th |Dr.|Suite 130|Mount Vernon|Scott County|TN|38482|United States|-6|single family| +34518|AAAAAAAAGNGIAAAA|555|Mill 1st|Court|Suite J|Woodville|Hancock County|IL|64289|United States|-6|apartment| +34519|AAAAAAAAHNGIAAAA|426|Spring |Street|Suite 410|Barnes|Putnam County|WV|23788|United States|-5|apartment| +34520|AAAAAAAAINGIAAAA|644|13th Woodland|Circle|Suite 90|Highland|Covington County|AL|39454|United States|-6|condo| +34521|AAAAAAAAJNGIAAAA|637|Lincoln |Way|Suite U|Fairview|Fentress County|TN|35709|United States|-5|single family| +34522|AAAAAAAAKNGIAAAA|798|View |Circle|Suite 200|Oak Ridge|Coleman County|TX|78371|United States|-6|condo| +34523|AAAAAAAALNGIAAAA|300|Ninth |Avenue|Suite 60|Concord|Montgomery County|TN|34107|United States|-6|single family| +34524|AAAAAAAAMNGIAAAA|950|South View|Way|Suite 30|Shady Grove|Lincoln County|MS|52812|United States|-6|apartment| +34525|AAAAAAAANNGIAAAA|600|12th |Ct.|Suite 390|Woodville|Champaign County|OH|44289|United States|-5|condo| +34526|AAAAAAAAONGIAAAA|646|15th 5th|Road|Suite 90|Harmony|Jackson County|KY|45804|United States|-6|single family| +34527|AAAAAAAAPNGIAAAA|516|Dogwood Park|Street|Suite O|Belmont|Atascosa County|TX|70191|United States|-6|condo| +34528|AAAAAAAAAOGIAAAA|863|12th |Ave|Suite E|Avoca|North Slope Borough|AK|90540|United States|-9|apartment| +34529|AAAAAAAABOGIAAAA|454|West |Dr.|Suite C|Lakewood|Doddridge County|WV|28877|United States|-5|single family| +34530|AAAAAAAACOGIAAAA|769|6th Dogwood|Avenue|Suite 470|Concord|Lafayette County|AR|74107|United States|-6|single family| +34531|AAAAAAAADOGIAAAA|65|14th |Lane|Suite K|Greenfield|Charlotte County|FL|35038|United States|-5|condo| +34532|AAAAAAAAEOGIAAAA|363|Main Washington|Blvd|Suite K|Brownsville|Rutherford County|NC|29310|United States|-5|condo| +34533|AAAAAAAAFOGIAAAA|898|Park 6th|Ct.|Suite 230|Lakeside|Northampton County|VA|29532|United States|-5|apartment| +34534|AAAAAAAAGOGIAAAA|986|7th Jackson|Circle|Suite V|Antioch|Montgomery County|TX|78605|United States|-6|condo| +34535|AAAAAAAAHOGIAAAA|137|Franklin 6th|Cir.|Suite Y|Mount Vernon|Gem County|ID|88482|United States|-7|apartment| +34536|AAAAAAAAIOGIAAAA|585|Elevnth |Blvd|Suite X|California|Buffalo County|WI|50141|United States|-6|condo| +34537|AAAAAAAAJOGIAAAA|542|Maple 6th|Blvd|Suite S|Oak Grove|Adair County|MO|68370|United States|-6|condo| +34538|AAAAAAAAKOGIAAAA|628|Meadow Sunset|Circle|Suite 210|Deerfield|Jackson County|AR|79840|United States|-6|condo| +34539|AAAAAAAALOGIAAAA||Eigth 15th|Cir.|||Trimble County|KY|49310|United States|-5|| +34540|AAAAAAAAMOGIAAAA|602|5th Park|Avenue|Suite X|Highland|Lincoln County|GA|39454|United States|-5|single family| +34541|AAAAAAAANOGIAAAA|662|4th |Circle|Suite 210|Harmony|Wabasha County|MN|55804|United States|-6|single family| +34542|AAAAAAAAOOGIAAAA|551||Cir.|Suite J|Maple Grove|||68252||-6|| +34543|AAAAAAAAPOGIAAAA|620|Spring Lake|Road|Suite B|Hillcrest|Madison County|MS|53003|United States|-6|single family| +34544|AAAAAAAAAPGIAAAA|970|Spring |ST|Suite V|Sulphur Springs|Wright County|MN|58354|United States|-6|condo| +34545|AAAAAAAABPGIAAAA|57|Pine Spruce|Ct.|Suite 100|Union Hill|Douglas County|CO|87746|United States|-7|condo| +34546|AAAAAAAACPGIAAAA|765|Fourth Laurel|Lane|Suite Y|Macedonia|Towner County|ND|51087|United States|-6|single family| +34547|AAAAAAAADPGIAAAA|257|River Forest|Circle|Suite I|Mount Zion|Stone County|MO|68054|United States|-6|apartment| +34548|AAAAAAAAEPGIAAAA|212|Valley |Way|Suite 190|Harmony|Itawamba County|MS|55804|United States|-6|single family| +34549|AAAAAAAAFPGIAAAA|649|Meadow 8th|Ln|Suite L|Farmington|Howard County|MO|69145|United States|-6|single family| +34550|AAAAAAAAGPGIAAAA|865|7th |Lane|Suite 80|Greenville|Wakulla County|FL|31387|United States|-5|single family| +34551|AAAAAAAAHPGIAAAA|954|2nd Washington|Avenue|Suite 490|Unionville|Harrison County|MO|61711|United States|-6|apartment| +34552|AAAAAAAAIPGIAAAA|||Pkwy||||||United States||| +34553|AAAAAAAAJPGIAAAA|279|Central 7th|Pkwy|Suite A|Greenfield|Kent County|MI|45038|United States|-5|single family| +34554|AAAAAAAAKPGIAAAA|556|5th |Pkwy|Suite Y|Spring Valley|Elliott County|KY|46060|United States|-6|single family| +34555|AAAAAAAALPGIAAAA|493|6th |ST|Suite R|Bridgeport|Dakota County|MN|55817|United States|-6|single family| +34556|AAAAAAAAMPGIAAAA|187|Main |Ct.|Suite G|New Hope|Dawes County|NE|69431|United States|-6|condo| +34557|AAAAAAAANPGIAAAA|965|Lake |Road|Suite G|Five Forks|Turner County|GA|32293|United States|-5|apartment| +34558|AAAAAAAAOPGIAAAA|306|Hill Pine|ST|Suite Y|Newport|Karnes County|TX|71521|United States|-6|apartment| +34559|AAAAAAAAPPGIAAAA|847|Sunset Railroad|Pkwy|Suite 20|Highland Park|Reno County|KS|66534|United States|-6|condo| +34560|AAAAAAAAAAHIAAAA|446|1st |Road|Suite 490|Red Hill|Tompkins County|NY|14338|United States|-5|single family| +34561|AAAAAAAABAHIAAAA|810|Spring Jefferson|Circle|Suite 50|Mount Zion|Franklin County|WA|98054|United States|-8|condo| +34562|AAAAAAAACAHIAAAA|693|Elm |Blvd|Suite G|Bethel|Logan County|OK|75281|United States|-6|apartment| +34563|AAAAAAAADAHIAAAA|709|Park |Circle|Suite D|Lakewood|San Benito County|CA|98877|United States|-8|condo| +34564|AAAAAAAAEAHIAAAA|641|Johnson Park|Ave|Suite O|Ryan|Richland Parish|LA|70525|United States|-6|apartment| +34565|AAAAAAAAFAHIAAAA|837|Broadway |Parkway|Suite K|Hillcrest|Douglas County|IL|63003|United States|-6|condo| +34566|AAAAAAAAGAHIAAAA|611|Chestnut Elm|Circle|Suite I|Mount Vernon|Goliad County|TX|78482|United States|-6|condo| +34567|AAAAAAAAHAHIAAAA|973|West |RD|Suite 460|Jackson|Bureau County|IL|69583|United States|-6|single family| +34568|AAAAAAAAIAHIAAAA|96|Chestnut Washington|Ave|Suite Y|Oakland|Lincoln County|OK|79843|United States|-6|single family| +34569|AAAAAAAAJAHIAAAA|757|East |Way|Suite 470|Flint|Carbon County|PA|18909|United States|-5|apartment| +34570|AAAAAAAAKAHIAAAA|212|First |ST|Suite T|Florence|Reynolds County|MO|63394|United States|-6|single family| +34571|AAAAAAAALAHIAAAA|697|Sunset |Blvd|Suite 170|Sulphur Springs|Shelby County|KY|48354|United States|-5|condo| +34572|AAAAAAAAMAHIAAAA|245|First |ST|Suite 230|Kingston|Calhoun County|FL|34975|United States|-5|apartment| +34573|AAAAAAAANAHIAAAA|707|Williams |Ave|Suite 460|Concord|Garvin County|OK|74107|United States|-6|apartment| +34574|AAAAAAAAOAHIAAAA|16|Cedar Hickory|Blvd|Suite F|Clifton|Lunenburg County|VA|28014|United States|-5|single family| +34575|AAAAAAAAPAHIAAAA|185|First |Dr.|Suite 230|Five Forks|Warren County|MO|62293|United States|-6|single family| +34576|AAAAAAAAABHIAAAA|277|Tenth |Road|Suite 430|Fairfield|Hillsborough County|FL|36192|United States|-5|condo| +34577|AAAAAAAABBHIAAAA|44|Franklin Fourth|Ave|Suite M|Pleasant Hill|East Baton Rouge Parish|LA|73604|United States|-6|apartment| +34578|AAAAAAAACBHIAAAA|381|4th Wilson|Drive|Suite 100|Marion|Fairfax County|VA|20399|United States|-5|single family| +34579|AAAAAAAADBHIAAAA|748|Jackson |Blvd|Suite G|Salem|Lee County|AL|38048|United States|-6|condo| +34580|AAAAAAAAEBHIAAAA|935|Sunset Sunset|Ct.|Suite 20|Unionville|Slope County|ND|51711|United States|-6|single family| +34581|AAAAAAAAFBHIAAAA|774|Center Elm|Ln|Suite H|Springdale|Scott County|MS|58883|United States|-6|apartment| +34582|AAAAAAAAGBHIAAAA|782|11th Lakeview|Avenue|Suite Q|Maple Grove|Campbell County|VA|28252|United States|-5|condo| +34583|AAAAAAAAHBHIAAAA|695|Forest Valley|Ct.|Suite I|Centerville|Furnas County|NE|60059|United States|-6|apartment| +34584|AAAAAAAAIBHIAAAA|351|Fourteenth Cedar|Circle|Suite 240|Hopewell|Fairbanks North Star Borough|AK|90587|United States|-9|apartment| +34585|AAAAAAAAJBHIAAAA|846|Miller |Boulevard|Suite 110|Woodville|Musselshell County|MT|64289|United States|-7|single family| +34586|AAAAAAAAKBHIAAAA|137|Mill |Street|Suite V|New Hope|McDowell County|WV|29431|United States|-5|apartment| +34587|AAAAAAAALBHIAAAA|24|Park Woodland|Way|Suite 470|Concord|Alamosa County|CO|84107|United States|-7|apartment| +34588|AAAAAAAAMBHIAAAA|902|Meadow |ST|Suite I|Union Hill|Dallas County|AL|37746|United States|-6|condo| +34589|AAAAAAAANBHIAAAA|197|Lake 2nd||||||38252|United States||| +34590|AAAAAAAAOBHIAAAA|175|Center |Ave|Suite 460|Greenwood|Crane County|TX|78828|United States|-6|single family| +34591|AAAAAAAAPBHIAAAA|859|Pine |Ct.|Suite 450|Lakeside|Switzerland County|IN|49532|United States|-5|single family| +34592|AAAAAAAAACHIAAAA||Dogwood |Pkwy|Suite 440||||||-7|| +34593|AAAAAAAABCHIAAAA|699|Walnut |ST|Suite C|Hopewell|Licking County|OH|||-5|| +34594|AAAAAAAACCHIAAAA|814|12th Williams|Court|Suite 90|Liberty|Panola County|MS|53451|United States|-6|single family| +34595|AAAAAAAADCHIAAAA|828|Thirteenth |Parkway|Suite R|Shady Grove|Coffee County|AL|32812|United States|-6|single family| +34596|AAAAAAAAECHIAAAA|702|Woodland |Avenue|Suite 250|Marion|Carter County|OK|70399|United States|-6|apartment| +34597|AAAAAAAAFCHIAAAA|127|Lake |Ave|Suite 10|Friendship|Hinds County|MS|54536|United States|-6|single family| +34598|AAAAAAAAGCHIAAAA|587|Fifth |Dr.|Suite E|Summit|Cowley County|KS|60499|United States|-6|apartment| +34599|AAAAAAAAHCHIAAAA|273|Lee Third|Lane|Suite 220|Plainview|Rockbridge County|VA|23683|United States|-5|apartment| +34600|AAAAAAAAICHIAAAA|970|Williams Pine|Parkway|Suite 480|Springdale|Wahkiakum County|WA|98883|United States|-8|single family| +34601|AAAAAAAAJCHIAAAA|245|3rd |Lane|Suite 50|Oak Grove|Martin County|FL|38370|United States|-5|apartment| +34602|AAAAAAAAKCHIAAAA|248|Railroad Williams|Cir.|Suite 310|Pleasant Grove|Jefferson County|MS|54136|United States|-6|condo| +34603|AAAAAAAALCHIAAAA|113|Adams |RD|Suite 470|Hartland|Putnam County|WV|26594|United States|-5|apartment| +34604|AAAAAAAAMCHIAAAA|677|1st |Parkway|Suite 490|Springville|Jefferson Parish|LA|70702|United States|-6|condo| +34605|AAAAAAAANCHIAAAA|698|Church |Avenue|Suite X|Glendale|Wilkes County|NC|23951|United States|-5|condo| +34606|AAAAAAAAOCHIAAAA|42|College Elevnth|ST|Suite R|Riverside|Fort Bend County|TX|79231|United States|-6|apartment| +34607|AAAAAAAAPCHIAAAA|777|Walnut |Drive|Suite N|Flint|Hancock County|IN|48909|United States|-5|apartment| +34608|AAAAAAAAADHIAAAA|483|5th |Road|Suite 290|Bunker Hill|Madison County|MT|60150|United States|-7|condo| +34609|AAAAAAAABDHIAAAA|34|3rd Main|ST|Suite Y|Walnut Grove|Cherokee County|TX|77752|United States|-6|single family| +34610|AAAAAAAACDHIAAAA|591|13th |Pkwy|Suite 110|Forest Hills|Hughes County|OK|79237|United States|-6|single family| +34611|AAAAAAAADDHIAAAA|419|2nd |Boulevard|Suite T|Ashland|Henry County|MO|64244|United States|-6|condo| +34612|AAAAAAAAEDHIAAAA|635|Maple Maple|Ave|Suite 380|Green Acres|Shannon County|MO|67683|United States|-6|apartment| +34613|AAAAAAAAFDHIAAAA|270|10th 2nd|Parkway|Suite C|Shady Grove|Sully County|SD|52812|United States|-7|condo| +34614|AAAAAAAAGDHIAAAA|314|Maple |RD|Suite Q|Marion|Vernon County|MO|60399|United States|-6|condo| +34615|AAAAAAAAHDHIAAAA|119|Maple Williams|Blvd|Suite 120|Riverdale|Polk County|MO|69391|United States|-6|apartment| +34616|AAAAAAAAIDHIAAAA|620|Willow Fifth|RD|Suite 110|Deerfield|Stephens County|TX|79840|United States|-6|condo| +34617|AAAAAAAAJDHIAAAA||Walnut 8th|Lane|Suite 80|Hillcrest|McDowell County|||United States||| +34618|AAAAAAAAKDHIAAAA|747|Green |Road|Suite 370|Valley View|Harrison County|TX|75124|United States|-6|apartment| +34619|AAAAAAAALDHIAAAA|910|Poplar |Ct.|Suite 230|Riverdale|Macon County|NC|29391|United States|-5|condo| +34620|AAAAAAAAMDHIAAAA|430|Ridge |RD|Suite 230|Hamilton|McCreary County|KY|42808|United States|-5|condo| +34621|AAAAAAAANDHIAAAA|693|Lake |Street|Suite V|Oakdale|Washington County|TN|39584|United States|-6|condo| +34622|AAAAAAAAODHIAAAA|259|Maple |Cir.|Suite 310|Sulphur Springs|Hinds County|MS|58354|United States|-6|single family| +34623|AAAAAAAAPDHIAAAA|864|Miller 11th|Dr.|Suite S|Green Acres|Caledonia County|VT|08283|United States|-5|apartment| +34624|AAAAAAAAAEHIAAAA|297|Johnson |Way|Suite 10|Hillcrest|Traverse County|MN|53003|United States|-6|single family| +34625|AAAAAAAABEHIAAAA|500|Oak |Drive|Suite 260|Millbrook|Seneca County|NY|17529|United States|-5|apartment| +34626|AAAAAAAACEHIAAAA|387|Elm Willow|Circle|Suite 470|Vienna|Collingsworth County|TX|75119|United States|-6|apartment| +34627|AAAAAAAADEHIAAAA|81|Ash |Boulevard|Suite C|Woodlawn|Tyler County|WV|24098|United States|-5|condo| +34628|AAAAAAAAEEHIAAAA|478|7th Davis|RD|Suite 450|Mount Zion|Navajo County|AZ|88054|United States|-7|single family| +34629|AAAAAAAAFEHIAAAA|596|Hill |Road|Suite K|Bridgeport|Somervell County|TX|75817|United States|-6|condo| +34630|AAAAAAAAGEHIAAAA|719|Tenth |Circle|Suite P|Woodlawn|Gwinnett County|GA|34098|United States|-5|condo| +34631|AAAAAAAAHEHIAAAA|503|Oak |Street|Suite 40|Harmony|Kiowa County|KS|65804|United States|-6|single family| +34632|AAAAAAAAIEHIAAAA|19|Sunset |Pkwy|Suite Y|Highland Park|Butler County|KS|66534|United States|-6|condo| +34633|AAAAAAAAJEHIAAAA|707|South |Court|Suite 70|Jamestown|Black Hawk County|IA|56867|United States|-6|single family| +34634|AAAAAAAAKEHIAAAA|92|View |Street|Suite F|Lakeside|Broadwater County|MT|69532|United States|-7|condo| +34635|AAAAAAAALEHIAAAA|828|3rd |Street|Suite F|Midway|Chautauqua County|KS|61904|United States|-6|apartment| +34636|AAAAAAAAMEHIAAAA|978|Highland Poplar|Lane|Suite S|Murphy|Bartholomew County|IN|42105|United States|-5|single family| +34637|AAAAAAAANEHIAAAA|195|2nd |Blvd|Suite 130|Whitesville|Duval County|TX|75903|United States|-6|single family| +34638|AAAAAAAAOEHIAAAA|284|Locust |Ct.|Suite 40|Fairfield|Bulloch County|GA|36192|United States|-5|single family| +34639|AAAAAAAAPEHIAAAA|347|North First|Way|Suite 140|Hopewell|Crawford County|IL|60587|United States|-6|single family| +34640|AAAAAAAAAFHIAAAA|268|View |Lane|Suite T|Montpelier|Calhoun County|TX|78930|United States|-6|apartment| +34641|AAAAAAAABFHIAAAA|82|14th Adams|Ave|Suite R|Ashland|Isanti County|MN|54244|United States|-6|single family| +34642|AAAAAAAACFHIAAAA|218|Main |Drive|Suite 130|Oak Grove|Steele County|MN|58370|United States|-6|apartment| +34643|AAAAAAAADFHIAAAA|952|1st |Road|Suite 140|Wilson|Dickinson County|MI|46971|United States|-5|apartment| +34644|AAAAAAAAEFHIAAAA|784|Birch |Pkwy|Suite 470|Macedonia|Conejos County|CO|81087|United States|-7|apartment| +34645|AAAAAAAAFFHIAAAA|575|Spring 8th|Wy|Suite 100|Marion|Douglas County|IL|60399|United States|-6|single family| +34646|AAAAAAAAGFHIAAAA|197|Sycamore 10th|Circle|Suite U|Union Hill|Neshoba County|MS|57746|United States|-6|single family| +34647|AAAAAAAAHFHIAAAA|911|Walnut |Pkwy|Suite 450|Oak Ridge|Apache County|AZ|88371|United States|-7|single family| +34648|AAAAAAAAIFHIAAAA|780|||||Linn County|MO|65038|||condo| +34649|AAAAAAAAJFHIAAAA|356|Franklin |Drive|Suite Y|Wildwood|McDonald County|MO|66871|United States|-6|single family| +34650|AAAAAAAAKFHIAAAA|573|Ridge Hillcrest|Drive|Suite O|White Oak|Burnett County|WI|56668|United States|-6|apartment| +34651|AAAAAAAALFHIAAAA|868|Forest |Street|Suite S|Salem|Lenoir County|NC|28048|United States|-5|apartment| +34652|AAAAAAAAMFHIAAAA|323|Church |Ln|Suite 480|Buena Vista|Judith Basin County|MT|65752|United States|-7|apartment| +34653|AAAAAAAANFHIAAAA|427|Mill |Lane|Suite 310|Pine Grove|Willacy County|TX|74593|United States|-6|apartment| +34654|AAAAAAAAOFHIAAAA|476|Sycamore 15th|Boulevard|Suite S|Highland Park|Wood County|OH|46534|United States|-5|condo| +34655|AAAAAAAAPFHIAAAA|340|East Ash|Court|Suite 160|Enterprise|Salem city|VA|21757|United States|-5|condo| +34656|AAAAAAAAAGHIAAAA|523|Cedar South|Lane|Suite 390|New Hope|Jackson County|FL|39431|United States|-5|apartment| +34657|AAAAAAAABGHIAAAA|624|Railroad Laurel|Boulevard|Suite F|Farmington|Carroll County|IA|59145|United States|-6|condo| +34658|AAAAAAAACGHIAAAA|389|5th Main|Cir.|Suite L|Ludlow|Shelby County|KY|45566|United States|-5|condo| +34659|AAAAAAAADGHIAAAA|805|Madison |Pkwy|Suite 210|Lincoln|San Miguel County|NM|81289|United States|-7|condo| +34660|AAAAAAAAEGHIAAAA|26|Ridge |Boulevard|Suite 140|Green Acres|Des Moines County|IA|57683|United States|-6|apartment| +34661|AAAAAAAAFGHIAAAA|360|North |Ave|Suite I|Riverview|Roberts County|TX|79003|United States|-6|apartment| +34662|AAAAAAAAGGHIAAAA|760|Poplar |Ct.|Suite U|Cherry Valley|Lauderdale County|TN|30854|United States|-6|condo| +34663|AAAAAAAAHGHIAAAA|739|Walnut 9th|Dr.|Suite 340|Enterprise|Hamilton County|FL|31757|United States|-5|apartment| +34664|AAAAAAAAIGHIAAAA|527|Ridge |Blvd||Buena Vista|Bristol city|VA||||apartment| +34665|AAAAAAAAJGHIAAAA|393|Franklin |ST|Suite G|Lakewood|Pike County|IL|68877|United States|-6|single family| +34666|AAAAAAAAKGHIAAAA|830|12th Locust|Blvd|Suite 290|Hillcrest|Butler County|KY|43003|United States|-6|apartment| +34667|AAAAAAAALGHIAAAA|996|Adams |RD|Suite K|Hillcrest|Lancaster County|PA|13003|United States|-5|condo| +34668|AAAAAAAAMGHIAAAA|167|2nd |Road|Suite L|Sunnyside|Edgar County|IL|61952|United States|-6|single family| +34669|AAAAAAAANGHIAAAA|268|Sunset Willow|Cir.|Suite E|Bunker Hill|Wheatland County|MT|60150|United States|-7|apartment| +34670|AAAAAAAAOGHIAAAA|92|6th Lincoln|Dr.|Suite 350|Forest Hills|Salem city|VA|29237|United States|-5|apartment| +34671|AAAAAAAAPGHIAAAA|412|1st |Drive|Suite R|Enterprise|Oceana County|MI|41757|United States|-5|apartment| +34672|AAAAAAAAAHHIAAAA|649|13th Sixth|Pkwy|Suite 390|Mount Vernon|Winston County|MS|58482|United States|-6|apartment| +34673|AAAAAAAABHHIAAAA|112|13th |Wy|Suite 230|Mount Zion|Manitowoc County|WI|58054|United States|-6|single family| +34674|AAAAAAAACHHIAAAA|91|Meadow |Cir.|Suite 110|Highland Park|Suffolk County|NY|16534|United States|-5|apartment| +34675|AAAAAAAADHHIAAAA|835|3rd 12th|Parkway|Suite K|Oakwood|Lane County|OR|90169|United States|-8|apartment| +34676|AAAAAAAAEHHIAAAA|480|9th View|Lane|Suite G|Lakewood|Barber County|KS|68877|United States|-6|single family| +34677|AAAAAAAAFHHIAAAA|272|First |Blvd|Suite L|Pleasant Hill|Noble County|IN|43604|United States|-5|condo| +34678|AAAAAAAAGHHIAAAA|371|First 7th|Boulevard|Suite L|Lakeview|Clearwater County|MN|58579|United States|-6|single family| +34679|AAAAAAAAHHHIAAAA|656|Madison Pine|Dr.|Suite A|Williamsville|Tippah County|MS|58754|United States|-6|apartment| +34680|AAAAAAAAIHHIAAAA|334|Mill |Wy|Suite C|Florence|Floyd County|GA|33394|United States|-5|condo| +34681|AAAAAAAAJHHIAAAA|538|East Spring|Street|Suite 480|Pleasant Valley|Franklin County|OH|42477|United States|-5|apartment| +34682|AAAAAAAAKHHIAAAA|100|10th |Court|Suite L|Sunnyside|Bexar County|TX|71952|United States|-6|apartment| +34683|AAAAAAAALHHIAAAA|628|Oak |Court|Suite A|Brownsville|Jefferson County|NE|69310|United States|-7|apartment| +34684|AAAAAAAAMHHIAAAA|552|Walnut |Circle|Suite W|Fairview|Blackford County|IN|45709|United States|-5|single family| +34685|AAAAAAAANHHIAAAA|||||Hamilton|Calhoun County|||||| +34686|AAAAAAAAOHHIAAAA|251|10th Sixth|Way|Suite 390|Hopewell|Carbon County|MT|60587|United States|-7|apartment| +34687|AAAAAAAAPHHIAAAA|63|First Smith|Lane|Suite 480|Centerville|Acadia Parish|LA|70059|United States|-6|single family| +34688|AAAAAAAAAIHIAAAA|413|Broadway Elm|Boulevard|Suite G|Friendship|Santa Cruz County|AZ|84536|United States|-7|condo| +34689|AAAAAAAABIHIAAAA|277|Spruce Main|RD|Suite 420|Gary|Hardin County|KY|40418|United States|-6|condo| +34690|AAAAAAAACIHIAAAA|255|4th Birch|Parkway|Suite K|Greenwood|Crenshaw County|AL|38828|United States|-6|single family| +34691|AAAAAAAADIHIAAAA|193|Meadow Locust|Parkway|Suite E|New Hope|Beaver County|OK|79431|United States|-6|single family| +34692|AAAAAAAAEIHIAAAA|759|Main Elm|ST|Suite S|Greenfield|Asotin County|WA|95038|United States|-8|single family| +34693|AAAAAAAAFIHIAAAA|702|4th |Lane|Suite P|Springdale|Conejos County|CO|88883|United States|-7|single family| +34694|AAAAAAAAGIHIAAAA|9|12th East|Ave|Suite 280|Farmington|Sampson County|NC|29145|United States|-5|condo| +34695|AAAAAAAAHIHIAAAA|123|Church Seventh|Street|Suite 140|Shiloh|Perry County|OH|49275|United States|-5|condo| +34696|AAAAAAAAIIHIAAAA|716|13th |Parkway|Suite Q|Newtown|Transylvania County|NC|21749|United States|-5|condo| +34697|AAAAAAAAJIHIAAAA|115|Sunset Johnson|ST|Suite D|Oak Ridge|Wake County|NC|28371|United States|-5|single family| +34698|AAAAAAAAKIHIAAAA|104|Mill |Street|Suite A|Bunker Hill|Greene County|IA|50150|United States|-6|single family| +34699|AAAAAAAALIHIAAAA|125|Wilson Sixth|Lane|Suite R|Wildwood|Grant County|SD|56871|United States|-6|condo| +34700|AAAAAAAAMIHIAAAA|268|Davis |Dr.|Suite G|Riverside|Phillips County|KS|69231|United States|-6|apartment| +34701|AAAAAAAANIHIAAAA|986|12th |Avenue|Suite 440|Hillcrest|Cowlitz County|WA|93003|United States|-8|condo| +34702|AAAAAAAAOIHIAAAA|139|Sunset |Avenue|Suite X|Jamestown|Jefferson County|IL|66867|United States|-6|single family| +34703|AAAAAAAAPIHIAAAA|218|12th Chestnut|Pkwy|Suite K|Green Acres|Roanoke County|VA|27683|United States|-5|single family| +34704|AAAAAAAAAJHIAAAA|93|Third |ST|Suite Y|Greenfield|Plymouth County|IA|55038|United States|-6|single family| +34705|AAAAAAAABJHIAAAA|647|13th Poplar|Circle|Suite 240|Kingston|Colquitt County|GA|34975|United States|-5|condo| +34706|AAAAAAAACJHIAAAA|436|Ridge Smith|Wy|Suite R|Edgewood|Pleasants County|WV|20069|United States|-5|apartment| +34707|AAAAAAAADJHIAAAA|5|Park |Blvd|Suite 230|Red Hill|Nassau County|NY|14338|United States|-5|single family| +34708|AAAAAAAAEJHIAAAA|192|Oak College|ST|Suite 230|Woodlawn|Holt County|MO|64098|United States|-6|apartment| +34709|AAAAAAAAFJHIAAAA|88|Railroad |Way|Suite 300|Lakeside|Pottawatomie County|KS|69532|United States|-6|apartment| +34710|AAAAAAAAGJHIAAAA|940|Walnut Sunset|Dr.|Suite 310|Lakewood|Sac County|IA|58877|United States|-6|single family| +34711|AAAAAAAAHJHIAAAA|583|Sixth |Drive|Suite 0|Unionville|Cavalier County|ND|51711|United States|-6|single family| +34712|AAAAAAAAIJHIAAAA|417|North Walnut|Lane|Suite 120|Red Hill|Murray County|GA|34338|United States|-5|single family| +34713|AAAAAAAAJJHIAAAA|186|Woodland Oak|Cir.|Suite E|Farmington|Audubon County|IA|59145|United States|-6|single family| +34714|AAAAAAAAKJHIAAAA|289|Valley |Street|Suite L|Lebanon|Massac County|IL|62898|United States|-6|single family| +34715|AAAAAAAALJHIAAAA|131|9th 12th|Blvd|Suite Y|Five Points|Woods County|OK|76098|United States|-6|single family| +34716|AAAAAAAAMJHIAAAA|883|Lincoln |Dr.|Suite C|Enterprise|Brown County|SD|51757|United States|-6|apartment| +34717|AAAAAAAANJHIAAAA|825|Meadow |Pkwy|Suite 350|Waterloo|Lincoln County|TN|31675|United States|-6|single family| +34718|AAAAAAAAOJHIAAAA|618|Main |Ave|Suite B|Woodville|Buncombe County|NC|24289|United States|-5|single family| +34719|AAAAAAAAPJHIAAAA|172|Meadow |Avenue|Suite 300|Woodland|Calhoun County|AR|74854|United States|-6|single family| +34720|AAAAAAAAAKHIAAAA|530|4th |Ln|Suite C|Newport|Gallatin County|IL|61521|United States|-6|apartment| +34721|AAAAAAAABKHIAAAA|164|Mill 8th|Drive|Suite 190|Unionville|Boone County|KY|41711|United States|-6|condo| +34722|AAAAAAAACKHIAAAA|236|Cherry Adams|Boulevard|Suite 230|Summerville|Moore County|TX|72033|United States|-6|condo| +34723|AAAAAAAADKHIAAAA||Elm ||||Harrison County||||-6|| +34724|AAAAAAAAEKHIAAAA|63|5th |Street|Suite 50|Woodland|Santa Fe County|NM|84854|United States|-7|apartment| +34725|AAAAAAAAFKHIAAAA|698|Main Oak|Street|Suite M|Springfield|Washington County|NE|69303|United States|-7|single family| +34726|AAAAAAAAGKHIAAAA|530|Cherry Oak|Way|Suite 390|Lincoln|Leelanau County|MI|41289|United States|-5|apartment| +34727|AAAAAAAAHKHIAAAA|901|Main Sixth|Blvd|Suite F|Lebanon|Champaign County|OH|42898|United States|-5|condo| +34728|AAAAAAAAIKHIAAAA|644|4th |Ln|Suite H|Bridgeport|Ottawa County|KS|65817|United States|-6|single family| +34729|AAAAAAAAJKHIAAAA|15|Jackson |Ave|Suite J|Centerville|Monroe County|MI|40059|United States|-5|single family| +34730|AAAAAAAAKKHIAAAA|402|10th North|Avenue|Suite 450|Lebanon|Sandusky County|OH|42898|United States|-5|condo| +34731|AAAAAAAALKHIAAAA||Lake 6th|ST||Clinton||ID|88222|||single family| +34732|AAAAAAAAMKHIAAAA|334|3rd |Wy|Suite B|Farmington|Hillsborough County|NH|09745|United States|-5|condo| +34733|AAAAAAAANKHIAAAA|722|Franklin 9th|Way|Suite A|Greenfield|Juneau County|WI|55038|United States|-6|single family| +34734|AAAAAAAAOKHIAAAA|575|Park Locust|Court|Suite X|Bethel|Shelby County|IL|65281|United States|-6|single family| +34735|AAAAAAAAPKHIAAAA|363|Hill Hill|Court|Suite G|Mount Pleasant|Telfair County|GA|31933|United States|-5|single family| +34736|AAAAAAAAALHIAAAA|758|Washington |Pkwy|Suite P|Unionville|Jasper County|IN|41711|United States|-5|single family| +34737|AAAAAAAABLHIAAAA|887|10th |Ln|Suite 360|Union|Clay County|MO|68721|United States|-6|apartment| +34738|AAAAAAAACLHIAAAA|373|Poplar |Boulevard|Suite B|Walnut Grove|Milwaukee County|WI|57752|United States|-6|condo| +34739|AAAAAAAADLHIAAAA|978|Jefferson 1st|ST|Suite 220|Pleasant Valley|Windham County|CT|03077|United States|-5|condo| +34740|AAAAAAAAELHIAAAA|397|First |Ln|Suite 430|Highland|Fremont County|IA|59454|United States|-6|single family| +34741|AAAAAAAAFLHIAAAA|308|9th Jackson|ST|Suite 480|Crossroads|Rockland County|NY|10534|United States|-5|condo| +34742|AAAAAAAAGLHIAAAA|950|East Lake|Drive|Suite 310|Riverview|Adams County|IN|49003|United States|-5|single family| +34743|AAAAAAAAHLHIAAAA|||||||KS|62293||-6|apartment| +34744|AAAAAAAAILHIAAAA|761|River Oak|Pkwy|Suite 390|Woodlawn|Scott County|AR|74098|United States|-6|single family| +34745|AAAAAAAAJLHIAAAA|604|1st |Wy|Suite W|Hamilton|Santa Barbara County|CA|92808|United States|-8|apartment| +34746|AAAAAAAAKLHIAAAA|973|Smith Oak|Avenue|Suite M|Bethel|Washington County|AR|75281|United States|-6|single family| +34747|AAAAAAAALLHIAAAA|960|1st |Way|Suite 440|Jackson|Warren County|IA|59583|United States|-6|apartment| +34748|AAAAAAAAMLHIAAAA|212|8th 5th|Boulevard|Suite X|Enterprise|Rabun County|GA|31757|United States|-5|condo| +34749|AAAAAAAANLHIAAAA|406|Fifth |Ct.|Suite E|Pleasant Hill|Pend Oreille County|WA|93604|United States|-8|condo| +34750|AAAAAAAAOLHIAAAA|835|Locust Highland|Ln|Suite 20|Cedar Grove|Pembina County|ND|50411|United States|-6|single family| +34751|AAAAAAAAPLHIAAAA|727|Ash |Ave|Suite N|Lakewood|Linn County|KS|68877|United States|-6|condo| +34752|AAAAAAAAAMHIAAAA|232|West |Lane|Suite L|Franklin|Miller County|GA|39101|United States|-5|single family| +34753|AAAAAAAABMHIAAAA|452|Park |Pkwy|Suite T|Riverdale|Alpena County|MI|49391|United States|-5|condo| +34754|AAAAAAAACMHIAAAA|54|Hickory Pine|Ave|Suite 70|Oak Ridge|Bergen County|NJ|08971|United States|-5|single family| +34755|AAAAAAAADMHIAAAA|777|4th |Ct.|Suite M|Greenwood|Pope County|IL|68828|United States|-6|single family| +34756|AAAAAAAAEMHIAAAA|590|Lake |Ave|Suite E|Rankin|Jessamine County|KY|42621|United States|-6|single family| +34757|AAAAAAAAFMHIAAAA|824|Church |Cir.|Suite U|Bunker Hill|Stoddard County|MO|60150|United States|-6|single family| +34758|AAAAAAAAGMHIAAAA|481|Spruce 10th|Street|Suite 400|Five Points|Yuba County|CA|96098|United States|-8|apartment| +34759|AAAAAAAAHMHIAAAA||||Suite T|Taft|Multnomah County||90589|United States||condo| +34760|AAAAAAAAIMHIAAAA|597|12th Third|Lane|Suite S|Waterloo|Leon County|TX|71675|United States|-6|condo| +34761|AAAAAAAAJMHIAAAA|449|Laurel Ridge|Street|Suite 490|Liberty|Campbell County|WY|83451|United States|-7|condo| +34762|AAAAAAAAKMHIAAAA|599|Hillcrest |Ln|Suite S|Clifton|Garfield County|NE|68014|United States|-6|apartment| +34763|AAAAAAAALMHIAAAA|579|Lee |Blvd|Suite 280|Hamilton|Grand Isle County|VT|03408|United States|-5|apartment| +34764|AAAAAAAAMMHIAAAA|131|Broadway |Court|Suite 100|Sulphur Springs|Ashtabula County|OH|48354|United States|-5|single family| +34765|AAAAAAAANMHIAAAA|496|Second |Pkwy|Suite 60|Oak Grove|Bergen County|NJ|08970|United States|-5|condo| +34766|AAAAAAAAOMHIAAAA|484|Third |Pkwy|Suite D|Buena Vista|Gallia County|OH|45752|United States|-5|condo| +34767|AAAAAAAAPMHIAAAA|156|Willow |Street|Suite H|Oak Ridge|Marinette County|WI|58371|United States|-6|apartment| +34768|AAAAAAAAANHIAAAA|21|West Valley|Dr.|Suite 110|Mechanicsburg|Portage County|WI|52219|United States|-6|single family| +34769|AAAAAAAABNHIAAAA|565|||Suite 210|Hartland||||United States||| +34770|AAAAAAAACNHIAAAA|236|Mill 6th|Drive|Suite 0|Highland|Stephens County|TX|79454|United States|-6|apartment| +34771|AAAAAAAADNHIAAAA|820|Laurel |Court|Suite L|Kelly|Cass County|MO|62738|United States|-6|apartment| +34772|AAAAAAAAENHIAAAA|388|Lincoln 14th|Way|Suite T|Jackson|White County|TN|39583|United States|-6|apartment| +34773|AAAAAAAAFNHIAAAA|844|Second Highland|Parkway|Suite 300|Hopewell|Gaines County|TX|70587|United States|-6|apartment| +34774|AAAAAAAAGNHIAAAA|641|12th |Lane|Suite 210|Lakewood|Bastrop County|TX|78877|United States|-6|single family| +34775|AAAAAAAAHNHIAAAA|738|Valley |Court|Suite F|Oakland|Charleston County|SC|29843|United States|-5|condo| +34776|AAAAAAAAINHIAAAA|141|Park |Ln|Suite T|Kingston|Mercer County|NJ|05575|United States|-5|single family| +34777|AAAAAAAAJNHIAAAA|353|West Pine|Road|Suite H|Newtown|Custer County|CO|81749|United States|-7|condo| +34778|AAAAAAAAKNHIAAAA|599|Johnson |Court|Suite S|Crossroads|Greene County|MO|60534|United States|-6|apartment| +34779|AAAAAAAALNHIAAAA|690|9th |Pkwy|Suite 400|Spring Hill|Ashtabula County|OH|46787|United States|-5|single family| +34780|AAAAAAAAMNHIAAAA|540|12th Forest|Drive|Suite 140|Marion|Lynn County|TX|70399|United States|-6|condo| +34781|AAAAAAAANNHIAAAA|515|Franklin Ridge|Blvd|Suite N|Greenfield|Dickinson County|MI|45038|United States|-5|condo| +34782|AAAAAAAAONHIAAAA|268|Maple 6th|Cir.|Suite 330|Cumberland|Dearborn County|IN|48971|United States|-5|condo| +34783|AAAAAAAAPNHIAAAA|623|10th Cedar|Ln|Suite M|Clifton|Norton County|KS|68014|United States|-6|apartment| +34784|AAAAAAAAAOHIAAAA|703|1st |Blvd|Suite 180|Deerfield|Jackson County|IN|49840|United States|-5|apartment| +34785|AAAAAAAABOHIAAAA|||||Edgewood||||United States||apartment| +34786|AAAAAAAACOHIAAAA|958|Church River|Wy|Suite E|Riverview|Riley County|KS|69003|United States|-6|single family| +34787|AAAAAAAADOHIAAAA|955|Miller |RD|Suite U|Georgetown|Moody County|SD|57057|United States|-7|single family| +34788|AAAAAAAAEOHIAAAA|513|13th Church|Road|Suite 150|Belmont|Judith Basin County|MT|60191|United States|-7|condo| +34789|AAAAAAAAFOHIAAAA|51|Dogwood |Ct.|Suite 140|Cherry Valley|Craighead County|AR|70854|United States|-6|single family| +34790|AAAAAAAAGOHIAAAA|345|7th Sunset|ST|Suite 430|Mount Pleasant|Albemarle County|VA|21933|United States|-5|condo| +34791|AAAAAAAAHOHIAAAA|310|Center |Circle|Suite J|Oak Grove|Van Wert County|OH|48370|United States|-5|apartment| +34792|AAAAAAAAIOHIAAAA|303|5th Smith|Road|Suite E|Valley View|Graham County|KS|65124|United States|-6|single family| +34793|AAAAAAAAJOHIAAAA|976|Fourth Seventh|Avenue|Suite 340|Enterprise|Breckinridge County|KY|41757|United States|-6|single family| +34794|AAAAAAAAKOHIAAAA|163|13th |Street|Suite E|Crossroads|Walworth County|WI|50534|United States|-6|single family| +34795|AAAAAAAALOHIAAAA|471|Laurel |Court|Suite 370|Lincoln|Allegheny County|PA|11289|United States|-5|apartment| +34796|AAAAAAAAMOHIAAAA|632|Park |Ln|Suite 490||Grand Traverse County|MI|40059|United States||condo| +34797|AAAAAAAANOHIAAAA|693|Franklin |Circle|Suite U|Walnut Grove|Scotland County|NC|27752|United States|-5|apartment| +34798|AAAAAAAAOOHIAAAA|342|Lincoln 3rd|Cir.|Suite 170|Norwood|Sumter County|SC|23989|United States|-5|condo| +34799|AAAAAAAAPOHIAAAA|485|Walnut 7th|Cir.|Suite 10|Woodland|Brantley County|GA|34854|United States|-5|apartment| +34800|AAAAAAAAAPHIAAAA|743|Cherry |Ln|Suite 430|Bunker Hill|Lincoln County|TN|30150|United States|-6|apartment| +34801|AAAAAAAABPHIAAAA|712||Boulevard|Suite X|||LA||United States|-6|| +34802|AAAAAAAACPHIAAAA|394|Cedar Main|Boulevard|Suite 330|Pine Grove|Livingston County|MI|44593|United States|-5|apartment| +34803|AAAAAAAADPHIAAAA|112|River Jefferson|Drive|Suite K|Liberty|James City County|VA|23451|United States|-5|single family| +34804|AAAAAAAAEPHIAAAA|830|Jackson |RD|Suite 350|Woodland|Henderson County|TN|34854|United States|-5|single family| +34805|AAAAAAAAFPHIAAAA|792|Jefferson Sycamore|Road|||Kittson County||||-6|single family| +34806|AAAAAAAAGPHIAAAA|889|Madison Spring|Boulevard|Suite E|Florence|Adair County|IA|53394|United States|-6|apartment| +34807|AAAAAAAAHPHIAAAA|647|Lincoln |Court|Suite X|Forest Hills|Wright County|IA|59237|United States|-6|apartment| +34808|AAAAAAAAIPHIAAAA|428|Laurel Main|Street|Suite D|Oakwood|Susquehanna County|PA|10169|United States|-5|apartment| +34809|AAAAAAAAJPHIAAAA|299|1st |Wy|Suite 280||||46060||-5|| +34810|AAAAAAAAKPHIAAAA|422|Chestnut |Ln|Suite 40|Springdale|Haskell County|KS|68883|United States|-6|apartment| +34811|AAAAAAAALPHIAAAA|978|Laurel College|Cir.|Suite P|Highland|Prairie County|MT|69454|United States|-7|apartment| +34812|AAAAAAAAMPHIAAAA|489|Church |Court|Suite T|Riverview|Clark County|KS|69003|United States|-6|apartment| +34813|AAAAAAAANPHIAAAA|63|13th |Street|Suite 220|Richville|Skagit County|WA|95945|United States|-8|apartment| +34814|AAAAAAAAOPHIAAAA|323|Third Hickory|Lane|Suite Q|Fairview|Hancock County|ME|06309|United States|-5|apartment| +34815|AAAAAAAAPPHIAAAA|706|Highland |Ave|Suite K|Clinton|Highland County|VA|28222|United States|-5|condo| +34816|AAAAAAAAAAIIAAAA|783|14th Broadway|Avenue|Suite S|Springfield|Carroll County|TN|39303|United States|-5|condo| +34817|AAAAAAAABAIIAAAA|5|View Eigth|Way|Suite R|Ashland|Duchesne County|UT|84244|United States|-7|apartment| +34818|AAAAAAAACAIIAAAA|591|Franklin 7th|ST|Suite F|Newtown|Independence County|AR|71749|United States|-6|apartment| +34819|AAAAAAAADAIIAAAA|453|8th 3rd|ST|Suite K|Shady Grove|Casey County|KY|42812|United States|-6|single family| +34820|AAAAAAAAEAIIAAAA|366|1st Birch|Ave|Suite 180|Oak Hill|Pawnee County|NE|67838|United States|-7|single family| +34821|AAAAAAAAFAIIAAAA|497|7th |Circle|Suite 20|Greenwood|Bourbon County|KS|68828|United States|-6|single family| +34822|AAAAAAAAGAIIAAAA|607|5th Poplar|Dr.|Suite 450|Mountain View|Stearns County|MN|54466|United States|-6|single family| +34823|AAAAAAAAHAIIAAAA|665|View Miller|Court|Suite S|Crossroads|Washtenaw County|MI|40534|United States|-5|apartment| +34824|AAAAAAAAIAIIAAAA|687|Maple |Ave|Suite 140|Green Acres|Taylor County|KY|47683|United States|-5|condo| +34825|AAAAAAAAJAIIAAAA|825|Madison Cherry|Ct.|Suite 20|Crossroads|Okeechobee County|FL|30534|United States|-5|condo| +34826|AAAAAAAAKAIIAAAA|676|North Ash|Cir.|Suite P|Newport|Russell County|AL|31521|United States|-6|single family| +34827|AAAAAAAALAIIAAAA|491|Eigth Hillcrest|Court|Suite K|Leesville|Colonial Heights city|VA|25423|United States|-5|single family| +34828|AAAAAAAAMAIIAAAA|985|Park |Cir.|Suite 290|Deerfield|Walker County|GA|39840|United States|-5|condo| +34829|AAAAAAAANAIIAAAA|983|4th Spring|Road|Suite E|Bunker Hill|Christian County|IL|60150|United States|-6|apartment| +34830|AAAAAAAAOAIIAAAA|957|Pine |Pkwy|Suite 210|Highland Park|Ingham County|MI|46534|United States|-5|condo| +34831|AAAAAAAAPAIIAAAA|565|Lincoln |Way|Suite 60|Oak Ridge|Kerr County|TX|78371|United States|-6|single family| +34832|AAAAAAAAABIIAAAA|583|6th East|Ct.|Suite 90|Hopewell|Knox County|OH|40587|United States|-5|apartment| +34833|AAAAAAAABBIIAAAA|186|1st 3rd|Drive|Suite R|Friendship|Moody County|SD|54536|United States|-7|condo| +34834|AAAAAAAACBIIAAAA|679|12th |Ln|Suite 410|Oak Grove|Stokes County|NC|28370|United States|-5|single family| +34835|AAAAAAAADBIIAAAA|95|Maple Spruce|Drive|Suite 160|Five Points|Latah County|ID|86098|United States|-7|apartment| +34836|AAAAAAAAEBIIAAAA|545|6th Jefferson|Ct.|Suite Q|Lebanon|Marion County|MS|52898|United States|-6|apartment| +34837|AAAAAAAAFBIIAAAA||1st |Ct.||Wilson||AR|||-6|condo| +34838|AAAAAAAAGBIIAAAA|30|Jefferson Church|Pkwy|Suite O|Edgewood|Norman County|MN|50069|United States|-6|condo| +34839|AAAAAAAAHBIIAAAA|278|Walnut |Road|Suite 290|Bunker Hill|Dixie County|FL|30150|United States|-5|apartment| +34840|AAAAAAAAIBIIAAAA|595|Green |Circle|Suite 490|Walnut Grove|Owen County|KY|47752|United States|-5|apartment| +34841|AAAAAAAAJBIIAAAA|960|Center 12th|ST|Suite M|Friendship|Real County|TX|74536|United States|-6|apartment| +34842|AAAAAAAAKBIIAAAA|229|Miller |Avenue|Suite M|Kingston|La Salle Parish|LA|74975|United States|-6|single family| +34843|AAAAAAAALBIIAAAA|337|Adams |Parkway|Suite X|Glenwood|Medina County|OH|43511|United States|-5|condo| +34844|AAAAAAAAMBIIAAAA|184|Lake Cherry|Ave|Suite I|Five Points|Leon County|TX|76098|United States|-6|apartment| +34845|AAAAAAAANBIIAAAA|226|6th |Dr.|Suite 240|Edgewood|Boyd County|NE|60069|United States|-6|single family| +34846|AAAAAAAAOBIIAAAA|268|1st 2nd|Ave|Suite 130|Pleasant Hill|Dodge County|WI|53604|United States|-6|single family| +34847|AAAAAAAAPBIIAAAA|967|14th Smith|Parkway|Suite 390|Pierce|Ogemaw County|MI|43360|United States|-5|single family| +34848|AAAAAAAAACIIAAAA|962|Cedar Park|Boulevard|Suite N|Waterloo|Prairie County|AR|71675|United States|-6|apartment| +34849|AAAAAAAABCIIAAAA|881|3rd Dogwood|Drive|Suite 40|Pleasant Grove|York County|PA|14136|United States|-5|single family| +34850|AAAAAAAACCIIAAAA|212|Smith Elm|Cir.|Suite B|Riverdale|Brantley County|GA|39391|United States|-5|apartment| +34851|AAAAAAAADCIIAAAA|52|Sycamore |ST|Suite J|Farmington|Clermont County|OH|49145|United States|-5|condo| +34852|AAAAAAAAECIIAAAA|476|Hickory |Road|Suite C|Sunnyside|Stark County|OH|41952|United States|-5|condo| +34853|AAAAAAAAFCIIAAAA|285|2nd 10th|RD|Suite 160|Oakland|Cochran County|TX|79843|United States|-6|apartment| +34854|AAAAAAAAGCIIAAAA|961|Maple |Ct.|Suite 120|Midway|Conejos County|CO|81904|United States|-7|single family| +34855|AAAAAAAAHCIIAAAA|633|Hill Park|Court|Suite R|Unionville|Ascension Parish|LA|71711|United States|-6|single family| +34856|AAAAAAAAICIIAAAA|713|Hickory Woodland|Cir.|Suite D|Five Points|Colleton County|SC|26098|United States|-5|single family| +34857|AAAAAAAAJCIIAAAA|316|Church |Parkway|Suite N|Fairview|Liberty County|GA|35709|United States|-5|apartment| +34858|AAAAAAAAKCIIAAAA|700|Park |Street|Suite R|West Liberty|Aiken County|SC|24752|United States|-5|condo| +34859|AAAAAAAALCIIAAAA|680|Miller |Lane|Suite W|Liberty|Bastrop County|TX|73451|United States|-6|single family| +34860|AAAAAAAAMCIIAAAA|923|Main Elm|Ave|Suite X|Union|Chesapeake city|VA|28721|United States|-5|condo| +34861|AAAAAAAANCIIAAAA|831|5th |Boulevard|Suite 180|Prosperity|Wichita County|TX|79089|United States|-6|apartment| +34862|AAAAAAAAOCIIAAAA|77|Spring Hill|Lane|Suite 10|Hamilton|Crawford County|IN|42808|United States|-5|apartment| +34863|AAAAAAAAPCIIAAAA||Willow River||||Utah County|UT|82808|United States||single family| +34864|AAAAAAAAADIIAAAA|360|View Highland|Way|Suite 50|Payne|Spencer County|KY|46134|United States|-5|condo| +34865|AAAAAAAABDIIAAAA|37|Walnut Poplar|ST|Suite P|White Oak|Pittsylvania County|VA|26668|United States|-5|condo| +34866|AAAAAAAACDIIAAAA|15|Birch |RD|Suite B|Sunnyside|Uinta County|WY|81952|United States|-7|single family| +34867|AAAAAAAADDIIAAAA|195|Lincoln Fourth|Way|Suite 370|Cedar Grove|Clinton County|MO|60411|United States|-6|apartment| +34868|AAAAAAAAEDIIAAAA|802|1st Sunset|Ct.|Suite 170|Mount Vernon|Pottawatomie County|KS|68482|United States|-6|single family| +34869|AAAAAAAAFDIIAAAA|787|6th |Dr.|Suite 50|Lakeside|Bullock County|AL|39532|United States|-6|single family| +34870|AAAAAAAAGDIIAAAA|40|Church |Wy|Suite 0|Five Points|Kenton County|KY|46098|United States|-5|single family| +34871|AAAAAAAAHDIIAAAA|17|Eigth 8th|Court|Suite H|Highland|Colusa County|CA|99454|United States|-8|apartment| +34872|AAAAAAAAIDIIAAAA|871|Cedar Pine|Way|Suite 260|Plainview|Lamar County|GA|33683|United States|-5|condo| +34873|AAAAAAAAJDIIAAAA|559|1st 8th|RD|Suite 60|Mount Olive|Sully County|SD|58059|United States|-7|apartment| +34874|AAAAAAAAKDIIAAAA|991|Madison |Ave|Suite 430|Lakeview|Riverside County|CA|98579|United States|-8|single family| +34875|AAAAAAAALDIIAAAA|175|1st 9th|Parkway|Suite R|Edgewood|Guadalupe County|TX|70069|United States|-6|single family| +34876|AAAAAAAAMDIIAAAA|640|Fourth 12th|RD|Suite C|Ferguson|Karnes County|TX|71821|United States|-6|condo| +34877|AAAAAAAANDIIAAAA|913|Lake Lake||Suite E||Furnas County|||United States||single family| +34878|AAAAAAAAODIIAAAA|116|Fourth |ST|Suite G|Mount Olive|Huerfano County|CO|88059|United States|-7|condo| +34879|AAAAAAAAPDIIAAAA|472|Locust |Ct.|Suite C|Mount Vernon|Surry County|VA|28482|United States|-5|condo| +34880|AAAAAAAAAEIIAAAA|137|River |ST|Suite 220|Jackson|Nassau County|FL|39583|United States|-5|apartment| +34881|AAAAAAAABEIIAAAA|948|Franklin 8th|Way|Suite 370|Georgetown|Valley County|NE|67057|United States|-7|apartment| +34882|AAAAAAAACEIIAAAA|494|Cherry Spruce|Way|Suite 280|Fairfield|Oxford County|ME|06792|United States|-5|condo| +34883|AAAAAAAADEIIAAAA|32|Walnut Main|Circle|Suite B|Stringtown|Goochland County|VA|20162|United States|-5|condo| +34884|AAAAAAAAEEIIAAAA|654|Birch Davis|Boulevard|Suite 310|New Hope|Camden County|MO|69431|United States|-6|single family| +34885|AAAAAAAAFEIIAAAA|900|West |Court|Suite 60|Northwood|Chelan County|WA|94104|United States|-8|condo| +34886|AAAAAAAAGEIIAAAA|836|College |Ct.|Suite D|Lebanon|Guernsey County|OH|42898|United States|-5|condo| +34887|AAAAAAAAHEIIAAAA|180|Jefferson Valley|ST|Suite T|Lebanon|Roane County|WV|22898|United States|-5|apartment| +34888|AAAAAAAAIEIIAAAA|450|Washington 12th|Circle|Suite 430|Newport|Pacific County|WA|91521|United States|-8|condo| +34889|AAAAAAAAJEIIAAAA|706|Pine Fourth|Drive|Suite 380|Red Hill|Murray County|OK|74338|United States|-6|condo| +34890|AAAAAAAAKEIIAAAA|377|Ridge Sunset|Wy|Suite 160|Wilson|Jefferson County|OR|96971|United States|-8|apartment| +34891|AAAAAAAALEIIAAAA|171|Jackson |Ave|Suite 60|Concord|Cache County|UT|84107|United States|-7|single family| +34892|AAAAAAAAMEIIAAAA|95||Street|Suite 110|Spring Hill|Murray County|GA|36787||-5|| +34893|AAAAAAAANEIIAAAA|746|Washington Maple|Boulevard|Suite K|Greenville|Custer County|CO|81387|United States|-7|apartment| +34894|AAAAAAAAOEIIAAAA|323|East Hickory|Ln|Suite 310|Woodlawn|Crawford County|GA|34098|United States|-5|condo| +34895|AAAAAAAAPEIIAAAA|919|West East|Lane|Suite H|Summit|Oneida County|ID|80499|United States|-7|condo| +34896|AAAAAAAAAFIIAAAA|37|Lake Center|Wy|Suite Y|New Hope|Coos County|NH|09431|United States|-5|apartment| +34897|AAAAAAAABFIIAAAA|1000|Hill |Blvd|Suite B|Shelby|Mason County|WV|26575|United States|-5|single family| +34898|AAAAAAAACFIIAAAA|512|3rd Elm|Ave|Suite 490|Maple Grove|Crawford County|PA|18252|United States|-5|condo| +34899|AAAAAAAADFIIAAAA|838|Sunset East|Boulevard|Suite 300|Harmony|Lincoln County|GA|35804|United States|-5|condo| +34900|AAAAAAAAEFIIAAAA|373|Hill |Boulevard|Suite Y|Jamestown|De Soto Parish|LA|76867|United States|-6|apartment| +34901|AAAAAAAAFFIIAAAA|711|3rd |Circle|Suite I|Oak Ridge|Kittitas County|WA|98371|United States|-8|apartment| +34902|AAAAAAAAGFIIAAAA|602|Forest River|Court|Suite X|Delmar|Polk County|WI|53957|United States|-6|single family| +34903|AAAAAAAAHFIIAAAA|878|Lake |RD|Suite T|Red Hill|Solano County|CA|94338|United States|-8|single family| +34904|AAAAAAAAIFIIAAAA|777|Lake |Parkway|Suite 350|Stringtown|Hamilton County|TN|30162|United States|-5|apartment| +34905|AAAAAAAAJFIIAAAA||Park |Circle|||Benton County|MS|56614|||| +34906|AAAAAAAAKFIIAAAA|994|Fourth 8th|Blvd|Suite Y|Bunker Hill|Red River Parish|LA|70150|United States|-6|condo| +34907|AAAAAAAALFIIAAAA|983|Washington 10th|Avenue|Suite T|Liberty|Logan County|KS|63451|United States|-6|apartment| +34908|AAAAAAAAMFIIAAAA|333|Main |Avenue|Suite O|Riverdale|Scott County|MN|59391|United States|-6|apartment| +34909|AAAAAAAANFIIAAAA|722|Pine |Avenue|Suite 190|Providence|Tioga County|NY|16614|United States|-5|apartment| +34910|AAAAAAAAOFIIAAAA|405|2nd |ST|Suite P|Oakwood|Hardy County|WV|20169|United States|-5|condo| +34911|AAAAAAAAPFIIAAAA|478|Chestnut |Avenue|Suite A|Greenwood|Mariposa County|CA|98828|United States|-8|single family| +34912|AAAAAAAAAGIIAAAA|797|3rd |RD|Suite 70|Greenville|Wabash County|IL|61387|United States|-6|condo| +34913|AAAAAAAABGIIAAAA|801|Pine |Avenue|Suite D|Shady Grove|Jefferson County|CO|82812|United States|-7|single family| +34914|AAAAAAAACGIIAAAA|827|6th View|Boulevard|Suite K|New Hope|Rogers County|OK|79431|United States|-6|apartment| +34915|AAAAAAAADGIIAAAA|235|1st |Drive|Suite H|Brownsville|Fairfax city|VA|29310|United States|-5|apartment| +34916|AAAAAAAAEGIIAAAA|441|Church |Lane|Suite 360|Newport|Kiowa County|OK|71521|United States|-6|condo| +34917|AAAAAAAAFGIIAAAA|417|View |Ave|Suite C|Valley View|Calhoun County|IL|65124|United States|-6|apartment| +34918|AAAAAAAAGGIIAAAA|437|7th South|Boulevard|Suite P|Belmont|Jeff Davis County|TX|70191|United States|-6|single family| +34919|AAAAAAAAHGIIAAAA|909|Green |Avenue|Suite 70|Farmersville|Wilkes County|GA|39305|United States|-5|condo| +34920|AAAAAAAAIGIIAAAA|412|Cherry |Blvd|Suite C|Concord|Mariposa County|CA|94107|United States|-8|condo| +34921|AAAAAAAAJGIIAAAA|767|Jefferson Jackson|Court|Suite 20|Providence|Boone County|IA|56614|United States|-6|single family| +34922|AAAAAAAAKGIIAAAA|88|River |Drive|Suite N|Ashland|Pike County|IL|64244|United States|-6|single family| +34923|AAAAAAAALGIIAAAA|431|Cedar |Boulevard|Suite O|Bunker Hill|Lyman County|SD|50150|United States|-7|apartment| +34924|AAAAAAAAMGIIAAAA|206|Third Cherry|Court|Suite Y|Ashley|Iroquois County|IL|64324|United States|-6|condo| +34925|AAAAAAAANGIIAAAA|733|||Suite D|Woodland|McDonough County|IL||United States|-6|condo| +34926|AAAAAAAAOGIIAAAA|663|Lake |Blvd|Suite L|Highland Park|Sabine Parish|LA|76534|United States|-6|condo| +34927|AAAAAAAAPGIIAAAA|724|Oak Woodland|Way|Suite 420|San Jose|Cecil County|MD|28003|United States|-5|apartment| +34928|AAAAAAAAAHIIAAAA|485|15th 2nd|Circle|Suite S|Fairview|Fulton County|IL|65709|United States|-6|condo| +34929|AAAAAAAABHIIAAAA|456|Hillcrest Jackson|Lane|Suite O|Fox|Lee County|NC|20631|United States|-5|condo| +34930|AAAAAAAACHIIAAAA|897|Washington Washington|Dr.|Suite I|Lincoln|Maries County|MO|61289|United States|-6|condo| +34931|AAAAAAAADHIIAAAA|143|West |Ct.|Suite S|Newport|Platte County|NE|61521|United States|-7|condo| +34932|AAAAAAAAEHIIAAAA|811|Cherry |RD|Suite L|Antioch|Bremer County|IA|58605|United States|-6|apartment| +34933|AAAAAAAAFHIIAAAA|625|4th Hickory|ST|Suite 130|Mount Pleasant|Campbell County|SD|51933|United States|-6|single family| +34934|AAAAAAAAGHIIAAAA|496|Park |Court|Suite N|Cedar Grove|Fulton County|IN|40411|United States|-5|apartment| +34935|AAAAAAAAHHIIAAAA|783|Jefferson |Boulevard|Suite 440|Kingston|Switzerland County|IN|44975|United States|-5|condo| +34936|AAAAAAAAIHIIAAAA|379|Pine |Ct.|Suite P|Edwards|Jay County|IN|41409|United States|-5|single family| +34937|AAAAAAAAJHIIAAAA|172|1st Maple|Parkway|Suite X|Hillcrest|Clinton County|IL|63003|United States|-6|condo| +34938|AAAAAAAAKHIIAAAA|485|Pine |Ct.|Suite X|Lincoln|Washoe County|NV|81289|United States|-8|apartment| +34939|AAAAAAAALHIIAAAA|828|9th Meadow|Way|Suite 340|Wesley|Brown County|KS|61218|United States|-6|condo| +34940|AAAAAAAAMHIIAAAA|645|Central |Circle|Suite I|White Oak|Lauderdale County|AL|36668|United States|-6|condo| +34941|AAAAAAAANHIIAAAA|769|East 5th|Road|Suite 110|Shiloh|Marion County|IL|69275|United States|-6|condo| +34942|AAAAAAAAOHIIAAAA|786|Lincoln |Street|Suite 360|Woodville|Elkhart County|IN|44289|United States|-5|condo| +34943|AAAAAAAAPHIIAAAA|693|2nd |Court|Suite 50|Newtown|Calhoun County|MS|51749|United States|-6|single family| +34944|AAAAAAAAAIIIAAAA|500|Ridge 8th|RD|Suite U|Glenwood|Boundary County|ID|83511|United States|-7|apartment| +34945|AAAAAAAABIIIAAAA|||||||OK||United States||| +34946|AAAAAAAACIIIAAAA|586|Sunset Hickory|Avenue|Suite O|Philadelphia|Tipton County|IN|45591|United States|-5|condo| +34947|AAAAAAAADIIIAAAA|267|Washington |Wy|Suite 110|Red Hill|Pierce County|NE|64338|United States|-7|condo| +34948|AAAAAAAAEIIIAAAA|438|Spring |Blvd|Suite 120|Macedonia|Deaf Smith County|TX|71087|United States|-6|single family| +34949|AAAAAAAAFIIIAAAA|89|Birch Hickory|Ln|Suite D|Brownsville|Parker County|TX|79310|United States|-6|single family| +34950|AAAAAAAAGIIIAAAA|548|View |RD|Suite N|Gravel Hill|Raleigh County|WV|21944|United States|-5|apartment| +34951|AAAAAAAAHIIIAAAA|795|Spruce Park|Ct.|Suite 210|Woodville|Clinton County|PA|14289|United States|-5|single family| +34952|AAAAAAAAIIIIAAAA|649|Sunset Eigth|Court|Suite E|Enterprise|Tioga County|NY|11757|United States|-5|condo| +34953|AAAAAAAAJIIIAAAA|754|Second |Parkway|Suite 420|Stringtown|Fayette County|WV|20162|United States|-5|apartment| +34954|AAAAAAAAKIIIAAAA|20|Railroad |Way|Suite 490|Glenwood|Ogemaw County|MI|43511|United States|-5|condo| +34955|AAAAAAAALIIIAAAA|641|Birch |Court|Suite P|Lakewood|Crisp County|GA|38877|United States|-5|apartment| +34956|AAAAAAAAMIIIAAAA|869|13th |Lane|Suite B|Friendship|Victoria County|TX|74536|United States|-6|apartment| +34957|AAAAAAAANIIIAAAA|620|Broadway |Ct.|Suite 350|Fairfield|Roseau County|MN|56192|United States|-6|apartment| +34958|AAAAAAAAOIIIAAAA|587|5th First|Street|Suite 110|Hillcrest|Polk County|MO|63003|United States|-6|single family| +34959|AAAAAAAAPIIIAAAA|234|Elm |Boulevard|Suite I|Concord|Clark County|ID|84107|United States|-7|single family| +34960|AAAAAAAAAJIIAAAA|453|1st Franklin|Lane|Suite R|Shady Grove|Angelina County|TX|72812|United States|-6|apartment| +34961|AAAAAAAABJIIAAAA|725|Hill |Ave|Suite R|Oak Hill|Hillsdale County|MI|47838|United States|-5|condo| +34962|AAAAAAAACJIIAAAA|465|Miller Jefferson|Ln|Suite S|Plainview|Lucas County|OH|43683|United States|-5|single family| +34963|AAAAAAAADJIIAAAA|747|5th Elevnth|Court|Suite 80|Maple Grove|Yakutat Borough|AK|98252|United States|-9|apartment| +34964|AAAAAAAAEJIIAAAA|515|Hickory |Street|Suite J|Adrian|Westchester County|NY|13301|United States|-5|condo| +34965|AAAAAAAAFJIIAAAA|539|14th Wilson|Street|Suite X|Valley View|Wright County|MN|55124|United States|-6|apartment| +34966|AAAAAAAAGJIIAAAA|258|Hill |Drive|Suite V|Woodland|Dade County|MO|64854|United States|-6|single family| +34967|AAAAAAAAHJIIAAAA|702|11th 2nd|Ln|Suite 130|Hamilton|Tallapoosa County|AL|32808|United States|-6|single family| +34968|AAAAAAAAIJIIAAAA|751|Chestnut |Pkwy|Suite 20|Newtown|Latah County|ID|81749|United States|-7|single family| +34969|AAAAAAAAJJIIAAAA|64|Hill Tenth|Ct.|Suite D|Marion|Dyer County|TN|30399|United States|-5|condo| +34970|AAAAAAAAKJIIAAAA|392|14th Wilson|Pkwy|Suite 440|Hamilton|Merced County|CA|92808|United States|-8|apartment| +34971|AAAAAAAALJIIAAAA|895|South |ST|Suite Y|Spring Hill|Owen County|KY|46787|United States|-5|condo| +34972|AAAAAAAAMJIIAAAA|344|10th Smith|Cir.|Suite 480|Highland Park|Hitchcock County|NE|66534|United States|-6|single family| +34973|AAAAAAAANJIIAAAA|725|Laurel First|Parkway|Suite O|Pleasant Valley|Candler County|GA|32477|United States|-5|condo| +34974|AAAAAAAAOJIIAAAA|831|Valley 4th|Way|Suite J|Salem|Jeff Davis County|TX|78048|United States|-6|condo| +34975|AAAAAAAAPJIIAAAA|970|1st 2nd|RD|Suite P|Marion|Renville County|ND|50399|United States|-6|single family| +34976|AAAAAAAAAKIIAAAA|724|Dogwood Smith|Drive|Suite 340|Florence|Trinity County|TX|73394|United States|-6|single family| +34977|AAAAAAAABKIIAAAA|891|Lakeview Lake|Circle|Suite 360|Newport|Suffolk County|MA|02121|United States|-5|apartment| +34978|AAAAAAAACKIIAAAA|656|1st Spring|Drive|Suite 280|Concord|Kershaw County|SC|24107|United States|-5|condo| +34979|AAAAAAAADKIIAAAA|621|Oak Park|Boulevard|Suite 150|Red Hill|Reagan County|TX|74338|United States|-6|apartment| +34980|AAAAAAAAEKIIAAAA|192|10th |RD|Suite M|Newport|Custer County|CO|81521|United States|-7|apartment| +34981|AAAAAAAAFKIIAAAA|766|6th |Dr.|Suite 200|Highland|Cherry County|NE|69454|United States|-6|apartment| +34982|AAAAAAAAGKIIAAAA|640|Cherry Park|Court|Suite A|Mount Olive|Cooke County|TX|78059|United States|-6|single family| +34983|AAAAAAAAHKIIAAAA|600||Wy|Suite Y|Springfield|Phillips County|MT|69303|United States|-7|| +34984|AAAAAAAAIKIIAAAA|147|9th |Blvd|Suite X|Marion|Sequatchie County|TN|30399|United States|-6|apartment| +34985|AAAAAAAAJKIIAAAA|583|Seventh View|Court|Suite X|Red Oak|Allegany County|NY|15018|United States|-5|condo| +34986|AAAAAAAAKKIIAAAA|765|Oak |Drive|Suite 100|Sunnyside|Haskell County|OK|71952|United States|-6|single family| +34987|AAAAAAAALKIIAAAA|320|Hillcrest Woodland|Blvd|Suite T|Liberty|Greene County|IL|63451|United States|-6|apartment| +34988|AAAAAAAAMKIIAAAA|592|4th Hill|Lane|Suite 240|Five Forks|Schoolcraft County|MI|42293|United States|-5|condo| +34989|AAAAAAAANKIIAAAA|837|Laurel North|Ln|Suite E|Empire|Dakota County|NE|64145|United States|-6|apartment| +34990|AAAAAAAAOKIIAAAA|998|Park Lincoln|Blvd|Suite J|Macedonia|Autauga County|AL|31087|United States|-6|condo| +34991|AAAAAAAAPKIIAAAA|391|Lake Lakeview|Drive|Suite 460|Cedar Grove|Oktibbeha County|MS|50411|United States|-6|apartment| +34992|AAAAAAAAALIIAAAA||||Suite I|Bethel|||||-7|single family| +34993|AAAAAAAABLIIAAAA|508|Cedar |Avenue|Suite L|Riverdale|Montgomery County|IL|69391|United States|-6|apartment| +34994|AAAAAAAACLIIAAAA|722|6th |Ave|Suite S|Appleton|Alexandria city|VA|24240|United States|-5|single family| +34995|AAAAAAAADLIIAAAA|833|Washington |Blvd|Suite R|Sawyer|Lake County|TN|36045|United States|-6|apartment| +34996|AAAAAAAAELIIAAAA|172|Spring |Road|Suite 450|Mount Pleasant|Ottawa County|OH|41933|United States|-5|single family| +34997|AAAAAAAAFLIIAAAA|246|Spring 15th|Circle|Suite 280|Lakewood|James City County|VA|28877|United States|-5|apartment| +34998|AAAAAAAAGLIIAAAA|305|Walnut |Dr.|Suite 300|Amity|Yoakum County|TX|70766|United States|-6|apartment| +34999|AAAAAAAAHLIIAAAA|560|River |Boulevard|Suite 120|Salem|Arthur County|NE|68048|United States|-6|apartment| +35000|AAAAAAAAILIIAAAA|440|Mill |Road||Midway|Goochland County|VA|||-5|apartment| +35001|AAAAAAAAJLIIAAAA|308|Central Spring|Ave|Suite A|Lakeside|Alexander County|IL|69532|United States|-6|single family| +35002|AAAAAAAAKLIIAAAA|357|Sycamore |Dr.|Suite Q|Fairview|West Baton Rouge Parish|LA|75709|United States|-6|condo| +35003|AAAAAAAALLIIAAAA|263|Jefferson Hickory|Cir.|Suite 300|Newtown|Tallahatchie County|MS|51749|United States|-6|single family| +35004|AAAAAAAAMLIIAAAA|||Ave|Suite 20||||68525|United States||| +35005|AAAAAAAANLIIAAAA|406|Willow |Boulevard|Suite C|Caledonia|Cherry County|NE|67411|United States|-6|condo| +35006|AAAAAAAAOLIIAAAA|463|West Miller|Pkwy|Suite 60|Midway|Drew County|AR|71904|United States|-6|condo| +35007|AAAAAAAAPLIIAAAA|544|Ash Washington|Ct.|Suite 460|Greenville|Claiborne County|TN|31387|United States|-5|single family| +35008|AAAAAAAAAMIIAAAA|809|Birch Highland|Blvd|Suite 10|Whitesville|Polk County|NE|65903|United States|-7|apartment| +35009|AAAAAAAABMIIAAAA|120|6th Oak|Way|Suite 200|Farmington|Davison County|SD|59145|United States|-6|apartment| +35010|AAAAAAAACMIIAAAA|901|Hickory Pine|Wy|Suite 220|Springdale|Dona Ana County|NM|88883|United States|-7|apartment| +35011|AAAAAAAADMIIAAAA|627|Franklin Woodland|Drive|Suite 80|Wilson|Worcester County|MA|07571|United States|-5|single family| +35012|AAAAAAAAEMIIAAAA|218|12th 15th|Wy|Suite P|Glenwood|Juab County|UT|83511|United States|-7|apartment| +35013|AAAAAAAAFMIIAAAA|||Dr.|Suite 280|Pleasant Grove|||||-5|apartment| +35014|AAAAAAAAGMIIAAAA|6|Davis |Ln|Suite Q|Ryan|Adams County|MS|50525|United States|-6|single family| +35015|AAAAAAAAHMIIAAAA|935|Maple South|Lane|Suite 310|Springfield|Cass County|IN|49303|United States|-5|single family| +35016|AAAAAAAAIMIIAAAA|954|Park |Blvd|Suite X|Spring Valley|Woodbury County|IA|56060|United States|-6|single family| +35017|AAAAAAAAJMIIAAAA|421|Seventh 9th|Ave|Suite 110|Lakewood|Powder River County|MT|68877|United States|-7|condo| +35018|AAAAAAAAKMIIAAAA|639|Railroad |Blvd|Suite X|Jackson|Montgomery County|MD|29583|United States|-5|single family| +35019|AAAAAAAALMIIAAAA|610|Miller Lake|Cir.|Suite 250|Mount Zion|Richland Parish|LA|78054|United States|-6|single family| +35020|AAAAAAAAMMIIAAAA|92|Hickory Valley|Avenue|Suite 420|Riverview|Mason County|WV|29003|United States|-5|single family| +35021|AAAAAAAANMIIAAAA|984|7th Highland|Pkwy|Suite 90|Ashland|Banks County|GA|34244|United States|-5|condo| +35022|AAAAAAAAOMIIAAAA|143|Lee Dogwood|Blvd|Suite 270|Plainview|Johnson County|TX|73683|United States|-6|condo| +35023|AAAAAAAAPMIIAAAA|6|Central Elm|Blvd|Suite V|Midway|Washington County|MS|51904|United States|-6|single family| +35024|AAAAAAAAANIIAAAA|904|Spruce Church|Blvd|Suite C|Maple Grove|Kemper County|MS|58252|United States|-6|apartment| +35025|AAAAAAAABNIIAAAA|440|Park Lake|Cir.|Suite F|Kingston|Sumter County|SC|24975|United States|-5|single family| +35026|AAAAAAAACNIIAAAA|946|2nd |Avenue|Suite 350|Glendale|Laramie County|WY|83951|United States|-7|apartment| +35027|AAAAAAAADNIIAAAA|220|River |Street|Suite H|Brownsville|Boone County|MO|69310|United States|-6|apartment| +35028|AAAAAAAAENIIAAAA|987|Madison |Court|Suite 30|Florence|Otero County|NM|83394|United States|-7|condo| +35029|AAAAAAAAFNIIAAAA|677|View Hillcrest|Drive|Suite 490|Mountain View|Pittsburg County|OK|74466|United States|-6|single family| +35030|AAAAAAAAGNIIAAAA|420|Washington |Road|Suite 270|Fairview|Gilliam County|OR|95709|United States|-8|condo| +35031|AAAAAAAAHNIIAAAA|150|Adams |Ln|Suite L|Wayland|Decatur County|GA|35115|United States|-5|apartment| +35032|AAAAAAAAINIIAAAA|585|Lakeview |Lane|Suite 130|Valley View|Grant County|KY|45124|United States|-6|condo| +35033|AAAAAAAAJNIIAAAA|229|View Jefferson|Blvd|Suite O|Florence|Randolph County|AL|33394|United States|-6|condo| +35034|AAAAAAAAKNIIAAAA|847|Lakeview Washington|Road|Suite G|Shiloh|Marinette County|WI|59275|United States|-6|condo| +35035|AAAAAAAALNIIAAAA|121|Dogwood |Ave|Suite H|Harmony|Oceana County|MI|45804|United States|-5|apartment| +35036|AAAAAAAAMNIIAAAA|200|Central |Road|Suite J|Arlington|Harrison County|MS|56557|United States|-6|single family| +35037|AAAAAAAANNIIAAAA|908|Oak Smith|Way|Suite 270|Friendship|Waller County|TX|74536|United States|-6|condo| +35038|AAAAAAAAONIIAAAA|451|Walnut |Parkway|Suite G|Stringtown|Tift County|GA|30162|United States|-5|apartment| +35039|AAAAAAAAPNIIAAAA|807|Adams |Way|Suite V|Springdale|Avery County|NC|28883|United States|-5|single family| +35040|AAAAAAAAAOIIAAAA|46|2nd Mill|Dr.|Suite V|Newtown|Dawson County|GA|31749|United States|-5|condo| +35041|AAAAAAAABOIIAAAA|702|Forest |Boulevard|Suite Y|Fairview|Belknap County|NH|06309|United States|-5|apartment| +35042|AAAAAAAACOIIAAAA|611|Valley Cedar|RD|Suite 450|Valley View|Livingston County|KY|45124|United States|-5|condo| +35043|AAAAAAAADOIIAAAA|307|Ridge |RD|Suite E|Friendship|Monroe County|MS|54536|United States|-6|apartment| +35044|AAAAAAAAEOIIAAAA|159|2nd |Boulevard|Suite 150|Ashland|Polk County|MN|54244|United States|-6|condo| +35045|AAAAAAAAFOIIAAAA|38|Cedar Main|Pkwy|Suite 290|Buena Vista|Shawnee County|KS|65752|United States|-6|single family| +35046|AAAAAAAAGOIIAAAA|546|Park Meadow|Way|Suite 380|Maple Grove|Jones County|SD|58252|United States|-7|single family| +35047|AAAAAAAAHOIIAAAA|900|Maple Railroad|Avenue|Suite 240|Newtown|Effingham County|GA|31749|United States|-5|apartment| +35048|AAAAAAAAIOIIAAAA|391|Lee |Ave|Suite 450|Greenville|Wise County|TX|71387|United States|-6|apartment| +35049|AAAAAAAAJOIIAAAA|173|River Lincoln|Boulevard|Suite W|Friendship|Calhoun County|TX|74536|United States|-6|single family| +35050|AAAAAAAAKOIIAAAA|953|Locust Locust|Dr.|Suite 60|Walnut|Calloway County|KY|46245|United States|-6|condo| +35051|AAAAAAAALOIIAAAA|111|West |Court|Suite 280|Antioch|Nowata County|OK|78605|United States|-6|single family| +35052|AAAAAAAAMOIIAAAA|345|College Green|RD|Suite R|Hardy|Tuolumne County|CA|95354|United States|-8|single family| +35053|AAAAAAAANOIIAAAA|300|Maple |Ln|Suite 210|Saint Paul|Live Oak County|TX|75720|United States|-6|apartment| +35054|AAAAAAAAOOIIAAAA|399|Williams |Court|Suite M|Riverview|Ozaukee County|WI|59003|United States|-6|condo| +35055|AAAAAAAAPOIIAAAA||Walnut 4th|Road|||||79391||-6|condo| +35056|AAAAAAAAAPIIAAAA|38|4th Lincoln|Boulevard|Suite H|Newport|Grant County|WV|21521|United States|-5|condo| +35057|AAAAAAAABPIIAAAA|573|Locust North|Avenue|Suite N|Clinton|Seward County|NE|68222|United States|-7|condo| +35058|AAAAAAAACPIIAAAA|855|Lincoln |RD|Suite 140|Woodville|Shelby County|TX|74289|United States|-6|condo| +35059|AAAAAAAADPIIAAAA||||Suite 80|||||||| +35060|AAAAAAAAEPIIAAAA|846|River Spring|ST|Suite 440|Pinhook|Allen Parish|LA|79398|United States|-6|single family| +35061|AAAAAAAAFPIIAAAA|39|View Spring|Way|Suite 120|Marion|Cobb County|GA|30399|United States|-5|condo| +35062|AAAAAAAAGPIIAAAA|980|Williams |Road|Suite F|Plainview|Caldwell County|KY|43683|United States|-6|condo| +35063|AAAAAAAAHPIIAAAA|442|10th |Drive|Suite Y|Oak Grove|Mariposa County|CA|98370|United States|-8|condo| +35064|AAAAAAAAIPIIAAAA|726|Washington |Circle|Suite 90|Shiloh|Nye County|NV|89275|United States|-8|single family| +35065|AAAAAAAAJPIIAAAA|762|Locust Poplar|Blvd|Suite I|Waterloo|Ballard County|KY|41675|United States|-6|single family| +35066|AAAAAAAAKPIIAAAA||Pine Spring|Pkwy|||Marengo County|||United States||| +35067|AAAAAAAALPIIAAAA|613|7th |Ct.|Suite F|Murray|Charlottesville city|VA|22150|United States|-5|apartment| +35068|AAAAAAAAMPIIAAAA|775|4th |RD|Suite V|Highland|Saratoga County|NY|19454|United States|-5|condo| +35069|AAAAAAAANPIIAAAA|120|Elm |Boulevard|Suite 440|Riverside|Le Flore County|OK|79231|United States|-6|condo| +35070|AAAAAAAAOPIIAAAA|142|Forest |Circle|Suite 310|Forest Hills|Stearns County|MN|59237|United States|-6|single family| +35071|AAAAAAAAPPIIAAAA|425|8th 6th|Wy|Suite U|Rockford|Floyd County|VA|21606|United States|-5|apartment| +35072|AAAAAAAAAAJIAAAA|230|Elm |Drive|Suite 240|Lewis|Sheridan County|KS|67066|United States|-6|apartment| +35073|AAAAAAAABAJIAAAA||2nd Maple|Street||Union|Luzerne County||18721||-5|| +35074|AAAAAAAACAJIAAAA|358|East |Circle|Suite X|Maple Grove|Klickitat County|WA|98252|United States|-8|condo| +35075|AAAAAAAADAJIAAAA|749|Cherry |Ct.|Suite 70|Fairfield|Kane County|IL|66192|United States|-6|apartment| +35076|AAAAAAAAEAJIAAAA|||||Macedonia||GA|31087|United States||single family| +35077|AAAAAAAAFAJIAAAA|432|Lincoln Poplar|Ct.|Suite 380|Arcola|Lake County|IL|61654|United States|-6|condo| +35078|AAAAAAAAGAJIAAAA|419|East |Drive|Suite 100|Fairfield|Campbell County|KY|46192|United States|-6|single family| +35079|AAAAAAAAHAJIAAAA|174|Park |Drive|Suite 30|Marion|Bailey County|TX|70399|United States|-6|apartment| +35080|AAAAAAAAIAJIAAAA|377|Main Walnut|Lane|Suite S|Pleasant Hill|Colfax County|NM|83604|United States|-7|apartment| +35081|AAAAAAAAJAJIAAAA|47|4th 2nd|ST|Suite 190|Glenwood|Adams County|OH|43511|United States|-5|single family| +35082|AAAAAAAAKAJIAAAA|656|Willow |Pkwy|Suite 330|Oakland|Mineral County|MT|69843|United States|-7|apartment| +35083|AAAAAAAALAJIAAAA|429|Ridge |Pkwy|Suite V|Crossroads|Palo Alto County|IA|50534|United States|-6|single family| +35084|AAAAAAAAMAJIAAAA|555|Lincoln |Cir.|Suite 290|Forest Hills|Scott County|AR|79237|United States|-6|apartment| +35085|AAAAAAAANAJIAAAA|285|Davis Maple|Dr.|Suite 380|Walnut Grove|Cass County|IL|67752|United States|-6|condo| +35086|AAAAAAAAOAJIAAAA|339|Spring |Ave|Suite J|Clifton|Lafayette Parish|LA|78014|United States|-6|apartment| +35087|AAAAAAAAPAJIAAAA|982|3rd Madison|RD|Suite 200|Red Hill|Warren County|GA|34338|United States|-5|condo| +35088|AAAAAAAAABJIAAAA|980|4th |Drive|Suite E|Edgewood|DeKalb County|GA|30069|United States|-5|condo| +35089|AAAAAAAABBJIAAAA|965|Church Central|RD|Suite S|Mount Pleasant|Jefferson County|IN|41933|United States|-5|single family| +35090|AAAAAAAACBJIAAAA|861|Green Cherry|Avenue|Suite 470|Walnut Grove|McIntosh County|GA|37752|United States|-5|apartment| +35091|AAAAAAAADBJIAAAA|965|Woodland |Court|Suite I|Riverview|Winnebago County|IL|69003|United States|-6|condo| +35092|AAAAAAAAEBJIAAAA|489|Third Pine|Ln|Suite 40|Marion|Oneida County|ID|80399|United States|-7|single family| +35093|AAAAAAAAFBJIAAAA|31|2nd |Ct.|Suite 150|Macedonia|Guadalupe County|TX|71087|United States|-6|condo| +35094|AAAAAAAAGBJIAAAA|945|Walnut Cedar|ST|Suite U|Summit|Osage County|OK|70499|United States|-6|condo| +35095|AAAAAAAAHBJIAAAA|216|Spruce Walnut|Parkway|Suite 290|Willow|Pottawattamie County|IA|56798|United States|-6|apartment| +35096|AAAAAAAAIBJIAAAA|609|Maple 12th|Avenue|Suite 190|Hopewell|Prairie County|AR|70587|United States|-6|condo| +35097|AAAAAAAAJBJIAAAA|493|Hill |Parkway|Suite V|Newtown|Washington County|UT|81749|United States|-7|condo| +35098|AAAAAAAAKBJIAAAA||Center Miller|Ln|Suite 220|Valley View|Jones County||||-6|condo| +35099|AAAAAAAALBJIAAAA|407|First |RD|Suite W|Lincoln|Stafford County|KS|61289|United States|-6|condo| +35100|AAAAAAAAMBJIAAAA|939|Elm |Court|Suite X|Macedonia|San Luis Obispo County|CA|91087|United States|-8|single family| +35101|AAAAAAAANBJIAAAA|311|Valley Valley|Ct.|Suite F|Harmony|Tehama County|CA|95804|United States|-8|apartment| +35102|AAAAAAAAOBJIAAAA|208|Fifth |Ct.|Suite R|Pleasant Valley|Marquette County|WI|52477|United States|-6|single family| +35103|AAAAAAAAPBJIAAAA|820|6th |Ct.|Suite 480|Summit|Lincoln County|ME|01099|United States|-5|apartment| +35104|AAAAAAAAACJIAAAA|103|Birch |Court|Suite 270|Shady Grove|Warren County|NC|22812|United States|-5|single family| +35105|AAAAAAAABCJIAAAA|310|Ridge View|Lane|Suite M|Riverdale|Washington County|IL|69391|United States|-6|apartment| +35106|AAAAAAAACCJIAAAA|||||Woodlawn|Northampton County|||||| +35107|AAAAAAAADCJIAAAA|95|5th |Wy|Suite R|Deerfield|Clay County|SD|59840|United States|-6|single family| +35108|AAAAAAAAECJIAAAA|836|Park |Court|Suite D|Oak Hill|Caldwell County|KY|47838|United States|-6|apartment| +35109|AAAAAAAAFCJIAAAA|412|1st |Court|Suite M|Concord|Washington County|MS|54107|United States|-6|apartment| +35110|AAAAAAAAGCJIAAAA|240|College |Ln|Suite O|Crossroads|Clay County|TN|30534|United States|-5|single family| +35111|AAAAAAAAHCJIAAAA|290|Walnut Woodland|Lane|Suite 390|Spring Valley|Etowah County|AL|36060|United States|-6|apartment| +35112|AAAAAAAAICJIAAAA|100|7th |Ave|Suite 90|Macedonia|Seneca County|OH|41087|United States|-5|apartment| +35113|AAAAAAAAJCJIAAAA|956|Railroad 11th|Ave|Suite F|Fairview|Perry County|OH|45709|United States|-5|apartment| +35114|AAAAAAAAKCJIAAAA|859|Park |Court|Suite I|Clifton|Loudoun County|VA|28014|United States|-5|single family| +35115|AAAAAAAALCJIAAAA|97|Main |Ave|Suite 170|Three Forks|Bannock County|ID|81338|United States|-7|single family| +35116|AAAAAAAAMCJIAAAA|877||Blvd|Suite 150|Franklin|Adams County|||||single family| +35117|AAAAAAAANCJIAAAA|130|4th Cedar|Court|Suite B|Union Hill|Pierce County|ND|57746|United States|-6|single family| +35118|AAAAAAAAOCJIAAAA|139|Green |Boulevard|Suite 200|Bethel|Harding County|NM|85281|United States|-7|single family| +35119|AAAAAAAAPCJIAAAA|123|Church Ninth|Pkwy|Suite 130|Clearview|Nevada County|AR|75495|United States|-6|condo| +35120|AAAAAAAAADJIAAAA|593|Wilson |Drive|Suite 340|Florence|Carbon County|MT|63394|United States|-7|single family| +35121|AAAAAAAABDJIAAAA|491|Mill Miller|Wy|Suite M|Mount Vernon|Livingston County|NY|18482|United States|-5|condo| +35122|AAAAAAAACDJIAAAA|449|Davis |RD|Suite 150|Birmingham|Steuben County|NY|13372|United States|-5|condo| +35123|AAAAAAAADDJIAAAA|90|Miller Church|Way|Suite B|Mount Pleasant|Holt County|MO|61933|United States|-6|single family| +35124|AAAAAAAAEDJIAAAA|645|Green |Drive|Suite I|Highland Park|Dawes County|NE|66534|United States|-6|condo| +35125|AAAAAAAAFDJIAAAA|31|South Sunset|Dr.|Suite 190|Mount Pleasant|Kane County|UT|81933|United States|-7|single family| +35126|AAAAAAAAGDJIAAAA|640|Lake |Street|Suite T|Appleton|Franklin County|KY|44240|United States|-6|single family| +35127|AAAAAAAAHDJIAAAA|361|Oak Washington|Drive|Suite J|Perkins|Hendry County|FL|31852|United States|-5|condo| +35128|AAAAAAAAIDJIAAAA|319|Center 8th|Ave|Suite 110|Riverdale|Lincoln County|AR|79391|United States|-6|single family| +35129|AAAAAAAAJDJIAAAA|892|5th |Drive|Suite 130|Riverview|Pembina County|ND|59003|United States|-6|condo| +35130|AAAAAAAAKDJIAAAA|242|Lee |Circle|Suite J|Summerville|Vilas County|WI|52033|United States|-6|single family| +35131|AAAAAAAALDJIAAAA|203|Railroad |Drive|Suite O|Lakeside|Spartanburg County|SC|29532|United States|-5|condo| +35132|AAAAAAAAMDJIAAAA|978|11th |RD|Suite M|Greenville|Jasper County|TX|71387|United States|-6|condo| +35133|AAAAAAAANDJIAAAA|488|3rd |Drive|Suite I|Oak Grove|New Haven County|CT|08970|United States|-5|single family| +35134|AAAAAAAAODJIAAAA|11|North 5th|Boulevard|Suite 100|Greenville|Colquitt County|GA|31387|United States|-5|single family| +35135|AAAAAAAAPDJIAAAA|964|Meadow Center|Road|Suite C|Oakland|Carroll County|MS|59843|United States|-6|single family| +35136|AAAAAAAAAEJIAAAA|245|9th |Court|Suite 30|Snug Harbor|Nassau County|NY|17936|United States|-5|apartment| +35137|AAAAAAAABEJIAAAA|138|6th |Court|Suite D|Brownsville|Delaware County|PA|19310|United States|-5|single family| +35138|AAAAAAAACEJIAAAA|489|Main |Ave|Suite 300|Mount Vernon|Maricopa County|AZ|88482|United States|-7|condo| +35139|AAAAAAAADEJIAAAA|664|Adams |Ln|Suite 10|Brownsville|Montgomery County|VA|29310|United States|-5|condo| +35140|AAAAAAAAEEJIAAAA|39|South 2nd|RD|Suite 290|Newport|Prince Edward County|VA|21521|United States|-5|apartment| +35141|AAAAAAAAFEJIAAAA|151|Hickory |Dr.|Suite 210|Antioch|Pierce County|WI|58605|United States|-6|single family| +35142|AAAAAAAAGEJIAAAA|14|Spruce Locust|Pkwy|Suite X|Highland|Outagamie County|WI|59454|United States|-6|condo| +35143|AAAAAAAAHEJIAAAA|944|3rd Willow|Boulevard|Suite T|Hillcrest|Oneida County|ID|83003|United States|-7|single family| +35144|AAAAAAAAIEJIAAAA|542|9th 3rd|Circle|Suite I|Walnut Grove|McNairy County|TN|37752|United States|-6|condo| +35145|AAAAAAAAJEJIAAAA|275|Laurel Fourth|Court|Suite R|Oakland|Custer County|ID|89843|United States|-7|single family| +35146|AAAAAAAAKEJIAAAA|22|Fifteenth |Cir.|Suite 90|Five Forks|Carter County|OK|72293|United States|-6|condo| +35147|AAAAAAAALEJIAAAA|392|Church Eigth|Dr.|Suite M|Oak Hill|Wabaunsee County|KS|67838|United States|-6|condo| +35148|AAAAAAAAMEJIAAAA|353|West |Avenue|Suite 340|Sheffield|Wichita County|KS|66896|United States|-6|condo| +35149|AAAAAAAANEJIAAAA|750|4th Sycamore|Road|Suite E|Five Points|Wayne County|IL|66098|United States|-6|apartment| +35150|AAAAAAAAOEJIAAAA|212|Broadway |Cir.|Suite L|Buena Vista|Decatur County|KS|65752|United States|-6|apartment| +35151|AAAAAAAAPEJIAAAA|5|13th |Circle|Suite 70|Unionville|Crawford County|MI|41711|United States|-5|condo| +35152|AAAAAAAAAFJIAAAA|408|North 9th|Pkwy|Suite 340|Fairview|Kingman County|KS|65709|United States|-6|condo| +35153|AAAAAAAABFJIAAAA|834|Green |Wy|Suite W|Wilson|Grant County|WI|56971|United States|-6|single family| +35154|AAAAAAAACFJIAAAA|367|Lincoln Miller|Wy|Suite Q|Oak Grove|Richland County|IL|68370|United States|-6|single family| +35155|AAAAAAAADFJIAAAA|823|Woodland |Boulevard|Suite 90|Lincoln|Benzie County|MI|41289|United States|-5|single family| +35156|AAAAAAAAEFJIAAAA|488|Sunset |Ct.|Suite 50|Forest Hills|Lapeer County|MI|49237|United States|-5|apartment| +35157|AAAAAAAAFFJIAAAA|395|Ridge |Blvd|Suite E|Liberty|Calumet County|WI|53451|United States|-6|single family| +35158|AAAAAAAAGFJIAAAA|806|3rd First|Street|Suite B|Wildwood|Sullivan County|NH|07471|United States|-5|single family| +35159|AAAAAAAAHFJIAAAA|688|Railroad Washington|Street|Suite 250|Mount Vernon|Pinal County|AZ|88482|United States|-7|apartment| +35160|AAAAAAAAIFJIAAAA|911|Lakeview |Boulevard|Suite 470|Unionville|Hempstead County|AR|71711|United States|-6|single family| +35161|AAAAAAAAJFJIAAAA|335|Meadow First|Lane|Suite 490|Oak Ridge|Alfalfa County|OK|78371|United States|-6|apartment| +35162|AAAAAAAAKFJIAAAA|180|Sunset 4th|Pkwy|Suite M|Belmont|Keya Paha County|NE|60191|United States|-7|single family| +35163|AAAAAAAALFJIAAAA|131|Third 10th|Road|Suite 260|Shiloh|Hartford County|CT|09875|United States|-5|apartment| +35164|AAAAAAAAMFJIAAAA|434|3rd West|Way|Suite S|Oakwood|Bureau County|IL|60169|United States|-6|single family| +35165|AAAAAAAANFJIAAAA|816|10th 1st|Avenue|Suite 230|Lakeside|Chippewa County|WI|59532|United States|-6|single family| +35166|AAAAAAAAOFJIAAAA|443|Fifteenth |Circle|Suite 80|Springfield|Santa Clara County|CA|99303|United States|-8|condo| +35167|AAAAAAAAPFJIAAAA|516|Broadway |Circle|Suite 160|Franklin|Moore County|NC|29101|United States|-5|single family| +35168|AAAAAAAAAGJIAAAA|508|15th Green|Dr.|Suite 400|Midway|Union County|NM|81904|United States|-7|apartment| +35169|AAAAAAAABGJIAAAA||Spring |||||OK||United States|-6|| +35170|AAAAAAAACGJIAAAA|884|9th Sycamore|Ln|Suite 330|Jamestown|Middlesex County|NJ|07467|United States|-5|apartment| +35171|AAAAAAAADGJIAAAA|890|Franklin |Wy|Suite A|Edgewood|Dillingham Census Area|AK|90069|United States|-9|condo| +35172|AAAAAAAAEGJIAAAA|575|Cedar |Drive|Suite 370|Sunnyside|Trempealeau County|WI|51952|United States|-6|single family| +35173|AAAAAAAAFGJIAAAA|708|Birch Johnson|Circle|Suite 200|Pleasant Grove|Waupaca County|WI|54136|United States|-6|apartment| +35174|AAAAAAAAGGJIAAAA|736|2nd Poplar|Cir.|Suite 410|Pleasant Hill|Saline County|IL|63604|United States|-6|single family| +35175|AAAAAAAAHGJIAAAA|274|West |Lane|Suite 10|Red Hill|Lafayette County|MS|54338|United States|-6|apartment| +35176|AAAAAAAAIGJIAAAA|230|Sunset Lee|Pkwy|Suite O|Hillcrest|Box Elder County|UT|83003|United States|-7|single family| +35177|AAAAAAAAJGJIAAAA|472|Ridge Lincoln|Blvd|Suite 330|Unionville|Houghton County|MI|41711|United States|-5|apartment| +35178|AAAAAAAAKGJIAAAA|34|1st |Dr.|Suite C|Ashland|Lumpkin County|GA|34244|United States|-5|apartment| +35179|AAAAAAAALGJIAAAA|704|1st Broadway|Wy|Suite 380|Crossroads|Koochiching County|MN|50534|United States|-6|single family| +35180|AAAAAAAAMGJIAAAA|579|Fifth |ST|Suite 240|Summit|Anson County|NC|20499|United States|-5|condo| +35181|AAAAAAAANGJIAAAA|||Cir.||Lebanon|McCook County||52898|United States||| +35182|AAAAAAAAOGJIAAAA|745|Lincoln Woodland|Avenue|Suite 60|Centerville|Obion County|TN|30059|United States|-6|apartment| +35183|AAAAAAAAPGJIAAAA|880|Fourth |Street|Suite 370|Friendship|Jefferson County|AR|74536|United States|-6|condo| +35184|AAAAAAAAAHJIAAAA|632|Hillcrest Second|Cir.|Suite T|Mount Vernon|Pottawattamie County|IA|58482|United States|-6|apartment| +35185|AAAAAAAABHJIAAAA||||Suite 150|||MS|||-6|| +35186|AAAAAAAACHJIAAAA|534|Laurel 6th|Way|Suite H|Hamilton|Trigg County|KY|42808|United States|-5|apartment| +35187|AAAAAAAADHJIAAAA|869|First 8th|Cir.|Suite T|Blanchard|Madison County|TX|75985|United States|-6|single family| +35188|AAAAAAAAEHJIAAAA|810|Lee |Parkway|Suite 490|Green Acres|Bureau County|IL|67683|United States|-6|single family| +35189|AAAAAAAAFHJIAAAA|212|Maple Park|RD|Suite 320|Westminster|Jasper County|TX|76549|United States|-6|apartment| +35190|AAAAAAAAGHJIAAAA|310|Franklin Dogwood|Ave|Suite 470|Brownsville|Wheatland County|MT|69310|United States|-7|single family| +35191|AAAAAAAAHHJIAAAA|464|East 7th|Court|Suite 440|Hopewell|Price County|WI|50587|United States|-6|apartment| +35192|AAAAAAAAIHJIAAAA|856|Davis |RD|Suite A|Hopewell|Wilkes County|NC|20587|United States|-5|apartment| +35193|AAAAAAAAJHJIAAAA|867|Willow 1st|Cir.|Suite N|Leesburg|Bronx County|NY|15605|United States|-5|single family| +35194|AAAAAAAAKHJIAAAA|267|Sunset |Drive|Suite 150|Mount Zion|Greene County|IA|58054|United States|-6|apartment| +35195|AAAAAAAALHJIAAAA|772|1st Oak|Circle|Suite 170|Salem|Nicholas County|WV|28048|United States|-5|apartment| +35196|AAAAAAAAMHJIAAAA|772|Washington Seventh|Way|Suite M|Newport|Walton County|GA|31521|United States|-5|condo| +35197|AAAAAAAANHJIAAAA|125|Wilson View|Way|Suite R|Hopewell|Barton County|MO|60587|United States|-6|single family| +35198|AAAAAAAAOHJIAAAA|165|Green Walnut|Ln|Suite E|Pleasant Valley|Calaveras County|CA|92477|United States|-8|single family| +35199|AAAAAAAAPHJIAAAA|385|2nd |Ave|Suite N|Buena Vista|Webb County|TX|75752|United States|-6|condo| +35200|AAAAAAAAAIJIAAAA|366|East 13th|ST|Suite R|Sulphur Springs|Kay County|OK|78354|United States|-6|apartment| +35201|AAAAAAAABIJIAAAA|394|8th ||Suite A|Green Acres|||27683|United States||| +35202|AAAAAAAACIJIAAAA|939|Park Main|Parkway|Suite 180|Union Hill|Yukon-Koyukuk Census Area|AK|97746|United States|-9|condo| +35203|AAAAAAAADIJIAAAA|884|Oak |Blvd|Suite C|Carpenter|Codington County|SD|51147|United States|-6|condo| +35204|AAAAAAAAEIJIAAAA|502|Willow Park|Cir.|Suite E|Dewey|Colquitt County|GA|31160|United States|-5|apartment| +35205|AAAAAAAAFIJIAAAA|426|2nd |Road|Suite 260|Plainview|Kiowa County|OK|73683|United States|-6|single family| +35206|AAAAAAAAGIJIAAAA|388|Fifth Madison|Road|Suite 10|Springfield|Runnels County|TX|79303|United States|-6|apartment| +35207|AAAAAAAAHIJIAAAA|||||Midway||TX||United States|-6|| +35208|AAAAAAAAIIJIAAAA|810|Church |Circle|Suite 480|Sulphur Springs|Knott County|KY|48354|United States|-5|apartment| +35209|AAAAAAAAJIJIAAAA|434|Lake Valley|Boulevard|Suite C|Florence|Lee County|GA|33394|United States|-5|single family| +35210|AAAAAAAAKIJIAAAA|503|Cedar Spring|Ct.|Suite P|New Hope|Shelby County|TX|79431|United States|-6|apartment| +35211|AAAAAAAALIJIAAAA|154|Seventh |Ave|Suite 320|Charleston|Carson City|NV|88018|United States|-8|condo| +35212|AAAAAAAAMIJIAAAA|155|East |Boulevard|Suite Y|Clifton|Brown County|TX|78014|United States|-6|apartment| +35213|AAAAAAAANIJIAAAA|97|15th Ninth|Drive|Suite M|Ashland|Niobrara County|WY|84244|United States|-7|single family| +35214|AAAAAAAAOIJIAAAA|983|Hickory |Drive|Suite 460|Hopewell|Gregory County|SD|50587|United States|-6|condo| +35215|AAAAAAAAPIJIAAAA|748|6th Green|Avenue|Suite 380|Shady Grove|Deuel County|NE|62812|United States|-6|condo| +35216|AAAAAAAAAJJIAAAA|266|Johnson Washington|Cir.|Suite 460|Concord|Poquoson city|VA|24107|United States|-5|apartment| +35217|AAAAAAAABJJIAAAA|856|Oak |Court|Suite 470|Union Hill|Greenwood County|KS|67746|United States|-6|condo| +35218|AAAAAAAACJJIAAAA|484|Johnson |Street|Suite B|Shady Grove|Miller County|MO|62812|United States|-6|single family| +35219|AAAAAAAADJJIAAAA|889|12th Williams|Blvd|Suite K|Antioch|Stanislaus County|CA|98605|United States|-8|condo| +35220|AAAAAAAAEJJIAAAA|665|Spring Maple|Ln|Suite P|Buena Vista|Cheyenne County|CO|85752|United States|-7|single family| +35221|AAAAAAAAFJJIAAAA|244|College Forest|Way|Suite L|Valley View|Stoddard County|MO|65124|United States|-6|apartment| +35222|AAAAAAAAGJJIAAAA|813|College Third|Ave|Suite 300|Riverdale|Watonwan County|MN|59391|United States|-6|single family| +35223|AAAAAAAAHJJIAAAA|694|Lee |Boulevard|Suite N|Friendship|Dinwiddie County|VA|24536|United States|-5|condo| +35224|AAAAAAAAIJJIAAAA|397|Washington Park|Ln|Suite 410|Lakewood|Hempstead County|AR|78877|United States|-6|condo| +35225|AAAAAAAAJJJIAAAA|559|Sunset Second|ST|Suite 470|Greenwood|Alameda County|CA|98828|United States|-8|condo| +35226|AAAAAAAAKJJIAAAA|14|Highland |Avenue|Suite R|Arthur|Aurora County|SD|55965|United States|-6|apartment| +35227|AAAAAAAALJJIAAAA|628|4th |Dr.|Suite 230|Oak Ridge|Tucker County|WV|28371|United States|-5|apartment| +35228|AAAAAAAAMJJIAAAA|811|Lincoln |Street|Suite 80|Spring Valley|Lake County|IL|66060|United States|-6|apartment| +35229|AAAAAAAANJJIAAAA|953|Jackson 2nd|Road|Suite D|Mountain View|Emmet County|MI|44466|United States|-5|apartment| +35230|AAAAAAAAOJJIAAAA|217|Fifteenth |Court|Suite U|Farmington||KS||||| +35231|AAAAAAAAPJJIAAAA|42|Ash Lincoln|Dr.|Suite F|New Hope|Denton County|TX|79431|United States|-6|apartment| +35232|AAAAAAAAAKJIAAAA|324|1st |Wy|Suite 110|Union|Olmsted County|MN|58721|United States|-6|condo| +35233|AAAAAAAABKJIAAAA|940|Madison |Ct.|Suite R|Hillcrest|Jefferson County|WV|23003|United States|-5|single family| +35234|AAAAAAAACKJIAAAA|773|Lee |Ct.|Suite 360|Hillcrest|Patrick County|VA|23003|United States|-5|apartment| +35235|AAAAAAAADKJIAAAA|891|Church 15th|Cir.|Suite 470|Gilmore|Bottineau County|ND|55464|United States|-6|condo| +35236|AAAAAAAAEKJIAAAA|298|Main Jackson|Court|Suite K|Belmont|Marshall County|MS|50191|United States|-6|condo| +35237|AAAAAAAAFKJIAAAA|415|1st Mill|Court|Suite C|Empire|Williamsburg County|SC|24145|United States|-5|apartment| +35238|AAAAAAAAGKJIAAAA|944|Birch River|Drive|Suite W|Deerfield|Harrison County|MO|69840|United States|-6|apartment| +35239|AAAAAAAAHKJIAAAA|821|View |Way|Suite H|Georgetown|Perquimans County|NC|27057|United States|-5|apartment| +35240|AAAAAAAAIKJIAAAA|815|Seventh Washington|Way|Suite 30|Springfield|Alpena County|MI|49303|United States|-5|condo| +35241|AAAAAAAAJKJIAAAA|822|West Lakeview|ST|Suite M|Stringtown|Monroe County|FL|30162|United States|-5|condo| +35242|AAAAAAAAKKJIAAAA|287|Highland |Road|Suite 80|Liberty|Hertford County|NC|23451|United States|-5|apartment| +35243|AAAAAAAALKJIAAAA|94|Sunset |Pkwy|Suite 480|Springdale|Morgan County|OH|48883|United States|-5|condo| +35244|AAAAAAAAMKJIAAAA||||Suite L|Ashland|Tillamook County|OR||||| +35245|AAAAAAAANKJIAAAA|368|Hill |Way|Suite 100|Brownsville|Pike County|AR|79310|United States|-6|single family| +35246|AAAAAAAAOKJIAAAA|121|Green 1st|RD|Suite 130|Lakeview|Walker County|TX|78579|United States|-6|condo| +35247|AAAAAAAAPKJIAAAA|281|Forest |Lane|Suite V|Edgewood|Sac County|IA|50069|United States|-6|condo| +35248|AAAAAAAAALJIAAAA|651|Maple |Ct.|Suite X|Stringtown|Tucker County|WV|20162|United States|-5|single family| +35249|AAAAAAAABLJIAAAA|885|Park |Cir.|Suite L|Mount Vernon|Fallon County|MT|68482|United States|-7|condo| +35250|AAAAAAAACLJIAAAA|394|Railroad |Parkway|Suite 260|Woodville|Sanborn County|SD|54289|United States|-7|condo| +35251|AAAAAAAADLJIAAAA|31|Willow 12th|Parkway|Suite O|Harmony|Westmoreland County|PA|15804|United States|-5|condo| +35252|AAAAAAAAELJIAAAA|158|5th 6th|Drive|Suite R|Howell|Preston County|WV|24854|United States|-5|single family| +35253|AAAAAAAAFLJIAAAA|2|Washington Jefferson|ST|Suite E|Lincoln|Pacific County|WA|91289|United States|-8|single family| +35254|AAAAAAAAGLJIAAAA|446|Jackson 1st|Court|Suite H|Bethel|Menifee County|KY|45281|United States|-5|single family| +35255|AAAAAAAAHLJIAAAA|77|South Chestnut|Ave|Suite 180|Bunker Hill|Vernon Parish|LA|70150|United States|-6|apartment| +35256|AAAAAAAAILJIAAAA|837|Hill |Blvd|Suite 10|Lakeview|Candler County|GA|38579|United States|-5|condo| +35257|AAAAAAAAJLJIAAAA|44|11th Forest|Way|Suite 370|Glendale|Catoosa County|GA|33951|United States|-5|condo| +35258|AAAAAAAAKLJIAAAA||Madison |Way||Enterprise|||51757|United States|-7|| +35259|AAAAAAAALLJIAAAA|480|Oak Elevnth|Ave|Suite 50|Riverview|Shasta County|CA|99003|United States|-8|condo| +35260|AAAAAAAAMLJIAAAA|60|Cedar Spruce|RD|Suite M|Mount Pleasant|Upton County|TX|71933|United States|-6|single family| +35261|AAAAAAAANLJIAAAA|78|Pine Davis|Way|Suite 40|Arlington|Lincoln County|GA|36557|United States|-5|single family| +35262|AAAAAAAAOLJIAAAA|531|Maple |RD|Suite 190|Leon|Presidio County|TX|70913|United States|-6|apartment| +35263|AAAAAAAAPLJIAAAA|61|East |Parkway|Suite 450|Cedar Grove|Stafford County|KS|60411|United States|-6|single family| +35264|AAAAAAAAAMJIAAAA|65|Laurel 4th|Pkwy|Suite 360|Woodville|Houston County|TX|74289|United States|-6|single family| +35265|AAAAAAAABMJIAAAA|178|5th 7th|Avenue|Suite W|Brownsville|Jones County|IA|59310|United States|-6|condo| +35266|AAAAAAAACMJIAAAA|940|Lake Chestnut|Parkway|Suite 320|Pomona|Okmulgee County|OK|74153|United States|-6|condo| +35267|AAAAAAAADMJIAAAA|709|Williams |Lane|Suite 370|Walnut Grove|Wayne County|MS|57752|United States|-6|apartment| +35268|AAAAAAAAEMJIAAAA|237|Main |Way|Suite G|Valley View|Fall River County|SD|55124|United States|-6|condo| +35269|AAAAAAAAFMJIAAAA|194|12th |Circle|Suite 310|Stringtown|Black Hawk County|IA|50162|United States|-6|single family| +35270|AAAAAAAAGMJIAAAA|683|Cherry Broadway|Road|Suite 250|Woodland|Park County|MT|64854|United States|-7|apartment| +35271|AAAAAAAAHMJIAAAA|898|Willow |Court|Suite 410|Post Oak|Story County|IA|58567|United States|-6|condo| +35272|AAAAAAAAIMJIAAAA|80|10th Ash|Blvd|Suite T|Green Acres|Brown County|NE|67683|United States|-6|apartment| +35273|AAAAAAAAJMJIAAAA|433|View Woodland|Blvd|Suite G|Belmont|Sullivan County|IN|40191|United States|-5|condo| +35274|AAAAAAAAKMJIAAAA|251|2nd 10th|Court|Suite 20|Riverview|Botetourt County|VA|29003|United States|-5|apartment| +35275|AAAAAAAALMJIAAAA|205|6th |Cir.|Suite H|Kingston|Klamath County|OR|94975|United States|-8|condo| +35276|AAAAAAAAMMJIAAAA||2nd North||Suite J||||||-5|| +35277|AAAAAAAANMJIAAAA|141|13th Miller|Cir.|Suite D|Brownsville|Santa Cruz County|AZ|89310|United States|-7|apartment| +35278|AAAAAAAAOMJIAAAA|774|View Main|Court|Suite B|Green Acres|Ferry County|WA|97683|United States|-8|condo| +35279|AAAAAAAAPMJIAAAA|181|Walnut |Wy|Suite W|Glenwood|Barren County|KY|43511|United States|-6|apartment| +35280|AAAAAAAAANJIAAAA|274|Elm |Parkway|Suite 130|Enterprise|Jefferson County|IL|61757|United States|-6|condo| +35281|AAAAAAAABNJIAAAA|758|Jefferson |Ct.|Suite X|Red Hill|Talbot County|MD|24338|United States|-5|condo| +35282|AAAAAAAACNJIAAAA|586|Madison |Ave|Suite 210|Sunnyside|Madison County|NC|21952|United States|-5|condo| +35283|AAAAAAAADNJIAAAA|959|Fourth Lincoln|Circle|Suite 260|Peru|Kodiak Island Borough|AK|90302|United States|-9|single family| +35284|AAAAAAAAENJIAAAA|15|8th Willow|Ln|Suite 470|Pine Grove|Campbell County|SD|54593|United States|-6|apartment| +35285|AAAAAAAAFNJIAAAA|15|Oak |Drive|Suite 400|Lakeview|Baldwin County|AL|38579|United States|-6|apartment| +35286|AAAAAAAAGNJIAAAA|400|Spring Jefferson|Road|Suite 230|Highland|Cheshire County|NH|09454|United States|-5|condo| +35287|AAAAAAAAHNJIAAAA|700|Ridge |Ct.|Suite 20|Edgewood|Douglas County|NE|60069|United States|-6|apartment| +35288|AAAAAAAAINJIAAAA|360|Washington |Road|Suite S|Stringtown|Stark County|OH|40162|United States|-5|single family| +35289|AAAAAAAAJNJIAAAA|301|Lincoln |Parkway|Suite X|Highland|Madison County|MO|69454|United States|-6|single family| +35290|AAAAAAAAKNJIAAAA|331|Laurel Oak|Ln|Suite Y|Wilson|Bottineau County|ND|56971|United States|-6|apartment| +35291|AAAAAAAALNJIAAAA|285|Locust Maple|Ave|Suite M|Georgetown|Reno County|KS|67057|United States|-6|apartment| +35292|AAAAAAAAMNJIAAAA|796|Hill |Way|Suite V|Pleasant Hill|Marion County|OR|93604|United States|-8|single family| +35293|AAAAAAAANNJIAAAA|444|Park Fifth|Blvd|Suite P|Mount Olive|Douglas County|MO|68059|United States|-6|condo| +35294|AAAAAAAAONJIAAAA|354|Maple Dogwood|Street|Suite W|Denmark|Marquette County|MI|45576|United States|-5|condo| +35295|AAAAAAAAPNJIAAAA|167|8th Park|Street|Suite 70|Jamestown|Hays County|TX|76867|United States|-6|apartment| +35296|AAAAAAAAAOJIAAAA|935|Williams |Dr.|Suite 320|Riverside|Barton County|KS|69231|United States|-6|condo| +35297|AAAAAAAABOJIAAAA|825|Tenth Meadow|Ct.|Suite 100|Mountain View|Hampden County|MA|05066|United States|-5|apartment| +35298|AAAAAAAACOJIAAAA|531|Park 7th|Pkwy|Suite 120|Fairfield|Caldwell County|KY|46192|United States|-6|condo| +35299|AAAAAAAADOJIAAAA|688|Willow |Pkwy|Suite Y|Richfield|Suffolk County|MA|06796|United States|-5|condo| +35300|AAAAAAAAEOJIAAAA|915|Pine Davis|Lane|Suite 240|Ashland|Arkansas County|AR|74244|United States|-6|condo| +35301|AAAAAAAAFOJIAAAA|656|5th |Drive|Suite 400|Glendale|Christian County|KY|43951|United States|-6|apartment| +35302|AAAAAAAAGOJIAAAA|593|14th Oak|Lane|Suite 120|Red Hill|Franklin County|FL|34338|United States|-5|single family| +35303|AAAAAAAAHOJIAAAA|749|Ash Madison|Cir.|Suite B|Bridgeport|Sevier County|AR|75817|United States|-6|apartment| +35304|AAAAAAAAIOJIAAAA|319|Central |Lane|Suite 440|Kingston|Trinity County|CA|94975|United States|-8|apartment| +35305|AAAAAAAAJOJIAAAA|631|Poplar 6th|Court|Suite 440|Mount Vernon|Wayne County|IN|48482|United States|-5|apartment| +35306|AAAAAAAAKOJIAAAA|155|Sycamore |Court|Suite C|Union Hill|District of Columbia|DC|27746|United States|-5|single family| +35307|AAAAAAAALOJIAAAA|830|12th |Ave|Suite V|White Oak|Okeechobee County|FL|36668|United States|-5|apartment| +35308|AAAAAAAAMOJIAAAA|34|1st 8th|Court|Suite 70|Macedonia|Kearney County|NE|61087|United States|-7|condo| +35309|AAAAAAAANOJIAAAA|26|Railroad Ninth|Drive|Suite 230|Springhill|Mecosta County|MI|44602|United States|-5|apartment| +35310|AAAAAAAAOOJIAAAA|426|6th 7th|Ave|Suite 100|Glenwood|Alger County|MI|43511|United States|-5|single family| +35311|AAAAAAAAPOJIAAAA|364|10th 7th|RD|Suite 340|Greenville|Thayer County|NE|61387|United States|-7|single family| +35312|AAAAAAAAAPJIAAAA|699|Maple |Parkway|Suite V|Sulphur Springs|Morton County|KS|68354|United States|-6|single family| +35313|AAAAAAAABPJIAAAA|69|3rd |ST|Suite J|Edgewood|Coffee County|GA|30069|United States|-5|condo| +35314|AAAAAAAACPJIAAAA|510|8th Center|Wy|Suite R|Providence|Tippecanoe County|IN|46614|United States|-5|apartment| +35315|AAAAAAAADPJIAAAA|920|Church |ST|Suite 410|Elkton|Grant County|KY|43481|United States|-6|condo| +35316|AAAAAAAAEPJIAAAA|382|Woodland |Ave|Suite 30|Bunker Hill|Crittenden County|KY|40150|United States|-6|condo| +35317|AAAAAAAAFPJIAAAA|764|Miller |Dr.|Suite L|Montague|Flathead County|MT|64062|United States|-7|single family| +35318|AAAAAAAAGPJIAAAA|821|Center Church|Blvd|Suite 240|Woodlawn|Wilkinson County|GA|34098|United States|-5|condo| +35319|AAAAAAAAHPJIAAAA|121|Church |Ln|Suite G|Hidden Valley|Pinellas County|FL|35521|United States|-5|single family| +35320|AAAAAAAAIPJIAAAA|581|3rd |Blvd|Suite 320|Louisville|Jackson County|AL|34464|United States|-6|condo| +35321|AAAAAAAAJPJIAAAA|763|11th Valley|Ave|Suite 450|Woodville|Granite County|MT|64289|United States|-7|apartment| +35322|AAAAAAAAKPJIAAAA|3||Avenue|||Van Wert County|OH|||-5|| +35323|AAAAAAAALPJIAAAA||||Suite 350||Schenectady County||||-5|| +35324|AAAAAAAAMPJIAAAA|486|1st Chestnut|Ave|Suite T|Newport|Lanier County|GA|31521|United States|-5|apartment| +35325|AAAAAAAANPJIAAAA|588|Elm Green|Avenue|Suite X|Philadelphia|Monroe County|MI|45591|United States|-5|condo| +35326|AAAAAAAAOPJIAAAA|572|Railroad |Avenue|Suite U|Summit|Avery County|NC|20499|United States|-5|single family| +35327|AAAAAAAAPPJIAAAA|571|12th Broadway|Lane|Suite 250|Kingston|Yadkin County|NC|24975|United States|-5|condo| +35328|AAAAAAAAAAKIAAAA|90|First 7th|RD|Suite X|Ellsworth|Bingham County|ID|85079|United States|-7|single family| +35329|AAAAAAAABAKIAAAA|888|Adams |Cir.|Suite 460|Pleasant Valley|Marion County|MS|52477|United States|-6|apartment| +35330|AAAAAAAACAKIAAAA|711|10th |RD|Suite 240|Lakeside|Ventura County|CA|99532|United States|-8|condo| +35331|AAAAAAAADAKIAAAA|302|8th |Street|Suite Y|Glenwood|Del Norte County|CA|93511|United States|-8|condo| +35332|AAAAAAAAEAKIAAAA|439|Davis Green|Street|Suite G|Greenville|Chase County|KS|61387|United States|-6|single family| +35333|AAAAAAAAFAKIAAAA|271|South Twelfth|Avenue|Suite 440|Franklin|Comanche County|OK|79101|United States|-6|condo| +35334|AAAAAAAAGAKIAAAA|699|Williams Lake|Court|Suite B|Altamont|Gibson County|TN|39387|United States|-5|condo| +35335|AAAAAAAAHAKIAAAA|450|Spruce 1st|Circle|Suite 170|Ashland|Lawrence County|AR|74244|United States|-6|apartment| +35336|AAAAAAAAIAKIAAAA|764|11th Pine|Pkwy|Suite U|Newtown|Bailey County|TX|71749|United States|-6|single family| +35337|AAAAAAAAJAKIAAAA|494|Cedar Willow|Ln|Suite 80|Brownsville|Owyhee County|ID|89310|United States|-7|apartment| +35338|AAAAAAAAKAKIAAAA|587|South Locust|Circle|Suite L|Fairfield|Nueces County|TX|76192|United States|-6|condo| +35339|AAAAAAAALAKIAAAA|441|10th |Wy|Suite G|Friendship|Lake County|TN|34536|United States|-6|condo| +35340|AAAAAAAAMAKIAAAA|613|Park |Road|Suite O|Hopewell|Grenada County|MS|50587|United States|-6|single family| +35341|AAAAAAAANAKIAAAA|53|Fourth Seventh|Ln|Suite 200|Highland|Okfuskee County|OK|79454|United States|-6|condo| +35342|AAAAAAAAOAKIAAAA|352|Laurel |Dr.|Suite V|Simpson|Prince George County|MD|29865|United States|-5|condo| +35343|AAAAAAAAPAKIAAAA|788|12th |Drive|Suite 330|Franklin|Rapides Parish|LA|79101|United States|-6|single family| +35344|AAAAAAAAABKIAAAA|310|Lake |Ct.|Suite Q|Clinton|Pike County|PA|18222|United States|-5|condo| +35345|AAAAAAAABBKIAAAA|114|South 4th|Boulevard|Suite V|Union|Washington County|FL|38721|United States|-5|condo| +35346|AAAAAAAACBKIAAAA|566|Pine Cedar|Ave|Suite K|Highland|Madison Parish|LA|79454|United States|-6|apartment| +35347|AAAAAAAADBKIAAAA|568|Church Birch|Road|Suite 420|Union|Ben Hill County|GA|38721|United States|-5|apartment| +35348|AAAAAAAAEBKIAAAA|568|Green Locust|Pkwy|Suite 440|Hopewell|Bee County|TX|70587|United States|-6|apartment| +35349|AAAAAAAAFBKIAAAA|609|Valley |Pkwy|Suite 350|Glendale|Chelan County|WA|93951|United States|-8|single family| +35350|AAAAAAAAGBKIAAAA|917|Sixth |Ct.|Suite N|Providence|Berrien County|MI|46614|United States|-5|apartment| +35351|AAAAAAAAHBKIAAAA|222|11th |Cir.|Suite 180|Calhoun|Tompkins County|NY|16909|United States|-5|condo| +35352|AAAAAAAAIBKIAAAA|977|Mill Smith|Wy|Suite P|Bethel|Gallatin County|IL|65281|United States|-6|single family| +35353|AAAAAAAAJBKIAAAA|6|North River|Way|Suite 50|Five Points|Madison County|GA|36098|United States|-5|apartment| +35354|AAAAAAAAKBKIAAAA|969|Highland Cherry|Ave|Suite G|Belmont|Bourbon County|KS|60191|United States|-6|single family| +35355|AAAAAAAALBKIAAAA|305|Birch |Cir.|Suite 440|Marion|Gaston County|NC|20399|United States|-5|apartment| +35356|AAAAAAAAMBKIAAAA|50|Lakeview Jackson|Pkwy|Suite C|Springdale|Westmoreland County|PA|18883|United States|-5|apartment| +35357|AAAAAAAANBKIAAAA|49||Avenue||Maple Grove|Niobrara County||88252|||| +35358|AAAAAAAAOBKIAAAA|518|Willow |Avenue|Suite J|Mountain View|Miller County|MO|64466|United States|-6|apartment| +35359|AAAAAAAAPBKIAAAA|147|Valley |Drive|Suite V|Wilson|Essex County|VT|07571|United States|-5|condo| +35360|AAAAAAAAACKIAAAA|207|Ridge Park|Court|Suite 30|Wilson|Audrain County|MO|66971|United States|-6|single family| +35361|AAAAAAAABCKIAAAA|840|Highland |Way|Suite 250|Springdale|Silver Bow County|MT|68883|United States|-7|apartment| +35362|AAAAAAAACCKIAAAA|949|Maple Center|Ln|Suite 60|Oak Hill|McCulloch County|TX|77838|United States|-6|single family| +35363|AAAAAAAADCKIAAAA|904|Sycamore Center|Street|Suite Y|Plainview|Coffee County|AL|33683|United States|-6|apartment| +35364|AAAAAAAAECKIAAAA|366|Hill Sycamore|Circle|Suite 70|Marion|Panola County|TX|70399|United States|-6|apartment| +35365|AAAAAAAAFCKIAAAA|685|Meadow |Blvd|Suite 350|Greenfield|Fremont County|IA|55038|United States|-6|apartment| +35366|AAAAAAAAGCKIAAAA|461|Church Jackson|Wy|Suite F|Plainview|Mohave County|AZ|83683|United States|-7|apartment| +35367|AAAAAAAAHCKIAAAA|831|Walnut Green|Drive|Suite 390|Cedar Grove|Pamlico County|NC|20411|United States|-5|apartment| +35368|AAAAAAAAICKIAAAA|432|Williams Spring|Wy|Suite V|Clifton|Dorchester County|SC|28014|United States|-5|single family| +35369|AAAAAAAAJCKIAAAA|993|Valley |Avenue|Suite G|New Hope|Pike County|MS|59431|United States|-6|apartment| +35370|AAAAAAAAKCKIAAAA|847|Hill South|Wy|Suite 90|Jenkins|Lake County|CA|97292|United States|-8|condo| +35371|AAAAAAAALCKIAAAA|220|Park |ST|Suite 130|Youngstown|Auglaize County|OH|40001|United States|-5|single family| +35372|AAAAAAAAMCKIAAAA|102|Sycamore |Avenue|Suite H|Friendship|Erath County|TX|74536|United States|-6|condo| +35373|AAAAAAAANCKIAAAA||10th |Circle||Sulphur Springs|Cherokee County|IA||United States||| +35374|AAAAAAAAOCKIAAAA|667|Cedar 4th|Circle|Suite H|Marion|Manatee County|FL|30399|United States|-5|condo| +35375|AAAAAAAAPCKIAAAA|69|15th |Blvd|Suite 320|Riverview|Clinton County|PA|19003|United States|-5|single family| +35376|AAAAAAAAADKIAAAA|866|11th |Road|Suite 450|Oakdale|Franklin County|TN|39584|United States|-5|apartment| +35377|AAAAAAAABDKIAAAA|977|Cedar Dogwood|Court|Suite T|Lakeside|Webster County|KY|49532|United States|-5|apartment| +35378|AAAAAAAACDKIAAAA|393|Park East|Wy|Suite G|Edgewood|Saline County|IL|60069|United States|-6|condo| +35379|AAAAAAAADDKIAAAA|339|Hickory |Street|Suite V|Oak Ridge|Colorado County|TX|78371|United States|-6|condo| +35380|AAAAAAAAEDKIAAAA|979|Franklin 8th|Way|Suite A|Fairfield|Boone County|AR|76192|United States|-6|apartment| +35381|AAAAAAAAFDKIAAAA|291|15th Park|Court|Suite 380|Edgewood|Panola County|MS|50069|United States|-6|single family| +35382|AAAAAAAAGDKIAAAA|173|Third |Blvd|Suite 130|Farmington|Sandoval County|NM|89145|United States|-7|apartment| +35383|AAAAAAAAHDKIAAAA|484|Cedar |Pkwy|Suite 300|Woodland|Amite County|MS|54854|United States|-6|condo| +35384|AAAAAAAAIDKIAAAA|568|Park South|Cir.|Suite V|Waterloo|Rockingham County|NH|02275|United States|-5|apartment| +35385|AAAAAAAAJDKIAAAA|275|Adams |Street|Suite P|Springfield|Lauderdale County|TN|39303|United States|-6|single family| +35386|AAAAAAAAKDKIAAAA|756|Elm 8th|Court|Suite 340|Ashland|Aransas County|TX|74244|United States|-6|apartment| +35387|AAAAAAAALDKIAAAA|509|Hickory |Parkway|Suite X|Woodville|Paulding County|GA|34289|United States|-5|apartment| +35388|AAAAAAAAMDKIAAAA|766|7th Railroad|Cir.|Suite O|Five Forks|Jefferson County|FL|32293|United States|-5|single family| +35389|AAAAAAAANDKIAAAA|818|Elm Hill|Dr.|Suite Y|Franklin|Reno County|KS|69101|United States|-6|single family| +35390|AAAAAAAAODKIAAAA|684|Washington Ash|Avenue|Suite R|Shady Grove|Aransas County|TX|72812|United States|-6|single family| +35391|AAAAAAAAPDKIAAAA|153|Lincoln College|Wy|Suite O|Clifton|Dubois County|IN|48014|United States|-5|condo| +35392|AAAAAAAAAEKIAAAA|905|Green |Blvd|Suite 360|Springtown|Crawford County|MI|49858|United States|-5|condo| +35393|AAAAAAAABEKIAAAA|235|Highland |Circle|Suite C|Hamilton|Scotland County|MO|62808|United States|-6|condo| +35394|AAAAAAAACEKIAAAA||||Suite 180|||ND|51521||-6|| +35395|AAAAAAAADEKIAAAA|418|||Suite 400|||||United States|-5|| +35396|AAAAAAAAEEKIAAAA|190|Sixth |Wy|Suite A|Deerfield|Lafayette County|FL|39840|United States|-5|apartment| +35397|AAAAAAAAFEKIAAAA|494|Dogwood Ash|Wy|Suite 60|Glendale|Wapello County|IA|53951|United States|-6|apartment| +35398|AAAAAAAAGEKIAAAA|82|Maple |Boulevard|Suite 340|Enterprise|Kent County|MD|21757|United States|-5|condo| +35399|AAAAAAAAHEKIAAAA|539|Adams |Ave|Suite P|Woodville|Franklin County|FL|34289|United States|-5|condo| +35400|AAAAAAAAIEKIAAAA|927|14th |Ave|Suite 240|Macedonia|Richmond County|GA|31087|United States|-5|single family| +35401|AAAAAAAAJEKIAAAA|73|5th |Avenue|Suite I|Greenville|Pike County|IL|61387|United States|-6|single family| +35402|AAAAAAAAKEKIAAAA|747|Park Cedar|Way|Suite X|Midway|Gilliam County|OR|91904|United States|-8|apartment| +35403|AAAAAAAALEKIAAAA|890|Elm Mill|Ct.|Suite 260|Greenfield|Fulton County|GA|35038|United States|-5|apartment| +35404|AAAAAAAAMEKIAAAA|870|11th 6th|Way|Suite 30|Midway|Tazewell County|IL|61904|United States|-6|apartment| +35405|AAAAAAAANEKIAAAA|591|Franklin Jefferson|Ln|Suite 260|Plainview|Hidalgo County|TX|73683|United States|-6|apartment| +35406|AAAAAAAAOEKIAAAA|453|5th Poplar|Wy|Suite U|Charleston|Madison County|FL|38018|United States|-5|apartment| +35407|AAAAAAAAPEKIAAAA|41|6th 13th|Ave|Suite 10|Leon|Marion County|OH|40913|United States|-5|single family| +35408|AAAAAAAAAFKIAAAA|49|4th 15th|Ave|Suite 120|Sulphur Springs|Knox County|OH|48354|United States|-5|single family| +35409|AAAAAAAABFKIAAAA|417|Pine |Wy|Suite 150|Cedar Grove|Pinellas County|FL|30411|United States|-5|condo| +35410|AAAAAAAACFKIAAAA|887|Ridge |Avenue|Suite S|Pleasant Grove|Jackson County|WV|24136|United States|-5|apartment| +35411|AAAAAAAADFKIAAAA|65|8th Meadow|Dr.|Suite D|Riceville|Massac County|IL|65867|United States|-6|condo| +35412|AAAAAAAAEFKIAAAA|159|Central |ST|Suite 200|Providence|Coffee County|TN|36614|United States|-5|condo| +35413|AAAAAAAAFFKIAAAA|149|Church |Dr.|Suite V|Springfield|Robertson County|TX|79303|United States|-6|apartment| +35414|AAAAAAAAGFKIAAAA|982|Main River|Pkwy|Suite 270|Centerville|Schuyler County|MO|60059|||| +35415|AAAAAAAAHFKIAAAA|161|10th 4th|Cir.|Suite 340|Plainview|Los Angeles County|CA|93683|United States|-8|condo| +35416|AAAAAAAAIFKIAAAA|700|10th Sixth|Boulevard|Suite 200|Buena Vista|Roscommon County|MI|45752|United States|-5|condo| +35417|AAAAAAAAJFKIAAAA|106|Elm Oak|Street|Suite U|Pine Grove|Johnston County|OK|74593|United States|-6|condo| +35418|AAAAAAAAKFKIAAAA||15th Second|||White Oak|||56668|United States|-6|| +35419|AAAAAAAALFKIAAAA|795|2nd 7th|Circle|Suite G|Enterprise|Scioto County|OH|41757|United States|-5|single family| +35420|AAAAAAAAMFKIAAAA|509|Cherry |Pkwy|Suite 330|Jackson|Lincoln County|MO|69583|United States|-6|apartment| +35421|AAAAAAAANFKIAAAA|763|||Suite 80|Richville|Monroe County||||-6|| +35422|AAAAAAAAOFKIAAAA|831|River |Cir.|Suite A|Summit|Owen County|KY|40499|United States|-5|single family| +35423|AAAAAAAAPFKIAAAA|496|Ash Williams|RD|Suite K|Florence|Jefferson County|KY|43394|United States|-6|single family| +35424|AAAAAAAAAGKIAAAA|261|Lake Sunset|Blvd|Suite I|Greenfield|Clayton County|IA|55038|United States|-6|apartment| +35425|AAAAAAAABGKIAAAA|911|Fourth Lake|Wy|Suite 0|Florence|Yell County|AR|73394|United States|-6|condo| +35426|AAAAAAAACGKIAAAA|945|6th Central|Circle|Suite 160|Ashland|Durham County|NC|24244|United States|-5|single family| +35427|AAAAAAAADGKIAAAA|773|Railroad |Drive|Suite O|Waterloo|Norfolk city|VA|21675|United States|-5|single family| +35428|AAAAAAAAEGKIAAAA||North |Parkway||Woodlawn||OH||United States||| +35429|AAAAAAAAFGKIAAAA|729|4th Fifth|Cir.|Suite 230|Franklin|Franklin County|NC|29101|United States|-5|apartment| +35430|AAAAAAAAGGKIAAAA|687|Pine |Cir.|Suite 390|Oak Grove|Cortland County|NY|18370|United States|-5|apartment| +35431|AAAAAAAAHGKIAAAA|231|Cedar 8th|Circle|Suite K|Bridgeport|Henderson County|KY|45817|United States|-6|apartment| +35432|AAAAAAAAIGKIAAAA|||Street|Suite 90|Florence|Le Sueur County|MN||United States||| +35433|AAAAAAAAJGKIAAAA|331|Walnut 3rd|Ave|Suite 320|Brunswick|Montgomery County|MD|24642|United States|-5|single family| +35434|AAAAAAAAKGKIAAAA|709|Railroad |ST|Suite 350|Riverdale|Volusia County|FL|39391|United States|-5|single family| +35435|AAAAAAAALGKIAAAA|360|Center Pine|Pkwy|Suite A|Plainview|Worcester County|MA|04283|United States|-5|condo| +35436|AAAAAAAAMGKIAAAA||||||San Juan County|CO|89231||-7|condo| +35437|AAAAAAAANGKIAAAA|458|South |Ave|Suite 190|Woodlawn|Cherokee County|OK|74098|United States|-6|single family| +35438|AAAAAAAAOGKIAAAA|397|13th |Street|Suite 110|Fairview|Cooke County|TX|75709|United States|-6|condo| +35439|AAAAAAAAPGKIAAAA|709|Railroad |Boulevard|Suite F|Bridgeport|Dare County|NC|25817|United States|-5|apartment| +35440|AAAAAAAAAHKIAAAA|317|Cedar Forest|Court|Suite K|Sunnyside|Kimball County|NE|61952|United States|-7|apartment| +35441|AAAAAAAABHKIAAAA|110|Mill |Circle|Suite 440|Mount Zion|Coahoma County|MS|58054|United States|-6|condo| +35442|AAAAAAAACHKIAAAA|353|Cherry |Pkwy|Suite 400|Glenwood|Hendry County|FL|33511|United States|-5|condo| +35443|AAAAAAAADHKIAAAA|985|South 9th|Avenue|Suite V|Langdon|Meade County|KS|60852|United States|-6|condo| +35444|AAAAAAAAEHKIAAAA|796|Elm East|Way|Suite 150|Oak Ridge|Bastrop County|TX|78371|United States|-6|apartment| +35445|AAAAAAAAFHKIAAAA|649|College |Circle|Suite 390|Clinton|Fairfield County|OH|48222|United States|-5|single family| +35446|AAAAAAAAGHKIAAAA|857|Mill |Boulevard|Suite 130|Jackson|Bear Lake County|ID|89583|United States|-7|condo| +35447|AAAAAAAAHHKIAAAA|950|Sycamore |Court|Suite 280|Florence|Wallace County|KS|63394|United States|-6|apartment| +35448|AAAAAAAAIHKIAAAA|500|6th |Lane|Suite 440|Edgewood|Scott County|AR|70069|United States|-6|apartment| +35449|AAAAAAAAJHKIAAAA|303|Washington Williams|Dr.|Suite 430|Hopewell|Bartholomew County|IN|40587|United States|-5|single family| +35450|AAAAAAAAKHKIAAAA|483|Spruce |Cir.|Suite 250|Edwards|Greene County|TN|31409|United States|-5|single family| +35451|AAAAAAAALHKIAAAA|96|1st |Lane|Suite 180|Oakwood|Richmond County|NY|10169|United States|-5|single family| +35452|AAAAAAAAMHKIAAAA|203|Maple Smith|Ave|Suite J|Newport|Lee County|SC|21521|United States|-5|apartment| +35453|AAAAAAAANHKIAAAA|52|Seventh 15th|Avenue|Suite 30|Macedonia|Colfax County|NM|81087|United States|-7|apartment| +35454|AAAAAAAAOHKIAAAA|816|Lee |Drive|Suite V|Liberty|Northampton County|VA|23451|United States|-5|apartment| +35455|AAAAAAAAPHKIAAAA|211|Washington Davis|Cir.|Suite 220|Shiloh|Lincoln County|CO|89275|United States|-7|apartment| +35456|AAAAAAAAAIKIAAAA|688|Hickory 6th|RD|Suite P|Three Forks|Treasure County|MT|61338|United States|-7|condo| +35457|AAAAAAAABIKIAAAA|828|North Wilson|Dr.|Suite Y|Green Acres|Douglas County|CO|87683|United States|-7|single family| +35458|AAAAAAAACIKIAAAA|276|Green Highland|Drive|Suite U|Stringtown|Nevada County|AR|70162|United States|-6|condo| +35459|AAAAAAAADIKIAAAA|959|View Lee|Parkway|Suite T|Jenkins|Moody County|SD|57292|United States|-7|condo| +35460|AAAAAAAAEIKIAAAA|128||Way||Woodland|Washington County|||United States||| +35461|AAAAAAAAFIKIAAAA|348|1st Fifth|Ave|Suite J|Marion|Bowman County|ND|50399|United States|-6|apartment| +35462|AAAAAAAAGIKIAAAA|234|South |Way|Suite 110|Springdale|Pulaski County|IL|68883|United States|-6|condo| +35463|AAAAAAAAHIKIAAAA|930|Pine |Way|Suite U|Liberty|Cooper County|MO|63451|United States|-6|single family| +35464|AAAAAAAAIIKIAAAA|900|View |Boulevard|Suite R|Springdale|Monongalia County|WV|28883|United States|-5|condo| +35465|AAAAAAAAJIKIAAAA|33|Railroad Mill|Pkwy|Suite 320|Mount Olive|Barber County|KS|68059|United States|-6|condo| +35466|AAAAAAAAKIKIAAAA|675|11th Jackson|Drive|Suite X|Enterprise|Esmeralda County|NV|81757|United States|-8|single family| +35467|AAAAAAAALIKIAAAA|565|Ridge Hill|Blvd|Suite Y|Woodbine|Albany County|NY|14253|United States|-5|apartment| +35468|AAAAAAAAMIKIAAAA|756|Third Eigth|Blvd||Highland|Charlton County|||United States|-5|| +35469|AAAAAAAANIKIAAAA|381|9th Main|Parkway|Suite 140|Deerfield|White County|IN|49840|United States|-5|single family| +35470|AAAAAAAAOIKIAAAA|988|Meadow |Cir.|Suite 260|Omega|White County|GA|36364|United States|-5|single family| +35471|AAAAAAAAPIKIAAAA|24|Main |Avenue|Suite 370|Oakland|Duval County|TX|79843|United States|-6|condo| +35472|AAAAAAAAAJKIAAAA||Maple ||||Ashland County|OH||United States|-5|| +35473|AAAAAAAABJKIAAAA|918|Mill |RD|Suite 10|Woodlawn|Dade County|GA|34098|United States|-5|condo| +35474|AAAAAAAACJKIAAAA|1000|Elm |Ln|Suite 410|Caledonia|Jackson County|FL|37411|United States|-5|single family| +35475|AAAAAAAADJKIAAAA|129|Eigth Adams|Lane|Suite 180|Midway|Cumberland County||||-6|| +35476|AAAAAAAAEJKIAAAA|49|Hillcrest |RD|Suite 100|Walnut Grove|Starr County|TX|77752|United States|-6|condo| +35477|AAAAAAAAFJKIAAAA|222|4th 2nd|RD|Suite P|Green Acres|Denton County|TX|77683|United States|-6|apartment| +35478|AAAAAAAAGJKIAAAA|388|Williams Hillcrest|Ln|Suite 220|Indian Village|Boyle County|KY|41075|United States|-6|single family| +35479|AAAAAAAAHJKIAAAA|535|Johnson Washington|Pkwy|Suite N|Lincoln|San Saba County|TX|71289|United States|-6|single family| +35480|AAAAAAAAIJKIAAAA|896|Lakeview |Cir.|Suite 440|Springfield|Dade County|MO|69303|United States|-6|condo| +35481|AAAAAAAAJJKIAAAA|871|Main |Court|Suite T|Edgewood|Essex County|VA|20069|United States|-5|condo| +35482|AAAAAAAAKJKIAAAA|858|Fourth |Way|Suite K|Antioch|Orleans Parish|LA|78605|United States|-6|apartment| +35483|AAAAAAAALJKIAAAA|190|Railroad Second|Dr.|Suite J|Stringtown|Morton County|ND|50162|United States|-6|apartment| +35484|AAAAAAAAMJKIAAAA|977|Jefferson |Circle|Suite 190|Sunnyside|Red River Parish|LA|71952|United States|-6|apartment| +35485|AAAAAAAANJKIAAAA|679|Second Ridge|Avenue|Suite 190|New Town|Coconino County|AZ|89634|United States|-7|single family| +35486|AAAAAAAAOJKIAAAA|360|Hillcrest |Cir.|Suite 340|Omega|Clarke County|AL|36364|United States|-6|single family| +35487|AAAAAAAAPJKIAAAA|717|Walnut |Ln|Suite M|Green Acres|Cook County|IL|67683|United States|-6|apartment| +35488|AAAAAAAAAKKIAAAA|135|Woodland Elm|Dr.|Suite H|Belfast|Pope County|MN|50125|United States|-6|apartment| +35489|AAAAAAAABKKIAAAA|796|1st Ninth|ST|Suite 490|Bunker Hill|San Diego County|CA|90150|United States|-8|single family| +35490|AAAAAAAACKKIAAAA|394|Lincoln |Cir.|Suite D|Glenwood|Crook County|WY|83511|United States|-7|condo| +35491|AAAAAAAADKKIAAAA|86|Church Washington|Drive|Suite Y|Newport|Santa Clara County|CA|91521|United States|-8|apartment| +35492|AAAAAAAAEKKIAAAA|858|Chestnut 11th|Ct.|Suite 440|Forest Hills|Hale County|AL|39237|United States|-6|apartment| +35493|AAAAAAAAFKKIAAAA|28|Ridge Fourth|Lane|Suite G|Georgetown|Clare County|MI|47057|United States|-5|single family| +35494|AAAAAAAAGKKIAAAA|979|Lake |Pkwy|Suite D|Unionville|Pointe Coupee Parish|LA|71711|United States|-6|condo| +35495|AAAAAAAAHKKIAAAA|615|Elm Smith|Ct.|Suite V|Hamilton|Irion County|TX|72808|United States|-6|apartment| +35496|AAAAAAAAIKKIAAAA|928|Woodland Jackson|RD|Suite 190|Highland|Brooke County|WV|29454|United States|-5|apartment| +35497|AAAAAAAAJKKIAAAA|217|Fifth Fourth|Lane|Suite Q|Highland Park|DeKalb County|GA|36534|United States|-5|apartment| +35498|AAAAAAAAKKKIAAAA|164|Fifth |Way|Suite S|Forest Hills|San Patricio County|TX|79237|United States|-6|apartment| +35499|AAAAAAAALKKIAAAA|876|Miller |Way|Suite 40|Riverside|Adair County|KY|49231|United States|-6|single family| +35500|AAAAAAAAMKKIAAAA|554|Spring |Drive|Suite O|Owens|Sevier County|UT|82324|United States|-7|apartment| +35501|AAAAAAAANKKIAAAA|143|15th 9th|Avenue|Suite 370|Oakdale|Bastrop County|TX|79584|United States|-6|condo| +35502|AAAAAAAAOKKIAAAA|626|Lee |Wy|Suite 290|Florence|Paulding County|GA|33394|United States|-5|condo| +35503|AAAAAAAAPKKIAAAA|574|Maple College|Pkwy|Suite B|Arlington|Guilford County|NC|26557|United States|-5|condo| +35504|AAAAAAAAALKIAAAA|164|11th Ridge|ST|Suite J|Harmony|Essex County|VA|25804|United States|-5|condo| +35505|AAAAAAAABLKIAAAA|749|15th |Dr.|Suite 100|Sleepy Hollow|Goliad County|TX|73592|United States|-6|apartment| +35506|AAAAAAAACLKIAAAA|331|13th First|Ave|Suite 410|Union Hill|Grant County|AR|77746|United States|-6|single family| +35507|AAAAAAAADLKIAAAA|607|Sunset 2nd|Court|Suite 300|Red Hill|Blount County|AL|34338|United States|-6|condo| +35508|AAAAAAAAELKIAAAA|423|Park College|Lane|Suite 340|Lakeside|Oldham County|TX|79532|United States|-6|apartment| +35509|AAAAAAAAFLKIAAAA|762|Elm |Cir.|Suite 50|Midway|Honolulu County|HI|91904|United States|-10|condo| +35510|AAAAAAAAGLKIAAAA|83|Cedar |Drive|Suite 160|Glenwood|Sibley County|MN|53511|United States|-6|apartment| +35511|AAAAAAAAHLKIAAAA|913|Maple |Court|Suite M|Springdale|Clay County|TX|78883|United States|-6|single family| +35512|AAAAAAAAILKIAAAA|759|||Suite I|Oakland|Lunenburg County||29843|United States|-5|single family| +35513|AAAAAAAAJLKIAAAA|938|Pine 2nd|Ln|Suite T|Highland Park|Fairfield County|SC|26534|United States|-5|single family| +35514|AAAAAAAAKLKIAAAA|870|Franklin |Avenue|Suite C|Spring Hill|Richland County|IL|66787|United States|-6|condo| +35515|AAAAAAAALLKIAAAA|821|3rd Railroad|Ct.|Suite 410|Brownsville|Washburn County|WI|59310|United States|-6|single family| +35516|AAAAAAAAMLKIAAAA|696|Davis Fifth|Wy|Suite 440|Concord|Red Lake County|MN|54107|United States|-6|condo| +35517|AAAAAAAANLKIAAAA|730|Sunset |Dr.|Suite F|Sulphur Springs|Hudson County|NJ|08954|United States|-5|apartment| +35518|AAAAAAAAOLKIAAAA|478|3rd |Avenue|Suite R|Harmony|Wilson County|NC|25804|United States|-5|single family| +35519|AAAAAAAAPLKIAAAA|481|Maple Park|Way|Suite 320|Waterloo|Hocking County|OH|41675|United States|-5|condo| +35520|AAAAAAAAAMKIAAAA|384|Fourth |Street|Suite 380|Providence|Lynchburg city|VA|26614|United States|-5|apartment| +35521|AAAAAAAABMKIAAAA|508|South Elm|Ct.|Suite 370|Valley View|Warren County|IN|45124|United States|-5|single family| +35522|AAAAAAAACMKIAAAA|313|Sunset Highland||Suite 40||Dewey County|||||condo| +35523|AAAAAAAADMKIAAAA|544|East |Street|Suite K|Red Hill|Klickitat County|WA|94338|United States|-8|apartment| +35524|AAAAAAAAEMKIAAAA|57|5th |Ct.|Suite X|Spring Hill|Cherokee County|TX|76787|United States|-6|condo| +35525|AAAAAAAAFMKIAAAA|895|North Ridge|Avenue|Suite A|Harmony|Minidoka County|ID|85804|United States|-7|single family| +35526|AAAAAAAAGMKIAAAA|107|First |Ave|Suite 480|Edgewood|Accomack County|VA|20069|United States|-5|apartment| +35527|AAAAAAAAHMKIAAAA|317|Washington Franklin|Road|Suite 40|Oakland|Sullivan County|NH|09843|United States|-5|apartment| +35528|AAAAAAAAIMKIAAAA|448|1st Hillcrest|Ln|Suite 280|Forest Hills|Dallas County|AR|79237|United States|-6|condo| +35529|AAAAAAAAJMKIAAAA|511|8th |Ln|Suite J|Highland Park|Benson County|ND|56534|United States|-6|apartment| +35530|AAAAAAAAKMKIAAAA|430|Main Chestnut|Pkwy|Suite 210|Sulphur Springs|Rooks County|KS|68354|United States|-6|condo| +35531|AAAAAAAALMKIAAAA|601|Cedar Park|Road|Suite 100|Clinton|Monona County|IA|58222|United States|-6|single family| +35532|AAAAAAAAMMKIAAAA|478|Washington View|Way|Suite W|Farmington|Logan County|KS|69145|United States|-6|single family| +35533|AAAAAAAANMKIAAAA|393|Walnut Smith|Parkway|Suite Y|Oakland|Comal County|TX|79843|United States|-6|condo| +35534|AAAAAAAAOMKIAAAA|642|Seventh First|Ct.|Suite 340|Rosewood|Kearney County|NE|62205|United States|-7|condo| +35535|AAAAAAAAPMKIAAAA|881|5th 4th|Ct.|Suite 480|Concord|Jackson County|KY|44107|United States|-6|apartment| +35536|AAAAAAAAANKIAAAA|747|Church View|Drive|Suite 20|Greenfield|Polk County|AR|75038|United States|-6|condo| +35537|AAAAAAAABNKIAAAA|446|Miller |Ave|Suite U|Woodland|Eau Claire County|WI|54854|United States|-6|condo| +35538|AAAAAAAACNKIAAAA|530|Spring 1st|Ave|Suite V|Arlington|Johnson County|IL|66557|United States|-6|single family| +35539|AAAAAAAADNKIAAAA|503|8th Fourth|Ct.|Suite 340|Sunnyside|Fayette County|GA|31952|United States|-5|apartment| +35540|AAAAAAAAENKIAAAA|794|Smith |Dr.|Suite O|Woodlawn|Pope County|IL|64098|United States|-6|condo| +35541|AAAAAAAAFNKIAAAA|490|10th |Court|Suite L|Newtown|Crow Wing County|MN|51749|United States|-6|condo| +35542|AAAAAAAAGNKIAAAA|618|7th South|Lane|Suite U|New Hope|Cole County|MO|69431|United States|-6|apartment| +35543|AAAAAAAAHNKIAAAA|54|Twelfth |Parkway|Suite 180|Lakeside|Scott County|AR|79532|United States|-6|single family| +35544|AAAAAAAAINKIAAAA|655|North Park|Cir.|Suite Q|Clifton|Hinds County|MS|58014|United States|-6|apartment| +35545|AAAAAAAAJNKIAAAA|317|River Broadway|Wy|Suite 490|Sunnyside|Taylor County|WV|21952|United States|-5|condo| +35546|AAAAAAAAKNKIAAAA|825|Hill Lake|Wy|Suite T|Waterloo|Lewis County|ID|81675|United States|-7|apartment| +35547|AAAAAAAALNKIAAAA|443|Seventh |Drive|Suite 460|Marion|Ouachita County|AR|70399|United States|-6|condo| +35548|AAAAAAAAMNKIAAAA|474|Willow |ST|Suite 50|Marion|Columbia County|WI|50399|United States|-6|apartment| +35549|AAAAAAAANNKIAAAA|758|View Jackson|Blvd|Suite U|Allison|Valley County|MT|64167|United States|-7|condo| +35550|AAAAAAAAONKIAAAA|265|Broadway |Wy|Suite 290|Forest Hills|Scott County|KS|69237|United States|-6|condo| +35551|AAAAAAAAPNKIAAAA|972|Cedar |Pkwy|Suite 260|Tanglewood|Winchester city|VA|28994|United States|-5|apartment| +35552|AAAAAAAAAOKIAAAA|431|9th |Pkwy|Suite 480|Lake Forest|Harnett County|NC|26000|United States|-5|apartment| +35553|AAAAAAAABOKIAAAA|843|Lincoln Hillcrest|Ct.|Suite 330|Mountain View|Chemung County|NY|14466|United States|-5|single family| +35554|AAAAAAAACOKIAAAA|223|Third 8th|ST|Suite 130|Marion|Chester County|TN|30399|United States|-5|single family| +35555|AAAAAAAADOKIAAAA|322|Birch Oak|Blvd|Suite 10|Bunker Hill|Contra Costa County|CA|90150|United States|-8|apartment| +35556|AAAAAAAAEOKIAAAA|921|Fourth College|Avenue|Suite D|Waterloo|Jones County|IA|51675|United States|-6|apartment| +35557|AAAAAAAAFOKIAAAA|785|Woodland Smith|Street|Suite H|Shiloh|Roosevelt County|MT|69275|United States|-7|condo| +35558|AAAAAAAAGOKIAAAA|970|Meadow |Court|Suite H|Pleasant Grove|Jackson County|OH|44136|United States|-5|apartment| +35559|AAAAAAAAHOKIAAAA|861|Church Park|Parkway|Suite 90|Bridgeport|Teller County|CO|85817|United States|-7|apartment| +35560|AAAAAAAAIOKIAAAA|474|Lincoln Washington|Drive|Suite K|Superior|Presque Isle County|MI|42562|United States|-5|apartment| +35561|AAAAAAAAJOKIAAAA|395|Central |ST|Suite 420|Clifton|Christian County|MO|68014|United States|-6|condo| +35562|AAAAAAAAKOKIAAAA|492|12th |Lane|Suite 120|Enterprise|Talladega County|AL|31757|United States|-6|apartment| +35563|AAAAAAAALOKIAAAA|570|9th Ridge|Parkway|Suite D|Lakeview|Polk County|WI|58579|United States|-6|single family| +35564|AAAAAAAAMOKIAAAA|489|Church |Dr.|Suite F|Pleasant Valley|Anson County|NC|22477|United States|-5|condo| +35565|AAAAAAAANOKIAAAA|124|3rd |RD|Suite O|Wildwood|Calhoun County|IA|56871|United States|-6|condo| +35566|AAAAAAAAOOKIAAAA|213|Spring |Ct.|Suite 310|Greenfield|Pike County|AL|35038|United States|-6|single family| +35567|AAAAAAAAPOKIAAAA|843|Chestnut |RD|Suite 30|Clinton|Grayson County|VA|28222|United States|-5|single family| +35568|AAAAAAAAAPKIAAAA||||Suite 140||Big Stone County|MN||||apartment| +35569|AAAAAAAABPKIAAAA|195|2nd Valley|Street|Suite E|Five Forks|Appanoose County|IA|52293|United States|-6|apartment| +35570|AAAAAAAACPKIAAAA|71|Church |Wy|Suite Y|Sulphur Springs|Meade County|SD|58354|United States|-7|single family| +35571|AAAAAAAADPKIAAAA|785|4th |Road|Suite 440|New Hope|Lawrence County|AL|39431|United States|-6|condo| +35572|AAAAAAAAEPKIAAAA|469|Sunset |Avenue|Suite 200|Lakeview|Clay County|KY|48579|United States|-6|apartment| +35573|AAAAAAAAFPKIAAAA|646|Johnson Cedar|Pkwy|Suite 380|Deerfield|Whiteside County|IL|69840|United States|-6|single family| +35574|AAAAAAAAGPKIAAAA|239|Lincoln |Street|Suite T|Woodland|Anderson County|TN|34854|United States|-5|apartment| +35575|AAAAAAAAHPKIAAAA|878|Railroad 4th|Circle|Suite 210|Summit|Cameron Parish|LA|70499|United States|-6|apartment| +35576|AAAAAAAAIPKIAAAA|399|Washington |Drive|Suite 240|White Oak|Vance County|NC|26668|United States|-5|single family| +35577|AAAAAAAAJPKIAAAA|309|6th 14th|Ave|Suite 330|Mountain View|Prince William County|VA|24466|United States|-5|condo| +35578|AAAAAAAAKPKIAAAA|349|Jackson |Way|Suite Q|Greenville|Prince George County|MD|21387|United States|-5|condo| +35579|AAAAAAAALPKIAAAA|868|Pine |Cir.|Suite 310|Five Forks|Nash County|NC|22293|United States|-5|condo| +35580|AAAAAAAAMPKIAAAA|119|Washington |Avenue|Suite 210|Highland Park|Franklin County|OH|46534|United States|-5|single family| +35581|AAAAAAAANPKIAAAA|85|||Suite 110||Jackson County|||||| +35582|AAAAAAAAOPKIAAAA|335|7th |RD|Suite F|Buena Vista|Polk County|OR|95752|United States|-8|apartment| +35583|AAAAAAAAPPKIAAAA|595|Poplar |Lane|Suite 80|Marion|Hall County|GA|30399|United States|-5|condo| +35584|AAAAAAAAAALIAAAA|883|||Suite 130|Lebanon|||72898|||| +35585|AAAAAAAABALIAAAA|666|Pine Adams|Parkway|Suite S|White Oak|Lynn County|TX|76668|United States|-6|condo| +35586|AAAAAAAACALIAAAA|160|College Church|Boulevard|Suite V|Red Hill|Perquimans County|NC|24338|United States|-5|condo| +35587|AAAAAAAADALIAAAA|180|Lee Railroad|Wy|Suite 0|Edgewood|Rio Blanco County|CO|80069|United States|-7|single family| +35588|AAAAAAAAEALIAAAA|690|Pine |Ave|Suite 20|Oakwood|Polk County|WI|50169|United States|-6|apartment| +35589|AAAAAAAAFALIAAAA|761|Central 1st|Court|Suite A|Oak Grove|Estill County|KY|48370|United States|-6|apartment| +35590|AAAAAAAAGALIAAAA|144|First Ridge|Street|Suite 90|Mount Pleasant|Montmorency County|MI|41933|United States|-5|single family| +35591|AAAAAAAAHALIAAAA|896|View |Road|Suite N|Flint|Attala County|MS|58909|United States|-6|apartment| +35592|AAAAAAAAIALIAAAA|152|13th |RD|Suite I|Franklin|Giles County|TN|39101|United States|-5|single family| +35593|AAAAAAAAJALIAAAA|269|Railroad |Street|Suite J|Edgewood|Campbell County|KY|40069|United States|-6|condo| +35594|AAAAAAAAKALIAAAA|733|Washington |Lane|Suite M|Wilson|Clay County|NE|66971|United States|-6|single family| +35595|AAAAAAAALALIAAAA|465|Ridge |Lane|Suite V|Farmington|Meagher County|MT|69145|United States|-7|single family| +35596|AAAAAAAAMALIAAAA|47|Sycamore 2nd|Road|Suite H|Kingston|Todd County|SD|54975|United States|-7|single family| +35597|AAAAAAAANALIAAAA|256|Ninth Second|Blvd|Suite 290|Five Forks|Haines Borough|AK|92293|United States|-9|single family| +35598|AAAAAAAAOALIAAAA|700|8th Wilson|Ct.|Suite K|Crossroads|District of Columbia|DC|20534|United States|-5|apartment| +35599|AAAAAAAAPALIAAAA|361|Davis |ST|Suite C|Wilson|McLean County|ND|56971|United States|-6|condo| +35600|AAAAAAAAABLIAAAA|685|14th Maple|ST|Suite 250|Oak Ridge|Sawyer County|WI|58371|United States|-6|apartment| +35601|AAAAAAAABBLIAAAA|187|7th |Parkway|Suite V|Lebanon|Davidson County|NC|22898|United States|-5|condo| +35602|AAAAAAAACBLIAAAA|728|Lakeview |Street|Suite E|Woodland|Greene County|MS|54854|United States|-6|condo| +35603|AAAAAAAADBLIAAAA|947|15th Thirteenth|Boulevard|Suite 340|Kelly|Harford County|MD|22738|United States|-5|apartment| +35604|AAAAAAAAEBLIAAAA|165|Tenth 8th|Pkwy|Suite I|Bethel|Prentiss County|MS|55281|United States|-6|condo| +35605|AAAAAAAAFBLIAAAA|73|Lee Franklin|Street|Suite T|Cedar Grove|Sullivan County|PA|10411|United States|-5|single family| +35606|AAAAAAAAGBLIAAAA|814|Fourth Third|Street|Suite C|Brookville|Liberty County|MT|63524|United States|-7|apartment| +35607|AAAAAAAAHBLIAAAA|785|Davis Locust|Ln|Suite V|Midway|Lowndes County|MS|51904|United States|-6|single family| +35608|AAAAAAAAIBLIAAAA|443|Jefferson |Street|Suite 340|Bethel|Park County|WY|85281|United States|-7|apartment| +35609|AAAAAAAAJBLIAAAA|229|Lake |Road|Suite 440|Bridgeport|Miller County|MO|65817|United States|-6|apartment| +35610|AAAAAAAAKBLIAAAA|893|Cherry Lincoln|Ave|Suite P|Morris|Peoria County|IL|66696|United States|-6|single family| +35611|AAAAAAAALBLIAAAA|668|Lincoln |Pkwy|Suite 40|Hopewell|Lewis County|ID|80587|United States|-7|single family| +35612|AAAAAAAAMBLIAAAA||||Suite 190|Washington Heights||GA||United States|-5|apartment| +35613|AAAAAAAANBLIAAAA|532|North 3rd|RD|Suite Q|Springdale|Floyd County|VA|28883|United States|-5|apartment| +35614|AAAAAAAAOBLIAAAA|523|Ridge Park|RD|Suite E|Clifton|Catoosa County|GA|38014|United States|-5|condo| +35615|AAAAAAAAPBLIAAAA|604|Johnson 7th|Boulevard|Suite 110|Bunker Hill|Menard County|TX|70150|United States|-6|single family| +35616|AAAAAAAAACLIAAAA|692|Williams |RD|Suite 330|Jamestown|Fayette County|PA|16867|United States|-5|condo| +35617|AAAAAAAABCLIAAAA|731|Broadway West|Ln|Suite 380|Oakland|Missoula County|MT|69843|United States|-7|single family| +35618|AAAAAAAACCLIAAAA|460|Main |Ave|Suite J|Fremont|Idaho County|ID|81851|United States|-7|single family| +35619|AAAAAAAADCLIAAAA|136|River Park|Street|Suite P|Whitesville|Lee County|IA|55903|United States|-6|condo| +35620|AAAAAAAAECLIAAAA|236|Washington |Parkway|Suite 210|Philadelphia|Dawson County|NE|65591|United States|-6|condo| +35621|AAAAAAAAFCLIAAAA|738|Main 3rd|Lane|Suite 290|Shady Grove|Ida County|IA|52812|United States|-6|condo| +35622|AAAAAAAAGCLIAAAA|168|4th 1st|Wy|Suite 160|Fairview|Lyon County|IA|55709|United States|-6|apartment| +35623|AAAAAAAAHCLIAAAA|605|4th |Blvd|Suite Y|Mountain View|Jackson County|MN|54466|United States|-6|single family| +35624|AAAAAAAAICLIAAAA|936|Park |RD|Suite A|Pleasant Grove|Treasure County|MT|64136|United States|-7|apartment| +35625|AAAAAAAAJCLIAAAA||Tenth |ST|Suite C|Stringtown|Lucas County|IA|50162|||| +35626|AAAAAAAAKCLIAAAA|883|10th Spring|Blvd|Suite T|Friendship|Washington County|MN|54536|United States|-6|single family| +35627|AAAAAAAALCLIAAAA|845|View Chestnut|Wy|Suite 190|Ashland|Haskell County|OK|74244|United States|-6|condo| +35628|AAAAAAAAMCLIAAAA|436|Maple |Lane|Suite 400|Shady Grove|Bonneville County|ID|82812|United States|-7|apartment| +35629|AAAAAAAANCLIAAAA|494|Meadow |Wy|Suite 110|Florence|Washington County|VT|03994|United States|-5|single family| +35630|AAAAAAAAOCLIAAAA|631|Woodland Lake|Parkway|Suite O|Franklin|Carson County|TX|79101|United States|-6|single family| +35631|AAAAAAAAPCLIAAAA|853|Pine Lincoln|RD|Suite V|Brownsville|Butler County|IA|59310|United States|-6|condo| +35632|AAAAAAAAADLIAAAA|962|Hillcrest |RD|Suite 240|Fairfield|Ouachita Parish|LA|76192|United States|-6|apartment| +35633|AAAAAAAABDLIAAAA|929|Highland 10th|Court|Suite A|Bunker Hill|Polk County|TX|70150|United States|-6|condo| +35634|AAAAAAAACDLIAAAA|606|Seventh |Parkway|Suite 450|Salem|Nottoway County|VA|28048|United States|-5|condo| +35635|AAAAAAAADDLIAAAA|721|2nd 2nd|RD|Suite B|Woodlawn|Volusia County|FL|34098|United States|-5|single family| +35636|AAAAAAAAEDLIAAAA|246|14th 6th|Street|Suite 20|Ashland|Putnam County|IN|44244|United States|-5|apartment| +35637|AAAAAAAAFDLIAAAA|668|12th 7th|Boulevard|Suite J|Newtown|Kenedy County|TX|71749|United States|-6|apartment| +35638|AAAAAAAAGDLIAAAA|723|12th Maple|ST|Suite 490|Morgan|Clark County|NV|86397|United States|-8|apartment| +35639|AAAAAAAAHDLIAAAA|523|Lakeview Laurel|Street|Suite 370|White Oak|Luna County|NM|86668|United States|-7|apartment| +35640|AAAAAAAAIDLIAAAA|976|Lakeview |Drive|Suite V|Crossroads|Grand Forks County|ND|50534|United States|-6|condo| +35641|AAAAAAAAJDLIAAAA|848|Third |Lane|Suite Y|Walnut Grove|Lanier County|GA|37752|United States|-5|apartment| +35642|AAAAAAAAKDLIAAAA|319|View |Lane|Suite U|Pleasant Hill|Scotland County|NC|23604|United States|-5|condo| +35643|AAAAAAAALDLIAAAA|562|Wilson Central|Blvd|Suite S|Kingston|Alcorn County|MS|54975|United States|-6|condo| +35644|AAAAAAAAMDLIAAAA|187|Hillcrest |RD|Suite F|Jackson|Terry County|TX|79583|United States|-6|single family| +35645|AAAAAAAANDLIAAAA|799|4th |Lane|Suite 420|Belmont|Thurston County|NE|60191|United States|-7|single family| +35646|AAAAAAAAODLIAAAA|367|Sycamore Miller|Dr.|Suite 490|Mount Vernon|Bradley County|TN|38482|United States|-5|single family| +35647|AAAAAAAAPDLIAAAA|487|Davis Green|Way|Suite 190|Springdale|Braxton County|WV|28883|United States|-5|condo| +35648|AAAAAAAAAELIAAAA|102|Oak Main|Avenue|Suite G|Greenfield|Meeker County|MN|55038|United States|-6|condo| +35649|AAAAAAAABELIAAAA|957|1st Elm|Wy|Suite X|Lebanon|Randall County|TX|72898|United States|-6|single family| +35650|AAAAAAAACELIAAAA|879|Pine 9th|Parkway|Suite 340|Smith|Cortland County|NY|17317|United States|-5|condo| +35651|AAAAAAAADELIAAAA|799|5th |RD|Suite E|Maywood|Dickinson County|IA|55681|United States|-6|apartment| +35652|AAAAAAAAEELIAAAA|||Cir.||Florence|||33394||-6|| +35653|AAAAAAAAFELIAAAA|176|Third |Way|Suite 270|Crossroads|Leon County|TX|70534|United States|-6|apartment| +35654|AAAAAAAAGELIAAAA|708|5th Fifth|Lane|Suite M|Lakeview|Dallas County|AL|38579|United States|-6|apartment| +35655|AAAAAAAAHELIAAAA|552|River |Drive|Suite B|Bunker Hill|Henrico County|VA|20150|United States|-5|single family| +35656|AAAAAAAAIELIAAAA||Washington |||Springfield|Montgomery County|IA|59303|United States|-6|| +35657|AAAAAAAAJELIAAAA|75|Poplar |Drive|Suite X|Brownsville|Atkinson County|GA|39310|United States|-5|condo| +35658|AAAAAAAAKELIAAAA|808|First |Court|Suite C|Jackson|Palo Pinto County|TX|79583|United States|-6|apartment| +35659|AAAAAAAALELIAAAA|551|13th Maple|Lane|Suite 80|Greenville|Potter County|PA|11387|United States|-5|apartment| +35660|AAAAAAAAMELIAAAA|307|2nd |Court|Suite 330|Glendale|Chicot County|AR|73951|United States|-6|single family| +35661|AAAAAAAANELIAAAA|951|Twelfth ||Suite E|||||United States|-6|condo| +35662|AAAAAAAAOELIAAAA|890|Jefferson |Ct.|Suite 260|Green Acres|Conejos County|CO|87683|United States|-7|apartment| +35663|AAAAAAAAPELIAAAA|323|Adams 2nd|Street|Suite 130|Woodlawn|Sherman County|NE|64098|United States|-7|condo| +35664|AAAAAAAAAFLIAAAA|369|Lincoln |RD|Suite 410|Dallas|Escambia County|AL|33628|United States|-6|condo| +35665|AAAAAAAABFLIAAAA|467|Oak |Cir.|Suite 340|Appleton|Swisher County|TX|74240|United States|-6|condo| +35666|AAAAAAAACFLIAAAA|900|Lincoln Ridge|Cir.|Suite 290|Jamestown|Kenai Peninsula Borough|AK|96867|United States|-9|apartment| +35667|AAAAAAAADFLIAAAA|256|6th 3rd|Ct.|Suite 400|Bunker Hill|Woodruff County|AR|70150|United States|-6|apartment| +35668|AAAAAAAAEFLIAAAA|660|Spring Park|RD|Suite 160|Saint Clair|Rabun County|GA|35294|United States|-5|condo| +35669|AAAAAAAAFFLIAAAA|124|Spruce |ST|Suite 220|Sunnyside|Anderson County|TX|71952|United States|-6|apartment| +35670|AAAAAAAAGFLIAAAA|600|Sycamore North|Avenue|Suite O|Woodville|Choctaw County|AL|34289|United States|-6|condo| +35671|AAAAAAAAHFLIAAAA|283|5th Highland|Ln|Suite N|Marion|Elbert County|GA|30399|United States|-5|single family| +35672|AAAAAAAAIFLIAAAA|42|Central View|Ln|Suite 300|Forest Hills|Martin County|FL|39237|United States|-5|apartment| +35673|AAAAAAAAJFLIAAAA|325|Oak 5th|Boulevard|Suite 480|Antioch|Chickasaw County|MS|58605|United States|-6|single family| +35674|AAAAAAAAKFLIAAAA|841|Washington |ST|Suite C|Lake Forest|Sheridan County|KS|66000|United States|-6|single family| +35675|AAAAAAAALFLIAAAA|898|Poplar |Way|Suite 240|Oak Grove|Clark County|AR|78370|United States|-6|single family| +35676|AAAAAAAAMFLIAAAA|116|9th |Avenue|Suite R|Lincoln|Kennebec County|ME|01889|United States|-5|apartment| +35677|AAAAAAAANFLIAAAA|600|Lincoln |Way|Suite A|Plainview|Lycoming County|PA|13683|United States|-5|single family| +35678|AAAAAAAAOFLIAAAA|19|Valley |RD|Suite 190|Woodlawn|Jefferson County|PA|14098|United States|-5|single family| +35679|AAAAAAAAPFLIAAAA|76|Lake |ST|Suite 340|Newport|Carver County|MN|51521|United States|-6|single family| +35680|AAAAAAAAAGLIAAAA|286|Center |Way|Suite P|Riverdale|Harrison County|MS|59391|United States|-6|apartment| +35681|AAAAAAAABGLIAAAA|939|Second 10th|Road|Suite 150|Edgewood|Alger County|MI|40069|United States|-5|single family| +35682|AAAAAAAACGLIAAAA|802|Ridge |Cir.|Suite R|Farmington|Butler County|KS|69145|United States|-6|condo| +35683|AAAAAAAADGLIAAAA|172|Wilson Lincoln|Drive|Suite T|Highland Park|Kenai Peninsula Borough|AK|96534|United States|-9|apartment| +35684|AAAAAAAAEGLIAAAA|597|Meadow |Lane|Suite B|Union Hill|Furnas County|NE|67746|United States|-6|single family| +35685|AAAAAAAAFGLIAAAA|750|8th |Road|Suite K|Riverdale|Vermilion County|IL|69391|United States|-6|condo| +35686|AAAAAAAAGGLIAAAA|325|Walnut Dogwood|Ln|Suite 290|Northwood|Finney County|KS|64104|United States|-6|condo| +35687|AAAAAAAAHGLIAAAA|414|Seventh |Cir.|Suite U|Highland Park|Boone County|MO|66534|United States|-6|apartment| +35688|AAAAAAAAIGLIAAAA|297|Willow |Lane|Suite 450|Forest Hills|Spalding County|GA|39237|United States|-5|apartment| +35689|AAAAAAAAJGLIAAAA|850|Hill East|RD|Suite 40|Lakeside|Athens County|OH|49532|United States|-5|single family| +35690|AAAAAAAAKGLIAAAA|554|Poplar Main|Parkway|Suite 90|Woodland|Adams County|IN|44854|United States|-5|apartment| +35691|AAAAAAAALGLIAAAA|778|Elm |RD|Suite 60|Shelby|Spencer County|KY|46575|United States|-5|condo| +35692|AAAAAAAAMGLIAAAA|605|7th Lee|Wy|Suite X|Clinton|La Salle County|IL|68222|United States|-6|single family| +35693|AAAAAAAANGLIAAAA|731|2nd 11th|Ln|Suite P|Woodlawn|Miller County|MO|64098|United States|-6|condo| +35694|AAAAAAAAOGLIAAAA|873|15th |Circle|Suite 360|Saint Clair|Kitsap County|WA|95294|United States|-8|single family| +35695|AAAAAAAAPGLIAAAA|911|Ridge Highland|Avenue|Suite 440|Lakeside|Meigs County|OH|49532|United States|-5|condo| +35696|AAAAAAAAAHLIAAAA|528|Fourth |Cir.|Suite T|Clifton|McPherson County|NE|68014|United States|-7|apartment| +35697|AAAAAAAABHLIAAAA|629|Poplar |Circle|Suite P|Macedonia|Gloucester County|NJ|01687|United States|-5|single family| +35698|AAAAAAAACHLIAAAA|510|Spring Woodland|Ln|Suite C|Maple Grove|Elkhart County|IN|48252|United States|-5|apartment| +35699|AAAAAAAADHLIAAAA|37|8th |Blvd|Suite 180|Harmony|Trego County|KS|65804|United States|-6|apartment| +35700|AAAAAAAAEHLIAAAA|295|Sunset |Circle|Suite 330|Union Hill|Haines Borough|AK|97746|United States|-9|condo| +35701|AAAAAAAAFHLIAAAA|247|Spring Lee|Pkwy|Suite T|White Oak|Roane County|WV|26668|United States|-5|condo| +35702|AAAAAAAAGHLIAAAA|896|Main |ST|Suite 170|Midway|Andrews County|TX|71904|United States|-6|condo| +35703|AAAAAAAAHHLIAAAA|521|Dogwood |Court|Suite W|Wyoming|Scott County|VA|20216|United States|-5|condo| +35704|AAAAAAAAIHLIAAAA|117|Laurel Valley|Circle|Suite A|Marion|Monona County|IA|50399|United States|-6|condo| +35705|AAAAAAAAJHLIAAAA|270|6th 3rd|Lane|Suite B|Bethel|Washington County|OK|75281|United States|-6|apartment| +35706|AAAAAAAAKHLIAAAA|377|1st Johnson|Way|Suite 480|Liberty|Marshall County|IA|53451|United States|-6|apartment| +35707|AAAAAAAALHLIAAAA|727|Seventh Forest|Ct.|Suite 470|Greenfield|Acadia Parish|LA|75038|United States|-6|apartment| +35708|AAAAAAAAMHLIAAAA|998|6th |Drive|Suite U|Highland|Sandoval County|NM|89454|United States|-7|condo| +35709|AAAAAAAANHLIAAAA|258|Oak |Court|Suite 380|New Hope|Taylor County|WI|59431|United States|-6|apartment| +35710|AAAAAAAAOHLIAAAA|875|Cedar |ST|Suite J|Buena Vista|Cedar County|IA|55752|United States|-6|condo| +35711|AAAAAAAAPHLIAAAA|523|6th |Boulevard|Suite M|Clinton|Surry County|VA|28222|United States|-5|condo| +35712|AAAAAAAAAILIAAAA|83|Ash |Way|Suite 270|Enterprise|Bourbon County|KY|41757|United States|-6|condo| +35713|AAAAAAAABILIAAAA|954|River |Court|Suite 470|Sulphur Springs|Clay County|TX|78354|United States|-6|apartment| +35714|AAAAAAAACILIAAAA|183|Mill |Cir.|Suite R|Sunnyside|Warren County|TN|31952|United States|-6|single family| +35715|AAAAAAAADILIAAAA|575|2nd |ST|Suite N|Leland|Nicholas County|KY|49452|United States|-5|apartment| +35716|AAAAAAAAEILIAAAA|776|1st |Ave|Suite 230|Mount Pleasant|Putnam County|GA|31933|United States|-5|condo| +35717|AAAAAAAAFILIAAAA|418|Cherry |Avenue|Suite 470|Hillcrest|Macoupin County|IL|63003|United States|-6|apartment| +35718|AAAAAAAAGILIAAAA|367|Spring |RD|Suite G|Oakland|McPherson County|KS|69843|United States|-6|single family| +35719|AAAAAAAAHILIAAAA|91|Third 10th|Cir.|Suite 290|Bennett|Columbia County|WI|51715|United States|-6|apartment| +35720|AAAAAAAAIILIAAAA|531|Main |Avenue|Suite 270|Five Points|Cowley County|KS|66098|United States|-6|single family| +35721|AAAAAAAAJILIAAAA|157|Cherry Miller|Parkway|Suite A|Woodville|Rich County|UT|84289|United States|-7|condo| +35722|AAAAAAAAKILIAAAA|364|Sunset |Pkwy|Suite K|Red Hill|McKean County|PA|14338|United States|-5|condo| +35723|AAAAAAAALILIAAAA|719|Jefferson |Street|Suite 290|Summit|Morgan County|IL|60499|United States|-6|condo| +35724|AAAAAAAAMILIAAAA|7|Dogwood |Avenue|Suite E|Valley View|Pottawatomie County|OK|75124|United States|-6|apartment| +35725|AAAAAAAANILIAAAA|806|Hickory |Pkwy|Suite 210|Centerville|McPherson County|KS|60059|United States|-6|condo| +35726|AAAAAAAAOILIAAAA|717|Spring Oak|Pkwy|Suite Q|Glendale|Atkinson County|GA|33951|United States|-5|condo| +35727|AAAAAAAAPILIAAAA|576|1st Park|Drive|Suite 100|Wildwood|Hampden County|MA|07471|United States|-5|apartment| +35728|AAAAAAAAAJLIAAAA|67|Sunset 1st|Cir.|Suite I|Greenwood|Cheatham County|TN|38828|United States|-5|single family| +35729|AAAAAAAABJLIAAAA|95|Fourth |ST|Suite 130|Pleasant Grove|Douglas County|WA|94136|United States|-8|condo| +35730|AAAAAAAACJLIAAAA|490|Hillcrest Central|Cir.|Suite 180|Waterloo|Turner County|SD|51675|United States|-7|single family| +35731|AAAAAAAADJLIAAAA|899|Center |Avenue|Suite 60|Springfield|Wake County|NC|29303|United States|-5|apartment| +35732|AAAAAAAAEJLIAAAA|84|3rd |Drive|Suite T|Jackson|Crenshaw County|AL|39583|United States|-6|apartment| +35733|AAAAAAAAFJLIAAAA|80|Walnut Davis|Court|Suite 240|Centerville|Washington County|MN|50059|United States|-6|condo| +35734|AAAAAAAAGJLIAAAA|322|Chestnut |Lane|Suite N|Longwood|Dauphin County|PA|17021|United States|-5|apartment| +35735|AAAAAAAAHJLIAAAA|192|12th |Circle|Suite G|Wildwood|Wayne County|NC|26871|United States|-5|single family| +35736|AAAAAAAAIJLIAAAA|747|Lake Cedar|Lane|Suite H|Mount Olive|Boyle County|KY|48059|United States|-6|condo| +35737|AAAAAAAAJJLIAAAA|518|Cedar Pine|Avenue|Suite 390|Peru|Grant County|AR|70302|United States|-6|single family| +35738|AAAAAAAAKJLIAAAA|249|Main |Ave|Suite 150|Fairview|Paulding County|OH|45709|United States|-5|single family| +35739|AAAAAAAALJLIAAAA|336|Spring |Street|Suite 340|Farmington|Jeff Davis County|TX|79145|United States|-6|apartment| +35740|AAAAAAAAMJLIAAAA|299|8th |Dr.|Suite 150|Greenwood|Roger Mills County|OK|78828|United States|-6|single family| +35741|AAAAAAAANJLIAAAA|210|Spring |Pkwy|Suite H|Pleasant Hill|Hennepin County|MN|53604|United States|-6|single family| +35742|AAAAAAAAOJLIAAAA|761|Main Birch|Boulevard|Suite 10|Liberty|Fremont County|WY|83451|United States|-7|apartment| +35743|AAAAAAAAPJLIAAAA|577|Spring |Drive|Suite T|Clinton|Onondaga County|NY|18222|United States|-5|condo| +35744|AAAAAAAAAKLIAAAA|458|5th |Boulevard|Suite K|Oak Grove|Elkhart County|IN|48370|United States|-5|single family| +35745|AAAAAAAABKLIAAAA|639|Pine 10th|Circle|Suite R|Green Acres|Scurry County|TX|77683|United States|-6|single family| +35746|AAAAAAAACKLIAAAA|128|1st Walnut|Blvd|Suite 290|Lincoln|Houston County|MN|51289|United States|-6|single family| +35747|AAAAAAAADKLIAAAA|612|Davis 3rd|Parkway|Suite 80|Jamestown|Caledonia County|VT|07467|United States|-5|condo| +35748|AAAAAAAAEKLIAAAA|316|Sixth |Road|Suite 140|Midway|Lexington city|VA|21904|United States|-5|condo| +35749|AAAAAAAAFKLIAAAA|818|College |Circle|Suite 400|Lakeview|Eureka County|NV|88579|United States|-8|condo| +35750|AAAAAAAAGKLIAAAA||||||Cedar County|MO||United States|-6|apartment| +35751|AAAAAAAAHKLIAAAA|183|Chestnut Railroad|Drive|Suite 80|Mount Pleasant|Shiawassee County|MI|41933|United States|-5|single family| +35752|AAAAAAAAIKLIAAAA|503|7th Highland|Blvd|Suite W|Jamestown|Monongalia County|WV|26867|United States|-5|single family| +35753|AAAAAAAAJKLIAAAA|630|14th South|Court|Suite L|Crossroads|Owen County|IN|40534|United States|-5|apartment| +35754|AAAAAAAAKKLIAAAA|11|Center Lake|Ave|Suite 190|Buckingham|Durham County|NC|24092|United States|-5|single family| +35755|AAAAAAAALKLIAAAA|817|Locust |Parkway|Suite 160|Spring Hill|Clarendon County|SC|26787|United States|-5|single family| +35756|AAAAAAAAMKLIAAAA|912|Main Lincoln|Drive|Suite N|Oak Ridge|Randolph County|IL|68371|United States|-6|apartment| +35757|AAAAAAAANKLIAAAA|810|North |ST|Suite R|Plainview|Burke County|NC|23683|United States|-5|condo| +35758|AAAAAAAAOKLIAAAA|329|Hill 2nd|Ln|Suite P|Lenox|Madison County|AR|71143|United States|-6|apartment| +35759|AAAAAAAAPKLIAAAA|228|Ridge Forest|Way|Suite S|Greenville|Yuma County|AZ|81387|United States|-7|single family| +35760|AAAAAAAAALLIAAAA|459|4th |Dr.|Suite 370|Pleasant Hill|Greenwood County|KS|63604|United States|-6|condo| +35761|AAAAAAAABLLIAAAA|718|8th |Way|Suite V|Woodville|Clark County|WA|94289|United States|-8|apartment| +35762|AAAAAAAACLLIAAAA|42|Forest North|Circle|Suite R|Granite|Nemaha County|NE|66284|United States|-7|apartment| +35763|AAAAAAAADLLIAAAA|48|Poplar |Court|Suite U|Avoca|Mills County|IA|50540|United States|-6|apartment| +35764|AAAAAAAAELLIAAAA|1000|Johnson 10th|Circle|Suite 70|Cedar Grove|Crawford County|IL|60411|United States|-6|condo| +35765|AAAAAAAAFLLIAAAA|377|Williams |Avenue|Suite F|Spring Hill|Effingham County|GA|36787|United States|-5|condo| +35766|AAAAAAAAGLLIAAAA|975|Mill Pine|Road|Suite B|Shady Grove|Polk County|NE|62812|United States|-7|condo| +35767|AAAAAAAAHLLIAAAA|846|Ash |Road|Suite 250|Woodland|Boyd County|NE|64854|United States|-6|condo| +35768|AAAAAAAAILLIAAAA|108|Johnson Park|Avenue|Suite 10|Springdale|Union County|TN|38883|United States|-6|condo| +35769|AAAAAAAAJLLIAAAA||Spring 7th|||Centerville|Lincoln County||70059|||apartment| +35770|AAAAAAAAKLLIAAAA|250|1st |Cir.|Suite 320|Valley View|McHenry County|IL|65124|United States|-6|apartment| +35771|AAAAAAAALLLIAAAA|909|Broadway |Road|Suite W|Lakeside|Wyoming County|PA|19532|United States|-5|apartment| +35772|AAAAAAAAMLLIAAAA|41|North |RD|Suite Q|Mountain View|Deuel County|NE|64466|United States|-6|apartment| +35773|AAAAAAAANLLIAAAA|339|First |Lane|Suite 170|Wilson|Webster County|WV|26971|United States|-5|single family| +35774|AAAAAAAAOLLIAAAA|505|Hill |Wy|Suite 220|Shady Grove|Sherman County|NE|62812|United States|-7|apartment| +35775|AAAAAAAAPLLIAAAA|207|3rd |ST|Suite 370|Walnut Grove|Perry County|IL|67752|United States|-6|single family| +35776|AAAAAAAAAMLIAAAA|180|Lincoln Woodland|Circle|Suite W|Hamilton|Sterling County|TX|72808|United States|-6|apartment| +35777|AAAAAAAABMLIAAAA|157|Seventh 2nd|Cir.|Suite 260|Harmony|Pearl River County|MS|55804|United States|-6|apartment| +35778|AAAAAAAACMLIAAAA|714|Main |Ave|Suite O|Fairview|Kenosha County|WI|55709|United States|-6|condo| +35779|AAAAAAAADMLIAAAA|624|Walnut |Ct.|Suite G|Leon|Blaine County|ID|80913|United States|-7|single family| +35780|AAAAAAAAEMLIAAAA|573|5th |Lane|Suite M|Hillcrest|Kent County|MI|43003|United States|-5|single family| +35781|AAAAAAAAFMLIAAAA|778|Wilson |Cir.|Suite 190|Oakland|Kauai County|HI|99843|United States|-10|condo| +35782|AAAAAAAAGMLIAAAA|795|Lake Main|Avenue|Suite 20|Mountain View|Mecklenburg County|VA|24466|United States|-5|single family| +35783|AAAAAAAAHMLIAAAA|20|Main |Ave|Suite 230|Five Forks|Breathitt County|KY|42293|United States|-6|condo| +35784|AAAAAAAAIMLIAAAA|674|Chestnut |Circle|Suite 180|Shiloh|Finney County|KS|69275|United States|-6|condo| +35785|AAAAAAAAJMLIAAAA|571|9th 6th|Cir.|Suite 10|Summit|Polk County|MO|60499|United States|-6|single family| +35786|AAAAAAAAKMLIAAAA||Chestnut ||Suite 50|Pleasant Valley||TX|72477|||apartment| +35787|AAAAAAAALMLIAAAA|725|5th |Cir.|Suite 140|Rankin|Wabash County|IL|62621|United States|-6|single family| +35788|AAAAAAAAMMLIAAAA|626|Williams |Ct.|Suite 50|Newport|Atkinson County|GA|31521|United States|-5|condo| +35789|AAAAAAAANMLIAAAA|288|Cherry Main|Road|Suite D|Greenfield|Nome Census Area|AK|95038|United States|-9|apartment| +35790|AAAAAAAAOMLIAAAA|75|7th |Blvd|Suite 480|Sunnyside|New Castle County|DE|11952|United States|-5|condo| +35791|AAAAAAAAPMLIAAAA|989|Woodland |Ave|Suite V|Bridgeport|Otsego County|NY|15817|United States|-5|condo| +35792|AAAAAAAAANLIAAAA|810|Elevnth 11th|Boulevard|Suite Q|Mount Zion|Ogemaw County|MI|48054|United States|-5|condo| +35793|AAAAAAAABNLIAAAA|384|View |Ct.|Suite H|Newport|Walton County|FL|31521|United States|-5|condo| +35794|AAAAAAAACNLIAAAA|890|Birch 5th|Street|Suite 140|Oak Hill|Emanuel County|GA|37838|United States|-5|apartment| +35795|AAAAAAAADNLIAAAA|389|Elm |Drive|Suite Y|Pleasant Grove|Mineral County|WV|24136|United States|-5|single family| +35796|AAAAAAAAENLIAAAA|239|15th |Way|Suite I|Spring Hill|Polk County|WI|56787|United States|-6|apartment| +35797|AAAAAAAAFNLIAAAA|440|10th Miller|ST|Suite 280|Marion|Fayette County|IL|60399|United States|-6|apartment| +35798|AAAAAAAAGNLIAAAA|141|Locust Oak|Way|Suite Q|Fisher|Warrick County|IN|42819|United States|-5|condo| +35799|AAAAAAAAHNLIAAAA|650|Miller Main|Way|Suite 160|Riverview|Bergen County|NJ|09603|United States|-5|condo| +35800|AAAAAAAAINLIAAAA|755|Center Dogwood|Road|Suite Q|Kingston|Cuyahoga County|OH|44975|United States|-5|single family| +35801|AAAAAAAAJNLIAAAA|776|1st Pine|Way|Suite V|Woodville|San Augustine County|TX|74289|United States|-6|apartment| +35802|AAAAAAAAKNLIAAAA|68|Hill |Blvd|Suite Q|Cedar Grove|Hot Springs County|WY|80411|United States|-7|single family| +35803|AAAAAAAALNLIAAAA|99|Church View|Blvd|Suite 150|Watkins|Schuyler County|MO|61732|United States|-6|condo| +35804|AAAAAAAAMNLIAAAA|711|Oak View|Wy|Suite A|Oak Hill|Republic County|KS|67838|United States|-6|single family| +35805|AAAAAAAANNLIAAAA|947|Woodland |Cir.|Suite 380|Greenville|Vanderburgh County|IN|41387|United States|-5|condo| +35806|AAAAAAAAONLIAAAA|783|Wilson |Avenue|Suite 100|Pleasant Grove|Bertie County|NC|24136|United States|-5|condo| +35807|AAAAAAAAPNLIAAAA|793|View Highland|ST|Suite 370|Five Forks|Campbell County|KY|42293|United States|-6|condo| +35808|AAAAAAAAAOLIAAAA|305|Birch Highland|Blvd|Suite 120|Riverdale|Cavalier County|ND|59391|United States|-6|single family| +35809|AAAAAAAABOLIAAAA|835||Drive|Suite 230|White Oak|Hamblen County|TN|36668|United States|-5|apartment| +35810|AAAAAAAACOLIAAAA|508|4th |Cir.|Suite Y|Lincoln|Jeff Davis County|GA|31289|United States|-5|condo| +35811|AAAAAAAADOLIAAAA|948|Chestnut |Drive|Suite A|Woodstock|Bailey County|TX|74422|United States|-6|condo| +35812|AAAAAAAAEOLIAAAA|547|9th |Court|Suite G|Lakewood|Kidder County|ND|58877|United States|-6|apartment| +35813|AAAAAAAAFOLIAAAA|880|Elm |Drive|Suite 150|Woodland|Ada County|ID|84854|United States|-7|apartment| +35814|AAAAAAAAGOLIAAAA|392|Washington 4th|Ct.|Suite F|Pleasant Hill|Grant County|WI|53604|United States|-6|apartment| +35815|AAAAAAAAHOLIAAAA|618|4th |Boulevard|Suite H|Friendship|Guthrie County|IA|54536|United States|-6|single family| +35816|AAAAAAAAIOLIAAAA|146|Mill 4th|Cir.|Suite G|Hopewell|Erath County|TX|70587|United States|-6|apartment| +35817|AAAAAAAAJOLIAAAA|940|Elevnth Seventh|Cir.|Suite 350|Lake Forest|Hillsborough County|NH|06600|United States|-5|condo| +35818|AAAAAAAAKOLIAAAA|||Road|Suite M|Salem|||78048|United States||| +35819|AAAAAAAALOLIAAAA|441|7th |Circle|Suite 380|Jackson|Ward County|TX|79583|United States|-6|apartment| +35820|AAAAAAAAMOLIAAAA|684|Jackson |Parkway|Suite I|Doyle|Poquoson city|VA|28434|United States|-5|apartment| +35821|AAAAAAAANOLIAAAA|543|East Poplar|Pkwy|Suite 50|Franklin|Haywood County|TN|39101|United States|-5|apartment| +35822|AAAAAAAAOOLIAAAA|441|Hickory |Circle|Suite 350|Summit|Calhoun County|MI|40499|United States|-5|condo| +35823|AAAAAAAAPOLIAAAA|883|Church Elm|Drive|Suite C|Walnut Grove|Belknap County|NH|08352|United States|-5|condo| +35824|AAAAAAAAAPLIAAAA|852|1st |Drive|Suite 160|Plainville|Calloway County|KY|46115|United States|-6|apartment| +35825|AAAAAAAABPLIAAAA|748|Woodland Ash|Pkwy|Suite E|Cherry Grove|Curry County|NM|81513|United States|-7|condo| +35826|AAAAAAAACPLIAAAA|30|Cherry Williams|Road|Suite 360|Greenfield|Spotsylvania County|VA|25038|United States|-5|single family| +35827|AAAAAAAADPLIAAAA|944|South Sunset|Street|Suite X|Enterprise|Austin County|TX|71757|United States|-6|single family| +35828|AAAAAAAAEPLIAAAA|545|9th Main|Drive|Suite F|Mount Vernon|Wayne County|UT|88482|United States|-7|condo| +35829|AAAAAAAAFPLIAAAA|498|Smith |Drive|Suite 10|Kingston|Williamson County|TX|74975|United States|-6|condo| +35830|AAAAAAAAGPLIAAAA|973|2nd |ST|Suite 120|Sunnyside|Donley County|TX|71952|United States|-6|condo| +35831|AAAAAAAAHPLIAAAA|634|4th Spring|Avenue|Suite 30|Woodlawn|Marengo County|AL|34098|United States|-6|condo| +35832|AAAAAAAAIPLIAAAA|782|7th 1st|Dr.|Suite B|Fairfield|Gallatin County|MT|66192|United States|-7|apartment| +35833|AAAAAAAAJPLIAAAA|710|Washington |Parkway|Suite 200|Shady Grove|Benton County|MS|52812|United States|-6|single family| +35834|AAAAAAAAKPLIAAAA|631|Davis View|Lane|Suite E|Allison|Erie County|OH|44167|United States|-5|condo| +35835|AAAAAAAALPLIAAAA|660|7th Williams|Avenue|Suite 370|Greenville|Guthrie County|IA|51387|United States|-6|apartment| +35836|AAAAAAAAMPLIAAAA|305|Railroad |Wy|Suite 20|Clifton|Grant County|OK|78014|United States|-6|single family| +35837|AAAAAAAANPLIAAAA|199|Hill |RD|Suite 0|Union Hill|Garfield County|MT|67746|United States|-7|single family| +35838|AAAAAAAAOPLIAAAA|867|Maple |Ln|Suite W|Saratoga|Wilkes County|NC|22123|United States|-5|condo| +35839|AAAAAAAAPPLIAAAA|518|Hill |Ave|Suite 170|Valley View|Cavalier County|ND|55124|United States|-6|apartment| +35840|AAAAAAAAAAMIAAAA|820|Ridge |Lane|Suite H|New Hope|Madison County|GA|39431|United States|-5|apartment| +35841|AAAAAAAABAMIAAAA|801|Cherry |Ave|Suite K|Greenwood|Muskogee County|OK|78828|United States|-6|single family| +35842|AAAAAAAACAMIAAAA|868|North Maple|Court|Suite 330|Woodlawn|Adams County|ND|54098|United States|-6|condo| +35843|AAAAAAAADAMIAAAA|491|Laurel |Circle|Suite B|Newport|Lorain County|OH|41521|United States|-5|apartment| +35844|AAAAAAAAEAMIAAAA|761|4th |Cir.|Suite 240|Five Forks|Anderson County|KS|62293|United States|-6|single family| +35845|AAAAAAAAFAMIAAAA|443|Miller 9th|Road|Suite L|Greenfield|Burleigh County|ND|55038|United States|-6|apartment| +35846|AAAAAAAAGAMIAAAA|706|Smith First|Ln|Suite A|Belmont|Pleasants County|WV|20191|United States|-5|single family| +35847|AAAAAAAAHAMIAAAA|339|Spring |Pkwy|Suite P|Montrose|Washington County|TN|34435|United States|-6|condo| +35848|AAAAAAAAIAMIAAAA|684|South |Ln|Suite V|Phoenix|Harvey County|KS|62276|United States|-6|single family| +35849|AAAAAAAAJAMIAAAA|875|Johnson Valley|RD|Suite 30|Kingston|Ocean County|NJ|05575|United States|-5|apartment| +35850|AAAAAAAAKAMIAAAA|713|Willow Franklin|Court|Suite 130|California|Jefferson County|WA|90141|United States|-8|single family| +35851|AAAAAAAALAMIAAAA|949|Walnut Third|RD|Suite 480|Shiloh|Scott County|IL|69275|United States|-6|condo| +35852|AAAAAAAAMAMIAAAA|365|Central |Avenue|Suite J|Woodlawn|Ontonagon County|MI|44098|United States|-5|apartment| +35853|AAAAAAAANAMIAAAA|720|Church Main|Lane|Suite R|Ashland|Dale County|AL|34244|United States|-6|condo| +35854|AAAAAAAAOAMIAAAA|334|Sunset Chestnut|Blvd|Suite I|Newtown|Richland County|ND|51749|United States|-6|condo| +35855|AAAAAAAAPAMIAAAA|531|East 1st|Street|Suite B|Clifton|Ness County|KS|68014|United States|-6|condo| +35856|AAAAAAAAABMIAAAA|808|College Cedar|Road|Suite 150|Plainview|Delaware County|PA|13683|United States|-5|condo| +35857|AAAAAAAABBMIAAAA|20|Washington |Drive|Suite H|Woodland|Polk County|AR|74854|United States|-6|condo| +35858|AAAAAAAACBMIAAAA|204|West Lake|Parkway|Suite 220|Mechanicsburg|Lincoln County|SD|52219|United States|-7|single family| +35859|AAAAAAAADBMIAAAA|224|Green 6th|Avenue|Suite 220|Antioch|Fallon County|MT|68605|United States|-7|apartment| +35860|AAAAAAAAEBMIAAAA|127|3rd Main|Way|Suite 130|Five Points|Sullivan County|NY|16098|United States|-5|condo| +35861|AAAAAAAAFBMIAAAA|19|Hill |Drive|Suite 80|Aberdeen|Morris County|KS|67995|United States|-6|condo| +35862|AAAAAAAAGBMIAAAA|168|Maple |Parkway|Suite 280|Macedonia|Washington Parish|LA|71087|United States|-6|apartment| +35863|AAAAAAAAHBMIAAAA|858|7th |Dr.|Suite 460|Stringtown|Boone County|KY|40162|United States|-6|condo| +35864|AAAAAAAAIBMIAAAA|671|Center |Ct.|Suite 430|Oakland|Fairfield County|OH|49843|United States|-5|condo| +35865|AAAAAAAAJBMIAAAA|714|2nd 10th|Circle|Suite 30|Edgewood|Hamilton County|IA|50069|United States|-6|apartment| +35866|AAAAAAAAKBMIAAAA|556|Meadow College|Court|Suite Y|Riverdale|Huntingdon County|PA|19391|United States|-5|apartment| +35867|AAAAAAAALBMIAAAA|169|9th |Parkway|Suite 190|Mount Zion|Lee County|GA|38054|United States|-5|single family| +35868|AAAAAAAAMBMIAAAA|479|10th |Boulevard|Suite T|Friendship|Jones County|IA|54536|United States|-6|apartment| +35869|AAAAAAAANBMIAAAA|452|Cherry |RD|Suite 390|Centerville|Corson County|SD|50059|United States|-6|condo| +35870|AAAAAAAAOBMIAAAA|55|3rd |Way|Suite Y|Florence|Starke County|IN|43394|United States|-5|single family| +35871|AAAAAAAAPBMIAAAA|356|Third |ST|Suite 450|Oakdale|Champaign County|IL|69584|United States|-6|single family| +35872|AAAAAAAAACMIAAAA|385|North |Lane|Suite 20|Spring Hill|Sabine Parish|LA|76787|United States|-6|condo| +35873|AAAAAAAABCMIAAAA|973||Court|Suite U|Derby|Routt County|||||single family| +35874|AAAAAAAACCMIAAAA|573|Sunset 11th|||Sheffield||MN|56896|United States||single family| +35875|AAAAAAAADCMIAAAA|390|First 9th|Dr.|Suite 390|Union|Clinton County|MI|48721|United States|-5|single family| +35876|AAAAAAAAECMIAAAA|97|Davis |Court|Suite 430|Farmington|Fort Bend County|TX|79145|United States|-6|apartment| +35877|AAAAAAAAFCMIAAAA|24|Adams 3rd|Ave|Suite 90|Georgetown|Nevada County|CA|97057|United States|-8|condo| +35878|AAAAAAAAGCMIAAAA|812|Center |Drive|Suite Y|Summit|Logan County|CO|80499|United States|-7|single family| +35879|AAAAAAAAHCMIAAAA|661|Jackson 12th|Street|Suite W|Pleasant Hill|Rock Island County|IL|63604|United States|-6|condo| +35880|AAAAAAAAICMIAAAA|115|Madison Laurel|Ave|Suite K|Caledonia|Bonneville County|ID|87411|United States|-7|single family| +35881|AAAAAAAAJCMIAAAA|249||||Marshall|Independence County|||United States||| +35882|AAAAAAAAKCMIAAAA|217|12th |Pkwy|Suite 390|Five Points|DeSoto County|FL|36098|United States|-5|condo| +35883|AAAAAAAALCMIAAAA|690|15th Forest|Ln|Suite K|Highland Park|Limestone County|AL|36534|United States|-6|apartment| +35884|AAAAAAAAMCMIAAAA|702|Church |Boulevard|Suite 320|Lakewood|Yuba County|CA|98877|United States|-8|apartment| +35885|AAAAAAAANCMIAAAA|870|Smith |Circle|Suite L|Brownsville|McKean County|PA|19310|United States|-5|apartment| +35886|AAAAAAAAOCMIAAAA|263|Pine |Lane|Suite B|Arlington|Washington County|TX|76557|United States|-6|condo| +35887|AAAAAAAAPCMIAAAA|380|Washington |Lane|Suite 200|Spring Hill|Tucker County|WV|26787|United States|-5|apartment| +35888|AAAAAAAAADMIAAAA|716|3rd Highland|Parkway|Suite H|Waterloo|Putnam County|IL|61675|United States|-6|condo| +35889|AAAAAAAABDMIAAAA|566|Cedar |RD|Suite 250|Hamilton|Cook County|IL|62808|United States|-6|single family| +35890|AAAAAAAACDMIAAAA|911|Cherry |Lane|Suite 150|Concord|Dickinson County|KS|64107|United States|-6|condo| +35891|AAAAAAAADDMIAAAA|511|Cedar Ash|Pkwy|Suite 60|Woodville|Sharp County|AR|74289|United States|-6|apartment| +35892|AAAAAAAAEDMIAAAA|337|11th |RD|Suite I|Macedonia|Pinellas County|FL|31087|United States|-5|single family| +35893|AAAAAAAAFDMIAAAA|21|Smith |Cir.|Suite T|Spring Valley|Richland County|SC|26060|United States|-5|condo| +35894|AAAAAAAAGDMIAAAA|522|River 10th|Street|Suite X|Franklin|Montgomery County|OH|49101|United States|-5|single family| +35895|AAAAAAAAHDMIAAAA|255|Pine 8th|Lane|Suite X|Riverview|Natrona County|WY|89003|United States|-7|condo| +35896|AAAAAAAAIDMIAAAA|363|Mill |Circle|Suite 310|Bunker Hill|Randolph County|GA|30150|United States|-5|single family| +35897|AAAAAAAAJDMIAAAA|212|Church Lee|Court|Suite 70|Pleasant Grove|Kenedy County|TX|74136|United States|-6|condo| +35898|AAAAAAAAKDMIAAAA|333|Washington First|Court|Suite H|Sulphur Springs|Madison County|IL|68354|United States|-6|single family| +35899|AAAAAAAALDMIAAAA|80|2nd |Court|Suite Q|Greenville|Kenai Peninsula Borough|AK|91387|United States|-9|condo| +35900|AAAAAAAAMDMIAAAA|499|Cherry |Parkway|Suite N|Oak Grove|Lincoln County|SD|58370|United States|-7|condo| +35901|AAAAAAAANDMIAAAA|465|Jackson Park|Way|Suite L|Waterloo|Daviess County|IN|41675|United States|-5|condo| +35902|AAAAAAAAODMIAAAA|243|View Park|Ave|Suite 460|Crossroads|Custer County|MT|60534|United States|-7|apartment| +35903|AAAAAAAAPDMIAAAA|792|Cedar College|Court|Suite B|Riverdale|Gilchrist County|FL|39391|United States|-5|apartment| +35904|AAAAAAAAAEMIAAAA|16|Lake 10th|Blvd|Suite A|Pleasant Hill|Boise County|ID|83604|United States|-7|apartment| +35905|AAAAAAAABEMIAAAA|851|Williams Cedar|Lane|Suite 100|Pleasant Hill|Phillips County|CO|83604|United States|-7|condo| +35906|AAAAAAAACEMIAAAA|782|Maple 13th|Road|Suite J|Concord|Wayne County|IN|44107|United States|-5|condo| +35907|AAAAAAAADEMIAAAA|433|Park 2nd|Boulevard|Suite Q|Riverside|Terrell County|TX|79231|United States|-6|condo| +35908|AAAAAAAAEEMIAAAA||Sycamore Lincoln||||||65752|||apartment| +35909|AAAAAAAAFEMIAAAA|375|Willow |Pkwy|Suite 60|Salem|Beaver County|OK|78048|United States|-6|condo| +35910|AAAAAAAAGEMIAAAA|795|Spring Jefferson|Ave|Suite 360|Greenville|Orange County|VT|01987|United States|-5|single family| +35911|AAAAAAAAHEMIAAAA|833|12th |Way|Suite S|Sulphur Springs|Summit County|CO|88354|United States|-7|single family| +35912|AAAAAAAAIEMIAAAA|289|4th Forest|Circle|Suite T|Greenfield|Pickens County|SC|25038|United States|-5|single family| +35913|AAAAAAAAJEMIAAAA|147|Woodland |Cir.|Suite 190|Riverview|Traverse County|MN|59003|United States|-6|condo| +35914|AAAAAAAAKEMIAAAA|623|Fifth |Drive|Suite 10|Fairfield|Montgomery County|AL|36192|United States|-6|single family| +35915|AAAAAAAALEMIAAAA|390|Forest Main|Way|Suite E|Fairview|Lafayette Parish|LA|75709|United States|-6|condo| +35916|AAAAAAAAMEMIAAAA|576|Sunset |Lane|Suite S|Spring Valley|Iron County|UT|86060|United States|-7|single family| +35917|AAAAAAAANEMIAAAA|249|Seventh Railroad|Dr.|Suite L|Rolling Hills|Clark County|ID|87272|United States|-7|apartment| +35918|AAAAAAAAOEMIAAAA|905|1st |Wy|Suite 50|Green Acres|Buffalo County|SD|57683|United States|-6|apartment| +35919|AAAAAAAAPEMIAAAA|540|Walnut 10th|Ln|Suite N|Five Forks|Northampton County|NC|22293|United States|-5|single family| +35920|AAAAAAAAAFMIAAAA|120|Pine 1st|Pkwy|Suite X|Mount Olive|Madison Parish|LA|78059|United States|-6|single family| +35921|AAAAAAAABFMIAAAA|428|8th South|ST|Suite 0|Farmington|Washington County|MS|59145|United States|-6|single family| +35922|AAAAAAAACFMIAAAA|696|South |Way|Suite 230|Spring Hill|Dawes County|NE|66787|United States|-6|apartment| +35923|AAAAAAAADFMIAAAA|294|Oak |Cir.|Suite 270|Brownsville|Shawano County|WI|59310|United States|-6|single family| +35924|AAAAAAAAEFMIAAAA|72|Park South|Lane|Suite S|Mountain View|Todd County|KY|44466|United States|-5|condo| +35925|AAAAAAAAFFMIAAAA|246|Church Hill|Street|Suite 330|Five Points|Hand County|SD|56098|United States|-7|apartment| +35926|AAAAAAAAGFMIAAAA|76|Broadway Hickory|Road|Suite F|New Hope|Clallam County|WA|99431|United States|-8|single family| +35927|AAAAAAAAHFMIAAAA|855|First |Ln|Suite 340|Harmony|Clinton County|MI|45804|United States|-5|condo| +35928|AAAAAAAAIFMIAAAA|119|Park Elm|Ave|Suite 40|Midway|Richmond County|GA|31904|United States|-5|apartment| +35929|AAAAAAAAJFMIAAAA|876|River |Blvd|Suite I|Oak Hill|Marion County|OH|47838|United States|-5|condo| +35930|AAAAAAAAKFMIAAAA|898|Main |Wy|Suite V|Waterloo|Washita County|OK|71675|United States|-6|condo| +35931|AAAAAAAALFMIAAAA||||Suite 490|Sawyer|||36045|||apartment| +35932|AAAAAAAAMFMIAAAA|966|Maple 9th|Ct.|Suite I|Springdale|Iberville Parish|LA|78883|United States|-6|apartment| +35933|AAAAAAAANFMIAAAA|655|Cherry 14th|RD|Suite M|Walnut Grove|Grand County|UT|87752|United States|-7|single family| +35934|AAAAAAAAOFMIAAAA|761|Fourth |Dr.|Suite 390|Forest|Kearney County|NE|67537|United States|-7|condo| +35935|AAAAAAAAPFMIAAAA|416|Lake |Cir.|Suite 220|Fairview|Harrison County|TX|75709|United States|-6|condo| +35936|AAAAAAAAAGMIAAAA|810|8th 1st|Way|Suite O|Mount Zion|Cannon County|TN|38054|United States|-5|apartment| +35937|AAAAAAAABGMIAAAA|333|2nd |Ave|Suite 160|Riverview|Tipton County|IN|49003|United States|-5|apartment| +35938|AAAAAAAACGMIAAAA|677|8th 10th|Lane|Suite 360|Fairfield|Tazewell County|VA|26192|United States|-5|condo| +35939|AAAAAAAADGMIAAAA|173|5th Elm|Lane|Suite H|Bethel|Highland County|OH|45281|United States|-5|apartment| +35940|AAAAAAAAEGMIAAAA|141|Hillcrest |Wy|Suite Q|Ridgeville|Sumter County|SC|29306|United States|-5|apartment| +35941|AAAAAAAAFGMIAAAA|773|Pine |Ave|Suite 40|Millbrook|Breathitt County|KY|47529|United States|-6|apartment| +35942|AAAAAAAAGGMIAAAA|393|Fourth |Blvd|Suite C|Cedar Grove|Haskell County|OK|70411|United States|-6|apartment| +35943|AAAAAAAAHGMIAAAA|70|9th |RD|Suite 290|Lincoln|Muskogee County|OK|71289|United States|-6|single family| +35944|AAAAAAAAIGMIAAAA|959|2nd |Parkway|Suite 0|Newtown|Pike County|OH|41749|United States|-5|single family| +35945|AAAAAAAAJGMIAAAA|702|Cherry |Parkway|Suite I|Pine Grove|Independence County|AR|74593|United States|-6|apartment| +35946|AAAAAAAAKGMIAAAA|604|Oak Spring|Wy|Suite U|Summit|Winneshiek County|IA|50499|United States|-6|single family| +35947|AAAAAAAALGMIAAAA|504|Willow |Court|Suite R|Red Hill|Wayne County|MO|64338|United States|-6|single family| +35948|AAAAAAAAMGMIAAAA|645|Davis East|Ct.|Suite 340|Franklin|Newton County|TX|79101|United States|-6|condo| +35949|AAAAAAAANGMIAAAA|252|Green |ST|Suite M|Green Acres|Ingham County|MI|47683|United States|-5|apartment| +35950|AAAAAAAAOGMIAAAA|404|Lincoln 3rd|Parkway|Suite 480|Oak Hill|San Juan County|WA|97838|United States|-8|apartment| +35951|AAAAAAAAPGMIAAAA|833|River 15th|RD|Suite H|Midway|Plymouth County|IA|51904|United States|-6|single family| +35952|AAAAAAAAAHMIAAAA|867|Maple Fourth|Ln|Suite 240|Clinton|Mills County|IA|58222|United States|-6|condo| +35953|AAAAAAAABHMIAAAA|671|8th First|Ct.|Suite F|Summit|Greene County|NY|10499|United States|-5|apartment| +35954|AAAAAAAACHMIAAAA|697|Chestnut Spring|Court|Suite T|Oak Hill|Skagit County|WA|97838|United States|-8|apartment| +35955|AAAAAAAADHMIAAAA|513|10th |Way|Suite 140|Sunnyside|Hart County|KY|41952|United States|-6|apartment| +35956|AAAAAAAAEHMIAAAA|206|Jefferson |Parkway|Suite 110|Oakwood|Woodson County|KS|60169|United States|-6|apartment| +35957|AAAAAAAAFHMIAAAA|481|15th |Circle|Suite I|Marion|Wheeler County|TX|70399|United States|-6|condo| +35958|AAAAAAAAGHMIAAAA|||Cir.|Suite M|White Oak|||56668|||condo| +35959|AAAAAAAAHHMIAAAA|598|Central |Avenue|Suite F|Glenwood|Sioux County|IA|53511|United States|-6|single family| +35960|AAAAAAAAIHMIAAAA|557|11th 13th|Blvd|Suite B|Church Hill|Barbour County|AL|33790|United States|-6|apartment| +35961|AAAAAAAAJHMIAAAA|780|Davis |Boulevard|Suite I|Macedonia|Juneau Borough|AK|91087|United States|-9|single family| +35962|AAAAAAAAKHMIAAAA|228|River Spring|Road|Suite 120|Oak Ridge|Clay County|TX|78371|United States|-6|single family| +35963|AAAAAAAALHMIAAAA|635|Wilson |Dr.|Suite 170|Forest Hills|Tazewell County|VA|29237|United States|-5|single family| +35964|AAAAAAAAMHMIAAAA|390|Forest |Court|Suite Q|Riverview|Gulf County|FL|39003|United States|-5|apartment| +35965|AAAAAAAANHMIAAAA|503|Center |Ct.|Suite 470|Bridgeport|Radford city|VA|25817|United States|-5|single family| +35966|AAAAAAAAOHMIAAAA|26|Johnson Cedar|Road|Suite 190|Woodville|Manistee County|MI|44289|United States|-5|condo| +35967|AAAAAAAAPHMIAAAA|676|Maple Central|Ct.|Suite C|Franklin|Screven County|GA|39101|United States|-5|single family| +35968|AAAAAAAAAIMIAAAA|223|Sunset 11th|Ave|Suite L|Sulphur Springs|Hale County|AL|38354|United States|-6|apartment| +35969|AAAAAAAABIMIAAAA|671|Center |Street|Suite 320|Maple Grove|Overton County|TN|38252|United States|-6|condo| +35970|AAAAAAAACIMIAAAA|773|Poplar Park|RD|Suite J|Pine Grove|Wapello County|IA|54593|United States|-6|single family| +35971|AAAAAAAADIMIAAAA|762|Main Pine|Pkwy|Suite S|Pleasant Valley|McHenry County|IL|62477|United States|-6|condo| +35972|AAAAAAAAEIMIAAAA|403|Railroad |Wy|Suite U|Five Points|Garvin County|OK|76098|United States|-6|condo| +35973|AAAAAAAAFIMIAAAA|925|Park |Ct.|Suite 230|Kingston|Carroll County|IA|54975|United States|-6|apartment| +35974|AAAAAAAAGIMIAAAA|77|7th 6th|Ln|Suite Q|Shannon|El Paso County|CO|84120|United States|-7|single family| +35975|AAAAAAAAHIMIAAAA|400|Fourth Mill|Road|Suite 150|Union|Will County|IL|68721|United States|-6|condo| +35976|AAAAAAAAIIMIAAAA|673|Meadow |Boulevard|Suite 490|Springdale|Stone County|MO|68883|United States|-6|single family| +35977|AAAAAAAAJIMIAAAA|556|5th |Parkway|Suite V|Oakland|Gooding County|ID|89843|United States|-7|condo| +35978|AAAAAAAAKIMIAAAA|664|Oak |Drive|Suite 280|Proctor|Atlantic County|NJ|08740|United States|-5|apartment| +35979|AAAAAAAALIMIAAAA|304|Washington |Street|Suite 370|Fox|Galax city|VA|20631|United States|-5|condo| +35980|AAAAAAAAMIMIAAAA|567||Street||||LA||United States||| +35981|AAAAAAAANIMIAAAA|287|3rd |Ave|Suite N|Lebanon|New Castle County|DE|12898|United States|-5|single family| +35982|AAAAAAAAOIMIAAAA|407|Third |Way|Suite 20|Woodville|Wyoming County|WV|24289|United States|-5|single family| +35983|AAAAAAAAPIMIAAAA|479|4th Jefferson|Parkway|Suite F|Deerfield|Park County|CO|89840|United States|-7|condo| +35984|AAAAAAAAAJMIAAAA|162|14th Twelfth|Cir.|Suite T|New Hope|Chambers County|AL|39431|United States|-6|single family| +35985|AAAAAAAABJMIAAAA|689|Sycamore |Road|Suite T|Macedonia|Rusk County|TX|71087|United States|-6|single family| +35986|AAAAAAAACJMIAAAA|158|Meadow Chestnut|Way|Suite W|Greenfield|Berkeley County|SC|25038|United States|-5|single family| +35987|AAAAAAAADJMIAAAA|67|12th 7th|Ave|Suite 220|Glendale|Lewis County|WV|23951|United States|-5|apartment| +35988|AAAAAAAAEJMIAAAA|488|2nd Main|Ave|Suite Q|Unionville|Smith County|TX|71711|United States|-6|single family| +35989|AAAAAAAAFJMIAAAA|919|Spring |Circle|Suite F|Springfield|Bracken County|KY|49303|United States|-6|single family| +35990|AAAAAAAAGJMIAAAA||12th Maple|Cir.|Suite E|Midway||MI||United States|-5|| +35991|AAAAAAAAHJMIAAAA|108|Highland |ST|Suite 480|Union|Franklin County|NY|18721|United States|-5|single family| +35992|AAAAAAAAIJMIAAAA|623|Main |Dr.|Suite R|Welcome|San Miguel County|NM|86386|United States|-7|single family| +35993|AAAAAAAAJJMIAAAA|46|Woodland |Drive|Suite 0|Farmington|Osage County|MO|69145|United States|-6|single family| +35994|AAAAAAAAKJMIAAAA|116|Central |ST|Suite L|Farmington|Powder River County|MT|69145|United States|-7|condo| +35995|AAAAAAAALJMIAAAA|925|6th |Parkway|Suite G|Riverdale|Gloucester County|VA|29391|United States|-5|condo| +35996|AAAAAAAAMJMIAAAA|137|4th Hill|Parkway|Suite 440|Pleasant Grove|Ford County|IL|64136|United States|-6|single family| +35997|AAAAAAAANJMIAAAA|383|Spruce Forest|ST|Suite I|Greenwood|Muskingum County|OH|48828|United States|-5|condo| +35998|AAAAAAAAOJMIAAAA|936|13th 6th|Blvd|Suite E|Shady Grove|Madison County|NY|12812|United States|-5|single family| +35999|AAAAAAAAPJMIAAAA|282|Jefferson |Lane|Suite F|Whispering Pines|Mohave County|AZ|87609|United States|-7|apartment| +36000|AAAAAAAAAKMIAAAA|936|View Mill|Boulevard|Suite 170|Glenwood|Lancaster County|NE|63511|United States|-7|single family| +36001|AAAAAAAABKMIAAAA|661|Meadow Main|Circle|Suite H|Union|Huerfano County|CO|88721|United States|-7|single family| +36002|AAAAAAAACKMIAAAA|355|9th |Parkway|Suite V|Spring Valley|Union County|OR|96060|United States|-8|single family| +36003|AAAAAAAADKMIAAAA|998|Smith |RD|Suite 460|Buena Vista|Durham County|NC|25752|United States|-5|apartment| +36004|AAAAAAAAEKMIAAAA|68|Sunset |Lane|Suite 350|Brownsville|Clay County|TN|39310|United States|-5|condo| +36005|AAAAAAAAFKMIAAAA|936|Jackson Lincoln|Ct.|Suite 170|Pine Grove|Bertie County|NC|24593|United States|-5|single family| +36006|AAAAAAAAGKMIAAAA|717|Adams |Way|Suite F|Arlington|Kodiak Island Borough|AK|96557|United States|-9|single family| +36007|AAAAAAAAHKMIAAAA|251|Valley Fifth|Way|Suite P|Lewis|Ramsey County|ND|57066|United States|-6|single family| +36008|AAAAAAAAIKMIAAAA|214|East |Drive|Suite 260|Arlington|Laurens County|GA|36557|United States|-5|apartment| +36009|AAAAAAAAJKMIAAAA|117|Smith Eigth|Ct.|Suite M|Pleasant Grove|Scott County|KY|44136|United States|-5|condo| +36010|AAAAAAAAKKMIAAAA|669|4th 10th|Drive|Suite S|Fairfield|George County|MS|56192|United States|-6|single family| +36011|AAAAAAAALKMIAAAA|921|Pine |Way|Suite 390|Riverdale|Walker County|GA|39391|United States|-5|apartment| +36012|AAAAAAAAMKMIAAAA|96|West |Ct.|Suite 160|Oakdale|Oklahoma County|OK|79584|United States|-6|single family| +36013|AAAAAAAANKMIAAAA|98|Center |Ave|Suite Y|Buena Vista|Knox County|IL|65752|United States|-6|single family| +36014|AAAAAAAAOKMIAAAA|90|Mill 12th|Way|Suite 430|Lakewood|Ingham County|MI|48877|United States|-5|apartment| +36015|AAAAAAAAPKMIAAAA|327|Johnson |Cir.|Suite 100|Georgetown|Walker County|GA|37057|United States|-5|apartment| +36016|AAAAAAAAALMIAAAA|517|South Jefferson|Blvd|Suite 20|Jamestown|Burt County|NE|66867|United States|-6|apartment| +36017|AAAAAAAABLMIAAAA|873|Railroad |Street|Suite V|Sunnyside|Banks County|GA|31952|United States|-5|single family| +36018|AAAAAAAACLMIAAAA|232|Mill 6th|Blvd|Suite 100|Fairfield|Huron County|OH|46192|United States|-5|condo| +36019|AAAAAAAADLMIAAAA|239|Meadow |Pkwy|Suite 190|Shiloh|Labette County|KS|69275|United States|-6|condo| +36020|AAAAAAAAELMIAAAA|140|Franklin Dogwood|Cir.|Suite 290|Mount Pleasant|Mason County|WV|21933|United States|-5|single family| +36021|AAAAAAAAFLMIAAAA|963|Birch Highland|RD|Suite F|Oakdale|Manistee County|MI|49584|United States|-5|condo| +36022|AAAAAAAAGLMIAAAA|591|Willow Highland|Road|Suite 110|Westgate|Wayne County|OH|42366|United States|-5|condo| +36023|AAAAAAAAHLMIAAAA|939|Pine |Circle|Suite 100|Bunker Hill|Muskingum County|OH|40150|United States|-5|single family| +36024|AAAAAAAAILMIAAAA|377|South 1st|RD|Suite U|Newtown|Lowndes County|GA|31749|United States|-5|apartment| +36025|AAAAAAAAJLMIAAAA|53|Cedar |ST|Suite 150|Woodland|Franklin County|NE|64854|United States|-6|apartment| +36026|AAAAAAAAKLMIAAAA|302|Park Meadow|Ave|Suite 230|Forest Hills|Ramsey County|ND|59237|United States|-6|single family| +36027|AAAAAAAALLMIAAAA|617|Green 13th|Drive|Suite W|Jackson|Clinch County|GA|39583|United States|-5|condo| +36028|AAAAAAAAMLMIAAAA|745|Church 5th|Boulevard|Suite 70|Lincoln|Piatt County|IL|61289|United States|-6|apartment| +36029|AAAAAAAANLMIAAAA|383|Willow |ST|Suite 140|Franklin|Ritchie County|WV|29101|United States|-5|single family| +36030|AAAAAAAAOLMIAAAA|140|Cherry 5th|Dr.|Suite 50|Green Acres|Essex County|VT|08283|United States|-5|condo| +36031|AAAAAAAAPLMIAAAA|834|Fourteenth Cedar|Avenue|Suite S|Mount Olive|Kinney County|TX|78059|United States|-6|apartment| +36032|AAAAAAAAAMMIAAAA|689|Fifth South|RD|Suite 320|Sutton|Morrow County|OR|95413|United States|-8|apartment| +36033|AAAAAAAABMMIAAAA|749|Cherry |Lane|Suite O|Bethel|Buena Vista city|VA|25281|United States|-5|condo| +36034|AAAAAAAACMMIAAAA|593|3rd 15th|Way|Suite 480|Summerfield|Colfax County|NM|80634|United States|-7|single family| +36035|AAAAAAAADMMIAAAA|251|Jackson 5th|Street|Suite U|Brownsville|Wheatland County|MT|69310|United States|-7|apartment| +36036|AAAAAAAAEMMIAAAA|832|Maple Tenth|Wy|Suite L|Greenfield|Morris County|NJ|05638|United States|-5|condo| +36037|AAAAAAAAFMMIAAAA|885|Lincoln Broadway|Wy|Suite 490|Five Points|Benton County|MO|66098|United States|-6|condo| +36038|AAAAAAAAGMMIAAAA|391|Spruce |Street|Suite H|Pine Grove|Winston County|MS|54593|United States|-6|single family| +36039|AAAAAAAAHMMIAAAA|627|Park |Dr.|Suite T|Buckingham|Somervell County|TX|74092|United States|-6|condo| +36040|AAAAAAAAIMMIAAAA|807|||||Monroe County|MO||||| +36041|AAAAAAAAJMMIAAAA|154|12th Lake|Blvd|Suite 70|New Hope|Hall County|TX|79431|United States|-6|condo| +36042|AAAAAAAAKMMIAAAA|578|Oak |ST|Suite O|Sunnyside|Douglas County|NV|81952|United States|-8|single family| +36043|AAAAAAAALMMIAAAA|507|Second |Court|Suite S|Mount Zion|Clinton County|KY|48054|United States|-6|apartment| +36044|AAAAAAAAMMMIAAAA|888|2nd Smith|Boulevard|Suite M|Greenville|Prince George County|MD|21387|United States|-5|apartment| +36045|AAAAAAAANMMIAAAA|779|Fifth |ST|Suite E|Wildwood|Craig County|VA|26871|United States|-5|apartment| +36046|AAAAAAAAOMMIAAAA|328|Pine |ST|Suite I|Wilson|Perry County|IL|66971|United States|-6|single family| +36047|AAAAAAAAPMMIAAAA|488|North Main|Lane|Suite H|Greenwood|Sherman County|KS|68828|United States|-6|condo| +36048|AAAAAAAAANMIAAAA|359|Chestnut Franklin|Way|Suite W|Pine Grove|Pope County|AR|74593|United States|-6|apartment| +36049|AAAAAAAABNMIAAAA|528|15th First|Ave|Suite A|Springfield|Washington County|VA|29303|United States|-5|apartment| +36050|AAAAAAAACNMIAAAA|373|Lee 3rd|Court|Suite F|Crossroads|Pontotoc County|MS|50534|United States|-6|apartment| +36051|AAAAAAAADNMIAAAA|822|Fourth Park|Court|Suite 240|Pine Grove|Bowie County|TX|74593|United States|-6|condo| +36052|AAAAAAAAENMIAAAA|317|2nd Third|Cir.|Suite 410|Jackson|Uvalde County|TX|79583|United States|-6|apartment| +36053|AAAAAAAAFNMIAAAA|||||Walnut Grove|||77752|United States||single family| +36054|AAAAAAAAGNMIAAAA|480|Center |Road|Suite 440|Oak Grove|Oglethorpe County|GA|38370|United States|-5|condo| +36055|AAAAAAAAHNMIAAAA|569|Spring Williams|Court|Suite U|Spring Hill|Ellsworth County|KS|66787|United States|-6|apartment| +36056|AAAAAAAAINMIAAAA|279|Walnut |RD|Suite B|Maple Grove|Monroe County|OH|48252|United States|-5|single family| +36057|AAAAAAAAJNMIAAAA|467|Main |Avenue|Suite Q|Antioch|Seminole County|GA|38605|United States|-5|condo| +36058|AAAAAAAAKNMIAAAA|714|Elm |Avenue|Suite T|Pleasant Valley|Lawrence County|PA|12477|United States|-5|apartment| +36059|AAAAAAAALNMIAAAA|477|Hillcrest |Way|Suite I|Church Hill|Los Angeles County|CA|93790|United States|-8|apartment| +36060|AAAAAAAAMNMIAAAA|436|East Cherry|Boulevard|Suite 450|Walnut Grove|Audubon County|IA|57752|United States|-6|apartment| +36061|AAAAAAAANNMIAAAA|860|East Sunset|Lane|Suite L|Sulphur Springs|Craig County|VA|28354|United States|-5|single family| +36062|AAAAAAAAONMIAAAA|538|Dogwood 6th|Way|Suite Y|Valley View|Orleans County|NY|15124|United States|-5|single family| +36063|AAAAAAAAPNMIAAAA|658|Wilson Valley|Ln|Suite 360|Macedonia|Platte County|WY|81087|United States|-7|apartment| +36064|AAAAAAAAAOMIAAAA|544|Park Jefferson|Ave|Suite V|Crossroads|Livingston County|MO|60534|United States|-6|single family| +36065|AAAAAAAABOMIAAAA|426|||Suite B|Ashland|||34244||-5|| +36066|AAAAAAAACOMIAAAA|750|Laurel |Avenue|Suite 460|Ashland|Charles Mix County|SD|54244|United States|-6|condo| +36067|AAAAAAAADOMIAAAA|98|Jackson |Ave|Suite W|Pleasant Grove|Prairie County|AR|74136|United States|-6|apartment| +36068|AAAAAAAAEOMIAAAA|738|2nd |Dr.|Suite 490|Edgewood|Greene County|IL|60069|United States|-6|apartment| +36069|AAAAAAAAFOMIAAAA|459|North 7th|Ln|Suite 220|Wyoming|Brevard County|FL|30216|United States|-5|condo| +36070|AAAAAAAAGOMIAAAA|979|8th |Ave|Suite G|Brownsville|Greene County|IN|49310|United States|-5|condo| +36071|AAAAAAAAHOMIAAAA|924|Maple |Drive|Suite 20|Oakland|Union County|NJ|09843|United States|-5|single family| +36072|AAAAAAAAIOMIAAAA|701|Green |Street|Suite 390|Farmington|Butler County|KY|49145|United States|-6|apartment| +36073|AAAAAAAAJOMIAAAA|689|Ridge |Pkwy|Suite 210|Oakwood|Barry County|MI|40169|United States|-5|single family| +36074|AAAAAAAAKOMIAAAA|163|Cherry |Ln|Suite S|Deerfield|Dakota County|MN|59840|United States|-6|single family| +36075|AAAAAAAALOMIAAAA|560|South Sycamore|Avenue|Suite H|Bunker Hill|Clay County|AR|70150|United States|-6|apartment| +36076|AAAAAAAAMOMIAAAA|337|Park |Cir.|Suite 200|Buena Vista|Lincoln County|ID|85752|United States|-7|condo| +36077|AAAAAAAANOMIAAAA|455|15th Maple|Wy|Suite O|Springfield|Spencer County|IN|49303|United States|-5|condo| +36078|AAAAAAAAOOMIAAAA|888|Cherry |Dr.|Suite R|Pleasant Valley|Lac qui Parle County|MN|52477|United States|-6|apartment| +36079|AAAAAAAAPOMIAAAA|316|Mill |Way|Suite 360|Walnut Grove|Thomas County|NE|67752|United States|-7|single family| +36080|AAAAAAAAAPMIAAAA|126|College Thirteenth|Ct.|Suite 280|Lakeview|Adams County|IN|48579|United States|-5|apartment| +36081|AAAAAAAABPMIAAAA|430|5th |Ave|Suite J|Lakewood|Randolph County|WV|28877|United States|-5|apartment| +36082|AAAAAAAACPMIAAAA|366|Sycamore Walnut|Lane|Suite 360|Lebanon|Jasper County|GA|32898|United States|-5|condo| +36083|AAAAAAAADPMIAAAA|||Drive|Suite 120||Calhoun County|GA||United States||condo| +36084|AAAAAAAAEPMIAAAA|898|Center Birch|Court|Suite 100|Mount Zion|Pennington County|SD|58054|United States|-7|condo| +36085|AAAAAAAAFPMIAAAA|44|Center |Street|Suite 440|Midway|Summit County|UT|81904|United States|-7|apartment| +36086|AAAAAAAAGPMIAAAA|719|Franklin Pine|Ln|Suite 470|Mount Olive|Cassia County|ID|88059|United States|-7|condo| +36087|AAAAAAAAHPMIAAAA|695|Seventh |RD|Suite 60|Oakdale|Baker County|OR|99584|United States|-8|single family| +36088|AAAAAAAAIPMIAAAA|594|Willow |Court|Suite 100|Stringtown|Kent County|MI|40162|United States|-5|condo| +36089|AAAAAAAAJPMIAAAA|422|Williams |Avenue|Suite 480|Lakewood|Thayer County|NE|68877|United States|-7|condo| +36090|AAAAAAAAKPMIAAAA|762|Miller |RD|Suite B|Woodlawn|Jefferson County|MT|64098|United States|-7|single family| +36091|AAAAAAAALPMIAAAA|951|Mill 1st|Circle|Suite O|Arlington|Clay County|AR|76557|United States|-6|single family| +36092|AAAAAAAAMPMIAAAA|||Ave||Oakland|Clallam County||99843|||single family| +36093|AAAAAAAANPMIAAAA|421|First |Parkway|Suite C|Green Acres|Denton County|TX|77683|United States|-6|apartment| +36094|AAAAAAAAOPMIAAAA|927|Ridge Lake|Road|Suite I|Glendale|Adams County|ND|53951|United States|-6|condo| +36095|AAAAAAAAPPMIAAAA|884|Lincoln |Ct.|Suite 430|Unionville|Dickinson County|MI|41711|United States|-5|apartment| +36096|AAAAAAAAAANIAAAA|324|East Highland|Avenue|Suite 350|Jackson|Washington County|WI|59583|United States|-6|single family| +36097|AAAAAAAABANIAAAA|883|Williams Poplar|Dr.|Suite 170|Arlington|Goochland County|VA|26557|United States|-5|single family| +36098|AAAAAAAACANIAAAA||Davis |||Concord|Wilcox County|GA||United States|-5|| +36099|AAAAAAAADANIAAAA|785|Hickory Tenth|Court|Suite 210|Riverside|Columbia County|FL|39231|United States|-5|apartment| +36100|AAAAAAAAEANIAAAA|507|Walnut |Ct.|Suite 300|Leland|Harrison County|KY|49452|United States|-6|single family| +36101|AAAAAAAAFANIAAAA|921|1st 6th|Wy|Suite 310|Green Acres|Richland County|MT|67683|United States|-7|condo| +36102|AAAAAAAAGANIAAAA|980|Johnson |Wy|Suite K|Bunker Hill|Lewis County|KY|40150|United States|-5|condo| +36103|AAAAAAAAHANIAAAA|117|Lincoln |Circle|Suite K|Cordova|Swift County|MN|56938|United States|-6|condo| +36104|AAAAAAAAIANIAAAA|132|Highland |Boulevard|Suite 410|Ashland|Grafton County|NH|04844|United States|-5|condo| +36105|AAAAAAAAJANIAAAA|344|10th |RD|Suite 180|Arlington|Essex County|MA|07157|United States|-5|single family| +36106|AAAAAAAAKANIAAAA|184|Adams |Boulevard|Suite T|Lake Forest|Greenwood County|KS|66000|United States|-6|single family| +36107|AAAAAAAALANIAAAA|62|7th 1st|Way|Suite X|Shady Grove|Thomas County|NE|62812|United States|-7|apartment| +36108|AAAAAAAAMANIAAAA|313|View |Drive|Suite C|Florence|Saginaw County|MI|43394|United States|-5|apartment| +36109|AAAAAAAANANIAAAA|191|Laurel View|Ct.|Suite 120|Guilford|Okaloosa County|FL|34408|United States|-5|condo| +36110|AAAAAAAAOANIAAAA|430|East |Lane|Suite 240|Summit|Aransas County|TX|70499|United States|-6|single family| +36111|AAAAAAAAPANIAAAA|686|Mill |Avenue|Suite Y|Union|Robeson County|NC|28721|United States|-5|apartment| +36112|AAAAAAAAABNIAAAA|128|Highland |Road|Suite 30|Cedar Grove|Norton city|VA|20411|United States|-5|condo| +36113|AAAAAAAABBNIAAAA|624|Cedar Jefferson|Court|Suite X|Lakeside|Yamhill County|OR|99532|United States|-8|apartment| +36114|AAAAAAAACBNIAAAA|87|Park Spruce|Boulevard|Suite 460|Harmony|Charles Mix County|SD|55804|United States|-6|single family| +36115|AAAAAAAADBNIAAAA||||Suite I||Grant County|||United States||| +36116|AAAAAAAAEBNIAAAA|723|14th |Drive|Suite E|Crossroads|Guthrie County|IA|50534|United States|-6|apartment| +36117|AAAAAAAAFBNIAAAA|513|River |Way|Suite A|Georgetown|Washington County|NC|27057|United States|-5|condo| +36118|AAAAAAAAGBNIAAAA|453|Central |Ave|Suite 430|Greenwood|Belmont County|OH|48828|United States|-5|apartment| +36119|AAAAAAAAHBNIAAAA|712|Adams Jackson|Ave|Suite 250|Georgetown|De Kalb County|IN|47057|United States|-5|condo| +36120|AAAAAAAAIBNIAAAA|381|College |Drive|Suite K|Concord|Schoolcraft County|MI|44107|United States|-5|apartment| +36121|AAAAAAAAJBNIAAAA|690|Railroad |Court|Suite G|Crossroads|Van Buren County|MI|40534|United States|-5|apartment| +36122|AAAAAAAAKBNIAAAA|803|Sycamore |Street|Suite 300|Enterprise|Champaign County|OH|41757|United States|-5|single family| +36123|AAAAAAAALBNIAAAA|195||Road||Fairview|||95709|United States|-8|| +36124|AAAAAAAAMBNIAAAA|855||RD|Suite 20|Salem|Jefferson County|PA|18048|||condo| +36125|AAAAAAAANBNIAAAA|832|2nd Chestnut|Boulevard|Suite C|Lakeview|Cambria County|PA|18579|United States|-5|condo| +36126|AAAAAAAAOBNIAAAA|5|Davis |Court|Suite U|Valley View|Lee County|MS|55124|United States|-6|condo| +36127|AAAAAAAAPBNIAAAA|680|Fourth Forest|Ave|Suite 400|Wilson|Los Angeles County|CA|96971|United States|-8|single family| +36128|AAAAAAAAACNIAAAA|500|1st 15th|Street|Suite 410|Maple Grove|Iron County|MI|48252|United States|-5|condo| +36129|AAAAAAAABCNIAAAA|656|Valley 8th|Way|Suite 480|Jackson|Grundy County|TN|39583|United States|-5|condo| +36130|AAAAAAAACCNIAAAA|435||Ln||||||United States|-8|single family| +36131|AAAAAAAADCNIAAAA|672|Cedar Seventh|Boulevard|Suite Q|Hillcrest|Trigg County|KY|43003|United States|-5|single family| +36132|AAAAAAAAECNIAAAA|692|Spruce 4th|Avenue|Suite C|Clifton|Jerauld County|SD|58014|United States|-7|condo| +36133|AAAAAAAAFCNIAAAA|302|Ninth Park|Lane|Suite 380|Wilson|Parker County|TX|76971|United States|-6|single family| +36134|AAAAAAAAGCNIAAAA|719|Miller 4th|Boulevard|Suite 320|Lake Forest|Anderson County|TX|76000|United States|-6|single family| +36135|AAAAAAAAHCNIAAAA|461|Hickory Jackson|Pkwy|Suite A|Oak Grove|Patrick County|VA|28370|United States|-5|condo| +36136|AAAAAAAAICNIAAAA|831|Forest |Court|Suite X|Peru|Carson City|NV|80302|United States|-8|single family| +36137|AAAAAAAAJCNIAAAA|399|Madison Main|Boulevard|Suite W|Woodlawn|Fulton County|IL|64098|United States|-6|condo| +36138|AAAAAAAAKCNIAAAA|62|Miller |ST|Suite 420|Lincoln|Lincoln County|OK|71289|United States|-6|condo| +36139|AAAAAAAALCNIAAAA|716|Adams Church|Wy|Suite 320|Walnut Grove|Strafford County|NH|08352|United States|-5|single family| +36140|AAAAAAAAMCNIAAAA|909|Fifth |Parkway|Suite G|Springfield|Plymouth County|MA|09903|United States|-5|apartment| +36141|AAAAAAAANCNIAAAA|860|10th 9th|RD|Suite 270|Ashland|Elmore County|ID|84244|United States|-7|single family| +36142|AAAAAAAAOCNIAAAA|712|4th 1st|Court|Suite X|Glenwood|Tift County|GA|33511|United States|-5|apartment| +36143|AAAAAAAAPCNIAAAA|369|Walnut |Cir.|Suite 30|Oakwood|Blair County|PA|10169|United States|-5|single family| +36144|AAAAAAAAADNIAAAA|921|Main 6th|Boulevard|Suite H|Newport|Hale County|AL|31521|United States|-6|single family| +36145|AAAAAAAABDNIAAAA|816|Main |Court|Suite P|Wilson|Door County|WI|56971|United States|-6|condo| +36146|AAAAAAAACDNIAAAA|172||||Five Forks||FL|32293||-5|single family| +36147|AAAAAAAADDNIAAAA|410|5th |Road|Suite 350|Ashland|Sublette County|WY|84244|United States|-7|apartment| +36148|AAAAAAAAEDNIAAAA|776|East Oak|Pkwy|Suite Q|Harmony|Reeves County|TX|75804|United States|-6|apartment| +36149|AAAAAAAAFDNIAAAA|568|West |Street|Suite U|Lakeview|Lee County|VA|28579|United States|-5|apartment| +36150|AAAAAAAAGDNIAAAA|825|East |Ave|Suite B|Five Points|Arenac County|MI|46098|United States|-5|single family| +36151|AAAAAAAAHDNIAAAA|95|Smith Miller|ST|Suite 200|Waterloo|Butler County|IA|51675|United States|-6|single family| +36152|AAAAAAAAIDNIAAAA|465|Maple Walnut|Street|Suite 10|Oak Grove|Essex County|NJ|08970|United States|-5|single family| +36153|AAAAAAAAJDNIAAAA|698|Chestnut |Circle|Suite 140|Summit|Saline County|MO|60499|United States|-6|apartment| +36154|AAAAAAAAKDNIAAAA|286|Maple Lake|Parkway|Suite 180|Brownsville|Harnett County|NC|29310|United States|-5|apartment| +36155|AAAAAAAALDNIAAAA|553|Third Ninth|Road|Suite 440|Stafford|Terrell County|TX|74980|United States|-6|condo| +36156|AAAAAAAAMDNIAAAA|936|Johnson 8th|RD|Suite A|Shady Grove|Campbell County|TN|32812|United States|-5|apartment| +36157|AAAAAAAANDNIAAAA|275|Thirteenth |Street|Suite X|Pleasant Hill|Middlesex County|NJ|04204|United States|-5|apartment| +36158|AAAAAAAAODNIAAAA|942|Park Central|Ct.|Suite X|Frenchtown|Greene County|MS|52629|United States|-6|apartment| +36159|AAAAAAAAPDNIAAAA|983|Washington |Pkwy|Suite 420|Lebanon|Hickman County|KY|42898|United States|-6|apartment| +36160|AAAAAAAAAENIAAAA|874|Johnson Jackson|Boulevard|Suite E|Shady Grove|Gilmer County|WV|22812|United States|-5|condo| +36161|AAAAAAAABENIAAAA|832|Highland |Way|Suite 160|Mount Olive|Chesapeake city|VA|28059|United States|-5|single family| +36162|AAAAAAAACENIAAAA|93|Cedar |Ln|Suite 210|Newport|Chautauqua County|KS|61521|United States|-6|condo| +36163|AAAAAAAADENIAAAA|626|Washington 14th|Blvd|Suite 250|Providence|Chippewa County|WI|56614|United States|-6|condo| +36164|AAAAAAAAEENIAAAA|917|West |Cir.|Suite N|Highland|McHenry County|ND|59454|United States|-6|condo| +36165|AAAAAAAAFENIAAAA|493|Jackson |Way|Suite 440|Stringtown|Saunders County|NE|60162|United States|-7|condo| +36166|AAAAAAAAGENIAAAA|894|2nd Hillcrest|Road|Suite D|Enterprise|Jerauld County|SD|51757|United States|-7|apartment| +36167|AAAAAAAAHENIAAAA|385|Eigth 11th|Boulevard|Suite 450|Liberty|Terrebonne Parish|LA|73451|United States|-6|single family| +36168|AAAAAAAAIENIAAAA|257|4th |Drive|Suite U|Maple Grove|Marquette County|MI|48252|United States|-5|single family| +36169|AAAAAAAAJENIAAAA|971|Pine 7th|Parkway|Suite 410|Clifton|Madison County|NC|28014|United States|-5|apartment| +36170|AAAAAAAAKENIAAAA|174|5th Sixth|ST|Suite 320|Waterloo|Trinity County|CA|91675|United States|-8|condo| +36171|AAAAAAAALENIAAAA|383|Railroad |Avenue|Suite W|Woodland|Yankton County|SD|54854|United States|-6|apartment| +36172|AAAAAAAAMENIAAAA|332|View View|Ave|Suite 370|Springfield|El Paso County|CO|89303|United States|-7|condo| +36173|AAAAAAAANENIAAAA|134|Willow |Ln|Suite 220|Georgetown|Millard County|UT|87057|United States|-7|condo| +36174|AAAAAAAAOENIAAAA|158|Eigth |||||MO|64244|United States||single family| +36175|AAAAAAAAPENIAAAA|570|Chestnut |Drive|Suite P|Five Points|Cuyahoga County|OH|46098|United States|-5|apartment| +36176|AAAAAAAAAFNIAAAA|758|Walnut |Ct.|Suite K|Arlington|Franklin County|KY|46557|United States|-6|single family| +36177|AAAAAAAABFNIAAAA|117|Lakeview Seventh|Circle|Suite 370|White Oak|Okaloosa County|FL|36668|United States|-5|apartment| +36178|AAAAAAAACFNIAAAA|967|12th |Boulevard|Suite 70|Greenfield|Robertson County|TX|75038|United States|-6|condo| +36179|AAAAAAAADFNIAAAA|182|3rd East|Boulevard|Suite 70|Marion|Isanti County|MN|50399|United States|-6|condo| +36180|AAAAAAAAEFNIAAAA|607|Hillcrest Pine|Way|Suite I|Pleasant Valley|Chattahoochee County|GA|32477|United States|-5|condo| +36181|AAAAAAAAFFNIAAAA|884|Maple |Way|Suite 340|Freeman|Pike County|GA|32297|United States|-5|single family| +36182|AAAAAAAAGFNIAAAA|704|Poplar |Cir.|Suite I|Salem|Greer County|OK|78048|United States|-6|condo| +36183|AAAAAAAAHFNIAAAA|434|West 1st|Road|Suite R|Providence|Donley County|TX|76614|United States|-6|apartment| +36184|AAAAAAAAIFNIAAAA|814|Park |Ln|Suite 200|Midway|Rankin County|MS|51904|United States|-6|apartment| +36185|AAAAAAAAJFNIAAAA|637|Eigth 1st|Way|Suite P|Clifton|Reynolds County|MO|68014|United States|-6|condo| +36186|AAAAAAAAKFNIAAAA|138||Way|||||29583|United States||| +36187|AAAAAAAALFNIAAAA|753|Oak |Ave|Suite 160|Union|Audubon County|IA|58721|United States|-6|apartment| +36188|AAAAAAAAMFNIAAAA|137|Cherry |Blvd|Suite 240|Cedar Grove|Mercer County|MO|60411|United States|-6|apartment| +36189|AAAAAAAANFNIAAAA|779|Lee Washington|Boulevard|Suite Q|Lakewood|Broadwater County|MT|68877|United States|-7|condo| +36190|AAAAAAAAOFNIAAAA|631|Hickory Elm|Ct.|Suite T|Washington Heights|Bingham County|ID|88167|United States|-7|single family| +36191|AAAAAAAAPFNIAAAA|368|Center |Drive|Suite 290|Belmont|Calhoun County|MI|40191|United States|-5|condo| +36192|AAAAAAAAAGNIAAAA|246|Forest |Lane|Suite 100|Pleasant Hill|Marion County|OR|93604|United States|-8|single family| +36193|AAAAAAAABGNIAAAA|681|North |Court|Suite X|Spring Grove|Butler County|NE|66719|United States|-6|apartment| +36194|AAAAAAAACGNIAAAA|458|Hickory South|Road|Suite V|Enterprise|East Feliciana Parish|LA|71757|United States|-6|single family| +36195|AAAAAAAADGNIAAAA|269|Forest Washington|RD|Suite 250|Arlington|Waynesboro city|VA|26557|United States|-5|single family| +36196|AAAAAAAAEGNIAAAA|730|Jackson Lake|Dr.|Suite 30|Edgewood|Clay County|IA|50069|United States|-6|condo| +36197|AAAAAAAAFGNIAAAA|308|3rd 2nd|Ct.|Suite Y|Milo|Carroll County|TN|30116|United States|-5|single family| +36198|AAAAAAAAGGNIAAAA|904|Laurel |Way|Suite H|Shady Grove|Summit County|UT|82812|United States|-7|single family| +36199|AAAAAAAAHGNIAAAA|334|Sycamore Adams|Court|Suite M|Farmington|Livingston County|NY|19145|United States|-5|single family| +36200|AAAAAAAAIGNIAAAA|529|Cherry |Street|Suite M|Florence|Emanuel County|GA|33394|United States|-5|condo| +36201|AAAAAAAAJGNIAAAA|311|8th Wilson|RD|Suite M|Harmony|Pike County|AR|75804|United States|-6|apartment| +36202|AAAAAAAAKGNIAAAA|560|Lakeview |Parkway|Suite F|Red Hill|Miami County|IN|44338|United States|-5|single family| +36203|AAAAAAAALGNIAAAA|389|Chestnut |Blvd|Suite D|Valley View|Los Alamos County|NM|85124|United States|-7|single family| +36204|AAAAAAAAMGNIAAAA|777|North Sunset|Court|Suite D|Highland Park|Pike County|IL|66534|United States|-6|condo| +36205|AAAAAAAANGNIAAAA||South ||Suite M|Ashland||ND|||-6|| +36206|AAAAAAAAOGNIAAAA|686|Elm |Parkway|Suite 460|Hillcrest|Suwannee County|FL|33003|United States|-5|apartment| +36207|AAAAAAAAPGNIAAAA|584|Franklin |Way|Suite 490|Cedar Grove|Red River County|TX|70411|United States|-6|condo| +36208|AAAAAAAAAHNIAAAA|97|Lakeview 1st|Dr.|Suite 260|Mount Pleasant|Alpine County|CA|91933|United States|-8|apartment| +36209|AAAAAAAABHNIAAAA|502|First 9th|Ln|Suite C|Waterloo|Madison County|MT|61675|United States|-7|single family| +36210|AAAAAAAACHNIAAAA|520|Davis |Cir.|Suite P|Red Hill|Yazoo County|MS|54338|United States|-6|single family| +36211|AAAAAAAADHNIAAAA|432|Elm |Boulevard|Suite M|New Hope|Madison County|MT|69431|United States|-7|apartment| +36212|AAAAAAAAEHNIAAAA|512|Maple ||Suite 30|Lakewood|Warren County|GA|38877|||condo| +36213|AAAAAAAAFHNIAAAA|75|Fifth 8th|Circle|Suite 450|Perkins|Solano County|CA|91852|United States|-8|condo| +36214|AAAAAAAAGHNIAAAA|389|Oak Hickory|Drive|Suite D|Franklin|Martin County|FL|39101|United States|-5|single family| +36215|AAAAAAAAHHNIAAAA|338|11th |Cir.|Suite X|Crossroads|Wilkes County|NC|20534|United States|-5|single family| +36216|AAAAAAAAIHNIAAAA|769|First |Cir.|Suite 440|Saint Clair|Wright County|MN|55294|United States|-6|single family| +36217|AAAAAAAAJHNIAAAA|95|College Lake|Ave|Suite Q|Maple Grove|Wayne County|MS|58252|United States|-6|condo| +36218|AAAAAAAAKHNIAAAA|56|Main |Cir.|Suite C|Frogtown|East Carroll Parish|LA|78784|United States|-6|single family| +36219|AAAAAAAALHNIAAAA|372|Lincoln Locust|Blvd|Suite C|Oakwood|Morgan County|IL|60169|United States|-6|apartment| +36220|AAAAAAAAMHNIAAAA|708|Second |Boulevard|Suite 400|Doyle|Houston County|GA|38434|United States|-5|apartment| +36221|AAAAAAAANHNIAAAA|85|Central Lakeview|Pkwy|Suite 270|Bridgeport|Henderson County|KY|45817|United States|-6|apartment| +36222|AAAAAAAAOHNIAAAA|972|Madison |Way|Suite 280|Franklin|Okanogan County|WA|99101|United States|-8|apartment| +36223|AAAAAAAAPHNIAAAA|188|Chestnut Chestnut|RD|Suite M|Edgewood|Marion County|OR|90069|United States|-8|condo| +36224|AAAAAAAAAINIAAAA|653|3rd |Drive|Suite K|Riverdale|Franklin County|PA|19391|United States|-5|single family| +36225|AAAAAAAABINIAAAA|155|West |Street|Suite V|Ashland|Lancaster County|NE|64244|United States|-7|condo| +36226|AAAAAAAACINIAAAA|588|Woodland 5th|Way|Suite O|Waterloo|Chesterfield County|SC|21675|United States|-5|apartment| +36227|AAAAAAAADINIAAAA|914|Elm Main|Parkway|Suite 330|Cedar Grove|Stanton County|NE|60411|United States|-7|condo| +36228|AAAAAAAAEINIAAAA|808|Johnson Wilson|Parkway|Suite 20|Howell|Jackson Parish|LA|74854|United States|-6|single family| +36229|AAAAAAAAFINIAAAA|869|13th |Parkway|Suite C|Greenfield|Morgan County|IN|45038|United States|-5|single family| +36230|AAAAAAAAGINIAAAA|820|2nd |Drive|Suite I|Clinton|Androscoggin County|ME|08822|United States|-5|condo| +36231|AAAAAAAAHINIAAAA|114|Cherry Maple|Street|Suite S|Oak Grove|Jasper County|TX|78370|United States|-6|condo| +36232|AAAAAAAAIINIAAAA|3|1st Poplar|Circle|Suite 230|Spring Hill|Autauga County|AL|36787|United States|-6|single family| +36233|AAAAAAAAJINIAAAA|231|Cedar Eigth|Court|Suite 400|Mount Vernon|Weakley County|TN|38482|United States|-6|apartment| +36234|AAAAAAAAKINIAAAA|801|West Ridge|RD|Suite O|Brownsville|Arthur County|NE|69310|United States|-6|apartment| +36235|AAAAAAAALINIAAAA|306|Cedar Locust|Cir.|Suite M|Greenfield|Allegheny County|PA|15038|United States|-5|single family| +36236|AAAAAAAAMINIAAAA|670|Lincoln |Street|Suite 10|Farmington|Philadelphia County|PA|19145|United States|-5|condo| +36237|AAAAAAAANINIAAAA|146|Oak |Parkway|Suite 430|Maywood|Nodaway County|MO|65681|United States|-6|single family| +36238|AAAAAAAAOINIAAAA|639|Main 15th|Dr.|Suite O|Mount Pleasant|Muscogee County|GA|31933|United States|-5|condo| +36239|AAAAAAAAPINIAAAA|40|Pine Ash|Drive|Suite 40|New Hope|Perry County|OH|49431|United States|-5|apartment| +36240|AAAAAAAAAJNIAAAA|383|2nd |Ct.|Suite E|Bethel|DeSoto County|MS|55281|United States|-6|condo| +36241|AAAAAAAABJNIAAAA|779|Park |Lane|Suite U|Georgetown|Stafford County|VA|27057|United States|-5|single family| +36242|AAAAAAAACJNIAAAA|48|Franklin |Way|Suite G|Macedonia|Walton County|FL|31087|United States|-5|single family| +36243|AAAAAAAADJNIAAAA|149|Hickory Jackson|Street|Suite P|Arthur|Allen County|KS|65965|United States|-6|condo| +36244|AAAAAAAAEJNIAAAA|584|Oak Maple|Parkway|Suite 320|Gladstone|Haralson County|GA|30894|United States|-5|single family| +36245|AAAAAAAAFJNIAAAA|643|Railroad |Cir.|Suite 140|Lincoln|McMinn County|TN|31289|United States|-6|condo| +36246|AAAAAAAAGJNIAAAA|49|Smith |Boulevard|Suite Q|Springfield|Mayes County|OK|79303|United States|-6|single family| +36247|AAAAAAAAHJNIAAAA|600|Spring Hillcrest|Wy|Suite 290|Wilson|Pointe Coupee Parish|LA|76971|United States|-6|single family| +36248|AAAAAAAAIJNIAAAA|496|4th |Cir.|Suite I|Midway|Montgomery County|PA|11904|United States|-5|single family| +36249|AAAAAAAAJJNIAAAA|167|Ash |Court|Suite 260|Stringtown|Clinton County|MI|40162|United States|-5|single family| +36250|AAAAAAAAKJNIAAAA|925|8th Lincoln|Wy|Suite X|Jamestown|Johnson County|WY|86867|United States|-7|condo| +36251|AAAAAAAALJNIAAAA|997|Spruce |Pkwy|Suite U|Fairfield|Parmer County|TX|76192|United States|-6|condo| +36252|AAAAAAAAMJNIAAAA|926|Highland 1st|Avenue|Suite I|Concord|Fayette County|IL|64107|United States|-6|apartment| +36253|AAAAAAAANJNIAAAA|500|Laurel First|Ln|Suite 180|Arlington|Sandoval County|NM|86557|United States|-7|apartment| +36254|AAAAAAAAOJNIAAAA|390|Ridge East|Drive|Suite 30|Richardson|Madison County|KY|47687|United States|-5|condo| +36255|AAAAAAAAPJNIAAAA||Mill 2nd|Ct.|Suite B|Woodville||TX|74289||-6|apartment| +36256|AAAAAAAAAKNIAAAA|209|College |Avenue|Suite 320|Belleville|Williamson County|TN|32924|United States|-5|condo| +36257|AAAAAAAABKNIAAAA|243|View Seventh|Avenue|Suite 220|Plainview|Hickman County|TN|33683|United States|-5|single family| +36258|AAAAAAAACKNIAAAA|147|Pine Williams|Ave|Suite G|Belmont|Buchanan County|IA|50191|United States|-6|apartment| +36259|AAAAAAAADKNIAAAA|54|Ridge |Cir.|Suite L|Pine Grove|Montgomery County|VA|24593|United States|-5|apartment| +36260|AAAAAAAAEKNIAAAA|567|3rd |Avenue|Suite 450|Stringtown|Chase County|KS|60162|United States|-6|condo| +36261|AAAAAAAAFKNIAAAA|140|Smith Main|Drive|Suite 100|Concord|Mecklenburg County|NC|24107|United States|-5|apartment| +36262|AAAAAAAAGKNIAAAA|943|Wilson |Circle|Suite N|Chestnut Ridge|Milwaukee County|WI|57334|United States|-6|condo| +36263|AAAAAAAAHKNIAAAA|250|Lakeview |Boulevard|Suite P|Highland|Wayne County|KY|49454|United States|-5|single family| +36264|AAAAAAAAIKNIAAAA|742|Washington Highland|Circle|Suite 360|Hamilton|Williamsburg County|SC|22808|United States|-5|condo| +36265|AAAAAAAAJKNIAAAA|806|Main |Boulevard|Suite 460|Red Hill|Morris County|TX|74338|United States|-6|apartment| +36266|AAAAAAAAKKNIAAAA|574|Park |Cir.|Suite 230|Florence|Fairfield County|SC|23394|United States|-5|apartment| +36267|AAAAAAAALKNIAAAA|2|Railroad |Lane|Suite 440|Valley View|Socorro County|NM|85124|United States|-7|single family| +36268|AAAAAAAAMKNIAAAA|524|North |Avenue|Suite 460|Stringtown|Cabell County|WV|20162|United States|-5|apartment| +36269|AAAAAAAANKNIAAAA|174|Mill |Lane|Suite B|Sunnyside|McLeod County|MN|51952|United States|-6|condo| +36270|AAAAAAAAOKNIAAAA|660|1st |RD|Suite U|Newport|Adams County|CO|81521|United States|-7|apartment| +36271|AAAAAAAAPKNIAAAA|209|Jefferson Franklin|Drive|Suite T|Walnut Grove|Allegheny County|PA|17752|United States|-5|single family| +36272|AAAAAAAAALNIAAAA|973|Birch |Pkwy|Suite F|Plainview|Parmer County|TX|73683|United States|-6|single family| +36273|AAAAAAAABLNIAAAA|||||||OK|76098|United States||apartment| +36274|AAAAAAAACLNIAAAA|394|14th Woodland|Street|Suite 380|Springfield|Douglas County|WA|99303|United States|-8|condo| +36275|AAAAAAAADLNIAAAA|715|7th Park|Drive|Suite 200|Concord|Gloucester County|NJ|04707|United States|-5|apartment| +36276|AAAAAAAAELNIAAAA|559|Thirteenth |Blvd|Suite K|Nottingham|Karnes County|TX|74074|United States|-6|apartment| +36277|AAAAAAAAFLNIAAAA|599|2nd |Pkwy|Suite 330|Woodbury|Faribault County|MN|54489|United States|-6|condo| +36278|AAAAAAAAGLNIAAAA|695|4th |Dr.|Suite B|Woodlawn|Wayne County|OH|44098|United States|-5|condo| +36279|AAAAAAAAHLNIAAAA|829|2nd Jefferson|Boulevard|Suite 20|Shiloh|Gulf County|FL|39275|United States|-5|apartment| +36280|AAAAAAAAILNIAAAA|8|Maple South|Court|Suite 30|Summit|Plymouth County|MA|01099|United States|-5|condo| +36281|AAAAAAAAJLNIAAAA|213|View |Cir.|Suite H|Lakewood|Maricopa County|AZ|88877|United States|-7|single family| +36282|AAAAAAAAKLNIAAAA|462|Oak |Dr.|Suite S|Oakwood|Okeechobee County|FL|30169|United States|-5|single family| +36283|AAAAAAAALLNIAAAA|511|Fourth |ST|Suite 90|Clifton|Marquette County|MI|48014|United States|-5|condo| +36284|AAAAAAAAMLNIAAAA|340|Hill |Dr.|Suite T|Five Forks|Alfalfa County|OK|72293|United States|-6|single family| +36285|AAAAAAAANLNIAAAA||Green ||Suite 220|Red Hill|Loudoun County|||United States||single family| +36286|AAAAAAAAOLNIAAAA|897|Laurel |Street|Suite J|Farmington|Lafayette County|MO|69145|United States|-6|apartment| +36287|AAAAAAAAPLNIAAAA|224|4th Park|Lane|Suite 90|Wilton|Jefferson County|PA|16997|United States|-5|apartment| +36288|AAAAAAAAAMNIAAAA|975|4th |Way|Suite 260|Cedar Grove|Walthall County|MS|50411|United States|-6|single family| +36289|AAAAAAAABMNIAAAA|377|13th |Drive|Suite N|Shady Grove|Dimmit County|TX|72812|United States|-6|apartment| +36290|AAAAAAAACMNIAAAA|729|Maple 2nd|Blvd|Suite 190|Buena Vista|Hampshire County|MA|06352|United States|-5|single family| +36291|AAAAAAAADMNIAAAA|122|Hill |Wy|Suite P|Highland Park|Montgomery County|KS|66534|United States|-6|condo| +36292|AAAAAAAAEMNIAAAA|224|Highland Washington|Lane|Suite Y|Ferguson|Cameron Parish|LA|71821|United States|-6|condo| +36293|AAAAAAAAFMNIAAAA|||Blvd|Suite B|Fairfield||GA|36192|United States|-5|| +36294|AAAAAAAAGMNIAAAA|251|Fourth |RD|Suite D|Florence|Fayette County|WV|23394|United States|-5|condo| +36295|AAAAAAAAHMNIAAAA|301|Poplar |Ct.|Suite H|Harmony|Maury County|TN|35804|United States|-6|condo| +36296|AAAAAAAAIMNIAAAA|329|10th Spring|Ln|Suite T|Riverview|Sherman County|KS|69003|United States|-6|condo| +36297|AAAAAAAAJMNIAAAA|722|Railroad |Ave|Suite T|Mount Zion|Grant County|OR|98054|United States|-8|condo| +36298|AAAAAAAAKMNIAAAA|14|Lake Second|Ave|Suite C|Sunnyside|Thomas County|NE|61952|United States|-7|apartment| +36299|AAAAAAAALMNIAAAA|775|2nd |Boulevard|Suite N|Springdale|Cameron County|PA|18883|United States|-5|apartment| +36300|AAAAAAAAMMNIAAAA|335|Spruce First|Dr.|Suite N|Georgetown|Osceola County|IA|57057|United States|-6|condo| +36301|AAAAAAAANMNIAAAA|735|7th Cedar|ST|Suite Y|Clinton|Crook County|OR|98222|United States|-8|apartment| +36302|AAAAAAAAOMNIAAAA|947|Washington |Circle|Suite 260|Woodland|Barnstable County|MA|05454|United States|-5|apartment| +36303|AAAAAAAAPMNIAAAA|496|Eigth |Ct.|Suite U|Lebanon|Milwaukee County|WI|52898|United States|-6|apartment| +36304|AAAAAAAAANNIAAAA|714|Spring 12th|Way|Suite N|Lebanon|Clark County|KY|42898|United States|-6|apartment| +36305|AAAAAAAABNNIAAAA|113|14th South|Circle|Suite 180|Jackson|Wood County|OH|49583|United States|-5|apartment| +36306|AAAAAAAACNNIAAAA|665|Walnut |Wy|Suite R|White Oak|Johnson County|IN|46668|United States|-5|condo| +36307|AAAAAAAADNNIAAAA|449|Willow 6th|Road|Suite E|Liberty|Columbia County|FL|33451|United States|-5|apartment| +36308|AAAAAAAAENNIAAAA|952|1st View|Lane|Suite 330|Oak Hill|Dallas County|AL|37838|United States|-6|apartment| +36309|AAAAAAAAFNNIAAAA|706|11th |Street|Suite G|Harmony|Massac County|IL|65804|United States|-6|single family| +36310|AAAAAAAAGNNIAAAA|418|River Williams|Ave|Suite M|Georgetown|Stafford County|KS|67057|United States|-6|apartment| +36311|AAAAAAAAHNNIAAAA|632|Hickory |Way|Suite T|Mount Vernon|Fort Bend County|TX|78482|United States|-6|condo| +36312|AAAAAAAAINNIAAAA|975|2nd Broadway|Avenue|Suite W|Hopewell|Calloway County|KY|40587|United States|-6|single family| +36313|AAAAAAAAJNNIAAAA|235|Fourth 12th|Circle|Suite 390|Glendale|Jim Hogg County|TX|73951|United States|-6|condo| +36314|AAAAAAAAKNNIAAAA|372|Woodland |Ave|Suite G|Mountain View|Wabash County|IL|64466|United States|-6|apartment| +36315|AAAAAAAALNNIAAAA|815|Lincoln |Dr.|Suite Q|Jerome|Walworth County|SD|59920|United States|-7|apartment| +36316|AAAAAAAAMNNIAAAA|334|View |Boulevard|Suite 330|Concord|Kenedy County|TX|74107|United States|-6|apartment| +36317|AAAAAAAANNNIAAAA|843|Walnut Sycamore|Wy|Suite Q|Spring Valley|Alleghany County|VA|26060|United States|-5|apartment| +36318|AAAAAAAAONNIAAAA|||||Georgetown||IA|||-6|| +36319|AAAAAAAAPNNIAAAA|16|Railroad |Court|Suite 330|Franklin|Twin Falls County|ID|89101|United States|-7|single family| +36320|AAAAAAAAAONIAAAA|182|Pine 8th|Circle|Suite F|Edgewood|Champaign County|OH|40069|United States|-5|condo| +36321|AAAAAAAABONIAAAA|272|Sunset 13th|Road|Suite R|Lakewood|Putnam County|FL|38877|United States|-5|single family| +36322|AAAAAAAACONIAAAA|418|Laurel Hickory|Court|Suite 300|Woodland|Chattahoochee County|GA|34854|United States|-5|apartment| +36323|AAAAAAAADONIAAAA|32|North |Street|Suite 40|Oak Ridge|Deuel County|SD|58371|United States|-6|apartment| +36324|AAAAAAAAEONIAAAA|882|Main Third|Court|Suite U|Valley View|Deaf Smith County|TX|75124|United States|-6|condo| +36325|AAAAAAAAFONIAAAA|258|Spring Adams|Boulevard|Suite U|Liberty|Garfield County|OK|73451|United States|-6|apartment| +36326|AAAAAAAAGONIAAAA|86|Fifth |Avenue|Suite 250|Stringtown|Uinta County|WY|80162|United States|-7|single family| +36327|AAAAAAAAHONIAAAA|776|Sixth North|Ct.|Suite 450|Crossroads|Aleutians West Census Area|AK|90534|United States|-9|single family| +36328|AAAAAAAAIONIAAAA|661|Locust 15th|Ave|Suite 70|Valley View|Mingo County|WV|25124|United States|-5|single family| +36329|AAAAAAAAJONIAAAA|658|Madison |Dr.|Suite 220|Oakland|Benewah County|ID|89843|United States|-7|single family| +36330|AAAAAAAAKONIAAAA|275|3rd |Street|Suite 200|Centerville|Major County|OK|70059|United States|-6|condo| +36331|AAAAAAAALONIAAAA|576|Park |Pkwy|Suite A|Fairfield|Coke County|TX|76192|United States|-6|single family| +36332|AAAAAAAAMONIAAAA|790|3rd Ash|Boulevard|Suite C|Lincoln|Cortland County|NY|11289|United States|-5|condo| +36333|AAAAAAAANONIAAAA|591|Spring |Ct.|Suite 150|Wilton|Chippewa County|WI|56997|United States|-6|condo| +36334|AAAAAAAAOONIAAAA|145|Center Forest|Ave|Suite 490|Hamilton|Orleans County|NY|12808|United States|-5|single family| +36335|AAAAAAAAPONIAAAA|508|Oak Walnut|Court|Suite M|Salem|Greenup County|KY|48048|United States|-6|single family| +36336|AAAAAAAAAPNIAAAA|979|1st River|Court|Suite 220|Macedonia|Hancock County|IL|61087|United States|-6|apartment| +36337|AAAAAAAABPNIAAAA|102|River 6th|Road|Suite L|Bunker Hill|Lake County|OR|90150|United States|-8|single family| +36338|AAAAAAAACPNIAAAA|485|North 11th|Ave|Suite 320|Oak Grove|Beaufort County|NC|28370|United States|-5|apartment| +36339|AAAAAAAADPNIAAAA|605|Broadway Hill|Ct.|Suite 330|Woodbury|Grant County|NM|84489|United States|-7|single family| +36340|AAAAAAAAEPNIAAAA|556|7th Main|Blvd|Suite V|Antioch|Stafford County|VA|28605|United States|-5|condo| +36341|AAAAAAAAFPNIAAAA|819|Wilson |Pkwy|Suite W|Greenfield|Leslie County|KY|45038|United States|-5|single family| +36342|AAAAAAAAGPNIAAAA|402|Forest Fourth|Ave|Suite V|Bridgeport|Morris County|NJ|06417|United States|-5|apartment| +36343|AAAAAAAAHPNIAAAA|583|1st 10th|RD|Suite D|Green Acres|Morgan County|IN|47683|United States|-5|apartment| +36344|AAAAAAAAIPNIAAAA|72|Ridge West|Parkway|Suite H|Mount Vernon|Rockingham County|VA|28482|United States|-5|condo| +36345|AAAAAAAAJPNIAAAA|952|Second Spring|Blvd|Suite Y|Lakeside|Scott County|VA|29532|United States|-5|condo| +36346|AAAAAAAAKPNIAAAA|211|Broadway Park|Circle|Suite V|Florence|Hardin County|OH|43394|United States|-5|single family| +36347|AAAAAAAALPNIAAAA|959|Jefferson 5th|Boulevard|Suite 390|Amity|Howard County|NE|60766|United States|-7|condo| +36348|AAAAAAAAMPNIAAAA|308|13th |Cir.|Suite G|Forestville|Rutherford County|TN|33027|United States|-6|single family| +36349|AAAAAAAANPNIAAAA|400|Mill 2nd|Ct.|Suite 430|Woodbine|Warren County|VA|24253|United States|-5|condo| +36350|AAAAAAAAOPNIAAAA|703|Lake North|Cir.|Suite A|Sumner|Russell County|AL|30519|United States|-6|single family| +36351|AAAAAAAAPPNIAAAA|78|Oak 5th|ST|Suite 70|Lebanon|Orange County|NC|22898|United States|-5|condo| +36352|AAAAAAAAAAOIAAAA|795|Ash Central|Wy|Suite W|Bridgeport|Scurry County|TX|75817|United States|-6|apartment| +36353|AAAAAAAABAOIAAAA|389|Dogwood |Street|Suite 140|Sunset Beach|Sweet Grass County|MT|69611|United States|-7|apartment| +36354|AAAAAAAACAOIAAAA|827|Spring |Circle|Suite H|Pleasant Valley|Jerauld County|SD|52477|United States|-7|apartment| +36355|AAAAAAAADAOIAAAA|468|Spruce |Ct.|Suite V|Red Hill|Chase County|NE|64338|United States|-6|apartment| +36356|AAAAAAAAEAOIAAAA|56|Oak |Ct.|Suite 330|Farmington|Hamilton County|KS|69145|United States|-6|apartment| +36357|AAAAAAAAFAOIAAAA|994|Sycamore |ST|Suite 490|Marion|Madison County|FL|30399|United States|-5|condo| +36358|AAAAAAAAGAOIAAAA|44|Park Pine|Ave|Suite R|White Oak|Boise County|ID|86668|United States|-7|apartment| +36359|AAAAAAAAHAOIAAAA|781|Laurel Hickory|Parkway|Suite J|Lebanon|Penobscot County|ME|03498|United States|-5|apartment| +36360|AAAAAAAAIAOIAAAA|222|Oak 10th|Court|Suite N|Edgewood|Preston County|WV|20069|United States|-5|apartment| +36361|AAAAAAAAJAOIAAAA|481|Willow 1st|Wy|Suite 430|Lebanon|Minnehaha County|SD|52898|United States|-7|condo| +36362|AAAAAAAAKAOIAAAA|691|West |ST|Suite 260|Newport|Fergus County|MT|61521|United States|-7|apartment| +36363|AAAAAAAALAOIAAAA|179|Eigth Miller|Ave|Suite H|Woodland|Lincoln County|CO|84854|United States|-7|condo| +36364|AAAAAAAAMAOIAAAA|549|Smith Sycamore|Parkway|Suite 410|Oak Grove|Dawson County|TX|78370|United States|-6|condo| +36365|AAAAAAAANAOIAAAA|751|First 12th|Blvd|Suite 60|Union Hill|Harrison County|MO|67746|United States|-6|single family| +36366|AAAAAAAAOAOIAAAA|704|Hickory Sunset|Avenue|Suite 140|Green Acres|Marshall County|KS|67683|United States|-6|single family| +36367|AAAAAAAAPAOIAAAA|729|Poplar |Dr.|Suite 60|Red Hill|Stark County|ND|54338|United States|-6|apartment| +36368|AAAAAAAAABOIAAAA|604|Miller Main|Blvd|Suite P|Waterloo|Big Horn County|WY|81675|United States|-7|apartment| +36369|AAAAAAAABBOIAAAA|898|7th |Dr.|Suite 60|Belmont|Ketchikan Gateway Borough|AK|90191|United States|-9|apartment| +36370|AAAAAAAACBOIAAAA|885|Spruce Jefferson|RD|Suite 70|Pleasant Grove|Campbell County|KY|44136|United States|-6|single family| +36371|AAAAAAAADBOIAAAA|965|10th Eigth|Cir.|Suite K|Spring Hill|Smith County|MS|56787|United States|-6|apartment| +36372|AAAAAAAAEBOIAAAA|331|Hickory |ST|Suite 80|Freedom|Calhoun County|TX|71651|United States|-6|single family| +36373|AAAAAAAAFBOIAAAA|830|View |Ave|Suite 270|Sunnyside|Meade County|SD|51952|United States|-7|apartment| +36374|AAAAAAAAGBOIAAAA|895|Poplar |Blvd|Suite 190|Oak Hill|Penobscot County|ME|08438|United States|-5|condo| +36375|AAAAAAAAHBOIAAAA|501|15th Pine|Ave|Suite Y|Valley View|Johnson County|NE|65124|United States|-7|apartment| +36376|AAAAAAAAIBOIAAAA|541|3rd |Blvd|Suite 0|Lakeside|Russell County|KS|69532|United States|-6|single family| +36377|AAAAAAAAJBOIAAAA|144|Washington |Court|Suite D|Enterprise|Flathead County|MT|61757|United States|-7|condo| +36378|AAAAAAAAKBOIAAAA|333|Jackson View|Avenue|Suite P|Crossroads|Edgefield County|SC|20534|United States|-5|apartment| +36379|AAAAAAAALBOIAAAA|972|Smith 6th|Circle|Suite 0|Mount Pleasant|Barron County|WI|51933|United States|-6|single family| +36380|AAAAAAAAMBOIAAAA|449|Madison |Avenue|Suite B|Valley View|Henry County|IN|45124|United States|-5|single family| +36381|AAAAAAAANBOIAAAA|5|Franklin |Circle|Suite K|White Oak|Knox County|OH|46668|United States|-5|condo| +36382|AAAAAAAAOBOIAAAA|797|Adams |Road|Suite 210|Riverview|Hamilton County|FL|39003|United States|-5|condo| +36383|AAAAAAAAPBOIAAAA|619|South |Cir.|Suite 490|Stratford|Rock County|NE|66668|United States|-7|apartment| +36384|AAAAAAAAACOIAAAA|323|West |Circle|Suite 230|Farmington|Wilbarger County|TX|79145|United States|-6|single family| +36385|AAAAAAAABCOIAAAA|454|Mill Wilson|Drive|Suite 340|Salem|Worcester County|MD|28048|United States|-5|single family| +36386|AAAAAAAACCOIAAAA|346|4th Hill|Ave|Suite 450|Summit|Garfield County|CO|80499|United States|-7|apartment| +36387|AAAAAAAADCOIAAAA|115|3rd Oak|Avenue|Suite V|Unionville|Washington County|ME|02311|United States|-5|condo| +36388|AAAAAAAAECOIAAAA|316|Chestnut |Drive|Suite L|Springfield|Douglas County|GA|39303|United States|-5|apartment| +36389|AAAAAAAAFCOIAAAA|43|Central |Road|Suite X|Harmony|Brazoria County|TX|75804|United States|-6|condo| +36390|AAAAAAAAGCOIAAAA|694|10th |Dr.|Suite 120|Buena Vista|Poweshiek County|IA|55752|United States|-6|apartment| +36391|AAAAAAAAHCOIAAAA|678|Lakeview |Court|Suite X|Dewey|Cherokee County|GA|31160|United States|-5|single family| +36392|AAAAAAAAICOIAAAA|784|||Suite M|Mount Pleasant|||71933|||| +36393|AAAAAAAAJCOIAAAA|181|2nd East|Ave|Suite D|Farmington|Wilson County|TN|39145|United States|-5|apartment| +36394|AAAAAAAAKCOIAAAA|6|Spring Park|Court|Suite 60|Florence|Miami County|KS|63394|United States|-6|single family| +36395|AAAAAAAALCOIAAAA||Ridge |Dr.|Suite 10|||MN||||condo| +36396|AAAAAAAAMCOIAAAA|498|Walnut |Wy|Suite 430|Valley View|Pittsburg County|OK|75124|United States|-6|condo| +36397|AAAAAAAANCOIAAAA|786|Sixth Miller|ST|Suite 440|Marion|Newton County|MS|50399|United States|-6|single family| +36398|AAAAAAAAOCOIAAAA|194|Jackson Lincoln|Dr.|Suite 460|Wilson|Sevier County|TN|36971|United States|-6|apartment| +36399|AAAAAAAAPCOIAAAA|131|Main |Ave|Suite 300|Greenfield|Arthur County|NE|65038|United States|-6|single family| +36400|AAAAAAAAADOIAAAA||Church |Ave|||Alleghany County|VA|20162|United States||| +36401|AAAAAAAABDOIAAAA|868|14th |Circle|Suite E|Perry|Hancock County|OH|41830|United States|-5|condo| +36402|AAAAAAAACDOIAAAA|532|Franklin Maple|Wy|Suite 180|Spring Valley|Boone County|IL|66060|United States|-6|single family| +36403|AAAAAAAADDOIAAAA|626|Maple Fourth|Ln|Suite 300|Farmington|La Plata County|CO|89145|United States|-7|apartment| +36404|AAAAAAAAEDOIAAAA|964|7th |Lane|Suite Q|Mount Pleasant|Warren County|IA|51933|United States|-6|condo| +36405|AAAAAAAAFDOIAAAA||Spruce Maple|Road||Mount Olive|Hitchcock County|NE||United States|-6|apartment| +36406|AAAAAAAAGDOIAAAA|350|Oak |Boulevard|Suite 220|Royal|Carroll County|AR|75819|United States|-6|condo| +36407|AAAAAAAAHDOIAAAA|423|North |Dr.|Suite 270|Cedar Grove|Gunnison County|CO|80411|United States|-7|apartment| +36408|AAAAAAAAIDOIAAAA|687|4th Birch|Parkway|Suite 330|Pierce|Franklin Parish|LA|73360|United States|-6|condo| +36409|AAAAAAAAJDOIAAAA|502|Twelfth |Boulevard|Suite U|Unionville|Marin County|CA|91711|United States|-8|single family| +36410|AAAAAAAAKDOIAAAA|922|College |Lane|Suite F|Bethel|Jefferson County|WA|95281|United States|-8|apartment| +36411|AAAAAAAALDOIAAAA|769|Smith |Wy|Suite 260|Brownsville|Christian County|MO|69310|United States|-6|condo| +36412|AAAAAAAAMDOIAAAA||||||Bosque County|||United States|-6|apartment| +36413|AAAAAAAANDOIAAAA|647|7th Highland|Drive|Suite 260|Mount Olive|Peach County|GA|38059|United States|-5|single family| +36414|AAAAAAAAODOIAAAA|394|River |Ave|Suite P|Union|Wade Hampton Census Area|AK|98721|United States|-9|condo| +36415|AAAAAAAAPDOIAAAA|713|Williams |ST|Suite 380|Lakewood|Greene County|AR|78877|United States|-6|single family| +36416|AAAAAAAAAEOIAAAA|767|9th Pine|Ln|Suite N|Fairview|Johnson County|MO|65709|United States|-6|apartment| +36417|AAAAAAAABEOIAAAA||||||||59584|||single family| +36418|AAAAAAAACEOIAAAA|298|2nd 11th|Road|Suite 470|Crossroads|Ben Hill County|GA|30534|United States|-5|condo| +36419|AAAAAAAADEOIAAAA|686|Lake Ridge|Ln|Suite T|Union Hill|Giles County|TN|37746|United States|-5|apartment| +36420|AAAAAAAAEEOIAAAA|307|Pine |ST|Suite P|Hamilton|Columbia County|NY|12808|United States|-5|single family| +36421|AAAAAAAAFEOIAAAA|575|Laurel Valley|ST|Suite 180|Forest Hills|Crow Wing County|MN|59237|United States|-6|condo| +36422|AAAAAAAAGEOIAAAA|443|Adams |Pkwy|Suite C|Clinton|York County|PA|18222|United States|-5|single family| +36423|AAAAAAAAHEOIAAAA|521|Broadway |Dr.|Suite 420|Jamestown|Nye County|NV|86867|United States|-8|single family| +36424|AAAAAAAAIEOIAAAA|376|Forest Lake|Boulevard|Suite I|Greenville|Jefferson County|KY|41387|United States|-6|single family| +36425|AAAAAAAAJEOIAAAA|482|Sycamore |Blvd|Suite K|Pleasant Grove|Lancaster County|VA|24136|United States|-5|single family| +36426|AAAAAAAAKEOIAAAA|799|Lakeview |Dr.|Suite 230|Oak Ridge|Howard County|TX|78371|United States|-6|condo| +36427|AAAAAAAALEOIAAAA||Willow ||||Butler County||||-6|| +36428|AAAAAAAAMEOIAAAA|407|Park Meadow|Wy|Suite G|Wyoming|White Pine County|NV|80216|United States|-8|single family| +36429|AAAAAAAANEOIAAAA|931|Davis Smith|Parkway|Suite F|Hamilton|Jasper County|IN|42808|United States|-5|apartment| +36430|AAAAAAAAOEOIAAAA|535|4th |Lane|Suite 40|Wilson|Thurston County|NE|66971|United States|-7|single family| +36431|AAAAAAAAPEOIAAAA|15|Hickory College|Ln|Suite 280|Friendship|El Dorado County|CA|94536|United States|-8|single family| +36432|AAAAAAAAAFOIAAAA|499|4th Wilson|Avenue|Suite 440|Morris|Crawford County|MI|46696|United States|-5|single family| +36433|AAAAAAAABFOIAAAA|604|Hillcrest |Dr.|Suite A|Walnut|Berkeley County|SC|26245|United States|-5|apartment| +36434|AAAAAAAACFOIAAAA|61|11th |Cir.|Suite G|Franklin|Wayne County|WV|29101|United States|-5|single family| +36435|AAAAAAAADFOIAAAA|611|4th Park|RD|Suite E|Deerfield|Jones County|NC|29840|United States|-5|condo| +36436|AAAAAAAAEFOIAAAA|781|Highland Dogwood|Road|Suite L|Greenville|Ramsey County|MN|51387|United States|-6|apartment| +36437|AAAAAAAAFFOIAAAA|397|Cedar |Ct.|Suite 180|Highland|Lane County|KS|69454|United States|-6|single family| +36438|AAAAAAAAGFOIAAAA|186|8th Wilson|ST|Suite A|Greenwood|Jones County|SD|58828|United States|-7|condo| +36439|AAAAAAAAHFOIAAAA|949|Green Green|Avenue|Suite 260|Owens|Boone County|IL|62324|United States|-6|single family| +36440|AAAAAAAAIFOIAAAA|555|Smith |ST|Suite 90|Maple Grove|Bonneville County|ID|88252|United States|-7|single family| +36441|AAAAAAAAJFOIAAAA|287|Cedar |Circle|Suite O|Lebanon|Saratoga County|NY|12898|United States|-5|apartment| +36442|AAAAAAAAKFOIAAAA|929|Hill |Drive|Suite 420|Mount Pleasant|Bland County|VA|21933|United States|-5|condo| +36443|AAAAAAAALFOIAAAA|709|Sycamore Franklin|Boulevard|Suite J|Shelby|Boone County|IL|66575|United States|-6|single family| +36444|AAAAAAAAMFOIAAAA|438|Dogwood |Wy|Suite 460|Wayland|Piatt County|IL|65115|United States|-6|apartment| +36445|AAAAAAAANFOIAAAA|568|Birch |Parkway|Suite F|Oakwood|Cortland County|NY|10169|United States|-5|apartment| +36446|AAAAAAAAOFOIAAAA|794|Sunset Lincoln|Pkwy|Suite 480|Greenfield|Pitt County|NC|25038|United States|-5|apartment| +36447|AAAAAAAAPFOIAAAA|877|West |Boulevard|Suite D|Shiloh|Alachua County|FL|39275|United States|-5|condo| +36448|AAAAAAAAAGOIAAAA|306|10th Washington|ST|Suite E|Spring Hill|Barren County|KY|46787|United States|-6|single family| +36449|AAAAAAAABGOIAAAA|939|9th 5th|Street|Suite 340|Wilson|Lamar County|MS|56971|United States|-6|single family| +36450|AAAAAAAACGOIAAAA|22|Main Forest|Drive|Suite V|Saint Johns|Tarrant County|TX|75717|United States|-6|single family| +36451|AAAAAAAADGOIAAAA|967|Lake 1st|Pkwy|Suite 350|Riverview|Fauquier County|VA|29003|United States|-5|apartment| +36452|AAAAAAAAEGOIAAAA|352|View |Wy|Suite 30|Shiloh|Wheeler County|NE|69275|United States|-7|single family| +36453|AAAAAAAAFGOIAAAA|245|Sunset |Boulevard|Suite 150|Ashland|Chaffee County|CO|84244|United States|-7|apartment| +36454|AAAAAAAAGGOIAAAA|770|Broadway Pine|Pkwy|Suite 90|Macedonia|Nuckolls County|NE|61087|United States|-7|single family| +36455|AAAAAAAAHGOIAAAA|977|Third Williams|Drive|Suite H|Friendship|Walworth County|SD|54536|United States|-7|apartment| +36456|AAAAAAAAIGOIAAAA|969|Oak College|Boulevard|Suite 150|Woodland Park|Cleveland County|OK|71934|United States|-6|condo| +36457|AAAAAAAAJGOIAAAA|65|Spring |ST|Suite 180|Riverside|Union County|OH|49231|United States|-5|apartment| +36458|AAAAAAAAKGOIAAAA|55|College |Pkwy|Suite F|Kingston|Shannon County|SD|54975|United States|-7|single family| +36459|AAAAAAAALGOIAAAA|674|South |Court|Suite 300|Greenfield|Spalding County|GA|35038|United States|-5|apartment| +36460|AAAAAAAAMGOIAAAA|569|Railroad |Court|Suite V|Carthage|Duplin County|NC|21529|United States|-5|single family| +36461|AAAAAAAANGOIAAAA|347|Cherry West|Court|Suite 190|Ashland|Bladen County|NC|24244|United States|-5|condo| +36462|AAAAAAAAOGOIAAAA|165|Cedar Miller|Court|Suite 250|Florence|Morgan County|WV|23394|United States|-5|apartment| +36463|AAAAAAAAPGOIAAAA|177|Mill |Ct.|Suite 450|Roberts|Steuben County|NY|10757|United States|-5|apartment| +36464|AAAAAAAAAHOIAAAA|90|Church |Wy|Suite 20|Jamestown|Fentress County|TN|36867|United States|-5|apartment| +36465|AAAAAAAABHOIAAAA|597|Wilson |Drive|Suite G|Lebanon|Lincoln County|NV|82898|United States|-8|single family| +36466|AAAAAAAACHOIAAAA|163|1st |Ave|Suite 380|Jamestown|Lyman County|SD|56867|United States|-7|single family| +36467|AAAAAAAADHOIAAAA|854|South |Parkway|Suite D|Deerfield|Red Lake County|MN|59840|United States|-6|condo| +36468|AAAAAAAAEHOIAAAA|238|3rd 4th|Court|Suite C|Kelly|Tama County|IA|52738|United States|-6|condo| +36469|AAAAAAAAFHOIAAAA|396|College Hillcrest|Avenue|Suite X|Sulphur Springs|Hardin County|IA|58354|United States|-6|single family| +36470|AAAAAAAAGHOIAAAA|525|Williams Johnson|RD|Suite 190|Pleasant Grove|Douglas County|GA|34136|United States|-5|single family| +36471|AAAAAAAAHHOIAAAA|18|Green View|Court|Suite J|Woodville|Butts County|GA|34289|United States|-5|single family| +36472|AAAAAAAAIHOIAAAA|362|East |Circle|Suite 30|Bridgeport|Carlisle County|KY|45817|United States|-6|condo| +36473|AAAAAAAAJHOIAAAA|292|6th |Way|Suite W|Midway|Calhoun County|TX|71904|United States|-6|apartment| +36474|AAAAAAAAKHOIAAAA|800|Highland Railroad|Lane|Suite 240|Summit|Whitman County|WA|90499|United States|-8|condo| +36475|AAAAAAAALHOIAAAA|||ST|Suite N|Trinity|Refugio County|||United States|-6|condo| +36476|AAAAAAAAMHOIAAAA|904|Chestnut Center|Ct.|Suite 450|Oakland|Stanley County|SD|59843|United States|-7|condo| +36477|AAAAAAAANHOIAAAA||Pine Williams|Cir.|Suite P|Leland|||||-5|condo| +36478|AAAAAAAAOHOIAAAA|38|Pine |Drive|Suite 60|Oakwood|Jefferson County|MO|60169|United States|-6|single family| +36479|AAAAAAAAPHOIAAAA|460|Railroad |Way|Suite 210|Lakeview|Union County|KY|48579|United States|-5|single family| +36480|AAAAAAAAAIOIAAAA|497|Oak |ST|Suite V|Wilson|Petersburg city|VA|26971|United States|-5|apartment| +36481|AAAAAAAABIOIAAAA|47|Cedar Elm|ST|Suite 10|Waterloo|Lincoln County|CO|81675|United States|-7|condo| +36482|AAAAAAAACIOIAAAA|122|Broadway Smith|Drive|Suite Q|Greenfield|Douglas County|MN|55038|United States|-6|condo| +36483|AAAAAAAADIOIAAAA|60|Elm 1st|RD|Suite 130|Waterloo|Simpson County|KY|41675|United States|-5|condo| +36484|AAAAAAAAEIOIAAAA|808|Williams |Way|Suite M|Star|Knox County|KY|40725|United States|-5|single family| +36485|AAAAAAAAFIOIAAAA|853|Hill Sunset|Boulevard|Suite L|Mount Zion|Essex County|VA|28054|United States|-5|single family| +36486|AAAAAAAAGIOIAAAA|323|Sixth |Parkway|Suite 180|Union Hill|Brooks County|TX|77746|United States|-6|condo| +36487|AAAAAAAAHIOIAAAA|742|Lake |Street|Suite V|Harvey|Glades County|FL|35858|United States|-5|condo| +36488|AAAAAAAAIIOIAAAA|608|Sunset |Blvd|Suite 80|Glendale|Atchison County|MO|63951|United States|-6|condo| +36489|AAAAAAAAJIOIAAAA|298|View Lee|Dr.|Suite 320|Wilson|McLean County|KY|46971|United States|-5|single family| +36490|AAAAAAAAKIOIAAAA|623|Fourth East|Wy|Suite Y|Maple Hill|Washington County|OH|48095|United States|-5|condo| +36491|AAAAAAAALIOIAAAA|3|11th |Cir.|Suite K|Friendship|Colquitt County|GA|34536|United States|-5|condo| +36492|AAAAAAAAMIOIAAAA|656|Fourth 4th|Circle|Suite W|Jackson|Stutsman County|ND|59583|United States|-6|single family| +36493|AAAAAAAANIOIAAAA|889|9th |Ave|Suite S|Highland Park|Madison County|TX|76534|United States|-6|single family| +36494|AAAAAAAAOIOIAAAA|991|Hillcrest |Parkway|Suite R|Union|Madera County|CA|98721|United States|-8|condo| +36495|AAAAAAAAPIOIAAAA|915|4th Seventh|RD|Suite S|Greenfield|Wood County|WI|55038|United States|-6|condo| +36496|AAAAAAAAAJOIAAAA|354|Washington Maple|Ave|Suite 10|Highland|Grand Traverse County|MI|49454|United States|-5|single family| +36497|AAAAAAAABJOIAAAA|186|Park |Street|Suite G|Shiloh|Riverside County|CA|99275|United States|-8|condo| +36498|AAAAAAAACJOIAAAA|106|Willow |Road|Suite 190|Mount Vernon|Gallatin County|KY|48482|United States|-6|apartment| +36499|AAAAAAAADJOIAAAA|713|15th |RD|Suite H|Edgewood|Hendry County|FL|30069|United States|-5|apartment| +36500|AAAAAAAAEJOIAAAA|805|15th |Drive|Suite 460|Union Hill|Delta County|MI|47746|United States|-5|single family| +36501|AAAAAAAAFJOIAAAA|74||||Ashland|Oklahoma County|||||condo| +36502|AAAAAAAAGJOIAAAA|867|College Cedar|Drive|Suite 170|Lakewood|Jefferson County|AR|78877|United States|-6|condo| +36503|AAAAAAAAHJOIAAAA|350|2nd |Pkwy|Suite 170|Jackson|Jackson County|AL|39583|United States|-6|apartment| +36504|AAAAAAAAIJOIAAAA|935||Pkwy||Whitney|Sweet Grass County|MT||United States|-7|| +36505|AAAAAAAAJJOIAAAA|681|1st Johnson|Ct.|Suite F|Union Hill|Cleveland County|OK|77746|United States|-6|apartment| +36506|AAAAAAAAKJOIAAAA|404|Main |Lane|Suite E|Martinsville|Harrison County|KY|40419|United States|-6|condo| +36507|AAAAAAAALJOIAAAA|341|Washington Maple|Boulevard|Suite H|Shiloh|Baker County|FL|39275|United States|-5|single family| +36508|AAAAAAAAMJOIAAAA|354|3rd |Wy|Suite F|Belfast|Nottoway County|VA|20125|United States|-5|condo| +36509|AAAAAAAANJOIAAAA|28|Sycamore |Drive|Suite 220|Greenfield|Henderson County|KY|45038|United States|-6|single family| +36510|AAAAAAAAOJOIAAAA|571|11th Lakeview|Circle|Suite 90|Perkins|Johnson County|NE|61852|United States|-7|single family| +36511|AAAAAAAAPJOIAAAA|332|Elm |Pkwy|Suite 70|Oak Hill|Keya Paha County|NE|67838|United States|-7|single family| +36512|AAAAAAAAAKOIAAAA|745|Madison 12th|Drive|Suite 260|Texas|Ashland County|WI|53342|United States|-6|condo| +36513|AAAAAAAABKOIAAAA|728|Meadow Washington|Wy|Suite R|Woodland|Luzerne County|PA|14854|United States|-5|single family| +36514|AAAAAAAACKOIAAAA|124|||Suite 470|Cherry Valley|Rensselaer County|NY||United States|-5|| +36515|AAAAAAAADKOIAAAA|453|Cherry Willow|Court|Suite J|Chestnut Hill|Bibb County|GA|36801|United States|-5|condo| +36516|AAAAAAAAEKOIAAAA|2|Adams |Ave|Suite 60|Spring Valley|Phillips County|AR|76060|United States|-6|condo| +36517|AAAAAAAAFKOIAAAA|373|Dogwood 14th|Parkway|Suite 60|New Hope|Atkinson County|GA|39431|United States|-5|single family| +36518|AAAAAAAAGKOIAAAA|573|East |RD|Suite V|Mount Pleasant|Jasper County|MO|61933|United States|-6|condo| +36519|AAAAAAAAHKOIAAAA|22|Ninth |Lane|Suite 30|Colfax|Miami County|IN|42565|United States|-5|condo| +36520|AAAAAAAAIKOIAAAA|710|2nd |Wy|Suite W|Lakeview|Madison Parish|LA|78579|United States|-6|apartment| +36521|AAAAAAAAJKOIAAAA|898|Woodland Cherry|Cir.|Suite Q|Cedar Grove|Umatilla County|OR|90411|United States|-8|condo| +36522|AAAAAAAAKKOIAAAA|617|Main Locust|Avenue|Suite G|Walnut Grove|Gila County|AZ|87752|United States|-7|single family| +36523|AAAAAAAALKOIAAAA|467|Jefferson |Dr.|Suite 350|Moore|Hudspeth County|TX|74614|United States|-6|single family| +36524|AAAAAAAAMKOIAAAA|796|Elm |Court|Suite X|Centerville|Lorain County|OH|40059|United States|-5|condo| +36525|AAAAAAAANKOIAAAA|417|Madison |Wy|Suite M|Hopewell|Riley County|KS|60587|United States|-6|condo| +36526|AAAAAAAAOKOIAAAA|183|13th |Pkwy|Suite X|Bridgeport|Clay County|IA|55817|United States|-6|condo| +36527|AAAAAAAAPKOIAAAA|802|5th Dogwood|Pkwy|Suite A|Antioch|Hanson County|SD|58605|United States|-7|apartment| +36528|AAAAAAAAALOIAAAA|431|8th |Lane|Suite 240|Midway|Mille Lacs County|MN|51904|United States|-6|single family| +36529|AAAAAAAABLOIAAAA|154|Lake |Lane|Suite 460|Fairview|McIntosh County|OK|75709|United States|-6|condo| +36530|AAAAAAAACLOIAAAA|812|Park Fourth|RD|Suite O|Fairview|Letcher County|KY|45709|United States|-5|apartment| +36531|AAAAAAAADLOIAAAA|177|Maple 11th|Court|Suite 340|Riverview|King and Queen County|VA|29003|United States|-5|apartment| +36532|AAAAAAAAELOIAAAA|858|First Lakeview|Ct.|Suite 250|Glendale|Jefferson County|WI|53951|United States|-6|single family| +36533|AAAAAAAAFLOIAAAA|982|11th 12th|Pkwy|Suite 390|Spring Valley|Imperial County|CA|96060|United States|-8|condo| +36534|AAAAAAAAGLOIAAAA|20|North 11th|Road|Suite Y|Red Hill|Hamilton County|KS|64338|United States|-6|condo| +36535|AAAAAAAAHLOIAAAA|74|3rd Main|Ln|Suite E|Red Hill|Mifflin County|PA|14338|United States|-5|condo| +36536|AAAAAAAAILOIAAAA|22|Sunset |Pkwy|Suite Y|Greenwood|Tazewell County|VA|28828|United States|-5|single family| +36537|AAAAAAAAJLOIAAAA|7|Jackson |Lane|Suite U|Lakeview|Tipton County|TN|38579|United States|-6|single family| +36538|AAAAAAAAKLOIAAAA|304|Laurel |Pkwy|Suite 280|Mount Vernon|Sangamon County|IL|68482|United States|-6|apartment| +36539|AAAAAAAALLOIAAAA|322|Park Hickory|Boulevard|Suite N|Riverview|Pawnee County|KS|69003|United States|-6|condo| +36540|AAAAAAAAMLOIAAAA|352|Madison |Court|Suite 200|Edgewood|La Plata County|CO|80069|United States|-7|condo| +36541|AAAAAAAANLOIAAAA|227|Smith 14th|Ave|Suite 340|Forest Hills|Franklin County|IN|49237|United States|-5|single family| +36542|AAAAAAAAOLOIAAAA|486|First |Cir.|Suite 370|Providence|Logan County|WV|26614|United States|-5|single family| +36543|AAAAAAAAPLOIAAAA|735|11th |Avenue|Suite 250|Centerville|Trego County|KS|60059|United States|-6|condo| +36544|AAAAAAAAAMOIAAAA|209|2nd Maple|Wy|Suite H|Murphy|Charles City County|VA|22105|United States|-5|apartment| +36545|AAAAAAAABMOIAAAA|897|Poplar Ridge|Wy|Suite 160|Woodville|Reagan County|TX|74289|United States|-6|single family| +36546|AAAAAAAACMOIAAAA|950|10th |Dr.|Suite C|Little River|Allegany County|NY|10319|United States|-5|single family| +36547|AAAAAAAADMOIAAAA|491|Ash |Road|Suite A|Unionville|Trego County|KS|61711|United States|-6|apartment| +36548|AAAAAAAAEMOIAAAA|151|North Mill|Pkwy|Suite 140|Spring Valley|Berkeley County|WV|26060|United States|-5|single family| +36549|AAAAAAAAFMOIAAAA|714|College |RD|Suite 90|Georgetown|Linn County|MO|67057|United States|-6|single family| +36550|AAAAAAAAGMOIAAAA|322|Wilson |Boulevard|Suite M|Newport|Johnson County|KS|61521|United States|-6|apartment| +36551|AAAAAAAAHMOIAAAA|778|East |Drive|Suite 250|White Oak|Flagler County|FL|36668|United States|-5|single family| +36552|AAAAAAAAIMOIAAAA||Main |Ln|Suite 0||Mineral County||29681||-5|condo| +36553|AAAAAAAAJMOIAAAA|229|Lakeview |Drive|Suite 390|Hopewell|White County|TN|30587|United States|-6|apartment| +36554|AAAAAAAAKMOIAAAA|72|6th |Blvd|Suite X|Fairfield|San Miguel County|CO|86192|United States|-7|condo| +36555|AAAAAAAALMOIAAAA|487|Oak 1st|Avenue|Suite Y|Littleton|Glynn County|GA|36074|United States|-5|single family| +36556|AAAAAAAAMMOIAAAA|884|Johnson |Avenue|Suite N|Spring Hill|Henry County|TN|36787|United States|-5|apartment| +36557|AAAAAAAANMOIAAAA|58|Spring Chestnut|Court|Suite 210|Clifton|Knox County|IL|68014|United States|-6|condo| +36558|AAAAAAAAOMOIAAAA|60|College |Way|Suite 10|Concord|Martin County|TX|74107|United States|-6|condo| +36559|AAAAAAAAPMOIAAAA|499|Ash Dogwood|Street|Suite G|Stringtown|Wayne County|GA|30162|United States|-5|single family| +36560|AAAAAAAAANOIAAAA||||Suite E||Holt County|NE|69843|||single family| +36561|AAAAAAAABNOIAAAA|897|Pine Hillcrest|RD|Suite W|Marion|Bond County|IL|60399|United States|-6|condo| +36562|AAAAAAAACNOIAAAA|424|Birch Central|Dr.|Suite E|Summit|Marshall County|WV|20499|United States|-5|single family| +36563|AAAAAAAADNOIAAAA|539|Mill |Road|Suite L|Valley View|Bledsoe County|TN|35124|United States|-5|single family| +36564|AAAAAAAAENOIAAAA|68|8th |Pkwy|Suite 0|Columbia|Marion County|WV|23292|United States|-5|condo| +36565|AAAAAAAAFNOIAAAA|58|Dogwood Walnut|RD|Suite 220|Glenwood|Wayne County|MO|63511|United States|-6|condo| +36566|AAAAAAAAGNOIAAAA|87|Lincoln |Parkway|Suite E|Farmington|Ada County|ID|89145|United States|-7|single family| +36567|AAAAAAAAHNOIAAAA|78|7th Williams|Ct.|Suite W|Lakeview|Crawford County|IN|48579|United States|-5|single family| +36568|AAAAAAAAINOIAAAA|466|7th Cedar|Parkway|Suite 0|Friendship|Williams County|ND|54536|United States|-6|single family| +36569|AAAAAAAAJNOIAAAA|673|8th |Road|Suite K|Greenville|Colleton County|SC|21387|United States|-5|condo| +36570|AAAAAAAAKNOIAAAA|327|Church |Court|Suite 460|Ashland|Cherokee County|TX|74244|United States|-6|single family| +36571|AAAAAAAALNOIAAAA|351|Meadow |Cir.|Suite N|Mountain View|Bear Lake County|ID|84466|United States|-7|single family| +36572|AAAAAAAAMNOIAAAA|89|Valley 10th|Pkwy|Suite P|Newtown|Carson City|NV|81749|United States|-8|single family| +36573|AAAAAAAANNOIAAAA|338|Smith Main|Wy|Suite 60|Springfield|Macon County|TN|39303|United States|-6|apartment| +36574|AAAAAAAAONOIAAAA|23||Boulevard|Suite 450|Newport||NM|81521|||| +36575|AAAAAAAAPNOIAAAA|325|3rd |Street|Suite 330|Crossroads|Miami County|OH|40534|United States|-5|apartment| +36576|AAAAAAAAAOOIAAAA|||Ln|||Madison County|IN|43003|United States|-5|single family| +36577|AAAAAAAABOOIAAAA|324|Fifth |Lane|Suite A|Woodlawn|Craig County|OK|74098|United States|-6|apartment| +36578|AAAAAAAACOOIAAAA|198|West |Wy|Suite L|Mount Pleasant|Mercer County|WV|21933|United States|-5|apartment| +36579|AAAAAAAADOOIAAAA|765|Fourth |Street|Suite L|Kingston|Cherry County|NE|64975|United States|-6|single family| +36580|AAAAAAAAEOOIAAAA|453|9th Miller|Parkway|Suite Q|Jamestown|Taliaferro County|GA|36867|United States|-5|condo| +36581|AAAAAAAAFOOIAAAA|918|Spruce Adams|Circle|Suite N|Enterprise|Gila County|AZ|81757|United States|-7|single family| +36582|AAAAAAAAGOOIAAAA|980|Maple |Ct.|Suite 80|Deerfield|Marion County|IA|59840|United States|-6|single family| +36583|AAAAAAAAHOOIAAAA|774|Meadow First|Circle|Suite 90|Sulphur Springs|Wichita County|KS|68354|United States|-6|condo| +36584|AAAAAAAAIOOIAAAA|293|Park Washington|Ln|Suite 380|Murphy|Franklin County|FL|32105|United States|-5|condo| +36585|AAAAAAAAJOOIAAAA|733|Ridge Hill|Drive|Suite 220|Stringtown|Auglaize County|OH|40162|United States|-5|single family| +36586|AAAAAAAAKOOIAAAA|438|10th Hill|RD|Suite 180|Woodville|Vermilion Parish|LA|74289|United States|-6|apartment| +36587|AAAAAAAALOOIAAAA|939|Second Meadow|Lane|Suite 280|Montague|Haywood County|NC|24062|United States|-5|single family| +36588|AAAAAAAAMOOIAAAA|157|Ridge |Parkway|Suite 130|Newport|Rowan County|NC|21521|United States|-5|apartment| +36589|AAAAAAAANOOIAAAA|256|Franklin 14th|Drive|Suite 490|Greenville|Franklin County|AR|71387|United States|-6|apartment| +36590|AAAAAAAAOOOIAAAA|226|Jefferson 12th|Circle|Suite F|Springdale|Wayne County|TN|38883|United States|-6|single family| +36591|AAAAAAAAPOOIAAAA|||ST|Suite I|Woodville|||44289|United States||single family| +36592|AAAAAAAAAPOIAAAA||Oak |||||NM||||| +36593|AAAAAAAABPOIAAAA|62|Hill Ash|Ave|Suite K|Jackson|Miami County|IN|49583|United States|-5|condo| +36594|AAAAAAAACPOIAAAA|437|Valley |Wy|Suite K|Friendship|Herkimer County|NY|14536|United States|-5|apartment| +36595|AAAAAAAADPOIAAAA|360|3rd |Road|Suite G|Friendship|Blaine County|NE|64536|United States|-6|condo| +36596|AAAAAAAAEPOIAAAA|879|6th |Way|Suite 130|Waterloo|Clark County|IN|41675|United States|-5|condo| +36597|AAAAAAAAFPOIAAAA|806|South Mill|Way|Suite 70|Fairfield|Madison County|NC|26192|United States|-5|condo| +36598|AAAAAAAAGPOIAAAA|412|8th 11th|Way|Suite 330|Hopewell|Barry County|MO|60587|United States|-6|condo| +36599|AAAAAAAAHPOIAAAA|719|Elm |Ln|Suite I|Lewis|Harrison County|KY|47066|United States|-6|condo| +36600|AAAAAAAAIPOIAAAA|79|Dogwood 5th|Blvd|Suite A|Brownsville|Charleston County|SC|29310|United States|-5|single family| +36601|AAAAAAAAJPOIAAAA|602|Main |Street|Suite 490|Stringtown|Boone County|IN|40162|United States|-5|single family| +36602|AAAAAAAAKPOIAAAA|49|2nd Park|Drive|Suite W|Franklin|Mineral County|WV|29101|United States|-5|single family| +36603|AAAAAAAALPOIAAAA|385|5th 5th|Ave|Suite H|New Town|Utah County|UT|89634|United States|-7|single family| +36604|AAAAAAAAMPOIAAAA|357|Sycamore |Pkwy|Suite 90|Shady Grove|Starke County|IN|42812|United States|-5|single family| +36605|AAAAAAAANPOIAAAA|247|Hillcrest |RD|Suite D|Belmont|Rabun County|GA|30191|United States|-5|condo| +36606|AAAAAAAAOPOIAAAA|789|Park 6th|Dr.|Suite 50|Franklin|Canyon County|ID|89101|United States|-7|single family| +36607|AAAAAAAAPPOIAAAA|136|Adams |Boulevard|Suite 0|Stringtown|Cleveland County|AR|70162|United States|-6|apartment| +36608|AAAAAAAAAAPIAAAA|802|Chestnut |Ct.|Suite I|Valley View|Beaver County|PA|15124|United States|-5|condo| +36609|AAAAAAAABAPIAAAA|548|3rd Cedar|Boulevard|Suite 300|Woodland|Noxubee County|MS|54854|United States|-6|apartment| +36610|AAAAAAAACAPIAAAA|992|Johnson Cherry|Lane|Suite Y|Woodville|Frio County|TX|74289|United States|-6|condo| +36611|AAAAAAAADAPIAAAA|303|7th |Street|Suite B|Riverside|Granville County|NC|29231|United States|-5|apartment| +36612|AAAAAAAAEAPIAAAA|563|Spring Park|Ave|Suite F|Mount Zion|Pacific County|WA|98054|United States|-8|single family| +36613|AAAAAAAAFAPIAAAA|612|Willow |Way|Suite N|Mount Pleasant|Jefferson County|AL|31933|United States|-6|single family| +36614|AAAAAAAAGAPIAAAA|784|Broadway |Lane|Suite 300|Salem|Douglas County|OR|98048|United States|-8|condo| +36615|AAAAAAAAHAPIAAAA|682|Jefferson |Court|Suite L|Riverdale|Martin County|KY|49391|United States|-5|condo| +36616|AAAAAAAAIAPIAAAA|994|Hillcrest Hickory|Ln|Suite 470|Hamilton|Williamsburg County|SC|22808|United States|-5|single family| +36617|AAAAAAAAJAPIAAAA||Miller ||Suite T|||||United States|-7|condo| +36618|AAAAAAAAKAPIAAAA|702|Pine East|||Midway|Russell County|||United States||| +36619|AAAAAAAALAPIAAAA|86|Hill |Lane|Suite 50|Oakland|Clay County|MS|59843|United States|-6|apartment| +36620|AAAAAAAAMAPIAAAA|619|1st North|Street|Suite 210|Spring Valley|Nuckolls County|NE|66060|United States|-7|apartment| +36621|AAAAAAAANAPIAAAA|938|Willow Fourth|Boulevard|Suite Q|Walnut|McHenry County|ND|56245|United States|-6|single family| +36622|AAAAAAAAOAPIAAAA|664|Church |Pkwy|Suite 320|Farmington|Potter County|PA|19145|United States|-5|condo| +36623|AAAAAAAAPAPIAAAA|341|Valley Pine|Drive|Suite 320|Star|Essex County|MA|01325|United States|-5|apartment| +36624|AAAAAAAAABPIAAAA|596|Cedar |Way|Suite G|Lakewood|Harvey County|KS|68877|United States|-6|apartment| +36625|AAAAAAAABBPIAAAA|210|Hillcrest |Parkway|Suite 20|Springfield|Jasper County|IL|69303|United States|-6|single family| +36626|AAAAAAAACBPIAAAA|65|Willow |Court|Suite 130|Greenville|Hardin County|TX|71387|United States|-6|condo| +36627|AAAAAAAADBPIAAAA|576|Lincoln Main|Cir.|Suite I|Crossroads|Gilchrist County|FL|30534|United States|-5|apartment| +36628|AAAAAAAAEBPIAAAA|445|Lake |Blvd|Suite 390|Green Acres|Hancock County|WV|27683|United States|-5|condo| +36629|AAAAAAAAFBPIAAAA|915|12th Valley|RD|Suite 210|Oak Ridge|Rockingham County|NH|08971|United States|-5|apartment| +36630|AAAAAAAAGBPIAAAA|603|Cedar Ridge|Lane|Suite X|Sunnyside|Limestone County|TX|71952|United States|-6|single family| +36631|AAAAAAAAHBPIAAAA|503|Franklin 3rd|Drive|Suite W|Jackson|Livingston County|NY|19583|United States|-5|condo| +36632|AAAAAAAAIBPIAAAA|562|River |ST|Suite 290|Hopewell|Philadelphia County|PA|10587|United States|-5|condo| +36633|AAAAAAAAJBPIAAAA|297|Sunset |Ave|Suite X|Ashland|Liberty County|GA|34244|United States|-5|apartment| +36634|AAAAAAAAKBPIAAAA|860|Pine |Wy|Suite Y|Spring Hill|Wayne County|WV|26787|United States|-5|single family| +36635|AAAAAAAALBPIAAAA|495|Railroad Sixth|Road|Suite 490|Greenwood|Penobscot County|ME|09428|United States|-5|apartment| +36636|AAAAAAAAMBPIAAAA|741|Hickory Lake|Ave|Suite 380|Mount Vernon|Logan County|OH|48482|United States|-5|single family| +36637|AAAAAAAANBPIAAAA|551|1st Hickory|Dr.|Suite J|Woodlawn|Logan County|OK|74098|United States|-6|single family| +36638|AAAAAAAAOBPIAAAA|420|9th |Cir.|Suite 420|Green Acres|Burleigh County|ND|57683|United States|-6|single family| +36639|AAAAAAAAPBPIAAAA|648|Sunset |Lane|Suite C|Plainview|Uintah County|UT|83683|United States|-7|apartment| +36640|AAAAAAAAACPIAAAA|928|Meadow Johnson|Avenue|Suite 170|Lakeside|Perry County|AR|79532|United States|-6|single family| +36641|AAAAAAAABCPIAAAA|304|Ninth 1st|Ave|Suite 440|Fairview|Kane County|IL|65709|United States|-6|condo| +36642|AAAAAAAACCPIAAAA|54|Lake |Boulevard|Suite A|Shady Grove|Mesa County|CO|82812|United States|-7|apartment| +36643|AAAAAAAADCPIAAAA|376|Ridge |Ct.|Suite 280|Buena Vista|Towns County|GA|35752|United States|-5|apartment| +36644|AAAAAAAAECPIAAAA|717|Pine Pine|Avenue|Suite 490|Lincoln|Banks County|GA|31289|United States|-5|single family| +36645|AAAAAAAAFCPIAAAA|119|Chestnut 12th|Circle|Suite J|Crossroads|Gallatin County|KY|40534|United States|-6|apartment| +36646|AAAAAAAAGCPIAAAA|591|Franklin |Boulevard|Suite B|Riverview|Jenkins County|GA|39003|United States|-5|apartment| +36647|AAAAAAAAHCPIAAAA|162|South |Way|Suite 470|Louisville|Pike County|IL|64464|United States|-6|single family| +36648|AAAAAAAAICPIAAAA||Williams |||||||||| +36649|AAAAAAAAJCPIAAAA|648|Oak |ST|Suite I|Friendship|Union County|SD|54536|United States|-7|apartment| +36650|AAAAAAAAKCPIAAAA|928|8th |Court|Suite L|Hopewell|Ector County|TX|70587|United States|-6|apartment| +36651|AAAAAAAALCPIAAAA|745|Valley |Pkwy|Suite 340|Sunnyside|Gallia County|OH|41952|United States|-5|single family| +36652|AAAAAAAAMCPIAAAA|621|Hickory East|ST|Suite S|Springdale|Taliaferro County|GA|38883|United States|-5|apartment| +36653|AAAAAAAANCPIAAAA|313|11th |Ln|Suite Q|Wilson|Pike County|KY|46971|United States|-5|apartment| +36654|AAAAAAAAOCPIAAAA|574|Dogwood Birch|ST|Suite F|Riverview|Lewis County|WV|29003|United States|-5|condo| +36655|AAAAAAAAPCPIAAAA||Cedar Church||Suite 220||Riley County||62812||-6|apartment| +36656|AAAAAAAAADPIAAAA|584|Park |Lane|Suite X|Wildwood|Cleveland County|NC|26871|United States|-5|apartment| +36657|AAAAAAAABDPIAAAA|571|Maple Center|Street|Suite M|Hillcrest|Wasco County|OR|93003|United States|-8|condo| +36658|AAAAAAAACDPIAAAA||||Suite P|Sunnyside||||||apartment| +36659|AAAAAAAADDPIAAAA|77|6th |Parkway|Suite 440|Springdale|Clay County|SD|58883|United States|-6|apartment| +36660|AAAAAAAAEDPIAAAA|132|South Pine|Wy|Suite V|Lakewood|Scott County|KS|68877|United States|-6|apartment| +36661|AAAAAAAAFDPIAAAA|959|Church Forest|Road|Suite J|Lincoln|Tunica County|MS|51289|United States|-6|apartment| +36662|AAAAAAAAGDPIAAAA|800|Lake Main|Drive|Suite A|Spring Valley|Ozaukee County|WI|56060|United States|-6|condo| +36663|AAAAAAAAHDPIAAAA|156|Locust Laurel|Way|Suite F|Liberty|Haywood County|TN|33451|United States|-5|condo| +36664|AAAAAAAAIDPIAAAA|242|Dogwood Ridge|Avenue|Suite U|Buena Vista|Deuel County|NE|65752|United States|-6|apartment| +36665|AAAAAAAAJDPIAAAA|678|Ridge |Blvd|Suite S|Antioch|Douglas County|IL|68605|United States|-6|single family| +36666|AAAAAAAAKDPIAAAA|532|East Elm|Street|Suite N|Valley View|Menominee County|MI|45124|United States|-5|apartment| +36667|AAAAAAAALDPIAAAA|740|Hickory Washington|Parkway|Suite O|Harmony|Hancock County|MS|55804|United States|-6|apartment| +36668|AAAAAAAAMDPIAAAA|237|Third Willow|Way|Suite 320|Clinton|Washington County|KS|68222|United States|-6|condo| +36669|AAAAAAAANDPIAAAA|195|1st 2nd|Ln|Suite A|Antioch|Hillsborough County|FL|38605|United States|-5|apartment| +36670|AAAAAAAAODPIAAAA|40|Mill |Street|Suite U|Clifford|Chattooga County|GA|38164|United States|-5|single family| +36671|AAAAAAAAPDPIAAAA|23|Park Walnut|Avenue|Suite 200|Greenville|Claiborne Parish|LA|71387|United States|-6|condo| +36672|AAAAAAAAAEPIAAAA|880|2nd 5th|Avenue|Suite W|Fairfield|McPherson County|KS|66192|United States|-6|condo| +36673|AAAAAAAABEPIAAAA|613|Lincoln 3rd|Boulevard|Suite 180|Oak Ridge|Greenbrier County|WV|28371|United States|-5|single family| +36674|AAAAAAAACEPIAAAA|932|Williams Forest|Ln|Suite 10|Allentown|Long County|GA|31838|United States|-5|single family| +36675|AAAAAAAADEPIAAAA|681|East |Cir.|Suite 250|Antioch|Augusta County|VA|28605|United States|-5|condo| +36676|AAAAAAAAEEPIAAAA|457|Valley Lakeview|Pkwy|Suite 230|Waterloo|Aleutians East Borough|AK|91675|United States|-9|single family| +36677|AAAAAAAAFEPIAAAA|463|7th |Blvd||Highland||IL|||-6|single family| +36678|AAAAAAAAGEPIAAAA|895|Park Park|RD|Suite 220|Buena Vista|Oldham County|KY|45752|United States|-5|single family| +36679|AAAAAAAAHEPIAAAA|197|Williams 1st|Avenue|Suite 490|Greenville|Grenada County|MS|51387|United States|-6|single family| +36680|AAAAAAAAIEPIAAAA||||Suite 220||Kershaw County|SC||United States|-5|apartment| +36681|AAAAAAAAJEPIAAAA|126|Lakeview View|Parkway|Suite 490||Cass County|IL||United States|-6|condo| +36682|AAAAAAAAKEPIAAAA|314|Fourth |Court|Suite 110|Rutland|Chippewa County|MI|48375|United States|-5|condo| +36683|AAAAAAAALEPIAAAA|666|Ninth |Lane|Suite 200|Sleepy Hollow|Madison County|GA|33592|United States|-5|single family| +36684|AAAAAAAAMEPIAAAA|52|Washington |Avenue|Suite P|Newport|Abbeville County|SC|21521|United States|-5|apartment| +36685|AAAAAAAANEPIAAAA|460|Lakeview Park|Parkway|Suite 30|Summit|Hoke County|NC|20499|United States|-5|condo| +36686|AAAAAAAAOEPIAAAA|113|Cherry |Lane|Suite 90|Oak Grove|Ramsey County|MN|58370|United States|-6|apartment| +36687|AAAAAAAAPEPIAAAA|443||Avenue||Bridgeport|McPherson County|SD||||| +36688|AAAAAAAAAFPIAAAA|956|14th Main|Ln|Suite 280|Woodbine|Gloucester County|VA|24253|United States|-5|condo| +36689|AAAAAAAABFPIAAAA|88|Ash 2nd|Ln|Suite 410|Pleasant Hill|Lawrence County|OH|43604|United States|-5|single family| +36690|AAAAAAAACFPIAAAA|793|5th |Way|Suite 390|Forest Hills|Washington County|VT|09837|United States|-5|apartment| +36691|AAAAAAAADFPIAAAA|584|Seventh |Circle|Suite I|Concord|Knott County|KY|44107|United States|-5|condo| +36692|AAAAAAAAEFPIAAAA|344|Oak |Road|Suite C|Edgewood|Rogers County|OK|70069|United States|-6|single family| +36693|AAAAAAAAFFPIAAAA|289|Spring 2nd|Parkway|Suite 140|Greenfield|Boone County|IN|45038|United States|-5|apartment| +36694|AAAAAAAAGFPIAAAA|44|Maple |Parkway|Suite 20|Jamestown|Lyman County|SD|56867|United States|-7|condo| +36695|AAAAAAAAHFPIAAAA|151|Woodland |Blvd|Suite V|Lakewood|Allen County|KY|48877|United States|-6|apartment| +36696|AAAAAAAAIFPIAAAA|285|Hickory Central|Pkwy|Suite I|Highland|Cattaraugus County|NY|19454|United States|-5|apartment| +36697|AAAAAAAAJFPIAAAA|873|Ninth |Boulevard|Suite 360|Highland Park|Nome Census Area|AK|96534|United States|-9|single family| +36698|AAAAAAAAKFPIAAAA|296|Wilson |Dr.|Suite 360|Oak Grove|Broadwater County|MT|68370|United States|-7|condo| +36699|AAAAAAAALFPIAAAA|770|5th |Way|Suite Y|Pleasant Grove|Cameron County|PA|14136|United States|-5|condo| +36700|AAAAAAAAMFPIAAAA|912|Railroad Hill|Pkwy|Suite 90|Glendale|Mohave County|AZ|83951|United States|-7|apartment| +36701|AAAAAAAANFPIAAAA|827|First |Cir.|Suite M|Vienna|Jefferson County|NE|65119|United States|-7|condo| +36702|AAAAAAAAOFPIAAAA|281|7th |ST|Suite C|Farmington|Jerauld County|SD|59145|United States|-7|apartment| +36703|AAAAAAAAPFPIAAAA|133|1st |Blvd|Suite 230|Waterloo|Schoolcraft County|MI|41675|United States|-5|single family| +36704|AAAAAAAAAGPIAAAA|882|Meadow River|Circle|Suite L|Pine Grove|Auglaize County|OH|44593|United States|-5|apartment| +36705|AAAAAAAABGPIAAAA|194|6th 10th|Avenue|Suite 290|Glendale|Amite County|MS|53951|United States|-6|condo| +36706|AAAAAAAACGPIAAAA|194|9th Green|Blvd|Suite Y|Farmington|Nodaway County|MO|69145|United States|-6|apartment| +36707|AAAAAAAADGPIAAAA|131|River |Lane|Suite V|Five Points|Island County|WA|96098|United States|-8|apartment| +36708|AAAAAAAAEGPIAAAA|480|Laurel Johnson|ST|Suite 150|Pine Grove|Ringgold County|IA|54593|United States|-6|apartment| +36709|AAAAAAAAFGPIAAAA|170|Jefferson |Parkway|Suite 300|Salem|Cameron County|PA|18048|United States|-5|single family| +36710|AAAAAAAAGGPIAAAA|15|Main |Blvd|Suite 330|Clearview|Day County|SD|55495|United States|-6|apartment| +36711|AAAAAAAAHGPIAAAA|590|7th Walnut|Street|Suite B|Glendale|Hubbard County|MN|53951|United States|-6|apartment| +36712|AAAAAAAAIGPIAAAA|671|Ridge |RD|Suite Y|Glenwood|Turner County|SD|53511|United States|-7|apartment| +36713|AAAAAAAAJGPIAAAA|6|Wilson Willow|Pkwy|Suite 300|Glenwood|Mahoning County|OH|43511|United States|-5|condo| +36714|AAAAAAAAKGPIAAAA|746|10th Wilson|Lane|Suite J|Stringtown|Leslie County|KY|40162|United States|-5|apartment| +36715|AAAAAAAALGPIAAAA|106|River |Drive|Suite 410|Edgewood|Sweet Grass County|MT|60069|United States|-7|apartment| +36716|AAAAAAAAMGPIAAAA|807||Drive||Taft|||||-5|single family| +36717|AAAAAAAANGPIAAAA|115|Maple Ridge|Court|Suite 230|Newport|Hampton County|SC|21521|United States|-5|apartment| +36718|AAAAAAAAOGPIAAAA|866|2nd |Circle|Suite D|Maple Grove|Prairie County|AR|78252|United States|-6|single family| +36719|AAAAAAAAPGPIAAAA|241|First Sixth|Drive|Suite C|Walnut Grove|Grant County|SD|57752|United States|-6|single family| +36720|AAAAAAAAAHPIAAAA|607|Main 5th|Dr.|Suite T|Macedonia|Champaign County|OH|41087|United States|-5|apartment| +36721|AAAAAAAABHPIAAAA|576|Oak |Blvd|Suite 280|Jamestown|Humboldt County|IA|56867|United States|-6|apartment| +36722|AAAAAAAACHPIAAAA|827|||Suite Y|Hillcrest|Saluda County|SC||United States|-5|| +36723|AAAAAAAADHPIAAAA||Pine |Lane|Suite 210|||VA|29840|United States||single family| +36724|AAAAAAAAEHPIAAAA|536|Cedar |Street|Suite D|Deerfield|Bristol County|RI|09840|United States|-5|single family| +36725|AAAAAAAAFHPIAAAA|230|Hickory |Drive|Suite L|Pleasant Valley|Greene County|IL|62477|United States|-6|single family| +36726|AAAAAAAAGHPIAAAA|571|Miller Adams|ST|Suite Q|Salem|Nacogdoches County|TX|78048|United States|-6|condo| +36727|AAAAAAAAHHPIAAAA|689|Spring 7th|Ln|Suite C|Crossroads|Swain County|NC|20534|United States|-5|single family| +36728|AAAAAAAAIHPIAAAA|876|Fifth Oak|Lane|Suite 430|New Hope|Perquimans County|NC|29431|United States|-5|condo| +36729|AAAAAAAAJHPIAAAA|14|Church |Court|Suite S|Pine Grove|Macoupin County|IL|64593|United States|-6|apartment| +36730|AAAAAAAAKHPIAAAA|917|Main Lee|Boulevard|Suite V|Edgewood|Crawford County|GA|30069|United States|-5|condo| +36731|AAAAAAAALHPIAAAA|579|College Highland|Ct.|Suite K|Saint Clair|Moffat County|CO|85294|United States|-7|condo| +36732|AAAAAAAAMHPIAAAA|121|College Dogwood|Pkwy|Suite T|Newport|Heard County|GA|31521|United States|-5|single family| +36733|AAAAAAAANHPIAAAA|411|Maple 15th|Ln|Suite U|Shiloh|Cape Girardeau County|MO|69275|United States|-6|single family| +36734|AAAAAAAAOHPIAAAA|287|Laurel |Lane|Suite 30|Maple Grove|Monroe County|MI|48252|United States|-5|single family| +36735|AAAAAAAAPHPIAAAA|632|Maple |Ct.|Suite 30|Oakwood|Scott County|AR|70169|United States|-6|single family| +36736|AAAAAAAAAIPIAAAA|777|||Suite K||Union County||97317|United States|-8|| +36737|AAAAAAAABIPIAAAA|70|Forest Jackson|Circle|Suite E|Concord|Carlton County|MN|54107|United States|-6|condo| +36738|AAAAAAAACIPIAAAA|923|Jefferson Chestnut|Drive|Suite 50|Greenville|Harding County|SD|51387|United States|-7|condo| +36739|AAAAAAAADIPIAAAA|217|Wilson |Blvd|Suite 50|Highland Park|Macon County|AL|36534|United States|-6|condo| +36740|AAAAAAAAEIPIAAAA|738|Sixth Fifth|Ln|Suite P|New Hope|Benton County|TN|39431|United States|-5|apartment| +36741|AAAAAAAAFIPIAAAA|802|View |Avenue|Suite B|Greenfield|Cameron County|PA|15038|United States|-5|apartment| +36742|AAAAAAAAGIPIAAAA|998|Willow Center|Way|Suite V|Franklin|Choctaw County|AL|39101|United States|-6|single family| +36743|AAAAAAAAHIPIAAAA|467|10th 4th|Drive|Suite 200|Florence|Jasper County|MS|53394|United States|-6|single family| +36744|AAAAAAAAIIPIAAAA|592|Locust Lincoln|Road|Suite 400|Pleasant Grove|Tom Green County|TX|74136|United States|-6|condo| +36745|AAAAAAAAJIPIAAAA|647|Forest |Street|||||||-5|single family| +36746|AAAAAAAAKIPIAAAA|944|South |Drive|Suite V|Unionville|Ramsey County|MN|51711|United States|-6|apartment| +36747|AAAAAAAALIPIAAAA|934|North |Circle|Suite C|Oakdale|Caldwell County|MO|69584|United States|-6|condo| +36748|AAAAAAAAMIPIAAAA|669|5th Central|Dr.|Suite T|Mount Olive|Essex County|VA|28059|United States|-5|apartment| +36749|AAAAAAAANIPIAAAA|293|Wilson |Cir.|Suite 330|Cedar Grove|Morehouse Parish|LA|70411|United States|-6|apartment| +36750|AAAAAAAAOIPIAAAA|956|Mill |Drive|Suite 50|Greenwood|Coryell County|TX|78828|United States|-6|apartment| +36751|AAAAAAAAPIPIAAAA|691|Miller Cedar|Boulevard|Suite U|Midway|Le Sueur County|MN|51904|United States|-6|condo| +36752|AAAAAAAAAJPIAAAA|564|Church |Way|Suite 180|Oak Ridge|Essex County|VT|08971|United States|-5|apartment| +36753|AAAAAAAABJPIAAAA|876|Maple 3rd|Ln|Suite 270|Avoca|Logan County|ND|50540|United States|-6|condo| +36754|AAAAAAAACJPIAAAA|463|Hickory East|Ave|Suite 340|Huntington|Iron County|MO|62095|United States|-6|single family| +36755|AAAAAAAADJPIAAAA|673|Fifth |Street|Suite 100|Globe|Taney County|MO|69614|United States|-6|apartment| +36756|AAAAAAAAEJPIAAAA|940|Oak |Street|Suite K|Amity|Grant County|SD|50766|United States|-6|condo| +36757|AAAAAAAAFJPIAAAA|457|Willow Main|Drive|Suite 430|Newtown|Scott County|IA|51749|United States|-6|apartment| +36758|AAAAAAAAGJPIAAAA|450|East Fourth|Way|Suite Y|Pleasant Valley|Oldham County|TX|72477|United States|-6|single family| +36759|AAAAAAAAHJPIAAAA|229||Ave|Suite M|Woodland|Modoc County||||-8|single family| +36760|AAAAAAAAIJPIAAAA|20|Second |Avenue|Suite 380|Crossroads|Lipscomb County|TX|70534|United States|-6|single family| +36761|AAAAAAAAJJPIAAAA|599|View |Circle|Suite Q|Woodcrest|Fremont County|IA|54919|United States|-6|single family| +36762|AAAAAAAAKJPIAAAA|192|Pine 8th|Street|Suite R|Greenville|Marion County|GA|31387|United States|-5|condo| +36763|AAAAAAAALJPIAAAA|85|Johnson Lincoln|Way|Suite 280|Oakland|Jack County|TX|79843|United States|-6|apartment| +36764|AAAAAAAAMJPIAAAA|529|Mill |Ct.|Suite V|Highland Park|Grimes County|TX|76534|United States|-6|single family| +36765|AAAAAAAANJPIAAAA||||||Pitt County|NC|29584|||apartment| +36766|AAAAAAAAOJPIAAAA|299|14th Center|Court|Suite 0|Sunnyside|Callahan County|TX|71952|United States|-6|apartment| +36767|AAAAAAAAPJPIAAAA|||Circle|Suite 80|Four Points|Barnwell County|SC||||single family| +36768|AAAAAAAAAKPIAAAA|391|Miller Hickory|Pkwy|Suite C|Buena Vista|Polk County|AR|75752|United States|-6|condo| +36769|AAAAAAAABKPIAAAA|716|Birch |Lane|Suite 260|Clifton|Brantley County|GA|38014|United States|-5|single family| +36770|AAAAAAAACKPIAAAA|887|Oak Lincoln|Circle|Suite X|Howell|Rutherford County|TN|34854|United States|-6|single family| +36771|AAAAAAAADKPIAAAA|968|View Miller|ST|Suite C|Riverside|Catahoula Parish|LA|79231|United States|-6|single family| +36772|AAAAAAAAEKPIAAAA|705|Ash Park|Road|Suite 410|Midway|Knox County|ME|02504|United States|-5|condo| +36773|AAAAAAAAFKPIAAAA|519|5th Ash|Parkway|Suite 240|Pine Grove|Christian County|KY|44593|United States|-6|condo| +36774|AAAAAAAAGKPIAAAA|583|8th Tenth|ST|Suite 410|Centerville|Toombs County|GA|30059|United States|-5|apartment| +36775|AAAAAAAAHKPIAAAA|919|First |Road|Suite A|Stringtown|Nye County|NV|80162|United States|-8|apartment| +36776|AAAAAAAAIKPIAAAA|781|View |Blvd|Suite U|Cumberland|Bath County|KY|48971|United States|-6|apartment| +36777|AAAAAAAAJKPIAAAA|71|Williams Cedar|Avenue|Suite H|Union|Keith County|NE|68721|United States|-7|condo| +36778|AAAAAAAAKKPIAAAA|350|5th |Circle|Suite 100|Five Forks|Butler County|OH|42293|United States|-5|apartment| +36779|AAAAAAAALKPIAAAA|158|Maple Broadway|Cir.|Suite X|Friendship|Waupaca County|WI|54536|United States|-6|apartment| +36780|AAAAAAAAMKPIAAAA|675|Second 10th|Boulevard|Suite N|Walton|Holmes County|OH|46389|United States|-5|single family| +36781|AAAAAAAANKPIAAAA|280|Meadow |Road|Suite 310|Green Acres|Owsley County|KY|47683|United States|-5|condo| +36782|AAAAAAAAOKPIAAAA|20|Williams 8th|Circle|Suite X|Springfield|Carroll County|GA|39303|United States|-5|apartment| +36783|AAAAAAAAPKPIAAAA|330|Ridge |Circle|Suite Q|Paxton|Ontonagon County|MI|45669|United States|-5|single family| +36784|AAAAAAAAALPIAAAA|934|West |Street|Suite Y|Mountain View|Macon County|GA|34466|United States|-5|single family| +36785|AAAAAAAABLPIAAAA|62|1st |Cir.|Suite U|Enterprise|Mellette County|SD|51757|United States|-7|condo| +36786|AAAAAAAACLPIAAAA|750|Spring |Dr.|Suite L|Springdale|Bradford County|PA|18883|United States|-5|condo| +36787|AAAAAAAADLPIAAAA|707|Green Church|Parkway|Suite 130|Summerfield|Lamb County|TX|70634|United States|-6|apartment| +36788|AAAAAAAAELPIAAAA|255|Lee Lee|Court|Suite 110|Providence|Middlesex County|NJ|07214|United States|-5|apartment| +36789|AAAAAAAAFLPIAAAA|780|Hickory Ridge|Circle|Suite 480|Glenwood|Belmont County|OH|43511|United States|-5|apartment| +36790|AAAAAAAAGLPIAAAA|993|5th Washington|Lane|Suite 250|Jamestown|Seminole County|FL|36867|United States|-5|condo| +36791|AAAAAAAAHLPIAAAA|238|Center Ridge|Dr.|Suite E|Mount Vernon|Alachua County|FL|38482|United States|-5|condo| +36792|AAAAAAAAILPIAAAA|482|Green |Street|Suite 350|Lakeside|Gladwin County|MI|49532|United States|-5|single family| +36793|AAAAAAAAJLPIAAAA||Poplar |||Salem||IN||||condo| +36794|AAAAAAAAKLPIAAAA|257|8th |Way|Suite 130|Five Points|Bastrop County|TX|76098|United States|-6|condo| +36795|AAAAAAAALLPIAAAA|690|West 9th|Drive|Suite K|Newtown|Brown County|NE|61749|United States|-6|condo| +36796|AAAAAAAAMLPIAAAA|501|Park |Parkway|Suite G|Harmony|Fairfax County|VA|25804|United States|-5|apartment| +36797|AAAAAAAANLPIAAAA|600|13th Highland|RD|Suite W|Spring Valley|Banks County|GA|36060|United States|-5|apartment| +36798|AAAAAAAAOLPIAAAA||||||Fisher County|TX|||-6|| +36799|AAAAAAAAPLPIAAAA|998|Spruce |Boulevard|Suite 250|Bunker Hill|Lafayette County|WI|50150|United States|-6|single family| +36800|AAAAAAAAAMPIAAAA|723|River Hill|Pkwy|Suite N|Fairfield|Jones County|NC|26192|United States|-5|apartment| +36801|AAAAAAAABMPIAAAA|235|Ash View|Ct.|Suite H|Farmington|Edgecombe County|NC|29145|United States|-5|condo| +36802|AAAAAAAACMPIAAAA|138|Spruce |Drive|Suite 100|Lakewood|Hooker County|NE|68877|United States|-7|condo| +36803|AAAAAAAADMPIAAAA|664|Elm 4th|Ave|Suite 320|Glenwood|McDuffie County|GA|33511|United States|-5|condo| +36804|AAAAAAAAEMPIAAAA|||Ln|Suite K||Carroll County|IL||||single family| +36805|AAAAAAAAFMPIAAAA|131|First Sunset|Ave|Suite 90|Wilson|Shelby County|KY|46971|United States|-5|apartment| +36806|AAAAAAAAGMPIAAAA|521|Pine |Pkwy|Suite 90|Ashland|Yell County|AR|74244|United States|-6|condo| +36807|AAAAAAAAHMPIAAAA|452|4th |Pkwy|Suite 490|Hopewell|Fentress County|TN|30587|United States|-5|condo| +36808|AAAAAAAAIMPIAAAA|249|Second |Boulevard|Suite V|Union|Pend Oreille County|WA|98721|United States|-8|single family| +36809|AAAAAAAAJMPIAAAA|276|Hill |Court|Suite G|Pleasant Hill|Zavala County|TX|73604|United States|-6|condo| +36810|AAAAAAAAKMPIAAAA|313|Willow ||Suite 160||||37838|||apartment| +36811|AAAAAAAALMPIAAAA|589|Park Lee|ST|Suite 490|Edgewood|Real County|TX|70069|United States|-6|apartment| +36812|AAAAAAAAMMPIAAAA|863|Poplar |Pkwy|Suite M|Mount Olive|Bailey County|TX|78059|United States|-6|condo| +36813|AAAAAAAANMPIAAAA|101|Miller |Parkway|Suite 200|Mountain View|Washington County|ID|84466|United States|-7|condo| +36814|AAAAAAAAOMPIAAAA|484|13th Sycamore|Cir.|Suite K|Oakland|Panola County|MS|59843|United States|-6|apartment| +36815|AAAAAAAAPMPIAAAA|61|Ridge |Ct.|Suite V|Highland Park|Knox County|OH|46534|United States|-5|single family| +36816|AAAAAAAAANPIAAAA|166|View Elm|Lane|Suite 460|Five Points|Hyde County|SD|56098|United States|-7|condo| +36817|AAAAAAAABNPIAAAA|762|Adams |Way|Suite 90|Enterprise|Van Buren County|AR|71757|United States|-6|apartment| +36818|AAAAAAAACNPIAAAA|332|West |Wy|Suite 200|Macedonia|Hamblen County|TN|31087|United States|-5|condo| +36819|AAAAAAAADNPIAAAA|176|Cedar Spruce|Wy|Suite 180|Kingston|York County|PA|14975|United States|-5|apartment| +36820|AAAAAAAAENPIAAAA|589|East |Lane|Suite G|Clearview|Wicomico County|MD|25495|United States|-5|condo| +36821|AAAAAAAAFNPIAAAA|907|Meadow |Street|Suite 470|Buena Vista|Clayton County|IA|55752|United States|-6|single family| +36822|AAAAAAAAGNPIAAAA|81|3rd Forest|Pkwy|Suite S|Mount Olive|Trumbull County|OH|48059|United States|-5|condo| +36823|AAAAAAAAHNPIAAAA|824|Willow 14th|RD|Suite X|King|Uintah County|UT|80008|United States|-7|apartment| +36824|AAAAAAAAINPIAAAA|954|Maple Jackson|Lane|Suite 310|Greenfield|Miami County|IN|45038|United States|-5|single family| +36825|AAAAAAAAJNPIAAAA|107|14th College|Parkway|Suite I|Maple Grove|Knott County|KY|48252|United States|-5|single family| +36826|AAAAAAAAKNPIAAAA|560|Dogwood Adams|RD|Suite 290|Wolf Creek|Todd County|SD|52455|United States|-7|single family| +36827|AAAAAAAALNPIAAAA|591|Elm |Ave|Suite 230|Plainview|Perry County|AL|33683|United States|-6|apartment| +36828|AAAAAAAAMNPIAAAA|173|Lake Hickory|Ct.|Suite 410|Pine Grove|Modoc County|CA|94593|United States|-8|single family| +36829|AAAAAAAANNPIAAAA|447|6th |Dr.|Suite 460|Mount Pleasant|McLean County|KY|41933|United States|-5|apartment| +36830|AAAAAAAAONPIAAAA|513|Poplar |RD|Suite O|Plainview|Ness County|KS|63683|United States|-6|condo| +36831|AAAAAAAAPNPIAAAA|201|9th 8th|Way|Suite 60|Newtown|Lincoln County|CO|81749|United States|-7|single family| +36832|AAAAAAAAAOPIAAAA|444|North Cedar|RD|Suite J|Fairview|Mathews County|VA|25709|United States|-5|condo| +36833|AAAAAAAABOPIAAAA|869|River |Road|Suite 390||Grundy County|TN||United States|-5|| +36834|AAAAAAAACOPIAAAA|108|Ash |Street|Suite Y|Sunnyside|Lawrence County|AR|71952|United States|-6|condo| +36835|AAAAAAAADOPIAAAA|608|Main Hillcrest|Way|Suite 430|Woodland|Lee County|AL|34854|United States|-6|single family| +36836|AAAAAAAAEOPIAAAA|175|Oak Ridge|Lane|Suite 270|Midway|Arenac County|MI|41904|United States|-5|single family| +36837|AAAAAAAAFOPIAAAA|972|Forest Spruce|Ct.|Suite Q|Friendship|Montgomery County|MS|54536|United States|-6|condo| +36838|AAAAAAAAGOPIAAAA|697|First 9th|Drive|Suite G|Brownsville|Osage County|MO|69310|United States|-6|condo| +36839|AAAAAAAAHOPIAAAA||8th ||Suite O|Liberty||OH||United States|-5|| +36840|AAAAAAAAIOPIAAAA|35|12th |ST|Suite 180|Plainview|Somerset County|MD|23683|United States|-5|condo| +36841|AAAAAAAAJOPIAAAA|535|Hillcrest Hill|Wy|Suite 240|Jamestown|Banner County|NE|66867|United States|-6|single family| +36842|AAAAAAAAKOPIAAAA|845|Ridge |Wy|Suite 10|Georgetown|Columbia County|PA|17057|United States|-5|apartment| +36843|AAAAAAAALOPIAAAA|656|View Davis|Pkwy|Suite 160|Hillcrest|Suffolk city|VA|23003|United States|-5|condo| +36844|AAAAAAAAMOPIAAAA|321|Hickory Wilson|Street|Suite 250|Clinton|Bollinger County|MO|68222|United States|-6|condo| +36845|AAAAAAAANOPIAAAA||1st Spring|Parkway|Suite 360|Waterloo|Johnson County||71675|United States|-6|| +36846|AAAAAAAAOOPIAAAA|371|Oak |Circle|Suite 0|Maple Grove|Polk County|NE|68252|United States|-7|condo| +36847|AAAAAAAAPOPIAAAA|820|15th |Lane|Suite 390|Woodville|Kingfisher County|OK|74289|United States|-6|apartment| +36848|AAAAAAAAAPPIAAAA|||Court|Suite W|Liberty||||||apartment| +36849|AAAAAAAABPPIAAAA|309||Way|Suite A|Enterprise||NE||||| +36850|AAAAAAAACPPIAAAA|752|Jackson 7th|Cir.|Suite T|Midway|Greene County|PA|11904|United States|-5|condo| +36851|AAAAAAAADPPIAAAA|932|Maple Mill|Blvd|Suite 190|Oakland|Smith County|TN|39843|United States|-6|apartment| +36852|AAAAAAAAEPPIAAAA|935|Washington Miller|Avenue|Suite L|Woodland|Guadalupe County|NM|84854|United States|-7|apartment| +36853|AAAAAAAAFPPIAAAA|457|11th |Court|Suite X|Macedonia|Lancaster County|VA|21087|United States|-5|single family| +36854|AAAAAAAAGPPIAAAA|702|First |Ave|Suite 440|Farmington|Clinton County|MI|49145|United States|-5|condo| +36855|AAAAAAAAHPPIAAAA|632|Railroad |Cir.|Suite U|Franklin|Nantucket County|MA|09701|United States|-5|condo| +36856|AAAAAAAAIPPIAAAA|239|Lee Maple|Ave|Suite V|Georgetown|Bedford County|VA|27057|United States|-5|single family| +36857|AAAAAAAAJPPIAAAA|256|Fifteenth |Ct.|Suite 100|Clearwater|Mason County|KY|49534|United States|-5|single family| +36858|AAAAAAAAKPPIAAAA|614|||Suite 450|||TN|31289|United States||| +36859|AAAAAAAALPPIAAAA|963|Madison |Ln|Suite 140|Waterloo|Wasatch County|UT|81675|United States|-7|apartment| +36860|AAAAAAAAMPPIAAAA|883|Lake |Wy|Suite 170|Arlington|Payette County|ID|86557|United States|-7|apartment| +36861|AAAAAAAANPPIAAAA|313|Forest Laurel||Suite 210|||VT|09454|United States||single family| +36862|AAAAAAAAOPPIAAAA|992|Lincoln First|Avenue|Suite G|Deerfield|Boone County|IN|49840|United States|-5|apartment| +36863|AAAAAAAAPPPIAAAA|599|8th 1st|ST|Suite I|Edgewood|Lampasas County|TX|70069|United States|-6|apartment| +36864|AAAAAAAAAAAJAAAA|515|Forest |Wy|Suite 200|Springdale|Lane County|OR|98883|United States|-8|condo| +36865|AAAAAAAABAAJAAAA|822|River Mill|Way|Suite 120|Oak Hill|Isle of Wight County|VA|27838|United States|-5|condo| +36866|AAAAAAAACAAJAAAA|278|Sycamore Railroad|Pkwy|Suite 400|Ashland|Mora County|NM|84244|United States|-7|single family| +36867|AAAAAAAADAAJAAAA|316|8th Laurel|Avenue|Suite N|Liberty|Hempstead County|AR|73451|United States|-6|condo| +36868|AAAAAAAAEAAJAAAA|842|Birch Lincoln|RD|Suite U|Bunker Hill|McClain County|OK|70150|United States|-6|apartment| +36869|AAAAAAAAFAAJAAAA|||||Springdale|Washington County|||United States|-5|apartment| +36870|AAAAAAAAGAAJAAAA|223|Washington |Road|Suite F|Clifton|Greene County|AR|78014|United States|-6|apartment| +36871|AAAAAAAAHAAJAAAA|485|Jackson Second|Circle|Suite 420|Lebanon|Lipscomb County|TX|72898|United States|-6|condo| +36872|AAAAAAAAIAAJAAAA|512|Ridge Ash|Dr.|Suite 60|Mount Pleasant|Niagara County|NY|11933|United States|-5|condo| +36873|AAAAAAAAJAAJAAAA|247|4th |Lane|Suite V|Maple Grove|Nodaway County|MO|68252|United States|-6|single family| +36874|AAAAAAAAKAAJAAAA|608|6th |Avenue|Suite G|Cedar Grove|Petroleum County|MT|60411|United States|-7|single family| +36875|AAAAAAAALAAJAAAA|990|5th |Road|Suite 150|Greenwood|Howard County|MD|28828|United States|-5|condo| +36876|AAAAAAAAMAAJAAAA|23|Miller 12th|Drive|Suite 470|Jackson|Sanders County|MT|69583|United States|-7|single family| +36877|AAAAAAAANAAJAAAA|978|Hill |Avenue|Suite S|Franklin|Prowers County|CO|89101|United States|-7|single family| +36878|AAAAAAAAOAAJAAAA|239|Fifth |Blvd|Suite 390|Unionville|Tallapoosa County|AL|31711|United States|-6|apartment| +36879|AAAAAAAAPAAJAAAA|188|Meadow Lake|Road|Suite S|Springfield|Harvey County|KS|69303|United States|-6|apartment| +36880|AAAAAAAAABAJAAAA|848|Adams Second|Way|Suite X|Springfield|Roanoke County|VA|29303|United States|-5|single family| +36881|AAAAAAAABBAJAAAA|431|Elm |Wy|Suite K|Fayetteville|Woodson County|KS|61732|United States|-6|apartment| +36882|AAAAAAAACBAJAAAA|390|Washington Williams|Ln|Suite 210|Riverside|Story County|IA|59231|United States|-6|apartment| +36883|AAAAAAAADBAJAAAA||||Suite 390|Lakeside||AZ|89532|United States|-7|| +36884|AAAAAAAAEBAJAAAA|459|Smith |Blvd|Suite 250|Liberty|Stevens County|KS|63451|United States|-6|single family| +36885|AAAAAAAAFBAJAAAA|941|East |Drive|Suite 200|Lakeside|Guernsey County|OH|49532|United States|-5|single family| +36886|AAAAAAAAGBAJAAAA|243|Johnson |Pkwy|Suite S|Greenfield|Franklin County|TN|35038|United States|-5|condo| +36887|AAAAAAAAHBAJAAAA|965|8th |Pkwy|Suite M|Lakewood|Madison County|IA|58877|United States|-6|single family| +36888|AAAAAAAAIBAJAAAA|883|4th 6th|Wy|Suite 470|Fairfield|Dallas County|TX|76192|United States|-6|condo| +36889|AAAAAAAAJBAJAAAA|321|Lakeview Fourth|Blvd|Suite G|Concord|Edmunds County|SD|54107|United States|-6|condo| +36890|AAAAAAAAKBAJAAAA|639|Davis Lake|RD|Suite 120|Lakeview|Charlotte County|FL|38579|United States|-5|condo| +36891|AAAAAAAALBAJAAAA|477|Hickory Spring|Court|Suite M|Philadelphia|Union County|NM|85591|United States|-7|condo| +36892|AAAAAAAAMBAJAAAA|631|3rd |Pkwy|Suite 460|Sunnyside|Hughes County|OK|71952|United States|-6|single family| +36893|AAAAAAAANBAJAAAA|57|Hickory 2nd|Wy|Suite 240|Springfield|Lee County|VA|29303|United States|-5|condo| +36894|AAAAAAAAOBAJAAAA|90|1st Park|ST|Suite 420|Granite|Brooks County|TX|76284|United States|-6|apartment| +36895|AAAAAAAAPBAJAAAA|603|Forest 7th|Dr.|Suite U|Lakeside|Cherokee County|KS|69532|United States|-6|single family| +36896|AAAAAAAAACAJAAAA|412|Smith |Avenue|Suite B|Buena Vista|Lawrence County|PA|15752|United States|-5|single family| +36897|AAAAAAAABCAJAAAA|690|6th Adams|Parkway|Suite C|Brownsville|Middlesex County|VA|29310|United States|-5|single family| +36898|AAAAAAAACCAJAAAA|664|2nd Second|Dr.|Suite J|Salem|Las Animas County|CO|88048|United States|-7|condo| +36899|AAAAAAAADCAJAAAA|896|2nd 5th|ST|Suite 410|Marion|Prairie County|MT|60399|United States|-7|condo| +36900|AAAAAAAAECAJAAAA|433|Spruce |Ave|Suite 70|Brownsville|Hillsborough County|NH|09910|United States|-5|single family| +36901|AAAAAAAAFCAJAAAA|884|Ridge Highland|Ct.|Suite 200|Woodlawn|Buckingham County|VA|24098|United States|-5|apartment| +36902|AAAAAAAAGCAJAAAA|798|4th |Street|Suite O|Woodland|Williamson County|IL|64854|United States|-6|condo| +36903|AAAAAAAAHCAJAAAA||Cedar ||Suite J|Riverview|Clinton County||49003||-6|| +36904|AAAAAAAAICAJAAAA|865|Ridge |Circle|Suite E|Cherry Valley|San Juan County|UT|80854|United States|-7|condo| +36905|AAAAAAAAJCAJAAAA|968|7th West|Blvd|Suite 230|Farmington|Montezuma County|CO|89145|United States|-7|apartment| +36906|AAAAAAAAKCAJAAAA|319|2nd Valley|Blvd|Suite S|Rockwood|Louisa County|IA|51545|United States|-6|condo| +36907|AAAAAAAALCAJAAAA|200|Washington 8th|Lane|Suite 390|Pleasant Grove|Harrison County|KY|44136|United States|-6|apartment| +36908|AAAAAAAAMCAJAAAA|744|Jefferson 1st|RD|Suite 150|Bunker Hill|Windham County|VT|00750|United States|-5|single family| +36909|AAAAAAAANCAJAAAA|414|8th |Cir.|Suite S|Murphy|Cherokee County|TX|72105|United States|-6|apartment| +36910|AAAAAAAAOCAJAAAA|286|College Cedar|Cir.|Suite T|Centerville|Harnett County|NC|20059|United States|-5|single family| +36911|AAAAAAAAPCAJAAAA|649|3rd Tenth|Ave|Suite A|Hillcrest|Wayne County|KY|43003|United States|-5|apartment| +36912|AAAAAAAAADAJAAAA|181|Jackson Smith|Blvd|Suite F|Farmington|Morgan County|WV|29145|United States|-5|apartment| +36913|AAAAAAAABDAJAAAA|976|View Birch|Parkway|Suite 180|Brownsville|Cambria County|PA|19310|United States|-5|condo| +36914|AAAAAAAACDAJAAAA|601|Hill |Avenue|Suite 50|Hillcrest|Ross County|OH|43003|United States|-5|apartment| +36915|AAAAAAAADDAJAAAA|329|7th |Boulevard|Suite N|Valley View|McKean County|PA|15124|United States|-5|single family| +36916|AAAAAAAAEDAJAAAA|475|5th Fourth|Ave|Suite 240|Mount Pleasant|Faulk County|SD|51933|United States|-6|single family| +36917|AAAAAAAAFDAJAAAA|270|Walnut Valley|Cir.|Suite 270|Hillcrest|Jefferson County|IN|43003|United States|-5|apartment| +36918|AAAAAAAAGDAJAAAA|156|Third |Road|Suite K|Highland|Clearwater County|ID|89454|United States|-7|condo| +36919|AAAAAAAAHDAJAAAA|504|Lake Locust|Street|Suite 180|Salem|Williamsburg city|VA|28048|United States|-5|single family| +36920|AAAAAAAAIDAJAAAA|465|Sunset Park|Ln|Suite M|Hillcrest|Ouachita Parish|LA|73003|United States|-6|condo| +36921|AAAAAAAAJDAJAAAA|52|Locust |Boulevard|Suite 150|Mount Pleasant|Stokes County|NC|21933|United States|-5|single family| +36922|AAAAAAAAKDAJAAAA|602|Franklin |Street|Suite 250|Clinton|Hudson County|NJ|08822|United States|-5|apartment| +36923|AAAAAAAALDAJAAAA|34|Forest Spruce|Circle|Suite O|Concord|Texas County|MO|64107|United States|-6|condo| +36924|AAAAAAAAMDAJAAAA|702|Adams |Dr.|Suite K|Marion|Harlan County|NE|60399|United States|-6|single family| +36925|AAAAAAAANDAJAAAA|826|Sixth |Boulevard|Suite 470|Hartland|McKenzie County|ND|56594|United States|-6|single family| +36926|AAAAAAAAODAJAAAA|40|View Green|Street|Suite U|Morris|Stanton County|NE|66696|United States|-7|single family| +36927|AAAAAAAAPDAJAAAA|847|Adams Elm|Way|Suite 340|White Oak|Dickson County|TN|36668|United States|-5|condo| +36928|AAAAAAAAAEAJAAAA|260|1st |ST|Suite 440|White Oak|McKinley County|NM|86668|United States|-7|apartment| +36929|AAAAAAAABEAJAAAA|481||||Mountain View|Clay County|AL||United States||condo| +36930|AAAAAAAACEAJAAAA|916|View Walnut|ST|Suite H|Greenfield|Kenedy County|TX|75038|United States|-6|condo| +36931|AAAAAAAADEAJAAAA|532|Elm |Ct.|Suite 100|Ellisville|Seminole County|OK|76820|United States|-6|single family| +36932|AAAAAAAAEEAJAAAA|117|Railroad |Street|Suite P|Hopewell|Benton County|AR|70587|United States|-6|single family| +36933|AAAAAAAAFEAJAAAA|87|Forest Madison|Parkway|Suite D|Fairmont|Lubbock County|TX|75987|United States|-6|condo| +36934|AAAAAAAAGEAJAAAA|931|Adams |Cir.|Suite 320|Springfield|Tulsa County|OK|79303|United States|-6|condo| +36935|AAAAAAAAHEAJAAAA|664|West |Blvd|Suite 300|Midway|Clay County|MN|51904|United States|-6|single family| +36936|AAAAAAAAIEAJAAAA|230|4th |Court|Suite 440|Bethel|Geauga County|OH|45281|United States|-5|apartment| +36937|AAAAAAAAJEAJAAAA|545|Cedar Sycamore|Avenue|Suite 330|Tracy|Alpena County|MI|46340|United States|-5|condo| +36938|AAAAAAAAKEAJAAAA|489|1st Elm|ST|Suite G|Glendale|Campbell County|VA|23951|United States|-5|single family| +36939|AAAAAAAALEAJAAAA|275|Davis Broadway|Circle|Suite 340|Pleasant Valley|Somerset County|PA|12477|United States|-5|single family| +36940|AAAAAAAAMEAJAAAA|977|12th |Way|Suite 120|Kingston|Carroll County|KY|44975|United States|-6|condo| +36941|AAAAAAAANEAJAAAA|194|15th |Street|Suite S|Providence|Ida County|IA|56614|United States|-6|apartment| +36942|AAAAAAAAOEAJAAAA|611|Third |Ave|Suite O|Waterloo|Gates County|NC|21675|United States|-5|apartment| +36943|AAAAAAAAPEAJAAAA|615|7th Spring|Way|Suite 380|Providence|Warren County|MS|56614|United States|-6|single family| +36944|AAAAAAAAAFAJAAAA|705|Tenth |Drive|Suite B|Mount Olive|Benton County|IA|58059|United States|-6|apartment| +36945|AAAAAAAABFAJAAAA|601|Cherry 1st|Cir.|Suite J|Midway|Webster County|MS|51904|United States|-6|condo| +36946|AAAAAAAACFAJAAAA|971|Cedar Wilson|Drive|Suite 160|Five Points|Roane County|TN|36098|United States|-6|condo| +36947|AAAAAAAADFAJAAAA|659|View |Avenue|Suite Y|Pleasant Grove|Clay County|KS|64136|United States|-6|single family| +36948|AAAAAAAAEFAJAAAA|101|Main College|Wy|Suite 260|Spring Hill|Colonial Heights city|VA|26787|United States|-5|apartment| +36949|AAAAAAAAFFAJAAAA|524|Willow |Street|Suite X|Highland|Nye County|NV|89454|United States|-8|apartment| +36950|AAAAAAAAGFAJAAAA|935|Center Maple|Dr.|Suite D|Riverdale|Taylor County|WI|59391|United States|-6|apartment| +36951|AAAAAAAAHFAJAAAA|927|11th |Circle|Suite V|Woodville|Livingston County|NY|14289|United States|-5|apartment| +36952|AAAAAAAAIFAJAAAA|645|Davis |Ln|Suite 70|Wildwood|Childress County|TX|76871|United States|-6|single family| +36953|AAAAAAAAJFAJAAAA|414|2nd |Street|Suite Y|Oak Hill|Holt County|NE|67838|United States|-7|single family| +36954|AAAAAAAAKFAJAAAA|797|Elm Highland|Blvd|Suite 10|Union Hill|Madison County|IA|57746|United States|-6|single family| +36955|AAAAAAAALFAJAAAA|438|Willow |Ln|Suite H|Pleasant Hill|Vigo County|IN|43604|United States|-5|apartment| +36956|AAAAAAAAMFAJAAAA|876|Green |Lane|Suite Y|Bunker Hill|Tucker County|WV|20150|United States|-5|apartment| +36957|AAAAAAAANFAJAAAA|138|Maple |Parkway|Suite E|Mount Zion|Mora County|NM|88054|United States|-7|single family| +36958|AAAAAAAAOFAJAAAA|324|Highland Center|Lane|Suite 170|Oakland|Stark County|IL|69843|United States|-6|single family| +36959|AAAAAAAAPFAJAAAA|56|South |Blvd|Suite U|Green Valley|Lafayette County|MO|68337|United States|-6|apartment| +36960|AAAAAAAAAGAJAAAA|276|Fourth |Ave|Suite 470|Florence|Ontonagon County|MI|43394|United States|-5|single family| +36961|AAAAAAAABGAJAAAA|657|Lakeview Cherry|Boulevard|Suite R|Fairfield|Amherst County|VA|26192|United States|-5|apartment| +36962|AAAAAAAACGAJAAAA|176|Willow |Court|Suite 90|Forest Hills|Lawrence County|AR|79237|United States|-6|condo| +36963|AAAAAAAADGAJAAAA|884|Third Broadway|Court|Suite 10|Farmington|Patrick County|VA|29145|United States|-5|condo| +36964|AAAAAAAAEGAJAAAA|742|Highland Forest|Street|Suite 260|Walnut Grove|Montgomery County|TN|37752|United States|-6|single family| +36965|AAAAAAAAFGAJAAAA|378|1st Broadway|Avenue|Suite T|Berea|Carson County|TX|73804|United States|-6|single family| +36966|AAAAAAAAGGAJAAAA|911|Locust |Drive|Suite 270|Oakwood|Taylor County|IA|50169|United States|-6|condo| +36967|AAAAAAAAHGAJAAAA|323|Third 1st|RD|Suite 200|Paradise|Prince George County|MD|21439|United States|-5|single family| +36968|AAAAAAAAIGAJAAAA|53|Green |Blvd|Suite 130|Oakland|Louisa County|IA|59843|United States|-6|single family| +36969|AAAAAAAAJGAJAAAA|709|11th |Road|Suite S|Lakeside|Le Flore County|OK|79532|United States|-6|single family| +36970|AAAAAAAAKGAJAAAA|473|4th |Parkway|Suite 210|Providence|Washington County|OH|46614|United States|-5|single family| +36971|AAAAAAAALGAJAAAA|517|11th |Way|Suite 370|Pine Grove|Morgan County|GA|34593|United States|-5|condo| +36972|AAAAAAAAMGAJAAAA|830|3rd Washington|RD|Suite E|Jackson|Pawnee County|NE|69583|United States|-7|apartment| +36973|AAAAAAAANGAJAAAA|707|Spruce Cedar|Circle|Suite 170|Ashland|Gladwin County|MI|44244|United States|-5|apartment| +36974|AAAAAAAAOGAJAAAA|782|Jackson Lee|Blvd|Suite G|Woodville|Blackford County|IN|44289|United States|-5|condo| +36975|AAAAAAAAPGAJAAAA|539|Oak |Road|Suite A|Cedar Grove|Carson City|NV|80411|United States|-8|single family| +36976|AAAAAAAAAHAJAAAA|977|Ash Church|Wy|Suite 300|Salem|Clark County|KS|68048|United States|-6|condo| +36977|AAAAAAAABHAJAAAA|107|Pine Ash|Court|Suite 120|Highland Park|Blackford County|IN|46534|United States|-5|single family| +36978|AAAAAAAACHAJAAAA|44|Third |Parkway|Suite 490|Springtown|Androscoggin County|ME|09858|United States|-5|single family| +36979|AAAAAAAADHAJAAAA|245|Seventh 2nd|RD|Suite U|Hamilton|Pointe Coupee Parish|LA|72808|United States|-6|apartment| +36980|AAAAAAAAEHAJAAAA|788|Valley 2nd|Drive|Suite S|Paxton|Colfax County|NE|65669|United States|-6|single family| +36981|AAAAAAAAFHAJAAAA|283|Oak Sixth|Wy|Suite X|Stringtown|Lanier County|GA|30162|United States|-5|apartment| +36982|AAAAAAAAGHAJAAAA|427|Birch |Dr.|Suite W|Oakdale|Polk County|FL|39584|United States|-5|condo| +36983|AAAAAAAAHHAJAAAA|676|Jackson North|Street|Suite 360|Glenwood|Keya Paha County|NE|63511|United States|-7|single family| +36984|AAAAAAAAIHAJAAAA|600|Walnut |Pkwy|Suite 140|Plainview|Barrow County|GA|33683|United States|-5|apartment| +36985|AAAAAAAAJHAJAAAA|76|Central Wilson|Way|Suite 320|Liberty|Chatham County|GA|33451|United States|-5|single family| +36986|AAAAAAAAKHAJAAAA|304|Elm Locust|Avenue|Suite 380|Unionville|Montgomery County|OH|41711|United States|-5|apartment| +36987|AAAAAAAALHAJAAAA|382|7th 6th|Lane|Suite 130|Stringtown|Screven County|GA|30162|United States|-5|condo| +36988|AAAAAAAAMHAJAAAA|175|Williams Lee|Circle|Suite E|Springfield|Sumter County|FL|39303|United States|-5|apartment| +36989|AAAAAAAANHAJAAAA|159|North |Dr.||Lewisburg|Greene County|MS|||-6|| +36990|AAAAAAAAOHAJAAAA|944|Highland |Boulevard|Suite 250|Red Hill|Jackson County|OR|94338|United States|-8|condo| +36991|AAAAAAAAPHAJAAAA|841|River |Parkway|Suite 160|Langdon|Branch County|MI|40852|United States|-5|single family| +36992|AAAAAAAAAIAJAAAA|318|View Broadway|ST|Suite H|Greenville|Fairfield County|CT|01987|United States|-5|condo| +36993|AAAAAAAABIAJAAAA|674|Spruce |Avenue|Suite Y|Lee|Phillips County|CO|80408|United States|-7|condo| +36994|AAAAAAAACIAJAAAA|727|Tenth |Cir.|Suite 110|Lakeside|Jones County|SD|59532|United States|-7|apartment| +36995|AAAAAAAADIAJAAAA|850|Johnson |Way|Suite O|Lebanon|Terry County|TX|72898|United States|-6|condo| +36996|AAAAAAAAEIAJAAAA|489|5th |Parkway|Suite N|Belmont|Bedford city|VA|20191|United States|-5|condo| +36997|AAAAAAAAFIAJAAAA|473|Lake 8th|Ct.|Suite 370|Macedonia|Pettis County|MO|61087|United States|-6|single family| +36998|AAAAAAAAGIAJAAAA|322||Ct.||Antioch||KY|48605|||condo| +36999|AAAAAAAAHIAJAAAA|219|Seventh Maple|Dr.|Suite J|Woodville|Buffalo County|NE|64289|United States|-6|single family| +37000|AAAAAAAAIIAJAAAA|||Dr.|Suite A|||||||| +37001|AAAAAAAAJIAJAAAA|580|Center 13th|Circle|Suite E|Arlington|Essex County|VA|26557|United States|-5|single family| +37002|AAAAAAAAKIAJAAAA|167|Franklin |RD|Suite 470|Shady Grove|Covington County|MS|52812|United States|-6|apartment| +37003|AAAAAAAALIAJAAAA|156|Willow 6th|Street|Suite Q|Deerfield|Clinton County|MO|69840|United States|-6|condo| +37004|AAAAAAAAMIAJAAAA|858||Circle|Suite 150|||AL||United States||single family| +37005|AAAAAAAANIAJAAAA|323|Meadow |Parkway|Suite T|Woodlawn|Floyd County|IN|44098|United States|-5|single family| +37006|AAAAAAAAOIAJAAAA|779|Lincoln |Parkway|Suite D|Fairview|Pawnee County|KS|65709|United States|-6|condo| +37007|AAAAAAAAPIAJAAAA|637|Johnson |ST|Suite X|Five Points|Bath County|KY|46098|United States|-6|condo| +37008|AAAAAAAAAJAJAAAA|28|Cedar |Pkwy|Suite X|Wilson|Pittsylvania County|VA|26971|United States|-5|condo| +37009|AAAAAAAABJAJAAAA|426|Railroad Hill|Ct.|Suite I|Wilson|Reynolds County|MO|66971|United States|-6|single family| +37010|AAAAAAAACJAJAAAA|996|River Third|Circle|Suite 180|Riverdale|Randolph County|AR|79391|United States|-6|condo| +37011|AAAAAAAADJAJAAAA|473|Maple |Blvd|Suite 330|Riverdale|Beckham County|OK|79391|United States|-6|condo| +37012|AAAAAAAAEJAJAAAA|552|Highland Railroad|RD|Suite E|Farmington|Grundy County|IA|59145|United States|-6|condo| +37013|AAAAAAAAFJAJAAAA|900|Pine |Blvd|Suite Y|Fairview|Trempealeau County|WI|55709|United States|-6|single family| +37014|AAAAAAAAGJAJAAAA|211|8th Park|Cir.|Suite N|Antioch|Fountain County|IN|48605|United States|-5|single family| +37015|AAAAAAAAHJAJAAAA|34|Mill South|RD|Suite G|Oak Hill|Putnam County|GA|37838|United States|-5|apartment| +37016|AAAAAAAAIJAJAAAA|794|Mill Sunset|Street|Suite I|Woodlawn|Daniels County|MT|64098|United States|-7|condo| +37017|AAAAAAAAJJAJAAAA|751|Cherry Lakeview|Ave|Suite E|Clifton|Wicomico County|MD|28014|United States|-5|condo| +37018|AAAAAAAAKJAJAAAA|962|Cherry 6th|Ct.|Suite O|Plainview|Fayette County|OH|43683|United States|-5|apartment| +37019|AAAAAAAALJAJAAAA|609|Green Lincoln|Avenue|Suite 240|White Oak|Lee County|FL|36668|United States|-5|condo| +37020|AAAAAAAAMJAJAAAA|929|Lincoln Center|ST|Suite 130|Glendale|Butler County|KS|63951|United States|-6|single family| +37021|AAAAAAAANJAJAAAA|315|Lakeview |ST|Suite L|Riverview|Brown County|OH|49003|United States|-5|single family| +37022|AAAAAAAAOJAJAAAA|591|Oak |Parkway|Suite 30|Stringtown|Perkins County|NE|60162|United States|-7|single family| +37023|AAAAAAAAPJAJAAAA|405|Franklin |Court|Suite X|Providence|Clark County|NV|86614|United States|-8|single family| +37024|AAAAAAAAAKAJAAAA|294|Cedar |Boulevard|Suite Y|Riverdale|Wilkin County|MN|59391|United States|-6|single family| +37025|AAAAAAAABKAJAAAA|998|Highland |Parkway|Suite V|Oak Grove|Rensselaer County|NY|18370|United States|-5|condo| +37026|AAAAAAAACKAJAAAA|415|Oak |Ave|Suite S|Lincoln|Hertford County|NC|21289|United States|-5|apartment| +37027|AAAAAAAADKAJAAAA|949|Fourth Sixth|Ct.|Suite 150|Bunker Hill|Carroll County|AR|70150|United States|-6|single family| +37028|AAAAAAAAEKAJAAAA|469|Lincoln Railroad|Road|Suite K|Summit|Klamath County|OR|90499|United States|-8|condo| +37029|AAAAAAAAFKAJAAAA|130|Sixth Fifth|Circle|Suite 350|Lebanon|Gallatin County|KY|42898|United States|-6|single family| +37030|AAAAAAAAGKAJAAAA|486|5th Meadow|Ln|Suite O|Mount Olive|Perry County|AR|78059|United States|-6|apartment| +37031|AAAAAAAAHKAJAAAA|657|River Center|Drive|Suite 340|Highland Park|Midland County|MI|46534|United States|-5|single family| +37032|AAAAAAAAIKAJAAAA|658|13th 4th|Court|Suite W|Clinton|Adair County|MO|68222|United States|-6|single family| +37033|AAAAAAAAJKAJAAAA|772|Washington |RD|Suite 80|Maple Grove|Greene County|MO|68252|United States|-6|condo| +37034|AAAAAAAAKKAJAAAA|396|5th |Road|Suite 180|Oakwood|Allamakee County|IA|50169|United States|-6|apartment| +37035|AAAAAAAALKAJAAAA|773|Locust |RD|Suite O|Mount Zion|El Paso County|TX|78054|United States|-6|apartment| +37036|AAAAAAAAMKAJAAAA|30|5th Seventh|Blvd|Suite 470|Wilson|Palo Alto County|IA|56971|United States|-6|apartment| +37037|AAAAAAAANKAJAAAA|802|River Pine|Avenue|Suite Y|Farmington|Warren County|OH|49145|United States|-5|single family| +37038|AAAAAAAAOKAJAAAA|504|Forest |Ct.|Suite L|Oakland|Union County|GA|39843|United States|-5|condo| +37039|AAAAAAAAPKAJAAAA|888|Mill Johnson|Parkway|Suite X|Newport|Preble County|OH|41521|United States|-5|condo| +37040|AAAAAAAAALAJAAAA|111|6th |Court|Suite F|Liberty|Rock Island County|IL|63451|United States|-6|apartment| +37041|AAAAAAAABLAJAAAA|555|Spring Center|Street|Suite 90|Antioch|Harvey County|KS|68605|United States|-6|apartment| +37042|AAAAAAAACLAJAAAA|952|Fourth |Boulevard|Suite 340|Spring Hill|Carroll County|AR|76787|United States|-6|apartment| +37043|AAAAAAAADLAJAAAA|212|Spring |Drive|Suite Y|Pleasant Grove|Wyoming County|WV|24136|United States|-5|condo| +37044|AAAAAAAAELAJAAAA|564|Washington |ST|Suite O|Oak Ridge|Monroe County|MS|58371|United States|-6|apartment| +37045|AAAAAAAAFLAJAAAA|711|East Pine|RD|Suite 70|Spring Valley|Aleutians East Borough|AK|96060|United States|-9|single family| +37046|AAAAAAAAGLAJAAAA|59|River 11th|Ct.|Suite 70|Mount Zion|Lanier County|GA|38054|United States|-5|single family| +37047|AAAAAAAAHLAJAAAA|700|14th |Wy|Suite L|Milo|Surry County|VA|20116|United States|-5|single family| +37048|AAAAAAAAILAJAAAA|717|Railroad |Cir.|Suite X|Farmington|Jerauld County|SD|59145|United States|-7|condo| +37049|AAAAAAAAJLAJAAAA|980|First View|Dr.|Suite 360|Newport|Kent County|MD|21521|United States|-5|apartment| +37050|AAAAAAAAKLAJAAAA|929|South |ST|Suite 100|Union Hill|Jasper County|MO|67746|United States|-6|single family| +37051|AAAAAAAALLAJAAAA||||Suite I||Stafford County||||-6|| +37052|AAAAAAAAMLAJAAAA|832|Hill 8th|ST|Suite 350|Macedonia|Johnson County|TN|31087|United States|-6|single family| +37053|AAAAAAAANLAJAAAA|116|Woodland |Street|Suite 150|Lakeview|Adams County|NE|68579|United States|-6|condo| +37054|AAAAAAAAOLAJAAAA|215|Main Central|Road|Suite O|Spring Valley|Multnomah County|OR|96060|United States|-8|single family| +37055|AAAAAAAAPLAJAAAA|182|Elevnth |Avenue|Suite 60|Sulphur Springs|Nash County|NC|28354|United States|-5|condo| +37056|AAAAAAAAAMAJAAAA|95|4th Mill|Blvd|Suite C|Riverdale|Nolan County|TX|79391|United States|-6|condo| +37057|AAAAAAAABMAJAAAA|899|Walnut |Cir.|Suite 340|Shiloh|Charlotte County|VA|29275|United States|-5|condo| +37058|AAAAAAAACMAJAAAA|12|Madison Walnut|Lane|Suite 80|Crossroads|Clinton County|PA|10534|United States|-5|single family| +37059|AAAAAAAADMAJAAAA|258|Main Smith|RD|Suite D|Mount Zion|Douglas County|WI|58054|United States|-6|single family| +37060|AAAAAAAAEMAJAAAA|706|13th Hill|Circle|Suite F|Summit|Bath County|VA|20499|United States|-5|apartment| +37061|AAAAAAAAFMAJAAAA|263|Sunset |Lane|Suite 250|Hillcrest|Mineral County|NV|83003|United States|-8|condo| +37062|AAAAAAAAGMAJAAAA||||Suite V|Greenville||AL||||condo| +37063|AAAAAAAAHMAJAAAA|316|Lake |Drive|Suite L|Hillcrest|Maricopa County|AZ|83003|United States|-7|apartment| +37064|AAAAAAAAIMAJAAAA|33|Elm |Way|Suite W|Walnut Grove|Lincoln County|KS|67752|United States|-6|single family| +37065|AAAAAAAAJMAJAAAA|629|Chestnut |Avenue|Suite D|Holland|Harrison County|OH|47751|United States|-5|condo| +37066|AAAAAAAAKMAJAAAA|593|Pine 7th|Lane|Suite 150|Union City|Runnels County|TX|78087|United States|-6|single family| +37067|AAAAAAAALMAJAAAA|313|River |Pkwy|Suite 400|Arlington|Spencer County|KY|46557|United States|-5|condo| +37068|AAAAAAAAMMAJAAAA|101|Ash Sycamore|Ave|Suite 210|Edgewood|Turner County|SD|50069|United States|-7|apartment| +37069|AAAAAAAANMAJAAAA|656|Lake |Blvd|Suite F|Milan|Issaquena County|MS|56697|United States|-6|condo| +37070|AAAAAAAAOMAJAAAA|868|Elm 3rd|Dr.|Suite 320|Tracy|Crawford County|MI|46340|United States|-5|apartment| +37071|AAAAAAAAPMAJAAAA|540|Park Ridge|Lane|Suite 20|Highland|Clear Creek County|CO|89454|United States|-7|single family| +37072|AAAAAAAAANAJAAAA|665|North Cherry|Way|Suite Q|Bethel|Barry County|MI|45281|United States|-5|condo| +37073|AAAAAAAABNAJAAAA|438|Valley |Circle|Suite 90|Pine Grove|Mason County|WA|94593|United States|-8|condo| +37074|AAAAAAAACNAJAAAA|72|Park |Ln|Suite 130|Stringtown|Humphreys County|TN|30162|United States|-5|single family| +37075|AAAAAAAADNAJAAAA|766|Locust 11th|Street|Suite 460|Franklin|LaMoure County|ND|59101|United States|-6|apartment| +37076|AAAAAAAAENAJAAAA|712|Cedar Broadway|Wy|Suite B|Lakewood|Lafayette County|AR|78877|United States|-6|condo| +37077|AAAAAAAAFNAJAAAA|790|Meadow |RD|Suite 220|Mount Vernon|Sumter County|FL|38482|United States|-5|apartment| +37078|AAAAAAAAGNAJAAAA|769|River |Court|Suite U|Clinton|Anson County|NC|28222|United States|-5|single family| +37079|AAAAAAAAHNAJAAAA|306|Main |ST|Suite 270|Caledonia|Franklin County|IA|57411|United States|-6|apartment| +37080|AAAAAAAAINAJAAAA|203|Park |Blvd|Suite A|Friendship|Choctaw County|OK|74536|United States|-6|single family| +37081|AAAAAAAAJNAJAAAA|455|River Green|Cir.|Suite I|Mount Zion|York County|VA|28054|United States|-5|apartment| +37082|AAAAAAAAKNAJAAAA|427|Williams Lake|Blvd|Suite W|Gravel Hill|Adams County|IA|51944|United States|-6|single family| +37083|AAAAAAAALNAJAAAA|109|Railroad |Way|Suite 140|Hillcrest|Shelby County|IA|53003|United States|-6|apartment| +37084|AAAAAAAAMNAJAAAA|153|Madison View|Cir.|Suite W|Walnut Grove|District of Columbia|DC|27752|United States|-5|apartment| +37085|AAAAAAAANNAJAAAA|89|2nd |ST|Suite 440|Shiloh|Lake County|IN|49275|United States|-5|condo| +37086|AAAAAAAAONAJAAAA|541|Lincoln 6th|Parkway|Suite 220|Buena Vista|Sharkey County|MS|55752|United States|-6|condo| +37087|AAAAAAAAPNAJAAAA||Seventh |||New Hope|Lincoln County|WY|89431||-7|single family| +37088|AAAAAAAAAOAJAAAA|138|1st |Parkway|Suite U|Greenville|Anoka County|MN|51387|United States|-6|condo| +37089|AAAAAAAABOAJAAAA|979|Seventh |Circle|Suite G|Lakeview|Leon County|FL|38579|United States|-5|single family| +37090|AAAAAAAACOAJAAAA|503|1st Walnut|Pkwy|Suite 280|Florence|Ray County|MO|63394|United States|-6|apartment| +37091|AAAAAAAADOAJAAAA|733|Madison |Parkway|Suite F|Mount Zion|Chesterfield County|VA|28054|United States|-5|condo| +37092|AAAAAAAAEOAJAAAA|985|Third Lakeview|Avenue|Suite L|Arlington|Perry County|AL|36557|United States|-6|apartment| +37093|AAAAAAAAFOAJAAAA|84|Fourth Main|Avenue|Suite 370|Oakland|Turner County|SD|59843|United States|-7|single family| +37094|AAAAAAAAGOAJAAAA|374|Main |Dr.|Suite C|Shannon|Juniata County|PA|14120|United States|-5|apartment| +37095|AAAAAAAAHOAJAAAA|36|Lincoln Sycamore|Cir.|Suite B|Clinton|King County|WA|98222|United States|-8|condo| +37096|AAAAAAAAIOAJAAAA|71|Miller |Circle|Suite I|Oak Hill|Hancock County|KY|47838|United States|-6|single family| +37097|AAAAAAAAJOAJAAAA|120|Elm Spring|Boulevard|Suite 210|Sunnyside|Caldwell County|TX|71952|United States|-6|single family| +37098|AAAAAAAAKOAJAAAA|730|Oak 2nd|Avenue|Suite N|Fairfield|Sumter County|AL|36192|United States|-6|apartment| +37099|AAAAAAAALOAJAAAA|844|13th Smith|Lane|Suite 250|Riverview|Acadia Parish|LA|79003|United States|-6|single family| +37100|AAAAAAAAMOAJAAAA||||||Lebanon County|PA||United States|-5|apartment| +37101|AAAAAAAANOAJAAAA|808|3rd |Blvd|Suite B|Mount Pleasant|Macon County|NC|21933|United States|-5|single family| +37102|AAAAAAAAOOAJAAAA|905|Railroad |Pkwy|Suite A|Mayfield|Multnomah County|OR|95491|United States|-8|single family| +37103|AAAAAAAAPOAJAAAA|92||||Riverdale||MI|49391|United States||condo| +37104|AAAAAAAAAPAJAAAA|93|8th Ninth|Court|Suite 30|Rolling Hills|Cheatham County|TN|37272|United States|-5|condo| +37105|AAAAAAAABPAJAAAA|133|14th Smith|Cir.|Suite L|Kingston|Madison County|IN|44975|United States|-5|condo| +37106|AAAAAAAACPAJAAAA|404|Park |Drive|Suite G|Summit|Bollinger County|MO|60499|United States|-6|condo| +37107|AAAAAAAADPAJAAAA||North |RD|||Warren County|IN||United States||| +37108|AAAAAAAAEPAJAAAA|528|Lincoln |Wy|Suite 390|Shiloh|Monona County|IA|59275|United States|-6|apartment| +37109|AAAAAAAAFPAJAAAA|32|Center Willow|Ave|Suite J|Leesburg|Guernsey County|OH|45605|United States|-5|apartment| +37110|AAAAAAAAGPAJAAAA|216|Mill 12th|Way|Suite 400|Glendale|Hamilton County|OH|43951|United States|-5|condo| +37111|AAAAAAAAHPAJAAAA|225|4th |Court|Suite Q|Country Club Estates|Bailey County|TX|76964|United States|-6|single family| +37112|AAAAAAAAIPAJAAAA|227|Hill Lincoln|Blvd|Suite 350|Riverdale|Emporia city|VA|29391|United States|-5|condo| +37113|AAAAAAAAJPAJAAAA|303|3rd 2nd|Dr.|Suite U|Valley View|Ferry County|WA|95124|United States|-8|condo| +37114|AAAAAAAAKPAJAAAA|910|9th Main|Lane|Suite 260|Chestnut Ridge|Adams County|PA|17334|United States|-5|single family| +37115|AAAAAAAALPAJAAAA|942|Hill |Ct.|Suite O|Riverdale|Blaine County|OK|79391|United States|-6|apartment| +37116|AAAAAAAAMPAJAAAA|300|3rd College|ST|Suite F|Salem|Rockbridge County|VA|28048|United States|-5|single family| +37117|AAAAAAAANPAJAAAA|164|Lakeview Wilson|Boulevard|Suite U|Riverview|Ziebach County|SD|59003|United States|-6|single family| +37118|AAAAAAAAOPAJAAAA|697|Lincoln 3rd|Road|Suite 220|Enterprise|Randolph County|IL|61757|United States|-6|apartment| +37119|AAAAAAAAPPAJAAAA|751|Johnson Green|ST|Suite X|Bethel|Indian River County|FL|35281|United States|-5|condo| +37120|AAAAAAAAAABJAAAA|746|Williams Hillcrest|Road|Suite 150|Glenwood|Shackelford County|TX|73511|United States|-6|condo| +37121|AAAAAAAABABJAAAA|997|Spruce Main|Avenue|Suite 490|Summit|Desha County|AR|70499|United States|-6|apartment| +37122|AAAAAAAACABJAAAA|35|Park 5th|Pkwy|Suite Y|Five Forks|Chambers County|TX|72293|United States|-6|apartment| +37123|AAAAAAAADABJAAAA|336|Pine |Drive|Suite 80|Friendship|Borden County|TX|74536|United States|-6|apartment| +37124|AAAAAAAAEABJAAAA|496|9th Fourth|Lane|Suite D|Tipton|Laurel County|KY|48773|United States|-5|single family| +37125|AAAAAAAAFABJAAAA|69|Jefferson Oak|Dr.|Suite 490|Kingston|Roger Mills County|OK|74975|United States|-6|single family| +37126|AAAAAAAAGABJAAAA|804|9th Eigth|Ct.|Suite 40|Marion|Baldwin County|AL|30399|United States|-6|apartment| +37127|AAAAAAAAHABJAAAA|958|Railroad Green|Cir.|Suite F|Cedar Grove|Guthrie County|IA|50411|United States|-6|single family| +37128|AAAAAAAAIABJAAAA|591|Hill 3rd|Wy|Suite S|Riverview|Plymouth County|MA|09603|United States|-5|condo| +37129|AAAAAAAAJABJAAAA|777|College |Avenue|Suite 260|Westville|Carroll County|IA|51649|United States|-6|apartment| +37130|AAAAAAAAKABJAAAA|678|2nd |Circle|Suite M|Shiloh|Cambria County|PA|19275|United States|-5|condo| +37131|AAAAAAAALABJAAAA|46|Second Chestnut|Avenue|Suite 100|Mount Olive|Montgomery County|AR|78059|United States|-6|single family| +37132|AAAAAAAAMABJAAAA|701|7th 2nd|Ct.|Suite 50|Mount Zion|Jenkins County|GA|38054|United States|-5|single family| +37133|AAAAAAAANABJAAAA|69|Sunset Johnson|Drive|Suite 180|Hopewell|McPherson County|SD|50587|United States|-7|apartment| +37134|AAAAAAAAOABJAAAA|301|10th 4th|Circle|Suite 440|Georgetown|Martin County|KY|47057|United States|-5|single family| +37135|AAAAAAAAPABJAAAA|448|Third |Way|Suite C|Avery|Colleton County|SC|20194|United States|-5|apartment| +37136|AAAAAAAAABBJAAAA|502|4th |Blvd|Suite X|Friendship|Schuyler County|IL|64536|United States|-6|single family| +37137|AAAAAAAABBBJAAAA|711|Elm Walnut|Street|Suite B|Mount Vernon|Oneida County|WI|58482|United States|-6|single family| +37138|AAAAAAAACBBJAAAA|899||Cir.|Suite C|Jenkins|Clay County|||||single family| +37139|AAAAAAAADBBJAAAA|495|Wilson 3rd|Parkway|Suite J|Glendale|Galveston County|TX|73951|United States|-6|single family| +37140|AAAAAAAAEBBJAAAA|349|Mill Walnut|Parkway|Suite 280|Unionville|Clackamas County|OR|91711|United States|-8|apartment| +37141|AAAAAAAAFBBJAAAA|816|Ridge Walnut|Ln|Suite F|Belmont|Benton County|MS|50191|United States|-6|condo| +37142|AAAAAAAAGBBJAAAA|495|First |Wy|Suite J|Fairview|Talbot County|GA|35709|United States|-5|apartment| +37143|AAAAAAAAHBBJAAAA|765|Madison |Ct.|Suite D|Georgetown|Whitman County|WA|97057|United States|-8|apartment| +37144|AAAAAAAAIBBJAAAA||Center Lee||Suite 480|Lakewood|Ocean County|NJ||United States||apartment| +37145|AAAAAAAAJBBJAAAA|320|River |Wy|Suite 430|Antioch|Troup County|GA|38605|United States|-5|condo| +37146|AAAAAAAAKBBJAAAA||Park Spruce|Lane|Suite 220|Hamilton||||||apartment| +37147|AAAAAAAALBBJAAAA|440|Park Wilson|Lane|Suite 30|Springfield|Johnson County|KS|69303|United States|-6|single family| +37148|AAAAAAAAMBBJAAAA|695|11th |Pkwy|Suite P|Friendship|Alameda County|CA|94536|United States|-8|single family| +37149|AAAAAAAANBBJAAAA|367|Cherry |Road|Suite A|Georgetown|Bastrop County|TX|77057|United States|-6|condo| +37150|AAAAAAAAOBBJAAAA|511|Park |Lane|Suite J|Hillcrest|Ascension Parish|LA|73003|United States|-6|apartment| +37151|AAAAAAAAPBBJAAAA|930|Railroad Elm|Ln|Suite J|Lakeside|Minnehaha County|SD|59532|United States|-7|condo| +37152|AAAAAAAAACBJAAAA|827|Church Maple|ST|Suite Y|Bunker Hill|Marion County|KS|60150|United States|-6|apartment| +37153|AAAAAAAABCBJAAAA|647|East Poplar|Road|Suite 320|Kingston|Carver County|MN|54975|United States|-6|apartment| +37154|AAAAAAAACCBJAAAA|810|Spring |Drive|Suite 130|Franklin|Union County|GA|||-5|condo| +37155|AAAAAAAADCBJAAAA|420|Williams Washington|ST|Suite S|Harmony|Bastrop County|TX|75804|United States|-6|apartment| +37156|AAAAAAAAECBJAAAA|530|Central Valley|RD|Suite V|Union Hill|Midland County|TX|77746|United States|-6|condo| +37157|AAAAAAAAFCBJAAAA|644|Miller Willow|Boulevard|Suite 390|Crossroads|Chippewa County|MN|50534|United States|-6|single family| +37158|AAAAAAAAGCBJAAAA|867|4th 13th|Way|Suite U|Florence|Dubois County|IN|43394|United States|-5|condo| +37159|AAAAAAAAHCBJAAAA|557|3rd |Court|Suite L|Brownsville|Milwaukee County|WI|59310|United States|-6|apartment| +37160|AAAAAAAAICBJAAAA|594|Park Willow|RD|Suite 240|Enterprise|Taylor County|IA|51757|United States|-6|apartment| +37161|AAAAAAAAJCBJAAAA|403|View Washington|Boulevard|Suite 10|Deerfield|Steuben County|IN|49840|United States|-5|condo| +37162|AAAAAAAAKCBJAAAA|578|Forest |Avenue|Suite 270|Springfield|Manassas Park city|VA|29303|United States|-5|single family| +37163|AAAAAAAALCBJAAAA||South Green|Road||Greenwood|||||-5|| +37164|AAAAAAAAMCBJAAAA|8|Cedar Oak|Boulevard|Suite 490|Ashland|Los Angeles County|CA|94244|United States|-8|condo| +37165|AAAAAAAANCBJAAAA|863|North Elm|Ave|Suite 490|Mountain View|Jasper County|MO|64466|United States|-6|single family| +37166|AAAAAAAAOCBJAAAA|77|Sycamore |ST|Suite Y|Walnut Grove|Westmoreland County|VA|27752|United States|-5|single family| +37167|AAAAAAAAPCBJAAAA|975|Hill Sixth|Wy|Suite S|Enterprise|Hamilton County|TN|31757|United States|-5|single family| +37168|AAAAAAAAADBJAAAA|786|North |Ave|Suite Y|Concord|Bee County|TX|74107|United States|-6|single family| +37169|AAAAAAAABDBJAAAA|359|Laurel Williams|Pkwy|Suite N|Enterprise|Stone County|MS|51757|United States|-6|condo| +37170|AAAAAAAACDBJAAAA||||||Northwest Arctic Borough|||United States|-9|| +37171|AAAAAAAADDBJAAAA|724|3rd |Circle|Suite 110|Glenwood|Wayne County|NY|13511|United States|-5|condo| +37172|AAAAAAAAEDBJAAAA|576|Valley |Pkwy|Suite Y|Pleasant Grove|Platte County|NE|64136|United States|-7|condo| +37173|AAAAAAAAFDBJAAAA|835|Lake |Boulevard|Suite 0|Franklin|Gallatin County|IL|69101|United States|-6|condo| +37174|AAAAAAAAGDBJAAAA|581|10th Green|Ct.|Suite Q|New Hope|Williams County|ND|59431|United States|-6|single family| +37175|AAAAAAAAHDBJAAAA|596|9th West|Ave|Suite 100|Pleasant Grove|Clark County|NV|84136|United States|-8|condo| +37176|AAAAAAAAIDBJAAAA|896|Miller Birch|Street|Suite M|Oakwood|Osceola County|MI|40169|United States|-5|apartment| +37177|AAAAAAAAJDBJAAAA|533|13th |ST|Suite B|Saratoga|McMinn County|TN|32123|United States|-6|single family| +37178|AAAAAAAAKDBJAAAA|322|River |Pkwy|Suite R|Florence|Choctaw County|OK|73394|United States|-6|single family| +37179|AAAAAAAALDBJAAAA|65|Green |Ln|Suite 400|Wildwood|Sullivan County|IN|46871|United States|-5|apartment| +37180|AAAAAAAAMDBJAAAA|68|Poplar |Ave|Suite R|Fairview|Jefferson Davis Parish|LA|75709|United States|-6|condo| +37181|AAAAAAAANDBJAAAA|793|15th |Court|Suite 420|Concord|Spartanburg County|SC|24107|United States|-5|condo| +37182|AAAAAAAAODBJAAAA|741|Eigth |Ln|Suite U|Arthur|Randolph County|AL|35965|United States|-6|condo| +37183|AAAAAAAAPDBJAAAA|548|Lee Elm|ST|Suite 20|Glenwood|Greene County|AR|73511|United States|-6|single family| +37184|AAAAAAAAAEBJAAAA|773|Fourth Seventh|ST|Suite 220|Blue Springs|Bastrop County|TX|74686|United States|-6|apartment| +37185|AAAAAAAABEBJAAAA|160|7th |Pkwy|Suite 290|Concord|Chouteau County|MT|64107|United States|-7|apartment| +37186|AAAAAAAACEBJAAAA|781|4th Maple|Drive|Suite 160|Georgetown|Franklin County|MA|07657|United States|-5|single family| +37187|AAAAAAAADEBJAAAA|410|River |Pkwy|Suite 400|Lone Oak|Edgefield County|SC|26893|United States|-5|apartment| +37188|AAAAAAAAEEBJAAAA|627|Spring Willow|Cir.|Suite 380|Green Acres|Morrill County|NE|67683|United States|-7|single family| +37189|AAAAAAAAFEBJAAAA|644|Jefferson |Parkway|Suite O|Shiloh|Medina County|OH|49275|United States|-5|condo| +37190|AAAAAAAAGEBJAAAA|334|Washington ||||Caledonia County||05881|United States||condo| +37191|AAAAAAAAHEBJAAAA|||Avenue|||Platte County|MO|60587|United States||| +37192|AAAAAAAAIEBJAAAA|456|||Suite J|Centerville|Dolores County||80059||-7|| +37193|AAAAAAAAJEBJAAAA|948|Madison |Avenue|Suite 260|Clifton|Red Lake County|MN|58014|United States|-6|single family| +37194|AAAAAAAAKEBJAAAA|677|Church |Circle|Suite X|Springdale|Dinwiddie County|VA|28883|United States|-5|single family| +37195|AAAAAAAALEBJAAAA||Lakeview Elevnth||Suite O||Orange County|IN|43951|||single family| +37196|AAAAAAAAMEBJAAAA|876|Franklin Park|Cir.|Suite 60|Riverview|McDuffie County|GA|39003|United States|-5|apartment| +37197|AAAAAAAANEBJAAAA|644|Oak |Ln|Suite U|Hidden Valley|Lawrence County|MS|55521|United States|-6|single family| +37198|AAAAAAAAOEBJAAAA|528|15th Spring|Cir.|Suite 90|Woodland|Dickenson County|VA|24854|United States|-5|single family| +37199|AAAAAAAAPEBJAAAA|46|Dogwood First|Cir.|Suite 300|Ellsworth|White County|IN|45079|United States|-5|condo| +37200|AAAAAAAAAFBJAAAA|689|7th Madison|Road|Suite X|Carlisle|Taliaferro County|GA|31620|United States|-5|single family| +37201|AAAAAAAABFBJAAAA|897|Jefferson |RD|Suite 340|Sunnyside|Duchesne County|UT|81952|United States|-7|single family| +37202|AAAAAAAACFBJAAAA|357|Twelfth Sunset|Ln|Suite R|Woodville|Hamilton County|KS|64289|United States|-6|single family| +37203|AAAAAAAADFBJAAAA|681|Second |Pkwy|Suite 380|Oak Grove|Clare County|MI|48370|United States|-5|single family| +37204|AAAAAAAAEFBJAAAA|7|Park 1st|Court|Suite 120|Valley View|Wabasha County|MN|55124|United States|-6|apartment| +37205|AAAAAAAAFFBJAAAA|671|View |Dr.|Suite 300|Harmony|Bergen County|NJ|06404|United States|-5|apartment| +37206|AAAAAAAAGFBJAAAA|59|Oak Locust|Avenue|Suite 220|Frogtown|Bryan County|GA|38784|United States|-5|condo| +37207|AAAAAAAAHFBJAAAA|39|Oak South|Wy|Suite D|Fairfield|Scott County|VA|26192|United States|-5|single family| +37208|AAAAAAAAIFBJAAAA|776|Fifth |Blvd|Suite T|Enterprise|Knox County|NE|61757|United States|-7|condo| +37209|AAAAAAAAJFBJAAAA|820|Locust Center|Circle|Suite Q|Summit|Washington County|MN|50499|United States|-6|condo| +37210|AAAAAAAAKFBJAAAA|16|Locust |Blvd|Suite Y|Spring Hill|Pike County|PA|16787|United States|-5|apartment| +37211|AAAAAAAALFBJAAAA|405|12th |Ct.|Suite T|Forest Hills|Weld County|CO|89237|United States|-7|condo| +37212|AAAAAAAAMFBJAAAA|773|Jackson Ash|Wy|Suite 420|Rutland|Garfield County|MT|68375|United States|-7|condo| +37213|AAAAAAAANFBJAAAA|205|Lake |Drive|Suite L|Jamestown|Lincoln County|TN|36867|United States|-6|apartment| +37214|AAAAAAAAOFBJAAAA|957|13th |RD|Suite 490|Deerfield|Jim Wells County|TX|79840|United States|-6|apartment| +37215|AAAAAAAAPFBJAAAA|723|South |Drive|Suite A|Green Acres|Wayne County|IN|47683|United States|-5|condo| +37216|AAAAAAAAAGBJAAAA|238|Valley Lake|Drive|Suite A|Cordova|Comanche County|TX|76938|United States|-6|apartment| +37217|AAAAAAAABGBJAAAA|348|6th 3rd|Boulevard|Suite 390|Burns|Ector County|TX|75272|United States|-6|apartment| +37218|AAAAAAAACGBJAAAA|580|2nd Sunset|Parkway|Suite 450|Lincoln|Durham County|NC|21289|United States|-5|single family| +37219|AAAAAAAADGBJAAAA|842|Oak 6th|||Salem|Greenville County|SC||United States|-5|| +37220|AAAAAAAAEGBJAAAA|752|Fifth |Cir.|Suite X|Mount Pleasant|Bexar County|TX|71933|United States|-6|condo| +37221|AAAAAAAAFGBJAAAA|510|Main Hill|Boulevard|Suite 170|Harmony|Stewart County|TN|35804|United States|-6|single family| +37222|AAAAAAAAGGBJAAAA|900|Park |Ave|Suite S|Lebanon|Bond County|IL|62898|United States|-6|apartment| +37223|AAAAAAAAHGBJAAAA|741|Second 9th|Ct.|Suite 80|Lenox|Douglas County|MO|61143|United States|-6|apartment| +37224|AAAAAAAAIGBJAAAA|637|Oak |Avenue|Suite J|Globe|Tazewell County|IL|69614|United States|-6|condo| +37225|AAAAAAAAJGBJAAAA|819|Cedar Jackson|Ave|Suite 130|Oak Hill|Randolph County|NC|27838|United States|-5|single family| +37226|AAAAAAAAKGBJAAAA|680|Highland Cedar|Parkway|Suite 190|Bridgeport|Butler County|KY|45817|United States|-6|condo| +37227|AAAAAAAALGBJAAAA|880|Poplar Lincoln|Blvd|Suite T|Woodville|Osceola County|IA|54289|United States|-6|single family| +37228|AAAAAAAAMGBJAAAA|409|Second |Road|Suite 260|Spring Valley|Clark County|WA|96060|United States|-8|condo| +37229|AAAAAAAANGBJAAAA|727|5th |Road|Suite W|Harmon|Ramsey County|MN|55623|United States|-6|single family| +37230|AAAAAAAAOGBJAAAA|701|Cherry |Circle|Suite A|Highland|Lawrence County|OH|49454|United States|-5|apartment| +37231|AAAAAAAAPGBJAAAA|849|Adams 9th|Wy|Suite D|Greenville|Elbert County|GA|31387|United States|-5|single family| +37232|AAAAAAAAAHBJAAAA|28|Elm |Pkwy|Suite 420|Maple Grove|Wilson County|KS|68252|United States|-6|condo| +37233|AAAAAAAABHBJAAAA|581|Washington Valley|Cir.|Suite R|Woodville|Reagan County|TX|74289|United States|-6|single family| +37234|AAAAAAAACHBJAAAA|109|14th |ST|Suite 230|Riverview|Gaines County|TX|79003|United States|-6|apartment| +37235|AAAAAAAADHBJAAAA|142|Oak Hickory|ST|Suite K|Union Hill|Allen County|OH|47746|United States|-5|apartment| +37236|AAAAAAAAEHBJAAAA|862|River Third|Dr.|Suite X|Greenville|San Miguel County|CO|81387|United States|-7|apartment| +37237|AAAAAAAAFHBJAAAA|427|12th |Drive|Suite 80|Waterloo|Kings County|CA|91675|United States|-8|condo| +37238|AAAAAAAAGHBJAAAA|685|Sixth Ridge|Avenue|Suite B|Shady Grove|Fairbanks North Star Borough|AK|92812|United States|-9|condo| +37239|AAAAAAAAHHBJAAAA|543|Locust |Avenue|Suite H|Liberty|Pike County|OH|43451|United States|-5|apartment| +37240|AAAAAAAAIHBJAAAA|436|Jackson |Way|Suite 390|Shady Grove|Wallace County|KS|62812|United States|-6|apartment| +37241|AAAAAAAAJHBJAAAA|134|Lake |Ct.|Suite 470|Hamilton|Cattaraugus County|NY|12808|United States|-5|condo| +37242|AAAAAAAAKHBJAAAA|368|Willow Main|Street|Suite 260|Five Points|Delaware County|IA|56098|United States|-6|condo| +37243|AAAAAAAALHBJAAAA|514|1st |Drive|Suite V|Spring Valley|Kane County|UT|86060|United States|-7|apartment| +37244|AAAAAAAAMHBJAAAA|735|Elm Woodland|Lane|Suite A|Deerfield|Ramsey County|MN|59840|United States|-6|condo| +37245|AAAAAAAANHBJAAAA|288|Lee |Wy|Suite E|Edwards|Charles City County|VA|21409|United States|-5|single family| +37246|AAAAAAAAOHBJAAAA|107|First |Way|Suite C|Highland Park|Stanislaus County|CA|96534|United States|-8|condo| +37247|AAAAAAAAPHBJAAAA|48|Maple |Lane|Suite 320|Acme|Crawford County|IN|40164|United States|-5|single family| +37248|AAAAAAAAAIBJAAAA|349|4th |Pkwy|Suite L|Pleasant Grove|Wilson County|NC|24136|United States|-5|apartment| +37249|AAAAAAAABIBJAAAA|592|7th |Circle|Suite 140|Sunnyside|Shelby County|IN|41952|United States|-5|single family| +37250|AAAAAAAACIBJAAAA|199|Fifth 5th|Cir.|Suite H|Lincoln|Newton County|MS|51289|United States|-6|single family| +37251|AAAAAAAADIBJAAAA|98|Meadow 6th|Pkwy|Suite 290|Five Points|Chester County|TN|36098|United States|-5|condo| +37252|AAAAAAAAEIBJAAAA|208|East |RD|Suite 160|Oak Ridge|Riverside County|CA|98371|United States|-8|condo| +37253|AAAAAAAAFIBJAAAA|542|Oak |Ave|Suite J|Forest Hills|Robertson County|TX|79237|United States|-6|single family| +37254|AAAAAAAAGIBJAAAA|734|Park |Pkwy|Suite 260|Edgewood|Owyhee County|ID|80069|United States|-7|apartment| +37255|AAAAAAAAHIBJAAAA|936|Laurel |Ct.|Suite 220|Greenville|Belknap County|NH|01987|United States|-5|apartment| +37256|AAAAAAAAIIBJAAAA|188|West |Road|Suite 130|Macedonia|Bradford County|PA|11087|United States|-5|condo| +37257|AAAAAAAAJIBJAAAA|99|2nd |Avenue|Suite D|Sunnyside|Coos County|OR|91952|United States|-8|single family| +37258|AAAAAAAAKIBJAAAA|353|South Elm|Parkway|Suite 460|Mount Zion|Dickinson County|MI|48054|United States|-5|apartment| +37259|AAAAAAAALIBJAAAA|439|Fifth Highland|Cir.|Suite 440|Hamilton|Stark County|OH|42808|United States|-5|condo| +37260|AAAAAAAAMIBJAAAA|554|Sunset |Wy|Suite Q|Providence|Cheboygan County|MI|46614|United States|-5|single family| +37261|AAAAAAAANIBJAAAA|878|Oak Ash|ST|Suite F|Jamestown|Cullman County|AL|36867|United States|-6|single family| +37262|AAAAAAAAOIBJAAAA|898|Railroad |Circle|Suite 380|Freeman|Franklin County|KS|62297|United States|-6|condo| +37263|AAAAAAAAPIBJAAAA|346|Church Lincoln|Blvd|Suite 100|Lebanon|Chattooga County|GA|32898|United States|-5|apartment| +37264|AAAAAAAAAJBJAAAA|621|Meadow |Way|Suite 420|Farmington|Holmes County|OH|49145|United States|-5|apartment| +37265|AAAAAAAABJBJAAAA|391|Cedar Sixth|Boulevard|Suite 20|Stringtown|Greene County|TN|30162|United States|-5|single family| +37266|AAAAAAAACJBJAAAA|72|Washington Ash|Boulevard|Suite 140|Summit|Wabash County|IN|40499|United States|-5|single family| +37267|AAAAAAAADJBJAAAA|375|Central |Pkwy|Suite R|Liberty|Montgomery County|IN|43451|United States|-5|apartment| +37268|AAAAAAAAEJBJAAAA|135|Ridge Lakeview|Dr.|Suite 260|Buena Vista|Parmer County|TX|75752|United States|-6|apartment| +37269|AAAAAAAAFJBJAAAA|128|2nd |Avenue|Suite S|Centerville|Terrebonne Parish|LA|70059|United States|-6|condo| +37270|AAAAAAAAGJBJAAAA|949|1st |Way|Suite 180|Hopewell|Tioga County|NY|10587|United States|-5|single family| +37271|AAAAAAAAHJBJAAAA|762|Sunset North|Wy|Suite 480|Farmington|Hyde County|NC|29145|United States|-5|condo| +37272|AAAAAAAAIJBJAAAA|248|Hill |Ave|Suite F|Maple Grove|Brazos County|TX|78252|United States|-6|apartment| +37273|AAAAAAAAJJBJAAAA|576|Elm |Road|Suite B|Weston|Buena Vista County|IA|51425|United States|-6|single family| +37274|AAAAAAAAKJBJAAAA|942|Main |Ln|Suite 20|Brunswick|Ripley County|MO|64642|United States|-6|single family| +37275|AAAAAAAALJBJAAAA|829|4th Woodland|Circle|Suite M|Leon|Brown County|IN|40913|United States|-5|condo| +37276|AAAAAAAAMJBJAAAA|593|Davis |Court|Suite X|Roxbury|Polk County|GA|35508|United States|-5|apartment| +37277|AAAAAAAANJBJAAAA|719|Locust |Court|Suite 240|Ashland|Bucks County|PA|14244|United States|-5|condo| +37278|AAAAAAAAOJBJAAAA|699|8th Woodland|Ave|Suite V|Mount Olive|Stevens County|WA|98059|United States|-8|condo| +37279|AAAAAAAAPJBJAAAA|228|2nd Fifth|Avenue|Suite M|White Oak|San Saba County|TX|76668|United States|-6|single family| +37280|AAAAAAAAAKBJAAAA|691|5th Spruce|Court|Suite 410|Red Hill|Decatur County|GA|34338|United States|-5|condo| +37281|AAAAAAAABKBJAAAA|481|2nd |ST|Suite K|Oakdale|Douglas County|CO|89584|United States|-7|single family| +37282|AAAAAAAACKBJAAAA|248|Center Ninth|Court|Suite 480|Forestville|Floyd County|GA|33027|United States|-5|apartment| +37283|AAAAAAAADKBJAAAA|775|Williams 12th|Way|Suite 220|Highland Park|Coconino County|AZ|86534|United States|-7|condo| +37284|AAAAAAAAEKBJAAAA|814|Miller Oak|Drive|Suite 310|Maple Grove|Plaquemines Parish|LA|78252|United States|-6|apartment| +37285|AAAAAAAAFKBJAAAA|101|Railroad Forest|Parkway|Suite G|Sulphur Springs|Hardin County|IA|58354|United States|-6|single family| +37286|AAAAAAAAGKBJAAAA|83|Franklin |Blvd|Suite F|Riverside|Adair County|IA|59231|United States|-6|single family| +37287|AAAAAAAAHKBJAAAA|409|Jackson |Parkway|Suite G|Whispering Pines|Livingston County|MI|47609|United States|-5|apartment| +37288|AAAAAAAAIKBJAAAA|791|7th 10th|Ave|Suite 80|Marion|Jersey County|IL|60399|United States|-6|condo| +37289|AAAAAAAAJKBJAAAA|231|Cherry |Drive|Suite 120|Hamilton|Scott County|IL|62808|United States|-6|single family| +37290|AAAAAAAAKKBJAAAA|44|Hickory |Road|Suite 400|Welcome|Guadalupe County|NM|86386|United States|-7|condo| +37291|AAAAAAAALKBJAAAA|908|Cedar Highland|RD|Suite A|Highland|Quitman County|GA|39454|United States|-5|single family| +37292|AAAAAAAAMKBJAAAA|309|Jefferson |Way|Suite 170|Harmony|Unicoi County|TN|35804|United States|-6|condo| +37293|AAAAAAAANKBJAAAA|489|Washington |Wy|Suite U|Greenfield|Hall County|TX|75038|United States|-6|single family| +37294|AAAAAAAAOKBJAAAA|5|Sunset Davis|Court|Suite G|Walnut|Henry County|IN|46245|United States|-5|single family| +37295|AAAAAAAAPKBJAAAA|513|Madison Franklin|Cir.|Suite M|Ashley|Webb County|TX|74324|United States|-6|condo| +37296|AAAAAAAAALBJAAAA|140|Lincoln |Cir.|Suite 430|Hillcrest|Tyrrell County|NC|23003|United States|-5|apartment| +37297|AAAAAAAABLBJAAAA|25|Walnut 2nd|ST|Suite 50|Mount Vernon|Milam County|TX|78482|United States|-6|condo| +37298|AAAAAAAACLBJAAAA|148|Johnson |Drive|Suite J|Union Hill|Erie County|NY|17746|United States|-5|single family| +37299|AAAAAAAADLBJAAAA|915|1st |Parkway|Suite 390|Bunker Hill|McPherson County|NE|60150|United States|-7|condo| +37300|AAAAAAAAELBJAAAA|814|Spring |Dr.|Suite 420|Highland Park|Franklin County|KY|46534|United States|-6|single family| +37301|AAAAAAAAFLBJAAAA|292|9th |Ln|Suite 50|Five Points|DeWitt County|TX|76098|United States|-6|apartment| +37302|AAAAAAAAGLBJAAAA|910|Spring 7th|Way|Suite 250|Sulphur Springs|Dale County|AL|38354|United States|-6|apartment| +37303|AAAAAAAAHLBJAAAA|984|Oak Dogwood|ST|Suite 440|New Hope|Franklin County|ID|89431|United States|-7|single family| +37304|AAAAAAAAILBJAAAA|623|11th |Ave|Suite X|Five Forks|Lincoln County|WA|92293|United States|-8|apartment| +37305|AAAAAAAAJLBJAAAA|255|1st |ST|Suite 170|Hopewell|Martin County|NC|20587|United States|-5|condo| +37306|AAAAAAAAKLBJAAAA|237|Elm Hickory|Cir.|Suite 90|Plainview|Eau Claire County|WI|53683|United States|-6|apartment| +37307|AAAAAAAALLBJAAAA|177|8th |ST|Suite G|Belmont|Knox County|NE|60191|United States|-7|apartment| +37308|AAAAAAAAMLBJAAAA|114|12th 10th|Avenue|Suite 360|||||||| +37309|AAAAAAAANLBJAAAA|782|Spring |Circle|Suite P|Brownsville|Randolph County|NC|29310|United States|-5|apartment| +37310|AAAAAAAAOLBJAAAA|731|Adams |RD|Suite O|Burns|Allegan County|MI|45272|United States|-5|condo| +37311|AAAAAAAAPLBJAAAA|78|2nd First|Cir.|Suite 120|Payne|Walla Walla County|WA|96134|United States|-8|apartment| +37312|AAAAAAAAAMBJAAAA|598|Jackson Hickory|Pkwy|Suite N|Sidney|Lucas County|IA|53530|United States|-6|condo| +37313|AAAAAAAABMBJAAAA|122|4th 1st|Boulevard|Suite P|Jamestown|Hopewell city|VA|26867|United States|-5|apartment| +37314|AAAAAAAACMBJAAAA|525|Third |Street|Suite 330|Marion|Fayette County|IN|40399|United States|-5|single family| +37315|AAAAAAAADMBJAAAA|224|West Sunset|Ln|Suite 250|Mount Pleasant|Zapata County|TX|71933|United States|-6|single family| +37316|AAAAAAAAEMBJAAAA|705|North Ridge|Wy|Suite O|White Oak|Dent County|MO|66668|United States|-6|single family| +37317|AAAAAAAAFMBJAAAA|990|Main |ST|Suite 310|Morris|Barry County|MO|66696|United States|-6|apartment| +37318|AAAAAAAAGMBJAAAA|358|Davis |Ave|Suite 90|Bridgeport|Jackson County|WV|25817|United States|-5|single family| +37319|AAAAAAAAHMBJAAAA|634|10th First|Blvd|Suite 470|Woodland|Winneshiek County|IA|54854|United States|-6|condo| +37320|AAAAAAAAIMBJAAAA|845|7th Pine|Court|Suite 430|Providence|Oswego County|NY|16614|United States|-5|single family| +37321|AAAAAAAAJMBJAAAA|357|9th |Lane|Suite L|Wildwood|Potter County|PA|16871|United States|-5|single family| +37322|AAAAAAAAKMBJAAAA||Fourteenth |Street||Hardy||MN|||-6|apartment| +37323|AAAAAAAALMBJAAAA|565|Washington Railroad|Ln|Suite 0|Newtown|Dunn County|ND|51749|United States|-6|apartment| +37324|AAAAAAAAMMBJAAAA|863|Hill |Dr.|Suite P|Bunker Hill|Calhoun County|SC|20150|United States|-5|condo| +37325|AAAAAAAANMBJAAAA|823|Miller |Road|Suite A|Midway|Turner County|SD|51904|United States|-7|condo| +37326|AAAAAAAAOMBJAAAA|67|9th |Court|Suite P|Kirkland|Modoc County|CA|97896|United States|-8|condo| +37327|AAAAAAAAPMBJAAAA|293|Park |Blvd|Suite T|New Hope|Virginia Beach city|VA|29431|United States|-5|condo| +37328|AAAAAAAAANBJAAAA|545|Second |Ln|Suite 360|Kingston|Livingston County|NY|14975|United States|-5|apartment| +37329|AAAAAAAABNBJAAAA|52|View Jefferson|Ln|Suite O|Louisville|Gaines County|TX|74464|United States|-6|apartment| +37330|AAAAAAAACNBJAAAA|127|Spring 12th|Drive|Suite M|Walnut Grove|McCone County|MT|67752|United States|-7|condo| +37331|AAAAAAAADNBJAAAA|962|Valley |Parkway|Suite G|Highland|Madison County|OH|49454|United States|-5|single family| +37332|AAAAAAAAENBJAAAA|681|Poplar |Drive|Suite 370|Hillcrest|Jones County|GA|33003|United States|-5|single family| +37333|AAAAAAAAFNBJAAAA|742|River 1st|Dr.|Suite 450|Mount Vernon|Cowley County|KS|68482|United States|-6|apartment| +37334|AAAAAAAAGNBJAAAA|617|1st College|Blvd|Suite X|Springfield|Highlands County|FL|39303|United States|-5|single family| +37335|AAAAAAAAHNBJAAAA|43|Sixth |RD|Suite T|Stringtown|Page County|VA|20162|United States|-5|single family| +37336|AAAAAAAAINBJAAAA|669|Washington |Lane|Suite 0|Sulphur Springs|Hall County|GA|38354|United States|-5|apartment| +37337|AAAAAAAAJNBJAAAA|495|Maple |Street|Suite 170|Forest Hills|Dakota County|MN|59237|United States|-6|apartment| +37338|AAAAAAAAKNBJAAAA|347|Lincoln North|Way|Suite T|Fairview|Caroline County|MD|25709|United States|-5|condo| +37339|AAAAAAAALNBJAAAA|909|Adams |Pkwy|Suite B|Mount Pleasant|Craven County|NC|21933|United States|-5|condo| +37340|AAAAAAAAMNBJAAAA|924|Ash |Cir.|Suite B|Greenwood|York County|VA|28828|United States|-5|condo| +37341|AAAAAAAANNBJAAAA|144|1st Washington|Pkwy|Suite 480|Harmony|Denali Borough|AK|95804|United States|-9|condo| +37342|AAAAAAAAONBJAAAA|132|Sunset |Avenue|Suite B|Wildwood|Franklin County|MS|56871|United States|-6|apartment| +37343|AAAAAAAAPNBJAAAA|277|Jackson Second|Parkway|Suite I|Deerfield|Chase County|NE|69840|United States|-6|apartment| +37344|AAAAAAAAAOBJAAAA|559|First |Street|Suite Q|Woodlawn|Clinch County|GA|34098|United States|-5|apartment| +37345|AAAAAAAABOBJAAAA|60|Central Walnut|Circle|Suite 340|Pleasant Hill|Grant County|NE|63604|United States|-6|apartment| +37346|AAAAAAAACOBJAAAA|587|8th |RD|Suite D|Marion|Bleckley County|GA|30399|United States|-5|single family| +37347|AAAAAAAADOBJAAAA|219|Cherry |Parkway|Suite Q|Bethel|Bristol County|MA|05881|United States|-5|condo| +37348|AAAAAAAAEOBJAAAA|393|Church |Dr.|Suite E|Jackson|Metcalfe County|KY|49583|United States|-5|condo| +37349|AAAAAAAAFOBJAAAA|388|Park |Avenue|Suite 130|Pine Grove|Escambia County|AL|34593|United States|-6|apartment| +37350|AAAAAAAAGOBJAAAA|227|Johnson 1st|Cir.|Suite 10|Lincoln|Durham County|NC|21289|United States|-5|apartment| +37351|AAAAAAAAHOBJAAAA|945|Spring |Boulevard|Suite 240|Five Forks|Ellsworth County|KS|62293|United States|-6|apartment| +37352|AAAAAAAAIOBJAAAA|30|Center |Way|Suite 360|Fairfield|Seward County|KS|66192|United States|-6|single family| +37353|AAAAAAAAJOBJAAAA|799|Maple 13th|Boulevard|Suite 30|Mountain View|Osceola County|MI|44466|United States|-5|single family| +37354|AAAAAAAAKOBJAAAA|89|2nd Walnut|Ct.|Suite 440|Hillcrest|Platte County|NE|63003|United States|-7|single family| +37355|AAAAAAAALOBJAAAA|172|View |Road|Suite K|Woodruff|Aurora County|SD|54174|United States|-6|condo| +37356|AAAAAAAAMOBJAAAA|894|Park Ninth|Wy|Suite 130|Andover|Butts County|GA|31639|United States|-5|condo| +37357|AAAAAAAANOBJAAAA|518|Main Second|Lane|Suite 10|Woodland|Towner County|ND|54854|United States|-6|apartment| +37358|AAAAAAAAOOBJAAAA|424|Forest |Circle|Suite 70|Fairfield|Williamsburg County|SC|26192|United States|-5|apartment| +37359|AAAAAAAAPOBJAAAA|983|Maple |Parkway|Suite R|||||United States||| +37360|AAAAAAAAAPBJAAAA|723|5th |Avenue|Suite M|White Oak|Panola County|TX|76668|United States|-6|apartment| +37361|AAAAAAAABPBJAAAA|106|Dogwood |Pkwy|Suite 160|Clinton|Somerset County|ME|08822|United States|-5|apartment| +37362|AAAAAAAACPBJAAAA||||Suite W||||29840|United States|-5|single family| +37363|AAAAAAAADPBJAAAA|254|Chestnut 1st|Ln|Suite 60|Riverside|Franklin County|KS|69231|United States|-6|apartment| +37364|AAAAAAAAEPBJAAAA|886|Jefferson 1st|Road|Suite 230|Hamilton|Page County|IA|52808|United States|-6|single family| +37365|AAAAAAAAFPBJAAAA|744|Hill |RD|Suite E|Summit|Chenango County|NY|10499|United States|-5|apartment| +37366|AAAAAAAAGPBJAAAA|391|Seventh |Lane|Suite 190|Brookwood|Hutchinson County|SD|50965|United States|-7|condo| +37367|AAAAAAAAHPBJAAAA|914|Meadow |ST|Suite 300|Enterprise|Knox County|NE|61757|United States|-7|apartment| +37368|AAAAAAAAIPBJAAAA|346|Lee 4th|ST|Suite J|Arlington|Archer County|TX|76557|United States|-6|single family| +37369|AAAAAAAAJPBJAAAA|281|Franklin |Pkwy|Suite 360|Plainview|Conejos County|CO|83683|United States|-7|condo| +37370|AAAAAAAAKPBJAAAA|832|Main |Avenue|Suite 320|Wyoming|Henderson County|NC|20216|United States|-5|apartment| +37371|AAAAAAAALPBJAAAA|257|11th Laurel|Ct.|Suite K|Sulphur Springs|Jefferson County|AR|78354|United States|-6|single family| +37372|AAAAAAAAMPBJAAAA|745|Birch Second|Road|Suite E|||||United States||| +37373|AAAAAAAANPBJAAAA|396|2nd |Street|Suite X|Kingston|Rock County|MN|54975|United States|-6|single family| +37374|AAAAAAAAOPBJAAAA|428|Third |Cir.|Suite 170|Shady Grove|Carson County|TX|72812|United States|-6|condo| +37375|AAAAAAAAPPBJAAAA|814|Main Walnut|Boulevard|Suite L|Spring Valley|Mower County|MN|56060|United States|-6|single family| +37376|AAAAAAAAAACJAAAA|138|First |Pkwy|Suite 490|Riverside|Monongalia County|WV|29231|United States|-5|apartment| +37377|AAAAAAAABACJAAAA|493|Park Cedar|Court|Suite C|Hopewell|Mesa County|CO|80587|United States|-7|apartment| +37378|AAAAAAAACACJAAAA|210|Cedar 2nd|Court|Suite O|Clifton|Coconino County|AZ|88014|United States|-7|single family| +37379|AAAAAAAADACJAAAA|874|Seventh |Street|Suite 370|Springdale|Laurens County|GA|38883|United States|-5|single family| +37380|AAAAAAAAEACJAAAA|693|Second |Street|Suite 120|Plainville|Chickasaw County|IA|56115|United States|-6|apartment| +37381|AAAAAAAAFACJAAAA|493|Chestnut Williams|Way|Suite D|Centerville|Ford County|IL|60059|United States|-6|condo| +37382|AAAAAAAAGACJAAAA|670|Elm |Street|Suite R|Cedar Grove|Turner County|SD|50411|United States|-7|apartment| +37383|AAAAAAAAHACJAAAA|875|||||Knox County||||-5|| +37384|AAAAAAAAIACJAAAA|799|Broadway Broadway|Street|Suite B|Salem|Cobb County|GA|38048|United States|-5|condo| +37385|AAAAAAAAJACJAAAA|297|Lincoln Wilson|Avenue|Suite C|Pine Grove|Stone County|AR|74593|United States|-6|apartment| +37386|AAAAAAAAKACJAAAA|696|Hickory |Boulevard|Suite 390|Sullivan|Hartley County|TX|70451|United States|-6|condo| +37387|AAAAAAAALACJAAAA|669|14th Sunset|Court|Suite N|Florence|Crockett County|TX|73394|United States|-6|apartment| +37388|AAAAAAAAMACJAAAA|59|Tenth |Street|Suite 130|Oakdale|Maury County|TN|39584|United States|-6|single family| +37389|AAAAAAAANACJAAAA|570|Mill |Way|Suite L|Walnut Grove|Grainger County|TN|37752|United States|-5|single family| +37390|AAAAAAAAOACJAAAA|341|Washington |Avenue|Suite 30|Friendship|Staunton city|VA|24536|United States|-5|apartment| +37391|AAAAAAAAPACJAAAA|429|Spring 12th|ST|Suite Y|Oak Hill|Gwinnett County|GA|37838|United States|-5|condo| +37392|AAAAAAAAABCJAAAA|815|Central |Pkwy|Suite 310|Bethel|Madison County|FL|35281|United States|-5|condo| +37393|AAAAAAAABBCJAAAA|536|7th |Pkwy|Suite H|Highland|Washington County|KY|49454|United States|-5|condo| +37394|AAAAAAAACBCJAAAA|598|Williams |Road|Suite J|Woodville|Guadalupe County|NM|84289|United States|-7|apartment| +37395|AAAAAAAADBCJAAAA|503|First |Road|Suite 270|Highland Park|Warren County|NC|26534|United States|-5|apartment| +37396|AAAAAAAAEBCJAAAA|122|Ridge |Parkway|Suite 290|Bethel|Hale County|TX|75281|United States|-6|single family| +37397|AAAAAAAAFBCJAAAA|70|Oak |Court|Suite 280|Oakland|Tyrrell County|NC|29843|United States|-5|condo| +37398|AAAAAAAAGBCJAAAA|267|Forest |Way|Suite 330|Salem|Garfield County|OK|78048|United States|-6|single family| +37399|AAAAAAAAHBCJAAAA|172|Hill Sunset|Way|Suite R|Pleasant Hill|Pulaski County|IL|63604|United States|-6|single family| +37400|AAAAAAAAIBCJAAAA|91|10th 6th|Dr.|Suite 0|Shady Grove|Morris County|KS|62812|United States|-6|condo| +37401|AAAAAAAAJBCJAAAA|852|East College|Way|Suite 50|Red Hill|Bon Homme County|SD|54338|United States|-6|apartment| +37402|AAAAAAAAKBCJAAAA|954|7th 12th|Cir.|Suite H|Hopewell|Sheridan County|ND|50587|United States|-6|apartment| +37403|AAAAAAAALBCJAAAA|391|Cedar Cherry|Pkwy|Suite Q|Kingston|Mason County|TX|74975|United States|-6|condo| +37404|AAAAAAAAMBCJAAAA|217|Mill |Way|Suite 130|Florence|Franklin County|ME|03994|United States|-5|single family| +37405|AAAAAAAANBCJAAAA|151|Woodland |Cir.|Suite V|Springdale|Monroe County|MO|68883|United States|-6|single family| +37406|AAAAAAAAOBCJAAAA|800|6th |Parkway|Suite 450|Buckingham|Wabaunsee County|KS|64092|United States|-6|single family| +37407|AAAAAAAAPBCJAAAA|427|Highland Oak|Street|Suite O|Antioch|Guernsey County|OH|48605|United States|-5|single family| +37408|AAAAAAAAACCJAAAA|779|3rd Highland|Court|Suite 10|Waterloo|Brown County|WI|51675|United States|-6|single family| +37409|AAAAAAAABCCJAAAA|679|First |Drive|Suite 380|Deerfield|Canyon County|ID|89840|United States|-7|single family| +37410|AAAAAAAACCCJAAAA|471|Green |Pkwy|Suite N|New Hope|Brazos County|TX|79431|United States|-6|condo| +37411|AAAAAAAADCCJAAAA|435|Hillcrest |Ct.|Suite N|Bridgeport|Hunt County|TX|75817|United States|-6|condo| +37412|AAAAAAAAECCJAAAA|968|Seventh Spruce|Boulevard|Suite 450|Forest Hills|Texas County|MO|69237|United States|-6|single family| +37413|AAAAAAAAFCCJAAAA|286|5th Spruce|Ln|Suite 350|Hamilton|Pacific County|WA|92808|United States|-8|single family| +37414|AAAAAAAAGCCJAAAA|510|Hickory Park|Parkway|Suite 180|Spring Hill|Van Buren County|MI|46787|United States|-5|single family| +37415|AAAAAAAAHCCJAAAA|835|Walnut West|Street|Suite K|Providence|Mason County|MI|46614|United States|-5|single family| +37416|AAAAAAAAICCJAAAA|334|Railroad |Lane|Suite 40|Arlington|Fremont County|CO|86557|United States|-7|single family| +37417|AAAAAAAAJCCJAAAA|661|Ridge Central|Dr.|Suite 280|Marion|Steuben County|IN|40399|United States|-5|single family| +37418|AAAAAAAAKCCJAAAA|659|Meadow Ash|Wy|Suite X|Lakewood|Albemarle County|VA|28877|United States|-5|apartment| +37419|AAAAAAAALCCJAAAA|862|Sycamore |Blvd|Suite 350|Jamestown|Dickinson County|IA|56867|United States|-6|apartment| +37420|AAAAAAAAMCCJAAAA|485|Jefferson Lee|Wy|Suite L|Kingston|Newaygo County|MI|44975|United States|-5|single family| +37421|AAAAAAAANCCJAAAA|728|||Suite R||||59583|United States|-6|condo| +37422|AAAAAAAAOCCJAAAA|951|6th Walnut|Dr.|Suite 410|Plainview|Ozaukee County|WI|53683|United States|-6|single family| +37423|AAAAAAAAPCCJAAAA|57|Third |Parkway|Suite 400|Concord|Amador County|CA|94107|United States|-8|apartment| +37424|AAAAAAAAADCJAAAA|968|Lee Oak|Ct.|Suite I|Hopewell|Alleghany County|VA|20587|United States|-5|single family| +37425|AAAAAAAABDCJAAAA|734|Third 11th|Ave|Suite 320|Highland|Cochran County|TX|79454|United States|-6|apartment| +37426|AAAAAAAACDCJAAAA|894|7th |Parkway|Suite U|Spring Valley|Menominee County|MI|46060|United States|-5|single family| +37427|AAAAAAAADDCJAAAA|768|Lincoln Sunset|Drive|Suite I|California|Loudon County|TN|30141|United States|-6|apartment| +37428|AAAAAAAAEDCJAAAA|340|Adams 7th|Circle|Suite 240|Georgetown|Chickasaw County|MS|57057|United States|-6|condo| +37429|AAAAAAAAFDCJAAAA|719|Fifth Central|Cir.|Suite 260|Lakewood|Cass County|TX|78877|United States|-6|condo| +37430|AAAAAAAAGDCJAAAA|327|Ninth |Ct.|Suite 240|Lakeside|Zavala County|TX|79532|United States|-6|single family| +37431|AAAAAAAAHDCJAAAA|92|Church |RD|Suite Q|Five Points|Washington County|MD|26098|United States|-5|single family| +37432|AAAAAAAAIDCJAAAA|838|North |Ct.|Suite 160|Mount Pleasant|Bear Lake County|ID|81933|United States|-7|single family| +37433|AAAAAAAAJDCJAAAA|489|Pine |Ct.|Suite 330|Oakwood|Webb County|TX|70169|United States|-6|apartment| +37434|AAAAAAAAKDCJAAAA|386|Church |Circle|Suite 150|Wesley|Hale County|AL|31218|United States|-6|single family| +37435|AAAAAAAALDCJAAAA|511|2nd Highland|Cir.|Suite 180|Calhoun|Clinton County|NY|16909|United States|-5|condo| +37436|AAAAAAAAMDCJAAAA|269|Cedar Oak|Way|Suite 170|Whitesville|Custer County|MT|65903|United States|-7|single family| +37437|AAAAAAAANDCJAAAA|29|Highland 2nd|Drive|Suite Q|White Oak|Wabash County|IN|46668|United States|-5|apartment| +37438|AAAAAAAAODCJAAAA|441|Johnson Walnut|Blvd|Suite 440|Morris|Greene County|IN|46696|United States|-5|single family| +37439|AAAAAAAAPDCJAAAA|874|12th Center|Avenue|Suite Y|Jamestown|San Francisco County|CA|96867|United States|-8|condo| +37440|AAAAAAAAAECJAAAA|757|View Fifth|Drive|Suite D|Glenwood|Nuckolls County|NE|63511|United States|-7|single family| +37441|AAAAAAAABECJAAAA|743|Cedar River|Pkwy|Suite L|Oak Grove|Ottawa County|OH|48370|United States|-5|apartment| +37442|AAAAAAAACECJAAAA|803|Dogwood Pine|Blvd|Suite T|Glenwood|Champaign County|IL|63511|United States|-6|single family| +37443|AAAAAAAADECJAAAA|547|Park |Pkwy|Suite 390|Plainview|Wasatch County|UT|83683|United States|-7|single family| +37444|AAAAAAAAEECJAAAA|87||Court|Suite S||Sierra County|CA|||-8|single family| +37445|AAAAAAAAFECJAAAA|250|Park |Court|Suite C|Springdale|Cibola County|NM|88883|United States|-7|single family| +37446|AAAAAAAAGECJAAAA|21|3rd South|Cir.|Suite A|Walnut Grove|Beaver County|PA|17752|United States|-5|single family| +37447|AAAAAAAAHECJAAAA|621|Walnut ||||Mountrail County|ND|56867|United States||| +37448|AAAAAAAAIECJAAAA|517|Elm |Avenue|Suite S|Sunnyside|Braxton County|WV|21952|United States|-5|condo| +37449|AAAAAAAAJECJAAAA|985|College |Avenue|Suite 450|Springdale|Hill County|MT|68883|United States|-7|apartment| +37450|AAAAAAAAKECJAAAA|573|Sunset River|Street|Suite O|Springfield|Bourbon County|KY|49303|United States|-6|single family| +37451|AAAAAAAALECJAAAA|323|1st |Circle|Suite R|Salem|McLean County|IL|68048|United States|-6|apartment| +37452|AAAAAAAAMECJAAAA|130|4th Johnson|Blvd|Suite M|Riverside|Knox County|KY|49231|United States|-5|condo| +37453|AAAAAAAANECJAAAA|547|Green |Circle|Suite S|Greenwood|Reeves County|TX|78828|United States|-6|apartment| +37454|AAAAAAAAOECJAAAA|600|Hickory |Court|Suite 410|Clinton|Eureka County|NV|88222|United States|-8|single family| +37455|AAAAAAAAPECJAAAA|754|Sycamore Lee|Cir.|Suite V|Riverdale|Clarke County|GA|39391|United States|-5|apartment| +37456|AAAAAAAAAFCJAAAA|923|Ridge Oak|Street|Suite H|Deerfield|Charlevoix County|MI|49840|United States|-5|single family| +37457|AAAAAAAABFCJAAAA|21|Lincoln |Street|Suite 290|Glendale|Greenville County|SC|23951|United States|-5|apartment| +37458|AAAAAAAACFCJAAAA|356|||Suite 130||||54975|||single family| +37459|AAAAAAAADFCJAAAA|938|2nd |Boulevard|Suite Y|Springfield|Vermilion County|IL|69303|United States|-6|condo| +37460|AAAAAAAAEFCJAAAA|320|Thirteenth 6th|Ave|Suite Y|Summit|Meigs County|TN|30499|United States|-6|single family| +37461|AAAAAAAAFFCJAAAA|759|Adams West|Blvd|Suite F|Belmont|Richland County|ND|50191|United States|-6|apartment| +37462|AAAAAAAAGFCJAAAA|251|Locust |Street|Suite 270|Centerville|Oklahoma County|OK|70059|United States|-6|apartment| +37463|AAAAAAAAHFCJAAAA|332|Ash |Avenue|Suite G|Lincoln|Laclede County|MO|61289|United States|-6|single family| +37464|AAAAAAAAIFCJAAAA|142|East 8th|Wy|Suite 280|Bridgeport|Barnwell County|SC|25817|United States|-5|apartment| +37465|AAAAAAAAJFCJAAAA|603|Willow Lincoln|Dr.|Suite 130|Highland|Strafford County|NH|09454|United States|-5|single family| +37466|AAAAAAAAKFCJAAAA|139|Center |Dr.|Suite 280|Buena Vista|Wheeler County|OR|95752|United States|-8|apartment| +37467|AAAAAAAALFCJAAAA|697|Lake |Road|Suite W|Union|Brunswick County|VA|28721|United States|-5|apartment| +37468|AAAAAAAAMFCJAAAA|826|1st Hickory|Ln|Suite 430|Mount Pleasant|Levy County|FL|31933|United States|-5|condo| +37469|AAAAAAAANFCJAAAA|885|Lake |Parkway|Suite R|Belfast|Rensselaer County|NY|10125|United States|-5|condo| +37470|AAAAAAAAOFCJAAAA|764|Ash Fifth|Wy|Suite 220|Greenwood|Pleasants County|WV|28828|United States|-5|apartment| +37471|AAAAAAAAPFCJAAAA|191|Main 10th|Lane|Suite 210|Hamilton|Calumet County|WI|52808|United States|-6|single family| +37472|AAAAAAAAAGCJAAAA|868|Eigth Hillcrest|Blvd|Suite 220|Sunnyside|McCurtain County|OK|71952|United States|-6|condo| +37473|AAAAAAAABGCJAAAA|59|Third 3rd|Court|Suite 480|Brownsville|Knox County|TN|39310|United States|-6|condo| +37474|AAAAAAAACGCJAAAA|542|Hill Oak|ST|Suite S|Newtown|Hillsborough County|FL|31749|United States|-5|apartment| +37475|AAAAAAAADGCJAAAA|385|Washington |Boulevard|Suite A|Lincoln|Rutland County|VT|01889|United States|-5|condo| +37476|AAAAAAAAEGCJAAAA|138|4th |Dr.|Suite J|Riverview|Dearborn County|IN|49003|United States|-5|single family| +37477|AAAAAAAAFGCJAAAA|477|Second |Dr.|Suite I|Shore Acres|Smith County|TX|72724|United States|-6|single family| +37478|AAAAAAAAGGCJAAAA|445|Hill South|Avenue|Suite 140|Scottsville|Lafourche Parish|LA|74190|United States|-6|condo| +37479|AAAAAAAAHGCJAAAA|955|11th 14th|Road|Suite I|Liberty|Power County|ID|83451|United States|-7|single family| +37480|AAAAAAAAIGCJAAAA|599|First Hillcrest|Ave|Suite 90|Lebanon|Johnson County|AR|72898|United States|-6|apartment| +37481|AAAAAAAAJGCJAAAA|149|4th |ST|Suite D|Mountain View|Clark County|KS|64466|United States|-6|apartment| +37482|AAAAAAAAKGCJAAAA|706|Wilson Hillcrest|Ave|Suite Q|Wilson|Cullman County|AL|36971|United States|-6|apartment| +37483|AAAAAAAALGCJAAAA|224|Elm Walnut|Lane|Suite 150|Oakwood|Coconino County|AZ|80169|United States|-7|condo| +37484|AAAAAAAAMGCJAAAA|582|4th East|Wy|Suite 150|Mount Pleasant|Tipton County|IN|41933|United States|-5|apartment| +37485|AAAAAAAANGCJAAAA|802|Elm Railroad|Lane|Suite 50|Oakdale|Jones County|TX|79584|United States|-6|single family| +37486|AAAAAAAAOGCJAAAA|808|8th Miller|Drive|Suite H|Hamilton|Spink County|SD|52808|United States|-7|apartment| +37487|AAAAAAAAPGCJAAAA||Highland 6th|Lane|Suite 90|||MS|50534|||condo| +37488|AAAAAAAAAHCJAAAA|662|West |Avenue|Suite 50|Lee|Union County|OH|40408|United States|-5|single family| +37489|AAAAAAAABHCJAAAA|646|Mill Green|Avenue|Suite 340|Spring Valley|Muskogee County|OK|76060|United States|-6|single family| +37490|AAAAAAAACHCJAAAA|383|Second Mill|Wy|Suite 180|Liberty|Bay County|FL|33451|United States|-5|single family| +37491|AAAAAAAADHCJAAAA|894|Ash |RD|Suite 380|Lakewood|Divide County|ND|58877|United States|-6|apartment| +37492|AAAAAAAAEHCJAAAA|572|River Willow|Dr.|Suite W|Kingston|Niobrara County|WY|84975|United States|-7|apartment| +37493|AAAAAAAAFHCJAAAA|||Circle||Centerville|Adams County|IA|||-6|apartment| +37494|AAAAAAAAGHCJAAAA|119|Sunset Park|Dr.|Suite N|Jackson|Androscoggin County|ME|09583|United States|-5|apartment| +37495|AAAAAAAAHHCJAAAA|989|River Chestnut|Court|Suite 190|Birmingham|Kennebec County|ME|03972|United States|-5|single family| +37496|AAAAAAAAIHCJAAAA|249|Washington |Circle|Suite 250|Newtown|Pipestone County|MN|51749|United States|-6|condo| +37497|AAAAAAAAJHCJAAAA|511|Green |Boulevard|Suite D|Cedar Grove|Aleutians East Borough|AK|90411|United States|-9|apartment| +37498|AAAAAAAAKHCJAAAA|917|3rd 12th|Drive|Suite 490|Glenwood|Decatur County|IA|53511|United States|-6|apartment| +37499|AAAAAAAALHCJAAAA|538|6th Central|RD|Suite 220|Glenwood|Hamilton County|IN|43511|United States|-5|single family| +37500|AAAAAAAAMHCJAAAA|220|Lincoln |Ave|Suite A|Kirkland|Christian County|MO|67896|United States|-6|apartment| +37501|AAAAAAAANHCJAAAA|838|Jackson |Wy|Suite 410|Waterloo|Nuckolls County|NE|61675|United States|-7|apartment| +37502|AAAAAAAAOHCJAAAA|581|10th |Parkway|Suite 130|Lewis|Cowlitz County|WA|97066|United States|-8|condo| +37503|AAAAAAAAPHCJAAAA|923|Birch 7th|Blvd|Suite 0|Walnut Grove|Menifee County|KY|47752|United States|-5|apartment| +37504|AAAAAAAAAICJAAAA|695|Miller |Drive|Suite A|Plainview|Limestone County|TX|73683|United States|-6|single family| +37505|AAAAAAAABICJAAAA|485|South |Wy|Suite B|Harmony|Williamson County|TN|35804|United States|-5|single family| +37506|AAAAAAAACICJAAAA|409|Main |Cir.|Suite F|Clinton|Monroe County|IA|58222|United States|-6|condo| +37507|AAAAAAAADICJAAAA|247|Adams Church|Avenue|Suite 470|Oakdale|Cherry County|NE|69584|United States|-6|apartment| +37508|AAAAAAAAEICJAAAA|463|South 7th|Dr.|Suite Q|Highland Park|Sevier County|UT|86534|United States|-7|single family| +37509|AAAAAAAAFICJAAAA|314|North |RD|Suite P|Hopewell|Colfax County|NE|60587|United States|-6|condo| +37510|AAAAAAAAGICJAAAA|867|Forest |Pkwy|Suite K|Bunker Hill|Vilas County|WI|50150|United States|-6|apartment| +37511|AAAAAAAAHICJAAAA|322|Railroad |Lane|Suite E|Forest Hills|New Madrid County|MO|69237|United States|-6|apartment| +37512|AAAAAAAAIICJAAAA|397|Valley |Lane|Suite 180|Farmington|Lyon County|MN|59145|United States|-6|condo| +37513|AAAAAAAAJICJAAAA|350|3rd |ST|Suite I|Oak Grove|Randolph County|IL|68370|United States|-6|apartment| +37514|AAAAAAAAKICJAAAA|540|Maple |Avenue|Suite B|Woodville|Warren County|NJ|04889|United States|-5|single family| +37515|AAAAAAAALICJAAAA|400|Railroad |Ln|Suite 70|Macedonia|McCreary County|KY|41087|United States|-5|apartment| +37516|AAAAAAAAMICJAAAA|777|Spring Jackson|Blvd|Suite 390|Maple Grove|Henry County|TN|38252|United States|-5|single family| +37517|AAAAAAAANICJAAAA|529|Madison |Blvd|Suite 70|Gary|Hernando County|FL|30418|United States|-5|condo| +37518|AAAAAAAAOICJAAAA|621|Ninth 13th|Drive|Suite J|Springfield|Schuyler County|NY|19303|United States|-5|condo| +37519|AAAAAAAAPICJAAAA|981|6th |Drive|Suite 450|Forestville|Mineral County|WV|23027|United States|-5|condo| +37520|AAAAAAAAAJCJAAAA|766|3rd |Cir.|Suite T|Mount Pleasant|Hancock County|ME|02533|United States|-5|apartment| +37521|AAAAAAAABJCJAAAA|153|College |RD|Suite 360|Clifton|Calhoun County|AL|38014|United States|-6|condo| +37522|AAAAAAAACJCJAAAA|403|Wilson |Avenue|Suite 0|Belmont|Carroll County|NH|00791|United States|-5|apartment| +37523|AAAAAAAADJCJAAAA|535|Washington |Lane|Suite 260|Woodlawn|Wheatland County|MT|64098|United States|-7|condo| +37524|AAAAAAAAEJCJAAAA|543|Madison |ST|Suite Y|Rolling Hills|Collingsworth County|TX|77272|United States|-6|single family| +37525|AAAAAAAAFJCJAAAA|643|9th |Parkway|Suite 390|Elba|Washington County|NE|60262|United States|-7|single family| +37526|AAAAAAAAGJCJAAAA|888|Lincoln Pine|Wy|Suite K|White Oak|Putnam County|TN|36668|United States|-6|single family| +37527|AAAAAAAAHJCJAAAA|244|Third Sunset|Lane|Suite K|White Oak|Cumberland County|ME|07268|United States|-5|condo| +37528|AAAAAAAAIJCJAAAA|600|Church |Dr.|Suite Q|Stringtown|Manassas city|VA|20162|United States|-5|apartment| +37529|AAAAAAAAJJCJAAAA|536|Elm |RD|Suite L|New Hope|Miller County|MO|69431|United States|-6|apartment| +37530|AAAAAAAAKJCJAAAA|3|Jackson |Blvd|Suite N|Summit|Shiawassee County|MI|40499|United States|-5|single family| +37531|AAAAAAAALJCJAAAA|157|River Hill|Ave|Suite H|Clinton|Forsyth County|GA|38222|United States|-5|apartment| +37532|AAAAAAAAMJCJAAAA|661|Lakeview |Ave|Suite J|Oak Grove|Lac qui Parle County|MN|58370|United States|-6|apartment| +37533|AAAAAAAANJCJAAAA|775|Ash 6th|ST|Suite Y|Greenville|Colusa County|CA|91387|United States|-8|single family| +37534|AAAAAAAAOJCJAAAA|231|8th Hickory|Wy|Suite D|Liberty|Howard County|NE|63451|United States|-7|single family| +37535|AAAAAAAAPJCJAAAA|44|Washington Lincoln|Ln|Suite 170|New Hope|Lincoln County|WV|29431|United States|-5|condo| +37536|AAAAAAAAAKCJAAAA|313|Third East|Way|Suite I|Clifton|Chickasaw County|IA|58014|United States|-6|condo| +37537|AAAAAAAABKCJAAAA|587|Third |Ct.|Suite W|Green Acres|Custer County|SD|57683|United States|-6|condo| +37538|AAAAAAAACKCJAAAA|476|Cherry Church|Avenue|Suite 230|Colfax|Clarke County|MS|52565|United States|-6|condo| +37539|AAAAAAAADKCJAAAA|347|View View|Ct.|Suite V|Greenwood|Montmorency County|MI|48828|United States|-5|single family| +37540|AAAAAAAAEKCJAAAA|608|Green Hillcrest|Parkway|Suite 400|Providence|Baker County|FL|36614|United States|-5|single family| +37541|AAAAAAAAFKCJAAAA|574|Johnson North|Ln|Suite 480|Mount Olive|Carbon County|UT|88059|United States|-7|condo| +37542|AAAAAAAAGKCJAAAA|238|Second Hickory|Street|Suite R|Lakewood|Durham County|NC|28877|United States|-5|condo| +37543|AAAAAAAAHKCJAAAA|272|Washington Wilson|Drive|Suite V|Lincoln|Muskogee County|OK|71289|United States|-6|condo| +37544|AAAAAAAAIKCJAAAA|414|Poplar Pine|Ave|Suite 150|Providence|Sumter County|GA|36614|United States|-5|apartment| +37545|AAAAAAAAJKCJAAAA|773|Central North|Avenue|Suite 270|Highland Park|Houghton County|MI|46534|United States|-5|condo| +37546|AAAAAAAAKKCJAAAA|182|Miller |Ave|Suite 20|Valley View|New York County|NY|15124|United States|-5|single family| +37547|AAAAAAAALKCJAAAA|12|Maple Seventh|Street|Suite V|Jackson|McMullen County|TX|79583|United States|-6|condo| +37548|AAAAAAAAMKCJAAAA|750|Willow |Wy|Suite O|Oakwood|Clare County|MI|40169|United States|-5|apartment| +37549|AAAAAAAANKCJAAAA|553|10th Fourth|RD|Suite N|New Town|Toole County|MT|69634|United States|-7|condo| +37550|AAAAAAAAOKCJAAAA|374|15th |Circle|Suite P|Bethel|Luzerne County|PA|15281|United States|-5|apartment| +37551|AAAAAAAAPKCJAAAA|643|View |Street|Suite 490|Berea|Black Hawk County|IA|53804|United States|-6|single family| +37552|AAAAAAAAALCJAAAA|761|12th Jackson|Blvd|Suite 180|Edgewood|Butler County|PA|10069|United States|-5|single family| +37553|AAAAAAAABLCJAAAA|||||Unionville|||71711||-6|| +37554|AAAAAAAACLCJAAAA|822|6th Spring||Suite R||||49310|||condo| +37555|AAAAAAAADLCJAAAA|333|11th |Pkwy|Suite 330|Bethel|Navajo County|AZ|85281|United States|-7|single family| +37556|AAAAAAAAELCJAAAA|359|Maple |Pkwy|Suite 240|Five Points|Tensas Parish|LA|76098|United States|-6|condo| +37557|AAAAAAAAFLCJAAAA|733|Hill 9th|Cir.|Suite J|Five Forks|Carteret County|NC|22293|United States|-5|single family| +37558|AAAAAAAAGLCJAAAA|136|Maple |Ct.|Suite 320|Woodlawn|Victoria County|TX|74098|United States|-6|single family| +37559|AAAAAAAAHLCJAAAA|608|Ridge |Road|Suite A|Harmony|Amite County|MS|55804|United States|-6|apartment| +37560|AAAAAAAAILCJAAAA|943|College First|Ave|Suite 290|Pine Grove|Franklin County|PA|14593|United States|-5|apartment| +37561|AAAAAAAAJLCJAAAA|727|Church River|Dr.|Suite 260|Wilson|Rockwall County|TX|76971|United States|-6|single family| +37562|AAAAAAAAKLCJAAAA|497|13th |Drive|Suite 320|Newport|Bradford County|FL|31521|United States|-5|condo| +37563|AAAAAAAALLCJAAAA|663|Franklin View|Wy|Suite K|Georgetown|McDowell County|WV|27057|United States|-5|apartment| +37564|AAAAAAAAMLCJAAAA|583|South Lee|Dr.|Suite P|Pleasant Hill|Sweet Grass County|MT|63604|United States|-7|condo| +37565|AAAAAAAANLCJAAAA|42|5th Jackson|Ct.|Suite 270|Pleasant Grove|Dawes County|NE|64136|United States|-6|apartment| +37566|AAAAAAAAOLCJAAAA|876|Highland |RD|Suite 160|Silver Springs|Rush County|IN|44843|United States|-5|condo| +37567|AAAAAAAAPLCJAAAA|512|Church 8th|Boulevard|Suite 50||||72477|United States|-6|apartment| +37568|AAAAAAAAAMCJAAAA|951|Sunset |Pkwy||Forest||SC||United States|-5|single family| +37569|AAAAAAAABMCJAAAA|680|Elm |Wy|Suite J|Springfield|Fulton County|NY|19303|United States|-5|single family| +37570|AAAAAAAACMCJAAAA|124|West 7th|Way|Suite L|Unionville|Sioux County|NE|61711|United States|-7|single family| +37571|AAAAAAAADMCJAAAA|121|Maple |Court|Suite 250|Longwood|Kiowa County|OK|77021|United States|-6|single family| +37572|AAAAAAAAEMCJAAAA|556|Wilson |Avenue|Suite 250|Forest Hills|Miller County|MO|69237|United States|-6|single family| +37573|AAAAAAAAFMCJAAAA|980|Center |Street|Suite 240|Pine Grove|Ohio County|IN|44593|United States|-5|apartment| +37574|AAAAAAAAGMCJAAAA|542|11th |Drive|Suite 310|Concord|Clinton County|OH|44107|United States|-5|condo| +37575|AAAAAAAAHMCJAAAA|667|Meadow 2nd|Pkwy|Suite 250|Mount Vernon|Berkeley County|SC|28482|United States|-5|single family| +37576|AAAAAAAAIMCJAAAA|225|Jackson |Road|Suite M|Springfield|Noble County|IN|49303|United States|-5|condo| +37577|AAAAAAAAJMCJAAAA|936|Sixth |Blvd|Suite 490|Plainview|Walker County|AL|33683|United States|-6|condo| +37578|AAAAAAAAKMCJAAAA|554|Broadway First|Dr.|Suite M|Marion|Meigs County|TN|30399|United States|-6|condo| +37579|AAAAAAAALMCJAAAA|810|Washington |Ln|Suite 70|Edgewood|La Salle Parish|LA|70069|United States|-6|apartment| +37580|AAAAAAAAMMCJAAAA|755|South |Wy|Suite Q|Philadelphia|Luna County|NM|85591|United States|-7|single family| +37581|AAAAAAAANMCJAAAA|248|Spring |Ln|Suite O|Fairfield|Canyon County|ID|86192|United States|-7|condo| +37582|AAAAAAAAOMCJAAAA|449|Cherry |Ave|Suite 90|Jackson|Oneida County|WI|59583|United States|-6|apartment| +37583|AAAAAAAAPMCJAAAA|521|Ridge |Ave|Suite I|Mount Vernon|Radford city|VA|28482|United States|-5|apartment| +37584|AAAAAAAAANCJAAAA|3|View |Drive|Suite K|Williamsville|Iron County|WI|58754|United States|-6|condo| +37585|AAAAAAAABNCJAAAA|348|Park South|Lane|Suite B|Mount Vernon|Nowata County|OK|78482|United States|-6|condo| +37586|AAAAAAAACNCJAAAA|587|Church |Parkway|Suite 490|Summit|Lee County|KY|40499|United States|-5|condo| +37587|AAAAAAAADNCJAAAA|145|Main |Court|Suite 180|Salem|Esmeralda County|NV|88048|United States|-8|single family| +37588|AAAAAAAAENCJAAAA|131|Park |Lane|Suite 200|Oak Hill|Kewaunee County|WI|57838|United States|-6|condo| +37589|AAAAAAAAFNCJAAAA|755|Center |Street|Suite K|Delmar|Okeechobee County|FL|33957|United States|-5|apartment| +37590|AAAAAAAAGNCJAAAA|998|3rd Cherry|Pkwy|Suite M|Woodville|Jo Daviess County|IL|64289|United States|-6|condo| +37591|AAAAAAAAHNCJAAAA|503|Johnson First|Drive|Suite 420|Kingston|Logan County|AR|74975|United States|-6|apartment| +37592|AAAAAAAAINCJAAAA|482|Johnson Adams|Blvd|Suite 20|Montpelier|Medina County|OH|48930|United States|-5|apartment| +37593|AAAAAAAAJNCJAAAA|791|Third |Ave|Suite A|Greenwood|Hockley County|TX|78828|United States|-6|single family| +37594|AAAAAAAAKNCJAAAA|280|Spruce |Drive|Suite 150|Springdale|Mackinac County|MI|48883|United States|-5|condo| +37595|AAAAAAAALNCJAAAA|923|7th |Boulevard|Suite 50|Riverview|Sabine Parish|LA|79003|United States|-6|condo| +37596|AAAAAAAAMNCJAAAA|549|4th 6th|Avenue|Suite O|Bunker Hill|Ballard County|KY|40150|United States|-6|apartment| +37597|AAAAAAAANNCJAAAA|539|Hill |Ct.|Suite 460|Antioch|Daniels County|MT|68605|United States|-7|apartment| +37598|AAAAAAAAONCJAAAA|2|Oak |Wy|Suite 80|Summit|Franklin County|AR|70499|United States|-6|apartment| +37599|AAAAAAAAPNCJAAAA|91|Highland |Pkwy|Suite M|Pleasant Hill|Fisher County|TX|73604|United States|-6|apartment| +37600|AAAAAAAAAOCJAAAA|800|Seventh |Way|Suite 470|Howell|Sully County|SD|54854|United States|-7|single family| +37601|AAAAAAAABOCJAAAA|863|Locust |Court|Suite 370|Unionville|Brooke County|WV|21711|United States|-5|single family| +37602|AAAAAAAACOCJAAAA|573|11th Birch|Ln|Suite 120|Summit|Greenwood County|KS|60499|United States|-6|single family| +37603|AAAAAAAADOCJAAAA|378|Main Cedar|Road|Suite 100|Riverdale|Baltimore County|MD|29391|United States|-5|apartment| +37604|AAAAAAAAEOCJAAAA|143|Sunset |Street|Suite Q|Springfield|Montgomery County|IA|59303|United States|-6|condo| +37605|AAAAAAAAFOCJAAAA|52|12th Main|Circle|Suite 30|Lebanon|Bennington County|VT|03498|United States|-5|apartment| +37606|AAAAAAAAGOCJAAAA|291|Lee |Parkway|Suite Y|Jackson|Marion County|AL|39583|United States|-6|condo| +37607|AAAAAAAAHOCJAAAA|729|Ridge |Ln|Suite 430|Clifton|Hinds County|MS|58014|United States|-6|apartment| +37608|AAAAAAAAIOCJAAAA|711|Central |Way|Suite 480|Oak Hill|Uinta County|WY|87838|United States|-7|single family| +37609|AAAAAAAAJOCJAAAA|787|8th |ST|Suite 220|Unionville|Chaffee County|CO|81711|United States|-7|condo| +37610|AAAAAAAAKOCJAAAA|410|Maple |Ln|Suite 160|Spring Valley|Morgan County|OH|46060|United States|-5|single family| +37611|AAAAAAAALOCJAAAA|980|Second Sixth|Street|Suite P|Kingston|Genesee County|MI|44975|United States|-5|single family| +37612|AAAAAAAAMOCJAAAA|84|1st |Avenue|Suite U|Newport|Roanoke city|VA|21521|United States|-5|apartment| +37613|AAAAAAAANOCJAAAA|967|Mill Franklin|Ln|Suite 420|Lakewood|Jackson County|IA|58877|United States|-6|condo| +37614|AAAAAAAAOOCJAAAA|390|Ash |Wy|Suite E|Riverview|Aleutians East Borough|AK|99003|United States|-9|apartment| +37615|AAAAAAAAPOCJAAAA|698|Fourth |Ave|Suite 450|Sulphur Springs|Otoe County|NE|68354|United States|-7|apartment| +37616|AAAAAAAAAPCJAAAA|696|Jackson |Blvd|Suite 110|Greenville|Hansford County|TX|71387|United States|-6|single family| +37617|AAAAAAAABPCJAAAA|335|Sunset |Dr.|Suite C|Liberty|Upshur County|TX|73451|United States|-6|single family| +37618|AAAAAAAACPCJAAAA|330|Ridge |Circle|Suite D|Georgetown|Washoe County|NV|87057|United States|-8|apartment| +37619|AAAAAAAADPCJAAAA|377|Railroad |Avenue|Suite 200|Belmont|Greene County|VA|20191|United States|-5|condo| +37620|AAAAAAAAEPCJAAAA|642|Oak |Drive|Suite 0|Pleasant Valley|Leelanau County|MI|42477|United States|-5|condo| +37621|AAAAAAAAFPCJAAAA|402|Washington |Ln|Suite 220|Oakland|Union County|KY|49843|United States|-5|condo| +37622|AAAAAAAAGPCJAAAA|786|13th |Avenue|Suite L|Northwood|Garfield County|NE|64104|United States|-6|condo| +37623|AAAAAAAAHPCJAAAA|158|View Birch|Street|Suite 270|Fairview|Pierce County|WA|95709|United States|-8|single family| +37624|AAAAAAAAIPCJAAAA|402|14th |Boulevard|Suite S|Waterloo|Rockwall County|TX|71675|United States|-6|condo| +37625|AAAAAAAAJPCJAAAA|494|Lakeview Center|Road|Suite 110|Mount Olive|Northampton County|PA|18059|United States|-5|apartment| +37626|AAAAAAAAKPCJAAAA|436|Laurel Walnut|Way|Suite 310|Lakeville|Sandusky County|OH|48811|United States|-5|single family| +37627|AAAAAAAALPCJAAAA|843||||Shaw||||||apartment| +37628|AAAAAAAAMPCJAAAA|659|Second Third|Blvd|Suite G|Oakdale|Coffee County|GA|39584|United States|-5|condo| +37629|AAAAAAAANPCJAAAA|694|Woodland 15th|Circle|Suite C|Concord|Huron County|MI|44107|United States|-5|apartment| +37630|AAAAAAAAOPCJAAAA|918|Elm |Cir.|Suite 380|Stringtown|Haralson County|GA|30162|United States|-5|single family| +37631|AAAAAAAAPPCJAAAA|682|Poplar |Drive|Suite 140|Farmington|Towns County|GA|39145|United States|-5|single family| +37632|AAAAAAAAAADJAAAA|375|Third Pine|Pkwy|Suite B|Greenfield|Johnson County|IA|55038|United States|-6|apartment| +37633|AAAAAAAABADJAAAA|447|15th Cedar|Circle|Suite Q|Fairview|Monmouth County|NJ|06309|United States|-5|apartment| +37634|AAAAAAAACADJAAAA|349|1st |Ave|Suite 180|Woodland|Choctaw County|OK|74854|United States|-6|apartment| +37635|AAAAAAAADADJAAAA|368|View |Pkwy|Suite 390|Conway|Franklin County|ID|84360|United States|-7|condo| +37636|AAAAAAAAEADJAAAA|203|4th |Pkwy|Suite V|Oak Grove|Pawnee County|OK|78370|United States|-6|condo| +37637|AAAAAAAAFADJAAAA|334|Oak |Ln|Suite 400|Oakwood|Cayuga County|NY|10169|United States|-5|apartment| +37638|AAAAAAAAGADJAAAA|993|Birch |Blvd|Suite V|Jamestown|Lexington County|SC|26867|United States|-5|condo| +37639|AAAAAAAAHADJAAAA|285|Spruce Park|Road|Suite 250|Waterloo|Payette County|ID|81675|United States|-7|apartment| +37640|AAAAAAAAIADJAAAA|268|Mill |RD|Suite F|Ashland|Pierce County|WI|54244|United States|-6|apartment| +37641|AAAAAAAAJADJAAAA|635|Cedar Cedar|Lane|Suite I|Springfield|Kiowa County|CO|89303|United States|-7|apartment| +37642|AAAAAAAAKADJAAAA|845|Eigth |Pkwy|Suite 450|Newport|Dearborn County|IN|41521|United States|-5|single family| +37643|AAAAAAAALADJAAAA|847|Sixth |Dr.|Suite U|Lebanon|Sandusky County|OH|42898|United States|-5|single family| +37644|AAAAAAAAMADJAAAA|500|3rd 8th|Boulevard|Suite V|Waterloo|Waller County|TX|71675|United States|-6|condo| +37645|AAAAAAAANADJAAAA|250|4th Seventh|Drive|Suite I|Green Acres|Scott County|IA|57683|United States|-6|apartment| +37646|AAAAAAAAOADJAAAA|389|East North|Ave|Suite 350|Plainview|Barry County|MI|43683|United States|-5|apartment| +37647|AAAAAAAAPADJAAAA|593|2nd |Ave|Suite 200|Shady Grove|Woodbury County|IA|52812|United States|-6|single family| +37648|AAAAAAAAABDJAAAA|957|Locust |Cir.|Suite 460|Crossroads|Aransas County|TX|70534|United States|-6|single family| +37649|AAAAAAAABBDJAAAA|840|River Dogwood|Pkwy|Suite 410|Green Acres|Cherokee County|IA|57683|United States|-6|condo| +37650|AAAAAAAACBDJAAAA|75|Spring First|Cir.|Suite M|Bridgeport|San Mateo County|CA|95817|United States|-8|apartment| +37651|AAAAAAAADBDJAAAA|909|Lee |Wy|Suite 350|Lakewood|Lebanon County|PA|18877|United States|-5|condo| +37652|AAAAAAAAEBDJAAAA|982|Woodland |Ave|Suite 410|Texas|Bennett County|SD|53342|United States|-6|apartment| +37653|AAAAAAAAFBDJAAAA|634|Church Walnut|Road|Suite K|Hopewell|Thayer County|NE|60587|United States|-7|apartment| +37654|AAAAAAAAGBDJAAAA|740|Ninth 6th|Circle|Suite 220|Fairfield|Marquette County|MI|46192|United States|-5|condo| +37655|AAAAAAAAHBDJAAAA|883|Twelfth |Road|Suite I|Greenwood|Van Buren County|AR|78828|United States|-6|single family| +37656|AAAAAAAAIBDJAAAA|330|Laurel Jefferson|Boulevard|Suite N|Greenfield|Stark County|ND|55038|United States|-6|single family| +37657|AAAAAAAAJBDJAAAA|827|14th |Street|Suite N|Wilson|Campbell County|WY|86971|United States|-7|apartment| +37658|AAAAAAAAKBDJAAAA|158|Walnut Poplar|Ln|Suite J|Brownsville|Guernsey County|OH|49310|United States|-5|apartment| +37659|AAAAAAAALBDJAAAA|118|2nd |Lane|Suite L|Mount Vernon|Upshur County|WV|28482|United States|-5|apartment| +37660|AAAAAAAAMBDJAAAA|22|Sunset Birch|Cir.|Suite M|Oakwood|Fleming County|KY|40169|United States|-6|condo| +37661|AAAAAAAANBDJAAAA|624|Washington |Dr.|Suite I|Newport|Greenwood County|SC|21521|United States|-5|apartment| +37662|AAAAAAAAOBDJAAAA|367|North 14th|Wy|Suite 450|Wildwood|San Miguel County|CO|86871|United States|-7|condo| +37663|AAAAAAAAPBDJAAAA|349|Elm South|ST|Suite 30|Blanchard|Talbot County|GA|35985|United States|-5|single family| +37664|AAAAAAAAACDJAAAA|907|Sycamore |ST|Suite 80|Plainview|Kenedy County|TX|73683|United States|-6|apartment| +37665|AAAAAAAABCDJAAAA|32|2nd |Circle|Suite 300|Arlington|Ouachita County|AR|76557|United States|-6|condo| +37666|AAAAAAAACCDJAAAA|136|3rd North|Cir.|Suite 330|Ashley|Anderson County|KY|44324|United States|-6|condo| +37667|AAAAAAAADCDJAAAA|421|Jefferson Central|Parkway|Suite K|Concord|Jasper County|MO|64107|United States|-6|apartment| +37668|AAAAAAAAECDJAAAA|661|Central Park|Blvd|Suite G|Ashland|McPherson County|SD|54244|United States|-7|condo| +37669|AAAAAAAAFCDJAAAA|600|10th |Dr.|Suite 440|Tipton|Osage County|MO|68773|United States|-6|single family| +37670|AAAAAAAAGCDJAAAA|720|Sixth |Avenue|Suite N|Centerville|Blaine County|MT|60059|United States|-7|single family| +37671|AAAAAAAAHCDJAAAA|88|Tenth |Ln|Suite 350|Oakland|Wheeler County|NE|69843|United States|-7|single family| +37672|AAAAAAAAICDJAAAA|917|13th |Ct.|Suite F|Newport|Jewell County|KS|61521|United States|-6|apartment| +37673|AAAAAAAAJCDJAAAA|410|Maple |Avenue|Suite 230|Mount Pleasant|Clarke County|VA|21933|United States|-5|single family| +37674|AAAAAAAAKCDJAAAA|851|West Locust|Dr.|Suite 350|Marion|Will County|IL|60399|United States|-6|apartment| +37675|AAAAAAAALCDJAAAA|851|Forest 2nd|Blvd|Suite 170|Mount Pleasant|Yolo County|CA|91933|United States|-8|single family| +37676|AAAAAAAAMCDJAAAA|||ST||Friendship|Harrison County|||||condo| +37677|AAAAAAAANCDJAAAA|897|Seventh Lake|Blvd|Suite U|Wilson|Valdez-Cordova Census Area|AK|96971|United States|-9|apartment| +37678|AAAAAAAAOCDJAAAA|533|12th |Court|Suite 340|Georgetown|Allendale County|SC|27057|United States|-5|condo| +37679|AAAAAAAAPCDJAAAA||3rd Poplar||||Palm Beach County|||United States||| +37680|AAAAAAAAADDJAAAA|483|Pine |Boulevard|Suite 30|Bunker Hill|Imperial County|CA|90150|United States|-8|apartment| +37681|AAAAAAAABDDJAAAA|90|Walnut |Way|Suite 450|Wilson|Butler County|NE|66971|United States|-6|single family| +37682|AAAAAAAACDDJAAAA|208|Maple |Street|Suite B|Belmont|Jefferson County|KY|40191|United States|-6|single family| +37683|AAAAAAAADDDJAAAA|100|3rd |ST|Suite E|Pleasant Grove|Putnam County|IN|44136|United States|-5|condo| +37684|AAAAAAAAEDDJAAAA|643|Eigth |Lane|Suite 70|Farmington|Orange County|NY|19145|United States|-5|apartment| +37685|AAAAAAAAFDDJAAAA||Dogwood ||Suite P|Wildwood|Marion County||76871|||| +37686|AAAAAAAAGDDJAAAA||||Suite W||Harper County|||United States||condo| +37687|AAAAAAAAHDDJAAAA|164|Walnut |Ave|Suite S|Lakeside|Addison County|VT|09532|United States|-5|single family| +37688|AAAAAAAAIDDJAAAA|287|3rd |Street|Suite T|Salem|Churchill County|NV|88048|United States|-8|apartment| +37689|AAAAAAAAJDDJAAAA|326|Highland |Way|Suite 380|Shiloh|Hancock County|MS|59275|United States|-6|apartment| +37690|AAAAAAAAKDDJAAAA|159|Center |Way|Suite 370|Farmington|Antelope County|NE|69145|United States|-6|single family| +37691|AAAAAAAALDDJAAAA|874|13th Chestnut|ST|Suite 330|Jackson|New Haven County|CT|09583|United States|-5|apartment| +37692|AAAAAAAAMDDJAAAA|395|East |Ct.|Suite 120|Wilson|Hopkins County|KY|46971|United States|-6|apartment| +37693|AAAAAAAANDDJAAAA|308|Broadway 4th|Lane|Suite 360|Sunnyside|Coffee County|GA|31952|United States|-5|apartment| +37694|AAAAAAAAODDJAAAA|428|13th 4th|Cir.|Suite W|Tremont|Christian County|IL|69515|United States|-6|single family| +37695|AAAAAAAAPDDJAAAA|21|View |Avenue|Suite A|Glenwood|Marion County|SC|23511|United States|-5|single family| +37696|AAAAAAAAAEDJAAAA|739|Hill Center|Street|Suite 90|Buena Vista|Green County|KY|45752|United States|-6|apartment| +37697|AAAAAAAABEDJAAAA|874|View |Lane|Suite 310|Salem|Valdez-Cordova Census Area|AK|98048|United States|-9|single family| +37698|AAAAAAAACEDJAAAA|500|Park Locust|Ave|Suite Y|Mount Olive|Lorain County|OH|48059|United States|-5|apartment| +37699|AAAAAAAADEDJAAAA|458|Oak |Pkwy|Suite W|Farmington|Evans County|GA|39145|United States|-5|apartment| +37700|AAAAAAAAEEDJAAAA|823|Church 1st|Parkway|Suite B|Pomona|San Juan County|NM|84153|United States|-7|single family| +37701|AAAAAAAAFEDJAAAA|347|Sunset 1st|Drive|Suite D|Shiloh|Montgomery County|GA|39275|United States|-5|single family| +37702|AAAAAAAAGEDJAAAA|146|9th 5th|Boulevard|Suite V|Hillcrest|Fremont County|CO|83003|United States|-7|single family| +37703|AAAAAAAAHEDJAAAA|151|Pine Maple|Cir.|Suite 400|Oak Grove|Catoosa County|GA|38370|United States|-5|condo| +37704|AAAAAAAAIEDJAAAA|340|Madison Lee|Road|Suite X|Gravel Hill|Gentry County|MO|61944|United States|-6|apartment| +37705|AAAAAAAAJEDJAAAA|658|Smith |Court|Suite U|Red Hill|Westchester County|NY|14338|United States|-5|condo| +37706|AAAAAAAAKEDJAAAA|558|Elm |Cir.|Suite 210|Pleasant Grove|Grant County|AR|74136|United States|-6|single family| +37707|AAAAAAAALEDJAAAA|806|Cedar |ST|Suite E|Maple Grove|Cook County|GA|38252|United States|-5|single family| +37708|AAAAAAAAMEDJAAAA|313|Cherry |Cir.|Suite L|Unionville|Douglas County|WI|51711|United States|-6|condo| +37709|AAAAAAAANEDJAAAA|467|Fourth |Circle|Suite 90|Fairfield|Steele County|ND|56192|United States|-6|condo| +37710|AAAAAAAAOEDJAAAA|455|North |Avenue|Suite Q|Oak Grove|Franklin County|TX|78370|United States|-6|single family| +37711|AAAAAAAAPEDJAAAA|941|Seventh Chestnut|Court|Suite F|Oak Ridge|Logan County|OK|78371|United States|-6|apartment| +37712|AAAAAAAAAFDJAAAA|873|2nd Cedar|Street|Suite 80|Valley View|Love County|OK|75124|United States|-6|apartment| +37713|AAAAAAAABFDJAAAA|548|7th |Boulevard|Suite 350|Lakewood|Richland County|MT|68877|United States|-7|condo| +37714|AAAAAAAACFDJAAAA|288|Center Hickory|Street|Suite 390|Red Hill|Essex County|VA|24338|United States|-5|single family| +37715|AAAAAAAADFDJAAAA|670|Lake Elevnth|Parkway|Suite V|Glenville|Kings County|NY|13445|United States|-5|condo| +37716|AAAAAAAAEFDJAAAA|242|2nd Green|Blvd|Suite N|Hillcrest|Otsego County|MI|43003|United States|-5|apartment| +37717|AAAAAAAAFFDJAAAA|648|6th Ash|RD|Suite R|Youngstown|Rockland County|NY|10001|United States|-5|apartment| +37718|AAAAAAAAGFDJAAAA|31|North |Road|Suite M|Lakeview|Howard County|TX|78579|United States|-6|single family| +37719|AAAAAAAAHFDJAAAA|951|12th |Parkway|Suite 450|Oakdale|Minidoka County|ID|89584|United States|-7|apartment| +37720|AAAAAAAAIFDJAAAA|930|Highland Miller|Lane|Suite 140|Sulphur Springs|Hickman County|KY|48354|United States|-6|condo| +37721|AAAAAAAAJFDJAAAA|115|Seventh Miller||Suite 380||||40069|United States|-5|| +37722|AAAAAAAAKFDJAAAA|682|Hill 3rd|Road|Suite A|Brownsville|York County|ME|09910|United States|-5|condo| +37723|AAAAAAAALFDJAAAA|221|Ash East|Way|Suite B|Pine Grove|Yuma County|AZ|84593|United States|-7|single family| +37724|AAAAAAAAMFDJAAAA|244|Willow Jefferson|Road|Suite L|Five Forks|Dickinson County|KS|62293|United States|-6|condo| +37725|AAAAAAAANFDJAAAA|986|Mill Dogwood|Pkwy|Suite 160|Waterloo|Jackson County|OH|41675|United States|-5|apartment| +37726|AAAAAAAAOFDJAAAA|647|College 6th|Wy|Suite 100|Clinton|Crawford County|MI|48222|United States|-5|condo| +37727|AAAAAAAAPFDJAAAA|547|Adams |Blvd|Suite G|Midway|Rice County|KS|61904|United States|-6|apartment| +37728|AAAAAAAAAGDJAAAA|437|Third |ST|Suite 130|Doyle|Cochise County|AZ|88434|United States|-7|single family| +37729|AAAAAAAABGDJAAAA|197|Pine 7th|Way|Suite X|Oakdale|Dawson County|NE|69584|United States|-6|apartment| +37730|AAAAAAAACGDJAAAA|562|Washington |Drive|Suite A|Brunswick|Saline County|NE|64642|United States|-7|single family| +37731|AAAAAAAADGDJAAAA|||ST|Suite 30|Glendale|Nueces County||||-6|apartment| +37732|AAAAAAAAEGDJAAAA|136|North Park|Street|Suite H|Five Points|Thurston County|WA|96098|United States|-8|condo| +37733|AAAAAAAAFGDJAAAA|871|Church |Blvd|Suite 480|Red Hill|Edwards County|KS|64338|United States|-6|condo| +37734|AAAAAAAAGGDJAAAA|551|Oak Fourteenth|Wy|Suite R|Eastwood|Richland County|MT|65843|United States|-7|condo| +37735|AAAAAAAAHGDJAAAA|414|Ash |Parkway|Suite H|Wilson|Kenosha County|WI|56971|United States|-6|apartment| +37736|AAAAAAAAIGDJAAAA|257|Hillcrest Main|Dr.|Suite 410|Derby|Butte County|CA|97702|United States|-8|condo| +37737|AAAAAAAAJGDJAAAA|445|Dogwood Main|RD|Suite L|Tyrone|Cleveland County|OK|71201|United States|-6|apartment| +37738|AAAAAAAAKGDJAAAA|828|Maple |Boulevard|Suite Q|Enterprise|Hamilton County|NY|11757|United States|-5|condo| +37739|AAAAAAAALGDJAAAA|310|Ridge |Cir.|Suite 320|Enterprise|Colonial Heights city|VA|21757|United States|-5|apartment| +37740|AAAAAAAAMGDJAAAA|856|Fifth Walnut|Cir.|Suite U|Lakeview|Sumner County|KS|68579|United States|-6|apartment| +37741|AAAAAAAANGDJAAAA|158|2nd Second|RD|Suite W|Georgetown|Cherokee County|KS|67057|United States|-6|condo| +37742|AAAAAAAAOGDJAAAA|986|Pine Williams|Court|Suite Q|Union|Hamilton County|OH|48721|United States|-5|condo| +37743|AAAAAAAAPGDJAAAA|649|12th Railroad|Road|Suite R|Wesley|Burke County|NC|21218|United States|-5|single family| +37744|AAAAAAAAAHDJAAAA|205|9th Jackson|Parkway|Suite L|Union Hill|Washington County|KY|47746|United States|-5|single family| +37745|AAAAAAAABHDJAAAA|482|Davis |Avenue|Suite U|Sulphur Springs|Dearborn County|IN|48354|United States|-5|condo| +37746|AAAAAAAACHDJAAAA|301|Spring |Blvd|Suite S|Liberty|Caswell County|NC|23451|United States|-5|apartment| +37747|AAAAAAAADHDJAAAA|519|West Forest|Blvd|Suite N|Sunnyside|Sweet Grass County|MT|61952|United States|-7|apartment| +37748|AAAAAAAAEHDJAAAA|819|Hickory Pine|Way|Suite T|Lakeside|Bledsoe County|TN|39532|United States|-5|single family| +37749|AAAAAAAAFHDJAAAA|243|Railroad |Pkwy|Suite X|Millbrook|Bartholomew County|IN|47529|United States|-5|apartment| +37750|AAAAAAAAGHDJAAAA|315|3rd |Road|Suite 480|Mount Vernon|Klickitat County|WA|98482|United States|-8|single family| +37751|AAAAAAAAHHDJAAAA||Jefferson |Avenue|Suite E||Howard County|AR|||-6|| +37752|AAAAAAAAIHDJAAAA|729|Jackson Maple|Dr.|Suite W|Carter|Boyd County|NE|60919|United States|-6|condo| +37753|AAAAAAAAJHDJAAAA|463|Spring View|Ln|Suite G|Five Points|Ozark County|MO|66098|United States|-6|apartment| +37754|AAAAAAAAKHDJAAAA|97|1st |Court|Suite A|Riceville|Custer County|CO|85867|United States|-7|single family| +37755|AAAAAAAALHDJAAAA|573|Center 11th|Road|Suite I|Little River|Calhoun County|SC|20319|United States|-5|single family| +37756|AAAAAAAAMHDJAAAA|906|River Pine|Way|Suite Y|Concord|Chicot County|AR|74107|United States|-6|condo| +37757|AAAAAAAANHDJAAAA|50|Valley |Pkwy|Suite 340|Bennett|York County|VA|21715|United States|-5|apartment| +37758|AAAAAAAAOHDJAAAA|482|First Church|Pkwy|Suite 90|Franklin|Pittsylvania County|VA|29101|United States|-5|apartment| +37759|AAAAAAAAPHDJAAAA|606|Main 9th|Street|Suite L|Mount Vernon|Roseau County|MN|58482|United States|-6|condo| +37760|AAAAAAAAAIDJAAAA|654||Pkwy|Suite O|Wildwood||||United States||| +37761|AAAAAAAABIDJAAAA|657|14th Pine|Street|Suite W|Kingston|Dutchess County|NY|14975|United States|-5|single family| +37762|AAAAAAAACIDJAAAA|518|Jackson 6th|Boulevard|Suite M|Lakewood|Morgan County|IL|68877|United States|-6|single family| +37763|AAAAAAAADIDJAAAA|968|Birch Green|Avenue|Suite B|Glendale|Jefferson County|KS|63951|United States|-6|condo| +37764|AAAAAAAAEIDJAAAA|320|14th Green|Way|Suite 40|Lincoln|Berkshire County|MA|01889|United States|-5|single family| +37765|AAAAAAAAFIDJAAAA|514|Pine Lakeview|Ave|Suite W|Oakdale|Baldwin County|AL|39584|United States|-6|apartment| +37766|AAAAAAAAGIDJAAAA|97|3rd |Way|Suite S|Belmont|Adams County|CO|80191|United States|-7|condo| +37767|AAAAAAAAHIDJAAAA|439|Oak |Parkway|Suite 490|Buena Vista|Cheyenne County|KS|65752|United States|-6|apartment| +37768|AAAAAAAAIIDJAAAA|2|Poplar 14th|Blvd|Suite S|Florence|Piatt County|IL|63394|United States|-6|condo| +37769|AAAAAAAAJIDJAAAA|102|View Smith|ST|Suite 130|Five Forks|Carbon County|PA|12293|United States|-5|apartment| +37770|AAAAAAAAKIDJAAAA|930|9th |Lane|Suite 390|Greenfield|Dickson County|TN|35038|United States|-5|apartment| +37771|AAAAAAAALIDJAAAA|639|11th 14th|Blvd|Suite F|Kingston|Beaverhead County|MT|64975|United States|-7|apartment| +37772|AAAAAAAAMIDJAAAA|81|Washington 15th|Dr.|Suite K|Georgetown|Chatham County|NC|27057|United States|-5|condo| +37773|AAAAAAAANIDJAAAA|499|Willow 13th|Street|Suite A|Jamestown|Broward County|FL|36867|United States|-5|condo| +37774|AAAAAAAAOIDJAAAA|640|Ninth Willow|Street|Suite K|San Jose|Sully County|SD|58003|United States|-7|apartment| +37775|AAAAAAAAPIDJAAAA|507|Seventh |Boulevard|Suite 290|Mountain View|Warren County|NC|24466|United States|-5|condo| +37776|AAAAAAAAAJDJAAAA|427|Mill 7th|Pkwy|Suite R|Pine Grove|Payette County|ID|84593|United States|-7|condo| +37777|AAAAAAAABJDJAAAA|191|Williams Second|Avenue|Suite 420|Brownsville|Coweta County|GA|39310|United States|-5|single family| +37778|AAAAAAAACJDJAAAA|614|Park |ST|Suite Q|Sulphur Springs|Huntingdon County|PA|18354|United States|-5|single family| +37779|AAAAAAAADJDJAAAA|960|Dogwood 15th|Pkwy|Suite 50|Lenox|Stark County|OH|41143|United States|-5|single family| +37780|AAAAAAAAEJDJAAAA|861|Railroad |Road|Suite 200|Clifton|Llano County|TX|78014|United States|-6|condo| +37781|AAAAAAAAFJDJAAAA|580|9th |Boulevard|Suite 490|Franklin|Wright County|MO|69101|United States|-6|single family| +37782|AAAAAAAAGJDJAAAA|409|2nd Fifth|Blvd|Suite A|Hopewell|Nassau County|NY|10587|United States|-5|single family| +37783|AAAAAAAAHJDJAAAA|420|Lincoln Chestnut|Boulevard|Suite N|Oak Grove|Park County|MT|68370|United States|-7|single family| +37784|AAAAAAAAIJDJAAAA|310|Second 12th|Parkway|Suite 270|Antioch|Cotton County|OK|78605|United States|-6|condo| +37785|AAAAAAAAJJDJAAAA|752|Elevnth 2nd|Avenue|Suite U|Woodland|Miami County|IN|44854|United States|-5|single family| +37786|AAAAAAAAKJDJAAAA|231|Dogwood West|Street|Suite 30|Highland|Lawrence County|IN|49454|United States|-5|single family| +37787|AAAAAAAALJDJAAAA|942|Laurel |Lane|Suite D|Highland|Union County|IN|49454|United States|-5|condo| +37788|AAAAAAAAMJDJAAAA|37|Chestnut |Lane|Suite 50|Liberty|Winston County|AL|33451|United States|-6|single family| +37789|AAAAAAAANJDJAAAA|532|Cherry Central|Ave|Suite 390|Mount Olive|Bibb County|AL|38059|United States|-6|condo| +37790|AAAAAAAAOJDJAAAA|693|Mill |Court|Suite P|Marion|Franklin County|VA|20399|United States|-5|condo| +37791|AAAAAAAAPJDJAAAA|725|Sixth Green|Way|Suite D|Greenville|Lake and Peninsula Borough|AK|91387|United States|-9|single family| +37792|AAAAAAAAAKDJAAAA||Sixth 11th|Avenue|Suite 90|Wildwood|Okfuskee County|OK|76871||-6|condo| +37793|AAAAAAAABKDJAAAA|669|Fifth |Cir.|Suite 370|Buena Vista|Armstrong County|TX|75752|United States|-6|condo| +37794|AAAAAAAACKDJAAAA|251|Broadway 7th|Parkway|Suite 10|Rolling Hills|Harney County|OR|97272|United States|-8|apartment| +37795|AAAAAAAADKDJAAAA|15|Lincoln |Way|Suite T|Friendship|Fayette County|KY|44536|United States|-6|apartment| +37796|AAAAAAAAEKDJAAAA|274|Broadway |Court|Suite 330|Mount Vernon|Vernon County|WI|58482|United States|-6|condo| +37797|AAAAAAAAFKDJAAAA|553||Ln||||MS|57057|||apartment| +37798|AAAAAAAAGKDJAAAA|904|Wilson Wilson|Road|Suite 270|Greenwood|Marion County|SC|28828|United States|-5|condo| +37799|AAAAAAAAHKDJAAAA|800|Green |Lane|Suite 480|Cedar Grove|Keya Paha County|NE|60411|United States|-7|apartment| +37800|AAAAAAAAIKDJAAAA|704|Ridge |Ave|Suite 110|Unionville|Phelps County|NE|61711|United States|-7|single family| +37801|AAAAAAAAJKDJAAAA|508|Jefferson |Lane|Suite 120|Waterloo|De Kalb County|IN|41675|United States|-5|apartment| +37802|AAAAAAAAKKDJAAAA|289|Davis 1st|Drive|Suite X|Jamestown|San Jacinto County|TX|76867|United States|-6|condo| +37803|AAAAAAAALKDJAAAA|377|Church |ST|Suite 30|Farmington|Power County|ID|89145|United States|-7|condo| +37804|AAAAAAAAMKDJAAAA|120|Poplar |Street|Suite 220|Antioch|Brazoria County|TX|78605|United States|-6|single family| +37805|AAAAAAAANKDJAAAA|983|South View|Circle|Suite 420|Hidden Valley|Butts County|GA|35521|United States|-5|single family| +37806|AAAAAAAAOKDJAAAA|76|Washington Cedar|Dr.|Suite H|Woodlawn|Sierra County|CA|94098|United States|-8|condo| +37807|AAAAAAAAPKDJAAAA|91|9th |Wy|Suite 460|Enterprise|Madison County|VA|21757|United States|-5|apartment| +37808|AAAAAAAAALDJAAAA|152|Lincoln 6th|RD|Suite I|Kingston|Miller County|GA|34975|United States|-5|single family| +37809|AAAAAAAABLDJAAAA|227|Washington |Lane|Suite 290|Brownsville|Clear Creek County|CO|89310|United States|-7|single family| +37810|AAAAAAAACLDJAAAA|447|Oak |Dr.|Suite Q|Clinton|Sutter County|CA|98222|United States|-8|single family| +37811|AAAAAAAADLDJAAAA|889|11th Williams|Circle|Suite 460|Oak Grove|Lee County|NC|28370|United States|-5|apartment| +37812|AAAAAAAAELDJAAAA|410|8th |Road|Suite V|Greenwood|Saunders County|NE|68828|United States|-7|apartment| +37813|AAAAAAAAFLDJAAAA|355|Jefferson Main|Wy|Suite R|Ashland|Tuscaloosa County|AL|34244|United States|-6|single family| +37814|AAAAAAAAGLDJAAAA|418|6th |Dr.|Suite 70|Edgewood|Loudon County|TN|30069|United States|-6|apartment| +37815|AAAAAAAAHLDJAAAA|117|Main |Ct.|Suite Q|Wilson|Worth County|IA|56971|United States|-6|apartment| +37816|AAAAAAAAILDJAAAA|394|Hill |RD|Suite K|Pleasant Grove|Haywood County|TN|34136|United States|-5|apartment| +37817|AAAAAAAAJLDJAAAA|21|Ash ||Suite F|||||United States||| +37818|AAAAAAAAKLDJAAAA|467|3rd 5th|ST|Suite B|Marion|Iroquois County|IL|60399|United States|-6|single family| +37819|AAAAAAAALLDJAAAA|285|Mill |Wy|Suite N|Marion|Greene County|OH|40399|United States|-5|condo| +37820|AAAAAAAAMLDJAAAA|333|Broadway Ridge|Cir.|Suite 380|Concord|Harrison County|TX|74107|United States|-6|single family| +37821|AAAAAAAANLDJAAAA|913|Poplar Oak|Parkway|Suite T|Lebanon|Karnes County|TX|72898|United States|-6|condo| +37822|AAAAAAAAOLDJAAAA|587|Maple West|RD|Suite 90|Hillcrest|Lincoln County|KY|43003|United States|-5|apartment| +37823|AAAAAAAAPLDJAAAA|860|South |Ave|Suite O|Antioch|Dawson County|GA|38605|United States|-5|single family| +37824|AAAAAAAAAMDJAAAA|44|Fourth Locust|Boulevard|Suite 480|Fairview|Codington County|SD|55709|United States|-6|apartment| +37825|AAAAAAAABMDJAAAA|803|1st Fifteenth|Avenue|Suite 50|Springfield|Union County|NJ|09903|United States|-5|condo| +37826|AAAAAAAACMDJAAAA|806|Lake Lake|Dr.|Suite Q|Georgetown|Venango County|PA|17057|United States|-5|apartment| +37827|AAAAAAAADMDJAAAA|821|Maple |Dr.|Suite 430|Glenwood|Graham County|NC|23511|United States|-5|single family| +37828|AAAAAAAAEMDJAAAA|617|Cherry Center|Dr.|Suite B|Harmony|Carbon County|MT|65804|United States|-7|single family| +37829|AAAAAAAAFMDJAAAA|909|Hickory |Pkwy|Suite I|Jamestown|Red River Parish|LA|76867|United States|-6|condo| +37830|AAAAAAAAGMDJAAAA|858|Wilson |Circle|Suite 0|Springfield|Jackson County|MI|49303|United States|-5|single family| +37831|AAAAAAAAHMDJAAAA|303|Adams Poplar|Parkway|Suite Y|Fairfield|Greeley County|KS|66192|United States|-6|condo| +37832|AAAAAAAAIMDJAAAA|262|Valley |Court|Suite 440|Lakeside|Dorchester County|SC|29532|United States|-5|condo| +37833|AAAAAAAAJMDJAAAA|603|Oak Oak|Circle|Suite W|Clifton|Crow Wing County|MN|58014|United States|-6|single family| +37834|AAAAAAAAKMDJAAAA|613|Ash North|RD|Suite 0|Clifton|Hamilton County|KS|68014|United States|-6|apartment| +37835|AAAAAAAALMDJAAAA|821|11th |Dr.|Suite R|Hillcrest|Allegan County|MI|43003|United States|-5|single family| +37836|AAAAAAAAMMDJAAAA|742|Green Fourteenth|RD|Suite K|Waterloo|Ottawa County|KS|61675|United States|-6|condo| +37837|AAAAAAAANMDJAAAA|457|Jackson |Ln|Suite 470|Union Hill|Hardee County|FL|37746|United States|-5|single family| +37838|AAAAAAAAOMDJAAAA|411|3rd |Drive|Suite Y|Mount Zion|Rankin County|MS|58054|United States|-6|condo| +37839|AAAAAAAAPMDJAAAA|727|Mill Sixth|Wy|Suite 260|Corinth|Bedford County|TN|37022|United States|-5|condo| +37840|AAAAAAAAANDJAAAA|933|Smith Hickory|Drive|Suite X|Kingston|Southampton County|VA|24975|United States|-5|apartment| +37841|AAAAAAAABNDJAAAA|454|River 14th|Parkway|Suite W|Hamilton|Florence County|WI|52808|United States|-6|apartment| +37842|AAAAAAAACNDJAAAA|494|9th Seventh|Drive|Suite 430|Kelly|Bee County|TX|72738|United States|-6|condo| +37843|AAAAAAAADNDJAAAA|693|8th |Avenue|Suite 40|Ashland|Calhoun County|MS|54244|United States|-6|single family| +37844|AAAAAAAAENDJAAAA|721|Miller |Ave|Suite 310|Marion|Dallas County|MO|60399|United States|-6|apartment| +37845|AAAAAAAAFNDJAAAA|177|Sunset Second|Parkway|Suite 360|Union Hill|Red Willow County|NE|67746|United States|-7|apartment| +37846|AAAAAAAAGNDJAAAA|902|Adams Woodland|Boulevard|Suite J|Elm Grove|Racine County|WI|53298|United States|-6|apartment| +37847|AAAAAAAAHNDJAAAA|917|12th |Blvd|Suite K|Pleasant Grove|McLennan County|TX|74136|United States|-6|condo| +37848|AAAAAAAAINDJAAAA|701|Woodland |Drive|Suite 110|Antioch|Lorain County|OH|48605|United States|-5|apartment| +37849|AAAAAAAAJNDJAAAA|923|5th |Wy|Suite 440|Florence|Henry County|AL|33394|United States|-6|single family| +37850|AAAAAAAAKNDJAAAA|428|Lake |RD|Suite A|Glenwood|Wyoming County|WV|23511|United States|-5|apartment| +37851|AAAAAAAALNDJAAAA|252|Park 8th|Cir.|Suite Q|Pleasant Grove|Okanogan County|WA|94136|United States|-8|condo| +37852|AAAAAAAAMNDJAAAA|200|7th South|Ct.|Suite 270|Green Acres|Woodward County|OK|77683|United States|-6|condo| +37853|AAAAAAAANNDJAAAA|452|Oak View|Cir.|Suite 410|Greenwood|Grant County|KS|68828|United States|-6|single family| +37854|AAAAAAAAONDJAAAA|26|Main |Circle|Suite M|Kingston|Flagler County|FL|34975|United States|-5|condo| +37855|AAAAAAAAPNDJAAAA|545|West 9th|Street|Suite K|Valley View|Putnam County|MO|65124|United States|-6|apartment| +37856|AAAAAAAAAODJAAAA|745|Walnut |Dr.|Suite 100|Hopewell|Chickasaw County|MS|50587|United States|-6|single family| +37857|AAAAAAAABODJAAAA|884|Hill |RD|Suite T|Greenwood|Bulloch County|GA|38828|United States|-5|condo| +37858|AAAAAAAACODJAAAA|14|Park |RD|Suite Y|Allentown|Douglas County|NE|61838|United States|-6|condo| +37859|AAAAAAAADODJAAAA|94|View Main|Road|Suite 60|Crossroads|Mahaska County|IA|50534|United States|-6|condo| +37860|AAAAAAAAEODJAAAA|855|4th Davis|RD|Suite 470|Shiloh|Monroe County|IL|69275|United States|-6|apartment| +37861|AAAAAAAAFODJAAAA|948|River Broadway|Ct.|Suite R|Woodville|Grayson County|KY|44289|United States|-6|apartment| +37862|AAAAAAAAGODJAAAA|119|Park |Boulevard|Suite 80|Providence|Union County|MS|56614|United States|-6|single family| +37863|AAAAAAAAHODJAAAA|48|Mill |RD|Suite V|Deerfield|Pittsburg County|OK|79840|United States|-6|single family| +37864|AAAAAAAAIODJAAAA|905|Williams First|Circle|Suite 130|Glendale|Miller County|GA|33951|United States|-5|single family| +37865|AAAAAAAAJODJAAAA|899|Willow Pine|ST|Suite 160|Clinton|Peach County|GA|38222|United States|-5|condo| +37866|AAAAAAAAKODJAAAA|192|River Jackson|Road|Suite 300|Spring Lake|Dodge County|GA|39275|United States|-5|single family| +37867|AAAAAAAALODJAAAA|119|Central River|Court|Suite 50|Walnut Grove|Kiowa County|KS|67752|United States|-6|single family| +37868|AAAAAAAAMODJAAAA|231|Maple |Blvd|Suite 250|Collinsville|Washington County|TX|72459|United States|-6|single family| +37869|AAAAAAAANODJAAAA|250|River |Street|Suite T|Providence|Bayfield County|WI|56614|United States|-6|single family| +37870|AAAAAAAAOODJAAAA|766|14th Sunset|Circle|Suite K|Shiloh|Wise County|TX|79275|United States|-6|single family| +37871|AAAAAAAAPODJAAAA|89|Forest Oak|Street|Suite 250|Buena Vista|Franklin County|TN|35752|United States|-5|apartment| +37872|AAAAAAAAAPDJAAAA|113|Sycamore 12th||||Kenton County||40069||-5|apartment| +37873|AAAAAAAABPDJAAAA|475|Davis 8th|Ln|Suite B|Mount Zion|Okmulgee County|OK|78054|United States|-6|apartment| +37874|AAAAAAAACPDJAAAA|501|Jackson Railroad|Boulevard|Suite 150|Lakewood|Kane County|IL|68877|United States|-6|apartment| +37875|AAAAAAAADPDJAAAA|909|Mill Madison|Blvd|Suite 470|Kingston|Blaine County|MT|64975|United States|-7|condo| +37876|AAAAAAAAEPDJAAAA|599|Oak |Avenue|Suite 280|Lakeview|Union County|IN|48579|United States|-5|apartment| +37877|AAAAAAAAFPDJAAAA|697|North Poplar|Ln|Suite 260|Unionville|Pittsburg County|OK|71711|United States|-6|apartment| +37878|AAAAAAAAGPDJAAAA|253|Main Birch|Court|Suite V|New Hope|Monongalia County|WV|29431|United States|-5|single family| +37879|AAAAAAAAHPDJAAAA|904|Center Wilson|Street|Suite R|Stringtown|Hopkins County|TX|70162|United States|-6|condo| +37880|AAAAAAAAIPDJAAAA|821|5th |Wy|Suite N|Cordova|Franklin city|VA|26938|United States|-5|condo| +37881|AAAAAAAAJPDJAAAA|152|Second Franklin|Ct.|Suite 30|Spring Hill|Lake of the Woods County|MN|56787|United States|-6|single family| +37882|AAAAAAAAKPDJAAAA|130|3rd |Lane|Suite W|Crossroads|Queens County|NY|10534|United States|-5|apartment| +37883|AAAAAAAALPDJAAAA|329|Mill |Dr.|Suite J|Lakewood|Burnett County|WI|58877|United States|-6|apartment| +37884|AAAAAAAAMPDJAAAA|615|Adams Smith|Wy|Suite B|Highland|Costilla County|CO|89454|United States|-7|single family| +37885|AAAAAAAANPDJAAAA|689|Green |Circle|Suite 120|Newport|Shannon County|MO|61521|United States|-6|single family| +37886|AAAAAAAAOPDJAAAA|929|West Johnson|Parkway|Suite F|Franklin|Indiana County|PA|19101|United States|-5|apartment| +37887|AAAAAAAAPPDJAAAA|238|1st View||Suite I||Plaquemines Parish||76614|||single family| +37888|AAAAAAAAAAEJAAAA|881|Mill |Circle|Suite M|Union|Chaves County|NM|88721|United States|-7|apartment| +37889|AAAAAAAABAEJAAAA|692|College Valley|Lane|Suite 440|Greenfield|Saguache County|CO|85038|United States|-7|single family| +37890|AAAAAAAACAEJAAAA|157|Jackson Walnut|Pkwy|Suite Q|Forest Hills|Garvin County|OK|79237|United States|-6|single family| +37891|AAAAAAAADAEJAAAA|468|North |Blvd|Suite C|Friendship|Andrew County|MO|64536|United States|-6|single family| +37892|AAAAAAAAEAEJAAAA|558|Main |Ave|Suite 490|Lakewood|Madison County|OH|48877|United States|-5|apartment| +37893|AAAAAAAAFAEJAAAA|624|Jefferson |Blvd|Suite 130|Williamsville|Okanogan County|WA|98754|United States|-8|condo| +37894|AAAAAAAAGAEJAAAA|724|4th |Blvd|Suite 260|Ferguson|Calvert County|MD|21821|United States|-5|condo| +37895|AAAAAAAAHAEJAAAA|44|Willow Pine|ST|Suite 450|Hamilton|Major County|OK|72808|United States|-6|apartment| +37896|AAAAAAAAIAEJAAAA|682|Johnson 2nd|Lane|Suite S|Pinhook|Yavapai County|AZ|89398|United States|-7|apartment| +37897|AAAAAAAAJAEJAAAA|246|Church 11th|Pkwy|Suite 30|Florence|Sacramento County|CA|93394|United States|-8|apartment| +37898|AAAAAAAAKAEJAAAA|523|3rd East|Lane|Suite U|Fairview|Dougherty County|GA|35709|United States|-5|apartment| +37899|AAAAAAAALAEJAAAA|807|Central |Way|Suite G|Highland Park|Lenawee County|MI|46534|United States|-5|condo| +37900|AAAAAAAAMAEJAAAA|909|Green Valley|ST|Suite 120|Lincoln|Fergus County|MT|61289|United States|-7|single family| +37901|AAAAAAAANAEJAAAA|519|Main Locust|Cir.|Suite K|Pine Grove|Williamsburg city|VA|24593|United States|-5|apartment| +37902|AAAAAAAAOAEJAAAA|41|Mill Walnut|Blvd|Suite A|Shiloh|Hughes County|OK|79275|United States|-6|single family| +37903|AAAAAAAAPAEJAAAA|453|Center View|Wy|Suite B|Brownsville|San Saba County|TX|79310|United States|-6|apartment| +37904|AAAAAAAAABEJAAAA|412|1st |Ln|Suite T|Wyoming|Harrison County|TX|70216|United States|-6|condo| +37905|AAAAAAAABBEJAAAA|994|Sunset |Parkway|Suite H|Springfield|Herkimer County|NY|19303|United States|-5|condo| +37906|AAAAAAAACBEJAAAA|326|6th 14th|Wy|Suite C|Liberty|Dukes County|MA|04051|United States|-5|apartment| +37907|AAAAAAAADBEJAAAA|571|Adams |Way|Suite D|Hamilton|Chouteau County|MT|62808|United States|-7|apartment| +37908|AAAAAAAAEBEJAAAA|488|Walnut |Road|Suite S|Harvey|Harris County|GA|35858|United States|-5|apartment| +37909|AAAAAAAAFBEJAAAA|852|Oak |Way|Suite D|Oak Hill|Franklin County|IN|47838|United States|-5|single family| +37910|AAAAAAAAGBEJAAAA|171|6th Jefferson|Dr.|Suite 420|Providence|Aleutians East Borough|AK|96614|United States|-9|single family| +37911|AAAAAAAAHBEJAAAA|181|Hillcrest Second|Road|Suite G|Jamestown|Spokane County|WA|96867|United States|-8|condo| +37912|AAAAAAAAIBEJAAAA|346|Fourth |Street|Suite 90|Pleasant Hill|Erie County|OH|43604|United States|-5|single family| +37913|AAAAAAAAJBEJAAAA|650|Washington Green|ST|Suite 450|Riverside|Pershing County|NV|89231|United States|-8|apartment| +37914|AAAAAAAAKBEJAAAA|732|Madison |Road|Suite Q|Georgetown|Sherman County|TX|77057|United States|-6|apartment| +37915|AAAAAAAALBEJAAAA|313|9th |Cir.|Suite 180|Red Hill|Motley County|TX|74338|United States|-6|single family| +37916|AAAAAAAAMBEJAAAA|116|Highland |Court|Suite P|Morton|Franklin County|VT|07349|United States|-5|apartment| +37917|AAAAAAAANBEJAAAA|888|Eigth |Lane|Suite H|Antioch|Worth County|GA|38605|United States|-5|single family| +37918|AAAAAAAAOBEJAAAA|938|Davis |ST|Suite 310|Crossroads|Kane County|IL|60534|United States|-6|apartment| +37919|AAAAAAAAPBEJAAAA|638|Lake Smith|Avenue|Suite 330|Lewisburg|Marion County|MO|67538|United States|-6|apartment| +37920|AAAAAAAAACEJAAAA|714|Church |RD|Suite B|Friendship|Pinellas County|FL|34536|United States|-5|single family| +37921|AAAAAAAABCEJAAAA|341|Adams |Street|Suite 420|Mount Vernon|Wayne County|NY|18482|United States|-5|apartment| +37922|AAAAAAAACCEJAAAA|921|Walnut |Road|Suite M|Summit|Coffey County|KS|60499|United States|-6|condo| +37923|AAAAAAAADCEJAAAA|790|5th |Parkway|Suite R|Oakland|Roseau County|MN|59843|United States|-6|apartment| +37924|AAAAAAAAECEJAAAA|491|Park 5th|Lane|Suite 60|Green Acres|Richland Parish|LA|77683|United States|-6|condo| +37925|AAAAAAAAFCEJAAAA|315|8th |Road|Suite K|Weldon|Wheeler County|TX|76277|United States|-6|condo| +37926|AAAAAAAAGCEJAAAA|79|Spruce |Lane|Suite 360|Woodlawn|Greene County|IL|64098|United States|-6|single family| +37927|AAAAAAAAHCEJAAAA|895|Jackson |Drive|Suite K|Harmony|Brown County|KS|65804|United States|-6|single family| +37928|AAAAAAAAICEJAAAA|214|Oak 1st|Boulevard|Suite 80|Green Acres|Jones County|NC|27683|United States|-5|condo| +37929|AAAAAAAAJCEJAAAA|21|Green Hickory|Pkwy|Suite U|Woodland|Morgan County|CO|84854|United States|-7|condo| +37930|AAAAAAAAKCEJAAAA|955|Lake |Pkwy|Suite 470|Fairview|Dickinson County|MI|45709|United States|-5|single family| +37931|AAAAAAAALCEJAAAA|671|View Green|Cir.|Suite Y|Spring Hill|Appanoose County|IA|56787|United States|-6|single family| +37932|AAAAAAAAMCEJAAAA|135|Forest |Ct.|Suite G|Edgewood|Newton County|MS|50069|United States|-6|single family| +37933|AAAAAAAANCEJAAAA|105|3rd 4th|Wy|Suite 50|New Hope|Adair County|IA|59431|United States|-6|apartment| +37934|AAAAAAAAOCEJAAAA|116|Wilson |Road|Suite E|Florence|Tompkins County|NY|13394|United States|-5|apartment| +37935|AAAAAAAAPCEJAAAA|213||Court|||Reeves County||71521||-6|apartment| +37936|AAAAAAAAADEJAAAA|212|Pine Maple|Ct.|Suite 150|Shady Grove|Jefferson County|OR|92812|United States|-8|condo| +37937|AAAAAAAABDEJAAAA|151|Green |Circle|Suite L|Midway|Carson County|TX|71904|United States|-6|apartment| +37938|AAAAAAAACDEJAAAA|332|Eigth 3rd|Court|Suite 300|Macedonia|Presidio County|TX|71087|United States|-6|condo| +37939|AAAAAAAADDEJAAAA|752|1st |Ln|Suite 190|Lebanon|Hampton County|SC|22898|United States|-5|apartment| +37940|AAAAAAAAEDEJAAAA|610|11th Ridge|Lane|Suite 380|Plainview|Jefferson Davis Parish|LA|73683|United States|-6|apartment| +37941|AAAAAAAAFDEJAAAA|265|College |Cir.|Suite D|Stringtown|Bell County|KY|40162|United States|-6|apartment| +37942|AAAAAAAAGDEJAAAA|990|Fifth |Ave|Suite 230|Walnut Grove|Patrick County|VA|27752|United States|-5|condo| +37943|AAAAAAAAHDEJAAAA|255|South |Cir.|Suite E|Lebanon|Meeker County|MN|52898|United States|-6|single family| +37944|AAAAAAAAIDEJAAAA|94|9th 7th|Way|Suite O|Macedonia|New Haven County|CT|01687|United States|-5|apartment| +37945|AAAAAAAAJDEJAAAA|341|Sunset |Boulevard|Suite 370|Unionville|Tipton County|IN|41711|United States|-5|single family| +37946|AAAAAAAAKDEJAAAA||College |ST|Suite 460|||MO|69840||-6|condo| +37947|AAAAAAAALDEJAAAA|531|Third Ash|Court|Suite D|Hamilton|Lauderdale County|TN|32808|United States|-6|apartment| +37948|AAAAAAAAMDEJAAAA|961|Elm Elm|Ave|Suite Y|Highland|Trinity County|CA|99454|United States|-8|apartment| +37949|AAAAAAAANDEJAAAA|998|Fifth Third|Parkway|Suite 110|Union|Van Zandt County|TX|78721|United States|-6|apartment| +37950|AAAAAAAAODEJAAAA|165|Sunset Church|Street|Suite 80|Pleasant Grove|Knox County|TN|34136|United States|-6|condo| +37951|AAAAAAAAPDEJAAAA|692|2nd |ST|Suite 30|Carter|Carson County|TX|70919|United States|-6|single family| +37952|AAAAAAAAAEEJAAAA|135|Railroad |Parkway|Suite 50|Crystal Springs|Centre County|PA|17912|United States|-5|condo| +37953|AAAAAAAABEEJAAAA|118|8th Pine|Avenue|Suite S|Pleasant Valley|Roberts County|TX|72477|United States|-6|condo| +37954|AAAAAAAACEEJAAAA|934|11th Main|Ave|Suite T|Greenfield|Tioga County|PA|15038|United States|-5|apartment| +37955|AAAAAAAADEEJAAAA|281|Meadow College|Way|Suite 290|Union Hill|Alexandria city|VA|27746|United States|-5|apartment| +37956|AAAAAAAAEEEJAAAA|815|Ash 5th|Drive|Suite F|Stony Point|Sumter County|SC|24255|United States|-5|condo| +37957|AAAAAAAAFEEJAAAA|546|Washington |Drive|Suite 140|Belmont|Cleveland County|OK|70191|United States|-6|single family| +37958|AAAAAAAAGEEJAAAA|929|Hill Fifth|Ave|Suite M|Marion|San Juan County|NM|80399|United States|-7|apartment| +37959|AAAAAAAAHEEJAAAA|91|6th |Avenue|Suite Y|Pleasant Valley|Grant County|OK|72477|United States|-6|apartment| +37960|AAAAAAAAIEEJAAAA|236|River Woodland|Dr.|Suite T|Shiloh|Rockwall County|TX|79275|United States|-6|condo| +37961|AAAAAAAAJEEJAAAA|766|Central |Court|Suite I|Florence|Livingston County|NY|13394|United States|-5|condo| +37962|AAAAAAAAKEEJAAAA|395|1st |Boulevard|Suite 20|Wilson|Washington County|MS|56971|United States|-6|condo| +37963|AAAAAAAALEEJAAAA|264|1st Lincoln|Ct.|Suite N|Hamilton|Mercer County|WV|22808|United States|-5|condo| +37964|AAAAAAAAMEEJAAAA|57|15th |Ct.|Suite 190|Unionville|Grayson County|VA|21711|United States|-5|condo| +37965|AAAAAAAANEEJAAAA|966|Spring |Dr.|Suite E|Mount Vernon|Cheyenne County|NE|68482|United States|-6|apartment| +37966|AAAAAAAAOEEJAAAA|902|Miller |Way|Suite X|Salem||||United States||| +37967|AAAAAAAAPEEJAAAA|68|Williams Park|Ave|Suite 250|Maple Grove|Bowman County|ND|58252|United States|-6|single family| +37968|AAAAAAAAAFEJAAAA|1|12th Church|Court|Suite S|Jamestown|Comanche County|KS|66867|United States|-6|condo| +37969|AAAAAAAABFEJAAAA|104|Elm Second|Cir.|Suite Q|Five Forks|Huerfano County|CO|82293|United States|-7|condo| +37970|AAAAAAAACFEJAAAA|682|Fifth |Blvd|Suite F|Harmony|Reno County|KS|65804|United States|-6|condo| +37971|AAAAAAAADFEJAAAA|114|Hickory Oak|Ln|Suite 360|Florence|Humboldt County|IA|53394|United States|-6|condo| +37972|AAAAAAAAEFEJAAAA|31|Washington Sixth|Street|Suite D|Farmington|Williamson County|TN|39145|United States|-5|apartment| +37973|AAAAAAAAFFEJAAAA|27|Ridge |Road|Suite K|Walnut Grove|Anderson County|TX|77752|United States|-6|condo| +37974|AAAAAAAAGFEJAAAA|758|7th Meadow|Parkway|Suite H|Bethel|Newton County|MO|65281|United States|-6|single family| +37975|AAAAAAAAHFEJAAAA|371|||||Stephenson County|IL||||| +37976|AAAAAAAAIFEJAAAA|776|12th Park|ST|Suite 400|Lakeside|New Kent County|VA|29532|United States|-5|condo| +37977|AAAAAAAAJFEJAAAA|690|||Suite O|Liberty|||33451||-5|condo| +37978|AAAAAAAAKFEJAAAA|856|Pine |Wy|Suite 290|Macedonia|Boulder County|CO|81087|United States|-7|single family| +37979|AAAAAAAALFEJAAAA|642||Way||Five Forks|Cascade County||62293|United States||single family| +37980|AAAAAAAAMFEJAAAA|547|Oak |Street|Suite 490|Midway|Wayne County|MS|51904|United States|-6|single family| +37981|AAAAAAAANFEJAAAA|455|River |Road|Suite 60|Union Hill|Sandusky County|OH|47746|United States|-5|condo| +37982|AAAAAAAAOFEJAAAA|21|South Walnut|Pkwy|Suite V|Griffin|Somerset County|PA|11204|United States|-5|single family| +37983|AAAAAAAAPFEJAAAA|436|Elm |Road|Suite 460|Cedar Grove|Grand County|UT|80411|United States|-7|single family| +37984|AAAAAAAAAGEJAAAA|672|Hill |Blvd|Suite 10|Avery|Iowa County|WI|50194|United States|-6|apartment| +37985|AAAAAAAABGEJAAAA|873|Forest Miller|Boulevard|Suite 400|Rutland|Tulare County|CA|98375|United States|-8|condo| +37986|AAAAAAAACGEJAAAA|929|Church Madison|Blvd|Suite 240|Pine Grove|Mercer County|IL|64593|United States|-6|condo| +37987|AAAAAAAADGEJAAAA|992|Walnut Woodland|Pkwy|Suite L|Ashland|Elk County|PA|14244|United States|-5|apartment| +37988|AAAAAAAAEGEJAAAA|691|Chestnut Willow|Lane|Suite 110|Parkwood|Montgomery County|VA|21669|United States|-5|condo| +37989|AAAAAAAAFGEJAAAA|887|Walnut |Dr.|Suite N|Brownsville|Person County|NC|29310|United States|-5|single family| +37990|AAAAAAAAGGEJAAAA|510|West Hickory|Circle|Suite 260|Greenwood|Juneau Borough|AK|98828|United States|-9|single family| +37991|AAAAAAAAHGEJAAAA|345|4th |Street|Suite V|Providence|Arenac County|MI|46614|United States|-5|condo| +37992|AAAAAAAAIGEJAAAA|348|Railroad |Ave|Suite 200|Valley View|Sacramento County|CA|95124|United States|-8|single family| +37993|AAAAAAAAJGEJAAAA|291|Willow Maple|Blvd|Suite 90|Buena Vista|Cumberland County|ME|06352|United States|-5|condo| +37994|AAAAAAAAKGEJAAAA|365|Third Johnson|Blvd|Suite B|Wilson|Gadsden County|FL|36971|United States|-5|single family| +37995|AAAAAAAALGEJAAAA|224|Forest Wilson|Avenue|Suite X|Nichols|Kittson County|MN|57940|United States|-6|apartment| +37996|AAAAAAAAMGEJAAAA|566|6th |Way|Suite T|Liberty|Carroll County|VA|23451|United States|-5|condo| +37997|AAAAAAAANGEJAAAA|||ST|Suite U|Oak Ridge||TX|78371|United States|-6|| +37998|AAAAAAAAOGEJAAAA|385|College 13th|Dr.|Suite 130|Royal|Talladega County|AL|35819|United States|-6|single family| +37999|AAAAAAAAPGEJAAAA|929|Valley Spruce|Boulevard|Suite 380|Riverdale|Newton County|IN|49391|United States|-5|condo| +38000|AAAAAAAAAHEJAAAA|412|Railroad Hill|Road|Suite 250|Riverview|Montgomery County|IN|49003|United States|-5|condo| +38001|AAAAAAAABHEJAAAA|297|West 7th|Circle|Suite 370|Lakeside|Clay County|GA|39532|United States|-5|apartment| +38002|AAAAAAAACHEJAAAA|598|8th |ST|Suite D|Greenfield|Liberty County|MT|65038|United States|-7|apartment| +38003|AAAAAAAADHEJAAAA|997|East |Court|Suite 490|Woodland|Clark County|ID|84854|United States|-7|single family| +38004|AAAAAAAAEHEJAAAA|404|5th |Drive|Suite U|Wildwood|Carbon County|UT|86871|United States|-7|single family| +38005|AAAAAAAAFHEJAAAA|872||||Stringtown|Schley County|||United States||| +38006|AAAAAAAAGHEJAAAA|508|15th |RD|Suite 80|Red Hill|Greenwood County|SC|24338|United States|-5|condo| +38007|AAAAAAAAHHEJAAAA|750|1st Fourth|Way|Suite 170|Pleasant Grove|Hopkins County|TX|74136|United States|-6|apartment| +38008|AAAAAAAAIHEJAAAA|984|Railroad First|Wy|Suite 260|Newtown|Ballard County|KY|41749|United States|-6|condo| +38009|AAAAAAAAJHEJAAAA|16|South |Blvd|Suite 370|Oak Hill|Beaver County|UT|87838|United States|-7|single family| +38010|AAAAAAAAKHEJAAAA|580|7th |Ave|Suite D|Buena Vista|Tarrant County|TX|75752|United States|-6|apartment| +38011|AAAAAAAALHEJAAAA|776|Spring 7th|Drive|Suite P|Woodcrest|Adair County|IA|54919|United States|-6|single family| +38012|AAAAAAAAMHEJAAAA|76|Johnson |Drive|Suite 210|Pleasant Grove|Claiborne Parish|LA|74136|United States|-6|condo| +38013|AAAAAAAANHEJAAAA|293|Forest 7th|Way|Suite 450|Franklin|Cabarrus County|NC|29101|United States|-5|condo| +38014|AAAAAAAAOHEJAAAA|784|Cedar Adams|Avenue|Suite 50|Oakdale|Winnebago County|WI|59584|United States|-6|condo| +38015|AAAAAAAAPHEJAAAA|157|Lee |Lane|Suite P|Sulphur Springs|Brown County|TX|78354|United States|-6|apartment| +38016|AAAAAAAAAIEJAAAA|629|Ash Elm|Avenue|Suite 230|Walnut Grove|Searcy County|AR|77752|United States|-6|apartment| +38017|AAAAAAAABIEJAAAA|113|Elm Church|Court|Suite 390|Glenwood|Cheboygan County|MI|43511|United States|-5|apartment| +38018|AAAAAAAACIEJAAAA|984|13th 7th|Circle|Suite T|Allentown|Monroe County|MI|41838|United States|-5|condo| +38019|AAAAAAAADIEJAAAA|211|Ash |Circle|Suite W|Lakewood|Harlan County|KY|48877|United States|-6|apartment| +38020|AAAAAAAAEIEJAAAA|184|Wilson Mill|Drive|Suite 70|Hopewell|Dougherty County|GA|30587|United States|-5|apartment| +38021|AAAAAAAAFIEJAAAA|980|Valley Lakeview|Ave|Suite M|Woodlawn|Washington County|IL|64098|United States|-6|condo| +38022|AAAAAAAAGIEJAAAA|863|Adams Third|ST|Suite 110|Midway|Houston County|MN|51904|United States|-6|condo| +38023|AAAAAAAAHIEJAAAA|741|1st 2nd|Street|Suite 380|Georgetown|Sumter County|GA|37057|United States|-5|condo| +38024|AAAAAAAAIIEJAAAA|743|Washington Willow|Drive|Suite 450|Spring Valley|Emporia city|VA|26060|United States|-5|condo| +38025|AAAAAAAAJIEJAAAA|503|North 11th|Ave|Suite W|Macedonia|Hamlin County|SD|51087|United States|-7|apartment| +38026|AAAAAAAAKIEJAAAA|155|Maple Lincoln||Suite E||||76871|United States|-6|condo| +38027|AAAAAAAALIEJAAAA|47|South Maple|Ct.|Suite D|Mount Vernon|Sevier County|AR|78482|United States|-6|apartment| +38028|AAAAAAAAMIEJAAAA|608|Willow |Avenue|Suite 130|Greenwood|Phillips County|MT|68828|United States|-7|single family| +38029|AAAAAAAANIEJAAAA|637|3rd Woodland|Avenue|Suite A|Lakeside|Coffey County|KS|69532|United States|-6|condo| +38030|AAAAAAAAOIEJAAAA|94|Johnson Davis|Pkwy|Suite B|Clinton|Calhoun County|AR|78222|United States|-6|apartment| +38031|AAAAAAAAPIEJAAAA|358|Sycamore 1st|Drive|Suite W|Maple Grove|Phillips County|MT|68252|United States|-7|apartment| +38032|AAAAAAAAAJEJAAAA|88|Lake Lee|Lane|Suite 30|Union Hill|Washington County|UT|87746|United States|-7|condo| +38033|AAAAAAAABJEJAAAA||||Suite P|||||United States||apartment| +38034|AAAAAAAACJEJAAAA|441|Madison |Cir.|Suite 490|Glendale|Noble County|IN|43951|United States|-5|condo| +38035|AAAAAAAADJEJAAAA|76|Hill |Way|Suite F|Stringtown|Malheur County|OR|90162|United States|-8|apartment| +38036|AAAAAAAAEJEJAAAA|255|Cedar |Drive|Suite 10|Unionville|Ross County|OH|41711|United States|-5|apartment| +38037|AAAAAAAAFJEJAAAA|410|Third Main|Dr.|Suite T|Ashland|Calhoun County|IA|54244|United States|-6|condo| +38038|AAAAAAAAGJEJAAAA|428|8th |Ct.|Suite T|Jackson|Marion County|OR|99583|United States|-8|apartment| +38039|AAAAAAAAHJEJAAAA|820|Pine Cherry|Way|Suite 30|Glendale|Roberts County|SD|53951|United States|-7|single family| +38040|AAAAAAAAIJEJAAAA|589|Hill Sunset|Wy|Suite 150|Riverdale|Buchanan County|MO|69391|United States|-6|condo| +38041|AAAAAAAAJJEJAAAA|991|Dogwood Lee|ST|Suite P|Pleasant Grove|Rush County|IN|44136|United States|-5|condo| +38042|AAAAAAAAKJEJAAAA|166|Laurel Meadow|Ln|Suite 360|Wildwood|Bristol County|RI|07471|United States|-5|condo| +38043|AAAAAAAALJEJAAAA|253|Sixth |Ln|Suite M|Brentwood|Ripley County|MO|64188|United States|-6|condo| +38044|AAAAAAAAMJEJAAAA|495|Hill |Boulevard|Suite 190|Unionville|Lewis County|KY|41711|United States|-5|apartment| +38045|AAAAAAAANJEJAAAA|64|Oak |Cir.|Suite 90|Waterloo|Hancock County|TN|31675|United States|-5|condo| +38046|AAAAAAAAOJEJAAAA|739|Lake |Cir.|Suite U|Sulphur Springs|Uintah County|UT|88354|United States|-7|apartment| +38047|AAAAAAAAPJEJAAAA|493|Fourth |Road|Suite R|Franklin|White County|IL|69101|United States|-6|single family| +38048|AAAAAAAAAKEJAAAA|303|Sycamore Second|Pkwy|Suite 200|Oak Grove|Pike County|IL|68370|United States|-6|condo| +38049|AAAAAAAABKEJAAAA|25|Locust 2nd|Blvd|Suite 100|Summit|Silver Bow County|MT|60499|United States|-7|single family| +38050|AAAAAAAACKEJAAAA|461|River |Dr.|Suite H|Fairview|Graham County|KS|65709|United States|-6|single family| +38051|AAAAAAAADKEJAAAA|26|Washington 4th|Road|Suite 440|Edgewood|Riverside County|CA|90069|United States|-8|single family| +38052|AAAAAAAAEKEJAAAA|616|Central View|Way|Suite 210|Woodland|Ulster County|NY|14854|United States|-5|single family| +38053|AAAAAAAAFKEJAAAA|293|Seventh 14th|Ct.|Suite D|Riverdale|Snohomish County|WA|99391|United States|-8|single family| +38054|AAAAAAAAGKEJAAAA|507|Lakeview 2nd|Boulevard|Suite 240|Oakland|El Dorado County|CA|99843|United States|-8|single family| +38055|AAAAAAAAHKEJAAAA|848|Birch |ST|Suite I|Oak Hill|Liberty County|MT|67838|United States|-7|condo| +38056|AAAAAAAAIKEJAAAA|83|West 5th|Street|Suite 410|Summit|Marion County|TN|30499|United States|-6|apartment| +38057|AAAAAAAAJKEJAAAA|479|Park 7th|Pkwy|Suite W|Wilson|Montgomery County|TX|76971|United States|-6|condo| +38058|AAAAAAAAKKEJAAAA|662|Railroad |Cir.|Suite P|Oakwood|Kent County|TX|70169|United States|-6|single family| +38059|AAAAAAAALKEJAAAA||||Suite U||Dallas County||||-6|| +38060|AAAAAAAAMKEJAAAA|532|Lincoln Oak|Boulevard|Suite A|Jackson|Niagara County|NY|19583|United States|-5|condo| +38061|AAAAAAAANKEJAAAA|41|Birch First|Lane|Suite O|Sulphur Springs|Franklin County|OH|48354|United States|-5|apartment| +38062|AAAAAAAAOKEJAAAA|348|Sycamore 2nd|Parkway|Suite 330|Siloam|Cabarrus County|NC|28948|United States|-5|apartment| +38063|AAAAAAAAPKEJAAAA|59|Broadway Oak|Ave|Suite 440|Bethel|Cedar County|MO|65281|United States|-6|condo| +38064|AAAAAAAAALEJAAAA|196|Laurel |Street|Suite W|Woodland|Lafayette County|MO|64854|United States|-6|single family| +38065|AAAAAAAABLEJAAAA|216|Highland |Circle|Suite E|Frenchtown|Clarke County|GA|32629|United States|-5|condo| +38066|AAAAAAAACLEJAAAA|127|Highland Fourteenth|Dr.|Suite 90|Riverview|Niagara County|NY|19003|United States|-5|apartment| +38067|AAAAAAAADLEJAAAA|871|Miller |Wy|Suite W|Greenville|Montgomery County|KS|61387|United States|-6|apartment| +38068|AAAAAAAAELEJAAAA|241|Cedar Ash|ST|Suite M|Edgewood|Montgomery County|TN|30069|United States|-6|apartment| +38069|AAAAAAAAFLEJAAAA|359|Madison |Road|Suite 450|Argyle|Putnam County|OH|48722|United States|-5|single family| +38070|AAAAAAAAGLEJAAAA|108|Davis |Dr.|Suite N|Antioch|Gage County|NE|68605|United States|-6|condo| +38071|AAAAAAAAHLEJAAAA|764|4th Center|Avenue|Suite 460|Bridgeport|Unicoi County|TN|35817|United States|-6|condo| +38072|AAAAAAAAILEJAAAA|620|Spring Birch|Pkwy|Suite A|Friendship|Boulder County|CO|84536|United States|-7|apartment| +38073|AAAAAAAAJLEJAAAA|898|Church |Dr.|Suite 180|Bethel|Yellow Medicine County|MN|55281|United States|-6|apartment| +38074|AAAAAAAAKLEJAAAA|784|Hickory |Cir.|Suite 110|Mount Pleasant|Madison County|MO|61933|United States|-6|condo| +38075|AAAAAAAALLEJAAAA|311|South |RD|Suite T|Cedar Grove|Woodford County|KY|40411|United States|-6|apartment| +38076|AAAAAAAAMLEJAAAA|276|Cedar |Ave|Suite B|Florence|Randall County|TX|73394|United States|-6|single family| +38077|AAAAAAAANLEJAAAA|178|Ninth |Circle|Suite O|Woodville|Muscatine County|IA|54289|United States|-6|condo| +38078|AAAAAAAAOLEJAAAA|922|Franklin |RD|Suite J|Clifton|Warren County|TN|38014|United States|-6|condo| +38079|AAAAAAAAPLEJAAAA|442|Laurel |Wy|Suite 250|Fairview|Lawrence County|MS|55709|United States|-6|apartment| +38080|AAAAAAAAAMEJAAAA|535|East 7th|Street|Suite 380|Riverdale|Perry County|KY|49391|United States|-5|apartment| +38081|AAAAAAAABMEJAAAA|875|Main Park|Parkway|Suite X|Salem|Sharp County|AR|78048|United States|-6|condo| +38082|AAAAAAAACMEJAAAA|568|4th |Dr.|Suite 80|Franklin|Newton County|TX|79101|United States|-6|apartment| +38083|AAAAAAAADMEJAAAA|507|Oak Third|Street|Suite 0|Sullivan|Madison County|ID|80451|United States|-7|single family| +38084|AAAAAAAAEMEJAAAA|245|1st Pine|Ct.|Suite 400|Kingston|Buena Vista County|IA|54975|United States|-6|condo| +38085|AAAAAAAAFMEJAAAA|559|Church |Wy|Suite M|Northwood|Clark County|IL|64104|United States|-6|single family| +38086|AAAAAAAAGMEJAAAA|42|Hill Jackson||Suite S||Hutchinson County||78482|United States|-6|condo| +38087|AAAAAAAAHMEJAAAA|116|Broadway Sycamore|Circle|Suite 110|Oak Grove|Polk County|MN|58370|United States|-6|single family| +38088|AAAAAAAAIMEJAAAA|374|Johnson ||Suite 420|Concord|||54107|United States||apartment| +38089|AAAAAAAAJMEJAAAA|803|Tenth |Ave|Suite D|Centerville|Flathead County|MT|60059|United States|-7|apartment| +38090|AAAAAAAAKMEJAAAA|559|Adams 5th|Boulevard|Suite 380|Fairview|Emanuel County|GA|35709|United States|-5|single family| +38091|AAAAAAAALMEJAAAA|650|5th |Street|Suite 310|Summit|Johnson County|AR|70499|United States|-6|condo| +38092|AAAAAAAAMMEJAAAA|290|Walnut |Boulevard|Suite 430|Belmont|Edwards County|TX|70191|United States|-6|condo| +38093|AAAAAAAANMEJAAAA|800|7th 4th|Circle|Suite 220|Lakewood|Beaufort County|SC|28877|United States|-5|single family| +38094|AAAAAAAAOMEJAAAA|609|Spring |Road|Suite O|Union Hill|Limestone County|AL|37746|United States|-6|condo| +38095|AAAAAAAAPMEJAAAA|908|Spring 12th|Ct.|Suite M|Walnut Grove|Hughes County|OK|77752|United States|-6|apartment| +38096|AAAAAAAAANEJAAAA|701|Elm Second|Court|Suite V|Oakland|Bay County|MI|49843|United States|-5|apartment| +38097|AAAAAAAABNEJAAAA|949|Willow |Dr.|Suite 180|Walnut Grove|Roanoke County|VA|27752|United States|-5|single family| +38098|AAAAAAAACNEJAAAA|138|Sycamore |Street|Suite 300||Red River Parish|LA||United States|-6|single family| +38099|AAAAAAAADNEJAAAA|692|10th |ST|Suite O|West Liberty|Saline County|AR|74752|United States|-6|single family| +38100|AAAAAAAAENEJAAAA|453|Ninth Miller|Lane|Suite 200|Hardy|Jackson County|KY|45354|United States|-6|condo| +38101|AAAAAAAAFNEJAAAA|323|Woodland |Parkway|Suite E|Salem|Lake County|MT|68048|United States|-7|apartment| +38102|AAAAAAAAGNEJAAAA|350|7th Birch|Dr.|Suite A|Springdale|Kidder County|ND|58883|United States|-6|condo| +38103|AAAAAAAAHNEJAAAA|263|7th |Cir.|Suite A|Buena Vista|Kingfisher County|OK|75752|United States|-6|apartment| +38104|AAAAAAAAINEJAAAA|728|Park |Road||||FL|30411|United States||single family| +38105|AAAAAAAAJNEJAAAA|498|Williams |Pkwy|Suite 290|Lincoln|Monroe County|PA|11289|United States|-5|apartment| +38106|AAAAAAAAKNEJAAAA|963|Ridge 3rd|Avenue|Suite G|Greenville|Mitchell County|IA|51387|United States|-6|apartment| +38107|AAAAAAAALNEJAAAA|577|Sycamore |Way|Suite A|Lakeview|Hamblen County|TN|38579|United States|-5|single family| +38108|AAAAAAAAMNEJAAAA|137|Lee Hillcrest|Lane|Suite N|Spring Valley|Eureka County|NV|86060|United States|-8|apartment| +38109|AAAAAAAANNEJAAAA|996|Franklin |||||||||| +38110|AAAAAAAAONEJAAAA|256|First Miller|Ct.|Suite B|Cumberland|Schoolcraft County|MI|48971|United States|-5|single family| +38111|AAAAAAAAPNEJAAAA|837|Cedar |Lane|Suite 80|Concord|Richmond County|GA|34107|United States|-5|single family| +38112|AAAAAAAAAOEJAAAA|973|Oak Lake|Dr.|Suite 250|Brownsville|Burt County|NE|69310|United States|-6|condo| +38113|AAAAAAAABOEJAAAA|69|10th |Drive|Suite B|Fairmount|Cherokee County|KS|66539|United States|-6|apartment| +38114|AAAAAAAACOEJAAAA|708|11th Birch|Cir.|Suite 100|Riverview|Turner County|SD|59003|United States|-7|apartment| +38115|AAAAAAAADOEJAAAA|5|Adams Wilson|Boulevard|Suite S|Riverview|Beaver County|OK|79003|United States|-6|condo| +38116|AAAAAAAAEOEJAAAA|247|Park |Cir.|Suite X|Unionville|Union County|NM|81711|United States|-7|apartment| +38117|AAAAAAAAFOEJAAAA|679|East 1st|Parkway|Suite Y|Lincoln|Dubois County|IN|41289|United States|-5|apartment| +38118|AAAAAAAAGOEJAAAA|903|Smith Lakeview|Way|Suite F|Crossroads|Linn County|IA|50534|United States|-6|apartment| +38119|AAAAAAAAHOEJAAAA|644|Eigth Ash|Way|Suite G|Georgetown|Cheyenne County|NE|67057|United States|-6|apartment| +38120|AAAAAAAAIOEJAAAA|429|Sunset Meadow|Dr.|Suite T|Red Hill|Grant County|KS|64338|United States|-6|apartment| +38121|AAAAAAAAJOEJAAAA|124|Sycamore |Parkway|Suite 140|Oakwood|Okanogan County|WA|90169|United States|-8|condo| +38122|AAAAAAAAKOEJAAAA|571|Cedar Sunset|Parkway|Suite N|Lincoln|Norfolk city|VA|21289|United States|-5|condo| +38123|AAAAAAAALOEJAAAA|68|Sunset Pine|RD|Suite 10|Woodland|Montague County|TX|74854|United States|-6|condo| +38124|AAAAAAAAMOEJAAAA|191|Mill Dogwood|Street|Suite L|Forest Hills|Brewster County|TX|79237|United States|-6|apartment| +38125|AAAAAAAANOEJAAAA|177|6th |Pkwy|Suite X|Glenwood|Lincoln County|GA|33511|United States|-5|single family| +38126|AAAAAAAAOOEJAAAA|888|6th |Wy|Suite 0|Oakdale|Park County|WY|89584|United States|-7|single family| +38127|AAAAAAAAPOEJAAAA|741|Lincoln |||Mount Olive||NY||||condo| +38128|AAAAAAAAAPEJAAAA|708|Miller |ST|Suite 370|Woodlawn|Blackford County|IN|44098|United States|-5|single family| +38129|AAAAAAAABPEJAAAA|291|Spruce 1st|Cir.|Suite 450|Centerville|Douglas County|GA|30059|United States|-5|condo| +38130|AAAAAAAACPEJAAAA|111|Railroad |Road|Suite T|Mount Vernon|Black Hawk County|IA|58482|United States|-6|single family| +38131|AAAAAAAADPEJAAAA|337|Maple College|Ln|Suite H|Waterloo|Jasper County|IL|61675|United States|-6|condo| +38132|AAAAAAAAEPEJAAAA|412|Franklin River|Avenue|Suite G|Oakland|Boise County|ID|89843|United States|-7|apartment| +38133|AAAAAAAAFPEJAAAA|649|4th |ST|Suite 270|Oak Grove|Calhoun County|TX|78370|United States|-6|single family| +38134|AAAAAAAAGPEJAAAA|663|4th 7th|Boulevard|Suite 290|Edgewood|Bremer County|IA|50069|United States|-6|condo| +38135|AAAAAAAAHPEJAAAA|951|Elm |Drive|Suite Q|Oakwood|Rockbridge County|VA|20169|United States|-5|single family| +38136|AAAAAAAAIPEJAAAA|675|Woodland |Street|Suite P|Arcola|Teton County|MT|61654|United States|-7|single family| +38137|AAAAAAAAJPEJAAAA|879|Spring Lincoln|Blvd|Suite 410|Oak Hill|Holt County|NE|67838|United States|-7|apartment| +38138|AAAAAAAAKPEJAAAA|410|8th |Pkwy|Suite E|Pleasant Grove|Okanogan County|WA|94136|United States|-8|single family| +38139|AAAAAAAALPEJAAAA|||Boulevard||||||United States|-6|| +38140|AAAAAAAAMPEJAAAA|660|Jefferson |Street|Suite 430|Jamestown|Calhoun County|MI|46867|United States|-5|condo| +38141|AAAAAAAANPEJAAAA|315|10th 1st|Cir.|Suite 420|Saint Johns|Washakie County|WY|85717|United States|-7|single family| +38142|AAAAAAAAOPEJAAAA|277|Laurel |ST|Suite G|Enterprise|Covington County|MS|51757|United States|-6|condo| +38143|AAAAAAAAPPEJAAAA|445|Elm |RD|Suite 420|Pleasant Grove|Lewis County|WV|24136|United States|-5|apartment| +38144|AAAAAAAAAAFJAAAA|452|3rd Park|Lane|Suite 370|Oakland|Lawrence County|MO|69843|United States|-6|apartment| +38145|AAAAAAAABAFJAAAA|257|South |Ln|Suite E|Jamestown|Lauderdale County|TN|36867|United States|-6|single family| +38146|AAAAAAAACAFJAAAA|754|Church |Ln|Suite X|Greenfield|Palm Beach County|FL|35038|United States|-5|apartment| +38147|AAAAAAAADAFJAAAA|110|Sycamore |Wy|Suite 270|Centerville|Dickinson County|MI|40059|United States|-5|condo| +38148|AAAAAAAAEAFJAAAA|748|Cedar |Blvd|Suite Q|Mount Zion|Mitchell County|GA|38054|United States|-5|single family| +38149|AAAAAAAAFAFJAAAA|118|Elm Johnson|Ave|Suite 400|Hopewell|Jackson County|KS|60587|United States|-6|apartment| +38150|AAAAAAAAGAFJAAAA|274|Spruce View|Dr.|Suite 300|Unionville|Aleutians East Borough|AK|91711|United States|-9|condo| +38151|AAAAAAAAHAFJAAAA|251|3rd |Way|Suite G|Lebanon|Lee County|FL|32898|United States|-5|condo| +38152|AAAAAAAAIAFJAAAA|191|Sixth |Ln|Suite R|Ashland|Gasconade County|MO|64244|United States|-6|condo| +38153|AAAAAAAAJAFJAAAA|594|Park |Blvd|Suite U|Kingston|San Augustine County|TX|74975|United States|-6|single family| +38154|AAAAAAAAKAFJAAAA|972|Broadway Chestnut|Street|Suite A|Union|Ness County|KS|68721|United States|-6|condo| +38155|AAAAAAAALAFJAAAA|934|Meadow 6th|Avenue|Suite 120|Highland|Cottle County|TX|79454|United States|-6|condo| +38156|AAAAAAAAMAFJAAAA|713|Fourth |Avenue|Suite 260|Mount Vernon|Cherokee County|NC|28482|United States|-5|condo| +38157|AAAAAAAANAFJAAAA|308|Madison |RD|Suite U|Riverview|New London County|CT|09603|United States|-5|condo| +38158|AAAAAAAAOAFJAAAA|274||||Friendship|King George County|||United States||condo| +38159|AAAAAAAAPAFJAAAA|482|Elm First|Street|Suite 50|Mount Zion|Ashley County|AR|78054|United States|-6|condo| +38160|AAAAAAAAABFJAAAA|430|7th |Ct.|Suite M|Bunker Hill|Arlington County|VA|20150|United States|-5|condo| +38161|AAAAAAAABBFJAAAA|128|Jackson |Circle|Suite 60|Bethel|Rock County|MN|55281|United States|-6|apartment| +38162|AAAAAAAACBFJAAAA|211|Lincoln Ridge|Street|Suite P|Valley View|Alleghany County|VA|25124|United States|-5|single family| +38163|AAAAAAAADBFJAAAA|953|Elm Eigth|Ct.|Suite L|Riverview|Iron County|WI|59003|United States|-6|apartment| +38164|AAAAAAAAEBFJAAAA|442|1st Sixth|Road|Suite A|Forest Hills|Shelby County|IN|49237|United States|-5|apartment| +38165|AAAAAAAAFBFJAAAA|56||Way|Suite 140|White Oak||||||| +38166|AAAAAAAAGBFJAAAA|262|Valley Locust|Road|Suite 320|Union|Zapata County|TX|78721|United States|-6|single family| +38167|AAAAAAAAHBFJAAAA|464|Twelfth |Dr.|Suite O|Greenville|Houston County|AL|31387|United States|-6|single family| +38168|AAAAAAAAIBFJAAAA|373|Cedar 15th|Boulevard|Suite 50|San Jose|Waushara County|WI|58003|United States|-6|single family| +38169|AAAAAAAAJBFJAAAA|107|8th Fourth|Circle|Suite 30|Bunker Hill|Clay County|IA|50150|United States|-6|single family| +38170|AAAAAAAAKBFJAAAA|347|Highland |Circle|Suite 230|Oak Ridge|Crook County|WY|88371|United States|-7|condo| +38171|AAAAAAAALBFJAAAA|595|Maple Walnut|Avenue|Suite W|White Hall|Charlotte County|VA|26955|United States|-5|condo| +38172|AAAAAAAAMBFJAAAA|157|Broadway Twelfth|RD|Suite 30|Youngstown|Rowan County|KY|40001|United States|-5|single family| +38173|AAAAAAAANBFJAAAA|727|Adams 7th|Street|Suite 480|Newtown|Monroe County|MO|61749|United States|-6|apartment| +38174|AAAAAAAAOBFJAAAA|293|Elevnth Main|Wy|Suite 310|Brownsville|Charlton County|GA|39310|United States|-5|single family| +38175|AAAAAAAAPBFJAAAA|808|Williams Franklin|RD|Suite G|Oakdale|Stewart County|GA|39584|United States|-5|apartment| +38176|AAAAAAAAACFJAAAA|301|3rd |Wy|Suite E|Providence|Spokane County|WA|96614|United States|-8|apartment| +38177|AAAAAAAABCFJAAAA|526|Elm |Boulevard|Suite Y|Mount Zion|Brown County|SD|58054|United States|-6|single family| +38178|AAAAAAAACCFJAAAA|2|Sunset Jefferson|Cir.|Suite 60|Jackson|Wayne County|PA|19583|United States|-5|apartment| +38179|AAAAAAAADCFJAAAA|568|Pine Washington|Wy|Suite W|Gilmore|Thurston County|NE|65464|United States|-7|single family| +38180|AAAAAAAAECFJAAAA|657|5th 1st|Lane|Suite 30|Five Points|Dallas County|AL|36098|United States|-6|single family| +38181|AAAAAAAAFCFJAAAA|573|Cherry |Street|Suite S|Hamilton|Custer County|SD|52808|United States|-6|condo| +38182|AAAAAAAAGCFJAAAA|375|College 2nd|Way|Suite 80|Union|Worth County|MO|68721|United States|-6|apartment| +38183|AAAAAAAAHCFJAAAA|902|Church Park|Ave|Suite 210|Lincoln|Okfuskee County|OK|71289|United States|-6|apartment| +38184|AAAAAAAAICFJAAAA|93|Jefferson First|Ct.|Suite S|Jamestown|Stone County|MS|56867|United States|-6|apartment| +38185|AAAAAAAAJCFJAAAA|794|Maple Forest|ST|Suite 260|Shiloh|Box Elder County|UT|89275|United States|-7|apartment| +38186|AAAAAAAAKCFJAAAA|996|Railroad |Street|Suite 440|Crossroads|Madison County|VA|20534|United States|-5|apartment| +38187|AAAAAAAALCFJAAAA|699|Ninth 8th|Wy|Suite 0|Maple Grove|Chautauqua County|NY|18252|United States|-5|single family| +38188|AAAAAAAAMCFJAAAA|929|Park |Blvd|Suite N|New Hope|Cedar County|IA|59431|United States|-6|apartment| +38189|AAAAAAAANCFJAAAA||||Suite B|||ID|89003|United States|-7|| +38190|AAAAAAAAOCFJAAAA|343|Eigth |Drive|Suite T|Providence|Pendleton County|KY|46614|United States|-5|single family| +38191|AAAAAAAAPCFJAAAA|580|Cherry Oak|Drive|Suite V|Shiloh|Galax city|VA|29275|United States|-5|condo| +38192|AAAAAAAAADFJAAAA|||Ln|||Nuckolls County|||||single family| +38193|AAAAAAAABDFJAAAA|407|6th |RD|Suite 20|Mount Pleasant|Bond County|IL|61933|United States|-6|condo| +38194|AAAAAAAACDFJAAAA|367|Adams Central|Parkway|Suite U|Delmar|Rowan County|NC|23957|United States|-5|condo| +38195|AAAAAAAADDFJAAAA|444|River North|RD|Suite X|Owens|Union County|AR|72324|United States|-6|apartment| +38196|AAAAAAAAEDFJAAAA|446|6th Locust|Wy|Suite 30|Maple Grove|Carroll County|NH|08852|United States|-5|apartment| +38197|AAAAAAAAFDFJAAAA|784|Lakeview |Blvd|Suite 70|Deerfield|Brevard County|FL|39840|United States|-5|single family| +38198|AAAAAAAAGDFJAAAA|623|Second |Drive|Suite 210|New Hope|Taylor County|WI|59431|United States|-6|single family| +38199|AAAAAAAAHDFJAAAA|517|Mill Cedar|Court|Suite V|Oakland|Union County|GA|39843|United States|-5|condo| +38200|AAAAAAAAIDFJAAAA|685|Forest Wilson|Street|Suite S|Woodville|Blair County|PA|14289|United States|-5|condo| +38201|AAAAAAAAJDFJAAAA|473|First |Drive|Suite O|Bethel|Henry County|TN|35281|United States|-5|condo| +38202|AAAAAAAAKDFJAAAA|70|Broadway Sycamore|ST|Suite 190|Unionville|Branch County|MI|41711|United States|-5|apartment| +38203|AAAAAAAALDFJAAAA|898|North |Ct.|Suite 0|Wildwood|Glasscock County|TX|76871|United States|-6|condo| +38204|AAAAAAAAMDFJAAAA|655|Washington |Drive|Suite Y|Lincoln|Bent County|CO|81289|United States|-7|single family| +38205|AAAAAAAANDFJAAAA|254|14th |Avenue|Suite N|Brookville|Murray County|GA|33524|United States|-5|single family| +38206|AAAAAAAAODFJAAAA|757|Johnson View|Avenue|Suite 330|Plainview|Fairfield County|CT|04283|United States|-5|apartment| +38207|AAAAAAAAPDFJAAAA|504|College |Cir.||Springdale|Thomas County|||||condo| +38208|AAAAAAAAAEFJAAAA|68|Lee |Ln|Suite 360|Oakland|Harper County|OK|79843|United States|-6|single family| +38209|AAAAAAAABEFJAAAA|742|Woodland |Way|Suite 350|Antioch|Spink County|SD|58605|United States|-7|apartment| +38210|AAAAAAAACEFJAAAA|139|Smith Sixth|Wy|Suite 280|Kingston|Boundary County|ID|84975|United States|-7|single family| +38211|AAAAAAAADEFJAAAA|661|Woodland |ST|Suite V|New Hope|Castro County|TX|79431|United States|-6|single family| +38212|AAAAAAAAEEFJAAAA||||Suite B||Victoria County|TX|71521|||| +38213|AAAAAAAAFEFJAAAA|423|12th |Street|Suite 60|Bethel|Mille Lacs County|MN|55281|United States|-6|single family| +38214|AAAAAAAAGEFJAAAA|987|Forest Spring|Ct.|Suite 110|Valley View|Custer County|NE|65124|United States|-6|apartment| +38215|AAAAAAAAHEFJAAAA|485|Oak |Blvd|Suite 230|Midway|Wayne County|NY|11904|United States|-5|condo| +38216|AAAAAAAAIEFJAAAA|882|View |RD|Suite G|Antioch|Whiteside County|IL|68605|United States|-6|apartment| +38217|AAAAAAAAJEFJAAAA|87|Forest |Ln|Suite B|Waterloo|Cass County|IN|41675|United States|-5|apartment| +38218|AAAAAAAAKEFJAAAA|762|Wilson |ST|Suite 240|Jamestown|Lincoln County|ID|86867|United States|-7|single family| +38219|AAAAAAAALEFJAAAA|323|Poplar |Ave|Suite K|Pleasant Valley|Scott County|IN|42477|United States|-5|single family| +38220|AAAAAAAAMEFJAAAA|973|Park North|Street|Suite 210|Red Hill|Preston County|WV|24338|United States|-5|condo| +38221|AAAAAAAANEFJAAAA|415|Williams |Road|Suite U|Pinhook|Cheyenne County|KS|69398|United States|-6|condo| +38222|AAAAAAAAOEFJAAAA|746|Adams |Drive|Suite 40|Bridgeport|Lake County|TN|35817|United States|-6|condo| +38223|AAAAAAAAPEFJAAAA|677|Birch 3rd|Ct.|Suite 240|Maple Grove|Saline County|AR|78252|United States|-6|condo| +38224|AAAAAAAAAFFJAAAA|788|10th East|Avenue|Suite 260|Salem|Long County|GA|38048|United States|-5|condo| +38225|AAAAAAAABFFJAAAA|562|Miller |Parkway|Suite 70|Pomona|Harrison County|MO|64153|United States|-6|condo| +38226|AAAAAAAACFFJAAAA|611|Forest |Ct.|Suite R|Wilson|Chesterfield County|SC|26971|United States|-5|condo| +38227|AAAAAAAADFFJAAAA|415|Lincoln |Ct.|Suite 460|Georgetown|Campbell County|SD|57057|United States|-6|single family| +38228|AAAAAAAAEFFJAAAA|59|Sixth |Drive|Suite I|Midway|Boundary County|ID|81904|United States|-7|apartment| +38229|AAAAAAAAFFFJAAAA|848|Broadway Lake|Way|Suite 30|Clinton|Crawford County|AR|78222|United States|-6|single family| +38230|AAAAAAAAGFFJAAAA|870|Locust Sunset|Court|Suite O|Avery|Morrill County|NE|60194|United States|-7|single family| +38231|AAAAAAAAHFFJAAAA|811|15th Dogwood|Lane|Suite C|Plainview|Winnebago County|WI|53683|United States|-6|condo| +38232|AAAAAAAAIFFJAAAA|935|15th Jackson|Parkway|Suite Q|Spring Valley|Oregon County|MO|66060|United States|-6|single family| +38233|AAAAAAAAJFFJAAAA|80|Sunset First|Circle|Suite 20|Springfield|George County|MS|59303|United States|-6|apartment| +38234|AAAAAAAAKFFJAAAA|392|Walnut Spring|Circle|Suite Q|Marion|King County|TX|70399|United States|-6|apartment| +38235|AAAAAAAALFFJAAAA|961|4th |Boulevard|Suite 270|Glendale|Christian County|KY|43951|United States|-6|apartment| +38236|AAAAAAAAMFFJAAAA|884|Lake |Wy|Suite 480|White Oak|Jones County|GA|36668|United States|-5|apartment| +38237|AAAAAAAANFFJAAAA|358|Spruce Pine|Blvd|Suite Q|Harmony|Cedar County|NE|65804|United States|-6|condo| +38238|AAAAAAAAOFFJAAAA|441|Woodland Pine|Way|Suite H|Green Acres|Sioux County|NE|67683|United States|-7|single family| +38239|AAAAAAAAPFFJAAAA|311|8th 6th|Ct.|Suite I|Elizabeth|Big Stone County|MN|52935|United States|-6|condo| +38240|AAAAAAAAAGFJAAAA|549|First Williams|Pkwy|Suite 280|Highland|McMullen County|TX|79454|United States|-6|apartment| +38241|AAAAAAAABGFJAAAA|451|Franklin Williams|Blvd|Suite 410|Mountain View|Anderson County|KS|64466|United States|-6|single family| +38242|AAAAAAAACGFJAAAA|472|Ash 12th|RD|Suite 450|Riverview|Stoddard County|MO|69003|United States|-6|single family| +38243|AAAAAAAADGFJAAAA|397|Park Willow|Avenue|Suite 140|Lakeview|Bee County|TX|78579|United States|-6|apartment| +38244|AAAAAAAAEGFJAAAA|526|1st |RD|Suite 230|Hamilton|Borden County|TX|72808|United States|-6|condo| +38245|AAAAAAAAFGFJAAAA|655|Main |Court|Suite 170|Fairview|Washington County|AL|35709|United States|-6|condo| +38246|AAAAAAAAGGFJAAAA|452|Central |Road|Suite 390|Newport|Gosper County|NE|61521|United States|-6|single family| +38247|AAAAAAAAHGFJAAAA||Adams |RD|Suite 70||Elbert County||86871|United States||condo| +38248|AAAAAAAAIGFJAAAA|766|4th |Road|Suite 150|Five Points|Frio County|TX|76098|United States|-6|single family| +38249|AAAAAAAAJGFJAAAA|225|8th |Court|Suite A|Unionville|Lawrence County|KY|41711|United States|-5|apartment| +38250|AAAAAAAAKGFJAAAA|285|10th 6th|Street|Suite T|Buena Vista|Jackson County|SD|55752|United States|-7|single family| +38251|AAAAAAAALGFJAAAA|113|Walnut |Cir.|Suite I|Stringtown|Kossuth County|IA|50162|United States|-6|single family| +38252|AAAAAAAAMGFJAAAA|46|Maple North|Way|Suite 490|Valley View|Monroe County|FL|35124|United States|-5|condo| +38253|AAAAAAAANGFJAAAA|498|East Mill|Ct.|Suite D|Five Points|Linn County|MO|66098|United States|-6|condo| +38254|AAAAAAAAOGFJAAAA|236|12th Park|Drive|Suite 200|Caledonia|Van Buren County|MI|47411|United States|-5|condo| +38255|AAAAAAAAPGFJAAAA|505|Central 9th|Drive|Suite 430|Riverside|||59231|United States|-6|| +38256|AAAAAAAAAHFJAAAA|||||Hamilton||TN|32808|United States||apartment| +38257|AAAAAAAABHFJAAAA|112|Spruce West|Wy|Suite 310|Deerfield|Prince George County|VA|29840|United States|-5|single family| +38258|AAAAAAAACHFJAAAA|147|Lincoln Smith|Lane|Suite X|Woodland|Natchitoches Parish|LA|74854|United States|-6|apartment| +38259|AAAAAAAADHFJAAAA|167|Wilson |Way|Suite L|Jamestown|McDonough County|IL|66867|United States|-6|single family| +38260|AAAAAAAAEHFJAAAA|70|Elm 6th|Boulevard|Suite 190|Unionville|Wood County|OH|41711|United States|-5|apartment| +38261|AAAAAAAAFHFJAAAA|309|Valley |Ln|Suite 80|Walnut Grove|Crawford County|AR|77752|United States|-6|single family| +38262|AAAAAAAAGHFJAAAA|151|East |Blvd|Suite 160|Fisher|Guthrie County|IA|52819|United States|-6|condo| +38263|AAAAAAAAHHFJAAAA|467|Walnut Sunset|Ct.|Suite V|Lakeside|Lawrence County|AR|79532|United States|-6|condo| +38264|AAAAAAAAIHFJAAAA|637|Cedar Washington|Boulevard|Suite J|Union Hill|Holmes County|MS|57746|United States|-6|apartment| +38265|AAAAAAAAJHFJAAAA|80|Adams |Ave|Suite M|Ludlow|Gallatin County|KY|45566|United States|-6|condo| +38266|AAAAAAAAKHFJAAAA|279|Oak Locust|Parkway|Suite 420|Pine Grove|Pike County|MO|64593|United States|-6|single family| +38267|AAAAAAAALHFJAAAA|83|Spring |ST|Suite K|Waterloo|Ashe County|NC|21675|United States|-5|condo| +38268|AAAAAAAAMHFJAAAA|366|Park |Street|Suite 420|Red Hill|Laclede County|MO|64338|United States|-6|single family| +38269|AAAAAAAANHFJAAAA|833|Walnut |Way|Suite H|Highland Park|Stone County|MS|56534|United States|-6|condo| +38270|AAAAAAAAOHFJAAAA|514|8th |Pkwy|Suite 170|Oak Ridge|Taliaferro County|GA|38371|United States|-5|condo| +38271|AAAAAAAAPHFJAAAA|797|Oak Ridge|Circle|Suite 150|Galena|Lipscomb County|TX|74369|United States|-6|condo| +38272|AAAAAAAAAIFJAAAA|437|First Davis|Drive|Suite 440|Hopewell|Lassen County|CA|90587|United States|-8|condo| +38273|AAAAAAAABIFJAAAA|17|||Suite 340|||||||| +38274|AAAAAAAACIFJAAAA|477|Central |Ave|Suite 250|Buena Vista|Stark County|ND|55752|United States|-6|single family| +38275|AAAAAAAADIFJAAAA|474|Chestnut Lee|Ct.|Suite 100|Spring Hill|Sagadahoc County|ME|07387|United States|-5|condo| +38276|AAAAAAAAEIFJAAAA|800|Adams |ST|Suite 400|Milan|Rockingham County|NH|07297|United States|-5|condo| +38277|AAAAAAAAFIFJAAAA|640|Green Smith|Ln|Suite 220|Pierce|Essex County|VA|23360|United States|-5|apartment| +38278|AAAAAAAAGIFJAAAA|152|6th Second|Parkway|Suite 40|Stringtown|Tuscola County|MI|40162|United States|-5|condo| +38279|AAAAAAAAHIFJAAAA|228|First Maple|Parkway|Suite 200|New Hope|Koochiching County|MN|59431|United States|-6|condo| +38280|AAAAAAAAIIFJAAAA|160|Washington 8th|Avenue|Suite 170|Woodland|Sequatchie County|TN|34854|United States|-6|single family| +38281|AAAAAAAAJIFJAAAA|671|Jackson Spruce|Boulevard|Suite Q|Bunker Hill|Tift County|GA|30150|United States|-5|single family| +38282|AAAAAAAAKIFJAAAA|318|Cedar |Blvd|Suite 0|Lakewood|Finney County|KS|68877|United States|-6|apartment| +38283|AAAAAAAALIFJAAAA|319|13th Sycamore|Pkwy|Suite 90|Hillcrest|Hoke County|NC|23003|United States|-5|single family| +38284|AAAAAAAAMIFJAAAA|704|East Smith|Avenue|Suite 450|Pleasant Valley|Strafford County|NH|03077|United States|-5|condo| +38285|AAAAAAAANIFJAAAA|434|Jefferson |Road|Suite T|Edgewood|Washington County|MS|50069|United States|-6|apartment| +38286|AAAAAAAAOIFJAAAA|950|1st 13th|Ave|Suite 110|Lakeview|Broadwater County|MT|68579|United States|-7|condo| +38287|AAAAAAAAPIFJAAAA|298|3rd |Parkway|Suite 30|Fairfield|Harris County|TX|76192|United States|-6|condo| +38288|AAAAAAAAAJFJAAAA|493|Main |Ct.|Suite J|Greenfield|Grant County|IN|45038|United States|-5|condo| +38289|AAAAAAAABJFJAAAA|917|Cherry |ST|Suite 340|Crossroads|Petersburg city|VA|20534|United States|-5|condo| +38290|AAAAAAAACJFJAAAA|655|Valley |Dr.|Suite I|Friendship|Madison County|MT|64536|United States|-7|single family| +38291|AAAAAAAADJFJAAAA|274|Cherry |Lane|Suite T|Summit|||50499|United States|-6|single family| +38292|AAAAAAAAEJFJAAAA|871|Highland Cedar|Circle|Suite H|Waterloo|Darke County|OH|41675|United States|-5|single family| +38293|AAAAAAAAFJFJAAAA|741|Sunset |Drive|Suite T|Fayetteville|DeSoto County|FL|31732|United States|-5|single family| +38294|AAAAAAAAGJFJAAAA|237|9th |ST|Suite J|Plainview|Wood County|WV|23683|United States|-5|apartment| +38295|AAAAAAAAHJFJAAAA|193|Fourth |Pkwy|Suite 100|Clifton|Callaway County|MO|68014|United States|-6|condo| +38296|AAAAAAAAIJFJAAAA|296|Walnut Valley|Circle|Suite R|Pleasant Hill|Greene County|MS|53604|United States|-6|condo| +38297|AAAAAAAAJJFJAAAA|358|Elm |Ln|Suite L|Five Points|Ellis County|TX|76098|United States|-6|condo| +38298|AAAAAAAAKJFJAAAA|497|Hillcrest 15th|Ln|Suite W|Newtown|Champaign County|OH|41749|United States|-5|condo| +38299|AAAAAAAALJFJAAAA|288|Maple Dogwood|Avenue|Suite L|Oakwood|Webster Parish|LA|70169|United States|-6|condo| +38300|AAAAAAAAMJFJAAAA|66|Meadow Park|Parkway|Suite Y|Riverside|Mitchell County|NC|29231|United States|-5|single family| +38301|AAAAAAAANJFJAAAA|847|Park |Court|Suite W|Freeport|Natrona County|WY|81844|United States|-7|apartment| +38302|AAAAAAAAOJFJAAAA|549|North |Blvd|Suite W|Spring Valley|Mason County|MI|46060|United States|-5|single family| +38303|AAAAAAAAPJFJAAAA|366|Jefferson East|Way|Suite I|Clifton|Campbell County|TN|38014|United States|-5|condo| +38304|AAAAAAAAAKFJAAAA|940|South |Circle|Suite 190|Belmont|Cooper County|MO|60191|United States|-6|condo| +38305|AAAAAAAABKFJAAAA|955|11th 8th|Ln|Suite 120|Allison|Montgomery County|GA|34167|United States|-5|single family| +38306|AAAAAAAACKFJAAAA|299|Mill |Court|Suite N|Georgetown|Trempealeau County|WI|57057|United States|-6|condo| +38307|AAAAAAAADKFJAAAA|75|Elm |Blvd|Suite D|Unionville|Union County|IA|51711|United States|-6|apartment| +38308|AAAAAAAAEKFJAAAA|821|Railroad Valley|Parkway|Suite T|Plainview|Windham County|VT|04283|United States|-5|condo| +38309|AAAAAAAAFKFJAAAA|797|Cherry |Blvd|Suite X|Greenwood|Carlton County|MN|58828|United States|-6|condo| +38310|AAAAAAAAGKFJAAAA|959|Locust Hillcrest|Ln|Suite 340|Franklin|Sandoval County|NM|89101|United States|-7|condo| +38311|AAAAAAAAHKFJAAAA|143|West |RD|Suite 40|Red Bank|Blanco County|TX|74975|United States|-6|condo| +38312|AAAAAAAAIKFJAAAA|93|Highland Second|Ave|Suite 350|Shady Grove|Pottawatomie County|OK|72812|United States|-6|apartment| +38313|AAAAAAAAJKFJAAAA|851|Fifth Smith|Pkwy|Suite O|Greenfield|Lexington city|VA|25038|United States|-5|single family| +38314|AAAAAAAAKKFJAAAA|974|6th Lee|Cir.|Suite S|Cedar Grove|Big Horn County|WY|80411|United States|-7|condo| +38315|AAAAAAAALKFJAAAA|||Boulevard|Suite 320|Lakewood||OK|78877||-6|apartment| +38316|AAAAAAAAMKFJAAAA|602|Main |Street|Suite R|Riverdale|Madison County|MO|69391|United States|-6|single family| +38317|AAAAAAAANKFJAAAA|463|Washington |Ct.|Suite 180|Mount Zion|East Baton Rouge Parish|LA|78054|United States|-6|apartment| +38318|AAAAAAAAOKFJAAAA|222|Central |Circle|Suite F|Maple Grove|Carroll County|OH|48252|United States|-5|condo| +38319|AAAAAAAAPKFJAAAA||Maple |Ln|Suite 400|Newport|Schuylkill County|PA||United States|-5|| +38320|AAAAAAAAALFJAAAA|210|Pine Walnut|Street|Suite 70|Highland Park|Santa Cruz County|CA|96534|United States|-8|apartment| +38321|AAAAAAAABLFJAAAA|708|Lincoln |Pkwy|Suite L|Siloam|Craig County|OK|78948|United States|-6|single family| +38322|AAAAAAAACLFJAAAA|284|Thirteenth Pine|Dr.|Suite H|Red Hill|San Joaquin County|CA|94338|United States|-8|apartment| +38323|AAAAAAAADLFJAAAA|651|Pine |Avenue|Suite B|Walnut Grove|Lancaster County|SC|27752|United States|-5|condo| +38324|AAAAAAAAELFJAAAA|35|Oak |Avenue|Suite V|Woodland|Pope County|AR|74854|United States|-6|apartment| +38325|AAAAAAAAFLFJAAAA|241|Poplar |Road|Suite 440|Whitesville|Lawrence County|IN|45903|United States|-5|apartment| +38326|AAAAAAAAGLFJAAAA|300|Center 3rd|Blvd|Suite L|Plainview|Franklin County|MS|53683|United States|-6|apartment| +38327|AAAAAAAAHLFJAAAA|452|6th |Court|Suite 0|Delmar|Wilkes County|NC|23957|United States|-5|single family| +38328|AAAAAAAAILFJAAAA|83|Spruce |Street|Suite Y|Lakeside|Teton County|MT|69532|United States|-7|condo| +38329|AAAAAAAAJLFJAAAA|734|Poplar |Boulevard|Suite 10|Greenville|Jasper County|IN|41387|United States|-5|condo| +38330|AAAAAAAAKLFJAAAA|735|10th 7th|RD|Suite 270|Oakdale|Lincoln County|TN|39584|United States|-6|condo| +38331|AAAAAAAALLFJAAAA|211|Cherry West|Ct.|Suite 0|Maple Grove|Chouteau County|MT|68252|United States|-7|condo| +38332|AAAAAAAAMLFJAAAA|529|||Suite G|Woodland|White Pine County|||United States|-8|single family| +38333|AAAAAAAANLFJAAAA|515|14th Cedar|Way|Suite 240|Oneida|Scurry County|TX|74027|United States|-6|condo| +38334|AAAAAAAAOLFJAAAA|262|||Suite G||Roane County|||||condo| +38335|AAAAAAAAPLFJAAAA|956|Green |Ave|Suite 190|Maple Grove|Randall County|TX|78252|United States|-6|single family| +38336|AAAAAAAAAMFJAAAA|963|Jefferson |Cir.|Suite 170|Wilson|Henry County|TN|36971|United States|-5|single family| +38337|AAAAAAAABMFJAAAA|273|3rd 5th|Dr.|Suite 490|Pleasant Grove|Coweta County|GA|34136|United States|-5|condo| +38338|AAAAAAAACMFJAAAA|73|4th Lakeview|Circle|Suite 20|Providence|Harding County|SD|56614|United States|-7|condo| +38339|AAAAAAAADMFJAAAA|968|Lake 5th|Wy|Suite Y|Williams|Dauphin County|PA|17564|United States|-5|single family| +38340|AAAAAAAAEMFJAAAA|179|Sunset Woodland|Ln|Suite T|Jamestown|Moniteau County|MO|66867|United States|-6|condo| +38341|AAAAAAAAFMFJAAAA|991|Main South|Avenue|Suite 220|Jenkins|Rockingham County|NH|07892|United States|-5|condo| +38342|AAAAAAAAGMFJAAAA|583|5th |Wy|Suite 470|Spring Valley|Claiborne County|MS|56060|United States|-6|single family| +38343|AAAAAAAAHMFJAAAA|23|West |Street|Suite B|Centerville|Grady County|GA|30059|United States|-5|condo| +38344|AAAAAAAAIMFJAAAA|468|Chestnut 1st|Street|Suite 450|Forest Hills|Hickory County|MO|69237|United States|-6|apartment| +38345|AAAAAAAAJMFJAAAA|304|South |ST|Suite G|Riverview|Montcalm County|MI|49003|United States|-5|single family| +38346|AAAAAAAAKMFJAAAA|161|Cherry |Road|Suite 230|Fox|Marion County|SC|20631|United States|-5|condo| +38347|AAAAAAAALMFJAAAA|866|Hill |Wy|Suite 160|Maple Grove|Warren County|OH|48252|United States|-5|apartment| +38348|AAAAAAAAMMFJAAAA|436|Eigth Oak|Parkway|Suite 320|Littleton|Wichita County|TX|76074|United States|-6|condo| +38349|AAAAAAAANMFJAAAA|308|Woodland |ST|Suite S|Cedar Grove|Gem County|ID|80411|United States|-7|single family| +38350|AAAAAAAAOMFJAAAA|415|Cherry |Drive|Suite G|Unionville|Nash County|NC|21711|United States|-5|condo| +38351|AAAAAAAAPMFJAAAA|620|North |Ave|Suite 150|Deerfield|Alpena County|MI|49840|United States|-5|condo| +38352|AAAAAAAAANFJAAAA|268|Highland |Avenue|Suite 290|Oakland|Marion County|OH|49843|United States|-5|single family| +38353|AAAAAAAABNFJAAAA|201|Meadow |Street|Suite 140|Farmington|Grays Harbor County|WA|99145|United States|-8|condo| +38354|AAAAAAAACNFJAAAA|906|Second |Court|Suite 30|Gravel Hill|Wadena County|MN|51944|United States|-6|apartment| +38355|AAAAAAAADNFJAAAA|927|Woodland |Street|Suite L|Union|Mitchell County|GA|38721|United States|-5|apartment| +38356|AAAAAAAAENFJAAAA|928|Fourth |Lane|Suite 380|Pierce|Van Buren County|TN|33360|United States|-6|single family| +38357|AAAAAAAAFNFJAAAA|751|Oak |Blvd|Suite E|Macedonia|Manistee County|MI|41087|United States|-5|single family| +38358|AAAAAAAAGNFJAAAA|395|Lakeview |Circle|Suite K|Riverside|Dixon County|NE|69231|United States|-6|apartment| +38359|AAAAAAAAHNFJAAAA|641|North 3rd|Ave|Suite 170|Oakdale|Hyde County|SD|59584|United States|-7|apartment| +38360|AAAAAAAAINFJAAAA|35|Elm |Road|Suite B|Oak Grove|Shiawassee County|MI|48370|United States|-5|single family| +38361|AAAAAAAAJNFJAAAA|409|Second |Boulevard|Suite P|Crossroads|Jackson County|IN|40534|United States|-5|single family| +38362|AAAAAAAAKNFJAAAA|590||Lane|||Gilmer County||39840|United States||condo| +38363|AAAAAAAALNFJAAAA|67|Hill 14th|Ln|Suite 160|Midway|Blackford County|IN|41904|United States|-5|condo| +38364|AAAAAAAAMNFJAAAA|332|5th |Court|Suite K|Shady Grove|Pierce County|WA|92812|United States|-8|apartment| +38365|AAAAAAAANNFJAAAA|952|Hillcrest Cedar|Parkway|Suite S|Greenville|Floyd County|VA|21387|United States|-5|apartment| +38366|AAAAAAAAONFJAAAA|902|Smith 6th|Road|Suite 80|Salem|Perkins County|SD|58048|United States|-7|apartment| +38367|AAAAAAAAPNFJAAAA|64|Park |Boulevard|Suite H|Forest Hills|Russell County|KY|49237|United States|-5|condo| +38368|AAAAAAAAAOFJAAAA|811|Walnut |Cir.|Suite 30|Florence|Noxubee County|MS|53394|United States|-6|single family| +38369|AAAAAAAABOFJAAAA|166||Ave|||Wheeler County||79920||-6|apartment| +38370|AAAAAAAACOFJAAAA|600|6th |Avenue|Suite 100|Sulphur Springs|Dixie County|FL|38354|United States|-5|apartment| +38371|AAAAAAAADOFJAAAA|408|Chestnut |Cir.|Suite P|Littleton|Lake County|CO|86074|United States|-7|apartment| +38372|AAAAAAAAEOFJAAAA|630|10th |Ct.|Suite 330|Salem|George County|MS|58048|United States|-6|apartment| +38373|AAAAAAAAFOFJAAAA|291|Main |Ln|Suite F|Florence|Yell County|AR|73394|United States|-6|single family| +38374|AAAAAAAAGOFJAAAA|101|Adams |Avenue|Suite 100|Waterloo|Logan County|ND|51675|United States|-6|single family| +38375|AAAAAAAAHOFJAAAA|701|6th Locust|Blvd|Suite H|Edgewood|Kandiyohi County|MN|50069|United States|-6|condo| +38376|AAAAAAAAIOFJAAAA|978|Jackson |Cir.|Suite V|Five Forks|Benton County|MS|52293|United States|-6|condo| +38377|AAAAAAAAJOFJAAAA|342|Main |Lane|Suite 190|Kingston|Wood County|WI|54975|United States|-6|condo| +38378|AAAAAAAAKOFJAAAA|692|River |Cir.|Suite D|Youngstown|Steuben County|IN|40001|United States|-5|single family| +38379|AAAAAAAALOFJAAAA|766|7th Eigth|Pkwy|Suite 60|Highland|Pickens County|GA|39454|United States|-5|condo| +38380|AAAAAAAAMOFJAAAA|68|3rd |Avenue|Suite N|Glenwood|York County|NE|63511|United States|-6|single family| +38381|AAAAAAAANOFJAAAA|521|Sunset Cedar|Way|Suite Q|Bethel|Holt County|NE|65281|United States|-7|condo| +38382|AAAAAAAAOOFJAAAA|399|Wilson |RD|Suite 330|Red Hill|Trousdale County|TN|34338|United States|-6|apartment| +38383|AAAAAAAAPOFJAAAA|349|Cherry |Avenue|Suite 60|Enterprise|Harrison County|KY|41757|United States|-6|condo| +38384|AAAAAAAAAPFJAAAA|449|1st Sixth|Circle|Suite F|Highland Park|Boulder County|CO|86534|United States|-7|single family| +38385|AAAAAAAABPFJAAAA|985|Sycamore Pine|Dr.|Suite 40|Decatur|Edgar County|IL|67932|United States|-6|condo| +38386|AAAAAAAACPFJAAAA|741|Jackson Chestnut|Avenue|Suite 450|Five Forks|Gadsden County|FL|32293|United States|-5|single family| +38387|AAAAAAAADPFJAAAA|23|Adams Birch|Parkway|Suite 150|Oakwood|Bath County|KY|40169|United States|-6|apartment| +38388|AAAAAAAAEPFJAAAA|779|Lincoln |Parkway|Suite R|Avoca|Geauga County|OH|40540|United States|-5|apartment| +38389|AAAAAAAAFPFJAAAA|400|Smith |Boulevard|Suite R|Hopewell|Lake County|IN|40587|United States|-5|condo| +38390|AAAAAAAAGPFJAAAA|577|Locust Lincoln|Way|Suite U|Riverview|Bingham County|ID|89003|United States|-7|condo| +38391|AAAAAAAAHPFJAAAA|785|Adams 15th|Cir.|Suite T|Franklin|Gregory County|SD|59101|United States|-6|apartment| +38392|AAAAAAAAIPFJAAAA|380|6th |Road|Suite 470|Philadelphia|Baltimore County|MD|25591|United States|-5|condo| +38393|AAAAAAAAJPFJAAAA|739|Walnut |Parkway|Suite O|Shiloh|Cameron County|TX|79275|United States|-6|single family| +38394|AAAAAAAAKPFJAAAA|815|East 5th|Street|Suite L|Bunker Hill|Golden Valley County|ND|50150|United States|-6|condo| +38395|AAAAAAAALPFJAAAA|802|Washington |Court|Suite 310|Shiloh|Hawaii County|HI|99275|United States|-10|single family| +38396|AAAAAAAAMPFJAAAA|689|Washington |Drive|Suite 330|Hamilton|Wagoner County|OK|72808|United States|-6|condo| +38397|AAAAAAAANPFJAAAA|385|View Miller|Street|Suite G|Riverdale|Cheyenne County|CO|89391|United States|-7|apartment| +38398|AAAAAAAAOPFJAAAA|279|Wilson |Circle|Suite 360|Liberty|Douglas County|SD|53451|United States|-6|condo| +38399|AAAAAAAAPPFJAAAA|910|Walnut |Circle|Suite L|Pleasant Grove|Habersham County|GA|34136|United States|-5|single family| +38400|AAAAAAAAAAGJAAAA|401|Spring |ST|Suite I|Cedar Grove|Powder River County|MT|60411|United States|-7|single family| +38401|AAAAAAAABAGJAAAA|578|11th Forest|Dr.|Suite 210|Whitney|Madison County|MS|58339|United States|-6|condo| +38402|AAAAAAAACAGJAAAA|780|First |Circle|Suite J|Georgetown|Conway County|AR|77057|United States|-6|apartment| +38403|AAAAAAAADAGJAAAA|768|Hill Hillcrest|Street|Suite U|Friendship|Kay County|OK|74536|United States|-6|single family| +38404|AAAAAAAAEAGJAAAA|346|Broadway |RD|Suite 240|Washington Heights|Chase County|NE|68167|United States|-6|condo| +38405|AAAAAAAAFAGJAAAA|649|Birch Washington|Avenue|Suite N|Pleasant Valley|Marshall County|WV|22477|United States|-5|single family| +38406|AAAAAAAAGAGJAAAA|268|Hill Church|Pkwy|Suite E|Woodland|Goochland County|VA|24854|United States|-5|condo| +38407|AAAAAAAAHAGJAAAA|463|Maple |Dr.|Suite G|Wildwood|Keweenaw County|MI|46871|United States|-5|condo| +38408|AAAAAAAAIAGJAAAA|900|Jackson |Dr.|Suite 410|Cherry Valley|Fremont County|ID|80854|United States|-7|apartment| +38409|AAAAAAAAJAGJAAAA|624|Ninth |Cir.|Suite T|Cedar Grove|Perry County|KY|40411|United States|-5|single family| +38410|AAAAAAAAKAGJAAAA|759|View |Dr.|Suite 180|Oakdale|Summit County|OH|49584|United States|-5|condo| +38411|AAAAAAAALAGJAAAA|374|Birch |Drive|Suite K|Franklin|McKean County|PA|19101|United States|-5|apartment| +38412|AAAAAAAAMAGJAAAA|892|Lakeview 8th|RD|Suite J|Monroe|Camden County|NJ|04584|United States|-5|apartment| +38413|AAAAAAAANAGJAAAA|823|Willow Park||Suite 280|||IL|||-6|apartment| +38414|AAAAAAAAOAGJAAAA|412|Park Main|Way|Suite 70|Edgewood|Horry County|SC|20069|United States|-5|single family| +38415|AAAAAAAAPAGJAAAA|339|Park |Pkwy|Suite 330|Woodlawn|Traverse County|MN|54098|United States|-6|condo| +38416|AAAAAAAAABGJAAAA|387|Church Sunset|Lane|Suite 20|Springdale|Quay County|NM|88883|United States|-7|single family| +38417|AAAAAAAABBGJAAAA|146|Green Mill|Parkway|Suite J|Oakdale|Clinton County|IL|69584|United States|-6|apartment| +38418|AAAAAAAACBGJAAAA|961|Highland |Street|Suite K|Highland Park|Marquette County|WI|56534|United States|-6|condo| +38419|AAAAAAAADBGJAAAA|761|8th 2nd|Boulevard|Suite 130|Yorktown|Esmeralda County|NV|80732|United States|-8|single family| +38420|AAAAAAAAEBGJAAAA|251|Wilson |Circle|Suite 230|White Oak|Telfair County|GA|36668|United States|-5|apartment| +38421|AAAAAAAAFBGJAAAA|135|Walnut |Circle|Suite 380|Providence|Wayne County|NE|66614|United States|-7|apartment| +38422|AAAAAAAAGBGJAAAA|912|Third Center|Court|Suite S|Valley View|Goodhue County|MN|55124|United States|-6|condo| +38423|AAAAAAAAHBGJAAAA|817|Ridge |Street|Suite F|Shiloh|Ellis County|TX|79275|United States|-6|apartment| +38424|AAAAAAAAIBGJAAAA|918|Lakeview |Court|Suite 170|White Oak|Pottawatomie County|OK|76668|United States|-6|apartment| +38425|AAAAAAAAJBGJAAAA|58|Third Oak|Street|Suite X|Pinecrest|Menifee County|KY|49981|United States|-5|apartment| +38426|AAAAAAAAKBGJAAAA|155|Oak |ST|Suite X|Macedonia|Seneca County|OH|41087|United States|-5|single family| +38427|AAAAAAAALBGJAAAA|463|Sunset Fourth|Dr.|Suite 280|Wilson|Blount County|TN|36971|United States|-5|single family| +38428|AAAAAAAAMBGJAAAA|257|Locust Railroad|Blvd|Suite 440|Union Hill|Stevens County|MN|57746|United States|-6|single family| +38429|AAAAAAAANBGJAAAA|626|Spring |Parkway|Suite 420|Argyle|Cook County|MN|58722|United States|-6|condo| +38430|AAAAAAAAOBGJAAAA|300|Sixth |Ct.|Suite 170|Garrison|Wright County|MO|68767|United States|-6|single family| +38431|AAAAAAAAPBGJAAAA|844|5th Poplar|Pkwy|Suite I|Hillcrest|Sequoyah County|OK|73003|United States|-6|condo| +38432|AAAAAAAAACGJAAAA|896|Walnut |Parkway|Suite N|Ellsworth|Henry County|MO|65079|United States|-6|apartment| +38433|AAAAAAAABCGJAAAA|626|Ridge |Drive|Suite I|Jamestown|Covington County|AL|36867|United States|-6|single family| +38434|AAAAAAAACCGJAAAA|89|Center |Ct.|Suite 240|Mount Olive|Amherst County|VA|28059|United States|-5|apartment| +38435|AAAAAAAADCGJAAAA|407|7th |Court|Suite Y|Belmont|Marion County|AR|70191|United States|-6|apartment| +38436|AAAAAAAAECGJAAAA|507|Park |Ln|Suite I|Lakeside|Adair County|IA|59532|United States|-6|apartment| +38437|AAAAAAAAFCGJAAAA|238|Ridge |Circle|Suite D|Bunker Hill|Wayne County|NC|20150|United States|-5|single family| +38438|AAAAAAAAGCGJAAAA|761|7th Cedar|ST|Suite H|New Hope|McDowell County|NC|29431|United States|-5|apartment| +38439|AAAAAAAAHCGJAAAA|337|Park |Ln|Suite G|Waterloo|Washington County|ME|02275|United States|-5|condo| +38440|AAAAAAAAICGJAAAA|206|Second |Lane|Suite B|Liberty|Clark County|MO|63451|United States|-6|condo| +38441|AAAAAAAAJCGJAAAA|242|3rd |Ln|Suite E|Antioch|Union County|OH|48605|United States|-5|condo| +38442|AAAAAAAAKCGJAAAA|641|Madison |Wy|Suite 430|Kingston|Flagler County|FL|34975|United States|-5|apartment| +38443|AAAAAAAALCGJAAAA|172|Pine |Way|Suite J|Greenfield|Lackawanna County|PA|15038|United States|-5|condo| +38444|AAAAAAAAMCGJAAAA|980|West 1st|Court|Suite 410|Oakwood|Randall County|TX|70169|United States|-6|apartment| +38445|AAAAAAAANCGJAAAA|524|Poplar |Ct.|Suite K|Oak Hill|Cherokee County|NC|27838|United States|-5|condo| +38446|AAAAAAAAOCGJAAAA|176|Lake |Blvd|Suite A|Midway|Marshall County|AL|31904|United States|-6|apartment| +38447|AAAAAAAAPCGJAAAA|565|Park |Boulevard|Suite 50|Wildwood|Cook County|IL|66871|United States|-6|single family| +38448|AAAAAAAAADGJAAAA|382|Fifth |Dr.|Suite 420|Wildwood|Bannock County|ID|86871|United States|-7|apartment| +38449|AAAAAAAABDGJAAAA||Cedar Ridge|||Walnut Grove|Auglaize County|||United States||single family| +38450|AAAAAAAACDGJAAAA|395|Ridge |Blvd|Suite 200|Providence|Norfolk County|MA|07214|United States|-5|single family| +38451|AAAAAAAADDGJAAAA|344|Second |Ave|Suite 390|Mount Pleasant|Cerro Gordo County|IA|51933|United States|-6|apartment| +38452|AAAAAAAAEDGJAAAA|621|Elm |Ave|Suite Q|Sulphur Springs|Hickman County|KY|48354|United States|-6|condo| +38453|AAAAAAAAFDGJAAAA|320|Fifth 6th|Lane|Suite D|Lakewood|Cocke County|TN|38877|United States|-5|condo| +38454|AAAAAAAAGDGJAAAA|212|College 4th|Parkway|Suite 20|Brownsville|Union County|TN|39310|United States|-6|apartment| +38455|AAAAAAAAHDGJAAAA|343|View |Court|Suite 360|Newport|Williams County|ND|51521|United States|-6|condo| +38456|AAAAAAAAIDGJAAAA|76|Lincoln |Circle|Suite 390|Sunnyside|Vernon County|MO|61952|United States|-6|condo| +38457|AAAAAAAAJDGJAAAA|822|4th Park|ST|Suite S|Riverside|Barry County|MO|69231|United States|-6|apartment| +38458|AAAAAAAAKDGJAAAA|419|Railroad |Wy|Suite D|Forest Hills|Crockett County|TX|79237|United States|-6|single family| +38459|AAAAAAAALDGJAAAA|46|Center |RD|Suite 390|Liberty|Collin County|TX|73451|United States|-6|single family| +38460|AAAAAAAAMDGJAAAA|967|Dogwood |Dr.|Suite Q|Riverdale|Lake and Peninsula Borough|AK|99391|United States|-9|single family| +38461|AAAAAAAANDGJAAAA|959|Pine Sycamore|Drive|Suite 30|Wildwood|Orange County|CA|96871|United States|-8|apartment| +38462|AAAAAAAAODGJAAAA|804|Ash |Boulevard|Suite A|Lakeside|Chippewa County|WI|59532|United States|-6|single family| +38463|AAAAAAAAPDGJAAAA|59|Cherry |Cir.|Suite 410|Woodlawn|Scott County|KS|64098|United States|-6|condo| +38464|AAAAAAAAAEGJAAAA|72|Ridge |Lane|Suite 110|Fairfield|Bourbon County|KY|46192|United States|-6|single family| +38465|AAAAAAAABEGJAAAA|765|Park Locust|Wy|Suite 460|Plainview|Person County|NC|23683|United States|-5|apartment| +38466|AAAAAAAACEGJAAAA|889|Oak |Wy|Suite 340|Lakeview|York County|NE|68579|United States|-6|single family| +38467|AAAAAAAADEGJAAAA|688|Ridge |Dr.|Suite A|Riverview|Montgomery County|GA|39003|United States|-5|single family| +38468|AAAAAAAAEEGJAAAA||||Suite V|Green Acres|||37683|||single family| +38469|AAAAAAAAFEGJAAAA|590|Park |Ave|Suite C|Hartland|Tensas Parish|LA|76594|United States|-6|condo| +38470|AAAAAAAAGEGJAAAA|933|First Oak|Drive|Suite L|Macedonia|Aleutians West Census Area|AK|91087|United States|-9|condo| +38471|AAAAAAAAHEGJAAAA|774|Maple 8th|Ln|Suite 140|Cedar Grove|Ohio County|WV|20411|United States|-5|condo| +38472|AAAAAAAAIEGJAAAA|277|Locust |Ln|Suite 90|Lebanon|Kent County|TX|72898|United States|-6|single family| +38473|AAAAAAAAJEGJAAAA|401|Park |Avenue|Suite V|Sulphur Springs|Marshall County|TN|38354|United States|-6|condo| +38474|AAAAAAAAKEGJAAAA|859|Cherry 8th|Pkwy|Suite 450|Maple Grove|Sac County|IA|58252|United States|-6|condo| +38475|AAAAAAAALEGJAAAA||Meadow |Boulevard|||Perry County|||United States|-5|single family| +38476|AAAAAAAAMEGJAAAA|391|8th |Drive|Suite Q|Flatwoods|Hays County|TX|74212|United States|-6|single family| +38477|AAAAAAAANEGJAAAA|639|Washington |RD|Suite 70|Sunnyside|Johnson County|WY|81952|United States|-7|apartment| +38478|AAAAAAAAOEGJAAAA|11|North Main|Dr.|Suite B|Spring Valley|Hemphill County|TX|76060|United States|-6|condo| +38479|AAAAAAAAPEGJAAAA|505|Dogwood Dogwood|Circle|Suite 150|Pleasant Grove|Goochland County|VA|24136|United States|-5|apartment| +38480|AAAAAAAAAFGJAAAA|751|Fifth Mill|Parkway|Suite 270|Riverdale|Edgefield County|SC|29391|United States|-5|single family| +38481|AAAAAAAABFGJAAAA|297|Laurel Adams|Dr.|Suite 280|Oak Ridge|Beauregard Parish|LA|78371|United States|-6|apartment| +38482|AAAAAAAACFGJAAAA|198|Lakeview |Street|Suite P|Lewisville|Crawford County|GA|30056|United States|-5|apartment| +38483|AAAAAAAADFGJAAAA|713|Central |Dr.|Suite A|Clinton|Kent County|DE|18222|United States|-5|condo| +38484|AAAAAAAAEFGJAAAA|848|Pine |Court|Suite 200|Hubbard|Toombs County|GA|36291|United States|-5|apartment| +38485|AAAAAAAAFFGJAAAA|836|West |Lane|Suite 210|Lincoln|Grayson County|VA|21289|United States|-5|condo| +38486|AAAAAAAAGFGJAAAA|531|Jefferson |Ct.|Suite 330|Deerfield|Oklahoma County|OK|79840|United States|-6|condo| +38487|AAAAAAAAHFGJAAAA|819|Lincoln |Dr.|Suite 370|Summit|Conecuh County|AL|30499|United States|-6|single family| +38488|AAAAAAAAIFGJAAAA|632|Wilson |Road|Suite B|Woodville|Kandiyohi County|MN|54289|United States|-6|single family| +38489|AAAAAAAAJFGJAAAA|974|9th 4th|Court|Suite H|Lakeside|Culberson County|TX|79532|United States|-6|single family| +38490|AAAAAAAAKFGJAAAA|174|Hill 1st|Court|Suite U|Oakwood|Union County|SD|50169|United States|-7|apartment| +38491|AAAAAAAALFGJAAAA|239|Birch |ST|Suite G|||TX|76192|||condo| +38492|AAAAAAAAMFGJAAAA|638|West |Boulevard|Suite 200|Morgantown|Esmeralda County|NV|89193|United States|-8|single family| +38493|AAAAAAAANFGJAAAA|484|||Suite M|Belmont||RI||United States|-5|condo| +38494|AAAAAAAAOFGJAAAA|928|Sunset Oak|Ln|Suite 210|Deerfield|Mahaska County|IA|59840|United States|-6|apartment| +38495|AAAAAAAAPFGJAAAA|467|Highland Laurel|Circle|Suite 310|Shiloh|McLean County|ND|59275|United States|-6|apartment| +38496|AAAAAAAAAGGJAAAA|181|10th |Parkway|Suite 290|Lakeville|Holt County|MO|68811|United States|-6|condo| +38497|AAAAAAAABGGJAAAA|535|15th Center|Road|Suite P|Florence|Bath County|VA|23394|United States|-5|single family| +38498|AAAAAAAACGGJAAAA|967|Railroad |Circle|Suite I|Kingston|Coffee County|TN|34975|United States|-5|condo| +38499|AAAAAAAADGGJAAAA|651|Oak East|ST|Suite 230|Milan|La Crosse County|WI|56697|United States|-6|condo| +38500|AAAAAAAAEGGJAAAA|309|Green Chestnut|Blvd|Suite Q|Oakdale|Bennett County|SD|59584|United States|-6|apartment| +38501|AAAAAAAAFGGJAAAA|218||Dr.||||WA||United States||| +38502|AAAAAAAAGGGJAAAA|588|10th |Street|Suite M|Deerfield|Bienville Parish|LA|79840|United States|-6|single family| +38503|AAAAAAAAHGGJAAAA|542|1st |Ln|Suite 290|Woodville|Liberty County|GA|34289|United States|-5|single family| +38504|AAAAAAAAIGGJAAAA|808|River Valley|Dr.|Suite F|Salem|Hancock County|KY|48048|United States|-6|single family| +38505|AAAAAAAAJGGJAAAA|247|1st |Road|Suite A|Lakeside|Sumter County|SC|29532|United States|-5|single family| +38506|AAAAAAAAKGGJAAAA|981|Spring |Wy|Suite 190|Valley View|Fulton County|OH|45124|United States|-5|condo| +38507|AAAAAAAALGGJAAAA|495|Cherry |Circle|Suite 100|Green Acres|Dickenson County|VA|27683|United States|-5|apartment| +38508|AAAAAAAAMGGJAAAA|569|Lake |Avenue|Suite 120|Summerville|Kleberg County|TX|72033|United States|-6|single family| +38509|AAAAAAAANGGJAAAA|185|Park Valley|Way|Suite 420|Lebanon|Sumter County|AL|32898|United States|-6|apartment| +38510|AAAAAAAAOGGJAAAA|609|Sycamore |Way|Suite G|Providence|Stevens County|MN|56614|United States|-6|apartment| +38511|AAAAAAAAPGGJAAAA|557|1st Lakeview|Cir.|Suite X|Wilson|Brown County|OH|46971|United States|-5|condo| +38512|AAAAAAAAAHGJAAAA|26|3rd Highland|Cir.|Suite 280|Hillcrest|Wayne County|MI|43003|United States|-5|single family| +38513|AAAAAAAABHGJAAAA|970|Ninth |Parkway|Suite B|Highland|McHenry County|ND|59454|United States|-6|apartment| +38514|AAAAAAAACHGJAAAA|824|Second View|Avenue|Suite K|Oak Grove|Niobrara County|WY|88370|United States|-7|single family| +38515|AAAAAAAADHGJAAAA|955|Cherry |Ct.|Suite O|Maple Hill|Beltrami County|MN|58095|United States|-6|single family| +38516|AAAAAAAAEHGJAAAA|688|2nd Second|Boulevard|Suite 240|Valley View|Humphreys County|MS|55124|United States|-6|apartment| +38517|AAAAAAAAFHGJAAAA|993|Highland |Lane|Suite 380|Five Points|Ferry County|WA|96098|United States|-8|single family| +38518|AAAAAAAAGHGJAAAA|890|Railroad Hickory|Avenue|Suite 220|Oak Ridge|Lagrange County|IN|48371|United States|-5|apartment| +38519|AAAAAAAAHHGJAAAA|114|2nd |Pkwy|Suite C|Friendship|Pulaski County|AR|74536|United States|-6|apartment| +38520|AAAAAAAAIHGJAAAA|492|8th Adams|RD|Suite 420|Lakeside|Calhoun County|GA|39532|United States|-5|single family| +38521|AAAAAAAAJHGJAAAA|149|Williams 11th|Way|Suite 480|Ashland|Rock County|WI|54244|United States|-6|condo| +38522|AAAAAAAAKHGJAAAA|362|4th |Wy|Suite 190|Hopewell|Armstrong County|PA|10587|United States|-5|apartment| +38523|AAAAAAAALHGJAAAA|96|Center |Pkwy|Suite 470|White Oak|Fayette County|WV|26668|United States|-5|apartment| +38524|AAAAAAAAMHGJAAAA|52|Spring 5th|Ave|Suite 10|Pine Grove|Grant Parish|LA|74593|United States|-6|apartment| +38525|AAAAAAAANHGJAAAA|21|Walnut 9th|Drive|Suite 60|Greenville|Kent County|MI|41387|United States|-5|apartment| +38526|AAAAAAAAOHGJAAAA|329|Highland 14th|Dr.|Suite 320|Waterloo|Summit County|UT|81675|United States|-7|apartment| +38527|AAAAAAAAPHGJAAAA|791||Circle|Suite I|Brownsville||KY||||| +38528|AAAAAAAAAIGJAAAA|925|Mill Lincoln|Cir.|Suite J|Burns|Attala County|MS|55272|United States|-6|single family| +38529|AAAAAAAABIGJAAAA|262|13th 13th|Pkwy|Suite 100|Red Hill|Chelan County|WA|94338|United States|-8|apartment| +38530|AAAAAAAACIGJAAAA|989|Main |Lane|Suite 120|Mountain View|Wade Hampton Census Area|AK|94466|United States|-9|condo| +38531|AAAAAAAADIGJAAAA|207|1st Walnut|Circle|Suite C|Brownsville|Jerome County|ID|89310|United States|-7|condo| +38532|AAAAAAAAEIGJAAAA|533|Chestnut |Blvd|Suite T|Woodville|Audrain County|MO|64289|United States|-6|single family| +38533|AAAAAAAAFIGJAAAA|278|Third Sunset|Ave|Suite 470||Shoshone County||86557||-7|| +38534|AAAAAAAAGIGJAAAA|882|Hill |Ave|Suite 380|Clifton|Wade Hampton Census Area|AK|98014|United States|-9|apartment| +38535|AAAAAAAAHIGJAAAA|952|Oak Walnut|RD|Suite 120|Macedonia|Bandera County|TX|71087|United States|-6|condo| +38536|AAAAAAAAIIGJAAAA|298|Church 4th|Parkway|Suite V|Wildwood|Fannin County|GA|36871|United States|-5|condo| +38537|AAAAAAAAJIGJAAAA|441|College 5th|Ave|Suite W|Lone Oak|Knott County|KY|46893|United States|-5|apartment| +38538|AAAAAAAAKIGJAAAA|662|Center Chestnut|Ln|Suite H|Centerville|Belmont County|OH|40059|United States|-5|apartment| +38539|AAAAAAAALIGJAAAA|701|Railroad |Parkway|Suite X|Greenville|Newport News city|VA|21387|United States|-5|condo| +38540|AAAAAAAAMIGJAAAA|472|Mill 5th|Dr.|Suite I|Midway|Wasco County|OR|91904|United States|-8|apartment| +38541|AAAAAAAANIGJAAAA|398|Elevnth |Ln|Suite 290|New Hope|Jack County|TX|79431|United States|-6|condo| +38542|AAAAAAAAOIGJAAAA|138|6th Main|Drive|Suite W|Belmont|Westchester County|NY|10191|United States|-5|single family| +38543|AAAAAAAAPIGJAAAA|974|Lincoln |Dr.|Suite 50|Buena Vista|Davison County|SD|55752|United States|-6|condo| +38544|AAAAAAAAAJGJAAAA|790|Birch |Boulevard|Suite N|Highland|Barry County|MI|49454|United States|-5|condo| +38545|AAAAAAAABJGJAAAA|591|Walnut Jackson|ST|Suite N|Hillcrest|Buffalo County|SD|53003|United States|-6|apartment| +38546|AAAAAAAACJGJAAAA|607|Laurel Wilson|Cir.|Suite I|Ruth|Alleghany County|VA|20309|United States|-5|single family| +38547|AAAAAAAADJGJAAAA|250|12th |Drive|Suite 470|Brownsville|Tyler County|TX|79310|United States|-6|single family| +38548|AAAAAAAAEJGJAAAA|772|Hill |Pkwy|Suite L|Lakeside|Walton County|FL|39532|United States|-5|single family| +38549|AAAAAAAAFJGJAAAA|771|Church 6th|Avenue|Suite E|Riverside|Dolores County|CO|89231|United States|-7|condo| +38550|AAAAAAAAGJGJAAAA|779|Railroad |Lane|Suite U|Royal|Mitchell County|TX|75819|United States|-6|condo| +38551|AAAAAAAAHJGJAAAA|345|4th |RD|Suite 60|Spring Valley|Hartford County|CT|06660|United States|-5|single family| +38552|AAAAAAAAIJGJAAAA|565|Church |Parkway|Suite 470|Summit|Muskegon County|MI|40499|United States|-5|single family| +38553|AAAAAAAAJJGJAAAA|937|Willow Cherry|Boulevard|Suite 20|Woodlawn|Smith County|TX|74098|United States|-6|apartment| +38554|AAAAAAAAKJGJAAAA|233|Spring Dogwood|Circle|Suite 390|Deerfield|Menominee County|MI|49840|United States|-5|condo| +38555|AAAAAAAALJGJAAAA|466|Main |Street|Suite D|Pine Grove|Jackson County|OR|94593|United States|-8|apartment| +38556|AAAAAAAAMJGJAAAA|380|15th Seventh|Cir.|Suite 120|Millbrook|Osceola County|FL|37529|United States|-5|condo| +38557|AAAAAAAANJGJAAAA|997|View |Street|Suite B|Shady Grove|Wright County|MO|62812|United States|-6|condo| +38558|AAAAAAAAOJGJAAAA|275|Dogwood Fourth|Pkwy|Suite A|Cedar Grove|Lancaster County|VA|20411|United States|-5|single family| +38559|AAAAAAAAPJGJAAAA|606|Wilson Chestnut|Avenue|Suite 210|Edgewood|Union County|NJ|00669|United States|-5|apartment| +38560|AAAAAAAAAKGJAAAA|684|2nd Tenth|Ave|Suite 400|Mountain View|Grant County|OR|94466|United States|-8|apartment| +38561|AAAAAAAABKGJAAAA|13|Green Spruce|Dr.|Suite J|Point Pleasant|Seminole County|OK|74749|United States|-6|single family| +38562|AAAAAAAACKGJAAAA|469|Spring |Ln|Suite E|Phoenix|Sumter County|FL|32276|United States|-5|condo| +38563|AAAAAAAADKGJAAAA|681|Walnut Poplar|Court|Suite A|Hopewell|McLennan County|TX|70587|United States|-6|single family| +38564|AAAAAAAAEKGJAAAA|673|Meadow |Drive|Suite 250|Pine Grove|Roanoke city|VA|24593|United States|-5|apartment| +38565|AAAAAAAAFKGJAAAA|832|3rd Highland|Road|Suite 280|Newtown|Hardeman County|TX|71749|United States|-6|condo| +38566|AAAAAAAAGKGJAAAA|516|Cedar Lincoln|Cir.|Suite O|Antioch|Spartanburg County|SC|28605|United States|-5|single family| +38567|AAAAAAAAHKGJAAAA|406|9th Spruce|Drive|Suite A|Oakwood|Juneau Borough|AK|90169|United States|-9|condo| +38568|AAAAAAAAIKGJAAAA|255|View West|Road|Suite B|Mountain View|Dodge County|NE|64466|United States|-6|condo| +38569|AAAAAAAAJKGJAAAA|79|3rd |Court|Suite I|Prosperity|Mackinac County|MI|49089|United States|-5|single family| +38570|AAAAAAAAKKGJAAAA|59|Willow 11th|Blvd|Suite A|Lakewood|Wakulla County|FL|38877|United States|-5|single family| +38571|AAAAAAAALKGJAAAA|333|Jackson Forest|Circle|Suite 450|Gilmore|Washington County|TN|35464|United States|-6|apartment| +38572|AAAAAAAAMKGJAAAA|10|Third |Blvd|Suite 200|Fairfield|Gaines County|TX|76192|United States|-6|condo| +38573|AAAAAAAANKGJAAAA|652|Oak |Ln|Suite Q|Green Acres|New Kent County|VA|27683|United States|-5|condo| +38574|AAAAAAAAOKGJAAAA|252|Mill |Circle|Suite V|Taft|Wayne County|MS|50589|United States|-6|apartment| +38575|AAAAAAAAPKGJAAAA|762|Miller 9th|Road|Suite C|Pleasant Valley|Iredell County|NC|22477|United States|-5|apartment| +38576|AAAAAAAAALGJAAAA|715|Willow |Parkway|Suite 370|Union|Yavapai County|AZ|88721|United States|-7|condo| +38577|AAAAAAAABLGJAAAA|218|6th |ST|Suite C|Mount Zion|Androscoggin County|ME|08654|United States|-5|apartment| +38578|AAAAAAAACLGJAAAA|183|9th |Lane|Suite Q|Belmont|Josephine County|OR|90191|United States|-8|single family| +38579|AAAAAAAADLGJAAAA||Jefferson Ninth||Suite B|Union|Stoddard County|MO|68721|United States|-6|| +38580|AAAAAAAAELGJAAAA|500|Miller Lake|Court|Suite K|Four Points|Gloucester County|NJ|01816|United States|-5|apartment| +38581|AAAAAAAAFLGJAAAA|226|Pine |Lane|Suite I|Rankin|Brown County|KS|62621|United States|-6|single family| +38582|AAAAAAAAGLGJAAAA|676|First |Parkway|Suite 390|Glenwood|Dauphin County|PA|13511|United States|-5|single family| +38583|AAAAAAAAHLGJAAAA|202|Madison |Lane|Suite J|New Hope|Poweshiek County|IA|59431|United States|-6|single family| +38584|AAAAAAAAILGJAAAA|558|4th Hickory|Street|Suite V|Liberty|Crosby County|TX|73451|United States|-6|condo| +38585|AAAAAAAAJLGJAAAA|356|2nd Lee|Cir.|Suite A|Hamilton|Noble County|IN|42808|United States|-5|condo| +38586|AAAAAAAAKLGJAAAA|787|Sixth Oak|Way|Suite 450|Pleasant Grove|White County|TN|34136|United States|-6|apartment| +38587|AAAAAAAALLGJAAAA|640|Oak Lake|Drive|Suite R|Pine Grove|Pawnee County|KS|64593|United States|-6|condo| +38588|AAAAAAAAMLGJAAAA|871|Davis |Wy|Suite X|Woodland|Morris County|KS|64854|United States|-6|apartment| +38589|AAAAAAAANLGJAAAA|297|West First|Circle|Suite 310|Riverdale|Garden County|NE|69391|United States|-6|condo| +38590|AAAAAAAAOLGJAAAA|474|Ash |Ave|Suite Y|Oak Ridge|Winneshiek County|IA|58371|United States|-6|condo| +38591|AAAAAAAAPLGJAAAA|470|Wilson Davis|Parkway|Suite 310|Woodlawn|Pike County|IN|44098|United States|-5|condo| +38592|AAAAAAAAAMGJAAAA|520|Railroad Main|Street|Suite 140|Oakland|Comanche County|KS|69843|United States|-6|apartment| +38593|AAAAAAAABMGJAAAA|167|Ash Center|Parkway|Suite F|Summit|Union County|NC|20499|United States|-5|single family| +38594|AAAAAAAACMGJAAAA|370|Oak |Drive|Suite I|Oak Ridge|Martin County|KY|48371|United States|-5|condo| +38595|AAAAAAAADMGJAAAA|745|South Main|Drive|Suite O|Union Hill|Baca County|CO|87746|United States|-7|condo| +38596|AAAAAAAAEMGJAAAA|973|10th |Parkway|Suite Y|Brownsville|Bolivar County|MS|59310|United States|-6|condo| +38597|AAAAAAAAFMGJAAAA|75|Laurel |Ln|Suite 20|Union|Clay County|AL|38721|United States|-6|condo| +38598|AAAAAAAAGMGJAAAA|540|4th 1st|Dr.|Suite J|Waterloo|Harrison County|MS|51675|United States|-6|condo| +38599|AAAAAAAAHMGJAAAA|188|West |Way|Suite C|Crossroads|Marion County|OH|40534|United States|-5|apartment| +38600|AAAAAAAAIMGJAAAA|126|Sixth College|Ave|Suite K|Avery|Perry County|MS|50194|United States|-6|single family| +38601|AAAAAAAAJMGJAAAA|966|Pine Sunset|Blvd|Suite B|Valley View|Nome Census Area|AK|95124|United States|-9|apartment| +38602|AAAAAAAAKMGJAAAA|405|Thirteenth Sixth|Lane|Suite U|Plainview|Jeff Davis County|TX|73683|United States|-6|apartment| +38603|AAAAAAAALMGJAAAA|482|Sunset Spring|Ave|Suite J|Mount Olive|Madison County|GA|38059|United States|-5|apartment| +38604|AAAAAAAAMMGJAAAA|83|2nd |Cir.|Suite C|Buena Vista|Corson County|SD|55752|United States|-6|single family| +38605|AAAAAAAANMGJAAAA|476|8th South|RD|Suite 140|Lincoln|Bradley County|TN|31289|United States|-5|single family| +38606|AAAAAAAAOMGJAAAA|899|Valley |Lane|Suite 290|Hillcrest|Walker County|AL|33003|United States|-6|single family| +38607|AAAAAAAAPMGJAAAA|831|Williams |Circle|Suite 30|Centerville|Houghton County|MI|40059|United States|-5|apartment| +38608|AAAAAAAAANGJAAAA|228|Jackson |Ln|Suite 170|Jackson|Boyd County|NE|69583|United States|-6|condo| +38609|AAAAAAAABNGJAAAA|43|Sycamore Second|Boulevard|Suite Q|Fairview|Hancock County|TN|35709|United States|-5|apartment| +38610|AAAAAAAACNGJAAAA|435|View East|Pkwy|Suite T|Waterloo|Vanderburgh County|IN|41675|United States|-5|apartment| +38611|AAAAAAAADNGJAAAA|735|2nd Walnut|Lane|Suite N|Salem|Manatee County|FL|38048|United States|-5|apartment| +38612|AAAAAAAAENGJAAAA|682|4th |Road|Suite 450|Birmingham|Lewis and Clark County|MT|63372|United States|-7|apartment| +38613|AAAAAAAAFNGJAAAA|675|Jefferson North|Ave|Suite 490|Owens|Beaverhead County|MT|62324|United States|-7|condo| +38614|AAAAAAAAGNGJAAAA|540|3rd 10th|Boulevard|Suite 20|Arlington|Bryan County|GA|36557|United States|-5|apartment| +38615|AAAAAAAAHNGJAAAA||South Franklin||Suite G|||||United States||apartment| +38616|AAAAAAAAINGJAAAA|361|13th |Parkway|Suite S|Glendale|Cherokee County|OK|73951|United States|-6|apartment| +38617|AAAAAAAAJNGJAAAA|313|Poplar |Drive|Suite 470|Pleasant Hill|Richland County|MT|63604|United States|-7|apartment| +38618|AAAAAAAAKNGJAAAA|939|Valley |Parkway|||Gentry County|MO|||-6|| +38619|AAAAAAAALNGJAAAA|831|Chestnut Dogwood|Dr.|Suite L|Cedar Grove|Marion County|AL|30411|United States|-6|condo| +38620|AAAAAAAAMNGJAAAA|607|Lincoln Hill|Circle|Suite 360|Red Hill|Lynn County|TX|74338|United States|-6|single family| +38621|AAAAAAAANNGJAAAA|548|Hill Ridge|Dr.|Suite J|Clifton|Larue County|KY|48014|United States|-5|apartment| +38622|AAAAAAAAONGJAAAA|885|College |Court|Suite B|Oak Ridge|Harrison County|MO|68371|United States|-6|apartment| +38623|AAAAAAAAPNGJAAAA|676|15th South|Court|Suite 380|Mount Olive|Wallace County|KS|68059|United States|-6|single family| +38624|AAAAAAAAAOGJAAAA|499|8th |Road|Suite 490|Shiloh|Adams County|IL|69275|United States|-6|apartment| +38625|AAAAAAAABOGJAAAA|845|Sycamore |Street|Suite T|Clifton|Beauregard Parish|LA|78014|United States|-6|apartment| +38626|AAAAAAAACOGJAAAA|531|Sunset |Street|Suite S|Highland Park|Salem County|NJ|07134|United States|-5|apartment| +38627|AAAAAAAADOGJAAAA|494|Smith First|Way|Suite 210|Bethel|Los Alamos County|NM|85281|United States|-7|apartment| +38628|AAAAAAAAEOGJAAAA|195|Franklin Maple|RD|Suite J|Marion|Patrick County|VA|20399|United States|-5|apartment| +38629|AAAAAAAAFOGJAAAA|149|Park Chestnut|Blvd|Suite X|Oak Grove|Sanpete County|UT|88370|United States|-7|condo| +38630|AAAAAAAAGOGJAAAA|765|7th |Dr.|Suite 380|Mount Olive|Clay County|IN|48059|United States|-5|condo| +38631|AAAAAAAAHOGJAAAA|433|Washington Pine|Court|Suite 90|Antioch|Potter County|SD|58605|United States|-7|condo| +38632|AAAAAAAAIOGJAAAA|577|Park Franklin|Court|Suite 410|Mount Vernon|Cass County|MO|68482|United States|-6|single family| +38633|AAAAAAAAJOGJAAAA|590|11th |Ln|Suite 350|Glenwood|McIntosh County|OK|73511|United States|-6|apartment| +38634|AAAAAAAAKOGJAAAA|348|Washington Green|Court|Suite J|Jackson|Harmon County|OK|79583|United States|-6|condo| +38635|AAAAAAAALOGJAAAA|63|Birch |Ave|Suite C|Sulphur Springs|Dixon County|NE|68354|United States|-6|apartment| +38636|AAAAAAAAMOGJAAAA|320|Spring |Street|Suite I|Marion|Bergen County|NJ|00999|United States|-5|single family| +38637|AAAAAAAANOGJAAAA|429|2nd Wilson|RD|Suite 260|Harmony|Hart County|GA|35804|United States|-5|single family| +38638|AAAAAAAAOOGJAAAA|832|Park Mill|Ct.|Suite D|Harmony|La Salle County|IL|65804|United States|-6|condo| +38639|AAAAAAAAPOGJAAAA|294|Willow |Dr.|Suite N|Bunker Hill|Stafford County|KS|60150|United States|-6|apartment| +38640|AAAAAAAAAPGJAAAA||Railroad Third||||Cabarrus County|||United States||apartment| +38641|AAAAAAAABPGJAAAA|889|Green West|Blvd|Suite 490|Sunnyside|Florence County|SC|21952|United States|-5|condo| +38642|AAAAAAAACPGJAAAA|12|Johnson |Lane|Suite 0|Pleasant Valley|Boise County|ID|82477|United States|-7|apartment| +38643|AAAAAAAADPGJAAAA|868|Hickory |Circle|Suite N|Harmony|Perkins County|NE|65804|United States|-7|apartment| +38644|AAAAAAAAEPGJAAAA||Sycamore Lee||Suite 130||Pike County||77838|United States||| +38645|AAAAAAAAFPGJAAAA|95|View Fifth|Parkway|Suite 290|Stringtown|Crosby County|TX|70162|United States|-6|apartment| +38646|AAAAAAAAGPGJAAAA|624|Park Ridge|Dr.|Suite R|Longwood|Jefferson County|AL|37021|United States|-6|apartment| +38647|AAAAAAAAHPGJAAAA|854|8th |Ave|Suite F|Fairbanks|Bland County|VA|26653|United States|-5|condo| +38648|AAAAAAAAIPGJAAAA|923|Sixth 10th|Pkwy|Suite S|Riverview|Lincoln Parish|LA|79003|United States|-6|single family| +38649|AAAAAAAAJPGJAAAA|132|Second |Court|Suite Y|Rockwood|Jefferson County|ID|81545|United States|-7|apartment| +38650|AAAAAAAAKPGJAAAA|601|3rd 7th|Dr.|Suite 90|Albany|Jackson County|MS|52834|United States|-6|apartment| +38651|AAAAAAAALPGJAAAA|14|Smith |Pkwy|Suite X|Highland|Carroll County|IN|49454|United States|-5|single family| +38652|AAAAAAAAMPGJAAAA|996|Main |Way|Suite A|Woodbury|Wythe County|VA|24489|United States|-5|apartment| +38653|AAAAAAAANPGJAAAA|865|View View|ST|Suite 50|Riverview|Clay County|IL|69003|United States|-6|apartment| +38654|AAAAAAAAOPGJAAAA|544|Laurel |Court|Suite 380|Oak Grove|Lane County|OR|98370|United States|-8|condo| +38655|AAAAAAAAPPGJAAAA|||Street||Mount Pleasant||GA|||-5|| +38656|AAAAAAAAAAHJAAAA|47|1st |Cir.|Suite 360|Union|Surry County|NC|28721|United States|-5|single family| +38657|AAAAAAAABAHJAAAA|141|Oak |Ct.|Suite 140|Deerfield|Henderson County|NC|29840|United States|-5|single family| +38658|AAAAAAAACAHJAAAA|591|Willow |Avenue|Suite X|Mount Olive|Cocke County|TN|38059|United States|-5|condo| +38659|AAAAAAAADAHJAAAA|452|Forest Walnut|Way|Suite 420|Oak Ridge|Queen Anne County|MD|28371|United States|-5|apartment| +38660|AAAAAAAAEAHJAAAA|661|Adams View|Blvd|Suite 310|Mount Vernon|La Salle Parish|LA|78482|United States|-6|condo| +38661|AAAAAAAAFAHJAAAA|855|15th 13th|Dr.|Suite 100|Florence|Marion County|SC|23394|United States|-5|single family| +38662|AAAAAAAAGAHJAAAA|362|Sycamore 14th|Pkwy|Suite A|Oak Grove|Howard County|TX|78370|United States|-6|apartment| +38663|AAAAAAAAHAHJAAAA|705|Smith |Wy|Suite K|Providence|Dearborn County|IN|46614|United States|-5|apartment| +38664|AAAAAAAAIAHJAAAA|770|2nd |Ave|Suite 180|Newtown|Hayes County|NE|61749|United States|-6|condo| +38665|AAAAAAAAJAHJAAAA|637|Center Hillcrest|ST|Suite E|Wright|Union County|TN|32814|United States|-6|apartment| +38666|AAAAAAAAKAHJAAAA|204|13th Lee|Road|Suite 90|Johnsonville|Greene County|MO|67745|United States|-6|apartment| +38667|AAAAAAAALAHJAAAA|||Lane||Salem|Benewah County||||-7|| +38668|AAAAAAAAMAHJAAAA|149|View Spruce|Ct.|Suite 200|Brownsville|Silver Bow County|MT|69310|United States|-7|single family| +38669|AAAAAAAANAHJAAAA|621|10th Cedar|Way|Suite 330|Hardy|Hancock County|WV|25354|United States|-5|condo| +38670|AAAAAAAAOAHJAAAA|698|Hill |Boulevard|Suite 300|Marion|Lewis County|NY|10399|United States|-5|condo| +38671|AAAAAAAAPAHJAAAA|693|Hillcrest 4th|Street|Suite 70|Spring Valley|Laramie County|WY|86060|United States|-7|single family| +38672|AAAAAAAAABHJAAAA|710|East Spring|Circle|Suite I|Crossroads|Charlevoix County|MI|40534|United States|-5|condo| +38673|AAAAAAAABBHJAAAA|215|Lake |Way|Suite Y|Farmington|Manitowoc County|WI|59145|United States|-6|condo| +38674|AAAAAAAACBHJAAAA|490|3rd |Parkway|Suite 240|Riverview|San Francisco County|CA|99003|United States|-8|single family| +38675|AAAAAAAADBHJAAAA|786|Ash |Lane|Suite S|Fairview|Neshoba County|MS|55709|United States|-6|apartment| +38676|AAAAAAAAEBHJAAAA|793|Green Sunset|Road|Suite 30|Pine Grove|Northampton County|NC|24593|United States|-5|condo| +38677|AAAAAAAAFBHJAAAA|274|Church 2nd|RD|Suite 110|Mountain View|Lenawee County|MI|44466|United States|-5|condo| +38678|AAAAAAAAGBHJAAAA|255|Johnson |Ct.|Suite 90|Highland|Lake County|FL|39454|United States|-5|apartment| +38679|AAAAAAAAHBHJAAAA|786|Miller |Ave|Suite 400|Clinton|Washington County|KS|68222|United States|-6|single family| +38680|AAAAAAAAIBHJAAAA|946|Lee Cherry|Lane|Suite 50|Sunnyside|Adams County|OH|41952|United States|-5|condo| +38681|AAAAAAAAJBHJAAAA|512|Smith |Blvd|Suite 450|Springtown|Cross County|AR|79858|United States|-6|condo| +38682|AAAAAAAAKBHJAAAA|840|7th |Parkway|Suite 80|Lincoln|Decatur County|GA|31289|United States|-5|apartment| +38683|AAAAAAAALBHJAAAA|88|2nd Poplar|Circle|Suite 280|Williamsville|Luce County|MI|48754|United States|-5|apartment| +38684|AAAAAAAAMBHJAAAA|734|Second 5th|Road|Suite O|Deerfield|Gilliam County|OR|99840|United States|-8|single family| +38685|AAAAAAAANBHJAAAA|46|Birch |Avenue|Suite H|Oakland|Stonewall County|TX|79843|United States|-6|single family| +38686|AAAAAAAAOBHJAAAA|476|2nd |Road|Suite J|Marion|Maries County|MO|60399|United States|-6|apartment| +38687|AAAAAAAAPBHJAAAA|917|Jackson |Wy|Suite 430|Forestville|Okaloosa County|FL|33027|United States|-5|condo| +38688|AAAAAAAAACHJAAAA|221|Second |Lane|Suite S|Texas|Johnson County|IL|63342|United States|-6|condo| +38689|AAAAAAAABCHJAAAA|828|Ash |Circle|Suite 160|Union Hill|Lenoir County|NC|27746|United States|-5|single family| +38690|AAAAAAAACCHJAAAA|550|Davis Green|ST|Suite M|Harmony|Butte County|CA|95804|United States|-8|single family| +38691|AAAAAAAADCHJAAAA|332|15th |Boulevard|Suite D|Wildwood|Lincoln County|ID|86871|United States|-7|apartment| +38692|AAAAAAAAECHJAAAA|887|Mill First|Boulevard|Suite A|Arcadia|Hamilton County|FL|32022|United States|-5|condo| +38693|AAAAAAAAFCHJAAAA|999|Cedar |Lane|Suite 480|Enterprise|Camas County|ID|81757|United States|-7|apartment| +38694|AAAAAAAAGCHJAAAA|241|Meadow |Wy|Suite 350|Woodville|Bradford County|PA|14289|United States|-5|apartment| +38695|AAAAAAAAHCHJAAAA|913|Birch Main|ST|Suite S|Spring Hill|York County|SC|26787|United States|-5|single family| +38696|AAAAAAAAICHJAAAA|117|Ninth Lincoln|Avenue|Suite Q|Brownsville|Wyoming County|PA|19310|United States|-5|condo| +38697|AAAAAAAAJCHJAAAA|684|6th |Avenue|Suite Q|Springfield|Sargent County|ND|59303|United States|-6|apartment| +38698|AAAAAAAAKCHJAAAA|480|Jefferson 2nd|Court|Suite Q|Pleasant Grove|Sumner County|TN|34136|United States|-6|apartment| +38699|AAAAAAAALCHJAAAA|771|Sixth Mill|Drive|Suite 450|Lakeside|Kane County|UT|89532|United States|-7|apartment| +38700|AAAAAAAAMCHJAAAA|712|Chestnut 10th|Court|Suite 100|Indian Village|Monmouth County|NJ|01675|United States|-5|single family| +38701|AAAAAAAANCHJAAAA|660|Lincoln |Ave|Suite O|Waterloo|Crawford County|MO|61675|United States|-6|apartment| +38702|AAAAAAAAOCHJAAAA|331|1st |Blvd|Suite 190|Midway|Rutherford County|NC|21904|United States|-5|condo| +38703|AAAAAAAAPCHJAAAA|286|Elevnth Railroad|RD|Suite 310|Hopewell|Boyd County|KY|40587|United States|-6|single family| +38704|AAAAAAAAADHJAAAA|547|Willow |Avenue|Suite C|Highland Park|Falls County|TX|76534|United States|-6|apartment| +38705|AAAAAAAABDHJAAAA|||Pkwy||Oak Ridge||||United States|-5|condo| +38706|AAAAAAAACDHJAAAA|772|Spring |Way|Suite 80|Springhill|Lincoln Parish|LA|74602|United States|-6|single family| +38707|AAAAAAAADDHJAAAA|167|13th |Road|Suite U|Denmark|Sumter County|AL|35576|United States|-6|single family| +38708|AAAAAAAAEDHJAAAA|753|Second |RD|Suite 80|Clifton|Champaign County|IL|68014|United States|-6|single family| +38709|AAAAAAAAFDHJAAAA|102|Adams |Blvd|Suite S|Greenfield|Nicholas County|KY|45038|United States|-5|condo| +38710|AAAAAAAAGDHJAAAA|471|Fourth |Blvd|Suite 180|Oakwood|Johnson County|IL|60169|United States|-6|single family| +38711|AAAAAAAAHDHJAAAA|420|Third 2nd|Ln|Suite 200|Highland Park|Hooker County|NE|66534|United States|-7|apartment| +38712|AAAAAAAAIDHJAAAA|113|Fourth 3rd|Road|Suite K|Brownsville|Kalamazoo County|MI|49310|United States|-5|apartment| +38713|AAAAAAAAJDHJAAAA|239|Maple |Pkwy|Suite 380|Buena Vista|Polk County|MN|55752|United States|-6|single family| +38714|AAAAAAAAKDHJAAAA|325|Seventh 5th|Drive|Suite 230|Andover|Morgan County|IN|41639|United States|-5|apartment| +38715|AAAAAAAALDHJAAAA|788|Lake Laurel|Boulevard|Suite 30|Providence|James City County|VA|26614|United States|-5|condo| +38716|AAAAAAAAMDHJAAAA|161|14th Lee|Road|Suite J|Mount Pleasant|Gratiot County|MI|41933|United States|-5|apartment| +38717|AAAAAAAANDHJAAAA|681|12th |Lane|Suite 470|Crossroads|Jackson County|KS|60534|United States|-6|condo| +38718|AAAAAAAAODHJAAAA|623|First |Way|Suite 380|Deerfield|Linn County|OR|99840|United States|-8|condo| +38719|AAAAAAAAPDHJAAAA|306|Broadway 7th|Cir.|Suite 90|Unionville|Medina County|TX|71711|United States|-6|condo| +38720|AAAAAAAAAEHJAAAA|441|Willow Wilson|Road|Suite 400|Jamestown|Fergus County|MT|66867|United States|-7|apartment| +38721|AAAAAAAABEHJAAAA|702|1st |Avenue|Suite 190|Union|Valley County|MT|68721|United States|-7|single family| +38722|AAAAAAAACEHJAAAA|439|4th 4th|Ave|Suite O|Oakdale|Holt County|MO|69584|United States|-6|apartment| +38723|AAAAAAAADEHJAAAA|219|Hill |Street|Suite T|Ellsworth|Emmet County|IA|55079|United States|-6|single family| +38724|AAAAAAAAEEHJAAAA|279|13th Oak|Parkway|Suite Q|Hillcrest|Brooks County|GA|33003|United States|-5|single family| +38725|AAAAAAAAFEHJAAAA|56|15th Woodland|Lane|Suite 320|Lebanon|Johnson County|MO|62898|United States|-6|apartment| +38726|AAAAAAAAGEHJAAAA|77|Pine |Road|Suite 280|Pine Grove|Ray County|MO|64593|United States|-6|condo| +38727|AAAAAAAAHEHJAAAA|791|Willow |Street|Suite L|New Hope|Onslow County|NC|29431|United States|-5|condo| +38728|AAAAAAAAIEHJAAAA|967|5th |Pkwy|Suite 260|Springdale|Honolulu County|HI|98883|United States|-10|condo| +38729|AAAAAAAAJEHJAAAA|291|Third |RD|Suite D|Mount Pleasant|Slope County|ND|51933|United States|-6|single family| +38730|AAAAAAAAKEHJAAAA|53|Spring |Lane|Suite 430|Woodland|Augusta County|VA|24854|United States|-5|apartment| +38731|AAAAAAAALEHJAAAA|715|Railroad Miller|RD|Suite 350|Union Hill|Chester County|TN|37746|United States|-5|condo| +38732|AAAAAAAAMEHJAAAA|102|Jefferson |Pkwy|Suite Y|Greenville|Garrard County|KY|41387|United States|-6|apartment| +38733|AAAAAAAANEHJAAAA|787|9th Willow|Ave|Suite W|Derby|Mercer County|ND|57702|United States|-6|single family| +38734|AAAAAAAAOEHJAAAA|516|9th |Blvd|Suite 250|Wilson|Scotts Bluff County|NE|66971|United States|-7|condo| +38735|AAAAAAAAPEHJAAAA|296||Ln||Oakwood||||United States||apartment| +38736|AAAAAAAAAFHJAAAA|70|Woodland |Dr.|Suite 150|Green Acres|Payette County|ID|87683|United States|-7|condo| +38737|AAAAAAAABFHJAAAA|585|Lincoln North|Pkwy|Suite 90|Marion|Moultrie County|IL|60399|United States|-6|apartment| +38738|AAAAAAAACFHJAAAA|803|Birch Elm|Ln|Suite O|Arlington|Baker County|GA|36557|United States|-5|apartment| +38739|AAAAAAAADFHJAAAA|851|14th |Blvd|Suite D|Glenwood|Holmes County|OH|43511|United States|-5|condo| +38740|AAAAAAAAEFHJAAAA|801|First Walnut|Dr.|Suite 350|Bethel||FL|35281|United States|-5|| +38741|AAAAAAAAFFHJAAAA|310|Park |Ave|Suite I|Maple Grove|Grant County|KY|48252|United States|-6|single family| +38742|AAAAAAAAGFHJAAAA|949|Second |Street|Suite 140|Walnut Grove|Bristol Bay Borough|AK|97752|United States|-9|apartment| +38743|AAAAAAAAHFHJAAAA|881|1st Lincoln|Ct.|Suite L|Greenfield|Orange County|VT|05638|United States|-5|condo| +38744|AAAAAAAAIFHJAAAA|736|Smith |Parkway|Suite X|Bethel|Roberts County|TX|75281|United States|-6|apartment| +38745|AAAAAAAAJFHJAAAA|619|North Washington|Boulevard|Suite R|Bethel|Mercer County|IL|65281|United States|-6|condo| +38746|AAAAAAAAKFHJAAAA|769|5th |Pkwy|Suite J|Greenwood|Hardin County|IL|68828|United States|-6|condo| +38747|AAAAAAAALFHJAAAA|536|Hillcrest |RD|Suite 290|Richville|DeWitt County|TX|75945|United States|-6|condo| +38748|AAAAAAAAMFHJAAAA|348|Birch Madison|Blvd|Suite 370|Newport|Merrimack County|NH|02121|United States|-5|apartment| +38749|AAAAAAAANFHJAAAA|359|Main 14th|Boulevard|Suite 210|Stringtown|Montgomery County|IA|50162|United States|-6|condo| +38750|AAAAAAAAOFHJAAAA|332|5th North|Way|Suite U|Riverside|Talbot County|MD|29231|United States|-5|single family| +38751|AAAAAAAAPFHJAAAA|541|Main |Lane|Suite 80|Cedar Grove|Pike County|AR|70411|United States|-6|single family| +38752|AAAAAAAAAGHJAAAA|511|Oak 6th|Circle|Suite 0|Fairview|Franklin County|TN|35709|United States|-5|single family| +38753|AAAAAAAABGHJAAAA|199|1st Third|Court|Suite 0|Plainview|Coryell County|TX|73683|United States|-6|apartment| +38754|AAAAAAAACGHJAAAA|487|Ash |Drive|Suite 170|Wallace|Tooele County|UT|87042|United States|-7|single family| +38755|AAAAAAAADGHJAAAA|617|Jackson |Street|Suite 100|Williamsville|Covington city|VA|28754|United States|-5|condo| +38756|AAAAAAAAEGHJAAAA|436|Walnut |Court|Suite A|Oakwood|Henry County|IN|40169|United States|-5|single family| +38757|AAAAAAAAFGHJAAAA|240|Mill 4th|Drive|Suite C|Five Points|Lamar County|GA|36098|United States|-5|condo| +38758|AAAAAAAAGGHJAAAA|911|Franklin Willow|Circle|Suite D|King|Vernon County|MO|60008|United States|-6|apartment| +38759|AAAAAAAAHGHJAAAA|596|Valley |Lane|Suite 410|Pleasant Valley|Decatur County|IA|52477|United States|-6|single family| +38760|AAAAAAAAIGHJAAAA|873|Washington |Blvd|Suite 150|Glenwood|Perry County|MO|63511|United States|-6|condo| +38761|AAAAAAAAJGHJAAAA|587|Sycamore |Street|Suite O|Harmony|Hendricks County|IN|45804|United States|-5|apartment| +38762|AAAAAAAAKGHJAAAA|282|Pine 4th|Street|Suite 300|Oakdale|Kiowa County|KS|69584|United States|-6|apartment| +38763|AAAAAAAALGHJAAAA|805|First First|Drive|Suite 240|Newport|Henry County|IA|51521|United States|-6|condo| +38764|AAAAAAAAMGHJAAAA|793|Park |ST|Suite S|Florence|Auglaize County|OH|43394|United States|-5|single family| +38765|AAAAAAAANGHJAAAA|269|12th |Wy|Suite A|Youngstown|Escambia County|AL|30001|United States|-6|single family| +38766|AAAAAAAAOGHJAAAA|392|Johnson Hickory|Cir.|Suite G|Glendale|Orleans County|NY|13951|United States|-5|apartment| +38767|AAAAAAAAPGHJAAAA|183|4th |ST|Suite 450|Glendale|Daviess County|KY|43951|United States|-6|condo| +38768|AAAAAAAAAHHJAAAA|176|7th |Circle|Suite U|Lincoln|Pacific County|WA|91289|United States|-8|single family| +38769|AAAAAAAABHHJAAAA|220|Chestnut Dogwood|Boulevard|Suite 450|Macedonia|Jasper County|SC|21087|United States|-5|single family| +38770|AAAAAAAACHHJAAAA|104|Hill |Wy|Suite K|Unionville|Carroll County|KY|41711|United States|-6|condo| +38771|AAAAAAAADHHJAAAA|556|North Valley|Cir.|Suite 70|Mount Olive|Missaukee County|MI|48059|United States|-5|apartment| +38772|AAAAAAAAEHHJAAAA|309|South Pine|Pkwy|Suite 150|Oak Hill|Fergus County|MT|67838|United States|-7|single family| +38773|AAAAAAAAFHHJAAAA|456|Fourth |Ln|Suite A|Riverview|Montgomery County|PA|19003|United States|-5|apartment| +38774|AAAAAAAAGHHJAAAA|918|Mill Jackson|Ct.|Suite P|Mount Vernon|Bond County|IL|68482|United States|-6|apartment| +38775|AAAAAAAAHHHJAAAA|12|Ridge First|Ave|Suite F|Pleasant Grove|Jefferson County|PA|14136|United States|-5|apartment| +38776|AAAAAAAAIHHJAAAA|591|Spruce Second|Street|Suite 120|Walnut|Houston County|AL|36245|United States|-6|apartment| +38777|AAAAAAAAJHHJAAAA|166|Sixth |Boulevard|Suite E|Spring Hill|Larimer County|CO|86787|United States|-7|single family| +38778|AAAAAAAAKHHJAAAA|192|Hill |ST|Suite G|Hamilton|Coleman County|TX|72808|United States|-6|apartment| +38779|AAAAAAAALHHJAAAA|501|Main 15th|Drive|Suite L|Marion|Lincoln Parish|LA|70399|United States|-6|condo| +38780|AAAAAAAAMHHJAAAA|436|Willow Franklin|Street|Suite P|Bunker Hill|Morton County|ND|50150|United States|-6|single family| +38781|AAAAAAAANHHJAAAA|497|Miller Park|Avenue|Suite U|Oak Grove|Boone County|KY|48370|United States|-6|single family| +38782|AAAAAAAAOHHJAAAA|970|Main Oak|Street|Suite 120|Crossroads|Lamoille County|VT|01134|United States|-5|condo| +38783|AAAAAAAAPHHJAAAA|353|1st |RD|Suite T|Tyrone|Oxford County|ME|01801|United States|-5|condo| +38784|AAAAAAAAAIHJAAAA|153|14th Lakeview|Dr.|Suite 200|Kingston|Crawford County|IA|54975|United States|-6|apartment| +38785|AAAAAAAABIHJAAAA|358|View 7th|Ct.|Suite 390|Bunker Hill|Morgan County|TN|30150|United States|-6|condo| +38786|AAAAAAAACIHJAAAA|586|Main Adams|Cir.|Suite G|Mount Zion|Scotland County|NC|28054|United States|-5|single family| +38787|AAAAAAAADIHJAAAA|816|Maple |RD|Suite 440|Harmony|Delta County|TX|75804|United States|-6|single family| +38788|AAAAAAAAEIHJAAAA|496|Main |Blvd|Suite G|Edgewood|Reynolds County|MO|60069|United States|-6|condo| +38789|AAAAAAAAFIHJAAAA|56|2nd First|Boulevard|Suite 420|Waterloo|Kane County|IL|61675|United States|-6|single family| +38790|AAAAAAAAGIHJAAAA|146|Oak Meadow|Cir.|Suite 70|Brownsville|Hale County|AL|39310|United States|-6|single family| +38791|AAAAAAAAHIHJAAAA|598|2nd Elm|Wy|Suite Y|Valley View|Benton County|AR|75124|United States|-6|condo| +38792|AAAAAAAAIIHJAAAA|846|11th Elm|Pkwy|Suite Y|Cooper|Grafton County|NH|05451|United States|-5|single family| +38793|AAAAAAAAJIHJAAAA|876|Highland View|Avenue|Suite J|Buena Vista|Schuyler County|NY|15752|United States|-5|condo| +38794|AAAAAAAAKIHJAAAA|175|Cherry View|Ave|Suite 250|Lone Pine|Washington County|GA|37441|United States|-5|condo| +38795|AAAAAAAALIHJAAAA|806|Main Williams||Suite B|Lakeside|||29532|United States|-5|condo| +38796|AAAAAAAAMIHJAAAA|407|Washington Park|Road|Suite Y|Brownsville|Routt County|CO|89310|United States|-7|apartment| +38797|AAAAAAAANIHJAAAA|544|Smith East|Lane|Suite 330|Jackson|Wallace County|KS|69583|United States|-6|condo| +38798|AAAAAAAAOIHJAAAA|521|2nd Hillcrest|RD|Suite G|Greenfield|Delaware County|PA|15038|United States|-5|condo| +38799|AAAAAAAAPIHJAAAA|752|Fifth Elm|Dr.|Suite 320|Riverview|Plymouth County|IA|59003|United States|-6|condo| +38800|AAAAAAAAAJHJAAAA|336|Pine Ridge|Lane|Suite 90|Jackson|Graham County|KS|69583|United States|-6|condo| +38801|AAAAAAAABJHJAAAA|277|Ninth Cedar|Ln|Suite H|Jackson|Shoshone County|ID|89583|United States|-7|single family| +38802|AAAAAAAACJHJAAAA|554|Woodland 6th|Blvd|Suite I|Crossroads|Albany County|NY|10534|United States|-5|apartment| +38803|AAAAAAAADJHJAAAA||Birch ||Suite I||Harford County||||-5|single family| +38804|AAAAAAAAEJHJAAAA|926|Meadow |Lane|Suite 0|Rocky Point|Yoakum County|TX|71209|United States|-6|single family| +38805|AAAAAAAAFJHJAAAA|829|Cherry |Blvd|Suite 200|Spring Hill|Ashtabula County|OH|46787|United States|-5|single family| +38806|AAAAAAAAGJHJAAAA|100||Lane|||Haakon County|SD||||single family| +38807|AAAAAAAAHJHJAAAA|958|3rd Meadow|Wy|Suite 430|Greenfield|Calhoun County|IA|55038|United States|-6|apartment| +38808|AAAAAAAAIJHJAAAA|21|Madison Ridge|Circle|Suite 320|Concord|Metcalfe County|KY|44107|United States|-5|condo| +38809|AAAAAAAAJJHJAAAA|806|Ash Hillcrest|Ln|Suite 430|Springdale|Grant County|NE|68883|United States|-6|single family| +38810|AAAAAAAAKJHJAAAA|419|First Birch|Street|Suite S|Riverview|Porter County|IN|49003|United States|-5|condo| +38811|AAAAAAAALJHJAAAA|998|College |Ln|Suite 60|Mount Olive|Navajo County|AZ|88059|United States|-7|single family| +38812|AAAAAAAAMJHJAAAA|708|Lincoln Railroad|Ave|Suite W|Mount Olive|Morgan County|KY|48059|United States|-5|condo| +38813|AAAAAAAANJHJAAAA|374|2nd 2nd|Wy|Suite G|Mountain View|Walker County|GA|34466|United States|-5|apartment| +38814|AAAAAAAAOJHJAAAA|52|Cedar |Lane|Suite K|Harmony|Yakima County|WA|95804|United States|-8|apartment| +38815|AAAAAAAAPJHJAAAA|878|Washington Spring|Ave|Suite 280|Antioch|Reynolds County|MO|68605|United States|-6|condo| +38816|AAAAAAAAAKHJAAAA|367|Lakeview Chestnut|||||||United States||single family| +38817|AAAAAAAABKHJAAAA|81|Hickory Oak|Boulevard|Suite 320|Lakeview|Perry County|IN|48579|United States|-5|single family| +38818|AAAAAAAACKHJAAAA|729|Church |Ave|Suite 40|Union Hill|Texas County|OK|77746|United States|-6|apartment| +38819|AAAAAAAADKHJAAAA|350|Railroad 1st|Road|Suite 390|Edgewood|Lamar County|TX|70069|United States|-6|condo| +38820|AAAAAAAAEKHJAAAA|830|Lee 6th|Dr.|Suite 200|Springdale|Hitchcock County|NE|68883|United States|-6|condo| +38821|AAAAAAAAFKHJAAAA|737|Lakeview |Court|Suite O|Shady Grove|Vanderburgh County|IN|42812|United States|-5|condo| +38822|AAAAAAAAGKHJAAAA|493|Church |Circle|Suite J|Oakland|Echols County|GA|39843|United States|-5|single family| +38823|AAAAAAAAHKHJAAAA|53|Willow 3rd|Ln|Suite 440|Enterprise|Barnwell County|SC|21757|United States|-5|apartment| +38824|AAAAAAAAIKHJAAAA|922|Adams Pine|Ave|Suite B|Cedar Grove|Latah County|ID|80411|United States|-7|condo| +38825|AAAAAAAAJKHJAAAA|||Avenue|Suite W|||||United States||single family| +38826|AAAAAAAAKKHJAAAA|624|Fifth |Circle|Suite P|Greenfield|Washington County|CO|85038|United States|-7|condo| +38827|AAAAAAAALKHJAAAA|179|Locust Dogwood|RD|Suite C|Lakeview|Scott County|IN|48579|United States|-5|condo| +38828|AAAAAAAAMKHJAAAA||Johnson ||||Albany County|WY||United States||single family| +38829|AAAAAAAANKHJAAAA|304|5th Fifth|Wy|Suite C|Clinton|Bolivar County|MS|58222|United States|-6|condo| +38830|AAAAAAAAOKHJAAAA|888|Sunset Ash|ST|Suite 120|Summit|Mercer County|MO|60499|United States|-6|condo| +38831|AAAAAAAAPKHJAAAA|729|North Railroad|Ave|Suite 300|Oakland|Webster County|WV|29843|United States|-5|condo| +38832|AAAAAAAAALHJAAAA|221|Williams |Ave|Suite B|Liberty|Texas County|OK|73451|United States|-6|apartment| +38833|AAAAAAAABLHJAAAA|740|Spring Walnut|Parkway|Suite Q|Shiloh|Fairfield County|OH|49275|United States|-5|apartment| +38834|AAAAAAAACLHJAAAA|798|Washington |Road|Suite R|Greenville|Monroe County|IL|61387|United States|-6|apartment| +38835|AAAAAAAADLHJAAAA|217|Johnson |Way|Suite B|Waterloo|Lee County|VA|21675|United States|-5|condo| +38836|AAAAAAAAELHJAAAA|30|Elm 3rd|Wy|Suite F|Hopewell|Greene County|IN|40587|United States|-5|single family| +38837|AAAAAAAAFLHJAAAA|||||Pleasant Hill|||73604|United States||condo| +38838|AAAAAAAAGLHJAAAA|728|Madison |Pkwy|Suite 270|Oneida|Wayne County|OH|44027|United States|-5|apartment| +38839|AAAAAAAAHLHJAAAA|563|Main |Drive|Suite 420|Cedar Grove|Bexar County|TX|70411|United States|-6|condo| +38840|AAAAAAAAILHJAAAA|756|Tenth |Parkway|Suite 330|Spring Valley|Lincoln County|ID|86060|United States|-7|single family| +38841|AAAAAAAAJLHJAAAA|163|Central Main|Pkwy|Suite 270|Highland Park|Macon County|TN|36534|United States|-6|apartment| +38842|AAAAAAAAKLHJAAAA|749|Ridge |Ct.|Suite 30|Highland|Bonner County|ID|89454|United States|-7|condo| +38843|AAAAAAAALLHJAAAA|268|Lincoln 5th|Court|Suite C|Arlington|Miami County|OH|46557|United States|-5|apartment| +38844|AAAAAAAAMLHJAAAA|387|Central 6th|Drive|Suite R|Bridgeport|Cook County|MN|55817|United States|-6|condo| +38845|AAAAAAAANLHJAAAA|136|Meadow |Dr.|Suite 130|Marion|Trinity County|CA|90399|United States|-8|single family| +38846|AAAAAAAAOLHJAAAA|342|Spruce Poplar|Way|Suite E|Ashland|Chaves County|NM|84244|United States|-7|single family| +38847|AAAAAAAAPLHJAAAA|244|Center Woodland|ST|Suite 260|Brownsville|Lander County|NV|89310|United States|-8|single family| +38848|AAAAAAAAAMHJAAAA|328|Miller |Parkway|Suite P|Liberty|Sumner County|KS|63451|United States|-6|single family| +38849|AAAAAAAABMHJAAAA|192|Third 4th|Circle|Suite 80|Cedar Grove|Monroe County|NY|10411|United States|-5|condo| +38850|AAAAAAAACMHJAAAA|993|Spruce |Lane|Suite S|Clinton|Rutherford County|TN|38222|United States|-6|apartment| +38851|AAAAAAAADMHJAAAA|646|Johnson 10th|Road|Suite D|Pleasant Valley|Wayne County|MO|62477|United States|-6|condo| +38852|AAAAAAAAEMHJAAAA|855|Railroad East|Street|Suite 290|Macedonia|Saunders County|NE|61087|United States|-7|single family| +38853|AAAAAAAAFMHJAAAA|851|7th |Drive|Suite E|Waterloo|Washington County|GA|31675|United States|-5|single family| +38854|AAAAAAAAGMHJAAAA|834|Birch South|RD|Suite 20|Liberty|Newton County|MO|63451|United States|-6|apartment| +38855|AAAAAAAAHMHJAAAA|800|Main |Dr.|Suite A|Newport|Dawson County|TX|71521|United States|-6|apartment| +38856|AAAAAAAAIMHJAAAA|30|Oak Hill|Lane|Suite 490|Union|Colquitt County|GA|38721|United States|-5|condo| +38857|AAAAAAAAJMHJAAAA|525|6th |Street|Suite 370|Colfax|Turner County|GA|32565|United States|-5|single family| +38858|AAAAAAAAKMHJAAAA|670|2nd |Ct.|Suite I|Mountain View|Grand Isle County|VT|05066|United States|-5|condo| +38859|AAAAAAAALMHJAAAA|713|Washington Laurel|Boulevard|Suite H|Bridgeport|Southampton County|VA|25817|United States|-5|apartment| +38860|AAAAAAAAMMHJAAAA||||||Contra Costa County|CA|98434|||condo| +38861|AAAAAAAANMHJAAAA|764|Washington |Parkway|Suite W|Farmington|Lowndes County|AL|39145|United States|-6|condo| +38862|AAAAAAAAOMHJAAAA|931|Third |Drive|Suite 350|Red Hill|Camden County|MO|64338|United States|-6|single family| +38863|AAAAAAAAPMHJAAAA|951|Sycamore |Ct.|Suite 260|Mount Zion|Douglas County|IL|68054|United States|-6|single family| +38864|AAAAAAAAANHJAAAA|188|Laurel Wilson|Ave|Suite H|Hamilton|Moore County|NC|22808|United States|-5|single family| +38865|AAAAAAAABNHJAAAA|||Drive|Suite 310|||MI||United States||condo| +38866|AAAAAAAACNHJAAAA|767|Mill 1st|Ct.|Suite 0|Crossroads|Trego County|KS|60534|United States|-6|condo| +38867|AAAAAAAADNHJAAAA|661|Wilson 13th|Cir.|Suite H|Green Acres|Sauk County|WI|57683|United States|-6|apartment| +38868|AAAAAAAAENHJAAAA|192|1st Walnut|Boulevard|Suite H|Brownsville|Suwannee County|FL|39310|United States|-5|single family| +38869|AAAAAAAAFNHJAAAA|370|Fifth |Pkwy|Suite N|Bay View|Pike County|AR|76457|United States|-6|single family| +38870|AAAAAAAAGNHJAAAA|817|Main Sunset|Street|Suite 70|Bunker Hill|Lawrence County|MO|60150|United States|-6|single family| +38871|AAAAAAAAHNHJAAAA|860|Willow |Road|Suite 330|Green Acres|Armstrong County|PA|17683|United States|-5|condo| +38872|AAAAAAAAINHJAAAA|32|Main Hillcrest|Parkway|Suite 340|Forest Hills|Saline County|AR|79237|United States|-6|condo| +38873|AAAAAAAAJNHJAAAA|364|Hill 3rd|Wy|Suite V|Farmington|Roanoke County|VA|29145|United States|-5|condo| +38874|AAAAAAAAKNHJAAAA|132|Adams Willow|Ave|Suite X|Arlington|Adams County|WI|56557|United States|-6|apartment| +38875|AAAAAAAALNHJAAAA|44|Birch |Pkwy|Suite 110|Woodlawn|Washita County|OK|74098|United States|-6|single family| +38876|AAAAAAAAMNHJAAAA|662|Walnut South|Cir.|Suite 400|Woodville|Sumter County|SC|24289|United States|-5|condo| +38877|AAAAAAAANNHJAAAA|617|Franklin Fifth|RD|Suite G|Franklin|Okfuskee County|OK|79101|United States|-6|condo| +38878|AAAAAAAAONHJAAAA|715|Elm |Ln|Suite O|Concord|Jackson County|MI|44107|United States|-5|apartment| +38879|AAAAAAAAPNHJAAAA|101|2nd |ST|Suite 370|Mount Vernon|Floyd County|TX|78482|United States|-6|apartment| +38880|AAAAAAAAAOHJAAAA|574|Laurel East|Wy|Suite 170|Riverside|Jersey County|IL|69231|United States|-6|condo| +38881|AAAAAAAABOHJAAAA|789|Birch 9th|Wy|Suite M|Centerville|Lavaca County|TX|70059|United States|-6|condo| +38882|AAAAAAAACOHJAAAA|344|Jackson 8th|Drive|Suite S|Fairfield|Hampden County|MA|06792|United States|-5|condo| +38883|AAAAAAAADOHJAAAA|347|9th |Road|Suite 140|Edgewood|Knott County|KY|40069|United States|-5|apartment| +38884|AAAAAAAAEOHJAAAA|609|Walnut |Wy|Suite 400|Riverview|Marion County|FL|39003|United States|-5|single family| +38885|AAAAAAAAFOHJAAAA|753|Lakeview |Ave|Suite 470|Pleasant Valley|Marshall County|AL|32477|United States|-6|condo| +38886|AAAAAAAAGOHJAAAA|864|Willow Church|Road|Suite 480|Franklin|Lewis County|TN|39101|United States|-6|condo| +38887|AAAAAAAAHOHJAAAA|404|Jackson 14th|Blvd|Suite C|Greenwood|Roberts County|SD|58828|United States|-7|single family| +38888|AAAAAAAAIOHJAAAA|581|Cherry South|Wy|Suite R|Jackson|Stephenson County|IL|69583|United States|-6|single family| +38889|AAAAAAAAJOHJAAAA|620|1st East|Road|Suite 90|Highland Park|Laurens County|SC|26534|United States|-5|single family| +38890|AAAAAAAAKOHJAAAA|418|Fourth Dogwood|Cir.|Suite 0|Riverside|Walla Walla County|WA|99231|United States|-8|condo| +38891|AAAAAAAALOHJAAAA|303|Spring South|Circle|Suite U|Gladstone|Clark County|SD|50894|United States|-6|condo| +38892|AAAAAAAAMOHJAAAA|523|3rd 6th|ST|Suite 430|Edgewood|Hamilton County|TN|30069|United States|-5|apartment| +38893|AAAAAAAANOHJAAAA|810|9th Spring|Way|Suite 320|Springdale|Bourbon County|KY|48883|United States|-6|single family| +38894|AAAAAAAAOOHJAAAA|710|7th Williams|Wy|Suite K|Greenfield|Owen County|KY|45038|United States|-5|apartment| +38895|AAAAAAAAPOHJAAAA|222|Dogwood |Wy|Suite B|Wildwood|Whatcom County|WA|96871|United States|-8|condo| +38896|AAAAAAAAAPHJAAAA|||Cir.|Suite 170|||IL||||| +38897|AAAAAAAABPHJAAAA|||Ln||Highland|Perquimans County|NC|29454|||apartment| +38898|AAAAAAAACPHJAAAA|522|8th Meadow|Pkwy|Suite 40|Lone Star|Franklin County|TX|77280|United States|-6|single family| +38899|AAAAAAAADPHJAAAA|576|South |Lane|Suite 320|Mountain View|Parmer County|TX|74466|United States|-6|condo| +38900|AAAAAAAAEPHJAAAA|864|5th 3rd|Lane|Suite F|Fairview|Jefferson County|NE|65709|United States|-7|condo| +38901|AAAAAAAAFPHJAAAA|||Lane|Suite 120|||OH|46787|United States||condo| +38902|AAAAAAAAGPHJAAAA|794|Cherry Maple|Ave|Suite N|Pomona|Tippah County|MS|54153|United States|-6|condo| +38903|AAAAAAAAHPHJAAAA|414|South Franklin|Drive|Suite V|Redland|Cochran County|TX|76343|United States|-6|single family| +38904|AAAAAAAAIPHJAAAA|649|8th Lincoln|Parkway|Suite 40|Tabor|Martinsville city|VA|28529|United States|-5|apartment| +38905|AAAAAAAAJPHJAAAA|325|Hickory 1st|Ct.||Mount Pleasant|Washington County|TX|||-6|single family| +38906|AAAAAAAAKPHJAAAA|325|Walnut |RD|Suite 330|Summit|Cass County|TX|70499|United States|-6|condo| +38907|AAAAAAAALPHJAAAA|855|Forest Sycamore|Parkway|Suite K|Union|Dearborn County|IN|48721|United States|-5|single family| +38908|AAAAAAAAMPHJAAAA|586|9th 4th|Ave|Suite X|Maple Grove|Atchison County|KS|68252|United States|-6|single family| +38909|AAAAAAAANPHJAAAA|832|Miller |Drive|Suite 470|Caledonia|Clarke County|VA|27411|United States|-5|single family| +38910|AAAAAAAAOPHJAAAA|359|6th Fourth|RD|Suite 330|Sunnyside|Fauquier County|VA|21952|United States|-5|apartment| +38911|AAAAAAAAPPHJAAAA|30|Park |Avenue|Suite D|Highland|Hunt County|TX|79454|United States|-6|apartment| +38912|AAAAAAAAAAIJAAAA|821|15th |Street|Suite Q|White Oak|Kimble County|TX|76668|United States|-6|single family| +38913|AAAAAAAABAIJAAAA|952|Lakeview |Parkway|Suite M|Brownsville|Otero County|NM|89310|United States|-7|apartment| +38914|AAAAAAAACAIJAAAA|103|River Lakeview|Avenue|Suite V|Clinton|Lake County|OH|48222|United States|-5|apartment| +38915|AAAAAAAADAIJAAAA|478|East |Ave|Suite S|New Hope|Seminole County|FL|39431|United States|-5|condo| +38916|AAAAAAAAEAIJAAAA|202|7th |Road|Suite A|Midway|Craig County|VA|21904|United States|-5|apartment| +38917|AAAAAAAAFAIJAAAA|515|8th 4th|Lane|Suite 220|Newport|Mesa County|CO|81521|United States|-7|condo| +38918|AAAAAAAAGAIJAAAA|272|Locust |Ave|Suite 200|Red Hill|Cedar County|MO|64338|United States|-6|condo| +38919|AAAAAAAAHAIJAAAA|897|Cedar |Ave|Suite 260|Riverview|Dawson County|MT|69003|United States|-7|single family| +38920|AAAAAAAAIAIJAAAA|329|Cedar Ridge|ST|Suite A|Pleasant Grove|Johnson County|GA|34136|United States|-5|condo| +38921|AAAAAAAAJAIJAAAA|742|Johnson |RD|Suite E|Woodland|Johnson County|KY|44854|United States|-6|condo| +38922|AAAAAAAAKAIJAAAA|473|Davis West|Lane|Suite 190|Bayview|Fredericksburg city|VA|29672|United States|-5|condo| +38923|AAAAAAAALAIJAAAA|572|3rd Cherry|Pkwy|Suite K|Fox|Norman County|MN|50631|United States|-6|apartment| +38924|AAAAAAAAMAIJAAAA|747|Wilson |Court|Suite V|Dallas|Stone County|AR|73628|United States|-6|condo| +38925|AAAAAAAANAIJAAAA|128||||Red Hill||MI||United States||apartment| +38926|AAAAAAAAOAIJAAAA|224|Ash |Way|Suite Q|Union Hill|Jefferson County|IL|67746|United States|-6|apartment| +38927|AAAAAAAAPAIJAAAA|31|Green Oak|Road|Suite 190|Mount Pleasant|Orange County|NC|21933|United States|-5|single family| +38928|AAAAAAAAABIJAAAA|890|Elm |Drive|Suite 420|Jackson|Berks County|PA|19583|United States|-5|condo| +38929|AAAAAAAABBIJAAAA|369|Center |RD|Suite G|Liberty|Lawrence County|KY|43451|United States|-5|apartment| +38930|AAAAAAAACBIJAAAA|501|Maple |RD|Suite R|Red Hill|Douglas County|GA|34338|United States|-5|apartment| +38931|AAAAAAAADBIJAAAA|325|Jefferson Spruce|Circle|Suite 440|Harmon|Ramsey County|MN|55623|United States|-6|single family| +38932|AAAAAAAAEBIJAAAA|133|Adams |Street|Suite C|Cottonwood|Boone County|WV|23647|United States|-5|single family| +38933|AAAAAAAAFBIJAAAA|147|Fifth Ridge|Blvd|Suite P|Greenwood|Pointe Coupee Parish|LA|78828|United States|-6|apartment| +38934|AAAAAAAAGBIJAAAA|701|Broadway 9th|Lane|Suite 290|Harmony|Ramsey County|ND|55804|United States|-6|apartment| +38935|AAAAAAAAHBIJAAAA|92|Davis |Circle|Suite 160|Providence|Thomas County|GA|36614|United States|-5|single family| +38936|AAAAAAAAIBIJAAAA|962|Cedar |Lane|Suite U|Star|McDuffie County|GA|30725|United States|-5|single family| +38937|AAAAAAAAJBIJAAAA|71|Walnut |ST|Suite H|Providence|Greene County|MS|56614|United States|-6|single family| +38938|AAAAAAAAKBIJAAAA|823|Madison Lee|Blvd|Suite S|Green Acres|Lincoln County|WA|97683|United States|-8|apartment| +38939|AAAAAAAALBIJAAAA|432|1st |ST|Suite T|Forest Hills|Pickaway County|OH|49237|United States|-5|apartment| +38940|AAAAAAAAMBIJAAAA|978|Sixth Davis|Road|Suite 290|Mount Vernon|Pacific County|WA|98482|United States|-8|condo| +38941|AAAAAAAANBIJAAAA|900|Fifth |Cir.|Suite D|Plainview|Morgan County|TN|33683|United States|-6|apartment| +38942|AAAAAAAAOBIJAAAA|941|Ash |Ct.|Suite W|Highland|Oxford County|ME|09454|United States|-5|single family| +38943|AAAAAAAAPBIJAAAA|528|West |Ct.|Suite 430|Fisher|Chippewa County|WI|52819|United States|-6|single family| +38944|AAAAAAAAACIJAAAA|789|Walnut |Pkwy|Suite Y|Lincoln|Lincoln County|ID|81289|United States|-7|condo| +38945|AAAAAAAABCIJAAAA|204|13th Green|Cir.|Suite F|Valley View|Mason County|MI|45124|United States|-5|condo| +38946|AAAAAAAACCIJAAAA|929|River |ST|Suite 180|Springfield|Cherokee County|KS|69303|United States|-6|condo| +38947|AAAAAAAADCIJAAAA|223|Park Oak|Avenue|Suite 450|Jamestown|Medina County|OH|46867|United States|-5|apartment| +38948|AAAAAAAAECIJAAAA|506|Johnson Hickory|Drive|Suite E|Springfield|Mellette County|SD|59303|United States|-7|apartment| +38949|AAAAAAAAFCIJAAAA|772|Jefferson 10th|Cir.|Suite X|Friendship|Maui County|HI|94536|United States|-10|single family| +38950|AAAAAAAAGCIJAAAA|316|Main |Circle|Suite 380|Pleasant Valley|Upton County|TX|72477|United States|-6|condo| +38951|AAAAAAAAHCIJAAAA|369|Railroad |Ave|Suite J|Oakwood|Mohave County|AZ|80169|United States|-7|condo| +38952|AAAAAAAAICIJAAAA|930|3rd View|Ct.|Suite 200|Bloomingdale|Rutherford County|TN|31824|United States|-6|single family| +38953|AAAAAAAAJCIJAAAA|205|College |Cir.|Suite U|Hardy|Laramie County|WY|85354|United States|-7|condo| +38954|AAAAAAAAKCIJAAAA|474|Chestnut 5th|Ave|Suite 50|Woodlawn|Ida County|IA|54098|United States|-6|condo| +38955|AAAAAAAALCIJAAAA|373|Railroad |Boulevard|Suite 50|Woodville|Douglas County|NE|64289|United States|-6|apartment| +38956|AAAAAAAAMCIJAAAA|55|Locust |Wy|Suite 140|Oakland|Henry County|MO|69843|United States|-6|condo| +38957|AAAAAAAANCIJAAAA|686|Sixth Lincoln|RD|Suite 290|Shiloh|Dillingham Census Area|AK|99275|United States|-9|condo| +38958|AAAAAAAAOCIJAAAA|489|3rd Poplar|Avenue|Suite 70|Springfield|Smith County|MS|59303|United States|-6|condo| +38959|AAAAAAAAPCIJAAAA|648|Williams Sunset|Dr.|Suite M|Plainview|Wabasha County|MN|53683|United States|-6|condo| +38960|AAAAAAAAADIJAAAA|68|Broadway |Court|Suite B|Woodlawn|Giles County|TN|34098|United States|-5|condo| +38961|AAAAAAAABDIJAAAA|768|Locust Madison|Ln|Suite R|Summit|Davie County|NC|20499|United States|-5|single family| +38962|AAAAAAAACDIJAAAA|53|Fourth |Cir.|Suite 200|White Oak|Marion County|FL|36668|United States|-5|single family| +38963|AAAAAAAADDIJAAAA|594||||||||United States||| +38964|AAAAAAAAEDIJAAAA|928|Seventh |Drive|Suite 350|Woodland|Manassas Park city|VA|24854|United States|-5|apartment| +38965|AAAAAAAAFDIJAAAA|773|Hill 12th|Circle|Suite M|Macedonia|Daniels County|MT|61087|United States|-7|single family| +38966|AAAAAAAAGDIJAAAA|190|10th |Pkwy|Suite G|Pleasant Hill|King County|TX|73604|United States|-6|single family| +38967|AAAAAAAAHDIJAAAA|607|Lake |Avenue|Suite J|Hamilton|Hardin County|IA|52808|United States|-6|apartment| +38968|AAAAAAAAIDIJAAAA|973|Miller |Pkwy|Suite G|Cedar Grove|Wasco County|OR|90411|United States|-8|single family| +38969|AAAAAAAAJDIJAAAA|739||Avenue||Milan||TX|||-6|condo| +38970|AAAAAAAAKDIJAAAA|292|3rd |Pkwy|Suite 0|Stratford|Washington County|VT|07268|United States|-5|condo| +38971|AAAAAAAALDIJAAAA|161|Center Willow|Pkwy|Suite 250|Fairfield|Claiborne County|TN|36192|United States|-5|apartment| +38972|AAAAAAAAMDIJAAAA|879|11th 2nd|Avenue|Suite 220|Sleepy Hollow|Fulton County|GA|33592|United States|-5|single family| +38973|AAAAAAAANDIJAAAA|620|Main |Way|Suite 80|Farmington|Stearns County|MN|59145|United States|-6|single family| +38974|AAAAAAAAODIJAAAA|114|Jackson Jackson|ST|Suite D|Pine Grove|Pocahontas County|IA|54593|United States|-6|single family| +38975|AAAAAAAAPDIJAAAA|475|Spring |Road|Suite 110|Forest Hills|Ward County|ND|59237|United States|-6|single family| +38976|AAAAAAAAAEIJAAAA|588|Fourth |Lane|Suite H|Jamestown|Cape May County|NJ|07467|United States|-5|single family| +38977|AAAAAAAABEIJAAAA|176|Willow |Parkway|Suite 200|Bridgeport|La Paz County|AZ|85817|United States|-7|single family| +38978|AAAAAAAACEIJAAAA|742|Cedar |RD|Suite K|Riverview|Sonoma County|CA|99003|United States|-8|single family| +38979|AAAAAAAADEIJAAAA|94|Jackson 13th|Parkway|Suite V|Liberty|Clinton County|MO|63451|United States|-6|condo| +38980|AAAAAAAAEEIJAAAA|374|7th |Parkway|Suite 470|Ashland|Luzerne County|PA|14244|United States|-5|apartment| +38981|AAAAAAAAFEIJAAAA|387|Dogwood |Street|Suite 40|Jamestown|Dallas County|IA|56867|United States|-6|single family| +38982|AAAAAAAAGEIJAAAA|127|Oak |Circle|Suite 230|Unionville|Madison County|KY|41711|United States|-5|apartment| +38983|AAAAAAAAHEIJAAAA|700|Washington |Street|Suite K|Hopewell|Roseau County|MN|50587|United States|-6|single family| +38984|AAAAAAAAIEIJAAAA|635|7th |ST|Suite 140|Walnut|Ferry County|WA|96245|United States|-8|single family| +38985|AAAAAAAAJEIJAAAA|96|3rd Church|ST|Suite X|King|Schoolcraft County|MI|40008|United States|-5|condo| +38986|AAAAAAAAKEIJAAAA|772|Birch Fifth|Cir.|Suite O|Fairview|Midland County|MI|45709|United States|-5|single family| +38987|AAAAAAAALEIJAAAA|40|Smith |Ave|Suite 280|Lakeview|Pottawatomie County|OK|78579|United States|-6|condo| +38988|AAAAAAAAMEIJAAAA|69|5th Spring|Ct.|Suite L|Crystal|Warren County|MS|55258|United States|-6|condo| +38989|AAAAAAAANEIJAAAA|698|Lakeview 7th|Dr.|Suite C|Riverside|Union County|SD|59231|United States|-7|single family| +38990|AAAAAAAAOEIJAAAA|548|Center Elm|Road|Suite M|Lakeside|Dorchester County|SC|29532|United States|-5|apartment| +38991|AAAAAAAAPEIJAAAA|136|Broadway |Wy|Suite J|Plainview|Carteret County|NC|23683|United States|-5|single family| +38992|AAAAAAAAAFIJAAAA|859|Third Eigth|Court|Suite T|Liberty|Johnson County|TX|73451|United States|-6|apartment| +38993|AAAAAAAABFIJAAAA|368|Jefferson |Parkway|Suite T|Oakdale|Dickinson County|MI|49584|United States|-5|condo| +38994|AAAAAAAACFIJAAAA|482|9th |Ct.|Suite 170|New Hope|Lewis County|KY|49431|United States|-5|apartment| +38995|AAAAAAAADFIJAAAA|770|Jefferson |Dr.|Suite J|Pleasant Hill|Finney County|KS|63604|United States|-6|single family| +38996|AAAAAAAAEFIJAAAA|756|Sunset 11th|Court|Suite W|White Oak|Kingfisher County|OK|76668|United States|-6|condo| +38997|AAAAAAAAFFIJAAAA|29|Second 8th|Cir.|Suite P|Oakwood|Ringgold County|IA|50169|United States|-6|single family| +38998|AAAAAAAAGFIJAAAA|572|Willow Oak|Boulevard|Suite 330|Five Forks|Greenlee County|AZ|82293|United States|-7|single family| +38999|AAAAAAAAHFIJAAAA|928|Railroad 13th|Ln|Suite W|Lebanon|Woodbury County|IA|52898|United States|-6|single family| +39000|AAAAAAAAIFIJAAAA|898|Wilson |RD|Suite 210|Shiloh|Allen Parish|LA|79275|United States|-6|apartment| +39001|AAAAAAAAJFIJAAAA|74|Madison Lakeview|Lane|Suite S|Phoenix|Morrison County|MN|52276|United States|-6|apartment| +39002|AAAAAAAAKFIJAAAA|295|View |Circle|Suite 0|Enterprise|Bailey County|TX|71757|United States|-6|condo| +39003|AAAAAAAALFIJAAAA|152|Sycamore |Way|Suite 210|Spring Hill|Powder River County|MT|66787|United States|-7|condo| +39004|AAAAAAAAMFIJAAAA|200|Elm Second|Dr.|Suite V|Wildwood|Evangeline Parish|LA|76871|United States|-6|single family| +39005|AAAAAAAANFIJAAAA|119|Third River|Ln|Suite 60|Forest Hills|Cleburne County|AL|39237|United States|-6|single family| +39006|AAAAAAAAOFIJAAAA|497|Fourth |Avenue|Suite S|Fairfield|Schoolcraft County|MI|46192|United States|-5|condo| +39007|AAAAAAAAPFIJAAAA|477|Central 6th|Avenue|Suite S|Mount Zion|Lawrence County|KY|48054|United States|-5|condo| +39008|AAAAAAAAAGIJAAAA|365|Maple |Drive|Suite 410|Belmont|Gloucester County|NJ|00791|United States|-5|apartment| +39009|AAAAAAAABGIJAAAA|512|Spring |Blvd|Suite P|Springdale|Dallas County|IA|58883|United States|-6|single family| +39010|AAAAAAAACGIJAAAA|529|4th |Parkway|Suite 400|Walnut Grove|Chautauqua County|NY|17752|United States|-5|condo| +39011|AAAAAAAADGIJAAAA|467|Green 8th|Ave|Suite N|Greenfield|DuPage County|IL|65038|United States|-6|condo| +39012|AAAAAAAAEGIJAAAA|339|Sunset |Circle|Suite Q|Bridgeport|Hennepin County|MN|55817|United States|-6|condo| +39013|AAAAAAAAFGIJAAAA|590|Railroad Elm|Parkway|Suite P|Union|Siskiyou County|CA|98721|United States|-8|single family| +39014|AAAAAAAAGGIJAAAA|298|Adams Chestnut|Road|Suite 250|Farmington|Lake of the Woods County|MN|59145|United States|-6|condo| +39015|AAAAAAAAHGIJAAAA|922|Fourth |Cir.|Suite X|Concord|Park County|MT|64107|United States|-7|apartment| +39016|AAAAAAAAIGIJAAAA|981|North |Parkway|Suite 30|Highland|Howard County|IA|59454|United States|-6|single family| +39017|AAAAAAAAJGIJAAAA|104|Jackson |ST|Suite 200|Greenwood|Westchester County|NY|18828|United States|-5|apartment| +39018|AAAAAAAAKGIJAAAA|920|Oak 5th|Cir.|Suite 180|Cherry Valley|Brown County|NE|60854|United States|-6|single family| +39019|AAAAAAAALGIJAAAA|696|South Hill|Street|Suite 190|Lakeside|Clay County|IN|49532|United States|-5|single family| +39020|AAAAAAAAMGIJAAAA|64|Maple 12th|Wy|Suite M|Salem|Etowah County|AL|38048|United States|-6|condo| +39021|AAAAAAAANGIJAAAA|114|Third |Ct.|Suite 50|Walnut Grove|Twiggs County|GA|37752|United States|-5|single family| +39022|AAAAAAAAOGIJAAAA|488|Oak 14th|ST|Suite C|Antioch|Summers County|WV|28605|United States|-5|apartment| +39023|AAAAAAAAPGIJAAAA|120|Third Second|ST|Suite 250|Cumberland|Greene County|AL|38971|United States|-6|apartment| +39024|AAAAAAAAAHIJAAAA|306|Jefferson 1st|Pkwy|Suite 190|New Hope|West Feliciana Parish|LA|79431|United States|-6|apartment| +39025|AAAAAAAABHIJAAAA|392|First |Cir.|Suite 40|Macedonia|Lewis County|WA|91087|United States|-8|apartment| +39026|AAAAAAAACHIJAAAA|738|Lakeview |Parkway|Suite 130|Longwood|Grady County|GA|37021|United States|-5|condo| +39027|AAAAAAAADHIJAAAA|52|1st |Ave|Suite A|New Hope|Armstrong County|TX|79431|United States|-6|condo| +39028|AAAAAAAAEHIJAAAA|430|3rd |Parkway|Suite M|Macedonia|Miller County|MO|61087|United States|-6|condo| +39029|AAAAAAAAFHIJAAAA|242|Franklin |Ln|Suite N|Midway|Menominee County|MI|41904|United States|-5|single family| +39030|AAAAAAAAGHIJAAAA|547|Cedar |Ct.|Suite 30|Harmon|Walla Walla County|WA|95623|United States|-8|single family| +39031|AAAAAAAAHHIJAAAA|768|3rd 7th|Blvd|Suite J|Wildwood|Delaware County|IA|56871|United States|-6|single family| +39032|AAAAAAAAIHIJAAAA|695|West Railroad|Ct.|Suite 190|Deerfield|Lake County|MN|59840|United States|-6|apartment| +39033|AAAAAAAAJHIJAAAA|907|Second Hillcrest|Parkway|Suite J|Shady Grove|Gooding County|ID|82812|United States|-7|apartment| +39034|AAAAAAAAKHIJAAAA|400|Oak 1st|Avenue|Suite 270|Mount Pleasant|Wayne County|TN|31933|United States|-6|apartment| +39035|AAAAAAAALHIJAAAA|961|8th |Wy|Suite B|Union Hill|Fentress County|TN|37746|United States|-5|apartment| +39036|AAAAAAAAMHIJAAAA|675|Hickory 4th|Ave|Suite 290|Cedar|Mercer County|KY|41229|United States|-5|single family| +39037|AAAAAAAANHIJAAAA|647|Seventh First|Ct.|Suite Y|Pleasant Hill|Madison County|MS|53604|United States|-6|apartment| +39038|AAAAAAAAOHIJAAAA|873|Valley Ridge|Ln|Suite 300|Oakwood|Humboldt County|IA|50169|United States|-6|single family| +39039|AAAAAAAAPHIJAAAA|296|Jackson Ninth|Wy|Suite E|Walnut Grove|Hancock County|IA|57752|United States|-6|single family| +39040|AAAAAAAAAIIJAAAA|728|4th |Wy|Suite 120|Springfield|Brooks County|GA|39303|United States|-5|single family| +39041|AAAAAAAABIIJAAAA|371|Sunset Forest|Pkwy|Suite 340|Riverdale|Newton County|TX|79391|United States|-6|apartment| +39042|AAAAAAAACIIJAAAA|250|Highland Maple|Boulevard|Suite 20|Hopewell|Elbert County|GA|30587|United States|-5|apartment| +39043|AAAAAAAADIIJAAAA|314|Hillcrest 3rd|Road|Suite Y|Walnut Grove|Monroe County|MS|57752|United States|-6|apartment| +39044|AAAAAAAAEIIJAAAA|477|12th |Ln|Suite 120|Sunnyside|Jay County|IN|41952|United States|-5|apartment| +39045|AAAAAAAAFIIJAAAA|906|1st Willow|Ave|Suite 0|Farmington|Van Buren County|IA|59145|United States|-6|condo| +39046|AAAAAAAAGIIJAAAA|317|7th Pine|Way|Suite G|Mount Vernon|Jasper County|IL|68482|United States|-6|apartment| +39047|AAAAAAAAHIIJAAAA|961|Williams |Court|Suite 260|Clinton|Polk County|NC|28222|United States|-5|apartment| +39048|AAAAAAAAIIIJAAAA|416|Thirteenth 4th|Circle|Suite 130|Oakdale|Uvalde County|TX|79584|United States|-6|single family| +39049|AAAAAAAAJIIJAAAA|1|Mill Third|Ct.|Suite 470|Providence|Costilla County|CO|86614|United States|-7|single family| +39050|AAAAAAAAKIIJAAAA|800|North |RD|Suite Y|Woodlawn|Caldwell County|MO|64098|United States|-6|apartment| +39051|AAAAAAAALIIJAAAA|283|Chestnut Sixth|Street|Suite 310|Cedar Grove|Park County|MT|60411|United States|-7|single family| +39052|AAAAAAAAMIIJAAAA|504|3rd |Court|Suite O|New Hope|Butte County|ID|89431|United States|-7|apartment| +39053|AAAAAAAANIIJAAAA|240|Sixth Spruce|Boulevard|Suite 310|Waterloo|Gregg County|TX|71675|United States|-6|apartment| +39054|AAAAAAAAOIIJAAAA|731|Ridge |Lane|Suite 160|New Hope|Bartow County|GA|39431|United States|-5|apartment| +39055|AAAAAAAAPIIJAAAA|7|Park |Boulevard|Suite O|Newport|Sweet Grass County|MT|61521|United States|-7|single family| +39056|AAAAAAAAAJIJAAAA|130|Pine |Ln|Suite M|Ashland|Kay County|OK|74244|United States|-6|condo| +39057|AAAAAAAABJIJAAAA|196|Forest Valley|Blvd|Suite Q|Mount Pleasant|Chase County|NE|61933|United States|-6|single family| +39058|AAAAAAAACJIJAAAA|375|Maple |Ln|Suite C|Crossroads|Crook County|WY|80534|United States|-7|condo| +39059|AAAAAAAADJIJAAAA|466|8th Central|Ct.|Suite A|Mount Olive|Lawrence County|KY|48059|United States|-5|condo| +39060|AAAAAAAAEJIJAAAA|879|5th Cedar|Street|Suite M|Woodville|Kaufman County|TX|74289|United States|-6|single family| +39061|AAAAAAAAFJIJAAAA|758|Hillcrest Second|Way|Suite 290|Lebanon|Smyth County|VA|22898|United States|-5|apartment| +39062|AAAAAAAAGJIJAAAA|990|5th Forest|Boulevard|Suite P|Riverside|Denton County|TX|79231|United States|-6|condo| +39063|AAAAAAAAHJIJAAAA|875|14th 2nd||||Warren County|||United States||apartment| +39064|AAAAAAAAIJIJAAAA|442|Madison 5th|Ct.|Suite E|Edgewood|Gentry County|MO|60069|United States|-6|single family| +39065|AAAAAAAAJJIJAAAA|972|Park Wilson|Boulevard|Suite U|White Oak|Lewis County|TN|36668|United States|-6|condo| +39066|AAAAAAAAKJIJAAAA|108|Cedar |Lane|Suite T|Jamestown|Ashland County|WI|56867|United States|-6|condo| +39067|AAAAAAAALJIJAAAA|896|6th |Way|Suite P|Mount Vernon|Russell County|VA|28482|United States|-5|condo| +39068|AAAAAAAAMJIJAAAA|372|4th Dogwood|Lane|Suite M|Glenwood|Norton County|KS|63511|United States|-6|single family| +39069|AAAAAAAANJIJAAAA|898|Cherry |Road|Suite C|Midway|Hamilton County|KS|61904|United States|-6|single family| +39070|AAAAAAAAOJIJAAAA|794|Washington |Drive|Suite 470|Pleasant Valley|Telfair County|GA|32477|United States|-5|apartment| +39071|AAAAAAAAPJIJAAAA|||||||MN|55804|United States||| +39072|AAAAAAAAAKIJAAAA|271|Highland |Cir.|Suite U|Newtown|Queens County|NY|11749|United States|-5|single family| +39073|AAAAAAAABKIJAAAA|278|Miller |Parkway|Suite 340|Crystal|Richland County|OH|45258|United States|-5|apartment| +39074|AAAAAAAACKIJAAAA|428|Pine Lincoln|Ave|Suite 480|Woodland|Coos County|NH|05454|United States|-5|condo| +39075|AAAAAAAADKIJAAAA|787|Willow |Avenue|Suite V|Bunker Hill|Benton County|IA|50150|United States|-6|apartment| +39076|AAAAAAAAEKIJAAAA|800|Davis College|Lane|Suite R|Wilson|Wayne County|MO|66971|United States|-6|condo| +39077|AAAAAAAAFKIJAAAA|910|Maple 10th|Road|Suite 440|Oakdale|Jefferson County|IA|59584|United States|-6|apartment| +39078|AAAAAAAAGKIJAAAA|75|Central Spruce|Circle|Suite 200|Ashland|Crenshaw County|AL|34244|United States|-6|condo| +39079|AAAAAAAAHKIJAAAA|966|Broadway 11th|Lane|Suite V|Union Hill|Taylor County|FL|37746|United States|-5|condo| +39080|AAAAAAAAIKIJAAAA|461|Smith North|Drive|Suite 360|Fairview|Rock County|MN|55709|United States|-6|single family| +39081|AAAAAAAAJKIJAAAA|10|15th 10th|Court|Suite B|Elizabeth|Power County|ID|82935|United States|-7|single family| +39082|AAAAAAAAKKIJAAAA|296|Tenth |Blvd|Suite B|Lakeside|Miller County|GA|39532|United States|-5|single family| +39083|AAAAAAAALKIJAAAA|89|Green |Ct.|Suite 430|Deerfield|Placer County|CA|99840|United States|-8|single family| +39084|AAAAAAAAMKIJAAAA|179|Lake Valley|ST|Suite H|Wright|San Saba County|TX|72814|United States|-6|condo| +39085|AAAAAAAANKIJAAAA|953|9th 1st|Road|Suite M|Providence|Monona County|IA|56614|United States|-6|condo| +39086|AAAAAAAAOKIJAAAA|424|Lakeview Woodland|Court|Suite T|Whispering Pines|Douglas County|NE|67609|United States|-6|condo| +39087|AAAAAAAAPKIJAAAA|505|4th |RD|Suite X|Bunker Hill|Leslie County|KY|40150|United States|-5|apartment| +39088|AAAAAAAAALIJAAAA|563||Road|Suite M|||GA||United States|-5|condo| +39089|AAAAAAAABLIJAAAA|134|6th |Avenue|Suite P|Arcola|Wicomico County|MD|21654|United States|-5|condo| +39090|AAAAAAAACLIJAAAA|5|Hillcrest |Lane|Suite E|Pleasant Grove|Prince George County|MD|24136|United States|-5|single family| +39091|AAAAAAAADLIJAAAA|858|Ash |Road|Suite T|Marion|Ida County|IA|50399|United States|-6|condo| +39092|AAAAAAAAELIJAAAA|189|Forest Broadway|Court|Suite 460|Wildwood|Gloucester County|NJ|07471|United States|-5|apartment| +39093|AAAAAAAAFLIJAAAA|928|Ridge |RD|Suite O|Clinton|Van Wert County|OH|48222|United States|-5|single family| +39094|AAAAAAAAGLIJAAAA|972|Lake |Dr.|Suite H|Avoca|Sac County|IA|50540|United States|-6|single family| +39095|AAAAAAAAHLIJAAAA|718|Sixth |Way|Suite 330|Deerfield|Bossier Parish|LA|79840|United States|-6|single family| +39096|AAAAAAAAILIJAAAA|923|Miller 8th|Court|Suite I|Chestnut Ridge|Oklahoma County|OK|77334|United States|-6|apartment| +39097|AAAAAAAAJLIJAAAA|236|Cedar |ST|||||34098||-5|single family| +39098|AAAAAAAAKLIJAAAA|306|Third |Ln|Suite L|Antioch|Walthall County|MS|58605|United States|-6|condo| +39099|AAAAAAAALLIJAAAA|231|Walnut |Street|Suite W|Greenville|Houston County|GA|31387|United States|-5|condo| +39100|AAAAAAAAMLIJAAAA|766|River |ST|Suite S|Riverside|Jasper County|TX|79231|United States|-6|apartment| +39101|AAAAAAAANLIJAAAA||1st Lake|Court|||||29237|||apartment| +39102|AAAAAAAAOLIJAAAA|805|Jefferson |Pkwy|Suite 400|Pleasant Hill|Sabine Parish|LA|73604|United States|-6|condo| +39103|AAAAAAAAPLIJAAAA|197|2nd |Blvd|Suite A|Woodland|Tishomingo County|MS|54854|United States|-6|condo| +39104|AAAAAAAAAMIJAAAA|||Wy|Suite 310|||MS|56787|||condo| +39105|AAAAAAAABMIJAAAA|144|Spring |Lane|Suite 420|Brownsville|Live Oak County|TX|79310|United States|-6|apartment| +39106|AAAAAAAACMIJAAAA|558|5th Hill|Ave|Suite 470|Ashland|Smith County|MS|54244|United States|-6|single family| +39107|AAAAAAAADMIJAAAA|434|Woodland |Court|Suite V|Kingston|Jones County|GA|34975|United States|-5|single family| +39108|AAAAAAAAEMIJAAAA|517|Second |Boulevard|Suite 10|Riverview|Pender County|NC|29003|United States|-5|condo| +39109|AAAAAAAAFMIJAAAA|145|West |Wy|Suite 300|Fairfield|Kewaunee County|WI|56192|United States|-6|condo| +39110|AAAAAAAAGMIJAAAA|296|Spring Church|Cir.|Suite 400|Riverview|Somervell County|TX|79003|United States|-6|condo| +39111|AAAAAAAAHMIJAAAA|697|7th 1st|Blvd|Suite V|Mount Vernon|Hood County|TX|78482|United States|-6|condo| +39112|AAAAAAAAIMIJAAAA|852|Valley Oak|RD|Suite R|Stringtown|Johnson County|AR|70162|United States|-6|condo| +39113|AAAAAAAAJMIJAAAA|990|Pine Park|Circle|Suite 160|Friendship|Oldham County|KY|44536|United States|-5|condo| +39114|AAAAAAAAKMIJAAAA|315|Cherry Cedar|Ave|Suite C|Kingston|Hutchinson County|TX|74975|United States|-6|apartment| +39115|AAAAAAAALMIJAAAA|819|Franklin |Lane|Suite C|Centerville|Thomas County|KS|60059|United States|-6|condo| +39116|AAAAAAAAMMIJAAAA|785|15th Madison|Dr.|Suite W|Summit|Oneida County|ID|80499|United States|-7|condo| +39117|AAAAAAAANMIJAAAA|351|West |Dr.|Suite 50|Collinsville|Monongalia County|WV|22459|United States|-5|apartment| +39118|AAAAAAAAOMIJAAAA|534|Valley Highland|Avenue|Suite I|Sulphur Springs|Gulf County|FL|38354|United States|-5|condo| +39119|AAAAAAAAPMIJAAAA|885|Church |Street|Suite 290|Newport|Cottonwood County|MN|51521|United States|-6|condo| +39120|AAAAAAAAANIJAAAA|505|Oak Center|Cir.|Suite 220|Lakeview|Rhea County|TN|38579|United States|-6|condo| +39121|AAAAAAAABNIJAAAA|335|1st South|Drive|Suite 340|Lebanon|Webster Parish|LA|72898|United States|-6|single family| +39122|AAAAAAAACNIJAAAA|753|Pine Elm|Cir.|Suite 230|Forest Hills|Fairfield County|OH|49237|United States|-5|single family| +39123|AAAAAAAADNIJAAAA||||Suite 310|Oakdale|Polk County|TX|79584|United States||apartment| +39124|AAAAAAAAENIJAAAA|799|Spruce |Ct.|Suite 0|Oak Grove|Oliver County|ND|58370|United States|-6|single family| +39125|AAAAAAAAFNIJAAAA|624|Williams Park|Ln|Suite 180|Woodland|Aransas County|TX|74854|United States|-6|apartment| +39126|AAAAAAAAGNIJAAAA|735|Lee Second|Ln||Kingston|San Mateo County|||United States||apartment| +39127|AAAAAAAAHNIJAAAA|133|Green Washington|Parkway|Suite J|Sutton|White County|AR|75413|United States|-6|apartment| +39128|AAAAAAAAINIJAAAA|927|Laurel |Ct.|Suite D|Oak Grove|Nelson County|ND|58370|United States|-6|apartment| +39129|AAAAAAAAJNIJAAAA|368|8th 10th|Ct.|Suite N|Hopewell|Clifton Forge city|VA|20587|United States|-5|apartment| +39130|AAAAAAAAKNIJAAAA|26|6th Fourth|Lane|Suite 410|Glenwood|Buena Vista County|IA|53511|United States|-6|single family| +39131|AAAAAAAALNIJAAAA|648|Main Maple|Street|Suite E|Kingston|Terrell County|GA|34975|United States|-5|single family| +39132|AAAAAAAAMNIJAAAA|349|Walnut Cherry|Ave|Suite 150|Maple Grove|Yukon-Koyukuk Census Area|AK|98252|United States|-9|apartment| +39133|AAAAAAAANNIJAAAA|771||Ct.||Greenfield|Perkins County|||United States|-7|| +39134|AAAAAAAAONIJAAAA|180|Maple First|ST|Suite 340|Oakdale|Bladen County|NC|29584|United States|-5|apartment| +39135|AAAAAAAAPNIJAAAA|163|Second Cherry|Cir.|Suite 120|Maywood|Tehama County|CA|95681|United States|-8|condo| +39136|AAAAAAAAAOIJAAAA|64|Madison |Circle|Suite V|Midway|Sumter County|SC|21904|United States|-5|apartment| +39137|AAAAAAAABOIJAAAA|576|Sycamore |Way|Suite 30|Greenfield|Pawnee County|OK|75038|United States|-6|apartment| +39138|AAAAAAAACOIJAAAA|663|4th |Lane|Suite E|Sunnyside|Sumter County|SC|21952|United States|-5|apartment| +39139|AAAAAAAADOIJAAAA|257|West Third|Lane|Suite 200|Valley View|Providence County|RI|05724|United States|-5|apartment| +39140|AAAAAAAAEOIJAAAA|306|Walnut Meadow|Ave|Suite 410|Greenwood|Dorchester County|SC|28828|United States|-5|condo| +39141|AAAAAAAAFOIJAAAA|564|Sixth Park||||Jackson County||26549|United States||| +39142|AAAAAAAAGOIJAAAA|923|Lincoln Seventh|Street|Suite 350|Marion|Putnam County|FL|30399|United States|-5|condo| +39143|AAAAAAAAHOIJAAAA|543|Mill |Court|Suite V|Springdale|Coconino County|AZ|88883|United States|-7|apartment| +39144|AAAAAAAAIOIJAAAA|122|9th |Boulevard|Suite 90|Fairview|James City County|VA|25709|United States|-5|apartment| +39145|AAAAAAAAJOIJAAAA|798|Woodland |Blvd|Suite A|Fairview|Tallahatchie County|MS|55709|United States|-6|apartment| +39146|AAAAAAAAKOIJAAAA|810|Cherry |Cir.|Suite 390|Shannon|Grundy County|TN|34120|United States|-5|apartment| +39147|AAAAAAAALOIJAAAA|787|Locust Walnut|Dr.|Suite R|Walnut Grove|Shelby County|AL|37752|United States|-6|condo| +39148|AAAAAAAAMOIJAAAA|656|2nd |RD|Suite 10|Springdale|Montgomery County|TX|78883|United States|-6|single family| +39149|AAAAAAAANOIJAAAA|732|4th |Court|Suite T|Woodville|Colonial Heights city|VA|24289|United States|-5|condo| +39150|AAAAAAAAOOIJAAAA|894|East |Drive|Suite 80|Wildwood|Lynchburg city|VA|26871|United States|-5|apartment| +39151|AAAAAAAAPOIJAAAA|289|Sunset |Ct.|Suite 260|Newport|Martin County|KY|41521|United States|-5|condo| +39152|AAAAAAAAAPIJAAAA|767|Valley Laurel|Parkway|Suite 390|Lakeside|Lorain County|OH|49532|United States|-5|apartment| +39153|AAAAAAAABPIJAAAA|679|Laurel |Court|Suite P|Summit|Conejos County|CO|80499|United States|-7|apartment| +39154|AAAAAAAACPIJAAAA|154|Dogwood Church|RD|Suite 200|Greenfield|Randolph County|IN|45038|United States|-5|apartment| +39155|AAAAAAAADPIJAAAA|33|Lincoln 6th|Road|Suite P|Forest Hills|Sullivan County|PA|19237|United States|-5|condo| +39156|AAAAAAAAEPIJAAAA|735|Chestnut |Dr.|Suite T|Stringtown|Brule County|SD|50162|United States|-6|condo| +39157|AAAAAAAAFPIJAAAA|596|Sunset |Avenue|Suite J|Riverview|Greene County|IN|49003|United States|-5|condo| +39158|AAAAAAAAGPIJAAAA|253|Laurel View|Blvd|Suite 270|Bethel|Lee County|AR|75281|United States|-6|apartment| +39159|AAAAAAAAHPIJAAAA|768|Washington Davis|Wy|Suite C|Lebanon|Deuel County|SD|52898|United States|-6|apartment| +39160|AAAAAAAAIPIJAAAA|725|Dogwood |Ct.|Suite 160|Summit|Garfield County|CO|80499|United States|-7|single family| +39161|AAAAAAAAJPIJAAAA|216|Lee |Boulevard|Suite 190|Glenwood|Collingsworth County|TX|73511|United States|-6|single family| +39162|AAAAAAAAKPIJAAAA|371|West 4th|Cir.|Suite D|Clifton|Richmond County|GA|38014|United States|-5|apartment| +39163|AAAAAAAALPIJAAAA|34|Fifth Washington|Court|Suite 240|Forest Hills|Lawrence County|TN|39237|United States|-6|single family| +39164|AAAAAAAAMPIJAAAA|201|6th First|Lane|Suite U|Oakdale|Lincoln County|AR|79584|United States|-6|single family| +39165|AAAAAAAANPIJAAAA|301|Eigth |Ave|Suite O|Lakewood|Ellis County|OK|78877|United States|-6|condo| +39166|AAAAAAAAOPIJAAAA|898|Jackson 1st|Street|Suite 450|Lakewood|Ector County|TX|78877|United States|-6|condo| +39167|AAAAAAAAPPIJAAAA|137|11th Washington|Pkwy|Suite M|Lewis|Nelson County|KY|47066|United States|-5|condo| +39168|AAAAAAAAAAJJAAAA|841|Willow |Cir.|Suite 460|Bunker Hill|Benzie County|MI|40150|United States|-5|condo| +39169|AAAAAAAABAJJAAAA|307|First 10th|ST|Suite E|Union|Alpena County|MI|48721|United States|-5|single family| +39170|AAAAAAAACAJJAAAA|654|1st Williams|Ln|Suite 50|Fairfield|Sauk County|WI|56192|United States|-6|single family| +39171|AAAAAAAADAJJAAAA|974|||Suite N|Wilson|Grand Traverse County||46971||-5|| +39172|AAAAAAAAEAJJAAAA|835|10th Chestnut|Street|Suite K|Midway|Dorchester County|MD|21904|United States|-5|single family| +39173|AAAAAAAAFAJJAAAA|||Pkwy||Rolling Hills||||||| +39174|AAAAAAAAGAJJAAAA|653|College Jackson|Ave|Suite 480|Fairfield|Crenshaw County|AL|36192|United States|-6|condo| +39175|AAAAAAAAHAJJAAAA|168|East Locust|Avenue|Suite 140|Cedar Grove|San Miguel County|CO|80411|United States|-7|single family| +39176|AAAAAAAAIAJJAAAA|76|East |Court|Suite U|Fairbanks|Kalkaska County|MI|46653|United States|-5|single family| +39177|AAAAAAAAJAJJAAAA|22|11th Franklin|Boulevard|Suite 120|Spring Valley|Nevada County|AR|76060|United States|-6|single family| +39178|AAAAAAAAKAJJAAAA|787|Church |Drive|Suite J|Glendale|Jackson County|AL|33951|United States|-6|condo| +39179|AAAAAAAALAJJAAAA|358|Lincoln |Ct.|Suite 420|Georgetown|McCreary County|KY|47057|United States|-5|condo| +39180|AAAAAAAAMAJJAAAA|550|Railroad Miller|Ave|Suite 420|Forest Hills|Bartow County|GA|39237|United States|-5|condo| +39181|AAAAAAAANAJJAAAA|713|||Suite G|||IA|54244|United States||| +39182|AAAAAAAAOAJJAAAA|518|10th West|Drive|Suite M|Wildwood|Jasper County|IA|56871|United States|-6|apartment| +39183|AAAAAAAAPAJJAAAA|676|Miller Pine|Avenue|Suite Q|Lakewood|Box Butte County|NE|68877|United States|-6|apartment| +39184|AAAAAAAAABJJAAAA||Pine |ST|||||||-6|condo| +39185|AAAAAAAABBJJAAAA|332|Fifteenth |Circle|Suite Q|Antioch|Alamosa County|CO|88605|United States|-7|apartment| +39186|AAAAAAAACBJJAAAA|178|View |Lane|Suite S|Argyle|Morrison County|MN|58722|United States|-6|single family| +39187|AAAAAAAADBJJAAAA|451|Central |Street|Suite O|Summit|Iroquois County|IL|60499|United States|-6|condo| +39188|AAAAAAAAEBJJAAAA|742|Miller |Pkwy|Suite 360|Harmony|Eaton County|MI|45804|United States|-5|condo| +39189|AAAAAAAAFBJJAAAA|456|Church Fifth|Cir.|Suite 300|Farmington|Wibaux County|MT|69145|United States|-7|condo| +39190|AAAAAAAAGBJJAAAA|495|1st Ridge|Drive|Suite T|Oak Grove|Tyrrell County|NC|28370|United States|-5|condo| +39191|AAAAAAAAHBJJAAAA|109|7th |Pkwy|Suite B|Cordova|Ohio County|KY|46938|United States|-5|single family| +39192|AAAAAAAAIBJJAAAA|34|2nd |Wy|Suite A|Farmington|Fulton County|PA|19145|United States|-5|single family| +39193|AAAAAAAAJBJJAAAA|713|12th River|Ave|Suite 490|Springfield|Gladwin County|MI|49303|United States|-5|apartment| +39194|AAAAAAAAKBJJAAAA|35|Hill |Way|Suite R|Cedar Grove|Lee County|TX|70411|United States|-6|apartment| +39195|AAAAAAAALBJJAAAA|580|Birch Maple|Lane|Suite 180|Hopewell|Lexington County|SC|20587|United States|-5|single family| +39196|AAAAAAAAMBJJAAAA|697|Park 3rd|Street|Suite K|Oakwood|Reeves County|TX|70169|United States|-6|condo| +39197|AAAAAAAANBJJAAAA|125|Sunset |Boulevard|Suite X|Woodlawn|Alpine County|CA|94098|United States|-8|single family| +39198|AAAAAAAAOBJJAAAA|758|8th |RD|Suite 330|Pleasant Grove|Kendall County|IL|64136|United States|-6|apartment| +39199|AAAAAAAAPBJJAAAA|594|Wilson Sunset|Boulevard|Suite C|Shady Grove|Marion County|AL|32812|United States|-6|condo| +39200|AAAAAAAAACJJAAAA|962|Smith |Parkway|Suite 300|Liberty|Ascension Parish|LA|73451|United States|-6|apartment| +39201|AAAAAAAABCJJAAAA|874|Lee |Wy|Suite T|Belmont|Roane County|WV|20191|United States|-5|condo| +39202|AAAAAAAACCJJAAAA|360|Fifth |Court|Suite V|Marion|De Kalb County|IN|40399|United States|-5|single family| +39203|AAAAAAAADCJJAAAA|511|River |RD|Suite H|Sulphur Springs|Jefferson Parish|LA|78354|United States|-6|apartment| +39204|AAAAAAAAECJJAAAA|283|Jefferson |Ave|Suite L|Shiloh|Arenac County|MI|49275|United States|-5|condo| +39205|AAAAAAAAFCJJAAAA|702|Washington Broadway|Drive|Suite U|Belmont|Sullivan County|PA|10191|United States|-5|condo| +39206|AAAAAAAAGCJJAAAA|724|8th 5th|Road|Suite 0|Pleasant Hill|Hughes County|SD|53604|United States|-7|condo| +39207|AAAAAAAAHCJJAAAA|642|Railroad North|RD|Suite N|Greenville|Yellowstone County|MT|61387|United States|-7|condo| +39208|AAAAAAAAICJJAAAA|100|Sunset Cherry|Ct.|Suite V|Union Hill|Calloway County|KY|47746|United States|-6|apartment| +39209|AAAAAAAAJCJJAAAA|71|6th Franklin|Parkway|Suite B|Pinecrest|Preston County|WV|29981|United States|-5|single family| +39210|AAAAAAAAKCJJAAAA|||Ln|||Middlesex County||05519|||condo| +39211|AAAAAAAALCJJAAAA|929|3rd Main|Parkway|Suite D|Lakewood|Beadle County|SD|58877|United States|-6|apartment| +39212|AAAAAAAAMCJJAAAA||Center Oak|Blvd|||Washington County||||-6|single family| +39213|AAAAAAAANCJJAAAA|568|Eigth |Boulevard|Suite D|Brownsville|Finney County|KS|69310|United States|-6|apartment| +39214|AAAAAAAAOCJJAAAA|659|College Willow|Way|Suite Q|Mount Olive|Republic County|KS|68059|United States|-6|single family| +39215|AAAAAAAAPCJJAAAA|90|Main Walnut|Ct.|Suite Q|Brownsville|Richland County|WI|59310|United States|-6|condo| +39216|AAAAAAAAADJJAAAA|24|12th |Drive|Suite 440|Union Hill|Hood County|TX|77746|United States|-6|condo| +39217|AAAAAAAABDJJAAAA|972|Walnut |Road|Suite U|Oakdale|Mercer County|OH|49584|United States|-5|single family| +39218|AAAAAAAACDJJAAAA|978|Maple College|Court|Suite A|Willis|Valley County|ID|86788|United States|-7|single family| +39219|AAAAAAAADDJJAAAA|824|Hillcrest |Ct.|Suite 340|Stringtown|Butts County|GA|30162|United States|-5|condo| +39220|AAAAAAAAEDJJAAAA|850|Thirteenth Woodland|Cir.|Suite 190|Deerfield|Cameron County|TX|79840|United States|-6|apartment| +39221|AAAAAAAAFDJJAAAA|641|Adams 1st|Pkwy|Suite 230|Highland Park|Pike County|KY|46534|United States|-5|single family| +39222|AAAAAAAAGDJJAAAA|758|Third |ST|Suite 10|Mount Vernon|Washington County|NC|28482|United States|-5|apartment| +39223|AAAAAAAAHDJJAAAA|990|Johnson Washington|Avenue|Suite 490|Cedar Grove|Kanawha County|WV|20411|United States|-5|apartment| +39224|AAAAAAAAIDJJAAAA|128|7th |Wy|Suite R|Fairview|Cass County|TX|75709|United States|-6|condo| +39225|AAAAAAAAJDJJAAAA|826|7th 7th|Road|Suite G|Flint|Clay County|IA|58909|United States|-6|single family| +39226|AAAAAAAAKDJJAAAA|90|Washington |ST|Suite O|Newport|Lee County|TX|71521|United States|-6|apartment| +39227|AAAAAAAALDJJAAAA|523|5th |Street|Suite J|Lakewood|McCurtain County|OK|78877|United States|-6|condo| +39228|AAAAAAAAMDJJAAAA|419|View |Parkway|Suite M|Oak Ridge|Bandera County|TX|78371|United States|-6|apartment| +39229|AAAAAAAANDJJAAAA|987|College 6th|Avenue|Suite 310|Wilson|Wilkes County|NC|26971|United States|-5|single family| +39230|AAAAAAAAODJJAAAA|215|9th View|Ave|Suite 390|Belmont|Howard County|NE|60191|United States|-7|apartment| +39231|AAAAAAAAPDJJAAAA|974|Main Jackson|RD|Suite 340|Spring Hill|Camden County|NC|26787|United States|-5|apartment| +39232|AAAAAAAAAEJJAAAA|782|3rd |Blvd|Suite A|Highland Park|Clinton County|IL|66534|United States|-6|single family| +39233|AAAAAAAABEJJAAAA|746|South |Ave|Suite A|Oak Grove|Hockley County|TX|78370|United States|-6|apartment| +39234|AAAAAAAACEJJAAAA|5|5th Hickory|Ln|Suite R|Mount Vernon|Rockwall County|TX|78482|United States|-6|single family| +39235|AAAAAAAADEJJAAAA|412|Oak |RD|Suite 120|Fairview|Northampton County|PA|15709|United States|-5|single family| +39236|AAAAAAAAEEJJAAAA|643|3rd Ash|||Friendship||AK|94536||-9|| +39237|AAAAAAAAFEJJAAAA|53|Adams Oak|Way|Suite 360|Fayetteville|Perkins County|SD|51732|United States|-7|single family| +39238|AAAAAAAAGEJJAAAA|189|Johnson |Street|Suite N|Sunnyside|Sanpete County|UT|81952|United States|-7|condo| +39239|AAAAAAAAHEJJAAAA|26|Locust Williams|Cir.|Suite Y|Sumner|Morgan County|TN|30519|United States|-6|single family| +39240|AAAAAAAAIEJJAAAA|635|2nd |Circle|Suite R|White Oak|Polk County|NE|66668|United States|-7|condo| +39241|AAAAAAAAJEJJAAAA|947|First |RD|Suite Y|Buena Vista|Chaves County|NM|85752|United States|-7|apartment| +39242|AAAAAAAAKEJJAAAA|616|Spring |ST|Suite N|Highland Park|San Jacinto County|TX|76534|United States|-6|apartment| +39243|AAAAAAAALEJJAAAA|637|4th Walnut|Wy|Suite 420|Five Points|Hyde County|SD|56098|United States|-7|apartment| +39244|AAAAAAAAMEJJAAAA|304|Willow |Dr.|Suite 270|Wesley|Sacramento County|CA|91218|United States|-8|apartment| +39245|AAAAAAAANEJJAAAA|674|Sycamore |Wy|Suite W|Lakewood|Pitt County|NC|28877|United States|-5|apartment| +39246|AAAAAAAAOEJJAAAA|569|1st |Drive|Suite A|Lakeview|Columbia County|NY|18579|United States|-5|single family| +39247|AAAAAAAAPEJJAAAA|75|Ridge Smith|Drive|Suite 280|Plainview|Carroll County|MD|23683|United States|-5|apartment| +39248|AAAAAAAAAFJJAAAA|273|Hill |Way|Suite R|Mount Vernon|Dodge County|MN|58482|United States|-6|single family| +39249|AAAAAAAABFJJAAAA|669|Center |Pkwy|Suite 220|Friendship|Holmes County|FL|34536|United States|-5|single family| +39250|AAAAAAAACFJJAAAA|36|Oak |Cir.|Suite J|Newport|Winona County|MN|51521|United States|-6|condo| +39251|AAAAAAAADFJJAAAA|249|Ninth |Parkway|Suite G|Indian Village|Guadalupe County|NM|81075|United States|-7|apartment| +39252|AAAAAAAAEFJJAAAA|169|Lee |Ln|Suite 160|Pleasant Grove|Sandusky County|OH|44136|United States|-5|condo| +39253|AAAAAAAAFFJJAAAA|381|6th Lincoln|Circle|Suite 60|Forest Hills|Greene County|IN|49237|United States|-5|condo| +39254|AAAAAAAAGFJJAAAA|972|Second |Court|Suite 140|Clifton|Jefferson County|WI|58014|United States|-6|single family| +39255|AAAAAAAAHFJJAAAA|514|1st Elm|Way|Suite R|Freeport|Gentry County|MO|61844|United States|-6|apartment| +39256|AAAAAAAAIFJJAAAA|508|River |Court|Suite T|Greenwood|Edgar County|IL|68828|United States|-6|apartment| +39257|AAAAAAAAJFJJAAAA|507|Second Dogwood|Blvd|Suite 400|Mount Pleasant|Danville city|VA|21933|United States|-5|single family| +39258|AAAAAAAAKFJJAAAA|764|Center |Pkwy|Suite 380|Oakdale|Wilson County|KS|69584|United States|-6|single family| +39259|AAAAAAAALFJJAAAA|212|Park Elm|Way|Suite Y|Riverdale|Powell County|KY|49391|United States|-5|apartment| +39260|AAAAAAAAMFJJAAAA|31|South 15th|Circle|Suite 190|Five Points|Taylor County|WV|26098|United States|-5|single family| +39261|AAAAAAAANFJJAAAA|947|8th |Wy|Suite K|Concord|Johnson County|TX|74107|United States|-6|condo| +39262|AAAAAAAAOFJJAAAA|89|Adams Cedar|RD|Suite 420|Green Acres|Franklin County|KS|67683|United States|-6|apartment| +39263|AAAAAAAAPFJJAAAA|512|East |Dr.|Suite 450|Springfield|Anderson County|SC|29303|United States|-5|condo| +39264|AAAAAAAAAGJJAAAA|501|Mill |ST|Suite B|Bennett|Garza County|TX|71715|United States|-6|condo| +39265|AAAAAAAABGJJAAAA|374|Cedar |Court|Suite 180|White Oak|Calvert County|MD|26668|United States|-5|condo| +39266|AAAAAAAACGJJAAAA|213|East |Ct.|Suite W|Riverdale|Butler County|AL|39391|United States|-6|condo| +39267|AAAAAAAADGJJAAAA|83|2nd |Drive|Suite C|Hamilton|Meade County|KS|62808|United States|-6|single family| +39268|AAAAAAAAEGJJAAAA|792|Third Hillcrest|Blvd|Suite Q|Springdale|Shelby County|AL|38883|United States|-6|apartment| +39269|AAAAAAAAFGJJAAAA|776|Walnut |Avenue|Suite 170|Spring Valley|Renville County|MN|56060|United States|-6|apartment| +39270|AAAAAAAAGGJJAAAA|491|7th Poplar|Court|Suite 410|Sunnyside|Brunswick County|NC|21952|United States|-5|single family| +39271|AAAAAAAAHGJJAAAA|409|Cedar Birch|Circle|Suite 30|Jackson|Union County|NJ|09583|United States|-5|condo| +39272|AAAAAAAAIGJJAAAA|514|8th |Ave|Suite 340|Ashland|Prince Edward County|VA|24244|United States|-5|single family| +39273|AAAAAAAAJGJJAAAA|983|6th Meadow|Street|Suite 280|Mount Zion|Edwards County|KS|68054|United States|-6|apartment| +39274|AAAAAAAAKGJJAAAA|221|Franklin |Pkwy|Suite 340|Five Points|Floyd County|KY|46098|United States|-6|apartment| +39275|AAAAAAAALGJJAAAA|920|Locust Cherry|Wy|Suite 490|Friendship|Hood River County|OR|94536|United States|-8|condo| +39276|AAAAAAAAMGJJAAAA|156|View |Parkway|Suite 490|Mount Vernon|Henry County|KY|48482|United States|-6|apartment| +39277|AAAAAAAANGJJAAAA|32|Lake Lincoln|Wy|Suite 30|Harmony|Wade Hampton Census Area|AK|95804|United States|-9|single family| +39278|AAAAAAAAOGJJAAAA|506|Jefferson Central|Boulevard|Suite P|White Oak|Bristol Bay Borough|AK|96668|United States|-9|apartment| +39279|AAAAAAAAPGJJAAAA|959|13th Locust|Blvd|Suite 0|Mount Pleasant|Rockdale County|GA|31933|United States|-5|condo| +39280|AAAAAAAAAHJJAAAA|914|Main Williams|Blvd|Suite 460|Franklin|Middlesex County|VA|29101|United States|-5|single family| +39281|AAAAAAAABHJJAAAA|170|First |Parkway|Suite R|Sawyer|Vigo County|IN|46045|United States|-5|apartment| +39282|AAAAAAAACHJJAAAA|181|Laurel |ST|Suite 70|Stringtown|Zavala County|TX|70162|United States|-6|apartment| +39283|AAAAAAAADHJJAAAA|353|1st 12th|ST|Suite N|Glendale|Beckham County|OK|73951|United States|-6|condo| +39284|AAAAAAAAEHJJAAAA|716|Main |Dr.|Suite T|Mountain View|Haywood County|TN|34466|United States|-5|single family| +39285|AAAAAAAAFHJJAAAA|687|Maple 8th|Drive|Suite D|Macedonia|Campbell County|KY|41087|United States|-6|single family| +39286|AAAAAAAAGHJJAAAA|539|||Suite Q|Sulphur Springs|Nicholas County|KY|48354|United States||| +39287|AAAAAAAAHHJJAAAA|71|10th |Way|Suite H|Forestville|Knox County|IL|63027|United States|-6|condo| +39288|AAAAAAAAIHJJAAAA||15th Williams||Suite 60|Fairview||MT||||| +39289|AAAAAAAAJHJJAAAA|257|3rd South|Road|Suite 300|Oakdale|Van Wert County|OH|49584|United States|-5|single family| +39290|AAAAAAAAKHJJAAAA|910|Lee 11th|Circle|Suite 0|Tremont|Barber County|KS|69515|United States|-6|condo| +39291|AAAAAAAALHJJAAAA|128|Davis Franklin|RD|Suite 230|Greenwood|Montour County|PA|18828|United States|-5|single family| +39292|AAAAAAAAMHJJAAAA|47|3rd |Boulevard|Suite G|Fairfield|Red River Parish|LA|76192|United States|-6|single family| +39293|AAAAAAAANHJJAAAA|901|Pine |Court|Suite M|Kingston|Wilkes County|NC|24975|United States|-5|apartment| +39294|AAAAAAAAOHJJAAAA|348|Adams Central|Way|Suite 350|Greenfield|Clatsop County|OR|95038|United States|-8|condo| +39295|AAAAAAAAPHJJAAAA|425|Maple Second|Boulevard|Suite 410|Perkins|Washington County|KS|61852|United States|-6|single family| +39296|AAAAAAAAAIJJAAAA|915|College |Street|Suite I|Kimball|La Salle County|TX|73595|United States|-6|single family| +39297|AAAAAAAABIJJAAAA|82|Pine |Boulevard|Suite D|Mountain View|Weld County|CO|84466|United States|-7|apartment| +39298|AAAAAAAACIJJAAAA|367|8th |Lane|Suite 420|Stringtown|Pottawatomie County|KS|60162|United States|-6|single family| +39299|AAAAAAAADIJJAAAA|215|Laurel |Cir.|Suite 410|Five Points|Barren County|KY|46098|United States|-6|single family| +39300|AAAAAAAAEIJJAAAA|419|7th |Circle|Suite B|Enterprise|Las Animas County|CO|81757|United States|-7|single family| +39301|AAAAAAAAFIJJAAAA|807|5th Spruce|ST|Suite 330|Riverview|Gulf County|FL|39003|United States|-5|apartment| +39302|AAAAAAAAGIJJAAAA|147|Broadway Second|Ct.|Suite 350|Union Hill|Mercer County|MO|67746|United States|-6|apartment| +39303|AAAAAAAAHIJJAAAA|705|Washington Hill|Cir.|Suite H|Franklin|Effingham County|IL|69101|United States|-6|condo| +39304|AAAAAAAAIIJJAAAA|680|Jefferson Adams|||||AR||United States|-6|| +39305|AAAAAAAAJIJJAAAA|181|Woodland Poplar|Court|Suite L|Forest Hills|McHenry County|ND|59237|United States|-6|condo| +39306|AAAAAAAAKIJJAAAA|134|2nd |Pkwy|Suite 130|Birmingham|Washington County|OH|43372|United States|-5|single family| +39307|AAAAAAAALIJJAAAA|608|Main Smith|Street|Suite M|Hillcrest|Roanoke city|VA|23003|United States|-5|condo| +39308|AAAAAAAAMIJJAAAA|||Way||Cedar Grove||PA||United States||condo| +39309|AAAAAAAANIJJAAAA|619|Jefferson |Boulevard|Suite 240|Woodlawn|Charles Mix County|SD|54098|United States|-6|single family| +39310|AAAAAAAAOIJJAAAA|438|Franklin |Boulevard|Suite K|Hillcrest|Essex County|VT|03603|United States|-5|apartment| +39311|AAAAAAAAPIJJAAAA|179|Cherry Dogwood|RD|Suite 480|Salem|Young County|TX|78048|United States|-6|condo| +39312|AAAAAAAAAJJJAAAA|99|5th Lakeview|Court|Suite K|Five Forks|Emanuel County|GA|32293|United States|-5|apartment| +39313|AAAAAAAABJJJAAAA|102|10th Pine|Boulevard|Suite 380|Marion|Seminole County|OK|70399|United States|-6|apartment| +39314|AAAAAAAACJJJAAAA|525|Park Sycamore|Cir.|Suite N|Edgewood|Graham County|NC|20069|United States|-5|single family| +39315|AAAAAAAADJJJAAAA|812|Lake 7th|Court|Suite 290|Greenfield|Nobles County|MN|55038|United States|-6|single family| +39316|AAAAAAAAEJJJAAAA|282|Jefferson Seventh|Street|Suite 450|Spring Hill|Greenville County|SC|26787|United States|-5|condo| +39317|AAAAAAAAFJJJAAAA|851|Elm |Blvd|Suite K|Shiloh|Crawford County|KS|69275|United States|-6|single family| +39318|AAAAAAAAGJJJAAAA|241|Valley |Wy|Suite W|Frenchtown|Goliad County|TX|72629|United States|-6|condo| +39319|AAAAAAAAHJJJAAAA|394|Laurel |Way|Suite 130|Jackson|Pershing County|NV|89583|United States|-8|condo| +39320|AAAAAAAAIJJJAAAA|451|Maple |Circle|Suite Q|Five Points|Moody County|SD|56098|United States|-7|condo| +39321|AAAAAAAAJJJJAAAA|283|Franklin |Ct.|Suite R|Oakland|Early County|GA|39843|United States|-5|single family| +39322|AAAAAAAAKJJJAAAA|219|Jefferson |Wy|Suite B|Stafford|Buckingham County|VA|24980|United States|-5|apartment| +39323|AAAAAAAALJJJAAAA|273|Washington Washington|Ln|Suite 40|Wildwood|Traill County|ND|56871|United States|-6|apartment| +39324|AAAAAAAAMJJJAAAA|186|6th Valley|Boulevard|Suite M|Greenfield|Livingston County|MO|65038|United States|-6|condo| +39325|AAAAAAAANJJJAAAA|365|Meadow Hill|Circle|Suite 300|Enterprise|Waller County|TX|71757|United States|-6|condo| +39326|AAAAAAAAOJJJAAAA|340|Sycamore Spruce|Ct.|Suite P|Oneida|Macon County|TN|34027|United States|-6|single family| +39327|AAAAAAAAPJJJAAAA|800|Mill North|Avenue|Suite M|Liberty|Hamilton County|IN|43451|United States|-5|apartment| +39328|AAAAAAAAAKJJAAAA||||Suite 390|Riverview|Pittsburg County|||United States|-6|condo| +39329|AAAAAAAABKJJAAAA|20|Ridge Green|Street|Suite 280|Mountain View|Johnson County|AR|74466|United States|-6|single family| +39330|AAAAAAAACKJJAAAA|638|Cedar |Dr.|Suite I|Pleasant Grove|Aitkin County|MN|54136|United States|-6|condo| +39331|AAAAAAAADKJJAAAA|71|4th Adams|Cir.|Suite 20|Unionville|Yakutat Borough|AK|91711|United States|-9|condo| +39332|AAAAAAAAEKJJAAAA|545|Hillcrest |Dr.|Suite 350|Valley View|Palm Beach County|FL|35124|United States|-5|apartment| +39333|AAAAAAAAFKJJAAAA|851|Main West|Blvd|Suite A|Providence|Griggs County|ND|56614|United States|-6|apartment| +39334|AAAAAAAAGKJJAAAA|997|Smith |Court|Suite E|Deerfield|Williamson County|TX|79840|United States|-6|condo| +39335|AAAAAAAAHKJJAAAA|114|Cherry Railroad|Lane|Suite 140|Gum Springs|Potter County|PA|12106|United States|-5|single family| +39336|AAAAAAAAIKJJAAAA|198|West 7th|Ln|Suite K|Riverdale|Finney County|KS|69391|United States|-6|apartment| +39337|AAAAAAAAJKJJAAAA|780|Forest |Avenue|Suite 360|Newport|Bristol Bay Borough|AK|91521|United States|-9|apartment| +39338|AAAAAAAAKKJJAAAA|99|5th |Drive|Suite 400|Yorktown|Foard County|TX|70732|United States|-6|single family| +39339|AAAAAAAALKJJAAAA|||||Lincoln||AR|71289|United States|-6|condo| +39340|AAAAAAAAMKJJAAAA|547|13th Park|Pkwy|Suite 340|Lincoln|Taylor County|IA|51289|United States|-6|condo| +39341|AAAAAAAANKJJAAAA|239|Pine Lee|Road|Suite 230|Bethel|Huron County|OH|45281|United States|-5|apartment| +39342|AAAAAAAAOKJJAAAA|143|Hill |Pkwy|Suite V|Riverside|Vilas County|WI|59231|United States|-6|apartment| +39343|AAAAAAAAPKJJAAAA|853|Washington West|Pkwy|Suite 250|Red Hill|Loving County|TX|74338|United States|-6|single family| +39344|AAAAAAAAALJJAAAA|678|Dogwood |Street|Suite O|Riverview|Yuma County|CO|89003|United States|-7|apartment| +39345|AAAAAAAABLJJAAAA|530|Wilson |Circle|Suite K|Green Acres|Colfax County|NM|87683|United States|-7|condo| +39346|AAAAAAAACLJJAAAA|945|South Franklin|Wy|Suite 440|Proctor|Jersey County|IL|68140|United States|-6|single family| +39347|AAAAAAAADLJJAAAA|357|Hill |Pkwy|Suite 200|Liberty|Yoakum County|TX|73451|United States|-6|condo| +39348|AAAAAAAAELJJAAAA|275|Center |Avenue|Suite 400|Greenfield|Lonoke County|AR|75038|United States|-6|single family| +39349|AAAAAAAAFLJJAAAA|21|Ridge Oak|Dr.|Suite P|Macedonia|Ellis County|KS|61087|United States|-6|apartment| +39350|AAAAAAAAGLJJAAAA|551|2nd |Street|Suite 0|Centerville|Campbell County|WY|80059|United States|-7|apartment| +39351|AAAAAAAAHLJJAAAA|187|Lee Dogwood|Ct.|Suite M|Ashland|Stone County|AR|74244|United States|-6|single family| +39352|AAAAAAAAILJJAAAA|94|Madison |Ave|Suite 290|Edgewood|Washtenaw County|MI|40069|United States|-5|single family| +39353|AAAAAAAAJLJJAAAA|427|First |Avenue|Suite S|Florence|San Augustine County|TX|73394|United States|-6|condo| +39354|AAAAAAAAKLJJAAAA|696|Smith |Boulevard|Suite L|Spring Valley|Montezuma County|CO|86060|United States|-7|condo| +39355|AAAAAAAALLJJAAAA|704|12th Green|Parkway|Suite I|State Line|Woodson County|KS|64312|United States|-6|apartment| +39356|AAAAAAAAMLJJAAAA|259|Willow |Lane|Suite D|Bethesda|Trumbull County|OH|45980|United States|-5|apartment| +39357|AAAAAAAANLJJAAAA|862|Hill Sunset|Ave|Suite A|Pine Grove|Saunders County|NE|64593|United States|-7|condo| +39358|AAAAAAAAOLJJAAAA|901|2nd Central|Pkwy|Suite H|Union|Litchfield County|CT|09321|United States|-5|condo| +39359|AAAAAAAAPLJJAAAA|779|Spring |Cir.|Suite 400|Woodlawn|Washington County|KS|64098|United States|-6|apartment| +39360|AAAAAAAAAMJJAAAA|293|Park |Ln|Suite M|Sunnyside|Dorchester County|SC|21952|United States|-5|single family| +39361|AAAAAAAABMJJAAAA|638|Seventh Ridge|Ln|Suite C|Hillcrest|Surry County|NC|23003|United States|-5|condo| +39362|AAAAAAAACMJJAAAA|685|8th |Ct.|Suite 220|Wildwood|Hanson County|SD|56871|United States|-7|condo| +39363|AAAAAAAADMJJAAAA|206|Center |Way|Suite 160|Georgetown|White County|IN|47057|United States|-5|apartment| +39364|AAAAAAAAEMJJAAAA||||Suite Q|Clinton||||United States|-5|single family| +39365|AAAAAAAAFMJJAAAA|175|3rd |Drive|Suite 170|Forest Hills|Cheyenne County|NE|69237|United States|-6|condo| +39366|AAAAAAAAGMJJAAAA|690|8th 10th|Avenue|Suite H|Oakdale|Jo Daviess County|IL|69584|United States|-6|single family| +39367|AAAAAAAAHMJJAAAA|291|7th |ST|Suite L|Ashland|Mobile County|AL|34244|United States|-6|single family| +39368|AAAAAAAAIMJJAAAA|855|11th Lincoln|Court|Suite M|Wright|Davis County|IA|52814|United States|-6|apartment| +39369|AAAAAAAAJMJJAAAA|520|Woodland Birch|Street|Suite C|Greenfield|Lee County|KY|45038|United States|-5|condo| +39370|AAAAAAAAKMJJAAAA|217|Fifth |RD|Suite 190|Price|Carroll County|VA|23824|United States|-5|apartment| +39371|AAAAAAAALMJJAAAA|468|3rd |Ct.|Suite S|Walnut Grove|Jackson County|GA|37752|United States|-5|single family| +39372|AAAAAAAAMMJJAAAA|152|Second First|Pkwy|Suite R|Springfield|Coal County|OK|79303|United States|-6|single family| +39373|AAAAAAAANMJJAAAA|109|Elm |Dr.|Suite N|Woodville|Sumter County|FL|34289|United States|-5|condo| +39374|AAAAAAAAOMJJAAAA|337|River Fifth|Wy|Suite 20|Stringtown|Clinton County|KY|40162|United States|-6|single family| +39375|AAAAAAAAPMJJAAAA|479|Johnson |Wy|Suite 150|Mountain View|Clark County|WI|54466|United States|-6|single family| +39376|AAAAAAAAANJJAAAA|690|Central South|ST|Suite D|Star|Warren County|NY|10725|United States|-5|apartment| +39377|AAAAAAAABNJJAAAA|255|4th Davis|ST|Suite J|Pleasant Hill|Moore County|TN|33604|United States|-6|single family| +39378|AAAAAAAACNJJAAAA|774|Third Smith|Ave|Suite 420|Wilton|Floyd County|KY|46997|United States|-6|single family| +39379|AAAAAAAADNJJAAAA|212|Chestnut |Ln|Suite 100|Brownsville|San Luis Obispo County|CA|99310|United States|-8|condo| +39380|AAAAAAAAENJJAAAA|833|South |Wy|Suite 0|Harmony|Bethel Census Area|AK|95804|United States|-9|single family| +39381|AAAAAAAAFNJJAAAA|808|View |ST|Suite 490|Vienna|Jack County|TX|75119|United States|-6|condo| +39382|AAAAAAAAGNJJAAAA|11|Second |Ave|Suite 250|Bunker Hill|Montgomery County|NC|20150|United States|-5|apartment| +39383|AAAAAAAAHNJJAAAA|724|Second |ST|Suite 20|Union Hill|Pulaski County|GA|37746|United States|-5|condo| +39384|AAAAAAAAINJJAAAA|346|South |Pkwy|Suite B|Pleasant Grove|Meade County|KS|64136|United States|-6|apartment| +39385|AAAAAAAAJNJJAAAA|577|1st |ST|Suite 400|Edgewood|Clinton County|KY|40069|United States|-6|apartment| +39386|AAAAAAAAKNJJAAAA|379|View Fifth|Ave|Suite N|Fowler|Jackson County|MI|41083|United States|-5|condo| +39387|AAAAAAAALNJJAAAA|322|Walnut |Ln|Suite 140|Union Hill|Madison County|AL|37746|United States|-6|apartment| +39388|AAAAAAAAMNJJAAAA|718|3rd 2nd|Ct.|Suite F|Mountain View|Gallatin County|IL|64466|United States|-6|single family| +39389|AAAAAAAANNJJAAAA|727|Walnut Mill|Ct.|Suite L|Pleasant Grove|Ouray County|CO|84136|United States|-7|apartment| +39390|AAAAAAAAONJJAAAA|209|Oak 13th|Boulevard|Suite G|Salem|McPherson County|SD|58048|United States|-7|condo| +39391|AAAAAAAAPNJJAAAA|626|View |Boulevard|Suite 430|Sulphur Springs|Kimball County|NE|68354|United States|-7|condo| +39392|AAAAAAAAAOJJAAAA|558|Elm 9th|Pkwy|Suite B|Five Forks|Beaver County|UT|82293|United States|-7|single family| +39393|AAAAAAAABOJJAAAA|11|Lee |Ln|Suite 110|Spring Hill|Mills County|TX|76787|United States|-6|condo| +39394|AAAAAAAACOJJAAAA|546|Church Park|Parkway|Suite S|Fairview|Jay County|IN|45709|United States|-5|single family| +39395|AAAAAAAADOJJAAAA|189|Woodland |Avenue|Suite 60|Mount Olive|Delta County|CO|88059|United States|-7|condo| +39396|AAAAAAAAEOJJAAAA|870|Sycamore Smith|Wy|Suite 290|Greenville|Carlisle County|KY|41387|United States|-6|condo| +39397|AAAAAAAAFOJJAAAA|965|Johnson Washington|Drive|Suite F|Sheffield|Catahoula Parish|LA|76896|United States|-6|condo| +39398|AAAAAAAAGOJJAAAA|109|Birch |Blvd|Suite G|Pleasant Grove|Jefferson Parish|LA|74136|United States|-6|apartment| +39399|AAAAAAAAHOJJAAAA|14|3rd |Way|Suite K|Forest Hills|Boone County|NE|69237|United States|-6|condo| +39400|AAAAAAAAIOJJAAAA|578|Willow First|Street|Suite 240|Shiloh|Kent County|RI|09875|United States|-5|apartment| +39401|AAAAAAAAJOJJAAAA|379|5th Woodland|Road|Suite K|Woodland|Saline County|IL|64854|United States|-6|single family| +39402|AAAAAAAAKOJJAAAA|78|14th |Court|Suite 160|Shady Grove|Humboldt County|CA|92812|United States|-8|single family| +39403|AAAAAAAALOJJAAAA|145|Fourth |ST|Suite M|Shiloh|Baker County|OR|99275|United States|-8|single family| +39404|AAAAAAAAMOJJAAAA|810|Madison Williams|Circle|Suite G|Belmont|Presidio County|TX|70191|United States|-6|apartment| +39405|AAAAAAAANOJJAAAA|865|South |Ave|Suite 460|Pine Grove|Valley County|NE|64593|United States|-7|apartment| +39406|AAAAAAAAOOJJAAAA|351|Pine |Lane|Suite 410|White Oak|Jackson County|AR|76668|United States|-6|condo| +39407|AAAAAAAAPOJJAAAA|9|7th |Road|Suite 100|New Hope|Wilson County|TN|39431|United States|-5|condo| +39408|AAAAAAAAAPJJAAAA|535|Oak Green|Cir.|Suite S|Arlington|Somerset County|NJ|07157|United States|-5|single family| +39409|AAAAAAAABPJJAAAA|61|Railroad |RD|Suite A|Pine Grove|Jackson County|OH|44593|United States|-5|condo| +39410|AAAAAAAACPJJAAAA|991|Park West|Drive|Suite 340|Waterloo|Kidder County|ND|51675|United States|-6|apartment| +39411|AAAAAAAADPJJAAAA|134|Ridge |Drive|Suite W|Allison|Turner County|GA|34167|United States|-5|apartment| +39412|AAAAAAAAEPJJAAAA|143|Sycamore |Boulevard|Suite I|Belmont|Murray County|GA|30191|United States|-5|single family| +39413|AAAAAAAAFPJJAAAA|255|Oak 12th|Pkwy|Suite 370|Ashland|Jackson County|WI|54244|United States|-6|apartment| +39414|AAAAAAAAGPJJAAAA|853|Green |Ct.|Suite 210|Bunker Hill|Portage County|WI|50150|United States|-6|single family| +39415|AAAAAAAAHPJJAAAA|403|Sixth Spring|Ct.|Suite 270|Unionville|Northampton County|VA|21711|United States|-5|condo| +39416|AAAAAAAAIPJJAAAA|150|4th Oak|Way|Suite 490|Walnut Grove|Hopewell city|VA|27752|United States|-5|single family| +39417|AAAAAAAAJPJJAAAA|836|Jackson |Dr.|Suite 280|Indian Village|Johnson County|KS|61075|United States|-6|apartment| +39418|AAAAAAAAKPJJAAAA|200|Jefferson Chestnut|Way|Suite 180|Ellisville|Hardin County|IL|66820|United States|-6|condo| +39419|AAAAAAAALPJJAAAA|923|Jackson Walnut|Dr.|Suite 230|Providence|Marquette County|WI|56614|United States|-6|single family| +39420|AAAAAAAAMPJJAAAA|62|7th Sunset|RD|Suite H|Freeman|O-Brien County|IA|52297|United States|-6|apartment| +39421|AAAAAAAANPJJAAAA||Jefferson |Parkway|||Franklin County|VA|23394|United States||apartment| +39422|AAAAAAAAOPJJAAAA|388|Cedar |Ave|Suite P|Newtown|Hansford County|TX|71749|United States|-6|apartment| +39423|AAAAAAAAPPJJAAAA|668|Second Church|Pkwy|Suite F|White Oak|Catawba County|NC|26668|United States|-5|condo| +39424|AAAAAAAAAAKJAAAA|574|Sycamore 8th|Cir.|Suite P|Riverside|Rooks County|KS|69231|United States|-6|single family| +39425|AAAAAAAABAKJAAAA|543|10th |Wy|Suite Y|Springdale|Owsley County|KY|48883|United States|-5|single family| +39426|AAAAAAAACAKJAAAA|84|Hill West|Court|Suite F|Shady Grove|Webster Parish|LA|72812|United States|-6|condo| +39427|AAAAAAAADAKJAAAA|959|Spring 2nd|Court|Suite 380|Spring Valley|Alameda County|CA|96060|United States|-8|condo| +39428|AAAAAAAAEAKJAAAA|631|Second |Ln|Suite 400|Watkins|Glades County|FL|31732|United States|-5|single family| +39429|AAAAAAAAFAKJAAAA|102|Franklin |RD|Suite 300|Farmington|Oneida County|WI|59145|United States|-6|condo| +39430|AAAAAAAAGAKJAAAA|847|Cedar |Way|Suite 380|Glenwood|Ulster County|NY|13511|United States|-5|apartment| +39431|AAAAAAAAHAKJAAAA|135|Church 3rd|Way|Suite V|Hartland|Lake County|MI|46594|United States|-5|apartment| +39432|AAAAAAAAIAKJAAAA|735|Elm 4th|Ct.|Suite O|Sunnyside|Orange County|VA|21952|United States|-5|single family| +39433|AAAAAAAAJAKJAAAA|993|Valley 10th|Parkway|Suite M|Lakewood|Pike County|MO|68877|United States|-6|apartment| +39434|AAAAAAAAKAKJAAAA|859|Mill |ST|Suite J|Newtown|Walthall County|MS|51749|United States|-6|single family| +39435|AAAAAAAALAKJAAAA|33|Hill West|Pkwy|Suite J|Bunker Hill|Fayette County|WV|20150|United States|-5|apartment| +39436|AAAAAAAAMAKJAAAA|224|Elm Maple|ST|Suite 160|Bloomingdale|Waukesha County|WI|51824|United States|-6|apartment| +39437|AAAAAAAANAKJAAAA|823|Pine |Road|Suite 300|Crossroads|Sevier County|TN|30534|United States|-6|condo| +39438|AAAAAAAAOAKJAAAA|278|3rd |Wy|Suite Y|Unionville|Jefferson County|IA|51711|United States|-6|condo| +39439|AAAAAAAAPAKJAAAA|353|Park |Drive|Suite T|Ashland|Suffolk County|NY|14244|United States|-5|condo| +39440|AAAAAAAAABKJAAAA|638|Locust Hickory|Lane|Suite 150|Jamestown|Nassau County|FL|36867|United States|-5|condo| +39441|AAAAAAAABBKJAAAA|465|Washington Davis|Ave|Suite 320|Franklin|Hancock County|IL|69101|United States|-6|single family| +39442|AAAAAAAACBKJAAAA|256|Pine Thirteenth|RD|Suite 300|Mount Zion|Potter County|PA|18054|United States|-5|apartment| +39443|AAAAAAAADBKJAAAA|297|12th Washington|Wy|Suite 20|Pleasant Valley|Warren County|NJ|03077|United States|-5|single family| +39444|AAAAAAAAEBKJAAAA|231|||Suite 190|Stringtown|||70162|United States|-6|condo| +39445|AAAAAAAAFBKJAAAA|711|Sycamore Park|Blvd|Suite R|||HI|||-10|single family| +39446|AAAAAAAAGBKJAAAA|231|Locust |Dr.|Suite K|Oak Ridge|Greene County|TN|38371|United States|-5|apartment| +39447|AAAAAAAAHBKJAAAA|338|12th |Ave|Suite 90|Brownsville|Maury County|TN|39310|United States|-6|single family| +39448|AAAAAAAAIBKJAAAA|660|5th Davis|Ave|Suite T|Saint Johns|Union Parish|LA|75717|United States|-6|condo| +39449|AAAAAAAAJBKJAAAA|284|South 4th|Road|Suite K|New Hope|Chaves County|NM|89431|United States|-7|single family| +39450|AAAAAAAAKBKJAAAA|227|Sunset |Ln|Suite U|Pine Grove|Lamb County|TX|74593|United States|-6|apartment| +39451|AAAAAAAALBKJAAAA|357|Dogwood |RD|Suite T|Cedar Grove|Bristol Bay Borough|AK|90411|United States|-9|single family| +39452|AAAAAAAAMBKJAAAA|459|Church Park|Wy|Suite 450|Centerville|Wise County|VA|20059|United States|-5|condo| +39453|AAAAAAAANBKJAAAA|624|9th |ST|Suite 280|Shaw|Teller County|CO|80618|United States|-7|apartment| +39454|AAAAAAAAOBKJAAAA|807|Cedar 3rd|Avenue|Suite Q|Winslow|Martin County|KY|48525|United States|-5|apartment| +39455|AAAAAAAAPBKJAAAA|85|8th |Parkway|Suite G|Walnut Grove|Adair County|IA|57752|United States|-6|single family| +39456|AAAAAAAAACKJAAAA|518|Ridge |Boulevard|Suite V|Glenwood|Umatilla County|OR|93511|United States|-8|apartment| +39457|AAAAAAAABCKJAAAA|51|Church |Cir.|Suite V|Maple Grove|Archuleta County|CO|88252|United States|-7|condo| +39458|AAAAAAAACCKJAAAA|330|Lake |Drive|Suite 80|Springfield|Anson County|NC|29303|United States|-5|condo| +39459|AAAAAAAADCKJAAAA|491|Ash Second|Avenue|Suite 120|Sunnyside|Suffolk County|MA|02552|United States|-5|single family| +39460|AAAAAAAAECKJAAAA|248|Miller Cedar|Ct.|Suite N|Farmington|Liberty County|GA|39145|United States|-5|apartment| +39461|AAAAAAAAFCKJAAAA|5|North Johnson|Ave|Suite V|Spring Hill|Forrest County|MS|56787|United States|-6|condo| +39462|AAAAAAAAGCKJAAAA|396|5th |Wy|Suite B|Arlington|Garfield County|OK|76557|United States|-6|single family| +39463|AAAAAAAAHCKJAAAA|203|8th Franklin|Blvd|Suite 40|Plainville|Hancock County|GA|36115|United States|-5|apartment| +39464|AAAAAAAAICKJAAAA|795|6th Dogwood|Wy|Suite 280|Sunnyside|Cumberland County|ME|02552|United States|-5|apartment| +39465|AAAAAAAAJCKJAAAA|958|Hickory |Court|Suite 90|Fairfield|Sonoma County|CA|96192|United States|-8|apartment| +39466|AAAAAAAAKCKJAAAA|239|Lake Meadow|Drive|Suite B|Riverdale|Culpeper County|VA|29391|United States|-5|condo| +39467|AAAAAAAALCKJAAAA|860|3rd |Lane|Suite W|Greenfield|Harney County|OR|95038|United States|-8|single family| +39468|AAAAAAAAMCKJAAAA|567|Spruce |Circle|Suite W|Mount Vernon|Washington County|MD|28482|United States|-5|condo| +39469|AAAAAAAANCKJAAAA|188|Main Highland|Road|Suite 160|Pleasant Grove|Trinity County|TX|74136|United States|-6|apartment| +39470|AAAAAAAAOCKJAAAA|978|Willow |Street|Suite 80|Mount Vernon|Lawrence County|TN|38482|United States|-6|apartment| +39471|AAAAAAAAPCKJAAAA|530|15th Miller|Lane|Suite 400|Red Oak|Jasper County|IN|45018|United States|-5|single family| +39472|AAAAAAAAADKJAAAA|647|4th |RD|Suite H|Greenfield|Valley County|ID|85038|United States|-7|apartment| +39473|AAAAAAAABDKJAAAA|398|Johnson |Blvd|Suite 180|Shiloh|Moultrie County|IL|69275|United States|-6|single family| +39474|AAAAAAAACDKJAAAA|119|Cherry |Circle|Suite D|Fairview|Woodruff County|AR|75709|United States|-6|condo| +39475|AAAAAAAADDKJAAAA|849|Park |Parkway|Suite X|Ellisville|Luce County|MI|46820|United States|-5|condo| +39476|AAAAAAAAEDKJAAAA|322|Tenth Green|Lane|Suite V|Bethel|O-Brien County|IA|55281|United States|-6|condo| +39477|AAAAAAAAFDKJAAAA|786|10th |Ave|Suite V|New Hope|Schuyler County|IL|69431|United States|-6|single family| +39478|AAAAAAAAGDKJAAAA|589|Cedar |RD|Suite V|Highland|Montgomery County|MO|69454|United States|-6|apartment| +39479|AAAAAAAAHDKJAAAA|692|Hickory |Dr.|Suite H|Macedonia|Cocke County|TN|31087|United States|-5|condo| +39480|AAAAAAAAIDKJAAAA|902|Valley |Blvd|Suite Q|Hillcrest|Cameron County|TX|73003|United States|-6|condo| +39481|AAAAAAAAJDKJAAAA|354|Cherry |Drive|Suite 80|Georgetown|Campbell County|VA|27057|United States|-5|condo| +39482|AAAAAAAAKDKJAAAA|391|Park 4th|Way|Suite C|Oak Ridge|Crane County|TX|78371|United States|-6|apartment| +39483|AAAAAAAALDKJAAAA|128|Church Eigth|Ave|Suite A|Barnes|Cass County|MN|53788|United States|-6|apartment| +39484|AAAAAAAAMDKJAAAA|299|Highland |Circle|Suite 450|Peoria|Niobrara County|WY|89818|United States|-7|apartment| +39485|AAAAAAAANDKJAAAA|446|Church |Street|Suite 100|Belmont|Lamar County|MS|50191|United States|-6|condo| +39486|AAAAAAAAODKJAAAA|484|Hill Willow|Pkwy|Suite 160|Forestville|Knox County|ME|03627|United States|-5|single family| +39487|AAAAAAAAPDKJAAAA|836|Hickory |ST|Suite 60|Williamsville|Union County|IL|68754|United States|-6|condo| +39488|AAAAAAAAAEKJAAAA|253|Sycamore 12th|ST|Suite Y|Wilson|Iosco County|MI|46971|United States|-5|single family| +39489|AAAAAAAABEKJAAAA|20|Church Maple|Ave|Suite 350|Forest|Morgan County|AL|37537|United States|-6|single family| +39490|AAAAAAAACEKJAAAA|679|Park Seventh|Ln|Suite 300|Riverside|Pulaski County|KY|49231|United States|-5|condo| +39491|AAAAAAAADEKJAAAA|942|Walnut View|Avenue|Suite 310|Highland|Boone County|WV|29454|United States|-5|condo| +39492|AAAAAAAAEEKJAAAA|868|Johnson Mill|Drive|Suite 270|Red Hill|Jim Hogg County|TX|74338|United States|-6|single family| +39493|AAAAAAAAFEKJAAAA|855|View Davis|Way|Suite P|Pleasant Hill|Leon County|TX|73604|United States|-6|apartment| +39494|AAAAAAAAGEKJAAAA|217|13th Ridge|ST|Suite 0|Salem|Rice County|KS|68048|United States|-6|apartment| +39495|AAAAAAAAHEKJAAAA|861|Willow Jackson|Parkway|Suite 440|White Oak|Slope County|ND|56668|United States|-6|condo| +39496|AAAAAAAAIEKJAAAA|36|Seventh |Ct.|Suite 460|Roscoe|Wheeler County|OR|91854|United States|-8|condo| +39497|AAAAAAAAJEKJAAAA|458|Church |Ln|Suite I|Blue Springs|Young County|TX|74686|United States|-6|apartment| +39498|AAAAAAAAKEKJAAAA|889|Maple |Lane|Suite F|Mount Olive|Van Zandt County|TX|78059|United States|-6|condo| +39499|AAAAAAAALEKJAAAA|129|Church |Drive|Suite E|Georgetown|Lyon County|IA|57057|United States|-6|condo| +39500|AAAAAAAAMEKJAAAA|231|8th 4th|Pkwy|Suite E|Elkton|Choctaw County|MS|53481|United States|-6|single family| +39501|AAAAAAAANEKJAAAA|70|Jefferson West|Ave|Suite L|Elizabeth|Adams County|CO|82935|United States|-7|apartment| +39502|AAAAAAAAOEKJAAAA|314|Washington |Avenue|Suite 150|Georgetown|Kit Carson County|CO|87057|United States|-7|apartment| +39503|AAAAAAAAPEKJAAAA|587|Lee |Court|Suite 90|Clifton|Montgomery County|IA|58014|United States|-6|apartment| +39504|AAAAAAAAAFKJAAAA|419|Main Main|Pkwy|Suite S|Oakland|Caswell County|NC|29843|United States|-5|apartment| +39505|AAAAAAAABFKJAAAA|160|South Mill|Ct.|Suite L|Kingston|Ford County|IL|64975|United States|-6|apartment| +39506|AAAAAAAACFKJAAAA|992|Third Lee|Blvd|Suite Y|Amherst|Taylor County|WV|28119|United States|-5|condo| +39507|AAAAAAAADFKJAAAA|749|Broadway |Circle|Suite U|New Hope|Mineral County|NV|89431|United States|-8|single family| +39508|AAAAAAAAEFKJAAAA|961|View Locust|Cir.|Suite L|Lakeview|Harrison County|TX|78579|United States|-6|apartment| +39509|AAAAAAAAFFKJAAAA|451|Walnut Main|Wy|Suite P|Greenville|Jenkins County|GA|31387|United States|-5|single family| +39510|AAAAAAAAGFKJAAAA|652||Wy|||Yavapai County||85709|||| +39511|AAAAAAAAHFKJAAAA|986|Lincoln Miller|Circle|Suite B|Springfield|Pepin County|WI|59303|United States|-6|apartment| +39512|AAAAAAAAIFKJAAAA|806|Sixth |Way|Suite 350|Mountain View|Camden County|NJ|05066|United States|-5|condo| +39513|AAAAAAAAJFKJAAAA|134|Ridge Ash|Drive|Suite N|Franklin|Taylor County|FL|39101|United States|-5|condo| +39514|AAAAAAAAKFKJAAAA|583|North Williams|Road|Suite K|Liberty|Heard County|GA|33451|United States|-5|condo| +39515|AAAAAAAALFKJAAAA|624|Fifth View|Wy|Suite 200|Bridgeport|Choctaw County|AL|35817|United States|-6|apartment| +39516|AAAAAAAAMFKJAAAA|852|3rd |Parkway|Suite D|Mount Vernon|Logan County|AR|78482|United States|-6|single family| +39517|AAAAAAAANFKJAAAA|598|Fourth 10th|Drive|Suite 260|Bunker Hill|Roger Mills County|OK|70150|United States|-6|condo| +39518|AAAAAAAAOFKJAAAA|927|Fifth |Parkway|Suite 310|Union|Silver Bow County|MT|68721|United States|-7|apartment| +39519|AAAAAAAAPFKJAAAA|827|||||Hayes County||||-6|| +39520|AAAAAAAAAGKJAAAA|880|Eigth |Street|Suite G|Ashland|Gove County|KS|64244|United States|-6|apartment| +39521|AAAAAAAABGKJAAAA|675|Spring 5th|Wy|Suite 70|Kingston|Lake County|CO|84975|United States|-7|condo| +39522|AAAAAAAACGKJAAAA|376|6th Maple|Ct.|Suite Q|Gladstone|Madison County|OH|40894|United States|-5|single family| +39523|AAAAAAAADGKJAAAA|436|Elm Highland|Pkwy|Suite K|Riverdale|Glenn County|CA|99391|United States|-8|apartment| +39524|AAAAAAAAEGKJAAAA|727|Twelfth |Avenue||Mountain View|Franklin County|||||| +39525|AAAAAAAAFGKJAAAA|877|Willow Cedar|Dr.|Suite 470|Spring Hill|Washington County|NC|26787|United States|-5|apartment| +39526|AAAAAAAAGGKJAAAA|743|North |Way|Suite F|Springfield|Boone County|WV|29303|United States|-5|condo| +39527|AAAAAAAAHGKJAAAA|910|7th Lake|ST|Suite E|Brookwood|Island County|WA|90965|United States|-8|apartment| +39528|AAAAAAAAIGKJAAAA|789|Lakeview North|Road|Suite O|Oakdale|Shelby County|MO|69584|United States|-6|condo| +39529|AAAAAAAAJGKJAAAA|475|5th |Ct.|Suite T|Hopewell|Davidson County|NC|20587|United States|-5|single family| +39530|AAAAAAAAKGKJAAAA|674|1st Forest|Dr.|Suite 20|Ferguson|Floyd County|KY|41821|United States|-6|single family| +39531|AAAAAAAALGKJAAAA|82|3rd |Boulevard|Suite C|Liberty|Nemaha County|NE|63451|United States|-7|single family| +39532|AAAAAAAAMGKJAAAA|985|5th Center|Dr.|Suite V|Hopewell|Wheatland County|MT|60587|United States|-7|single family| +39533|AAAAAAAANGKJAAAA|33|Seventh |Ct.|Suite E|Arlington|Highlands County|FL|36557|United States|-5|single family| +39534|AAAAAAAAOGKJAAAA|45|Park |Street|Suite 170|Clifton|Butler County|NE|68014|United States|-6|apartment| +39535|AAAAAAAAPGKJAAAA|645||Dr.||Newtown||KY||||apartment| +39536|AAAAAAAAAHKJAAAA||South College|Ave|Suite L|Oak Hill|Delta County|CO|87838|United States|-7|condo| +39537|AAAAAAAABHKJAAAA|609|Eigth Adams|Circle|Suite 80|Unionville|Okeechobee County|FL|31711|United States|-5|apartment| +39538|AAAAAAAACHKJAAAA|310|14th 15th|Way|Suite V|Glendale|Smith County|KS|63951|United States|-6|single family| +39539|AAAAAAAADHKJAAAA|353|Church |Wy|Suite 50|Glendale|Emanuel County|GA|33951|United States|-5|apartment| +39540|AAAAAAAAEHKJAAAA|514|6th Park|Way|Suite 410|San Jose|Itasca County|MN|58003|United States|-6|condo| +39541|AAAAAAAAFHKJAAAA|666|Walnut |Court|Suite Y|Waterloo|Maries County|MO|61675|United States|-6|single family| +39542|AAAAAAAAGHKJAAAA|994|West Park|Dr.|Suite 80|Edgewood|Douglas County|IL|60069|United States|-6|condo| +39543|AAAAAAAAHHKJAAAA|618|13th |Ave|Suite T|Five Forks|Scott County|VA|22293|United States|-5|apartment| +39544|AAAAAAAAIHKJAAAA|1000|Cedar |Ln|Suite X|Spring Hill|Adair County|MO|66787|United States|-6|apartment| +39545|AAAAAAAAJHKJAAAA|330|Woodland |Boulevard|Suite T|Wildwood|Leon County|FL|36871|United States|-5|single family| +39546|AAAAAAAAKHKJAAAA|293|Lincoln Washington|Court|Suite H|Springdale|Herkimer County|NY|18883|United States|-5|apartment| +39547|AAAAAAAALHKJAAAA|812|1st Hickory|Blvd|Suite 20|Bridgeport|Nevada County|CA|95817|United States|-8|apartment| +39548|AAAAAAAAMHKJAAAA|111|River Mill|Drive|Suite D|Northwood|Eddy County|ND|54104|United States|-6|condo| +39549|AAAAAAAANHKJAAAA|166|Willow |Road|Suite V|Bayview|Tensas Parish|LA|79672|United States|-6|single family| +39550|AAAAAAAAOHKJAAAA|295|East |Road|Suite J|Willow|Greene County|AR|76798|United States|-6|condo| +39551|AAAAAAAAPHKJAAAA|43|Chestnut Second|Drive|Suite 430|Lakeside|Stanton County|NE|69532|United States|-7|condo| +39552|AAAAAAAAAIKJAAAA|204|First |Pkwy|Suite 50|Hopewell|Allegan County|MI|40587|United States|-5|single family| +39553|AAAAAAAABIKJAAAA|738|Maple Jackson|Cir.|Suite J|Clifton|Crook County|OR|98014|United States|-8|condo| +39554|AAAAAAAACIKJAAAA|544|Washington Central|Drive|Suite 470|Greenfield|Bourbon County|KS|65038|United States|-6|condo| +39555|AAAAAAAADIKJAAAA|481|South Oak|ST|Suite I|Franklin|Jasper County|SC|29101|United States|-5|apartment| +39556|AAAAAAAAEIKJAAAA|389|Ridge |Pkwy|Suite R|Riverview|Berrien County|MI|49003|United States|-5|condo| +39557|AAAAAAAAFIKJAAAA|641|Jackson Ash|Parkway|Suite Y|Lincoln|Gallia County|OH|41289|United States|-5|single family| +39558|AAAAAAAAGIKJAAAA|977|Cedar Sycamore|Road|Suite 230|Lakewood|Renville County|MN|58877|United States|-6|single family| +39559|AAAAAAAAHIKJAAAA|464|6th 3rd|Lane|Suite T|Riverview|Clearfield County|PA|19003|United States|-5|condo| +39560|AAAAAAAAIIKJAAAA|803|Smith Johnson|Street|Suite G|Shady Grove|Cedar County|IA|52812|United States|-6|single family| +39561|AAAAAAAAJIKJAAAA|347|Fifth |Dr.|Suite O|Peru|Oklahoma County|OK|70302|United States|-6|apartment| +39562|AAAAAAAAKIKJAAAA|581|2nd Railroad|Lane|Suite 390|Plainview|Johnson County|KY|43683|United States|-6|apartment| +39563|AAAAAAAALIKJAAAA|93|Center 1st|Ln|Suite 270|Midway|Mayes County|OK|71904|United States|-6|apartment| +39564|AAAAAAAAMIKJAAAA|78|3rd Poplar|RD|Suite L|Mountain View|Pemiscot County|MO|64466|United States|-6|condo| +39565|AAAAAAAANIKJAAAA|103|7th |Circle|Suite I|Centerville|Shelby County|KY|40059|United States|-5|single family| +39566|AAAAAAAAOIKJAAAA|300|Oak Church|Ave|Suite 110|Macedonia|Westchester County|NY|11087|United States|-5|condo| +39567|AAAAAAAAPIKJAAAA|821|Cherry |Court|Suite Q|New Hope|Sevier County|AR|79431|United States|-6|apartment| +39568|AAAAAAAAAJKJAAAA|717|Johnson |RD|Suite N|Pleasant Valley|Martinsville city|VA|22477|United States|-5|apartment| +39569|AAAAAAAABJKJAAAA|804|Third Dogwood|Avenue|Suite 190|Springdale|Cumberland County|ME|09483|United States|-5|apartment| +39570|AAAAAAAACJKJAAAA||East |Road|||Washington County||||-6|apartment| +39571|AAAAAAAADJKJAAAA|573|Franklin Fifth|Road|Suite Q|Greenwood|Strafford County|NH|09428|United States|-5|apartment| +39572|AAAAAAAAEJKJAAAA|731|Walnut |Street|Suite Q|Lebanon|Cherokee County|SC|22898|United States|-5|single family| +39573|AAAAAAAAFJKJAAAA|778|Fourth |Cir.|Suite 200|Mountain View|Benton County|OR|94466|United States|-8|apartment| +39574|AAAAAAAAGJKJAAAA|861|3rd Meadow|Boulevard|Suite A|Oak Grove|Jefferson County|WI|58370|United States|-6|single family| +39575|AAAAAAAAHJKJAAAA|819|Railroad Meadow|Boulevard|Suite 30|Pine Grove|Monroe County|AR|74593|United States|-6|apartment| +39576|AAAAAAAAIJKJAAAA|376|2nd |Street|Suite J|Newport|Grant County|AR|71521|United States|-6|condo| +39577|AAAAAAAAJJKJAAAA|133||Court|Suite T|||MN|||-6|| +39578|AAAAAAAAKJKJAAAA|365|3rd |Lane|Suite 220|Carter|Madison County|FL|30919|United States|-5|single family| +39579|AAAAAAAALJKJAAAA|56|Mill |Drive|Suite I|Mount Pleasant|Shenandoah County|VA|21933|United States|-5|single family| +39580|AAAAAAAAMJKJAAAA|537|Lakeview Laurel|Ln|Suite 140|Derby|Perry County|AL|37702|United States|-6|single family| +39581|AAAAAAAANJKJAAAA|252|Johnson 4th|Parkway|Suite 0|Pine Grove|Lander County|NV|84593|United States|-8|single family| +39582|AAAAAAAAOJKJAAAA|11|College |Wy|Suite I|Red Hill|Honolulu County|HI|94338|United States|-10|condo| +39583|AAAAAAAAPJKJAAAA|831|Davis |Street|Suite 150|Longwood|Marion County|KS|67021|United States|-6|apartment| +39584|AAAAAAAAAKKJAAAA|942|Franklin |Drive|Suite 400|Hamilton|Pondera County|MT|62808|United States|-7|condo| +39585|AAAAAAAABKKJAAAA|221|Third |Ln|Suite Q|Highland Park|Cumberland County|IL|66534|United States|-6|apartment| +39586|AAAAAAAACKKJAAAA|108|Railroad Park|Drive|Suite L|Greenfield|Ketchikan Gateway Borough|AK|95038|United States|-9|single family| +39587|AAAAAAAADKKJAAAA|488|Center |Lane|Suite 280|Pleasant Valley|Gilliam County|OR|92477|United States|-8|single family| +39588|AAAAAAAAEKKJAAAA|684||Street|Suite H|Pleasant Valley|Lowndes County|AL|32477|United States||| +39589|AAAAAAAAFKKJAAAA|734|4th Church|Lane|Suite 380|Morris|Isle of Wight County|VA|26696|United States|-5|condo| +39590|AAAAAAAAGKKJAAAA|54|Valley |Street|Suite 470|Oakwood|Sherman County|TX|70169|United States|-6|single family| +39591|AAAAAAAAHKKJAAAA|907|Dogwood |Wy|Suite 220|Mount Pleasant|Stephens County|TX|71933|United States|-6|single family| +39592|AAAAAAAAIKKJAAAA|60|Highland Third|Lane|Suite 180|Woodlawn|Craig County|OK|74098|United States|-6|apartment| +39593|AAAAAAAAJKKJAAAA|708|Center Miller|Cir.|Suite E|Centerville|Washington County|MN|50059|United States|-6|single family| +39594|AAAAAAAAKKKJAAAA|46|1st |Lane|Suite 110|Hillcrest|Smith County|KS|63003|United States|-6|condo| +39595|AAAAAAAALKKJAAAA|200|12th |Wy|Suite 350|Sulphur Springs|Colquitt County|GA|38354|United States|-5|single family| +39596|AAAAAAAAMKKJAAAA|907|Lake Main|Street|Suite 390|Waterloo|Oneida County|ID|81675|United States|-7|condo| +39597|AAAAAAAANKKJAAAA|539|Oak |ST|Suite 490|Pleasant Grove|Guilford County|NC|24136|United States|-5|condo| +39598|AAAAAAAAOKKJAAAA|317|College Ninth|Ave|Suite M|Avery|Concho County|TX|70194|United States|-6|apartment| +39599|AAAAAAAAPKKJAAAA|805|West |Avenue|Suite F|Glendale|Decatur County|IA|53951|United States|-6|single family| +39600|AAAAAAAAALKJAAAA|878|Poplar Second|Court|Suite T|Mount Vernon|Wood County|OH|48482|United States|-5|single family| +39601|AAAAAAAABLKJAAAA|49|Broadway |Road|Suite A|Bunker Hill|Henderson County|IL|60150|United States|-6|single family| +39602|AAAAAAAACLKJAAAA|542|Park |Avenue|Suite Y|Hillcrest|Mountrail County|ND|53003|United States|-6|single family| +39603|AAAAAAAADLKJAAAA|118|Willow Dogwood|ST|Suite 370|Altamont|Irwin County|GA|39387|United States|-5|single family| +39604|AAAAAAAAELKJAAAA|847|14th |Wy|Suite 0|Wildwood|Douglas County|NV|86871|United States|-8|condo| +39605|AAAAAAAAFLKJAAAA|497|Locust |Pkwy|Suite X|Amherst|Wilkinson County|MS|58119|United States|-6|single family| +39606|AAAAAAAAGLKJAAAA|270|Fourth |Dr.|Suite 490|Lebanon|Mariposa County|CA|92898|United States|-8|single family| +39607|AAAAAAAAHLKJAAAA|607|Johnson College|RD|Suite X|Forest Hills|Buchanan County|VA|29237|United States|-5|condo| +39608|AAAAAAAAILKJAAAA|971|Pine |Ct.|Suite F|Pleasant Hill|Jefferson County|IL|63604|United States|-6|single family| +39609|AAAAAAAAJLKJAAAA|344|7th |Court|Suite Q|Lincoln|Carteret County|NC|21289|United States|-5|condo| +39610|AAAAAAAAKLKJAAAA|878|Jefferson Walnut|Avenue|Suite 390|Wilson|Wilcox County|GA|36971|United States|-5|apartment| +39611|AAAAAAAALLKJAAAA|994|15th |Circle|Suite O|Five Points|Mobile County|AL|36098|United States|-6|apartment| +39612|AAAAAAAAMLKJAAAA|373|Chestnut North|Street|Suite 250|Lakewood|Highland County|OH|48877|United States|-5|condo| +39613|AAAAAAAANLKJAAAA|487|College Center|Road|Suite 240|Deerfield|Stillwater County|MT|69840|United States|-7|single family| +39614|AAAAAAAAOLKJAAAA|748|Davis |Street|Suite 20|Pine Grove|Jackson County|OR|94593|United States|-8|condo| +39615|AAAAAAAAPLKJAAAA|841|North |Circle|Suite O|Georgetown|Muscatine County|IA|57057|United States|-6|apartment| +39616|AAAAAAAAAMKJAAAA|466|Adams |Ct.|Suite K|Hillcrest|McLean County|ND|53003|United States|-6|single family| +39617|AAAAAAAABMKJAAAA|821|View |ST|Suite 490|Bethel|Lewis and Clark County|MT|65281|United States|-7|apartment| +39618|AAAAAAAACMKJAAAA|679|Pine 3rd|Parkway|Suite 240|Oak Ridge|Polk County|FL|38371|United States|-5|single family| +39619|AAAAAAAADMKJAAAA|47|Lincoln 5th|Road|Suite 80|Pleasant Hill|Emery County|UT|83604|United States|-7|condo| +39620|AAAAAAAAEMKJAAAA|837|2nd |Wy|Suite 270|Greenville|Fulton County|PA|11387|United States|-5|condo| +39621|AAAAAAAAFMKJAAAA|690|Wilson 5th|Wy|Suite E|Enterprise|Henry County|MO|61757|United States|-6|apartment| +39622|AAAAAAAAGMKJAAAA|439|Eigth Church|Boulevard|Suite C|Enterprise|Keokuk County|IA|51757|United States|-6|condo| +39623|AAAAAAAAHMKJAAAA|286|Sunset |Street|Suite 250|Glenwood|Ingham County|MI|43511|United States|-5|condo| +39624|AAAAAAAAIMKJAAAA|169|Franklin |RD|Suite 50|Concord|Tioga County|NY|14107|United States|-5|condo| +39625|AAAAAAAAJMKJAAAA|684|Pine |Ln|Suite C|Glenwood|Buffalo County|NE|63511|United States|-6|single family| +39626|AAAAAAAAKMKJAAAA|15|7th 6th|Dr.|Suite U|Granite|Stanton County|NE|66284|United States|-7|condo| +39627|AAAAAAAALMKJAAAA|363|Adams Smith|Street|Suite U|Clifton|Lee County|SC|28014|United States|-5|single family| +39628|AAAAAAAAMMKJAAAA|810|Main |Circle|Suite T|Lakeside|Saratoga County|NY|19532|United States|-5|condo| +39629|AAAAAAAANMKJAAAA|159|Lake |Ct.|Suite 340|Highland|Whitley County|IN|49454|United States|-5|single family| +39630|AAAAAAAAOMKJAAAA|434|Mill 8th|Ct.|Suite A|Summerville|Perry County|AR|72033|United States|-6|condo| +39631|AAAAAAAAPMKJAAAA|14|Pine |Pkwy|Suite 330|Riverview|Owen County|KY|49003|United States|-5|single family| +39632|AAAAAAAAANKJAAAA|830|Seventh |Ln|Suite I|Cedar Grove|Campbell County|KY|40411|United States|-6|single family| +39633|AAAAAAAABNKJAAAA|439|Johnson |Pkwy|Suite G|Mount Olive|Coosa County|AL|38059|United States|-6|single family| +39634|AAAAAAAACNKJAAAA|595|Sunset |Dr.|Suite V|Clifton|Jackson County|OK|78014|United States|-6|condo| +39635|AAAAAAAADNKJAAAA|596|Second |Ln|Suite 220|Sulphur Springs|Marion County|IN|48354|United States|-5|apartment| +39636|AAAAAAAAENKJAAAA|453|Madison |Ct.|Suite 260|Brownsville|Emporia city|VA|29310|United States|-5|single family| +39637|AAAAAAAAFNKJAAAA|114|Cedar |Ave|Suite O|Providence|Hall County|GA|36614|United States|-5|apartment| +39638|AAAAAAAAGNKJAAAA|364|Spring |Blvd|Suite 100|Walnut Grove|Howard County|IN|47752|United States|-5|single family| +39639|AAAAAAAAHNKJAAAA|445|Railroad Willow|Street|Suite 380|Oakland|Calhoun County|MS|59843|United States|-6|apartment| +39640|AAAAAAAAINKJAAAA|33|Jefferson |ST|Suite M|Wilson|Broome County|NY|16971|United States|-5|apartment| +39641|AAAAAAAAJNKJAAAA|419|Maple |Way|Suite M|Deerfield|White County|IL|69840|United States|-6|apartment| +39642|AAAAAAAAKNKJAAAA||14th ||||Baraga County|MI|||-5|condo| +39643|AAAAAAAALNKJAAAA|571|Lake 4th|ST|Suite 430|Pleasant Grove|Knox County|KY|44136|United States|-5|apartment| +39644|AAAAAAAAMNKJAAAA|977|Pine Eigth|Circle|Suite 260|Glendale|Monroe County|MI|43951|United States|-5|condo| +39645|AAAAAAAANNKJAAAA|139|West Church|Lane|Suite 470|Crossroads|Montgomery County|NC|20534|United States|-5|condo| +39646|AAAAAAAAONKJAAAA|98|Cedar Johnson|Cir.|Suite 210|Brownsville|Coconino County|AZ|89310|United States|-7|condo| +39647|AAAAAAAAPNKJAAAA|598|Willow Fourth|Lane|Suite M|Pleasant Grove|Northwest Arctic Borough|AK|94136|United States|-9|condo| +39648|AAAAAAAAAOKJAAAA|177|6th |Court|Suite 200|Providence|Henry County|TN|36614|United States|-5|condo| +39649|AAAAAAAABOKJAAAA|254|Hill Church|Blvd|Suite B|Kingston|Waller County|TX|74975|United States|-6|apartment| +39650|AAAAAAAACOKJAAAA|32|Third Wilson|Way|Suite 450|Jamestown|Van Buren County|MI|46867|United States|-5|single family| +39651|AAAAAAAADOKJAAAA|126|Ridge |RD|Suite H|Somerville|McCulloch County|TX|77783|United States|-6|apartment| +39652|AAAAAAAAEOKJAAAA|766|View |Ct.|Suite P|Newtown|Cheboygan County|MI|41749|United States|-5|condo| +39653|AAAAAAAAFOKJAAAA|799|Center Park|Circle|Suite N|Fayetteville|Coffey County|KS|61732|United States|-6|condo| +39654|AAAAAAAAGOKJAAAA||Locust Park|Way|Suite 40|Gum Springs||IL|||-6|condo| +39655|AAAAAAAAHOKJAAAA|497|Hillcrest Fourth|Cir.|Suite G|Brownsville|Matagorda County|TX|79310|United States|-6|condo| +39656|AAAAAAAAIOKJAAAA|617|2nd |Court|Suite X|Belmont|Macon County|IL|60191|United States|-6|single family| +39657|AAAAAAAAJOKJAAAA||||Suite S||||10965|United States|-5|| +39658|AAAAAAAAKOKJAAAA|399|Laurel |Ct.|Suite N|Bayside|McCurtain County|OK|79550|United States|-6|apartment| +39659|AAAAAAAALOKJAAAA|191|Sixth Elm|Court|Suite H|Franklin|Alexander County|NC|29101|United States|-5|apartment| +39660|AAAAAAAAMOKJAAAA|111|Davis Cherry|Wy|Suite J|Brownsville|Ashe County|NC|29310|United States|-5|single family| +39661|AAAAAAAANOKJAAAA|341|Park Hill|Wy|Suite S|Walnut Grove|Eagle County|CO|87752|United States|-7|apartment| +39662|AAAAAAAAOOKJAAAA|682|Church Forest|Wy|Suite 50|Hopewell|Box Elder County|UT|80587|United States|-7|apartment| +39663|AAAAAAAAPOKJAAAA|10|Spring |Road|Suite I|Oakdale|Vernon County|MO|69584|United States|-6|apartment| +39664|AAAAAAAAAPKJAAAA|894|Church Lee|Ln|Suite Q|Edgewood|Titus County|TX|70069|United States|-6|apartment| +39665|AAAAAAAABPKJAAAA|857|Fifth Jackson|Ct.|Suite 290|Greenville|Steuben County|NY|11387|United States|-5|apartment| +39666|AAAAAAAACPKJAAAA|269|Lakeview Hill|Lane|Suite 170|Fairfield|Franklin County|KY|46192|United States|-6|condo| +39667|AAAAAAAADPKJAAAA|139|3rd Ridge|ST|Suite T|Springdale|Jefferson County|OH|48883|United States|-5|single family| +39668|AAAAAAAAEPKJAAAA|367|Eigth |Ct.|Suite 430|Oak Hill|San Bernardino County|CA|97838|United States|-8|single family| +39669|AAAAAAAAFPKJAAAA|865|Davis Fourth|Road|Suite 180|Leesville|Adams County|CO|85423|United States|-7|condo| +39670|AAAAAAAAGPKJAAAA|734|Lincoln |Ln|Suite O|Wildwood|Lenoir County|NC|26871|United States|-5|single family| +39671|AAAAAAAAHPKJAAAA|397|South 11th|Dr.|Suite F|Sulphur Springs|Gray County|KS|68354|United States|-6|single family| +39672|AAAAAAAAIPKJAAAA|73|Ash |Street|Suite 250|Hopewell|Trinity County|CA|90587|United States|-8|condo| +39673|AAAAAAAAJPKJAAAA|496|12th |Avenue|Suite N|Glenwood|Caswell County|NC|23511|United States|-5|apartment| +39674|AAAAAAAAKPKJAAAA|857|1st 11th|Ln|Suite 150|Shady Grove|Trinity County|CA|92812|United States|-8|single family| +39675|AAAAAAAALPKJAAAA|651|Johnson Sunset|Way|Suite 370|Highland Park|Emporia city|VA|26534|United States|-5|apartment| +39676|AAAAAAAAMPKJAAAA|429|5th Dogwood|Street|Suite 300|Enterprise|Grant County|OR|91757|United States|-8|apartment| +39677|AAAAAAAANPKJAAAA|476|Miller |RD|Suite 260|Oak Hill|Tipton County|IN|47838|United States|-5|single family| +39678|AAAAAAAAOPKJAAAA|281|Oak |Circle|Suite 300|Salem|Lamoille County|VT|08648|United States|-5|condo| +39679|AAAAAAAAPPKJAAAA|705|Hillcrest |Pkwy|Suite C|Marion|Renville County|MN|50399|United States|-6|condo| +39680|AAAAAAAAAALJAAAA|638|Park |Circle|Suite J|Antioch|Lee County|IL|68605|United States|-6|condo| +39681|AAAAAAAABALJAAAA|545|Second 8th|Circle|Suite 50|Hamilton|Harrison County|KY|42808|United States|-6|apartment| +39682|AAAAAAAACALJAAAA|130|Willow |Boulevard|Suite 150|Shady Grove|Erie County|OH|42812|United States|-5|condo| +39683|AAAAAAAADALJAAAA|583|4th Lake|Blvd|Suite I|Shiloh|Henry County|VA|29275|United States|-5|single family| +39684|AAAAAAAAEALJAAAA|150|Forest Railroad|RD|Suite 80|White Oak|Tyler County|WV|26668|United States|-5|apartment| +39685|AAAAAAAAFALJAAAA|768|Second |Lane|Suite B|Edgewood|Clay County|FL|30069|United States|-5|apartment| +39686|AAAAAAAAGALJAAAA|665|Park |Lane|Suite B|Highland Park|Etowah County|AL|36534|United States|-6|condo| +39687|AAAAAAAAHALJAAAA|989|Oak Washington|Lane|Suite 360|Highland|Calhoun County|IA|59454|United States|-6|single family| +39688|AAAAAAAAIALJAAAA|38|Broadway Mill|Blvd|Suite 420|Riverview|Page County|VA|29003|United States|-5|condo| +39689|AAAAAAAAJALJAAAA|667|2nd |Drive|Suite K|Oakland|Webster County|GA|39843|United States|-5|apartment| +39690|AAAAAAAAKALJAAAA|769|Green Woodland|RD|Suite X|Enterprise|Contra Costa County|CA|91757|United States|-8|apartment| +39691|AAAAAAAALALJAAAA|422|Main 7th|Drive|Suite J|Pine Grove|Moore County|TN|34593|United States|-6|single family| +39692|AAAAAAAAMALJAAAA|715|Oak 13th|Way|Suite R|Greenwood|Laurel County|KY|48828|United States|-5|apartment| +39693|AAAAAAAANALJAAAA|28|Johnson |Circle|Suite N|Harmony|Edgefield County|SC|25804|United States|-5|apartment| +39694|AAAAAAAAOALJAAAA|378|Hillcrest |Cir.|Suite 170|Macon|Barber County|KS|60369|United States|-6|single family| +39695|AAAAAAAAPALJAAAA|991|Railroad Smith|Lane|Suite 370|Forest Hills|Emery County|UT|89237|United States|-7|apartment| +39696|AAAAAAAAABLJAAAA|929|Main |Circle|Suite E|Mountain View|Custer County|MT|64466|United States|-7|condo| +39697|AAAAAAAABBLJAAAA|815|Church Willow|ST|Suite 40|Ashland|Storey County|NV|84244|United States|-8|apartment| +39698|AAAAAAAACBLJAAAA|483|Sixth |Boulevard|Suite L|Woodlawn|Columbia County|FL|34098|United States|-5|single family| +39699|AAAAAAAADBLJAAAA|29|Green |Avenue|Suite I|Oak Ridge|Macon County|AL|38371|United States|-6|single family| +39700|AAAAAAAAEBLJAAAA|220|Spruce |Avenue|Suite Y|Lincoln|Boone County|WV|21289|United States|-5|single family| +39701|AAAAAAAAFBLJAAAA|591|Williams |Boulevard|Suite N|Liberty|Bayfield County|WI|53451|United States|-6|single family| +39702|AAAAAAAAGBLJAAAA|209|Pine Sunset|Lane|Suite 130|Riverside|Rockingham County|NC|29231|United States|-5|single family| +39703|AAAAAAAAHBLJAAAA|57|Broadway Fifteenth|Parkway|Suite T|Newport|Sullivan County|PA|11521|United States|-5|condo| +39704|AAAAAAAAIBLJAAAA|895|Williams |Pkwy|Suite C|Ashland|Billings County|ND|54244|United States|-6|single family| +39705|AAAAAAAAJBLJAAAA|776|Second |Dr.|Suite S|Bridgeport|Brookings County|SD|55817|United States|-6|condo| +39706|AAAAAAAAKBLJAAAA|47|7th Main|Wy|Suite X|Bethel|Dallas County|TX|75281|United States|-6|single family| +39707|AAAAAAAALBLJAAAA|186|Johnson Spruce|Drive|Suite 180|Sulphur Springs|Marshall County|AL|38354|United States|-6|single family| +39708|AAAAAAAAMBLJAAAA|849|Fifth |Court|Suite O|Stringtown|Boone County|KY|40162|United States|-6|single family| +39709|AAAAAAAANBLJAAAA|346|Poplar Sunset|Ct.|Suite 270|Five Forks|Orange County|TX|72293|United States|-6|single family| +39710|AAAAAAAAOBLJAAAA|867|Tenth |Blvd|Suite 450|Mountain View|Bexar County|TX|74466|United States|-6|condo| +39711|AAAAAAAAPBLJAAAA|983|Park Sycamore|Lane|Suite M|Globe|Portsmouth city|VA|29614|United States|-5|single family| +39712|AAAAAAAAACLJAAAA|527|Park |Ct.|Suite 310|Cedar Grove|Daviess County|MO|60411|United States|-6|single family| +39713|AAAAAAAABCLJAAAA|133|Pine |Street|Suite 50|Oakdale|Franklin County|GA|39584|United States|-5|single family| +39714|AAAAAAAACCLJAAAA|371|Elm |Drive|Suite B|Kingston|Tehama County|CA|94975|United States|-8|apartment| +39715|AAAAAAAADCLJAAAA|309|6th |Way|Suite T|Clinton|Crockett County|TX|78222|United States|-6|condo| +39716|AAAAAAAAECLJAAAA|308|Seventh Hill|Way|Suite 70|Harmony|Fentress County|TN|35804|United States|-5|condo| +39717|AAAAAAAAFCLJAAAA|481|Sixth |Street|Suite 370|Newport|Kossuth County|IA|51521|United States|-6|condo| +39718|AAAAAAAAGCLJAAAA|324|Williams Highland|Cir.|Suite 490|Oakland|Franklin County|NY|19843|United States|-5|condo| +39719|AAAAAAAAHCLJAAAA|170|5th 1st|Drive|Suite 220|Midway|Sublette County|WY|81904|United States|-7|condo| +39720|AAAAAAAAICLJAAAA|240|4th 3rd|Way|Suite 160|Unionville|Custer County|CO|81711|United States|-7|condo| +39721|AAAAAAAAJCLJAAAA|751|Washington Oak|ST|Suite 490|Bridgeport|Pinal County|AZ|85817|United States|-7|condo| +39722|AAAAAAAAKCLJAAAA|988|Broadway |Avenue|Suite 40|Mount Pleasant|Otero County|NM|81933|United States|-7|condo| +39723|AAAAAAAALCLJAAAA|901|Maple |Ave|Suite 70|Carter|Piatt County|IL|60919|United States|-6|apartment| +39724|AAAAAAAAMCLJAAAA|32|Cherry Wilson|Dr.|Suite 90|Ashland|Lincoln County|GA|34244|United States|-5|single family| +39725|AAAAAAAANCLJAAAA|798|Pine 12th|Boulevard|Suite J|Woodlawn|Roberts County|TX|74098|United States|-6|apartment| +39726|AAAAAAAAOCLJAAAA|984|View |Pkwy|Suite K|Mount Zion|Webster County|MS|58054|United States|-6|condo| +39727|AAAAAAAAPCLJAAAA|464|Main |Street|Suite D|Kingston|Zapata County|TX|74975|United States|-6|single family| +39728|AAAAAAAAADLJAAAA|872|Maple |Blvd|Suite 390|Riverview|Upton County|TX|79003|United States|-6|apartment| +39729|AAAAAAAABDLJAAAA|560|Park Franklin|Circle|Suite 20|Clinton|Pondera County|MT|68222|United States|-7|single family| +39730|AAAAAAAACDLJAAAA|989|Meadow |Cir.|Suite Y|Waterloo|Granville County|NC|21675|United States|-5|condo| +39731|AAAAAAAADDLJAAAA|681|Mill |Cir.|Suite 130|Greenfield|Franklin County|WA|95038|United States|-8|single family| +39732|AAAAAAAAEDLJAAAA|361|15th |Wy|Suite X|Buena Vista|Shoshone County|ID|85752|United States|-7|apartment| +39733|AAAAAAAAFDLJAAAA|962|Sixth Cedar|Lane|Suite 100|Sulphur Springs|Shackelford County|TX|78354|United States|-6|single family| +39734|AAAAAAAAGDLJAAAA|479|3rd |Avenue|Suite 160|Shady Grove|Cumberland County|NC|22812|United States|-5|apartment| +39735|AAAAAAAAHDLJAAAA|633|First Lincoln|Boulevard|Suite 440|Bunker Hill|Zapata County|TX|70150|United States|-6|apartment| +39736|AAAAAAAAIDLJAAAA|995|Washington Park|Road|Suite 50|Jamestown|Wibaux County|MT|66867|United States|-7|apartment| +39737|AAAAAAAAJDLJAAAA|891|7th 3rd|Drive|Suite X|Oak Hill|Stephens County|GA|37838|United States|-5|single family| +39738|AAAAAAAAKDLJAAAA|622|15th Elm|Ln|Suite 190|Lakewood|Orleans County|VT|09477|United States|-5|apartment| +39739|AAAAAAAALDLJAAAA|34|Walnut |Road|Suite 250|Woodville|Stoddard County|MO|64289|United States|-6|single family| +39740|AAAAAAAAMDLJAAAA|716|Cedar Central|Court|Suite 280|Forest Hills|Bandera County|TX|79237|United States|-6|apartment| +39741|AAAAAAAANDLJAAAA|321|14th |Drive|Suite P|Highland Park|Neosho County|KS|66534|United States|-6|single family| +39742|AAAAAAAAODLJAAAA|198|15th Church|Ln|Suite 40|Bridgeport|Navarro County|TX|75817|United States|-6|condo| +39743|AAAAAAAAPDLJAAAA|238|Center Lake|Ave|Suite B|Providence|Kitsap County|WA|96614|United States|-8|condo| +39744|AAAAAAAAAELJAAAA|40|Madison |Cir.|Suite G|Providence|Whitley County|KY|46614|United States|-5|condo| +39745|AAAAAAAABELJAAAA|323|Second College|Boulevard|Suite E|Florence|Falls County|TX|73394|United States|-6|single family| +39746|AAAAAAAACELJAAAA|462|Chestnut Sixth|Cir.|Suite 150|Plainview|Crisp County|GA|33683|United States|-5|single family| +39747|AAAAAAAADELJAAAA|||Street|||Fulton County|PA|15281||-5|single family| +39748|AAAAAAAAEELJAAAA|989|9th |RD|Suite 150|Mount Olive|Bertie County|NC|28059|United States|-5|apartment| +39749|AAAAAAAAFELJAAAA|799|6th |Court|Suite 370|Mount Olive|Burleson County|TX|78059|United States|-6|condo| +39750|AAAAAAAAGELJAAAA|585|14th Main|Way|Suite T|Sulphur Springs|Overton County|TN|38354|United States|-6|apartment| +39751|AAAAAAAAHELJAAAA|384|Mill Cherry|Parkway|Suite F|Riverside|Glascock County|GA|39231|United States|-5|condo| +39752|AAAAAAAAIELJAAAA|496|Locust |ST|Suite N|New Hope|Hill County|TX|79431|United States|-6|single family| +39753|AAAAAAAAJELJAAAA|427|Spring Dogwood|Parkway|Suite P|Shady Grove|Anderson County|SC|22812|United States|-5|apartment| +39754|AAAAAAAAKELJAAAA|674|Cedar 2nd|Wy|Suite F|Whitesville|Matanuska-Susitna Borough|AK|95903|United States|-9|condo| +39755|AAAAAAAALELJAAAA|420|1st |ST|Suite 30|Marion|Wicomico County|MD|20399|United States|-5|single family| +39756|AAAAAAAAMELJAAAA|656||Ave||||||United States||| +39757|AAAAAAAANELJAAAA|512|Thirteenth |Way|Suite 340|Summit|Mercer County|IL|60499|United States|-6|condo| +39758|AAAAAAAAOELJAAAA|29|11th Third|Court|Suite G|Jamestown|Crittenden County|AR|76867|United States|-6|single family| +39759|AAAAAAAAPELJAAAA|758|4th |Ln|Suite 50|Springfield|Monroe County|FL|39303|United States|-5|single family| +39760|AAAAAAAAAFLJAAAA|86|Fourth Highland|Wy|Suite 210|Jackson|Hamilton County|OH|49583|United States|-5|single family| +39761|AAAAAAAABFLJAAAA|249|Park Seventh|RD|Suite D|New Hope|Jackson County|TN|39431|United States|-5|condo| +39762|AAAAAAAACFLJAAAA|318|River Meadow|Avenue|Suite 430|Mount Olive|Spotsylvania County|VA|28059|United States|-5|apartment| +39763|AAAAAAAADFLJAAAA|156|First Fifth|Drive|Suite A|Mountain View|Copiah County|MS|54466|United States|-6|apartment| +39764|AAAAAAAAEFLJAAAA|497|Maple |Blvd|Suite 300|Fairview|Noble County|OH|45709|United States|-5|condo| +39765|AAAAAAAAFFLJAAAA|148|Maple |Circle|Suite J|Woodland|Fulton County|AR|74854|United States|-6|condo| +39766|AAAAAAAAGFLJAAAA|520|10th 13th|Drive|Suite C|Cedar Grove|Union County|OR|90411|United States|-8|single family| +39767|AAAAAAAAHFLJAAAA|967|8th Main|Court|Suite 350|Maple Grove|Wyoming County|WV|28252|United States|-5|condo| +39768|AAAAAAAAIFLJAAAA|642|2nd |Pkwy|Suite 290|Mount Vernon|Potter County|TX|78482|United States|-6|apartment| +39769|AAAAAAAAJFLJAAAA|677|11th 3rd|Cir.|Suite C|Crossroads|Lake and Peninsula Borough|AK|90534|United States|-9|single family| +39770|AAAAAAAAKFLJAAAA|209|Hickory |Dr.|Suite 90|Providence|McPherson County|NE|66614|United States|-7|apartment| +39771|AAAAAAAALFLJAAAA|175|Lincoln Hillcrest|Lane|Suite 340|Oakland|Benton County|MN|59843|United States|-6|single family| +39772|AAAAAAAAMFLJAAAA|805|Spring |Blvd|Suite L|Wildwood|Webster County|IA|56871|United States|-6|single family| +39773|AAAAAAAANFLJAAAA|521|Wilson Lincoln|RD|Suite 10|Macedonia|Dorchester County|SC|21087|United States|-5|single family| +39774|AAAAAAAAOFLJAAAA|502|Park |Parkway|Suite E|Delmar|Cameron County|PA|13957|United States|-5|single family| +39775|AAAAAAAAPFLJAAAA|670|River View|Lane|Suite H|Mountain View|Plymouth County|IA|54466|United States|-6|apartment| +39776|AAAAAAAAAGLJAAAA|353|West Jackson|Street|Suite X|Newtown|Greensville County|VA|21749|United States|-5|single family| +39777|AAAAAAAABGLJAAAA|809|8th Sycamore|Dr.|Suite G|Edgewood|Pierce County|NE|60069|United States|-7|single family| +39778|AAAAAAAACGLJAAAA|559|12th Birch|Ct.|Suite M|Liberty|Harford County|MD|23451|United States|-5|apartment| +39779|AAAAAAAADGLJAAAA|825|Willow Cedar|Cir.|Suite 470|Altamont|Mendocino County|CA|99387|United States|-8|apartment| +39780|AAAAAAAAEGLJAAAA|738|Willow Adams|Avenue|Suite 370|Smith|Cedar County|IA|57317|United States|-6|condo| +39781|AAAAAAAAFGLJAAAA|816|Lakeview Central|Pkwy|Suite U|Belmont|Butte County|CA|90191|United States|-8|single family| +39782|AAAAAAAAGGLJAAAA|266|4th Wilson|Blvd|Suite Q|Franklin|Ohio County|WV|29101|United States|-5|condo| +39783|AAAAAAAAHGLJAAAA|339|Second |Way|Suite 150|Oak Grove|Lincoln County|MO|68370|United States|-6|apartment| +39784|AAAAAAAAIGLJAAAA|260|Smith |Pkwy|Suite 60|Oakwood|Amador County|CA|90169|United States|-8|single family| +39785|AAAAAAAAJGLJAAAA|962|Green Park|Way|Suite Q|Greenville|Caledonia County|VT|01987|United States|-5|condo| +39786|AAAAAAAAKGLJAAAA|149|Pine Williams|Drive|Suite N|Newport|Willacy County|TX|71521|United States|-6|single family| +39787|AAAAAAAALGLJAAAA|235|View 11th|Wy|Suite F|Waterloo|Limestone County|TX|71675|United States|-6|single family| +39788|AAAAAAAAMGLJAAAA|79|Johnson |Ln|Suite 200|Bridgeport|Gallia County|OH|45817|United States|-5|condo| +39789|AAAAAAAANGLJAAAA|924|Ridge |Cir.|Suite 30|Kingston|Poquoson city|VA|24975|United States|-5|apartment| +39790|AAAAAAAAOGLJAAAA|774|Sunset |Boulevard|Suite 380|Macedonia|Sevier County|UT|81087|United States|-7|apartment| +39791|AAAAAAAAPGLJAAAA|758|Park |Ln|Suite C|Jamestown|Hyde County|NC|26867|United States|-5|single family| +39792|AAAAAAAAAHLJAAAA|667|Pine 6th|Pkwy|Suite 340|Greenwood|Upson County|GA|38828|United States|-5|condo| +39793|AAAAAAAABHLJAAAA|3|Franklin |Road|Suite 200|Providence|Lemhi County|ID|86614|United States|-7|single family| +39794|AAAAAAAACHLJAAAA|906|Park Franklin|Boulevard|Suite Y|Woodland|Lewis County|KY|44854|United States|-5|condo| +39795|AAAAAAAADHLJAAAA|200|Lakeview |Dr.|Suite 390|Spring Hill|Juneau County|WI|56787|United States|-6|single family| +39796|AAAAAAAAEHLJAAAA|598|Chestnut Sixth|Road|Suite K|Unionville|Sanilac County|MI|41711|United States|-5|single family| +39797|AAAAAAAAFHLJAAAA|542|Sixth |Drive|Suite 490|Milan|Union County|OR|96697|United States|-8|single family| +39798|AAAAAAAAGHLJAAAA|750|Lincoln |Lane|Suite 90|Mount Vernon|Roane County|TN|38482|United States|-6|single family| +39799|AAAAAAAAHHLJAAAA|627|Washington |Ct.|Suite 280|Shore Acres|Morgan County|KY|42724|United States|-5|apartment| +39800|AAAAAAAAIHLJAAAA|212|Madison 12th|Blvd|Suite W|Five Forks|Nodaway County|MO|62293|United States|-6|single family| +39801|AAAAAAAAJHLJAAAA|||Court|Suite 420|Franklin|Richland County||59101|United States|-6|apartment| +39802|AAAAAAAAKHLJAAAA|28|North |Pkwy|Suite 490|Clifton|Windham County|CT|08614|United States|-5|apartment| +39803|AAAAAAAALHLJAAAA|966|South |Ln|Suite 210|Salem|Kent County|MD|28048|United States|-5|apartment| +39804|AAAAAAAAMHLJAAAA|749|Hill |Boulevard|Suite F|Stringtown|Charlottesville city|VA|20162|United States|-5|apartment| +39805|AAAAAAAANHLJAAAA|763|Railroad |Court|Suite 380|Bethesda|Benton County|IA|55980|United States|-6|single family| +39806|AAAAAAAAOHLJAAAA|838|Park |Court|Suite B|Clinton|Barbour County|WV|28222|United States|-5|condo| +39807|AAAAAAAAPHLJAAAA|996|Smith |Ct.|Suite 240|Greenville|Manitowoc County|WI|51387|United States|-6|single family| +39808|AAAAAAAAAILJAAAA|217|Ash |Avenue|Suite U|Maple Grove|Webster County|WV|28252|United States|-5|condo| +39809|AAAAAAAABILJAAAA|390|Lakeview Center|Drive|Suite X|Edgewood|Marshall County|KY|40069|United States|-5|apartment| +39810|AAAAAAAACILJAAAA|896|Main Miller|Street|Suite 400|Hopewell|Wayne County|TN|30587|United States|-6|condo| +39811|AAAAAAAADILJAAAA|315|Sycamore |Pkwy|Suite O|Mount Pleasant|Tyler County|TX|71933|United States|-6|apartment| +39812|AAAAAAAAEILJAAAA|359|Spring |Court|Suite 420|Forest Hills|East Carroll Parish|LA|79237|United States|-6|single family| +39813|AAAAAAAAFILJAAAA|673|3rd |Avenue|Suite 170|Greenfield|Tioga County|PA|15038|United States|-5|apartment| +39814|AAAAAAAAGILJAAAA|428|Highland Broadway|Wy|Suite 220|Greenwood|Rockwall County|TX|78828|United States|-6|single family| +39815|AAAAAAAAHILJAAAA|102|Oak |Avenue|Suite 140|Concord|Pulaski County|AR|74107|United States|-6|condo| +39816|AAAAAAAAIILJAAAA|948|Church Meadow|Parkway|Suite 440|Clifton|Valley County|NE|68014|United States|-7|apartment| +39817|AAAAAAAAJILJAAAA|555|7th |Lane|Suite Y|Clifton|Park County|MT|68014|United States|-7|apartment| +39818|AAAAAAAAKILJAAAA|834|Church Spruce|Circle|Suite 20|Salem|Holmes County|MS|58048|United States|-6|apartment| +39819|AAAAAAAALILJAAAA|357|15th |Court|Suite P|Woodland|Etowah County|AL|34854|United States|-6|apartment| +39820|AAAAAAAAMILJAAAA|22|Madison |Dr.|Suite K|Georgetown|Barbour County|WV|27057|United States|-5|apartment| +39821|AAAAAAAANILJAAAA|208|4th Sunset|Road|Suite M|Arlington|Martinsville city|VA|26557|United States|-5|apartment| +39822|AAAAAAAAOILJAAAA|28|Cherry |RD|Suite T|Riverside|Hall County|GA|39231|United States|-5|apartment| +39823|AAAAAAAAPILJAAAA|75|Sycamore Hickory|Street|Suite 340|Newtown|Columbia County|WA|91749|United States|-8|apartment| +39824|AAAAAAAAAJLJAAAA|512|Oak Birch|Boulevard|Suite 10|Oakwood|Kosciusko County|IN|40169|United States|-5|single family| +39825|AAAAAAAABJLJAAAA|249|Williams |RD|Suite K|Sulphur Springs|Taos County|NM|88354|United States|-7|apartment| +39826|AAAAAAAACJLJAAAA|22|Forest Forest|Cir.|Suite 350|Sulphur Springs|Lane County|OR|98354|United States|-8|apartment| +39827|AAAAAAAADJLJAAAA|69|Main 15th|Pkwy|Suite 270|Kingston|Orange County|CA|94975|United States|-8|apartment| +39828|AAAAAAAAEJLJAAAA|367|2nd Miller|Road|Suite L|Providence|Finney County|KS|66614|United States|-6|apartment| +39829|AAAAAAAAFJLJAAAA|906|Jackson |Parkway|Suite U|Woodville|Cook County|IL|64289|United States|-6|apartment| +39830|AAAAAAAAGJLJAAAA|880|Maple |Court|Suite 340|Newport|Tarrant County|TX|71521|United States|-6|apartment| +39831|AAAAAAAAHJLJAAAA|47|Adams |Parkway|Suite 470|Oakland|Orange County|TX|79843|United States|-6|apartment| +39832|AAAAAAAAIJLJAAAA|606|Locust |Way|Suite 170|New Hope|Dawson County|TX|79431|United States|-6|condo| +39833|AAAAAAAAJJLJAAAA||Broadway |Dr.|Suite F|||GA||||single family| +39834|AAAAAAAAKJLJAAAA|610|11th |Dr.|Suite S|Woodville|Rutherford County|NC|24289|United States|-5|single family| +39835|AAAAAAAALJLJAAAA|622|Willow Cedar|Lane|Suite X|Plainville|Polk County|MO|66115|United States|-6|apartment| +39836|AAAAAAAAMJLJAAAA|544|Church Sixth|Way|Suite B|Belmont|Dawson County|MT|60191|United States|-7|apartment| +39837|AAAAAAAANJLJAAAA|260|2nd Lee|Ln|Suite 150|Point Pleasant|Los Alamos County|NM|84749|United States|-7|single family| +39838|AAAAAAAAOJLJAAAA|336|Pine |Parkway|Suite 420|Williamsville|Bee County|TX|78754|United States|-6|condo| +39839|AAAAAAAAPJLJAAAA|500|Pine Smith|Dr.|Suite J|Florence|Lucas County|OH|43394|United States|-5|single family| +39840|AAAAAAAAAKLJAAAA|133|7th |RD|Suite T|Spring Hill|Hardin County|KY|46787|United States|-6|apartment| +39841|AAAAAAAABKLJAAAA|761|10th Park|Ln|Suite 20|Buckingham|Pierce County|GA|34092|United States|-5|apartment| +39842|AAAAAAAACKLJAAAA|711|Hill |RD|Suite B|Woodland|Marion County|FL|34854|United States|-5|condo| +39843|AAAAAAAADKLJAAAA|909|3rd |Lane|Suite W|Clinton|Walsh County|ND|58222|United States|-6|apartment| +39844|AAAAAAAAEKLJAAAA|512|Seventh |Road|Suite 230|Union City|Elko County|NV|88087|United States|-8|single family| +39845|AAAAAAAAFKLJAAAA|717|4th Laurel|Street|Suite X|Providence|Hitchcock County|NE|66614|United States|-6|apartment| +39846|AAAAAAAAGKLJAAAA|165|Lake |Circle|Suite 280|Providence|Bulloch County|GA|36614|United States|-5|single family| +39847|AAAAAAAAHKLJAAAA|243|Miller Johnson|Way|Suite Q|Woodland|Lincoln County|WV|24854|United States|-5|apartment| +39848|AAAAAAAAIKLJAAAA|103|Oak 10th|Way|Suite 330|Hillcrest|Bacon County|GA|33003|United States|-5|single family| +39849|AAAAAAAAJKLJAAAA|633|First Laurel|Dr.|Suite I|Mountain View|Saluda County|SC|24466|United States|-5|apartment| +39850|AAAAAAAAKKLJAAAA|551|Walnut Lincoln|Ave|Suite O|Wilson|Mason County|WV|26971|United States|-5|apartment| +39851|AAAAAAAALKLJAAAA|787|Oak Madison|Blvd|Suite 420|Riverview|Hale County|TX|79003|United States|-6|single family| +39852|AAAAAAAAMKLJAAAA|434|Johnson |Drive|Suite F|Frenchtown|McPherson County|SD|52629|United States|-7|condo| +39853|AAAAAAAANKLJAAAA|316|Forest |Parkway|Suite L|Deerfield|||59840|United States|-6|| +39854|AAAAAAAAOKLJAAAA|939|Williams West|Pkwy|Suite 40|Edgewood|Laurens County|GA|30069|United States|-5|apartment| +39855|AAAAAAAAPKLJAAAA|686|1st Highland|Blvd|Suite K|Fairview|Coryell County|TX|75709|United States|-6|apartment| +39856|AAAAAAAAALLJAAAA|686|3rd 6th|Pkwy|Suite 190|Union Hill|Lubbock County|TX|77746|United States|-6|single family| +39857|AAAAAAAABLLJAAAA|||Blvd|Suite X||||65819|United States||single family| +39858|AAAAAAAACLLJAAAA|458|Dogwood |Dr.|Suite 50|Arlington|Loudon County|TN|36557|United States|-6|single family| +39859|AAAAAAAADLLJAAAA|452|Ridge 5th|Dr.|Suite T|Maple Grove|Multnomah County|OR|98252|United States|-8|apartment| +39860|AAAAAAAAELLJAAAA|217|1st Cedar|Dr.|Suite R|Shiloh|Lee County|VA|29275|United States|-5|apartment| +39861|AAAAAAAAFLLJAAAA|172|Second |Dr.|Suite 170|Union|Ellis County|OK|78721|United States|-6|apartment| +39862|AAAAAAAAGLLJAAAA|762|Park |Court|Suite V|Argyle|Clay County|NE|68722|United States|-6|condo| +39863|AAAAAAAAHLLJAAAA|337|Sunset 3rd|Way|Suite W|Unionville|Summers County|WV|21711|United States|-5|apartment| +39864|AAAAAAAAILLJAAAA|372|Cherry |RD|Suite 350|Mount Pleasant|Lincoln County|ID|81933|United States|-7|condo| +39865|AAAAAAAAJLLJAAAA|839|1st Smith|Dr.|Suite Q|Littleton|Franklin County|AR|76074|United States|-6|apartment| +39866|AAAAAAAAKLLJAAAA|799|1st |Lane|Suite 470|Green Acres|Licking County|OH|47683|United States|-5|single family| +39867|AAAAAAAALLLJAAAA|92|Park |Street|Suite 440|Woodland|Telfair County|GA|34854|United States|-5|condo| +39868|AAAAAAAAMLLJAAAA|95|North Ridge|Avenue|Suite M|Fairfield|Cullman County|AL|36192|United States|-6|condo| +39869|AAAAAAAANLLJAAAA|835|Center Wilson|Court|Suite L|Buena Vista|Pinal County|AZ|85752|United States|-7|apartment| +39870|AAAAAAAAOLLJAAAA|38|Highland Maple|Parkway|Suite 70|Wilson|Cooper County|MO|66971|United States|-6|condo| +39871|AAAAAAAAPLLJAAAA|735|Seventh |Boulevard|Suite W|Georgetown|Worth County|GA|37057|United States|-5|apartment| +39872|AAAAAAAAAMLJAAAA|547|14th |Ct.|Suite 340|Pleasant Grove|Portage County|OH|44136|United States|-5|apartment| +39873|AAAAAAAABMLJAAAA|763|9th 4th|Road|Suite 170|Empire|Dundy County|NE|64145|United States|-6|condo| +39874|AAAAAAAACMLJAAAA|95|8th Pine|RD|Suite K|Riverview|Douglas County|GA|39003|United States|-5|apartment| +39875|AAAAAAAADMLJAAAA|119|Railroad 13th|Parkway|Suite V|Edgewood|Florence County|WI|50069|United States|-6|condo| +39876|AAAAAAAAEMLJAAAA|8|Woodland Second|Ct.|Suite 460|Springfield|Shannon County|SD|59303|United States|-7|condo| +39877|AAAAAAAAFMLJAAAA|638|Forest 3rd|Blvd|Suite 100|Redland|Lamar County|GA|36343|United States|-5|single family| +39878|AAAAAAAAGMLJAAAA|508|Mill Pine|Pkwy|Suite 270|Marion|Linn County|IA|50399|United States|-6|condo| +39879|AAAAAAAAHMLJAAAA|675|Lincoln Locust|Pkwy|Suite 230|Oakwood|Kendall County|IL|60169|United States|-6|condo| +39880|AAAAAAAAIMLJAAAA|826|Johnson |Avenue|Suite 400|Arcola|Charles County|MD|21654|United States|-5|apartment| +39881|AAAAAAAAJMLJAAAA|934|Pine |Boulevard|Suite 80|Riverdale|Greene County|MO|69391|United States|-6|apartment| +39882|AAAAAAAAKMLJAAAA|889|12th |Ave|Suite 30|Green Acres|Clay County|SD|57683|United States|-6|apartment| +39883|AAAAAAAALMLJAAAA|356|South |Parkway|Suite 400|Green Acres|Adair County|MO|67683|United States|-6|condo| +39884|AAAAAAAAMMLJAAAA|801|Twelfth |Lane|Suite H|Highland|Jackson County|OK|79454|United States|-6|single family| +39885|AAAAAAAANMLJAAAA|357|Ridge |Road|Suite Q|Avery|Carroll County|IN|40194|United States|-5|apartment| +39886|AAAAAAAAOMLJAAAA|936|Fourth East|Pkwy|Suite 460|Enon|Chickasaw County|IA|50965|United States|-6|single family| +39887|AAAAAAAAPMLJAAAA|384|6th Spruce|ST|Suite P|Summit|Knox County|TN|30499|United States|-6|condo| +39888|AAAAAAAAANLJAAAA|983|Ash Valley|Road|Suite 150|Union|Ottawa County|OK|78721|United States|-6|apartment| +39889|AAAAAAAABNLJAAAA|26|Cherry 10th|Road|Suite 480|Valley View|Osborne County|KS|65124|United States|-6|apartment| +39890|AAAAAAAACNLJAAAA|876|Maple 2nd|Boulevard|Suite 470|Mount Zion|Archuleta County|CO|88054|United States|-7|apartment| +39891|AAAAAAAADNLJAAAA|182|Highland |Drive|Suite 170|Unionville|Hutchinson County|TX|71711|United States|-6|apartment| +39892|AAAAAAAAENLJAAAA|826|Hill Birch|Ave|Suite 290|Stringtown|Meigs County|TN|30162|United States|-6|apartment| +39893|AAAAAAAAFNLJAAAA|830|Central |Ct.|Suite 240|Oakland|Monona County|IA|59843|United States|-6|apartment| +39894|AAAAAAAAGNLJAAAA|126|Cedar Miller|Circle|Suite T|White Oak|Greenwood County|SC|26668|United States|-5|condo| +39895|AAAAAAAAHNLJAAAA|578|3rd |Ln|Suite 100|Mount Olive|Mitchell County|TX|78059|United States|-6|single family| +39896|AAAAAAAAINLJAAAA|613|9th |Avenue|Suite X|Cedar Creek|Fayette County|AL|30311|United States|-6|single family| +39897|AAAAAAAAJNLJAAAA|663|Davis 1st|Drive|Suite R|Midway|Walthall County|MS|51904|United States|-6|single family| +39898|AAAAAAAAKNLJAAAA|179|Cedar Hill|Blvd|Suite I|White Oak|Lincoln County|NE|66668|United States|-7|single family| +39899|AAAAAAAALNLJAAAA|604|8th |Dr.|Suite 360|Edgewood|Wyoming County|WV|20069|United States|-5|apartment| +39900|AAAAAAAAMNLJAAAA|748|Main Railroad|Lane|Suite J|Wilson|Burke County|ND|56971|United States|-6|condo| +39901|AAAAAAAANNLJAAAA|899|Seventh |Boulevard|Suite 170|Fairview|Delaware County|IA|55709|United States|-6|condo| +39902|AAAAAAAAONLJAAAA|704|View Center|Way|Suite Q|Montague|Kossuth County|IA|54062|United States|-6|apartment| +39903|AAAAAAAAPNLJAAAA|895|Sunset |Parkway|Suite 110|Waterloo|Hall County|GA|31675|United States|-5|apartment| +39904|AAAAAAAAAOLJAAAA|944|3rd |Ln|Suite I|Springdale|Canadian County|OK|78883|United States|-6|apartment| +39905|AAAAAAAABOLJAAAA|978|Fifth 6th|Drive|Suite 320|Kingston|Bedford County|PA|14975|United States|-5|apartment| +39906|AAAAAAAACOLJAAAA|356|1st |Ln|Suite Y|Rosewood|Cotton County|OK|72205|United States|-6|condo| +39907|AAAAAAAADOLJAAAA|322|Oak |Ct.|Suite V|Lewisburg|Todd County|KY|47538|United States|-5|single family| +39908|AAAAAAAAEOLJAAAA|8|7th Cedar|Blvd|Suite M|Sunnyside|Nuckolls County|NE|61952|United States|-7|apartment| +39909|AAAAAAAAFOLJAAAA|477|Washington |RD|Suite 410|Clifton|Prentiss County|MS|58014|United States|-6|single family| +39910|AAAAAAAAGOLJAAAA|503|3rd |Lane|Suite 330|Unionville|Monroe County|MS|51711|United States|-6|single family| +39911|AAAAAAAAHOLJAAAA|298|View |Avenue|Suite 140|Five Forks|Clermont County|OH|42293|United States|-5|apartment| +39912|AAAAAAAAIOLJAAAA|101|Hickory Ash|Wy|Suite 210|Enterprise|Bertie County|NC|21757|United States|-5|condo| +39913|AAAAAAAAJOLJAAAA|824|Green Eigth|Boulevard|Suite X|Franklin|Ontonagon County|MI|49101|United States|-5|condo| +39914|AAAAAAAAKOLJAAAA|404|Broadway Valley|ST|Suite H|Pleasant Valley|Treasure County|MT|62477|United States|-7|apartment| +39915|AAAAAAAALOLJAAAA|976|Pine |Ct.|Suite J|Clearview|Hart County|GA|35495|United States|-5|condo| +39916|AAAAAAAAMOLJAAAA|706|Washington |Street|Suite K|Belmont|Webster County|WV|20191|United States|-5|single family| +39917|AAAAAAAANOLJAAAA|29|Church 15th|Ln|Suite 460|Pine Grove|Lincoln County|GA|34593|United States|-5|single family| +39918|AAAAAAAAOOLJAAAA|560|Fifth |Ct.|Suite 140|Sunnyside|Luzerne County|PA|11952|United States|-5|single family| +39919|AAAAAAAAPOLJAAAA|394|Miller Wilson|Drive|Suite Q|Lebanon|Butler County|KS|62898|United States|-6|apartment| +39920|AAAAAAAAAPLJAAAA|717|1st 11th|Dr.|Suite 410|Edgewood|Gila County|AZ|80069|United States|-7|condo| +39921|AAAAAAAABPLJAAAA|32|12th |Way|Suite N||Stone County|MS||||condo| +39922|AAAAAAAACPLJAAAA|466|Miller |Circle|Suite 250|Union|Allegheny County|PA|18721|United States|-5|single family| +39923|AAAAAAAADPLJAAAA|446|13th |Circle|Suite 60|Riverside|Sherman County|OR|99231|United States|-8|condo| +39924|AAAAAAAAEPLJAAAA|842|Elm East|Circle||||LA||||| +39925|AAAAAAAAFPLJAAAA|810||Way|Suite I|Friendship|Newton County|GA||||| +39926|AAAAAAAAGPLJAAAA|379|10th Lake|Blvd|Suite M|Williamsville|Lawrence County|KY|48754|United States|-5|condo| +39927|AAAAAAAAHPLJAAAA|418|2nd 15th|Dr.|Suite R|Buckingham|Drew County|AR|74092|United States|-6|condo| +39928|AAAAAAAAIPLJAAAA|529|8th Cherry|Dr.|Suite 440|Wilson|Dawson County|TX|76971|United States|-6|apartment| +39929|AAAAAAAAJPLJAAAA|481|||Suite I||||71711|||apartment| +39930|AAAAAAAAKPLJAAAA|641|Spring Hill|Pkwy|Suite S|Mount Zion|Johnson County|KS|68054|United States|-6|condo| +39931|AAAAAAAALPLJAAAA|756|Jefferson |Ave|Suite N|Friendship|Crawford County|MI|44536|United States|-5|single family| +39932|AAAAAAAAMPLJAAAA|641|Fifth 6th|ST|Suite 410|Greenfield|Trimble County|KY|45038|United States|-5|single family| +39933|AAAAAAAANPLJAAAA|250|Forest Main|Blvd|Suite Q|Walnut Grove|Randolph County|IN|47752|United States|-5|apartment| +39934|AAAAAAAAOPLJAAAA|781|Lakeview 13th|Parkway|Suite 410|Liberty|Greenbrier County|WV|23451|United States|-5|single family| +39935|AAAAAAAAPPLJAAAA|782|Mill |Ln|Suite K|Oak Hill|Union County|OR|97838|United States|-8|single family| +39936|AAAAAAAAAAMJAAAA|268|5th |Dr.|Suite V|Newport|Portage County|OH|41521|United States|-5|single family| +39937|AAAAAAAABAMJAAAA|91|Eigth Lake|Ct.|Suite X|Fairview|Monroe County|KY|45709|United States|-5|apartment| +39938|AAAAAAAACAMJAAAA|101|Tenth |Road|Suite 0|Waterloo|Shelby County|IA|51675|United States|-6|single family| +39939|AAAAAAAADAMJAAAA|297|5th |Blvd|Suite 380|Fairfield|Conecuh County|AL|36192|United States|-6|apartment| +39940|AAAAAAAAEAMJAAAA|781|Locust |Ct.|Suite 370|Salem|Camden County|GA|38048|United States|-5|apartment| +39941|AAAAAAAAFAMJAAAA|926|2nd Franklin|RD|Suite P|Lakewood|Jefferson County|FL|38877|United States|-5|apartment| +39942|AAAAAAAAGAMJAAAA|901|Main |Ct.|Suite 130|Jamestown|Marion County|IA|56867|United States|-6|condo| +39943|AAAAAAAAHAMJAAAA|475|Jackson |Pkwy|Suite S|Riverdale|Macon County|NC|29391|United States|-5|condo| +39944|AAAAAAAAIAMJAAAA|835|Madison Lee|Circle|Suite W|Mount Pleasant|Plymouth County|IA|51933|United States|-6|condo| +39945|AAAAAAAAJAMJAAAA|593|Railroad Davis|Drive|Suite W|Summit|Grand County|CO|80499|United States|-7|condo| +39946|AAAAAAAAKAMJAAAA|756|4th |Street|Suite I|Valley View|Bond County|IL|65124|United States|-6|single family| +39947|AAAAAAAALAMJAAAA|409|Walnut |Drive|Suite K|Wilson|Perry County|AL|36971|United States|-6|single family| +39948|AAAAAAAAMAMJAAAA|123|5th |Ave|Suite Y|Edgewood|Thomas County|KS|60069|United States|-6|apartment| +39949|AAAAAAAANAMJAAAA|181|Lake 8th|Road|Suite 390|Brownsville|Fulton County|IN|49310|United States|-5|apartment| +39950|AAAAAAAAOAMJAAAA|454|Lincoln Maple|RD|Suite 150|Oakland|Iowa County|IA|59843|United States|-6|condo| +39951|AAAAAAAAPAMJAAAA|730|West |Avenue|Suite 480|Greenfield|Saratoga County|NY|15038|United States|-5|single family| +39952|AAAAAAAAABMJAAAA|928|Railroad |Ave|Suite N|Jamestown|Ontario County|NY|16867|United States|-5|condo| +39953|AAAAAAAABBMJAAAA|574|Park |Road|Suite 140|Buena Vista|Nacogdoches County|TX|75752|United States|-6|condo| +39954|AAAAAAAACBMJAAAA|111|Forest |ST|Suite 0|Glenwood|McDonough County|IL|63511|United States|-6|single family| +39955|AAAAAAAADBMJAAAA|591|Park 8th|Street|Suite I|Lakewood|Love County|OK|78877|United States|-6|condo| +39956|AAAAAAAAEBMJAAAA|368|Chestnut |Blvd|Suite 270|Hillcrest|McNairy County|TN|33003|United States|-6|single family| +39957|AAAAAAAAFBMJAAAA|669|Cedar |Way|Suite 260|Woodlawn|Sandusky County|OH|44098|United States|-5|apartment| +39958|AAAAAAAAGBMJAAAA|102|Franklin |Pkwy|Suite 250|Carpenter|Monroe County|WV|21147|United States|-5|single family| +39959|AAAAAAAAHBMJAAAA|728|Pine 2nd|RD|Suite J|Cordova|Roosevelt County|NM|86938|United States|-7|apartment| +39960|AAAAAAAAIBMJAAAA|669|Maple Cedar|Ln|Suite 370|Belmont|Fulton County|NY|10191|United States|-5|single family| +39961|AAAAAAAAJBMJAAAA|314|Park |Road|Suite 310|Georgetown|Washington County|MD|27057|United States|-5|single family| +39962|AAAAAAAAKBMJAAAA|898|Spring |Avenue|Suite 360|Concord|Franklin County|KY|44107|United States|-6|condo| +39963|AAAAAAAALBMJAAAA|412|8th Johnson|Lane|Suite T|Elkton|Bradley County|TN|33481|United States|-5|condo| +39964|AAAAAAAAMBMJAAAA|902|Laurel Jefferson|Ct.|Suite 480|Five Forks|Rawlins County|KS|62293|United States|-6|condo| +39965|AAAAAAAANBMJAAAA|304|Main 14th|Way|Suite 170|Bridgeport|Logan County|KY|45817|United States|-5|condo| +39966|AAAAAAAAOBMJAAAA|958|Sixth |Avenue|Suite L|Valley View|Essex County|MA|05724|United States|-5|condo| +39967|AAAAAAAAPBMJAAAA|809|3rd Cedar|Ave|Suite 300|Guilford|Kent County|MI|44408|United States|-5|apartment| +39968|AAAAAAAAACMJAAAA|356|Poplar Poplar|Court|Suite J|Hamilton|Pettis County|MO|62808|United States|-6|single family| +39969|AAAAAAAABCMJAAAA|291|Center |Ln|Suite Y|Concord|New Castle County|DE|14107|United States|-5|apartment| +39970|AAAAAAAACCMJAAAA|592|Johnson Church|Drive|Suite G|Shady Grove|Wabash County|IN|42812|United States|-5|single family| +39971|AAAAAAAADCMJAAAA|891|Sixth |ST|Suite N|Wright|Lauderdale County|MS|52814|United States|-6|apartment| +39972|AAAAAAAAECMJAAAA|111|Broadway Fourth|Drive|Suite 200|Harmony|Island County|WA|95804|United States|-8|apartment| +39973|AAAAAAAAFCMJAAAA|375|8th South|Boulevard|Suite D|Springfield|Putnam County|GA|39303|United States|-5|apartment| +39974|AAAAAAAAGCMJAAAA|916|5th 8th|Street|Suite V|Edgewood|Carroll County|IL|60069|United States|-6|condo| +39975|AAAAAAAAHCMJAAAA|353|Spruce Oak|Parkway|Suite 80|Highland Park|Otter Tail County|MN|56534|United States|-6|condo| +39976|AAAAAAAAICMJAAAA|873|Sunset |RD|Suite C|Buena Vista|Schoolcraft County|MI|45752|United States|-5|apartment| +39977|AAAAAAAAJCMJAAAA|260|Valley |Ln|Suite 470|Highland|Switzerland County|IN|49454|United States|-5|single family| +39978|AAAAAAAAKCMJAAAA|956|Washington 11th|Blvd|Suite V|Highland Park|Leelanau County|MI|46534|United States|-5|apartment| +39979|AAAAAAAALCMJAAAA|960|Cedar 1st|Blvd|Suite G|Buena Vista|Washington County|MD|25752|United States|-5|apartment| +39980|AAAAAAAAMCMJAAAA|868|Wilson Sixth|Ln|Suite 380|Mount Zion|Yell County|AR|78054|United States|-6|single family| +39981|AAAAAAAANCMJAAAA|278|Willow |Pkwy|Suite E|Dewey|Early County|GA|31160|United States|-5|apartment| +39982|AAAAAAAAOCMJAAAA|602|6th |Lane|Suite 470|Antioch|Santa Cruz County|CA|98605|United States|-8|condo| +39983|AAAAAAAAPCMJAAAA|35|First |Lane|Suite P|Kingston|Eau Claire County|WI|54975|United States|-6|single family| +39984|AAAAAAAAADMJAAAA|988|Hickory Third|Cir.|Suite 260|Hamilton|Santa Fe County|NM|82808|United States|-7|condo| +39985|AAAAAAAABDMJAAAA|631|Third Spruce|Dr.|Suite 380|Newport|Montgomery County|PA|11521|United States|-5|apartment| +39986|AAAAAAAACDMJAAAA|733|Green Laurel|ST|Suite 160|Greenwood|Cameron County|TX|78828|United States|-6|single family| +39987|AAAAAAAADDMJAAAA|641|1st Sycamore|Ct.|Suite R|Freeman|Wells County|IN|42297|United States|-5|apartment| +39988|AAAAAAAAEDMJAAAA|786|Mill Sycamore|Parkway|Suite 60|Fairview|Noble County|OK|75709|United States|-6|apartment| +39989|AAAAAAAAFDMJAAAA|942|Ash Ash|Circle|Suite D|Deerfield|Jeff Davis County|GA|39840|United States|-5|single family| +39990|AAAAAAAAGDMJAAAA|484|Railroad Third|Ct.|Suite N|Centerville|Allegheny County|PA|10059|United States|-5|single family| +39991|AAAAAAAAHDMJAAAA|361|4th |Drive|Suite 490|Edgewater|Washington County|MS|50635|United States|-6|single family| +39992|AAAAAAAAIDMJAAAA|368|Dogwood 3rd|Ave|Suite 80|Oakwood|Wake County|NC|20169|United States|-5|condo| +39993|AAAAAAAAJDMJAAAA|382|Main |Boulevard|Suite 390|Union Hill|Fayette County|WV|27746|United States|-5|single family| +39994|AAAAAAAAKDMJAAAA|891|Sycamore |ST|Suite R|Pine Grove|Lancaster County|NE|64593|United States|-7|condo| +39995|AAAAAAAALDMJAAAA|866|Railroad Smith|Wy|Suite K|Pleasant Hill|Linn County|MO|63604|United States|-6|apartment| +39996|AAAAAAAAMDMJAAAA|211|Cherry |Street|Suite 480|Shady Grove|Guthrie County|IA|52812|United States|-6|condo| +39997|AAAAAAAANDMJAAAA|22|Jefferson |Ct.|Suite 260|Highland Park|Wayne County|MO|66534|United States|-6|condo| +39998|AAAAAAAAODMJAAAA|155|Lake Elm|Parkway|Suite 340|Georgetown|Duchesne County|UT|87057|United States|-7|apartment| +39999|AAAAAAAAPDMJAAAA|739|Elm 13th|Wy|Suite 230|Newport|Robeson County|NC|21521|United States|-5|apartment| +40000|AAAAAAAAAEMJAAAA|340|College |RD|Suite B|Star|Grant County|MN|50725|United States|-6|apartment| +40001|AAAAAAAABEMJAAAA|949|11th |Ct.|Suite P|Lebanon|Nance County|NE|62898|United States|-7|single family| +40002|AAAAAAAACEMJAAAA|514|West 12th|Blvd|Suite 70|Glenville|Hill County|TX|73445|United States|-6|condo| +40003|AAAAAAAADEMJAAAA|914|Cherry 7th|Dr.|Suite P|Mount Zion|Wise County|TX|78054|United States|-6|apartment| +40004|AAAAAAAAEEMJAAAA|263|Main |RD|Suite A|Mount Olive|Fulton County|KY|48059|United States|-6|single family| +40005|AAAAAAAAFEMJAAAA|510|Chestnut |Blvd|Suite R|Youngstown|Rockingham County|NH|00601|United States|-5|single family| +40006|AAAAAAAAGEMJAAAA|246|9th |Boulevard|Suite E|Oakdale|Caribou County|ID|89584|United States|-7|single family| +40007|AAAAAAAAHEMJAAAA|332|Sunset |ST|Suite I|Harmony|Limestone County|AL|35804|United States|-6|condo| +40008|AAAAAAAAIEMJAAAA|843|Birch |Parkway|Suite L|Georgetown|Jim Hogg County|TX|77057|United States|-6|condo| +40009|AAAAAAAAJEMJAAAA|443|East |Cir.|Suite 20|Unionville|McLean County|KY|41711|United States|-5|condo| +40010|AAAAAAAAKEMJAAAA|254|Park |Drive|Suite 80|Shiloh|Hampton County|SC|29275|United States|-5|single family| +40011|AAAAAAAALEMJAAAA|837|Chestnut |Dr.|Suite S|Lakeside|Haywood County|TN|39532|United States|-5|single family| +40012|AAAAAAAAMEMJAAAA|770|15th Washington|Avenue|Suite 10|Fayetteville|Clark County|MO|61732|United States|-6|single family| +40013|AAAAAAAANEMJAAAA||14th |Circle||||NV|84536|United States|-8|| +40014|AAAAAAAAOEMJAAAA|715|Adams |Avenue|Suite 40|Elkton|Bradley County|TN|33481|United States|-5|single family| +40015|AAAAAAAAPEMJAAAA|287|Adams |Boulevard|Suite 450|Oak Hill|Worth County|MO|67838|United States|-6|condo| +40016|AAAAAAAAAFMJAAAA|980|5th |Street|Suite P|Pleasant Valley|Orange County|FL|32477|United States|-5|single family| +40017|AAAAAAAABFMJAAAA|65|15th 1st|Ave|Suite 180|Summit|Calhoun County|MI|40499|United States|-5|single family| +40018|AAAAAAAACFMJAAAA|178|Cedar |Wy|Suite 300|Waterloo|Glacier County|MT|61675|United States|-7|single family| +40019|AAAAAAAADFMJAAAA|441|Sunset Valley|Drive|Suite 200|Shady Grove|Colfax County|NE|62812|United States|-6|single family| +40020|AAAAAAAAEFMJAAAA|721|Pine Church|Cir.|Suite E|Rosebud|Susquehanna County|PA|12244|United States|-5|apartment| +40021|AAAAAAAAFFMJAAAA|209|Cedar |Ave|Suite I|Plainview|Medina County|TX|73683|United States|-6|condo| +40022|AAAAAAAAGFMJAAAA|254|Ash |Parkway|Suite N|Wildwood|Webster County|GA|36871|United States|-5|apartment| +40023|AAAAAAAAHFMJAAAA|313|Oak |Circle|Suite 290|Hopewell|Waynesboro city|VA|20587|United States|-5|apartment| +40024|AAAAAAAAIFMJAAAA|252|Lincoln |Boulevard|Suite D|Newtown|Chaves County|NM|81749|United States|-7|single family| +40025|AAAAAAAAJFMJAAAA|491|Pine Meadow|Lane|Suite 460|Plainview|Lincoln County|OK|73683|United States|-6|apartment| +40026|AAAAAAAAKFMJAAAA|462|Meadow |RD|Suite 440|Mount Vernon|Crawford County|KS|68482|United States|-6|condo| +40027|AAAAAAAALFMJAAAA||||Suite K|Hamilton|Sumner County|||United States||condo| +40028|AAAAAAAAMFMJAAAA|550|West 15th|Ave|Suite N|Rocky Point|Wichita County|KS|61209|United States|-6|single family| +40029|AAAAAAAANFMJAAAA|539|14th Park|Avenue|Suite I|Maple Grove|Garfield County|MT|68252|United States|-7|single family| +40030|AAAAAAAAOFMJAAAA|201|11th |Lane|Suite X|Brownsville|DeBaca County|NM|89310|United States|-7|single family| +40031|AAAAAAAAPFMJAAAA|349|6th Poplar|Way|Suite 210|Lucas|Douglas County|GA|34554|United States|-5|condo| +40032|AAAAAAAAAGMJAAAA|385|View 11th|Circle|Suite D|Greenwood|Burke County|NC|28828|United States|-5|condo| +40033|AAAAAAAABGMJAAAA|422|8th |Dr.|Suite 430|Springfield|Jefferson County|IN|49303|United States|-5|single family| +40034|AAAAAAAACGMJAAAA|129|Walnut |Ct.|Suite 110|Franklin|Suwannee County|FL|39101|United States|-5|single family| +40035|AAAAAAAADGMJAAAA|193|Laurel Cherry|Ct.|Suite 80|Hopewell|Jefferson County|OK|70587|United States|-6|condo| +40036|AAAAAAAAEGMJAAAA|701|Smith |Boulevard|Suite F|Belmont|Marion County|IA|50191|United States|-6|apartment| +40037|AAAAAAAAFGMJAAAA|750|Spring |Street|Suite U|Clifton|Vernon County|MO|68014|United States|-6|apartment| +40038|AAAAAAAAGGMJAAAA|659|Davis |Boulevard|Suite 460|Forest Hills|DeKalb County|AL|39237|United States|-6|apartment| +40039|AAAAAAAAHGMJAAAA|77|View Walnut|Court|Suite B|New Hope|Upton County|TX|79431|United States|-6|single family| +40040|AAAAAAAAIGMJAAAA||1st ||Suite B|Spring Hill|Montmorency County||46787||-5|single family| +40041|AAAAAAAAJGMJAAAA|54|Johnson Forest|Avenue|Suite 350|Green Acres|Phelps County|MO|67683|United States|-6|apartment| +40042|AAAAAAAAKGMJAAAA|793|7th 4th|Avenue|Suite 440|Hillcrest|Warren County|IA|53003|United States|-6|condo| +40043|AAAAAAAALGMJAAAA|570|Ninth North|Court|Suite 200|Lakewood|DeSoto County|MS|58877|United States|-6|apartment| +40044|AAAAAAAAMGMJAAAA|521|Park Second|Lane|Suite T|Woodland|Merrimack County|NH|05454|United States|-5|condo| +40045|AAAAAAAANGMJAAAA|272|Central |Street|Suite 60|Jackson|Martin County|IN|49583|United States|-5|condo| +40046|AAAAAAAAOGMJAAAA|116|1st |Street|Suite E|Cedar Grove|Hunterdon County|NJ|01011|United States|-5|condo| +40047|AAAAAAAAPGMJAAAA|859|Johnson Church|Ct.|Suite C|Texas|Aransas County|TX|73342|United States|-6|condo| +40048|AAAAAAAAAHMJAAAA|818|Hill |Cir.|Suite 480|Arlington|Cumberland County|NC|26557|United States|-5|apartment| +40049|AAAAAAAABHMJAAAA|350|Fourth |Avenue|Suite 180|Woodlawn|Sequoyah County|OK|74098|United States|-6|condo| +40050|AAAAAAAACHMJAAAA|742|North 1st|Ln|Suite 220|Hillcrest|Powder River County|MT|63003|United States|-7|single family| +40051|AAAAAAAADHMJAAAA|380|Church |Parkway|Suite 150|Oak Hill|Marion County|MS|57838|United States|-6|condo| +40052|AAAAAAAAEHMJAAAA|165|14th Railroad|Road|Suite 400|Glendale|San Saba County|TX|73951|United States|-6|single family| +40053|AAAAAAAAFHMJAAAA|887|Wilson |Wy|Suite 330|Pleasant Hill|Pike County|AL|33604|United States|-6|single family| +40054|AAAAAAAAGHMJAAAA|41|5th |Parkway|Suite 80|Walnut Grove|Carbon County|UT|87752|United States|-7|condo| +40055|AAAAAAAAHHMJAAAA|247|Washington |Avenue|Suite C|Wildwood|Arapahoe County|CO|86871|United States|-7|single family| +40056|AAAAAAAAIHMJAAAA|780|1st |Road|Suite 50|Owens|Cherokee County|AL|32324|United States|-6|apartment| +40057|AAAAAAAAJHMJAAAA|193|Main |Circle|Suite P|Pine Grove|Colquitt County|GA|34593|United States|-5|apartment| +40058|AAAAAAAAKHMJAAAA|145|Adams 6th|Drive|Suite 430|Maple Grove|Johnson County|TN|38252|United States|-6|apartment| +40059|AAAAAAAALHMJAAAA|982|Spring Madison|Ln|Suite T|Antioch|Searcy County|AR|78605|United States|-6|condo| +40060|AAAAAAAAMHMJAAAA|124|Cherry |Circle|Suite V|Buena Vista|Gonzales County|TX|75752|United States|-6|condo| +40061|AAAAAAAANHMJAAAA|304|Central |Pkwy|Suite 360|Oak Hill|Marion County|AR|77838|United States|-6|condo| +40062|AAAAAAAAOHMJAAAA|571|5th 2nd|Parkway|Suite 240|Concord|Lyon County|KY|44107|United States|-5|apartment| +40063|AAAAAAAAPHMJAAAA|667|Broadway Maple|Ln|Suite 270|Pleasant Valley|Plaquemines Parish|LA|72477|United States|-6|condo| +40064|AAAAAAAAAIMJAAAA|575|Center Hill|Parkway|Suite K|Pleasant Grove|Coconino County|AZ|84136|United States|-7|apartment| +40065|AAAAAAAABIMJAAAA|794|Hill |Wy|Suite 390|Pleasant Valley|Columbia County|AR|72477|United States|-6|single family| +40066|AAAAAAAACIMJAAAA|277|Park Cherry|Ave|Suite K|Edgewood|Madison County|IN|40069|United States|-5|single family| +40067|AAAAAAAADIMJAAAA|86|Railroad Pine|Boulevard|Suite I|Brookville|Randolph County|AR|73524|United States|-6|condo| +40068|AAAAAAAAEIMJAAAA|345|4th Hickory|Circle|Suite 310|Ashland|Washtenaw County|MI|44244|United States|-5|single family| +40069|AAAAAAAAFIMJAAAA|154|3rd |RD|Suite R|Riverview|Ventura County|CA|99003|United States|-8|condo| +40070|AAAAAAAAGIMJAAAA|641|Willow Chestnut|Avenue|Suite 150|Edgewood|Fentress County|TN|30069|United States|-5|condo| +40071|AAAAAAAAHIMJAAAA|8|Jackson |Road|Suite D|Mountain View|Grady County|OK|74466|United States|-6|apartment| +40072|AAAAAAAAIIMJAAAA|167|Main Poplar|Ave|Suite L|Clinton|Clearwater County|MN|58222|United States|-6|apartment| +40073|AAAAAAAAJIMJAAAA|391|14th |Way|Suite 150|Bridgeport|Lafayette County|AR|75817|United States|-6|condo| +40074|AAAAAAAAKIMJAAAA|569|1st Spruce|Pkwy|Suite X|Florence|Pope County|IL|63394|United States|-6|single family| +40075|AAAAAAAALIMJAAAA|961|Walnut 9th|ST|Suite R|Valley View|Mercer County|KY|45124|United States|-5|apartment| +40076|AAAAAAAAMIMJAAAA|106|Wilson |Dr.|Suite C|Pine Grove|Yuba County|CA|94593|United States|-8|condo| +40077|AAAAAAAANIMJAAAA||View Lee||||Archuleta County||||-7|| +40078|AAAAAAAAOIMJAAAA|678|Williams |ST|Suite 230|Stringtown|Butler County|NE|60162|United States|-6|apartment| +40079|AAAAAAAAPIMJAAAA|534|Locust |Pkwy|Suite B|Jackson|Yazoo County|MS|59583|United States|-6|condo| +40080|AAAAAAAAAJMJAAAA|665|9th Ninth|Road|Suite 290|Summit|Charlotte County|FL|30499|United States|-5|condo| +40081|AAAAAAAABJMJAAAA|169|First |Cir.|Suite X|Concord|Lee County|NC|24107|United States|-5|condo| +40082|AAAAAAAACJMJAAAA|71|11th Walnut|Court|Suite 70|Five Forks|Wakulla County|FL|32293|United States|-5|single family| +40083|AAAAAAAADJMJAAAA|765|Chestnut South|Drive|Suite A|Antioch|Daggett County|UT|88605|United States|-7|condo| +40084|AAAAAAAAEJMJAAAA|471|Davis |Cir.|Suite L|Brownsville|Brewster County|TX|79310|United States|-6|apartment| +40085|AAAAAAAAFJMJAAAA|958|North Washington|Road|Suite U|Edgewood|Treutlen County|GA|30069|United States|-5|apartment| +40086|AAAAAAAAGJMJAAAA|19|Eigth |Dr.|Suite B|Stringtown|Caroline County|MD|20162|United States|-5|apartment| +40087|AAAAAAAAHJMJAAAA|427|Cedar Railroad|Blvd|Suite S|Enterprise|Transylvania County|NC|21757|United States|-5|apartment| +40088|AAAAAAAAIJMJAAAA|615|Willow |RD|Suite F|Belmont|Jefferson County|CO|80191|United States|-7|condo| +40089|AAAAAAAAJJMJAAAA|641|4th |Ct.|Suite T|Arlington|Guthrie County|IA|56557|United States|-6|condo| +40090|AAAAAAAAKJMJAAAA|940|Williams |Court|Suite 180|Centerville|Kenosha County|WI|50059|United States|-6|single family| +40091|AAAAAAAALJMJAAAA|847|Jackson 4th|Boulevard|Suite T|Summit|Graham County|NC|20499|United States|-5|apartment| +40092|AAAAAAAAMJMJAAAA|98|Miller 1st|Wy|Suite P|Bethel|Woodward County|OK|75281|United States|-6|apartment| +40093|AAAAAAAANJMJAAAA|651|Lincoln |Street|Suite 150|Spring Hill|Dare County|NC|26787|United States|-5|apartment| +40094|AAAAAAAAOJMJAAAA|843|Smith |RD|Suite 70|Pleasant Valley|Jackson County|CO|82477|United States|-7|condo| +40095|AAAAAAAAPJMJAAAA|659|Spring |Blvd|Suite K|Springfield|Dyer County|TN|39303|United States|-5|single family| +40096|AAAAAAAAAKMJAAAA|155|Central |Blvd|Suite S|Woodland|Adams County|ND|54854|United States|-6|condo| +40097|AAAAAAAABKMJAAAA|216|Laurel |Ln|Suite 390|Newtown|Laclede County|MO|61749|United States|-6|condo| +40098|AAAAAAAACKMJAAAA|604|Washington Main|Boulevard|Suite X|Crossroads|Madison County|NC|20534|United States|-5|apartment| +40099|AAAAAAAADKMJAAAA|768|Woodland |Parkway|Suite I|La Grange|Sumner County|KS|67941|United States|-6|apartment| +40100|AAAAAAAAEKMJAAAA|308|Mill |Court|Suite I|Sulphur Springs|Cherokee County|IA|58354|United States|-6|condo| +40101|AAAAAAAAFKMJAAAA|733|Mill |RD|Suite A|Crossroads|Maury County|TN|30534|United States|-6|apartment| +40102|AAAAAAAAGKMJAAAA|646|Smith |Pkwy|Suite 160|Jackson|Emporia city|VA|29583|United States|-5|apartment| +40103|AAAAAAAAHKMJAAAA|385|Sycamore 7th|Ct.|Suite 230|Cedar Grove|Washington County|CO|80411|United States|-7|apartment| +40104|AAAAAAAAIKMJAAAA|286|View 1st|Avenue|Suite I|Brownsville|Otter Tail County|MN|59310|United States|-6|apartment| +40105|AAAAAAAAJKMJAAAA|358|Spring First|Way|Suite 150|Shady Grove|Livingston County|NY|12812|United States|-5|single family| +40106|AAAAAAAAKKMJAAAA|408|Pine Birch|ST|Suite 50|Bridgeport|Jefferson County|WI|55817|United States|-6|single family| +40107|AAAAAAAALKMJAAAA|396|Walnut |Cir.|Suite P|Athens|Baldwin County|GA|36796|United States|-5|single family| +40108|AAAAAAAAMKMJAAAA|351|Meadow 1st|Way|Suite W|Fairfield|Waushara County|WI|56192|United States|-6|apartment| +40109|AAAAAAAANKMJAAAA|207|Cedar Broadway|Dr.|Suite G|Clifton|Baltimore County|MD|28014|United States|-5|condo| +40110|AAAAAAAAOKMJAAAA|939|9th |Dr.|Suite 20|Elizabeth|Lynn County|TX|72935|United States|-6|condo| +40111|AAAAAAAAPKMJAAAA|171|13th |Avenue|Suite 60|Marion|Camden County|NC|20399|United States|-5|single family| +40112|AAAAAAAAALMJAAAA|519|Lakeview |Ct.|Suite 130|Shiloh|Nelson County|ND|59275|United States|-6|condo| +40113|AAAAAAAABLMJAAAA|419|Chestnut 4th|Wy|Suite 40|Farmington|Covington County|AL|39145|United States|-6|apartment| +40114|AAAAAAAACLMJAAAA|426|Third |Boulevard|Suite N|Caledonia|Blaine County|NE|67411|United States|-6|condo| +40115|AAAAAAAADLMJAAAA|180|West 8th|RD|Suite 280|Ashland|Cibola County|NM|84244|United States|-7|condo| +40116|AAAAAAAAELMJAAAA|539|Washington Oak|RD|Suite E|Highland Park|Beckham County|OK|76534|United States|-6|single family| +40117|AAAAAAAAFLMJAAAA|564|Third Meadow|Circle|Suite V|Bridgeport|Gage County|NE|65817|United States|-6|condo| +40118|AAAAAAAAGLMJAAAA|907|Wilson |Pkwy|Suite N|Oak Ridge|Caroline County|VA|28371|United States|-5|apartment| +40119|AAAAAAAAHLMJAAAA|306|North |Avenue|Suite 170|Shore Acres|Wahkiakum County|WA|92724|United States|-8|single family| +40120|AAAAAAAAILMJAAAA|403|Mill |Ave|Suite A|Riverdale|Wood County|TX|79391|United States|-6|condo| +40121|AAAAAAAAJLMJAAAA|175|Forest Fifteenth|Cir.|Suite 120|Red Hill|Trego County|KS|64338|United States|-6|single family| +40122|AAAAAAAAKLMJAAAA|565|Second |ST|Suite 60|Peru|Winona County|MN|50302|United States|-6|apartment| +40123|AAAAAAAALLMJAAAA|878|Dogwood 13th|Avenue|Suite N|Pleasant Grove|Tallahatchie County|MS|54136|United States|-6|single family| +40124|AAAAAAAAMLMJAAAA|||Parkway||||||United States||condo| +40125|AAAAAAAANLMJAAAA|171|North Madison|Boulevard|Suite M|Enterprise|Henry County|AL|31757|United States|-6|condo| +40126|AAAAAAAAOLMJAAAA|420|Woodland Davis|Dr.|Suite 320|Shiloh|Lake County|OH|49275|United States|-5|condo| +40127|AAAAAAAAPLMJAAAA|122|First Third|Avenue|Suite 370|Mount Vernon|Schuylkill County|PA|18482|United States|-5|condo| +40128|AAAAAAAAAMMJAAAA|742|First 1st|ST|Suite J|Mount Pleasant|Cooper County|MO|61933|United States|-6|apartment| +40129|AAAAAAAABMMJAAAA|236|Highland Central|Wy|Suite M|Stringtown|Essex County|NJ|00762|United States|-5|condo| +40130|AAAAAAAACMMJAAAA|878|Sixth |Boulevard|Suite 300|Plainview|Richland County|WI|53683|United States|-6|apartment| +40131|AAAAAAAADMMJAAAA|776|Adams |Court|Suite 20|Plainview|Upshur County|TX|73683|United States|-6|condo| +40132|AAAAAAAAEMMJAAAA|5|8th Seventh|Avenue|Suite 440|Bunker Hill|Vigo County|IN|40150|United States|-5|condo| +40133|AAAAAAAAFMMJAAAA|360|Johnson |Cir.|Suite N|Clifton|Arenac County|MI|48014|United States|-5|apartment| +40134|AAAAAAAAGMMJAAAA|971|Park |Way|Suite F|Red Hill|Cherokee County|SC|24338|United States|-5|single family| +40135|AAAAAAAAHMMJAAAA|446|Forest |Court|Suite K|Union|Mobile County|AL|38721|United States|-6|single family| +40136|AAAAAAAAIMMJAAAA|840|Park |Way|Suite H|Jackson|Lawrence County|SD|59583|United States|-7|apartment| +40137|AAAAAAAAJMMJAAAA|340|Washington 3rd|Lane|Suite 30|Salem|Wayne County|NE|68048|United States|-7|single family| +40138|AAAAAAAAKMMJAAAA|273|Dogwood |Boulevard|Suite W|Greenfield|Hardin County|TN|35038|United States|-5|apartment| +40139|AAAAAAAALMMJAAAA|303|Birch |Wy|Suite M|Sulphur Springs|Rockingham County|NH|08954|United States|-5|single family| +40140|AAAAAAAAMMMJAAAA|431|Lake |Ct.|Suite N|Hillcrest|Montrose County|CO|83003|United States|-7|single family| +40141|AAAAAAAANMMJAAAA|478|||||Marion County|AL|38773|||| +40142|AAAAAAAAOMMJAAAA|816|First 12th|Circle|Suite 110|Newtown|Avery County|NC|21749|United States|-5|apartment| +40143|AAAAAAAAPMMJAAAA|412|5th Ridge|Lane|Suite 370|Spring Grove|Jefferson County|KS|66719|United States|-6|condo| +40144|AAAAAAAAANMJAAAA|28|Willow Sycamore|Way|Suite 320|Mountain View|Woodbury County|IA|54466|United States|-6|single family| +40145|AAAAAAAABNMJAAAA|778|3rd |Street|Suite 250|Summit|Clinch County|GA|30499|United States|-5|condo| +40146|AAAAAAAACNMJAAAA|693|Willow West|Boulevard|Suite 490|Centerville|Knox County|OH|40059|United States|-5|single family| +40147|AAAAAAAADNMJAAAA|505|Park Poplar|Street|Suite L|Concord|Madison County|MS|54107|United States|-6|condo| +40148|AAAAAAAAENMJAAAA|||Wy|Suite H||Dooly County|GA||United States||| +40149|AAAAAAAAFNMJAAAA|28|Pine 3rd|Wy|Suite V|The Meadows|Mellette County|SD|50026|United States|-7|apartment| +40150|AAAAAAAAGNMJAAAA|817|Locust Lincoln|Road|Suite N|Fairfield|Glasscock County|TX|76192|United States|-6|single family| +40151|AAAAAAAAHNMJAAAA|607|East |Circle|Suite 150|Globe|Hancock County|GA|39614|United States|-5|single family| +40152|AAAAAAAAINMJAAAA||Wilson |Avenue|||Hood County||70169|||| +40153|AAAAAAAAJNMJAAAA|525|Hill |Road|Suite 400|Yorktown|Chicot County|AR|70732|United States|-6|condo| +40154|AAAAAAAAKNMJAAAA|757|Fourth Oak|Court|Suite 270|Fairfield|Pottawatomie County|OK|76192|United States|-6|apartment| +40155|AAAAAAAALNMJAAAA|859|Fifth |Avenue|Suite Q|Bethel|Anne Arundel County|MD|25281|United States|-5|condo| +40156|AAAAAAAAMNMJAAAA|336|North |Wy|Suite 190|Edgewood|Ida County|IA|50069|United States|-6|apartment| +40157|AAAAAAAANNMJAAAA|242|Birch Miller|ST|Suite 270|Greenville|Seminole County|FL|31387|United States|-5|single family| +40158|AAAAAAAAONMJAAAA|635|Smith |Blvd|Suite R|Forest|Christian County|KY|47537|United States|-6|condo| +40159|AAAAAAAAPNMJAAAA|351|Fourth |Road|Suite 310|Bennett|Edmonson County|KY|41715|United States|-6|condo| +40160|AAAAAAAAAOMJAAAA|295|Fifth 12th|Cir.|Suite N|Oakdale|Coconino County|AZ|89584|United States|-7|condo| +40161|AAAAAAAABOMJAAAA|127|Sycamore |Dr.|Suite 370|Stringtown|Maries County|MO|60162|United States|-6|apartment| +40162|AAAAAAAACOMJAAAA|97|Oak 1st|Wy|Suite 150|Spring Hill|Roberts County|TX|76787|United States|-6|apartment| +40163|AAAAAAAADOMJAAAA|696|Locust Laurel|Avenue|Suite N|Pleasant Valley|Elmore County|AL|32477|United States|-6|single family| +40164|AAAAAAAAEOMJAAAA|110|Mill First|Pkwy|Suite 350|Edgewood|Hocking County|OH|40069|United States|-5|single family| +40165|AAAAAAAAFOMJAAAA|830|Walnut Ridge|Parkway|Suite 440|Lakeville|Queens County|NY|18811|United States|-5|apartment| +40166|AAAAAAAAGOMJAAAA|353|Church |Pkwy|Suite 200|Springfield|Midland County|TX|79303|United States|-6|apartment| +40167|AAAAAAAAHOMJAAAA|733|Park Lincoln|Parkway|Suite C|Arlington|Pierce County|GA|36557|United States|-5|single family| +40168|AAAAAAAAIOMJAAAA|568|Park 15th|Blvd|Suite E|Glenwood|Candler County|GA|33511|United States|-5|condo| +40169|AAAAAAAAJOMJAAAA|687|Park |Drive|Suite 330|Mount Zion|Columbia County|GA|38054|United States|-5|single family| +40170|AAAAAAAAKOMJAAAA|702|Dogwood 2nd|ST|Suite R|Mechanicsburg|Kingsbury County|SD|52219|United States|-7|condo| +40171|AAAAAAAALOMJAAAA|873|Ridge |Parkway|Suite 280|Hardy|Lincoln County|WY|85354|United States|-7|condo| +40172|AAAAAAAAMOMJAAAA|617|Cedar |Lane|Suite M|Antioch|Newport County|RI|09205|United States|-5|single family| +40173|AAAAAAAANOMJAAAA|260|View ||Suite 80|||NY|19391||-5|| +40174|AAAAAAAAOOMJAAAA|639|Broadway Meadow|Ln|Suite 10|Liberty|Richmond County|NC|23451|United States|-5|single family| +40175|AAAAAAAAPOMJAAAA|70|Madison |Circle|Suite B|Pleasant Valley|Brown County|MN|52477|United States|-6|single family| +40176|AAAAAAAAAPMJAAAA|172|Mill |ST|Suite 240|Glenwood|Wise County|TX|73511|United States|-6|single family| +40177|AAAAAAAABPMJAAAA|990|2nd |Parkway|Suite 210|Perkins|Knox County|TX|71852|United States|-6|single family| +40178|AAAAAAAACPMJAAAA|407|First |Pkwy|Suite 250|Union|Treasure County|MT|68721|United States|-7|single family| +40179|AAAAAAAADPMJAAAA|634|Laurel |ST|Suite R|Five Forks|Tuscaloosa County|AL|32293|United States|-6|apartment| +40180|AAAAAAAAEPMJAAAA|357|Sixth East|Road|Suite X|Sulphur Springs|Phelps County|NE|68354|United States|-7|single family| +40181|AAAAAAAAFPMJAAAA|826|8th |Lane|Suite 450|Edgewood|Mesa County|CO|80069|United States|-7|condo| +40182|AAAAAAAAGPMJAAAA|62|Maple |Circle|Suite M|Antioch|Fulton County|IL|68605|United States|-6|single family| +40183|AAAAAAAAHPMJAAAA|654|||||||31675|United States|-6|apartment| +40184|AAAAAAAAIPMJAAAA|954|Jackson 8th|Court|||||73511|United States|-6|apartment| +40185|AAAAAAAAJPMJAAAA|979|River West|Parkway|Suite 140|Mount Zion|Northampton County|PA|18054|United States|-5|condo| +40186|AAAAAAAAKPMJAAAA|470|Smith |Street|Suite 40|Bethel|Chase County|NE|65281|United States|-6|apartment| +40187|AAAAAAAALPMJAAAA|545|5th |Avenue|Suite E|Green Acres|Runnels County|TX|77683|United States|-6|condo| +40188|AAAAAAAAMPMJAAAA|805|Cedar Lakeview|Court|Suite E|Woodland|Muskogee County|OK|74854|United States|-6|single family| +40189|AAAAAAAANPMJAAAA|556|Miller View|Way|Suite 60|Springfield|Horry County|SC|29303|United States|-5|condo| +40190|AAAAAAAAOPMJAAAA|929|Spring |Boulevard|Suite 210|Mount Pleasant|Jackson County|TN|31933|United States|-5|condo| +40191|AAAAAAAAPPMJAAAA|114|Lake Oak|Cir.|Suite 100|Walnut Grove|Alpine County|CA|97752|United States|-8|single family| +40192|AAAAAAAAAANJAAAA|697|Church Lake|Circle|Suite 470|Spring Hill|Chambers County|TX|76787|United States|-6|apartment| +40193|AAAAAAAABANJAAAA|457|Main |Road|Suite 70|Warwick|DeWitt County|TX|71398|United States|-6|condo| +40194|AAAAAAAACANJAAAA|399|Forest 6th|Pkwy|Suite N|Sunnyside|Appomattox County|VA|21952|United States|-5|single family| +40195|AAAAAAAADANJAAAA|163|Ash Main|Road|Suite F|White Oak|Mille Lacs County|MN|56668|United States|-6|condo| +40196|AAAAAAAAEANJAAAA|693|South |Ln|Suite X|Sunnyside|Sullivan County|MO|61952|United States|-6|condo| +40197|AAAAAAAAFANJAAAA|634|Hickory |Blvd|Suite B|Hopewell|Stutsman County|ND|50587|United States|-6|condo| +40198|AAAAAAAAGANJAAAA|519|Lake |Cir.|Suite 410|Antioch|Conejos County|CO|88605|United States|-7|condo| +40199|AAAAAAAAHANJAAAA|40|Elm |Street|Suite 190|Marion|Ashe County|NC|20399|United States|-5|condo| +40200|AAAAAAAAIANJAAAA|98|Hickory 7th|Avenue|Suite G|Mount Zion|Placer County|CA|98054|United States|-8|condo| +40201|AAAAAAAAJANJAAAA||Lee ||Suite 80|Riverside||||United States||| +40202|AAAAAAAAKANJAAAA|935|Park Elm|Boulevard|Suite 320|Pleasant Hill|Mercer County|PA|13604|United States|-5|apartment| +40203|AAAAAAAALANJAAAA|334|Sixth Central|Road|Suite P|Arlington|Santa Rosa County|FL|36557|United States|-5|single family| +40204|AAAAAAAAMANJAAAA|535|Walnut 8th|Street|Suite 120|Buena Vista|East Baton Rouge Parish|LA|75752|United States|-6|single family| +40205|AAAAAAAANANJAAAA|127|South |Lane|Suite 80|Sunnyside|Denton County|TX|71952|United States|-6|condo| +40206|AAAAAAAAOANJAAAA|919|Dogwood 8th|Ave|Suite Q|Mount Olive|Warren County|OH|48059|United States|-5|single family| +40207|AAAAAAAAPANJAAAA|18|Lincoln Fourth|Court|Suite N|Oak Hill|Sanpete County|UT|87838|United States|-7|single family| +40208|AAAAAAAAABNJAAAA|66|11th Chestnut|Drive|Suite I|Oakwood|Richmond County|GA|30169|United States|-5|apartment| +40209|AAAAAAAABBNJAAAA|718|Woodland |Road|Suite I|Farmington|Tillman County|OK|79145|United States|-6|single family| +40210|AAAAAAAACBNJAAAA|137|Forest 8th|Court|Suite K|Sunnyside|Monroe County|IA|51952|United States|-6|apartment| +40211|AAAAAAAADBNJAAAA|687|8th Williams|Ct.|Suite 170|Brownsville|Floyd County|IN|49310|United States|-5|single family| +40212|AAAAAAAAEBNJAAAA|288|10th Dogwood|Circle|Suite J|Georgetown|Upshur County|WV|27057|United States|-5|single family| +40213|AAAAAAAAFBNJAAAA|539|Walnut |Street|Suite 0|Newtown|Moffat County|CO|81749|United States|-7|single family| +40214|AAAAAAAAGBNJAAAA|||Court||Hamilton|Will County|IL|62808|||| +40215|AAAAAAAAHBNJAAAA|808|Lee Wilson|Road|Suite O|Woodville|Greene County|IA|54289|United States|-6|condo| +40216|AAAAAAAAIBNJAAAA|694|10th Park|Pkwy|Suite 350|Cedar Grove|Los Angeles County|CA|90411|United States|-8|condo| +40217|AAAAAAAAJBNJAAAA|156|Walnut |Boulevard|Suite P|Spring Hill|Los Alamos County|NM|86787|United States|-7|condo| +40218|AAAAAAAAKBNJAAAA|443|8th Lake|Ave|Suite 160|Greenfield|Wolfe County|KY|45038|United States|-5|single family| +40219|AAAAAAAALBNJAAAA|924|Third |Blvd|Suite F|Friendship|Burke County|ND|54536|United States|-6|single family| +40220|AAAAAAAAMBNJAAAA|876|Jackson |Avenue|Suite S|Hopewell|Orange County|TX|70587|United States|-6|condo| +40221|AAAAAAAANBNJAAAA|619|First North|Boulevard|Suite A|Mount Olive|Grenada County|MS|58059|United States|-6|single family| +40222|AAAAAAAAOBNJAAAA|900|Washington |Lane|Suite 330|Kingston|Cumberland County|NC|24975|United States|-5|apartment| +40223|AAAAAAAAPBNJAAAA|597|Second River|Road|Suite 80|Shiloh|Beadle County|SD|59275|United States|-6|apartment| +40224|AAAAAAAAACNJAAAA|562|Hillcrest Railroad|Circle|Suite 90|Harmony|Cedar County|IA|55804|United States|-6|single family| +40225|AAAAAAAABCNJAAAA|131|Pine 5th|Ct.|Suite 80|Bridgeport|Armstrong County|TX|75817|United States|-6|apartment| +40226|AAAAAAAACCNJAAAA|527|East |Drive|Suite N|Mount Pleasant|Mercer County|KY|41933|United States|-5|condo| +40227|AAAAAAAADCNJAAAA|222|Jefferson Dogwood|RD|Suite 90|Mount Zion|Kent County|TX|78054|United States|-6|single family| +40228|AAAAAAAAECNJAAAA|779|Ninth Miller|Ct.|Suite N|Oak Grove|Harding County|NM|88370|United States|-7|condo| +40229|AAAAAAAAFCNJAAAA|525|Green |Ave|Suite 10|Clifton|Summit County|UT|88014|United States|-7|condo| +40230|AAAAAAAAGCNJAAAA|469|10th 12th|Ave|Suite 200|Ashland|Boyd County|KY|44244|United States|-6|apartment| +40231|AAAAAAAAHCNJAAAA|290|1st Smith|Ave|Suite 400|Five Forks|Tate County|MS|52293|United States|-6|condo| +40232|AAAAAAAAICNJAAAA|906|River Fifth|Blvd|Suite C|Georgetown|Menard County|TX|77057|United States|-6|condo| +40233|AAAAAAAAJCNJAAAA|233|Main |RD|Suite 150|Concord|Converse County|WY|84107|United States|-7|condo| +40234|AAAAAAAAKCNJAAAA|721|Oak 2nd|Court|Suite P|Springdale|Orange County|NY|18883|United States|-5|condo| +40235|AAAAAAAALCNJAAAA|388|11th 7th|Court|Suite 370|Woodlawn|Franklin County|PA|14098|United States|-5|condo| +40236|AAAAAAAAMCNJAAAA|866|8th Forest|ST|Suite W|Cedar Grove|Stanislaus County|CA|90411|United States|-8|condo| +40237|AAAAAAAANCNJAAAA|766|Ridge 7th|Ave|Suite 330|Winchester|Shelby County|TX|73252|United States|-6|apartment| +40238|AAAAAAAAOCNJAAAA|720|East |Dr.|Suite L|Union|Morgan County|KY|48721|United States|-5|single family| +40239|AAAAAAAAPCNJAAAA|320|Lincoln Fourth|Blvd|Suite G|Summerville|Wilcox County|GA|32033|United States|-5|condo| +40240|AAAAAAAAADNJAAAA|354|Meadow Lee|Street|Suite 270|Georgetown|Garfield County|CO|87057|United States|-7|single family| +40241|AAAAAAAABDNJAAAA|861|Maple West|Ct.|Suite 160|Hillcrest|Jefferson County|IN|43003|United States|-5|single family| +40242|AAAAAAAACDNJAAAA|108|North Williams|Circle|Suite J|Woodlawn|Corson County|SD|54098|United States|-6|single family| +40243|AAAAAAAADDNJAAAA|950|3rd 2nd|Avenue|Suite 130|Lakeview|Bastrop County|TX|78579|United States|-6|condo| +40244|AAAAAAAAEDNJAAAA|649|7th Green|Street|Suite X|Union|Hancock County|OH|48721|United States|-5|apartment| +40245|AAAAAAAAFDNJAAAA|315|Mill |Blvd|Suite Y|Jamestown|Hidalgo County|NM|86867|United States|-7|apartment| +40246|AAAAAAAAGDNJAAAA|975|North |Way|Suite 190|Spring Valley|Beaufort County|SC|26060|United States|-5|condo| +40247|AAAAAAAAHDNJAAAA|599|North |Street|Suite G|Jackson|Yellow Medicine County|MN|59583|United States|-6|apartment| +40248|AAAAAAAAIDNJAAAA|290|Railroad Smith|Blvd|Suite 310|Unionville|Chattooga County|GA|31711|United States|-5|condo| +40249|AAAAAAAAJDNJAAAA|248|Spring 9th|ST|Suite J|Walnut Grove|Smith County|TN|37752|United States|-6|condo| +40250|AAAAAAAAKDNJAAAA|166|10th |Drive|Suite T|Lakeview|Humboldt County|IA|58579|United States|-6|apartment| +40251|AAAAAAAALDNJAAAA|475|Davis Washington|Ln|Suite 220|Maple Grove|Galax city|VA|28252|United States|-5|condo| +40252|AAAAAAAAMDNJAAAA|||Road|Suite 80||Hampton city|VA||United States|-5|| +40253|AAAAAAAANDNJAAAA|245|9th |ST|Suite Q|Pleasant Hill|Bond County|IL|63604|United States|-6|apartment| +40254|AAAAAAAAODNJAAAA|748|Oak |Dr.|Suite M|Providence|Slope County|ND|56614|United States|-6|apartment| +40255|AAAAAAAAPDNJAAAA|88|Fourth Lake|Boulevard|Suite 300|Forest Hills|Jackson County|MI|49237|United States|-5|apartment| +40256|AAAAAAAAAENJAAAA|940|2nd Madison|Dr.|Suite U|Arlington|Wabash County|IL|66557|United States|-6|apartment| +40257|AAAAAAAABENJAAAA|733|Lakeview |Cir.|Suite 450|Fairview|Echols County|GA|35709|United States|-5|single family| +40258|AAAAAAAACENJAAAA|127|2nd View|Road|Suite 300|Walnut Grove|Beaufort County|NC|27752|United States|-5|apartment| +40259|AAAAAAAADENJAAAA|697|Cedar 9th|Road|Suite O|Lebanon|Northampton County|PA|12898|United States|-5|condo| +40260|AAAAAAAAEENJAAAA|616|Woodland Church|Circle|Suite C|Providence|Mower County|MN|56614|United States|-6|condo| +40261|AAAAAAAAFENJAAAA|718|8th |Boulevard|Suite 190|Springdale|Indian River County|FL|38883|United States|-5|single family| +40262|AAAAAAAAGENJAAAA|3|Sixth Tenth|Court|Suite 100|Jackson|Stephenson County|IL|69583|United States|-6|condo| +40263|AAAAAAAAHENJAAAA|246|9th Sunset|Boulevard|Suite L|New Hope|Jefferson County|WA|99431|United States|-8|single family| +40264|AAAAAAAAIENJAAAA|983|Hill Sixth|Road|Suite O|Maple Grove|Union County|GA|38252|United States|-5|single family| +40265|AAAAAAAAJENJAAAA|801|Maple |Ln|Suite W|Brownsville|Hickman County|TN|39310|United States|-5|apartment| +40266|AAAAAAAAKENJAAAA|962|4th |Avenue|Suite I|Shady Grove|Lawrence County|PA|12812|United States|-5|single family| +40267|AAAAAAAALENJAAAA|53|Highland Center|Blvd|Suite 430|Riverdale|Oxford County|ME|09991|United States|-5|condo| +40268|AAAAAAAAMENJAAAA|686|Sixth |Drive|Suite R|Wayland|Rutherford County|NC|25115|United States|-5|apartment| +40269|AAAAAAAANENJAAAA|919|Hillcrest |Boulevard|Suite N|Pleasant Valley|Broward County|FL|32477|United States|-5|apartment| +40270|AAAAAAAAOENJAAAA|853|Walnut |Pkwy|Suite A|Oakwood|Jefferson Davis County|MS|50169|United States|-6|apartment| +40271|AAAAAAAAPENJAAAA|183|3rd River|Road|Suite 450|Linden|Bandera County|TX|71187|United States|-6|single family| +40272|AAAAAAAAAFNJAAAA|652|7th |Ave|Suite I|Arlington|Dawson County|NE|66557|United States|-6|condo| +40273|AAAAAAAABFNJAAAA|410|Cherry |Ave|Suite D|Sanford|Monroe County|WI|59737|United States|-6|condo| +40274|AAAAAAAACFNJAAAA|292|15th |Blvd|Suite 460|Roy|Onondaga County|NY|10744|United States|-5|apartment| +40275|AAAAAAAADFNJAAAA|421|Washington |Street|Suite S|Edwards|Barry County|MO|61409|United States|-6|single family| +40276|AAAAAAAAEFNJAAAA|988|Maple |RD|Suite 190|Spring Hill|Ford County|IL|66787|United States|-6|single family| +40277|AAAAAAAAFFNJAAAA|39|1st North|Avenue|Suite C|Antioch|Atlantic County|NJ|09205|United States|-5|single family| +40278|AAAAAAAAGFNJAAAA|730|Ninth |Pkwy|Suite K|White Oak|Taney County|MO|66668|United States|-6|condo| +40279|AAAAAAAAHFNJAAAA|269|||||||51904|United States|-6|apartment| +40280|AAAAAAAAIFNJAAAA|968|Third Cedar|RD|Suite N|Bunker Hill|Tattnall County|GA|30150|United States|-5|apartment| +40281|AAAAAAAAJFNJAAAA|973|Johnson 6th|Wy|Suite Y|Harmony|Chariton County|MO|65804|United States|-6|apartment| +40282|AAAAAAAAKFNJAAAA|69|Maple |Street|Suite Q|Lebanon|San Juan County|WA|92898|United States|-8|single family| +40283|AAAAAAAALFNJAAAA|773|North |Way|Suite 400|Mountain View|Decatur County|IA|54466|United States|-6|single family| +40284|AAAAAAAAMFNJAAAA|262|Highland |ST|Suite 180|Greenville|Cooper County|MO|61387|United States|-6|apartment| +40285|AAAAAAAANFNJAAAA|457|Adams Davis|Circle|Suite V|Willard|Hamilton County|NY|17039|United States|-5|condo| +40286|AAAAAAAAOFNJAAAA|64|4th Johnson|Road|Suite H|Greenville|Broward County|FL|31387|United States|-5|single family| +40287|AAAAAAAAPFNJAAAA|629|Walnut Broadway|Way|Suite N|Lakeview|Kendall County|IL|68579|United States|-6|single family| +40288|AAAAAAAAAGNJAAAA|759|Third |Wy|Suite Q|Page|Sheridan County|MT|60296|United States|-7|apartment| +40289|AAAAAAAABGNJAAAA|53|Smith |Boulevard|Suite C|Pleasant Valley|Fremont County|CO|82477|United States|-7|apartment| +40290|AAAAAAAACGNJAAAA|727|Railroad |Ct.|Suite 230|Perkins|Jackson County|OR|91852|United States|-8|single family| +40291|AAAAAAAADGNJAAAA|440|Woodland Meadow|Ln|Suite 160|Highland|Bacon County|GA|39454|United States|-5|apartment| +40292|AAAAAAAAEGNJAAAA|478|Wilson |Lane|Suite B|Pleasant Grove|Orangeburg County|SC|24136|United States|-5|apartment| +40293|AAAAAAAAFGNJAAAA|||Cir.|Suite R||Knox County|NE|67838|||apartment| +40294|AAAAAAAAGGNJAAAA|311|Ash Hickory|Avenue|Suite E|Union|Upshur County|WV|28721|United States|-5|condo| +40295|AAAAAAAAHGNJAAAA|787|Birch Oak|Lane|Suite L|Fairview|Pueblo County|CO|85709|United States|-7|single family| +40296|AAAAAAAAIGNJAAAA|983|Ridge |Road|Suite 60|Shady Grove|Morris County|NJ|03412|United States|-5|single family| +40297|AAAAAAAAJGNJAAAA|304|Forest |Street|Suite S|Frogtown|Rowan County|KY|48784|United States|-5|condo| +40298|AAAAAAAAKGNJAAAA|616|Broadway |Lane|Suite 80|Spring Hill|Renville County|ND|56787|United States|-6|condo| +40299|AAAAAAAALGNJAAAA|917|Ash |Blvd|Suite 140|Farmington|Dona Ana County|NM|89145|United States|-7|apartment| +40300|AAAAAAAAMGNJAAAA|896|Chestnut Church|Court|Suite 420|Argyle|Crane County|TX|78722|United States|-6|condo| +40301|AAAAAAAANGNJAAAA|130|Williams River|Circle|Suite V|Jackson|Howard County|IN|49583|United States|-5|single family| +40302|AAAAAAAAOGNJAAAA|661|Railroad 9th|Circle|Suite 50|Glendale|Crawford County|IL|63951|United States|-6|single family| +40303|AAAAAAAAPGNJAAAA|343|Ninth Oak|ST|Suite V|Lakeview|Anderson County|TN|38579|United States|-5|single family| +40304|AAAAAAAAAHNJAAAA|325|Oak |Parkway|Suite I|Deerfield|Henry County|IN|49840|United States|-5|condo| +40305|AAAAAAAABHNJAAAA|168|Oak |Street|Suite 70|Oakland|Fairfax city|VA|29843|United States|-5|apartment| +40306|AAAAAAAACHNJAAAA|||||Highland|Wilson County|||||| +40307|AAAAAAAADHNJAAAA|927|Railroad |Court|Suite 460|New Hope|Chaves County|NM|89431|United States|-7|single family| +40308|AAAAAAAAEHNJAAAA|921|Willow Park|Pkwy|Suite S|Greendale|Jackson County|FL|39343|United States|-5|single family| +40309|AAAAAAAAFHNJAAAA|604|West |ST|Suite 60|Waterloo|Guernsey County|OH|41675|United States|-5|apartment| +40310|AAAAAAAAGHNJAAAA|2|Maple |Blvd|Suite 490|Riverside|Upton County|TX|79231|United States|-6|single family| +40311|AAAAAAAAHHNJAAAA|477|Elevnth Highland|Circle|Suite 360|Stringtown|Jackson County|IA|50162|United States|-6|single family| +40312|AAAAAAAAIHNJAAAA|691|First Birch|Ave|Suite W|Maple Grove|Cleburne County|AR|78252|United States|-6|single family| +40313|AAAAAAAAJHNJAAAA|877|Ash 4th|Wy|Suite A|Sulphur Springs|Kingman County|KS|68354|United States|-6|single family| +40314|AAAAAAAAKHNJAAAA|17|River |Ln|Suite I|Shiloh|Buffalo County|WI|59275|United States|-6|single family| +40315|AAAAAAAALHNJAAAA|242|Smith |Cir.|Suite B|Lebanon|Osborne County|KS|62898|United States|-6|condo| +40316|AAAAAAAAMHNJAAAA|314|Jefferson Mill|Ave|Suite 320|Nottingham|Knox County|OH|44074|United States|-5|condo| +40317|AAAAAAAANHNJAAAA|829|Poplar 8th|Ave|Suite 460|Hillcrest|Lee County|FL|33003|United States|-5|apartment| +40318|AAAAAAAAOHNJAAAA|651|West |Boulevard|Suite A|Forest Hills|Polk County|FL|39237|United States|-5|condo| +40319|AAAAAAAAPHNJAAAA|422|Ridge |Wy|Suite D|Concord|Mahnomen County|MN|54107|United States|-6|condo| +40320|AAAAAAAAAINJAAAA|897|8th |Cir.|Suite E|Highland Park|Lauderdale County|MS|56534|United States|-6|condo| +40321|AAAAAAAABINJAAAA|48|Lake |ST|Suite O|Riverview|Tioga County|NY|19003|United States|-5|single family| +40322|AAAAAAAACINJAAAA|470|Jefferson |Way|Suite 40|Glendale|Jeff Davis County|GA|33951|United States|-5|single family| +40323|AAAAAAAADINJAAAA|110|Oak 14th|Wy|Suite 480|Plainview|Floyd County|IA|53683|United States|-6|condo| +40324|AAAAAAAAEINJAAAA|792|Mill |Drive|Suite 40|Maple Grove|Boone County|MO|68252|United States|-6|single family| +40325|AAAAAAAAFINJAAAA|708|Lee |Cir.|Suite 330|Greenwood|Graham County|KS|68828|United States|-6|condo| +40326|AAAAAAAAGINJAAAA|692|Laurel |Court|Suite 210|Mountain View|Winkler County|TX|74466|United States|-6|condo| +40327|AAAAAAAAHINJAAAA|593|Ridge Spruce|RD|Suite 100|Crystal|Mecosta County|MI|45258|United States|-5|single family| +40328|AAAAAAAAIINJAAAA|890|6th |Pkwy|Suite 110|Hamilton|New Haven County|CT|03408|United States|-5|single family| +40329|AAAAAAAAJINJAAAA|26|Cedar |Ln|Suite 370|Valley View|Logan County|NE|65124|United States|-7|apartment| +40330|AAAAAAAAKINJAAAA|446|Hillcrest |Boulevard|Suite X|Valley View|Cross County|AR|75124|United States|-6|condo| +40331|AAAAAAAALINJAAAA|493|River 2nd|Road|Suite N|Mount Vernon|Sully County|SD|58482|United States|-7|single family| +40332|AAAAAAAAMINJAAAA|939|10th |Street|Suite R|Enterprise|Glascock County|GA|31757|United States|-5|single family| +40333|AAAAAAAANINJAAAA|674|Chestnut |Lane|Suite 270|Harvey|San Miguel County|NM|85858|United States|-7|apartment| +40334|AAAAAAAAOINJAAAA|708|Sunset |Wy|Suite 160|Stringtown|Ross County|OH|40162|United States|-5|single family| +40335|AAAAAAAAPINJAAAA|806|East Spruce|Ct.|Suite 40|Enterprise|Wright County|MO|61757|United States|-6|apartment| +40336|AAAAAAAAAJNJAAAA|728|Meadow 11th|RD|Suite 430|Florence|Vernon County|WI|53394|United States|-6|condo| +40337|AAAAAAAABJNJAAAA|543|Meadow |Ln|Suite V|Cedar Grove|Clark County|KY|40411|United States|-6|single family| +40338|AAAAAAAACJNJAAAA|531|Lee Highland|ST|Suite T|Mount Pleasant|Jim Wells County|TX|71933|United States|-6|apartment| +40339|AAAAAAAADJNJAAAA|459|South View|Ave|Suite 320|Friendship|McPherson County|KS|64536|United States|-6|single family| +40340|AAAAAAAAEJNJAAAA|216|West |Way|Suite 350|Liberty|Clarke County|AL|33451|United States|-6|condo| +40341|AAAAAAAAFJNJAAAA|24|Oak |Dr.|Suite X|Jackson|Rabun County|GA|39583|United States|-5|apartment| +40342|AAAAAAAAGJNJAAAA|120|4th Washington|Lane|Suite 40|Lincoln|Bernalillo County|NM|81289|United States|-7|single family| +40343|AAAAAAAAHJNJAAAA|135|Spruce Highland|RD|Suite W|Hardy|Lincoln County|MT|65354|United States|-7|single family| +40344|AAAAAAAAIJNJAAAA|967|Elm |Ct.|Suite 260|Woodrow|Highland County|VA|24273|United States|-5|condo| +40345|AAAAAAAAJJNJAAAA|602|5th |Wy|Suite G|Spring Valley|Union County|AR|76060|United States|-6|apartment| +40346|AAAAAAAAKJNJAAAA|260|Johnson |Ln|Suite 270|Summerfield|Taylor County|FL|30634|United States|-5|single family| +40347|AAAAAAAALJNJAAAA|948|Poplar Wilson|Dr.|Suite 10|Pleasant Valley|Cheyenne County|KS|62477|United States|-6|condo| +40348|AAAAAAAAMJNJAAAA||Hill View|ST|Suite 220|Spring Valley|Douglas County|WI|56060||-6|condo| +40349|AAAAAAAANJNJAAAA|642|Williams Church|Pkwy|Suite 20|Arlington|Greenbrier County|WV|26557|United States|-5|apartment| +40350|AAAAAAAAOJNJAAAA|750|College Sunset|Drive|Suite X|Riverview|Davis County|UT|89003|United States|-7|condo| +40351|AAAAAAAAPJNJAAAA|937|2nd |Way|Suite U|Pleasant Valley|Cherokee County|TX|72477|United States|-6|apartment| +40352|AAAAAAAAAKNJAAAA|375|1st |Boulevard|Suite 320|Union Hill|Adair County|KY|47746|United States|-6|apartment| +40353|AAAAAAAABKNJAAAA|840|Hill Oak|Circle|Suite 440|Hopewell|Barbour County|WV|20587|United States|-5|condo| +40354|AAAAAAAACKNJAAAA|171|Johnson |Street|Suite 340|Maple Grove|Washington County|NY|18252|United States|-5|condo| +40355|AAAAAAAADKNJAAAA|64|2nd |Pkwy|Suite T|Bunker Hill|Hancock County|IL|60150|United States|-6|apartment| +40356|AAAAAAAAEKNJAAAA|125|Wilson Poplar|Court|Suite 80|Oak Ridge|Bon Homme County|SD|58371|United States|-6|single family| +40357|AAAAAAAAFKNJAAAA|500|Johnson |Cir.|Suite G|Enterprise|Lake County|OH|41757|United States|-5|single family| +40358|AAAAAAAAGKNJAAAA|380|3rd Fourth|Avenue|Suite 460|Lakewood|Dorchester County|MD|28877|United States|-5|apartment| +40359|AAAAAAAAHKNJAAAA|305|Mill Smith|Boulevard|Suite 390|Shady Grove|Reeves County|TX|72812|United States|-6|apartment| +40360|AAAAAAAAIKNJAAAA|639|Oak |Lane|Suite 160|Glenwood|Dorchester County|MD|23511|United States|-5|single family| +40361|AAAAAAAAJKNJAAAA|542|Church |Parkway|Suite 100|Florence|Daggett County|UT|83394|United States|-7|single family| +40362|AAAAAAAAKKNJAAAA|430|Park Main|Court|Suite N|Newtown|Simpson County|KY|41749|United States|-5|apartment| +40363|AAAAAAAALKNJAAAA|136|Sunset |Lane|Suite F|Salem|Jewell County|KS|68048|United States|-6|apartment| +40364|AAAAAAAAMKNJAAAA|801|Meadow Birch|Avenue|Suite 130|Glenwood|Madison County|IL|63511|United States|-6|single family| +40365|AAAAAAAANKNJAAAA|797|Cherry |Blvd|Suite 490|Hillcrest|Scott County|MN|53003|United States|-6|single family| +40366|AAAAAAAAOKNJAAAA|307|4th Church|Drive|Suite N|Hamilton|Issaquena County|MS|52808|United States|-6|single family| +40367|AAAAAAAAPKNJAAAA|885|Park Hill|Ave|Suite 140|Owens|Berrien County|MI|42324|United States|-5|condo| +40368|AAAAAAAAALNJAAAA|210|Hill Dogwood|Ln|Suite 170|Oak Hill|Dane County|WI|57838|United States|-6|single family| +40369|AAAAAAAABLNJAAAA|794|Pine |Lane|Suite 130|Mount Zion|Minnehaha County|SD|58054|United States|-7|apartment| +40370|AAAAAAAACLNJAAAA|507|3rd |Way|Suite 490|Florence|Flagler County|FL|33394|United States|-5|apartment| +40371|AAAAAAAADLNJAAAA|742|Sixth 4th|Avenue|Suite F|Ferguson|Wright County|MO|61821|United States|-6|condo| +40372|AAAAAAAAELNJAAAA|262|9th |Lane|Suite B|Arlington|San Mateo County|CA|96557|United States|-8|apartment| +40373|AAAAAAAAFLNJAAAA|736|4th Wilson|Drive|Suite H|Mountain View|Freestone County|TX|74466|United States|-6|condo| +40374|AAAAAAAAGLNJAAAA|119|Park East|Road|Suite 410|Macedonia|Hampshire County|MA|01687|United States|-5|condo| +40375|AAAAAAAAHLNJAAAA|707|Ash |Dr.|Suite 440|Plainview|Wayne County|TN|33683|United States|-6|condo| +40376|AAAAAAAAILNJAAAA|580|Valley Center|Court|Suite 170|Oak Hill|Clark County|ID|87838|United States|-7|apartment| +40377|AAAAAAAAJLNJAAAA|96|River |Circle|Suite D|Red Hill|San Jacinto County|TX|74338|United States|-6|condo| +40378|AAAAAAAAKLNJAAAA|967|Woodland South|Wy|Suite 290|Arlington|Dawson County|TX|76557|United States|-6|single family| +40379|AAAAAAAALLNJAAAA|226|3rd Walnut|Street|Suite V|Maple Grove|Sioux County|ND|58252|United States|-6|condo| +40380|AAAAAAAAMLNJAAAA|120|Hill |Wy|Suite 420|Lakeview|Carlton County|MN|58579|United States|-6|single family| +40381|AAAAAAAANLNJAAAA|156|Park |Street|Suite 190|Georgetown|Monroe County|WV|27057|United States|-5|single family| +40382|AAAAAAAAOLNJAAAA|389|14th |Parkway|Suite T|Union Hill|Story County|IA|57746|United States|-6|condo| +40383|AAAAAAAAPLNJAAAA|563|Miller |Avenue|Suite 40|Greenfield|Webster County|NE|65038|United States|-7|condo| +40384|AAAAAAAAAMNJAAAA|237|Williams |Blvd|Suite F|Woodlawn|McDonough County|IL|64098|United States|-6|single family| +40385|AAAAAAAABMNJAAAA|335|15th |Street|Suite 260|Phoenix|Knott County|KY|42276|United States|-5|condo| +40386|AAAAAAAACMNJAAAA|616|Spring |ST|Suite 490|Shiloh|Grafton County|NH|09875|United States|-5|apartment| +40387|AAAAAAAADMNJAAAA|936|10th |Ln|Suite 350|Mount Pleasant|Cheyenne County|NE|61933|United States|-6|single family| +40388|AAAAAAAAEMNJAAAA|691|Cedar 4th|Avenue|Suite K|Liberty|Hall County|GA|33451|United States|-5|single family| +40389|AAAAAAAAFMNJAAAA|712|Smith Pine|Ln|Suite A|Mount Zion|Oconee County|GA|38054|United States|-5|single family| +40390|AAAAAAAAGMNJAAAA|686|River |Ln|Suite 80|Salem|Buffalo County|SD|58048|United States|-6|apartment| +40391|AAAAAAAAHMNJAAAA|910|1st |Cir.|Suite 300|Ashland|Martinsville city|VA|24244|United States|-5|condo| +40392|AAAAAAAAIMNJAAAA|147|Cedar 4th|Dr.|Suite J|Page|Grand County|UT|80296|United States|-7|single family| +40393|AAAAAAAAJMNJAAAA|30|Third |Pkwy|Suite 420|Florence|Putnam County|TN|33394|United States|-6|single family| +40394|AAAAAAAAKMNJAAAA|43|11th Jackson|Parkway|Suite Q|White Oak|Bedford County|VA|26668|United States|-5|apartment| +40395|AAAAAAAALMNJAAAA|135|Chestnut |Dr.|Suite I|Edgewood|Roscommon County|MI|40069|United States|-5|condo| +40396|AAAAAAAAMMNJAAAA|409|13th Highland|Court|Suite G|Belmont|Rockbridge County|VA|20191|United States|-5|apartment| +40397|AAAAAAAANMNJAAAA|720|West Broadway|Street|Suite X|Lebanon|Dallas County|MO|62898|United States|-6|single family| +40398|AAAAAAAAOMNJAAAA|410|Oak Lee|Ln|Suite 380|Lakeview|Davie County|NC|28579|United States|-5|single family| +40399|AAAAAAAAPMNJAAAA|792|Central |Wy|Suite 340|Lakeside|Forest County|PA|19532|United States|-5|condo| +40400|AAAAAAAAANNJAAAA|437|15th Birch|Dr.|Suite W|Hillcrest|Boulder County|CO|83003|United States|-7|single family| +40401|AAAAAAAABNNJAAAA|408|Pine |Parkway|Suite 210|Stringtown|Jerauld County|SD|50162|United States|-7|condo| +40402|AAAAAAAACNNJAAAA|169|Broadway 10th|Lane|Suite 140|Summit|Judith Basin County|MT|60499|United States|-7|apartment| +40403|AAAAAAAADNNJAAAA|924|8th 4th|Ln|Suite J|Sumner|Edmonson County|KY|40519|United States|-6|apartment| +40404|AAAAAAAAENNJAAAA|497|Church |Wy|Suite 240|Springdale|Sequatchie County|TN|38883|United States|-6|single family| +40405|AAAAAAAAFNNJAAAA|351|9th Maple|Street|Suite 420|Red Bank|Slope County|ND|54975|United States|-6|condo| +40406|AAAAAAAAGNNJAAAA|580|Lake |Pkwy|Suite 130|Bunker Hill|Franklin County|OH|40150|United States|-5|single family| +40407|AAAAAAAAHNNJAAAA|||Court|Suite 10|Union|||58721||-6|apartment| +40408|AAAAAAAAINNJAAAA|535|Locust West|ST|Suite 290|Spring Grove|Mercer County|PA|16719|United States|-5|condo| +40409|AAAAAAAAJNNJAAAA|729|5th North|Parkway|Suite 230|Dewey|Rains County|TX|71160|United States|-6|single family| +40410|AAAAAAAAKNNJAAAA|709|11th 3rd|Circle|Suite 180|Woodland|Cannon County|TN|34854|United States|-5|apartment| +40411|AAAAAAAALNNJAAAA|34|4th |Parkway|Suite 130|Deerfield|Cambria County|PA|19840|United States|-5|apartment| +40412|AAAAAAAAMNNJAAAA|||Wy||Wilson|Suffolk city|VA||United States||| +40413|AAAAAAAANNNJAAAA|26|Sixth Ninth|Blvd|Suite 0|Sulphur Springs|Kent County|TX|78354|United States|-6|condo| +40414|AAAAAAAAONNJAAAA|450|Johnson |Avenue|Suite K|Walnut Grove|Lincoln County|MO|67752|United States|-6|apartment| +40415|AAAAAAAAPNNJAAAA|942|Forest |Pkwy|Suite X|Mountain View|Adams County|IN|44466|United States|-5|single family| +40416|AAAAAAAAAONJAAAA|303|Highland Cherry|Way|Suite T|Highland|Rockbridge County|VA|29454|United States|-5|apartment| +40417|AAAAAAAABONJAAAA|833|Johnson 4th|Circle|Suite A|Valley View|Garfield County|OK|75124|United States|-6|apartment| +40418|AAAAAAAACONJAAAA|674|Locust |Wy|Suite T|Crossroads|Otter Tail County|MN|50534|United States|-6|single family| +40419|AAAAAAAADONJAAAA|811|Tenth 5th|Ct.|Suite 100|Wildwood|Johnson County|KY|46871|United States|-6|single family| +40420|AAAAAAAAEONJAAAA|861|Seventh Lee|Drive|Suite V|Riverview|Lake County|TN|39003|United States|-6|condo| +40421|AAAAAAAAFONJAAAA|||||Brownsville||MT||United States||| +40422|AAAAAAAAGONJAAAA|201|Sunset Walnut|Avenue|Suite 90|Sulphur Springs|Culpeper County|VA|28354|United States|-5|single family| +40423|AAAAAAAAHONJAAAA|||Street|Suite 460|Highland|Gallatin County||69454|United States||condo| +40424|AAAAAAAAIONJAAAA|654|Mill View|Dr.|Suite 260|Oak Grove|Henderson County|IL|68370|United States|-6|single family| +40425|AAAAAAAAJONJAAAA|379|North Locust|RD|Suite 450|Wildwood|Rhea County|TN|36871|United States|-6|condo| +40426|AAAAAAAAKONJAAAA|424|Dogwood |Ct.|Suite 0|Oak Grove|Benzie County|MI|48370|United States|-5|apartment| +40427|AAAAAAAALONJAAAA|600|2nd |Ln|Suite 450|Sulphur Springs|Blaine County|MT|68354|United States|-7|apartment| +40428|AAAAAAAAMONJAAAA|2|Lake |Pkwy|Suite 440|Rolling Hills|Passaic County|NJ|07872|United States|-5|apartment| +40429|AAAAAAAANONJAAAA|487|Oak |Road|Suite 250|Five Forks|Columbia County|AR|72293|United States|-6|single family| +40430|AAAAAAAAOONJAAAA|761|Cherry |Cir.||Jackson||IL||United States|-6|single family| +40431|AAAAAAAAPONJAAAA|448|Walnut |Road|Suite B|Salem|Prince George County|VA|28048|United States|-5|condo| +40432|AAAAAAAAAPNJAAAA|48|2nd 5th|Road|Suite 130|Greenfield|Portage County|WI|55038|United States|-6|condo| +40433|AAAAAAAABPNJAAAA|361|Main Locust|RD|Suite 170|Clinton|Jackson County|TN|38222|United States|-5|single family| +40434|AAAAAAAACPNJAAAA|279|Lakeview Cherry|Street|Suite X|Mountain View|Nantucket County|MA|05066|United States|-5|apartment| +40435|AAAAAAAADPNJAAAA|228|Main |Street|Suite Q|Sunnyside|Scott County|IN|41952|United States|-5|single family| +40436|AAAAAAAAEPNJAAAA|936|Sixth Williams|Boulevard|Suite S|Lebanon|Lincoln County|MN|52898|United States|-6|condo| +40437|AAAAAAAAFPNJAAAA|254|2nd |Pkwy|Suite 440|Greenville|Jefferson County|NY|11387|United States|-5|condo| +40438|AAAAAAAAGPNJAAAA|480|River |Drive|Suite 440|Forest Hills|Macon County|NC|29237|United States|-5|condo| +40439|AAAAAAAAHPNJAAAA|288|Lakeview |Road|Suite S|Hillcrest|Crawford County|PA|13003|United States|-5|single family| +40440|AAAAAAAAIPNJAAAA||||Suite 430||Richland County|IL|68605||-6|| +40441|AAAAAAAAJPNJAAAA|456|Willow |Ct.|Suite L|Wildwood|Cherokee County|OK|76871|United States|-6|apartment| +40442|AAAAAAAAKPNJAAAA|609|6th Elm|Pkwy|Suite 450|Liberty|Custer County|MT|63451|United States|-7|condo| +40443|AAAAAAAALPNJAAAA|320|Williams 5th|Street|Suite L|Midway|Hampton County|SC|21904|United States|-5|apartment| +40444|AAAAAAAAMPNJAAAA|823|Davis 3rd|Wy|Suite 490|Highland Park|Clay County|AR|76534|United States|-6|single family| +40445|AAAAAAAANPNJAAAA|684|2nd Railroad|Ct.|Suite D|Riverdale|Marion County|IL|69391|United States|-6|single family| +40446|AAAAAAAAOPNJAAAA|916|Main 13th|Court|Suite W|Farmington|Murray County|GA|39145|United States|-5|single family| +40447|AAAAAAAAPPNJAAAA|312|Spruce Madison|Dr.|Suite J|Arlington|Caribou County|ID|86557|United States|-7|condo| +40448|AAAAAAAAAAOJAAAA|597|Ridge First|Boulevard|Suite 130|Arlington|Carroll County|KY|46557|United States|-6|condo| +40449|AAAAAAAABAOJAAAA|12|6th Willow|Pkwy|Suite 20|Nichols|Musselshell County|MT|67940|United States|-7|single family| +40450|AAAAAAAACAOJAAAA|965|Railroad 3rd|Pkwy|Suite 20|Unionville|Wyandotte County|KS|61711|United States|-6|single family| +40451|AAAAAAAADAOJAAAA|398|Walnut |Parkway|Suite 190|Salem|Monroe County|IL|68048|United States|-6|apartment| +40452|AAAAAAAAEAOJAAAA|673||||Franklin|||69101|United States|-7|apartment| +40453|AAAAAAAAFAOJAAAA|6|Hickory Locust|Court|Suite 120|Deerfield|Hays County|TX|79840|United States|-6|condo| +40454|AAAAAAAAGAOJAAAA|711|Johnson |Pkwy|Suite 140|Weldon|Branch County|MI|46277|United States|-5|single family| +40455|AAAAAAAAHAOJAAAA|768|Pine Williams|Drive|Suite M|Jones|Sussex County|DE|12686|United States|-5|condo| +40456|AAAAAAAAIAOJAAAA|590|Second |Road|Suite O|Unionville|Centre County|PA|11711|United States|-5|condo| +40457|AAAAAAAAJAOJAAAA|988|Locust Laurel|Boulevard|Suite 350|Sunnyside|Maverick County|TX|71952|United States|-6|condo| +40458|AAAAAAAAKAOJAAAA|659|Hill |Blvd|Suite 80|Cedar Grove|Houston County|MN|50411|United States|-6|apartment| +40459|AAAAAAAALAOJAAAA|368|Davis North|Lane|Suite C|Union Hill|Furnas County|NE|67746|United States|-6|condo| +40460|AAAAAAAAMAOJAAAA|639|5th Lakeview|Avenue|Suite 260|Marion|Muscogee County|GA|30399|United States|-5|apartment| +40461|AAAAAAAANAOJAAAA|624|Pine |ST|Suite J|Hamilton|Hughes County|SD|52808|United States|-7|single family| +40462|AAAAAAAAOAOJAAAA|735|Pine Jackson|ST|Suite 390|Sunnyside|Grand Isle County|VT|02552|United States|-5|single family| +40463|AAAAAAAAPAOJAAAA|908|2nd |Dr.|Suite 30|Enterprise|Bowman County|ND|51757|United States|-6|single family| +40464|AAAAAAAAABOJAAAA|392|Sunset Hill|Parkway|Suite 460|Wildwood|Grant County|MN|56871|United States|-6|apartment| +40465|AAAAAAAABBOJAAAA|796|Washington |Way|Suite 160|Greenville|Richland County|SC|21387|United States|-5|single family| +40466|AAAAAAAACBOJAAAA|421|Seventh |Circle|Suite P|Harmony|Cambria County|PA|15804|United States|-5|apartment| +40467|AAAAAAAADBOJAAAA|810|Church |Court|Suite 190|Oneida|Stone County|AR|74027|United States|-6|apartment| +40468|AAAAAAAAEBOJAAAA|484|Maple Green|Court|Suite 220|Gladstone|Brown County|NE|60894|United States|-6|condo| +40469|AAAAAAAAFBOJAAAA|175|Mill |Road|Suite X|Sulphur Springs|Washington County|MS|58354|United States|-6|apartment| +40470|AAAAAAAAGBOJAAAA|434|Ninth |||||||||single family| +40471|AAAAAAAAHBOJAAAA|769|Cedar Chestnut|Ave|Suite 60|Midway|Bremer County|IA|51904|United States|-6|apartment| +40472|AAAAAAAAIBOJAAAA|348|Walnut |Court|Suite I|Marion|Stark County|ND|50399|United States|-6|single family| +40473|AAAAAAAAJBOJAAAA|334|Park |Court|Suite 20|Woodville|Woods County|OK|74289|United States|-6|apartment| +40474|AAAAAAAAKBOJAAAA|918|5th |Blvd|Suite 30|Riverside|Marshall County|AL|39231|United States|-6|condo| +40475|AAAAAAAALBOJAAAA|583|Hillcrest |Road|Suite P|Oakland|Clay County|TX|79843|United States|-6|apartment| +40476|AAAAAAAAMBOJAAAA|619|Fifth |Lane|Suite 400|Green Acres|Marlboro County|SC|27683|United States|-5|condo| +40477|AAAAAAAANBOJAAAA|79|4th |ST|Suite O|Greenwood|Pulaski County|KY|48828|United States|-5|condo| +40478|AAAAAAAAOBOJAAAA|690|West |Road|Suite S|Mount Zion|Woodford County|IL|68054|United States|-6|condo| +40479|AAAAAAAAPBOJAAAA|188|Hill View|Ln|Suite M|Sulphur Springs|Crawford County|OH|48354|United States|-5|apartment| +40480|AAAAAAAAACOJAAAA|227|Ash |Ln|Suite P|Greenwood|Kodiak Island Borough|AK|98828|United States|-9|condo| +40481|AAAAAAAABCOJAAAA|96|Walnut 4th|Avenue|Suite D|Springfield|Johnson County|KS|69303|United States|-6|single family| +40482|AAAAAAAACCOJAAAA|251|8th Cherry|Circle|Suite 170|Oak Hill|Orangeburg County|SC|27838|United States|-5|condo| +40483|AAAAAAAADCOJAAAA|83|Cedar |Blvd|Suite 340|Wilson|Ware County|GA|36971|United States|-5|apartment| +40484|AAAAAAAAECOJAAAA|535|West Spruce|Blvd|Suite C|Mount Vernon|Galveston County|TX|78482|United States|-6|single family| +40485|AAAAAAAAFCOJAAAA|898|7th Spruce|Ave|Suite 120|Fairfield|Jones County|IA|56192|United States|-6|single family| +40486|AAAAAAAAGCOJAAAA|270|Ninth |Cir.|Suite U|Red Oak|Laramie County|WY|85018|United States|-7|apartment| +40487|AAAAAAAAHCOJAAAA|584|Highland |Circle|Suite 470|Newtown|Kimball County|NE|61749|United States|-7|condo| +40488|AAAAAAAAICOJAAAA|103|Woodland |Ln|Suite 260|Oakland|Cortland County|NY|19843|United States|-5|apartment| +40489|AAAAAAAAJCOJAAAA|624|Hickory |Road|Suite 270|Clifton|Knox County|TX|78014|United States|-6|single family| +40490|AAAAAAAAKCOJAAAA|442|Railroad Madison|Drive|Suite 210|Buena Vista|Merced County|CA|95752|United States|-8|condo| +40491|AAAAAAAALCOJAAAA|540|Cedar |Boulevard|Suite V|Hillcrest|Gooding County|ID|83003|United States|-7|apartment| +40492|AAAAAAAAMCOJAAAA|843|View |Pkwy|Suite 100|Edgewood|Andrews County|TX|70069|United States|-6|condo| +40493|AAAAAAAANCOJAAAA|5|7th |Street|Suite 360|Millbrook|Valdez-Cordova Census Area|AK|97529|United States|-9|single family| +40494|AAAAAAAAOCOJAAAA|299|First 6th|Blvd|Suite 460|Spring Hill|Pratt County|KS|66787|United States|-6|condo| +40495|AAAAAAAAPCOJAAAA|250|Second |Circle|Suite M|White Oak|Sitka Borough|AK|96668|United States|-9|condo| +40496|AAAAAAAAADOJAAAA|17|2nd Lake|Circle|Suite Y|Shiloh|Iron County|MO|69275|United States|-6|condo| +40497|AAAAAAAABDOJAAAA|33|Birch Meadow|Lane|Suite E|Glenwood|Kendall County|IL|63511|United States|-6|single family| +40498|AAAAAAAACDOJAAAA|161|6th Spring|Wy|Suite T|Riverdale|Bradley County|TN|39391|United States|-5|condo| +40499|AAAAAAAADDOJAAAA|684|4th |Ln|Suite 90|Spring Hill|Pendleton County|KY|46787|United States|-5|condo| +40500|AAAAAAAAEDOJAAAA|802|Valley East|Dr.|Suite 100|Sulphur Springs|Santa Cruz County|CA|98354|United States|-8|apartment| +40501|AAAAAAAAFDOJAAAA|689|Jefferson Fourteenth|RD|Suite 430|Lakeside|Henry County|VA|29532|United States|-5|condo| +40502|AAAAAAAAGDOJAAAA|510|First |Wy|Suite R|Lakewood|Telfair County|GA|38877|United States|-5|single family| +40503|AAAAAAAAHDOJAAAA|872|Pine Sycamore|Ave|Suite 110|Ashland|Rice County|MN|54244|United States|-6|condo| +40504|AAAAAAAAIDOJAAAA|632|West |Pkwy|Suite 30|Pleasant Hill|Robertson County|TX|73604|United States|-6|apartment| +40505|AAAAAAAAJDOJAAAA|631|1st Second|Ave|Suite B|Newport|Warren County|IN|41521|United States|-5|apartment| +40506|AAAAAAAAKDOJAAAA|798|4th Oak|Avenue|Suite 240|Sunnyside|Salem County|NJ|02552|United States|-5|condo| +40507|AAAAAAAALDOJAAAA||Oak Oak||Suite D||Rhea County|TN|||-6|apartment| +40508|AAAAAAAAMDOJAAAA|680|Church |Court|Suite R|Maxwell|Bear Lake County|ID|84925|United States|-7|apartment| +40509|AAAAAAAANDOJAAAA|595|Washington Madison|Wy|Suite 330|Marion|Bullock County|AL|30399|United States|-6|apartment| +40510|AAAAAAAAODOJAAAA|376|Third |Drive|Suite 450|Midway|Washita County|OK|71904|United States|-6|apartment| +40511|AAAAAAAAPDOJAAAA||Highland Cedar|Boulevard|Suite 50|||TX|||-6|| +40512|AAAAAAAAAEOJAAAA|||Court|Suite 380|Providence|McHenry County||56614|||single family| +40513|AAAAAAAABEOJAAAA|566|Elm |Way|Suite 470|Hopewell|Lee County|AL|30587|United States|-6|single family| +40514|AAAAAAAACEOJAAAA|709|Madison |Blvd|Suite 140|Maple Grove|Charles Mix County|SD|58252|United States|-6|condo| +40515|AAAAAAAADEOJAAAA|748|Valley West|Blvd|Suite 230|Oak Grove|Pasco County|FL|38370|United States|-5|condo| +40516|AAAAAAAAEEOJAAAA|558|Spring 4th|Wy|Suite L|Lawrenceville|Jerome County|ID|84462|United States|-7|condo| +40517|AAAAAAAAFEOJAAAA|365|Dogwood Jackson|Parkway|Suite 330|Jamestown|Haines Borough|AK|96867|United States|-9|single family| +40518|AAAAAAAAGEOJAAAA|316|2nd |Circle|Suite 170|Pine Grove|New York County|NY|14593|United States|-5|single family| +40519|AAAAAAAAHEOJAAAA|272|Sunset Franklin|Cir.|Suite J|Ashland|Dallas County|TX|74244|United States|-6|single family| +40520|AAAAAAAAIEOJAAAA|766|1st Birch|Dr.|Suite T|Woodville|Washington County|CO|84289|United States|-7|condo| +40521|AAAAAAAAJEOJAAAA|181|Williams |Circle|Suite 140|Mount Vernon|Lancaster County|SC|28482|United States|-5|apartment| +40522|AAAAAAAAKEOJAAAA|838|Elm |Lane|Suite 480|Kingston|Brooke County|WV|24975|United States|-5|apartment| +40523|AAAAAAAALEOJAAAA|489|Pine |Ln|Suite 480|Highland Park|Neosho County|KS|66534|United States|-6|single family| +40524|AAAAAAAAMEOJAAAA|536|Main |Road|Suite T|Jackson|Juneau Borough|AK|99583|United States|-9|apartment| +40525|AAAAAAAANEOJAAAA|731|Franklin Railroad|Ave|Suite 280|Bunker Hill|Leon County|FL|30150|United States|-5|apartment| +40526|AAAAAAAAOEOJAAAA|189|Spring 2nd|Wy|Suite 460|Newtown|Belknap County|NH|02349|United States|-5|apartment| +40527|AAAAAAAAPEOJAAAA|17|Pine |RD|Suite V|Highland|Vilas County|WI|59454|United States|-6|condo| +40528|AAAAAAAAAFOJAAAA|255|Main Seventh|Road|Suite 80|Mount Vernon|Taylor County|WI|58482|United States|-6|condo| +40529|AAAAAAAABFOJAAAA|15|Meadow |Court|Suite J|Stringtown|Clark County|WA|90162|United States|-8|condo| +40530|AAAAAAAACFOJAAAA|41|11th Woodland|Blvd|Suite E|Barnes|Jefferson Davis County|MS|53788|United States|-6|condo| +40531|AAAAAAAADFOJAAAA|378|Spruce Elm|Court|Suite 370|Glenwood|Wood County|OH|43511|United States|-5|single family| +40532|AAAAAAAAEFOJAAAA|527|Meadow Seventh|Circle|Suite 50|Mount Pleasant|Granite County|MT|61933|United States|-7|single family| +40533|AAAAAAAAFFOJAAAA|814|View Oak|Boulevard|Suite 20|Hamilton|Morgan County|KY|42808|United States|-5|condo| +40534|AAAAAAAAGFOJAAAA|334|Poplar East|Road|Suite 480|Lebanon|Wagoner County|OK|72898|United States|-6|condo| +40535|AAAAAAAAHFOJAAAA|||Pkwy||Shady Grove|Elmore County||||-6|| +40536|AAAAAAAAIFOJAAAA|914|Willow Lakeview|Blvd|Suite 210|Sunnyside|Iberia Parish|LA|71952|United States|-6|single family| +40537|AAAAAAAAJFOJAAAA|518|Fourth |Parkway|Suite I|Lebanon|Cache County|UT|82898|United States|-7|condo| +40538|AAAAAAAAKFOJAAAA|789|Second |Lane|Suite B|Glenwood|Hemphill County|TX|73511|United States|-6|single family| +40539|AAAAAAAALFOJAAAA|290|Wilson River|Pkwy|Suite A|Greenfield|Houston County|TN|35038|United States|-5|condo| +40540|AAAAAAAAMFOJAAAA|398|Hill Center|Boulevard|Suite 100|Oakland|Smyth County|VA|29843|United States|-5|single family| +40541|AAAAAAAANFOJAAAA|891|Lake |Lane|Suite 400|Springhill|New Haven County|CT|05202|United States|-5|apartment| +40542|AAAAAAAAOFOJAAAA|342|Walnut Sunset|Way|Suite M|Summit|Williamsburg County|SC|20499|United States|-5|single family| +40543|AAAAAAAAPFOJAAAA|351|Fifth Pine|Wy|Suite W|Walnut Grove|Barrow County|GA|37752|United States|-5|apartment| +40544|AAAAAAAAAGOJAAAA|932|Oak Poplar|Circle|Suite 340|Tracy|Gosper County|NE|66340|United States|-6|condo| +40545|AAAAAAAABGOJAAAA|950|Sunset Lee|Dr.|Suite 210|Pleasant Hill|Reeves County|TX|73604|United States|-6|condo| +40546|AAAAAAAACGOJAAAA|158|Fifth Lake|Street|Suite 290|Friendship|Fayette County|IL|64536|United States|-6|single family| +40547|AAAAAAAADGOJAAAA|727|Dogwood Highland|Boulevard|Suite W|Northwood|Summit County|CO|84104|United States|-7|apartment| +40548|AAAAAAAAEGOJAAAA|207|Hickory Maple|Boulevard|Suite C|Glendale|Riley County|KS|63951|United States|-6|condo| +40549|AAAAAAAAFGOJAAAA|||||Liberty||AR|73451|United States||| +40550|AAAAAAAAGGOJAAAA|859|Hill |Court|Suite 420|Shiloh|Gratiot County|MI|49275|United States|-5|condo| +40551|AAAAAAAAHGOJAAAA|660|Maple Poplar|RD|Suite H|Mount Olive|Marinette County|WI|58059|United States|-6|apartment| +40552|AAAAAAAAIGOJAAAA|380|Lee |Ave|Suite 250|Walnut Grove|Breckinridge County|KY|47752|United States|-6|single family| +40553|AAAAAAAAJGOJAAAA|973|Sixth |Parkway|Suite 210|New Hope|Comanche County|OK|79431|United States|-6|condo| +40554|AAAAAAAAKGOJAAAA|580|||Suite X|Forest Hills|Henrico County|VA||||condo| +40555|AAAAAAAALGOJAAAA|146|Park |Way|Suite 250|Spring Hill|Montour County|PA|16787|United States|-5|single family| +40556|AAAAAAAAMGOJAAAA|338||Court|Suite V|Argyle|||38722|United States|-6|condo| +40557|AAAAAAAANGOJAAAA|912|Main |Court|Suite 320|Antioch|Erie County|PA|18605|United States|-5|single family| +40558|AAAAAAAAOGOJAAAA|898|Cherry |Ln|Suite C|Oakwood|Decatur County|KS|60169|United States|-6|apartment| +40559|AAAAAAAAPGOJAAAA|502|Church |Pkwy|Suite 120|Maple Grove|Jefferson County|TX|78252|United States|-6|single family| +40560|AAAAAAAAAHOJAAAA|228|View Johnson|Wy|Suite 140|Glenville|Donley County|TX|73445|United States|-6|apartment| +40561|AAAAAAAABHOJAAAA|402|Walnut |Street|Suite 110|Hillcrest|Wolfe County|KY|43003|United States|-5|apartment| +40562|AAAAAAAACHOJAAAA|230|North 4th|Circle|Suite 460|Highland Park|Windsor County|VT|07134|United States|-5|single family| +40563|AAAAAAAADHOJAAAA|52|Jefferson |Parkway|Suite 390|Brownsville|Dallas County|TX|79310|United States|-6|apartment| +40564|AAAAAAAAEHOJAAAA|274|Third |Ln|Suite Y|Unionville|Kent County|RI|02311|United States|-5|apartment| +40565|AAAAAAAAFHOJAAAA|485|10th |Pkwy|Suite V|Red Hill|Skamania County|WA|94338|United States|-8|single family| +40566|AAAAAAAAGHOJAAAA|369|Fifth |Road|Suite 170|Riverview|Platte County|MO|69003|United States|-6|condo| +40567|AAAAAAAAHHOJAAAA|596|Fourth 1st|Lane|Suite 450|Colonial Heights|Taos County|NM|83425|United States|-7|single family| +40568|AAAAAAAAIHOJAAAA|976|Second 7th|Street|Suite 210|Jamestown|Becker County|MN|56867|United States|-6|single family| +40569|AAAAAAAAJHOJAAAA|443|Church Green|Blvd|Suite R|Centerville|Putnam County|GA|30059|United States|-5|apartment| +40570|AAAAAAAAKHOJAAAA|869|Locust Ridge|Ct.|Suite 280|Pine Grove|Dickey County|ND|54593|United States|-6|condo| +40571|AAAAAAAALHOJAAAA|866|Willow Second|Avenue|Suite E|Red Hill|Buchanan County|MO|64338|United States|-6|apartment| +40572|AAAAAAAAMHOJAAAA|12||||Waterloo||IN|41675|United States||apartment| +40573|AAAAAAAANHOJAAAA|142|Smith View|Avenue|Suite 320|Salem|Kalamazoo County|MI|48048|United States|-5|single family| +40574|AAAAAAAAOHOJAAAA|523|Madison |Drive|Suite V|Glendale|Bronx County|NY|13951|United States|-5|single family| +40575|AAAAAAAAPHOJAAAA|699|11th |Street|Suite C|Midway|Garfield County|MT|61904|United States|-7|single family| +40576|AAAAAAAAAIOJAAAA|685|Washington Hickory|Avenue|Suite 120|Oak Ridge|Emporia city|VA|28371|United States|-5|single family| +40577|AAAAAAAABIOJAAAA|869|Ash |Cir.|Suite 60|Unionville|Hartley County|TX|71711|United States|-6|apartment| +40578|AAAAAAAACIOJAAAA|668|Maple |Road|Suite G|Floyd|Bexar County|TX|73235|United States|-6|apartment| +40579|AAAAAAAADIOJAAAA|236|9th |RD|Suite T|Fairfield|Pulaski County|IN|46192|United States|-5|single family| +40580|AAAAAAAAEIOJAAAA|311|Cherry |Wy|Suite 350|Hillcrest|Hardeman County|TN|33003|United States|-5|apartment| +40581|AAAAAAAAFIOJAAAA|554|Main |Ave|Suite 170|Oak Grove|Gasconade County|MO|68370|United States|-6|single family| +40582|AAAAAAAAGIOJAAAA|227|Jefferson |Parkway|Suite 290|Cedar Grove|Clay County|MN|50411|United States|-6|apartment| +40583|AAAAAAAAHIOJAAAA|201|River Elm|Ct.|Suite 200|Highland|Richmond County|GA|39454|United States|-5|condo| +40584|AAAAAAAAIIOJAAAA|867|Fifteenth Lakeview|Parkway|Suite 430|Woodland|Frederick County|MD|24854|United States|-5|condo| +40585|AAAAAAAAJIOJAAAA|106|Hill 14th|Ave|Suite H|Georgetown|Madison County|NC|27057|United States|-5|apartment| +40586|AAAAAAAAKIOJAAAA|676|2nd South|Dr.|Suite L|Concord|Adams County|WA|94107|United States|-8|single family| +40587|AAAAAAAALIOJAAAA|847|Main |Avenue|Suite 420|Shady Grove|Bullock County|AL|32812|United States|-6|single family| +40588|AAAAAAAAMIOJAAAA|234|Park 12th|Dr.|Suite G|Lincoln|Bryan County|OK|71289|United States|-6|condo| +40589|AAAAAAAANIOJAAAA|322|Oak |Court|Suite 410|Centerville|Freeborn County|MN|50059|United States|-6|single family| +40590|AAAAAAAAOIOJAAAA|469|Walnut Walnut|Road|Suite X|Georgetown|Fulton County|OH|47057|United States|-5|single family| +40591|AAAAAAAAPIOJAAAA|585|Sunset |Blvd|Suite R|Bethel|Washington County|ID|85281|United States|-7|apartment| +40592|AAAAAAAAAJOJAAAA|684|Lake Pine|Drive|Suite 150|Plainview|Nevada County|AR|73683|United States|-6|apartment| +40593|AAAAAAAABJOJAAAA|576|Locust Sycamore|Ct.|Suite 270|Greenfield|Bee County|TX|75038|United States|-6|apartment| +40594|AAAAAAAACJOJAAAA|58|Sunset |Road|Suite 30|Woodlawn|Grainger County|TN|34098|United States|-5|apartment| +40595|AAAAAAAADJOJAAAA|91|Chestnut Park|Cir.|Suite O|Forest Hills|Vance County|NC|29237|United States|-5|condo| +40596|AAAAAAAAEJOJAAAA|487|Ridge Smith|RD|Suite 100|Glenwood|Las Animas County|CO|83511|United States|-7|condo| +40597|AAAAAAAAFJOJAAAA|630|Hill |Circle|Suite 270|Gravel Hill|Mercer County|WV|21944|United States|-5|condo| +40598|AAAAAAAAGJOJAAAA|669|Miller Hill|Drive||Walnut Grove|Putnam County|NY|17752||-5|condo| +40599|AAAAAAAAHJOJAAAA|940|Madison |RD|Suite W|Edgewood|Winston County|MS|50069|United States|-6|apartment| +40600|AAAAAAAAIJOJAAAA|559|Lake Woodland|Drive|Suite 370|Antioch|San Juan County|UT|88605|United States|-7|condo| +40601|AAAAAAAAJJOJAAAA|966|South 12th|Ct.|Suite 350|Macedonia|Mathews County|VA|21087|United States|-5|single family| +40602|AAAAAAAAKJOJAAAA|629|Ninth Davis|Ave|Suite 310|Woodville|Page County|VA|24289|United States|-5|single family| +40603|AAAAAAAALJOJAAAA|956|6th Cedar|Road|Suite O|Summit|McNairy County|TN|30499|United States|-6|single family| +40604|AAAAAAAAMJOJAAAA|822|Jefferson |Blvd|Suite 120|Plainview|Woodford County|IL|63683|United States|-6|single family| +40605|AAAAAAAANJOJAAAA|998|5th Locust|Wy|Suite X|Ashland|Smith County|MS|54244|United States|-6|condo| +40606|AAAAAAAAOJOJAAAA|941|First |ST|Suite 390|Mount Vernon|Lamar County|GA|38482|United States|-5|apartment| +40607|AAAAAAAAPJOJAAAA|645|View |Ave|Suite J|Waterloo|Washington County|MD|21675|United States|-5|single family| +40608|AAAAAAAAAKOJAAAA|240|Lake Poplar|Ct.|Suite O|Clearwater|Peach County|GA|39534|United States|-5|condo| +40609|AAAAAAAABKOJAAAA|698|Spring |Court|Suite U|Union|Rio Grande County|CO|88721|United States|-7|apartment| +40610|AAAAAAAACKOJAAAA|376|Willow West|Wy|Suite 160|Antioch|Randolph County|GA|38605|United States|-5|apartment| +40611|AAAAAAAADKOJAAAA|199|Main |Boulevard|Suite 20|Marion|Calumet County|WI|50399|United States|-6|apartment| +40612|AAAAAAAAEKOJAAAA|373|6th |Ct.|Suite 460|Frenchtown|Choctaw County|MS|52629|United States|-6|apartment| +40613|AAAAAAAAFKOJAAAA|120|First North|Street|Suite 50|Providence|Toole County|MT|66614|United States|-7|single family| +40614|AAAAAAAAGKOJAAAA|899|Sixth |Road|Suite 110|Jamestown|Clay County|TX|76867|United States|-6|single family| +40615|AAAAAAAAHKOJAAAA|107|Woodland Valley|Way|Suite E|Greenwood|Bell County|TX|78828|United States|-6|apartment| +40616|AAAAAAAAIKOJAAAA|564|15th Walnut|Cir.|Suite E|Highland Park|Mitchell County|KS|66534|United States|-6|condo| +40617|AAAAAAAAJKOJAAAA|560|Hickory |Lane|Suite X|Walnut|Fall River County|SD|56245|United States|-6|apartment| +40618|AAAAAAAAKKOJAAAA|623|Sunset 5th|Road|Suite B|Granite|Monroe County|MI|46284|United States|-5|apartment| +40619|AAAAAAAALKOJAAAA|935|2nd Highland|Way|Suite 440|Centerville|Zavala County|TX|70059|United States|-6|single family| +40620|AAAAAAAAMKOJAAAA|310|Mill Dogwood|Ave|Suite F|Oak Hill|Dade County|GA|37838|United States|-5|apartment| +40621|AAAAAAAANKOJAAAA|228|Jefferson |Dr.|Suite 150|Clifton|Oldham County|KY|48014|United States|-5|condo| +40622|AAAAAAAAOKOJAAAA|268|Walnut Miller|Ln|Suite 430|Pleasant Hill|Hawaii County|HI|93604|United States|-10|apartment| +40623|AAAAAAAAPKOJAAAA|528|5th |Court|Suite 450|Centerville|Grundy County|IL|60059|United States|-6|apartment| +40624|AAAAAAAAALOJAAAA|220|Poplar Oak|ST|Suite H|Green Acres|Pacific County|WA|97683|United States|-8|condo| +40625|AAAAAAAABLOJAAAA|397|Maple 12th|Avenue|Suite Q|Brownsville|Tom Green County|TX|79310|United States|-6|apartment| +40626|AAAAAAAACLOJAAAA|293|Green |Wy|Suite 10|Red Hill|Humboldt County|CA|94338|United States|-8|single family| +40627|AAAAAAAADLOJAAAA|871|4th |Road|Suite A|Florence|Isle of Wight County|VA|23394|United States|-5|condo| +40628|AAAAAAAAELOJAAAA|227|Lincoln Main|Ave|Suite Y|Sunnyside|Pender County|NC|21952|United States|-5|apartment| +40629|AAAAAAAAFLOJAAAA|195|Pine |Pkwy|Suite S|Franklin|Kittitas County|WA|99101|United States|-8|apartment| +40630|AAAAAAAAGLOJAAAA|654|Adams |RD|Suite 30|Mount Olive|Dickey County|ND|58059|United States|-6|single family| +40631|AAAAAAAAHLOJAAAA|315|Thirteenth |Parkway|Suite W|Riverside|Crow Wing County|MN|59231|United States|-6|apartment| +40632|AAAAAAAAILOJAAAA|723|South East|Boulevard|Suite 310|Siloam|Plaquemines Parish|LA|78948|United States|-6|single family| +40633|AAAAAAAAJLOJAAAA|573|Walnut |Drive|Suite H|Franklin|Orange County|CA|99101|United States|-8|apartment| +40634|AAAAAAAAKLOJAAAA|367|South Valley|Ave|Suite 230|Oakwood|Nantucket County|MA|00769|United States|-5|single family| +40635|AAAAAAAALLOJAAAA|566|8th |Parkway||Marion|||60399|||| +40636|AAAAAAAAMLOJAAAA|943|East |Drive|Suite 390|Sawyer|Vernon County|MO|66045|United States|-6|condo| +40637|AAAAAAAANLOJAAAA|685|5th Sunset|ST|Suite I|Mount Zion|Washington County|WI|58054|United States|-6|condo| +40638|AAAAAAAAOLOJAAAA|985|11th River|Lane|Suite P|Langdon|Geneva County|AL|30852|United States|-6|condo| +40639|AAAAAAAAPLOJAAAA|138|Laurel |Circle|Suite L|Hampton|Jefferson County|FL|32522|United States|-5|apartment| +40640|AAAAAAAAAMOJAAAA|208|Jackson |Boulevard|Suite I|Clinton|Edgar County|IL|68222|United States|-6|single family| +40641|AAAAAAAABMOJAAAA|649|Spring |Parkway|Suite 370|Centerville|McCracken County|KY|40059|United States|-5|condo| +40642|AAAAAAAACMOJAAAA|51|Meadow Main|Avenue|Suite T|New Hope|Minidoka County|ID|89431|United States|-7|single family| +40643|AAAAAAAADMOJAAAA|909|Jefferson |Dr.|Suite 120|Woodlawn|Logan County|OH|44098|United States|-5|apartment| +40644|AAAAAAAAEMOJAAAA|321|Walnut Laurel|Way|Suite H|Hopewell|Morton County|ND|50587|United States|-6|condo| +40645|AAAAAAAAFMOJAAAA|802|Elm Oak|Ct.|Suite 130|Orange|Weston County|WY|81764|United States|-7|condo| +40646|AAAAAAAAGMOJAAAA|189|Cedar 3rd|Avenue|Suite D|Liberty|Siskiyou County|CA|93451|United States|-8|condo| +40647|AAAAAAAAHMOJAAAA|944|Oak |Ave|Suite 400|Red Hill|Larue County|KY|44338|United States|-5|single family| +40648|AAAAAAAAIMOJAAAA|469|Cedar Mill|Street|Suite 390|Sulphur Springs|Alger County|MI|48354|United States|-5|condo| +40649|AAAAAAAAJMOJAAAA|387|Elm |Boulevard|Suite N|Bunker Hill|Fulton County|IL|60150|United States|-6|condo| +40650|AAAAAAAAKMOJAAAA|54|8th Smith|Boulevard|Suite Y|Spring Valley|Polk County|TX|76060|United States|-6|condo| +40651|AAAAAAAALMOJAAAA|757|1st |Boulevard|Suite Y|Mount Olive|Bee County|TX|78059|United States|-6|single family| +40652|AAAAAAAAMMOJAAAA|144|Hillcrest |Road|Suite 480|Mount Zion|Emporia city|VA|28054|United States|-5|condo| +40653|AAAAAAAANMOJAAAA|950|Ash |Drive|Suite G|Griffin|Benewah County|ID|81204|United States|-7|apartment| +40654|AAAAAAAAOMOJAAAA|381|Wilson Washington|Street|Suite 260|Bridgeport|Webster County|IA|55817|United States|-6|condo| +40655|AAAAAAAAPMOJAAAA|749|Sunset Sunset|Pkwy|Suite E|Farmington|Malheur County|OR|99145|United States|-8|apartment| +40656|AAAAAAAAANOJAAAA|258|Second |Road|Suite H|Wyoming|Taylor County|KY|40216|United States|-5|apartment| +40657|AAAAAAAABNOJAAAA|364|6th 11th|Ln|Suite M|Bunker Hill|Yakima County|WA|90150|United States|-8|condo| +40658|AAAAAAAACNOJAAAA|82|Spruce Cedar|Dr.|Suite 150|Fairfield|Ramsey County|ND|56192|United States|-6|apartment| +40659|AAAAAAAADNOJAAAA|392|Williams |Pkwy|Suite X|Franklin|Decatur County|IA|59101|United States|-6|single family| +40660|AAAAAAAAENOJAAAA|470|Maple |Parkway|Suite 70|Bunker Hill|New Kent County|VA|20150|United States|-5|apartment| +40661|AAAAAAAAFNOJAAAA|760|Tenth Jefferson|ST|Suite 40|Woodland|Breckinridge County|KY|44854|United States|-6|condo| +40662|AAAAAAAAGNOJAAAA|||||Sullivan||||United States|-5|| +40663|AAAAAAAAHNOJAAAA|49|1st Jefferson|Cir.|Suite 480|Crossroads|Prairie County|MT|60534|United States|-7|single family| +40664|AAAAAAAAINOJAAAA|841|Lee Ash|Ave|Suite H|Newtown|Bleckley County|GA|31749|United States|-5|single family| +40665|AAAAAAAAJNOJAAAA|366|View |Pkwy|Suite 60|Deerfield|Calhoun County|MS|59840|United States|-6|apartment| +40666|AAAAAAAAKNOJAAAA|436|Ridge |Court|Suite 210|Jackson|Lewis County|NY|19583|United States|-5|apartment| +40667|AAAAAAAALNOJAAAA|885|Adams Meadow|Way|Suite T|Webb|Putnam County|WV|20899|United States|-5|condo| +40668|AAAAAAAAMNOJAAAA|169|Walnut |Ct.|Suite I|Woodlawn|Columbia County|WA|94098|United States|-8|single family| +40669|AAAAAAAANNOJAAAA|658|Central |Avenue|Suite 70|Fairfield|Emporia city|VA|26192|United States|-5|apartment| +40670|AAAAAAAAONOJAAAA|345|Elm |Avenue|Suite 180|Oak Hill|Rush County|IN|47838|United States|-5|single family| +40671|AAAAAAAAPNOJAAAA|147|Spring Oak|Avenue|Suite G|Plainview|Peach County|GA|33683|United States|-5|apartment| +40672|AAAAAAAAAOOJAAAA|16|Highland |ST|Suite P|Ashland|Hill County|TX|74244|United States|-6|single family| +40673|AAAAAAAABOOJAAAA|616|Smith 14th|Avenue|Suite 100|Lebanon|Judith Basin County|MT|62898|United States|-7|single family| +40674|AAAAAAAACOOJAAAA|106|9th Spruce|Road|Suite Q|Macedonia||||||apartment| +40675|AAAAAAAADOOJAAAA|287|Johnson Woodland|RD|Suite W|Pine Grove|Cook County|IL|64593|United States|-6|apartment| +40676|AAAAAAAAEOOJAAAA||Elm Mill||Suite A|||MT|||-7|| +40677|AAAAAAAAFOOJAAAA|589|15th View|Court|Suite U|Greenfield|Glasscock County|TX|75038|United States|-6|single family| +40678|AAAAAAAAGOOJAAAA||7th Oak|Ct.|||Jackson County|NC||United States||condo| +40679|AAAAAAAAHOOJAAAA|928|Willow 7th|Ct.|Suite 470|Valley View|Cecil County|MD|25124|United States|-5|condo| +40680|AAAAAAAAIOOJAAAA|791|River |Way|Suite 110|Belmont|Alleghany County|NC|20191|United States|-5|condo| +40681|AAAAAAAAJOOJAAAA|205|1st |Dr.|Suite U|Summerville|Morgan County|UT|82033|United States|-7|single family| +40682|AAAAAAAAKOOJAAAA|155|3rd |Street|Suite 70|Union|Tucker County|WV|28721|United States|-5|apartment| +40683|AAAAAAAALOOJAAAA|899|Park |Drive|Suite Y|Woodville|Mecklenburg County|VA|24289|United States|-5|single family| +40684|AAAAAAAAMOOJAAAA|808|Sunset Washington|Avenue|Suite W|Georgetown|Upson County|GA|37057|United States|-5|condo| +40685|AAAAAAAANOOJAAAA|485|8th Highland|Parkway|Suite 410|Woodlawn|Kent County|TX|74098|United States|-6|apartment| +40686|AAAAAAAAOOOJAAAA|56|2nd South|Dr.|Suite 430|Woodland|Clare County|MI|44854|United States|-5|apartment| +40687|AAAAAAAAPOOJAAAA|317|6th Johnson|Court|Suite B|Lakeside|Greene County|NY|19532|United States|-5|single family| +40688|AAAAAAAAAPOJAAAA|669|Willow |Circle|Suite 260|Hamilton|Ontario County|NY|12808|United States|-5|condo| +40689|AAAAAAAABPOJAAAA|349|Woodland 3rd|RD|Suite 490|Tabor|Rock County|MN|58529|United States|-6|condo| +40690|AAAAAAAACPOJAAAA|420|West |Cir.|Suite 340|Spring Valley|Echols County|GA|36060|United States|-5|apartment| +40691|AAAAAAAADPOJAAAA|930|11th Cherry|Pkwy|Suite L|Stringtown|Washington County|OK|70162|United States|-6|apartment| +40692|AAAAAAAAEPOJAAAA|951|Oak |Dr.|Suite Y|Red Hill|San Juan County|WA|94338|United States|-8|single family| +40693|AAAAAAAAFPOJAAAA|86|Third |Wy|Suite 350|Woodville|Knox County|KY|44289|United States|-5|condo| +40694|AAAAAAAAGPOJAAAA|320|East |Pkwy|Suite 260|Glendale|Oldham County|TX|73951|United States|-6|apartment| +40695|AAAAAAAAHPOJAAAA|43|Maple Cherry|Ave|Suite 230|Farmington|Walthall County|MS|59145|United States|-6|apartment| +40696|AAAAAAAAIPOJAAAA|268|14th |Wy|Suite J|Stringtown|Pointe Coupee Parish|LA|70162|United States|-6|single family| +40697|AAAAAAAAJPOJAAAA|455|Ridge Walnut|Boulevard|Suite P|Macedonia|Keweenaw County|MI|41087|United States|-5|condo| +40698|AAAAAAAAKPOJAAAA|448|Main Lee|Blvd|Suite R|Midway|Nicholas County|WV|21904|United States|-5|condo| +40699|AAAAAAAALPOJAAAA|625|North |Court|Suite C|Pleasant Valley|Jackson County|IN|42477|United States|-5|condo| +40700|AAAAAAAAMPOJAAAA|607|Oak |Drive|Suite 160|Salem|Mille Lacs County|MN|58048|United States|-6|single family| +40701|AAAAAAAANPOJAAAA|309|Hickory Spring|Street|Suite M|Warwick|Henry County|OH|41398|United States|-5|apartment| +40702|AAAAAAAAOPOJAAAA|447|Miller |RD|Suite 430|Glenwood|Geary County|KS|63511|United States|-6|single family| +40703|AAAAAAAAPPOJAAAA|40|Meadow Cedar|Road|Suite 350|Riley|Clark County|ID|81692|United States|-7|apartment| +40704|AAAAAAAAAAPJAAAA|993|Mill |Ct.|Suite 230|Five Forks|James City County|VA|22293|United States|-5|condo| +40705|AAAAAAAABAPJAAAA|9|Dogwood |Ave|Suite K|Mount Olive|Bristol County|RI|08659|United States|-5|single family| +40706|AAAAAAAACAPJAAAA|958|Central 6th|Parkway|Suite 20|Ridgeville|Sioux County|NE|69306|United States|-7|condo| +40707|AAAAAAAADAPJAAAA|216|Hillcrest |Drive|Suite 100|Spring Valley|Granville County|NC|26060|United States|-5|apartment| +40708|AAAAAAAAEAPJAAAA|758|Central |Road|Suite H|Oak Hill|Bristol Bay Borough|AK|97838|United States|-9|single family| +40709|AAAAAAAAFAPJAAAA|891|First Mill|Ct.|Suite 140|Mountain View|Marion County|TN|34466|United States|-6|condo| +40710|AAAAAAAAGAPJAAAA|111|Walnut Valley|Way|Suite N|Belmont|Seward County|KS|60191|United States|-6|single family| +40711|AAAAAAAAHAPJAAAA||2nd Seventh|Wy|Suite 320||Stone County|MS|50169|United States||| +40712|AAAAAAAAIAPJAAAA|116|1st |Ave|Suite F|Bunker Hill|Tangipahoa Parish|LA|70150|United States|-6|condo| +40713|AAAAAAAAJAPJAAAA|963|Main 3rd|Avenue|Suite L|Greenfield|De Soto Parish|LA|75038|United States|-6|apartment| +40714|AAAAAAAAKAPJAAAA|932|Lincoln |Parkway|Suite K|Oak Ridge|Fillmore County|NE|68371|United States|-6|apartment| +40715|AAAAAAAALAPJAAAA|644|Poplar |Drive|Suite 480|Oak Hill|Jasper County|MS|57838|United States|-6|condo| +40716|AAAAAAAAMAPJAAAA|831|4th Sixth|Boulevard|Suite 260|Salem|Baltimore city|MD|28048|United States|-5|apartment| +40717|AAAAAAAANAPJAAAA|67|Meadow Ridge|ST|Suite 80|Kingston|Johnson County|WY|84975|United States|-7|condo| +40718|AAAAAAAAOAPJAAAA|426|Railroad River|Pkwy|Suite L|Murphy|Madison County|TN|32105|United States|-6|condo| +40719|AAAAAAAAPAPJAAAA|229|Elm |Circle|Suite 470|Wesley|Gove County|KS|61218|United States|-6|single family| +40720|AAAAAAAAABPJAAAA|763|5th |Wy|Suite V|Belmont|Linn County|OR|90191|United States|-8|condo| +40721|AAAAAAAABBPJAAAA|601|6th |Boulevard|Suite 390|Jackson|Alamance County|NC|29583|United States|-5|condo| +40722|AAAAAAAACBPJAAAA|757|2nd First|Dr.|Suite V|Newtown|Sussex County|VA|21749|United States|-5|apartment| +40723|AAAAAAAADBPJAAAA|565|Park Birch|Boulevard|Suite P|Green Acres|Mohave County|AZ|87683|United States|-7|condo| +40724|AAAAAAAAEBPJAAAA|977|Maple View|Dr.|Suite 300|Pine Grove|Covington County|AL|34593|United States|-6|apartment| +40725|AAAAAAAAFBPJAAAA|170|Ridge Pine|Pkwy|Suite 270|Unionville|Grays Harbor County|WA|91711|United States|-8|condo| +40726|AAAAAAAAGBPJAAAA|324|Second |Circle|Suite 20|Franklin|Goochland County|VA|29101|United States|-5|single family| +40727|AAAAAAAAHBPJAAAA|117|Williams |||Farmington||TX|79145|United States||single family| +40728|AAAAAAAAIBPJAAAA|486|Second |Pkwy|Suite P|Harmony|Park County|MT|65804|United States|-7|single family| +40729|AAAAAAAAJBPJAAAA|836|Third |Ave|Suite E|Summit|Wilson County|TN|30499|United States|-5|apartment| +40730|AAAAAAAAKBPJAAAA|959|Madison Ash|Boulevard|Suite C|Midway|Lawrence County|SD|51904|United States|-7|apartment| +40731|AAAAAAAALBPJAAAA|553|Second Hillcrest|Drive|Suite 40|Friendship|Maricopa County|AZ|84536|United States|-7|single family| +40732|AAAAAAAAMBPJAAAA|121|Williams Hickory|Dr.|Suite 440|Lakewood|Heard County|GA|38877|United States|-5|apartment| +40733|AAAAAAAANBPJAAAA|143|Valley |ST|Suite B|Franklin|Marshall County|IA|59101|United States|-6|single family| +40734|AAAAAAAAOBPJAAAA|717|First |Ln|Suite 200|Summit|Emery County|UT|80499|United States|-7|apartment| +40735|AAAAAAAAPBPJAAAA|707|3rd |Ave|Suite X|Brownsville|Gates County|NC|29310|United States|-5|apartment| +40736|AAAAAAAAACPJAAAA|324|Second |Way|Suite J|Valley View|Rensselaer County|NY|15124|United States|-5|single family| +40737|AAAAAAAABCPJAAAA|852|Elm Jackson|Avenue|Suite 220|Mount Zion|Marshall County|KS|68054|United States|-6|apartment| +40738|AAAAAAAACCPJAAAA|631|View Seventh|Cir.|Suite 130|Buckingham|Clay County|TN|34092|United States|-5|apartment| +40739|AAAAAAAADCPJAAAA|854|Willow West|ST|Suite W|Harmony|Lee County|MS|55804|United States|-6|apartment| +40740|AAAAAAAAECPJAAAA|890|Johnson Main|Drive|Suite J|Fairview|Clay County|SD|55709|United States|-6|condo| +40741|AAAAAAAAFCPJAAAA|922|Oak Adams|Blvd|Suite O|Glenwood|Green Lake County|WI|53511|United States|-6|single family| +40742|AAAAAAAAGCPJAAAA|418|Park |Blvd|Suite 360|Leesville|Kennebec County|ME|06023|United States|-5|single family| +40743|AAAAAAAAHCPJAAAA|686|Laurel |Street|Suite 260|White Oak|Washington County|NY|16668|United States|-5|apartment| +40744|AAAAAAAAICPJAAAA|16|Meadow |Drive|Suite W|Ruth|Mellette County|SD|50309|United States|-7|condo| +40745|AAAAAAAAJCPJAAAA|22|Forest Second|Pkwy|Suite A|Cedar Grove|Stark County|IL|60411|United States|-6|single family| +40746|AAAAAAAAKCPJAAAA|316|Sycamore |Pkwy|Suite 100|Valley View|Saline County|NE|65124|United States|-7|single family| +40747|AAAAAAAALCPJAAAA|258|Elevnth Park|Wy|Suite 240|Oak Grove|Bertie County|NC|28370|United States|-5|single family| +40748|AAAAAAAAMCPJAAAA|358|3rd Mill|Boulevard|Suite H|Edgewood|Broward County|FL|30069|United States|-5|condo| +40749|AAAAAAAANCPJAAAA|227|5th |Court|Suite 50|Newtown|Bertie County|NC|21749|United States|-5|single family| +40750|AAAAAAAAOCPJAAAA|546|Fourth |Ave|Suite 460|Bennett|Morgan County|IN|41715|United States|-5|condo| +40751|AAAAAAAAPCPJAAAA|511|Fourteenth Williams|Parkway|Suite 40|Macedonia|Lee County|VA|21087|United States|-5|single family| +40752|AAAAAAAAADPJAAAA|354|2nd Oak|Ave|Suite 280|Greenfield|Cheboygan County|MI|45038|United States|-5|condo| +40753|AAAAAAAABDPJAAAA|570|Adams Sycamore|Wy|Suite G|Spring Hill|Kauai County|HI|96787|United States|-10|single family| +40754|AAAAAAAACDPJAAAA||Hill |Court|Suite 330||||93394|||apartment| +40755|AAAAAAAADDPJAAAA|49|Adams |Avenue|Suite T|Plainview|Holmes County|FL|33683|United States|-5|apartment| +40756|AAAAAAAAEDPJAAAA|450|Pine |Cir.|Suite 350|Pleasant Valley|Alexandria city|VA|22477|United States|-5|condo| +40757|AAAAAAAAFDPJAAAA|254|Oak |Drive|Suite W|Glenwood|Sunflower County|MS|53511|United States|-6|single family| +40758|AAAAAAAAGDPJAAAA|328|Lake Oak|Drive|Suite F|Harmony|Pointe Coupee Parish|LA|75804|United States|-6|condo| +40759|AAAAAAAAHDPJAAAA|822|8th |Dr.|Suite 340|Burns|Clinton County|IA|55272|United States|-6|apartment| +40760|AAAAAAAAIDPJAAAA|394|Smith Fourth|Pkwy|Suite 130|Edgewood|Hertford County|NC|20069|United States|-5|single family| +40761|AAAAAAAAJDPJAAAA|501|Madison Broadway|Ave|Suite 460|Wilson|Titus County|TX|76971|United States|-6|apartment| +40762|AAAAAAAAKDPJAAAA|80|Pine 12th|Boulevard|Suite L|Florence|Garden County|NE|63394|United States|-6|single family| +40763|AAAAAAAALDPJAAAA|739|Williams |RD|Suite R|Ashland|La Plata County|CO|84244|United States|-7|apartment| +40764|AAAAAAAAMDPJAAAA|609|Miller |Wy|Suite 40|Lincoln|Dougherty County|GA|31289|United States|-5|apartment| +40765|AAAAAAAANDPJAAAA|314|Spruce Wilson|Road|Suite 240|Deerfield|Garrard County|KY|49840|United States|-6|condo| +40766|AAAAAAAAODPJAAAA|583|Davis 7th|Avenue|Suite Q|Pleasant Valley|Lamb County|TX|72477|United States|-6|single family| +40767|AAAAAAAAPDPJAAAA|788|2nd Williams|Ave|Suite 220|Centerville|Monroe County|WV|20059|United States|-5|apartment| +40768|AAAAAAAAAEPJAAAA|29|Center |Wy|Suite E|Stringtown|Howard County|TX|70162|United States|-6|apartment| +40769|AAAAAAAABEPJAAAA|911|Pine Franklin|Lane|Suite 410|Lakeside|Jefferson County|OR|99532|United States|-8|condo| +40770|AAAAAAAACEPJAAAA|342|Meadow Lake|Wy|Suite Y|Walnut Grove|Bay County|FL|37752|United States|-5|apartment| +40771|AAAAAAAADEPJAAAA|248|Madison North|Pkwy|Suite 170|Harmony|Gaines County|TX|75804|United States|-6|condo| +40772|AAAAAAAAEEPJAAAA|134|Miller Williams|Street|Suite H|Cumberland|White County|TN|38971|United States|-6|apartment| +40773|AAAAAAAAFEPJAAAA|194|Maple Oak|Way|Suite 200|Spring Valley|Hansford County|TX|76060|United States|-6|condo| +40774|AAAAAAAAGEPJAAAA|219|Park |Circle|Suite S|Pleasant Grove|Lawrence County|SD|54136|United States|-7|condo| +40775|AAAAAAAAHEPJAAAA|190|3rd Hill|Street|Suite 380|Providence|San Diego County|CA|96614|United States|-8|single family| +40776|AAAAAAAAIEPJAAAA|631|Sycamore East|Ave|Suite 80|Pleasant Hill|Salem city|VA|23604|United States|-5|single family| +40777|AAAAAAAAJEPJAAAA|1000|Lee |Pkwy|Suite O|Riverdale|Jefferson County|TN|39391|United States|-6|apartment| +40778|AAAAAAAAKEPJAAAA|510|Green Hill|Wy|Suite 470|Fairfield|Gray County|TX|76192|United States|-6|condo| +40779|AAAAAAAALEPJAAAA|248|Main |Road|Suite 380|Plainview|West Baton Rouge Parish|LA|73683|United States|-6|condo| +40780|AAAAAAAAMEPJAAAA|783|Green |Ave|Suite 180|Greenwood|Ferry County|WA|98828|United States|-8|condo| +40781|AAAAAAAANEPJAAAA|844|Center |Blvd|Suite U|Salem|Hertford County|NC|28048|United States|-5|condo| +40782|AAAAAAAAOEPJAAAA|338|View Main|Pkwy|Suite W|New Hope|Carroll County|VA|29431|United States|-5|condo| +40783|AAAAAAAAPEPJAAAA|71|Park Hillcrest|Blvd|Suite B|Stringtown|Irwin County|GA|30162|United States|-5|apartment| +40784|AAAAAAAAAFPJAAAA|844|Walnut 10th|Boulevard|Suite 150|Knollwood|Perry County|AL|33368|United States|-6|single family| +40785|AAAAAAAABFPJAAAA||||Suite 120|Spring Valley||PA|16060|United States|-5|| +40786|AAAAAAAACFPJAAAA|816|12th 9th|Blvd|Suite S|Brownsville|Cook County|MN|59310|United States|-6|single family| +40787|AAAAAAAADFPJAAAA|458|Main Willow|Ln|Suite X|Greenfield|Lake County|MT|65038|United States|-7|condo| +40788|AAAAAAAAEFPJAAAA|747|Lincoln |Wy|Suite B|West Liberty|Pope County|AR|74752|United States|-6|single family| +40789|AAAAAAAAFFPJAAAA|974|1st Central|Court|Suite E|Green Acres|Albemarle County|VA|27683|United States|-5|apartment| +40790|AAAAAAAAGFPJAAAA|446|Spruce Madison|Pkwy|Suite 40|Maple Grove|Leon County|FL|38252|United States|-5|single family| +40791|AAAAAAAAHFPJAAAA|761|2nd Wilson|Court|Suite 30|Shiloh|Gove County|KS|69275|United States|-6|single family| +40792|AAAAAAAAIFPJAAAA|473|2nd Birch|RD|Suite 210|Summit|Outagamie County|WI|50499|United States|-6|single family| +40793|AAAAAAAAJFPJAAAA|984|Church Miller|ST|Suite L|Red Hill|Essex County|MA|04938|United States|-5|single family| +40794|AAAAAAAAKFPJAAAA|890|Fourth Main|Parkway|Suite 350|Wesley|Comanche County|TX|71218|United States|-6|single family| +40795|AAAAAAAALFPJAAAA|719|Wilson |Lane|Suite 480|Glenwood|Carbon County|WY|83511|United States|-7|condo| +40796|AAAAAAAAMFPJAAAA|667|Maple |Ave|Suite W|Georgetown|Warren County|NC|27057|United States|-5|single family| +40797|AAAAAAAANFPJAAAA|145|9th Williams|Cir.|Suite Q|Amity|Brown County|TX|70766|United States|-6|condo| +40798|AAAAAAAAOFPJAAAA|182|Oak ||||Columbia County|FL|37838|||apartment| +40799|AAAAAAAAPFPJAAAA|266|5th Valley|Blvd|Suite 370|Oakland|Santa Fe County|NM|89843|United States|-7|single family| +40800|AAAAAAAAAGPJAAAA|530|Lincoln |Ln|Suite 50|Hillcrest|Major County|OK|73003|United States|-6|condo| +40801|AAAAAAAABGPJAAAA|825|Spruce |Ct.|Suite Y|Oakdale|Colusa County|CA|99584|United States|-8|apartment| +40802|AAAAAAAACGPJAAAA|590|First |Ave|Suite S|Macedonia|Sierra County|NM|81087|United States|-7|apartment| +40803|AAAAAAAADGPJAAAA|620|Lake First|Road|Suite T|Deerfield|Cowlitz County|WA|99840|United States|-8|condo| +40804|AAAAAAAAEGPJAAAA|720|Highland |Road|Suite 330|Plainview|Lipscomb County|TX|73683|United States|-6|single family| +40805|AAAAAAAAFGPJAAAA|942|Davis |ST|Suite 190|Harmony|Issaquena County|MS|55804|United States|-6|apartment| +40806|AAAAAAAAGGPJAAAA|211|Fifteenth |Circle|Suite K|Brownsville|Chester County|TN|39310|United States|-5|condo| +40807|AAAAAAAAHGPJAAAA|408|Oak |Boulevard|Suite 310|Wilson|Grant County|ND|56971|United States|-6|condo| +40808|AAAAAAAAIGPJAAAA|590|Sunset |Avenue|Suite B|Highland Park|Eaton County|MI|46534|United States|-5|condo| +40809|AAAAAAAAJGPJAAAA|207|Oak Lakeview|Pkwy|Suite V|Greenwood|Worth County|GA|38828|United States|-5|condo| +40810|AAAAAAAAKGPJAAAA|1|Washington |Way|Suite 340|Enterprise|Douglas County|MN|51757|United States|-6|condo| +40811|AAAAAAAALGPJAAAA|184|Locust Dogwood|Street|Suite M|Friendship|Stafford County|KS|64536|United States|-6|apartment| +40812|AAAAAAAAMGPJAAAA|495|2nd |Road|Suite 330|Lebanon|Union County|SC|22898|United States|-5|condo| +40813|AAAAAAAANGPJAAAA|431|Madison |RD|Suite 100|Brunswick|McLean County|ND|54642|United States|-6|condo| +40814|AAAAAAAAOGPJAAAA|543|South |RD|Suite 250|Helena|Fairfax County|VA|21649|United States|-5|condo| +40815|AAAAAAAAPGPJAAAA|865|15th |Boulevard|Suite J|Bethel|Forest County|PA|15281|United States|-5|condo| +40816|AAAAAAAAAHPJAAAA|897|East |RD|Suite J|Fairfield|Highland County|OH|46192|United States|-5|single family| +40817|AAAAAAAABHPJAAAA|986|Williams |RD|Suite 410|Georgetown|Essex County|MA|07657|United States|-5|single family| +40818|AAAAAAAACHPJAAAA|624|Madison 5th|Way|Suite E|Barnes|Forest County|PA|13788|United States|-5|single family| +40819|AAAAAAAADHPJAAAA|191|4th Church|Drive|Suite 110|Brownsville|Berrien County|GA|39310|United States|-5|single family| +40820|AAAAAAAAEHPJAAAA|877|Lee Elm|Circle|Suite 280|Mount Vernon|Norfolk city|VA|28482|United States|-5|single family| +40821|AAAAAAAAFHPJAAAA|867|Second |Avenue|Suite 490|Macedonia|Canadian County|OK|71087|United States|-6|apartment| +40822|AAAAAAAAGHPJAAAA|195|Davis Second|Road|Suite 360|Centerville|Providence County|RI|00659|United States|-5|apartment| +40823|AAAAAAAAHHPJAAAA|787|||Suite X|Mount Vernon|||58482|United States|-7|apartment| +40824|AAAAAAAAIHPJAAAA|897|Central |Boulevard|Suite 50|Valley View|Carroll County|GA|35124|United States|-5|apartment| +40825|AAAAAAAAJHPJAAAA|||Circle|Suite H|Oak Hill|Charlton County|GA||||| +40826|AAAAAAAAKHPJAAAA|550|Washington |Parkway|Suite 40|Greenwood|Greenup County|KY|48828|United States|-6|condo| +40827|AAAAAAAALHPJAAAA|300|6th |Ct.|Suite A|Rossville|Bronx County|NY|12503|United States|-5|condo| +40828|AAAAAAAAMHPJAAAA|698|Green |Pkwy|Suite 200|Jackson|Dixie County|FL|39583|United States|-5|single family| +40829|AAAAAAAANHPJAAAA|862|Ash Hickory|Court|Suite B|New Hope|Crook County|OR|99431|United States|-8|apartment| +40830|AAAAAAAAOHPJAAAA|575|Madison Fourth|Ln|Suite 260|Midway|Cameron Parish|LA|71904|United States|-6|apartment| +40831|AAAAAAAAPHPJAAAA|794|Fourth Main|Street|Suite G|Rockwood|Licking County|OH|41545|United States|-5|apartment| +40832|AAAAAAAAAIPJAAAA|966|Broadway |Lane|Suite G|Antioch|Rockbridge County|VA|28605|United States|-5|condo| +40833|AAAAAAAABIPJAAAA|61|Woodland |Street|Suite A|Ashland|Wilkes County|GA|34244|United States|-5|single family| +40834|AAAAAAAACIPJAAAA|602|Center |Ct.|Suite X|Roy|Alamosa County|CO|80744|United States|-7|apartment| +40835|AAAAAAAADIPJAAAA|695|Main |Drive|Suite 190|Pleasant Grove|Conecuh County|AL|34136|United States|-6|apartment| +40836|AAAAAAAAEIPJAAAA|83|Walnut 8th|ST|Suite E|Nottingham|Hamilton County|TX|74074|United States|-6|condo| +40837|AAAAAAAAFIPJAAAA|379|West |Wy|Suite 200|Brownsville|Osceola County|IA|59310|United States|-6|apartment| +40838|AAAAAAAAGIPJAAAA|782|Chestnut Davis|Ave|Suite N|Florence|Payette County|ID|83394|United States|-7|single family| +40839|AAAAAAAAHIPJAAAA|297|Walnut 4th|Boulevard|Suite I|Fairfield|Panola County|TX|76192|United States|-6|condo| +40840|AAAAAAAAIIPJAAAA|641|Railroad |Pkwy|Suite 60|Globe|Bell County|KY|49614|United States|-6|apartment| +40841|AAAAAAAAJIPJAAAA|647|First 1st|RD|Suite 30|Shady Grove|Lancaster County|NE|62812|United States|-7|apartment| +40842|AAAAAAAAKIPJAAAA|454|Lakeview |Blvd|Suite K|Harmony|Hodgeman County|KS|65804|United States|-6|single family| +40843|AAAAAAAALIPJAAAA|195|West |Drive|Suite S|Riverside|Summit County|OH|49231|United States|-5|apartment| +40844|AAAAAAAAMIPJAAAA|371|Forest Johnson|Ct.|Suite B|Oak Hill|Culpeper County|VA|27838|United States|-5|single family| +40845|AAAAAAAANIPJAAAA|454|Mill |Street|Suite V|Maple Grove|La Salle County|IL|68252|United States|-6|single family| +40846|AAAAAAAAOIPJAAAA|510|Main |Wy|Suite F|Valley View|Clearwater County|ID|85124|United States|-7|single family| +40847|AAAAAAAAPIPJAAAA|413|Cherry Green|Ave|Suite 90|Lebanon|Codington County|SD|52898|United States|-6|condo| +40848|AAAAAAAAAJPJAAAA|741|4th Hill|Ave|Suite I|Lakeside|Shoshone County|ID|89532|United States|-7|apartment| +40849|AAAAAAAABJPJAAAA|875|Church |RD|Suite X|Salem|King William County|VA|28048|United States|-5|single family| +40850|AAAAAAAACJPJAAAA|75|Hill |Dr.|Suite 210|Riverdale|Wayne County|KY|49391|United States|-5|apartment| +40851|AAAAAAAADJPJAAAA|255|2nd |Ct.|Suite 220|Red Hill|Torrance County|NM|84338|United States|-7|single family| +40852|AAAAAAAAEJPJAAAA|586|Park Third|Boulevard|Suite N|Shiloh|Branch County|MI|49275|United States|-5|apartment| +40853|AAAAAAAAFJPJAAAA|659|North 11th|Lane|Suite 330|Riceville|Emmons County|ND|55867|United States|-6|apartment| +40854|AAAAAAAAGJPJAAAA|148|North |Circle|Suite I|Antioch|Hughes County|OK|78605|United States|-6|apartment| +40855|AAAAAAAAHJPJAAAA|604|Maple |Parkway|Suite L|Clinton|Clay County|MO|68222|United States|-6|apartment| +40856|AAAAAAAAIJPJAAAA|||||Forest Hills|||59237|United States|-6|| +40857|AAAAAAAAJJPJAAAA|785|6th |Road|Suite L|Clifton|Orange County|CA|98014|United States|-8|single family| +40858|AAAAAAAAKJPJAAAA|351|North Spring|Street|Suite 30|Pine Grove|Crisp County|GA|34593|United States|-5|single family| +40859|AAAAAAAALJPJAAAA|398|Pine |Ave|Suite R|Belmont|Warren County|TN|30191|United States|-6|single family| +40860|AAAAAAAAMJPJAAAA|224|Adams Elm|Pkwy|Suite Q|Mount Olive|Henry County|KY|48059|United States|-6|apartment| +40861|AAAAAAAANJPJAAAA|150|Mill View|Road|Suite L|Cedar Grove|Ripley County|IN|40411|United States|-5|single family| +40862|AAAAAAAAOJPJAAAA|171|Washington |RD|Suite 160|Highland|Sevier County|TN|39454|United States|-6|apartment| +40863|AAAAAAAAPJPJAAAA|235|11th |RD|Suite P|Wilson|Spokane County|WA|96971|United States|-8|single family| +40864|AAAAAAAAAKPJAAAA|||||Oak Grove|Outagamie County|WI||United States|-6|apartment| +40865|AAAAAAAABKPJAAAA|206|Green |Boulevard|Suite O|Oak Hill|Elbert County|GA|37838|United States|-5|single family| +40866|AAAAAAAACKPJAAAA|898|Willow |Way|Suite 270|Birmingham|McIntosh County|ND|53372|United States|-6|condo| +40867|AAAAAAAADKPJAAAA|391|Hickory 5th|Drive|Suite F|Midway|Ochiltree County|TX|71904|United States|-6|single family| +40868|AAAAAAAAEKPJAAAA|974|Meadow |Ln|Suite 460|Deerfield|Durham County|NC|29840|United States|-5|apartment| +40869|AAAAAAAAFKPJAAAA|442|Forest 1st|Boulevard|Suite 330|Lebanon|Clay County|SD|52898|United States|-6|single family| +40870|AAAAAAAAGKPJAAAA|279||Pkwy|Suite E|Green Acres|Powder River County|MT|67683||-7|apartment| +40871|AAAAAAAAHKPJAAAA|612|Oak |Drive|Suite X|Spring Valley|Arenac County|MI|46060|United States|-5|condo| +40872|AAAAAAAAIKPJAAAA|||||Pine Grove|Johnson County||34593|United States||| +40873|AAAAAAAAJKPJAAAA|834|West |Blvd|Suite 480|Hopewell|Houston County|TN|30587|United States|-5|apartment| +40874|AAAAAAAAKKPJAAAA|258|Chestnut 11th|Drive|Suite R|Clinton|DeSoto County|FL|38222|United States|-5|single family| +40875|AAAAAAAALKPJAAAA|160|North Central|Lane|Suite 240|Plainview|Renville County|MN|53683|United States|-6|condo| +40876|AAAAAAAAMKPJAAAA|755|Seventh Poplar|Blvd|Suite 410|Mountain View|Lycoming County|PA|14466|United States|-5|condo| +40877|AAAAAAAANKPJAAAA|684|West |Parkway|Suite 310|Riverdale|Oklahoma County|OK|79391|United States|-6|condo| +40878|AAAAAAAAOKPJAAAA|10|6th Wilson|Court|Suite T|Red Hill|Dane County|WI|54338|United States|-6|single family| +40879|AAAAAAAAPKPJAAAA|141|Hickory |Pkwy|Suite T|Oakdale|Okeechobee County|FL|39584|United States|-5|single family| +40880|AAAAAAAAALPJAAAA|465|River |Court|Suite G|Hopewell|Taylor County|GA|30587|United States|-5|apartment| +40881|AAAAAAAABLPJAAAA|171|13th |Ave|Suite 420|Kingston|Elkhart County|IN|44975|United States|-5|single family| +40882|AAAAAAAACLPJAAAA|351|Ridge Jefferson|Avenue|Suite 40|Lincoln|Colusa County|CA|91289|United States|-8|single family| +40883|AAAAAAAADLPJAAAA|708|South Sunset|Lane|Suite 270|Bridgeport|Lenoir County|NC|25817|United States|-5|single family| +40884|AAAAAAAAELPJAAAA|549|Church Cedar|ST|Suite 180|Hartland|Rosebud County|MT|66594|United States|-7|single family| +40885|AAAAAAAAFLPJAAAA|549|Park North|Lane|Suite O|Salem|Moniteau County|MO|68048|United States|-6|single family| +40886|AAAAAAAAGLPJAAAA|732|Johnson Green|Street|Suite 210|Brunswick|Woodford County|KY|44642|United States|-6|condo| +40887|AAAAAAAAHLPJAAAA|348|2nd |RD|Suite 10|Bayside|Kent County|DE|19550|United States|-5|condo| +40888|AAAAAAAAILPJAAAA|796|Sixth |Ave|Suite 30|Mount Pleasant|Stanley County|SD|51933|United States|-7|apartment| +40889|AAAAAAAAJLPJAAAA|796|Forest Fourteenth|Ct.|Suite 370|Mount Vernon|Polk County|NC|28482|United States|-5|single family| +40890|AAAAAAAAKLPJAAAA|607|Franklin Fifth|RD|Suite 480|Union Hill|Dearborn County|IN|47746|United States|-5|condo| +40891|AAAAAAAALLPJAAAA|441|Pine |Dr.|Suite X|Clinton|Baker County|OR|98222|United States|-8|apartment| +40892|AAAAAAAAMLPJAAAA|616|East |||||GA|38252||-5|| +40893|AAAAAAAANLPJAAAA|613|6th |Ave|Suite J|Midway|Fergus County|MT|61904|United States|-7|condo| +40894|AAAAAAAAOLPJAAAA|343|Franklin View|Circle|Suite 70|Friendship|Columbia County|FL|34536|United States|-5|apartment| +40895|AAAAAAAAPLPJAAAA|578|4th 13th|Court|Suite 270|Shiloh|Kane County|IL|69275|United States|-6|single family| +40896|AAAAAAAAAMPJAAAA|820|Franklin |Dr.|Suite S|Midway|Woodruff County|AR|71904|United States|-6|condo| +40897|AAAAAAAABMPJAAAA|672|Jackson |ST|Suite 260|Quincy|Pasquotank County|NC|23868|United States|-5|apartment| +40898|AAAAAAAACMPJAAAA|543|Elm Cedar|||||MO|||-6|| +40899|AAAAAAAADMPJAAAA|606|Lee |Blvd|Suite D|Ashland|Morgan County|CO|84244|United States|-7|condo| +40900|AAAAAAAAEMPJAAAA|471|4th 2nd|Street|Suite 220|Hillcrest|Knox County|ME|03603|United States|-5|apartment| +40901|AAAAAAAAFMPJAAAA|583|Mill |Pkwy|Suite 310|Five Points|Woodward County|OK|76098|United States|-6|single family| +40902|AAAAAAAAGMPJAAAA|749|Washington 10th|ST|Suite A|Kingston|Franklin County|ME|05575|United States|-5|condo| +40903|AAAAAAAAHMPJAAAA|797|Maple |Ave|Suite D|Cedar Grove|Kaufman County|TX|70411|United States|-6|apartment| +40904|AAAAAAAAIMPJAAAA|692|11th |Street|Suite B|Oakwood|Loudoun County|VA|20169|United States|-5|condo| +40905|AAAAAAAAJMPJAAAA|845|8th 5th|Road|Suite 230|Clinton|Guthrie County|IA|58222|United States|-6|apartment| +40906|AAAAAAAAKMPJAAAA|900|||Suite 320|Franklin|Mora County||89101||-7|apartment| +40907|AAAAAAAALMPJAAAA|639||ST||||KY|48721||-6|apartment| +40908|AAAAAAAAMMPJAAAA|412|Birch |Wy|Suite I|Crossroads|Lander County|NV|80534|United States|-8|apartment| +40909|AAAAAAAANMPJAAAA|679|9th East|RD|Suite S|Union|Yakima County|WA|98721|United States|-8|apartment| +40910|AAAAAAAAOMPJAAAA|340|7th |Way|Suite 410|Maple Grove|Campbell County|VA|28252|United States|-5|apartment| +40911|AAAAAAAAPMPJAAAA|251|Poplar |Ave|Suite 40|Highland Park|Hardin County|TX|76534|United States|-6|apartment| +40912|AAAAAAAAANPJAAAA|152|West Sunset|Avenue|Suite 320|Springdale|Douglas County|MN|58883|United States|-6|condo| +40913|AAAAAAAABNPJAAAA|295|||Suite F||||60059|United States|-7|apartment| +40914|AAAAAAAACNPJAAAA|934|Lincoln |Ct.|Suite 480|Jackson|Crow Wing County|MN|59583|United States|-6|apartment| +40915|AAAAAAAADNPJAAAA|453|Walnut Cedar|Wy|Suite F|Walnut Grove|Rooks County|KS|67752|United States|-6|apartment| +40916|AAAAAAAAENPJAAAA|571|Walnut |Court|Suite K|Highland Park|Wallace County|KS|66534|United States|-6|apartment| +40917|AAAAAAAAFNPJAAAA|656|13th Meadow|Parkway|Suite H|Harmony|Weston County|WY|85804|United States|-7|apartment| +40918|AAAAAAAAGNPJAAAA|514|Central |Lane|Suite 460|Five Points|Madison County|VA|26098|United States|-5|single family| +40919|AAAAAAAAHNPJAAAA|210|Valley |Circle|Suite 410|Antioch|Tom Green County|TX|78605|United States|-6|single family| +40920|AAAAAAAAINPJAAAA|135|Center |Ave|Suite 400|Pleasant Valley|Washington County|IL|62477|United States|-6|condo| +40921|AAAAAAAAJNPJAAAA|163|North |Ct.|Suite B|Florence|Louisa County|IA|53394|United States|-6|apartment| +40922|AAAAAAAAKNPJAAAA|358|Cedar Oak|Ln|Suite 420|Fairview|Hanson County|SD|55709|United States|-7|single family| +40923|AAAAAAAALNPJAAAA|599|West 6th|Cir.|Suite O|Riverdale|Polk County|WI|59391|United States|-6|apartment| +40924|AAAAAAAAMNPJAAAA|223|Church 1st|Ln|Suite C|Enterprise|Cherokee County|IA|51757|United States|-6|condo| +40925|AAAAAAAANNPJAAAA|223|Locust 6th|Way|Suite V|Belmont|Fluvanna County|VA|20191|United States|-5|single family| +40926|AAAAAAAAONPJAAAA|128|Park |Ave|Suite 210|Gum Springs|Weld County|CO|82106|United States|-7|apartment| +40927|AAAAAAAAPNPJAAAA|198|View |Street|Suite 470|Liberty|Jackson County|KY|43451|United States|-6|apartment| +40928|AAAAAAAAAOPJAAAA|24|Willow |Ln|Suite X|Greenwood|Glacier County|MT|68828|United States|-7|condo| +40929|AAAAAAAABOPJAAAA|204|Elm Lakeview|Blvd|Suite 10|Harmony|Beaufort County|NC|25804|United States|-5|condo| +40930|AAAAAAAACOPJAAAA|825|Ninth Chestnut|Pkwy|Suite E|Mount Zion|Iroquois County|IL|68054|United States|-6|condo| +40931|AAAAAAAADOPJAAAA|309|Fourth Mill|Street|Suite 410|Florence|Mercer County|WV|23394|United States|-5|condo| +40932|AAAAAAAAEOPJAAAA|711|Park |Parkway|Suite G|Clifford|La Plata County|CO|88164|United States|-7|single family| +40933|AAAAAAAAFOPJAAAA|793|Miller Woodland|ST|Suite F|Oak Hill|Washington County|AR|77838|United States|-6|apartment| +40934|AAAAAAAAGOPJAAAA|358|Dogwood Railroad|ST|Suite W|Buena Vista|Missoula County|MT|65752|United States|-7|condo| +40935|AAAAAAAAHOPJAAAA|12|Lee Miller|Lane|Suite L|Payne|Audubon County|IA|56134|United States|-6|apartment| +40936|AAAAAAAAIOPJAAAA|6|Oak 4th|Court|Suite O|Mount Pleasant|Calhoun County|TX|71933|United States|-6|condo| +40937|AAAAAAAAJOPJAAAA|308|Meadow |Pkwy|Suite 360|Harmony|Morrill County|NE|65804|United States|-7|condo| +40938|AAAAAAAAKOPJAAAA|423|Broadway |Street|Suite U|Saratoga|Monroe County|TN|32123|United States|-6|condo| +40939|AAAAAAAALOPJAAAA|598|Dogwood 8th|Cir.|Suite 110|Buena Vista|Franklin County|ME|06352|United States|-5|condo| +40940|AAAAAAAAMOPJAAAA|336|Hickory |Boulevard|Suite 410|Hopewell|Washington County|IL|60587|United States|-6|single family| +40941|AAAAAAAANOPJAAAA|725|Lincoln |ST|Suite C|Liberty|Saline County|IL|63451|United States|-6|apartment| +40942|AAAAAAAAOOPJAAAA|710|College River|Boulevard|Suite 120|Riverview|Seneca County|OH|49003|United States|-5|apartment| +40943|AAAAAAAAPOPJAAAA|931|Walnut |Court|Suite H|Greenville|Gregg County|TX|71387|United States|-6|apartment| +40944|AAAAAAAAAPPJAAAA|630|Walnut Twelfth|Cir.|Suite 190|Bath|Wasco County|OR|90573|United States|-8|condo| +40945|AAAAAAAABPPJAAAA|313|9th |Wy|Suite 470|Carpenter|Logan County|CO|81147|United States|-7|single family| +40946|AAAAAAAACPPJAAAA|620|Maple 5th|RD|Suite J|Riverside|Swain County|NC|29231|United States|-5|condo| +40947|AAAAAAAADPPJAAAA|596|North Hillcrest|Dr.|Suite U|Wildwood|Allen Parish|LA|76871|United States|-6|condo| +40948|AAAAAAAAEPPJAAAA|775|Pine |Ave|Suite 160|Antioch|Pike County|MO|68605|United States|-6|apartment| +40949|AAAAAAAAFPPJAAAA|516|Cedar Sixth|Boulevard|Suite 130|Lakeside|Eagle County|CO|89532|United States|-7|apartment| +40950|AAAAAAAAGPPJAAAA|444|Sixth Sunset|Boulevard|Suite C|Buena Vista|Amador County|CA|95752|United States|-8|apartment| +40951|AAAAAAAAHPPJAAAA|338|5th 10th|RD|Suite 120|Montague|Yukon-Koyukuk Census Area|AK|94062|United States|-9|condo| +40952|AAAAAAAAIPPJAAAA|409|11th Dogwood|Parkway|Suite F|Deerfield|Nye County|NV|89840|United States|-8|condo| +40953|AAAAAAAAJPPJAAAA|712|3rd Main|Lane|Suite Y|Marion|Clarke County|AL|30399|United States|-6|apartment| +40954|AAAAAAAAKPPJAAAA|876|Tenth First|Court|Suite N|Plainview|Washington County|VA|23683|United States|-5|apartment| +40955|AAAAAAAALPPJAAAA|915|4th Second|ST|Suite 340|Belmont|Lincoln County|NE|60191|United States|-7|apartment| +40956|AAAAAAAAMPPJAAAA|954|Willow Jackson|Ct.|Suite S|Springfield|Fillmore County|MN|59303|United States|-6|condo| +40957|AAAAAAAANPPJAAAA|684|4th East|Wy|Suite 320|Oakwood|Carroll County|OH|40169|United States|-5|single family| +40958|AAAAAAAAOPPJAAAA|145|Miller Mill|Circle|Suite A|Georgetown|Plymouth County|IA|57057|United States|-6|single family| +40959|AAAAAAAAPPPJAAAA|577|Johnson 7th|Pkwy|Suite 480|Franklin|McDonough County|IL|69101|United States|-6|apartment| +40960|AAAAAAAAAAAKAAAA|172|3rd East|Ct.|Suite 410|Woodlawn|Van Buren County|TN|34098|United States|-6|condo| +40961|AAAAAAAABAAKAAAA|968|5th |RD|Suite 110|Highland Park|Newton County|IN|46534|United States|-5|single family| +40962|AAAAAAAACAAKAAAA|465||||||WI|59454||-6|| +40963|AAAAAAAADAAKAAAA|156||Parkway|Suite I||Marion County|||||single family| +40964|AAAAAAAAEAAKAAAA|582|First |Ct.|Suite R|Fairview|Monona County|IA|55709|United States|-6|single family| +40965|AAAAAAAAFAAKAAAA|477|8th |Way|Suite O|Mount Olive|Morgan County|CO|88059|United States|-7|condo| +40966|AAAAAAAAGAAKAAAA|856|Oak |Pkwy|Suite H|Pleasant Grove|Dallas County|IA|54136|United States|-6|apartment| +40967|AAAAAAAAHAAKAAAA|146|Hickory College|Ln|Suite C|Hartland|Bremer County|IA|56594|United States|-6|apartment| +40968|AAAAAAAAIAAKAAAA|773|Broadway 1st|Street|Suite 30|Wilson|Moore County|TN|36971|United States|-6|apartment| +40969|AAAAAAAAJAAKAAAA||Highland |Ave||Riverview||IA|59003|||apartment| +40970|AAAAAAAAKAAKAAAA|903|Meadow |ST|Suite 320|Stringtown|Cecil County|MD|20162|United States|-5|apartment| +40971|AAAAAAAALAAKAAAA|265|2nd |Road|Suite 230|Riverside|Isabella County|MI|49231|United States|-5|condo| +40972|AAAAAAAAMAAKAAAA|139|Cedar |Lane|Suite 420|Cedar Grove|Bourbon County|KY|40411|United States|-6|apartment| +40973|AAAAAAAANAAKAAAA|395|Central Spruce|RD|Suite T|Fairfield|Ramsey County|ND|56192|United States|-6|condo| +40974|AAAAAAAAOAAKAAAA|943|1st Williams|Circle|Suite S|Clinton|Belmont County|OH|48222|United States|-5|condo| +40975|AAAAAAAAPAAKAAAA|560|Willow Railroad|Lane|Suite 440|Woodlawn|Fillmore County|MN|54098|United States|-6|apartment| +40976|AAAAAAAAABAKAAAA|379|3rd |Parkway|Suite S|Woodland|Pamlico County|NC|24854|United States|-5|single family| +40977|AAAAAAAABBAKAAAA|244|Ridge 12th|Avenue|Suite W|New Hope|Marshall County|MS|59431|United States|-6|single family| +40978|AAAAAAAACBAKAAAA|900|Smith |Circle|Suite 180|Macedonia|Jefferson County|MO|61087|United States|-6|condo| +40979|AAAAAAAADBAKAAAA|282|1st |ST|Suite W|Lakeview|Durham County|NC|28579|United States|-5|single family| +40980|AAAAAAAAEBAKAAAA|676|Center |Lane|Suite 360|Bear Creek|Johnson County|MO|63075|United States|-6|apartment| +40981|AAAAAAAAFBAKAAAA|47|West 8th|Drive|Suite 170|Oakland|Jefferson Davis Parish|LA|79843|United States|-6|single family| +40982|AAAAAAAAGBAKAAAA|772|Park Maple|Court|Suite J|White Oak|Stonewall County|TX|76668|United States|-6|condo| +40983|AAAAAAAAHBAKAAAA|456|Meadow |Boulevard|Suite D|Oakland|Coles County|IL|69843|United States|-6|condo| +40984|AAAAAAAAIBAKAAAA|969|Chestnut Elm|Ct.|Suite P|Woodville|Citrus County|FL|34289|United States|-5|condo| +40985|AAAAAAAAJBAKAAAA|830|First |Pkwy|Suite E|Springfield|Thayer County|NE|69303|United States|-7|single family| +40986|AAAAAAAAKBAKAAAA|483|Main |Cir.|Suite 120|Stewart|Sioux County|ND|58041|United States|-6|single family| +40987|AAAAAAAALBAKAAAA|485|12th |Boulevard|Suite N|Bunker Hill|Dade County|GA|30150|United States|-5|apartment| +40988|AAAAAAAAMBAKAAAA|618|10th Dogwood|Ln|Suite I|Mount Pleasant|Hale County|AL|31933|United States|-6|single family| +40989|AAAAAAAANBAKAAAA|258|Chestnut Jefferson|Ln|Suite 200|Glenwood|Lincoln County|MS|53511|United States|-6|single family| +40990|AAAAAAAAOBAKAAAA|338|Second |Circle|Suite 40|Maple Grove|Marshall County|OK|78252|United States|-6|single family| +40991|AAAAAAAAPBAKAAAA|56|Central |Ct.|Suite 320|Fairview|Musselshell County|MT|65709|United States|-7|single family| +40992|AAAAAAAAACAKAAAA|978|11th River|Cir.|Suite 330|Providence|Waseca County|MN|56614|United States|-6|condo| +40993|AAAAAAAABCAKAAAA|2|East |Parkway|Suite 190|Greenville|Bay County|FL|31387|United States|-5|single family| +40994|AAAAAAAACCAKAAAA|262|Elm Lincoln|Court|Suite D|Mount Olive|Asotin County|WA|98059|United States|-8|single family| +40995|AAAAAAAADCAKAAAA|242|Seventh |Circle|Suite 260|Newport|Clark County|OH|41521|United States|-5|condo| +40996|AAAAAAAAECAKAAAA|616|Park First|Lane|Suite 20|Rossville|Sunflower County|MS|52503|United States|-6|apartment| +40997|AAAAAAAAFCAKAAAA|8|Jackson Cedar|Court|Suite 50|Highland Park|Hinds County|MS|56534|United States|-6|condo| +40998|AAAAAAAAGCAKAAAA|481|10th |Blvd|Suite 230|Oakwood|Pulaski County|MO|60169|United States|-6|single family| +40999|AAAAAAAAHCAKAAAA|491|Willow Hickory|Road|Suite 100|Five Forks|Putnam County|NY|12293|United States|-5|apartment| +41000|AAAAAAAAICAKAAAA||Hickory Johnson|Parkway||||NJ||United States|-5|| +41001|AAAAAAAAJCAKAAAA|860|Chestnut 3rd|Court|Suite 0|Stringtown|Cloud County|KS|60162|United States|-6|condo| +41002|AAAAAAAAKCAKAAAA|744|12th Elm|Avenue|Suite N|Greenville|Cuming County|NE|61387|United States|-6|condo| +41003|AAAAAAAALCAKAAAA|450|South |Wy|Suite 320|Summit|Holmes County|MS|50499|United States|-6|apartment| +41004|AAAAAAAAMCAKAAAA|679|College Lake|Ln|Suite K|Hillcrest|Todd County|SD|53003|United States|-7|single family| +41005|AAAAAAAANCAKAAAA||||Suite 80||Boone County||62477||-6|condo| +41006|AAAAAAAAOCAKAAAA|324|Miller |Cir.|Suite J|Springfield|Elkhart County|IN|49303|United States|-5|condo| +41007|AAAAAAAAPCAKAAAA|914|8th Miller|Boulevard|Suite 240|Liberty|Owen County|IN|43451|United States|-5|apartment| +41008|AAAAAAAAADAKAAAA|620|7th 3rd|Road|Suite 180|Lebanon|Norton city|VA|22898|United States|-5|condo| +41009|AAAAAAAABDAKAAAA|430|Railroad |Wy|Suite 320|Pine Grove|Jackson County|IA|54593|United States|-6|single family| +41010|AAAAAAAACDAKAAAA|46|7th |Ct.|Suite D|Highland|Itasca County|MN|59454|United States|-6|condo| +41011|AAAAAAAADDAKAAAA|62|1st |Cir.|Suite 80|Waterloo|Union County|OR|91675|United States|-8|single family| +41012|AAAAAAAAEDAKAAAA|174|Adams Laurel|Ave|Suite H|Arcola|Hardin County|IA|51654|United States|-6|condo| +41013|AAAAAAAAFDAKAAAA|793|3rd |Way|Suite 140|Hamilton|Chesterfield County|VA|22808|United States|-5|apartment| +41014|AAAAAAAAGDAKAAAA||1st |Pkwy||||TN|39843|United States||condo| +41015|AAAAAAAAHDAKAAAA|136|Second Meadow|Lane|Suite 350|Hillcrest|Hill County|MT|63003|United States|-7|condo| +41016|AAAAAAAAIDAKAAAA|920|Cedar Pine|Ln|Suite 120|Friendship|Chatham County|NC|24536|United States|-5|condo| +41017|AAAAAAAAJDAKAAAA|578|Spring |Dr.|Suite 20|Franklin|Lowndes County|MS|59101|United States|-6|apartment| +41018|AAAAAAAAKDAKAAAA|91|1st |Ct.|Suite 210|Elizabeth|Ramsey County|ND|52935|United States|-6|apartment| +41019|AAAAAAAALDAKAAAA|402|Main |RD|Suite W|Carthage|Adams County|OH|41529|United States|-5|apartment| +41020|AAAAAAAAMDAKAAAA|993|7th Fifth|Road|Suite 170|Ellisville|Ida County|IA|56820|United States|-6|single family| +41021|AAAAAAAANDAKAAAA|743|South |Road|Suite A|Brownsville|Rockdale County|GA|39310|United States|-5|apartment| +41022|AAAAAAAAODAKAAAA|686|Cherry 7th|ST|Suite 490|Mountain View|Rice County|KS|64466|United States|-6|apartment| +41023|AAAAAAAAPDAKAAAA|94|Main Walnut|Court|Suite S|Greenfield|Douglas County|CO|85038|United States|-7|single family| +41024|AAAAAAAAAEAKAAAA|762|Cherry Franklin|Way|Suite 180|Oakland|Brule County|SD|59843|United States|-6|single family| +41025|AAAAAAAABEAKAAAA|798|5th |Blvd|Suite D|Centerville|Bee County|TX|70059|United States|-6|condo| +41026|AAAAAAAACEAKAAAA|445|View |Way|Suite G|Parkwood|Lafayette County|FL|31669|United States|-5|apartment| +41027|AAAAAAAADEAKAAAA|769|Hill Poplar|RD|Suite 100|Riverdale|Bristol County|RI|09991|United States|-5|single family| +41028|AAAAAAAAEEAKAAAA|470|Sunset 3rd|Drive||||AL|35804|||apartment| +41029|AAAAAAAAFEAKAAAA|927|Woodland Smith|Lane|Suite Y|Grandview|Sauk County|WI|56475|United States|-6|single family| +41030|AAAAAAAAGEAKAAAA|903|Cedar Pine|Wy|Suite 330|Buena Vista|Tallapoosa County|AL|35752|United States|-6|apartment| +41031|AAAAAAAAHEAKAAAA|194|Meadow |Avenue|Suite 40|Salem|Gentry County|MO|68048|United States|-6|condo| +41032|AAAAAAAAIEAKAAAA|723|Adams |Circle|Suite K|Pomona|Harney County|OR|94153|United States|-8|single family| +41033|AAAAAAAAJEAKAAAA|124|9th |Circle|Suite H|Greenville|Russell County|KY|41387|United States|-5|condo| +41034|AAAAAAAAKEAKAAAA|723|Laurel |Circle|Suite Q|Fairfield|Pine County|MN|56192|United States|-6|condo| +41035|AAAAAAAALEAKAAAA|766|2nd |Boulevard|Suite W|Five Points|Paulding County|OH|46098|United States|-5|condo| +41036|AAAAAAAAMEAKAAAA|847|Pine Birch|Street|Suite R|Clinton|Brooke County|WV|28222|United States|-5|single family| +41037|AAAAAAAANEAKAAAA|181|4th |Court|Suite X|Liberty|Kent County|MI|43451|United States|-5|condo| +41038|AAAAAAAAOEAKAAAA|496|Ash |Lane|Suite M|Greenville|Iosco County|MI|41387|United States|-5|condo| +41039|AAAAAAAAPEAKAAAA|27|Elm Main|Dr.|Suite 490|Union|Boone County|AR|78721|United States|-6|apartment| +41040|AAAAAAAAAFAKAAAA|925|Johnson |RD|Suite 490|Spring Valley|Johnson County|KY|46060|United States|-6|condo| +41041|AAAAAAAABFAKAAAA|491|Railroad |Street|Suite S|Enterprise|Phillips County|MT|61757|United States|-7|single family| +41042|AAAAAAAACFAKAAAA|946|Cherry Forest|Wy|Suite N|Shaw|Macon County|TN|30618|United States|-6|condo| +41043|AAAAAAAADFAKAAAA|171|Lake |Drive|Suite 140|Bunker Hill|Calhoun County|AR|70150|United States|-6|condo| +41044|AAAAAAAAEFAKAAAA|829|Hill |Wy|Suite Y|Lakewood|Wolfe County|KY|48877|United States|-5|single family| +41045|AAAAAAAAFFAKAAAA|288|Mill |Circle|Suite 310|Lakeview|Wood County|WV|28579|United States|-5|single family| +41046|AAAAAAAAGFAKAAAA|392|River |Wy|Suite 230|Wildwood|Sussex County|DE|16871|United States|-5|condo| +41047|AAAAAAAAHFAKAAAA|606|6th 3rd|ST|Suite 260|Franklin|Tooele County|UT|89101|United States|-7|condo| +41048|AAAAAAAAIFAKAAAA|387|Birch 1st|Ct.|Suite U|Brownsville|Raleigh County|WV|29310|United States|-5|single family| +41049|AAAAAAAAJFAKAAAA||Walnut ||Suite 140||||91387|||single family| +41050|AAAAAAAAKFAKAAAA|481|Maple 7th|RD|Suite Q|Bunker Hill|Humphreys County|MS|50150|United States|-6|condo| +41051|AAAAAAAALFAKAAAA|968|Sunset |Lane|Suite 340|Mount Zion|Benzie County|MI|48054|United States|-5|condo| +41052|AAAAAAAAMFAKAAAA|818|Woodland |Wy|Suite L|Farmington|Independence County|AR|79145|United States|-6|apartment| +41053|AAAAAAAANFAKAAAA|34|Jackson Walnut|Drive|Suite V|Oakwood|Emanuel County|GA|30169|United States|-5|condo| +41054|AAAAAAAAOFAKAAAA|424|Franklin |Court|Suite 100|Oakland|Allen County|KS|69843|United States|-6|apartment| +41055|AAAAAAAAPFAKAAAA|776|Third Johnson|Way|Suite Y|Belmont|Bollinger County|MO|60191|United States|-6|apartment| +41056|AAAAAAAAAGAKAAAA|751|Forest Sycamore|Ct.|Suite Q|Brownsville|Stone County|AR|79310|United States|-6|apartment| +41057|AAAAAAAABGAKAAAA|584|Oak 8th|Parkway|Suite U|Union Hill|Vernon Parish|LA|77746|United States|-6|condo| +41058|AAAAAAAACGAKAAAA|634|14th |Boulevard|Suite N|Woodlawn|Woods County|OK|74098|United States|-6|apartment| +41059|AAAAAAAADGAKAAAA|33|Church |Road|Suite M|Waterloo|Franklin County|MO|61675|United States|-6|single family| +41060|AAAAAAAAEGAKAAAA|29|Poplar Maple|Pkwy|Suite 100|Springfield|Nevada County|AR|79303|United States|-6|single family| +41061|AAAAAAAAFGAKAAAA|165|Seventh |Wy|Suite J|Pleasant Hill|Desha County|AR|73604|United States|-6|apartment| +41062|AAAAAAAAGGAKAAAA|777|3rd |Lane|Suite 40|Mount Vernon|Armstrong County|PA|18482|United States|-5|single family| +41063|AAAAAAAAHGAKAAAA|8|Miller Second|Blvd|Suite R|Sulphur Springs|Hamilton County|OH|48354|United States|-5|apartment| +41064|AAAAAAAAIGAKAAAA|77|West Spring|Avenue|Suite X|Lakeside|Walworth County|SD|59532|United States|-7|apartment| +41065|AAAAAAAAJGAKAAAA|242|4th Franklin|Dr.|Suite 270|Pine Grove|Washington County|MS|54593|United States|-6|apartment| +41066|AAAAAAAAKGAKAAAA|559|Valley |Lane|Suite 410|Greenwood|Monterey County|CA|98828|United States|-8|apartment| +41067|AAAAAAAALGAKAAAA|196|Birch |RD|Suite H|Green Acres|Butte County|CA|97683|United States|-8|condo| +41068|AAAAAAAAMGAKAAAA|161|Sunset View|Boulevard|Suite 80|Springfield|Jones County|NC|29303|United States|-5|apartment| +41069|AAAAAAAANGAKAAAA|77|Central 2nd|Boulevard|Suite 160|Concord|Yolo County|CA|94107|United States|-8|condo| +41070|AAAAAAAAOGAKAAAA|847|Williams Wilson|Circle|Suite P|Green Acres|Archuleta County|CO|87683|United States|-7|single family| +41071|AAAAAAAAPGAKAAAA|169|Main Park|Way|Suite 180|Wildwood|Grant County|WA|96871|United States|-8|apartment| +41072|AAAAAAAAAHAKAAAA|853|First |Parkway|Suite 230|Springdale|Adams County|ID|88883|United States|-7|condo| +41073|AAAAAAAABHAKAAAA|783|Miller 8th|ST|Suite Q|Bunker Hill|Pendleton County|WV|20150|United States|-5|apartment| +41074|AAAAAAAACHAKAAAA|411|Lee Ash|Parkway|Suite E|Denmark|Morgan County|GA|35576|United States|-5|single family| +41075|AAAAAAAADHAKAAAA|136|Walnut |ST|Suite W|Stringtown|Cocke County|TN|30162|United States|-5|apartment| +41076|AAAAAAAAEHAKAAAA|440|6th |Dr.|Suite 120|Oakland|Daviess County|IN|49843|United States|-5|apartment| +41077|AAAAAAAAFHAKAAAA|922|River 8th|Cir.|Suite 480|Oak Ridge|Elmore County|AL|38371|United States|-6|single family| +41078|AAAAAAAAGHAKAAAA|159|Elm |Road|Suite 180|Deerfield|Miller County|AR|79840|United States|-6|apartment| +41079|AAAAAAAAHHAKAAAA|82|Cherry |Ln|Suite M|Georgetown|Jerome County|ID|87057|United States|-7|single family| +41080|AAAAAAAAIHAKAAAA|544|3rd |Way|Suite I|Highland Park|Marshall County|OK|76534|United States|-6|apartment| +41081|AAAAAAAAJHAKAAAA|328|2nd |Wy|Suite V|Hamilton|Scott County|IL|62808|United States|-6|single family| +41082|AAAAAAAAKHAKAAAA|477|Mill |Lane|Suite 80|Antioch|Delaware County|IN|48605|United States|-5|apartment| +41083|AAAAAAAALHAKAAAA|221|Cherry 6th|RD|Suite W|Derby|Clinton County|MI|47702|United States|-5|apartment| +41084|AAAAAAAAMHAKAAAA|798|Main |Court|Suite V|Ashland|Greene County|PA|14244|United States|-5|apartment| +41085|AAAAAAAANHAKAAAA||Laurel ||Suite 150|||NC|||-5|| +41086|AAAAAAAAOHAKAAAA|682|Church |Ln|Suite 350|Lakeview|Delaware County|IA|58579|United States|-6|single family| +41087|AAAAAAAAPHAKAAAA|968|Elm 2nd|Street|Suite 190|Mount Olive|Baltimore County|MD|28059|United States|-5|condo| +41088|AAAAAAAAAIAKAAAA|49|3rd |Avenue|Suite 90|Sunnyside|Banks County|GA|31952|United States|-5|condo| +41089|AAAAAAAABIAKAAAA|628|Park Maple|ST|Suite 70|Wilson|Thomas County|GA|36971|United States|-5|apartment| +41090|AAAAAAAACIAKAAAA|388|Adams Broadway|Court|Suite P|Plainview|Sioux County|IA|53683|United States|-6|apartment| +41091|AAAAAAAADIAKAAAA|428|Woodland Jackson|Avenue|Suite Q|Harmony|Jones County|GA|35804|United States|-5|single family| +41092|AAAAAAAAEIAKAAAA|773|Hillcrest Smith|Cir.|Suite 0|Rockwood|Shelby County|IN|41545|United States|-5|condo| +41093|AAAAAAAAFIAKAAAA|900|Main Oak|Pkwy|Suite I|Sulphur Springs|Jefferson County|AL|38354|United States|-6|condo| +41094|AAAAAAAAGIAKAAAA|964|Third Johnson|Pkwy|Suite 130|Pleasant Grove|Perkins County|SD|54136|United States|-7|condo| +41095|AAAAAAAAHIAKAAAA|288|Eigth |Circle|Suite F|Plainview|Fayette County|GA|33683|United States|-5|condo| +41096|AAAAAAAAIIAKAAAA|830|13th |Street|Suite 270|Riverdale|Spencer County|IN|49391|United States|-5|apartment| +41097|AAAAAAAAJIAKAAAA|650|Ninth River|Cir.|Suite C|Woodlawn|Kenosha County|WI|54098|United States|-6|apartment| +41098|AAAAAAAAKIAKAAAA|362|Jackson Fourth|Boulevard|Suite L|Fairfield|Door County|WI|56192|United States|-6|apartment| +41099|AAAAAAAALIAKAAAA|948|1st |Street|Suite J|Salem|Santa Rosa County|FL|38048|United States|-5|single family| +41100|AAAAAAAAMIAKAAAA|823|Lake 8th|Dr.|Suite 170|Spring Valley|Westchester County|NY|16060|United States|-5|apartment| +41101|AAAAAAAANIAKAAAA|167|5th |Avenue|Suite 260|Liberty|Wood County|WV|23451|United States|-5|condo| +41102|AAAAAAAAOIAKAAAA|505|Sunset Park|Blvd|Suite 330|Wilson|Meigs County|OH|46971|United States|-5|single family| +41103|AAAAAAAAPIAKAAAA|103|Chestnut Oak|Circle|Suite Q|Green Acres|Keokuk County|IA|57683|United States|-6|apartment| +41104|AAAAAAAAAJAKAAAA|283|Washington Central|Wy|Suite 190|Salem|Barry County|MI|48048|United States|-5|condo| +41105|AAAAAAAABJAKAAAA|199|Railroad |Court|Suite 260|Plainview|Cedar County|IA|53683|United States|-6|condo| +41106|AAAAAAAACJAKAAAA|265|3rd |Dr.|Suite 480|Oak Hill|Washington County|ID|87838|United States|-7|single family| +41107|AAAAAAAADJAKAAAA|954|2nd Birch|ST|Suite N|Sulphur Springs|Wilson County|NC|28354|United States|-5|single family| +41108|AAAAAAAAEJAKAAAA|590|Willow Sunset|RD|Suite W|Pleasant Hill|Harlan County|KY|43604|United States|-6|apartment| +41109|AAAAAAAAFJAKAAAA|618|Poplar View|Drive|Suite W|Wildwood|Washington County|GA|36871|United States|-5|condo| +41110|AAAAAAAAGJAKAAAA|690|Hillcrest |Boulevard|Suite J|Valley View|Sioux County|IA|55124|United States|-6|single family| +41111|AAAAAAAAHJAKAAAA|336|7th |Pkwy|Suite W|Highland Park|La Salle County|IL|66534|United States|-6|apartment| +41112|AAAAAAAAIJAKAAAA|164|Dogwood |Way|Suite 130|Stringtown|Linn County|KS|60162|United States|-6|single family| +41113|AAAAAAAAJJAKAAAA|866|9th 3rd|Ln|Suite J|Providence|Culberson County|TX|76614|United States|-6|condo| +41114|AAAAAAAAKJAKAAAA|307|Oak Lake|Court|Suite 360|Springdale|Clayton County|GA|38883|United States|-5|single family| +41115|AAAAAAAALJAKAAAA|107|6th Center|Street|Suite P|Bennett|Bosque County|TX|71715|United States|-6|single family| +41116|AAAAAAAAMJAKAAAA|792|Lincoln |Drive|Suite Y|Newtown|Noble County|OH|41749|United States|-5|condo| +41117|AAAAAAAANJAKAAAA|829|Church Forest|Pkwy|Suite 320|Franklin|Wilkes County|NC|29101|United States|-5|apartment| +41118|AAAAAAAAOJAKAAAA|373|7th Park|Ln|Suite F|Mountain View|Black Hawk County|IA|54466|United States|-6|apartment| +41119|AAAAAAAAPJAKAAAA|252|Park Smith|Road|Suite 70|Sunnyside|Reno County|KS|61952|United States|-6|apartment| +41120|AAAAAAAAAKAKAAAA|883|Third 7th|Street|Suite S|Colfax|Adams County|PA|12565|United States|-5|apartment| +41121|AAAAAAAABKAKAAAA|855|1st |Court|Suite N|Ashland|Orange County|FL|34244|United States|-5|condo| +41122|AAAAAAAACKAKAAAA|561|Eigth |Ave|Suite 480|Redland|Foster County|ND|56343|United States|-6|single family| +41123|AAAAAAAADKAKAAAA|891|Poplar |Parkway|Suite I|Edgewood|Potter County|TX|70069|United States|-6|apartment| +41124|AAAAAAAAEKAKAAAA|318|Cedar |Circle|Suite 300|Walnut Grove|Bonneville County|ID|87752|United States|-7|apartment| +41125|AAAAAAAAFKAKAAAA|803|Park |Dr.|Suite 490|Lakeside|Pawnee County|OK|79532|United States|-6|condo| +41126|AAAAAAAAGKAKAAAA|254|West 2nd|Ct.|Suite 230|White Oak|Fredericksburg city|VA|26668|United States|-5|apartment| +41127|AAAAAAAAHKAKAAAA|498|5th Forest|Dr.|Suite 30|Newtown|Larue County|KY|41749|United States|-5|single family| +41128|AAAAAAAAIKAKAAAA|454|Fifth |ST|Suite C|Friendship|Gem County|ID|84536|United States|-7|single family| +41129|AAAAAAAAJKAKAAAA|108|Fourth |Dr.|Suite C|Lakeview|Lewis and Clark County|MT|68579|United States|-7|single family| +41130|AAAAAAAAKKAKAAAA|707|Third |Wy|Suite R|Hopewell|Pitt County|NC|20587|United States|-5|single family| +41131|AAAAAAAALKAKAAAA|903|Church |Ct.|Suite 210|Shady Grove|Wadena County|MN|52812|United States|-6|condo| +41132|AAAAAAAAMKAKAAAA|85|1st 6th|Lane|Suite R|Greenwood|Dent County|MO|68828|United States|-6|apartment| +41133|AAAAAAAANKAKAAAA|282|Maple West|Cir.|Suite 400|Florence|Pasco County|FL|33394|United States|-5|apartment| +41134|AAAAAAAAOKAKAAAA|25|Cherry 6th|Avenue|Suite I|Concord|Wapello County|IA|54107|United States|-6|condo| +41135|AAAAAAAAPKAKAAAA|646|Fourth Spruce|Pkwy|Suite 80|Richfield|Lincoln County|TN|36196|United States|-6|apartment| +41136|AAAAAAAAALAKAAAA|||||||GA||United States||| +41137|AAAAAAAABLAKAAAA|362|West |Ln|Suite 290|Hillcrest|Taylor County|WV|23003|United States|-5|apartment| +41138|AAAAAAAACLAKAAAA|310|Jackson |Blvd|Suite 480|Antioch|Franklin County|IA|58605|United States|-6|condo| +41139|AAAAAAAADLAKAAAA|36|4th Ridge|Circle|Suite 440|Newtown|Pondera County|MT|61749|United States|-7|apartment| +41140|AAAAAAAAELAKAAAA|205|North |Way|Suite Q|Union Hill|Madison County|GA|37746|United States|-5|apartment| +41141|AAAAAAAAFLAKAAAA|133|8th |Way|Suite 100|Burns|Power County|ID|85272|United States|-7|condo| +41142|AAAAAAAAGLAKAAAA|154|Church Fifth|Lane|Suite 30|Edgewood|Buchanan County|MO|60069|United States|-6|single family| +41143|AAAAAAAAHLAKAAAA|257|Mill Chestnut|Wy|Suite 430|Wildwood|Greene County|TN|36871|United States|-5|apartment| +41144|AAAAAAAAILAKAAAA|208|First |Pkwy|Suite Q|Proctor|Flathead County|MT|68140|United States|-7|apartment| +41145|AAAAAAAAJLAKAAAA|325|Elm |Lane|Suite 280|Hopewell|Murray County|OK|70587|United States|-6|condo| +41146|AAAAAAAAKLAKAAAA|584|6th |Boulevard|Suite 140|Highland|Clarendon County|SC|29454|United States|-5|apartment| +41147|AAAAAAAALLAKAAAA|108|Center |Pkwy|Suite Y|Liberty|Hardee County|FL|33451|United States|-5|single family| +41148|AAAAAAAAMLAKAAAA|539|Lincoln |Way|Suite H|Belleview|White County|IN|40492|United States|-5|condo| +41149|AAAAAAAANLAKAAAA|620|Washington Elm|Ave|Suite 280|Shady Grove|Van Buren County|AR|72812|United States|-6|apartment| +41150|AAAAAAAAOLAKAAAA|33|8th |Circle|Suite 370|Wolf Creek|Logan County|AR|72455|United States|-6|single family| +41151|AAAAAAAAPLAKAAAA|712|West Elm|Avenue|Suite O|Oakdale|Centre County|PA|19584|United States|-5|apartment| +41152|AAAAAAAAAMAKAAAA|496|Highland Main|Ave|Suite L|Pine Grove|Fredericksburg city|VA|24593|United States|-5|apartment| +41153|AAAAAAAABMAKAAAA|681|Seventh |Court|Suite 290|Farmington|Caroline County|VA|29145|United States|-5|single family| +41154|AAAAAAAACMAKAAAA|489|Ash Cedar|Lane|Suite 20|Woodlawn|Marshall County|AL|34098|United States|-6|single family| +41155|AAAAAAAADMAKAAAA|674|South Ridge|Drive|Suite D|Red Hill|Fulton County|NY|14338|United States|-5|condo| +41156|AAAAAAAAEMAKAAAA|248|West |Ln|Suite U|Stringtown|Crittenden County|KY|40162|United States|-6|apartment| +41157|AAAAAAAAFMAKAAAA|67|Church |Pkwy|Suite 190|Salem|Sierra County|CA|98048|United States|-8|condo| +41158|AAAAAAAAGMAKAAAA|540|Seventh College|Blvd|Suite R|Glenwood|Franklin County|KY|43511|United States|-6|condo| +41159|AAAAAAAAHMAKAAAA|221|8th |Way|Suite 330|Jackson|Bristol County|RI|09583|United States|-5|apartment| +41160|AAAAAAAAIMAKAAAA|136|Third Jackson|Blvd|Suite Q|Pleasant Hill|Sutter County|CA|93604|United States|-8|single family| +41161|AAAAAAAAJMAKAAAA|161|3rd |Boulevard|Suite 270|Antioch|Morgan County|MO|68605|United States|-6|apartment| +41162|AAAAAAAAKMAKAAAA|152|Madison |RD|Suite I|Lakewood|Lincoln County|ME|09477|United States|-5|condo| +41163|AAAAAAAALMAKAAAA|875|8th Willow|Blvd|Suite 240|Glenwood|Fairbanks North Star Borough|AK|93511|United States|-9|condo| +41164|AAAAAAAAMMAKAAAA|143|Wilson Main|Circle|Suite G|Five Points|Dakota County|MN|56098|United States|-6|apartment| +41165|AAAAAAAANMAKAAAA|677|Sixth Lake|Road|Suite 240|Bunker Hill|Maverick County|TX|70150|United States|-6|apartment| +41166|AAAAAAAAOMAKAAAA|732|Locust 13th|Court|Suite E|Pleasant Valley|Val Verde County|TX|72477|United States|-6|single family| +41167|AAAAAAAAPMAKAAAA|370|Maple 2nd|Dr.|Suite 200|Marion|Itawamba County|MS|50399|United States|-6|condo| +41168|AAAAAAAAANAKAAAA|103|Fifth Center|Pkwy|Suite T|Mount Pleasant|La Plata County|CO|81933|United States|-7|single family| +41169|AAAAAAAABNAKAAAA|649|Park |Ln|Suite K|Guilford|Green County|WI|54408|United States|-6|condo| +41170|AAAAAAAACNAKAAAA|942|Hickory |Ln|Suite V|Providence|Evangeline Parish|LA|76614|United States|-6|single family| +41171|AAAAAAAADNAKAAAA|14|Poplar |Street|Suite G|Barnes|Lamar County|MS|53788|United States|-6|condo| +41172|AAAAAAAAENAKAAAA|133|Main |Street|Suite A|Oak Grove|Reynolds County|MO|68370|United States|-6|apartment| +41173|AAAAAAAAFNAKAAAA|715|Central Ridge|Avenue|Suite 230|Springfield|Huntington County|IN|49303|United States|-5|condo| +41174|AAAAAAAAGNAKAAAA|93|Walnut Valley|Way|Suite S|Woodland|Franklin County|ID|84854|United States|-7|apartment| +41175|AAAAAAAAHNAKAAAA|259|Valley Cherry|Way|Suite J|Mount Zion|Nevada County|AR|78054|United States|-6|condo| +41176|AAAAAAAAINAKAAAA|625|Lee |Circle|Suite 0|Shady Grove|Saluda County|SC|22812|United States|-5|condo| +41177|AAAAAAAAJNAKAAAA|8|Spruce |Wy|Suite H|Oak Hill|Dakota County|NE|67838|United States|-6|condo| +41178|AAAAAAAAKNAKAAAA|697|15th |Parkway|Suite 330|Macedonia|Douglas County|MN|51087|United States|-6|condo| +41179|AAAAAAAALNAKAAAA|314|Oak |Boulevard|Suite D|Mount Pleasant|Leelanau County|MI|41933|United States|-5|apartment| +41180|AAAAAAAAMNAKAAAA|556|10th Elm|Ln|Suite W|Mount Vernon|Mercer County|PA|18482|United States|-5|condo| +41181|AAAAAAAANNAKAAAA|167|Jefferson River|Cir.|Suite 30|Oak Ridge|Coryell County|TX|78371|United States|-6|apartment| +41182|AAAAAAAAONAKAAAA|763|1st |Ave|Suite 480|Deerfield|Ravalli County|MT|69840|United States|-7|single family| +41183|AAAAAAAAPNAKAAAA|822|South |Pkwy|Suite 410|Jamestown|Wilkinson County|GA|36867|United States|-5|apartment| +41184|AAAAAAAAAOAKAAAA|237|Willow |Street|Suite 380|Jackson|Hamilton County|NY|19583|United States|-5|single family| +41185|AAAAAAAABOAKAAAA|120|Main 5th|Ct.|Suite 330|Harmony|Grant County|MN|55804|United States|-6|condo| +41186|AAAAAAAACOAKAAAA||5th Washington||||Tripp County||||-7|| +41187|AAAAAAAADOAKAAAA|436|8th River|ST|Suite J|Riverside|Orange County|VA|29231|United States|-5|single family| +41188|AAAAAAAAEOAKAAAA|460|6th |Blvd|Suite 450|Springfield|Greene County|NY|19303|United States|-5|single family| +41189|AAAAAAAAFOAKAAAA|423|Forest Seventh|Court|Suite 480|Belfast|Wheeler County|NE|60125|United States|-7|single family| +41190|AAAAAAAAGOAKAAAA|424|Highland |Blvd|Suite 210|Chestnut Ridge|Los Angeles County|CA|97334|United States|-8|condo| +41191|AAAAAAAAHOAKAAAA|290|Highland |Drive|Suite L|Mount Zion|Graham County|KS|68054|United States|-6|single family| +41192|AAAAAAAAIOAKAAAA|899|Center |||Greenville|||||-8|single family| +41193|AAAAAAAAJOAKAAAA|425|Poplar Maple|Wy|Suite F|Newport|Burlington County|NJ|02121|United States|-5|single family| +41194|AAAAAAAAKOAKAAAA|98|Maple Laurel|Wy|Suite X|Marion|Sierra County|CA|90399|United States|-8|apartment| +41195|AAAAAAAALOAKAAAA|632|14th Walnut|Avenue|Suite M|Franklin|Republic County|KS|69101|United States|-6|single family| +41196|AAAAAAAAMOAKAAAA|307|Maple Willow|Blvd|Suite 80|Oak Hill|Emery County|UT|87838|United States|-7|apartment| +41197|AAAAAAAANOAKAAAA|283|View 14th|Dr.|Suite 270|Gravel Hill|Benton County|IA|51944|United States|-6|condo| +41198|AAAAAAAAOOAKAAAA|673|River 6th|Cir.|Suite 120|Berea|Chatham County|GA|33804|United States|-5|single family| +41199|AAAAAAAAPOAKAAAA|595|Main 1st|Way|Suite 380|Lakeside|Fayette County|KY|49532|United States|-6|apartment| +41200|AAAAAAAAAPAKAAAA|414|Main |Way|Suite D|Walnut Grove|Nottoway County|VA|27752|United States|-5|single family| +41201|AAAAAAAABPAKAAAA|236|Cherry |ST|Suite 250|Valley View|San Bernardino County|CA|95124|United States|-8|condo| +41202|AAAAAAAACPAKAAAA|11|Locust Oak|RD|Suite X|Greenfield|Calhoun County|SC|25038|United States|-5|condo| +41203|AAAAAAAADPAKAAAA|62|Oak Lee|Circle|Suite M|Green Acres|Lewis County|MO|67683|United States|-6|apartment| +41204|AAAAAAAAEPAKAAAA|571|14th |Ave|Suite 410|Kingston|Jones County|MS|54975|United States|-6|single family| +41205|AAAAAAAAFPAKAAAA|30|Hickory Seventh|Street|Suite B|Washington Heights|Horry County|SC|28167|United States|-5|condo| +41206|AAAAAAAAGPAKAAAA|545|Ridge |Drive|Suite 80|Hamilton|Prairie County|AR|72808|United States|-6|single family| +41207|AAAAAAAAHPAKAAAA|749|4th |Dr.|Suite 170|Jerome|Fayette County|GA|39920|United States|-5|condo| +41208|AAAAAAAAIPAKAAAA|779|Wilson |Pkwy|Suite 340|Oak Hill|Carroll County|IL|67838|United States|-6|condo| +41209|AAAAAAAAJPAKAAAA|267|Washington 6th|Street|Suite N|Kingston|Marion County|OH|44975|United States|-5|single family| +41210|AAAAAAAAKPAKAAAA|833|Park Johnson|Lane|Suite T|Union|Bates County|MO|68721|United States|-6|condo| +41211|AAAAAAAALPAKAAAA|168|Maple 13th|Way|Suite G|Cedar Grove|Caledonia County|VT|01011|United States|-5|single family| +41212|AAAAAAAAMPAKAAAA|243|||Suite V|Frankfort|||39681||-5|| +41213|AAAAAAAANPAKAAAA|373|Church Johnson|Ct.|Suite 310|Lakewood|Cowley County|KS|68877|United States|-6|condo| +41214|AAAAAAAAOPAKAAAA|478|Maple |Wy|Suite H|Lakeview|Broward County|FL|38579|United States|-5|apartment| +41215|AAAAAAAAPPAKAAAA|106|Lee Oak|Ave|Suite 490|Riverdale|Warren County|VA|29391|United States|-5|apartment| +41216|AAAAAAAAAABKAAAA|208|River |Ln|Suite S|Lakeview|Douglas County|NV|88579|United States|-8|single family| +41217|AAAAAAAABABKAAAA|938|Miller |Dr.|Suite 430|Clifton|Schuylkill County|PA|18014|United States|-5|apartment| +41218|AAAAAAAACABKAAAA|426|Second Birch|Way|Suite R|Bunker Hill|Thomas County|KS|60150|United States|-6|condo| +41219|AAAAAAAADABKAAAA|527||Drive|Suite 370|Lakeside|Iron County|MI|49532|United States|-5|| +41220|AAAAAAAAEABKAAAA|88|Maple 7th|ST|Suite U|Hardy|Deaf Smith County|TX|75354|United States|-6|single family| +41221|AAAAAAAAFABKAAAA|929|Madison 5th|Pkwy|Suite 480|Oakland|Caldwell County|NC|29843|United States|-5|single family| +41222|AAAAAAAAGABKAAAA|360|10th |Dr.|Suite L|Proctor|Union County|IL|68140|United States|-6|apartment| +41223|AAAAAAAAHABKAAAA|941|15th Davis|Road|Suite H|Jamestown|Scurry County|TX|76867|United States|-6|apartment| +41224|AAAAAAAAIABKAAAA|475|Lake |ST|Suite 360|Greenfield|Grant County|OR|95038|United States|-8|single family| +41225|AAAAAAAAJABKAAAA|272|12th |ST|Suite 70|Pleasant Grove|Nodaway County|MO|64136|United States|-6|apartment| +41226|AAAAAAAAKABKAAAA|359|15th Green|Wy|Suite 300|Enterprise|Greene County|OH|41757|United States|-5|apartment| +41227|AAAAAAAALABKAAAA|177|Oak Meadow|Parkway|Suite C|Greenfield|West Feliciana Parish|LA|75038|United States|-6|single family| +41228|AAAAAAAAMABKAAAA|600|Hill Thirteenth|Dr.|Suite B|Salem|Erie County|OH|48048|United States|-5|condo| +41229|AAAAAAAANABKAAAA|925|Third |Court|Suite 100|Oak Ridge|Moore County|TX|78371|United States|-6|condo| +41230|AAAAAAAAOABKAAAA|13|Highland |Ct.|Suite 210|Liberty|Craig County|OK|73451|United States|-6|apartment| +41231|AAAAAAAAPABKAAAA|972|Smith 12th|Ln|Suite W|Newburg|Cobb County|GA|31150|United States|-5|single family| +41232|AAAAAAAAABBKAAAA|548|Fourth Sixth|Ave|Suite 100|Lakeview|Dawson County|NE|68579|United States|-6|condo| +41233|AAAAAAAABBBKAAAA|267|View |Ln|Suite 470|Mount Olive|Uvalde County|TX|78059|United States|-6|single family| +41234|AAAAAAAACBBKAAAA|499|Adams |Ln|Suite 100|Marion|Clinton County|KY|40399|United States|-6|apartment| +41235|AAAAAAAADBBKAAAA|187|5th Woodland|Ct.|Suite 40|Antioch|Anderson County|SC|28605|United States|-5|condo| +41236|AAAAAAAAEBBKAAAA|406|Green |Ln|Suite 310|Stafford|Musselshell County|MT|64980|United States|-7|apartment| +41237|AAAAAAAAFBBKAAAA|690|Lakeview |Lane|Suite H|Woodlawn|Ashe County|NC|24098|United States|-5|condo| +41238|AAAAAAAAGBBKAAAA|713|Sunset |Avenue|Suite 160|Brownsville|Washington County|IL|69310|United States|-6|apartment| +41239|AAAAAAAAHBBKAAAA|106|Eigth 1st|Ct.|Suite 220|Highland Park|Adair County|IA|56534|United States|-6|condo| +41240|AAAAAAAAIBBKAAAA|911|North Chestnut|Ave|Suite D|Mount Olive|Smith County|MS|58059|United States|-6|single family| +41241|AAAAAAAAJBBKAAAA|522|North |Court|Suite L|Omega|Powder River County|MT|66364|United States|-7|apartment| +41242|AAAAAAAAKBBKAAAA|||Blvd|||Jackson County|||||apartment| +41243|AAAAAAAALBBKAAAA|82|Main 2nd|Dr.|Suite 350|Unionville|Delaware County|IN|41711|United States|-5|single family| +41244|AAAAAAAAMBBKAAAA|567|Hickory 15th|Boulevard|Suite M|Farmington|Grayson County|KY|49145|United States|-6|condo| +41245|AAAAAAAANBBKAAAA|683|Jackson Spring|Dr.|Suite 310|Glenwood|Island County|WA|93511|United States|-8|apartment| +41246|AAAAAAAAOBBKAAAA|296|Ridge River|Wy|Suite A|Tyrone|Orleans Parish|LA|71201|United States|-6|condo| +41247|AAAAAAAAPBBKAAAA|944|6th |Road|Suite 140|Cedar Grove|Karnes County|TX|70411|United States|-6|condo| +41248|AAAAAAAAACBKAAAA|654|Ninth |Pkwy|Suite 80|Mount Vernon|Bell County|KY|48482|United States|-6|single family| +41249|AAAAAAAABCBKAAAA|191|Adams |Dr.|Suite 10|Newtown|Somerset County|MD|21749|United States|-5|apartment| +41250|AAAAAAAACCBKAAAA|502|Lake |Boulevard|Suite M|Fairfield|Otsego County|MI|46192|United States|-5|condo| +41251|AAAAAAAADCBKAAAA|902|Broadway |Avenue|Suite G|Shady Grove|Bedford County|VA|22812|United States|-5|single family| +41252|AAAAAAAAECBKAAAA|171|Third Main|RD|Suite 100|Woodlawn|Hopkins County|TX|74098|United States|-6|condo| +41253|AAAAAAAAFCBKAAAA|58|Washington |Blvd|Suite 170|Lakeville|Lincoln County|NC|28811|United States|-5|single family| +41254|AAAAAAAAGCBKAAAA|740|Spruce Spruce|RD|Suite 30|Friendship|Habersham County|GA|34536|United States|-5|condo| +41255|AAAAAAAAHCBKAAAA|196|West Pine|Dr.|Suite 170|Riverside|Asotin County|WA|99231|United States|-8|single family| +41256|AAAAAAAAICBKAAAA|454|Ridge Meadow|Court|Suite 310|Hamilton|Wahkiakum County|WA|92808|United States|-8|apartment| +41257|AAAAAAAAJCBKAAAA|745|8th Willow|Parkway|Suite 220|Woodlawn|Charlottesville city|VA|24098|United States|-5|single family| +41258|AAAAAAAAKCBKAAAA|326|Madison |Boulevard|Suite E|Bethel|La Salle Parish|LA|75281|United States|-6|apartment| +41259|AAAAAAAALCBKAAAA|744|Pine Park|Ct.|Suite 120|Antioch|Gates County|NC|28605|United States|-5|condo| +41260|AAAAAAAAMCBKAAAA|290|Park |Dr.|Suite 360|Pleasant Hill|Winona County|MN|53604|United States|-6|single family| +41261|AAAAAAAANCBKAAAA|744|1st |Court|Suite 310|Greenwood|Jefferson County|MO|68828|United States|-6|condo| +41262|AAAAAAAAOCBKAAAA|139|2nd |Dr.|Suite V|Pleasant Hill|Butte County|CA|93604|United States|-8|apartment| +41263|AAAAAAAAPCBKAAAA|983|Franklin Johnson|Road|Suite 170|Wilson|Boyd County|KY|46971|United States|-6|apartment| +41264|AAAAAAAAADBKAAAA|360|Jefferson Jefferson|Ln|Suite 60|Marion|Tangipahoa Parish|LA|70399|United States|-6|single family| +41265|AAAAAAAABDBKAAAA|713|Meadow |Drive|Suite A|Oak Hill|Minidoka County|ID|87838|United States|-7|single family| +41266|AAAAAAAACDBKAAAA|404|Third |Court|Suite 80|Marion|Wilson County|TN|30399|United States|-5|condo| +41267|AAAAAAAADDBKAAAA|863|Lincoln |Pkwy|Suite 250|Oakwood|Charlton County|GA|30169|United States|-5|condo| +41268|AAAAAAAAEDBKAAAA|59|River |Ave|Suite V|Kingston|Chilton County|AL|34975|United States|-6|condo| +41269|AAAAAAAAFDBKAAAA|109|Laurel |Avenue|Suite I|Newtown|Webster County|WV|21749|United States|-5|condo| +41270|AAAAAAAAGDBKAAAA|504|Second |Ave|Suite 350|Lakeview|Plymouth County|MA|09179|United States|-5|condo| +41271|AAAAAAAAHDBKAAAA|426|Main Smith|Blvd|Suite A|Friendship|Carter County|OK|74536|United States|-6|apartment| +41272|AAAAAAAAIDBKAAAA|117|Hill |Avenue|Suite O|Pleasant Valley|Yalobusha County|MS|52477|United States|-6|single family| +41273|AAAAAAAAJDBKAAAA|685|6th |Blvd|Suite 450|Bridgeport|Lynchburg city|VA|25817|United States|-5|single family| +41274|AAAAAAAAKDBKAAAA|92|Forest |Dr.|Suite 460|Ashland|Haywood County|TN|34244|United States|-5|condo| +41275|AAAAAAAALDBKAAAA|793|Lincoln |Street|Suite 240|Woodland|Natchitoches Parish|LA|74854|United States|-6|condo| +41276|AAAAAAAAMDBKAAAA|961|Walnut |Road|Suite I|Red Hill|Marinette County|WI|54338|United States|-6|single family| +41277|AAAAAAAANDBKAAAA|464|Park Hill|Road|Suite G|Georgetown|Robeson County|NC|27057|United States|-5|apartment| +41278|AAAAAAAAODBKAAAA|512|Hill |Parkway|Suite 390|Greenwood|Newton County|MS|58828|United States|-6|condo| +41279|AAAAAAAAPDBKAAAA|895|Dogwood |Pkwy|Suite W|Pleasant Hill|Dickey County|ND|53604|United States|-6|condo| +41280|AAAAAAAAAEBKAAAA|80|Chestnut |Blvd|Suite 380|Greenfield|Wayne County|OH|45038|United States|-5|condo| +41281|AAAAAAAABEBKAAAA|550|8th Hickory|Ave|Suite 380|Oakwood|Lapeer County|MI|40169|United States|-5|single family| +41282|AAAAAAAACEBKAAAA|347|Walnut |Ln|Suite I|Five Forks|Holmes County|FL|32293|United States|-5|single family| +41283|AAAAAAAADEBKAAAA|325|Park 6th|RD|Suite X|Mount Olive|Woodbury County|IA|58059|United States|-6|condo| +41284|AAAAAAAAEEBKAAAA|746|Washington |Way|Suite E|Ridgeville|Houghton County|MI|49306|United States|-5|condo| +41285|AAAAAAAAFEBKAAAA|879|Smith Oak|RD|Suite 350|Shiloh|Elbert County|GA|39275|United States|-5|single family| +41286|AAAAAAAAGEBKAAAA|394|North Locust|Cir.|Suite 60|Marion|Warren County|NC|20399|United States|-5|apartment| +41287|AAAAAAAAHEBKAAAA|531|Park Elm|Pkwy|Suite P|Pleasant Hill|Kent County|MD|23604|United States|-5|condo| +41288|AAAAAAAAIEBKAAAA|636|6th Walnut|Wy|Suite M|Oak Hill|Latimer County|OK|77838|United States|-6|apartment| +41289|AAAAAAAAJEBKAAAA|734|Hill |Avenue|Suite 240|Five Points|Chickasaw County|IA|56098|United States|-6|single family| +41290|AAAAAAAAKEBKAAAA|520|College Madison|Parkway|Suite 340|Denmark|Fulton County|IL|65576|United States|-6|apartment| +41291|AAAAAAAALEBKAAAA|666|Second 4th|Dr.|Suite S|Greenwood|Mellette County|SD|58828|United States|-7|single family| +41292|AAAAAAAAMEBKAAAA|24|Maple Washington|Ct.|Suite D|Longwood|Glenn County|CA|97021|United States|-8|apartment| +41293|AAAAAAAANEBKAAAA|465|Lake |Road|Suite 40|Five Forks|Guadalupe County|NM|82293|United States|-7|apartment| +41294|AAAAAAAAOEBKAAAA|122|Sunset Hill|Avenue|Suite U|Franklin|Henry County|IA|59101|United States|-6|apartment| +41295|AAAAAAAAPEBKAAAA|724|14th |Lane|Suite 30|Green Acres|Atlantic County|NJ|08283|United States|-5|apartment| +41296|AAAAAAAAAFBKAAAA|200|Forest Davis|Blvd|Suite F|Highland|Monroe County|GA|39454|United States|-5|single family| +41297|AAAAAAAABFBKAAAA|774|6th Ash|Way|Suite 80|Oakwood|Oliver County|ND|50169|United States|-6|single family| +41298|AAAAAAAACFBKAAAA|926|Adams Chestnut|Boulevard|Suite Y|Edgewood|Walworth County|WI|50069|United States|-6|condo| +41299|AAAAAAAADFBKAAAA|496|Hill |Way|Suite 490|Pleasant Hill|Ray County|MO|63604|United States|-6|apartment| +41300|AAAAAAAAEFBKAAAA|236|East |Wy|Suite 100|Riverdale|Abbeville County|SC|29391|United States|-5|single family| +41301|AAAAAAAAFFBKAAAA|548|Broadway |Pkwy|Suite S|Mount Zion|Sullivan County|MO|68054|United States|-6|apartment| +41302|AAAAAAAAGFBKAAAA|632|West Williams|ST|Suite T|Glenwood|Ritchie County|WV|23511|United States|-5|condo| +41303|AAAAAAAAHFBKAAAA|620|Lakeview |Blvd|Suite 350|Bridgeport|Pickens County|AL|35817|United States|-6|condo| +41304|AAAAAAAAIFBKAAAA|293|3rd |Ln|Suite 130|Mount Olive|Rabun County|GA|38059|United States|-5|condo| +41305|AAAAAAAAJFBKAAAA|226|Pine Chestnut|RD|Suite K|Franklin|Cochise County|AZ|89101|United States|-7|condo| +41306|AAAAAAAAKFBKAAAA||River ||||Kern County|CA||||single family| +41307|AAAAAAAALFBKAAAA|993|Highland |Ct.|Suite R|Highland Park|Sanders County|MT|66534|United States|-7|apartment| +41308|AAAAAAAAMFBKAAAA|822|Hill Meadow|Boulevard|Suite 180|Spring Hill|Suwannee County|FL|36787|United States|-5|condo| +41309|AAAAAAAANFBKAAAA|507|Washington |Blvd|Suite X|Oakwood|Howard County|IN|40169|United States|-5|condo| +41310|AAAAAAAAOFBKAAAA|714|Second |Road|Suite 90|Plainview|Orange County|NY|13683|United States|-5|single family| +41311|AAAAAAAAPFBKAAAA|940|South |Drive|Suite C|Belmont|Emmons County|ND|50191|United States|-6|condo| +41312|AAAAAAAAAGBKAAAA|273|3rd View|RD|Suite W|Salem|Cooper County|MO|68048|United States|-6|apartment| +41313|AAAAAAAABGBKAAAA|350|1st 9th|Cir.|Suite X|Liberty|Brown County|WI|53451|United States|-6|apartment| +41314|AAAAAAAACGBKAAAA|116|Lee Second|Parkway|Suite 140|Newtown|Texas County|OK|71749|United States|-6|single family| +41315|AAAAAAAADGBKAAAA|840|Oak Chestnut|Road|Suite D|Willis|Cass County|IL|66788|United States|-6|condo| +41316|AAAAAAAAEGBKAAAA|595|Meadow |Parkway|Suite M|Glenwood|Ocean County|NJ|04111|United States|-5|apartment| +41317|AAAAAAAAFGBKAAAA|694|Sixth |Lane|Suite L|Fox|Lincoln County|MS|50631|United States|-6|apartment| +41318|AAAAAAAAGGBKAAAA|890|6th Maple|Dr.|Suite 50|Harmony|Windham County|VT|06404|United States|-5|apartment| +41319|AAAAAAAAHGBKAAAA|313|River |Lane|Suite N|Wilson|Blaine County|NE|66971|United States|-6|single family| +41320|AAAAAAAAIGBKAAAA|639|5th |Ct.|Suite 370|Oak Hill|McHenry County|IL|67838|United States|-6|apartment| +41321|AAAAAAAAJGBKAAAA|427|Cherry Sycamore|Ave|Suite D|Mount Pleasant|Braxton County|WV|21933|United States|-5|condo| +41322|AAAAAAAAKGBKAAAA|702|North Ridge|Circle|Suite 260|Kingston|Warren County|PA|14975|United States|-5|apartment| +41323|AAAAAAAALGBKAAAA|755|7th |Boulevard|Suite X|Springdale|Cole County|MO|68883|United States|-6|apartment| +41324|AAAAAAAAMGBKAAAA|291|10th West|Blvd|Suite P|Langdon|Clark County|SD|50852|United States|-6|condo| +41325|AAAAAAAANGBKAAAA|712|Birch |Pkwy|Suite B|San Jose|Anchorage Borough|AK|98003|United States|-9|single family| +41326|AAAAAAAAOGBKAAAA|767|Main |Parkway|Suite A|Valley View|Pickens County|AL|35124|United States|-6|condo| +41327|AAAAAAAAPGBKAAAA|242|Oak |ST|Suite 110|Wildwood|Delaware County|PA|16871|United States|-5|single family| +41328|AAAAAAAAAHBKAAAA|537|3rd Lake|Street|Suite J|Jamestown|Wilcox County|GA|36867|United States|-5|single family| +41329|AAAAAAAABHBKAAAA|822|Eigth |Ave|Suite 130|Bunker Hill|Burleson County|TX|70150|United States|-6|apartment| +41330|AAAAAAAACHBKAAAA|920|Madison |Parkway|Suite 350|New Hope|Pinellas County|FL|39431|United States|-5|apartment| +41331|AAAAAAAADHBKAAAA|80|2nd Broadway|Cir.|Suite Q|Fairfield|Hunt County|TX|76192|United States|-6|apartment| +41332|AAAAAAAAEHBKAAAA|235|Ash Main|Cir.|Suite 190|Waterloo|Hardee County|FL|31675|United States|-5|condo| +41333|AAAAAAAAFHBKAAAA|430|9th |Circle|Suite 190|Summerfield|Fulton County|NY|10634|United States|-5|condo| +41334|AAAAAAAAGHBKAAAA|198|Sixth |Ave|Suite L|Woodville|Iberville Parish|LA|74289|United States|-6|single family| +41335|AAAAAAAAHHBKAAAA|896|Walnut |Circle|Suite 280|Highland Park|Dade County|MO|66534|United States|-6|single family| +41336|AAAAAAAAIHBKAAAA|354|15th |Blvd|Suite 430|Belleville|Kerr County|TX|72924|United States|-6|apartment| +41337|AAAAAAAAJHBKAAAA|914|Fourth Lakeview|Wy|Suite U|Buena Vista|Poquoson city|VA|25752|United States|-5|single family| +41338|AAAAAAAAKHBKAAAA|779|River Elm|Street|Suite D|Hopewell|Juniata County|PA|10587|United States|-5|apartment| +41339|AAAAAAAALHBKAAAA|814|Lake |Ln|Suite B|Bethel|Franklin County|AR|75281|United States|-6|single family| +41340|AAAAAAAAMHBKAAAA|653|Hill 12th|Ct.|Suite 270|Antioch|Kittson County|MN|58605|United States|-6|condo| +41341|AAAAAAAANHBKAAAA|234|Ridge |Wy|Suite 130|Mount Zion|Davidson County|TN|38054|United States|-5|apartment| +41342|AAAAAAAAOHBKAAAA|569|Woodland Miller|Way|Suite U|Edgewood|Ellis County|TX|70069|United States|-6|condo| +41343|AAAAAAAAPHBKAAAA|675|Birch Oak|Circle|Suite X|Belmont|Collingsworth County|TX|70191|United States|-6|single family| +41344|AAAAAAAAAIBKAAAA|757|4th |Way|Suite 310|Valley View|Uvalde County|TX|75124|United States|-6|apartment| +41345|AAAAAAAABIBKAAAA|578|Elm |Avenue|Suite F|Friendship|Baylor County|TX|74536|United States|-6|condo| +41346|AAAAAAAACIBKAAAA|64|||Suite U||||25281||-5|| +41347|AAAAAAAADIBKAAAA|232|7th Elm|Lane|Suite 60|Fairfield|Cook County|IL|66192|United States|-6|condo| +41348|AAAAAAAAEIBKAAAA|208|3rd Lakeview|ST|Suite 360|Shiloh|Oneida County|ID|89275|United States|-7|single family| +41349|AAAAAAAAFIBKAAAA|312|Lake |Lane|Suite 170|Pleasant Valley|Cerro Gordo County|IA|52477|United States|-6|single family| +41350|AAAAAAAAGIBKAAAA|859|Laurel |Pkwy|Suite O|Harmony|Washington County|FL|35804|United States|-5|condo| +41351|AAAAAAAAHIBKAAAA|496|Sunset Laurel|Way|Suite H|Cumberland|Holmes County|OH|48971|United States|-5|condo| +41352|AAAAAAAAIIBKAAAA|956|Lakeview Jackson||Suite 70|West Point|Kearney County|NE||||| +41353|AAAAAAAAJIBKAAAA|832|Madison |RD|Suite I|Brownsville|Allegheny County|PA|19310|United States|-5|apartment| +41354|AAAAAAAAKIBKAAAA|187|Dogwood First|Ct.|Suite 270|Red Hill|Thurston County|WA|94338|United States|-8|single family| +41355|AAAAAAAALIBKAAAA|822|8th Cedar|RD|Suite O|Lebanon|Ford County|KS|62898|United States|-6|condo| +41356|AAAAAAAAMIBKAAAA|967|View Park|Cir.|Suite S|Bethel|Darke County|OH|45281|United States|-5|condo| +41357|AAAAAAAANIBKAAAA|274|1st |Boulevard|Suite B|Fairview|Ravalli County|MT|65709|United States|-7|condo| +41358|AAAAAAAAOIBKAAAA|129|Spring |Wy|Suite 300|Oak Hill|Hart County|KY|47838|United States|-6|single family| +41359|AAAAAAAAPIBKAAAA|792|Lee North|Cir.|Suite K|Deerfield|Calaveras County|CA|99840|United States|-8|single family| +41360|AAAAAAAAAJBKAAAA|691|East |Circle|Suite 170|Ashland|Brown County|WI|54244|United States|-6|condo| +41361|AAAAAAAABJBKAAAA|454|Dogwood Williams|Pkwy|Suite G|Hyde Park|Bradford County|FL|38239|United States|-5|apartment| +41362|AAAAAAAACJBKAAAA|77|Ninth |Lane|Suite Y|Bunker Hill|Sweet Grass County|MT|60150|United States|-7|condo| +41363|AAAAAAAADJBKAAAA|896|Lee Center|Road|Suite 490|Greendale|Oxford County|ME|09943|United States|-5|apartment| +41364|AAAAAAAAEJBKAAAA|423|Davis |Dr.|Suite S|Jamestown|La Porte County|IN|46867|United States|-5|apartment| +41365|AAAAAAAAFJBKAAAA|116|Railroad Spring|Parkway|Suite 440|Georgetown|Racine County|WI|57057|United States|-6|condo| +41366|AAAAAAAAGJBKAAAA|392|Ridge Park|Pkwy|Suite B|Friendship|Camas County|ID|84536|United States|-7|condo| +41367|AAAAAAAAHJBKAAAA|831|Ash |Lane|Suite 210|Newtown|Washington County|OK|71749|United States|-6|condo| +41368|AAAAAAAAIJBKAAAA||||||||62477||-7|| +41369|AAAAAAAAJJBKAAAA|447|Fourth Elm|Lane|Suite 470|Stringtown|Hale County|TX|70162|United States|-6|single family| +41370|AAAAAAAAKJBKAAAA|413|Cherry |Parkway|Suite W|Greenfield|Sumner County|KS|65038|United States|-6|condo| +41371|AAAAAAAALJBKAAAA||3rd ||Suite C||||78054|United States|-6|| +41372|AAAAAAAAMJBKAAAA|135|11th 2nd|Street|Suite 370|Highland|Penobscot County|ME|09454|United States|-5|apartment| +41373|AAAAAAAANJBKAAAA|143|Hickory Oak|RD|Suite 330|Bethel|Stark County|ND|55281|United States|-6|single family| +41374|AAAAAAAAOJBKAAAA|932|North Hill|Avenue|Suite X|Greenville|Evans County|GA|31387|United States|-5|single family| +41375|AAAAAAAAPJBKAAAA|953|Second Lee|Boulevard|Suite 220|Greenville|Hendricks County|IN|41387|United States|-5|single family| +41376|AAAAAAAAAKBKAAAA|829|Hill 1st|ST|Suite 260|Georgetown|Carbon County|MT|67057|United States|-7|apartment| +41377|AAAAAAAABKBKAAAA|455|Broadway View|Ln|Suite D|Forest Hills|Decatur County|IN|49237|United States|-5|apartment| +41378|AAAAAAAACKBKAAAA|418||||Arcola|Jefferson Davis County|||United States|-6|| +41379|AAAAAAAADKBKAAAA|146|View 2nd|Lane|Suite U|Kingston|Cheyenne County|KS|64975|United States|-6|condo| +41380|AAAAAAAAEKBKAAAA|681|3rd |Wy|Suite 240|Georgetown|Cherokee County|GA|37057|United States|-5|apartment| +41381|AAAAAAAAFKBKAAAA|206|Broadway |Ct.|Suite 280|Midway|Jefferson County|KY|41904|United States|-6|apartment| +41382|AAAAAAAAGKBKAAAA|391|Jackson |Pkwy|Suite 140|Lakeside|Addison County|VT|09532|United States|-5|single family| +41383|AAAAAAAAHKBKAAAA|626|Central Ash|Parkway|Suite 390|Woodland|Pierce County|GA|34854|United States|-5|apartment| +41384|AAAAAAAAIKBKAAAA|743|Woodland |Drive|Suite 170|Green Acres|Habersham County|GA|37683|United States|-5|apartment| +41385|AAAAAAAAJKBKAAAA|637|Willow |Lane|Suite N|Sulphur Springs|Allamakee County|IA|58354|United States|-6|apartment| +41386|AAAAAAAAKKBKAAAA|355|Main |Blvd|Suite T|Mountain View|Terrebonne Parish|LA|74466|United States|-6|condo| +41387|AAAAAAAALKBKAAAA|717|11th Ninth|Road|Suite 350|Greenfield|Greenbrier County|WV|25038|United States|-5|apartment| +41388|AAAAAAAAMKBKAAAA|726|Cedar |Avenue|Suite O|Tanglewood|Switzerland County|IN|48994|United States|-5|single family| +41389|AAAAAAAANKBKAAAA|909|Johnson Broadway|Dr.|Suite K|Galena|Dawson County|MT|64369|United States|-7|condo| +41390|AAAAAAAAOKBKAAAA|632|North Pine|Street|Suite 330|Springdale|Fisher County|TX|78883|United States|-6|single family| +41391|AAAAAAAAPKBKAAAA|110|Hickory Birch|Ct.|Suite R|Langdon|Crenshaw County|AL|30852|United States|-6|single family| +41392|AAAAAAAAALBKAAAA|538|Wilson Oak|Ct.|Suite Q|Oakwood|Ingham County|MI|40169|United States|-5|condo| +41393|AAAAAAAABLBKAAAA|802|Broadway |Street|Suite 470|Marion|Marshall County|IL|60399|United States|-6|apartment| +41394|AAAAAAAACLBKAAAA|303|Fourth Park|Cir.|Suite B|Hopewell|Jackson County|GA|30587|United States|-5|single family| +41395|AAAAAAAADLBKAAAA|577|4th North|Pkwy|Suite N|Woodlawn|Fillmore County|NE|64098|United States|-6|single family| +41396|AAAAAAAAELBKAAAA|20|Maple |Ave|Suite P|Jamestown|Caldwell Parish|LA|76867|United States|-6|apartment| +41397|AAAAAAAAFLBKAAAA|875|Johnson Valley|Parkway|Suite 470|Woodland|Craighead County|AR|74854|United States|-6|apartment| +41398|AAAAAAAAGLBKAAAA|565|Mill Valley|Parkway|Suite 420|Walnut Grove|Fluvanna County|VA|27752|United States|-5|condo| +41399|AAAAAAAAHLBKAAAA|311|10th 4th|Blvd|Suite B|Glenwood|Fergus County|MT|63511|United States|-7|single family| +41400|AAAAAAAAILBKAAAA|242|North Main|Way|Suite Q|Friendship|Aleutians East Borough|AK|94536|United States|-9|apartment| +41401|AAAAAAAAJLBKAAAA|842|Cedar Park|Ln|Suite M|Pleasant Hill|Anne Arundel County|MD|23604|United States|-5|apartment| +41402|AAAAAAAAKLBKAAAA|79|10th |Ln|Suite S|Sunnyside|Shelby County|IN|41952|United States|-5|apartment| +41403|AAAAAAAALLBKAAAA|670|Main Hillcrest|Road|Suite 360|Union Hill|Pine County|MN|57746|United States|-6|single family| +41404|AAAAAAAAMLBKAAAA|295|Center Green|ST|Suite 450|Greenville|Costilla County|CO|81387|United States|-7|single family| +41405|AAAAAAAANLBKAAAA|590|Elm |Court|Suite L|Liberty|Monroe County|FL|33451|United States|-5|condo| +41406|AAAAAAAAOLBKAAAA|696|Elm 5th|Way|Suite 480|Macedonia|Huerfano County|CO|81087|United States|-7|single family| +41407|AAAAAAAAPLBKAAAA|592|Smith |Drive|Suite 190|Wildwood|Hand County|SD|56871|United States|-7|apartment| +41408|AAAAAAAAAMBKAAAA|333|Cedar 1st|ST|Suite 110|Five Forks|Throckmorton County|TX|72293|United States|-6|condo| +41409|AAAAAAAABMBKAAAA|624|View Franklin|Boulevard|Suite 180|Lakeside|Pointe Coupee Parish|LA|79532|United States|-6|single family| +41410|AAAAAAAACMBKAAAA|396|14th |Avenue|Suite V|Mount Olive|Harding County|SD|58059|United States|-7|condo| +41411|AAAAAAAADMBKAAAA|783|4th Hickory|Road|Suite 460|Wilson|Osage County|MO|66971|United States|-6|condo| +41412|AAAAAAAAEMBKAAAA|551|Walnut Lee|Cir.|Suite 410|Clifton|Harrison County|OH|48014|United States|-5|single family| +41413|AAAAAAAAFMBKAAAA|887|Jefferson |Boulevard|Suite 470|Lakewood|Clackamas County|OR|98877|United States|-8|apartment| +41414|AAAAAAAAGMBKAAAA|550|Hickory |Court|Suite Q|Fairfield|Dodge County|WI|56192|United States|-6|apartment| +41415|AAAAAAAAHMBKAAAA|493|10th |Cir.|Suite C|Roy|Kidder County|ND|50744|United States|-6|condo| +41416|AAAAAAAAIMBKAAAA|185|Madison 1st|Circle|Suite 360|Bridgeport|Snohomish County|WA|95817|United States|-8|apartment| +41417|AAAAAAAAJMBKAAAA|40|2nd |Court|Suite I|Oak Hill|Mitchell County|TX|77838|United States|-6|single family| +41418|AAAAAAAAKMBKAAAA|454|6th |Road|Suite K|Plainview|Buchanan County|VA|23683|United States|-5|apartment| +41419|AAAAAAAALMBKAAAA|531|View |Way|Suite R|Shiloh|Petroleum County|MT|69275|United States|-7|apartment| +41420|AAAAAAAAMMBKAAAA|108|11th Second|Cir.|Suite 420|Oak Hill|Hampshire County|MA|08438|United States|-5|condo| +41421|AAAAAAAANMBKAAAA|254|Wilson |Court|Suite 410|Ashland|Washington Parish|LA|74244|United States|-6|condo| +41422|AAAAAAAAOMBKAAAA|287|Dogwood |RD|Suite Y|Florence|Bradford County|PA|13394|United States|-5|apartment| +41423|AAAAAAAAPMBKAAAA|453|7th Lake|ST|Suite 160|Ferguson|Dallam County|TX|71821|United States|-6|single family| +41424|AAAAAAAAANBKAAAA|230|Valley Wilson|Circle|Suite P|Lakewood|Ashland County|OH|48877|United States|-5|single family| +41425|AAAAAAAABNBKAAAA|898|Third Spring|Ct.|Suite L|Crossroads|Cannon County|TN|30534|United States|-5|apartment| +41426|AAAAAAAACNBKAAAA|961|Washington |ST|Suite 430|Florence|Rapides Parish|LA|73394|United States|-6|single family| +41427|AAAAAAAADNBKAAAA|395|Spring 5th|Ct.|Suite 400|Lakeside|Greeley County|NE|69532|United States|-6|condo| +41428|AAAAAAAAENBKAAAA|418|Elm Pine|RD|Suite O|Woodlawn|Evans County|GA|34098|United States|-5|condo| +41429|AAAAAAAAFNBKAAAA|471|Poplar Spruce|Drive|Suite 270|Centerville|McKinley County|NM|80059|United States|-7|single family| +41430|AAAAAAAAGNBKAAAA|815|Lakeview |Ct.|Suite 340|Concord|Rich County|UT|84107|United States|-7|apartment| +41431|AAAAAAAAHNBKAAAA|39|1st 9th|Parkway|Suite 10|Forest Hills|Boone County|KY|49237|United States|-6|single family| +41432|AAAAAAAAINBKAAAA|166|8th |Dr.|Suite 200|Marion|Kearny County|KS|60399|United States|-6|apartment| +41433|AAAAAAAAJNBKAAAA|314|Jackson |Blvd|Suite 450|Clinton|Copiah County|MS|58222|United States|-6|single family| +41434|AAAAAAAAKNBKAAAA|493|View |Pkwy|Suite 260|Highland Park|Adams County|ID|86534|United States|-7|condo| +41435|AAAAAAAALNBKAAAA|820|Mill Maple|Blvd|Suite 200|Liberty|Clark County|IN|43451|United States|-5|condo| +41436|AAAAAAAAMNBKAAAA|639|Birch |Court|Suite 280|New Hope|Lemhi County|ID|89431|United States|-7|single family| +41437|AAAAAAAANNBKAAAA|286|3rd |Wy|Suite O|Weldon|Lonoke County|AR|76277|United States|-6|condo| +41438|AAAAAAAAONBKAAAA|150|Johnson |Drive|Suite 10|Richardson|Walker County|AL|37687|United States|-6|apartment| +41439|AAAAAAAAPNBKAAAA|892|Sixth |Boulevard|Suite 310|Lakeview|Walton County|FL|38579|United States|-5|apartment| +41440|AAAAAAAAAOBKAAAA|547|Maple South|RD|Suite 160|Woodlawn|Logan County|AR|74098|United States|-6|condo| +41441|AAAAAAAABOBKAAAA|680|Lakeview |Lane|Suite 420|Antioch|Santa Barbara County|CA|98605|United States|-8|condo| +41442|AAAAAAAACOBKAAAA|158|Oak Park|Lane|Suite 80|Salem|Stewart County|GA|38048|United States|-5|condo| +41443|AAAAAAAADOBKAAAA|783|2nd |Blvd|Suite 90|Pine Grove|Montgomery County|AR|74593|United States|-6|condo| +41444|AAAAAAAAEOBKAAAA||||Suite 130||||69532|||single family| +41445|AAAAAAAAFOBKAAAA|397|North |Avenue|Suite 400|Belmont|Stark County|IL|60191|United States|-6|single family| +41446|AAAAAAAAGOBKAAAA|27|Mill |Lane|Suite Y|Sumner|Howard County|NE|60519|United States|-7|apartment| +41447|AAAAAAAAHOBKAAAA|620|10th |Circle|Suite 410|Crossroads|Jackson County|OH|40534|United States|-5|condo| +41448|AAAAAAAAIOBKAAAA|661|10th |Court|Suite 310|Riverdale|Hardin County|KY|49391|United States|-6|single family| +41449|AAAAAAAAJOBKAAAA|770|2nd |Boulevard|Suite P|Bethel|Jones County|IA|55281|United States|-6|condo| +41450|AAAAAAAAKOBKAAAA|648|7th |Court|Suite X|Vienna|Meade County|KY|45119|United States|-5|single family| +41451|AAAAAAAALOBKAAAA|580|9th Smith|Dr.|Suite 10|Lakewood|Cameron County|TX|78877|United States|-6|single family| +41452|AAAAAAAAMOBKAAAA|953|Lee 1st|Ln|Suite 60|Farmington|Burke County|GA|39145|United States|-5|condo| +41453|AAAAAAAANOBKAAAA|303|College Franklin|Avenue|Suite 240|Woodland|Frio County|TX|74854|United States|-6|condo| +41454|AAAAAAAAOOBKAAAA|502|Park |Wy|Suite P|Macedonia|Spotsylvania County|VA|21087|United States|-5|condo| +41455|AAAAAAAAPOBKAAAA|702|Hill First|Boulevard|Suite 190|Maple Hill|McDowell County|NC|28095|United States|-5|single family| +41456|AAAAAAAAAPBKAAAA|595|Highland |Ave|Suite L|Newport|Jefferson County|PA|11521|United States|-5|condo| +41457|AAAAAAAABPBKAAAA|719|Smith |Dr.|Suite 410|Perkins|Saguache County|CO|81852|United States|-7|condo| +41458|AAAAAAAACPBKAAAA|58|Wilson |Boulevard|Suite G|Clifton|Calhoun County|IA|58014|United States|-6|apartment| +41459|AAAAAAAADPBKAAAA|747|Willow |Avenue|Suite A|Riverview|Benton County|IN|49003|United States|-5|condo| +41460|AAAAAAAAEPBKAAAA|828|Cedar |Avenue|Suite 400|Five Forks|Stark County|ND|52293|United States|-6|single family| +41461|AAAAAAAAFPBKAAAA|762|South Main|Blvd|Suite 100|Highland|Armstrong County|PA|19454|United States|-5|single family| +41462|AAAAAAAAGPBKAAAA|838|River |Ave|Suite C|Liberty|Lancaster County|NE|63451|United States|-7|condo| +41463|AAAAAAAAHPBKAAAA|472|Chestnut |Blvd|Suite R|Maple Grove|Noble County|OK|78252|United States|-6|single family| +41464|AAAAAAAAIPBKAAAA|378|Broadway |Ave|Suite 120|Brownsville|Fayette County|IA|59310|United States|-6|single family| +41465|AAAAAAAAJPBKAAAA|502|Park Cedar|Parkway|Suite 410|Jackson|Sutton County|TX|79583|United States|-6|single family| +41466|AAAAAAAAKPBKAAAA|226|Broadway |RD|Suite 300|Bunker Hill|Baker County|GA|30150|United States|-5|single family| +41467|AAAAAAAALPBKAAAA|109|11th Washington|Court|Suite 100|Woodcrest|Lane County|KS|64919|United States|-6|condo| +41468|AAAAAAAAMPBKAAAA|239|6th |Circle|Suite 350|Crossroads|Champaign County|IL|60534|United States|-6|condo| +41469|AAAAAAAANPBKAAAA|476|5th |Street|Suite 350|Arlington|Hickman County|KY|46557|United States|-6|apartment| +41470|AAAAAAAAOPBKAAAA|331|Oak West|Avenue|Suite 340|Bunker Hill|Cass County|NE|60150|United States|-6|condo| +41471|AAAAAAAAPPBKAAAA|968|1st North|Circle|Suite C|Forest Hills|Columbia County|PA|19237|United States|-5|apartment| +41472|AAAAAAAAAACKAAAA|705|Chestnut Fifth|Road|Suite D|Union Hill|Highland County|VA|27746|United States|-5|condo| +41473|AAAAAAAABACKAAAA|369|8th Madison|Drive|Suite 290|Woodland|Lincoln County|KS|64854|United States|-6|single family| +41474|AAAAAAAACACKAAAA|873|4th Madison|Wy|Suite 230|Lakewood|Woodford County|IL|68877|United States|-6|apartment| +41475|AAAAAAAADACKAAAA|408|Maple |Parkway|Suite 410|Jamestown|Ford County|KS|66867|United States|-6|apartment| +41476|AAAAAAAAEACKAAAA|454|3rd Sunset|Wy|Suite 300|Antioch|Duchesne County|UT|88605|United States|-7|apartment| +41477|AAAAAAAAFACKAAAA|131|Ridge Smith|Way|Suite 110|Clinton|Hickman County|KY|48222|United States|-6|single family| +41478|AAAAAAAAGACKAAAA|230|Spring Forest|Lane|Suite 250|Riverdale|Olmsted County|MN|59391|United States|-6|condo| +41479|AAAAAAAAHACKAAAA|737|Highland |Street|Suite 220|Lakewood|Medina County|TX|78877|United States|-6|condo| +41480|AAAAAAAAIACKAAAA|310|Lee |ST|Suite 230|Glenwood|Lake County|IL|63511|United States|-6|single family| +41481|AAAAAAAAJACKAAAA|651|Jefferson 9th|Boulevard|Suite P|Marion|Westmoreland County|VA|20399|United States|-5|single family| +41482|AAAAAAAAKACKAAAA|941|Lincoln College|Pkwy|Suite 430|Woodville|Clay County|SD|54289|United States|-6|condo| +41483|AAAAAAAALACKAAAA|441|Spruce Main|Circle|Suite R|Fairfield|Clearwater County|ID|86192|United States|-7|single family| +41484|AAAAAAAAMACKAAAA|484|Spruce Jefferson|Drive|Suite 50|Wilson|Gentry County|MO|66971|United States|-6|condo| +41485|AAAAAAAANACKAAAA|532|Lincoln |Ln|Suite I|Pleasant Hill|Franklin County|OH|43604|United States|-5|apartment| +41486|AAAAAAAAOACKAAAA|960|Main Main|Ln|Suite S|Lebanon|Mono County|CA|92898|United States|-8|single family| +41487|AAAAAAAAPACKAAAA|222|Sycamore |Avenue|Suite 60|Summit|Lewis County|NY|10499|United States|-5|single family| +41488|AAAAAAAAABCKAAAA|780|Park ||Suite S|Mountain View|Matanuska-Susitna Borough|AK||United States|-9|condo| +41489|AAAAAAAABBCKAAAA|271|Spruce Lincoln|Lane|Suite 0|Cedar Grove|Wibaux County|MT|60411|United States|-7|condo| +41490|AAAAAAAACBCKAAAA|345|Washington Fourth|ST|Suite L|Springfield|New London County|CT|09903|United States|-5|condo| +41491|AAAAAAAADBCKAAAA|832|6th |Lane|Suite U|Union|Lincoln County|NM|88721|United States|-7|single family| +41492|AAAAAAAAEBCKAAAA|249|College |Avenue|Suite N|Wildwood|Dougherty County|GA|36871|United States|-5|apartment| +41493|AAAAAAAAFBCKAAAA|146|Fifth |Wy|Suite H|Forest Hills|Whitfield County|GA|39237|United States|-5|apartment| +41494|AAAAAAAAGBCKAAAA|861|Church |Drive|Suite 290|Georgetown|Miller County|MO|67057|United States|-6|apartment| +41495|AAAAAAAAHBCKAAAA|143|Seventh South|Wy|Suite P|Lakeside|Shiawassee County|MI|49532|United States|-5|condo| +41496|AAAAAAAAIBCKAAAA|108|Oak 1st|Drive|Suite 140|Rose Hill|Evans County|GA|32024|United States|-5|condo| +41497|AAAAAAAAJBCKAAAA|990|Second Oak|Circle|Suite E|Shady Grove|Bell County|KY|42812|United States|-6|single family| +41498|AAAAAAAAKBCKAAAA|561|First |Road|Suite 350|Woodlawn|Mitchell County|GA|34098|United States|-5|apartment| +41499|AAAAAAAALBCKAAAA|759|East Meadow|Wy|Suite 210|Oak Ridge|Kane County|IL|68371|United States|-6|apartment| +41500|AAAAAAAAMBCKAAAA|834|Center |Ct.|Suite T|Fairview|Westmoreland County|PA|15709|United States|-5|apartment| +41501|AAAAAAAANBCKAAAA|756|13th Third|Road|Suite 80|Weldon|Seward County|KS|66277|United States|-6|single family| +41502|AAAAAAAAOBCKAAAA||||Suite 200||Fairbanks North Star Borough||97683|United States||condo| +41503|AAAAAAAAPBCKAAAA|343|13th |Circle|Suite 160|Westgate|Chase County|KS|62366|United States|-6|condo| +41504|AAAAAAAAACCKAAAA|278|Main |Wy|Suite 250|Summit|Traill County|ND|50499|United States|-6|single family| +41505|AAAAAAAABCCKAAAA|877|Walnut |Blvd|Suite 420|Griffin|Marion County|TX|71204|United States|-6|single family| +41506|AAAAAAAACCCKAAAA|555|10th Fifth|Ct.|Suite 250|Shiloh|Adams County|WA|99275|United States|-8|single family| +41507|AAAAAAAADCCKAAAA|263|Elm Sixth|Ave|Suite 310|Five Forks|Decatur County|KS|62293|United States|-6|condo| +41508|AAAAAAAAECCKAAAA|744|Fourth 9th|Way|Suite 80|Pleasant Hill|Brown County|SD|53604|United States|-6|apartment| +41509|AAAAAAAAFCCKAAAA|141|Adams |Dr.|Suite 80|Hamilton|Lincoln County|NC|22808|United States|-5|single family| +41510|AAAAAAAAGCCKAAAA|214|8th |Boulevard|Suite M|Jackson|Leon County|TX|79583|United States|-6|apartment| +41511|AAAAAAAAHCCKAAAA|||||Red Hill|Livingston County|NY||United States||apartment| +41512|AAAAAAAAICCKAAAA|715|Hill |RD|Suite I|Wilson|Dickinson County|IA|56971|United States|-6|condo| +41513|AAAAAAAAJCCKAAAA|524|Elm |Wy|Suite O|Belmont|Rawlins County|KS|60191|United States|-6|apartment| +41514|AAAAAAAAKCCKAAAA|100|Franklin |Cir.|Suite 120|Georgetown|Missaukee County|MI|47057|United States|-5|condo| +41515|AAAAAAAALCCKAAAA|243|River |Ave|Suite 250|Crossroads|Geneva County|AL|30534|United States|-6|single family| +41516|AAAAAAAAMCCKAAAA|677|6th |ST|Suite N|Howell|Dale County|AL|34854|United States|-6|condo| +41517|AAAAAAAANCCKAAAA|409|Second |Avenue|Suite Y|Walnut Grove|Russell County|AL|37752|United States|-6|single family| +41518|AAAAAAAAOCCKAAAA|261|Tenth |RD|Suite T|Harmony|Houston County|MN|55804|United States|-6|apartment| +41519|AAAAAAAAPCCKAAAA|304|Sycamore |Blvd|Suite 170|Greenville|Logan County|ND|51387|United States|-6|single family| +41520|AAAAAAAAADCKAAAA|745|Green Sunset|Parkway|Suite 380|Liberty|White Pine County|NV|83451|United States|-8|apartment| +41521|AAAAAAAABDCKAAAA|956|South |Boulevard|Suite J|Florence|Buncombe County|NC|23394|United States|-5|condo| +41522|AAAAAAAACDCKAAAA|864|Walnut |Ln|Suite 360|Bloomingdale|Pamlico County|NC|21824|United States|-5|single family| +41523|AAAAAAAADDCKAAAA|699|Elm Cherry|Pkwy|Suite B|Highland|Calaveras County|CA|99454|United States|-8|apartment| +41524|AAAAAAAAEDCKAAAA|279|Elm |Boulevard|Suite 60|Georgetown|Box Elder County|UT|87057|United States|-7|single family| +41525|AAAAAAAAFDCKAAAA|82|Main Oak|Blvd|Suite 310|Lincoln|Walker County|AL|31289|United States|-6|apartment| +41526|AAAAAAAAGDCKAAAA|336|4th |Street|Suite 400|Five Forks|Teton County|WY|82293|United States|-7|condo| +41527|AAAAAAAAHDCKAAAA|610|View |Wy|Suite 430|Oakwood|Barren County|KY|40169|United States|-6|single family| +41528|AAAAAAAAIDCKAAAA|966|Washington |Blvd|Suite 220|Lincoln|Nicholas County|WV|21289|United States|-5|single family| +41529|AAAAAAAAJDCKAAAA|373|Pine Lee|Avenue|Suite C|Wilson|Bland County|VA|26971|United States|-5|apartment| +41530|AAAAAAAAKDCKAAAA|870|Meadow |Ave|Suite 170|Unionville|Latah County|ID|81711|United States|-7|condo| +41531|AAAAAAAALDCKAAAA|877|6th |Blvd|Suite C|Plainview|Marion County|AL|33683|United States|-6|condo| +41532|AAAAAAAAMDCKAAAA|366|Pine |Lane|Suite 250|Oak Ridge|Vigo County|IN|48371|United States|-5|condo| +41533|AAAAAAAANDCKAAAA|213|College |ST|Suite 270|Riverdale|Buchanan County|IA|59391|United States|-6|single family| +41534|AAAAAAAAODCKAAAA|198|Williams 2nd|Dr.|Suite 410|Forest Hills|Becker County|MN|59237|United States|-6|condo| +41535|AAAAAAAAPDCKAAAA|383|Walnut |Street|Suite V|Greenfield|Harrison County|WV|25038|United States|-5|single family| +41536|AAAAAAAAAECKAAAA|919|Mill |Ln|Suite U|Amherst|Lake County|CA|98119|United States|-8|condo| +41537|AAAAAAAABECKAAAA|||Lane|||Mercer County|OH||United States||apartment| +41538|AAAAAAAACECKAAAA|368|4th Franklin|Drive|Suite 200|Stringtown|Ward County|ND|50162|United States|-6|single family| +41539|AAAAAAAADECKAAAA|75|Third Williams|Court|Suite Y|Florence|Hinds County|MS|53394|United States|-6|single family| +41540|AAAAAAAAEECKAAAA|842|Park |RD|Suite H|Springdale|Garfield County|NE|68883|United States|-6|condo| +41541|AAAAAAAAFECKAAAA|517|Woodland |Blvd|Suite 310|Oak Ridge|McPherson County|KS|68371|United States|-6|condo| +41542|AAAAAAAAGECKAAAA|340|5th Birch|Court|Suite 440|Newport|Fayette County|OH|41521|United States|-5|apartment| +41543|AAAAAAAAHECKAAAA|578|River Mill|RD|Suite 450|Harmony|Stanton County|KS|65804|United States|-6|condo| +41544|AAAAAAAAIECKAAAA|52|Hickory |Road|Suite Y|Sunnyside|Chesterfield County|SC|21952|United States|-5|single family| +41545|AAAAAAAAJECKAAAA|93|Oak |Parkway|Suite L|Midway|Ferry County|WA|91904|United States|-8|single family| +41546|AAAAAAAAKECKAAAA|628|Eigth Main|Avenue|Suite 220|Bunker Hill|Wilkes County|NC|20150|United States|-5|apartment| +41547|AAAAAAAALECKAAAA|589|4th |Ln|Suite G|Kent|Koochiching County|MN|50317|United States|-6|single family| +41548|AAAAAAAAMECKAAAA|768|Madison |Pkwy|Suite J|Union Hill|Madison County|NE|67746|United States|-7|single family| +41549|AAAAAAAANECKAAAA|590|15th First|Parkway|Suite 170|Highland Park|Posey County|IN|46534|United States|-5|apartment| +41550|AAAAAAAAOECKAAAA|204|Elm Oak|Way|Suite S|Georgetown|Berkeley County|SC|27057|United States|-5|single family| +41551|AAAAAAAAPECKAAAA|177|Poplar Cedar|Blvd|Suite D|Glenwood|Oklahoma County|OK|73511|United States|-6|condo| +41552|AAAAAAAAAFCKAAAA|552|Ridge Sycamore|Lane|Suite 370|Lake Forest|Fulton County|AR|76000|United States|-6|condo| +41553|AAAAAAAABFCKAAAA|224|Washington |Cir.|Suite L|Oakdale|Chaves County|NM|89584|United States|-7|single family| +41554|AAAAAAAACFCKAAAA|121|10th |Parkway|Suite 40|Mount Zion|Portage County|OH|48054|United States|-5|condo| +41555|AAAAAAAADFCKAAAA|796|4th |Circle|Suite W|Greenfield|Faulk County|SD|55038|United States|-6|single family| +41556|AAAAAAAAEFCKAAAA|186|Hickory West|Dr.|Suite 140|Louisville|Lenawee County|MI|44464|United States|-5|single family| +41557|AAAAAAAAFFCKAAAA|703|3rd |Boulevard|Suite 10|Florence|Walla Walla County|WA|93394|United States|-8|single family| +41558|AAAAAAAAGFCKAAAA|126|Birch Madison|Boulevard|Suite 400|Bethel|Monroe County|MO|65281|United States|-6|condo| +41559|AAAAAAAAHFCKAAAA|521|Railroad |Boulevard|Suite O|Georgetown|Braxton County|WV|27057|United States|-5|condo| +41560|AAAAAAAAIFCKAAAA|258|Hillcrest |RD|Suite H|Oakwood|Nobles County|MN|50169|United States|-6|condo| +41561|AAAAAAAAJFCKAAAA|843|Fourteenth |Lane|Suite R|Clifton|Ripley County|MO|68014|United States|-6|condo| +41562|AAAAAAAAKFCKAAAA|122|North |Avenue|Suite 480|Shiloh|McCormick County|SC|29275|United States|-5|condo| +41563|AAAAAAAALFCKAAAA|363|Wilson |Drive|Suite 40|Enterprise|Modoc County|CA|91757|United States|-8|condo| +41564|AAAAAAAAMFCKAAAA|||Circle|||Guadalupe County|TX|74464|United States||single family| +41565|AAAAAAAANFCKAAAA|292|Ridge Locust|Dr.|Suite U|Clifton|Trigg County|KY|48014|United States|-5|condo| +41566|AAAAAAAAOFCKAAAA|358|Tenth |Boulevard|Suite 10|Marion|Wibaux County|MT|60399|United States|-7|apartment| +41567|AAAAAAAAPFCKAAAA|171|Davis |Blvd|Suite 140|Cedar Grove|Gaston County|NC|20411|United States|-5|apartment| +41568|AAAAAAAAAGCKAAAA|356|6th |Circle|||Sanborn County|SD|56098|||| +41569|AAAAAAAABGCKAAAA|701|Oak River|Pkwy|Suite 60|Hillcrest|Mohave County|AZ|83003|United States|-7|single family| +41570|AAAAAAAACGCKAAAA||College |Road|Suite 230|||WV||United States|-5|| +41571|AAAAAAAADGCKAAAA|720|Central |Lane|Suite C|Hopewell|Sandoval County|NM|80587|United States|-7|single family| +41572|AAAAAAAAEGCKAAAA|162|South |Ct.|Suite E|Greenwood|Franklin County|AL|38828|United States|-6|single family| +41573|AAAAAAAAFGCKAAAA|725|Laurel |Lane|Suite 30|Five Points|Mineral County|WV|26098|United States|-5|single family| +41574|AAAAAAAAGGCKAAAA|364|1st Railroad||Suite 30|Riley|||51692||-6|| +41575|AAAAAAAAHGCKAAAA|748|Sycamore |Avenue|Suite A|Westgate|Macon County|IL|62366|United States|-6|condo| +41576|AAAAAAAAIGCKAAAA|707|View |Blvd|Suite W|New Hope|Perry County|AR|79431|United States|-6|condo| +41577|AAAAAAAAJGCKAAAA|142|Williams |Ln|Suite 170|Oak Ridge|Keweenaw County|MI|48371|United States|-5|single family| +41578|AAAAAAAAKGCKAAAA|592|Spruce Jefferson|Pkwy|Suite S|Delmar|Trigg County|KY|43957|United States|-5|condo| +41579|AAAAAAAALGCKAAAA|57|Central Willow|Street|Suite 30|Guilford|Dodge County|GA|34408|United States|-5|apartment| +41580|AAAAAAAAMGCKAAAA|991|Willow Center|Lane|Suite 390|Salem|Butler County|IA|58048|United States|-6|condo| +41581|AAAAAAAANGCKAAAA|87|3rd |Street|Suite T|Clifton|Clinton County|NY|18014|United States|-5|apartment| +41582|AAAAAAAAOGCKAAAA|449|View Main|Parkway|Suite C|Lakeview|Fremont County|ID|88579|United States|-7|apartment| +41583|AAAAAAAAPGCKAAAA|926|Elm 6th|Dr.|Suite B|White Oak|Weakley County|TN|36668|United States|-6|single family| +41584|AAAAAAAAAHCKAAAA|||Parkway|Suite 30||Franklin County|WA|98252||-8|| +41585|AAAAAAAABHCKAAAA|812|Elm 14th|RD|Suite 10|Walnut Grove|Bailey County|TX|77752|United States|-6|single family| +41586|AAAAAAAACHCKAAAA|776|East Ridge|Cir.|Suite I|Maple Grove|Brooks County|TX|78252|United States|-6|single family| +41587|AAAAAAAADHCKAAAA|48|Highland |Road|Suite 90|Oak Ridge|Bucks County|PA|18371|United States|-5|condo| +41588|AAAAAAAAEHCKAAAA|297|Park |Ave|Suite N|Stringtown|Garrett County|MD|20162|United States|-5|condo| +41589|AAAAAAAAFHCKAAAA|76|River Ash|Boulevard|Suite J|Edgewood|Knox County|OH|40069|United States|-5|apartment| +41590|AAAAAAAAGHCKAAAA|505|Davis Maple|Lane|Suite 0|Greenwood|Dickinson County|IA|58828|United States|-6|condo| +41591|AAAAAAAAHHCKAAAA|106|Spruce 10th|Wy|Suite 210|Proctor|Webster County|GA|38140|United States|-5|single family| +41592|AAAAAAAAIHCKAAAA|8|Fourth Adams|Street|Suite Y|Walnut Grove|Limestone County|AL|37752|United States|-6|condo| +41593|AAAAAAAAJHCKAAAA|526|Dogwood |Circle|Suite D|New Hope|Habersham County|GA|39431|United States|-5|apartment| +41594|AAAAAAAAKHCKAAAA|706|Williams |Pkwy|Suite Y|Antioch|Lanier County|GA|38605|United States|-5|single family| +41595|AAAAAAAALHCKAAAA|916|Jefferson Lake|Blvd|Suite Q|Concord|Lamar County|GA|34107|United States|-5|single family| +41596|AAAAAAAAMHCKAAAA|971|Dogwood |Parkway|Suite A|Pleasant Grove|Van Buren County|AR|74136|United States|-6|single family| +41597|AAAAAAAANHCKAAAA|982|14th |Dr.|Suite 250|Antioch|Swift County|MN|58605|United States|-6|apartment| +41598|AAAAAAAAOHCKAAAA|820|Madison 5th|Boulevard|Suite 320|Shady Grove|Pondera County|MT|62812|United States|-7|condo| +41599|AAAAAAAAPHCKAAAA|775|5th |Court|Suite C|Springdale|Taylor County|TX|78883|United States|-6|condo| +41600|AAAAAAAAAICKAAAA|226|2nd 6th|Circle|Suite 160|Buena Vista|Grayson County|KY|45752|United States|-6|single family| +41601|AAAAAAAABICKAAAA|827|Laurel Spruce|Drive|Suite 230|Lakeview|Callahan County|TX|78579|United States|-6|apartment| +41602|AAAAAAAACICKAAAA|777|4th |Dr.|Suite O|Willow|Lac qui Parle County|MN|56798|United States|-6|condo| +41603|AAAAAAAADICKAAAA|475|Sycamore Church|Dr.|Suite 350|Riverdale|Hampton city|VA|29391|United States|-5|condo| +41604|AAAAAAAAEICKAAAA|403|Park |||||OK||||single family| +41605|AAAAAAAAFICKAAAA|946|Pine |Pkwy|Suite 210|Fairview|Perry County|MS|55709|United States|-6|single family| +41606|AAAAAAAAGICKAAAA|817|Elm |Wy|Suite 420|Pleasant Grove|Churchill County|NV|84136|United States|-8|single family| +41607|AAAAAAAAHICKAAAA|516|Hill |Circle|Suite 330|Woodville|Washakie County|WY|84289|United States|-7|apartment| +41608|AAAAAAAAIICKAAAA|865|10th Highland|Ln|Suite 210|Arthur|Independence County|AR|75965|United States|-6|apartment| +41609|AAAAAAAAJICKAAAA|351|West Central|ST|Suite 160|Lincoln|Rio Blanco County|CO|81289|United States|-7|single family| +41610|AAAAAAAAKICKAAAA|205|Spruce |Boulevard|Suite Q|Springfield|Allamakee County|IA|59303|United States|-6|apartment| +41611|AAAAAAAALICKAAAA|395|Poplar |Avenue|Suite 210|Midway|Fayette County|IL|61904|United States|-6|single family| +41612|AAAAAAAAMICKAAAA|984|West Hillcrest|Avenue|Suite 260|Ashland|Henderson County|TX|74244|United States|-6|single family| +41613|AAAAAAAANICKAAAA|308|Adams |Pkwy|Suite D|New Hope|Henry County|VA|29431|United States|-5|apartment| +41614|AAAAAAAAOICKAAAA||Smith Center|Wy|Suite T|Newport||AR|||-6|condo| +41615|AAAAAAAAPICKAAAA|469|Maple |Street|Suite U|Springdale|Saline County|NE|68883|United States|-7|single family| +41616|AAAAAAAAAJCKAAAA|999|Sunset |Avenue|Suite P|Sleepy Hollow|Coffee County|GA|33592|United States|-5|apartment| +41617|AAAAAAAABJCKAAAA|142|Jackson |Blvd|Suite X|Maple Grove|Wise County|TX|78252|United States|-6|condo| +41618|AAAAAAAACJCKAAAA|700|Franklin |Drive|Suite 480|Farmington|Dickey County|ND|59145|United States|-6|condo| +41619|AAAAAAAADJCKAAAA|896|1st |Wy|Suite 470|Saint George|Henry County|IL|65281|United States|-6|condo| +41620|AAAAAAAAEJCKAAAA|315|2nd |Road||||||||| +41621|AAAAAAAAFJCKAAAA|381|3rd Ridge|Street|Suite U|White Oak|Pulaski County|KY|46668|United States|-5|condo| +41622|AAAAAAAAGJCKAAAA|592|1st |RD|Suite 80|Crawford|Pierce County|NE|68654|United States|-7|apartment| +41623|AAAAAAAAHJCKAAAA|543|Elm Church|RD|Suite G|Brownsville|Nash County|NC|29310|United States|-5|apartment| +41624|AAAAAAAAIJCKAAAA|244|Oak 11th|Parkway|Suite 80|Five Forks|Jackson County|TX|72293|United States|-6|condo| +41625|AAAAAAAAJJCKAAAA|232|Railroad Railroad|Wy|Suite V|Brownsville|Simpson County|KY|49310|United States|-5|condo| +41626|AAAAAAAAKJCKAAAA|691|Park |Lane|Suite H|Sherwood Forest|Starke County|IN|46702|United States|-5|apartment| +41627|AAAAAAAALJCKAAAA|850|Church Fifth|Way|Suite L|Edgewood|Green County|KY|40069|United States|-6|single family| +41628|AAAAAAAAMJCKAAAA|791|Jefferson Hill|Way|Suite A|Clifton|Laclede County|MO|68014|United States|-6|single family| +41629|AAAAAAAANJCKAAAA|650|Cherry Hillcrest|Circle|Suite F|Five Points|Bureau County|IL|66098|United States|-6|single family| +41630|AAAAAAAAOJCKAAAA|268||Court||Waterloo|Warren County|MS|||-6|| +41631|AAAAAAAAPJCKAAAA|377|North |Ave|||Stillwater County|MT|67683|United States|-7|single family| +41632|AAAAAAAAAKCKAAAA|207|South View|ST|Suite U|Edgewood|Dare County|NC|20069|United States|-5|single family| +41633|AAAAAAAABKCKAAAA|271|Elm |Ct.|Suite K|Oak Ridge|Spartanburg County|SC|28371|United States|-5|apartment| +41634|AAAAAAAACKCKAAAA|128|3rd 14th|Circle|Suite P|Woodville|Baltimore city|MD|24289|United States|-5|single family| +41635|AAAAAAAADKCKAAAA|412|7th ||Suite A||Muskingum County|||||condo| +41636|AAAAAAAAEKCKAAAA|360|Adams Dogwood|Pkwy|Suite 150|Bath|Galveston County|TX|70573|United States|-6|apartment| +41637|AAAAAAAAFKCKAAAA|347|Washington |ST|Suite 30|Greenfield|Montmorency County|MI|45038|United States|-5|condo| +41638|AAAAAAAAGKCKAAAA|928|Cedar Mill|Road|Suite 480|Shiloh|Madison County|NY|19275|United States|-5|apartment| +41639|AAAAAAAAHKCKAAAA|460|1st Valley|Lane|Suite 150|Hamilton|Sheboygan County|WI|52808|United States|-6|apartment| +41640|AAAAAAAAIKCKAAAA|86|Pine 3rd|Court|Suite 400|Fulton|Rockdale County|GA|39317|United States|-5|single family| +41641|AAAAAAAAJKCKAAAA|934|11th |Drive|Suite 240|Highland|Imperial County|CA|99454|United States|-8|condo| +41642|AAAAAAAAKKCKAAAA|324|Elm Park|Blvd|Suite L|Walnut|Racine County|WI|56245|United States|-6|apartment| +41643|AAAAAAAALKCKAAAA|963|Dogwood |Way|Suite H|Hillcrest|Lewis County|ID|83003|United States|-7|condo| +41644|AAAAAAAAMKCKAAAA|197|Church |Dr.|Suite K|Marion|Sussex County|DE|10399|United States|-5|single family| +41645|AAAAAAAANKCKAAAA||||Suite I|||MN|59145||-6|| +41646|AAAAAAAAOKCKAAAA|217|Hickory East|Circle|Suite J|Leland|Plymouth County|IA|59452|United States|-6|apartment| +41647|AAAAAAAAPKCKAAAA|758|Mill |ST|Suite 290|Lakeville|Adams County|WA|98811|United States|-8|apartment| +41648|AAAAAAAAALCKAAAA|242|Park Main|Boulevard|Suite Q|Greenfield|Park County|MT|65038|United States|-7|condo| +41649|AAAAAAAABLCKAAAA|990|Walnut College|Circle|Suite B|Spring Valley|Aiken County|SC|26060|United States|-5|single family| +41650|AAAAAAAACLCKAAAA|106|9th Eigth|RD|Suite W|Belmont|Fayette County|OH|40191|United States|-5|condo| +41651|AAAAAAAADLCKAAAA|405|Church |Road|Suite 430|Midway|Jefferson County|ID|81904|United States|-7|single family| +41652|AAAAAAAAELCKAAAA|439|2nd |Boulevard|Suite 40|Hopewell|Victoria County|TX|70587|United States|-6|single family| +41653|AAAAAAAAFLCKAAAA|684|Jackson |Circle|Suite L|Gladstone|Nevada County|CA|90894|United States|-8|apartment| +41654|AAAAAAAAGLCKAAAA|59|Smith |Dr.|Suite 180|Mount Pleasant|Crow Wing County|MN|51933|United States|-6|single family| +41655|AAAAAAAAHLCKAAAA|896|Chestnut 7th|Cir.|Suite 470|Fairview|Montague County|TX|75709|United States|-6|apartment| +41656|AAAAAAAAILCKAAAA|817|Fourth Jackson|Wy|Suite M|Franklin|Moffat County|CO|89101|United States|-7|condo| +41657|AAAAAAAAJLCKAAAA|601|Laurel |Blvd|Suite 260|Johnsonville|Kearney County|NE|67745|United States|-7|condo| +41658|AAAAAAAAKLCKAAAA|752|Maple |Court|Suite V|Oakland|Blanco County|TX|79843|United States|-6|condo| +41659|AAAAAAAALLCKAAAA|736|Lakeview Walnut|Ln|Suite K|Green Acres|Chester County|TN|37683|United States|-5|apartment| +41660|AAAAAAAAMLCKAAAA|612|Birch |Cir.|Suite Y|Stringtown|Richmond city|VA|20162|United States|-5|single family| +41661|AAAAAAAANLCKAAAA|811|||Suite 440||Seneca County||44107||-5|| +41662|AAAAAAAAOLCKAAAA|633|2nd Ninth|Boulevard|Suite 360|Plainview|Saline County|AR|73683|United States|-6|apartment| +41663|AAAAAAAAPLCKAAAA|283|Spring |Ct.|Suite 0|Lee|Scott County|VA|20408|United States|-5|single family| +41664|AAAAAAAAAMCKAAAA|747|8th 5th|ST|Suite 370|Kingston|Scott County|IN|44975|United States|-5|apartment| +41665|AAAAAAAABMCKAAAA|597|View |Street|Suite 100|Springfield|Long County|GA|39303|United States|-5|apartment| +41666|AAAAAAAACMCKAAAA|723|14th |Wy|Suite 100|Hopewell|Fillmore County|MN|50587|United States|-6|condo| +41667|AAAAAAAADMCKAAAA|372|Davis 3rd|Dr.|Suite M|Newport|Chase County|NE|61521|United States|-6|condo| +41668|AAAAAAAAEMCKAAAA|374|Chestnut |Blvd|Suite 70|Woodville|Hamblen County|TN|34289|United States|-5|condo| +41669|AAAAAAAAFMCKAAAA|686|5th |Way|Suite X|Unionville|Bucks County|PA|11711|United States|-5|single family| +41670|AAAAAAAAGMCKAAAA|742|East North|Parkway|Suite V|La Grange|Minidoka County|ID|87941|United States|-7|apartment| +41671|AAAAAAAAHMCKAAAA||11th Lake||Suite U|Sulphur Springs||||United States|-5|condo| +41672|AAAAAAAAIMCKAAAA|998|3rd Spruce|Avenue|Suite 280|Oakdale|Rooks County|KS|69584|United States|-6|apartment| +41673|AAAAAAAAJMCKAAAA|197|Valley Second|Ct.|Suite 50|Collinsville|Sebastian County|AR|72459|United States|-6|apartment| +41674|AAAAAAAAKMCKAAAA|920|Oak |RD|Suite F|Turner|Roscommon County|MI|40875|United States|-5|condo| +41675|AAAAAAAALMCKAAAA|972|Fourth Mill|Drive|Suite B|Highland|Jefferson County|GA|39454|United States|-5|single family| +41676|AAAAAAAAMMCKAAAA|165|Hillcrest |Road||Glenwood||IN||||single family| +41677|AAAAAAAANMCKAAAA|718|Jefferson Forest|Pkwy|Suite M|Spring Valley|Harrison County|KY|46060|United States|-6|condo| +41678|AAAAAAAAOMCKAAAA|103|10th |Ave|Suite 460|Belmont|Hawkins County|TN|30191|United States|-5|apartment| +41679|AAAAAAAAPMCKAAAA|754|College |Ln|Suite R|Farmington|Izard County|AR|79145|United States|-6|condo| +41680|AAAAAAAAANCKAAAA|186|Locust |Parkway|Suite 360|Sutton|York County|NE|65413|United States|-6|apartment| +41681|AAAAAAAABNCKAAAA|396|Adams 15th|Way|Suite O|Plainview|Warren County|VA|23683|United States|-5|condo| +41682|AAAAAAAACNCKAAAA|418|4th 12th|Boulevard|Suite J|Leesville|Hubbard County|MN|55423|United States|-6|apartment| +41683|AAAAAAAADNCKAAAA|633|2nd Laurel|Dr.|Suite V|Wilson|Perry County|AL|36971|United States|-6|apartment| +41684|AAAAAAAAENCKAAAA|204|Smith |Way|Suite F|Woodville|Monroe County|FL|34289|United States|-5|condo| +41685|AAAAAAAAFNCKAAAA|332|Williams Pine|Parkway|Suite 230|Spring Hill|Washington County|VA|26787|United States|-5|condo| +41686|AAAAAAAAGNCKAAAA|76|Sycamore 6th|Ave|Suite O|Friendship|Kinney County|TX|74536|United States|-6|apartment| +41687|AAAAAAAAHNCKAAAA|993|River |Boulevard|Suite K|Five Points|Allen Parish|LA|76098|United States|-6|apartment| +41688|AAAAAAAAINCKAAAA|490|Davis First|ST|Suite H|Mount Zion|Saline County|KS|68054|United States|-6|apartment| +41689|AAAAAAAAJNCKAAAA|549|Jackson |Blvd|Suite P|Shiloh|Tama County|IA|59275|United States|-6|apartment| +41690|AAAAAAAAKNCKAAAA|511|Lake Walnut|Dr.|Suite P|Five Forks|Ada County|ID|82293|United States|-7|condo| +41691|AAAAAAAALNCKAAAA|714|5th |Court|Suite 150|Maple Grove|Schleicher County|TX|78252|United States|-6|single family| +41692|AAAAAAAAMNCKAAAA|596|Poplar |Blvd|Suite 490|Mechanicsburg|Tishomingo County|MS|52219|United States|-6|single family| +41693|AAAAAAAANNCKAAAA|738|6th |Street|Suite 40|Sunnyside|Lincoln County|NE|61952|United States|-7|single family| +41694|AAAAAAAAONCKAAAA|848|Woodland |Lane|Suite 40|Shiloh|Sevier County|UT|89275|United States|-7|single family| +41695|AAAAAAAAPNCKAAAA|287|9th Madison|RD|Suite O|Bethel|Green County|KY|45281|United States|-6|single family| +41696|AAAAAAAAAOCKAAAA|405|Birch Fifth|Pkwy|Suite L|Concord|Grundy County|TN|34107|United States|-5|single family| +41697|AAAAAAAABOCKAAAA|160|3rd |ST|Suite E|New Hope|Warren County|IA|59431|United States|-6|condo| +41698|AAAAAAAACOCKAAAA|803|East |Wy|Suite G|Fairview|Faulkner County|AR|75709|United States|-6|condo| +41699|AAAAAAAADOCKAAAA|661|Jefferson |Lane|Suite V|Unionville|Greenville County|SC|21711|United States|-5|apartment| +41700|AAAAAAAAEOCKAAAA|946|Madison |RD|Suite 190|Highland Park|Tuscaloosa County|AL|36534|United States|-6|single family| +41701|AAAAAAAAFOCKAAAA|510|Park Hickory|ST|Suite B|Pine Grove|Graham County|AZ|84593|United States|-7|single family| +41702|AAAAAAAAGOCKAAAA|542|Sycamore |Ct.|Suite J|Green Acres|Clarion County|PA|17683|United States|-5|apartment| +41703|AAAAAAAAHOCKAAAA|341|4th |Drive|Suite F|Wildwood|Powder River County|MT|66871|United States|-7|single family| +41704|AAAAAAAAIOCKAAAA|894|Elm |Way|Suite 250|Cedar Grove|Kodiak Island Borough|AK|90411|United States|-9|condo| +41705|AAAAAAAAJOCKAAAA|39|4th |Dr.|Suite W|Edgewood|Saline County|MO|60069|United States|-6|condo| +41706|AAAAAAAAKOCKAAAA|190|Lee Fifteenth|Ave|Suite A|Centerville|Perkins County|SD|50059|United States|-7|apartment| +41707|AAAAAAAALOCKAAAA|83|6th Adams|Ct.|Suite A|Post Oak|Randolph County|NC|28567|United States|-5|single family| +41708|AAAAAAAAMOCKAAAA|418|Ridge Lee|Avenue|Suite 260|Pleasant Grove|Cheshire County|NH|04736|United States|-5|apartment| +41709|AAAAAAAANOCKAAAA|688|2nd Williams|Ln|Suite 310|Antioch|Bourbon County|KY|48605|United States|-6|condo| +41710|AAAAAAAAOOCKAAAA|488|Railroad |Circle|Suite 340|Hillcrest|Webb County|TX|73003|United States|-6|single family| +41711|AAAAAAAAPOCKAAAA|959|Fourth Park|Parkway|Suite 490|Five Points|Red River County|TX|76098|United States|-6|single family| +41712|AAAAAAAAAPCKAAAA|436|Birch |Street|Suite S|Springdale|Allamakee County|IA|58883|United States|-6|apartment| +41713|AAAAAAAABPCKAAAA|309|Oak Tenth|Parkway|Suite 410|Lakewood|Bath County|KY|48877|United States|-6|single family| +41714|AAAAAAAACPCKAAAA|699|Church Ash|Cir.|Suite X|Jackson|Itasca County|MN|59583|United States|-6|single family| +41715|AAAAAAAADPCKAAAA|104|Lee Broadway|Court|Suite 150|Cedar Grove|Marion County|WV|20411|United States|-5|single family| +41716|AAAAAAAAEPCKAAAA|12|13th |Ln|Suite 200|Pleasant Hill|Dillingham Census Area|AK|93604|United States|-9|single family| +41717|AAAAAAAAFPCKAAAA|703|Pine 3rd|Circle|Suite 250|Nottingham|Nicholas County|KY|44074|United States|-5|single family| +41718|AAAAAAAAGPCKAAAA|321|5th 7th|Ln|Suite X|Jackson|Morton County|ND|59583|United States|-6|single family| +41719|AAAAAAAAHPCKAAAA||||Suite W||||67057|United States||apartment| +41720|AAAAAAAAIPCKAAAA|884|3rd |Drive|Suite X|Riverview|Kershaw County|SC|29003|United States|-5|condo| +41721|AAAAAAAAJPCKAAAA|918|Valley |Avenue|Suite S|Westminster|Mitchell County|TX|76549|United States|-6|single family| +41722|AAAAAAAAKPCKAAAA|246|Main |Avenue|Suite V|Bridgeport|Freeborn County|MN|55817|United States|-6|condo| +41723|AAAAAAAALPCKAAAA|72|Jackson |Road|Suite M|Marion|Howard County|IA|50399|United States|-6|single family| +41724|AAAAAAAAMPCKAAAA|79|East Hickory|Ave|Suite A|Jackson|Buckingham County|VA|29583|United States|-5|condo| +41725|AAAAAAAANPCKAAAA|150|Dogwood Laurel|Parkway|Suite 470|Langdon|Independence County|AR|70852|United States|-6|single family| +41726|AAAAAAAAOPCKAAAA|554|Cedar |Wy|Suite 310|Union Hill|Montgomery County|KS|67746|United States|-6|single family| +41727|AAAAAAAAPPCKAAAA|411|Walnut Johnson|Cir.|Suite I|Sullivan|Avoyelles Parish|LA|70451|United States|-6|apartment| +41728|AAAAAAAAAADKAAAA|724|Fourth 3rd|Street|Suite 170|Red Hill|Hampton County|SC|24338|United States|-5|single family| +41729|AAAAAAAABADKAAAA|266|Lincoln |ST|Suite 10|Crossroads|Craven County|NC|20534|United States|-5|apartment| +41730|AAAAAAAACADKAAAA|528|15th |ST|Suite C|Glendale|Chase County|KS|63951|United States|-6|apartment| +41731|AAAAAAAADADKAAAA||13th ||Suite 370||Shelby County|OH||||| +41732|AAAAAAAAEADKAAAA|492|Main Wilson|Lane|Suite H|Lakeside|Shelby County|MO|69532|United States|-6|condo| +41733|AAAAAAAAFADKAAAA|302|Lakeview |RD|Suite 0|Ashland|Russell County|KS|64244|United States|-6|condo| +41734|AAAAAAAAGADKAAAA|424|11th |RD|Suite X|Brownsville|Hanson County|SD|59310|United States|-7|condo| +41735|AAAAAAAAHADKAAAA|873|Pine |Street|Suite T|Greenwood|Monmouth County|NJ|09428|United States|-5|condo| +41736|AAAAAAAAIADKAAAA|95|Cedar |Pkwy|Suite 120|Dallas|New York County|NY|13628|United States|-5|apartment| +41737|AAAAAAAAJADKAAAA|591|Poplar |Cir.|Suite 490|Concord|Blaine County|ID|84107|United States|-7|condo| +41738|AAAAAAAAKADKAAAA|37|Main |Parkway|Suite W|Shiloh|Huron County|OH|49275|United States|-5|condo| +41739|AAAAAAAALADKAAAA|937|5th Park|Blvd|Suite L|Union Hill|Carroll County|MD|27746|United States|-5|apartment| +41740|AAAAAAAAMADKAAAA|231|4th |Boulevard|Suite 390|Marion|Warrick County|IN|40399|United States|-5|single family| +41741|AAAAAAAANADKAAAA|92|9th |Parkway|Suite U|Farmersville|Curry County|NM|89305|United States|-7|apartment| +41742|AAAAAAAAOADKAAAA|955|Highland Fourteenth|RD|Suite 10|Cedar Grove|Jennings County|IN|40411|United States|-5|single family| +41743|AAAAAAAAPADKAAAA|961|Lake |Wy|Suite 230|Pleasant Grove|Jackson County|FL|34136|United States|-5|apartment| +41744|AAAAAAAAABDKAAAA|949|Park 8th|Dr.|Suite H|Lakeview|Leon County|FL|38579|United States|-5|single family| +41745|AAAAAAAABBDKAAAA|296|Forest Broadway|Street|Suite Y|Fairfield|Van Buren County|MI|46192|United States|-5|apartment| +41746|AAAAAAAACBDKAAAA|165|Hillcrest |Drive|Suite 260|Buena Vista|Dickson County|TN|35752|United States|-5|condo| +41747|AAAAAAAADBDKAAAA|258|Oak |Wy|Suite 460|Bethel|Lancaster County|PA|15281|United States|-5|apartment| +41748|AAAAAAAAEBDKAAAA|955|Lincoln 6th|Dr.|Suite D|Spring Hill|Fairbanks North Star Borough|AK|96787|United States|-9|single family| +41749|AAAAAAAAFBDKAAAA|486|Maple |RD|Suite D|Edgewood|Okeechobee County|FL|30069|United States|-5|single family| +41750|AAAAAAAAGBDKAAAA|540|Main |Wy|Suite 360|Cedar Grove|Hand County|SD|50411|United States|-7|apartment| +41751|AAAAAAAAHBDKAAAA|179|Spruce Laurel|Wy|Suite 0|Lakeview|Laurens County|SC|28579|United States|-5|condo| +41752|AAAAAAAAIBDKAAAA|269|South |ST|Suite 490|Peoria|Upton County|TX|79818|United States|-6|apartment| +41753|AAAAAAAAJBDKAAAA|989|5th Sunset|Boulevard|Suite E|Centerville|Washington County|AR|70059|United States|-6|apartment| +41754|AAAAAAAAKBDKAAAA|860|Jefferson |Parkway|Suite 470|Union|Kingfisher County|OK|78721|United States|-6|apartment| +41755|AAAAAAAALBDKAAAA|400|1st Spruce|Cir.|Suite 420|Brownsville|Alpena County|MI|49310|United States|-5|apartment| +41756|AAAAAAAAMBDKAAAA|66|15th |Ln|Suite L|Highland|Stevens County|MN|59454|United States|-6|single family| +41757|AAAAAAAANBDKAAAA|194|8th Meadow|Boulevard|Suite 90|Pleasant Grove|Powhatan County|VA|24136|United States|-5|single family| +41758|AAAAAAAAOBDKAAAA|842|Railroad |Drive|Suite R|Stratford|Charles Mix County|SD|56668|United States|-6|condo| +41759|AAAAAAAAPBDKAAAA|153|Ridge Ash|Ln|Suite 90|Stringtown|DeKalb County|AL|30162|United States|-6|apartment| +41760|AAAAAAAAACDKAAAA|79|11th |Dr.|Suite 490|Union Hill|Huerfano County|CO|87746|United States|-7|single family| +41761|AAAAAAAABCDKAAAA|873|River |Circle|Suite 160|White Oak|Perkins County|SD|56668|United States|-7|condo| +41762|AAAAAAAACCDKAAAA|863|Madison |Ln|Suite A|Concord|Mason County|IL|64107|United States|-6|single family| +41763|AAAAAAAADCDKAAAA|||||Sheridan||MI|47219|United States||| +41764|AAAAAAAAECDKAAAA|183|Oak |Ct.|Suite L|White Oak|Benton County|OR|96668|United States|-8|apartment| +41765|AAAAAAAAFCDKAAAA|74|11th Wilson|Pkwy|Suite 310|Lincoln|Dukes County|MA|01889|United States|-5|apartment| +41766|AAAAAAAAGCDKAAAA|864|7th |ST|Suite 250|Clinton|Bent County|CO|88222|United States|-7|condo| +41767|AAAAAAAAHCDKAAAA|412|Poplar |Road|Suite M|Union|Modoc County|CA|98721|United States|-8|single family| +41768|AAAAAAAAICDKAAAA|831|4th Oak|Pkwy|Suite E|Concord|Sweet Grass County|MT|64107|United States|-7|apartment| +41769|AAAAAAAAJCDKAAAA|580|First Chestnut|Lane|Suite F|Lakewood|Scotts Bluff County|NE|68877|United States|-7|single family| +41770|AAAAAAAAKCDKAAAA|450||||Mount Pleasant|Denver County|CO||United States|-7|| +41771|AAAAAAAALCDKAAAA|514|Park |Ave|Suite 470|Oak Ridge|Randolph County|IL|68371|United States|-6|condo| +41772|AAAAAAAAMCDKAAAA|228|Lake Lake|Ave|Suite Q|Centerville|Beaver County|UT|80059|United States|-7|apartment| +41773|AAAAAAAANCDKAAAA|506|View Laurel|Parkway|Suite 350|New Hope|Concordia Parish|LA|79431|United States|-6|apartment| +41774|AAAAAAAAOCDKAAAA|392|5th Second|Court|Suite A|Pleasant Valley|Oswego County|NY|12477|United States|-5|apartment| +41775|AAAAAAAAPCDKAAAA|66|Fifth |Boulevard|Suite 120|Lebanon|Rockingham County|NH|03498|United States|-5|single family| +41776|AAAAAAAAADDKAAAA|490|Pine |Ln|Suite 20|Sunnyside|Montmorency County|MI|41952|United States|-5|single family| +41777|AAAAAAAABDDKAAAA|323|Johnson 2nd|Way|Suite T|Oak Hill|Bethel Census Area|AK|97838|United States|-9|condo| +41778|AAAAAAAACDDKAAAA|140|Poplar |Pkwy|Suite U|Lincoln|Imperial County|CA|91289|United States|-8|condo| +41779|AAAAAAAADDDKAAAA|28|8th |Lane|Suite 490|Pleasant Grove|Plymouth County|MA|04736|United States|-5|apartment| +41780|AAAAAAAAEDDKAAAA|568|3rd |Road|Suite Y|Greenville|Logan County|NE|61387|United States|-7|apartment| +41781|AAAAAAAAFDDKAAAA|580|Adams |ST|Suite L|Glendale|Erie County|NY|13951|United States|-5|single family| +41782|AAAAAAAAGDDKAAAA|150|9th |ST|Suite W|Kimball|Montgomery County|VA|23595|United States|-5|single family| +41783|AAAAAAAAHDDKAAAA|497|Park Woodland|Ave|Suite F|Jamestown|Fulton County|AR|76867|United States|-6|condo| +41784|AAAAAAAAIDDKAAAA|320|Franklin 2nd|Street|Suite 260|Five Forks|Vigo County|IN|42293|United States|-5|condo| +41785|AAAAAAAAJDDKAAAA|652|Cherry |ST|Suite 230|Oakland|Waldo County|ME|09843|United States|-5|condo| +41786|AAAAAAAAKDDKAAAA|929|10th Washington|Circle|Suite 440|Cumberland|Bandera County|TX|78971|United States|-6|apartment| +41787|AAAAAAAALDDKAAAA|684|Willow |Avenue|Suite 480|Sunnyside|Carter County|MT|61952|United States|-7|single family| +41788|AAAAAAAAMDDKAAAA|72|Locust |Dr.|Suite 70|Welcome|McPherson County|SD|56386|United States|-7|condo| +41789|AAAAAAAANDDKAAAA|57|Main |Ct.|Suite 300|Green Acres|Augusta County|VA|27683|United States|-5|single family| +41790|AAAAAAAAODDKAAAA|967|1st Ash|Avenue|Suite L|Mount Olive|Armstrong County|TX|78059|United States|-6|apartment| +41791|AAAAAAAAPDDKAAAA|64|7th 1st|Street|Suite 270|Edgewood|Laurel County|KY|40069|United States|-5|apartment| +41792|AAAAAAAAAEDKAAAA||Third Wilson||||Union County|AR|74107|United States|-6|| +41793|AAAAAAAABEDKAAAA|12|Railroad 4th|Boulevard|Suite 430|Hillcrest|Henrico County|VA|23003|United States|-5|single family| +41794|AAAAAAAACEDKAAAA|644|Woodland |Lane|Suite 340|Maple Grove|Harper County|OK|78252|United States|-6|apartment| +41795|AAAAAAAADEDKAAAA|29|Madison Sixth|Dr.|Suite 220|New Hope|Santa Clara County|CA|99431|United States|-8|single family| +41796|AAAAAAAAEEDKAAAA|607|South Mill|Avenue|Suite 280|Galena|Pueblo County|CO|84369|United States|-7|single family| +41797|AAAAAAAAFEDKAAAA|687|Lakeview |Wy|Suite 120|Forest Hills|Middlesex County||09837||-5|condo| +41798|AAAAAAAAGEDKAAAA|703|Park |Street|Suite 420|Greenville|Cattaraugus County|NY|11387|United States|-5|apartment| +41799|AAAAAAAAHEDKAAAA|551|View Park|RD|Suite 320|Glendale|Monroe County|FL|33951|United States|-5|condo| +41800|AAAAAAAAIEDKAAAA|528|Lake |Way|Suite 90|Ashland|Wilson County|NC|24244|United States|-5|condo| +41801|AAAAAAAAJEDKAAAA|509|South |Drive|Suite 130|Glenwood|Crawford County|OH|43511|United States|-5|apartment| +41802|AAAAAAAAKEDKAAAA|628|11th |Avenue|Suite K|Freeport|Manassas Park city|VA|21844|United States|-5|apartment| +41803|AAAAAAAALEDKAAAA|947|6th |Lane|Suite 10|Lakeside|Chenango County|NY|19532|United States|-5|single family| +41804|AAAAAAAAMEDKAAAA|470|Lake |Drive|Suite 190|Centerville|Montcalm County|MI|40059|United States|-5|condo| +41805|AAAAAAAANEDKAAAA|118|West |Boulevard|Suite 480|Frogtown|Turner County|SD|58784|United States|-7|condo| +41806|AAAAAAAAOEDKAAAA|94|Green 13th|Way|Suite A|Bridgeport|Whitley County|IN|45817|United States|-5|condo| +41807|AAAAAAAAPEDKAAAA|620|Oak Locust|Court|Suite 480|Riverview|Boone County|MO|69003|United States|-6|apartment| +41808|AAAAAAAAAFDKAAAA|390|Smith |Pkwy|Suite O|Salem|Calhoun County|WV|28048|United States|-5|apartment| +41809|AAAAAAAABFDKAAAA|356|Lincoln |Dr.|Suite 40|Franklin|Marshall County|MN|59101|United States|-6|apartment| +41810|AAAAAAAACFDKAAAA|549|Williams Railroad|RD|Suite 320|Buena Vista|Vance County|NC|25752|United States|-5|single family| +41811|AAAAAAAADFDKAAAA|954|Jackson |RD|Suite Y|Five Forks|Christian County|MO|62293|United States|-6|condo| +41812|AAAAAAAAEFDKAAAA|693|Locust |Wy|Suite F|Woodlawn|Logan County|KS|64098|United States|-6|apartment| +41813|AAAAAAAAFFDKAAAA|431|Main Spring|Ln|Suite V|Fairview|Cameron County|TX|75709|United States|-6|single family| +41814|AAAAAAAAGFDKAAAA|939|Hill |Wy|Suite Y|Hillcrest|Hanover County|VA|23003|United States|-5|apartment| +41815|AAAAAAAAHFDKAAAA|302|5th |Lane|Suite 130|Oakland|Kodiak Island Borough|AK|99843|United States|-9|apartment| +41816|AAAAAAAAIFDKAAAA|140|Fifth |Circle|Suite 290|Lakeview|Jefferson County|TN|38579|United States|-6|apartment| +41817|AAAAAAAAJFDKAAAA|297|Walnut |Wy|Suite 330|Antioch|McCook County|SD|58605|United States|-7|single family| +41818|AAAAAAAAKFDKAAAA|106|Smith |Circle|Suite X|Burns|Otsego County|NY|15272|United States|-5|condo| +41819|AAAAAAAALFDKAAAA|15|Adams Center|Ln|Suite 230|Lebanon|Grundy County|IA|52898|United States|-6|single family| +41820|AAAAAAAAMFDKAAAA|132|Main Hill|Parkway|Suite 170|Georgetown|Falls County|TX|77057|United States|-6|apartment| +41821|AAAAAAAANFDKAAAA|590|Jefferson |Drive|Suite K|Acme|Walker County|AL|30164|United States|-6|condo| +41822|AAAAAAAAOFDKAAAA|940|Fifth |Avenue|Suite E|Antioch|Northwest Arctic Borough|AK|98605|United States|-9|condo| +41823|AAAAAAAAPFDKAAAA|730|Pine Smith|Drive|Suite 380|Highland|Hancock County|GA|39454|United States|-5|apartment| +41824|AAAAAAAAAGDKAAAA|236||Ave|Suite T|||WV|26971|||| +41825|AAAAAAAABGDKAAAA|813|1st East|RD|Suite 280|Mount Pleasant|Kittson County|MN|51933|United States|-6|single family| +41826|AAAAAAAACGDKAAAA|507|River 6th|Avenue|Suite 470|Maple Grove|Rogers County|OK|78252|United States|-6|single family| +41827|AAAAAAAADGDKAAAA|874|Oak |Lane|Suite 100|Crossroads|McPherson County|SD|50534|United States|-7|single family| +41828|AAAAAAAAEGDKAAAA|560|North |ST|Suite K|Red Hill|Fayette County|KY|44338|United States|-6|apartment| +41829|AAAAAAAAFGDKAAAA|917|5th Meadow|Parkway|Suite B|Sulphur Springs|White Pine County|NV|88354|United States|-8|apartment| +41830|AAAAAAAAGGDKAAAA|201|Broadway 7th|Cir.|Suite 210|Deerfield|Livingston County|MO|69840|United States|-6|apartment| +41831|AAAAAAAAHGDKAAAA|59|Williams |Lane|Suite 270|Payne|Union County|TN|36134|United States|-6|single family| +41832|AAAAAAAAIGDKAAAA|279|||Suite I||Greene County|NC|24338||-5|condo| +41833|AAAAAAAAJGDKAAAA|581|Fourth South|RD|Suite 150|Springtown|Telfair County|GA|39858|United States|-5|single family| +41834|AAAAAAAAKGDKAAAA|261|Smith |Street|Suite 160|Greenfield|Grand Traverse County|MI|45038|United States|-5|condo| +41835|AAAAAAAALGDKAAAA|348|Johnson First|RD|Suite Q|Fairview|Greene County|NY|15709|United States|-5|single family| +41836|AAAAAAAAMGDKAAAA|550|Ridge |Pkwy|Suite Q|Newport|Crawford County|IN|41521|United States|-5|single family| +41837|AAAAAAAANGDKAAAA|232||Road|Suite S|New Hope||||United States|-6|single family| +41838|AAAAAAAAOGDKAAAA|110|Hickory 9th|Blvd|Suite 270|Green Acres|Yates County|NY|17683|United States|-5|condo| +41839|AAAAAAAAPGDKAAAA|399|Ridge |Pkwy|Suite C|Centerville|Jackson County|AR|70059|United States|-6|single family| +41840|AAAAAAAAAHDKAAAA|548|Cedar 5th|Ct.|Suite H|Concord|Winkler County|TX|74107|United States|-6|single family| +41841|AAAAAAAABHDKAAAA|593|Lake |Circle|Suite Q|Five Forks|Gove County|KS|62293|United States|-6|condo| +41842|AAAAAAAACHDKAAAA|932|Lee |Lane|Suite 0|Greenville|Tipton County|IN|41387|United States|-5|apartment| +41843|AAAAAAAADHDKAAAA|941|Madison Davis|Circle|Suite Y|Brownsville|Johnston County|NC|29310|United States|-5|single family| +41844|AAAAAAAAEHDKAAAA|688|Church Pine|ST|Suite I|Belmont|Hopkins County|KY|40191|United States|-6|single family| +41845|AAAAAAAAFHDKAAAA|554|Davis |Boulevard|Suite 460|Lincoln|Preston County|WV|21289|United States|-5|single family| +41846|AAAAAAAAGHDKAAAA|99|Eigth 1st|Boulevard|Suite A|Woodland|Nelson County|KY|44854|United States|-5|condo| +41847|AAAAAAAAHHDKAAAA|410|7th Maple|ST|Suite U|Summit|Meriwether County|GA|30499|United States|-5|apartment| +41848|AAAAAAAAIHDKAAAA|311|Elm Birch|Parkway|Suite Q|Oneida|Brown County|KS|64027|United States|-6|single family| +41849|AAAAAAAAJHDKAAAA|776|Valley |ST|Suite 130|Hillcrest|Breathitt County|KY|43003|United States|-6|single family| +41850|AAAAAAAAKHDKAAAA|124|Maple |Blvd|Suite N|Greenwood|Ochiltree County|TX|78828|United States|-6|single family| +41851|AAAAAAAALHDKAAAA|329|Seventh 13th|Pkwy|Suite 490|Burns|Clinton County|NY|15272|United States|-5|apartment| +41852|AAAAAAAAMHDKAAAA|745|Oak Washington|Court|Suite H|Riverview|Bates County|MO|69003|United States|-6|single family| +41853|AAAAAAAANHDKAAAA|927|Miller 8th|Lane|Suite 70|Clifton|Little River County|AR|78014|United States|-6|apartment| +41854|AAAAAAAAOHDKAAAA|961|Broadway |Dr.|Suite D|Lakeview|Mitchell County|NC|28579|United States|-5|apartment| +41855|AAAAAAAAPHDKAAAA|313|Fifth |Road|Suite 470|Fairview|Carter County|MT|65709|United States|-7|single family| +41856|AAAAAAAAAIDKAAAA|267|Williams |Boulevard|Suite I|Lincoln|Archer County|TX|71289|United States|-6|apartment| +41857|AAAAAAAABIDKAAAA|564|Lincoln |Wy|Suite 310|Spring Valley|Davie County|NC|26060|United States|-5|apartment| +41858|AAAAAAAACIDKAAAA|425|Maple Valley|Ave|Suite 200|Shore Acres|Stark County|OH|42724|United States|-5|apartment| +41859|AAAAAAAADIDKAAAA|494|Cedar South|Dr.|Suite 440|Superior|Oneida County|WI|52562|United States|-6|condo| +41860|AAAAAAAAEIDKAAAA|395|View |Circle|Suite P|Friendship|Lincoln County|MO|64536|United States|-6|condo| +41861|AAAAAAAAFIDKAAAA|655|1st |Drive|Suite K|Sulphur Springs|Daviess County|KY|48354|United States|-6|single family| +41862|AAAAAAAAGIDKAAAA|975|Fourth Hill|Court|Suite 220|Pleasant Valley|Union County|OR|92477|United States|-8|single family| +41863|AAAAAAAAHIDKAAAA|635|Lakeview |Avenue|Suite 440|Plainview|Butte County|SD|53683|United States|-6|condo| +41864|AAAAAAAAIIDKAAAA|394|5th Birch|Ave|Suite 30|Pine Grove|Jackson Parish|LA|74593|United States|-6|apartment| +41865|AAAAAAAAJIDKAAAA|296|Chestnut Green|Court|Suite K|Hamilton|Hardin County|TN|32808|United States|-5|apartment| +41866|AAAAAAAAKIDKAAAA|479|Highland 4th|Parkway|Suite B|Pine Grove|Jenkins County|GA|34593|United States|-5|condo| +41867|AAAAAAAALIDKAAAA|528|5th |Way|Suite G|Mount Zion|Lancaster County|SC|28054|United States|-5|apartment| +41868|AAAAAAAAMIDKAAAA|645|Elm |RD|Suite 220|Pleasant Valley|Grayson County|TX|72477|United States|-6|condo| +41869|AAAAAAAANIDKAAAA|504|Second |Cir.|Suite Y|Unionville|Shelby County|IL|61711|United States|-6|condo| +41870|AAAAAAAAOIDKAAAA|590|Walnut |Lane|Suite X|Richville|Jones County|IA|55945|United States|-6|apartment| +41871|AAAAAAAAPIDKAAAA|387|Main |Ave|Suite V|Riverdale|Webb County|TX|79391|United States|-6|apartment| +41872|AAAAAAAAAJDKAAAA|558|Main South|Street|Suite G|Brownsville|Calhoun County|IA|59310|United States|-6|apartment| +41873|AAAAAAAABJDKAAAA|114|12th Johnson|Ave|Suite D|Stratford|Ware County|GA|36668|United States|-5|single family| +41874|AAAAAAAACJDKAAAA|599|Elm |Pkwy|Suite K|Spring Hill|Wyoming County|NY|16787|United States|-5|condo| +41875|AAAAAAAADJDKAAAA|513|10th Lake|Ln|Suite T|Greenfield|Walthall County|MS|55038|United States|-6|condo| +41876|AAAAAAAAEJDKAAAA|285|10th 5th|Way|Suite L|Mount Vernon|Monroe County|IA|58482|United States|-6|single family| +41877|AAAAAAAAFJDKAAAA|517|2nd |Ave|Suite O|Jamestown|Murray County|OK|76867|United States|-6|apartment| +41878|AAAAAAAAGJDKAAAA|193|Cherry |RD|Suite 320|Lakewood|Latimer County|OK|78877|United States|-6|condo| +41879|AAAAAAAAHJDKAAAA|146|Hillcrest Locust|Street|Suite 440|Clifton|Manassas city|VA|28014|United States|-5|apartment| +41880|AAAAAAAAIJDKAAAA|366|Jackson Meadow|Boulevard|Suite 300|Mountain View|Platte County|MO|64466|United States|-6|condo| +41881|AAAAAAAAJJDKAAAA|550|Maple |Wy|Suite B|Hopewell|Chaves County|NM|80587|United States|-7|condo| +41882|AAAAAAAAKJDKAAAA|898|South Woodland|Court|Suite 310|West Liberty|Randolph County|GA|34752|United States|-5|apartment| +41883|AAAAAAAALJDKAAAA|154|Hillcrest |Cir.|Suite T|Marion|Hampden County|MA|00999|United States|-5|apartment| +41884|AAAAAAAAMJDKAAAA|53|3rd |Lane|Suite F|Maple Grove|Rabun County|GA|38252|United States|-5|condo| +41885|AAAAAAAANJDKAAAA|592|6th Center|Way|Suite X|Cedar Grove|Dakota County|NE|60411|United States|-6|apartment| +41886|AAAAAAAAOJDKAAAA|81|7th 4th|Cir.|Suite 40|Woodlawn|Midland County|TX|74098|United States|-6|apartment| +41887|AAAAAAAAPJDKAAAA|294|East 2nd|Street|Suite E|Hillcrest|Yuma County|AZ|83003|United States|-7|single family| +41888|AAAAAAAAAKDKAAAA|251|Hickory |Circle|Suite 170|Oak Grove|Davis County|UT|88370|United States|-7|apartment| +41889|AAAAAAAABKDKAAAA|389|1st Dogwood|Court|Suite 120|Fairfield|Tangipahoa Parish|LA|76192|United States|-6|apartment| +41890|AAAAAAAACKDKAAAA|547|Lakeview Sunset|Wy|Suite L|Summit|Sussex County|VA|20499|United States|-5|apartment| +41891|AAAAAAAADKDKAAAA|814|Locust |RD|Suite Q|Fairfield|Floyd County|KY|46192|United States|-6|condo| +41892|AAAAAAAAEKDKAAAA|561|1st Adams|Parkway|Suite S|Valley View|Iron County|MI|45124|United States|-5|single family| +41893|AAAAAAAAFKDKAAAA||15th |Lane|Suite W|||NJ||United States|-5|| +41894|AAAAAAAAGKDKAAAA|84|Adams |Avenue|Suite L|Webb|Oakland County|MI|40899|United States|-5|apartment| +41895|AAAAAAAAHKDKAAAA|643|Smith |Cir.|Suite 240|Wilson|Larimer County|CO|86971|United States|-7|condo| +41896|AAAAAAAAIKDKAAAA||Johnson ||||||96787||-8|single family| +41897|AAAAAAAAJKDKAAAA|403|Hill West|Way|Suite C|Wilson|Red River County|TX|76971|United States|-6|condo| +41898|AAAAAAAAKKDKAAAA|618|Oak Third|Dr.|Suite 50|Summit|Sullivan County|MO|60499|United States|-6|single family| +41899|AAAAAAAALKDKAAAA|285|Mill Jefferson|ST|Suite L|Woodruff|Ingham County|MI|44174|United States|-5|apartment| +41900|AAAAAAAAMKDKAAAA|370|Mill |RD|Suite 310|Five Points|Iron County|MI|46098|United States|-5|single family| +41901|AAAAAAAANKDKAAAA|76|Main West|Lane|Suite S|Crystal|Wayne County|IN|45258|United States|-5|apartment| +41902|AAAAAAAAOKDKAAAA|340|11th |Court|Suite D|Oakdale|Pamlico County|NC|29584|United States|-5|apartment| +41903|AAAAAAAAPKDKAAAA|515|Jackson |Boulevard|Suite J|Forest Hills|Nome Census Area|AK|99237|United States|-9|apartment| +41904|AAAAAAAAALDKAAAA|352|12th |Dr.|Suite 70|Greenville|Lenoir County|NC|21387|United States|-5|single family| +41905|AAAAAAAABLDKAAAA|266|Ridge |Way|Suite A|Shady Grove|Bates County|MO|62812|United States|-6|apartment| +41906|AAAAAAAACLDKAAAA|828|Church Pine||Suite V|Bath|Ingham County|||United States|-5|single family| +41907|AAAAAAAADLDKAAAA|384|4th Park|Street|Suite J|Newtown|Pike County|OH|41749|United States|-5|single family| +41908|AAAAAAAAELDKAAAA|200|Railroad |RD|Suite 40|Highland Park|Lincoln County|TN|36534|United States|-6|apartment| +41909|AAAAAAAAFLDKAAAA|479|Tenth Jackson|Lane|Suite Q|Concord|North Slope Borough|AK|94107|United States|-9|single family| +41910|AAAAAAAAGLDKAAAA|||||Barnes|||63788|United States|-6|condo| +41911|AAAAAAAAHLDKAAAA|465|Spring |Avenue|Suite 240|Deerfield|Macon County|GA|39840|United States|-5|single family| +41912|AAAAAAAAILDKAAAA|70|Fourth 12th|Cir.|Suite 190|Oak Grove|Crawford County|IL|68370|United States|-6|condo| +41913|AAAAAAAAJLDKAAAA|755|Maple Hickory|Avenue|Suite 30|Sulphur Springs|Dodge County|WI|58354|United States|-6|condo| +41914|AAAAAAAAKLDKAAAA|917|Jackson Oak|Boulevard|Suite R|Bunker Hill|Lancaster County|VA|20150|United States|-5|apartment| +41915|AAAAAAAALLDKAAAA|579|2nd 5th|Court|Suite U|Georgetown|Hancock County|WV|27057|United States|-5|single family| +41916|AAAAAAAAMLDKAAAA|998|Ridge Main|||Jackson||||||| +41917|AAAAAAAANLDKAAAA|862|Broadway 11th|Parkway|Suite M|Union|Yukon-Koyukuk Census Area|AK|98721|United States|-9|single family| +41918|AAAAAAAAOLDKAAAA|974|South |Dr.|Suite 70|Newport|Catahoula Parish|LA|71521|United States|-6|apartment| +41919|AAAAAAAAPLDKAAAA|197|Willow |RD|Suite G|Union Hill|Greene County|MO|67746|United States|-6|condo| +41920|AAAAAAAAAMDKAAAA|565|Elm 11th|Lane|Suite 0|Crossroads|Lewis County|NY|10534|United States|-5|apartment| +41921|AAAAAAAABMDKAAAA|542|Maple Walnut|Circle|Suite 340|Lakeview|Lanier County|GA|38579|United States|-5|apartment| +41922|AAAAAAAACMDKAAAA|853|4th Walnut|Ct.|Suite N|Riverview|Mason County|KY|49003|United States|-5|condo| +41923|AAAAAAAADMDKAAAA|860|Washington |Way|Suite 470|Summit|Platte County|NE|60499|United States|-7|single family| +41924|AAAAAAAAEMDKAAAA|59|4th Highland|Blvd|Suite O|Proctor|York County|SC|28140|United States|-5|condo| +41925|AAAAAAAAFMDKAAAA|706|Chestnut Lincoln|Ave|Suite 250|Plainview|Marshall County|AL|33683|United States|-6|condo| +41926|AAAAAAAAGMDKAAAA|3|13th |Parkway|Suite 490|Red Hill|Alleghany County|NC|24338|United States|-5|single family| +41927|AAAAAAAAHMDKAAAA|335|East |Ave|Suite K|Greenville|Wicomico County|MD|21387|United States|-5|apartment| +41928|AAAAAAAAIMDKAAAA|736|Poplar |Ln|Suite 30|Plainview|Roanoke city|VA|23683|United States|-5|condo| +41929|AAAAAAAAJMDKAAAA|280|Central Broadway|Wy|Suite 270|Green Acres|Muskegon County|MI|47683|United States|-5|condo| +41930|AAAAAAAAKMDKAAAA|277|Hillcrest 5th|Road|Suite 50|Oakwood|Isle of Wight County|VA|20169|United States|-5|single family| +41931|AAAAAAAALMDKAAAA|569|9th Miller|ST|Suite D|Bridgeport|Norton County|KS|65817|United States|-6|apartment| +41932|AAAAAAAAMMDKAAAA|81|Madison Elm|Avenue|Suite X|Riverview|Miner County|SD|59003|United States|-7|single family| +41933|AAAAAAAANMDKAAAA|404|Birch |Blvd|Suite 330|Mount Zion|Licking County|OH|48054|United States|-5|single family| +41934|AAAAAAAAOMDKAAAA|258|Highland 13th|Boulevard|Suite A|Sunnyside|New Hanover County|NC|21952|United States|-5|single family| +41935|AAAAAAAAPMDKAAAA|268|Maple 6th|ST|Suite I|Johnstown|Martin County|TX|79785|United States|-6|single family| +41936|AAAAAAAAANDKAAAA|482|Johnson |Ln|Suite 430|Ridgeville|Comanche County|TX|79306|United States|-6|apartment| +41937|AAAAAAAABNDKAAAA|747|Ninth |Boulevard|Suite T|Richfield|Chautauqua County|KS|66196|United States|-6|apartment| +41938|AAAAAAAACNDKAAAA|158|Forest First|Court|Suite T|Mount Pleasant|Buchanan County|IA|51933|United States|-6|apartment| +41939|AAAAAAAADNDKAAAA|450|Sixth 2nd|Ave|Suite C|Point Pleasant|Searcy County|AR|74749|United States|-6|condo| +41940|AAAAAAAAENDKAAAA|493|9th Cedar|Way|Suite L|Ashland|Lee County|FL|34244|United States|-5|apartment| +41941|AAAAAAAAFNDKAAAA|668||ST|||Montgomery County||64593|||| +41942|AAAAAAAAGNDKAAAA|270|Church |RD|Suite 480|Highland Park|Terrell County|GA|36534|United States|-5|single family| +41943|AAAAAAAAHNDKAAAA|853|Green Main|Ln|Suite 390|Oakdale|Spokane County|WA|99584|United States|-8|apartment| +41944|AAAAAAAAINDKAAAA|994|River |Pkwy|Suite 160|Spring Hill|Boone County|MO|66787|United States|-6|single family| +41945|AAAAAAAAJNDKAAAA|685|Locust |ST|Suite O|Harvey|Langlade County|WI|55858|United States|-6|apartment| +41946|AAAAAAAAKNDKAAAA|133|Jefferson Railroad|Circle|Suite T|Highland Park|Boise County|ID|86534|United States|-7|single family| +41947|AAAAAAAALNDKAAAA|477|Third Main|Cir.|Suite 430|Fox|Nueces County|TX|70631|United States|-6|condo| +41948|AAAAAAAAMNDKAAAA|115|Forest |Dr.|Suite 230|Mountain View|Lawrence County|IN|44466|United States|-5|single family| +41949|AAAAAAAANNDKAAAA|318|Main 4th|Parkway|Suite J|Fairbanks|Brooke County|WV|26653|United States|-5|single family| +41950|AAAAAAAAONDKAAAA|292|Woodland |Road|Suite 370|Lakeview|Hopkins County|TX|78579|United States|-6|apartment| +41951|AAAAAAAAPNDKAAAA|956|Birch |Dr.|Suite O|Stringtown|Menard County|TX|70162|United States|-6|condo| +41952|AAAAAAAAAODKAAAA|194|Sunset |Street|Suite F|Ashland|Sangamon County|IL|64244|United States|-6|condo| +41953|AAAAAAAABODKAAAA|154|Wilson |Road|Suite G|Smith|Evans County|GA|37317|United States|-5|single family| +41954|AAAAAAAACODKAAAA|959|Fourth Jefferson||Suite N|Glendale|||93951|United States||| +41955|AAAAAAAADODKAAAA|596|Laurel Miller|Dr.|Suite 130|Centerville|Gray County|KS|60059|United States|-6|single family| +41956|AAAAAAAAEODKAAAA|565|Dogwood |Road|Suite 460|Woodlawn|Comanche County|KS|64098|United States|-6|condo| +41957|AAAAAAAAFODKAAAA|949|Green |Wy|Suite 310|Midway|Pendleton County|KY|41904|United States|-5|single family| +41958|AAAAAAAAGODKAAAA|354|Park |Dr.|Suite W|Midway|Butler County|KS|61904|United States|-6|single family| +41959|AAAAAAAAHODKAAAA|||Drive|Suite K|Woodlawn|Fairfield County|SC||United States||| +41960|AAAAAAAAIODKAAAA|703|River 1st|Lane|Suite X|Sunnyside|Kay County|OK|71952|United States|-6|apartment| +41961|AAAAAAAAJODKAAAA|241|Main |Cir.|Suite S|Providence|Emery County|UT|86614|United States|-7|condo| +41962|AAAAAAAAKODKAAAA|948|Oak |Dr.|Suite 180|Lakeview|Delta County|MI|48579|United States|-5|condo| +41963|AAAAAAAALODKAAAA|835|Lake |Cir.|Suite L|Riverdale|Somervell County|TX|79391|United States|-6|apartment| +41964|AAAAAAAAMODKAAAA|202|Pine 4th|Circle|Suite 270|Kingston|Summit County|CO|84975|United States|-7|apartment| +41965|AAAAAAAANODKAAAA|48|View 4th|Wy|Suite Y|Clinton|Coffee County|GA|38222|United States|-5|single family| +41966|AAAAAAAAOODKAAAA|28|Park Ridge|Pkwy|Suite W|Arlington|Bond County|IL|66557|United States|-6|condo| +41967|AAAAAAAAPODKAAAA|937|Pine Hill|RD|Suite O|Franklin|Phelps County|NE|69101|United States|-7|apartment| +41968|AAAAAAAAAPDKAAAA|169|West |Ave|Suite X|Kingston|Genesee County|NY|14975|United States|-5|single family| +41969|AAAAAAAABPDKAAAA|561|14th |Dr.|Suite 140|Hillcrest|Terrell County|GA|33003|United States|-5|single family| +41970|AAAAAAAACPDKAAAA|73|Seventh |Road|Suite 330|Ashland|Escambia County|FL|34244|United States|-5|single family| +41971|AAAAAAAADPDKAAAA|147|Lakeview 5th|Dr.|Suite I|Bethel|Mecklenburg County|NC|25281|United States|-5|condo| +41972|AAAAAAAAEPDKAAAA|301|Dogwood |RD|Suite L|Valley View|Pitt County|NC|25124|United States|-5|apartment| +41973|AAAAAAAAFPDKAAAA|476|14th |Ln|Suite 360|Lebanon|Lincoln County|TN|32898|United States|-6|single family| +41974|AAAAAAAAGPDKAAAA||4th Church|Pkwy||||AR||||apartment| +41975|AAAAAAAAHPDKAAAA|104|14th |Road|Suite Q|Sulphur Springs|Robertson County|TX|78354|United States|-6|single family| +41976|AAAAAAAAIPDKAAAA|182|Maple |ST|Suite 290|Lakeview|Platte County|MO|68579|United States|-6|apartment| +41977|AAAAAAAAJPDKAAAA|966|5th |Boulevard|Suite 160|Walnut Grove|Somerset County|PA|17752|United States|-5|condo| +41978|AAAAAAAAKPDKAAAA|885|1st |Circle|Suite 310|Kingston|Yolo County|CA|94975|United States|-8|condo| +41979|AAAAAAAALPDKAAAA|345|Poplar Second|Street|Suite 420|Pleasant Grove|Washington County|ME|04736|United States|-5|single family| +41980|AAAAAAAAMPDKAAAA|566|Elm Washington|RD|Suite N|Enterprise|McPherson County|KS|61757|United States|-6|condo| +41981|AAAAAAAANPDKAAAA|409|Cedar Jefferson|Street|Suite T|Walnut Grove|Jefferson County|TN|37752|United States|-6|condo| +41982|AAAAAAAAOPDKAAAA|977|3rd Hill|Dr.|Suite V|Pleasant Grove|Okfuskee County|OK|74136|United States|-6|single family| +41983|AAAAAAAAPPDKAAAA|584|9th Pine|Boulevard|Suite L|Five Points|Stone County|MO|66098|United States|-6|apartment| +41984|AAAAAAAAAAEKAAAA|598|2nd |Boulevard|Suite I|Shore Acres|Stewart County|TN|32724|United States|-6|single family| +41985|AAAAAAAABAEKAAAA|237|Central |Ave|Suite E|Oakwood|Jefferson County|MT|60169|United States|-7|single family| +41986|AAAAAAAACAEKAAAA|138|Hill |Wy|Suite 220|Newtown|Racine County|WI|51749|United States|-6|single family| +41987|AAAAAAAADAEKAAAA|720|Cherry Birch|Street|Suite L|Waterloo|Polk County|AR|71675|United States|-6|single family| +41988|AAAAAAAAEAEKAAAA|671|Franklin |Drive|Suite 430|Florence|Crawford County|MI|43394|United States|-5|single family| +41989|AAAAAAAAFAEKAAAA|665|Williams |Parkway|Suite 150|Five Points|Berks County|PA|16098|United States|-5|single family| +41990|AAAAAAAAGAEKAAAA|636|Poplar |Drive|Suite G|Forest Hills|Trego County|KS|69237|United States|-6|single family| +41991|AAAAAAAAHAEKAAAA|671|Main |Dr.|Suite 140|Greenville|Hayes County|NE|61387|United States|-6|apartment| +41992|AAAAAAAAIAEKAAAA|895|Center Hill|Blvd|Suite 300|Gravel Hill|Cimarron County|OK|71944|United States|-6|single family| +41993|AAAAAAAAJAEKAAAA|531|Dogwood |Avenue|Suite L|Pleasant Hill|Dewey County|SD|53604|United States|-6|single family| +41994|AAAAAAAAKAEKAAAA|819|5th |Blvd|Suite U|Appleton|Pembina County|ND|54240|United States|-6|single family| +41995|AAAAAAAALAEKAAAA|205|5th Smith|Parkway|Suite 150|Red Hill|Claiborne County|MS|54338|United States|-6|single family| +41996|AAAAAAAAMAEKAAAA|777|North Williams|Boulevard|Suite 110|Kingston|Lane County|OR|94975|United States|-8|condo| +41997|AAAAAAAANAEKAAAA|425|10th Lakeview|Court|Suite I|Springdale|Montgomery County|MD|28883|United States|-5|single family| +41998|AAAAAAAAOAEKAAAA|755|Mill |ST|Suite W|Bethel|Douglas County|OR|95281|United States|-8|single family| +41999|AAAAAAAAPAEKAAAA|906|Third |ST|Suite 150|Hillcrest|Tippecanoe County|IN|43003|United States|-5|apartment| +42000|AAAAAAAAABEKAAAA|914|Willow 6th|Circle|Suite 250|Riverview|Robertson County|TX|79003|United States|-6|apartment| +42001|AAAAAAAABBEKAAAA|699|Cedar 14th|Avenue|Suite A|Plainview|Woodruff County|AR|73683|United States|-6|apartment| +42002|AAAAAAAACBEKAAAA|909|Forest |Avenue|Suite 340|Glendale|Santa Fe County|NM|83951|United States|-7|apartment| +42003|AAAAAAAADBEKAAAA|348|11th Railroad|Blvd|Suite K|Newtown|Meade County|SD|51749|United States|-7|condo| +42004|AAAAAAAAEBEKAAAA|163|1st |Court|Suite E|Sulphur Springs|Montrose County|CO|88354|United States|-7|apartment| +42005|AAAAAAAAFBEKAAAA|375|Madison |Ct.|Suite M|Stringtown|Franklin County|NY|10162|United States|-5|apartment| +42006|AAAAAAAAGBEKAAAA||||Suite 170|Maple Grove|Willacy County|TX||United States|-6|| +42007|AAAAAAAAHBEKAAAA|784|3rd |Dr.|Suite N|Riverside|Hardeman County|TX|79231|United States|-6|condo| +42008|AAAAAAAAIBEKAAAA|762|Hill Hickory|Pkwy|Suite F|Midway|Huntingdon County|PA|11904|United States|-5|apartment| +42009|AAAAAAAAJBEKAAAA|79|Ash |Pkwy|Suite A|Lebanon|Clay County|IA|52898|United States|-6|single family| +42010|AAAAAAAAKBEKAAAA|492|Lincoln 3rd|Blvd|Suite 10|Stringtown|Perry County|MO|60162|United States|-6|condo| +42011|AAAAAAAALBEKAAAA|244|Jefferson Main|ST|Suite 10|Clinton|Wayne County|IN|48222|United States|-5|condo| +42012|AAAAAAAAMBEKAAAA|325|6th |RD|Suite 420|Hamilton|Meade County|KS|62808|United States|-6|apartment| +42013|AAAAAAAANBEKAAAA|361|7th |Boulevard|Suite 110|Hazelwood|Lincoln County|AR|71206|United States|-6|condo| +42014|AAAAAAAAOBEKAAAA|501|Railroad |Avenue|Suite X|Five Forks|Jackson County|IL|62293|United States|-6|condo| +42015|AAAAAAAAPBEKAAAA|960|5th Walnut|Dr.|Suite R|Woodville|Clark County|IN|44289|United States|-5|single family| +42016|AAAAAAAAACEKAAAA|621|Railroad |Blvd|Suite 150|Springdale|Buckingham County|VA|28883|United States|-5|condo| +42017|AAAAAAAABCEKAAAA|242|Willow |Court|Suite B|Mount Zion|Woodson County|KS|68054|United States|-6|single family| +42018|AAAAAAAACCEKAAAA|252|5th |Drive|Suite S|Marion|Penobscot County|ME|00999|United States|-5|apartment| +42019|AAAAAAAADCEKAAAA|58|West Broadway|Lane|Suite Q|Salem|Day County|SD|58048|United States|-6|condo| +42020|AAAAAAAAECEKAAAA|991|Hickory Cedar||Suite 430|Buena Vista||NH||||| +42021|AAAAAAAAFCEKAAAA|895|Maple Cedar|Court|Suite W|Hillcrest|Martin County|FL|33003|United States|-5|apartment| +42022|AAAAAAAAGCEKAAAA|908|1st 10th|Street|Suite F|Summit|Beltrami County|MN|50499|United States|-6|condo| +42023|AAAAAAAAHCEKAAAA|235|Davis |Ave|Suite A|Sulphur Springs|Iberville Parish|LA|78354|United States|-6|condo| +42024|AAAAAAAAICEKAAAA|804|Fifth Maple|Blvd|Suite 80|Salem|Aiken County|SC|28048|United States|-5|apartment| +42025|AAAAAAAAJCEKAAAA|677|3rd |Blvd|Suite 240|Centerville|Glascock County|GA|30059|United States|-5|condo| +42026|AAAAAAAAKCEKAAAA|131|Lake ||Suite R|Wilson|Heard County|GA|36971|United States|-5|single family| +42027|AAAAAAAALCEKAAAA|287|10th Center|RD|Suite E|Sheffield|Concho County|TX|76896|United States|-6|condo| +42028|AAAAAAAAMCEKAAAA|990|Green 14th|Ct.|Suite 350|Highland|Gooding County|ID|89454|United States|-7|apartment| +42029|AAAAAAAANCEKAAAA|330|Smith |Road|Suite Q|Oakdale|Keokuk County|IA|59584|United States|-6|apartment| +42030|AAAAAAAAOCEKAAAA|821|Highland |Cir.|Suite Q|Buena Vista|Noble County|OH|45752|United States|-5|condo| +42031|AAAAAAAAPCEKAAAA|291|13th |Court|Suite 280|Bridgeport|Union County|IN|45817|United States|-5|apartment| +42032|AAAAAAAAADEKAAAA|467|River |Wy|Suite 50|Green Acres|Posey County|IN|47683|United States|-5|apartment| +42033|AAAAAAAABDEKAAAA|654|Main |Boulevard|Suite 450|Pine Grove|Houghton County|MI|44593|United States|-5|apartment| +42034|AAAAAAAACDEKAAAA|73|Center Green|Street|Suite N|Shady Grove|Manassas city|VA|22812|United States|-5|single family| +42035|AAAAAAAADDEKAAAA|285|10th |Street|Suite 380|Pleasant Hill|Clark County|IN|43604|United States|-5|condo| +42036|AAAAAAAAEDEKAAAA|690|Lakeview |Street|Suite X|Woodville|Wirt County|WV|24289|United States|-5|condo| +42037|AAAAAAAAFDEKAAAA|833|Ninth |Dr.|Suite A|Woodland|Lee County|GA|34854|United States|-5|single family| +42038|AAAAAAAAGDEKAAAA|953|Valley |Road|Suite 160|Woodville|Lancaster County|PA|14289|United States|-5|single family| +42039|AAAAAAAAHDEKAAAA|263|Maple |Way|Suite 70|Mount Zion|Hardin County|KY|48054|United States|-6|single family| +42040|AAAAAAAAIDEKAAAA|801|Smith Hill|Lane|Suite 210|Bridgeport|Dinwiddie County|VA|25817|United States|-5|apartment| +42041|AAAAAAAAJDEKAAAA|367|Fifth |Dr.|Suite P|Riverdale|Cameron County|TX|79391|United States|-6|condo| +42042|AAAAAAAAKDEKAAAA|503|Willow |Pkwy|Suite K|Fairfield|Crawford County|MO|66192|United States|-6|condo| +42043|AAAAAAAALDEKAAAA|382|North Third|Parkway|Suite G|Salem|Hamilton County|IL|68048|United States|-6|condo| +42044|AAAAAAAAMDEKAAAA|94||||||CO|82812|United States||| +42045|AAAAAAAANDEKAAAA|510|Oak Cedar|Ave|Suite 80|Ryan|Wright County|IA|50525|United States|-6|apartment| +42046|AAAAAAAAODEKAAAA|27|Jefferson Maple|RD|Suite Y|Willis|Scott County|VA|26788|United States|-5|condo| +42047|AAAAAAAAPDEKAAAA|440|Forest 10th|Street|Suite 0|Waterloo|Portage County|OH|41675|United States|-5|single family| +42048|AAAAAAAAAEEKAAAA|474|Ash |Ave|Suite 460|Clearview|Fulton County|GA|35495|United States|-5|single family| +42049|AAAAAAAABEEKAAAA|27|Hickory Madison|ST|Suite 140|Fairview|Suffolk County|NY|15709|United States|-5|apartment| +42050|AAAAAAAACEEKAAAA|985|Hill 1st|Avenue|Suite 460|Bethel|Pulaski County|IN|45281|United States|-5|condo| +42051|AAAAAAAADEEKAAAA|551|3rd 12th|Street|Suite D|Oak Ridge|Marathon County|WI|58371|United States|-6|single family| +42052|AAAAAAAAEEEKAAAA|278|Jackson Hillcrest|Road|Suite 60|Saint James|Knox County|KY|45799|United States|-5|apartment| +42053|AAAAAAAAFEEKAAAA|352|River |Lane|Suite U|West Liberty|Colonial Heights city|VA|24752|United States|-5|condo| +42054|AAAAAAAAGEEKAAAA|750|Jefferson |Ct.|Suite 270|Fairview|Piute County|UT|85709|United States|-7|condo| +42055|AAAAAAAAHEEKAAAA|319|3rd Central|RD|Suite 430|Spring Valley|Winneshiek County|IA|56060|United States|-6|apartment| +42056|AAAAAAAAIEEKAAAA|607|Lincoln |Avenue|Suite E|Marion|Vance County|NC|20399|United States|-5|apartment| +42057|AAAAAAAAJEEKAAAA|685|View 1st|Pkwy|Suite 50|Hopewell|Grayson County|TX|70587|United States|-6|condo| +42058|AAAAAAAAKEEKAAAA|905|Ash |Avenue|Suite 310|Pleasant Grove|Shenandoah County|VA|24136|United States|-5|apartment| +42059|AAAAAAAALEEKAAAA||6th |Cir.|Suite 340|Enterprise||OR||United States|-8|single family| +42060|AAAAAAAAMEEKAAAA|530|Oak |Drive|Suite 480||Adams County|||United States|-6|| +42061|AAAAAAAANEEKAAAA|340|Willow View|Court|Suite 440|Valley View|Washington County|IN|45124|United States|-5|apartment| +42062|AAAAAAAAOEEKAAAA|50|1st |Ave|Suite 160|Plainview|Nash County|NC|23683|United States|-5|condo| +42063|AAAAAAAAPEEKAAAA|497|Poplar Spruce|Dr.|Suite 370|Macedonia|Stark County|OH|41087|United States|-5|condo| +42064|AAAAAAAAAFEKAAAA||Lake |Ct.||Woodrow||MS||||| +42065|AAAAAAAABFEKAAAA|618|Maple Jackson|Cir.|Suite 300|Mount Pleasant|Mohave County|AZ|81933|United States|-7|single family| +42066|AAAAAAAACFEKAAAA|282|Sunset |Road|Suite 420|Springfield|Mora County|NM|89303|United States|-7|condo| +42067|AAAAAAAADFEKAAAA|643|Church |Dr.|Suite 20|Hillcrest|Orleans County|VT|03603|United States|-5|condo| +42068|AAAAAAAAEFEKAAAA|268|Oak |Cir.|Suite X|Waterloo|Lincoln County|WA|91675|United States|-8|apartment| +42069|AAAAAAAAFFEKAAAA|452|Davis Spring|Pkwy|Suite Y|Glenwood|Jefferson County|TX|73511|United States|-6|apartment| +42070|AAAAAAAAGFEKAAAA|718|Dogwood |Road|Suite H|Glendale|Lincoln County|NV|83951|United States|-8|condo| +42071|AAAAAAAAHFEKAAAA|679|4th |Lane|Suite 470|Spring Valley|Mahaska County|IA|56060|United States|-6|condo| +42072|AAAAAAAAIFEKAAAA|787|9th Miller|Drive|Suite 120|Woodville|Sherburne County|MN|54289|United States|-6|single family| +42073|AAAAAAAAJFEKAAAA|975|Sixth |ST|Suite 210|Oakwood|Golden Valley County|MT|60169|United States|-7|apartment| +42074|AAAAAAAAKFEKAAAA|776|Willow |Ave|Suite 330|Pleasant Hill|Jersey County|IL|63604|United States|-6|apartment| +42075|AAAAAAAALFEKAAAA|485|South 4th|Avenue|Suite D|Colfax|Stephenson County|IL|62565|United States|-6|apartment| +42076|AAAAAAAAMFEKAAAA|424|Forest 13th|Boulevard|Suite 0|Hamilton|Benton County|IA|52808|United States|-6|single family| +42077|AAAAAAAANFEKAAAA|616|East Oak|Dr.|Suite G|Gravel Hill|Jones County|TX|71944|United States|-6|single family| +42078|AAAAAAAAOFEKAAAA|774|Willow |RD|Suite B|Cedar Grove|Madera County|CA|90411|United States|-8|condo| +42079|AAAAAAAAPFEKAAAA|756|Chestnut |Ct.|Suite 330|Arlington|Hopkins County|TX|76557|United States|-6|apartment| +42080|AAAAAAAAAGEKAAAA|861|Green South|Wy|Suite 80|Union Hill|Allen County|IN|47746|United States|-5|apartment| +42081|AAAAAAAABGEKAAAA|55|Madison Pine|Ave|Suite U|Plainview|Fayette County|KY|43683|United States|-6|apartment| +42082|AAAAAAAACGEKAAAA|1000|Mill |Cir.|Suite W|Macedonia|Rappahannock County|VA|21087|United States|-5|condo| +42083|AAAAAAAADGEKAAAA|247|Ridge |Way|Suite I|Maple Grove|Claiborne County|TN|38252|United States|-5|apartment| +42084|AAAAAAAAEGEKAAAA|828|Cherry Cedar|Blvd|Suite 340|Bridgeport|Waukesha County|WI|55817|United States|-6|apartment| +42085|AAAAAAAAFGEKAAAA|632|Miller |Court|Suite E|Crossroads|Sitka Borough|AK|90534|United States|-9|condo| +42086|AAAAAAAAGGEKAAAA|903|Oak |Blvd|Suite 40|Sleepy Hollow|Boone County|IA|53592|United States|-6|condo| +42087|AAAAAAAAHGEKAAAA|998|Spring |Parkway|Suite N|Highland|Allegan County|MI|49454|United States|-5|single family| +42088|AAAAAAAAIGEKAAAA|610|Ridge |Drive|Suite 160|Royal|East Feliciana Parish|LA|75819|United States|-6|apartment| +42089|AAAAAAAAJGEKAAAA|||ST||||||||single family| +42090|AAAAAAAAKGEKAAAA|526|River |Ln|Suite 340|Springdale|Montgomery County|AL|38883|United States|-6|apartment| +42091|AAAAAAAALGEKAAAA|157|Davis |Street|Suite V|Greenfield|Walla Walla County|WA|95038|United States|-8|apartment| +42092|AAAAAAAAMGEKAAAA||6th Franklin|Wy|Suite Q||||||-6|| +42093|AAAAAAAANGEKAAAA|154|Lakeview Hickory|Parkway|Suite 490|Lakeview|Alexander County|NC|28579|United States|-5|apartment| +42094|AAAAAAAAOGEKAAAA|27|West |Parkway|Suite 220|Wildwood|Clay County|MO|66871|United States|-6|single family| +42095|AAAAAAAAPGEKAAAA|280|Sycamore 10th|Ct.|Suite D|Woodville|Cooper County|MO|64289|United States|-6|single family| +42096|AAAAAAAAAHEKAAAA|356|Second South|Parkway|Suite B|Green Acres|Botetourt County|VA|27683|United States|-5|single family| +42097|AAAAAAAABHEKAAAA|349|13th 7th|Avenue|Suite 30|Bethel|Medina County|TX|75281|United States|-6|single family| +42098|AAAAAAAACHEKAAAA|680|Oak |Ln|Suite Y|Willow|Wilkinson County|MS|56798|United States|-6|apartment| +42099|AAAAAAAADHEKAAAA|958|1st Ash|Wy|Suite 240|Glendale|Madison County|VA|23951|United States|-5|single family| +42100|AAAAAAAAEHEKAAAA|129|Birch Lake|Road|Suite S|Newtown|Gregg County|TX|71749|United States|-6|single family| +42101|AAAAAAAAFHEKAAAA|744|Locust Adams|Court|Suite 220|Brownsville|Yalobusha County|MS|59310|United States|-6|condo| +42102|AAAAAAAAGHEKAAAA|835|View 4th|ST|Suite 240|Bridgeport|Albany County|NY|15817|United States|-5|apartment| +42103|AAAAAAAAHHEKAAAA|96|Laurel |Ct.|Suite K|Pleasant Grove|Crawford County|IL|64136|United States|-6|condo| +42104|AAAAAAAAIHEKAAAA|773|3rd |RD|Suite G|Harmony|Eureka County|NV|85804|United States|-8|apartment| +42105|AAAAAAAAJHEKAAAA|143|Hill |Parkway|Suite R|Riverdale|Lorain County|OH|49391|United States|-5|single family| +42106|AAAAAAAAKHEKAAAA|856|Maple 9th|Street|Suite N|Guilford|Pennington County|SD|54408|United States|-7|apartment| +42107|AAAAAAAALHEKAAAA|100|Washington Main|Wy|Suite M|Lincoln|Coffee County|GA|31289|United States|-5|condo| +42108|AAAAAAAAMHEKAAAA|729|Wilson Ridge|Lane|Suite 90|Tabor|Osage County|KS|68529|United States|-6|condo| +42109|AAAAAAAANHEKAAAA|634|Ash |Parkway|Suite W|Acme|Woodruff County|AR|70164|United States|-6|condo| +42110|AAAAAAAAOHEKAAAA|696|12th |Boulevard|Suite A|Brownsville|Jefferson Davis Parish|LA|79310|United States|-6|condo| +42111|AAAAAAAAPHEKAAAA|466|Second Johnson|Way|Suite 190|Sutton|Pendleton County|WV|25413|United States|-5|apartment| +42112|AAAAAAAAAIEKAAAA|418|Park |Ave|Suite 150|Harmony|Harris County|GA|35804|United States|-5|condo| +42113|AAAAAAAABIEKAAAA|189|Park |Wy|Suite E|Union Hill|Benton County|OR|97746|United States|-8|condo| +42114|AAAAAAAACIEKAAAA|944|Franklin Forest|Lane|Suite H|Providence|Lac qui Parle County|MN|56614|United States|-6|single family| +42115|AAAAAAAADIEKAAAA|882|Madison Chestnut|Blvd|Suite 370|Farmington|Sherburne County|MN|59145|United States|-6|condo| +42116|AAAAAAAAEIEKAAAA|832|Dogwood Willow|Ct.|Suite N|Red Hill|Monroe County|IN|44338|United States|-5|single family| +42117|AAAAAAAAFIEKAAAA|383|Washington |Ct.|Suite P|Woodville|Fulton County|AR|74289|United States|-6|single family| +42118|AAAAAAAAGIEKAAAA|960|Willow Church|Ct.|Suite 480|Springhill|Benewah County|ID|84602|United States|-7|single family| +42119|AAAAAAAAHIEKAAAA|114|1st Cherry|Boulevard|Suite 150|Glendale|Granville County|NC|23951|United States|-5|single family| +42120|AAAAAAAAIIEKAAAA|311|4th |Dr.|Suite T|Woodland|Adams County|WI|54854|United States|-6|apartment| +42121|AAAAAAAAJIEKAAAA|30|Oak |ST|Suite 390|Clifton|Finney County|KS|68014|United States|-6|condo| +42122|AAAAAAAAKIEKAAAA|611|3rd Sycamore|Wy|Suite 120|Hardy|Fayette County|PA|15354|United States|-5|condo| +42123|AAAAAAAALIEKAAAA|611|Locust Hickory|Boulevard|Suite D|Arlington|Jefferson County|KS|66557|United States|-6|single family| +42124|AAAAAAAAMIEKAAAA|505|9th |Blvd|Suite 60|Wesley|Washington County|OH|41218|United States|-5|condo| +42125|AAAAAAAANIEKAAAA||||Suite P|Riverside|Putnam County|OH||United States||| +42126|AAAAAAAAOIEKAAAA||||Suite 90||||||-6|apartment| +42127|AAAAAAAAPIEKAAAA|202|6th Cherry|RD|Suite O|Centerville|Fayette County|TX|70059|United States|-6|single family| +42128|AAAAAAAAAJEKAAAA|649|13th |Ave|Suite 200|Deerfield|Fulton County|KY|49840|United States|-6|condo| +42129|AAAAAAAABJEKAAAA|902|Jackson Chestnut|Boulevard|Suite L|Walnut Grove|Cedar County|NE|67752|United States|-6|apartment| +42130|AAAAAAAACJEKAAAA|825|Oak |Wy|Suite W|Glendale|Perquimans County|NC|23951|United States|-5|apartment| +42131|AAAAAAAADJEKAAAA|632|Spruce |Wy|Suite R|Farmington|Asotin County|WA|99145|United States|-8|apartment| +42132|AAAAAAAAEJEKAAAA|467|Dogwood View|Ct.|Suite 170|Midway|Montgomery County|VA|21904|United States|-5|apartment| +42133|AAAAAAAAFJEKAAAA|700|4th Lake|Drive|Suite P|Arlington|Scott County|IA|56557|United States|-6|single family| +42134|AAAAAAAAGJEKAAAA|621|Jackson |Dr.|Suite 270|Shiloh|Trumbull County|OH|49275|United States|-5|apartment| +42135|AAAAAAAAHJEKAAAA|782|Main |Cir.|Suite 0|Farmington|Lipscomb County|TX|79145|United States|-6|single family| +42136|AAAAAAAAIJEKAAAA|801|Lakeview |ST|Suite J|Farmington|Sedgwick County|KS|69145|United States|-6|single family| +42137|AAAAAAAAJJEKAAAA|935|First Valley|Wy|Suite 470|Clifton|Rains County|TX|78014|United States|-6|condo| +42138|AAAAAAAAKJEKAAAA|949|Franklin Jefferson|ST|Suite 470|Franklin|Prowers County|CO|89101|United States|-7|single family| +42139|AAAAAAAALJEKAAAA|252|East |Drive|Suite E|Mountain View|Carter County|KY|44466|United States|-6|apartment| +42140|AAAAAAAAMJEKAAAA||Ridge 8th|Road|Suite H||||61952|United States||apartment| +42141|AAAAAAAANJEKAAAA|859|Sycamore Fourth|Ct.|Suite 370|Buena Vista|Athens County|OH|45752|United States|-5|apartment| +42142|AAAAAAAAOJEKAAAA|301|South Broadway|Drive|Suite 470|Caldwell|Howard County|MO|66913|United States|-6|single family| +42143|AAAAAAAAPJEKAAAA|640|Hill |RD|Suite 300|Doyle|Big Horn County|MT|68434|United States|-7|apartment| +42144|AAAAAAAAAKEKAAAA|236|Elm |Ct.|Suite 330|Enterprise|Monroe County|MO|61757|United States|-6|single family| +42145|AAAAAAAABKEKAAAA|530|Jefferson |Avenue|Suite G|Shaw|Telfair County|GA|30618|United States|-5|apartment| +42146|AAAAAAAACKEKAAAA|764|Green |ST|Suite R|Liberty|Johnson County|NE|63451|United States|-7|condo| +42147|AAAAAAAADKEKAAAA|587|1st Woodland|Ln|Suite 120|Liberty|Glasscock County|TX|73451|United States|-6|apartment| +42148|AAAAAAAAEKEKAAAA|562|South Cherry|Blvd|Suite 460|Clifton|Cameron Parish|LA|78014|United States|-6|condo| +42149|AAAAAAAAFKEKAAAA|348|Oak Jefferson|Avenue|Suite 380|Globe|Escambia County|AL|39614|United States|-6|single family| +42150|AAAAAAAAGKEKAAAA|981|Birch |ST|Suite G|White Oak|Monroe County|IA|56668|United States|-6|condo| +42151|AAAAAAAAHKEKAAAA|482|South Hill|Wy|Suite F|Crossroads|Isanti County|MN|50534|United States|-6|single family| +42152|AAAAAAAAIKEKAAAA|455|7th Dogwood|Pkwy|Suite P|Kingston|Polk County|MN|54975|United States|-6|condo| +42153|AAAAAAAAJKEKAAAA|392|Park Williams|Lane|Suite 200|Newport|Darke County|OH|41521|United States|-5|single family| +42154|AAAAAAAAKKEKAAAA|729|Central Lakeview|Lane|Suite I|Clinton|Cowlitz County|WA|98222|United States|-8|condo| +42155|AAAAAAAALKEKAAAA|38|Park |Circle|Suite 360|Florence|Pike County|GA|33394|United States|-5|apartment| +42156|AAAAAAAAMKEKAAAA|260|Walnut |Ln|Suite 350|Oakland|Fremont County|CO|89843|United States|-7|apartment| +42157|AAAAAAAANKEKAAAA|714|Lincoln |ST|Suite 190|Shady Grove|Abbeville County|SC|22812|United States|-5|condo| +42158|AAAAAAAAOKEKAAAA|836|Elm Ash|Road|Suite 490|Woodcrest|Lee County|IL|64919|United States|-6|condo| +42159|AAAAAAAAPKEKAAAA|148|3rd Franklin|Way|Suite 480|Bethel|Addison County|VT|05881|United States|-5|apartment| +42160|AAAAAAAAALEKAAAA|125|Park Birch|Ct.|Suite O|Glenwood|Lake County|MI|43511|United States|-5|apartment| +42161|AAAAAAAABLEKAAAA|985|Chestnut |Ln|Suite V|Bunker Hill|Harper County|KS|60150|United States|-6|single family| +42162|AAAAAAAACLEKAAAA|558|Railroad Willow|Pkwy|Suite S|Friendship|Warren County|PA|14536|United States|-5|single family| +42163|AAAAAAAADLEKAAAA|809|Valley Woodland|Wy|Suite T|Oak Grove|Iron County|WI|58370|United States|-6|single family| +42164|AAAAAAAAELEKAAAA|114|Park |Court|Suite 80|Clifton|Inyo County|CA|98014|United States|-8|apartment| +42165|AAAAAAAAFLEKAAAA|977|Sunset |Parkway|Suite 260|Valley View|Hancock County|MS|55124|United States|-6|single family| +42166|AAAAAAAAGLEKAAAA|565|Spruce Jefferson|Drive|Suite F|Hamilton|Eastland County|TX|72808|United States|-6|condo| +42167|AAAAAAAAHLEKAAAA|303|North Seventh|Lane|Suite L|Phoenix|Jasper County|IN|42276|United States|-5|apartment| +42168|AAAAAAAAILEKAAAA|517|8th Wilson|Street|Suite 330|Bethel|Lincoln County|WI|55281|United States|-6|single family| +42169|AAAAAAAAJLEKAAAA|214|Sunset Lake|Cir.|Suite 190|Pleasant Hill|Gaines County|TX|73604|United States|-6|apartment| +42170|AAAAAAAAKLEKAAAA|937|4th Fourth|Lane|Suite W|Mount Vernon|Newberry County|SC|28482|United States|-5|apartment| +42171|AAAAAAAALLEKAAAA|693|Adams Fourth|Road|Suite B|Fairfield|Onslow County|NC|26192|United States|-5|condo| +42172|AAAAAAAAMLEKAAAA|154|Maple |Parkway|Suite 220|Pleasant Valley|Jackson County|CO|82477|United States|-7|single family| +42173|AAAAAAAANLEKAAAA|87|East |Dr.|Suite 170|Shiloh|Colbert County|AL|39275|United States|-6|condo| +42174|AAAAAAAAOLEKAAAA|203|9th 10th|Parkway|Suite 130|Greenwood|Scott County|VA|28828|United States|-5|condo| +42175|AAAAAAAAPLEKAAAA|957|Elm Hill|Parkway|Suite Y|Vista|Orange County|VA|24694|United States|-5|apartment| +42176|AAAAAAAAAMEKAAAA|595|||Suite U||Columbia County|GA|||-5|| +42177|AAAAAAAABMEKAAAA|670|Cedar |ST|Suite G|Wilson|Carlton County|MN|56971|United States|-6|apartment| +42178|AAAAAAAACMEKAAAA|680|Jackson Cedar|Blvd|Suite 200|Colonial Heights|Clarion County|PA|13425|United States|-5|single family| +42179|AAAAAAAADMEKAAAA|905|Cherry 2nd|Court|||||41904|United States||| +42180|AAAAAAAAEMEKAAAA|166|15th |Way|Suite V|Hopewell|Ohio County|WV|20587|United States|-5|apartment| +42181|AAAAAAAAFMEKAAAA|467|Green 3rd|Blvd|Suite U|Greenfield|Sioux County|ND|55038|United States|-6|condo| +42182|AAAAAAAAGMEKAAAA|6|Ash |Way|Suite Q|Shady Grove|Santa Rosa County|FL|32812|United States|-5|condo| +42183|AAAAAAAAHMEKAAAA|841|1st |Street|Suite W|Hidden Valley|Prince William County|VA|25521|United States|-5|condo| +42184|AAAAAAAAIMEKAAAA|640||Road|Suite M|||VA||United States|-5|| +42185|AAAAAAAAJMEKAAAA|317|4th |RD|Suite J|Longwood|Chambers County|TX|77021|United States|-6|single family| +42186|AAAAAAAAKMEKAAAA|519|Mill |Lane|Suite J|Marion|Yadkin County|NC|20399|United States|-5|single family| +42187|AAAAAAAALMEKAAAA|212|Church 4th|Circle|Suite N|Spring Hill|Bristol County|RI|07387|United States|-5|single family| +42188|AAAAAAAAMMEKAAAA|821|Willow |Street|Suite 290|Bethel|Blanco County|TX|75281|United States|-6|condo| +42189|AAAAAAAANMEKAAAA|671|Lee Jackson|RD|Suite 40|Oak Ridge|Allegany County|MD|28371|United States|-5|apartment| +42190|AAAAAAAAOMEKAAAA|591|Hill |Blvd|Suite 100|Hillcrest|Garfield County|NE|63003|United States|-6|apartment| +42191|AAAAAAAAPMEKAAAA|63|4th Jackson|Ct.|Suite 170|Spring Hill|Okaloosa County|FL|36787|United States|-5|apartment| +42192|AAAAAAAAANEKAAAA|880|Spring |Street|Suite U|Farmington|King and Queen County|VA|29145|United States|-5|apartment| +42193|AAAAAAAABNEKAAAA|457|Forest College|Road|Suite 110|Oak Hill|Buncombe County|NC|27838|United States|-5|apartment| +42194|AAAAAAAACNEKAAAA|944|Main East|Ct.|Suite E|Centerville|DeKalb County|IL|60059|United States|-6|single family| +42195|AAAAAAAADNEKAAAA|38|Walnut |Street|Suite 310|Woodville|Geneva County|AL|34289|United States|-6|apartment| +42196|AAAAAAAAENEKAAAA|282|View Poplar|Ct.|Suite 240|Concord|Putnam County|FL|34107|United States|-5|condo| +42197|AAAAAAAAFNEKAAAA|824|Lincoln 7th|ST|Suite 100|Summerville|Roanoke city|VA|22033|United States|-5|single family| +42198|AAAAAAAAGNEKAAAA|192|Mill 15th|Lane|Suite K|White Hall|Greenville County|SC|26955|United States|-5|condo| +42199|AAAAAAAAHNEKAAAA|187|College South|Dr.|Suite 370|Unionville|Cochran County|TX|71711|United States|-6|condo| +42200|AAAAAAAAINEKAAAA|1|10th 9th|Boulevard|Suite 460|Tanglewood|Sutter County|CA|98994|United States|-8|single family| +42201|AAAAAAAAJNEKAAAA|717|Ridge 4th|Road|Suite 170|Fairview|Cook County|MN|55709|United States|-6|condo| +42202|AAAAAAAAKNEKAAAA|62|Railroad |Court|Suite 340|Stratford|Lauderdale County|AL|36668|United States|-6|apartment| +42203|AAAAAAAALNEKAAAA|548|9th East|Cir.|Suite 50|Clinton|Brown County|NE|68222|United States|-6|apartment| +42204|AAAAAAAAMNEKAAAA|77|2nd |Boulevard|Suite 70|Wilson|Greenwood County|KS|66971|United States|-6|apartment| +42205|AAAAAAAANNEKAAAA|565|Eigth 2nd|Lane|Suite 10|Guilford|Stonewall County|TX|74408|United States|-6|apartment| +42206|AAAAAAAAONEKAAAA|514|Locust |Boulevard|Suite Q|Red Hill|Webster County|NE|64338|United States|-7|single family| +42207|AAAAAAAAPNEKAAAA|487|Franklin Broadway|Ln|Suite D|Jamestown|Shannon County|SD|56867|United States|-7|apartment| +42208|AAAAAAAAAOEKAAAA|342|Jackson |Avenue|Suite V|Farmington|Goliad County|TX|79145|United States|-6|apartment| +42209|AAAAAAAABOEKAAAA|262|Park |Way|Suite Y|Stringtown|Huron County|MI|40162|United States|-5|condo| +42210|AAAAAAAACOEKAAAA|105|Broadway Davis|Street|Suite 10|Midway|Union County|NM|81904|United States|-7|condo| +42211|AAAAAAAADOEKAAAA|30|West Walnut|Cir.|Suite V|Lakewood|Greeley County|NE|68877|United States|-6|apartment| +42212|AAAAAAAAEOEKAAAA|911|10th Sunset|Cir.|Suite P|Riverside|Bleckley County|GA|39231|United States|-5|single family| +42213|AAAAAAAAFOEKAAAA|438|6th |Ct.|Suite V|Friendship|Jackson County|OH|44536|United States|-5|condo| +42214|AAAAAAAAGOEKAAAA|262|Meadow Main|Street|Suite 360|Macedonia|Hendry County|FL|31087|United States|-5|single family| +42215|AAAAAAAAHOEKAAAA|314|Maple |Dr.|Suite 0|Macedonia|Warren County|IA|51087|United States|-6|apartment| +42216|AAAAAAAAIOEKAAAA|586|6th |Avenue|Suite U|Mount Zion|Campbell County|WY|88054|United States|-7|condo| +42217|AAAAAAAAJOEKAAAA|527|College Hillcrest|Avenue|Suite G|Deerfield|Gilmer County|WV|29840|United States|-5|apartment| +42218|AAAAAAAAKOEKAAAA|212|Willow 5th|Ln|Suite L|Farmington|Blount County|TN|39145|United States|-5|condo| +42219|AAAAAAAALOEKAAAA|6|3rd |Circle|Suite M|Pleasant Valley|Canadian County|OK|72477|United States|-6|condo| +42220|AAAAAAAAMOEKAAAA|363|15th |Way|Suite 20|Texas|Newton County|GA|33342|United States|-5|condo| +42221|AAAAAAAANOEKAAAA||Chestnut |Pkwy||||MS|54136|||condo| +42222|AAAAAAAAOOEKAAAA|854|Cedar |Ln|Suite X|Amherst|Stanley County|SD|58119|United States|-7|condo| +42223|AAAAAAAAPOEKAAAA|969|Forest |Cir.|Suite L|Liberty|Marshall County|SD|53451|United States|-7|condo| +42224|AAAAAAAAAPEKAAAA|157|Pine 5th|Blvd|Suite 400|Greenfield|Langlade County|WI|55038|United States|-6|single family| +42225|AAAAAAAABPEKAAAA|143|1st Mill|RD|Suite C|Arcadia|Tuscarawas County|OH|42022|United States|-5|condo| +42226|AAAAAAAACPEKAAAA|95|Miller Elevnth|Pkwy|Suite Y|Ashland|Decatur County|TN|34244|United States|-5|single family| +42227|AAAAAAAADPEKAAAA|568|Broadway |Way|Suite 100|Five Forks|Knox County|TX|72293|United States|-6|apartment| +42228|AAAAAAAAEPEKAAAA|101||Avenue|Suite H||Dauphin County|PA||United States||condo| +42229|AAAAAAAAFPEKAAAA|955|Johnson Spruce|Pkwy|Suite 450|Farmington|Parmer County|TX|79145|United States|-6|apartment| +42230|AAAAAAAAGPEKAAAA|60|Sunset |ST|Suite S|Florence|Grand Traverse County|MI|43394|United States|-5|condo| +42231|AAAAAAAAHPEKAAAA|402|Madison Birch|Cir.|Suite 340|Fairview|Mobile County|AL|35709|United States|-6|apartment| +42232|AAAAAAAAIPEKAAAA|729|Woodland Spruce|ST|Suite X|Lincoln|Wilkes County|GA|31289|United States|-5|condo| +42233|AAAAAAAAJPEKAAAA|221|8th Fourth|Ln|Suite J|Lakeview|Marion County|SC|28579|United States|-5|single family| +42234|AAAAAAAAKPEKAAAA|919|Cherry 6th|Cir.|Suite U|Maple Grove|Webb County|TX|78252|United States|-6|single family| +42235|AAAAAAAALPEKAAAA|560|3rd |Ln|Suite 200|Forest Hills|Poinsett County|AR|79237|United States|-6|apartment| +42236|AAAAAAAAMPEKAAAA|320|6th Pine|ST|Suite 300|Stringtown|Baxter County|AR|70162|United States|-6|single family| +42237|AAAAAAAANPEKAAAA|620|8th Walnut|Avenue|Suite 150|New Hope|Greenup County|KY|49431|United States|-6|apartment| +42238|AAAAAAAAOPEKAAAA|839|Hillcrest |Parkway|Suite 340|Pleasant Grove|Canadian County|OK|74136|United States|-6|condo| +42239|AAAAAAAAPPEKAAAA|122|Maple North||Suite 410||Cumberland County|||||| +42240|AAAAAAAAAAFKAAAA|587|Oak Maple|Cir.|Suite 440|Sugar Hill|Harnett County|NC|25114|United States|-5|apartment| +42241|AAAAAAAABAFKAAAA|311|Hillcrest |ST|Suite 330|Providence|Jefferson County|WA|96614|United States|-8|apartment| +42242|AAAAAAAACAFKAAAA|689|Maple Railroad|Blvd|Suite E|White Oak|Powell County|MT|66668|United States|-7|apartment| +42243|AAAAAAAADAFKAAAA|374|Hickory |Ave|Suite G|Kingston|Teller County|CO|84975|United States|-7|single family| +42244|AAAAAAAAEAFKAAAA|155|Johnson |ST|Suite R|Highland Park|Ocean County|NJ|07134|United States|-5|apartment| +42245|AAAAAAAAFAFKAAAA|683|Third Pine|Ct.|Suite 120|Waterloo|Ohio County|KY|41675|United States|-5|single family| +42246|AAAAAAAAGAFKAAAA|256|Lake South|Road|Suite 380|Lincoln|Lorain County|OH|41289|United States|-5|condo| +42247|AAAAAAAAHAFKAAAA|497|Main Woodland|RD|Suite 340|Springfield|Clay County|IL|69303|United States|-6|condo| +42248|AAAAAAAAIAFKAAAA|387|River |Blvd|Suite P|Edgewood|Barry County|MO|60069|United States|-6|apartment| +42249|AAAAAAAAJAFKAAAA|793|Hill Jefferson|Way|Suite S|Franklin|Jefferson County|IL|69101|United States|-6|single family| +42250|AAAAAAAAKAFKAAAA|786|West |Dr.|Suite G|Clifton|Horry County|SC|28014|United States|-5|single family| +42251|AAAAAAAALAFKAAAA|320|Lake |Ln|Suite 0|Pine Grove|Sumter County|GA|34593|United States|-5|apartment| +42252|AAAAAAAAMAFKAAAA|308|13th 3rd|Street|Suite D|Lincoln|Orange County|FL|31289|United States|-5|single family| +42253|AAAAAAAANAFKAAAA|919|South View|Ln|Suite 410|Five Points|Monroe County|MO|66098|United States|-6|single family| +42254|AAAAAAAAOAFKAAAA|184|2nd Walnut|Dr.|Suite 490|Shiloh|Titus County|TX|79275|United States|-6|single family| +42255|AAAAAAAAPAFKAAAA|687|Central |Ct.|Suite P|Mountain View|Clinton County|IN|44466|United States|-5|condo| +42256|AAAAAAAAABFKAAAA|413|College |Boulevard|Suite 40|Hopewell|Richmond County|NC|20587|United States|-5|condo| +42257|AAAAAAAABBFKAAAA|857|Pine |Circle|Suite M|Oak Hill|Clarke County|GA|37838|United States|-5|condo| +42258|AAAAAAAACBFKAAAA|118|Maple Williams|Cir.|Suite O|Centerville|Val Verde County|TX|70059|United States|-6|condo| +42259|AAAAAAAADBFKAAAA|654||Way||Oak Ridge|||78371|United States||| +42260|AAAAAAAAEBFKAAAA|||Cir.|Suite 230|||SD||United States|-7|| +42261|AAAAAAAAFBFKAAAA|315|Forest |Pkwy|Suite 490|Four Points|Santa Clara County|CA|91216|United States|-8|condo| +42262|AAAAAAAAGBFKAAAA|659|Jackson |Dr.|Suite X|Mount Vernon|Edmunds County|SD|58482|United States|-6|single family| +42263|AAAAAAAAHBFKAAAA|200|Church |Drive|Suite 190|Friendship|Terrell County|TX|74536|United States|-6|single family| +42264|AAAAAAAAIBFKAAAA|822|First 2nd|Blvd|Suite 490|Shady Grove|Lonoke County|AR|72812|United States|-6|condo| +42265|AAAAAAAAJBFKAAAA|172|Second 9th|Blvd|Suite O|Appleton|Doddridge County|WV|24240|United States|-5|condo| +42266|AAAAAAAAKBFKAAAA|559|Franklin Seventh|ST|Suite 340|Midway|Sedgwick County|CO|81904|United States|-7|apartment| +42267|AAAAAAAALBFKAAAA|293|Broadway Lake|ST|Suite 310|Woodville|Shoshone County|ID|84289|United States|-7|single family| +42268|AAAAAAAAMBFKAAAA|817|Lake Second|ST|Suite 330|Stringtown|Putnam County|OH|40162|United States|-5|apartment| +42269|AAAAAAAANBFKAAAA|253|Maple |Boulevard|Suite 430|Franklin|Glascock County|GA|39101|United States|-5|apartment| +42270|AAAAAAAAOBFKAAAA|123|Walnut |ST|Suite 80|Bunker Hill|Menard County|IL|60150|United States|-6|condo| +42271|AAAAAAAAPBFKAAAA|107|Hillcrest Fifth|Ave|Suite H|Wildwood|Evans County|GA|36871|United States|-5|condo| +42272|AAAAAAAAACFKAAAA|888|9th Elm|Cir.|Suite B|Oak Hill|Iron County|WI|57838|United States|-6|apartment| +42273|AAAAAAAABCFKAAAA|309|Third Main|Way|Suite P|Midway|Pike County|KY|41904|United States|-5|condo| +42274|AAAAAAAACCFKAAAA|667|Jackson Cedar|Pkwy|Suite 10|Red Bank|Pulaski County|MO|64975|United States|-6|apartment| +42275|AAAAAAAADCFKAAAA|925|Spring 11th|Street|Suite 320|Five Forks|Macomb County|MI|42293|United States|-5|single family| +42276|AAAAAAAAECFKAAAA|66|North Hill|Ct.|Suite 30|Sunnyside|Kauai County|HI|91952|United States|-10|single family| +42277|AAAAAAAAFCFKAAAA|592|Highland Oak|Dr.|Suite H|Shiloh|Sarpy County|NE|69275|United States|-7|single family| +42278|AAAAAAAAGCFKAAAA|473|View |ST|Suite C|Jamestown|Carroll County|MD|26867|United States|-5|apartment| +42279|AAAAAAAAHCFKAAAA|532|Eigth |Blvd|Suite T|Edgewood|Brunswick County|NC|20069|United States|-5|single family| +42280|AAAAAAAAICFKAAAA|156|1st |Wy|Suite K|Crossroads|Genesee County|NY|10534|United States|-5|condo| +42281|AAAAAAAAJCFKAAAA|487|Third |Circle|Suite T|Clinton|Cambria County|PA|18222|United States|-5|single family| +42282|AAAAAAAAKCFKAAAA|274|Hill Maple|Circle|Suite 60|Bethel|Saline County|AR|75281|United States|-6|apartment| +42283|AAAAAAAALCFKAAAA|186|3rd Hill|Cir.|Suite P|Riverview|Johnson County|TX|79003|United States|-6|single family| +42284|AAAAAAAAMCFKAAAA|155|View Sycamore|Pkwy|Suite G|Spring Valley|Baker County|OR|96060|United States|-8|apartment| +42285|AAAAAAAANCFKAAAA|815|8th Broadway|Ct.|Suite V|Jackson|Muskegon County|MI|49583|United States|-5|condo| +42286|AAAAAAAAOCFKAAAA||Cedar |||Greenville|Jeff Davis County|GA|||-5|condo| +42287|AAAAAAAAPCFKAAAA|871|Railroad East|Way|Suite A|Morgantown|New Haven County|CT|09793|United States|-5|condo| +42288|AAAAAAAAADFKAAAA|385|Park |Wy|Suite 450|Unionville|Richland County|IL|61711|United States|-6|single family| +42289|AAAAAAAABDFKAAAA|289|Birch |Blvd|Suite 480|Hopewell|Dallas County|MO|60587|United States|-6|single family| +42290|AAAAAAAACDFKAAAA|35|View |Court|Suite K|Lakewood|Daviess County|IN|48877|United States|-5|single family| +42291|AAAAAAAADDFKAAAA|878|11th |Parkway|Suite 40|Marion|Houston County|AL|30399|United States|-6|single family| +42292|AAAAAAAAEDFKAAAA|472|Highland |Parkway|Suite 110|Mountain View|Colonial Heights city|VA|24466|United States|-5|apartment| +42293|AAAAAAAAFDFKAAAA|470|Main |Circle|Suite M|Rutland|Chelan County|WA|98375|United States|-8|single family| +42294|AAAAAAAAGDFKAAAA|599|Davis |Pkwy|Suite T|Oak Grove|Boone County|WV|28370|United States|-5|condo| +42295|AAAAAAAAHDFKAAAA|234|1st |Cir.|Suite 170|Five Forks|Nantucket County|MA|02893|United States|-5|condo| +42296|AAAAAAAAIDFKAAAA|647|Cedar Center|Wy|Suite O|Mount Olive|Baker County|OR|98059|United States|-8|apartment| +42297|AAAAAAAAJDFKAAAA|92|Sunset Lake|RD|Suite Y|Woodland|Sweet Grass County|MT|64854|United States|-7|condo| +42298|AAAAAAAAKDFKAAAA|272|||Suite B|Glendale|||63951|United States||| +42299|AAAAAAAALDFKAAAA|316|Broadway |Blvd|Suite K|Murray|Campbell County|WY|82150|United States|-7|apartment| +42300|AAAAAAAAMDFKAAAA|532|Willow Ridge|Cir.|Suite V|Greenville|Cook County|IL|61387|United States|-6|condo| +42301|AAAAAAAANDFKAAAA|792|Willow 3rd|Lane|Suite 60|Hopewell|Webster County|GA|30587|United States|-5|single family| +42302|AAAAAAAAODFKAAAA|38|First |Dr.|Suite 420|Maple Grove|Logan County|OK|78252|United States|-6|apartment| +42303|AAAAAAAAPDFKAAAA|900|Cedar |Ln|Suite 280|Guthrie|Crow Wing County|MN|51423|United States|-6|apartment| +42304|AAAAAAAAAEFKAAAA|712|Church Main|RD|Suite 10|Pleasant Valley|Bond County|IL|62477|United States|-6|condo| +42305|AAAAAAAABEFKAAAA|831|Railroad |ST|Suite 140|Edgewood|San Bernardino County|CA|90069|United States|-8|condo| +42306|AAAAAAAACEFKAAAA|205|Highland |Blvd|Suite 480|Summit|La Salle County|IL|60499|United States|-6|single family| +42307|AAAAAAAADEFKAAAA|265|Wilson |Dr.|Suite C|Guthrie|Queen Anne County|MD|21423|United States|-5|single family| +42308|AAAAAAAAEEFKAAAA|763|Sycamore |Road|Suite R|Mount Pleasant|Greer County|OK|71933|United States|-6|apartment| +42309|AAAAAAAAFEFKAAAA|330|9th Center|Ln|Suite L|Waterloo|Lemhi County|ID|81675|United States|-7|condo| +42310|AAAAAAAAGEFKAAAA|920|4th |Pkwy|Suite 80|Waterloo|Winn Parish|LA|71675|United States|-6|single family| +42311|AAAAAAAAHEFKAAAA|450|3rd 3rd|Wy|Suite 310|Summerville|Hampshire County|MA|02633|United States|-5|condo| +42312|AAAAAAAAIEFKAAAA|166|Franklin |Dr.|Suite 450|Forest Hills|Leon County|TX|79237|United States|-6|condo| +42313|AAAAAAAAJEFKAAAA|289|Laurel Park|Boulevard|Suite R|Vienna|Burnett County|WI|55119|United States|-6|apartment| +42314|AAAAAAAAKEFKAAAA|459|Adams 2nd|Lane|Suite S|Bridgeport|Bates County|MO|65817|United States|-6|condo| +42315|AAAAAAAALEFKAAAA||||||Bland County|VA|29431|United States||single family| +42316|AAAAAAAAMEFKAAAA|96|Oak Twelfth|Dr.|Suite J|Sherwood Forest|Orange County|VT|07302|United States|-5|apartment| +42317|AAAAAAAANEFKAAAA|137|Wilson 3rd|Lane|Suite C|Bunker Hill|Clay County|NE|60150|United States|-6|condo| +42318|AAAAAAAAOEFKAAAA|746|Third Cedar|Wy|Suite 220|Oakland|Wolfe County|KY|49843|United States|-5|condo| +42319|AAAAAAAAPEFKAAAA|706|Forest |Parkway|Suite 360|Marion|Washington County|TN|30399|United States|-6|apartment| +42320|AAAAAAAAAFFKAAAA|329|3rd |Ct.|Suite 360|Oak Ridge|Jones County|SD|58371|United States|-7|single family| +42321|AAAAAAAABFFKAAAA|64|Green Eigth|RD|Suite H|Marion|Essex County|VA|20399|United States|-5|apartment| +42322|AAAAAAAACFFKAAAA|348|1st Davis|Parkway|Suite Y|Brownsville|Shannon County|SD|59310|United States|-7|single family| +42323|AAAAAAAADFFKAAAA|599|8th |Street|Suite H|Franklin|Cochran County|TX|79101|United States|-6|single family| +42324|AAAAAAAAEFFKAAAA|304|Dogwood Meadow|Dr.|Suite H|Greenville|Oldham County|TX|71387|United States|-6|apartment| +42325|AAAAAAAAFFFKAAAA|446|Broadway |Avenue|Suite 420|Cedar Grove|Schuyler County|MO|60411|United States|-6|apartment| +42326|AAAAAAAAGFFKAAAA|588|Green Jackson|Ct.|Suite 270|Riverview|Clallam County|WA|99003|United States|-8|single family| +42327|AAAAAAAAHFFKAAAA|631|Smith |Drive|Suite Y|New Hope|Lauderdale County|AL|39431|United States|-6|apartment| +42328|AAAAAAAAIFFKAAAA|61|Jefferson |Circle|Suite Y|Oakwood|Sitka Borough|AK|90169|United States|-9|condo| +42329|AAAAAAAAJFFKAAAA|671|Birch |Boulevard|Suite 210|Pleasant Hill|Radford city|VA|23604|United States|-5|condo| +42330|AAAAAAAAKFFKAAAA|693|Ridge |ST|Suite 160|Lakeview|Putnam County|TN|38579|United States|-6|condo| +42331|AAAAAAAALFFKAAAA|770|Smith |Drive|Suite 280|Woodlawn|Alexander County|IL|64098|United States|-6|apartment| +42332|AAAAAAAAMFFKAAAA|654|6th Poplar|Parkway|Suite 490|Harvey|Orange County|IN|45858|United States|-5|condo| +42333|AAAAAAAANFFKAAAA|756|Highland Oak|Court|Suite D|Waterloo|Newport News city|VA|21675|United States|-5|apartment| +42334|AAAAAAAAOFFKAAAA|445|9th |Parkway|Suite 350|Mount Zion|Terrell County|TX|78054|United States|-6|single family| +42335|AAAAAAAAPFFKAAAA|762|Pine |Drive|Suite W|Hillcrest|Ellis County|TX|73003|United States|-6|single family| +42336|AAAAAAAAAGFKAAAA|559|11th Davis|Lane|Suite W|Afton|Trigg County|KY|40029|United States|-5|single family| +42337|AAAAAAAABGFKAAAA|357|Johnson |Dr.|Suite 250|Hillcrest|Porter County|IN|43003|United States|-5|apartment| +42338|AAAAAAAACGFKAAAA|831|Sunset Lincoln|Circle|Suite W|Bunker Hill|Nome Census Area|AK|90150|United States|-9|apartment| +42339|AAAAAAAADGFKAAAA|56|1st |ST|Suite 240|Five Points|Botetourt County|VA|26098|United States|-5|single family| +42340|AAAAAAAAEGFKAAAA|652|River |Ave|Suite 150|Providence|Macon County|GA|36614|United States|-5|single family| +42341|AAAAAAAAFGFKAAAA|700|Walnut Valley|Court|Suite P|Riverdale|Dickson County|TN|39391|United States|-5|apartment| +42342|AAAAAAAAGGFKAAAA|710|Wilson |Drive|Suite U|Stringtown|Bedford County|PA|10162|United States|-5|single family| +42343|AAAAAAAAHGFKAAAA|434|East |Cir.|Suite 290|Tabor|Sagadahoc County|ME|09129|United States|-5|single family| +42344|AAAAAAAAIGFKAAAA|734|Fifth |Ave|Suite Y|Crossroads|Emmet County|IA|50534|United States|-6|condo| +42345|AAAAAAAAJGFKAAAA|384|Poplar |Avenue|Suite 390|New Hope|Moniteau County|MO|69431|United States|-6|single family| +42346|AAAAAAAAKGFKAAAA|565|Spring Lee|ST|Suite H|Stringtown|Boundary County|ID|80162|United States|-7|single family| +42347|AAAAAAAALGFKAAAA|356|Cedar |Ct.|Suite 400|White Oak|Grant County|KS|66668|United States|-6|condo| +42348|AAAAAAAAMGFKAAAA|32|Green Hickory|Lane|Suite 250|Stringtown|Marshall County|AL|30162|United States|-6|single family| +42349|AAAAAAAANGFKAAAA|738|Lake Lake|Blvd|Suite 10|Oakwood|Clay County|MO|60169|United States|-6|single family| +42350|AAAAAAAAOGFKAAAA|740|Valley College|Ave|Suite 310|Oakwood|Bienville Parish|LA|70169|United States|-6|apartment| +42351|AAAAAAAAPGFKAAAA|24|Birch |Street|Suite C|Brownsville|Placer County|CA|99310|United States|-8|single family| +42352|AAAAAAAAAHFKAAAA|271|Railroad |ST|Suite A|Vance|Person County|NC|20268|United States|-5|apartment| +42353|AAAAAAAABHFKAAAA|815|8th Main|Avenue|Suite 430|Woodland|Bandera County|TX|74854|United States|-6|apartment| +42354|AAAAAAAACHFKAAAA|||Ave||Newport||||United States||single family| +42355|AAAAAAAADHFKAAAA|116|View Third|Drive|Suite Y|Shiloh|Clinton County|KY|49275|United States|-6|single family| +42356|AAAAAAAAEHFKAAAA|211|Chestnut |Way|Suite J|Unionville|Big Stone County|MN|51711|United States|-6|single family| +42357|AAAAAAAAFHFKAAAA|159|Willow Johnson|Road|Suite 480|Sunnyside|Mille Lacs County|MN|51952|United States|-6|apartment| +42358|AAAAAAAAGHFKAAAA|51|Spruce |RD|Suite 370|Indian Village|Worcester County|MD|21075|United States|-5|single family| +42359|AAAAAAAAHHFKAAAA|764|Miller |Ave|Suite W|Wilson|Santa Clara County|CA|96971|United States|-8|condo| +42360|AAAAAAAAIHFKAAAA|514|Park |Blvd|Suite 40|Valley View|Berkshire County|MA|05724|United States|-5|apartment| +42361|AAAAAAAAJHFKAAAA|660|Williams East|Court|Suite 60|Spring Valley|Cumberland County|IL|66060|United States|-6|condo| +42362|AAAAAAAAKHFKAAAA|468|2nd |ST|Suite F|Friendship|Roanoke County|VA|24536|United States|-5|single family| +42363|AAAAAAAALHFKAAAA|428|Cherry Jefferson|Dr.|Suite Y|Edgewood|Jackson County|IL|60069|United States|-6|condo| +42364|AAAAAAAAMHFKAAAA|96|Johnson |Wy|Suite H|Five Forks|Ashley County|AR|72293|United States|-6|single family| +42365|AAAAAAAANHFKAAAA|359|Valley Davis|Avenue|Suite K|Brownsville|Woodson County|KS|69310|United States|-6|single family| +42366|AAAAAAAAOHFKAAAA|268|Chestnut South|Ln|Suite 300|Oakland|Adams County|IN|49843|United States|-5|single family| +42367|AAAAAAAAPHFKAAAA|782|Jackson South|Road|Suite P|Midway|White County|TN|31904|United States|-6|apartment| +42368|AAAAAAAAAIFKAAAA|979|1st |Pkwy|Suite W|Wilson|Silver Bow County|MT|66971|United States|-7|apartment| +42369|AAAAAAAABIFKAAAA|57|Cherry Jefferson|Drive|Suite L|Friendship|Lewis and Clark County|MT|64536|United States|-7|apartment| +42370|AAAAAAAACIFKAAAA|674|Hill |Road|Suite U|Kingston|McCracken County|KY|44975|United States|-5|single family| +42371|AAAAAAAADIFKAAAA|643|Smith |Wy|Suite C|Oakdale|Wapello County|IA|59584|United States|-6|single family| +42372|AAAAAAAAEIFKAAAA|562|Maple Ridge|Blvd|Suite A|Concord|Bristol Bay Borough|AK|94107|United States|-9|condo| +42373|AAAAAAAAFIFKAAAA|||Avenue||Shiloh|Tangipahoa Parish|LA||United States||| +42374|AAAAAAAAGIFKAAAA|268|Hill Hill|Ave|Suite 360|Lakeside|Maries County|MO|69532|United States|-6|single family| +42375|AAAAAAAAHIFKAAAA|162|Oak 7th|Avenue|Suite A|Oak Ridge|Jessamine County|KY|48371|United States|-6|condo| +42376|AAAAAAAAIIFKAAAA|947|Seventh |Ave|Suite J|Argyle|Madison County|TX|78722|United States|-6|apartment| +42377|AAAAAAAAJIFKAAAA|340|Oak |Blvd|Suite E|Salem|Washington County|IA|58048|United States|-6|condo| +42378|AAAAAAAAKIFKAAAA|73|Willow First|Court|Suite K|Macedonia|Cocke County|TN|31087|United States|-5|condo| +42379|AAAAAAAALIFKAAAA|487|Adams |Boulevard|Suite M|Riverview|Harlan County|NE|69003|United States|-6|single family| +42380|AAAAAAAAMIFKAAAA|800|Cedar |Street|Suite I|Lebanon|Kay County|OK|72898|United States|-6|condo| +42381|AAAAAAAANIFKAAAA|549|North Valley|Boulevard|Suite 90|Clinton|West Feliciana Parish|LA|78222|United States|-6|condo| +42382|AAAAAAAAOIFKAAAA|712|First |Way|Suite 430|Sunnyside|Cavalier County|ND|51952|United States|-6|condo| +42383|AAAAAAAAPIFKAAAA|468|West |Boulevard|Suite N|Wildwood|Latah County|ID|86871|United States|-7|apartment| +42384|AAAAAAAAAJFKAAAA|708|Meadow |Court|Suite 310|Clifton|La Salle Parish|LA|78014|United States|-6|single family| +42385|AAAAAAAABJFKAAAA|417|Walnut Hillcrest|Circle|Suite 300|Brownsville|Randall County|TX|79310|United States|-6|apartment| +42386|AAAAAAAACJFKAAAA|70|Ash Ridge|Cir.|Suite B|Marion|Mesa County|CO|80399|United States|-7|apartment| +42387|AAAAAAAADJFKAAAA|634|Lake |Boulevard|Suite O|Enterprise|Covington city|VA|21757|United States|-5|single family| +42388|AAAAAAAAEJFKAAAA|37|8th Wilson|Blvd|Suite 110|Lebanon|Marion County|MO|62898|United States|-6|condo| +42389|AAAAAAAAFJFKAAAA|809|Washington Dogwood|Drive|Suite 340|Woodville|Edgecombe County|NC|24289|United States|-5|apartment| +42390|AAAAAAAAGJFKAAAA|502|Ridge Elevnth|Pkwy|Suite K|Oakland|Windsor County|VT|09843|United States|-5|single family| +42391|AAAAAAAAHJFKAAAA|462|Pine 14th|Avenue|Suite 340|Lakeview|Cape Girardeau County|MO|68579|United States|-6|single family| +42392|AAAAAAAAIJFKAAAA|48|8th |Dr.|Suite B|Union Hill|Coweta County|GA|37746|United States|-5|apartment| +42393|AAAAAAAAJJFKAAAA|153|Locust Maple|Drive|Suite 250|Jackson|Richland County|WI|59583|United States|-6|single family| +42394|AAAAAAAAKJFKAAAA|46|Cedar |Circle|Suite N|Valley View|Ziebach County|SD|55124|United States|-6|single family| +42395|AAAAAAAALJFKAAAA|636|Park |Lane|Suite W|Oak Hill|Autauga County|AL|37838|United States|-6|apartment| +42396|AAAAAAAAMJFKAAAA|789|6th |Wy|Suite I|Ashland|Lancaster County|PA|14244|United States|-5|condo| +42397|AAAAAAAANJFKAAAA|428|Oak |Cir.|Suite 300|Lebanon|Hancock County|IL|62898|United States|-6|single family| +42398|AAAAAAAAOJFKAAAA|983|Cedar |Street|Suite D|Union Hill|Minnehaha County|SD|57746|United States|-7|single family| +42399|AAAAAAAAPJFKAAAA|304|Smith |Boulevard|Suite 390|Springfield|Lanier County|GA|39303|United States|-5|apartment| +42400|AAAAAAAAAKFKAAAA|309|Meadow |Pkwy|Suite M|Riverview|Richland County|IL|69003|United States|-6|single family| +42401|AAAAAAAABKFKAAAA|750|8th |Street|Suite H|Providence|Adams County|WA|96614|United States|-8|single family| +42402|AAAAAAAACKFKAAAA|754|Main Pine|Blvd||Crossroads|Kodiak Island Borough||||-9|| +42403|AAAAAAAADKFKAAAA|563|Center Maple|Way|Suite E|Fairfield|Wabash County|IL|66192|United States|-6|apartment| +42404|AAAAAAAAEKFKAAAA|199|2nd 11th|Ln|Suite Y|Ashland|Howard County|NE|64244|United States|-7|apartment| +42405|AAAAAAAAFKFKAAAA|797|Franklin Meadow|Blvd|Suite 390|Mount Zion|Caledonia County|VT|08654|United States|-5|condo| +42406|AAAAAAAAGKFKAAAA|674|5th |Ln|Suite F|Union Hill|Quay County|NM|87746|United States|-7|single family| +42407|AAAAAAAAHKFKAAAA|962|Davis Jackson|Blvd|Suite 350|Florence|Becker County|MN|53394|United States|-6|single family| +42408|AAAAAAAAIKFKAAAA|134|Dogwood |Road|Suite 440|Forest Hills|Greene County|IN|49237|United States|-5|condo| +42409|AAAAAAAAJKFKAAAA|176|Highland |Dr.|Suite Y|Oakdale|Wexford County|MI|49584|United States|-5|condo| +42410|AAAAAAAAKKFKAAAA|459|Birch |Dr.|Suite 0|Willis|Skamania County|WA|96788|United States|-8|apartment| +42411|AAAAAAAALKFKAAAA||College River||||Franklin County|||United States||| +42412|AAAAAAAAMKFKAAAA|445|Church 5th|Lane|Suite I|Five Forks|Wayne County|PA|12293|United States|-5|condo| +42413|AAAAAAAANKFKAAAA|711|3rd |Wy|Suite M|Concord|San Augustine County|TX|74107|United States|-6|single family| +42414|AAAAAAAAOKFKAAAA|165|Maple Hickory|RD|Suite 400|Belmont|Delaware County|IA|50191|United States|-6|condo| +42415|AAAAAAAAPKFKAAAA|251|Meadow First|ST|Suite K|Fairfield|Dawson County|MT|66192|United States|-7|condo| +42416|AAAAAAAAALFKAAAA|536|Church |Court|Suite W|Brownsville|De Kalb County|IN|49310|United States|-5|condo| +42417|AAAAAAAABLFKAAAA|964|Spruce Lincoln|Court|Suite Q|Greenville|Bergen County|NJ|01987|United States|-5|apartment| +42418|AAAAAAAACLFKAAAA|521|14th |Boulevard|Suite 120|Sulphur Springs|Louisa County|VA|28354|United States|-5|single family| +42419|AAAAAAAADLFKAAAA|611|Oak Broadway|Road|Suite 310|Star|Knox County|MO|60725|United States|-6|condo| +42420|AAAAAAAAELFKAAAA|167|Franklin |Wy|Suite 260|Lebanon|Shawano County|WI|52898|United States|-6|single family| +42421|AAAAAAAAFLFKAAAA|636|Birch |Blvd|Suite V|Oakland|Hickman County|KY|49843|United States|-6|condo| +42422|AAAAAAAAGLFKAAAA|91|Lakeview Woodland|Blvd|Suite 260|Stringtown|Elk County|PA|10162|United States|-5|apartment| +42423|AAAAAAAAHLFKAAAA|423|Lake |Circle|Suite 370|Four Points|Fillmore County|NE|61216|United States|-6|single family| +42424|AAAAAAAAILFKAAAA|355|9th Lee|Lane|Suite D|Antioch|Red River County|TX|78605|United States|-6|condo| +42425|AAAAAAAAJLFKAAAA|42|Railroad Pine|Pkwy|Suite R|Buena Vista|Collingsworth County|TX|75752|United States|-6|single family| +42426|AAAAAAAAKLFKAAAA|183|View Davis|Avenue|Suite 90|Newport|Lancaster County|NE|61521|United States|-7|condo| +42427|AAAAAAAALLFKAAAA|846|Pine |Ln|Suite L|Woodland|Richmond County|VA|24854|United States|-5|condo| +42428|AAAAAAAAMLFKAAAA|94|Highland Washington|Ln|Suite W|Fairfield|Refugio County|TX|76192|United States|-6|single family| +42429|AAAAAAAANLFKAAAA|80|Sunset Pine|Way|Suite 80|Brookwood|Schuyler County|MO|60965|United States|-6|single family| +42430|AAAAAAAAOLFKAAAA|744|Madison |Parkway|Suite L|Sulphur Springs|Jefferson County|GA|38354|United States|-5|single family| +42431|AAAAAAAAPLFKAAAA|868|8th 1st|Ct.|Suite 260|Amity|East Carroll Parish|LA|70766|United States|-6|single family| +42432|AAAAAAAAAMFKAAAA|528|Cedar |Blvd|Suite D|Wildwood|Jasper County|IL|66871|United States|-6|single family| +42433|AAAAAAAABMFKAAAA|863|Valley View|Pkwy|Suite 160|Pleasant Grove|Fayette County|IL|64136|United States|-6|apartment| +42434|AAAAAAAACMFKAAAA|167|6th Maple|Court|Suite 50|White Oak|Prowers County|CO|86668|United States|-7|apartment| +42435|AAAAAAAADMFKAAAA|212|Railroad |RD|Suite 190|Franklin|Gadsden County|FL|39101|United States|-5|condo| +42436|AAAAAAAAEMFKAAAA|10|Church Chestnut|Dr.|Suite 230|Fisher|Mississippi County|MO|62819|United States|-6|apartment| +42437|AAAAAAAAFMFKAAAA|414|Lake |Parkway|Suite 100|Deerfield|Bedford County|VA|29840|United States|-5|condo| +42438|AAAAAAAAGMFKAAAA|584|Oak |Dr.|Suite 480|Florence|Chippewa County|MI|43394|United States|-5|single family| +42439|AAAAAAAAHMFKAAAA|969|10th Center|Ct.|Suite Y|Lone Oak|Russell County|VA|26893|United States|-5|condo| +42440|AAAAAAAAIMFKAAAA|400|West Elm|ST|Suite 40|Woodland|Washington County|VT|05454|United States|-5|condo| +42441|AAAAAAAAJMFKAAAA|609|Laurel |Street|Suite 220|Antioch|Lipscomb County|TX|78605|United States|-6|apartment| +42442|AAAAAAAAKMFKAAAA|913|View Second|Way|Suite C|Forest Hills|Edgar County|IL|69237|United States|-6|condo| +42443|AAAAAAAALMFKAAAA|411|Hill |Boulevard|Suite E|Brownsville|Kanabec County|MN|59310|United States|-6|single family| +42444|AAAAAAAAMMFKAAAA|407|9th 5th|Way|Suite C|Lakewood|Winnebago County|WI|58877|United States|-6|apartment| +42445|AAAAAAAANMFKAAAA|376|Locust Center|Pkwy|Suite 280|Sunnyside|Hampden County|MA|02552|United States|-5|single family| +42446|AAAAAAAAOMFKAAAA|532|Center |Pkwy|Suite 220|Franklin|Essex County|NY|19101|United States|-5|condo| +42447|AAAAAAAAPMFKAAAA|944|Third |Circle|Suite 20|Wilton|Davidson County|TN|36997|United States|-5|apartment| +42448|AAAAAAAAANFKAAAA|596|Dogwood |Boulevard|Suite 140|Pleasant Valley|Bristol city|VA|22477|United States|-5|condo| +42449|AAAAAAAABNFKAAAA|390|Ash |RD|Suite 450|Salem|Wayne County|MO|68048|United States|-6|single family| +42450|AAAAAAAACNFKAAAA|760|7th |Court|Suite 330|Mount Pleasant|Washington County|NE|61933|United States|-7|single family| +42451|AAAAAAAADNFKAAAA|30|View |Way|Suite 270|Greenfield|Douglas County|WA|95038|United States|-8|single family| +42452|AAAAAAAAENFKAAAA|230|Cherry |Ln|Suite A|Brownsville|Hickman County|KY|49310|United States|-6|apartment| +42453|AAAAAAAAFNFKAAAA|904|Main |Cir.|Suite H|Woodville|Bonneville County|ID|84289|United States|-7|single family| +42454|AAAAAAAAGNFKAAAA|989|Washington |Circle|Suite I|Hopewell|Adams County|IL|60587|United States|-6|condo| +42455|AAAAAAAAHNFKAAAA|551|Walnut |Boulevard|Suite C|Clinton|Brooke County|WV|28222|United States|-5|apartment| +42456|AAAAAAAAINFKAAAA|868|7th |Parkway|Suite G|Lebanon|Lawrence County|IN|42898|United States|-5|apartment| +42457|AAAAAAAAJNFKAAAA|792|5th Mill|Street|Suite 190|Pleasant Hill|Benton County|MN|53604|United States|-6|single family| +42458|AAAAAAAAKNFKAAAA|130|Maple |Ct.|Suite E|Marion|Otter Tail County|MN|50399|United States|-6|single family| +42459|AAAAAAAALNFKAAAA|800|North Second|Parkway|Suite R|Buena Vista|Lyon County|IA|55752|United States|-6|condo| +42460|AAAAAAAAMNFKAAAA||Third Wilson|Cir.||Mount Pleasant|Tarrant County||||-6|| +42461|AAAAAAAANNFKAAAA|658|Highland |Drive|Suite J|Farmington|Henderson County|IL|69145|United States|-6|single family| +42462|AAAAAAAAONFKAAAA|517|College |Circle|Suite 200|Mount Vernon|Cheshire County|NH|09082|United States|-5|apartment| +42463|AAAAAAAAPNFKAAAA|918|Second |Drive|Suite 300|Antioch|Lee County|IL|68605|United States|-6|single family| +42464|AAAAAAAAAOFKAAAA|915|First |Avenue|Suite M|Plainville|Spokane County|WA|96115|United States|-8|single family| +42465|AAAAAAAABOFKAAAA|197|Eigth |Parkway|Suite Q|Lebanon|Wade Hampton Census Area|AK|92898|United States|-9|single family| +42466|AAAAAAAACOFKAAAA|65|West |Street|Suite G|Hillcrest|Wade Hampton Census Area|AK|93003|United States|-9|apartment| +42467|AAAAAAAADOFKAAAA|807|Hillcrest |Pkwy|Suite 190|Hopewell|Yankton County|SD|50587|United States|-6|condo| +42468|AAAAAAAAEOFKAAAA|855|Cherry 6th|Parkway|Suite 320|Five Forks|Garfield County|UT|82293|United States|-7|single family| +42469|AAAAAAAAFOFKAAAA|771|Oak Franklin|Cir.|Suite 430|Oak Ridge|Johnson County|MO|68371|United States|-6|single family| +42470|AAAAAAAAGOFKAAAA|188|Fifth |ST|Suite G|Rosebud|Clay County|NC|22244|United States|-5|condo| +42471|AAAAAAAAHOFKAAAA|701|Fourth 1st|Ln|Suite B|Plainview|Bradley County|AR|73683|United States|-6|single family| +42472|AAAAAAAAIOFKAAAA|502|Main |Lane|Suite 370|Deerfield|Winona County|MN|59840|United States|-6|condo| +42473|AAAAAAAAJOFKAAAA|277|Lake Smith|Lane|Suite N|Omega|Matanuska-Susitna Borough|AK|96364|United States|-9|condo| +42474|AAAAAAAAKOFKAAAA|44|Lee Ridge|Road|Suite H|Mountain View|Barren County|KY|44466|United States|-6|single family| +42475|AAAAAAAALOFKAAAA|717|Oak 9th|Ave|Suite Q|Youngstown|Boyd County|KY|40001|United States|-6|single family| +42476|AAAAAAAAMOFKAAAA|967|Pine |Wy|Suite 470|Brownsville|Gunnison County|CO|89310|United States|-7|condo| +42477|AAAAAAAANOFKAAAA|669|Spruce Lakeview|Blvd|Suite Y|White Oak|Johnson County|KY|46668|United States|-6|condo| +42478|AAAAAAAAOOFKAAAA|604|Meadow Lakeview|Wy|Suite 430|Lebanon|Potter County|SD|52898|United States|-7|single family| +42479|AAAAAAAAPOFKAAAA|395|1st Pine|Blvd|Suite 350|Evans|Hancock County|IA|52284|United States|-6|apartment| +42480|AAAAAAAAAPFKAAAA|515|Pine |Lane|Suite 180|Shelby|Union County|OR|96575|United States|-8|apartment| +42481|AAAAAAAABPFKAAAA|958|River Main|Street|Suite E|Pleasant Valley|Seward County|NE|62477|United States|-7|apartment| +42482|AAAAAAAACPFKAAAA|889|7th Second|Pkwy|Suite T|Greenville|Worth County|GA|31387|United States|-5|single family| +42483|AAAAAAAADPFKAAAA|67|Cherry Hillcrest|Ave|Suite L|Pleasant Valley|Lewis County|MO|62477|United States|-6|single family| +42484|AAAAAAAAEPFKAAAA|968|Davis Railroad|Circle|Suite 410|Lebanon|El Paso County|TX|72898|United States|-6|apartment| +42485|AAAAAAAAFPFKAAAA|362|13th Hill|RD|Suite 160|Plainview|Richmond County|NY|13683|United States|-5|apartment| +42486|AAAAAAAAGPFKAAAA|734|River Church|Ct.|Suite W|Glendale|McPherson County|NE|63951|United States|-7|single family| +42487|AAAAAAAAHPFKAAAA|675|Franklin 3rd|RD|Suite B|Kingston|Collin County|TX|74975|United States|-6|apartment| +42488|AAAAAAAAIPFKAAAA|637|Adams |Ave|Suite J|Bunker Hill|Grant County|SD|50150|United States|-6|single family| +42489|AAAAAAAAJPFKAAAA|423|North |Ln|Suite G|Lebanon|Granville County|NC|22898|United States|-5|single family| +42490|AAAAAAAAKPFKAAAA|668|View 3rd|Street|Suite 10|Hopewell|Isanti County|MN|50587|United States|-6|single family| +42491|AAAAAAAALPFKAAAA|380|Washington |Wy|Suite 180|Plainview|Jackson County|TN|33683|United States|-5|condo| +42492|AAAAAAAAMPFKAAAA|138|Elm Johnson|Court|Suite U|Elkton|Warren County|TN|33481|United States|-6|apartment| +42493|AAAAAAAANPFKAAAA|104|Highland View|Wy|Suite 390|Spring Valley|Jefferson County|KS|66060|United States|-6|single family| +42494|AAAAAAAAOPFKAAAA|470|Smith 4th|Drive|Suite G|Jamestown|Madison County|MT|66867|United States|-7|apartment| +42495|AAAAAAAAPPFKAAAA|553|Poplar North|Ct.|Suite 460|Springfield|Palo Alto County|IA|59303|United States|-6|apartment| +42496|AAAAAAAAAAGKAAAA|163|Willow 5th|Lane|Suite 60|Belmont|Pasco County|FL|30191|United States|-5|single family| +42497|AAAAAAAABAGKAAAA|195|Valley 3rd|Parkway|Suite 100|Mount Olive|Steuben County|IN|48059|United States|-5|apartment| +42498|AAAAAAAACAGKAAAA|304|Main |Blvd|Suite 260|Midway|Pike County|AL|31904|United States|-6|condo| +42499|AAAAAAAADAGKAAAA|993|College Adams|Wy|Suite 410|Farmington|Floyd County|VA|29145|United States|-5|single family| +42500|AAAAAAAAEAGKAAAA|207|Washington |Pkwy|Suite X|Springfield|Clarke County|GA|39303|United States|-5|apartment| +42501|AAAAAAAAFAGKAAAA|318|7th |Pkwy|Suite K|Harmony|Dewey County|OK|75804|United States|-6|apartment| +42502|AAAAAAAAGAGKAAAA|841|Smith |Ln|Suite H|Unionville|Lee County|AR|71711|United States|-6|apartment| +42503|AAAAAAAAHAGKAAAA|56|Adams 8th|ST|Suite 450|Centerville|Albemarle County|VA|20059|United States|-5|single family| +42504|AAAAAAAAIAGKAAAA||Laurel |Dr.|Suite M|Greenwood|Wells County|ND||||single family| +42505|AAAAAAAAJAGKAAAA|826|Cherry Maple|Boulevard|Suite 130|Shiloh|Washington County|ID|89275|United States|-7|single family| +42506|AAAAAAAAKAGKAAAA|14|South 10th|Pkwy|Suite 70|Plainview|Karnes County|TX|73683|United States|-6|apartment| +42507|AAAAAAAALAGKAAAA|64|Center |Ct.|Suite T|Ashland|Linn County|OR|94244|United States|-8|single family| +42508|AAAAAAAAMAGKAAAA|609|Cedar |Dr.|Suite 90|Oakwood|Runnels County|TX|70169|United States|-6|condo| +42509|AAAAAAAANAGKAAAA|727|View |Court|Suite 330|Mount Olive|Livingston County|NY|18059|United States|-5|apartment| +42510|AAAAAAAAOAGKAAAA|641|Ash |Way|Suite 350|Stringtown|Jefferson County|MT|60162|United States|-7|condo| +42511|AAAAAAAAPAGKAAAA|983|6th Hickory|Drive|Suite 330|Walnut Grove|Bienville Parish|LA|77752|United States|-6|single family| +42512|AAAAAAAAABGKAAAA|55|Franklin Wilson|ST|Suite 260|Five Forks|Geary County|KS|62293|United States|-6|apartment| +42513|AAAAAAAABBGKAAAA|411|1st Oak|Dr.|Suite R|Hopewell|Howard County|AR|70587|United States|-6|condo| +42514|AAAAAAAACBGKAAAA||Park |Circle|Suite H|Summit|Granville County|NC|20499|United States|-5|| +42515|AAAAAAAADBGKAAAA|251|Meadow |RD|Suite 20|Lakewood|Brown County|KS|68877|United States|-6|single family| +42516|AAAAAAAAEBGKAAAA|169|Oak Mill|Ct.|Suite T|Five Forks|Lake County|IN|42293|United States|-5|apartment| +42517|AAAAAAAAFBGKAAAA|290|Williams |ST|Suite F|Washington Heights|Big Horn County|MT|68167|United States|-7|apartment| +42518|AAAAAAAAGBGKAAAA|747|6th 7th|Drive|Suite 460|Hopewell|Oneida County|ID|80587|United States|-7|single family| +42519|AAAAAAAAHBGKAAAA|122|8th Oak|RD|Suite V|Oakland|Runnels County|TX|79843|United States|-6|condo| +42520|AAAAAAAAIBGKAAAA|369|13th |Cir.|Suite P|Hamilton|Cameron Parish|LA|72808|United States|-6|single family| +42521|AAAAAAAAJBGKAAAA|562|3rd |Boulevard|Suite 280|Jackson|Somerset County|PA|19583|United States|-5|single family| +42522|AAAAAAAAKBGKAAAA|227|Oak |Boulevard|Suite 150|Forest Hills|Jeff Davis County|GA|39237|United States|-5|apartment| +42523|AAAAAAAALBGKAAAA|585|Maple |Circle|Suite S|Oak Hill|Newport County|RI|08438|United States|-5|condo| +42524|AAAAAAAAMBGKAAAA|264|Adams |Circle|Suite W|Highland Park|Sheridan County|KS|66534|United States|-6|condo| +42525|AAAAAAAANBGKAAAA|344|Sixth |Cir.|Suite 20|Arlington|Prentiss County|MS|56557|United States|-6|condo| +42526|AAAAAAAAOBGKAAAA|92|Williams |ST|Suite O|Newtown|Bremer County|IA|51749|United States|-6|condo| +42527|AAAAAAAAPBGKAAAA|258|1st East|Lane|Suite U|Pine Grove|El Dorado County|CA|94593|United States|-8|apartment| +42528|AAAAAAAAACGKAAAA|319|View 15th|Drive|Suite 310|Clifton|Washington County|TX|78014|United States|-6|single family| +42529|AAAAAAAABCGKAAAA|405|North Franklin|Street|Suite 350|Buena Vista|Scott County|IA|55752|United States|-6|condo| +42530|AAAAAAAACCGKAAAA|850|Davis |Parkway|Suite 370|Glendale|Lauderdale County|TN|33951|United States|-6|condo| +42531|AAAAAAAADCGKAAAA|89|Poplar Spring|Dr.|Suite 390|Hillcrest|Calhoun County|FL|33003|United States|-5|condo| +42532|AAAAAAAAECGKAAAA|917|Main |Blvd|Suite 380|Waterloo|Dolores County|CO|81675|United States|-7|apartment| +42533|AAAAAAAAFCGKAAAA|818|Sunset |Ln|Suite 340|Union|Iron County|MO|68721|United States|-6|condo| +42534|AAAAAAAAGCGKAAAA|764|Valley |ST|Suite 120|Floyd|Wakulla County|FL|33235|United States|-5|apartment| +42535|AAAAAAAAHCGKAAAA|847|Lake |Dr.|Suite 460|Kelly|Chatham County|NC|22738|United States|-5|apartment| +42536|AAAAAAAAICGKAAAA|608|Central Lake|Ct.|Suite 460|New Hope|Sherman County|TX|79431|United States|-6|condo| +42537|AAAAAAAAJCGKAAAA|237|Johnson |Parkway|Suite H|Pleasant Hill|Cameron Parish|LA|73604|United States|-6|condo| +42538|AAAAAAAAKCGKAAAA|889|1st |Drive|Suite 440|Harmony|Henderson County|TX|75804|United States|-6|condo| +42539|AAAAAAAALCGKAAAA|129|Ash East|Cir.|Suite F|Stringtown|Josephine County|OR|90162|United States|-8|single family| +42540|AAAAAAAAMCGKAAAA|709|South |Way|Suite 230|Florence|Ottawa County|OH|43394|United States|-5|single family| +42541|AAAAAAAANCGKAAAA|366|Meadow |Blvd|Suite C|Clifton|Menard County|IL|68014|United States|-6|apartment| +42542|AAAAAAAAOCGKAAAA|300|Washington Washington|Street|Suite 300|Clinton|San Francisco County|CA|98222|United States|-8|condo| +42543|AAAAAAAAPCGKAAAA|229|8th |Lane|Suite 120|Oakdale|Palo Pinto County|TX|79584|United States|-6|condo| +42544|AAAAAAAAADGKAAAA|626|Laurel South|Blvd|Suite A|Farmington|Anchorage Borough|AK|99145|United States|-9|apartment| +42545|AAAAAAAABDGKAAAA|375|Hill Elm|Parkway|Suite 460|Pleasant Hill|Dorchester County|MD|23604|United States|-5|condo| +42546|AAAAAAAACDGKAAAA|656|Forest Railroad|Ln|Suite R|Highland Park|Broadwater County|MT|66534|United States|-7|condo| +42547|AAAAAAAADDGKAAAA|408|Adams Jefferson|Blvd|Suite 200|Fairfield|Sumter County|AL|36192|United States|-6|apartment| +42548|AAAAAAAAEDGKAAAA|883|Pine Maple|Dr.|Suite Y|Five Forks|Columbia County|NY|12293|United States|-5|single family| +42549|AAAAAAAAFDGKAAAA|865|Sycamore Spruce|Ct.|Suite 390|Newtown|Dawson County|MT|61749|United States|-7|single family| +42550|AAAAAAAAGDGKAAAA|856|South Fourteenth|Way|Suite V|Arlington|Crenshaw County|AL|36557|United States|-6|apartment| +42551|AAAAAAAAHDGKAAAA|644|13th |RD|Suite R|Arlington|Castro County|TX|76557|United States|-6|apartment| +42552|AAAAAAAAIDGKAAAA|921|2nd |Drive|Suite 320|Georgetown|Douglas County|WI|57057|United States|-6|apartment| +42553|AAAAAAAAJDGKAAAA|880|3rd 6th|RD|Suite Q|Lakeside|Otoe County|NE|69532|United States|-7|apartment| +42554|AAAAAAAAKDGKAAAA|57|Jackson Oak|Court|Suite 480|Church Hill|Penobscot County|ME|04390|United States|-5|condo| +42555|AAAAAAAALDGKAAAA|115|11th Sunset|Avenue|Suite 140|Georgetown|Camden County|NC|27057|United States|-5|apartment| +42556|AAAAAAAAMDGKAAAA|982|Williams College|RD|Suite 470|Bridgeport|Cape Girardeau County|MO|65817|United States|-6|apartment| +42557|AAAAAAAANDGKAAAA|392|Meadow |Cir.|Suite 40|Deerfield|Mifflin County|PA|19840|United States|-5|condo| +42558|AAAAAAAAODGKAAAA|684|East Walnut|RD|Suite M|Frogtown|Coles County|IL|68784|United States|-6|single family| +42559|AAAAAAAAPDGKAAAA|859|Davis 2nd|Dr.|Suite M|Belmont|Okmulgee County|OK|70191|United States|-6|condo| +42560|AAAAAAAAAEGKAAAA|239|Hill |RD|Suite H|Harmony|Campbell County|VA|25804|United States|-5|condo| +42561|AAAAAAAABEGKAAAA|20|10th |Circle|Suite 30|Sulphur Springs|Texas County|MO|68354|United States|-6|apartment| +42562|AAAAAAAACEGKAAAA|163|Smith Maple|Lane|Suite 40|Wilson|Jefferson County|IA|56971|United States|-6|condo| +42563|AAAAAAAADEGKAAAA|592|Washington |ST|Suite 100|Pleasant Valley|Mille Lacs County|MN|52477|United States|-6|apartment| +42564|AAAAAAAAEEGKAAAA|118|Park |Court|Suite S|Spring Hill|Brantley County|GA|36787|United States|-5|apartment| +42565|AAAAAAAAFEGKAAAA|39|Hickory |Way|Suite 140|Ashland|Morgan County|IN|44244|United States|-5|condo| +42566|AAAAAAAAGEGKAAAA|44|Second |Cir.|Suite S|Woodland|Dorchester County|SC|24854|United States|-5|apartment| +42567|AAAAAAAAHEGKAAAA|217|2nd |Way|Suite M|Sunnyside|Clinton County|OH|41952|United States|-5|condo| +42568|AAAAAAAAIEGKAAAA|58|Green 4th|Dr.|Suite 280|Dewey|Shiawassee County|MI|41160|United States|-5|condo| +42569|AAAAAAAAJEGKAAAA|211|College |Circle|Suite 480|Lakewood|DeKalb County|MO|68877|United States|-6|single family| +42570|AAAAAAAAKEGKAAAA|531|Cedar North|Dr.|Suite U|Spring Valley|Aleutians East Borough|AK|96060|United States|-9|single family| +42571|AAAAAAAALEGKAAAA|102|Main |Blvd|Suite K|Green Acres|Salem County|NJ|08283|United States|-5|condo| +42572|AAAAAAAAMEGKAAAA|654|Fourteenth 10th|Court|Suite L|Lakeview|Hidalgo County|TX|78579|United States|-6|apartment| +42573|AAAAAAAANEGKAAAA|149|Highland |Blvd|Suite A|Springfield|Lee County|IA|59303|United States|-6|single family| +42574|AAAAAAAAOEGKAAAA|919|Jackson River|Wy|Suite Q|Lakeside|Yavapai County|AZ|89532|United States|-7|single family| +42575|AAAAAAAAPEGKAAAA|997|Lincoln |Cir.|Suite 40|Union Hill|Franklin County|FL|37746|United States|-5|condo| +42576|AAAAAAAAAFGKAAAA||Hill Wilson|Circle||Springdale|Clay County||38883|United States||condo| +42577|AAAAAAAABFGKAAAA|861|Park |Drive|Suite E|Summit|Perquimans County|NC|20499|United States|-5|single family| +42578|AAAAAAAACFGKAAAA|322|Adams |Boulevard|Suite 290|Crossroads|New Kent County|VA|20534|United States|-5|apartment| +42579|AAAAAAAADFGKAAAA|948|Third |Pkwy|Suite D|Buena Vista|Bay County|MI|45752|United States|-5|apartment| +42580|AAAAAAAAEFGKAAAA|606|Lee Wilson|Wy|Suite H|Lakeside|Spalding County|GA|39532|United States|-5|apartment| +42581|AAAAAAAAFFGKAAAA|477|View Cedar|Drive|Suite 470|Lakeview|Hunt County|TX|78579|United States|-6|condo| +42582|AAAAAAAAGFGKAAAA|775|Wilson 5th|Avenue|Suite E|Cedar Grove|McPherson County|NE|60411|United States|-7|single family| +42583|AAAAAAAAHFGKAAAA|688|Pine Forest|Ct.|Suite 440|Mount Pleasant|Christian County|IL|61933|United States|-6|condo| +42584|AAAAAAAAIFGKAAAA|210|Hillcrest |Lane|Suite D|Deerfield|Oktibbeha County|MS|59840|United States|-6|condo| +42585|AAAAAAAAJFGKAAAA|125|Cedar |Way|Suite R|Clinton|Wood County|TX|78222|United States|-6|condo| +42586|AAAAAAAAKFGKAAAA|15|Fourth |Drive|Suite B|Bridgeport|Weston County|WY|85817|United States|-7|single family| +42587|AAAAAAAALFGKAAAA|141|Lakeview |Pkwy|Suite K|Lakeview|Amador County|CA|98579|United States|-8|apartment| +42588|AAAAAAAAMFGKAAAA|776|College 5th|Street|Suite L|Five Forks|Cloud County|KS|62293|United States|-6|condo| +42589|AAAAAAAANFGKAAAA|493|Birch |Pkwy|Suite J|Jackson|Anson County|NC|29583|United States|-5|apartment| +42590|AAAAAAAAOFGKAAAA|123|Ridge Sixth|Road|Suite 240|Pleasant Valley|Butler County|KS|62477|United States|-6|apartment| +42591|AAAAAAAAPFGKAAAA|13|4th 10th|Dr.|Suite G|Unionville|Lincoln County|MS|51711|United States|-6|single family| +42592|AAAAAAAAAGGKAAAA|702|10th |Way|Suite J|Sulphur Springs|Aitkin County|MN|58354|United States|-6|condo| +42593|AAAAAAAABGGKAAAA|803|Pine Main|Way|Suite O|Pleasant Valley|Stephens County|TX|72477|United States|-6|single family| +42594|AAAAAAAACGGKAAAA|88|11th |Avenue|Suite J|Union|Franklin County|NY|18721|United States|-5|single family| +42595|AAAAAAAADGGKAAAA|608|Miller First|Dr.|Suite V|Lakeside|Portage County|OH|49532|United States|-5|apartment| +42596|AAAAAAAAEGGKAAAA|632|9th Davis|ST|Suite Y|Waterloo|Allen County|OH|41675|United States|-5|apartment| +42597|AAAAAAAAFGGKAAAA|700|Smith Third|Drive|Suite 120|Spring Hill|Montgomery County|GA|36787|United States|-5|single family| +42598|AAAAAAAAGGGKAAAA|437|2nd |Court|Suite 60|Lone Pine|Hansford County|TX|77441|United States|-6|apartment| +42599|AAAAAAAAHGGKAAAA|733|Hill Seventh|Pkwy|Suite 150|Spring Hill|Dolores County|CO|86787|United States|-7|single family| +42600|AAAAAAAAIGGKAAAA|33|5th Walnut|Dr.|Suite R|Valley View|Haskell County|TX|75124|United States|-6|single family| +42601|AAAAAAAAJGGKAAAA|400|Church |Avenue|Suite 390|San Jose|Stanley County|SD|58003|United States|-7|single family| +42602|AAAAAAAAKGGKAAAA|153|Hickory Birch|Lane|Suite 140|Enterprise|Sacramento County|CA|91757|United States|-8|apartment| +42603|AAAAAAAALGGKAAAA|184|Second |Cir.|Suite C|Mount Pleasant|Twin Falls County|ID|81933|United States|-7|apartment| +42604|AAAAAAAAMGGKAAAA|554|Pine 10th|Dr.|Suite 130|Springfield|Washington County|ME|09903|United States|-5|single family| +42605|AAAAAAAANGGKAAAA|223|Oak Main|Avenue|Suite X|Evans|Dawes County|NE|62284|United States|-6|condo| +42606|AAAAAAAAOGGKAAAA|808|Elevnth |Road|Suite S|Pine Grove|Woodford County|KY|44593|United States|-6|single family| +42607|AAAAAAAAPGGKAAAA|904|Fifth Lake|Way|Suite W|Shady Grove|Wells County|ND|52812|United States|-6|apartment| +42608|AAAAAAAAAHGKAAAA|303|10th |Boulevard|Suite 250|Deerfield|Converse County|WY|89840|United States|-7|condo| +42609|AAAAAAAABHGKAAAA|389|Fourth |Ct.|Suite 450|Lakeview|Cleveland County|OK|78579|United States|-6|condo| +42610|AAAAAAAACHGKAAAA|905|First |Avenue|Suite L|Forest Hills|Butte County|SD|59237|United States|-6|single family| +42611|AAAAAAAADHGKAAAA|832|Sycamore |Boulevard|Suite 460|Liberty|Wood County|WV|23451|United States|-5|single family| +42612|AAAAAAAAEHGKAAAA|686|Hill |Parkway|Suite I|Highland|Mills County|IA|59454|United States|-6|single family| +42613|AAAAAAAAFHGKAAAA|724||Street|Suite X|||||United States|-6|condo| +42614|AAAAAAAAGHGKAAAA|532|Church First|Street|Suite 320|Pleasant Grove|Columbia County|PA|14136|United States|-5|apartment| +42615|AAAAAAAAHHGKAAAA|722|Cherry |Road|Suite L|Avery|Albany County|NY|10194|United States|-5|single family| +42616|AAAAAAAAIHGKAAAA|208|First |Dr.|Suite H|Georgetown|Hooker County|NE|67057|United States|-7|apartment| +42617|AAAAAAAAJHGKAAAA|224|Pine |Drive|Suite V|Florence|Unicoi County|TN|33394|United States|-6|condo| +42618|AAAAAAAAKHGKAAAA|667|Jackson Sycamore|Ave|Suite 300|Pleasant Hill|Todd County|MN|53604|United States|-6|single family| +42619|AAAAAAAALHGKAAAA|988|Jackson Railroad|Parkway|Suite E|Rossville|Itawamba County|MS|52503|United States|-6|single family| +42620|AAAAAAAAMHGKAAAA|623|South Wilson|Circle|Suite 40|Salem|Mitchell County|GA|38048|United States|-5|condo| +42621|AAAAAAAANHGKAAAA|328|Spring Cedar|Boulevard|Suite E|Riverdale|Johnson County|IA|59391|United States|-6|apartment| +42622|AAAAAAAAOHGKAAAA|352|5th Church|Ct.|Suite C|Union|Ottawa County|MI|48721|United States|-5|condo| +42623|AAAAAAAAPHGKAAAA|217|Williams View|Parkway|Suite G|Proctor|Allegan County|MI|48140|United States|-5|apartment| +42624|AAAAAAAAAIGKAAAA|679|Third |Lane|Suite 230|Green Acres|Hamilton County|IN|47683|United States|-5|apartment| +42625|AAAAAAAABIGKAAAA|549|Park |Cir.|Suite N|Caldwell|Union County|SD|56913|United States|-7|apartment| +42626|AAAAAAAACIGKAAAA|354|Highland |Dr.|Suite J|Oak Grove|Vilas County|WI|58370|United States|-6|single family| +42627|AAAAAAAADIGKAAAA|826|Lincoln |Street|Suite R|Waterloo|Champaign County|OH|41675|United States|-5|single family| +42628|AAAAAAAAEIGKAAAA|219|South |RD|Suite Q|Summit|Wadena County|MN|50499|United States|-6|condo| +42629|AAAAAAAAFIGKAAAA|950|4th Lake|Dr.|Suite R|Brookville|Montgomery County|VA|23524|United States|-5|condo| +42630|AAAAAAAAGIGKAAAA|884|First |Cir.|Suite C|Texas|Greene County|NC|23342|United States|-5|condo| +42631|AAAAAAAAHIGKAAAA|277|Jackson College|RD|Suite 110|Walnut Grove|Leelanau County|MI|47752|United States|-5|condo| +42632|AAAAAAAAIIGKAAAA|652|River Sycamore|Cir.|Suite R|Murray|Moody County|SD|52150|United States|-7|condo| +42633|AAAAAAAAJIGKAAAA|421|First |Circle|Suite 160|Harmony|Lincoln County|NC|25804|United States|-5|condo| +42634|AAAAAAAAKIGKAAAA|47||Ave|Suite 210||Polk County|MO|65858||-6|apartment| +42635|AAAAAAAALIGKAAAA|655|Seventh |Ct.|Suite 30|Glendale|Roseau County|MN|53951|United States|-6|apartment| +42636|AAAAAAAAMIGKAAAA|677|7th Third|Ct.|Suite I|Jones|Preston County|WV|22686|United States|-5|apartment| +42637|AAAAAAAANIGKAAAA|640|Lake 5th|Pkwy|Suite 350|Mount Vernon|Alameda County|CA|98482|United States|-8|condo| +42638|AAAAAAAAOIGKAAAA|142|East |Pkwy|Suite A|Greenville|Fluvanna County|VA|21387|United States|-5|apartment| +42639|AAAAAAAAPIGKAAAA|323|River Oak|Road|Suite 120|Leesville|Cook County|IL|65423|United States|-6|single family| +42640|AAAAAAAAAJGKAAAA|839|14th Pine|Circle|Suite Y|Georgetown|Sweet Grass County|MT|67057|United States|-7|single family| +42641|AAAAAAAABJGKAAAA|28|Ash |Pkwy|Suite O|Bunker Hill|Rock County|WI|50150|United States|-6|apartment| +42642|AAAAAAAACJGKAAAA|144|8th Meadow|Blvd|Suite P|Pine Grove|Palo Alto County|IA|54593|United States|-6|condo| +42643|AAAAAAAADJGKAAAA|756|Lake |Ln|Suite E|Georgetown|Hood County|TX|77057|United States|-6|condo| +42644|AAAAAAAAEJGKAAAA|893|2nd 11th|Drive|Suite 390|Spring Hill|Fisher County|TX||United States||| +42645|AAAAAAAAFJGKAAAA|207|Madison |Avenue|Suite U|Glendale|Lake County|CA|93951|United States|-8|condo| +42646|AAAAAAAAGJGKAAAA|568|2nd |Road|Suite V|Howell|Brantley County|GA|34854|United States|-5|condo| +42647|AAAAAAAAHJGKAAAA|362|Meadow West|Boulevard|Suite 120|Lakeview|Sharp County|AR|78579|United States|-6|apartment| +42648|AAAAAAAAIJGKAAAA|58|View |Road|Suite G|Liberty|Coos County|NH|04051|United States|-5|apartment| +42649|AAAAAAAAJJGKAAAA|417|Willow |Road|Suite 360|Ashland|Manatee County|FL|34244|United States|-5|apartment| +42650|AAAAAAAAKJGKAAAA|388|||Suite O|Wright||UT|||-7|| +42651|AAAAAAAALJGKAAAA|438|Washington Green|Ave|Suite 80|Bunker Hill|Morgan County|CO|80150|United States|-7|single family| +42652|AAAAAAAAMJGKAAAA|407|River |Road|Suite 420|Flint|Eastland County|TX|78909|United States|-6|single family| +42653|AAAAAAAANJGKAAAA|343|Ridge |Street|Suite J|Kingston|Nelson County|KY|44975|United States|-5|condo| +42654|AAAAAAAAOJGKAAAA|121|Ash 4th|Court|Suite Y|Hillcrest|Lewis County|TN|33003|United States|-6|apartment| +42655|AAAAAAAAPJGKAAAA|410|Fifth |Parkway|Suite 400|Avoca|Jackson County|AL|30540|United States|-6|condo| +42656|AAAAAAAAAKGKAAAA|143|Birch |Blvd|Suite 360|Berlin|Manitowoc County|WI|54039|United States|-6|apartment| +42657|AAAAAAAABKGKAAAA|121|Highland River|Drive|Suite 430|Antioch|Cascade County|MT|68605|United States|-7|apartment| +42658|AAAAAAAACKGKAAAA|134|Fifth Cherry|ST|Suite 80|Webb|Lake of the Woods County|MN|50899|United States|-6|condo| +42659|AAAAAAAADKGKAAAA|520|Main Lincoln|Cir.|Suite W|New Hope|Clark County|WI|59431|United States|-6|apartment| +42660|AAAAAAAAEKGKAAAA|303|College Maple|Dr.|Suite O|Riverdale|Irion County|TX|79391|United States|-6|apartment| +42661|AAAAAAAAFKGKAAAA|449|6th |Circle|Suite 330|Greenwood|Hanson County|SD|58828|United States|-7|condo| +42662|AAAAAAAAGKGKAAAA|867|Fifth |Pkwy|Suite S|Bayside|Greene County|TN|39550|United States|-5|condo| +42663|AAAAAAAAHKGKAAAA|109|Oak |Cir.|Suite J|Shelby|Lincoln County|WI|56575|United States|-6|apartment| +42664|AAAAAAAAIKGKAAAA|639|Dogwood Ridge|Boulevard|Suite 340|Five Forks|Stanton County|KS|62293|United States|-6|condo| +42665|AAAAAAAAJKGKAAAA|988|Church Center|Street|Suite P|Kingston|Hale County|TX|74975|United States|-6|apartment| +42666|AAAAAAAAKKGKAAAA|917|4th 3rd|Ln|Suite A|Woodville|Klamath County|OR|94289|United States|-8|apartment| +42667|AAAAAAAALKGKAAAA|986|Spring 6th||Suite 340|White Oak|Roanoke city|VA|26668|||apartment| +42668|AAAAAAAAMKGKAAAA|830|North Elm|Street|Suite W|Oakland|Grant County|KS|69843|United States|-6|apartment| +42669|AAAAAAAANKGKAAAA|234|Ridge |Ln|Suite G|Millbrook|Alpine County|CA|97529|United States|-8|single family| +42670|AAAAAAAAOKGKAAAA|212|2nd 1st|Ln|Suite K|Sulphur Springs|Baraga County|MI|48354|United States|-5|apartment| +42671|AAAAAAAAPKGKAAAA|434|Wilson |Cir.|||Martinsville city|||United States||| +42672|AAAAAAAAALGKAAAA|231|5th Cedar|Street|Suite 280|Summit|Moultrie County|IL|60499|United States|-6|single family| +42673|AAAAAAAABLGKAAAA|141|Oak |Ave|Suite 360||||74466||-6|apartment| +42674|AAAAAAAACLGKAAAA|342|Maple |Court|Suite 430|Oakwood|Independence County|AR|70169|United States|-6|single family| +42675|AAAAAAAADLGKAAAA|124|3rd East|Wy|Suite G|Marion|Greene County|IL|60399|United States|-6|apartment| +42676|AAAAAAAAELGKAAAA|||Boulevard||Shiloh||TX|79275|United States||| +42677|AAAAAAAAFLGKAAAA|318|6th Second|Ln|Suite J|Pleasant Grove|Northampton County|VA|24136|United States|-5|single family| +42678|AAAAAAAAGLGKAAAA|947|College Lincoln|Ln|Suite 200|Millbrook|Hunt County|TX|77529|United States|-6|single family| +42679|AAAAAAAAHLGKAAAA|772|Ridge |Court|Suite L|Jackson|Harrison County|WV|29583|United States|-5|single family| +42680|AAAAAAAAILGKAAAA|74|Laurel Maple|Cir.|Suite 420|Ashland|Orange County|TX|74244|United States|-6|condo| +42681|AAAAAAAAJLGKAAAA|493|15th |Parkway|Suite 260|Crossroads|Menard County|TX|70534|United States|-6|apartment| +42682|AAAAAAAAKLGKAAAA|268|Cedar 4th|Court|Suite 380|Oakwood|Bradley County|TN|30169|United States|-5|condo| +42683|AAAAAAAALLGKAAAA|755|5th |Avenue|Suite 360|Gilmore|Logan County|AR|75464|United States|-6|single family| +42684|AAAAAAAAMLGKAAAA||||Suite 390|Maple Grove|Madison County||||-6|single family| +42685|AAAAAAAANLGKAAAA|691|Park Pine|Ct.|Suite 320|Salem|Sherman County|TX|78048|United States|-6|apartment| +42686|AAAAAAAAOLGKAAAA|670|Elevnth |Ct.|Suite I|Oak Hill|Lee County|SC|27838|United States|-5|apartment| +42687|AAAAAAAAPLGKAAAA|270|Chestnut Dogwood|Cir.|Suite 420|Lewis|Sioux County|NE|67066|United States|-7|apartment| +42688|AAAAAAAAAMGKAAAA|62|Highland Poplar|Lane|Suite M|Mount Pleasant|Hancock County|IN|41933|United States|-5|condo| +42689|AAAAAAAABMGKAAAA|373|Meadow |Court|Suite S|Springfield|Montgomery County|MO|69303|United States|-6|condo| +42690|AAAAAAAACMGKAAAA|756|Third |Road|Suite 70|Red Rock|Hawaii County|HI|94169|United States|-10|apartment| +42691|AAAAAAAADMGKAAAA|166|Sycamore Railroad|Ln|Suite 370|Harmony|Custer County|CO|85804|United States|-7|single family| +42692|AAAAAAAAEMGKAAAA|792|2nd |Ave|Suite L|Harmony|Starr County|TX|75804|United States|-6|apartment| +42693|AAAAAAAAFMGKAAAA|990|Elm |Drive|Suite 470|Harmony|Stone County|AR|75804|United States|-6|apartment| +42694|AAAAAAAAGMGKAAAA|101|Willow Spruce|Cir.|Suite 330|Riverdale|San Diego County|CA|99391|United States|-8|condo| +42695|AAAAAAAAHMGKAAAA|495|Sycamore |Pkwy|Suite O|Union|Morton County|ND|58721|United States|-6|condo| +42696|AAAAAAAAIMGKAAAA|17|Willow Fourteenth|Cir.|Suite 80|Sulphur Springs|Bradley County|AR|78354|United States|-6|apartment| +42697|AAAAAAAAJMGKAAAA|808|Oak Thirteenth|Drive|Suite E|Riverview|Harrison County|IN|49003|United States|-5|condo| +42698|AAAAAAAAKMGKAAAA|254|Dogwood Chestnut|RD|Suite 100|Five Points|Imperial County|CA|96098|United States|-8|apartment| +42699|AAAAAAAALMGKAAAA|821|Spring Church|Dr.|Suite N|Mount Zion|McLeod County|MN|58054|United States|-6|condo| +42700|AAAAAAAAMMGKAAAA|605|Woodland |Way|Suite I|White Oak|Davison County|SD|56668|United States|-6|condo| +42701|AAAAAAAANMGKAAAA|938|Pine |Road|Suite 360|Waterloo|Mora County|NM|81675|United States|-7|condo| +42702|AAAAAAAAOMGKAAAA|779|Park North|Parkway|Suite 20|Woodland|Siskiyou County|CA|94854|United States|-8|single family| +42703|AAAAAAAAPMGKAAAA|548|Cherry |Ct.|Suite 470|Willis|Washington County|IA|56788|United States|-6|apartment| +42704|AAAAAAAAANGKAAAA|17|Park |Drive|Suite E|Kingston|Arapahoe County|CO|84975|United States|-7|apartment| +42705|AAAAAAAABNGKAAAA|671|Laurel |Road|Suite A|Maple Grove|Monroe County|IN|48252|United States|-5|condo| +42706|AAAAAAAACNGKAAAA|122|6th Laurel|Circle|Suite X|Yorktown|Perkins County|NE|60732|United States|-7|apartment| +42707|AAAAAAAADNGKAAAA|973|North |Cir.|Suite 50|Georgetown|Pinal County|AZ|87057|United States|-7|apartment| +42708|AAAAAAAAENGKAAAA|692|Valley 5th|Street|Suite 330|Spring Valley|Watauga County|NC|26060|United States|-5|condo| +42709|AAAAAAAAFNGKAAAA|374|Elm Willow|Blvd|Suite 220|Spring Hill|Ashland County|OH|46787|United States|-5|condo| +42710|AAAAAAAAGNGKAAAA|364|Forest View|Ln|Suite I|Hillcrest|Ellsworth County|KS|63003|United States|-6|single family| +42711|AAAAAAAAHNGKAAAA|885|Johnson Spruce|Pkwy|Suite M|New Hope|Madison County|KY|49431|United States|-5|apartment| +42712|AAAAAAAAINGKAAAA|372|8th |Parkway|Suite E|Mount Olive|Walker County|AL|38059|United States|-6|condo| +42713|AAAAAAAAJNGKAAAA|489|Oak Sycamore|Ave|Suite 480|Bridgeport|Kimball County|NE|65817|United States|-7|condo| +42714|AAAAAAAAKNGKAAAA|90|Williams 14th|Dr.|Suite 440|Fairview|Chickasaw County|IA|55709|United States|-6|condo| +42715|AAAAAAAALNGKAAAA|716|Walnut |Boulevard|Suite W|Oakwood|Rusk County|WI|50169|United States|-6|condo| +42716|AAAAAAAAMNGKAAAA|492|1st Lincoln|Parkway|Suite W|Macedonia|Miller County|AR|71087|United States|-6|single family| +42717|AAAAAAAANNGKAAAA|870|Ridge Maple|Avenue|Suite W|Union Hill|Walworth County|WI|57746|United States|-6|condo| +42718|AAAAAAAAONGKAAAA|719|12th |Ct.|Suite 330|Sulphur Springs|Flagler County|FL|38354|United States|-5|condo| +42719|AAAAAAAAPNGKAAAA|349|Park |RD|Suite 380|Oakland|Roanoke city|VA|29843|United States|-5|apartment| +42720|AAAAAAAAAOGKAAAA|488|Chestnut Johnson|Pkwy|Suite T|Greenwood|Rankin County|MS|58828|United States|-6|condo| +42721|AAAAAAAABOGKAAAA|701|Williams Second|Street|Suite X|Waterloo|Wright County|MN|51675|United States|-6|apartment| +42722|AAAAAAAACOGKAAAA|865|10th Meadow|RD|Suite 380|Green Acres|Callaway County|MO|67683|United States|-6|single family| +42723|AAAAAAAADOGKAAAA|258|Oak 8th|ST|Suite 410|Florence|Webster County|GA|33394|United States|-5|condo| +42724|AAAAAAAAEOGKAAAA|227|Walnut Broadway|Cir.|Suite X|Pleasant Grove|McDonald County|MO|64136|United States|-6|condo| +42725|AAAAAAAAFOGKAAAA|997|13th View|Ave|Suite 40|Greenville|Rabun County|GA|31387|United States|-5|condo| +42726|AAAAAAAAGOGKAAAA|331|West |Avenue|Suite I|Mechanicsburg|Allegany County|NY|12219|United States|-5|condo| +42727|AAAAAAAAHOGKAAAA|342|1st |Circle|Suite 60|Lakewood|Livingston County|NY|18877|United States|-5|condo| +42728|AAAAAAAAIOGKAAAA|919|Forest |ST|Suite A|Fox|Reno County|KS|60631|United States|-6|single family| +42729|AAAAAAAAJOGKAAAA|789|Highland |Ave|Suite N|Hopewell|Cassia County|ID|80587|United States|-7|single family| +42730|AAAAAAAAKOGKAAAA|864|Maple Pine|Ln|Suite B|Hillcrest|San Augustine County|TX|73003|United States|-6|condo| +42731|AAAAAAAALOGKAAAA|433|Davis |Pkwy|Suite 300|Mount Olive|Kandiyohi County|MN|58059|United States|-6|apartment| +42732|AAAAAAAAMOGKAAAA|184|2nd West|Avenue|Suite 240|Oakwood|Pocahontas County|IA|50169|United States|-6|condo| +42733|AAAAAAAANOGKAAAA|345|Adams |Cir.|Suite J|Five Forks|Los Angeles County|CA|92293|United States|-8|condo| +42734|AAAAAAAAOOGKAAAA|583|Central Locust|Ct.|Suite C|Nichols|Grant County|KY|47940|United States|-6|condo| +42735|AAAAAAAAPOGKAAAA|697||Wy|||||64289|United States||| +42736|AAAAAAAAAPGKAAAA|587|Fifth |Parkway|Suite 270|New Hope|Clinton County|OH|49431|United States|-5|apartment| +42737|AAAAAAAABPGKAAAA|207|Main |Cir.|Suite 0|Midway|Pawnee County|NE|61904|United States|-7|apartment| +42738|AAAAAAAACPGKAAAA|237|1st Johnson|Dr.|Suite H|Mount Olive|Jefferson County|PA|18059|United States|-5|apartment| +42739|AAAAAAAADPGKAAAA|677|River Ninth|Pkwy|Suite A|Lakeside|Christian County|IL|69532|United States|-6|apartment| +42740|AAAAAAAAEPGKAAAA|974|Davis Spring|Drive|Suite O|Bear Creek|Lawrence County|MO|63075|United States|-6|condo| +42741|AAAAAAAAFPGKAAAA|566|Center |Lane|Suite V|Glenwood|Surry County|VA|23511|United States|-5|apartment| +42742|AAAAAAAAGPGKAAAA|397|Jefferson |Ln|Suite 120|Riverview|Gulf County|FL|39003|United States|-5|single family| +42743|AAAAAAAAHPGKAAAA|283|Railroad |Pkwy|Suite 310|Mount Pleasant|Seminole County|FL|31933|United States|-5|single family| +42744|AAAAAAAAIPGKAAAA|936|Second |Pkwy|Suite 490|White Oak|Delaware County|OK|76668|United States|-6|condo| +42745|AAAAAAAAJPGKAAAA|817||ST||Hillcrest||FL||||| +42746|AAAAAAAAKPGKAAAA|384|15th 8th|RD|Suite 460|Shady Grove|La Paz County|AZ|82812|United States|-7|apartment| +42747|AAAAAAAALPGKAAAA|305|Sunset Wilson|Street|Suite J|Mount Olive|Jackson County|GA|38059|United States|-5|condo| +42748|AAAAAAAAMPGKAAAA|742|Walnut Eigth|Lane|Suite R|Mount Pleasant|Miller County|AR|71933|United States|-6|condo| +42749|AAAAAAAANPGKAAAA|879||Street|Suite 50|Fairfield||MO|66192||-6|condo| +42750|AAAAAAAAOPGKAAAA|751|Williams 4th|Way|Suite F|Liberty|Warren County|TN|33451|United States|-6|single family| +42751|AAAAAAAAPPGKAAAA|||||||NE|60059|||single family| +42752|AAAAAAAAAAHKAAAA|606|Jefferson |Cir.|Suite 50|Greenfield|Hood River County|OR|95038|United States|-8|single family| +42753|AAAAAAAABAHKAAAA|739|Washington |Boulevard|Suite 440|Bridgeport|Chickasaw County|MS|55817|United States|-6|apartment| +42754|AAAAAAAACAHKAAAA|409|Hickory |Court|Suite V|Ashland|Montgomery County|OH|44244|United States|-5|single family| +42755|AAAAAAAADAHKAAAA|819|Wilson 6th|Cir.|Suite 340|Riverside|Knox County|ME|09831|United States|-5|condo| +42756|AAAAAAAAEAHKAAAA|357|Madison 12th|Circle|Suite W|Marion|Alamance County|NC|20399|United States|-5|single family| +42757|AAAAAAAAFAHKAAAA|887|Poplar |Drive|Suite X|Lakeview|McCook County|SD|58579|United States|-7|apartment| +42758|AAAAAAAAGAHKAAAA|106|Lincoln |Blvd|Suite 380|Marion|Leon County|FL|30399|United States|-5|single family| +42759|AAAAAAAAHAHKAAAA|866|Jackson Hickory|Court|Suite 70|Hamilton|Hinsdale County|CO|82808|United States|-7|apartment| +42760|AAAAAAAAIAHKAAAA|416|2nd Chestnut|Lane|Suite F|Riverside|Carter County|OK|79231|United States|-6|condo| +42761|AAAAAAAAJAHKAAAA|583|Maple Hickory|Boulevard|Suite I|Edgewood|Grays Harbor County|WA|90069|United States|-8|condo| +42762|AAAAAAAAKAHKAAAA|656|West ||Suite F|Hillcrest|Rice County||63003|United States||| +42763|AAAAAAAALAHKAAAA|131|Fourth |RD|Suite W|Jamestown|Mifflin County|PA|16867|United States|-5|single family| +42764|AAAAAAAAMAHKAAAA|783|Oak |Ct.|Suite J|Crossroads|Napa County|CA|90534|United States|-8|condo| +42765|AAAAAAAANAHKAAAA|624|14th 1st|Dr.|Suite I|Bloomingdale|Cortland County|NY|11824|United States|-5|apartment| +42766|AAAAAAAAOAHKAAAA|638|Wilson |Ct.|Suite C|Florence|McPherson County|SD|53394|United States|-7|condo| +42767|AAAAAAAAPAHKAAAA|983|3rd |Drive|Suite 360|Antioch|Greensville County|VA|28605|United States|-5|single family| +42768|AAAAAAAAABHKAAAA|406|Park |Cir.|Suite D|Antioch|Carroll County|MO|68605|United States|-6|single family| +42769|AAAAAAAABBHKAAAA|386|Ninth |ST|Suite B|Glenwood|Henry County|MO|63511|United States|-6|single family| +42770|AAAAAAAACBHKAAAA|201|6th |Ln|Suite K|Pleasant Hill|Knox County|IN|43604|United States|-5|condo| +42771|AAAAAAAADBHKAAAA||Seventh |Circle||Antioch|Putnam County|NY|18605|United States|-5|| +42772|AAAAAAAAEBHKAAAA|103|Laurel ||Suite 30|Farmington|Somervell County|||||| +42773|AAAAAAAAFBHKAAAA|116|Park |Drive|Suite I|Fairfield|Stephens County|OK|76192|United States|-6|single family| +42774|AAAAAAAAGBHKAAAA|401|8th Main|Way|Suite 370|Glenwood|Gilmer County|GA|33511|United States|-5|apartment| +42775|AAAAAAAAHBHKAAAA|888|3rd |Road|Suite E|Buena Vista|Curry County|NM|85752|United States|-7|condo| +42776|AAAAAAAAIBHKAAAA|108|Pine |Road|Suite 370|Oakwood|Osage County|MO|60169|United States|-6|apartment| +42777|AAAAAAAAJBHKAAAA|662|1st |Dr.|Suite G|Green Acres|Johnson County|WY|87683|United States|-7|apartment| +42778|AAAAAAAAKBHKAAAA|317|Ninth 3rd|Ln|Suite 140|Shady Grove|Barbour County|WV|22812|United States|-5|apartment| +42779|AAAAAAAALBHKAAAA|305|6th West|Ct.|Suite R|Georgetown|Lawrence County|IN|47057|United States|-5|condo| +42780|AAAAAAAAMBHKAAAA|507|Pine |Wy|Suite 170|Unionville|Ingham County|MI|41711|United States|-5|single family| +42781|AAAAAAAANBHKAAAA|822|Lakeview |Ln|Suite 220|Lakewood|McIntosh County|ND|58877|United States|-6|single family| +42782|AAAAAAAAOBHKAAAA|350|12th |Boulevard|Suite K|Antioch|Sedgwick County|KS|68605|United States|-6|single family| +42783|AAAAAAAAPBHKAAAA|299||Pkwy|Suite Q||||98721|||apartment| +42784|AAAAAAAAACHKAAAA|480|Davis 8th|Wy|Suite J|Woodland|Clark County|SD|54854|United States|-6|apartment| +42785|AAAAAAAABCHKAAAA|150|Johnson |Lane|Suite Q|Mount Vernon|Warren County|OH|48482|United States|-5|condo| +42786|AAAAAAAACCHKAAAA|939|Birch |Ln|Suite 390|Marion|Green County|KY|40399|United States|-6|condo| +42787|AAAAAAAADCHKAAAA|449|5th Elm|Ct.|Suite A|Mountain View|Atoka County|OK|74466|United States|-6|single family| +42788|AAAAAAAAECHKAAAA|867|6th |RD|Suite 130|Macedonia|Laramie County|WY|81087|United States|-7|single family| +42789|AAAAAAAAFCHKAAAA|48|Sunset Spruce|Drive|Suite U|Marion|Warrick County|IN|40399|United States|-5|apartment| +42790|AAAAAAAAGCHKAAAA|940|Adams |Ave|Suite 420|Marion|Twin Falls County|ID|80399|United States|-7|single family| +42791|AAAAAAAAHCHKAAAA|285|Jefferson Sunset|Ct.|Suite 470|Midway|Wilcox County|AL|31904|United States|-6|condo| +42792|AAAAAAAAICHKAAAA|723|Second |Drive|Suite 30|Antioch|Durham County|NC|28605|United States|-5|single family| +42793|AAAAAAAAJCHKAAAA|845|5th North|Ct.|Suite X|Brunswick|Tuscola County|MI|44642|United States|-5|apartment| +42794|AAAAAAAAKCHKAAAA|59|River Poplar|Wy|Suite 280|Harmony|Forsyth County|NC|25804|United States|-5|condo| +42795|AAAAAAAALCHKAAAA|953|Birch |Cir.|Suite V|Arlington|Calhoun County|WV|26557|United States|-5|apartment| +42796|AAAAAAAAMCHKAAAA|303|Mill |Drive|Suite F|Avoca|Howard County|IA|50540|United States|-6|single family| +42797|AAAAAAAANCHKAAAA|760|9th 2nd|Dr.|Suite 440|Crossroads|Dickens County|TX|70534|United States|-6|single family| +42798|AAAAAAAAOCHKAAAA|521|Lincoln |Lane|Suite 390|Forest Hills|Effingham County|GA|39237|United States|-5|single family| +42799|AAAAAAAAPCHKAAAA|298|South |Ct.|Suite L|Plainview|Bell County|KY|43683|United States|-6|condo| +42800|AAAAAAAAADHKAAAA|988|4th |Parkway|Suite 250|Lebanon|Bradford County|FL|32898|United States|-5|condo| +42801|AAAAAAAABDHKAAAA|138|||Suite 460||Champaign County||41904|United States||| +42802|AAAAAAAACDHKAAAA|381|Pine |Way|Suite 430|Green Acres|Natchitoches Parish|LA|77683|United States|-6|single family| +42803|AAAAAAAADDHKAAAA|369|6th |Circle|Suite X|Clinton|Goliad County|TX|78222|United States|-6|single family| +42804|AAAAAAAAEDHKAAAA|9|Smith |Ln|Suite W|Shiloh|Lea County|NM|89275|United States|-7|apartment| +42805|AAAAAAAAFDHKAAAA|282|Chestnut |Ave|Suite 380|Sugar Hill|Rooks County|KS|65114|United States|-6|condo| +42806|AAAAAAAAGDHKAAAA|346|Park |Ave|Suite 420|Brownsville|Madison County|FL|39310|United States|-5|single family| +42807|AAAAAAAAHDHKAAAA|603|Washington |Street|Suite J|Woodland|Mellette County|SD|54854|United States|-7|apartment| +42808|AAAAAAAAIDHKAAAA|457|Cherry Smith|Way|Suite W|Wilson|Bath County|KY|46971|United States|-6|single family| +42809|AAAAAAAAJDHKAAAA|311|Pine |RD|Suite Q|Union Hill|Franklin County|NC|27746|United States|-5|single family| +42810|AAAAAAAAKDHKAAAA|909|11th |Street|Suite Y|Avoca|King George County|VA|20540|United States|-5|single family| +42811|AAAAAAAALDHKAAAA|802|4th |ST|Suite 420|Lakewood|McLean County|ND|58877|United States|-6|apartment| +42812|AAAAAAAAMDHKAAAA|728|Park 15th|Way|Suite 300|Enterprise|Hockley County|TX|71757|United States|-6|apartment| +42813|AAAAAAAANDHKAAAA|909|6th Spring|Ln|Suite A|Wildwood|Shawano County|WI|56871|United States|-6|single family| +42814|AAAAAAAAODHKAAAA|852|First Cedar|Road|Suite 190|Sleepy Hollow|Burt County|NE|63592|United States|-6|single family| +42815|AAAAAAAAPDHKAAAA|963|3rd |Circle|Suite P|Georgetown|Webb County|TX|77057|United States|-6|apartment| +42816|AAAAAAAAAEHKAAAA|501|Fourth Washington|Circle|Suite B|Buena Vista|Bonner County|ID|85752|United States|-7|condo| +42817|AAAAAAAABEHKAAAA|68|Elm Wilson|Circle|Suite 200|Woodland|Daviess County|MO|64854|United States|-6|apartment| +42818|AAAAAAAACEHKAAAA|405|15th Sunset|Boulevard|Suite R|Sunnyside|Madison County|NC|21952|United States|-5|single family| +42819|AAAAAAAADEHKAAAA|283|Hillcrest |Ct.|Suite 360|Newtown|Bulloch County|GA|31749|United States|-5|apartment| +42820|AAAAAAAAEEHKAAAA|740|Jefferson |Circle|Suite 80|Riverside|Tipton County|TN|39231|United States|-6|condo| +42821|AAAAAAAAFEHKAAAA|655|Wilson Lakeview|Pkwy|Suite 240|Bayview|Yakima County|WA|99672|United States|-8|condo| +42822|AAAAAAAAGEHKAAAA|190|Fifth Meadow|ST|Suite 140|Highland|Pickaway County|OH|49454|United States|-5|condo| +42823|AAAAAAAAHEHKAAAA|412|Ninth |Avenue|Suite 50|Mount Olive|Columbia County|GA|38059|United States|-5|single family| +42824|AAAAAAAAIEHKAAAA|864|Mill Church|Dr.|Suite P|Midway|Petersburg city|VA|21904|United States|-5|single family| +42825|AAAAAAAAJEHKAAAA|484|8th |Drive|Suite S|Spring Hill|Washington County|NY|16787|United States|-5|apartment| +42826|AAAAAAAAKEHKAAAA|580|3rd |Way|Suite E|Gilmore|Vernon Parish|LA|75464|United States|-6|condo| +42827|AAAAAAAALEHKAAAA|772|3rd Miller|ST|Suite 400|Oak Hill|Charles City County|VA|27838|United States|-5|condo| +42828|AAAAAAAAMEHKAAAA|474|Washington |Parkway|Suite J|Woodlawn|Amador County|CA|94098|United States|-8|apartment| +42829|AAAAAAAANEHKAAAA|81|South Lake|Pkwy|Suite 390|Valley View|Pacific County|WA|95124|United States|-8|condo| +42830|AAAAAAAAOEHKAAAA|||Cir.|Suite 240|Jackson|Greensville County|||United States||| +42831|AAAAAAAAPEHKAAAA|974|Lake 6th|ST|Suite 90|Salem|Loudoun County|VA|28048|United States|-5|apartment| +42832|AAAAAAAAAFHKAAAA|392|1st |Lane|Suite W|Cordova|Bolivar County|MS|56938|United States|-6|apartment| +42833|AAAAAAAABFHKAAAA|784|Spring 13th|Cir.|Suite H|Mount Vernon|Eau Claire County|WI|58482|United States|-6|single family| +42834|AAAAAAAACFHKAAAA|17|Pine |Drive|Suite T|Friendship|Pasquotank County|NC|24536|United States|-5|single family| +42835|AAAAAAAADFHKAAAA|391|5th |Way|Suite P|Cedar Grove|Lumpkin County|GA|30411|United States|-5|condo| +42836|AAAAAAAAEFHKAAAA|523|Lincoln View|Boulevard|Suite C|Crossroads|Steuben County|IN|40534|United States|-5|single family| +42837|AAAAAAAAFFHKAAAA|58|Ninth Forest|Circle|Suite 280|Wilson|Bannock County|ID|86971|United States|-7|apartment| +42838|AAAAAAAAGFHKAAAA|802|View |Way|Suite Q|Five Forks|Whatcom County|WA|92293|United States|-8|apartment| +42839|AAAAAAAAHFHKAAAA|842|Fifteenth Oak|Parkway|Suite D|Valley View|Benewah County|ID|85124|United States|-7|apartment| +42840|AAAAAAAAIFHKAAAA|951|North View|Drive|Suite 280|Shiloh|Clay County|IA|59275|United States|-6|condo| +42841|AAAAAAAAJFHKAAAA|399|Hill Jefferson|Circle|Suite I|Oakwood|Franklin County|ID|80169|United States|-7|apartment| +42842|AAAAAAAAKFHKAAAA|359|Oak 3rd|Boulevard|Suite 20|Centerville|Hall County|NE|60059|United States|-6|single family| +42843|AAAAAAAALFHKAAAA|25|Hill |Street|Suite 400|Green Acres|Morgan County|AL|37683|United States|-6|single family| +42844|AAAAAAAAMFHKAAAA|27|Madison Wilson|Avenue|Suite 250|Shady Grove|Rockingham County|VA|22812|United States|-5|single family| +42845|AAAAAAAANFHKAAAA|625|Park |Cir.|Suite 110|Jamestown|Westmoreland County|VA|26867|United States|-5|single family| +42846|AAAAAAAAOFHKAAAA|384|7th Oak|Ln|Suite 450|Belmont|Burke County|ND|50191|United States|-6|apartment| +42847|AAAAAAAAPFHKAAAA|130|Jefferson Green|Cir.|Suite E|Friendship|Campbell County|KY|44536|United States|-6|single family| +42848|AAAAAAAAAGHKAAAA|260|College |ST|Suite 180|Springfield|Greenwood County|KS|69303|United States|-6|single family| +42849|AAAAAAAABGHKAAAA|895|Smith Elm|Wy|Suite 350|Bunker Hill|Neosho County|KS|60150|United States|-6|single family| +42850|AAAAAAAACGHKAAAA|740|Main |Way|Suite B|Red Hill|Union County|KY|44338|United States|-5|apartment| +42851|AAAAAAAADGHKAAAA|908|7th |Ct.|Suite D|Woodville|Dukes County|MA|04889|United States|-5|apartment| +42852|AAAAAAAAEGHKAAAA|593|Green Main|Cir.|Suite 270|Waterloo|Butte County|CA|91675|United States|-8|condo| +42853|AAAAAAAAFGHKAAAA|198|Oak |Wy|Suite 50|Five Forks|Forest County|PA|12293|United States|-5|condo| +42854|AAAAAAAAGGHKAAAA|461|West Oak|Way|Suite H|Riverside|Lewis County|NY|19231|United States|-5|single family| +42855|AAAAAAAAHGHKAAAA|895|Main |Circle|Suite Y|Marion|Glasscock County|TX|70399|United States|-6|single family| +42856|AAAAAAAAIGHKAAAA|554|Lake Eigth|Ave|Suite 420|Brownsville|Darlington County|SC|29310|United States|-5|apartment| +42857|AAAAAAAAJGHKAAAA|905|Green Willow|Circle|Suite O|Five Forks|Payne County|OK|72293|United States|-6|single family| +42858|AAAAAAAAKGHKAAAA|274||||Shiloh|||59275|||condo| +42859|AAAAAAAALGHKAAAA|||Way|Suite Q||Wichita County||||-6|condo| +42860|AAAAAAAAMGHKAAAA|417|Hillcrest |Ln|Suite V|Antioch|Wayne County|IL|68605|United States|-6|single family| +42861|AAAAAAAANGHKAAAA|120|12th |Avenue|Suite U|Centerville|Accomack County|VA|20059|United States|-5|condo| +42862|AAAAAAAAOGHKAAAA|854|5th |Blvd|Suite 110|Valley View|Rowan County|KY|45124|United States|-5|apartment| +42863|AAAAAAAAPGHKAAAA|116|Madison |Pkwy|Suite N|Pine Grove|Marlboro County|SC|24593|United States|-5|single family| +42864|AAAAAAAAAHHKAAAA|180|Center |Dr.|Suite T|Five Forks|Logan County|OK|72293|United States|-6|apartment| +42865|AAAAAAAABHHKAAAA|183|Spring Walnut|Ln|Suite 480|Lincoln|Porter County|IN|41289|United States|-5|single family| +42866|AAAAAAAACHHKAAAA|288|Ridge Forest|Ave|Suite C|Farmington|Monterey County|CA|99145|United States|-8|apartment| +42867|AAAAAAAADHHKAAAA|50|Williams |Circle|Suite O|Pleasant Hill|Pratt County|KS|63604|United States|-6|single family| +42868|AAAAAAAAEHHKAAAA|118|Miller |Wy|Suite U|Unionville|Collier County|FL|31711|United States|-5|apartment| +42869|AAAAAAAAFHHKAAAA|485|College Davis|Way|Suite 150|Mount Zion|Hamilton County|TN|38054|United States|-5|apartment| +42870|AAAAAAAAGHHKAAAA|248|6th |Dr.|Suite H|Cedar Grove|Pennington County|MN|50411|United States|-6|condo| +42871|AAAAAAAAHHHKAAAA|890|Eigth Sunset|Dr.|Suite M|Waterloo|Grayson County|VA|21675|United States|-5|condo| +42872|AAAAAAAAIHHKAAAA|770|1st Lincoln|Ct.|Suite Q|Summit|Salem city|VA|20499|United States|-5|single family| +42873|AAAAAAAAJHHKAAAA|812|Poplar Adams|Ln|Suite V|Lincoln|Dallas County|AL|31289|United States|-6|condo| +42874|AAAAAAAAKHHKAAAA|669|Franklin |Ct.|Suite O|Shiloh|Scott County|MO|69275|United States|-6|condo| +42875|AAAAAAAALHHKAAAA|696|4th Ridge|Parkway|Suite 200|Brunswick|Sherman County|TX|74642|United States|-6|apartment| +42876|AAAAAAAAMHHKAAAA|776|Washington Hickory|Street|Suite 260|Mount Zion|Pottawatomie County|OK|78054|United States|-6|condo| +42877|AAAAAAAANHHKAAAA|247|North |Avenue|Suite 370|Mount Vernon|West Baton Rouge Parish|LA|78482|United States|-6|condo| +42878|AAAAAAAAOHHKAAAA|445|Valley |Lane|Suite 220|Jackson|Taylor County|FL|39583|United States|-5|single family| +42879|AAAAAAAAPHHKAAAA|798|Third Dogwood|Avenue|Suite 210|Hopewell|Snyder County|PA|10587|United States|-5|apartment| +42880|AAAAAAAAAIHKAAAA|363|Poplar |Court|Suite L|Fairfield|Hillsborough County|NH|06792|United States|-5|single family| +42881|AAAAAAAABIHKAAAA|688|Central |Ave|Suite 310|Newtown|Okaloosa County|FL|31749|United States|-5|apartment| +42882|AAAAAAAACIHKAAAA|33|||Suite I|||||United States||| +42883|AAAAAAAADIHKAAAA|220|5th Locust|Parkway|Suite 400|Hamilton|Keweenaw County|MI|42808|United States|-5|apartment| +42884|AAAAAAAAEIHKAAAA|875|Lake |RD|Suite Q|Ashland|Mercer County|IL|64244|United States|-6|condo| +42885|AAAAAAAAFIHKAAAA|303|Smith River|Ln|Suite 470|Maple Hill|Chautauqua County|NY|18095|United States|-5|single family| +42886|AAAAAAAAGIHKAAAA|109|Adams Jackson|Ct.|Suite B|Woodville|Martin County|MN|54289|United States|-6|apartment| +42887|AAAAAAAAHIHKAAAA|60|Ridge Meadow|Cir.|Suite 320|Riverview|Cochise County|AZ|89003|United States|-7|condo| +42888|AAAAAAAAIIHKAAAA|289|9th First|ST|Suite 290|Hillcrest|Rockland County|NY|13003|United States|-5|apartment| +42889|AAAAAAAAJIHKAAAA|212|Railroad |Dr.|Suite L|Plainview|Sunflower County|MS|53683|United States|-6|single family| +42890|AAAAAAAAKIHKAAAA|276|Main |Blvd|Suite 50|Lincoln|Laurens County|SC|21289|United States|-5|single family| +42891|AAAAAAAALIHKAAAA|264|3rd Second|Ln|Suite R|Stringtown|Henry County|IA|50162|United States|-6|condo| +42892|AAAAAAAAMIHKAAAA|377|Main 5th|Parkway|Suite 380|Littleton|Trimble County|KY|46074|United States|-5|apartment| +42893|AAAAAAAANIHKAAAA|361|Ridge |Pkwy|Suite 310|Oakland|McDonough County|IL|69843|United States|-6|apartment| +42894|AAAAAAAAOIHKAAAA|956|Seventh |Street|Suite 0|Springdale|Jackson County|SD|58883|United States|-7|single family| +42895|AAAAAAAAPIHKAAAA|345|Hillcrest 1st|Road|Suite 190|Ashland|York County|SC|24244|United States|-5|single family| +42896|AAAAAAAAAJHKAAAA|979|Seventh |RD|Suite 490|Jackson|Lyon County|KY|49583|United States|-5|single family| +42897|AAAAAAAABJHKAAAA|849|Birch |Boulevard|Suite U|Riverside|Columbia County|FL|39231|United States|-5|apartment| +42898|AAAAAAAACJHKAAAA|36|South Walnut|Dr.|Suite J|Valley View|East Carroll Parish|LA|75124|United States|-6|single family| +42899|AAAAAAAADJHKAAAA|283|9th Jefferson|Avenue|Suite W|Church Hill|Henry County|GA|33790|United States|-5|apartment| +42900|AAAAAAAAEJHKAAAA|190|Walnut Madison|Wy|Suite 300|Mount Pleasant|Roane County|TN|31933|United States|-6|single family| +42901|AAAAAAAAFJHKAAAA|857|12th Washington|Cir.|Suite X|Antioch|Muskogee County|OK|78605|United States|-6|single family| +42902|AAAAAAAAGJHKAAAA|550|River |Road|Suite 360|Summit|Herkimer County|NY|10499|United States|-5|condo| +42903|AAAAAAAAHJHKAAAA|765|River 2nd|Avenue|Suite S|Springdale|Fayette County|PA|18883|United States|-5|single family| +42904|AAAAAAAAIJHKAAAA|877|Jefferson Spruce|Pkwy|Suite P|Stringtown|Hudspeth County|TX|70162|United States|-6|single family| +42905|AAAAAAAAJJHKAAAA||Meadow Walnut|||||||United States||| +42906|AAAAAAAAKJHKAAAA|814|North Hickory|Road|Suite Y|Woodland|Neshoba County|MS|54854|United States|-6|condo| +42907|AAAAAAAALJHKAAAA|606|Sycamore 6th|Cir.|Suite 280|Antioch|Harrison County|WV|28605|United States|-5|condo| +42908|AAAAAAAAMJHKAAAA|644|Sycamore |Parkway|Suite R|Ashland|Uinta County|WY|84244|United States|-7|condo| +42909|AAAAAAAANJHKAAAA|||Lane|||Perry County||41289|United States|-5|| +42910|AAAAAAAAOJHKAAAA|67|Twelfth |Wy|Suite P|Pierce|Wahkiakum County|WA|93360|United States|-8|single family| +42911|AAAAAAAAPJHKAAAA|295|5th |RD|Suite C|Newtown|Rockcastle County|KY|41749|United States|-5|apartment| +42912|AAAAAAAAAKHKAAAA|913|Oak |Cir.|Suite 390|Reno|Christian County|MO|60344|United States|-6|single family| +42913|AAAAAAAABKHKAAAA|358|5th |RD|Suite X|Woodland|Henry County|GA|34854|United States|-5|condo| +42914|AAAAAAAACKHKAAAA|2|Fourth |Parkway|Suite 330|Oak Ridge|Giles County|TN|38371|United States|-5|apartment| +42915|AAAAAAAADKHKAAAA|522|Davis |Ave|Suite R|Five Points|Kiowa County|CO|86098|United States|-7|apartment| +42916|AAAAAAAAEKHKAAAA|222|1st Oak|Parkway|Suite 30|Berea|Chester County|SC|23804|United States|-5|apartment| +42917|AAAAAAAAFKHKAAAA|130|12th |Pkwy|Suite 190|White Oak|Rogers County|OK|76668|United States|-6|single family| +42918|AAAAAAAAGKHKAAAA|803|Pine Park|Avenue|Suite R|Wildwood|Chilton County|AL|36871|United States|-6|single family| +42919|AAAAAAAAHKHKAAAA|610|Highland |Wy|Suite D|Woodlawn|Greene County|MO|64098|United States|-6|single family| +42920|AAAAAAAAIKHKAAAA|374|Elevnth |Pkwy|Suite T|Riverview|Washington County|VA|29003|United States|-5|condo| +42921|AAAAAAAAJKHKAAAA|359|2nd |Avenue|Suite 150|Wildwood|Kewaunee County|WI|56871|United States|-6|apartment| +42922|AAAAAAAAKKHKAAAA|77|Church |Dr.|Suite 80|Clinton|Westmoreland County|PA|18222|United States|-5|condo| +42923|AAAAAAAALKHKAAAA|426|Valley |Way|Suite Q|Glendale|Washtenaw County|MI|43951|United States|-5|condo| +42924|AAAAAAAAMKHKAAAA|110|Elm |Ln|Suite 210|Riverdale|Crook County|WY|89391|United States|-7|single family| +42925|AAAAAAAANKHKAAAA|147|Oak Main|Lane|Suite A|Greenville|Summit County|UT|81387|United States|-7|condo| +42926|AAAAAAAAOKHKAAAA|393|Hickory Elm|Street|Suite W|Mount Vernon|Andrews County|TX|78482|United States|-6|single family| +42927|AAAAAAAAPKHKAAAA|596|Elm |ST|Suite 410|Wildwood|Haskell County|KS|66871|United States|-6|condo| +42928|AAAAAAAAALHKAAAA|||||Antioch|Huron County|MI|||-5|| +42929|AAAAAAAABLHKAAAA|571|2nd Willow|Ct.|Suite 70|Mount Olive|Larimer County|CO|88059|United States|-7|condo| +42930|AAAAAAAACLHKAAAA|428|2nd View|Cir.|Suite 90|Lone Pine|Mohave County|AZ|87441|United States|-7|apartment| +42931|AAAAAAAADLHKAAAA|678|Laurel Center|Road|Suite 290|Stringtown|Gallatin County|IL|60162|United States|-6|single family| +42932|AAAAAAAAELHKAAAA|97|Hill |Ct.|Suite D|Oak Ridge|Kearny County|KS|68371|United States|-6|apartment| +42933|AAAAAAAAFLHKAAAA|760|Meadow 13th|Pkwy|Suite 470|Wildwood|Rosebud County|MT|66871|United States|-7|condo| +42934|AAAAAAAAGLHKAAAA|50|South Hillcrest|Dr.|Suite 300|Buena Vista|Jim Wells County|TX|75752|United States|-6|condo| +42935|AAAAAAAAHLHKAAAA|385|Ridge |Way|Suite D|Warwick|Warren County|MO|61398|United States|-6|single family| +42936|AAAAAAAAILHKAAAA|805|Highland |Way|Suite O|Woodville|Marquette County|WI|54289|United States|-6|condo| +42937|AAAAAAAAJLHKAAAA|657||Cir.|Suite 110||Woods County|OK||United States||condo| +42938|AAAAAAAAKLHKAAAA|953|Main 10th|RD|Suite 220|San Jose|Southampton County|VA|28003|United States|-5|condo| +42939|AAAAAAAALLHKAAAA|356|Walnut |Street|Suite 330|Midway|Pulaski County|GA|31904|United States|-5|apartment| +42940|AAAAAAAAMLHKAAAA|790|Dogwood 10th|Way|Suite L|Cedar|Marshall County|IN|41229|United States|-5|condo| +42941|AAAAAAAANLHKAAAA|130|11th |ST|Suite D|Oakdale|Robertson County|TN|39584|United States|-6|single family| +42942|AAAAAAAAOLHKAAAA|525|Park |Road|Suite 230|Ashland|Carroll County|MD|24244|United States|-5|apartment| +42943|AAAAAAAAPLHKAAAA|870|Center 2nd|Ln|Suite Q|Five Forks|Henderson County|IL|62293|United States|-6|single family| +42944|AAAAAAAAAMHKAAAA|153|Spring Mill|Pkwy|Suite O|Oak Hill|Fulton County|KY|47838|United States|-6|condo| +42945|AAAAAAAABMHKAAAA|823|Maple |Lane|Suite I|Mechanicsburg|Laurens County|SC|22219|United States|-5|single family| +42946|AAAAAAAACMHKAAAA|91|Seventh Broadway|Avenue|Suite 90|Springfield|Roane County|TN|39303|United States|-6|single family| +42947|AAAAAAAADMHKAAAA|216|Williams |Court|Suite K|Bayside|Wade Hampton Census Area|AK|99550|United States|-9|condo| +42948|AAAAAAAAEMHKAAAA|412|Hill |Street|Suite 70|Hillcrest|Steuben County|NY|13003|United States|-5|single family| +42949|AAAAAAAAFMHKAAAA|323|Sycamore 5th|Circle|Suite Q|Highland|Richmond County|NC|29454|United States|-5|apartment| +42950|AAAAAAAAGMHKAAAA|998|North Wilson|Blvd|Suite 200|Greenville|Whitley County|IN|41387|United States|-5|apartment| +42951|AAAAAAAAHMHKAAAA|666|West |Road|Suite F|Mount Zion|Lyman County|SD|58054|United States|-7|single family| +42952|AAAAAAAAIMHKAAAA|582|4th 1st|RD|Suite 70|Lincoln|Delta County|CO|81289|United States|-7|condo| +42953|AAAAAAAAJMHKAAAA|275|Hill Jackson|Lane|Suite I|Highland|Pettis County|MO|69454|United States|-6|condo| +42954|AAAAAAAAKMHKAAAA|445|Jackson Walnut|RD|Suite 320|Greenfield|Fayette County|IA|55038|United States|-6|single family| +42955|AAAAAAAALMHKAAAA|249|Franklin Seventh|Court|Suite E|Stringtown|Jones County|MS|50162|United States|-6|condo| +42956|AAAAAAAAMMHKAAAA|567|Hill |Blvd|Suite L|White Oak|Pierce County|WI|56668|United States|-6|single family| +42957|AAAAAAAANMHKAAAA|289|Third 7th|Ln|Suite 130|Belmont|Rains County|TX|70191|United States|-6|single family| +42958|AAAAAAAAOMHKAAAA|922|Williams 13th|Pkwy|Suite 260|Pleasant Grove|Highland County|OH|44136|United States|-5|condo| +42959|AAAAAAAAPMHKAAAA|514||Court||Greenville|Addison County|||||| +42960|AAAAAAAAANHKAAAA|25|Locust Sycamore|ST|Suite 190|Ellsworth|Edgecombe County|NC|25079|United States|-5|condo| +42961|AAAAAAAABNHKAAAA|347|Central Hill|Avenue|Suite 460|Liberty|Grant County|WV|23451|United States|-5|single family| +42962|AAAAAAAACNHKAAAA|515|Maple |Dr.|Suite C|Providence|Malheur County|OR|96614|United States|-8|condo| +42963|AAAAAAAADNHKAAAA|167|Mill |Cir.|Suite 110|Edgewood|Hart County|GA|30069|United States|-5|single family| +42964|AAAAAAAAENHKAAAA|595|Lake Fifth|Drive|Suite L|Pleasant Hill|Day County|SD|53604|United States|-6|single family| +42965|AAAAAAAAFNHKAAAA|336|Eigth Ridge|ST|Suite M|Woodland|Ontonagon County|MI|44854|United States|-5|apartment| +42966|AAAAAAAAGNHKAAAA|869|Willow |Parkway|Suite 100|Bay View|Valley County|ID|86457|United States|-7|condo| +42967|AAAAAAAAHNHKAAAA|187|Central |Ln|Suite 460|Leesville|Jefferson County|AL|35423|United States|-6|single family| +42968|AAAAAAAAINHKAAAA|783|Lincoln Ridge|RD|Suite 390|Bridgeport|Larimer County|CO|85817|United States|-7|apartment| +42969|AAAAAAAAJNHKAAAA|428|Jackson 4th|Lane|Suite E|Crossroads|Presque Isle County|MI|40534|United States|-5|single family| +42970|AAAAAAAAKNHKAAAA|751|Meadow Franklin|Circle|Suite 460|Brownsville|Eastland County|TX|79310|United States|-6|condo| +42971|AAAAAAAALNHKAAAA|899|Oak |Lane|Suite A|Richardson|Marion County|IL|67687|United States|-6|apartment| +42972|AAAAAAAAMNHKAAAA|891|Elevnth Lincoln|Way|Suite 230|Forest Hills|Cooper County|MO|69237|United States|-6|single family| +42973|AAAAAAAANNHKAAAA|608|4th Elm|Street|Suite 100|Buena Vista|Jefferson Davis Parish|LA|75752|United States|-6|apartment| +42974|AAAAAAAAONHKAAAA|147|Cedar Valley|Wy|Suite M|Lebanon|Rock County|MN|52898|United States|-6|single family| +42975|AAAAAAAAPNHKAAAA|749|Jackson 7th|Lane|Suite U|Lee|Jefferson County|AL|30408|United States|-6|single family| +42976|AAAAAAAAAOHKAAAA|463|3rd Cherry|Street|Suite T|Hopewell|Botetourt County|VA|20587|United States|-5|apartment| +42977|AAAAAAAABOHKAAAA||||Suite E|||AL||United States||single family| +42978|AAAAAAAACOHKAAAA|705|Lake |Road|Suite Y|Mount Pleasant|Newton County|TX|71933|United States|-6|condo| +42979|AAAAAAAADOHKAAAA||||Suite 220|||KY|49089|United States|-6|| +42980|AAAAAAAAEOHKAAAA|386|Lake Dogwood|Court|Suite H|Forest Hills|Boyle County|KY|49237|United States|-6|single family| +42981|AAAAAAAAFOHKAAAA|322|Willow |Street|Suite N|Brownsville|Richmond County|NC|29310|United States|-5|single family| +42982|AAAAAAAAGOHKAAAA|515|Church |Parkway|Suite O|Clinton|Okaloosa County|FL|38222|United States|-5|single family| +42983|AAAAAAAAHOHKAAAA|992|Madison |Ct.|Suite A|Friendship|Swain County|NC|24536|United States|-5|apartment| +42984|AAAAAAAAIOHKAAAA|845|2nd Laurel|Way|Suite U|Sunnyside|Staunton city|VA|21952|United States|-5|apartment| +42985|AAAAAAAAJOHKAAAA|42|13th |Ln|Suite B|Stewart|Roosevelt County|NM|88041|United States|-7|apartment| +42986|AAAAAAAAKOHKAAAA|153|Hill |Avenue|Suite 350|Summit|Berkeley County|SC|20499|United States|-5|apartment| +42987|AAAAAAAALOHKAAAA|939|Madison West|Drive|Suite 0|Lincoln|Golden Valley County|ND|51289|United States|-6|apartment| +42988|AAAAAAAAMOHKAAAA|853|13th |Road|Suite H|Deerfield|Jerome County|ID|89840|United States|-7|apartment| +42989|AAAAAAAANOHKAAAA|325|Sycamore 15th|ST|Suite 320|Hopewell|Cherokee County|OK|70587|United States|-6|condo| +42990|AAAAAAAAOOHKAAAA|71|5th View|Way|Suite 100|Enterprise|Mitchell County|TX|71757|United States|-6|apartment| +42991|AAAAAAAAPOHKAAAA|77|1st Ash|Street|Suite 80|Walnut Grove|Clinton County|NY|17752|United States|-5|apartment| +42992|AAAAAAAAAPHKAAAA|201|Williams 5th|Pkwy|Suite 370|Pleasant Grove|Richland County|OH|44136|United States|-5|condo| +42993|AAAAAAAABPHKAAAA|532|Dogwood |Pkwy|Suite 440|Green Acres|Gove County|KS|67683|United States|-6|single family| +42994|AAAAAAAACPHKAAAA|100|College 2nd|Road|Suite L|Ashland|Caldwell County|TX|74244|United States|-6|single family| +42995|AAAAAAAADPHKAAAA|749|1st |Court|Suite 10|Leon|Ingham County|MI|40913|United States|-5|single family| +42996|AAAAAAAAEPHKAAAA|529|Birch |Cir.|Suite 50|Union Hill|Knox County|NE|67746|United States|-7|single family| +42997|AAAAAAAAFPHKAAAA|84|8th Jefferson|Wy|Suite M|Sunnyside|Richland County|OH|41952|United States|-5|apartment| +42998|AAAAAAAAGPHKAAAA|17|4th |Lane|Suite 260|New Town|Treasure County|MT|69634|United States|-7|apartment| +42999|AAAAAAAAHPHKAAAA|937|Franklin |RD|Suite B|Greenwood|Archuleta County|CO|88828|United States|-7|condo| +43000|AAAAAAAAIPHKAAAA|385|3rd Church|Parkway|Suite 110|Mountain View|Guadalupe County|NM|84466|United States|-7|apartment| +43001|AAAAAAAAJPHKAAAA|588|Spring |Pkwy|Suite D|New Salem|Wayne County|MS|59568|United States|-6|condo| +43002|AAAAAAAAKPHKAAAA|356|Forest |Ln|Suite 60|Forest Hills|Marshall County|IA|59237|United States|-6|apartment| +43003|AAAAAAAALPHKAAAA|461|10th Church|Wy|Suite G|Sunnyside|Lapeer County|MI|41952|United States|-5|condo| +43004|AAAAAAAAMPHKAAAA|916|Spruce |Wy|Suite S|Greenwood|Brunswick County|NC|28828|United States|-5|apartment| +43005|AAAAAAAANPHKAAAA|47|Willow |Blvd|Suite 200|Bethel|Saline County|KS|65281|United States|-6|apartment| +43006|AAAAAAAAOPHKAAAA|563|Cherry 2nd|Boulevard|Suite 90|Union Hill|Seminole County|GA|37746|United States|-5|condo| +43007|AAAAAAAAPPHKAAAA|301|Central Ninth|Street|Suite P|Lebanon|Macoupin County|IL|62898|United States|-6|apartment| +43008|AAAAAAAAAAIKAAAA|150|Jefferson |Ct.|Suite 350|Spring Hill|Howard County|TX|76787|United States|-6|single family| +43009|AAAAAAAABAIKAAAA|385|Lincoln River|Drive|Suite J|Mountain View|Walker County|GA|34466|United States|-5|condo| +43010|AAAAAAAACAIKAAAA|811|River |Lane|Suite I|Newtown|Mason County|MI|41749|United States|-5|apartment| +43011|AAAAAAAADAIKAAAA|634|Second Center|Ct.|Suite 80|Shiloh|Gulf County|FL|39275|United States|-5|apartment| +43012|AAAAAAAAEAIKAAAA|411|||||Washburn County|WI||||condo| +43013|AAAAAAAAFAIKAAAA|161|15th Spruce|RD|Suite X|Hamilton|Lemhi County|ID|82808|United States|-7|single family| +43014|AAAAAAAAGAIKAAAA|197|1st 12th|Ln|Suite 80|Siloam|Yukon-Koyukuk Census Area|AK|98948|United States|-9|single family| +43015|AAAAAAAAHAIKAAAA|15|North |Circle|Suite U|Summit|Manassas Park city|VA|20499|United States|-5|apartment| +43016|AAAAAAAAIAIKAAAA|464|Pine |Avenue|Suite L|Union Hill|Mille Lacs County|MN|57746|United States|-6|single family| +43017|AAAAAAAAJAIKAAAA|74|Chestnut Willow|Road|Suite G|Sulphur Springs|Posey County|IN|48354|United States|-5|condo| +43018|AAAAAAAAKAIKAAAA|156|Park Jefferson|RD|Suite 20|Waterloo|Bedford city|VA|21675|United States|-5|condo| +43019|AAAAAAAALAIKAAAA|64|Walnut Main|Ave|Suite 110|Northwood|Linn County|OR|94104|United States|-8|single family| +43020|AAAAAAAAMAIKAAAA|887|Park Second|Boulevard|Suite T|Woodland|Oneida County|NY|14854|United States|-5|condo| +43021|AAAAAAAANAIKAAAA|644|Dogwood Main|RD|Suite A|Wilson|Carson City|NV|86971|United States|-8|single family| +43022|AAAAAAAAOAIKAAAA||West Center|Blvd|||Rawlins County||||-6|single family| +43023|AAAAAAAAPAIKAAAA|171|Meadow |Lane|Suite 110|Garrison|Union County|NC|28767|United States|-5|condo| +43024|AAAAAAAAABIKAAAA|235|Valley |RD|Suite U|Mountain View|Beaverhead County|MT|64466|United States|-7|single family| +43025|AAAAAAAABBIKAAAA|488|Cherry |Ln|Suite F|Red Hill|Cannon County|TN|34338|United States|-5|single family| +43026|AAAAAAAACBIKAAAA|493|Jackson |Drive|Suite P|Shiloh|Burt County|NE|69275|United States|-6|condo| +43027|AAAAAAAADBIKAAAA|951|10th Spring|Parkway|Suite 210|Midway|Washington County|KS|61904|United States|-6|single family| +43028|AAAAAAAAEBIKAAAA|964|Center |Ln|Suite 420|Deerfield|Preston County|WV|29840|United States|-5|apartment| +43029|AAAAAAAAFBIKAAAA|305|6th Mill|Ave|Suite A|Concord|Washington County|WI|54107|United States|-6|apartment| +43030|AAAAAAAAGBIKAAAA|320|Lincoln Valley|Court|Suite A|Antioch|Bonneville County|ID|88605|United States|-7|apartment| +43031|AAAAAAAAHBIKAAAA|436|Eigth South|Road|Suite 140|Lakewood|Kennebec County|ME|09477|United States|-5|apartment| +43032|AAAAAAAAIBIKAAAA|49|Lincoln |Cir.|Suite U|Five Points|Rowan County|NC|26098|United States|-5|condo| +43033|AAAAAAAAJBIKAAAA|780|Highland ||Suite K||||88828||-7|| +43034|AAAAAAAAKBIKAAAA|482|Third Spring|Parkway|Suite N|Buena Vista|McLeod County|MN|55752|United States|-6|single family| +43035|AAAAAAAALBIKAAAA|422|15th |RD|Suite 310|Pleasant Valley|Accomack County|VA|22477|United States|-5|apartment| +43036|AAAAAAAAMBIKAAAA|225|Elm |RD|Suite 380|Macedonia|Jackson County|MO|61087|United States|-6|single family| +43037|AAAAAAAANBIKAAAA|398|Maple |Dr.|Suite K|Enterprise|Burke County|NC|21757|United States|-5|apartment| +43038|AAAAAAAAOBIKAAAA|692|River Hill|Ln|Suite 80|Crossroads|Mayes County|OK|70534|United States|-6|apartment| +43039|AAAAAAAAPBIKAAAA|837|7th Woodland|Parkway|Suite T|Mount Olive|Lewis County|KY|48059|United States|-5|apartment| +43040|AAAAAAAAACIKAAAA|638|7th |Street|Suite 240|Springdale|Uvalde County|TX|78883|United States|-6|apartment| +43041|AAAAAAAABCIKAAAA|7|6th |Drive|Suite 460|Pleasant Hill|Franklin County|IN|43604|United States|-5|single family| +43042|AAAAAAAACCIKAAAA|437|Pine ||Suite C|Spring Hill|Moffat County|CO|86787|United States||| +43043|AAAAAAAADCIKAAAA|362|Second |Road|Suite 220|Williamsburg|Douglas County|OR|98336|United States|-8|single family| +43044|AAAAAAAAECIKAAAA|17|2nd Park|Blvd|Suite 30|Pleasant Valley|San Luis Obispo County|CA|92477|United States|-8|apartment| +43045|AAAAAAAAFCIKAAAA|898|Railroad |ST|Suite Q|Harmony|Baldwin County|AL|35804|United States|-6|condo| +43046|AAAAAAAAGCIKAAAA|300|Hill Hill|Boulevard|Suite 220|Clifton|Marengo County|AL|38014|United States|-6|single family| +43047|AAAAAAAAHCIKAAAA|531|Spring |Way|Suite 110|Riverside|Lumpkin County|GA|39231|United States|-5|apartment| +43048|AAAAAAAAICIKAAAA|701|Chestnut |Way|Suite 330|Mount Pleasant|Webster County|WV|21933|United States|-5|condo| +43049|AAAAAAAAJCIKAAAA|951|Willow Elm|Ln|Suite M|Cedar Grove|Sitka Borough|AK|90411|United States|-9|apartment| +43050|AAAAAAAAKCIKAAAA|129|Spring Cherry|Pkwy|Suite Q|Midway|Holmes County|MS|51904|United States|-6|condo| +43051|AAAAAAAALCIKAAAA|649|Forest Hill|Cir.|Suite 370|Avery|Yukon-Koyukuk Census Area|AK|90194|United States|-9|apartment| +43052|AAAAAAAAMCIKAAAA|447|Ninth 2nd|Wy|Suite I|Five Forks|Reynolds County|MO|62293|United States|-6|apartment| +43053|AAAAAAAANCIKAAAA|738|View |Blvd|Suite K|Lone Pine|Green County|WI|57441|United States|-6|single family| +43054|AAAAAAAAOCIKAAAA|20|Railroad Cherry|Avenue|Suite 10|Pleasant Grove|Tuscola County|MI|44136|United States|-5|single family| +43055|AAAAAAAAPCIKAAAA|933|Seventh |Pkwy|Suite K|Valley View|Edwards County|KS|65124|United States|-6|apartment| +43056|AAAAAAAAADIKAAAA|278|Green 5th|Wy|Suite 340|Macedonia|Delta County|CO|81087|United States|-7|condo| +43057|AAAAAAAABDIKAAAA|459|Ninth |Way|Suite 180|Woodville|Shelby County|KY|44289|United States|-5|condo| +43058|AAAAAAAACDIKAAAA|145|||Suite F|||||United States||condo| +43059|AAAAAAAADDIKAAAA|422|Locust |Drive|Suite O|Liberty|Perry County|AL|33451|United States|-6|apartment| +43060|AAAAAAAAEDIKAAAA|714|1st ||Suite U|Jackson|Catawba County|NC||United States|-5|single family| +43061|AAAAAAAAFDIKAAAA|277||Boulevard||Stringtown|Carteret County|||United States|-5|| +43062|AAAAAAAAGDIKAAAA||||Suite 390|Bay View||CA|||-8|| +43063|AAAAAAAAHDIKAAAA|39|Elm Spruce|Ln|Suite 200|Woodlawn|Putnam County|MO|64098|United States|-6|single family| +43064|AAAAAAAAIDIKAAAA|615|Meadow |Boulevard|Suite 450|Stringtown|West Baton Rouge Parish|LA|70162|United States|-6|single family| +43065|AAAAAAAAJDIKAAAA|672|Second Adams|Road|Suite 490|Union Hill|Taylor County|WV|27746|United States|-5|apartment| +43066|AAAAAAAAKDIKAAAA|492|Locust Wilson|Wy|Suite 80|Highland Park|Mecklenburg County|NC|26534|United States|-5|apartment| +43067|AAAAAAAALDIKAAAA|848|Franklin Elm|Avenue|Suite B|Pleasant Hill|Burleigh County|ND|53604|United States|-6|apartment| +43068|AAAAAAAAMDIKAAAA|731|Central |Cir.|Suite M|New Hope|Banner County|NE|69431|United States|-6|apartment| +43069|AAAAAAAANDIKAAAA|335|Maple |Road|Suite V|Franklin|Quitman County|GA|39101|United States|-5|single family| +43070|AAAAAAAAODIKAAAA|591|14th 7th|Drive|Suite Q|Summit|Howard County|MD|20499|United States|-5|condo| +43071|AAAAAAAAPDIKAAAA|801|Johnson ||Suite 480|Friendship|||||-6|| +43072|AAAAAAAAAEIKAAAA|781|Sixth North|Court|Suite 370|Lakewood|York County|SC|28877|United States|-5|condo| +43073|AAAAAAAABEIKAAAA|712|Dogwood |Lane|Suite T|Greenfield|Dallas County|IA|55038|United States|-6|single family| +43074|AAAAAAAACEIKAAAA|298|First |Circle|Suite M|Richville|Wayne County|MI|45945|United States|-5|apartment| +43075|AAAAAAAADEIKAAAA|442|Jackson Lincoln|Road|Suite 0|Antioch|Oklahoma County|OK|78605|United States|-6|single family| +43076|AAAAAAAAEEIKAAAA|602|Jefferson Lake|Boulevard|Suite M|Bunker Hill|Switzerland County|IN|40150|United States|-5|single family| +43077|AAAAAAAAFEIKAAAA|502|Jackson Highland|Ln|Suite 0|Five Points|Marion County|SC|26098|United States|-5|single family| +43078|AAAAAAAAGEIKAAAA|51|Hill |ST|Suite O|Centerville|Hendry County|FL|30059|United States|-5|condo| +43079|AAAAAAAAHEIKAAAA|327|Forest |Court|Suite D|Evans|Newton County|MS|52284|United States|-6|single family| +43080|AAAAAAAAIEIKAAAA|||||||||||| +43081|AAAAAAAAJEIKAAAA|241|Main North|Way|Suite Q|Union|Covington County|AL|38721|United States|-6|single family| +43082|AAAAAAAAKEIKAAAA|102|Lake Highland|Road|Suite 120|Riverside|Lincoln County|SD|59231|United States|-7|single family| +43083|AAAAAAAALEIKAAAA|93|8th Pine|Court|Suite Q|Lakeview|Banner County|NE|68579|United States|-6|condo| +43084|AAAAAAAAMEIKAAAA|222|Park |Drive|Suite 110|Buena Vista|Lincoln County|KS|65752|United States|-6|condo| +43085|AAAAAAAANEIKAAAA|866|9th Maple|Blvd|Suite 110|Fairfield|Latimer County|OK|76192|United States|-6|apartment| +43086|AAAAAAAAOEIKAAAA|386|Elm |Boulevard|Suite 170|Woodlawn|Wise County|TX|74098|United States|-6|condo| +43087|AAAAAAAAPEIKAAAA|894|West |Lane|Suite 140|Shady Grove|Teton County|WY|82812|United States|-7|condo| +43088|AAAAAAAAAFIKAAAA|307|Sunset |Way|Suite K|Mount Zion|Deschutes County|OR|98054|United States|-8|condo| +43089|AAAAAAAABFIKAAAA||||Suite I|Plainview|Jeff Davis County|TX|||-6|| +43090|AAAAAAAACFIKAAAA|286|1st |Boulevard|Suite 170|Greenfield|Audubon County|IA|55038|United States|-6|condo| +43091|AAAAAAAADFIKAAAA|930|14th |Wy|Suite Y|Jamestown|Winn Parish|LA|76867|United States|-6|apartment| +43092|AAAAAAAAEFIKAAAA|370|Green Miller|Lane|Suite H|Edgewood|Sabine Parish|LA|70069|United States|-6|apartment| +43093|AAAAAAAAFFIKAAAA|151|Sunset Miller|ST|Suite A|Oakwood|Preble County|OH|40169|United States|-5|condo| +43094|AAAAAAAAGFIKAAAA|765|Walnut 2nd|Ln|Suite H|Springfield|Clark County|OH|49303|United States|-5|apartment| +43095|AAAAAAAAHFIKAAAA|600|Elm |Ln|Suite E|Glenwood|Lagrange County|IN|43511|United States|-5|single family| +43096|AAAAAAAAIFIKAAAA|468|Locust Church|Wy|Suite 20|Kingston|Carroll County|MD|24975|United States|-5|apartment| +43097|AAAAAAAAJFIKAAAA|343|13th |RD|Suite R|Pleasant Hill|Foard County|TX|73604|United States|-6|single family| +43098|AAAAAAAAKFIKAAAA|360|Main Ridge|ST|Suite 270|Highland|Lake County|FL|39454|United States|-5|single family| +43099|AAAAAAAALFIKAAAA|858|West Adams|Blvd|Suite 10|Springdale|Union County|NC|28883|United States|-5|condo| +43100|AAAAAAAAMFIKAAAA|833|Oak 2nd|Parkway|Suite 100|Wildwood|Jewell County|KS|66871|United States|-6|condo| +43101|AAAAAAAANFIKAAAA|944|Cedar Washington|Pkwy|Suite H|Owens|Klickitat County|WA|92324|United States|-8|single family| +43102|AAAAAAAAOFIKAAAA|255|First Oak|Boulevard|Suite 0|Providence|Montgomery County|AR|76614|United States|-6|condo| +43103|AAAAAAAAPFIKAAAA|293|Smith |Pkwy|Suite N|Edgewood|Saluda County|SC|20069|United States|-5|apartment| +43104|AAAAAAAAAGIKAAAA|497|Madison First|Lane|Suite D|Mount Zion|Montague County|TX|78054|United States|-6|single family| +43105|AAAAAAAABGIKAAAA|196|Lake |Wy|Suite 30|Bridgeport|Alfalfa County|OK|75817|United States|-6|apartment| +43106|AAAAAAAACGIKAAAA|793|Thirteenth Main|Road|Suite A|Hillcrest|Greene County|AL|33003|United States|-6|single family| +43107|AAAAAAAADGIKAAAA|601|10th |Wy|Suite O|Greenville|Fisher County|TX|71387|United States|-6|condo| +43108|AAAAAAAAEGIKAAAA|550|Walnut 8th|Road|Suite 190|Mount Pleasant|Lyon County|IA|51933|United States|-6|condo| +43109|AAAAAAAAFGIKAAAA|21|Sycamore River|Ave|Suite N|Sawyer|Taylor County|WV|26045|United States|-5|apartment| +43110|AAAAAAAAGGIKAAAA|336|7th 13th|Road|Suite 130|Bethel|Sharp County|AR|75281|United States|-6|apartment| +43111|AAAAAAAAHGIKAAAA|484|3rd 1st|Lane|Suite E|Kingston|Washington County|OK|74975|United States|-6|apartment| +43112|AAAAAAAAIGIKAAAA|499|8th Second|Dr.|Suite F|Plainview|Liberty County|TX|73683|United States|-6|condo| +43113|AAAAAAAAJGIKAAAA|498|Lake |Way|Suite J|Buckingham|Jefferson County|NY|14092|United States|-5|apartment| +43114|AAAAAAAAKGIKAAAA|589|Lincoln |Parkway|Suite Y|Wilson|Logan County|IL|66971|United States|-6|single family| +43115|AAAAAAAALGIKAAAA|443|3rd Miller|Parkway|Suite M|Royal|Southampton County|VA|25819|United States|-5|single family| +43116|AAAAAAAAMGIKAAAA|844|Lincoln |Ave|Suite 290|Hopewell|Jackson County|OR|90587|United States|-8|apartment| +43117|AAAAAAAANGIKAAAA|538|Central North|Court|Suite 350|Five Points|Middlesex County|NJ|06698|United States|-5|apartment| +43118|AAAAAAAAOGIKAAAA|677|Davis |Circle|Suite 400|Harmony|Union County|SD|55804|United States|-7|single family| +43119|AAAAAAAAPGIKAAAA|240|Eigth |ST|Suite E|Lakeview|Blaine County|OK|78579|United States|-6|apartment| +43120|AAAAAAAAAHIKAAAA|943|Ninth Main|Wy|Suite I|Hamilton|Russell County|VA|22808|United States|-5|apartment| +43121|AAAAAAAABHIKAAAA|650|Adams 4th|Drive|Suite 470|Gary|Cibola County|NM|80418|United States|-7|apartment| +43122|AAAAAAAACHIKAAAA|953|Chestnut |Cir.|Suite 330|Enterprise|Gallia County|OH|41757|United States|-5|apartment| +43123|AAAAAAAADHIKAAAA|914|8th |Wy|Suite 140|Hamilton|Harlan County|KY|42808|United States|-6|apartment| +43124|AAAAAAAAEHIKAAAA|409|Second |Drive|Suite 390|Enterprise|||31757|United States|-6|apartment| +43125|AAAAAAAAFHIKAAAA|758|Hillcrest 7th|RD|Suite 120|Plainview|Clark County|OH|43683|United States|-5|condo| +43126|AAAAAAAAGHIKAAAA|8|Elm |Court|Suite G|Hamilton|McLennan County|TX|72808|United States|-6|single family| +43127|AAAAAAAAHHIKAAAA|396|Fifth 3rd|Lane||Union Hill||||United States|-5|condo| +43128|AAAAAAAAIHIKAAAA||Woodland Green|Drive||Green Acres||OK||United States|-6|| +43129|AAAAAAAAJHIKAAAA|600|1st Cherry|Drive|Suite 450|Greenfield|Benton County|OR|95038|United States|-8|apartment| +43130|AAAAAAAAKHIKAAAA|340|Oak |Street|Suite O|Oak Hill|Greene County|OH|47838|United States|-5|condo| +43131|AAAAAAAALHIKAAAA|810|Laurel 7th|Boulevard|Suite 190|Lakeview|Rio Blanco County|CO|88579|United States|-7|condo| +43132|AAAAAAAAMHIKAAAA|783|Smith |Wy|Suite V|Glendale|Benton County|TN|33951|United States|-5|apartment| +43133|AAAAAAAANHIKAAAA|729|Fourteenth Spring|Ct.|Suite G|Pine Grove|Webster Parish|LA|74593|United States|-6|condo| +43134|AAAAAAAAOHIKAAAA|446|Wilson Cedar|Circle|Suite 250|Green Acres|Missoula County|MT|67683|United States|-7|single family| +43135|AAAAAAAAPHIKAAAA|40|1st |Circle|Suite 200|Spring Valley|Leelanau County|MI|46060|United States|-5|condo| +43136|AAAAAAAAAIIKAAAA|484|Maple Third|Road|Suite 30|Bethel|Schuyler County|IL|65281|United States|-6|condo| +43137|AAAAAAAABIIKAAAA|550|Fourth |Way|Suite 420|Spring Valley|Dunn County|WI|56060|United States|-6|single family| +43138|AAAAAAAACIIKAAAA|96|Meadow |RD|Suite S|Pleasant Grove|Rockwall County|TX|74136|United States|-6|single family| +43139|AAAAAAAADIIKAAAA|268|Valley |Blvd|Suite 360|Marion|Grundy County|IA|50399|United States|-6|apartment| +43140|AAAAAAAAEIIKAAAA|480|Poplar Mill|Dr.|Suite C|Jamestown|Lincoln County|MT|66867|United States|-7|apartment| +43141|AAAAAAAAFIIKAAAA|508|Miller Spring|ST|Suite 270|Jackson|Bethel Census Area|AK|99583|United States|-9|single family| +43142|AAAAAAAAGIIKAAAA|87|Park |Blvd|Suite U|Cumberland|Logan County|IL|68971|United States|-6|condo| +43143|AAAAAAAAHIIKAAAA|842|Lake Elm|ST|Suite J|Spring Hill|Pine County|MN|56787|United States|-6|condo| +43144|AAAAAAAAIIIKAAAA|41|Mill |RD|Suite 90|Five Points|New Madrid County|MO|66098|United States|-6|apartment| +43145|AAAAAAAAJIIKAAAA|216|Mill |Lane|Suite H|Friendship|Big Horn County|MT|64536|United States|-7|condo| +43146|AAAAAAAAKIIKAAAA|222|Laurel 5th|Street|Suite S|Shiloh|Pendleton County|KY|49275|United States|-5|single family| +43147|AAAAAAAALIIKAAAA|450|13th |Avenue|Suite 380|Pleasant Hill|Banner County|NE|63604|United States|-6|condo| +43148|AAAAAAAAMIIKAAAA|647|Laurel |Circle|Suite 380|Kingston|Powder River County|MT|64975|United States|-7|condo| +43149|AAAAAAAANIIKAAAA|154|Mill |Parkway|Suite 440|Edgewood|Skamania County|WA|90069|United States|-8|apartment| +43150|AAAAAAAAOIIKAAAA|806|Fourth 12th|Dr.|Suite 310|Union|Wood County|WI|58721|United States|-6|condo| +43151|AAAAAAAAPIIKAAAA|119|Laurel Center|Parkway|Suite 40|Wilson|Lane County|OR|96971|United States|-8|single family| +43152|AAAAAAAAAJIKAAAA|833|Jefferson |Ct.|Suite K|Mount Vernon|McDonald County|MO|68482|United States|-6|condo| +43153|AAAAAAAABJIKAAAA|251|9th |Way|Suite H|Unionville|Laurel County|KY|41711|United States|-5|condo| +43154|AAAAAAAACJIKAAAA|838|Washington |Cir.|Suite 450|Brownsville|Sweet Grass County|MT|69310|United States|-7|apartment| +43155|AAAAAAAADJIKAAAA|721|Main Birch|Court|Suite O|Union Hill|Spartanburg County|SC|27746|United States|-5|single family| +43156|AAAAAAAAEJIKAAAA|942|Elevnth |Pkwy|Suite 80|Mount Vernon|Contra Costa County|CA|98482|United States|-8|apartment| +43157|AAAAAAAAFJIKAAAA|516|2nd |Dr.|Suite 340|Clifton|Dillon County|SC|28014|United States|-5|condo| +43158|AAAAAAAAGJIKAAAA|647|Adams Hill|Cir.|Suite 240|Montpelier|Polk County|WI|58930|United States|-6|apartment| +43159|AAAAAAAAHJIKAAAA|626|10th 9th|Wy|Suite 440|Forest Hills|Montgomery County|IA|59237|United States|-6|apartment| +43160|AAAAAAAAIJIKAAAA|520|Park |Ln|Suite V|Enterprise|Lake County|CO|81757|United States|-7|apartment| +43161|AAAAAAAAJJIKAAAA|405|Miller |Circle|Suite U|Highland Park|Okfuskee County|OK|76534|United States|-6|condo| +43162|AAAAAAAAKJIKAAAA|398|Williams Chestnut|Road|Suite L|Buena Vista|Neshoba County|MS|55752|United States|-6|condo| +43163|AAAAAAAALJIKAAAA|97|7th 12th|RD|Suite C|Springfield|Gulf County|FL|39303|United States|-5|condo| +43164|AAAAAAAAMJIKAAAA|842|1st Spruce|Circle|Suite 60|Spring Hill|Knox County|OH|46787|United States|-5|apartment| +43165|AAAAAAAANJIKAAAA|73|Oak Ridge|Street|Suite 190|Rossville|Polk County|FL|32503|United States|-5|single family| +43166|AAAAAAAAOJIKAAAA|561|West |ST|Suite N|Mount Pleasant|Racine County|WI|51933|United States|-6|condo| +43167|AAAAAAAAPJIKAAAA|188|Walnut |Road|Suite 100|Wildwood|Lake County|OH|46871|United States|-5|apartment| +43168|AAAAAAAAAKIKAAAA|788|Park |Street|Suite 380|Morgantown|Luzerne County|PA|19193|United States|-5|condo| +43169|AAAAAAAABKIKAAAA|859|9th East|Ln|Suite A|Oak Hill|Presque Isle County|MI|47838|United States|-5|condo| +43170|AAAAAAAACKIKAAAA|661|First Highland|Ct.|Suite 310|Unionville|Carroll County|IA|51711|United States|-6|single family| +43171|AAAAAAAADKIKAAAA|327|Spring |Parkway|Suite 440|Liberty|Highland County|VA|23451|United States|-5|apartment| +43172|AAAAAAAAEKIKAAAA|634|2nd View|RD|Suite S|Maple Grove|Custer County|ID|88252|United States|-7|single family| +43173|AAAAAAAAFKIKAAAA|574|Railroad South|Avenue|Suite Q|Walnut Grove|Accomack County|VA|27752|United States|-5|apartment| +43174|AAAAAAAAGKIKAAAA|999|Meadow |RD|Suite 330|Ellisville|Juniata County|PA|16820|United States|-5|condo| +43175|AAAAAAAAHKIKAAAA|820|Willow Park|Boulevard|Suite L|New Hope|Clark County|OH|49431|United States|-5|condo| +43176|AAAAAAAAIKIKAAAA|82|8th Smith|Drive|Suite I|Oak Ridge|Le Flore County|OK|78371|United States|-6|apartment| +43177|AAAAAAAAJKIKAAAA|758|2nd Miller|Dr.|Suite D|Pleasant Valley|Dickinson County|KS|62477|United States|-6|single family| +43178|AAAAAAAAKKIKAAAA|830|College 5th|Ln|Suite 200|Georgetown|Ulster County|NY|17057|United States|-5|single family| +43179|AAAAAAAALKIKAAAA|370|Park |Boulevard|Suite 180|Union Hill|Kent County|MI|47746|United States|-5|apartment| +43180|AAAAAAAAMKIKAAAA|847|Jackson |RD|Suite 470|Plainview|Wyandot County|OH|43683|United States|-5|condo| +43181|AAAAAAAANKIKAAAA|935|Hill |Ct.|Suite 480|Plainview|Franklin County|PA|13683|United States|-5|apartment| +43182|AAAAAAAAOKIKAAAA|477|Fourth |Drive|Suite I|Hurricane|Missoula County|MT|67644|United States|-7|single family| +43183|AAAAAAAAPKIKAAAA|351|4th |RD|Suite 40|Green Acres|Cambria County|PA|17683|United States|-5|apartment| +43184|AAAAAAAAALIKAAAA|122|College 3rd||Suite 490|Fairfield|Lake County||||-5|condo| +43185|AAAAAAAABLIKAAAA|796|6th 2nd|Lane|Suite 490|Frankfort|Monroe County|MI|49681|United States|-5|single family| +43186|AAAAAAAACLIKAAAA|834|11th |Ln|Suite I|White Oak|Pacific County|WA|96668|United States|-8|condo| +43187|AAAAAAAADLIKAAAA|154|Green Lincoln|Blvd|Suite Q|Lincoln|Osage County|MO|61289|United States|-6|single family| +43188|AAAAAAAAELIKAAAA|130|Eigth |Drive|Suite 380|Jamestown|Monroe County|KY|46867|United States|-5|single family| +43189|AAAAAAAAFLIKAAAA||College West||||Woodruff County||74107||-6|apartment| +43190|AAAAAAAAGLIKAAAA|165|Lee |RD|Suite G|Oakland|Wood County|WV|29843|United States|-5|single family| +43191|AAAAAAAAHLIKAAAA|66|Dogwood Elm|Lane|Suite U|Farmington|San Miguel County|CO|89145|United States|-7|apartment| +43192|AAAAAAAAILIKAAAA|638|Lee |Road|Suite 450|Hopewell|Tucker County|WV|20587|United States|-5|apartment| +43193|AAAAAAAAJLIKAAAA|533|North |Lane|Suite Y|Buena Vista|Buena Vista County|IA|55752|United States|-6|apartment| +43194|AAAAAAAAKLIKAAAA|189|Hillcrest |Court|Suite 340|Clifton|Morgan County|WV|28014|United States|-5|apartment| +43195|AAAAAAAALLIKAAAA|726|Second Lake|Circle|Suite 400|Sunnyside|Beaver County|OK|71952|United States|-6|apartment| +43196|AAAAAAAAMLIKAAAA|676|2nd |Lane|Suite 220|Edgewood|Lake County|TN|30069|United States|-6|single family| +43197|AAAAAAAANLIKAAAA|737|Maple |Dr.|Suite 480|Springfield|Warren County|NJ|09903|United States|-5|condo| +43198|AAAAAAAAOLIKAAAA|771|Broadway |RD|Suite X|Ashland|Jefferson County|MS|54244|United States|-6|single family| +43199|AAAAAAAAPLIKAAAA|817||Ave||Lakeview|Richmond city|||||| +43200|AAAAAAAAAMIKAAAA|897|4th |Cir.|Suite 310|Hubbard|Jim Hogg County|TX|76291|United States|-6|apartment| +43201|AAAAAAAABMIKAAAA|930|Spring Third|Ct.|Suite Y|Bethel|Fulton County|IL|65281|United States|-6|condo| +43202|AAAAAAAACMIKAAAA|358|Chestnut |Way|Suite Y|Centerville|Outagamie County|WI|50059|United States|-6|apartment| +43203|AAAAAAAADMIKAAAA|109|Washington |Wy|Suite J|Five Forks|Pierce County|NE|62293|United States|-7|single family| +43204|AAAAAAAAEMIKAAAA|419|Chestnut Locust|Dr.|Suite 50|Franklin|Marion County|MO|69101|United States|-6|apartment| +43205|AAAAAAAAFMIKAAAA|644|5th Central|Cir.|Suite 430|Stringtown|Cache County|UT|80162|United States|-7|single family| +43206|AAAAAAAAGMIKAAAA|783|Walnut |Way|Suite 110|Concord|Dickey County|ND|54107|United States|-6|single family| +43207|AAAAAAAAHMIKAAAA|392|7th Hillcrest|Pkwy|Suite B|Glenville|Orange County|NY|13445|United States|-5|apartment| +43208|AAAAAAAAIMIKAAAA|821|Cedar Oak|Dr.|Suite 300|Woodland|Guthrie County|IA|54854|United States|-6|single family| +43209|AAAAAAAAJMIKAAAA|531|Washington |Drive|Suite 340|Marion|Luna County|NM|80399|United States|-7|single family| +43210|AAAAAAAAKMIKAAAA||Valley Poplar||Suite P||Carroll County||54854|United States|-6|condo| +43211|AAAAAAAALMIKAAAA|291|Franklin 15th|RD|Suite R|Woodlawn|Wilkes County|GA|34098|United States|-5|condo| +43212|AAAAAAAAMMIKAAAA|271|Hill |Ave|Suite N|Oakwood|Crawford County|OH|40169|United States|-5|apartment| +43213|AAAAAAAANMIKAAAA|159|Pine 3rd|Ct.|Suite C|Crossroads|Huron County|OH|40534|United States|-5|condo| +43214|AAAAAAAAOMIKAAAA|421|Poplar Walnut|Drive|Suite P|Salem|Alameda County|CA|98048|United States|-8|condo| +43215|AAAAAAAAPMIKAAAA|294|10th |Avenue|Suite O|White Oak|Burke County|ND|56668|United States|-6|apartment| +43216|AAAAAAAAANIKAAAA|897|4th West|Ct.|Suite 80|Pleasant Hill|Washington County|TN|33604|United States|-6|condo| +43217|AAAAAAAABNIKAAAA|271|Birch |Road|Suite T|Wildwood|Dakota County|NE|66871|United States|-6|condo| +43218|AAAAAAAACNIKAAAA|303|First |Boulevard|Suite 30|Providence|Hempstead County|AR|76614|United States|-6|single family| +43219|AAAAAAAADNIKAAAA|831|Hillcrest Dogwood|Court|Suite M|Brownsville|Baker County|GA|39310|United States|-5|single family| +43220|AAAAAAAAENIKAAAA|384|Oak |Court|Suite 430|Oakland|Orange County|NC|29843|United States|-5|apartment| +43221|AAAAAAAAFNIKAAAA|448|Woodland |Pkwy|Suite R|Sunnyside|Hancock County|GA|31952|United States|-5|apartment| +43222|AAAAAAAAGNIKAAAA|514|8th 8th|Drive|Suite X|Lebanon|Roane County|TN|32898|United States|-6|apartment| +43223|AAAAAAAAHNIKAAAA|659|Cherry |Circle|Suite E|Friendship|Washita County|OK|74536|United States|-6|condo| +43224|AAAAAAAAINIKAAAA|455|Maple Center|Ct.|Suite W|West Liberty|Ozark County|MO|64752|United States|-6|single family| +43225|AAAAAAAAJNIKAAAA|119|Walnut Johnson|Avenue|Suite C|Arlington|Childress County|TX|76557|United States|-6|apartment| +43226|AAAAAAAAKNIKAAAA|280|5th 2nd|Drive|Suite 330|Crossroads|Henry County|GA|30534|United States|-5|condo| +43227|AAAAAAAALNIKAAAA|144|Johnson |Blvd|Suite O|Greenfield|Nowata County|OK|75038|United States|-6|condo| +43228|AAAAAAAAMNIKAAAA|410|Poplar |Boulevard|Suite F|Harvey|Franklin County|NC|25858|United States|-5|apartment| +43229|AAAAAAAANNIKAAAA|755|Jackson Washington|Street|Suite 250|Riverdale|Jones County|SD|59391|United States|-7|condo| +43230|AAAAAAAAONIKAAAA|726|Elm Pine|Ct.|Suite 160|Bunker Hill|Tyler County|WV|20150|United States|-5|apartment| +43231|AAAAAAAAPNIKAAAA|643|Hillcrest Sycamore|Ave|Suite I|Maple Grove|Lassen County|CA|98252|United States|-8|condo| +43232|AAAAAAAAAOIKAAAA|437|Park South|RD|Suite 490|Sunnyside|Tolland County|CT|02552|United States|-5|condo| +43233|AAAAAAAABOIKAAAA|54|First |Ave|Suite 50|Jackson|Tuscaloosa County|AL|39583|United States|-6|single family| +43234|AAAAAAAACOIKAAAA|992|Lincoln |Drive|Suite Y|Glenwood|Avoyelles Parish|LA|73511|United States|-6|condo| +43235|AAAAAAAADOIKAAAA|539|Sunset 6th|Parkway|Suite U|Hillcrest|Bailey County|TX|73003|United States|-6|single family| +43236|AAAAAAAAEOIKAAAA|107|South |Avenue|Suite C|Maple Grove|Morton County|ND|58252|United States|-6|condo| +43237|AAAAAAAAFOIKAAAA|736|8th Broadway|Dr.|Suite N|Harmony|Martin County|FL|35804|United States|-5|apartment| +43238|AAAAAAAAGOIKAAAA|305|Fourth |Ave|Suite 220|Bridgeport|Fremont County|CO|85817|United States|-7|apartment| +43239|AAAAAAAAHOIKAAAA|798|Walnut |Street|Suite 390|Enterprise|Lake County|CA|91757|United States|-8|single family| +43240|AAAAAAAAIOIKAAAA|706|Central Highland|Street|Suite G|Mount Zion|Lawrence County|MS|58054|United States|-6|apartment| +43241|AAAAAAAAJOIKAAAA|181|Walnut |Cir.|Suite A|Hopewell|Douglas County|NE|60587|United States|-6|condo| +43242|AAAAAAAAKOIKAAAA|504|2nd |Lane|Suite B|Oakland|Washoe County|NV|89843|United States|-8|condo| +43243|AAAAAAAALOIKAAAA|734|Williams Spring|RD|Suite 40|Pine Grove|Chippewa County|MN|54593|United States|-6|single family| +43244|AAAAAAAAMOIKAAAA|922|Mill |Ln|Suite 180|Pine Grove|Traverse County|MN|54593|United States|-6|apartment| +43245|AAAAAAAANOIKAAAA|321|West |Avenue|Suite 350|Oak Hill|Columbia County|WI|57838|United States|-6|condo| +43246|AAAAAAAAOOIKAAAA|947|Park Wilson|Ln|Suite 270|Cedar Grove|Angelina County|TX|70411|United States|-6|condo| +43247|AAAAAAAAPOIKAAAA|459|Fourth Third|Court|Suite F|Red Hill|Jackson County|OH|44338|United States|-5|apartment| +43248|AAAAAAAAAPIKAAAA|628|10th |Dr.|Suite Y|Five Forks|Camp County|TX|72293|United States|-6|apartment| +43249|AAAAAAAABPIKAAAA|859|13th 1st|Court|Suite G|Shiloh|Lafayette County|MO|69275|United States|-6|single family| +43250|AAAAAAAACPIKAAAA|667|Spring |Circle|Suite 150|Leland|Wallowa County|OR|99452|United States|-8|single family| +43251|AAAAAAAADPIKAAAA|358|Highland 9th|Street|Suite 0|Cedar Grove|Upshur County|WV|20411|United States|-5|single family| +43252|AAAAAAAAEPIKAAAA|498|Park |Ct.|Suite 480|Granite|Richmond County|NC|26284|United States|-5|condo| +43253|AAAAAAAAFPIKAAAA|107|3rd |Dr.|Suite 120|Riverdale|Acadia Parish|LA|79391|United States|-6|single family| +43254|AAAAAAAAGPIKAAAA|936|Williams |Avenue|Suite U|Fairview|Cass County|IA|55709|United States|-6|apartment| +43255|AAAAAAAAHPIKAAAA|445|6th |Dr.|Suite P|Centerville|Delta County|MI|40059|United States|-5|single family| +43256|AAAAAAAAIPIKAAAA|570|Oak Center|RD|Suite 90|Mount Olive|Niobrara County|WY|88059|United States|-7|condo| +43257|AAAAAAAAJPIKAAAA|545|9th Seventh|Circle|Suite 300|Hillcrest|Monongalia County|WV|23003|United States|-5|condo| +43258|AAAAAAAAKPIKAAAA|126|Oak Second|Way|Suite 160|Greenwood|Bosque County|TX|78828|United States|-6|single family| +43259|AAAAAAAALPIKAAAA|763|Spring Highland|Circle|Suite 210|Summit|Bourbon County|KS|60499|United States|-6|single family| +43260|AAAAAAAAMPIKAAAA|343|Lake Park|Avenue|Suite S|Fairfield|Montgomery County|TX|76192|United States|-6|apartment| +43261|AAAAAAAANPIKAAAA|216|First |Ln|Suite 70|Newtown|Cache County|UT|81749|United States|-7|condo| +43262|AAAAAAAAOPIKAAAA|247|Fourteenth 7th|Avenue|Suite 260|Pleasant Hill|Chesapeake city|VA|23604|United States|-5|condo| +43263|AAAAAAAAPPIKAAAA||||Suite A|||SC||United States||apartment| +43264|AAAAAAAAAAJKAAAA|66|Laurel First|Circle|Suite 460|Forest Hills|Dodge County|WI|59237|United States|-6|condo| +43265|AAAAAAAABAJKAAAA|802|12th Mill|Circle|Suite F|Woodlawn|Middlesex County|MA|04698|United States|-5|condo| +43266|AAAAAAAACAJKAAAA|466|Chestnut 7th|Pkwy|Suite 270|Walnut Grove|Tompkins County|NY|17752|United States|-5|apartment| +43267|AAAAAAAADAJKAAAA|||Wy|Suite 280|||TX|70534|||| +43268|AAAAAAAAEAJKAAAA|734|||Suite A|||FL|36971|||apartment| +43269|AAAAAAAAFAJKAAAA|68|7th |Road|Suite U|Oak Hill|Coal County|OK|77838|United States|-6|apartment| +43270|AAAAAAAAGAJKAAAA|663|Birch Sixth|Ln|Suite 270|Edgewood|Hernando County|FL|30069|United States|-5|single family| +43271|AAAAAAAAHAJKAAAA|42|Spring |Ct.|Suite 240|Franklin|Campbell County|VA|29101|United States|-5|single family| +43272|AAAAAAAAIAJKAAAA|974|4th |Boulevard|Suite B|Crossroads|Davis County|IA|50534|United States|-6|single family| +43273|AAAAAAAAJAJKAAAA|299|Valley Railroad|Ln|Suite S|Waterloo|Bell County|KY|41675|United States|-6|apartment| +43274|AAAAAAAAKAJKAAAA|92|15th Elm|Wy|Suite 420|New Hope|Putnam County|GA|39431|United States|-5|apartment| +43275|AAAAAAAALAJKAAAA|877|Cherry |Avenue|Suite I|Pleasant Grove|Ocean County|NJ|04736|United States|-5|apartment| +43276|AAAAAAAAMAJKAAAA|309|Lincoln |Ct.|Suite 360|Union Hill|Jefferson County|AR|77746|United States|-6|apartment| +43277|AAAAAAAANAJKAAAA|331|13th 3rd|ST|Suite 160|Concord|Smyth County|VA|24107|United States|-5|condo| +43278|AAAAAAAAOAJKAAAA|488|South Chestnut|Lane|Suite G|Lebanon|Putnam County|NY|12898|United States|-5|apartment| +43279|AAAAAAAAPAJKAAAA|493|Wilson Spruce|Road|Suite U|Unionville|Assumption Parish|LA|71711|United States|-6|apartment| +43280|AAAAAAAAABJKAAAA|335|Mill Chestnut|Road|Suite D|Salem|Tuolumne County|CA|98048|United States|-8|single family| +43281|AAAAAAAABBJKAAAA|630|View |Ln|Suite 290|Wildwood|Hancock County|IN|46871|United States|-5|condo| +43282|AAAAAAAACBJKAAAA|104|Maple Forest|Street|Suite 200|Riverview|Gila County|AZ|89003|United States|-7|single family| +43283|AAAAAAAADBJKAAAA|457|Eigth |Blvd|Suite Y|Plainview|Sherman County|TX|73683|United States|-6|condo| +43284|AAAAAAAAEBJKAAAA|907|Forest |RD|Suite E|Woodland|Sharkey County|MS|54854|United States|-6|apartment| +43285|AAAAAAAAFBJKAAAA|371|9th |Ct.|Suite 110|Shady Grove|Ochiltree County|TX|72812|United States|-6|single family| +43286|AAAAAAAAGBJKAAAA|865|First Lake|Avenue|Suite N|Kimball|Highland County|OH|43595|United States|-5|condo| +43287|AAAAAAAAHBJKAAAA|378|Sycamore |RD|Suite 60|Snug Harbor|Martin County|KY|47936|United States|-5|condo| +43288|AAAAAAAAIBJKAAAA|620|Laurel Park|Parkway|Suite 80|Summit|Washington County|MO|60499|United States|-6|apartment| +43289|AAAAAAAAJBJKAAAA|124|11th |Pkwy|Suite K|Shady Grove|Miller County|GA|32812|United States|-5|single family| +43290|AAAAAAAAKBJKAAAA|129|East |Ave|Suite P|Woodland|Greene County|AR|74854|United States|-6|apartment| +43291|AAAAAAAALBJKAAAA|139|Central Cedar|Pkwy|Suite A|Hillcrest|Barron County|WI|53003|United States|-6|apartment| +43292|AAAAAAAAMBJKAAAA|908|15th |Ave|Suite D|Crossroads|Oxford County|ME|01134|United States|-5|single family| +43293|AAAAAAAANBJKAAAA|728|North |Dr.|Suite 330|Woodville|Clay County|NE|64289|United States|-6|condo| +43294|AAAAAAAAOBJKAAAA|454|Walnut Johnson|Boulevard|Suite G|Jackson|Gilmer County|WV|29583|United States|-5|apartment| +43295|AAAAAAAAPBJKAAAA|971||||Wilson||||||| +43296|AAAAAAAAACJKAAAA|132|Tenth |Lane|Suite Q|Clifton|Bee County|TX|78014|United States|-6|single family| +43297|AAAAAAAABCJKAAAA|741|15th |Wy|Suite K|Bridgeport|Pulaski County|AR|75817|United States|-6|single family| +43298|AAAAAAAACCJKAAAA|209|Hickory |Street|Suite 120|Clifton|Huron County|OH|48014|United States|-5|condo| +43299|AAAAAAAADCJKAAAA|301|3rd Park|Lane|Suite 370|Chestnut Ridge|Fulton County|OH|47334|United States|-5|single family| +43300|AAAAAAAAECJKAAAA|602|3rd |RD|Suite 20|Shady Grove|Burleigh County|ND|52812|United States|-6|condo| +43301|AAAAAAAAFCJKAAAA|303|10th |Court|Suite K|Edgewood|El Paso County|TX|70069|United States|-6|single family| +43302|AAAAAAAAGCJKAAAA|854|Ash |Lane|Suite L|Hillcrest|Lewis County|KY|43003|United States|-5|condo| +43303|AAAAAAAAHCJKAAAA|711|7th Madison|Court|Suite 300|Summit|Bannock County|ID|80499|United States|-7|single family| +43304|AAAAAAAAICJKAAAA|606|Pine |Court|Suite X|Deerfield|Limestone County|AL|39840|United States|-6|apartment| +43305|AAAAAAAAJCJKAAAA|904|Hill |ST|Suite C|Mount Olive|Otsego County|NY|18059|United States|-5|single family| +43306|AAAAAAAAKCJKAAAA|582|Church Hillcrest|Way|Suite C|Wright|Nottoway County|VA|22814|United States|-5|apartment| +43307|AAAAAAAALCJKAAAA|571|Woodland Oak|Blvd|Suite Y|Lee|Jefferson County|FL|30408|United States|-5|condo| +43308|AAAAAAAAMCJKAAAA|407|7th River|Ct.|Suite 30|Deerfield|Colorado County|TX|79840|United States|-6|apartment| +43309|AAAAAAAANCJKAAAA|380|North View|Avenue|Suite 160|Centerville|Richardson County|NE|60059|United States|-7|apartment| +43310|AAAAAAAAOCJKAAAA|912|13th Chestnut|Way|Suite 370|Farmington|Randolph County|MO|69145|United States|-6|condo| +43311|AAAAAAAAPCJKAAAA|169|Jefferson |Wy|Suite S|Sulphur Springs|Polk County|AR|78354|United States|-6|apartment| +43312|AAAAAAAAADJKAAAA|295|5th |Avenue|Suite 60|Edgewood|Oconee County|SC|20069|United States|-5|condo| +43313|AAAAAAAABDJKAAAA|751|3rd |Dr.|Suite X|Greenville|Ohio County|KY|41387|United States|-5|apartment| +43314|AAAAAAAACDJKAAAA|744|2nd Hill|Boulevard|Suite E|Bunker Hill|Addison County|VT|00750|United States|-5|apartment| +43315|AAAAAAAADDJKAAAA|877|Locust |Ln|Suite V|Cedar Grove|Decatur County|KS|60411|United States|-6|single family| +43316|AAAAAAAAEDJKAAAA|304|Second |RD|Suite X|Lebanon|Coffey County|KS|62898|United States|-6|condo| +43317|AAAAAAAAFDJKAAAA|703|6th |Pkwy|Suite Y|Lakewood|Divide County|ND|58877|United States|-6|apartment| +43318|AAAAAAAAGDJKAAAA|988|Poplar |Pkwy|Suite 300|Wheatland|Thurston County|WA|94760|United States|-8|apartment| +43319|AAAAAAAAHDJKAAAA|793|Spruce |Blvd|Suite V|Ashland|Hinds County|MS|54244|United States|-6|apartment| +43320|AAAAAAAAIDJKAAAA|894|Williams Madison|Ln|Suite 80|Pine Grove|Monroe County|MS|54593|United States|-6|apartment| +43321|AAAAAAAAJDJKAAAA|904|Cherry First|Court|Suite X|Greenfield|Stutsman County|ND|55038|United States|-6|condo| +43322|AAAAAAAAKDJKAAAA|714|Park |Blvd|Suite 430|Phoenix|Upshur County|WV|22276|United States|-5|condo| +43323|AAAAAAAALDJKAAAA||Washington |||||||||condo| +43324|AAAAAAAAMDJKAAAA|547|Jackson Spring|Ave|Suite 450|Springdale|Lawrence County|AL|38883|United States|-6|single family| +43325|AAAAAAAANDJKAAAA|103|Highland 7th|Court|Suite I|Wilson|Jenkins County|GA|36971|United States|-5|single family| +43326|AAAAAAAAODJKAAAA|256|5th |Blvd||Crossroads|Fairfax city|VA|20534||-5|apartment| +43327|AAAAAAAAPDJKAAAA|86|Highland Spring|RD|Suite 240|Deerfield|Uvalde County|TX|79840|United States|-6|single family| +43328|AAAAAAAAAEJKAAAA|217|2nd Pine|Drive|Suite 220|Friendship|Clay County|WV|24536|United States|-5|single family| +43329|AAAAAAAABEJKAAAA|391|Seventh |Road|Suite Q|Oakwood|Liberty County|GA|30169|United States|-5|single family| +43330|AAAAAAAACEJKAAAA|287|Main |Blvd|Suite 330|Lakewood|Spotsylvania County|VA|28877|United States|-5|condo| +43331|AAAAAAAADEJKAAAA|224|Fifth |Ct.|Suite 300|Wayland|Cook County|MN|55115|United States|-6|apartment| +43332|AAAAAAAAEEJKAAAA|425|Pine |Ln|Suite 10|Plainview|Cass County|IN|43683|United States|-5|condo| +43333|AAAAAAAAFEJKAAAA|205|9th Hillcrest|Dr.|Suite 80|Spring Hill|Alexander County|NC|26787|United States|-5|apartment| +43334|AAAAAAAAGEJKAAAA|477|3rd Pine|ST|Suite Y|Glenwood|Coke County|TX|73511|United States|-6|single family| +43335|AAAAAAAAHEJKAAAA|287|Railroad Second|Way|Suite 440|New Hope|Hamlin County|SD|59431|United States|-7|single family| +43336|AAAAAAAAIEJKAAAA|679|13th Davis|Wy|Suite 340|Bennett|Rappahannock County|VA|21715|United States|-5|condo| +43337|AAAAAAAAJEJKAAAA|724|Lincoln Jefferson|Road|Suite 200|Wildwood|Fremont County|WY|86871|United States|-7|condo| +43338|AAAAAAAAKEJKAAAA|373|Oak Franklin|Blvd|Suite 340|Kingston|Dade County|MO|64975|United States|-6|apartment| +43339|AAAAAAAALEJKAAAA|7|Main 4th|Blvd|Suite K|Unionville|Washington County|VA|21711|United States|-5|single family| +43340|AAAAAAAAMEJKAAAA|271|Park |Ave|Suite 330|Woodlawn|Mingo County|WV|24098|United States|-5|apartment| +43341|AAAAAAAANEJKAAAA|401|Elm |Pkwy|Suite W|Ridgeville|Cotton County|OK|79306|United States|-6|single family| +43342|AAAAAAAAOEJKAAAA|545|Church 11th|Drive|Suite 300|Murphy|Clay County|WV|22105|United States|-5|apartment| +43343|AAAAAAAAPEJKAAAA|584|Chestnut Lee|ST|Suite M|Shaw|Culberson County|TX|70618|United States|-6|apartment| +43344|AAAAAAAAAFJKAAAA|848|Ridge |||||GA||United States|-5|single family| +43345|AAAAAAAABFJKAAAA|967|Miller |Court|Suite D|Newtown|Mitchell County|NC|21749|United States|-5|apartment| +43346|AAAAAAAACFJKAAAA|92|South Wilson|Ct.|Suite 80|Midway|Washington County|UT|81904|United States|-7|apartment| +43347|AAAAAAAADFJKAAAA|308|10th |Boulevard|Suite J|Crossroads|Aroostook County|ME|01134|United States|-5|condo| +43348|AAAAAAAAEFJKAAAA|541|4th |Boulevard|Suite T|Gladstone|Lamar County|GA|30894|United States|-5|condo| +43349|AAAAAAAAFFJKAAAA|371|Elevnth North|Avenue|Suite O|Cedar Grove|Lake County|MN|50411|United States|-6|condo| +43350|AAAAAAAAGFJKAAAA|137|Johnson |Parkway|Suite 450|Lakeview|Crawford County|IA|58579|United States|-6|single family| +43351|AAAAAAAAHFJKAAAA|488|Lee |Road|Suite 230|Spring Valley|Audrain County|MO|66060|United States|-6|apartment| +43352|AAAAAAAAIFJKAAAA|421|Franklin Williams|Way|Suite 440|Mount Pleasant|Bristol County|MA|02533|United States|-5|condo| +43353|AAAAAAAAJFJKAAAA|456|4th |Court|Suite 170|Nottingham|Edgecombe County|NC|24074|United States|-5|condo| +43354|AAAAAAAAKFJKAAAA|603|Maple Second|RD|Suite 230|Glendale|Williamson County|TX|73951|United States|-6|apartment| +43355|AAAAAAAALFJKAAAA|275|Lake |Wy|Suite 270|Highland Park|Calhoun County|GA|36534|United States|-5|condo| +43356|AAAAAAAAMFJKAAAA|910|Green 3rd|Street|Suite 140|Union|Fountain County|IN|48721|United States|-5|condo| +43357|AAAAAAAANFJKAAAA|995|Walnut |Road|Suite I|Maple Grove|Wayne County|IL|68252|United States|-6|single family| +43358|AAAAAAAAOFJKAAAA|816|River |RD|Suite O|Highland|Crockett County|TX|79454|United States|-6|condo| +43359|AAAAAAAAPFJKAAAA||View |Dr.||Oak Hill||AZ|||-7|apartment| +43360|AAAAAAAAAGJKAAAA|837|Franklin |Lane|Suite 310|Belleville|Athens County|OH|42924|United States|-5|single family| +43361|AAAAAAAABGJKAAAA|44|Park |Parkway|Suite 120|Maple Grove|Chesapeake city|VA|28252|United States|-5|apartment| +43362|AAAAAAAACGJKAAAA|677|Hillcrest |Dr.|Suite I|Hamilton|Washington County|RI|03408|United States|-5|single family| +43363|AAAAAAAADGJKAAAA|367|8th |Circle|Suite W|Lakeville|Donley County|TX|78811|United States|-6|single family| +43364|AAAAAAAAEGJKAAAA|257|Fourth |Ave|Suite T|Hopewell|Bartholomew County|IN|40587|United States|-5|apartment| +43365|AAAAAAAAFGJKAAAA|674|Elevnth Twelfth|Cir.|Suite 220|Fisher|Ritchie County|WV|22819|United States|-5|apartment| +43366|AAAAAAAAGGJKAAAA|580|Madison |ST|Suite N|Antioch|Halifax County|NC|28605|United States|-5|condo| +43367|AAAAAAAAHGJKAAAA|41|11th |Drive|Suite 460|Maple Grove|Putnam County|TN|38252|United States|-6|apartment| +43368|AAAAAAAAIGJKAAAA|483|Smith 4th|Way|Suite L|Stringtown|Harmon County|OK|70162|United States|-6|apartment| +43369|AAAAAAAAJGJKAAAA|12|Church Hill|Blvd|Suite 130|Union|Young County|TX|78721|United States|-6|apartment| +43370|AAAAAAAAKGJKAAAA|418|South 8th|RD|Suite 300|Johnsonville|Clay County|GA|37745|United States|-5|condo| +43371|AAAAAAAALGJKAAAA|203|Hillcrest Church|Ln|Suite 10|Harmony|Poinsett County|AR|75804|United States|-6|apartment| +43372|AAAAAAAAMGJKAAAA|795|Willow Pine|Dr.|Suite 210|Amity|Hardin County|TN|30766|United States|-5|condo| +43373|AAAAAAAANGJKAAAA|804|Park Center|Avenue|Suite H|Spring Hill|Sullivan County|PA|16787|United States|-5|condo| +43374|AAAAAAAAOGJKAAAA|217|7th Adams|Blvd|Suite 310|Spring Valley|Coffee County|AL|36060|United States|-6|condo| +43375|AAAAAAAAPGJKAAAA|250|Adams |Lane|Suite 320|Snug Harbor|Columbiana County|OH|47936|United States|-5|single family| +43376|AAAAAAAAAHJKAAAA|452|14th Railroad|Way|Suite 460|Brookwood|Warren County|MS|50965|United States|-6|apartment| +43377|AAAAAAAABHJKAAAA|971|North View|Parkway|Suite 10|Shiloh|Jefferson County|GA|39275|United States|-5|single family| +43378|AAAAAAAACHJKAAAA|691|South |RD|Suite O|Woodland|Upson County|GA|34854|United States|-5|condo| +43379|AAAAAAAADHJKAAAA|911|Pine Railroad|ST|Suite 400|Oakland|Southampton County|VA|29843|United States|-5|apartment| +43380|AAAAAAAAEHJKAAAA|534|Elevnth Park|Circle|Suite K|Arlington|Winston County|MS|56557|United States|-6|condo| +43381|AAAAAAAAFHJKAAAA|268|4th Johnson|Road|Suite 240|Hopewell|Carter County|TN|30587|United States|-5|condo| +43382|AAAAAAAAGHJKAAAA|894|12th Washington|Lane|Suite 230|New Hope|Pike County|MO|69431|United States|-6|apartment| +43383|AAAAAAAAHHJKAAAA|76||Cir.|||Manassas city||20191|||apartment| +43384|AAAAAAAAIHJKAAAA|881|Maple |Cir.|Suite 110|Wilson|Wyoming County|NY|16971|United States|-5|condo| +43385|AAAAAAAAJHJKAAAA|173|Fifth |Drive|Suite U|Union Hill|Franklin County|PA|17746|United States|-5|condo| +43386|AAAAAAAAKHJKAAAA|782|Oak River|Street|Suite S|Waterloo|Barry County|MO|61675|United States|-6|apartment| +43387|AAAAAAAALHJKAAAA|787|Main West|Dr.|Suite U|Marion|Monroe County|PA|10399|United States|-5|single family| +43388|AAAAAAAAMHJKAAAA|132|East |Ct.|Suite 70|Lakeview|Yates County|NY|18579|United States|-5|single family| +43389|AAAAAAAANHJKAAAA|317|Hickory |Lane|Suite 50|Highland|Schuyler County|IL|69454|United States|-6|single family| +43390|AAAAAAAAOHJKAAAA|792|Lake |Wy|Suite N|Red Hill|Plymouth County|IA|54338|United States|-6|condo| +43391|AAAAAAAAPHJKAAAA|290|Meadow 3rd|Ave|Suite 250|Hopewell|Chippewa County|WI|50587|United States|-6|apartment| +43392|AAAAAAAAAIJKAAAA|895|Ash |Way|Suite 290|Woodland|Humphreys County|TN|34854|United States|-5|condo| +43393|AAAAAAAABIJKAAAA|832|Davis Pine|Street|Suite 140|Fairfield|Jefferson County|OR|96192|United States|-8|condo| +43394|AAAAAAAACIJKAAAA|770|Williams 4th|RD|Suite V|Flint|Holt County|MO|68909|United States|-6|condo| +43395|AAAAAAAADIJKAAAA|305|View Second|Road|Suite V|Midway|Howard County|TX|71904|United States|-6|single family| +43396|AAAAAAAAEIJKAAAA|811|Jefferson |Ave|Suite T|Waterloo|Charlevoix County|MI|41675|United States|-5|single family| +43397|AAAAAAAAFIJKAAAA|359|Valley |Ave|Suite 200|Red Hill|Kenton County|KY|44338|United States|-5|single family| +43398|AAAAAAAAGIJKAAAA|862|View |Court|Suite 380|Franklin|Burleson County|TX|79101|United States|-6|single family| +43399|AAAAAAAAHIJKAAAA|481|6th |Court|Suite 320|Kingston|Yamhill County|OR|94975|United States|-8|single family| +43400|AAAAAAAAIIJKAAAA|492|Franklin Park|RD|Suite Q|Forest Hills|Manistee County|MI|49237|United States|-5|apartment| +43401|AAAAAAAAJIJKAAAA|991|6th |Drive|Suite 80|Phoenix|Williams County|OH|42276|United States|-5|single family| +43402|AAAAAAAAKIJKAAAA|740|Lake |Wy|Suite D|Georgetown|Wilson County|TN|37057|United States|-5|apartment| +43403|AAAAAAAALIJKAAAA|378|Cedar Highland|Lane|Suite N|Cedar|Lander County|NV|81229|United States|-8|apartment| +43404|AAAAAAAAMIJKAAAA|780|Lake Second|Road|Suite 470|Walnut Grove|Lewis County|MO|67752|United States|-6|condo| +43405|AAAAAAAANIJKAAAA|834|Meadow Smith|ST|Suite 230|Highland|Yakutat Borough|AK|99454|United States|-9|apartment| +43406|AAAAAAAAOIJKAAAA|251|8th 9th|RD|Suite V|Antioch|Ottawa County|KS|68605|United States|-6|single family| +43407|AAAAAAAAPIJKAAAA|576|Laurel |ST|Suite 110|Waterloo|Grant County|WI|51675|United States|-6|single family| +43408|AAAAAAAAAJJKAAAA|464|Cherry |Wy|Suite 140|Pierce|Jackson County|MS|53360|United States|-6|single family| +43409|AAAAAAAABJJKAAAA|807|Oak Forest|Pkwy|Suite 270|Cedar Grove|Albemarle County|VA|20411|United States|-5|apartment| +43410|AAAAAAAACJJKAAAA|603|14th Walnut|Lane|Suite 490|Greenville|Storey County|NV|81387|United States|-8|single family| +43411|AAAAAAAADJJKAAAA|383|Ridge |Court|Suite 480|Amherst|Minidoka County|ID|88119|United States|-7|single family| +43412|AAAAAAAAEJJKAAAA|759|Forest |Blvd|Suite 210|Maple Grove|Crisp County|GA|38252|United States|-5|apartment| +43413|AAAAAAAAFJJKAAAA|160|Center |Way|Suite 120|Fairview|Tensas Parish|LA|75709|United States|-6|single family| +43414|AAAAAAAAGJJKAAAA|754|Lake |Ln|Suite 60|Georgetown|Butte County|SD|57057|United States|-6|apartment| +43415|AAAAAAAAHJJKAAAA|914|Elm Lincoln|Parkway|Suite H|Jamestown|Grant County|NE|66867|United States|-6|condo| +43416|AAAAAAAAIJJKAAAA|734||Ave|Suite 420|||||||| +43417|AAAAAAAAJJJKAAAA|743|Willow |RD|||Prince George County|VA||United States|-5|condo| +43418|AAAAAAAAKJJKAAAA|145|Cedar |Parkway|Suite C|Providence|Washington County|WI|56614|United States|-6|single family| +43419|AAAAAAAALJJKAAAA|329|Ridge |Street|Suite 390|Mount Olive|Sagadahoc County|ME|08659|United States|-5|apartment| +43420|AAAAAAAAMJJKAAAA|607|Willow |Court|Suite 490|Bethel|Clinton County|MO|65281|United States|-6|apartment| +43421|AAAAAAAANJJKAAAA|214|12th Highland|Circle|Suite 120|Enterprise|Wyoming County|NY|11757|United States|-5|single family| +43422|AAAAAAAAOJJKAAAA|57|Smith |Ave|Suite 450|Pleasant Hill|Montgomery County|KY|43604|United States|-5|single family| +43423|AAAAAAAAPJJKAAAA|691|4th Sixth|Lane|Suite A|Forestville|Howard County|TX|73027|United States|-6|single family| +43424|AAAAAAAAAKJKAAAA|90|Meadow |Court|Suite U|Shady Grove|Phelps County|NE|62812|United States|-7|apartment| +43425|AAAAAAAABKJKAAAA||East |Pkwy||Newport|Jackson County||31521|||| +43426|AAAAAAAACKJKAAAA|861|Hickory Spring|Parkway|Suite W|Crossroads|Hudson County|NJ|01134|United States|-5|single family| +43427|AAAAAAAADKJKAAAA|439|Lake 9th|Way|Suite 60|Five Points|Monroe County|MI|46098|United States|-5|apartment| +43428|AAAAAAAAEKJKAAAA|588|12th |Circle|Suite 490|Glendale|White Pine County|NV|83951|United States|-8|single family| +43429|AAAAAAAAFKJKAAAA|593|Lakeview Pine|Lane|Suite 220|Pleasant Valley|Pueblo County|CO|82477|United States|-7|apartment| +43430|AAAAAAAAGKJKAAAA|28|Franklin |Ct.|Suite 390|Jackson|Vermilion County|IL|69583|United States|-6|single family| +43431|AAAAAAAAHKJKAAAA|596|Maple |Wy|Suite 360|Church Hill|Bracken County|KY|43790|United States|-6|single family| +43432|AAAAAAAAIKJKAAAA|556|Williams Main|RD|Suite 0|Highland Park|Caldwell County|MO|66534|United States|-6|single family| +43433|AAAAAAAAJKJKAAAA|283|Sunset 9th|Way|Suite 110|Belmont|Big Horn County|WY|80191|United States|-7|apartment| +43434|AAAAAAAAKKJKAAAA|184|Madison |Way|Suite K|Pine Grove|Dillon County|SC|24593|United States|-5|condo| +43435|AAAAAAAALKJKAAAA|903|Woodland Sunset|RD|Suite C|Antioch|Gates County|NC|28605|United States|-5|apartment| +43436|AAAAAAAAMKJKAAAA|639|9th Hill|Street|Suite N|Forest Hills|Garrett County|MD|29237|United States|-5|apartment| +43437|AAAAAAAANKJKAAAA|34|Ridge Sunset|Cir.|Suite N|Newtown|Valley County|MT|61749|United States|-7|condo| +43438|AAAAAAAAOKJKAAAA|406|8th |Avenue|Suite Q|Walnut Grove|Jackson County|CO|87752|United States|-7|single family| +43439|AAAAAAAAPKJKAAAA|345|Willow |Avenue|Suite 210|Unionville|Webster Parish|LA|71711|United States|-6|single family| +43440|AAAAAAAAALJKAAAA|186|Miller Davis|Cir.|Suite 170|Buckingham|Douglas County|GA|34092|United States|-5|condo| +43441|AAAAAAAABLJKAAAA|364|Spring Smith|Blvd|Suite 50|Woodlawn|Lamar County|MS|54098|United States|-6|condo| +43442|AAAAAAAACLJKAAAA|251|East Johnson|Court|Suite N|Sulphur Springs|Gregg County|TX|78354|United States|-6|condo| +43443|AAAAAAAADLJKAAAA|439|4th |Boulevard|Suite A|Riverside|Clark County|AR|79231|United States|-6|condo| +43444|AAAAAAAAELJKAAAA|949|5th |Ln|Suite 270|Red Hill|Gallia County|OH|44338|United States|-5|apartment| +43445|AAAAAAAAFLJKAAAA|977|5th |Wy|Suite R|Hamilton|Shelby County|IL|62808|United States|-6|single family| +43446|AAAAAAAAGLJKAAAA|761|12th Franklin|Ave|Suite G|Jackson|North Slope Borough|AK|99583|United States|-9|single family| +43447|AAAAAAAAHLJKAAAA|594|3rd 7th|Street|Suite 480|Clinton|Carbon County|PA|18222|United States|-5|apartment| +43448|AAAAAAAAILJKAAAA|379|Ridge Walnut|Dr.|Suite Y|Belmont|Phillips County|CO|80191|United States|-7|single family| +43449|AAAAAAAAJLJKAAAA|934|Park Madison|Pkwy|Suite 260|Red Hill|Tyler County|TX|74338|United States|-6|apartment| +43450|AAAAAAAAKLJKAAAA|16|Hillcrest Dogwood|Drive|Suite F|Lakeview|Pershing County|NV|88579|United States|-8|condo| +43451|AAAAAAAALLJKAAAA|716|Center Jackson|Pkwy|Suite 290|Centerville|Hamilton County|TX|70059|United States|-6|condo| +43452|AAAAAAAAMLJKAAAA|984|Main |Ave|Suite 90|Sunnyside|Prairie County|MT|61952|United States|-7|condo| +43453|AAAAAAAANLJKAAAA|810|Church |Ave|Suite S|Springfield|Knott County|KY|49303|United States|-5|condo| +43454|AAAAAAAAOLJKAAAA|639|4th |Way|Suite K|Buena Vista|Butler County|KY|45752|United States|-6|apartment| +43455|AAAAAAAAPLJKAAAA|784|2nd Valley|Dr.|Suite 330|Unionville|Starke County|IN|41711|United States|-5|single family| +43456|AAAAAAAAAMJKAAAA|893|Hill River|Wy|Suite I|Highland Park|Dunn County|ND|56534|United States|-6|condo| +43457|AAAAAAAABMJKAAAA|433|North |Lane|Suite 140|Rockwood|DeKalb County|TN|31545|United States|-5|single family| +43458|AAAAAAAACMJKAAAA|507|Oak 1st|Blvd|Suite H|Oakdale|Renville County|MN|59584|United States|-6|condo| +43459|AAAAAAAADMJKAAAA|95|Green |Way|Suite W|Jamestown|Grayson County|TX|76867|United States|-6|single family| +43460|AAAAAAAAEMJKAAAA|367|Poplar |Way|Suite K|Centerville|Platte County|MO|60059|United States|-6|single family| +43461|AAAAAAAAFMJKAAAA|674|Third Cedar|Wy|Suite 0|Union Hill|Graham County|AZ|87746|United States|-7|condo| +43462|AAAAAAAAGMJKAAAA|938|Walnut |Ave|Suite 10|Harvey|Wayne County|GA|35858|United States|-5|condo| +43463|AAAAAAAAHMJKAAAA|135|Third Oak|Dr.|Suite 280|Pleasant Valley|Polk County|WI|52477|United States|-6|single family| +43464|AAAAAAAAIMJKAAAA|687|Maple Dogwood|Ln|Suite 60|Unionville|Uintah County|UT|81711|United States|-7|single family| +43465|AAAAAAAAJMJKAAAA|609|Oak Ridge|Dr.|Suite 50|Spring Valley|Androscoggin County|ME|06660|United States|-5|apartment| +43466|AAAAAAAAKMJKAAAA|488|Chestnut |Pkwy|Suite L|Enterprise|Borden County|TX|71757|United States|-6|apartment| +43467|AAAAAAAALMJKAAAA|57|Hillcrest 10th|Road|Suite X|Allentown|Iowa County|IA|51838|United States|-6|apartment| +43468|AAAAAAAAMMJKAAAA|709|Elm Jefferson|Ct.|Suite 140|Centerville|Box Elder County|UT|80059|United States|-7|condo| +43469|AAAAAAAANMJKAAAA|432|Green |Ave|Suite A|Macedonia|Craig County|OK|71087|United States|-6|single family| +43470|AAAAAAAAOMJKAAAA|400|Main |Road|Suite B|Arlington|Logan County|AR|76557|United States|-6|single family| +43471|AAAAAAAAPMJKAAAA|39|Sycamore Sixth|Boulevard|Suite X|Spring Valley|Webster Parish|LA|76060|United States|-6|apartment| +43472|AAAAAAAAANJKAAAA|911|1st Oak|Way|Suite 440|Deerfield|Emmet County|MI|49840|United States|-5|apartment| +43473|AAAAAAAABNJKAAAA|542|River Wilson|Ave|Suite O|Mount Vernon|Toombs County|GA|38482|United States|-5|apartment| +43474|AAAAAAAACNJKAAAA|471|5th |Wy|Suite A|Fairview|Oktibbeha County|MS|55709|United States|-6|single family| +43475|AAAAAAAADNJKAAAA|364|Green Second|Avenue|Suite 70|Woodland|Crockett County|TX|74854|United States|-6|apartment| +43476|AAAAAAAAENJKAAAA|706|Spruce |Circle|Suite X|Woodland|Keweenaw County|MI|44854|United States|-5|single family| +43477|AAAAAAAAFNJKAAAA||Oak |Circle|||Laclede County|||||| +43478|AAAAAAAAGNJKAAAA|340|11th |Dr.|Suite 70|Stringtown|Lyon County|KS|60162|United States|-6|condo| +43479|AAAAAAAAHNJKAAAA|416|Dogwood Spruce|Wy|Suite P|Woodland|Cleburne County|AL|34854|United States|-6|condo| +43480|AAAAAAAAINJKAAAA|480|15th |Pkwy|Suite R|Wilson|Clay County|IL|66971|United States|-6|apartment| +43481|AAAAAAAAJNJKAAAA|566|Highland |Pkwy|Suite I|Pleasant Grove|Worth County|GA|34136|United States|-5|condo| +43482|AAAAAAAAKNJKAAAA|18|3rd Wilson|Drive|Suite 200|Sulphur Springs|Independence County|AR|78354|United States|-6|apartment| +43483|AAAAAAAALNJKAAAA|636|Cedar |Wy|Suite Q|Unionville|Crawford County|IL|61711|United States|-6|apartment| +43484|AAAAAAAAMNJKAAAA|516|Lee 1st|Blvd|Suite L|Frogtown|Duval County|FL|38784|United States|-5|condo| +43485|AAAAAAAANNJKAAAA|216|Elm Main|Wy|Suite 170|Midway|Eureka County|NV|81904|United States|-8|apartment| +43486|AAAAAAAAONJKAAAA|999|Third Central|Circle|Suite 0|Valley View|Kanawha County|WV|25124|United States|-5|single family| +43487|AAAAAAAAPNJKAAAA|586|Locust |Lane|Suite 230|Montpelier|Blaine County|MT|68930|United States|-7|single family| +43488|AAAAAAAAAOJKAAAA|826|Green |Wy|Suite 470|Georgetown|Fredericksburg city|VA|27057|United States|-5|apartment| +43489|AAAAAAAABOJKAAAA|436|Washington |Wy|Suite 290|Glendale|Barnwell County|SC|23951|United States|-5|apartment| +43490|AAAAAAAACOJKAAAA||||||||||-6|single family| +43491|AAAAAAAADOJKAAAA|40|Wilson |Pkwy|Suite 340|Franklin|Flathead County|MT|69101|United States|-7|single family| +43492|AAAAAAAAEOJKAAAA|697|Pine Spruce|Circle|Suite S|Cedar Grove|Georgetown County|SC|20411|United States|-5|single family| +43493|AAAAAAAAFOJKAAAA|510|College |Parkway|Suite 130|Pleasant Valley|Cheyenne County|KS|62477|United States|-6|apartment| +43494|AAAAAAAAGOJKAAAA|176|Smith |Boulevard|Suite 420|Green Acres|Lexington County|SC|27683|United States|-5|single family| +43495|AAAAAAAAHOJKAAAA|685|Seventh |Boulevard|Suite 250|Edgewood|Pine County|MN|50069|United States|-6|apartment| +43496|AAAAAAAAIOJKAAAA||1st 14th|Wy|Suite 170|Pleasant Hill|||04204|United States||| +43497|AAAAAAAAJOJKAAAA|689|Green Main|ST|Suite S|Brownsville|Bay County|FL|39310|United States|-5|single family| +43498|AAAAAAAAKOJKAAAA|24|Elm 3rd|Court|Suite I|Glenwood|Jasper County|TX|73511|United States|-6|condo| +43499|AAAAAAAALOJKAAAA|283|River Lincoln|Drive|Suite 460|Concord|Augusta County|VA|24107|United States|-5|single family| +43500|AAAAAAAAMOJKAAAA|104|Broadway |Ct.|Suite J|Concord|Yakutat Borough|AK|94107|United States|-9|condo| +43501|AAAAAAAANOJKAAAA|679|6th Lincoln|Way|Suite 410|Franklin|Wayne County|NE|69101|United States|-7|condo| +43502|AAAAAAAAOOJKAAAA|495|Adams Meadow|Pkwy|Suite R|Hillcrest|Lawrence County|AR|73003|United States|-6|condo| +43503|AAAAAAAAPOJKAAAA|525|Sixth Poplar|Ave|Suite J|Macedonia|Middlesex County|CT|01687|United States|-5|condo| +43504|AAAAAAAAAPJKAAAA|892|Church |Cir.|Suite 490|Jamestown|Phelps County|NE|66867|United States|-7|condo| +43505|AAAAAAAABPJKAAAA|7|College 12th|Drive|Suite 420|Bunker Hill|Kay County|OK|70150|United States|-6|single family| +43506|AAAAAAAACPJKAAAA|657|Washington |Wy|Suite H|Youngstown|Clay County|SD|50001|United States|-6|apartment| +43507|AAAAAAAADPJKAAAA|727|6th Fifth|Ave|Suite 40|Oakdale|Gwinnett County|GA|39584|United States|-5|condo| +43508|AAAAAAAAEPJKAAAA|140|Lincoln |Court|Suite R|Lakewood|Idaho County|ID|88877|United States|-7|condo| +43509|AAAAAAAAFPJKAAAA|623|Main |Ln|Suite H|Sullivan|Lynn County|TX|70451|United States|-6|condo| +43510|AAAAAAAAGPJKAAAA|489|Locust Franklin|Boulevard|Suite S|Brownsville|Lancaster County|NE|69310|United States|-7|condo| +43511|AAAAAAAAHPJKAAAA|57|Hickory Ninth|Ln|Suite V|Providence|Irion County|TX|76614|United States|-6|condo| +43512|AAAAAAAAIPJKAAAA|664|1st First|Wy|Suite 130|Greenwood|Benton County|IN|48828|United States|-5|condo| +43513|AAAAAAAAJPJKAAAA|973|1st |Parkway|Suite 280|Mount Zion|Miami County|OH|48054|United States|-5|apartment| +43514|AAAAAAAAKPJKAAAA|344|2nd |Drive|Suite 270|Georgetown|Houghton County|MI|47057|United States|-5|condo| +43515|AAAAAAAALPJKAAAA|438|Twelfth |Cir.|Suite V|Bunker Hill|Carbon County|WY|80150|United States|-7|condo| +43516|AAAAAAAAMPJKAAAA|945|Cedar 2nd|Cir.|Suite 310|Allison|Greene County|PA|14167|United States|-5|single family| +43517|AAAAAAAANPJKAAAA|962|Lake |Parkway|Suite N|Edgewood|Parker County|TX|70069|United States|-6|apartment| +43518|AAAAAAAAOPJKAAAA|631|Fifth |Ln|Suite W|Wyoming|Shenandoah County|VA|20216|United States|-5|condo| +43519|AAAAAAAAPPJKAAAA|907|Meadow Elm|Circle|Suite 40|Concord|Platte County|NE|64107|United States|-7|apartment| +43520|AAAAAAAAAAKKAAAA|568|Johnson Park|Lane|Suite S|Chester|Sutter County|CA|96088|United States|-8|apartment| +43521|AAAAAAAABAKKAAAA|135|1st |Street|Suite 110|Sutton|Trinity County|TX|75413|United States|-6|condo| +43522|AAAAAAAACAKKAAAA|308|Highland Chestnut|Court|Suite Y|Elizabeth|Hinds County|MS|52935|United States|-6|single family| +43523|AAAAAAAADAKKAAAA|387|Forest 10th|Blvd|Suite 330|Ashland|Iron County|MO|64244|United States|-6|apartment| +43524|AAAAAAAAEAKKAAAA|704|Highland Main|Ln|Suite B|Five Forks|Dickenson County|VA|22293|United States|-5|apartment| +43525|AAAAAAAAFAKKAAAA|904|Fourth |Parkway|Suite 140|Woodlawn|Callahan County|TX|74098|United States|-6|apartment| +43526|AAAAAAAAGAKKAAAA|233|Church |Wy|Suite 270|Fairfield|Crow Wing County|MN|56192|United States|-6|single family| +43527|AAAAAAAAHAKKAAAA|908|North 2nd|Pkwy|Suite L|Sulphur Springs|Huntingdon County|PA|18354|United States|-5|single family| +43528|AAAAAAAAIAKKAAAA|620|Central Oak|Dr.|Suite S|Bridgeport|Lagrange County|IN|45817|United States|-5|apartment| +43529|AAAAAAAAJAKKAAAA|952|Franklin |ST|Suite 180|Enterprise|Grant County|SD|51757|United States|-6|apartment| +43530|AAAAAAAAKAKKAAAA|820|Smith South|Circle|Suite X|Providence|Craig County|OK|76614|United States|-6|apartment| +43531|AAAAAAAALAKKAAAA|852|Washington Hill|Boulevard|Suite M|Lebanon|Sevier County|UT|82898|United States|-7|apartment| +43532|AAAAAAAAMAKKAAAA|12|Hill Hillcrest|Wy|Suite 160|Pine Grove|Putnam County|MO|64593|United States|-6|single family| +43533|AAAAAAAANAKKAAAA|218|||Suite Q|Mount Vernon|Daniels County||||-7|apartment| +43534|AAAAAAAAOAKKAAAA|757|Meadow |ST|Suite 370|Buena Vista|Beaver County|UT|85752|United States|-7|single family| +43535|AAAAAAAAPAKKAAAA|677|3rd Oak|Way|Suite O|New Hope|Loup County|NE|69431|United States|-7|single family| +43536|AAAAAAAAABKKAAAA|284|Cherry |Dr.|Suite 360|Bridgeport|Lee County|GA|35817|United States|-5|apartment| +43537|AAAAAAAABBKKAAAA|956|Second |Avenue|Suite 450|Riverdale|Morrill County|NE|69391|United States|-7|single family| +43538|AAAAAAAACBKKAAAA|514|Wilson |Dr.|Suite 230|Springdale|Holmes County|MS|58883|United States|-6|condo| +43539|AAAAAAAADBKKAAAA|35|Maple |Circle|Suite 170|Lucas|Greene County|MS|54554|United States|-6|apartment| +43540|AAAAAAAAEBKKAAAA|802|Oak Ash|Avenue|Suite 100|White Oak|Saratoga County|NY|16668|United States|-5|single family| +43541|AAAAAAAAFBKKAAAA|875|Elm |Pkwy|Suite 60|Enterprise|Washington County|OR|91757|United States|-8|condo| +43542|AAAAAAAAGBKKAAAA|189|4th Railroad|Boulevard|Suite O|Maywood|Polk County|OR|95681|United States|-8|condo| +43543|AAAAAAAAHBKKAAAA|917|Spring Chestnut|Drive|Suite 90|Franklin|Beauregard Parish|LA|79101|United States|-6|single family| +43544|AAAAAAAAIBKKAAAA||Adams Center|Drive|Suite Q|Spring Hill||NE||||condo| +43545|AAAAAAAAJBKKAAAA|148|Chestnut |Lane|Suite I|Farmington|Throckmorton County|TX|79145|United States|-6|apartment| +43546|AAAAAAAAKBKKAAAA|583|Lake Maple|Lane|Suite 360|Spring Valley|Calhoun County|GA|36060|United States|-5|apartment| +43547|AAAAAAAALBKKAAAA|183|Railroad |Boulevard|Suite S|Liberty|Schenectady County|NY|13451|United States|-5|single family| +43548|AAAAAAAAMBKKAAAA|424||Dr.|Suite A|||||United States||apartment| +43549|AAAAAAAANBKKAAAA|143|Lincoln 15th|Boulevard|Suite T|Highland Park|Elk County|PA|16534|United States|-5|apartment| +43550|AAAAAAAAOBKKAAAA|803|Hill Cherry|Dr.|Suite T|Lucas|Broadwater County|MT|64554|United States|-7|condo| +43551|AAAAAAAAPBKKAAAA|311|15th North|Parkway|Suite 40|Oak Grove|Hertford County|NC|28370|United States|-5|single family| +43552|AAAAAAAAACKKAAAA|289|Fifth Oak|Street|Suite I|Mount Vernon|Adair County|IA|58482|United States|-6|condo| +43553|AAAAAAAABCKKAAAA|732|Sixth |Drive|Suite J|Hamilton|Chautauqua County|NY|12808|United States|-5|apartment| +43554|AAAAAAAACCKKAAAA|485|Elm |Dr.|Suite Q|Sunnyside|Ogemaw County|MI|41952|United States|-5|condo| +43555|AAAAAAAADCKKAAAA|741|Church |Ln|Suite J|Saint James|Bronx County|NY|15799|United States|-5|apartment| +43556|AAAAAAAAECKKAAAA|282|Laurel Church|Ave|Suite 470|Enterprise|Lake County|CO|81757|United States|-7|single family| +43557|AAAAAAAAFCKKAAAA|515|9th Washington|RD|Suite 70|King|Benton County|IN|40008|United States|-5|single family| +43558|AAAAAAAAGCKKAAAA|461|Second |Parkway|Suite T|Forest Hills|Tippah County|MS|59237|United States|-6|apartment| +43559|AAAAAAAAHCKKAAAA|879|Poplar 15th|Ave|Suite 490|Amity|Seward County|NE|60766|United States|-7|condo| +43560|AAAAAAAAICKKAAAA|283|Eigth Willow|Lane|Suite I|Wayland|Taliaferro County|GA|35115|United States|-5|apartment| +43561|AAAAAAAAJCKKAAAA|25|River |Ln|Suite I|Belmont|Willacy County|TX|70191|United States|-6|condo| +43562|AAAAAAAAKCKKAAAA|736|Birch Adams|Street|Suite P|Lakeside|Trumbull County|OH|49532|United States|-5|condo| +43563|AAAAAAAALCKKAAAA|830|Wilson First|Ct.|Suite 160|Friendship|San Augustine County|TX|74536|United States|-6|apartment| +43564|AAAAAAAAMCKKAAAA|941|View 12th|Drive|Suite 150|Summit|Clay County|IN|40499|United States|-5|apartment| +43565|AAAAAAAANCKKAAAA|676|Cherry |Way|Suite L|Oakland|Staunton city|VA|29843|United States|-5|apartment| +43566|AAAAAAAAOCKKAAAA|114|Valley |Dr.|Suite E|Cedar Grove|Linn County|OR|90411|United States|-8|condo| +43567|AAAAAAAAPCKKAAAA|415|Sunset |Blvd||Kelly|Harrison County||||-6|| +43568|AAAAAAAAADKKAAAA|442|Washington Walnut|Drive|Suite 160|Belmont|Carlisle County|KY|40191|United States|-6|apartment| +43569|AAAAAAAABDKKAAAA|654|Cherry Chestnut|Ave|Suite 420|Greenville|Blaine County|OK|71387|United States|-6|single family| +43570|AAAAAAAACDKKAAAA|131|2nd Central|ST|Suite U|Jackson|Hamilton County|IL|69583|United States|-6|condo| +43571|AAAAAAAADDKKAAAA|373|Broadway Valley|||||MS|||-6|| +43572|AAAAAAAAEDKKAAAA|134|Hill |RD|Suite 430|Woodland|Fleming County|KY|44854|United States|-6|apartment| +43573|AAAAAAAAFDKKAAAA|794|2nd Dogwood|Road|Suite W|Pleasant Hill|Noble County|OH|43604|United States|-5|single family| +43574|AAAAAAAAGDKKAAAA|297|10th Poplar|Blvd|Suite 470|Oakland|Ross County|OH|49843|United States|-5|single family| +43575|AAAAAAAAHDKKAAAA|381|Hill Ridge|Circle|Suite K|Fairfield|Emmons County|ND|56192|United States|-6|apartment| +43576|AAAAAAAAIDKKAAAA|632|Willow Ridge|Blvd|Suite 190|Woodville|Buena Vista city|VA|24289|United States|-5|single family| +43577|AAAAAAAAJDKKAAAA|18|10th 11th|Cir.|Suite T|Lakewood|Morgan County|KY|48877|United States|-5|apartment| +43578|AAAAAAAAKDKKAAAA|413|1st |Blvd|Suite 300|Lake Forest|Garden County|NE|66000|United States|-6|apartment| +43579|AAAAAAAALDKKAAAA|435|Hill River|Court|Suite X|Belmont|Franklin County|NY|10191|United States|-5|condo| +43580|AAAAAAAAMDKKAAAA|457|Cedar Meadow|Wy|Suite G|Pleasant Hill|Pacific County|WA|93604|United States|-8|condo| +43581|AAAAAAAANDKKAAAA|253|Locust View|Street|Suite 200|Salem|Clarke County|VA|28048|United States|-5|single family| +43582|AAAAAAAAODKKAAAA|665|11th Laurel|Wy|Suite U|Mountain View|Valley County|MT|64466|United States|-7|condo| +43583|AAAAAAAAPDKKAAAA|472|Church |Street|Suite 450|Colonial Heights|Haralson County|GA|33425|United States|-5|apartment| +43584|AAAAAAAAAEKKAAAA|711|Lake Oak|Wy|Suite 110|Spring Hill|Cleveland County|AR|76787|United States|-6|apartment| +43585|AAAAAAAABEKKAAAA|520|Mill |Court|Suite 340|Plainview|Erie County|PA|13683|United States|-5|apartment| +43586|AAAAAAAACEKKAAAA|791|9th |Ave|Suite K|Mount Vernon|San Francisco County|CA|98482|United States|-8|single family| +43587|AAAAAAAADEKKAAAA|221|Hickory 6th|Avenue|Suite I|Forest Hills|Ottawa County|OK|79237|United States|-6|condo| +43588|AAAAAAAAEEKKAAAA|226|4th |Dr.|Suite 210|Newtown|Macon County|NC|21749|United States|-5|single family| +43589|AAAAAAAAFEKKAAAA|847|Park |Street|Suite 240|Franklin|Meigs County|OH|49101|United States|-5|single family| +43590|AAAAAAAAGEKKAAAA|676|Highland Fifth|Court|Suite 50|Deerfield|Clay County|NC|29840|United States|-5|single family| +43591|AAAAAAAAHEKKAAAA|235|Lee Pine|Drive|Suite P|Antioch|Daviess County|IN|48605|United States|-5|apartment| +43592|AAAAAAAAIEKKAAAA|89|Cedar |Blvd|Suite A|Bridgeport|Del Norte County|CA|95817|United States|-8|single family| +43593|AAAAAAAAJEKKAAAA|671|Green Walnut|Cir.|Suite K|Tyrone|Jefferson County|PA|11201|United States|-5|single family| +43594|AAAAAAAAKEKKAAAA|81|6th Central|Dr.|Suite J|Friendship|Union County|GA|34536|United States|-5|single family| +43595|AAAAAAAALEKKAAAA|604|13th |Dr.|Suite 280|Plainview|Macon County|IL|63683|United States|-6|single family| +43596|AAAAAAAAMEKKAAAA|961|Second 1st|Cir.|Suite 330|Siloam|Cumberland County|TN|38948|United States|-5|single family| +43597|AAAAAAAANEKKAAAA|978|Valley Walnut|Ave|Suite J|Union|Laclede County|MO|68721|United States|-6|single family| +43598|AAAAAAAAOEKKAAAA|338|Davis |Cir.|Suite 380|Spring Hill|Oxford County|ME|07387|United States|-5|condo| +43599|AAAAAAAAPEKKAAAA|80|1st |Court|Suite 350|Highland|Nelson County|VA|29454|United States|-5|single family| +43600|AAAAAAAAAFKKAAAA|500|2nd |Ln|Suite G|Bethel|Choctaw County|AL|35281|United States|-6|condo| +43601|AAAAAAAABFKKAAAA|149|Adams |Blvd|Suite V|Clinton|Wythe County|VA|28222|United States|-5|apartment| +43602|AAAAAAAACFKKAAAA||||||Wayne County|||||| +43603|AAAAAAAADFKKAAAA|962|Park |Ct.|Suite 310|Flatwoods|Jewell County|KS|64212|United States|-6|apartment| +43604|AAAAAAAAEFKKAAAA|457|Ridge |RD|Suite 440|Kingston|Pickaway County|OH|44975|United States|-5|condo| +43605|AAAAAAAAFFKKAAAA|172|Fourth 2nd|Lane|Suite 350|Galena|Lee County|AR|74369|United States|-6|apartment| +43606|AAAAAAAAGFKKAAAA|144|3rd |Blvd|Suite 220|Pine Grove|Columbiana County|OH|44593|United States|-5|apartment| +43607|AAAAAAAAHFKKAAAA|889|Madison 5th|Circle|Suite M|Summit|Sully County|SD|50499|United States|-7|single family| +43608|AAAAAAAAIFKKAAAA|909|Maple 3rd|Wy|Suite Y|Red Oak|Walsh County|ND|55018|United States|-6|single family| +43609|AAAAAAAAJFKKAAAA|499|Church |Lane|Suite K|Maple Grove|Blue Earth County|MN|58252|United States|-6|single family| +43610|AAAAAAAAKFKKAAAA|62|Cedar |Wy|Suite 170|Fox|Renville County|MN|50631|United States|-6|single family| +43611|AAAAAAAALFKKAAAA|56|Valley |Ave|Suite G|Bunker Hill|Pontotoc County|MS|50150|United States|-6|apartment| +43612|AAAAAAAAMFKKAAAA|431|Maple Lincoln|Wy|Suite 280|Plainview|Windsor County|VT|04283|United States|-5|single family| +43613|AAAAAAAANFKKAAAA|456|Miller Walnut|Court|Suite B|Greenwood|McHenry County|IL|68828|United States|-6|condo| +43614|AAAAAAAAOFKKAAAA|55|5th Lake|Boulevard|Suite E|Lincoln|Franklin County|AL|31289|United States|-6|single family| +43615|AAAAAAAAPFKKAAAA|631|Broadway 7th|Ln|Suite 260|Oak Grove|Coles County|IL|68370|United States|-6|single family| +43616|AAAAAAAAAGKKAAAA|242|6th Spring|Street|Suite 320|Providence|Rosebud County|MT|66614|United States|-7|condo| +43617|AAAAAAAABGKKAAAA|397|15th 1st|Court|Suite J|Providence|Le Sueur County|MN|56614|United States|-6|condo| +43618|AAAAAAAACGKKAAAA|757||ST||Highland Park|Sterling County|TX|76534|United States||condo| +43619|AAAAAAAADGKKAAAA|590|Park |Drive|Suite I|Springdale|Lincoln County|ME|09483|United States|-5|single family| +43620|AAAAAAAAEGKKAAAA|744|Sycamore Park|RD|Suite 210|Liberty|Tazewell County|VA|23451|United States|-5|condo| +43621|AAAAAAAAFGKKAAAA|261|14th |Road|Suite E|Bethel|Seminole County|FL|35281|United States|-5|single family| +43622|AAAAAAAAGGKKAAAA|329|12th Locust|Way|Suite H|Mount Olive|Hennepin County|MN|58059|United States|-6|condo| +43623|AAAAAAAAHGKKAAAA|923|Dogwood |Ave|Suite V|Providence|Marshall County|IN|46614|United States|-5|single family| +43624|AAAAAAAAIGKKAAAA|958|Meadow Fourth|Circle|Suite J|Harmony|Martin County|KY|45804|United States|-5|apartment| +43625|AAAAAAAAJGKKAAAA||9th |Ct.||Shiloh||NC|29275|United States|-5|condo| +43626|AAAAAAAAKGKKAAAA|927|Main |ST|Suite L|Cedar Grove|Laurel County|KY|40411|United States|-5|condo| +43627|AAAAAAAALGKKAAAA|587|Elevnth |Lane|Suite 140|Union|Brown County|OH|48721|United States|-5|single family| +43628|AAAAAAAAMGKKAAAA|76|Main |Drive|Suite A|Siloam|Richardson County|NE|68948|United States|-7|apartment| +43629|AAAAAAAANGKKAAAA|974|Adams |Court|Suite H|Summit|Delaware County|IA|50499|United States|-6|apartment| +43630|AAAAAAAAOGKKAAAA|178|Birch |Circle|Suite X|Payne|Venango County|PA|16134|United States|-5|apartment| +43631|AAAAAAAAPGKKAAAA|321|Johnson |Court|Suite 120|Mount Pleasant|Wichita County|TX|71933|United States|-6|apartment| +43632|AAAAAAAAAHKKAAAA|467|Johnson Park|Parkway|Suite K|Jackson|Barrow County|GA|39583|United States|-5|apartment| +43633|AAAAAAAABHKKAAAA|21|Laurel Mill|ST|Suite P|Valley View|Dallas County|AL|35124|United States|-6|single family| +43634|AAAAAAAACHKKAAAA|692|1st 9th|Blvd|Suite A|Fairview|Phillips County|AR|75709|United States|-6|condo| +43635|AAAAAAAADHKKAAAA|375|Maple |Boulevard|Suite 320|Friendship|Chesterfield County|VA|24536|United States|-5|condo| +43636|AAAAAAAAEHKKAAAA|403|Park Washington|Circle|Suite 490|Wildwood|Refugio County|TX|76871|United States|-6|single family| +43637|AAAAAAAAFHKKAAAA|425|Main |Cir.|Suite D|Wildwood|Meriwether County|GA|36871|United States|-5|single family| +43638|AAAAAAAAGHKKAAAA|755|Mill 2nd|Blvd|Suite 150|Highland|Cabell County|WV|29454|United States|-5|condo| +43639|AAAAAAAAHHKKAAAA|987|Main |Pkwy|Suite 490|Dewey|Juab County|UT|81160|United States|-7|single family| +43640|AAAAAAAAIHKKAAAA|862|Madison |Parkway|Suite Q|Woodland|Lackawanna County|PA|14854|United States|-5|single family| +43641|AAAAAAAAJHKKAAAA|46|Lee |Drive|Suite 300|Kingston|Webster County|NE|64975|United States|-7|single family| +43642|AAAAAAAAKHKKAAAA|606|Lincoln Spruce|RD|Suite 350|Wildwood|Bracken County|KY|46871|United States|-6|condo| +43643|AAAAAAAALHKKAAAA|466|2nd Willow|Dr.|Suite 420|Salem|Dallas County|AL|38048|United States|-6|single family| +43644|AAAAAAAAMHKKAAAA||Wilson Second|Lane|Suite A|Plainville|Kay County|OK|76115|||| +43645|AAAAAAAANHKKAAAA|791|Lakeview Broadway|Circle|Suite L|Shady Grove|Day County|SD|52812|United States|-6|apartment| +43646|AAAAAAAAOHKKAAAA|291|Willow Forest|Court|Suite 90|Sulphur Springs|Dodge County|GA|38354|United States|-5|single family| +43647|AAAAAAAAPHKKAAAA|649|3rd |Cir.|Suite 390|Pleasant Hill|Union County|MS|53604|United States|-6|condo| +43648|AAAAAAAAAIKKAAAA|868|Lincoln 9th|Pkwy|Suite 450|Taft|Stevens County|MN|50589|United States|-6|condo| +43649|AAAAAAAABIKKAAAA|874|6th |Cir.|Suite 180|Glenwood|New Haven County|CT|04111|United States|-5|single family| +43650|AAAAAAAACIKKAAAA|32|1st Pine|Street|Suite 300|Arlington|Schley County|GA|36557|United States|-5|apartment| +43651|AAAAAAAADIKKAAAA|478|Lakeview Elm|ST|Suite W|Oakdale|Jefferson County|MS|59584|United States|-6|apartment| +43652|AAAAAAAAEIKKAAAA|559|9th Poplar|Dr.|Suite 90|Concord|Marion County|TN|34107|United States|-6|condo| +43653|AAAAAAAAFIKKAAAA|621|View First|Drive|Suite 460|Midway|Chesapeake city|VA|21904|United States|-5|condo| +43654|AAAAAAAAGIKKAAAA|521|Fourth |Ct.|Suite 30|Pleasant Valley|Chenango County|NY|12477|United States|-5|condo| +43655|AAAAAAAAHIKKAAAA|461|Smith Laurel|Ave|Suite G|Five Points|Hutchinson County|SD|56098|United States|-7|condo| +43656|AAAAAAAAIIKKAAAA|745|Sunset |Wy|Suite 20|Tanglewood|Kent County|MI|48994|United States|-5|apartment| +43657|AAAAAAAAJIKKAAAA||Madison |||||IL||United States|-6|single family| +43658|AAAAAAAAKIKKAAAA|988|Highland Spring|Circle|Suite 60|Centerville|Cleburne County|AR|70059|United States|-6|apartment| +43659|AAAAAAAALIKKAAAA|927|Park 3rd|Ln|Suite K|Green Acres|Clay County|IN|47683|United States|-5|apartment| +43660|AAAAAAAAMIKKAAAA|325|Smith Chestnut|Parkway|Suite 340|Cedar Grove|Washington County|WI|50411|United States|-6|single family| +43661|AAAAAAAANIKKAAAA|288|2nd |Pkwy|Suite F|Glendale|Muskingum County|OH|43951|United States|-5|condo| +43662|AAAAAAAAOIKKAAAA|954|Lake |Pkwy|Suite 50|Burns|Navajo County|AZ|85272|United States|-7|single family| +43663|AAAAAAAAPIKKAAAA|887|Second |Street|Suite 100|Hillcrest|Brown County|TX|73003|United States|-6|apartment| +43664|AAAAAAAAAJKKAAAA|130|Green |Parkway|Suite K|Pleasant Hill|Oneida County|WI|53604|United States|-6|condo| +43665|AAAAAAAABJKKAAAA|834|Meadow 4th|Lane|Suite 250|Woodland|Ringgold County|IA|54854|United States|-6|apartment| +43666|AAAAAAAACJKKAAAA|942|2nd Main|Blvd|Suite 10|King|Weston County|WY|80008|United States|-7|condo| +43667|AAAAAAAADJKKAAAA|898|13th |Ln|Suite F|Macedonia|Wichita County|TX|71087|United States|-6|single family| +43668|AAAAAAAAEJKKAAAA|62|2nd |Way|Suite 470|Mount Vernon|Chatham County|GA|38482|United States|-5|condo| +43669|AAAAAAAAFJKKAAAA|388|Lincoln Ridge|Ave|Suite 430|Centerville|Harnett County|NC|20059|United States|-5|single family| +43670|AAAAAAAAGJKKAAAA|47|Spruce Pine|Pkwy|Suite I|Newport|Polk County|GA|31521|United States|-5|condo| +43671|AAAAAAAAHJKKAAAA|671|Ash 12th|Street|Suite 160|Lakeside|Rusk County|WI|59532|United States|-6|apartment| +43672|AAAAAAAAIJKKAAAA|650|Franklin Oak|Pkwy|Suite 350|Hardy|Tazewell County|VA|25354|United States|-5|single family| +43673|AAAAAAAAJJKKAAAA|477|First |Dr.|Suite O|Wilson|DeSoto County|MS|56971|United States|-6|apartment| +43674|AAAAAAAAKJKKAAAA|165|Seventh |RD|Suite E|Riverside|De Kalb County|IN|49231|United States|-5|condo| +43675|AAAAAAAALJKKAAAA|298|Pine |Road|Suite N|Barnes|Teton County|MT|63788|United States|-7|apartment| +43676|AAAAAAAAMJKKAAAA||North West|Ave|Suite 330||Fulton County||49231|||condo| +43677|AAAAAAAANJKKAAAA||Fourth |||Hopewell|Aroostook County||01187||-5|| +43678|AAAAAAAAOJKKAAAA|700|Lake |Lane|Suite O|Mount Vernon|Ringgold County|IA|58482|United States|-6|apartment| +43679|AAAAAAAAPJKKAAAA|93|View Sunset|Court|Suite S|Highland|Rogers County|OK|||-6|single family| +43680|AAAAAAAAAKKKAAAA|615|Wilson |Ct.|Suite 200|Mount Pleasant|White Pine County|NV|81933|United States|-8|condo| +43681|AAAAAAAABKKKAAAA|47|Sunset |Blvd|Suite H|Brownsville|McDowell County|NC|29310|United States|-5|single family| +43682|AAAAAAAACKKKAAAA||||||||68828|||| +43683|AAAAAAAADKKKAAAA|945|Cedar |Avenue|Suite G|Spring Valley|York County|NE|66060|United States|-6|single family| +43684|AAAAAAAAEKKKAAAA|382|2nd Pine|Dr.|Suite 70|||NC|20419||-5|apartment| +43685|AAAAAAAAFKKKAAAA|359|15th |Street|Suite 430|Fairfield|Glynn County|GA|36192|United States|-5|single family| +43686|AAAAAAAAGKKKAAAA|286|Woodland |RD|Suite S|Hartland|Orange County|VA|26594|United States|-5|condo| +43687|AAAAAAAAHKKKAAAA|165|Birch Cedar|Way||||||United States|-6|condo| +43688|AAAAAAAAIKKKAAAA|960|Adams |Ave|Suite 430|Liberty|Warren County|NY|13451|United States|-5|condo| +43689|AAAAAAAAJKKKAAAA|621|Sunset |Circle|Suite 230|Clifton|Edmonson County|KY|48014|United States|-6|apartment| +43690|AAAAAAAAKKKKAAAA|411|Maple |Circle|Suite X|Ashland|Franklin County|NC|24244|United States|-5|single family| +43691|AAAAAAAALKKKAAAA|900|1st View|Ln|Suite L|Fairview|Wells County|ND|55709|United States|-6|condo| +43692|AAAAAAAAMKKKAAAA|808|Park |Parkway|Suite C|Shiloh|Floyd County|KY|49275|United States|-6|condo| +43693|AAAAAAAANKKKAAAA|231|Central Lakeview|Avenue|Suite 300|Green Acres|Montague County|TX|77683|United States|-6|apartment| +43694|AAAAAAAAOKKKAAAA|995|2nd View|Parkway|Suite O|Red Hill|Harrison County|OH|44338|United States|-5|single family| +43695|AAAAAAAAPKKKAAAA|109|First 4th|Lane|Suite 10|Fairview|Duval County|FL|35709|United States|-5|condo| +43696|AAAAAAAAALKKAAAA|172|Cherry |Boulevard|Suite L|Wilton|Hart County|GA|36997|United States|-5|condo| +43697|AAAAAAAABLKKAAAA|88|Spring |Dr.|Suite 160|Greenfield|Winston County|MS|55038|United States|-6|single family| +43698|AAAAAAAACLKKAAAA|913|Laurel |Road|Suite J|Antioch|Kent County|MI|48605|United States|-5|single family| +43699|AAAAAAAADLKKAAAA|406|Pine Franklin|Ave|Suite S|White Oak|Honolulu County|HI|96668|United States|-10|single family| +43700|AAAAAAAAELKKAAAA|419|Ridge |Circle|Suite 120|Lebanon|Morris County|KS|62898|United States|-6|apartment| +43701|AAAAAAAAFLKKAAAA|987|College Pine|Ave|Suite 90|Greendale|Vanderburgh County|IN|49343|United States|-5|apartment| +43702|AAAAAAAAGLKKAAAA|185|5th |Drive|Suite Q|Springfield|Santa Rosa County|FL|39303|United States|-5|apartment| +43703|AAAAAAAAHLKKAAAA|671|River |Ave|Suite P|Floyd|Bedford city|VA|23235|United States|-5|apartment| +43704|AAAAAAAAILKKAAAA|174|3rd 13th|Dr.|Suite 20|Woodlawn|Cherokee County|GA|34098|United States|-5|condo| +43705|AAAAAAAAJLKKAAAA|664|Lake |Way|Suite O|Gladstone|Coos County|OR|90894|United States|-8|condo| +43706|AAAAAAAAKLKKAAAA|534|Park Ash|ST|Suite 420|Five Forks|Fayette County|IA|52293|United States|-6|apartment| +43707|AAAAAAAALLKKAAAA|698|Oak |Road|Suite 80|Unionville|Sheridan County|ND|51711|United States|-6|condo| +43708|AAAAAAAAMLKKAAAA|198|East South|Road|Suite 390|Macedonia|Robertson County|TX|71087|United States|-6|single family| +43709|AAAAAAAANLKKAAAA|67|First |Lane|Suite B|Friendship|Petersburg city|VA|24536|United States|-5|single family| +43710|AAAAAAAAOLKKAAAA|160|3rd Spruce|RD|Suite L|Clifton|Bethel Census Area|AK|98014|United States|-9|apartment| +43711|AAAAAAAAPLKKAAAA|438|Lincoln |Boulevard|Suite 140|Deerfield|McCurtain County|OK|79840|United States|-6|apartment| +43712|AAAAAAAAAMKKAAAA|758|1st |Court|Suite 120|Oakwood|Dawson County|GA|30169|United States|-5|condo| +43713|AAAAAAAABMKKAAAA||||Suite Y|Summit||OK||United States||| +43714|AAAAAAAACMKKAAAA|639|West 6th|Court|Suite F|Green Acres|Yazoo County|MS|57683|United States|-6|apartment| +43715|AAAAAAAADMKKAAAA|990|Valley |Boulevard|Suite 50|Redland|Bay County|FL|36343|United States|-5|apartment| +43716|AAAAAAAAEMKKAAAA|218|View |Boulevard|Suite H|Brownsville|Cass County|IA|59310|United States|-6|apartment| +43717|AAAAAAAAFMKKAAAA|577|Park |Pkwy|Suite Q|Forest Hills|Fannin County|TX|79237|United States|-6|single family| +43718|AAAAAAAAGMKKAAAA|955|9th Willow|Circle|Suite 100|Glendale|Union County|IN|43951|United States|-5|condo| +43719|AAAAAAAAHMKKAAAA|778|Woodland |RD|Suite 290|Enterprise|Hettinger County|ND|51757|United States|-6|condo| +43720|AAAAAAAAIMKKAAAA|143|11th 15th|Circle|Suite 420|Waterloo|Lamar County|TX|71675|United States|-6|apartment| +43721|AAAAAAAAJMKKAAAA|344|First View|Street|Suite 380|Wildwood|Morris County|TX|76871|United States|-6|single family| +43722|AAAAAAAAKMKKAAAA|750|View South|Avenue|Suite R|Quincy|Stevens County|MN|53868|United States|-6|condo| +43723|AAAAAAAALMKKAAAA|360|2nd Elm|Dr.|Suite 230|Greendale|Toole County|MT|69343|United States|-7|apartment| +43724|AAAAAAAAMMKKAAAA|873|Franklin |ST|Suite 190|Walnut Grove|Bronx County|NY|17752|United States|-5|condo| +43725|AAAAAAAANMKKAAAA|549|Ridge West|Cir.|Suite V|Concord|Fauquier County|VA|24107|United States|-5|single family| +43726|AAAAAAAAOMKKAAAA|839|4th |Court|Suite G|Riverside|Arthur County|NE|69231|United States|-6|single family| +43727|AAAAAAAAPMKKAAAA|800|Miller |Dr.|Suite 350|Jamestown|Chester County|TN|36867|United States|-5|condo| +43728|AAAAAAAAANKKAAAA|286|River Maple|Ct.|Suite N|Evans|Hidalgo County|NM|82284|United States|-7|single family| +43729|AAAAAAAABNKKAAAA|154|Main Spring|Wy|Suite 410|Clinton|Greene County|PA|18222|United States|-5|apartment| +43730|AAAAAAAACNKKAAAA|605|College |Way|Suite 110|Walnut Grove|Taylor County|KY|47752|United States|-5|single family| +43731|AAAAAAAADNKKAAAA|873|Lincoln Chestnut|Boulevard|Suite 240|White Oak|Graham County|KS|66668|United States|-6|condo| +43732|AAAAAAAAENKKAAAA|999|Locust Willow|Parkway|Suite W|Lincoln|Summers County|WV|21289|United States|-5|single family| +43733|AAAAAAAAFNKKAAAA|134|Oak Railroad|Ln|Suite 160|Spring Valley|Winchester city|VA|26060|United States|-5|single family| +43734|AAAAAAAAGNKKAAAA|836|Central 6th|Ln|Suite K|Red Hill|Furnas County|NE|64338|United States|-6|condo| +43735|AAAAAAAAHNKKAAAA|546|Lee |Wy|Suite 220|Riverview|Wayne County|KY|49003|United States|-5|condo| +43736|AAAAAAAAINKKAAAA|88|Birch |Road|Suite 200|White Plains|Faulkner County|AR|76622|United States|-6|apartment| +43737|AAAAAAAAJNKKAAAA|624|Ridge Franklin|ST|Suite W|Jackson|Bollinger County|MO|69583|United States|-6|single family| +43738|AAAAAAAAKNKKAAAA|623|Broadway Seventh|Boulevard|Suite 450|Shady Grove|Yancey County|NC|22812|United States|-5|single family| +43739|AAAAAAAALNKKAAAA|465|11th Fifth|Cir.|Suite 480|Oakwood|Northampton County|PA|10169|United States|-5|apartment| +43740|AAAAAAAAMNKKAAAA|508|North 11th|Drive|Suite 410|Marion|Darke County|OH|40399|United States|-5|condo| +43741|AAAAAAAANNKKAAAA|919|Jackson South|ST|Suite 0|Oakwood|Cleveland County|OK|70169|United States|-6|condo| +43742|AAAAAAAAONKKAAAA|946|Laurel Hill|Cir.|Suite F|Brunswick|Lander County|NV|84642|United States|-8|apartment| +43743|AAAAAAAAPNKKAAAA|339|12th |Parkway|Suite 0|Concord|Cottonwood County|MN|54107|United States|-6|condo| +43744|AAAAAAAAAOKKAAAA|232|2nd |Court|Suite 40|Clinton|Baltimore city|MD|28222|United States|-5|apartment| +43745|AAAAAAAABOKKAAAA|824|Main |Ave|Suite 10|Oak Ridge|Rooks County|KS|68371|United States|-6|condo| +43746|AAAAAAAACOKKAAAA|67|View |Wy|Suite I|Stafford|Ozaukee County|WI|54980|United States|-6|single family| +43747|AAAAAAAADOKKAAAA|595|Broadway Sunset|Avenue|Suite 190|Friendship|Lee County|NC|24536|United States|-5|single family| +43748|AAAAAAAAEOKKAAAA|803|10th |Wy|Suite 130|Friendship|Jasper County|GA|34536|United States|-5|single family| +43749|AAAAAAAAFOKKAAAA|565|Smith |Wy|Suite D|Sutton|McPherson County|SD|55413|United States|-7|apartment| +43750|AAAAAAAAGOKKAAAA|858|2nd |RD|Suite 150|Glenwood|Virginia Beach city|VA|23511|United States|-5|condo| +43751|AAAAAAAAHOKKAAAA|595|View |Dr.|Suite K|Cedar Grove|Wayne County|PA|10411|United States|-5|apartment| +43752|AAAAAAAAIOKKAAAA|840|Fifth Maple|Ave|Suite S|Globe|Platte County|MO|69614|United States|-6|apartment| +43753|AAAAAAAAJOKKAAAA|844|Adams Main|Boulevard|Suite Q|Sulphur Springs|Union County|KY|48354|United States|-5|apartment| +43754|AAAAAAAAKOKKAAAA||||||Gallatin County|MT|||-7|condo| +43755|AAAAAAAALOKKAAAA|552|3rd 13th|Wy|Suite 90|Waterloo|Kitsap County|WA|91675|United States|-8|condo| +43756|AAAAAAAAMOKKAAAA|148|1st |Avenue|Suite A|Proctor|Teton County|WY|88140|United States|-7|condo| +43757|AAAAAAAANOKKAAAA|680|Ridge |Way|Suite D|Liberty|Pickens County|AL|33451|United States|-6|single family| +43758|AAAAAAAAOOKKAAAA|211|First Davis|Blvd|Suite L|Crystal|Elbert County|GA|35258|United States|-5|single family| +43759|AAAAAAAAPOKKAAAA||Birch Valley|Avenue|Suite A|||AR|78222||-6|condo| +43760|AAAAAAAAAPKKAAAA|222|Second |Circle|Suite T|Riverdale|Cumberland County|IL|69391|United States|-6|apartment| +43761|AAAAAAAABPKKAAAA|955|Spring |Cir.|Suite R|Harmony|Wilson County|NC|25804|United States|-5|single family| +43762|AAAAAAAACPKKAAAA|977|Elm |Street|Suite X|Franklin|Grady County|GA|39101|United States|-5|apartment| +43763|AAAAAAAADPKKAAAA|47|Highland Oak|ST|Suite 450|Walnut Grove|Todd County|KY|47752|United States|-5|single family| +43764|AAAAAAAAEPKKAAAA|726|Jackson |Dr.|Suite 270|Shiloh|Swain County|NC|29275|United States|-5|condo| +43765|AAAAAAAAFPKKAAAA|910|First |Blvd|Suite X|Oak Hill|Comanche County|OK|77838|United States|-6|single family| +43766|AAAAAAAAGPKKAAAA|110|Second Lincoln|Street|Suite 380|Ashland|Forest County|PA|14244|United States|-5|condo| +43767|AAAAAAAAHPKKAAAA|222|Main Main|Wy|Suite R|Pleasant Valley|Montague County|TX|72477|United States|-6|single family| +43768|AAAAAAAAIPKKAAAA|583|Broadway Washington|Drive|Suite G|Georgetown|Butler County|NE|67057|United States|-6|condo| +43769|AAAAAAAAJPKKAAAA|8|Jackson Oak|RD|Suite J|Oak Grove|Refugio County|TX|78370|United States|-6|condo| +43770|AAAAAAAAKPKKAAAA|765|Church |Lane|Suite W|Lakewood|Lane County|KS|68877|United States|-6|condo| +43771|AAAAAAAALPKKAAAA|718|10th |Court|Suite 360|Five Forks|Ocean County|NJ|02893|United States|-5|condo| +43772|AAAAAAAAMPKKAAAA|285|Birch |Avenue|Suite Q|Mountain View|Chouteau County|MT|64466|United States|-7|condo| +43773|AAAAAAAANPKKAAAA|359|Johnson |Court|Suite 130|Lakewood|Chelan County|WA|98877|United States|-8|condo| +43774|AAAAAAAAOPKKAAAA|649|Second South|Ave|Suite 400|Marion|Decatur County|TN|30399|United States|-5|condo| +43775|AAAAAAAAPPKKAAAA|828|Davis |Circle|Suite I|Ashland|Coshocton County|OH|44244|United States|-5|condo| +43776|AAAAAAAAAALKAAAA|125|Cedar |Ln|Suite X|Walnut Grove|Aleutians East Borough|AK|97752|United States|-9|apartment| +43777|AAAAAAAABALKAAAA|924|Washington |Ct.|Suite W|Saratoga|Yamhill County|OR|92123|United States|-8|single family| +43778|AAAAAAAACALKAAAA|353|Maple South|Court|Suite Y|Edgewood|Richmond County|NY|10069|United States|-5|single family| +43779|AAAAAAAADALKAAAA|125|River Park|Court|Suite 390|Hopewell|Sullivan County|TN|30587|United States|-6|condo| +43780|AAAAAAAAEALKAAAA|658|12th Ash|Dr.|Suite 320|Glendale|Placer County|CA|93951|United States|-8|condo| +43781|AAAAAAAAFALKAAAA|933|7th |Way|Suite 340|Woodland|Deschutes County|OR|94854|United States|-8|single family| +43782|AAAAAAAAGALKAAAA|534|13th |Wy|Suite 460|Newport|Ellis County|OK|71521|United States|-6|single family| +43783|AAAAAAAAHALKAAAA|187|Davis |Way|Suite 420|Friendship|Switzerland County|IN|44536|United States|-5|single family| +43784|AAAAAAAAIALKAAAA|712|5th Spring|Lane|Suite 50|Harmony|Morgan County|UT|85804|United States|-7|condo| +43785|AAAAAAAAJALKAAAA|500|College Miller|Ave|Suite 360|Newtown|Kittitas County|WA|91749|United States|-8|apartment| +43786|AAAAAAAAKALKAAAA|834|Sunset |Pkwy|Suite X|Post Oak|Franklin County|WA|98567|United States|-8|single family| +43787|AAAAAAAALALKAAAA|754|2nd |Ave|Suite 350|Centerville|Prince Edward County|VA|20059|United States|-5|apartment| +43788|AAAAAAAAMALKAAAA|324||Wy|Suite 170||Davis County|UT|||-7|condo| +43789|AAAAAAAANALKAAAA|581|13th Elm|Pkwy|Suite 300|Belmont|Rockwall County|TX|70191|United States|-6|condo| +43790|AAAAAAAAOALKAAAA|908|North North|Boulevard|Suite X|Walnut Grove|Logan County|OK|77752|United States|-6|apartment| +43791|AAAAAAAAPALKAAAA|344|Oak Valley|Blvd|Suite 10|Bunker Hill|Staunton city|VA|20150|United States|-5|apartment| +43792|AAAAAAAAABLKAAAA|728|Spring College|Parkway|Suite V|Fairfield|Scurry County|TX|76192|United States|-6|single family| +43793|AAAAAAAABBLKAAAA|724|Ash Adams|Circle|Suite 190|Clifton|Abbeville County|SC|28014|United States|-5|apartment| +43794|AAAAAAAACBLKAAAA|308|Spruce |RD|Suite B|Five Points|Oldham County|KY|46098|United States|-5|condo| +43795|AAAAAAAADBLKAAAA|895|Ash |Parkway|Suite O|Riverview|Winona County|MN|59003|United States|-6|condo| +43796|AAAAAAAAEBLKAAAA|713|8th 1st|Dr.||Mount Olive||CA||United States||| +43797|AAAAAAAAFBLKAAAA|885|Washington 1st|Pkwy|Suite Q|Oakwood|Wilkes County|NC|20169|United States|-5|single family| +43798|AAAAAAAAGBLKAAAA|914|Forest |RD|Suite 290|Newport|Mono County|CA|91521|United States|-8|apartment| +43799|AAAAAAAAHBLKAAAA|345|Elm |Ave|Suite 10|Liberty|Columbus County|NC|23451|United States|-5|apartment| +43800|AAAAAAAAIBLKAAAA|60|8th East|Boulevard|Suite B|Hamilton|Snohomish County|WA|92808|United States|-8|condo| +43801|AAAAAAAAJBLKAAAA|665|7th |Lane|Suite B|Centerville|Huntingdon County|PA|10059|United States|-5|apartment| +43802|AAAAAAAAKBLKAAAA|548|6th 11th|Parkway|Suite 190|Arthur|Florence County|WI|55965|United States|-6|single family| +43803|AAAAAAAALBLKAAAA|611|Mill |Ave|Suite X|Waterloo|Logan County|NE|61675|United States|-7|single family| +43804|AAAAAAAAMBLKAAAA|893|Hillcrest Main|Parkway|Suite F|Tremont|Rockland County|NY|19515|United States|-5|single family| +43805|AAAAAAAANBLKAAAA|216|Woodland Jackson|Drive|Suite J|Centerville|Macoupin County|IL|60059|United States|-6|condo| +43806|AAAAAAAAOBLKAAAA|999|4th |Court|Suite 380|Spring Valley|Pickens County|SC|26060|United States|-5|condo| +43807|AAAAAAAAPBLKAAAA|132|Central Ridge|Drive|Suite 470|Macedonia|Freeborn County|MN|51087|United States|-6|single family| +43808|AAAAAAAAACLKAAAA|874|Lake Washington|Dr.|Suite 230|Riverside|Pasco County|FL|39231|United States|-5|single family| +43809|AAAAAAAABCLKAAAA|||Drive|||Roanoke County||25817||-5|| +43810|AAAAAAAACCLKAAAA|925|10th |Cir.|Suite 290|Pierce|Marquette County|MI|43360|United States|-5|single family| +43811|AAAAAAAADCLKAAAA|69|10th |Court|Suite 110|Riverdale|Ada County|ID|89391|United States|-7|condo| +43812|AAAAAAAAECLKAAAA|149|Dogwood Sunset|RD|Suite F|Richmond|Salem County|NJ|06575|United States|-5|apartment| +43813|AAAAAAAAFCLKAAAA|539|Broadway |Court|Suite M|Wildwood|Jefferson County|AL|36871|United States|-6|single family| +43814|AAAAAAAAGCLKAAAA|710|Elm |Road|Suite G|Spring Valley|Preble County|OH|46060|United States|-5|apartment| +43815|AAAAAAAAHCLKAAAA|202|Poplar |Circle|Suite C|Midway|Lee County|IL|61904|United States|-6|condo| +43816|AAAAAAAAICLKAAAA|614|Railroad |Road|Suite 200|Springfield|Sevier County|AR|79303|United States|-6|apartment| +43817|AAAAAAAAJCLKAAAA|780|Ash |Street|Suite Y|Maple Grove|Clark County|MO|68252|United States|-6|single family| +43818|AAAAAAAAKCLKAAAA|476|6th |Court|Suite W|Pleasant Valley|Pocahontas County|WV|22477|United States|-5|single family| +43819|AAAAAAAALCLKAAAA|600|Hill Ridge|Street|Suite M|Spring Hill|Prince William County|VA|26787|United States|-5|condo| +43820|AAAAAAAAMCLKAAAA|610|||Suite 170|||MS||United States||apartment| +43821|AAAAAAAANCLKAAAA|17|2nd |Ln|Suite 390|Glendale|Fresno County|CA|93951|United States|-8|apartment| +43822|AAAAAAAAOCLKAAAA|651|College East|Ave|Suite 100|Oak Grove|Rice County|MN|58370|United States|-6|condo| +43823|AAAAAAAAPCLKAAAA|507|1st |Court|Suite 110|Lebanon|Washington County|MO|62898|United States|-6|apartment| +43824|AAAAAAAAADLKAAAA|77|Church |Road|Suite V|Spring Hill|Guadalupe County|NM|86787|United States|-7|apartment| +43825|AAAAAAAABDLKAAAA|551|Wilson Poplar|Ave|Suite J|Summit|Jefferson County|AR|70499|United States|-6|single family| +43826|AAAAAAAACDLKAAAA|863|Park |Drive|Suite 390|Hamilton|Jasper County|GA|32808|United States|-5|condo| +43827|AAAAAAAADDLKAAAA|930|5th |Road|Suite 400|Enterprise|Dallas County|AR|71757|United States|-6|apartment| +43828|AAAAAAAAEDLKAAAA|481|Oak Laurel|Street|Suite 350|Longwood|Jeff Davis County|TX|77021|United States|-6|apartment| +43829|AAAAAAAAFDLKAAAA|957|Laurel 1st|Parkway|Suite 50|Glenwood|Cullman County|AL|33511|United States|-6|condo| +43830|AAAAAAAAGDLKAAAA|595|First Miller|Ct.|Suite 300|Wilson|Lewis County|ID|86971|United States|-7|single family| +43831|AAAAAAAAHDLKAAAA|582|Hillcrest Maple|Ct.|Suite 240|Glenwood|Real County|TX|73511|United States|-6|condo| +43832|AAAAAAAAIDLKAAAA|9|2nd |Ave|Suite 170|Bridgeport|Pueblo County|CO|85817|United States|-7|condo| +43833|AAAAAAAAJDLKAAAA|734|Spruce Ridge|Ave|Suite H|Stringtown|Potter County|PA|10162|United States|-5|condo| +43834|AAAAAAAAKDLKAAAA|178|River |Parkway|Suite U|Oak Ridge|Stanley County|SD|58371|United States|-7|apartment| +43835|AAAAAAAALDLKAAAA|613|Cedar |ST|Suite S|Mount Pleasant|Morehouse Parish|LA|71933|United States|-6|apartment| +43836|AAAAAAAAMDLKAAAA|513|Park |Way|Suite F|Springdale|Cabell County|WV|28883|United States|-5|apartment| +43837|AAAAAAAANDLKAAAA|367|West |Dr.|Suite 220|Jamestown|Vilas County|WI|56867|United States|-6|apartment| +43838|AAAAAAAAODLKAAAA|670|7th River|Parkway|Suite D|Mount Zion|Martin County|KY|48054|United States|-5|condo| +43839|AAAAAAAAPDLKAAAA|830|Laurel |Cir.|Suite 100|White Oak|DuPage County|IL|66668|United States|-6|single family| +43840|AAAAAAAAAELKAAAA|396|Railroad |Blvd|Suite N|Shaw|Audubon County|IA|50618|United States|-6|condo| +43841|AAAAAAAABELKAAAA|968|River Mill|Ave|Suite 420|Mount Pleasant|Calhoun County|AL|31933|United States|-6|apartment| +43842|AAAAAAAACELKAAAA|772|Johnson Seventh|Drive|Suite 10|Mount Pleasant|Sunflower County|MS|51933|United States|-6|condo| +43843|AAAAAAAADELKAAAA|172|6th |Dr.|Suite M|Concord|Suffolk County|NY|14107|United States|-5|condo| +43844|AAAAAAAAEELKAAAA|330|Pine Franklin|Avenue|Suite 150|Silver Creek|Berrien County|GA|34546|United States|-5|single family| +43845|AAAAAAAAFELKAAAA|957|9th Main|Avenue|Suite 220|Macedonia|Kershaw County|SC|21087|United States|-5|apartment| +43846|AAAAAAAAGELKAAAA|432|Birch |Parkway|Suite 90|Mount Olive|Tattnall County|GA|38059|United States|-5|single family| +43847|AAAAAAAAHELKAAAA|246|Broadway |RD|Suite A|Shiloh|Early County|GA|39275|United States|-5|single family| +43848|AAAAAAAAIELKAAAA|545|Washington |Street|Suite H|Barnes|Gillespie County|TX|73788|United States|-6|apartment| +43849|AAAAAAAAJELKAAAA|421|Chestnut |Dr.|Suite A|Bethel|Hamilton County|KS|65281|United States|-6|condo| +43850|AAAAAAAAKELKAAAA|985|Central |Ln|Suite H|Red Bank|Simpson County|MS|54975|United States|-6|single family| +43851|AAAAAAAALELKAAAA|260|Green |Circle|Suite 300|Belmont|Carroll County|NH|00791|United States|-5|condo| +43852|AAAAAAAAMELKAAAA|946|Forest |Court|Suite K|King|Augusta County|VA|20008|United States|-5|single family| +43853|AAAAAAAANELKAAAA|772|Willow |Ln|Suite 90|Enterprise|Rolette County|ND|51757|United States|-6|condo| +43854|AAAAAAAAOELKAAAA|997|8th 6th|RD|Suite 310|Buena Vista|Decatur County|TN|35752|United States|-5|single family| +43855|AAAAAAAAPELKAAAA|794|Woodland |Avenue|Suite 160|Enterprise|Harrison County|MO|61757|United States|-6|single family| +43856|AAAAAAAAAFLKAAAA|270|4th Madison|Blvd|Suite W|Midway|Galveston County|TX|71904|United States|-6|condo| +43857|AAAAAAAABFLKAAAA|604|Cherry 6th|ST|Suite V|Bethel|Russell County|KS|65281|United States|-6|condo| +43858|AAAAAAAACFLKAAAA|180|1st |RD|Suite 450|Ashland|Gooding County|ID|84244|United States|-7|condo| +43859|AAAAAAAADFLKAAAA|132|Cedar 2nd|Court|Suite 440|Waterloo|Weld County|CO|81675|United States|-7|single family| +43860|AAAAAAAAEFLKAAAA|915|Sycamore Fifth|Lane|Suite 410|Providence|Clay County|WV|26614|United States|-5|apartment| +43861|AAAAAAAAFFLKAAAA|853|Meadow Birch|Wy|Suite A|Mount Olive|Hemphill County|TX|78059|United States|-6|apartment| +43862|AAAAAAAAGFLKAAAA|100|Spring Laurel|Ct.|Suite 300|Jerome|Lackawanna County|PA|19920|United States|-5|condo| +43863|AAAAAAAAHFLKAAAA|833|Jefferson 15th|Drive|Suite N|Lakeside|Ross County|OH|49532|United States|-5|apartment| +43864|AAAAAAAAIFLKAAAA|401|Ridge River|Ct.|Suite 400|Shiloh|York County|NE|69275|United States|-6|apartment| +43865|AAAAAAAAJFLKAAAA|976|Main |Court|Suite 220|Pleasant Hill|Bibb County|GA|33604|United States|-5|apartment| +43866|AAAAAAAAKFLKAAAA|250|First Forest|Ct.|Suite K|Mount Vernon|Davis County|IA|58482|United States|-6|apartment| +43867|AAAAAAAALFLKAAAA|832|North Park|ST|Suite 140|Plainview|Cayuga County|NY|13683|United States|-5|condo| +43868|AAAAAAAAMFLKAAAA|838|Ridge |Circle|Suite 360|Forest Hills|Westchester County|NY|19237|United States|-5|single family| +43869|AAAAAAAANFLKAAAA|484|3rd Broadway|Parkway|Suite 320|Brownsville|Montgomery County|MD|29310|United States|-5|condo| +43870|AAAAAAAAOFLKAAAA|708|Park Railroad|Way|Suite 470|Liberty|Transylvania County|NC|23451|United States|-5|single family| +43871|AAAAAAAAPFLKAAAA|243|Johnson Ridge|RD|Suite 60|Concord|Lehigh County|PA|14107|United States|-5|condo| +43872|AAAAAAAAAGLKAAAA|908|Ridge Park|Blvd|Suite 150|Lakeside|Butler County|AL|39532|United States|-6|condo| +43873|AAAAAAAABGLKAAAA|||Cir.|Suite 400|Harmony|San Juan County|||||condo| +43874|AAAAAAAACGLKAAAA|746|8th Ridge|RD|Suite 250|Highland|Clayton County|IA|59454|United States|-6|condo| +43875|AAAAAAAADGLKAAAA|951|Park Highland|Street|Suite O|Rosebud|Carteret County|NC|22244|United States|-5|apartment| +43876|AAAAAAAAEGLKAAAA|683|Church |ST|Suite 70|Riverview|Crowley County|CO|89003|United States|-7|condo| +43877|AAAAAAAAFGLKAAAA|515|3rd |Pkwy|Suite S|Union|Nassau County|FL|38721|United States|-5|apartment| +43878|AAAAAAAAGGLKAAAA|220|College |Boulevard|Suite V|Lakeside|Clark County|IL|69532|United States|-6|apartment| +43879|AAAAAAAAHGLKAAAA|766|Seventh Dogwood|Parkway|Suite C|Bridgeport|Lincoln County|MN|55817|United States|-6|condo| +43880|AAAAAAAAIGLKAAAA|608|West |Blvd|Suite 300|Union Hill|Madison Parish|LA|77746|United States|-6|condo| +43881|AAAAAAAAJGLKAAAA|476|9th Cedar|Parkway|Suite 400|Oak Grove|Hot Spring County|AR|78370|United States|-6|apartment| +43882|AAAAAAAAKGLKAAAA|994|4th |ST|Suite 20|Georgetown|La Salle County|TX|77057|United States|-6|apartment| +43883|AAAAAAAALGLKAAAA|687|Lincoln |Blvd|Suite 70|Enterprise|Calhoun County|AR|71757|United States|-6|single family| +43884|AAAAAAAAMGLKAAAA|771|Hillcrest Oak|Ave|Suite 290|Walnut Grove|Brooke County|WV|27752|United States|-5|condo| +43885|AAAAAAAANGLKAAAA|434|Adams |Pkwy|Suite 180|Five Forks|Marion County|TN|32293|United States|-6|single family| +43886|AAAAAAAAOGLKAAAA|109|Forest Ash|Cir.|Suite L|Pleasant Grove|Noble County|OK|74136|United States|-6|apartment| +43887|AAAAAAAAPGLKAAAA|733|Cherry |Court|Suite P|Fairfield|Kane County|IL|66192|United States|-6|condo| +43888|AAAAAAAAAHLKAAAA|962|River |Cir.|Suite 120|Marion|Jackson County|MO|60399|United States|-6|apartment| +43889|AAAAAAAABHLKAAAA|626|Birch 2nd|RD|Suite 250|Ashland|Cameron County|PA|14244|United States|-5|single family| +43890|AAAAAAAACHLKAAAA|744|Valley Madison|||Oakland|||99843||-8|apartment| +43891|AAAAAAAADHLKAAAA|115|Main |Street|Suite U|Freeman|Utah County|UT|82297|United States|-7|single family| +43892|AAAAAAAAEHLKAAAA|141|5th 10th|ST|Suite 350|Lakeside|Orangeburg County|SC|29532|United States|-5|condo| +43893|AAAAAAAAFHLKAAAA|794|Washington Williams|Parkway|Suite 270|Edgewood|Big Horn County|MT|60069|United States|-7|single family| +43894|AAAAAAAAGHLKAAAA|789|Jefferson |Blvd|Suite C|Woodland|Dixie County|FL|34854|United States|-5|apartment| +43895|AAAAAAAAHHLKAAAA|877|Spring 12th|Ln|Suite B|Union Hill|Sitka Borough|AK|97746|United States|-9|single family| +43896|AAAAAAAAIHLKAAAA|924|6th Lake|Ct.|Suite M|Hamilton|Presque Isle County|MI|42808|United States|-5|condo| +43897|AAAAAAAAJHLKAAAA|727|2nd 1st||||Frontier County||60150|||| +43898|AAAAAAAAKHLKAAAA|992|Park Spruce|Circle|Suite S|Plainview|Tippecanoe County|IN|43683|United States|-5|condo| +43899|AAAAAAAALHLKAAAA|197|Adams |Avenue|Suite S|Five Forks|Ziebach County|SD|52293|United States|-6|condo| +43900|AAAAAAAAMHLKAAAA|732|Fifth Railroad|Avenue|Suite S|Edgewood|Madison County|IL|60069|United States|-6|single family| +43901|AAAAAAAANHLKAAAA|678|Spring Meadow|Dr.|Suite T|Glendale|Lincoln County|WY|83951|United States|-7|apartment| +43902|AAAAAAAAOHLKAAAA|686|4th Birch|Circle|Suite 160|Lakewood|Fluvanna County|VA|28877|United States|-5|condo| +43903|AAAAAAAAPHLKAAAA|57|Ninth |Lane|Suite 440|Springfield|Pemiscot County|MO|69303|United States|-6|condo| +43904|AAAAAAAAAILKAAAA|69|Seventh Park|Wy|Suite I|Salem|Highlands County|FL|38048|United States|-5|condo| +43905|AAAAAAAABILKAAAA|94|Center Jefferson|Dr.|Suite F|Oakdale|Polk County|MN|59584|United States|-6|apartment| +43906|AAAAAAAACILKAAAA|521|Elm |Parkway|Suite 410|Kingston|Mesa County|CO|84975|United States|-7|single family| +43907|AAAAAAAADILKAAAA|149|Oak |Circle|Suite 450|Lebanon|Lee County|SC|22898|United States|-5|apartment| +43908|AAAAAAAAEILKAAAA|153|1st Ninth|Drive|Suite 70|Pleasant Hill|Iron County|WI|53604|United States|-6|condo| +43909|AAAAAAAAFILKAAAA|521|Church |Drive|Suite H|Oakwood|Madison County|ID|80169|United States|-7|condo| +43910|AAAAAAAAGILKAAAA|65|Second |Wy|Suite M|Ashland|Muskingum County|OH|44244|United States|-5|single family| +43911|AAAAAAAAHILKAAAA|801|First |Court|Suite O|Mount Vernon|Allen County|OH|48482|United States|-5|apartment| +43912|AAAAAAAAIILKAAAA|579|East |Road||Woodville||TX|74289|||condo| +43913|AAAAAAAAJILKAAAA|690|Hillcrest Adams|Avenue|Suite I|Greenwood|Douglas County|NE|68828|United States|-6|single family| +43914|AAAAAAAAKILKAAAA|879|Main Mill|Ln|Suite D|Lakewood|Jasper County|IA|58877|United States|-6|condo| +43915|AAAAAAAALILKAAAA|916|12th Pine|Drive|Suite 350|Mount Olive|Genesee County|MI|48059|United States|-5|condo| +43916|AAAAAAAAMILKAAAA|229|Jackson |Circle|Suite A|Belmont|Northumberland County|PA|10191|United States|-5|apartment| +43917|AAAAAAAANILKAAAA||Broadway |||Buckingham|Issaquena County|MS|54092|||| +43918|AAAAAAAAOILKAAAA|729|Fifth Washington|Way|Suite 180|Perkins|Adams County|ID|81852|United States|-7|apartment| +43919|AAAAAAAAPILKAAAA|273|Sycamore Lee|Boulevard|Suite D|Oak Ridge|Gallatin County|MT|68371|United States|-7|condo| +43920|AAAAAAAAAJLKAAAA|700|View Sunset|Dr.|Suite 350|Empire|Fulton County|AR|74145|United States|-6|single family| +43921|AAAAAAAABJLKAAAA|527|Jackson |ST|Suite 340|Clinton|Jasper County|MO|68222|United States|-6|apartment| +43922|AAAAAAAACJLKAAAA|230|College 2nd|ST|Suite 290|Mountain View|Sequoyah County|OK|74466|United States|-6|apartment| +43923|AAAAAAAADJLKAAAA|420|Hillcrest |Wy|Suite 450|Lakeview|Monroe County|IA|58579|United States|-6|apartment| +43924|AAAAAAAAEJLKAAAA|14|Main |Road|Suite L|Hopewell|Nacogdoches County|TX|70587|United States|-6|apartment| +43925|AAAAAAAAFJLKAAAA|720|West |RD|Suite P|Deerfield|Clark County|NV|89840|United States|-8|apartment| +43926|AAAAAAAAGJLKAAAA|381|2nd |Parkway|Suite H|Maple Grove|Emmet County|IA|58252|United States|-6|apartment| +43927|AAAAAAAAHJLKAAAA|148|Broadway |Circle|Suite 460|Lebanon|Winn Parish|LA|72898|United States|-6|condo| +43928|AAAAAAAAIJLKAAAA|124|Sunset Cedar|ST|Suite 300|Liberty|Lake County|OH|43451|United States|-5|condo| +43929|AAAAAAAAJJLKAAAA|473|Main |Ave|Suite 240|Clifton|Harris County|TX|78014|United States|-6|condo| +43930|AAAAAAAAKJLKAAAA|911|Lee |Dr.|Suite T|Jamestown|Shelby County|IL|66867|United States|-6|single family| +43931|AAAAAAAALJLKAAAA|617|Third |Drive|Suite Y|Mount Zion|Watauga County|NC|28054|United States|-5|apartment| +43932|AAAAAAAAMJLKAAAA|425|Fourteenth Hillcrest|Avenue|Suite 490|Hillcrest|Caldwell Parish|LA|73003|United States|-6|single family| +43933|AAAAAAAANJLKAAAA|953||Cir.||Enterprise|Wayne County|IN|41757|United States||| +43934|AAAAAAAAOJLKAAAA|862|Davis 15th|RD|Suite G|Greenfield|Williams County|OH|45038|United States|-5|condo| +43935|AAAAAAAAPJLKAAAA|898|Locust |Boulevard|Suite B|Hopewell|Hutchinson County|SD|50587|United States|-7|condo| +43936|AAAAAAAAAKLKAAAA|736|Eigth |Blvd|Suite R|Forest Hills|Brown County|IL|69237|United States|-6|condo| +43937|AAAAAAAABKLKAAAA|948|Valley |Street|Suite Y|Woodlawn|Monona County|IA|54098|United States|-6|condo| +43938|AAAAAAAACKLKAAAA|220|Sixth |Dr.|Suite 110|Riceville|Chowan County|NC|25867|United States|-5|single family| +43939|AAAAAAAADKLKAAAA|301||Parkway|||Kankakee County|IL|61087|United States||| +43940|AAAAAAAAEKLKAAAA|730|Hill |Parkway|Suite 340|Arlington|Lincoln County|GA|36557|United States|-5|apartment| +43941|AAAAAAAAFKLKAAAA|15|Main Pine|ST|Suite K|Sulphur Springs|Jackson County|OH|48354|United States|-5|condo| +43942|AAAAAAAAGKLKAAAA|1000|Elm Wilson|Cir.|Suite W|Fairfield|Garrett County|MD|26192|United States|-5|apartment| +43943|AAAAAAAAHKLKAAAA|88|Jackson 6th|Cir.|Suite 280|Caledonia|Harrison County|OH|47411|United States|-5|single family| +43944|AAAAAAAAIKLKAAAA|479|4th |Pkwy|Suite 470|Unionville|Alexander County|NC|21711|United States|-5|apartment| +43945|AAAAAAAAJKLKAAAA|433|View |Pkwy|Suite 110|Providence|McCormick County|SC|26614|United States|-5|single family| +43946|AAAAAAAAKKLKAAAA||Hillcrest |Ave||Red Hill||||United States|-5|| +43947|AAAAAAAALKLKAAAA|449|14th |Blvd|Suite 270|Newtown|Powell County|KY|41749|United States|-5|single family| +43948|AAAAAAAAMKLKAAAA|870|Lincoln Fourteenth|Dr.|Suite J|Midway|Lawrence County|PA|11904|United States|-5|condo| +43949|AAAAAAAANKLKAAAA|734|North Green|Way|Suite 20|White Oak|Hart County|KY|46668|United States|-6|single family| +43950|AAAAAAAAOKLKAAAA|727|Third |Boulevard|Suite 200|Mount Vernon|Putnam County|OH|48482|United States|-5|apartment| +43951|AAAAAAAAPKLKAAAA|843|Lincoln |Blvd|Suite J|Wildwood|Dorchester County|MD|26871|United States|-5|single family| +43952|AAAAAAAAALLKAAAA|916|1st |Way|Suite 290|Maple Grove|Love County|OK|78252|United States|-6|apartment| +43953|AAAAAAAABLLKAAAA|736|Ridge Hill|ST|Suite H|Walnut Grove|Jefferson County|AR|77752|United States|-6|apartment| +43954|AAAAAAAACLLKAAAA|191|College Green|Drive|Suite 160|Stringtown|Wilkes County|NC|20162|United States|-5|condo| +43955|AAAAAAAADLLKAAAA|785|Tenth |Ave|Suite 180|White Oak|Ozaukee County|WI|56668|United States|-6|apartment| +43956|AAAAAAAAELLKAAAA|211|Walnut |Dr.|Suite D|Maple Grove|Cloud County|KS|68252|United States|-6|single family| +43957|AAAAAAAAFLLKAAAA|377|Center |Court|Suite 80|Winslow|Walsh County|ND|58525|United States|-6|single family| +43958|AAAAAAAAGLLKAAAA|243|6th 6th|RD|Suite G|Oakdale|Wright County|IA|59584|United States|-6|single family| +43959|AAAAAAAAHLLKAAAA|573|Ash 13th|Blvd|Suite 120|Enterprise|Hyde County|NC|21757|United States|-5|apartment| +43960|AAAAAAAAILLKAAAA||Walnut Cedar||||Searcy County||||-6|| +43961|AAAAAAAAJLLKAAAA|823|Third Willow|Drive|Suite 290|Red Hill|Labette County|KS|64338|United States|-6|single family| +43962|AAAAAAAAKLLKAAAA|394|2nd Ash|Road|Suite T|Forest Hills|Rockwall County|TX|79237|United States|-6|apartment| +43963|AAAAAAAALLLKAAAA|751|Dogwood College|Street|Suite V|Mount Olive|Aiken County|SC|28059|United States|-5|single family| +43964|AAAAAAAAMLLKAAAA|116|First Sixth|Avenue|Suite 240|Coldwater|Nantucket County|MA|03973|United States|-5|condo| +43965|AAAAAAAANLLKAAAA|529|Ridge |Street|Suite A|Georgetown|Garfield County|CO|87057|United States|-7|condo| +43966|AAAAAAAAOLLKAAAA|116|Sunset North|Circle|Suite D|Mountain View|Henry County|OH|44466|United States|-5|apartment| +43967|AAAAAAAAPLLKAAAA|839|Railroad College|Blvd|Suite E|Maywood|Alamance County|NC|25681|United States|-5|condo| +43968|AAAAAAAAAMLKAAAA|245|Fourth Forest|Ct.|Suite I|Union Hill|Boundary County|ID|87746|United States|-7|single family| +43969|AAAAAAAABMLKAAAA|429|Washington |Pkwy|Suite 270|Sunnyside|Elk County|PA|11952|United States|-5|condo| +43970|AAAAAAAACMLKAAAA|858|Maple |Court|Suite 390|Rockwood|Weston County|WY|81545|United States|-7|single family| +43971|AAAAAAAADMLKAAAA|410|Church |Dr.|Suite K|Oakdale|Irwin County|GA|39584|United States|-5|single family| +43972|AAAAAAAAEMLKAAAA|173|Maple |Circle|Suite 360|Green Acres|Jefferson County|MO|67683|United States|-6|condo| +43973|AAAAAAAAFMLKAAAA|854|Maple Forest|RD|Suite 240|Woodland|Holmes County|MS|54854|United States|-6|condo| +43974|AAAAAAAAGMLKAAAA|241|Eigth |Parkway|Suite M|Georgetown|San Juan County|NM|87057|United States|-7|condo| +43975|AAAAAAAAHMLKAAAA|612|Eigth |Drive|Suite 260|Hopewell|Fulton County|KY|40587|United States|-6|condo| +43976|AAAAAAAAIMLKAAAA|697|2nd Sunset|Ct.|Suite T|Shady Grove|Columbia County|NY|12812|United States|-5|single family| +43977|AAAAAAAAJMLKAAAA|172|1st |Ct.|Suite W|Forest Hills|Loving County|TX|79237|United States|-6|condo| +43978|AAAAAAAAKMLKAAAA|88|View |Ct.|Suite K|Willis|McCreary County|KY|46788|United States|-5|apartment| +43979|AAAAAAAALMLKAAAA|176|Lake |Avenue|Suite 190|Franklin|Boulder County|CO|89101|United States|-7|apartment| +43980|AAAAAAAAMMLKAAAA|711|Hillcrest |Circle|Suite I|Macedonia|Calhoun County|WV|21087|United States|-5|single family| +43981|AAAAAAAANMLKAAAA|388|2nd |Ave|Suite 460|Oakwood|Cherokee County|AL|30169|United States|-6|single family| +43982|AAAAAAAAOMLKAAAA|295|9th Pine|Ln|Suite 280|Springdale|Santa Cruz County|CA|98883|United States|-8|single family| +43983|AAAAAAAAPMLKAAAA|233|6th 5th|Dr.|Suite T|Riverview|Comal County|TX|79003|United States|-6|condo| +43984|AAAAAAAAANLKAAAA|210|Main Park|Parkway|Suite J|Midway|Hardin County|TX|71904|United States|-6|condo| +43985|AAAAAAAABNLKAAAA|42|Hillcrest |Way|Suite 100|Salem|Haywood County|TN|38048|United States|-5|apartment| +43986|AAAAAAAACNLKAAAA|975|North |Pkwy|Suite 130|Lakewood|Tulare County|CA|98877|United States|-8|apartment| +43987|AAAAAAAADNLKAAAA|805|Walnut |Road|Suite 70|Ellsworth|Anson County|NC|25079|United States|-5|apartment| +43988|AAAAAAAAENLKAAAA|753|Main |Lane|Suite V|Florence|Sublette County|WY|83394|United States|-7|condo| +43989|AAAAAAAAFNLKAAAA|374|Birch 4th|Ct.|Suite 160|Summit|Lyman County|SD|50499|United States|-7|apartment| +43990|AAAAAAAAGNLKAAAA|565|Park Smith|Way|Suite 390|Globe|Oxford County|ME|09614|United States|-5|condo| +43991|AAAAAAAAHNLKAAAA|614|Lincoln |RD|Suite 410|Pleasant Valley|Milwaukee County|WI|52477|United States|-6|single family| +43992|AAAAAAAAINLKAAAA|993|Ninth |Drive|Suite 180|Shady Grove|Cherry County|NE|62812|United States|-6|apartment| +43993|AAAAAAAAJNLKAAAA|441|Woodland |Lane|Suite J|Centerville|Anderson County|TX|70059|United States|-6|condo| +43994|AAAAAAAAKNLKAAAA|59|6th Sunset|Avenue|Suite M|Liberty|Lowndes County|AL|33451|United States|-6|condo| +43995|AAAAAAAALNLKAAAA|701|Walnut |Dr.|Suite P|Mount Vernon|Mora County|NM|88482|United States|-7|apartment| +43996|AAAAAAAAMNLKAAAA|164|3rd Jackson|Ln|Suite O|Clifton|Milwaukee County|WI|58014|United States|-6|single family| +43997|AAAAAAAANNLKAAAA|49|Willow |ST|Suite R|Perkins|Spotsylvania County|VA|21852|United States|-5|single family| +43998|AAAAAAAAONLKAAAA|879|Willow |Blvd|Suite 300|Newport|Williamson County|IL|61521|United States|-6|single family| +43999|AAAAAAAAPNLKAAAA|||Court||Union Hill|Atascosa County||77746|United States|-6|| +44000|AAAAAAAAAOLKAAAA|624|12th |Blvd|Suite S|Salem|Becker County|MN|58048|United States|-6|apartment| +44001|AAAAAAAABOLKAAAA|553|Third Church|Circle|Suite 240|Five Points|Dorchester County|MD|26098|United States|-5|single family| +44002|AAAAAAAACOLKAAAA||||||Union County|AR|78567|United States||| +44003|AAAAAAAADOLKAAAA|390|Miller 2nd|Cir.|Suite 290|Belmont|Mora County|NM|80191|United States|-7|condo| +44004|AAAAAAAAEOLKAAAA|181|North Smith|Boulevard|Suite X|Hopewell|Oliver County|ND|50587|United States|-6|condo| +44005|AAAAAAAAFOLKAAAA|294|3rd |Court|Suite M|Mount Olive|Fulton County|OH|48059|United States|-5|apartment| +44006|AAAAAAAAGOLKAAAA|929|4th |Avenue|Suite V|Montezuma|Dade County|GA|32150|United States|-5|apartment| +44007|AAAAAAAAHOLKAAAA|611|Ash |Blvd|Suite U|Kingston|Pima County|AZ|84975|United States|-7|condo| +44008|AAAAAAAAIOLKAAAA|324|8th |ST|Suite 340|Buena Vista|Litchfield County|CT|06352|United States|-5|apartment| +44009|AAAAAAAAJOLKAAAA|822|Oak |ST|Suite 100|Lebanon|Hamblen County|TN|32898|United States|-5|apartment| +44010|AAAAAAAAKOLKAAAA|81|Main Sycamore|Avenue|Suite T|Stringtown|Tulsa County|OK|70162|United States|-6|condo| +44011|AAAAAAAALOLKAAAA|171|View 6th|Road|Suite R|Kingston|Washington County|AL|34975|United States|-6|apartment| +44012|AAAAAAAAMOLKAAAA|995|Walnut |ST|Suite M|Oakdale|Powder River County|MT|69584|United States|-7|condo| +44013|AAAAAAAANOLKAAAA|440|East Lee|Street|Suite 230|Oakdale|San Benito County|CA|99584|United States|-8|condo| +44014|AAAAAAAAOOLKAAAA|212|10th |Ct.|Suite 410|Wilson|Defiance County|OH|46971|United States|-5|single family| +44015|AAAAAAAAPOLKAAAA|56|Highland |Pkwy|Suite 270|Mountain View|Johnson County|TN|34466|United States|-6|condo| +44016|AAAAAAAAAPLKAAAA|28|Hickory Fourth|ST|Suite 460|Shady Grove|Dunn County|ND|52812|United States|-6|condo| +44017|AAAAAAAABPLKAAAA|649|Birch |Way|Suite 340|Edgewood|Young County|TX|70069|United States|-6|single family| +44018|AAAAAAAACPLKAAAA|48|15th Jefferson|RD|Suite R|Hamilton|Todd County|KY|42808|United States|-5|apartment| +44019|AAAAAAAADPLKAAAA|124|Sixth Oak|Court|Suite R|Mount Zion|Holmes County|MS|58054|United States|-6|apartment| +44020|AAAAAAAAEPLKAAAA|442|Second |Blvd|Suite O|Stringtown|Martinsville city|VA|20162|United States|-5|apartment| +44021|AAAAAAAAFPLKAAAA|440|Main Davis|Wy|Suite 210|Friendship|Edwards County|IL|64536|United States|-6|condo| +44022|AAAAAAAAGPLKAAAA|74|Lincoln |Ct.|Suite 240|Summit|Trinity County|CA|90499|United States|-8|apartment| +44023|AAAAAAAAHPLKAAAA|548|Oak |Court|Suite 250|Woodlawn|Jackson County|FL|34098|United States|-5|apartment| +44024|AAAAAAAAIPLKAAAA|627|5th Birch|Cir.|Suite C|Sunnyside|Schuyler County|NY|11952|United States|-5|apartment| +44025|AAAAAAAAJPLKAAAA||Broadway ||Suite A|Woodland|||54854|United States||| +44026|AAAAAAAAKPLKAAAA|699|Sixth 6th|Pkwy|Suite 330|Jackson|Sierra County|NM|89583|United States|-7|single family| +44027|AAAAAAAALPLKAAAA|389|3rd Seventh|Drive|Suite 200|Union|Marion County|OH|48721|United States|-5|apartment| +44028|AAAAAAAAMPLKAAAA|596|Valley Cedar|Parkway|Suite 230|Stratford|Kenosha County|WI|56668|United States|-6|condo| +44029|AAAAAAAANPLKAAAA|486|Maple |Drive|Suite 200|Newport|Wright County|MN|51521|United States|-6|single family| +44030|AAAAAAAAOPLKAAAA|921|Fifth Hickory|Street|Suite P|Florence|Gordon County|GA|33394|United States|-5|apartment| +44031|AAAAAAAAPPLKAAAA|236|First 3rd|Ave|Suite 300|Crossroads|Rockingham County|NH|01134|United States|-5|condo| +44032|AAAAAAAAAAMKAAAA|993|North 5th|Pkwy|Suite 40|Sulphur Springs|Comanche County|KS|68354|United States|-6|apartment| +44033|AAAAAAAABAMKAAAA|545|9th |Cir.|Suite D|Oakdale|East Carroll Parish|LA|79584|United States|-6|condo| +44034|AAAAAAAACAMKAAAA|487|Second Cedar|Boulevard|Suite R|Oak Grove|Clay County|NC|28370|United States|-5|condo| +44035|AAAAAAAADAMKAAAA|54|Washington |Ave|Suite 20|Concord|Windham County|CT|04707|United States|-5|apartment| +44036|AAAAAAAAEAMKAAAA|400|Third 8th|Street|Suite E|Ashland|New Kent County|VA|24244|United States|-5|condo| +44037|AAAAAAAAFAMKAAAA|954|Johnson |RD|Suite 0|Oak Hill|Cape Girardeau County|MO|67838|United States|-6|condo| +44038|AAAAAAAAGAMKAAAA|149|Lake Hill|Avenue|Suite 220|Shady Grove|Clatsop County|OR|92812|United States|-8|single family| +44039|AAAAAAAAHAMKAAAA|890|North 10th|ST|Suite 360|Silver Creek|Laurens County|GA|34546|United States|-5|condo| +44040|AAAAAAAAIAMKAAAA|382|1st |Ct.|Suite 150|Maple Grove|Sedgwick County|CO|88252|United States|-7|condo| +44041|AAAAAAAAJAMKAAAA|983|12th |Ave|Suite T|Lakeside|Sherman County|TX|79532|||| +44042|AAAAAAAAKAMKAAAA|607|Miller Ninth|Ln|Suite H|Brownsville|Richmond County|VA|29310|United States|-5|apartment| +44043|AAAAAAAALAMKAAAA|209|Hill |Pkwy|Suite 180|Marion|Grant County|WA|90399|United States|-8|apartment| +44044|AAAAAAAAMAMKAAAA|||||||AR|70587|United States||| +44045|AAAAAAAANAMKAAAA|224|Mill |Parkway|Suite S|Oakdale|Madison County|FL|39584|United States|-5|condo| +44046|AAAAAAAAOAMKAAAA|930|Oak Spring|Cir.|Suite 10|Sunnyside|Texas County|MO|61952|United States|-6|apartment| +44047|AAAAAAAAPAMKAAAA|484|11th |Ave|Suite R|Jamestown|Coke County|TX|76867|United States|-6|single family| +44048|AAAAAAAAABMKAAAA|966|6th |Wy|Suite 230|Shelby|Henderson County|TX|76575|United States|-6|single family| +44049|AAAAAAAABBMKAAAA|646|Johnson |Road|Suite O|Salem|Huntington County|IN|48048|United States|-5|condo| +44050|AAAAAAAACBMKAAAA|678|Second |Court|Suite P|Jamestown|Harrison County|TX|76867|United States|-6|condo| +44051|AAAAAAAADBMKAAAA|27|Oak Third|Circle|Suite 340|Cedar Grove|Rio Grande County|CO|80411|United States|-7|apartment| +44052|AAAAAAAAEBMKAAAA|968|Fifth View|Drive|Suite S|Five Points|Santa Rosa County|FL|36098|United States|-5|apartment| +44053|AAAAAAAAFBMKAAAA|630|Central |Avenue|Suite 140|Five Points|Barrow County|GA|36098|United States|-5|apartment| +44054|AAAAAAAAGBMKAAAA|850|Sunset |Boulevard|Suite 370|Oak Ridge|Charlotte County|FL|38371|United States|-5|condo| +44055|AAAAAAAAHBMKAAAA|351|10th |Court|Suite A|Newtown|Sandusky County|OH|41749|United States|-5|apartment| +44056|AAAAAAAAIBMKAAAA|960|Willow |Avenue|Suite E|Five Forks|Jackson County|MS|52293|United States|-6|condo| +44057|AAAAAAAAJBMKAAAA|976|Madison Smith|Boulevard|Suite E|Five Forks|Iron County|MO|62293|United States|-6|condo| +44058|AAAAAAAAKBMKAAAA|329|Eigth 3rd|Street|Suite X|Riverdale|Brown County|NE|69391|United States|-6|apartment| +44059|AAAAAAAALBMKAAAA|132|Poplar Elm|Ave|Suite H|Derby|Lubbock County|TX|77702|United States|-6|condo| +44060|AAAAAAAAMBMKAAAA|61|Cedar |RD|Suite 200|Maple Grove|Kankakee County|IL|68252|United States|-6|condo| +44061|AAAAAAAANBMKAAAA|928|Pine |Way|Suite 60|Greenwood|Dallas County|AL|38828|United States|-6|condo| +44062|AAAAAAAAOBMKAAAA|115|Sunset |Parkway|Suite V|Saint James|Pike County|OH|45799|United States|-5|single family| +44063|AAAAAAAAPBMKAAAA|721|View |Ct.|Suite S|Dallas|Perkins County|SD|53628|United States|-7|condo| +44064|AAAAAAAAACMKAAAA|494|Green |Ave|Suite X|Oak Grove|Catron County|NM|88370|United States|-7|condo| +44065|AAAAAAAABCMKAAAA|264|Third |Avenue|Suite 260|Pleasant Valley|Campbell County|VA|22477|United States|-5|single family| +44066|AAAAAAAACCMKAAAA|926|Park Broadway|Parkway|Suite E|Mount Vernon|Suffolk County|NY|18482|United States|-5|single family| +44067|AAAAAAAADCMKAAAA|285|Fifth Oak|Ave|Suite 0|Lincoln|Pike County|MS|51289|United States|-6|condo| +44068|AAAAAAAAECMKAAAA|499|Walnut 10th|Court|Suite 450|Collinsville|Noble County|IN|42459|United States|-5|apartment| +44069|AAAAAAAAFCMKAAAA|499|Mill |Way|Suite 270|Lebanon|Claiborne County|TN|32898|United States|-5|condo| +44070|AAAAAAAAGCMKAAAA|870|Main |Drive|Suite Y|Franklin|Perkins County|NE|69101|United States|-7|condo| +44071|AAAAAAAAHCMKAAAA|2|Elm Forest|Ct.|Suite J|Harmony|Boyle County|KY|45804|United States|-6|single family| +44072|AAAAAAAAICMKAAAA|976|Second Church|Dr.|Suite M|Wilson|Elliott County|KY|46971|United States|-6|single family| +44073|AAAAAAAAJCMKAAAA|||Circle|Suite 360||Wasatch County|UT||United States||| +44074|AAAAAAAAKCMKAAAA|14|Walnut |Avenue|Suite G|Franklin|Lander County|NV|89101|United States|-8|apartment| +44075|AAAAAAAALCMKAAAA|199|11th Chestnut|Blvd|Suite N|Mount Zion|Santa Rosa County|FL|38054|United States|-5|apartment| +44076|AAAAAAAAMCMKAAAA|798|3rd |Boulevard|Suite O|Newtown|Boone County|MO|61749|United States|-6|apartment| +44077|AAAAAAAANCMKAAAA|712|1st Park|Lane|Suite N|Five Points|Currituck County|NC|26098|United States|-5|apartment| +44078|AAAAAAAAOCMKAAAA||Spruce |||Hamilton|Mower County||52808|United States||| +44079|AAAAAAAAPCMKAAAA|86|Center |Pkwy|Suite 20|Mountain View|Pueblo County|CO|84466|United States|-7|condo| +44080|AAAAAAAAADMKAAAA|658|Laurel |ST|Suite Q|Sunnyside|Madison County|OH|41952|United States|-5|single family| +44081|AAAAAAAABDMKAAAA|906|Elm |Pkwy|Suite A|Kelly|Cook County|IL|62738|United States|-6|condo| +44082|AAAAAAAACDMKAAAA|568|View |Court|Suite R|Rosewood|Craven County|NC|22205|United States|-5|apartment| +44083|AAAAAAAADDMKAAAA|31|3rd Green|Ave|Suite 240|Union|Long County|GA|38721|United States|-5|condo| +44084|AAAAAAAAEDMKAAAA|934|Maple Maple|Court|Suite 350|Wilson|Creek County|OK|76971|United States|-6|single family| +44085|AAAAAAAAFDMKAAAA|10|Spring |Parkway|Suite 250|Ashland|Pontotoc County|OK|74244|United States|-6|single family| +44086|AAAAAAAAGDMKAAAA|994|Hillcrest 8th|Ave|Suite V|White Oak|Stone County|MS|56668|United States|-6|condo| +44087|AAAAAAAAHDMKAAAA|608|Wilson Walnut|Way|Suite 20|Macedonia|Blount County|TN|31087|United States|-5|condo| +44088|AAAAAAAAIDMKAAAA|368|View Hill|Court|Suite E|Pine Grove|Monroe County|FL|34593|United States|-5|condo| +44089|AAAAAAAAJDMKAAAA||1st |RD|Suite 350|Cunningham||||United States|-5|condo| +44090|AAAAAAAAKDMKAAAA|732|Jackson |Avenue|Suite 280|Union|Logan County|KS|68721|United States|-6|apartment| +44091|AAAAAAAALDMKAAAA|982|Cherry |Road|Suite 380|Clifton|Rutherford County|NC|28014|United States|-5|apartment| +44092|AAAAAAAAMDMKAAAA|909|Williams |Parkway|Suite 360|Antioch|Quitman County|MS|58605|United States|-6|apartment| +44093|AAAAAAAANDMKAAAA|726|Main 4th|Ave|Suite F|White Hall|Lawrence County|PA|16955|United States|-5|single family| +44094|AAAAAAAAODMKAAAA|782|Chestnut |Wy|Suite I|Union Hill|Charlottesville city|VA|27746|United States|-5|single family| +44095|AAAAAAAAPDMKAAAA|133|College |Dr.|Suite E|Macedonia|Pickens County|SC|21087|United States|-5|single family| +44096|AAAAAAAAAEMKAAAA|124|4th 6th|Wy|Suite S|Sulphur Springs|Angelina County|TX|78354|United States|-6|apartment| +44097|AAAAAAAABEMKAAAA|320|Seventh Miller|Pkwy|Suite T|Youngstown|Winn Parish|LA|70001|United States|-6|condo| +44098|AAAAAAAACEMKAAAA|745|Walnut Spring|Way|Suite 480|Valley View|Northampton County|PA|15124|United States|-5|apartment| +44099|AAAAAAAADEMKAAAA|289|Fourth Meadow|Avenue|Suite M|Spring Valley|Wayne County|WV|26060|United States|-5|apartment| +44100|AAAAAAAAEEMKAAAA|50|South |Ave|Suite 270|Union|Assumption Parish|LA|78721|United States|-6|apartment| +44101|AAAAAAAAFEMKAAAA|632|Main Pine|Court|Suite 130|Antioch|Sanilac County|MI|48605|United States|-5|condo| +44102|AAAAAAAAGEMKAAAA|893|Sunset |Circle|Suite S|Clifton|Mason County|MI|48014|United States|-5|condo| +44103|AAAAAAAAHEMKAAAA|582|Walnut |Cir.|Suite D|Pine Grove|Fulton County|AR|74593|United States|-6|apartment| +44104|AAAAAAAAIEMKAAAA|6|Third |Cir.|Suite 270|Crossroads|Caswell County|NC|20534|United States|-5|single family| +44105|AAAAAAAAJEMKAAAA|577|Church Willow|Blvd|Suite U|Concord|Shannon County|SD|54107|United States|-7|apartment| +44106|AAAAAAAAKEMKAAAA|281|5th |Wy|Suite 280|Cedar Grove|Susquehanna County|PA|10411|United States|-5|apartment| +44107|AAAAAAAALEMKAAAA|805|Main |Ave|Suite 10|Marion|Lackawanna County|PA|10399|United States|-5|condo| +44108|AAAAAAAAMEMKAAAA|745|Washington Main|Circle|Suite 70|Crossroads|Poweshiek County|IA|50534|United States|-6|apartment| +44109|AAAAAAAANEMKAAAA|919|1st 14th|Dr.|Suite 90|Pleasant Valley|Hyde County|SD|52477|United States|-7|apartment| +44110|AAAAAAAAOEMKAAAA|75|Poplar Second|Blvd|Suite N|Pleasant Hill|Wise County|TX|73604|United States|-6|single family| +44111|AAAAAAAAPEMKAAAA|177|Forest |Parkway|Suite Q|Lakeside|Lexington County|SC|29532|United States|-5|condo| +44112|AAAAAAAAAFMKAAAA|391|Washington |Ct.|Suite K|Oak Grove|Jefferson County|AL|38370|United States|-6|condo| +44113|AAAAAAAABFMKAAAA|998|Sixth Railroad|Pkwy|Suite 430|Lakewood|Schleicher County|TX|78877|United States|-6|apartment| +44114|AAAAAAAACFMKAAAA|847|Ridge |Lane|Suite Y|Pleasant Grove|Rockbridge County|VA|24136|United States|-5|apartment| +44115|AAAAAAAADFMKAAAA|566|River |Circle|Suite J|Newport|Acadia Parish|LA|71521|United States|-6|condo| +44116|AAAAAAAAEFMKAAAA|||Court|||Manassas Park city||||-5|| +44117|AAAAAAAAFFMKAAAA|151|Davis |Ln|Suite R|Riverside|Prowers County|CO|89231|United States|-7|apartment| +44118|AAAAAAAAGFMKAAAA|897|3rd River|Parkway|Suite 450|Hopewell|Cerro Gordo County|IA|50587|United States|-6|apartment| +44119|AAAAAAAAHFMKAAAA|323|Willow Walnut|RD|Suite H|Jamestown|Kinney County|TX|76867|United States|-6|apartment| +44120|AAAAAAAAIFMKAAAA|609|Wilson |Drive|Suite X|Five Points|Wilkinson County|MS|56098|United States|-6|apartment| +44121|AAAAAAAAJFMKAAAA|263|Main |Boulevard|Suite J|Greenwood|Audubon County|IA|58828|United States|-6|apartment| +44122|AAAAAAAAKFMKAAAA|297|15th |Ct.|Suite D|Lakeside|Conway County|AR|79532|United States|-6|condo| +44123|AAAAAAAALFMKAAAA|168|Sunset Walnut|Ave|Suite 190|Wayland|Columbiana County|OH|45115|United States|-5|condo| +44124|AAAAAAAAMFMKAAAA|575|Jefferson |Blvd|Suite U|Pine Grove|Greene County|TN|34593|United States|-5|apartment| +44125|AAAAAAAANFMKAAAA|383|East 7th|Drive|Suite O|Thomas|Miller County|GA|30330|United States|-5|condo| +44126|AAAAAAAAOFMKAAAA|550|Ash |Dr.|Suite 380|Gum Springs|Calhoun County|AR|72106|United States|-6|single family| +44127|AAAAAAAAPFMKAAAA|408|First Hill|Court|Suite R|Salem|Lyon County|KS|68048|United States|-6|condo| +44128|AAAAAAAAAGMKAAAA|462|West Wilson|Court|Suite 200|Bunker Hill|Campbell County|KY|40150|United States|-6|apartment| +44129|AAAAAAAABGMKAAAA|658|Williams |ST|Suite N|Sulphur Springs|Lamar County|TX|78354|United States|-6|single family| +44130|AAAAAAAACGMKAAAA||||||||50411|United States|-6|condo| +44131|AAAAAAAADGMKAAAA|541|Sycamore Meadow|Pkwy|Suite 400|Valley View|San Bernardino County|CA|95124|United States|-8|single family| +44132|AAAAAAAAEGMKAAAA|992|Spruce |Dr.|Suite S|Jackson|La Crosse County|WI|59583|United States|-6|apartment| +44133|AAAAAAAAFGMKAAAA|473|Fifth |Court|Suite 20|Spring Valley|Attala County|MS|56060|United States|-6|single family| +44134|AAAAAAAAGGMKAAAA|801|2nd Lake|Court|Suite V|Five Points|Winnebago County|IL|66098|United States|-6|condo| +44135|AAAAAAAAHGMKAAAA|819|Walnut Madison|Court|Suite 270|Harmony|Fallon County|MT|65804|United States|-7|condo| +44136|AAAAAAAAIGMKAAAA|278|Mill |Street|Suite 130|Enterprise|Halifax County|NC|21757|United States|-5|single family| +44137|AAAAAAAAJGMKAAAA|278|Birch Cherry|Parkway|Suite B|Walnut Grove|Fayette County|TN|37752|United States|-5|apartment| +44138|AAAAAAAAKGMKAAAA|339|3rd |RD|Suite P|Union Hill|Red Willow County|NE|67746|United States|-7|single family| +44139|AAAAAAAALGMKAAAA|420|3rd Smith|Street|Suite F|Waterloo|Huntington County|IN|41675|United States|-5|apartment| +44140|AAAAAAAAMGMKAAAA|738|Madison Locust|Way|Suite 340|Green Acres|Jackson County|AL|37683|United States|-6|condo| +44141|AAAAAAAANGMKAAAA|192|9th |Street|Suite U|Providence|Shackelford County|TX|76614|United States|-6|condo| +44142|AAAAAAAAOGMKAAAA|556|5th |RD|Suite 220|Providence|Bleckley County|GA|36614|United States|-5|condo| +44143|AAAAAAAAPGMKAAAA|841|Hillcrest |Dr.|Suite O|Fairfield|Mineral County|CO|86192|United States|-7|condo| +44144|AAAAAAAAAHMKAAAA|31|River Washington|Street|Suite 430|Lakewood|Benson County|ND|58877|United States|-6|apartment| +44145|AAAAAAAABHMKAAAA|15|Jackson Lee|Blvd|Suite 30|Tracy|Boyle County|KY|46340|United States|-6|single family| +44146|AAAAAAAACHMKAAAA|602|Birch |Court|Suite 260|Glenville|Box Elder County|UT|83445|United States|-7|apartment| +44147|AAAAAAAADHMKAAAA|957|Main Franklin|Ct.|Suite H|Arlington|Madison County|AR|76557|United States|-6|condo| +44148|AAAAAAAAEHMKAAAA|259|Woodland Johnson|Parkway|Suite E|Oak Grove|Greer County|OK|78370|United States|-6|single family| +44149|AAAAAAAAFHMKAAAA|193|Hickory View|Cir.|Suite 250|Hillcrest|Osage County|OK|73003|United States|-6|single family| +44150|AAAAAAAAGHMKAAAA|883|Cedar Oak|Boulevard|Suite D|Hopewell|Walla Walla County|WA|90587|United States|-8|condo| +44151|AAAAAAAAHHMKAAAA|542|Locust Highland|RD|Suite C|New Hope|Franklin Parish|LA|79431|United States|-6|condo| +44152|AAAAAAAAIHMKAAAA|995|Walnut |Ave|Suite 80|Highland|Decatur County|IN|49454|United States|-5|apartment| +44153|AAAAAAAAJHMKAAAA|43|Woodland |Circle|Suite 390|Kingston|Chemung County|NY|14975|United States|-5|single family| +44154|AAAAAAAAKHMKAAAA|316|Hickory Sunset|Boulevard|Suite A||Loup County||||-7|| +44155|AAAAAAAALHMKAAAA|618|Park |Boulevard|Suite 290|Brownsville|Stanley County|SD|59310|United States|-7|condo| +44156|AAAAAAAAMHMKAAAA|326|Spring |Street|Suite T|Five Points|Lake County|CA|96098|United States|-8|single family| +44157|AAAAAAAANHMKAAAA|424|5th |Street|Suite 450|Enterprise|Clay County|NE|61757|United States|-6|condo| +44158|AAAAAAAAOHMKAAAA|927|2nd Green|Circle|Suite Q|Red Hill|Richland County|OH|44338|United States|-5|condo| +44159|AAAAAAAAPHMKAAAA|44|Fourth |Ave|Suite A|Kingston|Mason County|WA|94975|United States|-8|apartment| +44160|AAAAAAAAAIMKAAAA|9|Valley |Street|Suite G|Hamilton|Fentress County|TN|32808|United States|-5|condo| +44161|AAAAAAAABIMKAAAA|354|Miller |Ln|Suite F|Franklin|Allen County|KS|69101|United States|-6|condo| +44162|AAAAAAAACIMKAAAA|396|Maple Johnson|Avenue|Suite R|Crossroads|Hall County|GA|30534|United States|-5|condo| +44163|AAAAAAAADIMKAAAA|839|Fifth Forest|Ln|Suite A|Woodville|Westmoreland County|VA|24289|United States|-5|single family| +44164|AAAAAAAAEIMKAAAA|337|Johnson |Road|Suite D|Five Points|Union County|IN|46098|United States|-5|single family| +44165|AAAAAAAAFIMKAAAA|26|Hill View|Dr.|Suite 340|Lakeview|Huron County|MI|48579|United States|-5|single family| +44166|AAAAAAAAGIMKAAAA|458|Spring |Ln|Suite T|White Oak|Oneida County|ID|86668|United States|-7|condo| +44167|AAAAAAAAHIMKAAAA|992|West Twelfth|Drive|Suite N|Derby|Surry County|NC|27702|United States|-5|condo| +44168|AAAAAAAAIIMKAAAA|785|Mill 13th|Avenue|Suite K|Arlington|Marathon County|WI|56557|United States|-6|single family| +44169|AAAAAAAAJIMKAAAA|340|Williams |Way|Suite 220|Fairfield|Walton County|FL|36192|United States|-5|single family| +44170|AAAAAAAAKIMKAAAA|959|3rd 2nd|Court|Suite 270|Riverview|Riley County|KS|69003|United States|-6|single family| +44171|AAAAAAAALIMKAAAA|960|7th |Ave|Suite A|Mount Olive|Yancey County|NC|28059|United States|-5|single family| +44172|AAAAAAAAMIMKAAAA|904|Fifth Seventh|Wy|Suite 70|Franklin|Tioga County|PA|19101|United States|-5|condo| +44173|AAAAAAAANIMKAAAA|544|Railroad 10th|Circle|Suite 160|Macedonia|Forest County|WI|51087|United States|-6|single family| +44174|AAAAAAAAOIMKAAAA|165|Ridge |Pkwy|Suite W|Valley View|Bedford County|VA|25124|United States|-5|single family| +44175|AAAAAAAAPIMKAAAA|621|Ridge River|RD|Suite 450|Florence|Clay County|KS|63394|United States|-6|single family| +44176|AAAAAAAAAJMKAAAA|807|First |Way|Suite U|Kingston|Clayton County|GA|34975|United States|-5|single family| +44177|AAAAAAAABJMKAAAA|274|Smith |ST|Suite 90|Walnut Grove|Jefferson County|FL|37752|United States|-5|apartment| +44178|AAAAAAAACJMKAAAA|55|Sycamore |Ln|Suite 480|Springdale|Yuma County|CO|88883|United States|-7|condo| +44179|AAAAAAAADJMKAAAA|186|1st |Wy|Suite Y|Green Acres|Perry County|IL|67683|United States|-6|condo| +44180|AAAAAAAAEJMKAAAA|613|Church Second|Boulevard|Suite 260|Lincoln|Arlington County|VA|21289|United States|-5|single family| +44181|AAAAAAAAFJMKAAAA|880|6th |Lane|Suite 330|Union|Graham County|NC|28721|United States|-5|single family| +44182|AAAAAAAAGJMKAAAA|7|Walnut |Dr.|Suite 20|Red Hill|Barron County|WI|54338|United States|-6|single family| +44183|AAAAAAAAHJMKAAAA|669|Pine |ST|Suite 200|Hillcrest|East Baton Rouge Parish|LA|73003|United States|-6|condo| +44184|AAAAAAAAIJMKAAAA|114|Washington |Dr.|Suite 470|Shiloh|McMullen County|TX|79275|United States|-6|single family| +44185|AAAAAAAAJJMKAAAA|75|4th Lake|Ct.|Suite W|Georgetown|Larimer County|CO|87057|United States|-7|apartment| +44186|AAAAAAAAKJMKAAAA|54|Second Walnut|Circle|Suite 260|Mount Olive|Penobscot County|ME|08659|United States|-5|single family| +44187|AAAAAAAALJMKAAAA|671|Cherry Oak|Parkway|Suite O|Woodlawn|Ada County|ID|84098|United States|-7|condo| +44188|AAAAAAAAMJMKAAAA|609|7th |Road|Suite T|Lebanon|Burke County|GA|32898|United States|-5|single family| +44189|AAAAAAAANJMKAAAA|827|9th Poplar|Parkway|Suite I|Edgewood|Seneca County|OH|40069|United States|-5|condo| +44190|AAAAAAAAOJMKAAAA|872||Street|||Marshall County|KS|65817|||single family| +44191|AAAAAAAAPJMKAAAA|912|Central |ST|Suite 110|Lebanon|Union County|TN|32898|United States|-6|apartment| +44192|AAAAAAAAAKMKAAAA|732|Washington Lincoln|Wy|Suite I|Woodland|Cleburne County|AR|74854|United States|-6|condo| +44193|AAAAAAAABKMKAAAA|489|Forest |Ct.|Suite C|Hopewell|Wilbarger County|TX|70587|United States|-6|single family| +44194|AAAAAAAACKMKAAAA|997|Washington Central|RD|Suite 440|Salem|Kit Carson County|CO|88048|United States|-7|single family| +44195|AAAAAAAADKMKAAAA|924|Second |Street|Suite E|Pleasant Valley|McIntosh County|OK|72477|United States|-6|condo| +44196|AAAAAAAAEKMKAAAA|683|Ridge Hillcrest|Cir.|Suite F|Sulphur Springs|Beaver County|PA|18354|United States|-5|apartment| +44197|AAAAAAAAFKMKAAAA|647|5th Cherry|Ct.|Suite P|Spring Valley|Stevens County|WA|96060|United States|-8|single family| +44198|AAAAAAAAGKMKAAAA|996|Broadway |Court|Suite H|Jackson|Polk County|WI|59583|United States|-6|condo| +44199|AAAAAAAAHKMKAAAA|166|Hickory |Blvd|Suite U|Riverdale|Sevier County|AR|79391|United States|-6|condo| +44200|AAAAAAAAIKMKAAAA|827|Elm 5th|Boulevard|Suite 280|Summit|Reagan County|TX|70499|United States|-6|single family| +44201|AAAAAAAAJKMKAAAA|902|Walnut 8th|Cir.|Suite J|Union Hill|Upson County|GA|37746|United States|-5|single family| +44202|AAAAAAAAKKMKAAAA|918|Cedar Hickory|Drive|Suite I|Leesburg|Hinsdale County|CO|85605|United States|-7|apartment| +44203|AAAAAAAALKMKAAAA|38|6th Pine|Avenue|Suite U|Midway|Franklin County|VA|21904|United States|-5|single family| +44204|AAAAAAAAMKMKAAAA|16|View |Parkway|Suite F|Summit|Iredell County|NC|20499|United States|-5|single family| +44205|AAAAAAAANKMKAAAA|382|Main |RD|Suite 190|Five Points|Taylor County|TX|76098|United States|-6|condo| +44206|AAAAAAAAOKMKAAAA|8|Johnson |Dr.|Suite 310|Glenwood|Tripp County|SD|53511|United States|-7|condo| +44207|AAAAAAAAPKMKAAAA|922|Center Smith|Drive|Suite M|Concord|Marshall County|MS|54107|United States|-6|apartment| +44208|AAAAAAAAALMKAAAA|||Parkway|Suite W||||79391|United States|-6|single family| +44209|AAAAAAAABLMKAAAA|540|Lake Main|Ct.|Suite 340|Lakewood|Emery County|UT|88877|United States|-7|apartment| +44210|AAAAAAAACLMKAAAA|422|2nd |Street|Suite 280|Forest Hills|Crane County|TX|79237|United States|-6|apartment| +44211|AAAAAAAADLMKAAAA||North 8th|||||NY|11160||-5|apartment| +44212|AAAAAAAAELMKAAAA|850|Adams Sixth|Cir.|Suite T|Oakland|Floyd County|IN|49843|United States|-5|apartment| +44213|AAAAAAAAFLMKAAAA|373|East |Parkway|Suite S|Lakeview|Hall County|TX|78579|United States|-6|apartment| +44214|AAAAAAAAGLMKAAAA|343|Railroad |Cir.|Suite T|Plainview|Oakland County|MI|43683|United States|-5|single family| +44215|AAAAAAAAHLMKAAAA|234|Eigth Locust|Court|Suite O|Forest Hills|Lauderdale County|AL|39237|United States|-6|apartment| +44216|AAAAAAAAILMKAAAA|67|Park |Court|Suite I|Greenfield|Lincoln County|TN|35038|United States|-6|condo| +44217|AAAAAAAAJLMKAAAA|117|Oak Broadway|Wy|Suite V|Pleasant Grove|Maury County|TN|34136|United States|-6|condo| +44218|AAAAAAAAKLMKAAAA|494|Sycamore |Court|Suite 330|Providence|Orange County|FL|36614|United States|-5|condo| +44219|AAAAAAAALLMKAAAA|387|Washington 3rd|ST|Suite G|Hamilton|Ascension Parish|LA|72808|United States|-6|apartment| +44220|AAAAAAAAMLMKAAAA|967|Central |Parkway|Suite A|Fairfield|Newton County|MO|66192|United States|-6|single family| +44221|AAAAAAAANLMKAAAA|566|College |Way|Suite 340|Jamestown|Wabash County|IL|66867|United States|-6|apartment| +44222|AAAAAAAAOLMKAAAA|780|15th |Court|Suite 50|Friendship|Lincoln County|GA|34536|United States|-5|single family| +44223|AAAAAAAAPLMKAAAA|894|2nd |Circle|Suite 230|Pinecrest|Dawson County|TX|79981|United States|-6|apartment| +44224|AAAAAAAAAMMKAAAA|828|Spring 12th|Lane|Suite 380|Belleview|George County|MS|50492|United States|-6|apartment| +44225|AAAAAAAABMMKAAAA|824|Lakeview |Lane|Suite 440|Mount Olive|Boyle County|KY|48059|United States|-6|apartment| +44226|AAAAAAAACMMKAAAA|74|College College|Blvd|Suite Y|Sunnyside|Gilchrist County|FL|31952|United States|-5|apartment| +44227|AAAAAAAADMMKAAAA|879|1st Pine|Cir.|Suite J|Walnut Grove|Venango County|PA|17752|United States|-5|single family| +44228|AAAAAAAAEMMKAAAA|164|7th 9th|Circle|Suite L|Wildwood|Washington County|OH|46871|United States|-5|condo| +44229|AAAAAAAAFMMKAAAA|11|Jackson |Wy|Suite S|Union Hill|Walker County|TX|77746|United States|-6|single family| +44230|AAAAAAAAGMMKAAAA|303|First |Dr.|Suite 370|Bethel|Tillamook County|OR|95281|United States|-8|single family| +44231|AAAAAAAAHMMKAAAA|305|Hillcrest |Court|Suite 220|Greenville|Kanawha County|WV|21387|United States|-5|single family| +44232|AAAAAAAAIMMKAAAA|285|Elm |Avenue|Suite J|Lebanon|Washington County|IL|62898|||| +44233|AAAAAAAAJMMKAAAA|||||Newtown||KS||United States||| +44234|AAAAAAAAKMMKAAAA|757|9th |Court|Suite 370|Mountain View|Edwards County|TX|74466|United States|-6|apartment| +44235|AAAAAAAALMMKAAAA|640|10th Maple|ST|Suite Y|Highland Park|Audubon County|IA|56534|United States|-6|single family| +44236|AAAAAAAAMMMKAAAA|264|7th |Street|Suite K|Riceville|Okanogan County|WA|95867|United States|-8|apartment| +44237|AAAAAAAANMMKAAAA|350|Oak 2nd|Wy|Suite 80|Pine Grove|Decatur County|KS|64593|United States|-6|apartment| +44238|AAAAAAAAOMMKAAAA|691|Lincoln Central|Blvd|Suite 270|Pine Grove|Jefferson County|FL|34593|United States|-5|condo| +44239|AAAAAAAAPMMKAAAA|659|Fifteenth West|ST|Suite C|Stringtown|Choctaw County|AL|30162|United States|-6|single family| +44240|AAAAAAAAANMKAAAA|724|Birch Sixth|Blvd|Suite L|Lakewood|Terrebonne Parish|LA|78877|United States|-6|apartment| +44241|AAAAAAAABNMKAAAA|133|Lake |RD|Suite C|Antioch|Chattooga County|GA|38605|United States|-5|single family| +44242|AAAAAAAACNMKAAAA|100|2nd |Boulevard|Suite D|Salem|Kandiyohi County|MN|58048|United States|-6|single family| +44243|AAAAAAAADNMKAAAA|463|Miller |Lane|Suite Y|Woodland|Walton County|GA|34854|United States|-5|apartment| +44244|AAAAAAAAENMKAAAA|389|Dogwood Laurel|Way|Suite U|White Oak|Tuscaloosa County|AL|36668|United States|-6|apartment| +44245|AAAAAAAAFNMKAAAA|249|11th Pine|Ln|Suite 340|Plainview|Lamar County|MS|53683|United States|-6|single family| +44246|AAAAAAAAGNMKAAAA|202|4th |Wy|Suite 220|Franklin|Bossier Parish|LA|79101|United States|-6|single family| +44247|AAAAAAAAHNMKAAAA|178|Fourth |Avenue|Suite 430|Hamilton|Tyrrell County|NC|22808|United States|-5|condo| +44248|AAAAAAAAINMKAAAA|870|Center Thirteenth|Avenue|Suite 380|Five Forks|Huron County|MI|42293|United States|-5|condo| +44249|AAAAAAAAJNMKAAAA|305|Willow 14th|Wy|Suite 200|Cedar Grove|Washington County|KY|40411|United States|-5|apartment| +44250|AAAAAAAAKNMKAAAA|828|5th Lincoln|Street|Suite 260|Lakeview|Upton County|TX|78579|United States|-6|single family| +44251|AAAAAAAALNMKAAAA|267|Main Spring|RD|Suite 460|Hamilton|Chester County|TN|32808|United States|-5|single family| +44252|AAAAAAAAMNMKAAAA|362|Hill 8th|Dr.|Suite 350|Waterloo|Los Alamos County|NM|81675|United States|-7|condo| +44253|AAAAAAAANNMKAAAA|234|Ash 2nd|Pkwy|Suite 120|Antioch|Arenac County|MI|48605|United States|-5|single family| +44254|AAAAAAAAONMKAAAA|424||Cir.|Suite L|Oakland|||39843|United States||condo| +44255|AAAAAAAAPNMKAAAA|761|Center Church|RD|Suite 370|Red Hill|Worcester County|MD|24338|United States|-5|apartment| +44256|AAAAAAAAAOMKAAAA|870|6th |Avenue|Suite 300|Edgewood|Nevada County|AR|70069|United States|-6|condo| +44257|AAAAAAAABOMKAAAA|623|15th |RD|Suite 230|Antioch|Coffee County|AL|38605|United States|-6|apartment| +44258|AAAAAAAACOMKAAAA|142|Lake Main|Dr.|Suite W|Antioch|Hanover County|VA|28605|United States|-5|apartment| +44259|AAAAAAAADOMKAAAA|613|Jackson Davis|ST|Suite X|Jamestown|Pima County|AZ|86867|United States|-7|apartment| +44260|AAAAAAAAEOMKAAAA|350|Pine |RD|Suite C|Providence|Franklin County|MA|07214|United States|-5|condo| +44261|AAAAAAAAFOMKAAAA|987|Mill |Road|Suite 200|Pleasant Hill|Adair County|OK|73604|United States|-6|apartment| +44262|AAAAAAAAGOMKAAAA|854|4th |Cir.|Suite R|Brownsville|Lincoln County|WY|89310|United States|-7|apartment| +44263|AAAAAAAAHOMKAAAA|400|4th Cedar|Wy|Suite 70|Centerville|Winchester city|VA|20059|United States|-5|single family| +44264|AAAAAAAAIOMKAAAA|605|Oak 3rd|Drive|Suite Q|Mount Vernon|Suwannee County|FL|38482|United States|-5|condo| +44265|AAAAAAAAJOMKAAAA|75|3rd |Pkwy|Suite U|Milo|Clarke County|IA|50116|United States|-6|condo| +44266|AAAAAAAAKOMKAAAA|866|Washington |Road|Suite W|Oakdale|Ohio County|WV|29584|United States|-5|single family| +44267|AAAAAAAALOMKAAAA|304|11th |ST|Suite 40|Waterloo|McPherson County|SD|51675|United States|-7|condo| +44268|AAAAAAAAMOMKAAAA||||Suite K|Newport|Haskell County||61521||-6|single family| +44269|AAAAAAAANOMKAAAA|960|Second Washington|Ct.|Suite H|Wesley|Oregon County|MO|61218|United States|-6|single family| +44270|AAAAAAAAOOMKAAAA|798|Adams 14th|Way|Suite R|Omega|Cass County|MI|46364|United States|-5|apartment| +44271|AAAAAAAAPOMKAAAA|954|Madison Lee|Ln|Suite V|Clinton|Orange County|VT|08822|United States|-5|condo| +44272|AAAAAAAAAPMKAAAA|320|2nd |Wy|Suite 160|West Liberty|Howard County|IA|54752|United States|-6|condo| +44273|AAAAAAAABPMKAAAA|641|Chestnut Spring|ST|Suite G|Mount Zion|Warren County|MS|58054|United States|-6|single family| +44274|AAAAAAAACPMKAAAA|113|Cedar |Blvd|Suite 130|Plainview|Lane County|KS|63683|United States|-6|condo| +44275|AAAAAAAADPMKAAAA|747|4th |Street|Suite 220|Mount Olive|Big Horn County|MT|68059|United States|-7|single family| +44276|AAAAAAAAEPMKAAAA||Franklin Dogwood|Circle||||||United States||| +44277|AAAAAAAAFPMKAAAA|256|Lee Church|Wy|Suite U|Highland Park|Mendocino County|CA|96534|United States|-8|single family| +44278|AAAAAAAAGPMKAAAA|316|Washington |Lane|Suite 220|Mount Zion|Rockland County|NY|18054|United States|-5|single family| +44279|AAAAAAAAHPMKAAAA|546|Central Hill|Dr.|Suite U|Union Hill|Harford County|MD|27746|United States|-5|apartment| +44280|AAAAAAAAIPMKAAAA|543|5th |Blvd|Suite D|Hopewell|Adams County|NE|60587|United States|-6|condo| +44281|AAAAAAAAJPMKAAAA|208|Franklin |ST|Suite 460|Walnut Grove|Marion County|IA|57752|United States|-6|condo| +44282|AAAAAAAAKPMKAAAA|285|6th 2nd|Cir.|Suite 430|Willis|Lake County|SD|56788|United States|-7|apartment| +44283|AAAAAAAALPMKAAAA|136|Forest Davis|Blvd|Suite B|Cedar Grove|Mingo County|WV|20411|United States|-5|single family| +44284|AAAAAAAAMPMKAAAA|614|Spring Forest|Ct.|Suite E|Lebanon|Little River County|AR|72898|United States|-6|apartment| +44285|AAAAAAAANPMKAAAA|421|Forest |RD|Suite 460|Sulphur Springs|San Juan County|UT|88354|United States|-7|apartment| +44286|AAAAAAAAOPMKAAAA|60|Fourth |ST|Suite T|Greenfield|Lincoln County|ID|85038|United States|-7|apartment| +44287|AAAAAAAAPPMKAAAA|330|Birch |Cir.|Suite 190|Jackson|Cumberland County|TN|39583|United States|-5|condo| +44288|AAAAAAAAAANKAAAA|791|Ash |Dr.|Suite 290|Fairfield|Butler County|KY|46192|United States|-6|single family| +44289|AAAAAAAABANKAAAA|619|View Lee|Lane|Suite 290|Lebanon|Wilcox County|GA|32898|United States|-5|apartment| +44290|AAAAAAAACANKAAAA|356|Wilson |Drive|Suite 260|Georgetown|Swain County|NC|27057|United States|-5|apartment| +44291|AAAAAAAADANKAAAA|1|Park 2nd|Blvd|Suite I|Spring Hill|Sandusky County|OH|46787|United States|-5|condo| +44292|AAAAAAAAEANKAAAA|461|Lincoln Elm|Way|Suite 440|Pleasant Grove|Kodiak Island Borough|AK|94136|United States|-9|apartment| +44293|AAAAAAAAFANKAAAA|716|Oak |Parkway|Suite 350|Hamilton|Union County|IA|52808|United States|-6|apartment| +44294|AAAAAAAAGANKAAAA|656|||||Laclede County|MO||United States|-6|condo| +44295|AAAAAAAAHANKAAAA|648|Walnut Park|Circle|Suite F|Stratford|Orange County|NY|16668|United States|-5|single family| +44296|AAAAAAAAIANKAAAA|733|Valley Main|RD|Suite 60|Greenville|Suffolk County|MA|01987|United States|-5|condo| +44297|AAAAAAAAJANKAAAA|342|Poplar College|Parkway|Suite B|Pleasant Valley|Humboldt County|NV|82477|United States|-8|single family| +44298|AAAAAAAAKANKAAAA|528|View South|RD|Suite 320|Spring Hill|Dickinson County|KS|66787|United States|-6|condo| +44299|AAAAAAAALANKAAAA|||Ave||Belmont|||40191|United States||| +44300|AAAAAAAAMANKAAAA|132|Oak |Cir.|Suite 70|Bay View|Sussex County|VA|26457|United States|-5|condo| +44301|AAAAAAAANANKAAAA|530|Railroad |Street|Suite 340|Red Hill|Glacier County|MT|64338|United States|-7|single family| +44302|AAAAAAAAOANKAAAA|117|Pine West|Road|Suite W|Saint Johns|Lincoln County|WV|25717|United States|-5|condo| +44303|AAAAAAAAPANKAAAA|798|Cherry |Pkwy|Suite 210|Lakewood|Graham County|NC|28877|United States|-5|condo| +44304|AAAAAAAAABNKAAAA|778|North 15th|ST|Suite 70|Stringtown|Jefferson County|OK|70162|United States|-6|apartment| +44305|AAAAAAAABBNKAAAA|707|Poplar 8th|ST|Suite L|Pine Grove|Renville County|ND|54593|United States|-6|single family| +44306|AAAAAAAACBNKAAAA|398|Lincoln |Drive|Suite 470|Riverview|Bergen County|NJ|09603|United States|-5|condo| +44307|AAAAAAAADBNKAAAA|97|Elm |Court|Suite 260|Antioch|Brown County|MN|58605|United States|-6|single family| +44308|AAAAAAAAEBNKAAAA|783|North |Road|Suite 250|Lakeview|Hardin County|TN|38579|United States|-5|single family| +44309|AAAAAAAAFBNKAAAA|558|Center Lake|Boulevard|Suite M|Edgewood|Mille Lacs County|MN|50069|United States|-6|condo| +44310|AAAAAAAAGBNKAAAA|216|East |Lane|Suite X|Oak Ridge|Stanley County|SD|58371|United States|-7|condo| +44311|AAAAAAAAHBNKAAAA|947|Highland Sixth|Boulevard|Suite X|Hazelwood|Madison Parish|LA|71206|United States|-6|condo| +44312|AAAAAAAAIBNKAAAA|279|8th |Ave|Suite N|Colfax|Perry County|KY|42565|United States|-5|condo| +44313|AAAAAAAAJBNKAAAA|787|Johnson |Ct.|Suite H|Kingston|Chouteau County|MT|64975|United States|-7|condo| +44314|AAAAAAAAKBNKAAAA|502|Main |Boulevard|Suite R|Mount Zion|Dickenson County|VA|28054|United States|-5|condo| +44315|AAAAAAAALBNKAAAA|997|Second |Blvd|Suite 10|Jamestown|Cumberland County|NJ|07467|United States|-5|condo| +44316|AAAAAAAAMBNKAAAA|455|9th |Circle|Suite 160|Ashland|Bates County|MO|64244|United States|-6|single family| +44317|AAAAAAAANBNKAAAA|563|Sunset |ST|Suite B|Bridgeport|Kimble County|TX|75817|United States|-6|single family| +44318|AAAAAAAAOBNKAAAA|783|Green Highland|Dr.|Suite A|Fairfield|Waller County|TX|76192|United States|-6|condo| +44319|AAAAAAAAPBNKAAAA|487|2nd |Ct.|Suite 290|Five Points|Bartow County|GA|36098|United States|-5|single family| +44320|AAAAAAAAACNKAAAA|829|Walnut |ST|Suite 50|Mount Vernon|Troup County|GA|38482|United States|-5|apartment| +44321|AAAAAAAABCNKAAAA|102|Smith Woodland|Road|Suite Y|Florence|Richland County|ND|53394|United States|-6|single family| +44322|AAAAAAAACCNKAAAA|589|Highland Sunset|ST|Suite 60|Walnut Grove|Throckmorton County|TX|77752|United States|-6|single family| +44323|AAAAAAAADCNKAAAA|412|River Madison|Avenue|Suite N|Providence|Elbert County|GA|36614|United States|-5|condo| +44324|AAAAAAAAECNKAAAA|116|7th 14th|Dr.|Suite 180|Woodville|Monroe County|WV|24289|United States|-5|apartment| +44325|AAAAAAAAFCNKAAAA|973|Center |Blvd|Suite Y|Red Hill|Big Horn County|WY|84338|United States|-7|apartment| +44326|AAAAAAAAGCNKAAAA|262|South View|Dr.|Suite U|Clearview|Windham County|VT|06095|United States|-5|apartment| +44327|AAAAAAAAHCNKAAAA|||Way|||Middlesex County|MA|07467|United States|-5|| +44328|AAAAAAAAICNKAAAA|540|Ash |Parkway|Suite H|Wildwood|Montgomery County|NC|26871|United States|-5|condo| +44329|AAAAAAAAJCNKAAAA|173|Jackson Lakeview|Dr.|Suite 40|Riverdale|McHenry County|IL|69391|United States|-6|single family| +44330|AAAAAAAAKCNKAAAA|796|Central Lee|Street|Suite 50|Riverside|Honolulu County|HI|99231|United States|-10|condo| +44331|AAAAAAAALCNKAAAA|898|Madison Hill|Drive|Suite 300|Lebanon|Lincoln County|MS|52898|United States|-6|single family| +44332|AAAAAAAAMCNKAAAA|110|Chestnut |Ln|Suite 140|Antioch|Davison County|SD|58605|United States|-6|condo| +44333|AAAAAAAANCNKAAAA|||Lane|Suite 470|Vance|Camp County||||-6|| +44334|AAAAAAAAOCNKAAAA||||Suite 290|||NJ||||single family| +44335|AAAAAAAAPCNKAAAA|336|Williams |Lane|Suite S|Summit|Cass County|TX|70499|United States|-6|condo| +44336|AAAAAAAAADNKAAAA|52|Wilson Pine|Dr.|Suite A|Stewart|Umatilla County|OR|98041|United States|-8|condo| +44337|AAAAAAAABDNKAAAA|484|7th |Dr.|Suite R|Jackson|Larue County|KY|49583|United States|-5|condo| +44338|AAAAAAAACDNKAAAA|636|Meadow |Lane|Suite O|Leland|Washington County|ME|09452|United States|-5|condo| +44339|AAAAAAAADDNKAAAA|467|Lee Adams|ST|Suite 230|Howell|Carter County|KY|44854|United States|-6|condo| +44340|AAAAAAAAEDNKAAAA|178|1st |Avenue|Suite 90|Arlington|Seneca County|OH|46557|United States|-5|single family| +44341|AAAAAAAAFDNKAAAA|824|Second 12th|Street|Suite 300|Leland|Hickman County|TN|39452|United States|-5|condo| +44342|AAAAAAAAGDNKAAAA|959|College |Street|Suite 30|Oakland|Olmsted County|MN|59843|United States|-6|apartment| +44343|AAAAAAAAHDNKAAAA|423|Park |Avenue|Suite W|Florence|Laclede County|MO|63394|United States|-6|condo| +44344|AAAAAAAAIDNKAAAA|535|Maple Madison|Blvd|Suite X|Lakeview|Custer County|ID|88579|United States|-7|condo| +44345|AAAAAAAAJDNKAAAA|60|Washington |Street|Suite 360|Clifton|Pushmataha County|OK|78014|United States|-6|condo| +44346|AAAAAAAAKDNKAAAA|114|Woodland Washington||Suite 170||||49391|United States||| +44347|AAAAAAAALDNKAAAA|136|Maple Poplar|Parkway|Suite 180|Freeman|Russell County|VA|22297|United States|-5|condo| +44348|AAAAAAAAMDNKAAAA|334|South |Parkway|Suite 220|Concord|Chelan County|WA|94107|United States|-8|condo| +44349|AAAAAAAANDNKAAAA|670|Pine South|Boulevard|Suite 380|Waterloo|Shelby County|MO|61675|United States|-6|apartment| +44350|AAAAAAAAODNKAAAA|717|4th |Ct.|Suite O|Lakeside|Jefferson County|MO|69532|United States|-6|condo| +44351|AAAAAAAAPDNKAAAA|841|Lake 3rd|Dr.|Suite 210|Newtown|Tift County|GA|31749|United States|-5|apartment| +44352|AAAAAAAAAENKAAAA|280|6th Main|Avenue|Suite 230|Pleasant Grove|Madison Parish|LA|74136|United States|-6|apartment| +44353|AAAAAAAABENKAAAA|752|Church Sycamore|Lane|Suite 390|Fairview|Butler County|AL|35709|United States|-6|condo| +44354|AAAAAAAACENKAAAA|849|14th |Parkway|Suite 120|Enterprise|Lowndes County|AL|31757|United States|-6|condo| +44355|AAAAAAAADENKAAAA|20|Davis Maple|Dr.|Suite 440|Fairfield|Comanche County|TX|76192|United States|-6|single family| +44356|AAAAAAAAEENKAAAA||Poplar ||Suite 110|Franklin|Hardin County|||United States||apartment| +44357|AAAAAAAAFENKAAAA|665|Maple Main|Ln|Suite V|Valley View|Berrien County|GA|35124|United States|-5|single family| +44358|AAAAAAAAGENKAAAA|508|View Hill|Lane|Suite 300|Brookwood|Garfield County|MT|60965|United States|-7|apartment| +44359|AAAAAAAAHENKAAAA|241|Maple |Ave|Suite 440|Jamestown|Graves County|KY|46867|United States|-6|condo| +44360|AAAAAAAAIENKAAAA|72|1st Meadow|Avenue|Suite 60|Highland Park|Toombs County|GA|36534|United States|-5|condo| +44361|AAAAAAAAJENKAAAA|569|Pine |Ln|Suite X|Cedar Grove|Josephine County|OR|90411|United States|-8|apartment| +44362|AAAAAAAAKENKAAAA|260|Willow |Parkway|Suite 330|Midway|Payette County|ID|81904|United States|-7|apartment| +44363|AAAAAAAALENKAAAA||||||Barnstable County||05575||-5|| +44364|AAAAAAAAMENKAAAA|820|Hickory |RD|Suite P|Mount Olive|Gaston County|NC|28059|United States|-5|single family| +44365|AAAAAAAANENKAAAA|728|Spring |Way|Suite W|Woodlawn|Kanawha County|WV|24098|United States|-5|condo| +44366|AAAAAAAAOENKAAAA|959|Lake Madison|Ave|Suite 280|Macedonia|Middlesex County|NJ|01687|United States|-5|apartment| +44367|AAAAAAAAPENKAAAA|72|Cedar 2nd|RD|Suite 410|Brownsville|Randolph County|WV|29310|United States|-5|apartment| +44368|AAAAAAAAAFNKAAAA|825|Church |ST|Suite 320|Red Hill|Marshall County|IN|44338|United States|-5|single family| +44369|AAAAAAAABFNKAAAA|343|Ridge |Dr.|Suite F|Willow|Boyle County|KY|46798|United States|-6|single family| +44370|AAAAAAAACFNKAAAA|607|Park |Blvd|Suite R|Colfax|Grainger County|TN|32565|United States|-5|apartment| +44371|AAAAAAAADFNKAAAA|110|Hickory |Lane|Suite 380|Lake Forest|Sullivan County|TN|36000|United States|-6|single family| +44372|AAAAAAAAEFNKAAAA|541|Smith |ST|Suite 330|Kelly|Craven County|NC|22738|United States|-5|apartment| +44373|AAAAAAAAFFNKAAAA|607|South Washington|Pkwy|Suite 180|Newport|Chester County|TN|31521|United States|-5|apartment| +44374|AAAAAAAAGFNKAAAA|941|Ridge |Boulevard|Suite L|Lincoln|Appanoose County|IA|51289|United States|-6|single family| +44375|AAAAAAAAHFNKAAAA|992|Park |Lane|Suite 350|New Hope|Fisher County|TX|79431|United States|-6|apartment| +44376|AAAAAAAAIFNKAAAA|||||Owens|Monroe County|IN|42324|United States||| +44377|AAAAAAAAJFNKAAAA|416|2nd Ash|Lane|Suite C|Warwick|Lucas County|IA|51398|United States|-6|apartment| +44378|AAAAAAAAKFNKAAAA|579|Laurel |Dr.|Suite G|Galena|Schuyler County|NY|14369|United States|-5|apartment| +44379|AAAAAAAALFNKAAAA|590|6th East|Parkway|Suite C|Woodville|Saginaw County|MI|44289|United States|-5|single family| +44380|AAAAAAAAMFNKAAAA|126|Mill Davis|Circle|Suite 50|Pine Grove|Pacific County|WA|94593|United States|-8|single family| +44381|AAAAAAAANFNKAAAA|496|View Maple|Lane|Suite I|Greenwood|George County|MS|58828|United States|-6|apartment| +44382|AAAAAAAAOFNKAAAA|858|Park Central|Cir.|Suite 10|Cedar Grove|Leflore County|MS|50411|United States|-6|condo| +44383|AAAAAAAAPFNKAAAA|797|Third Broadway|Lane|Suite 460|Wilson|Frederick County|VA|26971|United States|-5|single family| +44384|AAAAAAAAAGNKAAAA|148|Cedar Franklin|Pkwy|Suite G|Georgetown|Rio Arriba County|NM|87057|United States|-7|condo| +44385|AAAAAAAABGNKAAAA|67|College |Ct.|Suite F|Maple Grove|Pacific County|WA|98252|United States|-8|condo| +44386|AAAAAAAACGNKAAAA||Poplar |ST|Suite 370|||||United States|-5|single family| +44387|AAAAAAAADGNKAAAA|156|Spring 8th|Circle|Suite C|Union Hill|Barbour County|AL|37746|United States|-6|apartment| +44388|AAAAAAAAEGNKAAAA|193|13th West|Parkway|Suite 40|Wilson|Barber County|KS|66971|United States|-6|apartment| +44389|AAAAAAAAFGNKAAAA|289|Spring Meadow|Boulevard|Suite 270|Buena Vista|Greene County|OH|45752|United States|-5|condo| +44390|AAAAAAAAGGNKAAAA|429|13th Fifth|Court|Suite 450|Wilson|Avoyelles Parish|LA|76971|United States|-6|apartment| +44391|AAAAAAAAHGNKAAAA||Main Park|||||NY|16787|United States|-5|condo| +44392|AAAAAAAAIGNKAAAA|551|7th |Road|Suite 220|Hamilton|Greenville County|SC|22808|United States|-5|condo| +44393|AAAAAAAAJGNKAAAA|952|Hill |Dr.|Suite 240|Dewey|Andrew County|MO|61160|United States|-6|apartment| +44394|AAAAAAAAKGNKAAAA|276|15th Broadway|Street|Suite 210|Deerfield|Arlington County|VA|29840|United States|-5|apartment| +44395|AAAAAAAALGNKAAAA|421|Lake Elm|Road|Suite K|Newport|San Miguel County|NM|81521|United States|-7|single family| +44396|AAAAAAAAMGNKAAAA|431|Elm |Ln|Suite 10|Liberty|Moffat County|CO|83451|United States|-7|condo| +44397|AAAAAAAANGNKAAAA|||Boulevard|Suite W||||31387|United States||| +44398|AAAAAAAAOGNKAAAA|209|6th |ST|Suite Y|Mount Olive|Carbon County|PA|18059|United States|-5|single family| +44399|AAAAAAAAPGNKAAAA|659|Jackson Second|Ct.|Suite 200|Jericho|Oglethorpe County|GA|39778|United States|-5|single family| +44400|AAAAAAAAAHNKAAAA|444|Park Green|Ave|Suite 200|Montpelier|Young County|TX|78930|United States|-6|single family| +44401|AAAAAAAABHNKAAAA|727|Jackson |ST|Suite O|Thompsonville|Park County|WY|89651|United States|-7|apartment| +44402|AAAAAAAACHNKAAAA|996|Oak |Road|Suite 100|Friendship|Webster County|GA|34536|United States|-5|single family| +44403|AAAAAAAADHNKAAAA|538|Walnut View|Ct.|Suite L|Mount Olive|Marathon County|WI|58059|United States|-6|single family| +44404|AAAAAAAAEHNKAAAA|610|Lincoln 2nd|Dr.|Suite J|Five Forks|Dawson County|GA|32293|United States|-5|single family| +44405|AAAAAAAAFHNKAAAA|349|Oak |ST|Suite 250|Centerville|Clackamas County|OR|90059|United States|-8|apartment| +44406|AAAAAAAAGHNKAAAA|735|Woodland |Parkway|Suite N|Farmington|Kane County|IL|69145|United States|-6|single family| +44407|AAAAAAAAHHNKAAAA||Hill Pine|Parkway|Suite 310|||||United States||| +44408|AAAAAAAAIHNKAAAA|600|Lakeview Walnut|Cir.|Suite E|Antioch|Harrison County|TX|78605|United States|-6|apartment| +44409|AAAAAAAAJHNKAAAA|406|Third |Dr.|Suite 470|Valley View|Dauphin County|PA|15124|United States|-5|condo| +44410|AAAAAAAAKHNKAAAA|282|Ninth |Cir.|Suite 70|Buena Vista|Franklin County|IA|55752|United States|-6|condo| +44411|AAAAAAAALHNKAAAA|97|Chestnut |Wy|Suite N|Mountain View|Mississippi County|AR|74466|United States|-6|condo| +44412|AAAAAAAAMHNKAAAA|926|12th |Court|Suite V|Pine Grove|Tolland County|CT|05193|United States|-5|single family| +44413|AAAAAAAANHNKAAAA|305|Birch |Court|Suite Q|Fairfield|Madison County|AR|76192|United States|-6|condo| +44414|AAAAAAAAOHNKAAAA|582|East |Cir.|Suite 440|Jerome|Garfield County|UT|89920|United States|-7|apartment| +44415|AAAAAAAAPHNKAAAA|126|Forest 5th|Street|Suite 210|Jamestown|Grand Traverse County|MI|46867|United States|-5|condo| +44416|AAAAAAAAAINKAAAA|151|Jackson |Pkwy|Suite 480|Greenville|Northumberland County|PA|11387|United States|-5|apartment| +44417|AAAAAAAABINKAAAA|434|Washington |RD|Suite X|Macedonia|Tehama County|CA|91087|United States|-8|apartment| +44418|AAAAAAAACINKAAAA|649|Mill Ridge|Pkwy|Suite 380|Bunker Hill|Hardin County|KY|40150|United States|-6|apartment| +44419|AAAAAAAADINKAAAA|888|Park Williams|Parkway|Suite D|Oakwood|Lincoln County|WY|80169|United States|-7|apartment| +44420|AAAAAAAAEINKAAAA|332|Pine Johnson|Circle|Suite U|Prosperity|Sevier County|AR|79089|United States|-6|single family| +44421|AAAAAAAAFINKAAAA|225|Franklin First|Avenue|Suite R|Unionville|Dare County|NC|21711|United States|-5|condo| +44422|AAAAAAAAGINKAAAA|874|Third Woodland|RD|Suite F|Greenville|Lackawanna County|PA|11387|United States|-5|single family| +44423|AAAAAAAAHINKAAAA|780|Lake |Street|Suite 140|Springdale|Cumberland County|IL|68883|United States|-6|apartment| +44424|AAAAAAAAIINKAAAA|578|Hickory Chestnut|Wy|Suite 240|Greenville|Fairfield County|OH|41387|United States|-5|condo| +44425|AAAAAAAAJINKAAAA|616|Smith |Dr.|Suite D|Shady Grove|Hillsborough County|FL|32812|United States|-5|condo| +44426|AAAAAAAAKINKAAAA|148|Park Sunset|Wy|Suite F|Newport|Hanson County|SD|51521|United States|-7|condo| +44427|AAAAAAAALINKAAAA|205|North |||Pleasant Hill|Barrow County|||United States|-5|| +44428|AAAAAAAAMINKAAAA|737|Jackson |Ln|Suite K|Fairfield|Hardin County|OH|46192|United States|-5|single family| +44429|AAAAAAAANINKAAAA|469|Madison Laurel|Way|Suite J|Woodlawn|Nowata County|OK|74098|United States|-6|apartment| +44430|AAAAAAAAOINKAAAA|229|Williams |Parkway|Suite P|Farmington|Winn Parish|LA|79145|United States|-6|single family| +44431|AAAAAAAAPINKAAAA|987|Mill College|Road|Suite T|Lakeview|Kingfisher County|OK|78579|United States|-6|single family| +44432|AAAAAAAAAJNKAAAA|184|Hillcrest 2nd|Circle|Suite 270|Franklin|Maverick County|TX|79101|United States|-6|single family| +44433|AAAAAAAABJNKAAAA|779|Seventh |RD|Suite C|Woodlawn|Ottawa County|KS|64098|United States|-6|condo| +44434|AAAAAAAACJNKAAAA|93|View |Ln|Suite 190|Hopewell|Cottonwood County|MN|50587|United States|-6|apartment| +44435|AAAAAAAADJNKAAAA|212|Ash |Parkway|Suite 320|Florence|Reynolds County|MO|63394|United States|-6|condo| +44436|AAAAAAAAEJNKAAAA|109|Fourth |ST|Suite 470|Mount Vernon|Magoffin County|KY|48482|United States|-5|single family| +44437|AAAAAAAAFJNKAAAA|294|Madison Park|ST|Suite 480|Edwards|Orleans Parish|LA|71409|United States|-6|condo| +44438|AAAAAAAAGJNKAAAA|668|2nd |Road|Suite 170|Green Acres|Isle of Wight County|VA|27683|United States|-5|condo| +44439|AAAAAAAAHJNKAAAA|272|Forest |Boulevard|Suite P|Royal|Dillingham Census Area|AK|95819|United States|-9|condo| +44440|AAAAAAAAIJNKAAAA|441|Elevnth |Boulevard|Suite Q|Ashland|Franklin County|KS|64244|United States|-6|condo| +44441|AAAAAAAAJJNKAAAA|480|South Ninth|Avenue|Suite 370|Harmony|Essex County|MA|06404|United States|-5|condo| +44442|AAAAAAAAKJNKAAAA|754|Central Jefferson|Pkwy|Suite 240|Ludlow|Carroll County|VA|25566|United States|-5|single family| +44443|AAAAAAAALJNKAAAA|184|Ninth Mill|ST|Suite J|Pleasant Valley|Benewah County|ID|82477|United States|-7|single family| +44444|AAAAAAAAMJNKAAAA|838|Elm |Wy|Suite 30|Lake Forest|Atchison County|MO|66000|United States|-6|condo| +44445|AAAAAAAANJNKAAAA|668|3rd |Blvd|Suite 70|Belmont|Loup County|NE|60191|United States|-7|single family| +44446|AAAAAAAAOJNKAAAA|379|East Walnut|Drive|Suite 60|Union Hill|Pitt County|NC|27746|United States|-5|single family| +44447|AAAAAAAAPJNKAAAA||Lake Willow||||||34854|United States||condo| +44448|AAAAAAAAAKNKAAAA|981|Fourth Madison|Blvd|Suite N|Mechanicsburg|Northampton County|PA|12219|United States|-5|single family| +44449|AAAAAAAABKNKAAAA|909|Ridge |Blvd|Suite A|Fairview|Warren County|MS|55709|United States|-6|single family| +44450|AAAAAAAACKNKAAAA|527|North |Court|Suite 400|Kingston|Winston County|MS|54975|United States|-6|condo| +44451|AAAAAAAADKNKAAAA|524|First Chestnut|Way|Suite 230|Waterloo|Claiborne County|TN|31675|United States|-5|apartment| +44452|AAAAAAAAEKNKAAAA|548|Willow |Pkwy|Suite 230|Mount Vernon|Menard County|IL|68482|United States|-6|condo| +44453|AAAAAAAAFKNKAAAA|565|Spruce |Ave|Suite M|Hamilton|Stephens County|OK|72808|United States|-6|single family| +44454|AAAAAAAAGKNKAAAA|764|Dogwood |RD|Suite U|Burns|Holmes County|MS|55272|United States|-6|condo| +44455|AAAAAAAAHKNKAAAA|930|7th 5th|Ln|Suite 0|Fairview|Horry County|SC|25709|United States|-5|condo| +44456|AAAAAAAAIKNKAAAA|731|Poplar |Court|Suite X|Deerfield|Wayne County|NC|29840|United States|-5|single family| +44457|AAAAAAAAJKNKAAAA|352|Park |Way|Suite 340|Pleasant Grove|Macon County|GA|34136|United States|-5|apartment| +44458|AAAAAAAAKKNKAAAA|264|Cedar Spring|Avenue|Suite 390|Florence|Ellis County|TX|73394|United States|-6|condo| +44459|AAAAAAAALKNKAAAA|898|Johnson Railroad|Dr.|Suite J|Friendship|Wyandot County|OH|44536|United States|-5|apartment| +44460|AAAAAAAAMKNKAAAA|17|14th |Ln|Suite C|New Hope|Chilton County|AL|39431|United States|-6|single family| +44461|AAAAAAAANKNKAAAA|739|Lake Miller|Circle|Suite 140|Lakewood|Floyd County|VA|28877|United States|-5|apartment| +44462|AAAAAAAAOKNKAAAA|547|Johnson Park|Dr.|||||||-8|single family| +44463|AAAAAAAAPKNKAAAA|44|Willow Main|Road|Suite X|Enterprise|Harney County|OR|91757|United States|-8|apartment| +44464|AAAAAAAAALNKAAAA|15|Park |Pkwy|Suite 20|Friendship|Washington County|VA|24536|United States|-5|single family| +44465|AAAAAAAABLNKAAAA|440|7th |Wy|Suite 430|Allentown|Jasper County|MS|51838|United States|-6|single family| +44466|AAAAAAAACLNKAAAA|840|Miller 1st|Boulevard|Suite L|Lakeview|Addison County|VT|09179|United States|-5|condo| +44467|AAAAAAAADLNKAAAA|267|Sunset |Lane|Suite 310|Deerfield|Lyon County|KY|49840|United States|-5|apartment| +44468|AAAAAAAAELNKAAAA|702|Oak 8th|Blvd|Suite E|Cedar Grove|Anson County|NC|20411|United States|-5|single family| +44469|AAAAAAAAFLNKAAAA|376|5th 3rd|Ave|Suite 430|Valley View|Hancock County|IN|45124|United States|-5|condo| +44470|AAAAAAAAGLNKAAAA|586|Maple |Road|Suite 250|Fairview|Williamson County|TX|75709|United States|-6|apartment| +44471|AAAAAAAAHLNKAAAA|||Ln||Greenville|||61387|||condo| +44472|AAAAAAAAILNKAAAA|43|3rd Third|Street|Suite T|Five Points|DuPage County|IL|66098|United States|-6|apartment| +44473|AAAAAAAAJLNKAAAA|564|Central Sunset|Wy|Suite 20|Leon|Wythe County|VA|20913|United States|-5|single family| +44474|AAAAAAAAKLNKAAAA|896|Willow |Court|Suite 440|Arlington|San Francisco County|CA|96557|United States|-8|condo| +44475|AAAAAAAALLNKAAAA|455|2nd 9th|Road|Suite V|Greenwood|McCone County|MT|68828|United States|-7|condo| +44476|AAAAAAAAMLNKAAAA|298|6th 11th|Blvd|Suite 440|Jackson|Leon County|FL|39583|United States|-5|single family| +44477|AAAAAAAANLNKAAAA|263|Center |Boulevard|Suite 190|Buena Vista|Ramsey County|MN|55752|United States|-6|condo| +44478|AAAAAAAAOLNKAAAA|907|Smith College|Avenue|Suite W|Doyle|Lenoir County|NC|28434|United States|-5|apartment| +44479|AAAAAAAAPLNKAAAA|158|Woodland West|Lane|Suite 390|Union|Laurens County|GA|38721|United States|-5|condo| +44480|AAAAAAAAAMNKAAAA|703|Lakeview 6th|RD|Suite D|Union Hill|Gage County|NE|67746|United States|-6|condo| +44481|AAAAAAAABMNKAAAA|612|Lakeview 14th|Ave|Suite 100|Springdale|Carroll County|IL|68883|United States|-6|condo| +44482|AAAAAAAACMNKAAAA|730|Williams |Drive|Suite 290|Clinton|Lycoming County|PA|18222|United States|-5|condo| +44483|AAAAAAAADMNKAAAA|565|Hill Elm|ST|Suite 250|Greenwood|Vermilion County|IL|68828|United States|-6|condo| +44484|AAAAAAAAEMNKAAAA|208|3rd |Ct.|Suite 60|Jackson|Madison County|TN|39583|United States|-6|single family| +44485|AAAAAAAAFMNKAAAA|115|1st |ST|Suite W|Valley View|Heard County|GA|35124|United States|-5|condo| +44486|AAAAAAAAGMNKAAAA|730|Washington 8th|Parkway|Suite 460|Florence|Mecosta County|MI|43394|United States|-5|single family| +44487|AAAAAAAAHMNKAAAA|826|Walnut |Parkway|Suite W|Green Acres|Towner County|ND|57683|United States|-6|condo| +44488|AAAAAAAAIMNKAAAA|399|11th |Boulevard|Suite F|Lebanon|Warren County|IA|52898|United States|-6|apartment| +44489|AAAAAAAAJMNKAAAA|172|Miller |Ave|Suite 410|Riverside|Franklin County|KY|49231|United States|-6|apartment| +44490|AAAAAAAAKMNKAAAA|472|Third Birch|ST|Suite I|Green Acres|Wayne County|MI|47683|United States|-5|condo| +44491|AAAAAAAALMNKAAAA|78|Spring |Court|Suite R|Sunnyside|Decatur County|IN|41952|United States|-5|single family| +44492|AAAAAAAAMMNKAAAA|920|Park |Avenue|Suite X|Mount Pleasant|Fairbanks North Star Borough|AK|91933|United States|-9|single family| +44493|AAAAAAAANMNKAAAA|875||||Hillcrest|||23003|United States|-5|single family| +44494|AAAAAAAAOMNKAAAA|554|College River|Parkway|Suite A|Bethel|Spokane County|WA|95281|United States|-8|apartment| +44495|AAAAAAAAPMNKAAAA|578|Sunset |Ct.|Suite S|Red Hill|Marathon County|WI|54338|United States|-6|condo| +44496|AAAAAAAAANNKAAAA|167|Sunset |Wy|Suite Y|Mount Vernon|Keokuk County|IA|58482|United States|-6|apartment| +44497|AAAAAAAABNNKAAAA|326|Washington Cedar|Lane|Suite C|Unionville|Holt County|NE|61711|United States|-7|single family| +44498|AAAAAAAACNNKAAAA|903|6th Walnut|Pkwy|Suite H|Stafford|Malheur County|OR|94980|United States|-8|single family| +44499|AAAAAAAADNNKAAAA|615|Walnut North|ST|Suite 400|Oak Ridge|Santa Cruz County|AZ|88371|United States|-7|apartment| +44500|AAAAAAAAENNKAAAA|133|2nd Cedar|Avenue|Suite 320|Oakwood|Montrose County|CO|80169|United States|-7|single family| +44501|AAAAAAAAFNNKAAAA|702|Pine 1st|Circle|Suite M|Andover|Dundy County|NE|61639|United States|-6|single family| +44502|AAAAAAAAGNNKAAAA|19|North 1st|Avenue|Suite I|Silver Springs|Shelby County|IA|54843|United States|-6|condo| +44503|AAAAAAAAHNNKAAAA|746|Central Johnson|Ln|Suite 100|Highland Park|Laurens County|GA|36534|United States|-5|single family| +44504|AAAAAAAAINNKAAAA|468|Cedar |Circle|Suite 120|Woodland|Grundy County|IL|64854|United States|-6|apartment| +44505|AAAAAAAAJNNKAAAA|935|Johnson 4th|Cir.|Suite 450|Clifton|Raleigh County|WV|28014|United States|-5|single family| +44506|AAAAAAAAKNNKAAAA|267|Pine |Ave|Suite 230|Newtown|Troup County|GA|31749|United States|-5|apartment| +44507|AAAAAAAALNNKAAAA|339|12th |Blvd|Suite S|Hamilton|Banks County|GA|32808|United States|-5|condo| +44508|AAAAAAAAMNNKAAAA|257|Walnut Washington|Ln|Suite 330|Walnut Grove|Hodgeman County|KS|67752|United States|-6|single family| +44509|AAAAAAAANNNKAAAA|181|Jefferson |Wy|Suite E|Cedar Grove|Van Buren County|AR|70411|United States|-6|condo| +44510|AAAAAAAAONNKAAAA|261|Ash Cedar|Dr.|Suite 280|Wildwood|Seward County|KS|66871|United States|-6|condo| +44511|AAAAAAAAPNNKAAAA||6th |Lane|||Warren County||37683||-5|single family| +44512|AAAAAAAAAONKAAAA|542|Fifth |Parkway|Suite 10|Antioch|Audrain County|MO|68605|United States|-6|condo| +44513|AAAAAAAABONKAAAA|299|Adams Sixth|Dr.|Suite K|Paxton|Pope County|AR|75669|United States|-6|condo| +44514|AAAAAAAACONKAAAA|948|North |Boulevard|Suite H|Centerville|York County|VA|20059|United States|-5|single family| +44515|AAAAAAAADONKAAAA|617|College |Way|Suite I|Buena Vista|Sussex County|VA|25752|United States|-5|single family| +44516|AAAAAAAAEONKAAAA|796|2nd |Ct.|Suite R|Center|Marion County|AR|78519|United States|-6|apartment| +44517|AAAAAAAAFONKAAAA|825|Lee Maple|Boulevard|Suite W|Pleasant Hill|Osceola County|FL|33604|United States|-5|single family| +44518|AAAAAAAAGONKAAAA|886|Cedar |Parkway|Suite O|Newport|Jackson County|NC|21521|United States|-5|single family| +44519|AAAAAAAAHONKAAAA|49|Willow Poplar|Pkwy|Suite T|Lincoln|Copiah County|MS|51289|United States|-6|single family| +44520|AAAAAAAAIONKAAAA|545|Pine |Boulevard|Suite X|Union Hill|Warren County|MO|67746|United States|-6|apartment| +44521|AAAAAAAAJONKAAAA|557|Mill |Blvd|Suite 220|Newport|Will County|IL|61521|United States|-6|apartment| +44522|AAAAAAAAKONKAAAA|100|9th 1st|Road|Suite G|Salem|Madison County|IN|48048|United States|-5|single family| +44523|AAAAAAAALONKAAAA|161|Elm Fifth|Cir.|Suite 320|Hillcrest|Howard County|MO|63003|United States|-6|condo| +44524|AAAAAAAAMONKAAAA|623|Railroad Tenth|Pkwy|Suite K|Highland Park|Morrow County|OR|96534|United States|-8|condo| +44525|AAAAAAAANONKAAAA|190|View 4th|Circle|Suite S|Newport|Marshall County|AL|31521|United States|-6|condo| +44526|AAAAAAAAOONKAAAA|783|Hickory Pine|Avenue|Suite 20|Union Hill|Sitka Borough|AK|97746|United States|-9|apartment| +44527|AAAAAAAAPONKAAAA|414|View Spring|ST|Suite F|Riverdale|Jefferson County|NY|19391|United States|-5|apartment| +44528|AAAAAAAAAPNKAAAA||Lincoln Broadway|Circle||||OH|||-5|| +44529|AAAAAAAABPNKAAAA|241|Fourth Third|Way|Suite 210|Shiloh|Berks County|PA|19275|United States|-5|single family| +44530|AAAAAAAACPNKAAAA|138|5th Cherry|Road|Suite 480|Wilson|Saguache County|CO|86971|United States|-7|apartment| +44531|AAAAAAAADPNKAAAA|272|East Willow|Ct.|Suite P|Riverdale|Chenango County|NY|19391|United States|-5|apartment| +44532|AAAAAAAAEPNKAAAA|585|Park |Ct.|Suite J|Bridgeport|Bulloch County|GA|35817|United States|-5|single family| +44533|AAAAAAAAFPNKAAAA|478|Cedar |Pkwy|Suite L|Newtown|Edgefield County|SC|21749|United States|-5|single family| +44534|AAAAAAAAGPNKAAAA|703|Spring Church|Ln|Suite 470|Shaw|Hardeman County|TX|70618|United States|-6|single family| +44535|AAAAAAAAHPNKAAAA|975|2nd |Wy|Suite 0|Sunnyside|Frederick County|VA|21952|United States|-5|condo| +44536|AAAAAAAAIPNKAAAA|290|2nd Main|Street|Suite 20|Cedar Grove|Lincoln County|TN|30411|United States|-6|condo| +44537|AAAAAAAAJPNKAAAA|863|Birch |Lane|Suite W|Unionville|Yellowstone County|MT|61711|United States|-7|apartment| +44538|AAAAAAAAKPNKAAAA|49|Central Woodland|Blvd|Suite 170|Wildwood|Coke County|TX|76871|United States|-6|single family| +44539|AAAAAAAALPNKAAAA|865|8th Hillcrest|Circle|Suite D|Vance|Surry County|VA|20268|United States|-5|apartment| +44540|AAAAAAAAMPNKAAAA|553|10th Maple|Blvd|Suite 390|Mount Olive|Washington County|NC|28059|United States|-5|apartment| +44541|AAAAAAAANPNKAAAA|618|Eigth Walnut|Parkway|Suite 230|Glenwood|Oklahoma County|OK|73511|United States|-6|apartment| +44542|AAAAAAAAOPNKAAAA|316|1st 13th|ST|Suite D|Harmony|Pottawatomie County|OK|75804|United States|-6|apartment| +44543|AAAAAAAAPPNKAAAA||Walnut Hill|Dr.|||Marshall County||43003|United States||single family| +44544|AAAAAAAAAAOKAAAA|57|Smith |Ct.|Suite B|Jacksonville|Hanson County|SD|58223|United States|-7|condo| +44545|AAAAAAAABAOKAAAA|746|10th |Court|Suite U|Jones|Dawson County|GA|32686|United States|-5|condo| +44546|AAAAAAAACAOKAAAA|188|River |Ln|Suite Y|Glenwood|Newton County|IN|43511|United States|-5|condo| +44547|AAAAAAAADAOKAAAA|124|Washington Center|Dr.|Suite 300|White Oak|Lake County|CA|96668|United States|-8|single family| +44548|AAAAAAAAEAOKAAAA|909|Elm |Street|Suite 350|Peru|Flathead County|MT|60302|United States|-7|condo| +44549|AAAAAAAAFAOKAAAA|17|Forest |Lane|Suite 270|Five Forks|Comanche County|OK|72293|United States|-6|condo| +44550|AAAAAAAAGAOKAAAA|111|Central Fifth|Court|Suite 450|Lakeview|Copiah County|MS|58579|United States|-6|condo| +44551|AAAAAAAAHAOKAAAA|32|Pine Ninth|Court|Suite O|Fairview|Montgomery County|IL|65709|United States|-6|single family| +44552|AAAAAAAAIAOKAAAA|95|View North|Road|Suite M|Pleasant Grove|Reagan County|TX|74136|United States|-6|condo| +44553|AAAAAAAAJAOKAAAA|670|Hill Walnut|Drive|Suite U|Marion|Horry County|SC|20399|United States|-5|single family| +44554|AAAAAAAAKAOKAAAA|175|Lake 1st|Road|Suite G|Friendship|Osage County|KS|64536|United States|-6|condo| +44555|AAAAAAAALAOKAAAA|701|3rd |Cir.|Suite O|Enterprise|Habersham County|GA|31757|United States|-5|apartment| +44556|AAAAAAAAMAOKAAAA|313|Broadway Poplar|Circle|Suite 370|Clifton|Torrance County|NM|88014|United States|-7|condo| +44557|AAAAAAAANAOKAAAA|792|Park |Road|Suite U|Pleasant Grove|Falls Church city|VA|24136|United States|-5|apartment| +44558|AAAAAAAAOAOKAAAA|187|Eigth Church|Ct.|Suite 80|Midway|Jasper County|MO|61904|United States|-6|condo| +44559|AAAAAAAAPAOKAAAA|68|Sixth |Avenue|Suite A|Oak Grove|Howard County|IN|48370|United States|-5|single family| +44560|AAAAAAAAABOKAAAA|273|Davis Locust|Blvd|Suite S|Kingston|Lincoln County|KS|64975|United States|-6|condo| +44561|AAAAAAAABBOKAAAA|358|Madison |Wy|Suite N|New Hope|Chase County|NE|69431|United States|-6|apartment| +44562|AAAAAAAACBOKAAAA|374|Jackson Oak|Cir.|Suite Y|Belmont|Warren County|KY|40191|United States|-5|single family| +44563|AAAAAAAADBOKAAAA|890|Meadow Spring|Cir.|Suite 250|Pomona|Hamilton County|NE|64153|United States|-6|single family| +44564|AAAAAAAAEBOKAAAA|767|Fifth |Avenue|Suite N|Oak Ridge|King William County|VA|28371|United States|-5|condo| +44565|AAAAAAAAFBOKAAAA|198|Main |Drive|Suite H|Hillcrest|Alcona County|MI|43003|United States|-5|condo| +44566|AAAAAAAAGBOKAAAA|782|Ash |Cir.|Suite X|Crossroads|Cameron County|PA|10534|United States|-5|apartment| +44567|AAAAAAAAHBOKAAAA|234|Eigth |Parkway|Suite 130|Springfield|Nye County|NV|89303|United States|-8|condo| +44568|AAAAAAAAIBOKAAAA|750|Madison |Drive|Suite 490|Five Points|Park County|WY|86098|United States|-7|apartment| +44569|AAAAAAAAJBOKAAAA|823|6th |Court|Suite P|Oakland|Bronx County|NY|19843|United States|-5|apartment| +44570|AAAAAAAAKBOKAAAA|587|Oak Green|Parkway|Suite R|Pleasant Hill|West Carroll Parish|LA|73604|United States|-6|single family| +44571|AAAAAAAALBOKAAAA|687|||Suite Y|||KS||||apartment| +44572|AAAAAAAAMBOKAAAA|84|Jefferson |Drive|Suite L|Hamilton|Gates County|NC|22808|United States|-5|single family| +44573|AAAAAAAANBOKAAAA|657|Mill |Circle|Suite G|Pleasant Grove|King George County|VA|24136|United States|-5|condo| +44574|AAAAAAAAOBOKAAAA|689|Green |Road|Suite 180|Belmont|Brown County|NE|60191|United States|-6|single family| +44575|AAAAAAAAPBOKAAAA|856|Cedar |Road|Suite 430|Newport|Oakland County|MI|41521|United States|-5|condo| +44576|AAAAAAAAACOKAAAA|942|Birch |Pkwy|Suite 220|Summit|Huntington County|IN|40499|United States|-5|apartment| +44577|AAAAAAAABCOKAAAA|711|Hickory |Ave|Suite O|Lebanon|Jefferson County|AR|72898|United States|-6|condo| +44578|AAAAAAAACCOKAAAA|225|9th Pine|Ave|Suite 290|Riverview|Benton County|MN|59003|United States|-6|condo| +44579|AAAAAAAADCOKAAAA|976|Third Adams|Ln|Suite 310|Hamilton|Broome County|NY|12808|United States|-5|condo| +44580|AAAAAAAAECOKAAAA|36|Central |Street|Suite Y|Riverdale|Elko County|NV|89391|United States|-8|single family| +44581|AAAAAAAAFCOKAAAA|545|Fifth |Street|Suite Y|New Hope|Adair County|MO|69431|United States|-6|single family| +44582|AAAAAAAAGCOKAAAA|642|4th Main|Ln|Suite E|Salem|Fairbanks North Star Borough|AK|98048|United States|-9|single family| +44583|AAAAAAAAHCOKAAAA|228|Madison 13th|Ln|Suite 140|Fairfield|Schley County|GA|36192|United States|-5|condo| +44584|AAAAAAAAICOKAAAA|905|2nd 8th|Road|Suite 300|Deerfield|Webster County|WV|29840|United States|-5|apartment| +44585|AAAAAAAAJCOKAAAA|205|9th Oak|Wy|Suite 160|Union|Herkimer County|NY|18721|United States|-5|condo| +44586|AAAAAAAAKCOKAAAA|562|5th Walnut|Road|Suite N|Spring Hill|Oswego County|NY|16787|United States|-5|condo| +44587|AAAAAAAALCOKAAAA|984|Spring Pine|Dr.|Suite T|Red Hill|Houston County|MN|54338|United States|-6|single family| +44588|AAAAAAAAMCOKAAAA|994|Birch View|Ct.|Suite K|Riverview|Dawson County|GA|39003|United States|-5|condo| +44589|AAAAAAAANCOKAAAA|43|Jefferson Woodland|Ln|Suite K|Union City|Honolulu County|HI|98087|United States|-10|apartment| +44590|AAAAAAAAOCOKAAAA|629|College Elm|Drive|Suite 180|Highland|Hopkins County|TX|79454|United States|-6|apartment| +44591|AAAAAAAAPCOKAAAA|194|9th |Blvd|Suite 410|Lakewood|Oldham County|KY|48877|United States|-5|apartment| +44592|AAAAAAAAADOKAAAA|232|13th Main|RD|Suite 330|Clifton|Lawrence County|IN|48014|United States|-5|single family| +44593|AAAAAAAABDOKAAAA|162|Lincoln |Ct.|Suite W|Lincoln|Crawford County|IL|61289|United States|-6|single family| +44594|AAAAAAAACDOKAAAA|697|Sunset |Blvd|Suite 240|Plainview|Trinity County|CA|93683|United States|-8|apartment| +44595|AAAAAAAADDOKAAAA|270|Cherry View|Way|Suite 430|Pinhook|Pipestone County|MN|59398|United States|-6|apartment| +44596|AAAAAAAAEDOKAAAA|692|Elm 3rd|Parkway|Suite 380|Edgewood|Litchfield County|CT|00669|United States|-5|single family| +44597|AAAAAAAAFDOKAAAA|216|Park |Ct.|Suite L|Lincoln|Valley County|NE|61289|United States|-7|single family| +44598|AAAAAAAAGDOKAAAA|674|Park Davis|Ave|Suite 440|Antioch|Monongalia County|WV|28605|United States|-5|condo| +44599|AAAAAAAAHDOKAAAA|985|East |Cir.|Suite Y|Jackson|Orange County|VT|09583|United States|-5|condo| +44600|AAAAAAAAIDOKAAAA|925|Mill Woodland|Street|Suite 460|Mount Zion|Winn Parish|LA|78054|United States|-6|single family| +44601|AAAAAAAAJDOKAAAA|453|South |Ave|Suite 470|Deerfield|Pulaski County|GA|39840|United States|-5|condo| +44602|AAAAAAAAKDOKAAAA|855|Walnut 3rd|Avenue|Suite V|Unionville|Halifax County|NC|21711|United States|-5|condo| +44603|AAAAAAAALDOKAAAA|562|Green Center|Ave|Suite P|Edgewood|Franklin County|OH|40069|United States|-5|apartment| +44604|AAAAAAAAMDOKAAAA|306|4th 2nd|Way|Suite 170|Glendale|Anderson County|KY|43951|United States|-6|apartment| +44605|AAAAAAAANDOKAAAA|354|Maple 2nd|Court|Suite 310|Georgetown|Marin County|CA|97057|United States|-8|condo| +44606|AAAAAAAAODOKAAAA|285|Washington Pine|Pkwy|Suite 170|Pleasant Grove|Seminole County|FL|34136|United States|-5|condo| +44607|AAAAAAAAPDOKAAAA|856|Central |Boulevard|Suite 300|Lakeside|Kossuth County|IA|59532|United States|-6|condo| +44608|AAAAAAAAAEOKAAAA|15|Third 4th|Ct.|Suite 410|Waterloo|Ramsey County|MN|51675|United States|-6|apartment| +44609|AAAAAAAABEOKAAAA||||Suite M|Centerville|Bee County|TX|||-6|| +44610|AAAAAAAACEOKAAAA|434|Park Sixth|Circle|Suite 0|Bunker Hill|Ellis County|TX|70150|United States|-6|condo| +44611|AAAAAAAADEOKAAAA|916|Wilson |Street|Suite 360|Bethel|Iberville Parish|LA|75281|United States|-6|single family| +44612|AAAAAAAAEEOKAAAA|71|14th |Avenue|Suite 270|Franklin|Wayne County|NE|69101|United States|-7|condo| +44613|AAAAAAAAFEOKAAAA|717|Ninth |Pkwy|Suite U|Bunker Hill|Hancock County|GA|30150|United States|-5|apartment| +44614|AAAAAAAAGEOKAAAA|742|Ridge Elm|Avenue|Suite V|Five Forks|Montgomery County|TX|72293|United States|-6|condo| +44615|AAAAAAAAHEOKAAAA|947|Maple Spring|Lane|Suite 160|Sunnyside|Nowata County|OK|71952|United States|-6|condo| +44616|AAAAAAAAIEOKAAAA|240|Franklin |RD|Suite 40|Woodland|Clinton County|IN|44854|United States|-5|single family| +44617|AAAAAAAAJEOKAAAA|230|Birch Railroad|Blvd|Suite X|Martinsville|Craig County|OK|70419|United States|-6|condo| +44618|AAAAAAAAKEOKAAAA|212|Main |ST|Suite W|Harmony|Oneida County|WI|55804|United States|-6|apartment| +44619|AAAAAAAALEOKAAAA|253|First |Ave|Suite W|Waterville|Oldham County|TX|74355|United States|-6|condo| +44620|AAAAAAAAMEOKAAAA|304|Sunset Lake|ST|Suite S|Salem|Garrett County|MD|28048|United States|-5|condo| +44621|AAAAAAAANEOKAAAA|292|Park |Cir.|Suite V|Hopewell|Clinton County|KY|40587|United States|-6|condo| +44622|AAAAAAAAOEOKAAAA|70|Eigth 14th|ST|Suite C|Pleasant Hill|McLean County|KY|43604|United States|-5|single family| +44623|AAAAAAAAPEOKAAAA|171|Jefferson Sunset|Blvd|Suite M|Macedonia|Lincoln County|NC|21087|United States|-5|apartment| +44624|AAAAAAAAAFOKAAAA|497|Willow Sixth|RD|Suite C|Clinton|Lancaster County|NE|68222|United States|-7|single family| +44625|AAAAAAAABFOKAAAA|51|Eigth |Parkway|Suite 10|Edgewater|Christian County|MO|60635|United States|-6|apartment| +44626|AAAAAAAACFOKAAAA|681|Cherry 2nd|Ave|Suite B|Fairfield|Hancock County|IL|66192|United States|-6|apartment| +44627|AAAAAAAADFOKAAAA|934|Park |Lane|Suite K|Lakewood|Muscogee County|GA|38877|United States|-5|condo| +44628|AAAAAAAAEFOKAAAA|315|Lincoln Birch|Circle|Suite 30|Antioch|Red Willow County|NE|68605|United States|-7|apartment| +44629|AAAAAAAAFFOKAAAA|440|Cherry |RD|Suite B|Midway|Jones County|SD|51904|United States|-7|single family| +44630|AAAAAAAAGFOKAAAA|118|14th |Boulevard|Suite 380|Bunker Hill|Overton County|TN|30150|United States|-6|apartment| +44631|AAAAAAAAHFOKAAAA|739|Woodland |Cir.|Suite 10|Oak Grove|Amite County|MS|58370|United States|-6|apartment| +44632|AAAAAAAAIFOKAAAA|320|Oak |Blvd|Suite J|Scottsville|Angelina County|TX|74190|United States|-6|single family| +44633|AAAAAAAAJFOKAAAA|596|Pine Washington|Court|Suite O|Oakland|Lauderdale County|MS|59843|United States|-6|apartment| +44634|AAAAAAAAKFOKAAAA|749|7th |Wy|Suite V|Spring Hill|Wheeler County|GA|36787|United States|-5|condo| +44635|AAAAAAAALFOKAAAA|928|Walnut First|Circle|Suite 210|Shiloh|Duplin County|NC|29275|United States|-5|condo| +44636|AAAAAAAAMFOKAAAA|162|Oak |Ct.|Suite M|Woodlawn|Clay County|AL|34098|United States|-6|condo| +44637|AAAAAAAANFOKAAAA|139|Cedar Park|Way|Suite 340|Bethel|Cambria County|PA|15281|United States|-5|condo| +44638|AAAAAAAAOFOKAAAA|643|10th |Court|Suite H|Lenox|Pershing County|NV|81143|United States|-8|apartment| +44639|AAAAAAAAPFOKAAAA|303|Mill Forest|Road|Suite 160|Centerville|Clermont County|OH|40059|United States|-5|single family| +44640|AAAAAAAAAGOKAAAA|297|Poplar Woodland|Ct.|Suite 380|Mount Pleasant|Washington County|UT|81933|United States|-7|single family| +44641|AAAAAAAABGOKAAAA|880|Birch |Court|Suite U|Mount Pleasant|Chippewa County|MI|41933|United States|-5|condo| +44642|AAAAAAAACGOKAAAA|861|Oak Valley|Pkwy|Suite 40|Ashland|Tattnall County|GA|34244|United States|-5|condo| +44643|AAAAAAAADGOKAAAA|188|Maple |Court|Suite 130|Bethel|Gladwin County|MI|45281|United States|-5|condo| +44644|AAAAAAAAEGOKAAAA|377|Third |Ave|Suite 190|Harmony|Bandera County|TX|75804|United States|-6|apartment| +44645|AAAAAAAAFGOKAAAA|682|North Hill|Pkwy|Suite 80|Greenfield|Butler County|KY|45038|United States|-6|apartment| +44646|AAAAAAAAGGOKAAAA|343|Hickory Cedar|Drive|Suite S|Woodville|Clay County|TX|74289|United States|-6|apartment| +44647|AAAAAAAAHGOKAAAA|149|12th Adams|Cir.|Suite 370|Midway|Platte County|MO|61904|United States|-6|apartment| +44648|AAAAAAAAIGOKAAAA|496|Chestnut |Boulevard|Suite 280|Arlington|Ramsey County|ND|56557|United States|-6|single family| +44649|AAAAAAAAJGOKAAAA|136|Cherry |Boulevard|Suite L|Fairview|Saguache County|CO|85709|United States|-7|condo| +44650|AAAAAAAAKGOKAAAA|151|11th Mill|Wy|Suite T|Mount Pleasant|McMinn County|TN|31933|United States|-6|single family| +44651|AAAAAAAALGOKAAAA|519|1st 8th|Wy|Suite Q|Pleasant Hill|Lyman County|SD|53604|United States|-7|apartment| +44652|AAAAAAAAMGOKAAAA|446|Fourth |Blvd|Suite Q|Harmony|Rogers County|OK|75804|United States|-6|condo| +44653|AAAAAAAANGOKAAAA|655|Third Highland|Pkwy|Suite C|Jamestown|Caledonia County|VT|07467|United States|-5|apartment| +44654|AAAAAAAAOGOKAAAA|910|11th |Avenue|Suite J|Friendship|Tama County|IA|54536|United States|-6|condo| +44655|AAAAAAAAPGOKAAAA|517|Washington 1st|Way|Suite A|Red Hill|Sharp County|AR|74338|United States|-6|single family| +44656|AAAAAAAAAHOKAAAA|553|Walnut Third|Avenue|Suite 360|Riverdale|Matanuska-Susitna Borough|AK|99391|United States|-9|condo| +44657|AAAAAAAABHOKAAAA|996|Walnut |Road|Suite 160|Fairview|Harrison County|TX|75709|United States|-6|apartment| +44658|AAAAAAAACHOKAAAA|533|Jefferson |RD|Suite 50|Fairfax|Flagler County|FL|36550|United States|-5|apartment| +44659|AAAAAAAADHOKAAAA|888|Fifth 10th|Drive|Suite 370|Lincoln|Gilchrist County|FL|31289|United States|-5|condo| +44660|AAAAAAAAEHOKAAAA|243|East Spring|Lane|Suite 280|Woodland Park|Morgan County|MO|61934|United States|-6|single family| +44661|AAAAAAAAFHOKAAAA|144|15th Church|Court|Suite 200|Lakeview|Payette County|ID|88579|United States|-7|condo| +44662|AAAAAAAAGHOKAAAA|46|Oak 14th|Street|Suite 440|New Hope|Foster County|ND|59431|United States|-6|apartment| +44663|AAAAAAAAHHOKAAAA|337|Spruce Thirteenth|Boulevard|Suite 300|Oakland|Yellowstone County|MT|69843|United States|-7|single family| +44664|AAAAAAAAIHOKAAAA|211|Hill Jefferson|Wy|Suite 240|Spring Hill|Park County|CO|86787|United States|-7|single family| +44665|AAAAAAAAJHOKAAAA|823|2nd |Drive|Suite 430|Harmony|Major County|OK|75804|United States|-6|condo| +44666|AAAAAAAAKHOKAAAA|354|Washington Lake|Street|Suite A|Oak Ridge|Bedford County|TN|38371|United States|-5|apartment| +44667|AAAAAAAALHOKAAAA|734|Lakeview |Cir.|Suite I|Hillcrest|Pratt County|KS|63003|United States|-6|condo| +44668|AAAAAAAAMHOKAAAA|51|Broadway 2nd|Road|Suite D|Riverdale|McKinley County|NM|89391|United States|-7|apartment| +44669|AAAAAAAANHOKAAAA|52|9th |Boulevard|Suite J|Newtown|Danville city|VA|21749|United States|-5|condo| +44670|AAAAAAAAOHOKAAAA|88|Cedar Lincoln|Parkway|Suite O|Valley View|Parke County|IN|45124|United States|-5|condo| +44671|AAAAAAAAPHOKAAAA|959|8th Birch|Court|Suite J|Belmont|Knox County|TN|30191|United States|-6|condo| +44672|AAAAAAAAAIOKAAAA|742|Laurel Second|Circle|Suite 310|Oakdale|Ripley County|MO|69584|United States|-6|condo| +44673|AAAAAAAABIOKAAAA|123|Birch |ST|Suite 190|Spring Valley|Allegheny County|PA|16060|United States|-5|condo| +44674|AAAAAAAACIOKAAAA|402|2nd Pine|Avenue|Suite 480|Glendale|Barren County|KY|43951|United States|-6|single family| +44675|AAAAAAAADIOKAAAA|555|Elm |Ct.|Suite H|Brownsville|Lawrence County|OH|49310|United States|-5|apartment| +44676|AAAAAAAAEIOKAAAA|238|2nd |Ave|Suite I|Jackson|Yolo County|CA|99583|United States|-8|single family| +44677|AAAAAAAAFIOKAAAA|546|Railroad |Parkway|Suite 150|Lakewood|Livingston County|NY|18877|United States|-5|single family| +44678|AAAAAAAAGIOKAAAA|781|Cherry 11th|Dr.|Suite 90|Lakewood|Mason County|MI|48877|United States|-5|condo| +44679|AAAAAAAAHIOKAAAA|713|Central First|ST|Suite L|Greenville|Fountain County|IN|41387|United States|-5|single family| +44680|AAAAAAAAIIOKAAAA|766|Pine Mill|Circle|Suite 130|Stringtown|Banner County|NE|60162|United States|-6|apartment| +44681|AAAAAAAAJIOKAAAA|307|Third |Blvd|Suite 50|Providence|Warren County|KY|46614|United States|-5|apartment| +44682|AAAAAAAAKIOKAAAA|828|Highland Sycamore|Circle|Suite 370|Woodland|Mercer County|IL|64854|United States|-6|single family| +44683|AAAAAAAALIOKAAAA|672|Hillcrest |Way|Suite 340|Lakeview|Douglas County|NE|68579|United States|-6|apartment| +44684|AAAAAAAAMIOKAAAA|847|Oak |Avenue|Suite Y|Friendship|Lancaster County|PA|14536|United States|-5|single family| +44685|AAAAAAAANIOKAAAA|342|Green Madison|Court|Suite H|Riverside|Aransas County|TX|79231|United States|-6|condo| +44686|AAAAAAAAOIOKAAAA|||Court||Midway|Pulaski County|IL||||apartment| +44687|AAAAAAAAPIOKAAAA|917|Railroad Washington|ST|Suite E|Marion|Pinellas County|FL|30399|United States|-5|condo| +44688|AAAAAAAAAJOKAAAA|954|Fifth 2nd|RD|Suite 480|Springdale|Marlboro County|SC|28883|United States|-5|apartment| +44689|AAAAAAAABJOKAAAA|295|Maple Sycamore|Ave|Suite Y|Kingston|Issaquena County|MS|54975|United States|-6|apartment| +44690|AAAAAAAACJOKAAAA|227|West Lincoln|Drive|Suite 460|Bunker Hill|Shannon County|SD|50150|United States|-7|single family| +44691|AAAAAAAADJOKAAAA|18|4th |Road|Suite M|Oakwood|Mason County|WA|90169|United States|-8|condo| +44692|AAAAAAAAEJOKAAAA|617|Hill River|Court|Suite I|Marion|Payette County|ID|80399|United States|-7|apartment| +44693|AAAAAAAAFJOKAAAA|96|Cedar Main|Blvd|Suite T|Lebanon|Pike County|IL|62898|United States|-6|single family| +44694|AAAAAAAAGJOKAAAA|686|First |Street|Suite O|Green Acres|Whitley County|IN|47683|United States|-5|condo| +44695|AAAAAAAAHJOKAAAA|555|First Woodland|Wy|Suite 0|Enterprise|Sheridan County|ND|51757|United States|-6|single family| +44696|AAAAAAAAIJOKAAAA|847|Main East|Ln|Suite J|Newport|Knox County|OH|41521|United States|-5|single family| +44697|AAAAAAAAJJOKAAAA|316|Railroad |Parkway|Suite 170|Mount Pleasant|Manassas Park city|VA|21933|United States|-5|condo| +44698|AAAAAAAAKJOKAAAA|996|West Green|Cir.|Suite 290|Buena Vista|McDowell County|NC|25752|United States|-5|apartment| +44699|AAAAAAAALJOKAAAA||Lincoln Pine||Suite 180||||53298||-6|single family| +44700|AAAAAAAAMJOKAAAA|803|Woodland View|Street|Suite 450|Cedar Grove|Holmes County|OH|40411|United States|-5|single family| +44701|AAAAAAAANJOKAAAA|485|8th |Blvd|Suite R|Waterloo|Ziebach County|SD|51675|United States|-6|condo| +44702|AAAAAAAAOJOKAAAA|772|North Green|Circle|Suite 450|Franklin|Sumner County|TN|39101|United States|-6|single family| +44703|AAAAAAAAPJOKAAAA|593|1st |Blvd|Suite 190|Pleasant Valley|Cameron County|TX|72477|United States|-6|condo| +44704|AAAAAAAAAKOKAAAA|427|Willow |Road|Suite 270|Buena Vista|Taylor County|FL|35752|United States|-5|single family| +44705|AAAAAAAABKOKAAAA|329|Ash Spruce|Pkwy|Suite S|Pleasant Valley|Waupaca County|WI|52477|United States|-6|single family| +44706|AAAAAAAACKOKAAAA|985|Lake 11th|Ct.|Suite 180|Pleasant Valley|Conway County|AR|72477|United States|-6|apartment| +44707|AAAAAAAADKOKAAAA|633|Thirteenth |Lane|Suite L|Pierce|Warren County|NC|23360|United States|-5|apartment| +44708|AAAAAAAAEKOKAAAA||Park |Parkway|||Pendleton County|||United States|-5|| +44709|AAAAAAAAFKOKAAAA|464|8th |Blvd|Suite R|Shiloh|Benton County|MO|69275|United States|-6|apartment| +44710|AAAAAAAAGKOKAAAA|39|Smith Locust|Wy|Suite P|Five Points|Owsley County|KY|46098|United States|-5|single family| +44711|AAAAAAAAHKOKAAAA|503|Main Park|Ln|Suite E|Belleville|Beckham County|OK|72924|United States|-6|condo| +44712|AAAAAAAAIKOKAAAA|397|Walnut |Wy|Suite 420|Providence|Murray County|MN|56614|United States|-6|apartment| +44713|AAAAAAAAJKOKAAAA|623|North |||Greenfield|Fall River County|SD|55038|United States|-6|apartment| +44714|AAAAAAAAKKOKAAAA|826||Ct.|||Graham County|AZ||||condo| +44715|AAAAAAAALKOKAAAA|769|North Sunset|Circle|Suite 400||Sumter County|||United States||| +44716|AAAAAAAAMKOKAAAA|810|Valley Walnut|Wy|Suite A|Brownsville|Auglaize County|OH|49310|United States|-5|condo| +44717|AAAAAAAANKOKAAAA|667|Madison |Ave|Suite F|Amity|San Bernardino County|CA|90766|United States|-8|single family| +44718|AAAAAAAAOKOKAAAA|145|Lincoln |Circle|Suite U|Pine Grove|Lexington County|SC|24593|United States|-5|single family| +44719|AAAAAAAAPKOKAAAA|462|Forest Main|ST|Suite 240|Farmington|Putnam County|IL|69145|United States|-6|apartment| +44720|AAAAAAAAALOKAAAA|758|2nd Hill|Ave|Suite 160|Mount Pleasant|Grant County|OR|91933|United States|-8|single family| +44721|AAAAAAAABLOKAAAA|940|Willow |Dr.|Suite 140|Centerville|Ravalli County|MT|60059|United States|-7|single family| +44722|AAAAAAAACLOKAAAA|833|East ||||||||-6|condo| +44723|AAAAAAAADLOKAAAA|352|Highland |Avenue|Suite I|Union Hill|Ouachita County|AR|77746|United States|-6|apartment| +44724|AAAAAAAAELOKAAAA|860|1st |Ln|Suite 380|Pleasant Grove|Pope County|AR|74136|United States|-6|apartment| +44725|AAAAAAAAFLOKAAAA|132|7th |Court|Suite 20|Jerome|Bear Lake County|ID|89920|United States|-7|single family| +44726|AAAAAAAAGLOKAAAA|677|River Adams|Pkwy|Suite V|Five Forks|Barnwell County|SC|22293|United States|-5|single family| +44727|AAAAAAAAHLOKAAAA|589|4th |Drive|Suite S|Arlington|Pittsylvania County|VA|26557|United States|-5|apartment| +44728|AAAAAAAAILOKAAAA|788|North Railroad|Pkwy|Suite 470|Floyd|Chester County|SC|23235|United States|-5|single family| +44729|AAAAAAAAJLOKAAAA|634|2nd View|Ln|Suite F|Maple Grove|Laclede County|MO|68252|United States|-6|apartment| +44730|AAAAAAAAKLOKAAAA|858|Ash |Avenue|Suite P|Bridgeport|Lawrence County|AR|75817|United States|-6|apartment| +44731|AAAAAAAALLOKAAAA|368|Railroad 1st|Way|Suite 160|Sulphur Springs|Marshall County|MN|58354|United States|-6|apartment| +44732|AAAAAAAAMLOKAAAA|122|Valley Second|Boulevard|Suite 260|Florence|Clark County|OH|43394|United States|-5|condo| +44733|AAAAAAAANLOKAAAA|574|Johnson |Avenue|Suite 430|Forest Hills|Irion County|TX|79237|United States|-6|single family| +44734|AAAAAAAAOLOKAAAA|529|11th Spruce|Avenue|Suite 420|Marion|Peoria County|IL|60399|United States|-6|apartment| +44735|AAAAAAAAPLOKAAAA|666|Jefferson |Wy|Suite O|Hillcrest|Albemarle County|VA|23003|United States|-5|single family| +44736|AAAAAAAAAMOKAAAA|244|Lee Ridge|Lane|Suite T|Riverside|Clarion County|PA|19231|United States|-5|condo| +44737|AAAAAAAABMOKAAAA|986|10th |Drive|Suite 210|Georgetown|Surry County|NC|27057|United States|-5|apartment| +44738|AAAAAAAACMOKAAAA|774|Spruce |Street|Suite N|Rankin|Shenandoah County|VA|22621|United States|-5|condo| +44739|AAAAAAAADMOKAAAA|27|9th |Drive|Suite 60|Greenville|Carroll County|MO|61387|United States|-6|condo| +44740|AAAAAAAAEMOKAAAA|428|East Jefferson|Cir.|Suite 290|Sunnyside|Mahoning County|OH|41952|United States|-5|single family| +44741|AAAAAAAAFMOKAAAA|792|Dogwood 11th|Boulevard|Suite 110|Pleasant Hill|Texas County|MO|63604|United States|-6|apartment| +44742|AAAAAAAAGMOKAAAA|107|North |Court|Suite 160|New Hope|Wyoming County|NY|19431|United States|-5|single family| +44743|AAAAAAAAHMOKAAAA|764|First Smith|Cir.|Suite Q|Spring Hill|Isle of Wight County|VA|26787|United States|-5|condo| +44744|AAAAAAAAIMOKAAAA|922|Fourth Oak|Drive|Suite W|Oak Hill|Coos County|OR|97838|United States|-8|condo| +44745|AAAAAAAAJMOKAAAA|756|Second |Blvd|Suite 340|Stringtown|Scott County|AR|70162|United States|-6|condo| +44746|AAAAAAAAKMOKAAAA|968|Pine Main|Wy|Suite 320|Oak Ridge|Camden County|NC|28371|United States|-5|single family| +44747|AAAAAAAALMOKAAAA|880|8th |Drive|Suite 300|Shiloh|Johnston County|OK|79275|United States|-6|condo| +44748|AAAAAAAAMMOKAAAA|735|Church |Pkwy|Suite 60|Arlington|Rio Blanco County|CO|86557|United States|-7|condo| +44749|AAAAAAAANMOKAAAA|124|Willow |Court|Suite 380|Brownsville|Essex County|NY|19310|United States|-5|single family| +44750|AAAAAAAAOMOKAAAA|676|Railroad |||||OK||||single family| +44751|AAAAAAAAPMOKAAAA|237|Ninth |Boulevard|Suite Q|Plainview|Menifee County|KY|43683|United States|-5|apartment| +44752|AAAAAAAAANOKAAAA|391|View |RD|Suite 300|Marion|Pocahontas County|IA|50399|United States|-6|apartment| +44753|AAAAAAAABNOKAAAA|276|Seventh Central|Lane|Suite J|Buena Vista|Campbell County|TN|35752|United States|-5|single family| +44754|AAAAAAAACNOKAAAA|875|Lincoln |Court|Suite 50|Hamilton|Madison County|TX|72808|United States|-6|apartment| +44755|AAAAAAAADNOKAAAA|856|Miller |Dr.|Suite 200|Wildwood|Pushmataha County|OK|76871|United States|-6|condo| +44756|AAAAAAAAENOKAAAA|343|15th Railroad|Pkwy|Suite V|Evans|Ross County|OH|42284|United States|-5|apartment| +44757|AAAAAAAAFNOKAAAA|542|Ridge |Circle|Suite 60|Oakdale|Minnehaha County|SD|59584|United States|-7|condo| +44758|AAAAAAAAGNOKAAAA|522|View Hill|Wy|Suite W|Woodruff|Oconee County|GA|34174|United States|-5|apartment| +44759|AAAAAAAAHNOKAAAA|755|Sycamore Seventh|Road|Suite G|Jamestown|Winchester city|VA|26867|United States|-5|apartment| +44760|AAAAAAAAINOKAAAA|457|East 10th|Cir.|Suite 430|Springfield|Adams County|MS|59303|United States|-6|single family| +44761|AAAAAAAAJNOKAAAA|85|Woodland Park|Ln|Suite 400|Lakeville|Seneca County|OH|48811|United States|-5|condo| +44762|AAAAAAAAKNOKAAAA|400|8th River|Court|Suite 130|Mount Olive|Newton County|MS|58059|United States|-6|single family| +44763|AAAAAAAALNOKAAAA|781|3rd |RD|Suite W|Oakdale|Franklin County|AR|79584|United States|-6|single family| +44764|AAAAAAAAMNOKAAAA|749|Main Birch|Lane|Suite 140|Lakewood|Kenai Peninsula Borough|AK|98877|United States|-9|condo| +44765|AAAAAAAANNOKAAAA|648|Main Meadow|Blvd|Suite U|Mountain View|Carroll County|IL|64466|United States|-6|apartment| +44766|AAAAAAAAONOKAAAA|878|Madison |Avenue|Suite L|New Hope|Greene County|OH|49431|United States|-5|apartment| +44767|AAAAAAAAPNOKAAAA|647|Spruce |Boulevard|Suite 240|Oak Hill|Imperial County|CA|97838|United States|-8|condo| +44768|AAAAAAAAAOOKAAAA|564|Willow |Parkway|Suite H|Snug Harbor|East Carroll Parish|LA|77936|United States|-6|condo| +44769|AAAAAAAABOOKAAAA|525|Ash Johnson|Court|Suite 250|Maple Grove|McNairy County|TN|38252|United States|-6|apartment| +44770|AAAAAAAACOOKAAAA|140|5th |Street|Suite 490|Yorktown|Washington County|CO|80732|United States|-7|single family| +44771|AAAAAAAADOOKAAAA|636|South Spring|Circle|Suite I|Ashland|DeSoto County|FL|34244|United States|-5|condo| +44772|AAAAAAAAEOOKAAAA|465|3rd Wilson|Ave|Suite 160|Glendale|Platte County|NE|63951|United States|-7|apartment| +44773|AAAAAAAAFOOKAAAA|652|Hill Center|Cir.|Suite 460|Oakwood|Missaukee County|MI|40169|United States|-5|single family| +44774|AAAAAAAAGOOKAAAA|184|4th |Dr.|Suite 250|Liberty|Kent County|TX|73451|United States|-6|apartment| +44775|AAAAAAAAHOOKAAAA|375|Second 1st|Cir.|Suite N|Oakland|Benton County|AR|79843|United States|-6|condo| +44776|AAAAAAAAIOOKAAAA|607|Park Fourth|Road|Suite B|Arlington|Clay County|AL|36557|United States|-6|condo| +44777|AAAAAAAAJOOKAAAA|794|Sunset Sunset|Drive|Suite 470|Glendale|Calhoun County|GA|33951|United States|-5|condo| +44778|AAAAAAAAKOOKAAAA|571|Sunset |Lane|Suite H|Sulphur Springs|Jones County|GA|38354|United States|-5|single family| +44779|AAAAAAAALOOKAAAA||2nd ||Suite 250|Five Forks|Tuolumne County|||||single family| +44780|AAAAAAAAMOOKAAAA|708|Lake |ST|Suite 60|Wesley|Trempealeau County|WI|51218|United States|-6|apartment| +44781|AAAAAAAANOOKAAAA|704|Highland |Drive|Suite J|Lincoln|Putnam County|NY|11289|United States|-5|condo| +44782|AAAAAAAAOOOKAAAA|141|Lincoln |Ln|Suite 120|Unionville|Tallahatchie County|MS|51711|United States|-6|apartment| +44783|AAAAAAAAPOOKAAAA|330|Cherry |Way|Suite 220|Stringtown|Dallas County|AR|70162|United States|-6|apartment| +44784|AAAAAAAAAPOKAAAA|539|View |Boulevard|Suite 320|Sunnyside|Cameron Parish|LA|71952|United States|-6|apartment| +44785|AAAAAAAABPOKAAAA|45|Johnson Hickory|Ave|Suite Q|Midway|Gloucester County|VA|21904|United States|-5|single family| +44786|AAAAAAAACPOKAAAA|631|Elevnth |Boulevard|Suite A|Point Pleasant|Randolph County|WV|24749|United States|-5|apartment| +44787|AAAAAAAADPOKAAAA|557|Meadow Park|Ave|Suite 490|Spring Hill|Archer County|TX|76787|United States|-6|apartment| +44788|AAAAAAAAEPOKAAAA|636|Ninth Spring|Parkway|Suite R|Concord|Elbert County|CO|84107|United States|-7|single family| +44789|AAAAAAAAFPOKAAAA|894|Hickory |Drive|Suite 10|Ruth|Marshall County|OK|70309|United States|-6|apartment| +44790|AAAAAAAAGPOKAAAA|604|Ridge Jefferson|Cir.|Suite 340|New Hope|Whatcom County|WA|99431|United States|-8|single family| +44791|AAAAAAAAHPOKAAAA|103|Seventh |Ct.|Suite 70|Clinton|Hayes County|NE|68222|United States|-6|apartment| +44792|AAAAAAAAIPOKAAAA|207|3rd |Cir.|Suite U|Centerville|Denver County|CO|80059|United States|-7|condo| +44793|AAAAAAAAJPOKAAAA|991|12th Hill|Ln|Suite 180|Salem|Shelby County|IL|68048|United States|-6|apartment| +44794|AAAAAAAAKPOKAAAA|165|Sunset |Court|Suite 300|Crossroads|Butler County|KY|40534|United States|-6|single family| +44795|AAAAAAAALPOKAAAA|216|Park |Parkway|Suite 280|Franklin|Kane County|UT|89101|United States|-7|condo| +44796|AAAAAAAAMPOKAAAA|139|Smith Lincoln|Lane|Suite 360|Stringtown|Coweta County|GA|30162|United States|-5|condo| +44797|AAAAAAAANPOKAAAA|686|Cedar Green|RD|Suite 0|Brownsville|Clinton County|PA|19310|United States|-5|apartment| +44798|AAAAAAAAOPOKAAAA|194|Fourth |Blvd|Suite 40|Oakwood|Dallas County|AL|30169|United States|-6|single family| +44799|AAAAAAAAPPOKAAAA|59|12th Main|Circle|Suite 60|Pine Grove|Lincoln County|GA|34593|United States|-5|condo| +44800|AAAAAAAAAAPKAAAA|672|Laurel Johnson|Road|Suite 150|Antioch|Holmes County|MS|58605|United States|-6|single family| +44801|AAAAAAAABAPKAAAA|33|Elm Fifth|Road|Suite T|Afton|Putnam County|OH|40029|United States|-5|single family| +44802|AAAAAAAACAPKAAAA|444|Pine Lincoln|Ln|Suite 480|Glenville|Perry County|PA|13445|United States|-5|condo| +44803|AAAAAAAADAPKAAAA|508|East Poplar|Court|Suite 50|Pine Grove|Orange County|FL|34593|United States|-5|condo| +44804|AAAAAAAAEAPKAAAA|226|Madison West|Dr.|Suite 490|Red Hill|Lonoke County|AR|74338|United States|-6|condo| +44805|AAAAAAAAFAPKAAAA|764|Mill 2nd|RD|Suite D|Bethel|Dixon County|NE|65281|United States|-6|condo| +44806|AAAAAAAAGAPKAAAA|227|Second |RD|Suite 190|Harmony|Norton County|KS|65804|United States|-6|condo| +44807|AAAAAAAAHAPKAAAA|225|Oak |Court|Suite I|Walnut Grove|La Salle County|TX|77752|United States|-6|apartment| +44808|AAAAAAAAIAPKAAAA|1|Cedar Adams|Street|Suite 430|Oak Hill|Hutchinson County|SD|57838|United States|-7|single family| +44809|AAAAAAAAJAPKAAAA|382|Sycamore 1st|Boulevard|Suite N|Walnut Grove|Pasquotank County|NC|27752|United States|-5|apartment| +44810|AAAAAAAAKAPKAAAA|161|2nd Walnut|Avenue|Suite O|Mount Vernon|Caldwell Parish|LA|78482|United States|-6|apartment| +44811|AAAAAAAALAPKAAAA|592|Locust Maple|Circle|Suite 380|Riverview|Lincoln County|NV|89003|United States|-8|apartment| +44812|AAAAAAAAMAPKAAAA|19|Hickory Hillcrest|Ln|Suite M|Glendale|San Benito County|CA|93951|United States|-8|condo| +44813|AAAAAAAANAPKAAAA|467|Walnut Sycamore|RD|Suite P|Cedar Grove|Columbia County|FL|30411|United States|-5|single family| +44814|AAAAAAAAOAPKAAAA|704|Cedar Hickory|Road|Suite V|Bethel|Reynolds County|MO|65281|United States|-6|apartment| +44815|AAAAAAAAPAPKAAAA|40|4th Poplar|Parkway|Suite V|Scottsville|Iron County|UT|84190|United States|-7|single family| +44816|AAAAAAAAABPKAAAA|193|Washington Chestnut|||Glenwood||||||apartment| +44817|AAAAAAAABBPKAAAA|263|Park Laurel|Way|Suite 310|Liberty|Benton County|IN|43451|United States|-5|apartment| +44818|AAAAAAAACBPKAAAA|||RD|||Menominee County|WI|55817|United States|-6|single family| +44819|AAAAAAAADBPKAAAA|823|Sunset Jackson|Blvd|Suite I|White Oak|McCone County|MT|66668|United States|-7|condo| +44820|AAAAAAAAEBPKAAAA|255|Cherry |Wy|Suite 200|Spring Hill|Pearl River County|MS|56787|United States|-6|single family| +44821|AAAAAAAAFBPKAAAA|||Ave|||||55819|||condo| +44822|AAAAAAAAGBPKAAAA|953|Green 1st|Street|Suite J|Arlington|Washington County|AL|36557|United States|-6|apartment| +44823|AAAAAAAAHBPKAAAA|383|Adams 6th|Boulevard|Suite P|Oak Hill|Newton County|MO|67838|United States|-6|apartment| +44824|AAAAAAAAIBPKAAAA|448|7th |Blvd|Suite K|Riverside|Unicoi County|TN|39231|United States|-6|apartment| +44825|AAAAAAAAJBPKAAAA|731|Birch |Wy|Suite 460|Midway|Monroe County|IA|51904|United States|-6|apartment| +44826|AAAAAAAAKBPKAAAA|554|Highland 4th|Avenue|Suite M|Highland Park|Greene County|AL|36534|United States|-6|single family| +44827|AAAAAAAALBPKAAAA|635|Ash Lake|Dr.|Suite 100|Ashland|Richland County|MT|64244|United States|-7|condo| +44828|AAAAAAAAMBPKAAAA|85|Laurel |Way|Suite P|Bridgeport|Schoolcraft County|MI|45817|United States|-5|single family| +44829|AAAAAAAANBPKAAAA|651|Church |Boulevard|Suite 340|Five Forks|Montgomery County|MO|62293|United States|-6|single family| +44830|AAAAAAAAOBPKAAAA|64|14th |ST|Suite 130|Enterprise|Young County|TX|71757|United States|-6|single family| +44831|AAAAAAAAPBPKAAAA|504|Chestnut 14th|Street|Suite X|Highland Park|Carlton County|MN|56534|United States|-6|single family| +44832|AAAAAAAAACPKAAAA|518|Woodland Wilson|Cir.|Suite A|Riverview|Calhoun County|WV|29003|United States|-5|single family| +44833|AAAAAAAABCPKAAAA|727|7th |Road|Suite 0|Kingston|Curry County|NM|84975|United States|-7|single family| +44834|AAAAAAAACCPKAAAA|834|Sycamore 8th|Court|Suite 50|Forest Hills|Leflore County|MS|59237|United States|-6|single family| +44835|AAAAAAAADCPKAAAA|401|Sycamore |Drive|Suite W|Hamilton|Linn County|OR|92808|United States|-8|apartment| +44836|AAAAAAAAECPKAAAA|527|Pine Walnut|Court|Suite W|Walnut Grove|King and Queen County|VA|27752|United States|-5|condo| +44837|AAAAAAAAFCPKAAAA|743|Central |Ln|Suite U|Brookwood|Dickens County|TX|70965|United States|-6|condo| +44838|AAAAAAAAGCPKAAAA|867|Park |Ln|Suite 140|Cedar Grove|Kane County|UT|80411|United States|-7|condo| +44839|AAAAAAAAHCPKAAAA|911|Birch |Pkwy|Suite 280|Sullivan|Miami County|OH|40451|United States|-5|apartment| +44840|AAAAAAAAICPKAAAA|395|Tenth Hill|Drive|Suite 140|Florence|Anderson County|KS|63394|United States|-6|single family| +44841|AAAAAAAAJCPKAAAA|369|Poplar Ridge|Ave|Suite 470|Jackson|Holt County|NE|69583|United States|-7|single family| +44842|AAAAAAAAKCPKAAAA|71|Fifth |Dr.|Suite O|Bunker Hill|Bullitt County|KY|40150|United States|-6|single family| +44843|AAAAAAAALCPKAAAA|700|Walnut Johnson|Parkway|Suite 190|Elba|Mercer County|OH|40262|United States|-5|apartment| +44844|AAAAAAAAMCPKAAAA|520|Tenth 7th|Wy|Suite Q|Newtown|Door County|WI|51749|United States|-6|condo| +44845|AAAAAAAANCPKAAAA|311|West |Avenue|Suite T|Oakwood|Craig County|VA|20169|United States|-5|condo| +44846|AAAAAAAAOCPKAAAA|760|Meadow Maple|Pkwy|Suite R|Newport|Henry County|TN|31521|United States|-5|single family| +44847|AAAAAAAAPCPKAAAA||Highland |Dr.|||Calhoun County|||||condo| +44848|AAAAAAAAADPKAAAA|289|Pine 1st|Ct.|Suite 410|Mount Vernon|Harper County|KS|68482|United States|-6|apartment| +44849|AAAAAAAABDPKAAAA|123|North Fourth|Boulevard|Suite 170|Friendship|Dewey County|SD|54536|United States|-6|single family| +44850|AAAAAAAACDPKAAAA|499|Dogwood |Boulevard|Suite H|White Oak|Lamar County|AL|36668|United States|-6|condo| +44851|AAAAAAAADDPKAAAA|35|Williams |Road|Suite L|Kingston|Floyd County|KY|44975|United States|-6|single family| +44852|AAAAAAAAEDPKAAAA|589|Williams Pine|Ct.|Suite F|Salem|McPherson County|NE|68048|United States|-7|single family| +44853|AAAAAAAAFDPKAAAA|750|9th |Wy|Suite T|Union|Cumberland County|VA|28721|United States|-5|single family| +44854|AAAAAAAAGDPKAAAA|645|First Green|Ln|Suite H|Bridgeport|Camp County|TX|75817|United States|-6|apartment| +44855|AAAAAAAAHDPKAAAA|672|13th |RD|Suite O|Mount Olive|Logan County|OH|48059|United States|-5|apartment| +44856|AAAAAAAAIDPKAAAA|436|Lake |Pkwy|Suite 420|Mountain View|Franklin Parish|LA|74466|United States|-6|condo| +44857|AAAAAAAAJDPKAAAA|751|Walnut Pine|Parkway|Suite J|Walnut Grove|Uvalde County|TX|77752|United States|-6|single family| +44858|AAAAAAAAKDPKAAAA|730|Center |Pkwy|Suite B|Vance|Morgan County|KY|40268|United States|-5|condo| +44859|AAAAAAAALDPKAAAA|834|Cherry |Road|Suite G|Macedonia|Washington County|OR|91087|United States|-8|single family| +44860|AAAAAAAAMDPKAAAA|348|Jackson Smith|Ct.|Suite 430|Union Hill|Garrett County|MD|27746|United States|-5|single family| +44861|AAAAAAAANDPKAAAA|28|Poplar 6th|Street|Suite 150|Belmont|Fairfield County|CT|00791|United States|-5|apartment| +44862|AAAAAAAAODPKAAAA|906|1st South|Pkwy|Suite 260|Fowler|Muscogee County|GA|31083|United States|-5|condo| +44863|AAAAAAAAPDPKAAAA|723|Oak Chestnut|Avenue|Suite B|Georgetown|Lawrence County|TN|37057|United States|-6|condo| +44864|AAAAAAAAAEPKAAAA|239|Chestnut South|Road|Suite 160|Hillcrest|Titus County|TX|73003|United States|-6|single family| +44865|AAAAAAAABEPKAAAA|219|North 3rd|Parkway|Suite B|Friendship|Roscommon County|MI|44536|United States|-5|condo| +44866|AAAAAAAACEPKAAAA|49|Park |Cir.|Suite 120|Peoria|Natchitoches Parish|LA|79818|United States|-6|single family| +44867|AAAAAAAADEPKAAAA|106|Cherry |Ave||Elm Grove|Lenoir County|NC|23298||-5|apartment| +44868|AAAAAAAAEEPKAAAA|619||Circle|Suite C|||GA||United States|-5|apartment| +44869|AAAAAAAAFEPKAAAA|643|View Johnson|Way|Suite 380|Clinton|Jackson County|MI|48222|United States|-5|condo| +44870|AAAAAAAAGEPKAAAA|318|3rd 9th|Court|Suite 10|Arlington|Gloucester County|NJ|07157|United States|-5|apartment| +44871|AAAAAAAAHEPKAAAA|761|Locust North|Way|Suite 340|Oakwood|Schuylkill County|PA|10169|United States|-5|apartment| +44872|AAAAAAAAIEPKAAAA|128|Poplar Green|Ln|Suite 100|Lakeview|Yoakum County|TX|78579|United States|-6|single family| +44873|AAAAAAAAJEPKAAAA||Hickory |Boulevard||Fairview|Randolph County|GA||United States|-5|| +44874|AAAAAAAAKEPKAAAA|977|5th Park|Ave|Suite T|Woodland|Mendocino County|CA|94854|United States|-8|apartment| +44875|AAAAAAAALEPKAAAA|859|Hickory 12th|Lane|Suite 400|Greenfield|Dickinson County|IA|55038|United States|-6|condo| +44876|AAAAAAAAMEPKAAAA|202|Oak |Cir.|Suite 0|Pleasant Grove|Madison County|IA|54136|United States|-6|single family| +44877|AAAAAAAANEPKAAAA|707|Adams 6th|Ave|Suite 340|Post Oak|Pickens County|SC|28567|United States|-5|apartment| +44878|AAAAAAAAOEPKAAAA|53|Green |Cir.|Suite S|Valley View|Pawnee County|OK|75124|United States|-6|apartment| +44879|AAAAAAAAPEPKAAAA|||||Greenfield|Dearborn County|IN|45038|United States||| +44880|AAAAAAAAAFPKAAAA|130|West |Parkway|Suite S|Oakland|Bullock County|AL|39843|United States|-6|apartment| +44881|AAAAAAAABFPKAAAA|636|Lee |Way|Suite B|Five Points|Lac qui Parle County|MN|56098|United States|-6|single family| +44882|AAAAAAAACFPKAAAA|549|Forest |Avenue|Suite 40|Wilton|Vilas County|WI|56997|United States|-6|condo| +44883|AAAAAAAADFPKAAAA|233|Second ||||Fisher County|||United States||| +44884|AAAAAAAAEFPKAAAA|110|6th Ridge|Lane|Suite M|Farmington|Dawson County|NE|69145|United States|-6|single family| +44885|AAAAAAAAFFPKAAAA|12|Valley Washington|Drive|Suite 220|Waterloo|Candler County|GA|31675|United States|-5|single family| +44886|AAAAAAAAGFPKAAAA|261|Lake |Wy|Suite 70|Union|Garfield County|MT|68721|United States|-7|condo| +44887|AAAAAAAAHFPKAAAA|51|Ridge Cherry|Court|Suite 100|Green Acres|Cullman County|AL|37683|United States|-6|apartment| +44888|AAAAAAAAIFPKAAAA|396|Chestnut |Street|Suite T|Shiloh|Bremer County|IA|59275|United States|-6|single family| +44889|AAAAAAAAJFPKAAAA|467|Laurel 3rd|Ct.|Suite F|Five Forks|Walworth County|WI|52293|United States|-6|single family| +44890|AAAAAAAAKFPKAAAA|825|Main |Court|Suite 310|Fairview|Wheeler County|NE|65709|United States|-7|apartment| +44891|AAAAAAAALFPKAAAA|754|Hill Dogwood|Boulevard|Suite 440|Oak Hill|Washoe County|NV|87838|United States|-8|single family| +44892|AAAAAAAAMFPKAAAA|569|1st Cherry|Boulevard|Suite I|Macedonia|Reynolds County|MO|61087|United States|-6|condo| +44893|AAAAAAAANFPKAAAA|947|Spruce |Ave|Suite P|Riverside|Monroe County|IN|49231|United States|-5|single family| +44894|AAAAAAAAOFPKAAAA|111|South Sunset|Parkway|Suite F|Fairview|McLean County|IL|65709|United States|-6|apartment| +44895|AAAAAAAAPFPKAAAA|719|Maple Spruce|Lane|Suite 390|Hopewell|Vernon Parish|LA|70587|United States|-6|apartment| +44896|AAAAAAAAAGPKAAAA|741|Woodland 11th|Avenue|Suite H|Maple Grove|Socorro County|NM|88252|United States|-7|apartment| +44897|AAAAAAAABGPKAAAA|538|Hillcrest Dogwood|Drive|Suite K|Mount Pleasant|Hutchinson County|TX|71933|United States|-6|condo| +44898|AAAAAAAACGPKAAAA|667|Fourth Jefferson|Ln|Suite 0|Belleview|Decatur County|GA|30492|United States|-5|single family| +44899|AAAAAAAADGPKAAAA|974|Locust Davis|Way|Suite 60|Sawyer|Greene County|NC|26045|United States|-5|single family| +44900|AAAAAAAAEGPKAAAA|126|Poplar |Court|Suite E|Whitesville|Carlisle County|KY|45903|United States|-6|single family| +44901|AAAAAAAAFGPKAAAA|986|Cedar |Cir.|Suite K|Glendale|Edwards County|KS|63951|United States|-6|condo| +44902|AAAAAAAAGGPKAAAA|309|13th 3rd|Ln|Suite 340|Enterprise|Gage County|NE|61757|United States|-6|condo| +44903|AAAAAAAAHGPKAAAA|484|Eigth Walnut|Wy|Suite 320|Belmont|Williamson County|IL|60191|United States|-6|condo| +44904|AAAAAAAAIGPKAAAA|852|North College|Court|Suite A|White Plains|Lake County|TN|36622|United States|-6|apartment| +44905|AAAAAAAAJGPKAAAA|304|Elm |Lane|Suite 270|Bethel|Summit County|CO|85281|United States|-7|apartment| +44906|AAAAAAAAKGPKAAAA|607|Cedar Lincoln|Blvd|Suite 50|Green Acres|Carson City|NV|87683|United States|-8|single family| +44907|AAAAAAAALGPKAAAA|375|Davis Fifth|Court|Suite 40|Green Acres|Cochran County|TX|77683|United States|-6|single family| +44908|AAAAAAAAMGPKAAAA|659|Walnut |ST|Suite E|Bridgeport|Oldham County|KY|45817|United States|-5|apartment| +44909|AAAAAAAANGPKAAAA|597|Lee |Avenue|Suite 60|Shelby|Lagrange County|IN|46575|United States|-5|single family| +44910|AAAAAAAAOGPKAAAA||Fourteenth ||||La Salle County|IL|60059|United States||condo| +44911|AAAAAAAAPGPKAAAA|696|Lincoln View|Road|Suite F|Arlington|Fauquier County|VA|26557|United States|-5|single family| +44912|AAAAAAAAAHPKAAAA|37|Valley |Way|Suite 30|Lawrenceville|Jewell County|KS|64462|United States|-6|single family| +44913|AAAAAAAABHPKAAAA|702|4th |Way|Suite P|Hillcrest|Otero County|NM|83003|United States|-7|single family| +44914|AAAAAAAACHPKAAAA|194|12th Lincoln|Pkwy|Suite H|Plainview|Noble County|IN|43683|United States|-5|single family| +44915|AAAAAAAADHPKAAAA|996|Eigth 5th|Parkway|Suite Q|White Hall|Renville County|ND|56955|United States|-6|single family| +44916|AAAAAAAAEHPKAAAA|247|15th |Parkway|Suite X|Edgewood|Baylor County|TX|70069|United States|-6|single family| +44917|AAAAAAAAFHPKAAAA|456|Center |Ave|Suite S|Riverview|Benton County|AR|79003|United States|-6|condo| +44918|AAAAAAAAGHPKAAAA|971|Pine Park|Dr.|Suite C|Macon|Bledsoe County|TN|30369|United States|-5|apartment| +44919|AAAAAAAAHHPKAAAA|561|Hill |Pkwy|Suite 360|Jamestown|Schleicher County|TX|76867|United States|-6|apartment| +44920|AAAAAAAAIHPKAAAA|976|Spruce |Way|Suite 250|Kingston|Houston County|MN|54975|United States|-6|single family| +44921|AAAAAAAAJHPKAAAA|437|Second |ST|Suite 410|Springfield|Hunt County|TX|79303|United States|-6|apartment| +44922|AAAAAAAAKHPKAAAA|796|Ridge |Circle|Suite D|Providence|Griggs County|ND|56614|United States|-6|condo| +44923|AAAAAAAALHPKAAAA|674|10th |Circle|Suite I|Clinton|Mecosta County|MI|48222|United States|-5|condo| +44924|AAAAAAAAMHPKAAAA|575|5th Walnut|Parkway|Suite 360|Wilson|Dewey County|OK|76971|United States|-6|single family| +44925|AAAAAAAANHPKAAAA|17|Ash Park|Cir.|Suite 180|Arlington|Kiowa County|CO|86557|United States|-7|condo| +44926|AAAAAAAAOHPKAAAA|456|Maple Oak|Wy|Suite C|Montague|Livingston Parish|LA|74062|United States|-6|condo| +44927|AAAAAAAAPHPKAAAA|840|College |Street|Suite 90|Belmont|Eddy County|NM|80191|United States|-7|apartment| +44928|AAAAAAAAAIPKAAAA|283|Lincoln Third|Circle|Suite I|Newtown|Pottawatomie County|OK|71749|United States|-6|condo| +44929|AAAAAAAABIPKAAAA|667|Main Eigth|Cir.|Suite 410|Green Acres|Richland County|IL|67683|United States|-6|single family| +44930|AAAAAAAACIPKAAAA|203|Park |Lane|Suite 190|Greenfield|Lehigh County|PA|15038|United States|-5|single family| +44931|AAAAAAAADIPKAAAA|54|Meadow |Dr.|Suite 380|Kingston|Roane County|TN|34975|United States|-6|single family| +44932|AAAAAAAAEIPKAAAA|35|Green Ash|Parkway|Suite U|Hopewell|Cumberland County|KY|40587|United States|-6|single family| +44933|AAAAAAAAFIPKAAAA|251|Elm Church|RD|Suite 210|Forest Hills|Mason County|TX|79237|United States|-6|condo| +44934|AAAAAAAAGIPKAAAA|570|View |Wy|Suite 310|Spring Hill|Richland County|OH|46787|United States|-5|single family| +44935|AAAAAAAAHIPKAAAA|337|Davis |Wy|Suite F|Georgetown|San Juan County|NM|87057|United States|-7|condo| +44936|AAAAAAAAIIPKAAAA|16|Lake Poplar|Court|Suite L|Oak Hill|Washoe County|NV|87838|United States|-8|condo| +44937|AAAAAAAAJIPKAAAA|230|Meadow |Ave|Suite 20|Greenfield|Cumberland County|KY|45038|United States|-6|single family| +44938|AAAAAAAAKIPKAAAA|271|Madison 3rd|Cir.|Suite X|Clinton|Sweet Grass County|MT|68222|United States|-7|single family| +44939|AAAAAAAALIPKAAAA|486|1st 10th|Lane|Suite 60|Walnut Grove|Frio County|TX|77752|United States|-6|apartment| +44940|AAAAAAAAMIPKAAAA|836|Locust Railroad|Circle|Suite N|Garrison|Cass County|IN|48767|United States|-5|single family| +44941|AAAAAAAANIPKAAAA|880|3rd Laurel|Cir.|Suite 170|Newtown|Clay County|MO|61749|United States|-6|single family| +44942|AAAAAAAAOIPKAAAA|926|Tenth Cherry|Drive|Suite 60|Highland Park|Effingham County|GA|36534|United States|-5|apartment| +44943|AAAAAAAAPIPKAAAA|370|3rd 9th|Lane|Suite V|Antioch|Guthrie County|IA|58605|United States|-6|condo| +44944|AAAAAAAAAJPKAAAA|385|3rd |Boulevard|Suite 430|Jamestown|Rio Arriba County|NM|86867|United States|-7|apartment| +44945|AAAAAAAABJPKAAAA|710|14th Lee||||||81757||-7|| +44946|AAAAAAAACJPKAAAA|463|Second |Ct.|Suite 270|Snug Harbor|Warren County|PA|17936|United States|-5|apartment| +44947|AAAAAAAADJPKAAAA|520|Pine |Ln|Suite I|Providence|Oxford County|ME|07214|United States|-5|condo| +44948|AAAAAAAAEJPKAAAA|640|South Ridge|Ln|Suite E|Five Points|Red Lake County|MN|56098|United States|-6|condo| +44949|AAAAAAAAFJPKAAAA|166|West 5th|Avenue|Suite S|Kingston|Cascade County|MT|64975|United States|-7|condo| +44950|AAAAAAAAGJPKAAAA|612|Woodland Hillcrest|RD|Suite 230|Union Hill|Guadalupe County|NM|87746|United States|-7|single family| +44951|AAAAAAAAHJPKAAAA|11|South Highland|Parkway|Suite 490|Greenville|Tillman County|OK|71387|United States|-6|condo| +44952|AAAAAAAAIJPKAAAA|843|Park |Drive|Suite H|Spring Hill|Cortland County|NY|16787|United States|-5|condo| +44953|AAAAAAAAJJPKAAAA|201|Ridge |Drive|Suite P|Spring Valley|Corson County|SD|56060|United States|-6|condo| +44954|AAAAAAAAKJPKAAAA|232|Sunset Willow|Road|Suite 370|Deerfield|Jackson County|AR|79840|United States|-6|condo| +44955|AAAAAAAALJPKAAAA|677|Spring |Pkwy|Suite 340|Glenwood|King County|WA|93511|United States|-8|single family| +44956|AAAAAAAAMJPKAAAA|799|Mill 1st|Pkwy|Suite 480|Ashland|White County|AR|74244|United States|-6|condo| +44957|AAAAAAAANJPKAAAA|258|Sixth Maple|ST|Suite S|Providence|Monroe County|IN|46614|United States|-5|apartment| +44958|AAAAAAAAOJPKAAAA|627|Lee Walnut|Avenue|Suite E|Sunnyside|Warren County|PA|11952|United States|-5|single family| +44959|AAAAAAAAPJPKAAAA|142|Valley |Way|Suite V|Lakeside|Audrain County|MO|69532|United States|-6|apartment| +44960|AAAAAAAAAKPKAAAA|654|Locust |Parkway|Suite 420|Wilton|Scioto County|OH|46997|United States|-5|condo| +44961|AAAAAAAABKPKAAAA|411|Highland |Drive|Suite O|Deerfield|Kershaw County|SC|29840|United States|-5|condo| +44962|AAAAAAAACKPKAAAA|268|11th Jefferson|RD|Suite 450|Wilson|Sublette County|WY|86971|United States|-7|condo| +44963|AAAAAAAADKPKAAAA|771|Ash |ST|Suite E|White Oak|Box Butte County|NE|66668|United States|-6|condo| +44964|AAAAAAAAEKPKAAAA|250|Hillcrest |Lane|Suite 210|Greenwood|Fulton County|GA|38828|United States|-5|apartment| +44965|AAAAAAAAFKPKAAAA|134|Ash |Blvd|Suite V|Five Forks|Bladen County|NC|22293|United States|-5|condo| +44966|AAAAAAAAGKPKAAAA|406|10th |RD|Suite 410|Oakland|Lake County|SD|59843|United States|-7|condo| +44967|AAAAAAAAHKPKAAAA|712|Jackson |Circle|Suite 210|Deerfield|Whatcom County|WA|99840|United States|-8|condo| +44968|AAAAAAAAIKPKAAAA|603|1st Highland|Wy|Suite J|Jackson|Madison County|MO|69583|United States|-6|single family| +44969|AAAAAAAAJKPKAAAA|579|Wilson Laurel|Avenue|Suite B|Stringtown|Meriwether County|GA|30162|United States|-5|apartment| +44970|AAAAAAAAKKPKAAAA|761|Park |ST|Suite G|Pine Grove|Platte County|WY|84593|United States|-7|apartment| +44971|AAAAAAAALKPKAAAA|726|Willow Washington|Court|Suite U|Highland Park|McDowell County|NC|26534|United States|-5|apartment| +44972|AAAAAAAAMKPKAAAA|643|Davis Oak|Ct.|Suite 360|Oakland|Faulkner County|AR|79843|United States|-6|apartment| +44973|AAAAAAAANKPKAAAA|391|Lakeview |Circle|Suite V|Antioch|Baltimore County|MD|28605|United States|-5|condo| +44974|AAAAAAAAOKPKAAAA|574|Chestnut Oak|Parkway|Suite 360|Liberty|Teton County|WY|83451|United States|-7|apartment| +44975|AAAAAAAAPKPKAAAA|134|Broadway Sixth|Court|Suite 240|Mount Zion|Knox County|NE|68054|United States|-7|condo| +44976|AAAAAAAAALPKAAAA|753|Adams |Drive|Suite M|Mountain View|Williamsburg County|SC|24466|United States|-5|single family| +44977|AAAAAAAABLPKAAAA|569|Ridge |Ct.|Suite 80|Springfield|Chilton County|AL|39303|United States|-6|single family| +44978|AAAAAAAACLPKAAAA|635|Davis Lakeview|Pkwy|Suite S|Mount Zion|Sublette County|WY|88054|United States|-7|apartment| +44979|AAAAAAAADLPKAAAA|309|5th |Circle|Suite 0|Lebanon|Roanoke County|VA|22898|United States|-5|single family| +44980|AAAAAAAAELPKAAAA|4|Pine |Cir.|Suite K|Shore Acres|Kinney County|TX|72724|United States|-6|apartment| +44981|AAAAAAAAFLPKAAAA|386|||Suite L||Cumberland County|||United States||apartment| +44982|AAAAAAAAGLPKAAAA|579|Lake Hickory|Street|Suite 470|Kirkland|Brown County|IN|47896|United States|-5|condo| +44983|AAAAAAAAHLPKAAAA|966|Park Main|Drive|Suite 180|Providence|Scott County|MS|56614|United States|-6|apartment| +44984|AAAAAAAAILPKAAAA|72|Lake Park|ST|Suite R|Oak Ridge|Cheyenne County|KS|68371|United States|-6|condo| +44985|AAAAAAAAJLPKAAAA|479|5th |Road|Suite 360|Clinton|Duplin County|NC|28222|United States|-5|single family| +44986|AAAAAAAAKLPKAAAA|87|North |Blvd|Suite 110|Stringtown|Greenwood County|SC|20162|United States|-5|apartment| +44987|AAAAAAAALLPKAAAA|473|Jackson Walnut|Street|Suite J|Springdale|Cattaraugus County|NY|18883|United States|-5|single family| +44988|AAAAAAAAMLPKAAAA|438|View |Parkway|Suite 430|Lakewood|Gilmer County|WV|28877|United States|-5|condo| +44989|AAAAAAAANLPKAAAA|756|Johnson 8th|Blvd|Suite K|Oak Grove|Bear Lake County|ID|88370|United States|-7|apartment| +44990|AAAAAAAAOLPKAAAA|137|1st |Parkway|Suite 390|Mount Olive|Cherokee County|OK|78059|United States|-6|condo| +44991|AAAAAAAAPLPKAAAA|460|Park Church|Dr.|Suite 350|Cedar Grove|Nicholas County|WV|20411|United States|-5|single family| +44992|AAAAAAAAAMPKAAAA|576|Chestnut |Way|Suite 150|Farmington|Hartley County|TX|79145|United States|-6|single family| +44993|AAAAAAAABMPKAAAA|982|Walnut Church|Drive|Suite W|Pleasant Valley|Taney County|MO|62477|United States|-6|apartment| +44994|AAAAAAAACMPKAAAA|540|Park 1st|Ct.|Suite 440|Newtown|Clark County|IN|41749|United States|-5|single family| +44995|AAAAAAAADMPKAAAA|742|Park |Drive|Suite 220|Mount Pleasant|Loudoun County|VA|21933|United States|-5|apartment| +44996|AAAAAAAAEMPKAAAA|70|Valley First|Street|Suite 410|Spring Valley|Lincoln County|KS|66060|United States|-6|apartment| +44997|AAAAAAAAFMPKAAAA|332|Jackson 12th|ST|Suite L|Wildwood|Van Wert County|OH|46871|United States|-5|single family| +44998|AAAAAAAAGMPKAAAA|966|3rd Eigth|Lane|Suite M|La Grange|Crawford County|AR|77941|United States|-6|apartment| +44999|AAAAAAAAHMPKAAAA|735|Hill 4th|Dr.|Suite N|Valley View|Beaver County|OK|75124|United States|-6|condo| +45000|AAAAAAAAIMPKAAAA|625|12th Dogwood|Drive|Suite T|Riverside|Sabine County|TX|79231|United States|-6|apartment| +45001|AAAAAAAAJMPKAAAA|294|4th |Ave|Suite 420|Springdale|Union County|IA|58883|United States|-6|single family| +45002|AAAAAAAAKMPKAAAA|729|Jefferson |Circle|Suite L|Lakeview|Hickory County|MO|68579|United States|-6|single family| +45003|AAAAAAAALMPKAAAA|40|Highland Second|Ave|Suite C|Clifton|Wyandot County|OH|48014|United States|-5|condo| +45004|AAAAAAAAMMPKAAAA|473|Sixth First|Boulevard|Suite L|Jamestown|Midland County|TX|76867|United States|-6|apartment| +45005|AAAAAAAANMPKAAAA|984|Center |Road|Suite 410|Georgetown|Washtenaw County|MI|47057|United States|-5|apartment| +45006|AAAAAAAAOMPKAAAA|785|Jackson |Wy|Suite A|Macedonia|Cannon County|TN|31087|United States|-5|condo| +45007|AAAAAAAAPMPKAAAA|238|Woodland |Wy|Suite O|Hopewell|Cedar County|IA|50587|United States|-6|apartment| +45008|AAAAAAAAANPKAAAA|998|Park Meadow|Street|Suite 360|Mount Pleasant|Lancaster County|PA|11933|United States|-5|condo| +45009|AAAAAAAABNPKAAAA|432|Hill |Circle|Suite 440|Maywood|Noble County|OK|75681|United States|-6|single family| +45010|AAAAAAAACNPKAAAA|460|Fourth Maple|Boulevard|Suite O|Riverside|Chippewa County|MI|49231|United States|-5|condo| +45011|AAAAAAAADNPKAAAA|807|Washington Fourth|Boulevard|Suite 220|Edgewood|Yellow Medicine County|MN|50069|United States|-6|apartment| +45012|AAAAAAAAENPKAAAA|616|6th Mill|Ln|Suite 460|Midway|Lyon County|MN|51904|United States|-6|condo| +45013|AAAAAAAAFNPKAAAA|105|Woodland |Ln|Suite 150|Oak Ridge|Stevens County|WA|98371|United States|-8|single family| +45014|AAAAAAAAGNPKAAAA|149|9th |Street|Suite D|Mount Zion|Aitkin County|MN|58054|United States|-6|single family| +45015|AAAAAAAAHNPKAAAA|395|5th Third|Wy|Suite 20|Harmony|Sumter County|FL|35804|United States|-5|condo| +45016|AAAAAAAAINPKAAAA|843|Lake |Wy|Suite C|Cedar Grove|Putnam County|MO|60411|United States|-6|apartment| +45017|AAAAAAAAJNPKAAAA|669|Smith 2nd|Drive|Suite O|Wildwood|Christian County|KY|46871|United States|-6|apartment| +45018|AAAAAAAAKNPKAAAA|737|5th |Drive|Suite K|Clearview|Allen County|KS|65495|United States|-6|apartment| +45019|AAAAAAAALNPKAAAA|570|8th View|Road|Suite 360|Springfield|Mecosta County|MI|49303|United States|-5|apartment| +45020|AAAAAAAAMNPKAAAA|151|Pine |Ave|Suite 480|Pleasant Grove|Logan County|OH|44136|United States|-5|condo| +45021|AAAAAAAANNPKAAAA|352|Hickory 1st|Ct.|Suite 420|White Oak|Greene County|IA|56668|United States|-6|condo| +45022|AAAAAAAAONPKAAAA|188|Eigth 7th|Lane|Suite 350|Mount Zion|Grundy County|MO|68054|United States|-6|condo| +45023|AAAAAAAAPNPKAAAA||10th Sunset||||Wyoming County||21904|United States|-5|| +45024|AAAAAAAAAOPKAAAA|69|Spring |Boulevard|Suite 470|Clinton|Ben Hill County|GA|38222|United States|-5|condo| +45025|AAAAAAAABOPKAAAA|780|4th |Avenue|Suite C|Fayetteville|Fond du Lac County|WI|51732|United States|-6|apartment| +45026|AAAAAAAACOPKAAAA|404|Oak Green|RD|Suite B|Springfield|Johnson County|MO|69303|United States|-6|single family| +45027|AAAAAAAADOPKAAAA|604|10th 7th|Blvd|Suite 370|Spring Valley|Boone County|AR|76060|United States|-6|condo| +45028|AAAAAAAAEOPKAAAA|667|College |Way|Suite Y|Unionville|Yazoo County|MS|51711|United States|-6|single family| +45029|AAAAAAAAFOPKAAAA|570|River |Street|Suite 350|Enterprise|Knox County|TN|31757|United States|-6|condo| +45030|AAAAAAAAGOPKAAAA|749|Lee Ridge|Court|Suite 90|Cedar|Sherman County|OR|91229|United States|-8|apartment| +45031|AAAAAAAAHOPKAAAA|124|Hickory View|Avenue|Suite S|Sulphur Springs|Pleasants County|WV|28354|United States|-5|apartment| +45032|AAAAAAAAIOPKAAAA|839|6th Railroad|Lane|Suite O|Antioch|New Haven County|CT|09205|United States|-5|apartment| +45033|AAAAAAAAJOPKAAAA|691|12th |Way|Suite 200|Pleasant Hill|Wayne County|PA|13604|United States|-5|single family| +45034|AAAAAAAAKOPKAAAA||||Suite O||Crowley County|||United States|-7|condo| +45035|AAAAAAAALOPKAAAA||Highland Ridge|Road|Suite 30|||NY|||-5|apartment| +45036|AAAAAAAAMOPKAAAA|595|Pine |Blvd|Suite A|Woodland|Bedford County|VA|24854|United States|-5|single family| +45037|AAAAAAAANOPKAAAA|1000|Walnut |Ct.|Suite R|Five Forks|Jackson County|AL|32293|United States|-6|single family| +45038|AAAAAAAAOOPKAAAA|33|3rd |Ct.|Suite Y|Owens|Ochiltree County|TX|72324|United States|-6|single family| +45039|AAAAAAAAPOPKAAAA|592|View |Avenue|Suite U|Hillcrest|Searcy County|AR|73003|United States|-6|single family| +45040|AAAAAAAAAPPKAAAA||Jefferson Wilson|Ct.|Suite T||||69843|||| +45041|AAAAAAAABPPKAAAA|701|Wilson |Wy|Suite 340|Stratford|Newton County|TX|76668|United States|-6|condo| +45042|AAAAAAAACPPKAAAA|417|View |Road|Suite 320|Maple Grove|York County|PA|18252|United States|-5|apartment| +45043|AAAAAAAADPPKAAAA|876|Park 12th|Blvd||||||United States|-6|| +45044|AAAAAAAAEPPKAAAA||Birch View||Suite 410|Sunnyside|Dubois County|IN|41952||-5|| +45045|AAAAAAAAFPPKAAAA|964|First |Ct.|Suite 330|Florence|Cumberland County|NJ|03994|United States|-5|condo| +45046|AAAAAAAAGPPKAAAA|369|Wilson |RD|Suite 420|Riverdale|Lauderdale County|TN|39391|United States|-6|apartment| +45047|AAAAAAAAHPPKAAAA|800|Broadway Pine|Dr.|Suite V|Delmar|Giles County|TN|33957|United States|-5|condo| +45048|AAAAAAAAIPPKAAAA|590|North |Circle|Suite 160|Bethel|Mono County|CA|95281|United States|-8|single family| +45049|AAAAAAAAJPPKAAAA|206|Hickory |Circle|Suite 470|Lincoln|Clinton County|IA|51289|United States|-6|single family| +45050|AAAAAAAAKPPKAAAA|65|Center |Way|Suite M|Sulphur Springs|Cheyenne County|KS|68354|United States|-6|single family| +45051|AAAAAAAALPPKAAAA|888|8th Cedar|Way|Suite 110|Forest Hills|Multnomah County|OR|99237|United States|-8|condo| +45052|AAAAAAAAMPPKAAAA|517|Main |Street|Suite 350|Walnut Grove|Goodhue County|MN|57752|United States|-6|single family| +45053|AAAAAAAANPPKAAAA|747|Lake Maple|Ln|Suite 170|Hopewell|Noble County|OK|70587|United States|-6|single family| +45054|AAAAAAAAOPPKAAAA|19|Mill |Road|Suite 130|Pleasant Valley|Montgomery County|IL|62477|United States|-6|condo| +45055|AAAAAAAAPPPKAAAA|328|Meadow |Road|Suite 20|Springfield|Williamson County|IL|69303|United States|-6|apartment| +45056|AAAAAAAAAAALAAAA|966|Third |Circle|Suite 40|Woodville|Mobile County|AL|34289|United States|-6|apartment| +45057|AAAAAAAABAALAAAA|725|South |ST|Suite 0|Highland Park|Dooly County|GA|36534|United States|-5|single family| +45058|AAAAAAAACAALAAAA|904|Willow First|Boulevard|Suite M|Sunnyside|Barnes County|ND|51952|United States|-6|condo| +45059|AAAAAAAADAALAAAA|491|1st Highland|Cir.|Suite 130|Vance|Penobscot County|ME|00868|United States|-5|apartment| +45060|AAAAAAAAEAALAAAA|806|Cherry 11th|Wy|Suite X|White Plains|Meade County|SD|56622|United States|-7|single family| +45061|AAAAAAAAFAALAAAA|873|Ridge |Parkway|Suite Y|Plainview|Johnson County|IL|63683|United States|-6|single family| +45062|AAAAAAAAGAALAAAA|417|Ridge 9th|Court|Suite 20|Pine Grove|Palo Pinto County|TX|74593|United States|-6|apartment| +45063|AAAAAAAAHAALAAAA|737|Jefferson |RD|Suite C|Cedar Grove|Black Hawk County|IA|50411|United States|-6|single family| +45064|AAAAAAAAIAALAAAA|805|Smith |RD|Suite 280|Concord|Davidson County|TN|34107|United States|-5|single family| +45065|AAAAAAAAJAALAAAA|95|Williams |Pkwy|Suite J|Mount Pleasant|Johnson County|KS|61933|United States|-6|apartment| +45066|AAAAAAAAKAALAAAA|262|1st |Avenue|Suite 250|Marion|Fountain County|IN|40399|United States|-5|apartment| +45067|AAAAAAAALAALAAAA|580|Smith Hill|Road|Suite 460|Providence|Haskell County|KS|66614|United States|-6|single family| +45068|AAAAAAAAMAALAAAA|||Ln|Suite 200|Walnut Grove|||37752||-5|| +45069|AAAAAAAANAALAAAA|97|1st |Wy|Suite 300|Buena Vista|Merrick County|NE|65752|United States|-7|condo| +45070|AAAAAAAAOAALAAAA|331|Cedar 9th|Wy|Suite M|Lebanon|Tehama County|CA|92898|United States|-8|condo| +45071|AAAAAAAAPAALAAAA|392|Maple Main|Court|Suite 110|Woodland|Edmonson County|KY|44854|United States|-6|apartment| +45072|AAAAAAAAABALAAAA|951|Hickory Valley|Parkway|Suite 130|Elba|Montgomery County|MD|20262|United States|-5|single family| +45073|AAAAAAAABBALAAAA|275|Meadow Davis|Blvd|Suite 400|Waterloo|Page County|VA|21675|United States|-5|apartment| +45074|AAAAAAAACBALAAAA|32|Forest |Cir.|Suite I|Highland Park|Webster County|GA|36534|United States|-5|single family| +45075|AAAAAAAADBALAAAA|426|Washington |Boulevard|Suite O||Graham County|NC||United States||| +45076|AAAAAAAAEBALAAAA|639|Williams |Ln|Suite 200|Riverview|Florence County|WI|59003|United States|-6|apartment| +45077|AAAAAAAAFBALAAAA|956|Williams |RD|Suite M|Greenville|Thomas County|GA|31387|United States|-5|apartment| +45078|AAAAAAAAGBALAAAA|469|Oak |Drive|Suite Q|Pine Grove|Trigg County|KY|44593|United States|-5|apartment| +45079|AAAAAAAAHBALAAAA|856|Elm |RD|Suite A|Marion|Harrison County|MS|50399|United States|-6|single family| +45080|AAAAAAAAIBALAAAA|294|Elevnth Center|Road|Suite H|Plainview|Muskingum County|OH|43683|United States|-5|apartment| +45081|AAAAAAAAJBALAAAA|595|Oak |Ct.|Suite 370|Hamilton|Douglas County|OR|92808|United States|-8|apartment| +45082|AAAAAAAAKBALAAAA|349|Mill |Pkwy|Suite P|Jamestown|Koochiching County|MN|56867|United States|-6|apartment| +45083|AAAAAAAALBALAAAA|439|3rd 5th|Ave|Suite 240|Granite|Grant County|OK|76284|United States|-6|condo| +45084|AAAAAAAAMBALAAAA|683|Oak Highland|Ln|Suite C|Greenville|Tripp County|SD|51387|United States|-7|condo| +45085|AAAAAAAANBALAAAA||Jefferson |Circle||||AK||||| +45086|AAAAAAAAOBALAAAA|590|South Lake|Dr.|Suite X|Plainview|Crook County|WY|83683|United States|-7|single family| +45087|AAAAAAAAPBALAAAA|581|Smith |Lane|Suite 230|Newport|Valdez-Cordova Census Area|AK|91521|United States|-9|condo| +45088|AAAAAAAAACALAAAA|730|Dogwood Third|Avenue|Suite H|Frenchtown|Madison County|OH|42629|United States|-5|single family| +45089|AAAAAAAABCALAAAA|822|Elevnth |Street|Suite K|Valley View|Orleans Parish|LA|75124|United States|-6|apartment| +45090|AAAAAAAACCALAAAA|670|Valley Lincoln|Way|Suite H|Shiloh|Atchison County|MO|69275|United States|-6|apartment| +45091|AAAAAAAADCALAAAA|693|Walnut View|Ct.|Suite S|Stringtown|Pickens County|SC|20162|United States|-5|single family| +45092|AAAAAAAAECALAAAA|199|1st Central|Road|Suite 270|Riverside|Brown County|IN|49231|United States|-5|condo| +45093|AAAAAAAAFCALAAAA|726|Woodland 11th|Ct.|Suite A|Springdale|Day County|SD|58883|United States|-6|apartment| +45094|AAAAAAAAGCALAAAA|705|Sycamore 1st|Road|Suite 490|Leesville|Twin Falls County|ID|85423|United States|-7|apartment| +45095|AAAAAAAAHCALAAAA|237|Laurel |Cir.|Suite 300|Midway|Steuben County|NY|11904|United States|-5|apartment| +45096|AAAAAAAAICALAAAA|63|Park North|Road|Suite 60|Hopewell|Caldwell County|NC|20587|United States|-5|single family| +45097|AAAAAAAAJCALAAAA|847|West |Cir.|Suite 30|Hillcrest|Bates County|MO|63003|United States|-6|apartment| +45098|AAAAAAAAKCALAAAA|26|Maple East|Wy|Suite C|Newport|Haywood County|NC|21521|United States|-5|condo| +45099|AAAAAAAALCALAAAA|378|Maple Lake|Ct.|Suite 260|Oak Ridge|Itasca County|MN|58371|United States|-6|apartment| +45100|AAAAAAAAMCALAAAA|739|Second Second|Boulevard|Suite U|Crossroads|Santa Barbara County|CA|90534|United States|-8|single family| +45101|AAAAAAAANCALAAAA|942|Elm View|Way|Suite C|Glendale|Poinsett County|AR|73951|United States|-6|apartment| +45102|AAAAAAAAOCALAAAA|239|Hill Walnut|Court|Suite 220|Newport|Kalamazoo County|MI|41521|United States|-5|apartment| +45103|AAAAAAAAPCALAAAA|182|Hillcrest First|Circle|Suite 410|Mount Zion|Douglas County|NV|88054|United States|-8|condo| +45104|AAAAAAAAADALAAAA|118|Forest Williams|RD|Suite 90|New Hope|Brown County|NE|69431|United States|-6|apartment| +45105|AAAAAAAABDALAAAA|748|Cherry |Dr.|Suite S|Springfield|Southampton County|VA|29303|United States|-5|single family| +45106|AAAAAAAACDALAAAA|304|Spring Poplar|Boulevard|Suite 150|Lakewood|Amelia County|VA|28877|United States|-5|apartment| +45107|AAAAAAAADDALAAAA|217|View |Ct.|Suite O|Belmont|Monroe County|IL|60191|United States|-6|condo| +45108|AAAAAAAAEDALAAAA|555|Maple |Road|Suite 170|Belmont|Perry County|PA|10191|United States|-5|condo| +45109|AAAAAAAAFDALAAAA|154|||Suite O|Brownsville|Grand Traverse County|MI||||| +45110|AAAAAAAAGDALAAAA|3|Park Cherry|Way|Suite 470|Buena Vista|Hidalgo County|NM|85752|United States|-7|single family| +45111|AAAAAAAAHDALAAAA|465|5th |Ct.|Suite 50|Oak Grove|Lampasas County|TX|78370|United States|-6|single family| +45112|AAAAAAAAIDALAAAA|975|3rd |Boulevard|Suite Q|Mount Olive|Medina County|TX|78059|United States|-6|apartment| +45113|AAAAAAAAJDALAAAA|566|College 1st|Avenue|Suite 420|Union Hill|Lake County|OR|97746|United States|-8|condo| +45114|AAAAAAAAKDALAAAA|373|Ridge |Way|Suite 460|Lebanon|Todd County|KY|42898|United States|-5|single family| +45115|AAAAAAAALDALAAAA|946|Lincoln |RD|Suite 440|Reno|Moody County|SD|50344|United States|-7|single family| +45116|AAAAAAAAMDALAAAA|477|Mill |Ct.|Suite G|Riverside|Elko County|NV|89231|United States|-8|apartment| +45117|AAAAAAAANDALAAAA|82|Jefferson Pine|Circle|Suite R|Woodville|Highland County|VA|24289|United States|-5|condo| +45118|AAAAAAAAODALAAAA|916|Poplar Main|Dr.|Suite Q|Salem|Clatsop County|OR|98048|United States|-8|condo| +45119|AAAAAAAAPDALAAAA|858|Cedar Birch|Parkway|Suite U|Woodland|Carbon County|UT|84854|United States|-7|condo| +45120|AAAAAAAAAEALAAAA|540|Lakeview |Cir.|Suite X|Glendale|Valley County|NE|63951|United States|-7|apartment| +45121|AAAAAAAABEALAAAA|920|Locust Wilson|Road|Suite 90|Brownsville|Dodge County|GA|39310|United States|-5|single family| +45122|AAAAAAAACEALAAAA|715|Sunset |Street|Suite T|Rocky Point|Adams County|IA|51209|United States|-6|apartment| +45123|AAAAAAAADEALAAAA|635|Ridge |Way|Suite Q|Liberty|Tazewell County|VA|23451|United States|-5|single family| +45124|AAAAAAAAEEALAAAA|91|Railroad |Ln|Suite 450|Marion|Alpine County|CA|90399|United States|-8|apartment| +45125|AAAAAAAAFEALAAAA|925|Smith Lee|Drive|Suite 310|Webb|Williams County|ND|50899|United States|-6|apartment| +45126|AAAAAAAAGEALAAAA|691|Ash |Dr.|Suite 480|Mount Olive|Lehigh County|PA|18059|United States|-5|apartment| +45127|AAAAAAAAHEALAAAA|751|Willow |RD|Suite 490|Valley View|Montgomery County|TX|75124|United States|-6|apartment| +45128|AAAAAAAAIEALAAAA|678|Laurel |Pkwy|Suite 410|Wilson|Ottawa County|OK|76971|United States|-6|single family| +45129|AAAAAAAAJEALAAAA|214|Ash |ST|Suite 240|Griffin|Lake County|IL|61204|United States|-6|single family| +45130|AAAAAAAAKEALAAAA|693|Sixth 10th|Dr.|Suite N|Oak Grove|Bullitt County|KY|48370|United States|-6|apartment| +45131|AAAAAAAALEALAAAA|698|5th Cherry|Way|Suite P|Unionville|Monmouth County|NJ|02311|United States|-5|single family| +45132|AAAAAAAAMEALAAAA|772||||Fairview|||45709|||condo| +45133|AAAAAAAANEALAAAA|668|Ash Lake|ST|Suite G|Summit|Carter County|MT|60499|United States|-7|condo| +45134|AAAAAAAAOEALAAAA|180|Pine |Pkwy|Suite N|Brownsville|Marion County|FL|39310|United States|-5|single family| +45135|AAAAAAAAPEALAAAA|246|View 10th|Street|Suite S|Buena Vista|Clark County|WI|55752|United States|-6|condo| +45136|AAAAAAAAAFALAAAA|984|River Pine|Dr.|Suite 440|Oakwood|Webster County|WV|20169|United States|-5|single family| +45137|AAAAAAAABFALAAAA|75|Oak |Dr.|Suite R|Woodland|Attala County|MS|54854|United States|-6|condo| +45138|AAAAAAAACFALAAAA|228|Hickory 1st|Road|Suite 90|Edgewood|Hot Springs County|WY|80069|United States|-7|single family| +45139|AAAAAAAADFALAAAA|210|Oak |Wy|Suite R|Pleasant Hill|Kern County|CA|93604|United States|-8|condo| +45140|AAAAAAAAEFALAAAA|60|Lakeview |Court|Suite 100|Highland Park|Panola County|MS|56534|United States|-6|condo| +45141|AAAAAAAAFFALAAAA||5th Birch||Suite 270|Harvey||TN||United States||| +45142|AAAAAAAAGFALAAAA|943|Oak |Ave|Suite D|Sulphur Springs|Van Buren County|TN|38354|United States|-6|single family| +45143|AAAAAAAAHFALAAAA|340|Laurel Woodland|Lane|Suite 300|Lakeside|Claiborne County|TN|39532|United States|-5|apartment| +45144|AAAAAAAAIFALAAAA|451|Sycamore |Ln|Suite 20|Bunker Hill|Ada County|ID|80150|United States|-7|single family| +45145|AAAAAAAAJFALAAAA|560|10th |Court|Suite 80|White Oak|Barren County|KY|46668|United States|-6|condo| +45146|AAAAAAAAKFALAAAA|807|Poplar |Ct.|Suite 310|Oakdale|Real County|TX|79584|United States|-6|apartment| +45147|AAAAAAAALFALAAAA|680|5th Park|Circle|Suite 390|Pleasant Hill|Knox County|TN|33604|United States|-6|apartment| +45148|AAAAAAAAMFALAAAA|913|First |Boulevard|Suite 90|Lakeside|Baca County|CO|89532|United States|-7|single family| +45149|AAAAAAAANFALAAAA|82|Elm |Avenue|Suite 280|Wildwood|Calhoun County|IA|56871|United States|-6|single family| +45150|AAAAAAAAOFALAAAA|529|Highland 10th|Way|Suite 130|Union Hill|Caldwell County|NC|27746|United States|-5|apartment| +45151|AAAAAAAAPFALAAAA|900|River |Parkway|Suite M|Salem|Harrison County|TX|78048|United States|-6|condo| +45152|AAAAAAAAAGALAAAA|514|15th |Way|Suite I|Oakland|Box Elder County|UT|89843|United States|-7|single family| +45153|AAAAAAAABGALAAAA|843|5th Smith|Dr.|Suite 320|Florence|Butler County|KS|63394|United States|-6|apartment| +45154|AAAAAAAACGALAAAA|871|South |Cir.|Suite C|Florence|Wayne County|UT|83394|United States|-7|single family| +45155|AAAAAAAADGALAAAA|206|Park Lee|Drive|Suite Q|Wilson|Randolph County|WV|26971|United States|-5|condo| +45156|AAAAAAAAEGALAAAA|675|Lincoln Sunset|Court|Suite 400|Bloomingdale|Scott County|VA|21824|United States|-5|condo| +45157|AAAAAAAAFGALAAAA|128|2nd |ST|Suite H|Ashland|Buchanan County|MO|64244|United States|-6|condo| +45158|AAAAAAAAGGALAAAA|348|10th River|Avenue|Suite U|Crossroads|Morris County|NJ|01134|United States|-5|condo| +45159|AAAAAAAAHGALAAAA|665|5th |ST|Suite D|Hopewell|Halifax County|VA|20587|United States|-5|condo| +45160|AAAAAAAAIGALAAAA||||Suite R|Hartland|Dickinson County|MI|46594|||condo| +45161|AAAAAAAAJGALAAAA|101|Pine Willow|Dr.|Suite 330|Glenwood|Baldwin County|GA|33511|United States|-5|condo| +45162|AAAAAAAAKGALAAAA|305|Smith |Ct.|Suite B|Wilson|West Feliciana Parish|LA|76971|United States|-6|single family| +45163|AAAAAAAALGALAAAA|37|East Pine|ST|Suite 470|Mount Pleasant|Whitley County|IN|41933|United States|-5|single family| +45164|AAAAAAAAMGALAAAA||4th |Pkwy|||Sterling County|TX|70499|||single family| +45165|AAAAAAAANGALAAAA|830|Cedar Sunset|Street|Suite 70|Kingston|Seneca County|OH|44975|United States|-5|single family| +45166|AAAAAAAAOGALAAAA|687|6th |Way|Suite T|Deerfield|Lincoln County|MT|69840|United States|-7|condo| +45167|AAAAAAAAPGALAAAA|616|Railroad |Cir.|Suite 230|Unionville|Washington County|ME|02311|United States|-5|single family| +45168|AAAAAAAAAHALAAAA|836|Johnson Valley|Way|Suite W|Oakwood|Dunn County|ND|50169|United States|-6|apartment| +45169|AAAAAAAABHALAAAA|255|7th Center|Court|Suite 230|Ashland|Gooding County|ID|84244|United States|-7|single family| +45170|AAAAAAAACHALAAAA|143|8th Wilson|Court|Suite W|Pleasant Hill|Tuolumne County|CA|93604|United States|-8|condo| +45171|AAAAAAAADHALAAAA|961|Smith |Cir.|Suite H|Midway|Bremer County|IA|51904|United States|-6|condo| +45172|AAAAAAAAEHALAAAA|249|View |Blvd|Suite 230|Liberty|Harding County|SD|53451|United States|-7|condo| +45173|AAAAAAAAFHALAAAA|331|Green |Drive|Suite 20|Farmington|Red River Parish|LA|79145|United States|-6|condo| +45174|AAAAAAAAGHALAAAA|214|Central |Blvd|Suite 310|Bunker Hill|Webster County|IA|50150|United States|-6|single family| +45175|AAAAAAAAHHALAAAA|833|College |RD|Suite G|Salem|Polk County|TN|38048|United States|-6|single family| +45176|AAAAAAAAIHALAAAA|408|Woodland Center|Ct.|Suite 180|Forest Hills|Hanover County|VA|29237|United States|-5|single family| +45177|AAAAAAAAJHALAAAA|595|6th |Blvd|Suite S|Belleville|Porter County|IN|42924|United States|-5|apartment| +45178|AAAAAAAAKHALAAAA|683|Broadway 5th|Lane|Suite 10|Spring Valley|Miami County|IN|46060|United States|-5|condo| +45179|AAAAAAAALHALAAAA|583|Walnut |Blvd|Suite A|Woodville|Piute County|UT|84289|United States|-7|apartment| +45180|AAAAAAAAMHALAAAA|72|Dogwood |Cir.|Suite 250|Sleepy Hollow|Carroll County|VA|23592|United States|-5|condo| +45181|AAAAAAAANHALAAAA|681|Meadow |Dr.|Suite 160|Rossville|Mahnomen County|MN|52503|United States|-6|single family| +45182|AAAAAAAAOHALAAAA|386|Jackson |Circle|Suite 110|Oakwood|Martin County|KY|40169|United States|-5|apartment| +45183|AAAAAAAAPHALAAAA|26|Main |Circle|Suite T|Oak Grove|Jeff Davis County|GA|38370|United States|-5|single family| +45184|AAAAAAAAAIALAAAA|202|Railroad |Wy|Suite 350|Greenville|LaMoure County|ND|51387|United States|-6|single family| +45185|AAAAAAAABIALAAAA|639|Walnut 1st|Circle|Suite P|Jamestown|Muskegon County|MI|46867|United States|-5|single family| +45186|AAAAAAAACIALAAAA|330|3rd Lincoln|Cir.|Suite 240|White Oak|Holmes County|MS|56668|United States|-6|apartment| +45187|AAAAAAAADIALAAAA|232|Park Wilson|Parkway|Suite 100|New Hope|Gates County|NC|29431|United States|-5|single family| +45188|AAAAAAAAEIALAAAA|85|Dogwood Cherry|Boulevard|Suite 110|Saratoga|Clay County|IL|62123|United States|-6|single family| +45189|AAAAAAAAFIALAAAA|233|Cedar |Drive|Suite I|Providence|Buffalo County|NE|66614|United States|-6|apartment| +45190|AAAAAAAAGIALAAAA|105|1st Main|Avenue|Suite E|Jackson|Menard County|IL|69583|United States|-6|single family| +45191|AAAAAAAAHIALAAAA|907|Lee |Street|Suite 350|Liberty|Anoka County|MN|53451|United States|-6|single family| +45192|AAAAAAAAIIALAAAA|446|Willow Lakeview|Pkwy|Suite J|Glenwood|Lipscomb County|TX|73511|United States|-6|single family| +45193|AAAAAAAAJIALAAAA|677|Seventh Johnson|Way||Stringtown|Pike County|IL|60162||-6|condo| +45194|AAAAAAAAKIALAAAA|107|1st |ST|Suite 240|Oak Hill|Roane County|WV|27838|United States|-5|condo| +45195|AAAAAAAALIALAAAA|53||Boulevard|||Hardee County|FL|30150|||| +45196|AAAAAAAAMIALAAAA|395|2nd |Ct.|Suite 150|Oakdale|Fayette County|KY|49584|United States|-6|apartment| +45197|AAAAAAAANIALAAAA|52|12th Ash|Wy|Suite I|Riverview|Trego County|KS|69003|United States|-6|single family| +45198|AAAAAAAAOIALAAAA|373|Miller |Court|Suite A|Woodlawn|Crowley County|CO|84098|United States|-7|single family| +45199|AAAAAAAAPIALAAAA|689|Davis Elm|Court|Suite R|Shiloh|Echols County|GA|39275|United States|-5|condo| +45200|AAAAAAAAAJALAAAA|1000|Chestnut Williams|Dr.|Suite 330|Waterloo|Ross County|OH|41675|United States|-5|single family| +45201|AAAAAAAABJALAAAA|337|8th Cedar|Circle|Suite O|Summit|Harrison County|OH|40499|United States|-5|apartment| +45202|AAAAAAAACJALAAAA|741|Chestnut |Blvd|Suite 290|Liberty|Cherokee County|AL|33451|United States|-6|apartment| +45203|AAAAAAAADJALAAAA|||Circle||||PA|||-5|| +45204|AAAAAAAAEJALAAAA|520|Franklin Hill|Boulevard|Suite B|Amity|Johnson County|IA|50766|United States|-6|single family| +45205|AAAAAAAAFJALAAAA|76|2nd 9th|Circle|Suite 430|Clifton|Red Willow County|NE|68014|United States|-7|single family| +45206|AAAAAAAAGJALAAAA|213|Jackson |Ln|Suite 260|Wilson|Rock County|MN|56971|United States|-6|condo| +45207|AAAAAAAAHJALAAAA|968|Hickory South|RD|Suite U|Florence|Clare County|MI|43394|United States|-5|single family| +45208|AAAAAAAAIJALAAAA|216|Second Spruce||Suite 230|Enterprise|||||-7|condo| +45209|AAAAAAAAJJALAAAA|719|11th Cedar|Road|Suite F|Argyle|Kern County|CA|98722|United States|-8|single family| +45210|AAAAAAAAKJALAAAA|699|Center Meadow|Ave|Suite 240|Glenwood|Juniata County|PA|13511|United States|-5|apartment| +45211|AAAAAAAALJALAAAA|172|10th 9th|Avenue|Suite V|Oak Ridge|LaMoure County|ND|58371|United States|-6|single family| +45212|AAAAAAAAMJALAAAA|587|Willow Mill|Road|Suite 180|Plainview|Lee County|SC|23683|United States|-5|single family| +45213|AAAAAAAANJALAAAA|861|Hill |Drive|Suite 460|Oak Grove|Cayuga County|NY|18370|United States|-5|apartment| +45214|AAAAAAAAOJALAAAA|804|Washington Adams|Ave|Suite 10|Northwood|Pike County|MS|54104|United States|-6|single family| +45215|AAAAAAAAPJALAAAA|725|Pine |Way|Suite 450|Red Hill|Ohio County|IN|44338|United States|-5|condo| +45216|AAAAAAAAAKALAAAA|499|Green |ST|Suite T|Bridgeport|Lincoln County|WY|85817|United States|-7|single family| +45217|AAAAAAAABKALAAAA|155|4th |Drive|Suite 450|Salem|Atchison County|MO|68048|United States|-6|condo| +45218|AAAAAAAACKALAAAA|101|Hill North|Lane|Suite 210|Providence|Lonoke County|AR|76614|United States|-6|condo| +45219|AAAAAAAADKALAAAA|704|Park Hill|ST|Suite 470|Riverview|Sangamon County|IL|69003|United States|-6|condo| +45220|AAAAAAAAEKALAAAA|4|Smith |Street|Suite Y|Macedonia|Summers County|WV|21087|United States|-5|condo| +45221|AAAAAAAAFKALAAAA|580|View |Way|Suite R|Redland|Clark County|MO|66343|United States|-6|apartment| +45222|AAAAAAAAGKALAAAA|229|2nd Fourth|Circle|Suite 430|Lakeside|DeKalb County|MO|69532|United States|-6|condo| +45223|AAAAAAAAHKALAAAA||Highland 15th|Ln||Midway|Bonneville County||81904|United States||apartment| +45224|AAAAAAAAIKALAAAA|16|9th Willow|Pkwy|Suite 340|Deerfield|Fremont County|CO|89840|United States|-7|apartment| +45225|AAAAAAAAJKALAAAA|968|2nd |Way|Suite L|Springdale|Graham County|NC|28883|United States|-5|apartment| +45226|AAAAAAAAKKALAAAA|533|Third South|Court|Suite O|Jamestown|Union County|NC|26867|United States|-5|single family| +45227|AAAAAAAALKALAAAA|611|Lakeview Main|Ct.|Suite C|Farmington|Stevens County|WA|99145|United States|-8|condo| +45228|AAAAAAAAMKALAAAA|149|Madison Dogwood|Lane|Suite G|Shiloh|Taylor County|WV|29275|United States|-5|condo| +45229|AAAAAAAANKALAAAA|263|Williams Chestnut|Wy|Suite J|Bunker Hill|Russell County|VA|20150|United States|-5|single family| +45230|AAAAAAAAOKALAAAA|461|Sycamore |Pkwy|Suite K|Lakewood|Rogers County|OK|78877|United States|-6|single family| +45231|AAAAAAAAPKALAAAA|912|Poplar Woodland|Drive|Suite 130|Plainview|McLean County|KY|43683|United States|-5|apartment| +45232|AAAAAAAAALALAAAA|432|Oak View|Blvd|Suite U|Belmont|Jackson County|IA|50191|United States|-6|single family| +45233|AAAAAAAABLALAAAA|292|Oak |Court|Suite G|Spring Hill|Christian County|MO|66787|United States|-6|condo| +45234|AAAAAAAACLALAAAA|563|Walnut |Ave|Suite 420|Woodland|Scott County|IA|54854|United States|-6|apartment| +45235|AAAAAAAADLALAAAA|537|Woodland Ridge|Road|Suite O|Walnut Grove|Williamson County|TX|77752|United States|-6|apartment| +45236|AAAAAAAAELALAAAA|978|10th |Pkwy|Suite 10|Woodland|Lafayette Parish|LA|74854|United States|-6|condo| +45237|AAAAAAAAFLALAAAA|129|4th |ST|Suite G|Kingston|Door County|WI|54975|United States|-6|apartment| +45238|AAAAAAAAGLALAAAA|753|Ridge Railroad|Pkwy|Suite 170|Clifton|Charles County|MD|28014|United States|-5|apartment| +45239|AAAAAAAAHLALAAAA|28||Court|||||42898|United States||| +45240|AAAAAAAAILALAAAA|||Avenue|Suite I|Unionville||CA||United States|-8|| +45241|AAAAAAAAJLALAAAA|643|14th Pine|Court|Suite 290|Glenwood|Bourbon County|KY|43511|United States|-6|apartment| +45242|AAAAAAAAKLALAAAA|41|Davis |Pkwy|Suite G|Woodlawn|Caldwell County|MO|64098|United States|-6|apartment| +45243|AAAAAAAALLALAAAA|259|9th View|Drive|Suite 30|Mount Zion|Miami County|OH|48054|United States|-5|single family| +45244|AAAAAAAAMLALAAAA|534|3rd 12th|Dr.|Suite 20|Carter|Kittitas County|WA|90919|United States|-8|single family| +45245|AAAAAAAANLALAAAA|177|Pine Third|Circle|Suite B|Pine Grove|Fulton County|AR|74593|United States|-6|condo| +45246|AAAAAAAAOLALAAAA|85|Pine Forest|Wy|Suite 360|Harmony|Ascension Parish|LA|75804|United States|-6|condo| +45247|AAAAAAAAPLALAAAA|327|Hill Second|Ln|Suite N|Concord|Northampton County|PA|14107|United States|-5|condo| +45248|AAAAAAAAAMALAAAA|745|Davis 7th|Circle|Suite G|Pleasant Hill|Jackson County|AR|73604|United States|-6|apartment| +45249|AAAAAAAABMALAAAA|92|11th |Road|Suite 360|Franklin|Avoyelles Parish|LA|79101|United States|-6|single family| +45250|AAAAAAAACMALAAAA|4|Sunset College|Street|Suite 260|Maple Grove|Sioux County|ND|58252|United States|-6|apartment| +45251|AAAAAAAADMALAAAA|227|Cedar |Avenue|Suite 360|Red Hill|Valencia County|NM|84338|United States|-7|apartment| +45252|AAAAAAAAEMALAAAA|154|5th |ST|Suite 410|Glendale|Buckingham County|VA|23951|United States|-5|apartment| +45253|AAAAAAAAFMALAAAA|258|11th |Avenue|Suite C|Harmony|Wallowa County|OR|95804|United States|-8|apartment| +45254|AAAAAAAAGMALAAAA|716|Elevnth Valley|RD|Suite V|Franklin|New London County|CT|09701|United States|-5|apartment| +45255|AAAAAAAAHMALAAAA|740|Elm |Blvd|Suite 240|Woodland|Glascock County|GA|34854|United States|-5|single family| +45256|AAAAAAAAIMALAAAA|884|1st |Pkwy|Suite C|Amherst|La Salle Parish|LA|78119|United States|-6|single family| +45257|AAAAAAAAJMALAAAA|396|6th Chestnut|Pkwy|Suite 210|Lakeside|Mason County|KY|49532|United States|-5|apartment| +45258|AAAAAAAAKMALAAAA|472|Lee |ST|Suite B|Pleasant Grove|Carroll County|IL|64136|United States|-6|apartment| +45259|AAAAAAAALMALAAAA|475|Park South|Road|Suite 320|Jackson|Breathitt County|KY|49583|United States|-6|single family| +45260|AAAAAAAAMMALAAAA|743|Pine Smith|Avenue|Suite T|Wilson|Hamblen County|TN|36971|United States|-5|condo| +45261|AAAAAAAANMALAAAA|182|Park |Road|Suite R|Shore Acres|Coffee County|AL|32724|United States|-6|single family| +45262|AAAAAAAAOMALAAAA|609|Forest |Boulevard|Suite L|Green Acres|Butler County|AL|37683|United States|-6|apartment| +45263|AAAAAAAAPMALAAAA|786|Jefferson 10th|Blvd|Suite 220|Valley View|Surry County|VA|25124|United States|-5|condo| +45264|AAAAAAAAANALAAAA|896||||||||United States||| +45265|AAAAAAAABNALAAAA|381|Hill |Street|Suite 110|Belmont|Lincoln County|TN|30191|United States|-6|apartment| +45266|AAAAAAAACNALAAAA|499|9th Church|ST|Suite J|Maple Grove|Tuscaloosa County|AL|38252|United States|-6|single family| +45267|AAAAAAAADNALAAAA|816|Madison |Parkway|Suite 90|Farmington|Wilkinson County|MS|59145|United States|-6|apartment| +45268|AAAAAAAAENALAAAA|989|Lake |Ct.|Suite 240|Oakdale|Dane County|WI|59584|United States|-6|condo| +45269|AAAAAAAAFNALAAAA|661|2nd |Boulevard|Suite N|Georgetown|Rio Blanco County|CO|87057|United States|-7|single family| +45270|AAAAAAAAGNALAAAA|588|View Center|Boulevard|Suite 30|Harmony|Butler County|KY|45804|United States|-6|apartment| +45271|AAAAAAAAHNALAAAA|53|Chestnut Jackson|Circle|Suite X|Waterloo|Rapides Parish|LA|71675|United States|-6|condo| +45272|AAAAAAAAINALAAAA|661|8th |Blvd|Suite 260|Sunnyside|Lake County|IL|61952|United States|-6|condo| +45273|AAAAAAAAJNALAAAA|76|Woodland 6th|Dr.|Suite D|||MT||||| +45274|AAAAAAAAKNALAAAA|220|Maple |RD|Suite 260|Deerfield|Nassau County|FL|39840|United States|-5|single family| +45275|AAAAAAAALNALAAAA|184|Green Sixth|Blvd|Suite 220|Maple Grove|Adams County|PA|18252|United States|-5|apartment| +45276|AAAAAAAAMNALAAAA|448|1st |Ln|Suite C|Oak Grove|Wilkinson County|MS|58370|United States|-6|condo| +45277|AAAAAAAANNALAAAA|249|6th Maple|Boulevard|Suite 370|Oakdale|Brazoria County|TX|79584|United States|-6|apartment| +45278|AAAAAAAAONALAAAA|241|Williams |Avenue|Suite 90|Lebanon|Marshall County|OK|72898|United States|-6|apartment| +45279|AAAAAAAAPNALAAAA|861|Johnson 5th|Boulevard|Suite 390|Greenville|Kent County|MD|21387|United States|-5|apartment| +45280|AAAAAAAAAOALAAAA|707|Lakeview |Way|Suite 360|Bunker Hill|Park County|WY|80150|United States|-7|apartment| +45281|AAAAAAAABOALAAAA|51|5th Laurel|Way|Suite S|Pine Grove|Jackson County|TN|34593|United States|-5|condo| +45282|AAAAAAAACOALAAAA|793|View 2nd|RD|Suite 190|Lakeview|Harris County|TX|78579|United States|-6|single family| +45283|AAAAAAAADOALAAAA|65|Lee |Road|Suite U|Mount Vernon|Canyon County|ID|88482|United States|-7|condo| +45284|AAAAAAAAEOALAAAA|730|Oak Franklin|ST|Suite 170|Midway|Washington County|WI|51904|United States|-6|condo| +45285|AAAAAAAAFOALAAAA|703|Ash |Avenue|Suite 280|Pleasant Hill|Nassau County|NY|13604|United States|-5|apartment| +45286|AAAAAAAAGOALAAAA|858|Franklin Franklin|Boulevard|Suite 380|Midway|Presidio County|TX|71904|United States|-6|apartment| +45287|AAAAAAAAHOALAAAA|866|Main |Avenue|Suite 380|Springfield|Dawson County|MT|69303|United States|-7|single family| +45288|AAAAAAAAIOALAAAA|587|Park 8th|Dr.|Suite A|Saint Clair|Candler County|GA|35294|United States|-5|apartment| +45289|AAAAAAAAJOALAAAA|184|7th Oak|Court|Suite N|Ashland|Medina County|OH|44244|United States|-5|condo| +45290|AAAAAAAAKOALAAAA|733|Mill Poplar|Ct.|Suite 490|Jackson|Robertson County|TN|39583|United States|-6|apartment| +45291|AAAAAAAALOALAAAA|648|Pine 5th|Pkwy|Suite 390|Spring Hill|Somerset County|PA|16787|United States|-5|apartment| +45292|AAAAAAAAMOALAAAA|522|Madison 11th|Boulevard|Suite O|Pleasant Grove|Berkshire County|MA|04736|United States|-5|single family| +45293|AAAAAAAANOALAAAA|71|Williams 14th|Way|Suite 160|Newport|San Saba County|TX|71521|United States|-6|single family| +45294|AAAAAAAAOOALAAAA|966|Spring |Avenue|Suite E|Frogtown|Wabaunsee County|KS|68784|United States|-6|single family| +45295|AAAAAAAAPOALAAAA|283|Birch |Pkwy|Suite 320|Unionville|Gray County|TX|71711|United States|-6|apartment| +45296|AAAAAAAAAPALAAAA|711|Mill 3rd|Pkwy|Suite I|Greenville|Jeff Davis County|TX|71387|United States|-6|apartment| +45297|AAAAAAAABPALAAAA|676|Washington |Drive|Suite 410|Red Hill|Schoolcraft County|MI|44338|United States|-5|apartment| +45298|AAAAAAAACPALAAAA|46|East |Avenue|Suite 480|Weldon|Sherman County|KS|66277|United States|-6|apartment| +45299|AAAAAAAADPALAAAA|350|Sunset Washington|RD|Suite S|Fairfield|Dallas County|AR|76192|United States|-6|single family| +45300|AAAAAAAAEPALAAAA|287|2nd Lincoln|Wy|Suite J|Kingston|Valencia County|NM|84975|United States|-7|condo| +45301|AAAAAAAAFPALAAAA|10|Forest Spring|Ct.|Suite 340|Lakewood|Knox County|TX|78877|United States|-6|apartment| +45302|AAAAAAAAGPALAAAA|107|14th |Dr.|Suite 430|Liberty|Moniteau County|MO|63451|United States|-6|apartment| +45303|AAAAAAAAHPALAAAA|99|College 7th|Dr.|Suite 390|Pine Grove|Gogebic County|MI|44593|United States|-5|single family| +45304|AAAAAAAAIPALAAAA|592|Cherry Central|Ct.|Suite 50|Pleasant Grove|Onondaga County|NY|14136|United States|-5|condo| +45305|AAAAAAAAJPALAAAA|411|Central |Ln|Suite 440|Forest Hills|Lee County|VA|29237|United States|-5|apartment| +45306|AAAAAAAAKPALAAAA|646|Railroad Jackson|Way|Suite Y|Five Forks|Dickinson County|KS|62293|United States|-6|condo| +45307|AAAAAAAALPALAAAA|254|Main Pine|Court|Suite 80|Cedar Grove|Hutchinson County|TX|70411|United States|-6|single family| +45308|AAAAAAAAMPALAAAA|291|View |Road|Suite J|Woodville|Macoupin County|IL|64289|United States|-6|single family| +45309|AAAAAAAANPALAAAA|222|Main Locust|RD|Suite 120|Arlington|Pondera County|MT|66557|United States|-7|condo| +45310|AAAAAAAAOPALAAAA|13|1st 5th|Lane|Suite 190|Summerville|Hampton city|VA|22033|United States|-5|single family| +45311|AAAAAAAAPPALAAAA|837|Chestnut Elm|Street|Suite 270|Riverview|Stark County|OH|49003|United States|-5|apartment| +45312|AAAAAAAAAABLAAAA|381|Highland Washington|Boulevard|Suite 480|Liberty|Bradley County|TN|33451|United States|-5|apartment| +45313|AAAAAAAABABLAAAA|498|Cherry Highland|Wy|Suite 190|Florence|Mitchell County|KS|63394|United States|-6|apartment| +45314|AAAAAAAACABLAAAA|366|14th |Wy|Suite 100|Forest Hills|La Plata County|CO|89237|United States|-7|single family| +45315|AAAAAAAADABLAAAA|108|Maple South|Ct.|Suite S|Liberty|Middlesex County|MA|04051|United States|-5|condo| +45316|AAAAAAAAEABLAAAA|394|Davis |Dr.|Suite Q|Ashland|Boone County|WV|24244|United States|-5|apartment| +45317|AAAAAAAAFABLAAAA|747|Johnson Poplar|Blvd|Suite O|Green Acres|Oklahoma County|OK|77683|United States|-6|condo| +45318|AAAAAAAAGABLAAAA|757|Green |RD|Suite 200|Pine Grove|Wilkes County|NC|24593|United States|-5|single family| +45319|AAAAAAAAHABLAAAA|350|Lakeview Pine|Parkway|Suite 170|Riverside|Lewis County|WV|29231|United States|-5|single family| +45320|AAAAAAAAIABLAAAA|932|6th 7th|Ln|Suite 20|Calhoun|Venango County|PA|16909|United States|-5|single family| +45321|AAAAAAAAJABLAAAA|257|River |Way|Suite R|Bethel|Colbert County|AL|35281|United States|-6|single family| +45322|AAAAAAAAKABLAAAA|318|Cedar 3rd|Lane|Suite 50|Fairview|Sampson County|NC|25709|United States|-5|condo| +45323|AAAAAAAALABLAAAA|736|10th Pine|||Oakland|Aroostook County|ME|||-5|| +45324|AAAAAAAAMABLAAAA|297|Jackson Railroad|Wy|Suite F|Franklin|Judith Basin County|MT|69101|United States|-7|single family| +45325|AAAAAAAANABLAAAA|159|12th |RD|Suite 310|Bunker Hill|Hardeman County|TX|70150|United States|-6|apartment| +45326|AAAAAAAAOABLAAAA|538|Main |Wy|Suite O|Fairfield|Chesapeake city|VA|26192|United States|-5|single family| +45327|AAAAAAAAPABLAAAA|526|Main |Boulevard|Suite 370|Pleasant Hill|Luce County|MI|43604|United States|-5|single family| +45328|AAAAAAAAABBLAAAA|352|Walnut |Road|Suite 290|Wildwood|Elmore County|AL|36871|United States|-6|condo| +45329|AAAAAAAABBBLAAAA|250|Laurel |Ct.|Suite B|Lakewood|Montgomery County|KY|48877|United States|-5|condo| +45330|AAAAAAAACBBLAAAA|62|Meadow |Ln|Suite 50|Marion|Franklin city|VA|20399|United States|-5|apartment| +45331|AAAAAAAADBBLAAAA|828|7th |Drive|Suite 470|Fowler|Hooker County|NE|61083|United States|-7|apartment| +45332|AAAAAAAAEBBLAAAA|733|3rd Main|Wy|Suite 190|Providence|Cedar County|MO|66614|United States|-6|single family| +45333|AAAAAAAAFBBLAAAA|755|12th Washington|Avenue|Suite J|Glendale|Mineral County|CO|83951|United States|-7|condo| +45334|AAAAAAAAGBBLAAAA|480|West |Street|Suite A|Woodlawn|Latah County|ID|84098|United States|-7|condo| +45335|AAAAAAAAHBBLAAAA|895|West |Wy|Suite 360|Jamestown|Wilson County|KS|66867|United States|-6|single family| +45336|AAAAAAAAIBBLAAAA|586|Center Highland|Circle|Suite 120|Spring Hill|Clearwater County|ID|86787|United States|-7|condo| +45337|AAAAAAAAJBBLAAAA|184|Church Willow|Ave|Suite 490|Oakdale|Iberia Parish|LA|79584|United States|-6|apartment| +45338|AAAAAAAAKBBLAAAA|403|Madison 10th|Blvd|Suite 490|Lincoln|Marquette County|WI|51289|United States|-6|condo| +45339|AAAAAAAALBBLAAAA|188|West Lee|Drive|Suite 430|Franklin|Washington County|KS|69101|United States|-6|single family| +45340|AAAAAAAAMBBLAAAA|463|Cedar Forest|Cir.|Suite E|Arlington|Appling County|GA|36557|United States|-5|single family| +45341|AAAAAAAANBBLAAAA|883|Cedar |Street|Suite 240|Harmony|Obion County|TN|35804|United States|-6|apartment| +45342|AAAAAAAAOBBLAAAA|648|Chestnut |Circle|Suite O|Shiloh|Henderson County|NC|29275|United States|-5|single family| +45343|AAAAAAAAPBBLAAAA|727|Maple |Boulevard|Suite 30|Oak Hill|Darlington County|SC|27838|United States|-5|condo| +45344|AAAAAAAAACBLAAAA|535|Sunset 4th|RD|Suite 20|Waterloo|Dallas County|AL|31675|United States|-6|apartment| +45345|AAAAAAAABCBLAAAA|699|Dogwood |ST|Suite 90|Spring Hill|Charlottesville city|VA|26787|United States|-5|condo| +45346|AAAAAAAACCBLAAAA|556|Lee Sunset|Ct.|Suite K|Amity|Searcy County|AR|70766|United States|-6|apartment| +45347|AAAAAAAADCBLAAAA|759|Lake Woodland|Pkwy|Suite A|Springdale|Duplin County|NC|28883|United States|-5|single family| +45348|AAAAAAAAECBLAAAA|617|Forest |Ln|Suite 430|Oak Grove|Kit Carson County|CO|88370|United States|-7|single family| +45349|AAAAAAAAFCBLAAAA|554|Poplar |Wy|Suite E|Sunnyside|Adams County|WI|51952|United States|-6|condo| +45350|AAAAAAAAGCBLAAAA|124|Johnson |Ln|Suite 270|Salem|Henry County|TN|38048|United States|-5|condo| +45351|AAAAAAAAHCBLAAAA|882|Pine |Street|Suite D|Summit|Loving County|TX|70499|United States|-6|condo| +45352|AAAAAAAAICBLAAAA|987|North Oak|Avenue|Suite J|Crossroads|Hooker County|NE|60534|United States|-7|single family| +45353|AAAAAAAAJCBLAAAA|142|Pine |Drive|Suite K|Whitney|Mayes County|OK|78339|United States|-6|single family| +45354|AAAAAAAAKCBLAAAA|562|Sycamore |Street|Suite B|Woodville|Poweshiek County|IA|54289|United States|-6|single family| +45355|AAAAAAAALCBLAAAA|496|View Hill|Way|Suite E|Arlington|Wayne County|KY|46557|United States|-5|apartment| +45356|AAAAAAAAMCBLAAAA|890|Walnut Oak|Parkway|Suite 50|Marion|Houston County|AL|30399|United States|-6|apartment| +45357|AAAAAAAANCBLAAAA|917|Willow 12th|Pkwy|Suite G|Valley View|Sumter County|AL|35124|United States|-6|apartment| +45358|AAAAAAAAOCBLAAAA|499|Center 5th|Circle|Suite R|Riverside|Fremont County|ID|89231|United States|-7|condo| +45359|AAAAAAAAPCBLAAAA|626|Oak Washington|Dr.|Suite K|Lebanon|Plumas County|CA|92898|United States|-8|apartment| +45360|AAAAAAAAADBLAAAA|626|Cedar |ST|Suite X|Franklin|Walworth County|WI|59101|United States|-6|apartment| +45361|AAAAAAAABDBLAAAA|286|North Green|Dr.|Suite 10|Spring Hill|Warren County|IN|46787|United States|-5|single family| +45362|AAAAAAAACDBLAAAA|530|Highland West|Drive|Suite K|Morris|Tama County|IA|56696|United States|-6|condo| +45363|AAAAAAAADDBLAAAA|58|5th Elm|ST|Suite Q|Clinton|Solano County|CA|98222|United States|-8|apartment| +45364|AAAAAAAAEDBLAAAA|634|Wilson |Wy|Suite X|Lakeside|Florence County|SC|29532|United States|-5|single family| +45365|AAAAAAAAFDBLAAAA|655|Third View|Ln|Suite P|Belmont|Benton County|AR|70191|United States|-6|condo| +45366|AAAAAAAAGDBLAAAA|636|Wilson |Pkwy|Suite 200|Springdale|Phillips County|CO|88883|United States|-7|single family| +45367|AAAAAAAAHDBLAAAA|819|Fifth |Way|Suite I|Mountain View|Surry County|NC|24466|United States|-5|condo| +45368|AAAAAAAAIDBLAAAA|73|Valley |Blvd|Suite E|Pleasant Grove|Lafourche Parish|LA|74136|United States|-6|single family| +45369|AAAAAAAAJDBLAAAA|93|Hillcrest |Pkwy|Suite 210|Buena Vista|Herkimer County|NY|15752|United States|-5|condo| +45370|AAAAAAAAKDBLAAAA|904|Oak Mill|Cir.|||Meagher County|MT||||| +45371|AAAAAAAALDBLAAAA|925|7th |Avenue|Suite 420|Waterloo|North Slope Borough|AK|91675|United States|-9|single family| +45372|AAAAAAAAMDBLAAAA|478|Railroad |Ave|Suite P|Green Acres|Bowman County|ND|57683|United States|-6|condo| +45373|AAAAAAAANDBLAAAA|280|4th |Blvd|Suite 10|Union|Montgomery County|KY|48721|United States|-5|single family| +45374|AAAAAAAAODBLAAAA|508|Green |Blvd|Suite 450|Goshen|Jim Hogg County|TX|75247|United States|-6|single family| +45375|AAAAAAAAPDBLAAAA||||||Preston County|WV|29454|United States|-5|apartment| +45376|AAAAAAAAAEBLAAAA|668|Meadow Dogwood|Way|Suite U|Shady Grove|Essex County|VT|03412|United States|-5|apartment| +45377|AAAAAAAABEBLAAAA|841|Locust |Boulevard|Suite V|Hamilton|Copiah County|MS|52808|United States|-6|apartment| +45378|AAAAAAAACEBLAAAA|183|Main |Court|Suite A|Wildwood|Bell County|KY|46871|United States|-6|single family| +45379|AAAAAAAADEBLAAAA|402|Meadow Elm|ST|Suite U|Oak Hill|Larimer County|CO|87838|United States|-7|apartment| +45380|AAAAAAAAEEBLAAAA|||Drive||Arlington|Greene County||66557|||| +45381|AAAAAAAAFEBLAAAA|363|Woodland Park|Dr.|Suite 460|Webb|Buena Vista city|VA|20899|United States|-5|condo| +45382|AAAAAAAAGEBLAAAA|951|Jefferson |Ln|Suite A|Shiloh|Metcalfe County|KY|49275|United States|-5|condo| +45383|AAAAAAAAHEBLAAAA|455|Church Spruce|Dr.|Suite U|Waterloo|Polk County|WI|51675|United States|-6|apartment| +45384|AAAAAAAAIEBLAAAA|555|Center 1st|Court|Suite N|Oak Grove|Sarasota County|FL|38370|United States|-5|single family| +45385|AAAAAAAAJEBLAAAA|468|Washington |Ave|Suite 70|Valley View|Toole County|MT|65124|United States|-7|condo| +45386|AAAAAAAAKEBLAAAA|596|Spruce |ST|Suite R|Sunnyside|Alpena County|MI|41952|United States|-5|condo| +45387|AAAAAAAALEBLAAAA|555|Madison |Cir.|Suite U|Edgewood|Ferry County|WA|90069|United States|-8|condo| +45388|AAAAAAAAMEBLAAAA|649|Main Laurel|Ave|Suite M|Jamestown|Tishomingo County|MS|56867|United States|-6|single family| +45389|AAAAAAAANEBLAAAA|299|Main Spring|||||CA|96871||-8|| +45390|AAAAAAAAOEBLAAAA|450|View 6th|Dr.|Suite U|Crossroads|Lucas County|IA|50534|United States|-6|apartment| +45391|AAAAAAAAPEBLAAAA|458|Valley |RD|Suite P|Crossroads|Clay County|IA|50534|United States|-6|single family| +45392|AAAAAAAAAFBLAAAA|909|8th |Ave|Suite T|Red Hill|Douglas County|MO|64338|United States|-6|single family| +45393|AAAAAAAABFBLAAAA|598|Meadow Mill|Parkway|Suite 350|Mount Olive|Saline County|AR|78059|United States|-6|single family| +45394|AAAAAAAACFBLAAAA|842|Sunset |Ln|Suite 190|Bethel|Franklin County|TN|35281|United States|-5|single family| +45395|AAAAAAAADFBLAAAA|94|4th |Pkwy|Suite K|Spring Hill|Trinity County|TX|76787|United States|-6|single family| +45396|AAAAAAAAEFBLAAAA|3|3rd |Court|Suite P|Fairfield|Muskegon County|MI|46192|United States|-5|apartment| +45397|AAAAAAAAFFBLAAAA|991|Johnson Fourth|Wy|Suite 80|Newport|Rio Blanco County|CO|81521|United States|-7|apartment| +45398|AAAAAAAAGFBLAAAA|761|Maple Railroad|Drive|Suite 470|Antioch|Poquoson city|VA|28605|United States|-5|single family| +45399|AAAAAAAAHFBLAAAA|385|Smith Lee|Ln|Suite I|Harmony|Terrebonne Parish|LA|75804|United States|-6|apartment| +45400|AAAAAAAAIFBLAAAA|972|Center 7th|Court|Suite W|Friendship|Wallowa County|OR|94536|United States|-8|apartment| +45401|AAAAAAAAJFBLAAAA|528|Mill 7th|Circle|Suite D|Concord|Izard County|AR|74107|United States|-6|apartment| +45402|AAAAAAAAKFBLAAAA|971|Ridge Spruce|Way|Suite V|Oak Hill|Phillips County|CO|87838|United States|-7|condo| +45403|AAAAAAAALFBLAAAA|258|Forest Fifth|Way|Suite V|Kingston|Lee County|IA|54975|United States|-6|condo| +45404|AAAAAAAAMFBLAAAA|396|||||Ida County|IA|54324|||| +45405|AAAAAAAANFBLAAAA|697|Pine 8th|Lane|Suite 380|Greenwood|Brule County|SD|58828|United States|-6|condo| +45406|AAAAAAAAOFBLAAAA|603|Lake Washington|Dr.|Suite X|Springfield|Oswego County|NY|19303|United States|-5|single family| +45407|AAAAAAAAPFBLAAAA|46|Meadow |Road|Suite C|Somerville|Toole County|MT|67783|United States|-7|apartment| +45408|AAAAAAAAAGBLAAAA|650|14th West|Parkway|Suite Q|Mount Vernon|Marion County|TN|38482|United States|-6|single family| +45409|AAAAAAAABGBLAAAA|609|East South|Way|Suite Q|Oak Hill|Iron County|WI|57838|United States|-6|condo| +45410|AAAAAAAACGBLAAAA|684|Main Highland|Drive|Suite 110|Riverview|Greene County|IL|69003|United States|-6|single family| +45411|AAAAAAAADGBLAAAA|569|2nd Cedar|Pkwy|Suite B|Sulphur Springs|Coos County|NH|08954|United States|-5|apartment| +45412|AAAAAAAAEGBLAAAA|34|Cedar |ST|Suite 90|Pleasant Hill|Charlotte County|FL|33604|United States|-5|apartment| +45413|AAAAAAAAFGBLAAAA|977|Main 4th|Ct.|Suite T|Ashland|Jefferson County|MS|54244|United States|-6|apartment| +45414|AAAAAAAAGGBLAAAA|932|River Oak|Street|Suite D|Glendale|Fulton County|GA|33951|United States|-5|single family| +45415|AAAAAAAAHGBLAAAA|696|3rd Center|Parkway|Suite J|Taft|Blair County|PA|10589|United States|-5|apartment| +45416|AAAAAAAAIGBLAAAA|548|Wilson |Wy|Suite 360|Newport|Androscoggin County|ME|02121|United States|-5|apartment| +45417|AAAAAAAAJGBLAAAA|277|First Chestnut|Road|Suite V|Plainview|Hitchcock County|NE|63683|United States|-6|apartment| +45418|AAAAAAAAKGBLAAAA|555|Elm |Blvd|Suite 390|Pleasant Grove|Pueblo County|CO|84136|United States|-7|apartment| +45419|AAAAAAAALGBLAAAA|332|View 12th|Wy|Suite R|Kingston|Wise County|TX|74975|United States|-6|single family| +45420|AAAAAAAAMGBLAAAA|985|8th 1st|Drive|Suite 370|Springfield|Kauai County|HI|99303|United States|-10|condo| +45421|AAAAAAAANGBLAAAA|671|1st |RD|Suite 120|Bridgeport|Henry County|AL|35817|United States|-6|single family| +45422|AAAAAAAAOGBLAAAA|410|Franklin Mill|Drive|Suite V|Fisher|Forrest County|MS|52819|United States|-6|condo| +45423|AAAAAAAAPGBLAAAA|145|Third |Blvd|Suite M|Crossroads|Martin County|TX|70534|United States|-6|single family| +45424|AAAAAAAAAHBLAAAA|264|Oak Chestnut|Circle|Suite 80|Highland Park|Dona Ana County|NM|86534|United States|-7|single family| +45425|AAAAAAAABHBLAAAA|335|8th |ST|Suite D|Pleasant Valley|Franklin city|VA|22477|United States|-5|apartment| +45426|AAAAAAAACHBLAAAA||Second ||Suite 420|Glenwood|Grayson County|VA|||-5|single family| +45427|AAAAAAAADHBLAAAA|179|Highland River|Street|Suite 390|Bethel|Jackson County|WV|25281|United States|-5|single family| +45428|AAAAAAAAEHBLAAAA|840|Twelfth 2nd|Street|Suite 300|Greenwood|Sutter County|CA|98828|United States|-8|single family| +45429|AAAAAAAAFHBLAAAA|739|Hill Park|Court|Suite I|Cedar Grove|Trigg County|KY|40411|United States|-5|condo| +45430|AAAAAAAAGHBLAAAA|412|Lake |Cir.|Suite 30|Newtown|Erath County|TX|71749|United States|-6|condo| +45431|AAAAAAAAHHBLAAAA|940|Wilson |Wy|Suite 250|Sulphur Springs|Butler County|AL|38354|United States|-6|apartment| +45432|AAAAAAAAIHBLAAAA|399|Central |Ct.|Suite 150|Harmony|Gilliam County|OR|95804|United States|-8|single family| +45433|AAAAAAAAJHBLAAAA|719|12th Woodland|Ave|Suite 120|Sunnyside|Webster County|MO|61952|United States|-6|apartment| +45434|AAAAAAAAKHBLAAAA|286|Elm |Dr.|Suite 430|Lincoln|El Paso County|TX|71289|United States|-6|apartment| +45435|AAAAAAAALHBLAAAA|535|Jefferson West|Drive|Suite 390|Greenville|Taylor County|KY|41387|United States|-5|apartment| +45436|AAAAAAAAMHBLAAAA|847|First |Way|Suite 80|Sulphur Springs|Howard County|TX|78354|United States|-6|condo| +45437|AAAAAAAANHBLAAAA|72|Fifth Second|Parkway|Suite 290|Greenwood|Pottawatomie County|KS|68828|United States|-6|apartment| +45438|AAAAAAAAOHBLAAAA|644|14th |Way|Suite 110|Deerfield|Schley County|GA|39840|United States|-5|apartment| +45439|AAAAAAAAPHBLAAAA|234|Locust Birch|Circle|Suite K|Oakwood|Lincoln County|NC|20169|United States|-5|single family| +45440|AAAAAAAAAIBLAAAA|317|Sixth |Parkway|Suite F|Woodlawn|Colorado County|TX|74098|United States|-6|condo| +45441|AAAAAAAABIBLAAAA|884|6th |Drive|Suite 80|Lakeview|Lander County|NV|88579|United States|-8|single family| +45442|AAAAAAAACIBLAAAA|290|2nd Cedar|Lane|Suite S|Lenox|Cedar County|IA|51143|United States|-6|apartment| +45443|AAAAAAAADIBLAAAA|966|Cherry Spruce|Ave|Suite 110|Lakeside|Amherst County|VA|29532|United States|-5|condo| +45444|AAAAAAAAEIBLAAAA|692|Sixth West|Road|Suite 190|Newport|Wabasha County|MN|51521|United States|-6|apartment| +45445|AAAAAAAAFIBLAAAA|99|5th |Blvd|Suite J|Crossroads|Hamlin County|SD|50534|United States|-7|single family| +45446|AAAAAAAAGIBLAAAA|100|Dogwood |Avenue|Suite 350|Forest Hills|Bristol Bay Borough|AK|99237|United States|-9|apartment| +45447|AAAAAAAAHIBLAAAA|999|Hill 6th|Avenue|Suite J|Woodlawn|Wilkes County|GA|34098|United States|-5|apartment| +45448|AAAAAAAAIIBLAAAA|549|Adams Jackson|Cir.|Suite 80|Bethel|Colleton County|SC|25281|United States|-5|apartment| +45449|AAAAAAAAJIBLAAAA|994|Miller Oak|RD|Suite 40|Springfield|Carroll County|MS|59303|United States|-6|single family| +45450|AAAAAAAAKIBLAAAA|10|9th 9th|Circle|Suite C|Harmony|Watonwan County|MN|55804|United States|-6|apartment| +45451|AAAAAAAALIBLAAAA|699|14th |Circle|Suite W|Ryan|Baxter County|AR|70525|United States|-6|apartment| +45452|AAAAAAAAMIBLAAAA|287|7th Laurel|Boulevard|Suite 50|Pleasant Valley|Union County|IL|62477|United States|-6|condo| +45453|AAAAAAAANIBLAAAA|552|Church |Boulevard|Suite Y|Oakwood|Clearwater County|MN|50169|United States|-6|apartment| +45454|AAAAAAAAOIBLAAAA|882|10th Mill|Ln|Suite G|Galena|Gogebic County|MI|44369|United States|-5|apartment| +45455|AAAAAAAAPIBLAAAA|389|First Pine|Way|Suite O|Spring Hill|Allegany County|NY|16787|United States|-5|apartment| +45456|AAAAAAAAAJBLAAAA|614|South Pine|ST|Suite K|Hillcrest|Wyandot County|OH|43003|United States|-5|single family| +45457|AAAAAAAABJBLAAAA|34|Franklin |Dr.|Suite 110|Bethel|Howard County|IN|45281|United States|-5|single family| +45458|AAAAAAAACJBLAAAA|345|2nd Adams|Way|Suite 440|Sunnyside|Sarpy County|NE|61952|United States|-7|single family| +45459|AAAAAAAADJBLAAAA|690|Locust |Ct.|Suite A|Crossroads|Traverse County|MN|50534|United States|-6|apartment| +45460|AAAAAAAAEJBLAAAA|132|Mill |Boulevard|Suite 90|Shiloh|Macon County|MO|69275|United States|-6|apartment| +45461|AAAAAAAAFJBLAAAA|670|1st |Wy|Suite G|Five Points|Woods County|OK|76098|United States|-6|apartment| +45462|AAAAAAAAGJBLAAAA|590|Walnut Park|Wy|Suite Y|Greenville|Lagrange County|IN|41387|United States|-5|apartment| +45463|AAAAAAAAHJBLAAAA|221|Jefferson |Parkway|Suite 370|Concord|Columbia County|GA|34107|United States|-5|condo| +45464|AAAAAAAAIJBLAAAA|70|Third Main|Circle|Suite J|Oakland|Sullivan County|TN|39843|United States|-6|condo| +45465|AAAAAAAAJJBLAAAA|942|Ridge Spruce|Cir.|Suite 350|Glendale|Jefferson County|NY|13951|United States|-5|apartment| +45466|AAAAAAAAKJBLAAAA|302|Fourth |Wy|Suite S|Glendale|Nottoway County|VA|23951|United States|-5|condo| +45467|AAAAAAAALJBLAAAA|3|Railroad |Blvd|Suite C|Oak Hill|Gregg County|TX|77838|United States|-6|apartment| +45468|AAAAAAAAMJBLAAAA|439|Johnson Ridge|ST|Suite C|Forest Hills|Clay County|GA|39237|United States|-5|single family| +45469|AAAAAAAANJBLAAAA|844|4th 1st|Dr.|Suite Y|Lincoln|Pierce County|NE|61289|United States|-7|condo| +45470|AAAAAAAAOJBLAAAA|24|Elevnth 9th|Ave|Suite U|Lucas|Woodson County|KS|64554|United States|-6|condo| +45471|AAAAAAAAPJBLAAAA|236|Cedar |RD|Suite X|Wilson|Obion County|TN|36971|United States|-6|condo| +45472|AAAAAAAAAKBLAAAA|735|Locust ||Suite U||Alameda County||96909|||apartment| +45473|AAAAAAAABKBLAAAA|991|View |Street|Suite T|Shiloh|Clatsop County|OR|99275|United States|-8|single family| +45474|AAAAAAAACKBLAAAA|543|Hill |RD|Suite G|Stringtown|Campbell County|SD|50162|United States|-6|single family| +45475|AAAAAAAADKBLAAAA|816|Lake |Parkway|Suite E|Greenville|Allen County|OH|41387|United States|-5|apartment| +45476|AAAAAAAAEKBLAAAA|606|15th |Cir.|Suite F|Oak Hill|Hampshire County|WV|27838|United States|-5|condo| +45477|AAAAAAAAFKBLAAAA|801|Franklin View|Street|Suite W|Glendale|Butte County|ID|83951|United States|-7|condo| +45478|AAAAAAAAGKBLAAAA|817|Third |Avenue|Suite 390|Mount Vernon|Macon County|MO|68482|United States|-6|single family| +45479|AAAAAAAAHKBLAAAA|529|Hickory Highland|Boulevard|Suite 280|Hopewell|Hinsdale County|CO|80587|United States|-7|condo| +45480|AAAAAAAAIKBLAAAA|801|Eigth |ST|Suite 390|Woodville|Choctaw County|AL|34289|United States|-6|apartment| +45481|AAAAAAAAJKBLAAAA|75|Dogwood |Wy|Suite I|Green Acres|Clear Creek County|CO|87683|United States|-7|apartment| +45482|AAAAAAAAKKBLAAAA|266|Williams |Pkwy|Suite 150|Franklin|Anchorage Borough|AK|99101|United States|-9|condo| +45483|AAAAAAAALKBLAAAA|469|13th Washington|Road|Suite Q|Westgate|Redwood County|MN|52366|United States|-6|apartment| +45484|AAAAAAAAMKBLAAAA|251|Park |Ave|Suite 190|Waterloo|Skamania County|WA|91675|United States|-8|apartment| +45485|AAAAAAAANKBLAAAA|673|Main |Dr.|Suite 490|Lebanon|Uinta County|WY|82898|United States|-7|single family| +45486|AAAAAAAAOKBLAAAA|650|Cherry Pine|Lane|Suite R|Lakewood|Richland County|IL|68877|United States|-6|apartment| +45487|AAAAAAAAPKBLAAAA|560|Sixth |Pkwy|Suite 340|Macon|Blount County|AL|30369|United States|-6|apartment| +45488|AAAAAAAAALBLAAAA|504|1st |Road|Suite 140|Pine Grove|Butler County|KS|64593|United States|-6|single family| +45489|AAAAAAAABLBLAAAA|124|Adams Highland|Way|Suite J|Pleasant Grove|Cole County|MO|64136|United States|-6|condo| +45490|AAAAAAAACLBLAAAA|300|5th |Dr.|Suite 170|Crossroads|Kimball County|NE|60534|United States|-7|condo| +45491|AAAAAAAADLBLAAAA|305|2nd 6th|Drive|Suite 70|Salem|Ascension Parish|LA|78048|United States|-6|single family| +45492|AAAAAAAAELBLAAAA|971|Main Miller|Pkwy|Suite 480|Riverside|Ashe County|NC|29231|United States|-5|apartment| +45493|AAAAAAAAFLBLAAAA|443|Johnson 15th|Ln|Suite D|New Town|Maricopa County|AZ|89634|United States|-7|single family| +45494|AAAAAAAAGLBLAAAA|676|North 5th|ST|Suite M|Page|Racine County|WI|50296|United States|-6|apartment| +45495|AAAAAAAAHLBLAAAA|313|Spring Poplar|Road|Suite 0|Arlington|Chicot County|AR|76557|United States|-6|apartment| +45496|AAAAAAAAILBLAAAA|640|Willow Pine|Ave|Suite S|Union|Pickaway County|OH|48721|United States|-5|condo| +45497|AAAAAAAAJLBLAAAA|233|Park |Wy|Suite 410|Belmont|Pine County|MN|50191|United States|-6|apartment| +45498|AAAAAAAAKLBLAAAA|967|Mill |ST|Suite B|Midway|Granville County|NC|21904|United States|-5|condo| +45499|AAAAAAAALLBLAAAA|715|Miller |Pkwy|Suite V|Ashland|Clay County|KY|44244|United States|-6|apartment| +45500|AAAAAAAAMLBLAAAA|719|10th |Parkway|Suite 150|Woodlawn|Polk County|TX|74098|United States|-6|single family| +45501|AAAAAAAANLBLAAAA|255|Railroad |Boulevard|Suite 300|Bethel|Evangeline Parish|LA|75281|United States|-6|apartment| +45502|AAAAAAAAOLBLAAAA|684|South 7th|Cir.|Suite 90|Florence|Hansford County|TX|73394|United States|-6|condo| +45503|AAAAAAAAPLBLAAAA|247|Hickory Woodland|Circle|Suite 70|Plainview|Milwaukee County|WI|53683|United States|-6|condo| +45504|AAAAAAAAAMBLAAAA|878|5th Smith|Road||Cedar Grove|||40411|United States||apartment| +45505|AAAAAAAABMBLAAAA|204|3rd |Wy|Suite B|Spring Valley|Sheridan County|KS|66060|United States|-6|condo| +45506|AAAAAAAACMBLAAAA|944|College Davis|Ave|Suite F|Sunnyside|Somerset County|ME|02552|United States|-5|apartment| +45507|AAAAAAAADMBLAAAA|26|Center |Ave|Suite O|Stringtown|Pembina County|ND|50162|United States|-6|condo| +45508|AAAAAAAAEMBLAAAA|898|6th Highland|Circle|Suite O|Four Points|San Benito County|CA|91216|United States|-8|apartment| +45509|AAAAAAAAFMBLAAAA|27|Center Hickory|Parkway|Suite P|Belmont|Cimarron County|OK|70191|United States|-6|single family| +45510|AAAAAAAAGMBLAAAA|693|Williams |Ln|Suite 400|Hillcrest|Clark County|KS|63003|United States|-6|condo| +45511|AAAAAAAAHMBLAAAA|895|East |Way|Suite 280|Riverview|Taylor County|WV|29003|United States|-5|apartment| +45512|AAAAAAAAIMBLAAAA|94|South |Pkwy|Suite 230|Montpelier|Portsmouth city|VA|28930|United States|-5|condo| +45513|AAAAAAAAJMBLAAAA|277|Oak 1st|Cir.|Suite S|Five Points|Pipestone County|MN|56098|United States|-6|apartment| +45514|AAAAAAAAKMBLAAAA|526|Hillcrest First|ST|Suite 430|Red Hill|Arlington County|VA|24338|United States|-5|single family| +45515|AAAAAAAALMBLAAAA|765|Washington |Ct.|Suite 230|Centerville|Chattahoochee County|GA|30059|United States|-5|single family| +45516|AAAAAAAAMMBLAAAA|605|Poplar |Boulevard|Suite T|Riverview|Malheur County|OR|99003|United States|-8|apartment| +45517|AAAAAAAANMBLAAAA|73|Dogwood Davis|Street|Suite T|Texas|Vigo County|IN|43342|United States|-5|condo| +45518|AAAAAAAAOMBLAAAA|943|South Adams|Cir.|Suite H|Lakeville|Clark County|IL|68811|United States|-6|apartment| +45519|AAAAAAAAPMBLAAAA|633|4th Cherry|Cir.|Suite 150|Wilson|Kershaw County|SC|26971|United States|-5|single family| +45520|AAAAAAAAANBLAAAA|363|Maple Cedar|RD|Suite 0|Five Points|Woodward County|OK|76098|United States|-6|apartment| +45521|AAAAAAAABNBLAAAA|756|Lake |Ln|Suite F|Woodland Park|Franklin County|MA|02534|United States|-5|single family| +45522|AAAAAAAACNBLAAAA|670|Main Hickory|Ct.|Suite 40|Pleasant Grove|Mora County|NM|84136|United States|-7|apartment| +45523|AAAAAAAADNBLAAAA|277|Locust |Lane|Suite 100|Greenville|Bexar County|TX|71387|United States|-6|single family| +45524|AAAAAAAAENBLAAAA|396|10th Park|Court|Suite 100|Tyrone|Mitchell County|NC|21201|United States|-5|single family| +45525|AAAAAAAAFNBLAAAA|205|Spring 2nd|Avenue|Suite 190|Union|Mitchell County|NC|28721|United States|-5|apartment| +45526|AAAAAAAAGNBLAAAA|127|Eigth Lincoln|Dr.|Suite B|York|Sullivan County|NH|00836|United States|-5|single family| +45527|AAAAAAAAHNBLAAAA|622|River North|Dr.|Suite C|Mount Vernon|Jasper County|IN|48482|United States|-5|single family| +45528|AAAAAAAAINBLAAAA|811|Main River|ST|Suite 380|Salem|Lagrange County|IN|48048|United States|-5|apartment| +45529|AAAAAAAAJNBLAAAA|905|Ash Washington|Street|Suite 140|Shiloh|Putnam County|TN|39275|United States|-6|apartment| +45530|AAAAAAAAKNBLAAAA|249|Cherry |RD|Suite D|Clinton|Waller County|TX|78222|United States|-6|apartment| +45531|AAAAAAAALNBLAAAA|17|View |Ct.|Suite 230|Greenfield|Lancaster County|VA|25038|United States|-5|condo| +45532|AAAAAAAAMNBLAAAA|615|South Park|Dr.|Suite X|Liberty|Ashland County|OH|43451|United States|-5|condo| +45533|AAAAAAAANNBLAAAA|54|Washington Ash|RD|Suite 60|Summit|Owsley County|KY|40499|United States|-5|condo| +45534|AAAAAAAAONBLAAAA|775|Wilson |Cir.|Suite 130||Knox County||61933|United States||| +45535|AAAAAAAAPNBLAAAA|664|Elm |Way|Suite P|Walnut Grove|Rowan County|KY|47752|United States|-5|single family| +45536|AAAAAAAAAOBLAAAA|525|Ash Main|Ct.|Suite 380|Nottingham|McLeod County|MN|54074|United States|-6|condo| +45537|AAAAAAAABOBLAAAA|289|Valley |Parkway||Glenwood|Sheridan County||63511|United States|-7|apartment| +45538|AAAAAAAACOBLAAAA|784|12th Lake|Avenue|Suite 200|Riverview|Cascade County|MT|69003|United States|-7|single family| +45539|AAAAAAAADOBLAAAA|51|9th |Road|Suite Y|Five Forks|Steele County|MN|52293|United States|-6|apartment| +45540|AAAAAAAAEOBLAAAA|717|First Pine|Way|Suite T|Redland|Floyd County|VA|26343|United States|-5|single family| +45541|AAAAAAAAFOBLAAAA|175|Elm |RD|Suite O|Oak Grove|North Slope Borough|AK|98370|United States|-9|single family| +45542|AAAAAAAAGOBLAAAA|275|Woodland |Drive|Suite 30|Arlington|Cibola County|NM|86557|United States|-7|condo| +45543|AAAAAAAAHOBLAAAA|771|Park Park|Wy|Suite S|Mount Pleasant|Gloucester County|VA|21933|United States|-5|apartment| +45544|AAAAAAAAIOBLAAAA|154|Laurel Maple|Wy|Suite 60|Bridgeport|Essex County|VA|25817|United States|-5|apartment| +45545|AAAAAAAAJOBLAAAA||Lakeview ||Suite 100|Hopewell||FL|||-5|| +45546|AAAAAAAAKOBLAAAA|867|Lakeview Sixth|Boulevard|Suite U|Newport|Perquimans County|NC|21521|United States|-5|single family| +45547|AAAAAAAALOBLAAAA|873|14th 13th|Way|Suite U|Newport|Butler County|MO|61521|United States|-6|single family| +45548|AAAAAAAAMOBLAAAA|570|Second |ST|Suite 490|Macedonia|Yancey County|NC|21087|United States|-5|condo| +45549|AAAAAAAANOBLAAAA|669|South View|Court|Suite 210|Greenville|Martin County|IN|41387|United States|-5|apartment| +45550|AAAAAAAAOOBLAAAA|97|Valley 6th|Ln|Suite X|Concord|Harris County|TX|74107|United States|-6|apartment| +45551|AAAAAAAAPOBLAAAA|975|Maple Jackson|RD|Suite W|Fairfield|Fayette County|OH|46192|United States|-5|single family| +45552|AAAAAAAAAPBLAAAA|680|Valley |Road|Suite 200|Summit|Pike County|IN|40499|United States|-5|condo| +45553|AAAAAAAABPBLAAAA|126|East Smith|RD|Suite 150|Riverview|Custer County|SD|59003|United States|-6|condo| +45554|AAAAAAAACPBLAAAA|358|Meadow 4th|Circle|Suite 190|Salem|Norfolk city|VA|28048|United States|-5|single family| +45555|AAAAAAAADPBLAAAA|13|Lincoln |Pkwy|Suite X|Fairview|Columbia County|PA|15709|United States|-5|apartment| +45556|AAAAAAAAEPBLAAAA|381|5th 2nd|Court|Suite S|White Oak|Floyd County|IN|46668|United States|-5|condo| +45557|AAAAAAAAFPBLAAAA|120|Laurel Green|Blvd|Suite F|Centerville|Andrew County|MO|60059|United States|-6|single family| +45558|AAAAAAAAGPBLAAAA|799|6th |Ct.|Suite Y|Valley View|Colquitt County|GA|35124|United States|-5|condo| +45559|AAAAAAAAHPBLAAAA|872|10th Williams|Ln|Suite F|Mount Zion|Edmunds County|SD|58054|United States|-6|condo| +45560|AAAAAAAAIPBLAAAA|857|1st Spruce|Wy|Suite Y|Riverside|Whatcom County|WA|99231|United States|-8|single family| +45561|AAAAAAAAJPBLAAAA|489|8th |Ct.|Suite H|Newport|Hernando County|FL|31521|United States|-5|condo| +45562|AAAAAAAAKPBLAAAA|393|Park Forest|Ave|Suite 80|Clinton|Desha County|AR|78222|United States|-6|apartment| +45563|AAAAAAAALPBLAAAA|376|Jefferson South|Lane|Suite 30|Mount Vernon|Greenup County|KY|48482|United States|-6|single family| +45564|AAAAAAAAMPBLAAAA|226|First |Boulevard|Suite O|Jamestown|Chilton County|AL|36867|United States|-6|single family| +45565|AAAAAAAANPBLAAAA|130|Lakeview Washington|Boulevard|Suite H|Greenfield|Clay County|IN|45038|United States|-5|single family| +45566|AAAAAAAAOPBLAAAA|460|Miller Washington|Circle|Suite O|Wildwood|Wayne County|NY|16871|United States|-5|single family| +45567|AAAAAAAAPPBLAAAA|379|Adams Jackson|Wy|Suite E|Riverside|Johnson County|MO|69231|United States|-6|apartment| +45568|AAAAAAAAAACLAAAA|794|Maple 2nd|Wy|Suite X|Florence|Leslie County|KY|43394|United States|-5|condo| +45569|AAAAAAAABACLAAAA|648|Hickory |Ct.|Suite 370|Riverdale|Gratiot County|MI|49391|United States|-5|condo| +45570|AAAAAAAACACLAAAA|451|7th Maple|Cir.|Suite 470|Gladstone|Delaware County|IN|40894|United States|-5|single family| +45571|AAAAAAAADACLAAAA|398|View Oak|Lane|Suite 200|Forest Hills|Yakima County|WA|99237|United States|-8|single family| +45572|AAAAAAAAEACLAAAA|||||Crossroads|Shelby County|TX|||-6|| +45573|AAAAAAAAFACLAAAA|520|Franklin Franklin|Blvd|Suite J|Louisville|Clearwater County|MN|54464|United States|-6|apartment| +45574|AAAAAAAAGACLAAAA|105|Williams |Drive|Suite 80|Green Acres|Bryan County|OK|77683|United States|-6|condo| +45575|AAAAAAAAHACLAAAA|293|Oak |Wy|Suite 120|Lincoln|Nash County|NC|21289|United States|-5|condo| +45576|AAAAAAAAIACLAAAA|735|Dogwood |Ct.|Suite 430|Wilson|Monroe County|FL|36971|United States|-5|apartment| +45577|AAAAAAAAJACLAAAA|993|3rd Spruce|RD|Suite M|Newport|Henry County|IL|61521|United States|-6|condo| +45578|AAAAAAAAKACLAAAA|117|Cedar |Wy|Suite 280|Maple Grove|Kossuth County|IA|58252|United States|-6|single family| +45579|AAAAAAAALACLAAAA|799|Fifth Lakeview|Boulevard|Suite 440|Riverview|Kent County|RI|09603|United States|-5|condo| +45580|AAAAAAAAMACLAAAA|21|3rd |ST|Suite T|Belmont|Giles County|VA|20191|United States|-5|apartment| +45581|AAAAAAAANACLAAAA|793|Ninth |Ct.|Suite K|Five Points|Aiken County|SC|26098|United States|-5|condo| +45582|AAAAAAAAOACLAAAA|222|Mill |Ct.|Suite 460|Westgate|Clark County|KS|62366|United States|-6|single family| +45583|AAAAAAAAPACLAAAA|373|Maple |Street|Suite A|Peru|Calcasieu Parish|LA|70302|United States|-6|apartment| +45584|AAAAAAAAABCLAAAA|570|11th |Circle|Suite U|Oakland|Placer County|CA|99843|United States|-8|apartment| +45585|AAAAAAAABBCLAAAA|259|Wilson |Street|Suite C|Monticello|DeKalb County|MO|64146|United States|-6|single family| +45586|AAAAAAAACBCLAAAA|400|8th 13th|Court|Suite E|Hopewell|Clark County|WI|50587|United States|-6|condo| +45587|AAAAAAAADBCLAAAA|387|Maple |Dr.|Suite K|Midway|Nemaha County|KS|61904|United States|-6|apartment| +45588|AAAAAAAAEBCLAAAA|879|Meadow |Lane|Suite K|Elkton|Bethel Census Area|AK|93481|United States|-9|apartment| +45589|AAAAAAAAFBCLAAAA|368|Cherry Walnut|Ln|Suite V|Centerville|Skagit County|WA|90059|United States|-8|apartment| +45590|AAAAAAAAGBCLAAAA|153|Woodland |Court|Suite 250|Kingston|Fulton County|KY|44975|United States|-6|apartment| +45591|AAAAAAAAHBCLAAAA|753|Thirteenth West|Parkway|Suite 140|Forest Hills|Pulaski County|IN|49237|United States|-5|condo| +45592|AAAAAAAAIBCLAAAA|205|Adams |Ct.|Suite 490|Green Acres|Rockingham County|NH|08283|United States|-5|apartment| +45593|AAAAAAAAJBCLAAAA|573|Sycamore |Way|Suite 370|Friendship|Logan County|IL|64536|United States|-6|single family| +45594|AAAAAAAAKBCLAAAA|338|View 7th|Boulevard|Suite 20|Wildwood|Roseau County|MN|56871|United States|-6|single family| +45595|AAAAAAAALBCLAAAA|375|Spruce Green|Court|Suite S|Lincoln|Parker County|TX|71289|United States|-6|single family| +45596|AAAAAAAAMBCLAAAA|139|Wilson 2nd|Drive|Suite 290|Clinton|Mellette County|SD|58222|United States|-7|condo| +45597|AAAAAAAANBCLAAAA|393|Pine |Cir.|Suite S|Riverdale|Saratoga County|NY|19391|United States|-5|single family| +45598|AAAAAAAAOBCLAAAA|715|Mill |Ct.|Suite F|Weldon|McLean County|IL|66277|United States|-6|single family| +45599|AAAAAAAAPBCLAAAA|910|Fifth |Drive|Suite 70|Stringtown|Hardin County|KY|40162|United States|-6|single family| +45600|AAAAAAAAACCLAAAA|51|View Fifteenth|Street|Suite T|Glendale|Chippewa County|WI|53951|United States|-6|condo| +45601|AAAAAAAABCCLAAAA|79|First |Avenue|Suite 200|Wildwood|Harney County|OR|96871|United States|-8|condo| +45602|AAAAAAAACCCLAAAA|746|Elm Center|Parkway|Suite 470|Mount Vernon|Polk County|TX|78482|United States|-6|single family| +45603|AAAAAAAADCCLAAAA|451|North |Court|Suite C|Deerfield|Coles County|IL|69840|United States|-6|condo| +45604|AAAAAAAAECCLAAAA||Hickory Sunset|Wy|Suite A|Spring Valley|Cocke County|TN|36060||-5|| +45605|AAAAAAAAFCCLAAAA|824|Thirteenth Park|Dr.|Suite X|Spring Grove|Pike County|IL|66719|United States|-6|single family| +45606|AAAAAAAAGCCLAAAA|858|Maple Third|RD|Suite E|Ashland|Caddo County|OK|74244|United States|-6|single family| +45607|AAAAAAAAHCCLAAAA|832|Fifth Cherry|Drive|Suite D|Oak Grove|Grant County|OR|98370|United States|-8|single family| +45608|AAAAAAAAICCLAAAA|203|Dogwood |Cir.|Suite 140|Fisher|Madison Parish|LA|72819|United States|-6|single family| +45609|AAAAAAAAJCCLAAAA|324|Lakeview Cherry|Parkway|Suite U|Lakewood|Yolo County|CA|98877|United States|-8|condo| +45610|AAAAAAAAKCCLAAAA|745|Oak Lee|Wy|Suite K|Fairview|Trousdale County|TN|35709|United States|-6|condo| +45611|AAAAAAAALCCLAAAA|22|Sixth 7th|Wy|Suite D|Five Points|Frontier County|NE|66098|United States|-6|apartment| +45612|AAAAAAAAMCCLAAAA|714|3rd |ST|Suite Y|Pleasant Hill|White Pine County|NV|83604|United States|-8|apartment| +45613|AAAAAAAANCCLAAAA|46|Main 8th|Blvd|Suite L|Ashland|Okanogan County|WA|94244|United States|-8|apartment| +45614|AAAAAAAAOCCLAAAA|580|Meadow |Court|Suite E|Macedonia|Ramsey County|MN|51087|United States|-6|condo| +45615|AAAAAAAAPCCLAAAA|645|River |Wy|Suite 130|Riverview|Clatsop County|OR|99003|United States|-8|single family| +45616|AAAAAAAAADCLAAAA|961|12th Davis|Cir.|Suite R|Hillcrest|Erie County|NY|13003|United States|-5|condo| +45617|AAAAAAAABDCLAAAA|376|Sunset |Dr.|Suite S|Riverdale|Hamilton County|IN|49391|United States|-5|condo| +45618|AAAAAAAACDCLAAAA|548|Church |Dr.|Suite 350|Hopewell|Cherokee County|GA|30587|United States|-5|single family| +45619|AAAAAAAADDCLAAAA|197|Central |Cir.|Suite S|Union Hill|Rockingham County|NC|27746|United States|-5|single family| +45620|AAAAAAAAEDCLAAAA|548|5th Church|Pkwy|Suite N|Spring Valley|Cherokee County|SC|26060|United States|-5|condo| +45621|AAAAAAAAFDCLAAAA|527|South |Avenue|Suite 70|Clifton|Cumberland County|IL|68014|United States|-6|condo| +45622|AAAAAAAAGDCLAAAA|121|Elm Mill|Boulevard|Suite 230|Springfield|Coleman County|TX|79303|United States|-6|condo| +45623|AAAAAAAAHDCLAAAA|993|Railroad |Cir.|Suite 290|Walnut Grove|Twin Falls County|ID|87752|United States|-7|condo| +45624|AAAAAAAAIDCLAAAA|201|1st Mill|Circle|Suite 330|Pleasant Valley|Pershing County|NV|82477|United States|-8|single family| +45625|AAAAAAAAJDCLAAAA|768|Laurel 6th|Lane|Suite 480|Greenwood|Woodbury County|IA|58828|United States|-6|single family| +45626|AAAAAAAAKDCLAAAA|34|Park |Wy|Suite 440|Maywood|Elmore County|AL|35681|United States|-6|single family| +45627|AAAAAAAALDCLAAAA|498|1st |Circle|Suite 390|Bunker Hill|Kenosha County|WI|50150|United States|-6|single family| +45628|AAAAAAAAMDCLAAAA|865|Hill Fourth|Blvd|Suite 270|Hillcrest|Davidson County|TN|33003|United States|-5|single family| +45629|AAAAAAAANDCLAAAA|560|Fifth Williams|Avenue|Suite Q|Brownsville|Mason County|KY|49310|United States|-5|condo| +45630|AAAAAAAAODCLAAAA|82|Park |Court|Suite P|Five Forks|Pike County|PA|12293|United States|-5|single family| +45631|AAAAAAAAPDCLAAAA||Main |Pkwy|Suite 60||Franklin County||||-5|apartment| +45632|AAAAAAAAAECLAAAA|377|10th Williams|Ct.|Suite P|Woodland|Licking County|OH|44854|United States|-5|condo| +45633|AAAAAAAABECLAAAA|572|Fourth |Street|Suite 400|Oak Ridge|Brown County|IN|48371|United States|-5|apartment| +45634|AAAAAAAACECLAAAA|741|Fourth 12th|Street|Suite 190|Summit|Carbon County|WY|80499|United States|-7|condo| +45635|AAAAAAAADECLAAAA|488|First |Road|Suite E|Greenfield|Wayne County|TN|35038|United States|-6|single family| +45636|AAAAAAAAEECLAAAA|806|Thirteenth |RD|Suite R|Fairview|Uvalde County|TX|75709|United States|-6|apartment| +45637|AAAAAAAAFECLAAAA|127|13th East|Lane|Suite G|Riverside|Dunklin County|MO|69231|United States|-6|apartment| +45638|AAAAAAAAGECLAAAA|211|Elm 13th|Parkway|Suite 50|New Hope|Hancock County|IA|59431|United States|-6|condo| +45639|AAAAAAAAHECLAAAA|435|Washington |Avenue|Suite 260|Shady Grove|Dolores County|CO|82812|United States|-7|apartment| +45640|AAAAAAAAIECLAAAA|473|Elm College|Way|Suite G|Woodcrest|Haskell County|OK|74919|United States|-6|apartment| +45641|AAAAAAAAJECLAAAA|857|Laurel |Lane|Suite 110|Stringtown|Niagara County|NY|10162|United States|-5|single family| +45642|AAAAAAAAKECLAAAA|489|Oak |Boulevard|Suite 260|Deerfield|Itawamba County|MS|59840|United States|-6|single family| +45643|AAAAAAAALECLAAAA|868|Spruce 7th|Avenue|Suite E|Pleasant Valley|Harvey County|KS|62477|United States|-6|apartment| +45644|AAAAAAAAMECLAAAA|237|Walnut |Way|Suite 20|Franklin|Casey County|KY|49101|United States|-6|condo| +45645|AAAAAAAANECLAAAA|838|5th |Circle|Suite D|Greenwood|Eddy County|ND|58828|United States|-6|single family| +45646|AAAAAAAAOECLAAAA|1000|13th Fourth|Ave|Suite 90|Woodville|Terrell County|GA|34289|United States|-5|single family| +45647|AAAAAAAAPECLAAAA|362|Ridge Meadow|Way|Suite 480|Midway|Contra Costa County|CA|91904|United States|-8|apartment| +45648|AAAAAAAAAFCLAAAA|102|Valley Mill|Cir.|Suite 370|Antioch|Wheatland County|MT|68605|United States|-7|single family| +45649|AAAAAAAABFCLAAAA|141|First |Avenue|Suite 380|Greenfield|Rock County|MN|55038|United States|-6|condo| +45650|AAAAAAAACFCLAAAA|579|Willow 5th|Court|Suite B|Belleville|Pine County|MN|52924|United States|-6|single family| +45651|AAAAAAAADFCLAAAA|746|Cedar |Court|Suite 390|Providence|Franklin County|AR|76614|United States|-6|apartment| +45652|AAAAAAAAEFCLAAAA|718|Lincoln Main|Avenue|Suite I|Bridgeport|Wayne County|GA|35817|United States|-5|single family| +45653|AAAAAAAAFFCLAAAA|721|College |Pkwy|Suite T|Jackson|Arapahoe County|CO|89583|United States|-7|condo| +45654|AAAAAAAAGFCLAAAA|731||Dr.|||Angelina County|TX|71952|||| +45655|AAAAAAAAHFCLAAAA|453|Williams Washington|RD|Suite C|Union|Williamson County|TX|78721|United States|-6|condo| +45656|AAAAAAAAIFCLAAAA|870|Pine 8th|Road|Suite M|Woodlawn|King George County|VA|24098|United States|-5|apartment| +45657|AAAAAAAAJFCLAAAA|426|Cherry Center|RD|Suite 470|Lebanon|Franklin County|PA|12898|United States|-5|single family| +45658|AAAAAAAAKFCLAAAA|397|Lake 4th|Street|Suite 190|Clifton|Vanderburgh County|IN|48014|United States|-5|apartment| +45659|AAAAAAAALFCLAAAA|312|1st |Drive|Suite Q|Oakwood|Trimble County|KY|40169|United States|-5|apartment| +45660|AAAAAAAAMFCLAAAA|209|Third Spring|Ln|Suite 420|Brentwood|Wasatch County|UT|84188|United States|-7|single family| +45661|AAAAAAAANFCLAAAA|876|Central |Court|Suite 360|Brownsville|Duplin County|NC|29310|United States|-5|apartment| +45662|AAAAAAAAOFCLAAAA|950|Main |Blvd|Suite 390|Hamilton|Kendall County|IL|62808|United States|-6|condo| +45663|AAAAAAAAPFCLAAAA|562|Williams |Circle|Suite U|Providence|Guthrie County|IA|56614|United States|-6|condo| +45664|AAAAAAAAAGCLAAAA|395|11th |Ln|Suite 430|Flatwoods|Clinton County|MO|64212|United States|-6|condo| +45665|AAAAAAAABGCLAAAA|629|Laurel West|Boulevard|Suite K|Mount Zion|Jefferson County|ID|88054|United States|-7|apartment| +45666|AAAAAAAACGCLAAAA|978|Center |Drive|Suite 220|Harmony|Winnebago County|WI|55804|United States|-6|apartment| +45667|AAAAAAAADGCLAAAA|947|River |Road|Suite 330|Paxton|Marshall County|SD|55669|United States|-7|condo| +45668|AAAAAAAAEGCLAAAA|550|Lincoln |Blvd|Suite 130|Oak Ridge|Sevier County|AR|78371|United States|-6|apartment| +45669|AAAAAAAAFGCLAAAA|918|Laurel 5th|Parkway|Suite 290|Woodland|Pointe Coupee Parish|LA|74854|United States|-6|apartment| +45670|AAAAAAAAGGCLAAAA|725|Park |Ln|Suite 100|Pleasant Grove|Hancock County|WV|24136|United States|-5|apartment| +45671|AAAAAAAAHGCLAAAA|167|9th |Parkway|Suite 200|Buena Vista|Titus County|TX|75752|United States|-6|apartment| +45672|AAAAAAAAIGCLAAAA|767|2nd |Blvd|Suite D|Unionville|Allegany County|MD|21711|United States|-5|single family| +45673|AAAAAAAAJGCLAAAA|426|Hillcrest Smith|Ct.|Suite 120|Macedonia|Pike County|AR|71087|United States|-6|condo| +45674|AAAAAAAAKGCLAAAA|500|Elm |Drive|Suite 190|Sulphur Springs|Bibb County|AL|38354|United States|-6|apartment| +45675|AAAAAAAALGCLAAAA|967|Washington View|Wy|Suite 470|Marion|Webster Parish|LA|70399|United States|-6|single family| +45676|AAAAAAAAMGCLAAAA|818|Spring |Dr.|Suite M|Clearwater|Lee County|MS|59534|United States|-6|apartment| +45677|AAAAAAAANGCLAAAA|983|Elm |Wy|Suite 0|Bridgeport|Pecos County|TX|75817|United States|-6|single family| +45678|AAAAAAAAOGCLAAAA|202|Woodland |Street|Suite 10|Wildwood|Fleming County|KY|46871|United States|-6|condo| +45679|AAAAAAAAPGCLAAAA|753|Sunset 9th|Boulevard|Suite U|Richville|Gilpin County|CO|85945|United States|-7|condo| +45680|AAAAAAAAAHCLAAAA|580|Pine |Way|Suite 40|Friendship|Walla Walla County|WA|94536|United States|-8|single family| +45681|AAAAAAAABHCLAAAA|504|Church Maple|Ln|Suite 0|Plainview|Sweetwater County|WY|83683|United States|-7|apartment| +45682|AAAAAAAACHCLAAAA|908|Hillcrest |Cir.|Suite X|Hillcrest|Todd County|MN|53003|United States|-6|condo| +45683|AAAAAAAADHCLAAAA|284|Elm |Cir.|Suite C|Fairview|Pittsylvania County|VA|25709|United States|-5|condo| +45684|AAAAAAAAEHCLAAAA|387|Walnut South|Way|Suite H|Riverview|Iredell County|NC|29003|United States|-5|condo| +45685|AAAAAAAAFHCLAAAA|26|Third |Cir.|Suite 200|Wildwood|Burke County|NC|26871|United States|-5|apartment| +45686|AAAAAAAAGHCLAAAA|997|Third |RD|Suite 340|Glendale|Talbot County|MD|23951|United States|-5|apartment| +45687|AAAAAAAAHHCLAAAA|416|9th |Court|Suite 290|Pleasant Hill|Saline County|MO|63604|United States|-6|apartment| +45688|AAAAAAAAIHCLAAAA|799|Elm Franklin|RD|Suite F|Woodlawn|Marion County|IL|64098|United States|-6|single family| +45689|AAAAAAAAJHCLAAAA|372|Jefferson |Drive|Suite 370|Pomona|Putnam County|NY|14153|United States|-5|apartment| +45690|AAAAAAAAKHCLAAAA|332|Locust Main|Cir.|Suite 20|Church Hill|Chickasaw County|IA|53790|United States|-6|apartment| +45691|AAAAAAAALHCLAAAA|970|Miller 7th|RD|Suite 420|Riverview|Butte County|ID|89003|United States|-7|apartment| +45692|AAAAAAAAMHCLAAAA|534|Central |Ln|Suite 70|Edgewood|Christian County|MO|60069|United States|-6|apartment| +45693|AAAAAAAANHCLAAAA|64|Walnut 6th|Court|Suite 440|Spring Valley|Bay County|FL|36060|United States|-5|apartment| +45694|AAAAAAAAOHCLAAAA|488|Williams South|Lane|Suite 60|Highland Park|Walton County|GA|36534|United States|-5|single family| +45695|AAAAAAAAPHCLAAAA|231|Mill Adams|Ave|Suite C|White Oak|Dubuque County|IA|56668|United States|-6|single family| +45696|AAAAAAAAAICLAAAA|614|Fourth 7th|Court|Suite E|Red Hill|Thomas County|NE|64338|United States|-7|condo| +45697|AAAAAAAABICLAAAA|135|Spruce |Avenue|Suite N|Hopewell|Vigo County|IN|40587|United States|-5|apartment| +45698|AAAAAAAACICLAAAA|713|East First|Avenue|Suite O|Lone Oak|Bradford County|PA|16893|United States|-5|condo| +45699|AAAAAAAADICLAAAA|997|Spring |Ct.|Suite 180|Bunker Hill|Bandera County|TX|70150|United States|-6|single family| +45700|AAAAAAAAEICLAAAA|975|Mill 6th|Circle|Suite 280|Oakland|Mellette County|SD|59843|United States|-7|condo| +45701|AAAAAAAAFICLAAAA|177|Pine Fourth|Blvd|Suite V|Walnut Grove|Madison County|OH|47752|United States|-5|apartment| +45702|AAAAAAAAGICLAAAA|||||Riverview||NJ||United States|-5|| +45703|AAAAAAAAHICLAAAA|524|View |Court|Suite P|Jamestown|Trego County|KS|66867|United States|-6|single family| +45704|AAAAAAAAIICLAAAA|||Ct.|Suite 250|||NM|||-7|apartment| +45705|AAAAAAAAJICLAAAA|589|First Woodland|Pkwy|Suite M|Unionville|Noble County|IN|41711|United States|-5|condo| +45706|AAAAAAAAKICLAAAA|994|Elm Pine|Ave|Suite A|Pine Grove|Renville County|MN|54593|United States|-6|condo| +45707|AAAAAAAALICLAAAA|393|First Lincoln|ST|Suite L|Edgewood|Somervell County|TX|70069|United States|-6|condo| +45708|AAAAAAAAMICLAAAA|276|Valley |Blvd|Suite 400|Martinsville|Jeff Davis County|TX|70419|United States|-6|condo| +45709|AAAAAAAANICLAAAA|131|Pine |Street|Suite W|Franklin|Grafton County|NH|09701|United States|-5|condo| +45710|AAAAAAAAOICLAAAA|645|Cedar |||Forest Hills||TX||||single family| +45711|AAAAAAAAPICLAAAA|256|Maple Second|Wy|Suite 380|Glenwood|Meigs County|TN|33511|United States|-6|single family| +45712|AAAAAAAAAJCLAAAA|323|Lakeview |Road|Suite 210|Jamestown|Mathews County|VA|26867|United States|-5|condo| +45713|AAAAAAAABJCLAAAA|189|8th Broadway|Way|Suite X|Arlington|Cassia County|ID|86557|United States|-7|apartment| +45714|AAAAAAAACJCLAAAA|726|Second |Ct.|Suite 460|Pleasant Hill|Grant County|OK|73604|United States|-6|single family| +45715|AAAAAAAADJCLAAAA|726|Oak Birch|Ln|Suite P|Riverview|Costilla County|CO|89003|United States|-7|apartment| +45716|AAAAAAAAEJCLAAAA|261|Sunset Hickory|Wy|Suite 350|Pleasant Hill|Genesee County|NY|13604|United States|-5|single family| +45717|AAAAAAAAFJCLAAAA|602|Fourth Park|Ave|Suite V|Bethel|Campbell County|WY|85281|United States|-7|single family| +45718|AAAAAAAAGJCLAAAA|127|2nd Maple|Parkway|Suite D|Riverdale|Grundy County|IA|59391|United States|-6|single family| +45719|AAAAAAAAHJCLAAAA|695|Davis |Ln|Suite Q|Liberty|Weber County|UT|83451|United States|-7|condo| +45720|AAAAAAAAIJCLAAAA|14|Central |ST|Suite 120|Newport|Knox County|OH|41521|United States|-5|condo| +45721|AAAAAAAAJJCLAAAA|456|View |Wy|Suite S|Royal|Phelps County|NE|65819|United States|-7|single family| +45722|AAAAAAAAKJCLAAAA|487|Chestnut Green|Lane|Suite 280|Riverview|Knox County|IL|69003|United States|-6|apartment| +45723|AAAAAAAALJCLAAAA|876|Johnson East|Ave|Suite R|Kingston|Jackson County|TX|74975|United States|-6|single family| +45724|AAAAAAAAMJCLAAAA|558|Pine |ST|Suite E|Unionville|Gladwin County|MI|41711|United States|-5|condo| +45725|AAAAAAAANJCLAAAA|211|Washington Woodland|Pkwy|Suite P|Bunker Hill|Montgomery County|AL|30150|United States|-6|apartment| +45726|AAAAAAAAOJCLAAAA|759|Forest |Pkwy|Suite N|Wolf Creek|Assumption Parish|LA|72455|United States|-6|condo| +45727|AAAAAAAAPJCLAAAA|602|South |Blvd|Suite 50|Sunnyside|Alamosa County|CO|81952|United States|-7|apartment| +45728|AAAAAAAAAKCLAAAA|765|Pine |Dr.|Suite 140|Salem|Schoolcraft County|MI|48048|United States|-5|apartment| +45729|AAAAAAAABKCLAAAA|588|Sycamore |Blvd|Suite X|Arlington|Wabash County|IL|66557|United States|-6|condo| +45730|AAAAAAAACKCLAAAA|281|Walnut |Lane|Suite G|Wildwood|Jackson County|SD|56871|United States|-7|condo| +45731|AAAAAAAADKCLAAAA|669|Mill Davis|RD|Suite 70|Ashland|Dawson County|GA|34244|United States|-5|apartment| +45732|AAAAAAAAEKCLAAAA|572|Spruce |Boulevard|Suite 430|Sunnyside|Crawford County|OH|41952|United States|-5|condo| +45733|AAAAAAAAFKCLAAAA|107|Woodland |Ct.|Suite 350|Maple Grove|Marshall County|IA|58252|United States|-6|condo| +45734|AAAAAAAAGKCLAAAA|716|Center |Parkway|Suite 490|Marion|Tuscarawas County|OH|40399|United States|-5|apartment| +45735|AAAAAAAAHKCLAAAA|337|Pine 15th|Blvd|Suite 360|Mount Pleasant|Adams County|WI|51933|United States|-6|condo| +45736|AAAAAAAAIKCLAAAA|416|Lincoln Main|Ln|Suite 480|Waterloo|Webster County|NE|61675|United States|-7|condo| +45737|AAAAAAAAJKCLAAAA|99|2nd |Ct.|Suite M|Valley View|Harrison County|OH|45124|United States|-5|condo| +45738|AAAAAAAAKKCLAAAA|984|7th Woodland|Road|Suite 390|Unionville|Morgan County|IN|41711|United States|-5|condo| +45739|AAAAAAAALKCLAAAA|42|Willow |Road|Suite O|Green Acres|Huntingdon County|PA|17683|United States|-5|condo| +45740|AAAAAAAAMKCLAAAA|44|Washington 8th|Blvd|Suite 240|Fairfield|Mineral County|NV|86192|United States|-8|apartment| +45741|AAAAAAAANKCLAAAA|395|Park |Circle|Suite V|Harmony|Wibaux County|MT|65804|United States|-7|single family| +45742|AAAAAAAAOKCLAAAA|537|Central |Boulevard|Suite 310|Oak Grove|Fredericksburg city|VA|28370|United States|-5|apartment| +45743|AAAAAAAAPKCLAAAA|144|Sycamore Smith|Boulevard|Suite O|Union Hill|Lake of the Woods County|MN|57746|United States|-6|apartment| +45744|AAAAAAAAALCLAAAA|787|Wilson |Wy|Suite A|Newtown|Carbon County|UT|81749|United States|-7|apartment| +45745|AAAAAAAABLCLAAAA|747|Fifth 2nd|Wy|Suite 180|Lakewood|Ravalli County|MT|68877|United States|-7|apartment| +45746|AAAAAAAACLCLAAAA|242|East Washington|Road|Suite Q|Riverdale|Wallowa County|OR|99391|United States|-8|single family| +45747|AAAAAAAADLCLAAAA|430|Church |Ave|Suite 340|Hillcrest|Hopewell city|VA|23003|United States|-5|apartment| +45748|AAAAAAAAELCLAAAA|574|West Locust|Avenue|Suite A|Oakwood|Isle of Wight County|VA|20169|United States|-5|single family| +45749|AAAAAAAAFLCLAAAA|653|Lee Poplar|Ln|Suite H|Liberty|Suffolk County|NY|13451|United States|-5|apartment| +45750|AAAAAAAAGLCLAAAA|847|12th 3rd|Lane|Suite 40|Sumner|Union County|NM|80519|United States|-7|condo| +45751|AAAAAAAAHLCLAAAA|691|North 3rd|Court|Suite L|Lakeview|Orange County|NC|28579|United States|-5|condo| +45752|AAAAAAAAILCLAAAA|723|Davis |Ln|Suite W|Forest Hills|Kent County|MI|49237|United States|-5|single family| +45753|AAAAAAAAJLCLAAAA|307|Meadow Railroad|Street|Suite I|Oak Ridge|Lawrence County|AL|38371|United States|-6|single family| +45754|AAAAAAAAKLCLAAAA|234|Walnut Green|Wy|Suite M|Waterloo|Santa Cruz County|CA|91675|United States|-8|single family| +45755|AAAAAAAALLCLAAAA|926|North |Ln|Suite P|Royal|East Baton Rouge Parish|LA|75819|United States|-6|apartment| +45756|AAAAAAAAMLCLAAAA|680|Church 13th|Street|Suite T|Glenwood|Indian River County|FL|33511|United States|-5|apartment| +45757|AAAAAAAANLCLAAAA|383|Second |Dr.|Suite 50|Willis|Jefferson County|OH|46788|United States|-5|apartment| +45758|AAAAAAAAOLCLAAAA|74|Wilson 5th|ST|Suite 330|Arlington|Boone County|NE|66557|United States|-6|apartment| +45759|AAAAAAAAPLCLAAAA|659|Seventh Main|Parkway|Suite F|Cedar Grove|Titus County|TX|70411|United States|-6|apartment| +45760|AAAAAAAAAMCLAAAA|191|4th |Circle|Suite 410|Midway|Potter County|PA|11904|United States|-5|condo| +45761|AAAAAAAABMCLAAAA|994|Walnut |Way|Suite L|Deerfield|La Salle Parish|LA|79840|United States|-6|single family| +45762|AAAAAAAACMCLAAAA|134|West |Road|Suite 30|Pine Grove|Morgan County|WV|24593|United States|-5|apartment| +45763|AAAAAAAADMCLAAAA|311|Johnson |ST|Suite D|Pleasant Grove|Webster County|MS|54136|United States|-6|single family| +45764|AAAAAAAAEMCLAAAA|110|15th |Ave|Suite N|Pleasant Grove|Knox County|TX|74136|United States|-6|single family| +45765|AAAAAAAAFMCLAAAA|24|12th |Circle|Suite 210|Florence|Nance County|NE|63394|United States|-7|single family| +45766|AAAAAAAAGMCLAAAA|648|15th |Wy|Suite R|Ashland|Christian County|IL|64244|United States|-6|apartment| +45767|AAAAAAAAHMCLAAAA|283|Meadow |Ct.|Suite 490|Hamilton|Marshall County|AL|32808|United States|-6|condo| +45768|AAAAAAAAIMCLAAAA|563|13th 3rd|RD|Suite 140|Forest Hills|Lampasas County|TX|79237|United States|-6|apartment| +45769|AAAAAAAAJMCLAAAA|884|Oak 8th|Circle|Suite 140|Maple Grove|Bedford County|VA|28252|United States|-5|single family| +45770|AAAAAAAAKMCLAAAA|37|Franklin |Dr.|Suite C|Riverside|Jerome County|ID|89231|United States|-7|single family| +45771|AAAAAAAALMCLAAAA|333|Lincoln Adams|Dr.|Suite V|Arlington|Mason County|MI|46557|United States|-5|condo| +45772|AAAAAAAAMMCLAAAA|69|8th Railroad|Avenue|Suite 260|Sunnyside|Jackson County|TX|71952|United States|-6|condo| +45773|AAAAAAAANMCLAAAA|458|13th |Court|Suite J|Forest Hills|Park County|CO|89237|United States|-7|single family| +45774|AAAAAAAAOMCLAAAA|563|Second Maple|Lane|Suite N|Greenville|Wilbarger County|TX|71387|United States|-6|single family| +45775|AAAAAAAAPMCLAAAA|305|Wilson |RD|Suite 100|Lakeside|Sarasota County|FL|39532|United States|-5|single family| +45776|AAAAAAAAANCLAAAA|465|Wilson |Circle|Suite U|Lincoln|Rock County|WI|51289|United States|-6|single family| +45777|AAAAAAAABNCLAAAA|839|Davis |Ct.|Suite 80|Watkins|Fairfield County|CT|02332|United States|-5|condo| +45778|AAAAAAAACNCLAAAA|766|Fifth Spruce|Road|Suite X|Riverside|Collin County|TX|79231|United States|-6|condo| +45779|AAAAAAAADNCLAAAA|296|Elm Maple|Ave|Suite 300|Union|Chase County|NE|68721|United States|-6|condo| +45780|AAAAAAAAENCLAAAA|344|Park |Way|Suite M|Forest Hills|Arthur County|NE|69237|United States|-6|condo| +45781|AAAAAAAAFNCLAAAA|826|Poplar 12th|Ct.|Suite 330|Newtown|Albany County|WY|81749|United States|-7|condo| +45782|AAAAAAAAGNCLAAAA|759|Thirteenth West|Drive|Suite 470|Woodland Park|Des Moines County|IA|51934|United States|-6|condo| +45783|AAAAAAAAHNCLAAAA|101|Adams Walnut|Court|Suite O|Oak Grove|Potter County|SD|58370|United States|-7|apartment| +45784|AAAAAAAAINCLAAAA|536|Willow Fifth|RD|Suite 480|Hillcrest|Cabarrus County|NC|23003|United States|-5|condo| +45785|AAAAAAAAJNCLAAAA|871|Oak Jackson|Boulevard|Suite B|Liberty|Day County|SD|53451|United States|-6|single family| +45786|AAAAAAAAKNCLAAAA|48|Willow 9th|Blvd|Suite N|Sunnyside|Amador County|CA|91952|United States|-8|apartment| +45787|AAAAAAAALNCLAAAA|320|Sixth |Road|Suite A|Providence|Orleans County|NY|16614|United States|-5|condo| +45788|AAAAAAAAMNCLAAAA|703|Sycamore |Court|Suite M|Mountain View|Walton County|GA|34466|United States|-5|single family| +45789|AAAAAAAANNCLAAAA|376|Adams |Ln|Suite 10|Mount Zion|Jeff Davis County|TX|78054|United States|-6|single family| +45790|AAAAAAAAONCLAAAA|129|||Suite K||Custer County|||United States||| +45791|AAAAAAAAPNCLAAAA|167|Lee |Road|Suite 100|Jackson|Barry County|MI|49583|United States|-5|apartment| +45792|AAAAAAAAAOCLAAAA|378|Laurel |Court|Suite 310|Green Acres|Caroline County|MD|27683|United States|-5|single family| +45793|AAAAAAAABOCLAAAA|604|Fourth Sycamore|Wy|Suite 290|Greenville|Hamblen County|TN|31387|United States|-5|apartment| +45794|AAAAAAAACOCLAAAA|81|Pine Elm|Drive|Suite 30|Sulphur Springs|Weld County|CO|88354|United States|-7|single family| +45795|AAAAAAAADOCLAAAA|219|Maple River|Blvd|Suite 320|Buena Vista|Cooper County|MO|65752|United States|-6|single family| +45796|AAAAAAAAEOCLAAAA|547|Willow Thirteenth|Avenue|Suite 320|Mount Pleasant|Washington County|MO|61933|United States|-6|single family| +45797|AAAAAAAAFOCLAAAA|66|15th Meadow|Road|||||73481|United States|-6|| +45798|AAAAAAAAGOCLAAAA|395|Broadway 2nd|Way|Suite K|Five Forks|Walworth County|SD|52293|United States|-7|condo| +45799|AAAAAAAAHOCLAAAA|367|10th Poplar|Avenue|Suite 180|Maple Grove|Morgan County|CO|88252|United States|-7|condo| +45800|AAAAAAAAIOCLAAAA|791|||Suite 170||Centre County||||-5|condo| +45801|AAAAAAAAJOCLAAAA|258|View |Avenue|Suite O|Oakdale|Levy County|FL|39584|United States|-5|apartment| +45802|AAAAAAAAKOCLAAAA|671|Hill Mill|Way|Suite I|Bunker Hill|Halifax County|NC|20150|United States|-5|condo| +45803|AAAAAAAALOCLAAAA|843|Walnut Second|Ln|Suite F|Lincoln|Glynn County|GA|31289|United States|-5|apartment| +45804|AAAAAAAAMOCLAAAA|186|Sunset 11th|Pkwy|Suite M|Walnut Grove|Poweshiek County|IA|57752|United States|-6|single family| +45805|AAAAAAAANOCLAAAA|||Circle||Midway||||||| +45806|AAAAAAAAOOCLAAAA|113|South College|Court|Suite 420|Oakland|Woodson County|KS|69843|United States|-6|apartment| +45807|AAAAAAAAPOCLAAAA|666|Laurel |Ct.|Suite W|Pleasant Grove|Leslie County|KY|44136|United States|-5|condo| +45808|AAAAAAAAAPCLAAAA|28|Sunset |Way|Suite V|New Hope|Putnam County|WV|29431|United States|-5|single family| +45809|AAAAAAAABPCLAAAA|417|Birch 13th|Wy|Suite 10|Fairfield|Butts County|GA|36192|United States|-5|single family| +45810|AAAAAAAACPCLAAAA|409|Franklin |Road|Suite 240|Clifford|Douglas County|WI|58164|United States|-6|condo| +45811|AAAAAAAADPCLAAAA|67|Fourth Church|Court|Suite 20|Woodland|Volusia County|FL|34854|United States|-5|apartment| +45812|AAAAAAAAEPCLAAAA|919|Ash Meadow|Pkwy|Suite Y|Riverside|Owsley County|KY|49231|United States|-5|apartment| +45813|AAAAAAAAFPCLAAAA|580|Mill Williams|Court|Suite 290|Lawrenceville|Callahan County|TX|74462|United States|-6|single family| +45814|AAAAAAAAGPCLAAAA|784|Smith Elm|Dr.|Suite G|Brownsville|Ogemaw County|MI|49310|United States|-5|condo| +45815|AAAAAAAAHPCLAAAA|362|Oak 4th|Lane|Suite 470|Colfax|Alexandria city|VA|22565|United States|-5|condo| +45816|AAAAAAAAIPCLAAAA|598|Spring |Parkway|Suite 450|Lakeview|Howard County|MD|28579|United States|-5|single family| +45817|AAAAAAAAJPCLAAAA|383|River 5th|Way|Suite Y|Ridgeville|Washington County|KS|69306|United States|-6|condo| +45818|AAAAAAAAKPCLAAAA|648|Church |Drive|Suite V|Riverview|Garland County|AR|79003|United States|-6|apartment| +45819|AAAAAAAALPCLAAAA|886|Cedar Broadway|Parkway|Suite 330|Enterprise|Lake and Peninsula Borough|AK|91757|United States|-9|single family| +45820|AAAAAAAAMPCLAAAA|952|Oak Fifth|RD|Suite 20|Greenwood|Lewis County|KY|48828|United States|-5|single family| +45821|AAAAAAAANPCLAAAA|614|3rd Cherry|Ct.|Suite 310|Rossville|Judith Basin County|MT|62503|United States|-7|single family| +45822|AAAAAAAAOPCLAAAA|155|4th |RD|Suite S|Deerfield|El Paso County|CO|89840|United States|-7|apartment| +45823|AAAAAAAAPPCLAAAA|793|North |Circle|Suite H|Glenville|Frio County|TX|73445|United States|-6|single family| +45824|AAAAAAAAAADLAAAA|940|Hill Hill|Ct.|Suite E|Lincoln|Burt County|NE|61289|United States|-6|condo| +45825|AAAAAAAABADLAAAA|469|Spring 1st|Dr.|Suite 240|Shady Grove|Canyon County|ID|82812|United States|-7|single family| +45826|AAAAAAAACADLAAAA|262|Spruce Woodland|Boulevard|Suite C|Stringtown|Catawba County|NC|20162|United States|-5|condo| +45827|AAAAAAAADADLAAAA|554|Fifth Jefferson|Ln|Suite M|Pleasant Hill|Grand Forks County|ND|53604|United States|-6|condo| +45828|AAAAAAAAEADLAAAA|844|Fourth Lincoln|Cir.|Suite 430|Newtown|Butler County|IA|51749|United States|-6|apartment| +45829|AAAAAAAAFADLAAAA|287|Church First|RD|Suite W|Enterprise|Lincoln County|GA|31757|United States|-5|apartment| +45830|AAAAAAAAGADLAAAA|690|8th |Blvd|Suite I|Five Points|Coal County|OK|76098|United States|-6|single family| +45831|AAAAAAAAHADLAAAA|854|Franklin |Lane|Suite 300|Friendship|Allendale County|SC|24536|United States|-5|single family| +45832|AAAAAAAAIADLAAAA|286|Park |ST|Suite K|Shady Grove|Pittsylvania County|VA|22812|United States|-5|apartment| +45833|AAAAAAAAJADLAAAA|106|Pine 1st|Wy|Suite M|Lakeview|Indian River County|FL|38579|United States|-5|condo| +45834|AAAAAAAAKADLAAAA|731|Williams Jefferson|Lane|Suite G|Hillcrest|Fond du Lac County|WI|53003|United States|-6|apartment| +45835|AAAAAAAALADLAAAA|671|12th Pine|RD|Suite 470|Union Hill|Coffee County|TN|37746|United States|-5|condo| +45836|AAAAAAAAMADLAAAA|979|Lake Oak|Wy|Suite 60|Macedonia|Windsor County|VT|01687|United States|-5|condo| +45837|AAAAAAAANADLAAAA|857|Fourteenth Park|Boulevard|Suite 40|Bethel|Franklin County|MO|65281|United States|-6|single family| +45838|AAAAAAAAOADLAAAA|40|Highland Forest|Street|Suite 380|Hamilton|Nodaway County|MO|62808|United States|-6|single family| +45839|AAAAAAAAPADLAAAA|775|Cedar 9th|Cir.|Suite C|Fairfield|Barbour County|WV|26192|United States|-5|condo| +45840|AAAAAAAAABDLAAAA|430|Madison 13th|Ln|Suite 90|Kingston|Sioux County|NE|64975|United States|-7|condo| +45841|AAAAAAAABBDLAAAA|48|Cherry Smith|Court|Suite 310|Centerville|Louisa County|IA|50059|United States|-6|apartment| +45842|AAAAAAAACBDLAAAA|934|13th Cherry|Blvd|Suite O|Antioch|Haralson County|GA|38605|United States|-5|condo| +45843|AAAAAAAADBDLAAAA|333|13th |Dr.|Suite 90|Salem|Shelby County|KY|48048|United States|-5|single family| +45844|AAAAAAAAEBDLAAAA|189|2nd |Street|Suite 130|Farmington|Hampshire County|WV|29145|United States|-5|apartment| +45845|AAAAAAAAFBDLAAAA|974|Park Smith|Parkway|Suite 220|New Hope|Jackson County|IL|69431|United States|-6|condo| +45846|AAAAAAAAGBDLAAAA|645|14th 4th|Parkway|Suite I|Glenwood|Shelby County|TX|73511|United States|-6|single family| +45847|AAAAAAAAHBDLAAAA|322|Willow Spring|Street|Suite L|Greenwood|Prince George County|MD|28828|United States|-5|apartment| +45848|AAAAAAAAIBDLAAAA|290|Seventh Lincoln|Ave|Suite 150|Doyle|Crawford County|KS|68434|United States|-6|apartment| +45849|AAAAAAAAJBDLAAAA|529|Cedar Hillcrest|Blvd|Suite 120|Bridgeport|Becker County|MN|55817|United States|-6|condo| +45850|AAAAAAAAKBDLAAAA|847|Highland |Avenue|Suite C|Woodlawn|Tyler County|TX|74098|United States|-6|apartment| +45851|AAAAAAAALBDLAAAA|565|Cedar Broadway|Way|Suite 320|Pleasant Grove|Madison Parish|LA|74136|United States|-6|apartment| +45852|AAAAAAAAMBDLAAAA|692|Maple |Dr.|Suite D|Elkton|Somerset County|PA|13481|United States|-5|apartment| +45853|AAAAAAAANBDLAAAA|400|Davis |Circle|Suite O|Crossroads|Barry County|MI|40534|United States|-5|single family| +45854|AAAAAAAAOBDLAAAA|35|First Meadow|Lane|Suite R|Wilton|Lincoln County|WV|26997|United States|-5|apartment| +45855|AAAAAAAAPBDLAAAA|299|Church 3rd|Dr.|Suite 250|Oak Hill|Foster County|ND|57838|United States|-6|apartment| +45856|AAAAAAAAACDLAAAA|276|Miller |Boulevard|Suite 380|Pleasant Valley|Pulaski County|GA|32477|United States|-5|condo| +45857|AAAAAAAABCDLAAAA|905|Lee Valley|Circle|Suite I|Rosewood|Clearfield County|PA|12205|United States|-5|apartment| +45858|AAAAAAAACCDLAAAA|||Circle|||Liberty County||||-5|apartment| +45859|AAAAAAAADCDLAAAA|269|Hill |Road|Suite V|Centerville|Perry County|OH|40059|United States|-5|single family| +45860|AAAAAAAAECDLAAAA|65|Park |Ct.|Suite 320|Oneida|Lee County|TX|74027|United States|-6|apartment| +45861|AAAAAAAAFCDLAAAA|63|Jackson 5th|Cir.|Suite 390|Riverdale|Muhlenberg County|KY|49391|United States|-5|single family| +45862|AAAAAAAAGCDLAAAA|575|Broadway |Ave|Suite I|Five Points|Chisago County|MN|56098|United States|-6|condo| +45863|AAAAAAAAHCDLAAAA|628|Walnut Cherry|ST|Suite 330|Clearview|Chickasaw County|IA|55495|United States|-6|apartment| +45864|AAAAAAAAICDLAAAA|452|South Washington|Drive|Suite 410|Hillcrest|Cleveland County|OK|73003|United States|-6|condo| +45865|AAAAAAAAJCDLAAAA|738|Valley Lake|RD|Suite 130|Edgewood|Otsego County|MI|40069|United States|-5|single family| +45866|AAAAAAAAKCDLAAAA|298|Washington Poplar|Court|Suite 410|Riverview|Nance County|NE|69003|United States|-7|single family| +45867|AAAAAAAALCDLAAAA|348|1st |Court|Suite W|Mountain View|Carson City|NV|84466|United States|-8|apartment| +45868|AAAAAAAAMCDLAAAA|862|Maple |RD|Suite 0|Greenwood|Foster County|ND|58828|United States|-6|single family| +45869|AAAAAAAANCDLAAAA|75|Willow |Road|Suite 270|Five Points|El Paso County|CO|86098|United States|-7|single family| +45870|AAAAAAAAOCDLAAAA|549|13th Cedar|Drive|Suite 130|Wilson|Lake County|CO|86971|United States|-7|condo| +45871|AAAAAAAAPCDLAAAA|642|First Pine|Parkway|Suite 110|Lebanon|Tangipahoa Parish|LA|72898|United States|-6|apartment| +45872|AAAAAAAAADDLAAAA|876|Valley Hill|Dr.|Suite 300|Marion|Dodge County|NE|60399|United States|-6|apartment| +45873|AAAAAAAABDDLAAAA|63|Sunset |ST|Suite 10|Farmersville|Allen County|IN|49305|United States|-5|condo| +45874|AAAAAAAACDDLAAAA|412|Sunset 3rd|Way|Suite 290|Oak Hill|East Baton Rouge Parish|LA|77838|United States|-6|condo| +45875|AAAAAAAADDDLAAAA|361|Elm Chestnut|Drive|Suite N|Cherry Valley|Kidder County|ND|50854|United States|-6|single family| +45876|AAAAAAAAEDDLAAAA|697|6th |ST|Suite Y|Centerville|Oneida County|ID|80059|United States|-7|condo| +45877|AAAAAAAAFDDLAAAA|683|4th Washington|Pkwy|Suite 130|Sulphur Springs|Lawrence County|PA|18354|United States|-5|apartment| +45878|AAAAAAAAGDDLAAAA|436|Oak Fifth|ST|Suite L|Pleasant Grove|Butler County|PA|14136|United States|-5|apartment| +45879|AAAAAAAAHDDLAAAA|328|Lake 11th|Street|Suite D|Sullivan|Goochland County|VA|20451|United States|-5|single family| +45880|AAAAAAAAIDDLAAAA|879|7th East|Cir.|Suite H|Oak Ridge|Wahkiakum County|WA|98371|United States|-8|apartment| +45881|AAAAAAAAJDDLAAAA|229|Park |Way|Suite 390|Woodville|Ottawa County|OK|74289|United States|-6|apartment| +45882|AAAAAAAAKDDLAAAA|914|Broadway 4th|Court|Suite V|Stringtown|Williams County|OH|40162|United States|-5|single family| +45883|AAAAAAAALDDLAAAA|423|Jefferson |Road|Suite S|Lone Oak|Rock County|WI|56893|United States|-6|single family| +45884|AAAAAAAAMDDLAAAA|939|Meadow |Pkwy|Suite 100|Marion|Mineral County|NV|80399|United States|-8|condo| +45885|AAAAAAAANDDLAAAA|569|Fifth Williams|Circle|Suite G|Pleasant Grove|Cecil County|MD|24136|United States|-5|apartment| +45886|AAAAAAAAODDLAAAA|79|Fifth Sycamore|Cir.|Suite 110|Enterprise|Van Buren County|TN|31757|United States|-6|single family| +45887|AAAAAAAAPDDLAAAA|101|14th |Street|Suite 430|Buena Vista|Gonzales County|TX|75752|United States|-6|single family| +45888|AAAAAAAAAEDLAAAA|100|Pine First|ST|Suite 170|Clifton|Torrance County|NM|88014|United States|-7|condo| +45889|AAAAAAAABEDLAAAA|786|Park Davis|Drive|Suite N|Bridgeport|Oktibbeha County|MS|55817|United States|-6|apartment| +45890|AAAAAAAACEDLAAAA|596|Walnut Birch|Avenue|Suite B|Sulphur Springs|Karnes County|TX|78354|United States|-6|single family| +45891|AAAAAAAADEDLAAAA|103|Central Third|Drive|Suite 400|Five Forks|Lowndes County|GA|32293|United States|-5|single family| +45892|AAAAAAAAEEDLAAAA|687|5th Park|Cir.|Suite X|Fairview|Murray County|GA|35709|United States|-5|condo| +45893|AAAAAAAAFEDLAAAA|527|Fourth |Ave|Suite 430|Clifton|Pottawatomie County|OK|78014|United States|-6|condo| +45894|AAAAAAAAGEDLAAAA|666|4th Birch|ST|Suite 20|Franklin|Richmond County|NC|29101|United States|-5|single family| +45895|AAAAAAAAHEDLAAAA|690|East Highland|Street|Suite 10|Farmington|Jones County|SD|59145|United States|-7|apartment| +45896|AAAAAAAAIEDLAAAA|721|Franklin |Drive|Suite G|Lakewood|Northumberland County|VA|28877|United States|-5|apartment| +45897|AAAAAAAAJEDLAAAA|139|8th Second|Ave|Suite I|New Hope|Cook County|GA|39431|United States|-5|apartment| +45898|AAAAAAAAKEDLAAAA|909||Parkway|Suite 440|Quincy|Rockbridge County|VA|23868|United States|-5|| +45899|AAAAAAAALEDLAAAA|499|Chestnut |Way|Suite 70|Page|Chariton County|MO|60296|United States|-6|apartment| +45900|AAAAAAAAMEDLAAAA|230|3rd Walnut|Boulevard|Suite I|Providence|Jefferson County|MO|66614|United States|-6|single family| +45901|AAAAAAAANEDLAAAA|504|1st |Road|Suite O|Belmont|Grenada County|MS|50191|United States|-6|apartment| +45902|AAAAAAAAOEDLAAAA|493|Broadway |Drive|Suite Q|Tremont|Shelby County|IA|59515|United States|-6|single family| +45903|AAAAAAAAPEDLAAAA|709|Ash Valley|Way|Suite L|Highland|Wilson County|TX|79454|United States|-6|condo| +45904|AAAAAAAAAFDLAAAA|540|Third |Lane|Suite 280|Wildwood|Gilmer County|WV|26871|United States|-5|single family| +45905|AAAAAAAABFDLAAAA|925|Washington Maple|Parkway|Suite X|Summit|Elk County|PA|10499|United States|-5|condo| +45906|AAAAAAAACFDLAAAA|793|Pine Hill|Ln|Suite 210|Hillcrest|Wise County|TX|73003|United States|-6|condo| +45907|AAAAAAAADFDLAAAA|529|Lincoln |Drive|Suite 50|Pleasant Hill|Lamar County|AL|33604|United States|-6|apartment| +45908|AAAAAAAAEFDLAAAA|972|Adams |Ct.|Suite Q|Sunnyside|Monroe County|MI|41952|United States|-5|apartment| +45909|AAAAAAAAFFDLAAAA|605|11th Spring|Cir.|Suite L|Riverdale|Ashley County|AR|79391|United States|-6|condo| +45910|AAAAAAAAGFDLAAAA|803|5th 3rd|Ave|Suite 60|Shady Grove|Centre County|PA|12812|United States|-5|condo| +45911|AAAAAAAAHFDLAAAA|992|Franklin |Ln|Suite 240|Oakland|Churchill County|NV|89843|United States|-8|apartment| +45912|AAAAAAAAIFDLAAAA|290|2nd Lee|Circle|Suite 200|Georgetown|Graves County|KY|47057|United States|-6|condo| +45913|AAAAAAAAJFDLAAAA|993|Central Hill|Way|Suite 400|Enterprise|Douglas County|OR|91757|United States|-8|single family| +45914|AAAAAAAAKFDLAAAA|249|Madison |Boulevard|Suite R|Richardson|Marion County|OH|47687|United States|-5|condo| +45915|AAAAAAAALFDLAAAA|531|Franklin Adams|RD|Suite V|Friendship|Big Horn County|MT|64536|United States|-7|condo| +45916|AAAAAAAAMFDLAAAA|206|4th Locust|Boulevard|Suite 440|Green Acres|Logan County|KS|67683|United States|-6|apartment| +45917|AAAAAAAANFDLAAAA|839|Park Maple|Circle|Suite 410|Mount Olive|Stewart County|GA|38059|United States|-5|apartment| +45918|AAAAAAAAOFDLAAAA|389|Green Lee|Pkwy|Suite 400|Greenfield|White County|TN|35038|United States|-6|condo| +45919|AAAAAAAAPFDLAAAA|530|First |Avenue|Suite C|Lakeview|Iredell County|NC|28579|United States|-5|apartment| +45920|AAAAAAAAAGDLAAAA|914|Park |Street|Suite 210|Enterprise|Otoe County|NE|61757|United States|-7|apartment| +45921|AAAAAAAABGDLAAAA|368|Ninth Sixth|Wy|Suite J|Farmington|Brown County|SD|59145|United States|-6|condo| +45922|AAAAAAAACGDLAAAA|453|Locust |Street|Suite J|Union|Isle of Wight County|VA|28721|United States|-5|apartment| +45923|AAAAAAAADGDLAAAA|524|Hickory Cedar|Circle|Suite 150|Pleasant Valley|Gates County|NC|22477|United States|-5|single family| +45924|AAAAAAAAEGDLAAAA|372|Smith Elm|Cir.|Suite 480|Clinton|Bethel Census Area|AK|98222|United States|-9|apartment| +45925|AAAAAAAAFGDLAAAA|195|Dogwood |Circle|Suite P|Deerfield|Noble County|OK|79840|United States|-6|apartment| +45926|AAAAAAAAGGDLAAAA|521|Ash |Ave|Suite 100|Mount Zion|Summit County|CO|88054|United States|-7|apartment| +45927|AAAAAAAAHGDLAAAA|975|Walnut Forest|Circle|Suite 130|Lakeview|Brewster County|TX|78579|United States|-6|condo| +45928|AAAAAAAAIGDLAAAA|101|Center North|Avenue|Suite E|Clifford|Buchanan County|IA|58164|United States|-6|apartment| +45929|AAAAAAAAJGDLAAAA|439|Park Hickory|||Cedar Grove||IN|||-5|condo| +45930|AAAAAAAAKGDLAAAA|28|11th |Cir.|Suite 420|Bunker Hill|Lane County|KS|60150|United States|-6|apartment| +45931|AAAAAAAALGDLAAAA|510|Poplar |Wy|Suite 350|Argyle|Sanders County|MT|68722|United States|-7|apartment| +45932|AAAAAAAAMGDLAAAA|392|South Meadow|Pkwy|Suite Q|Woodland Park|Logan County|ND|51934|United States|-6|condo| +45933|AAAAAAAANGDLAAAA|574|Cherry Elm|Ct.|Suite 300|Weldon|Pembina County|ND|56277|United States|-6|apartment| +45934|AAAAAAAAOGDLAAAA|189|Meadow Main|Way|Suite 250|Sulphur Springs|Pocahontas County|IA|58354|United States|-6|condo| +45935|AAAAAAAAPGDLAAAA|963|Cherry |RD|Suite X|Newport|Posey County|IN|41521|United States|-5|single family| +45936|AAAAAAAAAHDLAAAA|301|Walnut View|Boulevard|Suite 130|Springfield|Brunswick County|NC|29303|United States|-5|single family| +45937|AAAAAAAABHDLAAAA|565|Williams College|Court|Suite H|Wilson|Sumner County|TN|36971|United States|-6|single family| +45938|AAAAAAAACHDLAAAA|309|Sunset Mill|Ave|Suite K|Summit|Mississippi County|MO|60499|United States|-6|apartment| +45939|AAAAAAAADHDLAAAA|776|Wilson Center|Circle|Suite C|Walnut Grove|Faulk County|SD|57752|United States|-6|condo| +45940|AAAAAAAAEHDLAAAA|838|North 5th|Blvd|Suite 230|Bridgeport|Blackford County|IN|45817|United States|-5|apartment| +45941|AAAAAAAAFHDLAAAA|748|Wilson West|Drive|Suite B|Forest Hills|Coffee County|TN|39237|United States|-5|condo| +45942|AAAAAAAAGHDLAAAA|400|Maple Oak|Ct.|Suite V|Salem|Jones County|SD|58048|United States|-7|single family| +45943|AAAAAAAAHHDLAAAA||Lincoln Sunset|Parkway||Woodland|Northampton County|NC|||-5|| +45944|AAAAAAAAIHDLAAAA|297|Second Cherry|RD|Suite 260|Spring Hill|Tioga County|PA|16787|United States|-5|condo| +45945|AAAAAAAAJHDLAAAA|725|Lincoln Washington|Court|Suite T|Twin Oaks|Morris County|KS|60663|United States|-6|apartment| +45946|AAAAAAAAKHDLAAAA|4|Oak 1st|Way|Suite 120|Fairview|Clay County|AL|35709|United States|-6|condo| +45947|AAAAAAAALHDLAAAA|261|Maple |Way|Suite 150|New Hope|Garfield County|CO|89431|United States|-7|condo| +45948|AAAAAAAAMHDLAAAA|357|Ash Lake|ST|Suite A|Ashland|Rankin County|MS|54244|United States|-6|condo| +45949|AAAAAAAANHDLAAAA|68|Spruce |Wy|Suite 230|Union Hill|Hickman County|KY|47746|United States|-6|single family| +45950|AAAAAAAAOHDLAAAA|362|Walnut Center|Wy|Suite G|Bunker Hill|Clarke County|MS|50150|United States|-6|single family| +45951|AAAAAAAAPHDLAAAA|692|Locust Oak|Road|Suite 220|Mount Pleasant|Liberty County|GA|31933|United States|-5|condo| +45952|AAAAAAAAAIDLAAAA|775|14th Main|Ave|Suite 100|Highland|Berkshire County|MA|09454|United States|-5|condo| +45953|AAAAAAAABIDLAAAA||River |||Clifton||SC||United States|-5|| +45954|AAAAAAAACIDLAAAA|1000|Cherry Sycamore|Wy|Suite R|Midway|Geauga County|OH|41904|United States|-5|apartment| +45955|AAAAAAAADIDLAAAA|529|6th Williams|Boulevard|Suite 100|Frankfort|Randolph County|IN|49681|United States|-5|condo| +45956|AAAAAAAAEIDLAAAA|61|Center |Road|Suite T|Unionville|Jefferson County|IL|61711|United States|-6|apartment| +45957|AAAAAAAAFIDLAAAA|533|Chestnut Oak|Street|Suite 290|White Oak|Childress County|TX|76668|United States|-6|single family| +45958|AAAAAAAAGIDLAAAA|916|Sixth 9th|Road|Suite 470|Elm Grove|Adams County|PA|13298|United States|-5|single family| +45959|AAAAAAAAHIDLAAAA|105|Forest Fourth|Parkway|Suite 440|Riverview|Sacramento County|CA|99003|United States|-8|condo| +45960|AAAAAAAAIIDLAAAA|479|Walnut |Parkway|Suite X|Ridgeville|Kimball County|NE|69306|United States|-7|single family| +45961|AAAAAAAAJIDLAAAA|229|Valley 7th|Boulevard|Suite Y|Midway|Jones County|IA|51904|United States|-6|condo| +45962|AAAAAAAAKIDLAAAA|780|Jackson Laurel|Ave|Suite T|Mount Vernon|Klamath County|OR|98482|United States|-8|single family| +45963|AAAAAAAALIDLAAAA|977|View Park|Court|Suite 290|Jackson|Callaway County|MO|69583|United States|-6|apartment| +45964|AAAAAAAAMIDLAAAA|985|7th |Dr.|Suite 220|Centerville|Banks County|GA|30059|United States|-5|apartment| +45965|AAAAAAAANIDLAAAA|453|Locust |ST|Suite 430|Pleasant Grove|Bacon County|GA|34136|United States|-5|single family| +45966|AAAAAAAAOIDLAAAA|658|Seventh |Pkwy|Suite 490|Cedar Grove|Lenawee County|MI|40411|United States|-5|apartment| +45967|AAAAAAAAPIDLAAAA|975|Ridge |Ln|Suite G|Lakeview|Webster County|MS|58579|United States|-6|apartment| +45968|AAAAAAAAAJDLAAAA|392|Park |Cir.|Suite Q|Rutland|Calhoun County|TX|78375|United States|-6|apartment| +45969|AAAAAAAABJDLAAAA|475|Third |Dr.|Suite 380|Pleasant Hill|Sedgwick County|CO|83604|United States|-7|apartment| +45970|AAAAAAAACJDLAAAA|376|3rd Fourth|Boulevard|Suite B|Avoca|Arenac County|MI|40540|United States|-5|apartment| +45971|AAAAAAAADJDLAAAA|641|Lee Seventh|Street|Suite 50|Farmington|Red River County|TX|79145|United States|-6|condo| +45972|AAAAAAAAEJDLAAAA|598||Way|||Delaware County||16668|United States|-5|| +45973|AAAAAAAAFJDLAAAA|719|View |Ct.|Suite R|Providence|Rabun County|GA|36614|United States|-5|condo| +45974|AAAAAAAAGJDLAAAA|8|View |Street|Suite M|Unionville|Ashland County|WI|51711|United States|-6|single family| +45975|AAAAAAAAHJDLAAAA|253|7th |Court|Suite 210|Plainview|Forrest County|MS|53683|United States|-6|single family| +45976|AAAAAAAAIJDLAAAA|163|5th |ST|Suite E|Maple Grove|Navarro County|TX|78252|United States|-6|condo| +45977|AAAAAAAAJJDLAAAA|469|Dogwood |Cir.|Suite 60|Page|Tift County|GA|30296|United States|-5|condo| +45978|AAAAAAAAKJDLAAAA|136|First Dogwood|Avenue|Suite 200|Glenwood|Hood County|TX|73511|United States|-6|single family| +45979|AAAAAAAALJDLAAAA|86|7th |Parkway|Suite U|Pleasant Hill|Culberson County|TX|73604|United States|-6|single family| +45980|AAAAAAAAMJDLAAAA|662|Washington |RD|Suite Y|Tyler|Madison Parish|LA|76853|United States|-6|single family| +45981|AAAAAAAANJDLAAAA|726|3rd |Ln|Suite 40|Highland|Brown County|NE|69454|United States|-6|single family| +45982|AAAAAAAAOJDLAAAA|672|4th |Way|Suite 410|Woodlawn|Washington County|TX|74098|United States|-6|apartment| +45983|AAAAAAAAPJDLAAAA|319|Main 5th|Drive|Suite 450|Shaw|Calhoun County|TX|70618|United States|-6|apartment| +45984|AAAAAAAAAKDLAAAA|940|Railroad 4th|Ave|Suite 230|Mount Vernon|Switzerland County|IN|48482|United States|-5|single family| +45985|AAAAAAAABKDLAAAA|344|Poplar Hillcrest|Wy|Suite 90|Edgewood|Alleghany County|VA|20069|United States|-5|single family| +45986|AAAAAAAACKDLAAAA|382|College |Ct.|Suite Y|Stringtown|Smyth County|VA|20162|United States|-5|single family| +45987|AAAAAAAADKDLAAAA|721|Hill Hickory|ST|Suite N|Richfield|Garfield County|WA|96196|United States|-8|condo| +45988|AAAAAAAAEKDLAAAA|614|Willow |Pkwy|Suite 90|Wildwood|Buncombe County|NC|26871|United States|-5|apartment| +45989|AAAAAAAAFKDLAAAA|252|Fifth |RD|Suite 100|Liberty|Greer County|OK|73451|United States|-6|apartment| +45990|AAAAAAAAGKDLAAAA|755|Oak 11th|Ave|Suite V|Five Forks|Stark County|ND|52293|United States|-6|apartment| +45991|AAAAAAAAHKDLAAAA|278|3rd |Street|Suite 110|Springdale|Ritchie County|WV|28883|United States|-5|condo| +45992|AAAAAAAAIKDLAAAA|632|Sunset Lakeview|Ave|Suite 410|Greenfield|Bourbon County|KY|45038|United States|-6|single family| +45993|AAAAAAAAJKDLAAAA|645|2nd Walnut|Parkway|Suite 410|Four Points|Thomas County|GA|31216|United States|-5|apartment| +45994|AAAAAAAAKKDLAAAA|333|Park |Wy|Suite N|Vista|Tippecanoe County|IN|44694|United States|-5|condo| +45995|AAAAAAAALKDLAAAA|6|13th |Boulevard|Suite D|Unionville|Schuyler County|MO|61711|United States|-6|single family| +45996|AAAAAAAAMKDLAAAA|850|North |ST|Suite 400|Brownsville|Franklin County|MS|59310|United States|-6|single family| +45997|AAAAAAAANKDLAAAA|482|Hill River|Court|Suite M|White Hall|Morgan County|TN|36955|United States|-6|single family| +45998|AAAAAAAAOKDLAAAA|428|Madison |Avenue|Suite I|Glendale|Berkshire County|MA|04551|United States|-5|apartment| +45999|AAAAAAAAPKDLAAAA|219|First |Ln|Suite 130|Enterprise|Page County|IA|51757|United States|-6|apartment| +46000|AAAAAAAAALDLAAAA|533|Cherry |Street|Suite Q|Riverdale|Charles City County|VA|29391|United States|-5|condo| +46001|AAAAAAAABLDLAAAA|529|North |ST|Suite B|Stafford|Brown County|KS|64980|United States|-6|single family| +46002|AAAAAAAACLDLAAAA|636|Forest |Wy|Suite 90|Fairfield|Oneida County|ID|86192|United States|-7|apartment| +46003|AAAAAAAADLDLAAAA|363|Lincoln 8th|Lane|Suite K|Edgewood|Long County|GA|30069|United States|-5|apartment| +46004|AAAAAAAAELDLAAAA|273|Wilson |Ave|Suite 340|Edgewood|O-Brien County|IA|50069|United States|-6|apartment| +46005|AAAAAAAAFLDLAAAA|969|Pine |Dr.|Suite B|Wildwood|Edmonson County|KY|46871|United States|-6|condo| +46006|AAAAAAAAGLDLAAAA|519|Sixth Johnson|Circle|Suite 240|Oak Hill|Kent County|RI|08438|United States|-5|single family| +46007|AAAAAAAAHLDLAAAA|564|View Main|Wy|Suite 60|Cedar|Alameda County|CA|91229|United States|-8|condo| +46008|AAAAAAAAILDLAAAA||Dogwood ||Suite W||Orange County|IN|47752||-5|| +46009|AAAAAAAAJLDLAAAA|238|Cedar Jackson|Street|Suite 310|Sunnyside|Delta County|MI|41952|United States|-5|condo| +46010|AAAAAAAAKLDLAAAA|822|Lakeview |Court|Suite 470|Enterprise|Franklin County|NE|61757|United States|-6|condo| +46011|AAAAAAAALLDLAAAA|393|Poplar Pine|Street|Suite V|Fairfield|Queen Anne County|MD|26192|United States|-5|apartment| +46012|AAAAAAAAMLDLAAAA|170|Hickory |Road|Suite 440|Franklin|Anchorage Borough|AK|99101|United States|-9|condo| +46013|AAAAAAAANLDLAAAA|560|7th |Lane|Suite 280|Stringtown|Grayson County|VA|20162|United States|-5|single family| +46014|AAAAAAAAOLDLAAAA|91|Park |Street|Suite 260|Shady Grove|Valley County|MT|62812|United States|-7|apartment| +46015|AAAAAAAAPLDLAAAA|515|Spring First|Road|Suite D|Mount Vernon|Wirt County|WV|28482|United States|-5|apartment| +46016|AAAAAAAAAMDLAAAA|542|Ridge |Street|Suite L|Plainview|Morton County|KS|63683|United States|-6|single family| +46017|AAAAAAAABMDLAAAA|175|Highland Main|RD|Suite 70|Murphy|Trousdale County|TN|32105|United States|-6|apartment| +46018|AAAAAAAACMDLAAAA|954|Johnson Washington|RD|Suite B|Newport|Ellis County|OK|71521|United States|-6|apartment| +46019|AAAAAAAADMDLAAAA|312|Jackson Hill|Lane|Suite X|Oakdale|Dougherty County|GA|39584|United States|-5|single family| +46020|AAAAAAAAEMDLAAAA|530|Pine |Cir.|Suite 260|Saint Clair|Burleigh County|ND|55294|United States|-6|apartment| +46021|AAAAAAAAFMDLAAAA|875|Meadow |Road|Suite Y|Marion|Smith County|MS|50399|United States|-6|single family| +46022|AAAAAAAAGMDLAAAA|506|Hill 2nd|Parkway|Suite 260|Oak Hill|Lincoln County|CO|87838|United States|-7|single family| +46023|AAAAAAAAHMDLAAAA|477|Chestnut Meadow|Boulevard|Suite A|Arthur|Boise County|ID|85965|United States|-7|apartment| +46024|AAAAAAAAIMDLAAAA|379|North Broadway|Drive|Suite O|Sunnyside|Butler County|AL|31952|United States|-6|single family| +46025|AAAAAAAAJMDLAAAA|842|Highland Adams|Blvd|Suite 80|Union|Major County|OK|78721|United States|-6|apartment| +46026|AAAAAAAAKMDLAAAA|107|2nd |RD|Suite 350|Woodlawn|Stark County|ND|54098|United States|-6|apartment| +46027|AAAAAAAALMDLAAAA|24|West North|Court|Suite 410|Lincoln|Hyde County|SD|51289|United States|-7|single family| +46028|AAAAAAAAMMDLAAAA|731|Meadow Elm|Pkwy|Suite T|Nottingham|Kings County|NY|14074|United States|-5|condo| +46029|AAAAAAAANMDLAAAA|438|Railroad 3rd|Parkway|Suite 250|Buena Vista|Tyler County|WV|25752|United States|-5|single family| +46030|AAAAAAAAOMDLAAAA|8|6th |RD|Suite E|Mountain View|Madison County|OH|44466|United States|-5|apartment| +46031|AAAAAAAAPMDLAAAA|463|View |Pkwy|Suite O|Watkins|Cassia County|ID|81732|United States|-7|single family| +46032|AAAAAAAAANDLAAAA|180|Lake |Ct.|Suite 130|Clinton|Barron County|WI|58222|United States|-6|condo| +46033|AAAAAAAABNDLAAAA|223|Railroad |Wy|Suite P|Edgewood|Ritchie County|WV|20069|United States|-5|single family| +46034|AAAAAAAACNDLAAAA|834|Sunset |Ct.|Suite U|Marion|Newton County|AR|70399|United States|-6|apartment| +46035|AAAAAAAADNDLAAAA|409|Jefferson Willow|Avenue|Suite Y|Antioch|Dimmit County|TX|78605|United States|-6|apartment| +46036|AAAAAAAAENDLAAAA|320|Pine 8th|Ave|Suite 70|Summit|Poweshiek County|IA|50499|United States|-6|single family| +46037|AAAAAAAAFNDLAAAA|770|Eigth Fourth|Ct.|Suite 470|Sulphur Springs|Hawaii County|HI|98354|United States|-10|single family| +46038|AAAAAAAAGNDLAAAA|982|Third |ST|Suite 430|Snug Harbor|Westchester County|NY|17936|United States|-5|condo| +46039|AAAAAAAAHNDLAAAA|463|North |Boulevard|Suite 390|Georgetown|Union County|FL|37057|United States|-5|apartment| +46040|AAAAAAAAINDLAAAA|358|Pine |Court|Suite 70|Brownsville|Steele County|ND|59310|United States|-6|condo| +46041|AAAAAAAAJNDLAAAA|462|Dogwood Maple|Blvd|Suite 170|Belmont|Hardin County|OH|40191|United States|-5|condo| +46042|AAAAAAAAKNDLAAAA|929|Johnson |Street|Suite G|Arlington|Irion County|TX|76557|United States|-6|single family| +46043|AAAAAAAALNDLAAAA|563|West |Pkwy|Suite 290|Riverside|Kendall County|TX|79231|United States|-6|apartment| +46044|AAAAAAAAMNDLAAAA|206|4th |ST|Suite 220|Shiloh|Tulare County|CA|99275|United States|-8|single family| +46045|AAAAAAAANNDLAAAA|280|Birch |Ct.|Suite K|Denmark|Santa Barbara County|CA|95576|United States|-8|apartment| +46046|AAAAAAAAONDLAAAA|716|Washington Jackson|Road|Suite Y|Avoca|Jefferson County|AL|30540|United States|-6|apartment| +46047|AAAAAAAAPNDLAAAA|62|7th Cedar|ST|Suite K|Clinton|Jersey County|IL|68222|United States|-6|condo| +46048|AAAAAAAAAODLAAAA|876|Central |Way|Suite 20|Georgetown|Unicoi County|TN|37057|United States|-6|single family| +46049|AAAAAAAABODLAAAA|656|Broadway |RD|Suite Y|Sunnyside|Okaloosa County|FL|31952|United States|-5|apartment| +46050|AAAAAAAACODLAAAA|554|Williams |RD|Suite P|Kingston|Burt County|NE|64975|United States|-6|apartment| +46051|AAAAAAAADODLAAAA|705|2nd Green|Boulevard|Suite 400|Clifton|Dillon County|SC|28014|United States|-5|condo| +46052|AAAAAAAAEODLAAAA|986|Wilson Meadow|Way|Suite 370|Spring Valley|Poquoson city|VA|26060|United States|-5|single family| +46053|AAAAAAAAFODLAAAA|217|6th Walnut|Way|Suite N|Buena Vista|Huntington County|IN|45752|United States|-5|apartment| +46054|AAAAAAAAGODLAAAA|367|9th |Road|Suite T|Mount Olive|Wayne County|MI|48059|United States|-5|condo| +46055|AAAAAAAAHODLAAAA|812|Main |Ave|Suite V|Five Forks|Nicollet County|MN|52293|United States|-6|single family| +46056|AAAAAAAAIODLAAAA|920|Oak |Dr.|Suite O|Riverside|Pulaski County|IL|69231|United States|-6|single family| +46057|AAAAAAAAJODLAAAA|330|Spring |Blvd|Suite U|Pleasant Grove|Bolivar County|MS|54136|United States|-6|single family| +46058|AAAAAAAAKODLAAAA|973|Maple |Court|Suite 240|Springfield|Calhoun County|IA|59303|United States|-6|single family| +46059|AAAAAAAALODLAAAA|174|Maple Park|Wy|Suite R|Greenville|Pope County|MN|51387|United States|-6|condo| +46060|AAAAAAAAMODLAAAA|538|3rd Sunset|Ct.|Suite T|Pleasant Valley|Sandoval County|NM|82477|United States|-7|condo| +46061|AAAAAAAANODLAAAA|358|Oak |Cir.|Suite C|Highland Park|Alexandria city|VA|26534|United States|-5|single family| +46062|AAAAAAAAOODLAAAA|736|View |ST|Suite 60|Centerville|Polk County|AR|70059|United States|-6|condo| +46063|AAAAAAAAPODLAAAA|607|Cherry Third|ST|Suite V|Newport|Des Moines County|IA|51521|United States|-6|condo| +46064|AAAAAAAAAPDLAAAA|281|Forest |Road|Suite 380|Clinton|Lancaster County|PA|18222|United States|-5|apartment| +46065|AAAAAAAABPDLAAAA|698|Dogwood East|Ct.|Suite J|Buckingham|Jefferson County|WA|94092|United States|-8|apartment| +46066|AAAAAAAACPDLAAAA|573|Park |Lane|||Parmer County|||United States|-6|apartment| +46067|AAAAAAAADPDLAAAA|977|West North|ST|Suite U|Jamestown|Autauga County|AL|36867|United States|-6|apartment| +46068|AAAAAAAAEPDLAAAA|38|Park |Ct.|Suite R|Fisher|Ritchie County|WV|22819|United States|-5|apartment| +46069|AAAAAAAAFPDLAAAA|305|Lakeview |Blvd|Suite 400|Jamestown|Cullman County|AL|36867|United States|-6|condo| +46070|AAAAAAAAGPDLAAAA|61|Oak |ST|Suite 240|Hillcrest|Mason County|MI|43003|United States|-5|apartment| +46071|AAAAAAAAHPDLAAAA|792|Woodland |Blvd|Suite 370|Highland|Morrow County|OR|99454|United States|-8|condo| +46072|AAAAAAAAIPDLAAAA|490|Poplar Park|Parkway|Suite F|Spring Hill|Shelby County|OH|46787|United States|-5|single family| +46073|AAAAAAAAJPDLAAAA|218|Maple Hill|Road|Suite X|Kingston|Bedford County|PA|14975|United States|-5|condo| +46074|AAAAAAAAKPDLAAAA|301|11th |Wy|Suite 300|Oak Grove|Warrick County|IN|48370|United States|-5|condo| +46075|AAAAAAAALPDLAAAA|868|10th |Avenue|Suite F|Sulphur Springs|Stonewall County|TX|78354|United States|-6|apartment| +46076|AAAAAAAAMPDLAAAA|653|West Hill|Ln|Suite V|Pleasant Valley|Vance County|NC|22477|United States|-5|single family| +46077|AAAAAAAANPDLAAAA|239|Broadway Davis|Avenue|Suite 180|Westfield|Smith County|TX|71900|United States|-6|single family| +46078|AAAAAAAAOPDLAAAA|794|10th |Circle|Suite L|Lakewood|Pinellas County|FL|38877|United States|-5|apartment| +46079|AAAAAAAAPPDLAAAA|274|Green Second|Road|Suite 270|Valley View|Bent County|CO|85124|United States|-7|single family| +46080|AAAAAAAAAAELAAAA|330|Center Washington||Suite F|Clinton||KS||||| +46081|AAAAAAAABAELAAAA|734|Locust |Circle|Suite Q|Wright|Schoolcraft County|MI|42814|United States|-5|apartment| +46082|AAAAAAAACAELAAAA|793|First |Parkway|Suite S|Springdale|Richmond County|NY|18883|United States|-5|condo| +46083|AAAAAAAADAELAAAA|803|Walnut Smith|Boulevard|Suite W|Johnsonville|Sanilac County|MI|47745|United States|-5|apartment| +46084|AAAAAAAAEAELAAAA|990|Spruce |Pkwy|Suite S|Jackson|Ashley County|AR|79583|United States|-6|condo| +46085|AAAAAAAAFAELAAAA|267|Madison |Ave|Suite E|Springdale|Henderson County|IL|68883|United States|-6|apartment| +46086|AAAAAAAAGAELAAAA|870|Davis 4th|Court|Suite 40|Enterprise|Rich County|UT|81757|United States|-7|apartment| +46087|AAAAAAAAHAELAAAA|321|Pine 8th|Way|Suite 310|Guthrie|Hardin County|IL|61423|United States|-6|apartment| +46088|AAAAAAAAIAELAAAA|938|Davis |Boulevard|Suite U|Woodlawn|Schenectady County|NY|14098|United States|-5|apartment| +46089|AAAAAAAAJAELAAAA|939|10th 15th|Wy|Suite 50|Red Hill|Clay County|TN|34338|United States|-5|single family| +46090|AAAAAAAAKAELAAAA|554|Cedar Spring|Wy|Suite E|Bethel|Norton city|VA|25281|United States|-5|single family| +46091|AAAAAAAALAELAAAA|271|Ridge |Ave|Suite 30|Union Hill|Dolores County|CO|87746|United States|-7|condo| +46092|AAAAAAAAMAELAAAA|974|Oak West|Ln|Suite A|Oakwood|Cass County|MO|60169|United States|-6|apartment| +46093|AAAAAAAANAELAAAA|166|Washington Ridge|Drive|Suite 150|Mount Olive|Trinity County|TX|78059|United States|-6|apartment| +46094|AAAAAAAAOAELAAAA|148|6th |Dr.|Suite U|Glendale|Okanogan County|WA|93951|United States|-8|single family| +46095|AAAAAAAAPAELAAAA|649|Adams Hill|Blvd|Suite G|Riverview|Miner County|SD|59003|United States|-7|condo| +46096|AAAAAAAAABELAAAA|534|North |Dr.|Suite 20|Fairfield|Lamar County|TX|76192|United States|-6|single family| +46097|AAAAAAAABBELAAAA|353|Chestnut 8th|Circle|Suite 90|Sunnyside|Bland County|VA|21952|United States|-5|single family| +46098|AAAAAAAACBELAAAA|214|Williams |Ave|Suite X|Fairfield|Washburn County|WI|56192|United States|-6|condo| +46099|AAAAAAAADBELAAAA|858|11th |Ct.|Suite 410|Pleasant Grove|Nicholas County|WV|24136|United States|-5|condo| +46100|AAAAAAAAEBELAAAA|231|Lincoln Birch|Road|Suite 440|New Hope|Monongalia County|WV|29431|United States|-5|condo| +46101|AAAAAAAAFBELAAAA|413|Dogwood |Way|Suite 330|Westgate|Barry County|MO|62366|United States|-6|single family| +46102|AAAAAAAAGBELAAAA|12|14th 13th|Avenue|Suite U|Deerfield|Dooly County|GA|39840|United States|-5|apartment| +46103|AAAAAAAAHBELAAAA|858|West |ST|Suite 490|Carter|Augusta County|VA|20919|United States|-5|apartment| +46104|AAAAAAAAIBELAAAA|395|6th |Ln|Suite 220|Buena Vista|Shasta County|CA|95752|United States|-8|condo| +46105|AAAAAAAAJBELAAAA|976|Lake Oak|Parkway|Suite I|Crossroads|Aleutians East Borough|AK|90534|United States|-9|single family| +46106|AAAAAAAAKBELAAAA|549|Madison |Drive|Suite 70|Springdale|Grand County|UT|88883|United States|-7|condo| +46107|AAAAAAAALBELAAAA|368|Mill Hill|Cir.|Suite 110|Liberty|Lawrence County|AR|73451|United States|-6|apartment| +46108|AAAAAAAAMBELAAAA|709|Main |Cir.|Suite X|Lenox|Monroe County|IN|41143|United States|-5|condo| +46109|AAAAAAAANBELAAAA|34|Poplar Elevnth|Ln|Suite Y|Walnut Grove|Clay County|AR|77752|United States|-6|condo| +46110|AAAAAAAAOBELAAAA|207|Spruce |Lane|Suite 310|Five Forks|Juab County|UT|82293|United States|-7|condo| +46111|AAAAAAAAPBELAAAA|732|8th |Ave|Suite W|Crossroads|Glades County|FL|30534|United States|-5|single family| +46112|AAAAAAAAACELAAAA|714|West |Dr.|Suite K|Tanglewood|Hitchcock County|NE|68994|United States|-6|apartment| +46113|AAAAAAAABCELAAAA||||Suite 240|Harmony|||85804|||apartment| +46114|AAAAAAAACCELAAAA|977|Poplar 3rd|Wy|Suite 430|Willow|Alger County|MI|46798|United States|-5|single family| +46115|AAAAAAAADCELAAAA|299|Second Highland|RD|Suite 380|Shady Grove|Sagadahoc County|ME|03412|United States|-5|apartment| +46116|AAAAAAAAECELAAAA|755|Maple |Street|Suite 180|Peru|Pasquotank County|NC|20302|United States|-5|apartment| +46117|AAAAAAAAFCELAAAA|758|Spruce Hillcrest|Wy|Suite G|Plainview|Foard County|TX|73683|United States|-6|single family| +46118|AAAAAAAAGCELAAAA|404|Walnut |Blvd|Suite 30|Highland|Live Oak County|TX|79454|United States|-6|apartment| +46119|AAAAAAAAHCELAAAA|556|Spring 13th|Avenue|Suite I|Cedar Grove|Ward County|ND|50411|United States|-6|single family| +46120|AAAAAAAAICELAAAA|157|6th Railroad|Road|Suite 240|Union Hill|Carroll County|KY|47746|United States|-6|condo| +46121|AAAAAAAAJCELAAAA|729|Maple |Boulevard|Suite 40|Oak Grove|Madison County|OH|48370|United States|-5|apartment| +46122|AAAAAAAAKCELAAAA|973|Cedar |Road|Suite 300|Springfield|Richardson County|NE|69303|United States|-7|apartment| +46123|AAAAAAAALCELAAAA|716|5th |Street|Suite O|New Hope|Putnam County|NY|19431|United States|-5|apartment| +46124|AAAAAAAAMCELAAAA|256|4th Laurel|Drive|Suite 80|Concord|Baker County|FL|34107|United States|-5|apartment| +46125|AAAAAAAANCELAAAA|507|8th Hillcrest|RD|Suite V|Farmington|Lenoir County|NC|29145|United States|-5|apartment| +46126|AAAAAAAAOCELAAAA|915|Main Church|Street|Suite 220|Woodland|Wasatch County|UT|84854|United States|-7|apartment| +46127|AAAAAAAAPCELAAAA|874|Cherry Third|Drive|Suite 10|Harmony|Roberts County|TX|75804|United States|-6|single family| +46128|AAAAAAAAADELAAAA|71|Lee |Ct.|Suite 420|Green Acres|Amador County|CA|97683|United States|-8|single family| +46129|AAAAAAAABDELAAAA|60|Johnson College|Lane|Suite 480|Franklin|Martin County|MN|59101|United States|-6|apartment| +46130|AAAAAAAACDELAAAA|476|Railroad |Parkway|Suite 390|Lakewood|Fremont County|ID|88877|United States|-7|single family| +46131|AAAAAAAADDELAAAA|210|Maple |Street|Suite 370|Wildwood|Hutchinson County|TX|76871|United States|-6|apartment| +46132|AAAAAAAAEDELAAAA|783|Sycamore 5th|Drive|Suite 150|Georgetown|Jefferson County|KS|67057|United States|-6|condo| +46133|AAAAAAAAFDELAAAA|326|Church 3rd|ST|Suite W|Glenwood|Montgomery County|TX|73511|United States|-6|condo| +46134|AAAAAAAAGDELAAAA|321|4th |Blvd|Suite 160|Enterprise|Park County|CO|81757|United States|-7|condo| +46135|AAAAAAAAHDELAAAA|15|10th Lake|Avenue|Suite W|Bethel|Foster County|ND|55281|United States|-6|single family| +46136|AAAAAAAAIDELAAAA|1000|Oak Tenth|Ct.|Suite E|Five Forks|Liberty County|FL|32293|United States|-5|single family| +46137|AAAAAAAAJDELAAAA|328|Fifth 4th|ST|Suite N|Bridgeport|Fauquier County|VA|25817|United States|-5|condo| +46138|AAAAAAAAKDELAAAA|691|Meadow |Ct.|Suite 10|Wildwood|Wyoming County|WV|26871|United States|-5|condo| +46139|AAAAAAAALDELAAAA|473|Jackson |Wy|Suite 320|Red Hill|Marshall County|SD|54338|United States|-7|apartment| +46140|AAAAAAAAMDELAAAA|746|||Suite 220||Cole County|MO||||| +46141|AAAAAAAANDELAAAA|875|Spruce |Wy|Suite 340|Franklin|Gillespie County|TX|79101|United States|-6|condo| +46142|AAAAAAAAODELAAAA|182|Poplar Main|Lane|Suite H|Newport|Yancey County|NC|21521|United States|-5|apartment| +46143|AAAAAAAAPDELAAAA|104|Lake |Ct.|Suite 490|Ryan|Walker County|AL|30525|United States|-6|apartment| +46144|AAAAAAAAAEELAAAA|641|Church |RD|Suite G|Midway|Harlan County|KY|41904|United States|-6|condo| +46145|AAAAAAAABEELAAAA|454|9th |Parkway|Suite 10|Woodlawn|Seward County|NE|64098|United States|-7|condo| +46146|AAAAAAAACEELAAAA|170|View |Pkwy|Suite 50|Lakeview|Turner County|SD|58579|United States|-7|apartment| +46147|AAAAAAAADEELAAAA|143|Elm 15th|Avenue|Suite 430|Lakewood|Yalobusha County|MS|58877|United States|-6|condo| +46148|AAAAAAAAEEELAAAA|358|8th 8th|Road|Suite B|Shady Grove|Shelby County|IA|52812|United States|-6|condo| +46149|AAAAAAAAFEELAAAA|194|Park |ST||Huntsville||TX|77570|United States||condo| +46150|AAAAAAAAGEELAAAA|796|Mill West|Road|Suite U|Pleasant Grove|Henderson County|TX|74136|United States|-6|apartment| +46151|AAAAAAAAHEELAAAA|323|View Chestnut|ST|Suite 70|Newtown|Okmulgee County|OK|71749|United States|-6|single family| +46152|AAAAAAAAIEELAAAA|146|Pine |Ave|Suite 10|Clifton|Ravalli County|MT|68014|United States|-7|apartment| +46153|AAAAAAAAJEELAAAA||Wilson Elm|Ave||Clinton|Oneida County|NY||||apartment| +46154|AAAAAAAAKEELAAAA|436|Elevnth Smith|Blvd|Suite S|Forest Hills|Haralson County|GA|39237|United States|-5|condo| +46155|AAAAAAAALEELAAAA|227|Park Lakeview|Blvd|Suite 240|Maple Grove|Prince William County|VA|28252|United States|-5|condo| +46156|AAAAAAAAMEELAAAA||7th |Boulevard|Suite Q|Five Forks||MI|42293|United States|-5|single family| +46157|AAAAAAAANEELAAAA|381|West Valley|RD|Suite 300|Greenfield|Weakley County|TN|35038|United States|-6|single family| +46158|AAAAAAAAOEELAAAA|2|||Suite 50||Cumberland County|KY|48048||-6|condo| +46159|AAAAAAAAPEELAAAA|989|Lincoln |Blvd|Suite J|Snug Harbor|Lehigh County|PA|17936|United States|-5|condo| +46160|AAAAAAAAAFELAAAA||Jackson Park||Suite D|Centerville|Palo Pinto County|TX|||-6|single family| +46161|AAAAAAAABFELAAAA|713|Elm Railroad|Blvd|Suite 160|Pleasant Grove|Chickasaw County|MS|54136|United States|-6|single family| +46162|AAAAAAAACFELAAAA|600|2nd |Boulevard|Suite Y|Spring Hill|Wilkinson County|GA|36787|United States|-5|condo| +46163|AAAAAAAADFELAAAA|331|River Franklin|ST|Suite 170|Union|Magoffin County|KY|48721|United States|-5|condo| +46164|AAAAAAAAEFELAAAA|816|8th |Boulevard|Suite A|Riverdale|Whitfield County|GA|39391|United States|-5|single family| +46165|AAAAAAAAFFELAAAA|481|Chestnut |Circle|Suite 100|Mountain View|Sussex County|VA|24466|United States|-5|apartment| +46166|AAAAAAAAGFELAAAA|777|South |Circle|Suite N|Jamestown|Greeley County|NE|66867|United States|-6|apartment| +46167|AAAAAAAAHFELAAAA|963|Locust |Dr.|Suite 70|Farmington|Morton County|KS|69145|United States|-6|apartment| +46168|AAAAAAAAIFELAAAA|747|Elm |Blvd|Suite U|Woodland|Lyman County|SD|54854|United States|-7|apartment| +46169|AAAAAAAAJFELAAAA|284|11th |Circle|Suite G|Clinton|Cameron Parish|LA|78222|United States|-6|single family| +46170|AAAAAAAAKFELAAAA|634|Railroad Hillcrest|Wy|Suite P|Red Hill|Wichita County|KS|64338|United States|-6|single family| +46171|AAAAAAAALFELAAAA|999|East |Drive|Suite 170|Buena Vista|Adams County|IA|55752|United States|-6|apartment| +46172|AAAAAAAAMFELAAAA|897|Spring Wilson|Pkwy|Suite O|Pleasant Hill|McIntosh County|OK|73604|United States|-6|apartment| +46173|AAAAAAAANFELAAAA|245|2nd |Ave|Suite 330|Red Hill|Newton County|IN|44338|United States|-5|single family| +46174|AAAAAAAAOFELAAAA|389|River |Dr.|Suite E|Newport|Cass County|IA|51521|United States|-6|apartment| +46175|AAAAAAAAPFELAAAA|521|13th |Circle|Suite I|Glenwood|Yakima County|WA|93511|United States|-8|condo| +46176|AAAAAAAAAGELAAAA|653|2nd Elm|Cir.|Suite 90|Cedar Grove|Simpson County|MS|50411|United States|-6|apartment| +46177|AAAAAAAABGELAAAA|123|Second |Way|Suite 230|Brownsville|Hopkins County|KY|49310|United States|-6|apartment| +46178|AAAAAAAACGELAAAA|112|Church Meadow|Ct.|Suite 290|Bethel|Seminole County|OK|75281|United States|-6|condo| +46179|AAAAAAAADGELAAAA|802|First |Ct.|Suite 200|Jamestown|Keokuk County|IA|56867|United States|-6|apartment| +46180|AAAAAAAAEGELAAAA|613|Lakeview |Cir.|Suite W|Newtown|Anderson County|TN|31749|United States|-5|apartment| +46181|AAAAAAAAFGELAAAA|87|1st Highland|Drive|Suite 170|Clifton|Martinsville city|VA|28014|United States|-5|condo| +46182|AAAAAAAAGGELAAAA|244|Oak |Avenue|Suite 70|Union|Logan County|KY|48721|United States|-5|condo| +46183|AAAAAAAAHGELAAAA|436|10th |Parkway|Suite 150|Glenwood|Lincoln County|NE|63511|United States|-7|single family| +46184|AAAAAAAAIGELAAAA|269|Locust Davis|Wy|Suite 110|Sunnyside|Spotsylvania County|VA|21952|United States|-5|condo| +46185|AAAAAAAAJGELAAAA|621|Washington |Wy|Suite M|Stringtown|Darlington County|SC|20162|United States|-5|single family| +46186|AAAAAAAAKGELAAAA|420|Center |Wy|Suite 250|Tyrone|Emmons County|ND|51201|United States|-6|apartment| +46187|AAAAAAAALGELAAAA|392|Lincoln |Cir.|Suite 100|Plainview|Custer County|SD|53683|United States|-6|condo| +46188|AAAAAAAAMGELAAAA||5th |Boulevard|Suite N||McPherson County||66668||-7|condo| +46189|AAAAAAAANGELAAAA|839|Valley |Street|Suite S|Fairfield|Oconee County|GA|36192|United States|-5|apartment| +46190|AAAAAAAAOGELAAAA|755|4th Cedar|Ln|Suite P|Centerville|Whatcom County|WA|90059|United States|-8|condo| +46191|AAAAAAAAPGELAAAA|800|Church Laurel|Court|Suite B|Fairfield|Hancock County|TN|36192|United States|-5|apartment| +46192|AAAAAAAAAHELAAAA|343|Miller |Street|Suite Q|Riverview|Allegany County|MD|29003|United States|-5|apartment| +46193|AAAAAAAABHELAAAA|712|8th Maple|Avenue|Suite 440|Lakeview|Pine County|MN|58579|United States|-6|condo| +46194|AAAAAAAACHELAAAA|961|||||||58370|United States||| +46195|AAAAAAAADHELAAAA|180|Oak |Blvd|Suite 10|Granite|Ellsworth County|KS|66284|United States|-6|condo| +46196|AAAAAAAAEHELAAAA|963|Ridge 6th|Court|Suite 380|Greenville|Campbell County|KY|41387|United States|-6|apartment| +46197|AAAAAAAAFHELAAAA|637|Center 3rd|ST|Suite 0|Mount Zion|Cape May County|NJ|08654|United States|-5|single family| +46198|AAAAAAAAGHELAAAA|853|1st 7th|RD|Suite 40|Stringtown|Bee County|TX|70162|United States|-6|apartment| +46199|AAAAAAAAHHELAAAA|616|Ninth |Court|Suite K|Plainview|Ouray County|CO|83683|United States|-7|apartment| +46200|AAAAAAAAIHELAAAA|40|13th |Avenue|Suite B|Florence|Antrim County|MI|43394|United States|-5|condo| +46201|AAAAAAAAJHELAAAA|277|Chestnut Seventh|Drive|Suite Q|Mount Pleasant|Clay County|GA|31933|United States|-5|single family| +46202|AAAAAAAAKHELAAAA|375|Walnut |RD|Suite B|Oak Hill|Butler County|NE|67838|United States|-6|single family| +46203|AAAAAAAALHELAAAA|335|Church |Boulevard|Suite 160|Green Acres|Iroquois County|IL|67683|United States|-6|apartment| +46204|AAAAAAAAMHELAAAA|96|Jefferson Birch|Way|Suite O|Shady Grove|Pitt County|NC|22812|United States|-5|condo| +46205|AAAAAAAANHELAAAA|418|Washington Spring|Wy|Suite 360|Mount Pleasant|Crawford County|IA|51933|United States|-6|condo| +46206|AAAAAAAAOHELAAAA|296|1st |Avenue|Suite E|Midway|Moniteau County|MO|61904|United States|-6|condo| +46207|AAAAAAAAPHELAAAA|514|Hickory |Cir.|Suite 110|Highland Park|Uinta County|WY|86534|United States|-7|condo| +46208|AAAAAAAAAIELAAAA|752|Adams |Dr.|Suite D|Liberty|Lea County|NM|83451|United States|-7|apartment| +46209|AAAAAAAABIELAAAA|497|Ash |Boulevard|Suite V|Woodville|Perry County|IN|44289|United States|-5|apartment| +46210|AAAAAAAACIELAAAA|334|Elm |Parkway|Suite K|Oakland|Butler County|AL|39843|United States|-6|single family| +46211|AAAAAAAADIELAAAA|750|13th |Court|Suite F|Woodville|Isabella County|MI|44289|United States|-5|apartment| +46212|AAAAAAAAEIELAAAA|872|Second |Boulevard|Suite N|Friendship|Greene County|IN|44536|United States|-5|single family| +46213|AAAAAAAAFIELAAAA|549|Maple |Boulevard|Suite 230|Macedonia|Weston County|WY|81087|United States|-7|single family| +46214|AAAAAAAAGIELAAAA|593|East |Ln|Suite V|Brunswick|Webster County|IA|54642|United States|-6|condo| +46215|AAAAAAAAHIELAAAA|779|First Lakeview|Court|Suite 390|Hamilton|Pike County|OH|42808|United States|-5|apartment| +46216|AAAAAAAAIIELAAAA|506|Ash |Court|Suite 330|Kingston|Shackelford County|TX|74975|United States|-6|condo| +46217|AAAAAAAAJIELAAAA|553|Oak |RD|Suite 370|Riverview|Payette County|ID|89003|United States|-7|apartment| +46218|AAAAAAAAKIELAAAA|50|Sixth |Ln|Suite E|Hillcrest|Nacogdoches County|TX|73003|United States|-6|single family| +46219|AAAAAAAALIELAAAA|315|Railroad Meadow|Ln|Suite 360|White Oak|Smith County|TX|76668|United States|-6|apartment| +46220|AAAAAAAAMIELAAAA|602|Cherry Adams|Pkwy|Suite L|Enterprise|Suffolk city|VA|21757|United States|-5|condo| +46221|AAAAAAAANIELAAAA|463|Lincoln |Street|Suite F|Fowler|Charles City County|VA|21083|United States|-5|condo| +46222|AAAAAAAAOIELAAAA|327|10th Second|Blvd|Suite X|Midway|Mineral County|CO|81904|United States|-7|apartment| +46223|AAAAAAAAPIELAAAA|466|Cedar Central|ST|Suite I|Lebanon|Franklin County|MO|62898|United States|-6|single family| +46224|AAAAAAAAAJELAAAA|677|View 1st|Cir.|Suite V|Ashland|Hamblen County|TN|34244|United States|-5|condo| +46225|AAAAAAAABJELAAAA|853|Highland Sunset|Lane|Suite 410|Shady Grove|Yates County|NY|12812|United States|-5|single family| +46226|AAAAAAAACJELAAAA|671|Oak |RD|Suite 30|Maple Grove|Norman County|MN|58252|United States|-6|condo| +46227|AAAAAAAADJELAAAA|408|Fourth |Blvd|Suite 300|Lincoln|Randolph County|AL|31289|United States|-6|apartment| +46228|AAAAAAAAEJELAAAA|589|Ridge Meadow|Street|Suite C|Concord|Menard County|TX|74107|United States|-6|condo| +46229|AAAAAAAAFJELAAAA|775|Green |ST|Suite 330|Oak Hill|Gove County|KS|67838|United States|-6|condo| +46230|AAAAAAAAGJELAAAA|288|River |Ln|Suite 110|Bridgeport|Garland County|AR|75817|United States|-6|single family| +46231|AAAAAAAAHJELAAAA|670|Smith 15th|Street|Suite Q|Buena Vista|Sumter County|FL|35752|United States|-5|single family| +46232|AAAAAAAAIJELAAAA|501|Laurel Railroad|Drive|Suite H|Jenkins|Tripp County|SD|57292|United States|-7|single family| +46233|AAAAAAAAJJELAAAA|721|Ninth 2nd|Court|Suite E|Indian Village|Irwin County|GA|31075|United States|-5|apartment| +46234|AAAAAAAAKJELAAAA|108|Washington View|ST|Suite P|Greenville|Bladen County|NC|21387|United States|-5|single family| +46235|AAAAAAAALJELAAAA|858|Fifth Washington|Road|Suite 260|Lincoln|Madison County|TX|71289|United States|-6|apartment| +46236|AAAAAAAAMJELAAAA|534|13th |Cir.|Suite B|Plainview|Hale County|AL|33683|United States|-6|condo| +46237|AAAAAAAANJELAAAA|159|12th Laurel|Blvd|Suite 120|Greenwood|Pulaski County|IL|68828|United States|-6|condo| +46238|AAAAAAAAOJELAAAA|502|7th Mill|Ct.|Suite 40|Flatwoods|Torrance County|NM|84212|United States|-7|condo| +46239|AAAAAAAAPJELAAAA|221|Third Cherry|Court|Suite 100|Sullivan|Cerro Gordo County|IA|50451|United States|-6|condo| +46240|AAAAAAAAAKELAAAA|9|12th East|Ln|Suite 300|Antioch|Stephenson County|IL|68605|United States|-6|apartment| +46241|AAAAAAAABKELAAAA|429|Ash |Lane|Suite 360|Glenwood|Decatur County|IN|43511|United States|-5|single family| +46242|AAAAAAAACKELAAAA|838|Wilson |Lane|Suite I|Pleasant Hill|Swain County|NC|23604|United States|-5|single family| +46243|AAAAAAAADKELAAAA|328|8th |RD|Suite S|New Hope|Lincoln County|OK|79431|United States|-6|condo| +46244|AAAAAAAAEKELAAAA|423|Pine Washington|Road|Suite 0|Providence|Schley County|GA|36614|United States|-5|condo| +46245|AAAAAAAAFKELAAAA|298|12th |Wy|Suite C|Argyle|Gallatin County|IL|68722|United States|-6|condo| +46246|AAAAAAAAGKELAAAA|178|Lakeview 8th|Court|Suite 80|Oak Ridge|Currituck County|NC|28371|United States|-5|condo| +46247|AAAAAAAAHKELAAAA|911|Center |Cir.|Suite W|Hamilton|Chaffee County|CO|82808|United States|-7|condo| +46248|AAAAAAAAIKELAAAA|413|Jefferson |Cir.|Suite K|Woodland|Marquette County|WI|54854|United States|-6|condo| +46249|AAAAAAAAJKELAAAA|735|Johnson 8th|Ct.|Suite 30|Friendship|Salem city|VA|24536|United States|-5|apartment| +46250|AAAAAAAAKKELAAAA|435|Elevnth |Road|Suite D|Red Bank|DeKalb County|IL|64975|United States|-6|apartment| +46251|AAAAAAAALKELAAAA|621|Lake Fourth|Cir.|Suite 110|Mountain View|Laclede County|MO|64466|United States|-6|single family| +46252|AAAAAAAAMKELAAAA|636|Maple |Cir.|Suite S|Spring Hill|Halifax County|NC|26787|United States|-5|apartment| +46253|AAAAAAAANKELAAAA|519|Washington |Court|Suite 250|Pleasant Grove|Cattaraugus County|NY|14136|United States|-5|single family| +46254|AAAAAAAAOKELAAAA|956|Walnut |Parkway|Suite I|New Hope|Menard County|TX|79431|United States|-6|condo| +46255|AAAAAAAAPKELAAAA|59||Avenue||Oakdale|Knott County||||-5|condo| +46256|AAAAAAAAALELAAAA|759|8th |Cir.|Suite 470|Hidden Valley|Richmond County|VA|25521|United States|-5|apartment| +46257|AAAAAAAABLELAAAA|863|Williams |ST|Suite K|Newport|Brooks County|TX|71521|United States|-6|apartment| +46258|AAAAAAAACLELAAAA|745|Cedar |Ln|Suite 40|Bunker Hill|Jasper County|GA|30150|United States|-5|apartment| +46259|AAAAAAAADLELAAAA|103|Walnut Cedar|Pkwy|Suite 240|Buena Vista|Aiken County|SC|25752|United States|-5|single family| +46260|AAAAAAAAELELAAAA|696|10th Mill|Road|Suite W|Woodland|Carter County|OK|74854|United States|-6|condo| +46261|AAAAAAAAFLELAAAA|361|Pine College|Pkwy|Suite 120|Oak Grove|Brantley County|GA|38370|United States|-5|apartment| +46262|AAAAAAAAGLELAAAA|337|Central |Boulevard|Suite 80|Unionville|Meeker County|MN|51711|United States|-6|condo| +46263|AAAAAAAAHLELAAAA|633|Oak |Cir.|Suite 440|Sulphur Springs|Panola County|MS|58354|United States|-6|apartment| +46264|AAAAAAAAILELAAAA|984|Hill 8th|Avenue|Suite 300|Pleasant Hill|Bingham County|ID|83604|United States|-7|single family| +46265|AAAAAAAAJLELAAAA|673|5th Oak|Ln|Suite H|Jamestown|Mason County|WA|96867|United States|-8|apartment| +46266|AAAAAAAAKLELAAAA|712|Central |Ave|Suite J|Sunnyside|Surry County|NC|21952|United States|-5|condo| +46267|AAAAAAAALLELAAAA|796|Hickory Smith|Circle|Suite O|Hillcrest|Guadalupe County|TX|73003|United States|-6|condo| +46268|AAAAAAAAMLELAAAA|831|1st |Ct.|Suite 310|Newtown|Terry County|TX|71749|United States|-6|single family| +46269|AAAAAAAANLELAAAA|703|Lake 12th|Ct.|Suite 350|Hamilton|Dinwiddie County|VA|22808|United States|-5|apartment| +46270|AAAAAAAAOLELAAAA|69|Spring |Cir.|Suite V|White Oak|Prowers County|CO|86668|United States|-7|single family| +46271|AAAAAAAAPLELAAAA|843|Oak |ST|Suite 140|Oakdale|Gosper County|NE|69584|United States|-6|apartment| +46272|AAAAAAAAAMELAAAA|848|Pine |Pkwy|Suite V|Mountain View|Greene County|GA|34466|United States|-5|apartment| +46273|AAAAAAAABMELAAAA|986|Pine Lake|Pkwy|Suite 50|Brookwood|Whiteside County|IL|60965|United States|-6|condo| +46274|AAAAAAAACMELAAAA|266|Ridge Oak|Boulevard|Suite C|Greenwood|Faribault County|MN|58828|United States|-6|condo| +46275|AAAAAAAADMELAAAA|744|Birch 4th|Ct.|Suite X|Franklin|Trego County|KS|69101|United States|-6|single family| +46276|AAAAAAAAEMELAAAA|484|Smith |Street|Suite J|Macedonia|Medina County|TX|71087|United States|-6|single family| +46277|AAAAAAAAFMELAAAA|893|1st |Street|Suite O|Florence|Cowley County|KS|63394|United States|-6|single family| +46278|AAAAAAAAGMELAAAA|674|Jackson Cherry|Ct.|Suite 180|Mountain View|Hooker County|NE|64466|United States|-7|condo| +46279|AAAAAAAAHMELAAAA|918|Jackson Park|Blvd|Suite 40|White Oak|Jefferson County|WI|56668|United States|-6|apartment| +46280|AAAAAAAAIMELAAAA|760|Walnut |Ln|Suite 420|Oakdale|Calumet County|WI|59584|United States|-6|apartment| +46281|AAAAAAAAJMELAAAA|819|Mill 1st|Ave|Suite T|Jamestown|McCulloch County|TX|76867|United States|-6|condo| +46282|AAAAAAAAKMELAAAA|67|Church Meadow|Road|Suite 320|Oakland|Cimarron County|OK|79843|United States|-6|condo| +46283|AAAAAAAALMELAAAA|21|Hillcrest |Wy|Suite M|Macedonia|Morgan County|MO|61087|United States|-6|condo| +46284|AAAAAAAAMMELAAAA|316|West Cedar|Way|Suite Y|Riverview|Surry County|VA|29003|United States|-5|single family| +46285|AAAAAAAANMELAAAA|151|Woodland |Drive|Suite G|Five Forks|Beltrami County|MN|52293|United States|-6|apartment| +46286|AAAAAAAAOMELAAAA|172|Highland |Ct.|Suite 170|Unionville|Dallam County|TX|71711|United States|-6|single family| +46287|AAAAAAAAPMELAAAA|533|7th |Circle|Suite 450|Arthur|Forrest County|MS|55965|United States|-6|single family| +46288|AAAAAAAAANELAAAA||Poplar Park|Blvd|Suite D|Woodville|Clay County||74289|||apartment| +46289|AAAAAAAABNELAAAA|711|Oak |Street|Suite 470|Liberty|Tyler County|TX|73451|United States|-6|single family| +46290|AAAAAAAACNELAAAA|262|Valley |RD|Suite F|Lakeview|Brown County|NE|68579|United States|-6|single family| +46291|AAAAAAAADNELAAAA||Wilson Jefferson||||||77292|||| +46292|AAAAAAAAENELAAAA|396|Elm |Lane|Suite 320|Bethel|Morton County|KS|65281|United States|-6|single family| +46293|AAAAAAAAFNELAAAA|652|Jefferson |Boulevard|Suite S|Liberty|McCreary County|KY|43451|United States|-5|single family| +46294|AAAAAAAAGNELAAAA|222|View |Ave|Suite E|Harmony|Siskiyou County|CA|95804|United States|-8|condo| +46295|AAAAAAAAHNELAAAA|93|First Locust|Ln|Suite 430|Tanglewood|Wilkinson County|MS|58994|United States|-6|apartment| +46296|AAAAAAAAINELAAAA|369|1st |Avenue|Suite J|Highland Park|Richmond County|NC|26534|United States|-5|condo| +46297|AAAAAAAAJNELAAAA|32|Washington First|RD||Salem||KY|48048|United States|-5|single family| +46298|AAAAAAAAKNELAAAA|786|South |Court|Suite 30|Five Points|Camden County|GA|36098|United States|-5|condo| +46299|AAAAAAAALNELAAAA|945|Williams |Pkwy|Suite 90|Bridgeport|Baraga County|MI|45817|United States|-5|apartment| +46300|AAAAAAAAMNELAAAA|500|First |RD|Suite M|Wildwood|Bibb County|GA|36871|United States|-5|apartment| +46301|AAAAAAAANNELAAAA|678|View 4th|Avenue|Suite 350|Pleasant Valley|Tioga County|PA|12477|United States|-5|single family| +46302|AAAAAAAAONELAAAA|54|Walnut |Ln|Suite 70|Pleasant Hill|Nacogdoches County|TX|73604|United States|-6|apartment| +46303|AAAAAAAAPNELAAAA|560|Hickory South|Avenue|Suite 370|Ashland|Greene County|PA|14244|United States|-5|single family| +46304|AAAAAAAAAOELAAAA|667|13th |Drive|Suite G|Glenwood|Russell County|AL|33511|United States|-6|condo| +46305|AAAAAAAABOELAAAA|389|College |Drive|Suite 300|Belmont|Hardin County|TN|30191|United States|-5|condo| +46306|AAAAAAAACOELAAAA|794|Hill Park|Court|Suite 330|Hopewell|Navarro County|TX|70587|United States|-6|single family| +46307|AAAAAAAADOELAAAA|153|Lincoln |Ct.|Suite 90|Jackson|Bourbon County|KY|49583|United States|-6|single family| +46308|AAAAAAAAEOELAAAA|959|6th |Lane|Suite B|Spring Valley|Elliott County|KY|46060|United States|-6|condo| +46309|AAAAAAAAFOELAAAA|437|Pine |Street|Suite T|Farmington|Washakie County|WY|89145|United States|-7|single family| +46310|AAAAAAAAGOELAAAA|611|Miller 13th|Way|Suite 190|Hamilton|Sierra County|NM|82808|United States|-7|condo| +46311|AAAAAAAAHOELAAAA|677|South 5th|Dr.|Suite M|Lake View|Anderson County|KY|48589|United States|-6|condo| +46312|AAAAAAAAIOELAAAA|569||Cir.||||KY|43394|United States|-5|| +46313|AAAAAAAAJOELAAAA|504|Center |Road|Suite S|Rutland|Iowa County|WI|58375|United States|-6|apartment| +46314|AAAAAAAAKOELAAAA|198|2nd |Pkwy|Suite V|Hopewell|Wright County|IA|50587|United States|-6|single family| +46315|AAAAAAAALOELAAAA|829|Maple Forest|Lane|Suite 270|Spring Valley|Seminole County|FL|36060|United States|-5|apartment| +46316|AAAAAAAAMOELAAAA|451|Smith Hillcrest|Way|Suite 170|Stringtown|Providence County|RI|00762|United States|-5|condo| +46317|AAAAAAAANOELAAAA|34|Park 13th|Lane|Suite E|Elm Grove|Morton County|KS|63298|United States|-6|condo| +46318|AAAAAAAAOOELAAAA|850|9th Davis|Ave|Suite 190|Bunker Hill|Suwannee County|FL|30150|United States|-5|single family| +46319|AAAAAAAAPOELAAAA|355|4th 3rd|Court|Suite 60|Marion|Duval County|FL|30399|United States|-5|single family| +46320|AAAAAAAAAPELAAAA|174|Main |||Oak Ridge||OR||United States|-8|| +46321|AAAAAAAABPELAAAA|97|8th Miller|Ave|Suite 440|Antioch|Merced County|CA|98605|United States|-8|condo| +46322|AAAAAAAACPELAAAA|91|Willow |Ave|Suite 0|Union|Delta County|CO|88721|United States|-7|single family| +46323|AAAAAAAADPELAAAA|673|6th Lake|ST|Suite G|Lakeview|Traill County|ND|58579|United States|-6|apartment| +46324|AAAAAAAAEPELAAAA|270|South Meadow|Court|Suite E|Five Forks|Spokane County|WA|92293|United States|-8|single family| +46325|AAAAAAAAFPELAAAA|171||||Midway|||91904|||| +46326|AAAAAAAAGPELAAAA|766|3rd |Dr.|Suite T|Bethel|Union County|GA|35281|United States|-5|condo| +46327|AAAAAAAAHPELAAAA|488|Jefferson |Blvd|Suite 380|Woodville|Ashland County|WI|54289|United States|-6|condo| +46328|AAAAAAAAIPELAAAA|763|Elm |Pkwy|Suite 200|Jackson|Conway County|AR|79583|United States|-6|single family| +46329|AAAAAAAAJPELAAAA|905|Railroad View|Ln|Suite 280|Mount Olive|Williamson County|TN|38059|United States|-5|apartment| +46330|AAAAAAAAKPELAAAA|612|Mill |Cir.|Suite 40|Shady Grove|Vernon County|WI|52812|United States|-6|condo| +46331|AAAAAAAALPELAAAA|235|Highland |Wy|Suite 440|Franklin|Cooper County|MO|69101|United States|-6|condo| +46332|AAAAAAAAMPELAAAA|85|View |Parkway|Suite S|Unionville|Schuylkill County|PA|11711|United States|-5|condo| +46333|AAAAAAAANPELAAAA|409|15th 12th|Wy|Suite J|Stringtown|Beaver County|OK|70162|United States|-6|single family| +46334|AAAAAAAAOPELAAAA|943|Church South|ST|Suite P|Lakeview|Bristol city|VA|28579|United States|-5|apartment| +46335|AAAAAAAAPPELAAAA|332|Third West|Street|Suite 250|Oak Ridge|Lee County|AL|38371|United States|-6|apartment| +46336|AAAAAAAAAAFLAAAA|34|Hill Birch|Court|Suite Q|Salem|Green Lake County|WI|58048|United States|-6|single family| +46337|AAAAAAAABAFLAAAA|599|Lincoln 11th|Road|Suite 470|Friendship|Hoke County|NC|24536|United States|-5|condo| +46338|AAAAAAAACAFLAAAA|589|Lincoln |Road|Suite 130|Waterloo|Lafourche Parish|LA|71675|United States|-6|single family| +46339|AAAAAAAADAFLAAAA|311|Spruce Third|Boulevard|Suite R|Spring Hill|Walla Walla County|WA|96787|United States|-8|apartment| +46340|AAAAAAAAEAFLAAAA|360|Williams Valley|Boulevard|Suite 170|Gary|Duval County|FL|30418|United States|-5|single family| +46341|AAAAAAAAFAFLAAAA|418|Maple Locust|Drive|Suite B|Crossroads|Calhoun County|AR|70534|United States|-6|condo| +46342|AAAAAAAAGAFLAAAA|817|Walnut |Pkwy|Suite C|Highland Park|Bear Lake County|ID|86534|United States|-7|condo| +46343|AAAAAAAAHAFLAAAA|464|Third Park|Wy|Suite 310|Highland|Huntington County|IN|49454|United States|-5|condo| +46344|AAAAAAAAIAFLAAAA|530|Pine College|Cir.|Suite 260|Red Hill|Wayne County|KY|44338|United States|-5|single family| +46345|AAAAAAAAJAFLAAAA|763|Second |Lane|Suite 470|Clifton|Bedford County|VA|28014|United States|-5|condo| +46346|AAAAAAAAKAFLAAAA|466|Oak |Dr.|Suite N|Arlington|Woodford County|IL|66557|United States|-6|apartment| +46347|AAAAAAAALAFLAAAA|634|Ridge Highland|Parkway|Suite 50|Sulphur Springs|Benton County|MS|58354|United States|-6|apartment| +46348|AAAAAAAAMAFLAAAA|578|Green |Avenue|Suite 40|Arlington|Cass County|IA|56557|United States|-6|single family| +46349|AAAAAAAANAFLAAAA|810|South Second|Pkwy|Suite 230|Manchester|Gibson County|TN|32884|United States|-5|single family| +46350|AAAAAAAAOAFLAAAA|257|Hill |Court|Suite Q|Bunker Hill|Kosciusko County|IN|40150|United States|-5|apartment| +46351|AAAAAAAAPAFLAAAA|893|Birch |Avenue|Suite O|Summit|Edmonson County|KY|40499|United States|-6|apartment| +46352|AAAAAAAAABFLAAAA|509|Forest Eigth|Wy|Suite J|Hopewell|Middlesex County|VA|20587|United States|-5|single family| +46353|AAAAAAAABBFLAAAA|516|1st |Drive|Suite S|Hopewell|Callaway County|MO|60587|United States|-6|single family| +46354|AAAAAAAACBFLAAAA|538||Lane||Green Acres||||||| +46355|AAAAAAAADBFLAAAA|394|4th 3rd|Ct.|Suite L|Five Points|Warren County|GA|36098|United States|-5|condo| +46356|AAAAAAAAEBFLAAAA|803|Oak Hickory|Pkwy|Suite N|Marion|Stark County|OH|40399|United States|-5|single family| +46357|AAAAAAAAFBFLAAAA|299|Meadow Park|Cir.|Suite I|Glenwood|Shelby County|OH|43511|United States|-5|condo| +46358|AAAAAAAAGBFLAAAA|716|Dogwood Elm|Cir.|Suite 320|Florence|Muscatine County|IA|53394|United States|-6|condo| +46359|AAAAAAAAHBFLAAAA|935|Mill |Avenue|Suite E|Five Forks|Morgan County|IL|62293|United States|-6|condo| +46360|AAAAAAAAIBFLAAAA|831|North Willow|Parkway|Suite K|Centerville|Judith Basin County|MT|60059|United States|-7|single family| +46361|AAAAAAAAJBFLAAAA|56|Elevnth East|Dr.|Suite 140|Country Club Estates|Ascension Parish|LA|76964|United States|-6|apartment| +46362|AAAAAAAAKBFLAAAA|704|Second Adams|Way|Suite 0|Kingston|Edgecombe County|NC|24975|United States|-5|apartment| +46363|AAAAAAAALBFLAAAA|652|Smith |Wy|Suite L|Centerville|Chase County|NE|60059|United States|-6|apartment| +46364|AAAAAAAAMBFLAAAA|376|Laurel Madison|Drive|Suite 250|Peoria|Rockingham County|NC|29818|United States|-5|condo| +46365|AAAAAAAANBFLAAAA|633|Ash |Drive|Suite V|Green Acres|Yates County|NY|17683|United States|-5|apartment| +46366|AAAAAAAAOBFLAAAA|688|West Dogwood|Drive|Suite 470|Fairview|Nez Perce County|ID|85709|United States|-7|apartment| +46367|AAAAAAAAPBFLAAAA|969|Forest |Circle|Suite X|Highland|Berrien County|GA|39454|United States|-5|apartment| +46368|AAAAAAAAACFLAAAA|130|North |Cir.|Suite 250|Antioch|Nacogdoches County|TX|78605|United States|-6|apartment| +46369|AAAAAAAABCFLAAAA|450|Johnson |Ln|Suite 130|Weldon|Cedar County|NE|66277|United States|-6|apartment| +46370|AAAAAAAACCFLAAAA|842|Madison |Blvd|Suite Y|Shiloh|Barry County|MI|49275|United States|-5|apartment| +46371|AAAAAAAADCFLAAAA|609|2nd Meadow|Ct.|Suite U|Hubbard|Lafayette County|MO|66291|United States|-6|condo| +46372|AAAAAAAAECFLAAAA|201|Cedar |Road|Suite 270|Midway|Yolo County|CA|91904|United States|-8|single family| +46373|AAAAAAAAFCFLAAAA|63|1st Oak|Drive|Suite O|Lakewood|Bee County|TX|78877|United States|-6|condo| +46374|AAAAAAAAGCFLAAAA|592|Hickory Washington|Court|Suite 100|Fairview|McMullen County|TX||||single family| +46375|AAAAAAAAHCFLAAAA|794|Church Cedar|Way|Suite V|Stringtown|Jackson County|KS|60162|United States|-6|condo| +46376|AAAAAAAAICFLAAAA|474||Drive|Suite 490||Phillips County||71749|||apartment| +46377|AAAAAAAAJCFLAAAA|811|Franklin Spruce||Suite Q|Antioch|Hancock County|KY||||condo| +46378|AAAAAAAAKCFLAAAA|911|Sunset |Ave|Suite 50|Pleasant Valley|Pottawatomie County|KS|62477|United States|-6|condo| +46379|AAAAAAAALCFLAAAA|538|1st |Ln|Suite T|Brownsville|Whitfield County|GA|39310|United States|-5|single family| +46380|AAAAAAAAMCFLAAAA|563|Forest Jackson|Circle|Suite Q|New Hope|Butte County|CA|99431|United States|-8|condo| +46381|AAAAAAAANCFLAAAA|613|Ash |Lane|Suite P|Bethel|Henry County|MO|65281|United States|-6|single family| +46382|AAAAAAAAOCFLAAAA|568|Woodland Washington|Blvd|Suite 420|Springdale|Hughes County|SD|58883|United States|-7|apartment| +46383|AAAAAAAAPCFLAAAA|707|Pine Valley|Lane|Suite 10|Weldon|Kings County|CA|96277|United States|-8|single family| +46384|AAAAAAAAADFLAAAA|807|Woodland |Ln|Suite K|Jackson|Lincoln County|KS|69583|United States|-6|single family| +46385|AAAAAAAABDFLAAAA|363|Fifth |Way|Suite Y|Fisher|Los Alamos County|NM|82819|United States|-7|single family| +46386|AAAAAAAACDFLAAAA|102|Hillcrest |Ct.|Suite 470|Oakdale|Owen County|KY|49584|United States|-5|condo| +46387|AAAAAAAADDFLAAAA|716|Sixth Adams|Road|Suite F|Spring Valley|Toombs County|GA|36060|United States|-5|single family| +46388|AAAAAAAAEDFLAAAA|256|Washington |Pkwy|Suite C|Lakewood|Craig County|VA|28877|United States|-5|apartment| +46389|AAAAAAAAFDFLAAAA|228|Washington Ridge|Dr.|Suite G|Oak Ridge|Boyd County|NE|68371|United States|-6|single family| +46390|AAAAAAAAGDFLAAAA|107|Jackson Spruce|Avenue|Suite 20|New Hope|Buffalo County|SD|59431|United States|-6|single family| +46391|AAAAAAAAHDFLAAAA|40|Jackson Center|Way|Suite 170|Whitesville|Vernon County|MO|65903|United States|-6|condo| +46392|AAAAAAAAIDFLAAAA|622|Lincoln |Blvd|Suite 290|Lincoln|Montgomery County|AL|31289|United States|-6|condo| +46393|AAAAAAAAJDFLAAAA|157|14th |Boulevard|Suite U|Lincoln|Dearborn County|IN|41289|United States|-5|apartment| +46394|AAAAAAAAKDFLAAAA|995|5th |Way|Suite 90|Lone Pine|Worcester County|MD|27441|United States|-5|apartment| +46395|AAAAAAAALDFLAAAA|143|Central View|Court|Suite 460|Bridgeport|Sullivan County|PA|15817|United States|-5|single family| +46396|AAAAAAAAMDFLAAAA|281|Locust Dogwood|RD|Suite I|Sunnyside|Culberson County|TX|71952|United States|-6|single family| +46397|AAAAAAAANDFLAAAA|75|Mill 4th|Court|Suite K|Oak Ridge|West Feliciana Parish|LA|78371|United States|-6|single family| +46398|AAAAAAAAODFLAAAA||6th |Pkwy||||AL|33394|||condo| +46399|AAAAAAAAPDFLAAAA|396|13th Locust|Blvd|Suite Q|Midway|Morrow County|OR|91904|United States|-8|single family| +46400|AAAAAAAAAEFLAAAA|108|Ash Railroad|Circle|Suite 440|Springfield|Mineral County|CO|89303|United States|-7|single family| +46401|AAAAAAAABEFLAAAA|316|Elm |RD|Suite 240|Clinton|Knox County|TX|78222|United States|-6|condo| +46402|AAAAAAAACEFLAAAA|598|West Lakeview|Drive|Suite 460|Woodland|Rowan County|NC|24854|United States|-5|condo| +46403|AAAAAAAADEFLAAAA|175|East |Circle|Suite 310|Lakeview|Amherst County|VA|28579|United States|-5|condo| +46404|AAAAAAAAEEFLAAAA|21|Cedar 6th|Road|Suite 70|Lakeview|Crawford County|GA|38579|United States|-5|single family| +46405|AAAAAAAAFEFLAAAA|377|Maple |Avenue|Suite 70|Riverview|Burnet County|TX|79003|United States|-6|condo| +46406|AAAAAAAAGEFLAAAA|62|Sunset |Parkway|Suite Y|Riverdale|Douglas County|WA|99391|United States|-8|single family| +46407|AAAAAAAAHEFLAAAA|668|Ninth |Cir.|Suite M|Lakeside|Riley County|KS|69532|United States|-6|single family| +46408|AAAAAAAAIEFLAAAA|423|Maple |ST|Suite 400|Greenwood|Staunton city|VA|28828|United States|-5|single family| +46409|AAAAAAAAJEFLAAAA|962|Ninth |Ave|Suite V|Farmington|Lawrence County|MS|59145|United States|-6|apartment| +46410|AAAAAAAAKEFLAAAA|911|2nd |Wy|Suite 300|Lebanon|Jessamine County|KY|42898|United States|-6|condo| +46411|AAAAAAAALEFLAAAA||Hill Chestnut||||Yalobusha County||55752||-6|condo| +46412|AAAAAAAAMEFLAAAA|720|Williams |Court|Suite 210|Oakland|Defiance County|OH|49843|United States|-5|apartment| +46413|AAAAAAAANEFLAAAA|383|Church |Boulevard|Suite 400|Unionville|Oxford County|ME|02311|United States|-5|condo| +46414|AAAAAAAAOEFLAAAA|92|Lake Lakeview|Lane|Suite 140|Clifton|Nicollet County|MN|58014|United States|-6|single family| +46415|AAAAAAAAPEFLAAAA|861|4th |RD|Suite 360|Fairview|Montgomery County|MO|65709|United States|-6|apartment| +46416|AAAAAAAAAFFLAAAA|475|Willow |Circle|Suite 410|Mountain View|Bartholomew County|IN|44466|United States|-5|apartment| +46417|AAAAAAAABFFLAAAA|216|Lincoln |Cir.|Suite T|Hamilton|Monroe County|OH|42808|United States|-5|apartment| +46418|AAAAAAAACFFLAAAA|550|11th |Ln|Suite 160|Florence|Saguache County|CO|83394|United States|-7|apartment| +46419|AAAAAAAADFFLAAAA|298|Adams |Court|Suite H|Pleasant Hill|Sanborn County|SD|53604|United States|-7|apartment| +46420|AAAAAAAAEFFLAAAA|763|Hickory Jackson|Circle|Suite K|Jamestown|Middlesex County|VA|26867|United States|-5|apartment| +46421|AAAAAAAAFFFLAAAA|386|Spring |Dr.|Suite I|Clifton|Oregon County|MO|68014|United States|-6|single family| +46422|AAAAAAAAGFFLAAAA|144|Center |Ave|Suite C|Maple Grove|Surry County|NC|28252|United States|-5|single family| +46423|AAAAAAAAHFFLAAAA|798|Ash |Way|Suite K|Spring Valley|Clay County|MN|56060|United States|-6|apartment| +46424|AAAAAAAAIFFLAAAA|545|Franklin |Ave|Suite H|Buena Vista|Union Parish|LA|75752|United States|-6|apartment| +46425|AAAAAAAAJFFLAAAA|441|Elm |Cir.|Suite 0|Belmont|Harvey County|KS|60191|United States|-6|condo| +46426|AAAAAAAAKFFLAAAA|749|Eigth |Drive|Suite X|Freeport|Fayette County|IN|41844|United States|-5|condo| +46427|AAAAAAAALFFLAAAA|96|3rd Smith|Drive|Suite 410|Five Forks|Appling County|GA|32293|United States|-5|condo| +46428|AAAAAAAAMFFLAAAA|759|6th North|Ave|Suite 160|Oak Hill|Oconee County|SC|27838|United States|-5|condo| +46429|AAAAAAAANFFLAAAA|544|Forest Railroad|Way|Suite Y|Unionville|Van Wert County|OH|41711|United States|-5|condo| +46430|AAAAAAAAOFFLAAAA|91|Locust |Ln|Suite T|Mount Olive|Bacon County|GA|38059|United States|-5|apartment| +46431|AAAAAAAAPFFLAAAA|113|Williams |Avenue|Suite M|Union|Hansford County|TX|78721|United States|-6|single family| +46432|AAAAAAAAAGFLAAAA|80|Valley South|Drive|Suite C|Lakewood|Vance County|NC|28877|United States|-5|single family| +46433|AAAAAAAABGFLAAAA|544|Lake |ST|Suite K|Harmony|Morrison County|MN|55804|United States|-6|single family| +46434|AAAAAAAACGFLAAAA|749|Cherry Thirteenth|Boulevard|Suite 30|White Oak|Cass County|IA|56668|United States|-6|condo| +46435|AAAAAAAADGFLAAAA|665|4th |Circle|Suite I|Newtown|Jones County|GA|31749|United States|-5|apartment| +46436|AAAAAAAAEGFLAAAA|595|Oak |Road|Suite E|Birmingham|Malheur County|OR|93372|United States|-8|single family| +46437|AAAAAAAAFGFLAAAA|662|River 3rd|Cir.|Suite U|Mountain View|Allamakee County|IA|54466|United States|-6|apartment| +46438|AAAAAAAAGGFLAAAA|590|Franklin 2nd|Circle|Suite J|Forest Hills|Hernando County|FL|39237|United States|-5|condo| +46439|AAAAAAAAHGFLAAAA|576|Fourth |Cir.|Suite 50|Riverdale|Ward County|ND|59391|United States|-6|condo| +46440|AAAAAAAAIGFLAAAA|496|Miller Maple|Pkwy|Suite 350|Valley View|Pemiscot County|MO|65124|United States|-6|single family| +46441|AAAAAAAAJGFLAAAA|829|Pine Jefferson|RD|Suite 340|Harmony|Gaines County|TX|75804|United States|-6|apartment| +46442|AAAAAAAAKGFLAAAA|413|1st Park|Ave|Suite 460|Oak Ridge|Charlevoix County|MI|48371|United States|-5|single family| +46443|AAAAAAAALGFLAAAA|895|Tenth Park|Road|Suite C|Greenville|Towns County|GA|31387|United States|-5|condo| +46444|AAAAAAAAMGFLAAAA|930|Third |ST|Suite 410|Concord|Boone County|KY|44107|United States|-6|apartment| +46445|AAAAAAAANGFLAAAA|969|Birch |Circle|Suite X|Woodlawn|Wayne County|TN|34098|United States|-6|single family| +46446|AAAAAAAAOGFLAAAA|854|Park 2nd|Way|Suite 260|Edgewood|Mitchell County|GA|30069|United States|-5|condo| +46447|AAAAAAAAPGFLAAAA|467|Poplar North|Cir.|Suite J|Deerfield|Hopewell city|VA|29840|United States|-5|condo| +46448|AAAAAAAAAHFLAAAA|188|Lincoln Jefferson|Boulevard|Suite 70|Oakland|Wilson County|KS|69843|United States|-6|condo| +46449|AAAAAAAABHFLAAAA|183|13th Dogwood|Blvd|Suite 80|Jamestown|Haines Borough|AK|96867|United States|-9|condo| +46450|AAAAAAAACHFLAAAA|15|Elm |Street|Suite 360|Valley View|Calhoun County|WV|25124|United States|-5|apartment| +46451|AAAAAAAADHFLAAAA|309|Cherry |Ln|Suite 210|Pierce|San Bernardino County|CA|93360|United States|-8|single family| +46452|AAAAAAAAEHFLAAAA|60|Center |Ct.|Suite V|Bunker Hill|Houston County|TX|70150|United States|-6|single family| +46453|AAAAAAAAFHFLAAAA|929|Oak |Cir.|Suite H|New Hope|Bourbon County|KS|69431|United States|-6|condo| +46454|AAAAAAAAGHFLAAAA|822|Hill |Pkwy|Suite 200|Roy|Chisago County|MN|50744|United States|-6|condo| +46455|AAAAAAAAHHFLAAAA|47||||Highland||TX|79454|United States||apartment| +46456|AAAAAAAAIHFLAAAA|15|Cedar Ash|Way|Suite T|Unionville|Glades County|FL|31711|United States|-5|single family| +46457|AAAAAAAAJHFLAAAA|542|Lakeview 5th|Avenue|Suite 170|Springfield|Cameron County|PA|19303|United States|-5|condo| +46458|AAAAAAAAKHFLAAAA|730|14th Green|Ln|Suite 160|Lebanon|Washington County|NY|12898|United States|-5|single family| +46459|AAAAAAAALHFLAAAA|709|Fourth |Ct.|Suite 100|Arlington|Lawrence County|TN|36557|United States|-6|apartment| +46460|AAAAAAAAMHFLAAAA|28|3rd |Boulevard|Suite 90|Mount Olive|Crockett County|TN|38059|United States|-5|condo| +46461|AAAAAAAANHFLAAAA|372|Laurel |Avenue|||Westmoreland County|||United States||| +46462|AAAAAAAAOHFLAAAA|351|12th |Parkway|Suite 370|Howell|Clinton County|NY|14854|United States|-5|apartment| +46463|AAAAAAAAPHFLAAAA|161|West |Court|Suite 400|Mount Zion|Union Parish|LA|78054|United States|-6|condo| +46464|AAAAAAAAAIFLAAAA|294|Highland 1st|ST|Suite 430|Lakeview|Stark County|IL|68579|United States|-6|single family| +46465|AAAAAAAABIFLAAAA|39|Cherry |Wy|Suite O|Lakeview|McHenry County|IL|68579|United States|-6|apartment| +46466|AAAAAAAACIFLAAAA|402|Park Main|Way|Suite B|Five Forks|Crisp County|GA|32293|United States|-5|single family| +46467|AAAAAAAADIFLAAAA|608|Hickory Pine|Way|Suite 430|Oak Grove|Greene County|AR|78370|United States|-6|condo| +46468|AAAAAAAAEIFLAAAA|724|6th Oak|Ct.|Suite J|Five Points|Dickenson County|VA|26098|United States|-5|single family| +46469|AAAAAAAAFIFLAAAA|508|Hickory |Road|Suite U|Clifton|Mecklenburg County|NC|28014|United States|-5|single family| +46470|AAAAAAAAGIFLAAAA|583|Madison North|Ln|Suite E|Woodlawn|Boone County|KY|44098|United States|-6|apartment| +46471|AAAAAAAAHIFLAAAA|120|Maple Hickory|Way|Suite 0|Unionville|Sunflower County|MS|51711|United States|-6|condo| +46472|AAAAAAAAIIFLAAAA|858|4th Elm|Drive|Suite 90|Woodville|Payne County|OK|74289|United States|-6|single family| +46473|AAAAAAAAJIFLAAAA|93|Meadow |Cir.|Suite 20|Lincoln|Boulder County|CO|81289|United States|-7|condo| +46474|AAAAAAAAKIFLAAAA|447|Sycamore |Pkwy|Suite H|Forest Hills|Jeff Davis County|GA|39237|United States|-5|single family| +46475|AAAAAAAALIFLAAAA|724|Maple |Court|Suite J|Harmony|Monroe County|AR|75804|United States|-6|condo| +46476|AAAAAAAAMIFLAAAA|118|Pine Center|Circle|Suite 90|Woodrow|Dawson County|MT|64273|United States|-7|apartment| +46477|AAAAAAAANIFLAAAA|803|4th |RD|Suite V|Greenfield|Lagrange County|IN|45038|United States|-5|condo| +46478|AAAAAAAAOIFLAAAA|839|Church |Cir.|Suite S|Green Acres|York County|VA|27683|United States|-5|condo| +46479|AAAAAAAAPIFLAAAA|723|Williams Seventh|Wy|Suite 140|Texas|Osage County|MO|63342|United States|-6|apartment| +46480|AAAAAAAAAJFLAAAA|996|Hickory Railroad|Cir.|Suite L|Lebanon|Tuscarawas County|OH|42898|United States|-5|single family| +46481|AAAAAAAABJFLAAAA|722|Forest Mill|Dr.|Suite W|Waterloo|White Pine County|NV|81675|United States|-8|single family| +46482|AAAAAAAACJFLAAAA|891|4th |Way|Suite 290|Jackson|Lake County|SD|59583|United States|-7|single family| +46483|AAAAAAAADJFLAAAA|432|West Wilson|Lane|Suite 180|Salem|Johnston County|OK|78048|United States|-6|condo| +46484|AAAAAAAAEJFLAAAA|319|Walnut 10th|Ave|Suite 60|Friendship|Juniata County|PA|14536|United States|-5|single family| +46485|AAAAAAAAFJFLAAAA|892|Chestnut |Dr.|Suite S|Bunker Hill|Marion County|GA|30150|United States|-5|apartment| +46486|AAAAAAAAGJFLAAAA|373|Johnson |Way|Suite C|Brownsville|McCook County|SD|59310|United States|-7|condo| +46487|AAAAAAAAHJFLAAAA|429|Sycamore |Blvd|Suite 90|Riverdale|Wayne County|MO|69391|United States|-6|single family| +46488|AAAAAAAAIJFLAAAA|395|Cherry |Ln|Suite Q|New Hope|Defiance County|OH|49431|United States|-5|condo| +46489|AAAAAAAAJJFLAAAA|445|Ridge |Ct.|Suite 300|Five Forks|Cooke County|TX|72293|United States|-6|single family| +46490|AAAAAAAAKJFLAAAA|972|1st Ridge|Dr.|Suite 250|Wright|Cattaraugus County|NY|12814|United States|-5|condo| +46491|AAAAAAAALJFLAAAA|||Pkwy||Liberty|Newton County|MS|||-6|apartment| +46492|AAAAAAAAMJFLAAAA|261|Smith |Court|Suite 460|Nottingham|Hamlin County|SD|54074|United States|-7|condo| +46493|AAAAAAAANJFLAAAA|797|Franklin |Parkway|Suite S|Argyle|Rich County|UT|88722|United States|-7|condo| +46494|AAAAAAAAOJFLAAAA|764|College |Blvd|Suite X|Oakdale|Fremont County|IA|59584|United States|-6|condo| +46495|AAAAAAAAPJFLAAAA|777|Third |Blvd|Suite 430|Highland Park|Hart County|KY|46534|United States|-6|condo| +46496|AAAAAAAAAKFLAAAA|26|3rd Elm|Circle|Suite K|Concord|La Salle County|TX|74107|United States|-6|single family| +46497|AAAAAAAABKFLAAAA|269|Ash |Blvd|Suite O|Warwick|Pickens County|GA|31398|United States|-5|condo| +46498|AAAAAAAACKFLAAAA|552|Wilson |Ct.|Suite F|Long Branch|Baker County|FL|37682|United States|-5|single family| +46499|AAAAAAAADKFLAAAA|24|Fifth |Dr.|Suite 100|Lincoln|Knox County|KY|41289|United States|-5|condo| +46500|AAAAAAAAEKFLAAAA|420|Forest Second|Road|Suite 110|Midway|Appomattox County|VA|21904|United States|-5|condo| +46501|AAAAAAAAFKFLAAAA|840|Mill 14th|Lane|Suite 100|Edgewood|Koochiching County|MN|50069|United States|-6|condo| +46502|AAAAAAAAGKFLAAAA|486|15th Main|Ave|Suite A|Providence|Jefferson County|KY|46614|United States|-6|single family| +46503|AAAAAAAAHKFLAAAA|402|View |Road|Suite D|Providence|Morgan County|CO|86614|United States|-7|single family| +46504|AAAAAAAAIKFLAAAA||Jefferson |||Summit|Marshall County||70499|||| +46505|AAAAAAAAJKFLAAAA|525|Spruce 1st|Avenue|Suite K|Jamestown|Monroe County|KY|46867|United States|-5|condo| +46506|AAAAAAAAKKFLAAAA|754|Poplar 14th|Road|Suite 60|Shady Grove|Wright County|IA|52812|United States|-6|single family| +46507|AAAAAAAALKFLAAAA|295|12th Lee|Dr.|Suite N|Lee|Le Flore County|OK|70408|United States|-6|condo| +46508|AAAAAAAAMKFLAAAA|330|Lakeview Adams|RD|Suite 150|Shiloh|Kauai County|HI|99275|United States|-10|single family| +46509|AAAAAAAANKFLAAAA|89|Woodland |Ave|Suite 310|Spring Hill|Skagit County|WA|96787|United States|-8|apartment| +46510|AAAAAAAAOKFLAAAA|572|View Miller|Blvd|Suite T|Greenville|Cass County|MO|61387|United States|-6|condo| +46511|AAAAAAAAPKFLAAAA|823|Laurel |Ave|Suite E|Providence|Delta County|MI|46614|United States|-5|apartment| +46512|AAAAAAAAALFLAAAA|567|Eigth |Drive|Suite 170|Arlington|Orleans Parish|LA|76557|United States|-6|condo| +46513|AAAAAAAABLFLAAAA|492|Hill |Street|Suite I|Greenville|Bath County|KY|41387|United States|-6|apartment| +46514|AAAAAAAACLFLAAAA|36|1st 13th|Way|Suite M|Edgewood|Columbus County|NC|20069|United States|-5|single family| +46515|AAAAAAAADLFLAAAA|732|1st |Ln|Suite 100|Maywood|Trigg County|KY|45681|United States|-5|single family| +46516|AAAAAAAAELFLAAAA|527|Hill Walnut|Circle|Suite 250|Valley View|Day County|SD|55124|United States|-6|single family| +46517|AAAAAAAAFLFLAAAA|707|10th Spring|Dr.|Suite E|Mount Zion|Allen County|KS|68054|United States|-6|single family| +46518|AAAAAAAAGLFLAAAA|302|Church Forest|Ln|Suite K|Bridgeport|Big Stone County|MN|55817|United States|-6|condo| +46519|AAAAAAAAHLFLAAAA|428|College |Circle|Suite 180|Georgetown|San Miguel County|NM|87057|United States|-7|apartment| +46520|AAAAAAAAILFLAAAA|569|First |RD|Suite 440|Farmington|Williamson County|TX|79145|United States|-6|condo| +46521|AAAAAAAAJLFLAAAA|937|6th |Drive|Suite V|Lincoln|Pointe Coupee Parish|LA|71289|United States|-6|condo| +46522|AAAAAAAAKLFLAAAA|380|View Hillcrest|Way|Suite 10|Greenville|Nelson County|VA|21387|United States|-5|apartment| +46523|AAAAAAAALLFLAAAA|725|Valley 6th|ST|Suite I|Lakeview|Cherokee County|AL|38579|United States|-6|condo| +46524|AAAAAAAAMLFLAAAA|263|Cedar Washington|Road|Suite 30|Harmony|Butler County|KY|45804|United States|-6|condo| +46525|AAAAAAAANLFLAAAA|100|15th |Ct.|Suite Q|Clinton|Fillmore County|NE|68222|United States|-6|condo| +46526|AAAAAAAAOLFLAAAA|638|Spruce |Wy|Suite 80|Philadelphia|Daggett County|UT|85591|United States|-7|condo| +46527|AAAAAAAAPLFLAAAA|185|Smith Lake|Way|Suite 150|Brookwood|Goliad County|TX|70965|United States|-6|apartment| +46528|AAAAAAAAAMFLAAAA|52|Pine |Lane||||||United States|-8|single family| +46529|AAAAAAAABMFLAAAA|739|Lakeview Spruce|Parkway|Suite R|Lakewood|Kleberg County|TX|78877|United States|-6|single family| +46530|AAAAAAAACMFLAAAA|830|Maple |Dr.|Suite 400|Oak Grove|Plaquemines Parish|LA|78370|United States|-6|single family| +46531|AAAAAAAADMFLAAAA|628|3rd Railroad|Drive|Suite K|Ashland|Woodson County|KS|64244|United States|-6|condo| +46532|AAAAAAAAEMFLAAAA|94|Thirteenth 5th|Blvd|Suite 260|Harmony|Natchitoches Parish|LA|75804|United States|-6|condo| +46533|AAAAAAAAFMFLAAAA|9|Railroad |Court|Suite L|Concord|Clay County|TN|34107|United States|-5|apartment| +46534|AAAAAAAAGMFLAAAA|309|Oak Ash|Court|Suite 330|Pleasant Valley|Fairfax County|VA|22477|United States|-5|apartment| +46535|AAAAAAAAHMFLAAAA|166|Oak 8th|Road|Suite 260|Forest Hills|Dolores County|CO|89237|United States|-7|single family| +46536|AAAAAAAAIMFLAAAA|120|Hill |Street|Suite 420|Five Forks|Seminole County|GA|32293|United States|-5|single family| +46537|AAAAAAAAJMFLAAAA|998|Pine |Blvd|Suite 340|Mountain View|Muskegon County|MI|44466|United States|-5|single family| +46538|AAAAAAAAKMFLAAAA|792|Walnut |Pkwy|Suite 430|Lakeside|Panola County|TX|79532|United States|-6|condo| +46539|AAAAAAAALMFLAAAA|472|Washington |Circle|Suite 280|Georgetown|Upshur County|WV|27057|United States|-5|single family| +46540|AAAAAAAAMMFLAAAA|163|View Sunset|RD|Suite 370|Hidden Valley|Nemaha County|KS|65521|United States|-6|condo| +46541|AAAAAAAANMFLAAAA|906|Maple 1st|Avenue|Suite 330|Farmington|New Castle County|DE|19145|United States|-5|apartment| +46542|AAAAAAAAOMFLAAAA|132|Park |Street|Suite Y|Pleasant Hill|Mariposa County|CA|93604|United States|-8|single family| +46543|AAAAAAAAPMFLAAAA|231|8th |Ct.|Suite 310|Pine Grove|Allen County|KY|44593|United States|-6|condo| +46544|AAAAAAAAANFLAAAA|602|Pine Lake|Street|Suite 440|Fairview|Skagit County|WA|95709|United States|-8|apartment| +46545|AAAAAAAABNFLAAAA|643|West 2nd|Avenue|Suite L|Mount Olive|Fayette County|IL|68059|United States|-6|condo| +46546|AAAAAAAACNFLAAAA|297|Walnut |Ave|Suite 490|Plainview|Pontotoc County|MS|53683|United States|-6|single family| +46547|AAAAAAAADNFLAAAA|544|Spring |Ct.|Suite 130|Oak Hill|Kalkaska County|MI|47838|United States|-5|condo| +46548|AAAAAAAAENFLAAAA|80|Madison Church|Pkwy|Suite 340|Riverside|Teller County|CO|89231|United States|-7|apartment| +46549|AAAAAAAAFNFLAAAA|903|Cherry |Way|Suite 90|Summit|Shawnee County|KS|60499|United States|-6|single family| +46550|AAAAAAAAGNFLAAAA|370|Maple |Avenue|Suite K|Sulphur Springs|Alachua County|FL|38354|United States|-5|apartment| +46551|AAAAAAAAHNFLAAAA|358|Third |Dr.|Suite 160|New Hope|Parke County|IN|49431|United States|-5|condo| +46552|AAAAAAAAINFLAAAA|933|Hickory |Parkway|Suite I|Red Hill|Mercer County|NJ|04938|United States|-5|apartment| +46553|AAAAAAAAJNFLAAAA|854|First Railroad|Way|Suite K|Harmon|Angelina County|TX|75623|United States|-6|apartment| +46554|AAAAAAAAKNFLAAAA|225|8th |Ln|Suite N|Buena Vista|Wayne County|MO|65752|United States|-6|condo| +46555|AAAAAAAALNFLAAAA|128|Locust Johnson|Street|Suite X|Greenville|Anchorage Borough|AK|91387|United States|-9|apartment| +46556|AAAAAAAAMNFLAAAA|710|Main |Road|Suite 10|Florence|Moody County|SD|53394|United States|-7|single family| +46557|AAAAAAAANNFLAAAA|881|5th Park|Boulevard|Suite 460|Lakeside|Vermilion Parish|LA|79532|United States|-6|single family| +46558|AAAAAAAAONFLAAAA|728|6th Elm|Avenue|Suite 410|Kingston|Middlesex County|NJ|05575|United States|-5|condo| +46559|AAAAAAAAPNFLAAAA|309|Oak Meadow|Circle|Suite 400|Lincoln|Caswell County|NC|21289|United States|-5|single family| +46560|AAAAAAAAAOFLAAAA|610|Eigth River|Lane|Suite W|Bunker Hill|Clarke County|GA|30150|United States|-5|apartment| +46561|AAAAAAAABOFLAAAA|936|Church |Way|Suite E|Walnut Grove|Baca County|CO|87752|United States|-7|apartment| +46562|AAAAAAAACOFLAAAA|631|Sixth |Boulevard|Suite 340|Lebanon|Noble County|IN|42898|United States|-5|apartment| +46563|AAAAAAAADOFLAAAA|639|10th |||Jamestown||IL||||apartment| +46564|AAAAAAAAEOFLAAAA|805|Hillcrest Laurel|Parkway|Suite 90|Hillcrest|Saratoga County|NY|13003|United States|-5|apartment| +46565|AAAAAAAAFOFLAAAA|355|Ridge Sunset|Boulevard|Suite 270|Forest Hills|Missaukee County|MI|49237|United States|-5|apartment| +46566|AAAAAAAAGOFLAAAA|751|8th Pine|Avenue|Suite 280|Highland Park|King County|WA|96534|United States|-8|single family| +46567|AAAAAAAAHOFLAAAA|56|Spring Hill|ST|Suite 160|Jamestown|Calhoun County|AL|36867|United States|-6|single family| +46568|AAAAAAAAIOFLAAAA|496|College Ridge|Lane|Suite Q|Spring Hill|Red Willow County|NE|66787|United States|-7|apartment| +46569|AAAAAAAAJOFLAAAA|5|Center |Cir.|Suite M|Maple Grove|Hood River County|OR|98252|United States|-8|condo| +46570|AAAAAAAAKOFLAAAA|245|Cedar 15th|Court|Suite R|Jackson|Denali Borough|AK|99583|United States|-9|condo| +46571|AAAAAAAALOFLAAAA|634|10th |Blvd|Suite P|Belmont|Harnett County|NC|20191|United States|-5|condo| +46572|AAAAAAAAMOFLAAAA|564|Jackson Valley|Ct.|Suite 170|Pleasant Grove|Piscataquis County|ME|04736|United States|-5|apartment| +46573|AAAAAAAANOFLAAAA|865|4th |Ave|Suite 120|Plainview|Santa Barbara County|CA|93683|United States|-8|apartment| +46574|AAAAAAAAOOFLAAAA|459|5th |Dr.|Suite 350|Liberty|Allegany County|NY|13451|United States|-5|condo| +46575|AAAAAAAAPOFLAAAA|560|14th 2nd|Road|Suite 10|Whitney|Wythe County|VA|28339|United States|-5|condo| +46576|AAAAAAAAAPFLAAAA|87|First Adams|Street||||MO||United States|-6|| +46577|AAAAAAAABPFLAAAA|686|View 5th|Drive|Suite C|Macedonia|Cross County|AR|71087|United States|-6|apartment| +46578|AAAAAAAACPFLAAAA|611|Willow |Ave|Suite 150|Youngstown|Mason County|KY|40001|United States|-5|condo| +46579|AAAAAAAADPFLAAAA|643|Hillcrest |Parkway|Suite 270|Enterprise|Pleasants County|WV|21757|United States|-5|single family| +46580|AAAAAAAAEPFLAAAA|566|10th |RD|Suite 80|Springfield|Chester County|PA|19303|United States|-5|single family| +46581|AAAAAAAAFPFLAAAA|252|Park Ridge|Drive|Suite B|Newport|Palo Pinto County|TX|71521|United States|-6|apartment| +46582|AAAAAAAAGPFLAAAA|284|Washington |Avenue|Suite 470|Hamilton|Pierce County|GA|32808|United States|-5|condo| +46583|AAAAAAAAHPFLAAAA|190|Mill North|Wy|Suite 490|Washington Heights|Bullock County|AL|38167|United States|-6|condo| +46584|AAAAAAAAIPFLAAAA|8|10th Hill|Court|Suite U|Franklin|Crosby County|TX|79101|United States|-6|single family| +46585|AAAAAAAAJPFLAAAA|574|Oak |Parkway|Suite L|Pleasant Valley|Reno County|KS|62477|United States|-6|apartment| +46586|AAAAAAAAKPFLAAAA|689|Seventh |Boulevard|Suite U|Clifton|Page County|IA|58014|United States|-6|condo| +46587|AAAAAAAALPFLAAAA|618|Williams |Blvd|Suite U|Riverside|Lincoln County|MN|59231|United States|-6|apartment| +46588|AAAAAAAAMPFLAAAA|795|6th Highland|Road|Suite N|Empire|Jefferson County|IL|64145|United States|-6|condo| +46589|AAAAAAAANPFLAAAA|178|5th 6th|Avenue|Suite M|Bethel|Clay County|AL|35281|United States|-6|apartment| +46590|AAAAAAAAOPFLAAAA|406|Jefferson Pine|Ct.|Suite 260|Deerfield|Louisa County|IA|59840|United States|-6|single family| +46591|AAAAAAAAPPFLAAAA|449|Lake Jefferson|ST|Suite N|Lincoln|White County|AR|71289|United States|-6|single family| +46592|AAAAAAAAAAGLAAAA|773|5th Sunset|Lane|Suite 320|Oakland|Polk County|NE|69843|United States|-7|single family| +46593|AAAAAAAABAGLAAAA|768|Ninth 1st|RD|Suite 230|Franklin|Martin County|MN|59101|United States|-6|condo| +46594|AAAAAAAACAGLAAAA|507|Forest 3rd|Circle|Suite 30|Red Oak|Sauk County|WI|55018|United States|-6|apartment| +46595|AAAAAAAADAGLAAAA|783|Railroad Willow|Circle|Suite I|Lakeside|Stafford County|KS|69532|United States|-6|apartment| +46596|AAAAAAAAEAGLAAAA|666|14th |Ave|Suite X|Riverview|Keya Paha County|NE|69003|United States|-7|condo| +46597|AAAAAAAAFAGLAAAA|15|View |Dr.|Suite 30|Macedonia|Burlington County|NJ|01687|United States|-5|apartment| +46598|AAAAAAAAGAGLAAAA|211|6th |Road|Suite 310|Sutton|Comanche County|OK|75413|United States|-6|apartment| +46599|AAAAAAAAHAGLAAAA||Center ||Suite 310|Oakland|Van Buren County||||-5|apartment| +46600|AAAAAAAAIAGLAAAA|151|Fifth Hillcrest|Boulevard|Suite L|Thompsonville|Franklin County|MS|59651|United States|-6|apartment| +46601|AAAAAAAAJAGLAAAA|755|Church |Ct.|Suite 190|Mount Pleasant|Pitkin County|CO|81933|United States|-7|condo| +46602|AAAAAAAAKAGLAAAA|877|Dogwood Maple|Wy|Suite F|Walnut Grove|Shannon County|MO|67752|United States|-6|apartment| +46603|AAAAAAAALAGLAAAA|721|Ridge |Road|Suite 160|Saratoga|Grand County|UT|82123|United States|-7|apartment| +46604|AAAAAAAAMAGLAAAA|767|Park 2nd|Way|Suite X|Bridgeport|Llano County|TX|75817|United States|-6|single family| +46605|AAAAAAAANAGLAAAA|876|Washington South|Ave|Suite 170|Crystal|Calhoun County|TX|75258|United States|-6|single family| +46606|AAAAAAAAOAGLAAAA|890|Green |Circle||Oakdale||MN|||-6|single family| +46607|AAAAAAAAPAGLAAAA|643|Church Broadway|Dr.|Suite 180|Shiloh|Burlington County|NJ|09875|United States|-5|apartment| +46608|AAAAAAAAABGLAAAA|456|Center |Court|Suite O|Greenwood|Logan County|WV|28828|United States|-5|apartment| +46609|AAAAAAAABBGLAAAA|477|6th Jefferson|Court|Suite 460|Wilson|Aitkin County|MN|56971|United States|-6|single family| +46610|AAAAAAAACBGLAAAA|310||Wy|Suite V||||39454|United States||condo| +46611|AAAAAAAADBGLAAAA|4|View Miller|Road|Suite V|Union|Polk County|MN|58721|United States|-6|condo| +46612|AAAAAAAAEBGLAAAA|618|North |Dr.|Suite R|Pleasant Grove|Augusta County|VA|24136|United States|-5|apartment| +46613|AAAAAAAAFBGLAAAA|904|Green Railroad|Lane|Suite 370|Marion|Duval County|TX|70399|United States|-6|condo| +46614|AAAAAAAAGBGLAAAA|237|Elm |Boulevard|Suite 490|Forest Hills|Hamilton County|IA|59237|United States|-6|condo| +46615|AAAAAAAAHBGLAAAA||||Suite X|Waterloo||AK|91675|United States||| +46616|AAAAAAAAIBGLAAAA|965|View |Parkway|Suite 300|Greenwood|Mercer County|OH|48828|United States|-5|apartment| +46617|AAAAAAAAJBGLAAAA|20|Oak Elm|Ct.|Suite T|Macedonia|Frio County|TX|71087|United States|-6|condo| +46618|AAAAAAAAKBGLAAAA|689|Main |Drive|Suite G|Midway|Caldwell County|MO|61904|United States|-6|single family| +46619|AAAAAAAALBGLAAAA|117|Birch North|Pkwy|Suite 60|Oak Ridge|Darke County|OH|48371|United States|-5|condo| +46620|AAAAAAAAMBGLAAAA|627|Madison |Street|Suite 270|Liberty|Franklin County|ID|83451|United States|-7|condo| +46621|AAAAAAAANBGLAAAA|297|Jackson Pine|Wy|Suite 130|Riverdale|San Bernardino County|CA|99391|United States|-8|apartment| +46622|AAAAAAAAOBGLAAAA|467|Washington View|Ct.|Suite 300|Florence|Lincoln County|WV|23394|United States|-5|single family| +46623|AAAAAAAAPBGLAAAA|986|Meadow East|Ln|Suite N|Woodland|Sargent County|ND|54854|United States|-6|condo| +46624|AAAAAAAAACGLAAAA|524|Railroad |Cir.|Suite T|Mount Zion|Chatham County|GA|38054|United States|-5|single family| +46625|AAAAAAAABCGLAAAA|787|Hillcrest |Way|Suite 310|Friendship|Trempealeau County|WI|54536|United States|-6|condo| +46626|AAAAAAAACCGLAAAA|338|12th |Ave|Suite 230|Fairview|Koochiching County|MN|55709|United States|-6|condo| +46627|AAAAAAAADCGLAAAA|605|Eigth |Circle|Suite A|Centerville|Putnam County|IL|60059|United States|-6|single family| +46628|AAAAAAAAECGLAAAA|854|Pine |Ct.|Suite 410|Friendship|White County|AR|74536|United States|-6|condo| +46629|AAAAAAAAFCGLAAAA|42|Woodland Lake|Ct.|Suite D|Plainview|Grant County|KY|43683|United States|-6|apartment| +46630|AAAAAAAAGCGLAAAA|440|13th Hill|Parkway|Suite 140|Farmington|Crow Wing County|MN|59145|United States|-6|condo| +46631|AAAAAAAAHCGLAAAA|708|8th Fourth|Wy|Suite 30|Bunker Hill|Nance County|NE|60150|United States|-7|single family| +46632|AAAAAAAAICGLAAAA|571|Poplar |Ct.|Suite T|Stringtown|Hale County|AL|30162|United States|-6|apartment| +46633|AAAAAAAAJCGLAAAA|862|11th Mill|Way|Suite 90|Jackson|Graham County|AZ|89583|United States|-7|single family| +46634|AAAAAAAAKCGLAAAA|897|Second |Court|Suite U|Sunnyside|Lafayette County|MS|51952|United States|-6|condo| +46635|AAAAAAAALCGLAAAA|699|7th |Lane|Suite 330|Guthrie|Shelby County|AL|31423|United States|-6|single family| +46636|AAAAAAAAMCGLAAAA|592|Johnson 2nd|Drive|Suite Q|Union Hill|Hancock County|ME|08346|United States|-5|condo| +46637|AAAAAAAANCGLAAAA|195|Pine Adams|Drive|Suite 20|Guthrie|Wirt County|WV|21423|United States|-5|condo| +46638|AAAAAAAAOCGLAAAA|756|Pine |Blvd|Suite 190|Mount Olive|Madison County|ID|88059|United States|-7|single family| +46639|AAAAAAAAPCGLAAAA|943|Oak |Ave|Suite Y|Walnut Grove|Watonwan County|MN|57752|United States|-6|single family| +46640|AAAAAAAAADGLAAAA|737|Third Maple|Court|Suite F|Glendale|Grant County|NE|63951|United States|-6|condo| +46641|AAAAAAAABDGLAAAA|8|4th |Lane|Suite 50|Unionville|Tate County|MS|51711|United States|-6|apartment| +46642|AAAAAAAACDGLAAAA|71|2nd |Ct.|Suite U|Lakeview|Shawano County|WI|58579|United States|-6|apartment| +46643|AAAAAAAADDGLAAAA|608|First |ST|Suite 80|Shiloh|Swift County|MN|59275|United States|-6|condo| +46644|AAAAAAAAEDGLAAAA|43|7th |Blvd|Suite N|Springfield|Bacon County|GA|39303|United States|-5|apartment| +46645|AAAAAAAAFDGLAAAA|302|Park Hill|Wy|Suite 460|Georgetown|Orange County|VA|27057|United States|-5|single family| +46646|AAAAAAAAGDGLAAAA|522|Adams |Cir.|Suite Y|Liberty|Portage County|OH|43451|United States|-5|condo| +46647|AAAAAAAAHDGLAAAA|894|8th Maple|Way|Suite 350|Hopewell|Platte County|MO|60587|United States|-6|apartment| +46648|AAAAAAAAIDGLAAAA|652|River 4th|Way|Suite D|Liberty|Jefferson County|PA|13451|United States|-5|apartment| +46649|AAAAAAAAJDGLAAAA|630|Hill |Ct.|Suite 360|Springfield|Musselshell County|MT|69303|United States|-7|condo| +46650|AAAAAAAAKDGLAAAA|342|3rd Lake|Drive|Suite 90|Unionville|Eureka County|NV|81711|United States|-8|condo| +46651|AAAAAAAALDGLAAAA|447|Main Cherry|RD|Suite X|Clinton|Boone County|NE|68222|United States|-6|apartment| +46652|AAAAAAAAMDGLAAAA|223|Fourth |Circle|Suite 170|Wilson|Stone County|MO|66971|United States|-6|single family| +46653|AAAAAAAANDGLAAAA|264|Church |Ave|Suite 420|Hillcrest|Hillsborough County|NH|03603|United States|-5|single family| +46654|AAAAAAAAODGLAAAA|141|7th |ST|Suite 70|Clifton|Brookings County|SD|58014|United States|-6|apartment| +46655|AAAAAAAAPDGLAAAA|192|West Meadow|Court|Suite 40|Clinton|Nevada County|CA|98222|United States|-8|single family| +46656|AAAAAAAAAEGLAAAA|374|Railroad 11th|Drive|Suite J|Georgetown|Baylor County|TX|77057|United States|-6|apartment| +46657|AAAAAAAABEGLAAAA|643|8th |Ave|Suite K|Hopewell|Custer County|OK|70587|United States|-6|single family| +46658|AAAAAAAACEGLAAAA|||||||IL|66557|United States||condo| +46659|AAAAAAAADEGLAAAA||Washington 6th|ST|||Matanuska-Susitna Borough||96534|United States|-9|| +46660|AAAAAAAAEEGLAAAA|190|14th |Avenue|Suite N|Stringtown|Johnston County|OK|70162|United States|-6|apartment| +46661|AAAAAAAAFEGLAAAA|746|Fourteenth Ash|Ave|Suite 450|Bridgeport|Green County|WI|55817|United States|-6|apartment| +46662|AAAAAAAAGEGLAAAA|596|Mill |RD|Suite R|Plainview|Knox County|TN|33683|United States|-6|apartment| +46663|AAAAAAAAHEGLAAAA|735|Oak 15th|Boulevard|Suite S|Shady Grove|Richmond city|VA|22812|United States|-5|single family| +46664|AAAAAAAAIEGLAAAA|357|Maple 5th|Lane|Suite K|Vance|Lackawanna County|PA|10268|United States|-5|condo| +46665|AAAAAAAAJEGLAAAA|949|Cedar |Drive|Suite G|Wildwood|Jones County|GA|36871|United States|-5|apartment| +46666|AAAAAAAAKEGLAAAA|609|10th |Drive|Suite D|Pine Grove|Fountain County|IN|44593|United States|-5|condo| +46667|AAAAAAAALEGLAAAA|216|Church |Blvd|Suite 490|Lakeview|Laurens County|SC|28579|United States|-5|condo| +46668|AAAAAAAAMEGLAAAA|238|Tenth |Pkwy|Suite 410|Weldon|Torrance County|NM|86277|United States|-7|single family| +46669|AAAAAAAANEGLAAAA|331|Poplar Broadway|Parkway|Suite K|Plainview|Kimble County|TX|73683|United States|-6|single family| +46670|AAAAAAAAOEGLAAAA|552|2nd |RD|Suite 140|Mount Olive|Latah County|ID|88059|United States|-7|apartment| +46671|AAAAAAAAPEGLAAAA|235|Main |Way|Suite J|Belmont|Elmore County|ID|80191|United States|-7|condo| +46672|AAAAAAAAAFGLAAAA|190|Mill |Avenue|Suite 90|Colfax|Juneau Borough|AK|92565|United States|-9|condo| +46673|AAAAAAAABFGLAAAA|740|Ninth |Road|Suite 390|Bunker Hill|Jefferson County|AR|70150|United States|-6|apartment| +46674|AAAAAAAACFGLAAAA|525|Hickory |Way|Suite 230|Oak Ridge|Gentry County|MO|68371|United States|-6|apartment| +46675|AAAAAAAADFGLAAAA|591|8th |Wy|Suite O|Yorktown|Otter Tail County|MN|50732|United States|-6|apartment| +46676|AAAAAAAAEFGLAAAA|454|Elm |Court|Suite 30|Newport|Dauphin County|PA|11521|United States|-5|single family| +46677|AAAAAAAAFFGLAAAA|||||Harvey||MN|55858||-6|| +46678|AAAAAAAAGFGLAAAA|1|4th Seventh|Way|Suite Q|Concord|Calhoun County|AL|34107|United States|-6|condo| +46679|AAAAAAAAHFGLAAAA|153|Spring 5th|Pkwy|Suite A|Oakland|Bergen County|NJ|09843|United States|-5|apartment| +46680|AAAAAAAAIFGLAAAA|409|Walnut 10th|Drive|Suite R|Bunker Hill|Webster County|NE|60150|United States|-7|single family| +46681|AAAAAAAAJFGLAAAA|861|Hill |Ln|Suite X|Friendship|Charleston County|SC|24536|United States|-5|condo| +46682|AAAAAAAAKFGLAAAA|745|2nd |Cir.|Suite V|Riverdale|Grant County|SD|59391|United States|-6|single family| +46683|AAAAAAAALFGLAAAA|613|Ninth |Pkwy|Suite W|Harmony|Wharton County|TX|75804|United States|-6|single family| +46684|AAAAAAAAMFGLAAAA|272|Jackson |Ln|Suite 60|Newtown|Fillmore County|NE|61749|United States|-6|condo| +46685|AAAAAAAANFGLAAAA|486|Pine 9th|Ln|Suite 360|Oakwood|Harrison County|KY|40169|United States|-6|apartment| +46686|AAAAAAAAOFGLAAAA|956|River Forest|Pkwy|Suite M|Bethel|Archuleta County|CO|85281|United States|-7|single family| +46687|AAAAAAAAPFGLAAAA|941|Ninth |Way|Suite V|Shiloh|Russell County|VA|29275|United States|-5|single family| +46688|AAAAAAAAAGGLAAAA|315|Elm Main|RD|Suite M|Springtown|Fleming County|KY|49858|United States|-6|apartment| +46689|AAAAAAAABGGLAAAA|739|||Suite 380|Greenfield||AK||United States||single family| +46690|AAAAAAAACGGLAAAA|975|Laurel |Wy|Suite S|Glenwood|Warren County|GA|33511|United States|-5|condo| +46691|AAAAAAAADGGLAAAA|336|Spruce |Boulevard|Suite R|Unionville|Tillamook County|OR|91711|United States|-8|condo| +46692|AAAAAAAAEGGLAAAA||Railroad Main||Suite 180||McMullen County|TX|78579|||| +46693|AAAAAAAAFGGLAAAA|486|5th View|Blvd|Suite 280|Centerville|Summers County|WV|20059|United States|-5|single family| +46694|AAAAAAAAGGGLAAAA|643|Pine |Boulevard|Suite W|Fairfield|Hardin County|TX|76192|United States|-6|single family| +46695|AAAAAAAAHGGLAAAA|926|4th |Avenue|Suite D|Jackson|Nueces County|TX|79583|United States|-6|single family| +46696|AAAAAAAAIGGLAAAA|757|Hickory |Wy|Suite 130|Hopewell|Kootenai County|ID|80587|United States|-7|single family| +46697|AAAAAAAAJGGLAAAA|132|Ash |Boulevard|Suite K|White Hall|Mercer County|WV|26955|United States|-5|single family| +46698|AAAAAAAAKGGLAAAA|110|Pine Pine|Blvd|Suite 20|Millbrook|Osceola County|IA|57529|United States|-6|apartment| +46699|AAAAAAAALGGLAAAA|881|Forest |Circle|Suite Y|Wilson|Deschutes County|OR|96971|United States|-8|apartment| +46700|AAAAAAAAMGGLAAAA|411|Main |Ln|Suite F|Riverview|Saginaw County|MI|49003|United States|-5|condo| +46701|AAAAAAAANGGLAAAA|323|Maple Hickory|Circle|Suite O|Newport|Forest County|WI|51521|United States|-6|single family| +46702|AAAAAAAAOGGLAAAA|40|Lakeview Third|Blvd|Suite F|New Hope|Hale County|AL|39431|United States|-6|apartment| +46703|AAAAAAAAPGGLAAAA|67|Lake |Ave|Suite 150|Arlington|District of Columbia|DC|26557|United States|-5|condo| +46704|AAAAAAAAAHGLAAAA|913|Main |Wy|Suite 290|Oak Hill|Van Buren County|AR|77838|United States|-6|single family| +46705|AAAAAAAABHGLAAAA|79|Park Fifth|Court|Suite V|Hamilton|Grant County|KS|62808|United States|-6|condo| +46706|AAAAAAAACHGLAAAA|305|Highland |Blvd|Suite W|Lakeside|Perry County|MO|69532|United States|-6|apartment| +46707|AAAAAAAADHGLAAAA|425|Broadway |Street|Suite B|Winslow|Pawnee County|OK|78525|United States|-6|single family| +46708|AAAAAAAAEHGLAAAA|194|Elm Railroad|Street|Suite N|Farmington|Hudson County|NJ|09745|United States|-5|single family| +46709|AAAAAAAAFHGLAAAA|45|Chestnut Mill|Wy|Suite H|Glendale|Fayette County|TX|73951|United States|-6|condo| +46710|AAAAAAAAGHGLAAAA|586||Circle|||Adams County|IN|41087|||| +46711|AAAAAAAAHHGLAAAA|112|Johnson 2nd|ST|Suite 400|Valley View|Henry County|VA|25124|United States|-5|single family| +46712|AAAAAAAAIHGLAAAA|139|Meadow |Wy|Suite P|Highland Park|Charlotte County|VA|26534|United States|-5|condo| +46713|AAAAAAAAJHGLAAAA|858|Forest |Cir.|Suite 180|California|Jefferson County|WV|20141|United States|-5|condo| +46714|AAAAAAAAKHGLAAAA|462|Jackson |Ln|Suite 330|Glendale|Washington County|GA|33951|United States|-5|condo| +46715|AAAAAAAALHGLAAAA|819|Pine |Way|Suite 350|Lakeview|Pike County|IN|48579|United States|-5|apartment| +46716|AAAAAAAAMHGLAAAA|660|Main Maple|Court|Suite Y|Bridgeport|Fisher County|TX|75817|United States|-6|single family| +46717|AAAAAAAANHGLAAAA|578|2nd View|Circle|Suite C|Springfield|Santa Cruz County|CA|99303|United States|-8|condo| +46718|AAAAAAAAOHGLAAAA|936|9th |Avenue|Suite L|Wildwood|Irion County|TX|76871|United States|-6|apartment| +46719|AAAAAAAAPHGLAAAA|480|Park |Blvd|Suite K|Harmony|Harford County|MD|25804|United States|-5|condo| +46720|AAAAAAAAAIGLAAAA|862|Chestnut |Court|Suite C|Fairview|Buchanan County|MO|65709|United States|-6|apartment| +46721|AAAAAAAABIGLAAAA|419|Lincoln Fourth|Circle|Suite F|Montpelier|Coal County|OK|78930|United States|-6|single family| +46722|AAAAAAAACIGLAAAA|986|9th View|RD|Suite L|Ashton|Tripp County|SD|59981|United States|-7|single family| +46723|AAAAAAAADIGLAAAA|523||Ct.|Suite Q||Berkeley County|SC||United States||condo| +46724|AAAAAAAAEIGLAAAA|451|Birch |Road|Suite G|Franklin|Hardin County|TN|39101|United States|-5|condo| +46725|AAAAAAAAFIGLAAAA|62|Cedar |Street|Suite 280|Mountain View|Richmond city|VA|24466|United States|-5|condo| +46726|AAAAAAAAGIGLAAAA|987|View Miller|Ct.|Suite A|Unionville|Sumner County|KS|61711|United States|-6|condo| +46727|AAAAAAAAHIGLAAAA|696|Washington |RD|Suite S|Greenwood|Fremont County|ID|88828|United States|-7|condo| +46728|AAAAAAAAIIGLAAAA|207|Johnson Adams|Drive|Suite 240|Oakland|Clay County|GA|39843|United States|-5|single family| +46729|AAAAAAAAJIGLAAAA|862|Center |Circle|Suite C|Sunnyside|Live Oak County|TX|71952|United States|-6|apartment| +46730|AAAAAAAAKIGLAAAA|651|Jackson |Drive|Suite I|Woodlawn|Meade County|SD|54098|United States|-7|apartment| +46731|AAAAAAAALIGLAAAA|817|Hillcrest Second|Lane|Suite E|Riverview|Kanabec County|MN|59003|United States|-6|condo| +46732|AAAAAAAAMIGLAAAA|454|9th |Ct.|Suite Q|Hamilton|Colorado County|TX|72808|United States|-6|single family| +46733|AAAAAAAANIGLAAAA|472|Jackson 4th|Blvd|Suite 140|Concord|Lassen County|CA|94107|United States|-8|apartment| +46734|AAAAAAAAOIGLAAAA||West Seventh|Way||Wilson|||16971||-5|| +46735|AAAAAAAAPIGLAAAA|674|First |Pkwy|Suite B|Florence|Hunterdon County|NJ|03994|United States|-5|condo| +46736|AAAAAAAAAJGLAAAA|243|Ridge |Dr.|Suite 180|Concord|Hanover County|VA|24107|United States|-5|condo| +46737|AAAAAAAABJGLAAAA|430|Elm |Ln|Suite 30|Cedar Grove|Morrill County|NE|60411|United States|-7|apartment| +46738|AAAAAAAACJGLAAAA|156|Sycamore Center|Circle|Suite O|Plainview|Morrison County|MN|53683|United States|-6|single family| +46739|AAAAAAAADJGLAAAA|246|Church |Wy|Suite 440|Highland|Door County|WI|59454|United States|-6|single family| +46740|AAAAAAAAEJGLAAAA|361|1st |Lane|Suite H|Lewisburg|Benton County|IA|57538|United States|-6|condo| +46741|AAAAAAAAFJGLAAAA|122|Center |Lane|Suite 260|Enterprise|Taylor County|KY|41757|United States|-5|apartment| +46742|AAAAAAAAGJGLAAAA||14th |Road|Suite 220|||||United States|-5|| +46743|AAAAAAAAHJGLAAAA|685|East |Boulevard|Suite 310|Hamilton|Christian County|IL|62808|United States|-6|apartment| +46744|AAAAAAAAIJGLAAAA|885|5th Sunset|Lane|Suite B|Arlington|Wayne County|WV|26557|United States|-5|single family| +46745|AAAAAAAAJJGLAAAA|640|Locust Hickory|Road|Suite J|Bridgeport|Blanco County|TX|75817|United States|-6|condo| +46746|AAAAAAAAKJGLAAAA|119|Chestnut Woodland|Parkway|Suite 220|Fairview|Allen County|OH|45709|United States|-5|single family| +46747|AAAAAAAALJGLAAAA|407|Birch Main||Suite 150|||CA||||| +46748|AAAAAAAAMJGLAAAA|592|13th North|Court|Suite 410|Star|Quay County|NM|80725|United States|-7|single family| +46749|AAAAAAAANJGLAAAA|967|Locust Broadway|Dr.|Suite A|Riverview|Lawrence County|MS|59003|United States|-6|single family| +46750|AAAAAAAAOJGLAAAA|643|Fifth |Cir.|Suite N|Five Points|Clermont County|OH|46098|United States|-5|condo| +46751|AAAAAAAAPJGLAAAA|829|Seventh |Wy|Suite 170|Cedar Grove|Montgomery County|PA|10411|United States|-5|condo| +46752|AAAAAAAAAKGLAAAA||||||Trinity County|||United States||condo| +46753|AAAAAAAABKGLAAAA|424|First Spring|Street|Suite V|Newport|Schuyler County|IL|61521|United States|-6|single family| +46754|AAAAAAAACKGLAAAA|334|Elevnth |RD|Suite 150|Spring Hill|Alger County|MI|46787|United States|-5|condo| +46755|AAAAAAAADKGLAAAA|3|Locust 8th|Court|Suite J|Riverdale|Gosper County|NE|69391|United States|-6|condo| +46756|AAAAAAAAEKGLAAAA|781|View River|Road|Suite C|Pinhook|Richland County|SC|29398|United States|-5|apartment| +46757|AAAAAAAAFKGLAAAA|729|Hillcrest |Dr.|Suite 10|Unionville|Monroe County|PA|11711|United States|-5|apartment| +46758|AAAAAAAAGKGLAAAA|841|Miller |Way|Suite 260|Sulphur Springs|McCracken County|KY|48354|United States|-5|condo| +46759|AAAAAAAAHKGLAAAA|511|1st West|Ave|Suite S|Richville|Iroquois County|IL|65945|United States|-6|condo| +46760|AAAAAAAAIKGLAAAA|531|East |Lane|Suite 260|Riverside|Carbon County|UT|89231|United States|-7|apartment| +46761|AAAAAAAAJKGLAAAA|445|Seventh |Cir.|Suite 70|Fairfield|Lewis and Clark County|MT|66192|United States|-7|apartment| +46762|AAAAAAAAKKGLAAAA|170|Main |Ct.|Suite 470|Springfield|Duchesne County|UT|89303|United States|-7|single family| +46763|AAAAAAAALKGLAAAA|926|College Laurel|Cir.|Suite J|Maple Grove|Lafayette County|AR|78252|United States|-6|condo| +46764|AAAAAAAAMKGLAAAA|682|14th |Ave|Suite 90|Red Hill|Delaware County|IA|54338|United States|-6|condo| +46765|AAAAAAAANKGLAAAA|863|Pine Green|Ave|Suite Y|Spring Valley|Polk County|IA|56060|United States|-6|apartment| +46766|AAAAAAAAOKGLAAAA|801|Walnut Pine|Ave|Suite 440|Bridgeport|Clay County|KS|65817|United States|-6|apartment| +46767|AAAAAAAAPKGLAAAA|178|Central Park|Wy|Suite 470|Centerville|Osceola County|IA|50059|United States|-6|single family| +46768|AAAAAAAAALGLAAAA|834|Lake Mill|Dr.|Suite 310|Bunker Hill|Madison County|ID|80150|United States|-7|condo| +46769|AAAAAAAABLGLAAAA|11|2nd 4th|Way|Suite 160|Enterprise|Boyd County|NE|61757|United States|-6|apartment| +46770|AAAAAAAACLGLAAAA|203|Broadway 5th|Street|Suite 230|Sleepy Hollow|Walker County|GA|33592|United States|-5|single family| +46771|AAAAAAAADLGLAAAA|462|Cherry |Cir.|Suite 30|Rolling Hills|Muskogee County|OK|77272|United States|-6|single family| +46772|AAAAAAAAELGLAAAA|307|Maple |Road|Suite 260|Spring Hill|Hampshire County|WV|26787|United States|-5|single family| +46773|AAAAAAAAFLGLAAAA||||Suite 340||Madison County||38579|United States||| +46774|AAAAAAAAGLGLAAAA|747|Maple Park|Road|Suite L|Friendship|Briscoe County|TX|74536|United States|-6|single family| +46775|AAAAAAAAHLGLAAAA|233|Willow Williams|Ct.|Suite Y|Centerville|Butler County|NE|60059|United States|-6|single family| +46776|AAAAAAAAILGLAAAA|282|9th Fifteenth|Avenue|Suite G|White Oak|Harrison County|TX|76668|United States|-6|single family| +46777|AAAAAAAAJLGLAAAA|414|Pine Oak|Court|Suite 190|Summit|Montgomery County|MD|20499|United States|-5|apartment| +46778|AAAAAAAAKLGLAAAA|714|South 3rd|Wy|Suite 70|Sulphur Springs|Wasatch County|UT|88354|United States|-7|condo| +46779|AAAAAAAALLGLAAAA|445|First |Road|Suite W|Glendale|Phelps County|MO|63951|United States|-6|condo| +46780|AAAAAAAAMLGLAAAA|813|Third |Court|Suite 480|Oak Ridge|Orangeburg County|SC|28371|United States|-5|single family| +46781|AAAAAAAANLGLAAAA|366|Green Fourth|Boulevard|Suite 80|Shiloh|Crockett County|TX|79275|United States|-6|single family| +46782|AAAAAAAAOLGLAAAA|614|Meadow |Ave|Suite M|||||||condo| +46783|AAAAAAAAPLGLAAAA|724|Spring Walnut|Avenue|Suite M|White Oak|Essex County|NY|16668|United States|-5|condo| +46784|AAAAAAAAAMGLAAAA|288|Main |Avenue|Suite 490|Jackson|Hyde County|SD|59583|United States|-7|apartment| +46785|AAAAAAAABMGLAAAA|297|3rd |Road|Suite 220|Lakeview|Stafford County|VA|28579|United States|-5|apartment| +46786|AAAAAAAACMGLAAAA|57|Ridge Oak|ST|Suite K|Five Forks|East Feliciana Parish|LA|72293|United States|-6|condo| +46787|AAAAAAAADMGLAAAA|91|10th |Parkway|Suite O|Clinton|Calhoun County|SC|28222|United States|-5|condo| +46788|AAAAAAAAEMGLAAAA|402|Hill |Dr.|Suite L|Salem|McDowell County|NC|28048|United States|-5|condo| +46789|AAAAAAAAFMGLAAAA|114|Church |Ct.|Suite O|Lincoln|Sequoyah County|OK|71289|United States|-6|apartment| +46790|AAAAAAAAGMGLAAAA|971|Walnut |ST|Suite O|Spring Hill|Tift County|GA|36787|United States|-5|single family| +46791|AAAAAAAAHMGLAAAA|403|Willow West|Court|Suite P|Leon|Lafayette County|AR|70913|United States|-6|condo| +46792|AAAAAAAAIMGLAAAA|237|7th |Parkway|Suite 200|Newport|Pima County|AZ|81521|United States|-7|single family| +46793|AAAAAAAAJMGLAAAA|863|Third 7th|Wy|Suite J|Fairview|Montrose County|CO|85709|United States|-7|single family| +46794|AAAAAAAAKMGLAAAA|295|Jackson |Court|Suite W|Green Acres|Racine County|WI|57683|United States|-6|apartment| +46795|AAAAAAAALMGLAAAA|903|1st Sunset|ST|Suite C|Sunnyside|Lancaster County|VA|21952|United States|-5|apartment| +46796|AAAAAAAAMMGLAAAA|464|Lake |||Five Points||SC||United States|-5|condo| +46797|AAAAAAAANMGLAAAA|749|Maple |Pkwy|Suite 280|Lakewood|Saginaw County|MI|48877|United States|-5|condo| +46798|AAAAAAAAOMGLAAAA|159|Locust Pine|Avenue|Suite 320|Belmont|Unicoi County|TN|30191|United States|-6|apartment| +46799|AAAAAAAAPMGLAAAA|335|Chestnut View|Wy|Suite 440|Williamsville|Cheyenne County|KS|68754|United States|-6|condo| +46800|AAAAAAAAANGLAAAA|218|Smith |Pkwy|Suite 190|Blanchard|Saratoga County|NY|15985|United States|-5|condo| +46801|AAAAAAAABNGLAAAA|562|Third |Lane|Suite 210|Tyrone|Macomb County|MI|41201|United States|-5|condo| +46802|AAAAAAAACNGLAAAA|711|4th |Wy|Suite U|Lakeview|Butler County|AL|38579|United States|-6|single family| +46803|AAAAAAAADNGLAAAA|116|Park |Cir.|Suite 350|Buckingham|Weston County|WY|84092|United States|-7|apartment| +46804|AAAAAAAAENGLAAAA|334|4th Ninth|Dr.|Suite 400|Plainview|Wallowa County|OR|93683|United States|-8|apartment| +46805|AAAAAAAAFNGLAAAA|345|Park Lake||Suite 410||Jefferson County||82898|United States||| +46806|AAAAAAAAGNGLAAAA|224|Railroad |Court|Suite 310|Georgetown|Crawford County|MO|67057|United States|-6|apartment| +46807|AAAAAAAAHNGLAAAA|619|First |Ave|Suite 50|Highland|Garfield County|OK|79454|United States|-6|single family| +46808|AAAAAAAAINGLAAAA|640|North Mill|Ln|Suite 370|Sulphur Springs|Butler County|NE|68354|United States|-6|condo| +46809|AAAAAAAAJNGLAAAA|920|11th |Way|Suite N|Macedonia|Florence County|SC|21087|United States|-5|single family| +46810|AAAAAAAAKNGLAAAA|529|Third Fourth|Parkway|Suite V|Bethesda|Orange County|TX|75980|United States|-6|condo| +46811|AAAAAAAALNGLAAAA|441|Hill |Circle|Suite U|Sunnyside|Washburn County|WI|51952|United States|-6|apartment| +46812|AAAAAAAAMNGLAAAA|668|Spring |Parkway|Suite 80|Salem|Edgar County|IL|68048|United States|-6|condo| +46813|AAAAAAAANNGLAAAA|176|Cherry 4th|Boulevard|Suite A|Woodlawn|Monroe County|WV|24098|United States|-5|apartment| +46814|AAAAAAAAONGLAAAA|394|Hickory 9th|Way|Suite Q|Lebanon|Clarke County|IA|52898|United States|-6|condo| +46815|AAAAAAAAPNGLAAAA|506|Mill Pine|Boulevard|Suite 130|Hillcrest|Keya Paha County|NE|63003|United States|-7|single family| +46816|AAAAAAAAAOGLAAAA||Pine Maple||||||28059|United States|-5|apartment| +46817|AAAAAAAABOGLAAAA|551|Cherry |Ct.|Suite 460|Woodville|Haskell County|OK|74289|United States|-6|condo| +46818|AAAAAAAACOGLAAAA|347|Meadow |Ln|Suite I|Edgewood|Arkansas County|AR|70069|United States|-6|single family| +46819|AAAAAAAADOGLAAAA|503|Dogwood Highland|RD|Suite C|Georgetown|Douglas County|OR|97057|United States|-8|condo| +46820|AAAAAAAAEOGLAAAA|208|8th |Circle|Suite 120|Salem|Huron County|MI|48048|United States|-5|single family| +46821|AAAAAAAAFOGLAAAA|188|Lake Highland|Blvd|Suite 100|Fayetteville|Logan County|IL|61732|United States|-6|condo| +46822|AAAAAAAAGOGLAAAA|877|10th Second|Blvd|Suite D|Bridgeport|Davidson County|TN|35817|United States|-5|condo| +46823|AAAAAAAAHOGLAAAA|730|Elm Hill|Circle|Suite 360|Waterloo|Beaver County|OK|71675|United States|-6|single family| +46824|AAAAAAAAIOGLAAAA|873|Fourth |Blvd|Suite 160|Hopewell|Martin County|MN|50587|United States|-6|condo| +46825|AAAAAAAAJOGLAAAA|302|Valley Church|Blvd|Suite 220|Hopewell|Person County|NC|20587|United States|-5|single family| +46826|AAAAAAAAKOGLAAAA|750|Hickory |Lane|Suite L|Macedonia|Lawrence County|AL|31087|United States|-6|single family| +46827|AAAAAAAALOGLAAAA|974|Lincoln |RD|Suite C|Riverdale|Dubois County|IN|49391|United States|-5|apartment| +46828|AAAAAAAAMOGLAAAA|212|Washington Woodland|Court|Suite 60|Spring Valley|Wasatch County|UT|86060|United States|-7|condo| +46829|AAAAAAAANOGLAAAA|942|Eigth Oak|Ave|Suite I|Macedonia|Amherst County|VA|21087|United States|-5|single family| +46830|AAAAAAAAOOGLAAAA|590|Fifth |RD|Suite X|Fairfield|Renville County|ND|56192|United States|-6|single family| +46831|AAAAAAAAPOGLAAAA|141|7th Elm|RD|Suite 200|Centerville|Berrien County|MI|40059|United States|-5|single family| +46832|AAAAAAAAAPGLAAAA|147|14th |Ct.|Suite 390|Greenwood|Essex County|VA|28828|United States|-5|apartment| +46833|AAAAAAAABPGLAAAA|332|Fifth 4th|Blvd|Suite 260|Red Hill|Jefferson County|GA|34338|United States|-5|condo| +46834|AAAAAAAACPGLAAAA|240|Third |Ave|Suite D|Oakwood|Cook County|GA|30169|United States|-5|single family| +46835|AAAAAAAADPGLAAAA|623|Mill |Way|Suite 210|Macedonia|Winchester city|VA|21087|United States|-5|single family| +46836|AAAAAAAAEPGLAAAA|822|Sunset Dogwood|ST|Suite T|Pine Grove|Hockley County|TX|74593|United States|-6|condo| +46837|AAAAAAAAFPGLAAAA|16|Spring |Way|Suite H|Oakland|Audrain County|MO|69843|United States|-6|single family| +46838|AAAAAAAAGPGLAAAA|934|4th |Way|Suite B|Lakeview|Gage County|NE|68579|United States|-6|single family| +46839|AAAAAAAAHPGLAAAA|274|Third Hickory|Lane|Suite P|Maple Grove|Trigg County|KY|48252|United States|-5|apartment| +46840|AAAAAAAAIPGLAAAA|404|Main |Dr.|Suite 480|Maple Grove|Garfield County|NE|68252|United States|-6|condo| +46841|AAAAAAAAJPGLAAAA|625|Lakeview Jackson|Ct.|Suite P|Ferguson|Frontier County|NE|61821|United States|-6|apartment| +46842|AAAAAAAAKPGLAAAA|932|Valley |RD|Suite G|Clinton|Conecuh County|AL|38222|United States|-6|condo| +46843|AAAAAAAALPGLAAAA|311|Birch |Parkway|Suite O|Jamestown|York County|NE|66867|United States|-6|condo| +46844|AAAAAAAAMPGLAAAA|442|11th Johnson|Ct.|Suite P|Florence|Oconee County|GA|33394|United States|-5|apartment| +46845|AAAAAAAANPGLAAAA|88|Valley Park|Drive|Suite 380|Union|Republic County|KS|68721|United States|-6|apartment| +46846|AAAAAAAAOPGLAAAA|153|Forest 4th|Court|Suite N|Hopewell|Cooke County|TX|70587|United States|-6|apartment| +46847|AAAAAAAAPPGLAAAA|992|Washington |Circle|Suite 240|Cedar Grove|Calloway County|KY|40411|United States|-6|condo| +46848|AAAAAAAAAAHLAAAA|204|Ridge Highland|Ct.|Suite J|Bunker Hill|Huerfano County|CO|80150|United States|-7|single family| +46849|AAAAAAAABAHLAAAA|203|Walnut |RD|Suite Q|Springfield|Snyder County|PA|19303|United States|-5|apartment| +46850|AAAAAAAACAHLAAAA|855|Jefferson 3rd|Parkway|Suite S|Fairfield|Pickett County|TN|36192|United States|-6|condo| +46851|AAAAAAAADAHLAAAA|635||||Spring Hill|Rockingham County|NC||United States||| +46852|AAAAAAAAEAHLAAAA|589|Dogwood 2nd|Street|Suite 160|Marion|Benton County|OR|90399|United States|-8|condo| +46853|AAAAAAAAFAHLAAAA|465|Hill Maple|Boulevard|Suite H|Union|Bryan County|GA|38721|United States|-5|single family| +46854|AAAAAAAAGAHLAAAA|616|Chestnut Birch|Ct.|Suite Q|Spring Hill|Shawano County|WI|56787|United States|-6|apartment| +46855|AAAAAAAAHAHLAAAA|254|Pine |Blvd|Suite 250|Siloam|Audrain County|MO|68948|United States|-6|single family| +46856|AAAAAAAAIAHLAAAA|277|East |RD|Suite 380|Friendship|Henry County|IN|44536|United States|-5|condo| +46857|AAAAAAAAJAHLAAAA|516|Broadway |Ave|Suite S|Greenwood|Sarasota County|FL|38828|United States|-5|apartment| +46858|AAAAAAAAKAHLAAAA|502|2nd Hickory|Circle|Suite 170|Watkins|Montour County|PA|11732|United States|-5|apartment| +46859|AAAAAAAALAHLAAAA|496|First 6th|Blvd|Suite J|Oak Hill|Bay County|FL|37838|United States|-5|condo| +46860|AAAAAAAAMAHLAAAA|492|1st Fourth|Dr.|Suite 340|Springdale|Kitsap County|WA|98883|United States|-8|condo| +46861|AAAAAAAANAHLAAAA|915|South |Wy|Suite 270|Pleasant Hill|Decatur County|GA|33604|United States|-5|apartment| +46862|AAAAAAAAOAHLAAAA|513|Wilson |Boulevard|Suite S|Stringtown|Kent County|MD|20162|United States|-5|apartment| +46863|AAAAAAAAPAHLAAAA|180|Pine Church|Parkway|Suite 450|Greenwood|Logan County|AR|78828|United States|-6|single family| +46864|AAAAAAAAABHLAAAA|305|Wilson Hickory|Blvd|Suite 370|Edgewood|East Carroll Parish|LA|70069|United States|-6|single family| +46865|AAAAAAAABBHLAAAA|221|Davis |Drive|Suite 110|Pine Grove|Falls County|TX|74593|United States|-6|condo| +46866|AAAAAAAACBHLAAAA|272|Sixth Highland|Lane|Suite 130|Springdale|Bolivar County|MS|58883|United States|-6|condo| +46867|AAAAAAAADBHLAAAA|915|Twelfth Fourth|Cir.|Suite M|Ashland|Lafayette County|WI|54244|United States|-6|single family| +46868|AAAAAAAAEBHLAAAA|207|1st Laurel|Ct.|Suite G|Centerville|Marion County|TN|30059|United States|-6|apartment| +46869|AAAAAAAAFBHLAAAA|889|14th |ST|Suite O|Pleasant Valley|Lincoln County|WY|82477|United States|-7|single family| +46870|AAAAAAAAGBHLAAAA|557|North |Dr.|Suite F|Greenwood|Marshall County|KY|48828|United States|-5|apartment| +46871|AAAAAAAAHBHLAAAA|447|Sixth Franklin|Blvd|Suite F|Woodbine|Honolulu County|HI|94253|United States|-10|condo| +46872|AAAAAAAAIBHLAAAA|||Cir.||Lincoln||MA||United States|-5|| +46873|AAAAAAAAJBHLAAAA|832|Mill |Cir.|Suite 260|Wesley|Fredericksburg city|VA|21218|United States|-5|apartment| +46874|AAAAAAAAKBHLAAAA|325|Miller 7th|Circle|Suite J|Summit|Butler County|AL|30499|United States|-6|apartment| +46875|AAAAAAAALBHLAAAA|64|Willow |RD|Suite 120|Jackson|Ward County|TX|79583|United States|-6|apartment| +46876|AAAAAAAAMBHLAAAA|165|7th |Ln|Suite O|Mount Pleasant|Valdez-Cordova Census Area|AK|91933|United States|-9|condo| +46877|AAAAAAAANBHLAAAA||Pine |||Riverdale|Lincoln County|CO|89391||-7|| +46878|AAAAAAAAOBHLAAAA|916|Third |RD|Suite 360|Newtown|Madison Parish|LA|71749|United States|-6|condo| +46879|AAAAAAAAPBHLAAAA|260|13th 1st|Court|Suite 190|Maple Grove|Mellette County|SD|58252|United States|-7|single family| +46880|AAAAAAAAACHLAAAA|678|Park |Pkwy|Suite R|Centerville|Franklin County|AR|70059|United States|-6|condo| +46881|AAAAAAAABCHLAAAA|144|Center |Ave|Suite 20|Glenwood|Lee County|AR|73511|United States|-6|condo| +46882|AAAAAAAACCHLAAAA|431|Birch |ST|Suite 180|Woodlawn|Ellis County|OK|74098|United States|-6|condo| +46883|AAAAAAAADCHLAAAA|69|15th Willow|Pkwy|Suite B|Clinton|Reynolds County|MO|68222|United States|-6|single family| +46884|AAAAAAAAECHLAAAA|8|Woodland |Avenue|Suite 60|Riverside|Beaverhead County|MT|69231|United States|-7|apartment| +46885|AAAAAAAAFCHLAAAA|277|Meadow |Circle|Suite Q|Little River|Yancey County|NC|20319|United States|-5|single family| +46886|AAAAAAAAGCHLAAAA|803|Washington View|RD|Suite 470|Five Points|Costilla County|CO|86098|United States|-7|single family| +46887|AAAAAAAAHCHLAAAA||||||Buffalo County|||United States||| +46888|AAAAAAAAICHLAAAA|474||Parkway||||WI||United States|-6|condo| +46889|AAAAAAAAJCHLAAAA|720|Mill 9th|Way|Suite 290|White Oak|Okfuskee County|OK|76668|United States|-6|condo| +46890|AAAAAAAAKCHLAAAA|480|First |Circle|Suite 260|Marion|Lamar County|TX|70399|United States|-6|condo| +46891|AAAAAAAALCHLAAAA|332|Lake |Ct.|Suite 330|Enterprise|Garfield County|CO|81757|United States|-7|single family| +46892|AAAAAAAAMCHLAAAA|940|Hickory Main|Wy|Suite 130|Arlington|Lorain County|OH|46557|United States|-5|condo| +46893|AAAAAAAANCHLAAAA|897|College Valley|Road|Suite 130|Crossroads|Alameda County|CA|90534|United States|-8|condo| +46894|AAAAAAAAOCHLAAAA|394|Forest 1st|ST|Suite W|Hillcrest|Burke County|NC|23003|United States|-5|single family| +46895|AAAAAAAAPCHLAAAA|645|Jackson |Boulevard|Suite X|Riverview|Chase County|NE|69003|United States|-6|single family| +46896|AAAAAAAAADHLAAAA|673|Smith Chestnut|Ct.|Suite A|Oakwood|Churchill County|NV|80169|United States|-8|single family| +46897|AAAAAAAABDHLAAAA|413|Center |Avenue|Suite U|Forest Hills|Schuylkill County|PA|19237|United States|-5|apartment| +46898|AAAAAAAACDHLAAAA||4th 12th||Suite 170||Pepin County||||-6|| +46899|AAAAAAAADDHLAAAA|243|Highland Woodland|Circle|Suite 150|Glenwood|Gooding County|ID|83511|United States|-7|condo| +46900|AAAAAAAAEDHLAAAA|887|Green |Court|Suite 110|Wildwood|Macomb County|MI|46871|United States|-5|condo| +46901|AAAAAAAAFDHLAAAA|994|Sixth |Avenue|Suite J|Sunnyside|Brown County|MN|51952|United States|-6|apartment| +46902|AAAAAAAAGDHLAAAA|496|Meadow 1st|Ave|Suite N||Beaufort County|||||single family| +46903|AAAAAAAAHDHLAAAA|610|Ridge Ash|Parkway|Suite E|Bridgeport|Harris County|TX|75817|United States|-6|apartment| +46904|AAAAAAAAIDHLAAAA|94|Birch |Wy|Suite 120|Farmington|Cabarrus County|NC|29145|United States|-5|apartment| +46905|AAAAAAAAJDHLAAAA|252|9th |Dr.|Suite 10|Lakewood|Santa Fe County|NM|88877|United States|-7|single family| +46906|AAAAAAAAKDHLAAAA|210|Willow |Ct.|Suite V|Green Acres|Tippah County|MS|57683|United States|-6|single family| +46907|AAAAAAAALDHLAAAA|295|14th 2nd|Way|Suite L|Shore Acres|Lipscomb County|TX|72724|United States|-6|apartment| +46908|AAAAAAAAMDHLAAAA|501|Dogwood Tenth|Lane|Suite N|Marion|Blair County|PA|10399|United States|-5|apartment| +46909|AAAAAAAANDHLAAAA|56|Poplar 5th|Drive|Suite R|Mount Zion|Shiawassee County|MI|48054|United States|-5|single family| +46910|AAAAAAAAODHLAAAA|331|3rd |Avenue|Suite C|Green Acres|Sumner County|TN|37683|United States|-6|single family| +46911|AAAAAAAAPDHLAAAA|827|Elm |Ave|Suite P|Mount Vernon|Cedar County|NE|68482|United States|-6|condo| +46912|AAAAAAAAAEHLAAAA|114|4th |Lane|Suite Y|Green Acres|Weld County|CO|87683|United States|-7|condo| +46913|AAAAAAAABEHLAAAA|910|Main |Wy|Suite 480|||||||| +46914|AAAAAAAACEHLAAAA|982|1st 3rd|Street|Suite 130|Edgewater|Cabell County|WV|20635|United States|-5|apartment| +46915|AAAAAAAADEHLAAAA|842|Franklin Wilson|Street|Suite 220|Clinton|Juneau Borough|AK|98222|United States|-9|apartment| +46916|AAAAAAAAEEHLAAAA|721||||Lebanon|Wayne County|MO|62898|United States||single family| +46917|AAAAAAAAFEHLAAAA|209|Willow |Ct.|Suite B|Evansville|Starr County|TX|75274|United States|-6|single family| +46918|AAAAAAAAGEHLAAAA|556|West |Parkway|Suite L|Phoenix|Fulton County|IL|62276|United States|-6|single family| +46919|AAAAAAAAHEHLAAAA|724|Hickory |Ln|Suite B|Friendship|Forsyth County|NC|24536|United States|-5|condo| +46920|AAAAAAAAIEHLAAAA|777|Meadow |Lane|Suite T|Pleasant Valley|Marion County|TN|32477|United States|-6|apartment| +46921|AAAAAAAAJEHLAAAA|641|Cedar Wilson|Road|Suite 410|Crossroads|Paulding County|OH|40534|United States|-5|single family| +46922|AAAAAAAAKEHLAAAA|508|4th |Lane|Suite L|Mountain View|Belknap County|NH|05066|United States|-5|single family| +46923|AAAAAAAALEHLAAAA|703|||Suite 120||New Madrid County|||||apartment| +46924|AAAAAAAAMEHLAAAA|331|Sixth |Wy|Suite R|Greenwood|Gosper County|NE|68828|United States|-6|condo| +46925|AAAAAAAANEHLAAAA|708|Eigth Sixth|Avenue|Suite S|Church Hill|Dodge County|NE|63790|United States|-6|single family| +46926|AAAAAAAAOEHLAAAA|207|Main |Cir.|Suite 270|Jackson|Franklin city|VA|29583|United States|-5|apartment| +46927|AAAAAAAAPEHLAAAA|789|Highland |RD|Suite P|Oak Hill|Fisher County|TX|77838|United States|-6|apartment| +46928|AAAAAAAAAFHLAAAA|131|13th |Blvd|Suite 70|New Hope|Washington County|VA|29431|United States|-5|condo| +46929|AAAAAAAABFHLAAAA|852|Jackson Chestnut|Street|Suite 490|Farmington|Jackson County|KS|69145|United States|-6|apartment| +46930|AAAAAAAACFHLAAAA|902|Second South|Circle|Suite C|Oak Hill|Prentiss County|MS|57838|United States|-6|condo| +46931|AAAAAAAADFHLAAAA|10|Washington |Way|Suite D|Pleasant Grove|Barbour County|AL|34136|United States|-6|single family| +46932|AAAAAAAAEFHLAAAA|523|Hill |Wy|Suite D|Hartland|Clay County|IL|66594|United States|-6|single family| +46933|AAAAAAAAFFHLAAAA|492|3rd |Avenue|Suite P|Pleasant Valley|Dodge County|WI|52477|United States|-6|single family| +46934|AAAAAAAAGFHLAAAA|409|Valley |Lane|Suite 270|Newport|Madison County|MO|61521|United States|-6|condo| +46935|AAAAAAAAHFHLAAAA|478|Cherry 5th|Parkway|Suite 140|California|Clark County|ID|80141|United States|-7|apartment| +46936|AAAAAAAAIFHLAAAA|945|First |Boulevard|Suite 190|Sanford|Nantucket County|MA|09737|United States|-5|single family| +46937|AAAAAAAAJFHLAAAA|234|Ridge |Drive|Suite 230|Littleton|Fountain County|IN|46074|United States|-5|condo| +46938|AAAAAAAAKFHLAAAA|524|River 1st|Road|Suite 300|Cedar Grove|Fremont County|IA|50411|United States|-6|apartment| +46939|AAAAAAAALFHLAAAA|236|Chestnut View|Dr.|Suite V|Lakewood|Sabine Parish|LA|78877|United States|-6|condo| +46940|AAAAAAAAMFHLAAAA|789|2nd |Ave|Suite Q|Brownsville|Mingo County|WV|29310|United States|-5|single family| +46941|AAAAAAAANFHLAAAA|734|Sycamore |Dr.|Suite 350|Greenwood|Conway County|AR|78828|United States|-6|condo| +46942|AAAAAAAAOFHLAAAA|287|Hill |Avenue|Suite 400|Summit|Jasper County|GA|30499|United States|-5|condo| +46943|AAAAAAAAPFHLAAAA|951|Sunset |ST|Suite V|Belmont|Presidio County|TX|70191|United States|-6|condo| +46944|AAAAAAAAAGHLAAAA|452|3rd |Avenue|Suite 140|Kingston|Golden Valley County|MT|64975|United States|-7|apartment| +46945|AAAAAAAABGHLAAAA|102|Woodland Park|Boulevard|Suite B|Silver Springs|Mineral County|MT|64843|United States|-7|single family| +46946|AAAAAAAACGHLAAAA|576|South Main|Parkway|Suite 50|Red Hill|Jackson Parish|LA|74338|United States|-6|apartment| +46947|AAAAAAAADGHLAAAA|160|Lake |ST|Suite 410|Highland Park|Middlesex County|NJ|07134|United States|-5|single family| +46948|AAAAAAAAEGHLAAAA|285|Third 14th|ST|Suite 490|Valley View|Pearl River County|MS|55124|United States|-6|condo| +46949|AAAAAAAAFGHLAAAA|231|Park Hickory|Dr.|Suite K|Glenwood|McLean County|ND|53511|United States|-6|condo| +46950|AAAAAAAAGGHLAAAA||View View||Suite 350||Burnett County||51075|United States|-6|| +46951|AAAAAAAAHGHLAAAA|186|Second |Avenue|Suite W|Fairfield|Warren County|GA|36192|United States|-5|condo| +46952|AAAAAAAAIGHLAAAA|174|East |RD|Suite 270|Summit|Fayette County|WV|20499|United States|-5|apartment| +46953|AAAAAAAAJGHLAAAA|581|Elm |Dr.|Suite P|Clifton|Clay County|NC|28014|United States|-5|apartment| +46954|AAAAAAAAKGHLAAAA|897|Washington |ST|Suite 70|Plainview|Parmer County|TX|73683|United States|-6|single family| +46955|AAAAAAAALGHLAAAA|749|Cedar Dogwood|Ct.|Suite 80|Glendale|Haskell County|KS|63951|United States|-6|condo| +46956|AAAAAAAAMGHLAAAA|768|Fourteenth Washington|ST|Suite S|Pleasant Valley|Bay County|FL|32477|United States|-5|condo| +46957|AAAAAAAANGHLAAAA|272|Sycamore Fifth|Ct.|Suite 440|Providence|Marion County|AL|36614|United States|-6|single family| +46958|AAAAAAAAOGHLAAAA|274|Main Washington|Parkway|Suite 300|Harmony|Sevier County|TN|35804|United States|-6|condo| +46959|AAAAAAAAPGHLAAAA|144|Spruce |Boulevard|Suite P|Bayside|Carver County|MN|59550|United States|-6|apartment| +46960|AAAAAAAAAHHLAAAA|363|College |Road|Suite P|Mount Zion|Aroostook County|ME|08654|United States|-5|condo| +46961|AAAAAAAABHHLAAAA|608|7th |Court|Suite 0|Forest Hills|Red River Parish|LA|79237|United States|-6|single family| +46962|AAAAAAAACHHLAAAA|515|Dogwood |Pkwy|Suite 390|Stringtown|Greenwood County|KS|60162|United States|-6|apartment| +46963|AAAAAAAADHHLAAAA|832|10th 1st|RD|Suite 430|Wilson|Carter County|OK|76971|United States|-6|apartment| +46964|AAAAAAAAEHHLAAAA|536|River Lake|RD|Suite H|Bayside|Kootenai County|ID|89550|United States|-7|apartment| +46965|AAAAAAAAFHHLAAAA|500|Lincoln Sunset|ST|Suite W|Hillcrest|Mason County|WV|23003|United States|-5|apartment| +46966|AAAAAAAAGHHLAAAA|247|4th Woodland|Blvd|Suite 420|Mount Olive|Cavalier County|ND|58059|United States|-6|condo| +46967|AAAAAAAAHHHLAAAA|729|1st 6th|Road|Suite E|Union Hill|Assumption Parish|LA|77746|United States|-6|single family| +46968|AAAAAAAAIHHLAAAA|660|9th 15th|Street|Suite 270|Deerfield|Bowman County|ND|59840|United States|-6|single family| +46969|AAAAAAAAJHHLAAAA|317|Cherry |Cir.|Suite 480|Union|Carroll County|KY|48721|United States|-6|condo| +46970|AAAAAAAAKHHLAAAA|602|Center Second|Cir.|Suite G|Midway|Montgomery County|PA|11904|United States|-5|condo| +46971|AAAAAAAALHHLAAAA|899|Cherry |Drive|Suite A|Lakewood|Scott County|VA|28877|United States|-5|condo| +46972|AAAAAAAAMHHLAAAA|345|10th Washington|Street|Suite 400|Mount Pleasant|Wood County|OH|41933|United States|-5|apartment| +46973|AAAAAAAANHHLAAAA|260|View Fourth|Lane|Suite C|Oakdale|Cochise County|AZ|89584|United States|-7|condo| +46974|AAAAAAAAOHHLAAAA|||Avenue|Suite C|Cedar Grove|Calhoun County|IL||United States||| +46975|AAAAAAAAPHHLAAAA|452|Forest Central|Drive|Suite D|California|Clarke County|VA|20141|United States|-5|apartment| +46976|AAAAAAAAAIHLAAAA|791|Ash Meadow|Ave|Suite 90|Oakwood|Mono County|CA|90169|United States|-8|apartment| +46977|AAAAAAAABIHLAAAA|406|Jefferson Laurel|Pkwy|Suite K|Star|Pope County|IL|60725|United States|-6|condo| +46978|AAAAAAAACIHLAAAA|33|Main Oak|Ln|Suite J|Lincoln|Grand Traverse County|MI|41289|United States|-5|single family| +46979|AAAAAAAADIHLAAAA|730|3rd |Parkway|Suite G|Bethel|Coke County|TX|75281|United States|-6|apartment| +46980|AAAAAAAAEIHLAAAA|489|Williams Woodland|Wy|Suite P|Shady Grove|Kern County|CA|92812|United States|-8|condo| +46981|AAAAAAAAFIHLAAAA|387|Park |Drive|Suite E|Five Points|Valencia County|NM|86098|United States|-7|condo| +46982|AAAAAAAAGIHLAAAA|502|10th |Court|Suite J|Five Forks|Brazos County|TX|72293|United States|-6|condo| +46983|AAAAAAAAHIHLAAAA|522|Jackson 2nd|Street|Suite 150|Ashland|Taylor County|WV|24244|United States|-5|condo| +46984|AAAAAAAAIIHLAAAA|492|Lincoln |Court|Suite A|Hamilton|Wexford County|MI|42808|United States|-5|single family| +46985|AAAAAAAAJIHLAAAA|528|4th 10th|Boulevard|Suite P|Forest Hills|Campbell County|WY|89237|United States|-7|condo| +46986|AAAAAAAAKIHLAAAA|322|College Birch|Circle|Suite P|Sulphur Springs|Jackson County|SD|58354|United States|-7|condo| +46987|AAAAAAAALIHLAAAA|669|8th 1st|Pkwy|Suite 370|Brentwood|Garrard County|KY|44188|United States|-6|condo| +46988|AAAAAAAAMIHLAAAA|490|Forest |ST|Suite H|Walnut Grove|Providence County|RI|08352|United States|-5|apartment| +46989|AAAAAAAANIHLAAAA|53|Park North|ST|Suite Y|Greenwood|Douglas County|IL|68828|United States|-6|single family| +46990|AAAAAAAAOIHLAAAA|251|Forest |ST|Suite K|Deerfield|Clay County|SD|59840|United States|-6|single family| +46991|AAAAAAAAPIHLAAAA|98|7th |Boulevard|Suite 450|Salem|Hinds County|MS|58048|United States|-6|apartment| +46992|AAAAAAAAAJHLAAAA|476|Smith |Ln|Suite 100|Macedonia|Sawyer County|WI|51087|United States|-6|condo| +46993|AAAAAAAABJHLAAAA|568|Broadway |Court|Suite T|Woodlawn|Kittson County|MN|54098|United States|-6|condo| +46994|AAAAAAAACJHLAAAA|92|Birch |Parkway|Suite X|Mountain View|Marion County|AL|34466|United States|-6|single family| +46995|AAAAAAAADJHLAAAA|171|Jackson Jackson|Ct.|Suite 20|Union Hill|Onslow County|NC|27746|United States|-5|single family| +46996|AAAAAAAAEJHLAAAA|671|Lake Johnson|Ct.|Suite R|Bethel|Dyer County|TN|35281|United States|-5|condo| +46997|AAAAAAAAFJHLAAAA|654|12th 2nd|Road|Suite 10|Millbrook|Searcy County|AR|77529|United States|-6|single family| +46998|AAAAAAAAGJHLAAAA|379|Cedar |Lane|Suite 250|White Oak|Itawamba County|MS|56668|United States|-6|condo| +46999|AAAAAAAAHJHLAAAA|981|Fourth |Ave|Suite 40|Bethel|Columbia County|NY|15281|United States|-5|apartment| +47000|AAAAAAAAIJHLAAAA|716|River 7th|ST|Suite K|Arlington|Callahan County|TX|76557|United States|-6|condo| +47001|AAAAAAAAJJHLAAAA|594|Wilson Jefferson|Boulevard|Suite 360|Mount Pleasant|Franklin County|ME|02533|United States|-5|condo| +47002|AAAAAAAAKJHLAAAA|344|North |Parkway|Suite 300|Enterprise|Clay County|MS|51757|United States|-6|single family| +47003|AAAAAAAALJHLAAAA|441|Davis Park|Ave|Suite U|Midway|Lyon County|NV|81904|United States|-8|condo| +47004|AAAAAAAAMJHLAAAA|470|12th |Ave|Suite 230|Brunswick|Macon County|MO|64642|United States|-6|single family| +47005|AAAAAAAANJHLAAAA|405|13th Franklin|Dr.|Suite D|Glenwood|Alpena County|MI|43511|United States|-5|apartment| +47006|AAAAAAAAOJHLAAAA|725|Hill Center|Blvd|Suite N|Pleasant Grove|Washington County|OK|74136|United States|-6|apartment| +47007|AAAAAAAAPJHLAAAA|60|Smith Park|Blvd|Suite 150|Plainview|Titus County|TX|73683|United States|-6|apartment| +47008|AAAAAAAAAKHLAAAA|360|2nd First|Court|Suite 310|Woodville|Pennington County|SD|54289|United States|-7|condo| +47009|AAAAAAAABKHLAAAA|972|2nd Third|Road|Suite 120|Crossroads|Jackson County|GA|30534|United States|-5|apartment| +47010|AAAAAAAACKHLAAAA|129|Lake |Avenue|Suite 360|Union Hill|Marshall County|WV|27746|United States|-5|condo| +47011|AAAAAAAADKHLAAAA|126|Seventh Maple|Circle|Suite 60|Midway|Republic County|KS|61904|United States|-6|single family| +47012|AAAAAAAAEKHLAAAA|785|Seventh |Way|Suite 330|Highland Park|Garfield County|WA|96534|United States|-8|apartment| +47013|AAAAAAAAFKHLAAAA|698|Oak Franklin|Cir.|Suite 70|Enterprise|Scotland County|MO|61757|United States|-6|single family| +47014|AAAAAAAAGKHLAAAA|652|South Green|Drive|Suite P|Spring Valley|Bergen County|NJ|06660|United States|-5|single family| +47015|AAAAAAAAHKHLAAAA|118||ST|Suite T|Franklin|||39101||-5|| +47016|AAAAAAAAIKHLAAAA|53|10th |Avenue|Suite 30|Shiloh|Carson County|TX|79275|United States|-6|single family| +47017|AAAAAAAAJKHLAAAA|320|Sunset |Court|Suite 180|Shore Acres|Atchison County|KS|62724|United States|-6|single family| +47018|AAAAAAAAKKHLAAAA|662|8th Franklin|Circle|Suite 90|Bunker Hill|Fulton County|NY|10150|United States|-5|apartment| +47019|AAAAAAAALKHLAAAA|949|Adams |Road|Suite C|Newport|Appanoose County|IA|51521|United States|-6|condo| +47020|AAAAAAAAMKHLAAAA|917|Hickory |Lane|Suite F|Valley View|Morgan County|KY|45124|United States|-5|single family| +47021|AAAAAAAANKHLAAAA|644|Lake |Road|Suite 300|Forest Hills|Hood County|TX|79237|United States|-6|apartment| +47022|AAAAAAAAOKHLAAAA|498|Railroad |Boulevard|Suite 330|Sulphur Springs|York County|NE|68354|United States|-6|single family| +47023|AAAAAAAAPKHLAAAA|651|Pine |Blvd|Suite 40|Enterprise|Refugio County|TX|71757|United States|-6|single family| +47024|AAAAAAAAALHLAAAA|279|4th 2nd|Parkway|Suite X|Pine Grove|Baker County|OR|94593|United States|-8|condo| +47025|AAAAAAAABLHLAAAA|275|Elevnth |Cir.|Suite C|White Oak|Carlton County|MN|56668|United States|-6|single family| +47026|AAAAAAAACLHLAAAA|162|Lincoln |Ct.|Suite 180|Pleasant Valley|Grainger County|TN|32477|United States|-5|condo| +47027|AAAAAAAADLHLAAAA|365|Hill Lake|Wy|Suite 430|Mount Pleasant|Switzerland County|IN|41933|United States|-5|condo| +47028|AAAAAAAAELHLAAAA|751|Pine |RD|Suite I|Fairview|Washington County|MO|65709|United States|-6|apartment| +47029|AAAAAAAAFLHLAAAA|791|Spring Park|Court|Suite 350|Unionville|Sagadahoc County|ME|02311|United States|-5|apartment| +47030|AAAAAAAAGLHLAAAA|595|Dogwood Cedar|Dr.|Suite E|Maple Grove|Monroe County|GA|38252|United States|-5|condo| +47031|AAAAAAAAHLHLAAAA|327|Sycamore 5th|RD|Suite T|Saint George|Franklin County|VA|25281|United States|-5|apartment| +47032|AAAAAAAAILHLAAAA|909|River |Pkwy|Suite E|Lakewood|Carroll County|MS|58877|United States|-6|condo| +47033|AAAAAAAAJLHLAAAA|556|15th |Circle|Suite 490|Clinton|Randolph County|IL|68222|United States|-6|single family| +47034|AAAAAAAAKLHLAAAA|449|South |Road|Suite 470|Glenwood|Franklin County|MS|53511|United States|-6|apartment| +47035|AAAAAAAALLHLAAAA|933|Poplar Cedar|Circle|Suite S|Unionville|Brown County|KS|61711|United States|-6|condo| +47036|AAAAAAAAMLHLAAAA|899|Eigth |Circle|Suite Q|Stringtown|Cumberland County|TN|30162|United States|-5|single family| +47037|AAAAAAAANLHLAAAA|164|14th Jefferson|ST|Suite M|Greenville|Claiborne County|MS|51387|United States|-6|condo| +47038|AAAAAAAAOLHLAAAA|664|Forest 7th|RD|Suite 480|Lakewood|Kern County|CA|98877|United States|-8|single family| +47039|AAAAAAAAPLHLAAAA|302|14th |Lane|Suite 190|Buena Vista|Koochiching County|MN|55752|United States|-6|condo| +47040|AAAAAAAAAMHLAAAA|442|Park |Dr.|Suite 210|Roscoe|Texas County|OK|71854|United States|-6|condo| +47041|AAAAAAAABMHLAAAA|500|First |Ave|Suite 350|Woodlawn|Grenada County|MS|54098|United States|-6|apartment| +47042|AAAAAAAACMHLAAAA|416|Railroad |Avenue|Suite G|Springdale|Merced County|CA|98883|United States|-8|condo| +47043|AAAAAAAADMHLAAAA|500|Park |Lane|Suite 160|Newtown|Merrick County|NE|61749|United States|-7|apartment| +47044|AAAAAAAAEMHLAAAA|959|Elm |Parkway|Suite 460|Little River|Ida County|IA|50319|United States|-6|apartment| +47045|AAAAAAAAFMHLAAAA|870|Ninth 15th|Blvd|Suite 310|Millbrook|Curry County|OR|97529|United States|-8|single family| +47046|AAAAAAAAGMHLAAAA|137|5th Dogwood|Ln|Suite T|Pleasant Hill|Upton County|TX|73604|United States|-6|single family| +47047|AAAAAAAAHMHLAAAA|959|Lake 3rd|Ln|Suite G|Shiloh|Mercer County|OH|49275|United States|-5|single family| +47048|AAAAAAAAIMHLAAAA|810|Spruce |ST|Suite 220|Antioch|Rusk County|WI|58605|United States|-6|single family| +47049|AAAAAAAAJMHLAAAA|||Boulevard|Suite 270|Georgetown|Dunklin County|||United States||apartment| +47050|AAAAAAAAKMHLAAAA|121|Hill |Court|Suite S|Walnut Grove|Chesapeake city|VA|27752|United States|-5|condo| +47051|AAAAAAAALMHLAAAA|406|Second West|Way|Suite 60|Greenfield|Boone County|KY|45038|United States|-6|apartment| +47052|AAAAAAAAMMHLAAAA|19|3rd |Pkwy|Suite R|Franklin|Roane County|TN|39101|United States|-6|condo| +47053|AAAAAAAANMHLAAAA|398|Hillcrest Smith|Cir.|Suite 50|Summit|Bennett County|SD|50499|United States|-6|condo| +47054|AAAAAAAAOMHLAAAA|20|Oak |Drive|Suite V|Avery|Buncombe County|NC|20194|United States|-5|apartment| +47055|AAAAAAAAPMHLAAAA|436|Oak West|Drive|Suite O|Unionville|Butler County|MO|61711|United States|-6|apartment| +47056|AAAAAAAAANHLAAAA|739|Pine Ridge|Cir.|Suite 30|Richfield|Knox County|IL|66196|United States|-6|condo| +47057|AAAAAAAABNHLAAAA|898|Birch |Boulevard|Suite D|Cedar Grove|Steuben County|IN|40411|United States|-5|single family| +47058|AAAAAAAACNHLAAAA||Railroad River|Parkway||Bunker Hill||FL|30150|||| +47059|AAAAAAAADNHLAAAA|735|Washington |Ln|Suite 50|Highland Park|Alexander County|IL|66534|United States|-6|single family| +47060|AAAAAAAAENHLAAAA|941|Third |ST|Suite B|Lone Pine|Jasper County|IN|47441|United States|-5|apartment| +47061|AAAAAAAAFNHLAAAA|603|1st |Parkway|Suite 480|Pleasant Hill|Luzerne County|PA|13604|United States|-5|single family| +47062|AAAAAAAAGNHLAAAA|719|Lakeview |Wy|Suite 390|Spring Valley|Yamhill County|OR|96060|United States|-8|condo| +47063|AAAAAAAAHNHLAAAA|236|Sycamore |RD|Suite J|Friendship|Benton County|AR|74536|United States|-6|apartment| +47064|AAAAAAAAINHLAAAA|467|Maple |Avenue|Suite 200|Wilson|Allen County|IN|46971|United States|-5|single family| +47065|AAAAAAAAJNHLAAAA|526|Cedar Third|Court|Suite 180|Riverside|Clifton Forge city|VA|29231|United States|-5|condo| +47066|AAAAAAAAKNHLAAAA|746|Poplar |Lane|Suite K|Unionville|York County|SC|21711|United States|-5|single family| +47067|AAAAAAAALNHLAAAA|385|Lincoln |Lane|Suite P|Rutland|Rabun County|GA|38375|United States|-5|single family| +47068|AAAAAAAAMNHLAAAA|101|Park Cedar|Boulevard|Suite U|Antioch|Dickey County|ND|58605|United States|-6|single family| +47069|AAAAAAAANNHLAAAA||Main |Pkwy|Suite 430|||MI|40150|United States|-5|apartment| +47070|AAAAAAAAONHLAAAA|560|Valley |Drive|Suite V|Florence|Hamilton County|KS|63394|United States|-6|apartment| +47071|AAAAAAAAPNHLAAAA|948|Meadow 15th|Ct.|Suite 310|Hazelwood|Beadle County|SD|51206|United States|-6|apartment| +47072|AAAAAAAAAOHLAAAA|288|2nd |Road|Suite 370|Belmont|Hall County|NE|60191|United States|-6|apartment| +47073|AAAAAAAABOHLAAAA|936|Elm |Pkwy|Suite P|Georgetown|Grafton County|NH|07657|United States|-5|condo| +47074|AAAAAAAACOHLAAAA|357|Cherry 1st||Suite N||Fulton County|GA||United States||condo| +47075|AAAAAAAADOHLAAAA|123|East Washington|RD|Suite E|Woodland|Neosho County|KS|64854|United States|-6|condo| +47076|AAAAAAAAEOHLAAAA|899|Maple 10th|Circle|Suite I|Ashland|McLennan County|TX|74244|United States|-6|single family| +47077|AAAAAAAAFOHLAAAA|872|Mill Dogwood|Blvd|Suite 150|Adrian|Bullock County|AL|33301|United States|-6|single family| +47078|AAAAAAAAGOHLAAAA|812|Miller |Court|Suite 90|Hillcrest|Allen County|KY|43003|United States|-6|condo| +47079|AAAAAAAAHOHLAAAA|688|Franklin Cedar|Road|Suite A|Antioch|Tioga County|NY|18605|United States|-5|condo| +47080|AAAAAAAAIOHLAAAA|646|13th |Drive|Suite P|Oakland|Haywood County|NC|29843|United States|-5|single family| +47081|AAAAAAAAJOHLAAAA|880|2nd |Cir.|Suite Y|Antioch|Lowndes County|MS|58605|United States|-6|condo| +47082|AAAAAAAAKOHLAAAA|664|2nd 3rd|RD|Suite G|Springfield|Shawnee County|KS|69303|United States|-6|single family| +47083|AAAAAAAALOHLAAAA|168|Maple |RD|Suite F|Hopewell|Valley County|MT|60587|United States|-7|single family| +47084|AAAAAAAAMOHLAAAA|316|1st Lincoln|ST|Suite 350|Waterloo|Highland County|VA|21675|United States|-5|condo| +47085|AAAAAAAANOHLAAAA|452|Wilson |Ct.|Suite A|Mountain View|Sherman County|OR|94466|United States|-8|apartment| +47086|AAAAAAAAOOHLAAAA|313|4th 5th|Cir.|Suite R|Salem|Jackson County|MN|58048|United States|-6|condo| +47087|AAAAAAAAPOHLAAAA|53|Johnson |Wy|Suite L|Newport|Foster County|ND|51521|United States|-6|condo| +47088|AAAAAAAAAPHLAAAA|206|Poplar Madison|Way|Suite 20|Fairfield|Stewart County|GA|36192|United States|-5|single family| +47089|AAAAAAAABPHLAAAA|986|Franklin Forest|Court|Suite 430|Woodville|Hancock County|ME|04889|United States|-5|condo| +47090|AAAAAAAACPHLAAAA|980|5th |Parkway|Suite R|Arlington|Adams County|CO|86557|United States|-7|condo| +47091|AAAAAAAADPHLAAAA|127|8th |RD|Suite J|Parkwood|Hancock County|OH|41669|United States|-5|apartment| +47092|AAAAAAAAEPHLAAAA|797|Fourth |Pkwy|Suite A|Fairfield|Dane County|WI|56192|United States|-6|apartment| +47093|AAAAAAAAFPHLAAAA|963|15th |Pkwy|Suite 480|Crossroads|Keith County|NE|60534|United States|-7|condo| +47094|AAAAAAAAGPHLAAAA|549|Mill |Dr.|Suite T|Woodlawn|Rowan County|NC|24098|United States|-5|apartment| +47095|AAAAAAAAHPHLAAAA|629|Jackson |Blvd|Suite 460|Cedar Grove|Amherst County|VA|20411|United States|-5|single family| +47096|AAAAAAAAIPHLAAAA|971|Williams Park|RD|Suite 170|Sunnyside|Arenac County|MI|41952|United States|-5|condo| +47097|AAAAAAAAJPHLAAAA|358|Hill Maple|Boulevard|Suite W|Mount Olive|Maricopa County|AZ|88059|United States|-7|apartment| +47098|AAAAAAAAKPHLAAAA|150|Railroad |Pkwy|Suite 250|Hopewell|Putnam County|GA|30587|United States|-5|apartment| +47099|AAAAAAAALPHLAAAA|71|Adams |Cir.|Suite 420|Amherst|Llano County|TX|78119|United States|-6|single family| +47100|AAAAAAAAMPHLAAAA|901|Lee |ST|Suite 200|Mountain View|Durham County|NC|24466|United States|-5|apartment| +47101|AAAAAAAANPHLAAAA|640|Wilson |Pkwy|Suite 330|Concord|Haralson County|GA|34107|United States|-5|apartment| +47102|AAAAAAAAOPHLAAAA|538|Central |Dr.|Suite Q|Florence|San Juan County|WA|93394|United States|-8|single family| +47103|AAAAAAAAPPHLAAAA|839|Willow 9th|Ave|Suite 80|Bunker Hill|Alexander County|NC|20150|United States|-5|apartment| +47104|AAAAAAAAAAILAAAA|219|Spring First|Drive|Suite 340|Pleasant Valley|Comanche County|TX|72477|United States|-6|apartment| +47105|AAAAAAAABAILAAAA|146|Sunset Cherry|Street|Suite 230|Prospect|Montgomery County|OH|48578|United States|-5|apartment| +47106|AAAAAAAACAILAAAA|692|Park Elevnth|RD|Suite 300|Denmark|Shenandoah County|VA|25576|United States|-5|apartment| +47107|AAAAAAAADAILAAAA|461|Railroad |Boulevard|Suite 240|Clifton|Waupaca County|WI|58014|United States|-6|condo| +47108|AAAAAAAAEAILAAAA|707|Elm |Parkway|Suite Y|Oakdale|Seneca County|OH|49584|United States|-5|condo| +47109|AAAAAAAAFAILAAAA|597|Davis Third|Ave|Suite H|Marion|Creek County|OK|70399|United States|-6|apartment| +47110|AAAAAAAAGAILAAAA|442|3rd |Dr.|Suite T|Riverside|Montgomery County|OH|49231|United States|-5|condo| +47111|AAAAAAAAHAILAAAA|865|Oak Maple|Lane|Suite R|Clifton|Union Parish|LA|78014|United States|-6|single family| +47112|AAAAAAAAIAILAAAA|474|Birch |Lane|Suite A|Red Hill|Humboldt County|IA|54338|United States|-6|apartment| +47113|AAAAAAAAJAILAAAA|509|Fourteenth |Court|Suite Y|Lakeview|Jay County|IN|48579|United States|-5|apartment| +47114|AAAAAAAAKAILAAAA|631|7th |Lane|Suite 280|Spring Hill|Kane County|IL|66787|United States|-6|condo| +47115|AAAAAAAALAILAAAA|592|Fourteenth |Blvd|Suite W|Clifton|Floyd County|TX|78014|United States|-6|single family| +47116|AAAAAAAAMAILAAAA|477|College Spring|Circle|Suite 480|New Hope|Watonwan County|MN|59431|United States|-6|apartment| +47117|AAAAAAAANAILAAAA|524|Cedar 9th|ST|Suite V|Green Acres|Branch County|MI|47683|United States|-5|single family| +47118|AAAAAAAAOAILAAAA|153|Sunset |Way|Suite F|Newport|Humboldt County|NV|81521|United States|-8|condo| +47119|AAAAAAAAPAILAAAA|402|Davis |Wy|Suite 10|Woodlawn|Wayne County|MO|64098|United States|-6|single family| +47120|AAAAAAAAABILAAAA|7|8th Elm|Ln|Suite 440|Mountain View|Glacier County|MT|64466|United States|-7|single family| +47121|AAAAAAAABBILAAAA|16|11th Cherry|ST|Suite L|Edwards|Hendry County|FL|31409|United States|-5|apartment| +47122|AAAAAAAACBILAAAA|937|Hillcrest Williams|Boulevard|Suite 210|Cedar Grove|Richmond County|NY|10411|United States|-5|condo| +47123|AAAAAAAADBILAAAA|436|Railroad Mill|Ave|Suite 420|Providence|Allen County|OH|46614|United States|-5|single family| +47124|AAAAAAAAEBILAAAA|813|Main |Avenue|Suite 10|Mountain View|Irion County|TX|74466|United States|-6|condo| +47125|AAAAAAAAFBILAAAA|573|Madison |Way|Suite L|Arthur|Cattaraugus County|NY|15965|United States|-5|single family| +47126|AAAAAAAAGBILAAAA|828|Laurel |ST|Suite 160|Brownsville|Ward County|TX|79310|United States|-6|condo| +47127|AAAAAAAAHBILAAAA|651|Railroad 11th|Blvd|Suite L|New Hope|Augusta County|VA|29431|United States|-5|apartment| +47128|AAAAAAAAIBILAAAA|758|Highland Jackson|Cir.|Suite N|Bunker Hill|Ontario County|NY|10150|United States|-5|condo| +47129|AAAAAAAAJBILAAAA|972|2nd |Ave|Suite 320|Newtown|Worcester County|MA|02349|United States|-5|apartment| +47130|AAAAAAAAKBILAAAA|||ST|Suite 110|Allison|||34167|||| +47131|AAAAAAAALBILAAAA||Maple Tenth|Cir.||Shiloh|||69275|United States||condo| +47132|AAAAAAAAMBILAAAA|829|7th Maple|Court|Suite I|Pleasant Grove|Ravalli County|MT|64136|United States|-7|condo| +47133|AAAAAAAANBILAAAA|766|6th Main|Ln|Suite 60|Hamilton|Sherman County|KS|62808|United States|-6|single family| +47134|AAAAAAAAOBILAAAA|40|Park |Blvd|Suite Q|Sulphur Springs|Holmes County|FL|38354|United States|-5|single family| +47135|AAAAAAAAPBILAAAA|418|Highland |Ln|Suite 240|Jamestown|Moffat County|CO|86867|United States|-7|condo| +47136|AAAAAAAAACILAAAA|215|Church Ash|Road|Suite 430|Liberty|Hall County|TX|73451|United States|-6|single family| +47137|AAAAAAAABCILAAAA|753|Sunset Jackson|Way|Suite 470|Johnsonville|Lincoln County|OR|97745|United States|-8|apartment| +47138|AAAAAAAACCILAAAA|711|Mill Cherry|Wy|Suite H|Roxbury|DeKalb County|GA|35508|United States|-5|single family| +47139|AAAAAAAADCILAAAA|541|Ridge |Ct.|Suite 360|Oakdale|Douglas County|NE|69584|United States|-6|single family| +47140|AAAAAAAAECILAAAA|995|Seventh 12th|Boulevard|Suite 350|Enterprise|Grant County|AR|71757|United States|-6|apartment| +47141|AAAAAAAAFCILAAAA|640|College |Road|Suite 30|Shady Grove|Tompkins County|NY|12812|United States|-5|condo| +47142|AAAAAAAAGCILAAAA|512|River |Court|Suite 70|Buena Vista|Pueblo County|CO|85752|United States|-7|condo| +47143|AAAAAAAAHCILAAAA|239|6th Jefferson|Ln|Suite K|Farmington|Wheeler County|OR|99145|United States|-8|single family| +47144|AAAAAAAAICILAAAA|342|1st |Pkwy|Suite 150|Gravel Hill|Sullivan County|NH|02544|United States|-5|condo| +47145|AAAAAAAAJCILAAAA|500|Ninth Franklin|Court|Suite 480|Bunker Hill|Montgomery County|KY|40150|United States|-5|condo| +47146|AAAAAAAAKCILAAAA|741|Locust Center|Ct.|Suite K|Plainview|Greene County|MO|63683|United States|-6|single family| +47147|AAAAAAAALCILAAAA|416|4th |Ave|Suite 430|Parkwood|Fauquier County|VA|21669|United States|-5|apartment| +47148|AAAAAAAAMCILAAAA|820|13th Oak|Lane|Suite X|Waterloo|Putnam County|OH|41675|United States|-5|apartment| +47149|AAAAAAAANCILAAAA|752|Cedar Cherry|Blvd|Suite 0|Hopewell|Washington County|OH|40587|United States|-5|condo| +47150|AAAAAAAAOCILAAAA|262|View Adams|Way|Suite S|Green Acres|Douglas County|IL|67683|United States|-6|condo| +47151|AAAAAAAAPCILAAAA|426|Center Locust|RD|Suite G|Brownsville|Edwards County|KS|69310|United States|-6|single family| +47152|AAAAAAAAADILAAAA|262|Elm Jackson|Ln|Suite U|Macedonia|Lewis County|MO|61087|United States|-6|apartment| +47153|AAAAAAAABDILAAAA|249|Elevnth 6th|Drive|Suite 300|Friendship|Warren County|IA|54536|United States|-6|apartment| +47154|AAAAAAAACDILAAAA|827|Hickory |Boulevard|Suite 50|White Oak|Custer County|MT|66668|United States|-7|apartment| +47155|AAAAAAAADDILAAAA|805|Pine |Ct.|Suite 420|Mount Pleasant|Beaver County|UT|81933|United States|-7|condo| +47156|AAAAAAAAEDILAAAA|426|Elm Madison|Wy|Suite A|Farmington|Liberty County|MT|69145|United States|-7|apartment| +47157|AAAAAAAAFDILAAAA||Maple ||||Salem city||23451|United States|-5|| +47158|AAAAAAAAGDILAAAA|139|Maple |Court|Suite 150|Concord|Howard County|MO|64107|United States|-6|single family| +47159|AAAAAAAAHDILAAAA|961|Sycamore Forest|Dr.|Suite S|Leesville|Rutland County|VT|06023|United States|-5|condo| +47160|AAAAAAAAIDILAAAA|63|Pine |Blvd|Suite 140|Pinhook|Ida County|IA|59398|United States|-6|single family| +47161|AAAAAAAAJDILAAAA|725|5th 9th|Blvd|Suite 240|Cedar Grove|Darke County|OH|40411|United States|-5|single family| +47162|AAAAAAAAKDILAAAA|303|Franklin 6th|Boulevard|Suite 280|Wildwood|Uinta County|WY|86871|United States|-7|apartment| +47163|AAAAAAAALDILAAAA|418|Church |Ave|Suite 360|Mount Zion|Chesapeake city|VA|28054|United States|-5|condo| +47164|AAAAAAAAMDILAAAA|374|Park 2nd|Ln|Suite 10|Brentwood|Chaves County|NM|84188|United States|-7|condo| +47165|AAAAAAAANDILAAAA|700|Chestnut Wilson|Circle|Suite 260|Sunnyside|Ferry County|WA|91952|United States|-8|single family| +47166|AAAAAAAAODILAAAA|553|Main |Dr.|Suite 490|Friendship|Lyon County|MN|54536|United States|-6|condo| +47167|AAAAAAAAPDILAAAA|466|9th |Ave|Suite 390|Pleasant Grove|Yellow Medicine County|MN|54136|United States|-6|single family| +47168|AAAAAAAAAEILAAAA|13|Central |Circle|Suite 350|Oakland|Butler County|MO|69843|United States|-6|condo| +47169|AAAAAAAABEILAAAA|618|Mill Forest|Boulevard|Suite X|Cedar Grove|Fergus County|MT|60411|United States|-7|single family| +47170|AAAAAAAACEILAAAA|723|Park |Court|Suite U|Kingston|Sabine County|TX|74975|United States|-6|apartment| +47171|AAAAAAAADEILAAAA|901|Park |Parkway|Suite 150|Plainview|McIntosh County|OK|73683|United States|-6|single family| +47172|AAAAAAAAEEILAAAA|150|4th Eigth|Boulevard|Suite M|Bath|Logan County|ND|50573|United States|-6|apartment| +47173|AAAAAAAAFEILAAAA|778|||Suite W||||38605|United States||| +47174|AAAAAAAAGEILAAAA|612|11th |Wy|Suite F|Oak Ridge|Yell County|AR|78371|United States|-6|condo| +47175|AAAAAAAAHEILAAAA|638|Chestnut |Street|Suite W|Lakeside|Northumberland County|VA|29532|United States|-5|condo| +47176|AAAAAAAAIEILAAAA|938|Ash |Court|Suite L|Pleasant Grove|Lake and Peninsula Borough|AK|94136|United States|-9|apartment| +47177|AAAAAAAAJEILAAAA|382|6th |Ln|Suite X|Mount Pleasant|Monroe County|IA|51933|United States|-6|condo| +47178|AAAAAAAAKEILAAAA|774|Sixth |Court|Suite 430|Hamilton|Huron County|MI|42808|United States|-5|single family| +47179|AAAAAAAALEILAAAA|251|View 7th|Circle|Suite G|Winchester|Franklin County|NY|13252|United States|-5|condo| +47180|AAAAAAAAMEILAAAA|204|Willow |Circle|Suite 160|Farmington|Linn County|OR|99145|United States|-8|apartment| +47181|AAAAAAAANEILAAAA|316|Hickory Birch|Court|Suite V|Providence|Clay County|MS|56614|United States|-6|apartment| +47182|AAAAAAAAOEILAAAA|739|7th |Drive|Suite 400|Bridgeport|Churchill County|NV|85817|United States|-8|single family| +47183|AAAAAAAAPEILAAAA|465|Main |Cir.|||||||-7|| +47184|AAAAAAAAAFILAAAA|777|First |Parkway|Suite K|Oakdale|Riley County|KS|69584|United States|-6|apartment| +47185|AAAAAAAABFILAAAA|262|Cedar |Circle|Suite B|Hillcrest|Roane County|TN|33003|United States|-6|condo| +47186|AAAAAAAACFILAAAA|12|Center |Parkway|Suite Y|Five Points|Pike County|KY|46098|United States|-5|apartment| +47187|AAAAAAAADFILAAAA|413|View Franklin|ST|Suite 420|Bridgeport|Texas County|OK|75817|United States|-6|apartment| +47188|AAAAAAAAEFILAAAA|463|Oak |Ct.|Suite L|Highland Park|Cherokee County|NC|26534|United States|-5|condo| +47189|AAAAAAAAFFILAAAA|951|Fifth |Cir.|Suite 90|Gary|Bibb County|GA|30418|United States|-5|condo| +47190|AAAAAAAAGFILAAAA|662|Davis Washington|Pkwy|Suite J|Clinton|Ozaukee County|WI|58222|United States|-6|single family| +47191|AAAAAAAAHFILAAAA|967|Walnut 1st|Ct.|Suite Y|Five Forks|James City County|VA|22293|United States|-5|apartment| +47192|AAAAAAAAIFILAAAA|736|Hickory |Drive|Suite F|Red Bank|Fannin County|TX|74975|United States|-6|apartment| +47193|AAAAAAAAJFILAAAA|83|College |Cir.|Suite J|Antioch|Cass County|MN|58605|United States|-6|apartment| +47194|AAAAAAAAKFILAAAA|735|Main Elm|Lane|Suite Y|Mount Zion|Ellis County|KS|68054|United States|-6|condo| +47195|AAAAAAAALFILAAAA|181|Williams Oak|Court|Suite T|Jordan|Wolfe County|KY|45391|United States|-5|apartment| +47196|AAAAAAAAMFILAAAA|133|4th |Pkwy|Suite 300|Pleasant Grove|Madison County|OH|44136|United States|-5|condo| +47197|AAAAAAAANFILAAAA|344|Adams |Wy|Suite W|Glenwood|Washington County|CO|83511|United States|-7|apartment| +47198|AAAAAAAAOFILAAAA|568|6th Adams|Ct.||Lebanon|Chautauqua County||12898|United States|-5|condo| +47199|AAAAAAAAPFILAAAA|928|Ridge |ST|Suite 400|Georgetown|Meeker County|MN|57057|United States|-6|single family| +47200|AAAAAAAAAGILAAAA|100|12th |Court|Suite T|Wildwood|Lincoln County|OK|76871|United States|-6|single family| +47201|AAAAAAAABGILAAAA|262|Tenth Madison|Boulevard|Suite 430|Shiloh|Butler County|AL|39275|United States|-6|condo| +47202|AAAAAAAACGILAAAA|488|Oak |Street|Suite 260|Concord|Cedar County|NE|64107|United States|-6|single family| +47203|AAAAAAAADGILAAAA|791|Walnut |Ct.|Suite R|Sunnyside|Richland County|ND|51952|United States|-6|single family| +47204|AAAAAAAAEGILAAAA|887|Forest |Street|Suite X|Salem|Marshall County|SD|58048|United States|-7|apartment| +47205|AAAAAAAAFGILAAAA|247|Valley Miller|Ln|Suite 370|New Hope|Ohio County|WV|29431|United States|-5|condo| +47206|AAAAAAAAGGILAAAA|82|3rd |Ln|Suite L|Glendale|Martin County|MN|53951|United States|-6|apartment| +47207|AAAAAAAAHGILAAAA|470|West |Court|Suite 100|Brownsville|Denali Borough|AK|99310|United States|-9|apartment| +47208|AAAAAAAAIGILAAAA||Second ||Suite P||Wayne County|NY|||-5|apartment| +47209|AAAAAAAAJGILAAAA|394|Miller North|Ave|Suite 20|Bethel|Marion County|MS|55281|United States|-6|single family| +47210|AAAAAAAAKGILAAAA|905|Valley South|Ave|Suite L|White Oak|Crosby County|TX|76668|United States|-6|single family| +47211|AAAAAAAALGILAAAA|636|Park Fourth|Ln|Suite I|Valley View|Delaware County|PA|15124|United States|-5|apartment| +47212|AAAAAAAAMGILAAAA|170|Spring |Pkwy|Suite 380|Oakland|Sierra County|CA|99843|United States|-8|single family| +47213|AAAAAAAANGILAAAA|327|Fourth |Lane|Suite B|Wilson|Wadena County|MN|56971|United States|-6|apartment| +47214|AAAAAAAAOGILAAAA|240|Green |Ln|Suite E|Spring Valley|Sioux County|ND|56060|United States|-6|apartment| +47215|AAAAAAAAPGILAAAA|38|River 1st|Parkway|Suite 10|Spring Valley|Magoffin County|KY|46060|United States|-5|single family| +47216|AAAAAAAAAHILAAAA|994|8th |Parkway|Suite 170|Five Points|Lafourche Parish|LA|76098|United States|-6|single family| +47217|AAAAAAAABHILAAAA|529|Cedar |Drive|Suite Q|Deerfield|Red Willow County|NE|69840|United States|-7|single family| +47218|AAAAAAAACHILAAAA|348|Fifth |RD|Suite B|Green Acres|Clay County|TX|77683|United States|-6|single family| +47219|AAAAAAAADHILAAAA|100|Franklin |Cir.|Suite 460|Mount Zion|Calhoun County|WV|28054|United States|-5|apartment| +47220|AAAAAAAAEHILAAAA|755|Hillcrest 15th|Ln|Suite O|Clinton|Weber County|UT|88222|United States|-7|condo| +47221|AAAAAAAAFHILAAAA|961|4th |Circle|Suite D|Oak Grove|Union County|SD|58370|United States|-7|condo| +47222|AAAAAAAAGHILAAAA|347|Main River|Drive|Suite 170|Concord|Collier County|FL|34107|United States|-5|single family| +47223|AAAAAAAAHHILAAAA|645|Main 15th|Boulevard|Suite 200|Bethany|Butler County|KS|65460|United States|-6|condo| +47224|AAAAAAAAIHILAAAA|815|6th |RD|Suite 390|Shiloh|East Carroll Parish|LA|79275|United States|-6|condo| +47225|AAAAAAAAJHILAAAA|14|Center Meadow|Parkway|Suite 240|Lebanon|Barnes County|ND|52898|United States|-6|single family| +47226|AAAAAAAAKHILAAAA|662|View |Wy|Suite N|Harmony|Rawlins County|KS|65804|United States|-6|condo| +47227|AAAAAAAALHILAAAA|661|South Lee|Road|Suite R|Antioch||IA||United States|-6|condo| +47228|AAAAAAAAMHILAAAA|570|Green |Dr.|Suite 0|Woodville|Middlesex County|VA|24289|United States|-5|single family| +47229|AAAAAAAANHILAAAA|8|Franklin |Avenue|Suite 260|Woodville|Kimball County|NE|64289|United States|-7|single family| +47230|AAAAAAAAOHILAAAA|890|Sunset |Road|Suite H|Clifton|Prince Edward County|VA|28014|United States|-5|apartment| +47231|AAAAAAAAPHILAAAA|749|Railroad |Dr.|Suite L|Jackson|Taos County|NM|89583|United States|-7|apartment| +47232|AAAAAAAAAIILAAAA|456|Forest |Cir.|Suite B|Stringtown|Winnebago County|IA|50162|United States|-6|single family| +47233|AAAAAAAABIILAAAA|412|Main Sycamore|Street|Suite 330|Greenwood|Elkhart County|IN|48828|United States|-5|apartment| +47234|AAAAAAAACIILAAAA|158|Mill |Drive|Suite V|Glendale|Grant County|KY|43951|United States|-6|single family| +47235|AAAAAAAADIILAAAA|869|Second River|Parkway|Suite 490|Oak Hill|Sussex County|VA|27838|United States|-5|apartment| +47236|AAAAAAAAEIILAAAA|996||Pkwy|Suite 230|Green Acres|||67683|United States||| +47237|AAAAAAAAFIILAAAA|945|Elm |Cir.|Suite X|Highland|Ralls County|MO|69454|United States|-6|apartment| +47238|AAAAAAAAGIILAAAA|2|North 12th|Street|Suite K|Stringtown|Boundary County|ID|80162|United States|-7|single family| +47239|AAAAAAAAHIILAAAA|84|Washington |Ct.|Suite 170|Wilson|Gallatin County|IL|66971|United States|-6|condo| +47240|AAAAAAAAIIILAAAA|20|4th |Ln|Suite 270|Clifton|Winston County|AL|38014|United States|-6|condo| +47241|AAAAAAAAJIILAAAA|712|Cherry |Ln|Suite H|Arlington|Roger Mills County|OK|76557|United States|-6|single family| +47242|AAAAAAAAKIILAAAA|420|Valley |Ln|Suite 330|Oak Ridge|Vernon County|MO|68371|United States|-6|condo| +47243|AAAAAAAALIILAAAA|301|Forest Laurel|Street|Suite Y|Highland|Lea County|NM|89454|United States|-7|single family| +47244|AAAAAAAAMIILAAAA|499|Woodland |Way|Suite 230|Highland Park|Morgan County|KY|46534|United States|-5|single family| +47245|AAAAAAAANIILAAAA|20|||||Barton County|MO|||-6|| +47246|AAAAAAAAOIILAAAA|183|Green ||||||||-5|| +47247|AAAAAAAAPIILAAAA|843|13th 3rd|Road|Suite T|Lee|Chicot County|AR|70408|United States|-6|single family| +47248|AAAAAAAAAJILAAAA|620|8th |Pkwy|Suite 300|Florence|Putnam County|IL|63394|United States|-6|apartment| +47249|AAAAAAAABJILAAAA|226|4th Spruce|Avenue|Suite W|Bridgeport|Clark County|SD|55817|United States|-6|single family| +47250|AAAAAAAACJILAAAA|478|Jackson 3rd|Cir.|Suite 270|Royal|Franklin County|MA|06419|United States|-5|single family| +47251|AAAAAAAADJILAAAA|212|Eigth 14th|Court|Suite 200|Lebanon|Decatur County|IN|42898|United States|-5|apartment| +47252|AAAAAAAAEJILAAAA|43|Lake |ST|Suite X|Springdale|Houston County|AL|38883|United States|-6|single family| +47253|AAAAAAAAFJILAAAA|297|Johnson |Drive|Suite 390|Greenwood|Beaver County|UT|88828|United States|-7|condo| +47254|AAAAAAAAGJILAAAA|918|Smith |Road|Suite 150|Five Forks|Tillamook County|OR|92293|United States|-8|condo| +47255|AAAAAAAAHJILAAAA|304|Fourteenth Main|Wy|Suite 110|Millbrook|Durham County|NC|27529|United States|-5|single family| +47256|AAAAAAAAIJILAAAA|570|Smith |ST|Suite N|Bethel|Waushara County|WI|55281|United States|-6|condo| +47257|AAAAAAAAJJILAAAA|180|Lake |Boulevard|Suite Y|Paxton|Marion County|IA|55669|United States|-6|single family| +47258|AAAAAAAAKJILAAAA|772|7th 10th|ST|Suite 60|Bridgeport|Willacy County|TX|75817|United States|-6|condo| +47259|AAAAAAAALJILAAAA|329|Meadow Miller|Dr.|Suite 110|Green Acres|Juneau County|WI|57683|United States|-6|single family| +47260|AAAAAAAAMJILAAAA|194|Woodland |Ln|Suite F|Clinton|Clay County|TX|78222|United States|-6|condo| +47261|AAAAAAAANJILAAAA|424|Fifth |Drive|Suite D|Mount Pleasant|Crawford County|AR|71933|United States|-6|single family| +47262|AAAAAAAAOJILAAAA|249|1st |Ave|Suite Q|Tanglewood|Murray County|MN|58994|United States|-6|apartment| +47263|AAAAAAAAPJILAAAA|572|Maple |RD|Suite T|Deerfield|Fulton County|AR|79840|United States|-6|condo| +47264|AAAAAAAAAKILAAAA|399|Park Forest|Ct.|Suite G|Sulphur Springs|Carroll County|MD|28354|United States|-5|apartment| +47265|AAAAAAAABKILAAAA|902|Willow River|Road|Suite C|Antioch|Newton County|MO|68605|United States|-6|apartment| +47266|AAAAAAAACKILAAAA|769|Maple |Way|Suite 310|Woodlawn|Wilcox County|GA|34098|United States|-5|condo| +47267|AAAAAAAADKILAAAA|143|2nd Broadway|Lane|Suite V|Ashland|Craighead County|AR|74244|United States|-6|single family| +47268|AAAAAAAAEKILAAAA|870|Spruce Wilson|Drive|Suite R|Waterloo|||71675|||| +47269|AAAAAAAAFKILAAAA|52|Central Sycamore|Ct.|Suite 380|Enterprise|Butte County|ID|81757|United States|-7|condo| +47270|AAAAAAAAGKILAAAA|197|11th |Blvd|Suite S|Marion|Fayette County|AL|30399|United States|-6|apartment| +47271|AAAAAAAAHKILAAAA|885|13th |Street|Suite V|Farmington|Bannock County|ID|89145|United States|-7|apartment| +47272|AAAAAAAAIKILAAAA|110|1st Birch|Ln|Suite 170|Oak Hill|Lawrence County|IL|67838|United States|-6|apartment| +47273|AAAAAAAAJKILAAAA|896|Park |Boulevard|Suite 350|Deerfield|Crockett County|TX|79840|United States|-6|single family| +47274|AAAAAAAAKKILAAAA|949|Spring |Wy|Suite F|Newport|Venango County|PA|11521|United States|-5|apartment| +47275|AAAAAAAALKILAAAA|389|Railroad Fourth|Circle|Suite K|Wilson|Garvin County|OK|76971|United States|-6|apartment| +47276|AAAAAAAAMKILAAAA|824|East |Street|Suite K|Concord|Polk County|TX|74107|United States|-6|single family| +47277|AAAAAAAANKILAAAA|251|6th |Pkwy|Suite 250|Highland Park|Kingsbury County|SD|56534|United States|-7|apartment| +47278|AAAAAAAAOKILAAAA|144|Spruce 2nd|Circle|Suite J|Highland Park|Hinds County|MS|56534|United States|-6|apartment| +47279|AAAAAAAAPKILAAAA|353|Poplar Maple|Lane|Suite 150|Oakland|Lampasas County|TX|79843|United States|-6|single family| +47280|AAAAAAAAALILAAAA|641|Lake Wilson|Ct.|Suite L|Oakwood|Green Lake County|WI|50169|United States|-6|condo| +47281|AAAAAAAABLILAAAA|698|Willow |Parkway|Suite 220|Hamilton|Crow Wing County|MN|52808|United States|-6|single family| +47282|AAAAAAAACLILAAAA|615|Jefferson Hickory|Ct.|Suite 450|Fairview|Jefferson County|IL|65709|United States|-6|condo| +47283|AAAAAAAADLILAAAA|892|First |Drive|Suite O|Valley View|Rockingham County|NC|25124|United States|-5|single family| +47284|AAAAAAAAELILAAAA|657|Third |Dr.|Suite 90|Oakdale|Meade County|KY|49584|United States|-5|condo| +47285|AAAAAAAAFLILAAAA|309|Second |Avenue|Suite A|Hamilton|Northwest Arctic Borough|AK|92808|United States|-9|apartment| +47286|AAAAAAAAGLILAAAA|843|3rd |Ln|Suite T|Ruby|Paulding County|OH|40858|United States|-5|apartment| +47287|AAAAAAAAHLILAAAA|80|Hill 9th|Street|Suite 270|Oakdale|Mitchell County|GA|39584|United States|-5|single family| +47288|AAAAAAAAILILAAAA|828|South Broadway|Ave|Suite 270|Glendale|Nassau County|FL|33951|United States|-5|apartment| +47289|AAAAAAAAJLILAAAA|939|Spring |Way|Suite H|Cherry Valley|Davie County|NC|20854|United States|-5|condo| +47290|AAAAAAAAKLILAAAA|724|2nd |ST|Suite 20|Greenville|Walworth County|SD|51387|United States|-7|single family| +47291|AAAAAAAALLILAAAA|807|Lee Elm|Lane|Suite K|Antioch|Ada County|ID|88605|United States|-7|apartment| +47292|AAAAAAAAMLILAAAA|737|North 3rd|Court|Suite R|Mountain View|Stewart County|GA|34466|United States|-5|apartment| +47293|AAAAAAAANLILAAAA|942|4th Park|Road|Suite 340|Clinton|Stutsman County|ND|58222|United States|-6|single family| +47294|AAAAAAAAOLILAAAA|821|Eigth |Avenue|Suite F|Pomona|San Joaquin County|CA|94153|United States|-8|apartment| +47295|AAAAAAAAPLILAAAA|148|Ridge |Court|Suite V|Spring Hill|Bradley County|TN|36787|United States|-5|apartment| +47296|AAAAAAAAAMILAAAA|636|Cedar Maple|Wy|Suite 170|Marion|Hopkins County|TX|70399|United States|-6|apartment| +47297|AAAAAAAABMILAAAA|226|Franklin View|RD|Suite 60|Springdale|Greene County|OH|48883|United States|-5|apartment| +47298|AAAAAAAACMILAAAA|205|15th |Cir.|Suite K|Mount Pleasant|Jackson County|SD|51933|United States|-7|single family| +47299|AAAAAAAADMILAAAA|448|1st Railroad|Dr.|Suite 260|Jamestown|Wallace County|KS|66867|United States|-6|single family| +47300|AAAAAAAAEMILAAAA|832|North 1st|Street|Suite 480|Woodlawn|James City County|VA|24098|United States|-5|single family| +47301|AAAAAAAAFMILAAAA|423|Meadow |Parkway|Suite 350|Oak Hill|Effingham County|IL|67838|United States|-6|single family| +47302|AAAAAAAAGMILAAAA|681|Park |Avenue|Suite 200|Crossroads|Middlesex County|CT|01134|United States|-5|apartment| +47303|AAAAAAAAHMILAAAA|754|Smith |Parkway|Suite L|New Hope|Clark County|WA|99431|United States|-8|single family| +47304|AAAAAAAAIMILAAAA|958|Cedar 5th|Ct.|Suite 300|Macedonia|Mason County|KY|41087|United States|-5|single family| +47305|AAAAAAAAJMILAAAA|635|Jackson |Avenue|Suite 140|Sulphur Springs|Morgan County|GA|38354|United States|-5|apartment| +47306|AAAAAAAAKMILAAAA|186|Lincoln |Lane|Suite A|Sulphur Springs|Crawford County|KS|68354|United States|-6|condo| +47307|AAAAAAAALMILAAAA|583|6th Jackson|Street|Suite P|Springdale|Graves County|KY|48883|United States|-6|apartment| +47308|AAAAAAAAMMILAAAA|487|Laurel |RD|Suite 60|Philadelphia|Minidoka County|ID|85591|United States|-7|single family| +47309|AAAAAAAANMILAAAA|950|Meadow |Court|Suite C|Brownsville|Hardin County|KY|49310|United States|-6|apartment| +47310|AAAAAAAAOMILAAAA|964|11th Main|Dr.|Suite S|Oak Hill|Pike County|GA|37838|United States|-5|condo| +47311|AAAAAAAAPMILAAAA|1|3rd |RD|Suite 420|Ingleside|Sarasota County|FL|34356|United States|-5|single family| +47312|AAAAAAAAANILAAAA|11|Spruce |Dr.|Suite X|Union|Morgan County|IN|48721|United States|-5|single family| +47313|AAAAAAAABNILAAAA|31|4th North|Street|Suite T|Jamestown|Lauderdale County|TN|36867|United States|-6|single family| +47314|AAAAAAAACNILAAAA|698|Cherry |Street|Suite 330|Pine Grove|Ripley County|IN|44593|United States|-5|single family| +47315|AAAAAAAADNILAAAA|837|||Suite 140|Springfield|||39303|||| +47316|AAAAAAAAENILAAAA|557|Second Spring|Blvd|Suite 100|Stringtown|Cedar County|IA|50162|United States|-6|apartment| +47317|AAAAAAAAFNILAAAA|310|Second |Drive|Suite N|Crossroads|Glades County|FL|30534|United States|-5|apartment| +47318|AAAAAAAAGNILAAAA|993|Woodland |Ct.|Suite 60|Centerville|Lee County|MS|50059|United States|-6|single family| +47319|AAAAAAAAHNILAAAA|42|Wilson Broadway|Blvd|Suite M|Lakewood|Wise County|TX|78877|United States|-6|apartment| +47320|AAAAAAAAINILAAAA|126|Washington 15th|Circle|Suite J|Lakeside|Garrard County|KY|49532|United States|-6|single family| +47321|AAAAAAAAJNILAAAA|854||Court|Suite 80|Wildwood||||United States|-5|condo| +47322|AAAAAAAAKNILAAAA|31|3rd River|Blvd|Suite J|Shiloh|Chester County|TN|39275|United States|-5|condo| +47323|AAAAAAAALNILAAAA||8th |Dr.|Suite G|Gilmore|||55464|||| +47324|AAAAAAAAMNILAAAA|200|Hill |Avenue|Suite W|Antioch|Manitowoc County|WI|58605|United States|-6|condo| +47325|AAAAAAAANNILAAAA|567|Davis |Road|Suite 30|Georgetown|Nemaha County|NE|67057|United States|-7|single family| +47326|AAAAAAAAONILAAAA|68|Maple Davis|Circle|Suite 10|Spring Valley|Franklin County|WA|96060|United States|-8|condo| +47327|AAAAAAAAPNILAAAA|||Road||Oakwood||||United States||| +47328|AAAAAAAAAOILAAAA|874|River |Street|Suite 480|Woodlawn|Elko County|NV|84098|United States|-8|apartment| +47329|AAAAAAAABOILAAAA|657|5th Lincoln|Blvd|Suite N|Newport|Riley County|KS|61521|United States|-6|apartment| +47330|AAAAAAAACOILAAAA|883||Wy|Suite F|Edgewater|Brazos County|TX|70635||-6|| +47331|AAAAAAAADOILAAAA|774|11th 11th|Drive|Suite 380|Highland|Grenada County|MS|59454|United States|-6|apartment| +47332|AAAAAAAAEOILAAAA||Laurel |Circle||Liberty|Alger County|MI|43451||-5|single family| +47333|AAAAAAAAFOILAAAA|882|Oak |Blvd|Suite 140|Liberty|Mecklenburg County|NC|23451|United States|-5|condo| +47334|AAAAAAAAGOILAAAA|655|Seventh Pine|Cir.|Suite T|Farmington|Piscataquis County|ME|09745|United States|-5|single family| +47335|AAAAAAAAHOILAAAA|241|Second Spruce|Ct.|Suite D|Shady Grove|Posey County|IN|42812|United States|-5|apartment| +47336|AAAAAAAAIOILAAAA|43|Jackson Woodland|Blvd|Suite 290|Mount Olive|Richland County|SC|28059|United States|-5|single family| +47337|AAAAAAAAJOILAAAA|610|4th Broadway|Blvd|Suite I|Franklin|Pike County|GA|39101|United States|-5|apartment| +47338|AAAAAAAAKOILAAAA|854|7th |Blvd|Suite 330|Hurricane|Jackson County|KS|67644|United States|-6|condo| +47339|AAAAAAAALOILAAAA|476|Ash Cedar|ST|||Henry County|OH||||| +47340|AAAAAAAAMOILAAAA|251|Ash |Parkway|Suite X|Arcola|Dolores County|CO|81654|United States|-7|condo| +47341|AAAAAAAANOILAAAA|757|Broadway Walnut|Pkwy|Suite 130|Mountain View|Greene County|MS|54466|United States|-6|apartment| +47342|AAAAAAAAOOILAAAA|540|1st Tenth|Ave|Suite D|Wilton|Adams County|CO|86997|United States|-7|condo| +47343|AAAAAAAAPOILAAAA|83|Central 7th|Parkway|Suite 180|Macedonia|Box Butte County|NE|61087|United States|-6|condo| +47344|AAAAAAAAAPILAAAA|49|Elm Lee|ST|Suite 260|Post Oak|Butler County|OH|48567|United States|-5|condo| +47345|AAAAAAAABPILAAAA|348|1st Poplar|Blvd|Suite 430|Lakeview|Dallas County|IA|58579|United States|-6|condo| +47346|AAAAAAAACPILAAAA|606|1st Elm|Road|Suite A|Spring Valley|Harvey County|KS|66060|United States|-6|single family| +47347|AAAAAAAADPILAAAA|905|Adams |RD|Suite H|Wesley|Jennings County|IN|41218|United States|-5|apartment| +47348|AAAAAAAAEPILAAAA|598|10th Franklin|RD|Suite 260|New Hope|Kittitas County|WA|99431|United States|-8|apartment| +47349|AAAAAAAAFPILAAAA|938|West Madison|RD|Suite H|Newport|Pulaski County|IL|61521|United States|-6|single family| +47350|AAAAAAAAGPILAAAA|301|Sixth |Circle|Suite 30|Summerfield|Monroe County|AL|30634|United States|-6|apartment| +47351|AAAAAAAAHPILAAAA|745|Wilson Miller|Ct.|Suite D|Riverdale|Sharkey County|MS|59391|United States|-6|apartment| +47352|AAAAAAAAIPILAAAA|876|Laurel |Street|Suite 110|Five Forks|Montmorency County|MI|42293|United States|-5|apartment| +47353|AAAAAAAAJPILAAAA|953|Oak 1st|Ln|Suite N|Amity|Bell County|TX|70766|United States|-6|apartment| +47354|AAAAAAAAKPILAAAA|761|Highland Sycamore|Wy|Suite N|Florence|Sequoyah County|OK|73394|United States|-6|single family| +47355|AAAAAAAALPILAAAA|513|Forest |Pkwy|Suite 350|Mount Pleasant|Chesterfield County|VA|21933|United States|-5|apartment| +47356|AAAAAAAAMPILAAAA|808|Park Broadway|Pkwy|Suite 170|Belmont|Chippewa County|MN|50191|United States|-6|apartment| +47357|AAAAAAAANPILAAAA|337|11th |Lane|Suite X|Walnut Grove|Dodge County|GA|37752|United States|-5|apartment| +47358|AAAAAAAAOPILAAAA|439|1st Church|Ct.|Suite 380|Mount Zion|Montcalm County|MI|48054|United States|-5|condo| +47359|AAAAAAAAPPILAAAA|46|Pine |Ave|Suite A|Pleasant Grove|Raleigh County|WV|24136|United States|-5|condo| +47360|AAAAAAAAAAJLAAAA|331|Broadway |Court|Suite B|White Oak|York County|VA|26668|United States|-5|condo| +47361|AAAAAAAABAJLAAAA|582|View |Dr.|Suite E|Fairfield|Cabarrus County|NC|26192|United States|-5|single family| +47362|AAAAAAAACAJLAAAA|256|6th Poplar|Drive|Suite A|Taft|Donley County|TX|70589|United States|-6|condo| +47363|AAAAAAAADAJLAAAA|932|Woodland |||Sunnyside|||91952|||| +47364|AAAAAAAAEAJLAAAA|910|Woodland Johnson|ST|Suite P|Centerville|Madison County|IA|50059|United States|-6|condo| +47365|AAAAAAAAFAJLAAAA|750|First |Blvd|Suite V|Newtown|Churchill County|NV|81749|United States|-8|single family| +47366|AAAAAAAAGAJLAAAA|943|Adams |Lane|Suite 170|Hartland|Scott County|IA|56594|United States|-6|single family| +47367|AAAAAAAAHAJLAAAA|661|Meadow Walnut|Pkwy|Suite A|Enterprise|Letcher County|KY|41757|United States|-5|condo| +47368|AAAAAAAAIAJLAAAA|272|Maple |Blvd|Suite K|Hillcrest|Seminole County|FL|33003|United States|-5|condo| +47369|AAAAAAAAJAJLAAAA|827|Johnson |Boulevard|Suite 240|Clifton|Milwaukee County|WI|58014|United States|-6|apartment| +47370|AAAAAAAAKAJLAAAA|16|Walnut |Wy|Suite G|Union|Hartford County|CT|09321|United States|-5|condo| +47371|AAAAAAAALAJLAAAA|905|Railroad Fourth|Parkway|Suite T|Farmington|Bryan County|OK|79145|United States|-6|single family| +47372|AAAAAAAAMAJLAAAA|302|Woodland Johnson|Circle|Suite E|Springfield|Spotsylvania County|VA|29303|United States|-5|apartment| +47373|AAAAAAAANAJLAAAA|959|10th Church|Street|Suite K|Cedar Grove|Sussex County|DE|10411|United States|-5|apartment| +47374|AAAAAAAAOAJLAAAA|713|Cedar Oak|Circle|Suite 340|Flint|Wood County|WI|58909|United States|-6|apartment| +47375|AAAAAAAAPAJLAAAA|902|Chestnut |Road|Suite 270|Sunnyside|Paulding County|OH|41952|United States|-5|apartment| +47376|AAAAAAAAABJLAAAA|918|1st Hill|Circle|Suite K|Woodland|Nye County|NV|84854|United States|-8|apartment| +47377|AAAAAAAABBJLAAAA|712|Dogwood |Avenue|Suite U|Greenwood|Atoka County|OK|78828|United States|-6|single family| +47378|AAAAAAAACBJLAAAA|558|Lee |Parkway|Suite 0|Montpelier|Montgomery County|MD|28930|United States|-5|condo| +47379|AAAAAAAADBJLAAAA|1|Spruce 8th|Street|Suite 160|Macedonia|Wayne County|KY|41087|United States|-5|condo| +47380|AAAAAAAAEBJLAAAA|528|Miller Green|Way|Suite 430|Oak Hill|Johnson County|GA|37838|United States|-5|condo| +47381|AAAAAAAAFBJLAAAA|947|Eigth |Boulevard|Suite 100|Lakeside|Kleberg County|TX|79532|United States|-6|apartment| +47382|AAAAAAAAGBJLAAAA|801|6th |Circle|Suite E|Bethel|Asotin County|WA|95281|United States|-8|condo| +47383|AAAAAAAAHBJLAAAA|20|Birch |Avenue|Suite 440|Frenchtown|McCormick County|SC|22629|United States|-5|condo| +47384|AAAAAAAAIBJLAAAA|650|6th Ridge|Street|Suite N|Bethel|Terrell County|TX|75281|United States|-6|condo| +47385|AAAAAAAAJBJLAAAA|582|Cedar Maple|Way|Suite 230|Ludlow|Park County|WY|85566|United States|-7|apartment| +47386|AAAAAAAAKBJLAAAA|256|Locust |Pkwy|Suite 230|Glenwood|Washington County|TN|33511|United States|-6|condo| +47387|AAAAAAAALBJLAAAA|668|Franklin Williams|Blvd|Suite 420|Hamilton|El Paso County|CO|82808|United States|-7|condo| +47388|AAAAAAAAMBJLAAAA|||Blvd|||||48222|United States|-5|single family| +47389|AAAAAAAANBJLAAAA|999|Elm Miller|ST|Suite 340|Newport|Howard County|NE|61521|United States|-7|apartment| +47390|AAAAAAAAOBJLAAAA|594|Laurel South|Lane|Suite E|Lakewood|Mellette County|SD|58877|United States|-7|single family| +47391|AAAAAAAAPBJLAAAA|840|Sunset |Ct.|Suite 220|Spring Hill||TX||United States|-6|apartment| +47392|AAAAAAAAACJLAAAA|793|8th |Parkway|Suite F|White Oak|Hillsborough County|NH|07268|United States|-5|condo| +47393|AAAAAAAABCJLAAAA|182|Miller West|Way|Suite V|Kimball|Milam County|TX|73595|United States|-6|apartment| +47394|AAAAAAAACCJLAAAA|359|Ridge Hill|Dr.|Suite 350|Glenwood|Missoula County|MT|63511|United States|-7|condo| +47395|AAAAAAAADCJLAAAA|266|Maple Pine|Avenue|Suite Y|Kingston|Pembina County|ND|54975|United States|-6|condo| +47396|AAAAAAAAECJLAAAA|771|Willow |Ave|Suite 110|Sulphur Springs|Sequatchie County|TN|38354|United States|-6|apartment| +47397|AAAAAAAAFCJLAAAA|934|Spring |Parkway|Suite O|Oakland|Oldham County|TX|79843|United States|-6|condo| +47398|AAAAAAAAGCJLAAAA|953|Birch |Ln|Suite 420|Bridgeport|Scott County|TN|35817|United States|-6|single family| +47399|AAAAAAAAHCJLAAAA|717|7th Jackson|Road|Suite 300|Belmont|Montrose County|CO|80191|United States|-7|apartment| +47400|AAAAAAAAICJLAAAA|674|Woodland |Avenue|Suite K|Spring Hill|Effingham County|IL|66787|United States|-6|single family| +47401|AAAAAAAAJCJLAAAA|455|Chestnut 5th|Lane|Suite 100|Highland|Harney County|OR|99454|United States|-8|single family| +47402|AAAAAAAAKCJLAAAA|567|Forest |Parkway|Suite K|Mount Zion|Tooele County|UT|88054|United States|-7|condo| +47403|AAAAAAAALCJLAAAA|805||Blvd||Glendale||MN||||apartment| +47404|AAAAAAAAMCJLAAAA|847|1st Center|Ln|Suite K|Highland|Marion County|IL|69454|United States|-6|condo| +47405|AAAAAAAANCJLAAAA|741|Elm |Drive|Suite 190|Woodland|Custer County|CO|84854|United States|-7|condo| +47406|AAAAAAAAOCJLAAAA|688|Cherry Dogwood|Circle|Suite P|New Hope|Williamson County|IL|69431|United States|-6|condo| +47407|AAAAAAAAPCJLAAAA|328|1st |Ave|Suite 410|Highland Park|Green County|KY|46534|United States|-6|single family| +47408|AAAAAAAAADJLAAAA|891|11th |Avenue|Suite 450|Florence|Pitkin County|CO|83394|United States|-7|condo| +47409|AAAAAAAABDJLAAAA|606|Seventh ||||||45752||-5|apartment| +47410|AAAAAAAACDJLAAAA|225|View |Avenue|Suite W|Belmont|Kings County|CA|90191|United States|-8|condo| +47411|AAAAAAAADDJLAAAA|92|Fifth |ST|Suite 230|Midway|Bethel Census Area|AK|91904|United States|-9|condo| +47412|AAAAAAAAEDJLAAAA|463|Fifth Poplar|Circle|Suite Q|Walnut Grove|Menifee County|KY|47752|United States|-5|condo| +47413|AAAAAAAAFDJLAAAA|609|Eigth Chestnut|Road|Suite 200|Lakeview|Lee County|GA|38579|United States|-5|apartment| +47414|AAAAAAAAGDJLAAAA|381|7th Hickory|Drive|Suite 380|Greenwood|Bulloch County|GA|38828|United States|-5|apartment| +47415|AAAAAAAAHDJLAAAA|403|Elevnth Maple|Court|Suite S|Lebanon|Pittsburg County|OK|72898|United States|-6|condo| +47416|AAAAAAAAIDJLAAAA|480|Highland |Ct.|Suite 470|Lakewood|Chaffee County|CO|88877|United States|-7|apartment| +47417|AAAAAAAAJDJLAAAA|342|Main Jackson|Ct.|Suite K|Highland|Winona County|MN|59454|United States|-6|apartment| +47418|AAAAAAAAKDJLAAAA|955|Valley |Court|Suite 390|Five Points|Lemhi County|ID|86098|United States|-7|apartment| +47419|AAAAAAAALDJLAAAA|378|Elm Sycamore|Street|Suite J|Shady Grove|Monroe County|IN|42812|United States|-5|apartment| +47420|AAAAAAAAMDJLAAAA|827|Mill |Ct.|Suite 200|Glenwood|Grant County|ND|53511|United States|-6|apartment| +47421|AAAAAAAANDJLAAAA|675|15th |Court|Suite 90|Pleasant Hill|Harrison County|OH|43604|United States|-5|condo| +47422|AAAAAAAAODJLAAAA|686|Church |Street|Suite R|Clifton|Donley County|TX|78014|United States|-6|condo| +47423|AAAAAAAAPDJLAAAA|504|Sunset Valley|RD|Suite I|Guthrie|Bremer County|IA|51423|United States|-6|condo| +47424|AAAAAAAAAEJLAAAA|128|Lake Highland|Pkwy|Suite 460|Shelby|Ozaukee County|WI|56575|United States|-6|apartment| +47425|AAAAAAAABEJLAAAA|549|Woodland Green|ST|Suite H|Lakewood|Barrow County|GA|38877|United States|-5|single family| +47426|AAAAAAAACEJLAAAA|191|3rd 10th|Ln|Suite F|Walnut Grove|Shannon County|MO|67752|United States|-6|condo| +47427|AAAAAAAADEJLAAAA|958|Park |Blvd|Suite I|Greenwood|Lee County|IL|68828|United States|-6|apartment| +47428|AAAAAAAAEEJLAAAA|483|Madison |Avenue|Suite K|Belmont|Cecil County|MD|20191|United States|-5|single family| +47429|AAAAAAAAFEJLAAAA|125|Park 8th|Lane|Suite 80|Newtown|Newton County|TX|71749|United States|-6|condo| +47430|AAAAAAAAGEJLAAAA|440|Mill 7th|Ct.|Suite N|Mountain View|Barron County|WI|54466|United States|-6|apartment| +47431|AAAAAAAAHEJLAAAA|312|Eigth |Street|Suite 480|Plainview|Pittsylvania County|VA|23683|United States|-5|condo| +47432|AAAAAAAAIEJLAAAA|23|Ridge |Dr.|Suite 90|Acme|Clinton County|NY|10164|United States|-5|condo| +47433|AAAAAAAAJEJLAAAA|292|11th First|Ave|Suite 20|Union|Mecosta County|MI|48721|United States|-5|apartment| +47434|AAAAAAAAKEJLAAAA|902|Madison Church|Lane|Suite N|Five Forks|Knox County|TN|32293|United States|-6|single family| +47435|AAAAAAAALEJLAAAA|998|13th |Ave|Suite K|Greenwood|Williamson County|TX|78828|United States|-6|condo| +47436|AAAAAAAAMEJLAAAA|91|Cherry Main|Blvd|Suite 70|Clearwater|Douglas County|WI|59534|United States|-6|apartment| +47437|AAAAAAAANEJLAAAA|31|Lee Mill|Avenue|Suite 420|Red Hill|Freestone County|TX|74338|United States|-6|single family| +47438|AAAAAAAAOEJLAAAA|400|College 15th|Circle|Suite J|Summit|Bedford County|PA|10499|United States|-5|single family| +47439|AAAAAAAAPEJLAAAA|447|Hill 1st|Dr.|Suite M|Crossroads|Gaines County|TX|70534|United States|-6|apartment| +47440|AAAAAAAAAFJLAAAA|515|6th West|Lane|Suite R|Marion|Roger Mills County|OK|70399|United States|-6|condo| +47441|AAAAAAAABFJLAAAA|994|Lincoln Forest|Ct.|Suite 280|Highland Park|Story County|IA|56534|United States|-6|apartment| +47442|AAAAAAAACFJLAAAA|517|Third |Street|Suite 200|Parkwood|Island County|WA|91669|United States|-8|single family| +47443|AAAAAAAADFJLAAAA|80|Madison |Ln|Suite 220|Lakewood|Valley County|NE|68877|United States|-7|apartment| +47444|AAAAAAAAEFJLAAAA|149|Smith Valley|Lane|Suite 60|Hillcrest|Wilcox County|AL|33003|United States|-6|condo| +47445|AAAAAAAAFFJLAAAA|294|Franklin |Drive|Suite 10|Maple Hill|Richmond city|VA|28095|United States|-5|condo| +47446|AAAAAAAAGFJLAAAA|904|Meadow Pine|Drive|Suite H|Green Acres|Dallam County|TX|77683|United States|-6|condo| +47447|AAAAAAAAHFJLAAAA|120|Twelfth 9th|Cir.|Suite 340|Salem|Murray County|GA|38048|United States|-5|condo| +47448|AAAAAAAAIFJLAAAA|650|5th Laurel|Ln|Suite 30|Mountain View|Madison County|IL|64466|United States|-6|single family| +47449|AAAAAAAAJFJLAAAA|475|Franklin Johnson|Street|Suite 270|Shiloh|Blaine County|NE|69275|United States|-6|single family| +47450|AAAAAAAAKFJLAAAA|312|Central |Ave|Suite S|Hamilton|Marion County|WV|22808|United States|-5|condo| +47451|AAAAAAAALFJLAAAA|53|Fourth |ST|Suite 260|Highland|Uvalde County|TX|79454|United States|-6|single family| +47452|AAAAAAAAMFJLAAAA|492|Birch |Avenue|Suite 370|Fairfield|Vanderburgh County|IN|46192|United States|-5|single family| +47453|AAAAAAAANFJLAAAA|510|9th |Cir.|Suite S|Sulphur Springs|Polk County|WI|58354|United States|-6|condo| +47454|AAAAAAAAOFJLAAAA|858|Jefferson Oak|Ct.|Suite O|Valley View|Monroe County|IL|65124|United States|-6|single family| +47455|AAAAAAAAPFJLAAAA|684|Wilson |ST|Suite 90|Oak Ridge|Carroll County|GA|38371|United States|-5|apartment| +47456|AAAAAAAAAGJLAAAA|668|5th Valley|Dr.|Suite S|Florence|Madison County|KY|43394|United States|-5|apartment| +47457|AAAAAAAABGJLAAAA|501|Eigth Tenth|Ct.|||Franklin County||86668|United States||condo| +47458|AAAAAAAACGJLAAAA|889|Third |Street|Suite Q|Bethel|San Joaquin County|CA|95281|United States|-8|single family| +47459|AAAAAAAADGJLAAAA|346|Fourth |Cir.|Suite 290|Greenwood|Tipton County|IN|48828|United States|-5|single family| +47460|AAAAAAAAEGJLAAAA|441|Broadway View|Cir.|Suite 210|Springfield|Bacon County|GA|39303|United States|-5|single family| +47461|AAAAAAAAFGJLAAAA|910|Park Pine|Court|Suite I|Shiloh|Chesapeake city|VA|29275|United States|-5|apartment| +47462|AAAAAAAAGGJLAAAA|438|Park 1st|Wy|Suite 390|Green Acres|Chatham County|NC|27683|United States|-5|condo| +47463|AAAAAAAAHGJLAAAA|529|Oak |Blvd|Suite 100|Mountain View|Anson County|NC|24466|United States|-5|apartment| +47464|AAAAAAAAIGJLAAAA|817|1st |Wy|Suite 170|Mount Olive|Crook County|OR|98059|United States|-8|single family| +47465|AAAAAAAAJGJLAAAA|929|Locust 3rd|Street|Suite P|Springdale|Lee County|NC|28883|United States|-5|single family| +47466|AAAAAAAAKGJLAAAA|196|Fourth Spruce|Drive|Suite V|Hillcrest|Ozaukee County|WI|53003|United States|-6|apartment| +47467|AAAAAAAALGJLAAAA|371|Main View|Street|Suite K|Greenwood|Esmeralda County|NV|88828|United States|-8|single family| +47468|AAAAAAAAMGJLAAAA|728|Lincoln |Parkway|Suite 120|Sawyer|Fulton County|AR|76045|United States|-6|condo| +47469|AAAAAAAANGJLAAAA|83|Jackson North|Avenue|Suite L|Stringtown|Muskegon County|MI|40162|United States|-5|apartment| +47470|AAAAAAAAOGJLAAAA|650|Hill Laurel|Cir.|Suite J|Greenwood|Beadle County|SD|58828|United States|-6|single family| +47471|AAAAAAAAPGJLAAAA|917|6th Cherry|Wy|Suite 350|Woodlawn|Wilbarger County|TX|74098|United States|-6|condo| +47472|AAAAAAAAAHJLAAAA|455|Ridge 2nd|Way|Suite T|Sunnyside|Ector County|TX|71952|United States|-6|condo| +47473|AAAAAAAABHJLAAAA|967|Ninth Hill|Avenue|Suite E|Kingston|Hampton city|VA|24975|United States|-5|single family| +47474|AAAAAAAACHJLAAAA|772|Main Davis|Lane|Suite N|Glenwood|Madison County|MO|63511|United States|-6|single family| +47475|AAAAAAAADHJLAAAA|748|Spring |Ct.|Suite 400|Green Acres|Florence County|WI|57683|United States|-6|condo| +47476|AAAAAAAAEHJLAAAA|344|Adams |Parkway|Suite 0|Five Forks|Johnson County|WY|82293|United States|-7|condo| +47477|AAAAAAAAFHJLAAAA|945|Cedar |Pkwy|Suite 400|Gary|Jackson County|MO|60418|United States|-6|apartment| +47478|AAAAAAAAGHJLAAAA|164|Ridge Church|Ln|Suite 320|Buena Vista|Marshall County|MS|55752|United States|-6|condo| +47479|AAAAAAAAHHJLAAAA|739|Main Lake|Court|Suite G|Red Bank|Dale County|AL|34975|United States|-6|condo| +47480|AAAAAAAAIHJLAAAA|949|Lake ||Suite X|Forest Hills||IN|49237|United States||condo| +47481|AAAAAAAAJHJLAAAA|731|Park |Circle|Suite N|Macedonia|Blaine County|NE|61087|United States|-6|single family| +47482|AAAAAAAAKHJLAAAA|401|3rd Smith|Drive|Suite 480|Mount Olive|Manatee County|FL|38059|United States|-5|condo| +47483|AAAAAAAALHJLAAAA|104|Adams Mill|Drive|Suite 160|Greenville|Morgan County|TN|31387|United States|-6|condo| +47484|AAAAAAAAMHJLAAAA|759|Elm |Drive|Suite 230|Woodland|Harrison County|KY|44854|United States|-6|single family| +47485|AAAAAAAANHJLAAAA|659|Walnut 6th|ST|Suite 200|Springdale|Scott County|MO|68883|United States|-6|condo| +47486|AAAAAAAAOHJLAAAA|847|5th |Cir.|Suite 470|Mount Olive|Venango County|PA|18059|United States|-5|apartment| +47487|AAAAAAAAPHJLAAAA|776|Hill |RD|Suite N|Ashland|Miller County|GA|34244|United States|-5|condo| +47488|AAAAAAAAAIJLAAAA|343|Madison Locust|Drive|Suite U|Pleasant Hill|Brooks County|GA|33604|United States|-5|condo| +47489|AAAAAAAABIJLAAAA|826|Park |Dr.|Suite U|Newtown|Chelan County|WA|91749|United States|-8|single family| +47490|AAAAAAAACIJLAAAA|160|Walnut 4th|ST|Suite E|Blanchard|Noble County|IN|45985|United States|-5|condo| +47491|AAAAAAAADIJLAAAA|377|Spring Maple|Way|Suite 30|Siloam|Kingsbury County|SD|58948|United States|-7|condo| +47492|AAAAAAAAEIJLAAAA|845|8th Madison|Ave|Suite U|Jamestown|Lafayette County|WI|56867|United States|-6|single family| +47493|AAAAAAAAFIJLAAAA|152|Highland |Wy|Suite M|Reno|Perquimans County|NC|20344|United States|-5|single family| +47494|AAAAAAAAGIJLAAAA|153|1st Second|Court|Suite 40|Mount Pleasant|Aiken County|SC|21933|United States|-5|condo| +47495|AAAAAAAAHIJLAAAA|178|Johnson Hillcrest|Wy|Suite 170|Oakdale|Ringgold County|IA|59584|United States|-6|condo| +47496|AAAAAAAAIIJLAAAA|907|7th Maple|Drive|Suite 310|Salem|Benton County|TN|38048|United States|-5|single family| +47497|AAAAAAAAJIJLAAAA|295|11th |Dr.|Suite W|White Oak|Wright County|MN|56668|United States|-6|apartment| +47498|AAAAAAAAKIJLAAAA|131|Cherry Railroad|Street|Suite 420|Oak Hill|Jefferson County|FL|37838|United States|-5|apartment| +47499|AAAAAAAALIJLAAAA|129|Lake |Way|Suite 210|Five Points|Lincoln County|WV|26098|United States|-5|condo| +47500|AAAAAAAAMIJLAAAA|303|Sycamore |Dr.|Suite 130|Wildwood|Jay County|IN|46871|United States|-5|single family| +47501|AAAAAAAANIJLAAAA|841|Franklin |Ave|Suite O|Enterprise|Clinton County|IA|51757|United States|-6|single family| +47502|AAAAAAAAOIJLAAAA|||Ave||Antioch||||||| +47503|AAAAAAAAPIJLAAAA|405|Main Hickory|Blvd|Suite 410|Mount Pleasant|Lake County|MI|41933|United States|-5|apartment| +47504|AAAAAAAAAJJLAAAA|338|North |Ln|Suite 300|Pleasant Valley|Fannin County|TX|72477|United States|-6|single family| +47505|AAAAAAAABJJLAAAA|257|Walnut |Ave|Suite 170|Franklin|Johnston County|NC|29101|United States|-5|condo| +47506|AAAAAAAACJJLAAAA|100|4th First|Parkway|Suite 90|Greenville|Lincoln County|NE|61387|United States|-7|condo| +47507|AAAAAAAADJJLAAAA|||Ave|Suite 320|Lakeside||||United States|-6|| +47508|AAAAAAAAEJJLAAAA|730|Main Fifth|Ave|Suite 10|Shiloh|Santa Fe County|NM|89275|United States|-7|condo| +47509|AAAAAAAAFJJLAAAA|564|5th North|Circle|Suite 220|Arlington|Lubbock County|TX|76557|United States|-6|single family| +47510|AAAAAAAAGJJLAAAA|45|Sixth |ST|Suite C|Plainview|Wilkin County|MN|53683|United States|-6|single family| +47511|AAAAAAAAHJJLAAAA|289|Chestnut 1st|Avenue|Suite 40|Hopewell|Hampton County|SC|20587|United States|-5|condo| +47512|AAAAAAAAIJJLAAAA|905|Fifth North|Wy|Suite S|Highland|Steele County|MN|59454|United States|-6|single family| +47513|AAAAAAAAJJJLAAAA|388|7th |Court|Suite 290|Chestnut Ridge|Taylor County|IA|57334|United States|-6|single family| +47514|AAAAAAAAKJJLAAAA|420|Park |Wy|Suite 440|Brownsville|Dorchester County|SC|29310|United States|-5|apartment| +47515|AAAAAAAALJJLAAAA|588|2nd Johnson|Lane|Suite K|Springfield|Knott County|KY|49303|United States|-5|single family| +47516|AAAAAAAAMJJLAAAA|334|Willow |Blvd|Suite 220|Lebanon|Potter County|PA|12898|United States|-5|apartment| +47517|AAAAAAAANJJLAAAA|86|Eigth Franklin|Circle|Suite E|Hillcrest|Eagle County|CO|83003|United States|-7|single family| +47518|AAAAAAAAOJJLAAAA|801|Spring Cedar|Avenue|Suite 180|Highland|Douglas County|CO|89454|United States|-7|apartment| +47519|AAAAAAAAPJJLAAAA|977|2nd Forest||Suite V||||80411|United States|-7|| +47520|AAAAAAAAAKJLAAAA|754|6th |Circle|Suite J|Cherry Valley|Marin County|CA|90854|United States|-8|apartment| +47521|AAAAAAAABKJLAAAA|441|Birch |Way|Suite 280|Highland Park|Lowndes County|GA|36534|United States|-5|apartment| +47522|AAAAAAAACKJLAAAA|790|Madison 5th|Drive|Suite 440|Oak Grove|Morrow County|OH|48370|United States|-5|condo| +47523|AAAAAAAADKJLAAAA|760|Hill |Road|Suite 420|Belmont|Gibson County|IN|40191|United States|-5|single family| +47524|AAAAAAAAEKJLAAAA|985|Willow Oak|ST|Suite A|Mount Olive|Doddridge County|WV|28059|United States|-5|condo| +47525|AAAAAAAAFKJLAAAA|433|Poplar |Drive|Suite Q|Springfield|Union Parish|LA|79303|United States|-6|single family| +47526|AAAAAAAAGKJLAAAA|578|12th 7th|Wy|Suite 150|Buena Vista|Payette County|ID|85752|United States|-7|condo| +47527|AAAAAAAAHKJLAAAA|409|Washington 3rd|Circle|Suite D|Pine Grove|Carroll County|KY|44593|United States|-6|apartment| +47528|AAAAAAAAIKJLAAAA|841|Johnson |Cir.|Suite V|Clinton|Langlade County|WI|58222|United States|-6|apartment| +47529|AAAAAAAAJKJLAAAA|365|Church |Ct.|Suite T|Concord|Jackson County|WI|54107|United States|-6|apartment| +47530|AAAAAAAAKKJLAAAA|13|Franklin 1st|Way|Suite 320|Florence|Clay County|NE|63394|United States|-6|apartment| +47531|AAAAAAAALKJLAAAA|409|Laurel Dogwood|ST|Suite 230|Liberty|Preble County|OH|43451|United States|-5|apartment| +47532|AAAAAAAAMKJLAAAA|283|Third |Ln|Suite 280|Oak Grove|Musselshell County|MT|68370|United States|-7|apartment| +47533|AAAAAAAANKJLAAAA|||ST||Pleasant Valley|Jefferson County|WI|52477|United States|-6|| +47534|AAAAAAAAOKJLAAAA|948|Lake |Pkwy|Suite 60|Highland|Lincoln County|KY|49454|United States|-5|single family| +47535|AAAAAAAAPKJLAAAA|230|Pine |Avenue|Suite E|Providence|De Witt County|IL|66614|United States|-6|condo| +47536|AAAAAAAAALJLAAAA|215|9th |Road|Suite A|Plainview|Franklin County|AL|33683|United States|-6|condo| +47537|AAAAAAAABLJLAAAA|798|Meadow Birch|Boulevard|Suite 220|Sulphur Springs|Crowley County|CO|88354|United States|-7|condo| +47538|AAAAAAAACLJLAAAA|695|Ridge Fourth|Way|Suite Y|Mountain View|Newaygo County|MI|44466|United States|-5|condo| +47539|AAAAAAAADLJLAAAA|169|Park |Pkwy|Suite 370|Riverside|Ashland County|OH|49231|United States|-5|condo| +47540|AAAAAAAAELJLAAAA|191|8th Jackson|Blvd|Suite W|Clinton|Holt County|NE|68222|United States|-7|apartment| +47541|AAAAAAAAFLJLAAAA|476|Adams Railroad|Lane|Suite M|Glenwood|Montgomery County|AL|33511|United States|-6|single family| +47542|AAAAAAAAGLJLAAAA|729|12th 4th|Circle|Suite C|Pleasant Grove|Camden County|NC|24136|United States|-5|apartment| +47543|AAAAAAAAHLJLAAAA|28|Seventh Green|Dr.|Suite 200|Hamilton|Hall County|GA|32808|United States|-5|single family| +47544|AAAAAAAAILJLAAAA|602|15th 11th|Avenue|Suite 0|Mount Zion|Loudoun County|VA|28054|United States|-5|single family| +47545|AAAAAAAAJLJLAAAA|144|Spruce Jefferson|Ave|Suite J|Riverdale|Clay County|KY|49391|United States|-6|condo| +47546|AAAAAAAAKLJLAAAA|616|Wilson |Dr.|Suite F|White Oak|Orange County|IN|46668|United States|-5|single family| +47547|AAAAAAAALLJLAAAA|309|Willow |Lane|Suite W|Hamilton|Winchester city|VA|22808|United States|-5|condo| +47548|AAAAAAAAMLJLAAAA|344|Pine |Wy|Suite 430|Spring Valley|Citrus County|FL|36060|United States|-5|condo| +47549|AAAAAAAANLJLAAAA|944|Ash Oak|Ln|Suite X|Enterprise|Nowata County|OK|71757|United States|-6|apartment| +47550|AAAAAAAAOLJLAAAA|544|Second First|RD|Suite M|Pleasant Hill|Clackamas County|OR|93604|United States|-8|single family| +47551|AAAAAAAAPLJLAAAA|550|Maple |Way|Suite 310|Highland|Henry County|AL|39454|United States|-6|single family| +47552|AAAAAAAAAMJLAAAA|5|Hill Cedar|Road|Suite 390|Walnut Grove|Dale County|AL|37752|United States|-6|apartment| +47553|AAAAAAAABMJLAAAA|713|North Ash|Way|Suite 110|Deerfield|Taney County|MO|69840|United States|-6|apartment| +47554|AAAAAAAACMJLAAAA|992|Park Fourth|Wy|Suite P|Forestville|Colorado County|TX|73027|United States|-6|single family| +47555|AAAAAAAADMJLAAAA|568|Davis |Wy|Suite S|Concord|Columbia County|FL|34107|United States|-5|single family| +47556|AAAAAAAAEMJLAAAA|84|1st |RD|Suite 100|Lebanon|Monroe County|TN|32898|United States|-6|apartment| +47557|AAAAAAAAFMJLAAAA|559|Elm Lake|Court|Suite 360|Riverdale|Jefferson County|TX|79391|United States|-6|condo| +47558|AAAAAAAAGMJLAAAA|771|Main |Blvd|Suite D|Union Hill|Pipestone County|MN|57746|United States|-6|single family| +47559|AAAAAAAAHMJLAAAA|436|Church Dogwood|Circle|Suite 0|Waterloo|Morris County|KS|61675|United States|-6|apartment| +47560|AAAAAAAAIMJLAAAA|69|Ash 8th|Drive|Suite W|Maple Grove|Hampton city|VA|28252|United States|-5|single family| +47561|AAAAAAAAJMJLAAAA|623|Hickory |Court|Suite K|Bunker Hill|Hamblen County|TN|30150|United States|-5|condo| +47562|AAAAAAAAKMJLAAAA|918|Woodland |Pkwy|Suite 50|Clinton|Morgan County|AL|38222|United States|-6|condo| +47563|AAAAAAAALMJLAAAA|307|Lake |Street|Suite I|Kent|Dawson County|MT|60317|United States|-7|apartment| +47564|AAAAAAAAMMJLAAAA|545|7th |Circle|Suite 320|White Oak|Martin County|MN|56668|United States|-6|single family| +47565|AAAAAAAANMJLAAAA|640|8th Ridge|Circle|Suite T|Union Hill|Windham County|VT|08346|United States|-5|single family| +47566|AAAAAAAAOMJLAAAA|751|1st |Wy|Suite 440|Union Hill|Hamilton County|IL|67746|United States|-6|single family| +47567|AAAAAAAAPMJLAAAA|99|Birch Jackson|Dr.|Suite 440|Sulphur Springs|Lancaster County|NE|68354|United States|-7|single family| +47568|AAAAAAAAANJLAAAA|173|Walnut Elm|RD|Suite V|Union|Issaquena County|MS|58721|United States|-6|condo| +47569|AAAAAAAABNJLAAAA|730|Park |Cir.|Suite E|Buena Vista|Union County|MS|55752|United States|-6|apartment| +47570|AAAAAAAACNJLAAAA|300|12th |Ct.|Suite 50|Mount Vernon|Morgan County|MO|68482|United States|-6|condo| +47571|AAAAAAAADNJLAAAA|703|Lincoln 11th|Ave|Suite W|Bunker Hill|Midland County|MI|40150|United States|-5|apartment| +47572|AAAAAAAAENJLAAAA|968|Smith |Street|Suite 370|Mountain View|Barren County|KY|44466|United States|-6|apartment| +47573|AAAAAAAAFNJLAAAA|697|Park Church|Avenue|Suite A|Antioch|Ionia County|MI|48605|United States|-5|condo| +47574|AAAAAAAAGNJLAAAA|448|Mill Lake|Way|Suite 160|Sulphur Springs|Highland County|VA|28354|United States|-5|condo| +47575|AAAAAAAAHNJLAAAA|192|4th |Dr.|Suite C|Star|Allendale County|SC|20725|United States|-5|single family| +47576|AAAAAAAAINJLAAAA|488|Broadway Lakeview|Ct.|Suite R|Glendale|McKenzie County|ND|53951|United States|-6|apartment| +47577|AAAAAAAAJNJLAAAA|56|Laurel 4th|Road|Suite H|Newtown|Madison County|NC|21749|United States|-5|single family| +47578|AAAAAAAAKNJLAAAA|991|10th 7th|Dr.|Suite 320|Oakdale|Winnebago County|IL|69584|United States|-6|single family| +47579|AAAAAAAALNJLAAAA|485|Seventh |Court|Suite 280|Enterprise|Nueces County|TX|71757|United States|-6|condo| +47580|AAAAAAAAMNJLAAAA|611|Center |Lane|Suite 140|Greenville|Shelby County|MO|61387|United States|-6|apartment| +47581|AAAAAAAANNJLAAAA|246|Franklin Oak|Ave|Suite 230|Belmont|Cleveland County|AR|70191|United States|-6|single family| +47582|AAAAAAAAONJLAAAA|222|2nd |Road|Suite 150|Spring Valley|Baker County|GA|36060|United States|-5|single family| +47583|AAAAAAAAPNJLAAAA|118|4th |Lane|Suite 150|Shannon|Benton County|MS|54120|United States|-6|condo| +47584|AAAAAAAAAOJLAAAA|940|Eigth Meadow|Blvd|Suite 30|Glendale|Price County|WI|53951|United States|-6|condo| +47585|AAAAAAAABOJLAAAA|640|Dogwood |Circle|Suite H|Blanchard|Bourbon County|KY|45985|United States|-6|condo| +47586|AAAAAAAACOJLAAAA|811|Walnut |Court|Suite 320|Glendale|Camden County|MO|63951|United States|-6|apartment| +47587|AAAAAAAADOJLAAAA|56|Main View|Way|Suite 190|Bennett|Clinton County|IL|61715|United States|-6|condo| +47588|AAAAAAAAEOJLAAAA|285|10th |Ave|Suite 60|Arlington|Hocking County|OH|46557|United States|-5|single family| +47589|AAAAAAAAFOJLAAAA|449|Second Second|Street|Suite W|Bridgeport|Runnels County|TX|75817|United States|-6|condo| +47590|AAAAAAAAGOJLAAAA|382|Mill |Lane|Suite 150|Belmont|Worth County|GA|30191|United States|-5|apartment| +47591|AAAAAAAAHOJLAAAA|332|Jackson Spruce|Court|Suite W|Greenville|Riley County|KS|61387|United States|-6|condo| +47592|AAAAAAAAIOJLAAAA|125|West |Dr.|Suite 150|Spring Valley|Hooker County|NE|66060|United States|-7|apartment| +47593|AAAAAAAAJOJLAAAA|344|Maple |Ct.|Suite 210|Kingston|King William County|VA|24975|United States|-5|condo| +47594|AAAAAAAAKOJLAAAA|11|Maple Center|Dr.|Suite B|Lakewood|Henry County|TN|38877|United States|-5|single family| +47595|AAAAAAAALOJLAAAA|868|Birch |Pkwy|Suite 0|Spring Valley|Cumberland County|KY|46060|United States|-6|condo| +47596|AAAAAAAAMOJLAAAA|18|Smith 6th|Drive|Suite I|Cedar|Vermilion County|IL|61229|United States|-6|single family| +47597|AAAAAAAANOJLAAAA|284|Park |Dr.|Suite 490|Pleasant Valley|Campbell County|VA|22477|United States|-5|single family| +47598|AAAAAAAAOOJLAAAA|551|Sunset 9th|Pkwy|Suite W|Shiloh|Wilson County|TX|79275|United States|-6|single family| +47599|AAAAAAAAPOJLAAAA|308|Church Dogwood|Boulevard|Suite Q|Valley View|Storey County|NV|85124|United States|-8|single family| +47600|AAAAAAAAAPJLAAAA|812|Sixth |Ln|Suite K|Woodland|Door County|WI|54854|United States|-6|condo| +47601|AAAAAAAABPJLAAAA|791|Pine |Pkwy|Suite 300|Farmington|Columbia County|PA|19145|United States|-5|apartment| +47602|AAAAAAAACPJLAAAA|89|1st Hill|Lane|Suite 450|Glendale|Brown County|TX|73951|United States|-6|apartment| +47603|AAAAAAAADPJLAAAA|343|3rd |Lane|Suite 160|Montague|Redwood County|MN|54062|United States|-6|apartment| +47604|AAAAAAAAEPJLAAAA|75|Hill |Blvd|Suite X|Hamilton|Lewis County|WV|22808|United States|-5|apartment| +47605|AAAAAAAAFPJLAAAA|41|Lee 5th|Boulevard|Suite M|Newtown|Navarro County|TX|71749|United States|-6|condo| +47606|AAAAAAAAGPJLAAAA|78|Lakeview Jackson|Ave|Suite 220|Concord|Carroll County|IA|54107|United States|-6|apartment| +47607|AAAAAAAAHPJLAAAA|20|Tenth Lakeview|Blvd|Suite 300|Fairview|Marin County|CA|95709|United States|-8|condo| +47608|AAAAAAAAIPJLAAAA|532|Pine Park|ST|Suite 170|Pine Grove|Brazoria County|TX|74593|United States|-6|apartment| +47609|AAAAAAAAJPJLAAAA|670|Center Meadow|Cir.|Suite G|Farmington|Seward County|KS|69145|United States|-6|apartment| +47610|AAAAAAAAKPJLAAAA|813|Sycamore Lincoln|Boulevard|Suite 380|Clifton|Gladwin County|MI|48014|United States|-5|single family| +47611|AAAAAAAALPJLAAAA|674|River 6th|Ave|Suite 30|Pine Grove|Alleghany County|VA|24593|United States|-5|condo| +47612|AAAAAAAAMPJLAAAA|707|Park Sycamore|ST|Suite 290|Edgewood|Pulaski County|IN|40069|United States|-5|single family| +47613|AAAAAAAANPJLAAAA|538|Oak |Parkway|Suite J|Edgewood|Wilcox County|AL|30069|United States|-6|apartment| +47614|AAAAAAAAOPJLAAAA|718|4th Jefferson|Way|Suite 0|Liberty|Jefferson County|IL|63451|United States|-6|apartment| +47615|AAAAAAAAPPJLAAAA|785|Central |Circle|Suite 90|Glenwood|Somervell County|TX|73511|United States|-6|apartment| +47616|AAAAAAAAAAKLAAAA|615|Johnson North|Parkway|Suite 430|Spring Hill|Charles County|MD|26787|United States|-5|single family| +47617|AAAAAAAABAKLAAAA|418|Sycamore Oak|Way|Suite G|White Oak|Yolo County|CA|96668|United States|-8|condo| +47618|AAAAAAAACAKLAAAA|589|West Ridge|RD|Suite V|Mount Zion|Cottle County|TX|78054|United States|-6|apartment| +47619|AAAAAAAADAKLAAAA|220||Ave|||Renville County||55038|||| +47620|AAAAAAAAEAKLAAAA|704|1st 5th|Lane|Suite 160|Fairbanks|Aleutians West Census Area|AK|96653|United States|-9|apartment| +47621|AAAAAAAAFAKLAAAA|380|Second Elm|Lane|Suite 480|Clinton|Pike County|AR|78222|United States|-6|condo| +47622|AAAAAAAAGAKLAAAA|239|Lee |RD|Suite C|Macedonia|Yamhill County|OR|91087|United States|-8|single family| +47623|AAAAAAAAHAKLAAAA|908|Lee 12th|Parkway|Suite 410|Centerville|Bristol County|RI|00659|United States|-5|single family| +47624|AAAAAAAAIAKLAAAA|858|Walnut Willow|Ave|Suite 470|Sulphur Springs|Lyman County|SD|58354|United States|-7|single family| +47625|AAAAAAAAJAKLAAAA|420|Hill |RD|Suite T|Bunker Hill|Chester County|SC|20150|United States|-5|condo| +47626|AAAAAAAAKAKLAAAA|||Ave|Suite U|Highland|Brewster County|TX|79454|||| +47627|AAAAAAAALAKLAAAA|642|9th |Lane|Suite 50|Buena Vista|Union County|PA|15752|United States|-5|single family| +47628|AAAAAAAAMAKLAAAA|48|Center |Way|Suite Y|Midway|East Baton Rouge Parish|LA|71904|United States|-6|single family| +47629|AAAAAAAANAKLAAAA|579|Smith |Circle|Suite W|Florence|Plaquemines Parish|LA|73394|United States|-6|apartment| +47630|AAAAAAAAOAKLAAAA|310|Green |Pkwy|Suite S|Greenwood|Cowley County|KS|68828|United States|-6|apartment| +47631|AAAAAAAAPAKLAAAA|906|Elm |Ave|Suite 260|Berea|Gogebic County|MI|43804|United States|-5|condo| +47632|AAAAAAAAABKLAAAA|901|West |Circle|Suite 400|Centerville|Toole County|MT|60059|United States|-7|single family| +47633|AAAAAAAABBKLAAAA|156|Center Davis|Road|Suite 260|Cedar Grove|Lewis County|WA|90411|United States|-8|single family| +47634|AAAAAAAACBKLAAAA|168|First Sunset|Road|Suite 70|Woodville|Ellis County|TX|74289|United States|-6|single family| +47635|AAAAAAAADBKLAAAA|921|4th |Ln|Suite 190|Bethel|Fulton County|OH|45281|United States|-5|condo| +47636|AAAAAAAAEBKLAAAA|300|5th |Parkway|Suite V|Shelby|Lamb County|TX|76575|United States|-6|single family| +47637|AAAAAAAAFBKLAAAA|784|View Lincoln|Court|Suite 40|Bunker Hill|Allen Parish|LA|70150|United States|-6|condo| +47638|AAAAAAAAGBKLAAAA|564|Walnut Valley|Lane|Suite M|Clinton|Sauk County|WI|58222|United States|-6|apartment| +47639|AAAAAAAAHBKLAAAA|595|Lee Main|Wy|Suite F|Liberty|Cowlitz County|WA|93451|United States|-8|apartment| +47640|AAAAAAAAIBKLAAAA|255|View |Blvd|Suite X|Springfield|Winston County|MS|59303|United States|-6|condo| +47641|AAAAAAAAJBKLAAAA||View Church||Suite 460||Elk County|KS|60302|United States||apartment| +47642|AAAAAAAAKBKLAAAA|241|3rd |Drive|Suite 60|New Hope|Hood County|TX|79431|United States|-6|single family| +47643|AAAAAAAALBKLAAAA||Park |Parkway|Suite 180|Kingston|Forest County|PA||United States|-5|| +47644|AAAAAAAAMBKLAAAA|745|Main Fifth|Parkway|Suite C|Pleasant Hill|Ouachita County|AR|73604|United States|-6|apartment| +47645|AAAAAAAANBKLAAAA|340|Main |Avenue|Suite 110|Friendship|Norfolk city|VA|24536|United States|-5|single family| +47646|AAAAAAAAOBKLAAAA|549|Davis |Ct.|Suite J|Greenfield|Polk County|MN|55038|United States|-6|single family| +47647|AAAAAAAAPBKLAAAA|299|Park Eigth|Drive|Suite M|Hamilton|Boise County|ID|82808|United States|-7|single family| +47648|AAAAAAAAACKLAAAA|64|7th |Cir.|Suite C|Pleasant Grove|Rock County|WI|54136|United States|-6|single family| +47649|AAAAAAAABCKLAAAA|480|Sixth 4th|Dr.|Suite 70|Midway|McPherson County|KS|61904|United States|-6|apartment| +47650|AAAAAAAACCKLAAAA|398|First |Ln|Suite 200|Stringtown|Kendall County|TX|70162|United States|-6|apartment| +47651|AAAAAAAADCKLAAAA|444|Center |Wy|Suite 340|Macedonia|Lancaster County|SC|21087|United States|-5|apartment| +47652|AAAAAAAAECKLAAAA|594|Sixth |Parkway|Suite C|Crossroads|Martin County|TX|70534|United States|-6|apartment| +47653|AAAAAAAAFCKLAAAA|54|Woodland Park|Circle|Suite 370|Mount Zion|Swain County|NC|28054|United States|-5|apartment| +47654|AAAAAAAAGCKLAAAA|824|Mill Oak|Court|Suite 250|Centerville|Ballard County|KY|40059|United States|-6|condo| +47655|AAAAAAAAHCKLAAAA|286|Birch |Dr.|Suite J|Mount Olive|Lee County|IL|68059|United States|-6|apartment| +47656|AAAAAAAAICKLAAAA|414|Hickory |Ln|Suite I|Pleasant Hill|Brown County|KS|63604|United States|-6|single family| +47657|AAAAAAAAJCKLAAAA|38|Sunset Lee|Parkway|Suite 60|Woodlawn|Clarke County|MS|54098|United States|-6|apartment| +47658|AAAAAAAAKCKLAAAA|329|Center |Ct.|Suite X|Fairview|Bradford County|FL|35709|United States|-5|condo| +47659|AAAAAAAALCKLAAAA|644|Broadway Birch|Road|Suite L|Hopewell|Jackson County|TX|70587|United States|-6|single family| +47660|AAAAAAAAMCKLAAAA|638|Park Highland|Ave|Suite 40|Red Hill|Martin County|TX|74338|United States|-6|single family| +47661|AAAAAAAANCKLAAAA|337|Fourth |Dr.|Suite A|Belmont|Emanuel County|GA|30191|United States|-5|condo| +47662|AAAAAAAAOCKLAAAA|485|Pine |Lane|Suite 490|Stringtown|Calhoun County|WV|20162|United States|-5|apartment| +47663|AAAAAAAAPCKLAAAA|636|Sycamore |Pkwy|Suite J|Maple Grove|White County|AR|78252|United States|-6|condo| +47664|AAAAAAAAADKLAAAA|803|Chestnut Lincoln|Pkwy|Suite F|Mount Pleasant|Bath County|VA|21933|United States|-5|condo| +47665|AAAAAAAABDKLAAAA|597|Cedar Hillcrest|Drive|Suite 310|Jamestown|Franklin Parish|LA|76867|United States|-6|condo| +47666|AAAAAAAACDKLAAAA|546|2nd Elm|Parkway|Suite A|Oakdale|Worth County|GA|39584|United States|-5|condo| +47667|AAAAAAAADDKLAAAA|370|10th Hickory|Ct.|Suite 480|Woodville|Williams County|ND|54289|United States|-6|single family| +47668|AAAAAAAAEDKLAAAA|623|Woodland 7th|ST|Suite 270|Oak Grove|Freestone County|TX|78370|United States|-6|single family| +47669|AAAAAAAAFDKLAAAA|138|Locust 2nd|Avenue|Suite K|Belmont|Dakota County|NE|60191|United States|-6|condo| +47670|AAAAAAAAGDKLAAAA|341|Main |Lane|Suite 320|Woodville|Burnet County|TX|74289|United States|-6|apartment| +47671|AAAAAAAAHDKLAAAA|531|4th |Drive|Suite V|Sulphur Springs|Montgomery County|GA|38354|United States|-5|single family| +47672|AAAAAAAAIDKLAAAA|216|Willow |Ave|Suite V|Lebanon|Okeechobee County|FL|32898|United States|-5|single family| +47673|AAAAAAAAJDKLAAAA|317|College Central|ST|Suite 470|Lakeside|Iroquois County|IL|69532|United States|-6|single family| +47674|AAAAAAAAKDKLAAAA|569|Hillcrest |Lane|Suite 290|Springdale|Dorchester County|SC|28883|United States|-5|condo| +47675|AAAAAAAALDKLAAAA|335|Hillcrest Lake|Street|Suite 140|Woodlawn|Kings County|CA|94098|United States|-8|single family| +47676|AAAAAAAAMDKLAAAA|369|Ridge |Avenue|Suite 340|Arlington|Greene County|NC|26557|United States|-5|condo| +47677|AAAAAAAANDKLAAAA|539|||Suite 100|New Hope|Perry County||69431|United States|-6|| +47678|AAAAAAAAODKLAAAA|278|6th 2nd|Lane|Suite U|Wildwood|Carroll County|MS|56871|United States|-6|condo| +47679|AAAAAAAAPDKLAAAA||||Suite O|||||||single family| +47680|AAAAAAAAAEKLAAAA|606|Davis |Street|Suite K|Woodlawn|Dixie County|FL|34098|United States|-5|apartment| +47681|AAAAAAAABEKLAAAA|846|Park Railroad|Ave|Suite O|Ashland|Plaquemines Parish|LA|74244|United States|-6|apartment| +47682|AAAAAAAACEKLAAAA|895|Willow |ST|Suite C|Newport|Clark County|WI|51521|United States|-6|apartment| +47683|AAAAAAAADEKLAAAA|547|River View|Parkway|Suite 320|Centerville|Forest County|WI|50059|United States|-6|apartment| +47684|AAAAAAAAEEKLAAAA|479|Sixth 9th|Parkway|Suite C|Hopewell|Plaquemines Parish|LA|70587|United States|-6|apartment| +47685|AAAAAAAAFEKLAAAA|133|Hickory Jefferson|Road|Suite 180|Ashland|Macomb County|MI|44244|United States|-5|single family| +47686|AAAAAAAAGEKLAAAA|793|Mill Wilson|Cir.|Suite K|Mount Vernon|Doddridge County|WV|28482|United States|-5|apartment| +47687|AAAAAAAAHEKLAAAA|42|2nd 5th|Blvd|Suite V|Newtown|Berrien County|GA|31749|United States|-5|apartment| +47688|AAAAAAAAIEKLAAAA|123|Adams Lincoln|Way|Suite 420|Woodland|Lincoln County|WA|94854|United States|-8|condo| +47689|AAAAAAAAJEKLAAAA|430|1st College|Ct.|Suite 270|Summit|Clackamas County|OR|90499|United States|-8|single family| +47690|AAAAAAAAKEKLAAAA|1|Jefferson Wilson|Court|Suite 10|Highland Park|Jefferson County|WA|96534|United States|-8|single family| +47691|AAAAAAAALEKLAAAA|910|Oak Hill|Street|Suite B|Waterloo|Klickitat County|WA|91675|United States|-8|condo| +47692|AAAAAAAAMEKLAAAA|908|Green Adams|Boulevard|Suite H|Jamestown|Caldwell County|NC|26867|United States|-5|apartment| +47693|AAAAAAAANEKLAAAA|391|Lee Johnson|Cir.|Suite 130|Deerfield|Hickory County|MO|69840|United States|-6|condo| +47694|AAAAAAAAOEKLAAAA|972|4th Smith|Road|Suite Y|Colfax|Franklin County|WA|92565|United States|-8|apartment| +47695|AAAAAAAAPEKLAAAA|172|Main |Parkway|Suite R|Florence|Allendale County|SC|23394|United States|-5|apartment| +47696|AAAAAAAAAFKLAAAA|891|1st Woodland|ST|Suite S|Wesley|Madison County|NE|61218|United States|-7|condo| +47697|AAAAAAAABFKLAAAA|374|Johnson |Circle|Suite B|Newport|Franklin city|VA|21521|United States|-5|condo| +47698|AAAAAAAACFKLAAAA|779|Oak |Boulevard|Suite L|Macedonia|Wyandotte County|KS|61087|United States|-6|apartment| +47699|AAAAAAAADFKLAAAA|460|Sunset Railroad|Street|Suite T|Gary|Dixie County|FL|30418|United States|-5|apartment| +47700|AAAAAAAAEFKLAAAA|842|Oak |Blvd|Suite A|Pine Grove|Lewis and Clark County|MT|64593|United States|-7|apartment| +47701|AAAAAAAAFFKLAAAA|294|Forest |Wy|Suite U|Highland|Sweet Grass County|MT|69454|United States|-7|apartment| +47702|AAAAAAAAGFKLAAAA|319|Oak Forest|Wy|Suite 440|Oakdale|Brooks County|TX|79584|United States|-6|condo| +47703|AAAAAAAAHFKLAAAA|404|Laurel Broadway|RD|Suite 480|Georgetown|Jefferson Davis Parish|LA|77057|United States|-6|condo| +47704|AAAAAAAAIFKLAAAA|584|Lake 3rd|Wy|Suite Q|Lakeside|Essex County|VA|29532|United States|-5|single family| +47705|AAAAAAAAJFKLAAAA|533|Mill 6th|Boulevard|Suite 480|Lakewood|Warren County|NY|18877|United States|-5|apartment| +47706|AAAAAAAAKFKLAAAA|221|Maple River|Road|Suite 30|Waterloo|Clark County|ID|81675|United States|-7|apartment| +47707|AAAAAAAALFKLAAAA|527|North West|Avenue|Suite 60|Springdale|Rowan County|KY|48883|United States|-5|apartment| +47708|AAAAAAAAMFKLAAAA|509|Wilson Park|Parkway|Suite 460|Fairfield|Oldham County|TX|76192|United States|-6|single family| +47709|AAAAAAAANFKLAAAA|537|Highland 3rd|Way|Suite L|Union|San Luis Obispo County|CA|98721|United States|-8|apartment| +47710|AAAAAAAAOFKLAAAA|175|10th Davis|Circle|Suite D|Oakdale|Adams County|OH|49584|United States|-5|condo| +47711|AAAAAAAAPFKLAAAA|373|Maple |Court|Suite 280|Deerfield|Salem County|NJ|09840|United States|-5|single family| +47712|AAAAAAAAAGKLAAAA|829|Lincoln 6th|Cir.|Suite 230|Pleasant Hill|Wayne County|MI|43604|United States|-5|single family| +47713|AAAAAAAABGKLAAAA|230|Meadow |Pkwy|Suite U|Lebanon|San Diego County|CA|92898|United States|-8|single family| +47714|AAAAAAAACGKLAAAA|54|Lee |Road|Suite 80|Gilmore|Knox County|IL|65464|United States|-6|single family| +47715|AAAAAAAADGKLAAAA|703|Ninth Hickory|Boulevard|Suite 330|Greenville|Teton County|WY|81387|United States|-7|condo| +47716|AAAAAAAAEGKLAAAA|567|Church Highland|ST|Suite 370|Wilson|Glades County|FL|36971|United States|-5|apartment| +47717|AAAAAAAAFGKLAAAA|796|Lincoln Meadow|Wy|Suite F|Concord|Lawrence County|AL|34107|United States|-6|apartment| +47718|AAAAAAAAGGKLAAAA|271|Fourth Locust|Ave|Suite 200|Clinton|Moultrie County|IL|68222|United States|-6|condo| +47719|AAAAAAAAHGKLAAAA|346|Pine Willow|Parkway|Suite 420|Liberty|Webb County|TX|73451|United States|-6|condo| +47720|AAAAAAAAIGKLAAAA|304|Center |Cir.|Suite U|Harmony|Hart County|KY|45804|United States|-6|condo| +47721|AAAAAAAAJGKLAAAA|49|Park Fourteenth|RD|Suite M|Oak Hill|Bleckley County|GA|37838|United States|-5|apartment| +47722|AAAAAAAAKGKLAAAA|175|Main Fifteenth|Court|Suite 370|Paxton|New London County|CT|06269|United States|-5|condo| +47723|AAAAAAAALGKLAAAA|844|Laurel |Way|Suite 110|Brownsville|Jackson County|KY|49310|United States|-6|condo| +47724|AAAAAAAAMGKLAAAA|723|Ash Ridge|ST|Suite 200|Five Points|Portsmouth city|VA|26098|United States|-5|single family| +47725|AAAAAAAANGKLAAAA|202|North |Avenue|Suite 310|Arlington|Burlington County|NJ|07157|United States|-5|single family| +47726|AAAAAAAAOGKLAAAA|503|8th |Road|Suite 140|Fairfield|Harrison County|OH|46192|United States|-5|apartment| +47727|AAAAAAAAPGKLAAAA|605|Davis |Avenue|Suite F|Fairfield|Decatur County|GA|36192|United States|-5|condo| +47728|AAAAAAAAAHKLAAAA|606|Fifteenth Hill|Circle|Suite H|Oakdale|Grand Traverse County|MI|49584|United States|-5|single family| +47729|AAAAAAAABHKLAAAA|959|Hillcrest Woodland|Dr.|Suite T|Farmington|Horry County|SC|29145|United States|-5|condo| +47730|AAAAAAAACHKLAAAA|57|Lee |Cir.|Suite P|Oakland|Denton County|TX|79843|United States|-6|apartment| +47731|AAAAAAAADHKLAAAA|685|7th Hillcrest|Drive|Suite 100|Sulphur Springs|Dillon County|SC|28354|United States|-5|apartment| +47732|AAAAAAAAEHKLAAAA|72|Maple |Avenue|Suite C|Oakwood|Fayette County|KY|40169|United States|-6|single family| +47733|AAAAAAAAFHKLAAAA|338|8th Lincoln|Boulevard|Suite 40|Waterloo|Towns County|GA|31675|United States|-5|apartment| +47734|AAAAAAAAGHKLAAAA|966|Chestnut |RD|Suite 200|Highland Park|Lucas County|IA|56534|United States|-6|single family| +47735|AAAAAAAAHHKLAAAA|742|Hillcrest |RD|Suite 50|Harmony|Phillips County|CO|85804|United States|-7|apartment| +47736|AAAAAAAAIHKLAAAA|599|14th 8th|RD|Suite M|Florence|Comal County|TX|73394|United States|-6|single family| +47737|AAAAAAAAJHKLAAAA|21|Willow 7th|Ln|Suite 330|Five Points|Richmond County|GA|36098|United States|-5|condo| +47738|AAAAAAAAKHKLAAAA|691|Mill Willow|Cir.|Suite A|Greenville|Stephenson County|IL|61387|United States|-6|condo| +47739|AAAAAAAALHKLAAAA|356|Willow |Lane|Suite 340|Clifton|Big Stone County|MN|58014|United States|-6|apartment| +47740|AAAAAAAAMHKLAAAA|922|Main 8th|ST|Suite 390|Kingston|Iron County|MO|64975|United States|-6|single family| +47741|AAAAAAAANHKLAAAA|189|Lake |Circle|Suite 270|Clinton|Posey County|IN|48222|United States|-5|single family| +47742|AAAAAAAAOHKLAAAA|475|Sunset |Drive|Suite F|Friendship|Franklin County|NC|24536|United States|-5|single family| +47743|AAAAAAAAPHKLAAAA|224|9th |Parkway|Suite 240|Bethel|Jackson County|FL|35281|United States|-5|single family| +47744|AAAAAAAAAIKLAAAA|737|12th Sycamore|RD|Suite 220|Lenox|Harmon County|OK|71143|United States|-6|single family| +47745|AAAAAAAABIKLAAAA|968|Park |Way|Suite C|Greenwood|Dougherty County|GA|38828|United States|-5|single family| +47746|AAAAAAAACIKLAAAA|439|Poplar Second|Way|Suite W|Woodville|Stone County|MO|64289|United States|-6|apartment| +47747|AAAAAAAADIKLAAAA|86|Lake |RD|Suite H|Edgewood|Wyoming County|NY|10069|United States|-5|condo| +47748|AAAAAAAAEIKLAAAA|443|River |Street|Suite 190|Fairview|Prentiss County|MS|55709|United States|-6|apartment| +47749|AAAAAAAAFIKLAAAA|74|Highland |Ave|Suite O|Warwick|Montgomery County|AL|31398|United States|-6|apartment| +47750|AAAAAAAAGIKLAAAA|438|2nd |Parkway|Suite 400|Unionville|Ohio County|IN|41711|United States|-5|single family| +47751|AAAAAAAAHIKLAAAA|741|Hickory Meadow|Road|Suite I|Shiloh|Lexington city|VA|29275|United States|-5|apartment| +47752|AAAAAAAAIIKLAAAA|312|3rd 15th|Blvd|Suite 210|Maple Grove|Charleston County|SC|28252|United States|-5|condo| +47753|AAAAAAAAJIKLAAAA|149|Highland |Ct.|Suite 340|Wildwood|Treutlen County|GA|36871|United States|-5|condo| +47754|AAAAAAAAKIKLAAAA|206|Third |Parkway|Suite P|Bridgeport|Tyler County|WV|25817|United States|-5|single family| +47755|AAAAAAAALIKLAAAA|463|Park First|Ave|Suite B|Midway|Colfax County|NE|61904|United States|-6|single family| +47756|AAAAAAAAMIKLAAAA|730|Mill Park|Avenue|Suite 480|Liberty|Garfield County|OK|73451|United States|-6|condo| +47757|AAAAAAAANIKLAAAA|795|Elm |Avenue|Suite N|Mount Zion|Cowlitz County|WA|98054|United States|-8|apartment| +47758|AAAAAAAAOIKLAAAA|629|South Spruce||Suite 80|||FL|34466||-5|| +47759|AAAAAAAAPIKLAAAA|199|Fourth 2nd|Boulevard|Suite N|Lakewood|Jasper County|IN|48877|United States|-5|apartment| +47760|AAAAAAAAAJKLAAAA|946|Main 2nd|Street|Suite Y|Franklin|Taylor County|GA|39101|United States|-5|apartment| +47761|AAAAAAAABJKLAAAA|478|Oak Park|Parkway|Suite G|Providence|Dodge County|NE|66614|United States|-6|apartment| +47762|AAAAAAAACJKLAAAA|536|Lincoln Elm|Way|Suite 300|Newtown|Pinal County|AZ|81749|United States|-7|apartment| +47763|AAAAAAAADJKLAAAA|355|Fifth Poplar|Pkwy|Suite 70|Wildwood|Jackson County|AR|76871|United States|-6|apartment| +47764|AAAAAAAAEJKLAAAA|632|Maple 14th|Ln|Suite 220|Union Hill|Otero County|CO|87746|United States|-7|condo| +47765|AAAAAAAAFJKLAAAA|219|Johnson |RD|Suite 220|Sulphur Springs|Walla Walla County|WA|98354|United States|-8|condo| +47766|AAAAAAAAGJKLAAAA|448|4th |Road|Suite 80|Hillcrest|Washington County|WI|53003|United States|-6|apartment| +47767|AAAAAAAAHJKLAAAA|723|Elm |Pkwy|Suite 190|Providence|Sauk County|WI|56614|United States|-6|single family| +47768|AAAAAAAAIJKLAAAA|122|North Hillcrest|Cir.|Suite 140|Spring Valley|Whiteside County|IL|66060|United States|-6|apartment| +47769|AAAAAAAAJJKLAAAA|748|Willow |Street|Suite J|Shady Grove|Morgan County|WV|22812|United States|-5|condo| +47770|AAAAAAAAKJKLAAAA|986|Pine |ST|Suite 210|Fairfield|Perry County|IN|46192|United States|-5|condo| +47771|AAAAAAAALJKLAAAA|103|Fourth Park|Street|Suite 60|Gravel Hill|Hidalgo County|NM|81944|United States|-7|single family| +47772|AAAAAAAAMJKLAAAA|117|East |Wy|Suite O|Union|Rolette County|ND|58721|United States|-6|apartment| +47773|AAAAAAAANJKLAAAA|540|Franklin |RD|Suite 70|Georgetown|Lowndes County|AL|37057|United States|-6|single family| +47774|AAAAAAAAOJKLAAAA|97|Meadow |Avenue|Suite E|Pleasant Valley|Hawaii County|HI|92477|United States|-10|apartment| +47775|AAAAAAAAPJKLAAAA|926|Locust 4th|Avenue|Suite B|Georgetown|Hood County|TX|77057|United States|-6|condo| +47776|AAAAAAAAAKKLAAAA|691|Cherry West|Ln|Suite X|Oak Ridge|El Paso County|TX|78371|United States|-6|condo| +47777|AAAAAAAABKKLAAAA|55|2nd Park|Ave|Suite H|New Hope|Matanuska-Susitna Borough|AK|99431|United States|-9|single family| +47778|AAAAAAAACKKLAAAA|987|Ash Davis|Circle|Suite 430|Unionville|Hancock County|KY|41711|United States|-6|apartment| +47779|AAAAAAAADKKLAAAA|266|First |Street|Suite U|Five Forks|Tipton County|IN|42293|United States|-5|apartment| +47780|AAAAAAAAEKKLAAAA|691|Fourth Second|Parkway|Suite 240|Frenchtown|Collier County|FL|32629|United States|-5|apartment| +47781|AAAAAAAAFKKLAAAA|288|Miller |Road|Suite J|Crossroads|Marshall County|SD|50534|United States|-7|single family| +47782|AAAAAAAAGKKLAAAA|697|East |Ln|Suite 10|Pleasant Hill|Woodruff County|AR|73604|United States|-6|apartment| +47783|AAAAAAAAHKKLAAAA|780|Poplar |Ave|Suite 240|Stringtown|Grayson County|KY|40162|United States|-6|condo| +47784|AAAAAAAAIKKLAAAA|412|1st |Ln|Suite U|Glendale|Pleasants County|WV|23951|United States|-5|single family| +47785|AAAAAAAAJKKLAAAA|955|Ridge Miller|Parkway|Suite 170|Walnut Grove|Foster County|ND|57752|United States|-6|single family| +47786|AAAAAAAAKKKLAAAA|550|3rd Park|Lane|Suite 170|Clifton|Ravalli County|MT|68014|United States|-7|apartment| +47787|AAAAAAAALKKLAAAA|411|Adams |Lane|Suite 20|Ashland|Perry County|IL|64244|United States|-6|condo| +47788|AAAAAAAAMKKLAAAA|835|Franklin |Parkway|Suite A|Allison|Treutlen County|GA|34167|United States|-5|single family| +47789|AAAAAAAANKKLAAAA|966|2nd Ridge|Ln|Suite 320|Greenwood|Cameron County|PA|18828|United States|-5|single family| +47790|AAAAAAAAOKKLAAAA||Sunset |Ave|Suite R||||55709|United States|-6|| +47791|AAAAAAAAPKKLAAAA|297|Lincoln Adams|ST|Suite X|Arlington|Guthrie County|IA|56557|United States|-6|apartment| +47792|AAAAAAAAALKLAAAA|539|South Highland|Circle|Suite F|Woodland|Craig County|VA|24854|United States|-5|single family| +47793|AAAAAAAABLKLAAAA|659|Cedar Park|Boulevard|Suite E|Oakland|Schoharie County|NY|19843|United States|-5|condo| +47794|AAAAAAAACLKLAAAA|524|River Laurel|Parkway|Suite A|Forest Hills|Kenton County|KY|49237|United States|-5|condo| +47795|AAAAAAAADLKLAAAA|81|1st |Ave|Suite 80|Liberty|Irwin County|GA|33451|United States|-5|condo| +47796|AAAAAAAAELKLAAAA|625|Lincoln |Court|Suite K|Clearview|Burt County|NE|65495|United States|-6|condo| +47797|AAAAAAAAFLKLAAAA|857|4th |Ct.|Suite N|Newtown|Le Flore County|OK|71749|United States|-6|condo| +47798|AAAAAAAAGLKLAAAA|923|1st 12th|Ct.|Suite 230|Springfield|Rusk County|TX|79303|United States|-6|single family| +47799|AAAAAAAAHLKLAAAA|202|Main |Drive|Suite A|Franklin|Beaver County|UT|89101|United States|-7|apartment| +47800|AAAAAAAAILKLAAAA|321|Oak Valley|Road|Suite E|White Oak|Seminole County|OK|76668|United States|-6|condo| +47801|AAAAAAAAJLKLAAAA|397|Williams Laurel|Circle|Suite 450|Edgewood|Lamar County|GA|30069|United States|-5|single family| +47802|AAAAAAAAKLKLAAAA|491|Mill First|RD|Suite F|Jackson|Franklin County|AL|39583|United States|-6|single family| +47803|AAAAAAAALLKLAAAA|166|Washington |Boulevard|Suite L|Spring Hill|Crawford County|IL|66787|United States|-6|apartment| +47804|AAAAAAAAMLKLAAAA|648|Sixth Park|ST|Suite D|Wilson|Morris County|KS|66971|United States|-6|apartment| +47805|AAAAAAAANLKLAAAA|189|Forest Wilson|ST|Suite G|Union Hill|Mecklenburg County|NC|27746|United States|-5|apartment| +47806|AAAAAAAAOLKLAAAA|216|West |Blvd|Suite 470|Murray|Faribault County|MN|52150|United States|-6|single family| +47807|AAAAAAAAPLKLAAAA|793|Jackson 1st|Boulevard|Suite 70|Peru|Lane County|OR|90302|United States|-8|single family| +47808|AAAAAAAAAMKLAAAA|522|Lee Lee|Circle|Suite 100|Oakdale|Richland County|MT|69584|United States|-7|apartment| +47809|AAAAAAAABMKLAAAA|859|Lake Park|Ln|Suite Q|Oneida|Hancock County|OH|44027|United States|-5|apartment| +47810|AAAAAAAACMKLAAAA|863|Jackson 8th|Court|Suite 310|Spring Grove|Polk County|WI|56719|United States|-6|single family| +47811|AAAAAAAADMKLAAAA|792|Broadway Lake|Blvd|Suite 100|Evergreen|Boone County|IA|55621|United States|-6|single family| +47812|AAAAAAAAEMKLAAAA|314|Adams 8th|Ln|Suite 210|Glenwood|Dooly County|GA|33511|United States|-5|condo| +47813|AAAAAAAAFMKLAAAA|804|Broadway |Dr.|Suite P|Kingston|Dickinson County|IA|54975|United States|-6|single family| +47814|AAAAAAAAGMKLAAAA|196|5th |Dr.|Suite 280|Friendship|Union County|GA|34536|United States|-5|single family| +47815|AAAAAAAAHMKLAAAA|740|Church Elm|Cir.|Suite 260|Highland|Adams County|MS|59454|United States|-6|condo| +47816|AAAAAAAAIMKLAAAA|117|1st |Boulevard|Suite E|Mount Pleasant|Harding County|SD|51933|United States|-7|single family| +47817|AAAAAAAAJMKLAAAA|559|Johnson |Avenue|Suite P|Wildwood|Calhoun County|MI|46871|United States|-5|single family| +47818|AAAAAAAAKMKLAAAA|904|Church |Avenue|Suite 10|Red Hill|Russell County|KY|44338|United States|-5|apartment| +47819|AAAAAAAALMKLAAAA|103|3rd Oak|Ave|Suite J|Midway|Lake County|FL|31904|United States|-5|condo| +47820|AAAAAAAAMMKLAAAA|810|Woodland 3rd|Blvd|Suite X|Belmont|Bartholomew County|IN|40191|United States|-5|single family| +47821|AAAAAAAANMKLAAAA|807|Sunset 13th|Avenue|Suite R|Waterloo|Monroe County|IA|51675|United States|-6|apartment| +47822|AAAAAAAAOMKLAAAA|92|13th Forest|Way|Suite S|Unionville|Washington County|NC|21711|United States|-5|condo| +47823|AAAAAAAAPMKLAAAA|436|Railroad Adams|Avenue|Suite 60|Mountain View|Fayette County|PA|14466|United States|-5|apartment| +47824|AAAAAAAAANKLAAAA|730|Highland Cherry|Way|Suite F|Cumberland|Wayne County|MS|58971|United States|-6|single family| +47825|AAAAAAAABNKLAAAA|289|Franklin |Circle|Suite I|Oak Grove|Wise County|VA|28370|United States|-5|single family| +47826|AAAAAAAACNKLAAAA|775|14th Willow|Pkwy|Suite I|Lebanon|Hillsborough County|NH|03498|United States|-5|single family| +47827|AAAAAAAADNKLAAAA|987|Ridge Thirteenth|Way|Suite R|Fairfield|Chemung County|NY|16192|United States|-5|condo| +47828|AAAAAAAAENKLAAAA|832|8th Jackson|Parkway|Suite O|Plainview|Shoshone County|ID|83683|United States|-7|condo| +47829|AAAAAAAAFNKLAAAA|892|Fourth Broadway|Ave|Suite 380|Pleasant Grove|Dooly County|GA|34136|United States|-5|apartment| +47830|AAAAAAAAGNKLAAAA|264|14th |Street|Suite X|Harmon|Platte County|MO|65623|United States|-6|single family| +47831|AAAAAAAAHNKLAAAA|24|Walnut Oak|Ct.|Suite 360|Summit|Nolan County|TX|70499|United States|-6|single family| +47832|AAAAAAAAINKLAAAA|665|9th |Way|Suite 50|Yorktown|Cross County|AR|70732|United States|-6|condo| +47833|AAAAAAAAJNKLAAAA|231|Cedar Pine|Way|Suite 50|Pleasant Hill|Alameda County|CA|93604|United States|-8|condo| +47834|AAAAAAAAKNKLAAAA|963|2nd Miller|Way|Suite S|Ellisville|Lake County|CA|96820|United States|-8|single family| +47835|AAAAAAAALNKLAAAA|706|View |Road|Suite F|Pleasant Grove|Houston County|TN|34136|United States|-5|condo| +47836|AAAAAAAAMNKLAAAA|621|Main Chestnut|Circle|Suite F|Forest Hills|Salem city|VA|29237|United States|-5|single family| +47837|AAAAAAAANNKLAAAA|36|West |Dr.|Suite 490|Georgetown|Bay County|FL|37057|United States|-5|condo| +47838|AAAAAAAAONKLAAAA|69|2nd |ST|Suite A|Walnut Grove|Yuma County|CO|87752|United States|-7|apartment| +47839|AAAAAAAAPNKLAAAA|204|Sixth |Lane|Suite W|Bunker Hill|Putnam County|WV|20150|United States|-5|apartment| +47840|AAAAAAAAAOKLAAAA||Pine Johnson||Suite 10|||SC|20059|United States|-5|| +47841|AAAAAAAABOKLAAAA|803|3rd |Cir.|Suite I|Woodland|Rosebud County|MT|64854|United States|-7|condo| +47842|AAAAAAAACOKLAAAA|293|Valley Oak|Drive|Suite 290|Sunnyside|Jackson County|FL|31952|United States|-5|apartment| +47843|AAAAAAAADOKLAAAA|212|Third East|Blvd|Suite 310|Springdale|Randolph County|NC|28883|United States|-5|condo| +47844|AAAAAAAAEOKLAAAA|664|Wilson First|Ln|Suite 260|Midway|Sequoyah County|OK|71904|United States|-6|single family| +47845|AAAAAAAAFOKLAAAA|395|West 4th|Wy|Suite 270|Highland|Lucas County|IA|59454|United States|-6|condo| +47846|AAAAAAAAGOKLAAAA|227|14th |Way|Suite 490|Georgetown|Yamhill County|OR|97057|United States|-8|condo| +47847|AAAAAAAAHOKLAAAA|963|View |Drive|Suite Y|Wildwood|Wells County|IN|46871|United States|-5|apartment| +47848|AAAAAAAAIOKLAAAA|405|South |Drive|Suite 60|Enterprise|Clifton Forge city|VA|21757|United States|-5|condo| +47849|AAAAAAAAJOKLAAAA|613|13th Sycamore|Blvd|Suite 40|Stratford|Phillips County|AR|76668|United States|-6|single family| +47850|AAAAAAAAKOKLAAAA|605|Wilson |Ln|Suite V|Pleasant Valley|Frio County|TX|72477|United States|-6|condo| +47851|AAAAAAAALOKLAAAA|695|East |Dr.|Suite 280|Jamestown|Ventura County|CA|96867|United States|-8|single family| +47852|AAAAAAAAMOKLAAAA|629|Franklin Williams|Drive|Suite J|Shady Grove|Genesee County|NY|12812|United States|-5|single family| +47853|AAAAAAAANOKLAAAA|837|12th |Street|Suite 200|Elba|Pembina County|ND|50262|United States|-6|condo| +47854|AAAAAAAAOOKLAAAA|208|Highland Adams|Blvd|Suite A|Plainview|Fond du Lac County|WI|53683|United States|-6|single family| +47855|AAAAAAAAPOKLAAAA|678|Park |Wy|Suite X|Arlington|Gila County|AZ|86557|United States|-7|apartment| +47856|AAAAAAAAAPKLAAAA|169|Center Hillcrest|RD|Suite R|Red Hill|Placer County|CA|94338|United States|-8|single family| +47857|AAAAAAAABPKLAAAA|149|5th |Wy|Suite J|Forest Hills|Mercer County|ND|59237|United States|-6|apartment| +47858|AAAAAAAACPKLAAAA|406|Cherry |Pkwy|Suite H|Friendship|Aiken County|SC|24536|United States|-5|apartment| +47859|AAAAAAAADPKLAAAA|668|2nd 11th|Blvd|Suite N|Lebanon|Jackson County|IA|52898|United States|-6|condo| +47860|AAAAAAAAEPKLAAAA|839|North |Lane|Suite T|Plainview|Sanpete County|UT|83683|United States|-7|single family| +47861|AAAAAAAAFPKLAAAA|404|Hillcrest Broadway|Way|Suite 70|Roscoe|Volusia County|FL|31854|United States|-5|apartment| +47862|AAAAAAAAGPKLAAAA|52|Williams |Wy|Suite G|Doyle|Lake County|TN|38434|United States|-6|apartment| +47863|AAAAAAAAHPKLAAAA|370|1st Maple|Ct.|Suite S|Greenwood|Bedford County|TN|38828|United States|-5|apartment| +47864|AAAAAAAAIPKLAAAA|342|Spring |Pkwy|Suite M|Jamestown|Hamlin County|SD|56867|United States|-7|condo| +47865|AAAAAAAAJPKLAAAA|546||Cir.|Suite Y|Mountain View|Butte County|CA||United States|-8|apartment| +47866|AAAAAAAAKPKLAAAA|502|Chestnut Second|Dr.|Suite 240|Milan|Chaffee County|CO|86697|United States|-7|condo| +47867|AAAAAAAALPKLAAAA|940|Fifth South|Ln|Suite O|Hamilton|Fayette County|TN|32808|United States|-5|condo| +47868|AAAAAAAAMPKLAAAA|477|1st |Ct.|Suite S|Hamilton|Wayne County|IL|62808|United States|-6|apartment| +47869|AAAAAAAANPKLAAAA|549|North |Parkway|Suite T|Mount Pleasant|Crawford County|IL|61933|United States|-6|apartment| +47870|AAAAAAAAOPKLAAAA|110|14th |Court|Suite V|Crystal|Jefferson County|AL|35258|United States|-6|apartment| +47871|AAAAAAAAPPKLAAAA|106|Elm |Avenue|Suite M|Ryan|Valley County|MT|60525|United States|-7|single family| +47872|AAAAAAAAAALLAAAA|950|Fourth |Ct.|Suite 190|Midway|Scott County|VA|21904|United States|-5|condo| +47873|AAAAAAAABALLAAAA|267|Poplar Ash|Parkway|Suite 290|Riverdale|Caldwell County|KY|49391|United States|-6|apartment| +47874|AAAAAAAACALLAAAA|133|First Jackson|Blvd|Suite U|Clifton|Clarke County|IA|58014|United States|-6|apartment| +47875|AAAAAAAADALLAAAA|111|Mill Ash|Street|Suite 40|Mountain View|Shawano County|WI|54466|United States|-6|single family| +47876|AAAAAAAAEALLAAAA|846|6th |Cir.|Suite 60|Silver Creek|Hall County|GA|34546|United States|-5|apartment| +47877|AAAAAAAAFALLAAAA|766|15th |Way|Suite 470|Pine Grove|Arkansas County|AR|74593|United States|-6|condo| +47878|AAAAAAAAGALLAAAA|26|West |Avenue|Suite 20|Edgewood|Hart County|GA|30069|United States|-5|apartment| +47879|AAAAAAAAHALLAAAA|464|Hillcrest |Avenue|Suite I|Chapel Hill|Monroe County|IL|68124|United States|-6|apartment| +47880|AAAAAAAAIALLAAAA||Railroad 5th||Suite 170|Farmington|Hardin County|IA||United States||| +47881|AAAAAAAAJALLAAAA|238|East 5th|Dr.|Suite A|Woodville|Tama County|IA|54289|United States|-6|single family| +47882|AAAAAAAAKALLAAAA|398|View |Street|Suite 240|Clifton|Beauregard Parish|LA|78014|United States|-6|apartment| +47883|AAAAAAAALALLAAAA|990|Park Smith|Dr.|Suite 410|Silver Creek|Iberville Parish|LA|74546|United States|-6|condo| +47884|AAAAAAAAMALLAAAA|445||Wy||||GA|38252||-5|apartment| +47885|AAAAAAAANALLAAAA|356|6th Forest|Lane|Suite 470|Hamilton|Clay County|WV|22808|United States|-5|apartment| +47886|AAAAAAAAOALLAAAA|600|Fourth 1st|Boulevard|Suite Y|Newport|Iron County|UT|81521|United States|-7|single family| +47887|AAAAAAAAPALLAAAA|75|View ||Suite 60|||NC|||-5|| +47888|AAAAAAAAABLLAAAA|194|Eigth |Avenue|Suite 20|Harmony|Blaine County|NE|65804|United States|-6|single family| +47889|AAAAAAAABBLLAAAA|32|Hill |Drive|Suite 10|Forest Hills|Potter County|SD|59237|United States|-7|single family| +47890|AAAAAAAACBLLAAAA|741|Sunset |Court|Suite I|Edgewood|Madison County|OH|40069|United States|-5|condo| +47891|AAAAAAAADBLLAAAA|936|5th Park|Court|Suite I|Oak Grove|Wadena County|MN|58370|United States|-6|condo| +47892|AAAAAAAAEBLLAAAA|244|Oak |Ct.|Suite J|Globe|Stanton County|KS|69614|United States|-6|apartment| +47893|AAAAAAAAFBLLAAAA|517|Park Williams|Way|Suite 20|Concord|Mobile County|AL|34107|United States|-6|condo| +47894|AAAAAAAAGBLLAAAA|273|Sunset |Wy|Suite 100|Clinton|White County|IN|48222|United States|-5|single family| +47895|AAAAAAAAHBLLAAAA|502|Ridge First|Street|Suite G|Summit|Miner County|SD|50499|United States|-7|apartment| +47896|AAAAAAAAIBLLAAAA|939|Meadow |Dr.|Suite M|Newtown|Bexar County|TX|71749|United States|-6|condo| +47897|AAAAAAAAJBLLAAAA|522|12th |Blvd|Suite J|Star|Gates County|NC|20725|United States|-5|condo| +47898|AAAAAAAAKBLLAAAA|528|Jackson |Circle|Suite D|Georgetown|Jackson County|KS|67057|United States|-6|apartment| +47899|AAAAAAAALBLLAAAA|919|Laurel Elm|Boulevard|Suite 380|Brownsville|Jefferson County|ID|89310|United States|-7|condo| +47900|AAAAAAAAMBLLAAAA|935|Fourth |Ave|Suite 230|White Oak|Addison County|VT|07268|United States|-5|apartment| +47901|AAAAAAAANBLLAAAA|127|8th Wilson|Lane|Suite H|New Hope|Marshall County|IA|59431|United States|-6|single family| +47902|AAAAAAAAOBLLAAAA|254|Sycamore Sixth|Wy|Suite 290|Clifton|Hooker County|NE|68014|United States|-7|condo| +47903|AAAAAAAAPBLLAAAA|948|Franklin |Wy|Suite 480|Mount Vernon|Hardin County|IL|68482|United States|-6|apartment| +47904|AAAAAAAAACLLAAAA|258|Broadway North|Avenue|Suite W|Mount Pleasant|Powhatan County|VA|21933|United States|-5|apartment| +47905|AAAAAAAABCLLAAAA|809|Broadway |Drive|Suite W|Wolf Creek|Warren County|TN|32455|United States|-6|condo| +47906|AAAAAAAACCLLAAAA||Park |||Enterprise|Lapeer County||||-5|| +47907|AAAAAAAADCLLAAAA|381|Fifth Cherry|ST|Suite 370|Newtown|Ransom County|ND|51749|United States|-6|single family| +47908|AAAAAAAAECLLAAAA|475|West |Court|Suite 260|Bunker Hill|Candler County|GA|30150|United States|-5|condo| +47909|AAAAAAAAFCLLAAAA|153|Sixth Ridge|Lane|Suite 60|Bunker Hill|Baca County|CO|80150|United States|-7|single family| +47910|AAAAAAAAGCLLAAAA|992|Dogwood Maple|Ct.|Suite A|Franklin|Essex County|NY|19101|United States|-5|apartment| +47911|AAAAAAAAHCLLAAAA|985|Central Green|Parkway|Suite 420|Woodlawn|Rio Blanco County|CO|84098|United States|-7|condo| +47912|AAAAAAAAICLLAAAA|547|Lakeview 2nd|Road|Suite 350|Spring Hill|Boone County|KY|46787|United States|-6|single family| +47913|AAAAAAAAJCLLAAAA|739|14th 15th|Way|Suite G|Woodville|Crawford County|IL|64289|United States|-6|condo| +47914|AAAAAAAAKCLLAAAA|160|5th View|Way|Suite 480|Pleasant Hill|Columbia County|FL|33604|United States|-5|apartment| +47915|AAAAAAAALCLLAAAA|94|Sunset |Road|Suite 440|Valley View|Taylor County|GA|35124|United States|-5|single family| +47916|AAAAAAAAMCLLAAAA|975|Willow |Court|Suite G|Mount Zion|Adair County|IA|58054|United States|-6|condo| +47917|AAAAAAAANCLLAAAA|622|River 10th|Boulevard|Suite I|Newtown|Somerset County|ME|02349|United States|-5|apartment| +47918|AAAAAAAAOCLLAAAA|158|Central |Lane|Suite R|Red Hill|Blackford County|IN|44338|United States|-5|apartment| +47919|AAAAAAAAPCLLAAAA|478|Jefferson Woodland|Ln|Suite 390|Friendship|Tillamook County|OR|94536|United States|-8|condo| +47920|AAAAAAAAADLLAAAA|364|View |Drive|Suite 320|Buena Vista|Genesee County|NY|15752|United States|-5|condo| +47921|AAAAAAAABDLLAAAA|590|2nd Walnut|Parkway|Suite G|Oakdale|Lipscomb County|TX|79584|United States|-6|single family| +47922|AAAAAAAACDLLAAAA|992|View |Ct.|Suite R|Fairfield|Butte County|SD|56192|United States|-6|apartment| +47923|AAAAAAAADDLLAAAA|748|3rd Hill|Court|Suite 50|Oakwood|Polk County|MO|60169|United States|-6|single family| +47924|AAAAAAAAEDLLAAAA|413|Poplar 1st|Road|Suite 180|White Oak|Jasper County|TX|76668|United States|-6|single family| +47925|AAAAAAAAFDLLAAAA|949|Lake 2nd|Ave|Suite 190|Woodland|Wilson County|TX|74854|United States|-6|condo| +47926|AAAAAAAAGDLLAAAA|837|Park |Drive|Suite B|Five Forks|Roscommon County|MI|42293|United States|-5|single family| +47927|AAAAAAAAHDLLAAAA|356|6th Oak|Way|Suite X|Friendship|Gogebic County|MI|44536|United States|-5|single family| +47928|AAAAAAAAIDLLAAAA|3|6th |Wy|Suite 170|Farmington|Ionia County|MI|49145|United States|-5|condo| +47929|AAAAAAAAJDLLAAAA|267|Oak Woodland|Court|Suite 270|Ashland|Carter County|TN|34244|United States|-5|condo| +47930|AAAAAAAAKDLLAAAA|367|12th |Wy|Suite 250|Franklin|Bossier Parish|LA|79101|United States|-6|apartment| +47931|AAAAAAAALDLLAAAA|473|View |Way|Suite 320|Hillcrest|Perry County|TN|33003|United States|-6|condo| +47932|AAAAAAAAMDLLAAAA|719|5th East|Ct.|Suite 460|Pleasant Grove|Dougherty County|GA|34136|United States|-5|apartment| +47933|AAAAAAAANDLLAAAA|843|Madison Washington|Avenue|Suite 340|Pleasant Valley|Russell County|KY|42477|United States|-5|apartment| +47934|AAAAAAAAODLLAAAA|189|Cedar |Way|Suite N|Edgewood|Hartley County|TX|70069|United States|-6|condo| +47935|AAAAAAAAPDLLAAAA|774|Birch |Blvd|Suite H|Greenville|McCracken County|KY|41387|United States|-5|apartment| +47936|AAAAAAAAAELLAAAA|985|Park Center|ST|Suite S|Mountain View|Lapeer County|MI|44466|United States|-5|single family| +47937|AAAAAAAABELLAAAA|223|Lakeview |Pkwy|Suite R|Pleasant Valley|Teton County|WY|82477|United States|-7|single family| +47938|AAAAAAAACELLAAAA|||Way||Green Acres|Yakima County|WA||United States||single family| +47939|AAAAAAAADELLAAAA|||||Riverview||IN||||| +47940|AAAAAAAAEELLAAAA|599|||Suite 330|Hamilton|Stone County||72808|||apartment| +47941|AAAAAAAAFELLAAAA|433|View Lincoln|Road|Suite X|Valley View|Johnston County|OK|75124|United States|-6|condo| +47942|AAAAAAAAGELLAAAA|720|Park |Circle|Suite X|Lakeview|Marion County|MO|68579|United States|-6|single family| +47943|AAAAAAAAHELLAAAA|852|Oak |Cir.|Suite P|Kingston|Lamar County|AL|34975|United States|-6|single family| +47944|AAAAAAAAIELLAAAA|478|Church Birch|Parkway|Suite T|Guilford|Stokes County|NC|24408|United States|-5|apartment| +47945|AAAAAAAAJELLAAAA|281|Willow |Ln|Suite A|Five Forks|Henrico County|VA|22293|United States|-5|apartment| +47946|AAAAAAAAKELLAAAA|533|Forest Fifth|Blvd|Suite I|Jamestown|Boone County|AR|76867|United States|-6|single family| +47947|AAAAAAAALELLAAAA|762|Cedar Meadow|ST|Suite 90|Spring Hill|Huron County|MI|46787|United States|-5|single family| +47948|AAAAAAAAMELLAAAA|895|Oak |Blvd|Suite I|Oakdale|Lynchburg city|VA|29584|United States|-5|single family| +47949|AAAAAAAANELLAAAA|176|West |Circle|Suite 250|Five Points|Hardee County|FL|36098|United States|-5|single family| +47950|AAAAAAAAOELLAAAA|886|South Ridge|Pkwy|Suite 340|Waterloo|Rice County|KS|61675|United States|-6|apartment| +47951|AAAAAAAAPELLAAAA|107|Highland |Pkwy|Suite X|Franklin|Giles County|VA|29101|United States|-5|apartment| +47952|AAAAAAAAAFLLAAAA|679|9th |Wy|Suite 10|Rossville|Decatur County|IA|52503|United States|-6|apartment| +47953|AAAAAAAABFLLAAAA|967|Laurel |ST|Suite 180|Farmington|Murray County|OK|79145|United States|-6|condo| +47954|AAAAAAAACFLLAAAA|32|Ridge 5th|Ln|Suite W|Springfield|Izard County|AR|79303|United States|-6|apartment| +47955|AAAAAAAADFLLAAAA|977|Johnson Laurel|Road|Suite Y|Cordova|Scurry County|TX|76938|United States|-6|apartment| +47956|AAAAAAAAEFLLAAAA|954|Williams River|Cir.|Suite 250|Woodlawn|Heard County|GA|34098|United States|-5|single family| +47957|AAAAAAAAFFLLAAAA|55|1st |Street|Suite 390|Bethel|Barton County|KS|65281|United States|-6|single family| +47958|AAAAAAAAGFLLAAAA|189|River |Pkwy|Suite E|Enterprise|Duval County|FL|31757|United States|-5|apartment| +47959|AAAAAAAAHFLLAAAA|361|Railroad |Ave|||Decatur County|GA|36557||-5|apartment| +47960|AAAAAAAAIFLLAAAA|131|Hill |Circle|Suite D|Valley View|Gray County|KS|65124|United States|-6|condo| +47961|AAAAAAAAJFLLAAAA||||Suite M|Fairfield||UT||||| +47962|AAAAAAAAKFLLAAAA|542|7th Willow|Boulevard|Suite 330|New Hope|Knox County|TN|39431|United States|-6|condo| +47963|AAAAAAAALFLLAAAA|497|Lake North|Street|Suite K|Concord|Gosper County|NE|64107|United States|-6|apartment| +47964|AAAAAAAAMFLLAAAA|485|||Suite D|Oakdale||VA||United States|-5|| +47965|AAAAAAAANFLLAAAA|250|13th |Cir.|Suite R|Bunker Hill|Hood River County|OR|90150|United States|-8|single family| +47966|AAAAAAAAOFLLAAAA|394|Green |Ln|Suite G|Shiloh|Sabine Parish|LA|79275|United States|-6|apartment| +47967|AAAAAAAAPFLLAAAA|737|Cedar Main|Street|Suite M|Shiloh|Drew County|AR|79275|United States|-6|condo| +47968|AAAAAAAAAGLLAAAA|42|Green |Avenue|Suite E|Mount Pleasant|Delta County|MI|41933|United States|-5|single family| +47969|AAAAAAAABGLLAAAA|393|Maple |Ct.|Suite S|Arlington|Camden County|MO|66557|United States|-6|apartment| +47970|AAAAAAAACGLLAAAA|755|3rd 7th|Drive|Suite U|Newport|Reeves County|TX|71521|United States|-6|single family| +47971|AAAAAAAADGLLAAAA|659|Elm College|ST|Suite 30|Avery|Chattooga County|GA|30194|United States|-5|single family| +47972|AAAAAAAAEGLLAAAA|26|Sunset |Circle|Suite S|Roxbury|Roane County|WV|25508|United States|-5|condo| +47973|AAAAAAAAFGLLAAAA|738|Jackson |RD|Suite 470|Woodlawn|Eaton County|MI|44098|United States|-5|apartment| +47974|AAAAAAAAGGLLAAAA|494|Hillcrest |Street|Suite 130|Stringtown|Montgomery County|MD|20162|United States|-5|apartment| +47975|AAAAAAAAHGLLAAAA|739|Woodland Church|Court|Suite 460|Centerville|Jefferson County|WV|20059|United States|-5|condo| +47976|AAAAAAAAIGLLAAAA|903|Spring |Circle|Suite 400|Riverdale|Marshall County|IL|69391|United States|-6|apartment| +47977|AAAAAAAAJGLLAAAA|992|Ridge |Avenue|Suite 80|Jamestown|Grant County|WA|96867|United States|-8|condo| +47978|AAAAAAAAKGLLAAAA|173|West |RD|Suite D|Summit|Powder River County|MT|60499|United States|-7|single family| +47979|AAAAAAAALGLLAAAA|505|4th Poplar|Boulevard|Suite 40|Shady Grove|Tate County|MS|52812|United States|-6|single family| +47980|AAAAAAAAMGLLAAAA|79|Chestnut |Dr.|Suite 100|Lakeside|Clay County|KY|49532|United States|-6|apartment| +47981|AAAAAAAANGLLAAAA|294|Maple |Ct.|Suite S|Providence|McPherson County|SD|56614|United States|-7|condo| +47982|AAAAAAAAOGLLAAAA|391|Park 11th|Court|Suite V|Ashland|Martinsville city|VA|24244|United States|-5|single family| +47983|AAAAAAAAPGLLAAAA||6th Pine|||Glendale||||United States||| +47984|AAAAAAAAAHLLAAAA|||Street||Greenfield|Madison Parish|LA|75038|||| +47985|AAAAAAAABHLLAAAA|301|East |Cir.|Suite 280|Liberty|Washington County|NC|23451|United States|-5|apartment| +47986|AAAAAAAACHLLAAAA|490|Madison 1st|Ave|Suite B|Crossroads|Jennings County|IN|40534|United States|-5|apartment| +47987|AAAAAAAADHLLAAAA|541|2nd |Dr.|Suite M|Pleasant Grove|Gordon County|GA|34136|United States|-5|apartment| +47988|AAAAAAAAEHLLAAAA|319|Center |Dr.|Suite L|Wildwood|Monroe County|MS|56871|United States|-6|apartment| +47989|AAAAAAAAFHLLAAAA|962|12th Forest|Parkway|Suite 270|Union|Jenkins County|GA|38721|United States|-5|condo| +47990|AAAAAAAAGHLLAAAA|982|Spring |Way|Suite F|Red Hill|Vernon County|MO|64338|United States|-6|single family| +47991|AAAAAAAAHHLLAAAA|831|East |Parkway|Suite 360|Shiloh|Muskingum County|OH|49275|United States|-5|single family| +47992|AAAAAAAAIHLLAAAA|992|Main River|Street|Suite 430|Midway|Lowndes County|MS|51904|United States|-6|single family| +47993|AAAAAAAAJHLLAAAA|||Drive|Suite L|||LA|79840|||condo| +47994|AAAAAAAAKHLLAAAA|641|College |Lane|Suite J|Highland Park|Mono County|CA|96534|United States|-8|condo| +47995|AAAAAAAALHLLAAAA|793|North |Dr.|Suite 200|Lincoln|Hamilton County|IN|41289|United States|-5|apartment| +47996|AAAAAAAAMHLLAAAA|818|Church Adams|Cir.|Suite 10|Brownsville|Oswego County|NY|19310|United States|-5|single family| +47997|AAAAAAAANHLLAAAA|562|7th Park|Road|Suite 320|Edgewood|Troup County|GA|30069|United States|-5|apartment| +47998|AAAAAAAAOHLLAAAA|1|North 2nd|Circle|Suite 300|King|Paulding County|OH|40008|United States|-5|apartment| +47999|AAAAAAAAPHLLAAAA|267|13th |Ave|||||54098|||apartment| +48000|AAAAAAAAAILLAAAA|27|Williams |RD|Suite S|Ashland|Humboldt County|IA|54244|United States|-6|single family| +48001|AAAAAAAABILLAAAA|759|3rd Park|Wy|Suite K|Centerville|Clarke County|VA|20059|United States|-5|condo| +48002|AAAAAAAACILLAAAA|708|Broadway |Court|Suite E|Hamilton|Leon County|FL|32808|United States|-5|condo| +48003|AAAAAAAADILLAAAA|789|View Maple|ST|Suite 270|Harmony|Carter County|TN|35804|United States|-5|apartment| +48004|AAAAAAAAEILLAAAA|84|8th Maple|Ln|Suite K|Oakwood|Thomas County|KS|60169|United States|-6|single family| +48005|AAAAAAAAFILLAAAA|872|Miller |Cir.|Suite W|Fairview|Koochiching County|MN|55709|United States|-6|apartment| +48006|AAAAAAAAGILLAAAA|825|Elm |Dr.|Suite A|Post Oak|Lincoln County|ID|88567|United States|-7|single family| +48007|AAAAAAAAHILLAAAA|851|Central 1st|Street|Suite 150|Fairfield|Knox County|IL|66192|United States|-6|apartment| +48008|AAAAAAAAIILLAAAA|466|Hill Main|Cir.|Suite I|Stringtown|Alachua County|FL|30162|United States|-5|single family| +48009|AAAAAAAAJILLAAAA|461|Forest |Ln|Suite W|Unionville|Reeves County|TX|71711|United States|-6|apartment| +48010|AAAAAAAAKILLAAAA|271|1st South|ST|Suite Y|Arlington|Liberty County|MT|66557|United States|-7|condo| +48011|AAAAAAAALILLAAAA|457|Birch |Lane|Suite T|Woodville|Gratiot County|MI|44289|United States|-5|single family| +48012|AAAAAAAAMILLAAAA|980|Smith |Dr.|Suite A|Wesley|Oconee County|GA|31218|United States|-5|condo| +48013|AAAAAAAANILLAAAA|904|Maple |Road|Suite 250|Buena Vista|Martin County|TX|75752|United States|-6|apartment| +48014|AAAAAAAAOILLAAAA|109|Oak Adams|Avenue|Suite 50|Glenwood|Knox County|NE|63511|United States|-7|condo| +48015|AAAAAAAAPILLAAAA|247|Cherry Walnut|Wy|Suite T|Macedonia|Hickman County|TN|31087|United States|-5|single family| +48016|AAAAAAAAAJLLAAAA|826|Wilson Locust|Blvd|Suite 80|Lake Forest|Terrell County|GA|36000|United States|-5|apartment| +48017|AAAAAAAABJLLAAAA||Hickory |Road|Suite U|||||||apartment| +48018|AAAAAAAACJLLAAAA|731|6th |Lane|Suite R|Highland|Dodge County|MN|59454|United States|-6|apartment| +48019|AAAAAAAADJLLAAAA|775|Sunset Railroad|Court|Suite 340|Shady Grove|Fremont County|WY|82812|United States|-7|apartment| +48020|AAAAAAAAEJLLAAAA|886|Hickory |Parkway|Suite W|Valley View|Roberts County|TX|75124|United States|-6|condo| +48021|AAAAAAAAFJLLAAAA|656|First |Parkway|Suite 420|Highland|Dinwiddie County|VA|29454|United States|-5|condo| +48022|AAAAAAAAGJLLAAAA|133|Washington |RD|Suite W|Florence|Lawrence County|IL|63394|United States|-6|single family| +48023|AAAAAAAAHJLLAAAA|598|Laurel |Pkwy|Suite 240|Oakdale|Westmoreland County|PA|19584|United States|-5|apartment| +48024|AAAAAAAAIJLLAAAA|46|Park |Ct.|Suite E|Pine Hill|Porter County|IN|49236|United States|-5|single family| +48025|AAAAAAAAJJLLAAAA|166|Lake |Circle|Suite O|Walnut Grove|Carbon County|WY|87752|United States|-7|condo| +48026|AAAAAAAAKJLLAAAA|87|Valley Birch|Circle|Suite E|Bethel|San Patricio County|TX|75281|United States|-6|apartment| +48027|AAAAAAAALJLLAAAA|711|Willow Hill|Blvd|Suite G|Providence|Twiggs County|GA|36614|United States|-5|single family| +48028|AAAAAAAAMJLLAAAA|600|Sixth 7th|Avenue|Suite 220|Shelby|Fayette County|OH|46575|United States|-5|condo| +48029|AAAAAAAANJLLAAAA|78|Wilson Hill|Court|Suite S|Waterloo|Roosevelt County|MT|61675|United States|-7|single family| +48030|AAAAAAAAOJLLAAAA|952|Cedar |Court|Suite 130|Woodlawn|Kings County|CA|94098|United States|-8|condo| +48031|AAAAAAAAPJLLAAAA||||Suite A|Mount Pleasant||IL||United States||| +48032|AAAAAAAAAKLLAAAA|460|1st |Road|Suite 340|Hopewell|Mahaska County|IA|50587|United States|-6|condo| +48033|AAAAAAAABKLLAAAA|80|Twelfth Dogwood|Ct.|Suite M|Glendale|Brevard County|FL|33951|United States|-5|condo| +48034|AAAAAAAACKLLAAAA|247|Maple Wilson|Road|Suite F|Hopewell|Jones County|NC|20587|United States|-5|single family| +48035|AAAAAAAADKLLAAAA|469|Smith |Parkway|Suite 110|New Hope|La Salle County|IL|69431|United States|-6|single family| +48036|AAAAAAAAEKLLAAAA|916|4th Main|Way|Suite 180|Elm Grove|Ochiltree County|TX|73298|United States|-6|condo| +48037|AAAAAAAAFKLLAAAA|329|Miller |ST|Suite I|Stratford|Douglas County|WA|96668|United States|-8|apartment| +48038|AAAAAAAAGKLLAAAA|810|6th 5th|Road|Suite N|Jimtown|Real County|TX|77389|United States|-6|apartment| +48039|AAAAAAAAHKLLAAAA|741|Third 4th|Way|Suite 290|Pleasant Hill|Washington County|AR|73604|United States|-6|condo| +48040|AAAAAAAAIKLLAAAA|263|10th |Road|Suite 60|Sunnyside|Iowa County|WI|51952|United States|-6|single family| +48041|AAAAAAAAJKLLAAAA|994|Sycamore |Street|Suite 210|Greenwood|Lexington County|SC|28828|United States|-5|condo| +48042|AAAAAAAAKKLLAAAA|875|5th Cherry|ST|Suite I|Georgetown|Harrison County|OH|47057|United States|-5|single family| +48043|AAAAAAAALKLLAAAA|513|1st |Boulevard|Suite 230|Lakeside|Chautauqua County|KS|69532|United States|-6|condo| +48044|AAAAAAAAMKLLAAAA|336|North Dogwood|Street|Suite D|Woodlawn|Sherman County|OR|94098|United States|-8|condo| +48045|AAAAAAAANKLLAAAA|350|Hill |Avenue|Suite S|Liberty|Cuyahoga County|OH|43451|United States|-5|condo| +48046|AAAAAAAAOKLLAAAA|248|Main 7th|Avenue|Suite N|Red Hill|Collier County|FL|34338|United States|-5|apartment| +48047|AAAAAAAAPKLLAAAA|46|Willow |Boulevard|Suite Y|Five Forks|Dawson County|TX|72293|United States|-6|condo| +48048|AAAAAAAAALLLAAAA|141|Seventh 7th|Street|Suite L|Clinton|Cocke County|TN|38222|United States|-5|condo| +48049|AAAAAAAABLLLAAAA|570|Hillcrest |Road|Suite 20|Ellisville|Aleutians East Borough|AK|96820|United States|-9|condo| +48050|AAAAAAAACLLLAAAA|886|6th Walnut|Lane|Suite 40|Belmont|Griggs County|ND|50191|United States|-6|single family| +48051|AAAAAAAADLLLAAAA|246|7th |Court|Suite A|Bunker Hill|Scott County|MO|60150|United States|-6|condo| +48052|AAAAAAAAELLLAAAA|766|8th Lincoln|Ln|Suite K|Mount Olive|Red River Parish|LA|78059|United States|-6|single family| +48053|AAAAAAAAFLLLAAAA|106|Franklin |ST|Suite 30|Wolf Creek|Morgan County|UT|82455|United States|-7|condo| +48054|AAAAAAAAGLLLAAAA|693|Tenth Mill|Lane|Suite Y|Stringtown|McPherson County|NE|60162|United States|-7|condo| +48055|AAAAAAAAHLLLAAAA|466|Main Mill|ST|Suite 400|Riverdale|Fayette County|AL|39391|United States|-6|apartment| +48056|AAAAAAAAILLLAAAA|628|4th |Lane|Suite L|Jamestown|Powder River County|MT|66867|United States|-7|condo| +48057|AAAAAAAAJLLLAAAA|94|1st |Ave|Suite 210|Red Hill|Russell County|VA|24338|United States|-5|condo| +48058|AAAAAAAAKLLLAAAA|84|2nd |Court|Suite N|Clifton|Pasco County|FL|38014|United States|-5|condo| +48059|AAAAAAAALLLLAAAA|81|Wilson |Cir.|Suite O|Bethel|Northampton County|VA|25281|United States|-5|apartment| +48060|AAAAAAAAMLLLAAAA|321|3rd Park|ST|Suite R|Clinton|Wayne County|UT|88222|United States|-7|apartment| +48061|AAAAAAAANLLLAAAA|805|Elm West|Road|Suite 230|Enterprise|Stewart County|GA|31757|United States|-5|single family| +48062|AAAAAAAAOLLLAAAA|760|River |Road|Suite 50|Pinecrest|Washington County|ID|89981|United States|-7|single family| +48063|AAAAAAAAPLLLAAAA|923|Sunset |Lane|Suite 400|Concord|Clay County|TN|34107|United States|-5|condo| +48064|AAAAAAAAAMLLAAAA|685|Cedar Fifth|Blvd|Suite E|Walnut Grove|Union County|PA|17752|United States|-5|single family| +48065|AAAAAAAABMLLAAAA|91|Main Spruce|Street|Suite 430|Newport|Bandera County|TX|71521|United States|-6|condo| +48066|AAAAAAAACMLLAAAA|371|Williams |Ave|Suite 80|Springfield|Lyon County|MN|59303|United States|-6|apartment| +48067|AAAAAAAADMLLAAAA|632|Main |Avenue|Suite 110|Bunker Hill|Cass County|IN|40150|United States|-5|apartment| +48068|AAAAAAAAEMLLAAAA|360|2nd |Road|Suite 20|Providence|Putnam County|NY|16614|United States|-5|single family| +48069|AAAAAAAAFMLLAAAA|893|7th |Dr.|Suite 480|Highland|Knox County|OH|49454|United States|-5|apartment| +48070|AAAAAAAAGMLLAAAA|624|Hickory Fourth|Drive|Suite I|Langdon|Fayette County|IL|60852|United States|-6|apartment| +48071|AAAAAAAAHMLLAAAA|103|Second |Ln|Suite Y|Brentwood|Butte County|SD|54188|United States|-6|condo| +48072|AAAAAAAAIMLLAAAA|879|Maple Lake|Drive|Suite T|Antioch|Riley County|KS|68605|United States|-6|single family| +48073|AAAAAAAAJMLLAAAA|228|East Fourth|Wy|Suite Q|Arlington|Ulster County|NY|16557|United States|-5|single family| +48074|AAAAAAAAKMLLAAAA|635|Meadow Railroad|Blvd|Suite L|Summit|Livingston County|NY|10499|United States|-5|apartment| +48075|AAAAAAAALMLLAAAA|751|14th |Dr.|Suite 210|Cordova|Custer County|CO|86938|United States|-7|single family| +48076|AAAAAAAAMMLLAAAA|76|Third |Court|Suite A|Stafford|Evans County|GA|34980|United States|-5|condo| +48077|AAAAAAAANMLLAAAA|896|1st |Ave|Suite C|Oakland|Randall County|TX|79843|United States|-6|condo| +48078|AAAAAAAAOMLLAAAA|420|2nd View|Wy|Suite 340|Plainview|Palm Beach County|FL|33683|United States|-5|single family| +48079|AAAAAAAAPMLLAAAA|425|Chestnut |Pkwy|Suite 290|Summerville|Chattooga County|GA|32033|United States|-5|condo| +48080|AAAAAAAAANLLAAAA|119|Railroad |Avenue|Suite D|Harmony|Solano County|CA|95804|United States|-8|single family| +48081|AAAAAAAABNLLAAAA|360|Lakeview Jackson|Way|Suite 240|Macedonia|Wayne County|KY|41087|United States|-5|apartment| +48082|AAAAAAAACNLLAAAA|614||Drive||Waterloo|Martin County|KY||United States|-5|single family| +48083|AAAAAAAADNLLAAAA|588|West |Ave|Suite B|Kingston|Bannock County|ID|84975|United States|-7|condo| +48084|AAAAAAAAENLLAAAA|376|Second |Ln|Suite H|Lakeview|Edgefield County|SC|28579|United States|-5|single family| +48085|AAAAAAAAFNLLAAAA|39|College |Cir.|Suite E|New Hope|Sterling County|TX|79431|United States|-6|apartment| +48086|AAAAAAAAGNLLAAAA|516|6th |Drive|Suite D|Five Forks|Macon County|IL|62293|United States|-6|condo| +48087|AAAAAAAAHNLLAAAA|699|Franklin |Ln|Suite Q|Valley View|Halifax County|NC|25124|United States|-5|apartment| +48088|AAAAAAAAINLLAAAA|676|2nd Hickory|Ct.|Suite S|Springfield|Harding County|SD|59303|United States|-7|single family| +48089|AAAAAAAAJNLLAAAA|374|Davis Madison|Cir.|Suite 400|Marion|Renville County|ND|50399|United States|-6|single family| +48090|AAAAAAAAKNLLAAAA|4|Lincoln |Ln|Suite N|Woodville|Frio County|TX|74289|United States|-6|condo| +48091|AAAAAAAALNLLAAAA|||Road||Harmony|Mineral County||||-7|apartment| +48092|AAAAAAAAMNLLAAAA|3|5th |Circle|Suite 490|Oak Ridge|Lyon County|NV|88371|United States|-8|apartment| +48093|AAAAAAAANNLLAAAA|291|Jefferson Forest|Drive|Suite J|Clinton|Webster County|IA|58222|United States|-6|single family| +48094|AAAAAAAAONLLAAAA|736|Main Dogwood|Cir.|Suite E|Pine Grove|Marengo County|AL|34593|United States|-6|apartment| +48095|AAAAAAAAPNLLAAAA|475|Laurel |Avenue|Suite 220|Pleasant Valley|Wadena County|MN|52477|United States|-6|condo| +48096|AAAAAAAAAOLLAAAA|331|River |Ln|Suite N|Highland|Harvey County|KS|69454|United States|-6|single family| +48097|AAAAAAAABOLLAAAA|111|Highland |Circle|Suite 460|Hillcrest|Cecil County|MD|23003|United States|-5|single family| +48098|AAAAAAAACOLLAAAA|785|Hill Cedar|Pkwy|Suite 130|Lakeview|Mason County|KY|48579|United States|-5|apartment| +48099|AAAAAAAADOLLAAAA|326|Lake Cherry|Street|Suite 10|Belmont|Yolo County|CA|90191|United States|-8|apartment| +48100|AAAAAAAAEOLLAAAA|263|South Chestnut|Court|Suite O|Enterprise|Franklin County|IN|41757|United States|-5|condo| +48101|AAAAAAAAFOLLAAAA|881|Birch |Cir.|Suite 430|Waterloo|Woodford County|IL|61675|United States|-6|apartment| +48102|AAAAAAAAGOLLAAAA|590|Tenth |Ave|Suite I|Fairfield|Swain County|NC|26192|United States|-5|single family| +48103|AAAAAAAAHOLLAAAA|396|Third 2nd|Way|Suite W|Oakdale|Pettis County|MO|69584|United States|-6|condo| +48104|AAAAAAAAIOLLAAAA|875|Ridge Washington|Parkway|Suite 330|Fairview|Grant Parish|LA|75709|United States|-6|apartment| +48105|AAAAAAAAJOLLAAAA|142|Forest Forest|Blvd|Suite R|Arlington|Franklin County|MA|07157|United States|-5|condo| +48106|AAAAAAAAKOLLAAAA|528||Lane|Suite B|||OK|74593||-6|single family| +48107|AAAAAAAALOLLAAAA|469|Eigth |Wy|Suite 340|Stringtown|Jasper County|MS|50162|United States|-6|apartment| +48108|AAAAAAAAMOLLAAAA|588|Maple |Circle|Suite V|Oakdale|Washington County|CO|89584|United States|-7|single family| +48109|AAAAAAAANOLLAAAA|789|Ridge Main|Ct.|Suite 200|Hillcrest|Hill County|MT|63003|United States|-7|apartment| +48110|AAAAAAAAOOLLAAAA|64|Pine |Lane|Suite 400|Kingston|Hardin County|IA|54975|United States|-6|single family| +48111|AAAAAAAAPOLLAAAA|50|Cedar |Way|Suite 160|Woodland|Pennington County|MN|54854|United States|-6|apartment| +48112|AAAAAAAAAPLLAAAA|945|Park 7th|Parkway|Suite J|Bethesda|Deer Lodge County|||||| +48113|AAAAAAAABPLLAAAA|243|Sunset Spring|Court|Suite 490|Providence|Orange County|NY|16614|United States|-5|condo| +48114|AAAAAAAACPLLAAAA|586|Pine Cherry|Street|Suite U|Deerfield|Muscogee County|GA|39840|United States|-5|condo| +48115|AAAAAAAADPLLAAAA|503|Park |Circle|Suite 240|Midway|Madison County|ID|81904|United States|-7|single family| +48116|AAAAAAAAEPLLAAAA|387|East Lake|Boulevard|Suite 80|Antioch|Luce County|MI|48605|United States|-5|apartment| +48117|AAAAAAAAFPLLAAAA|571|Center Second|Ct.|Suite G|Mount Pleasant|Attala County|MS|51933|United States|-6|apartment| +48118|AAAAAAAAGPLLAAAA|616|Spring Fourth|Street|Suite H|Five Points|Roger Mills County|OK|76098|United States|-6|condo| +48119|AAAAAAAAHPLLAAAA|993|1st Miller|Lane|Suite N|Macedonia|Calhoun County|MI|41087|United States|-5|single family| +48120|AAAAAAAAIPLLAAAA|995|Railroad |Court|Suite 170|Bridgeport|Benson County|ND|55817|United States|-6|apartment| +48121|AAAAAAAAJPLLAAAA|754|2nd |Lane|Suite V|Riverview|Hampton County|SC|29003|United States|-5|single family| +48122|AAAAAAAAKPLLAAAA|538|South Broadway|Court|Suite R|Unionville|Putnam County|IN|41711|United States|-5|condo| +48123|AAAAAAAALPLLAAAA|203|Park Cherry|Parkway|Suite 430|Greenfield|San Juan County|WA|95038|United States|-8|apartment| +48124|AAAAAAAAMPLLAAAA|666|Hickory 5th|Court|Suite G|Oak Hill|Mercer County|NJ|08438|United States|-5|condo| +48125|AAAAAAAANPLLAAAA|143|Valley |Way|Suite 190|Mount Vernon|Edwards County|IL|68482|United States|-6|single family| +48126|AAAAAAAAOPLLAAAA||Spring East|||||WA|96867||-8|single family| +48127|AAAAAAAAPPLLAAAA|728|Sunset 5th|Parkway|Suite 370|Mountain View|Taos County|NM|84466|United States|-7|apartment| +48128|AAAAAAAAAAMLAAAA|866|Maple 5th|Street|Suite A|Hamilton|Dale County|AL|32808|United States|-6|single family| +48129|AAAAAAAABAMLAAAA|514|Franklin Franklin|ST|Suite T|Riverside|Citrus County|FL|39231|United States|-5|single family| +48130|AAAAAAAACAMLAAAA|892|Jefferson Smith|Parkway|Suite 340|Clifton|De Soto Parish|LA|78014|United States|-6|apartment| +48131|AAAAAAAADAMLAAAA|201|Wilson |Circle|Suite 110|Summit|Dougherty County|GA|30499|United States|-5|apartment| +48132|AAAAAAAAEAMLAAAA|601|2nd |Pkwy|Suite C|Mount Olive|Hemphill County|TX|78059|United States|-6|apartment| +48133|AAAAAAAAFAMLAAAA|272|Central |Ct.|Suite 370|Pleasant Hill|Winnebago County|IL|63604|United States|-6|condo| +48134|AAAAAAAAGAMLAAAA|763|Locust |Avenue|Suite 110|Wilson|Montgomery County|TN|36971|United States|-6|apartment| +48135|AAAAAAAAHAMLAAAA|322|Woodland Broadway|Boulevard|Suite 340|Riverside|Kodiak Island Borough|AK|99231|United States|-9|condo| +48136|AAAAAAAAIAMLAAAA|794|Second 2nd|Road|Suite R|Cedar Grove|Beaver County|PA|10411|United States|-5|apartment| +48137|AAAAAAAAJAMLAAAA|993|6th Franklin|Road|Suite T|Forest Hills|Willacy County|TX|79237|United States|-6|apartment| +48138|AAAAAAAAKAMLAAAA|80|Lakeview |RD|Suite 310|Bethel|Dyer County|TN|35281|United States|-5|apartment| +48139|AAAAAAAALAMLAAAA|97|Madison |Drive|Suite M|Parkwood|Johnston County|NC|21669|United States|-5|single family| +48140|AAAAAAAAMAMLAAAA|626|4th |RD|Suite 170|Mount Vernon|Susquehanna County|PA|18482|United States|-5|condo| +48141|AAAAAAAANAMLAAAA|802|Woodland Forest|Drive|Suite 70|Pine Grove|Campbell County|TN|34593|United States|-5|condo| +48142|AAAAAAAAOAMLAAAA|372|Hickory Ash|Circle|Suite 110|Bethel|Adams County|CO|85281|United States|-7|apartment| +48143|AAAAAAAAPAMLAAAA|856|Johnson |Lane|Suite 130|Watkins|Burke County|NC|21732|United States|-5|single family| +48144|AAAAAAAAABMLAAAA|209|Washington |ST|Suite 340|Spring Hill|Chippewa County|WI|56787|United States|-6|single family| +48145|AAAAAAAABBMLAAAA|751|Oak Oak|Ct.|Suite H|Lincoln|Sussex County|NJ|01889|United States|-5|single family| +48146|AAAAAAAACBMLAAAA|249|North |ST|Suite 210|Greenville|Brunswick County|NC|21387|United States|-5|single family| +48147|AAAAAAAADBMLAAAA|221|6th 6th|Road|Suite K|Forest Hills|Ohio County|KY|49237|United States|-5|apartment| +48148|AAAAAAAAEBMLAAAA|564|Adams Walnut|RD|Suite 130|Five Points|Payette County|ID|86098|United States|-7|condo| +48149|AAAAAAAAFBMLAAAA|159|Center Forest|Dr.|Suite 450|Highland|Wood County|TX|79454|United States|-6|apartment| +48150|AAAAAAAAGBMLAAAA|980|First |Court|Suite 410|Harmony|Roanoke County|VA|25804|United States|-5|condo| +48151|AAAAAAAAHBMLAAAA|500|1st |Circle|Suite A|Clifton|Greene County|OH|48014|United States|-5|single family| +48152|AAAAAAAAIBMLAAAA|318|Locust Central|Ct.|Suite 230|Fairbanks|Moffat County|CO|86653|United States|-7|apartment| +48153|AAAAAAAAJBMLAAAA|39|Maple Jackson|Way|Suite 110|Indian Village|Russell County|KY|41075|United States|-5|apartment| +48154|AAAAAAAAKBMLAAAA|6|First |Ave|Suite 200|Union Hill|Tillman County|OK|77746|United States|-6|condo| +48155|AAAAAAAALBMLAAAA|350|14th Smith|Dr.|Suite K|Mountain View|Seward County|NE|64466|United States|-7|apartment| +48156|AAAAAAAAMBMLAAAA|407|Ridge |Lane|Suite S|Florence|Monroe County|PA|13394|United States|-5|apartment| +48157|AAAAAAAANBMLAAAA|923|Church |Drive|Suite 150|New Hope|Sedgwick County|KS|69431|United States|-6|condo| +48158|AAAAAAAAOBMLAAAA|240|Highland |Road|Suite 310|Waterloo|Jeff Davis County|TX|71675|United States|-6|apartment| +48159|AAAAAAAAPBMLAAAA|19|Lakeview |Boulevard|Suite 230|Omega|Dent County|MO|66364|United States|-6|single family| +48160|AAAAAAAAACMLAAAA|480|Cedar |Lane|Suite N|Lincoln|Lincoln County|MN|51289|United States|-6|condo| +48161|AAAAAAAABCMLAAAA|883|3rd Maple|Drive|Suite Y|Wilson|Greene County|NC|26971|United States|-5|condo| +48162|AAAAAAAACCMLAAAA|604|Spruce |Drive|Suite 100|Lakeview|Blackford County|IN|48579|United States|-5|condo| +48163|AAAAAAAADCMLAAAA|170|Cherry |Pkwy|Suite Y|Oak Hill|Sitka Borough|AK|97838|United States|-9|single family| +48164|AAAAAAAAECMLAAAA||West Fourth|Dr.|Suite 230||Lincoln County||38721|United States|-5|| +48165|AAAAAAAAFCMLAAAA|695|14th Fifth|Road|Suite H|Farmington|McNairy County|TN|39145|United States|-6|apartment| +48166|AAAAAAAAGCMLAAAA|908|Hillcrest Lee|RD|Suite V|Macedonia|Crosby County|TX|71087|United States|-6|single family| +48167|AAAAAAAAHCMLAAAA|108|Oak |Avenue|Suite 380|Jamestown|Wahkiakum County|WA|96867|United States|-8|condo| +48168|AAAAAAAAICMLAAAA||Elm Williams||Suite 350|Springhill|||||-6|condo| +48169|AAAAAAAAJCMLAAAA|443|13th Second|Avenue|Suite 290|Mount Vernon|Dixie County|FL|38482|United States|-5|condo| +48170|AAAAAAAAKCMLAAAA|836|Johnson |Ct.|Suite F|Lebanon|Clay County|WV|22898|United States|-5|condo| +48171|AAAAAAAALCMLAAAA|914|Twelfth 3rd|Pkwy|Suite 60|Bridgeport|San Joaquin County|CA|95817|United States|-8|condo| +48172|AAAAAAAAMCMLAAAA|539|1st |Wy|Suite V|Forest Hills|Tift County|GA|39237|United States|-5|condo| +48173|AAAAAAAANCMLAAAA|700|Maple Hickory|Ave|Suite Q|Providence|Wallowa County|OR|96614|United States|-8|condo| +48174|AAAAAAAAOCMLAAAA|101|Williams North|Court|Suite L|Five Points|Lee County|KY|46098|United States|-5|apartment| +48175|AAAAAAAAPCMLAAAA|107|Sixth Spruce|Cir.|Suite P|Glenwood|Carroll County|IA|53511|United States|-6|apartment| +48176|AAAAAAAAADMLAAAA|893|Locust |Dr.|Suite 470|Lincoln|Wayne County|MS|51289|United States|-6|apartment| +48177|AAAAAAAABDMLAAAA|842|Williams Locust|Boulevard|Suite 440|Red Hill|Cache County|UT|84338|United States|-7|apartment| +48178|AAAAAAAACDMLAAAA|363|Church |Blvd|Suite G|Bridgeport|Gillespie County|TX|75817|United States|-6|apartment| +48179|AAAAAAAADDMLAAAA|624|Johnson |Pkwy|Suite 40|Mount Olive|Sevier County|AR|78059|United States|-6|condo| +48180|AAAAAAAAEDMLAAAA|751|Jackson Oak|Parkway|Suite U|Union Hill|Midland County|TX|77746|United States|-6|apartment| +48181|AAAAAAAAFDMLAAAA|||||Deerfield|Franklin County|||United States|-5|single family| +48182|AAAAAAAAGDMLAAAA|232|Woodland |Avenue|Suite 200|Valley View|Kleberg County|TX|75124|United States|-6|apartment| +48183|AAAAAAAAHDMLAAAA|808|Washington |Ave|Suite 280|Union Hill|Lee County|IA|57746|United States|-6|apartment| +48184|AAAAAAAAIDMLAAAA|143|Walnut Cherry|Drive|Suite 70|Johnsonville|Nuckolls County|NE|67745|United States|-7|single family| +48185|AAAAAAAAJDMLAAAA|352|Railroad |Ln|Suite 350|Vienna|Sullivan County|PA|15119|United States|-5|condo| +48186|AAAAAAAAKDMLAAAA|980|Chestnut |Road|Suite 340|Five Points|Wythe County|VA|26098|United States|-5|condo| +48187|AAAAAAAALDMLAAAA|938|Cherry 4th|Ave|Suite 340|Pine Grove|Lassen County|CA|94593|United States|-8|single family| +48188|AAAAAAAAMDMLAAAA|795|Davis |Drive|Suite Y|Highland|Ford County|IL|69454|United States|-6|apartment| +48189|AAAAAAAANDMLAAAA|571|North |ST|Suite S|Cedar Grove|Whitley County|IN|40411|United States|-5|condo| +48190|AAAAAAAAODMLAAAA|245|9th |RD|Suite 360|Midway|Wasco County|OR|91904|United States|-8|apartment| +48191|AAAAAAAAPDMLAAAA|59|College Park|Pkwy|Suite 320|Hillcrest|Glasscock County|TX|73003|United States|-6|single family| +48192|AAAAAAAAAEMLAAAA|74||Way|Suite 160||Hamilton County|||||condo| +48193|AAAAAAAABEMLAAAA|143|12th Park|RD|Suite F|Highland Park|Columbia County|FL|36534|United States|-5|condo| +48194|AAAAAAAACEMLAAAA|309|Tenth Main|RD|Suite E|Sutton|Georgetown County|SC|25413|United States|-5|condo| +48195|AAAAAAAADEMLAAAA|269|4th |ST|Suite P|Greenwood|Meade County|KS|68828|United States|-6|apartment| +48196|AAAAAAAAEEMLAAAA|189|Second 12th|Pkwy|Suite E|Palmyra|Roosevelt County|MT|67646|United States|-7|condo| +48197|AAAAAAAAFEMLAAAA|769|Oak Pine|Circle|Suite 340|Greenville|Keith County|NE|61387|United States|-7|condo| +48198|AAAAAAAAGEMLAAAA|161|Cedar Lake|Wy|Suite 20|Glendale|Sedgwick County|KS||United States|-6|| +48199|AAAAAAAAHEMLAAAA|400|Maple Walnut|Ave|Suite 460|Spring Valley|Emmet County|MI|46060|United States|-5|apartment| +48200|AAAAAAAAIEMLAAAA|762|Hickory 14th|Dr.|Suite 350|Wildwood|Telfair County|GA|36871|United States|-5|single family| +48201|AAAAAAAAJEMLAAAA|396|Cedar Pine|Way|Suite 90|Lakeside|Colleton County|SC|29532|United States|-5|single family| +48202|AAAAAAAAKEMLAAAA|73|Third Birch|Pkwy|Suite 180|Salem|Caddo Parish|LA|78048|United States|-6|single family| +48203|AAAAAAAALEMLAAAA|536|Ridge Sixth|Ave|Suite 150|Peoria|Garland County|AR|79818|United States|-6|apartment| +48204|AAAAAAAAMEMLAAAA|207|5th |Ave|Suite B|Fairview|Osage County|KS|65709|United States|-6|single family| +48205|AAAAAAAANEMLAAAA|90|Dogwood |Lane|Suite 260|Glendale|Navajo County|AZ|83951|United States|-7|condo| +48206|AAAAAAAAOEMLAAAA|966|6th Franklin|Cir.|Suite J|Greenwood|Camden County|MO|68828|United States|-6|apartment| +48207|AAAAAAAAPEMLAAAA|26|10th 4th|ST|Suite 490|Woodland|Shelby County|TN|34854|United States|-6|condo| +48208|AAAAAAAAAFMLAAAA|691|Park |Ln|Suite E|Glendale|Livingston Parish|LA|73951|United States|-6|apartment| +48209|AAAAAAAABFMLAAAA|322|Fourth 3rd|Street|Suite 40|Union Hill|Sumter County|AL|37746|United States|-6|apartment| +48210|AAAAAAAACFMLAAAA||Madison ||Suite D|||UT|82898||-7|| +48211|AAAAAAAADFMLAAAA|656|Fifth |Avenue|Suite 330|Green Acres|Allegany County|NY|17683|United States|-5|apartment| +48212|AAAAAAAAEFMLAAAA|641|Sycamore |Way|Suite S|Arlington|Bradley County|TN|36557|United States|-5|condo| +48213|AAAAAAAAFFMLAAAA|899|Locust |Cir.|Suite 280|Shiloh|Jim Hogg County|TX|79275|United States|-6|condo| +48214|AAAAAAAAGFMLAAAA|918|Main |Dr.|Suite H|Mount Vernon|Martin County|NC|28482|United States|-5|apartment| +48215|AAAAAAAAHFMLAAAA|760|Woodland Sunset|Circle|Suite Q|Blair|Bedford County|VA|25465|United States|-5|apartment| +48216|AAAAAAAAIFMLAAAA|731|Main Spring|Blvd|Suite N|Shannon|Alleghany County|VA|24120|United States|-5|single family| +48217|AAAAAAAAJFMLAAAA|70|Second River|Parkway|Suite V|Wildwood|Latimer County|OK|76871|United States|-6|apartment| +48218|AAAAAAAAKFMLAAAA|882|Elm Franklin|Blvd|Suite 20|Fairview|Yellowstone County|MT|65709|United States|-7|single family| +48219|AAAAAAAALFMLAAAA|538|Johnson Second|Boulevard|Suite 490|Mount Olive|Lincoln Parish|LA|78059|United States|-6|apartment| +48220|AAAAAAAAMFMLAAAA|156|Valley Church|Ln|Suite 0|Edgewood|Rusk County|WI|50069|United States|-6|condo| +48221|AAAAAAAANFMLAAAA|477|Franklin |Lane|Suite K|Stratford|Carroll County|MS|56668|United States|-6|apartment| +48222|AAAAAAAAOFMLAAAA|411|North Cedar|Avenue|Suite 10|Pleasant Hill|McLean County|ND|53604|United States|-6|apartment| +48223|AAAAAAAAPFMLAAAA|89|10th Ridge|Blvd|Suite 280|Edgewood|Monroe County|IL|60069|United States|-6|single family| +48224|AAAAAAAAAGMLAAAA|182|5th |Avenue|Suite X|Pleasant Hill|Polk County|AR|73604|United States|-6|condo| +48225|AAAAAAAABGMLAAAA|404|View Elm|Circle|Suite T|Hamilton|Williamson County|TN|32808|United States|-5|condo| +48226|AAAAAAAACGMLAAAA|482|Meadow 7th|Drive|Suite 30|Greenville|Will County|IL|61387|United States|-6|apartment| +48227|AAAAAAAADGMLAAAA|931|Washington Jackson|Way|Suite V|Marion|Gallatin County|MT|60399|United States|-7|single family| +48228|AAAAAAAAEGMLAAAA|708|View 2nd|Way|Suite P|Mount Pleasant|Kendall County|IL|61933|United States|-6|single family| +48229|AAAAAAAAFGMLAAAA|10|Highland |Boulevard|Suite U|Westgate|Lake County|CO|82366|United States|-7|single family| +48230|AAAAAAAAGGMLAAAA|760|North |Lane|Suite R|Wilson|Howard County|NE|66971|United States|-7|single family| +48231|AAAAAAAAHGMLAAAA|210|Center |Ave|Suite 30|Northwood|Lawrence County|TN|34104|United States|-6|single family| +48232|AAAAAAAAIGMLAAAA|290|6th |Wy|Suite Y|Deerfield|Taylor County|WV|29840|United States|-5|condo| +48233|AAAAAAAAJGMLAAAA|866|6th Broadway|Pkwy|Suite X|White Oak|Emporia city|VA|26668|United States|-5|condo| +48234|AAAAAAAAKGMLAAAA|447|Wilson |Dr.|Suite G|Pleasant Grove|Clayton County|IA|54136|United States|-6|apartment| +48235|AAAAAAAALGMLAAAA|97|Hickory |Lane|Suite 410|Union Hill|Carter County|MO|67746|United States|-6|condo| +48236|AAAAAAAAMGMLAAAA|240|Willow Chestnut|Parkway|Suite F|Spring Valley|Robeson County|NC|26060|United States|-5|single family| +48237|AAAAAAAANGMLAAAA|720|South |Boulevard|Suite 60|Unionville|Addison County|VT|02311|United States|-5|condo| +48238|AAAAAAAAOGMLAAAA|470|Cedar Franklin|Street|Suite W|Lincoln|Marion County|IN|41289|United States|-5|condo| +48239|AAAAAAAAPGMLAAAA|515|Lincoln Church|Ct.|Suite X|Lakeview|East Carroll Parish|LA|78579|United States|-6|single family| +48240|AAAAAAAAAHMLAAAA|753|Sycamore Railroad|Way|Suite 390|Bridgeport|Wayne County|KY|45817|United States|-5|condo| +48241|AAAAAAAABHMLAAAA|984|Walnut |Circle|Suite C|Mount Olive|Smyth County|VA|28059|United States|-5|single family| +48242|AAAAAAAACHMLAAAA|337|Johnson 1st|Road|Suite 200|Newport|Gates County|NC|21521|United States|-5|condo| +48243|AAAAAAAADHMLAAAA|203|Park Pine|Court|Suite 290|Valley View|Kent County|DE|15124|United States|-5|condo| +48244|AAAAAAAAEHMLAAAA|536|Locust Sixth|ST|Suite 260|Rosebud|Sheridan County|NE|62244|United States|-7|apartment| +48245|AAAAAAAAFHMLAAAA|798|Main |Ct.|Suite Q|Florence|Camden County|NJ|03994|United States|-5|apartment| +48246|AAAAAAAAGHMLAAAA|472|9th Forest|Cir.|Suite N|Pleasant Valley|La Paz County|AZ|82477|United States|-7|single family| +48247|AAAAAAAAHHMLAAAA|388|Ridge |ST|Suite 300|Lakeview|Appling County|GA|38579|United States|-5|single family| +48248|AAAAAAAAIHMLAAAA|475|Oak Madison|ST|Suite 370|Union|Camden County|MO|68721|United States|-6|apartment| +48249|AAAAAAAAJHMLAAAA|809|Mill Poplar|Ct.|Suite B|Franklin|Rabun County|GA|39101|United States|-5|apartment| +48250|AAAAAAAAKHMLAAAA|487|Jackson 11th|Ct.|Suite 140|Pleasant Valley|Berkeley County|WV|22477|United States|-5|single family| +48251|AAAAAAAALHMLAAAA|649|Walnut |Ln|Suite M|Red Hill|Fayette County|PA|14338|United States|-5|single family| +48252|AAAAAAAAMHMLAAAA|735|Maple Washington|RD|Suite D|Franklin|Cuming County|NE|69101|United States|-6|condo| +48253|AAAAAAAANHMLAAAA|400|Park Valley|RD|Suite R|Highland|Okfuskee County|OK|79454|United States|-6|apartment| +48254|AAAAAAAAOHMLAAAA|70|7th Maple|ST|Suite H|Greenville|Summit County|OH|41387|United States|-5|apartment| +48255|AAAAAAAAPHMLAAAA|585|Spring |Ct.|Suite 460|Pine Grove|Randall County|TX|74593|United States|-6|single family| +48256|AAAAAAAAAIMLAAAA|701|Smith |Cir.|Suite E|Jamestown|Santa Cruz County|AZ|86867|United States|-7|apartment| +48257|AAAAAAAABIMLAAAA|158|15th Meadow|Cir.|Suite 380|Amity|Bennington County|VT|01366|United States|-5|single family| +48258|AAAAAAAACIMLAAAA|153|Lake Park|Blvd|Suite 270|Springtown|Buffalo County|SD|59858|United States|-6|apartment| +48259|AAAAAAAADIMLAAAA|369|View |Parkway|Suite M|Macedonia|Logan County|IL|61087|United States|-6|single family| +48260|AAAAAAAAEIMLAAAA|885|Park Park|Circle|Suite W|Bath|Boise County|ID|80573|United States|-7|single family| +48261|AAAAAAAAFIMLAAAA|914|Park |Ave|Suite X|Lakeside|Archuleta County|CO|89532|United States|-7|single family| +48262|AAAAAAAAGIMLAAAA|786|Seventh 5th|Avenue|Suite 410|Franklin|Bourbon County|KY|49101|United States|-6|condo| +48263|AAAAAAAAHIMLAAAA|627|Third 3rd||Suite T|Oak Grove||IA|58370|||| +48264|AAAAAAAAIIMLAAAA|602|Lake |Circle|Suite 130|Hopewell|Morrison County|MN|50587|United States|-6|single family| +48265|AAAAAAAAJIMLAAAA|405|8th |RD|Suite 10|Oakwood|Lucas County|OH|40169|United States|-5|single family| +48266|AAAAAAAAKIMLAAAA|271|Railroad |Road|Suite 110|Kingston|Orangeburg County|SC|24975|United States|-5|apartment| +48267|AAAAAAAALIMLAAAA|239|Franklin 7th|Street|Suite C|Liberty|Henderson County|TX|73451|United States|-6|condo| +48268|AAAAAAAAMIMLAAAA|625|River |Avenue|Suite 230|Siloam|Montezuma County|CO|88948|United States|-7|apartment| +48269|AAAAAAAANIMLAAAA|519|Washington |Wy|Suite 180|Lakeside|Swisher County|TX|79532|United States|-6|apartment| +48270|AAAAAAAAOIMLAAAA|208|15th Pine|Ct.|Suite J|Deerfield|Jefferson County|TX|79840|United States|-6|condo| +48271|AAAAAAAAPIMLAAAA|78|Third |Road|Suite K|Brownsville|York County|VA|29310|United States|-5|single family| +48272|AAAAAAAAAJMLAAAA|609|North |Lane|Suite C|Clifford|Ogemaw County|MI|48164|United States|-5|apartment| +48273|AAAAAAAABJMLAAAA|104|First |Road|Suite 360|Wildwood|Coshocton County|OH|46871|United States|-5|apartment| +48274|AAAAAAAACJMLAAAA|912|Cedar |Wy|Suite 0|Union|Baca County|CO|88721|United States|-7|condo| +48275|AAAAAAAADJMLAAAA|817|Dogwood South|Parkway|Suite 180|Avoca|Comanche County|TX|70540|United States|-6|single family| +48276|AAAAAAAAEJMLAAAA|791|6th Central|Road|Suite 320|Springdale|Monmouth County|NJ|09483|United States|-5|apartment| +48277|AAAAAAAAFJMLAAAA|658|Dogwood East|Way|Suite V|Farmington|Vermilion County|IL|69145|United States|-6|apartment| +48278|AAAAAAAAGJMLAAAA|371|8th Davis|Avenue|Suite E|Deerfield|Rolette County|ND|59840|United States|-6|single family| +48279|AAAAAAAAHJMLAAAA|608|Jackson Third|Dr.|Suite N|Summit|Marion County|TX|70499|United States|-6|condo| +48280|AAAAAAAAIJMLAAAA|251|Davis Madison|Lane|Suite D|Wilson|Grant County|OK|76971|United States|-6|condo| +48281|AAAAAAAAJJMLAAAA|31|Hickory Johnson|RD|Suite V|Shiloh|Robertson County|TN|39275|United States|-6|single family| +48282|AAAAAAAAKJMLAAAA|664|Railroad |Lane|Suite 230|Midway|Tucker County|WV|21904|United States|-5|condo| +48283|AAAAAAAALJMLAAAA|43|4th |Parkway|Suite 140|Oakdale|Coffee County|GA|39584|United States|-5|apartment| +48284|AAAAAAAAMJMLAAAA|474|Johnson View|Boulevard|Suite D|Ryan|Marion County|KY|40525|United States|-5|single family| +48285|AAAAAAAANJMLAAAA|341|West |Court|Suite 400|Harmony|Bradley County|TN|35804|United States|-5|condo| +48286|AAAAAAAAOJMLAAAA|510|Lincoln |Cir.|Suite T|Lebanon|Montgomery County|MD|22898|United States|-5|single family| +48287|AAAAAAAAPJMLAAAA|2|2nd |Parkway|Suite 310|Sunnyside|Benton County|WA|91952|United States|-8|condo| +48288|AAAAAAAAAKMLAAAA|274|Pine |RD|Suite 240|Fairview|Cherokee County|TX|75709|United States|-6|single family| +48289|AAAAAAAABKMLAAAA|521|Fifth |Pkwy|Suite D|Woodlawn|Clark County|WI|54098|United States|-6|single family| +48290|AAAAAAAACKMLAAAA|932|Walnut |Boulevard|Suite 380|Flatwoods|Morrill County|NE|64212|United States|-7|condo| +48291|AAAAAAAADKMLAAAA|838|Dogwood Adams|RD|Suite 390|Mechanicsburg|Decatur County|KS|62219|United States|-6|single family| +48292|AAAAAAAAEKMLAAAA|||||||AL||United States||apartment| +48293|AAAAAAAAFKMLAAAA|676|Oak |Cir.|Suite 460|Mount Olive|Madison County|OH|48059|United States|-5|apartment| +48294|AAAAAAAAGKMLAAAA|988|Woodland |Wy|Suite A|Summerville|Carroll County|IL|62033|United States|-6|single family| +48295|AAAAAAAAHKMLAAAA|482|South 2nd|Parkway|Suite 320|Union|Angelina County|TX|78721|United States|-6|single family| +48296|AAAAAAAAIKMLAAAA|513|Wilson |Lane|Suite K|Lakeside|Grand Traverse County|MI|49532|United States|-5|single family| +48297|AAAAAAAAJKMLAAAA|481|Fifteenth Ash|Cir.|Suite G|Sulphur Springs|Cass County|MO|68354|United States|-6|apartment| +48298|AAAAAAAAKKMLAAAA|593|7th Highland|Court|Suite B|Lenox|Florence County|WI|51143|United States|-6|single family| +48299|AAAAAAAALKMLAAAA|574|Elm Jefferson|RD|Suite 370|Greenwood|Los Alamos County|NM|88828|United States|-7|single family| +48300|AAAAAAAAMKMLAAAA|905|3rd |Parkway|Suite Q|Unionville|Howard County|NE|61711|United States|-7|apartment| +48301|AAAAAAAANKMLAAAA|772|Oak Laurel|Boulevard|Suite 180|California|Wahkiakum County|WA|90141|United States|-8|condo| +48302|AAAAAAAAOKMLAAAA|935|Church |ST|Suite T|Spring Hill|Grant County|NE|66787|United States|-6|condo| +48303|AAAAAAAAPKMLAAAA|426|Miller |Dr.|Suite C|Cedar Grove|Story County|IA|50411|United States|-6|apartment| +48304|AAAAAAAAALMLAAAA|944|Main Broadway|Cir.|Suite 360|Scottsville|Tompkins County|NY|14190|United States|-5|condo| +48305|AAAAAAAABLMLAAAA|411|West Ridge|Court|Suite O|Oakwood|Yates County|NY|10169|United States|-5|single family| +48306|AAAAAAAACLMLAAAA|216|Second |Way|Suite 40|Green Acres|Menard County|IL|67683|United States|-6|apartment| +48307|AAAAAAAADLMLAAAA|502|Lee |Cir.|Suite O|Harmony|Ingham County|MI|45804|United States|-5|condo| +48308|AAAAAAAAELMLAAAA|387|Lincoln Lake|Ln|Suite 210|Lakeside|Pointe Coupee Parish|LA|79532|United States|-6|apartment| +48309|AAAAAAAAFLMLAAAA|772|Valley ||||Pecos County|||United States||apartment| +48310|AAAAAAAAGLMLAAAA|281|Franklin |Street|Suite 70|Spring Hill|Levy County|FL|36787|United States|-5|apartment| +48311|AAAAAAAAHLMLAAAA|664|Main Lake|||Summit|Lafayette County|FL||United States||single family| +48312|AAAAAAAAILMLAAAA|664|Main Cherry|Circle|Suite E|Valley View|Randolph County|IN|45124|United States|-5|apartment| +48313|AAAAAAAAJLMLAAAA|275|Second |Drive|Suite H|New Hope|Morgan County|IN|49431|United States|-5|apartment| +48314|AAAAAAAAKLMLAAAA|104|5th Poplar|Court|Suite Q|Mount Zion|Franklin County|KY|48054|United States|-6|single family| +48315|AAAAAAAALLMLAAAA|784|12th |Ln|Suite U|Crossroads|Santa Barbara County|CA|90534|United States|-8|condo| +48316|AAAAAAAAMLMLAAAA|815|6th Seventh|RD|Suite S|Clifton|Shasta County|CA|98014|United States|-8|condo| +48317|AAAAAAAANLMLAAAA|553|Woodland 2nd|Lane|Suite 470|Lincoln|Iowa County|WI|51289|United States|-6|single family| +48318|AAAAAAAAOLMLAAAA|52|Oak Central|ST|Suite 490|Providence|Lake County|TN|36614|United States|-6|single family| +48319|AAAAAAAAPLMLAAAA|454|Birch |Street|Suite B|Wilson|Williamson County|IL|66971|United States|-6|single family| +48320|AAAAAAAAAMMLAAAA|4|Mill Jackson|Ct.|Suite 270|Mount Pleasant|Cheyenne County|CO|81933|United States|-7|condo| +48321|AAAAAAAABMMLAAAA|283|Maple |Ave|Suite 190|Walnut Grove|Loving County|TX|77752|United States|-6|single family| +48322|AAAAAAAACMMLAAAA|895|North |Way|Suite K|Pleasant Valley|Mecosta County|MI|42477|United States|-5|apartment| +48323|AAAAAAAADMMLAAAA|305|River |Cir.|Suite 380|Mount Zion|Inyo County|CA|98054|United States|-8|single family| +48324|AAAAAAAAEMMLAAAA|373|Elm ||Suite H||||||-5|| +48325|AAAAAAAAFMMLAAAA|964|Park View|ST|Suite C|Walnut Grove|Perkins County|NE|67752|United States|-7|condo| +48326|AAAAAAAAGMMLAAAA|745|Forest |Avenue|Suite A|New Hope|Livingston Parish|LA|79431|United States|-6|condo| +48327|AAAAAAAAHMMLAAAA|14|Main |Road|Suite Q|Union|Randolph County|AR|78721|United States|-6|condo| +48328|AAAAAAAAIMMLAAAA|960|Cedar Highland|Street|Suite 260|Springdale|Montague County|TX|78883|United States|-6|condo| +48329|AAAAAAAAJMMLAAAA|636|Laurel Dogwood|Blvd|Suite 220|Amity|McDuffie County|GA|30766|United States|-5|single family| +48330|AAAAAAAAKMMLAAAA|903|Laurel |Drive|Suite 120|Oakland|Oxford County|ME|09843|United States|-5|condo| +48331|AAAAAAAALMMLAAAA|430|11th |||||MN|||-6|apartment| +48332|AAAAAAAAMMMLAAAA|470|Johnson |Dr.|Suite K|Greenfield|Waynesboro city|VA|25038|United States|-5|single family| +48333|AAAAAAAANMMLAAAA|217|Center 3rd|Lane|Suite 470|Concord|Graham County|AZ|84107|United States|-7|single family| +48334|AAAAAAAAOMMLAAAA|701|5th |Circle|Suite 470|Hamilton|Bedford County|TN|32808|United States|-5|apartment| +48335|AAAAAAAAPMMLAAAA|698|Pine |Pkwy|Suite 340|Kirkland|Clinton County|IL|67896|United States|-6|condo| +48336|AAAAAAAAANMLAAAA|113|Sunset Broadway|Cir.|Suite 80|Green Acres|Fayette County|OH|47683|United States|-5|single family| +48337|AAAAAAAABNMLAAAA|157|Main |Lane|Suite R|Hillcrest|Cass County|IA|53003|United States|-6|apartment| +48338|AAAAAAAACNMLAAAA|525|8th |Road|Suite Q|Georgetown|Lamoille County|VT|07657|United States|-5|single family| +48339|AAAAAAAADNMLAAAA|340|3rd Cedar|Lane|Suite 240|Arlington|Ferry County|WA|96557|United States|-8|apartment| +48340|AAAAAAAAENMLAAAA|288|3rd |Cir.|Suite 70|Cedar Grove|Clay County|GA|30411|United States|-5|single family| +48341|AAAAAAAAFNMLAAAA|530|15th |Cir.|Suite X|Fairfield|Coffee County|AL|36192|United States|-6|single family| +48342|AAAAAAAAGNMLAAAA|769|Center 7th|Avenue|Suite K|Deerfield|Ottawa County|OK|79840|United States|-6|condo| +48343|AAAAAAAAHNMLAAAA|902|Main View|Parkway|Suite 250|Farmington|Rock County|MN|59145|United States|-6|single family| +48344|AAAAAAAAINMLAAAA|776|Ridge Wilson|Pkwy|Suite O|Woodlawn|Carter County|MO|64098|United States|-6|condo| +48345|AAAAAAAAJNMLAAAA|745|Main |Boulevard|Suite 290|Pleasant Grove|Greer County|OK|74136|United States|-6|apartment| +48346|AAAAAAAAKNMLAAAA|89|Second Lincoln|Blvd|Suite A|Shiloh|Snyder County|PA|19275|United States|-5|condo| +48347|AAAAAAAALNMLAAAA|977|4th |RD|Suite W|Springfield|Union County|OR|99303|United States|-8|apartment| +48348|AAAAAAAAMNMLAAAA|815|Lincoln Cherry|Road|Suite 200|Stringtown|Goshen County|WY|80162|United States|-7|condo| +48349|AAAAAAAANNMLAAAA|156|Highland 11th|Wy|Suite X|Wilson|Dorchester County|SC|26971|United States|-5|apartment| +48350|AAAAAAAAONMLAAAA|936|1st |Blvd|Suite 20|Oakwood|Columbus County|NC|20169|United States|-5|apartment| +48351|AAAAAAAAPNMLAAAA|783|Meadow |Parkway|Suite 60|Union Hill|Pulaski County|IL|67746|United States|-6|apartment| +48352|AAAAAAAAAOMLAAAA|202|Hickory |Way|Suite P|Edgewood|Riley County|KS|60069|United States|-6|apartment| +48353|AAAAAAAABOMLAAAA|974|Jefferson |Road|Suite 200|Red Bank|Chelan County|WA|94975|United States|-8|condo| +48354|AAAAAAAACOMLAAAA|171|East North|Cir.|Suite W|Arlington|Rooks County|KS|66557|United States|-6|condo| +48355|AAAAAAAADOMLAAAA|446|Chestnut |Wy|Suite 220|Enterprise|Jefferson County|IL|61757|United States|-6|apartment| +48356|AAAAAAAAEOMLAAAA|809|Fifth Elm|Lane|Suite Q|Wildwood|Sitka Borough|AK|96871|United States|-9|apartment| +48357|AAAAAAAAFOMLAAAA|956|11th |Ct.|Suite N|Mount Vernon|Jefferson County|AL|38482|United States|-6|condo| +48358|AAAAAAAAGOMLAAAA|94|Cedar |Ln|Suite Y|Peoria|White Pine County|NV|89818|United States|-8|single family| +48359|AAAAAAAAHOMLAAAA|670|View |Dr.|Suite 250|Clifton|Blaine County|ID|88014|United States|-7|apartment| +48360|AAAAAAAAIOMLAAAA|803|2nd |Lane|Suite 10|Brownsville|Trigg County|KY|49310|United States|-5|apartment| +48361|AAAAAAAAJOMLAAAA|210|Green Washington|Pkwy|Suite 330|San Jose|Loving County|TX|78003|United States|-6|condo| +48362|AAAAAAAAKOMLAAAA|86|Jackson Sunset|Boulevard|Suite M|Mount Zion|Belknap County|NH|08654|United States|-5|single family| +48363|AAAAAAAALOMLAAAA|298|Maple |Avenue|Suite 400|Oak Grove|Franklin County|ID|88370|United States|-7|single family| +48364|AAAAAAAAMOMLAAAA|87|Oak Park|Blvd|Suite B|Spring Hill|Bartow County|GA|36787|United States|-5|condo| +48365|AAAAAAAANOMLAAAA|557|Davis 5th|Boulevard|Suite 100|Woodlawn|Montgomery County|MO|64098|United States|-6|condo| +48366|AAAAAAAAOOMLAAAA|83|Spring |Pkwy|Suite N|Marion|Floyd County|IN|40399|United States|-5|apartment| +48367|AAAAAAAAPOMLAAAA|65|11th |Ct.|Suite W|Newport|Van Buren County|AR|71521|United States|-6|condo| +48368|AAAAAAAAAPMLAAAA|617|4th 6th|Boulevard|Suite 460|Roscoe|Monroe County|WI|51854|United States|-6|condo| +48369|AAAAAAAABPMLAAAA|655|Williams |Blvd|Suite 460|Greenfield|Marquette County|WI|55038|United States|-6|apartment| +48370|AAAAAAAACPMLAAAA|85|2nd 8th|Avenue|Suite 470|Edgewater|Suffolk city|VA|20635|United States|-5|single family| +48371|AAAAAAAADPMLAAAA|217|Oak 4th|Avenue|Suite 400|Midway|Mono County|CA|91904|United States|-8|condo| +48372|AAAAAAAAEPMLAAAA|429|Elm |Avenue|Suite D|Royal|Upshur County|WV|25819|United States|-5|single family| +48373|AAAAAAAAFPMLAAAA|||ST||Oakdale||IA||United States||apartment| +48374|AAAAAAAAGPMLAAAA|10|Railroad |Boulevard|Suite 380|Superior|Madison County|NE|62562|United States|-7|apartment| +48375|AAAAAAAAHPMLAAAA|645|Highland West|Avenue|Suite X|Union|Franklin County|NY|18721|United States|-5|apartment| +48376|AAAAAAAAIPMLAAAA|843|10th |Avenue|Suite N|Oakwood|Osceola County|IA|50169|United States|-6|single family| +48377|AAAAAAAAJPMLAAAA|591|Cedar Valley|Cir.|Suite U|White Oak|Yuba County|CA|96668|United States|-8|single family| +48378|AAAAAAAAKPMLAAAA|396|3rd |ST|Suite C|Bunker Hill|Lee County|NC|20150|United States|-5|apartment| +48379|AAAAAAAALPMLAAAA|817|Maple Highland|Dr.|Suite 290|Union Hill|Kearny County|KS|67746|United States|-6|single family| +48380|AAAAAAAAMPMLAAAA|941|8th Sunset|Circle|Suite 170|Lakeview|McNairy County|TN|38579|United States|-6|single family| +48381|AAAAAAAANPMLAAAA|243|Spruce |Pkwy|Suite D|Fairfield|Dare County|NC|26192|United States|-5|single family| +48382|AAAAAAAAOPMLAAAA|289|Sycamore 13th|Wy|Suite 120|Florence|Lee County|AL|33394|United States|-6|condo| +48383|AAAAAAAAPPMLAAAA|860|South |Ct.|Suite K|Hillcrest|Paulding County|OH|43003|United States|-5|apartment| +48384|AAAAAAAAAANLAAAA|961|Third 6th|Road|Suite 300|Clifton|Jersey County|IL|68014|United States|-6|single family| +48385|AAAAAAAABANLAAAA|284|View |Ave|Suite A|Sunnyside|McDonough County|IL|61952|United States|-6|condo| +48386|AAAAAAAACANLAAAA|807|2nd |Ln|Suite 310|Valley View|Franklin County|AR|75124|United States|-6|condo| +48387|AAAAAAAADANLAAAA|496|Main 6th|Dr.|Suite 300|Spring Hill|Kemper County|MS|56787|United States|-6|apartment| +48388|AAAAAAAAEANLAAAA|854|View 7th|Cir.|Suite M|Forest|Waukesha County|WI|57537|United States|-6|condo| +48389|AAAAAAAAFANLAAAA|693|7th |Dr.||||||||| +48390|AAAAAAAAGANLAAAA|590|Spring Cherry|Dr.|Suite S|Springfield|Turner County|GA|39303|United States|-5|condo| +48391|AAAAAAAAHANLAAAA|912|Lake |Wy|Suite T|Union Hill|Sumner County|TN|37746|United States|-6|single family| +48392|AAAAAAAAIANLAAAA|138|Second |Parkway|Suite 50|Greenville|Washoe County|NV|81387|United States|-8|single family| +48393|AAAAAAAAJANLAAAA|579|Elm |Parkway|Suite 250|Leesburg|Ozark County|MO|65605|United States|-6|condo| +48394|AAAAAAAAKANLAAAA|428|Dogwood |Wy|Suite 400|Bridgeport|Greenwood County|SC|25817|United States|-5|single family| +48395|AAAAAAAALANLAAAA|776|3rd Miller|ST|Suite H|Ludlow|White Pine County|NV|85566|United States|-8|apartment| +48396|AAAAAAAAMANLAAAA|96|Park |Boulevard|Suite 90|Greenwood|Vermillion County|IN|48828|United States|-5|single family| +48397|AAAAAAAANANLAAAA|831|Lincoln |Cir.|Suite W|Fairfield|Erie County|NY|16192|United States|-5|single family| +48398|AAAAAAAAOANLAAAA|534|Madison Woodland|Ave|Suite 320|Oak Grove|Winston County|MS|58370|United States|-6|single family| +48399|AAAAAAAAPANLAAAA|288|Willow Dogwood|Pkwy|Suite 440|Glenwood|Sabine County|TX|73511|United States|-6|apartment| +48400|AAAAAAAAABNLAAAA|570|3rd |Ct.|Suite 490|Belmont|Union County|AR|70191|United States|-6|condo| +48401|AAAAAAAABBNLAAAA|185|Williams ||Suite 0||White County|IN|44136|||apartment| +48402|AAAAAAAACBNLAAAA|259|Washington |Avenue|Suite 320|Oakland|Grafton County|NH|09843|United States|-5|single family| +48403|AAAAAAAADBNLAAAA|192|Spring 9th|Circle|Suite W|Sulphur Springs|Washington County|PA|18354|United States|-5|apartment| +48404|AAAAAAAAEBNLAAAA|35|River 10th|Ct.|Suite 270|Jones|Cumberland County|ME|03286|United States|-5|single family| +48405|AAAAAAAAFBNLAAAA|323|Johnson Maple|Blvd|Suite 110|Frenchtown|Monroe County|TN|32629|United States|-6|single family| +48406|AAAAAAAAGBNLAAAA|69|Spring |Blvd|Suite 430|Maple Grove|Washington County|OK|78252|United States|-6|condo| +48407|AAAAAAAAHBNLAAAA|877|Ash Forest|Boulevard|Suite 420|Oakdale|Salt Lake County|UT|89584|United States|-7|condo| +48408|AAAAAAAAIBNLAAAA|679|East |Parkway|Suite 180|Gilmore|Starr County|TX|75464|United States|-6|condo| +48409|AAAAAAAAJBNLAAAA|95|Third |Avenue|Suite U|Snug Harbor|Norton County|KS|67936|United States|-6|single family| +48410|AAAAAAAAKBNLAAAA|667|Sycamore Sixth|Cir.|Suite 40|Riverside|Montgomery County|NC|29231|United States|-5|condo| +48411|AAAAAAAALBNLAAAA|247|Woodland 9th|Dr.|Suite 460|Fairfield|Adams County|IN|46192|United States|-5|single family| +48412|AAAAAAAAMBNLAAAA|731|Washington Adams|Lane|Suite E|Cordova|Baca County|CO|86938|United States|-7|single family| +48413|AAAAAAAANBNLAAAA|383|Broadway |Lane|Suite 450|Spring Hill|Santa Rosa County|FL|36787|United States|-5|condo| +48414|AAAAAAAAOBNLAAAA|510|Spruce Park|Ave|Suite 60|Lebanon|Okaloosa County|FL|32898|United States|-5|apartment| +48415|AAAAAAAAPBNLAAAA|746|Franklin Johnson|Ln|Suite B|Fairfield|Pershing County|NV|86192|United States|-8|condo| +48416|AAAAAAAAACNLAAAA|177|West Jackson|Ct.|Suite 60|Highland Park|Meagher County|MT|66534|United States|-7|condo| +48417|AAAAAAAABCNLAAAA|211|Johnson Walnut|Dr.|Suite U|Fairfield|Hutchinson County|SD|56192|United States|-7|apartment| +48418|AAAAAAAACCNLAAAA|284|Pine |Blvd|Suite V|Pleasant Hill|Crockett County|TN|33604|United States|-5|single family| +48419|AAAAAAAADCNLAAAA|941|Tenth Spring|Dr.|Suite 120|Greenville|Winchester city|VA|21387|United States|-5|single family| +48420|AAAAAAAAECNLAAAA|655|Park Laurel|Blvd|Suite 400|Woodland|Windham County|CT|05454|United States|-5|condo| +48421|AAAAAAAAFCNLAAAA|833|Green |Street|Suite 340|Concord|Marion County|WV|24107|United States|-5|single family| +48422|AAAAAAAAGCNLAAAA|280|3rd Ninth|RD|Suite F|Thompsonville|Albemarle County|VA|29651|United States|-5|apartment| +48423|AAAAAAAAHCNLAAAA|701|Green |Boulevard|Suite G|Oakdale|Washington County|OR|99584|United States|-8|condo| +48424|AAAAAAAAICNLAAAA|613|8th |Ln|Suite 170|Valley View|Wise County|VA|25124|United States|-5|apartment| +48425|AAAAAAAAJCNLAAAA|438|14th |Boulevard|Suite T|Midway|Cheshire County|NH|02504|United States|-5|single family| +48426|AAAAAAAAKCNLAAAA|681|Valley |Lane|Suite 70|Stringtown|Victoria County|TX|70162|United States|-6|condo| +48427|AAAAAAAALCNLAAAA|761|River 5th|Pkwy||Shiloh||KY||United States|-6|single family| +48428|AAAAAAAAMCNLAAAA|839|Park |ST|Suite K|Clifton|Boone County|WV|28014|United States|-5|single family| +48429|AAAAAAAANCNLAAAA|199|Poplar Mill|Parkway|Suite 20|Summit|Crisp County|GA|30499|United States|-5|condo| +48430|AAAAAAAAOCNLAAAA|386|Spruce |Court|Suite P|Woodville|Pipestone County|MN|54289|United States|-6|apartment| +48431|AAAAAAAAPCNLAAAA|521|Ridge |Ct.|Suite 470|Waterloo|Nantucket County|MA|02275|United States|-5|condo| +48432|AAAAAAAAADNLAAAA|747|Walnut |Parkway|Suite Y|Clinton|Sullivan County|NH|08822|United States|-5|condo| +48433|AAAAAAAABDNLAAAA|76|Lincoln |Blvd|Suite 200|Glendale|Madison County|IA|53951|United States|-6|apartment| +48434|AAAAAAAACDNLAAAA|133|Railroad |Drive|Suite 370|Concord|Yavapai County|AZ|84107|United States|-7|apartment| +48435|AAAAAAAADDNLAAAA|643|14th |Cir.|Suite I|Deerfield|Johnson County|IA|59840|United States|-6|apartment| +48436|AAAAAAAAEDNLAAAA|253|Park 2nd||Suite 360||Bradford County|PA|17057|United States|-5|| +48437|AAAAAAAAFDNLAAAA|282|Poplar 4th|Ct.|Suite 180|Pleasant Valley|Clinton County|IL|62477|United States|-6|apartment| +48438|AAAAAAAAGDNLAAAA|88|8th |Court|Suite R|Woodlawn|Dunn County|WI|54098|United States|-6|condo| +48439|AAAAAAAAHDNLAAAA|538|North 10th|Street|Suite 380|Woodland|Bingham County|ID|84854|United States|-7|apartment| +48440|AAAAAAAAIDNLAAAA|443|Forest Church|Parkway|Suite 480|Greenwood|Brunswick County|NC|28828|United States|-5|single family| +48441|AAAAAAAAJDNLAAAA|12|Sycamore |Way|Suite 440|Oakwood|Wheeler County|GA|30169|United States|-5|condo| +48442|AAAAAAAAKDNLAAAA|470|Cherry Sunset|Drive|Suite 280|Edgewood|Maverick County|TX|70069|United States|-6|condo| +48443|AAAAAAAALDNLAAAA|721|Elm |Blvd|Suite 220|Greenwood|Hennepin County|MN|58828|United States|-6|apartment| +48444|AAAAAAAAMDNLAAAA|80|East 7th|Avenue|Suite N|Pinecrest|Garden County|NE|69981|United States|-6|single family| +48445|AAAAAAAANDNLAAAA|449|12th Park|ST|Suite V|Gravel Hill|James City County|VA|21944|United States|-5|single family| +48446|AAAAAAAAODNLAAAA|381|Johnson Jefferson|Cir.|Suite G|Spring Hill|Fremont County|ID|86787|United States|-7|condo| +48447|AAAAAAAAPDNLAAAA|178|Mill |Way|Suite H|Farmington|Kimball County|NE|69145|United States|-7|apartment| +48448|AAAAAAAAAENLAAAA|423|Cedar Pine|Road|Suite 260|White Oak|Kleberg County|TX|76668|United States|-6|condo| +48449|AAAAAAAABENLAAAA|146|Oak North|Avenue|Suite 40|Wilson|Wright County|IA|56971|United States|-6|single family| +48450|AAAAAAAACENLAAAA|190|15th 4th|Lane|Suite 260|Gilmore|Alcona County|MI|45464|United States|-5|single family| +48451|AAAAAAAADENLAAAA|275|Pine |Way|Suite 250|Cedar Grove|Appling County|GA|30411|United States|-5|apartment| +48452|AAAAAAAAEENLAAAA|708|Jackson 6th|Street|Suite 340|Brownsville|Meade County|KY|49310|United States|-5|single family| +48453|AAAAAAAAFENLAAAA|966|5th 13th|Circle|Suite 330|Greenwood|Harding County|SD|58828|United States|-7|single family| +48454|AAAAAAAAGENLAAAA|754|4th Pine|Way|Suite 180|Bridgeport|Antelope County|NE|65817|United States|-6|condo| +48455|AAAAAAAAHENLAAAA|206|South |Drive|Suite 460|Mountain View|Pend Oreille County|WA|94466|United States|-8|single family| +48456|AAAAAAAAIENLAAAA|329|Hill |Circle|Suite C|Providence|Lee County|TX|76614|United States|-6|condo| +48457|AAAAAAAAJENLAAAA|902|6th |RD|Suite 360|Lakeview|Saline County|AR|78579|United States|-6|condo| +48458|AAAAAAAAKENLAAAA|196|14th |Ln|Suite 130|Woodland|Haskell County|KS|64854|United States|-6|condo| +48459|AAAAAAAALENLAAAA|216|Davis |Ave|Suite F|Glenwood|Bland County|VA|23511|United States|-5|condo| +48460|AAAAAAAAMENLAAAA|526|Hill 3rd|Parkway|Suite D|White Oak|Throckmorton County|TX|76668|United States|-6|condo| +48461|AAAAAAAANENLAAAA|596|Sunset |Ct.|Suite G|Lincoln|Black Hawk County|IA|51289|United States|-6|condo| +48462|AAAAAAAAOENLAAAA|740|North |Drive|Suite 20|Wilson|Livingston County|KY|46971|United States|-5|apartment| +48463|AAAAAAAAPENLAAAA|361|Lincoln Thirteenth|Lane|Suite E|Highland Park|Roanoke County|VA|26534|United States|-5|condo| +48464|AAAAAAAAAFNLAAAA|1|Meadow |Ave|Suite G|Greenwood|Richland Parish|LA|78828|United States|-6|single family| +48465|AAAAAAAABFNLAAAA|672|First Mill|Cir.|Suite G|New Hope|Berrien County|MI|49431|United States|-5|single family| +48466|AAAAAAAACFNLAAAA|879|Lincoln Spruce|Cir.|Suite 360|Farmington|Washington County|MN|59145|United States|-6|single family| +48467|AAAAAAAADFNLAAAA|49|3rd |Road|Suite 180|Midway|Juneau County|WI|51904|United States|-6|condo| +48468|AAAAAAAAEFNLAAAA|730|College |ST|Suite 90|Woodland|Pontotoc County|OK|74854|United States|-6|apartment| +48469|AAAAAAAAFFNLAAAA|441|2nd Church|RD|Suite 440|Watkins|Kingman County|KS|61732|United States|-6|apartment| +48470|AAAAAAAAGFNLAAAA|265|Meadow Ridge|Drive|Suite I|Crossroads|Middlesex County|NJ|01134|United States|-5|apartment| +48471|AAAAAAAAHFNLAAAA|455|Park 5th|Ln|Suite I|Newtown|Shenandoah County|VA|21749|United States|-5|condo| +48472|AAAAAAAAIFNLAAAA|203|Highland Meadow|Cir.|Suite 120|Welcome|Calumet County|WI|56386|United States|-6|single family| +48473|AAAAAAAAJFNLAAAA|138||Avenue|Suite O|Ashland||CA|94244|United States|-8|condo| +48474|AAAAAAAAKFNLAAAA|313|4th |Pkwy|Suite C|Greenfield|Clinton County|MO|65038|United States|-6|single family| +48475|AAAAAAAALFNLAAAA|808|Hill |Pkwy|Suite N|Georgetown|Codington County|SD|57057|United States|-6|single family| +48476|AAAAAAAAMFNLAAAA|630|10th River|Cir.|Suite Q|Five Forks|Haskell County|OK|72293|United States|-6|single family| +48477|AAAAAAAANFNLAAAA|251|Jackson |Blvd|Suite 90|Union Hill|Clinton County|PA|17746|United States|-5|apartment| +48478|AAAAAAAAOFNLAAAA|116|Lincoln |Drive|Suite 30|Union Hill|Adair County|KY|47746|United States|-6|single family| +48479|AAAAAAAAPFNLAAAA|256|Park Lake|Blvd|Suite 470|Riverview|Powhatan County|VA|29003|United States|-5|condo| +48480|AAAAAAAAAGNLAAAA|580|Highland |Wy|Suite X|Lincoln|Todd County|KY|41289|United States|-5|single family| +48481|AAAAAAAABGNLAAAA|388|Ash |Cir.|Suite 40|Enterprise|Williamson County|TN|31757|United States|-5|single family| +48482|AAAAAAAACGNLAAAA|296|Forest |Dr.|Suite N|Greenfield|Armstrong County|PA|15038|United States|-5|apartment| +48483|AAAAAAAADGNLAAAA|315|Jackson |Ave|Suite 390|Riverdale|Roanoke city|VA|29391|United States|-5|single family| +48484|AAAAAAAAEGNLAAAA|88|12th |Road|Suite 90|New Hope|Major County|OK|79431|United States|-6|condo| +48485|AAAAAAAAFGNLAAAA|520|11th Second|Ln|Suite K|Highland|Treasure County|MT|69454|United States|-7|single family| +48486|AAAAAAAAGGNLAAAA||||Suite F|Buena Vista||GA|35752|United States||condo| +48487|AAAAAAAAHGNLAAAA|19|Main Poplar|Court|Suite 100|Buena Vista|Lipscomb County|TX|75752|United States|-6|single family| +48488|AAAAAAAAIGNLAAAA|415|Johnson Hill|Blvd|Suite 310|Midway|Riley County|KS|61904|United States|-6|apartment| +48489|AAAAAAAAJGNLAAAA|96|Willow |RD|Suite D|Bridgeport|Victoria County|TX|75817|United States|-6|single family| +48490|AAAAAAAAKGNLAAAA|330|Hickory |Dr.|Suite 100|Patterson|Carbon County|UT|83175|United States|-7|condo| +48491|AAAAAAAALGNLAAAA|221|7th Franklin|Drive|Suite S|Deerfield|Lemhi County|ID|89840|United States|-7|apartment| +48492|AAAAAAAAMGNLAAAA|432|2nd |Pkwy|Suite N|Oak Ridge|Clay County|KS|68371|United States|-6|condo| +48493|AAAAAAAANGNLAAAA|799|West 15th|Boulevard|Suite F|Belmont|Wilson County|TX|70191|United States|-6|condo| +48494|AAAAAAAAOGNLAAAA|656|4th |Pkwy|Suite Y|Centerville|Coal County|OK|70059|United States|-6|condo| +48495|AAAAAAAAPGNLAAAA|844|Sycamore |Drive|Suite Y|Ludlow|Upshur County|TX|75566|United States|-6|single family| +48496|AAAAAAAAAHNLAAAA|496|Poplar |ST|Suite I|Red Hill|Boundary County|ID|84338|United States|-7|apartment| +48497|AAAAAAAABHNLAAAA|60|4th |ST|Suite 120|Pleasant Hill|Carroll County|MS|53604|United States|-6|condo| +48498|AAAAAAAACHNLAAAA|41|Pine |Ln|Suite 460|Crossroads|Gibson County|IN|40534|United States|-5|condo| +48499|AAAAAAAADHNLAAAA|44|1st |Road|Suite M|Mountain View|Cumberland County|NJ|05066|United States|-5|condo| +48500|AAAAAAAAEHNLAAAA|87|Dogwood Ridge|Ct.|Suite 130|Greenville|York County|SC|21387|United States|-5|single family| +48501|AAAAAAAAFHNLAAAA|293|Davis Second|Ct.|Suite M|Brownsville|Mendocino County|CA|99310|United States|-8|apartment| +48502|AAAAAAAAGHNLAAAA|936|Pine Central|Parkway|Suite 130|Yorktown|Union County|OR|90732|United States|-8|single family| +48503|AAAAAAAAHHNLAAAA|976|Locust Maple|Wy|Suite 400|Franklin|Columbia County|FL|39101|United States|-5|single family| +48504|AAAAAAAAIHNLAAAA|373|Miller |Drive|Suite E|Springfield|Carlton County|MN|59303|United States|-6|apartment| +48505|AAAAAAAAJHNLAAAA|238|10th |Dr.|Suite Y|Glendale|Chautauqua County|KS|63951|United States|-6|apartment| +48506|AAAAAAAAKHNLAAAA|224|Smith Williams|Way|Suite S|Oak Hill|Kent County|RI|08438|United States|-5|apartment| +48507|AAAAAAAALHNLAAAA|621|Chestnut |Lane|Suite 360|Woodland|Walworth County|SD|54854|United States|-7|single family| +48508|AAAAAAAAMHNLAAAA|949|Hill Central|RD|Suite I|Deerfield|Beadle County|SD|59840|United States|-6|condo| +48509|AAAAAAAANHNLAAAA|711|Park |Ln|Suite N|Smith|Gwinnett County|GA|37317|United States|-5|single family| +48510|AAAAAAAAOHNLAAAA|59|College 14th|Road|Suite 20|Woodbine|Grant County|NE|64253|United States|-6|apartment| +48511|AAAAAAAAPHNLAAAA|407|River Dogwood|RD|Suite 180|Plainview|Cameron County|PA|13683|United States|-5|apartment| +48512|AAAAAAAAAINLAAAA|53|Elm |RD|Suite 470|Five Forks|Carroll County|MS|52293|United States|-6|condo| +48513|AAAAAAAABINLAAAA|492|3rd |Avenue|Suite 70|Shady Grove|Taylor County|FL|32812|United States|-5|condo| +48514|AAAAAAAACINLAAAA|774|Lake |Drive|Suite 450|Glendale|Murray County|MN|53951|United States|-6|apartment| +48515|AAAAAAAADINLAAAA|390|Poplar |Drive|Suite D|Bethel|De Soto Parish|LA|75281|United States|-6|single family| +48516|AAAAAAAAEINLAAAA|642|Main Hill|Circle|Suite S|Oak Grove|Clinton County|KY|48370|United States|-6|apartment| +48517|AAAAAAAAFINLAAAA|757|Elm Sixth|RD|Suite 290|Franklin|Mifflin County|PA|19101|United States|-5|apartment| +48518|AAAAAAAAGINLAAAA|861|Sunset |Road|Suite 370|Lone Pine|Cook County|IL|67441|United States|-6|single family| +48519|AAAAAAAAHINLAAAA|40|Sycamore |Ave|Suite 100|Buena Vista|Coosa County|AL|35752|United States|-6|apartment| +48520|AAAAAAAAIINLAAAA|494|Main 10th|Court|||||||-5|condo| +48521|AAAAAAAAJINLAAAA|535|Broadway Ash|Blvd|Suite 340|Wilson|Benewah County|ID|86971|United States|-7|single family| +48522|AAAAAAAAKINLAAAA|544|Third |Street|Suite K|Fairview|Union County|OR|95709|United States|-8|condo| +48523|AAAAAAAALINLAAAA|797|Park |Drive|Suite N|Woodland|Casey County|KY|44854|United States|-6|apartment| +48524|AAAAAAAAMINLAAAA|100|7th 4th|Pkwy|Suite 390|Saint George|Allen County|OH|45281|United States|-5|single family| +48525|AAAAAAAANINLAAAA|851|North |Lane|Suite 470|Greenville|Lauderdale County|AL|31387|United States|-6|apartment| +48526|AAAAAAAAOINLAAAA|729|View Lake|Court|Suite G|Union Hill|Solano County|CA|97746|United States|-8|apartment| +48527|AAAAAAAAPINLAAAA|3|West Hill|Parkway|Suite 290|White Oak|Woodson County|KS|66668|United States|-6|condo| +48528|AAAAAAAAAJNLAAAA|334|4th |ST|Suite N|Mount Olive|Bennett County|SD|58059|United States|-6|condo| +48529|AAAAAAAABJNLAAAA|338|Fourth Walnut|RD|Suite 230|Plainview|Kent County|RI|04283|United States|-5|condo| +48530|AAAAAAAACJNLAAAA|375|Seventh |ST|Suite 490|Rock Springs|Tyler County|TX|77018|United States|-6|apartment| +48531|AAAAAAAADJNLAAAA|325|14th Maple|Street|Suite 380|Glenwood|Franklin County|GA|33511|United States|-5|single family| +48532|AAAAAAAAEJNLAAAA|902|8th |Wy|Suite S|Riverview|Pottawatomie County|KS|69003|United States|-6|single family| +48533|AAAAAAAAFJNLAAAA|938|15th |RD|Suite S|Friendship|Hancock County|OH|44536|United States|-5|single family| +48534|AAAAAAAAGJNLAAAA|413|3rd Second|Drive|Suite G|Walnut Grove|Lee County|IL|67752|United States|-6|condo| +48535|AAAAAAAAHJNLAAAA|77|Jackson |Boulevard|Suite S|Ashton|Matagorda County|TX|79981|United States|-6|condo| +48536|AAAAAAAAIJNLAAAA|779|Sycamore |Circle|Suite V|Sunnyside|Mercer County|ND|51952|United States|-6|apartment| +48537|AAAAAAAAJJNLAAAA|579|South Johnson|Ct.|Suite 260|Ellsworth|Douglas County|WI|55079|United States|-6|single family| +48538|AAAAAAAAKJNLAAAA|129|Spring |Street|Suite 460|Springdale|Itasca County|MN|58883|United States|-6|condo| +48539|AAAAAAAALJNLAAAA|902|Jefferson Lincoln|Ct.|Suite 230|Jackson|Muhlenberg County|KY|49583|United States|-5|apartment| +48540|AAAAAAAAMJNLAAAA|336|South Ridge|Drive|Suite 370|Mount Zion|Adams County|WI|58054|United States|-6|apartment| +48541|AAAAAAAANJNLAAAA|429|River 9th|Dr.|Suite 380|Pinhook|Power County|ID|89398|United States|-7|single family| +48542|AAAAAAAAOJNLAAAA|919|Lake |Boulevard|Suite Y|Liberty|Bergen County|NJ|04051|United States|-5|apartment| +48543|AAAAAAAAPJNLAAAA||Sunset ||Suite 300||Lincoln County|||United States|-7|| +48544|AAAAAAAAAKNLAAAA|959|Birch Park|Ave|Suite O|Montpelier|Becker County|MN|58930|United States|-6|single family| +48545|AAAAAAAABKNLAAAA|797|View Main|Street|Suite 390|Farmington|Adams County|PA|19145|United States|-5|single family| +48546|AAAAAAAACKNLAAAA|869|Green |Boulevard|Suite 70|Belmont|Rock County|WI|50191|United States|-6|apartment| +48547|AAAAAAAADKNLAAAA|124|Johnson |Lane|Suite K|Unionville|Geauga County|OH|41711|United States|-5|apartment| +48548|AAAAAAAAEKNLAAAA|355|Walnut Fifth|Street|Suite 240|Kingston|Barren County|KY|44975|United States|-6|condo| +48549|AAAAAAAAFKNLAAAA|223|College |Ave|Suite 320|Glendale|Towner County|ND|53951|United States|-6|apartment| +48550|AAAAAAAAGKNLAAAA|402|7th Main|Cir.|Suite 450|Shady Grove|Preston County|WV|22812|United States|-5|apartment| +48551|AAAAAAAAHKNLAAAA|76|12th |RD|Suite 90|Jackson|Aleutians East Borough|AK|99583|United States|-9|single family| +48552|AAAAAAAAIKNLAAAA|870|Cedar |Parkway|Suite S|Buena Vista|Gallatin County|MT|65752|United States|-7|apartment| +48553|AAAAAAAAJKNLAAAA|226|First |Blvd|Suite E|Clinton|Reynolds County|MO|68222|United States|-6|single family| +48554|AAAAAAAAKKNLAAAA|863|Park |Cir.|Suite Y|Antioch|Tioga County|NY|18605|United States|-5|apartment| +48555|AAAAAAAALKNLAAAA|683|Sunset |Blvd|Suite F|Glenwood|McCook County|SD|53511|United States|-7|single family| +48556|AAAAAAAAMKNLAAAA|663|River Lake|Dr.|Suite 80|Summerfield|Natchitoches Parish|LA|70634|United States|-6|apartment| +48557|AAAAAAAANKNLAAAA|876|Green |Blvd|Suite 310|Springfield|Dare County|NC|29303|United States|-5|apartment| +48558|AAAAAAAAOKNLAAAA|247|West Second|Dr.|Suite 440|Waterloo|Gray County|KS|61675|United States|-6|apartment| +48559|AAAAAAAAPKNLAAAA|736|Meadow Locust|ST|Suite 380|Lakeside|Red River Parish|LA|79532|United States|-6|condo| +48560|AAAAAAAAALNLAAAA|216|Washington Park|Cir.|Suite 70|Cedar Grove|Noble County|IN|40411|United States|-5|single family| +48561|AAAAAAAABLNLAAAA|703|Tenth 3rd|Dr.|Suite O|Green Acres|Osborne County|KS|67683|United States|-6|single family| +48562|AAAAAAAACLNLAAAA|22|Meadow |Lane|Suite P|Spring Hill|Vilas County|WI|56787|United States|-6|single family| +48563|AAAAAAAADLNLAAAA|84|Davis |Way|Suite 440|Jackson|Harris County|TX|79583|United States|-6|apartment| +48564|AAAAAAAAELNLAAAA|78|2nd |ST|Suite Y|Berea|Fayette County|IA|53804|United States|-6|condo| +48565|AAAAAAAAFLNLAAAA||||Suite R||Putnam County||||-5|| +48566|AAAAAAAAGLNLAAAA|419|Maple |Ln|Suite 270|Georgetown|Putnam County|GA|37057|United States|-5|condo| +48567|AAAAAAAAHLNLAAAA|558|Valley |Ct.|Suite K|Harmony|Rock County|MN|55804|United States|-6|single family| +48568|AAAAAAAAILNLAAAA|409|Broadway |Wy|Suite A|Marion|Madison County|IA|50399|United States|-6|condo| +48569|AAAAAAAAJLNLAAAA|151|Forest Green|Pkwy|Suite 190|Mount Pleasant|Lowndes County|GA|31933|United States|-5|single family| +48570|AAAAAAAAKLNLAAAA|337|Spruce |Cir.|Suite W|Glenwood|Greene County|PA|13511|United States|-5|single family| +48571|AAAAAAAALLNLAAAA|86|Lake |Boulevard|Suite C|Crossroads|Jay County|IN|40534|United States|-5|single family| +48572|AAAAAAAAMLNLAAAA|649|Park Jefferson|Ave|Suite 300|Cedar Grove|Cass County|MO|60411|United States|-6|single family| +48573|AAAAAAAANLNLAAAA|606|West 13th|Dr.|Suite 160|Oakland|Jasper County|TX|79843|United States|-6|apartment| +48574|AAAAAAAAOLNLAAAA|1000|1st West|Road|Suite R|Ellsworth|Ramsey County|ND|55079|United States|-6|condo| +48575|AAAAAAAAPLNLAAAA|336|Park |Circle|Suite K|Valley View|Crockett County|TX|75124|United States|-6|single family| +48576|AAAAAAAAAMNLAAAA|291|Ridge |Circle|Suite 50|Lakeview|DeSoto County|MS|58579|United States|-6|single family| +48577|AAAAAAAABMNLAAAA|235|Willow |Drive|Suite 20|Marion|Dolores County|CO|80399|United States|-7|condo| +48578|AAAAAAAACMNLAAAA|751|Jackson |Dr.|Suite 250|Bennett|Routt County|CO|81715|United States|-7|condo| +48579|AAAAAAAADMNLAAAA|131|5th Green|RD|Suite 90|Pine Grove|Washington County|MD|24593|United States|-5|condo| +48580|AAAAAAAAEMNLAAAA|804|Elm 2nd|Court|Suite P|White Oak|Chase County|KS|66668|United States|-6|single family| +48581|AAAAAAAAFMNLAAAA|741|3rd |Dr.|Suite S|Shiloh|Randolph County|MO|69275|United States|-6|apartment| +48582|AAAAAAAAGMNLAAAA|183|Madison |Dr.|Suite 20|Providence|Willacy County|TX|76614|United States|-6|condo| +48583|AAAAAAAAHMNLAAAA|616|Elm |ST|Suite 420|Acme|Madison County|NY|10164|United States|-5|apartment| +48584|AAAAAAAAIMNLAAAA|86|Lincoln |Street|Suite A|Shiloh|Montgomery County|VA|29275|United States|-5|single family| +48585|AAAAAAAAJMNLAAAA|337|Green |Ln|Suite G|Five Forks|Hidalgo County|TX|72293|United States|-6|condo| +48586|AAAAAAAAKMNLAAAA|329|Third Jefferson|RD|Suite Y|Shiloh|Dorchester County|SC|29275|United States|-5|condo| +48587|AAAAAAAALMNLAAAA|641|Hickory |Boulevard|Suite X|Brownsville|McHenry County|ND|59310|United States|-6|single family| +48588|AAAAAAAAMMNLAAAA|167|North |Way|Suite G|Highland Park|Worcester County|MA|07134|United States|-5|single family| +48589|AAAAAAAANMNLAAAA|541|Main Elevnth|Lane|Suite 240|Jamestown|Lincoln County|ID|86867|United States|-7|apartment| +48590|AAAAAAAAOMNLAAAA|944|Lake |Ct.|Suite T|Valley View|Hopkins County|KY|45124|United States|-6|single family| +48591|AAAAAAAAPMNLAAAA|777|3rd |Cir.|Suite R|Mount Vernon|Medina County|TX|78482|United States|-6|single family| +48592|AAAAAAAAANNLAAAA|424|1st Chestnut|Avenue|Suite W|Green Acres|Wilkes County|NC|27683|United States|-5|condo| +48593|AAAAAAAABNNLAAAA|788|Fourteenth Hillcrest|Avenue|Suite 80|Jackson|Wayne County|NC|29583|United States|-5|condo| +48594|AAAAAAAACNNLAAAA|510|Wilson Spruce|Avenue|Suite L|Riverside|Lyon County|MN|59231|United States|-6|single family| +48595|AAAAAAAADNNLAAAA|810|6th Park|Boulevard|Suite A|West Liberty|Lamar County|MS|54752|United States|-6|apartment| +48596|AAAAAAAAENNLAAAA|657|11th |Avenue|Suite K|Summit|Mountrail County|ND|50499|United States|-6|apartment| +48597|AAAAAAAAFNNLAAAA|128|Hill 7th|Lane|Suite E|Springdale|Wilcox County|AL|38883|United States|-6|single family| +48598|AAAAAAAAGNNLAAAA|94|Ridge Oak|Pkwy|Suite F|Friendship|Montgomery County|NC|24536|United States|-5|apartment| +48599|AAAAAAAAHNNLAAAA|409|Main |Road|Suite 160|Valley View|Hancock County|MS|55124|United States|-6|apartment| +48600|AAAAAAAAINNLAAAA|192|Central Dogwood|Circle|Suite G|Highland Park|Appanoose County|IA|56534|United States|-6|condo| +48601|AAAAAAAAJNNLAAAA|495|Elevnth West|Ct.|Suite 470|Glenwood|Grimes County|TX|73511|United States|-6|condo| +48602|AAAAAAAAKNNLAAAA|978|Sycamore |Ln|Suite A|Mount Zion|Erie County|NY|18054|United States|-5|condo| +48603|AAAAAAAALNNLAAAA|629|North Franklin|Court|Suite 190|Bunker Hill|Rio Arriba County|NM|80150|United States|-7|condo| +48604|AAAAAAAAMNNLAAAA|190|1st |Pkwy||||IL|60894|United States||| +48605|AAAAAAAANNNLAAAA|36|Main |Road|Suite 190|Red Hill|Lyman County|SD|54338|United States|-7|single family| +48606|AAAAAAAAONNLAAAA|615|West |Dr.|Suite 450|Riceville|Greene County|IL|65867|United States|-6|apartment| +48607|AAAAAAAAPNNLAAAA|863|Central Main|Road|Suite S|Greenwood|Phillips County|KS|68828|United States|-6|single family| +48608|AAAAAAAAAONLAAAA|459|1st 15th|Parkway|Suite 270|Hopewell|Iosco County|MI|40587|United States|-5|single family| +48609|AAAAAAAABONLAAAA|749|13th 1st|Ave|Suite 40|Glendale|Forest County|WI|53951|United States|-6|condo| +48610|AAAAAAAACONLAAAA|17|Cherry |Road|Suite 170|Clifton|Fayette County|TN|38014|United States|-5|single family| +48611|AAAAAAAADONLAAAA|311|1st Third|Ln|Suite 30|Five Points|Cumberland County|VA|26098|United States|-5|apartment| +48612|AAAAAAAAEONLAAAA|932|Second |ST|Suite N|Union|Shannon County|MO|68721|United States|-6|single family| +48613|AAAAAAAAFONLAAAA|732|Elm |Way|Suite 430|Riverview|Nome Census Area|AK|99003|United States|-9|apartment| +48614|AAAAAAAAGONLAAAA|671|Central Johnson|Court|Suite T|Riverside|Gladwin County|MI|49231|United States|-5|condo| +48615|AAAAAAAAHONLAAAA|126|Elm Mill|Pkwy|Suite 120|Summit|Montgomery County|IA|50499|United States|-6|apartment| +48616|AAAAAAAAIONLAAAA|516|Highland 3rd|Drive|Suite 440|Greenville|Otoe County|NE|61387|United States|-7|condo| +48617|AAAAAAAAJONLAAAA|959|Green North|Wy|Suite 130|Clinton|Hudson County|NJ|08822|United States|-5|condo| +48618|AAAAAAAAKONLAAAA|782|Walnut Maple|Road|Suite P|Concord|Wheeler County|GA|34107|United States|-5|apartment| +48619|AAAAAAAALONLAAAA|32|Ash Main|Ave|Suite 330|Harmony|Wexford County|MI|45804|United States|-5|apartment| +48620|AAAAAAAAMONLAAAA||North Center|||Macedonia|||41087|United States||apartment| +48621|AAAAAAAANONLAAAA|||Avenue|Suite 90||||07571||-5|condo| +48622|AAAAAAAAOONLAAAA|937|Davis 8th|ST|Suite 290|Sulphur Springs|El Paso County|TX|78354|United States|-6|single family| +48623|AAAAAAAAPONLAAAA|747|Hillcrest Maple|RD|Suite O|Lakeview|Douglas County|MN|58579|United States|-6|single family| +48624|AAAAAAAAAPNLAAAA|98|View Woodland|Cir.|Suite 110|Newport|Suffolk County|NY|11521|United States|-5|apartment| +48625|AAAAAAAABPNLAAAA|749|Elm 9th|Ave|Suite 390|Midway|San Miguel County|NM|81904|United States|-7|apartment| +48626|AAAAAAAACPNLAAAA|550|Ridge View|Pkwy|Suite M|Wildwood|Lehigh County|PA|16871|United States|-5|condo| +48627|AAAAAAAADPNLAAAA|352|3rd |Wy|Suite 340|Lakeside|Fairfax County|VA|29532|United States|-5|single family| +48628|AAAAAAAAEPNLAAAA|140|Adams |Cir.|Suite 400|Wilson|Polk County|TN|36971|United States|-6|condo| +48629|AAAAAAAAFPNLAAAA|392|4th Spruce|RD|Suite 110|Riley|Winston County|MS|51692|United States|-6|condo| +48630|AAAAAAAAGPNLAAAA|175|Ash |Parkway|Suite D|Lone Pine|Logan County|OK|77441|United States|-6|condo| +48631|AAAAAAAAHPNLAAAA|718|Oak Washington|Pkwy|Suite F|Highland Park|Kenosha County|WI|56534|United States|-6|apartment| +48632|AAAAAAAAIPNLAAAA|535|4th Jefferson|Ct.|Suite A|Georgetown|Columbia County|FL|37057|United States|-5|apartment| +48633|AAAAAAAAJPNLAAAA|998|Sixth Spruce|Ct.|Suite 250|Pomona|Tazewell County|IL|64153|United States|-6|apartment| +48634|AAAAAAAAKPNLAAAA|465|Maple Valley|Lane|Suite Y|Newport|Monroe County|IN|41521|United States|-5|single family| +48635|AAAAAAAALPNLAAAA|634|Park Birch|RD|Suite U|Red Hill|Robeson County|NC|24338|United States|-5|apartment| +48636|AAAAAAAAMPNLAAAA|861|Johnson 5th|Cir.|Suite 60|Providence|Dawson County|NE|66614|United States|-6|condo| +48637|AAAAAAAANPNLAAAA|786|Main Fourth|Ct.|Suite H|Oakland|Greene County|OH|49843|United States|-5|condo| +48638|AAAAAAAAOPNLAAAA|466|Center |Ave|Suite 300|Sulphur Springs|Radford city|VA|28354|United States|-5|apartment| +48639|AAAAAAAAPPNLAAAA|314|3rd |Ln|Suite 390|Arlington|Sunflower County|MS|56557|United States|-6|apartment| +48640|AAAAAAAAAAOLAAAA|795|Lakeview Sunset|Dr.|Suite 200|Pinecrest|Dade County|MO|69981|United States|-6|apartment| +48641|AAAAAAAABAOLAAAA|965|Smith 13th|Dr.|Suite Y|Bethel|Westmoreland County|PA|15281|United States|-5|single family| +48642|AAAAAAAACAOLAAAA|528|Valley |Avenue|Suite V|Riverdale|Lyon County|MN|59391|United States|-6|apartment| +48643|AAAAAAAADAOLAAAA|680|First Railroad|Circle|Suite 460|Five Points|Sumter County|AL|36098|United States|-6|apartment| +48644|AAAAAAAAEAOLAAAA|247|Chestnut Main|Blvd|Suite T|Hillcrest|Walworth County|SD|53003|United States|-7|single family| +48645|AAAAAAAAFAOLAAAA|462|Valley |Parkway|Suite W|Union Hill|Marshall County|SD|57746|United States|-7|condo| +48646|AAAAAAAAGAOLAAAA|444|Fourth |Street|Suite 380|Blair|Jones County|NC|25465|United States|-5|single family| +48647|AAAAAAAAHAOLAAAA|82|Oak Mill|Avenue|Suite C|Unionville|Guadalupe County|NM|81711|United States|-7|single family| +48648|AAAAAAAAIAOLAAAA|928|Main Green|Avenue|Suite 50|Spring Valley|Miner County|SD|56060|United States|-7|single family| +48649|AAAAAAAAJAOLAAAA|89|5th |RD|Suite E|Lakewood|Wapello County|IA|58877|United States|-6|apartment| +48650|AAAAAAAAKAOLAAAA|76|Maple Center|Avenue|Suite A|Jackson|Benton County|AR|79583|United States|-6|condo| +48651|AAAAAAAALAOLAAAA|881|Cedar Park|Street|Suite K|Woodlawn|Ashe County|NC|24098|United States|-5|single family| +48652|AAAAAAAAMAOLAAAA|725|2nd Ridge|Lane|Suite I|Belmont|Jefferson County|FL|30191|United States|-5|apartment| +48653|AAAAAAAANAOLAAAA|519|15th |Ct.|Suite 50|Oak Hill|Comal County|TX|77838|United States|-6|single family| +48654|AAAAAAAAOAOLAAAA|352|1st ||||Henry County|IL|67752|United States||condo| +48655|AAAAAAAAPAOLAAAA|547|Highland 7th|Road|Suite 30|Page|Spink County|SD|50296|United States|-7|apartment| +48656|AAAAAAAAABOLAAAA|427|North Jefferson|Ct.|Suite W|Bunker Hill|Lamar County|AL|30150|United States|-6|single family| +48657|AAAAAAAABBOLAAAA|26|Washington |Circle|Suite 40|Centerville|Washington County|TN|30059|United States|-6|condo| +48658|AAAAAAAACBOLAAAA||||Suite Q|Buena Vista||||United States|-8|| +48659|AAAAAAAADBOLAAAA|249|Lake |Court|Suite R|Springfield|Beaufort County|NC|29303|United States|-5|single family| +48660|AAAAAAAAEBOLAAAA|92|Williams Park|Ln|Suite F|Spring Valley|Ray County|MO|66060|United States|-6|condo| +48661|AAAAAAAAFBOLAAAA|814|Chestnut |ST|Suite O|Five Points|Goshen County|WY|86098|United States|-7|single family| +48662|AAAAAAAAGBOLAAAA|860|Maple |ST|Suite 430|Woodland|Somervell County|TX|74854|United States|-6|apartment| +48663|AAAAAAAAHBOLAAAA|707|Church |ST|Suite 290|Harmony|Hancock County|GA|35804|United States|-5|single family| +48664|AAAAAAAAIBOLAAAA|806|Ridge |Ave|Suite Y|Woodland|Arlington County|VA|24854|United States|-5|apartment| +48665|AAAAAAAAJBOLAAAA|569|4th Cherry|||White Oak|Franklin County|||United States||| +48666|AAAAAAAAKBOLAAAA|156|Ash |Boulevard|Suite T|Stringtown|Anderson County|SC|20162|United States|-5|apartment| +48667|AAAAAAAALBOLAAAA|52|Thirteenth |Circle|Suite B|Enterprise|Carroll County|AR|71757|United States|-6|condo| +48668|AAAAAAAAMBOLAAAA|581|Chestnut |Court|Suite 160|Lakewood|Athens County|OH|48877|United States|-5|condo| +48669|AAAAAAAANBOLAAAA|303|Jefferson |Circle|Suite 180|Maple Grove|Williamsburg County|SC|28252|United States|-5|single family| +48670|AAAAAAAAOBOLAAAA|200|Jefferson 4th|Road|Suite F|Wildwood|Charlton County|GA|36871|United States|-5|single family| +48671|AAAAAAAAPBOLAAAA||Franklin |Parkway||Green Acres||MT||||| +48672|AAAAAAAAACOLAAAA|201|9th 2nd|Drive|Suite C|Ryan|Marlboro County|SC|20525|United States|-5|condo| +48673|AAAAAAAABCOLAAAA|171|Walnut Maple|Way|Suite O|Pleasant Hill|Howard County|AR|73604|United States|-6|condo| +48674|AAAAAAAACCOLAAAA|422|Washington Birch|Ave|Suite 400|Jackson|Duval County|TX|79583|United States|-6|apartment| +48675|AAAAAAAADCOLAAAA|21|Locust Oak|Ct.|Suite 0|Salem|Porter County|IN|48048|United States|-5|apartment| +48676|AAAAAAAAECOLAAAA|820|2nd |ST|Suite 390|Fairfield|Logan County|KS|66192|United States|-6|apartment| +48677|AAAAAAAAFCOLAAAA|194|Center |Road|Suite 320|Oakwood|Benton County|MN|50169|United States|-6|single family| +48678|AAAAAAAAGCOLAAAA|648|Third |Drive|Suite U|Johnsonville|Santa Clara County|CA|97745|United States|-8|single family| +48679|AAAAAAAAHCOLAAAA|156|4th Broadway|Blvd|Suite V|Lenox|Edwards County|IL|61143|United States|-6|apartment| +48680|AAAAAAAAICOLAAAA|515|Tenth |Lane|Suite 210|Clinton|Williams County|ND|58222|United States|-6|apartment| +48681|AAAAAAAAJCOLAAAA|656|Lakeview Cedar|Way|Suite 120|Enterprise|Carroll County|AR|71757|United States|-6|single family| +48682|AAAAAAAAKCOLAAAA|664|Lakeview |ST|Suite 60|Mountain View|Franklin County|GA|34466|United States|-5|apartment| +48683|AAAAAAAALCOLAAAA|51|Laurel |RD|Suite 430|Glenwood|Lawrence County|SD|53511|United States|-7|condo| +48684|AAAAAAAAMCOLAAAA|688|10th |Drive|Suite 340|Mount Vernon|Elmore County|ID|88482|United States|-7|apartment| +48685|AAAAAAAANCOLAAAA|819|View |Court|Suite W|Salem|Somerset County|NJ|08648|United States|-5|condo| +48686|AAAAAAAAOCOLAAAA|384|7th Washington|Ct.|Suite 180|Spring Valley|Columbia County|NY|16060|United States|-5|apartment| +48687|AAAAAAAAPCOLAAAA|947|Locust |Court|Suite Y|Glenwood|Glades County|FL|33511|United States|-5|single family| +48688|AAAAAAAAADOLAAAA|301|Center Washington|Circle|Suite 260|Cedar Grove|Tishomingo County|MS|50411|United States|-6|single family| +48689|AAAAAAAABDOLAAAA|742|6th Third|Street|Suite 220|Stringtown|Osceola County|IA|50162|United States|-6|single family| +48690|AAAAAAAACDOLAAAA|529|Lee 12th|Boulevard|Suite 310|Bethel|Fisher County|TX|75281|United States|-6|single family| +48691|AAAAAAAADDOLAAAA|499|2nd Fifth|Ct.|Suite 280|Waterloo|Bristol County|RI|02275|United States|-5|apartment| +48692|AAAAAAAAEDOLAAAA|279|Lakeview Maple|Way|Suite 230|Oak Ridge|Green Lake County|WI|58371|United States|-6|condo| +48693|AAAAAAAAFDOLAAAA|65|2nd |Parkway|Suite U|Clinton|Faulk County|SD|58222|United States|-6|single family| +48694|AAAAAAAAGDOLAAAA|976|Adams Oak|Circle|Suite 270|Bethel|Taylor County|TX|75281|United States|-6|condo| +48695|AAAAAAAAHDOLAAAA|366|1st Main|Street|Suite 420|Belmont|Bryan County|GA|30191|United States|-5|apartment| +48696|AAAAAAAAIDOLAAAA|473|Lake 4th|Ave|Suite Q|Oakdale|Lea County|NM|89584|United States|-7|single family| +48697|AAAAAAAAJDOLAAAA|939|Highland Ash|Cir.|Suite 350|Lakeview|Rappahannock County|VA|28579|United States|-5|single family| +48698|AAAAAAAAKDOLAAAA|313|Ridge Johnson|Ave|Suite 250|Farmington|Lawrence County|AR|79145|United States|-6|apartment| +48699|AAAAAAAALDOLAAAA|986|5th Hickory|Way|Suite K|Greenfield|Harnett County|NC|25038|United States|-5|single family| +48700|AAAAAAAAMDOLAAAA|23|Spruce |Wy|Suite 380|Mount Zion|Perry County|AR|78054|United States|-6|apartment| +48701|AAAAAAAANDOLAAAA|344|Davis |Pkwy||||PA||United States|-5|| +48702|AAAAAAAAODOLAAAA|937|Pine 9th|Circle|Suite Y|Stringtown|Steele County|ND|50162|United States|-6|single family| +48703|AAAAAAAAPDOLAAAA|282|4th |Pkwy|Suite A|Carthage|Noxubee County|MS|51529|United States|-6|single family| +48704|AAAAAAAAAEOLAAAA|108|Willow |Ct.|Suite A|Forest Hills|Staunton city|VA|29237|United States|-5|single family| +48705|AAAAAAAABEOLAAAA|674|Davis |Cir.|Suite 310|Clinton|Nowata County|OK|78222|United States|-6|apartment| +48706|AAAAAAAACEOLAAAA|410|Lincoln Sunset|Ave|Suite H|Oakwood|Wythe County|VA|20169|United States|-5|apartment| +48707|AAAAAAAADEOLAAAA|919|Fifteenth |Lane|Suite 300|Walnut Grove|Jackson County|OK|77752|United States|-6|single family| +48708|AAAAAAAAEEOLAAAA|259|Second |Drive|Suite 330|Shore Acres|Charles County|MD|22724|United States|-5|condo| +48709|AAAAAAAAFEOLAAAA|147|Pine Davis|Blvd|Suite 150|Stringtown|Jefferson County|AL|30162|United States|-6|single family| +48710|AAAAAAAAGEOLAAAA|43|Oak Elm|Ln|Suite N|Waterloo|Jefferson Davis County|MS|51675|United States|-6|apartment| +48711|AAAAAAAAHEOLAAAA|311|Locust |Lane|Suite H|Stringtown|Martin County|MN|50162|United States|-6|apartment| +48712|AAAAAAAAIEOLAAAA|380|Third |Parkway|Suite S|Marion|Jones County|IA|50399|United States|-6|condo| +48713|AAAAAAAAJEOLAAAA|748|Second Washington|Street|Suite 400|Belmont|Floyd County|TX|70191|United States|-6|single family| +48714|AAAAAAAAKEOLAAAA|275|Davis |Ct.|Suite E|Riverside|Woods County|OK|79231|United States|-6|condo| +48715|AAAAAAAALEOLAAAA|728|Poplar Lake|Blvd|Suite M|Wilson|Albany County|NY|16971|United States|-5|single family| +48716|AAAAAAAAMEOLAAAA|331|Cedar |Pkwy|Suite V|Oakdale|Choctaw County|AL|39584|United States|-6|single family| +48717|AAAAAAAANEOLAAAA|757|North |Dr.|Suite 100|Garrison|Natchitoches Parish|LA|78767|United States|-6|apartment| +48718|AAAAAAAAOEOLAAAA|161|Park |Way|Suite C|Greenville|Adams County|OH|41387|United States|-5|condo| +48719|AAAAAAAAPEOLAAAA|315|Poplar |Wy|Suite H|Glenwood|Hartford County|CT|04111|United States|-5|condo| +48720|AAAAAAAAAFOLAAAA|811|8th South|Circle|Suite 50|Florence|Bergen County|NJ|03994|United States|-5|condo| +48721|AAAAAAAABFOLAAAA|61|Adams Park|Dr.|Suite 150|Sunnyside|Darlington County|SC|21952|United States|-5|condo| +48722|AAAAAAAACFOLAAAA|30|Railroad |Circle|Suite N|Hillcrest|Del Norte County|CA|93003|United States|-8|condo| +48723|AAAAAAAADFOLAAAA|672|Park Eigth|Ave|Suite L|Riverside|Obion County|TN|39231|United States|-6|single family| +48724|AAAAAAAAEFOLAAAA|845|6th 15th|Blvd|Suite 200|Harmony|Pawnee County|OK|75804|United States|-6|condo| +48725|AAAAAAAAFFOLAAAA|671|||Suite 380||Warren County||20499|United States|-5|condo| +48726|AAAAAAAAGFOLAAAA|856|Main |Avenue|Suite J|Valley View|Ritchie County|WV|25124|United States|-5|single family| +48727|AAAAAAAAHFOLAAAA|501|Park |ST|Suite P|Florence|Lee County|FL|33394|United States|-5|condo| +48728|AAAAAAAAIFOLAAAA|719|Meadow |Road|Suite 270|Shady Grove|Payne County|OK|72812|United States|-6|apartment| +48729|AAAAAAAAJFOLAAAA|35|Washington |ST|Suite F|Wolf Creek|Sullivan County|IN|42455|United States|-5|condo| +48730|AAAAAAAAKFOLAAAA|23|South |Drive|Suite 180|Colonial Heights|Kittitas County|WA|93425|United States|-8|apartment| +48731|AAAAAAAALFOLAAAA|209|9th 7th|Street|Suite K|Jamestown|McIntosh County|GA|36867|United States|-5|apartment| +48732|AAAAAAAAMFOLAAAA|85|Chestnut |Way|Suite 150|Enterprise|Kaufman County|TX|71757|United States|-6|condo| +48733|AAAAAAAANFOLAAAA|606|Johnson |Circle|Suite 50|Midway|Saguache County|CO|81904|United States|-7|condo| +48734|AAAAAAAAOFOLAAAA|987|||Suite 340|Pleasant Valley|Cheyenne County|||United States|-6|apartment| +48735|AAAAAAAAPFOLAAAA|553|2nd |Wy|Suite 190|Pleasant Hill|Washington County|TN|33604|United States|-6|apartment| +48736|AAAAAAAAAGOLAAAA|768|Miller |Cir.|Suite 410|Farmington|Madison County|IL|69145|United States|-6|condo| +48737|AAAAAAAABGOLAAAA|970|12th |Ave|Suite N|Summerfield|Hendry County|FL|30634|United States|-5|single family| +48738|AAAAAAAACGOLAAAA|894|3rd |Wy|Suite J|Franklin|Schleicher County|TX|79101|United States|-6|apartment| +48739|AAAAAAAADGOLAAAA|96|Oak Spring|Drive|Suite 100|Nottingham|Chautauqua County|NY|14074|United States|-5|condo| +48740|AAAAAAAAEGOLAAAA|282|Franklin Second|Drive|Suite W|Salem|Harper County|KS|68048|United States|-6|single family| +48741|AAAAAAAAFGOLAAAA|895|12th |Wy|Suite T|Antioch|La Porte County|IN|48605|United States|-5|condo| +48742|AAAAAAAAGGOLAAAA|595|Cedar Lincoln|Dr.|Suite 290|Pleasant Grove|King County|TX|74136|United States|-6|condo| +48743|AAAAAAAAHGOLAAAA|209|View Central|Cir.|Suite 340|New Hope|Catawba County|NC|29431|United States|-5|apartment| +48744|AAAAAAAAIGOLAAAA|941|Second Main|Parkway|Suite F|Spring Valley|Aleutians West Census Area|AK|96060|United States|-9|apartment| +48745|AAAAAAAAJGOLAAAA|90|Pine Third|Dr.|Suite K|Hopewell|Gage County|NE|60587|United States|-6|single family| +48746|AAAAAAAAKGOLAAAA|515|Cherry |Pkwy|Suite 30|Spring Valley|Wibaux County|MT|66060|United States|-7|condo| +48747|AAAAAAAALGOLAAAA|312|Meadow Ridge|Blvd|Suite 470|Oakland|Crawford County|WI|59843|United States|-6|apartment| +48748|AAAAAAAAMGOLAAAA|821|Ridge |Drive|Suite 170|Riverdale|Washington County|MO|69391|United States|-6|single family| +48749|AAAAAAAANGOLAAAA|368|Walnut Sixth|Blvd|Suite D|Oak Ridge|Perry County|TN|38371|United States|-6|condo| +48750|AAAAAAAAOGOLAAAA|48|11th |ST|Suite C|Shiloh|Franklin County|VA|29275|United States|-5|apartment| +48751|AAAAAAAAPGOLAAAA|462|Dogwood Forest|RD|Suite V|Plainview|Lincoln County|NE|63683|United States|-7|condo| +48752|AAAAAAAAAHOLAAAA|36|7th West|Wy|Suite Y|Fairfield|Yates County|NY|16192|United States|-5|apartment| +48753|AAAAAAAABHOLAAAA|150|Twelfth Jackson|Boulevard|Suite F|Maple Grove|Atchison County|KS|68252|United States|-6|condo| +48754|AAAAAAAACHOLAAAA|697|Third |Drive|Suite T|Antioch|Simpson County|KY|48605|United States|-5|single family| +48755|AAAAAAAADHOLAAAA|713|Johnson Park|Wy|Suite A|Oakwood|Richland County|WI|50169|United States|-6|apartment| +48756|AAAAAAAAEHOLAAAA|941|View Central|Avenue|||Coffey County||62819||-6|apartment| +48757|AAAAAAAAFHOLAAAA|352|Laurel |Pkwy|Suite 320|Marion|Hamilton County|TX|70399|United States|-6|condo| +48758|AAAAAAAAGHOLAAAA|730|5th Washington|Way|Suite 70|Shady Grove|Lake County|IN|42812|United States|-5|condo| +48759|AAAAAAAAHHOLAAAA|176|Johnson |ST|Suite 190|Waterloo|Barry County|MI|41675|United States|-5|apartment| +48760|AAAAAAAAIHOLAAAA|567|Maple 2nd|Road|Suite R|Providence|Brown County|IN|46614|United States|-5|single family| +48761|AAAAAAAAJHOLAAAA|690|Miller Lake|Cir.|Suite N|Oakland|Milam County|TX|79843|United States|-6|condo| +48762|AAAAAAAAKHOLAAAA|413|Pine |Lane|Suite P|Farmington|Bullock County|AL|39145|United States|-6|single family| +48763|AAAAAAAALHOLAAAA|111|Wilson Ash|Blvd|Suite V|Kingston|Darlington County|SC|24975|United States|-5|condo| +48764|AAAAAAAAMHOLAAAA|412|View 5th|Circle|Suite O|Lakeview|Tipton County|IN|48579|United States|-5|single family| +48765|AAAAAAAANHOLAAAA|21|Jefferson |Way|Suite M|Unionville|Covington County|MS|51711|United States|-6|apartment| +48766|AAAAAAAAOHOLAAAA|545|6th Franklin|Ct.|Suite V|Midway|Lyon County|KY|41904|United States|-5|single family| +48767|AAAAAAAAPHOLAAAA|110|Elm |Blvd|Suite 290|Riverview|Indiana County|PA|19003|United States|-5|single family| +48768|AAAAAAAAAIOLAAAA|264|Laurel |Pkwy|||Pender County|NC||||| +48769|AAAAAAAABIOLAAAA|923|Third Chestnut|Dr.|Suite 20|Calhoun|Henry County|IA|56909|United States|-6|single family| +48770|AAAAAAAACIOLAAAA|287|Dogwood |Way|Suite 120|Spring Valley|Braxton County|WV|26060|United States|-5|apartment| +48771|AAAAAAAADIOLAAAA|866|North Pine|Ln|Suite Q|Green Acres|Hawkins County|TN|37683|United States|-5|apartment| +48772|AAAAAAAAEIOLAAAA|384|Walnut |Wy|Suite D|Enterprise|Pendleton County|WV|21757|United States|-5|apartment| +48773|AAAAAAAAFIOLAAAA|967|Eigth Walnut|Way|Suite 10|Bridgeport|Washington County|AR|75817|United States|-6|apartment| +48774|AAAAAAAAGIOLAAAA|568|Dogwood Main|Boulevard|Suite 200|Guthrie|Columbia County|AR|71423|United States|-6|apartment| +48775|AAAAAAAAHIOLAAAA|124|5th 4th|Cir.|Suite Y|Stringtown|Cross County|AR|70162|United States|-6|condo| +48776|AAAAAAAAIIOLAAAA|230|1st |ST|Suite P|Lakeside|Pawnee County|NE|69532|United States|-7|single family| +48777|AAAAAAAAJIOLAAAA|696|4th |Court|Suite U|Greenfield|Dallas County|MO|65038|United States|-6|condo| +48778|AAAAAAAAKIOLAAAA|328|Lake Green|Drive|Suite F|White Oak|Hickman County|TN|36668|United States|-5|single family| +48779|AAAAAAAALIOLAAAA|374|2nd Smith|RD|Suite W|Ashland|Hill County|MT|64244|United States|-7|condo| +48780|AAAAAAAAMIOLAAAA|365|Oak View|Ct.|Suite D|Bridgeport|Otoe County|NE|65817|United States|-7|single family| +48781|AAAAAAAANIOLAAAA|324|Third Forest|Wy|Suite W|Oak Grove|Hudson County|NJ|08970|United States|-5|single family| +48782|AAAAAAAAOIOLAAAA|808|Davis Oak|Avenue|Suite Q|White Oak|Calhoun County|||||| +48783|AAAAAAAAPIOLAAAA|827|Jackson Main|Parkway|Suite C|Gardner|Tyrrell County|NC|29352|United States|-5|apartment| +48784|AAAAAAAAAJOLAAAA|22|Broadway |Boulevard|Suite V|Five Points|DeKalb County|TN|36098|United States|-5|single family| +48785|AAAAAAAABJOLAAAA|515|First |Lane|Suite 460|Waterloo|Kingman County|KS|61675|United States|-6|condo| +48786|AAAAAAAACJOLAAAA|498|1st Maple|Pkwy|Suite Y|Union Hill|Talbot County|MD|27746|United States|-5|condo| +48787|AAAAAAAADJOLAAAA|582|View |Ct.|Suite 60|Oak Ridge|Clay County|AR|78371|United States|-6|apartment| +48788|AAAAAAAAEJOLAAAA|531|Ash |Cir.|Suite 80|Salem|Leflore County|MS|58048|United States|-6|apartment| +48789|AAAAAAAAFJOLAAAA|151|8th 7th|Ct.|Suite N|Oakwood|Caldwell County|TX|70169|United States|-6|condo| +48790|AAAAAAAAGJOLAAAA|627|Willow |Wy|Suite 470|Oakdale|Pima County|AZ|89584|United States|-7|condo| +48791|AAAAAAAAHJOLAAAA|473|Center River|Ct.|Suite B|Pleasant Valley|Kings County|NY|12477|United States|-5|condo| +48792|AAAAAAAAIJOLAAAA|242|Locust First|Ln|Suite 350|Hamilton|Logan County|OH|42808|United States|-5|apartment| +48793|AAAAAAAAJJOLAAAA|314|Jefferson |Parkway|Suite 260|Deerfield|James City County|VA|29840|United States|-5|condo| +48794|AAAAAAAAKJOLAAAA|611|Elevnth |Avenue|Suite G|Woodland|Uinta County|WY|84854|United States|-7|apartment| +48795|AAAAAAAALJOLAAAA|302|Sycamore |Ct.|Suite V|Fairview|Fairfax County|VA|25709|United States|-5|condo| +48796|AAAAAAAAMJOLAAAA|233|Main |Cir.|Suite 160|Riverside|Scott County|MN|59231|United States|-6|single family| +48797|AAAAAAAANJOLAAAA|359|Dogwood Johnson|Road|Suite 100|Georgetown|Oconto County|WI|57057|United States|-6|single family| +48798|AAAAAAAAOJOLAAAA|566|Cedar |Wy|Suite G|Riceville|Wheeler County|GA|35867|United States|-5|apartment| +48799|AAAAAAAAPJOLAAAA|4|Washington |Ct.|Suite 460|Murray|Beaver County|PA|12150|United States|-5|condo| +48800|AAAAAAAAAKOLAAAA|710|Ash Maple|Boulevard|Suite 400|Newport|Monroe County|AR|71521|United States|-6|single family| +48801|AAAAAAAABKOLAAAA|902|1st First|Circle|Suite 370|Clifton|Carroll County|IL|68014|United States|-6|condo| +48802|AAAAAAAACKOLAAAA|883|14th 1st|Blvd|Suite 300|Clinton|Potter County|SD|58222|United States|-7|condo| +48803|AAAAAAAADKOLAAAA|893|Birch |Blvd|Suite 480|Bethel|Hartford County|CT|05881|United States|-5|condo| +48804|AAAAAAAAEKOLAAAA|224|4th 5th|ST|Suite G|Spring Hill|Sierra County|CA|96787|United States|-8|single family| +48805|AAAAAAAAFKOLAAAA|510|Madison Sunset|Dr.|Suite E|Valley View|Choctaw County|MS|55124|United States|-6|single family| +48806|AAAAAAAAGKOLAAAA|346|Ridge |Avenue|Suite F|Sugar Hill|Somerset County|ME|05714|United States|-5|apartment| +48807|AAAAAAAAHKOLAAAA|630|North 9th|Parkway|Suite N|Buena Vista|Lee County|SC|25752|United States|-5|apartment| +48808|AAAAAAAAIKOLAAAA|647|Lincoln |Parkway|Suite Q|Mount Pleasant|Washington County|IL|61933|United States|-6|apartment| +48809|AAAAAAAAJKOLAAAA|368|Cherry |RD|Suite P|Mount Zion|Cumberland County|PA|18054|United States|-5|condo| +48810|AAAAAAAAKKOLAAAA|891|Center |Avenue|Suite K|Clifton|Holmes County|MS|58014|United States|-6|condo| +48811|AAAAAAAALKOLAAAA|336|Walnut |Street|Suite F|Clifford|Pickaway County|OH|48164|United States|-5|condo| +48812|AAAAAAAAMKOLAAAA|5|15th Fifteenth|Pkwy|Suite W|Oakland|Howard County|IA|59843|United States|-6|single family| +48813|AAAAAAAANKOLAAAA|618|4th Oak|Lane|Suite W|Flatwoods|Garfield County|NE|64212|United States|-6|condo| +48814|AAAAAAAAOKOLAAAA|927|Mill |Dr.|Suite H|Newtown|Terrell County|TX|71749|United States|-6|apartment| +48815|AAAAAAAAPKOLAAAA|610|Elm Green|Road|Suite N|Brownsville|Morgan County|KY|49310|United States|-5|single family| +48816|AAAAAAAAALOLAAAA|987|8th Meadow|Street|Suite G|New Hope|Harper County|OK|79431|United States|-6|apartment| +48817|AAAAAAAABLOLAAAA|679|Main |Cir.|Suite F|Ashland|Butler County|AL|34244|United States|-6|condo| +48818|AAAAAAAACLOLAAAA|415|Green Church|RD|Suite E|Florence|Isle of Wight County|VA|23394|United States|-5|condo| +48819|AAAAAAAADLOLAAAA|665|5th |RD|Suite 440|Guthrie|Pottawatomie County|KS|61423|United States|-6|condo| +48820|AAAAAAAAELOLAAAA|439|7th |Pkwy|Suite 310|Newport|Westmoreland County|PA|11521|United States|-5|single family| +48821|AAAAAAAAFLOLAAAA|835|Miller 11th|Cir.|Suite 310|Harmony|Edwards County|IL|65804|United States|-6|single family| +48822|AAAAAAAAGLOLAAAA|102|5th |Cir.|Suite E|Five Forks|Jefferson Parish|LA|72293|United States|-6|single family| +48823|AAAAAAAAHLOLAAAA|488|West River|Pkwy|Suite 330|Lincoln|Buffalo County|WI|51289|United States|-6|condo| +48824|AAAAAAAAILOLAAAA|93|Church 7th|RD|Suite T|Oak Hill|Brown County|WI|57838|United States|-6|condo| +48825|AAAAAAAAJLOLAAAA|210|Jefferson |Way|Suite W|Ashland|Columbia County|AR|74244|United States|-6|condo| +48826|AAAAAAAAKLOLAAAA|475|View Williams|Parkway|Suite 10|Salem|Lincoln Parish|LA|78048|United States|-6|single family| +48827|AAAAAAAALLOLAAAA|352|Meadow |Way|Suite 420|Lakeside|Jackson County|SD|59532|United States|-7|apartment| +48828|AAAAAAAAMLOLAAAA|37|Washington Sunset|Ave|Suite 330|Cordova|Merced County|CA|96938|United States|-8|apartment| +48829|AAAAAAAANLOLAAAA|763|6th Hickory|Road|Suite F|Oakwood|Douglas County|OR|90169|United States|-8|single family| +48830|AAAAAAAAOLOLAAAA|94|Fourth |ST|Suite 110|Lakeview|Saline County|AR|78579|United States|-6|single family| +48831|AAAAAAAAPLOLAAAA|561|Davis |Wy|Suite F|Providence|Cedar County|NE|66614|United States|-6|apartment| +48832|AAAAAAAAAMOLAAAA|448|Hickory North|Court|Suite F|Waterloo|Berrien County|MI|41675|United States|-5|condo| +48833|AAAAAAAABMOLAAAA|381|Elm |Ln|Suite T|Unionville|Marion County|WV|21711|United States|-5|apartment| +48834|AAAAAAAACMOLAAAA|646|Ninth Woodland|Ave|Suite L|Hopewell|Rusk County|TX|70587|United States|-6|condo| +48835|AAAAAAAADMOLAAAA|574|Jefferson Poplar|Lane|Suite S|Mount Pleasant|Butts County|GA|31933|United States|-5|apartment| +48836|AAAAAAAAEMOLAAAA|703|Smith |RD|Suite H|Milford|Orleans Parish|LA|77137|United States|-6|single family| +48837|AAAAAAAAFMOLAAAA|383|13th |RD|Suite E|Sulphur Springs|Delta County|TX|78354|United States|-6|condo| +48838|AAAAAAAAGMOLAAAA|877|Lake Park|Boulevard|Suite 380|Oak Ridge|Marion County|WV|28371|United States|-5|apartment| +48839|AAAAAAAAHMOLAAAA|128||Drive||Hamilton|Roanoke County|||United States||condo| +48840|AAAAAAAAIMOLAAAA|654|12th |Circle|Suite N|Highland Park|Clatsop County|OR|96534|United States|-8|apartment| +48841|AAAAAAAAJMOLAAAA|528|River |Ct.|Suite I|Pine Grove|Teton County|ID|84593|United States|-7|single family| +48842|AAAAAAAAKMOLAAAA|929|View |Avenue|Suite L|Mount Zion|Otero County|CO|88054|United States|-7|apartment| +48843|AAAAAAAALMOLAAAA|254|Central Ridge|Street|Suite 280|Hopewell|Acadia Parish|LA|70587|United States|-6|apartment| +48844|AAAAAAAAMMOLAAAA|336|3rd Fourth|Blvd|Suite 160|Deerfield|Kearney County|NE|69840|United States|-7|condo| +48845|AAAAAAAANMOLAAAA|330|Pine Lake|ST|Suite Q|Leon|Windham County|VT|01513|United States|-5|condo| +48846|AAAAAAAAOMOLAAAA|120|West Park|Circle|Suite 190|Bayside|Columbia County|PA|19550|United States|-5|single family| +48847|AAAAAAAAPMOLAAAA|330|Spring 3rd|Circle|Suite L|Union|Wabaunsee County|KS|68721|United States|-6|single family| +48848|AAAAAAAAANOLAAAA|261|Park Maple|Ct.|Suite 190|Brookwood|Runnels County|TX|70965|United States|-6|apartment| +48849|AAAAAAAABNOLAAAA|35|Broadway Oak|Ln|Suite M|Edgewood|Merrick County|NE|60069|United States|-7|single family| +48850|AAAAAAAACNOLAAAA|325|Forest 12th|Ln|Suite M|Marion|Drew County|AR|70399|United States|-6|apartment| +48851|AAAAAAAADNOLAAAA|575|Maple |Wy|Suite T|Highland Park|Herkimer County|NY|16534|United States|-5|apartment| +48852|AAAAAAAAENOLAAAA|221|Ash |Drive|Suite W|Peoria|Volusia County|FL|39818|United States|-5|condo| +48853|AAAAAAAAFNOLAAAA|769|Sunset |Lane|Suite G|Ashland|Wheeler County|TX|74244|United States|-6|single family| +48854|AAAAAAAAGNOLAAAA|876|Sunset College|Ct.|Suite E|Five Points|Schoharie County|NY|16098|United States|-5|single family| +48855|AAAAAAAAHNOLAAAA|530|Eigth |Circle|Suite Y|Deerfield|Cochise County|AZ|89840|United States|-7|condo| +48856|AAAAAAAAINOLAAAA|98|Center |Ave|Suite 180|Georgetown|Martin County|IN|47057|United States|-5|single family| +48857|AAAAAAAAJNOLAAAA||Dogwood |Ln||Oakdale|Wicomico County|MD||||| +48858|AAAAAAAAKNOLAAAA|42|Forest First|Way|Suite 350|Lincoln|Chattooga County|GA|31289|United States|-5|apartment| +48859|AAAAAAAALNOLAAAA|448|7th South|Boulevard|Suite K|Centerville|Henry County|AL|30059|United States|-6|condo| +48860|AAAAAAAAMNOLAAAA|944|Pine Locust|Pkwy|Suite 40|Arlington|Stillwater County|MT|66557|United States|-7|single family| +48861|AAAAAAAANNOLAAAA|972|Lincoln 12th|Road|Suite 410|Highland Park|Norton city|VA|26534|United States|-5|apartment| +48862|AAAAAAAAONOLAAAA|976|Laurel Maple|Pkwy|Suite B|Edgewood|Winkler County|TX|70069|United States|-6|apartment| +48863|AAAAAAAAPNOLAAAA|284|Ash 1st|Ave|Suite 100|Lebanon|Covington city|VA|22898|United States|-5|single family| +48864|AAAAAAAAAOOLAAAA|108|Forest |Drive|Suite 250|New Hope|Mercer County|MO|69431|United States|-6|apartment| +48865|AAAAAAAABOOLAAAA|642|Cherry |Road|Suite M|Highland Park|Latah County|ID|86534|United States|-7|single family| +48866|AAAAAAAACOOLAAAA|833|10th 8th|RD|Suite 290|Oakwood|Essex County|MA|00769|United States|-5|single family| +48867|AAAAAAAADOOLAAAA|691|9th East|Pkwy|Suite 450|Marion|Marion County|IA|50399|United States|-6|apartment| +48868|AAAAAAAAEOOLAAAA|47|South |Circle|Suite 190|Oak Grove|DeWitt County|TX|78370|United States|-6|single family| +48869|AAAAAAAAFOOLAAAA|386|Maple Valley|RD|Suite 480|Jamestown|Hardin County|TX|76867|United States|-6|condo| +48870|AAAAAAAAGOOLAAAA|864|Adams |Lane|Suite 110|Georgetown|Callahan County|TX|77057|United States|-6|apartment| +48871|AAAAAAAAHOOLAAAA|955|View Fifth|Court|Suite N|Arlington|Elbert County|CO|86557|United States|-7|condo| +48872|AAAAAAAAIOOLAAAA|186|1st Pine|Blvd|Suite 240|Five Points|Macoupin County|IL|66098|United States|-6|condo| +48873|AAAAAAAAJOOLAAAA|972|2nd Third|Pkwy|Suite 290|Marion|Lane County|OR|90399|United States|-8|apartment| +48874|AAAAAAAAKOOLAAAA|901|3rd |Dr.|Suite Q|Lincoln|Panola County|MS|51289|United States|-6|apartment| +48875|AAAAAAAALOOLAAAA|166|Johnson |Drive|Suite M|Springfield|Columbia County|GA|39303|United States|-5|condo| +48876|AAAAAAAAMOOLAAAA|113|8th |Ave|Suite V|Edgewood|Jackson County|OR|90069|United States|-8|apartment| +48877|AAAAAAAANOOLAAAA|819|Second |Road|Suite 450|Summit|Putnam County|TN|30499|United States|-6|condo| +48878|AAAAAAAAOOOLAAAA|402|4th 2nd|RD|Suite M|Wilson|Iowa County|WI|56971|United States|-6|apartment| +48879|AAAAAAAAPOOLAAAA|286|Dogwood First|Circle|Suite 280|Union Hill|Crook County|OR|97746|United States|-8|apartment| +48880|AAAAAAAAAPOLAAAA|770|7th River|Circle|Suite 460|Clinton|Harnett County|NC|28222|United States|-5|condo| +48881|AAAAAAAABPOLAAAA|44|First |ST|Suite Q|Sulphur Springs|Harmon County|OK|78354|United States|-6|single family| +48882|AAAAAAAACPOLAAAA|125|10th |Wy|Suite 460|Red Hill|Marin County|CA|94338|United States|-8|apartment| +48883|AAAAAAAADPOLAAAA|329|Park |Ave|Suite 110|Riverdale|Stone County|AR|79391|United States|-6|single family| +48884|AAAAAAAAEPOLAAAA|102|Railroad Hickory|Ave|Suite 320|Oakland|Tuscola County|MI|49843|United States|-5|condo| +48885|AAAAAAAAFPOLAAAA|548|4th Lincoln|Circle|Suite P|Hamilton|Johnson County|TN|32808|United States|-6|apartment| +48886|AAAAAAAAGPOLAAAA|218|Oak 2nd|Way|Suite 0|Valley View|Sherman County|TX|75124|United States|-6|apartment| +48887|AAAAAAAAHPOLAAAA|746|Wilson |Blvd|Suite 60|Farmington|Coosa County|AL|39145|United States|-6|single family| +48888|AAAAAAAAIPOLAAAA|313|First Central|Avenue|Suite J|Greenfield|Sullivan County|PA|15038|United States|-5|single family| +48889|AAAAAAAAJPOLAAAA|712|Lakeview Walnut|Circle|Suite 240|Stringtown|Bradley County|AR|70162|United States|-6|single family| +48890|AAAAAAAAKPOLAAAA||North ||Suite 350|Bayview||PA||United States||apartment| +48891|AAAAAAAALPOLAAAA|479|Jefferson |Blvd|Suite I|Waterloo|Orangeburg County|SC|21675|United States|-5|apartment| +48892|AAAAAAAAMPOLAAAA|188|4th Lincoln|Street|Suite 200|Five Points|Vermilion Parish|LA|76098|United States|-6|apartment| +48893|AAAAAAAANPOLAAAA|229|First 3rd|Ave|Suite 80|Ward|Sawyer County|WI|50613|United States|-6|single family| +48894|AAAAAAAAOPOLAAAA|186|South Main|Ave|Suite R|Sulphur Springs|Washburn County|WI|58354|United States|-6|apartment| +48895|AAAAAAAAPPOLAAAA|56|Elm Broadway|Avenue|Suite 60|Marion|Edmonson County|KY|40399|United States|-6|apartment| +48896|AAAAAAAAAAPLAAAA|51|2nd |Parkway|Suite 220|Shady Grove|Bayfield County|WI|52812|United States|-6|condo| +48897|AAAAAAAABAPLAAAA|583|Chestnut Second|Wy|Suite 260|Glenwood|McMinn County|TN|33511|United States|-6|condo| +48898|AAAAAAAACAPLAAAA|549|West 2nd|Wy|Suite O|Fairview|Madison County|OH|45709|United States|-5|apartment| +48899|AAAAAAAADAPLAAAA||2nd |Parkway|Suite K|||GA||United States||| +48900|AAAAAAAAEAPLAAAA|297|Laurel |Boulevard|Suite Y|Liberty|Greensville County|VA|23451|United States|-5|condo| +48901|AAAAAAAAFAPLAAAA|21|Railroad Elm|Circle|Suite S|Hillcrest|Big Horn County|MT|63003|United States|-7|condo| +48902|AAAAAAAAGAPLAAAA|52|Pine Cedar|Way|Suite 60|Shiloh|Anne Arundel County|MD|29275|United States|-5|condo| +48903|AAAAAAAAHAPLAAAA|855|2nd |Lane|Suite K|Waterloo|Sawyer County|WI|51675|United States|-6|apartment| +48904|AAAAAAAAIAPLAAAA|194|11th |Ave|Suite K|Valley View|Shelby County|OH|45124|United States|-5|apartment| +48905|AAAAAAAAJAPLAAAA|945|Woodland |Avenue|Suite 220|Lakeside|Kewaunee County|WI|59532|United States|-6|single family| +48906|AAAAAAAAKAPLAAAA|596|Spruce |ST|Suite P|Spring Valley|Moultrie County|IL|66060|United States|-6|condo| +48907|AAAAAAAALAPLAAAA|895|Ash |Circle|Suite R|Brownsville|Ripley County|MO|69310|United States|-6|condo| +48908|AAAAAAAAMAPLAAAA|284|Hillcrest Maple|Blvd|Suite H|Hidden Valley|Otsego County|NY|15521|United States|-5|apartment| +48909|AAAAAAAANAPLAAAA|736|Third Park|ST|Suite H|Springdale|Smith County|MS|58883|United States|-6|single family| +48910|AAAAAAAAOAPLAAAA|12|15th Eigth|Road|Suite B|White Plains|Lyon County|KY|46622|United States|-5|single family| +48911|AAAAAAAAPAPLAAAA|751|10th Railroad|Blvd|Suite 160|Five Points|Juniata County|PA|16098|United States|-5|condo| +48912|AAAAAAAAABPLAAAA|940|Central Central|Cir.|Suite 300|Enterprise|Marion County|WV|21757|United States|-5|single family| +48913|AAAAAAAABBPLAAAA|481|Meadow |Blvd|Suite W|Oak Hill|Rush County|IN|47838|United States|-5|single family| +48914|AAAAAAAACBPLAAAA|19|Sycamore |Dr.|Suite X|Pleasant Hill|Monroe County|IL|63604|United States|-6|condo| +48915|AAAAAAAADBPLAAAA|166|15th Washington|RD|Suite 150|Farmington|Walworth County|WI|59145|United States|-6|condo| +48916|AAAAAAAAEBPLAAAA|307|12th Park|Pkwy|Suite 420|Spring Hill|Lincoln County|NE|66787|United States|-7|single family| +48917|AAAAAAAAFBPLAAAA|861|Maple Williams|RD|Suite L|Jamestown|Walker County|AL|36867|United States|-6|condo| +48918|AAAAAAAAGBPLAAAA|435|Park Miller|Ln|Suite 240|Greenwood|Douglas County|WI|58828|United States|-6|condo| +48919|AAAAAAAAHBPLAAAA|697|Hill Pine|Parkway|Suite C|Morgantown|Warren County|KY|49193|United States|-5|single family| +48920|AAAAAAAAIBPLAAAA|219|Woodland |Ave|Suite K|Woodlawn|Union County|OR|94098|United States|-8|single family| +48921|AAAAAAAAJBPLAAAA|181|5th 1st|Ave|Suite 340|Wildwood|Florence County|WI|56871|United States|-6|condo| +48922|AAAAAAAAKBPLAAAA|684|Elm |Wy|Suite Q|Hastings|Oldham County|KY|46888|United States|-5|apartment| +48923|AAAAAAAALBPLAAAA|352|Green |Lane|Suite N|Red Hill|Franklin County|NC|24338|United States|-5|condo| +48924|AAAAAAAAMBPLAAAA|877|Cedar Oak|Ln|Suite 330|Concord|Pulaski County|IL|64107|United States|-6|apartment| +48925|AAAAAAAANBPLAAAA|449|River 2nd|Ln|Suite 270|Glenwood|Washington County|RI|04111|United States|-5|condo| +48926|AAAAAAAAOBPLAAAA|13|8th |Pkwy|Suite 290|Forest Hills|Mecosta County|MI|49237|United States|-5|apartment| +48927|AAAAAAAAPBPLAAAA|836|Maple First|Blvd|Suite H|Highland Park|Lee County|NC|26534|United States|-5|apartment| +48928|AAAAAAAAACPLAAAA|900|4th |Wy|Suite T|Bethel|Monroe County|MS|55281|United States|-6|condo| +48929|AAAAAAAABCPLAAAA|84|13th Cherry|Blvd|Suite J|Hartland|Somervell County|TX|76594|United States|-6|apartment| +48930|AAAAAAAACCPLAAAA|766|Miller |RD|Suite O|Dover|Bonner County|ID|86237|United States|-7|single family| +48931|AAAAAAAADCPLAAAA|816|First |Pkwy|Suite 80|Five Points|Phillips County|KS|66098|United States|-6|apartment| +48932|AAAAAAAAECPLAAAA|655|10th Park|Boulevard|Suite R|Antioch|Valencia County|NM|88605|United States|-7|condo| +48933|AAAAAAAAFCPLAAAA|47|Third |Boulevard|Suite 150|Oakland|Shawano County|WI|59843|United States|-6|apartment| +48934|AAAAAAAAGCPLAAAA|287|11th |Ave|Suite Q|Midway|Lewis County|TN|31904|United States|-6|condo| +48935|AAAAAAAAHCPLAAAA|381|Walnut Sunset|Dr.|Suite 0|Mount Pleasant|Lake County|IL|61933|United States|-6|condo| +48936|AAAAAAAAICPLAAAA|516|Center College|Dr.|Suite 30|Florence|O-Brien County|IA|53394|United States|-6|apartment| +48937|AAAAAAAAJCPLAAAA|258|Lincoln Fourth|Drive|Suite W|Doyle|Pierce County|GA|38434|United States|-5|single family| +48938|AAAAAAAAKCPLAAAA|||||Spring Hill|||96787||-8|condo| +48939|AAAAAAAALCPLAAAA|485|North Dogwood|Drive|Suite 210|Riverside|Unicoi County|TN|39231|United States|-6|single family| +48940|AAAAAAAAMCPLAAAA|681|North Highland|Lane|Suite L|Greenwood|Harlan County|NE|68828|United States|-6|single family| +48941|AAAAAAAANCPLAAAA|836|Ash |Lane|Suite 150|Red Hill|Custer County|SD|54338|United States|-6|single family| +48942|AAAAAAAAOCPLAAAA|763|Elm |Parkway|Suite 340|Red Hill|Lawrence County|AR|74338|United States|-6|single family| +48943|AAAAAAAAPCPLAAAA||Fourth |||Shiloh|Dearborn County||49275|United States|-5|| +48944|AAAAAAAAADPLAAAA|789|Hill |Ct.|Suite E|Riverview|Hickman County|KY|49003|United States|-6|apartment| +48945|AAAAAAAABDPLAAAA|484|1st |Boulevard|Suite 370|Pleasant Hill|Onslow County|NC|23604|United States|-5|single family| +48946|AAAAAAAACDPLAAAA|785|Second |Avenue|Suite 440|Greenville|Butler County|PA|11387|United States|-5|single family| +48947|AAAAAAAADDPLAAAA|810|South |Way|Suite R|White Oak|Adair County|KY|46668|United States|-6|condo| +48948|AAAAAAAAEDPLAAAA|317|Smith 7th|Pkwy|Suite T|Preston|Dodge County|GA|31792|United States|-5|apartment| +48949|AAAAAAAAFDPLAAAA|532|6th North|ST|Suite E|Enterprise|Brown County|OH|41757|United States|-5|condo| +48950|AAAAAAAAGDPLAAAA|683|Birch |Dr.|Suite J|Harmony|Cass County|MO|65804|United States|-6|apartment| +48951|AAAAAAAAHDPLAAAA|196|College Washington|RD|Suite 450|Maple Grove|Sullivan County|PA|18252|United States|-5|single family| +48952|AAAAAAAAIDPLAAAA|354|Poplar |Wy|Suite J|Glenwood|Twiggs County|GA|33511|United States|-5|single family| +48953|AAAAAAAAJDPLAAAA|923|4th |Wy|Suite 460|Hillcrest|Hopewell city|VA|23003|United States|-5|single family| +48954|AAAAAAAAKDPLAAAA|260|Cedar |Street|Suite G|Woodland|Searcy County|AR|74854|United States|-6|single family| +48955|AAAAAAAALDPLAAAA|215|4th Spruce|Ln|Suite 140|Salem|Woodbury County|IA|58048|United States|-6|condo| +48956|AAAAAAAAMDPLAAAA|539|Valley |Dr.|Suite U|Pine Hill|Greene County|MS|59236|United States|-6|single family| +48957|AAAAAAAANDPLAAAA||Church 7th|Road|Suite B|Antioch||IA|58605|United States||apartment| +48958|AAAAAAAAODPLAAAA|949|Valley North|RD|Suite 410|Pine Grove|Macon County|NC|24593|United States|-5|condo| +48959|AAAAAAAAPDPLAAAA|213|Third |Circle|Suite J|Pleasant Grove|Coweta County|GA|34136|United States|-5|condo| +48960|AAAAAAAAAEPLAAAA|805|Main Woodland|Lane|Suite N|Enterprise|Juniata County|PA|11757|United States|-5|single family| +48961|AAAAAAAABEPLAAAA|206|Cedar |Pkwy|Suite G|Shady Grove|McPherson County|SD|52812|United States|-7|apartment| +48962|AAAAAAAACEPLAAAA|416|Fifth |Pkwy|Suite 160|Riverside|McCurtain County|OK|79231|United States|-6|single family| +48963|AAAAAAAADEPLAAAA|31|Hickory |Way|Suite S|Wilson|Maricopa County|AZ|86971|United States|-7|condo| +48964|AAAAAAAAEEPLAAAA|224|Lakeview View|Way|Suite 320|Macedonia|Eddy County|ND|51087|United States|-6|condo| +48965|AAAAAAAAFEPLAAAA|549|Lincoln Cherry|ST|Suite 360|Oak Grove|Lincoln County|GA|38370|United States|-5|condo| +48966|AAAAAAAAGEPLAAAA|505|View |Pkwy|Suite 370|Springfield|Dodge County|MN|59303|United States|-6|condo| +48967|AAAAAAAAHEPLAAAA|775|4th Elm|Wy|Suite X|Plainview|Lewis County|KY|43683|United States|-5|single family| +48968|AAAAAAAAIEPLAAAA|881|6th 11th|Ave|Suite G|Franklin|Kennebec County|ME|09701|United States|-5|single family| +48969|AAAAAAAAJEPLAAAA|298|8th |Boulevard|Suite 0|Shady Grove|Nottoway County|VA|22812|United States|-5|single family| +48970|AAAAAAAAKEPLAAAA|581|6th |Wy|Suite 460|Liberty|Green County|WI|53451|United States|-6|single family| +48971|AAAAAAAALEPLAAAA|753|15th |Boulevard|Suite T|Spring Hill|Monroe County|IL|66787|United States|-6|single family| +48972|AAAAAAAAMEPLAAAA|695|Jefferson |Lane|Suite V|Sunnyside|Wyandot County|OH|41952|United States|-5|condo| +48973|AAAAAAAANEPLAAAA|876|15th ||||Perquimans County|NC|26787|United States|-5|apartment| +48974|AAAAAAAAOEPLAAAA|395|Hickory |Pkwy|Suite U|Franklin|Montgomery County|KS|69101|United States|-6|apartment| +48975|AAAAAAAAPEPLAAAA|339|Oak |Cir.|Suite 300|Providence|Big Horn County|WY|86614|United States|-7|apartment| +48976|AAAAAAAAAFPLAAAA|871|8th |ST|Suite E|Macedonia|Columbia County|PA|11087|United States|-5|single family| +48977|AAAAAAAABFPLAAAA|92|Jackson Davis|Pkwy|Suite N|Wilson|Green Lake County|WI|56971|United States|-6|apartment| +48978|AAAAAAAACFPLAAAA|13|First |Cir.|Suite M|Stringtown|McLeod County|MN|50162|United States|-6|single family| +48979|AAAAAAAADFPLAAAA|718|Poplar |Road|Suite 430|Brownsville|Garfield County|WA|99310|United States|-8|single family| +48980|AAAAAAAAEFPLAAAA|123|Green Cherry|Blvd|Suite S|Springfield|||||-6|| +48981|AAAAAAAAFFPLAAAA|995|Adams |Parkway|Suite L|Marion|Suwannee County|FL|30399|United States|-5|single family| +48982|AAAAAAAAGFPLAAAA|200|College |Drive|Suite 370|Waterloo|Otsego County|NY|11675|United States|-5|single family| +48983|AAAAAAAAHFPLAAAA||||Suite 280||Windham County|||||condo| +48984|AAAAAAAAIFPLAAAA|699|Washington Woodland|Parkway|Suite 120|Friendship|Wheeler County|TX|74536|United States|-6|single family| +48985|AAAAAAAAJFPLAAAA|642|Elm |Cir.|Suite 230|Clinton|Perkins County|SD|58222|United States|-7|condo| +48986|AAAAAAAAKFPLAAAA|231|Jefferson |Blvd|Suite 140|Macedonia|Seward County|KS|61087|United States|-6|condo| +48987|AAAAAAAALFPLAAAA|370|Davis Park|Avenue|Suite P|Riverside|Stone County|AR|79231|United States|-6|apartment| +48988|AAAAAAAAMFPLAAAA|769|Oak |Court|Suite 390|Oak Grove|Tangipahoa Parish|LA|78370|United States|-6|apartment| +48989|AAAAAAAANFPLAAAA|647|College River|Ct.|Suite 320|Woodland|Crockett County|TX|74854|United States|-6|condo| +48990|AAAAAAAAOFPLAAAA|360|Maple |Drive|Suite G|Concord|Nez Perce County|ID|84107|United States|-7|condo| +48991|AAAAAAAAPFPLAAAA|30|Birch Maple|Drive|Suite 270|Maple Grove|Marshall County|TN|38252|United States|-6|single family| +48992|AAAAAAAAAGPLAAAA|477|Mill |ST|Suite Q|Cedar Grove|Kinney County|TX|70411|United States|-6|apartment| +48993|AAAAAAAABGPLAAAA|63|Highland |RD|Suite N|Arlington|Lee County|GA|36557|United States|-5|condo| +48994|AAAAAAAACGPLAAAA|398|Pine |Court|Suite L|Bunker Hill|Henderson County|TX|70150|United States|-6|condo| +48995|AAAAAAAADGPLAAAA|603|2nd Oak|Boulevard|Suite T|Newport|Onslow County|NC|21521|United States|-5|single family| +48996|AAAAAAAAEGPLAAAA|961|12th |Circle|Suite C|Highland|Canadian County|OK|79454|United States|-6|single family| +48997|AAAAAAAAFGPLAAAA|614|Broadway Cherry|Ct.|Suite 370|Shiloh|Perry County|PA|19275|United States|-5|condo| +48998|AAAAAAAAGGPLAAAA|151|Smith |Wy|Suite N|Bunker Hill|Wise County|VA|20150|United States|-5|condo| +48999|AAAAAAAAHGPLAAAA|139|Main |Avenue|Suite C|Springfield|Aurora County|SD|59303|United States|-6|apartment| +49000|AAAAAAAAIGPLAAAA|206|Hickory Maple|Road|Suite 200|Deerfield|Fresno County|CA|99840|United States|-8|single family| +49001|AAAAAAAAJGPLAAAA|697|View Central|RD|Suite 80|Woodlawn|Pickens County|GA|34098|United States|-5|condo| +49002|AAAAAAAAKGPLAAAA|154|Woodland |Drive|Suite Y|Mount Olive|Scott County|IL|68059|United States|-6|condo| +49003|AAAAAAAALGPLAAAA|698|4th |RD|Suite E|Saint Clair|Madison County|IL|65294|United States|-6|single family| +49004|AAAAAAAAMGPLAAAA|279|10th |Court|Suite 110|Floyd|Polk County|FL|33235|United States|-5|single family| +49005|AAAAAAAANGPLAAAA|468|11th |RD|Suite 90|Wildwood|Tift County|GA|36871|United States|-5|condo| +49006|AAAAAAAAOGPLAAAA|256|Church |Ln|Suite 220|Woodville|Jefferson Parish|LA|74289|United States|-6|apartment| +49007|AAAAAAAAPGPLAAAA|483|West Park|Ave|Suite 420|Oak Grove|Clifton Forge city|VA|28370|United States|-5|apartment| +49008|AAAAAAAAAHPLAAAA|147|7th 11th|Wy|Suite 350|Springdale|Dodge County|GA|38883|United States|-5|condo| +49009|AAAAAAAABHPLAAAA|396|Lee |Street|Suite 420|Waterloo|Westchester County|NY|11675|United States|-5|condo| +49010|AAAAAAAACHPLAAAA|726|Main |Street|Suite N|Concord|Boise County|ID|84107|United States|-7|apartment| +49011|AAAAAAAADHPLAAAA|659|Elm Washington|Boulevard|Suite J|Lakeview|Davidson County|NC|28579|United States|-5|apartment| +49012|AAAAAAAAEHPLAAAA|793|Walnut |Court|Suite J|Oak Grove|Baltimore County|MD|28370|United States|-5|single family| +49013|AAAAAAAAFHPLAAAA|250|8th Ninth|Boulevard|Suite V|Glenwood|Knott County|KY|43511|United States|-5|single family| +49014|AAAAAAAAGHPLAAAA|887|9th |Court|Suite X|Wilson|Nobles County|MN|56971|United States|-6|single family| +49015|AAAAAAAAHHPLAAAA|714|Spring Meadow|Way|Suite 410|Franklin|Sutter County|CA|99101|United States|-8|single family| +49016|AAAAAAAAIHPLAAAA|318|Park Cherry|ST|Suite M|Fairview|Baylor County|TX|75709|United States|-6|condo| +49017|AAAAAAAAJHPLAAAA|853|4th West|Road|Suite 450|Oakland|Logan County|ND|59843|United States|-6|apartment| +49018|AAAAAAAAKHPLAAAA|705|Sycamore |Ln|Suite 490|Plainview|Washington County|MS|53683|United States|-6|condo| +49019|AAAAAAAALHPLAAAA|178|Sycamore Mill|Pkwy|Suite 50|Harmony|Tallapoosa County|AL|35804|United States|-6|condo| +49020|AAAAAAAAMHPLAAAA|299|10th |Parkway|Suite 160|California|Alamosa County|CO|80141|United States|-7|single family| +49021|AAAAAAAANHPLAAAA|567|Hill Hill|Avenue|Suite V|Forestville|Moniteau County|MO|63027|United States|-6|single family| +49022|AAAAAAAAOHPLAAAA|4|Woodland |Court|Suite 320|Harmony|Cameron County|TX|75804|United States|-6|condo| +49023|AAAAAAAAPHPLAAAA||Ninth Smith|RD||New Hope|Perry County||39431|United States||| +49024|AAAAAAAAAIPLAAAA|253|Davis |Ct.|Suite G|Arlington|Penobscot County|ME|07157|United States|-5|apartment| +49025|AAAAAAAABIPLAAAA|314|7th Dogwood|Street|Suite E|Birmingham|Middlesex County|NJ|03972|United States|-5|condo| +49026|AAAAAAAACIPLAAAA|221|4th Forest|Street|Suite 140|Kingston|Tama County|IA|54975|United States|-6|single family| +49027|AAAAAAAADIPLAAAA|301|Woodland |Way|Suite W|New Hope|Elbert County|GA|39431|United States|-5|apartment| +49028|AAAAAAAAEIPLAAAA|432|Jackson |Drive|Suite U|New Hope|West Carroll Parish|LA|79431|United States|-6|apartment| +49029|AAAAAAAAFIPLAAAA|662|Highland Valley|ST|Suite V|Franklin|Lincoln County|WY|89101|United States|-7|single family| +49030|AAAAAAAAGIPLAAAA|417|Smith Ninth|ST|Suite 100|Liberty|Chaffee County|CO|83451|United States|-7|apartment| +49031|AAAAAAAAHIPLAAAA|764|5th |ST|Suite 470|Brownsville|Merced County|CA|99310|United States|-8|condo| +49032|AAAAAAAAIIPLAAAA|116|Third |Circle|Suite T|Oak Grove|De Witt County|IL|68370|United States|-6|apartment| +49033|AAAAAAAAJIPLAAAA|570|Hickory 15th|Ln|Suite N|Belfast|Latimer County|OK|70125|United States|-6|single family| +49034|AAAAAAAAKIPLAAAA|458|2nd West|RD|Suite E|Oakland|Lancaster County|NE|69843|United States|-7|single family| +49035|AAAAAAAALIPLAAAA|542|Hill |Ave|Suite 250|Summit|Lyon County|KS|60499|United States|-6|single family| +49036|AAAAAAAAMIPLAAAA|556|Wilson |Ln|Suite 280|Oakdale|Breathitt County|KY|49584|United States|-6|condo| +49037|AAAAAAAANIPLAAAA|249|5th |Court|Suite 70|Woodville|Kankakee County|IL|64289|United States|-6|condo| +49038|AAAAAAAAOIPLAAAA|610|9th 1st|Blvd|Suite 240|Spring Hill|Montezuma County|CO|86787|United States|-7|single family| +49039|AAAAAAAAPIPLAAAA||||Suite Y|Omega|||46364|United States|-5|condo| +49040|AAAAAAAAAJPLAAAA|568|View Jefferson|Road|Suite F|Buena Vista|Grant County|AR|75752|United States|-6|apartment| +49041|AAAAAAAABJPLAAAA|67|2nd |Boulevard|Suite S|Peoria|Chariton County|MO|69818|United States|-6|single family| +49042|AAAAAAAACJPLAAAA|340|Washington |Pkwy|Suite 50|Fisher|Pacific County|WA|92819|United States|-8|apartment| +49043|AAAAAAAADJPLAAAA|923|Pine Birch|Road|Suite Y|Liberty|Perry County|IN|43451|United States|-5|condo| +49044|AAAAAAAAEJPLAAAA|977|||Suite 400|Hillcrest||||||| +49045|AAAAAAAAFJPLAAAA|707|8th |Drive|Suite N|Marion|Cass County|IA|50399|United States|-6|apartment| +49046|AAAAAAAAGJPLAAAA|5|Woodland |Lane|Suite 0|Langdon|Carroll County|KY|40852|United States|-6|condo| +49047|AAAAAAAAHJPLAAAA|280|5th |Parkway|Suite D|New Hope|Smith County|TX|79431|United States|-6|apartment| +49048|AAAAAAAAIJPLAAAA|466|Sunset Oak|Wy|Suite 490|Farmington|McHenry County|IL|69145|United States|-6|condo| +49049|AAAAAAAAJJPLAAAA|310|Ash West|Boulevard|Suite I|Union Hill|Obion County|TN|37746|United States|-6|apartment| +49050|AAAAAAAAKJPLAAAA|139|6th East|Street|Suite 410|Richville|West Baton Rouge Parish|LA|75945|United States|-6|single family| +49051|AAAAAAAALJPLAAAA|424|Spring |Drive|Suite P|Highland|Kossuth County|IA|59454|United States|-6|condo| +49052|AAAAAAAAMJPLAAAA|47|9th |Ct.|Suite F|Grant|Gilliam County|OR|95933|United States|-8|condo| +49053|AAAAAAAANJPLAAAA|594|Lakeview College|Wy|Suite 440|Summit|Mahnomen County|MN|50499|United States|-6|apartment| +49054|AAAAAAAAOJPLAAAA|252|Adams |Cir.|Suite Y|Concord|Fall River County|SD|54107|United States|-6|single family| +49055|AAAAAAAAPJPLAAAA|933|10th Valley|Wy|Suite 90|Mountain View|Fallon County|MT|64466|United States|-7|single family| +49056|AAAAAAAAAKPLAAAA|901|Lakeview |Cir.|Suite E|Harmony|Burke County|GA|35804|United States|-5|apartment| +49057|AAAAAAAABKPLAAAA|449|Cedar |Ln|Suite H|Arlington|Polk County|OR|96557|United States|-8|apartment| +49058|AAAAAAAACKPLAAAA|215|10th |Cir.|Suite P|Union|Liberty County|FL|38721|United States|-5|condo| +49059|AAAAAAAADKPLAAAA|121|Third |Blvd|Suite M|Quincy|Montgomery County|MO|63868|United States|-6|apartment| +49060|AAAAAAAAEKPLAAAA|327|6th Elm|Drive|Suite 320|Oakwood|Owen County|KY|40169|United States|-5|apartment| +49061|AAAAAAAAFKPLAAAA|490|7th |Wy|Suite 50|Oakland|Ashtabula County|OH|49843|United States|-5|apartment| +49062|AAAAAAAAGKPLAAAA|565|Johnson Madison|Dr.|Suite F|Denmark|Dorchester County|MD|25576|United States|-5|apartment| +49063|AAAAAAAAHKPLAAAA|252|Pine Maple|Street|Suite 390|Woodland|Montgomery County|GA|34854|United States|-5|single family| +49064|AAAAAAAAIKPLAAAA|||||Pine Grove||||United States|-6|single family| +49065|AAAAAAAAJKPLAAAA|368|North |Avenue|Suite V|Greenville|Powell County|KY|41387|United States|-5|condo| +49066|AAAAAAAAKKPLAAAA|66|Pine Maple|Street|Suite J|Five Forks|McCormick County|SC|22293|United States|-5|single family| +49067|AAAAAAAALKPLAAAA|208|West Hill|Cir.|Suite B|Buena Vista|Delaware County|IN|45752|United States|-5|single family| +49068|AAAAAAAAMKPLAAAA|401|View 7th|Circle|Suite 230|Oakland|Warren County|IN|49843|United States|-5|apartment| +49069|AAAAAAAANKPLAAAA||1st ||Suite 110|||KY||||apartment| +49070|AAAAAAAAOKPLAAAA|162|South |Road|Suite J|Pleasant Grove|Douglas County|NV|84136|United States|-8|apartment| +49071|AAAAAAAAPKPLAAAA|762|10th |Avenue|Suite 10|Oakland|Magoffin County|KY|49843|United States|-5|apartment| +49072|AAAAAAAAALPLAAAA|645|Maple |Ave|Suite A|Walnut Grove|Butler County|OH|47752|United States|-5|apartment| +49073|AAAAAAAABLPLAAAA|555|Willow |Pkwy|Suite Q|Wilton|Baxter County|AR|76997|United States|-6|single family| +49074|AAAAAAAACLPLAAAA|701|6th |Parkway|Suite 240|Brookwood|DeWitt County|TX|70965|United States|-6|condo| +49075|AAAAAAAADLPLAAAA|613|North Railroad|Cir.|Suite 140|Sunnyside|Logan County|IL|61952|United States|-6|condo| +49076|AAAAAAAAELPLAAAA|919|7th |Circle|Suite 120|Union Hill|La Salle County|IL|67746|United States|-6|apartment| +49077|AAAAAAAAFLPLAAAA|958|College |Cir.|Suite Q|Riverside|Bartholomew County|IN|49231|United States|-5|apartment| +49078|AAAAAAAAGLPLAAAA|623|Main |Avenue|Suite J|Pleasant Grove|Morrow County|OR|94136|United States|-8|condo| +49079|AAAAAAAAHLPLAAAA|710|14th Sunset|Drive|Suite L|Midway|Baker County|FL|31904|United States|-5|single family| +49080|AAAAAAAAILPLAAAA|530|Hickory |Lane|Suite Q|Clinton|Piute County|UT|88222|United States|-7|apartment| +49081|AAAAAAAAJLPLAAAA|656|Fourteenth East|Pkwy|Suite 450|Mount Olive|Mitchell County|NC|28059|United States|-5|single family| +49082|AAAAAAAAKLPLAAAA|685|Elevnth |ST|Suite 130|Spring Valley|Weber County|UT|86060|United States|-7|single family| +49083|AAAAAAAALLPLAAAA|921|||Suite 390|||||||single family| +49084|AAAAAAAAMLPLAAAA|19|Smith |Court|Suite M|Lakewood|Palo Pinto County|TX|78877|United States|-6|apartment| +49085|AAAAAAAANLPLAAAA|117|Poplar East|Wy|Suite 160|Centerville|Gregg County|TX|70059|United States|-6|apartment| +49086|AAAAAAAAOLPLAAAA|953|Cherry |Lane|Suite 230|Harmony|Fillmore County|MN|55804|United States|-6|condo| +49087|AAAAAAAAPLPLAAAA|849|Center 7th|Circle|Suite 320|Highland Park|Lemhi County|ID|86534|United States|-7|apartment| +49088|AAAAAAAAAMPLAAAA|831|Church Ridge|Cir.|Suite O|Mount Olive|Jefferson County|PA|18059|United States|-5|single family| +49089|AAAAAAAABMPLAAAA|653|Maple Mill|Parkway|Suite W|Friendship|Carbon County|MT|64536|United States|-7|single family| +49090|AAAAAAAACMPLAAAA|4|Franklin Smith|Circle|Suite B|Glenwood|Furnas County|NE|63511|United States|-6|condo| +49091|AAAAAAAADMPLAAAA|150|3rd |Road|Suite 400|Maple Grove|Edgar County|IL|68252|United States|-6|condo| +49092|AAAAAAAAEMPLAAAA|398|Mill Eigth|Drive|Suite 320|Bayview|Lac qui Parle County|MN|59672|United States|-6|single family| +49093|AAAAAAAAFMPLAAAA|603|Fourth Dogwood|Cir.|Suite 50|Ashland|Pinal County|AZ|84244|United States|-7|condo| +49094|AAAAAAAAGMPLAAAA|904|Oak |Road|Suite 180|Georgetown|Meeker County|MN|57057|United States|-6|apartment| +49095|AAAAAAAAHMPLAAAA|713|Sycamore |Cir.|Suite 310|Spring Valley|Edgefield County|SC|26060|United States|-5|apartment| +49096|AAAAAAAAIMPLAAAA|13|5th Lincoln|Way|Suite 190|Greenfield|Northwest Arctic Borough|AK|95038|United States|-9|single family| +49097|AAAAAAAAJMPLAAAA|419|Ash |Way|Suite T|Red Hill|Wayne County|IN|44338|United States|-5|single family| +49098|AAAAAAAAKMPLAAAA|370|Williams |Ln|Suite X|Harmony|Weakley County|TN|35804|United States|-6|apartment| +49099|AAAAAAAALMPLAAAA|379|Ridge |Ct.|Suite D|Highland Park|Fulton County|IN|46534|United States|-5|single family| +49100|AAAAAAAAMMPLAAAA|477|Pine |Road|Suite L|Fairfield|Osage County|OK|76192|United States|-6|condo| +49101|AAAAAAAANMPLAAAA|427|9th |Road|Suite 420|Buena Vista|Mercer County|ND|55752|United States|-6|apartment| +49102|AAAAAAAAOMPLAAAA|48|13th Oak|Drive|Suite C|Pine Grove|Ouachita Parish|LA|74593|United States|-6|condo| +49103|AAAAAAAAPMPLAAAA|163|Woodland |Lane|Suite 190|Mount Vernon|Perkins County|SD|58482|United States|-7|apartment| +49104|AAAAAAAAANPLAAAA|773|Pine |Ave|Suite 10|Antioch|Benton County|IN|48605|United States|-5|single family| +49105|AAAAAAAABNPLAAAA|398|First |Boulevard|Suite C|Belmont|McCulloch County|TX|70191|United States|-6|apartment| +49106|AAAAAAAACNPLAAAA|728|Third Lake|Street|Suite E|Edgewood|Dimmit County|TX|70069|United States|-6|condo| +49107|AAAAAAAADNPLAAAA|818|Highland |Drive|Suite N|Bunker Hill|Keith County|NE|60150|United States|-7|condo| +49108|AAAAAAAAENPLAAAA|274|Maple |Ln|Suite C|Bridgeport|Medina County|TX|75817|United States|-6|condo| +49109|AAAAAAAAFNPLAAAA|245|Oak |Avenue|Suite 350|Salem|Camden County|GA|38048|United States|-5|apartment| +49110|AAAAAAAAGNPLAAAA|41|Cedar Lincoln|Street|Suite Y|Adrian|Colonial Heights city|VA|23301|United States|-5|condo| +49111|AAAAAAAAHNPLAAAA|333|Main |Lane|Suite K|Greenville|Smith County|TN|31387|United States|-6|apartment| +49112|AAAAAAAAINPLAAAA|727|First |Parkway|Suite 150|Shady Grove|Boise County|ID|82812|United States|-7|single family| +49113|AAAAAAAAJNPLAAAA|807|Meadow |Road|Suite J|Pine Grove|Crawford County|MO|64593|United States|-6|apartment| +49114|AAAAAAAAKNPLAAAA|325|Oak Wilson|Avenue|Suite 160|Glenwood|Wayne County|OH|43511|United States|-5|apartment| +49115|AAAAAAAALNPLAAAA|22|1st Mill|Wy|Suite 260|Clifton|Livingston County|MO|68014|United States|-6|condo| +49116|AAAAAAAAMNPLAAAA|151|Elevnth |Pkwy|Suite 330|Spring Valley|Douglas County|WI|56060|United States|-6|apartment| +49117|AAAAAAAANNPLAAAA|70|Davis |Ave|Suite 130|Riverview|King County|WA|99003|United States|-8|apartment| +49118|AAAAAAAAONPLAAAA|253|Oak River|Parkway|Suite 190|Greenfield|Gratiot County|MI|45038|United States|-5|apartment| +49119|AAAAAAAAPNPLAAAA|99|14th |Lane|Suite D|Wildwood|Berrien County|GA|36871|United States|-5|condo| +49120|AAAAAAAAAOPLAAAA|772|Johnson Meadow|Ct.|Suite 100|Chatham|Cuming County|NE|63289|United States|-6|condo| +49121|AAAAAAAABOPLAAAA|180|8th |Dr.|Suite 150|Shady Grove|Wilkes County|NC|22812|United States|-5|single family| +49122|AAAAAAAACOPLAAAA|372|Third Meadow|Boulevard|Suite A|Crossroads|Coleman County|TX|70534|United States|-6|condo| +49123|AAAAAAAADOPLAAAA|59|2nd |Ln|Suite 330|Lebanon|Uvalde County|TX|72898|United States|-6|apartment| +49124|AAAAAAAAEOPLAAAA|571|Sixth Hill|Circle|Suite D|Plainview|Clay County|NE|63683|United States|-6|condo| +49125|AAAAAAAAFOPLAAAA|244|12th Ash|Ave|Suite 200|Galena|Armstrong County|TX|74369|United States|-6|condo| +49126|AAAAAAAAGOPLAAAA|605|Cedar |Way|Suite M|Spring Valley|Keya Paha County|NE|66060|United States|-7|single family| +49127|AAAAAAAAHOPLAAAA|579|Third |Circle|Suite J|Providence|Beaver County|OK|76614|United States|-6|apartment| +49128|AAAAAAAAIOPLAAAA|105|River Mill|Parkway|Suite F|Glendale|Allen Parish|LA|73951|United States|-6|single family| +49129|AAAAAAAAJOPLAAAA|512|Park 10th|Road|Suite S|Enterprise|Lexington city|VA|21757|United States|-5|apartment| +49130|AAAAAAAAKOPLAAAA|48|Walnut College|RD|Suite 380|Pine Grove|Middlesex County|NJ|05193|United States|-5|condo| +49131|AAAAAAAALOPLAAAA|597|Jackson |Drive|Suite 350|Sulphur Springs|Grand County|CO|88354|United States|-7|condo| +49132|AAAAAAAAMOPLAAAA|883|Spring Sixth|Cir.|Suite 160|Lebanon|Schuyler County|NY|12898|United States|-5|condo| +49133|AAAAAAAANOPLAAAA|447|Cedar |Circle|Suite 470|Forest Hills|Cibola County|NM|89237|United States|-7|condo| +49134|AAAAAAAAOOPLAAAA|144|Center Eigth|Dr.|Suite 270|Greenfield|Arlington County|VA|25038|United States|-5|apartment| +49135|AAAAAAAAPOPLAAAA|745|Twelfth 14th|Dr.|Suite D|Waterloo|Lancaster County|PA|11675|United States|-5|condo| +49136|AAAAAAAAAPPLAAAA|366|Smith |Drive|Suite D|Walnut Grove|Fall River County|SD|57752|United States|-6|apartment| +49137|AAAAAAAABPPLAAAA|866|First |Parkway|Suite T|Woodlawn|Gogebic County|MI|44098|United States|-5|single family| +49138|AAAAAAAACPPLAAAA|623|Locust |Way|Suite 90|Riverview|Summers County|WV|29003|United States|-5|single family| +49139|AAAAAAAADPPLAAAA|217|Oak Cedar|Wy|Suite P|Jackson|Lancaster County|SC|29583|United States|-5|apartment| +49140|AAAAAAAAEPPLAAAA|454|Lakeview |Lane|Suite 70|Providence|Chisago County|MN|56614|United States|-6|condo| +49141|AAAAAAAAFPPLAAAA|18|North |Ave|Suite F|Bunker Hill|Powhatan County|VA|20150|United States|-5|apartment| +49142|AAAAAAAAGPPLAAAA|717|Main Fifth|Ln|Suite F|Oakdale|Champaign County|IL|69584|United States|-6|single family| +49143|AAAAAAAAHPPLAAAA|24|Church |Cir.|Suite 450|Marion|Ingham County|MI|40399|United States|-5|condo| +49144|AAAAAAAAIPPLAAAA|547|Pine |Way|Suite 290|Springdale|Clackamas County|OR|98883|United States|-8|single family| +49145|AAAAAAAAJPPLAAAA|516|10th Main|Boulevard|Suite S|Georgetown|Petersburg city|VA|27057|United States|-5|apartment| +49146|AAAAAAAAKPPLAAAA|720|Sunset Church|Ct.|Suite 120|Midway|Meade County|SD|51904|United States|-7|single family| +49147|AAAAAAAALPPLAAAA|272|Ridge |Circle|Suite W|Lakeside|Kearny County|KS|69532|United States|-6|apartment| +49148|AAAAAAAAMPPLAAAA|332|Davis East|ST|Suite 420|Buena Vista|Green County|KY|45752|United States|-6|condo| +49149|AAAAAAAANPPLAAAA|700|View |RD|Suite 290|Clinton|Cedar County|MO|68222|United States|-6|apartment| +49150|AAAAAAAAOPPLAAAA|728|Locust Meadow|Street|Suite 370|Bridgeport|Harrison County|WV|25817|United States|-5|condo| +49151|AAAAAAAAPPPLAAAA|559|2nd |Ln|Suite 240|Harmony|Iron County|WI|55804|United States|-6|apartment| +49152|AAAAAAAAAAAMAAAA|712|Smith |Ln|Suite 150|Arlington|Barrow County|GA|36557|United States|-5|apartment| +49153|AAAAAAAABAAMAAAA|804|Second |Lane|Suite 290|Pleasant Valley|Cass County|IL|62477|United States|-6|apartment| +49154|AAAAAAAACAAMAAAA|515|West 9th|Ln|Suite J|Greenfield|Searcy County|AR|75038|United States|-6|condo| +49155|AAAAAAAADAAMAAAA|800|Lincoln 8th|Ln|Suite 450|Shady Grove|Mason County|WV|22812|United States|-5|condo| +49156|AAAAAAAAEAAMAAAA|825|Jackson Fifth|Way|Suite E|Sawyer|Lincoln County|CO|86045|United States|-7|apartment| +49157|AAAAAAAAFAAMAAAA|555|Hillcrest Seventh|Street|Suite H|Carpenter|Crook County|OR|91147|United States|-8|single family| +49158|AAAAAAAAGAAMAAAA|333|Ninth |Drive|Suite 490|Mount Vernon|Wayne County|GA|38482|United States|-5|condo| +49159|AAAAAAAAHAAMAAAA|474|4th 10th|Road|Suite 490|Pine Grove|Roane County|TN|34593|United States|-6|condo| +49160|AAAAAAAAIAAMAAAA|875|Church Jackson|Pkwy|Suite L|Blue Springs|Clarke County|IA|54686|United States|-6|condo| +49161|AAAAAAAAJAAMAAAA|409|Third |Circle|Suite E|Highland|Floyd County|IA|59454|United States|-6|condo| +49162|AAAAAAAAKAAMAAAA|323|River Church|Boulevard|Suite 190|Maple Grove|Davis County|IA|58252|United States|-6|condo| +49163|AAAAAAAALAAMAAAA|679|North |Blvd|Suite 450|Mountain View|Mariposa County|CA|94466|United States|-8|apartment| +49164|AAAAAAAAMAAMAAAA|227|Fourth 3rd|Pkwy|Suite 360|Highland|Valley County|ID|89454|United States|-7|single family| +49165|AAAAAAAANAAMAAAA|180|6th |Circle|Suite W|Harmony|Colonial Heights city|VA|25804|United States|-5|apartment| +49166|AAAAAAAAOAAMAAAA|506|Jackson 5th|Drive|Suite 250|Georgetown|Iowa County|WI|57057|United States|-6|apartment| +49167|AAAAAAAAPAAMAAAA|604|||||Union County||42293|United States||single family| +49168|AAAAAAAAABAMAAAA|697|Church |Boulevard|Suite G|Hopewell|Covington County|AL|30587|United States|-6|apartment| +49169|AAAAAAAABBAMAAAA|371|Main |RD|Suite 100|Friendship|Mason County|WV|24536|United States|-5|condo| +49170|AAAAAAAACBAMAAAA|688|East View|Cir.|Suite X|Spring Hill|Gallia County|OH|46787|United States|-5|apartment| +49171|AAAAAAAADBAMAAAA|396|Oak Williams|Ln|Suite E|Lawrenceville|Seneca County|NY|14462|United States|-5|apartment| +49172|AAAAAAAAEBAMAAAA|548|Johnson |Court|Suite T|Clifton|Bell County|KY|48014|United States|-6|apartment| +49173|AAAAAAAAFBAMAAAA||Sunset |Court|||||86557|||| +49174|AAAAAAAAGBAMAAAA|152|View |Blvd|Suite 230|Greenfield|Kauai County|HI|95038|United States|-10|single family| +49175|AAAAAAAAHBAMAAAA|158|Main |Circle|Suite 400|Fairfield|Gregg County|TX|76192|United States|-6|condo| +49176|AAAAAAAAIBAMAAAA|979|Park |Wy|Suite H|Springdale|Delaware County|OK|78883|United States|-6|condo| +49177|AAAAAAAAJBAMAAAA|343|Chestnut Seventh|Ln|Suite 170|Buena Vista|Morton County|ND|55752|United States|-6|single family| +49178|AAAAAAAAKBAMAAAA|901|Central |Ave|Suite M|Red Hill|Henry County|IA|54338|United States|-6|single family| +49179|AAAAAAAALBAMAAAA|625|North College|Circle|Suite W|Midway|Atlantic County|NJ|02504|United States|-5|apartment| +49180|AAAAAAAAMBAMAAAA|739|Dogwood |Parkway|Suite O|King|Cochise County|AZ|80008|United States|-7|condo| +49181|AAAAAAAANBAMAAAA|326|North 2nd|ST|Suite A|Sunnyside|Surry County|NC|21952|United States|-5|condo| +49182|AAAAAAAAOBAMAAAA|503|Chestnut 5th|Dr.|Suite A|Shady Grove|Stanton County|NE|62812|United States|-7|apartment| +49183|AAAAAAAAPBAMAAAA|817|Mill 2nd|Dr.|Suite I|Liberty|Gilpin County|CO|83451|United States|-7|apartment| +49184|AAAAAAAAACAMAAAA|257|6th 1st|Avenue|Suite E|Five Points|Cabell County|WV|26098|United States|-5|condo| +49185|AAAAAAAABCAMAAAA|145|Maple Elevnth|Pkwy|Suite S|Maple Grove|Schuyler County|IL|68252|United States|-6|apartment| +49186|AAAAAAAACCAMAAAA|330|Hickory Willow|Drive|Suite 40|Brookwood|Pope County|AR|70965|United States|-6|condo| +49187|AAAAAAAADCAMAAAA|209|Church |Lane|Suite J|Riverdale|East Feliciana Parish|LA|79391|United States|-6|apartment| +49188|AAAAAAAAECAMAAAA|556|10th Madison|Ct.|Suite 460|Stringtown|Stark County|ND|50162|United States|-6|single family| +49189|AAAAAAAAFCAMAAAA|280|Maple 13th|Dr.|Suite F|Greenville|Seminole County|OK|71387|United States|-6|single family| +49190|AAAAAAAAGCAMAAAA|||||||MO||United States|-6|single family| +49191|AAAAAAAAHCAMAAAA|||Way||Oak Hill||IN||||| +49192|AAAAAAAAICAMAAAA|761|6th |Lane|Suite R|Friendship|Putnam County|WV|24536|United States|-5|single family| +49193|AAAAAAAAJCAMAAAA|398|7th |Ave|Suite 250|Jackson|Pawnee County|KS|69583|United States|-6|apartment| +49194|AAAAAAAAKCAMAAAA|909|Adams |Road|Suite 230|Quincy|Livingston County|MO|63868|United States|-6|apartment| +49195|AAAAAAAALCAMAAAA|583|Center |Boulevard|Suite F|Arlington|Tioga County|NY|16557|United States|-5|single family| +49196|AAAAAAAAMCAMAAAA|255|Railroad Railroad|Pkwy|Suite T|Forest Hills|Nome Census Area|AK|99237|United States|-9|condo| +49197|AAAAAAAANCAMAAAA|244|Center Hill|Boulevard|Suite 30|Bridgeport|Woodruff County|AR|75817|United States|-6|single family| +49198|AAAAAAAAOCAMAAAA|522|1st |Boulevard|Suite 90|Pleasant Valley|Hill County|MT|62477|United States|-7|condo| +49199|AAAAAAAAPCAMAAAA|821|12th |Road|Suite B|Oakdale|Williamson County|TN|39584|United States|-5|single family| +49200|AAAAAAAAADAMAAAA|931|Forest |Cir.|Suite Y|Five Forks|Hidalgo County|NM|82293|United States|-7|apartment| +49201|AAAAAAAABDAMAAAA|370|10th |Court|Suite J|Lebanon|Clark County|IN|42898|United States|-5|single family| +49202|AAAAAAAACDAMAAAA|282|South Elm|RD|Suite 360|Florence|Wilson County|KS|63394|United States|-6|apartment| +49203|AAAAAAAADDAMAAAA|814|Williams East|Boulevard|Suite 220|Providence|Addison County|VT|07214|United States|-5|apartment| +49204|AAAAAAAAEDAMAAAA|62|Highland Franklin|Wy|Suite 280|Plainview|Aitkin County|MN|53683|United States|-6|condo| +49205|AAAAAAAAFDAMAAAA|384|Ridge |Court|Suite W|Jamestown|Aiken County|SC|26867|United States|-5|apartment| +49206|AAAAAAAAGDAMAAAA|339|Main Elevnth|Lane|Suite 200|Lakeview|Benton County|MO|68579|United States|-6|condo| +49207|AAAAAAAAHDAMAAAA|884|Cedar Sunset|Road|Suite 360|Oakdale|Costilla County|CO|89584|United States|-7|single family| +49208|AAAAAAAAIDAMAAAA|150|7th |Road|Suite 120|Ashland|Cumberland County|NC|24244|United States|-5|condo| +49209|AAAAAAAAJDAMAAAA|839|Chestnut Wilson|Dr.|Suite T|Glendale|Ward County|TX|73951|United States|-6|condo| +49210|AAAAAAAAKDAMAAAA|216|Elm |Parkway|Suite J|Five Forks|Montgomery County|OH|42293|United States|-5|condo| +49211|AAAAAAAALDAMAAAA|537|Main |Way|Suite R|Fairbanks|Columbia County|NY|16653|United States|-5|apartment| +49212|AAAAAAAAMDAMAAAA|484|Lake |Wy|Suite 330|White Oak|Platte County|NE|66668|United States|-7|single family| +49213|AAAAAAAANDAMAAAA|767|Walnut |Ln|Suite 60|Belmont|Granville County|NC|20191|United States|-5|apartment| +49214|AAAAAAAAODAMAAAA|824|Cherry |RD|Suite 10|Oak Hill|Montgomery County|NC|27838|United States|-5|apartment| +49215|AAAAAAAAPDAMAAAA|254|Mill |Avenue|Suite 70|Unionville|Pottawatomie County|OK|71711|United States|-6|single family| +49216|AAAAAAAAAEAMAAAA|405|Third |Road|Suite 50|Lakeside|Perry County|MO|69532|United States|-6|condo| +49217|AAAAAAAABEAMAAAA|617||Boulevard|Suite 400||||||-6|| +49218|AAAAAAAACEAMAAAA|46|Miller |Wy|Suite 170|Guilford|Bedford County|PA|14408|United States|-5|single family| +49219|AAAAAAAADEAMAAAA|699|Adams |ST|Suite L|Greenville|Solano County|CA|91387|United States|-8|condo| +49220|AAAAAAAAEEAMAAAA|591|Smith 2nd|Road|Suite R|Mechanicsburg|Hancock County|WV|22219|United States|-5|single family| +49221|AAAAAAAAFEAMAAAA|485|Lake First|Court|Suite 420|Hillcrest|Limestone County|AL|33003|United States|-6|condo| +49222|AAAAAAAAGEAMAAAA|872|Jefferson 11th|Parkway|Suite 80|Florence|Mackinac County|MI|43394|United States|-5|apartment| +49223|AAAAAAAAHEAMAAAA|968|Williams |Street|Suite B|Greenwood|Montgomery County|NC|28828|United States|-5|single family| +49224|AAAAAAAAIEAMAAAA|27|Park Park|Parkway|Suite V|Woodland|Humboldt County|NV|84854|United States|-8|apartment| +49225|AAAAAAAAJEAMAAAA|647|Cedar Adams|Pkwy|Suite 10|Crossroads|Randolph County|WV|20534|United States|-5|condo| +49226|AAAAAAAAKEAMAAAA|427|4th |Lane|Suite 150|Enterprise|Surry County|VA|21757|United States|-5|apartment| +49227|AAAAAAAALEAMAAAA|801|Elm Spruce|Street|Suite 30|Oakwood|Calhoun County|IA|50169|United States|-6|apartment| +49228|AAAAAAAAMEAMAAAA|254|Oak |ST|Suite 280|Acme|DeKalb County|IL|60164|United States|-6|apartment| +49229|AAAAAAAANEAMAAAA|144|Ninth |Dr.|Suite R|Union|Salem County|NJ|09321|United States|-5|condo| +49230|AAAAAAAAOEAMAAAA|884|Ash |Drive|Suite Y|Lincoln|Hampton County|SC|21289|United States|-5|condo| +49231|AAAAAAAAPEAMAAAA|796|Spring |Ave|Suite 390|Lincoln|Washington County|TX|71289|United States|-6|apartment| +49232|AAAAAAAAAFAMAAAA|594|Oak |Avenue|Suite I|Pleasant Valley|Terrell County|GA|32477|United States|-5|condo| +49233|AAAAAAAABFAMAAAA|195|13th |Street|Suite R|Mount Vernon|Crawford County|MO|68482|United States|-6|single family| +49234|AAAAAAAACFAMAAAA|474|Lee Hillcrest|Ln|Suite S|Marion|La Salle Parish|LA|70399|United States|-6|single family| +49235|AAAAAAAADFAMAAAA||||Suite E|Oak Grove|Macon County|GA|38370|||single family| +49236|AAAAAAAAEFAMAAAA|801|View Chestnut|Court|Suite 120|Lakeville|Oswego County|NY|18811|United States|-5|condo| +49237|AAAAAAAAFFAMAAAA|924|Railroad View|Wy|Suite 450|Red Hill|Hamilton County|FL|34338|United States|-5|apartment| +49238|AAAAAAAAGFAMAAAA|236|Green Woodland|Lane|Suite S|Woodcrest|Jackson County|MN|54919|United States|-6|single family| +49239|AAAAAAAAHFAMAAAA|936|Cedar |Way|Suite 270|Kingston|Harding County|SD|54975|United States|-7|single family| +49240|AAAAAAAAIFAMAAAA|111|Locust |Ln|Suite E|Gilmore|Grant County|WI|55464|United States|-6|single family| +49241|AAAAAAAAJFAMAAAA|847|Central |Ct.|Suite U|Bristol|Sawyer County|WI|56830|United States|-6|apartment| +49242|AAAAAAAAKFAMAAAA|744|Park |Ave|Suite Q|Oak Hill|Manitowoc County|WI|57838|United States|-6|condo| +49243|AAAAAAAALFAMAAAA|297|5th Main|Parkway|Suite 460|Jackson|Searcy County|AR|79583|United States|-6|single family| +49244|AAAAAAAAMFAMAAAA|346|6th |Dr.|Suite 470|Franklin|Roosevelt County|MT|69101|United States|-7|single family| +49245|AAAAAAAANFAMAAAA|735|7th 3rd|Ave|Suite N|Friendship|Erie County|PA|14536|United States|-5|single family| +49246|AAAAAAAAOFAMAAAA|204|Pine 2nd|Dr.|Suite C|Enterprise|Montgomery County|MD|21757|United States|-5|condo| +49247|AAAAAAAAPFAMAAAA||||Suite W|Buena Vista||AR|75752||-6|apartment| +49248|AAAAAAAAAGAMAAAA|536|Locust |Road|Suite 240|Stafford|Hancock County|TN|34980|United States|-5|single family| +49249|AAAAAAAABGAMAAAA|612|Davis Cherry|Wy|Suite 160|Shady Grove|Camden County|NC|22812|United States|-5|condo| +49250|AAAAAAAACGAMAAAA|501|Johnson |Cir.|Suite X|Plainview|Jefferson County|WV|23683|United States|-5|single family| +49251|AAAAAAAADGAMAAAA|986|Lincoln Hill|RD|Suite 330|Woodville|Harrison County|WV|24289|United States|-5|condo| +49252|AAAAAAAAEGAMAAAA|847|Broadway |Lane|Suite 430|Belmont|Marion County|AL|30191|United States|-6|single family| +49253|AAAAAAAAFGAMAAAA|721|Main Maple|Avenue|Suite E|Oak Ridge|Knox County|OH|48371|United States|-5|single family| +49254|AAAAAAAAGGAMAAAA|503|Mill |Court|Suite 360|Buena Vista|Edmonson County|KY|45752|United States|-6|single family| +49255|AAAAAAAAHGAMAAAA|711|2nd |Pkwy|Suite 220|Bethel|Lake County|SD|55281|United States|-7|condo| +49256|AAAAAAAAIGAMAAAA|873|9th |RD|Suite M|Centerville|Greene County|GA|30059|United States|-5|condo| +49257|AAAAAAAAJGAMAAAA|351|Johnson |Blvd|Suite U|Spring Hill|Sullivan County|MO|66787|United States|-6|apartment| +49258|AAAAAAAAKGAMAAAA|475|Cherry Valley|Circle|Suite H|Springfield|Carroll County|IL|69303|United States|-6|single family| +49259|AAAAAAAALGAMAAAA|103|8th Railroad|Road|Suite M|Union|Shackelford County|TX|78721|United States|-6|condo| +49260|AAAAAAAAMGAMAAAA|241|Fourth |Way|Suite 160|Kingston|Escambia County|AL|34975|United States|-6|single family| +49261|AAAAAAAANGAMAAAA|1|Fourth Hickory|Street|Suite 360|Mount Vernon|Hertford County|NC|28482|United States|-5|apartment| +49262|AAAAAAAAOGAMAAAA|333|4th |Avenue|Suite J|Greenwood|Butler County|AL|38828|United States|-6|apartment| +49263|AAAAAAAAPGAMAAAA|809|11th |Street|Suite 110|Arlington|Dodge County|WI|56557|United States|-6|condo| +49264|AAAAAAAAAHAMAAAA|467|Wilson |Ct.|Suite J|Spring Valley|Jefferson County|CO|86060|United States|-7|apartment| +49265|AAAAAAAABHAMAAAA|684|Broadway Davis|Circle|Suite A|Oakwood|Jefferson County|IA|50169|United States|-6|condo| +49266|AAAAAAAACHAMAAAA|141|Park Lincoln|Drive|Suite 50|Wilson|Granite County|MT|66971|United States|-7|single family| +49267|AAAAAAAADHAMAAAA|814|Highland Valley|Circle|Suite 0|Arlington|DeKalb County|GA|36557|United States|-5|apartment| +49268|AAAAAAAAEHAMAAAA|998|Main Franklin|Boulevard|Suite W|Mount Pleasant|Eastland County|TX|71933|United States|-6|single family| +49269|AAAAAAAAFHAMAAAA|781|Mill Ash|Pkwy|Suite 300|Oakwood|Comal County|TX|70169|United States|-6|apartment| +49270|AAAAAAAAGHAMAAAA|503|1st |Blvd|Suite 20|Riverview|Allendale County|SC|29003|United States|-5|single family| +49271|AAAAAAAAHHAMAAAA|948|Laurel |Dr.|Suite R|White Oak|McClain County|OK|76668|United States|-6|condo| +49272|AAAAAAAAIHAMAAAA|744|Woodland Miller|Road|Suite 100|Harmony|Mohave County|AZ|85804|United States|-7|apartment| +49273|AAAAAAAAJHAMAAAA|193|Church |Ave|Suite S|Springfield|Washington County|IA|59303|United States|-6|apartment| +49274|AAAAAAAAKHAMAAAA|307|Main Spring|Avenue|Suite 300|Riverside|Platte County|NE|69231|United States|-7|condo| +49275|AAAAAAAALHAMAAAA|833|8th Jefferson|Boulevard|Suite X|Clifton|Boyd County|KY|48014|United States|-6|apartment| +49276|AAAAAAAAMHAMAAAA|616|Poplar |Cir.|Suite L|Hamilton|Talbot County|MD|22808|United States|-5|apartment| +49277|AAAAAAAANHAMAAAA|510|College |Ln|Suite 160|Sunnyside|Scott County|AR|71952|United States|-6|condo| +49278|AAAAAAAAOHAMAAAA|634|Willow |Lane|Suite S|Red Hill|Berks County|PA|14338|United States|-5|single family| +49279|AAAAAAAAPHAMAAAA|696|Jackson |Avenue|Suite V|Fairfield|Crowley County|CO|86192|United States|-7|condo| +49280|AAAAAAAAAIAMAAAA|98|15th Main|RD|Suite 160|Highland Park|West Feliciana Parish|LA|76534|United States|-6|apartment| +49281|AAAAAAAABIAMAAAA|929|Maple |Circle|Suite 260|Pleasant Hill|Cameron County|TX|73604|United States|-6|apartment| +49282|AAAAAAAACIAMAAAA|287|12th View|Lane|Suite 350|Springdale|Lac qui Parle County|MN|58883|United States|-6|single family| +49283|AAAAAAAADIAMAAAA|158|Hill |Road|Suite F|Gravel Hill|Powell County|KY|41944|United States|-5|condo| +49284|AAAAAAAAEIAMAAAA|125|Oak 9th|Boulevard|Suite 430|New Town|Edmonson County|KY|49634|United States|-6|single family| +49285|AAAAAAAAFIAMAAAA|897|Locust 3rd|Drive|Suite 90|Peru|Jersey County|IL|60302|United States|-6|condo| +49286|AAAAAAAAGIAMAAAA|202||Parkway||Clinton||||United States|-6|| +49287|AAAAAAAAHIAMAAAA|225|Railroad 8th|Wy|Suite Y|Riverview|Guthrie County|IA|59003|United States|-6|single family| +49288|AAAAAAAAIIAMAAAA|207|15th |Parkway|Suite F|Riverview|Mecosta County|MI|49003|United States|-5|condo| +49289|AAAAAAAAJIAMAAAA|426|Elevnth |Cir.|Suite 150|Mount Olive|Dimmit County|TX|78059|United States|-6|apartment| +49290|AAAAAAAAKIAMAAAA|313|Park Adams|Ct.|Suite O|Woodlawn|Madison County|MT|64098|United States|-7|condo| +49291|AAAAAAAALIAMAAAA|848|Willow |Drive|Suite H|Highland Park|Coosa County|AL|36534|United States|-6|apartment| +49292|AAAAAAAAMIAMAAAA|653|Center Hill|Way|Suite 440|Arlington|Sweet Grass County|MT|66557|United States|-7|condo| +49293|AAAAAAAANIAMAAAA|355|Adams |Pkwy|Suite 340|Lincoln|Okmulgee County|OK|71289|United States|-6|condo| +49294|AAAAAAAAOIAMAAAA|194|Third |Ct.|Suite 280|Waterloo|Lincoln County|ME|02275|United States|-5|condo| +49295|AAAAAAAAPIAMAAAA|310|Valley |Court|Suite W|Georgetown|Ripley County|IN|47057|United States|-5|single family| +49296|AAAAAAAAAJAMAAAA|337|College |RD|Suite X|Concord|Petersburg city|VA|24107|United States|-5|single family| +49297|AAAAAAAABJAMAAAA|703|Center Lee|Avenue|Suite 310|Spring Valley|Stephens County|OK|76060|United States|-6|condo| +49298|AAAAAAAACJAMAAAA|668|Hickory Cedar|Way|Suite 430|Bridgeport|Manassas Park city|VA|25817|United States|-5|condo| +49299|AAAAAAAADJAMAAAA|281|Elm 2nd|Street|Suite F|Lincoln|Garland County|AR|71289|United States|-6|condo| +49300|AAAAAAAAEJAMAAAA|791|Cherry |Cir.|Suite J|Newport|Hopkins County|TX|71521|United States|-6|condo| +49301|AAAAAAAAFJAMAAAA|543|Lincoln 2nd|Ave|Suite 480|Mount Pleasant|Flathead County|MT|61933|United States|-7|condo| +49302|AAAAAAAAGJAMAAAA|48|Sixth |Street|Suite 160|Antioch|Washington County|AL|38605|United States|-6|single family| +49303|AAAAAAAAHJAMAAAA|582|Williams |Wy|Suite B|Riverdale|Grundy County|TN|39391|United States|-5|single family| +49304|AAAAAAAAIJAMAAAA|577|Lee Pine|Wy|Suite F|Newtown|Jefferson Davis Parish|LA|71749|United States|-6|apartment| +49305|AAAAAAAAJJAMAAAA|209|6th |Drive|Suite 30|Union|Coffey County|KS|68721|United States|-6|apartment| +49306|AAAAAAAAKJAMAAAA|907|7th |Ln|Suite 290|Bunker Hill|Holmes County|OH|40150|United States|-5|apartment| +49307|AAAAAAAALJAMAAAA|183|Chestnut 2nd|Road|Suite C|Pleasant Valley|Kendall County|TX|72477|United States|-6|apartment| +49308|AAAAAAAAMJAMAAAA|21|Miller |Avenue|Suite J|Buena Vista|Morehouse Parish|LA|75752|United States|-6|apartment| +49309|AAAAAAAANJAMAAAA|326|Pine |Drive|Suite 230|Plainview|Asotin County|WA|93683|United States|-8|condo| +49310|AAAAAAAAOJAMAAAA|228|Oak 2nd|Drive|Suite 200|Springfield|Craighead County|AR|79303|United States|-6|apartment| +49311|AAAAAAAAPJAMAAAA|306|Chestnut Broadway|Ln|Suite F|Rockwood|Edmonson County|KY|41545|United States|-6|condo| +49312|AAAAAAAAAKAMAAAA|596|Wilson |Cir.|Suite 100|Red Hill|Nicholas County|KY|44338|United States|-5|condo| +49313|AAAAAAAABKAMAAAA|250|Broadway |Avenue|Suite O|Mountain View|Pembina County|ND|54466|United States|-6|single family| +49314|AAAAAAAACKAMAAAA|878|Second |Drive|Suite P|Newtown|Taylor County|WI|51749|United States|-6|apartment| +49315|AAAAAAAADKAMAAAA|148|Cherry First|Ave|Suite 320|Greenwood|Radford city|VA|28828|United States|-5|single family| +49316|AAAAAAAAEKAMAAAA|34|Pine |RD|Suite 400|Woodlawn|Jewell County|KS|64098|United States|-6|apartment| +49317|AAAAAAAAFKAMAAAA|464|Central |Drive|Suite T|Mount Vernon|Washington Parish|LA|78482|United States|-6|single family| +49318|AAAAAAAAGKAMAAAA|604|5th |Blvd|Suite 360|Union|Fort Bend County|TX|78721|United States|-6|apartment| +49319|AAAAAAAAHKAMAAAA|249|Valley |Parkway|Suite 290|Highland Park|Adair County|IA|56534|United States|-6|apartment| +49320|AAAAAAAAIKAMAAAA|760|Franklin |Street|Suite 210|New Hope|Hamilton County|OH|49431|United States|-5|apartment| +49321|AAAAAAAAJKAMAAAA|362|Spring |Ave|Suite Q|Salem|Franklin County|KS|68048|United States|-6|apartment| +49322|AAAAAAAAKKAMAAAA|428|Washington |Drive|Suite 90|Concord|Calhoun County|MS|54107|United States|-6|condo| +49323|AAAAAAAALKAMAAAA|75|Oak Laurel|Cir.|Suite G|Frogtown|Fisher County|TX|78784|United States|-6|condo| +49324|AAAAAAAAMKAMAAAA|425|Green |Pkwy|Suite 100|Enterprise|Montour County|PA|11757|United States|-5|single family| +49325|AAAAAAAANKAMAAAA|854|Pine |Ave|Suite S|Georgetown|Haskell County|TX|77057|United States|-6|single family| +49326|AAAAAAAAOKAMAAAA|735|Green |Blvd|Suite 280|Bunker Hill|Kenedy County|TX|70150|United States|-6|apartment| +49327|AAAAAAAAPKAMAAAA|923|Lakeview |Lane|Suite I|Sunnyside|Baker County|FL|31952|United States|-5|condo| +49328|AAAAAAAAALAMAAAA|61|Ninth Johnson|ST|Suite 370|Wildwood|Perry County|IN|46871|United States|-5|condo| +49329|AAAAAAAABLAMAAAA|767|Highland Lincoln|Ct.|Suite 380|Deerfield|Otsego County|NY|19840|United States|-5|single family| +49330|AAAAAAAACLAMAAAA|590|North Ridge|Ln|Suite 200|Clearview|Ramsey County|ND|55495|United States|-6|apartment| +49331|AAAAAAAADLAMAAAA|94|View |Parkway|Suite 250|Stringtown|Collier County|FL|30162|United States|-5|condo| +49332|AAAAAAAAELAMAAAA|817|Chestnut |ST|Suite 450|Mount Olive|Moniteau County|MO|68059|United States|-6|apartment| +49333|AAAAAAAAFLAMAAAA|127|Dogwood Wilson|Boulevard|Suite S|Antioch|Laramie County|WY|88605|United States|-7|condo| +49334|AAAAAAAAGLAMAAAA|434|Sixth |Lane|Suite A|Mount Olive|Johnson County|KY|48059|United States|-6|single family| +49335|AAAAAAAAHLAMAAAA|84|View |Ave|Suite H|Mountain View|Logan County|WV|24466|United States|-5|condo| +49336|AAAAAAAAILAMAAAA|86|7th |Drive|Suite M|Marion|Blaine County|MT|60399|United States|-7|condo| +49337|AAAAAAAAJLAMAAAA|382|4th |Pkwy|Suite S|Jackson|Osceola County|FL|39583|United States|-5|single family| +49338|AAAAAAAAKLAMAAAA|758|Smith |Circle|Suite G|Plainview|Stevens County|KS|63683|United States|-6|condo| +49339|AAAAAAAALLAMAAAA|961|10th |Way|Suite 0|Freeport|DeKalb County|IL|61844|United States|-6|apartment| +49340|AAAAAAAAMLAMAAAA|953|Third |Blvd|Suite M|Woodlawn|Mercer County|NJ|04698|United States|-5|condo| +49341|AAAAAAAANLAMAAAA|332|River Broadway|Boulevard|Suite 280|Fairview|Charlevoix County|MI|45709|United States|-5|apartment| +49342|AAAAAAAAOLAMAAAA|514|Sixth 4th|ST|Suite W|Mount Olive|Douglas County|OR|98059|United States|-8|apartment| +49343|AAAAAAAAPLAMAAAA|937|1st 13th|Avenue|Suite A|Oak Ridge|Harmon County|OK|78371|United States|-6|apartment| +49344|AAAAAAAAAMAMAAAA|286|Ridge |Blvd|Suite 480|Wilson|Adams County|IL|66971|United States|-6|apartment| +49345|AAAAAAAABMAMAAAA|856|Walnut Main|RD|Suite 100|Green Acres|Ottawa County|OH|47683|United States|-5|single family| +49346|AAAAAAAACMAMAAAA|661|Broadway Third|Street|Suite 110|Belmont|Clay County|NE|60191|United States|-6|condo| +49347|AAAAAAAADMAMAAAA|73|Park Spring|Court|Suite S|Willow|Raleigh County|WV|26798|United States|-5|single family| +49348|AAAAAAAAEMAMAAAA|910|14th Lake|RD|Suite 20|Lincoln|Shenandoah County|VA|21289|United States|-5|single family| +49349|AAAAAAAAFMAMAAAA|5|Railroad |Circle|Suite K|Florence|Lake County|SD|53394|United States|-7|single family| +49350|AAAAAAAAGMAMAAAA|164|3rd |Lane|Suite 430|Jackson|Chatham County|GA|39583|United States|-5|apartment| +49351|AAAAAAAAHMAMAAAA|690|3rd Pine|Ave|Suite O|Kingston|Harper County|KS|64975|United States|-6|single family| +49352|AAAAAAAAIMAMAAAA|775|Sycamore 2nd|Blvd|Suite 230|White Oak|Houston County|GA|36668|United States|-5|apartment| +49353|AAAAAAAAJMAMAAAA|675|West Wilson|Lane|Suite 400|New Hope|Foster County|ND|59431|United States|-6|apartment| +49354|AAAAAAAAKMAMAAAA|532|Lake |Blvd|Suite 360|Springdale|Haakon County|SD|58883|United States|-7|apartment| +49355|AAAAAAAALMAMAAAA|223|River |ST|Suite C|Brownsville|Washington County|AR|79310|United States|-6|single family| +49356|AAAAAAAAMMAMAAAA|368|Oak |Ln|Suite D|Friendship|Lehigh County|PA|14536|United States|-5|apartment| +49357|AAAAAAAANMAMAAAA|802|Ash |RD|Suite 310|Lakeview|James City County|VA|28579|United States|-5|single family| +49358|AAAAAAAAOMAMAAAA|793|Walnut East|Circle|Suite S|Bridgeport|Hardee County|FL|35817|United States|-5|single family| +49359|AAAAAAAAPMAMAAAA|732|Hill Forest|Parkway|Suite K|Mount Zion|Miller County|AR|78054|United States|-6|apartment| +49360|AAAAAAAAANAMAAAA|471|College |Road|Suite 0|Five Forks|Logan County|WV|22293|United States|-5|apartment| +49361|AAAAAAAABNAMAAAA|126|Second Park|Parkway|Suite E|Maple Grove|Crittenden County|KY|48252|United States|-6|apartment| +49362|AAAAAAAACNAMAAAA|830|11th |Blvd|Suite 380|Valley View|Providence County|RI|05724|United States|-5|condo| +49363|AAAAAAAADNAMAAAA|869|5th Elm|Dr.|Suite K|Plainview|Dewey County|OK|73683|United States|-6|apartment| +49364|AAAAAAAAENAMAAAA|800|6th Broadway|Dr.|Suite Y|New Town|Mountrail County|ND|59634|United States|-6|single family| +49365|AAAAAAAAFNAMAAAA|848|Forest View|Circle|Suite 230|Pleasant Hill|Brooke County|WV|23604|United States|-5|apartment| +49366|AAAAAAAAGNAMAAAA|575|Hillcrest |Avenue|Suite W|Waterloo|Plymouth County|IA|51675|United States|-6|apartment| +49367|AAAAAAAAHNAMAAAA|457|Main |Ave|Suite G|Sulphur Springs|Forsyth County|GA|38354|United States|-5|apartment| +49368|AAAAAAAAINAMAAAA|188|Second |Dr.|Suite 470|Valley View|Antrim County|MI|45124|United States|-5|condo| +49369|AAAAAAAAJNAMAAAA|917|7th |Boulevard|Suite 190|Kingston|Bedford County|TN|34975|United States|-5|apartment| +49370|AAAAAAAAKNAMAAAA|738|Second Forest|Pkwy|Suite T|Clearwater|Howard County|IN|49534|United States|-5|condo| +49371|AAAAAAAALNAMAAAA|998|1st |Boulevard|Suite 440|Shiloh|Grenada County|MS|59275|United States|-6|single family| +49372|AAAAAAAAMNAMAAAA|887|Lakeview Cedar|Street|Suite 490|Ashland|Virginia Beach city|VA|24244|United States|-5|apartment| +49373|AAAAAAAANNAMAAAA|679|Elm Sycamore|Dr.|Suite 0|Spring Valley|Barton County|KS|66060|United States|-6|single family| +49374|AAAAAAAAONAMAAAA|848|10th |Boulevard|Suite 230|Shady Grove|Marengo County|AL|32812|United States|-6|single family| +49375|AAAAAAAAPNAMAAAA|613|Lake 5th|Cir.|Suite 60|Clinton|Albemarle County|VA|28222|United States|-5|single family| +49376|AAAAAAAAAOAMAAAA|643|Laurel |RD|Suite 270|Murray|Klickitat County|WA|92150|United States|-8|apartment| +49377|AAAAAAAABOAMAAAA|738|10th 6th|Avenue|Suite 270|Oakland|Chowan County|NC|29843|United States|-5|apartment| +49378|AAAAAAAACOAMAAAA|32|8th |Circle|Suite G|Sunnyside|Spink County|SD|51952|United States|-7|single family| +49379|AAAAAAAADOAMAAAA|120|Ninth |Boulevard|Suite 60|Union|Oregon County|MO|68721|United States|-6|single family| +49380|AAAAAAAAEOAMAAAA|952|Smith |Boulevard|Suite T|Sumner|Kings County|CA|90519|United States|-8|condo| +49381|AAAAAAAAFOAMAAAA|247|Maple |Street|Suite E|Mount Pleasant|Simpson County|KY|41933|United States|-5|single family| +49382|AAAAAAAAGOAMAAAA|896|Central Maple|ST|Suite V|Harmony|Cumberland County|IL|65804|United States|-6|apartment| +49383|AAAAAAAAHOAMAAAA|539|First 15th|Drive|Suite A|Lebanon|Kenai Peninsula Borough|AK|92898|United States|-9|condo| +49384|AAAAAAAAIOAMAAAA|968||||Lebanon|||42898|||condo| +49385|AAAAAAAAJOAMAAAA|116|Mill |||Arlington||||United States|-5|| +49386|AAAAAAAAKOAMAAAA|667|Walnut |Cir.|Suite 400|Valley View|Kiowa County|CO|85124|United States|-7|apartment| +49387|AAAAAAAALOAMAAAA|552|3rd |RD|Suite 350|Shiloh|Comanche County|TX|79275|United States|-6|apartment| +49388|AAAAAAAAMOAMAAAA|416|First Tenth|Pkwy|Suite K|Fairview|Clinton County|IA|55709|United States|-6|apartment| +49389|AAAAAAAANOAMAAAA|421|Broadway Sixth|Street|Suite A|Valley View|Miner County|SD|55124|United States|-7|single family| +49390|AAAAAAAAOOAMAAAA|698|1st |Parkway|Suite 10|Crossroads|Polk County|GA|30534|United States|-5|condo| +49391|AAAAAAAAPOAMAAAA|861|2nd |Boulevard|Suite O|Mount Zion|Doddridge County|WV|28054|United States|-5|apartment| +49392|AAAAAAAAAPAMAAAA|608|Central Lake|RD|Suite L|Lakeside|Lee County|GA|39532|United States|-5|apartment| +49393|AAAAAAAABPAMAAAA|531|Dogwood |Parkway|Suite 340|Deerfield|Aiken County|SC|29840|United States|-5|apartment| +49394|AAAAAAAACPAMAAAA|328|River Cedar|Ln|Suite 10|Pine Grove|Dixon County|NE|64593|United States|-6|condo| +49395|AAAAAAAADPAMAAAA|705|4th College|Lane|Suite 120|Glendale|San Luis Obispo County|CA|93951|United States|-8|apartment| +49396|AAAAAAAAEPAMAAAA|496|Cherry |Blvd|Suite M|Waterloo|Queens County|NY|11675|United States|-5|condo| +49397|AAAAAAAAFPAMAAAA|374|Maple 7th|Avenue|Suite S|New Hope|Kerr County|TX|79431|United States|-6|condo| +49398|AAAAAAAAGPAMAAAA|441||Drive|||Bollinger County|MO||United States|-6|| +49399|AAAAAAAAHPAMAAAA|439|Madison First|Street|Suite 250|Macedonia|Cowlitz County|WA|91087|United States|-8|condo| +49400|AAAAAAAAIPAMAAAA|802|Elm |Ln|Suite 360|Union Hill|Broome County|NY|17746|United States|-5|single family| +49401|AAAAAAAAJPAMAAAA|456|Lincoln Lakeview|Circle|Suite T|Jackson|Arapahoe County|CO|89583|United States|-7|single family| +49402|AAAAAAAAKPAMAAAA|984|Sunset |Avenue|Suite K|Newport|Pawnee County|NE|61521|United States|-7|single family| +49403|AAAAAAAALPAMAAAA|683|Second Lee|Court|Suite Y|Shady Grove|Meeker County|MN|52812|United States|-6|condo| +49404|AAAAAAAAMPAMAAAA|129|Walnut Maple|Ct.|Suite 430|Newport|Hamilton County|NY|11521|United States|-5|condo| +49405|AAAAAAAANPAMAAAA|514|10th Oak|Drive|Suite E|Union|Hubbard County|MN|58721|United States|-6|condo| +49406|AAAAAAAAOPAMAAAA|633|12th Second|Ln|Suite 210|Enterprise|Okanogan County|WA|91757|United States|-8|apartment| +49407|AAAAAAAAPPAMAAAA|781|Meadow Washington|Pkwy|Suite 230|Sulphur Springs|Oldham County|KY|48354|United States|-5|single family| +49408|AAAAAAAAAABMAAAA|321|Mill Ridge|Wy|Suite T|Edgewood|Lake County|MN|50069|United States|-6|single family| +49409|AAAAAAAABABMAAAA|330|Madison Fifth|Pkwy|Suite 490|Pleasant Hill|Carlton County|MN|53604|United States|-6|single family| +49410|AAAAAAAACABMAAAA|788|Park Main|Way|Suite R|Oak Hill|Wilson County|KS|67838|United States|-6|apartment| +49411|AAAAAAAADABMAAAA|651|Woodland 12th|Cir.|Suite 0|Hillcrest|Pendleton County|KY|43003|United States|-5|condo| +49412|AAAAAAAAEABMAAAA|596|Jackson Dogwood|Parkway|Suite 270|Pleasant Valley|Pike County|MO|62477|United States|-6|apartment| +49413|AAAAAAAAFABMAAAA|939|South First|Ln|Suite 350|Wilton|Bottineau County|ND|56997|United States|-6|single family| +49414|AAAAAAAAGABMAAAA|719|Poplar First|Circle|Suite 150|Oak Hill|Choctaw County|MS|57838|United States|-6|apartment| +49415|AAAAAAAAHABMAAAA|65|2nd |Boulevard|Suite D|Greenwood|Moore County|TN|38828|United States|-6|apartment| +49416|AAAAAAAAIABMAAAA|169|Lakeview Sixth|Lane|Suite 130|Ashland|Jessamine County|KY|44244|United States|-6|single family| +49417|AAAAAAAAJABMAAAA|34|Davis |ST|Suite L|Florence|Linn County|KS|63394|United States|-6|condo| +49418|AAAAAAAAKABMAAAA|568|Valley |Way|Suite 110|Colonial Heights|Inyo County|CA|93425|United States|-8|condo| +49419|AAAAAAAALABMAAAA|47|Smith Wilson|Way|Suite E|Oakdale|Greene County|IA|59584|United States|-6|condo| +49420|AAAAAAAAMABMAAAA|685|Second |ST|Suite Y|Arlington|Madison County|KY|46557|United States|-5|apartment| +49421|AAAAAAAANABMAAAA|370|River |Drive|Suite 80|Spring Hill|Winkler County|TX|76787|United States|-6|single family| +49422|AAAAAAAAOABMAAAA|170|4th Spring|Parkway|Suite F|Clinton|Elbert County|GA|38222|United States|-5|single family| +49423|AAAAAAAAPABMAAAA|775|Williams 3rd|Avenue|Suite R|Enterprise|Christian County|MO|61757|United States|-6|apartment| +49424|AAAAAAAAABBMAAAA|36|2nd |Avenue|Suite 270|San Jose|Pierce County|GA|38003|United States|-5|condo| +49425|AAAAAAAABBBMAAAA|919|Cedar |Dr.|Suite H|Cedar Grove|Thomas County|NE|60411|United States|-7|apartment| +49426|AAAAAAAACBBMAAAA|258|1st West|Road|Suite 270|Wildwood|Lincoln County|NE|66871|United States|-7|condo| +49427|AAAAAAAADBBMAAAA|||Way|Suite 240|Macedonia|||41087|||| +49428|AAAAAAAAEBBMAAAA|64|First College|Lane|Suite 40|Macedonia|Phillips County|AR|71087|United States|-6|condo| +49429|AAAAAAAAFBBMAAAA|721|Central |ST|Suite 90|Shiloh|Crawford County|MO|69275|United States|-6|condo| +49430|AAAAAAAAGBBMAAAA|417|Jefferson |Pkwy|Suite F|Williamsville|Clay County|AL|38754|United States|-6|single family| +49431|AAAAAAAAHBBMAAAA|489|Mill Jackson|Dr.|Suite 60|Mount Vernon|Buena Vista County|IA|58482|United States|-6|single family| +49432|AAAAAAAAIBBMAAAA|603|3rd Main|Lane|Suite 230|Red Hill|Madison County|TN|34338|United States|-6|single family| +49433|AAAAAAAAJBBMAAAA|938|5th Center|Pkwy|Suite 440|Salem|White County|TN|38048|United States|-6|apartment| +49434|AAAAAAAAKBBMAAAA|231|Davis View|Ln|Suite R|Crossroads|Mason County|KY|40534|United States|-5|single family| +49435|AAAAAAAALBBMAAAA|916|Maple 14th|Way|Suite B|Bridgeport|Henry County|IL|65817|United States|-6|apartment| +49436|AAAAAAAAMBBMAAAA|77|Woodland Center|Ln|Suite M|Midway|Kennebec County|ME|02504|United States|-5|condo| +49437|AAAAAAAANBBMAAAA|633|Main Railroad|RD|Suite 420|Shady Grove|Cocke County|TN|32812|United States|-5|condo| +49438|AAAAAAAAOBBMAAAA|934|Maple Washington|Pkwy|Suite 360|New Hope|Jones County|IA|59431|United States|-6|condo| +49439|AAAAAAAAPBBMAAAA|256|Oak 3rd|Boulevard|Suite M|Highland Park|Deuel County|NE|66534|United States|-6|apartment| +49440|AAAAAAAAACBMAAAA|193|1st Central|Way|Suite C|Pleasant Valley|Fredericksburg city|VA|22477|United States|-5|apartment| +49441|AAAAAAAABCBMAAAA|160|Cherry |Parkway|Suite 280|Oak Hill|Marion County|GA|37838|United States|-5|single family| +49442|AAAAAAAACCBMAAAA|699|Church 6th|Circle|Suite 50|Oakdale|Erie County|PA|19584|United States|-5|single family| +49443|AAAAAAAADCBMAAAA|326|Main Spruce|Dr.|Suite 460|Colonial Heights|Camp County|TX|73425|United States|-6|condo| +49444|AAAAAAAAECBMAAAA|129|Third Franklin|Street|Suite 200|Five Points|Shannon County|MO|66098|United States|-6|apartment| +49445|AAAAAAAAFCBMAAAA|501|5th Fourth|Ave|Suite B|Five Forks|Pope County|MN|52293|United States|-6|condo| +49446|AAAAAAAAGCBMAAAA|337|South |Lane|Suite F|Centerville|Marion County|IL|60059|United States|-6|single family| +49447|AAAAAAAAHCBMAAAA|361|Green Forest|Cir.|Suite L|Franklin|Alfalfa County|OK|79101|United States|-6|apartment| +49448|AAAAAAAAICBMAAAA|115|Dogwood |Way|Suite 400|Kelly|Grant County|OR|92738|United States|-8|condo| +49449|AAAAAAAAJCBMAAAA|528|Broadway |Boulevard|Suite I|Providence|Harrisonburg city|VA|26614|United States|-5|apartment| +49450|AAAAAAAAKCBMAAAA|291|Hill |Cir.|Suite 20|Enterprise|Morgan County|GA|31757|United States|-5|apartment| +49451|AAAAAAAALCBMAAAA|513|10th Church|Dr.|Suite 170|Maple Grove|Colquitt County|GA|38252|United States|-5|condo| +49452|AAAAAAAAMCBMAAAA|907|Mill |Drive|Suite I|Harmony|Motley County|TX|75804|United States|-6|single family| +49453|AAAAAAAANCBMAAAA|722|Pine 2nd|Avenue|Suite 380|Midway|Phillips County|AR|71904|United States|-6|condo| +49454|AAAAAAAAOCBMAAAA|550|Oak 6th|Circle|Suite 190|Roy|Breckinridge County|KY|40744|United States|-6|condo| +49455|AAAAAAAAPCBMAAAA|331|Walnut |Dr.|Suite U|Saratoga|Houghton County|MI|42123|United States|-5|apartment| +49456|AAAAAAAAADBMAAAA||Hill View|RD|Suite 160|||UT||United States|-7|condo| +49457|AAAAAAAABDBMAAAA|53|Forest East|Lane|Suite 430|Riverside|Henry County|IN|49231|United States|-5|single family| +49458|AAAAAAAACDBMAAAA|202|East Wilson|Street|Suite 110|Springdale|Warren County|GA|38883|United States|-5|single family| +49459|AAAAAAAADDBMAAAA|360|1st |Street|Suite E|Deerfield|Reeves County|TX|79840|United States|-6|condo| +49460|AAAAAAAAEDBMAAAA|528|Valley |ST|Suite 270|Greenville|Clarke County|VA|21387|United States|-5|single family| +49461|AAAAAAAAFDBMAAAA|461|Valley |Wy|Suite 120|White Oak|Shannon County|MO|66668|United States|-6|condo| +49462|AAAAAAAAGDBMAAAA|886|Washington 1st|Dr.|Suite H|Pomona|Lewis County|WV|24153|United States|-5|condo| +49463|AAAAAAAAHDBMAAAA|331|Ridge 3rd|Ct.|Suite D|Kelly|Wayne County|PA|12738|United States|-5|single family| +49464|AAAAAAAAIDBMAAAA||2nd View||Suite D||Nacogdoches County|TX|76697|United States||single family| +49465|AAAAAAAAJDBMAAAA|34|Seventh Lake|Blvd|Suite 410|Springdale|Big Horn County|WY|88883|United States|-7|single family| +49466|AAAAAAAAKDBMAAAA|408||Cir.|||Colleton County|||United States||| +49467|AAAAAAAALDBMAAAA|316|Cedar |Dr.|Suite 120|Springhill|Glacier County|MT|64602|United States|-7|single family| +49468|AAAAAAAAMDBMAAAA|951|Walnut |Cir.|Suite 250|Cedar Grove|Warren County|IA|50411|United States|-6|single family| +49469|AAAAAAAANDBMAAAA|98|Oak |Road|Suite 370|Jamestown|Bibb County|GA|36867|United States|-5|condo| +49470|AAAAAAAAODBMAAAA|762|Park First|Road|Suite 270|Raymond|Union County|NJ|00875|United States|-5|apartment| +49471|AAAAAAAAPDBMAAAA|72|Locust |Street|Suite J|Florence|Van Buren County|AR|73394|United States|-6|condo| +49472|AAAAAAAAAEBMAAAA|128|Madison |Blvd|Suite 450|Pleasant Hill|Salem city|VA|23604|United States|-5|apartment| +49473|AAAAAAAABEBMAAAA|379|College |Pkwy|Suite U|Hamilton|Sedgwick County|CO|82808|United States|-7|apartment| +49474|AAAAAAAACEBMAAAA|403|2nd |Lane|Suite C|Jamestown|Leavenworth County|KS|66867|United States|-6|apartment| +49475|AAAAAAAADEBMAAAA|495|Dogwood |Way|Suite X|Liberty|Rusk County|TX|73451|United States|-6|condo| +49476|AAAAAAAAEEBMAAAA|812|Hill |Blvd|Suite W|Woodville|Lowndes County|AL|34289|United States|-6|condo| +49477|AAAAAAAAFEBMAAAA|949|6th Jefferson|Dr.|Suite X|Centerville|Bartholomew County|IN|40059|United States|-5|single family| +49478|AAAAAAAAGEBMAAAA|277|Spring |Avenue|Suite 370|Rocky Point|Alcorn County|MS|51209|United States|-6|apartment| +49479|AAAAAAAAHEBMAAAA|707|Sunset |Pkwy|Suite 30|Pomona|Lyon County|KY|44153|United States|-5|single family| +49480|AAAAAAAAIEBMAAAA|825|12th |Road|Suite 260|Oakdale|Union County|OH|49584|United States|-5|single family| +49481|AAAAAAAAJEBMAAAA|1|Lake |Ct.|Suite V|Sunnyside|Jasper County|SC|21952|United States|-5|single family| +49482|AAAAAAAAKEBMAAAA|711|Park |RD|Suite U|Owens|Caddo Parish|LA|72324|United States|-6|condo| +49483|AAAAAAAALEBMAAAA||||||||||-6|single family| +49484|AAAAAAAAMEBMAAAA|426|12th Cherry|Pkwy|Suite 470|Salem|Custer County|MT|68048|United States|-7|apartment| +49485|AAAAAAAANEBMAAAA||Green 6th||Suite 370||Jackson County|OR|90534|United States||| +49486|AAAAAAAAOEBMAAAA|650|Spring |Street|Suite 400|Plainview|Adams County|IL|63683|United States|-6|single family| +49487|AAAAAAAAPEBMAAAA|476|Lee |Street|Suite T|Red Hill|Garland County|AR|74338|United States|-6|apartment| +49488|AAAAAAAAAFBMAAAA||Broadway Jefferson||Suite X|||||||| +49489|AAAAAAAABFBMAAAA|779|Park |Street|Suite 210|Fairfield|Henry County|AL|36192|United States|-6|apartment| +49490|AAAAAAAACFBMAAAA|258|Pine Fourth|Drive|Suite J|Milo|Chesterfield County|SC|20116|United States|-5|apartment| +49491|AAAAAAAADFBMAAAA|75|10th |Circle|Suite C|Altamont|Clark County|MO|69387|United States|-6|apartment| +49492|AAAAAAAAEFBMAAAA|633|6th |Ct.|Suite 130|Fairview|Stone County|MS|55709|United States|-6|condo| +49493|AAAAAAAAFFBMAAAA|766|11th |Ave|Suite 10|Bunker Hill|Deuel County|SD|50150|United States|-6|condo| +49494|AAAAAAAAGFBMAAAA|978|East 8th|Court|Suite B|Forest Hills|Fulton County|AR|79237|United States|-6|apartment| +49495|AAAAAAAAHFBMAAAA|306|Mill Smith||||Runnels County|TX|78877|||| +49496|AAAAAAAAIFBMAAAA|471|Ridge Green|Ct.|Suite 350|Waterloo|Cumberland County|KY|41675|United States|-6|single family| +49497|AAAAAAAAJFBMAAAA|352|First 15th|Wy|Suite 150|Oakwood|Baldwin County|AL|30169|United States|-6|apartment| +49498|AAAAAAAAKFBMAAAA|117|14th |Ln|Suite K|Doyle|Cowley County|KS|68434|United States|-6|condo| +49499|AAAAAAAALFBMAAAA|223|Johnson Davis|ST|Suite 420|Oak Ridge|Franklin County|OH|48371|United States|-5|condo| +49500|AAAAAAAAMFBMAAAA|788|Williams |Cir.|Suite 190|Union|Wilson County|NC|28721|United States|-5|condo| +49501|AAAAAAAANFBMAAAA|369|North |Ln|Suite 280|Edgewood|Mecklenburg County|NC|20069|United States|-5|single family| +49502|AAAAAAAAOFBMAAAA|908|5th |ST|Suite R|Wildwood|Nevada County|AR|76871|United States|-6|single family| +49503|AAAAAAAAPFBMAAAA|719|Valley Walnut|Ct.|Suite U|Ludlow|Attala County|MS|55566|United States|-6|apartment| +49504|AAAAAAAAAGBMAAAA|996|Lee |Ave|Suite O|Calhoun|Muskogee County|OK|76909|United States|-6|condo| +49505|AAAAAAAABGBMAAAA|173|6th Lee|RD|Suite I|Mount Zion|Prince Edward County|VA|28054|United States|-5|apartment| +49506|AAAAAAAACGBMAAAA|231|East |Drive|Suite 300|Spring Hill|Yuba County|CA|96787|United States|-8|condo| +49507|AAAAAAAADGBMAAAA|557|Ridge View|Blvd|Suite 40|Spring Hill|Sullivan County|NH|07387|United States|-5|condo| +49508|AAAAAAAAEGBMAAAA|972|Center College|Street|Suite E|Spring Hill|Crisp County|GA|36787|United States|-5|condo| +49509|AAAAAAAAFGBMAAAA|11|7th Dogwood|Pkwy|Suite 0|Mount Pleasant|New Hanover County|NC|21933|United States|-5|single family| +49510|AAAAAAAAGGBMAAAA|915|Miller 2nd|RD|Suite 70|Deerfield|Talladega County|AL|39840|United States|-6|apartment| +49511|AAAAAAAAHGBMAAAA|989|Spruce |Road|Suite 110|Edgewood|Corson County|SD|50069|United States|-6|condo| +49512|AAAAAAAAIGBMAAAA|257|North Poplar|Boulevard|Suite 300|Arlington|Garfield County|OK|76557|United States|-6|condo| +49513|AAAAAAAAJGBMAAAA||Chestnut |||Five Forks||MS||||apartment| +49514|AAAAAAAAKGBMAAAA|717|Park |Ave|Suite 0|Oneida|Hempstead County|AR|74027|United States|-6|condo| +49515|AAAAAAAALGBMAAAA|205|Fifth |Drive|Suite D|Riverview|Sanilac County|MI|49003|United States|-5|single family| +49516|AAAAAAAAMGBMAAAA|697|Walnut |Boulevard|Suite E|Newport|Coryell County|TX|71521|United States|-6|apartment| +49517|AAAAAAAANGBMAAAA|663|Third 6th|ST|Suite 470|Georgetown|Jefferson County|IA|57057|United States|-6|apartment| +49518|AAAAAAAAOGBMAAAA|653|Willow Johnson|Street|Suite C|Pleasant Valley|Slope County|ND|52477|United States|-6|apartment| +49519|AAAAAAAAPGBMAAAA|931|View Spring|Boulevard|Suite 270|Oakwood|Jeff Davis County|GA|30169|United States|-5|single family| +49520|AAAAAAAAAHBMAAAA|770|Maple |ST|Suite 220|Oak Hill|Scott County|KS|67838|United States|-6|apartment| +49521|AAAAAAAABHBMAAAA|519|East |Lane|Suite R|Ashley|Wabash County|IN|44324|United States|-5|apartment| +49522|AAAAAAAACHBMAAAA|153|North |Road|Suite 90|Marion|Lincoln County|NV|80399|United States|-8|apartment| +49523|AAAAAAAADHBMAAAA|111|7th |Dr.|Suite 70|Green Acres|Grafton County|NH|08283|United States|-5|apartment| +49524|AAAAAAAAEHBMAAAA|717|Maple Johnson|Court|Suite 180|Greenwood|Horry County|SC|28828|United States|-5|apartment| +49525|AAAAAAAAFHBMAAAA|737|13th View|Circle|Suite 490|Pleasant Grove|McHenry County|ND|54136|United States|-6|single family| +49526|AAAAAAAAGHBMAAAA|409|Jackson Poplar|ST|Suite 380|Forest Hills|Lee County|IL|69237|United States|-6|single family| +49527|AAAAAAAAHHBMAAAA|595|2nd Miller|RD|Suite 410|Red Hill|Fairfax city|VA|24338|United States|-5|condo| +49528|AAAAAAAAIHBMAAAA|59|Birch South|Cir.|Suite N|Red Hill|Santa Barbara County|CA|94338|United States|-8|apartment| +49529|AAAAAAAAJHBMAAAA|712|Johnson Sunset|Ln|Suite O|Enterprise|Mifflin County|PA|11757|United States|-5|apartment| +49530|AAAAAAAAKHBMAAAA||8th |Blvd||Shiloh|La Salle County||||-6|| +49531|AAAAAAAALHBMAAAA|106|4th |Boulevard|Suite E|Glendale|Fayette County|IL|63951|United States|-6|single family| +49532|AAAAAAAAMHBMAAAA|257|Lincoln Walnut|Blvd|Suite R|Five Points|Cass County|IA|56098|United States|-6|condo| +49533|AAAAAAAANHBMAAAA|906|Twelfth Valley|Ln|Suite 390|Riverview|Martin County|FL|39003|United States|-5|single family| +49534|AAAAAAAAOHBMAAAA|885|First Lee|Drive|Suite I|Green Acres|Franklin County|MS|57683|United States|-6|single family| +49535|AAAAAAAAPHBMAAAA|710|Cedar West|Wy|Suite E|Oakwood|Wayne County|MO|60169|United States|-6|apartment| +49536|AAAAAAAAAIBMAAAA|277|Hickory |Lane|Suite 300|New Hope|Marshall County|KS|69431|United States|-6|single family| +49537|AAAAAAAABIBMAAAA|995|Hill |Street|Suite 290|Green Acres|Rains County|TX|77683|United States|-6|apartment| +49538|AAAAAAAACIBMAAAA|80|11th |Street|Suite R|Riverview|Fremont County|CO|89003|United States|-7|condo| +49539|AAAAAAAADIBMAAAA|268||Street|Suite 470||||06404|United States|-5|| +49540|AAAAAAAAEIBMAAAA|339|Sycamore |Circle|Suite W|Scottsville|Pushmataha County|OK|74190|United States|-6|single family| +49541|AAAAAAAAFIBMAAAA|710|||Suite 370||Thomas County||32293|United States|-5|condo| +49542|AAAAAAAAGIBMAAAA|186|View 13th|Cir.|Suite 310|Ashland|Montgomery County|IN|44244|United States|-5|apartment| +49543|AAAAAAAAHIBMAAAA|508|Park Main|Ct.|Suite B|Valley View|Clark County|KY|45124|United States|-6|condo| +49544|AAAAAAAAIIBMAAAA|910|Hickory ||Suite 490|Red Hill||||United States||| +49545|AAAAAAAAJIBMAAAA|974|Sunset Wilson|Ct.|Suite 120|Union Hill|Union County|NC|27746|United States|-5|condo| +49546|AAAAAAAAKIBMAAAA|337|6th |Cir.|Suite P|Wilson|Torrance County|NM|86971|United States|-7|apartment| +49547|AAAAAAAALIBMAAAA|816|6th West|Street|Suite G|Freeport|Crenshaw County|AL|31844|United States|-6|single family| +49548|AAAAAAAAMIBMAAAA|758|College North|Lane|Suite V|Fairfield|White Pine County|NV|86192|United States|-8|single family| +49549|AAAAAAAANIBMAAAA|355|Center Laurel|RD|Suite 210|Five Forks|Whitley County|KY|42293|United States|-5|single family| +49550|AAAAAAAAOIBMAAAA|312|Lincoln 9th|Boulevard|Suite X|Hillcrest|Butte County|ID|83003|United States|-7|single family| +49551|AAAAAAAAPIBMAAAA|385|Third Johnson|Lane|Suite X|Lincoln|Gladwin County|MI|41289|United States|-5|condo| +49552|AAAAAAAAAJBMAAAA|769|6th |Street|Suite O|Kingston|Wichita County|TX|74975|United States|-6|apartment| +49553|AAAAAAAABJBMAAAA|525|Cedar |Circle|Suite 240|Summit|McCormick County|SC|20499|United States|-5|apartment| +49554|AAAAAAAACJBMAAAA|843|Williams |Way|Suite S|Newtown|Garfield County|OK|71749|United States|-6|apartment| +49555|AAAAAAAADJBMAAAA|824|Adams |Ave|Suite 370|Maple Grove|Beaver County|PA|18252|United States|-5|condo| +49556|AAAAAAAAEJBMAAAA|854|Woodland |Blvd|Suite M|Hartland|Amite County|MS|56594|United States|-6|condo| +49557|AAAAAAAAFJBMAAAA|174|13th Miller|Parkway|Suite H|Clinton|Nolan County|TX|78222|United States|-6|single family| +49558|AAAAAAAAGJBMAAAA|||Ct.|Suite V|Clifton|Winchester city||28014|United States|-5|apartment| +49559|AAAAAAAAHJBMAAAA|692|Oak |Wy|Suite A|Somerville|Amador County|CA|97783|United States|-8|condo| +49560|AAAAAAAAIJBMAAAA|446|2nd |Road|Suite T|Deerfield|Sherman County|KS|69840|United States|-6|apartment| +49561|AAAAAAAAJJBMAAAA|581|Main Hill|Cir.|Suite W|Pine Grove|Hertford County|NC|24593|United States|-5|single family| +49562|AAAAAAAAKJBMAAAA|805|7th Hill|Drive|Suite C|Mount Zion|Branch County|MI|48054|United States|-5|apartment| +49563|AAAAAAAALJBMAAAA|884|Locust Hill|Pkwy|Suite 250|Lenox|Terry County|TX|71143|United States|-6|condo| +49564|AAAAAAAAMJBMAAAA|805|Johnson |Ln|Suite 390|Bayside|Stillwater County|MT|69550|United States|-7|condo| +49565|AAAAAAAANJBMAAAA|79|Cherry |Court|Suite B|Fairfield|Custer County|NE|66192|United States|-6|single family| +49566|AAAAAAAAOJBMAAAA||Valley ||Suite H|Riverdale||LA|79391|||| +49567|AAAAAAAAPJBMAAAA|252|2nd Jackson|Court|Suite S|Union|Breckinridge County|KY|48721|United States|-6|apartment| +49568|AAAAAAAAAKBMAAAA|116|Pine Davis|RD|Suite 290|Waterloo|Wayne County|OH|41675|United States|-5|single family| +49569|AAAAAAAABKBMAAAA|272|Sunset |Boulevard|Suite C|Oak Ridge|Montgomery County|IL|68371|United States|-6|apartment| +49570|AAAAAAAACKBMAAAA|957|Lake Wilson|Dr.|Suite 260|Wilson|Nye County|NV|86971|United States|-8|single family| +49571|AAAAAAAADKBMAAAA|626|Railroad |Road|Suite D|Pine Grove|Tunica County|MS|54593|United States|-6|single family| +49572|AAAAAAAAEKBMAAAA|228|7th |Parkway|Suite 430|Clinton|Oldham County|TX|78222|United States|-6|single family| +49573|AAAAAAAAFKBMAAAA|616|Eigth Walnut|Road|Suite V|Gladstone|Chambers County|AL|30894|United States|-6|single family| +49574|AAAAAAAAGKBMAAAA|611|Central |Pkwy|Suite 250|Five Forks|Pickens County|GA|32293|United States|-5|condo| +49575|AAAAAAAAHKBMAAAA|98|Johnson |Street|Suite 160|Friendship|Goodhue County|MN|54536|United States|-6|single family| +49576|AAAAAAAAIKBMAAAA|251|East 7th|Court|Suite 180|Woodlawn|Teller County|CO|84098|United States|-7|single family| +49577|AAAAAAAAJKBMAAAA|460|11th Lakeview|Parkway|Suite 390|Buena Vista|Nottoway County|VA|25752|United States|-5|condo| +49578|AAAAAAAAKKBMAAAA|468|Railroad |Court|Suite 50|Harmony|Bergen County|NJ|06404|United States|-5|condo| +49579|AAAAAAAALKBMAAAA||1st Washington||||Vanderburgh County|IN|40216|United States||apartment| +49580|AAAAAAAAMKBMAAAA|448|13th South|RD|Suite J|Pleasant Valley|Appanoose County|IA|52477|United States|-6|apartment| +49581|AAAAAAAANKBMAAAA|715|Madison Spring|Blvd|Suite 250|Sunnyside|Logan County|KY|41952|United States|-5|apartment| +49582|AAAAAAAAOKBMAAAA|853||RD|||Morrow County|OR|98721|United States|-8|condo| +49583|AAAAAAAAPKBMAAAA|823|7th Poplar|Wy|Suite 480|Pleasant Valley|Blue Earth County|MN|52477|United States|-6|condo| +49584|AAAAAAAAALBMAAAA|683|7th |Dr.|Suite F|Crossroads|Alcona County|MI|40534|United States|-5|apartment| +49585|AAAAAAAABLBMAAAA|769|7th |Pkwy|Suite F|Ashland|McCone County|MT|64244|United States|-7|apartment| +49586|AAAAAAAACLBMAAAA|40|5th Fifth|Avenue|Suite A|Cedar Grove|Seward County|NE|60411|United States|-7|apartment| +49587|AAAAAAAADLBMAAAA|965|Jackson Highland|ST|Suite N|Enterprise|Jefferson County|IA|51757|United States|-6|single family| +49588|AAAAAAAAELBMAAAA|547|11th Eigth|ST|Suite U|Macedonia|Fayette County|KY|41087|United States|-6|single family| +49589|AAAAAAAAFLBMAAAA|899|5th Wilson|Ct.|Suite K|Lakeside|Lincoln County|TN|39532|United States|-6|single family| +49590|AAAAAAAAGLBMAAAA|289|3rd |Avenue|Suite 0|Valley View|Iowa County|WI|55124|United States|-6|apartment| +49591|AAAAAAAAHLBMAAAA|190|Laurel 7th|Wy|Suite L|Florence|Winneshiek County|IA|53394|United States|-6|condo| +49592|AAAAAAAAILBMAAAA|555|11th |Way|Suite 440|Greenfield|Lea County|NM|85038|United States|-7|condo| +49593|AAAAAAAAJLBMAAAA|683|Dogwood |Avenue|Suite G|Clinton|Bedford city|VA|28222|United States|-5|condo| +49594|AAAAAAAAKLBMAAAA|754|Church Mill|Ave|Suite 260|Mount Pleasant|Will County|IL|61933|United States|-6|apartment| +49595|AAAAAAAALLBMAAAA|686|Davis Central|Wy|Suite 180|Omega|Lake County|SD|56364|United States|-7|condo| +49596|AAAAAAAAMLBMAAAA|817|Spring |Street|Suite 80|Pine Grove|Vilas County|WI|54593|United States|-6|condo| +49597|AAAAAAAANLBMAAAA|686|Washington Third|Dr.|Suite W|Mountain View|Orange County|NY|14466|United States|-5|single family| +49598|AAAAAAAAOLBMAAAA|548|2nd |Way|Suite P|Unionville|Ray County|MO|61711|United States|-6|single family| +49599|AAAAAAAAPLBMAAAA|819|East 4th|Blvd|Suite A|Belleview|Wayne County|NC|20492|United States|-5|single family| +49600|AAAAAAAAAMBMAAAA|320|Park |Pkwy|Suite N|Ellsworth|Del Norte County|CA|95079|United States|-8|condo| +49601|AAAAAAAABMBMAAAA|544|Lincoln East|Street|Suite N|New Hope|Putnam County|IN|49431|United States|-5|apartment| +49602|AAAAAAAACMBMAAAA|453|1st 10th|Drive|Suite 310|Maple Grove|Klamath County|OR|98252|United States|-8|single family| +49603|AAAAAAAADMBMAAAA|740|15th |Drive|Suite 110|Valley View|Howard County|AR|75124|United States|-6|apartment| +49604|AAAAAAAAEMBMAAAA|710|Pine |RD|Suite 400|Pleasant Hill|Ouachita Parish|LA|73604|United States|-6|apartment| +49605|AAAAAAAAFMBMAAAA|636|Lincoln |Lane|Suite E|Concord|Johnson County|NE|64107|United States|-7|single family| +49606|AAAAAAAAGMBMAAAA|44|Washington 3rd|Circle|Suite N|Highland Park|Umatilla County|OR|96534|United States|-8|condo| +49607|AAAAAAAAHMBMAAAA|88|Hickory |Avenue|Suite W|Kingston|Quitman County|GA|34975|United States|-5|single family| +49608|AAAAAAAAIMBMAAAA|258|Jackson Tenth|Way|Suite T|Five Forks|Osceola County|MI|42293|United States|-5|single family| +49609|AAAAAAAAJMBMAAAA|148|Meadow 12th|Pkwy|Suite 330|Clinton|Lamar County|MS|58222|United States|-6|condo| +49610|AAAAAAAAKMBMAAAA|722|Ash 1st|Blvd|Suite 130|Greenfield|Wibaux County|MT|65038|United States|-7|single family| +49611|AAAAAAAALMBMAAAA|462|2nd 1st|Lane|Suite K|Stratford|Lee County|FL|36668|United States|-5|condo| +49612|AAAAAAAAMMBMAAAA|34|Main |Lane|Suite U|Spring Valley|Jackson County|NC|26060|United States|-5|condo| +49613|AAAAAAAANMBMAAAA|779|Lincoln |ST|Suite V|Lakeside|Marshall County|IL|69532|United States|-6|single family| +49614|AAAAAAAAOMBMAAAA|253|Railroad |ST|Suite 0|Oakwood|Gentry County|MO|60169|United States|-6|apartment| +49615|AAAAAAAAPMBMAAAA|553|Second Hickory|Ln|Suite C|New Hope|Peach County|GA|39431|United States|-5|single family| +49616|AAAAAAAAANBMAAAA|491|Willow |Road|Suite 50|Mountain View|Hardin County|KY|44466|United States|-6|condo| +49617|AAAAAAAABNBMAAAA|813|Davis |Ct.|Suite 160|Fairview|Calhoun County|TX|75709|United States|-6|single family| +49618|AAAAAAAACNBMAAAA|266|Cedar Railroad|ST|Suite M|Deerfield|Uintah County|UT|89840|United States|-7|single family| +49619|AAAAAAAADNBMAAAA|274|Lincoln 8th|Pkwy|Suite 30|Cedar Grove|Miner County|SD|50411|United States|-7|condo| +49620|AAAAAAAAENBMAAAA|215|Woodland Fourth|Avenue|Suite G|Walnut Grove|Greene County|GA|37752|United States|-5|single family| +49621|AAAAAAAAFNBMAAAA|516|Valley Cherry|Road|Suite V|Pleasant Grove|De Kalb County|IN|44136|United States|-5|condo| +49622|AAAAAAAAGNBMAAAA|144|Cherry 14th|Road|Suite 300|Bunker Hill|Gooding County|ID|80150|United States|-7|condo| +49623|AAAAAAAAHNBMAAAA|917|Poplar Ridge|Drive|Suite 350|Harmony|Douglas County|NE|65804|United States|-6|condo| +49624|AAAAAAAAINBMAAAA|750|Wilson 11th|Blvd|Suite 290|Highland|Monroe County|NY|19454|United States|-5|condo| +49625|AAAAAAAAJNBMAAAA|685|Maple Birch|Pkwy|Suite 110|Oak Ridge|Waupaca County|WI|58371|United States|-6|condo| +49626|AAAAAAAAKNBMAAAA|969|14th 12th|Pkwy|Suite 410|Wildwood|Clarion County|PA|16871|United States|-5|condo| +49627|AAAAAAAALNBMAAAA|423|Highland 2nd|Ln|Suite 110|Kingston|Dawson County|TX|74975|United States|-6|single family| +49628|AAAAAAAAMNBMAAAA|840|South |Parkway|Suite G|Five Points|Tangipahoa Parish|LA|76098|United States|-6|condo| +49629|AAAAAAAANNBMAAAA|687|Second |Dr.|Suite K|Glendale|Polk County|MO|63951|United States|-6|condo| +49630|AAAAAAAAONBMAAAA|811|9th Laurel|Boulevard|Suite I|Lebanon|Bladen County|NC|22898|United States|-5|single family| +49631|AAAAAAAAPNBMAAAA|793|Willow |Ct.|Suite 10|Belmont|Cumberland County|VA|20191|United States|-5|condo| +49632|AAAAAAAAAOBMAAAA|106||||Union|McKinley County|NM|88721|United States|-7|apartment| +49633|AAAAAAAABOBMAAAA|930|West Seventh|Street|Suite 380|Webb|Lewis County|TN|30899|United States|-6|condo| +49634|AAAAAAAACOBMAAAA|56|7th North|Blvd|Suite 10|Springdale|Newberry County|SC|28883|United States|-5|apartment| +49635|AAAAAAAADOBMAAAA|208|Hill |Cir.|Suite 130|Union|Trousdale County|TN|38721|United States|-6|apartment| +49636|AAAAAAAAEOBMAAAA|677|Jackson 3rd|Drive|Suite 340|Arlington|Rains County|TX|76557|United States|-6|condo| +49637|AAAAAAAAFOBMAAAA|93|Lake |Wy|Suite M|Newtown|Hansford County|TX|71749|United States|-6|condo| +49638|AAAAAAAAGOBMAAAA|767|Chestnut |Parkway|Suite 440|Pleasant Valley|Palo Alto County|IA|52477|United States|-6|apartment| +49639|AAAAAAAAHOBMAAAA|32|1st Maple|RD|Suite 390|Enterprise|Murray County|OK|71757|United States|-6|condo| +49640|AAAAAAAAIOBMAAAA|668|Davis |Dr.|Suite 190|Lakeville|Louisa County|VA|28811|United States|-5|condo| +49641|AAAAAAAAJOBMAAAA|240|Hill |Dr.|Suite U|Jackson|Union County|NM|89583|United States|-7|condo| +49642|AAAAAAAAKOBMAAAA|608|Wilson Lincoln|Circle|Suite K|Mount Zion|Clallam County|WA|98054|United States|-8|single family| +49643|AAAAAAAALOBMAAAA|297|Mill |Way|Suite Q|Clinton|Patrick County|VA|28222|United States|-5|condo| +49644|AAAAAAAAMOBMAAAA|932|Pine South|Circle|Suite S|Sulphur Springs|Fleming County|KY|48354|United States|-6|single family| +49645|AAAAAAAANOBMAAAA|731|2nd Birch|Ln|Suite R|Ashland|Lake County|CA|94244|United States|-8|single family| +49646|AAAAAAAAOOBMAAAA|952|Third Mill|ST|Suite 280|Liberty|Van Buren County|AR|73451|United States|-6|apartment| +49647|AAAAAAAAPOBMAAAA|302|Sycamore |Cir.|Suite I|Hopewell|Blair County|PA|10587|United States|-5|apartment| +49648|AAAAAAAAAPBMAAAA|2|Woodland |Parkway|Suite G|Pleasant Hill|Muscogee County|GA|33604|United States|-5|condo| +49649|AAAAAAAABPBMAAAA|150|Cherry River|Drive|Suite F|Lakeview|Iredell County|NC|28579|United States|-5|condo| +49650|AAAAAAAACPBMAAAA|415|Pine Oak|Lane|Suite O|Weldon|Jefferson County|ID|86277|United States|-7|single family| +49651|AAAAAAAADPBMAAAA|933|Main |Road|Suite B|Spring Valley|Archuleta County|CO|86060|United States|-7|single family| +49652|AAAAAAAAEPBMAAAA|866|Main View|ST|Suite B|Valley View|Dunn County|ND|55124|United States|-6|condo| +49653|AAAAAAAAFPBMAAAA|536|Forest |Way|Suite 130|Five Forks|Dillon County|SC|22293|United States|-5|apartment| +49654|AAAAAAAAGPBMAAAA|168|Poplar |Pkwy|Suite 320|Oakland|Fulton County|OH|49843|United States|-5|apartment| +49655|AAAAAAAAHPBMAAAA|500|Davis |Court|Suite 270|Waterloo|Goshen County|WY|81675|United States|-7|apartment| +49656|AAAAAAAAIPBMAAAA|312|3rd |Parkway|Suite 0|Pleasant Valley|Galveston County|TX|72477|United States|-6|single family| +49657|AAAAAAAAJPBMAAAA|591|Main |Ln|Suite S|Arlington|Petroleum County|MT|66557|United States|-7|apartment| +49658|AAAAAAAAKPBMAAAA|136|1st |Ln|Suite A|Pleasant Valley|Fulton County|NY|12477|United States|-5|apartment| +49659|AAAAAAAALPBMAAAA|270|Maple |Avenue|Suite 430|Spring Hill|Garden County|NE|66787|United States|-6|condo| +49660|AAAAAAAAMPBMAAAA|257|Valley Poplar|Ct.|Suite P|Enterprise|Cottle County|TX|71757|United States|-6|single family| +49661|AAAAAAAANPBMAAAA|245|View |Street|Suite P|Bunker Hill|Kings County|NY|10150|United States|-5|single family| +49662|AAAAAAAAOPBMAAAA|77|Franklin Highland|Ave|Suite 460|Clinton|Lake County|SD|58222|United States|-7|apartment| +49663|AAAAAAAAPPBMAAAA|450|South 3rd|Ct.|Suite 180|Mount Pleasant|Kings County|NY|11933|United States|-5|apartment| +49664|AAAAAAAAAACMAAAA|377|Walnut |Ave|Suite F|Glendale|Angelina County|TX|73951|United States|-6|apartment| +49665|AAAAAAAABACMAAAA|753|Green |Blvd|Suite 340|Sunnyside|Carlisle County|KY|41952|United States|-6|single family| +49666|AAAAAAAACACMAAAA|93|Center 6th|Street|Suite 60|Belmont|Monroe County|IA|50191|United States|-6|single family| +49667|AAAAAAAADACMAAAA|511|Willow |Pkwy|Suite 220|Barnes|Kern County|CA|93788|United States|-8|single family| +49668|AAAAAAAAEACMAAAA|709|3rd |Ln|Suite 480|Greenfield|Hanover County|VA|25038|United States|-5|condo| +49669|AAAAAAAAFACMAAAA|572|11th |Ave|Suite V|Lakeside|Martinsville city|VA|29532|United States|-5|single family| +49670|AAAAAAAAGACMAAAA|4|Cherry |Road|Suite 40|Midway|Leslie County|KY|41904|United States|-5|single family| +49671|AAAAAAAAHACMAAAA|934|Maple South|Avenue|Suite 410|Springdale|Iron County|MI|48883|United States|-5|apartment| +49672|AAAAAAAAIACMAAAA|734|West Pine|Lane|Suite 490|Guthrie|Essex County|NY|11423|United States|-5|condo| +49673|AAAAAAAAJACMAAAA|784|Locust |Circle|Suite 360|Macedonia|Bibb County|AL|31087|United States|-6|apartment| +49674|AAAAAAAAKACMAAAA|938|Fifth |Road|Suite 120|Franklin|Johnson County|MO|69101|United States|-6|condo| +49675|AAAAAAAALACMAAAA|276|Maple |Avenue|Suite 170|Jackson|Dillon County|SC|29583|United States|-5|single family| +49676|AAAAAAAAMACMAAAA|229|Second Valley|Drive|Suite 170|Woodland|Valley County|NE|64854|United States|-7|apartment| +49677|AAAAAAAANACMAAAA|495|Hill Lincoln|Lane|Suite S|Flint|Douglas County|CO|88909|United States|-7|apartment| +49678|AAAAAAAAOACMAAAA|470|1st |Drive|Suite 200|Pine Valley|Benton County|IN|48209|United States|-5|single family| +49679|AAAAAAAAPACMAAAA|78|Maple |RD|Suite 170|Macedonia|Fort Bend County|TX|71087|United States|-6|single family| +49680|AAAAAAAAABCMAAAA|806|Adams 4th|ST|Suite V|Florence|Kennebec County|ME|03994|United States|-5|apartment| +49681|AAAAAAAABBCMAAAA|677|9th Fourth|Blvd|Suite 340|Florence|Piute County|UT|83394|United States|-7|condo| +49682|AAAAAAAACBCMAAAA|74|North |Circle|Suite F|Shiloh|Braxton County|WV|29275|United States|-5|condo| +49683|AAAAAAAADBCMAAAA|276|Hickory Johnson|ST|Suite 150|Antioch|Riley County|KS|68605|United States|-6|apartment| +49684|AAAAAAAAEBCMAAAA|419||Dr.|Suite Q|||WI||United States||condo| +49685|AAAAAAAAFBCMAAAA|464|14th Cedar|Lane|Suite I|Highland|Grand County|UT|89454|United States|-7|single family| +49686|AAAAAAAAGBCMAAAA|932|Sunset |Way|Suite 230|Riverview|Perry County|AR|79003|United States|-6|single family| +49687|AAAAAAAAHBCMAAAA|165|2nd Spring|Ct.|Suite B|Fairfield|Claiborne County|MS|56192|United States|-6|single family| +49688|AAAAAAAAIBCMAAAA|628|Green Oak|Dr.|Suite D|Lakeside|Carver County|MN|59532|United States|-6|single family| +49689|AAAAAAAAJBCMAAAA|214|12th 3rd|Boulevard|Suite 120|Ashland|Johnston County|OK|74244|United States|-6|condo| +49690|AAAAAAAAKBCMAAAA|694|Spring 8th|Ln|Suite 50|Newport|Bonneville County|ID|81521|United States|-7|apartment| +49691|AAAAAAAALBCMAAAA|576|North |Way|Suite 310|Wildwood|Defiance County|OH|46871|United States|-5|apartment| +49692|AAAAAAAAMBCMAAAA|374|Lincoln Railroad|Way|Suite J||Lexington County||28059|||condo| +49693|AAAAAAAANBCMAAAA|409|5th |Dr.|Suite R|Lakeside|Marshall County|TN|39532|United States|-6|single family| +49694|AAAAAAAAOBCMAAAA|521|3rd 7th|Road|Suite T|Fairview|Haakon County|SD|55709|United States|-7|apartment| +49695|AAAAAAAAPBCMAAAA|246|Main |Pkwy|Suite Y|Lakeview|Lewis County|KY|48579|United States|-5|condo| +49696|AAAAAAAAACCMAAAA|540|East |Avenue|Suite Y|Oak Grove|Cherokee County|TX|78370|United States|-6|single family| +49697|AAAAAAAABCCMAAAA|329|15th Lakeview|Lane|Suite 90|Spring Hill|Floyd County|KY|46787|United States|-6|single family| +49698|AAAAAAAACCCMAAAA|475|Center Forest||||Crawford County|||United States|-6|| +49699|AAAAAAAADCCMAAAA|430|Main |Blvd|Suite K|Red Hill|Kandiyohi County|MN|54338|United States|-6|apartment| +49700|AAAAAAAAECCMAAAA|490|4th Park|Dr.|Suite H|Florence|Linn County|KS|63394|United States|-6|single family| +49701|AAAAAAAAFCCMAAAA|777|Railroad |Road|Suite Q|Concord|Camp County|TX|74107|United States|-6|condo| +49702|AAAAAAAAGCCMAAAA|450|East 8th|Way|Suite R|Antioch|Campbell County|SD|58605|United States|-6|apartment| +49703|AAAAAAAAHCCMAAAA|504|Davis Spruce|Dr.|Suite 440|Woodland|Hamilton County|NY|14854|United States|-5|condo| +49704|AAAAAAAAICCMAAAA|583|13th Elm|Drive|Suite 10|Ashland|Nevada County|AR|74244|United States|-6|apartment| +49705|AAAAAAAAJCCMAAAA|405|4th Second|Ave|Suite 290|Hopewell|Daviess County|KY|40587|United States|-6|single family| +49706|AAAAAAAAKCCMAAAA|608|6th |RD|Suite 30|Riley|Perry County|OH|41692|United States|-5|apartment| +49707|AAAAAAAALCCMAAAA|193|South |Cir.|Suite E|Woodlawn|Ellis County|OK|74098|United States|-6|apartment| +49708|AAAAAAAAMCCMAAAA|657|Pine Pine|Dr.|Suite 60|Oakwood|Blaine County|OK|70169|United States|-6|condo| +49709|AAAAAAAANCCMAAAA|849|Fifth Adams|Parkway|Suite X|Pleasant Hill|Isle of Wight County|VA|23604|United States|-5|apartment| +49710|AAAAAAAAOCCMAAAA|216|Birch |Cir.|Suite 470|Wildwood|Foard County|TX|76871|United States|-6|condo| +49711|AAAAAAAAPCCMAAAA|639|10th |Road|Suite N|The Meadows|Orange County|VA|20026|United States|-5|condo| +49712|AAAAAAAAADCMAAAA|765|Chestnut |Circle|Suite J|Green Acres|Palo Pinto County|TX|77683|United States|-6|condo| +49713|AAAAAAAABDCMAAAA|64|Spruce Oak||Suite I||Judith Basin County|MT|64593|||apartment| +49714|AAAAAAAACDCMAAAA|690|Oak |Ave|Suite O|Hamilton|Monterey County|CA|92808|United States|-8|condo| +49715|AAAAAAAADDCMAAAA|273|Ridge 11th|||||||||single family| +49716|AAAAAAAAEDCMAAAA|201|Mill |Drive|Suite V|Macedonia|Vernon County|WI|51087|United States|-6|single family| +49717|AAAAAAAAFDCMAAAA|978|First East|Wy|Suite 70|Jamestown|Baca County|CO|86867|United States|-7|apartment| +49718|AAAAAAAAGDCMAAAA|368|8th |Pkwy|Suite 70|White Oak|Burnet County|TX|76668|United States|-6|single family| +49719|AAAAAAAAHDCMAAAA|883|Ash |Drive|Suite 90|Providence|McDowell County|NC|26614|United States|-5|apartment| +49720|AAAAAAAAIDCMAAAA|53|First |Dr.|Suite 440|Smith|Gaines County|TX|77317|United States|-6|condo| +49721|AAAAAAAAJDCMAAAA|628|Park Mill|Boulevard|Suite F|Pleasant Valley|Morris County|TX|72477|United States|-6|single family| +49722|AAAAAAAAKDCMAAAA|989|Sycamore Lake|RD|Suite M|Riverdale|Union County|OR|99391|United States|-8|apartment| +49723|AAAAAAAALDCMAAAA|366|Seventh Oak|Avenue|Suite S|Woodlawn|Caldwell Parish|LA|74098|United States|-6|single family| +49724|AAAAAAAAMDCMAAAA|635|Elm |Way|Suite 410|Georgetown|Kent County|RI|07657|United States|-5|condo| +49725|AAAAAAAANDCMAAAA|589|Lincoln Ash|Dr.|Suite 250|Longwood|Hardin County|IA|57021|United States|-6|condo| +49726|AAAAAAAAODCMAAAA|940|Church |Dr.|Suite L|Glenwood|Dinwiddie County|VA|23511|United States|-5|condo| +49727|AAAAAAAAPDCMAAAA|180|Broadway Maple|Ct.|Suite G|Union Hill|Fayette County|GA|37746|United States|-5|single family| +49728|AAAAAAAAAECMAAAA|483|River Woodland|Court|Suite M|Riverview|Lawrence County|MS|59003|United States|-6|single family| +49729|AAAAAAAABECMAAAA|265|Main |Cir.|Suite 240|Deerfield|Fairfield County|SC|29840|United States|-5|condo| +49730|AAAAAAAACECMAAAA|111|Madison |Dr.|Suite F|Stringtown|Dallam County|TX|70162|United States|-6|apartment| +49731|AAAAAAAADECMAAAA|162|Oak Cherry|Dr.|Suite 170|Bethel|Wythe County|VA|25281|United States|-5|single family| +49732|AAAAAAAAEECMAAAA|595|13th |Lane|Suite T|Perkins|Sheridan County|||United States||| +49733|AAAAAAAAFECMAAAA||||||||54338||-6|| +49734|AAAAAAAAGECMAAAA|52|Washington |Avenue|Suite A|Summit|Franklin County|KS|60499|United States|-6|single family| +49735|AAAAAAAAHECMAAAA|144|Chestnut |Ave|Suite 30|Belmont|Bristol County|RI|00791|United States|-5|apartment| +49736|AAAAAAAAIECMAAAA|248||||Liberty|Davis County|||United States|-7|apartment| +49737|AAAAAAAAJECMAAAA|861|View |Ave|Suite 20|Woodruff|Polk County|IA|54174|United States|-6|apartment| +49738|AAAAAAAAKECMAAAA|599|3rd |Ct.|Suite I|Forest Hills|Siskiyou County|CA|99237|United States|-8|apartment| +49739|AAAAAAAALECMAAAA|253|10th Valley|Parkway|Suite L|Oakwood|Mobile County|AL|30169|United States|-6|apartment| +49740|AAAAAAAAMECMAAAA|896|East |Lane|Suite C|Spring Hill|Prince William County|VA|26787|United States|-5|apartment| +49741|AAAAAAAANECMAAAA|276|Walnut |Cir.|Suite 90|Mount Olive|Ogemaw County|MI|48059|United States|-5|single family| +49742|AAAAAAAAOECMAAAA|74|Lake |Dr.|Suite 80|Springfield|Yakutat Borough|AK|99303|United States|-9|condo| +49743|AAAAAAAAPECMAAAA|548|Third |Ln|Suite 40|Springfield|Burke County|ND|59303|United States|-6|condo| +49744|AAAAAAAAAFCMAAAA|632|Cedar |Boulevard|Suite 420|Kingston|Goochland County|VA|24975|United States|-5|condo| +49745|AAAAAAAABFCMAAAA|618|Jackson View|Boulevard|Suite 110|Lakewood|Keweenaw County|MI|48877|United States|-5|single family| +49746|AAAAAAAACFCMAAAA|839|Church ||Suite N|Gum Springs|Dimmit County|TX||United States||| +49747|AAAAAAAADFCMAAAA|164|Sunset |Way|Suite 330|Shiloh|Limestone County|AL|39275|United States|-6|condo| +49748|AAAAAAAAEFCMAAAA|488|Ridge Broadway|Blvd|Suite J|Oak Ridge|Holt County|NE|68371|United States|-7|single family| +49749|AAAAAAAAFFCMAAAA|207|2nd Park|Drive|Suite F|Pleasant Grove|Lafayette County|MS|54136|United States|-6|apartment| +49750|AAAAAAAAGFCMAAAA|505|Oak Pine|Boulevard|Suite N|Oak Ridge|Cherokee County|IA|58371|United States|-6|apartment| +49751|AAAAAAAAHFCMAAAA|136|Eigth |RD|Suite B|Friendship|Lauderdale County|AL|34536|United States|-6|apartment| +49752|AAAAAAAAIFCMAAAA|162|Birch |Lane|Suite A|Harris|Pepin County|WI|55752|United States|-6|single family| +49753|AAAAAAAAJFCMAAAA|598|Cherry |Court|Suite 490|Woodland|Bourbon County|KS|64854|United States|-6|condo| +49754|AAAAAAAAKFCMAAAA|444|Ridge Walnut|Cir.|Suite 40|Glenwood|Guthrie County|IA|53511|United States|-6|single family| +49755|AAAAAAAALFCMAAAA|311|Railroad |Lane||Riverview|Hitchcock County|NE|||-6|single family| +49756|AAAAAAAAMFCMAAAA|730|3rd Walnut|Cir.|Suite D|Arlington|Jeff Davis County|GA|36557|United States|-5|condo| +49757|AAAAAAAANFCMAAAA||||||||58605|United States|-6|| +49758|AAAAAAAAOFCMAAAA|514|Ash Maple|Cir.|Suite 180|Deerfield|Carroll County|NH|09840|United States|-5|apartment| +49759|AAAAAAAAPFCMAAAA|402|Walnut 7th|Blvd|Suite U|Fairfield|Sandusky County|OH|46192|United States|-5|apartment| +49760|AAAAAAAAAGCMAAAA|864|Cedar 4th|Wy|Suite M|Harmony|Wilkes County|NC|25804|United States|-5|condo| +49761|AAAAAAAABGCMAAAA|660|Willow Franklin|Avenue|Suite 350|Calhoun|Steele County|MN|56909|United States|-6|apartment| +49762|AAAAAAAACGCMAAAA|933|5th |Pkwy|Suite 60|Cedar Grove|Parker County|TX|70411|United States|-6|apartment| +49763|AAAAAAAADGCMAAAA|201|Woodland Williams|Street|Suite U|Wildwood|Jones County|TX|76871|United States|-6|condo| +49764|AAAAAAAAEGCMAAAA|806|Willow |Ln|Suite 100|Macedonia|Fulton County|AR|71087|United States|-6|apartment| +49765|AAAAAAAAFGCMAAAA|980|Smith 9th|Ln|Suite G|Bethel|Glenn County|CA|95281|United States|-8|condo| +49766|AAAAAAAAGGCMAAAA|776|Sixth |Court|Suite X|Plainview|Lincoln County|MN|53683|United States|-6|single family| +49767|AAAAAAAAHGCMAAAA|551|Hickory |Boulevard|Suite J|Lincoln|Volusia County|FL|31289|United States|-5|single family| +49768|AAAAAAAAIGCMAAAA|256|West |Road|Suite 100|Providence|Rockbridge County|VA|26614|United States|-5|single family| +49769|AAAAAAAAJGCMAAAA|172|1st Cherry|Ct.||Spring Valley|Jersey County||66060|United States||condo| +49770|AAAAAAAAKGCMAAAA|618|Pine Franklin|Avenue|Suite A|Ashland|Stephenson County|IL|64244|United States|-6|single family| +49771|AAAAAAAALGCMAAAA|447|Smith Fourth|Blvd|Suite 150|Red Oak|Bear Lake County|ID|85018|United States|-7|single family| +49772|AAAAAAAAMGCMAAAA|99|Third Lakeview|Wy|Suite N|Gary|Frederick County|VA|20418|United States|-5|condo| +49773|AAAAAAAANGCMAAAA|33|9th Maple|Ct.|Suite 170|Stringtown|Montcalm County|MI|40162|United States|-5|apartment| +49774|AAAAAAAAOGCMAAAA|549|Spring 8th|Circle|Suite 430|Marion|Sedgwick County|KS|60399|United States|-6|single family| +49775|AAAAAAAAPGCMAAAA|||Way|Suite 480||Rock County|||United States||apartment| +49776|AAAAAAAAAHCMAAAA|620|Wilson Jefferson|Road|Suite U|Mount Pleasant|Vermillion County|IN|41933|United States|-5|single family| +49777|AAAAAAAABHCMAAAA|915|Dogwood |Parkway|Suite S|Milo|Colfax County|NM|80116|United States|-7|single family| +49778|AAAAAAAACHCMAAAA|264|Cedar Washington|Court|Suite M|Macedonia|Burleson County|TX|71087|United States|-6|single family| +49779|AAAAAAAADHCMAAAA|493|Valley |Lane|Suite 110|Glenwood|Converse County|WY|83511|United States|-7|condo| +49780|AAAAAAAAEHCMAAAA|19|2nd Lincoln|Drive|Suite M|Arlington|Cavalier County|ND|56557|United States|-6|single family| +49781|AAAAAAAAFHCMAAAA|674|Smith Franklin|Cir.|Suite H|Centerville|Terrell County|TX|70059|United States|-6|apartment| +49782|AAAAAAAAGHCMAAAA|817|8th |Lane|Suite 310|Cedar Grove|Bell County|TX|70411|United States|-6|condo| +49783|AAAAAAAAHHCMAAAA|878|Cedar |Wy|Suite 390|Oakland|Clayton County|GA|39843|United States|-5|single family| +49784|AAAAAAAAIHCMAAAA|31|5th |Court|Suite 40|Highland Park|Montgomery County|KY|46534|United States|-5|apartment| +49785|AAAAAAAAJHCMAAAA|169|Oak Third|ST|Suite M|Oakwood|Placer County|CA|90169|United States|-8|condo| +49786|AAAAAAAAKHCMAAAA|488|North |Blvd|Suite V|Guthrie|Collin County|TX|71423|United States|-6|single family| +49787|AAAAAAAALHCMAAAA|983|Walnut 13th|Pkwy|Suite 260|Jamestown|Jackson County|AL|36867|United States|-6|apartment| +49788|AAAAAAAAMHCMAAAA|574|Elm |Court|Suite 160|Deerfield|Livingston Parish|LA|79840|United States|-6|apartment| +49789|AAAAAAAANHCMAAAA|520|Hill |Avenue|Suite P|Jamestown|DeKalb County|TN|36867|United States|-5|condo| +49790|AAAAAAAAOHCMAAAA|987|Hickory |ST|Suite G|Oak Ridge|Emery County|UT|88371|United States|-7|apartment| +49791|AAAAAAAAPHCMAAAA|662|Sycamore Wilson|Parkway|Suite I|Farmington|Leake County|MS|59145|United States|-6|condo| +49792|AAAAAAAAAICMAAAA|228|4th 1st|Ln|Suite W|Mount Pleasant|Grand County|CO|81933|United States|-7|single family| +49793|AAAAAAAABICMAAAA|280|Eigth |Dr.|Suite M|Harmony|Cerro Gordo County|IA|55804|United States|-6|apartment| +49794|AAAAAAAACICMAAAA|650|Locust |Cir.|Suite 200|Oak Grove|Custer County|ID|88370|United States|-7|condo| +49795|AAAAAAAADICMAAAA|886|Smith Forest|Ct.|Suite G|Franklin|Roanoke County|VA|29101|United States|-5|condo| +49796|AAAAAAAAEICMAAAA|741|Ridge |Lane|Suite 290|||NY|11904|United States|-5|condo| +49797|AAAAAAAAFICMAAAA|528|Ash |Pkwy|Suite 150|Spring Valley|Union County|IN|46060|United States|-5|apartment| +49798|AAAAAAAAGICMAAAA|68|Highland 1st|Blvd|Suite N|Midway|Lake County|TN|31904|United States|-6|apartment| +49799|AAAAAAAAHICMAAAA|513|5th |Wy|Suite T|White Oak|Valdez-Cordova Census Area|AK|96668|United States|-9|apartment| +49800|AAAAAAAAIICMAAAA|345|Elm |ST|Suite 280|Oak Ridge|Laurel County|KY|48371|United States|-5|apartment| +49801|AAAAAAAAJICMAAAA|442|4th |ST|Suite P|Greenville|De Witt County|IL|61387|United States|-6|condo| +49802|AAAAAAAAKICMAAAA|548|Sunset |Wy|Suite 460|Deerfield|Levy County|FL|39840|United States|-5|single family| +49803|AAAAAAAALICMAAAA|195|Sycamore |Pkwy|Suite P|New Hope|Lee County|NC|29431|United States|-5|apartment| +49804|AAAAAAAAMICMAAAA|388|Woodland Forest|Boulevard|Suite 50|Fairview|Miller County|AR|75709|United States|-6|condo| +49805|AAAAAAAANICMAAAA|569|Walnut Park|Ave|Suite Q|Wilson|Watauga County|NC|26971|United States|-5|condo| +49806|AAAAAAAAOICMAAAA|377|5th Laurel|Way|Suite A|Red Hill|Marion County|MO|64338|United States|-6|single family| +49807|AAAAAAAAPICMAAAA|985|Meadow |Blvd|Suite 110|Centerville|Henry County|IA|50059|United States|-6|single family| +49808|AAAAAAAAAJCMAAAA|431|2nd |Parkway|Suite 160|Highland Park|Howard County|AR|76534|United States|-6|condo| +49809|AAAAAAAABJCMAAAA|||Ct.|Suite R|Cherry Valley|Louisa County|IA||||condo| +49810|AAAAAAAACJCMAAAA||Pine Hickory||Suite U|Highland|||69454|United States||condo| +49811|AAAAAAAADJCMAAAA|606|Green |Blvd|Suite 420|Hopewell|Carroll County|IA|50587|United States|-6|single family| +49812|AAAAAAAAEJCMAAAA||||Suite W|Greenfield|Page County|VA||United States||apartment| +49813|AAAAAAAAFJCMAAAA|440|13th East|ST|Suite 270|Greenwood|Carter County|MT|68828|United States|-7|single family| +49814|AAAAAAAAGJCMAAAA|726|Dogwood |Drive|Suite B|Oakdale|Alpena County|MI|49584|United States|-5|condo| +49815|AAAAAAAAHJCMAAAA|275|Cherry |Road|Suite N|Florence|Surry County|VA|23394|United States|-5|condo| +49816|AAAAAAAAIJCMAAAA|952|Walnut Sunset|Circle|Suite A|Deerfield|Obion County|TN|39840|United States|-6|apartment| +49817|AAAAAAAAJJCMAAAA|520|Ridge |Wy|Suite G|Clifton|Martin County|MN|58014|United States|-6|condo| +49818|AAAAAAAAKJCMAAAA|216|Washington Ash|RD|Suite 250|Lakewood|Lawrence County|TN|38877|United States|-6|condo| +49819|AAAAAAAALJCMAAAA||6th Fourth|||Lawrence||PA|17322|United States|-5|| +49820|AAAAAAAAMJCMAAAA|142|Williams Cedar|ST|Suite 450|Antioch|Livingston County|KY|48605|United States|-5|condo| +49821|AAAAAAAANJCMAAAA|306|Fifth |Blvd|Suite E|Deerfield|Elko County|NV|89840|United States|-8|condo| +49822|AAAAAAAAOJCMAAAA|814|Second View|Avenue|Suite R|Centerville|Fayette County|IN|40059|United States|-5|single family| +49823|AAAAAAAAPJCMAAAA|49|Fifth |ST|Suite 290|Hopewell|Saluda County|SC|20587|United States|-5|single family| +49824|AAAAAAAAAKCMAAAA|458|College Walnut|Circle|Suite 70|Bridgeport|Calhoun County|FL|35817|United States|-5|apartment| +49825|AAAAAAAABKCMAAAA|215|Maple |Ct.|Suite 470|Glenwood|Archuleta County|CO|83511|United States|-7|apartment| +49826|AAAAAAAACKCMAAAA|289|4th Jefferson|Wy|Suite 260|Bethel|Crook County|OR|95281|United States|-8|condo| +49827|AAAAAAAADKCMAAAA|112|10th Wilson|Lane|Suite 450|Valley View|Scott County|IL|65124|United States|-6|single family| +49828|AAAAAAAAEKCMAAAA|885|6th 9th|Ct.|Suite M|Enterprise|Jefferson County|IL|61757|United States|-6|apartment| +49829|AAAAAAAAFKCMAAAA|330|Locust |Dr.|Suite 80|Lincoln|Monona County|IA|51289|United States|-6|single family| +49830|AAAAAAAAGKCMAAAA|400|Meadow Lakeview|Ave|Suite 70|Mount Pleasant|Beckham County|OK|71933|United States|-6|apartment| +49831|AAAAAAAAHKCMAAAA|907|4th Oak|Circle|Suite W|Little River|Marion County|KS|60319|United States|-6|single family| +49832|AAAAAAAAIKCMAAAA|847|Fifteenth West|Circle|Suite 200|Glendale|Mesa County|CO|83951|United States|-7|apartment| +49833|AAAAAAAAJKCMAAAA|157|Fourth Hickory|Ct.|Suite 390|Spring Hill|Franklin County|NY|16787|United States|-5|apartment| +49834|AAAAAAAAKKCMAAAA|532|2nd |ST|Suite 100|Green Acres|Edmunds County|SD|57683|United States|-6|condo| +49835|AAAAAAAALKCMAAAA|926|Maple |Road|Suite 280|Oak Hill|Mono County|CA|97838|United States|-8|apartment| +49836|AAAAAAAAMKCMAAAA|4|Williams |Parkway|Suite M|Adrian|Kenai Peninsula Borough|AK|93301|United States|-9|condo| +49837|AAAAAAAANKCMAAAA|237|Meadow Pine|Ln|Suite I|Pleasant Grove|Caddo Parish|LA|74136|United States|-6|single family| +49838|AAAAAAAAOKCMAAAA|246|North River|Pkwy|Suite R|Valley View|Putnam County|WV|25124|United States|-5|single family| +49839|AAAAAAAAPKCMAAAA|318|7th |ST|Suite 280|Bunker Hill|Denver County|CO|80150|United States|-7|condo| +49840|AAAAAAAAALCMAAAA|13|2nd 2nd|Boulevard|Suite N|Forest Hills|Ferry County|WA|99237|United States|-8|condo| +49841|AAAAAAAABLCMAAAA|80|Sunset |Lane|Suite Q|Plainview|Baltimore city|MD|23683|United States|-5|single family| +49842|AAAAAAAACLCMAAAA|798|8th 2nd|Wy|Suite C|Summit|Knox County|IN|40499|United States|-5|apartment| +49843|AAAAAAAADLCMAAAA|582|Locust 1st|Parkway|Suite M|Pleasant Valley|Kiowa County|KS|62477|United States|-6|single family| +49844|AAAAAAAAELCMAAAA|413|Central |Parkway|Suite 240|Walnut Grove|Bienville Parish|LA|77752|United States|-6|condo| +49845|AAAAAAAAFLCMAAAA|908|First 4th|Pkwy|Suite 140|Vienna|Clark County|IN|45119|United States|-5|condo| +49846|AAAAAAAAGLCMAAAA|854|5th |Lane|Suite P|Waterloo|Chase County|KS|61675|United States|-6|apartment| +49847|AAAAAAAAHLCMAAAA|475|Adams Johnson|Way|Suite V|Liberty|Chattahoochee County|GA|33451|United States|-5|apartment| +49848|AAAAAAAAILCMAAAA|764|Cedar 3rd|Avenue|Suite C|Mountain View|Haskell County|KS|64466|United States|-6|condo| +49849|AAAAAAAAJLCMAAAA|978|Fourth |Ct.|Suite 170|Centerville|Lyon County|IA|50059|United States|-6|single family| +49850|AAAAAAAAKLCMAAAA|371|Pine |Boulevard|Suite B|Sunnyside|Texas County|OK|71952|United States|-6|single family| +49851|AAAAAAAALLCMAAAA|975|Hill |Circle|Suite 260|Vista|Stonewall County|TX|74694|United States|-6|condo| +49852|AAAAAAAAMLCMAAAA|939|Cherry |Drive|Suite T|Newport|Russell County|VA|21521|United States|-5|single family| +49853|AAAAAAAANLCMAAAA|987|8th East|Blvd|Suite 490|Lakeside|Christian County|KY|49532|United States|-6|single family| +49854|AAAAAAAAOLCMAAAA|775|Dogwood Spruce|Cir.|Suite C|Oakdale|Lamar County|AL|39584|United States|-6|apartment| +49855|AAAAAAAAPLCMAAAA|690|South Lake|Lane|Suite A|Jackson|Ripley County|IN|49583|United States|-5|apartment| +49856|AAAAAAAAAMCMAAAA|292|Adams 2nd|Cir.|Suite C|Glendale|Navajo County|AZ|83951|United States|-7|apartment| +49857|AAAAAAAABMCMAAAA|763|Park |Blvd|Suite 100|Fairfield|Rosebud County|MT|66192|United States|-7|condo| +49858|AAAAAAAACMCMAAAA|288|1st Wilson|Wy|Suite 260|Providence|Rush County|IN|46614|United States|-5|single family| +49859|AAAAAAAADMCMAAAA|975|Franklin Smith|Circle|Suite 230|Arlington|Lake County|MT|66557|United States|-7|apartment| +49860|AAAAAAAAEMCMAAAA|552|2nd |Way|Suite 250|Oak Hill|Tunica County|MS|57838|United States|-6|single family| +49861|AAAAAAAAFMCMAAAA|497|Church Elm|Ave|Suite 140|Lakeview|Marshall County|IA|58579|United States|-6|single family| +49862|AAAAAAAAGMCMAAAA|47|View |RD|Suite 490|Providence|Chesterfield County|SC|26614|United States|-5|condo| +49863|AAAAAAAAHMCMAAAA|899|Adams Fourth|Parkway|Suite 270|Mount Pleasant|San Bernardino County|CA|91933|United States|-8|single family| +49864|AAAAAAAAIMCMAAAA|276|Church Pine|Boulevard|Suite 410|Fairview|Lyon County|KY|45709|United States|-5|condo| +49865|AAAAAAAAJMCMAAAA|34|Ninth Maple|Avenue|Suite S|Wildwood|Humboldt County|CA|96871|United States|-8|apartment| +49866|AAAAAAAAKMCMAAAA|108|Madison Sycamore|Boulevard|Suite W|Belmont|Smith County|KS|60191|United States|-6|condo| +49867|AAAAAAAALMCMAAAA|931|Highland 11th|Road|Suite R|Kingston|McCook County|SD|54975|United States|-7|condo| +49868|AAAAAAAAMMCMAAAA|257|Cedar Lee|Street|Suite O|Bridgeport|Coffey County|KS|65817|United States|-6|single family| +49869|AAAAAAAANMCMAAAA|337|Park |Ave|Suite H|Woodlawn|Wagoner County|OK|74098|United States|-6|condo| +49870|AAAAAAAAOMCMAAAA|341|2nd |Court|Suite 390|Pleasant Hill|Moniteau County|MO|63604|United States|-6|condo| +49871|AAAAAAAAPMCMAAAA|82|Oak 1st|Drive|Suite 160|Midway|Cherokee County|AL|31904|United States|-6|single family| +49872|AAAAAAAAANCMAAAA|805|Johnson |Drive|Suite 410|Harmony|Lawrence County|IN|45804|United States|-5|single family| +49873|AAAAAAAABNCMAAAA|789|5th South|RD|Suite 280|Mount Olive|Barrow County|GA|38059|United States|-5|single family| +49874|AAAAAAAACNCMAAAA|538|6th Eigth|Pkwy|Suite T|Oak Grove|Oconto County|WI|58370|United States|-6|condo| +49875|AAAAAAAADNCMAAAA|82|12th Walnut|Road|Suite 240|Maple Grove|Delta County|CO|88252|United States|-7|apartment| +49876|AAAAAAAAENCMAAAA|552|Cherry Locust|Way|Suite O|Unionville|Eastland County|TX|71711|United States|-6|apartment| +49877|AAAAAAAAFNCMAAAA|224|Hillcrest |Drive|Suite Q|Brownsville|Whatcom County|WA|99310|United States|-8|single family| +49878|AAAAAAAAGNCMAAAA|667|Fifth |Blvd|Suite 370|Fisher|Dunklin County|MO|62819|United States|-6|condo| +49879|AAAAAAAAHNCMAAAA|414|Williams Church|Cir.|Suite G|Glenwood|Montour County|PA|13511|United States|-5|single family| +49880|AAAAAAAAINCMAAAA|943|Spruce Sunset|Road|Suite 400|Newport|Holmes County|FL|31521|United States|-5|single family| +49881|AAAAAAAAJNCMAAAA|352|Lake |Way|Suite 370|Harmony|Wapello County|IA|55804|United States|-6|single family| +49882|AAAAAAAAKNCMAAAA|820|Washington Walnut|Parkway|Suite 0|Maple Hill|Vermillion County|IN|48095|United States|-5|condo| +49883|AAAAAAAALNCMAAAA|624|Wilson Main|Drive|Suite 150|Red Hill|Glades County|FL|34338|United States|-5|apartment| +49884|AAAAAAAAMNCMAAAA|574|Ridge |Parkway|Suite 20|Rockwood|Accomack County|VA|21545|United States|-5|condo| +49885|AAAAAAAANNCMAAAA|897|Church |Wy|Suite 120|Forest Hills|Delaware County|OK|79237|United States|-6|apartment| +49886|AAAAAAAAONCMAAAA|539|Center Park|ST|Suite 100|Greenfield|Fresno County|CA|95038|United States|-8|single family| +49887|AAAAAAAAPNCMAAAA|785|Davis |RD|Suite 60|Union|Mahaska County|IA|58721|United States|-6|apartment| +49888|AAAAAAAAAOCMAAAA|892|Lake Forest|Way|Suite 400|Lakewood|Faribault County|MN|58877|United States|-6|single family| +49889|AAAAAAAABOCMAAAA||2nd |||Lebanon||FL|||-5|apartment| +49890|AAAAAAAACOCMAAAA|574|Cedar 7th|Wy|Suite D|Plainview|Forsyth County|NC|23683|United States|-5|single family| +49891|AAAAAAAADOCMAAAA|486|2nd |Dr.|Suite 300|Oak Ridge|Conejos County|CO|88371|United States|-7|apartment| +49892|AAAAAAAAEOCMAAAA|613|3rd |Parkway|Suite 80|Shiloh|Piute County|UT|89275|United States|-7|condo| +49893|AAAAAAAAFOCMAAAA|293|15th 11th|Parkway|Suite 90|Thompsonville|Louisa County|VA|29651|United States|-5|condo| +49894|AAAAAAAAGOCMAAAA|889|Pine 8th|Dr.|Suite 0|Greenville|Whitley County|KY|41387|United States|-5|condo| +49895|AAAAAAAAHOCMAAAA|626|East |Court|Suite 350|New Hope|Adams County|CO|89431|United States|-7|apartment| +49896|AAAAAAAAIOCMAAAA|679|Laurel |Way|Suite V|Glendale|Giles County|VA|23951|United States|-5|apartment| +49897|AAAAAAAAJOCMAAAA|436|Jefferson |Cir.|Suite Q|Adrian|Elbert County|GA|33301|United States|-5|apartment| +49898|AAAAAAAAKOCMAAAA|193|Pine |Ave|Suite 200|Evans|Schoharie County|NY|12284|United States|-5|single family| +49899|AAAAAAAALOCMAAAA|188|4th |ST|Suite 140|Georgetown|Emmet County|IA|57057|United States|-6|condo| +49900|AAAAAAAAMOCMAAAA|408|Jefferson North|Parkway|Suite 430|Forest Hills|Carroll County|GA|39237|United States|-5|condo| +49901|AAAAAAAANOCMAAAA|||ST|Suite 280|Woodlawn|Limestone County|TX||United States|-6|| +49902|AAAAAAAAOOCMAAAA|837|Woodland |Ct.|Suite N|Franklin|Sevier County|AR|79101|United States|-6|condo| +49903|AAAAAAAAPOCMAAAA|870|Center |Boulevard|Suite 360|Wilson|Sanborn County|SD|56971|United States|-7|condo| +49904|AAAAAAAAAPCMAAAA|659|Spring |Parkway|Suite 190|Enterprise|Ferry County|WA|91757|United States|-8|condo| +49905|AAAAAAAABPCMAAAA|176|Fifth |Cir.|Suite 60|Summit|Effingham County|IL|60499|United States|-6|single family| +49906|AAAAAAAACPCMAAAA|725|Cherry |Dr.|Suite 40|Mount Olive|Wise County|VA|28059|United States|-5|apartment| +49907|AAAAAAAADPCMAAAA|685|Sunset Mill|Boulevard|Suite 230|Highland Park|Franklin County|TN|36534|United States|-5|single family| +49908|AAAAAAAAEPCMAAAA|159|Walnut Seventh|ST|Suite S|Oakland|Aroostook County|ME|09843|United States|-5|single family| +49909|AAAAAAAAFPCMAAAA|104|Pine |Street|Suite 480|Stringtown|Newton County|MS|50162|United States|-6|single family| +49910|AAAAAAAAGPCMAAAA|370|Johnson |Ave|Suite M|Greenville|Wake County|NC|21387|United States|-5|condo| +49911|AAAAAAAAHPCMAAAA|960|Eigth |Way|Suite 240|Mountain View|Putnam County|GA|34466|United States|-5|apartment| +49912|AAAAAAAAIPCMAAAA|567|Oak |Pkwy|Suite 70|Forest Hills|Grundy County|TN|39237|United States|-5|apartment| +49913|AAAAAAAAJPCMAAAA|41|Park East|Drive|Suite 150|Harmony|Cabarrus County|NC|25804|United States|-5|condo| +49914|AAAAAAAAKPCMAAAA|324|South |Drive|Suite L|Unionville|Page County|VA|21711|United States|-5|single family| +49915|AAAAAAAALPCMAAAA|990|Ridge 10th|Way|Suite Q|Providence|Bamberg County|SC|26614|United States|-5|single family| +49916|AAAAAAAAMPCMAAAA|667|15th Laurel|Blvd|Suite B|Marion|Anne Arundel County|MD|20399|United States|-5|apartment| +49917|AAAAAAAANPCMAAAA|885|15th Third|ST|Suite C|Riverside|Elliott County|KY|49231|United States|-6|apartment| +49918|AAAAAAAAOPCMAAAA|106|Fifth |Court|Suite G|Florence|Trinity County|CA|93394|United States|-8|apartment| +49919|AAAAAAAAPPCMAAAA|938|Lakeview |Blvd|Suite H|Brownsville|Fannin County|TX|79310|United States|-6|apartment| +49920|AAAAAAAAAADMAAAA|543|South Fourth|Pkwy|Suite 230|Midway|Wise County|VA|21904|United States|-5|apartment| +49921|AAAAAAAABADMAAAA|885|Birch |Ct.|Suite 280|Cordova|Missaukee County|MI|46938|United States|-5|single family| +49922|AAAAAAAACADMAAAA|986|8th Laurel|Cir.|Suite 320|Forest Hills|Lee County|IA|59237|United States|-6|condo| +49923|AAAAAAAADADMAAAA|676|3rd Hickory|Cir.|Suite X|Montague|Foard County|TX|74062|United States|-6|apartment| +49924|AAAAAAAAEADMAAAA|155|Forest |Way|Suite M|Mountain View|Floyd County|VA|24466|United States|-5|single family| +49925|AAAAAAAAFADMAAAA||||Suite 220|Newport|Berkshire County|||||single family| +49926|AAAAAAAAGADMAAAA|979|12th Central|Cir.|Suite J|Pine Grove|Whatcom County|WA|94593|United States|-8|apartment| +49927|AAAAAAAAHADMAAAA|373|Hill |Lane|Suite N|Providence|Franklin County|IN|46614|United States|-5|single family| +49928|AAAAAAAAIADMAAAA||10th |Lane|||Los Alamos County|NM|||-7|single family| +49929|AAAAAAAAJADMAAAA|703|Main Center|Ln|Suite 20|Concord|Tolland County|CT|04707|United States|-5|apartment| +49930|AAAAAAAAKADMAAAA|974|Seventh |Way|Suite 190|Bridgeport|New Madrid County|MO|65817|United States|-6|apartment| +49931|AAAAAAAALADMAAAA|690|9th Washington|Boulevard|Suite A|Springdale|Hardy County|WV|28883|United States|-5|apartment| +49932|AAAAAAAAMADMAAAA|409|Church |Cir.|Suite 50|Lincoln|Ben Hill County|GA|31289|United States|-5|single family| +49933|AAAAAAAANADMAAAA|215|Park 12th|Dr.|Suite 160|Mount Zion|Marshall County|MN|58054|United States|-6|apartment| +49934|AAAAAAAAOADMAAAA|36|Birch 2nd|Dr.|Suite 80|Woodland|Yakima County|WA|94854|United States|-8|single family| +49935|AAAAAAAAPADMAAAA|796||Ln||Mount Olive|Logan County|IL||United States||| +49936|AAAAAAAAABDMAAAA|166|9th Second|Avenue|Suite 290|Riley|Carter County|OK|71692|United States|-6|apartment| +49937|AAAAAAAABBDMAAAA|888|Oak Woodland|Drive|Suite 10|Red Hill|Montgomery County|TX|74338|United States|-6|condo| +49938|AAAAAAAACBDMAAAA|805|7th |RD|Suite L|Woodville|Douglas County|NE|64289|United States|-6|single family| +49939|AAAAAAAADBDMAAAA|114|Pine |ST|Suite 220|Providence|Accomack County|VA|26614|United States|-5|condo| +49940|AAAAAAAAEBDMAAAA|767|Willow |Circle|Suite I|Pine Grove|Yavapai County|AZ|84593|United States|-7|condo| +49941|AAAAAAAAFBDMAAAA|989|Oak |Pkwy|Suite 90|Providence|New Hanover County|NC|26614|United States|-5|condo| +49942|AAAAAAAAGBDMAAAA|32|Park |Circle|Suite 440|Walnut Grove|Dodge County|GA|37752|United States|-5|single family| +49943|AAAAAAAAHBDMAAAA|46|Second Sunset|Ln|Suite 380|Providence|Franklin County|MO|66614|United States|-6|single family| +49944|AAAAAAAAIBDMAAAA|939|West Green|Ct.|Suite I|Lakeside|Cheatham County|TN|39532|United States|-5|single family| +49945|AAAAAAAAJBDMAAAA|493|Central |Dr.|Suite 0|Five Points|Redwood County|MN|56098|United States|-6|apartment| +49946|AAAAAAAAKBDMAAAA|184|Chestnut Sixth|Ave|Suite Q|Maple Grove|Maverick County|TX|78252|United States|-6|apartment| +49947|AAAAAAAALBDMAAAA|914|Oak Jackson|Road|Suite B|Mount Zion|Hopewell city|VA|28054|United States|-5|condo| +49948|AAAAAAAAMBDMAAAA|847|2nd |ST|Suite A|Bunker Hill|Sumner County|KS|60150|United States|-6|single family| +49949|AAAAAAAANBDMAAAA|640|14th Chestnut|Ln|Suite 200|Union Hill|Nelson County|KY|47746|United States|-5|condo| +49950|AAAAAAAAOBDMAAAA|794|4th Jackson|Lane|Suite 300|Buena Vista|Camden County|NJ|06352|United States|-5|apartment| +49951|AAAAAAAAPBDMAAAA|948|East |Pkwy|Suite 40|Valley View|Dundy County|NE|65124|United States|-6|single family| +49952|AAAAAAAAACDMAAAA|904|Central 15th|Dr.|Suite G|Springfield|Wabaunsee County|KS|69303|United States|-6|condo| +49953|AAAAAAAABCDMAAAA|354|Lake Pine|Avenue|Suite J|Salem|Pinal County|AZ|88048|United States|-7|single family| +49954|AAAAAAAACCDMAAAA|813|Cedar |Avenue|Suite J|Hopewell|Sonoma County|CA|90587|United States|-8|condo| +49955|AAAAAAAADCDMAAAA|177|Jackson 3rd|Pkwy|Suite N|Greenwood|Aitkin County|MN|58828|United States|-6|condo| +49956|AAAAAAAAECDMAAAA|791|Pine |Avenue|Suite 390|Wilson|Phillips County|MT|66971|United States|-7|condo| +49957|AAAAAAAAFCDMAAAA|227|Adams Highland|Drive|Suite S|Spring Valley|Pocahontas County|WV|26060|United States|-5|single family| +49958|AAAAAAAAGCDMAAAA|553|5th Third|Boulevard|Suite 170|Friendship|Carbon County|WY|84536|United States|-7|apartment| +49959|AAAAAAAAHCDMAAAA|3|Wilson ||||||40499|||| +49960|AAAAAAAAICDMAAAA|573|Main |ST|Suite M|Plainview|Beckham County|OK|73683|United States|-6|single family| +49961|AAAAAAAAJCDMAAAA||||||Granville County|||United States||| +49962|AAAAAAAAKCDMAAAA|344|Smith |Ln|Suite O|Jamestown|Larimer County|CO|86867|United States|-7|condo| +49963|AAAAAAAALCDMAAAA|404|Cedar |Ct.|Suite Y|Enterprise|Treutlen County|GA|31757|United States|-5|apartment| +49964|AAAAAAAAMCDMAAAA|80|Forest Cedar|Boulevard|Suite T|Bethel|Iberville Parish|LA|75281|United States|-6|condo| +49965|AAAAAAAANCDMAAAA|122|Washington |Dr.|Suite 170|Stringtown|Wilkinson County|MS|50162|United States|-6|apartment| +49966|AAAAAAAAOCDMAAAA|870|1st |Dr.|Suite 120|Owens|Hickman County|KY|42324|United States|-6|single family| +49967|AAAAAAAAPCDMAAAA|645|7th 9th|Ave|Suite H|Concord|Matanuska-Susitna Borough|AK|94107|United States|-9|apartment| +49968|AAAAAAAAADDMAAAA|89|Woodland |Court|Suite 380|Georgetown|Madison County|KY|47057|United States|-5|apartment| +49969|AAAAAAAABDDMAAAA|903|Valley |Pkwy|Suite J|Woodlawn|Lawrence County|OH|44098|United States|-5|apartment| +49970|AAAAAAAACDDMAAAA|359|West |Blvd|Suite 380|Belmont|Platte County|WY|80191|United States|-7|condo| +49971|AAAAAAAADDDMAAAA|518|Cherry |Parkway|Suite J|Five Points|Taylor County|WI|56098|United States|-6|single family| +49972|AAAAAAAAEDDMAAAA|816|Main |Avenue|Suite N|Union|Wexford County|MI|48721|United States|-5|condo| +49973|AAAAAAAAFDDMAAAA|15|Willow |Wy|Suite 370|Concord|Carteret County|NC|24107|United States|-5|apartment| +49974|AAAAAAAAGDDMAAAA|761|North Chestnut|Street|Suite 300|Unionville|Lee County|VA|21711|United States|-5|single family| +49975|AAAAAAAAHDDMAAAA|167|Johnson Railroad|Lane|Suite F|Riverdale|Madera County|CA|99391|United States|-8|condo| +49976|AAAAAAAAIDDMAAAA|302|Maple Johnson|Ave|Suite A|Enterprise|Bolivar County|MS|51757|United States|-6|apartment| +49977|AAAAAAAAJDDMAAAA||Third |||Plainview|Archuleta County|CO|83683|United States||| +49978|AAAAAAAAKDDMAAAA|78|Chestnut |Court|Suite D|Oak Grove|Hudson County|NJ|08970|United States|-5|condo| +49979|AAAAAAAALDDMAAAA|21|Cedar Lincoln|Blvd|Suite 120|Union|Sarasota County|FL|38721|United States|-5|apartment| +49980|AAAAAAAAMDDMAAAA|598|Elm Miller|Dr.|Suite 240|Centerville|Fannin County|TX|70059|United States|-6|condo| +49981|AAAAAAAANDDMAAAA|508|Hillcrest 9th|Court|Suite 160|Springfield|Jefferson County|AL|39303|United States|-6|condo| +49982|AAAAAAAAODDMAAAA|632|Laurel Wilson|Road|Suite R|Highland|Bristol Bay Borough|AK|99454|United States|-9|single family| +49983|AAAAAAAAPDDMAAAA|403|South 7th|Way|Suite 70|Riverview|Malheur County|OR|99003|United States|-8|single family| +49984|AAAAAAAAAEDMAAAA|980|River |Boulevard|Suite 180|Mount Vernon|Lincoln County|TN|38482|United States|-6|single family| +49985|AAAAAAAABEDMAAAA|439|View |Ave|Suite F|Fairfield|Sherman County|OR|96192|United States|-8|condo| +49986|AAAAAAAACEDMAAAA||Park |Dr.|||Daviess County|MO||United States||condo| +49987|AAAAAAAADEDMAAAA|472|Wilson 8th|RD|Suite X|Mount Pleasant|Madison Parish|LA|71933|United States|-6|condo| +49988|AAAAAAAAEEDMAAAA|483|Lincoln |Circle|Suite 300|Lebanon|Sublette County|WY|82898|United States|-7|single family| +49989|AAAAAAAAFEDMAAAA|8|South |Dr.|Suite 0|Mount Vernon|Warren County|MS|58482|United States|-6|condo| +49990|AAAAAAAAGEDMAAAA|575|12th |Ct.|Suite J|Pleasant Valley|Aleutians West Census Area|AK|92477|United States|-9|single family| +49991|AAAAAAAAHEDMAAAA|926|Johnson Walnut|Circle|Suite Q|Green Acres|Kent County|MD|27683|United States|-5|apartment| +49992|AAAAAAAAIEDMAAAA|18|Park Cherry|Cir.|Suite R|Springfield|Martin County|TX|79303|United States|-6|single family| +49993|AAAAAAAAJEDMAAAA|18|Oak Oak|Cir.|Suite D|Salem|Jay County|IN|48048|United States|-5|condo| +49994|AAAAAAAAKEDMAAAA|75|Sixth |Ln|Suite A|Sulphur Springs|Cass County|MI|48354|United States|-5|single family| +49995|AAAAAAAALEDMAAAA|316|Ninth |Court|Suite 170|Clifton|Bibb County|AL|38014|United States|-6|single family| +49996|AAAAAAAAMEDMAAAA|338|1st |Avenue|Suite E|Lakewood|Clearfield County|PA|18877|United States|-5|single family| +49997|AAAAAAAANEDMAAAA|891|Park 2nd|Pkwy|Suite 140|Lebanon|Richland County|MT|62898|United States|-7|apartment| +49998|AAAAAAAAOEDMAAAA|345|Ridge |Avenue|Suite 90|Oak Hill|Humboldt County|CA|97838|United States|-8|single family| +49999|AAAAAAAAPEDMAAAA|259|Maple 7th|Drive|Suite 20|Greenfield|Alameda County|CA|95038|United States|-8|apartment| +50000|AAAAAAAAAFDMAAAA|570|Sixth 3rd|Ave|Suite 350|Stringtown|Durham County|NC|20162|United States|-5|condo| diff --git a/spark-connector/spark-common/src/test/resources/data/tpcds/customer_demographics/customer_demographics.dat b/spark-connector/spark-common/src/test/resources/data/tpcds/customer_demographics/customer_demographics.dat new file mode 100644 index 00000000000..cff933af88b --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/data/tpcds/customer_demographics/customer_demographics.dat @@ -0,0 +1,19208 @@ +1|M|M|Primary|500|Good|0|0|0| +2|F|M|Primary|500|Good|0|0|0| +3|M|S|Primary|500|Good|0|0|0| +4|F|S|Primary|500|Good|0|0|0| +5|M|D|Primary|500|Good|0|0|0| +6|F|D|Primary|500|Good|0|0|0| +7|M|W|Primary|500|Good|0|0|0| +8|F|W|Primary|500|Good|0|0|0| +9|M|U|Primary|500|Good|0|0|0| +10|F|U|Primary|500|Good|0|0|0| +11|M|M|Secondary|500|Good|0|0|0| +12|F|M|Secondary|500|Good|0|0|0| +13|M|S|Secondary|500|Good|0|0|0| +14|F|S|Secondary|500|Good|0|0|0| +15|M|D|Secondary|500|Good|0|0|0| +16|F|D|Secondary|500|Good|0|0|0| +17|M|W|Secondary|500|Good|0|0|0| +18|F|W|Secondary|500|Good|0|0|0| +19|M|U|Secondary|500|Good|0|0|0| +20|F|U|Secondary|500|Good|0|0|0| +21|M|M|College|500|Good|0|0|0| +22|F|M|College|500|Good|0|0|0| +23|M|S|College|500|Good|0|0|0| +24|F|S|College|500|Good|0|0|0| +25|M|D|College|500|Good|0|0|0| +26|F|D|College|500|Good|0|0|0| +27|M|W|College|500|Good|0|0|0| +28|F|W|College|500|Good|0|0|0| +29|M|U|College|500|Good|0|0|0| +30|F|U|College|500|Good|0|0|0| +31|M|M|2 yr Degree|500|Good|0|0|0| +32|F|M|2 yr Degree|500|Good|0|0|0| +33|M|S|2 yr Degree|500|Good|0|0|0| +34|F|S|2 yr Degree|500|Good|0|0|0| +35|M|D|2 yr Degree|500|Good|0|0|0| +36|F|D|2 yr Degree|500|Good|0|0|0| +37|M|W|2 yr Degree|500|Good|0|0|0| +38|F|W|2 yr Degree|500|Good|0|0|0| +39|M|U|2 yr Degree|500|Good|0|0|0| +40|F|U|2 yr Degree|500|Good|0|0|0| +41|M|M|4 yr Degree|500|Good|0|0|0| +42|F|M|4 yr Degree|500|Good|0|0|0| +43|M|S|4 yr Degree|500|Good|0|0|0| +44|F|S|4 yr Degree|500|Good|0|0|0| +45|M|D|4 yr Degree|500|Good|0|0|0| +46|F|D|4 yr Degree|500|Good|0|0|0| +47|M|W|4 yr Degree|500|Good|0|0|0| +48|F|W|4 yr Degree|500|Good|0|0|0| +49|M|U|4 yr Degree|500|Good|0|0|0| +50|F|U|4 yr Degree|500|Good|0|0|0| +51|M|M|Advanced Degree|500|Good|0|0|0| +52|F|M|Advanced Degree|500|Good|0|0|0| +53|M|S|Advanced Degree|500|Good|0|0|0| +54|F|S|Advanced Degree|500|Good|0|0|0| +55|M|D|Advanced Degree|500|Good|0|0|0| +56|F|D|Advanced Degree|500|Good|0|0|0| +57|M|W|Advanced Degree|500|Good|0|0|0| +58|F|W|Advanced Degree|500|Good|0|0|0| +59|M|U|Advanced Degree|500|Good|0|0|0| +60|F|U|Advanced Degree|500|Good|0|0|0| +61|M|M|Unknown|500|Good|0|0|0| +62|F|M|Unknown|500|Good|0|0|0| +63|M|S|Unknown|500|Good|0|0|0| +64|F|S|Unknown|500|Good|0|0|0| +65|M|D|Unknown|500|Good|0|0|0| +66|F|D|Unknown|500|Good|0|0|0| +67|M|W|Unknown|500|Good|0|0|0| +68|F|W|Unknown|500|Good|0|0|0| +69|M|U|Unknown|500|Good|0|0|0| +70|F|U|Unknown|500|Good|0|0|0| +71|M|M|Primary|1000|Good|0|0|0| +72|F|M|Primary|1000|Good|0|0|0| +73|M|S|Primary|1000|Good|0|0|0| +74|F|S|Primary|1000|Good|0|0|0| +75|M|D|Primary|1000|Good|0|0|0| +76|F|D|Primary|1000|Good|0|0|0| +77|M|W|Primary|1000|Good|0|0|0| +78|F|W|Primary|1000|Good|0|0|0| +79|M|U|Primary|1000|Good|0|0|0| +80|F|U|Primary|1000|Good|0|0|0| +81|M|M|Secondary|1000|Good|0|0|0| +82|F|M|Secondary|1000|Good|0|0|0| +83|M|S|Secondary|1000|Good|0|0|0| +84|F|S|Secondary|1000|Good|0|0|0| +85|M|D|Secondary|1000|Good|0|0|0| +86|F|D|Secondary|1000|Good|0|0|0| +87|M|W|Secondary|1000|Good|0|0|0| +88|F|W|Secondary|1000|Good|0|0|0| +89|M|U|Secondary|1000|Good|0|0|0| +90|F|U|Secondary|1000|Good|0|0|0| +91|M|M|College|1000|Good|0|0|0| +92|F|M|College|1000|Good|0|0|0| +93|M|S|College|1000|Good|0|0|0| +94|F|S|College|1000|Good|0|0|0| +95|M|D|College|1000|Good|0|0|0| +96|F|D|College|1000|Good|0|0|0| +97|M|W|College|1000|Good|0|0|0| +98|F|W|College|1000|Good|0|0|0| +99|M|U|College|1000|Good|0|0|0| +100|F|U|College|1000|Good|0|0|0| +101|M|M|2 yr Degree|1000|Good|0|0|0| +102|F|M|2 yr Degree|1000|Good|0|0|0| +103|M|S|2 yr Degree|1000|Good|0|0|0| +104|F|S|2 yr Degree|1000|Good|0|0|0| +105|M|D|2 yr Degree|1000|Good|0|0|0| +106|F|D|2 yr Degree|1000|Good|0|0|0| +107|M|W|2 yr Degree|1000|Good|0|0|0| +108|F|W|2 yr Degree|1000|Good|0|0|0| +109|M|U|2 yr Degree|1000|Good|0|0|0| +110|F|U|2 yr Degree|1000|Good|0|0|0| +111|M|M|4 yr Degree|1000|Good|0|0|0| +112|F|M|4 yr Degree|1000|Good|0|0|0| +113|M|S|4 yr Degree|1000|Good|0|0|0| +114|F|S|4 yr Degree|1000|Good|0|0|0| +115|M|D|4 yr Degree|1000|Good|0|0|0| +116|F|D|4 yr Degree|1000|Good|0|0|0| +117|M|W|4 yr Degree|1000|Good|0|0|0| +118|F|W|4 yr Degree|1000|Good|0|0|0| +119|M|U|4 yr Degree|1000|Good|0|0|0| +120|F|U|4 yr Degree|1000|Good|0|0|0| +121|M|M|Advanced Degree|1000|Good|0|0|0| +122|F|M|Advanced Degree|1000|Good|0|0|0| +123|M|S|Advanced Degree|1000|Good|0|0|0| +124|F|S|Advanced Degree|1000|Good|0|0|0| +125|M|D|Advanced Degree|1000|Good|0|0|0| +126|F|D|Advanced Degree|1000|Good|0|0|0| +127|M|W|Advanced Degree|1000|Good|0|0|0| +128|F|W|Advanced Degree|1000|Good|0|0|0| +129|M|U|Advanced Degree|1000|Good|0|0|0| +130|F|U|Advanced Degree|1000|Good|0|0|0| +131|M|M|Unknown|1000|Good|0|0|0| +132|F|M|Unknown|1000|Good|0|0|0| +133|M|S|Unknown|1000|Good|0|0|0| +134|F|S|Unknown|1000|Good|0|0|0| +135|M|D|Unknown|1000|Good|0|0|0| +136|F|D|Unknown|1000|Good|0|0|0| +137|M|W|Unknown|1000|Good|0|0|0| +138|F|W|Unknown|1000|Good|0|0|0| +139|M|U|Unknown|1000|Good|0|0|0| +140|F|U|Unknown|1000|Good|0|0|0| +141|M|M|Primary|1500|Good|0|0|0| +142|F|M|Primary|1500|Good|0|0|0| +143|M|S|Primary|1500|Good|0|0|0| +144|F|S|Primary|1500|Good|0|0|0| +145|M|D|Primary|1500|Good|0|0|0| +146|F|D|Primary|1500|Good|0|0|0| +147|M|W|Primary|1500|Good|0|0|0| +148|F|W|Primary|1500|Good|0|0|0| +149|M|U|Primary|1500|Good|0|0|0| +150|F|U|Primary|1500|Good|0|0|0| +151|M|M|Secondary|1500|Good|0|0|0| +152|F|M|Secondary|1500|Good|0|0|0| +153|M|S|Secondary|1500|Good|0|0|0| +154|F|S|Secondary|1500|Good|0|0|0| +155|M|D|Secondary|1500|Good|0|0|0| +156|F|D|Secondary|1500|Good|0|0|0| +157|M|W|Secondary|1500|Good|0|0|0| +158|F|W|Secondary|1500|Good|0|0|0| +159|M|U|Secondary|1500|Good|0|0|0| +160|F|U|Secondary|1500|Good|0|0|0| +161|M|M|College|1500|Good|0|0|0| +162|F|M|College|1500|Good|0|0|0| +163|M|S|College|1500|Good|0|0|0| +164|F|S|College|1500|Good|0|0|0| +165|M|D|College|1500|Good|0|0|0| +166|F|D|College|1500|Good|0|0|0| +167|M|W|College|1500|Good|0|0|0| +168|F|W|College|1500|Good|0|0|0| +169|M|U|College|1500|Good|0|0|0| +170|F|U|College|1500|Good|0|0|0| +171|M|M|2 yr Degree|1500|Good|0|0|0| +172|F|M|2 yr Degree|1500|Good|0|0|0| +173|M|S|2 yr Degree|1500|Good|0|0|0| +174|F|S|2 yr Degree|1500|Good|0|0|0| +175|M|D|2 yr Degree|1500|Good|0|0|0| +176|F|D|2 yr Degree|1500|Good|0|0|0| +177|M|W|2 yr Degree|1500|Good|0|0|0| +178|F|W|2 yr Degree|1500|Good|0|0|0| +179|M|U|2 yr Degree|1500|Good|0|0|0| +180|F|U|2 yr Degree|1500|Good|0|0|0| +181|M|M|4 yr Degree|1500|Good|0|0|0| +182|F|M|4 yr Degree|1500|Good|0|0|0| +183|M|S|4 yr Degree|1500|Good|0|0|0| +184|F|S|4 yr Degree|1500|Good|0|0|0| +185|M|D|4 yr Degree|1500|Good|0|0|0| +186|F|D|4 yr Degree|1500|Good|0|0|0| +187|M|W|4 yr Degree|1500|Good|0|0|0| +188|F|W|4 yr Degree|1500|Good|0|0|0| +189|M|U|4 yr Degree|1500|Good|0|0|0| +190|F|U|4 yr Degree|1500|Good|0|0|0| +191|M|M|Advanced Degree|1500|Good|0|0|0| +192|F|M|Advanced Degree|1500|Good|0|0|0| +193|M|S|Advanced Degree|1500|Good|0|0|0| +194|F|S|Advanced Degree|1500|Good|0|0|0| +195|M|D|Advanced Degree|1500|Good|0|0|0| +196|F|D|Advanced Degree|1500|Good|0|0|0| +197|M|W|Advanced Degree|1500|Good|0|0|0| +198|F|W|Advanced Degree|1500|Good|0|0|0| +199|M|U|Advanced Degree|1500|Good|0|0|0| +200|F|U|Advanced Degree|1500|Good|0|0|0| +201|M|M|Unknown|1500|Good|0|0|0| +202|F|M|Unknown|1500|Good|0|0|0| +203|M|S|Unknown|1500|Good|0|0|0| +204|F|S|Unknown|1500|Good|0|0|0| +205|M|D|Unknown|1500|Good|0|0|0| +206|F|D|Unknown|1500|Good|0|0|0| +207|M|W|Unknown|1500|Good|0|0|0| +208|F|W|Unknown|1500|Good|0|0|0| +209|M|U|Unknown|1500|Good|0|0|0| +210|F|U|Unknown|1500|Good|0|0|0| +211|M|M|Primary|2000|Good|0|0|0| +212|F|M|Primary|2000|Good|0|0|0| +213|M|S|Primary|2000|Good|0|0|0| +214|F|S|Primary|2000|Good|0|0|0| +215|M|D|Primary|2000|Good|0|0|0| +216|F|D|Primary|2000|Good|0|0|0| +217|M|W|Primary|2000|Good|0|0|0| +218|F|W|Primary|2000|Good|0|0|0| +219|M|U|Primary|2000|Good|0|0|0| +220|F|U|Primary|2000|Good|0|0|0| +221|M|M|Secondary|2000|Good|0|0|0| +222|F|M|Secondary|2000|Good|0|0|0| +223|M|S|Secondary|2000|Good|0|0|0| +224|F|S|Secondary|2000|Good|0|0|0| +225|M|D|Secondary|2000|Good|0|0|0| +226|F|D|Secondary|2000|Good|0|0|0| +227|M|W|Secondary|2000|Good|0|0|0| +228|F|W|Secondary|2000|Good|0|0|0| +229|M|U|Secondary|2000|Good|0|0|0| +230|F|U|Secondary|2000|Good|0|0|0| +231|M|M|College|2000|Good|0|0|0| +232|F|M|College|2000|Good|0|0|0| +233|M|S|College|2000|Good|0|0|0| +234|F|S|College|2000|Good|0|0|0| +235|M|D|College|2000|Good|0|0|0| +236|F|D|College|2000|Good|0|0|0| +237|M|W|College|2000|Good|0|0|0| +238|F|W|College|2000|Good|0|0|0| +239|M|U|College|2000|Good|0|0|0| +240|F|U|College|2000|Good|0|0|0| +241|M|M|2 yr Degree|2000|Good|0|0|0| +242|F|M|2 yr Degree|2000|Good|0|0|0| +243|M|S|2 yr Degree|2000|Good|0|0|0| +244|F|S|2 yr Degree|2000|Good|0|0|0| +245|M|D|2 yr Degree|2000|Good|0|0|0| +246|F|D|2 yr Degree|2000|Good|0|0|0| +247|M|W|2 yr Degree|2000|Good|0|0|0| +248|F|W|2 yr Degree|2000|Good|0|0|0| +249|M|U|2 yr Degree|2000|Good|0|0|0| +250|F|U|2 yr Degree|2000|Good|0|0|0| +251|M|M|4 yr Degree|2000|Good|0|0|0| +252|F|M|4 yr Degree|2000|Good|0|0|0| +253|M|S|4 yr Degree|2000|Good|0|0|0| +254|F|S|4 yr Degree|2000|Good|0|0|0| +255|M|D|4 yr Degree|2000|Good|0|0|0| +256|F|D|4 yr Degree|2000|Good|0|0|0| +257|M|W|4 yr Degree|2000|Good|0|0|0| +258|F|W|4 yr Degree|2000|Good|0|0|0| +259|M|U|4 yr Degree|2000|Good|0|0|0| +260|F|U|4 yr Degree|2000|Good|0|0|0| +261|M|M|Advanced Degree|2000|Good|0|0|0| +262|F|M|Advanced Degree|2000|Good|0|0|0| +263|M|S|Advanced Degree|2000|Good|0|0|0| +264|F|S|Advanced Degree|2000|Good|0|0|0| +265|M|D|Advanced Degree|2000|Good|0|0|0| +266|F|D|Advanced Degree|2000|Good|0|0|0| +267|M|W|Advanced Degree|2000|Good|0|0|0| +268|F|W|Advanced Degree|2000|Good|0|0|0| +269|M|U|Advanced Degree|2000|Good|0|0|0| +270|F|U|Advanced Degree|2000|Good|0|0|0| +271|M|M|Unknown|2000|Good|0|0|0| +272|F|M|Unknown|2000|Good|0|0|0| +273|M|S|Unknown|2000|Good|0|0|0| +274|F|S|Unknown|2000|Good|0|0|0| +275|M|D|Unknown|2000|Good|0|0|0| +276|F|D|Unknown|2000|Good|0|0|0| +277|M|W|Unknown|2000|Good|0|0|0| +278|F|W|Unknown|2000|Good|0|0|0| +279|M|U|Unknown|2000|Good|0|0|0| +280|F|U|Unknown|2000|Good|0|0|0| +281|M|M|Primary|2500|Good|0|0|0| +282|F|M|Primary|2500|Good|0|0|0| +283|M|S|Primary|2500|Good|0|0|0| +284|F|S|Primary|2500|Good|0|0|0| +285|M|D|Primary|2500|Good|0|0|0| +286|F|D|Primary|2500|Good|0|0|0| +287|M|W|Primary|2500|Good|0|0|0| +288|F|W|Primary|2500|Good|0|0|0| +289|M|U|Primary|2500|Good|0|0|0| +290|F|U|Primary|2500|Good|0|0|0| +291|M|M|Secondary|2500|Good|0|0|0| +292|F|M|Secondary|2500|Good|0|0|0| +293|M|S|Secondary|2500|Good|0|0|0| +294|F|S|Secondary|2500|Good|0|0|0| +295|M|D|Secondary|2500|Good|0|0|0| +296|F|D|Secondary|2500|Good|0|0|0| +297|M|W|Secondary|2500|Good|0|0|0| +298|F|W|Secondary|2500|Good|0|0|0| +299|M|U|Secondary|2500|Good|0|0|0| +300|F|U|Secondary|2500|Good|0|0|0| +301|M|M|College|2500|Good|0|0|0| +302|F|M|College|2500|Good|0|0|0| +303|M|S|College|2500|Good|0|0|0| +304|F|S|College|2500|Good|0|0|0| +305|M|D|College|2500|Good|0|0|0| +306|F|D|College|2500|Good|0|0|0| +307|M|W|College|2500|Good|0|0|0| +308|F|W|College|2500|Good|0|0|0| +309|M|U|College|2500|Good|0|0|0| +310|F|U|College|2500|Good|0|0|0| +311|M|M|2 yr Degree|2500|Good|0|0|0| +312|F|M|2 yr Degree|2500|Good|0|0|0| +313|M|S|2 yr Degree|2500|Good|0|0|0| +314|F|S|2 yr Degree|2500|Good|0|0|0| +315|M|D|2 yr Degree|2500|Good|0|0|0| +316|F|D|2 yr Degree|2500|Good|0|0|0| +317|M|W|2 yr Degree|2500|Good|0|0|0| +318|F|W|2 yr Degree|2500|Good|0|0|0| +319|M|U|2 yr Degree|2500|Good|0|0|0| +320|F|U|2 yr Degree|2500|Good|0|0|0| +321|M|M|4 yr Degree|2500|Good|0|0|0| +322|F|M|4 yr Degree|2500|Good|0|0|0| +323|M|S|4 yr Degree|2500|Good|0|0|0| +324|F|S|4 yr Degree|2500|Good|0|0|0| +325|M|D|4 yr Degree|2500|Good|0|0|0| +326|F|D|4 yr Degree|2500|Good|0|0|0| +327|M|W|4 yr Degree|2500|Good|0|0|0| +328|F|W|4 yr Degree|2500|Good|0|0|0| +329|M|U|4 yr Degree|2500|Good|0|0|0| +330|F|U|4 yr Degree|2500|Good|0|0|0| +331|M|M|Advanced Degree|2500|Good|0|0|0| +332|F|M|Advanced Degree|2500|Good|0|0|0| +333|M|S|Advanced Degree|2500|Good|0|0|0| +334|F|S|Advanced Degree|2500|Good|0|0|0| +335|M|D|Advanced Degree|2500|Good|0|0|0| +336|F|D|Advanced Degree|2500|Good|0|0|0| +337|M|W|Advanced Degree|2500|Good|0|0|0| +338|F|W|Advanced Degree|2500|Good|0|0|0| +339|M|U|Advanced Degree|2500|Good|0|0|0| +340|F|U|Advanced Degree|2500|Good|0|0|0| +341|M|M|Unknown|2500|Good|0|0|0| +342|F|M|Unknown|2500|Good|0|0|0| +343|M|S|Unknown|2500|Good|0|0|0| +344|F|S|Unknown|2500|Good|0|0|0| +345|M|D|Unknown|2500|Good|0|0|0| +346|F|D|Unknown|2500|Good|0|0|0| +347|M|W|Unknown|2500|Good|0|0|0| +348|F|W|Unknown|2500|Good|0|0|0| +349|M|U|Unknown|2500|Good|0|0|0| +350|F|U|Unknown|2500|Good|0|0|0| +351|M|M|Primary|3000|Good|0|0|0| +352|F|M|Primary|3000|Good|0|0|0| +353|M|S|Primary|3000|Good|0|0|0| +354|F|S|Primary|3000|Good|0|0|0| +355|M|D|Primary|3000|Good|0|0|0| +356|F|D|Primary|3000|Good|0|0|0| +357|M|W|Primary|3000|Good|0|0|0| +358|F|W|Primary|3000|Good|0|0|0| +359|M|U|Primary|3000|Good|0|0|0| +360|F|U|Primary|3000|Good|0|0|0| +361|M|M|Secondary|3000|Good|0|0|0| +362|F|M|Secondary|3000|Good|0|0|0| +363|M|S|Secondary|3000|Good|0|0|0| +364|F|S|Secondary|3000|Good|0|0|0| +365|M|D|Secondary|3000|Good|0|0|0| +366|F|D|Secondary|3000|Good|0|0|0| +367|M|W|Secondary|3000|Good|0|0|0| +368|F|W|Secondary|3000|Good|0|0|0| +369|M|U|Secondary|3000|Good|0|0|0| +370|F|U|Secondary|3000|Good|0|0|0| +371|M|M|College|3000|Good|0|0|0| +372|F|M|College|3000|Good|0|0|0| +373|M|S|College|3000|Good|0|0|0| +374|F|S|College|3000|Good|0|0|0| +375|M|D|College|3000|Good|0|0|0| +376|F|D|College|3000|Good|0|0|0| +377|M|W|College|3000|Good|0|0|0| +378|F|W|College|3000|Good|0|0|0| +379|M|U|College|3000|Good|0|0|0| +380|F|U|College|3000|Good|0|0|0| +381|M|M|2 yr Degree|3000|Good|0|0|0| +382|F|M|2 yr Degree|3000|Good|0|0|0| +383|M|S|2 yr Degree|3000|Good|0|0|0| +384|F|S|2 yr Degree|3000|Good|0|0|0| +385|M|D|2 yr Degree|3000|Good|0|0|0| +386|F|D|2 yr Degree|3000|Good|0|0|0| +387|M|W|2 yr Degree|3000|Good|0|0|0| +388|F|W|2 yr Degree|3000|Good|0|0|0| +389|M|U|2 yr Degree|3000|Good|0|0|0| +390|F|U|2 yr Degree|3000|Good|0|0|0| +391|M|M|4 yr Degree|3000|Good|0|0|0| +392|F|M|4 yr Degree|3000|Good|0|0|0| +393|M|S|4 yr Degree|3000|Good|0|0|0| +394|F|S|4 yr Degree|3000|Good|0|0|0| +395|M|D|4 yr Degree|3000|Good|0|0|0| +396|F|D|4 yr Degree|3000|Good|0|0|0| +397|M|W|4 yr Degree|3000|Good|0|0|0| +398|F|W|4 yr Degree|3000|Good|0|0|0| +399|M|U|4 yr Degree|3000|Good|0|0|0| +400|F|U|4 yr Degree|3000|Good|0|0|0| +401|M|M|Advanced Degree|3000|Good|0|0|0| +402|F|M|Advanced Degree|3000|Good|0|0|0| +403|M|S|Advanced Degree|3000|Good|0|0|0| +404|F|S|Advanced Degree|3000|Good|0|0|0| +405|M|D|Advanced Degree|3000|Good|0|0|0| +406|F|D|Advanced Degree|3000|Good|0|0|0| +407|M|W|Advanced Degree|3000|Good|0|0|0| +408|F|W|Advanced Degree|3000|Good|0|0|0| +409|M|U|Advanced Degree|3000|Good|0|0|0| +410|F|U|Advanced Degree|3000|Good|0|0|0| +411|M|M|Unknown|3000|Good|0|0|0| +412|F|M|Unknown|3000|Good|0|0|0| +413|M|S|Unknown|3000|Good|0|0|0| +414|F|S|Unknown|3000|Good|0|0|0| +415|M|D|Unknown|3000|Good|0|0|0| +416|F|D|Unknown|3000|Good|0|0|0| +417|M|W|Unknown|3000|Good|0|0|0| +418|F|W|Unknown|3000|Good|0|0|0| +419|M|U|Unknown|3000|Good|0|0|0| +420|F|U|Unknown|3000|Good|0|0|0| +421|M|M|Primary|3500|Good|0|0|0| +422|F|M|Primary|3500|Good|0|0|0| +423|M|S|Primary|3500|Good|0|0|0| +424|F|S|Primary|3500|Good|0|0|0| +425|M|D|Primary|3500|Good|0|0|0| +426|F|D|Primary|3500|Good|0|0|0| +427|M|W|Primary|3500|Good|0|0|0| +428|F|W|Primary|3500|Good|0|0|0| +429|M|U|Primary|3500|Good|0|0|0| +430|F|U|Primary|3500|Good|0|0|0| +431|M|M|Secondary|3500|Good|0|0|0| +432|F|M|Secondary|3500|Good|0|0|0| +433|M|S|Secondary|3500|Good|0|0|0| +434|F|S|Secondary|3500|Good|0|0|0| +435|M|D|Secondary|3500|Good|0|0|0| +436|F|D|Secondary|3500|Good|0|0|0| +437|M|W|Secondary|3500|Good|0|0|0| +438|F|W|Secondary|3500|Good|0|0|0| +439|M|U|Secondary|3500|Good|0|0|0| +440|F|U|Secondary|3500|Good|0|0|0| +441|M|M|College|3500|Good|0|0|0| +442|F|M|College|3500|Good|0|0|0| +443|M|S|College|3500|Good|0|0|0| +444|F|S|College|3500|Good|0|0|0| +445|M|D|College|3500|Good|0|0|0| +446|F|D|College|3500|Good|0|0|0| +447|M|W|College|3500|Good|0|0|0| +448|F|W|College|3500|Good|0|0|0| +449|M|U|College|3500|Good|0|0|0| +450|F|U|College|3500|Good|0|0|0| +451|M|M|2 yr Degree|3500|Good|0|0|0| +452|F|M|2 yr Degree|3500|Good|0|0|0| +453|M|S|2 yr Degree|3500|Good|0|0|0| +454|F|S|2 yr Degree|3500|Good|0|0|0| +455|M|D|2 yr Degree|3500|Good|0|0|0| +456|F|D|2 yr Degree|3500|Good|0|0|0| +457|M|W|2 yr Degree|3500|Good|0|0|0| +458|F|W|2 yr Degree|3500|Good|0|0|0| +459|M|U|2 yr Degree|3500|Good|0|0|0| +460|F|U|2 yr Degree|3500|Good|0|0|0| +461|M|M|4 yr Degree|3500|Good|0|0|0| +462|F|M|4 yr Degree|3500|Good|0|0|0| +463|M|S|4 yr Degree|3500|Good|0|0|0| +464|F|S|4 yr Degree|3500|Good|0|0|0| +465|M|D|4 yr Degree|3500|Good|0|0|0| +466|F|D|4 yr Degree|3500|Good|0|0|0| +467|M|W|4 yr Degree|3500|Good|0|0|0| +468|F|W|4 yr Degree|3500|Good|0|0|0| +469|M|U|4 yr Degree|3500|Good|0|0|0| +470|F|U|4 yr Degree|3500|Good|0|0|0| +471|M|M|Advanced Degree|3500|Good|0|0|0| +472|F|M|Advanced Degree|3500|Good|0|0|0| +473|M|S|Advanced Degree|3500|Good|0|0|0| +474|F|S|Advanced Degree|3500|Good|0|0|0| +475|M|D|Advanced Degree|3500|Good|0|0|0| +476|F|D|Advanced Degree|3500|Good|0|0|0| +477|M|W|Advanced Degree|3500|Good|0|0|0| +478|F|W|Advanced Degree|3500|Good|0|0|0| +479|M|U|Advanced Degree|3500|Good|0|0|0| +480|F|U|Advanced Degree|3500|Good|0|0|0| +481|M|M|Unknown|3500|Good|0|0|0| +482|F|M|Unknown|3500|Good|0|0|0| +483|M|S|Unknown|3500|Good|0|0|0| +484|F|S|Unknown|3500|Good|0|0|0| +485|M|D|Unknown|3500|Good|0|0|0| +486|F|D|Unknown|3500|Good|0|0|0| +487|M|W|Unknown|3500|Good|0|0|0| +488|F|W|Unknown|3500|Good|0|0|0| +489|M|U|Unknown|3500|Good|0|0|0| +490|F|U|Unknown|3500|Good|0|0|0| +491|M|M|Primary|4000|Good|0|0|0| +492|F|M|Primary|4000|Good|0|0|0| +493|M|S|Primary|4000|Good|0|0|0| +494|F|S|Primary|4000|Good|0|0|0| +495|M|D|Primary|4000|Good|0|0|0| +496|F|D|Primary|4000|Good|0|0|0| +497|M|W|Primary|4000|Good|0|0|0| +498|F|W|Primary|4000|Good|0|0|0| +499|M|U|Primary|4000|Good|0|0|0| +500|F|U|Primary|4000|Good|0|0|0| +501|M|M|Secondary|4000|Good|0|0|0| +502|F|M|Secondary|4000|Good|0|0|0| +503|M|S|Secondary|4000|Good|0|0|0| +504|F|S|Secondary|4000|Good|0|0|0| +505|M|D|Secondary|4000|Good|0|0|0| +506|F|D|Secondary|4000|Good|0|0|0| +507|M|W|Secondary|4000|Good|0|0|0| +508|F|W|Secondary|4000|Good|0|0|0| +509|M|U|Secondary|4000|Good|0|0|0| +510|F|U|Secondary|4000|Good|0|0|0| +511|M|M|College|4000|Good|0|0|0| +512|F|M|College|4000|Good|0|0|0| +513|M|S|College|4000|Good|0|0|0| +514|F|S|College|4000|Good|0|0|0| +515|M|D|College|4000|Good|0|0|0| +516|F|D|College|4000|Good|0|0|0| +517|M|W|College|4000|Good|0|0|0| +518|F|W|College|4000|Good|0|0|0| +519|M|U|College|4000|Good|0|0|0| +520|F|U|College|4000|Good|0|0|0| +521|M|M|2 yr Degree|4000|Good|0|0|0| +522|F|M|2 yr Degree|4000|Good|0|0|0| +523|M|S|2 yr Degree|4000|Good|0|0|0| +524|F|S|2 yr Degree|4000|Good|0|0|0| +525|M|D|2 yr Degree|4000|Good|0|0|0| +526|F|D|2 yr Degree|4000|Good|0|0|0| +527|M|W|2 yr Degree|4000|Good|0|0|0| +528|F|W|2 yr Degree|4000|Good|0|0|0| +529|M|U|2 yr Degree|4000|Good|0|0|0| +530|F|U|2 yr Degree|4000|Good|0|0|0| +531|M|M|4 yr Degree|4000|Good|0|0|0| +532|F|M|4 yr Degree|4000|Good|0|0|0| +533|M|S|4 yr Degree|4000|Good|0|0|0| +534|F|S|4 yr Degree|4000|Good|0|0|0| +535|M|D|4 yr Degree|4000|Good|0|0|0| +536|F|D|4 yr Degree|4000|Good|0|0|0| +537|M|W|4 yr Degree|4000|Good|0|0|0| +538|F|W|4 yr Degree|4000|Good|0|0|0| +539|M|U|4 yr Degree|4000|Good|0|0|0| +540|F|U|4 yr Degree|4000|Good|0|0|0| +541|M|M|Advanced Degree|4000|Good|0|0|0| +542|F|M|Advanced Degree|4000|Good|0|0|0| +543|M|S|Advanced Degree|4000|Good|0|0|0| +544|F|S|Advanced Degree|4000|Good|0|0|0| +545|M|D|Advanced Degree|4000|Good|0|0|0| +546|F|D|Advanced Degree|4000|Good|0|0|0| +547|M|W|Advanced Degree|4000|Good|0|0|0| +548|F|W|Advanced Degree|4000|Good|0|0|0| +549|M|U|Advanced Degree|4000|Good|0|0|0| +550|F|U|Advanced Degree|4000|Good|0|0|0| +551|M|M|Unknown|4000|Good|0|0|0| +552|F|M|Unknown|4000|Good|0|0|0| +553|M|S|Unknown|4000|Good|0|0|0| +554|F|S|Unknown|4000|Good|0|0|0| +555|M|D|Unknown|4000|Good|0|0|0| +556|F|D|Unknown|4000|Good|0|0|0| +557|M|W|Unknown|4000|Good|0|0|0| +558|F|W|Unknown|4000|Good|0|0|0| +559|M|U|Unknown|4000|Good|0|0|0| +560|F|U|Unknown|4000|Good|0|0|0| +561|M|M|Primary|4500|Good|0|0|0| +562|F|M|Primary|4500|Good|0|0|0| +563|M|S|Primary|4500|Good|0|0|0| +564|F|S|Primary|4500|Good|0|0|0| +565|M|D|Primary|4500|Good|0|0|0| +566|F|D|Primary|4500|Good|0|0|0| +567|M|W|Primary|4500|Good|0|0|0| +568|F|W|Primary|4500|Good|0|0|0| +569|M|U|Primary|4500|Good|0|0|0| +570|F|U|Primary|4500|Good|0|0|0| +571|M|M|Secondary|4500|Good|0|0|0| +572|F|M|Secondary|4500|Good|0|0|0| +573|M|S|Secondary|4500|Good|0|0|0| +574|F|S|Secondary|4500|Good|0|0|0| +575|M|D|Secondary|4500|Good|0|0|0| +576|F|D|Secondary|4500|Good|0|0|0| +577|M|W|Secondary|4500|Good|0|0|0| +578|F|W|Secondary|4500|Good|0|0|0| +579|M|U|Secondary|4500|Good|0|0|0| +580|F|U|Secondary|4500|Good|0|0|0| +581|M|M|College|4500|Good|0|0|0| +582|F|M|College|4500|Good|0|0|0| +583|M|S|College|4500|Good|0|0|0| +584|F|S|College|4500|Good|0|0|0| +585|M|D|College|4500|Good|0|0|0| +586|F|D|College|4500|Good|0|0|0| +587|M|W|College|4500|Good|0|0|0| +588|F|W|College|4500|Good|0|0|0| +589|M|U|College|4500|Good|0|0|0| +590|F|U|College|4500|Good|0|0|0| +591|M|M|2 yr Degree|4500|Good|0|0|0| +592|F|M|2 yr Degree|4500|Good|0|0|0| +593|M|S|2 yr Degree|4500|Good|0|0|0| +594|F|S|2 yr Degree|4500|Good|0|0|0| +595|M|D|2 yr Degree|4500|Good|0|0|0| +596|F|D|2 yr Degree|4500|Good|0|0|0| +597|M|W|2 yr Degree|4500|Good|0|0|0| +598|F|W|2 yr Degree|4500|Good|0|0|0| +599|M|U|2 yr Degree|4500|Good|0|0|0| +600|F|U|2 yr Degree|4500|Good|0|0|0| +601|M|M|4 yr Degree|4500|Good|0|0|0| +602|F|M|4 yr Degree|4500|Good|0|0|0| +603|M|S|4 yr Degree|4500|Good|0|0|0| +604|F|S|4 yr Degree|4500|Good|0|0|0| +605|M|D|4 yr Degree|4500|Good|0|0|0| +606|F|D|4 yr Degree|4500|Good|0|0|0| +607|M|W|4 yr Degree|4500|Good|0|0|0| +608|F|W|4 yr Degree|4500|Good|0|0|0| +609|M|U|4 yr Degree|4500|Good|0|0|0| +610|F|U|4 yr Degree|4500|Good|0|0|0| +611|M|M|Advanced Degree|4500|Good|0|0|0| +612|F|M|Advanced Degree|4500|Good|0|0|0| +613|M|S|Advanced Degree|4500|Good|0|0|0| +614|F|S|Advanced Degree|4500|Good|0|0|0| +615|M|D|Advanced Degree|4500|Good|0|0|0| +616|F|D|Advanced Degree|4500|Good|0|0|0| +617|M|W|Advanced Degree|4500|Good|0|0|0| +618|F|W|Advanced Degree|4500|Good|0|0|0| +619|M|U|Advanced Degree|4500|Good|0|0|0| +620|F|U|Advanced Degree|4500|Good|0|0|0| +621|M|M|Unknown|4500|Good|0|0|0| +622|F|M|Unknown|4500|Good|0|0|0| +623|M|S|Unknown|4500|Good|0|0|0| +624|F|S|Unknown|4500|Good|0|0|0| +625|M|D|Unknown|4500|Good|0|0|0| +626|F|D|Unknown|4500|Good|0|0|0| +627|M|W|Unknown|4500|Good|0|0|0| +628|F|W|Unknown|4500|Good|0|0|0| +629|M|U|Unknown|4500|Good|0|0|0| +630|F|U|Unknown|4500|Good|0|0|0| +631|M|M|Primary|5000|Good|0|0|0| +632|F|M|Primary|5000|Good|0|0|0| +633|M|S|Primary|5000|Good|0|0|0| +634|F|S|Primary|5000|Good|0|0|0| +635|M|D|Primary|5000|Good|0|0|0| +636|F|D|Primary|5000|Good|0|0|0| +637|M|W|Primary|5000|Good|0|0|0| +638|F|W|Primary|5000|Good|0|0|0| +639|M|U|Primary|5000|Good|0|0|0| +640|F|U|Primary|5000|Good|0|0|0| +641|M|M|Secondary|5000|Good|0|0|0| +642|F|M|Secondary|5000|Good|0|0|0| +643|M|S|Secondary|5000|Good|0|0|0| +644|F|S|Secondary|5000|Good|0|0|0| +645|M|D|Secondary|5000|Good|0|0|0| +646|F|D|Secondary|5000|Good|0|0|0| +647|M|W|Secondary|5000|Good|0|0|0| +648|F|W|Secondary|5000|Good|0|0|0| +649|M|U|Secondary|5000|Good|0|0|0| +650|F|U|Secondary|5000|Good|0|0|0| +651|M|M|College|5000|Good|0|0|0| +652|F|M|College|5000|Good|0|0|0| +653|M|S|College|5000|Good|0|0|0| +654|F|S|College|5000|Good|0|0|0| +655|M|D|College|5000|Good|0|0|0| +656|F|D|College|5000|Good|0|0|0| +657|M|W|College|5000|Good|0|0|0| +658|F|W|College|5000|Good|0|0|0| +659|M|U|College|5000|Good|0|0|0| +660|F|U|College|5000|Good|0|0|0| +661|M|M|2 yr Degree|5000|Good|0|0|0| +662|F|M|2 yr Degree|5000|Good|0|0|0| +663|M|S|2 yr Degree|5000|Good|0|0|0| +664|F|S|2 yr Degree|5000|Good|0|0|0| +665|M|D|2 yr Degree|5000|Good|0|0|0| +666|F|D|2 yr Degree|5000|Good|0|0|0| +667|M|W|2 yr Degree|5000|Good|0|0|0| +668|F|W|2 yr Degree|5000|Good|0|0|0| +669|M|U|2 yr Degree|5000|Good|0|0|0| +670|F|U|2 yr Degree|5000|Good|0|0|0| +671|M|M|4 yr Degree|5000|Good|0|0|0| +672|F|M|4 yr Degree|5000|Good|0|0|0| +673|M|S|4 yr Degree|5000|Good|0|0|0| +674|F|S|4 yr Degree|5000|Good|0|0|0| +675|M|D|4 yr Degree|5000|Good|0|0|0| +676|F|D|4 yr Degree|5000|Good|0|0|0| +677|M|W|4 yr Degree|5000|Good|0|0|0| +678|F|W|4 yr Degree|5000|Good|0|0|0| +679|M|U|4 yr Degree|5000|Good|0|0|0| +680|F|U|4 yr Degree|5000|Good|0|0|0| +681|M|M|Advanced Degree|5000|Good|0|0|0| +682|F|M|Advanced Degree|5000|Good|0|0|0| +683|M|S|Advanced Degree|5000|Good|0|0|0| +684|F|S|Advanced Degree|5000|Good|0|0|0| +685|M|D|Advanced Degree|5000|Good|0|0|0| +686|F|D|Advanced Degree|5000|Good|0|0|0| +687|M|W|Advanced Degree|5000|Good|0|0|0| +688|F|W|Advanced Degree|5000|Good|0|0|0| +689|M|U|Advanced Degree|5000|Good|0|0|0| +690|F|U|Advanced Degree|5000|Good|0|0|0| +691|M|M|Unknown|5000|Good|0|0|0| +692|F|M|Unknown|5000|Good|0|0|0| +693|M|S|Unknown|5000|Good|0|0|0| +694|F|S|Unknown|5000|Good|0|0|0| +695|M|D|Unknown|5000|Good|0|0|0| +696|F|D|Unknown|5000|Good|0|0|0| +697|M|W|Unknown|5000|Good|0|0|0| +698|F|W|Unknown|5000|Good|0|0|0| +699|M|U|Unknown|5000|Good|0|0|0| +700|F|U|Unknown|5000|Good|0|0|0| +701|M|M|Primary|5500|Good|0|0|0| +702|F|M|Primary|5500|Good|0|0|0| +703|M|S|Primary|5500|Good|0|0|0| +704|F|S|Primary|5500|Good|0|0|0| +705|M|D|Primary|5500|Good|0|0|0| +706|F|D|Primary|5500|Good|0|0|0| +707|M|W|Primary|5500|Good|0|0|0| +708|F|W|Primary|5500|Good|0|0|0| +709|M|U|Primary|5500|Good|0|0|0| +710|F|U|Primary|5500|Good|0|0|0| +711|M|M|Secondary|5500|Good|0|0|0| +712|F|M|Secondary|5500|Good|0|0|0| +713|M|S|Secondary|5500|Good|0|0|0| +714|F|S|Secondary|5500|Good|0|0|0| +715|M|D|Secondary|5500|Good|0|0|0| +716|F|D|Secondary|5500|Good|0|0|0| +717|M|W|Secondary|5500|Good|0|0|0| +718|F|W|Secondary|5500|Good|0|0|0| +719|M|U|Secondary|5500|Good|0|0|0| +720|F|U|Secondary|5500|Good|0|0|0| +721|M|M|College|5500|Good|0|0|0| +722|F|M|College|5500|Good|0|0|0| +723|M|S|College|5500|Good|0|0|0| +724|F|S|College|5500|Good|0|0|0| +725|M|D|College|5500|Good|0|0|0| +726|F|D|College|5500|Good|0|0|0| +727|M|W|College|5500|Good|0|0|0| +728|F|W|College|5500|Good|0|0|0| +729|M|U|College|5500|Good|0|0|0| +730|F|U|College|5500|Good|0|0|0| +731|M|M|2 yr Degree|5500|Good|0|0|0| +732|F|M|2 yr Degree|5500|Good|0|0|0| +733|M|S|2 yr Degree|5500|Good|0|0|0| +734|F|S|2 yr Degree|5500|Good|0|0|0| +735|M|D|2 yr Degree|5500|Good|0|0|0| +736|F|D|2 yr Degree|5500|Good|0|0|0| +737|M|W|2 yr Degree|5500|Good|0|0|0| +738|F|W|2 yr Degree|5500|Good|0|0|0| +739|M|U|2 yr Degree|5500|Good|0|0|0| +740|F|U|2 yr Degree|5500|Good|0|0|0| +741|M|M|4 yr Degree|5500|Good|0|0|0| +742|F|M|4 yr Degree|5500|Good|0|0|0| +743|M|S|4 yr Degree|5500|Good|0|0|0| +744|F|S|4 yr Degree|5500|Good|0|0|0| +745|M|D|4 yr Degree|5500|Good|0|0|0| +746|F|D|4 yr Degree|5500|Good|0|0|0| +747|M|W|4 yr Degree|5500|Good|0|0|0| +748|F|W|4 yr Degree|5500|Good|0|0|0| +749|M|U|4 yr Degree|5500|Good|0|0|0| +750|F|U|4 yr Degree|5500|Good|0|0|0| +751|M|M|Advanced Degree|5500|Good|0|0|0| +752|F|M|Advanced Degree|5500|Good|0|0|0| +753|M|S|Advanced Degree|5500|Good|0|0|0| +754|F|S|Advanced Degree|5500|Good|0|0|0| +755|M|D|Advanced Degree|5500|Good|0|0|0| +756|F|D|Advanced Degree|5500|Good|0|0|0| +757|M|W|Advanced Degree|5500|Good|0|0|0| +758|F|W|Advanced Degree|5500|Good|0|0|0| +759|M|U|Advanced Degree|5500|Good|0|0|0| +760|F|U|Advanced Degree|5500|Good|0|0|0| +761|M|M|Unknown|5500|Good|0|0|0| +762|F|M|Unknown|5500|Good|0|0|0| +763|M|S|Unknown|5500|Good|0|0|0| +764|F|S|Unknown|5500|Good|0|0|0| +765|M|D|Unknown|5500|Good|0|0|0| +766|F|D|Unknown|5500|Good|0|0|0| +767|M|W|Unknown|5500|Good|0|0|0| +768|F|W|Unknown|5500|Good|0|0|0| +769|M|U|Unknown|5500|Good|0|0|0| +770|F|U|Unknown|5500|Good|0|0|0| +771|M|M|Primary|6000|Good|0|0|0| +772|F|M|Primary|6000|Good|0|0|0| +773|M|S|Primary|6000|Good|0|0|0| +774|F|S|Primary|6000|Good|0|0|0| +775|M|D|Primary|6000|Good|0|0|0| +776|F|D|Primary|6000|Good|0|0|0| +777|M|W|Primary|6000|Good|0|0|0| +778|F|W|Primary|6000|Good|0|0|0| +779|M|U|Primary|6000|Good|0|0|0| +780|F|U|Primary|6000|Good|0|0|0| +781|M|M|Secondary|6000|Good|0|0|0| +782|F|M|Secondary|6000|Good|0|0|0| +783|M|S|Secondary|6000|Good|0|0|0| +784|F|S|Secondary|6000|Good|0|0|0| +785|M|D|Secondary|6000|Good|0|0|0| +786|F|D|Secondary|6000|Good|0|0|0| +787|M|W|Secondary|6000|Good|0|0|0| +788|F|W|Secondary|6000|Good|0|0|0| +789|M|U|Secondary|6000|Good|0|0|0| +790|F|U|Secondary|6000|Good|0|0|0| +791|M|M|College|6000|Good|0|0|0| +792|F|M|College|6000|Good|0|0|0| +793|M|S|College|6000|Good|0|0|0| +794|F|S|College|6000|Good|0|0|0| +795|M|D|College|6000|Good|0|0|0| +796|F|D|College|6000|Good|0|0|0| +797|M|W|College|6000|Good|0|0|0| +798|F|W|College|6000|Good|0|0|0| +799|M|U|College|6000|Good|0|0|0| +800|F|U|College|6000|Good|0|0|0| +801|M|M|2 yr Degree|6000|Good|0|0|0| +802|F|M|2 yr Degree|6000|Good|0|0|0| +803|M|S|2 yr Degree|6000|Good|0|0|0| +804|F|S|2 yr Degree|6000|Good|0|0|0| +805|M|D|2 yr Degree|6000|Good|0|0|0| +806|F|D|2 yr Degree|6000|Good|0|0|0| +807|M|W|2 yr Degree|6000|Good|0|0|0| +808|F|W|2 yr Degree|6000|Good|0|0|0| +809|M|U|2 yr Degree|6000|Good|0|0|0| +810|F|U|2 yr Degree|6000|Good|0|0|0| +811|M|M|4 yr Degree|6000|Good|0|0|0| +812|F|M|4 yr Degree|6000|Good|0|0|0| +813|M|S|4 yr Degree|6000|Good|0|0|0| +814|F|S|4 yr Degree|6000|Good|0|0|0| +815|M|D|4 yr Degree|6000|Good|0|0|0| +816|F|D|4 yr Degree|6000|Good|0|0|0| +817|M|W|4 yr Degree|6000|Good|0|0|0| +818|F|W|4 yr Degree|6000|Good|0|0|0| +819|M|U|4 yr Degree|6000|Good|0|0|0| +820|F|U|4 yr Degree|6000|Good|0|0|0| +821|M|M|Advanced Degree|6000|Good|0|0|0| +822|F|M|Advanced Degree|6000|Good|0|0|0| +823|M|S|Advanced Degree|6000|Good|0|0|0| +824|F|S|Advanced Degree|6000|Good|0|0|0| +825|M|D|Advanced Degree|6000|Good|0|0|0| +826|F|D|Advanced Degree|6000|Good|0|0|0| +827|M|W|Advanced Degree|6000|Good|0|0|0| +828|F|W|Advanced Degree|6000|Good|0|0|0| +829|M|U|Advanced Degree|6000|Good|0|0|0| +830|F|U|Advanced Degree|6000|Good|0|0|0| +831|M|M|Unknown|6000|Good|0|0|0| +832|F|M|Unknown|6000|Good|0|0|0| +833|M|S|Unknown|6000|Good|0|0|0| +834|F|S|Unknown|6000|Good|0|0|0| +835|M|D|Unknown|6000|Good|0|0|0| +836|F|D|Unknown|6000|Good|0|0|0| +837|M|W|Unknown|6000|Good|0|0|0| +838|F|W|Unknown|6000|Good|0|0|0| +839|M|U|Unknown|6000|Good|0|0|0| +840|F|U|Unknown|6000|Good|0|0|0| +841|M|M|Primary|6500|Good|0|0|0| +842|F|M|Primary|6500|Good|0|0|0| +843|M|S|Primary|6500|Good|0|0|0| +844|F|S|Primary|6500|Good|0|0|0| +845|M|D|Primary|6500|Good|0|0|0| +846|F|D|Primary|6500|Good|0|0|0| +847|M|W|Primary|6500|Good|0|0|0| +848|F|W|Primary|6500|Good|0|0|0| +849|M|U|Primary|6500|Good|0|0|0| +850|F|U|Primary|6500|Good|0|0|0| +851|M|M|Secondary|6500|Good|0|0|0| +852|F|M|Secondary|6500|Good|0|0|0| +853|M|S|Secondary|6500|Good|0|0|0| +854|F|S|Secondary|6500|Good|0|0|0| +855|M|D|Secondary|6500|Good|0|0|0| +856|F|D|Secondary|6500|Good|0|0|0| +857|M|W|Secondary|6500|Good|0|0|0| +858|F|W|Secondary|6500|Good|0|0|0| +859|M|U|Secondary|6500|Good|0|0|0| +860|F|U|Secondary|6500|Good|0|0|0| +861|M|M|College|6500|Good|0|0|0| +862|F|M|College|6500|Good|0|0|0| +863|M|S|College|6500|Good|0|0|0| +864|F|S|College|6500|Good|0|0|0| +865|M|D|College|6500|Good|0|0|0| +866|F|D|College|6500|Good|0|0|0| +867|M|W|College|6500|Good|0|0|0| +868|F|W|College|6500|Good|0|0|0| +869|M|U|College|6500|Good|0|0|0| +870|F|U|College|6500|Good|0|0|0| +871|M|M|2 yr Degree|6500|Good|0|0|0| +872|F|M|2 yr Degree|6500|Good|0|0|0| +873|M|S|2 yr Degree|6500|Good|0|0|0| +874|F|S|2 yr Degree|6500|Good|0|0|0| +875|M|D|2 yr Degree|6500|Good|0|0|0| +876|F|D|2 yr Degree|6500|Good|0|0|0| +877|M|W|2 yr Degree|6500|Good|0|0|0| +878|F|W|2 yr Degree|6500|Good|0|0|0| +879|M|U|2 yr Degree|6500|Good|0|0|0| +880|F|U|2 yr Degree|6500|Good|0|0|0| +881|M|M|4 yr Degree|6500|Good|0|0|0| +882|F|M|4 yr Degree|6500|Good|0|0|0| +883|M|S|4 yr Degree|6500|Good|0|0|0| +884|F|S|4 yr Degree|6500|Good|0|0|0| +885|M|D|4 yr Degree|6500|Good|0|0|0| +886|F|D|4 yr Degree|6500|Good|0|0|0| +887|M|W|4 yr Degree|6500|Good|0|0|0| +888|F|W|4 yr Degree|6500|Good|0|0|0| +889|M|U|4 yr Degree|6500|Good|0|0|0| +890|F|U|4 yr Degree|6500|Good|0|0|0| +891|M|M|Advanced Degree|6500|Good|0|0|0| +892|F|M|Advanced Degree|6500|Good|0|0|0| +893|M|S|Advanced Degree|6500|Good|0|0|0| +894|F|S|Advanced Degree|6500|Good|0|0|0| +895|M|D|Advanced Degree|6500|Good|0|0|0| +896|F|D|Advanced Degree|6500|Good|0|0|0| +897|M|W|Advanced Degree|6500|Good|0|0|0| +898|F|W|Advanced Degree|6500|Good|0|0|0| +899|M|U|Advanced Degree|6500|Good|0|0|0| +900|F|U|Advanced Degree|6500|Good|0|0|0| +901|M|M|Unknown|6500|Good|0|0|0| +902|F|M|Unknown|6500|Good|0|0|0| +903|M|S|Unknown|6500|Good|0|0|0| +904|F|S|Unknown|6500|Good|0|0|0| +905|M|D|Unknown|6500|Good|0|0|0| +906|F|D|Unknown|6500|Good|0|0|0| +907|M|W|Unknown|6500|Good|0|0|0| +908|F|W|Unknown|6500|Good|0|0|0| +909|M|U|Unknown|6500|Good|0|0|0| +910|F|U|Unknown|6500|Good|0|0|0| +911|M|M|Primary|7000|Good|0|0|0| +912|F|M|Primary|7000|Good|0|0|0| +913|M|S|Primary|7000|Good|0|0|0| +914|F|S|Primary|7000|Good|0|0|0| +915|M|D|Primary|7000|Good|0|0|0| +916|F|D|Primary|7000|Good|0|0|0| +917|M|W|Primary|7000|Good|0|0|0| +918|F|W|Primary|7000|Good|0|0|0| +919|M|U|Primary|7000|Good|0|0|0| +920|F|U|Primary|7000|Good|0|0|0| +921|M|M|Secondary|7000|Good|0|0|0| +922|F|M|Secondary|7000|Good|0|0|0| +923|M|S|Secondary|7000|Good|0|0|0| +924|F|S|Secondary|7000|Good|0|0|0| +925|M|D|Secondary|7000|Good|0|0|0| +926|F|D|Secondary|7000|Good|0|0|0| +927|M|W|Secondary|7000|Good|0|0|0| +928|F|W|Secondary|7000|Good|0|0|0| +929|M|U|Secondary|7000|Good|0|0|0| +930|F|U|Secondary|7000|Good|0|0|0| +931|M|M|College|7000|Good|0|0|0| +932|F|M|College|7000|Good|0|0|0| +933|M|S|College|7000|Good|0|0|0| +934|F|S|College|7000|Good|0|0|0| +935|M|D|College|7000|Good|0|0|0| +936|F|D|College|7000|Good|0|0|0| +937|M|W|College|7000|Good|0|0|0| +938|F|W|College|7000|Good|0|0|0| +939|M|U|College|7000|Good|0|0|0| +940|F|U|College|7000|Good|0|0|0| +941|M|M|2 yr Degree|7000|Good|0|0|0| +942|F|M|2 yr Degree|7000|Good|0|0|0| +943|M|S|2 yr Degree|7000|Good|0|0|0| +944|F|S|2 yr Degree|7000|Good|0|0|0| +945|M|D|2 yr Degree|7000|Good|0|0|0| +946|F|D|2 yr Degree|7000|Good|0|0|0| +947|M|W|2 yr Degree|7000|Good|0|0|0| +948|F|W|2 yr Degree|7000|Good|0|0|0| +949|M|U|2 yr Degree|7000|Good|0|0|0| +950|F|U|2 yr Degree|7000|Good|0|0|0| +951|M|M|4 yr Degree|7000|Good|0|0|0| +952|F|M|4 yr Degree|7000|Good|0|0|0| +953|M|S|4 yr Degree|7000|Good|0|0|0| +954|F|S|4 yr Degree|7000|Good|0|0|0| +955|M|D|4 yr Degree|7000|Good|0|0|0| +956|F|D|4 yr Degree|7000|Good|0|0|0| +957|M|W|4 yr Degree|7000|Good|0|0|0| +958|F|W|4 yr Degree|7000|Good|0|0|0| +959|M|U|4 yr Degree|7000|Good|0|0|0| +960|F|U|4 yr Degree|7000|Good|0|0|0| +961|M|M|Advanced Degree|7000|Good|0|0|0| +962|F|M|Advanced Degree|7000|Good|0|0|0| +963|M|S|Advanced Degree|7000|Good|0|0|0| +964|F|S|Advanced Degree|7000|Good|0|0|0| +965|M|D|Advanced Degree|7000|Good|0|0|0| +966|F|D|Advanced Degree|7000|Good|0|0|0| +967|M|W|Advanced Degree|7000|Good|0|0|0| +968|F|W|Advanced Degree|7000|Good|0|0|0| +969|M|U|Advanced Degree|7000|Good|0|0|0| +970|F|U|Advanced Degree|7000|Good|0|0|0| +971|M|M|Unknown|7000|Good|0|0|0| +972|F|M|Unknown|7000|Good|0|0|0| +973|M|S|Unknown|7000|Good|0|0|0| +974|F|S|Unknown|7000|Good|0|0|0| +975|M|D|Unknown|7000|Good|0|0|0| +976|F|D|Unknown|7000|Good|0|0|0| +977|M|W|Unknown|7000|Good|0|0|0| +978|F|W|Unknown|7000|Good|0|0|0| +979|M|U|Unknown|7000|Good|0|0|0| +980|F|U|Unknown|7000|Good|0|0|0| +981|M|M|Primary|7500|Good|0|0|0| +982|F|M|Primary|7500|Good|0|0|0| +983|M|S|Primary|7500|Good|0|0|0| +984|F|S|Primary|7500|Good|0|0|0| +985|M|D|Primary|7500|Good|0|0|0| +986|F|D|Primary|7500|Good|0|0|0| +987|M|W|Primary|7500|Good|0|0|0| +988|F|W|Primary|7500|Good|0|0|0| +989|M|U|Primary|7500|Good|0|0|0| +990|F|U|Primary|7500|Good|0|0|0| +991|M|M|Secondary|7500|Good|0|0|0| +992|F|M|Secondary|7500|Good|0|0|0| +993|M|S|Secondary|7500|Good|0|0|0| +994|F|S|Secondary|7500|Good|0|0|0| +995|M|D|Secondary|7500|Good|0|0|0| +996|F|D|Secondary|7500|Good|0|0|0| +997|M|W|Secondary|7500|Good|0|0|0| +998|F|W|Secondary|7500|Good|0|0|0| +999|M|U|Secondary|7500|Good|0|0|0| +1000|F|U|Secondary|7500|Good|0|0|0| +1001|M|M|College|7500|Good|0|0|0| +1002|F|M|College|7500|Good|0|0|0| +1003|M|S|College|7500|Good|0|0|0| +1004|F|S|College|7500|Good|0|0|0| +1005|M|D|College|7500|Good|0|0|0| +1006|F|D|College|7500|Good|0|0|0| +1007|M|W|College|7500|Good|0|0|0| +1008|F|W|College|7500|Good|0|0|0| +1009|M|U|College|7500|Good|0|0|0| +1010|F|U|College|7500|Good|0|0|0| +1011|M|M|2 yr Degree|7500|Good|0|0|0| +1012|F|M|2 yr Degree|7500|Good|0|0|0| +1013|M|S|2 yr Degree|7500|Good|0|0|0| +1014|F|S|2 yr Degree|7500|Good|0|0|0| +1015|M|D|2 yr Degree|7500|Good|0|0|0| +1016|F|D|2 yr Degree|7500|Good|0|0|0| +1017|M|W|2 yr Degree|7500|Good|0|0|0| +1018|F|W|2 yr Degree|7500|Good|0|0|0| +1019|M|U|2 yr Degree|7500|Good|0|0|0| +1020|F|U|2 yr Degree|7500|Good|0|0|0| +1021|M|M|4 yr Degree|7500|Good|0|0|0| +1022|F|M|4 yr Degree|7500|Good|0|0|0| +1023|M|S|4 yr Degree|7500|Good|0|0|0| +1024|F|S|4 yr Degree|7500|Good|0|0|0| +1025|M|D|4 yr Degree|7500|Good|0|0|0| +1026|F|D|4 yr Degree|7500|Good|0|0|0| +1027|M|W|4 yr Degree|7500|Good|0|0|0| +1028|F|W|4 yr Degree|7500|Good|0|0|0| +1029|M|U|4 yr Degree|7500|Good|0|0|0| +1030|F|U|4 yr Degree|7500|Good|0|0|0| +1031|M|M|Advanced Degree|7500|Good|0|0|0| +1032|F|M|Advanced Degree|7500|Good|0|0|0| +1033|M|S|Advanced Degree|7500|Good|0|0|0| +1034|F|S|Advanced Degree|7500|Good|0|0|0| +1035|M|D|Advanced Degree|7500|Good|0|0|0| +1036|F|D|Advanced Degree|7500|Good|0|0|0| +1037|M|W|Advanced Degree|7500|Good|0|0|0| +1038|F|W|Advanced Degree|7500|Good|0|0|0| +1039|M|U|Advanced Degree|7500|Good|0|0|0| +1040|F|U|Advanced Degree|7500|Good|0|0|0| +1041|M|M|Unknown|7500|Good|0|0|0| +1042|F|M|Unknown|7500|Good|0|0|0| +1043|M|S|Unknown|7500|Good|0|0|0| +1044|F|S|Unknown|7500|Good|0|0|0| +1045|M|D|Unknown|7500|Good|0|0|0| +1046|F|D|Unknown|7500|Good|0|0|0| +1047|M|W|Unknown|7500|Good|0|0|0| +1048|F|W|Unknown|7500|Good|0|0|0| +1049|M|U|Unknown|7500|Good|0|0|0| +1050|F|U|Unknown|7500|Good|0|0|0| +1051|M|M|Primary|8000|Good|0|0|0| +1052|F|M|Primary|8000|Good|0|0|0| +1053|M|S|Primary|8000|Good|0|0|0| +1054|F|S|Primary|8000|Good|0|0|0| +1055|M|D|Primary|8000|Good|0|0|0| +1056|F|D|Primary|8000|Good|0|0|0| +1057|M|W|Primary|8000|Good|0|0|0| +1058|F|W|Primary|8000|Good|0|0|0| +1059|M|U|Primary|8000|Good|0|0|0| +1060|F|U|Primary|8000|Good|0|0|0| +1061|M|M|Secondary|8000|Good|0|0|0| +1062|F|M|Secondary|8000|Good|0|0|0| +1063|M|S|Secondary|8000|Good|0|0|0| +1064|F|S|Secondary|8000|Good|0|0|0| +1065|M|D|Secondary|8000|Good|0|0|0| +1066|F|D|Secondary|8000|Good|0|0|0| +1067|M|W|Secondary|8000|Good|0|0|0| +1068|F|W|Secondary|8000|Good|0|0|0| +1069|M|U|Secondary|8000|Good|0|0|0| +1070|F|U|Secondary|8000|Good|0|0|0| +1071|M|M|College|8000|Good|0|0|0| +1072|F|M|College|8000|Good|0|0|0| +1073|M|S|College|8000|Good|0|0|0| +1074|F|S|College|8000|Good|0|0|0| +1075|M|D|College|8000|Good|0|0|0| +1076|F|D|College|8000|Good|0|0|0| +1077|M|W|College|8000|Good|0|0|0| +1078|F|W|College|8000|Good|0|0|0| +1079|M|U|College|8000|Good|0|0|0| +1080|F|U|College|8000|Good|0|0|0| +1081|M|M|2 yr Degree|8000|Good|0|0|0| +1082|F|M|2 yr Degree|8000|Good|0|0|0| +1083|M|S|2 yr Degree|8000|Good|0|0|0| +1084|F|S|2 yr Degree|8000|Good|0|0|0| +1085|M|D|2 yr Degree|8000|Good|0|0|0| +1086|F|D|2 yr Degree|8000|Good|0|0|0| +1087|M|W|2 yr Degree|8000|Good|0|0|0| +1088|F|W|2 yr Degree|8000|Good|0|0|0| +1089|M|U|2 yr Degree|8000|Good|0|0|0| +1090|F|U|2 yr Degree|8000|Good|0|0|0| +1091|M|M|4 yr Degree|8000|Good|0|0|0| +1092|F|M|4 yr Degree|8000|Good|0|0|0| +1093|M|S|4 yr Degree|8000|Good|0|0|0| +1094|F|S|4 yr Degree|8000|Good|0|0|0| +1095|M|D|4 yr Degree|8000|Good|0|0|0| +1096|F|D|4 yr Degree|8000|Good|0|0|0| +1097|M|W|4 yr Degree|8000|Good|0|0|0| +1098|F|W|4 yr Degree|8000|Good|0|0|0| +1099|M|U|4 yr Degree|8000|Good|0|0|0| +1100|F|U|4 yr Degree|8000|Good|0|0|0| +1101|M|M|Advanced Degree|8000|Good|0|0|0| +1102|F|M|Advanced Degree|8000|Good|0|0|0| +1103|M|S|Advanced Degree|8000|Good|0|0|0| +1104|F|S|Advanced Degree|8000|Good|0|0|0| +1105|M|D|Advanced Degree|8000|Good|0|0|0| +1106|F|D|Advanced Degree|8000|Good|0|0|0| +1107|M|W|Advanced Degree|8000|Good|0|0|0| +1108|F|W|Advanced Degree|8000|Good|0|0|0| +1109|M|U|Advanced Degree|8000|Good|0|0|0| +1110|F|U|Advanced Degree|8000|Good|0|0|0| +1111|M|M|Unknown|8000|Good|0|0|0| +1112|F|M|Unknown|8000|Good|0|0|0| +1113|M|S|Unknown|8000|Good|0|0|0| +1114|F|S|Unknown|8000|Good|0|0|0| +1115|M|D|Unknown|8000|Good|0|0|0| +1116|F|D|Unknown|8000|Good|0|0|0| +1117|M|W|Unknown|8000|Good|0|0|0| +1118|F|W|Unknown|8000|Good|0|0|0| +1119|M|U|Unknown|8000|Good|0|0|0| +1120|F|U|Unknown|8000|Good|0|0|0| +1121|M|M|Primary|8500|Good|0|0|0| +1122|F|M|Primary|8500|Good|0|0|0| +1123|M|S|Primary|8500|Good|0|0|0| +1124|F|S|Primary|8500|Good|0|0|0| +1125|M|D|Primary|8500|Good|0|0|0| +1126|F|D|Primary|8500|Good|0|0|0| +1127|M|W|Primary|8500|Good|0|0|0| +1128|F|W|Primary|8500|Good|0|0|0| +1129|M|U|Primary|8500|Good|0|0|0| +1130|F|U|Primary|8500|Good|0|0|0| +1131|M|M|Secondary|8500|Good|0|0|0| +1132|F|M|Secondary|8500|Good|0|0|0| +1133|M|S|Secondary|8500|Good|0|0|0| +1134|F|S|Secondary|8500|Good|0|0|0| +1135|M|D|Secondary|8500|Good|0|0|0| +1136|F|D|Secondary|8500|Good|0|0|0| +1137|M|W|Secondary|8500|Good|0|0|0| +1138|F|W|Secondary|8500|Good|0|0|0| +1139|M|U|Secondary|8500|Good|0|0|0| +1140|F|U|Secondary|8500|Good|0|0|0| +1141|M|M|College|8500|Good|0|0|0| +1142|F|M|College|8500|Good|0|0|0| +1143|M|S|College|8500|Good|0|0|0| +1144|F|S|College|8500|Good|0|0|0| +1145|M|D|College|8500|Good|0|0|0| +1146|F|D|College|8500|Good|0|0|0| +1147|M|W|College|8500|Good|0|0|0| +1148|F|W|College|8500|Good|0|0|0| +1149|M|U|College|8500|Good|0|0|0| +1150|F|U|College|8500|Good|0|0|0| +1151|M|M|2 yr Degree|8500|Good|0|0|0| +1152|F|M|2 yr Degree|8500|Good|0|0|0| +1153|M|S|2 yr Degree|8500|Good|0|0|0| +1154|F|S|2 yr Degree|8500|Good|0|0|0| +1155|M|D|2 yr Degree|8500|Good|0|0|0| +1156|F|D|2 yr Degree|8500|Good|0|0|0| +1157|M|W|2 yr Degree|8500|Good|0|0|0| +1158|F|W|2 yr Degree|8500|Good|0|0|0| +1159|M|U|2 yr Degree|8500|Good|0|0|0| +1160|F|U|2 yr Degree|8500|Good|0|0|0| +1161|M|M|4 yr Degree|8500|Good|0|0|0| +1162|F|M|4 yr Degree|8500|Good|0|0|0| +1163|M|S|4 yr Degree|8500|Good|0|0|0| +1164|F|S|4 yr Degree|8500|Good|0|0|0| +1165|M|D|4 yr Degree|8500|Good|0|0|0| +1166|F|D|4 yr Degree|8500|Good|0|0|0| +1167|M|W|4 yr Degree|8500|Good|0|0|0| +1168|F|W|4 yr Degree|8500|Good|0|0|0| +1169|M|U|4 yr Degree|8500|Good|0|0|0| +1170|F|U|4 yr Degree|8500|Good|0|0|0| +1171|M|M|Advanced Degree|8500|Good|0|0|0| +1172|F|M|Advanced Degree|8500|Good|0|0|0| +1173|M|S|Advanced Degree|8500|Good|0|0|0| +1174|F|S|Advanced Degree|8500|Good|0|0|0| +1175|M|D|Advanced Degree|8500|Good|0|0|0| +1176|F|D|Advanced Degree|8500|Good|0|0|0| +1177|M|W|Advanced Degree|8500|Good|0|0|0| +1178|F|W|Advanced Degree|8500|Good|0|0|0| +1179|M|U|Advanced Degree|8500|Good|0|0|0| +1180|F|U|Advanced Degree|8500|Good|0|0|0| +1181|M|M|Unknown|8500|Good|0|0|0| +1182|F|M|Unknown|8500|Good|0|0|0| +1183|M|S|Unknown|8500|Good|0|0|0| +1184|F|S|Unknown|8500|Good|0|0|0| +1185|M|D|Unknown|8500|Good|0|0|0| +1186|F|D|Unknown|8500|Good|0|0|0| +1187|M|W|Unknown|8500|Good|0|0|0| +1188|F|W|Unknown|8500|Good|0|0|0| +1189|M|U|Unknown|8500|Good|0|0|0| +1190|F|U|Unknown|8500|Good|0|0|0| +1191|M|M|Primary|9000|Good|0|0|0| +1192|F|M|Primary|9000|Good|0|0|0| +1193|M|S|Primary|9000|Good|0|0|0| +1194|F|S|Primary|9000|Good|0|0|0| +1195|M|D|Primary|9000|Good|0|0|0| +1196|F|D|Primary|9000|Good|0|0|0| +1197|M|W|Primary|9000|Good|0|0|0| +1198|F|W|Primary|9000|Good|0|0|0| +1199|M|U|Primary|9000|Good|0|0|0| +1200|F|U|Primary|9000|Good|0|0|0| +1201|M|M|Secondary|9000|Good|0|0|0| +1202|F|M|Secondary|9000|Good|0|0|0| +1203|M|S|Secondary|9000|Good|0|0|0| +1204|F|S|Secondary|9000|Good|0|0|0| +1205|M|D|Secondary|9000|Good|0|0|0| +1206|F|D|Secondary|9000|Good|0|0|0| +1207|M|W|Secondary|9000|Good|0|0|0| +1208|F|W|Secondary|9000|Good|0|0|0| +1209|M|U|Secondary|9000|Good|0|0|0| +1210|F|U|Secondary|9000|Good|0|0|0| +1211|M|M|College|9000|Good|0|0|0| +1212|F|M|College|9000|Good|0|0|0| +1213|M|S|College|9000|Good|0|0|0| +1214|F|S|College|9000|Good|0|0|0| +1215|M|D|College|9000|Good|0|0|0| +1216|F|D|College|9000|Good|0|0|0| +1217|M|W|College|9000|Good|0|0|0| +1218|F|W|College|9000|Good|0|0|0| +1219|M|U|College|9000|Good|0|0|0| +1220|F|U|College|9000|Good|0|0|0| +1221|M|M|2 yr Degree|9000|Good|0|0|0| +1222|F|M|2 yr Degree|9000|Good|0|0|0| +1223|M|S|2 yr Degree|9000|Good|0|0|0| +1224|F|S|2 yr Degree|9000|Good|0|0|0| +1225|M|D|2 yr Degree|9000|Good|0|0|0| +1226|F|D|2 yr Degree|9000|Good|0|0|0| +1227|M|W|2 yr Degree|9000|Good|0|0|0| +1228|F|W|2 yr Degree|9000|Good|0|0|0| +1229|M|U|2 yr Degree|9000|Good|0|0|0| +1230|F|U|2 yr Degree|9000|Good|0|0|0| +1231|M|M|4 yr Degree|9000|Good|0|0|0| +1232|F|M|4 yr Degree|9000|Good|0|0|0| +1233|M|S|4 yr Degree|9000|Good|0|0|0| +1234|F|S|4 yr Degree|9000|Good|0|0|0| +1235|M|D|4 yr Degree|9000|Good|0|0|0| +1236|F|D|4 yr Degree|9000|Good|0|0|0| +1237|M|W|4 yr Degree|9000|Good|0|0|0| +1238|F|W|4 yr Degree|9000|Good|0|0|0| +1239|M|U|4 yr Degree|9000|Good|0|0|0| +1240|F|U|4 yr Degree|9000|Good|0|0|0| +1241|M|M|Advanced Degree|9000|Good|0|0|0| +1242|F|M|Advanced Degree|9000|Good|0|0|0| +1243|M|S|Advanced Degree|9000|Good|0|0|0| +1244|F|S|Advanced Degree|9000|Good|0|0|0| +1245|M|D|Advanced Degree|9000|Good|0|0|0| +1246|F|D|Advanced Degree|9000|Good|0|0|0| +1247|M|W|Advanced Degree|9000|Good|0|0|0| +1248|F|W|Advanced Degree|9000|Good|0|0|0| +1249|M|U|Advanced Degree|9000|Good|0|0|0| +1250|F|U|Advanced Degree|9000|Good|0|0|0| +1251|M|M|Unknown|9000|Good|0|0|0| +1252|F|M|Unknown|9000|Good|0|0|0| +1253|M|S|Unknown|9000|Good|0|0|0| +1254|F|S|Unknown|9000|Good|0|0|0| +1255|M|D|Unknown|9000|Good|0|0|0| +1256|F|D|Unknown|9000|Good|0|0|0| +1257|M|W|Unknown|9000|Good|0|0|0| +1258|F|W|Unknown|9000|Good|0|0|0| +1259|M|U|Unknown|9000|Good|0|0|0| +1260|F|U|Unknown|9000|Good|0|0|0| +1261|M|M|Primary|9500|Good|0|0|0| +1262|F|M|Primary|9500|Good|0|0|0| +1263|M|S|Primary|9500|Good|0|0|0| +1264|F|S|Primary|9500|Good|0|0|0| +1265|M|D|Primary|9500|Good|0|0|0| +1266|F|D|Primary|9500|Good|0|0|0| +1267|M|W|Primary|9500|Good|0|0|0| +1268|F|W|Primary|9500|Good|0|0|0| +1269|M|U|Primary|9500|Good|0|0|0| +1270|F|U|Primary|9500|Good|0|0|0| +1271|M|M|Secondary|9500|Good|0|0|0| +1272|F|M|Secondary|9500|Good|0|0|0| +1273|M|S|Secondary|9500|Good|0|0|0| +1274|F|S|Secondary|9500|Good|0|0|0| +1275|M|D|Secondary|9500|Good|0|0|0| +1276|F|D|Secondary|9500|Good|0|0|0| +1277|M|W|Secondary|9500|Good|0|0|0| +1278|F|W|Secondary|9500|Good|0|0|0| +1279|M|U|Secondary|9500|Good|0|0|0| +1280|F|U|Secondary|9500|Good|0|0|0| +1281|M|M|College|9500|Good|0|0|0| +1282|F|M|College|9500|Good|0|0|0| +1283|M|S|College|9500|Good|0|0|0| +1284|F|S|College|9500|Good|0|0|0| +1285|M|D|College|9500|Good|0|0|0| +1286|F|D|College|9500|Good|0|0|0| +1287|M|W|College|9500|Good|0|0|0| +1288|F|W|College|9500|Good|0|0|0| +1289|M|U|College|9500|Good|0|0|0| +1290|F|U|College|9500|Good|0|0|0| +1291|M|M|2 yr Degree|9500|Good|0|0|0| +1292|F|M|2 yr Degree|9500|Good|0|0|0| +1293|M|S|2 yr Degree|9500|Good|0|0|0| +1294|F|S|2 yr Degree|9500|Good|0|0|0| +1295|M|D|2 yr Degree|9500|Good|0|0|0| +1296|F|D|2 yr Degree|9500|Good|0|0|0| +1297|M|W|2 yr Degree|9500|Good|0|0|0| +1298|F|W|2 yr Degree|9500|Good|0|0|0| +1299|M|U|2 yr Degree|9500|Good|0|0|0| +1300|F|U|2 yr Degree|9500|Good|0|0|0| +1301|M|M|4 yr Degree|9500|Good|0|0|0| +1302|F|M|4 yr Degree|9500|Good|0|0|0| +1303|M|S|4 yr Degree|9500|Good|0|0|0| +1304|F|S|4 yr Degree|9500|Good|0|0|0| +1305|M|D|4 yr Degree|9500|Good|0|0|0| +1306|F|D|4 yr Degree|9500|Good|0|0|0| +1307|M|W|4 yr Degree|9500|Good|0|0|0| +1308|F|W|4 yr Degree|9500|Good|0|0|0| +1309|M|U|4 yr Degree|9500|Good|0|0|0| +1310|F|U|4 yr Degree|9500|Good|0|0|0| +1311|M|M|Advanced Degree|9500|Good|0|0|0| +1312|F|M|Advanced Degree|9500|Good|0|0|0| +1313|M|S|Advanced Degree|9500|Good|0|0|0| +1314|F|S|Advanced Degree|9500|Good|0|0|0| +1315|M|D|Advanced Degree|9500|Good|0|0|0| +1316|F|D|Advanced Degree|9500|Good|0|0|0| +1317|M|W|Advanced Degree|9500|Good|0|0|0| +1318|F|W|Advanced Degree|9500|Good|0|0|0| +1319|M|U|Advanced Degree|9500|Good|0|0|0| +1320|F|U|Advanced Degree|9500|Good|0|0|0| +1321|M|M|Unknown|9500|Good|0|0|0| +1322|F|M|Unknown|9500|Good|0|0|0| +1323|M|S|Unknown|9500|Good|0|0|0| +1324|F|S|Unknown|9500|Good|0|0|0| +1325|M|D|Unknown|9500|Good|0|0|0| +1326|F|D|Unknown|9500|Good|0|0|0| +1327|M|W|Unknown|9500|Good|0|0|0| +1328|F|W|Unknown|9500|Good|0|0|0| +1329|M|U|Unknown|9500|Good|0|0|0| +1330|F|U|Unknown|9500|Good|0|0|0| +1331|M|M|Primary|10000|Good|0|0|0| +1332|F|M|Primary|10000|Good|0|0|0| +1333|M|S|Primary|10000|Good|0|0|0| +1334|F|S|Primary|10000|Good|0|0|0| +1335|M|D|Primary|10000|Good|0|0|0| +1336|F|D|Primary|10000|Good|0|0|0| +1337|M|W|Primary|10000|Good|0|0|0| +1338|F|W|Primary|10000|Good|0|0|0| +1339|M|U|Primary|10000|Good|0|0|0| +1340|F|U|Primary|10000|Good|0|0|0| +1341|M|M|Secondary|10000|Good|0|0|0| +1342|F|M|Secondary|10000|Good|0|0|0| +1343|M|S|Secondary|10000|Good|0|0|0| +1344|F|S|Secondary|10000|Good|0|0|0| +1345|M|D|Secondary|10000|Good|0|0|0| +1346|F|D|Secondary|10000|Good|0|0|0| +1347|M|W|Secondary|10000|Good|0|0|0| +1348|F|W|Secondary|10000|Good|0|0|0| +1349|M|U|Secondary|10000|Good|0|0|0| +1350|F|U|Secondary|10000|Good|0|0|0| +1351|M|M|College|10000|Good|0|0|0| +1352|F|M|College|10000|Good|0|0|0| +1353|M|S|College|10000|Good|0|0|0| +1354|F|S|College|10000|Good|0|0|0| +1355|M|D|College|10000|Good|0|0|0| +1356|F|D|College|10000|Good|0|0|0| +1357|M|W|College|10000|Good|0|0|0| +1358|F|W|College|10000|Good|0|0|0| +1359|M|U|College|10000|Good|0|0|0| +1360|F|U|College|10000|Good|0|0|0| +1361|M|M|2 yr Degree|10000|Good|0|0|0| +1362|F|M|2 yr Degree|10000|Good|0|0|0| +1363|M|S|2 yr Degree|10000|Good|0|0|0| +1364|F|S|2 yr Degree|10000|Good|0|0|0| +1365|M|D|2 yr Degree|10000|Good|0|0|0| +1366|F|D|2 yr Degree|10000|Good|0|0|0| +1367|M|W|2 yr Degree|10000|Good|0|0|0| +1368|F|W|2 yr Degree|10000|Good|0|0|0| +1369|M|U|2 yr Degree|10000|Good|0|0|0| +1370|F|U|2 yr Degree|10000|Good|0|0|0| +1371|M|M|4 yr Degree|10000|Good|0|0|0| +1372|F|M|4 yr Degree|10000|Good|0|0|0| +1373|M|S|4 yr Degree|10000|Good|0|0|0| +1374|F|S|4 yr Degree|10000|Good|0|0|0| +1375|M|D|4 yr Degree|10000|Good|0|0|0| +1376|F|D|4 yr Degree|10000|Good|0|0|0| +1377|M|W|4 yr Degree|10000|Good|0|0|0| +1378|F|W|4 yr Degree|10000|Good|0|0|0| +1379|M|U|4 yr Degree|10000|Good|0|0|0| +1380|F|U|4 yr Degree|10000|Good|0|0|0| +1381|M|M|Advanced Degree|10000|Good|0|0|0| +1382|F|M|Advanced Degree|10000|Good|0|0|0| +1383|M|S|Advanced Degree|10000|Good|0|0|0| +1384|F|S|Advanced Degree|10000|Good|0|0|0| +1385|M|D|Advanced Degree|10000|Good|0|0|0| +1386|F|D|Advanced Degree|10000|Good|0|0|0| +1387|M|W|Advanced Degree|10000|Good|0|0|0| +1388|F|W|Advanced Degree|10000|Good|0|0|0| +1389|M|U|Advanced Degree|10000|Good|0|0|0| +1390|F|U|Advanced Degree|10000|Good|0|0|0| +1391|M|M|Unknown|10000|Good|0|0|0| +1392|F|M|Unknown|10000|Good|0|0|0| +1393|M|S|Unknown|10000|Good|0|0|0| +1394|F|S|Unknown|10000|Good|0|0|0| +1395|M|D|Unknown|10000|Good|0|0|0| +1396|F|D|Unknown|10000|Good|0|0|0| +1397|M|W|Unknown|10000|Good|0|0|0| +1398|F|W|Unknown|10000|Good|0|0|0| +1399|M|U|Unknown|10000|Good|0|0|0| +1400|F|U|Unknown|10000|Good|0|0|0| +1401|M|M|Primary|500|Low Risk|0|0|0| +1402|F|M|Primary|500|Low Risk|0|0|0| +1403|M|S|Primary|500|Low Risk|0|0|0| +1404|F|S|Primary|500|Low Risk|0|0|0| +1405|M|D|Primary|500|Low Risk|0|0|0| +1406|F|D|Primary|500|Low Risk|0|0|0| +1407|M|W|Primary|500|Low Risk|0|0|0| +1408|F|W|Primary|500|Low Risk|0|0|0| +1409|M|U|Primary|500|Low Risk|0|0|0| +1410|F|U|Primary|500|Low Risk|0|0|0| +1411|M|M|Secondary|500|Low Risk|0|0|0| +1412|F|M|Secondary|500|Low Risk|0|0|0| +1413|M|S|Secondary|500|Low Risk|0|0|0| +1414|F|S|Secondary|500|Low Risk|0|0|0| +1415|M|D|Secondary|500|Low Risk|0|0|0| +1416|F|D|Secondary|500|Low Risk|0|0|0| +1417|M|W|Secondary|500|Low Risk|0|0|0| +1418|F|W|Secondary|500|Low Risk|0|0|0| +1419|M|U|Secondary|500|Low Risk|0|0|0| +1420|F|U|Secondary|500|Low Risk|0|0|0| +1421|M|M|College|500|Low Risk|0|0|0| +1422|F|M|College|500|Low Risk|0|0|0| +1423|M|S|College|500|Low Risk|0|0|0| +1424|F|S|College|500|Low Risk|0|0|0| +1425|M|D|College|500|Low Risk|0|0|0| +1426|F|D|College|500|Low Risk|0|0|0| +1427|M|W|College|500|Low Risk|0|0|0| +1428|F|W|College|500|Low Risk|0|0|0| +1429|M|U|College|500|Low Risk|0|0|0| +1430|F|U|College|500|Low Risk|0|0|0| +1431|M|M|2 yr Degree|500|Low Risk|0|0|0| +1432|F|M|2 yr Degree|500|Low Risk|0|0|0| +1433|M|S|2 yr Degree|500|Low Risk|0|0|0| +1434|F|S|2 yr Degree|500|Low Risk|0|0|0| +1435|M|D|2 yr Degree|500|Low Risk|0|0|0| +1436|F|D|2 yr Degree|500|Low Risk|0|0|0| +1437|M|W|2 yr Degree|500|Low Risk|0|0|0| +1438|F|W|2 yr Degree|500|Low Risk|0|0|0| +1439|M|U|2 yr Degree|500|Low Risk|0|0|0| +1440|F|U|2 yr Degree|500|Low Risk|0|0|0| +1441|M|M|4 yr Degree|500|Low Risk|0|0|0| +1442|F|M|4 yr Degree|500|Low Risk|0|0|0| +1443|M|S|4 yr Degree|500|Low Risk|0|0|0| +1444|F|S|4 yr Degree|500|Low Risk|0|0|0| +1445|M|D|4 yr Degree|500|Low Risk|0|0|0| +1446|F|D|4 yr Degree|500|Low Risk|0|0|0| +1447|M|W|4 yr Degree|500|Low Risk|0|0|0| +1448|F|W|4 yr Degree|500|Low Risk|0|0|0| +1449|M|U|4 yr Degree|500|Low Risk|0|0|0| +1450|F|U|4 yr Degree|500|Low Risk|0|0|0| +1451|M|M|Advanced Degree|500|Low Risk|0|0|0| +1452|F|M|Advanced Degree|500|Low Risk|0|0|0| +1453|M|S|Advanced Degree|500|Low Risk|0|0|0| +1454|F|S|Advanced Degree|500|Low Risk|0|0|0| +1455|M|D|Advanced Degree|500|Low Risk|0|0|0| +1456|F|D|Advanced Degree|500|Low Risk|0|0|0| +1457|M|W|Advanced Degree|500|Low Risk|0|0|0| +1458|F|W|Advanced Degree|500|Low Risk|0|0|0| +1459|M|U|Advanced Degree|500|Low Risk|0|0|0| +1460|F|U|Advanced Degree|500|Low Risk|0|0|0| +1461|M|M|Unknown|500|Low Risk|0|0|0| +1462|F|M|Unknown|500|Low Risk|0|0|0| +1463|M|S|Unknown|500|Low Risk|0|0|0| +1464|F|S|Unknown|500|Low Risk|0|0|0| +1465|M|D|Unknown|500|Low Risk|0|0|0| +1466|F|D|Unknown|500|Low Risk|0|0|0| +1467|M|W|Unknown|500|Low Risk|0|0|0| +1468|F|W|Unknown|500|Low Risk|0|0|0| +1469|M|U|Unknown|500|Low Risk|0|0|0| +1470|F|U|Unknown|500|Low Risk|0|0|0| +1471|M|M|Primary|1000|Low Risk|0|0|0| +1472|F|M|Primary|1000|Low Risk|0|0|0| +1473|M|S|Primary|1000|Low Risk|0|0|0| +1474|F|S|Primary|1000|Low Risk|0|0|0| +1475|M|D|Primary|1000|Low Risk|0|0|0| +1476|F|D|Primary|1000|Low Risk|0|0|0| +1477|M|W|Primary|1000|Low Risk|0|0|0| +1478|F|W|Primary|1000|Low Risk|0|0|0| +1479|M|U|Primary|1000|Low Risk|0|0|0| +1480|F|U|Primary|1000|Low Risk|0|0|0| +1481|M|M|Secondary|1000|Low Risk|0|0|0| +1482|F|M|Secondary|1000|Low Risk|0|0|0| +1483|M|S|Secondary|1000|Low Risk|0|0|0| +1484|F|S|Secondary|1000|Low Risk|0|0|0| +1485|M|D|Secondary|1000|Low Risk|0|0|0| +1486|F|D|Secondary|1000|Low Risk|0|0|0| +1487|M|W|Secondary|1000|Low Risk|0|0|0| +1488|F|W|Secondary|1000|Low Risk|0|0|0| +1489|M|U|Secondary|1000|Low Risk|0|0|0| +1490|F|U|Secondary|1000|Low Risk|0|0|0| +1491|M|M|College|1000|Low Risk|0|0|0| +1492|F|M|College|1000|Low Risk|0|0|0| +1493|M|S|College|1000|Low Risk|0|0|0| +1494|F|S|College|1000|Low Risk|0|0|0| +1495|M|D|College|1000|Low Risk|0|0|0| +1496|F|D|College|1000|Low Risk|0|0|0| +1497|M|W|College|1000|Low Risk|0|0|0| +1498|F|W|College|1000|Low Risk|0|0|0| +1499|M|U|College|1000|Low Risk|0|0|0| +1500|F|U|College|1000|Low Risk|0|0|0| +1501|M|M|2 yr Degree|1000|Low Risk|0|0|0| +1502|F|M|2 yr Degree|1000|Low Risk|0|0|0| +1503|M|S|2 yr Degree|1000|Low Risk|0|0|0| +1504|F|S|2 yr Degree|1000|Low Risk|0|0|0| +1505|M|D|2 yr Degree|1000|Low Risk|0|0|0| +1506|F|D|2 yr Degree|1000|Low Risk|0|0|0| +1507|M|W|2 yr Degree|1000|Low Risk|0|0|0| +1508|F|W|2 yr Degree|1000|Low Risk|0|0|0| +1509|M|U|2 yr Degree|1000|Low Risk|0|0|0| +1510|F|U|2 yr Degree|1000|Low Risk|0|0|0| +1511|M|M|4 yr Degree|1000|Low Risk|0|0|0| +1512|F|M|4 yr Degree|1000|Low Risk|0|0|0| +1513|M|S|4 yr Degree|1000|Low Risk|0|0|0| +1514|F|S|4 yr Degree|1000|Low Risk|0|0|0| +1515|M|D|4 yr Degree|1000|Low Risk|0|0|0| +1516|F|D|4 yr Degree|1000|Low Risk|0|0|0| +1517|M|W|4 yr Degree|1000|Low Risk|0|0|0| +1518|F|W|4 yr Degree|1000|Low Risk|0|0|0| +1519|M|U|4 yr Degree|1000|Low Risk|0|0|0| +1520|F|U|4 yr Degree|1000|Low Risk|0|0|0| +1521|M|M|Advanced Degree|1000|Low Risk|0|0|0| +1522|F|M|Advanced Degree|1000|Low Risk|0|0|0| +1523|M|S|Advanced Degree|1000|Low Risk|0|0|0| +1524|F|S|Advanced Degree|1000|Low Risk|0|0|0| +1525|M|D|Advanced Degree|1000|Low Risk|0|0|0| +1526|F|D|Advanced Degree|1000|Low Risk|0|0|0| +1527|M|W|Advanced Degree|1000|Low Risk|0|0|0| +1528|F|W|Advanced Degree|1000|Low Risk|0|0|0| +1529|M|U|Advanced Degree|1000|Low Risk|0|0|0| +1530|F|U|Advanced Degree|1000|Low Risk|0|0|0| +1531|M|M|Unknown|1000|Low Risk|0|0|0| +1532|F|M|Unknown|1000|Low Risk|0|0|0| +1533|M|S|Unknown|1000|Low Risk|0|0|0| +1534|F|S|Unknown|1000|Low Risk|0|0|0| +1535|M|D|Unknown|1000|Low Risk|0|0|0| +1536|F|D|Unknown|1000|Low Risk|0|0|0| +1537|M|W|Unknown|1000|Low Risk|0|0|0| +1538|F|W|Unknown|1000|Low Risk|0|0|0| +1539|M|U|Unknown|1000|Low Risk|0|0|0| +1540|F|U|Unknown|1000|Low Risk|0|0|0| +1541|M|M|Primary|1500|Low Risk|0|0|0| +1542|F|M|Primary|1500|Low Risk|0|0|0| +1543|M|S|Primary|1500|Low Risk|0|0|0| +1544|F|S|Primary|1500|Low Risk|0|0|0| +1545|M|D|Primary|1500|Low Risk|0|0|0| +1546|F|D|Primary|1500|Low Risk|0|0|0| +1547|M|W|Primary|1500|Low Risk|0|0|0| +1548|F|W|Primary|1500|Low Risk|0|0|0| +1549|M|U|Primary|1500|Low Risk|0|0|0| +1550|F|U|Primary|1500|Low Risk|0|0|0| +1551|M|M|Secondary|1500|Low Risk|0|0|0| +1552|F|M|Secondary|1500|Low Risk|0|0|0| +1553|M|S|Secondary|1500|Low Risk|0|0|0| +1554|F|S|Secondary|1500|Low Risk|0|0|0| +1555|M|D|Secondary|1500|Low Risk|0|0|0| +1556|F|D|Secondary|1500|Low Risk|0|0|0| +1557|M|W|Secondary|1500|Low Risk|0|0|0| +1558|F|W|Secondary|1500|Low Risk|0|0|0| +1559|M|U|Secondary|1500|Low Risk|0|0|0| +1560|F|U|Secondary|1500|Low Risk|0|0|0| +1561|M|M|College|1500|Low Risk|0|0|0| +1562|F|M|College|1500|Low Risk|0|0|0| +1563|M|S|College|1500|Low Risk|0|0|0| +1564|F|S|College|1500|Low Risk|0|0|0| +1565|M|D|College|1500|Low Risk|0|0|0| +1566|F|D|College|1500|Low Risk|0|0|0| +1567|M|W|College|1500|Low Risk|0|0|0| +1568|F|W|College|1500|Low Risk|0|0|0| +1569|M|U|College|1500|Low Risk|0|0|0| +1570|F|U|College|1500|Low Risk|0|0|0| +1571|M|M|2 yr Degree|1500|Low Risk|0|0|0| +1572|F|M|2 yr Degree|1500|Low Risk|0|0|0| +1573|M|S|2 yr Degree|1500|Low Risk|0|0|0| +1574|F|S|2 yr Degree|1500|Low Risk|0|0|0| +1575|M|D|2 yr Degree|1500|Low Risk|0|0|0| +1576|F|D|2 yr Degree|1500|Low Risk|0|0|0| +1577|M|W|2 yr Degree|1500|Low Risk|0|0|0| +1578|F|W|2 yr Degree|1500|Low Risk|0|0|0| +1579|M|U|2 yr Degree|1500|Low Risk|0|0|0| +1580|F|U|2 yr Degree|1500|Low Risk|0|0|0| +1581|M|M|4 yr Degree|1500|Low Risk|0|0|0| +1582|F|M|4 yr Degree|1500|Low Risk|0|0|0| +1583|M|S|4 yr Degree|1500|Low Risk|0|0|0| +1584|F|S|4 yr Degree|1500|Low Risk|0|0|0| +1585|M|D|4 yr Degree|1500|Low Risk|0|0|0| +1586|F|D|4 yr Degree|1500|Low Risk|0|0|0| +1587|M|W|4 yr Degree|1500|Low Risk|0|0|0| +1588|F|W|4 yr Degree|1500|Low Risk|0|0|0| +1589|M|U|4 yr Degree|1500|Low Risk|0|0|0| +1590|F|U|4 yr Degree|1500|Low Risk|0|0|0| +1591|M|M|Advanced Degree|1500|Low Risk|0|0|0| +1592|F|M|Advanced Degree|1500|Low Risk|0|0|0| +1593|M|S|Advanced Degree|1500|Low Risk|0|0|0| +1594|F|S|Advanced Degree|1500|Low Risk|0|0|0| +1595|M|D|Advanced Degree|1500|Low Risk|0|0|0| +1596|F|D|Advanced Degree|1500|Low Risk|0|0|0| +1597|M|W|Advanced Degree|1500|Low Risk|0|0|0| +1598|F|W|Advanced Degree|1500|Low Risk|0|0|0| +1599|M|U|Advanced Degree|1500|Low Risk|0|0|0| +1600|F|U|Advanced Degree|1500|Low Risk|0|0|0| +1601|M|M|Unknown|1500|Low Risk|0|0|0| +1602|F|M|Unknown|1500|Low Risk|0|0|0| +1603|M|S|Unknown|1500|Low Risk|0|0|0| +1604|F|S|Unknown|1500|Low Risk|0|0|0| +1605|M|D|Unknown|1500|Low Risk|0|0|0| +1606|F|D|Unknown|1500|Low Risk|0|0|0| +1607|M|W|Unknown|1500|Low Risk|0|0|0| +1608|F|W|Unknown|1500|Low Risk|0|0|0| +1609|M|U|Unknown|1500|Low Risk|0|0|0| +1610|F|U|Unknown|1500|Low Risk|0|0|0| +1611|M|M|Primary|2000|Low Risk|0|0|0| +1612|F|M|Primary|2000|Low Risk|0|0|0| +1613|M|S|Primary|2000|Low Risk|0|0|0| +1614|F|S|Primary|2000|Low Risk|0|0|0| +1615|M|D|Primary|2000|Low Risk|0|0|0| +1616|F|D|Primary|2000|Low Risk|0|0|0| +1617|M|W|Primary|2000|Low Risk|0|0|0| +1618|F|W|Primary|2000|Low Risk|0|0|0| +1619|M|U|Primary|2000|Low Risk|0|0|0| +1620|F|U|Primary|2000|Low Risk|0|0|0| +1621|M|M|Secondary|2000|Low Risk|0|0|0| +1622|F|M|Secondary|2000|Low Risk|0|0|0| +1623|M|S|Secondary|2000|Low Risk|0|0|0| +1624|F|S|Secondary|2000|Low Risk|0|0|0| +1625|M|D|Secondary|2000|Low Risk|0|0|0| +1626|F|D|Secondary|2000|Low Risk|0|0|0| +1627|M|W|Secondary|2000|Low Risk|0|0|0| +1628|F|W|Secondary|2000|Low Risk|0|0|0| +1629|M|U|Secondary|2000|Low Risk|0|0|0| +1630|F|U|Secondary|2000|Low Risk|0|0|0| +1631|M|M|College|2000|Low Risk|0|0|0| +1632|F|M|College|2000|Low Risk|0|0|0| +1633|M|S|College|2000|Low Risk|0|0|0| +1634|F|S|College|2000|Low Risk|0|0|0| +1635|M|D|College|2000|Low Risk|0|0|0| +1636|F|D|College|2000|Low Risk|0|0|0| +1637|M|W|College|2000|Low Risk|0|0|0| +1638|F|W|College|2000|Low Risk|0|0|0| +1639|M|U|College|2000|Low Risk|0|0|0| +1640|F|U|College|2000|Low Risk|0|0|0| +1641|M|M|2 yr Degree|2000|Low Risk|0|0|0| +1642|F|M|2 yr Degree|2000|Low Risk|0|0|0| +1643|M|S|2 yr Degree|2000|Low Risk|0|0|0| +1644|F|S|2 yr Degree|2000|Low Risk|0|0|0| +1645|M|D|2 yr Degree|2000|Low Risk|0|0|0| +1646|F|D|2 yr Degree|2000|Low Risk|0|0|0| +1647|M|W|2 yr Degree|2000|Low Risk|0|0|0| +1648|F|W|2 yr Degree|2000|Low Risk|0|0|0| +1649|M|U|2 yr Degree|2000|Low Risk|0|0|0| +1650|F|U|2 yr Degree|2000|Low Risk|0|0|0| +1651|M|M|4 yr Degree|2000|Low Risk|0|0|0| +1652|F|M|4 yr Degree|2000|Low Risk|0|0|0| +1653|M|S|4 yr Degree|2000|Low Risk|0|0|0| +1654|F|S|4 yr Degree|2000|Low Risk|0|0|0| +1655|M|D|4 yr Degree|2000|Low Risk|0|0|0| +1656|F|D|4 yr Degree|2000|Low Risk|0|0|0| +1657|M|W|4 yr Degree|2000|Low Risk|0|0|0| +1658|F|W|4 yr Degree|2000|Low Risk|0|0|0| +1659|M|U|4 yr Degree|2000|Low Risk|0|0|0| +1660|F|U|4 yr Degree|2000|Low Risk|0|0|0| +1661|M|M|Advanced Degree|2000|Low Risk|0|0|0| +1662|F|M|Advanced Degree|2000|Low Risk|0|0|0| +1663|M|S|Advanced Degree|2000|Low Risk|0|0|0| +1664|F|S|Advanced Degree|2000|Low Risk|0|0|0| +1665|M|D|Advanced Degree|2000|Low Risk|0|0|0| +1666|F|D|Advanced Degree|2000|Low Risk|0|0|0| +1667|M|W|Advanced Degree|2000|Low Risk|0|0|0| +1668|F|W|Advanced Degree|2000|Low Risk|0|0|0| +1669|M|U|Advanced Degree|2000|Low Risk|0|0|0| +1670|F|U|Advanced Degree|2000|Low Risk|0|0|0| +1671|M|M|Unknown|2000|Low Risk|0|0|0| +1672|F|M|Unknown|2000|Low Risk|0|0|0| +1673|M|S|Unknown|2000|Low Risk|0|0|0| +1674|F|S|Unknown|2000|Low Risk|0|0|0| +1675|M|D|Unknown|2000|Low Risk|0|0|0| +1676|F|D|Unknown|2000|Low Risk|0|0|0| +1677|M|W|Unknown|2000|Low Risk|0|0|0| +1678|F|W|Unknown|2000|Low Risk|0|0|0| +1679|M|U|Unknown|2000|Low Risk|0|0|0| +1680|F|U|Unknown|2000|Low Risk|0|0|0| +1681|M|M|Primary|2500|Low Risk|0|0|0| +1682|F|M|Primary|2500|Low Risk|0|0|0| +1683|M|S|Primary|2500|Low Risk|0|0|0| +1684|F|S|Primary|2500|Low Risk|0|0|0| +1685|M|D|Primary|2500|Low Risk|0|0|0| +1686|F|D|Primary|2500|Low Risk|0|0|0| +1687|M|W|Primary|2500|Low Risk|0|0|0| +1688|F|W|Primary|2500|Low Risk|0|0|0| +1689|M|U|Primary|2500|Low Risk|0|0|0| +1690|F|U|Primary|2500|Low Risk|0|0|0| +1691|M|M|Secondary|2500|Low Risk|0|0|0| +1692|F|M|Secondary|2500|Low Risk|0|0|0| +1693|M|S|Secondary|2500|Low Risk|0|0|0| +1694|F|S|Secondary|2500|Low Risk|0|0|0| +1695|M|D|Secondary|2500|Low Risk|0|0|0| +1696|F|D|Secondary|2500|Low Risk|0|0|0| +1697|M|W|Secondary|2500|Low Risk|0|0|0| +1698|F|W|Secondary|2500|Low Risk|0|0|0| +1699|M|U|Secondary|2500|Low Risk|0|0|0| +1700|F|U|Secondary|2500|Low Risk|0|0|0| +1701|M|M|College|2500|Low Risk|0|0|0| +1702|F|M|College|2500|Low Risk|0|0|0| +1703|M|S|College|2500|Low Risk|0|0|0| +1704|F|S|College|2500|Low Risk|0|0|0| +1705|M|D|College|2500|Low Risk|0|0|0| +1706|F|D|College|2500|Low Risk|0|0|0| +1707|M|W|College|2500|Low Risk|0|0|0| +1708|F|W|College|2500|Low Risk|0|0|0| +1709|M|U|College|2500|Low Risk|0|0|0| +1710|F|U|College|2500|Low Risk|0|0|0| +1711|M|M|2 yr Degree|2500|Low Risk|0|0|0| +1712|F|M|2 yr Degree|2500|Low Risk|0|0|0| +1713|M|S|2 yr Degree|2500|Low Risk|0|0|0| +1714|F|S|2 yr Degree|2500|Low Risk|0|0|0| +1715|M|D|2 yr Degree|2500|Low Risk|0|0|0| +1716|F|D|2 yr Degree|2500|Low Risk|0|0|0| +1717|M|W|2 yr Degree|2500|Low Risk|0|0|0| +1718|F|W|2 yr Degree|2500|Low Risk|0|0|0| +1719|M|U|2 yr Degree|2500|Low Risk|0|0|0| +1720|F|U|2 yr Degree|2500|Low Risk|0|0|0| +1721|M|M|4 yr Degree|2500|Low Risk|0|0|0| +1722|F|M|4 yr Degree|2500|Low Risk|0|0|0| +1723|M|S|4 yr Degree|2500|Low Risk|0|0|0| +1724|F|S|4 yr Degree|2500|Low Risk|0|0|0| +1725|M|D|4 yr Degree|2500|Low Risk|0|0|0| +1726|F|D|4 yr Degree|2500|Low Risk|0|0|0| +1727|M|W|4 yr Degree|2500|Low Risk|0|0|0| +1728|F|W|4 yr Degree|2500|Low Risk|0|0|0| +1729|M|U|4 yr Degree|2500|Low Risk|0|0|0| +1730|F|U|4 yr Degree|2500|Low Risk|0|0|0| +1731|M|M|Advanced Degree|2500|Low Risk|0|0|0| +1732|F|M|Advanced Degree|2500|Low Risk|0|0|0| +1733|M|S|Advanced Degree|2500|Low Risk|0|0|0| +1734|F|S|Advanced Degree|2500|Low Risk|0|0|0| +1735|M|D|Advanced Degree|2500|Low Risk|0|0|0| +1736|F|D|Advanced Degree|2500|Low Risk|0|0|0| +1737|M|W|Advanced Degree|2500|Low Risk|0|0|0| +1738|F|W|Advanced Degree|2500|Low Risk|0|0|0| +1739|M|U|Advanced Degree|2500|Low Risk|0|0|0| +1740|F|U|Advanced Degree|2500|Low Risk|0|0|0| +1741|M|M|Unknown|2500|Low Risk|0|0|0| +1742|F|M|Unknown|2500|Low Risk|0|0|0| +1743|M|S|Unknown|2500|Low Risk|0|0|0| +1744|F|S|Unknown|2500|Low Risk|0|0|0| +1745|M|D|Unknown|2500|Low Risk|0|0|0| +1746|F|D|Unknown|2500|Low Risk|0|0|0| +1747|M|W|Unknown|2500|Low Risk|0|0|0| +1748|F|W|Unknown|2500|Low Risk|0|0|0| +1749|M|U|Unknown|2500|Low Risk|0|0|0| +1750|F|U|Unknown|2500|Low Risk|0|0|0| +1751|M|M|Primary|3000|Low Risk|0|0|0| +1752|F|M|Primary|3000|Low Risk|0|0|0| +1753|M|S|Primary|3000|Low Risk|0|0|0| +1754|F|S|Primary|3000|Low Risk|0|0|0| +1755|M|D|Primary|3000|Low Risk|0|0|0| +1756|F|D|Primary|3000|Low Risk|0|0|0| +1757|M|W|Primary|3000|Low Risk|0|0|0| +1758|F|W|Primary|3000|Low Risk|0|0|0| +1759|M|U|Primary|3000|Low Risk|0|0|0| +1760|F|U|Primary|3000|Low Risk|0|0|0| +1761|M|M|Secondary|3000|Low Risk|0|0|0| +1762|F|M|Secondary|3000|Low Risk|0|0|0| +1763|M|S|Secondary|3000|Low Risk|0|0|0| +1764|F|S|Secondary|3000|Low Risk|0|0|0| +1765|M|D|Secondary|3000|Low Risk|0|0|0| +1766|F|D|Secondary|3000|Low Risk|0|0|0| +1767|M|W|Secondary|3000|Low Risk|0|0|0| +1768|F|W|Secondary|3000|Low Risk|0|0|0| +1769|M|U|Secondary|3000|Low Risk|0|0|0| +1770|F|U|Secondary|3000|Low Risk|0|0|0| +1771|M|M|College|3000|Low Risk|0|0|0| +1772|F|M|College|3000|Low Risk|0|0|0| +1773|M|S|College|3000|Low Risk|0|0|0| +1774|F|S|College|3000|Low Risk|0|0|0| +1775|M|D|College|3000|Low Risk|0|0|0| +1776|F|D|College|3000|Low Risk|0|0|0| +1777|M|W|College|3000|Low Risk|0|0|0| +1778|F|W|College|3000|Low Risk|0|0|0| +1779|M|U|College|3000|Low Risk|0|0|0| +1780|F|U|College|3000|Low Risk|0|0|0| +1781|M|M|2 yr Degree|3000|Low Risk|0|0|0| +1782|F|M|2 yr Degree|3000|Low Risk|0|0|0| +1783|M|S|2 yr Degree|3000|Low Risk|0|0|0| +1784|F|S|2 yr Degree|3000|Low Risk|0|0|0| +1785|M|D|2 yr Degree|3000|Low Risk|0|0|0| +1786|F|D|2 yr Degree|3000|Low Risk|0|0|0| +1787|M|W|2 yr Degree|3000|Low Risk|0|0|0| +1788|F|W|2 yr Degree|3000|Low Risk|0|0|0| +1789|M|U|2 yr Degree|3000|Low Risk|0|0|0| +1790|F|U|2 yr Degree|3000|Low Risk|0|0|0| +1791|M|M|4 yr Degree|3000|Low Risk|0|0|0| +1792|F|M|4 yr Degree|3000|Low Risk|0|0|0| +1793|M|S|4 yr Degree|3000|Low Risk|0|0|0| +1794|F|S|4 yr Degree|3000|Low Risk|0|0|0| +1795|M|D|4 yr Degree|3000|Low Risk|0|0|0| +1796|F|D|4 yr Degree|3000|Low Risk|0|0|0| +1797|M|W|4 yr Degree|3000|Low Risk|0|0|0| +1798|F|W|4 yr Degree|3000|Low Risk|0|0|0| +1799|M|U|4 yr Degree|3000|Low Risk|0|0|0| +1800|F|U|4 yr Degree|3000|Low Risk|0|0|0| +1801|M|M|Advanced Degree|3000|Low Risk|0|0|0| +1802|F|M|Advanced Degree|3000|Low Risk|0|0|0| +1803|M|S|Advanced Degree|3000|Low Risk|0|0|0| +1804|F|S|Advanced Degree|3000|Low Risk|0|0|0| +1805|M|D|Advanced Degree|3000|Low Risk|0|0|0| +1806|F|D|Advanced Degree|3000|Low Risk|0|0|0| +1807|M|W|Advanced Degree|3000|Low Risk|0|0|0| +1808|F|W|Advanced Degree|3000|Low Risk|0|0|0| +1809|M|U|Advanced Degree|3000|Low Risk|0|0|0| +1810|F|U|Advanced Degree|3000|Low Risk|0|0|0| +1811|M|M|Unknown|3000|Low Risk|0|0|0| +1812|F|M|Unknown|3000|Low Risk|0|0|0| +1813|M|S|Unknown|3000|Low Risk|0|0|0| +1814|F|S|Unknown|3000|Low Risk|0|0|0| +1815|M|D|Unknown|3000|Low Risk|0|0|0| +1816|F|D|Unknown|3000|Low Risk|0|0|0| +1817|M|W|Unknown|3000|Low Risk|0|0|0| +1818|F|W|Unknown|3000|Low Risk|0|0|0| +1819|M|U|Unknown|3000|Low Risk|0|0|0| +1820|F|U|Unknown|3000|Low Risk|0|0|0| +1821|M|M|Primary|3500|Low Risk|0|0|0| +1822|F|M|Primary|3500|Low Risk|0|0|0| +1823|M|S|Primary|3500|Low Risk|0|0|0| +1824|F|S|Primary|3500|Low Risk|0|0|0| +1825|M|D|Primary|3500|Low Risk|0|0|0| +1826|F|D|Primary|3500|Low Risk|0|0|0| +1827|M|W|Primary|3500|Low Risk|0|0|0| +1828|F|W|Primary|3500|Low Risk|0|0|0| +1829|M|U|Primary|3500|Low Risk|0|0|0| +1830|F|U|Primary|3500|Low Risk|0|0|0| +1831|M|M|Secondary|3500|Low Risk|0|0|0| +1832|F|M|Secondary|3500|Low Risk|0|0|0| +1833|M|S|Secondary|3500|Low Risk|0|0|0| +1834|F|S|Secondary|3500|Low Risk|0|0|0| +1835|M|D|Secondary|3500|Low Risk|0|0|0| +1836|F|D|Secondary|3500|Low Risk|0|0|0| +1837|M|W|Secondary|3500|Low Risk|0|0|0| +1838|F|W|Secondary|3500|Low Risk|0|0|0| +1839|M|U|Secondary|3500|Low Risk|0|0|0| +1840|F|U|Secondary|3500|Low Risk|0|0|0| +1841|M|M|College|3500|Low Risk|0|0|0| +1842|F|M|College|3500|Low Risk|0|0|0| +1843|M|S|College|3500|Low Risk|0|0|0| +1844|F|S|College|3500|Low Risk|0|0|0| +1845|M|D|College|3500|Low Risk|0|0|0| +1846|F|D|College|3500|Low Risk|0|0|0| +1847|M|W|College|3500|Low Risk|0|0|0| +1848|F|W|College|3500|Low Risk|0|0|0| +1849|M|U|College|3500|Low Risk|0|0|0| +1850|F|U|College|3500|Low Risk|0|0|0| +1851|M|M|2 yr Degree|3500|Low Risk|0|0|0| +1852|F|M|2 yr Degree|3500|Low Risk|0|0|0| +1853|M|S|2 yr Degree|3500|Low Risk|0|0|0| +1854|F|S|2 yr Degree|3500|Low Risk|0|0|0| +1855|M|D|2 yr Degree|3500|Low Risk|0|0|0| +1856|F|D|2 yr Degree|3500|Low Risk|0|0|0| +1857|M|W|2 yr Degree|3500|Low Risk|0|0|0| +1858|F|W|2 yr Degree|3500|Low Risk|0|0|0| +1859|M|U|2 yr Degree|3500|Low Risk|0|0|0| +1860|F|U|2 yr Degree|3500|Low Risk|0|0|0| +1861|M|M|4 yr Degree|3500|Low Risk|0|0|0| +1862|F|M|4 yr Degree|3500|Low Risk|0|0|0| +1863|M|S|4 yr Degree|3500|Low Risk|0|0|0| +1864|F|S|4 yr Degree|3500|Low Risk|0|0|0| +1865|M|D|4 yr Degree|3500|Low Risk|0|0|0| +1866|F|D|4 yr Degree|3500|Low Risk|0|0|0| +1867|M|W|4 yr Degree|3500|Low Risk|0|0|0| +1868|F|W|4 yr Degree|3500|Low Risk|0|0|0| +1869|M|U|4 yr Degree|3500|Low Risk|0|0|0| +1870|F|U|4 yr Degree|3500|Low Risk|0|0|0| +1871|M|M|Advanced Degree|3500|Low Risk|0|0|0| +1872|F|M|Advanced Degree|3500|Low Risk|0|0|0| +1873|M|S|Advanced Degree|3500|Low Risk|0|0|0| +1874|F|S|Advanced Degree|3500|Low Risk|0|0|0| +1875|M|D|Advanced Degree|3500|Low Risk|0|0|0| +1876|F|D|Advanced Degree|3500|Low Risk|0|0|0| +1877|M|W|Advanced Degree|3500|Low Risk|0|0|0| +1878|F|W|Advanced Degree|3500|Low Risk|0|0|0| +1879|M|U|Advanced Degree|3500|Low Risk|0|0|0| +1880|F|U|Advanced Degree|3500|Low Risk|0|0|0| +1881|M|M|Unknown|3500|Low Risk|0|0|0| +1882|F|M|Unknown|3500|Low Risk|0|0|0| +1883|M|S|Unknown|3500|Low Risk|0|0|0| +1884|F|S|Unknown|3500|Low Risk|0|0|0| +1885|M|D|Unknown|3500|Low Risk|0|0|0| +1886|F|D|Unknown|3500|Low Risk|0|0|0| +1887|M|W|Unknown|3500|Low Risk|0|0|0| +1888|F|W|Unknown|3500|Low Risk|0|0|0| +1889|M|U|Unknown|3500|Low Risk|0|0|0| +1890|F|U|Unknown|3500|Low Risk|0|0|0| +1891|M|M|Primary|4000|Low Risk|0|0|0| +1892|F|M|Primary|4000|Low Risk|0|0|0| +1893|M|S|Primary|4000|Low Risk|0|0|0| +1894|F|S|Primary|4000|Low Risk|0|0|0| +1895|M|D|Primary|4000|Low Risk|0|0|0| +1896|F|D|Primary|4000|Low Risk|0|0|0| +1897|M|W|Primary|4000|Low Risk|0|0|0| +1898|F|W|Primary|4000|Low Risk|0|0|0| +1899|M|U|Primary|4000|Low Risk|0|0|0| +1900|F|U|Primary|4000|Low Risk|0|0|0| +1901|M|M|Secondary|4000|Low Risk|0|0|0| +1902|F|M|Secondary|4000|Low Risk|0|0|0| +1903|M|S|Secondary|4000|Low Risk|0|0|0| +1904|F|S|Secondary|4000|Low Risk|0|0|0| +1905|M|D|Secondary|4000|Low Risk|0|0|0| +1906|F|D|Secondary|4000|Low Risk|0|0|0| +1907|M|W|Secondary|4000|Low Risk|0|0|0| +1908|F|W|Secondary|4000|Low Risk|0|0|0| +1909|M|U|Secondary|4000|Low Risk|0|0|0| +1910|F|U|Secondary|4000|Low Risk|0|0|0| +1911|M|M|College|4000|Low Risk|0|0|0| +1912|F|M|College|4000|Low Risk|0|0|0| +1913|M|S|College|4000|Low Risk|0|0|0| +1914|F|S|College|4000|Low Risk|0|0|0| +1915|M|D|College|4000|Low Risk|0|0|0| +1916|F|D|College|4000|Low Risk|0|0|0| +1917|M|W|College|4000|Low Risk|0|0|0| +1918|F|W|College|4000|Low Risk|0|0|0| +1919|M|U|College|4000|Low Risk|0|0|0| +1920|F|U|College|4000|Low Risk|0|0|0| +1921|M|M|2 yr Degree|4000|Low Risk|0|0|0| +1922|F|M|2 yr Degree|4000|Low Risk|0|0|0| +1923|M|S|2 yr Degree|4000|Low Risk|0|0|0| +1924|F|S|2 yr Degree|4000|Low Risk|0|0|0| +1925|M|D|2 yr Degree|4000|Low Risk|0|0|0| +1926|F|D|2 yr Degree|4000|Low Risk|0|0|0| +1927|M|W|2 yr Degree|4000|Low Risk|0|0|0| +1928|F|W|2 yr Degree|4000|Low Risk|0|0|0| +1929|M|U|2 yr Degree|4000|Low Risk|0|0|0| +1930|F|U|2 yr Degree|4000|Low Risk|0|0|0| +1931|M|M|4 yr Degree|4000|Low Risk|0|0|0| +1932|F|M|4 yr Degree|4000|Low Risk|0|0|0| +1933|M|S|4 yr Degree|4000|Low Risk|0|0|0| +1934|F|S|4 yr Degree|4000|Low Risk|0|0|0| +1935|M|D|4 yr Degree|4000|Low Risk|0|0|0| +1936|F|D|4 yr Degree|4000|Low Risk|0|0|0| +1937|M|W|4 yr Degree|4000|Low Risk|0|0|0| +1938|F|W|4 yr Degree|4000|Low Risk|0|0|0| +1939|M|U|4 yr Degree|4000|Low Risk|0|0|0| +1940|F|U|4 yr Degree|4000|Low Risk|0|0|0| +1941|M|M|Advanced Degree|4000|Low Risk|0|0|0| +1942|F|M|Advanced Degree|4000|Low Risk|0|0|0| +1943|M|S|Advanced Degree|4000|Low Risk|0|0|0| +1944|F|S|Advanced Degree|4000|Low Risk|0|0|0| +1945|M|D|Advanced Degree|4000|Low Risk|0|0|0| +1946|F|D|Advanced Degree|4000|Low Risk|0|0|0| +1947|M|W|Advanced Degree|4000|Low Risk|0|0|0| +1948|F|W|Advanced Degree|4000|Low Risk|0|0|0| +1949|M|U|Advanced Degree|4000|Low Risk|0|0|0| +1950|F|U|Advanced Degree|4000|Low Risk|0|0|0| +1951|M|M|Unknown|4000|Low Risk|0|0|0| +1952|F|M|Unknown|4000|Low Risk|0|0|0| +1953|M|S|Unknown|4000|Low Risk|0|0|0| +1954|F|S|Unknown|4000|Low Risk|0|0|0| +1955|M|D|Unknown|4000|Low Risk|0|0|0| +1956|F|D|Unknown|4000|Low Risk|0|0|0| +1957|M|W|Unknown|4000|Low Risk|0|0|0| +1958|F|W|Unknown|4000|Low Risk|0|0|0| +1959|M|U|Unknown|4000|Low Risk|0|0|0| +1960|F|U|Unknown|4000|Low Risk|0|0|0| +1961|M|M|Primary|4500|Low Risk|0|0|0| +1962|F|M|Primary|4500|Low Risk|0|0|0| +1963|M|S|Primary|4500|Low Risk|0|0|0| +1964|F|S|Primary|4500|Low Risk|0|0|0| +1965|M|D|Primary|4500|Low Risk|0|0|0| +1966|F|D|Primary|4500|Low Risk|0|0|0| +1967|M|W|Primary|4500|Low Risk|0|0|0| +1968|F|W|Primary|4500|Low Risk|0|0|0| +1969|M|U|Primary|4500|Low Risk|0|0|0| +1970|F|U|Primary|4500|Low Risk|0|0|0| +1971|M|M|Secondary|4500|Low Risk|0|0|0| +1972|F|M|Secondary|4500|Low Risk|0|0|0| +1973|M|S|Secondary|4500|Low Risk|0|0|0| +1974|F|S|Secondary|4500|Low Risk|0|0|0| +1975|M|D|Secondary|4500|Low Risk|0|0|0| +1976|F|D|Secondary|4500|Low Risk|0|0|0| +1977|M|W|Secondary|4500|Low Risk|0|0|0| +1978|F|W|Secondary|4500|Low Risk|0|0|0| +1979|M|U|Secondary|4500|Low Risk|0|0|0| +1980|F|U|Secondary|4500|Low Risk|0|0|0| +1981|M|M|College|4500|Low Risk|0|0|0| +1982|F|M|College|4500|Low Risk|0|0|0| +1983|M|S|College|4500|Low Risk|0|0|0| +1984|F|S|College|4500|Low Risk|0|0|0| +1985|M|D|College|4500|Low Risk|0|0|0| +1986|F|D|College|4500|Low Risk|0|0|0| +1987|M|W|College|4500|Low Risk|0|0|0| +1988|F|W|College|4500|Low Risk|0|0|0| +1989|M|U|College|4500|Low Risk|0|0|0| +1990|F|U|College|4500|Low Risk|0|0|0| +1991|M|M|2 yr Degree|4500|Low Risk|0|0|0| +1992|F|M|2 yr Degree|4500|Low Risk|0|0|0| +1993|M|S|2 yr Degree|4500|Low Risk|0|0|0| +1994|F|S|2 yr Degree|4500|Low Risk|0|0|0| +1995|M|D|2 yr Degree|4500|Low Risk|0|0|0| +1996|F|D|2 yr Degree|4500|Low Risk|0|0|0| +1997|M|W|2 yr Degree|4500|Low Risk|0|0|0| +1998|F|W|2 yr Degree|4500|Low Risk|0|0|0| +1999|M|U|2 yr Degree|4500|Low Risk|0|0|0| +2000|F|U|2 yr Degree|4500|Low Risk|0|0|0| +2001|M|M|4 yr Degree|4500|Low Risk|0|0|0| +2002|F|M|4 yr Degree|4500|Low Risk|0|0|0| +2003|M|S|4 yr Degree|4500|Low Risk|0|0|0| +2004|F|S|4 yr Degree|4500|Low Risk|0|0|0| +2005|M|D|4 yr Degree|4500|Low Risk|0|0|0| +2006|F|D|4 yr Degree|4500|Low Risk|0|0|0| +2007|M|W|4 yr Degree|4500|Low Risk|0|0|0| +2008|F|W|4 yr Degree|4500|Low Risk|0|0|0| +2009|M|U|4 yr Degree|4500|Low Risk|0|0|0| +2010|F|U|4 yr Degree|4500|Low Risk|0|0|0| +2011|M|M|Advanced Degree|4500|Low Risk|0|0|0| +2012|F|M|Advanced Degree|4500|Low Risk|0|0|0| +2013|M|S|Advanced Degree|4500|Low Risk|0|0|0| +2014|F|S|Advanced Degree|4500|Low Risk|0|0|0| +2015|M|D|Advanced Degree|4500|Low Risk|0|0|0| +2016|F|D|Advanced Degree|4500|Low Risk|0|0|0| +2017|M|W|Advanced Degree|4500|Low Risk|0|0|0| +2018|F|W|Advanced Degree|4500|Low Risk|0|0|0| +2019|M|U|Advanced Degree|4500|Low Risk|0|0|0| +2020|F|U|Advanced Degree|4500|Low Risk|0|0|0| +2021|M|M|Unknown|4500|Low Risk|0|0|0| +2022|F|M|Unknown|4500|Low Risk|0|0|0| +2023|M|S|Unknown|4500|Low Risk|0|0|0| +2024|F|S|Unknown|4500|Low Risk|0|0|0| +2025|M|D|Unknown|4500|Low Risk|0|0|0| +2026|F|D|Unknown|4500|Low Risk|0|0|0| +2027|M|W|Unknown|4500|Low Risk|0|0|0| +2028|F|W|Unknown|4500|Low Risk|0|0|0| +2029|M|U|Unknown|4500|Low Risk|0|0|0| +2030|F|U|Unknown|4500|Low Risk|0|0|0| +2031|M|M|Primary|5000|Low Risk|0|0|0| +2032|F|M|Primary|5000|Low Risk|0|0|0| +2033|M|S|Primary|5000|Low Risk|0|0|0| +2034|F|S|Primary|5000|Low Risk|0|0|0| +2035|M|D|Primary|5000|Low Risk|0|0|0| +2036|F|D|Primary|5000|Low Risk|0|0|0| +2037|M|W|Primary|5000|Low Risk|0|0|0| +2038|F|W|Primary|5000|Low Risk|0|0|0| +2039|M|U|Primary|5000|Low Risk|0|0|0| +2040|F|U|Primary|5000|Low Risk|0|0|0| +2041|M|M|Secondary|5000|Low Risk|0|0|0| +2042|F|M|Secondary|5000|Low Risk|0|0|0| +2043|M|S|Secondary|5000|Low Risk|0|0|0| +2044|F|S|Secondary|5000|Low Risk|0|0|0| +2045|M|D|Secondary|5000|Low Risk|0|0|0| +2046|F|D|Secondary|5000|Low Risk|0|0|0| +2047|M|W|Secondary|5000|Low Risk|0|0|0| +2048|F|W|Secondary|5000|Low Risk|0|0|0| +2049|M|U|Secondary|5000|Low Risk|0|0|0| +2050|F|U|Secondary|5000|Low Risk|0|0|0| +2051|M|M|College|5000|Low Risk|0|0|0| +2052|F|M|College|5000|Low Risk|0|0|0| +2053|M|S|College|5000|Low Risk|0|0|0| +2054|F|S|College|5000|Low Risk|0|0|0| +2055|M|D|College|5000|Low Risk|0|0|0| +2056|F|D|College|5000|Low Risk|0|0|0| +2057|M|W|College|5000|Low Risk|0|0|0| +2058|F|W|College|5000|Low Risk|0|0|0| +2059|M|U|College|5000|Low Risk|0|0|0| +2060|F|U|College|5000|Low Risk|0|0|0| +2061|M|M|2 yr Degree|5000|Low Risk|0|0|0| +2062|F|M|2 yr Degree|5000|Low Risk|0|0|0| +2063|M|S|2 yr Degree|5000|Low Risk|0|0|0| +2064|F|S|2 yr Degree|5000|Low Risk|0|0|0| +2065|M|D|2 yr Degree|5000|Low Risk|0|0|0| +2066|F|D|2 yr Degree|5000|Low Risk|0|0|0| +2067|M|W|2 yr Degree|5000|Low Risk|0|0|0| +2068|F|W|2 yr Degree|5000|Low Risk|0|0|0| +2069|M|U|2 yr Degree|5000|Low Risk|0|0|0| +2070|F|U|2 yr Degree|5000|Low Risk|0|0|0| +2071|M|M|4 yr Degree|5000|Low Risk|0|0|0| +2072|F|M|4 yr Degree|5000|Low Risk|0|0|0| +2073|M|S|4 yr Degree|5000|Low Risk|0|0|0| +2074|F|S|4 yr Degree|5000|Low Risk|0|0|0| +2075|M|D|4 yr Degree|5000|Low Risk|0|0|0| +2076|F|D|4 yr Degree|5000|Low Risk|0|0|0| +2077|M|W|4 yr Degree|5000|Low Risk|0|0|0| +2078|F|W|4 yr Degree|5000|Low Risk|0|0|0| +2079|M|U|4 yr Degree|5000|Low Risk|0|0|0| +2080|F|U|4 yr Degree|5000|Low Risk|0|0|0| +2081|M|M|Advanced Degree|5000|Low Risk|0|0|0| +2082|F|M|Advanced Degree|5000|Low Risk|0|0|0| +2083|M|S|Advanced Degree|5000|Low Risk|0|0|0| +2084|F|S|Advanced Degree|5000|Low Risk|0|0|0| +2085|M|D|Advanced Degree|5000|Low Risk|0|0|0| +2086|F|D|Advanced Degree|5000|Low Risk|0|0|0| +2087|M|W|Advanced Degree|5000|Low Risk|0|0|0| +2088|F|W|Advanced Degree|5000|Low Risk|0|0|0| +2089|M|U|Advanced Degree|5000|Low Risk|0|0|0| +2090|F|U|Advanced Degree|5000|Low Risk|0|0|0| +2091|M|M|Unknown|5000|Low Risk|0|0|0| +2092|F|M|Unknown|5000|Low Risk|0|0|0| +2093|M|S|Unknown|5000|Low Risk|0|0|0| +2094|F|S|Unknown|5000|Low Risk|0|0|0| +2095|M|D|Unknown|5000|Low Risk|0|0|0| +2096|F|D|Unknown|5000|Low Risk|0|0|0| +2097|M|W|Unknown|5000|Low Risk|0|0|0| +2098|F|W|Unknown|5000|Low Risk|0|0|0| +2099|M|U|Unknown|5000|Low Risk|0|0|0| +2100|F|U|Unknown|5000|Low Risk|0|0|0| +2101|M|M|Primary|5500|Low Risk|0|0|0| +2102|F|M|Primary|5500|Low Risk|0|0|0| +2103|M|S|Primary|5500|Low Risk|0|0|0| +2104|F|S|Primary|5500|Low Risk|0|0|0| +2105|M|D|Primary|5500|Low Risk|0|0|0| +2106|F|D|Primary|5500|Low Risk|0|0|0| +2107|M|W|Primary|5500|Low Risk|0|0|0| +2108|F|W|Primary|5500|Low Risk|0|0|0| +2109|M|U|Primary|5500|Low Risk|0|0|0| +2110|F|U|Primary|5500|Low Risk|0|0|0| +2111|M|M|Secondary|5500|Low Risk|0|0|0| +2112|F|M|Secondary|5500|Low Risk|0|0|0| +2113|M|S|Secondary|5500|Low Risk|0|0|0| +2114|F|S|Secondary|5500|Low Risk|0|0|0| +2115|M|D|Secondary|5500|Low Risk|0|0|0| +2116|F|D|Secondary|5500|Low Risk|0|0|0| +2117|M|W|Secondary|5500|Low Risk|0|0|0| +2118|F|W|Secondary|5500|Low Risk|0|0|0| +2119|M|U|Secondary|5500|Low Risk|0|0|0| +2120|F|U|Secondary|5500|Low Risk|0|0|0| +2121|M|M|College|5500|Low Risk|0|0|0| +2122|F|M|College|5500|Low Risk|0|0|0| +2123|M|S|College|5500|Low Risk|0|0|0| +2124|F|S|College|5500|Low Risk|0|0|0| +2125|M|D|College|5500|Low Risk|0|0|0| +2126|F|D|College|5500|Low Risk|0|0|0| +2127|M|W|College|5500|Low Risk|0|0|0| +2128|F|W|College|5500|Low Risk|0|0|0| +2129|M|U|College|5500|Low Risk|0|0|0| +2130|F|U|College|5500|Low Risk|0|0|0| +2131|M|M|2 yr Degree|5500|Low Risk|0|0|0| +2132|F|M|2 yr Degree|5500|Low Risk|0|0|0| +2133|M|S|2 yr Degree|5500|Low Risk|0|0|0| +2134|F|S|2 yr Degree|5500|Low Risk|0|0|0| +2135|M|D|2 yr Degree|5500|Low Risk|0|0|0| +2136|F|D|2 yr Degree|5500|Low Risk|0|0|0| +2137|M|W|2 yr Degree|5500|Low Risk|0|0|0| +2138|F|W|2 yr Degree|5500|Low Risk|0|0|0| +2139|M|U|2 yr Degree|5500|Low Risk|0|0|0| +2140|F|U|2 yr Degree|5500|Low Risk|0|0|0| +2141|M|M|4 yr Degree|5500|Low Risk|0|0|0| +2142|F|M|4 yr Degree|5500|Low Risk|0|0|0| +2143|M|S|4 yr Degree|5500|Low Risk|0|0|0| +2144|F|S|4 yr Degree|5500|Low Risk|0|0|0| +2145|M|D|4 yr Degree|5500|Low Risk|0|0|0| +2146|F|D|4 yr Degree|5500|Low Risk|0|0|0| +2147|M|W|4 yr Degree|5500|Low Risk|0|0|0| +2148|F|W|4 yr Degree|5500|Low Risk|0|0|0| +2149|M|U|4 yr Degree|5500|Low Risk|0|0|0| +2150|F|U|4 yr Degree|5500|Low Risk|0|0|0| +2151|M|M|Advanced Degree|5500|Low Risk|0|0|0| +2152|F|M|Advanced Degree|5500|Low Risk|0|0|0| +2153|M|S|Advanced Degree|5500|Low Risk|0|0|0| +2154|F|S|Advanced Degree|5500|Low Risk|0|0|0| +2155|M|D|Advanced Degree|5500|Low Risk|0|0|0| +2156|F|D|Advanced Degree|5500|Low Risk|0|0|0| +2157|M|W|Advanced Degree|5500|Low Risk|0|0|0| +2158|F|W|Advanced Degree|5500|Low Risk|0|0|0| +2159|M|U|Advanced Degree|5500|Low Risk|0|0|0| +2160|F|U|Advanced Degree|5500|Low Risk|0|0|0| +2161|M|M|Unknown|5500|Low Risk|0|0|0| +2162|F|M|Unknown|5500|Low Risk|0|0|0| +2163|M|S|Unknown|5500|Low Risk|0|0|0| +2164|F|S|Unknown|5500|Low Risk|0|0|0| +2165|M|D|Unknown|5500|Low Risk|0|0|0| +2166|F|D|Unknown|5500|Low Risk|0|0|0| +2167|M|W|Unknown|5500|Low Risk|0|0|0| +2168|F|W|Unknown|5500|Low Risk|0|0|0| +2169|M|U|Unknown|5500|Low Risk|0|0|0| +2170|F|U|Unknown|5500|Low Risk|0|0|0| +2171|M|M|Primary|6000|Low Risk|0|0|0| +2172|F|M|Primary|6000|Low Risk|0|0|0| +2173|M|S|Primary|6000|Low Risk|0|0|0| +2174|F|S|Primary|6000|Low Risk|0|0|0| +2175|M|D|Primary|6000|Low Risk|0|0|0| +2176|F|D|Primary|6000|Low Risk|0|0|0| +2177|M|W|Primary|6000|Low Risk|0|0|0| +2178|F|W|Primary|6000|Low Risk|0|0|0| +2179|M|U|Primary|6000|Low Risk|0|0|0| +2180|F|U|Primary|6000|Low Risk|0|0|0| +2181|M|M|Secondary|6000|Low Risk|0|0|0| +2182|F|M|Secondary|6000|Low Risk|0|0|0| +2183|M|S|Secondary|6000|Low Risk|0|0|0| +2184|F|S|Secondary|6000|Low Risk|0|0|0| +2185|M|D|Secondary|6000|Low Risk|0|0|0| +2186|F|D|Secondary|6000|Low Risk|0|0|0| +2187|M|W|Secondary|6000|Low Risk|0|0|0| +2188|F|W|Secondary|6000|Low Risk|0|0|0| +2189|M|U|Secondary|6000|Low Risk|0|0|0| +2190|F|U|Secondary|6000|Low Risk|0|0|0| +2191|M|M|College|6000|Low Risk|0|0|0| +2192|F|M|College|6000|Low Risk|0|0|0| +2193|M|S|College|6000|Low Risk|0|0|0| +2194|F|S|College|6000|Low Risk|0|0|0| +2195|M|D|College|6000|Low Risk|0|0|0| +2196|F|D|College|6000|Low Risk|0|0|0| +2197|M|W|College|6000|Low Risk|0|0|0| +2198|F|W|College|6000|Low Risk|0|0|0| +2199|M|U|College|6000|Low Risk|0|0|0| +2200|F|U|College|6000|Low Risk|0|0|0| +2201|M|M|2 yr Degree|6000|Low Risk|0|0|0| +2202|F|M|2 yr Degree|6000|Low Risk|0|0|0| +2203|M|S|2 yr Degree|6000|Low Risk|0|0|0| +2204|F|S|2 yr Degree|6000|Low Risk|0|0|0| +2205|M|D|2 yr Degree|6000|Low Risk|0|0|0| +2206|F|D|2 yr Degree|6000|Low Risk|0|0|0| +2207|M|W|2 yr Degree|6000|Low Risk|0|0|0| +2208|F|W|2 yr Degree|6000|Low Risk|0|0|0| +2209|M|U|2 yr Degree|6000|Low Risk|0|0|0| +2210|F|U|2 yr Degree|6000|Low Risk|0|0|0| +2211|M|M|4 yr Degree|6000|Low Risk|0|0|0| +2212|F|M|4 yr Degree|6000|Low Risk|0|0|0| +2213|M|S|4 yr Degree|6000|Low Risk|0|0|0| +2214|F|S|4 yr Degree|6000|Low Risk|0|0|0| +2215|M|D|4 yr Degree|6000|Low Risk|0|0|0| +2216|F|D|4 yr Degree|6000|Low Risk|0|0|0| +2217|M|W|4 yr Degree|6000|Low Risk|0|0|0| +2218|F|W|4 yr Degree|6000|Low Risk|0|0|0| +2219|M|U|4 yr Degree|6000|Low Risk|0|0|0| +2220|F|U|4 yr Degree|6000|Low Risk|0|0|0| +2221|M|M|Advanced Degree|6000|Low Risk|0|0|0| +2222|F|M|Advanced Degree|6000|Low Risk|0|0|0| +2223|M|S|Advanced Degree|6000|Low Risk|0|0|0| +2224|F|S|Advanced Degree|6000|Low Risk|0|0|0| +2225|M|D|Advanced Degree|6000|Low Risk|0|0|0| +2226|F|D|Advanced Degree|6000|Low Risk|0|0|0| +2227|M|W|Advanced Degree|6000|Low Risk|0|0|0| +2228|F|W|Advanced Degree|6000|Low Risk|0|0|0| +2229|M|U|Advanced Degree|6000|Low Risk|0|0|0| +2230|F|U|Advanced Degree|6000|Low Risk|0|0|0| +2231|M|M|Unknown|6000|Low Risk|0|0|0| +2232|F|M|Unknown|6000|Low Risk|0|0|0| +2233|M|S|Unknown|6000|Low Risk|0|0|0| +2234|F|S|Unknown|6000|Low Risk|0|0|0| +2235|M|D|Unknown|6000|Low Risk|0|0|0| +2236|F|D|Unknown|6000|Low Risk|0|0|0| +2237|M|W|Unknown|6000|Low Risk|0|0|0| +2238|F|W|Unknown|6000|Low Risk|0|0|0| +2239|M|U|Unknown|6000|Low Risk|0|0|0| +2240|F|U|Unknown|6000|Low Risk|0|0|0| +2241|M|M|Primary|6500|Low Risk|0|0|0| +2242|F|M|Primary|6500|Low Risk|0|0|0| +2243|M|S|Primary|6500|Low Risk|0|0|0| +2244|F|S|Primary|6500|Low Risk|0|0|0| +2245|M|D|Primary|6500|Low Risk|0|0|0| +2246|F|D|Primary|6500|Low Risk|0|0|0| +2247|M|W|Primary|6500|Low Risk|0|0|0| +2248|F|W|Primary|6500|Low Risk|0|0|0| +2249|M|U|Primary|6500|Low Risk|0|0|0| +2250|F|U|Primary|6500|Low Risk|0|0|0| +2251|M|M|Secondary|6500|Low Risk|0|0|0| +2252|F|M|Secondary|6500|Low Risk|0|0|0| +2253|M|S|Secondary|6500|Low Risk|0|0|0| +2254|F|S|Secondary|6500|Low Risk|0|0|0| +2255|M|D|Secondary|6500|Low Risk|0|0|0| +2256|F|D|Secondary|6500|Low Risk|0|0|0| +2257|M|W|Secondary|6500|Low Risk|0|0|0| +2258|F|W|Secondary|6500|Low Risk|0|0|0| +2259|M|U|Secondary|6500|Low Risk|0|0|0| +2260|F|U|Secondary|6500|Low Risk|0|0|0| +2261|M|M|College|6500|Low Risk|0|0|0| +2262|F|M|College|6500|Low Risk|0|0|0| +2263|M|S|College|6500|Low Risk|0|0|0| +2264|F|S|College|6500|Low Risk|0|0|0| +2265|M|D|College|6500|Low Risk|0|0|0| +2266|F|D|College|6500|Low Risk|0|0|0| +2267|M|W|College|6500|Low Risk|0|0|0| +2268|F|W|College|6500|Low Risk|0|0|0| +2269|M|U|College|6500|Low Risk|0|0|0| +2270|F|U|College|6500|Low Risk|0|0|0| +2271|M|M|2 yr Degree|6500|Low Risk|0|0|0| +2272|F|M|2 yr Degree|6500|Low Risk|0|0|0| +2273|M|S|2 yr Degree|6500|Low Risk|0|0|0| +2274|F|S|2 yr Degree|6500|Low Risk|0|0|0| +2275|M|D|2 yr Degree|6500|Low Risk|0|0|0| +2276|F|D|2 yr Degree|6500|Low Risk|0|0|0| +2277|M|W|2 yr Degree|6500|Low Risk|0|0|0| +2278|F|W|2 yr Degree|6500|Low Risk|0|0|0| +2279|M|U|2 yr Degree|6500|Low Risk|0|0|0| +2280|F|U|2 yr Degree|6500|Low Risk|0|0|0| +2281|M|M|4 yr Degree|6500|Low Risk|0|0|0| +2282|F|M|4 yr Degree|6500|Low Risk|0|0|0| +2283|M|S|4 yr Degree|6500|Low Risk|0|0|0| +2284|F|S|4 yr Degree|6500|Low Risk|0|0|0| +2285|M|D|4 yr Degree|6500|Low Risk|0|0|0| +2286|F|D|4 yr Degree|6500|Low Risk|0|0|0| +2287|M|W|4 yr Degree|6500|Low Risk|0|0|0| +2288|F|W|4 yr Degree|6500|Low Risk|0|0|0| +2289|M|U|4 yr Degree|6500|Low Risk|0|0|0| +2290|F|U|4 yr Degree|6500|Low Risk|0|0|0| +2291|M|M|Advanced Degree|6500|Low Risk|0|0|0| +2292|F|M|Advanced Degree|6500|Low Risk|0|0|0| +2293|M|S|Advanced Degree|6500|Low Risk|0|0|0| +2294|F|S|Advanced Degree|6500|Low Risk|0|0|0| +2295|M|D|Advanced Degree|6500|Low Risk|0|0|0| +2296|F|D|Advanced Degree|6500|Low Risk|0|0|0| +2297|M|W|Advanced Degree|6500|Low Risk|0|0|0| +2298|F|W|Advanced Degree|6500|Low Risk|0|0|0| +2299|M|U|Advanced Degree|6500|Low Risk|0|0|0| +2300|F|U|Advanced Degree|6500|Low Risk|0|0|0| +2301|M|M|Unknown|6500|Low Risk|0|0|0| +2302|F|M|Unknown|6500|Low Risk|0|0|0| +2303|M|S|Unknown|6500|Low Risk|0|0|0| +2304|F|S|Unknown|6500|Low Risk|0|0|0| +2305|M|D|Unknown|6500|Low Risk|0|0|0| +2306|F|D|Unknown|6500|Low Risk|0|0|0| +2307|M|W|Unknown|6500|Low Risk|0|0|0| +2308|F|W|Unknown|6500|Low Risk|0|0|0| +2309|M|U|Unknown|6500|Low Risk|0|0|0| +2310|F|U|Unknown|6500|Low Risk|0|0|0| +2311|M|M|Primary|7000|Low Risk|0|0|0| +2312|F|M|Primary|7000|Low Risk|0|0|0| +2313|M|S|Primary|7000|Low Risk|0|0|0| +2314|F|S|Primary|7000|Low Risk|0|0|0| +2315|M|D|Primary|7000|Low Risk|0|0|0| +2316|F|D|Primary|7000|Low Risk|0|0|0| +2317|M|W|Primary|7000|Low Risk|0|0|0| +2318|F|W|Primary|7000|Low Risk|0|0|0| +2319|M|U|Primary|7000|Low Risk|0|0|0| +2320|F|U|Primary|7000|Low Risk|0|0|0| +2321|M|M|Secondary|7000|Low Risk|0|0|0| +2322|F|M|Secondary|7000|Low Risk|0|0|0| +2323|M|S|Secondary|7000|Low Risk|0|0|0| +2324|F|S|Secondary|7000|Low Risk|0|0|0| +2325|M|D|Secondary|7000|Low Risk|0|0|0| +2326|F|D|Secondary|7000|Low Risk|0|0|0| +2327|M|W|Secondary|7000|Low Risk|0|0|0| +2328|F|W|Secondary|7000|Low Risk|0|0|0| +2329|M|U|Secondary|7000|Low Risk|0|0|0| +2330|F|U|Secondary|7000|Low Risk|0|0|0| +2331|M|M|College|7000|Low Risk|0|0|0| +2332|F|M|College|7000|Low Risk|0|0|0| +2333|M|S|College|7000|Low Risk|0|0|0| +2334|F|S|College|7000|Low Risk|0|0|0| +2335|M|D|College|7000|Low Risk|0|0|0| +2336|F|D|College|7000|Low Risk|0|0|0| +2337|M|W|College|7000|Low Risk|0|0|0| +2338|F|W|College|7000|Low Risk|0|0|0| +2339|M|U|College|7000|Low Risk|0|0|0| +2340|F|U|College|7000|Low Risk|0|0|0| +2341|M|M|2 yr Degree|7000|Low Risk|0|0|0| +2342|F|M|2 yr Degree|7000|Low Risk|0|0|0| +2343|M|S|2 yr Degree|7000|Low Risk|0|0|0| +2344|F|S|2 yr Degree|7000|Low Risk|0|0|0| +2345|M|D|2 yr Degree|7000|Low Risk|0|0|0| +2346|F|D|2 yr Degree|7000|Low Risk|0|0|0| +2347|M|W|2 yr Degree|7000|Low Risk|0|0|0| +2348|F|W|2 yr Degree|7000|Low Risk|0|0|0| +2349|M|U|2 yr Degree|7000|Low Risk|0|0|0| +2350|F|U|2 yr Degree|7000|Low Risk|0|0|0| +2351|M|M|4 yr Degree|7000|Low Risk|0|0|0| +2352|F|M|4 yr Degree|7000|Low Risk|0|0|0| +2353|M|S|4 yr Degree|7000|Low Risk|0|0|0| +2354|F|S|4 yr Degree|7000|Low Risk|0|0|0| +2355|M|D|4 yr Degree|7000|Low Risk|0|0|0| +2356|F|D|4 yr Degree|7000|Low Risk|0|0|0| +2357|M|W|4 yr Degree|7000|Low Risk|0|0|0| +2358|F|W|4 yr Degree|7000|Low Risk|0|0|0| +2359|M|U|4 yr Degree|7000|Low Risk|0|0|0| +2360|F|U|4 yr Degree|7000|Low Risk|0|0|0| +2361|M|M|Advanced Degree|7000|Low Risk|0|0|0| +2362|F|M|Advanced Degree|7000|Low Risk|0|0|0| +2363|M|S|Advanced Degree|7000|Low Risk|0|0|0| +2364|F|S|Advanced Degree|7000|Low Risk|0|0|0| +2365|M|D|Advanced Degree|7000|Low Risk|0|0|0| +2366|F|D|Advanced Degree|7000|Low Risk|0|0|0| +2367|M|W|Advanced Degree|7000|Low Risk|0|0|0| +2368|F|W|Advanced Degree|7000|Low Risk|0|0|0| +2369|M|U|Advanced Degree|7000|Low Risk|0|0|0| +2370|F|U|Advanced Degree|7000|Low Risk|0|0|0| +2371|M|M|Unknown|7000|Low Risk|0|0|0| +2372|F|M|Unknown|7000|Low Risk|0|0|0| +2373|M|S|Unknown|7000|Low Risk|0|0|0| +2374|F|S|Unknown|7000|Low Risk|0|0|0| +2375|M|D|Unknown|7000|Low Risk|0|0|0| +2376|F|D|Unknown|7000|Low Risk|0|0|0| +2377|M|W|Unknown|7000|Low Risk|0|0|0| +2378|F|W|Unknown|7000|Low Risk|0|0|0| +2379|M|U|Unknown|7000|Low Risk|0|0|0| +2380|F|U|Unknown|7000|Low Risk|0|0|0| +2381|M|M|Primary|7500|Low Risk|0|0|0| +2382|F|M|Primary|7500|Low Risk|0|0|0| +2383|M|S|Primary|7500|Low Risk|0|0|0| +2384|F|S|Primary|7500|Low Risk|0|0|0| +2385|M|D|Primary|7500|Low Risk|0|0|0| +2386|F|D|Primary|7500|Low Risk|0|0|0| +2387|M|W|Primary|7500|Low Risk|0|0|0| +2388|F|W|Primary|7500|Low Risk|0|0|0| +2389|M|U|Primary|7500|Low Risk|0|0|0| +2390|F|U|Primary|7500|Low Risk|0|0|0| +2391|M|M|Secondary|7500|Low Risk|0|0|0| +2392|F|M|Secondary|7500|Low Risk|0|0|0| +2393|M|S|Secondary|7500|Low Risk|0|0|0| +2394|F|S|Secondary|7500|Low Risk|0|0|0| +2395|M|D|Secondary|7500|Low Risk|0|0|0| +2396|F|D|Secondary|7500|Low Risk|0|0|0| +2397|M|W|Secondary|7500|Low Risk|0|0|0| +2398|F|W|Secondary|7500|Low Risk|0|0|0| +2399|M|U|Secondary|7500|Low Risk|0|0|0| +2400|F|U|Secondary|7500|Low Risk|0|0|0| +2401|M|M|College|7500|Low Risk|0|0|0| +2402|F|M|College|7500|Low Risk|0|0|0| +2403|M|S|College|7500|Low Risk|0|0|0| +2404|F|S|College|7500|Low Risk|0|0|0| +2405|M|D|College|7500|Low Risk|0|0|0| +2406|F|D|College|7500|Low Risk|0|0|0| +2407|M|W|College|7500|Low Risk|0|0|0| +2408|F|W|College|7500|Low Risk|0|0|0| +2409|M|U|College|7500|Low Risk|0|0|0| +2410|F|U|College|7500|Low Risk|0|0|0| +2411|M|M|2 yr Degree|7500|Low Risk|0|0|0| +2412|F|M|2 yr Degree|7500|Low Risk|0|0|0| +2413|M|S|2 yr Degree|7500|Low Risk|0|0|0| +2414|F|S|2 yr Degree|7500|Low Risk|0|0|0| +2415|M|D|2 yr Degree|7500|Low Risk|0|0|0| +2416|F|D|2 yr Degree|7500|Low Risk|0|0|0| +2417|M|W|2 yr Degree|7500|Low Risk|0|0|0| +2418|F|W|2 yr Degree|7500|Low Risk|0|0|0| +2419|M|U|2 yr Degree|7500|Low Risk|0|0|0| +2420|F|U|2 yr Degree|7500|Low Risk|0|0|0| +2421|M|M|4 yr Degree|7500|Low Risk|0|0|0| +2422|F|M|4 yr Degree|7500|Low Risk|0|0|0| +2423|M|S|4 yr Degree|7500|Low Risk|0|0|0| +2424|F|S|4 yr Degree|7500|Low Risk|0|0|0| +2425|M|D|4 yr Degree|7500|Low Risk|0|0|0| +2426|F|D|4 yr Degree|7500|Low Risk|0|0|0| +2427|M|W|4 yr Degree|7500|Low Risk|0|0|0| +2428|F|W|4 yr Degree|7500|Low Risk|0|0|0| +2429|M|U|4 yr Degree|7500|Low Risk|0|0|0| +2430|F|U|4 yr Degree|7500|Low Risk|0|0|0| +2431|M|M|Advanced Degree|7500|Low Risk|0|0|0| +2432|F|M|Advanced Degree|7500|Low Risk|0|0|0| +2433|M|S|Advanced Degree|7500|Low Risk|0|0|0| +2434|F|S|Advanced Degree|7500|Low Risk|0|0|0| +2435|M|D|Advanced Degree|7500|Low Risk|0|0|0| +2436|F|D|Advanced Degree|7500|Low Risk|0|0|0| +2437|M|W|Advanced Degree|7500|Low Risk|0|0|0| +2438|F|W|Advanced Degree|7500|Low Risk|0|0|0| +2439|M|U|Advanced Degree|7500|Low Risk|0|0|0| +2440|F|U|Advanced Degree|7500|Low Risk|0|0|0| +2441|M|M|Unknown|7500|Low Risk|0|0|0| +2442|F|M|Unknown|7500|Low Risk|0|0|0| +2443|M|S|Unknown|7500|Low Risk|0|0|0| +2444|F|S|Unknown|7500|Low Risk|0|0|0| +2445|M|D|Unknown|7500|Low Risk|0|0|0| +2446|F|D|Unknown|7500|Low Risk|0|0|0| +2447|M|W|Unknown|7500|Low Risk|0|0|0| +2448|F|W|Unknown|7500|Low Risk|0|0|0| +2449|M|U|Unknown|7500|Low Risk|0|0|0| +2450|F|U|Unknown|7500|Low Risk|0|0|0| +2451|M|M|Primary|8000|Low Risk|0|0|0| +2452|F|M|Primary|8000|Low Risk|0|0|0| +2453|M|S|Primary|8000|Low Risk|0|0|0| +2454|F|S|Primary|8000|Low Risk|0|0|0| +2455|M|D|Primary|8000|Low Risk|0|0|0| +2456|F|D|Primary|8000|Low Risk|0|0|0| +2457|M|W|Primary|8000|Low Risk|0|0|0| +2458|F|W|Primary|8000|Low Risk|0|0|0| +2459|M|U|Primary|8000|Low Risk|0|0|0| +2460|F|U|Primary|8000|Low Risk|0|0|0| +2461|M|M|Secondary|8000|Low Risk|0|0|0| +2462|F|M|Secondary|8000|Low Risk|0|0|0| +2463|M|S|Secondary|8000|Low Risk|0|0|0| +2464|F|S|Secondary|8000|Low Risk|0|0|0| +2465|M|D|Secondary|8000|Low Risk|0|0|0| +2466|F|D|Secondary|8000|Low Risk|0|0|0| +2467|M|W|Secondary|8000|Low Risk|0|0|0| +2468|F|W|Secondary|8000|Low Risk|0|0|0| +2469|M|U|Secondary|8000|Low Risk|0|0|0| +2470|F|U|Secondary|8000|Low Risk|0|0|0| +2471|M|M|College|8000|Low Risk|0|0|0| +2472|F|M|College|8000|Low Risk|0|0|0| +2473|M|S|College|8000|Low Risk|0|0|0| +2474|F|S|College|8000|Low Risk|0|0|0| +2475|M|D|College|8000|Low Risk|0|0|0| +2476|F|D|College|8000|Low Risk|0|0|0| +2477|M|W|College|8000|Low Risk|0|0|0| +2478|F|W|College|8000|Low Risk|0|0|0| +2479|M|U|College|8000|Low Risk|0|0|0| +2480|F|U|College|8000|Low Risk|0|0|0| +2481|M|M|2 yr Degree|8000|Low Risk|0|0|0| +2482|F|M|2 yr Degree|8000|Low Risk|0|0|0| +2483|M|S|2 yr Degree|8000|Low Risk|0|0|0| +2484|F|S|2 yr Degree|8000|Low Risk|0|0|0| +2485|M|D|2 yr Degree|8000|Low Risk|0|0|0| +2486|F|D|2 yr Degree|8000|Low Risk|0|0|0| +2487|M|W|2 yr Degree|8000|Low Risk|0|0|0| +2488|F|W|2 yr Degree|8000|Low Risk|0|0|0| +2489|M|U|2 yr Degree|8000|Low Risk|0|0|0| +2490|F|U|2 yr Degree|8000|Low Risk|0|0|0| +2491|M|M|4 yr Degree|8000|Low Risk|0|0|0| +2492|F|M|4 yr Degree|8000|Low Risk|0|0|0| +2493|M|S|4 yr Degree|8000|Low Risk|0|0|0| +2494|F|S|4 yr Degree|8000|Low Risk|0|0|0| +2495|M|D|4 yr Degree|8000|Low Risk|0|0|0| +2496|F|D|4 yr Degree|8000|Low Risk|0|0|0| +2497|M|W|4 yr Degree|8000|Low Risk|0|0|0| +2498|F|W|4 yr Degree|8000|Low Risk|0|0|0| +2499|M|U|4 yr Degree|8000|Low Risk|0|0|0| +2500|F|U|4 yr Degree|8000|Low Risk|0|0|0| +2501|M|M|Advanced Degree|8000|Low Risk|0|0|0| +2502|F|M|Advanced Degree|8000|Low Risk|0|0|0| +2503|M|S|Advanced Degree|8000|Low Risk|0|0|0| +2504|F|S|Advanced Degree|8000|Low Risk|0|0|0| +2505|M|D|Advanced Degree|8000|Low Risk|0|0|0| +2506|F|D|Advanced Degree|8000|Low Risk|0|0|0| +2507|M|W|Advanced Degree|8000|Low Risk|0|0|0| +2508|F|W|Advanced Degree|8000|Low Risk|0|0|0| +2509|M|U|Advanced Degree|8000|Low Risk|0|0|0| +2510|F|U|Advanced Degree|8000|Low Risk|0|0|0| +2511|M|M|Unknown|8000|Low Risk|0|0|0| +2512|F|M|Unknown|8000|Low Risk|0|0|0| +2513|M|S|Unknown|8000|Low Risk|0|0|0| +2514|F|S|Unknown|8000|Low Risk|0|0|0| +2515|M|D|Unknown|8000|Low Risk|0|0|0| +2516|F|D|Unknown|8000|Low Risk|0|0|0| +2517|M|W|Unknown|8000|Low Risk|0|0|0| +2518|F|W|Unknown|8000|Low Risk|0|0|0| +2519|M|U|Unknown|8000|Low Risk|0|0|0| +2520|F|U|Unknown|8000|Low Risk|0|0|0| +2521|M|M|Primary|8500|Low Risk|0|0|0| +2522|F|M|Primary|8500|Low Risk|0|0|0| +2523|M|S|Primary|8500|Low Risk|0|0|0| +2524|F|S|Primary|8500|Low Risk|0|0|0| +2525|M|D|Primary|8500|Low Risk|0|0|0| +2526|F|D|Primary|8500|Low Risk|0|0|0| +2527|M|W|Primary|8500|Low Risk|0|0|0| +2528|F|W|Primary|8500|Low Risk|0|0|0| +2529|M|U|Primary|8500|Low Risk|0|0|0| +2530|F|U|Primary|8500|Low Risk|0|0|0| +2531|M|M|Secondary|8500|Low Risk|0|0|0| +2532|F|M|Secondary|8500|Low Risk|0|0|0| +2533|M|S|Secondary|8500|Low Risk|0|0|0| +2534|F|S|Secondary|8500|Low Risk|0|0|0| +2535|M|D|Secondary|8500|Low Risk|0|0|0| +2536|F|D|Secondary|8500|Low Risk|0|0|0| +2537|M|W|Secondary|8500|Low Risk|0|0|0| +2538|F|W|Secondary|8500|Low Risk|0|0|0| +2539|M|U|Secondary|8500|Low Risk|0|0|0| +2540|F|U|Secondary|8500|Low Risk|0|0|0| +2541|M|M|College|8500|Low Risk|0|0|0| +2542|F|M|College|8500|Low Risk|0|0|0| +2543|M|S|College|8500|Low Risk|0|0|0| +2544|F|S|College|8500|Low Risk|0|0|0| +2545|M|D|College|8500|Low Risk|0|0|0| +2546|F|D|College|8500|Low Risk|0|0|0| +2547|M|W|College|8500|Low Risk|0|0|0| +2548|F|W|College|8500|Low Risk|0|0|0| +2549|M|U|College|8500|Low Risk|0|0|0| +2550|F|U|College|8500|Low Risk|0|0|0| +2551|M|M|2 yr Degree|8500|Low Risk|0|0|0| +2552|F|M|2 yr Degree|8500|Low Risk|0|0|0| +2553|M|S|2 yr Degree|8500|Low Risk|0|0|0| +2554|F|S|2 yr Degree|8500|Low Risk|0|0|0| +2555|M|D|2 yr Degree|8500|Low Risk|0|0|0| +2556|F|D|2 yr Degree|8500|Low Risk|0|0|0| +2557|M|W|2 yr Degree|8500|Low Risk|0|0|0| +2558|F|W|2 yr Degree|8500|Low Risk|0|0|0| +2559|M|U|2 yr Degree|8500|Low Risk|0|0|0| +2560|F|U|2 yr Degree|8500|Low Risk|0|0|0| +2561|M|M|4 yr Degree|8500|Low Risk|0|0|0| +2562|F|M|4 yr Degree|8500|Low Risk|0|0|0| +2563|M|S|4 yr Degree|8500|Low Risk|0|0|0| +2564|F|S|4 yr Degree|8500|Low Risk|0|0|0| +2565|M|D|4 yr Degree|8500|Low Risk|0|0|0| +2566|F|D|4 yr Degree|8500|Low Risk|0|0|0| +2567|M|W|4 yr Degree|8500|Low Risk|0|0|0| +2568|F|W|4 yr Degree|8500|Low Risk|0|0|0| +2569|M|U|4 yr Degree|8500|Low Risk|0|0|0| +2570|F|U|4 yr Degree|8500|Low Risk|0|0|0| +2571|M|M|Advanced Degree|8500|Low Risk|0|0|0| +2572|F|M|Advanced Degree|8500|Low Risk|0|0|0| +2573|M|S|Advanced Degree|8500|Low Risk|0|0|0| +2574|F|S|Advanced Degree|8500|Low Risk|0|0|0| +2575|M|D|Advanced Degree|8500|Low Risk|0|0|0| +2576|F|D|Advanced Degree|8500|Low Risk|0|0|0| +2577|M|W|Advanced Degree|8500|Low Risk|0|0|0| +2578|F|W|Advanced Degree|8500|Low Risk|0|0|0| +2579|M|U|Advanced Degree|8500|Low Risk|0|0|0| +2580|F|U|Advanced Degree|8500|Low Risk|0|0|0| +2581|M|M|Unknown|8500|Low Risk|0|0|0| +2582|F|M|Unknown|8500|Low Risk|0|0|0| +2583|M|S|Unknown|8500|Low Risk|0|0|0| +2584|F|S|Unknown|8500|Low Risk|0|0|0| +2585|M|D|Unknown|8500|Low Risk|0|0|0| +2586|F|D|Unknown|8500|Low Risk|0|0|0| +2587|M|W|Unknown|8500|Low Risk|0|0|0| +2588|F|W|Unknown|8500|Low Risk|0|0|0| +2589|M|U|Unknown|8500|Low Risk|0|0|0| +2590|F|U|Unknown|8500|Low Risk|0|0|0| +2591|M|M|Primary|9000|Low Risk|0|0|0| +2592|F|M|Primary|9000|Low Risk|0|0|0| +2593|M|S|Primary|9000|Low Risk|0|0|0| +2594|F|S|Primary|9000|Low Risk|0|0|0| +2595|M|D|Primary|9000|Low Risk|0|0|0| +2596|F|D|Primary|9000|Low Risk|0|0|0| +2597|M|W|Primary|9000|Low Risk|0|0|0| +2598|F|W|Primary|9000|Low Risk|0|0|0| +2599|M|U|Primary|9000|Low Risk|0|0|0| +2600|F|U|Primary|9000|Low Risk|0|0|0| +2601|M|M|Secondary|9000|Low Risk|0|0|0| +2602|F|M|Secondary|9000|Low Risk|0|0|0| +2603|M|S|Secondary|9000|Low Risk|0|0|0| +2604|F|S|Secondary|9000|Low Risk|0|0|0| +2605|M|D|Secondary|9000|Low Risk|0|0|0| +2606|F|D|Secondary|9000|Low Risk|0|0|0| +2607|M|W|Secondary|9000|Low Risk|0|0|0| +2608|F|W|Secondary|9000|Low Risk|0|0|0| +2609|M|U|Secondary|9000|Low Risk|0|0|0| +2610|F|U|Secondary|9000|Low Risk|0|0|0| +2611|M|M|College|9000|Low Risk|0|0|0| +2612|F|M|College|9000|Low Risk|0|0|0| +2613|M|S|College|9000|Low Risk|0|0|0| +2614|F|S|College|9000|Low Risk|0|0|0| +2615|M|D|College|9000|Low Risk|0|0|0| +2616|F|D|College|9000|Low Risk|0|0|0| +2617|M|W|College|9000|Low Risk|0|0|0| +2618|F|W|College|9000|Low Risk|0|0|0| +2619|M|U|College|9000|Low Risk|0|0|0| +2620|F|U|College|9000|Low Risk|0|0|0| +2621|M|M|2 yr Degree|9000|Low Risk|0|0|0| +2622|F|M|2 yr Degree|9000|Low Risk|0|0|0| +2623|M|S|2 yr Degree|9000|Low Risk|0|0|0| +2624|F|S|2 yr Degree|9000|Low Risk|0|0|0| +2625|M|D|2 yr Degree|9000|Low Risk|0|0|0| +2626|F|D|2 yr Degree|9000|Low Risk|0|0|0| +2627|M|W|2 yr Degree|9000|Low Risk|0|0|0| +2628|F|W|2 yr Degree|9000|Low Risk|0|0|0| +2629|M|U|2 yr Degree|9000|Low Risk|0|0|0| +2630|F|U|2 yr Degree|9000|Low Risk|0|0|0| +2631|M|M|4 yr Degree|9000|Low Risk|0|0|0| +2632|F|M|4 yr Degree|9000|Low Risk|0|0|0| +2633|M|S|4 yr Degree|9000|Low Risk|0|0|0| +2634|F|S|4 yr Degree|9000|Low Risk|0|0|0| +2635|M|D|4 yr Degree|9000|Low Risk|0|0|0| +2636|F|D|4 yr Degree|9000|Low Risk|0|0|0| +2637|M|W|4 yr Degree|9000|Low Risk|0|0|0| +2638|F|W|4 yr Degree|9000|Low Risk|0|0|0| +2639|M|U|4 yr Degree|9000|Low Risk|0|0|0| +2640|F|U|4 yr Degree|9000|Low Risk|0|0|0| +2641|M|M|Advanced Degree|9000|Low Risk|0|0|0| +2642|F|M|Advanced Degree|9000|Low Risk|0|0|0| +2643|M|S|Advanced Degree|9000|Low Risk|0|0|0| +2644|F|S|Advanced Degree|9000|Low Risk|0|0|0| +2645|M|D|Advanced Degree|9000|Low Risk|0|0|0| +2646|F|D|Advanced Degree|9000|Low Risk|0|0|0| +2647|M|W|Advanced Degree|9000|Low Risk|0|0|0| +2648|F|W|Advanced Degree|9000|Low Risk|0|0|0| +2649|M|U|Advanced Degree|9000|Low Risk|0|0|0| +2650|F|U|Advanced Degree|9000|Low Risk|0|0|0| +2651|M|M|Unknown|9000|Low Risk|0|0|0| +2652|F|M|Unknown|9000|Low Risk|0|0|0| +2653|M|S|Unknown|9000|Low Risk|0|0|0| +2654|F|S|Unknown|9000|Low Risk|0|0|0| +2655|M|D|Unknown|9000|Low Risk|0|0|0| +2656|F|D|Unknown|9000|Low Risk|0|0|0| +2657|M|W|Unknown|9000|Low Risk|0|0|0| +2658|F|W|Unknown|9000|Low Risk|0|0|0| +2659|M|U|Unknown|9000|Low Risk|0|0|0| +2660|F|U|Unknown|9000|Low Risk|0|0|0| +2661|M|M|Primary|9500|Low Risk|0|0|0| +2662|F|M|Primary|9500|Low Risk|0|0|0| +2663|M|S|Primary|9500|Low Risk|0|0|0| +2664|F|S|Primary|9500|Low Risk|0|0|0| +2665|M|D|Primary|9500|Low Risk|0|0|0| +2666|F|D|Primary|9500|Low Risk|0|0|0| +2667|M|W|Primary|9500|Low Risk|0|0|0| +2668|F|W|Primary|9500|Low Risk|0|0|0| +2669|M|U|Primary|9500|Low Risk|0|0|0| +2670|F|U|Primary|9500|Low Risk|0|0|0| +2671|M|M|Secondary|9500|Low Risk|0|0|0| +2672|F|M|Secondary|9500|Low Risk|0|0|0| +2673|M|S|Secondary|9500|Low Risk|0|0|0| +2674|F|S|Secondary|9500|Low Risk|0|0|0| +2675|M|D|Secondary|9500|Low Risk|0|0|0| +2676|F|D|Secondary|9500|Low Risk|0|0|0| +2677|M|W|Secondary|9500|Low Risk|0|0|0| +2678|F|W|Secondary|9500|Low Risk|0|0|0| +2679|M|U|Secondary|9500|Low Risk|0|0|0| +2680|F|U|Secondary|9500|Low Risk|0|0|0| +2681|M|M|College|9500|Low Risk|0|0|0| +2682|F|M|College|9500|Low Risk|0|0|0| +2683|M|S|College|9500|Low Risk|0|0|0| +2684|F|S|College|9500|Low Risk|0|0|0| +2685|M|D|College|9500|Low Risk|0|0|0| +2686|F|D|College|9500|Low Risk|0|0|0| +2687|M|W|College|9500|Low Risk|0|0|0| +2688|F|W|College|9500|Low Risk|0|0|0| +2689|M|U|College|9500|Low Risk|0|0|0| +2690|F|U|College|9500|Low Risk|0|0|0| +2691|M|M|2 yr Degree|9500|Low Risk|0|0|0| +2692|F|M|2 yr Degree|9500|Low Risk|0|0|0| +2693|M|S|2 yr Degree|9500|Low Risk|0|0|0| +2694|F|S|2 yr Degree|9500|Low Risk|0|0|0| +2695|M|D|2 yr Degree|9500|Low Risk|0|0|0| +2696|F|D|2 yr Degree|9500|Low Risk|0|0|0| +2697|M|W|2 yr Degree|9500|Low Risk|0|0|0| +2698|F|W|2 yr Degree|9500|Low Risk|0|0|0| +2699|M|U|2 yr Degree|9500|Low Risk|0|0|0| +2700|F|U|2 yr Degree|9500|Low Risk|0|0|0| +2701|M|M|4 yr Degree|9500|Low Risk|0|0|0| +2702|F|M|4 yr Degree|9500|Low Risk|0|0|0| +2703|M|S|4 yr Degree|9500|Low Risk|0|0|0| +2704|F|S|4 yr Degree|9500|Low Risk|0|0|0| +2705|M|D|4 yr Degree|9500|Low Risk|0|0|0| +2706|F|D|4 yr Degree|9500|Low Risk|0|0|0| +2707|M|W|4 yr Degree|9500|Low Risk|0|0|0| +2708|F|W|4 yr Degree|9500|Low Risk|0|0|0| +2709|M|U|4 yr Degree|9500|Low Risk|0|0|0| +2710|F|U|4 yr Degree|9500|Low Risk|0|0|0| +2711|M|M|Advanced Degree|9500|Low Risk|0|0|0| +2712|F|M|Advanced Degree|9500|Low Risk|0|0|0| +2713|M|S|Advanced Degree|9500|Low Risk|0|0|0| +2714|F|S|Advanced Degree|9500|Low Risk|0|0|0| +2715|M|D|Advanced Degree|9500|Low Risk|0|0|0| +2716|F|D|Advanced Degree|9500|Low Risk|0|0|0| +2717|M|W|Advanced Degree|9500|Low Risk|0|0|0| +2718|F|W|Advanced Degree|9500|Low Risk|0|0|0| +2719|M|U|Advanced Degree|9500|Low Risk|0|0|0| +2720|F|U|Advanced Degree|9500|Low Risk|0|0|0| +2721|M|M|Unknown|9500|Low Risk|0|0|0| +2722|F|M|Unknown|9500|Low Risk|0|0|0| +2723|M|S|Unknown|9500|Low Risk|0|0|0| +2724|F|S|Unknown|9500|Low Risk|0|0|0| +2725|M|D|Unknown|9500|Low Risk|0|0|0| +2726|F|D|Unknown|9500|Low Risk|0|0|0| +2727|M|W|Unknown|9500|Low Risk|0|0|0| +2728|F|W|Unknown|9500|Low Risk|0|0|0| +2729|M|U|Unknown|9500|Low Risk|0|0|0| +2730|F|U|Unknown|9500|Low Risk|0|0|0| +2731|M|M|Primary|10000|Low Risk|0|0|0| +2732|F|M|Primary|10000|Low Risk|0|0|0| +2733|M|S|Primary|10000|Low Risk|0|0|0| +2734|F|S|Primary|10000|Low Risk|0|0|0| +2735|M|D|Primary|10000|Low Risk|0|0|0| +2736|F|D|Primary|10000|Low Risk|0|0|0| +2737|M|W|Primary|10000|Low Risk|0|0|0| +2738|F|W|Primary|10000|Low Risk|0|0|0| +2739|M|U|Primary|10000|Low Risk|0|0|0| +2740|F|U|Primary|10000|Low Risk|0|0|0| +2741|M|M|Secondary|10000|Low Risk|0|0|0| +2742|F|M|Secondary|10000|Low Risk|0|0|0| +2743|M|S|Secondary|10000|Low Risk|0|0|0| +2744|F|S|Secondary|10000|Low Risk|0|0|0| +2745|M|D|Secondary|10000|Low Risk|0|0|0| +2746|F|D|Secondary|10000|Low Risk|0|0|0| +2747|M|W|Secondary|10000|Low Risk|0|0|0| +2748|F|W|Secondary|10000|Low Risk|0|0|0| +2749|M|U|Secondary|10000|Low Risk|0|0|0| +2750|F|U|Secondary|10000|Low Risk|0|0|0| +2751|M|M|College|10000|Low Risk|0|0|0| +2752|F|M|College|10000|Low Risk|0|0|0| +2753|M|S|College|10000|Low Risk|0|0|0| +2754|F|S|College|10000|Low Risk|0|0|0| +2755|M|D|College|10000|Low Risk|0|0|0| +2756|F|D|College|10000|Low Risk|0|0|0| +2757|M|W|College|10000|Low Risk|0|0|0| +2758|F|W|College|10000|Low Risk|0|0|0| +2759|M|U|College|10000|Low Risk|0|0|0| +2760|F|U|College|10000|Low Risk|0|0|0| +2761|M|M|2 yr Degree|10000|Low Risk|0|0|0| +2762|F|M|2 yr Degree|10000|Low Risk|0|0|0| +2763|M|S|2 yr Degree|10000|Low Risk|0|0|0| +2764|F|S|2 yr Degree|10000|Low Risk|0|0|0| +2765|M|D|2 yr Degree|10000|Low Risk|0|0|0| +2766|F|D|2 yr Degree|10000|Low Risk|0|0|0| +2767|M|W|2 yr Degree|10000|Low Risk|0|0|0| +2768|F|W|2 yr Degree|10000|Low Risk|0|0|0| +2769|M|U|2 yr Degree|10000|Low Risk|0|0|0| +2770|F|U|2 yr Degree|10000|Low Risk|0|0|0| +2771|M|M|4 yr Degree|10000|Low Risk|0|0|0| +2772|F|M|4 yr Degree|10000|Low Risk|0|0|0| +2773|M|S|4 yr Degree|10000|Low Risk|0|0|0| +2774|F|S|4 yr Degree|10000|Low Risk|0|0|0| +2775|M|D|4 yr Degree|10000|Low Risk|0|0|0| +2776|F|D|4 yr Degree|10000|Low Risk|0|0|0| +2777|M|W|4 yr Degree|10000|Low Risk|0|0|0| +2778|F|W|4 yr Degree|10000|Low Risk|0|0|0| +2779|M|U|4 yr Degree|10000|Low Risk|0|0|0| +2780|F|U|4 yr Degree|10000|Low Risk|0|0|0| +2781|M|M|Advanced Degree|10000|Low Risk|0|0|0| +2782|F|M|Advanced Degree|10000|Low Risk|0|0|0| +2783|M|S|Advanced Degree|10000|Low Risk|0|0|0| +2784|F|S|Advanced Degree|10000|Low Risk|0|0|0| +2785|M|D|Advanced Degree|10000|Low Risk|0|0|0| +2786|F|D|Advanced Degree|10000|Low Risk|0|0|0| +2787|M|W|Advanced Degree|10000|Low Risk|0|0|0| +2788|F|W|Advanced Degree|10000|Low Risk|0|0|0| +2789|M|U|Advanced Degree|10000|Low Risk|0|0|0| +2790|F|U|Advanced Degree|10000|Low Risk|0|0|0| +2791|M|M|Unknown|10000|Low Risk|0|0|0| +2792|F|M|Unknown|10000|Low Risk|0|0|0| +2793|M|S|Unknown|10000|Low Risk|0|0|0| +2794|F|S|Unknown|10000|Low Risk|0|0|0| +2795|M|D|Unknown|10000|Low Risk|0|0|0| +2796|F|D|Unknown|10000|Low Risk|0|0|0| +2797|M|W|Unknown|10000|Low Risk|0|0|0| +2798|F|W|Unknown|10000|Low Risk|0|0|0| +2799|M|U|Unknown|10000|Low Risk|0|0|0| +2800|F|U|Unknown|10000|Low Risk|0|0|0| +2801|M|M|Primary|500|High Risk|0|0|0| +2802|F|M|Primary|500|High Risk|0|0|0| +2803|M|S|Primary|500|High Risk|0|0|0| +2804|F|S|Primary|500|High Risk|0|0|0| +2805|M|D|Primary|500|High Risk|0|0|0| +2806|F|D|Primary|500|High Risk|0|0|0| +2807|M|W|Primary|500|High Risk|0|0|0| +2808|F|W|Primary|500|High Risk|0|0|0| +2809|M|U|Primary|500|High Risk|0|0|0| +2810|F|U|Primary|500|High Risk|0|0|0| +2811|M|M|Secondary|500|High Risk|0|0|0| +2812|F|M|Secondary|500|High Risk|0|0|0| +2813|M|S|Secondary|500|High Risk|0|0|0| +2814|F|S|Secondary|500|High Risk|0|0|0| +2815|M|D|Secondary|500|High Risk|0|0|0| +2816|F|D|Secondary|500|High Risk|0|0|0| +2817|M|W|Secondary|500|High Risk|0|0|0| +2818|F|W|Secondary|500|High Risk|0|0|0| +2819|M|U|Secondary|500|High Risk|0|0|0| +2820|F|U|Secondary|500|High Risk|0|0|0| +2821|M|M|College|500|High Risk|0|0|0| +2822|F|M|College|500|High Risk|0|0|0| +2823|M|S|College|500|High Risk|0|0|0| +2824|F|S|College|500|High Risk|0|0|0| +2825|M|D|College|500|High Risk|0|0|0| +2826|F|D|College|500|High Risk|0|0|0| +2827|M|W|College|500|High Risk|0|0|0| +2828|F|W|College|500|High Risk|0|0|0| +2829|M|U|College|500|High Risk|0|0|0| +2830|F|U|College|500|High Risk|0|0|0| +2831|M|M|2 yr Degree|500|High Risk|0|0|0| +2832|F|M|2 yr Degree|500|High Risk|0|0|0| +2833|M|S|2 yr Degree|500|High Risk|0|0|0| +2834|F|S|2 yr Degree|500|High Risk|0|0|0| +2835|M|D|2 yr Degree|500|High Risk|0|0|0| +2836|F|D|2 yr Degree|500|High Risk|0|0|0| +2837|M|W|2 yr Degree|500|High Risk|0|0|0| +2838|F|W|2 yr Degree|500|High Risk|0|0|0| +2839|M|U|2 yr Degree|500|High Risk|0|0|0| +2840|F|U|2 yr Degree|500|High Risk|0|0|0| +2841|M|M|4 yr Degree|500|High Risk|0|0|0| +2842|F|M|4 yr Degree|500|High Risk|0|0|0| +2843|M|S|4 yr Degree|500|High Risk|0|0|0| +2844|F|S|4 yr Degree|500|High Risk|0|0|0| +2845|M|D|4 yr Degree|500|High Risk|0|0|0| +2846|F|D|4 yr Degree|500|High Risk|0|0|0| +2847|M|W|4 yr Degree|500|High Risk|0|0|0| +2848|F|W|4 yr Degree|500|High Risk|0|0|0| +2849|M|U|4 yr Degree|500|High Risk|0|0|0| +2850|F|U|4 yr Degree|500|High Risk|0|0|0| +2851|M|M|Advanced Degree|500|High Risk|0|0|0| +2852|F|M|Advanced Degree|500|High Risk|0|0|0| +2853|M|S|Advanced Degree|500|High Risk|0|0|0| +2854|F|S|Advanced Degree|500|High Risk|0|0|0| +2855|M|D|Advanced Degree|500|High Risk|0|0|0| +2856|F|D|Advanced Degree|500|High Risk|0|0|0| +2857|M|W|Advanced Degree|500|High Risk|0|0|0| +2858|F|W|Advanced Degree|500|High Risk|0|0|0| +2859|M|U|Advanced Degree|500|High Risk|0|0|0| +2860|F|U|Advanced Degree|500|High Risk|0|0|0| +2861|M|M|Unknown|500|High Risk|0|0|0| +2862|F|M|Unknown|500|High Risk|0|0|0| +2863|M|S|Unknown|500|High Risk|0|0|0| +2864|F|S|Unknown|500|High Risk|0|0|0| +2865|M|D|Unknown|500|High Risk|0|0|0| +2866|F|D|Unknown|500|High Risk|0|0|0| +2867|M|W|Unknown|500|High Risk|0|0|0| +2868|F|W|Unknown|500|High Risk|0|0|0| +2869|M|U|Unknown|500|High Risk|0|0|0| +2870|F|U|Unknown|500|High Risk|0|0|0| +2871|M|M|Primary|1000|High Risk|0|0|0| +2872|F|M|Primary|1000|High Risk|0|0|0| +2873|M|S|Primary|1000|High Risk|0|0|0| +2874|F|S|Primary|1000|High Risk|0|0|0| +2875|M|D|Primary|1000|High Risk|0|0|0| +2876|F|D|Primary|1000|High Risk|0|0|0| +2877|M|W|Primary|1000|High Risk|0|0|0| +2878|F|W|Primary|1000|High Risk|0|0|0| +2879|M|U|Primary|1000|High Risk|0|0|0| +2880|F|U|Primary|1000|High Risk|0|0|0| +2881|M|M|Secondary|1000|High Risk|0|0|0| +2882|F|M|Secondary|1000|High Risk|0|0|0| +2883|M|S|Secondary|1000|High Risk|0|0|0| +2884|F|S|Secondary|1000|High Risk|0|0|0| +2885|M|D|Secondary|1000|High Risk|0|0|0| +2886|F|D|Secondary|1000|High Risk|0|0|0| +2887|M|W|Secondary|1000|High Risk|0|0|0| +2888|F|W|Secondary|1000|High Risk|0|0|0| +2889|M|U|Secondary|1000|High Risk|0|0|0| +2890|F|U|Secondary|1000|High Risk|0|0|0| +2891|M|M|College|1000|High Risk|0|0|0| +2892|F|M|College|1000|High Risk|0|0|0| +2893|M|S|College|1000|High Risk|0|0|0| +2894|F|S|College|1000|High Risk|0|0|0| +2895|M|D|College|1000|High Risk|0|0|0| +2896|F|D|College|1000|High Risk|0|0|0| +2897|M|W|College|1000|High Risk|0|0|0| +2898|F|W|College|1000|High Risk|0|0|0| +2899|M|U|College|1000|High Risk|0|0|0| +2900|F|U|College|1000|High Risk|0|0|0| +2901|M|M|2 yr Degree|1000|High Risk|0|0|0| +2902|F|M|2 yr Degree|1000|High Risk|0|0|0| +2903|M|S|2 yr Degree|1000|High Risk|0|0|0| +2904|F|S|2 yr Degree|1000|High Risk|0|0|0| +2905|M|D|2 yr Degree|1000|High Risk|0|0|0| +2906|F|D|2 yr Degree|1000|High Risk|0|0|0| +2907|M|W|2 yr Degree|1000|High Risk|0|0|0| +2908|F|W|2 yr Degree|1000|High Risk|0|0|0| +2909|M|U|2 yr Degree|1000|High Risk|0|0|0| +2910|F|U|2 yr Degree|1000|High Risk|0|0|0| +2911|M|M|4 yr Degree|1000|High Risk|0|0|0| +2912|F|M|4 yr Degree|1000|High Risk|0|0|0| +2913|M|S|4 yr Degree|1000|High Risk|0|0|0| +2914|F|S|4 yr Degree|1000|High Risk|0|0|0| +2915|M|D|4 yr Degree|1000|High Risk|0|0|0| +2916|F|D|4 yr Degree|1000|High Risk|0|0|0| +2917|M|W|4 yr Degree|1000|High Risk|0|0|0| +2918|F|W|4 yr Degree|1000|High Risk|0|0|0| +2919|M|U|4 yr Degree|1000|High Risk|0|0|0| +2920|F|U|4 yr Degree|1000|High Risk|0|0|0| +2921|M|M|Advanced Degree|1000|High Risk|0|0|0| +2922|F|M|Advanced Degree|1000|High Risk|0|0|0| +2923|M|S|Advanced Degree|1000|High Risk|0|0|0| +2924|F|S|Advanced Degree|1000|High Risk|0|0|0| +2925|M|D|Advanced Degree|1000|High Risk|0|0|0| +2926|F|D|Advanced Degree|1000|High Risk|0|0|0| +2927|M|W|Advanced Degree|1000|High Risk|0|0|0| +2928|F|W|Advanced Degree|1000|High Risk|0|0|0| +2929|M|U|Advanced Degree|1000|High Risk|0|0|0| +2930|F|U|Advanced Degree|1000|High Risk|0|0|0| +2931|M|M|Unknown|1000|High Risk|0|0|0| +2932|F|M|Unknown|1000|High Risk|0|0|0| +2933|M|S|Unknown|1000|High Risk|0|0|0| +2934|F|S|Unknown|1000|High Risk|0|0|0| +2935|M|D|Unknown|1000|High Risk|0|0|0| +2936|F|D|Unknown|1000|High Risk|0|0|0| +2937|M|W|Unknown|1000|High Risk|0|0|0| +2938|F|W|Unknown|1000|High Risk|0|0|0| +2939|M|U|Unknown|1000|High Risk|0|0|0| +2940|F|U|Unknown|1000|High Risk|0|0|0| +2941|M|M|Primary|1500|High Risk|0|0|0| +2942|F|M|Primary|1500|High Risk|0|0|0| +2943|M|S|Primary|1500|High Risk|0|0|0| +2944|F|S|Primary|1500|High Risk|0|0|0| +2945|M|D|Primary|1500|High Risk|0|0|0| +2946|F|D|Primary|1500|High Risk|0|0|0| +2947|M|W|Primary|1500|High Risk|0|0|0| +2948|F|W|Primary|1500|High Risk|0|0|0| +2949|M|U|Primary|1500|High Risk|0|0|0| +2950|F|U|Primary|1500|High Risk|0|0|0| +2951|M|M|Secondary|1500|High Risk|0|0|0| +2952|F|M|Secondary|1500|High Risk|0|0|0| +2953|M|S|Secondary|1500|High Risk|0|0|0| +2954|F|S|Secondary|1500|High Risk|0|0|0| +2955|M|D|Secondary|1500|High Risk|0|0|0| +2956|F|D|Secondary|1500|High Risk|0|0|0| +2957|M|W|Secondary|1500|High Risk|0|0|0| +2958|F|W|Secondary|1500|High Risk|0|0|0| +2959|M|U|Secondary|1500|High Risk|0|0|0| +2960|F|U|Secondary|1500|High Risk|0|0|0| +2961|M|M|College|1500|High Risk|0|0|0| +2962|F|M|College|1500|High Risk|0|0|0| +2963|M|S|College|1500|High Risk|0|0|0| +2964|F|S|College|1500|High Risk|0|0|0| +2965|M|D|College|1500|High Risk|0|0|0| +2966|F|D|College|1500|High Risk|0|0|0| +2967|M|W|College|1500|High Risk|0|0|0| +2968|F|W|College|1500|High Risk|0|0|0| +2969|M|U|College|1500|High Risk|0|0|0| +2970|F|U|College|1500|High Risk|0|0|0| +2971|M|M|2 yr Degree|1500|High Risk|0|0|0| +2972|F|M|2 yr Degree|1500|High Risk|0|0|0| +2973|M|S|2 yr Degree|1500|High Risk|0|0|0| +2974|F|S|2 yr Degree|1500|High Risk|0|0|0| +2975|M|D|2 yr Degree|1500|High Risk|0|0|0| +2976|F|D|2 yr Degree|1500|High Risk|0|0|0| +2977|M|W|2 yr Degree|1500|High Risk|0|0|0| +2978|F|W|2 yr Degree|1500|High Risk|0|0|0| +2979|M|U|2 yr Degree|1500|High Risk|0|0|0| +2980|F|U|2 yr Degree|1500|High Risk|0|0|0| +2981|M|M|4 yr Degree|1500|High Risk|0|0|0| +2982|F|M|4 yr Degree|1500|High Risk|0|0|0| +2983|M|S|4 yr Degree|1500|High Risk|0|0|0| +2984|F|S|4 yr Degree|1500|High Risk|0|0|0| +2985|M|D|4 yr Degree|1500|High Risk|0|0|0| +2986|F|D|4 yr Degree|1500|High Risk|0|0|0| +2987|M|W|4 yr Degree|1500|High Risk|0|0|0| +2988|F|W|4 yr Degree|1500|High Risk|0|0|0| +2989|M|U|4 yr Degree|1500|High Risk|0|0|0| +2990|F|U|4 yr Degree|1500|High Risk|0|0|0| +2991|M|M|Advanced Degree|1500|High Risk|0|0|0| +2992|F|M|Advanced Degree|1500|High Risk|0|0|0| +2993|M|S|Advanced Degree|1500|High Risk|0|0|0| +2994|F|S|Advanced Degree|1500|High Risk|0|0|0| +2995|M|D|Advanced Degree|1500|High Risk|0|0|0| +2996|F|D|Advanced Degree|1500|High Risk|0|0|0| +2997|M|W|Advanced Degree|1500|High Risk|0|0|0| +2998|F|W|Advanced Degree|1500|High Risk|0|0|0| +2999|M|U|Advanced Degree|1500|High Risk|0|0|0| +3000|F|U|Advanced Degree|1500|High Risk|0|0|0| +3001|M|M|Unknown|1500|High Risk|0|0|0| +3002|F|M|Unknown|1500|High Risk|0|0|0| +3003|M|S|Unknown|1500|High Risk|0|0|0| +3004|F|S|Unknown|1500|High Risk|0|0|0| +3005|M|D|Unknown|1500|High Risk|0|0|0| +3006|F|D|Unknown|1500|High Risk|0|0|0| +3007|M|W|Unknown|1500|High Risk|0|0|0| +3008|F|W|Unknown|1500|High Risk|0|0|0| +3009|M|U|Unknown|1500|High Risk|0|0|0| +3010|F|U|Unknown|1500|High Risk|0|0|0| +3011|M|M|Primary|2000|High Risk|0|0|0| +3012|F|M|Primary|2000|High Risk|0|0|0| +3013|M|S|Primary|2000|High Risk|0|0|0| +3014|F|S|Primary|2000|High Risk|0|0|0| +3015|M|D|Primary|2000|High Risk|0|0|0| +3016|F|D|Primary|2000|High Risk|0|0|0| +3017|M|W|Primary|2000|High Risk|0|0|0| +3018|F|W|Primary|2000|High Risk|0|0|0| +3019|M|U|Primary|2000|High Risk|0|0|0| +3020|F|U|Primary|2000|High Risk|0|0|0| +3021|M|M|Secondary|2000|High Risk|0|0|0| +3022|F|M|Secondary|2000|High Risk|0|0|0| +3023|M|S|Secondary|2000|High Risk|0|0|0| +3024|F|S|Secondary|2000|High Risk|0|0|0| +3025|M|D|Secondary|2000|High Risk|0|0|0| +3026|F|D|Secondary|2000|High Risk|0|0|0| +3027|M|W|Secondary|2000|High Risk|0|0|0| +3028|F|W|Secondary|2000|High Risk|0|0|0| +3029|M|U|Secondary|2000|High Risk|0|0|0| +3030|F|U|Secondary|2000|High Risk|0|0|0| +3031|M|M|College|2000|High Risk|0|0|0| +3032|F|M|College|2000|High Risk|0|0|0| +3033|M|S|College|2000|High Risk|0|0|0| +3034|F|S|College|2000|High Risk|0|0|0| +3035|M|D|College|2000|High Risk|0|0|0| +3036|F|D|College|2000|High Risk|0|0|0| +3037|M|W|College|2000|High Risk|0|0|0| +3038|F|W|College|2000|High Risk|0|0|0| +3039|M|U|College|2000|High Risk|0|0|0| +3040|F|U|College|2000|High Risk|0|0|0| +3041|M|M|2 yr Degree|2000|High Risk|0|0|0| +3042|F|M|2 yr Degree|2000|High Risk|0|0|0| +3043|M|S|2 yr Degree|2000|High Risk|0|0|0| +3044|F|S|2 yr Degree|2000|High Risk|0|0|0| +3045|M|D|2 yr Degree|2000|High Risk|0|0|0| +3046|F|D|2 yr Degree|2000|High Risk|0|0|0| +3047|M|W|2 yr Degree|2000|High Risk|0|0|0| +3048|F|W|2 yr Degree|2000|High Risk|0|0|0| +3049|M|U|2 yr Degree|2000|High Risk|0|0|0| +3050|F|U|2 yr Degree|2000|High Risk|0|0|0| +3051|M|M|4 yr Degree|2000|High Risk|0|0|0| +3052|F|M|4 yr Degree|2000|High Risk|0|0|0| +3053|M|S|4 yr Degree|2000|High Risk|0|0|0| +3054|F|S|4 yr Degree|2000|High Risk|0|0|0| +3055|M|D|4 yr Degree|2000|High Risk|0|0|0| +3056|F|D|4 yr Degree|2000|High Risk|0|0|0| +3057|M|W|4 yr Degree|2000|High Risk|0|0|0| +3058|F|W|4 yr Degree|2000|High Risk|0|0|0| +3059|M|U|4 yr Degree|2000|High Risk|0|0|0| +3060|F|U|4 yr Degree|2000|High Risk|0|0|0| +3061|M|M|Advanced Degree|2000|High Risk|0|0|0| +3062|F|M|Advanced Degree|2000|High Risk|0|0|0| +3063|M|S|Advanced Degree|2000|High Risk|0|0|0| +3064|F|S|Advanced Degree|2000|High Risk|0|0|0| +3065|M|D|Advanced Degree|2000|High Risk|0|0|0| +3066|F|D|Advanced Degree|2000|High Risk|0|0|0| +3067|M|W|Advanced Degree|2000|High Risk|0|0|0| +3068|F|W|Advanced Degree|2000|High Risk|0|0|0| +3069|M|U|Advanced Degree|2000|High Risk|0|0|0| +3070|F|U|Advanced Degree|2000|High Risk|0|0|0| +3071|M|M|Unknown|2000|High Risk|0|0|0| +3072|F|M|Unknown|2000|High Risk|0|0|0| +3073|M|S|Unknown|2000|High Risk|0|0|0| +3074|F|S|Unknown|2000|High Risk|0|0|0| +3075|M|D|Unknown|2000|High Risk|0|0|0| +3076|F|D|Unknown|2000|High Risk|0|0|0| +3077|M|W|Unknown|2000|High Risk|0|0|0| +3078|F|W|Unknown|2000|High Risk|0|0|0| +3079|M|U|Unknown|2000|High Risk|0|0|0| +3080|F|U|Unknown|2000|High Risk|0|0|0| +3081|M|M|Primary|2500|High Risk|0|0|0| +3082|F|M|Primary|2500|High Risk|0|0|0| +3083|M|S|Primary|2500|High Risk|0|0|0| +3084|F|S|Primary|2500|High Risk|0|0|0| +3085|M|D|Primary|2500|High Risk|0|0|0| +3086|F|D|Primary|2500|High Risk|0|0|0| +3087|M|W|Primary|2500|High Risk|0|0|0| +3088|F|W|Primary|2500|High Risk|0|0|0| +3089|M|U|Primary|2500|High Risk|0|0|0| +3090|F|U|Primary|2500|High Risk|0|0|0| +3091|M|M|Secondary|2500|High Risk|0|0|0| +3092|F|M|Secondary|2500|High Risk|0|0|0| +3093|M|S|Secondary|2500|High Risk|0|0|0| +3094|F|S|Secondary|2500|High Risk|0|0|0| +3095|M|D|Secondary|2500|High Risk|0|0|0| +3096|F|D|Secondary|2500|High Risk|0|0|0| +3097|M|W|Secondary|2500|High Risk|0|0|0| +3098|F|W|Secondary|2500|High Risk|0|0|0| +3099|M|U|Secondary|2500|High Risk|0|0|0| +3100|F|U|Secondary|2500|High Risk|0|0|0| +3101|M|M|College|2500|High Risk|0|0|0| +3102|F|M|College|2500|High Risk|0|0|0| +3103|M|S|College|2500|High Risk|0|0|0| +3104|F|S|College|2500|High Risk|0|0|0| +3105|M|D|College|2500|High Risk|0|0|0| +3106|F|D|College|2500|High Risk|0|0|0| +3107|M|W|College|2500|High Risk|0|0|0| +3108|F|W|College|2500|High Risk|0|0|0| +3109|M|U|College|2500|High Risk|0|0|0| +3110|F|U|College|2500|High Risk|0|0|0| +3111|M|M|2 yr Degree|2500|High Risk|0|0|0| +3112|F|M|2 yr Degree|2500|High Risk|0|0|0| +3113|M|S|2 yr Degree|2500|High Risk|0|0|0| +3114|F|S|2 yr Degree|2500|High Risk|0|0|0| +3115|M|D|2 yr Degree|2500|High Risk|0|0|0| +3116|F|D|2 yr Degree|2500|High Risk|0|0|0| +3117|M|W|2 yr Degree|2500|High Risk|0|0|0| +3118|F|W|2 yr Degree|2500|High Risk|0|0|0| +3119|M|U|2 yr Degree|2500|High Risk|0|0|0| +3120|F|U|2 yr Degree|2500|High Risk|0|0|0| +3121|M|M|4 yr Degree|2500|High Risk|0|0|0| +3122|F|M|4 yr Degree|2500|High Risk|0|0|0| +3123|M|S|4 yr Degree|2500|High Risk|0|0|0| +3124|F|S|4 yr Degree|2500|High Risk|0|0|0| +3125|M|D|4 yr Degree|2500|High Risk|0|0|0| +3126|F|D|4 yr Degree|2500|High Risk|0|0|0| +3127|M|W|4 yr Degree|2500|High Risk|0|0|0| +3128|F|W|4 yr Degree|2500|High Risk|0|0|0| +3129|M|U|4 yr Degree|2500|High Risk|0|0|0| +3130|F|U|4 yr Degree|2500|High Risk|0|0|0| +3131|M|M|Advanced Degree|2500|High Risk|0|0|0| +3132|F|M|Advanced Degree|2500|High Risk|0|0|0| +3133|M|S|Advanced Degree|2500|High Risk|0|0|0| +3134|F|S|Advanced Degree|2500|High Risk|0|0|0| +3135|M|D|Advanced Degree|2500|High Risk|0|0|0| +3136|F|D|Advanced Degree|2500|High Risk|0|0|0| +3137|M|W|Advanced Degree|2500|High Risk|0|0|0| +3138|F|W|Advanced Degree|2500|High Risk|0|0|0| +3139|M|U|Advanced Degree|2500|High Risk|0|0|0| +3140|F|U|Advanced Degree|2500|High Risk|0|0|0| +3141|M|M|Unknown|2500|High Risk|0|0|0| +3142|F|M|Unknown|2500|High Risk|0|0|0| +3143|M|S|Unknown|2500|High Risk|0|0|0| +3144|F|S|Unknown|2500|High Risk|0|0|0| +3145|M|D|Unknown|2500|High Risk|0|0|0| +3146|F|D|Unknown|2500|High Risk|0|0|0| +3147|M|W|Unknown|2500|High Risk|0|0|0| +3148|F|W|Unknown|2500|High Risk|0|0|0| +3149|M|U|Unknown|2500|High Risk|0|0|0| +3150|F|U|Unknown|2500|High Risk|0|0|0| +3151|M|M|Primary|3000|High Risk|0|0|0| +3152|F|M|Primary|3000|High Risk|0|0|0| +3153|M|S|Primary|3000|High Risk|0|0|0| +3154|F|S|Primary|3000|High Risk|0|0|0| +3155|M|D|Primary|3000|High Risk|0|0|0| +3156|F|D|Primary|3000|High Risk|0|0|0| +3157|M|W|Primary|3000|High Risk|0|0|0| +3158|F|W|Primary|3000|High Risk|0|0|0| +3159|M|U|Primary|3000|High Risk|0|0|0| +3160|F|U|Primary|3000|High Risk|0|0|0| +3161|M|M|Secondary|3000|High Risk|0|0|0| +3162|F|M|Secondary|3000|High Risk|0|0|0| +3163|M|S|Secondary|3000|High Risk|0|0|0| +3164|F|S|Secondary|3000|High Risk|0|0|0| +3165|M|D|Secondary|3000|High Risk|0|0|0| +3166|F|D|Secondary|3000|High Risk|0|0|0| +3167|M|W|Secondary|3000|High Risk|0|0|0| +3168|F|W|Secondary|3000|High Risk|0|0|0| +3169|M|U|Secondary|3000|High Risk|0|0|0| +3170|F|U|Secondary|3000|High Risk|0|0|0| +3171|M|M|College|3000|High Risk|0|0|0| +3172|F|M|College|3000|High Risk|0|0|0| +3173|M|S|College|3000|High Risk|0|0|0| +3174|F|S|College|3000|High Risk|0|0|0| +3175|M|D|College|3000|High Risk|0|0|0| +3176|F|D|College|3000|High Risk|0|0|0| +3177|M|W|College|3000|High Risk|0|0|0| +3178|F|W|College|3000|High Risk|0|0|0| +3179|M|U|College|3000|High Risk|0|0|0| +3180|F|U|College|3000|High Risk|0|0|0| +3181|M|M|2 yr Degree|3000|High Risk|0|0|0| +3182|F|M|2 yr Degree|3000|High Risk|0|0|0| +3183|M|S|2 yr Degree|3000|High Risk|0|0|0| +3184|F|S|2 yr Degree|3000|High Risk|0|0|0| +3185|M|D|2 yr Degree|3000|High Risk|0|0|0| +3186|F|D|2 yr Degree|3000|High Risk|0|0|0| +3187|M|W|2 yr Degree|3000|High Risk|0|0|0| +3188|F|W|2 yr Degree|3000|High Risk|0|0|0| +3189|M|U|2 yr Degree|3000|High Risk|0|0|0| +3190|F|U|2 yr Degree|3000|High Risk|0|0|0| +3191|M|M|4 yr Degree|3000|High Risk|0|0|0| +3192|F|M|4 yr Degree|3000|High Risk|0|0|0| +3193|M|S|4 yr Degree|3000|High Risk|0|0|0| +3194|F|S|4 yr Degree|3000|High Risk|0|0|0| +3195|M|D|4 yr Degree|3000|High Risk|0|0|0| +3196|F|D|4 yr Degree|3000|High Risk|0|0|0| +3197|M|W|4 yr Degree|3000|High Risk|0|0|0| +3198|F|W|4 yr Degree|3000|High Risk|0|0|0| +3199|M|U|4 yr Degree|3000|High Risk|0|0|0| +3200|F|U|4 yr Degree|3000|High Risk|0|0|0| +3201|M|M|Advanced Degree|3000|High Risk|0|0|0| +3202|F|M|Advanced Degree|3000|High Risk|0|0|0| +3203|M|S|Advanced Degree|3000|High Risk|0|0|0| +3204|F|S|Advanced Degree|3000|High Risk|0|0|0| +3205|M|D|Advanced Degree|3000|High Risk|0|0|0| +3206|F|D|Advanced Degree|3000|High Risk|0|0|0| +3207|M|W|Advanced Degree|3000|High Risk|0|0|0| +3208|F|W|Advanced Degree|3000|High Risk|0|0|0| +3209|M|U|Advanced Degree|3000|High Risk|0|0|0| +3210|F|U|Advanced Degree|3000|High Risk|0|0|0| +3211|M|M|Unknown|3000|High Risk|0|0|0| +3212|F|M|Unknown|3000|High Risk|0|0|0| +3213|M|S|Unknown|3000|High Risk|0|0|0| +3214|F|S|Unknown|3000|High Risk|0|0|0| +3215|M|D|Unknown|3000|High Risk|0|0|0| +3216|F|D|Unknown|3000|High Risk|0|0|0| +3217|M|W|Unknown|3000|High Risk|0|0|0| +3218|F|W|Unknown|3000|High Risk|0|0|0| +3219|M|U|Unknown|3000|High Risk|0|0|0| +3220|F|U|Unknown|3000|High Risk|0|0|0| +3221|M|M|Primary|3500|High Risk|0|0|0| +3222|F|M|Primary|3500|High Risk|0|0|0| +3223|M|S|Primary|3500|High Risk|0|0|0| +3224|F|S|Primary|3500|High Risk|0|0|0| +3225|M|D|Primary|3500|High Risk|0|0|0| +3226|F|D|Primary|3500|High Risk|0|0|0| +3227|M|W|Primary|3500|High Risk|0|0|0| +3228|F|W|Primary|3500|High Risk|0|0|0| +3229|M|U|Primary|3500|High Risk|0|0|0| +3230|F|U|Primary|3500|High Risk|0|0|0| +3231|M|M|Secondary|3500|High Risk|0|0|0| +3232|F|M|Secondary|3500|High Risk|0|0|0| +3233|M|S|Secondary|3500|High Risk|0|0|0| +3234|F|S|Secondary|3500|High Risk|0|0|0| +3235|M|D|Secondary|3500|High Risk|0|0|0| +3236|F|D|Secondary|3500|High Risk|0|0|0| +3237|M|W|Secondary|3500|High Risk|0|0|0| +3238|F|W|Secondary|3500|High Risk|0|0|0| +3239|M|U|Secondary|3500|High Risk|0|0|0| +3240|F|U|Secondary|3500|High Risk|0|0|0| +3241|M|M|College|3500|High Risk|0|0|0| +3242|F|M|College|3500|High Risk|0|0|0| +3243|M|S|College|3500|High Risk|0|0|0| +3244|F|S|College|3500|High Risk|0|0|0| +3245|M|D|College|3500|High Risk|0|0|0| +3246|F|D|College|3500|High Risk|0|0|0| +3247|M|W|College|3500|High Risk|0|0|0| +3248|F|W|College|3500|High Risk|0|0|0| +3249|M|U|College|3500|High Risk|0|0|0| +3250|F|U|College|3500|High Risk|0|0|0| +3251|M|M|2 yr Degree|3500|High Risk|0|0|0| +3252|F|M|2 yr Degree|3500|High Risk|0|0|0| +3253|M|S|2 yr Degree|3500|High Risk|0|0|0| +3254|F|S|2 yr Degree|3500|High Risk|0|0|0| +3255|M|D|2 yr Degree|3500|High Risk|0|0|0| +3256|F|D|2 yr Degree|3500|High Risk|0|0|0| +3257|M|W|2 yr Degree|3500|High Risk|0|0|0| +3258|F|W|2 yr Degree|3500|High Risk|0|0|0| +3259|M|U|2 yr Degree|3500|High Risk|0|0|0| +3260|F|U|2 yr Degree|3500|High Risk|0|0|0| +3261|M|M|4 yr Degree|3500|High Risk|0|0|0| +3262|F|M|4 yr Degree|3500|High Risk|0|0|0| +3263|M|S|4 yr Degree|3500|High Risk|0|0|0| +3264|F|S|4 yr Degree|3500|High Risk|0|0|0| +3265|M|D|4 yr Degree|3500|High Risk|0|0|0| +3266|F|D|4 yr Degree|3500|High Risk|0|0|0| +3267|M|W|4 yr Degree|3500|High Risk|0|0|0| +3268|F|W|4 yr Degree|3500|High Risk|0|0|0| +3269|M|U|4 yr Degree|3500|High Risk|0|0|0| +3270|F|U|4 yr Degree|3500|High Risk|0|0|0| +3271|M|M|Advanced Degree|3500|High Risk|0|0|0| +3272|F|M|Advanced Degree|3500|High Risk|0|0|0| +3273|M|S|Advanced Degree|3500|High Risk|0|0|0| +3274|F|S|Advanced Degree|3500|High Risk|0|0|0| +3275|M|D|Advanced Degree|3500|High Risk|0|0|0| +3276|F|D|Advanced Degree|3500|High Risk|0|0|0| +3277|M|W|Advanced Degree|3500|High Risk|0|0|0| +3278|F|W|Advanced Degree|3500|High Risk|0|0|0| +3279|M|U|Advanced Degree|3500|High Risk|0|0|0| +3280|F|U|Advanced Degree|3500|High Risk|0|0|0| +3281|M|M|Unknown|3500|High Risk|0|0|0| +3282|F|M|Unknown|3500|High Risk|0|0|0| +3283|M|S|Unknown|3500|High Risk|0|0|0| +3284|F|S|Unknown|3500|High Risk|0|0|0| +3285|M|D|Unknown|3500|High Risk|0|0|0| +3286|F|D|Unknown|3500|High Risk|0|0|0| +3287|M|W|Unknown|3500|High Risk|0|0|0| +3288|F|W|Unknown|3500|High Risk|0|0|0| +3289|M|U|Unknown|3500|High Risk|0|0|0| +3290|F|U|Unknown|3500|High Risk|0|0|0| +3291|M|M|Primary|4000|High Risk|0|0|0| +3292|F|M|Primary|4000|High Risk|0|0|0| +3293|M|S|Primary|4000|High Risk|0|0|0| +3294|F|S|Primary|4000|High Risk|0|0|0| +3295|M|D|Primary|4000|High Risk|0|0|0| +3296|F|D|Primary|4000|High Risk|0|0|0| +3297|M|W|Primary|4000|High Risk|0|0|0| +3298|F|W|Primary|4000|High Risk|0|0|0| +3299|M|U|Primary|4000|High Risk|0|0|0| +3300|F|U|Primary|4000|High Risk|0|0|0| +3301|M|M|Secondary|4000|High Risk|0|0|0| +3302|F|M|Secondary|4000|High Risk|0|0|0| +3303|M|S|Secondary|4000|High Risk|0|0|0| +3304|F|S|Secondary|4000|High Risk|0|0|0| +3305|M|D|Secondary|4000|High Risk|0|0|0| +3306|F|D|Secondary|4000|High Risk|0|0|0| +3307|M|W|Secondary|4000|High Risk|0|0|0| +3308|F|W|Secondary|4000|High Risk|0|0|0| +3309|M|U|Secondary|4000|High Risk|0|0|0| +3310|F|U|Secondary|4000|High Risk|0|0|0| +3311|M|M|College|4000|High Risk|0|0|0| +3312|F|M|College|4000|High Risk|0|0|0| +3313|M|S|College|4000|High Risk|0|0|0| +3314|F|S|College|4000|High Risk|0|0|0| +3315|M|D|College|4000|High Risk|0|0|0| +3316|F|D|College|4000|High Risk|0|0|0| +3317|M|W|College|4000|High Risk|0|0|0| +3318|F|W|College|4000|High Risk|0|0|0| +3319|M|U|College|4000|High Risk|0|0|0| +3320|F|U|College|4000|High Risk|0|0|0| +3321|M|M|2 yr Degree|4000|High Risk|0|0|0| +3322|F|M|2 yr Degree|4000|High Risk|0|0|0| +3323|M|S|2 yr Degree|4000|High Risk|0|0|0| +3324|F|S|2 yr Degree|4000|High Risk|0|0|0| +3325|M|D|2 yr Degree|4000|High Risk|0|0|0| +3326|F|D|2 yr Degree|4000|High Risk|0|0|0| +3327|M|W|2 yr Degree|4000|High Risk|0|0|0| +3328|F|W|2 yr Degree|4000|High Risk|0|0|0| +3329|M|U|2 yr Degree|4000|High Risk|0|0|0| +3330|F|U|2 yr Degree|4000|High Risk|0|0|0| +3331|M|M|4 yr Degree|4000|High Risk|0|0|0| +3332|F|M|4 yr Degree|4000|High Risk|0|0|0| +3333|M|S|4 yr Degree|4000|High Risk|0|0|0| +3334|F|S|4 yr Degree|4000|High Risk|0|0|0| +3335|M|D|4 yr Degree|4000|High Risk|0|0|0| +3336|F|D|4 yr Degree|4000|High Risk|0|0|0| +3337|M|W|4 yr Degree|4000|High Risk|0|0|0| +3338|F|W|4 yr Degree|4000|High Risk|0|0|0| +3339|M|U|4 yr Degree|4000|High Risk|0|0|0| +3340|F|U|4 yr Degree|4000|High Risk|0|0|0| +3341|M|M|Advanced Degree|4000|High Risk|0|0|0| +3342|F|M|Advanced Degree|4000|High Risk|0|0|0| +3343|M|S|Advanced Degree|4000|High Risk|0|0|0| +3344|F|S|Advanced Degree|4000|High Risk|0|0|0| +3345|M|D|Advanced Degree|4000|High Risk|0|0|0| +3346|F|D|Advanced Degree|4000|High Risk|0|0|0| +3347|M|W|Advanced Degree|4000|High Risk|0|0|0| +3348|F|W|Advanced Degree|4000|High Risk|0|0|0| +3349|M|U|Advanced Degree|4000|High Risk|0|0|0| +3350|F|U|Advanced Degree|4000|High Risk|0|0|0| +3351|M|M|Unknown|4000|High Risk|0|0|0| +3352|F|M|Unknown|4000|High Risk|0|0|0| +3353|M|S|Unknown|4000|High Risk|0|0|0| +3354|F|S|Unknown|4000|High Risk|0|0|0| +3355|M|D|Unknown|4000|High Risk|0|0|0| +3356|F|D|Unknown|4000|High Risk|0|0|0| +3357|M|W|Unknown|4000|High Risk|0|0|0| +3358|F|W|Unknown|4000|High Risk|0|0|0| +3359|M|U|Unknown|4000|High Risk|0|0|0| +3360|F|U|Unknown|4000|High Risk|0|0|0| +3361|M|M|Primary|4500|High Risk|0|0|0| +3362|F|M|Primary|4500|High Risk|0|0|0| +3363|M|S|Primary|4500|High Risk|0|0|0| +3364|F|S|Primary|4500|High Risk|0|0|0| +3365|M|D|Primary|4500|High Risk|0|0|0| +3366|F|D|Primary|4500|High Risk|0|0|0| +3367|M|W|Primary|4500|High Risk|0|0|0| +3368|F|W|Primary|4500|High Risk|0|0|0| +3369|M|U|Primary|4500|High Risk|0|0|0| +3370|F|U|Primary|4500|High Risk|0|0|0| +3371|M|M|Secondary|4500|High Risk|0|0|0| +3372|F|M|Secondary|4500|High Risk|0|0|0| +3373|M|S|Secondary|4500|High Risk|0|0|0| +3374|F|S|Secondary|4500|High Risk|0|0|0| +3375|M|D|Secondary|4500|High Risk|0|0|0| +3376|F|D|Secondary|4500|High Risk|0|0|0| +3377|M|W|Secondary|4500|High Risk|0|0|0| +3378|F|W|Secondary|4500|High Risk|0|0|0| +3379|M|U|Secondary|4500|High Risk|0|0|0| +3380|F|U|Secondary|4500|High Risk|0|0|0| +3381|M|M|College|4500|High Risk|0|0|0| +3382|F|M|College|4500|High Risk|0|0|0| +3383|M|S|College|4500|High Risk|0|0|0| +3384|F|S|College|4500|High Risk|0|0|0| +3385|M|D|College|4500|High Risk|0|0|0| +3386|F|D|College|4500|High Risk|0|0|0| +3387|M|W|College|4500|High Risk|0|0|0| +3388|F|W|College|4500|High Risk|0|0|0| +3389|M|U|College|4500|High Risk|0|0|0| +3390|F|U|College|4500|High Risk|0|0|0| +3391|M|M|2 yr Degree|4500|High Risk|0|0|0| +3392|F|M|2 yr Degree|4500|High Risk|0|0|0| +3393|M|S|2 yr Degree|4500|High Risk|0|0|0| +3394|F|S|2 yr Degree|4500|High Risk|0|0|0| +3395|M|D|2 yr Degree|4500|High Risk|0|0|0| +3396|F|D|2 yr Degree|4500|High Risk|0|0|0| +3397|M|W|2 yr Degree|4500|High Risk|0|0|0| +3398|F|W|2 yr Degree|4500|High Risk|0|0|0| +3399|M|U|2 yr Degree|4500|High Risk|0|0|0| +3400|F|U|2 yr Degree|4500|High Risk|0|0|0| +3401|M|M|4 yr Degree|4500|High Risk|0|0|0| +3402|F|M|4 yr Degree|4500|High Risk|0|0|0| +3403|M|S|4 yr Degree|4500|High Risk|0|0|0| +3404|F|S|4 yr Degree|4500|High Risk|0|0|0| +3405|M|D|4 yr Degree|4500|High Risk|0|0|0| +3406|F|D|4 yr Degree|4500|High Risk|0|0|0| +3407|M|W|4 yr Degree|4500|High Risk|0|0|0| +3408|F|W|4 yr Degree|4500|High Risk|0|0|0| +3409|M|U|4 yr Degree|4500|High Risk|0|0|0| +3410|F|U|4 yr Degree|4500|High Risk|0|0|0| +3411|M|M|Advanced Degree|4500|High Risk|0|0|0| +3412|F|M|Advanced Degree|4500|High Risk|0|0|0| +3413|M|S|Advanced Degree|4500|High Risk|0|0|0| +3414|F|S|Advanced Degree|4500|High Risk|0|0|0| +3415|M|D|Advanced Degree|4500|High Risk|0|0|0| +3416|F|D|Advanced Degree|4500|High Risk|0|0|0| +3417|M|W|Advanced Degree|4500|High Risk|0|0|0| +3418|F|W|Advanced Degree|4500|High Risk|0|0|0| +3419|M|U|Advanced Degree|4500|High Risk|0|0|0| +3420|F|U|Advanced Degree|4500|High Risk|0|0|0| +3421|M|M|Unknown|4500|High Risk|0|0|0| +3422|F|M|Unknown|4500|High Risk|0|0|0| +3423|M|S|Unknown|4500|High Risk|0|0|0| +3424|F|S|Unknown|4500|High Risk|0|0|0| +3425|M|D|Unknown|4500|High Risk|0|0|0| +3426|F|D|Unknown|4500|High Risk|0|0|0| +3427|M|W|Unknown|4500|High Risk|0|0|0| +3428|F|W|Unknown|4500|High Risk|0|0|0| +3429|M|U|Unknown|4500|High Risk|0|0|0| +3430|F|U|Unknown|4500|High Risk|0|0|0| +3431|M|M|Primary|5000|High Risk|0|0|0| +3432|F|M|Primary|5000|High Risk|0|0|0| +3433|M|S|Primary|5000|High Risk|0|0|0| +3434|F|S|Primary|5000|High Risk|0|0|0| +3435|M|D|Primary|5000|High Risk|0|0|0| +3436|F|D|Primary|5000|High Risk|0|0|0| +3437|M|W|Primary|5000|High Risk|0|0|0| +3438|F|W|Primary|5000|High Risk|0|0|0| +3439|M|U|Primary|5000|High Risk|0|0|0| +3440|F|U|Primary|5000|High Risk|0|0|0| +3441|M|M|Secondary|5000|High Risk|0|0|0| +3442|F|M|Secondary|5000|High Risk|0|0|0| +3443|M|S|Secondary|5000|High Risk|0|0|0| +3444|F|S|Secondary|5000|High Risk|0|0|0| +3445|M|D|Secondary|5000|High Risk|0|0|0| +3446|F|D|Secondary|5000|High Risk|0|0|0| +3447|M|W|Secondary|5000|High Risk|0|0|0| +3448|F|W|Secondary|5000|High Risk|0|0|0| +3449|M|U|Secondary|5000|High Risk|0|0|0| +3450|F|U|Secondary|5000|High Risk|0|0|0| +3451|M|M|College|5000|High Risk|0|0|0| +3452|F|M|College|5000|High Risk|0|0|0| +3453|M|S|College|5000|High Risk|0|0|0| +3454|F|S|College|5000|High Risk|0|0|0| +3455|M|D|College|5000|High Risk|0|0|0| +3456|F|D|College|5000|High Risk|0|0|0| +3457|M|W|College|5000|High Risk|0|0|0| +3458|F|W|College|5000|High Risk|0|0|0| +3459|M|U|College|5000|High Risk|0|0|0| +3460|F|U|College|5000|High Risk|0|0|0| +3461|M|M|2 yr Degree|5000|High Risk|0|0|0| +3462|F|M|2 yr Degree|5000|High Risk|0|0|0| +3463|M|S|2 yr Degree|5000|High Risk|0|0|0| +3464|F|S|2 yr Degree|5000|High Risk|0|0|0| +3465|M|D|2 yr Degree|5000|High Risk|0|0|0| +3466|F|D|2 yr Degree|5000|High Risk|0|0|0| +3467|M|W|2 yr Degree|5000|High Risk|0|0|0| +3468|F|W|2 yr Degree|5000|High Risk|0|0|0| +3469|M|U|2 yr Degree|5000|High Risk|0|0|0| +3470|F|U|2 yr Degree|5000|High Risk|0|0|0| +3471|M|M|4 yr Degree|5000|High Risk|0|0|0| +3472|F|M|4 yr Degree|5000|High Risk|0|0|0| +3473|M|S|4 yr Degree|5000|High Risk|0|0|0| +3474|F|S|4 yr Degree|5000|High Risk|0|0|0| +3475|M|D|4 yr Degree|5000|High Risk|0|0|0| +3476|F|D|4 yr Degree|5000|High Risk|0|0|0| +3477|M|W|4 yr Degree|5000|High Risk|0|0|0| +3478|F|W|4 yr Degree|5000|High Risk|0|0|0| +3479|M|U|4 yr Degree|5000|High Risk|0|0|0| +3480|F|U|4 yr Degree|5000|High Risk|0|0|0| +3481|M|M|Advanced Degree|5000|High Risk|0|0|0| +3482|F|M|Advanced Degree|5000|High Risk|0|0|0| +3483|M|S|Advanced Degree|5000|High Risk|0|0|0| +3484|F|S|Advanced Degree|5000|High Risk|0|0|0| +3485|M|D|Advanced Degree|5000|High Risk|0|0|0| +3486|F|D|Advanced Degree|5000|High Risk|0|0|0| +3487|M|W|Advanced Degree|5000|High Risk|0|0|0| +3488|F|W|Advanced Degree|5000|High Risk|0|0|0| +3489|M|U|Advanced Degree|5000|High Risk|0|0|0| +3490|F|U|Advanced Degree|5000|High Risk|0|0|0| +3491|M|M|Unknown|5000|High Risk|0|0|0| +3492|F|M|Unknown|5000|High Risk|0|0|0| +3493|M|S|Unknown|5000|High Risk|0|0|0| +3494|F|S|Unknown|5000|High Risk|0|0|0| +3495|M|D|Unknown|5000|High Risk|0|0|0| +3496|F|D|Unknown|5000|High Risk|0|0|0| +3497|M|W|Unknown|5000|High Risk|0|0|0| +3498|F|W|Unknown|5000|High Risk|0|0|0| +3499|M|U|Unknown|5000|High Risk|0|0|0| +3500|F|U|Unknown|5000|High Risk|0|0|0| +3501|M|M|Primary|5500|High Risk|0|0|0| +3502|F|M|Primary|5500|High Risk|0|0|0| +3503|M|S|Primary|5500|High Risk|0|0|0| +3504|F|S|Primary|5500|High Risk|0|0|0| +3505|M|D|Primary|5500|High Risk|0|0|0| +3506|F|D|Primary|5500|High Risk|0|0|0| +3507|M|W|Primary|5500|High Risk|0|0|0| +3508|F|W|Primary|5500|High Risk|0|0|0| +3509|M|U|Primary|5500|High Risk|0|0|0| +3510|F|U|Primary|5500|High Risk|0|0|0| +3511|M|M|Secondary|5500|High Risk|0|0|0| +3512|F|M|Secondary|5500|High Risk|0|0|0| +3513|M|S|Secondary|5500|High Risk|0|0|0| +3514|F|S|Secondary|5500|High Risk|0|0|0| +3515|M|D|Secondary|5500|High Risk|0|0|0| +3516|F|D|Secondary|5500|High Risk|0|0|0| +3517|M|W|Secondary|5500|High Risk|0|0|0| +3518|F|W|Secondary|5500|High Risk|0|0|0| +3519|M|U|Secondary|5500|High Risk|0|0|0| +3520|F|U|Secondary|5500|High Risk|0|0|0| +3521|M|M|College|5500|High Risk|0|0|0| +3522|F|M|College|5500|High Risk|0|0|0| +3523|M|S|College|5500|High Risk|0|0|0| +3524|F|S|College|5500|High Risk|0|0|0| +3525|M|D|College|5500|High Risk|0|0|0| +3526|F|D|College|5500|High Risk|0|0|0| +3527|M|W|College|5500|High Risk|0|0|0| +3528|F|W|College|5500|High Risk|0|0|0| +3529|M|U|College|5500|High Risk|0|0|0| +3530|F|U|College|5500|High Risk|0|0|0| +3531|M|M|2 yr Degree|5500|High Risk|0|0|0| +3532|F|M|2 yr Degree|5500|High Risk|0|0|0| +3533|M|S|2 yr Degree|5500|High Risk|0|0|0| +3534|F|S|2 yr Degree|5500|High Risk|0|0|0| +3535|M|D|2 yr Degree|5500|High Risk|0|0|0| +3536|F|D|2 yr Degree|5500|High Risk|0|0|0| +3537|M|W|2 yr Degree|5500|High Risk|0|0|0| +3538|F|W|2 yr Degree|5500|High Risk|0|0|0| +3539|M|U|2 yr Degree|5500|High Risk|0|0|0| +3540|F|U|2 yr Degree|5500|High Risk|0|0|0| +3541|M|M|4 yr Degree|5500|High Risk|0|0|0| +3542|F|M|4 yr Degree|5500|High Risk|0|0|0| +3543|M|S|4 yr Degree|5500|High Risk|0|0|0| +3544|F|S|4 yr Degree|5500|High Risk|0|0|0| +3545|M|D|4 yr Degree|5500|High Risk|0|0|0| +3546|F|D|4 yr Degree|5500|High Risk|0|0|0| +3547|M|W|4 yr Degree|5500|High Risk|0|0|0| +3548|F|W|4 yr Degree|5500|High Risk|0|0|0| +3549|M|U|4 yr Degree|5500|High Risk|0|0|0| +3550|F|U|4 yr Degree|5500|High Risk|0|0|0| +3551|M|M|Advanced Degree|5500|High Risk|0|0|0| +3552|F|M|Advanced Degree|5500|High Risk|0|0|0| +3553|M|S|Advanced Degree|5500|High Risk|0|0|0| +3554|F|S|Advanced Degree|5500|High Risk|0|0|0| +3555|M|D|Advanced Degree|5500|High Risk|0|0|0| +3556|F|D|Advanced Degree|5500|High Risk|0|0|0| +3557|M|W|Advanced Degree|5500|High Risk|0|0|0| +3558|F|W|Advanced Degree|5500|High Risk|0|0|0| +3559|M|U|Advanced Degree|5500|High Risk|0|0|0| +3560|F|U|Advanced Degree|5500|High Risk|0|0|0| +3561|M|M|Unknown|5500|High Risk|0|0|0| +3562|F|M|Unknown|5500|High Risk|0|0|0| +3563|M|S|Unknown|5500|High Risk|0|0|0| +3564|F|S|Unknown|5500|High Risk|0|0|0| +3565|M|D|Unknown|5500|High Risk|0|0|0| +3566|F|D|Unknown|5500|High Risk|0|0|0| +3567|M|W|Unknown|5500|High Risk|0|0|0| +3568|F|W|Unknown|5500|High Risk|0|0|0| +3569|M|U|Unknown|5500|High Risk|0|0|0| +3570|F|U|Unknown|5500|High Risk|0|0|0| +3571|M|M|Primary|6000|High Risk|0|0|0| +3572|F|M|Primary|6000|High Risk|0|0|0| +3573|M|S|Primary|6000|High Risk|0|0|0| +3574|F|S|Primary|6000|High Risk|0|0|0| +3575|M|D|Primary|6000|High Risk|0|0|0| +3576|F|D|Primary|6000|High Risk|0|0|0| +3577|M|W|Primary|6000|High Risk|0|0|0| +3578|F|W|Primary|6000|High Risk|0|0|0| +3579|M|U|Primary|6000|High Risk|0|0|0| +3580|F|U|Primary|6000|High Risk|0|0|0| +3581|M|M|Secondary|6000|High Risk|0|0|0| +3582|F|M|Secondary|6000|High Risk|0|0|0| +3583|M|S|Secondary|6000|High Risk|0|0|0| +3584|F|S|Secondary|6000|High Risk|0|0|0| +3585|M|D|Secondary|6000|High Risk|0|0|0| +3586|F|D|Secondary|6000|High Risk|0|0|0| +3587|M|W|Secondary|6000|High Risk|0|0|0| +3588|F|W|Secondary|6000|High Risk|0|0|0| +3589|M|U|Secondary|6000|High Risk|0|0|0| +3590|F|U|Secondary|6000|High Risk|0|0|0| +3591|M|M|College|6000|High Risk|0|0|0| +3592|F|M|College|6000|High Risk|0|0|0| +3593|M|S|College|6000|High Risk|0|0|0| +3594|F|S|College|6000|High Risk|0|0|0| +3595|M|D|College|6000|High Risk|0|0|0| +3596|F|D|College|6000|High Risk|0|0|0| +3597|M|W|College|6000|High Risk|0|0|0| +3598|F|W|College|6000|High Risk|0|0|0| +3599|M|U|College|6000|High Risk|0|0|0| +3600|F|U|College|6000|High Risk|0|0|0| +3601|M|M|2 yr Degree|6000|High Risk|0|0|0| +3602|F|M|2 yr Degree|6000|High Risk|0|0|0| +3603|M|S|2 yr Degree|6000|High Risk|0|0|0| +3604|F|S|2 yr Degree|6000|High Risk|0|0|0| +3605|M|D|2 yr Degree|6000|High Risk|0|0|0| +3606|F|D|2 yr Degree|6000|High Risk|0|0|0| +3607|M|W|2 yr Degree|6000|High Risk|0|0|0| +3608|F|W|2 yr Degree|6000|High Risk|0|0|0| +3609|M|U|2 yr Degree|6000|High Risk|0|0|0| +3610|F|U|2 yr Degree|6000|High Risk|0|0|0| +3611|M|M|4 yr Degree|6000|High Risk|0|0|0| +3612|F|M|4 yr Degree|6000|High Risk|0|0|0| +3613|M|S|4 yr Degree|6000|High Risk|0|0|0| +3614|F|S|4 yr Degree|6000|High Risk|0|0|0| +3615|M|D|4 yr Degree|6000|High Risk|0|0|0| +3616|F|D|4 yr Degree|6000|High Risk|0|0|0| +3617|M|W|4 yr Degree|6000|High Risk|0|0|0| +3618|F|W|4 yr Degree|6000|High Risk|0|0|0| +3619|M|U|4 yr Degree|6000|High Risk|0|0|0| +3620|F|U|4 yr Degree|6000|High Risk|0|0|0| +3621|M|M|Advanced Degree|6000|High Risk|0|0|0| +3622|F|M|Advanced Degree|6000|High Risk|0|0|0| +3623|M|S|Advanced Degree|6000|High Risk|0|0|0| +3624|F|S|Advanced Degree|6000|High Risk|0|0|0| +3625|M|D|Advanced Degree|6000|High Risk|0|0|0| +3626|F|D|Advanced Degree|6000|High Risk|0|0|0| +3627|M|W|Advanced Degree|6000|High Risk|0|0|0| +3628|F|W|Advanced Degree|6000|High Risk|0|0|0| +3629|M|U|Advanced Degree|6000|High Risk|0|0|0| +3630|F|U|Advanced Degree|6000|High Risk|0|0|0| +3631|M|M|Unknown|6000|High Risk|0|0|0| +3632|F|M|Unknown|6000|High Risk|0|0|0| +3633|M|S|Unknown|6000|High Risk|0|0|0| +3634|F|S|Unknown|6000|High Risk|0|0|0| +3635|M|D|Unknown|6000|High Risk|0|0|0| +3636|F|D|Unknown|6000|High Risk|0|0|0| +3637|M|W|Unknown|6000|High Risk|0|0|0| +3638|F|W|Unknown|6000|High Risk|0|0|0| +3639|M|U|Unknown|6000|High Risk|0|0|0| +3640|F|U|Unknown|6000|High Risk|0|0|0| +3641|M|M|Primary|6500|High Risk|0|0|0| +3642|F|M|Primary|6500|High Risk|0|0|0| +3643|M|S|Primary|6500|High Risk|0|0|0| +3644|F|S|Primary|6500|High Risk|0|0|0| +3645|M|D|Primary|6500|High Risk|0|0|0| +3646|F|D|Primary|6500|High Risk|0|0|0| +3647|M|W|Primary|6500|High Risk|0|0|0| +3648|F|W|Primary|6500|High Risk|0|0|0| +3649|M|U|Primary|6500|High Risk|0|0|0| +3650|F|U|Primary|6500|High Risk|0|0|0| +3651|M|M|Secondary|6500|High Risk|0|0|0| +3652|F|M|Secondary|6500|High Risk|0|0|0| +3653|M|S|Secondary|6500|High Risk|0|0|0| +3654|F|S|Secondary|6500|High Risk|0|0|0| +3655|M|D|Secondary|6500|High Risk|0|0|0| +3656|F|D|Secondary|6500|High Risk|0|0|0| +3657|M|W|Secondary|6500|High Risk|0|0|0| +3658|F|W|Secondary|6500|High Risk|0|0|0| +3659|M|U|Secondary|6500|High Risk|0|0|0| +3660|F|U|Secondary|6500|High Risk|0|0|0| +3661|M|M|College|6500|High Risk|0|0|0| +3662|F|M|College|6500|High Risk|0|0|0| +3663|M|S|College|6500|High Risk|0|0|0| +3664|F|S|College|6500|High Risk|0|0|0| +3665|M|D|College|6500|High Risk|0|0|0| +3666|F|D|College|6500|High Risk|0|0|0| +3667|M|W|College|6500|High Risk|0|0|0| +3668|F|W|College|6500|High Risk|0|0|0| +3669|M|U|College|6500|High Risk|0|0|0| +3670|F|U|College|6500|High Risk|0|0|0| +3671|M|M|2 yr Degree|6500|High Risk|0|0|0| +3672|F|M|2 yr Degree|6500|High Risk|0|0|0| +3673|M|S|2 yr Degree|6500|High Risk|0|0|0| +3674|F|S|2 yr Degree|6500|High Risk|0|0|0| +3675|M|D|2 yr Degree|6500|High Risk|0|0|0| +3676|F|D|2 yr Degree|6500|High Risk|0|0|0| +3677|M|W|2 yr Degree|6500|High Risk|0|0|0| +3678|F|W|2 yr Degree|6500|High Risk|0|0|0| +3679|M|U|2 yr Degree|6500|High Risk|0|0|0| +3680|F|U|2 yr Degree|6500|High Risk|0|0|0| +3681|M|M|4 yr Degree|6500|High Risk|0|0|0| +3682|F|M|4 yr Degree|6500|High Risk|0|0|0| +3683|M|S|4 yr Degree|6500|High Risk|0|0|0| +3684|F|S|4 yr Degree|6500|High Risk|0|0|0| +3685|M|D|4 yr Degree|6500|High Risk|0|0|0| +3686|F|D|4 yr Degree|6500|High Risk|0|0|0| +3687|M|W|4 yr Degree|6500|High Risk|0|0|0| +3688|F|W|4 yr Degree|6500|High Risk|0|0|0| +3689|M|U|4 yr Degree|6500|High Risk|0|0|0| +3690|F|U|4 yr Degree|6500|High Risk|0|0|0| +3691|M|M|Advanced Degree|6500|High Risk|0|0|0| +3692|F|M|Advanced Degree|6500|High Risk|0|0|0| +3693|M|S|Advanced Degree|6500|High Risk|0|0|0| +3694|F|S|Advanced Degree|6500|High Risk|0|0|0| +3695|M|D|Advanced Degree|6500|High Risk|0|0|0| +3696|F|D|Advanced Degree|6500|High Risk|0|0|0| +3697|M|W|Advanced Degree|6500|High Risk|0|0|0| +3698|F|W|Advanced Degree|6500|High Risk|0|0|0| +3699|M|U|Advanced Degree|6500|High Risk|0|0|0| +3700|F|U|Advanced Degree|6500|High Risk|0|0|0| +3701|M|M|Unknown|6500|High Risk|0|0|0| +3702|F|M|Unknown|6500|High Risk|0|0|0| +3703|M|S|Unknown|6500|High Risk|0|0|0| +3704|F|S|Unknown|6500|High Risk|0|0|0| +3705|M|D|Unknown|6500|High Risk|0|0|0| +3706|F|D|Unknown|6500|High Risk|0|0|0| +3707|M|W|Unknown|6500|High Risk|0|0|0| +3708|F|W|Unknown|6500|High Risk|0|0|0| +3709|M|U|Unknown|6500|High Risk|0|0|0| +3710|F|U|Unknown|6500|High Risk|0|0|0| +3711|M|M|Primary|7000|High Risk|0|0|0| +3712|F|M|Primary|7000|High Risk|0|0|0| +3713|M|S|Primary|7000|High Risk|0|0|0| +3714|F|S|Primary|7000|High Risk|0|0|0| +3715|M|D|Primary|7000|High Risk|0|0|0| +3716|F|D|Primary|7000|High Risk|0|0|0| +3717|M|W|Primary|7000|High Risk|0|0|0| +3718|F|W|Primary|7000|High Risk|0|0|0| +3719|M|U|Primary|7000|High Risk|0|0|0| +3720|F|U|Primary|7000|High Risk|0|0|0| +3721|M|M|Secondary|7000|High Risk|0|0|0| +3722|F|M|Secondary|7000|High Risk|0|0|0| +3723|M|S|Secondary|7000|High Risk|0|0|0| +3724|F|S|Secondary|7000|High Risk|0|0|0| +3725|M|D|Secondary|7000|High Risk|0|0|0| +3726|F|D|Secondary|7000|High Risk|0|0|0| +3727|M|W|Secondary|7000|High Risk|0|0|0| +3728|F|W|Secondary|7000|High Risk|0|0|0| +3729|M|U|Secondary|7000|High Risk|0|0|0| +3730|F|U|Secondary|7000|High Risk|0|0|0| +3731|M|M|College|7000|High Risk|0|0|0| +3732|F|M|College|7000|High Risk|0|0|0| +3733|M|S|College|7000|High Risk|0|0|0| +3734|F|S|College|7000|High Risk|0|0|0| +3735|M|D|College|7000|High Risk|0|0|0| +3736|F|D|College|7000|High Risk|0|0|0| +3737|M|W|College|7000|High Risk|0|0|0| +3738|F|W|College|7000|High Risk|0|0|0| +3739|M|U|College|7000|High Risk|0|0|0| +3740|F|U|College|7000|High Risk|0|0|0| +3741|M|M|2 yr Degree|7000|High Risk|0|0|0| +3742|F|M|2 yr Degree|7000|High Risk|0|0|0| +3743|M|S|2 yr Degree|7000|High Risk|0|0|0| +3744|F|S|2 yr Degree|7000|High Risk|0|0|0| +3745|M|D|2 yr Degree|7000|High Risk|0|0|0| +3746|F|D|2 yr Degree|7000|High Risk|0|0|0| +3747|M|W|2 yr Degree|7000|High Risk|0|0|0| +3748|F|W|2 yr Degree|7000|High Risk|0|0|0| +3749|M|U|2 yr Degree|7000|High Risk|0|0|0| +3750|F|U|2 yr Degree|7000|High Risk|0|0|0| +3751|M|M|4 yr Degree|7000|High Risk|0|0|0| +3752|F|M|4 yr Degree|7000|High Risk|0|0|0| +3753|M|S|4 yr Degree|7000|High Risk|0|0|0| +3754|F|S|4 yr Degree|7000|High Risk|0|0|0| +3755|M|D|4 yr Degree|7000|High Risk|0|0|0| +3756|F|D|4 yr Degree|7000|High Risk|0|0|0| +3757|M|W|4 yr Degree|7000|High Risk|0|0|0| +3758|F|W|4 yr Degree|7000|High Risk|0|0|0| +3759|M|U|4 yr Degree|7000|High Risk|0|0|0| +3760|F|U|4 yr Degree|7000|High Risk|0|0|0| +3761|M|M|Advanced Degree|7000|High Risk|0|0|0| +3762|F|M|Advanced Degree|7000|High Risk|0|0|0| +3763|M|S|Advanced Degree|7000|High Risk|0|0|0| +3764|F|S|Advanced Degree|7000|High Risk|0|0|0| +3765|M|D|Advanced Degree|7000|High Risk|0|0|0| +3766|F|D|Advanced Degree|7000|High Risk|0|0|0| +3767|M|W|Advanced Degree|7000|High Risk|0|0|0| +3768|F|W|Advanced Degree|7000|High Risk|0|0|0| +3769|M|U|Advanced Degree|7000|High Risk|0|0|0| +3770|F|U|Advanced Degree|7000|High Risk|0|0|0| +3771|M|M|Unknown|7000|High Risk|0|0|0| +3772|F|M|Unknown|7000|High Risk|0|0|0| +3773|M|S|Unknown|7000|High Risk|0|0|0| +3774|F|S|Unknown|7000|High Risk|0|0|0| +3775|M|D|Unknown|7000|High Risk|0|0|0| +3776|F|D|Unknown|7000|High Risk|0|0|0| +3777|M|W|Unknown|7000|High Risk|0|0|0| +3778|F|W|Unknown|7000|High Risk|0|0|0| +3779|M|U|Unknown|7000|High Risk|0|0|0| +3780|F|U|Unknown|7000|High Risk|0|0|0| +3781|M|M|Primary|7500|High Risk|0|0|0| +3782|F|M|Primary|7500|High Risk|0|0|0| +3783|M|S|Primary|7500|High Risk|0|0|0| +3784|F|S|Primary|7500|High Risk|0|0|0| +3785|M|D|Primary|7500|High Risk|0|0|0| +3786|F|D|Primary|7500|High Risk|0|0|0| +3787|M|W|Primary|7500|High Risk|0|0|0| +3788|F|W|Primary|7500|High Risk|0|0|0| +3789|M|U|Primary|7500|High Risk|0|0|0| +3790|F|U|Primary|7500|High Risk|0|0|0| +3791|M|M|Secondary|7500|High Risk|0|0|0| +3792|F|M|Secondary|7500|High Risk|0|0|0| +3793|M|S|Secondary|7500|High Risk|0|0|0| +3794|F|S|Secondary|7500|High Risk|0|0|0| +3795|M|D|Secondary|7500|High Risk|0|0|0| +3796|F|D|Secondary|7500|High Risk|0|0|0| +3797|M|W|Secondary|7500|High Risk|0|0|0| +3798|F|W|Secondary|7500|High Risk|0|0|0| +3799|M|U|Secondary|7500|High Risk|0|0|0| +3800|F|U|Secondary|7500|High Risk|0|0|0| +3801|M|M|College|7500|High Risk|0|0|0| +3802|F|M|College|7500|High Risk|0|0|0| +3803|M|S|College|7500|High Risk|0|0|0| +3804|F|S|College|7500|High Risk|0|0|0| +3805|M|D|College|7500|High Risk|0|0|0| +3806|F|D|College|7500|High Risk|0|0|0| +3807|M|W|College|7500|High Risk|0|0|0| +3808|F|W|College|7500|High Risk|0|0|0| +3809|M|U|College|7500|High Risk|0|0|0| +3810|F|U|College|7500|High Risk|0|0|0| +3811|M|M|2 yr Degree|7500|High Risk|0|0|0| +3812|F|M|2 yr Degree|7500|High Risk|0|0|0| +3813|M|S|2 yr Degree|7500|High Risk|0|0|0| +3814|F|S|2 yr Degree|7500|High Risk|0|0|0| +3815|M|D|2 yr Degree|7500|High Risk|0|0|0| +3816|F|D|2 yr Degree|7500|High Risk|0|0|0| +3817|M|W|2 yr Degree|7500|High Risk|0|0|0| +3818|F|W|2 yr Degree|7500|High Risk|0|0|0| +3819|M|U|2 yr Degree|7500|High Risk|0|0|0| +3820|F|U|2 yr Degree|7500|High Risk|0|0|0| +3821|M|M|4 yr Degree|7500|High Risk|0|0|0| +3822|F|M|4 yr Degree|7500|High Risk|0|0|0| +3823|M|S|4 yr Degree|7500|High Risk|0|0|0| +3824|F|S|4 yr Degree|7500|High Risk|0|0|0| +3825|M|D|4 yr Degree|7500|High Risk|0|0|0| +3826|F|D|4 yr Degree|7500|High Risk|0|0|0| +3827|M|W|4 yr Degree|7500|High Risk|0|0|0| +3828|F|W|4 yr Degree|7500|High Risk|0|0|0| +3829|M|U|4 yr Degree|7500|High Risk|0|0|0| +3830|F|U|4 yr Degree|7500|High Risk|0|0|0| +3831|M|M|Advanced Degree|7500|High Risk|0|0|0| +3832|F|M|Advanced Degree|7500|High Risk|0|0|0| +3833|M|S|Advanced Degree|7500|High Risk|0|0|0| +3834|F|S|Advanced Degree|7500|High Risk|0|0|0| +3835|M|D|Advanced Degree|7500|High Risk|0|0|0| +3836|F|D|Advanced Degree|7500|High Risk|0|0|0| +3837|M|W|Advanced Degree|7500|High Risk|0|0|0| +3838|F|W|Advanced Degree|7500|High Risk|0|0|0| +3839|M|U|Advanced Degree|7500|High Risk|0|0|0| +3840|F|U|Advanced Degree|7500|High Risk|0|0|0| +3841|M|M|Unknown|7500|High Risk|0|0|0| +3842|F|M|Unknown|7500|High Risk|0|0|0| +3843|M|S|Unknown|7500|High Risk|0|0|0| +3844|F|S|Unknown|7500|High Risk|0|0|0| +3845|M|D|Unknown|7500|High Risk|0|0|0| +3846|F|D|Unknown|7500|High Risk|0|0|0| +3847|M|W|Unknown|7500|High Risk|0|0|0| +3848|F|W|Unknown|7500|High Risk|0|0|0| +3849|M|U|Unknown|7500|High Risk|0|0|0| +3850|F|U|Unknown|7500|High Risk|0|0|0| +3851|M|M|Primary|8000|High Risk|0|0|0| +3852|F|M|Primary|8000|High Risk|0|0|0| +3853|M|S|Primary|8000|High Risk|0|0|0| +3854|F|S|Primary|8000|High Risk|0|0|0| +3855|M|D|Primary|8000|High Risk|0|0|0| +3856|F|D|Primary|8000|High Risk|0|0|0| +3857|M|W|Primary|8000|High Risk|0|0|0| +3858|F|W|Primary|8000|High Risk|0|0|0| +3859|M|U|Primary|8000|High Risk|0|0|0| +3860|F|U|Primary|8000|High Risk|0|0|0| +3861|M|M|Secondary|8000|High Risk|0|0|0| +3862|F|M|Secondary|8000|High Risk|0|0|0| +3863|M|S|Secondary|8000|High Risk|0|0|0| +3864|F|S|Secondary|8000|High Risk|0|0|0| +3865|M|D|Secondary|8000|High Risk|0|0|0| +3866|F|D|Secondary|8000|High Risk|0|0|0| +3867|M|W|Secondary|8000|High Risk|0|0|0| +3868|F|W|Secondary|8000|High Risk|0|0|0| +3869|M|U|Secondary|8000|High Risk|0|0|0| +3870|F|U|Secondary|8000|High Risk|0|0|0| +3871|M|M|College|8000|High Risk|0|0|0| +3872|F|M|College|8000|High Risk|0|0|0| +3873|M|S|College|8000|High Risk|0|0|0| +3874|F|S|College|8000|High Risk|0|0|0| +3875|M|D|College|8000|High Risk|0|0|0| +3876|F|D|College|8000|High Risk|0|0|0| +3877|M|W|College|8000|High Risk|0|0|0| +3878|F|W|College|8000|High Risk|0|0|0| +3879|M|U|College|8000|High Risk|0|0|0| +3880|F|U|College|8000|High Risk|0|0|0| +3881|M|M|2 yr Degree|8000|High Risk|0|0|0| +3882|F|M|2 yr Degree|8000|High Risk|0|0|0| +3883|M|S|2 yr Degree|8000|High Risk|0|0|0| +3884|F|S|2 yr Degree|8000|High Risk|0|0|0| +3885|M|D|2 yr Degree|8000|High Risk|0|0|0| +3886|F|D|2 yr Degree|8000|High Risk|0|0|0| +3887|M|W|2 yr Degree|8000|High Risk|0|0|0| +3888|F|W|2 yr Degree|8000|High Risk|0|0|0| +3889|M|U|2 yr Degree|8000|High Risk|0|0|0| +3890|F|U|2 yr Degree|8000|High Risk|0|0|0| +3891|M|M|4 yr Degree|8000|High Risk|0|0|0| +3892|F|M|4 yr Degree|8000|High Risk|0|0|0| +3893|M|S|4 yr Degree|8000|High Risk|0|0|0| +3894|F|S|4 yr Degree|8000|High Risk|0|0|0| +3895|M|D|4 yr Degree|8000|High Risk|0|0|0| +3896|F|D|4 yr Degree|8000|High Risk|0|0|0| +3897|M|W|4 yr Degree|8000|High Risk|0|0|0| +3898|F|W|4 yr Degree|8000|High Risk|0|0|0| +3899|M|U|4 yr Degree|8000|High Risk|0|0|0| +3900|F|U|4 yr Degree|8000|High Risk|0|0|0| +3901|M|M|Advanced Degree|8000|High Risk|0|0|0| +3902|F|M|Advanced Degree|8000|High Risk|0|0|0| +3903|M|S|Advanced Degree|8000|High Risk|0|0|0| +3904|F|S|Advanced Degree|8000|High Risk|0|0|0| +3905|M|D|Advanced Degree|8000|High Risk|0|0|0| +3906|F|D|Advanced Degree|8000|High Risk|0|0|0| +3907|M|W|Advanced Degree|8000|High Risk|0|0|0| +3908|F|W|Advanced Degree|8000|High Risk|0|0|0| +3909|M|U|Advanced Degree|8000|High Risk|0|0|0| +3910|F|U|Advanced Degree|8000|High Risk|0|0|0| +3911|M|M|Unknown|8000|High Risk|0|0|0| +3912|F|M|Unknown|8000|High Risk|0|0|0| +3913|M|S|Unknown|8000|High Risk|0|0|0| +3914|F|S|Unknown|8000|High Risk|0|0|0| +3915|M|D|Unknown|8000|High Risk|0|0|0| +3916|F|D|Unknown|8000|High Risk|0|0|0| +3917|M|W|Unknown|8000|High Risk|0|0|0| +3918|F|W|Unknown|8000|High Risk|0|0|0| +3919|M|U|Unknown|8000|High Risk|0|0|0| +3920|F|U|Unknown|8000|High Risk|0|0|0| +3921|M|M|Primary|8500|High Risk|0|0|0| +3922|F|M|Primary|8500|High Risk|0|0|0| +3923|M|S|Primary|8500|High Risk|0|0|0| +3924|F|S|Primary|8500|High Risk|0|0|0| +3925|M|D|Primary|8500|High Risk|0|0|0| +3926|F|D|Primary|8500|High Risk|0|0|0| +3927|M|W|Primary|8500|High Risk|0|0|0| +3928|F|W|Primary|8500|High Risk|0|0|0| +3929|M|U|Primary|8500|High Risk|0|0|0| +3930|F|U|Primary|8500|High Risk|0|0|0| +3931|M|M|Secondary|8500|High Risk|0|0|0| +3932|F|M|Secondary|8500|High Risk|0|0|0| +3933|M|S|Secondary|8500|High Risk|0|0|0| +3934|F|S|Secondary|8500|High Risk|0|0|0| +3935|M|D|Secondary|8500|High Risk|0|0|0| +3936|F|D|Secondary|8500|High Risk|0|0|0| +3937|M|W|Secondary|8500|High Risk|0|0|0| +3938|F|W|Secondary|8500|High Risk|0|0|0| +3939|M|U|Secondary|8500|High Risk|0|0|0| +3940|F|U|Secondary|8500|High Risk|0|0|0| +3941|M|M|College|8500|High Risk|0|0|0| +3942|F|M|College|8500|High Risk|0|0|0| +3943|M|S|College|8500|High Risk|0|0|0| +3944|F|S|College|8500|High Risk|0|0|0| +3945|M|D|College|8500|High Risk|0|0|0| +3946|F|D|College|8500|High Risk|0|0|0| +3947|M|W|College|8500|High Risk|0|0|0| +3948|F|W|College|8500|High Risk|0|0|0| +3949|M|U|College|8500|High Risk|0|0|0| +3950|F|U|College|8500|High Risk|0|0|0| +3951|M|M|2 yr Degree|8500|High Risk|0|0|0| +3952|F|M|2 yr Degree|8500|High Risk|0|0|0| +3953|M|S|2 yr Degree|8500|High Risk|0|0|0| +3954|F|S|2 yr Degree|8500|High Risk|0|0|0| +3955|M|D|2 yr Degree|8500|High Risk|0|0|0| +3956|F|D|2 yr Degree|8500|High Risk|0|0|0| +3957|M|W|2 yr Degree|8500|High Risk|0|0|0| +3958|F|W|2 yr Degree|8500|High Risk|0|0|0| +3959|M|U|2 yr Degree|8500|High Risk|0|0|0| +3960|F|U|2 yr Degree|8500|High Risk|0|0|0| +3961|M|M|4 yr Degree|8500|High Risk|0|0|0| +3962|F|M|4 yr Degree|8500|High Risk|0|0|0| +3963|M|S|4 yr Degree|8500|High Risk|0|0|0| +3964|F|S|4 yr Degree|8500|High Risk|0|0|0| +3965|M|D|4 yr Degree|8500|High Risk|0|0|0| +3966|F|D|4 yr Degree|8500|High Risk|0|0|0| +3967|M|W|4 yr Degree|8500|High Risk|0|0|0| +3968|F|W|4 yr Degree|8500|High Risk|0|0|0| +3969|M|U|4 yr Degree|8500|High Risk|0|0|0| +3970|F|U|4 yr Degree|8500|High Risk|0|0|0| +3971|M|M|Advanced Degree|8500|High Risk|0|0|0| +3972|F|M|Advanced Degree|8500|High Risk|0|0|0| +3973|M|S|Advanced Degree|8500|High Risk|0|0|0| +3974|F|S|Advanced Degree|8500|High Risk|0|0|0| +3975|M|D|Advanced Degree|8500|High Risk|0|0|0| +3976|F|D|Advanced Degree|8500|High Risk|0|0|0| +3977|M|W|Advanced Degree|8500|High Risk|0|0|0| +3978|F|W|Advanced Degree|8500|High Risk|0|0|0| +3979|M|U|Advanced Degree|8500|High Risk|0|0|0| +3980|F|U|Advanced Degree|8500|High Risk|0|0|0| +3981|M|M|Unknown|8500|High Risk|0|0|0| +3982|F|M|Unknown|8500|High Risk|0|0|0| +3983|M|S|Unknown|8500|High Risk|0|0|0| +3984|F|S|Unknown|8500|High Risk|0|0|0| +3985|M|D|Unknown|8500|High Risk|0|0|0| +3986|F|D|Unknown|8500|High Risk|0|0|0| +3987|M|W|Unknown|8500|High Risk|0|0|0| +3988|F|W|Unknown|8500|High Risk|0|0|0| +3989|M|U|Unknown|8500|High Risk|0|0|0| +3990|F|U|Unknown|8500|High Risk|0|0|0| +3991|M|M|Primary|9000|High Risk|0|0|0| +3992|F|M|Primary|9000|High Risk|0|0|0| +3993|M|S|Primary|9000|High Risk|0|0|0| +3994|F|S|Primary|9000|High Risk|0|0|0| +3995|M|D|Primary|9000|High Risk|0|0|0| +3996|F|D|Primary|9000|High Risk|0|0|0| +3997|M|W|Primary|9000|High Risk|0|0|0| +3998|F|W|Primary|9000|High Risk|0|0|0| +3999|M|U|Primary|9000|High Risk|0|0|0| +4000|F|U|Primary|9000|High Risk|0|0|0| +4001|M|M|Secondary|9000|High Risk|0|0|0| +4002|F|M|Secondary|9000|High Risk|0|0|0| +4003|M|S|Secondary|9000|High Risk|0|0|0| +4004|F|S|Secondary|9000|High Risk|0|0|0| +4005|M|D|Secondary|9000|High Risk|0|0|0| +4006|F|D|Secondary|9000|High Risk|0|0|0| +4007|M|W|Secondary|9000|High Risk|0|0|0| +4008|F|W|Secondary|9000|High Risk|0|0|0| +4009|M|U|Secondary|9000|High Risk|0|0|0| +4010|F|U|Secondary|9000|High Risk|0|0|0| +4011|M|M|College|9000|High Risk|0|0|0| +4012|F|M|College|9000|High Risk|0|0|0| +4013|M|S|College|9000|High Risk|0|0|0| +4014|F|S|College|9000|High Risk|0|0|0| +4015|M|D|College|9000|High Risk|0|0|0| +4016|F|D|College|9000|High Risk|0|0|0| +4017|M|W|College|9000|High Risk|0|0|0| +4018|F|W|College|9000|High Risk|0|0|0| +4019|M|U|College|9000|High Risk|0|0|0| +4020|F|U|College|9000|High Risk|0|0|0| +4021|M|M|2 yr Degree|9000|High Risk|0|0|0| +4022|F|M|2 yr Degree|9000|High Risk|0|0|0| +4023|M|S|2 yr Degree|9000|High Risk|0|0|0| +4024|F|S|2 yr Degree|9000|High Risk|0|0|0| +4025|M|D|2 yr Degree|9000|High Risk|0|0|0| +4026|F|D|2 yr Degree|9000|High Risk|0|0|0| +4027|M|W|2 yr Degree|9000|High Risk|0|0|0| +4028|F|W|2 yr Degree|9000|High Risk|0|0|0| +4029|M|U|2 yr Degree|9000|High Risk|0|0|0| +4030|F|U|2 yr Degree|9000|High Risk|0|0|0| +4031|M|M|4 yr Degree|9000|High Risk|0|0|0| +4032|F|M|4 yr Degree|9000|High Risk|0|0|0| +4033|M|S|4 yr Degree|9000|High Risk|0|0|0| +4034|F|S|4 yr Degree|9000|High Risk|0|0|0| +4035|M|D|4 yr Degree|9000|High Risk|0|0|0| +4036|F|D|4 yr Degree|9000|High Risk|0|0|0| +4037|M|W|4 yr Degree|9000|High Risk|0|0|0| +4038|F|W|4 yr Degree|9000|High Risk|0|0|0| +4039|M|U|4 yr Degree|9000|High Risk|0|0|0| +4040|F|U|4 yr Degree|9000|High Risk|0|0|0| +4041|M|M|Advanced Degree|9000|High Risk|0|0|0| +4042|F|M|Advanced Degree|9000|High Risk|0|0|0| +4043|M|S|Advanced Degree|9000|High Risk|0|0|0| +4044|F|S|Advanced Degree|9000|High Risk|0|0|0| +4045|M|D|Advanced Degree|9000|High Risk|0|0|0| +4046|F|D|Advanced Degree|9000|High Risk|0|0|0| +4047|M|W|Advanced Degree|9000|High Risk|0|0|0| +4048|F|W|Advanced Degree|9000|High Risk|0|0|0| +4049|M|U|Advanced Degree|9000|High Risk|0|0|0| +4050|F|U|Advanced Degree|9000|High Risk|0|0|0| +4051|M|M|Unknown|9000|High Risk|0|0|0| +4052|F|M|Unknown|9000|High Risk|0|0|0| +4053|M|S|Unknown|9000|High Risk|0|0|0| +4054|F|S|Unknown|9000|High Risk|0|0|0| +4055|M|D|Unknown|9000|High Risk|0|0|0| +4056|F|D|Unknown|9000|High Risk|0|0|0| +4057|M|W|Unknown|9000|High Risk|0|0|0| +4058|F|W|Unknown|9000|High Risk|0|0|0| +4059|M|U|Unknown|9000|High Risk|0|0|0| +4060|F|U|Unknown|9000|High Risk|0|0|0| +4061|M|M|Primary|9500|High Risk|0|0|0| +4062|F|M|Primary|9500|High Risk|0|0|0| +4063|M|S|Primary|9500|High Risk|0|0|0| +4064|F|S|Primary|9500|High Risk|0|0|0| +4065|M|D|Primary|9500|High Risk|0|0|0| +4066|F|D|Primary|9500|High Risk|0|0|0| +4067|M|W|Primary|9500|High Risk|0|0|0| +4068|F|W|Primary|9500|High Risk|0|0|0| +4069|M|U|Primary|9500|High Risk|0|0|0| +4070|F|U|Primary|9500|High Risk|0|0|0| +4071|M|M|Secondary|9500|High Risk|0|0|0| +4072|F|M|Secondary|9500|High Risk|0|0|0| +4073|M|S|Secondary|9500|High Risk|0|0|0| +4074|F|S|Secondary|9500|High Risk|0|0|0| +4075|M|D|Secondary|9500|High Risk|0|0|0| +4076|F|D|Secondary|9500|High Risk|0|0|0| +4077|M|W|Secondary|9500|High Risk|0|0|0| +4078|F|W|Secondary|9500|High Risk|0|0|0| +4079|M|U|Secondary|9500|High Risk|0|0|0| +4080|F|U|Secondary|9500|High Risk|0|0|0| +4081|M|M|College|9500|High Risk|0|0|0| +4082|F|M|College|9500|High Risk|0|0|0| +4083|M|S|College|9500|High Risk|0|0|0| +4084|F|S|College|9500|High Risk|0|0|0| +4085|M|D|College|9500|High Risk|0|0|0| +4086|F|D|College|9500|High Risk|0|0|0| +4087|M|W|College|9500|High Risk|0|0|0| +4088|F|W|College|9500|High Risk|0|0|0| +4089|M|U|College|9500|High Risk|0|0|0| +4090|F|U|College|9500|High Risk|0|0|0| +4091|M|M|2 yr Degree|9500|High Risk|0|0|0| +4092|F|M|2 yr Degree|9500|High Risk|0|0|0| +4093|M|S|2 yr Degree|9500|High Risk|0|0|0| +4094|F|S|2 yr Degree|9500|High Risk|0|0|0| +4095|M|D|2 yr Degree|9500|High Risk|0|0|0| +4096|F|D|2 yr Degree|9500|High Risk|0|0|0| +4097|M|W|2 yr Degree|9500|High Risk|0|0|0| +4098|F|W|2 yr Degree|9500|High Risk|0|0|0| +4099|M|U|2 yr Degree|9500|High Risk|0|0|0| +4100|F|U|2 yr Degree|9500|High Risk|0|0|0| +4101|M|M|4 yr Degree|9500|High Risk|0|0|0| +4102|F|M|4 yr Degree|9500|High Risk|0|0|0| +4103|M|S|4 yr Degree|9500|High Risk|0|0|0| +4104|F|S|4 yr Degree|9500|High Risk|0|0|0| +4105|M|D|4 yr Degree|9500|High Risk|0|0|0| +4106|F|D|4 yr Degree|9500|High Risk|0|0|0| +4107|M|W|4 yr Degree|9500|High Risk|0|0|0| +4108|F|W|4 yr Degree|9500|High Risk|0|0|0| +4109|M|U|4 yr Degree|9500|High Risk|0|0|0| +4110|F|U|4 yr Degree|9500|High Risk|0|0|0| +4111|M|M|Advanced Degree|9500|High Risk|0|0|0| +4112|F|M|Advanced Degree|9500|High Risk|0|0|0| +4113|M|S|Advanced Degree|9500|High Risk|0|0|0| +4114|F|S|Advanced Degree|9500|High Risk|0|0|0| +4115|M|D|Advanced Degree|9500|High Risk|0|0|0| +4116|F|D|Advanced Degree|9500|High Risk|0|0|0| +4117|M|W|Advanced Degree|9500|High Risk|0|0|0| +4118|F|W|Advanced Degree|9500|High Risk|0|0|0| +4119|M|U|Advanced Degree|9500|High Risk|0|0|0| +4120|F|U|Advanced Degree|9500|High Risk|0|0|0| +4121|M|M|Unknown|9500|High Risk|0|0|0| +4122|F|M|Unknown|9500|High Risk|0|0|0| +4123|M|S|Unknown|9500|High Risk|0|0|0| +4124|F|S|Unknown|9500|High Risk|0|0|0| +4125|M|D|Unknown|9500|High Risk|0|0|0| +4126|F|D|Unknown|9500|High Risk|0|0|0| +4127|M|W|Unknown|9500|High Risk|0|0|0| +4128|F|W|Unknown|9500|High Risk|0|0|0| +4129|M|U|Unknown|9500|High Risk|0|0|0| +4130|F|U|Unknown|9500|High Risk|0|0|0| +4131|M|M|Primary|10000|High Risk|0|0|0| +4132|F|M|Primary|10000|High Risk|0|0|0| +4133|M|S|Primary|10000|High Risk|0|0|0| +4134|F|S|Primary|10000|High Risk|0|0|0| +4135|M|D|Primary|10000|High Risk|0|0|0| +4136|F|D|Primary|10000|High Risk|0|0|0| +4137|M|W|Primary|10000|High Risk|0|0|0| +4138|F|W|Primary|10000|High Risk|0|0|0| +4139|M|U|Primary|10000|High Risk|0|0|0| +4140|F|U|Primary|10000|High Risk|0|0|0| +4141|M|M|Secondary|10000|High Risk|0|0|0| +4142|F|M|Secondary|10000|High Risk|0|0|0| +4143|M|S|Secondary|10000|High Risk|0|0|0| +4144|F|S|Secondary|10000|High Risk|0|0|0| +4145|M|D|Secondary|10000|High Risk|0|0|0| +4146|F|D|Secondary|10000|High Risk|0|0|0| +4147|M|W|Secondary|10000|High Risk|0|0|0| +4148|F|W|Secondary|10000|High Risk|0|0|0| +4149|M|U|Secondary|10000|High Risk|0|0|0| +4150|F|U|Secondary|10000|High Risk|0|0|0| +4151|M|M|College|10000|High Risk|0|0|0| +4152|F|M|College|10000|High Risk|0|0|0| +4153|M|S|College|10000|High Risk|0|0|0| +4154|F|S|College|10000|High Risk|0|0|0| +4155|M|D|College|10000|High Risk|0|0|0| +4156|F|D|College|10000|High Risk|0|0|0| +4157|M|W|College|10000|High Risk|0|0|0| +4158|F|W|College|10000|High Risk|0|0|0| +4159|M|U|College|10000|High Risk|0|0|0| +4160|F|U|College|10000|High Risk|0|0|0| +4161|M|M|2 yr Degree|10000|High Risk|0|0|0| +4162|F|M|2 yr Degree|10000|High Risk|0|0|0| +4163|M|S|2 yr Degree|10000|High Risk|0|0|0| +4164|F|S|2 yr Degree|10000|High Risk|0|0|0| +4165|M|D|2 yr Degree|10000|High Risk|0|0|0| +4166|F|D|2 yr Degree|10000|High Risk|0|0|0| +4167|M|W|2 yr Degree|10000|High Risk|0|0|0| +4168|F|W|2 yr Degree|10000|High Risk|0|0|0| +4169|M|U|2 yr Degree|10000|High Risk|0|0|0| +4170|F|U|2 yr Degree|10000|High Risk|0|0|0| +4171|M|M|4 yr Degree|10000|High Risk|0|0|0| +4172|F|M|4 yr Degree|10000|High Risk|0|0|0| +4173|M|S|4 yr Degree|10000|High Risk|0|0|0| +4174|F|S|4 yr Degree|10000|High Risk|0|0|0| +4175|M|D|4 yr Degree|10000|High Risk|0|0|0| +4176|F|D|4 yr Degree|10000|High Risk|0|0|0| +4177|M|W|4 yr Degree|10000|High Risk|0|0|0| +4178|F|W|4 yr Degree|10000|High Risk|0|0|0| +4179|M|U|4 yr Degree|10000|High Risk|0|0|0| +4180|F|U|4 yr Degree|10000|High Risk|0|0|0| +4181|M|M|Advanced Degree|10000|High Risk|0|0|0| +4182|F|M|Advanced Degree|10000|High Risk|0|0|0| +4183|M|S|Advanced Degree|10000|High Risk|0|0|0| +4184|F|S|Advanced Degree|10000|High Risk|0|0|0| +4185|M|D|Advanced Degree|10000|High Risk|0|0|0| +4186|F|D|Advanced Degree|10000|High Risk|0|0|0| +4187|M|W|Advanced Degree|10000|High Risk|0|0|0| +4188|F|W|Advanced Degree|10000|High Risk|0|0|0| +4189|M|U|Advanced Degree|10000|High Risk|0|0|0| +4190|F|U|Advanced Degree|10000|High Risk|0|0|0| +4191|M|M|Unknown|10000|High Risk|0|0|0| +4192|F|M|Unknown|10000|High Risk|0|0|0| +4193|M|S|Unknown|10000|High Risk|0|0|0| +4194|F|S|Unknown|10000|High Risk|0|0|0| +4195|M|D|Unknown|10000|High Risk|0|0|0| +4196|F|D|Unknown|10000|High Risk|0|0|0| +4197|M|W|Unknown|10000|High Risk|0|0|0| +4198|F|W|Unknown|10000|High Risk|0|0|0| +4199|M|U|Unknown|10000|High Risk|0|0|0| +4200|F|U|Unknown|10000|High Risk|0|0|0| +4201|M|M|Primary|500|Unknown|0|0|0| +4202|F|M|Primary|500|Unknown|0|0|0| +4203|M|S|Primary|500|Unknown|0|0|0| +4204|F|S|Primary|500|Unknown|0|0|0| +4205|M|D|Primary|500|Unknown|0|0|0| +4206|F|D|Primary|500|Unknown|0|0|0| +4207|M|W|Primary|500|Unknown|0|0|0| +4208|F|W|Primary|500|Unknown|0|0|0| +4209|M|U|Primary|500|Unknown|0|0|0| +4210|F|U|Primary|500|Unknown|0|0|0| +4211|M|M|Secondary|500|Unknown|0|0|0| +4212|F|M|Secondary|500|Unknown|0|0|0| +4213|M|S|Secondary|500|Unknown|0|0|0| +4214|F|S|Secondary|500|Unknown|0|0|0| +4215|M|D|Secondary|500|Unknown|0|0|0| +4216|F|D|Secondary|500|Unknown|0|0|0| +4217|M|W|Secondary|500|Unknown|0|0|0| +4218|F|W|Secondary|500|Unknown|0|0|0| +4219|M|U|Secondary|500|Unknown|0|0|0| +4220|F|U|Secondary|500|Unknown|0|0|0| +4221|M|M|College|500|Unknown|0|0|0| +4222|F|M|College|500|Unknown|0|0|0| +4223|M|S|College|500|Unknown|0|0|0| +4224|F|S|College|500|Unknown|0|0|0| +4225|M|D|College|500|Unknown|0|0|0| +4226|F|D|College|500|Unknown|0|0|0| +4227|M|W|College|500|Unknown|0|0|0| +4228|F|W|College|500|Unknown|0|0|0| +4229|M|U|College|500|Unknown|0|0|0| +4230|F|U|College|500|Unknown|0|0|0| +4231|M|M|2 yr Degree|500|Unknown|0|0|0| +4232|F|M|2 yr Degree|500|Unknown|0|0|0| +4233|M|S|2 yr Degree|500|Unknown|0|0|0| +4234|F|S|2 yr Degree|500|Unknown|0|0|0| +4235|M|D|2 yr Degree|500|Unknown|0|0|0| +4236|F|D|2 yr Degree|500|Unknown|0|0|0| +4237|M|W|2 yr Degree|500|Unknown|0|0|0| +4238|F|W|2 yr Degree|500|Unknown|0|0|0| +4239|M|U|2 yr Degree|500|Unknown|0|0|0| +4240|F|U|2 yr Degree|500|Unknown|0|0|0| +4241|M|M|4 yr Degree|500|Unknown|0|0|0| +4242|F|M|4 yr Degree|500|Unknown|0|0|0| +4243|M|S|4 yr Degree|500|Unknown|0|0|0| +4244|F|S|4 yr Degree|500|Unknown|0|0|0| +4245|M|D|4 yr Degree|500|Unknown|0|0|0| +4246|F|D|4 yr Degree|500|Unknown|0|0|0| +4247|M|W|4 yr Degree|500|Unknown|0|0|0| +4248|F|W|4 yr Degree|500|Unknown|0|0|0| +4249|M|U|4 yr Degree|500|Unknown|0|0|0| +4250|F|U|4 yr Degree|500|Unknown|0|0|0| +4251|M|M|Advanced Degree|500|Unknown|0|0|0| +4252|F|M|Advanced Degree|500|Unknown|0|0|0| +4253|M|S|Advanced Degree|500|Unknown|0|0|0| +4254|F|S|Advanced Degree|500|Unknown|0|0|0| +4255|M|D|Advanced Degree|500|Unknown|0|0|0| +4256|F|D|Advanced Degree|500|Unknown|0|0|0| +4257|M|W|Advanced Degree|500|Unknown|0|0|0| +4258|F|W|Advanced Degree|500|Unknown|0|0|0| +4259|M|U|Advanced Degree|500|Unknown|0|0|0| +4260|F|U|Advanced Degree|500|Unknown|0|0|0| +4261|M|M|Unknown|500|Unknown|0|0|0| +4262|F|M|Unknown|500|Unknown|0|0|0| +4263|M|S|Unknown|500|Unknown|0|0|0| +4264|F|S|Unknown|500|Unknown|0|0|0| +4265|M|D|Unknown|500|Unknown|0|0|0| +4266|F|D|Unknown|500|Unknown|0|0|0| +4267|M|W|Unknown|500|Unknown|0|0|0| +4268|F|W|Unknown|500|Unknown|0|0|0| +4269|M|U|Unknown|500|Unknown|0|0|0| +4270|F|U|Unknown|500|Unknown|0|0|0| +4271|M|M|Primary|1000|Unknown|0|0|0| +4272|F|M|Primary|1000|Unknown|0|0|0| +4273|M|S|Primary|1000|Unknown|0|0|0| +4274|F|S|Primary|1000|Unknown|0|0|0| +4275|M|D|Primary|1000|Unknown|0|0|0| +4276|F|D|Primary|1000|Unknown|0|0|0| +4277|M|W|Primary|1000|Unknown|0|0|0| +4278|F|W|Primary|1000|Unknown|0|0|0| +4279|M|U|Primary|1000|Unknown|0|0|0| +4280|F|U|Primary|1000|Unknown|0|0|0| +4281|M|M|Secondary|1000|Unknown|0|0|0| +4282|F|M|Secondary|1000|Unknown|0|0|0| +4283|M|S|Secondary|1000|Unknown|0|0|0| +4284|F|S|Secondary|1000|Unknown|0|0|0| +4285|M|D|Secondary|1000|Unknown|0|0|0| +4286|F|D|Secondary|1000|Unknown|0|0|0| +4287|M|W|Secondary|1000|Unknown|0|0|0| +4288|F|W|Secondary|1000|Unknown|0|0|0| +4289|M|U|Secondary|1000|Unknown|0|0|0| +4290|F|U|Secondary|1000|Unknown|0|0|0| +4291|M|M|College|1000|Unknown|0|0|0| +4292|F|M|College|1000|Unknown|0|0|0| +4293|M|S|College|1000|Unknown|0|0|0| +4294|F|S|College|1000|Unknown|0|0|0| +4295|M|D|College|1000|Unknown|0|0|0| +4296|F|D|College|1000|Unknown|0|0|0| +4297|M|W|College|1000|Unknown|0|0|0| +4298|F|W|College|1000|Unknown|0|0|0| +4299|M|U|College|1000|Unknown|0|0|0| +4300|F|U|College|1000|Unknown|0|0|0| +4301|M|M|2 yr Degree|1000|Unknown|0|0|0| +4302|F|M|2 yr Degree|1000|Unknown|0|0|0| +4303|M|S|2 yr Degree|1000|Unknown|0|0|0| +4304|F|S|2 yr Degree|1000|Unknown|0|0|0| +4305|M|D|2 yr Degree|1000|Unknown|0|0|0| +4306|F|D|2 yr Degree|1000|Unknown|0|0|0| +4307|M|W|2 yr Degree|1000|Unknown|0|0|0| +4308|F|W|2 yr Degree|1000|Unknown|0|0|0| +4309|M|U|2 yr Degree|1000|Unknown|0|0|0| +4310|F|U|2 yr Degree|1000|Unknown|0|0|0| +4311|M|M|4 yr Degree|1000|Unknown|0|0|0| +4312|F|M|4 yr Degree|1000|Unknown|0|0|0| +4313|M|S|4 yr Degree|1000|Unknown|0|0|0| +4314|F|S|4 yr Degree|1000|Unknown|0|0|0| +4315|M|D|4 yr Degree|1000|Unknown|0|0|0| +4316|F|D|4 yr Degree|1000|Unknown|0|0|0| +4317|M|W|4 yr Degree|1000|Unknown|0|0|0| +4318|F|W|4 yr Degree|1000|Unknown|0|0|0| +4319|M|U|4 yr Degree|1000|Unknown|0|0|0| +4320|F|U|4 yr Degree|1000|Unknown|0|0|0| +4321|M|M|Advanced Degree|1000|Unknown|0|0|0| +4322|F|M|Advanced Degree|1000|Unknown|0|0|0| +4323|M|S|Advanced Degree|1000|Unknown|0|0|0| +4324|F|S|Advanced Degree|1000|Unknown|0|0|0| +4325|M|D|Advanced Degree|1000|Unknown|0|0|0| +4326|F|D|Advanced Degree|1000|Unknown|0|0|0| +4327|M|W|Advanced Degree|1000|Unknown|0|0|0| +4328|F|W|Advanced Degree|1000|Unknown|0|0|0| +4329|M|U|Advanced Degree|1000|Unknown|0|0|0| +4330|F|U|Advanced Degree|1000|Unknown|0|0|0| +4331|M|M|Unknown|1000|Unknown|0|0|0| +4332|F|M|Unknown|1000|Unknown|0|0|0| +4333|M|S|Unknown|1000|Unknown|0|0|0| +4334|F|S|Unknown|1000|Unknown|0|0|0| +4335|M|D|Unknown|1000|Unknown|0|0|0| +4336|F|D|Unknown|1000|Unknown|0|0|0| +4337|M|W|Unknown|1000|Unknown|0|0|0| +4338|F|W|Unknown|1000|Unknown|0|0|0| +4339|M|U|Unknown|1000|Unknown|0|0|0| +4340|F|U|Unknown|1000|Unknown|0|0|0| +4341|M|M|Primary|1500|Unknown|0|0|0| +4342|F|M|Primary|1500|Unknown|0|0|0| +4343|M|S|Primary|1500|Unknown|0|0|0| +4344|F|S|Primary|1500|Unknown|0|0|0| +4345|M|D|Primary|1500|Unknown|0|0|0| +4346|F|D|Primary|1500|Unknown|0|0|0| +4347|M|W|Primary|1500|Unknown|0|0|0| +4348|F|W|Primary|1500|Unknown|0|0|0| +4349|M|U|Primary|1500|Unknown|0|0|0| +4350|F|U|Primary|1500|Unknown|0|0|0| +4351|M|M|Secondary|1500|Unknown|0|0|0| +4352|F|M|Secondary|1500|Unknown|0|0|0| +4353|M|S|Secondary|1500|Unknown|0|0|0| +4354|F|S|Secondary|1500|Unknown|0|0|0| +4355|M|D|Secondary|1500|Unknown|0|0|0| +4356|F|D|Secondary|1500|Unknown|0|0|0| +4357|M|W|Secondary|1500|Unknown|0|0|0| +4358|F|W|Secondary|1500|Unknown|0|0|0| +4359|M|U|Secondary|1500|Unknown|0|0|0| +4360|F|U|Secondary|1500|Unknown|0|0|0| +4361|M|M|College|1500|Unknown|0|0|0| +4362|F|M|College|1500|Unknown|0|0|0| +4363|M|S|College|1500|Unknown|0|0|0| +4364|F|S|College|1500|Unknown|0|0|0| +4365|M|D|College|1500|Unknown|0|0|0| +4366|F|D|College|1500|Unknown|0|0|0| +4367|M|W|College|1500|Unknown|0|0|0| +4368|F|W|College|1500|Unknown|0|0|0| +4369|M|U|College|1500|Unknown|0|0|0| +4370|F|U|College|1500|Unknown|0|0|0| +4371|M|M|2 yr Degree|1500|Unknown|0|0|0| +4372|F|M|2 yr Degree|1500|Unknown|0|0|0| +4373|M|S|2 yr Degree|1500|Unknown|0|0|0| +4374|F|S|2 yr Degree|1500|Unknown|0|0|0| +4375|M|D|2 yr Degree|1500|Unknown|0|0|0| +4376|F|D|2 yr Degree|1500|Unknown|0|0|0| +4377|M|W|2 yr Degree|1500|Unknown|0|0|0| +4378|F|W|2 yr Degree|1500|Unknown|0|0|0| +4379|M|U|2 yr Degree|1500|Unknown|0|0|0| +4380|F|U|2 yr Degree|1500|Unknown|0|0|0| +4381|M|M|4 yr Degree|1500|Unknown|0|0|0| +4382|F|M|4 yr Degree|1500|Unknown|0|0|0| +4383|M|S|4 yr Degree|1500|Unknown|0|0|0| +4384|F|S|4 yr Degree|1500|Unknown|0|0|0| +4385|M|D|4 yr Degree|1500|Unknown|0|0|0| +4386|F|D|4 yr Degree|1500|Unknown|0|0|0| +4387|M|W|4 yr Degree|1500|Unknown|0|0|0| +4388|F|W|4 yr Degree|1500|Unknown|0|0|0| +4389|M|U|4 yr Degree|1500|Unknown|0|0|0| +4390|F|U|4 yr Degree|1500|Unknown|0|0|0| +4391|M|M|Advanced Degree|1500|Unknown|0|0|0| +4392|F|M|Advanced Degree|1500|Unknown|0|0|0| +4393|M|S|Advanced Degree|1500|Unknown|0|0|0| +4394|F|S|Advanced Degree|1500|Unknown|0|0|0| +4395|M|D|Advanced Degree|1500|Unknown|0|0|0| +4396|F|D|Advanced Degree|1500|Unknown|0|0|0| +4397|M|W|Advanced Degree|1500|Unknown|0|0|0| +4398|F|W|Advanced Degree|1500|Unknown|0|0|0| +4399|M|U|Advanced Degree|1500|Unknown|0|0|0| +4400|F|U|Advanced Degree|1500|Unknown|0|0|0| +4401|M|M|Unknown|1500|Unknown|0|0|0| +4402|F|M|Unknown|1500|Unknown|0|0|0| +4403|M|S|Unknown|1500|Unknown|0|0|0| +4404|F|S|Unknown|1500|Unknown|0|0|0| +4405|M|D|Unknown|1500|Unknown|0|0|0| +4406|F|D|Unknown|1500|Unknown|0|0|0| +4407|M|W|Unknown|1500|Unknown|0|0|0| +4408|F|W|Unknown|1500|Unknown|0|0|0| +4409|M|U|Unknown|1500|Unknown|0|0|0| +4410|F|U|Unknown|1500|Unknown|0|0|0| +4411|M|M|Primary|2000|Unknown|0|0|0| +4412|F|M|Primary|2000|Unknown|0|0|0| +4413|M|S|Primary|2000|Unknown|0|0|0| +4414|F|S|Primary|2000|Unknown|0|0|0| +4415|M|D|Primary|2000|Unknown|0|0|0| +4416|F|D|Primary|2000|Unknown|0|0|0| +4417|M|W|Primary|2000|Unknown|0|0|0| +4418|F|W|Primary|2000|Unknown|0|0|0| +4419|M|U|Primary|2000|Unknown|0|0|0| +4420|F|U|Primary|2000|Unknown|0|0|0| +4421|M|M|Secondary|2000|Unknown|0|0|0| +4422|F|M|Secondary|2000|Unknown|0|0|0| +4423|M|S|Secondary|2000|Unknown|0|0|0| +4424|F|S|Secondary|2000|Unknown|0|0|0| +4425|M|D|Secondary|2000|Unknown|0|0|0| +4426|F|D|Secondary|2000|Unknown|0|0|0| +4427|M|W|Secondary|2000|Unknown|0|0|0| +4428|F|W|Secondary|2000|Unknown|0|0|0| +4429|M|U|Secondary|2000|Unknown|0|0|0| +4430|F|U|Secondary|2000|Unknown|0|0|0| +4431|M|M|College|2000|Unknown|0|0|0| +4432|F|M|College|2000|Unknown|0|0|0| +4433|M|S|College|2000|Unknown|0|0|0| +4434|F|S|College|2000|Unknown|0|0|0| +4435|M|D|College|2000|Unknown|0|0|0| +4436|F|D|College|2000|Unknown|0|0|0| +4437|M|W|College|2000|Unknown|0|0|0| +4438|F|W|College|2000|Unknown|0|0|0| +4439|M|U|College|2000|Unknown|0|0|0| +4440|F|U|College|2000|Unknown|0|0|0| +4441|M|M|2 yr Degree|2000|Unknown|0|0|0| +4442|F|M|2 yr Degree|2000|Unknown|0|0|0| +4443|M|S|2 yr Degree|2000|Unknown|0|0|0| +4444|F|S|2 yr Degree|2000|Unknown|0|0|0| +4445|M|D|2 yr Degree|2000|Unknown|0|0|0| +4446|F|D|2 yr Degree|2000|Unknown|0|0|0| +4447|M|W|2 yr Degree|2000|Unknown|0|0|0| +4448|F|W|2 yr Degree|2000|Unknown|0|0|0| +4449|M|U|2 yr Degree|2000|Unknown|0|0|0| +4450|F|U|2 yr Degree|2000|Unknown|0|0|0| +4451|M|M|4 yr Degree|2000|Unknown|0|0|0| +4452|F|M|4 yr Degree|2000|Unknown|0|0|0| +4453|M|S|4 yr Degree|2000|Unknown|0|0|0| +4454|F|S|4 yr Degree|2000|Unknown|0|0|0| +4455|M|D|4 yr Degree|2000|Unknown|0|0|0| +4456|F|D|4 yr Degree|2000|Unknown|0|0|0| +4457|M|W|4 yr Degree|2000|Unknown|0|0|0| +4458|F|W|4 yr Degree|2000|Unknown|0|0|0| +4459|M|U|4 yr Degree|2000|Unknown|0|0|0| +4460|F|U|4 yr Degree|2000|Unknown|0|0|0| +4461|M|M|Advanced Degree|2000|Unknown|0|0|0| +4462|F|M|Advanced Degree|2000|Unknown|0|0|0| +4463|M|S|Advanced Degree|2000|Unknown|0|0|0| +4464|F|S|Advanced Degree|2000|Unknown|0|0|0| +4465|M|D|Advanced Degree|2000|Unknown|0|0|0| +4466|F|D|Advanced Degree|2000|Unknown|0|0|0| +4467|M|W|Advanced Degree|2000|Unknown|0|0|0| +4468|F|W|Advanced Degree|2000|Unknown|0|0|0| +4469|M|U|Advanced Degree|2000|Unknown|0|0|0| +4470|F|U|Advanced Degree|2000|Unknown|0|0|0| +4471|M|M|Unknown|2000|Unknown|0|0|0| +4472|F|M|Unknown|2000|Unknown|0|0|0| +4473|M|S|Unknown|2000|Unknown|0|0|0| +4474|F|S|Unknown|2000|Unknown|0|0|0| +4475|M|D|Unknown|2000|Unknown|0|0|0| +4476|F|D|Unknown|2000|Unknown|0|0|0| +4477|M|W|Unknown|2000|Unknown|0|0|0| +4478|F|W|Unknown|2000|Unknown|0|0|0| +4479|M|U|Unknown|2000|Unknown|0|0|0| +4480|F|U|Unknown|2000|Unknown|0|0|0| +4481|M|M|Primary|2500|Unknown|0|0|0| +4482|F|M|Primary|2500|Unknown|0|0|0| +4483|M|S|Primary|2500|Unknown|0|0|0| +4484|F|S|Primary|2500|Unknown|0|0|0| +4485|M|D|Primary|2500|Unknown|0|0|0| +4486|F|D|Primary|2500|Unknown|0|0|0| +4487|M|W|Primary|2500|Unknown|0|0|0| +4488|F|W|Primary|2500|Unknown|0|0|0| +4489|M|U|Primary|2500|Unknown|0|0|0| +4490|F|U|Primary|2500|Unknown|0|0|0| +4491|M|M|Secondary|2500|Unknown|0|0|0| +4492|F|M|Secondary|2500|Unknown|0|0|0| +4493|M|S|Secondary|2500|Unknown|0|0|0| +4494|F|S|Secondary|2500|Unknown|0|0|0| +4495|M|D|Secondary|2500|Unknown|0|0|0| +4496|F|D|Secondary|2500|Unknown|0|0|0| +4497|M|W|Secondary|2500|Unknown|0|0|0| +4498|F|W|Secondary|2500|Unknown|0|0|0| +4499|M|U|Secondary|2500|Unknown|0|0|0| +4500|F|U|Secondary|2500|Unknown|0|0|0| +4501|M|M|College|2500|Unknown|0|0|0| +4502|F|M|College|2500|Unknown|0|0|0| +4503|M|S|College|2500|Unknown|0|0|0| +4504|F|S|College|2500|Unknown|0|0|0| +4505|M|D|College|2500|Unknown|0|0|0| +4506|F|D|College|2500|Unknown|0|0|0| +4507|M|W|College|2500|Unknown|0|0|0| +4508|F|W|College|2500|Unknown|0|0|0| +4509|M|U|College|2500|Unknown|0|0|0| +4510|F|U|College|2500|Unknown|0|0|0| +4511|M|M|2 yr Degree|2500|Unknown|0|0|0| +4512|F|M|2 yr Degree|2500|Unknown|0|0|0| +4513|M|S|2 yr Degree|2500|Unknown|0|0|0| +4514|F|S|2 yr Degree|2500|Unknown|0|0|0| +4515|M|D|2 yr Degree|2500|Unknown|0|0|0| +4516|F|D|2 yr Degree|2500|Unknown|0|0|0| +4517|M|W|2 yr Degree|2500|Unknown|0|0|0| +4518|F|W|2 yr Degree|2500|Unknown|0|0|0| +4519|M|U|2 yr Degree|2500|Unknown|0|0|0| +4520|F|U|2 yr Degree|2500|Unknown|0|0|0| +4521|M|M|4 yr Degree|2500|Unknown|0|0|0| +4522|F|M|4 yr Degree|2500|Unknown|0|0|0| +4523|M|S|4 yr Degree|2500|Unknown|0|0|0| +4524|F|S|4 yr Degree|2500|Unknown|0|0|0| +4525|M|D|4 yr Degree|2500|Unknown|0|0|0| +4526|F|D|4 yr Degree|2500|Unknown|0|0|0| +4527|M|W|4 yr Degree|2500|Unknown|0|0|0| +4528|F|W|4 yr Degree|2500|Unknown|0|0|0| +4529|M|U|4 yr Degree|2500|Unknown|0|0|0| +4530|F|U|4 yr Degree|2500|Unknown|0|0|0| +4531|M|M|Advanced Degree|2500|Unknown|0|0|0| +4532|F|M|Advanced Degree|2500|Unknown|0|0|0| +4533|M|S|Advanced Degree|2500|Unknown|0|0|0| +4534|F|S|Advanced Degree|2500|Unknown|0|0|0| +4535|M|D|Advanced Degree|2500|Unknown|0|0|0| +4536|F|D|Advanced Degree|2500|Unknown|0|0|0| +4537|M|W|Advanced Degree|2500|Unknown|0|0|0| +4538|F|W|Advanced Degree|2500|Unknown|0|0|0| +4539|M|U|Advanced Degree|2500|Unknown|0|0|0| +4540|F|U|Advanced Degree|2500|Unknown|0|0|0| +4541|M|M|Unknown|2500|Unknown|0|0|0| +4542|F|M|Unknown|2500|Unknown|0|0|0| +4543|M|S|Unknown|2500|Unknown|0|0|0| +4544|F|S|Unknown|2500|Unknown|0|0|0| +4545|M|D|Unknown|2500|Unknown|0|0|0| +4546|F|D|Unknown|2500|Unknown|0|0|0| +4547|M|W|Unknown|2500|Unknown|0|0|0| +4548|F|W|Unknown|2500|Unknown|0|0|0| +4549|M|U|Unknown|2500|Unknown|0|0|0| +4550|F|U|Unknown|2500|Unknown|0|0|0| +4551|M|M|Primary|3000|Unknown|0|0|0| +4552|F|M|Primary|3000|Unknown|0|0|0| +4553|M|S|Primary|3000|Unknown|0|0|0| +4554|F|S|Primary|3000|Unknown|0|0|0| +4555|M|D|Primary|3000|Unknown|0|0|0| +4556|F|D|Primary|3000|Unknown|0|0|0| +4557|M|W|Primary|3000|Unknown|0|0|0| +4558|F|W|Primary|3000|Unknown|0|0|0| +4559|M|U|Primary|3000|Unknown|0|0|0| +4560|F|U|Primary|3000|Unknown|0|0|0| +4561|M|M|Secondary|3000|Unknown|0|0|0| +4562|F|M|Secondary|3000|Unknown|0|0|0| +4563|M|S|Secondary|3000|Unknown|0|0|0| +4564|F|S|Secondary|3000|Unknown|0|0|0| +4565|M|D|Secondary|3000|Unknown|0|0|0| +4566|F|D|Secondary|3000|Unknown|0|0|0| +4567|M|W|Secondary|3000|Unknown|0|0|0| +4568|F|W|Secondary|3000|Unknown|0|0|0| +4569|M|U|Secondary|3000|Unknown|0|0|0| +4570|F|U|Secondary|3000|Unknown|0|0|0| +4571|M|M|College|3000|Unknown|0|0|0| +4572|F|M|College|3000|Unknown|0|0|0| +4573|M|S|College|3000|Unknown|0|0|0| +4574|F|S|College|3000|Unknown|0|0|0| +4575|M|D|College|3000|Unknown|0|0|0| +4576|F|D|College|3000|Unknown|0|0|0| +4577|M|W|College|3000|Unknown|0|0|0| +4578|F|W|College|3000|Unknown|0|0|0| +4579|M|U|College|3000|Unknown|0|0|0| +4580|F|U|College|3000|Unknown|0|0|0| +4581|M|M|2 yr Degree|3000|Unknown|0|0|0| +4582|F|M|2 yr Degree|3000|Unknown|0|0|0| +4583|M|S|2 yr Degree|3000|Unknown|0|0|0| +4584|F|S|2 yr Degree|3000|Unknown|0|0|0| +4585|M|D|2 yr Degree|3000|Unknown|0|0|0| +4586|F|D|2 yr Degree|3000|Unknown|0|0|0| +4587|M|W|2 yr Degree|3000|Unknown|0|0|0| +4588|F|W|2 yr Degree|3000|Unknown|0|0|0| +4589|M|U|2 yr Degree|3000|Unknown|0|0|0| +4590|F|U|2 yr Degree|3000|Unknown|0|0|0| +4591|M|M|4 yr Degree|3000|Unknown|0|0|0| +4592|F|M|4 yr Degree|3000|Unknown|0|0|0| +4593|M|S|4 yr Degree|3000|Unknown|0|0|0| +4594|F|S|4 yr Degree|3000|Unknown|0|0|0| +4595|M|D|4 yr Degree|3000|Unknown|0|0|0| +4596|F|D|4 yr Degree|3000|Unknown|0|0|0| +4597|M|W|4 yr Degree|3000|Unknown|0|0|0| +4598|F|W|4 yr Degree|3000|Unknown|0|0|0| +4599|M|U|4 yr Degree|3000|Unknown|0|0|0| +4600|F|U|4 yr Degree|3000|Unknown|0|0|0| +4601|M|M|Advanced Degree|3000|Unknown|0|0|0| +4602|F|M|Advanced Degree|3000|Unknown|0|0|0| +4603|M|S|Advanced Degree|3000|Unknown|0|0|0| +4604|F|S|Advanced Degree|3000|Unknown|0|0|0| +4605|M|D|Advanced Degree|3000|Unknown|0|0|0| +4606|F|D|Advanced Degree|3000|Unknown|0|0|0| +4607|M|W|Advanced Degree|3000|Unknown|0|0|0| +4608|F|W|Advanced Degree|3000|Unknown|0|0|0| +4609|M|U|Advanced Degree|3000|Unknown|0|0|0| +4610|F|U|Advanced Degree|3000|Unknown|0|0|0| +4611|M|M|Unknown|3000|Unknown|0|0|0| +4612|F|M|Unknown|3000|Unknown|0|0|0| +4613|M|S|Unknown|3000|Unknown|0|0|0| +4614|F|S|Unknown|3000|Unknown|0|0|0| +4615|M|D|Unknown|3000|Unknown|0|0|0| +4616|F|D|Unknown|3000|Unknown|0|0|0| +4617|M|W|Unknown|3000|Unknown|0|0|0| +4618|F|W|Unknown|3000|Unknown|0|0|0| +4619|M|U|Unknown|3000|Unknown|0|0|0| +4620|F|U|Unknown|3000|Unknown|0|0|0| +4621|M|M|Primary|3500|Unknown|0|0|0| +4622|F|M|Primary|3500|Unknown|0|0|0| +4623|M|S|Primary|3500|Unknown|0|0|0| +4624|F|S|Primary|3500|Unknown|0|0|0| +4625|M|D|Primary|3500|Unknown|0|0|0| +4626|F|D|Primary|3500|Unknown|0|0|0| +4627|M|W|Primary|3500|Unknown|0|0|0| +4628|F|W|Primary|3500|Unknown|0|0|0| +4629|M|U|Primary|3500|Unknown|0|0|0| +4630|F|U|Primary|3500|Unknown|0|0|0| +4631|M|M|Secondary|3500|Unknown|0|0|0| +4632|F|M|Secondary|3500|Unknown|0|0|0| +4633|M|S|Secondary|3500|Unknown|0|0|0| +4634|F|S|Secondary|3500|Unknown|0|0|0| +4635|M|D|Secondary|3500|Unknown|0|0|0| +4636|F|D|Secondary|3500|Unknown|0|0|0| +4637|M|W|Secondary|3500|Unknown|0|0|0| +4638|F|W|Secondary|3500|Unknown|0|0|0| +4639|M|U|Secondary|3500|Unknown|0|0|0| +4640|F|U|Secondary|3500|Unknown|0|0|0| +4641|M|M|College|3500|Unknown|0|0|0| +4642|F|M|College|3500|Unknown|0|0|0| +4643|M|S|College|3500|Unknown|0|0|0| +4644|F|S|College|3500|Unknown|0|0|0| +4645|M|D|College|3500|Unknown|0|0|0| +4646|F|D|College|3500|Unknown|0|0|0| +4647|M|W|College|3500|Unknown|0|0|0| +4648|F|W|College|3500|Unknown|0|0|0| +4649|M|U|College|3500|Unknown|0|0|0| +4650|F|U|College|3500|Unknown|0|0|0| +4651|M|M|2 yr Degree|3500|Unknown|0|0|0| +4652|F|M|2 yr Degree|3500|Unknown|0|0|0| +4653|M|S|2 yr Degree|3500|Unknown|0|0|0| +4654|F|S|2 yr Degree|3500|Unknown|0|0|0| +4655|M|D|2 yr Degree|3500|Unknown|0|0|0| +4656|F|D|2 yr Degree|3500|Unknown|0|0|0| +4657|M|W|2 yr Degree|3500|Unknown|0|0|0| +4658|F|W|2 yr Degree|3500|Unknown|0|0|0| +4659|M|U|2 yr Degree|3500|Unknown|0|0|0| +4660|F|U|2 yr Degree|3500|Unknown|0|0|0| +4661|M|M|4 yr Degree|3500|Unknown|0|0|0| +4662|F|M|4 yr Degree|3500|Unknown|0|0|0| +4663|M|S|4 yr Degree|3500|Unknown|0|0|0| +4664|F|S|4 yr Degree|3500|Unknown|0|0|0| +4665|M|D|4 yr Degree|3500|Unknown|0|0|0| +4666|F|D|4 yr Degree|3500|Unknown|0|0|0| +4667|M|W|4 yr Degree|3500|Unknown|0|0|0| +4668|F|W|4 yr Degree|3500|Unknown|0|0|0| +4669|M|U|4 yr Degree|3500|Unknown|0|0|0| +4670|F|U|4 yr Degree|3500|Unknown|0|0|0| +4671|M|M|Advanced Degree|3500|Unknown|0|0|0| +4672|F|M|Advanced Degree|3500|Unknown|0|0|0| +4673|M|S|Advanced Degree|3500|Unknown|0|0|0| +4674|F|S|Advanced Degree|3500|Unknown|0|0|0| +4675|M|D|Advanced Degree|3500|Unknown|0|0|0| +4676|F|D|Advanced Degree|3500|Unknown|0|0|0| +4677|M|W|Advanced Degree|3500|Unknown|0|0|0| +4678|F|W|Advanced Degree|3500|Unknown|0|0|0| +4679|M|U|Advanced Degree|3500|Unknown|0|0|0| +4680|F|U|Advanced Degree|3500|Unknown|0|0|0| +4681|M|M|Unknown|3500|Unknown|0|0|0| +4682|F|M|Unknown|3500|Unknown|0|0|0| +4683|M|S|Unknown|3500|Unknown|0|0|0| +4684|F|S|Unknown|3500|Unknown|0|0|0| +4685|M|D|Unknown|3500|Unknown|0|0|0| +4686|F|D|Unknown|3500|Unknown|0|0|0| +4687|M|W|Unknown|3500|Unknown|0|0|0| +4688|F|W|Unknown|3500|Unknown|0|0|0| +4689|M|U|Unknown|3500|Unknown|0|0|0| +4690|F|U|Unknown|3500|Unknown|0|0|0| +4691|M|M|Primary|4000|Unknown|0|0|0| +4692|F|M|Primary|4000|Unknown|0|0|0| +4693|M|S|Primary|4000|Unknown|0|0|0| +4694|F|S|Primary|4000|Unknown|0|0|0| +4695|M|D|Primary|4000|Unknown|0|0|0| +4696|F|D|Primary|4000|Unknown|0|0|0| +4697|M|W|Primary|4000|Unknown|0|0|0| +4698|F|W|Primary|4000|Unknown|0|0|0| +4699|M|U|Primary|4000|Unknown|0|0|0| +4700|F|U|Primary|4000|Unknown|0|0|0| +4701|M|M|Secondary|4000|Unknown|0|0|0| +4702|F|M|Secondary|4000|Unknown|0|0|0| +4703|M|S|Secondary|4000|Unknown|0|0|0| +4704|F|S|Secondary|4000|Unknown|0|0|0| +4705|M|D|Secondary|4000|Unknown|0|0|0| +4706|F|D|Secondary|4000|Unknown|0|0|0| +4707|M|W|Secondary|4000|Unknown|0|0|0| +4708|F|W|Secondary|4000|Unknown|0|0|0| +4709|M|U|Secondary|4000|Unknown|0|0|0| +4710|F|U|Secondary|4000|Unknown|0|0|0| +4711|M|M|College|4000|Unknown|0|0|0| +4712|F|M|College|4000|Unknown|0|0|0| +4713|M|S|College|4000|Unknown|0|0|0| +4714|F|S|College|4000|Unknown|0|0|0| +4715|M|D|College|4000|Unknown|0|0|0| +4716|F|D|College|4000|Unknown|0|0|0| +4717|M|W|College|4000|Unknown|0|0|0| +4718|F|W|College|4000|Unknown|0|0|0| +4719|M|U|College|4000|Unknown|0|0|0| +4720|F|U|College|4000|Unknown|0|0|0| +4721|M|M|2 yr Degree|4000|Unknown|0|0|0| +4722|F|M|2 yr Degree|4000|Unknown|0|0|0| +4723|M|S|2 yr Degree|4000|Unknown|0|0|0| +4724|F|S|2 yr Degree|4000|Unknown|0|0|0| +4725|M|D|2 yr Degree|4000|Unknown|0|0|0| +4726|F|D|2 yr Degree|4000|Unknown|0|0|0| +4727|M|W|2 yr Degree|4000|Unknown|0|0|0| +4728|F|W|2 yr Degree|4000|Unknown|0|0|0| +4729|M|U|2 yr Degree|4000|Unknown|0|0|0| +4730|F|U|2 yr Degree|4000|Unknown|0|0|0| +4731|M|M|4 yr Degree|4000|Unknown|0|0|0| +4732|F|M|4 yr Degree|4000|Unknown|0|0|0| +4733|M|S|4 yr Degree|4000|Unknown|0|0|0| +4734|F|S|4 yr Degree|4000|Unknown|0|0|0| +4735|M|D|4 yr Degree|4000|Unknown|0|0|0| +4736|F|D|4 yr Degree|4000|Unknown|0|0|0| +4737|M|W|4 yr Degree|4000|Unknown|0|0|0| +4738|F|W|4 yr Degree|4000|Unknown|0|0|0| +4739|M|U|4 yr Degree|4000|Unknown|0|0|0| +4740|F|U|4 yr Degree|4000|Unknown|0|0|0| +4741|M|M|Advanced Degree|4000|Unknown|0|0|0| +4742|F|M|Advanced Degree|4000|Unknown|0|0|0| +4743|M|S|Advanced Degree|4000|Unknown|0|0|0| +4744|F|S|Advanced Degree|4000|Unknown|0|0|0| +4745|M|D|Advanced Degree|4000|Unknown|0|0|0| +4746|F|D|Advanced Degree|4000|Unknown|0|0|0| +4747|M|W|Advanced Degree|4000|Unknown|0|0|0| +4748|F|W|Advanced Degree|4000|Unknown|0|0|0| +4749|M|U|Advanced Degree|4000|Unknown|0|0|0| +4750|F|U|Advanced Degree|4000|Unknown|0|0|0| +4751|M|M|Unknown|4000|Unknown|0|0|0| +4752|F|M|Unknown|4000|Unknown|0|0|0| +4753|M|S|Unknown|4000|Unknown|0|0|0| +4754|F|S|Unknown|4000|Unknown|0|0|0| +4755|M|D|Unknown|4000|Unknown|0|0|0| +4756|F|D|Unknown|4000|Unknown|0|0|0| +4757|M|W|Unknown|4000|Unknown|0|0|0| +4758|F|W|Unknown|4000|Unknown|0|0|0| +4759|M|U|Unknown|4000|Unknown|0|0|0| +4760|F|U|Unknown|4000|Unknown|0|0|0| +4761|M|M|Primary|4500|Unknown|0|0|0| +4762|F|M|Primary|4500|Unknown|0|0|0| +4763|M|S|Primary|4500|Unknown|0|0|0| +4764|F|S|Primary|4500|Unknown|0|0|0| +4765|M|D|Primary|4500|Unknown|0|0|0| +4766|F|D|Primary|4500|Unknown|0|0|0| +4767|M|W|Primary|4500|Unknown|0|0|0| +4768|F|W|Primary|4500|Unknown|0|0|0| +4769|M|U|Primary|4500|Unknown|0|0|0| +4770|F|U|Primary|4500|Unknown|0|0|0| +4771|M|M|Secondary|4500|Unknown|0|0|0| +4772|F|M|Secondary|4500|Unknown|0|0|0| +4773|M|S|Secondary|4500|Unknown|0|0|0| +4774|F|S|Secondary|4500|Unknown|0|0|0| +4775|M|D|Secondary|4500|Unknown|0|0|0| +4776|F|D|Secondary|4500|Unknown|0|0|0| +4777|M|W|Secondary|4500|Unknown|0|0|0| +4778|F|W|Secondary|4500|Unknown|0|0|0| +4779|M|U|Secondary|4500|Unknown|0|0|0| +4780|F|U|Secondary|4500|Unknown|0|0|0| +4781|M|M|College|4500|Unknown|0|0|0| +4782|F|M|College|4500|Unknown|0|0|0| +4783|M|S|College|4500|Unknown|0|0|0| +4784|F|S|College|4500|Unknown|0|0|0| +4785|M|D|College|4500|Unknown|0|0|0| +4786|F|D|College|4500|Unknown|0|0|0| +4787|M|W|College|4500|Unknown|0|0|0| +4788|F|W|College|4500|Unknown|0|0|0| +4789|M|U|College|4500|Unknown|0|0|0| +4790|F|U|College|4500|Unknown|0|0|0| +4791|M|M|2 yr Degree|4500|Unknown|0|0|0| +4792|F|M|2 yr Degree|4500|Unknown|0|0|0| +4793|M|S|2 yr Degree|4500|Unknown|0|0|0| +4794|F|S|2 yr Degree|4500|Unknown|0|0|0| +4795|M|D|2 yr Degree|4500|Unknown|0|0|0| +4796|F|D|2 yr Degree|4500|Unknown|0|0|0| +4797|M|W|2 yr Degree|4500|Unknown|0|0|0| +4798|F|W|2 yr Degree|4500|Unknown|0|0|0| +4799|M|U|2 yr Degree|4500|Unknown|0|0|0| +4800|F|U|2 yr Degree|4500|Unknown|0|0|0| +4801|M|M|4 yr Degree|4500|Unknown|0|0|0| +4802|F|M|4 yr Degree|4500|Unknown|0|0|0| +4803|M|S|4 yr Degree|4500|Unknown|0|0|0| +4804|F|S|4 yr Degree|4500|Unknown|0|0|0| +4805|M|D|4 yr Degree|4500|Unknown|0|0|0| +4806|F|D|4 yr Degree|4500|Unknown|0|0|0| +4807|M|W|4 yr Degree|4500|Unknown|0|0|0| +4808|F|W|4 yr Degree|4500|Unknown|0|0|0| +4809|M|U|4 yr Degree|4500|Unknown|0|0|0| +4810|F|U|4 yr Degree|4500|Unknown|0|0|0| +4811|M|M|Advanced Degree|4500|Unknown|0|0|0| +4812|F|M|Advanced Degree|4500|Unknown|0|0|0| +4813|M|S|Advanced Degree|4500|Unknown|0|0|0| +4814|F|S|Advanced Degree|4500|Unknown|0|0|0| +4815|M|D|Advanced Degree|4500|Unknown|0|0|0| +4816|F|D|Advanced Degree|4500|Unknown|0|0|0| +4817|M|W|Advanced Degree|4500|Unknown|0|0|0| +4818|F|W|Advanced Degree|4500|Unknown|0|0|0| +4819|M|U|Advanced Degree|4500|Unknown|0|0|0| +4820|F|U|Advanced Degree|4500|Unknown|0|0|0| +4821|M|M|Unknown|4500|Unknown|0|0|0| +4822|F|M|Unknown|4500|Unknown|0|0|0| +4823|M|S|Unknown|4500|Unknown|0|0|0| +4824|F|S|Unknown|4500|Unknown|0|0|0| +4825|M|D|Unknown|4500|Unknown|0|0|0| +4826|F|D|Unknown|4500|Unknown|0|0|0| +4827|M|W|Unknown|4500|Unknown|0|0|0| +4828|F|W|Unknown|4500|Unknown|0|0|0| +4829|M|U|Unknown|4500|Unknown|0|0|0| +4830|F|U|Unknown|4500|Unknown|0|0|0| +4831|M|M|Primary|5000|Unknown|0|0|0| +4832|F|M|Primary|5000|Unknown|0|0|0| +4833|M|S|Primary|5000|Unknown|0|0|0| +4834|F|S|Primary|5000|Unknown|0|0|0| +4835|M|D|Primary|5000|Unknown|0|0|0| +4836|F|D|Primary|5000|Unknown|0|0|0| +4837|M|W|Primary|5000|Unknown|0|0|0| +4838|F|W|Primary|5000|Unknown|0|0|0| +4839|M|U|Primary|5000|Unknown|0|0|0| +4840|F|U|Primary|5000|Unknown|0|0|0| +4841|M|M|Secondary|5000|Unknown|0|0|0| +4842|F|M|Secondary|5000|Unknown|0|0|0| +4843|M|S|Secondary|5000|Unknown|0|0|0| +4844|F|S|Secondary|5000|Unknown|0|0|0| +4845|M|D|Secondary|5000|Unknown|0|0|0| +4846|F|D|Secondary|5000|Unknown|0|0|0| +4847|M|W|Secondary|5000|Unknown|0|0|0| +4848|F|W|Secondary|5000|Unknown|0|0|0| +4849|M|U|Secondary|5000|Unknown|0|0|0| +4850|F|U|Secondary|5000|Unknown|0|0|0| +4851|M|M|College|5000|Unknown|0|0|0| +4852|F|M|College|5000|Unknown|0|0|0| +4853|M|S|College|5000|Unknown|0|0|0| +4854|F|S|College|5000|Unknown|0|0|0| +4855|M|D|College|5000|Unknown|0|0|0| +4856|F|D|College|5000|Unknown|0|0|0| +4857|M|W|College|5000|Unknown|0|0|0| +4858|F|W|College|5000|Unknown|0|0|0| +4859|M|U|College|5000|Unknown|0|0|0| +4860|F|U|College|5000|Unknown|0|0|0| +4861|M|M|2 yr Degree|5000|Unknown|0|0|0| +4862|F|M|2 yr Degree|5000|Unknown|0|0|0| +4863|M|S|2 yr Degree|5000|Unknown|0|0|0| +4864|F|S|2 yr Degree|5000|Unknown|0|0|0| +4865|M|D|2 yr Degree|5000|Unknown|0|0|0| +4866|F|D|2 yr Degree|5000|Unknown|0|0|0| +4867|M|W|2 yr Degree|5000|Unknown|0|0|0| +4868|F|W|2 yr Degree|5000|Unknown|0|0|0| +4869|M|U|2 yr Degree|5000|Unknown|0|0|0| +4870|F|U|2 yr Degree|5000|Unknown|0|0|0| +4871|M|M|4 yr Degree|5000|Unknown|0|0|0| +4872|F|M|4 yr Degree|5000|Unknown|0|0|0| +4873|M|S|4 yr Degree|5000|Unknown|0|0|0| +4874|F|S|4 yr Degree|5000|Unknown|0|0|0| +4875|M|D|4 yr Degree|5000|Unknown|0|0|0| +4876|F|D|4 yr Degree|5000|Unknown|0|0|0| +4877|M|W|4 yr Degree|5000|Unknown|0|0|0| +4878|F|W|4 yr Degree|5000|Unknown|0|0|0| +4879|M|U|4 yr Degree|5000|Unknown|0|0|0| +4880|F|U|4 yr Degree|5000|Unknown|0|0|0| +4881|M|M|Advanced Degree|5000|Unknown|0|0|0| +4882|F|M|Advanced Degree|5000|Unknown|0|0|0| +4883|M|S|Advanced Degree|5000|Unknown|0|0|0| +4884|F|S|Advanced Degree|5000|Unknown|0|0|0| +4885|M|D|Advanced Degree|5000|Unknown|0|0|0| +4886|F|D|Advanced Degree|5000|Unknown|0|0|0| +4887|M|W|Advanced Degree|5000|Unknown|0|0|0| +4888|F|W|Advanced Degree|5000|Unknown|0|0|0| +4889|M|U|Advanced Degree|5000|Unknown|0|0|0| +4890|F|U|Advanced Degree|5000|Unknown|0|0|0| +4891|M|M|Unknown|5000|Unknown|0|0|0| +4892|F|M|Unknown|5000|Unknown|0|0|0| +4893|M|S|Unknown|5000|Unknown|0|0|0| +4894|F|S|Unknown|5000|Unknown|0|0|0| +4895|M|D|Unknown|5000|Unknown|0|0|0| +4896|F|D|Unknown|5000|Unknown|0|0|0| +4897|M|W|Unknown|5000|Unknown|0|0|0| +4898|F|W|Unknown|5000|Unknown|0|0|0| +4899|M|U|Unknown|5000|Unknown|0|0|0| +4900|F|U|Unknown|5000|Unknown|0|0|0| +4901|M|M|Primary|5500|Unknown|0|0|0| +4902|F|M|Primary|5500|Unknown|0|0|0| +4903|M|S|Primary|5500|Unknown|0|0|0| +4904|F|S|Primary|5500|Unknown|0|0|0| +4905|M|D|Primary|5500|Unknown|0|0|0| +4906|F|D|Primary|5500|Unknown|0|0|0| +4907|M|W|Primary|5500|Unknown|0|0|0| +4908|F|W|Primary|5500|Unknown|0|0|0| +4909|M|U|Primary|5500|Unknown|0|0|0| +4910|F|U|Primary|5500|Unknown|0|0|0| +4911|M|M|Secondary|5500|Unknown|0|0|0| +4912|F|M|Secondary|5500|Unknown|0|0|0| +4913|M|S|Secondary|5500|Unknown|0|0|0| +4914|F|S|Secondary|5500|Unknown|0|0|0| +4915|M|D|Secondary|5500|Unknown|0|0|0| +4916|F|D|Secondary|5500|Unknown|0|0|0| +4917|M|W|Secondary|5500|Unknown|0|0|0| +4918|F|W|Secondary|5500|Unknown|0|0|0| +4919|M|U|Secondary|5500|Unknown|0|0|0| +4920|F|U|Secondary|5500|Unknown|0|0|0| +4921|M|M|College|5500|Unknown|0|0|0| +4922|F|M|College|5500|Unknown|0|0|0| +4923|M|S|College|5500|Unknown|0|0|0| +4924|F|S|College|5500|Unknown|0|0|0| +4925|M|D|College|5500|Unknown|0|0|0| +4926|F|D|College|5500|Unknown|0|0|0| +4927|M|W|College|5500|Unknown|0|0|0| +4928|F|W|College|5500|Unknown|0|0|0| +4929|M|U|College|5500|Unknown|0|0|0| +4930|F|U|College|5500|Unknown|0|0|0| +4931|M|M|2 yr Degree|5500|Unknown|0|0|0| +4932|F|M|2 yr Degree|5500|Unknown|0|0|0| +4933|M|S|2 yr Degree|5500|Unknown|0|0|0| +4934|F|S|2 yr Degree|5500|Unknown|0|0|0| +4935|M|D|2 yr Degree|5500|Unknown|0|0|0| +4936|F|D|2 yr Degree|5500|Unknown|0|0|0| +4937|M|W|2 yr Degree|5500|Unknown|0|0|0| +4938|F|W|2 yr Degree|5500|Unknown|0|0|0| +4939|M|U|2 yr Degree|5500|Unknown|0|0|0| +4940|F|U|2 yr Degree|5500|Unknown|0|0|0| +4941|M|M|4 yr Degree|5500|Unknown|0|0|0| +4942|F|M|4 yr Degree|5500|Unknown|0|0|0| +4943|M|S|4 yr Degree|5500|Unknown|0|0|0| +4944|F|S|4 yr Degree|5500|Unknown|0|0|0| +4945|M|D|4 yr Degree|5500|Unknown|0|0|0| +4946|F|D|4 yr Degree|5500|Unknown|0|0|0| +4947|M|W|4 yr Degree|5500|Unknown|0|0|0| +4948|F|W|4 yr Degree|5500|Unknown|0|0|0| +4949|M|U|4 yr Degree|5500|Unknown|0|0|0| +4950|F|U|4 yr Degree|5500|Unknown|0|0|0| +4951|M|M|Advanced Degree|5500|Unknown|0|0|0| +4952|F|M|Advanced Degree|5500|Unknown|0|0|0| +4953|M|S|Advanced Degree|5500|Unknown|0|0|0| +4954|F|S|Advanced Degree|5500|Unknown|0|0|0| +4955|M|D|Advanced Degree|5500|Unknown|0|0|0| +4956|F|D|Advanced Degree|5500|Unknown|0|0|0| +4957|M|W|Advanced Degree|5500|Unknown|0|0|0| +4958|F|W|Advanced Degree|5500|Unknown|0|0|0| +4959|M|U|Advanced Degree|5500|Unknown|0|0|0| +4960|F|U|Advanced Degree|5500|Unknown|0|0|0| +4961|M|M|Unknown|5500|Unknown|0|0|0| +4962|F|M|Unknown|5500|Unknown|0|0|0| +4963|M|S|Unknown|5500|Unknown|0|0|0| +4964|F|S|Unknown|5500|Unknown|0|0|0| +4965|M|D|Unknown|5500|Unknown|0|0|0| +4966|F|D|Unknown|5500|Unknown|0|0|0| +4967|M|W|Unknown|5500|Unknown|0|0|0| +4968|F|W|Unknown|5500|Unknown|0|0|0| +4969|M|U|Unknown|5500|Unknown|0|0|0| +4970|F|U|Unknown|5500|Unknown|0|0|0| +4971|M|M|Primary|6000|Unknown|0|0|0| +4972|F|M|Primary|6000|Unknown|0|0|0| +4973|M|S|Primary|6000|Unknown|0|0|0| +4974|F|S|Primary|6000|Unknown|0|0|0| +4975|M|D|Primary|6000|Unknown|0|0|0| +4976|F|D|Primary|6000|Unknown|0|0|0| +4977|M|W|Primary|6000|Unknown|0|0|0| +4978|F|W|Primary|6000|Unknown|0|0|0| +4979|M|U|Primary|6000|Unknown|0|0|0| +4980|F|U|Primary|6000|Unknown|0|0|0| +4981|M|M|Secondary|6000|Unknown|0|0|0| +4982|F|M|Secondary|6000|Unknown|0|0|0| +4983|M|S|Secondary|6000|Unknown|0|0|0| +4984|F|S|Secondary|6000|Unknown|0|0|0| +4985|M|D|Secondary|6000|Unknown|0|0|0| +4986|F|D|Secondary|6000|Unknown|0|0|0| +4987|M|W|Secondary|6000|Unknown|0|0|0| +4988|F|W|Secondary|6000|Unknown|0|0|0| +4989|M|U|Secondary|6000|Unknown|0|0|0| +4990|F|U|Secondary|6000|Unknown|0|0|0| +4991|M|M|College|6000|Unknown|0|0|0| +4992|F|M|College|6000|Unknown|0|0|0| +4993|M|S|College|6000|Unknown|0|0|0| +4994|F|S|College|6000|Unknown|0|0|0| +4995|M|D|College|6000|Unknown|0|0|0| +4996|F|D|College|6000|Unknown|0|0|0| +4997|M|W|College|6000|Unknown|0|0|0| +4998|F|W|College|6000|Unknown|0|0|0| +4999|M|U|College|6000|Unknown|0|0|0| +5000|F|U|College|6000|Unknown|0|0|0| +5001|M|M|2 yr Degree|6000|Unknown|0|0|0| +5002|F|M|2 yr Degree|6000|Unknown|0|0|0| +5003|M|S|2 yr Degree|6000|Unknown|0|0|0| +5004|F|S|2 yr Degree|6000|Unknown|0|0|0| +5005|M|D|2 yr Degree|6000|Unknown|0|0|0| +5006|F|D|2 yr Degree|6000|Unknown|0|0|0| +5007|M|W|2 yr Degree|6000|Unknown|0|0|0| +5008|F|W|2 yr Degree|6000|Unknown|0|0|0| +5009|M|U|2 yr Degree|6000|Unknown|0|0|0| +5010|F|U|2 yr Degree|6000|Unknown|0|0|0| +5011|M|M|4 yr Degree|6000|Unknown|0|0|0| +5012|F|M|4 yr Degree|6000|Unknown|0|0|0| +5013|M|S|4 yr Degree|6000|Unknown|0|0|0| +5014|F|S|4 yr Degree|6000|Unknown|0|0|0| +5015|M|D|4 yr Degree|6000|Unknown|0|0|0| +5016|F|D|4 yr Degree|6000|Unknown|0|0|0| +5017|M|W|4 yr Degree|6000|Unknown|0|0|0| +5018|F|W|4 yr Degree|6000|Unknown|0|0|0| +5019|M|U|4 yr Degree|6000|Unknown|0|0|0| +5020|F|U|4 yr Degree|6000|Unknown|0|0|0| +5021|M|M|Advanced Degree|6000|Unknown|0|0|0| +5022|F|M|Advanced Degree|6000|Unknown|0|0|0| +5023|M|S|Advanced Degree|6000|Unknown|0|0|0| +5024|F|S|Advanced Degree|6000|Unknown|0|0|0| +5025|M|D|Advanced Degree|6000|Unknown|0|0|0| +5026|F|D|Advanced Degree|6000|Unknown|0|0|0| +5027|M|W|Advanced Degree|6000|Unknown|0|0|0| +5028|F|W|Advanced Degree|6000|Unknown|0|0|0| +5029|M|U|Advanced Degree|6000|Unknown|0|0|0| +5030|F|U|Advanced Degree|6000|Unknown|0|0|0| +5031|M|M|Unknown|6000|Unknown|0|0|0| +5032|F|M|Unknown|6000|Unknown|0|0|0| +5033|M|S|Unknown|6000|Unknown|0|0|0| +5034|F|S|Unknown|6000|Unknown|0|0|0| +5035|M|D|Unknown|6000|Unknown|0|0|0| +5036|F|D|Unknown|6000|Unknown|0|0|0| +5037|M|W|Unknown|6000|Unknown|0|0|0| +5038|F|W|Unknown|6000|Unknown|0|0|0| +5039|M|U|Unknown|6000|Unknown|0|0|0| +5040|F|U|Unknown|6000|Unknown|0|0|0| +5041|M|M|Primary|6500|Unknown|0|0|0| +5042|F|M|Primary|6500|Unknown|0|0|0| +5043|M|S|Primary|6500|Unknown|0|0|0| +5044|F|S|Primary|6500|Unknown|0|0|0| +5045|M|D|Primary|6500|Unknown|0|0|0| +5046|F|D|Primary|6500|Unknown|0|0|0| +5047|M|W|Primary|6500|Unknown|0|0|0| +5048|F|W|Primary|6500|Unknown|0|0|0| +5049|M|U|Primary|6500|Unknown|0|0|0| +5050|F|U|Primary|6500|Unknown|0|0|0| +5051|M|M|Secondary|6500|Unknown|0|0|0| +5052|F|M|Secondary|6500|Unknown|0|0|0| +5053|M|S|Secondary|6500|Unknown|0|0|0| +5054|F|S|Secondary|6500|Unknown|0|0|0| +5055|M|D|Secondary|6500|Unknown|0|0|0| +5056|F|D|Secondary|6500|Unknown|0|0|0| +5057|M|W|Secondary|6500|Unknown|0|0|0| +5058|F|W|Secondary|6500|Unknown|0|0|0| +5059|M|U|Secondary|6500|Unknown|0|0|0| +5060|F|U|Secondary|6500|Unknown|0|0|0| +5061|M|M|College|6500|Unknown|0|0|0| +5062|F|M|College|6500|Unknown|0|0|0| +5063|M|S|College|6500|Unknown|0|0|0| +5064|F|S|College|6500|Unknown|0|0|0| +5065|M|D|College|6500|Unknown|0|0|0| +5066|F|D|College|6500|Unknown|0|0|0| +5067|M|W|College|6500|Unknown|0|0|0| +5068|F|W|College|6500|Unknown|0|0|0| +5069|M|U|College|6500|Unknown|0|0|0| +5070|F|U|College|6500|Unknown|0|0|0| +5071|M|M|2 yr Degree|6500|Unknown|0|0|0| +5072|F|M|2 yr Degree|6500|Unknown|0|0|0| +5073|M|S|2 yr Degree|6500|Unknown|0|0|0| +5074|F|S|2 yr Degree|6500|Unknown|0|0|0| +5075|M|D|2 yr Degree|6500|Unknown|0|0|0| +5076|F|D|2 yr Degree|6500|Unknown|0|0|0| +5077|M|W|2 yr Degree|6500|Unknown|0|0|0| +5078|F|W|2 yr Degree|6500|Unknown|0|0|0| +5079|M|U|2 yr Degree|6500|Unknown|0|0|0| +5080|F|U|2 yr Degree|6500|Unknown|0|0|0| +5081|M|M|4 yr Degree|6500|Unknown|0|0|0| +5082|F|M|4 yr Degree|6500|Unknown|0|0|0| +5083|M|S|4 yr Degree|6500|Unknown|0|0|0| +5084|F|S|4 yr Degree|6500|Unknown|0|0|0| +5085|M|D|4 yr Degree|6500|Unknown|0|0|0| +5086|F|D|4 yr Degree|6500|Unknown|0|0|0| +5087|M|W|4 yr Degree|6500|Unknown|0|0|0| +5088|F|W|4 yr Degree|6500|Unknown|0|0|0| +5089|M|U|4 yr Degree|6500|Unknown|0|0|0| +5090|F|U|4 yr Degree|6500|Unknown|0|0|0| +5091|M|M|Advanced Degree|6500|Unknown|0|0|0| +5092|F|M|Advanced Degree|6500|Unknown|0|0|0| +5093|M|S|Advanced Degree|6500|Unknown|0|0|0| +5094|F|S|Advanced Degree|6500|Unknown|0|0|0| +5095|M|D|Advanced Degree|6500|Unknown|0|0|0| +5096|F|D|Advanced Degree|6500|Unknown|0|0|0| +5097|M|W|Advanced Degree|6500|Unknown|0|0|0| +5098|F|W|Advanced Degree|6500|Unknown|0|0|0| +5099|M|U|Advanced Degree|6500|Unknown|0|0|0| +5100|F|U|Advanced Degree|6500|Unknown|0|0|0| +5101|M|M|Unknown|6500|Unknown|0|0|0| +5102|F|M|Unknown|6500|Unknown|0|0|0| +5103|M|S|Unknown|6500|Unknown|0|0|0| +5104|F|S|Unknown|6500|Unknown|0|0|0| +5105|M|D|Unknown|6500|Unknown|0|0|0| +5106|F|D|Unknown|6500|Unknown|0|0|0| +5107|M|W|Unknown|6500|Unknown|0|0|0| +5108|F|W|Unknown|6500|Unknown|0|0|0| +5109|M|U|Unknown|6500|Unknown|0|0|0| +5110|F|U|Unknown|6500|Unknown|0|0|0| +5111|M|M|Primary|7000|Unknown|0|0|0| +5112|F|M|Primary|7000|Unknown|0|0|0| +5113|M|S|Primary|7000|Unknown|0|0|0| +5114|F|S|Primary|7000|Unknown|0|0|0| +5115|M|D|Primary|7000|Unknown|0|0|0| +5116|F|D|Primary|7000|Unknown|0|0|0| +5117|M|W|Primary|7000|Unknown|0|0|0| +5118|F|W|Primary|7000|Unknown|0|0|0| +5119|M|U|Primary|7000|Unknown|0|0|0| +5120|F|U|Primary|7000|Unknown|0|0|0| +5121|M|M|Secondary|7000|Unknown|0|0|0| +5122|F|M|Secondary|7000|Unknown|0|0|0| +5123|M|S|Secondary|7000|Unknown|0|0|0| +5124|F|S|Secondary|7000|Unknown|0|0|0| +5125|M|D|Secondary|7000|Unknown|0|0|0| +5126|F|D|Secondary|7000|Unknown|0|0|0| +5127|M|W|Secondary|7000|Unknown|0|0|0| +5128|F|W|Secondary|7000|Unknown|0|0|0| +5129|M|U|Secondary|7000|Unknown|0|0|0| +5130|F|U|Secondary|7000|Unknown|0|0|0| +5131|M|M|College|7000|Unknown|0|0|0| +5132|F|M|College|7000|Unknown|0|0|0| +5133|M|S|College|7000|Unknown|0|0|0| +5134|F|S|College|7000|Unknown|0|0|0| +5135|M|D|College|7000|Unknown|0|0|0| +5136|F|D|College|7000|Unknown|0|0|0| +5137|M|W|College|7000|Unknown|0|0|0| +5138|F|W|College|7000|Unknown|0|0|0| +5139|M|U|College|7000|Unknown|0|0|0| +5140|F|U|College|7000|Unknown|0|0|0| +5141|M|M|2 yr Degree|7000|Unknown|0|0|0| +5142|F|M|2 yr Degree|7000|Unknown|0|0|0| +5143|M|S|2 yr Degree|7000|Unknown|0|0|0| +5144|F|S|2 yr Degree|7000|Unknown|0|0|0| +5145|M|D|2 yr Degree|7000|Unknown|0|0|0| +5146|F|D|2 yr Degree|7000|Unknown|0|0|0| +5147|M|W|2 yr Degree|7000|Unknown|0|0|0| +5148|F|W|2 yr Degree|7000|Unknown|0|0|0| +5149|M|U|2 yr Degree|7000|Unknown|0|0|0| +5150|F|U|2 yr Degree|7000|Unknown|0|0|0| +5151|M|M|4 yr Degree|7000|Unknown|0|0|0| +5152|F|M|4 yr Degree|7000|Unknown|0|0|0| +5153|M|S|4 yr Degree|7000|Unknown|0|0|0| +5154|F|S|4 yr Degree|7000|Unknown|0|0|0| +5155|M|D|4 yr Degree|7000|Unknown|0|0|0| +5156|F|D|4 yr Degree|7000|Unknown|0|0|0| +5157|M|W|4 yr Degree|7000|Unknown|0|0|0| +5158|F|W|4 yr Degree|7000|Unknown|0|0|0| +5159|M|U|4 yr Degree|7000|Unknown|0|0|0| +5160|F|U|4 yr Degree|7000|Unknown|0|0|0| +5161|M|M|Advanced Degree|7000|Unknown|0|0|0| +5162|F|M|Advanced Degree|7000|Unknown|0|0|0| +5163|M|S|Advanced Degree|7000|Unknown|0|0|0| +5164|F|S|Advanced Degree|7000|Unknown|0|0|0| +5165|M|D|Advanced Degree|7000|Unknown|0|0|0| +5166|F|D|Advanced Degree|7000|Unknown|0|0|0| +5167|M|W|Advanced Degree|7000|Unknown|0|0|0| +5168|F|W|Advanced Degree|7000|Unknown|0|0|0| +5169|M|U|Advanced Degree|7000|Unknown|0|0|0| +5170|F|U|Advanced Degree|7000|Unknown|0|0|0| +5171|M|M|Unknown|7000|Unknown|0|0|0| +5172|F|M|Unknown|7000|Unknown|0|0|0| +5173|M|S|Unknown|7000|Unknown|0|0|0| +5174|F|S|Unknown|7000|Unknown|0|0|0| +5175|M|D|Unknown|7000|Unknown|0|0|0| +5176|F|D|Unknown|7000|Unknown|0|0|0| +5177|M|W|Unknown|7000|Unknown|0|0|0| +5178|F|W|Unknown|7000|Unknown|0|0|0| +5179|M|U|Unknown|7000|Unknown|0|0|0| +5180|F|U|Unknown|7000|Unknown|0|0|0| +5181|M|M|Primary|7500|Unknown|0|0|0| +5182|F|M|Primary|7500|Unknown|0|0|0| +5183|M|S|Primary|7500|Unknown|0|0|0| +5184|F|S|Primary|7500|Unknown|0|0|0| +5185|M|D|Primary|7500|Unknown|0|0|0| +5186|F|D|Primary|7500|Unknown|0|0|0| +5187|M|W|Primary|7500|Unknown|0|0|0| +5188|F|W|Primary|7500|Unknown|0|0|0| +5189|M|U|Primary|7500|Unknown|0|0|0| +5190|F|U|Primary|7500|Unknown|0|0|0| +5191|M|M|Secondary|7500|Unknown|0|0|0| +5192|F|M|Secondary|7500|Unknown|0|0|0| +5193|M|S|Secondary|7500|Unknown|0|0|0| +5194|F|S|Secondary|7500|Unknown|0|0|0| +5195|M|D|Secondary|7500|Unknown|0|0|0| +5196|F|D|Secondary|7500|Unknown|0|0|0| +5197|M|W|Secondary|7500|Unknown|0|0|0| +5198|F|W|Secondary|7500|Unknown|0|0|0| +5199|M|U|Secondary|7500|Unknown|0|0|0| +5200|F|U|Secondary|7500|Unknown|0|0|0| +5201|M|M|College|7500|Unknown|0|0|0| +5202|F|M|College|7500|Unknown|0|0|0| +5203|M|S|College|7500|Unknown|0|0|0| +5204|F|S|College|7500|Unknown|0|0|0| +5205|M|D|College|7500|Unknown|0|0|0| +5206|F|D|College|7500|Unknown|0|0|0| +5207|M|W|College|7500|Unknown|0|0|0| +5208|F|W|College|7500|Unknown|0|0|0| +5209|M|U|College|7500|Unknown|0|0|0| +5210|F|U|College|7500|Unknown|0|0|0| +5211|M|M|2 yr Degree|7500|Unknown|0|0|0| +5212|F|M|2 yr Degree|7500|Unknown|0|0|0| +5213|M|S|2 yr Degree|7500|Unknown|0|0|0| +5214|F|S|2 yr Degree|7500|Unknown|0|0|0| +5215|M|D|2 yr Degree|7500|Unknown|0|0|0| +5216|F|D|2 yr Degree|7500|Unknown|0|0|0| +5217|M|W|2 yr Degree|7500|Unknown|0|0|0| +5218|F|W|2 yr Degree|7500|Unknown|0|0|0| +5219|M|U|2 yr Degree|7500|Unknown|0|0|0| +5220|F|U|2 yr Degree|7500|Unknown|0|0|0| +5221|M|M|4 yr Degree|7500|Unknown|0|0|0| +5222|F|M|4 yr Degree|7500|Unknown|0|0|0| +5223|M|S|4 yr Degree|7500|Unknown|0|0|0| +5224|F|S|4 yr Degree|7500|Unknown|0|0|0| +5225|M|D|4 yr Degree|7500|Unknown|0|0|0| +5226|F|D|4 yr Degree|7500|Unknown|0|0|0| +5227|M|W|4 yr Degree|7500|Unknown|0|0|0| +5228|F|W|4 yr Degree|7500|Unknown|0|0|0| +5229|M|U|4 yr Degree|7500|Unknown|0|0|0| +5230|F|U|4 yr Degree|7500|Unknown|0|0|0| +5231|M|M|Advanced Degree|7500|Unknown|0|0|0| +5232|F|M|Advanced Degree|7500|Unknown|0|0|0| +5233|M|S|Advanced Degree|7500|Unknown|0|0|0| +5234|F|S|Advanced Degree|7500|Unknown|0|0|0| +5235|M|D|Advanced Degree|7500|Unknown|0|0|0| +5236|F|D|Advanced Degree|7500|Unknown|0|0|0| +5237|M|W|Advanced Degree|7500|Unknown|0|0|0| +5238|F|W|Advanced Degree|7500|Unknown|0|0|0| +5239|M|U|Advanced Degree|7500|Unknown|0|0|0| +5240|F|U|Advanced Degree|7500|Unknown|0|0|0| +5241|M|M|Unknown|7500|Unknown|0|0|0| +5242|F|M|Unknown|7500|Unknown|0|0|0| +5243|M|S|Unknown|7500|Unknown|0|0|0| +5244|F|S|Unknown|7500|Unknown|0|0|0| +5245|M|D|Unknown|7500|Unknown|0|0|0| +5246|F|D|Unknown|7500|Unknown|0|0|0| +5247|M|W|Unknown|7500|Unknown|0|0|0| +5248|F|W|Unknown|7500|Unknown|0|0|0| +5249|M|U|Unknown|7500|Unknown|0|0|0| +5250|F|U|Unknown|7500|Unknown|0|0|0| +5251|M|M|Primary|8000|Unknown|0|0|0| +5252|F|M|Primary|8000|Unknown|0|0|0| +5253|M|S|Primary|8000|Unknown|0|0|0| +5254|F|S|Primary|8000|Unknown|0|0|0| +5255|M|D|Primary|8000|Unknown|0|0|0| +5256|F|D|Primary|8000|Unknown|0|0|0| +5257|M|W|Primary|8000|Unknown|0|0|0| +5258|F|W|Primary|8000|Unknown|0|0|0| +5259|M|U|Primary|8000|Unknown|0|0|0| +5260|F|U|Primary|8000|Unknown|0|0|0| +5261|M|M|Secondary|8000|Unknown|0|0|0| +5262|F|M|Secondary|8000|Unknown|0|0|0| +5263|M|S|Secondary|8000|Unknown|0|0|0| +5264|F|S|Secondary|8000|Unknown|0|0|0| +5265|M|D|Secondary|8000|Unknown|0|0|0| +5266|F|D|Secondary|8000|Unknown|0|0|0| +5267|M|W|Secondary|8000|Unknown|0|0|0| +5268|F|W|Secondary|8000|Unknown|0|0|0| +5269|M|U|Secondary|8000|Unknown|0|0|0| +5270|F|U|Secondary|8000|Unknown|0|0|0| +5271|M|M|College|8000|Unknown|0|0|0| +5272|F|M|College|8000|Unknown|0|0|0| +5273|M|S|College|8000|Unknown|0|0|0| +5274|F|S|College|8000|Unknown|0|0|0| +5275|M|D|College|8000|Unknown|0|0|0| +5276|F|D|College|8000|Unknown|0|0|0| +5277|M|W|College|8000|Unknown|0|0|0| +5278|F|W|College|8000|Unknown|0|0|0| +5279|M|U|College|8000|Unknown|0|0|0| +5280|F|U|College|8000|Unknown|0|0|0| +5281|M|M|2 yr Degree|8000|Unknown|0|0|0| +5282|F|M|2 yr Degree|8000|Unknown|0|0|0| +5283|M|S|2 yr Degree|8000|Unknown|0|0|0| +5284|F|S|2 yr Degree|8000|Unknown|0|0|0| +5285|M|D|2 yr Degree|8000|Unknown|0|0|0| +5286|F|D|2 yr Degree|8000|Unknown|0|0|0| +5287|M|W|2 yr Degree|8000|Unknown|0|0|0| +5288|F|W|2 yr Degree|8000|Unknown|0|0|0| +5289|M|U|2 yr Degree|8000|Unknown|0|0|0| +5290|F|U|2 yr Degree|8000|Unknown|0|0|0| +5291|M|M|4 yr Degree|8000|Unknown|0|0|0| +5292|F|M|4 yr Degree|8000|Unknown|0|0|0| +5293|M|S|4 yr Degree|8000|Unknown|0|0|0| +5294|F|S|4 yr Degree|8000|Unknown|0|0|0| +5295|M|D|4 yr Degree|8000|Unknown|0|0|0| +5296|F|D|4 yr Degree|8000|Unknown|0|0|0| +5297|M|W|4 yr Degree|8000|Unknown|0|0|0| +5298|F|W|4 yr Degree|8000|Unknown|0|0|0| +5299|M|U|4 yr Degree|8000|Unknown|0|0|0| +5300|F|U|4 yr Degree|8000|Unknown|0|0|0| +5301|M|M|Advanced Degree|8000|Unknown|0|0|0| +5302|F|M|Advanced Degree|8000|Unknown|0|0|0| +5303|M|S|Advanced Degree|8000|Unknown|0|0|0| +5304|F|S|Advanced Degree|8000|Unknown|0|0|0| +5305|M|D|Advanced Degree|8000|Unknown|0|0|0| +5306|F|D|Advanced Degree|8000|Unknown|0|0|0| +5307|M|W|Advanced Degree|8000|Unknown|0|0|0| +5308|F|W|Advanced Degree|8000|Unknown|0|0|0| +5309|M|U|Advanced Degree|8000|Unknown|0|0|0| +5310|F|U|Advanced Degree|8000|Unknown|0|0|0| +5311|M|M|Unknown|8000|Unknown|0|0|0| +5312|F|M|Unknown|8000|Unknown|0|0|0| +5313|M|S|Unknown|8000|Unknown|0|0|0| +5314|F|S|Unknown|8000|Unknown|0|0|0| +5315|M|D|Unknown|8000|Unknown|0|0|0| +5316|F|D|Unknown|8000|Unknown|0|0|0| +5317|M|W|Unknown|8000|Unknown|0|0|0| +5318|F|W|Unknown|8000|Unknown|0|0|0| +5319|M|U|Unknown|8000|Unknown|0|0|0| +5320|F|U|Unknown|8000|Unknown|0|0|0| +5321|M|M|Primary|8500|Unknown|0|0|0| +5322|F|M|Primary|8500|Unknown|0|0|0| +5323|M|S|Primary|8500|Unknown|0|0|0| +5324|F|S|Primary|8500|Unknown|0|0|0| +5325|M|D|Primary|8500|Unknown|0|0|0| +5326|F|D|Primary|8500|Unknown|0|0|0| +5327|M|W|Primary|8500|Unknown|0|0|0| +5328|F|W|Primary|8500|Unknown|0|0|0| +5329|M|U|Primary|8500|Unknown|0|0|0| +5330|F|U|Primary|8500|Unknown|0|0|0| +5331|M|M|Secondary|8500|Unknown|0|0|0| +5332|F|M|Secondary|8500|Unknown|0|0|0| +5333|M|S|Secondary|8500|Unknown|0|0|0| +5334|F|S|Secondary|8500|Unknown|0|0|0| +5335|M|D|Secondary|8500|Unknown|0|0|0| +5336|F|D|Secondary|8500|Unknown|0|0|0| +5337|M|W|Secondary|8500|Unknown|0|0|0| +5338|F|W|Secondary|8500|Unknown|0|0|0| +5339|M|U|Secondary|8500|Unknown|0|0|0| +5340|F|U|Secondary|8500|Unknown|0|0|0| +5341|M|M|College|8500|Unknown|0|0|0| +5342|F|M|College|8500|Unknown|0|0|0| +5343|M|S|College|8500|Unknown|0|0|0| +5344|F|S|College|8500|Unknown|0|0|0| +5345|M|D|College|8500|Unknown|0|0|0| +5346|F|D|College|8500|Unknown|0|0|0| +5347|M|W|College|8500|Unknown|0|0|0| +5348|F|W|College|8500|Unknown|0|0|0| +5349|M|U|College|8500|Unknown|0|0|0| +5350|F|U|College|8500|Unknown|0|0|0| +5351|M|M|2 yr Degree|8500|Unknown|0|0|0| +5352|F|M|2 yr Degree|8500|Unknown|0|0|0| +5353|M|S|2 yr Degree|8500|Unknown|0|0|0| +5354|F|S|2 yr Degree|8500|Unknown|0|0|0| +5355|M|D|2 yr Degree|8500|Unknown|0|0|0| +5356|F|D|2 yr Degree|8500|Unknown|0|0|0| +5357|M|W|2 yr Degree|8500|Unknown|0|0|0| +5358|F|W|2 yr Degree|8500|Unknown|0|0|0| +5359|M|U|2 yr Degree|8500|Unknown|0|0|0| +5360|F|U|2 yr Degree|8500|Unknown|0|0|0| +5361|M|M|4 yr Degree|8500|Unknown|0|0|0| +5362|F|M|4 yr Degree|8500|Unknown|0|0|0| +5363|M|S|4 yr Degree|8500|Unknown|0|0|0| +5364|F|S|4 yr Degree|8500|Unknown|0|0|0| +5365|M|D|4 yr Degree|8500|Unknown|0|0|0| +5366|F|D|4 yr Degree|8500|Unknown|0|0|0| +5367|M|W|4 yr Degree|8500|Unknown|0|0|0| +5368|F|W|4 yr Degree|8500|Unknown|0|0|0| +5369|M|U|4 yr Degree|8500|Unknown|0|0|0| +5370|F|U|4 yr Degree|8500|Unknown|0|0|0| +5371|M|M|Advanced Degree|8500|Unknown|0|0|0| +5372|F|M|Advanced Degree|8500|Unknown|0|0|0| +5373|M|S|Advanced Degree|8500|Unknown|0|0|0| +5374|F|S|Advanced Degree|8500|Unknown|0|0|0| +5375|M|D|Advanced Degree|8500|Unknown|0|0|0| +5376|F|D|Advanced Degree|8500|Unknown|0|0|0| +5377|M|W|Advanced Degree|8500|Unknown|0|0|0| +5378|F|W|Advanced Degree|8500|Unknown|0|0|0| +5379|M|U|Advanced Degree|8500|Unknown|0|0|0| +5380|F|U|Advanced Degree|8500|Unknown|0|0|0| +5381|M|M|Unknown|8500|Unknown|0|0|0| +5382|F|M|Unknown|8500|Unknown|0|0|0| +5383|M|S|Unknown|8500|Unknown|0|0|0| +5384|F|S|Unknown|8500|Unknown|0|0|0| +5385|M|D|Unknown|8500|Unknown|0|0|0| +5386|F|D|Unknown|8500|Unknown|0|0|0| +5387|M|W|Unknown|8500|Unknown|0|0|0| +5388|F|W|Unknown|8500|Unknown|0|0|0| +5389|M|U|Unknown|8500|Unknown|0|0|0| +5390|F|U|Unknown|8500|Unknown|0|0|0| +5391|M|M|Primary|9000|Unknown|0|0|0| +5392|F|M|Primary|9000|Unknown|0|0|0| +5393|M|S|Primary|9000|Unknown|0|0|0| +5394|F|S|Primary|9000|Unknown|0|0|0| +5395|M|D|Primary|9000|Unknown|0|0|0| +5396|F|D|Primary|9000|Unknown|0|0|0| +5397|M|W|Primary|9000|Unknown|0|0|0| +5398|F|W|Primary|9000|Unknown|0|0|0| +5399|M|U|Primary|9000|Unknown|0|0|0| +5400|F|U|Primary|9000|Unknown|0|0|0| +5401|M|M|Secondary|9000|Unknown|0|0|0| +5402|F|M|Secondary|9000|Unknown|0|0|0| +5403|M|S|Secondary|9000|Unknown|0|0|0| +5404|F|S|Secondary|9000|Unknown|0|0|0| +5405|M|D|Secondary|9000|Unknown|0|0|0| +5406|F|D|Secondary|9000|Unknown|0|0|0| +5407|M|W|Secondary|9000|Unknown|0|0|0| +5408|F|W|Secondary|9000|Unknown|0|0|0| +5409|M|U|Secondary|9000|Unknown|0|0|0| +5410|F|U|Secondary|9000|Unknown|0|0|0| +5411|M|M|College|9000|Unknown|0|0|0| +5412|F|M|College|9000|Unknown|0|0|0| +5413|M|S|College|9000|Unknown|0|0|0| +5414|F|S|College|9000|Unknown|0|0|0| +5415|M|D|College|9000|Unknown|0|0|0| +5416|F|D|College|9000|Unknown|0|0|0| +5417|M|W|College|9000|Unknown|0|0|0| +5418|F|W|College|9000|Unknown|0|0|0| +5419|M|U|College|9000|Unknown|0|0|0| +5420|F|U|College|9000|Unknown|0|0|0| +5421|M|M|2 yr Degree|9000|Unknown|0|0|0| +5422|F|M|2 yr Degree|9000|Unknown|0|0|0| +5423|M|S|2 yr Degree|9000|Unknown|0|0|0| +5424|F|S|2 yr Degree|9000|Unknown|0|0|0| +5425|M|D|2 yr Degree|9000|Unknown|0|0|0| +5426|F|D|2 yr Degree|9000|Unknown|0|0|0| +5427|M|W|2 yr Degree|9000|Unknown|0|0|0| +5428|F|W|2 yr Degree|9000|Unknown|0|0|0| +5429|M|U|2 yr Degree|9000|Unknown|0|0|0| +5430|F|U|2 yr Degree|9000|Unknown|0|0|0| +5431|M|M|4 yr Degree|9000|Unknown|0|0|0| +5432|F|M|4 yr Degree|9000|Unknown|0|0|0| +5433|M|S|4 yr Degree|9000|Unknown|0|0|0| +5434|F|S|4 yr Degree|9000|Unknown|0|0|0| +5435|M|D|4 yr Degree|9000|Unknown|0|0|0| +5436|F|D|4 yr Degree|9000|Unknown|0|0|0| +5437|M|W|4 yr Degree|9000|Unknown|0|0|0| +5438|F|W|4 yr Degree|9000|Unknown|0|0|0| +5439|M|U|4 yr Degree|9000|Unknown|0|0|0| +5440|F|U|4 yr Degree|9000|Unknown|0|0|0| +5441|M|M|Advanced Degree|9000|Unknown|0|0|0| +5442|F|M|Advanced Degree|9000|Unknown|0|0|0| +5443|M|S|Advanced Degree|9000|Unknown|0|0|0| +5444|F|S|Advanced Degree|9000|Unknown|0|0|0| +5445|M|D|Advanced Degree|9000|Unknown|0|0|0| +5446|F|D|Advanced Degree|9000|Unknown|0|0|0| +5447|M|W|Advanced Degree|9000|Unknown|0|0|0| +5448|F|W|Advanced Degree|9000|Unknown|0|0|0| +5449|M|U|Advanced Degree|9000|Unknown|0|0|0| +5450|F|U|Advanced Degree|9000|Unknown|0|0|0| +5451|M|M|Unknown|9000|Unknown|0|0|0| +5452|F|M|Unknown|9000|Unknown|0|0|0| +5453|M|S|Unknown|9000|Unknown|0|0|0| +5454|F|S|Unknown|9000|Unknown|0|0|0| +5455|M|D|Unknown|9000|Unknown|0|0|0| +5456|F|D|Unknown|9000|Unknown|0|0|0| +5457|M|W|Unknown|9000|Unknown|0|0|0| +5458|F|W|Unknown|9000|Unknown|0|0|0| +5459|M|U|Unknown|9000|Unknown|0|0|0| +5460|F|U|Unknown|9000|Unknown|0|0|0| +5461|M|M|Primary|9500|Unknown|0|0|0| +5462|F|M|Primary|9500|Unknown|0|0|0| +5463|M|S|Primary|9500|Unknown|0|0|0| +5464|F|S|Primary|9500|Unknown|0|0|0| +5465|M|D|Primary|9500|Unknown|0|0|0| +5466|F|D|Primary|9500|Unknown|0|0|0| +5467|M|W|Primary|9500|Unknown|0|0|0| +5468|F|W|Primary|9500|Unknown|0|0|0| +5469|M|U|Primary|9500|Unknown|0|0|0| +5470|F|U|Primary|9500|Unknown|0|0|0| +5471|M|M|Secondary|9500|Unknown|0|0|0| +5472|F|M|Secondary|9500|Unknown|0|0|0| +5473|M|S|Secondary|9500|Unknown|0|0|0| +5474|F|S|Secondary|9500|Unknown|0|0|0| +5475|M|D|Secondary|9500|Unknown|0|0|0| +5476|F|D|Secondary|9500|Unknown|0|0|0| +5477|M|W|Secondary|9500|Unknown|0|0|0| +5478|F|W|Secondary|9500|Unknown|0|0|0| +5479|M|U|Secondary|9500|Unknown|0|0|0| +5480|F|U|Secondary|9500|Unknown|0|0|0| +5481|M|M|College|9500|Unknown|0|0|0| +5482|F|M|College|9500|Unknown|0|0|0| +5483|M|S|College|9500|Unknown|0|0|0| +5484|F|S|College|9500|Unknown|0|0|0| +5485|M|D|College|9500|Unknown|0|0|0| +5486|F|D|College|9500|Unknown|0|0|0| +5487|M|W|College|9500|Unknown|0|0|0| +5488|F|W|College|9500|Unknown|0|0|0| +5489|M|U|College|9500|Unknown|0|0|0| +5490|F|U|College|9500|Unknown|0|0|0| +5491|M|M|2 yr Degree|9500|Unknown|0|0|0| +5492|F|M|2 yr Degree|9500|Unknown|0|0|0| +5493|M|S|2 yr Degree|9500|Unknown|0|0|0| +5494|F|S|2 yr Degree|9500|Unknown|0|0|0| +5495|M|D|2 yr Degree|9500|Unknown|0|0|0| +5496|F|D|2 yr Degree|9500|Unknown|0|0|0| +5497|M|W|2 yr Degree|9500|Unknown|0|0|0| +5498|F|W|2 yr Degree|9500|Unknown|0|0|0| +5499|M|U|2 yr Degree|9500|Unknown|0|0|0| +5500|F|U|2 yr Degree|9500|Unknown|0|0|0| +5501|M|M|4 yr Degree|9500|Unknown|0|0|0| +5502|F|M|4 yr Degree|9500|Unknown|0|0|0| +5503|M|S|4 yr Degree|9500|Unknown|0|0|0| +5504|F|S|4 yr Degree|9500|Unknown|0|0|0| +5505|M|D|4 yr Degree|9500|Unknown|0|0|0| +5506|F|D|4 yr Degree|9500|Unknown|0|0|0| +5507|M|W|4 yr Degree|9500|Unknown|0|0|0| +5508|F|W|4 yr Degree|9500|Unknown|0|0|0| +5509|M|U|4 yr Degree|9500|Unknown|0|0|0| +5510|F|U|4 yr Degree|9500|Unknown|0|0|0| +5511|M|M|Advanced Degree|9500|Unknown|0|0|0| +5512|F|M|Advanced Degree|9500|Unknown|0|0|0| +5513|M|S|Advanced Degree|9500|Unknown|0|0|0| +5514|F|S|Advanced Degree|9500|Unknown|0|0|0| +5515|M|D|Advanced Degree|9500|Unknown|0|0|0| +5516|F|D|Advanced Degree|9500|Unknown|0|0|0| +5517|M|W|Advanced Degree|9500|Unknown|0|0|0| +5518|F|W|Advanced Degree|9500|Unknown|0|0|0| +5519|M|U|Advanced Degree|9500|Unknown|0|0|0| +5520|F|U|Advanced Degree|9500|Unknown|0|0|0| +5521|M|M|Unknown|9500|Unknown|0|0|0| +5522|F|M|Unknown|9500|Unknown|0|0|0| +5523|M|S|Unknown|9500|Unknown|0|0|0| +5524|F|S|Unknown|9500|Unknown|0|0|0| +5525|M|D|Unknown|9500|Unknown|0|0|0| +5526|F|D|Unknown|9500|Unknown|0|0|0| +5527|M|W|Unknown|9500|Unknown|0|0|0| +5528|F|W|Unknown|9500|Unknown|0|0|0| +5529|M|U|Unknown|9500|Unknown|0|0|0| +5530|F|U|Unknown|9500|Unknown|0|0|0| +5531|M|M|Primary|10000|Unknown|0|0|0| +5532|F|M|Primary|10000|Unknown|0|0|0| +5533|M|S|Primary|10000|Unknown|0|0|0| +5534|F|S|Primary|10000|Unknown|0|0|0| +5535|M|D|Primary|10000|Unknown|0|0|0| +5536|F|D|Primary|10000|Unknown|0|0|0| +5537|M|W|Primary|10000|Unknown|0|0|0| +5538|F|W|Primary|10000|Unknown|0|0|0| +5539|M|U|Primary|10000|Unknown|0|0|0| +5540|F|U|Primary|10000|Unknown|0|0|0| +5541|M|M|Secondary|10000|Unknown|0|0|0| +5542|F|M|Secondary|10000|Unknown|0|0|0| +5543|M|S|Secondary|10000|Unknown|0|0|0| +5544|F|S|Secondary|10000|Unknown|0|0|0| +5545|M|D|Secondary|10000|Unknown|0|0|0| +5546|F|D|Secondary|10000|Unknown|0|0|0| +5547|M|W|Secondary|10000|Unknown|0|0|0| +5548|F|W|Secondary|10000|Unknown|0|0|0| +5549|M|U|Secondary|10000|Unknown|0|0|0| +5550|F|U|Secondary|10000|Unknown|0|0|0| +5551|M|M|College|10000|Unknown|0|0|0| +5552|F|M|College|10000|Unknown|0|0|0| +5553|M|S|College|10000|Unknown|0|0|0| +5554|F|S|College|10000|Unknown|0|0|0| +5555|M|D|College|10000|Unknown|0|0|0| +5556|F|D|College|10000|Unknown|0|0|0| +5557|M|W|College|10000|Unknown|0|0|0| +5558|F|W|College|10000|Unknown|0|0|0| +5559|M|U|College|10000|Unknown|0|0|0| +5560|F|U|College|10000|Unknown|0|0|0| +5561|M|M|2 yr Degree|10000|Unknown|0|0|0| +5562|F|M|2 yr Degree|10000|Unknown|0|0|0| +5563|M|S|2 yr Degree|10000|Unknown|0|0|0| +5564|F|S|2 yr Degree|10000|Unknown|0|0|0| +5565|M|D|2 yr Degree|10000|Unknown|0|0|0| +5566|F|D|2 yr Degree|10000|Unknown|0|0|0| +5567|M|W|2 yr Degree|10000|Unknown|0|0|0| +5568|F|W|2 yr Degree|10000|Unknown|0|0|0| +5569|M|U|2 yr Degree|10000|Unknown|0|0|0| +5570|F|U|2 yr Degree|10000|Unknown|0|0|0| +5571|M|M|4 yr Degree|10000|Unknown|0|0|0| +5572|F|M|4 yr Degree|10000|Unknown|0|0|0| +5573|M|S|4 yr Degree|10000|Unknown|0|0|0| +5574|F|S|4 yr Degree|10000|Unknown|0|0|0| +5575|M|D|4 yr Degree|10000|Unknown|0|0|0| +5576|F|D|4 yr Degree|10000|Unknown|0|0|0| +5577|M|W|4 yr Degree|10000|Unknown|0|0|0| +5578|F|W|4 yr Degree|10000|Unknown|0|0|0| +5579|M|U|4 yr Degree|10000|Unknown|0|0|0| +5580|F|U|4 yr Degree|10000|Unknown|0|0|0| +5581|M|M|Advanced Degree|10000|Unknown|0|0|0| +5582|F|M|Advanced Degree|10000|Unknown|0|0|0| +5583|M|S|Advanced Degree|10000|Unknown|0|0|0| +5584|F|S|Advanced Degree|10000|Unknown|0|0|0| +5585|M|D|Advanced Degree|10000|Unknown|0|0|0| +5586|F|D|Advanced Degree|10000|Unknown|0|0|0| +5587|M|W|Advanced Degree|10000|Unknown|0|0|0| +5588|F|W|Advanced Degree|10000|Unknown|0|0|0| +5589|M|U|Advanced Degree|10000|Unknown|0|0|0| +5590|F|U|Advanced Degree|10000|Unknown|0|0|0| +5591|M|M|Unknown|10000|Unknown|0|0|0| +5592|F|M|Unknown|10000|Unknown|0|0|0| +5593|M|S|Unknown|10000|Unknown|0|0|0| +5594|F|S|Unknown|10000|Unknown|0|0|0| +5595|M|D|Unknown|10000|Unknown|0|0|0| +5596|F|D|Unknown|10000|Unknown|0|0|0| +5597|M|W|Unknown|10000|Unknown|0|0|0| +5598|F|W|Unknown|10000|Unknown|0|0|0| +5599|M|U|Unknown|10000|Unknown|0|0|0| +5600|F|U|Unknown|10000|Unknown|0|0|0| +5601|M|M|Primary|500|Good|1|0|0| +5602|F|M|Primary|500|Good|1|0|0| +5603|M|S|Primary|500|Good|1|0|0| +5604|F|S|Primary|500|Good|1|0|0| +5605|M|D|Primary|500|Good|1|0|0| +5606|F|D|Primary|500|Good|1|0|0| +5607|M|W|Primary|500|Good|1|0|0| +5608|F|W|Primary|500|Good|1|0|0| +5609|M|U|Primary|500|Good|1|0|0| +5610|F|U|Primary|500|Good|1|0|0| +5611|M|M|Secondary|500|Good|1|0|0| +5612|F|M|Secondary|500|Good|1|0|0| +5613|M|S|Secondary|500|Good|1|0|0| +5614|F|S|Secondary|500|Good|1|0|0| +5615|M|D|Secondary|500|Good|1|0|0| +5616|F|D|Secondary|500|Good|1|0|0| +5617|M|W|Secondary|500|Good|1|0|0| +5618|F|W|Secondary|500|Good|1|0|0| +5619|M|U|Secondary|500|Good|1|0|0| +5620|F|U|Secondary|500|Good|1|0|0| +5621|M|M|College|500|Good|1|0|0| +5622|F|M|College|500|Good|1|0|0| +5623|M|S|College|500|Good|1|0|0| +5624|F|S|College|500|Good|1|0|0| +5625|M|D|College|500|Good|1|0|0| +5626|F|D|College|500|Good|1|0|0| +5627|M|W|College|500|Good|1|0|0| +5628|F|W|College|500|Good|1|0|0| +5629|M|U|College|500|Good|1|0|0| +5630|F|U|College|500|Good|1|0|0| +5631|M|M|2 yr Degree|500|Good|1|0|0| +5632|F|M|2 yr Degree|500|Good|1|0|0| +5633|M|S|2 yr Degree|500|Good|1|0|0| +5634|F|S|2 yr Degree|500|Good|1|0|0| +5635|M|D|2 yr Degree|500|Good|1|0|0| +5636|F|D|2 yr Degree|500|Good|1|0|0| +5637|M|W|2 yr Degree|500|Good|1|0|0| +5638|F|W|2 yr Degree|500|Good|1|0|0| +5639|M|U|2 yr Degree|500|Good|1|0|0| +5640|F|U|2 yr Degree|500|Good|1|0|0| +5641|M|M|4 yr Degree|500|Good|1|0|0| +5642|F|M|4 yr Degree|500|Good|1|0|0| +5643|M|S|4 yr Degree|500|Good|1|0|0| +5644|F|S|4 yr Degree|500|Good|1|0|0| +5645|M|D|4 yr Degree|500|Good|1|0|0| +5646|F|D|4 yr Degree|500|Good|1|0|0| +5647|M|W|4 yr Degree|500|Good|1|0|0| +5648|F|W|4 yr Degree|500|Good|1|0|0| +5649|M|U|4 yr Degree|500|Good|1|0|0| +5650|F|U|4 yr Degree|500|Good|1|0|0| +5651|M|M|Advanced Degree|500|Good|1|0|0| +5652|F|M|Advanced Degree|500|Good|1|0|0| +5653|M|S|Advanced Degree|500|Good|1|0|0| +5654|F|S|Advanced Degree|500|Good|1|0|0| +5655|M|D|Advanced Degree|500|Good|1|0|0| +5656|F|D|Advanced Degree|500|Good|1|0|0| +5657|M|W|Advanced Degree|500|Good|1|0|0| +5658|F|W|Advanced Degree|500|Good|1|0|0| +5659|M|U|Advanced Degree|500|Good|1|0|0| +5660|F|U|Advanced Degree|500|Good|1|0|0| +5661|M|M|Unknown|500|Good|1|0|0| +5662|F|M|Unknown|500|Good|1|0|0| +5663|M|S|Unknown|500|Good|1|0|0| +5664|F|S|Unknown|500|Good|1|0|0| +5665|M|D|Unknown|500|Good|1|0|0| +5666|F|D|Unknown|500|Good|1|0|0| +5667|M|W|Unknown|500|Good|1|0|0| +5668|F|W|Unknown|500|Good|1|0|0| +5669|M|U|Unknown|500|Good|1|0|0| +5670|F|U|Unknown|500|Good|1|0|0| +5671|M|M|Primary|1000|Good|1|0|0| +5672|F|M|Primary|1000|Good|1|0|0| +5673|M|S|Primary|1000|Good|1|0|0| +5674|F|S|Primary|1000|Good|1|0|0| +5675|M|D|Primary|1000|Good|1|0|0| +5676|F|D|Primary|1000|Good|1|0|0| +5677|M|W|Primary|1000|Good|1|0|0| +5678|F|W|Primary|1000|Good|1|0|0| +5679|M|U|Primary|1000|Good|1|0|0| +5680|F|U|Primary|1000|Good|1|0|0| +5681|M|M|Secondary|1000|Good|1|0|0| +5682|F|M|Secondary|1000|Good|1|0|0| +5683|M|S|Secondary|1000|Good|1|0|0| +5684|F|S|Secondary|1000|Good|1|0|0| +5685|M|D|Secondary|1000|Good|1|0|0| +5686|F|D|Secondary|1000|Good|1|0|0| +5687|M|W|Secondary|1000|Good|1|0|0| +5688|F|W|Secondary|1000|Good|1|0|0| +5689|M|U|Secondary|1000|Good|1|0|0| +5690|F|U|Secondary|1000|Good|1|0|0| +5691|M|M|College|1000|Good|1|0|0| +5692|F|M|College|1000|Good|1|0|0| +5693|M|S|College|1000|Good|1|0|0| +5694|F|S|College|1000|Good|1|0|0| +5695|M|D|College|1000|Good|1|0|0| +5696|F|D|College|1000|Good|1|0|0| +5697|M|W|College|1000|Good|1|0|0| +5698|F|W|College|1000|Good|1|0|0| +5699|M|U|College|1000|Good|1|0|0| +5700|F|U|College|1000|Good|1|0|0| +5701|M|M|2 yr Degree|1000|Good|1|0|0| +5702|F|M|2 yr Degree|1000|Good|1|0|0| +5703|M|S|2 yr Degree|1000|Good|1|0|0| +5704|F|S|2 yr Degree|1000|Good|1|0|0| +5705|M|D|2 yr Degree|1000|Good|1|0|0| +5706|F|D|2 yr Degree|1000|Good|1|0|0| +5707|M|W|2 yr Degree|1000|Good|1|0|0| +5708|F|W|2 yr Degree|1000|Good|1|0|0| +5709|M|U|2 yr Degree|1000|Good|1|0|0| +5710|F|U|2 yr Degree|1000|Good|1|0|0| +5711|M|M|4 yr Degree|1000|Good|1|0|0| +5712|F|M|4 yr Degree|1000|Good|1|0|0| +5713|M|S|4 yr Degree|1000|Good|1|0|0| +5714|F|S|4 yr Degree|1000|Good|1|0|0| +5715|M|D|4 yr Degree|1000|Good|1|0|0| +5716|F|D|4 yr Degree|1000|Good|1|0|0| +5717|M|W|4 yr Degree|1000|Good|1|0|0| +5718|F|W|4 yr Degree|1000|Good|1|0|0| +5719|M|U|4 yr Degree|1000|Good|1|0|0| +5720|F|U|4 yr Degree|1000|Good|1|0|0| +5721|M|M|Advanced Degree|1000|Good|1|0|0| +5722|F|M|Advanced Degree|1000|Good|1|0|0| +5723|M|S|Advanced Degree|1000|Good|1|0|0| +5724|F|S|Advanced Degree|1000|Good|1|0|0| +5725|M|D|Advanced Degree|1000|Good|1|0|0| +5726|F|D|Advanced Degree|1000|Good|1|0|0| +5727|M|W|Advanced Degree|1000|Good|1|0|0| +5728|F|W|Advanced Degree|1000|Good|1|0|0| +5729|M|U|Advanced Degree|1000|Good|1|0|0| +5730|F|U|Advanced Degree|1000|Good|1|0|0| +5731|M|M|Unknown|1000|Good|1|0|0| +5732|F|M|Unknown|1000|Good|1|0|0| +5733|M|S|Unknown|1000|Good|1|0|0| +5734|F|S|Unknown|1000|Good|1|0|0| +5735|M|D|Unknown|1000|Good|1|0|0| +5736|F|D|Unknown|1000|Good|1|0|0| +5737|M|W|Unknown|1000|Good|1|0|0| +5738|F|W|Unknown|1000|Good|1|0|0| +5739|M|U|Unknown|1000|Good|1|0|0| +5740|F|U|Unknown|1000|Good|1|0|0| +5741|M|M|Primary|1500|Good|1|0|0| +5742|F|M|Primary|1500|Good|1|0|0| +5743|M|S|Primary|1500|Good|1|0|0| +5744|F|S|Primary|1500|Good|1|0|0| +5745|M|D|Primary|1500|Good|1|0|0| +5746|F|D|Primary|1500|Good|1|0|0| +5747|M|W|Primary|1500|Good|1|0|0| +5748|F|W|Primary|1500|Good|1|0|0| +5749|M|U|Primary|1500|Good|1|0|0| +5750|F|U|Primary|1500|Good|1|0|0| +5751|M|M|Secondary|1500|Good|1|0|0| +5752|F|M|Secondary|1500|Good|1|0|0| +5753|M|S|Secondary|1500|Good|1|0|0| +5754|F|S|Secondary|1500|Good|1|0|0| +5755|M|D|Secondary|1500|Good|1|0|0| +5756|F|D|Secondary|1500|Good|1|0|0| +5757|M|W|Secondary|1500|Good|1|0|0| +5758|F|W|Secondary|1500|Good|1|0|0| +5759|M|U|Secondary|1500|Good|1|0|0| +5760|F|U|Secondary|1500|Good|1|0|0| +5761|M|M|College|1500|Good|1|0|0| +5762|F|M|College|1500|Good|1|0|0| +5763|M|S|College|1500|Good|1|0|0| +5764|F|S|College|1500|Good|1|0|0| +5765|M|D|College|1500|Good|1|0|0| +5766|F|D|College|1500|Good|1|0|0| +5767|M|W|College|1500|Good|1|0|0| +5768|F|W|College|1500|Good|1|0|0| +5769|M|U|College|1500|Good|1|0|0| +5770|F|U|College|1500|Good|1|0|0| +5771|M|M|2 yr Degree|1500|Good|1|0|0| +5772|F|M|2 yr Degree|1500|Good|1|0|0| +5773|M|S|2 yr Degree|1500|Good|1|0|0| +5774|F|S|2 yr Degree|1500|Good|1|0|0| +5775|M|D|2 yr Degree|1500|Good|1|0|0| +5776|F|D|2 yr Degree|1500|Good|1|0|0| +5777|M|W|2 yr Degree|1500|Good|1|0|0| +5778|F|W|2 yr Degree|1500|Good|1|0|0| +5779|M|U|2 yr Degree|1500|Good|1|0|0| +5780|F|U|2 yr Degree|1500|Good|1|0|0| +5781|M|M|4 yr Degree|1500|Good|1|0|0| +5782|F|M|4 yr Degree|1500|Good|1|0|0| +5783|M|S|4 yr Degree|1500|Good|1|0|0| +5784|F|S|4 yr Degree|1500|Good|1|0|0| +5785|M|D|4 yr Degree|1500|Good|1|0|0| +5786|F|D|4 yr Degree|1500|Good|1|0|0| +5787|M|W|4 yr Degree|1500|Good|1|0|0| +5788|F|W|4 yr Degree|1500|Good|1|0|0| +5789|M|U|4 yr Degree|1500|Good|1|0|0| +5790|F|U|4 yr Degree|1500|Good|1|0|0| +5791|M|M|Advanced Degree|1500|Good|1|0|0| +5792|F|M|Advanced Degree|1500|Good|1|0|0| +5793|M|S|Advanced Degree|1500|Good|1|0|0| +5794|F|S|Advanced Degree|1500|Good|1|0|0| +5795|M|D|Advanced Degree|1500|Good|1|0|0| +5796|F|D|Advanced Degree|1500|Good|1|0|0| +5797|M|W|Advanced Degree|1500|Good|1|0|0| +5798|F|W|Advanced Degree|1500|Good|1|0|0| +5799|M|U|Advanced Degree|1500|Good|1|0|0| +5800|F|U|Advanced Degree|1500|Good|1|0|0| +5801|M|M|Unknown|1500|Good|1|0|0| +5802|F|M|Unknown|1500|Good|1|0|0| +5803|M|S|Unknown|1500|Good|1|0|0| +5804|F|S|Unknown|1500|Good|1|0|0| +5805|M|D|Unknown|1500|Good|1|0|0| +5806|F|D|Unknown|1500|Good|1|0|0| +5807|M|W|Unknown|1500|Good|1|0|0| +5808|F|W|Unknown|1500|Good|1|0|0| +5809|M|U|Unknown|1500|Good|1|0|0| +5810|F|U|Unknown|1500|Good|1|0|0| +5811|M|M|Primary|2000|Good|1|0|0| +5812|F|M|Primary|2000|Good|1|0|0| +5813|M|S|Primary|2000|Good|1|0|0| +5814|F|S|Primary|2000|Good|1|0|0| +5815|M|D|Primary|2000|Good|1|0|0| +5816|F|D|Primary|2000|Good|1|0|0| +5817|M|W|Primary|2000|Good|1|0|0| +5818|F|W|Primary|2000|Good|1|0|0| +5819|M|U|Primary|2000|Good|1|0|0| +5820|F|U|Primary|2000|Good|1|0|0| +5821|M|M|Secondary|2000|Good|1|0|0| +5822|F|M|Secondary|2000|Good|1|0|0| +5823|M|S|Secondary|2000|Good|1|0|0| +5824|F|S|Secondary|2000|Good|1|0|0| +5825|M|D|Secondary|2000|Good|1|0|0| +5826|F|D|Secondary|2000|Good|1|0|0| +5827|M|W|Secondary|2000|Good|1|0|0| +5828|F|W|Secondary|2000|Good|1|0|0| +5829|M|U|Secondary|2000|Good|1|0|0| +5830|F|U|Secondary|2000|Good|1|0|0| +5831|M|M|College|2000|Good|1|0|0| +5832|F|M|College|2000|Good|1|0|0| +5833|M|S|College|2000|Good|1|0|0| +5834|F|S|College|2000|Good|1|0|0| +5835|M|D|College|2000|Good|1|0|0| +5836|F|D|College|2000|Good|1|0|0| +5837|M|W|College|2000|Good|1|0|0| +5838|F|W|College|2000|Good|1|0|0| +5839|M|U|College|2000|Good|1|0|0| +5840|F|U|College|2000|Good|1|0|0| +5841|M|M|2 yr Degree|2000|Good|1|0|0| +5842|F|M|2 yr Degree|2000|Good|1|0|0| +5843|M|S|2 yr Degree|2000|Good|1|0|0| +5844|F|S|2 yr Degree|2000|Good|1|0|0| +5845|M|D|2 yr Degree|2000|Good|1|0|0| +5846|F|D|2 yr Degree|2000|Good|1|0|0| +5847|M|W|2 yr Degree|2000|Good|1|0|0| +5848|F|W|2 yr Degree|2000|Good|1|0|0| +5849|M|U|2 yr Degree|2000|Good|1|0|0| +5850|F|U|2 yr Degree|2000|Good|1|0|0| +5851|M|M|4 yr Degree|2000|Good|1|0|0| +5852|F|M|4 yr Degree|2000|Good|1|0|0| +5853|M|S|4 yr Degree|2000|Good|1|0|0| +5854|F|S|4 yr Degree|2000|Good|1|0|0| +5855|M|D|4 yr Degree|2000|Good|1|0|0| +5856|F|D|4 yr Degree|2000|Good|1|0|0| +5857|M|W|4 yr Degree|2000|Good|1|0|0| +5858|F|W|4 yr Degree|2000|Good|1|0|0| +5859|M|U|4 yr Degree|2000|Good|1|0|0| +5860|F|U|4 yr Degree|2000|Good|1|0|0| +5861|M|M|Advanced Degree|2000|Good|1|0|0| +5862|F|M|Advanced Degree|2000|Good|1|0|0| +5863|M|S|Advanced Degree|2000|Good|1|0|0| +5864|F|S|Advanced Degree|2000|Good|1|0|0| +5865|M|D|Advanced Degree|2000|Good|1|0|0| +5866|F|D|Advanced Degree|2000|Good|1|0|0| +5867|M|W|Advanced Degree|2000|Good|1|0|0| +5868|F|W|Advanced Degree|2000|Good|1|0|0| +5869|M|U|Advanced Degree|2000|Good|1|0|0| +5870|F|U|Advanced Degree|2000|Good|1|0|0| +5871|M|M|Unknown|2000|Good|1|0|0| +5872|F|M|Unknown|2000|Good|1|0|0| +5873|M|S|Unknown|2000|Good|1|0|0| +5874|F|S|Unknown|2000|Good|1|0|0| +5875|M|D|Unknown|2000|Good|1|0|0| +5876|F|D|Unknown|2000|Good|1|0|0| +5877|M|W|Unknown|2000|Good|1|0|0| +5878|F|W|Unknown|2000|Good|1|0|0| +5879|M|U|Unknown|2000|Good|1|0|0| +5880|F|U|Unknown|2000|Good|1|0|0| +5881|M|M|Primary|2500|Good|1|0|0| +5882|F|M|Primary|2500|Good|1|0|0| +5883|M|S|Primary|2500|Good|1|0|0| +5884|F|S|Primary|2500|Good|1|0|0| +5885|M|D|Primary|2500|Good|1|0|0| +5886|F|D|Primary|2500|Good|1|0|0| +5887|M|W|Primary|2500|Good|1|0|0| +5888|F|W|Primary|2500|Good|1|0|0| +5889|M|U|Primary|2500|Good|1|0|0| +5890|F|U|Primary|2500|Good|1|0|0| +5891|M|M|Secondary|2500|Good|1|0|0| +5892|F|M|Secondary|2500|Good|1|0|0| +5893|M|S|Secondary|2500|Good|1|0|0| +5894|F|S|Secondary|2500|Good|1|0|0| +5895|M|D|Secondary|2500|Good|1|0|0| +5896|F|D|Secondary|2500|Good|1|0|0| +5897|M|W|Secondary|2500|Good|1|0|0| +5898|F|W|Secondary|2500|Good|1|0|0| +5899|M|U|Secondary|2500|Good|1|0|0| +5900|F|U|Secondary|2500|Good|1|0|0| +5901|M|M|College|2500|Good|1|0|0| +5902|F|M|College|2500|Good|1|0|0| +5903|M|S|College|2500|Good|1|0|0| +5904|F|S|College|2500|Good|1|0|0| +5905|M|D|College|2500|Good|1|0|0| +5906|F|D|College|2500|Good|1|0|0| +5907|M|W|College|2500|Good|1|0|0| +5908|F|W|College|2500|Good|1|0|0| +5909|M|U|College|2500|Good|1|0|0| +5910|F|U|College|2500|Good|1|0|0| +5911|M|M|2 yr Degree|2500|Good|1|0|0| +5912|F|M|2 yr Degree|2500|Good|1|0|0| +5913|M|S|2 yr Degree|2500|Good|1|0|0| +5914|F|S|2 yr Degree|2500|Good|1|0|0| +5915|M|D|2 yr Degree|2500|Good|1|0|0| +5916|F|D|2 yr Degree|2500|Good|1|0|0| +5917|M|W|2 yr Degree|2500|Good|1|0|0| +5918|F|W|2 yr Degree|2500|Good|1|0|0| +5919|M|U|2 yr Degree|2500|Good|1|0|0| +5920|F|U|2 yr Degree|2500|Good|1|0|0| +5921|M|M|4 yr Degree|2500|Good|1|0|0| +5922|F|M|4 yr Degree|2500|Good|1|0|0| +5923|M|S|4 yr Degree|2500|Good|1|0|0| +5924|F|S|4 yr Degree|2500|Good|1|0|0| +5925|M|D|4 yr Degree|2500|Good|1|0|0| +5926|F|D|4 yr Degree|2500|Good|1|0|0| +5927|M|W|4 yr Degree|2500|Good|1|0|0| +5928|F|W|4 yr Degree|2500|Good|1|0|0| +5929|M|U|4 yr Degree|2500|Good|1|0|0| +5930|F|U|4 yr Degree|2500|Good|1|0|0| +5931|M|M|Advanced Degree|2500|Good|1|0|0| +5932|F|M|Advanced Degree|2500|Good|1|0|0| +5933|M|S|Advanced Degree|2500|Good|1|0|0| +5934|F|S|Advanced Degree|2500|Good|1|0|0| +5935|M|D|Advanced Degree|2500|Good|1|0|0| +5936|F|D|Advanced Degree|2500|Good|1|0|0| +5937|M|W|Advanced Degree|2500|Good|1|0|0| +5938|F|W|Advanced Degree|2500|Good|1|0|0| +5939|M|U|Advanced Degree|2500|Good|1|0|0| +5940|F|U|Advanced Degree|2500|Good|1|0|0| +5941|M|M|Unknown|2500|Good|1|0|0| +5942|F|M|Unknown|2500|Good|1|0|0| +5943|M|S|Unknown|2500|Good|1|0|0| +5944|F|S|Unknown|2500|Good|1|0|0| +5945|M|D|Unknown|2500|Good|1|0|0| +5946|F|D|Unknown|2500|Good|1|0|0| +5947|M|W|Unknown|2500|Good|1|0|0| +5948|F|W|Unknown|2500|Good|1|0|0| +5949|M|U|Unknown|2500|Good|1|0|0| +5950|F|U|Unknown|2500|Good|1|0|0| +5951|M|M|Primary|3000|Good|1|0|0| +5952|F|M|Primary|3000|Good|1|0|0| +5953|M|S|Primary|3000|Good|1|0|0| +5954|F|S|Primary|3000|Good|1|0|0| +5955|M|D|Primary|3000|Good|1|0|0| +5956|F|D|Primary|3000|Good|1|0|0| +5957|M|W|Primary|3000|Good|1|0|0| +5958|F|W|Primary|3000|Good|1|0|0| +5959|M|U|Primary|3000|Good|1|0|0| +5960|F|U|Primary|3000|Good|1|0|0| +5961|M|M|Secondary|3000|Good|1|0|0| +5962|F|M|Secondary|3000|Good|1|0|0| +5963|M|S|Secondary|3000|Good|1|0|0| +5964|F|S|Secondary|3000|Good|1|0|0| +5965|M|D|Secondary|3000|Good|1|0|0| +5966|F|D|Secondary|3000|Good|1|0|0| +5967|M|W|Secondary|3000|Good|1|0|0| +5968|F|W|Secondary|3000|Good|1|0|0| +5969|M|U|Secondary|3000|Good|1|0|0| +5970|F|U|Secondary|3000|Good|1|0|0| +5971|M|M|College|3000|Good|1|0|0| +5972|F|M|College|3000|Good|1|0|0| +5973|M|S|College|3000|Good|1|0|0| +5974|F|S|College|3000|Good|1|0|0| +5975|M|D|College|3000|Good|1|0|0| +5976|F|D|College|3000|Good|1|0|0| +5977|M|W|College|3000|Good|1|0|0| +5978|F|W|College|3000|Good|1|0|0| +5979|M|U|College|3000|Good|1|0|0| +5980|F|U|College|3000|Good|1|0|0| +5981|M|M|2 yr Degree|3000|Good|1|0|0| +5982|F|M|2 yr Degree|3000|Good|1|0|0| +5983|M|S|2 yr Degree|3000|Good|1|0|0| +5984|F|S|2 yr Degree|3000|Good|1|0|0| +5985|M|D|2 yr Degree|3000|Good|1|0|0| +5986|F|D|2 yr Degree|3000|Good|1|0|0| +5987|M|W|2 yr Degree|3000|Good|1|0|0| +5988|F|W|2 yr Degree|3000|Good|1|0|0| +5989|M|U|2 yr Degree|3000|Good|1|0|0| +5990|F|U|2 yr Degree|3000|Good|1|0|0| +5991|M|M|4 yr Degree|3000|Good|1|0|0| +5992|F|M|4 yr Degree|3000|Good|1|0|0| +5993|M|S|4 yr Degree|3000|Good|1|0|0| +5994|F|S|4 yr Degree|3000|Good|1|0|0| +5995|M|D|4 yr Degree|3000|Good|1|0|0| +5996|F|D|4 yr Degree|3000|Good|1|0|0| +5997|M|W|4 yr Degree|3000|Good|1|0|0| +5998|F|W|4 yr Degree|3000|Good|1|0|0| +5999|M|U|4 yr Degree|3000|Good|1|0|0| +6000|F|U|4 yr Degree|3000|Good|1|0|0| +6001|M|M|Advanced Degree|3000|Good|1|0|0| +6002|F|M|Advanced Degree|3000|Good|1|0|0| +6003|M|S|Advanced Degree|3000|Good|1|0|0| +6004|F|S|Advanced Degree|3000|Good|1|0|0| +6005|M|D|Advanced Degree|3000|Good|1|0|0| +6006|F|D|Advanced Degree|3000|Good|1|0|0| +6007|M|W|Advanced Degree|3000|Good|1|0|0| +6008|F|W|Advanced Degree|3000|Good|1|0|0| +6009|M|U|Advanced Degree|3000|Good|1|0|0| +6010|F|U|Advanced Degree|3000|Good|1|0|0| +6011|M|M|Unknown|3000|Good|1|0|0| +6012|F|M|Unknown|3000|Good|1|0|0| +6013|M|S|Unknown|3000|Good|1|0|0| +6014|F|S|Unknown|3000|Good|1|0|0| +6015|M|D|Unknown|3000|Good|1|0|0| +6016|F|D|Unknown|3000|Good|1|0|0| +6017|M|W|Unknown|3000|Good|1|0|0| +6018|F|W|Unknown|3000|Good|1|0|0| +6019|M|U|Unknown|3000|Good|1|0|0| +6020|F|U|Unknown|3000|Good|1|0|0| +6021|M|M|Primary|3500|Good|1|0|0| +6022|F|M|Primary|3500|Good|1|0|0| +6023|M|S|Primary|3500|Good|1|0|0| +6024|F|S|Primary|3500|Good|1|0|0| +6025|M|D|Primary|3500|Good|1|0|0| +6026|F|D|Primary|3500|Good|1|0|0| +6027|M|W|Primary|3500|Good|1|0|0| +6028|F|W|Primary|3500|Good|1|0|0| +6029|M|U|Primary|3500|Good|1|0|0| +6030|F|U|Primary|3500|Good|1|0|0| +6031|M|M|Secondary|3500|Good|1|0|0| +6032|F|M|Secondary|3500|Good|1|0|0| +6033|M|S|Secondary|3500|Good|1|0|0| +6034|F|S|Secondary|3500|Good|1|0|0| +6035|M|D|Secondary|3500|Good|1|0|0| +6036|F|D|Secondary|3500|Good|1|0|0| +6037|M|W|Secondary|3500|Good|1|0|0| +6038|F|W|Secondary|3500|Good|1|0|0| +6039|M|U|Secondary|3500|Good|1|0|0| +6040|F|U|Secondary|3500|Good|1|0|0| +6041|M|M|College|3500|Good|1|0|0| +6042|F|M|College|3500|Good|1|0|0| +6043|M|S|College|3500|Good|1|0|0| +6044|F|S|College|3500|Good|1|0|0| +6045|M|D|College|3500|Good|1|0|0| +6046|F|D|College|3500|Good|1|0|0| +6047|M|W|College|3500|Good|1|0|0| +6048|F|W|College|3500|Good|1|0|0| +6049|M|U|College|3500|Good|1|0|0| +6050|F|U|College|3500|Good|1|0|0| +6051|M|M|2 yr Degree|3500|Good|1|0|0| +6052|F|M|2 yr Degree|3500|Good|1|0|0| +6053|M|S|2 yr Degree|3500|Good|1|0|0| +6054|F|S|2 yr Degree|3500|Good|1|0|0| +6055|M|D|2 yr Degree|3500|Good|1|0|0| +6056|F|D|2 yr Degree|3500|Good|1|0|0| +6057|M|W|2 yr Degree|3500|Good|1|0|0| +6058|F|W|2 yr Degree|3500|Good|1|0|0| +6059|M|U|2 yr Degree|3500|Good|1|0|0| +6060|F|U|2 yr Degree|3500|Good|1|0|0| +6061|M|M|4 yr Degree|3500|Good|1|0|0| +6062|F|M|4 yr Degree|3500|Good|1|0|0| +6063|M|S|4 yr Degree|3500|Good|1|0|0| +6064|F|S|4 yr Degree|3500|Good|1|0|0| +6065|M|D|4 yr Degree|3500|Good|1|0|0| +6066|F|D|4 yr Degree|3500|Good|1|0|0| +6067|M|W|4 yr Degree|3500|Good|1|0|0| +6068|F|W|4 yr Degree|3500|Good|1|0|0| +6069|M|U|4 yr Degree|3500|Good|1|0|0| +6070|F|U|4 yr Degree|3500|Good|1|0|0| +6071|M|M|Advanced Degree|3500|Good|1|0|0| +6072|F|M|Advanced Degree|3500|Good|1|0|0| +6073|M|S|Advanced Degree|3500|Good|1|0|0| +6074|F|S|Advanced Degree|3500|Good|1|0|0| +6075|M|D|Advanced Degree|3500|Good|1|0|0| +6076|F|D|Advanced Degree|3500|Good|1|0|0| +6077|M|W|Advanced Degree|3500|Good|1|0|0| +6078|F|W|Advanced Degree|3500|Good|1|0|0| +6079|M|U|Advanced Degree|3500|Good|1|0|0| +6080|F|U|Advanced Degree|3500|Good|1|0|0| +6081|M|M|Unknown|3500|Good|1|0|0| +6082|F|M|Unknown|3500|Good|1|0|0| +6083|M|S|Unknown|3500|Good|1|0|0| +6084|F|S|Unknown|3500|Good|1|0|0| +6085|M|D|Unknown|3500|Good|1|0|0| +6086|F|D|Unknown|3500|Good|1|0|0| +6087|M|W|Unknown|3500|Good|1|0|0| +6088|F|W|Unknown|3500|Good|1|0|0| +6089|M|U|Unknown|3500|Good|1|0|0| +6090|F|U|Unknown|3500|Good|1|0|0| +6091|M|M|Primary|4000|Good|1|0|0| +6092|F|M|Primary|4000|Good|1|0|0| +6093|M|S|Primary|4000|Good|1|0|0| +6094|F|S|Primary|4000|Good|1|0|0| +6095|M|D|Primary|4000|Good|1|0|0| +6096|F|D|Primary|4000|Good|1|0|0| +6097|M|W|Primary|4000|Good|1|0|0| +6098|F|W|Primary|4000|Good|1|0|0| +6099|M|U|Primary|4000|Good|1|0|0| +6100|F|U|Primary|4000|Good|1|0|0| +6101|M|M|Secondary|4000|Good|1|0|0| +6102|F|M|Secondary|4000|Good|1|0|0| +6103|M|S|Secondary|4000|Good|1|0|0| +6104|F|S|Secondary|4000|Good|1|0|0| +6105|M|D|Secondary|4000|Good|1|0|0| +6106|F|D|Secondary|4000|Good|1|0|0| +6107|M|W|Secondary|4000|Good|1|0|0| +6108|F|W|Secondary|4000|Good|1|0|0| +6109|M|U|Secondary|4000|Good|1|0|0| +6110|F|U|Secondary|4000|Good|1|0|0| +6111|M|M|College|4000|Good|1|0|0| +6112|F|M|College|4000|Good|1|0|0| +6113|M|S|College|4000|Good|1|0|0| +6114|F|S|College|4000|Good|1|0|0| +6115|M|D|College|4000|Good|1|0|0| +6116|F|D|College|4000|Good|1|0|0| +6117|M|W|College|4000|Good|1|0|0| +6118|F|W|College|4000|Good|1|0|0| +6119|M|U|College|4000|Good|1|0|0| +6120|F|U|College|4000|Good|1|0|0| +6121|M|M|2 yr Degree|4000|Good|1|0|0| +6122|F|M|2 yr Degree|4000|Good|1|0|0| +6123|M|S|2 yr Degree|4000|Good|1|0|0| +6124|F|S|2 yr Degree|4000|Good|1|0|0| +6125|M|D|2 yr Degree|4000|Good|1|0|0| +6126|F|D|2 yr Degree|4000|Good|1|0|0| +6127|M|W|2 yr Degree|4000|Good|1|0|0| +6128|F|W|2 yr Degree|4000|Good|1|0|0| +6129|M|U|2 yr Degree|4000|Good|1|0|0| +6130|F|U|2 yr Degree|4000|Good|1|0|0| +6131|M|M|4 yr Degree|4000|Good|1|0|0| +6132|F|M|4 yr Degree|4000|Good|1|0|0| +6133|M|S|4 yr Degree|4000|Good|1|0|0| +6134|F|S|4 yr Degree|4000|Good|1|0|0| +6135|M|D|4 yr Degree|4000|Good|1|0|0| +6136|F|D|4 yr Degree|4000|Good|1|0|0| +6137|M|W|4 yr Degree|4000|Good|1|0|0| +6138|F|W|4 yr Degree|4000|Good|1|0|0| +6139|M|U|4 yr Degree|4000|Good|1|0|0| +6140|F|U|4 yr Degree|4000|Good|1|0|0| +6141|M|M|Advanced Degree|4000|Good|1|0|0| +6142|F|M|Advanced Degree|4000|Good|1|0|0| +6143|M|S|Advanced Degree|4000|Good|1|0|0| +6144|F|S|Advanced Degree|4000|Good|1|0|0| +6145|M|D|Advanced Degree|4000|Good|1|0|0| +6146|F|D|Advanced Degree|4000|Good|1|0|0| +6147|M|W|Advanced Degree|4000|Good|1|0|0| +6148|F|W|Advanced Degree|4000|Good|1|0|0| +6149|M|U|Advanced Degree|4000|Good|1|0|0| +6150|F|U|Advanced Degree|4000|Good|1|0|0| +6151|M|M|Unknown|4000|Good|1|0|0| +6152|F|M|Unknown|4000|Good|1|0|0| +6153|M|S|Unknown|4000|Good|1|0|0| +6154|F|S|Unknown|4000|Good|1|0|0| +6155|M|D|Unknown|4000|Good|1|0|0| +6156|F|D|Unknown|4000|Good|1|0|0| +6157|M|W|Unknown|4000|Good|1|0|0| +6158|F|W|Unknown|4000|Good|1|0|0| +6159|M|U|Unknown|4000|Good|1|0|0| +6160|F|U|Unknown|4000|Good|1|0|0| +6161|M|M|Primary|4500|Good|1|0|0| +6162|F|M|Primary|4500|Good|1|0|0| +6163|M|S|Primary|4500|Good|1|0|0| +6164|F|S|Primary|4500|Good|1|0|0| +6165|M|D|Primary|4500|Good|1|0|0| +6166|F|D|Primary|4500|Good|1|0|0| +6167|M|W|Primary|4500|Good|1|0|0| +6168|F|W|Primary|4500|Good|1|0|0| +6169|M|U|Primary|4500|Good|1|0|0| +6170|F|U|Primary|4500|Good|1|0|0| +6171|M|M|Secondary|4500|Good|1|0|0| +6172|F|M|Secondary|4500|Good|1|0|0| +6173|M|S|Secondary|4500|Good|1|0|0| +6174|F|S|Secondary|4500|Good|1|0|0| +6175|M|D|Secondary|4500|Good|1|0|0| +6176|F|D|Secondary|4500|Good|1|0|0| +6177|M|W|Secondary|4500|Good|1|0|0| +6178|F|W|Secondary|4500|Good|1|0|0| +6179|M|U|Secondary|4500|Good|1|0|0| +6180|F|U|Secondary|4500|Good|1|0|0| +6181|M|M|College|4500|Good|1|0|0| +6182|F|M|College|4500|Good|1|0|0| +6183|M|S|College|4500|Good|1|0|0| +6184|F|S|College|4500|Good|1|0|0| +6185|M|D|College|4500|Good|1|0|0| +6186|F|D|College|4500|Good|1|0|0| +6187|M|W|College|4500|Good|1|0|0| +6188|F|W|College|4500|Good|1|0|0| +6189|M|U|College|4500|Good|1|0|0| +6190|F|U|College|4500|Good|1|0|0| +6191|M|M|2 yr Degree|4500|Good|1|0|0| +6192|F|M|2 yr Degree|4500|Good|1|0|0| +6193|M|S|2 yr Degree|4500|Good|1|0|0| +6194|F|S|2 yr Degree|4500|Good|1|0|0| +6195|M|D|2 yr Degree|4500|Good|1|0|0| +6196|F|D|2 yr Degree|4500|Good|1|0|0| +6197|M|W|2 yr Degree|4500|Good|1|0|0| +6198|F|W|2 yr Degree|4500|Good|1|0|0| +6199|M|U|2 yr Degree|4500|Good|1|0|0| +6200|F|U|2 yr Degree|4500|Good|1|0|0| +6201|M|M|4 yr Degree|4500|Good|1|0|0| +6202|F|M|4 yr Degree|4500|Good|1|0|0| +6203|M|S|4 yr Degree|4500|Good|1|0|0| +6204|F|S|4 yr Degree|4500|Good|1|0|0| +6205|M|D|4 yr Degree|4500|Good|1|0|0| +6206|F|D|4 yr Degree|4500|Good|1|0|0| +6207|M|W|4 yr Degree|4500|Good|1|0|0| +6208|F|W|4 yr Degree|4500|Good|1|0|0| +6209|M|U|4 yr Degree|4500|Good|1|0|0| +6210|F|U|4 yr Degree|4500|Good|1|0|0| +6211|M|M|Advanced Degree|4500|Good|1|0|0| +6212|F|M|Advanced Degree|4500|Good|1|0|0| +6213|M|S|Advanced Degree|4500|Good|1|0|0| +6214|F|S|Advanced Degree|4500|Good|1|0|0| +6215|M|D|Advanced Degree|4500|Good|1|0|0| +6216|F|D|Advanced Degree|4500|Good|1|0|0| +6217|M|W|Advanced Degree|4500|Good|1|0|0| +6218|F|W|Advanced Degree|4500|Good|1|0|0| +6219|M|U|Advanced Degree|4500|Good|1|0|0| +6220|F|U|Advanced Degree|4500|Good|1|0|0| +6221|M|M|Unknown|4500|Good|1|0|0| +6222|F|M|Unknown|4500|Good|1|0|0| +6223|M|S|Unknown|4500|Good|1|0|0| +6224|F|S|Unknown|4500|Good|1|0|0| +6225|M|D|Unknown|4500|Good|1|0|0| +6226|F|D|Unknown|4500|Good|1|0|0| +6227|M|W|Unknown|4500|Good|1|0|0| +6228|F|W|Unknown|4500|Good|1|0|0| +6229|M|U|Unknown|4500|Good|1|0|0| +6230|F|U|Unknown|4500|Good|1|0|0| +6231|M|M|Primary|5000|Good|1|0|0| +6232|F|M|Primary|5000|Good|1|0|0| +6233|M|S|Primary|5000|Good|1|0|0| +6234|F|S|Primary|5000|Good|1|0|0| +6235|M|D|Primary|5000|Good|1|0|0| +6236|F|D|Primary|5000|Good|1|0|0| +6237|M|W|Primary|5000|Good|1|0|0| +6238|F|W|Primary|5000|Good|1|0|0| +6239|M|U|Primary|5000|Good|1|0|0| +6240|F|U|Primary|5000|Good|1|0|0| +6241|M|M|Secondary|5000|Good|1|0|0| +6242|F|M|Secondary|5000|Good|1|0|0| +6243|M|S|Secondary|5000|Good|1|0|0| +6244|F|S|Secondary|5000|Good|1|0|0| +6245|M|D|Secondary|5000|Good|1|0|0| +6246|F|D|Secondary|5000|Good|1|0|0| +6247|M|W|Secondary|5000|Good|1|0|0| +6248|F|W|Secondary|5000|Good|1|0|0| +6249|M|U|Secondary|5000|Good|1|0|0| +6250|F|U|Secondary|5000|Good|1|0|0| +6251|M|M|College|5000|Good|1|0|0| +6252|F|M|College|5000|Good|1|0|0| +6253|M|S|College|5000|Good|1|0|0| +6254|F|S|College|5000|Good|1|0|0| +6255|M|D|College|5000|Good|1|0|0| +6256|F|D|College|5000|Good|1|0|0| +6257|M|W|College|5000|Good|1|0|0| +6258|F|W|College|5000|Good|1|0|0| +6259|M|U|College|5000|Good|1|0|0| +6260|F|U|College|5000|Good|1|0|0| +6261|M|M|2 yr Degree|5000|Good|1|0|0| +6262|F|M|2 yr Degree|5000|Good|1|0|0| +6263|M|S|2 yr Degree|5000|Good|1|0|0| +6264|F|S|2 yr Degree|5000|Good|1|0|0| +6265|M|D|2 yr Degree|5000|Good|1|0|0| +6266|F|D|2 yr Degree|5000|Good|1|0|0| +6267|M|W|2 yr Degree|5000|Good|1|0|0| +6268|F|W|2 yr Degree|5000|Good|1|0|0| +6269|M|U|2 yr Degree|5000|Good|1|0|0| +6270|F|U|2 yr Degree|5000|Good|1|0|0| +6271|M|M|4 yr Degree|5000|Good|1|0|0| +6272|F|M|4 yr Degree|5000|Good|1|0|0| +6273|M|S|4 yr Degree|5000|Good|1|0|0| +6274|F|S|4 yr Degree|5000|Good|1|0|0| +6275|M|D|4 yr Degree|5000|Good|1|0|0| +6276|F|D|4 yr Degree|5000|Good|1|0|0| +6277|M|W|4 yr Degree|5000|Good|1|0|0| +6278|F|W|4 yr Degree|5000|Good|1|0|0| +6279|M|U|4 yr Degree|5000|Good|1|0|0| +6280|F|U|4 yr Degree|5000|Good|1|0|0| +6281|M|M|Advanced Degree|5000|Good|1|0|0| +6282|F|M|Advanced Degree|5000|Good|1|0|0| +6283|M|S|Advanced Degree|5000|Good|1|0|0| +6284|F|S|Advanced Degree|5000|Good|1|0|0| +6285|M|D|Advanced Degree|5000|Good|1|0|0| +6286|F|D|Advanced Degree|5000|Good|1|0|0| +6287|M|W|Advanced Degree|5000|Good|1|0|0| +6288|F|W|Advanced Degree|5000|Good|1|0|0| +6289|M|U|Advanced Degree|5000|Good|1|0|0| +6290|F|U|Advanced Degree|5000|Good|1|0|0| +6291|M|M|Unknown|5000|Good|1|0|0| +6292|F|M|Unknown|5000|Good|1|0|0| +6293|M|S|Unknown|5000|Good|1|0|0| +6294|F|S|Unknown|5000|Good|1|0|0| +6295|M|D|Unknown|5000|Good|1|0|0| +6296|F|D|Unknown|5000|Good|1|0|0| +6297|M|W|Unknown|5000|Good|1|0|0| +6298|F|W|Unknown|5000|Good|1|0|0| +6299|M|U|Unknown|5000|Good|1|0|0| +6300|F|U|Unknown|5000|Good|1|0|0| +6301|M|M|Primary|5500|Good|1|0|0| +6302|F|M|Primary|5500|Good|1|0|0| +6303|M|S|Primary|5500|Good|1|0|0| +6304|F|S|Primary|5500|Good|1|0|0| +6305|M|D|Primary|5500|Good|1|0|0| +6306|F|D|Primary|5500|Good|1|0|0| +6307|M|W|Primary|5500|Good|1|0|0| +6308|F|W|Primary|5500|Good|1|0|0| +6309|M|U|Primary|5500|Good|1|0|0| +6310|F|U|Primary|5500|Good|1|0|0| +6311|M|M|Secondary|5500|Good|1|0|0| +6312|F|M|Secondary|5500|Good|1|0|0| +6313|M|S|Secondary|5500|Good|1|0|0| +6314|F|S|Secondary|5500|Good|1|0|0| +6315|M|D|Secondary|5500|Good|1|0|0| +6316|F|D|Secondary|5500|Good|1|0|0| +6317|M|W|Secondary|5500|Good|1|0|0| +6318|F|W|Secondary|5500|Good|1|0|0| +6319|M|U|Secondary|5500|Good|1|0|0| +6320|F|U|Secondary|5500|Good|1|0|0| +6321|M|M|College|5500|Good|1|0|0| +6322|F|M|College|5500|Good|1|0|0| +6323|M|S|College|5500|Good|1|0|0| +6324|F|S|College|5500|Good|1|0|0| +6325|M|D|College|5500|Good|1|0|0| +6326|F|D|College|5500|Good|1|0|0| +6327|M|W|College|5500|Good|1|0|0| +6328|F|W|College|5500|Good|1|0|0| +6329|M|U|College|5500|Good|1|0|0| +6330|F|U|College|5500|Good|1|0|0| +6331|M|M|2 yr Degree|5500|Good|1|0|0| +6332|F|M|2 yr Degree|5500|Good|1|0|0| +6333|M|S|2 yr Degree|5500|Good|1|0|0| +6334|F|S|2 yr Degree|5500|Good|1|0|0| +6335|M|D|2 yr Degree|5500|Good|1|0|0| +6336|F|D|2 yr Degree|5500|Good|1|0|0| +6337|M|W|2 yr Degree|5500|Good|1|0|0| +6338|F|W|2 yr Degree|5500|Good|1|0|0| +6339|M|U|2 yr Degree|5500|Good|1|0|0| +6340|F|U|2 yr Degree|5500|Good|1|0|0| +6341|M|M|4 yr Degree|5500|Good|1|0|0| +6342|F|M|4 yr Degree|5500|Good|1|0|0| +6343|M|S|4 yr Degree|5500|Good|1|0|0| +6344|F|S|4 yr Degree|5500|Good|1|0|0| +6345|M|D|4 yr Degree|5500|Good|1|0|0| +6346|F|D|4 yr Degree|5500|Good|1|0|0| +6347|M|W|4 yr Degree|5500|Good|1|0|0| +6348|F|W|4 yr Degree|5500|Good|1|0|0| +6349|M|U|4 yr Degree|5500|Good|1|0|0| +6350|F|U|4 yr Degree|5500|Good|1|0|0| +6351|M|M|Advanced Degree|5500|Good|1|0|0| +6352|F|M|Advanced Degree|5500|Good|1|0|0| +6353|M|S|Advanced Degree|5500|Good|1|0|0| +6354|F|S|Advanced Degree|5500|Good|1|0|0| +6355|M|D|Advanced Degree|5500|Good|1|0|0| +6356|F|D|Advanced Degree|5500|Good|1|0|0| +6357|M|W|Advanced Degree|5500|Good|1|0|0| +6358|F|W|Advanced Degree|5500|Good|1|0|0| +6359|M|U|Advanced Degree|5500|Good|1|0|0| +6360|F|U|Advanced Degree|5500|Good|1|0|0| +6361|M|M|Unknown|5500|Good|1|0|0| +6362|F|M|Unknown|5500|Good|1|0|0| +6363|M|S|Unknown|5500|Good|1|0|0| +6364|F|S|Unknown|5500|Good|1|0|0| +6365|M|D|Unknown|5500|Good|1|0|0| +6366|F|D|Unknown|5500|Good|1|0|0| +6367|M|W|Unknown|5500|Good|1|0|0| +6368|F|W|Unknown|5500|Good|1|0|0| +6369|M|U|Unknown|5500|Good|1|0|0| +6370|F|U|Unknown|5500|Good|1|0|0| +6371|M|M|Primary|6000|Good|1|0|0| +6372|F|M|Primary|6000|Good|1|0|0| +6373|M|S|Primary|6000|Good|1|0|0| +6374|F|S|Primary|6000|Good|1|0|0| +6375|M|D|Primary|6000|Good|1|0|0| +6376|F|D|Primary|6000|Good|1|0|0| +6377|M|W|Primary|6000|Good|1|0|0| +6378|F|W|Primary|6000|Good|1|0|0| +6379|M|U|Primary|6000|Good|1|0|0| +6380|F|U|Primary|6000|Good|1|0|0| +6381|M|M|Secondary|6000|Good|1|0|0| +6382|F|M|Secondary|6000|Good|1|0|0| +6383|M|S|Secondary|6000|Good|1|0|0| +6384|F|S|Secondary|6000|Good|1|0|0| +6385|M|D|Secondary|6000|Good|1|0|0| +6386|F|D|Secondary|6000|Good|1|0|0| +6387|M|W|Secondary|6000|Good|1|0|0| +6388|F|W|Secondary|6000|Good|1|0|0| +6389|M|U|Secondary|6000|Good|1|0|0| +6390|F|U|Secondary|6000|Good|1|0|0| +6391|M|M|College|6000|Good|1|0|0| +6392|F|M|College|6000|Good|1|0|0| +6393|M|S|College|6000|Good|1|0|0| +6394|F|S|College|6000|Good|1|0|0| +6395|M|D|College|6000|Good|1|0|0| +6396|F|D|College|6000|Good|1|0|0| +6397|M|W|College|6000|Good|1|0|0| +6398|F|W|College|6000|Good|1|0|0| +6399|M|U|College|6000|Good|1|0|0| +6400|F|U|College|6000|Good|1|0|0| +6401|M|M|2 yr Degree|6000|Good|1|0|0| +6402|F|M|2 yr Degree|6000|Good|1|0|0| +6403|M|S|2 yr Degree|6000|Good|1|0|0| +6404|F|S|2 yr Degree|6000|Good|1|0|0| +6405|M|D|2 yr Degree|6000|Good|1|0|0| +6406|F|D|2 yr Degree|6000|Good|1|0|0| +6407|M|W|2 yr Degree|6000|Good|1|0|0| +6408|F|W|2 yr Degree|6000|Good|1|0|0| +6409|M|U|2 yr Degree|6000|Good|1|0|0| +6410|F|U|2 yr Degree|6000|Good|1|0|0| +6411|M|M|4 yr Degree|6000|Good|1|0|0| +6412|F|M|4 yr Degree|6000|Good|1|0|0| +6413|M|S|4 yr Degree|6000|Good|1|0|0| +6414|F|S|4 yr Degree|6000|Good|1|0|0| +6415|M|D|4 yr Degree|6000|Good|1|0|0| +6416|F|D|4 yr Degree|6000|Good|1|0|0| +6417|M|W|4 yr Degree|6000|Good|1|0|0| +6418|F|W|4 yr Degree|6000|Good|1|0|0| +6419|M|U|4 yr Degree|6000|Good|1|0|0| +6420|F|U|4 yr Degree|6000|Good|1|0|0| +6421|M|M|Advanced Degree|6000|Good|1|0|0| +6422|F|M|Advanced Degree|6000|Good|1|0|0| +6423|M|S|Advanced Degree|6000|Good|1|0|0| +6424|F|S|Advanced Degree|6000|Good|1|0|0| +6425|M|D|Advanced Degree|6000|Good|1|0|0| +6426|F|D|Advanced Degree|6000|Good|1|0|0| +6427|M|W|Advanced Degree|6000|Good|1|0|0| +6428|F|W|Advanced Degree|6000|Good|1|0|0| +6429|M|U|Advanced Degree|6000|Good|1|0|0| +6430|F|U|Advanced Degree|6000|Good|1|0|0| +6431|M|M|Unknown|6000|Good|1|0|0| +6432|F|M|Unknown|6000|Good|1|0|0| +6433|M|S|Unknown|6000|Good|1|0|0| +6434|F|S|Unknown|6000|Good|1|0|0| +6435|M|D|Unknown|6000|Good|1|0|0| +6436|F|D|Unknown|6000|Good|1|0|0| +6437|M|W|Unknown|6000|Good|1|0|0| +6438|F|W|Unknown|6000|Good|1|0|0| +6439|M|U|Unknown|6000|Good|1|0|0| +6440|F|U|Unknown|6000|Good|1|0|0| +6441|M|M|Primary|6500|Good|1|0|0| +6442|F|M|Primary|6500|Good|1|0|0| +6443|M|S|Primary|6500|Good|1|0|0| +6444|F|S|Primary|6500|Good|1|0|0| +6445|M|D|Primary|6500|Good|1|0|0| +6446|F|D|Primary|6500|Good|1|0|0| +6447|M|W|Primary|6500|Good|1|0|0| +6448|F|W|Primary|6500|Good|1|0|0| +6449|M|U|Primary|6500|Good|1|0|0| +6450|F|U|Primary|6500|Good|1|0|0| +6451|M|M|Secondary|6500|Good|1|0|0| +6452|F|M|Secondary|6500|Good|1|0|0| +6453|M|S|Secondary|6500|Good|1|0|0| +6454|F|S|Secondary|6500|Good|1|0|0| +6455|M|D|Secondary|6500|Good|1|0|0| +6456|F|D|Secondary|6500|Good|1|0|0| +6457|M|W|Secondary|6500|Good|1|0|0| +6458|F|W|Secondary|6500|Good|1|0|0| +6459|M|U|Secondary|6500|Good|1|0|0| +6460|F|U|Secondary|6500|Good|1|0|0| +6461|M|M|College|6500|Good|1|0|0| +6462|F|M|College|6500|Good|1|0|0| +6463|M|S|College|6500|Good|1|0|0| +6464|F|S|College|6500|Good|1|0|0| +6465|M|D|College|6500|Good|1|0|0| +6466|F|D|College|6500|Good|1|0|0| +6467|M|W|College|6500|Good|1|0|0| +6468|F|W|College|6500|Good|1|0|0| +6469|M|U|College|6500|Good|1|0|0| +6470|F|U|College|6500|Good|1|0|0| +6471|M|M|2 yr Degree|6500|Good|1|0|0| +6472|F|M|2 yr Degree|6500|Good|1|0|0| +6473|M|S|2 yr Degree|6500|Good|1|0|0| +6474|F|S|2 yr Degree|6500|Good|1|0|0| +6475|M|D|2 yr Degree|6500|Good|1|0|0| +6476|F|D|2 yr Degree|6500|Good|1|0|0| +6477|M|W|2 yr Degree|6500|Good|1|0|0| +6478|F|W|2 yr Degree|6500|Good|1|0|0| +6479|M|U|2 yr Degree|6500|Good|1|0|0| +6480|F|U|2 yr Degree|6500|Good|1|0|0| +6481|M|M|4 yr Degree|6500|Good|1|0|0| +6482|F|M|4 yr Degree|6500|Good|1|0|0| +6483|M|S|4 yr Degree|6500|Good|1|0|0| +6484|F|S|4 yr Degree|6500|Good|1|0|0| +6485|M|D|4 yr Degree|6500|Good|1|0|0| +6486|F|D|4 yr Degree|6500|Good|1|0|0| +6487|M|W|4 yr Degree|6500|Good|1|0|0| +6488|F|W|4 yr Degree|6500|Good|1|0|0| +6489|M|U|4 yr Degree|6500|Good|1|0|0| +6490|F|U|4 yr Degree|6500|Good|1|0|0| +6491|M|M|Advanced Degree|6500|Good|1|0|0| +6492|F|M|Advanced Degree|6500|Good|1|0|0| +6493|M|S|Advanced Degree|6500|Good|1|0|0| +6494|F|S|Advanced Degree|6500|Good|1|0|0| +6495|M|D|Advanced Degree|6500|Good|1|0|0| +6496|F|D|Advanced Degree|6500|Good|1|0|0| +6497|M|W|Advanced Degree|6500|Good|1|0|0| +6498|F|W|Advanced Degree|6500|Good|1|0|0| +6499|M|U|Advanced Degree|6500|Good|1|0|0| +6500|F|U|Advanced Degree|6500|Good|1|0|0| +6501|M|M|Unknown|6500|Good|1|0|0| +6502|F|M|Unknown|6500|Good|1|0|0| +6503|M|S|Unknown|6500|Good|1|0|0| +6504|F|S|Unknown|6500|Good|1|0|0| +6505|M|D|Unknown|6500|Good|1|0|0| +6506|F|D|Unknown|6500|Good|1|0|0| +6507|M|W|Unknown|6500|Good|1|0|0| +6508|F|W|Unknown|6500|Good|1|0|0| +6509|M|U|Unknown|6500|Good|1|0|0| +6510|F|U|Unknown|6500|Good|1|0|0| +6511|M|M|Primary|7000|Good|1|0|0| +6512|F|M|Primary|7000|Good|1|0|0| +6513|M|S|Primary|7000|Good|1|0|0| +6514|F|S|Primary|7000|Good|1|0|0| +6515|M|D|Primary|7000|Good|1|0|0| +6516|F|D|Primary|7000|Good|1|0|0| +6517|M|W|Primary|7000|Good|1|0|0| +6518|F|W|Primary|7000|Good|1|0|0| +6519|M|U|Primary|7000|Good|1|0|0| +6520|F|U|Primary|7000|Good|1|0|0| +6521|M|M|Secondary|7000|Good|1|0|0| +6522|F|M|Secondary|7000|Good|1|0|0| +6523|M|S|Secondary|7000|Good|1|0|0| +6524|F|S|Secondary|7000|Good|1|0|0| +6525|M|D|Secondary|7000|Good|1|0|0| +6526|F|D|Secondary|7000|Good|1|0|0| +6527|M|W|Secondary|7000|Good|1|0|0| +6528|F|W|Secondary|7000|Good|1|0|0| +6529|M|U|Secondary|7000|Good|1|0|0| +6530|F|U|Secondary|7000|Good|1|0|0| +6531|M|M|College|7000|Good|1|0|0| +6532|F|M|College|7000|Good|1|0|0| +6533|M|S|College|7000|Good|1|0|0| +6534|F|S|College|7000|Good|1|0|0| +6535|M|D|College|7000|Good|1|0|0| +6536|F|D|College|7000|Good|1|0|0| +6537|M|W|College|7000|Good|1|0|0| +6538|F|W|College|7000|Good|1|0|0| +6539|M|U|College|7000|Good|1|0|0| +6540|F|U|College|7000|Good|1|0|0| +6541|M|M|2 yr Degree|7000|Good|1|0|0| +6542|F|M|2 yr Degree|7000|Good|1|0|0| +6543|M|S|2 yr Degree|7000|Good|1|0|0| +6544|F|S|2 yr Degree|7000|Good|1|0|0| +6545|M|D|2 yr Degree|7000|Good|1|0|0| +6546|F|D|2 yr Degree|7000|Good|1|0|0| +6547|M|W|2 yr Degree|7000|Good|1|0|0| +6548|F|W|2 yr Degree|7000|Good|1|0|0| +6549|M|U|2 yr Degree|7000|Good|1|0|0| +6550|F|U|2 yr Degree|7000|Good|1|0|0| +6551|M|M|4 yr Degree|7000|Good|1|0|0| +6552|F|M|4 yr Degree|7000|Good|1|0|0| +6553|M|S|4 yr Degree|7000|Good|1|0|0| +6554|F|S|4 yr Degree|7000|Good|1|0|0| +6555|M|D|4 yr Degree|7000|Good|1|0|0| +6556|F|D|4 yr Degree|7000|Good|1|0|0| +6557|M|W|4 yr Degree|7000|Good|1|0|0| +6558|F|W|4 yr Degree|7000|Good|1|0|0| +6559|M|U|4 yr Degree|7000|Good|1|0|0| +6560|F|U|4 yr Degree|7000|Good|1|0|0| +6561|M|M|Advanced Degree|7000|Good|1|0|0| +6562|F|M|Advanced Degree|7000|Good|1|0|0| +6563|M|S|Advanced Degree|7000|Good|1|0|0| +6564|F|S|Advanced Degree|7000|Good|1|0|0| +6565|M|D|Advanced Degree|7000|Good|1|0|0| +6566|F|D|Advanced Degree|7000|Good|1|0|0| +6567|M|W|Advanced Degree|7000|Good|1|0|0| +6568|F|W|Advanced Degree|7000|Good|1|0|0| +6569|M|U|Advanced Degree|7000|Good|1|0|0| +6570|F|U|Advanced Degree|7000|Good|1|0|0| +6571|M|M|Unknown|7000|Good|1|0|0| +6572|F|M|Unknown|7000|Good|1|0|0| +6573|M|S|Unknown|7000|Good|1|0|0| +6574|F|S|Unknown|7000|Good|1|0|0| +6575|M|D|Unknown|7000|Good|1|0|0| +6576|F|D|Unknown|7000|Good|1|0|0| +6577|M|W|Unknown|7000|Good|1|0|0| +6578|F|W|Unknown|7000|Good|1|0|0| +6579|M|U|Unknown|7000|Good|1|0|0| +6580|F|U|Unknown|7000|Good|1|0|0| +6581|M|M|Primary|7500|Good|1|0|0| +6582|F|M|Primary|7500|Good|1|0|0| +6583|M|S|Primary|7500|Good|1|0|0| +6584|F|S|Primary|7500|Good|1|0|0| +6585|M|D|Primary|7500|Good|1|0|0| +6586|F|D|Primary|7500|Good|1|0|0| +6587|M|W|Primary|7500|Good|1|0|0| +6588|F|W|Primary|7500|Good|1|0|0| +6589|M|U|Primary|7500|Good|1|0|0| +6590|F|U|Primary|7500|Good|1|0|0| +6591|M|M|Secondary|7500|Good|1|0|0| +6592|F|M|Secondary|7500|Good|1|0|0| +6593|M|S|Secondary|7500|Good|1|0|0| +6594|F|S|Secondary|7500|Good|1|0|0| +6595|M|D|Secondary|7500|Good|1|0|0| +6596|F|D|Secondary|7500|Good|1|0|0| +6597|M|W|Secondary|7500|Good|1|0|0| +6598|F|W|Secondary|7500|Good|1|0|0| +6599|M|U|Secondary|7500|Good|1|0|0| +6600|F|U|Secondary|7500|Good|1|0|0| +6601|M|M|College|7500|Good|1|0|0| +6602|F|M|College|7500|Good|1|0|0| +6603|M|S|College|7500|Good|1|0|0| +6604|F|S|College|7500|Good|1|0|0| +6605|M|D|College|7500|Good|1|0|0| +6606|F|D|College|7500|Good|1|0|0| +6607|M|W|College|7500|Good|1|0|0| +6608|F|W|College|7500|Good|1|0|0| +6609|M|U|College|7500|Good|1|0|0| +6610|F|U|College|7500|Good|1|0|0| +6611|M|M|2 yr Degree|7500|Good|1|0|0| +6612|F|M|2 yr Degree|7500|Good|1|0|0| +6613|M|S|2 yr Degree|7500|Good|1|0|0| +6614|F|S|2 yr Degree|7500|Good|1|0|0| +6615|M|D|2 yr Degree|7500|Good|1|0|0| +6616|F|D|2 yr Degree|7500|Good|1|0|0| +6617|M|W|2 yr Degree|7500|Good|1|0|0| +6618|F|W|2 yr Degree|7500|Good|1|0|0| +6619|M|U|2 yr Degree|7500|Good|1|0|0| +6620|F|U|2 yr Degree|7500|Good|1|0|0| +6621|M|M|4 yr Degree|7500|Good|1|0|0| +6622|F|M|4 yr Degree|7500|Good|1|0|0| +6623|M|S|4 yr Degree|7500|Good|1|0|0| +6624|F|S|4 yr Degree|7500|Good|1|0|0| +6625|M|D|4 yr Degree|7500|Good|1|0|0| +6626|F|D|4 yr Degree|7500|Good|1|0|0| +6627|M|W|4 yr Degree|7500|Good|1|0|0| +6628|F|W|4 yr Degree|7500|Good|1|0|0| +6629|M|U|4 yr Degree|7500|Good|1|0|0| +6630|F|U|4 yr Degree|7500|Good|1|0|0| +6631|M|M|Advanced Degree|7500|Good|1|0|0| +6632|F|M|Advanced Degree|7500|Good|1|0|0| +6633|M|S|Advanced Degree|7500|Good|1|0|0| +6634|F|S|Advanced Degree|7500|Good|1|0|0| +6635|M|D|Advanced Degree|7500|Good|1|0|0| +6636|F|D|Advanced Degree|7500|Good|1|0|0| +6637|M|W|Advanced Degree|7500|Good|1|0|0| +6638|F|W|Advanced Degree|7500|Good|1|0|0| +6639|M|U|Advanced Degree|7500|Good|1|0|0| +6640|F|U|Advanced Degree|7500|Good|1|0|0| +6641|M|M|Unknown|7500|Good|1|0|0| +6642|F|M|Unknown|7500|Good|1|0|0| +6643|M|S|Unknown|7500|Good|1|0|0| +6644|F|S|Unknown|7500|Good|1|0|0| +6645|M|D|Unknown|7500|Good|1|0|0| +6646|F|D|Unknown|7500|Good|1|0|0| +6647|M|W|Unknown|7500|Good|1|0|0| +6648|F|W|Unknown|7500|Good|1|0|0| +6649|M|U|Unknown|7500|Good|1|0|0| +6650|F|U|Unknown|7500|Good|1|0|0| +6651|M|M|Primary|8000|Good|1|0|0| +6652|F|M|Primary|8000|Good|1|0|0| +6653|M|S|Primary|8000|Good|1|0|0| +6654|F|S|Primary|8000|Good|1|0|0| +6655|M|D|Primary|8000|Good|1|0|0| +6656|F|D|Primary|8000|Good|1|0|0| +6657|M|W|Primary|8000|Good|1|0|0| +6658|F|W|Primary|8000|Good|1|0|0| +6659|M|U|Primary|8000|Good|1|0|0| +6660|F|U|Primary|8000|Good|1|0|0| +6661|M|M|Secondary|8000|Good|1|0|0| +6662|F|M|Secondary|8000|Good|1|0|0| +6663|M|S|Secondary|8000|Good|1|0|0| +6664|F|S|Secondary|8000|Good|1|0|0| +6665|M|D|Secondary|8000|Good|1|0|0| +6666|F|D|Secondary|8000|Good|1|0|0| +6667|M|W|Secondary|8000|Good|1|0|0| +6668|F|W|Secondary|8000|Good|1|0|0| +6669|M|U|Secondary|8000|Good|1|0|0| +6670|F|U|Secondary|8000|Good|1|0|0| +6671|M|M|College|8000|Good|1|0|0| +6672|F|M|College|8000|Good|1|0|0| +6673|M|S|College|8000|Good|1|0|0| +6674|F|S|College|8000|Good|1|0|0| +6675|M|D|College|8000|Good|1|0|0| +6676|F|D|College|8000|Good|1|0|0| +6677|M|W|College|8000|Good|1|0|0| +6678|F|W|College|8000|Good|1|0|0| +6679|M|U|College|8000|Good|1|0|0| +6680|F|U|College|8000|Good|1|0|0| +6681|M|M|2 yr Degree|8000|Good|1|0|0| +6682|F|M|2 yr Degree|8000|Good|1|0|0| +6683|M|S|2 yr Degree|8000|Good|1|0|0| +6684|F|S|2 yr Degree|8000|Good|1|0|0| +6685|M|D|2 yr Degree|8000|Good|1|0|0| +6686|F|D|2 yr Degree|8000|Good|1|0|0| +6687|M|W|2 yr Degree|8000|Good|1|0|0| +6688|F|W|2 yr Degree|8000|Good|1|0|0| +6689|M|U|2 yr Degree|8000|Good|1|0|0| +6690|F|U|2 yr Degree|8000|Good|1|0|0| +6691|M|M|4 yr Degree|8000|Good|1|0|0| +6692|F|M|4 yr Degree|8000|Good|1|0|0| +6693|M|S|4 yr Degree|8000|Good|1|0|0| +6694|F|S|4 yr Degree|8000|Good|1|0|0| +6695|M|D|4 yr Degree|8000|Good|1|0|0| +6696|F|D|4 yr Degree|8000|Good|1|0|0| +6697|M|W|4 yr Degree|8000|Good|1|0|0| +6698|F|W|4 yr Degree|8000|Good|1|0|0| +6699|M|U|4 yr Degree|8000|Good|1|0|0| +6700|F|U|4 yr Degree|8000|Good|1|0|0| +6701|M|M|Advanced Degree|8000|Good|1|0|0| +6702|F|M|Advanced Degree|8000|Good|1|0|0| +6703|M|S|Advanced Degree|8000|Good|1|0|0| +6704|F|S|Advanced Degree|8000|Good|1|0|0| +6705|M|D|Advanced Degree|8000|Good|1|0|0| +6706|F|D|Advanced Degree|8000|Good|1|0|0| +6707|M|W|Advanced Degree|8000|Good|1|0|0| +6708|F|W|Advanced Degree|8000|Good|1|0|0| +6709|M|U|Advanced Degree|8000|Good|1|0|0| +6710|F|U|Advanced Degree|8000|Good|1|0|0| +6711|M|M|Unknown|8000|Good|1|0|0| +6712|F|M|Unknown|8000|Good|1|0|0| +6713|M|S|Unknown|8000|Good|1|0|0| +6714|F|S|Unknown|8000|Good|1|0|0| +6715|M|D|Unknown|8000|Good|1|0|0| +6716|F|D|Unknown|8000|Good|1|0|0| +6717|M|W|Unknown|8000|Good|1|0|0| +6718|F|W|Unknown|8000|Good|1|0|0| +6719|M|U|Unknown|8000|Good|1|0|0| +6720|F|U|Unknown|8000|Good|1|0|0| +6721|M|M|Primary|8500|Good|1|0|0| +6722|F|M|Primary|8500|Good|1|0|0| +6723|M|S|Primary|8500|Good|1|0|0| +6724|F|S|Primary|8500|Good|1|0|0| +6725|M|D|Primary|8500|Good|1|0|0| +6726|F|D|Primary|8500|Good|1|0|0| +6727|M|W|Primary|8500|Good|1|0|0| +6728|F|W|Primary|8500|Good|1|0|0| +6729|M|U|Primary|8500|Good|1|0|0| +6730|F|U|Primary|8500|Good|1|0|0| +6731|M|M|Secondary|8500|Good|1|0|0| +6732|F|M|Secondary|8500|Good|1|0|0| +6733|M|S|Secondary|8500|Good|1|0|0| +6734|F|S|Secondary|8500|Good|1|0|0| +6735|M|D|Secondary|8500|Good|1|0|0| +6736|F|D|Secondary|8500|Good|1|0|0| +6737|M|W|Secondary|8500|Good|1|0|0| +6738|F|W|Secondary|8500|Good|1|0|0| +6739|M|U|Secondary|8500|Good|1|0|0| +6740|F|U|Secondary|8500|Good|1|0|0| +6741|M|M|College|8500|Good|1|0|0| +6742|F|M|College|8500|Good|1|0|0| +6743|M|S|College|8500|Good|1|0|0| +6744|F|S|College|8500|Good|1|0|0| +6745|M|D|College|8500|Good|1|0|0| +6746|F|D|College|8500|Good|1|0|0| +6747|M|W|College|8500|Good|1|0|0| +6748|F|W|College|8500|Good|1|0|0| +6749|M|U|College|8500|Good|1|0|0| +6750|F|U|College|8500|Good|1|0|0| +6751|M|M|2 yr Degree|8500|Good|1|0|0| +6752|F|M|2 yr Degree|8500|Good|1|0|0| +6753|M|S|2 yr Degree|8500|Good|1|0|0| +6754|F|S|2 yr Degree|8500|Good|1|0|0| +6755|M|D|2 yr Degree|8500|Good|1|0|0| +6756|F|D|2 yr Degree|8500|Good|1|0|0| +6757|M|W|2 yr Degree|8500|Good|1|0|0| +6758|F|W|2 yr Degree|8500|Good|1|0|0| +6759|M|U|2 yr Degree|8500|Good|1|0|0| +6760|F|U|2 yr Degree|8500|Good|1|0|0| +6761|M|M|4 yr Degree|8500|Good|1|0|0| +6762|F|M|4 yr Degree|8500|Good|1|0|0| +6763|M|S|4 yr Degree|8500|Good|1|0|0| +6764|F|S|4 yr Degree|8500|Good|1|0|0| +6765|M|D|4 yr Degree|8500|Good|1|0|0| +6766|F|D|4 yr Degree|8500|Good|1|0|0| +6767|M|W|4 yr Degree|8500|Good|1|0|0| +6768|F|W|4 yr Degree|8500|Good|1|0|0| +6769|M|U|4 yr Degree|8500|Good|1|0|0| +6770|F|U|4 yr Degree|8500|Good|1|0|0| +6771|M|M|Advanced Degree|8500|Good|1|0|0| +6772|F|M|Advanced Degree|8500|Good|1|0|0| +6773|M|S|Advanced Degree|8500|Good|1|0|0| +6774|F|S|Advanced Degree|8500|Good|1|0|0| +6775|M|D|Advanced Degree|8500|Good|1|0|0| +6776|F|D|Advanced Degree|8500|Good|1|0|0| +6777|M|W|Advanced Degree|8500|Good|1|0|0| +6778|F|W|Advanced Degree|8500|Good|1|0|0| +6779|M|U|Advanced Degree|8500|Good|1|0|0| +6780|F|U|Advanced Degree|8500|Good|1|0|0| +6781|M|M|Unknown|8500|Good|1|0|0| +6782|F|M|Unknown|8500|Good|1|0|0| +6783|M|S|Unknown|8500|Good|1|0|0| +6784|F|S|Unknown|8500|Good|1|0|0| +6785|M|D|Unknown|8500|Good|1|0|0| +6786|F|D|Unknown|8500|Good|1|0|0| +6787|M|W|Unknown|8500|Good|1|0|0| +6788|F|W|Unknown|8500|Good|1|0|0| +6789|M|U|Unknown|8500|Good|1|0|0| +6790|F|U|Unknown|8500|Good|1|0|0| +6791|M|M|Primary|9000|Good|1|0|0| +6792|F|M|Primary|9000|Good|1|0|0| +6793|M|S|Primary|9000|Good|1|0|0| +6794|F|S|Primary|9000|Good|1|0|0| +6795|M|D|Primary|9000|Good|1|0|0| +6796|F|D|Primary|9000|Good|1|0|0| +6797|M|W|Primary|9000|Good|1|0|0| +6798|F|W|Primary|9000|Good|1|0|0| +6799|M|U|Primary|9000|Good|1|0|0| +6800|F|U|Primary|9000|Good|1|0|0| +6801|M|M|Secondary|9000|Good|1|0|0| +6802|F|M|Secondary|9000|Good|1|0|0| +6803|M|S|Secondary|9000|Good|1|0|0| +6804|F|S|Secondary|9000|Good|1|0|0| +6805|M|D|Secondary|9000|Good|1|0|0| +6806|F|D|Secondary|9000|Good|1|0|0| +6807|M|W|Secondary|9000|Good|1|0|0| +6808|F|W|Secondary|9000|Good|1|0|0| +6809|M|U|Secondary|9000|Good|1|0|0| +6810|F|U|Secondary|9000|Good|1|0|0| +6811|M|M|College|9000|Good|1|0|0| +6812|F|M|College|9000|Good|1|0|0| +6813|M|S|College|9000|Good|1|0|0| +6814|F|S|College|9000|Good|1|0|0| +6815|M|D|College|9000|Good|1|0|0| +6816|F|D|College|9000|Good|1|0|0| +6817|M|W|College|9000|Good|1|0|0| +6818|F|W|College|9000|Good|1|0|0| +6819|M|U|College|9000|Good|1|0|0| +6820|F|U|College|9000|Good|1|0|0| +6821|M|M|2 yr Degree|9000|Good|1|0|0| +6822|F|M|2 yr Degree|9000|Good|1|0|0| +6823|M|S|2 yr Degree|9000|Good|1|0|0| +6824|F|S|2 yr Degree|9000|Good|1|0|0| +6825|M|D|2 yr Degree|9000|Good|1|0|0| +6826|F|D|2 yr Degree|9000|Good|1|0|0| +6827|M|W|2 yr Degree|9000|Good|1|0|0| +6828|F|W|2 yr Degree|9000|Good|1|0|0| +6829|M|U|2 yr Degree|9000|Good|1|0|0| +6830|F|U|2 yr Degree|9000|Good|1|0|0| +6831|M|M|4 yr Degree|9000|Good|1|0|0| +6832|F|M|4 yr Degree|9000|Good|1|0|0| +6833|M|S|4 yr Degree|9000|Good|1|0|0| +6834|F|S|4 yr Degree|9000|Good|1|0|0| +6835|M|D|4 yr Degree|9000|Good|1|0|0| +6836|F|D|4 yr Degree|9000|Good|1|0|0| +6837|M|W|4 yr Degree|9000|Good|1|0|0| +6838|F|W|4 yr Degree|9000|Good|1|0|0| +6839|M|U|4 yr Degree|9000|Good|1|0|0| +6840|F|U|4 yr Degree|9000|Good|1|0|0| +6841|M|M|Advanced Degree|9000|Good|1|0|0| +6842|F|M|Advanced Degree|9000|Good|1|0|0| +6843|M|S|Advanced Degree|9000|Good|1|0|0| +6844|F|S|Advanced Degree|9000|Good|1|0|0| +6845|M|D|Advanced Degree|9000|Good|1|0|0| +6846|F|D|Advanced Degree|9000|Good|1|0|0| +6847|M|W|Advanced Degree|9000|Good|1|0|0| +6848|F|W|Advanced Degree|9000|Good|1|0|0| +6849|M|U|Advanced Degree|9000|Good|1|0|0| +6850|F|U|Advanced Degree|9000|Good|1|0|0| +6851|M|M|Unknown|9000|Good|1|0|0| +6852|F|M|Unknown|9000|Good|1|0|0| +6853|M|S|Unknown|9000|Good|1|0|0| +6854|F|S|Unknown|9000|Good|1|0|0| +6855|M|D|Unknown|9000|Good|1|0|0| +6856|F|D|Unknown|9000|Good|1|0|0| +6857|M|W|Unknown|9000|Good|1|0|0| +6858|F|W|Unknown|9000|Good|1|0|0| +6859|M|U|Unknown|9000|Good|1|0|0| +6860|F|U|Unknown|9000|Good|1|0|0| +6861|M|M|Primary|9500|Good|1|0|0| +6862|F|M|Primary|9500|Good|1|0|0| +6863|M|S|Primary|9500|Good|1|0|0| +6864|F|S|Primary|9500|Good|1|0|0| +6865|M|D|Primary|9500|Good|1|0|0| +6866|F|D|Primary|9500|Good|1|0|0| +6867|M|W|Primary|9500|Good|1|0|0| +6868|F|W|Primary|9500|Good|1|0|0| +6869|M|U|Primary|9500|Good|1|0|0| +6870|F|U|Primary|9500|Good|1|0|0| +6871|M|M|Secondary|9500|Good|1|0|0| +6872|F|M|Secondary|9500|Good|1|0|0| +6873|M|S|Secondary|9500|Good|1|0|0| +6874|F|S|Secondary|9500|Good|1|0|0| +6875|M|D|Secondary|9500|Good|1|0|0| +6876|F|D|Secondary|9500|Good|1|0|0| +6877|M|W|Secondary|9500|Good|1|0|0| +6878|F|W|Secondary|9500|Good|1|0|0| +6879|M|U|Secondary|9500|Good|1|0|0| +6880|F|U|Secondary|9500|Good|1|0|0| +6881|M|M|College|9500|Good|1|0|0| +6882|F|M|College|9500|Good|1|0|0| +6883|M|S|College|9500|Good|1|0|0| +6884|F|S|College|9500|Good|1|0|0| +6885|M|D|College|9500|Good|1|0|0| +6886|F|D|College|9500|Good|1|0|0| +6887|M|W|College|9500|Good|1|0|0| +6888|F|W|College|9500|Good|1|0|0| +6889|M|U|College|9500|Good|1|0|0| +6890|F|U|College|9500|Good|1|0|0| +6891|M|M|2 yr Degree|9500|Good|1|0|0| +6892|F|M|2 yr Degree|9500|Good|1|0|0| +6893|M|S|2 yr Degree|9500|Good|1|0|0| +6894|F|S|2 yr Degree|9500|Good|1|0|0| +6895|M|D|2 yr Degree|9500|Good|1|0|0| +6896|F|D|2 yr Degree|9500|Good|1|0|0| +6897|M|W|2 yr Degree|9500|Good|1|0|0| +6898|F|W|2 yr Degree|9500|Good|1|0|0| +6899|M|U|2 yr Degree|9500|Good|1|0|0| +6900|F|U|2 yr Degree|9500|Good|1|0|0| +6901|M|M|4 yr Degree|9500|Good|1|0|0| +6902|F|M|4 yr Degree|9500|Good|1|0|0| +6903|M|S|4 yr Degree|9500|Good|1|0|0| +6904|F|S|4 yr Degree|9500|Good|1|0|0| +6905|M|D|4 yr Degree|9500|Good|1|0|0| +6906|F|D|4 yr Degree|9500|Good|1|0|0| +6907|M|W|4 yr Degree|9500|Good|1|0|0| +6908|F|W|4 yr Degree|9500|Good|1|0|0| +6909|M|U|4 yr Degree|9500|Good|1|0|0| +6910|F|U|4 yr Degree|9500|Good|1|0|0| +6911|M|M|Advanced Degree|9500|Good|1|0|0| +6912|F|M|Advanced Degree|9500|Good|1|0|0| +6913|M|S|Advanced Degree|9500|Good|1|0|0| +6914|F|S|Advanced Degree|9500|Good|1|0|0| +6915|M|D|Advanced Degree|9500|Good|1|0|0| +6916|F|D|Advanced Degree|9500|Good|1|0|0| +6917|M|W|Advanced Degree|9500|Good|1|0|0| +6918|F|W|Advanced Degree|9500|Good|1|0|0| +6919|M|U|Advanced Degree|9500|Good|1|0|0| +6920|F|U|Advanced Degree|9500|Good|1|0|0| +6921|M|M|Unknown|9500|Good|1|0|0| +6922|F|M|Unknown|9500|Good|1|0|0| +6923|M|S|Unknown|9500|Good|1|0|0| +6924|F|S|Unknown|9500|Good|1|0|0| +6925|M|D|Unknown|9500|Good|1|0|0| +6926|F|D|Unknown|9500|Good|1|0|0| +6927|M|W|Unknown|9500|Good|1|0|0| +6928|F|W|Unknown|9500|Good|1|0|0| +6929|M|U|Unknown|9500|Good|1|0|0| +6930|F|U|Unknown|9500|Good|1|0|0| +6931|M|M|Primary|10000|Good|1|0|0| +6932|F|M|Primary|10000|Good|1|0|0| +6933|M|S|Primary|10000|Good|1|0|0| +6934|F|S|Primary|10000|Good|1|0|0| +6935|M|D|Primary|10000|Good|1|0|0| +6936|F|D|Primary|10000|Good|1|0|0| +6937|M|W|Primary|10000|Good|1|0|0| +6938|F|W|Primary|10000|Good|1|0|0| +6939|M|U|Primary|10000|Good|1|0|0| +6940|F|U|Primary|10000|Good|1|0|0| +6941|M|M|Secondary|10000|Good|1|0|0| +6942|F|M|Secondary|10000|Good|1|0|0| +6943|M|S|Secondary|10000|Good|1|0|0| +6944|F|S|Secondary|10000|Good|1|0|0| +6945|M|D|Secondary|10000|Good|1|0|0| +6946|F|D|Secondary|10000|Good|1|0|0| +6947|M|W|Secondary|10000|Good|1|0|0| +6948|F|W|Secondary|10000|Good|1|0|0| +6949|M|U|Secondary|10000|Good|1|0|0| +6950|F|U|Secondary|10000|Good|1|0|0| +6951|M|M|College|10000|Good|1|0|0| +6952|F|M|College|10000|Good|1|0|0| +6953|M|S|College|10000|Good|1|0|0| +6954|F|S|College|10000|Good|1|0|0| +6955|M|D|College|10000|Good|1|0|0| +6956|F|D|College|10000|Good|1|0|0| +6957|M|W|College|10000|Good|1|0|0| +6958|F|W|College|10000|Good|1|0|0| +6959|M|U|College|10000|Good|1|0|0| +6960|F|U|College|10000|Good|1|0|0| +6961|M|M|2 yr Degree|10000|Good|1|0|0| +6962|F|M|2 yr Degree|10000|Good|1|0|0| +6963|M|S|2 yr Degree|10000|Good|1|0|0| +6964|F|S|2 yr Degree|10000|Good|1|0|0| +6965|M|D|2 yr Degree|10000|Good|1|0|0| +6966|F|D|2 yr Degree|10000|Good|1|0|0| +6967|M|W|2 yr Degree|10000|Good|1|0|0| +6968|F|W|2 yr Degree|10000|Good|1|0|0| +6969|M|U|2 yr Degree|10000|Good|1|0|0| +6970|F|U|2 yr Degree|10000|Good|1|0|0| +6971|M|M|4 yr Degree|10000|Good|1|0|0| +6972|F|M|4 yr Degree|10000|Good|1|0|0| +6973|M|S|4 yr Degree|10000|Good|1|0|0| +6974|F|S|4 yr Degree|10000|Good|1|0|0| +6975|M|D|4 yr Degree|10000|Good|1|0|0| +6976|F|D|4 yr Degree|10000|Good|1|0|0| +6977|M|W|4 yr Degree|10000|Good|1|0|0| +6978|F|W|4 yr Degree|10000|Good|1|0|0| +6979|M|U|4 yr Degree|10000|Good|1|0|0| +6980|F|U|4 yr Degree|10000|Good|1|0|0| +6981|M|M|Advanced Degree|10000|Good|1|0|0| +6982|F|M|Advanced Degree|10000|Good|1|0|0| +6983|M|S|Advanced Degree|10000|Good|1|0|0| +6984|F|S|Advanced Degree|10000|Good|1|0|0| +6985|M|D|Advanced Degree|10000|Good|1|0|0| +6986|F|D|Advanced Degree|10000|Good|1|0|0| +6987|M|W|Advanced Degree|10000|Good|1|0|0| +6988|F|W|Advanced Degree|10000|Good|1|0|0| +6989|M|U|Advanced Degree|10000|Good|1|0|0| +6990|F|U|Advanced Degree|10000|Good|1|0|0| +6991|M|M|Unknown|10000|Good|1|0|0| +6992|F|M|Unknown|10000|Good|1|0|0| +6993|M|S|Unknown|10000|Good|1|0|0| +6994|F|S|Unknown|10000|Good|1|0|0| +6995|M|D|Unknown|10000|Good|1|0|0| +6996|F|D|Unknown|10000|Good|1|0|0| +6997|M|W|Unknown|10000|Good|1|0|0| +6998|F|W|Unknown|10000|Good|1|0|0| +6999|M|U|Unknown|10000|Good|1|0|0| +7000|F|U|Unknown|10000|Good|1|0|0| +7001|M|M|Primary|500|Low Risk|1|0|0| +7002|F|M|Primary|500|Low Risk|1|0|0| +7003|M|S|Primary|500|Low Risk|1|0|0| +7004|F|S|Primary|500|Low Risk|1|0|0| +7005|M|D|Primary|500|Low Risk|1|0|0| +7006|F|D|Primary|500|Low Risk|1|0|0| +7007|M|W|Primary|500|Low Risk|1|0|0| +7008|F|W|Primary|500|Low Risk|1|0|0| +7009|M|U|Primary|500|Low Risk|1|0|0| +7010|F|U|Primary|500|Low Risk|1|0|0| +7011|M|M|Secondary|500|Low Risk|1|0|0| +7012|F|M|Secondary|500|Low Risk|1|0|0| +7013|M|S|Secondary|500|Low Risk|1|0|0| +7014|F|S|Secondary|500|Low Risk|1|0|0| +7015|M|D|Secondary|500|Low Risk|1|0|0| +7016|F|D|Secondary|500|Low Risk|1|0|0| +7017|M|W|Secondary|500|Low Risk|1|0|0| +7018|F|W|Secondary|500|Low Risk|1|0|0| +7019|M|U|Secondary|500|Low Risk|1|0|0| +7020|F|U|Secondary|500|Low Risk|1|0|0| +7021|M|M|College|500|Low Risk|1|0|0| +7022|F|M|College|500|Low Risk|1|0|0| +7023|M|S|College|500|Low Risk|1|0|0| +7024|F|S|College|500|Low Risk|1|0|0| +7025|M|D|College|500|Low Risk|1|0|0| +7026|F|D|College|500|Low Risk|1|0|0| +7027|M|W|College|500|Low Risk|1|0|0| +7028|F|W|College|500|Low Risk|1|0|0| +7029|M|U|College|500|Low Risk|1|0|0| +7030|F|U|College|500|Low Risk|1|0|0| +7031|M|M|2 yr Degree|500|Low Risk|1|0|0| +7032|F|M|2 yr Degree|500|Low Risk|1|0|0| +7033|M|S|2 yr Degree|500|Low Risk|1|0|0| +7034|F|S|2 yr Degree|500|Low Risk|1|0|0| +7035|M|D|2 yr Degree|500|Low Risk|1|0|0| +7036|F|D|2 yr Degree|500|Low Risk|1|0|0| +7037|M|W|2 yr Degree|500|Low Risk|1|0|0| +7038|F|W|2 yr Degree|500|Low Risk|1|0|0| +7039|M|U|2 yr Degree|500|Low Risk|1|0|0| +7040|F|U|2 yr Degree|500|Low Risk|1|0|0| +7041|M|M|4 yr Degree|500|Low Risk|1|0|0| +7042|F|M|4 yr Degree|500|Low Risk|1|0|0| +7043|M|S|4 yr Degree|500|Low Risk|1|0|0| +7044|F|S|4 yr Degree|500|Low Risk|1|0|0| +7045|M|D|4 yr Degree|500|Low Risk|1|0|0| +7046|F|D|4 yr Degree|500|Low Risk|1|0|0| +7047|M|W|4 yr Degree|500|Low Risk|1|0|0| +7048|F|W|4 yr Degree|500|Low Risk|1|0|0| +7049|M|U|4 yr Degree|500|Low Risk|1|0|0| +7050|F|U|4 yr Degree|500|Low Risk|1|0|0| +7051|M|M|Advanced Degree|500|Low Risk|1|0|0| +7052|F|M|Advanced Degree|500|Low Risk|1|0|0| +7053|M|S|Advanced Degree|500|Low Risk|1|0|0| +7054|F|S|Advanced Degree|500|Low Risk|1|0|0| +7055|M|D|Advanced Degree|500|Low Risk|1|0|0| +7056|F|D|Advanced Degree|500|Low Risk|1|0|0| +7057|M|W|Advanced Degree|500|Low Risk|1|0|0| +7058|F|W|Advanced Degree|500|Low Risk|1|0|0| +7059|M|U|Advanced Degree|500|Low Risk|1|0|0| +7060|F|U|Advanced Degree|500|Low Risk|1|0|0| +7061|M|M|Unknown|500|Low Risk|1|0|0| +7062|F|M|Unknown|500|Low Risk|1|0|0| +7063|M|S|Unknown|500|Low Risk|1|0|0| +7064|F|S|Unknown|500|Low Risk|1|0|0| +7065|M|D|Unknown|500|Low Risk|1|0|0| +7066|F|D|Unknown|500|Low Risk|1|0|0| +7067|M|W|Unknown|500|Low Risk|1|0|0| +7068|F|W|Unknown|500|Low Risk|1|0|0| +7069|M|U|Unknown|500|Low Risk|1|0|0| +7070|F|U|Unknown|500|Low Risk|1|0|0| +7071|M|M|Primary|1000|Low Risk|1|0|0| +7072|F|M|Primary|1000|Low Risk|1|0|0| +7073|M|S|Primary|1000|Low Risk|1|0|0| +7074|F|S|Primary|1000|Low Risk|1|0|0| +7075|M|D|Primary|1000|Low Risk|1|0|0| +7076|F|D|Primary|1000|Low Risk|1|0|0| +7077|M|W|Primary|1000|Low Risk|1|0|0| +7078|F|W|Primary|1000|Low Risk|1|0|0| +7079|M|U|Primary|1000|Low Risk|1|0|0| +7080|F|U|Primary|1000|Low Risk|1|0|0| +7081|M|M|Secondary|1000|Low Risk|1|0|0| +7082|F|M|Secondary|1000|Low Risk|1|0|0| +7083|M|S|Secondary|1000|Low Risk|1|0|0| +7084|F|S|Secondary|1000|Low Risk|1|0|0| +7085|M|D|Secondary|1000|Low Risk|1|0|0| +7086|F|D|Secondary|1000|Low Risk|1|0|0| +7087|M|W|Secondary|1000|Low Risk|1|0|0| +7088|F|W|Secondary|1000|Low Risk|1|0|0| +7089|M|U|Secondary|1000|Low Risk|1|0|0| +7090|F|U|Secondary|1000|Low Risk|1|0|0| +7091|M|M|College|1000|Low Risk|1|0|0| +7092|F|M|College|1000|Low Risk|1|0|0| +7093|M|S|College|1000|Low Risk|1|0|0| +7094|F|S|College|1000|Low Risk|1|0|0| +7095|M|D|College|1000|Low Risk|1|0|0| +7096|F|D|College|1000|Low Risk|1|0|0| +7097|M|W|College|1000|Low Risk|1|0|0| +7098|F|W|College|1000|Low Risk|1|0|0| +7099|M|U|College|1000|Low Risk|1|0|0| +7100|F|U|College|1000|Low Risk|1|0|0| +7101|M|M|2 yr Degree|1000|Low Risk|1|0|0| +7102|F|M|2 yr Degree|1000|Low Risk|1|0|0| +7103|M|S|2 yr Degree|1000|Low Risk|1|0|0| +7104|F|S|2 yr Degree|1000|Low Risk|1|0|0| +7105|M|D|2 yr Degree|1000|Low Risk|1|0|0| +7106|F|D|2 yr Degree|1000|Low Risk|1|0|0| +7107|M|W|2 yr Degree|1000|Low Risk|1|0|0| +7108|F|W|2 yr Degree|1000|Low Risk|1|0|0| +7109|M|U|2 yr Degree|1000|Low Risk|1|0|0| +7110|F|U|2 yr Degree|1000|Low Risk|1|0|0| +7111|M|M|4 yr Degree|1000|Low Risk|1|0|0| +7112|F|M|4 yr Degree|1000|Low Risk|1|0|0| +7113|M|S|4 yr Degree|1000|Low Risk|1|0|0| +7114|F|S|4 yr Degree|1000|Low Risk|1|0|0| +7115|M|D|4 yr Degree|1000|Low Risk|1|0|0| +7116|F|D|4 yr Degree|1000|Low Risk|1|0|0| +7117|M|W|4 yr Degree|1000|Low Risk|1|0|0| +7118|F|W|4 yr Degree|1000|Low Risk|1|0|0| +7119|M|U|4 yr Degree|1000|Low Risk|1|0|0| +7120|F|U|4 yr Degree|1000|Low Risk|1|0|0| +7121|M|M|Advanced Degree|1000|Low Risk|1|0|0| +7122|F|M|Advanced Degree|1000|Low Risk|1|0|0| +7123|M|S|Advanced Degree|1000|Low Risk|1|0|0| +7124|F|S|Advanced Degree|1000|Low Risk|1|0|0| +7125|M|D|Advanced Degree|1000|Low Risk|1|0|0| +7126|F|D|Advanced Degree|1000|Low Risk|1|0|0| +7127|M|W|Advanced Degree|1000|Low Risk|1|0|0| +7128|F|W|Advanced Degree|1000|Low Risk|1|0|0| +7129|M|U|Advanced Degree|1000|Low Risk|1|0|0| +7130|F|U|Advanced Degree|1000|Low Risk|1|0|0| +7131|M|M|Unknown|1000|Low Risk|1|0|0| +7132|F|M|Unknown|1000|Low Risk|1|0|0| +7133|M|S|Unknown|1000|Low Risk|1|0|0| +7134|F|S|Unknown|1000|Low Risk|1|0|0| +7135|M|D|Unknown|1000|Low Risk|1|0|0| +7136|F|D|Unknown|1000|Low Risk|1|0|0| +7137|M|W|Unknown|1000|Low Risk|1|0|0| +7138|F|W|Unknown|1000|Low Risk|1|0|0| +7139|M|U|Unknown|1000|Low Risk|1|0|0| +7140|F|U|Unknown|1000|Low Risk|1|0|0| +7141|M|M|Primary|1500|Low Risk|1|0|0| +7142|F|M|Primary|1500|Low Risk|1|0|0| +7143|M|S|Primary|1500|Low Risk|1|0|0| +7144|F|S|Primary|1500|Low Risk|1|0|0| +7145|M|D|Primary|1500|Low Risk|1|0|0| +7146|F|D|Primary|1500|Low Risk|1|0|0| +7147|M|W|Primary|1500|Low Risk|1|0|0| +7148|F|W|Primary|1500|Low Risk|1|0|0| +7149|M|U|Primary|1500|Low Risk|1|0|0| +7150|F|U|Primary|1500|Low Risk|1|0|0| +7151|M|M|Secondary|1500|Low Risk|1|0|0| +7152|F|M|Secondary|1500|Low Risk|1|0|0| +7153|M|S|Secondary|1500|Low Risk|1|0|0| +7154|F|S|Secondary|1500|Low Risk|1|0|0| +7155|M|D|Secondary|1500|Low Risk|1|0|0| +7156|F|D|Secondary|1500|Low Risk|1|0|0| +7157|M|W|Secondary|1500|Low Risk|1|0|0| +7158|F|W|Secondary|1500|Low Risk|1|0|0| +7159|M|U|Secondary|1500|Low Risk|1|0|0| +7160|F|U|Secondary|1500|Low Risk|1|0|0| +7161|M|M|College|1500|Low Risk|1|0|0| +7162|F|M|College|1500|Low Risk|1|0|0| +7163|M|S|College|1500|Low Risk|1|0|0| +7164|F|S|College|1500|Low Risk|1|0|0| +7165|M|D|College|1500|Low Risk|1|0|0| +7166|F|D|College|1500|Low Risk|1|0|0| +7167|M|W|College|1500|Low Risk|1|0|0| +7168|F|W|College|1500|Low Risk|1|0|0| +7169|M|U|College|1500|Low Risk|1|0|0| +7170|F|U|College|1500|Low Risk|1|0|0| +7171|M|M|2 yr Degree|1500|Low Risk|1|0|0| +7172|F|M|2 yr Degree|1500|Low Risk|1|0|0| +7173|M|S|2 yr Degree|1500|Low Risk|1|0|0| +7174|F|S|2 yr Degree|1500|Low Risk|1|0|0| +7175|M|D|2 yr Degree|1500|Low Risk|1|0|0| +7176|F|D|2 yr Degree|1500|Low Risk|1|0|0| +7177|M|W|2 yr Degree|1500|Low Risk|1|0|0| +7178|F|W|2 yr Degree|1500|Low Risk|1|0|0| +7179|M|U|2 yr Degree|1500|Low Risk|1|0|0| +7180|F|U|2 yr Degree|1500|Low Risk|1|0|0| +7181|M|M|4 yr Degree|1500|Low Risk|1|0|0| +7182|F|M|4 yr Degree|1500|Low Risk|1|0|0| +7183|M|S|4 yr Degree|1500|Low Risk|1|0|0| +7184|F|S|4 yr Degree|1500|Low Risk|1|0|0| +7185|M|D|4 yr Degree|1500|Low Risk|1|0|0| +7186|F|D|4 yr Degree|1500|Low Risk|1|0|0| +7187|M|W|4 yr Degree|1500|Low Risk|1|0|0| +7188|F|W|4 yr Degree|1500|Low Risk|1|0|0| +7189|M|U|4 yr Degree|1500|Low Risk|1|0|0| +7190|F|U|4 yr Degree|1500|Low Risk|1|0|0| +7191|M|M|Advanced Degree|1500|Low Risk|1|0|0| +7192|F|M|Advanced Degree|1500|Low Risk|1|0|0| +7193|M|S|Advanced Degree|1500|Low Risk|1|0|0| +7194|F|S|Advanced Degree|1500|Low Risk|1|0|0| +7195|M|D|Advanced Degree|1500|Low Risk|1|0|0| +7196|F|D|Advanced Degree|1500|Low Risk|1|0|0| +7197|M|W|Advanced Degree|1500|Low Risk|1|0|0| +7198|F|W|Advanced Degree|1500|Low Risk|1|0|0| +7199|M|U|Advanced Degree|1500|Low Risk|1|0|0| +7200|F|U|Advanced Degree|1500|Low Risk|1|0|0| +7201|M|M|Unknown|1500|Low Risk|1|0|0| +7202|F|M|Unknown|1500|Low Risk|1|0|0| +7203|M|S|Unknown|1500|Low Risk|1|0|0| +7204|F|S|Unknown|1500|Low Risk|1|0|0| +7205|M|D|Unknown|1500|Low Risk|1|0|0| +7206|F|D|Unknown|1500|Low Risk|1|0|0| +7207|M|W|Unknown|1500|Low Risk|1|0|0| +7208|F|W|Unknown|1500|Low Risk|1|0|0| +7209|M|U|Unknown|1500|Low Risk|1|0|0| +7210|F|U|Unknown|1500|Low Risk|1|0|0| +7211|M|M|Primary|2000|Low Risk|1|0|0| +7212|F|M|Primary|2000|Low Risk|1|0|0| +7213|M|S|Primary|2000|Low Risk|1|0|0| +7214|F|S|Primary|2000|Low Risk|1|0|0| +7215|M|D|Primary|2000|Low Risk|1|0|0| +7216|F|D|Primary|2000|Low Risk|1|0|0| +7217|M|W|Primary|2000|Low Risk|1|0|0| +7218|F|W|Primary|2000|Low Risk|1|0|0| +7219|M|U|Primary|2000|Low Risk|1|0|0| +7220|F|U|Primary|2000|Low Risk|1|0|0| +7221|M|M|Secondary|2000|Low Risk|1|0|0| +7222|F|M|Secondary|2000|Low Risk|1|0|0| +7223|M|S|Secondary|2000|Low Risk|1|0|0| +7224|F|S|Secondary|2000|Low Risk|1|0|0| +7225|M|D|Secondary|2000|Low Risk|1|0|0| +7226|F|D|Secondary|2000|Low Risk|1|0|0| +7227|M|W|Secondary|2000|Low Risk|1|0|0| +7228|F|W|Secondary|2000|Low Risk|1|0|0| +7229|M|U|Secondary|2000|Low Risk|1|0|0| +7230|F|U|Secondary|2000|Low Risk|1|0|0| +7231|M|M|College|2000|Low Risk|1|0|0| +7232|F|M|College|2000|Low Risk|1|0|0| +7233|M|S|College|2000|Low Risk|1|0|0| +7234|F|S|College|2000|Low Risk|1|0|0| +7235|M|D|College|2000|Low Risk|1|0|0| +7236|F|D|College|2000|Low Risk|1|0|0| +7237|M|W|College|2000|Low Risk|1|0|0| +7238|F|W|College|2000|Low Risk|1|0|0| +7239|M|U|College|2000|Low Risk|1|0|0| +7240|F|U|College|2000|Low Risk|1|0|0| +7241|M|M|2 yr Degree|2000|Low Risk|1|0|0| +7242|F|M|2 yr Degree|2000|Low Risk|1|0|0| +7243|M|S|2 yr Degree|2000|Low Risk|1|0|0| +7244|F|S|2 yr Degree|2000|Low Risk|1|0|0| +7245|M|D|2 yr Degree|2000|Low Risk|1|0|0| +7246|F|D|2 yr Degree|2000|Low Risk|1|0|0| +7247|M|W|2 yr Degree|2000|Low Risk|1|0|0| +7248|F|W|2 yr Degree|2000|Low Risk|1|0|0| +7249|M|U|2 yr Degree|2000|Low Risk|1|0|0| +7250|F|U|2 yr Degree|2000|Low Risk|1|0|0| +7251|M|M|4 yr Degree|2000|Low Risk|1|0|0| +7252|F|M|4 yr Degree|2000|Low Risk|1|0|0| +7253|M|S|4 yr Degree|2000|Low Risk|1|0|0| +7254|F|S|4 yr Degree|2000|Low Risk|1|0|0| +7255|M|D|4 yr Degree|2000|Low Risk|1|0|0| +7256|F|D|4 yr Degree|2000|Low Risk|1|0|0| +7257|M|W|4 yr Degree|2000|Low Risk|1|0|0| +7258|F|W|4 yr Degree|2000|Low Risk|1|0|0| +7259|M|U|4 yr Degree|2000|Low Risk|1|0|0| +7260|F|U|4 yr Degree|2000|Low Risk|1|0|0| +7261|M|M|Advanced Degree|2000|Low Risk|1|0|0| +7262|F|M|Advanced Degree|2000|Low Risk|1|0|0| +7263|M|S|Advanced Degree|2000|Low Risk|1|0|0| +7264|F|S|Advanced Degree|2000|Low Risk|1|0|0| +7265|M|D|Advanced Degree|2000|Low Risk|1|0|0| +7266|F|D|Advanced Degree|2000|Low Risk|1|0|0| +7267|M|W|Advanced Degree|2000|Low Risk|1|0|0| +7268|F|W|Advanced Degree|2000|Low Risk|1|0|0| +7269|M|U|Advanced Degree|2000|Low Risk|1|0|0| +7270|F|U|Advanced Degree|2000|Low Risk|1|0|0| +7271|M|M|Unknown|2000|Low Risk|1|0|0| +7272|F|M|Unknown|2000|Low Risk|1|0|0| +7273|M|S|Unknown|2000|Low Risk|1|0|0| +7274|F|S|Unknown|2000|Low Risk|1|0|0| +7275|M|D|Unknown|2000|Low Risk|1|0|0| +7276|F|D|Unknown|2000|Low Risk|1|0|0| +7277|M|W|Unknown|2000|Low Risk|1|0|0| +7278|F|W|Unknown|2000|Low Risk|1|0|0| +7279|M|U|Unknown|2000|Low Risk|1|0|0| +7280|F|U|Unknown|2000|Low Risk|1|0|0| +7281|M|M|Primary|2500|Low Risk|1|0|0| +7282|F|M|Primary|2500|Low Risk|1|0|0| +7283|M|S|Primary|2500|Low Risk|1|0|0| +7284|F|S|Primary|2500|Low Risk|1|0|0| +7285|M|D|Primary|2500|Low Risk|1|0|0| +7286|F|D|Primary|2500|Low Risk|1|0|0| +7287|M|W|Primary|2500|Low Risk|1|0|0| +7288|F|W|Primary|2500|Low Risk|1|0|0| +7289|M|U|Primary|2500|Low Risk|1|0|0| +7290|F|U|Primary|2500|Low Risk|1|0|0| +7291|M|M|Secondary|2500|Low Risk|1|0|0| +7292|F|M|Secondary|2500|Low Risk|1|0|0| +7293|M|S|Secondary|2500|Low Risk|1|0|0| +7294|F|S|Secondary|2500|Low Risk|1|0|0| +7295|M|D|Secondary|2500|Low Risk|1|0|0| +7296|F|D|Secondary|2500|Low Risk|1|0|0| +7297|M|W|Secondary|2500|Low Risk|1|0|0| +7298|F|W|Secondary|2500|Low Risk|1|0|0| +7299|M|U|Secondary|2500|Low Risk|1|0|0| +7300|F|U|Secondary|2500|Low Risk|1|0|0| +7301|M|M|College|2500|Low Risk|1|0|0| +7302|F|M|College|2500|Low Risk|1|0|0| +7303|M|S|College|2500|Low Risk|1|0|0| +7304|F|S|College|2500|Low Risk|1|0|0| +7305|M|D|College|2500|Low Risk|1|0|0| +7306|F|D|College|2500|Low Risk|1|0|0| +7307|M|W|College|2500|Low Risk|1|0|0| +7308|F|W|College|2500|Low Risk|1|0|0| +7309|M|U|College|2500|Low Risk|1|0|0| +7310|F|U|College|2500|Low Risk|1|0|0| +7311|M|M|2 yr Degree|2500|Low Risk|1|0|0| +7312|F|M|2 yr Degree|2500|Low Risk|1|0|0| +7313|M|S|2 yr Degree|2500|Low Risk|1|0|0| +7314|F|S|2 yr Degree|2500|Low Risk|1|0|0| +7315|M|D|2 yr Degree|2500|Low Risk|1|0|0| +7316|F|D|2 yr Degree|2500|Low Risk|1|0|0| +7317|M|W|2 yr Degree|2500|Low Risk|1|0|0| +7318|F|W|2 yr Degree|2500|Low Risk|1|0|0| +7319|M|U|2 yr Degree|2500|Low Risk|1|0|0| +7320|F|U|2 yr Degree|2500|Low Risk|1|0|0| +7321|M|M|4 yr Degree|2500|Low Risk|1|0|0| +7322|F|M|4 yr Degree|2500|Low Risk|1|0|0| +7323|M|S|4 yr Degree|2500|Low Risk|1|0|0| +7324|F|S|4 yr Degree|2500|Low Risk|1|0|0| +7325|M|D|4 yr Degree|2500|Low Risk|1|0|0| +7326|F|D|4 yr Degree|2500|Low Risk|1|0|0| +7327|M|W|4 yr Degree|2500|Low Risk|1|0|0| +7328|F|W|4 yr Degree|2500|Low Risk|1|0|0| +7329|M|U|4 yr Degree|2500|Low Risk|1|0|0| +7330|F|U|4 yr Degree|2500|Low Risk|1|0|0| +7331|M|M|Advanced Degree|2500|Low Risk|1|0|0| +7332|F|M|Advanced Degree|2500|Low Risk|1|0|0| +7333|M|S|Advanced Degree|2500|Low Risk|1|0|0| +7334|F|S|Advanced Degree|2500|Low Risk|1|0|0| +7335|M|D|Advanced Degree|2500|Low Risk|1|0|0| +7336|F|D|Advanced Degree|2500|Low Risk|1|0|0| +7337|M|W|Advanced Degree|2500|Low Risk|1|0|0| +7338|F|W|Advanced Degree|2500|Low Risk|1|0|0| +7339|M|U|Advanced Degree|2500|Low Risk|1|0|0| +7340|F|U|Advanced Degree|2500|Low Risk|1|0|0| +7341|M|M|Unknown|2500|Low Risk|1|0|0| +7342|F|M|Unknown|2500|Low Risk|1|0|0| +7343|M|S|Unknown|2500|Low Risk|1|0|0| +7344|F|S|Unknown|2500|Low Risk|1|0|0| +7345|M|D|Unknown|2500|Low Risk|1|0|0| +7346|F|D|Unknown|2500|Low Risk|1|0|0| +7347|M|W|Unknown|2500|Low Risk|1|0|0| +7348|F|W|Unknown|2500|Low Risk|1|0|0| +7349|M|U|Unknown|2500|Low Risk|1|0|0| +7350|F|U|Unknown|2500|Low Risk|1|0|0| +7351|M|M|Primary|3000|Low Risk|1|0|0| +7352|F|M|Primary|3000|Low Risk|1|0|0| +7353|M|S|Primary|3000|Low Risk|1|0|0| +7354|F|S|Primary|3000|Low Risk|1|0|0| +7355|M|D|Primary|3000|Low Risk|1|0|0| +7356|F|D|Primary|3000|Low Risk|1|0|0| +7357|M|W|Primary|3000|Low Risk|1|0|0| +7358|F|W|Primary|3000|Low Risk|1|0|0| +7359|M|U|Primary|3000|Low Risk|1|0|0| +7360|F|U|Primary|3000|Low Risk|1|0|0| +7361|M|M|Secondary|3000|Low Risk|1|0|0| +7362|F|M|Secondary|3000|Low Risk|1|0|0| +7363|M|S|Secondary|3000|Low Risk|1|0|0| +7364|F|S|Secondary|3000|Low Risk|1|0|0| +7365|M|D|Secondary|3000|Low Risk|1|0|0| +7366|F|D|Secondary|3000|Low Risk|1|0|0| +7367|M|W|Secondary|3000|Low Risk|1|0|0| +7368|F|W|Secondary|3000|Low Risk|1|0|0| +7369|M|U|Secondary|3000|Low Risk|1|0|0| +7370|F|U|Secondary|3000|Low Risk|1|0|0| +7371|M|M|College|3000|Low Risk|1|0|0| +7372|F|M|College|3000|Low Risk|1|0|0| +7373|M|S|College|3000|Low Risk|1|0|0| +7374|F|S|College|3000|Low Risk|1|0|0| +7375|M|D|College|3000|Low Risk|1|0|0| +7376|F|D|College|3000|Low Risk|1|0|0| +7377|M|W|College|3000|Low Risk|1|0|0| +7378|F|W|College|3000|Low Risk|1|0|0| +7379|M|U|College|3000|Low Risk|1|0|0| +7380|F|U|College|3000|Low Risk|1|0|0| +7381|M|M|2 yr Degree|3000|Low Risk|1|0|0| +7382|F|M|2 yr Degree|3000|Low Risk|1|0|0| +7383|M|S|2 yr Degree|3000|Low Risk|1|0|0| +7384|F|S|2 yr Degree|3000|Low Risk|1|0|0| +7385|M|D|2 yr Degree|3000|Low Risk|1|0|0| +7386|F|D|2 yr Degree|3000|Low Risk|1|0|0| +7387|M|W|2 yr Degree|3000|Low Risk|1|0|0| +7388|F|W|2 yr Degree|3000|Low Risk|1|0|0| +7389|M|U|2 yr Degree|3000|Low Risk|1|0|0| +7390|F|U|2 yr Degree|3000|Low Risk|1|0|0| +7391|M|M|4 yr Degree|3000|Low Risk|1|0|0| +7392|F|M|4 yr Degree|3000|Low Risk|1|0|0| +7393|M|S|4 yr Degree|3000|Low Risk|1|0|0| +7394|F|S|4 yr Degree|3000|Low Risk|1|0|0| +7395|M|D|4 yr Degree|3000|Low Risk|1|0|0| +7396|F|D|4 yr Degree|3000|Low Risk|1|0|0| +7397|M|W|4 yr Degree|3000|Low Risk|1|0|0| +7398|F|W|4 yr Degree|3000|Low Risk|1|0|0| +7399|M|U|4 yr Degree|3000|Low Risk|1|0|0| +7400|F|U|4 yr Degree|3000|Low Risk|1|0|0| +7401|M|M|Advanced Degree|3000|Low Risk|1|0|0| +7402|F|M|Advanced Degree|3000|Low Risk|1|0|0| +7403|M|S|Advanced Degree|3000|Low Risk|1|0|0| +7404|F|S|Advanced Degree|3000|Low Risk|1|0|0| +7405|M|D|Advanced Degree|3000|Low Risk|1|0|0| +7406|F|D|Advanced Degree|3000|Low Risk|1|0|0| +7407|M|W|Advanced Degree|3000|Low Risk|1|0|0| +7408|F|W|Advanced Degree|3000|Low Risk|1|0|0| +7409|M|U|Advanced Degree|3000|Low Risk|1|0|0| +7410|F|U|Advanced Degree|3000|Low Risk|1|0|0| +7411|M|M|Unknown|3000|Low Risk|1|0|0| +7412|F|M|Unknown|3000|Low Risk|1|0|0| +7413|M|S|Unknown|3000|Low Risk|1|0|0| +7414|F|S|Unknown|3000|Low Risk|1|0|0| +7415|M|D|Unknown|3000|Low Risk|1|0|0| +7416|F|D|Unknown|3000|Low Risk|1|0|0| +7417|M|W|Unknown|3000|Low Risk|1|0|0| +7418|F|W|Unknown|3000|Low Risk|1|0|0| +7419|M|U|Unknown|3000|Low Risk|1|0|0| +7420|F|U|Unknown|3000|Low Risk|1|0|0| +7421|M|M|Primary|3500|Low Risk|1|0|0| +7422|F|M|Primary|3500|Low Risk|1|0|0| +7423|M|S|Primary|3500|Low Risk|1|0|0| +7424|F|S|Primary|3500|Low Risk|1|0|0| +7425|M|D|Primary|3500|Low Risk|1|0|0| +7426|F|D|Primary|3500|Low Risk|1|0|0| +7427|M|W|Primary|3500|Low Risk|1|0|0| +7428|F|W|Primary|3500|Low Risk|1|0|0| +7429|M|U|Primary|3500|Low Risk|1|0|0| +7430|F|U|Primary|3500|Low Risk|1|0|0| +7431|M|M|Secondary|3500|Low Risk|1|0|0| +7432|F|M|Secondary|3500|Low Risk|1|0|0| +7433|M|S|Secondary|3500|Low Risk|1|0|0| +7434|F|S|Secondary|3500|Low Risk|1|0|0| +7435|M|D|Secondary|3500|Low Risk|1|0|0| +7436|F|D|Secondary|3500|Low Risk|1|0|0| +7437|M|W|Secondary|3500|Low Risk|1|0|0| +7438|F|W|Secondary|3500|Low Risk|1|0|0| +7439|M|U|Secondary|3500|Low Risk|1|0|0| +7440|F|U|Secondary|3500|Low Risk|1|0|0| +7441|M|M|College|3500|Low Risk|1|0|0| +7442|F|M|College|3500|Low Risk|1|0|0| +7443|M|S|College|3500|Low Risk|1|0|0| +7444|F|S|College|3500|Low Risk|1|0|0| +7445|M|D|College|3500|Low Risk|1|0|0| +7446|F|D|College|3500|Low Risk|1|0|0| +7447|M|W|College|3500|Low Risk|1|0|0| +7448|F|W|College|3500|Low Risk|1|0|0| +7449|M|U|College|3500|Low Risk|1|0|0| +7450|F|U|College|3500|Low Risk|1|0|0| +7451|M|M|2 yr Degree|3500|Low Risk|1|0|0| +7452|F|M|2 yr Degree|3500|Low Risk|1|0|0| +7453|M|S|2 yr Degree|3500|Low Risk|1|0|0| +7454|F|S|2 yr Degree|3500|Low Risk|1|0|0| +7455|M|D|2 yr Degree|3500|Low Risk|1|0|0| +7456|F|D|2 yr Degree|3500|Low Risk|1|0|0| +7457|M|W|2 yr Degree|3500|Low Risk|1|0|0| +7458|F|W|2 yr Degree|3500|Low Risk|1|0|0| +7459|M|U|2 yr Degree|3500|Low Risk|1|0|0| +7460|F|U|2 yr Degree|3500|Low Risk|1|0|0| +7461|M|M|4 yr Degree|3500|Low Risk|1|0|0| +7462|F|M|4 yr Degree|3500|Low Risk|1|0|0| +7463|M|S|4 yr Degree|3500|Low Risk|1|0|0| +7464|F|S|4 yr Degree|3500|Low Risk|1|0|0| +7465|M|D|4 yr Degree|3500|Low Risk|1|0|0| +7466|F|D|4 yr Degree|3500|Low Risk|1|0|0| +7467|M|W|4 yr Degree|3500|Low Risk|1|0|0| +7468|F|W|4 yr Degree|3500|Low Risk|1|0|0| +7469|M|U|4 yr Degree|3500|Low Risk|1|0|0| +7470|F|U|4 yr Degree|3500|Low Risk|1|0|0| +7471|M|M|Advanced Degree|3500|Low Risk|1|0|0| +7472|F|M|Advanced Degree|3500|Low Risk|1|0|0| +7473|M|S|Advanced Degree|3500|Low Risk|1|0|0| +7474|F|S|Advanced Degree|3500|Low Risk|1|0|0| +7475|M|D|Advanced Degree|3500|Low Risk|1|0|0| +7476|F|D|Advanced Degree|3500|Low Risk|1|0|0| +7477|M|W|Advanced Degree|3500|Low Risk|1|0|0| +7478|F|W|Advanced Degree|3500|Low Risk|1|0|0| +7479|M|U|Advanced Degree|3500|Low Risk|1|0|0| +7480|F|U|Advanced Degree|3500|Low Risk|1|0|0| +7481|M|M|Unknown|3500|Low Risk|1|0|0| +7482|F|M|Unknown|3500|Low Risk|1|0|0| +7483|M|S|Unknown|3500|Low Risk|1|0|0| +7484|F|S|Unknown|3500|Low Risk|1|0|0| +7485|M|D|Unknown|3500|Low Risk|1|0|0| +7486|F|D|Unknown|3500|Low Risk|1|0|0| +7487|M|W|Unknown|3500|Low Risk|1|0|0| +7488|F|W|Unknown|3500|Low Risk|1|0|0| +7489|M|U|Unknown|3500|Low Risk|1|0|0| +7490|F|U|Unknown|3500|Low Risk|1|0|0| +7491|M|M|Primary|4000|Low Risk|1|0|0| +7492|F|M|Primary|4000|Low Risk|1|0|0| +7493|M|S|Primary|4000|Low Risk|1|0|0| +7494|F|S|Primary|4000|Low Risk|1|0|0| +7495|M|D|Primary|4000|Low Risk|1|0|0| +7496|F|D|Primary|4000|Low Risk|1|0|0| +7497|M|W|Primary|4000|Low Risk|1|0|0| +7498|F|W|Primary|4000|Low Risk|1|0|0| +7499|M|U|Primary|4000|Low Risk|1|0|0| +7500|F|U|Primary|4000|Low Risk|1|0|0| +7501|M|M|Secondary|4000|Low Risk|1|0|0| +7502|F|M|Secondary|4000|Low Risk|1|0|0| +7503|M|S|Secondary|4000|Low Risk|1|0|0| +7504|F|S|Secondary|4000|Low Risk|1|0|0| +7505|M|D|Secondary|4000|Low Risk|1|0|0| +7506|F|D|Secondary|4000|Low Risk|1|0|0| +7507|M|W|Secondary|4000|Low Risk|1|0|0| +7508|F|W|Secondary|4000|Low Risk|1|0|0| +7509|M|U|Secondary|4000|Low Risk|1|0|0| +7510|F|U|Secondary|4000|Low Risk|1|0|0| +7511|M|M|College|4000|Low Risk|1|0|0| +7512|F|M|College|4000|Low Risk|1|0|0| +7513|M|S|College|4000|Low Risk|1|0|0| +7514|F|S|College|4000|Low Risk|1|0|0| +7515|M|D|College|4000|Low Risk|1|0|0| +7516|F|D|College|4000|Low Risk|1|0|0| +7517|M|W|College|4000|Low Risk|1|0|0| +7518|F|W|College|4000|Low Risk|1|0|0| +7519|M|U|College|4000|Low Risk|1|0|0| +7520|F|U|College|4000|Low Risk|1|0|0| +7521|M|M|2 yr Degree|4000|Low Risk|1|0|0| +7522|F|M|2 yr Degree|4000|Low Risk|1|0|0| +7523|M|S|2 yr Degree|4000|Low Risk|1|0|0| +7524|F|S|2 yr Degree|4000|Low Risk|1|0|0| +7525|M|D|2 yr Degree|4000|Low Risk|1|0|0| +7526|F|D|2 yr Degree|4000|Low Risk|1|0|0| +7527|M|W|2 yr Degree|4000|Low Risk|1|0|0| +7528|F|W|2 yr Degree|4000|Low Risk|1|0|0| +7529|M|U|2 yr Degree|4000|Low Risk|1|0|0| +7530|F|U|2 yr Degree|4000|Low Risk|1|0|0| +7531|M|M|4 yr Degree|4000|Low Risk|1|0|0| +7532|F|M|4 yr Degree|4000|Low Risk|1|0|0| +7533|M|S|4 yr Degree|4000|Low Risk|1|0|0| +7534|F|S|4 yr Degree|4000|Low Risk|1|0|0| +7535|M|D|4 yr Degree|4000|Low Risk|1|0|0| +7536|F|D|4 yr Degree|4000|Low Risk|1|0|0| +7537|M|W|4 yr Degree|4000|Low Risk|1|0|0| +7538|F|W|4 yr Degree|4000|Low Risk|1|0|0| +7539|M|U|4 yr Degree|4000|Low Risk|1|0|0| +7540|F|U|4 yr Degree|4000|Low Risk|1|0|0| +7541|M|M|Advanced Degree|4000|Low Risk|1|0|0| +7542|F|M|Advanced Degree|4000|Low Risk|1|0|0| +7543|M|S|Advanced Degree|4000|Low Risk|1|0|0| +7544|F|S|Advanced Degree|4000|Low Risk|1|0|0| +7545|M|D|Advanced Degree|4000|Low Risk|1|0|0| +7546|F|D|Advanced Degree|4000|Low Risk|1|0|0| +7547|M|W|Advanced Degree|4000|Low Risk|1|0|0| +7548|F|W|Advanced Degree|4000|Low Risk|1|0|0| +7549|M|U|Advanced Degree|4000|Low Risk|1|0|0| +7550|F|U|Advanced Degree|4000|Low Risk|1|0|0| +7551|M|M|Unknown|4000|Low Risk|1|0|0| +7552|F|M|Unknown|4000|Low Risk|1|0|0| +7553|M|S|Unknown|4000|Low Risk|1|0|0| +7554|F|S|Unknown|4000|Low Risk|1|0|0| +7555|M|D|Unknown|4000|Low Risk|1|0|0| +7556|F|D|Unknown|4000|Low Risk|1|0|0| +7557|M|W|Unknown|4000|Low Risk|1|0|0| +7558|F|W|Unknown|4000|Low Risk|1|0|0| +7559|M|U|Unknown|4000|Low Risk|1|0|0| +7560|F|U|Unknown|4000|Low Risk|1|0|0| +7561|M|M|Primary|4500|Low Risk|1|0|0| +7562|F|M|Primary|4500|Low Risk|1|0|0| +7563|M|S|Primary|4500|Low Risk|1|0|0| +7564|F|S|Primary|4500|Low Risk|1|0|0| +7565|M|D|Primary|4500|Low Risk|1|0|0| +7566|F|D|Primary|4500|Low Risk|1|0|0| +7567|M|W|Primary|4500|Low Risk|1|0|0| +7568|F|W|Primary|4500|Low Risk|1|0|0| +7569|M|U|Primary|4500|Low Risk|1|0|0| +7570|F|U|Primary|4500|Low Risk|1|0|0| +7571|M|M|Secondary|4500|Low Risk|1|0|0| +7572|F|M|Secondary|4500|Low Risk|1|0|0| +7573|M|S|Secondary|4500|Low Risk|1|0|0| +7574|F|S|Secondary|4500|Low Risk|1|0|0| +7575|M|D|Secondary|4500|Low Risk|1|0|0| +7576|F|D|Secondary|4500|Low Risk|1|0|0| +7577|M|W|Secondary|4500|Low Risk|1|0|0| +7578|F|W|Secondary|4500|Low Risk|1|0|0| +7579|M|U|Secondary|4500|Low Risk|1|0|0| +7580|F|U|Secondary|4500|Low Risk|1|0|0| +7581|M|M|College|4500|Low Risk|1|0|0| +7582|F|M|College|4500|Low Risk|1|0|0| +7583|M|S|College|4500|Low Risk|1|0|0| +7584|F|S|College|4500|Low Risk|1|0|0| +7585|M|D|College|4500|Low Risk|1|0|0| +7586|F|D|College|4500|Low Risk|1|0|0| +7587|M|W|College|4500|Low Risk|1|0|0| +7588|F|W|College|4500|Low Risk|1|0|0| +7589|M|U|College|4500|Low Risk|1|0|0| +7590|F|U|College|4500|Low Risk|1|0|0| +7591|M|M|2 yr Degree|4500|Low Risk|1|0|0| +7592|F|M|2 yr Degree|4500|Low Risk|1|0|0| +7593|M|S|2 yr Degree|4500|Low Risk|1|0|0| +7594|F|S|2 yr Degree|4500|Low Risk|1|0|0| +7595|M|D|2 yr Degree|4500|Low Risk|1|0|0| +7596|F|D|2 yr Degree|4500|Low Risk|1|0|0| +7597|M|W|2 yr Degree|4500|Low Risk|1|0|0| +7598|F|W|2 yr Degree|4500|Low Risk|1|0|0| +7599|M|U|2 yr Degree|4500|Low Risk|1|0|0| +7600|F|U|2 yr Degree|4500|Low Risk|1|0|0| +7601|M|M|4 yr Degree|4500|Low Risk|1|0|0| +7602|F|M|4 yr Degree|4500|Low Risk|1|0|0| +7603|M|S|4 yr Degree|4500|Low Risk|1|0|0| +7604|F|S|4 yr Degree|4500|Low Risk|1|0|0| +7605|M|D|4 yr Degree|4500|Low Risk|1|0|0| +7606|F|D|4 yr Degree|4500|Low Risk|1|0|0| +7607|M|W|4 yr Degree|4500|Low Risk|1|0|0| +7608|F|W|4 yr Degree|4500|Low Risk|1|0|0| +7609|M|U|4 yr Degree|4500|Low Risk|1|0|0| +7610|F|U|4 yr Degree|4500|Low Risk|1|0|0| +7611|M|M|Advanced Degree|4500|Low Risk|1|0|0| +7612|F|M|Advanced Degree|4500|Low Risk|1|0|0| +7613|M|S|Advanced Degree|4500|Low Risk|1|0|0| +7614|F|S|Advanced Degree|4500|Low Risk|1|0|0| +7615|M|D|Advanced Degree|4500|Low Risk|1|0|0| +7616|F|D|Advanced Degree|4500|Low Risk|1|0|0| +7617|M|W|Advanced Degree|4500|Low Risk|1|0|0| +7618|F|W|Advanced Degree|4500|Low Risk|1|0|0| +7619|M|U|Advanced Degree|4500|Low Risk|1|0|0| +7620|F|U|Advanced Degree|4500|Low Risk|1|0|0| +7621|M|M|Unknown|4500|Low Risk|1|0|0| +7622|F|M|Unknown|4500|Low Risk|1|0|0| +7623|M|S|Unknown|4500|Low Risk|1|0|0| +7624|F|S|Unknown|4500|Low Risk|1|0|0| +7625|M|D|Unknown|4500|Low Risk|1|0|0| +7626|F|D|Unknown|4500|Low Risk|1|0|0| +7627|M|W|Unknown|4500|Low Risk|1|0|0| +7628|F|W|Unknown|4500|Low Risk|1|0|0| +7629|M|U|Unknown|4500|Low Risk|1|0|0| +7630|F|U|Unknown|4500|Low Risk|1|0|0| +7631|M|M|Primary|5000|Low Risk|1|0|0| +7632|F|M|Primary|5000|Low Risk|1|0|0| +7633|M|S|Primary|5000|Low Risk|1|0|0| +7634|F|S|Primary|5000|Low Risk|1|0|0| +7635|M|D|Primary|5000|Low Risk|1|0|0| +7636|F|D|Primary|5000|Low Risk|1|0|0| +7637|M|W|Primary|5000|Low Risk|1|0|0| +7638|F|W|Primary|5000|Low Risk|1|0|0| +7639|M|U|Primary|5000|Low Risk|1|0|0| +7640|F|U|Primary|5000|Low Risk|1|0|0| +7641|M|M|Secondary|5000|Low Risk|1|0|0| +7642|F|M|Secondary|5000|Low Risk|1|0|0| +7643|M|S|Secondary|5000|Low Risk|1|0|0| +7644|F|S|Secondary|5000|Low Risk|1|0|0| +7645|M|D|Secondary|5000|Low Risk|1|0|0| +7646|F|D|Secondary|5000|Low Risk|1|0|0| +7647|M|W|Secondary|5000|Low Risk|1|0|0| +7648|F|W|Secondary|5000|Low Risk|1|0|0| +7649|M|U|Secondary|5000|Low Risk|1|0|0| +7650|F|U|Secondary|5000|Low Risk|1|0|0| +7651|M|M|College|5000|Low Risk|1|0|0| +7652|F|M|College|5000|Low Risk|1|0|0| +7653|M|S|College|5000|Low Risk|1|0|0| +7654|F|S|College|5000|Low Risk|1|0|0| +7655|M|D|College|5000|Low Risk|1|0|0| +7656|F|D|College|5000|Low Risk|1|0|0| +7657|M|W|College|5000|Low Risk|1|0|0| +7658|F|W|College|5000|Low Risk|1|0|0| +7659|M|U|College|5000|Low Risk|1|0|0| +7660|F|U|College|5000|Low Risk|1|0|0| +7661|M|M|2 yr Degree|5000|Low Risk|1|0|0| +7662|F|M|2 yr Degree|5000|Low Risk|1|0|0| +7663|M|S|2 yr Degree|5000|Low Risk|1|0|0| +7664|F|S|2 yr Degree|5000|Low Risk|1|0|0| +7665|M|D|2 yr Degree|5000|Low Risk|1|0|0| +7666|F|D|2 yr Degree|5000|Low Risk|1|0|0| +7667|M|W|2 yr Degree|5000|Low Risk|1|0|0| +7668|F|W|2 yr Degree|5000|Low Risk|1|0|0| +7669|M|U|2 yr Degree|5000|Low Risk|1|0|0| +7670|F|U|2 yr Degree|5000|Low Risk|1|0|0| +7671|M|M|4 yr Degree|5000|Low Risk|1|0|0| +7672|F|M|4 yr Degree|5000|Low Risk|1|0|0| +7673|M|S|4 yr Degree|5000|Low Risk|1|0|0| +7674|F|S|4 yr Degree|5000|Low Risk|1|0|0| +7675|M|D|4 yr Degree|5000|Low Risk|1|0|0| +7676|F|D|4 yr Degree|5000|Low Risk|1|0|0| +7677|M|W|4 yr Degree|5000|Low Risk|1|0|0| +7678|F|W|4 yr Degree|5000|Low Risk|1|0|0| +7679|M|U|4 yr Degree|5000|Low Risk|1|0|0| +7680|F|U|4 yr Degree|5000|Low Risk|1|0|0| +7681|M|M|Advanced Degree|5000|Low Risk|1|0|0| +7682|F|M|Advanced Degree|5000|Low Risk|1|0|0| +7683|M|S|Advanced Degree|5000|Low Risk|1|0|0| +7684|F|S|Advanced Degree|5000|Low Risk|1|0|0| +7685|M|D|Advanced Degree|5000|Low Risk|1|0|0| +7686|F|D|Advanced Degree|5000|Low Risk|1|0|0| +7687|M|W|Advanced Degree|5000|Low Risk|1|0|0| +7688|F|W|Advanced Degree|5000|Low Risk|1|0|0| +7689|M|U|Advanced Degree|5000|Low Risk|1|0|0| +7690|F|U|Advanced Degree|5000|Low Risk|1|0|0| +7691|M|M|Unknown|5000|Low Risk|1|0|0| +7692|F|M|Unknown|5000|Low Risk|1|0|0| +7693|M|S|Unknown|5000|Low Risk|1|0|0| +7694|F|S|Unknown|5000|Low Risk|1|0|0| +7695|M|D|Unknown|5000|Low Risk|1|0|0| +7696|F|D|Unknown|5000|Low Risk|1|0|0| +7697|M|W|Unknown|5000|Low Risk|1|0|0| +7698|F|W|Unknown|5000|Low Risk|1|0|0| +7699|M|U|Unknown|5000|Low Risk|1|0|0| +7700|F|U|Unknown|5000|Low Risk|1|0|0| +7701|M|M|Primary|5500|Low Risk|1|0|0| +7702|F|M|Primary|5500|Low Risk|1|0|0| +7703|M|S|Primary|5500|Low Risk|1|0|0| +7704|F|S|Primary|5500|Low Risk|1|0|0| +7705|M|D|Primary|5500|Low Risk|1|0|0| +7706|F|D|Primary|5500|Low Risk|1|0|0| +7707|M|W|Primary|5500|Low Risk|1|0|0| +7708|F|W|Primary|5500|Low Risk|1|0|0| +7709|M|U|Primary|5500|Low Risk|1|0|0| +7710|F|U|Primary|5500|Low Risk|1|0|0| +7711|M|M|Secondary|5500|Low Risk|1|0|0| +7712|F|M|Secondary|5500|Low Risk|1|0|0| +7713|M|S|Secondary|5500|Low Risk|1|0|0| +7714|F|S|Secondary|5500|Low Risk|1|0|0| +7715|M|D|Secondary|5500|Low Risk|1|0|0| +7716|F|D|Secondary|5500|Low Risk|1|0|0| +7717|M|W|Secondary|5500|Low Risk|1|0|0| +7718|F|W|Secondary|5500|Low Risk|1|0|0| +7719|M|U|Secondary|5500|Low Risk|1|0|0| +7720|F|U|Secondary|5500|Low Risk|1|0|0| +7721|M|M|College|5500|Low Risk|1|0|0| +7722|F|M|College|5500|Low Risk|1|0|0| +7723|M|S|College|5500|Low Risk|1|0|0| +7724|F|S|College|5500|Low Risk|1|0|0| +7725|M|D|College|5500|Low Risk|1|0|0| +7726|F|D|College|5500|Low Risk|1|0|0| +7727|M|W|College|5500|Low Risk|1|0|0| +7728|F|W|College|5500|Low Risk|1|0|0| +7729|M|U|College|5500|Low Risk|1|0|0| +7730|F|U|College|5500|Low Risk|1|0|0| +7731|M|M|2 yr Degree|5500|Low Risk|1|0|0| +7732|F|M|2 yr Degree|5500|Low Risk|1|0|0| +7733|M|S|2 yr Degree|5500|Low Risk|1|0|0| +7734|F|S|2 yr Degree|5500|Low Risk|1|0|0| +7735|M|D|2 yr Degree|5500|Low Risk|1|0|0| +7736|F|D|2 yr Degree|5500|Low Risk|1|0|0| +7737|M|W|2 yr Degree|5500|Low Risk|1|0|0| +7738|F|W|2 yr Degree|5500|Low Risk|1|0|0| +7739|M|U|2 yr Degree|5500|Low Risk|1|0|0| +7740|F|U|2 yr Degree|5500|Low Risk|1|0|0| +7741|M|M|4 yr Degree|5500|Low Risk|1|0|0| +7742|F|M|4 yr Degree|5500|Low Risk|1|0|0| +7743|M|S|4 yr Degree|5500|Low Risk|1|0|0| +7744|F|S|4 yr Degree|5500|Low Risk|1|0|0| +7745|M|D|4 yr Degree|5500|Low Risk|1|0|0| +7746|F|D|4 yr Degree|5500|Low Risk|1|0|0| +7747|M|W|4 yr Degree|5500|Low Risk|1|0|0| +7748|F|W|4 yr Degree|5500|Low Risk|1|0|0| +7749|M|U|4 yr Degree|5500|Low Risk|1|0|0| +7750|F|U|4 yr Degree|5500|Low Risk|1|0|0| +7751|M|M|Advanced Degree|5500|Low Risk|1|0|0| +7752|F|M|Advanced Degree|5500|Low Risk|1|0|0| +7753|M|S|Advanced Degree|5500|Low Risk|1|0|0| +7754|F|S|Advanced Degree|5500|Low Risk|1|0|0| +7755|M|D|Advanced Degree|5500|Low Risk|1|0|0| +7756|F|D|Advanced Degree|5500|Low Risk|1|0|0| +7757|M|W|Advanced Degree|5500|Low Risk|1|0|0| +7758|F|W|Advanced Degree|5500|Low Risk|1|0|0| +7759|M|U|Advanced Degree|5500|Low Risk|1|0|0| +7760|F|U|Advanced Degree|5500|Low Risk|1|0|0| +7761|M|M|Unknown|5500|Low Risk|1|0|0| +7762|F|M|Unknown|5500|Low Risk|1|0|0| +7763|M|S|Unknown|5500|Low Risk|1|0|0| +7764|F|S|Unknown|5500|Low Risk|1|0|0| +7765|M|D|Unknown|5500|Low Risk|1|0|0| +7766|F|D|Unknown|5500|Low Risk|1|0|0| +7767|M|W|Unknown|5500|Low Risk|1|0|0| +7768|F|W|Unknown|5500|Low Risk|1|0|0| +7769|M|U|Unknown|5500|Low Risk|1|0|0| +7770|F|U|Unknown|5500|Low Risk|1|0|0| +7771|M|M|Primary|6000|Low Risk|1|0|0| +7772|F|M|Primary|6000|Low Risk|1|0|0| +7773|M|S|Primary|6000|Low Risk|1|0|0| +7774|F|S|Primary|6000|Low Risk|1|0|0| +7775|M|D|Primary|6000|Low Risk|1|0|0| +7776|F|D|Primary|6000|Low Risk|1|0|0| +7777|M|W|Primary|6000|Low Risk|1|0|0| +7778|F|W|Primary|6000|Low Risk|1|0|0| +7779|M|U|Primary|6000|Low Risk|1|0|0| +7780|F|U|Primary|6000|Low Risk|1|0|0| +7781|M|M|Secondary|6000|Low Risk|1|0|0| +7782|F|M|Secondary|6000|Low Risk|1|0|0| +7783|M|S|Secondary|6000|Low Risk|1|0|0| +7784|F|S|Secondary|6000|Low Risk|1|0|0| +7785|M|D|Secondary|6000|Low Risk|1|0|0| +7786|F|D|Secondary|6000|Low Risk|1|0|0| +7787|M|W|Secondary|6000|Low Risk|1|0|0| +7788|F|W|Secondary|6000|Low Risk|1|0|0| +7789|M|U|Secondary|6000|Low Risk|1|0|0| +7790|F|U|Secondary|6000|Low Risk|1|0|0| +7791|M|M|College|6000|Low Risk|1|0|0| +7792|F|M|College|6000|Low Risk|1|0|0| +7793|M|S|College|6000|Low Risk|1|0|0| +7794|F|S|College|6000|Low Risk|1|0|0| +7795|M|D|College|6000|Low Risk|1|0|0| +7796|F|D|College|6000|Low Risk|1|0|0| +7797|M|W|College|6000|Low Risk|1|0|0| +7798|F|W|College|6000|Low Risk|1|0|0| +7799|M|U|College|6000|Low Risk|1|0|0| +7800|F|U|College|6000|Low Risk|1|0|0| +7801|M|M|2 yr Degree|6000|Low Risk|1|0|0| +7802|F|M|2 yr Degree|6000|Low Risk|1|0|0| +7803|M|S|2 yr Degree|6000|Low Risk|1|0|0| +7804|F|S|2 yr Degree|6000|Low Risk|1|0|0| +7805|M|D|2 yr Degree|6000|Low Risk|1|0|0| +7806|F|D|2 yr Degree|6000|Low Risk|1|0|0| +7807|M|W|2 yr Degree|6000|Low Risk|1|0|0| +7808|F|W|2 yr Degree|6000|Low Risk|1|0|0| +7809|M|U|2 yr Degree|6000|Low Risk|1|0|0| +7810|F|U|2 yr Degree|6000|Low Risk|1|0|0| +7811|M|M|4 yr Degree|6000|Low Risk|1|0|0| +7812|F|M|4 yr Degree|6000|Low Risk|1|0|0| +7813|M|S|4 yr Degree|6000|Low Risk|1|0|0| +7814|F|S|4 yr Degree|6000|Low Risk|1|0|0| +7815|M|D|4 yr Degree|6000|Low Risk|1|0|0| +7816|F|D|4 yr Degree|6000|Low Risk|1|0|0| +7817|M|W|4 yr Degree|6000|Low Risk|1|0|0| +7818|F|W|4 yr Degree|6000|Low Risk|1|0|0| +7819|M|U|4 yr Degree|6000|Low Risk|1|0|0| +7820|F|U|4 yr Degree|6000|Low Risk|1|0|0| +7821|M|M|Advanced Degree|6000|Low Risk|1|0|0| +7822|F|M|Advanced Degree|6000|Low Risk|1|0|0| +7823|M|S|Advanced Degree|6000|Low Risk|1|0|0| +7824|F|S|Advanced Degree|6000|Low Risk|1|0|0| +7825|M|D|Advanced Degree|6000|Low Risk|1|0|0| +7826|F|D|Advanced Degree|6000|Low Risk|1|0|0| +7827|M|W|Advanced Degree|6000|Low Risk|1|0|0| +7828|F|W|Advanced Degree|6000|Low Risk|1|0|0| +7829|M|U|Advanced Degree|6000|Low Risk|1|0|0| +7830|F|U|Advanced Degree|6000|Low Risk|1|0|0| +7831|M|M|Unknown|6000|Low Risk|1|0|0| +7832|F|M|Unknown|6000|Low Risk|1|0|0| +7833|M|S|Unknown|6000|Low Risk|1|0|0| +7834|F|S|Unknown|6000|Low Risk|1|0|0| +7835|M|D|Unknown|6000|Low Risk|1|0|0| +7836|F|D|Unknown|6000|Low Risk|1|0|0| +7837|M|W|Unknown|6000|Low Risk|1|0|0| +7838|F|W|Unknown|6000|Low Risk|1|0|0| +7839|M|U|Unknown|6000|Low Risk|1|0|0| +7840|F|U|Unknown|6000|Low Risk|1|0|0| +7841|M|M|Primary|6500|Low Risk|1|0|0| +7842|F|M|Primary|6500|Low Risk|1|0|0| +7843|M|S|Primary|6500|Low Risk|1|0|0| +7844|F|S|Primary|6500|Low Risk|1|0|0| +7845|M|D|Primary|6500|Low Risk|1|0|0| +7846|F|D|Primary|6500|Low Risk|1|0|0| +7847|M|W|Primary|6500|Low Risk|1|0|0| +7848|F|W|Primary|6500|Low Risk|1|0|0| +7849|M|U|Primary|6500|Low Risk|1|0|0| +7850|F|U|Primary|6500|Low Risk|1|0|0| +7851|M|M|Secondary|6500|Low Risk|1|0|0| +7852|F|M|Secondary|6500|Low Risk|1|0|0| +7853|M|S|Secondary|6500|Low Risk|1|0|0| +7854|F|S|Secondary|6500|Low Risk|1|0|0| +7855|M|D|Secondary|6500|Low Risk|1|0|0| +7856|F|D|Secondary|6500|Low Risk|1|0|0| +7857|M|W|Secondary|6500|Low Risk|1|0|0| +7858|F|W|Secondary|6500|Low Risk|1|0|0| +7859|M|U|Secondary|6500|Low Risk|1|0|0| +7860|F|U|Secondary|6500|Low Risk|1|0|0| +7861|M|M|College|6500|Low Risk|1|0|0| +7862|F|M|College|6500|Low Risk|1|0|0| +7863|M|S|College|6500|Low Risk|1|0|0| +7864|F|S|College|6500|Low Risk|1|0|0| +7865|M|D|College|6500|Low Risk|1|0|0| +7866|F|D|College|6500|Low Risk|1|0|0| +7867|M|W|College|6500|Low Risk|1|0|0| +7868|F|W|College|6500|Low Risk|1|0|0| +7869|M|U|College|6500|Low Risk|1|0|0| +7870|F|U|College|6500|Low Risk|1|0|0| +7871|M|M|2 yr Degree|6500|Low Risk|1|0|0| +7872|F|M|2 yr Degree|6500|Low Risk|1|0|0| +7873|M|S|2 yr Degree|6500|Low Risk|1|0|0| +7874|F|S|2 yr Degree|6500|Low Risk|1|0|0| +7875|M|D|2 yr Degree|6500|Low Risk|1|0|0| +7876|F|D|2 yr Degree|6500|Low Risk|1|0|0| +7877|M|W|2 yr Degree|6500|Low Risk|1|0|0| +7878|F|W|2 yr Degree|6500|Low Risk|1|0|0| +7879|M|U|2 yr Degree|6500|Low Risk|1|0|0| +7880|F|U|2 yr Degree|6500|Low Risk|1|0|0| +7881|M|M|4 yr Degree|6500|Low Risk|1|0|0| +7882|F|M|4 yr Degree|6500|Low Risk|1|0|0| +7883|M|S|4 yr Degree|6500|Low Risk|1|0|0| +7884|F|S|4 yr Degree|6500|Low Risk|1|0|0| +7885|M|D|4 yr Degree|6500|Low Risk|1|0|0| +7886|F|D|4 yr Degree|6500|Low Risk|1|0|0| +7887|M|W|4 yr Degree|6500|Low Risk|1|0|0| +7888|F|W|4 yr Degree|6500|Low Risk|1|0|0| +7889|M|U|4 yr Degree|6500|Low Risk|1|0|0| +7890|F|U|4 yr Degree|6500|Low Risk|1|0|0| +7891|M|M|Advanced Degree|6500|Low Risk|1|0|0| +7892|F|M|Advanced Degree|6500|Low Risk|1|0|0| +7893|M|S|Advanced Degree|6500|Low Risk|1|0|0| +7894|F|S|Advanced Degree|6500|Low Risk|1|0|0| +7895|M|D|Advanced Degree|6500|Low Risk|1|0|0| +7896|F|D|Advanced Degree|6500|Low Risk|1|0|0| +7897|M|W|Advanced Degree|6500|Low Risk|1|0|0| +7898|F|W|Advanced Degree|6500|Low Risk|1|0|0| +7899|M|U|Advanced Degree|6500|Low Risk|1|0|0| +7900|F|U|Advanced Degree|6500|Low Risk|1|0|0| +7901|M|M|Unknown|6500|Low Risk|1|0|0| +7902|F|M|Unknown|6500|Low Risk|1|0|0| +7903|M|S|Unknown|6500|Low Risk|1|0|0| +7904|F|S|Unknown|6500|Low Risk|1|0|0| +7905|M|D|Unknown|6500|Low Risk|1|0|0| +7906|F|D|Unknown|6500|Low Risk|1|0|0| +7907|M|W|Unknown|6500|Low Risk|1|0|0| +7908|F|W|Unknown|6500|Low Risk|1|0|0| +7909|M|U|Unknown|6500|Low Risk|1|0|0| +7910|F|U|Unknown|6500|Low Risk|1|0|0| +7911|M|M|Primary|7000|Low Risk|1|0|0| +7912|F|M|Primary|7000|Low Risk|1|0|0| +7913|M|S|Primary|7000|Low Risk|1|0|0| +7914|F|S|Primary|7000|Low Risk|1|0|0| +7915|M|D|Primary|7000|Low Risk|1|0|0| +7916|F|D|Primary|7000|Low Risk|1|0|0| +7917|M|W|Primary|7000|Low Risk|1|0|0| +7918|F|W|Primary|7000|Low Risk|1|0|0| +7919|M|U|Primary|7000|Low Risk|1|0|0| +7920|F|U|Primary|7000|Low Risk|1|0|0| +7921|M|M|Secondary|7000|Low Risk|1|0|0| +7922|F|M|Secondary|7000|Low Risk|1|0|0| +7923|M|S|Secondary|7000|Low Risk|1|0|0| +7924|F|S|Secondary|7000|Low Risk|1|0|0| +7925|M|D|Secondary|7000|Low Risk|1|0|0| +7926|F|D|Secondary|7000|Low Risk|1|0|0| +7927|M|W|Secondary|7000|Low Risk|1|0|0| +7928|F|W|Secondary|7000|Low Risk|1|0|0| +7929|M|U|Secondary|7000|Low Risk|1|0|0| +7930|F|U|Secondary|7000|Low Risk|1|0|0| +7931|M|M|College|7000|Low Risk|1|0|0| +7932|F|M|College|7000|Low Risk|1|0|0| +7933|M|S|College|7000|Low Risk|1|0|0| +7934|F|S|College|7000|Low Risk|1|0|0| +7935|M|D|College|7000|Low Risk|1|0|0| +7936|F|D|College|7000|Low Risk|1|0|0| +7937|M|W|College|7000|Low Risk|1|0|0| +7938|F|W|College|7000|Low Risk|1|0|0| +7939|M|U|College|7000|Low Risk|1|0|0| +7940|F|U|College|7000|Low Risk|1|0|0| +7941|M|M|2 yr Degree|7000|Low Risk|1|0|0| +7942|F|M|2 yr Degree|7000|Low Risk|1|0|0| +7943|M|S|2 yr Degree|7000|Low Risk|1|0|0| +7944|F|S|2 yr Degree|7000|Low Risk|1|0|0| +7945|M|D|2 yr Degree|7000|Low Risk|1|0|0| +7946|F|D|2 yr Degree|7000|Low Risk|1|0|0| +7947|M|W|2 yr Degree|7000|Low Risk|1|0|0| +7948|F|W|2 yr Degree|7000|Low Risk|1|0|0| +7949|M|U|2 yr Degree|7000|Low Risk|1|0|0| +7950|F|U|2 yr Degree|7000|Low Risk|1|0|0| +7951|M|M|4 yr Degree|7000|Low Risk|1|0|0| +7952|F|M|4 yr Degree|7000|Low Risk|1|0|0| +7953|M|S|4 yr Degree|7000|Low Risk|1|0|0| +7954|F|S|4 yr Degree|7000|Low Risk|1|0|0| +7955|M|D|4 yr Degree|7000|Low Risk|1|0|0| +7956|F|D|4 yr Degree|7000|Low Risk|1|0|0| +7957|M|W|4 yr Degree|7000|Low Risk|1|0|0| +7958|F|W|4 yr Degree|7000|Low Risk|1|0|0| +7959|M|U|4 yr Degree|7000|Low Risk|1|0|0| +7960|F|U|4 yr Degree|7000|Low Risk|1|0|0| +7961|M|M|Advanced Degree|7000|Low Risk|1|0|0| +7962|F|M|Advanced Degree|7000|Low Risk|1|0|0| +7963|M|S|Advanced Degree|7000|Low Risk|1|0|0| +7964|F|S|Advanced Degree|7000|Low Risk|1|0|0| +7965|M|D|Advanced Degree|7000|Low Risk|1|0|0| +7966|F|D|Advanced Degree|7000|Low Risk|1|0|0| +7967|M|W|Advanced Degree|7000|Low Risk|1|0|0| +7968|F|W|Advanced Degree|7000|Low Risk|1|0|0| +7969|M|U|Advanced Degree|7000|Low Risk|1|0|0| +7970|F|U|Advanced Degree|7000|Low Risk|1|0|0| +7971|M|M|Unknown|7000|Low Risk|1|0|0| +7972|F|M|Unknown|7000|Low Risk|1|0|0| +7973|M|S|Unknown|7000|Low Risk|1|0|0| +7974|F|S|Unknown|7000|Low Risk|1|0|0| +7975|M|D|Unknown|7000|Low Risk|1|0|0| +7976|F|D|Unknown|7000|Low Risk|1|0|0| +7977|M|W|Unknown|7000|Low Risk|1|0|0| +7978|F|W|Unknown|7000|Low Risk|1|0|0| +7979|M|U|Unknown|7000|Low Risk|1|0|0| +7980|F|U|Unknown|7000|Low Risk|1|0|0| +7981|M|M|Primary|7500|Low Risk|1|0|0| +7982|F|M|Primary|7500|Low Risk|1|0|0| +7983|M|S|Primary|7500|Low Risk|1|0|0| +7984|F|S|Primary|7500|Low Risk|1|0|0| +7985|M|D|Primary|7500|Low Risk|1|0|0| +7986|F|D|Primary|7500|Low Risk|1|0|0| +7987|M|W|Primary|7500|Low Risk|1|0|0| +7988|F|W|Primary|7500|Low Risk|1|0|0| +7989|M|U|Primary|7500|Low Risk|1|0|0| +7990|F|U|Primary|7500|Low Risk|1|0|0| +7991|M|M|Secondary|7500|Low Risk|1|0|0| +7992|F|M|Secondary|7500|Low Risk|1|0|0| +7993|M|S|Secondary|7500|Low Risk|1|0|0| +7994|F|S|Secondary|7500|Low Risk|1|0|0| +7995|M|D|Secondary|7500|Low Risk|1|0|0| +7996|F|D|Secondary|7500|Low Risk|1|0|0| +7997|M|W|Secondary|7500|Low Risk|1|0|0| +7998|F|W|Secondary|7500|Low Risk|1|0|0| +7999|M|U|Secondary|7500|Low Risk|1|0|0| +8000|F|U|Secondary|7500|Low Risk|1|0|0| +8001|M|M|College|7500|Low Risk|1|0|0| +8002|F|M|College|7500|Low Risk|1|0|0| +8003|M|S|College|7500|Low Risk|1|0|0| +8004|F|S|College|7500|Low Risk|1|0|0| +8005|M|D|College|7500|Low Risk|1|0|0| +8006|F|D|College|7500|Low Risk|1|0|0| +8007|M|W|College|7500|Low Risk|1|0|0| +8008|F|W|College|7500|Low Risk|1|0|0| +8009|M|U|College|7500|Low Risk|1|0|0| +8010|F|U|College|7500|Low Risk|1|0|0| +8011|M|M|2 yr Degree|7500|Low Risk|1|0|0| +8012|F|M|2 yr Degree|7500|Low Risk|1|0|0| +8013|M|S|2 yr Degree|7500|Low Risk|1|0|0| +8014|F|S|2 yr Degree|7500|Low Risk|1|0|0| +8015|M|D|2 yr Degree|7500|Low Risk|1|0|0| +8016|F|D|2 yr Degree|7500|Low Risk|1|0|0| +8017|M|W|2 yr Degree|7500|Low Risk|1|0|0| +8018|F|W|2 yr Degree|7500|Low Risk|1|0|0| +8019|M|U|2 yr Degree|7500|Low Risk|1|0|0| +8020|F|U|2 yr Degree|7500|Low Risk|1|0|0| +8021|M|M|4 yr Degree|7500|Low Risk|1|0|0| +8022|F|M|4 yr Degree|7500|Low Risk|1|0|0| +8023|M|S|4 yr Degree|7500|Low Risk|1|0|0| +8024|F|S|4 yr Degree|7500|Low Risk|1|0|0| +8025|M|D|4 yr Degree|7500|Low Risk|1|0|0| +8026|F|D|4 yr Degree|7500|Low Risk|1|0|0| +8027|M|W|4 yr Degree|7500|Low Risk|1|0|0| +8028|F|W|4 yr Degree|7500|Low Risk|1|0|0| +8029|M|U|4 yr Degree|7500|Low Risk|1|0|0| +8030|F|U|4 yr Degree|7500|Low Risk|1|0|0| +8031|M|M|Advanced Degree|7500|Low Risk|1|0|0| +8032|F|M|Advanced Degree|7500|Low Risk|1|0|0| +8033|M|S|Advanced Degree|7500|Low Risk|1|0|0| +8034|F|S|Advanced Degree|7500|Low Risk|1|0|0| +8035|M|D|Advanced Degree|7500|Low Risk|1|0|0| +8036|F|D|Advanced Degree|7500|Low Risk|1|0|0| +8037|M|W|Advanced Degree|7500|Low Risk|1|0|0| +8038|F|W|Advanced Degree|7500|Low Risk|1|0|0| +8039|M|U|Advanced Degree|7500|Low Risk|1|0|0| +8040|F|U|Advanced Degree|7500|Low Risk|1|0|0| +8041|M|M|Unknown|7500|Low Risk|1|0|0| +8042|F|M|Unknown|7500|Low Risk|1|0|0| +8043|M|S|Unknown|7500|Low Risk|1|0|0| +8044|F|S|Unknown|7500|Low Risk|1|0|0| +8045|M|D|Unknown|7500|Low Risk|1|0|0| +8046|F|D|Unknown|7500|Low Risk|1|0|0| +8047|M|W|Unknown|7500|Low Risk|1|0|0| +8048|F|W|Unknown|7500|Low Risk|1|0|0| +8049|M|U|Unknown|7500|Low Risk|1|0|0| +8050|F|U|Unknown|7500|Low Risk|1|0|0| +8051|M|M|Primary|8000|Low Risk|1|0|0| +8052|F|M|Primary|8000|Low Risk|1|0|0| +8053|M|S|Primary|8000|Low Risk|1|0|0| +8054|F|S|Primary|8000|Low Risk|1|0|0| +8055|M|D|Primary|8000|Low Risk|1|0|0| +8056|F|D|Primary|8000|Low Risk|1|0|0| +8057|M|W|Primary|8000|Low Risk|1|0|0| +8058|F|W|Primary|8000|Low Risk|1|0|0| +8059|M|U|Primary|8000|Low Risk|1|0|0| +8060|F|U|Primary|8000|Low Risk|1|0|0| +8061|M|M|Secondary|8000|Low Risk|1|0|0| +8062|F|M|Secondary|8000|Low Risk|1|0|0| +8063|M|S|Secondary|8000|Low Risk|1|0|0| +8064|F|S|Secondary|8000|Low Risk|1|0|0| +8065|M|D|Secondary|8000|Low Risk|1|0|0| +8066|F|D|Secondary|8000|Low Risk|1|0|0| +8067|M|W|Secondary|8000|Low Risk|1|0|0| +8068|F|W|Secondary|8000|Low Risk|1|0|0| +8069|M|U|Secondary|8000|Low Risk|1|0|0| +8070|F|U|Secondary|8000|Low Risk|1|0|0| +8071|M|M|College|8000|Low Risk|1|0|0| +8072|F|M|College|8000|Low Risk|1|0|0| +8073|M|S|College|8000|Low Risk|1|0|0| +8074|F|S|College|8000|Low Risk|1|0|0| +8075|M|D|College|8000|Low Risk|1|0|0| +8076|F|D|College|8000|Low Risk|1|0|0| +8077|M|W|College|8000|Low Risk|1|0|0| +8078|F|W|College|8000|Low Risk|1|0|0| +8079|M|U|College|8000|Low Risk|1|0|0| +8080|F|U|College|8000|Low Risk|1|0|0| +8081|M|M|2 yr Degree|8000|Low Risk|1|0|0| +8082|F|M|2 yr Degree|8000|Low Risk|1|0|0| +8083|M|S|2 yr Degree|8000|Low Risk|1|0|0| +8084|F|S|2 yr Degree|8000|Low Risk|1|0|0| +8085|M|D|2 yr Degree|8000|Low Risk|1|0|0| +8086|F|D|2 yr Degree|8000|Low Risk|1|0|0| +8087|M|W|2 yr Degree|8000|Low Risk|1|0|0| +8088|F|W|2 yr Degree|8000|Low Risk|1|0|0| +8089|M|U|2 yr Degree|8000|Low Risk|1|0|0| +8090|F|U|2 yr Degree|8000|Low Risk|1|0|0| +8091|M|M|4 yr Degree|8000|Low Risk|1|0|0| +8092|F|M|4 yr Degree|8000|Low Risk|1|0|0| +8093|M|S|4 yr Degree|8000|Low Risk|1|0|0| +8094|F|S|4 yr Degree|8000|Low Risk|1|0|0| +8095|M|D|4 yr Degree|8000|Low Risk|1|0|0| +8096|F|D|4 yr Degree|8000|Low Risk|1|0|0| +8097|M|W|4 yr Degree|8000|Low Risk|1|0|0| +8098|F|W|4 yr Degree|8000|Low Risk|1|0|0| +8099|M|U|4 yr Degree|8000|Low Risk|1|0|0| +8100|F|U|4 yr Degree|8000|Low Risk|1|0|0| +8101|M|M|Advanced Degree|8000|Low Risk|1|0|0| +8102|F|M|Advanced Degree|8000|Low Risk|1|0|0| +8103|M|S|Advanced Degree|8000|Low Risk|1|0|0| +8104|F|S|Advanced Degree|8000|Low Risk|1|0|0| +8105|M|D|Advanced Degree|8000|Low Risk|1|0|0| +8106|F|D|Advanced Degree|8000|Low Risk|1|0|0| +8107|M|W|Advanced Degree|8000|Low Risk|1|0|0| +8108|F|W|Advanced Degree|8000|Low Risk|1|0|0| +8109|M|U|Advanced Degree|8000|Low Risk|1|0|0| +8110|F|U|Advanced Degree|8000|Low Risk|1|0|0| +8111|M|M|Unknown|8000|Low Risk|1|0|0| +8112|F|M|Unknown|8000|Low Risk|1|0|0| +8113|M|S|Unknown|8000|Low Risk|1|0|0| +8114|F|S|Unknown|8000|Low Risk|1|0|0| +8115|M|D|Unknown|8000|Low Risk|1|0|0| +8116|F|D|Unknown|8000|Low Risk|1|0|0| +8117|M|W|Unknown|8000|Low Risk|1|0|0| +8118|F|W|Unknown|8000|Low Risk|1|0|0| +8119|M|U|Unknown|8000|Low Risk|1|0|0| +8120|F|U|Unknown|8000|Low Risk|1|0|0| +8121|M|M|Primary|8500|Low Risk|1|0|0| +8122|F|M|Primary|8500|Low Risk|1|0|0| +8123|M|S|Primary|8500|Low Risk|1|0|0| +8124|F|S|Primary|8500|Low Risk|1|0|0| +8125|M|D|Primary|8500|Low Risk|1|0|0| +8126|F|D|Primary|8500|Low Risk|1|0|0| +8127|M|W|Primary|8500|Low Risk|1|0|0| +8128|F|W|Primary|8500|Low Risk|1|0|0| +8129|M|U|Primary|8500|Low Risk|1|0|0| +8130|F|U|Primary|8500|Low Risk|1|0|0| +8131|M|M|Secondary|8500|Low Risk|1|0|0| +8132|F|M|Secondary|8500|Low Risk|1|0|0| +8133|M|S|Secondary|8500|Low Risk|1|0|0| +8134|F|S|Secondary|8500|Low Risk|1|0|0| +8135|M|D|Secondary|8500|Low Risk|1|0|0| +8136|F|D|Secondary|8500|Low Risk|1|0|0| +8137|M|W|Secondary|8500|Low Risk|1|0|0| +8138|F|W|Secondary|8500|Low Risk|1|0|0| +8139|M|U|Secondary|8500|Low Risk|1|0|0| +8140|F|U|Secondary|8500|Low Risk|1|0|0| +8141|M|M|College|8500|Low Risk|1|0|0| +8142|F|M|College|8500|Low Risk|1|0|0| +8143|M|S|College|8500|Low Risk|1|0|0| +8144|F|S|College|8500|Low Risk|1|0|0| +8145|M|D|College|8500|Low Risk|1|0|0| +8146|F|D|College|8500|Low Risk|1|0|0| +8147|M|W|College|8500|Low Risk|1|0|0| +8148|F|W|College|8500|Low Risk|1|0|0| +8149|M|U|College|8500|Low Risk|1|0|0| +8150|F|U|College|8500|Low Risk|1|0|0| +8151|M|M|2 yr Degree|8500|Low Risk|1|0|0| +8152|F|M|2 yr Degree|8500|Low Risk|1|0|0| +8153|M|S|2 yr Degree|8500|Low Risk|1|0|0| +8154|F|S|2 yr Degree|8500|Low Risk|1|0|0| +8155|M|D|2 yr Degree|8500|Low Risk|1|0|0| +8156|F|D|2 yr Degree|8500|Low Risk|1|0|0| +8157|M|W|2 yr Degree|8500|Low Risk|1|0|0| +8158|F|W|2 yr Degree|8500|Low Risk|1|0|0| +8159|M|U|2 yr Degree|8500|Low Risk|1|0|0| +8160|F|U|2 yr Degree|8500|Low Risk|1|0|0| +8161|M|M|4 yr Degree|8500|Low Risk|1|0|0| +8162|F|M|4 yr Degree|8500|Low Risk|1|0|0| +8163|M|S|4 yr Degree|8500|Low Risk|1|0|0| +8164|F|S|4 yr Degree|8500|Low Risk|1|0|0| +8165|M|D|4 yr Degree|8500|Low Risk|1|0|0| +8166|F|D|4 yr Degree|8500|Low Risk|1|0|0| +8167|M|W|4 yr Degree|8500|Low Risk|1|0|0| +8168|F|W|4 yr Degree|8500|Low Risk|1|0|0| +8169|M|U|4 yr Degree|8500|Low Risk|1|0|0| +8170|F|U|4 yr Degree|8500|Low Risk|1|0|0| +8171|M|M|Advanced Degree|8500|Low Risk|1|0|0| +8172|F|M|Advanced Degree|8500|Low Risk|1|0|0| +8173|M|S|Advanced Degree|8500|Low Risk|1|0|0| +8174|F|S|Advanced Degree|8500|Low Risk|1|0|0| +8175|M|D|Advanced Degree|8500|Low Risk|1|0|0| +8176|F|D|Advanced Degree|8500|Low Risk|1|0|0| +8177|M|W|Advanced Degree|8500|Low Risk|1|0|0| +8178|F|W|Advanced Degree|8500|Low Risk|1|0|0| +8179|M|U|Advanced Degree|8500|Low Risk|1|0|0| +8180|F|U|Advanced Degree|8500|Low Risk|1|0|0| +8181|M|M|Unknown|8500|Low Risk|1|0|0| +8182|F|M|Unknown|8500|Low Risk|1|0|0| +8183|M|S|Unknown|8500|Low Risk|1|0|0| +8184|F|S|Unknown|8500|Low Risk|1|0|0| +8185|M|D|Unknown|8500|Low Risk|1|0|0| +8186|F|D|Unknown|8500|Low Risk|1|0|0| +8187|M|W|Unknown|8500|Low Risk|1|0|0| +8188|F|W|Unknown|8500|Low Risk|1|0|0| +8189|M|U|Unknown|8500|Low Risk|1|0|0| +8190|F|U|Unknown|8500|Low Risk|1|0|0| +8191|M|M|Primary|9000|Low Risk|1|0|0| +8192|F|M|Primary|9000|Low Risk|1|0|0| +8193|M|S|Primary|9000|Low Risk|1|0|0| +8194|F|S|Primary|9000|Low Risk|1|0|0| +8195|M|D|Primary|9000|Low Risk|1|0|0| +8196|F|D|Primary|9000|Low Risk|1|0|0| +8197|M|W|Primary|9000|Low Risk|1|0|0| +8198|F|W|Primary|9000|Low Risk|1|0|0| +8199|M|U|Primary|9000|Low Risk|1|0|0| +8200|F|U|Primary|9000|Low Risk|1|0|0| +8201|M|M|Secondary|9000|Low Risk|1|0|0| +8202|F|M|Secondary|9000|Low Risk|1|0|0| +8203|M|S|Secondary|9000|Low Risk|1|0|0| +8204|F|S|Secondary|9000|Low Risk|1|0|0| +8205|M|D|Secondary|9000|Low Risk|1|0|0| +8206|F|D|Secondary|9000|Low Risk|1|0|0| +8207|M|W|Secondary|9000|Low Risk|1|0|0| +8208|F|W|Secondary|9000|Low Risk|1|0|0| +8209|M|U|Secondary|9000|Low Risk|1|0|0| +8210|F|U|Secondary|9000|Low Risk|1|0|0| +8211|M|M|College|9000|Low Risk|1|0|0| +8212|F|M|College|9000|Low Risk|1|0|0| +8213|M|S|College|9000|Low Risk|1|0|0| +8214|F|S|College|9000|Low Risk|1|0|0| +8215|M|D|College|9000|Low Risk|1|0|0| +8216|F|D|College|9000|Low Risk|1|0|0| +8217|M|W|College|9000|Low Risk|1|0|0| +8218|F|W|College|9000|Low Risk|1|0|0| +8219|M|U|College|9000|Low Risk|1|0|0| +8220|F|U|College|9000|Low Risk|1|0|0| +8221|M|M|2 yr Degree|9000|Low Risk|1|0|0| +8222|F|M|2 yr Degree|9000|Low Risk|1|0|0| +8223|M|S|2 yr Degree|9000|Low Risk|1|0|0| +8224|F|S|2 yr Degree|9000|Low Risk|1|0|0| +8225|M|D|2 yr Degree|9000|Low Risk|1|0|0| +8226|F|D|2 yr Degree|9000|Low Risk|1|0|0| +8227|M|W|2 yr Degree|9000|Low Risk|1|0|0| +8228|F|W|2 yr Degree|9000|Low Risk|1|0|0| +8229|M|U|2 yr Degree|9000|Low Risk|1|0|0| +8230|F|U|2 yr Degree|9000|Low Risk|1|0|0| +8231|M|M|4 yr Degree|9000|Low Risk|1|0|0| +8232|F|M|4 yr Degree|9000|Low Risk|1|0|0| +8233|M|S|4 yr Degree|9000|Low Risk|1|0|0| +8234|F|S|4 yr Degree|9000|Low Risk|1|0|0| +8235|M|D|4 yr Degree|9000|Low Risk|1|0|0| +8236|F|D|4 yr Degree|9000|Low Risk|1|0|0| +8237|M|W|4 yr Degree|9000|Low Risk|1|0|0| +8238|F|W|4 yr Degree|9000|Low Risk|1|0|0| +8239|M|U|4 yr Degree|9000|Low Risk|1|0|0| +8240|F|U|4 yr Degree|9000|Low Risk|1|0|0| +8241|M|M|Advanced Degree|9000|Low Risk|1|0|0| +8242|F|M|Advanced Degree|9000|Low Risk|1|0|0| +8243|M|S|Advanced Degree|9000|Low Risk|1|0|0| +8244|F|S|Advanced Degree|9000|Low Risk|1|0|0| +8245|M|D|Advanced Degree|9000|Low Risk|1|0|0| +8246|F|D|Advanced Degree|9000|Low Risk|1|0|0| +8247|M|W|Advanced Degree|9000|Low Risk|1|0|0| +8248|F|W|Advanced Degree|9000|Low Risk|1|0|0| +8249|M|U|Advanced Degree|9000|Low Risk|1|0|0| +8250|F|U|Advanced Degree|9000|Low Risk|1|0|0| +8251|M|M|Unknown|9000|Low Risk|1|0|0| +8252|F|M|Unknown|9000|Low Risk|1|0|0| +8253|M|S|Unknown|9000|Low Risk|1|0|0| +8254|F|S|Unknown|9000|Low Risk|1|0|0| +8255|M|D|Unknown|9000|Low Risk|1|0|0| +8256|F|D|Unknown|9000|Low Risk|1|0|0| +8257|M|W|Unknown|9000|Low Risk|1|0|0| +8258|F|W|Unknown|9000|Low Risk|1|0|0| +8259|M|U|Unknown|9000|Low Risk|1|0|0| +8260|F|U|Unknown|9000|Low Risk|1|0|0| +8261|M|M|Primary|9500|Low Risk|1|0|0| +8262|F|M|Primary|9500|Low Risk|1|0|0| +8263|M|S|Primary|9500|Low Risk|1|0|0| +8264|F|S|Primary|9500|Low Risk|1|0|0| +8265|M|D|Primary|9500|Low Risk|1|0|0| +8266|F|D|Primary|9500|Low Risk|1|0|0| +8267|M|W|Primary|9500|Low Risk|1|0|0| +8268|F|W|Primary|9500|Low Risk|1|0|0| +8269|M|U|Primary|9500|Low Risk|1|0|0| +8270|F|U|Primary|9500|Low Risk|1|0|0| +8271|M|M|Secondary|9500|Low Risk|1|0|0| +8272|F|M|Secondary|9500|Low Risk|1|0|0| +8273|M|S|Secondary|9500|Low Risk|1|0|0| +8274|F|S|Secondary|9500|Low Risk|1|0|0| +8275|M|D|Secondary|9500|Low Risk|1|0|0| +8276|F|D|Secondary|9500|Low Risk|1|0|0| +8277|M|W|Secondary|9500|Low Risk|1|0|0| +8278|F|W|Secondary|9500|Low Risk|1|0|0| +8279|M|U|Secondary|9500|Low Risk|1|0|0| +8280|F|U|Secondary|9500|Low Risk|1|0|0| +8281|M|M|College|9500|Low Risk|1|0|0| +8282|F|M|College|9500|Low Risk|1|0|0| +8283|M|S|College|9500|Low Risk|1|0|0| +8284|F|S|College|9500|Low Risk|1|0|0| +8285|M|D|College|9500|Low Risk|1|0|0| +8286|F|D|College|9500|Low Risk|1|0|0| +8287|M|W|College|9500|Low Risk|1|0|0| +8288|F|W|College|9500|Low Risk|1|0|0| +8289|M|U|College|9500|Low Risk|1|0|0| +8290|F|U|College|9500|Low Risk|1|0|0| +8291|M|M|2 yr Degree|9500|Low Risk|1|0|0| +8292|F|M|2 yr Degree|9500|Low Risk|1|0|0| +8293|M|S|2 yr Degree|9500|Low Risk|1|0|0| +8294|F|S|2 yr Degree|9500|Low Risk|1|0|0| +8295|M|D|2 yr Degree|9500|Low Risk|1|0|0| +8296|F|D|2 yr Degree|9500|Low Risk|1|0|0| +8297|M|W|2 yr Degree|9500|Low Risk|1|0|0| +8298|F|W|2 yr Degree|9500|Low Risk|1|0|0| +8299|M|U|2 yr Degree|9500|Low Risk|1|0|0| +8300|F|U|2 yr Degree|9500|Low Risk|1|0|0| +8301|M|M|4 yr Degree|9500|Low Risk|1|0|0| +8302|F|M|4 yr Degree|9500|Low Risk|1|0|0| +8303|M|S|4 yr Degree|9500|Low Risk|1|0|0| +8304|F|S|4 yr Degree|9500|Low Risk|1|0|0| +8305|M|D|4 yr Degree|9500|Low Risk|1|0|0| +8306|F|D|4 yr Degree|9500|Low Risk|1|0|0| +8307|M|W|4 yr Degree|9500|Low Risk|1|0|0| +8308|F|W|4 yr Degree|9500|Low Risk|1|0|0| +8309|M|U|4 yr Degree|9500|Low Risk|1|0|0| +8310|F|U|4 yr Degree|9500|Low Risk|1|0|0| +8311|M|M|Advanced Degree|9500|Low Risk|1|0|0| +8312|F|M|Advanced Degree|9500|Low Risk|1|0|0| +8313|M|S|Advanced Degree|9500|Low Risk|1|0|0| +8314|F|S|Advanced Degree|9500|Low Risk|1|0|0| +8315|M|D|Advanced Degree|9500|Low Risk|1|0|0| +8316|F|D|Advanced Degree|9500|Low Risk|1|0|0| +8317|M|W|Advanced Degree|9500|Low Risk|1|0|0| +8318|F|W|Advanced Degree|9500|Low Risk|1|0|0| +8319|M|U|Advanced Degree|9500|Low Risk|1|0|0| +8320|F|U|Advanced Degree|9500|Low Risk|1|0|0| +8321|M|M|Unknown|9500|Low Risk|1|0|0| +8322|F|M|Unknown|9500|Low Risk|1|0|0| +8323|M|S|Unknown|9500|Low Risk|1|0|0| +8324|F|S|Unknown|9500|Low Risk|1|0|0| +8325|M|D|Unknown|9500|Low Risk|1|0|0| +8326|F|D|Unknown|9500|Low Risk|1|0|0| +8327|M|W|Unknown|9500|Low Risk|1|0|0| +8328|F|W|Unknown|9500|Low Risk|1|0|0| +8329|M|U|Unknown|9500|Low Risk|1|0|0| +8330|F|U|Unknown|9500|Low Risk|1|0|0| +8331|M|M|Primary|10000|Low Risk|1|0|0| +8332|F|M|Primary|10000|Low Risk|1|0|0| +8333|M|S|Primary|10000|Low Risk|1|0|0| +8334|F|S|Primary|10000|Low Risk|1|0|0| +8335|M|D|Primary|10000|Low Risk|1|0|0| +8336|F|D|Primary|10000|Low Risk|1|0|0| +8337|M|W|Primary|10000|Low Risk|1|0|0| +8338|F|W|Primary|10000|Low Risk|1|0|0| +8339|M|U|Primary|10000|Low Risk|1|0|0| +8340|F|U|Primary|10000|Low Risk|1|0|0| +8341|M|M|Secondary|10000|Low Risk|1|0|0| +8342|F|M|Secondary|10000|Low Risk|1|0|0| +8343|M|S|Secondary|10000|Low Risk|1|0|0| +8344|F|S|Secondary|10000|Low Risk|1|0|0| +8345|M|D|Secondary|10000|Low Risk|1|0|0| +8346|F|D|Secondary|10000|Low Risk|1|0|0| +8347|M|W|Secondary|10000|Low Risk|1|0|0| +8348|F|W|Secondary|10000|Low Risk|1|0|0| +8349|M|U|Secondary|10000|Low Risk|1|0|0| +8350|F|U|Secondary|10000|Low Risk|1|0|0| +8351|M|M|College|10000|Low Risk|1|0|0| +8352|F|M|College|10000|Low Risk|1|0|0| +8353|M|S|College|10000|Low Risk|1|0|0| +8354|F|S|College|10000|Low Risk|1|0|0| +8355|M|D|College|10000|Low Risk|1|0|0| +8356|F|D|College|10000|Low Risk|1|0|0| +8357|M|W|College|10000|Low Risk|1|0|0| +8358|F|W|College|10000|Low Risk|1|0|0| +8359|M|U|College|10000|Low Risk|1|0|0| +8360|F|U|College|10000|Low Risk|1|0|0| +8361|M|M|2 yr Degree|10000|Low Risk|1|0|0| +8362|F|M|2 yr Degree|10000|Low Risk|1|0|0| +8363|M|S|2 yr Degree|10000|Low Risk|1|0|0| +8364|F|S|2 yr Degree|10000|Low Risk|1|0|0| +8365|M|D|2 yr Degree|10000|Low Risk|1|0|0| +8366|F|D|2 yr Degree|10000|Low Risk|1|0|0| +8367|M|W|2 yr Degree|10000|Low Risk|1|0|0| +8368|F|W|2 yr Degree|10000|Low Risk|1|0|0| +8369|M|U|2 yr Degree|10000|Low Risk|1|0|0| +8370|F|U|2 yr Degree|10000|Low Risk|1|0|0| +8371|M|M|4 yr Degree|10000|Low Risk|1|0|0| +8372|F|M|4 yr Degree|10000|Low Risk|1|0|0| +8373|M|S|4 yr Degree|10000|Low Risk|1|0|0| +8374|F|S|4 yr Degree|10000|Low Risk|1|0|0| +8375|M|D|4 yr Degree|10000|Low Risk|1|0|0| +8376|F|D|4 yr Degree|10000|Low Risk|1|0|0| +8377|M|W|4 yr Degree|10000|Low Risk|1|0|0| +8378|F|W|4 yr Degree|10000|Low Risk|1|0|0| +8379|M|U|4 yr Degree|10000|Low Risk|1|0|0| +8380|F|U|4 yr Degree|10000|Low Risk|1|0|0| +8381|M|M|Advanced Degree|10000|Low Risk|1|0|0| +8382|F|M|Advanced Degree|10000|Low Risk|1|0|0| +8383|M|S|Advanced Degree|10000|Low Risk|1|0|0| +8384|F|S|Advanced Degree|10000|Low Risk|1|0|0| +8385|M|D|Advanced Degree|10000|Low Risk|1|0|0| +8386|F|D|Advanced Degree|10000|Low Risk|1|0|0| +8387|M|W|Advanced Degree|10000|Low Risk|1|0|0| +8388|F|W|Advanced Degree|10000|Low Risk|1|0|0| +8389|M|U|Advanced Degree|10000|Low Risk|1|0|0| +8390|F|U|Advanced Degree|10000|Low Risk|1|0|0| +8391|M|M|Unknown|10000|Low Risk|1|0|0| +8392|F|M|Unknown|10000|Low Risk|1|0|0| +8393|M|S|Unknown|10000|Low Risk|1|0|0| +8394|F|S|Unknown|10000|Low Risk|1|0|0| +8395|M|D|Unknown|10000|Low Risk|1|0|0| +8396|F|D|Unknown|10000|Low Risk|1|0|0| +8397|M|W|Unknown|10000|Low Risk|1|0|0| +8398|F|W|Unknown|10000|Low Risk|1|0|0| +8399|M|U|Unknown|10000|Low Risk|1|0|0| +8400|F|U|Unknown|10000|Low Risk|1|0|0| +8401|M|M|Primary|500|High Risk|1|0|0| +8402|F|M|Primary|500|High Risk|1|0|0| +8403|M|S|Primary|500|High Risk|1|0|0| +8404|F|S|Primary|500|High Risk|1|0|0| +8405|M|D|Primary|500|High Risk|1|0|0| +8406|F|D|Primary|500|High Risk|1|0|0| +8407|M|W|Primary|500|High Risk|1|0|0| +8408|F|W|Primary|500|High Risk|1|0|0| +8409|M|U|Primary|500|High Risk|1|0|0| +8410|F|U|Primary|500|High Risk|1|0|0| +8411|M|M|Secondary|500|High Risk|1|0|0| +8412|F|M|Secondary|500|High Risk|1|0|0| +8413|M|S|Secondary|500|High Risk|1|0|0| +8414|F|S|Secondary|500|High Risk|1|0|0| +8415|M|D|Secondary|500|High Risk|1|0|0| +8416|F|D|Secondary|500|High Risk|1|0|0| +8417|M|W|Secondary|500|High Risk|1|0|0| +8418|F|W|Secondary|500|High Risk|1|0|0| +8419|M|U|Secondary|500|High Risk|1|0|0| +8420|F|U|Secondary|500|High Risk|1|0|0| +8421|M|M|College|500|High Risk|1|0|0| +8422|F|M|College|500|High Risk|1|0|0| +8423|M|S|College|500|High Risk|1|0|0| +8424|F|S|College|500|High Risk|1|0|0| +8425|M|D|College|500|High Risk|1|0|0| +8426|F|D|College|500|High Risk|1|0|0| +8427|M|W|College|500|High Risk|1|0|0| +8428|F|W|College|500|High Risk|1|0|0| +8429|M|U|College|500|High Risk|1|0|0| +8430|F|U|College|500|High Risk|1|0|0| +8431|M|M|2 yr Degree|500|High Risk|1|0|0| +8432|F|M|2 yr Degree|500|High Risk|1|0|0| +8433|M|S|2 yr Degree|500|High Risk|1|0|0| +8434|F|S|2 yr Degree|500|High Risk|1|0|0| +8435|M|D|2 yr Degree|500|High Risk|1|0|0| +8436|F|D|2 yr Degree|500|High Risk|1|0|0| +8437|M|W|2 yr Degree|500|High Risk|1|0|0| +8438|F|W|2 yr Degree|500|High Risk|1|0|0| +8439|M|U|2 yr Degree|500|High Risk|1|0|0| +8440|F|U|2 yr Degree|500|High Risk|1|0|0| +8441|M|M|4 yr Degree|500|High Risk|1|0|0| +8442|F|M|4 yr Degree|500|High Risk|1|0|0| +8443|M|S|4 yr Degree|500|High Risk|1|0|0| +8444|F|S|4 yr Degree|500|High Risk|1|0|0| +8445|M|D|4 yr Degree|500|High Risk|1|0|0| +8446|F|D|4 yr Degree|500|High Risk|1|0|0| +8447|M|W|4 yr Degree|500|High Risk|1|0|0| +8448|F|W|4 yr Degree|500|High Risk|1|0|0| +8449|M|U|4 yr Degree|500|High Risk|1|0|0| +8450|F|U|4 yr Degree|500|High Risk|1|0|0| +8451|M|M|Advanced Degree|500|High Risk|1|0|0| +8452|F|M|Advanced Degree|500|High Risk|1|0|0| +8453|M|S|Advanced Degree|500|High Risk|1|0|0| +8454|F|S|Advanced Degree|500|High Risk|1|0|0| +8455|M|D|Advanced Degree|500|High Risk|1|0|0| +8456|F|D|Advanced Degree|500|High Risk|1|0|0| +8457|M|W|Advanced Degree|500|High Risk|1|0|0| +8458|F|W|Advanced Degree|500|High Risk|1|0|0| +8459|M|U|Advanced Degree|500|High Risk|1|0|0| +8460|F|U|Advanced Degree|500|High Risk|1|0|0| +8461|M|M|Unknown|500|High Risk|1|0|0| +8462|F|M|Unknown|500|High Risk|1|0|0| +8463|M|S|Unknown|500|High Risk|1|0|0| +8464|F|S|Unknown|500|High Risk|1|0|0| +8465|M|D|Unknown|500|High Risk|1|0|0| +8466|F|D|Unknown|500|High Risk|1|0|0| +8467|M|W|Unknown|500|High Risk|1|0|0| +8468|F|W|Unknown|500|High Risk|1|0|0| +8469|M|U|Unknown|500|High Risk|1|0|0| +8470|F|U|Unknown|500|High Risk|1|0|0| +8471|M|M|Primary|1000|High Risk|1|0|0| +8472|F|M|Primary|1000|High Risk|1|0|0| +8473|M|S|Primary|1000|High Risk|1|0|0| +8474|F|S|Primary|1000|High Risk|1|0|0| +8475|M|D|Primary|1000|High Risk|1|0|0| +8476|F|D|Primary|1000|High Risk|1|0|0| +8477|M|W|Primary|1000|High Risk|1|0|0| +8478|F|W|Primary|1000|High Risk|1|0|0| +8479|M|U|Primary|1000|High Risk|1|0|0| +8480|F|U|Primary|1000|High Risk|1|0|0| +8481|M|M|Secondary|1000|High Risk|1|0|0| +8482|F|M|Secondary|1000|High Risk|1|0|0| +8483|M|S|Secondary|1000|High Risk|1|0|0| +8484|F|S|Secondary|1000|High Risk|1|0|0| +8485|M|D|Secondary|1000|High Risk|1|0|0| +8486|F|D|Secondary|1000|High Risk|1|0|0| +8487|M|W|Secondary|1000|High Risk|1|0|0| +8488|F|W|Secondary|1000|High Risk|1|0|0| +8489|M|U|Secondary|1000|High Risk|1|0|0| +8490|F|U|Secondary|1000|High Risk|1|0|0| +8491|M|M|College|1000|High Risk|1|0|0| +8492|F|M|College|1000|High Risk|1|0|0| +8493|M|S|College|1000|High Risk|1|0|0| +8494|F|S|College|1000|High Risk|1|0|0| +8495|M|D|College|1000|High Risk|1|0|0| +8496|F|D|College|1000|High Risk|1|0|0| +8497|M|W|College|1000|High Risk|1|0|0| +8498|F|W|College|1000|High Risk|1|0|0| +8499|M|U|College|1000|High Risk|1|0|0| +8500|F|U|College|1000|High Risk|1|0|0| +8501|M|M|2 yr Degree|1000|High Risk|1|0|0| +8502|F|M|2 yr Degree|1000|High Risk|1|0|0| +8503|M|S|2 yr Degree|1000|High Risk|1|0|0| +8504|F|S|2 yr Degree|1000|High Risk|1|0|0| +8505|M|D|2 yr Degree|1000|High Risk|1|0|0| +8506|F|D|2 yr Degree|1000|High Risk|1|0|0| +8507|M|W|2 yr Degree|1000|High Risk|1|0|0| +8508|F|W|2 yr Degree|1000|High Risk|1|0|0| +8509|M|U|2 yr Degree|1000|High Risk|1|0|0| +8510|F|U|2 yr Degree|1000|High Risk|1|0|0| +8511|M|M|4 yr Degree|1000|High Risk|1|0|0| +8512|F|M|4 yr Degree|1000|High Risk|1|0|0| +8513|M|S|4 yr Degree|1000|High Risk|1|0|0| +8514|F|S|4 yr Degree|1000|High Risk|1|0|0| +8515|M|D|4 yr Degree|1000|High Risk|1|0|0| +8516|F|D|4 yr Degree|1000|High Risk|1|0|0| +8517|M|W|4 yr Degree|1000|High Risk|1|0|0| +8518|F|W|4 yr Degree|1000|High Risk|1|0|0| +8519|M|U|4 yr Degree|1000|High Risk|1|0|0| +8520|F|U|4 yr Degree|1000|High Risk|1|0|0| +8521|M|M|Advanced Degree|1000|High Risk|1|0|0| +8522|F|M|Advanced Degree|1000|High Risk|1|0|0| +8523|M|S|Advanced Degree|1000|High Risk|1|0|0| +8524|F|S|Advanced Degree|1000|High Risk|1|0|0| +8525|M|D|Advanced Degree|1000|High Risk|1|0|0| +8526|F|D|Advanced Degree|1000|High Risk|1|0|0| +8527|M|W|Advanced Degree|1000|High Risk|1|0|0| +8528|F|W|Advanced Degree|1000|High Risk|1|0|0| +8529|M|U|Advanced Degree|1000|High Risk|1|0|0| +8530|F|U|Advanced Degree|1000|High Risk|1|0|0| +8531|M|M|Unknown|1000|High Risk|1|0|0| +8532|F|M|Unknown|1000|High Risk|1|0|0| +8533|M|S|Unknown|1000|High Risk|1|0|0| +8534|F|S|Unknown|1000|High Risk|1|0|0| +8535|M|D|Unknown|1000|High Risk|1|0|0| +8536|F|D|Unknown|1000|High Risk|1|0|0| +8537|M|W|Unknown|1000|High Risk|1|0|0| +8538|F|W|Unknown|1000|High Risk|1|0|0| +8539|M|U|Unknown|1000|High Risk|1|0|0| +8540|F|U|Unknown|1000|High Risk|1|0|0| +8541|M|M|Primary|1500|High Risk|1|0|0| +8542|F|M|Primary|1500|High Risk|1|0|0| +8543|M|S|Primary|1500|High Risk|1|0|0| +8544|F|S|Primary|1500|High Risk|1|0|0| +8545|M|D|Primary|1500|High Risk|1|0|0| +8546|F|D|Primary|1500|High Risk|1|0|0| +8547|M|W|Primary|1500|High Risk|1|0|0| +8548|F|W|Primary|1500|High Risk|1|0|0| +8549|M|U|Primary|1500|High Risk|1|0|0| +8550|F|U|Primary|1500|High Risk|1|0|0| +8551|M|M|Secondary|1500|High Risk|1|0|0| +8552|F|M|Secondary|1500|High Risk|1|0|0| +8553|M|S|Secondary|1500|High Risk|1|0|0| +8554|F|S|Secondary|1500|High Risk|1|0|0| +8555|M|D|Secondary|1500|High Risk|1|0|0| +8556|F|D|Secondary|1500|High Risk|1|0|0| +8557|M|W|Secondary|1500|High Risk|1|0|0| +8558|F|W|Secondary|1500|High Risk|1|0|0| +8559|M|U|Secondary|1500|High Risk|1|0|0| +8560|F|U|Secondary|1500|High Risk|1|0|0| +8561|M|M|College|1500|High Risk|1|0|0| +8562|F|M|College|1500|High Risk|1|0|0| +8563|M|S|College|1500|High Risk|1|0|0| +8564|F|S|College|1500|High Risk|1|0|0| +8565|M|D|College|1500|High Risk|1|0|0| +8566|F|D|College|1500|High Risk|1|0|0| +8567|M|W|College|1500|High Risk|1|0|0| +8568|F|W|College|1500|High Risk|1|0|0| +8569|M|U|College|1500|High Risk|1|0|0| +8570|F|U|College|1500|High Risk|1|0|0| +8571|M|M|2 yr Degree|1500|High Risk|1|0|0| +8572|F|M|2 yr Degree|1500|High Risk|1|0|0| +8573|M|S|2 yr Degree|1500|High Risk|1|0|0| +8574|F|S|2 yr Degree|1500|High Risk|1|0|0| +8575|M|D|2 yr Degree|1500|High Risk|1|0|0| +8576|F|D|2 yr Degree|1500|High Risk|1|0|0| +8577|M|W|2 yr Degree|1500|High Risk|1|0|0| +8578|F|W|2 yr Degree|1500|High Risk|1|0|0| +8579|M|U|2 yr Degree|1500|High Risk|1|0|0| +8580|F|U|2 yr Degree|1500|High Risk|1|0|0| +8581|M|M|4 yr Degree|1500|High Risk|1|0|0| +8582|F|M|4 yr Degree|1500|High Risk|1|0|0| +8583|M|S|4 yr Degree|1500|High Risk|1|0|0| +8584|F|S|4 yr Degree|1500|High Risk|1|0|0| +8585|M|D|4 yr Degree|1500|High Risk|1|0|0| +8586|F|D|4 yr Degree|1500|High Risk|1|0|0| +8587|M|W|4 yr Degree|1500|High Risk|1|0|0| +8588|F|W|4 yr Degree|1500|High Risk|1|0|0| +8589|M|U|4 yr Degree|1500|High Risk|1|0|0| +8590|F|U|4 yr Degree|1500|High Risk|1|0|0| +8591|M|M|Advanced Degree|1500|High Risk|1|0|0| +8592|F|M|Advanced Degree|1500|High Risk|1|0|0| +8593|M|S|Advanced Degree|1500|High Risk|1|0|0| +8594|F|S|Advanced Degree|1500|High Risk|1|0|0| +8595|M|D|Advanced Degree|1500|High Risk|1|0|0| +8596|F|D|Advanced Degree|1500|High Risk|1|0|0| +8597|M|W|Advanced Degree|1500|High Risk|1|0|0| +8598|F|W|Advanced Degree|1500|High Risk|1|0|0| +8599|M|U|Advanced Degree|1500|High Risk|1|0|0| +8600|F|U|Advanced Degree|1500|High Risk|1|0|0| +8601|M|M|Unknown|1500|High Risk|1|0|0| +8602|F|M|Unknown|1500|High Risk|1|0|0| +8603|M|S|Unknown|1500|High Risk|1|0|0| +8604|F|S|Unknown|1500|High Risk|1|0|0| +8605|M|D|Unknown|1500|High Risk|1|0|0| +8606|F|D|Unknown|1500|High Risk|1|0|0| +8607|M|W|Unknown|1500|High Risk|1|0|0| +8608|F|W|Unknown|1500|High Risk|1|0|0| +8609|M|U|Unknown|1500|High Risk|1|0|0| +8610|F|U|Unknown|1500|High Risk|1|0|0| +8611|M|M|Primary|2000|High Risk|1|0|0| +8612|F|M|Primary|2000|High Risk|1|0|0| +8613|M|S|Primary|2000|High Risk|1|0|0| +8614|F|S|Primary|2000|High Risk|1|0|0| +8615|M|D|Primary|2000|High Risk|1|0|0| +8616|F|D|Primary|2000|High Risk|1|0|0| +8617|M|W|Primary|2000|High Risk|1|0|0| +8618|F|W|Primary|2000|High Risk|1|0|0| +8619|M|U|Primary|2000|High Risk|1|0|0| +8620|F|U|Primary|2000|High Risk|1|0|0| +8621|M|M|Secondary|2000|High Risk|1|0|0| +8622|F|M|Secondary|2000|High Risk|1|0|0| +8623|M|S|Secondary|2000|High Risk|1|0|0| +8624|F|S|Secondary|2000|High Risk|1|0|0| +8625|M|D|Secondary|2000|High Risk|1|0|0| +8626|F|D|Secondary|2000|High Risk|1|0|0| +8627|M|W|Secondary|2000|High Risk|1|0|0| +8628|F|W|Secondary|2000|High Risk|1|0|0| +8629|M|U|Secondary|2000|High Risk|1|0|0| +8630|F|U|Secondary|2000|High Risk|1|0|0| +8631|M|M|College|2000|High Risk|1|0|0| +8632|F|M|College|2000|High Risk|1|0|0| +8633|M|S|College|2000|High Risk|1|0|0| +8634|F|S|College|2000|High Risk|1|0|0| +8635|M|D|College|2000|High Risk|1|0|0| +8636|F|D|College|2000|High Risk|1|0|0| +8637|M|W|College|2000|High Risk|1|0|0| +8638|F|W|College|2000|High Risk|1|0|0| +8639|M|U|College|2000|High Risk|1|0|0| +8640|F|U|College|2000|High Risk|1|0|0| +8641|M|M|2 yr Degree|2000|High Risk|1|0|0| +8642|F|M|2 yr Degree|2000|High Risk|1|0|0| +8643|M|S|2 yr Degree|2000|High Risk|1|0|0| +8644|F|S|2 yr Degree|2000|High Risk|1|0|0| +8645|M|D|2 yr Degree|2000|High Risk|1|0|0| +8646|F|D|2 yr Degree|2000|High Risk|1|0|0| +8647|M|W|2 yr Degree|2000|High Risk|1|0|0| +8648|F|W|2 yr Degree|2000|High Risk|1|0|0| +8649|M|U|2 yr Degree|2000|High Risk|1|0|0| +8650|F|U|2 yr Degree|2000|High Risk|1|0|0| +8651|M|M|4 yr Degree|2000|High Risk|1|0|0| +8652|F|M|4 yr Degree|2000|High Risk|1|0|0| +8653|M|S|4 yr Degree|2000|High Risk|1|0|0| +8654|F|S|4 yr Degree|2000|High Risk|1|0|0| +8655|M|D|4 yr Degree|2000|High Risk|1|0|0| +8656|F|D|4 yr Degree|2000|High Risk|1|0|0| +8657|M|W|4 yr Degree|2000|High Risk|1|0|0| +8658|F|W|4 yr Degree|2000|High Risk|1|0|0| +8659|M|U|4 yr Degree|2000|High Risk|1|0|0| +8660|F|U|4 yr Degree|2000|High Risk|1|0|0| +8661|M|M|Advanced Degree|2000|High Risk|1|0|0| +8662|F|M|Advanced Degree|2000|High Risk|1|0|0| +8663|M|S|Advanced Degree|2000|High Risk|1|0|0| +8664|F|S|Advanced Degree|2000|High Risk|1|0|0| +8665|M|D|Advanced Degree|2000|High Risk|1|0|0| +8666|F|D|Advanced Degree|2000|High Risk|1|0|0| +8667|M|W|Advanced Degree|2000|High Risk|1|0|0| +8668|F|W|Advanced Degree|2000|High Risk|1|0|0| +8669|M|U|Advanced Degree|2000|High Risk|1|0|0| +8670|F|U|Advanced Degree|2000|High Risk|1|0|0| +8671|M|M|Unknown|2000|High Risk|1|0|0| +8672|F|M|Unknown|2000|High Risk|1|0|0| +8673|M|S|Unknown|2000|High Risk|1|0|0| +8674|F|S|Unknown|2000|High Risk|1|0|0| +8675|M|D|Unknown|2000|High Risk|1|0|0| +8676|F|D|Unknown|2000|High Risk|1|0|0| +8677|M|W|Unknown|2000|High Risk|1|0|0| +8678|F|W|Unknown|2000|High Risk|1|0|0| +8679|M|U|Unknown|2000|High Risk|1|0|0| +8680|F|U|Unknown|2000|High Risk|1|0|0| +8681|M|M|Primary|2500|High Risk|1|0|0| +8682|F|M|Primary|2500|High Risk|1|0|0| +8683|M|S|Primary|2500|High Risk|1|0|0| +8684|F|S|Primary|2500|High Risk|1|0|0| +8685|M|D|Primary|2500|High Risk|1|0|0| +8686|F|D|Primary|2500|High Risk|1|0|0| +8687|M|W|Primary|2500|High Risk|1|0|0| +8688|F|W|Primary|2500|High Risk|1|0|0| +8689|M|U|Primary|2500|High Risk|1|0|0| +8690|F|U|Primary|2500|High Risk|1|0|0| +8691|M|M|Secondary|2500|High Risk|1|0|0| +8692|F|M|Secondary|2500|High Risk|1|0|0| +8693|M|S|Secondary|2500|High Risk|1|0|0| +8694|F|S|Secondary|2500|High Risk|1|0|0| +8695|M|D|Secondary|2500|High Risk|1|0|0| +8696|F|D|Secondary|2500|High Risk|1|0|0| +8697|M|W|Secondary|2500|High Risk|1|0|0| +8698|F|W|Secondary|2500|High Risk|1|0|0| +8699|M|U|Secondary|2500|High Risk|1|0|0| +8700|F|U|Secondary|2500|High Risk|1|0|0| +8701|M|M|College|2500|High Risk|1|0|0| +8702|F|M|College|2500|High Risk|1|0|0| +8703|M|S|College|2500|High Risk|1|0|0| +8704|F|S|College|2500|High Risk|1|0|0| +8705|M|D|College|2500|High Risk|1|0|0| +8706|F|D|College|2500|High Risk|1|0|0| +8707|M|W|College|2500|High Risk|1|0|0| +8708|F|W|College|2500|High Risk|1|0|0| +8709|M|U|College|2500|High Risk|1|0|0| +8710|F|U|College|2500|High Risk|1|0|0| +8711|M|M|2 yr Degree|2500|High Risk|1|0|0| +8712|F|M|2 yr Degree|2500|High Risk|1|0|0| +8713|M|S|2 yr Degree|2500|High Risk|1|0|0| +8714|F|S|2 yr Degree|2500|High Risk|1|0|0| +8715|M|D|2 yr Degree|2500|High Risk|1|0|0| +8716|F|D|2 yr Degree|2500|High Risk|1|0|0| +8717|M|W|2 yr Degree|2500|High Risk|1|0|0| +8718|F|W|2 yr Degree|2500|High Risk|1|0|0| +8719|M|U|2 yr Degree|2500|High Risk|1|0|0| +8720|F|U|2 yr Degree|2500|High Risk|1|0|0| +8721|M|M|4 yr Degree|2500|High Risk|1|0|0| +8722|F|M|4 yr Degree|2500|High Risk|1|0|0| +8723|M|S|4 yr Degree|2500|High Risk|1|0|0| +8724|F|S|4 yr Degree|2500|High Risk|1|0|0| +8725|M|D|4 yr Degree|2500|High Risk|1|0|0| +8726|F|D|4 yr Degree|2500|High Risk|1|0|0| +8727|M|W|4 yr Degree|2500|High Risk|1|0|0| +8728|F|W|4 yr Degree|2500|High Risk|1|0|0| +8729|M|U|4 yr Degree|2500|High Risk|1|0|0| +8730|F|U|4 yr Degree|2500|High Risk|1|0|0| +8731|M|M|Advanced Degree|2500|High Risk|1|0|0| +8732|F|M|Advanced Degree|2500|High Risk|1|0|0| +8733|M|S|Advanced Degree|2500|High Risk|1|0|0| +8734|F|S|Advanced Degree|2500|High Risk|1|0|0| +8735|M|D|Advanced Degree|2500|High Risk|1|0|0| +8736|F|D|Advanced Degree|2500|High Risk|1|0|0| +8737|M|W|Advanced Degree|2500|High Risk|1|0|0| +8738|F|W|Advanced Degree|2500|High Risk|1|0|0| +8739|M|U|Advanced Degree|2500|High Risk|1|0|0| +8740|F|U|Advanced Degree|2500|High Risk|1|0|0| +8741|M|M|Unknown|2500|High Risk|1|0|0| +8742|F|M|Unknown|2500|High Risk|1|0|0| +8743|M|S|Unknown|2500|High Risk|1|0|0| +8744|F|S|Unknown|2500|High Risk|1|0|0| +8745|M|D|Unknown|2500|High Risk|1|0|0| +8746|F|D|Unknown|2500|High Risk|1|0|0| +8747|M|W|Unknown|2500|High Risk|1|0|0| +8748|F|W|Unknown|2500|High Risk|1|0|0| +8749|M|U|Unknown|2500|High Risk|1|0|0| +8750|F|U|Unknown|2500|High Risk|1|0|0| +8751|M|M|Primary|3000|High Risk|1|0|0| +8752|F|M|Primary|3000|High Risk|1|0|0| +8753|M|S|Primary|3000|High Risk|1|0|0| +8754|F|S|Primary|3000|High Risk|1|0|0| +8755|M|D|Primary|3000|High Risk|1|0|0| +8756|F|D|Primary|3000|High Risk|1|0|0| +8757|M|W|Primary|3000|High Risk|1|0|0| +8758|F|W|Primary|3000|High Risk|1|0|0| +8759|M|U|Primary|3000|High Risk|1|0|0| +8760|F|U|Primary|3000|High Risk|1|0|0| +8761|M|M|Secondary|3000|High Risk|1|0|0| +8762|F|M|Secondary|3000|High Risk|1|0|0| +8763|M|S|Secondary|3000|High Risk|1|0|0| +8764|F|S|Secondary|3000|High Risk|1|0|0| +8765|M|D|Secondary|3000|High Risk|1|0|0| +8766|F|D|Secondary|3000|High Risk|1|0|0| +8767|M|W|Secondary|3000|High Risk|1|0|0| +8768|F|W|Secondary|3000|High Risk|1|0|0| +8769|M|U|Secondary|3000|High Risk|1|0|0| +8770|F|U|Secondary|3000|High Risk|1|0|0| +8771|M|M|College|3000|High Risk|1|0|0| +8772|F|M|College|3000|High Risk|1|0|0| +8773|M|S|College|3000|High Risk|1|0|0| +8774|F|S|College|3000|High Risk|1|0|0| +8775|M|D|College|3000|High Risk|1|0|0| +8776|F|D|College|3000|High Risk|1|0|0| +8777|M|W|College|3000|High Risk|1|0|0| +8778|F|W|College|3000|High Risk|1|0|0| +8779|M|U|College|3000|High Risk|1|0|0| +8780|F|U|College|3000|High Risk|1|0|0| +8781|M|M|2 yr Degree|3000|High Risk|1|0|0| +8782|F|M|2 yr Degree|3000|High Risk|1|0|0| +8783|M|S|2 yr Degree|3000|High Risk|1|0|0| +8784|F|S|2 yr Degree|3000|High Risk|1|0|0| +8785|M|D|2 yr Degree|3000|High Risk|1|0|0| +8786|F|D|2 yr Degree|3000|High Risk|1|0|0| +8787|M|W|2 yr Degree|3000|High Risk|1|0|0| +8788|F|W|2 yr Degree|3000|High Risk|1|0|0| +8789|M|U|2 yr Degree|3000|High Risk|1|0|0| +8790|F|U|2 yr Degree|3000|High Risk|1|0|0| +8791|M|M|4 yr Degree|3000|High Risk|1|0|0| +8792|F|M|4 yr Degree|3000|High Risk|1|0|0| +8793|M|S|4 yr Degree|3000|High Risk|1|0|0| +8794|F|S|4 yr Degree|3000|High Risk|1|0|0| +8795|M|D|4 yr Degree|3000|High Risk|1|0|0| +8796|F|D|4 yr Degree|3000|High Risk|1|0|0| +8797|M|W|4 yr Degree|3000|High Risk|1|0|0| +8798|F|W|4 yr Degree|3000|High Risk|1|0|0| +8799|M|U|4 yr Degree|3000|High Risk|1|0|0| +8800|F|U|4 yr Degree|3000|High Risk|1|0|0| +8801|M|M|Advanced Degree|3000|High Risk|1|0|0| +8802|F|M|Advanced Degree|3000|High Risk|1|0|0| +8803|M|S|Advanced Degree|3000|High Risk|1|0|0| +8804|F|S|Advanced Degree|3000|High Risk|1|0|0| +8805|M|D|Advanced Degree|3000|High Risk|1|0|0| +8806|F|D|Advanced Degree|3000|High Risk|1|0|0| +8807|M|W|Advanced Degree|3000|High Risk|1|0|0| +8808|F|W|Advanced Degree|3000|High Risk|1|0|0| +8809|M|U|Advanced Degree|3000|High Risk|1|0|0| +8810|F|U|Advanced Degree|3000|High Risk|1|0|0| +8811|M|M|Unknown|3000|High Risk|1|0|0| +8812|F|M|Unknown|3000|High Risk|1|0|0| +8813|M|S|Unknown|3000|High Risk|1|0|0| +8814|F|S|Unknown|3000|High Risk|1|0|0| +8815|M|D|Unknown|3000|High Risk|1|0|0| +8816|F|D|Unknown|3000|High Risk|1|0|0| +8817|M|W|Unknown|3000|High Risk|1|0|0| +8818|F|W|Unknown|3000|High Risk|1|0|0| +8819|M|U|Unknown|3000|High Risk|1|0|0| +8820|F|U|Unknown|3000|High Risk|1|0|0| +8821|M|M|Primary|3500|High Risk|1|0|0| +8822|F|M|Primary|3500|High Risk|1|0|0| +8823|M|S|Primary|3500|High Risk|1|0|0| +8824|F|S|Primary|3500|High Risk|1|0|0| +8825|M|D|Primary|3500|High Risk|1|0|0| +8826|F|D|Primary|3500|High Risk|1|0|0| +8827|M|W|Primary|3500|High Risk|1|0|0| +8828|F|W|Primary|3500|High Risk|1|0|0| +8829|M|U|Primary|3500|High Risk|1|0|0| +8830|F|U|Primary|3500|High Risk|1|0|0| +8831|M|M|Secondary|3500|High Risk|1|0|0| +8832|F|M|Secondary|3500|High Risk|1|0|0| +8833|M|S|Secondary|3500|High Risk|1|0|0| +8834|F|S|Secondary|3500|High Risk|1|0|0| +8835|M|D|Secondary|3500|High Risk|1|0|0| +8836|F|D|Secondary|3500|High Risk|1|0|0| +8837|M|W|Secondary|3500|High Risk|1|0|0| +8838|F|W|Secondary|3500|High Risk|1|0|0| +8839|M|U|Secondary|3500|High Risk|1|0|0| +8840|F|U|Secondary|3500|High Risk|1|0|0| +8841|M|M|College|3500|High Risk|1|0|0| +8842|F|M|College|3500|High Risk|1|0|0| +8843|M|S|College|3500|High Risk|1|0|0| +8844|F|S|College|3500|High Risk|1|0|0| +8845|M|D|College|3500|High Risk|1|0|0| +8846|F|D|College|3500|High Risk|1|0|0| +8847|M|W|College|3500|High Risk|1|0|0| +8848|F|W|College|3500|High Risk|1|0|0| +8849|M|U|College|3500|High Risk|1|0|0| +8850|F|U|College|3500|High Risk|1|0|0| +8851|M|M|2 yr Degree|3500|High Risk|1|0|0| +8852|F|M|2 yr Degree|3500|High Risk|1|0|0| +8853|M|S|2 yr Degree|3500|High Risk|1|0|0| +8854|F|S|2 yr Degree|3500|High Risk|1|0|0| +8855|M|D|2 yr Degree|3500|High Risk|1|0|0| +8856|F|D|2 yr Degree|3500|High Risk|1|0|0| +8857|M|W|2 yr Degree|3500|High Risk|1|0|0| +8858|F|W|2 yr Degree|3500|High Risk|1|0|0| +8859|M|U|2 yr Degree|3500|High Risk|1|0|0| +8860|F|U|2 yr Degree|3500|High Risk|1|0|0| +8861|M|M|4 yr Degree|3500|High Risk|1|0|0| +8862|F|M|4 yr Degree|3500|High Risk|1|0|0| +8863|M|S|4 yr Degree|3500|High Risk|1|0|0| +8864|F|S|4 yr Degree|3500|High Risk|1|0|0| +8865|M|D|4 yr Degree|3500|High Risk|1|0|0| +8866|F|D|4 yr Degree|3500|High Risk|1|0|0| +8867|M|W|4 yr Degree|3500|High Risk|1|0|0| +8868|F|W|4 yr Degree|3500|High Risk|1|0|0| +8869|M|U|4 yr Degree|3500|High Risk|1|0|0| +8870|F|U|4 yr Degree|3500|High Risk|1|0|0| +8871|M|M|Advanced Degree|3500|High Risk|1|0|0| +8872|F|M|Advanced Degree|3500|High Risk|1|0|0| +8873|M|S|Advanced Degree|3500|High Risk|1|0|0| +8874|F|S|Advanced Degree|3500|High Risk|1|0|0| +8875|M|D|Advanced Degree|3500|High Risk|1|0|0| +8876|F|D|Advanced Degree|3500|High Risk|1|0|0| +8877|M|W|Advanced Degree|3500|High Risk|1|0|0| +8878|F|W|Advanced Degree|3500|High Risk|1|0|0| +8879|M|U|Advanced Degree|3500|High Risk|1|0|0| +8880|F|U|Advanced Degree|3500|High Risk|1|0|0| +8881|M|M|Unknown|3500|High Risk|1|0|0| +8882|F|M|Unknown|3500|High Risk|1|0|0| +8883|M|S|Unknown|3500|High Risk|1|0|0| +8884|F|S|Unknown|3500|High Risk|1|0|0| +8885|M|D|Unknown|3500|High Risk|1|0|0| +8886|F|D|Unknown|3500|High Risk|1|0|0| +8887|M|W|Unknown|3500|High Risk|1|0|0| +8888|F|W|Unknown|3500|High Risk|1|0|0| +8889|M|U|Unknown|3500|High Risk|1|0|0| +8890|F|U|Unknown|3500|High Risk|1|0|0| +8891|M|M|Primary|4000|High Risk|1|0|0| +8892|F|M|Primary|4000|High Risk|1|0|0| +8893|M|S|Primary|4000|High Risk|1|0|0| +8894|F|S|Primary|4000|High Risk|1|0|0| +8895|M|D|Primary|4000|High Risk|1|0|0| +8896|F|D|Primary|4000|High Risk|1|0|0| +8897|M|W|Primary|4000|High Risk|1|0|0| +8898|F|W|Primary|4000|High Risk|1|0|0| +8899|M|U|Primary|4000|High Risk|1|0|0| +8900|F|U|Primary|4000|High Risk|1|0|0| +8901|M|M|Secondary|4000|High Risk|1|0|0| +8902|F|M|Secondary|4000|High Risk|1|0|0| +8903|M|S|Secondary|4000|High Risk|1|0|0| +8904|F|S|Secondary|4000|High Risk|1|0|0| +8905|M|D|Secondary|4000|High Risk|1|0|0| +8906|F|D|Secondary|4000|High Risk|1|0|0| +8907|M|W|Secondary|4000|High Risk|1|0|0| +8908|F|W|Secondary|4000|High Risk|1|0|0| +8909|M|U|Secondary|4000|High Risk|1|0|0| +8910|F|U|Secondary|4000|High Risk|1|0|0| +8911|M|M|College|4000|High Risk|1|0|0| +8912|F|M|College|4000|High Risk|1|0|0| +8913|M|S|College|4000|High Risk|1|0|0| +8914|F|S|College|4000|High Risk|1|0|0| +8915|M|D|College|4000|High Risk|1|0|0| +8916|F|D|College|4000|High Risk|1|0|0| +8917|M|W|College|4000|High Risk|1|0|0| +8918|F|W|College|4000|High Risk|1|0|0| +8919|M|U|College|4000|High Risk|1|0|0| +8920|F|U|College|4000|High Risk|1|0|0| +8921|M|M|2 yr Degree|4000|High Risk|1|0|0| +8922|F|M|2 yr Degree|4000|High Risk|1|0|0| +8923|M|S|2 yr Degree|4000|High Risk|1|0|0| +8924|F|S|2 yr Degree|4000|High Risk|1|0|0| +8925|M|D|2 yr Degree|4000|High Risk|1|0|0| +8926|F|D|2 yr Degree|4000|High Risk|1|0|0| +8927|M|W|2 yr Degree|4000|High Risk|1|0|0| +8928|F|W|2 yr Degree|4000|High Risk|1|0|0| +8929|M|U|2 yr Degree|4000|High Risk|1|0|0| +8930|F|U|2 yr Degree|4000|High Risk|1|0|0| +8931|M|M|4 yr Degree|4000|High Risk|1|0|0| +8932|F|M|4 yr Degree|4000|High Risk|1|0|0| +8933|M|S|4 yr Degree|4000|High Risk|1|0|0| +8934|F|S|4 yr Degree|4000|High Risk|1|0|0| +8935|M|D|4 yr Degree|4000|High Risk|1|0|0| +8936|F|D|4 yr Degree|4000|High Risk|1|0|0| +8937|M|W|4 yr Degree|4000|High Risk|1|0|0| +8938|F|W|4 yr Degree|4000|High Risk|1|0|0| +8939|M|U|4 yr Degree|4000|High Risk|1|0|0| +8940|F|U|4 yr Degree|4000|High Risk|1|0|0| +8941|M|M|Advanced Degree|4000|High Risk|1|0|0| +8942|F|M|Advanced Degree|4000|High Risk|1|0|0| +8943|M|S|Advanced Degree|4000|High Risk|1|0|0| +8944|F|S|Advanced Degree|4000|High Risk|1|0|0| +8945|M|D|Advanced Degree|4000|High Risk|1|0|0| +8946|F|D|Advanced Degree|4000|High Risk|1|0|0| +8947|M|W|Advanced Degree|4000|High Risk|1|0|0| +8948|F|W|Advanced Degree|4000|High Risk|1|0|0| +8949|M|U|Advanced Degree|4000|High Risk|1|0|0| +8950|F|U|Advanced Degree|4000|High Risk|1|0|0| +8951|M|M|Unknown|4000|High Risk|1|0|0| +8952|F|M|Unknown|4000|High Risk|1|0|0| +8953|M|S|Unknown|4000|High Risk|1|0|0| +8954|F|S|Unknown|4000|High Risk|1|0|0| +8955|M|D|Unknown|4000|High Risk|1|0|0| +8956|F|D|Unknown|4000|High Risk|1|0|0| +8957|M|W|Unknown|4000|High Risk|1|0|0| +8958|F|W|Unknown|4000|High Risk|1|0|0| +8959|M|U|Unknown|4000|High Risk|1|0|0| +8960|F|U|Unknown|4000|High Risk|1|0|0| +8961|M|M|Primary|4500|High Risk|1|0|0| +8962|F|M|Primary|4500|High Risk|1|0|0| +8963|M|S|Primary|4500|High Risk|1|0|0| +8964|F|S|Primary|4500|High Risk|1|0|0| +8965|M|D|Primary|4500|High Risk|1|0|0| +8966|F|D|Primary|4500|High Risk|1|0|0| +8967|M|W|Primary|4500|High Risk|1|0|0| +8968|F|W|Primary|4500|High Risk|1|0|0| +8969|M|U|Primary|4500|High Risk|1|0|0| +8970|F|U|Primary|4500|High Risk|1|0|0| +8971|M|M|Secondary|4500|High Risk|1|0|0| +8972|F|M|Secondary|4500|High Risk|1|0|0| +8973|M|S|Secondary|4500|High Risk|1|0|0| +8974|F|S|Secondary|4500|High Risk|1|0|0| +8975|M|D|Secondary|4500|High Risk|1|0|0| +8976|F|D|Secondary|4500|High Risk|1|0|0| +8977|M|W|Secondary|4500|High Risk|1|0|0| +8978|F|W|Secondary|4500|High Risk|1|0|0| +8979|M|U|Secondary|4500|High Risk|1|0|0| +8980|F|U|Secondary|4500|High Risk|1|0|0| +8981|M|M|College|4500|High Risk|1|0|0| +8982|F|M|College|4500|High Risk|1|0|0| +8983|M|S|College|4500|High Risk|1|0|0| +8984|F|S|College|4500|High Risk|1|0|0| +8985|M|D|College|4500|High Risk|1|0|0| +8986|F|D|College|4500|High Risk|1|0|0| +8987|M|W|College|4500|High Risk|1|0|0| +8988|F|W|College|4500|High Risk|1|0|0| +8989|M|U|College|4500|High Risk|1|0|0| +8990|F|U|College|4500|High Risk|1|0|0| +8991|M|M|2 yr Degree|4500|High Risk|1|0|0| +8992|F|M|2 yr Degree|4500|High Risk|1|0|0| +8993|M|S|2 yr Degree|4500|High Risk|1|0|0| +8994|F|S|2 yr Degree|4500|High Risk|1|0|0| +8995|M|D|2 yr Degree|4500|High Risk|1|0|0| +8996|F|D|2 yr Degree|4500|High Risk|1|0|0| +8997|M|W|2 yr Degree|4500|High Risk|1|0|0| +8998|F|W|2 yr Degree|4500|High Risk|1|0|0| +8999|M|U|2 yr Degree|4500|High Risk|1|0|0| +9000|F|U|2 yr Degree|4500|High Risk|1|0|0| +9001|M|M|4 yr Degree|4500|High Risk|1|0|0| +9002|F|M|4 yr Degree|4500|High Risk|1|0|0| +9003|M|S|4 yr Degree|4500|High Risk|1|0|0| +9004|F|S|4 yr Degree|4500|High Risk|1|0|0| +9005|M|D|4 yr Degree|4500|High Risk|1|0|0| +9006|F|D|4 yr Degree|4500|High Risk|1|0|0| +9007|M|W|4 yr Degree|4500|High Risk|1|0|0| +9008|F|W|4 yr Degree|4500|High Risk|1|0|0| +9009|M|U|4 yr Degree|4500|High Risk|1|0|0| +9010|F|U|4 yr Degree|4500|High Risk|1|0|0| +9011|M|M|Advanced Degree|4500|High Risk|1|0|0| +9012|F|M|Advanced Degree|4500|High Risk|1|0|0| +9013|M|S|Advanced Degree|4500|High Risk|1|0|0| +9014|F|S|Advanced Degree|4500|High Risk|1|0|0| +9015|M|D|Advanced Degree|4500|High Risk|1|0|0| +9016|F|D|Advanced Degree|4500|High Risk|1|0|0| +9017|M|W|Advanced Degree|4500|High Risk|1|0|0| +9018|F|W|Advanced Degree|4500|High Risk|1|0|0| +9019|M|U|Advanced Degree|4500|High Risk|1|0|0| +9020|F|U|Advanced Degree|4500|High Risk|1|0|0| +9021|M|M|Unknown|4500|High Risk|1|0|0| +9022|F|M|Unknown|4500|High Risk|1|0|0| +9023|M|S|Unknown|4500|High Risk|1|0|0| +9024|F|S|Unknown|4500|High Risk|1|0|0| +9025|M|D|Unknown|4500|High Risk|1|0|0| +9026|F|D|Unknown|4500|High Risk|1|0|0| +9027|M|W|Unknown|4500|High Risk|1|0|0| +9028|F|W|Unknown|4500|High Risk|1|0|0| +9029|M|U|Unknown|4500|High Risk|1|0|0| +9030|F|U|Unknown|4500|High Risk|1|0|0| +9031|M|M|Primary|5000|High Risk|1|0|0| +9032|F|M|Primary|5000|High Risk|1|0|0| +9033|M|S|Primary|5000|High Risk|1|0|0| +9034|F|S|Primary|5000|High Risk|1|0|0| +9035|M|D|Primary|5000|High Risk|1|0|0| +9036|F|D|Primary|5000|High Risk|1|0|0| +9037|M|W|Primary|5000|High Risk|1|0|0| +9038|F|W|Primary|5000|High Risk|1|0|0| +9039|M|U|Primary|5000|High Risk|1|0|0| +9040|F|U|Primary|5000|High Risk|1|0|0| +9041|M|M|Secondary|5000|High Risk|1|0|0| +9042|F|M|Secondary|5000|High Risk|1|0|0| +9043|M|S|Secondary|5000|High Risk|1|0|0| +9044|F|S|Secondary|5000|High Risk|1|0|0| +9045|M|D|Secondary|5000|High Risk|1|0|0| +9046|F|D|Secondary|5000|High Risk|1|0|0| +9047|M|W|Secondary|5000|High Risk|1|0|0| +9048|F|W|Secondary|5000|High Risk|1|0|0| +9049|M|U|Secondary|5000|High Risk|1|0|0| +9050|F|U|Secondary|5000|High Risk|1|0|0| +9051|M|M|College|5000|High Risk|1|0|0| +9052|F|M|College|5000|High Risk|1|0|0| +9053|M|S|College|5000|High Risk|1|0|0| +9054|F|S|College|5000|High Risk|1|0|0| +9055|M|D|College|5000|High Risk|1|0|0| +9056|F|D|College|5000|High Risk|1|0|0| +9057|M|W|College|5000|High Risk|1|0|0| +9058|F|W|College|5000|High Risk|1|0|0| +9059|M|U|College|5000|High Risk|1|0|0| +9060|F|U|College|5000|High Risk|1|0|0| +9061|M|M|2 yr Degree|5000|High Risk|1|0|0| +9062|F|M|2 yr Degree|5000|High Risk|1|0|0| +9063|M|S|2 yr Degree|5000|High Risk|1|0|0| +9064|F|S|2 yr Degree|5000|High Risk|1|0|0| +9065|M|D|2 yr Degree|5000|High Risk|1|0|0| +9066|F|D|2 yr Degree|5000|High Risk|1|0|0| +9067|M|W|2 yr Degree|5000|High Risk|1|0|0| +9068|F|W|2 yr Degree|5000|High Risk|1|0|0| +9069|M|U|2 yr Degree|5000|High Risk|1|0|0| +9070|F|U|2 yr Degree|5000|High Risk|1|0|0| +9071|M|M|4 yr Degree|5000|High Risk|1|0|0| +9072|F|M|4 yr Degree|5000|High Risk|1|0|0| +9073|M|S|4 yr Degree|5000|High Risk|1|0|0| +9074|F|S|4 yr Degree|5000|High Risk|1|0|0| +9075|M|D|4 yr Degree|5000|High Risk|1|0|0| +9076|F|D|4 yr Degree|5000|High Risk|1|0|0| +9077|M|W|4 yr Degree|5000|High Risk|1|0|0| +9078|F|W|4 yr Degree|5000|High Risk|1|0|0| +9079|M|U|4 yr Degree|5000|High Risk|1|0|0| +9080|F|U|4 yr Degree|5000|High Risk|1|0|0| +9081|M|M|Advanced Degree|5000|High Risk|1|0|0| +9082|F|M|Advanced Degree|5000|High Risk|1|0|0| +9083|M|S|Advanced Degree|5000|High Risk|1|0|0| +9084|F|S|Advanced Degree|5000|High Risk|1|0|0| +9085|M|D|Advanced Degree|5000|High Risk|1|0|0| +9086|F|D|Advanced Degree|5000|High Risk|1|0|0| +9087|M|W|Advanced Degree|5000|High Risk|1|0|0| +9088|F|W|Advanced Degree|5000|High Risk|1|0|0| +9089|M|U|Advanced Degree|5000|High Risk|1|0|0| +9090|F|U|Advanced Degree|5000|High Risk|1|0|0| +9091|M|M|Unknown|5000|High Risk|1|0|0| +9092|F|M|Unknown|5000|High Risk|1|0|0| +9093|M|S|Unknown|5000|High Risk|1|0|0| +9094|F|S|Unknown|5000|High Risk|1|0|0| +9095|M|D|Unknown|5000|High Risk|1|0|0| +9096|F|D|Unknown|5000|High Risk|1|0|0| +9097|M|W|Unknown|5000|High Risk|1|0|0| +9098|F|W|Unknown|5000|High Risk|1|0|0| +9099|M|U|Unknown|5000|High Risk|1|0|0| +9100|F|U|Unknown|5000|High Risk|1|0|0| +9101|M|M|Primary|5500|High Risk|1|0|0| +9102|F|M|Primary|5500|High Risk|1|0|0| +9103|M|S|Primary|5500|High Risk|1|0|0| +9104|F|S|Primary|5500|High Risk|1|0|0| +9105|M|D|Primary|5500|High Risk|1|0|0| +9106|F|D|Primary|5500|High Risk|1|0|0| +9107|M|W|Primary|5500|High Risk|1|0|0| +9108|F|W|Primary|5500|High Risk|1|0|0| +9109|M|U|Primary|5500|High Risk|1|0|0| +9110|F|U|Primary|5500|High Risk|1|0|0| +9111|M|M|Secondary|5500|High Risk|1|0|0| +9112|F|M|Secondary|5500|High Risk|1|0|0| +9113|M|S|Secondary|5500|High Risk|1|0|0| +9114|F|S|Secondary|5500|High Risk|1|0|0| +9115|M|D|Secondary|5500|High Risk|1|0|0| +9116|F|D|Secondary|5500|High Risk|1|0|0| +9117|M|W|Secondary|5500|High Risk|1|0|0| +9118|F|W|Secondary|5500|High Risk|1|0|0| +9119|M|U|Secondary|5500|High Risk|1|0|0| +9120|F|U|Secondary|5500|High Risk|1|0|0| +9121|M|M|College|5500|High Risk|1|0|0| +9122|F|M|College|5500|High Risk|1|0|0| +9123|M|S|College|5500|High Risk|1|0|0| +9124|F|S|College|5500|High Risk|1|0|0| +9125|M|D|College|5500|High Risk|1|0|0| +9126|F|D|College|5500|High Risk|1|0|0| +9127|M|W|College|5500|High Risk|1|0|0| +9128|F|W|College|5500|High Risk|1|0|0| +9129|M|U|College|5500|High Risk|1|0|0| +9130|F|U|College|5500|High Risk|1|0|0| +9131|M|M|2 yr Degree|5500|High Risk|1|0|0| +9132|F|M|2 yr Degree|5500|High Risk|1|0|0| +9133|M|S|2 yr Degree|5500|High Risk|1|0|0| +9134|F|S|2 yr Degree|5500|High Risk|1|0|0| +9135|M|D|2 yr Degree|5500|High Risk|1|0|0| +9136|F|D|2 yr Degree|5500|High Risk|1|0|0| +9137|M|W|2 yr Degree|5500|High Risk|1|0|0| +9138|F|W|2 yr Degree|5500|High Risk|1|0|0| +9139|M|U|2 yr Degree|5500|High Risk|1|0|0| +9140|F|U|2 yr Degree|5500|High Risk|1|0|0| +9141|M|M|4 yr Degree|5500|High Risk|1|0|0| +9142|F|M|4 yr Degree|5500|High Risk|1|0|0| +9143|M|S|4 yr Degree|5500|High Risk|1|0|0| +9144|F|S|4 yr Degree|5500|High Risk|1|0|0| +9145|M|D|4 yr Degree|5500|High Risk|1|0|0| +9146|F|D|4 yr Degree|5500|High Risk|1|0|0| +9147|M|W|4 yr Degree|5500|High Risk|1|0|0| +9148|F|W|4 yr Degree|5500|High Risk|1|0|0| +9149|M|U|4 yr Degree|5500|High Risk|1|0|0| +9150|F|U|4 yr Degree|5500|High Risk|1|0|0| +9151|M|M|Advanced Degree|5500|High Risk|1|0|0| +9152|F|M|Advanced Degree|5500|High Risk|1|0|0| +9153|M|S|Advanced Degree|5500|High Risk|1|0|0| +9154|F|S|Advanced Degree|5500|High Risk|1|0|0| +9155|M|D|Advanced Degree|5500|High Risk|1|0|0| +9156|F|D|Advanced Degree|5500|High Risk|1|0|0| +9157|M|W|Advanced Degree|5500|High Risk|1|0|0| +9158|F|W|Advanced Degree|5500|High Risk|1|0|0| +9159|M|U|Advanced Degree|5500|High Risk|1|0|0| +9160|F|U|Advanced Degree|5500|High Risk|1|0|0| +9161|M|M|Unknown|5500|High Risk|1|0|0| +9162|F|M|Unknown|5500|High Risk|1|0|0| +9163|M|S|Unknown|5500|High Risk|1|0|0| +9164|F|S|Unknown|5500|High Risk|1|0|0| +9165|M|D|Unknown|5500|High Risk|1|0|0| +9166|F|D|Unknown|5500|High Risk|1|0|0| +9167|M|W|Unknown|5500|High Risk|1|0|0| +9168|F|W|Unknown|5500|High Risk|1|0|0| +9169|M|U|Unknown|5500|High Risk|1|0|0| +9170|F|U|Unknown|5500|High Risk|1|0|0| +9171|M|M|Primary|6000|High Risk|1|0|0| +9172|F|M|Primary|6000|High Risk|1|0|0| +9173|M|S|Primary|6000|High Risk|1|0|0| +9174|F|S|Primary|6000|High Risk|1|0|0| +9175|M|D|Primary|6000|High Risk|1|0|0| +9176|F|D|Primary|6000|High Risk|1|0|0| +9177|M|W|Primary|6000|High Risk|1|0|0| +9178|F|W|Primary|6000|High Risk|1|0|0| +9179|M|U|Primary|6000|High Risk|1|0|0| +9180|F|U|Primary|6000|High Risk|1|0|0| +9181|M|M|Secondary|6000|High Risk|1|0|0| +9182|F|M|Secondary|6000|High Risk|1|0|0| +9183|M|S|Secondary|6000|High Risk|1|0|0| +9184|F|S|Secondary|6000|High Risk|1|0|0| +9185|M|D|Secondary|6000|High Risk|1|0|0| +9186|F|D|Secondary|6000|High Risk|1|0|0| +9187|M|W|Secondary|6000|High Risk|1|0|0| +9188|F|W|Secondary|6000|High Risk|1|0|0| +9189|M|U|Secondary|6000|High Risk|1|0|0| +9190|F|U|Secondary|6000|High Risk|1|0|0| +9191|M|M|College|6000|High Risk|1|0|0| +9192|F|M|College|6000|High Risk|1|0|0| +9193|M|S|College|6000|High Risk|1|0|0| +9194|F|S|College|6000|High Risk|1|0|0| +9195|M|D|College|6000|High Risk|1|0|0| +9196|F|D|College|6000|High Risk|1|0|0| +9197|M|W|College|6000|High Risk|1|0|0| +9198|F|W|College|6000|High Risk|1|0|0| +9199|M|U|College|6000|High Risk|1|0|0| +9200|F|U|College|6000|High Risk|1|0|0| +9201|M|M|2 yr Degree|6000|High Risk|1|0|0| +9202|F|M|2 yr Degree|6000|High Risk|1|0|0| +9203|M|S|2 yr Degree|6000|High Risk|1|0|0| +9204|F|S|2 yr Degree|6000|High Risk|1|0|0| +9205|M|D|2 yr Degree|6000|High Risk|1|0|0| +9206|F|D|2 yr Degree|6000|High Risk|1|0|0| +9207|M|W|2 yr Degree|6000|High Risk|1|0|0| +9208|F|W|2 yr Degree|6000|High Risk|1|0|0| +9209|M|U|2 yr Degree|6000|High Risk|1|0|0| +9210|F|U|2 yr Degree|6000|High Risk|1|0|0| +9211|M|M|4 yr Degree|6000|High Risk|1|0|0| +9212|F|M|4 yr Degree|6000|High Risk|1|0|0| +9213|M|S|4 yr Degree|6000|High Risk|1|0|0| +9214|F|S|4 yr Degree|6000|High Risk|1|0|0| +9215|M|D|4 yr Degree|6000|High Risk|1|0|0| +9216|F|D|4 yr Degree|6000|High Risk|1|0|0| +9217|M|W|4 yr Degree|6000|High Risk|1|0|0| +9218|F|W|4 yr Degree|6000|High Risk|1|0|0| +9219|M|U|4 yr Degree|6000|High Risk|1|0|0| +9220|F|U|4 yr Degree|6000|High Risk|1|0|0| +9221|M|M|Advanced Degree|6000|High Risk|1|0|0| +9222|F|M|Advanced Degree|6000|High Risk|1|0|0| +9223|M|S|Advanced Degree|6000|High Risk|1|0|0| +9224|F|S|Advanced Degree|6000|High Risk|1|0|0| +9225|M|D|Advanced Degree|6000|High Risk|1|0|0| +9226|F|D|Advanced Degree|6000|High Risk|1|0|0| +9227|M|W|Advanced Degree|6000|High Risk|1|0|0| +9228|F|W|Advanced Degree|6000|High Risk|1|0|0| +9229|M|U|Advanced Degree|6000|High Risk|1|0|0| +9230|F|U|Advanced Degree|6000|High Risk|1|0|0| +9231|M|M|Unknown|6000|High Risk|1|0|0| +9232|F|M|Unknown|6000|High Risk|1|0|0| +9233|M|S|Unknown|6000|High Risk|1|0|0| +9234|F|S|Unknown|6000|High Risk|1|0|0| +9235|M|D|Unknown|6000|High Risk|1|0|0| +9236|F|D|Unknown|6000|High Risk|1|0|0| +9237|M|W|Unknown|6000|High Risk|1|0|0| +9238|F|W|Unknown|6000|High Risk|1|0|0| +9239|M|U|Unknown|6000|High Risk|1|0|0| +9240|F|U|Unknown|6000|High Risk|1|0|0| +9241|M|M|Primary|6500|High Risk|1|0|0| +9242|F|M|Primary|6500|High Risk|1|0|0| +9243|M|S|Primary|6500|High Risk|1|0|0| +9244|F|S|Primary|6500|High Risk|1|0|0| +9245|M|D|Primary|6500|High Risk|1|0|0| +9246|F|D|Primary|6500|High Risk|1|0|0| +9247|M|W|Primary|6500|High Risk|1|0|0| +9248|F|W|Primary|6500|High Risk|1|0|0| +9249|M|U|Primary|6500|High Risk|1|0|0| +9250|F|U|Primary|6500|High Risk|1|0|0| +9251|M|M|Secondary|6500|High Risk|1|0|0| +9252|F|M|Secondary|6500|High Risk|1|0|0| +9253|M|S|Secondary|6500|High Risk|1|0|0| +9254|F|S|Secondary|6500|High Risk|1|0|0| +9255|M|D|Secondary|6500|High Risk|1|0|0| +9256|F|D|Secondary|6500|High Risk|1|0|0| +9257|M|W|Secondary|6500|High Risk|1|0|0| +9258|F|W|Secondary|6500|High Risk|1|0|0| +9259|M|U|Secondary|6500|High Risk|1|0|0| +9260|F|U|Secondary|6500|High Risk|1|0|0| +9261|M|M|College|6500|High Risk|1|0|0| +9262|F|M|College|6500|High Risk|1|0|0| +9263|M|S|College|6500|High Risk|1|0|0| +9264|F|S|College|6500|High Risk|1|0|0| +9265|M|D|College|6500|High Risk|1|0|0| +9266|F|D|College|6500|High Risk|1|0|0| +9267|M|W|College|6500|High Risk|1|0|0| +9268|F|W|College|6500|High Risk|1|0|0| +9269|M|U|College|6500|High Risk|1|0|0| +9270|F|U|College|6500|High Risk|1|0|0| +9271|M|M|2 yr Degree|6500|High Risk|1|0|0| +9272|F|M|2 yr Degree|6500|High Risk|1|0|0| +9273|M|S|2 yr Degree|6500|High Risk|1|0|0| +9274|F|S|2 yr Degree|6500|High Risk|1|0|0| +9275|M|D|2 yr Degree|6500|High Risk|1|0|0| +9276|F|D|2 yr Degree|6500|High Risk|1|0|0| +9277|M|W|2 yr Degree|6500|High Risk|1|0|0| +9278|F|W|2 yr Degree|6500|High Risk|1|0|0| +9279|M|U|2 yr Degree|6500|High Risk|1|0|0| +9280|F|U|2 yr Degree|6500|High Risk|1|0|0| +9281|M|M|4 yr Degree|6500|High Risk|1|0|0| +9282|F|M|4 yr Degree|6500|High Risk|1|0|0| +9283|M|S|4 yr Degree|6500|High Risk|1|0|0| +9284|F|S|4 yr Degree|6500|High Risk|1|0|0| +9285|M|D|4 yr Degree|6500|High Risk|1|0|0| +9286|F|D|4 yr Degree|6500|High Risk|1|0|0| +9287|M|W|4 yr Degree|6500|High Risk|1|0|0| +9288|F|W|4 yr Degree|6500|High Risk|1|0|0| +9289|M|U|4 yr Degree|6500|High Risk|1|0|0| +9290|F|U|4 yr Degree|6500|High Risk|1|0|0| +9291|M|M|Advanced Degree|6500|High Risk|1|0|0| +9292|F|M|Advanced Degree|6500|High Risk|1|0|0| +9293|M|S|Advanced Degree|6500|High Risk|1|0|0| +9294|F|S|Advanced Degree|6500|High Risk|1|0|0| +9295|M|D|Advanced Degree|6500|High Risk|1|0|0| +9296|F|D|Advanced Degree|6500|High Risk|1|0|0| +9297|M|W|Advanced Degree|6500|High Risk|1|0|0| +9298|F|W|Advanced Degree|6500|High Risk|1|0|0| +9299|M|U|Advanced Degree|6500|High Risk|1|0|0| +9300|F|U|Advanced Degree|6500|High Risk|1|0|0| +9301|M|M|Unknown|6500|High Risk|1|0|0| +9302|F|M|Unknown|6500|High Risk|1|0|0| +9303|M|S|Unknown|6500|High Risk|1|0|0| +9304|F|S|Unknown|6500|High Risk|1|0|0| +9305|M|D|Unknown|6500|High Risk|1|0|0| +9306|F|D|Unknown|6500|High Risk|1|0|0| +9307|M|W|Unknown|6500|High Risk|1|0|0| +9308|F|W|Unknown|6500|High Risk|1|0|0| +9309|M|U|Unknown|6500|High Risk|1|0|0| +9310|F|U|Unknown|6500|High Risk|1|0|0| +9311|M|M|Primary|7000|High Risk|1|0|0| +9312|F|M|Primary|7000|High Risk|1|0|0| +9313|M|S|Primary|7000|High Risk|1|0|0| +9314|F|S|Primary|7000|High Risk|1|0|0| +9315|M|D|Primary|7000|High Risk|1|0|0| +9316|F|D|Primary|7000|High Risk|1|0|0| +9317|M|W|Primary|7000|High Risk|1|0|0| +9318|F|W|Primary|7000|High Risk|1|0|0| +9319|M|U|Primary|7000|High Risk|1|0|0| +9320|F|U|Primary|7000|High Risk|1|0|0| +9321|M|M|Secondary|7000|High Risk|1|0|0| +9322|F|M|Secondary|7000|High Risk|1|0|0| +9323|M|S|Secondary|7000|High Risk|1|0|0| +9324|F|S|Secondary|7000|High Risk|1|0|0| +9325|M|D|Secondary|7000|High Risk|1|0|0| +9326|F|D|Secondary|7000|High Risk|1|0|0| +9327|M|W|Secondary|7000|High Risk|1|0|0| +9328|F|W|Secondary|7000|High Risk|1|0|0| +9329|M|U|Secondary|7000|High Risk|1|0|0| +9330|F|U|Secondary|7000|High Risk|1|0|0| +9331|M|M|College|7000|High Risk|1|0|0| +9332|F|M|College|7000|High Risk|1|0|0| +9333|M|S|College|7000|High Risk|1|0|0| +9334|F|S|College|7000|High Risk|1|0|0| +9335|M|D|College|7000|High Risk|1|0|0| +9336|F|D|College|7000|High Risk|1|0|0| +9337|M|W|College|7000|High Risk|1|0|0| +9338|F|W|College|7000|High Risk|1|0|0| +9339|M|U|College|7000|High Risk|1|0|0| +9340|F|U|College|7000|High Risk|1|0|0| +9341|M|M|2 yr Degree|7000|High Risk|1|0|0| +9342|F|M|2 yr Degree|7000|High Risk|1|0|0| +9343|M|S|2 yr Degree|7000|High Risk|1|0|0| +9344|F|S|2 yr Degree|7000|High Risk|1|0|0| +9345|M|D|2 yr Degree|7000|High Risk|1|0|0| +9346|F|D|2 yr Degree|7000|High Risk|1|0|0| +9347|M|W|2 yr Degree|7000|High Risk|1|0|0| +9348|F|W|2 yr Degree|7000|High Risk|1|0|0| +9349|M|U|2 yr Degree|7000|High Risk|1|0|0| +9350|F|U|2 yr Degree|7000|High Risk|1|0|0| +9351|M|M|4 yr Degree|7000|High Risk|1|0|0| +9352|F|M|4 yr Degree|7000|High Risk|1|0|0| +9353|M|S|4 yr Degree|7000|High Risk|1|0|0| +9354|F|S|4 yr Degree|7000|High Risk|1|0|0| +9355|M|D|4 yr Degree|7000|High Risk|1|0|0| +9356|F|D|4 yr Degree|7000|High Risk|1|0|0| +9357|M|W|4 yr Degree|7000|High Risk|1|0|0| +9358|F|W|4 yr Degree|7000|High Risk|1|0|0| +9359|M|U|4 yr Degree|7000|High Risk|1|0|0| +9360|F|U|4 yr Degree|7000|High Risk|1|0|0| +9361|M|M|Advanced Degree|7000|High Risk|1|0|0| +9362|F|M|Advanced Degree|7000|High Risk|1|0|0| +9363|M|S|Advanced Degree|7000|High Risk|1|0|0| +9364|F|S|Advanced Degree|7000|High Risk|1|0|0| +9365|M|D|Advanced Degree|7000|High Risk|1|0|0| +9366|F|D|Advanced Degree|7000|High Risk|1|0|0| +9367|M|W|Advanced Degree|7000|High Risk|1|0|0| +9368|F|W|Advanced Degree|7000|High Risk|1|0|0| +9369|M|U|Advanced Degree|7000|High Risk|1|0|0| +9370|F|U|Advanced Degree|7000|High Risk|1|0|0| +9371|M|M|Unknown|7000|High Risk|1|0|0| +9372|F|M|Unknown|7000|High Risk|1|0|0| +9373|M|S|Unknown|7000|High Risk|1|0|0| +9374|F|S|Unknown|7000|High Risk|1|0|0| +9375|M|D|Unknown|7000|High Risk|1|0|0| +9376|F|D|Unknown|7000|High Risk|1|0|0| +9377|M|W|Unknown|7000|High Risk|1|0|0| +9378|F|W|Unknown|7000|High Risk|1|0|0| +9379|M|U|Unknown|7000|High Risk|1|0|0| +9380|F|U|Unknown|7000|High Risk|1|0|0| +9381|M|M|Primary|7500|High Risk|1|0|0| +9382|F|M|Primary|7500|High Risk|1|0|0| +9383|M|S|Primary|7500|High Risk|1|0|0| +9384|F|S|Primary|7500|High Risk|1|0|0| +9385|M|D|Primary|7500|High Risk|1|0|0| +9386|F|D|Primary|7500|High Risk|1|0|0| +9387|M|W|Primary|7500|High Risk|1|0|0| +9388|F|W|Primary|7500|High Risk|1|0|0| +9389|M|U|Primary|7500|High Risk|1|0|0| +9390|F|U|Primary|7500|High Risk|1|0|0| +9391|M|M|Secondary|7500|High Risk|1|0|0| +9392|F|M|Secondary|7500|High Risk|1|0|0| +9393|M|S|Secondary|7500|High Risk|1|0|0| +9394|F|S|Secondary|7500|High Risk|1|0|0| +9395|M|D|Secondary|7500|High Risk|1|0|0| +9396|F|D|Secondary|7500|High Risk|1|0|0| +9397|M|W|Secondary|7500|High Risk|1|0|0| +9398|F|W|Secondary|7500|High Risk|1|0|0| +9399|M|U|Secondary|7500|High Risk|1|0|0| +9400|F|U|Secondary|7500|High Risk|1|0|0| +9401|M|M|College|7500|High Risk|1|0|0| +9402|F|M|College|7500|High Risk|1|0|0| +9403|M|S|College|7500|High Risk|1|0|0| +9404|F|S|College|7500|High Risk|1|0|0| +9405|M|D|College|7500|High Risk|1|0|0| +9406|F|D|College|7500|High Risk|1|0|0| +9407|M|W|College|7500|High Risk|1|0|0| +9408|F|W|College|7500|High Risk|1|0|0| +9409|M|U|College|7500|High Risk|1|0|0| +9410|F|U|College|7500|High Risk|1|0|0| +9411|M|M|2 yr Degree|7500|High Risk|1|0|0| +9412|F|M|2 yr Degree|7500|High Risk|1|0|0| +9413|M|S|2 yr Degree|7500|High Risk|1|0|0| +9414|F|S|2 yr Degree|7500|High Risk|1|0|0| +9415|M|D|2 yr Degree|7500|High Risk|1|0|0| +9416|F|D|2 yr Degree|7500|High Risk|1|0|0| +9417|M|W|2 yr Degree|7500|High Risk|1|0|0| +9418|F|W|2 yr Degree|7500|High Risk|1|0|0| +9419|M|U|2 yr Degree|7500|High Risk|1|0|0| +9420|F|U|2 yr Degree|7500|High Risk|1|0|0| +9421|M|M|4 yr Degree|7500|High Risk|1|0|0| +9422|F|M|4 yr Degree|7500|High Risk|1|0|0| +9423|M|S|4 yr Degree|7500|High Risk|1|0|0| +9424|F|S|4 yr Degree|7500|High Risk|1|0|0| +9425|M|D|4 yr Degree|7500|High Risk|1|0|0| +9426|F|D|4 yr Degree|7500|High Risk|1|0|0| +9427|M|W|4 yr Degree|7500|High Risk|1|0|0| +9428|F|W|4 yr Degree|7500|High Risk|1|0|0| +9429|M|U|4 yr Degree|7500|High Risk|1|0|0| +9430|F|U|4 yr Degree|7500|High Risk|1|0|0| +9431|M|M|Advanced Degree|7500|High Risk|1|0|0| +9432|F|M|Advanced Degree|7500|High Risk|1|0|0| +9433|M|S|Advanced Degree|7500|High Risk|1|0|0| +9434|F|S|Advanced Degree|7500|High Risk|1|0|0| +9435|M|D|Advanced Degree|7500|High Risk|1|0|0| +9436|F|D|Advanced Degree|7500|High Risk|1|0|0| +9437|M|W|Advanced Degree|7500|High Risk|1|0|0| +9438|F|W|Advanced Degree|7500|High Risk|1|0|0| +9439|M|U|Advanced Degree|7500|High Risk|1|0|0| +9440|F|U|Advanced Degree|7500|High Risk|1|0|0| +9441|M|M|Unknown|7500|High Risk|1|0|0| +9442|F|M|Unknown|7500|High Risk|1|0|0| +9443|M|S|Unknown|7500|High Risk|1|0|0| +9444|F|S|Unknown|7500|High Risk|1|0|0| +9445|M|D|Unknown|7500|High Risk|1|0|0| +9446|F|D|Unknown|7500|High Risk|1|0|0| +9447|M|W|Unknown|7500|High Risk|1|0|0| +9448|F|W|Unknown|7500|High Risk|1|0|0| +9449|M|U|Unknown|7500|High Risk|1|0|0| +9450|F|U|Unknown|7500|High Risk|1|0|0| +9451|M|M|Primary|8000|High Risk|1|0|0| +9452|F|M|Primary|8000|High Risk|1|0|0| +9453|M|S|Primary|8000|High Risk|1|0|0| +9454|F|S|Primary|8000|High Risk|1|0|0| +9455|M|D|Primary|8000|High Risk|1|0|0| +9456|F|D|Primary|8000|High Risk|1|0|0| +9457|M|W|Primary|8000|High Risk|1|0|0| +9458|F|W|Primary|8000|High Risk|1|0|0| +9459|M|U|Primary|8000|High Risk|1|0|0| +9460|F|U|Primary|8000|High Risk|1|0|0| +9461|M|M|Secondary|8000|High Risk|1|0|0| +9462|F|M|Secondary|8000|High Risk|1|0|0| +9463|M|S|Secondary|8000|High Risk|1|0|0| +9464|F|S|Secondary|8000|High Risk|1|0|0| +9465|M|D|Secondary|8000|High Risk|1|0|0| +9466|F|D|Secondary|8000|High Risk|1|0|0| +9467|M|W|Secondary|8000|High Risk|1|0|0| +9468|F|W|Secondary|8000|High Risk|1|0|0| +9469|M|U|Secondary|8000|High Risk|1|0|0| +9470|F|U|Secondary|8000|High Risk|1|0|0| +9471|M|M|College|8000|High Risk|1|0|0| +9472|F|M|College|8000|High Risk|1|0|0| +9473|M|S|College|8000|High Risk|1|0|0| +9474|F|S|College|8000|High Risk|1|0|0| +9475|M|D|College|8000|High Risk|1|0|0| +9476|F|D|College|8000|High Risk|1|0|0| +9477|M|W|College|8000|High Risk|1|0|0| +9478|F|W|College|8000|High Risk|1|0|0| +9479|M|U|College|8000|High Risk|1|0|0| +9480|F|U|College|8000|High Risk|1|0|0| +9481|M|M|2 yr Degree|8000|High Risk|1|0|0| +9482|F|M|2 yr Degree|8000|High Risk|1|0|0| +9483|M|S|2 yr Degree|8000|High Risk|1|0|0| +9484|F|S|2 yr Degree|8000|High Risk|1|0|0| +9485|M|D|2 yr Degree|8000|High Risk|1|0|0| +9486|F|D|2 yr Degree|8000|High Risk|1|0|0| +9487|M|W|2 yr Degree|8000|High Risk|1|0|0| +9488|F|W|2 yr Degree|8000|High Risk|1|0|0| +9489|M|U|2 yr Degree|8000|High Risk|1|0|0| +9490|F|U|2 yr Degree|8000|High Risk|1|0|0| +9491|M|M|4 yr Degree|8000|High Risk|1|0|0| +9492|F|M|4 yr Degree|8000|High Risk|1|0|0| +9493|M|S|4 yr Degree|8000|High Risk|1|0|0| +9494|F|S|4 yr Degree|8000|High Risk|1|0|0| +9495|M|D|4 yr Degree|8000|High Risk|1|0|0| +9496|F|D|4 yr Degree|8000|High Risk|1|0|0| +9497|M|W|4 yr Degree|8000|High Risk|1|0|0| +9498|F|W|4 yr Degree|8000|High Risk|1|0|0| +9499|M|U|4 yr Degree|8000|High Risk|1|0|0| +9500|F|U|4 yr Degree|8000|High Risk|1|0|0| +9501|M|M|Advanced Degree|8000|High Risk|1|0|0| +9502|F|M|Advanced Degree|8000|High Risk|1|0|0| +9503|M|S|Advanced Degree|8000|High Risk|1|0|0| +9504|F|S|Advanced Degree|8000|High Risk|1|0|0| +9505|M|D|Advanced Degree|8000|High Risk|1|0|0| +9506|F|D|Advanced Degree|8000|High Risk|1|0|0| +9507|M|W|Advanced Degree|8000|High Risk|1|0|0| +9508|F|W|Advanced Degree|8000|High Risk|1|0|0| +9509|M|U|Advanced Degree|8000|High Risk|1|0|0| +9510|F|U|Advanced Degree|8000|High Risk|1|0|0| +9511|M|M|Unknown|8000|High Risk|1|0|0| +9512|F|M|Unknown|8000|High Risk|1|0|0| +9513|M|S|Unknown|8000|High Risk|1|0|0| +9514|F|S|Unknown|8000|High Risk|1|0|0| +9515|M|D|Unknown|8000|High Risk|1|0|0| +9516|F|D|Unknown|8000|High Risk|1|0|0| +9517|M|W|Unknown|8000|High Risk|1|0|0| +9518|F|W|Unknown|8000|High Risk|1|0|0| +9519|M|U|Unknown|8000|High Risk|1|0|0| +9520|F|U|Unknown|8000|High Risk|1|0|0| +9521|M|M|Primary|8500|High Risk|1|0|0| +9522|F|M|Primary|8500|High Risk|1|0|0| +9523|M|S|Primary|8500|High Risk|1|0|0| +9524|F|S|Primary|8500|High Risk|1|0|0| +9525|M|D|Primary|8500|High Risk|1|0|0| +9526|F|D|Primary|8500|High Risk|1|0|0| +9527|M|W|Primary|8500|High Risk|1|0|0| +9528|F|W|Primary|8500|High Risk|1|0|0| +9529|M|U|Primary|8500|High Risk|1|0|0| +9530|F|U|Primary|8500|High Risk|1|0|0| +9531|M|M|Secondary|8500|High Risk|1|0|0| +9532|F|M|Secondary|8500|High Risk|1|0|0| +9533|M|S|Secondary|8500|High Risk|1|0|0| +9534|F|S|Secondary|8500|High Risk|1|0|0| +9535|M|D|Secondary|8500|High Risk|1|0|0| +9536|F|D|Secondary|8500|High Risk|1|0|0| +9537|M|W|Secondary|8500|High Risk|1|0|0| +9538|F|W|Secondary|8500|High Risk|1|0|0| +9539|M|U|Secondary|8500|High Risk|1|0|0| +9540|F|U|Secondary|8500|High Risk|1|0|0| +9541|M|M|College|8500|High Risk|1|0|0| +9542|F|M|College|8500|High Risk|1|0|0| +9543|M|S|College|8500|High Risk|1|0|0| +9544|F|S|College|8500|High Risk|1|0|0| +9545|M|D|College|8500|High Risk|1|0|0| +9546|F|D|College|8500|High Risk|1|0|0| +9547|M|W|College|8500|High Risk|1|0|0| +9548|F|W|College|8500|High Risk|1|0|0| +9549|M|U|College|8500|High Risk|1|0|0| +9550|F|U|College|8500|High Risk|1|0|0| +9551|M|M|2 yr Degree|8500|High Risk|1|0|0| +9552|F|M|2 yr Degree|8500|High Risk|1|0|0| +9553|M|S|2 yr Degree|8500|High Risk|1|0|0| +9554|F|S|2 yr Degree|8500|High Risk|1|0|0| +9555|M|D|2 yr Degree|8500|High Risk|1|0|0| +9556|F|D|2 yr Degree|8500|High Risk|1|0|0| +9557|M|W|2 yr Degree|8500|High Risk|1|0|0| +9558|F|W|2 yr Degree|8500|High Risk|1|0|0| +9559|M|U|2 yr Degree|8500|High Risk|1|0|0| +9560|F|U|2 yr Degree|8500|High Risk|1|0|0| +9561|M|M|4 yr Degree|8500|High Risk|1|0|0| +9562|F|M|4 yr Degree|8500|High Risk|1|0|0| +9563|M|S|4 yr Degree|8500|High Risk|1|0|0| +9564|F|S|4 yr Degree|8500|High Risk|1|0|0| +9565|M|D|4 yr Degree|8500|High Risk|1|0|0| +9566|F|D|4 yr Degree|8500|High Risk|1|0|0| +9567|M|W|4 yr Degree|8500|High Risk|1|0|0| +9568|F|W|4 yr Degree|8500|High Risk|1|0|0| +9569|M|U|4 yr Degree|8500|High Risk|1|0|0| +9570|F|U|4 yr Degree|8500|High Risk|1|0|0| +9571|M|M|Advanced Degree|8500|High Risk|1|0|0| +9572|F|M|Advanced Degree|8500|High Risk|1|0|0| +9573|M|S|Advanced Degree|8500|High Risk|1|0|0| +9574|F|S|Advanced Degree|8500|High Risk|1|0|0| +9575|M|D|Advanced Degree|8500|High Risk|1|0|0| +9576|F|D|Advanced Degree|8500|High Risk|1|0|0| +9577|M|W|Advanced Degree|8500|High Risk|1|0|0| +9578|F|W|Advanced Degree|8500|High Risk|1|0|0| +9579|M|U|Advanced Degree|8500|High Risk|1|0|0| +9580|F|U|Advanced Degree|8500|High Risk|1|0|0| +9581|M|M|Unknown|8500|High Risk|1|0|0| +9582|F|M|Unknown|8500|High Risk|1|0|0| +9583|M|S|Unknown|8500|High Risk|1|0|0| +9584|F|S|Unknown|8500|High Risk|1|0|0| +9585|M|D|Unknown|8500|High Risk|1|0|0| +9586|F|D|Unknown|8500|High Risk|1|0|0| +9587|M|W|Unknown|8500|High Risk|1|0|0| +9588|F|W|Unknown|8500|High Risk|1|0|0| +9589|M|U|Unknown|8500|High Risk|1|0|0| +9590|F|U|Unknown|8500|High Risk|1|0|0| +9591|M|M|Primary|9000|High Risk|1|0|0| +9592|F|M|Primary|9000|High Risk|1|0|0| +9593|M|S|Primary|9000|High Risk|1|0|0| +9594|F|S|Primary|9000|High Risk|1|0|0| +9595|M|D|Primary|9000|High Risk|1|0|0| +9596|F|D|Primary|9000|High Risk|1|0|0| +9597|M|W|Primary|9000|High Risk|1|0|0| +9598|F|W|Primary|9000|High Risk|1|0|0| +9599|M|U|Primary|9000|High Risk|1|0|0| +9600|F|U|Primary|9000|High Risk|1|0|0| +9601|M|M|Secondary|9000|High Risk|1|0|0| +9602|F|M|Secondary|9000|High Risk|1|0|0| +9603|M|S|Secondary|9000|High Risk|1|0|0| +9604|F|S|Secondary|9000|High Risk|1|0|0| +9605|M|D|Secondary|9000|High Risk|1|0|0| +9606|F|D|Secondary|9000|High Risk|1|0|0| +9607|M|W|Secondary|9000|High Risk|1|0|0| +9608|F|W|Secondary|9000|High Risk|1|0|0| +9609|M|U|Secondary|9000|High Risk|1|0|0| +9610|F|U|Secondary|9000|High Risk|1|0|0| +9611|M|M|College|9000|High Risk|1|0|0| +9612|F|M|College|9000|High Risk|1|0|0| +9613|M|S|College|9000|High Risk|1|0|0| +9614|F|S|College|9000|High Risk|1|0|0| +9615|M|D|College|9000|High Risk|1|0|0| +9616|F|D|College|9000|High Risk|1|0|0| +9617|M|W|College|9000|High Risk|1|0|0| +9618|F|W|College|9000|High Risk|1|0|0| +9619|M|U|College|9000|High Risk|1|0|0| +9620|F|U|College|9000|High Risk|1|0|0| +9621|M|M|2 yr Degree|9000|High Risk|1|0|0| +9622|F|M|2 yr Degree|9000|High Risk|1|0|0| +9623|M|S|2 yr Degree|9000|High Risk|1|0|0| +9624|F|S|2 yr Degree|9000|High Risk|1|0|0| +9625|M|D|2 yr Degree|9000|High Risk|1|0|0| +9626|F|D|2 yr Degree|9000|High Risk|1|0|0| +9627|M|W|2 yr Degree|9000|High Risk|1|0|0| +9628|F|W|2 yr Degree|9000|High Risk|1|0|0| +9629|M|U|2 yr Degree|9000|High Risk|1|0|0| +9630|F|U|2 yr Degree|9000|High Risk|1|0|0| +9631|M|M|4 yr Degree|9000|High Risk|1|0|0| +9632|F|M|4 yr Degree|9000|High Risk|1|0|0| +9633|M|S|4 yr Degree|9000|High Risk|1|0|0| +9634|F|S|4 yr Degree|9000|High Risk|1|0|0| +9635|M|D|4 yr Degree|9000|High Risk|1|0|0| +9636|F|D|4 yr Degree|9000|High Risk|1|0|0| +9637|M|W|4 yr Degree|9000|High Risk|1|0|0| +9638|F|W|4 yr Degree|9000|High Risk|1|0|0| +9639|M|U|4 yr Degree|9000|High Risk|1|0|0| +9640|F|U|4 yr Degree|9000|High Risk|1|0|0| +9641|M|M|Advanced Degree|9000|High Risk|1|0|0| +9642|F|M|Advanced Degree|9000|High Risk|1|0|0| +9643|M|S|Advanced Degree|9000|High Risk|1|0|0| +9644|F|S|Advanced Degree|9000|High Risk|1|0|0| +9645|M|D|Advanced Degree|9000|High Risk|1|0|0| +9646|F|D|Advanced Degree|9000|High Risk|1|0|0| +9647|M|W|Advanced Degree|9000|High Risk|1|0|0| +9648|F|W|Advanced Degree|9000|High Risk|1|0|0| +9649|M|U|Advanced Degree|9000|High Risk|1|0|0| +9650|F|U|Advanced Degree|9000|High Risk|1|0|0| +9651|M|M|Unknown|9000|High Risk|1|0|0| +9652|F|M|Unknown|9000|High Risk|1|0|0| +9653|M|S|Unknown|9000|High Risk|1|0|0| +9654|F|S|Unknown|9000|High Risk|1|0|0| +9655|M|D|Unknown|9000|High Risk|1|0|0| +9656|F|D|Unknown|9000|High Risk|1|0|0| +9657|M|W|Unknown|9000|High Risk|1|0|0| +9658|F|W|Unknown|9000|High Risk|1|0|0| +9659|M|U|Unknown|9000|High Risk|1|0|0| +9660|F|U|Unknown|9000|High Risk|1|0|0| +9661|M|M|Primary|9500|High Risk|1|0|0| +9662|F|M|Primary|9500|High Risk|1|0|0| +9663|M|S|Primary|9500|High Risk|1|0|0| +9664|F|S|Primary|9500|High Risk|1|0|0| +9665|M|D|Primary|9500|High Risk|1|0|0| +9666|F|D|Primary|9500|High Risk|1|0|0| +9667|M|W|Primary|9500|High Risk|1|0|0| +9668|F|W|Primary|9500|High Risk|1|0|0| +9669|M|U|Primary|9500|High Risk|1|0|0| +9670|F|U|Primary|9500|High Risk|1|0|0| +9671|M|M|Secondary|9500|High Risk|1|0|0| +9672|F|M|Secondary|9500|High Risk|1|0|0| +9673|M|S|Secondary|9500|High Risk|1|0|0| +9674|F|S|Secondary|9500|High Risk|1|0|0| +9675|M|D|Secondary|9500|High Risk|1|0|0| +9676|F|D|Secondary|9500|High Risk|1|0|0| +9677|M|W|Secondary|9500|High Risk|1|0|0| +9678|F|W|Secondary|9500|High Risk|1|0|0| +9679|M|U|Secondary|9500|High Risk|1|0|0| +9680|F|U|Secondary|9500|High Risk|1|0|0| +9681|M|M|College|9500|High Risk|1|0|0| +9682|F|M|College|9500|High Risk|1|0|0| +9683|M|S|College|9500|High Risk|1|0|0| +9684|F|S|College|9500|High Risk|1|0|0| +9685|M|D|College|9500|High Risk|1|0|0| +9686|F|D|College|9500|High Risk|1|0|0| +9687|M|W|College|9500|High Risk|1|0|0| +9688|F|W|College|9500|High Risk|1|0|0| +9689|M|U|College|9500|High Risk|1|0|0| +9690|F|U|College|9500|High Risk|1|0|0| +9691|M|M|2 yr Degree|9500|High Risk|1|0|0| +9692|F|M|2 yr Degree|9500|High Risk|1|0|0| +9693|M|S|2 yr Degree|9500|High Risk|1|0|0| +9694|F|S|2 yr Degree|9500|High Risk|1|0|0| +9695|M|D|2 yr Degree|9500|High Risk|1|0|0| +9696|F|D|2 yr Degree|9500|High Risk|1|0|0| +9697|M|W|2 yr Degree|9500|High Risk|1|0|0| +9698|F|W|2 yr Degree|9500|High Risk|1|0|0| +9699|M|U|2 yr Degree|9500|High Risk|1|0|0| +9700|F|U|2 yr Degree|9500|High Risk|1|0|0| +9701|M|M|4 yr Degree|9500|High Risk|1|0|0| +9702|F|M|4 yr Degree|9500|High Risk|1|0|0| +9703|M|S|4 yr Degree|9500|High Risk|1|0|0| +9704|F|S|4 yr Degree|9500|High Risk|1|0|0| +9705|M|D|4 yr Degree|9500|High Risk|1|0|0| +9706|F|D|4 yr Degree|9500|High Risk|1|0|0| +9707|M|W|4 yr Degree|9500|High Risk|1|0|0| +9708|F|W|4 yr Degree|9500|High Risk|1|0|0| +9709|M|U|4 yr Degree|9500|High Risk|1|0|0| +9710|F|U|4 yr Degree|9500|High Risk|1|0|0| +9711|M|M|Advanced Degree|9500|High Risk|1|0|0| +9712|F|M|Advanced Degree|9500|High Risk|1|0|0| +9713|M|S|Advanced Degree|9500|High Risk|1|0|0| +9714|F|S|Advanced Degree|9500|High Risk|1|0|0| +9715|M|D|Advanced Degree|9500|High Risk|1|0|0| +9716|F|D|Advanced Degree|9500|High Risk|1|0|0| +9717|M|W|Advanced Degree|9500|High Risk|1|0|0| +9718|F|W|Advanced Degree|9500|High Risk|1|0|0| +9719|M|U|Advanced Degree|9500|High Risk|1|0|0| +9720|F|U|Advanced Degree|9500|High Risk|1|0|0| +9721|M|M|Unknown|9500|High Risk|1|0|0| +9722|F|M|Unknown|9500|High Risk|1|0|0| +9723|M|S|Unknown|9500|High Risk|1|0|0| +9724|F|S|Unknown|9500|High Risk|1|0|0| +9725|M|D|Unknown|9500|High Risk|1|0|0| +9726|F|D|Unknown|9500|High Risk|1|0|0| +9727|M|W|Unknown|9500|High Risk|1|0|0| +9728|F|W|Unknown|9500|High Risk|1|0|0| +9729|M|U|Unknown|9500|High Risk|1|0|0| +9730|F|U|Unknown|9500|High Risk|1|0|0| +9731|M|M|Primary|10000|High Risk|1|0|0| +9732|F|M|Primary|10000|High Risk|1|0|0| +9733|M|S|Primary|10000|High Risk|1|0|0| +9734|F|S|Primary|10000|High Risk|1|0|0| +9735|M|D|Primary|10000|High Risk|1|0|0| +9736|F|D|Primary|10000|High Risk|1|0|0| +9737|M|W|Primary|10000|High Risk|1|0|0| +9738|F|W|Primary|10000|High Risk|1|0|0| +9739|M|U|Primary|10000|High Risk|1|0|0| +9740|F|U|Primary|10000|High Risk|1|0|0| +9741|M|M|Secondary|10000|High Risk|1|0|0| +9742|F|M|Secondary|10000|High Risk|1|0|0| +9743|M|S|Secondary|10000|High Risk|1|0|0| +9744|F|S|Secondary|10000|High Risk|1|0|0| +9745|M|D|Secondary|10000|High Risk|1|0|0| +9746|F|D|Secondary|10000|High Risk|1|0|0| +9747|M|W|Secondary|10000|High Risk|1|0|0| +9748|F|W|Secondary|10000|High Risk|1|0|0| +9749|M|U|Secondary|10000|High Risk|1|0|0| +9750|F|U|Secondary|10000|High Risk|1|0|0| +9751|M|M|College|10000|High Risk|1|0|0| +9752|F|M|College|10000|High Risk|1|0|0| +9753|M|S|College|10000|High Risk|1|0|0| +9754|F|S|College|10000|High Risk|1|0|0| +9755|M|D|College|10000|High Risk|1|0|0| +9756|F|D|College|10000|High Risk|1|0|0| +9757|M|W|College|10000|High Risk|1|0|0| +9758|F|W|College|10000|High Risk|1|0|0| +9759|M|U|College|10000|High Risk|1|0|0| +9760|F|U|College|10000|High Risk|1|0|0| +9761|M|M|2 yr Degree|10000|High Risk|1|0|0| +9762|F|M|2 yr Degree|10000|High Risk|1|0|0| +9763|M|S|2 yr Degree|10000|High Risk|1|0|0| +9764|F|S|2 yr Degree|10000|High Risk|1|0|0| +9765|M|D|2 yr Degree|10000|High Risk|1|0|0| +9766|F|D|2 yr Degree|10000|High Risk|1|0|0| +9767|M|W|2 yr Degree|10000|High Risk|1|0|0| +9768|F|W|2 yr Degree|10000|High Risk|1|0|0| +9769|M|U|2 yr Degree|10000|High Risk|1|0|0| +9770|F|U|2 yr Degree|10000|High Risk|1|0|0| +9771|M|M|4 yr Degree|10000|High Risk|1|0|0| +9772|F|M|4 yr Degree|10000|High Risk|1|0|0| +9773|M|S|4 yr Degree|10000|High Risk|1|0|0| +9774|F|S|4 yr Degree|10000|High Risk|1|0|0| +9775|M|D|4 yr Degree|10000|High Risk|1|0|0| +9776|F|D|4 yr Degree|10000|High Risk|1|0|0| +9777|M|W|4 yr Degree|10000|High Risk|1|0|0| +9778|F|W|4 yr Degree|10000|High Risk|1|0|0| +9779|M|U|4 yr Degree|10000|High Risk|1|0|0| +9780|F|U|4 yr Degree|10000|High Risk|1|0|0| +9781|M|M|Advanced Degree|10000|High Risk|1|0|0| +9782|F|M|Advanced Degree|10000|High Risk|1|0|0| +9783|M|S|Advanced Degree|10000|High Risk|1|0|0| +9784|F|S|Advanced Degree|10000|High Risk|1|0|0| +9785|M|D|Advanced Degree|10000|High Risk|1|0|0| +9786|F|D|Advanced Degree|10000|High Risk|1|0|0| +9787|M|W|Advanced Degree|10000|High Risk|1|0|0| +9788|F|W|Advanced Degree|10000|High Risk|1|0|0| +9789|M|U|Advanced Degree|10000|High Risk|1|0|0| +9790|F|U|Advanced Degree|10000|High Risk|1|0|0| +9791|M|M|Unknown|10000|High Risk|1|0|0| +9792|F|M|Unknown|10000|High Risk|1|0|0| +9793|M|S|Unknown|10000|High Risk|1|0|0| +9794|F|S|Unknown|10000|High Risk|1|0|0| +9795|M|D|Unknown|10000|High Risk|1|0|0| +9796|F|D|Unknown|10000|High Risk|1|0|0| +9797|M|W|Unknown|10000|High Risk|1|0|0| +9798|F|W|Unknown|10000|High Risk|1|0|0| +9799|M|U|Unknown|10000|High Risk|1|0|0| +9800|F|U|Unknown|10000|High Risk|1|0|0| +9801|M|M|Primary|500|Unknown|1|0|0| +9802|F|M|Primary|500|Unknown|1|0|0| +9803|M|S|Primary|500|Unknown|1|0|0| +9804|F|S|Primary|500|Unknown|1|0|0| +9805|M|D|Primary|500|Unknown|1|0|0| +9806|F|D|Primary|500|Unknown|1|0|0| +9807|M|W|Primary|500|Unknown|1|0|0| +9808|F|W|Primary|500|Unknown|1|0|0| +9809|M|U|Primary|500|Unknown|1|0|0| +9810|F|U|Primary|500|Unknown|1|0|0| +9811|M|M|Secondary|500|Unknown|1|0|0| +9812|F|M|Secondary|500|Unknown|1|0|0| +9813|M|S|Secondary|500|Unknown|1|0|0| +9814|F|S|Secondary|500|Unknown|1|0|0| +9815|M|D|Secondary|500|Unknown|1|0|0| +9816|F|D|Secondary|500|Unknown|1|0|0| +9817|M|W|Secondary|500|Unknown|1|0|0| +9818|F|W|Secondary|500|Unknown|1|0|0| +9819|M|U|Secondary|500|Unknown|1|0|0| +9820|F|U|Secondary|500|Unknown|1|0|0| +9821|M|M|College|500|Unknown|1|0|0| +9822|F|M|College|500|Unknown|1|0|0| +9823|M|S|College|500|Unknown|1|0|0| +9824|F|S|College|500|Unknown|1|0|0| +9825|M|D|College|500|Unknown|1|0|0| +9826|F|D|College|500|Unknown|1|0|0| +9827|M|W|College|500|Unknown|1|0|0| +9828|F|W|College|500|Unknown|1|0|0| +9829|M|U|College|500|Unknown|1|0|0| +9830|F|U|College|500|Unknown|1|0|0| +9831|M|M|2 yr Degree|500|Unknown|1|0|0| +9832|F|M|2 yr Degree|500|Unknown|1|0|0| +9833|M|S|2 yr Degree|500|Unknown|1|0|0| +9834|F|S|2 yr Degree|500|Unknown|1|0|0| +9835|M|D|2 yr Degree|500|Unknown|1|0|0| +9836|F|D|2 yr Degree|500|Unknown|1|0|0| +9837|M|W|2 yr Degree|500|Unknown|1|0|0| +9838|F|W|2 yr Degree|500|Unknown|1|0|0| +9839|M|U|2 yr Degree|500|Unknown|1|0|0| +9840|F|U|2 yr Degree|500|Unknown|1|0|0| +9841|M|M|4 yr Degree|500|Unknown|1|0|0| +9842|F|M|4 yr Degree|500|Unknown|1|0|0| +9843|M|S|4 yr Degree|500|Unknown|1|0|0| +9844|F|S|4 yr Degree|500|Unknown|1|0|0| +9845|M|D|4 yr Degree|500|Unknown|1|0|0| +9846|F|D|4 yr Degree|500|Unknown|1|0|0| +9847|M|W|4 yr Degree|500|Unknown|1|0|0| +9848|F|W|4 yr Degree|500|Unknown|1|0|0| +9849|M|U|4 yr Degree|500|Unknown|1|0|0| +9850|F|U|4 yr Degree|500|Unknown|1|0|0| +9851|M|M|Advanced Degree|500|Unknown|1|0|0| +9852|F|M|Advanced Degree|500|Unknown|1|0|0| +9853|M|S|Advanced Degree|500|Unknown|1|0|0| +9854|F|S|Advanced Degree|500|Unknown|1|0|0| +9855|M|D|Advanced Degree|500|Unknown|1|0|0| +9856|F|D|Advanced Degree|500|Unknown|1|0|0| +9857|M|W|Advanced Degree|500|Unknown|1|0|0| +9858|F|W|Advanced Degree|500|Unknown|1|0|0| +9859|M|U|Advanced Degree|500|Unknown|1|0|0| +9860|F|U|Advanced Degree|500|Unknown|1|0|0| +9861|M|M|Unknown|500|Unknown|1|0|0| +9862|F|M|Unknown|500|Unknown|1|0|0| +9863|M|S|Unknown|500|Unknown|1|0|0| +9864|F|S|Unknown|500|Unknown|1|0|0| +9865|M|D|Unknown|500|Unknown|1|0|0| +9866|F|D|Unknown|500|Unknown|1|0|0| +9867|M|W|Unknown|500|Unknown|1|0|0| +9868|F|W|Unknown|500|Unknown|1|0|0| +9869|M|U|Unknown|500|Unknown|1|0|0| +9870|F|U|Unknown|500|Unknown|1|0|0| +9871|M|M|Primary|1000|Unknown|1|0|0| +9872|F|M|Primary|1000|Unknown|1|0|0| +9873|M|S|Primary|1000|Unknown|1|0|0| +9874|F|S|Primary|1000|Unknown|1|0|0| +9875|M|D|Primary|1000|Unknown|1|0|0| +9876|F|D|Primary|1000|Unknown|1|0|0| +9877|M|W|Primary|1000|Unknown|1|0|0| +9878|F|W|Primary|1000|Unknown|1|0|0| +9879|M|U|Primary|1000|Unknown|1|0|0| +9880|F|U|Primary|1000|Unknown|1|0|0| +9881|M|M|Secondary|1000|Unknown|1|0|0| +9882|F|M|Secondary|1000|Unknown|1|0|0| +9883|M|S|Secondary|1000|Unknown|1|0|0| +9884|F|S|Secondary|1000|Unknown|1|0|0| +9885|M|D|Secondary|1000|Unknown|1|0|0| +9886|F|D|Secondary|1000|Unknown|1|0|0| +9887|M|W|Secondary|1000|Unknown|1|0|0| +9888|F|W|Secondary|1000|Unknown|1|0|0| +9889|M|U|Secondary|1000|Unknown|1|0|0| +9890|F|U|Secondary|1000|Unknown|1|0|0| +9891|M|M|College|1000|Unknown|1|0|0| +9892|F|M|College|1000|Unknown|1|0|0| +9893|M|S|College|1000|Unknown|1|0|0| +9894|F|S|College|1000|Unknown|1|0|0| +9895|M|D|College|1000|Unknown|1|0|0| +9896|F|D|College|1000|Unknown|1|0|0| +9897|M|W|College|1000|Unknown|1|0|0| +9898|F|W|College|1000|Unknown|1|0|0| +9899|M|U|College|1000|Unknown|1|0|0| +9900|F|U|College|1000|Unknown|1|0|0| +9901|M|M|2 yr Degree|1000|Unknown|1|0|0| +9902|F|M|2 yr Degree|1000|Unknown|1|0|0| +9903|M|S|2 yr Degree|1000|Unknown|1|0|0| +9904|F|S|2 yr Degree|1000|Unknown|1|0|0| +9905|M|D|2 yr Degree|1000|Unknown|1|0|0| +9906|F|D|2 yr Degree|1000|Unknown|1|0|0| +9907|M|W|2 yr Degree|1000|Unknown|1|0|0| +9908|F|W|2 yr Degree|1000|Unknown|1|0|0| +9909|M|U|2 yr Degree|1000|Unknown|1|0|0| +9910|F|U|2 yr Degree|1000|Unknown|1|0|0| +9911|M|M|4 yr Degree|1000|Unknown|1|0|0| +9912|F|M|4 yr Degree|1000|Unknown|1|0|0| +9913|M|S|4 yr Degree|1000|Unknown|1|0|0| +9914|F|S|4 yr Degree|1000|Unknown|1|0|0| +9915|M|D|4 yr Degree|1000|Unknown|1|0|0| +9916|F|D|4 yr Degree|1000|Unknown|1|0|0| +9917|M|W|4 yr Degree|1000|Unknown|1|0|0| +9918|F|W|4 yr Degree|1000|Unknown|1|0|0| +9919|M|U|4 yr Degree|1000|Unknown|1|0|0| +9920|F|U|4 yr Degree|1000|Unknown|1|0|0| +9921|M|M|Advanced Degree|1000|Unknown|1|0|0| +9922|F|M|Advanced Degree|1000|Unknown|1|0|0| +9923|M|S|Advanced Degree|1000|Unknown|1|0|0| +9924|F|S|Advanced Degree|1000|Unknown|1|0|0| +9925|M|D|Advanced Degree|1000|Unknown|1|0|0| +9926|F|D|Advanced Degree|1000|Unknown|1|0|0| +9927|M|W|Advanced Degree|1000|Unknown|1|0|0| +9928|F|W|Advanced Degree|1000|Unknown|1|0|0| +9929|M|U|Advanced Degree|1000|Unknown|1|0|0| +9930|F|U|Advanced Degree|1000|Unknown|1|0|0| +9931|M|M|Unknown|1000|Unknown|1|0|0| +9932|F|M|Unknown|1000|Unknown|1|0|0| +9933|M|S|Unknown|1000|Unknown|1|0|0| +9934|F|S|Unknown|1000|Unknown|1|0|0| +9935|M|D|Unknown|1000|Unknown|1|0|0| +9936|F|D|Unknown|1000|Unknown|1|0|0| +9937|M|W|Unknown|1000|Unknown|1|0|0| +9938|F|W|Unknown|1000|Unknown|1|0|0| +9939|M|U|Unknown|1000|Unknown|1|0|0| +9940|F|U|Unknown|1000|Unknown|1|0|0| +9941|M|M|Primary|1500|Unknown|1|0|0| +9942|F|M|Primary|1500|Unknown|1|0|0| +9943|M|S|Primary|1500|Unknown|1|0|0| +9944|F|S|Primary|1500|Unknown|1|0|0| +9945|M|D|Primary|1500|Unknown|1|0|0| +9946|F|D|Primary|1500|Unknown|1|0|0| +9947|M|W|Primary|1500|Unknown|1|0|0| +9948|F|W|Primary|1500|Unknown|1|0|0| +9949|M|U|Primary|1500|Unknown|1|0|0| +9950|F|U|Primary|1500|Unknown|1|0|0| +9951|M|M|Secondary|1500|Unknown|1|0|0| +9952|F|M|Secondary|1500|Unknown|1|0|0| +9953|M|S|Secondary|1500|Unknown|1|0|0| +9954|F|S|Secondary|1500|Unknown|1|0|0| +9955|M|D|Secondary|1500|Unknown|1|0|0| +9956|F|D|Secondary|1500|Unknown|1|0|0| +9957|M|W|Secondary|1500|Unknown|1|0|0| +9958|F|W|Secondary|1500|Unknown|1|0|0| +9959|M|U|Secondary|1500|Unknown|1|0|0| +9960|F|U|Secondary|1500|Unknown|1|0|0| +9961|M|M|College|1500|Unknown|1|0|0| +9962|F|M|College|1500|Unknown|1|0|0| +9963|M|S|College|1500|Unknown|1|0|0| +9964|F|S|College|1500|Unknown|1|0|0| +9965|M|D|College|1500|Unknown|1|0|0| +9966|F|D|College|1500|Unknown|1|0|0| +9967|M|W|College|1500|Unknown|1|0|0| +9968|F|W|College|1500|Unknown|1|0|0| +9969|M|U|College|1500|Unknown|1|0|0| +9970|F|U|College|1500|Unknown|1|0|0| +9971|M|M|2 yr Degree|1500|Unknown|1|0|0| +9972|F|M|2 yr Degree|1500|Unknown|1|0|0| +9973|M|S|2 yr Degree|1500|Unknown|1|0|0| +9974|F|S|2 yr Degree|1500|Unknown|1|0|0| +9975|M|D|2 yr Degree|1500|Unknown|1|0|0| +9976|F|D|2 yr Degree|1500|Unknown|1|0|0| +9977|M|W|2 yr Degree|1500|Unknown|1|0|0| +9978|F|W|2 yr Degree|1500|Unknown|1|0|0| +9979|M|U|2 yr Degree|1500|Unknown|1|0|0| +9980|F|U|2 yr Degree|1500|Unknown|1|0|0| +9981|M|M|4 yr Degree|1500|Unknown|1|0|0| +9982|F|M|4 yr Degree|1500|Unknown|1|0|0| +9983|M|S|4 yr Degree|1500|Unknown|1|0|0| +9984|F|S|4 yr Degree|1500|Unknown|1|0|0| +9985|M|D|4 yr Degree|1500|Unknown|1|0|0| +9986|F|D|4 yr Degree|1500|Unknown|1|0|0| +9987|M|W|4 yr Degree|1500|Unknown|1|0|0| +9988|F|W|4 yr Degree|1500|Unknown|1|0|0| +9989|M|U|4 yr Degree|1500|Unknown|1|0|0| +9990|F|U|4 yr Degree|1500|Unknown|1|0|0| +9991|M|M|Advanced Degree|1500|Unknown|1|0|0| +9992|F|M|Advanced Degree|1500|Unknown|1|0|0| +9993|M|S|Advanced Degree|1500|Unknown|1|0|0| +9994|F|S|Advanced Degree|1500|Unknown|1|0|0| +9995|M|D|Advanced Degree|1500|Unknown|1|0|0| +9996|F|D|Advanced Degree|1500|Unknown|1|0|0| +9997|M|W|Advanced Degree|1500|Unknown|1|0|0| +9998|F|W|Advanced Degree|1500|Unknown|1|0|0| +9999|M|U|Advanced Degree|1500|Unknown|1|0|0| +10000|F|U|Advanced Degree|1500|Unknown|1|0|0| +10001|M|M|Unknown|1500|Unknown|1|0|0| +10002|F|M|Unknown|1500|Unknown|1|0|0| +10003|M|S|Unknown|1500|Unknown|1|0|0| +10004|F|S|Unknown|1500|Unknown|1|0|0| +10005|M|D|Unknown|1500|Unknown|1|0|0| +10006|F|D|Unknown|1500|Unknown|1|0|0| +10007|M|W|Unknown|1500|Unknown|1|0|0| +10008|F|W|Unknown|1500|Unknown|1|0|0| +10009|M|U|Unknown|1500|Unknown|1|0|0| +10010|F|U|Unknown|1500|Unknown|1|0|0| +10011|M|M|Primary|2000|Unknown|1|0|0| +10012|F|M|Primary|2000|Unknown|1|0|0| +10013|M|S|Primary|2000|Unknown|1|0|0| +10014|F|S|Primary|2000|Unknown|1|0|0| +10015|M|D|Primary|2000|Unknown|1|0|0| +10016|F|D|Primary|2000|Unknown|1|0|0| +10017|M|W|Primary|2000|Unknown|1|0|0| +10018|F|W|Primary|2000|Unknown|1|0|0| +10019|M|U|Primary|2000|Unknown|1|0|0| +10020|F|U|Primary|2000|Unknown|1|0|0| +10021|M|M|Secondary|2000|Unknown|1|0|0| +10022|F|M|Secondary|2000|Unknown|1|0|0| +10023|M|S|Secondary|2000|Unknown|1|0|0| +10024|F|S|Secondary|2000|Unknown|1|0|0| +10025|M|D|Secondary|2000|Unknown|1|0|0| +10026|F|D|Secondary|2000|Unknown|1|0|0| +10027|M|W|Secondary|2000|Unknown|1|0|0| +10028|F|W|Secondary|2000|Unknown|1|0|0| +10029|M|U|Secondary|2000|Unknown|1|0|0| +10030|F|U|Secondary|2000|Unknown|1|0|0| +10031|M|M|College|2000|Unknown|1|0|0| +10032|F|M|College|2000|Unknown|1|0|0| +10033|M|S|College|2000|Unknown|1|0|0| +10034|F|S|College|2000|Unknown|1|0|0| +10035|M|D|College|2000|Unknown|1|0|0| +10036|F|D|College|2000|Unknown|1|0|0| +10037|M|W|College|2000|Unknown|1|0|0| +10038|F|W|College|2000|Unknown|1|0|0| +10039|M|U|College|2000|Unknown|1|0|0| +10040|F|U|College|2000|Unknown|1|0|0| +10041|M|M|2 yr Degree|2000|Unknown|1|0|0| +10042|F|M|2 yr Degree|2000|Unknown|1|0|0| +10043|M|S|2 yr Degree|2000|Unknown|1|0|0| +10044|F|S|2 yr Degree|2000|Unknown|1|0|0| +10045|M|D|2 yr Degree|2000|Unknown|1|0|0| +10046|F|D|2 yr Degree|2000|Unknown|1|0|0| +10047|M|W|2 yr Degree|2000|Unknown|1|0|0| +10048|F|W|2 yr Degree|2000|Unknown|1|0|0| +10049|M|U|2 yr Degree|2000|Unknown|1|0|0| +10050|F|U|2 yr Degree|2000|Unknown|1|0|0| +10051|M|M|4 yr Degree|2000|Unknown|1|0|0| +10052|F|M|4 yr Degree|2000|Unknown|1|0|0| +10053|M|S|4 yr Degree|2000|Unknown|1|0|0| +10054|F|S|4 yr Degree|2000|Unknown|1|0|0| +10055|M|D|4 yr Degree|2000|Unknown|1|0|0| +10056|F|D|4 yr Degree|2000|Unknown|1|0|0| +10057|M|W|4 yr Degree|2000|Unknown|1|0|0| +10058|F|W|4 yr Degree|2000|Unknown|1|0|0| +10059|M|U|4 yr Degree|2000|Unknown|1|0|0| +10060|F|U|4 yr Degree|2000|Unknown|1|0|0| +10061|M|M|Advanced Degree|2000|Unknown|1|0|0| +10062|F|M|Advanced Degree|2000|Unknown|1|0|0| +10063|M|S|Advanced Degree|2000|Unknown|1|0|0| +10064|F|S|Advanced Degree|2000|Unknown|1|0|0| +10065|M|D|Advanced Degree|2000|Unknown|1|0|0| +10066|F|D|Advanced Degree|2000|Unknown|1|0|0| +10067|M|W|Advanced Degree|2000|Unknown|1|0|0| +10068|F|W|Advanced Degree|2000|Unknown|1|0|0| +10069|M|U|Advanced Degree|2000|Unknown|1|0|0| +10070|F|U|Advanced Degree|2000|Unknown|1|0|0| +10071|M|M|Unknown|2000|Unknown|1|0|0| +10072|F|M|Unknown|2000|Unknown|1|0|0| +10073|M|S|Unknown|2000|Unknown|1|0|0| +10074|F|S|Unknown|2000|Unknown|1|0|0| +10075|M|D|Unknown|2000|Unknown|1|0|0| +10076|F|D|Unknown|2000|Unknown|1|0|0| +10077|M|W|Unknown|2000|Unknown|1|0|0| +10078|F|W|Unknown|2000|Unknown|1|0|0| +10079|M|U|Unknown|2000|Unknown|1|0|0| +10080|F|U|Unknown|2000|Unknown|1|0|0| +10081|M|M|Primary|2500|Unknown|1|0|0| +10082|F|M|Primary|2500|Unknown|1|0|0| +10083|M|S|Primary|2500|Unknown|1|0|0| +10084|F|S|Primary|2500|Unknown|1|0|0| +10085|M|D|Primary|2500|Unknown|1|0|0| +10086|F|D|Primary|2500|Unknown|1|0|0| +10087|M|W|Primary|2500|Unknown|1|0|0| +10088|F|W|Primary|2500|Unknown|1|0|0| +10089|M|U|Primary|2500|Unknown|1|0|0| +10090|F|U|Primary|2500|Unknown|1|0|0| +10091|M|M|Secondary|2500|Unknown|1|0|0| +10092|F|M|Secondary|2500|Unknown|1|0|0| +10093|M|S|Secondary|2500|Unknown|1|0|0| +10094|F|S|Secondary|2500|Unknown|1|0|0| +10095|M|D|Secondary|2500|Unknown|1|0|0| +10096|F|D|Secondary|2500|Unknown|1|0|0| +10097|M|W|Secondary|2500|Unknown|1|0|0| +10098|F|W|Secondary|2500|Unknown|1|0|0| +10099|M|U|Secondary|2500|Unknown|1|0|0| +10100|F|U|Secondary|2500|Unknown|1|0|0| +10101|M|M|College|2500|Unknown|1|0|0| +10102|F|M|College|2500|Unknown|1|0|0| +10103|M|S|College|2500|Unknown|1|0|0| +10104|F|S|College|2500|Unknown|1|0|0| +10105|M|D|College|2500|Unknown|1|0|0| +10106|F|D|College|2500|Unknown|1|0|0| +10107|M|W|College|2500|Unknown|1|0|0| +10108|F|W|College|2500|Unknown|1|0|0| +10109|M|U|College|2500|Unknown|1|0|0| +10110|F|U|College|2500|Unknown|1|0|0| +10111|M|M|2 yr Degree|2500|Unknown|1|0|0| +10112|F|M|2 yr Degree|2500|Unknown|1|0|0| +10113|M|S|2 yr Degree|2500|Unknown|1|0|0| +10114|F|S|2 yr Degree|2500|Unknown|1|0|0| +10115|M|D|2 yr Degree|2500|Unknown|1|0|0| +10116|F|D|2 yr Degree|2500|Unknown|1|0|0| +10117|M|W|2 yr Degree|2500|Unknown|1|0|0| +10118|F|W|2 yr Degree|2500|Unknown|1|0|0| +10119|M|U|2 yr Degree|2500|Unknown|1|0|0| +10120|F|U|2 yr Degree|2500|Unknown|1|0|0| +10121|M|M|4 yr Degree|2500|Unknown|1|0|0| +10122|F|M|4 yr Degree|2500|Unknown|1|0|0| +10123|M|S|4 yr Degree|2500|Unknown|1|0|0| +10124|F|S|4 yr Degree|2500|Unknown|1|0|0| +10125|M|D|4 yr Degree|2500|Unknown|1|0|0| +10126|F|D|4 yr Degree|2500|Unknown|1|0|0| +10127|M|W|4 yr Degree|2500|Unknown|1|0|0| +10128|F|W|4 yr Degree|2500|Unknown|1|0|0| +10129|M|U|4 yr Degree|2500|Unknown|1|0|0| +10130|F|U|4 yr Degree|2500|Unknown|1|0|0| +10131|M|M|Advanced Degree|2500|Unknown|1|0|0| +10132|F|M|Advanced Degree|2500|Unknown|1|0|0| +10133|M|S|Advanced Degree|2500|Unknown|1|0|0| +10134|F|S|Advanced Degree|2500|Unknown|1|0|0| +10135|M|D|Advanced Degree|2500|Unknown|1|0|0| +10136|F|D|Advanced Degree|2500|Unknown|1|0|0| +10137|M|W|Advanced Degree|2500|Unknown|1|0|0| +10138|F|W|Advanced Degree|2500|Unknown|1|0|0| +10139|M|U|Advanced Degree|2500|Unknown|1|0|0| +10140|F|U|Advanced Degree|2500|Unknown|1|0|0| +10141|M|M|Unknown|2500|Unknown|1|0|0| +10142|F|M|Unknown|2500|Unknown|1|0|0| +10143|M|S|Unknown|2500|Unknown|1|0|0| +10144|F|S|Unknown|2500|Unknown|1|0|0| +10145|M|D|Unknown|2500|Unknown|1|0|0| +10146|F|D|Unknown|2500|Unknown|1|0|0| +10147|M|W|Unknown|2500|Unknown|1|0|0| +10148|F|W|Unknown|2500|Unknown|1|0|0| +10149|M|U|Unknown|2500|Unknown|1|0|0| +10150|F|U|Unknown|2500|Unknown|1|0|0| +10151|M|M|Primary|3000|Unknown|1|0|0| +10152|F|M|Primary|3000|Unknown|1|0|0| +10153|M|S|Primary|3000|Unknown|1|0|0| +10154|F|S|Primary|3000|Unknown|1|0|0| +10155|M|D|Primary|3000|Unknown|1|0|0| +10156|F|D|Primary|3000|Unknown|1|0|0| +10157|M|W|Primary|3000|Unknown|1|0|0| +10158|F|W|Primary|3000|Unknown|1|0|0| +10159|M|U|Primary|3000|Unknown|1|0|0| +10160|F|U|Primary|3000|Unknown|1|0|0| +10161|M|M|Secondary|3000|Unknown|1|0|0| +10162|F|M|Secondary|3000|Unknown|1|0|0| +10163|M|S|Secondary|3000|Unknown|1|0|0| +10164|F|S|Secondary|3000|Unknown|1|0|0| +10165|M|D|Secondary|3000|Unknown|1|0|0| +10166|F|D|Secondary|3000|Unknown|1|0|0| +10167|M|W|Secondary|3000|Unknown|1|0|0| +10168|F|W|Secondary|3000|Unknown|1|0|0| +10169|M|U|Secondary|3000|Unknown|1|0|0| +10170|F|U|Secondary|3000|Unknown|1|0|0| +10171|M|M|College|3000|Unknown|1|0|0| +10172|F|M|College|3000|Unknown|1|0|0| +10173|M|S|College|3000|Unknown|1|0|0| +10174|F|S|College|3000|Unknown|1|0|0| +10175|M|D|College|3000|Unknown|1|0|0| +10176|F|D|College|3000|Unknown|1|0|0| +10177|M|W|College|3000|Unknown|1|0|0| +10178|F|W|College|3000|Unknown|1|0|0| +10179|M|U|College|3000|Unknown|1|0|0| +10180|F|U|College|3000|Unknown|1|0|0| +10181|M|M|2 yr Degree|3000|Unknown|1|0|0| +10182|F|M|2 yr Degree|3000|Unknown|1|0|0| +10183|M|S|2 yr Degree|3000|Unknown|1|0|0| +10184|F|S|2 yr Degree|3000|Unknown|1|0|0| +10185|M|D|2 yr Degree|3000|Unknown|1|0|0| +10186|F|D|2 yr Degree|3000|Unknown|1|0|0| +10187|M|W|2 yr Degree|3000|Unknown|1|0|0| +10188|F|W|2 yr Degree|3000|Unknown|1|0|0| +10189|M|U|2 yr Degree|3000|Unknown|1|0|0| +10190|F|U|2 yr Degree|3000|Unknown|1|0|0| +10191|M|M|4 yr Degree|3000|Unknown|1|0|0| +10192|F|M|4 yr Degree|3000|Unknown|1|0|0| +10193|M|S|4 yr Degree|3000|Unknown|1|0|0| +10194|F|S|4 yr Degree|3000|Unknown|1|0|0| +10195|M|D|4 yr Degree|3000|Unknown|1|0|0| +10196|F|D|4 yr Degree|3000|Unknown|1|0|0| +10197|M|W|4 yr Degree|3000|Unknown|1|0|0| +10198|F|W|4 yr Degree|3000|Unknown|1|0|0| +10199|M|U|4 yr Degree|3000|Unknown|1|0|0| +10200|F|U|4 yr Degree|3000|Unknown|1|0|0| +10201|M|M|Advanced Degree|3000|Unknown|1|0|0| +10202|F|M|Advanced Degree|3000|Unknown|1|0|0| +10203|M|S|Advanced Degree|3000|Unknown|1|0|0| +10204|F|S|Advanced Degree|3000|Unknown|1|0|0| +10205|M|D|Advanced Degree|3000|Unknown|1|0|0| +10206|F|D|Advanced Degree|3000|Unknown|1|0|0| +10207|M|W|Advanced Degree|3000|Unknown|1|0|0| +10208|F|W|Advanced Degree|3000|Unknown|1|0|0| +10209|M|U|Advanced Degree|3000|Unknown|1|0|0| +10210|F|U|Advanced Degree|3000|Unknown|1|0|0| +10211|M|M|Unknown|3000|Unknown|1|0|0| +10212|F|M|Unknown|3000|Unknown|1|0|0| +10213|M|S|Unknown|3000|Unknown|1|0|0| +10214|F|S|Unknown|3000|Unknown|1|0|0| +10215|M|D|Unknown|3000|Unknown|1|0|0| +10216|F|D|Unknown|3000|Unknown|1|0|0| +10217|M|W|Unknown|3000|Unknown|1|0|0| +10218|F|W|Unknown|3000|Unknown|1|0|0| +10219|M|U|Unknown|3000|Unknown|1|0|0| +10220|F|U|Unknown|3000|Unknown|1|0|0| +10221|M|M|Primary|3500|Unknown|1|0|0| +10222|F|M|Primary|3500|Unknown|1|0|0| +10223|M|S|Primary|3500|Unknown|1|0|0| +10224|F|S|Primary|3500|Unknown|1|0|0| +10225|M|D|Primary|3500|Unknown|1|0|0| +10226|F|D|Primary|3500|Unknown|1|0|0| +10227|M|W|Primary|3500|Unknown|1|0|0| +10228|F|W|Primary|3500|Unknown|1|0|0| +10229|M|U|Primary|3500|Unknown|1|0|0| +10230|F|U|Primary|3500|Unknown|1|0|0| +10231|M|M|Secondary|3500|Unknown|1|0|0| +10232|F|M|Secondary|3500|Unknown|1|0|0| +10233|M|S|Secondary|3500|Unknown|1|0|0| +10234|F|S|Secondary|3500|Unknown|1|0|0| +10235|M|D|Secondary|3500|Unknown|1|0|0| +10236|F|D|Secondary|3500|Unknown|1|0|0| +10237|M|W|Secondary|3500|Unknown|1|0|0| +10238|F|W|Secondary|3500|Unknown|1|0|0| +10239|M|U|Secondary|3500|Unknown|1|0|0| +10240|F|U|Secondary|3500|Unknown|1|0|0| +10241|M|M|College|3500|Unknown|1|0|0| +10242|F|M|College|3500|Unknown|1|0|0| +10243|M|S|College|3500|Unknown|1|0|0| +10244|F|S|College|3500|Unknown|1|0|0| +10245|M|D|College|3500|Unknown|1|0|0| +10246|F|D|College|3500|Unknown|1|0|0| +10247|M|W|College|3500|Unknown|1|0|0| +10248|F|W|College|3500|Unknown|1|0|0| +10249|M|U|College|3500|Unknown|1|0|0| +10250|F|U|College|3500|Unknown|1|0|0| +10251|M|M|2 yr Degree|3500|Unknown|1|0|0| +10252|F|M|2 yr Degree|3500|Unknown|1|0|0| +10253|M|S|2 yr Degree|3500|Unknown|1|0|0| +10254|F|S|2 yr Degree|3500|Unknown|1|0|0| +10255|M|D|2 yr Degree|3500|Unknown|1|0|0| +10256|F|D|2 yr Degree|3500|Unknown|1|0|0| +10257|M|W|2 yr Degree|3500|Unknown|1|0|0| +10258|F|W|2 yr Degree|3500|Unknown|1|0|0| +10259|M|U|2 yr Degree|3500|Unknown|1|0|0| +10260|F|U|2 yr Degree|3500|Unknown|1|0|0| +10261|M|M|4 yr Degree|3500|Unknown|1|0|0| +10262|F|M|4 yr Degree|3500|Unknown|1|0|0| +10263|M|S|4 yr Degree|3500|Unknown|1|0|0| +10264|F|S|4 yr Degree|3500|Unknown|1|0|0| +10265|M|D|4 yr Degree|3500|Unknown|1|0|0| +10266|F|D|4 yr Degree|3500|Unknown|1|0|0| +10267|M|W|4 yr Degree|3500|Unknown|1|0|0| +10268|F|W|4 yr Degree|3500|Unknown|1|0|0| +10269|M|U|4 yr Degree|3500|Unknown|1|0|0| +10270|F|U|4 yr Degree|3500|Unknown|1|0|0| +10271|M|M|Advanced Degree|3500|Unknown|1|0|0| +10272|F|M|Advanced Degree|3500|Unknown|1|0|0| +10273|M|S|Advanced Degree|3500|Unknown|1|0|0| +10274|F|S|Advanced Degree|3500|Unknown|1|0|0| +10275|M|D|Advanced Degree|3500|Unknown|1|0|0| +10276|F|D|Advanced Degree|3500|Unknown|1|0|0| +10277|M|W|Advanced Degree|3500|Unknown|1|0|0| +10278|F|W|Advanced Degree|3500|Unknown|1|0|0| +10279|M|U|Advanced Degree|3500|Unknown|1|0|0| +10280|F|U|Advanced Degree|3500|Unknown|1|0|0| +10281|M|M|Unknown|3500|Unknown|1|0|0| +10282|F|M|Unknown|3500|Unknown|1|0|0| +10283|M|S|Unknown|3500|Unknown|1|0|0| +10284|F|S|Unknown|3500|Unknown|1|0|0| +10285|M|D|Unknown|3500|Unknown|1|0|0| +10286|F|D|Unknown|3500|Unknown|1|0|0| +10287|M|W|Unknown|3500|Unknown|1|0|0| +10288|F|W|Unknown|3500|Unknown|1|0|0| +10289|M|U|Unknown|3500|Unknown|1|0|0| +10290|F|U|Unknown|3500|Unknown|1|0|0| +10291|M|M|Primary|4000|Unknown|1|0|0| +10292|F|M|Primary|4000|Unknown|1|0|0| +10293|M|S|Primary|4000|Unknown|1|0|0| +10294|F|S|Primary|4000|Unknown|1|0|0| +10295|M|D|Primary|4000|Unknown|1|0|0| +10296|F|D|Primary|4000|Unknown|1|0|0| +10297|M|W|Primary|4000|Unknown|1|0|0| +10298|F|W|Primary|4000|Unknown|1|0|0| +10299|M|U|Primary|4000|Unknown|1|0|0| +10300|F|U|Primary|4000|Unknown|1|0|0| +10301|M|M|Secondary|4000|Unknown|1|0|0| +10302|F|M|Secondary|4000|Unknown|1|0|0| +10303|M|S|Secondary|4000|Unknown|1|0|0| +10304|F|S|Secondary|4000|Unknown|1|0|0| +10305|M|D|Secondary|4000|Unknown|1|0|0| +10306|F|D|Secondary|4000|Unknown|1|0|0| +10307|M|W|Secondary|4000|Unknown|1|0|0| +10308|F|W|Secondary|4000|Unknown|1|0|0| +10309|M|U|Secondary|4000|Unknown|1|0|0| +10310|F|U|Secondary|4000|Unknown|1|0|0| +10311|M|M|College|4000|Unknown|1|0|0| +10312|F|M|College|4000|Unknown|1|0|0| +10313|M|S|College|4000|Unknown|1|0|0| +10314|F|S|College|4000|Unknown|1|0|0| +10315|M|D|College|4000|Unknown|1|0|0| +10316|F|D|College|4000|Unknown|1|0|0| +10317|M|W|College|4000|Unknown|1|0|0| +10318|F|W|College|4000|Unknown|1|0|0| +10319|M|U|College|4000|Unknown|1|0|0| +10320|F|U|College|4000|Unknown|1|0|0| +10321|M|M|2 yr Degree|4000|Unknown|1|0|0| +10322|F|M|2 yr Degree|4000|Unknown|1|0|0| +10323|M|S|2 yr Degree|4000|Unknown|1|0|0| +10324|F|S|2 yr Degree|4000|Unknown|1|0|0| +10325|M|D|2 yr Degree|4000|Unknown|1|0|0| +10326|F|D|2 yr Degree|4000|Unknown|1|0|0| +10327|M|W|2 yr Degree|4000|Unknown|1|0|0| +10328|F|W|2 yr Degree|4000|Unknown|1|0|0| +10329|M|U|2 yr Degree|4000|Unknown|1|0|0| +10330|F|U|2 yr Degree|4000|Unknown|1|0|0| +10331|M|M|4 yr Degree|4000|Unknown|1|0|0| +10332|F|M|4 yr Degree|4000|Unknown|1|0|0| +10333|M|S|4 yr Degree|4000|Unknown|1|0|0| +10334|F|S|4 yr Degree|4000|Unknown|1|0|0| +10335|M|D|4 yr Degree|4000|Unknown|1|0|0| +10336|F|D|4 yr Degree|4000|Unknown|1|0|0| +10337|M|W|4 yr Degree|4000|Unknown|1|0|0| +10338|F|W|4 yr Degree|4000|Unknown|1|0|0| +10339|M|U|4 yr Degree|4000|Unknown|1|0|0| +10340|F|U|4 yr Degree|4000|Unknown|1|0|0| +10341|M|M|Advanced Degree|4000|Unknown|1|0|0| +10342|F|M|Advanced Degree|4000|Unknown|1|0|0| +10343|M|S|Advanced Degree|4000|Unknown|1|0|0| +10344|F|S|Advanced Degree|4000|Unknown|1|0|0| +10345|M|D|Advanced Degree|4000|Unknown|1|0|0| +10346|F|D|Advanced Degree|4000|Unknown|1|0|0| +10347|M|W|Advanced Degree|4000|Unknown|1|0|0| +10348|F|W|Advanced Degree|4000|Unknown|1|0|0| +10349|M|U|Advanced Degree|4000|Unknown|1|0|0| +10350|F|U|Advanced Degree|4000|Unknown|1|0|0| +10351|M|M|Unknown|4000|Unknown|1|0|0| +10352|F|M|Unknown|4000|Unknown|1|0|0| +10353|M|S|Unknown|4000|Unknown|1|0|0| +10354|F|S|Unknown|4000|Unknown|1|0|0| +10355|M|D|Unknown|4000|Unknown|1|0|0| +10356|F|D|Unknown|4000|Unknown|1|0|0| +10357|M|W|Unknown|4000|Unknown|1|0|0| +10358|F|W|Unknown|4000|Unknown|1|0|0| +10359|M|U|Unknown|4000|Unknown|1|0|0| +10360|F|U|Unknown|4000|Unknown|1|0|0| +10361|M|M|Primary|4500|Unknown|1|0|0| +10362|F|M|Primary|4500|Unknown|1|0|0| +10363|M|S|Primary|4500|Unknown|1|0|0| +10364|F|S|Primary|4500|Unknown|1|0|0| +10365|M|D|Primary|4500|Unknown|1|0|0| +10366|F|D|Primary|4500|Unknown|1|0|0| +10367|M|W|Primary|4500|Unknown|1|0|0| +10368|F|W|Primary|4500|Unknown|1|0|0| +10369|M|U|Primary|4500|Unknown|1|0|0| +10370|F|U|Primary|4500|Unknown|1|0|0| +10371|M|M|Secondary|4500|Unknown|1|0|0| +10372|F|M|Secondary|4500|Unknown|1|0|0| +10373|M|S|Secondary|4500|Unknown|1|0|0| +10374|F|S|Secondary|4500|Unknown|1|0|0| +10375|M|D|Secondary|4500|Unknown|1|0|0| +10376|F|D|Secondary|4500|Unknown|1|0|0| +10377|M|W|Secondary|4500|Unknown|1|0|0| +10378|F|W|Secondary|4500|Unknown|1|0|0| +10379|M|U|Secondary|4500|Unknown|1|0|0| +10380|F|U|Secondary|4500|Unknown|1|0|0| +10381|M|M|College|4500|Unknown|1|0|0| +10382|F|M|College|4500|Unknown|1|0|0| +10383|M|S|College|4500|Unknown|1|0|0| +10384|F|S|College|4500|Unknown|1|0|0| +10385|M|D|College|4500|Unknown|1|0|0| +10386|F|D|College|4500|Unknown|1|0|0| +10387|M|W|College|4500|Unknown|1|0|0| +10388|F|W|College|4500|Unknown|1|0|0| +10389|M|U|College|4500|Unknown|1|0|0| +10390|F|U|College|4500|Unknown|1|0|0| +10391|M|M|2 yr Degree|4500|Unknown|1|0|0| +10392|F|M|2 yr Degree|4500|Unknown|1|0|0| +10393|M|S|2 yr Degree|4500|Unknown|1|0|0| +10394|F|S|2 yr Degree|4500|Unknown|1|0|0| +10395|M|D|2 yr Degree|4500|Unknown|1|0|0| +10396|F|D|2 yr Degree|4500|Unknown|1|0|0| +10397|M|W|2 yr Degree|4500|Unknown|1|0|0| +10398|F|W|2 yr Degree|4500|Unknown|1|0|0| +10399|M|U|2 yr Degree|4500|Unknown|1|0|0| +10400|F|U|2 yr Degree|4500|Unknown|1|0|0| +10401|M|M|4 yr Degree|4500|Unknown|1|0|0| +10402|F|M|4 yr Degree|4500|Unknown|1|0|0| +10403|M|S|4 yr Degree|4500|Unknown|1|0|0| +10404|F|S|4 yr Degree|4500|Unknown|1|0|0| +10405|M|D|4 yr Degree|4500|Unknown|1|0|0| +10406|F|D|4 yr Degree|4500|Unknown|1|0|0| +10407|M|W|4 yr Degree|4500|Unknown|1|0|0| +10408|F|W|4 yr Degree|4500|Unknown|1|0|0| +10409|M|U|4 yr Degree|4500|Unknown|1|0|0| +10410|F|U|4 yr Degree|4500|Unknown|1|0|0| +10411|M|M|Advanced Degree|4500|Unknown|1|0|0| +10412|F|M|Advanced Degree|4500|Unknown|1|0|0| +10413|M|S|Advanced Degree|4500|Unknown|1|0|0| +10414|F|S|Advanced Degree|4500|Unknown|1|0|0| +10415|M|D|Advanced Degree|4500|Unknown|1|0|0| +10416|F|D|Advanced Degree|4500|Unknown|1|0|0| +10417|M|W|Advanced Degree|4500|Unknown|1|0|0| +10418|F|W|Advanced Degree|4500|Unknown|1|0|0| +10419|M|U|Advanced Degree|4500|Unknown|1|0|0| +10420|F|U|Advanced Degree|4500|Unknown|1|0|0| +10421|M|M|Unknown|4500|Unknown|1|0|0| +10422|F|M|Unknown|4500|Unknown|1|0|0| +10423|M|S|Unknown|4500|Unknown|1|0|0| +10424|F|S|Unknown|4500|Unknown|1|0|0| +10425|M|D|Unknown|4500|Unknown|1|0|0| +10426|F|D|Unknown|4500|Unknown|1|0|0| +10427|M|W|Unknown|4500|Unknown|1|0|0| +10428|F|W|Unknown|4500|Unknown|1|0|0| +10429|M|U|Unknown|4500|Unknown|1|0|0| +10430|F|U|Unknown|4500|Unknown|1|0|0| +10431|M|M|Primary|5000|Unknown|1|0|0| +10432|F|M|Primary|5000|Unknown|1|0|0| +10433|M|S|Primary|5000|Unknown|1|0|0| +10434|F|S|Primary|5000|Unknown|1|0|0| +10435|M|D|Primary|5000|Unknown|1|0|0| +10436|F|D|Primary|5000|Unknown|1|0|0| +10437|M|W|Primary|5000|Unknown|1|0|0| +10438|F|W|Primary|5000|Unknown|1|0|0| +10439|M|U|Primary|5000|Unknown|1|0|0| +10440|F|U|Primary|5000|Unknown|1|0|0| +10441|M|M|Secondary|5000|Unknown|1|0|0| +10442|F|M|Secondary|5000|Unknown|1|0|0| +10443|M|S|Secondary|5000|Unknown|1|0|0| +10444|F|S|Secondary|5000|Unknown|1|0|0| +10445|M|D|Secondary|5000|Unknown|1|0|0| +10446|F|D|Secondary|5000|Unknown|1|0|0| +10447|M|W|Secondary|5000|Unknown|1|0|0| +10448|F|W|Secondary|5000|Unknown|1|0|0| +10449|M|U|Secondary|5000|Unknown|1|0|0| +10450|F|U|Secondary|5000|Unknown|1|0|0| +10451|M|M|College|5000|Unknown|1|0|0| +10452|F|M|College|5000|Unknown|1|0|0| +10453|M|S|College|5000|Unknown|1|0|0| +10454|F|S|College|5000|Unknown|1|0|0| +10455|M|D|College|5000|Unknown|1|0|0| +10456|F|D|College|5000|Unknown|1|0|0| +10457|M|W|College|5000|Unknown|1|0|0| +10458|F|W|College|5000|Unknown|1|0|0| +10459|M|U|College|5000|Unknown|1|0|0| +10460|F|U|College|5000|Unknown|1|0|0| +10461|M|M|2 yr Degree|5000|Unknown|1|0|0| +10462|F|M|2 yr Degree|5000|Unknown|1|0|0| +10463|M|S|2 yr Degree|5000|Unknown|1|0|0| +10464|F|S|2 yr Degree|5000|Unknown|1|0|0| +10465|M|D|2 yr Degree|5000|Unknown|1|0|0| +10466|F|D|2 yr Degree|5000|Unknown|1|0|0| +10467|M|W|2 yr Degree|5000|Unknown|1|0|0| +10468|F|W|2 yr Degree|5000|Unknown|1|0|0| +10469|M|U|2 yr Degree|5000|Unknown|1|0|0| +10470|F|U|2 yr Degree|5000|Unknown|1|0|0| +10471|M|M|4 yr Degree|5000|Unknown|1|0|0| +10472|F|M|4 yr Degree|5000|Unknown|1|0|0| +10473|M|S|4 yr Degree|5000|Unknown|1|0|0| +10474|F|S|4 yr Degree|5000|Unknown|1|0|0| +10475|M|D|4 yr Degree|5000|Unknown|1|0|0| +10476|F|D|4 yr Degree|5000|Unknown|1|0|0| +10477|M|W|4 yr Degree|5000|Unknown|1|0|0| +10478|F|W|4 yr Degree|5000|Unknown|1|0|0| +10479|M|U|4 yr Degree|5000|Unknown|1|0|0| +10480|F|U|4 yr Degree|5000|Unknown|1|0|0| +10481|M|M|Advanced Degree|5000|Unknown|1|0|0| +10482|F|M|Advanced Degree|5000|Unknown|1|0|0| +10483|M|S|Advanced Degree|5000|Unknown|1|0|0| +10484|F|S|Advanced Degree|5000|Unknown|1|0|0| +10485|M|D|Advanced Degree|5000|Unknown|1|0|0| +10486|F|D|Advanced Degree|5000|Unknown|1|0|0| +10487|M|W|Advanced Degree|5000|Unknown|1|0|0| +10488|F|W|Advanced Degree|5000|Unknown|1|0|0| +10489|M|U|Advanced Degree|5000|Unknown|1|0|0| +10490|F|U|Advanced Degree|5000|Unknown|1|0|0| +10491|M|M|Unknown|5000|Unknown|1|0|0| +10492|F|M|Unknown|5000|Unknown|1|0|0| +10493|M|S|Unknown|5000|Unknown|1|0|0| +10494|F|S|Unknown|5000|Unknown|1|0|0| +10495|M|D|Unknown|5000|Unknown|1|0|0| +10496|F|D|Unknown|5000|Unknown|1|0|0| +10497|M|W|Unknown|5000|Unknown|1|0|0| +10498|F|W|Unknown|5000|Unknown|1|0|0| +10499|M|U|Unknown|5000|Unknown|1|0|0| +10500|F|U|Unknown|5000|Unknown|1|0|0| +10501|M|M|Primary|5500|Unknown|1|0|0| +10502|F|M|Primary|5500|Unknown|1|0|0| +10503|M|S|Primary|5500|Unknown|1|0|0| +10504|F|S|Primary|5500|Unknown|1|0|0| +10505|M|D|Primary|5500|Unknown|1|0|0| +10506|F|D|Primary|5500|Unknown|1|0|0| +10507|M|W|Primary|5500|Unknown|1|0|0| +10508|F|W|Primary|5500|Unknown|1|0|0| +10509|M|U|Primary|5500|Unknown|1|0|0| +10510|F|U|Primary|5500|Unknown|1|0|0| +10511|M|M|Secondary|5500|Unknown|1|0|0| +10512|F|M|Secondary|5500|Unknown|1|0|0| +10513|M|S|Secondary|5500|Unknown|1|0|0| +10514|F|S|Secondary|5500|Unknown|1|0|0| +10515|M|D|Secondary|5500|Unknown|1|0|0| +10516|F|D|Secondary|5500|Unknown|1|0|0| +10517|M|W|Secondary|5500|Unknown|1|0|0| +10518|F|W|Secondary|5500|Unknown|1|0|0| +10519|M|U|Secondary|5500|Unknown|1|0|0| +10520|F|U|Secondary|5500|Unknown|1|0|0| +10521|M|M|College|5500|Unknown|1|0|0| +10522|F|M|College|5500|Unknown|1|0|0| +10523|M|S|College|5500|Unknown|1|0|0| +10524|F|S|College|5500|Unknown|1|0|0| +10525|M|D|College|5500|Unknown|1|0|0| +10526|F|D|College|5500|Unknown|1|0|0| +10527|M|W|College|5500|Unknown|1|0|0| +10528|F|W|College|5500|Unknown|1|0|0| +10529|M|U|College|5500|Unknown|1|0|0| +10530|F|U|College|5500|Unknown|1|0|0| +10531|M|M|2 yr Degree|5500|Unknown|1|0|0| +10532|F|M|2 yr Degree|5500|Unknown|1|0|0| +10533|M|S|2 yr Degree|5500|Unknown|1|0|0| +10534|F|S|2 yr Degree|5500|Unknown|1|0|0| +10535|M|D|2 yr Degree|5500|Unknown|1|0|0| +10536|F|D|2 yr Degree|5500|Unknown|1|0|0| +10537|M|W|2 yr Degree|5500|Unknown|1|0|0| +10538|F|W|2 yr Degree|5500|Unknown|1|0|0| +10539|M|U|2 yr Degree|5500|Unknown|1|0|0| +10540|F|U|2 yr Degree|5500|Unknown|1|0|0| +10541|M|M|4 yr Degree|5500|Unknown|1|0|0| +10542|F|M|4 yr Degree|5500|Unknown|1|0|0| +10543|M|S|4 yr Degree|5500|Unknown|1|0|0| +10544|F|S|4 yr Degree|5500|Unknown|1|0|0| +10545|M|D|4 yr Degree|5500|Unknown|1|0|0| +10546|F|D|4 yr Degree|5500|Unknown|1|0|0| +10547|M|W|4 yr Degree|5500|Unknown|1|0|0| +10548|F|W|4 yr Degree|5500|Unknown|1|0|0| +10549|M|U|4 yr Degree|5500|Unknown|1|0|0| +10550|F|U|4 yr Degree|5500|Unknown|1|0|0| +10551|M|M|Advanced Degree|5500|Unknown|1|0|0| +10552|F|M|Advanced Degree|5500|Unknown|1|0|0| +10553|M|S|Advanced Degree|5500|Unknown|1|0|0| +10554|F|S|Advanced Degree|5500|Unknown|1|0|0| +10555|M|D|Advanced Degree|5500|Unknown|1|0|0| +10556|F|D|Advanced Degree|5500|Unknown|1|0|0| +10557|M|W|Advanced Degree|5500|Unknown|1|0|0| +10558|F|W|Advanced Degree|5500|Unknown|1|0|0| +10559|M|U|Advanced Degree|5500|Unknown|1|0|0| +10560|F|U|Advanced Degree|5500|Unknown|1|0|0| +10561|M|M|Unknown|5500|Unknown|1|0|0| +10562|F|M|Unknown|5500|Unknown|1|0|0| +10563|M|S|Unknown|5500|Unknown|1|0|0| +10564|F|S|Unknown|5500|Unknown|1|0|0| +10565|M|D|Unknown|5500|Unknown|1|0|0| +10566|F|D|Unknown|5500|Unknown|1|0|0| +10567|M|W|Unknown|5500|Unknown|1|0|0| +10568|F|W|Unknown|5500|Unknown|1|0|0| +10569|M|U|Unknown|5500|Unknown|1|0|0| +10570|F|U|Unknown|5500|Unknown|1|0|0| +10571|M|M|Primary|6000|Unknown|1|0|0| +10572|F|M|Primary|6000|Unknown|1|0|0| +10573|M|S|Primary|6000|Unknown|1|0|0| +10574|F|S|Primary|6000|Unknown|1|0|0| +10575|M|D|Primary|6000|Unknown|1|0|0| +10576|F|D|Primary|6000|Unknown|1|0|0| +10577|M|W|Primary|6000|Unknown|1|0|0| +10578|F|W|Primary|6000|Unknown|1|0|0| +10579|M|U|Primary|6000|Unknown|1|0|0| +10580|F|U|Primary|6000|Unknown|1|0|0| +10581|M|M|Secondary|6000|Unknown|1|0|0| +10582|F|M|Secondary|6000|Unknown|1|0|0| +10583|M|S|Secondary|6000|Unknown|1|0|0| +10584|F|S|Secondary|6000|Unknown|1|0|0| +10585|M|D|Secondary|6000|Unknown|1|0|0| +10586|F|D|Secondary|6000|Unknown|1|0|0| +10587|M|W|Secondary|6000|Unknown|1|0|0| +10588|F|W|Secondary|6000|Unknown|1|0|0| +10589|M|U|Secondary|6000|Unknown|1|0|0| +10590|F|U|Secondary|6000|Unknown|1|0|0| +10591|M|M|College|6000|Unknown|1|0|0| +10592|F|M|College|6000|Unknown|1|0|0| +10593|M|S|College|6000|Unknown|1|0|0| +10594|F|S|College|6000|Unknown|1|0|0| +10595|M|D|College|6000|Unknown|1|0|0| +10596|F|D|College|6000|Unknown|1|0|0| +10597|M|W|College|6000|Unknown|1|0|0| +10598|F|W|College|6000|Unknown|1|0|0| +10599|M|U|College|6000|Unknown|1|0|0| +10600|F|U|College|6000|Unknown|1|0|0| +10601|M|M|2 yr Degree|6000|Unknown|1|0|0| +10602|F|M|2 yr Degree|6000|Unknown|1|0|0| +10603|M|S|2 yr Degree|6000|Unknown|1|0|0| +10604|F|S|2 yr Degree|6000|Unknown|1|0|0| +10605|M|D|2 yr Degree|6000|Unknown|1|0|0| +10606|F|D|2 yr Degree|6000|Unknown|1|0|0| +10607|M|W|2 yr Degree|6000|Unknown|1|0|0| +10608|F|W|2 yr Degree|6000|Unknown|1|0|0| +10609|M|U|2 yr Degree|6000|Unknown|1|0|0| +10610|F|U|2 yr Degree|6000|Unknown|1|0|0| +10611|M|M|4 yr Degree|6000|Unknown|1|0|0| +10612|F|M|4 yr Degree|6000|Unknown|1|0|0| +10613|M|S|4 yr Degree|6000|Unknown|1|0|0| +10614|F|S|4 yr Degree|6000|Unknown|1|0|0| +10615|M|D|4 yr Degree|6000|Unknown|1|0|0| +10616|F|D|4 yr Degree|6000|Unknown|1|0|0| +10617|M|W|4 yr Degree|6000|Unknown|1|0|0| +10618|F|W|4 yr Degree|6000|Unknown|1|0|0| +10619|M|U|4 yr Degree|6000|Unknown|1|0|0| +10620|F|U|4 yr Degree|6000|Unknown|1|0|0| +10621|M|M|Advanced Degree|6000|Unknown|1|0|0| +10622|F|M|Advanced Degree|6000|Unknown|1|0|0| +10623|M|S|Advanced Degree|6000|Unknown|1|0|0| +10624|F|S|Advanced Degree|6000|Unknown|1|0|0| +10625|M|D|Advanced Degree|6000|Unknown|1|0|0| +10626|F|D|Advanced Degree|6000|Unknown|1|0|0| +10627|M|W|Advanced Degree|6000|Unknown|1|0|0| +10628|F|W|Advanced Degree|6000|Unknown|1|0|0| +10629|M|U|Advanced Degree|6000|Unknown|1|0|0| +10630|F|U|Advanced Degree|6000|Unknown|1|0|0| +10631|M|M|Unknown|6000|Unknown|1|0|0| +10632|F|M|Unknown|6000|Unknown|1|0|0| +10633|M|S|Unknown|6000|Unknown|1|0|0| +10634|F|S|Unknown|6000|Unknown|1|0|0| +10635|M|D|Unknown|6000|Unknown|1|0|0| +10636|F|D|Unknown|6000|Unknown|1|0|0| +10637|M|W|Unknown|6000|Unknown|1|0|0| +10638|F|W|Unknown|6000|Unknown|1|0|0| +10639|M|U|Unknown|6000|Unknown|1|0|0| +10640|F|U|Unknown|6000|Unknown|1|0|0| +10641|M|M|Primary|6500|Unknown|1|0|0| +10642|F|M|Primary|6500|Unknown|1|0|0| +10643|M|S|Primary|6500|Unknown|1|0|0| +10644|F|S|Primary|6500|Unknown|1|0|0| +10645|M|D|Primary|6500|Unknown|1|0|0| +10646|F|D|Primary|6500|Unknown|1|0|0| +10647|M|W|Primary|6500|Unknown|1|0|0| +10648|F|W|Primary|6500|Unknown|1|0|0| +10649|M|U|Primary|6500|Unknown|1|0|0| +10650|F|U|Primary|6500|Unknown|1|0|0| +10651|M|M|Secondary|6500|Unknown|1|0|0| +10652|F|M|Secondary|6500|Unknown|1|0|0| +10653|M|S|Secondary|6500|Unknown|1|0|0| +10654|F|S|Secondary|6500|Unknown|1|0|0| +10655|M|D|Secondary|6500|Unknown|1|0|0| +10656|F|D|Secondary|6500|Unknown|1|0|0| +10657|M|W|Secondary|6500|Unknown|1|0|0| +10658|F|W|Secondary|6500|Unknown|1|0|0| +10659|M|U|Secondary|6500|Unknown|1|0|0| +10660|F|U|Secondary|6500|Unknown|1|0|0| +10661|M|M|College|6500|Unknown|1|0|0| +10662|F|M|College|6500|Unknown|1|0|0| +10663|M|S|College|6500|Unknown|1|0|0| +10664|F|S|College|6500|Unknown|1|0|0| +10665|M|D|College|6500|Unknown|1|0|0| +10666|F|D|College|6500|Unknown|1|0|0| +10667|M|W|College|6500|Unknown|1|0|0| +10668|F|W|College|6500|Unknown|1|0|0| +10669|M|U|College|6500|Unknown|1|0|0| +10670|F|U|College|6500|Unknown|1|0|0| +10671|M|M|2 yr Degree|6500|Unknown|1|0|0| +10672|F|M|2 yr Degree|6500|Unknown|1|0|0| +10673|M|S|2 yr Degree|6500|Unknown|1|0|0| +10674|F|S|2 yr Degree|6500|Unknown|1|0|0| +10675|M|D|2 yr Degree|6500|Unknown|1|0|0| +10676|F|D|2 yr Degree|6500|Unknown|1|0|0| +10677|M|W|2 yr Degree|6500|Unknown|1|0|0| +10678|F|W|2 yr Degree|6500|Unknown|1|0|0| +10679|M|U|2 yr Degree|6500|Unknown|1|0|0| +10680|F|U|2 yr Degree|6500|Unknown|1|0|0| +10681|M|M|4 yr Degree|6500|Unknown|1|0|0| +10682|F|M|4 yr Degree|6500|Unknown|1|0|0| +10683|M|S|4 yr Degree|6500|Unknown|1|0|0| +10684|F|S|4 yr Degree|6500|Unknown|1|0|0| +10685|M|D|4 yr Degree|6500|Unknown|1|0|0| +10686|F|D|4 yr Degree|6500|Unknown|1|0|0| +10687|M|W|4 yr Degree|6500|Unknown|1|0|0| +10688|F|W|4 yr Degree|6500|Unknown|1|0|0| +10689|M|U|4 yr Degree|6500|Unknown|1|0|0| +10690|F|U|4 yr Degree|6500|Unknown|1|0|0| +10691|M|M|Advanced Degree|6500|Unknown|1|0|0| +10692|F|M|Advanced Degree|6500|Unknown|1|0|0| +10693|M|S|Advanced Degree|6500|Unknown|1|0|0| +10694|F|S|Advanced Degree|6500|Unknown|1|0|0| +10695|M|D|Advanced Degree|6500|Unknown|1|0|0| +10696|F|D|Advanced Degree|6500|Unknown|1|0|0| +10697|M|W|Advanced Degree|6500|Unknown|1|0|0| +10698|F|W|Advanced Degree|6500|Unknown|1|0|0| +10699|M|U|Advanced Degree|6500|Unknown|1|0|0| +10700|F|U|Advanced Degree|6500|Unknown|1|0|0| +10701|M|M|Unknown|6500|Unknown|1|0|0| +10702|F|M|Unknown|6500|Unknown|1|0|0| +10703|M|S|Unknown|6500|Unknown|1|0|0| +10704|F|S|Unknown|6500|Unknown|1|0|0| +10705|M|D|Unknown|6500|Unknown|1|0|0| +10706|F|D|Unknown|6500|Unknown|1|0|0| +10707|M|W|Unknown|6500|Unknown|1|0|0| +10708|F|W|Unknown|6500|Unknown|1|0|0| +10709|M|U|Unknown|6500|Unknown|1|0|0| +10710|F|U|Unknown|6500|Unknown|1|0|0| +10711|M|M|Primary|7000|Unknown|1|0|0| +10712|F|M|Primary|7000|Unknown|1|0|0| +10713|M|S|Primary|7000|Unknown|1|0|0| +10714|F|S|Primary|7000|Unknown|1|0|0| +10715|M|D|Primary|7000|Unknown|1|0|0| +10716|F|D|Primary|7000|Unknown|1|0|0| +10717|M|W|Primary|7000|Unknown|1|0|0| +10718|F|W|Primary|7000|Unknown|1|0|0| +10719|M|U|Primary|7000|Unknown|1|0|0| +10720|F|U|Primary|7000|Unknown|1|0|0| +10721|M|M|Secondary|7000|Unknown|1|0|0| +10722|F|M|Secondary|7000|Unknown|1|0|0| +10723|M|S|Secondary|7000|Unknown|1|0|0| +10724|F|S|Secondary|7000|Unknown|1|0|0| +10725|M|D|Secondary|7000|Unknown|1|0|0| +10726|F|D|Secondary|7000|Unknown|1|0|0| +10727|M|W|Secondary|7000|Unknown|1|0|0| +10728|F|W|Secondary|7000|Unknown|1|0|0| +10729|M|U|Secondary|7000|Unknown|1|0|0| +10730|F|U|Secondary|7000|Unknown|1|0|0| +10731|M|M|College|7000|Unknown|1|0|0| +10732|F|M|College|7000|Unknown|1|0|0| +10733|M|S|College|7000|Unknown|1|0|0| +10734|F|S|College|7000|Unknown|1|0|0| +10735|M|D|College|7000|Unknown|1|0|0| +10736|F|D|College|7000|Unknown|1|0|0| +10737|M|W|College|7000|Unknown|1|0|0| +10738|F|W|College|7000|Unknown|1|0|0| +10739|M|U|College|7000|Unknown|1|0|0| +10740|F|U|College|7000|Unknown|1|0|0| +10741|M|M|2 yr Degree|7000|Unknown|1|0|0| +10742|F|M|2 yr Degree|7000|Unknown|1|0|0| +10743|M|S|2 yr Degree|7000|Unknown|1|0|0| +10744|F|S|2 yr Degree|7000|Unknown|1|0|0| +10745|M|D|2 yr Degree|7000|Unknown|1|0|0| +10746|F|D|2 yr Degree|7000|Unknown|1|0|0| +10747|M|W|2 yr Degree|7000|Unknown|1|0|0| +10748|F|W|2 yr Degree|7000|Unknown|1|0|0| +10749|M|U|2 yr Degree|7000|Unknown|1|0|0| +10750|F|U|2 yr Degree|7000|Unknown|1|0|0| +10751|M|M|4 yr Degree|7000|Unknown|1|0|0| +10752|F|M|4 yr Degree|7000|Unknown|1|0|0| +10753|M|S|4 yr Degree|7000|Unknown|1|0|0| +10754|F|S|4 yr Degree|7000|Unknown|1|0|0| +10755|M|D|4 yr Degree|7000|Unknown|1|0|0| +10756|F|D|4 yr Degree|7000|Unknown|1|0|0| +10757|M|W|4 yr Degree|7000|Unknown|1|0|0| +10758|F|W|4 yr Degree|7000|Unknown|1|0|0| +10759|M|U|4 yr Degree|7000|Unknown|1|0|0| +10760|F|U|4 yr Degree|7000|Unknown|1|0|0| +10761|M|M|Advanced Degree|7000|Unknown|1|0|0| +10762|F|M|Advanced Degree|7000|Unknown|1|0|0| +10763|M|S|Advanced Degree|7000|Unknown|1|0|0| +10764|F|S|Advanced Degree|7000|Unknown|1|0|0| +10765|M|D|Advanced Degree|7000|Unknown|1|0|0| +10766|F|D|Advanced Degree|7000|Unknown|1|0|0| +10767|M|W|Advanced Degree|7000|Unknown|1|0|0| +10768|F|W|Advanced Degree|7000|Unknown|1|0|0| +10769|M|U|Advanced Degree|7000|Unknown|1|0|0| +10770|F|U|Advanced Degree|7000|Unknown|1|0|0| +10771|M|M|Unknown|7000|Unknown|1|0|0| +10772|F|M|Unknown|7000|Unknown|1|0|0| +10773|M|S|Unknown|7000|Unknown|1|0|0| +10774|F|S|Unknown|7000|Unknown|1|0|0| +10775|M|D|Unknown|7000|Unknown|1|0|0| +10776|F|D|Unknown|7000|Unknown|1|0|0| +10777|M|W|Unknown|7000|Unknown|1|0|0| +10778|F|W|Unknown|7000|Unknown|1|0|0| +10779|M|U|Unknown|7000|Unknown|1|0|0| +10780|F|U|Unknown|7000|Unknown|1|0|0| +10781|M|M|Primary|7500|Unknown|1|0|0| +10782|F|M|Primary|7500|Unknown|1|0|0| +10783|M|S|Primary|7500|Unknown|1|0|0| +10784|F|S|Primary|7500|Unknown|1|0|0| +10785|M|D|Primary|7500|Unknown|1|0|0| +10786|F|D|Primary|7500|Unknown|1|0|0| +10787|M|W|Primary|7500|Unknown|1|0|0| +10788|F|W|Primary|7500|Unknown|1|0|0| +10789|M|U|Primary|7500|Unknown|1|0|0| +10790|F|U|Primary|7500|Unknown|1|0|0| +10791|M|M|Secondary|7500|Unknown|1|0|0| +10792|F|M|Secondary|7500|Unknown|1|0|0| +10793|M|S|Secondary|7500|Unknown|1|0|0| +10794|F|S|Secondary|7500|Unknown|1|0|0| +10795|M|D|Secondary|7500|Unknown|1|0|0| +10796|F|D|Secondary|7500|Unknown|1|0|0| +10797|M|W|Secondary|7500|Unknown|1|0|0| +10798|F|W|Secondary|7500|Unknown|1|0|0| +10799|M|U|Secondary|7500|Unknown|1|0|0| +10800|F|U|Secondary|7500|Unknown|1|0|0| +10801|M|M|College|7500|Unknown|1|0|0| +10802|F|M|College|7500|Unknown|1|0|0| +10803|M|S|College|7500|Unknown|1|0|0| +10804|F|S|College|7500|Unknown|1|0|0| +10805|M|D|College|7500|Unknown|1|0|0| +10806|F|D|College|7500|Unknown|1|0|0| +10807|M|W|College|7500|Unknown|1|0|0| +10808|F|W|College|7500|Unknown|1|0|0| +10809|M|U|College|7500|Unknown|1|0|0| +10810|F|U|College|7500|Unknown|1|0|0| +10811|M|M|2 yr Degree|7500|Unknown|1|0|0| +10812|F|M|2 yr Degree|7500|Unknown|1|0|0| +10813|M|S|2 yr Degree|7500|Unknown|1|0|0| +10814|F|S|2 yr Degree|7500|Unknown|1|0|0| +10815|M|D|2 yr Degree|7500|Unknown|1|0|0| +10816|F|D|2 yr Degree|7500|Unknown|1|0|0| +10817|M|W|2 yr Degree|7500|Unknown|1|0|0| +10818|F|W|2 yr Degree|7500|Unknown|1|0|0| +10819|M|U|2 yr Degree|7500|Unknown|1|0|0| +10820|F|U|2 yr Degree|7500|Unknown|1|0|0| +10821|M|M|4 yr Degree|7500|Unknown|1|0|0| +10822|F|M|4 yr Degree|7500|Unknown|1|0|0| +10823|M|S|4 yr Degree|7500|Unknown|1|0|0| +10824|F|S|4 yr Degree|7500|Unknown|1|0|0| +10825|M|D|4 yr Degree|7500|Unknown|1|0|0| +10826|F|D|4 yr Degree|7500|Unknown|1|0|0| +10827|M|W|4 yr Degree|7500|Unknown|1|0|0| +10828|F|W|4 yr Degree|7500|Unknown|1|0|0| +10829|M|U|4 yr Degree|7500|Unknown|1|0|0| +10830|F|U|4 yr Degree|7500|Unknown|1|0|0| +10831|M|M|Advanced Degree|7500|Unknown|1|0|0| +10832|F|M|Advanced Degree|7500|Unknown|1|0|0| +10833|M|S|Advanced Degree|7500|Unknown|1|0|0| +10834|F|S|Advanced Degree|7500|Unknown|1|0|0| +10835|M|D|Advanced Degree|7500|Unknown|1|0|0| +10836|F|D|Advanced Degree|7500|Unknown|1|0|0| +10837|M|W|Advanced Degree|7500|Unknown|1|0|0| +10838|F|W|Advanced Degree|7500|Unknown|1|0|0| +10839|M|U|Advanced Degree|7500|Unknown|1|0|0| +10840|F|U|Advanced Degree|7500|Unknown|1|0|0| +10841|M|M|Unknown|7500|Unknown|1|0|0| +10842|F|M|Unknown|7500|Unknown|1|0|0| +10843|M|S|Unknown|7500|Unknown|1|0|0| +10844|F|S|Unknown|7500|Unknown|1|0|0| +10845|M|D|Unknown|7500|Unknown|1|0|0| +10846|F|D|Unknown|7500|Unknown|1|0|0| +10847|M|W|Unknown|7500|Unknown|1|0|0| +10848|F|W|Unknown|7500|Unknown|1|0|0| +10849|M|U|Unknown|7500|Unknown|1|0|0| +10850|F|U|Unknown|7500|Unknown|1|0|0| +10851|M|M|Primary|8000|Unknown|1|0|0| +10852|F|M|Primary|8000|Unknown|1|0|0| +10853|M|S|Primary|8000|Unknown|1|0|0| +10854|F|S|Primary|8000|Unknown|1|0|0| +10855|M|D|Primary|8000|Unknown|1|0|0| +10856|F|D|Primary|8000|Unknown|1|0|0| +10857|M|W|Primary|8000|Unknown|1|0|0| +10858|F|W|Primary|8000|Unknown|1|0|0| +10859|M|U|Primary|8000|Unknown|1|0|0| +10860|F|U|Primary|8000|Unknown|1|0|0| +10861|M|M|Secondary|8000|Unknown|1|0|0| +10862|F|M|Secondary|8000|Unknown|1|0|0| +10863|M|S|Secondary|8000|Unknown|1|0|0| +10864|F|S|Secondary|8000|Unknown|1|0|0| +10865|M|D|Secondary|8000|Unknown|1|0|0| +10866|F|D|Secondary|8000|Unknown|1|0|0| +10867|M|W|Secondary|8000|Unknown|1|0|0| +10868|F|W|Secondary|8000|Unknown|1|0|0| +10869|M|U|Secondary|8000|Unknown|1|0|0| +10870|F|U|Secondary|8000|Unknown|1|0|0| +10871|M|M|College|8000|Unknown|1|0|0| +10872|F|M|College|8000|Unknown|1|0|0| +10873|M|S|College|8000|Unknown|1|0|0| +10874|F|S|College|8000|Unknown|1|0|0| +10875|M|D|College|8000|Unknown|1|0|0| +10876|F|D|College|8000|Unknown|1|0|0| +10877|M|W|College|8000|Unknown|1|0|0| +10878|F|W|College|8000|Unknown|1|0|0| +10879|M|U|College|8000|Unknown|1|0|0| +10880|F|U|College|8000|Unknown|1|0|0| +10881|M|M|2 yr Degree|8000|Unknown|1|0|0| +10882|F|M|2 yr Degree|8000|Unknown|1|0|0| +10883|M|S|2 yr Degree|8000|Unknown|1|0|0| +10884|F|S|2 yr Degree|8000|Unknown|1|0|0| +10885|M|D|2 yr Degree|8000|Unknown|1|0|0| +10886|F|D|2 yr Degree|8000|Unknown|1|0|0| +10887|M|W|2 yr Degree|8000|Unknown|1|0|0| +10888|F|W|2 yr Degree|8000|Unknown|1|0|0| +10889|M|U|2 yr Degree|8000|Unknown|1|0|0| +10890|F|U|2 yr Degree|8000|Unknown|1|0|0| +10891|M|M|4 yr Degree|8000|Unknown|1|0|0| +10892|F|M|4 yr Degree|8000|Unknown|1|0|0| +10893|M|S|4 yr Degree|8000|Unknown|1|0|0| +10894|F|S|4 yr Degree|8000|Unknown|1|0|0| +10895|M|D|4 yr Degree|8000|Unknown|1|0|0| +10896|F|D|4 yr Degree|8000|Unknown|1|0|0| +10897|M|W|4 yr Degree|8000|Unknown|1|0|0| +10898|F|W|4 yr Degree|8000|Unknown|1|0|0| +10899|M|U|4 yr Degree|8000|Unknown|1|0|0| +10900|F|U|4 yr Degree|8000|Unknown|1|0|0| +10901|M|M|Advanced Degree|8000|Unknown|1|0|0| +10902|F|M|Advanced Degree|8000|Unknown|1|0|0| +10903|M|S|Advanced Degree|8000|Unknown|1|0|0| +10904|F|S|Advanced Degree|8000|Unknown|1|0|0| +10905|M|D|Advanced Degree|8000|Unknown|1|0|0| +10906|F|D|Advanced Degree|8000|Unknown|1|0|0| +10907|M|W|Advanced Degree|8000|Unknown|1|0|0| +10908|F|W|Advanced Degree|8000|Unknown|1|0|0| +10909|M|U|Advanced Degree|8000|Unknown|1|0|0| +10910|F|U|Advanced Degree|8000|Unknown|1|0|0| +10911|M|M|Unknown|8000|Unknown|1|0|0| +10912|F|M|Unknown|8000|Unknown|1|0|0| +10913|M|S|Unknown|8000|Unknown|1|0|0| +10914|F|S|Unknown|8000|Unknown|1|0|0| +10915|M|D|Unknown|8000|Unknown|1|0|0| +10916|F|D|Unknown|8000|Unknown|1|0|0| +10917|M|W|Unknown|8000|Unknown|1|0|0| +10918|F|W|Unknown|8000|Unknown|1|0|0| +10919|M|U|Unknown|8000|Unknown|1|0|0| +10920|F|U|Unknown|8000|Unknown|1|0|0| +10921|M|M|Primary|8500|Unknown|1|0|0| +10922|F|M|Primary|8500|Unknown|1|0|0| +10923|M|S|Primary|8500|Unknown|1|0|0| +10924|F|S|Primary|8500|Unknown|1|0|0| +10925|M|D|Primary|8500|Unknown|1|0|0| +10926|F|D|Primary|8500|Unknown|1|0|0| +10927|M|W|Primary|8500|Unknown|1|0|0| +10928|F|W|Primary|8500|Unknown|1|0|0| +10929|M|U|Primary|8500|Unknown|1|0|0| +10930|F|U|Primary|8500|Unknown|1|0|0| +10931|M|M|Secondary|8500|Unknown|1|0|0| +10932|F|M|Secondary|8500|Unknown|1|0|0| +10933|M|S|Secondary|8500|Unknown|1|0|0| +10934|F|S|Secondary|8500|Unknown|1|0|0| +10935|M|D|Secondary|8500|Unknown|1|0|0| +10936|F|D|Secondary|8500|Unknown|1|0|0| +10937|M|W|Secondary|8500|Unknown|1|0|0| +10938|F|W|Secondary|8500|Unknown|1|0|0| +10939|M|U|Secondary|8500|Unknown|1|0|0| +10940|F|U|Secondary|8500|Unknown|1|0|0| +10941|M|M|College|8500|Unknown|1|0|0| +10942|F|M|College|8500|Unknown|1|0|0| +10943|M|S|College|8500|Unknown|1|0|0| +10944|F|S|College|8500|Unknown|1|0|0| +10945|M|D|College|8500|Unknown|1|0|0| +10946|F|D|College|8500|Unknown|1|0|0| +10947|M|W|College|8500|Unknown|1|0|0| +10948|F|W|College|8500|Unknown|1|0|0| +10949|M|U|College|8500|Unknown|1|0|0| +10950|F|U|College|8500|Unknown|1|0|0| +10951|M|M|2 yr Degree|8500|Unknown|1|0|0| +10952|F|M|2 yr Degree|8500|Unknown|1|0|0| +10953|M|S|2 yr Degree|8500|Unknown|1|0|0| +10954|F|S|2 yr Degree|8500|Unknown|1|0|0| +10955|M|D|2 yr Degree|8500|Unknown|1|0|0| +10956|F|D|2 yr Degree|8500|Unknown|1|0|0| +10957|M|W|2 yr Degree|8500|Unknown|1|0|0| +10958|F|W|2 yr Degree|8500|Unknown|1|0|0| +10959|M|U|2 yr Degree|8500|Unknown|1|0|0| +10960|F|U|2 yr Degree|8500|Unknown|1|0|0| +10961|M|M|4 yr Degree|8500|Unknown|1|0|0| +10962|F|M|4 yr Degree|8500|Unknown|1|0|0| +10963|M|S|4 yr Degree|8500|Unknown|1|0|0| +10964|F|S|4 yr Degree|8500|Unknown|1|0|0| +10965|M|D|4 yr Degree|8500|Unknown|1|0|0| +10966|F|D|4 yr Degree|8500|Unknown|1|0|0| +10967|M|W|4 yr Degree|8500|Unknown|1|0|0| +10968|F|W|4 yr Degree|8500|Unknown|1|0|0| +10969|M|U|4 yr Degree|8500|Unknown|1|0|0| +10970|F|U|4 yr Degree|8500|Unknown|1|0|0| +10971|M|M|Advanced Degree|8500|Unknown|1|0|0| +10972|F|M|Advanced Degree|8500|Unknown|1|0|0| +10973|M|S|Advanced Degree|8500|Unknown|1|0|0| +10974|F|S|Advanced Degree|8500|Unknown|1|0|0| +10975|M|D|Advanced Degree|8500|Unknown|1|0|0| +10976|F|D|Advanced Degree|8500|Unknown|1|0|0| +10977|M|W|Advanced Degree|8500|Unknown|1|0|0| +10978|F|W|Advanced Degree|8500|Unknown|1|0|0| +10979|M|U|Advanced Degree|8500|Unknown|1|0|0| +10980|F|U|Advanced Degree|8500|Unknown|1|0|0| +10981|M|M|Unknown|8500|Unknown|1|0|0| +10982|F|M|Unknown|8500|Unknown|1|0|0| +10983|M|S|Unknown|8500|Unknown|1|0|0| +10984|F|S|Unknown|8500|Unknown|1|0|0| +10985|M|D|Unknown|8500|Unknown|1|0|0| +10986|F|D|Unknown|8500|Unknown|1|0|0| +10987|M|W|Unknown|8500|Unknown|1|0|0| +10988|F|W|Unknown|8500|Unknown|1|0|0| +10989|M|U|Unknown|8500|Unknown|1|0|0| +10990|F|U|Unknown|8500|Unknown|1|0|0| +10991|M|M|Primary|9000|Unknown|1|0|0| +10992|F|M|Primary|9000|Unknown|1|0|0| +10993|M|S|Primary|9000|Unknown|1|0|0| +10994|F|S|Primary|9000|Unknown|1|0|0| +10995|M|D|Primary|9000|Unknown|1|0|0| +10996|F|D|Primary|9000|Unknown|1|0|0| +10997|M|W|Primary|9000|Unknown|1|0|0| +10998|F|W|Primary|9000|Unknown|1|0|0| +10999|M|U|Primary|9000|Unknown|1|0|0| +11000|F|U|Primary|9000|Unknown|1|0|0| +11001|M|M|Secondary|9000|Unknown|1|0|0| +11002|F|M|Secondary|9000|Unknown|1|0|0| +11003|M|S|Secondary|9000|Unknown|1|0|0| +11004|F|S|Secondary|9000|Unknown|1|0|0| +11005|M|D|Secondary|9000|Unknown|1|0|0| +11006|F|D|Secondary|9000|Unknown|1|0|0| +11007|M|W|Secondary|9000|Unknown|1|0|0| +11008|F|W|Secondary|9000|Unknown|1|0|0| +11009|M|U|Secondary|9000|Unknown|1|0|0| +11010|F|U|Secondary|9000|Unknown|1|0|0| +11011|M|M|College|9000|Unknown|1|0|0| +11012|F|M|College|9000|Unknown|1|0|0| +11013|M|S|College|9000|Unknown|1|0|0| +11014|F|S|College|9000|Unknown|1|0|0| +11015|M|D|College|9000|Unknown|1|0|0| +11016|F|D|College|9000|Unknown|1|0|0| +11017|M|W|College|9000|Unknown|1|0|0| +11018|F|W|College|9000|Unknown|1|0|0| +11019|M|U|College|9000|Unknown|1|0|0| +11020|F|U|College|9000|Unknown|1|0|0| +11021|M|M|2 yr Degree|9000|Unknown|1|0|0| +11022|F|M|2 yr Degree|9000|Unknown|1|0|0| +11023|M|S|2 yr Degree|9000|Unknown|1|0|0| +11024|F|S|2 yr Degree|9000|Unknown|1|0|0| +11025|M|D|2 yr Degree|9000|Unknown|1|0|0| +11026|F|D|2 yr Degree|9000|Unknown|1|0|0| +11027|M|W|2 yr Degree|9000|Unknown|1|0|0| +11028|F|W|2 yr Degree|9000|Unknown|1|0|0| +11029|M|U|2 yr Degree|9000|Unknown|1|0|0| +11030|F|U|2 yr Degree|9000|Unknown|1|0|0| +11031|M|M|4 yr Degree|9000|Unknown|1|0|0| +11032|F|M|4 yr Degree|9000|Unknown|1|0|0| +11033|M|S|4 yr Degree|9000|Unknown|1|0|0| +11034|F|S|4 yr Degree|9000|Unknown|1|0|0| +11035|M|D|4 yr Degree|9000|Unknown|1|0|0| +11036|F|D|4 yr Degree|9000|Unknown|1|0|0| +11037|M|W|4 yr Degree|9000|Unknown|1|0|0| +11038|F|W|4 yr Degree|9000|Unknown|1|0|0| +11039|M|U|4 yr Degree|9000|Unknown|1|0|0| +11040|F|U|4 yr Degree|9000|Unknown|1|0|0| +11041|M|M|Advanced Degree|9000|Unknown|1|0|0| +11042|F|M|Advanced Degree|9000|Unknown|1|0|0| +11043|M|S|Advanced Degree|9000|Unknown|1|0|0| +11044|F|S|Advanced Degree|9000|Unknown|1|0|0| +11045|M|D|Advanced Degree|9000|Unknown|1|0|0| +11046|F|D|Advanced Degree|9000|Unknown|1|0|0| +11047|M|W|Advanced Degree|9000|Unknown|1|0|0| +11048|F|W|Advanced Degree|9000|Unknown|1|0|0| +11049|M|U|Advanced Degree|9000|Unknown|1|0|0| +11050|F|U|Advanced Degree|9000|Unknown|1|0|0| +11051|M|M|Unknown|9000|Unknown|1|0|0| +11052|F|M|Unknown|9000|Unknown|1|0|0| +11053|M|S|Unknown|9000|Unknown|1|0|0| +11054|F|S|Unknown|9000|Unknown|1|0|0| +11055|M|D|Unknown|9000|Unknown|1|0|0| +11056|F|D|Unknown|9000|Unknown|1|0|0| +11057|M|W|Unknown|9000|Unknown|1|0|0| +11058|F|W|Unknown|9000|Unknown|1|0|0| +11059|M|U|Unknown|9000|Unknown|1|0|0| +11060|F|U|Unknown|9000|Unknown|1|0|0| +11061|M|M|Primary|9500|Unknown|1|0|0| +11062|F|M|Primary|9500|Unknown|1|0|0| +11063|M|S|Primary|9500|Unknown|1|0|0| +11064|F|S|Primary|9500|Unknown|1|0|0| +11065|M|D|Primary|9500|Unknown|1|0|0| +11066|F|D|Primary|9500|Unknown|1|0|0| +11067|M|W|Primary|9500|Unknown|1|0|0| +11068|F|W|Primary|9500|Unknown|1|0|0| +11069|M|U|Primary|9500|Unknown|1|0|0| +11070|F|U|Primary|9500|Unknown|1|0|0| +11071|M|M|Secondary|9500|Unknown|1|0|0| +11072|F|M|Secondary|9500|Unknown|1|0|0| +11073|M|S|Secondary|9500|Unknown|1|0|0| +11074|F|S|Secondary|9500|Unknown|1|0|0| +11075|M|D|Secondary|9500|Unknown|1|0|0| +11076|F|D|Secondary|9500|Unknown|1|0|0| +11077|M|W|Secondary|9500|Unknown|1|0|0| +11078|F|W|Secondary|9500|Unknown|1|0|0| +11079|M|U|Secondary|9500|Unknown|1|0|0| +11080|F|U|Secondary|9500|Unknown|1|0|0| +11081|M|M|College|9500|Unknown|1|0|0| +11082|F|M|College|9500|Unknown|1|0|0| +11083|M|S|College|9500|Unknown|1|0|0| +11084|F|S|College|9500|Unknown|1|0|0| +11085|M|D|College|9500|Unknown|1|0|0| +11086|F|D|College|9500|Unknown|1|0|0| +11087|M|W|College|9500|Unknown|1|0|0| +11088|F|W|College|9500|Unknown|1|0|0| +11089|M|U|College|9500|Unknown|1|0|0| +11090|F|U|College|9500|Unknown|1|0|0| +11091|M|M|2 yr Degree|9500|Unknown|1|0|0| +11092|F|M|2 yr Degree|9500|Unknown|1|0|0| +11093|M|S|2 yr Degree|9500|Unknown|1|0|0| +11094|F|S|2 yr Degree|9500|Unknown|1|0|0| +11095|M|D|2 yr Degree|9500|Unknown|1|0|0| +11096|F|D|2 yr Degree|9500|Unknown|1|0|0| +11097|M|W|2 yr Degree|9500|Unknown|1|0|0| +11098|F|W|2 yr Degree|9500|Unknown|1|0|0| +11099|M|U|2 yr Degree|9500|Unknown|1|0|0| +11100|F|U|2 yr Degree|9500|Unknown|1|0|0| +11101|M|M|4 yr Degree|9500|Unknown|1|0|0| +11102|F|M|4 yr Degree|9500|Unknown|1|0|0| +11103|M|S|4 yr Degree|9500|Unknown|1|0|0| +11104|F|S|4 yr Degree|9500|Unknown|1|0|0| +11105|M|D|4 yr Degree|9500|Unknown|1|0|0| +11106|F|D|4 yr Degree|9500|Unknown|1|0|0| +11107|M|W|4 yr Degree|9500|Unknown|1|0|0| +11108|F|W|4 yr Degree|9500|Unknown|1|0|0| +11109|M|U|4 yr Degree|9500|Unknown|1|0|0| +11110|F|U|4 yr Degree|9500|Unknown|1|0|0| +11111|M|M|Advanced Degree|9500|Unknown|1|0|0| +11112|F|M|Advanced Degree|9500|Unknown|1|0|0| +11113|M|S|Advanced Degree|9500|Unknown|1|0|0| +11114|F|S|Advanced Degree|9500|Unknown|1|0|0| +11115|M|D|Advanced Degree|9500|Unknown|1|0|0| +11116|F|D|Advanced Degree|9500|Unknown|1|0|0| +11117|M|W|Advanced Degree|9500|Unknown|1|0|0| +11118|F|W|Advanced Degree|9500|Unknown|1|0|0| +11119|M|U|Advanced Degree|9500|Unknown|1|0|0| +11120|F|U|Advanced Degree|9500|Unknown|1|0|0| +11121|M|M|Unknown|9500|Unknown|1|0|0| +11122|F|M|Unknown|9500|Unknown|1|0|0| +11123|M|S|Unknown|9500|Unknown|1|0|0| +11124|F|S|Unknown|9500|Unknown|1|0|0| +11125|M|D|Unknown|9500|Unknown|1|0|0| +11126|F|D|Unknown|9500|Unknown|1|0|0| +11127|M|W|Unknown|9500|Unknown|1|0|0| +11128|F|W|Unknown|9500|Unknown|1|0|0| +11129|M|U|Unknown|9500|Unknown|1|0|0| +11130|F|U|Unknown|9500|Unknown|1|0|0| +11131|M|M|Primary|10000|Unknown|1|0|0| +11132|F|M|Primary|10000|Unknown|1|0|0| +11133|M|S|Primary|10000|Unknown|1|0|0| +11134|F|S|Primary|10000|Unknown|1|0|0| +11135|M|D|Primary|10000|Unknown|1|0|0| +11136|F|D|Primary|10000|Unknown|1|0|0| +11137|M|W|Primary|10000|Unknown|1|0|0| +11138|F|W|Primary|10000|Unknown|1|0|0| +11139|M|U|Primary|10000|Unknown|1|0|0| +11140|F|U|Primary|10000|Unknown|1|0|0| +11141|M|M|Secondary|10000|Unknown|1|0|0| +11142|F|M|Secondary|10000|Unknown|1|0|0| +11143|M|S|Secondary|10000|Unknown|1|0|0| +11144|F|S|Secondary|10000|Unknown|1|0|0| +11145|M|D|Secondary|10000|Unknown|1|0|0| +11146|F|D|Secondary|10000|Unknown|1|0|0| +11147|M|W|Secondary|10000|Unknown|1|0|0| +11148|F|W|Secondary|10000|Unknown|1|0|0| +11149|M|U|Secondary|10000|Unknown|1|0|0| +11150|F|U|Secondary|10000|Unknown|1|0|0| +11151|M|M|College|10000|Unknown|1|0|0| +11152|F|M|College|10000|Unknown|1|0|0| +11153|M|S|College|10000|Unknown|1|0|0| +11154|F|S|College|10000|Unknown|1|0|0| +11155|M|D|College|10000|Unknown|1|0|0| +11156|F|D|College|10000|Unknown|1|0|0| +11157|M|W|College|10000|Unknown|1|0|0| +11158|F|W|College|10000|Unknown|1|0|0| +11159|M|U|College|10000|Unknown|1|0|0| +11160|F|U|College|10000|Unknown|1|0|0| +11161|M|M|2 yr Degree|10000|Unknown|1|0|0| +11162|F|M|2 yr Degree|10000|Unknown|1|0|0| +11163|M|S|2 yr Degree|10000|Unknown|1|0|0| +11164|F|S|2 yr Degree|10000|Unknown|1|0|0| +11165|M|D|2 yr Degree|10000|Unknown|1|0|0| +11166|F|D|2 yr Degree|10000|Unknown|1|0|0| +11167|M|W|2 yr Degree|10000|Unknown|1|0|0| +11168|F|W|2 yr Degree|10000|Unknown|1|0|0| +11169|M|U|2 yr Degree|10000|Unknown|1|0|0| +11170|F|U|2 yr Degree|10000|Unknown|1|0|0| +11171|M|M|4 yr Degree|10000|Unknown|1|0|0| +11172|F|M|4 yr Degree|10000|Unknown|1|0|0| +11173|M|S|4 yr Degree|10000|Unknown|1|0|0| +11174|F|S|4 yr Degree|10000|Unknown|1|0|0| +11175|M|D|4 yr Degree|10000|Unknown|1|0|0| +11176|F|D|4 yr Degree|10000|Unknown|1|0|0| +11177|M|W|4 yr Degree|10000|Unknown|1|0|0| +11178|F|W|4 yr Degree|10000|Unknown|1|0|0| +11179|M|U|4 yr Degree|10000|Unknown|1|0|0| +11180|F|U|4 yr Degree|10000|Unknown|1|0|0| +11181|M|M|Advanced Degree|10000|Unknown|1|0|0| +11182|F|M|Advanced Degree|10000|Unknown|1|0|0| +11183|M|S|Advanced Degree|10000|Unknown|1|0|0| +11184|F|S|Advanced Degree|10000|Unknown|1|0|0| +11185|M|D|Advanced Degree|10000|Unknown|1|0|0| +11186|F|D|Advanced Degree|10000|Unknown|1|0|0| +11187|M|W|Advanced Degree|10000|Unknown|1|0|0| +11188|F|W|Advanced Degree|10000|Unknown|1|0|0| +11189|M|U|Advanced Degree|10000|Unknown|1|0|0| +11190|F|U|Advanced Degree|10000|Unknown|1|0|0| +11191|M|M|Unknown|10000|Unknown|1|0|0| +11192|F|M|Unknown|10000|Unknown|1|0|0| +11193|M|S|Unknown|10000|Unknown|1|0|0| +11194|F|S|Unknown|10000|Unknown|1|0|0| +11195|M|D|Unknown|10000|Unknown|1|0|0| +11196|F|D|Unknown|10000|Unknown|1|0|0| +11197|M|W|Unknown|10000|Unknown|1|0|0| +11198|F|W|Unknown|10000|Unknown|1|0|0| +11199|M|U|Unknown|10000|Unknown|1|0|0| +11200|F|U|Unknown|10000|Unknown|1|0|0| +11201|M|M|Primary|500|Good|2|0|0| +11202|F|M|Primary|500|Good|2|0|0| +11203|M|S|Primary|500|Good|2|0|0| +11204|F|S|Primary|500|Good|2|0|0| +11205|M|D|Primary|500|Good|2|0|0| +11206|F|D|Primary|500|Good|2|0|0| +11207|M|W|Primary|500|Good|2|0|0| +11208|F|W|Primary|500|Good|2|0|0| +11209|M|U|Primary|500|Good|2|0|0| +11210|F|U|Primary|500|Good|2|0|0| +11211|M|M|Secondary|500|Good|2|0|0| +11212|F|M|Secondary|500|Good|2|0|0| +11213|M|S|Secondary|500|Good|2|0|0| +11214|F|S|Secondary|500|Good|2|0|0| +11215|M|D|Secondary|500|Good|2|0|0| +11216|F|D|Secondary|500|Good|2|0|0| +11217|M|W|Secondary|500|Good|2|0|0| +11218|F|W|Secondary|500|Good|2|0|0| +11219|M|U|Secondary|500|Good|2|0|0| +11220|F|U|Secondary|500|Good|2|0|0| +11221|M|M|College|500|Good|2|0|0| +11222|F|M|College|500|Good|2|0|0| +11223|M|S|College|500|Good|2|0|0| +11224|F|S|College|500|Good|2|0|0| +11225|M|D|College|500|Good|2|0|0| +11226|F|D|College|500|Good|2|0|0| +11227|M|W|College|500|Good|2|0|0| +11228|F|W|College|500|Good|2|0|0| +11229|M|U|College|500|Good|2|0|0| +11230|F|U|College|500|Good|2|0|0| +11231|M|M|2 yr Degree|500|Good|2|0|0| +11232|F|M|2 yr Degree|500|Good|2|0|0| +11233|M|S|2 yr Degree|500|Good|2|0|0| +11234|F|S|2 yr Degree|500|Good|2|0|0| +11235|M|D|2 yr Degree|500|Good|2|0|0| +11236|F|D|2 yr Degree|500|Good|2|0|0| +11237|M|W|2 yr Degree|500|Good|2|0|0| +11238|F|W|2 yr Degree|500|Good|2|0|0| +11239|M|U|2 yr Degree|500|Good|2|0|0| +11240|F|U|2 yr Degree|500|Good|2|0|0| +11241|M|M|4 yr Degree|500|Good|2|0|0| +11242|F|M|4 yr Degree|500|Good|2|0|0| +11243|M|S|4 yr Degree|500|Good|2|0|0| +11244|F|S|4 yr Degree|500|Good|2|0|0| +11245|M|D|4 yr Degree|500|Good|2|0|0| +11246|F|D|4 yr Degree|500|Good|2|0|0| +11247|M|W|4 yr Degree|500|Good|2|0|0| +11248|F|W|4 yr Degree|500|Good|2|0|0| +11249|M|U|4 yr Degree|500|Good|2|0|0| +11250|F|U|4 yr Degree|500|Good|2|0|0| +11251|M|M|Advanced Degree|500|Good|2|0|0| +11252|F|M|Advanced Degree|500|Good|2|0|0| +11253|M|S|Advanced Degree|500|Good|2|0|0| +11254|F|S|Advanced Degree|500|Good|2|0|0| +11255|M|D|Advanced Degree|500|Good|2|0|0| +11256|F|D|Advanced Degree|500|Good|2|0|0| +11257|M|W|Advanced Degree|500|Good|2|0|0| +11258|F|W|Advanced Degree|500|Good|2|0|0| +11259|M|U|Advanced Degree|500|Good|2|0|0| +11260|F|U|Advanced Degree|500|Good|2|0|0| +11261|M|M|Unknown|500|Good|2|0|0| +11262|F|M|Unknown|500|Good|2|0|0| +11263|M|S|Unknown|500|Good|2|0|0| +11264|F|S|Unknown|500|Good|2|0|0| +11265|M|D|Unknown|500|Good|2|0|0| +11266|F|D|Unknown|500|Good|2|0|0| +11267|M|W|Unknown|500|Good|2|0|0| +11268|F|W|Unknown|500|Good|2|0|0| +11269|M|U|Unknown|500|Good|2|0|0| +11270|F|U|Unknown|500|Good|2|0|0| +11271|M|M|Primary|1000|Good|2|0|0| +11272|F|M|Primary|1000|Good|2|0|0| +11273|M|S|Primary|1000|Good|2|0|0| +11274|F|S|Primary|1000|Good|2|0|0| +11275|M|D|Primary|1000|Good|2|0|0| +11276|F|D|Primary|1000|Good|2|0|0| +11277|M|W|Primary|1000|Good|2|0|0| +11278|F|W|Primary|1000|Good|2|0|0| +11279|M|U|Primary|1000|Good|2|0|0| +11280|F|U|Primary|1000|Good|2|0|0| +11281|M|M|Secondary|1000|Good|2|0|0| +11282|F|M|Secondary|1000|Good|2|0|0| +11283|M|S|Secondary|1000|Good|2|0|0| +11284|F|S|Secondary|1000|Good|2|0|0| +11285|M|D|Secondary|1000|Good|2|0|0| +11286|F|D|Secondary|1000|Good|2|0|0| +11287|M|W|Secondary|1000|Good|2|0|0| +11288|F|W|Secondary|1000|Good|2|0|0| +11289|M|U|Secondary|1000|Good|2|0|0| +11290|F|U|Secondary|1000|Good|2|0|0| +11291|M|M|College|1000|Good|2|0|0| +11292|F|M|College|1000|Good|2|0|0| +11293|M|S|College|1000|Good|2|0|0| +11294|F|S|College|1000|Good|2|0|0| +11295|M|D|College|1000|Good|2|0|0| +11296|F|D|College|1000|Good|2|0|0| +11297|M|W|College|1000|Good|2|0|0| +11298|F|W|College|1000|Good|2|0|0| +11299|M|U|College|1000|Good|2|0|0| +11300|F|U|College|1000|Good|2|0|0| +11301|M|M|2 yr Degree|1000|Good|2|0|0| +11302|F|M|2 yr Degree|1000|Good|2|0|0| +11303|M|S|2 yr Degree|1000|Good|2|0|0| +11304|F|S|2 yr Degree|1000|Good|2|0|0| +11305|M|D|2 yr Degree|1000|Good|2|0|0| +11306|F|D|2 yr Degree|1000|Good|2|0|0| +11307|M|W|2 yr Degree|1000|Good|2|0|0| +11308|F|W|2 yr Degree|1000|Good|2|0|0| +11309|M|U|2 yr Degree|1000|Good|2|0|0| +11310|F|U|2 yr Degree|1000|Good|2|0|0| +11311|M|M|4 yr Degree|1000|Good|2|0|0| +11312|F|M|4 yr Degree|1000|Good|2|0|0| +11313|M|S|4 yr Degree|1000|Good|2|0|0| +11314|F|S|4 yr Degree|1000|Good|2|0|0| +11315|M|D|4 yr Degree|1000|Good|2|0|0| +11316|F|D|4 yr Degree|1000|Good|2|0|0| +11317|M|W|4 yr Degree|1000|Good|2|0|0| +11318|F|W|4 yr Degree|1000|Good|2|0|0| +11319|M|U|4 yr Degree|1000|Good|2|0|0| +11320|F|U|4 yr Degree|1000|Good|2|0|0| +11321|M|M|Advanced Degree|1000|Good|2|0|0| +11322|F|M|Advanced Degree|1000|Good|2|0|0| +11323|M|S|Advanced Degree|1000|Good|2|0|0| +11324|F|S|Advanced Degree|1000|Good|2|0|0| +11325|M|D|Advanced Degree|1000|Good|2|0|0| +11326|F|D|Advanced Degree|1000|Good|2|0|0| +11327|M|W|Advanced Degree|1000|Good|2|0|0| +11328|F|W|Advanced Degree|1000|Good|2|0|0| +11329|M|U|Advanced Degree|1000|Good|2|0|0| +11330|F|U|Advanced Degree|1000|Good|2|0|0| +11331|M|M|Unknown|1000|Good|2|0|0| +11332|F|M|Unknown|1000|Good|2|0|0| +11333|M|S|Unknown|1000|Good|2|0|0| +11334|F|S|Unknown|1000|Good|2|0|0| +11335|M|D|Unknown|1000|Good|2|0|0| +11336|F|D|Unknown|1000|Good|2|0|0| +11337|M|W|Unknown|1000|Good|2|0|0| +11338|F|W|Unknown|1000|Good|2|0|0| +11339|M|U|Unknown|1000|Good|2|0|0| +11340|F|U|Unknown|1000|Good|2|0|0| +11341|M|M|Primary|1500|Good|2|0|0| +11342|F|M|Primary|1500|Good|2|0|0| +11343|M|S|Primary|1500|Good|2|0|0| +11344|F|S|Primary|1500|Good|2|0|0| +11345|M|D|Primary|1500|Good|2|0|0| +11346|F|D|Primary|1500|Good|2|0|0| +11347|M|W|Primary|1500|Good|2|0|0| +11348|F|W|Primary|1500|Good|2|0|0| +11349|M|U|Primary|1500|Good|2|0|0| +11350|F|U|Primary|1500|Good|2|0|0| +11351|M|M|Secondary|1500|Good|2|0|0| +11352|F|M|Secondary|1500|Good|2|0|0| +11353|M|S|Secondary|1500|Good|2|0|0| +11354|F|S|Secondary|1500|Good|2|0|0| +11355|M|D|Secondary|1500|Good|2|0|0| +11356|F|D|Secondary|1500|Good|2|0|0| +11357|M|W|Secondary|1500|Good|2|0|0| +11358|F|W|Secondary|1500|Good|2|0|0| +11359|M|U|Secondary|1500|Good|2|0|0| +11360|F|U|Secondary|1500|Good|2|0|0| +11361|M|M|College|1500|Good|2|0|0| +11362|F|M|College|1500|Good|2|0|0| +11363|M|S|College|1500|Good|2|0|0| +11364|F|S|College|1500|Good|2|0|0| +11365|M|D|College|1500|Good|2|0|0| +11366|F|D|College|1500|Good|2|0|0| +11367|M|W|College|1500|Good|2|0|0| +11368|F|W|College|1500|Good|2|0|0| +11369|M|U|College|1500|Good|2|0|0| +11370|F|U|College|1500|Good|2|0|0| +11371|M|M|2 yr Degree|1500|Good|2|0|0| +11372|F|M|2 yr Degree|1500|Good|2|0|0| +11373|M|S|2 yr Degree|1500|Good|2|0|0| +11374|F|S|2 yr Degree|1500|Good|2|0|0| +11375|M|D|2 yr Degree|1500|Good|2|0|0| +11376|F|D|2 yr Degree|1500|Good|2|0|0| +11377|M|W|2 yr Degree|1500|Good|2|0|0| +11378|F|W|2 yr Degree|1500|Good|2|0|0| +11379|M|U|2 yr Degree|1500|Good|2|0|0| +11380|F|U|2 yr Degree|1500|Good|2|0|0| +11381|M|M|4 yr Degree|1500|Good|2|0|0| +11382|F|M|4 yr Degree|1500|Good|2|0|0| +11383|M|S|4 yr Degree|1500|Good|2|0|0| +11384|F|S|4 yr Degree|1500|Good|2|0|0| +11385|M|D|4 yr Degree|1500|Good|2|0|0| +11386|F|D|4 yr Degree|1500|Good|2|0|0| +11387|M|W|4 yr Degree|1500|Good|2|0|0| +11388|F|W|4 yr Degree|1500|Good|2|0|0| +11389|M|U|4 yr Degree|1500|Good|2|0|0| +11390|F|U|4 yr Degree|1500|Good|2|0|0| +11391|M|M|Advanced Degree|1500|Good|2|0|0| +11392|F|M|Advanced Degree|1500|Good|2|0|0| +11393|M|S|Advanced Degree|1500|Good|2|0|0| +11394|F|S|Advanced Degree|1500|Good|2|0|0| +11395|M|D|Advanced Degree|1500|Good|2|0|0| +11396|F|D|Advanced Degree|1500|Good|2|0|0| +11397|M|W|Advanced Degree|1500|Good|2|0|0| +11398|F|W|Advanced Degree|1500|Good|2|0|0| +11399|M|U|Advanced Degree|1500|Good|2|0|0| +11400|F|U|Advanced Degree|1500|Good|2|0|0| +11401|M|M|Unknown|1500|Good|2|0|0| +11402|F|M|Unknown|1500|Good|2|0|0| +11403|M|S|Unknown|1500|Good|2|0|0| +11404|F|S|Unknown|1500|Good|2|0|0| +11405|M|D|Unknown|1500|Good|2|0|0| +11406|F|D|Unknown|1500|Good|2|0|0| +11407|M|W|Unknown|1500|Good|2|0|0| +11408|F|W|Unknown|1500|Good|2|0|0| +11409|M|U|Unknown|1500|Good|2|0|0| +11410|F|U|Unknown|1500|Good|2|0|0| +11411|M|M|Primary|2000|Good|2|0|0| +11412|F|M|Primary|2000|Good|2|0|0| +11413|M|S|Primary|2000|Good|2|0|0| +11414|F|S|Primary|2000|Good|2|0|0| +11415|M|D|Primary|2000|Good|2|0|0| +11416|F|D|Primary|2000|Good|2|0|0| +11417|M|W|Primary|2000|Good|2|0|0| +11418|F|W|Primary|2000|Good|2|0|0| +11419|M|U|Primary|2000|Good|2|0|0| +11420|F|U|Primary|2000|Good|2|0|0| +11421|M|M|Secondary|2000|Good|2|0|0| +11422|F|M|Secondary|2000|Good|2|0|0| +11423|M|S|Secondary|2000|Good|2|0|0| +11424|F|S|Secondary|2000|Good|2|0|0| +11425|M|D|Secondary|2000|Good|2|0|0| +11426|F|D|Secondary|2000|Good|2|0|0| +11427|M|W|Secondary|2000|Good|2|0|0| +11428|F|W|Secondary|2000|Good|2|0|0| +11429|M|U|Secondary|2000|Good|2|0|0| +11430|F|U|Secondary|2000|Good|2|0|0| +11431|M|M|College|2000|Good|2|0|0| +11432|F|M|College|2000|Good|2|0|0| +11433|M|S|College|2000|Good|2|0|0| +11434|F|S|College|2000|Good|2|0|0| +11435|M|D|College|2000|Good|2|0|0| +11436|F|D|College|2000|Good|2|0|0| +11437|M|W|College|2000|Good|2|0|0| +11438|F|W|College|2000|Good|2|0|0| +11439|M|U|College|2000|Good|2|0|0| +11440|F|U|College|2000|Good|2|0|0| +11441|M|M|2 yr Degree|2000|Good|2|0|0| +11442|F|M|2 yr Degree|2000|Good|2|0|0| +11443|M|S|2 yr Degree|2000|Good|2|0|0| +11444|F|S|2 yr Degree|2000|Good|2|0|0| +11445|M|D|2 yr Degree|2000|Good|2|0|0| +11446|F|D|2 yr Degree|2000|Good|2|0|0| +11447|M|W|2 yr Degree|2000|Good|2|0|0| +11448|F|W|2 yr Degree|2000|Good|2|0|0| +11449|M|U|2 yr Degree|2000|Good|2|0|0| +11450|F|U|2 yr Degree|2000|Good|2|0|0| +11451|M|M|4 yr Degree|2000|Good|2|0|0| +11452|F|M|4 yr Degree|2000|Good|2|0|0| +11453|M|S|4 yr Degree|2000|Good|2|0|0| +11454|F|S|4 yr Degree|2000|Good|2|0|0| +11455|M|D|4 yr Degree|2000|Good|2|0|0| +11456|F|D|4 yr Degree|2000|Good|2|0|0| +11457|M|W|4 yr Degree|2000|Good|2|0|0| +11458|F|W|4 yr Degree|2000|Good|2|0|0| +11459|M|U|4 yr Degree|2000|Good|2|0|0| +11460|F|U|4 yr Degree|2000|Good|2|0|0| +11461|M|M|Advanced Degree|2000|Good|2|0|0| +11462|F|M|Advanced Degree|2000|Good|2|0|0| +11463|M|S|Advanced Degree|2000|Good|2|0|0| +11464|F|S|Advanced Degree|2000|Good|2|0|0| +11465|M|D|Advanced Degree|2000|Good|2|0|0| +11466|F|D|Advanced Degree|2000|Good|2|0|0| +11467|M|W|Advanced Degree|2000|Good|2|0|0| +11468|F|W|Advanced Degree|2000|Good|2|0|0| +11469|M|U|Advanced Degree|2000|Good|2|0|0| +11470|F|U|Advanced Degree|2000|Good|2|0|0| +11471|M|M|Unknown|2000|Good|2|0|0| +11472|F|M|Unknown|2000|Good|2|0|0| +11473|M|S|Unknown|2000|Good|2|0|0| +11474|F|S|Unknown|2000|Good|2|0|0| +11475|M|D|Unknown|2000|Good|2|0|0| +11476|F|D|Unknown|2000|Good|2|0|0| +11477|M|W|Unknown|2000|Good|2|0|0| +11478|F|W|Unknown|2000|Good|2|0|0| +11479|M|U|Unknown|2000|Good|2|0|0| +11480|F|U|Unknown|2000|Good|2|0|0| +11481|M|M|Primary|2500|Good|2|0|0| +11482|F|M|Primary|2500|Good|2|0|0| +11483|M|S|Primary|2500|Good|2|0|0| +11484|F|S|Primary|2500|Good|2|0|0| +11485|M|D|Primary|2500|Good|2|0|0| +11486|F|D|Primary|2500|Good|2|0|0| +11487|M|W|Primary|2500|Good|2|0|0| +11488|F|W|Primary|2500|Good|2|0|0| +11489|M|U|Primary|2500|Good|2|0|0| +11490|F|U|Primary|2500|Good|2|0|0| +11491|M|M|Secondary|2500|Good|2|0|0| +11492|F|M|Secondary|2500|Good|2|0|0| +11493|M|S|Secondary|2500|Good|2|0|0| +11494|F|S|Secondary|2500|Good|2|0|0| +11495|M|D|Secondary|2500|Good|2|0|0| +11496|F|D|Secondary|2500|Good|2|0|0| +11497|M|W|Secondary|2500|Good|2|0|0| +11498|F|W|Secondary|2500|Good|2|0|0| +11499|M|U|Secondary|2500|Good|2|0|0| +11500|F|U|Secondary|2500|Good|2|0|0| +11501|M|M|College|2500|Good|2|0|0| +11502|F|M|College|2500|Good|2|0|0| +11503|M|S|College|2500|Good|2|0|0| +11504|F|S|College|2500|Good|2|0|0| +11505|M|D|College|2500|Good|2|0|0| +11506|F|D|College|2500|Good|2|0|0| +11507|M|W|College|2500|Good|2|0|0| +11508|F|W|College|2500|Good|2|0|0| +11509|M|U|College|2500|Good|2|0|0| +11510|F|U|College|2500|Good|2|0|0| +11511|M|M|2 yr Degree|2500|Good|2|0|0| +11512|F|M|2 yr Degree|2500|Good|2|0|0| +11513|M|S|2 yr Degree|2500|Good|2|0|0| +11514|F|S|2 yr Degree|2500|Good|2|0|0| +11515|M|D|2 yr Degree|2500|Good|2|0|0| +11516|F|D|2 yr Degree|2500|Good|2|0|0| +11517|M|W|2 yr Degree|2500|Good|2|0|0| +11518|F|W|2 yr Degree|2500|Good|2|0|0| +11519|M|U|2 yr Degree|2500|Good|2|0|0| +11520|F|U|2 yr Degree|2500|Good|2|0|0| +11521|M|M|4 yr Degree|2500|Good|2|0|0| +11522|F|M|4 yr Degree|2500|Good|2|0|0| +11523|M|S|4 yr Degree|2500|Good|2|0|0| +11524|F|S|4 yr Degree|2500|Good|2|0|0| +11525|M|D|4 yr Degree|2500|Good|2|0|0| +11526|F|D|4 yr Degree|2500|Good|2|0|0| +11527|M|W|4 yr Degree|2500|Good|2|0|0| +11528|F|W|4 yr Degree|2500|Good|2|0|0| +11529|M|U|4 yr Degree|2500|Good|2|0|0| +11530|F|U|4 yr Degree|2500|Good|2|0|0| +11531|M|M|Advanced Degree|2500|Good|2|0|0| +11532|F|M|Advanced Degree|2500|Good|2|0|0| +11533|M|S|Advanced Degree|2500|Good|2|0|0| +11534|F|S|Advanced Degree|2500|Good|2|0|0| +11535|M|D|Advanced Degree|2500|Good|2|0|0| +11536|F|D|Advanced Degree|2500|Good|2|0|0| +11537|M|W|Advanced Degree|2500|Good|2|0|0| +11538|F|W|Advanced Degree|2500|Good|2|0|0| +11539|M|U|Advanced Degree|2500|Good|2|0|0| +11540|F|U|Advanced Degree|2500|Good|2|0|0| +11541|M|M|Unknown|2500|Good|2|0|0| +11542|F|M|Unknown|2500|Good|2|0|0| +11543|M|S|Unknown|2500|Good|2|0|0| +11544|F|S|Unknown|2500|Good|2|0|0| +11545|M|D|Unknown|2500|Good|2|0|0| +11546|F|D|Unknown|2500|Good|2|0|0| +11547|M|W|Unknown|2500|Good|2|0|0| +11548|F|W|Unknown|2500|Good|2|0|0| +11549|M|U|Unknown|2500|Good|2|0|0| +11550|F|U|Unknown|2500|Good|2|0|0| +11551|M|M|Primary|3000|Good|2|0|0| +11552|F|M|Primary|3000|Good|2|0|0| +11553|M|S|Primary|3000|Good|2|0|0| +11554|F|S|Primary|3000|Good|2|0|0| +11555|M|D|Primary|3000|Good|2|0|0| +11556|F|D|Primary|3000|Good|2|0|0| +11557|M|W|Primary|3000|Good|2|0|0| +11558|F|W|Primary|3000|Good|2|0|0| +11559|M|U|Primary|3000|Good|2|0|0| +11560|F|U|Primary|3000|Good|2|0|0| +11561|M|M|Secondary|3000|Good|2|0|0| +11562|F|M|Secondary|3000|Good|2|0|0| +11563|M|S|Secondary|3000|Good|2|0|0| +11564|F|S|Secondary|3000|Good|2|0|0| +11565|M|D|Secondary|3000|Good|2|0|0| +11566|F|D|Secondary|3000|Good|2|0|0| +11567|M|W|Secondary|3000|Good|2|0|0| +11568|F|W|Secondary|3000|Good|2|0|0| +11569|M|U|Secondary|3000|Good|2|0|0| +11570|F|U|Secondary|3000|Good|2|0|0| +11571|M|M|College|3000|Good|2|0|0| +11572|F|M|College|3000|Good|2|0|0| +11573|M|S|College|3000|Good|2|0|0| +11574|F|S|College|3000|Good|2|0|0| +11575|M|D|College|3000|Good|2|0|0| +11576|F|D|College|3000|Good|2|0|0| +11577|M|W|College|3000|Good|2|0|0| +11578|F|W|College|3000|Good|2|0|0| +11579|M|U|College|3000|Good|2|0|0| +11580|F|U|College|3000|Good|2|0|0| +11581|M|M|2 yr Degree|3000|Good|2|0|0| +11582|F|M|2 yr Degree|3000|Good|2|0|0| +11583|M|S|2 yr Degree|3000|Good|2|0|0| +11584|F|S|2 yr Degree|3000|Good|2|0|0| +11585|M|D|2 yr Degree|3000|Good|2|0|0| +11586|F|D|2 yr Degree|3000|Good|2|0|0| +11587|M|W|2 yr Degree|3000|Good|2|0|0| +11588|F|W|2 yr Degree|3000|Good|2|0|0| +11589|M|U|2 yr Degree|3000|Good|2|0|0| +11590|F|U|2 yr Degree|3000|Good|2|0|0| +11591|M|M|4 yr Degree|3000|Good|2|0|0| +11592|F|M|4 yr Degree|3000|Good|2|0|0| +11593|M|S|4 yr Degree|3000|Good|2|0|0| +11594|F|S|4 yr Degree|3000|Good|2|0|0| +11595|M|D|4 yr Degree|3000|Good|2|0|0| +11596|F|D|4 yr Degree|3000|Good|2|0|0| +11597|M|W|4 yr Degree|3000|Good|2|0|0| +11598|F|W|4 yr Degree|3000|Good|2|0|0| +11599|M|U|4 yr Degree|3000|Good|2|0|0| +11600|F|U|4 yr Degree|3000|Good|2|0|0| +11601|M|M|Advanced Degree|3000|Good|2|0|0| +11602|F|M|Advanced Degree|3000|Good|2|0|0| +11603|M|S|Advanced Degree|3000|Good|2|0|0| +11604|F|S|Advanced Degree|3000|Good|2|0|0| +11605|M|D|Advanced Degree|3000|Good|2|0|0| +11606|F|D|Advanced Degree|3000|Good|2|0|0| +11607|M|W|Advanced Degree|3000|Good|2|0|0| +11608|F|W|Advanced Degree|3000|Good|2|0|0| +11609|M|U|Advanced Degree|3000|Good|2|0|0| +11610|F|U|Advanced Degree|3000|Good|2|0|0| +11611|M|M|Unknown|3000|Good|2|0|0| +11612|F|M|Unknown|3000|Good|2|0|0| +11613|M|S|Unknown|3000|Good|2|0|0| +11614|F|S|Unknown|3000|Good|2|0|0| +11615|M|D|Unknown|3000|Good|2|0|0| +11616|F|D|Unknown|3000|Good|2|0|0| +11617|M|W|Unknown|3000|Good|2|0|0| +11618|F|W|Unknown|3000|Good|2|0|0| +11619|M|U|Unknown|3000|Good|2|0|0| +11620|F|U|Unknown|3000|Good|2|0|0| +11621|M|M|Primary|3500|Good|2|0|0| +11622|F|M|Primary|3500|Good|2|0|0| +11623|M|S|Primary|3500|Good|2|0|0| +11624|F|S|Primary|3500|Good|2|0|0| +11625|M|D|Primary|3500|Good|2|0|0| +11626|F|D|Primary|3500|Good|2|0|0| +11627|M|W|Primary|3500|Good|2|0|0| +11628|F|W|Primary|3500|Good|2|0|0| +11629|M|U|Primary|3500|Good|2|0|0| +11630|F|U|Primary|3500|Good|2|0|0| +11631|M|M|Secondary|3500|Good|2|0|0| +11632|F|M|Secondary|3500|Good|2|0|0| +11633|M|S|Secondary|3500|Good|2|0|0| +11634|F|S|Secondary|3500|Good|2|0|0| +11635|M|D|Secondary|3500|Good|2|0|0| +11636|F|D|Secondary|3500|Good|2|0|0| +11637|M|W|Secondary|3500|Good|2|0|0| +11638|F|W|Secondary|3500|Good|2|0|0| +11639|M|U|Secondary|3500|Good|2|0|0| +11640|F|U|Secondary|3500|Good|2|0|0| +11641|M|M|College|3500|Good|2|0|0| +11642|F|M|College|3500|Good|2|0|0| +11643|M|S|College|3500|Good|2|0|0| +11644|F|S|College|3500|Good|2|0|0| +11645|M|D|College|3500|Good|2|0|0| +11646|F|D|College|3500|Good|2|0|0| +11647|M|W|College|3500|Good|2|0|0| +11648|F|W|College|3500|Good|2|0|0| +11649|M|U|College|3500|Good|2|0|0| +11650|F|U|College|3500|Good|2|0|0| +11651|M|M|2 yr Degree|3500|Good|2|0|0| +11652|F|M|2 yr Degree|3500|Good|2|0|0| +11653|M|S|2 yr Degree|3500|Good|2|0|0| +11654|F|S|2 yr Degree|3500|Good|2|0|0| +11655|M|D|2 yr Degree|3500|Good|2|0|0| +11656|F|D|2 yr Degree|3500|Good|2|0|0| +11657|M|W|2 yr Degree|3500|Good|2|0|0| +11658|F|W|2 yr Degree|3500|Good|2|0|0| +11659|M|U|2 yr Degree|3500|Good|2|0|0| +11660|F|U|2 yr Degree|3500|Good|2|0|0| +11661|M|M|4 yr Degree|3500|Good|2|0|0| +11662|F|M|4 yr Degree|3500|Good|2|0|0| +11663|M|S|4 yr Degree|3500|Good|2|0|0| +11664|F|S|4 yr Degree|3500|Good|2|0|0| +11665|M|D|4 yr Degree|3500|Good|2|0|0| +11666|F|D|4 yr Degree|3500|Good|2|0|0| +11667|M|W|4 yr Degree|3500|Good|2|0|0| +11668|F|W|4 yr Degree|3500|Good|2|0|0| +11669|M|U|4 yr Degree|3500|Good|2|0|0| +11670|F|U|4 yr Degree|3500|Good|2|0|0| +11671|M|M|Advanced Degree|3500|Good|2|0|0| +11672|F|M|Advanced Degree|3500|Good|2|0|0| +11673|M|S|Advanced Degree|3500|Good|2|0|0| +11674|F|S|Advanced Degree|3500|Good|2|0|0| +11675|M|D|Advanced Degree|3500|Good|2|0|0| +11676|F|D|Advanced Degree|3500|Good|2|0|0| +11677|M|W|Advanced Degree|3500|Good|2|0|0| +11678|F|W|Advanced Degree|3500|Good|2|0|0| +11679|M|U|Advanced Degree|3500|Good|2|0|0| +11680|F|U|Advanced Degree|3500|Good|2|0|0| +11681|M|M|Unknown|3500|Good|2|0|0| +11682|F|M|Unknown|3500|Good|2|0|0| +11683|M|S|Unknown|3500|Good|2|0|0| +11684|F|S|Unknown|3500|Good|2|0|0| +11685|M|D|Unknown|3500|Good|2|0|0| +11686|F|D|Unknown|3500|Good|2|0|0| +11687|M|W|Unknown|3500|Good|2|0|0| +11688|F|W|Unknown|3500|Good|2|0|0| +11689|M|U|Unknown|3500|Good|2|0|0| +11690|F|U|Unknown|3500|Good|2|0|0| +11691|M|M|Primary|4000|Good|2|0|0| +11692|F|M|Primary|4000|Good|2|0|0| +11693|M|S|Primary|4000|Good|2|0|0| +11694|F|S|Primary|4000|Good|2|0|0| +11695|M|D|Primary|4000|Good|2|0|0| +11696|F|D|Primary|4000|Good|2|0|0| +11697|M|W|Primary|4000|Good|2|0|0| +11698|F|W|Primary|4000|Good|2|0|0| +11699|M|U|Primary|4000|Good|2|0|0| +11700|F|U|Primary|4000|Good|2|0|0| +11701|M|M|Secondary|4000|Good|2|0|0| +11702|F|M|Secondary|4000|Good|2|0|0| +11703|M|S|Secondary|4000|Good|2|0|0| +11704|F|S|Secondary|4000|Good|2|0|0| +11705|M|D|Secondary|4000|Good|2|0|0| +11706|F|D|Secondary|4000|Good|2|0|0| +11707|M|W|Secondary|4000|Good|2|0|0| +11708|F|W|Secondary|4000|Good|2|0|0| +11709|M|U|Secondary|4000|Good|2|0|0| +11710|F|U|Secondary|4000|Good|2|0|0| +11711|M|M|College|4000|Good|2|0|0| +11712|F|M|College|4000|Good|2|0|0| +11713|M|S|College|4000|Good|2|0|0| +11714|F|S|College|4000|Good|2|0|0| +11715|M|D|College|4000|Good|2|0|0| +11716|F|D|College|4000|Good|2|0|0| +11717|M|W|College|4000|Good|2|0|0| +11718|F|W|College|4000|Good|2|0|0| +11719|M|U|College|4000|Good|2|0|0| +11720|F|U|College|4000|Good|2|0|0| +11721|M|M|2 yr Degree|4000|Good|2|0|0| +11722|F|M|2 yr Degree|4000|Good|2|0|0| +11723|M|S|2 yr Degree|4000|Good|2|0|0| +11724|F|S|2 yr Degree|4000|Good|2|0|0| +11725|M|D|2 yr Degree|4000|Good|2|0|0| +11726|F|D|2 yr Degree|4000|Good|2|0|0| +11727|M|W|2 yr Degree|4000|Good|2|0|0| +11728|F|W|2 yr Degree|4000|Good|2|0|0| +11729|M|U|2 yr Degree|4000|Good|2|0|0| +11730|F|U|2 yr Degree|4000|Good|2|0|0| +11731|M|M|4 yr Degree|4000|Good|2|0|0| +11732|F|M|4 yr Degree|4000|Good|2|0|0| +11733|M|S|4 yr Degree|4000|Good|2|0|0| +11734|F|S|4 yr Degree|4000|Good|2|0|0| +11735|M|D|4 yr Degree|4000|Good|2|0|0| +11736|F|D|4 yr Degree|4000|Good|2|0|0| +11737|M|W|4 yr Degree|4000|Good|2|0|0| +11738|F|W|4 yr Degree|4000|Good|2|0|0| +11739|M|U|4 yr Degree|4000|Good|2|0|0| +11740|F|U|4 yr Degree|4000|Good|2|0|0| +11741|M|M|Advanced Degree|4000|Good|2|0|0| +11742|F|M|Advanced Degree|4000|Good|2|0|0| +11743|M|S|Advanced Degree|4000|Good|2|0|0| +11744|F|S|Advanced Degree|4000|Good|2|0|0| +11745|M|D|Advanced Degree|4000|Good|2|0|0| +11746|F|D|Advanced Degree|4000|Good|2|0|0| +11747|M|W|Advanced Degree|4000|Good|2|0|0| +11748|F|W|Advanced Degree|4000|Good|2|0|0| +11749|M|U|Advanced Degree|4000|Good|2|0|0| +11750|F|U|Advanced Degree|4000|Good|2|0|0| +11751|M|M|Unknown|4000|Good|2|0|0| +11752|F|M|Unknown|4000|Good|2|0|0| +11753|M|S|Unknown|4000|Good|2|0|0| +11754|F|S|Unknown|4000|Good|2|0|0| +11755|M|D|Unknown|4000|Good|2|0|0| +11756|F|D|Unknown|4000|Good|2|0|0| +11757|M|W|Unknown|4000|Good|2|0|0| +11758|F|W|Unknown|4000|Good|2|0|0| +11759|M|U|Unknown|4000|Good|2|0|0| +11760|F|U|Unknown|4000|Good|2|0|0| +11761|M|M|Primary|4500|Good|2|0|0| +11762|F|M|Primary|4500|Good|2|0|0| +11763|M|S|Primary|4500|Good|2|0|0| +11764|F|S|Primary|4500|Good|2|0|0| +11765|M|D|Primary|4500|Good|2|0|0| +11766|F|D|Primary|4500|Good|2|0|0| +11767|M|W|Primary|4500|Good|2|0|0| +11768|F|W|Primary|4500|Good|2|0|0| +11769|M|U|Primary|4500|Good|2|0|0| +11770|F|U|Primary|4500|Good|2|0|0| +11771|M|M|Secondary|4500|Good|2|0|0| +11772|F|M|Secondary|4500|Good|2|0|0| +11773|M|S|Secondary|4500|Good|2|0|0| +11774|F|S|Secondary|4500|Good|2|0|0| +11775|M|D|Secondary|4500|Good|2|0|0| +11776|F|D|Secondary|4500|Good|2|0|0| +11777|M|W|Secondary|4500|Good|2|0|0| +11778|F|W|Secondary|4500|Good|2|0|0| +11779|M|U|Secondary|4500|Good|2|0|0| +11780|F|U|Secondary|4500|Good|2|0|0| +11781|M|M|College|4500|Good|2|0|0| +11782|F|M|College|4500|Good|2|0|0| +11783|M|S|College|4500|Good|2|0|0| +11784|F|S|College|4500|Good|2|0|0| +11785|M|D|College|4500|Good|2|0|0| +11786|F|D|College|4500|Good|2|0|0| +11787|M|W|College|4500|Good|2|0|0| +11788|F|W|College|4500|Good|2|0|0| +11789|M|U|College|4500|Good|2|0|0| +11790|F|U|College|4500|Good|2|0|0| +11791|M|M|2 yr Degree|4500|Good|2|0|0| +11792|F|M|2 yr Degree|4500|Good|2|0|0| +11793|M|S|2 yr Degree|4500|Good|2|0|0| +11794|F|S|2 yr Degree|4500|Good|2|0|0| +11795|M|D|2 yr Degree|4500|Good|2|0|0| +11796|F|D|2 yr Degree|4500|Good|2|0|0| +11797|M|W|2 yr Degree|4500|Good|2|0|0| +11798|F|W|2 yr Degree|4500|Good|2|0|0| +11799|M|U|2 yr Degree|4500|Good|2|0|0| +11800|F|U|2 yr Degree|4500|Good|2|0|0| +11801|M|M|4 yr Degree|4500|Good|2|0|0| +11802|F|M|4 yr Degree|4500|Good|2|0|0| +11803|M|S|4 yr Degree|4500|Good|2|0|0| +11804|F|S|4 yr Degree|4500|Good|2|0|0| +11805|M|D|4 yr Degree|4500|Good|2|0|0| +11806|F|D|4 yr Degree|4500|Good|2|0|0| +11807|M|W|4 yr Degree|4500|Good|2|0|0| +11808|F|W|4 yr Degree|4500|Good|2|0|0| +11809|M|U|4 yr Degree|4500|Good|2|0|0| +11810|F|U|4 yr Degree|4500|Good|2|0|0| +11811|M|M|Advanced Degree|4500|Good|2|0|0| +11812|F|M|Advanced Degree|4500|Good|2|0|0| +11813|M|S|Advanced Degree|4500|Good|2|0|0| +11814|F|S|Advanced Degree|4500|Good|2|0|0| +11815|M|D|Advanced Degree|4500|Good|2|0|0| +11816|F|D|Advanced Degree|4500|Good|2|0|0| +11817|M|W|Advanced Degree|4500|Good|2|0|0| +11818|F|W|Advanced Degree|4500|Good|2|0|0| +11819|M|U|Advanced Degree|4500|Good|2|0|0| +11820|F|U|Advanced Degree|4500|Good|2|0|0| +11821|M|M|Unknown|4500|Good|2|0|0| +11822|F|M|Unknown|4500|Good|2|0|0| +11823|M|S|Unknown|4500|Good|2|0|0| +11824|F|S|Unknown|4500|Good|2|0|0| +11825|M|D|Unknown|4500|Good|2|0|0| +11826|F|D|Unknown|4500|Good|2|0|0| +11827|M|W|Unknown|4500|Good|2|0|0| +11828|F|W|Unknown|4500|Good|2|0|0| +11829|M|U|Unknown|4500|Good|2|0|0| +11830|F|U|Unknown|4500|Good|2|0|0| +11831|M|M|Primary|5000|Good|2|0|0| +11832|F|M|Primary|5000|Good|2|0|0| +11833|M|S|Primary|5000|Good|2|0|0| +11834|F|S|Primary|5000|Good|2|0|0| +11835|M|D|Primary|5000|Good|2|0|0| +11836|F|D|Primary|5000|Good|2|0|0| +11837|M|W|Primary|5000|Good|2|0|0| +11838|F|W|Primary|5000|Good|2|0|0| +11839|M|U|Primary|5000|Good|2|0|0| +11840|F|U|Primary|5000|Good|2|0|0| +11841|M|M|Secondary|5000|Good|2|0|0| +11842|F|M|Secondary|5000|Good|2|0|0| +11843|M|S|Secondary|5000|Good|2|0|0| +11844|F|S|Secondary|5000|Good|2|0|0| +11845|M|D|Secondary|5000|Good|2|0|0| +11846|F|D|Secondary|5000|Good|2|0|0| +11847|M|W|Secondary|5000|Good|2|0|0| +11848|F|W|Secondary|5000|Good|2|0|0| +11849|M|U|Secondary|5000|Good|2|0|0| +11850|F|U|Secondary|5000|Good|2|0|0| +11851|M|M|College|5000|Good|2|0|0| +11852|F|M|College|5000|Good|2|0|0| +11853|M|S|College|5000|Good|2|0|0| +11854|F|S|College|5000|Good|2|0|0| +11855|M|D|College|5000|Good|2|0|0| +11856|F|D|College|5000|Good|2|0|0| +11857|M|W|College|5000|Good|2|0|0| +11858|F|W|College|5000|Good|2|0|0| +11859|M|U|College|5000|Good|2|0|0| +11860|F|U|College|5000|Good|2|0|0| +11861|M|M|2 yr Degree|5000|Good|2|0|0| +11862|F|M|2 yr Degree|5000|Good|2|0|0| +11863|M|S|2 yr Degree|5000|Good|2|0|0| +11864|F|S|2 yr Degree|5000|Good|2|0|0| +11865|M|D|2 yr Degree|5000|Good|2|0|0| +11866|F|D|2 yr Degree|5000|Good|2|0|0| +11867|M|W|2 yr Degree|5000|Good|2|0|0| +11868|F|W|2 yr Degree|5000|Good|2|0|0| +11869|M|U|2 yr Degree|5000|Good|2|0|0| +11870|F|U|2 yr Degree|5000|Good|2|0|0| +11871|M|M|4 yr Degree|5000|Good|2|0|0| +11872|F|M|4 yr Degree|5000|Good|2|0|0| +11873|M|S|4 yr Degree|5000|Good|2|0|0| +11874|F|S|4 yr Degree|5000|Good|2|0|0| +11875|M|D|4 yr Degree|5000|Good|2|0|0| +11876|F|D|4 yr Degree|5000|Good|2|0|0| +11877|M|W|4 yr Degree|5000|Good|2|0|0| +11878|F|W|4 yr Degree|5000|Good|2|0|0| +11879|M|U|4 yr Degree|5000|Good|2|0|0| +11880|F|U|4 yr Degree|5000|Good|2|0|0| +11881|M|M|Advanced Degree|5000|Good|2|0|0| +11882|F|M|Advanced Degree|5000|Good|2|0|0| +11883|M|S|Advanced Degree|5000|Good|2|0|0| +11884|F|S|Advanced Degree|5000|Good|2|0|0| +11885|M|D|Advanced Degree|5000|Good|2|0|0| +11886|F|D|Advanced Degree|5000|Good|2|0|0| +11887|M|W|Advanced Degree|5000|Good|2|0|0| +11888|F|W|Advanced Degree|5000|Good|2|0|0| +11889|M|U|Advanced Degree|5000|Good|2|0|0| +11890|F|U|Advanced Degree|5000|Good|2|0|0| +11891|M|M|Unknown|5000|Good|2|0|0| +11892|F|M|Unknown|5000|Good|2|0|0| +11893|M|S|Unknown|5000|Good|2|0|0| +11894|F|S|Unknown|5000|Good|2|0|0| +11895|M|D|Unknown|5000|Good|2|0|0| +11896|F|D|Unknown|5000|Good|2|0|0| +11897|M|W|Unknown|5000|Good|2|0|0| +11898|F|W|Unknown|5000|Good|2|0|0| +11899|M|U|Unknown|5000|Good|2|0|0| +11900|F|U|Unknown|5000|Good|2|0|0| +11901|M|M|Primary|5500|Good|2|0|0| +11902|F|M|Primary|5500|Good|2|0|0| +11903|M|S|Primary|5500|Good|2|0|0| +11904|F|S|Primary|5500|Good|2|0|0| +11905|M|D|Primary|5500|Good|2|0|0| +11906|F|D|Primary|5500|Good|2|0|0| +11907|M|W|Primary|5500|Good|2|0|0| +11908|F|W|Primary|5500|Good|2|0|0| +11909|M|U|Primary|5500|Good|2|0|0| +11910|F|U|Primary|5500|Good|2|0|0| +11911|M|M|Secondary|5500|Good|2|0|0| +11912|F|M|Secondary|5500|Good|2|0|0| +11913|M|S|Secondary|5500|Good|2|0|0| +11914|F|S|Secondary|5500|Good|2|0|0| +11915|M|D|Secondary|5500|Good|2|0|0| +11916|F|D|Secondary|5500|Good|2|0|0| +11917|M|W|Secondary|5500|Good|2|0|0| +11918|F|W|Secondary|5500|Good|2|0|0| +11919|M|U|Secondary|5500|Good|2|0|0| +11920|F|U|Secondary|5500|Good|2|0|0| +11921|M|M|College|5500|Good|2|0|0| +11922|F|M|College|5500|Good|2|0|0| +11923|M|S|College|5500|Good|2|0|0| +11924|F|S|College|5500|Good|2|0|0| +11925|M|D|College|5500|Good|2|0|0| +11926|F|D|College|5500|Good|2|0|0| +11927|M|W|College|5500|Good|2|0|0| +11928|F|W|College|5500|Good|2|0|0| +11929|M|U|College|5500|Good|2|0|0| +11930|F|U|College|5500|Good|2|0|0| +11931|M|M|2 yr Degree|5500|Good|2|0|0| +11932|F|M|2 yr Degree|5500|Good|2|0|0| +11933|M|S|2 yr Degree|5500|Good|2|0|0| +11934|F|S|2 yr Degree|5500|Good|2|0|0| +11935|M|D|2 yr Degree|5500|Good|2|0|0| +11936|F|D|2 yr Degree|5500|Good|2|0|0| +11937|M|W|2 yr Degree|5500|Good|2|0|0| +11938|F|W|2 yr Degree|5500|Good|2|0|0| +11939|M|U|2 yr Degree|5500|Good|2|0|0| +11940|F|U|2 yr Degree|5500|Good|2|0|0| +11941|M|M|4 yr Degree|5500|Good|2|0|0| +11942|F|M|4 yr Degree|5500|Good|2|0|0| +11943|M|S|4 yr Degree|5500|Good|2|0|0| +11944|F|S|4 yr Degree|5500|Good|2|0|0| +11945|M|D|4 yr Degree|5500|Good|2|0|0| +11946|F|D|4 yr Degree|5500|Good|2|0|0| +11947|M|W|4 yr Degree|5500|Good|2|0|0| +11948|F|W|4 yr Degree|5500|Good|2|0|0| +11949|M|U|4 yr Degree|5500|Good|2|0|0| +11950|F|U|4 yr Degree|5500|Good|2|0|0| +11951|M|M|Advanced Degree|5500|Good|2|0|0| +11952|F|M|Advanced Degree|5500|Good|2|0|0| +11953|M|S|Advanced Degree|5500|Good|2|0|0| +11954|F|S|Advanced Degree|5500|Good|2|0|0| +11955|M|D|Advanced Degree|5500|Good|2|0|0| +11956|F|D|Advanced Degree|5500|Good|2|0|0| +11957|M|W|Advanced Degree|5500|Good|2|0|0| +11958|F|W|Advanced Degree|5500|Good|2|0|0| +11959|M|U|Advanced Degree|5500|Good|2|0|0| +11960|F|U|Advanced Degree|5500|Good|2|0|0| +11961|M|M|Unknown|5500|Good|2|0|0| +11962|F|M|Unknown|5500|Good|2|0|0| +11963|M|S|Unknown|5500|Good|2|0|0| +11964|F|S|Unknown|5500|Good|2|0|0| +11965|M|D|Unknown|5500|Good|2|0|0| +11966|F|D|Unknown|5500|Good|2|0|0| +11967|M|W|Unknown|5500|Good|2|0|0| +11968|F|W|Unknown|5500|Good|2|0|0| +11969|M|U|Unknown|5500|Good|2|0|0| +11970|F|U|Unknown|5500|Good|2|0|0| +11971|M|M|Primary|6000|Good|2|0|0| +11972|F|M|Primary|6000|Good|2|0|0| +11973|M|S|Primary|6000|Good|2|0|0| +11974|F|S|Primary|6000|Good|2|0|0| +11975|M|D|Primary|6000|Good|2|0|0| +11976|F|D|Primary|6000|Good|2|0|0| +11977|M|W|Primary|6000|Good|2|0|0| +11978|F|W|Primary|6000|Good|2|0|0| +11979|M|U|Primary|6000|Good|2|0|0| +11980|F|U|Primary|6000|Good|2|0|0| +11981|M|M|Secondary|6000|Good|2|0|0| +11982|F|M|Secondary|6000|Good|2|0|0| +11983|M|S|Secondary|6000|Good|2|0|0| +11984|F|S|Secondary|6000|Good|2|0|0| +11985|M|D|Secondary|6000|Good|2|0|0| +11986|F|D|Secondary|6000|Good|2|0|0| +11987|M|W|Secondary|6000|Good|2|0|0| +11988|F|W|Secondary|6000|Good|2|0|0| +11989|M|U|Secondary|6000|Good|2|0|0| +11990|F|U|Secondary|6000|Good|2|0|0| +11991|M|M|College|6000|Good|2|0|0| +11992|F|M|College|6000|Good|2|0|0| +11993|M|S|College|6000|Good|2|0|0| +11994|F|S|College|6000|Good|2|0|0| +11995|M|D|College|6000|Good|2|0|0| +11996|F|D|College|6000|Good|2|0|0| +11997|M|W|College|6000|Good|2|0|0| +11998|F|W|College|6000|Good|2|0|0| +11999|M|U|College|6000|Good|2|0|0| +12000|F|U|College|6000|Good|2|0|0| +12001|M|M|2 yr Degree|6000|Good|2|0|0| +12002|F|M|2 yr Degree|6000|Good|2|0|0| +12003|M|S|2 yr Degree|6000|Good|2|0|0| +12004|F|S|2 yr Degree|6000|Good|2|0|0| +12005|M|D|2 yr Degree|6000|Good|2|0|0| +12006|F|D|2 yr Degree|6000|Good|2|0|0| +12007|M|W|2 yr Degree|6000|Good|2|0|0| +12008|F|W|2 yr Degree|6000|Good|2|0|0| +12009|M|U|2 yr Degree|6000|Good|2|0|0| +12010|F|U|2 yr Degree|6000|Good|2|0|0| +12011|M|M|4 yr Degree|6000|Good|2|0|0| +12012|F|M|4 yr Degree|6000|Good|2|0|0| +12013|M|S|4 yr Degree|6000|Good|2|0|0| +12014|F|S|4 yr Degree|6000|Good|2|0|0| +12015|M|D|4 yr Degree|6000|Good|2|0|0| +12016|F|D|4 yr Degree|6000|Good|2|0|0| +12017|M|W|4 yr Degree|6000|Good|2|0|0| +12018|F|W|4 yr Degree|6000|Good|2|0|0| +12019|M|U|4 yr Degree|6000|Good|2|0|0| +12020|F|U|4 yr Degree|6000|Good|2|0|0| +12021|M|M|Advanced Degree|6000|Good|2|0|0| +12022|F|M|Advanced Degree|6000|Good|2|0|0| +12023|M|S|Advanced Degree|6000|Good|2|0|0| +12024|F|S|Advanced Degree|6000|Good|2|0|0| +12025|M|D|Advanced Degree|6000|Good|2|0|0| +12026|F|D|Advanced Degree|6000|Good|2|0|0| +12027|M|W|Advanced Degree|6000|Good|2|0|0| +12028|F|W|Advanced Degree|6000|Good|2|0|0| +12029|M|U|Advanced Degree|6000|Good|2|0|0| +12030|F|U|Advanced Degree|6000|Good|2|0|0| +12031|M|M|Unknown|6000|Good|2|0|0| +12032|F|M|Unknown|6000|Good|2|0|0| +12033|M|S|Unknown|6000|Good|2|0|0| +12034|F|S|Unknown|6000|Good|2|0|0| +12035|M|D|Unknown|6000|Good|2|0|0| +12036|F|D|Unknown|6000|Good|2|0|0| +12037|M|W|Unknown|6000|Good|2|0|0| +12038|F|W|Unknown|6000|Good|2|0|0| +12039|M|U|Unknown|6000|Good|2|0|0| +12040|F|U|Unknown|6000|Good|2|0|0| +12041|M|M|Primary|6500|Good|2|0|0| +12042|F|M|Primary|6500|Good|2|0|0| +12043|M|S|Primary|6500|Good|2|0|0| +12044|F|S|Primary|6500|Good|2|0|0| +12045|M|D|Primary|6500|Good|2|0|0| +12046|F|D|Primary|6500|Good|2|0|0| +12047|M|W|Primary|6500|Good|2|0|0| +12048|F|W|Primary|6500|Good|2|0|0| +12049|M|U|Primary|6500|Good|2|0|0| +12050|F|U|Primary|6500|Good|2|0|0| +12051|M|M|Secondary|6500|Good|2|0|0| +12052|F|M|Secondary|6500|Good|2|0|0| +12053|M|S|Secondary|6500|Good|2|0|0| +12054|F|S|Secondary|6500|Good|2|0|0| +12055|M|D|Secondary|6500|Good|2|0|0| +12056|F|D|Secondary|6500|Good|2|0|0| +12057|M|W|Secondary|6500|Good|2|0|0| +12058|F|W|Secondary|6500|Good|2|0|0| +12059|M|U|Secondary|6500|Good|2|0|0| +12060|F|U|Secondary|6500|Good|2|0|0| +12061|M|M|College|6500|Good|2|0|0| +12062|F|M|College|6500|Good|2|0|0| +12063|M|S|College|6500|Good|2|0|0| +12064|F|S|College|6500|Good|2|0|0| +12065|M|D|College|6500|Good|2|0|0| +12066|F|D|College|6500|Good|2|0|0| +12067|M|W|College|6500|Good|2|0|0| +12068|F|W|College|6500|Good|2|0|0| +12069|M|U|College|6500|Good|2|0|0| +12070|F|U|College|6500|Good|2|0|0| +12071|M|M|2 yr Degree|6500|Good|2|0|0| +12072|F|M|2 yr Degree|6500|Good|2|0|0| +12073|M|S|2 yr Degree|6500|Good|2|0|0| +12074|F|S|2 yr Degree|6500|Good|2|0|0| +12075|M|D|2 yr Degree|6500|Good|2|0|0| +12076|F|D|2 yr Degree|6500|Good|2|0|0| +12077|M|W|2 yr Degree|6500|Good|2|0|0| +12078|F|W|2 yr Degree|6500|Good|2|0|0| +12079|M|U|2 yr Degree|6500|Good|2|0|0| +12080|F|U|2 yr Degree|6500|Good|2|0|0| +12081|M|M|4 yr Degree|6500|Good|2|0|0| +12082|F|M|4 yr Degree|6500|Good|2|0|0| +12083|M|S|4 yr Degree|6500|Good|2|0|0| +12084|F|S|4 yr Degree|6500|Good|2|0|0| +12085|M|D|4 yr Degree|6500|Good|2|0|0| +12086|F|D|4 yr Degree|6500|Good|2|0|0| +12087|M|W|4 yr Degree|6500|Good|2|0|0| +12088|F|W|4 yr Degree|6500|Good|2|0|0| +12089|M|U|4 yr Degree|6500|Good|2|0|0| +12090|F|U|4 yr Degree|6500|Good|2|0|0| +12091|M|M|Advanced Degree|6500|Good|2|0|0| +12092|F|M|Advanced Degree|6500|Good|2|0|0| +12093|M|S|Advanced Degree|6500|Good|2|0|0| +12094|F|S|Advanced Degree|6500|Good|2|0|0| +12095|M|D|Advanced Degree|6500|Good|2|0|0| +12096|F|D|Advanced Degree|6500|Good|2|0|0| +12097|M|W|Advanced Degree|6500|Good|2|0|0| +12098|F|W|Advanced Degree|6500|Good|2|0|0| +12099|M|U|Advanced Degree|6500|Good|2|0|0| +12100|F|U|Advanced Degree|6500|Good|2|0|0| +12101|M|M|Unknown|6500|Good|2|0|0| +12102|F|M|Unknown|6500|Good|2|0|0| +12103|M|S|Unknown|6500|Good|2|0|0| +12104|F|S|Unknown|6500|Good|2|0|0| +12105|M|D|Unknown|6500|Good|2|0|0| +12106|F|D|Unknown|6500|Good|2|0|0| +12107|M|W|Unknown|6500|Good|2|0|0| +12108|F|W|Unknown|6500|Good|2|0|0| +12109|M|U|Unknown|6500|Good|2|0|0| +12110|F|U|Unknown|6500|Good|2|0|0| +12111|M|M|Primary|7000|Good|2|0|0| +12112|F|M|Primary|7000|Good|2|0|0| +12113|M|S|Primary|7000|Good|2|0|0| +12114|F|S|Primary|7000|Good|2|0|0| +12115|M|D|Primary|7000|Good|2|0|0| +12116|F|D|Primary|7000|Good|2|0|0| +12117|M|W|Primary|7000|Good|2|0|0| +12118|F|W|Primary|7000|Good|2|0|0| +12119|M|U|Primary|7000|Good|2|0|0| +12120|F|U|Primary|7000|Good|2|0|0| +12121|M|M|Secondary|7000|Good|2|0|0| +12122|F|M|Secondary|7000|Good|2|0|0| +12123|M|S|Secondary|7000|Good|2|0|0| +12124|F|S|Secondary|7000|Good|2|0|0| +12125|M|D|Secondary|7000|Good|2|0|0| +12126|F|D|Secondary|7000|Good|2|0|0| +12127|M|W|Secondary|7000|Good|2|0|0| +12128|F|W|Secondary|7000|Good|2|0|0| +12129|M|U|Secondary|7000|Good|2|0|0| +12130|F|U|Secondary|7000|Good|2|0|0| +12131|M|M|College|7000|Good|2|0|0| +12132|F|M|College|7000|Good|2|0|0| +12133|M|S|College|7000|Good|2|0|0| +12134|F|S|College|7000|Good|2|0|0| +12135|M|D|College|7000|Good|2|0|0| +12136|F|D|College|7000|Good|2|0|0| +12137|M|W|College|7000|Good|2|0|0| +12138|F|W|College|7000|Good|2|0|0| +12139|M|U|College|7000|Good|2|0|0| +12140|F|U|College|7000|Good|2|0|0| +12141|M|M|2 yr Degree|7000|Good|2|0|0| +12142|F|M|2 yr Degree|7000|Good|2|0|0| +12143|M|S|2 yr Degree|7000|Good|2|0|0| +12144|F|S|2 yr Degree|7000|Good|2|0|0| +12145|M|D|2 yr Degree|7000|Good|2|0|0| +12146|F|D|2 yr Degree|7000|Good|2|0|0| +12147|M|W|2 yr Degree|7000|Good|2|0|0| +12148|F|W|2 yr Degree|7000|Good|2|0|0| +12149|M|U|2 yr Degree|7000|Good|2|0|0| +12150|F|U|2 yr Degree|7000|Good|2|0|0| +12151|M|M|4 yr Degree|7000|Good|2|0|0| +12152|F|M|4 yr Degree|7000|Good|2|0|0| +12153|M|S|4 yr Degree|7000|Good|2|0|0| +12154|F|S|4 yr Degree|7000|Good|2|0|0| +12155|M|D|4 yr Degree|7000|Good|2|0|0| +12156|F|D|4 yr Degree|7000|Good|2|0|0| +12157|M|W|4 yr Degree|7000|Good|2|0|0| +12158|F|W|4 yr Degree|7000|Good|2|0|0| +12159|M|U|4 yr Degree|7000|Good|2|0|0| +12160|F|U|4 yr Degree|7000|Good|2|0|0| +12161|M|M|Advanced Degree|7000|Good|2|0|0| +12162|F|M|Advanced Degree|7000|Good|2|0|0| +12163|M|S|Advanced Degree|7000|Good|2|0|0| +12164|F|S|Advanced Degree|7000|Good|2|0|0| +12165|M|D|Advanced Degree|7000|Good|2|0|0| +12166|F|D|Advanced Degree|7000|Good|2|0|0| +12167|M|W|Advanced Degree|7000|Good|2|0|0| +12168|F|W|Advanced Degree|7000|Good|2|0|0| +12169|M|U|Advanced Degree|7000|Good|2|0|0| +12170|F|U|Advanced Degree|7000|Good|2|0|0| +12171|M|M|Unknown|7000|Good|2|0|0| +12172|F|M|Unknown|7000|Good|2|0|0| +12173|M|S|Unknown|7000|Good|2|0|0| +12174|F|S|Unknown|7000|Good|2|0|0| +12175|M|D|Unknown|7000|Good|2|0|0| +12176|F|D|Unknown|7000|Good|2|0|0| +12177|M|W|Unknown|7000|Good|2|0|0| +12178|F|W|Unknown|7000|Good|2|0|0| +12179|M|U|Unknown|7000|Good|2|0|0| +12180|F|U|Unknown|7000|Good|2|0|0| +12181|M|M|Primary|7500|Good|2|0|0| +12182|F|M|Primary|7500|Good|2|0|0| +12183|M|S|Primary|7500|Good|2|0|0| +12184|F|S|Primary|7500|Good|2|0|0| +12185|M|D|Primary|7500|Good|2|0|0| +12186|F|D|Primary|7500|Good|2|0|0| +12187|M|W|Primary|7500|Good|2|0|0| +12188|F|W|Primary|7500|Good|2|0|0| +12189|M|U|Primary|7500|Good|2|0|0| +12190|F|U|Primary|7500|Good|2|0|0| +12191|M|M|Secondary|7500|Good|2|0|0| +12192|F|M|Secondary|7500|Good|2|0|0| +12193|M|S|Secondary|7500|Good|2|0|0| +12194|F|S|Secondary|7500|Good|2|0|0| +12195|M|D|Secondary|7500|Good|2|0|0| +12196|F|D|Secondary|7500|Good|2|0|0| +12197|M|W|Secondary|7500|Good|2|0|0| +12198|F|W|Secondary|7500|Good|2|0|0| +12199|M|U|Secondary|7500|Good|2|0|0| +12200|F|U|Secondary|7500|Good|2|0|0| +12201|M|M|College|7500|Good|2|0|0| +12202|F|M|College|7500|Good|2|0|0| +12203|M|S|College|7500|Good|2|0|0| +12204|F|S|College|7500|Good|2|0|0| +12205|M|D|College|7500|Good|2|0|0| +12206|F|D|College|7500|Good|2|0|0| +12207|M|W|College|7500|Good|2|0|0| +12208|F|W|College|7500|Good|2|0|0| +12209|M|U|College|7500|Good|2|0|0| +12210|F|U|College|7500|Good|2|0|0| +12211|M|M|2 yr Degree|7500|Good|2|0|0| +12212|F|M|2 yr Degree|7500|Good|2|0|0| +12213|M|S|2 yr Degree|7500|Good|2|0|0| +12214|F|S|2 yr Degree|7500|Good|2|0|0| +12215|M|D|2 yr Degree|7500|Good|2|0|0| +12216|F|D|2 yr Degree|7500|Good|2|0|0| +12217|M|W|2 yr Degree|7500|Good|2|0|0| +12218|F|W|2 yr Degree|7500|Good|2|0|0| +12219|M|U|2 yr Degree|7500|Good|2|0|0| +12220|F|U|2 yr Degree|7500|Good|2|0|0| +12221|M|M|4 yr Degree|7500|Good|2|0|0| +12222|F|M|4 yr Degree|7500|Good|2|0|0| +12223|M|S|4 yr Degree|7500|Good|2|0|0| +12224|F|S|4 yr Degree|7500|Good|2|0|0| +12225|M|D|4 yr Degree|7500|Good|2|0|0| +12226|F|D|4 yr Degree|7500|Good|2|0|0| +12227|M|W|4 yr Degree|7500|Good|2|0|0| +12228|F|W|4 yr Degree|7500|Good|2|0|0| +12229|M|U|4 yr Degree|7500|Good|2|0|0| +12230|F|U|4 yr Degree|7500|Good|2|0|0| +12231|M|M|Advanced Degree|7500|Good|2|0|0| +12232|F|M|Advanced Degree|7500|Good|2|0|0| +12233|M|S|Advanced Degree|7500|Good|2|0|0| +12234|F|S|Advanced Degree|7500|Good|2|0|0| +12235|M|D|Advanced Degree|7500|Good|2|0|0| +12236|F|D|Advanced Degree|7500|Good|2|0|0| +12237|M|W|Advanced Degree|7500|Good|2|0|0| +12238|F|W|Advanced Degree|7500|Good|2|0|0| +12239|M|U|Advanced Degree|7500|Good|2|0|0| +12240|F|U|Advanced Degree|7500|Good|2|0|0| +12241|M|M|Unknown|7500|Good|2|0|0| +12242|F|M|Unknown|7500|Good|2|0|0| +12243|M|S|Unknown|7500|Good|2|0|0| +12244|F|S|Unknown|7500|Good|2|0|0| +12245|M|D|Unknown|7500|Good|2|0|0| +12246|F|D|Unknown|7500|Good|2|0|0| +12247|M|W|Unknown|7500|Good|2|0|0| +12248|F|W|Unknown|7500|Good|2|0|0| +12249|M|U|Unknown|7500|Good|2|0|0| +12250|F|U|Unknown|7500|Good|2|0|0| +12251|M|M|Primary|8000|Good|2|0|0| +12252|F|M|Primary|8000|Good|2|0|0| +12253|M|S|Primary|8000|Good|2|0|0| +12254|F|S|Primary|8000|Good|2|0|0| +12255|M|D|Primary|8000|Good|2|0|0| +12256|F|D|Primary|8000|Good|2|0|0| +12257|M|W|Primary|8000|Good|2|0|0| +12258|F|W|Primary|8000|Good|2|0|0| +12259|M|U|Primary|8000|Good|2|0|0| +12260|F|U|Primary|8000|Good|2|0|0| +12261|M|M|Secondary|8000|Good|2|0|0| +12262|F|M|Secondary|8000|Good|2|0|0| +12263|M|S|Secondary|8000|Good|2|0|0| +12264|F|S|Secondary|8000|Good|2|0|0| +12265|M|D|Secondary|8000|Good|2|0|0| +12266|F|D|Secondary|8000|Good|2|0|0| +12267|M|W|Secondary|8000|Good|2|0|0| +12268|F|W|Secondary|8000|Good|2|0|0| +12269|M|U|Secondary|8000|Good|2|0|0| +12270|F|U|Secondary|8000|Good|2|0|0| +12271|M|M|College|8000|Good|2|0|0| +12272|F|M|College|8000|Good|2|0|0| +12273|M|S|College|8000|Good|2|0|0| +12274|F|S|College|8000|Good|2|0|0| +12275|M|D|College|8000|Good|2|0|0| +12276|F|D|College|8000|Good|2|0|0| +12277|M|W|College|8000|Good|2|0|0| +12278|F|W|College|8000|Good|2|0|0| +12279|M|U|College|8000|Good|2|0|0| +12280|F|U|College|8000|Good|2|0|0| +12281|M|M|2 yr Degree|8000|Good|2|0|0| +12282|F|M|2 yr Degree|8000|Good|2|0|0| +12283|M|S|2 yr Degree|8000|Good|2|0|0| +12284|F|S|2 yr Degree|8000|Good|2|0|0| +12285|M|D|2 yr Degree|8000|Good|2|0|0| +12286|F|D|2 yr Degree|8000|Good|2|0|0| +12287|M|W|2 yr Degree|8000|Good|2|0|0| +12288|F|W|2 yr Degree|8000|Good|2|0|0| +12289|M|U|2 yr Degree|8000|Good|2|0|0| +12290|F|U|2 yr Degree|8000|Good|2|0|0| +12291|M|M|4 yr Degree|8000|Good|2|0|0| +12292|F|M|4 yr Degree|8000|Good|2|0|0| +12293|M|S|4 yr Degree|8000|Good|2|0|0| +12294|F|S|4 yr Degree|8000|Good|2|0|0| +12295|M|D|4 yr Degree|8000|Good|2|0|0| +12296|F|D|4 yr Degree|8000|Good|2|0|0| +12297|M|W|4 yr Degree|8000|Good|2|0|0| +12298|F|W|4 yr Degree|8000|Good|2|0|0| +12299|M|U|4 yr Degree|8000|Good|2|0|0| +12300|F|U|4 yr Degree|8000|Good|2|0|0| +12301|M|M|Advanced Degree|8000|Good|2|0|0| +12302|F|M|Advanced Degree|8000|Good|2|0|0| +12303|M|S|Advanced Degree|8000|Good|2|0|0| +12304|F|S|Advanced Degree|8000|Good|2|0|0| +12305|M|D|Advanced Degree|8000|Good|2|0|0| +12306|F|D|Advanced Degree|8000|Good|2|0|0| +12307|M|W|Advanced Degree|8000|Good|2|0|0| +12308|F|W|Advanced Degree|8000|Good|2|0|0| +12309|M|U|Advanced Degree|8000|Good|2|0|0| +12310|F|U|Advanced Degree|8000|Good|2|0|0| +12311|M|M|Unknown|8000|Good|2|0|0| +12312|F|M|Unknown|8000|Good|2|0|0| +12313|M|S|Unknown|8000|Good|2|0|0| +12314|F|S|Unknown|8000|Good|2|0|0| +12315|M|D|Unknown|8000|Good|2|0|0| +12316|F|D|Unknown|8000|Good|2|0|0| +12317|M|W|Unknown|8000|Good|2|0|0| +12318|F|W|Unknown|8000|Good|2|0|0| +12319|M|U|Unknown|8000|Good|2|0|0| +12320|F|U|Unknown|8000|Good|2|0|0| +12321|M|M|Primary|8500|Good|2|0|0| +12322|F|M|Primary|8500|Good|2|0|0| +12323|M|S|Primary|8500|Good|2|0|0| +12324|F|S|Primary|8500|Good|2|0|0| +12325|M|D|Primary|8500|Good|2|0|0| +12326|F|D|Primary|8500|Good|2|0|0| +12327|M|W|Primary|8500|Good|2|0|0| +12328|F|W|Primary|8500|Good|2|0|0| +12329|M|U|Primary|8500|Good|2|0|0| +12330|F|U|Primary|8500|Good|2|0|0| +12331|M|M|Secondary|8500|Good|2|0|0| +12332|F|M|Secondary|8500|Good|2|0|0| +12333|M|S|Secondary|8500|Good|2|0|0| +12334|F|S|Secondary|8500|Good|2|0|0| +12335|M|D|Secondary|8500|Good|2|0|0| +12336|F|D|Secondary|8500|Good|2|0|0| +12337|M|W|Secondary|8500|Good|2|0|0| +12338|F|W|Secondary|8500|Good|2|0|0| +12339|M|U|Secondary|8500|Good|2|0|0| +12340|F|U|Secondary|8500|Good|2|0|0| +12341|M|M|College|8500|Good|2|0|0| +12342|F|M|College|8500|Good|2|0|0| +12343|M|S|College|8500|Good|2|0|0| +12344|F|S|College|8500|Good|2|0|0| +12345|M|D|College|8500|Good|2|0|0| +12346|F|D|College|8500|Good|2|0|0| +12347|M|W|College|8500|Good|2|0|0| +12348|F|W|College|8500|Good|2|0|0| +12349|M|U|College|8500|Good|2|0|0| +12350|F|U|College|8500|Good|2|0|0| +12351|M|M|2 yr Degree|8500|Good|2|0|0| +12352|F|M|2 yr Degree|8500|Good|2|0|0| +12353|M|S|2 yr Degree|8500|Good|2|0|0| +12354|F|S|2 yr Degree|8500|Good|2|0|0| +12355|M|D|2 yr Degree|8500|Good|2|0|0| +12356|F|D|2 yr Degree|8500|Good|2|0|0| +12357|M|W|2 yr Degree|8500|Good|2|0|0| +12358|F|W|2 yr Degree|8500|Good|2|0|0| +12359|M|U|2 yr Degree|8500|Good|2|0|0| +12360|F|U|2 yr Degree|8500|Good|2|0|0| +12361|M|M|4 yr Degree|8500|Good|2|0|0| +12362|F|M|4 yr Degree|8500|Good|2|0|0| +12363|M|S|4 yr Degree|8500|Good|2|0|0| +12364|F|S|4 yr Degree|8500|Good|2|0|0| +12365|M|D|4 yr Degree|8500|Good|2|0|0| +12366|F|D|4 yr Degree|8500|Good|2|0|0| +12367|M|W|4 yr Degree|8500|Good|2|0|0| +12368|F|W|4 yr Degree|8500|Good|2|0|0| +12369|M|U|4 yr Degree|8500|Good|2|0|0| +12370|F|U|4 yr Degree|8500|Good|2|0|0| +12371|M|M|Advanced Degree|8500|Good|2|0|0| +12372|F|M|Advanced Degree|8500|Good|2|0|0| +12373|M|S|Advanced Degree|8500|Good|2|0|0| +12374|F|S|Advanced Degree|8500|Good|2|0|0| +12375|M|D|Advanced Degree|8500|Good|2|0|0| +12376|F|D|Advanced Degree|8500|Good|2|0|0| +12377|M|W|Advanced Degree|8500|Good|2|0|0| +12378|F|W|Advanced Degree|8500|Good|2|0|0| +12379|M|U|Advanced Degree|8500|Good|2|0|0| +12380|F|U|Advanced Degree|8500|Good|2|0|0| +12381|M|M|Unknown|8500|Good|2|0|0| +12382|F|M|Unknown|8500|Good|2|0|0| +12383|M|S|Unknown|8500|Good|2|0|0| +12384|F|S|Unknown|8500|Good|2|0|0| +12385|M|D|Unknown|8500|Good|2|0|0| +12386|F|D|Unknown|8500|Good|2|0|0| +12387|M|W|Unknown|8500|Good|2|0|0| +12388|F|W|Unknown|8500|Good|2|0|0| +12389|M|U|Unknown|8500|Good|2|0|0| +12390|F|U|Unknown|8500|Good|2|0|0| +12391|M|M|Primary|9000|Good|2|0|0| +12392|F|M|Primary|9000|Good|2|0|0| +12393|M|S|Primary|9000|Good|2|0|0| +12394|F|S|Primary|9000|Good|2|0|0| +12395|M|D|Primary|9000|Good|2|0|0| +12396|F|D|Primary|9000|Good|2|0|0| +12397|M|W|Primary|9000|Good|2|0|0| +12398|F|W|Primary|9000|Good|2|0|0| +12399|M|U|Primary|9000|Good|2|0|0| +12400|F|U|Primary|9000|Good|2|0|0| +12401|M|M|Secondary|9000|Good|2|0|0| +12402|F|M|Secondary|9000|Good|2|0|0| +12403|M|S|Secondary|9000|Good|2|0|0| +12404|F|S|Secondary|9000|Good|2|0|0| +12405|M|D|Secondary|9000|Good|2|0|0| +12406|F|D|Secondary|9000|Good|2|0|0| +12407|M|W|Secondary|9000|Good|2|0|0| +12408|F|W|Secondary|9000|Good|2|0|0| +12409|M|U|Secondary|9000|Good|2|0|0| +12410|F|U|Secondary|9000|Good|2|0|0| +12411|M|M|College|9000|Good|2|0|0| +12412|F|M|College|9000|Good|2|0|0| +12413|M|S|College|9000|Good|2|0|0| +12414|F|S|College|9000|Good|2|0|0| +12415|M|D|College|9000|Good|2|0|0| +12416|F|D|College|9000|Good|2|0|0| +12417|M|W|College|9000|Good|2|0|0| +12418|F|W|College|9000|Good|2|0|0| +12419|M|U|College|9000|Good|2|0|0| +12420|F|U|College|9000|Good|2|0|0| +12421|M|M|2 yr Degree|9000|Good|2|0|0| +12422|F|M|2 yr Degree|9000|Good|2|0|0| +12423|M|S|2 yr Degree|9000|Good|2|0|0| +12424|F|S|2 yr Degree|9000|Good|2|0|0| +12425|M|D|2 yr Degree|9000|Good|2|0|0| +12426|F|D|2 yr Degree|9000|Good|2|0|0| +12427|M|W|2 yr Degree|9000|Good|2|0|0| +12428|F|W|2 yr Degree|9000|Good|2|0|0| +12429|M|U|2 yr Degree|9000|Good|2|0|0| +12430|F|U|2 yr Degree|9000|Good|2|0|0| +12431|M|M|4 yr Degree|9000|Good|2|0|0| +12432|F|M|4 yr Degree|9000|Good|2|0|0| +12433|M|S|4 yr Degree|9000|Good|2|0|0| +12434|F|S|4 yr Degree|9000|Good|2|0|0| +12435|M|D|4 yr Degree|9000|Good|2|0|0| +12436|F|D|4 yr Degree|9000|Good|2|0|0| +12437|M|W|4 yr Degree|9000|Good|2|0|0| +12438|F|W|4 yr Degree|9000|Good|2|0|0| +12439|M|U|4 yr Degree|9000|Good|2|0|0| +12440|F|U|4 yr Degree|9000|Good|2|0|0| +12441|M|M|Advanced Degree|9000|Good|2|0|0| +12442|F|M|Advanced Degree|9000|Good|2|0|0| +12443|M|S|Advanced Degree|9000|Good|2|0|0| +12444|F|S|Advanced Degree|9000|Good|2|0|0| +12445|M|D|Advanced Degree|9000|Good|2|0|0| +12446|F|D|Advanced Degree|9000|Good|2|0|0| +12447|M|W|Advanced Degree|9000|Good|2|0|0| +12448|F|W|Advanced Degree|9000|Good|2|0|0| +12449|M|U|Advanced Degree|9000|Good|2|0|0| +12450|F|U|Advanced Degree|9000|Good|2|0|0| +12451|M|M|Unknown|9000|Good|2|0|0| +12452|F|M|Unknown|9000|Good|2|0|0| +12453|M|S|Unknown|9000|Good|2|0|0| +12454|F|S|Unknown|9000|Good|2|0|0| +12455|M|D|Unknown|9000|Good|2|0|0| +12456|F|D|Unknown|9000|Good|2|0|0| +12457|M|W|Unknown|9000|Good|2|0|0| +12458|F|W|Unknown|9000|Good|2|0|0| +12459|M|U|Unknown|9000|Good|2|0|0| +12460|F|U|Unknown|9000|Good|2|0|0| +12461|M|M|Primary|9500|Good|2|0|0| +12462|F|M|Primary|9500|Good|2|0|0| +12463|M|S|Primary|9500|Good|2|0|0| +12464|F|S|Primary|9500|Good|2|0|0| +12465|M|D|Primary|9500|Good|2|0|0| +12466|F|D|Primary|9500|Good|2|0|0| +12467|M|W|Primary|9500|Good|2|0|0| +12468|F|W|Primary|9500|Good|2|0|0| +12469|M|U|Primary|9500|Good|2|0|0| +12470|F|U|Primary|9500|Good|2|0|0| +12471|M|M|Secondary|9500|Good|2|0|0| +12472|F|M|Secondary|9500|Good|2|0|0| +12473|M|S|Secondary|9500|Good|2|0|0| +12474|F|S|Secondary|9500|Good|2|0|0| +12475|M|D|Secondary|9500|Good|2|0|0| +12476|F|D|Secondary|9500|Good|2|0|0| +12477|M|W|Secondary|9500|Good|2|0|0| +12478|F|W|Secondary|9500|Good|2|0|0| +12479|M|U|Secondary|9500|Good|2|0|0| +12480|F|U|Secondary|9500|Good|2|0|0| +12481|M|M|College|9500|Good|2|0|0| +12482|F|M|College|9500|Good|2|0|0| +12483|M|S|College|9500|Good|2|0|0| +12484|F|S|College|9500|Good|2|0|0| +12485|M|D|College|9500|Good|2|0|0| +12486|F|D|College|9500|Good|2|0|0| +12487|M|W|College|9500|Good|2|0|0| +12488|F|W|College|9500|Good|2|0|0| +12489|M|U|College|9500|Good|2|0|0| +12490|F|U|College|9500|Good|2|0|0| +12491|M|M|2 yr Degree|9500|Good|2|0|0| +12492|F|M|2 yr Degree|9500|Good|2|0|0| +12493|M|S|2 yr Degree|9500|Good|2|0|0| +12494|F|S|2 yr Degree|9500|Good|2|0|0| +12495|M|D|2 yr Degree|9500|Good|2|0|0| +12496|F|D|2 yr Degree|9500|Good|2|0|0| +12497|M|W|2 yr Degree|9500|Good|2|0|0| +12498|F|W|2 yr Degree|9500|Good|2|0|0| +12499|M|U|2 yr Degree|9500|Good|2|0|0| +12500|F|U|2 yr Degree|9500|Good|2|0|0| +12501|M|M|4 yr Degree|9500|Good|2|0|0| +12502|F|M|4 yr Degree|9500|Good|2|0|0| +12503|M|S|4 yr Degree|9500|Good|2|0|0| +12504|F|S|4 yr Degree|9500|Good|2|0|0| +12505|M|D|4 yr Degree|9500|Good|2|0|0| +12506|F|D|4 yr Degree|9500|Good|2|0|0| +12507|M|W|4 yr Degree|9500|Good|2|0|0| +12508|F|W|4 yr Degree|9500|Good|2|0|0| +12509|M|U|4 yr Degree|9500|Good|2|0|0| +12510|F|U|4 yr Degree|9500|Good|2|0|0| +12511|M|M|Advanced Degree|9500|Good|2|0|0| +12512|F|M|Advanced Degree|9500|Good|2|0|0| +12513|M|S|Advanced Degree|9500|Good|2|0|0| +12514|F|S|Advanced Degree|9500|Good|2|0|0| +12515|M|D|Advanced Degree|9500|Good|2|0|0| +12516|F|D|Advanced Degree|9500|Good|2|0|0| +12517|M|W|Advanced Degree|9500|Good|2|0|0| +12518|F|W|Advanced Degree|9500|Good|2|0|0| +12519|M|U|Advanced Degree|9500|Good|2|0|0| +12520|F|U|Advanced Degree|9500|Good|2|0|0| +12521|M|M|Unknown|9500|Good|2|0|0| +12522|F|M|Unknown|9500|Good|2|0|0| +12523|M|S|Unknown|9500|Good|2|0|0| +12524|F|S|Unknown|9500|Good|2|0|0| +12525|M|D|Unknown|9500|Good|2|0|0| +12526|F|D|Unknown|9500|Good|2|0|0| +12527|M|W|Unknown|9500|Good|2|0|0| +12528|F|W|Unknown|9500|Good|2|0|0| +12529|M|U|Unknown|9500|Good|2|0|0| +12530|F|U|Unknown|9500|Good|2|0|0| +12531|M|M|Primary|10000|Good|2|0|0| +12532|F|M|Primary|10000|Good|2|0|0| +12533|M|S|Primary|10000|Good|2|0|0| +12534|F|S|Primary|10000|Good|2|0|0| +12535|M|D|Primary|10000|Good|2|0|0| +12536|F|D|Primary|10000|Good|2|0|0| +12537|M|W|Primary|10000|Good|2|0|0| +12538|F|W|Primary|10000|Good|2|0|0| +12539|M|U|Primary|10000|Good|2|0|0| +12540|F|U|Primary|10000|Good|2|0|0| +12541|M|M|Secondary|10000|Good|2|0|0| +12542|F|M|Secondary|10000|Good|2|0|0| +12543|M|S|Secondary|10000|Good|2|0|0| +12544|F|S|Secondary|10000|Good|2|0|0| +12545|M|D|Secondary|10000|Good|2|0|0| +12546|F|D|Secondary|10000|Good|2|0|0| +12547|M|W|Secondary|10000|Good|2|0|0| +12548|F|W|Secondary|10000|Good|2|0|0| +12549|M|U|Secondary|10000|Good|2|0|0| +12550|F|U|Secondary|10000|Good|2|0|0| +12551|M|M|College|10000|Good|2|0|0| +12552|F|M|College|10000|Good|2|0|0| +12553|M|S|College|10000|Good|2|0|0| +12554|F|S|College|10000|Good|2|0|0| +12555|M|D|College|10000|Good|2|0|0| +12556|F|D|College|10000|Good|2|0|0| +12557|M|W|College|10000|Good|2|0|0| +12558|F|W|College|10000|Good|2|0|0| +12559|M|U|College|10000|Good|2|0|0| +12560|F|U|College|10000|Good|2|0|0| +12561|M|M|2 yr Degree|10000|Good|2|0|0| +12562|F|M|2 yr Degree|10000|Good|2|0|0| +12563|M|S|2 yr Degree|10000|Good|2|0|0| +12564|F|S|2 yr Degree|10000|Good|2|0|0| +12565|M|D|2 yr Degree|10000|Good|2|0|0| +12566|F|D|2 yr Degree|10000|Good|2|0|0| +12567|M|W|2 yr Degree|10000|Good|2|0|0| +12568|F|W|2 yr Degree|10000|Good|2|0|0| +12569|M|U|2 yr Degree|10000|Good|2|0|0| +12570|F|U|2 yr Degree|10000|Good|2|0|0| +12571|M|M|4 yr Degree|10000|Good|2|0|0| +12572|F|M|4 yr Degree|10000|Good|2|0|0| +12573|M|S|4 yr Degree|10000|Good|2|0|0| +12574|F|S|4 yr Degree|10000|Good|2|0|0| +12575|M|D|4 yr Degree|10000|Good|2|0|0| +12576|F|D|4 yr Degree|10000|Good|2|0|0| +12577|M|W|4 yr Degree|10000|Good|2|0|0| +12578|F|W|4 yr Degree|10000|Good|2|0|0| +12579|M|U|4 yr Degree|10000|Good|2|0|0| +12580|F|U|4 yr Degree|10000|Good|2|0|0| +12581|M|M|Advanced Degree|10000|Good|2|0|0| +12582|F|M|Advanced Degree|10000|Good|2|0|0| +12583|M|S|Advanced Degree|10000|Good|2|0|0| +12584|F|S|Advanced Degree|10000|Good|2|0|0| +12585|M|D|Advanced Degree|10000|Good|2|0|0| +12586|F|D|Advanced Degree|10000|Good|2|0|0| +12587|M|W|Advanced Degree|10000|Good|2|0|0| +12588|F|W|Advanced Degree|10000|Good|2|0|0| +12589|M|U|Advanced Degree|10000|Good|2|0|0| +12590|F|U|Advanced Degree|10000|Good|2|0|0| +12591|M|M|Unknown|10000|Good|2|0|0| +12592|F|M|Unknown|10000|Good|2|0|0| +12593|M|S|Unknown|10000|Good|2|0|0| +12594|F|S|Unknown|10000|Good|2|0|0| +12595|M|D|Unknown|10000|Good|2|0|0| +12596|F|D|Unknown|10000|Good|2|0|0| +12597|M|W|Unknown|10000|Good|2|0|0| +12598|F|W|Unknown|10000|Good|2|0|0| +12599|M|U|Unknown|10000|Good|2|0|0| +12600|F|U|Unknown|10000|Good|2|0|0| +12601|M|M|Primary|500|Low Risk|2|0|0| +12602|F|M|Primary|500|Low Risk|2|0|0| +12603|M|S|Primary|500|Low Risk|2|0|0| +12604|F|S|Primary|500|Low Risk|2|0|0| +12605|M|D|Primary|500|Low Risk|2|0|0| +12606|F|D|Primary|500|Low Risk|2|0|0| +12607|M|W|Primary|500|Low Risk|2|0|0| +12608|F|W|Primary|500|Low Risk|2|0|0| +12609|M|U|Primary|500|Low Risk|2|0|0| +12610|F|U|Primary|500|Low Risk|2|0|0| +12611|M|M|Secondary|500|Low Risk|2|0|0| +12612|F|M|Secondary|500|Low Risk|2|0|0| +12613|M|S|Secondary|500|Low Risk|2|0|0| +12614|F|S|Secondary|500|Low Risk|2|0|0| +12615|M|D|Secondary|500|Low Risk|2|0|0| +12616|F|D|Secondary|500|Low Risk|2|0|0| +12617|M|W|Secondary|500|Low Risk|2|0|0| +12618|F|W|Secondary|500|Low Risk|2|0|0| +12619|M|U|Secondary|500|Low Risk|2|0|0| +12620|F|U|Secondary|500|Low Risk|2|0|0| +12621|M|M|College|500|Low Risk|2|0|0| +12622|F|M|College|500|Low Risk|2|0|0| +12623|M|S|College|500|Low Risk|2|0|0| +12624|F|S|College|500|Low Risk|2|0|0| +12625|M|D|College|500|Low Risk|2|0|0| +12626|F|D|College|500|Low Risk|2|0|0| +12627|M|W|College|500|Low Risk|2|0|0| +12628|F|W|College|500|Low Risk|2|0|0| +12629|M|U|College|500|Low Risk|2|0|0| +12630|F|U|College|500|Low Risk|2|0|0| +12631|M|M|2 yr Degree|500|Low Risk|2|0|0| +12632|F|M|2 yr Degree|500|Low Risk|2|0|0| +12633|M|S|2 yr Degree|500|Low Risk|2|0|0| +12634|F|S|2 yr Degree|500|Low Risk|2|0|0| +12635|M|D|2 yr Degree|500|Low Risk|2|0|0| +12636|F|D|2 yr Degree|500|Low Risk|2|0|0| +12637|M|W|2 yr Degree|500|Low Risk|2|0|0| +12638|F|W|2 yr Degree|500|Low Risk|2|0|0| +12639|M|U|2 yr Degree|500|Low Risk|2|0|0| +12640|F|U|2 yr Degree|500|Low Risk|2|0|0| +12641|M|M|4 yr Degree|500|Low Risk|2|0|0| +12642|F|M|4 yr Degree|500|Low Risk|2|0|0| +12643|M|S|4 yr Degree|500|Low Risk|2|0|0| +12644|F|S|4 yr Degree|500|Low Risk|2|0|0| +12645|M|D|4 yr Degree|500|Low Risk|2|0|0| +12646|F|D|4 yr Degree|500|Low Risk|2|0|0| +12647|M|W|4 yr Degree|500|Low Risk|2|0|0| +12648|F|W|4 yr Degree|500|Low Risk|2|0|0| +12649|M|U|4 yr Degree|500|Low Risk|2|0|0| +12650|F|U|4 yr Degree|500|Low Risk|2|0|0| +12651|M|M|Advanced Degree|500|Low Risk|2|0|0| +12652|F|M|Advanced Degree|500|Low Risk|2|0|0| +12653|M|S|Advanced Degree|500|Low Risk|2|0|0| +12654|F|S|Advanced Degree|500|Low Risk|2|0|0| +12655|M|D|Advanced Degree|500|Low Risk|2|0|0| +12656|F|D|Advanced Degree|500|Low Risk|2|0|0| +12657|M|W|Advanced Degree|500|Low Risk|2|0|0| +12658|F|W|Advanced Degree|500|Low Risk|2|0|0| +12659|M|U|Advanced Degree|500|Low Risk|2|0|0| +12660|F|U|Advanced Degree|500|Low Risk|2|0|0| +12661|M|M|Unknown|500|Low Risk|2|0|0| +12662|F|M|Unknown|500|Low Risk|2|0|0| +12663|M|S|Unknown|500|Low Risk|2|0|0| +12664|F|S|Unknown|500|Low Risk|2|0|0| +12665|M|D|Unknown|500|Low Risk|2|0|0| +12666|F|D|Unknown|500|Low Risk|2|0|0| +12667|M|W|Unknown|500|Low Risk|2|0|0| +12668|F|W|Unknown|500|Low Risk|2|0|0| +12669|M|U|Unknown|500|Low Risk|2|0|0| +12670|F|U|Unknown|500|Low Risk|2|0|0| +12671|M|M|Primary|1000|Low Risk|2|0|0| +12672|F|M|Primary|1000|Low Risk|2|0|0| +12673|M|S|Primary|1000|Low Risk|2|0|0| +12674|F|S|Primary|1000|Low Risk|2|0|0| +12675|M|D|Primary|1000|Low Risk|2|0|0| +12676|F|D|Primary|1000|Low Risk|2|0|0| +12677|M|W|Primary|1000|Low Risk|2|0|0| +12678|F|W|Primary|1000|Low Risk|2|0|0| +12679|M|U|Primary|1000|Low Risk|2|0|0| +12680|F|U|Primary|1000|Low Risk|2|0|0| +12681|M|M|Secondary|1000|Low Risk|2|0|0| +12682|F|M|Secondary|1000|Low Risk|2|0|0| +12683|M|S|Secondary|1000|Low Risk|2|0|0| +12684|F|S|Secondary|1000|Low Risk|2|0|0| +12685|M|D|Secondary|1000|Low Risk|2|0|0| +12686|F|D|Secondary|1000|Low Risk|2|0|0| +12687|M|W|Secondary|1000|Low Risk|2|0|0| +12688|F|W|Secondary|1000|Low Risk|2|0|0| +12689|M|U|Secondary|1000|Low Risk|2|0|0| +12690|F|U|Secondary|1000|Low Risk|2|0|0| +12691|M|M|College|1000|Low Risk|2|0|0| +12692|F|M|College|1000|Low Risk|2|0|0| +12693|M|S|College|1000|Low Risk|2|0|0| +12694|F|S|College|1000|Low Risk|2|0|0| +12695|M|D|College|1000|Low Risk|2|0|0| +12696|F|D|College|1000|Low Risk|2|0|0| +12697|M|W|College|1000|Low Risk|2|0|0| +12698|F|W|College|1000|Low Risk|2|0|0| +12699|M|U|College|1000|Low Risk|2|0|0| +12700|F|U|College|1000|Low Risk|2|0|0| +12701|M|M|2 yr Degree|1000|Low Risk|2|0|0| +12702|F|M|2 yr Degree|1000|Low Risk|2|0|0| +12703|M|S|2 yr Degree|1000|Low Risk|2|0|0| +12704|F|S|2 yr Degree|1000|Low Risk|2|0|0| +12705|M|D|2 yr Degree|1000|Low Risk|2|0|0| +12706|F|D|2 yr Degree|1000|Low Risk|2|0|0| +12707|M|W|2 yr Degree|1000|Low Risk|2|0|0| +12708|F|W|2 yr Degree|1000|Low Risk|2|0|0| +12709|M|U|2 yr Degree|1000|Low Risk|2|0|0| +12710|F|U|2 yr Degree|1000|Low Risk|2|0|0| +12711|M|M|4 yr Degree|1000|Low Risk|2|0|0| +12712|F|M|4 yr Degree|1000|Low Risk|2|0|0| +12713|M|S|4 yr Degree|1000|Low Risk|2|0|0| +12714|F|S|4 yr Degree|1000|Low Risk|2|0|0| +12715|M|D|4 yr Degree|1000|Low Risk|2|0|0| +12716|F|D|4 yr Degree|1000|Low Risk|2|0|0| +12717|M|W|4 yr Degree|1000|Low Risk|2|0|0| +12718|F|W|4 yr Degree|1000|Low Risk|2|0|0| +12719|M|U|4 yr Degree|1000|Low Risk|2|0|0| +12720|F|U|4 yr Degree|1000|Low Risk|2|0|0| +12721|M|M|Advanced Degree|1000|Low Risk|2|0|0| +12722|F|M|Advanced Degree|1000|Low Risk|2|0|0| +12723|M|S|Advanced Degree|1000|Low Risk|2|0|0| +12724|F|S|Advanced Degree|1000|Low Risk|2|0|0| +12725|M|D|Advanced Degree|1000|Low Risk|2|0|0| +12726|F|D|Advanced Degree|1000|Low Risk|2|0|0| +12727|M|W|Advanced Degree|1000|Low Risk|2|0|0| +12728|F|W|Advanced Degree|1000|Low Risk|2|0|0| +12729|M|U|Advanced Degree|1000|Low Risk|2|0|0| +12730|F|U|Advanced Degree|1000|Low Risk|2|0|0| +12731|M|M|Unknown|1000|Low Risk|2|0|0| +12732|F|M|Unknown|1000|Low Risk|2|0|0| +12733|M|S|Unknown|1000|Low Risk|2|0|0| +12734|F|S|Unknown|1000|Low Risk|2|0|0| +12735|M|D|Unknown|1000|Low Risk|2|0|0| +12736|F|D|Unknown|1000|Low Risk|2|0|0| +12737|M|W|Unknown|1000|Low Risk|2|0|0| +12738|F|W|Unknown|1000|Low Risk|2|0|0| +12739|M|U|Unknown|1000|Low Risk|2|0|0| +12740|F|U|Unknown|1000|Low Risk|2|0|0| +12741|M|M|Primary|1500|Low Risk|2|0|0| +12742|F|M|Primary|1500|Low Risk|2|0|0| +12743|M|S|Primary|1500|Low Risk|2|0|0| +12744|F|S|Primary|1500|Low Risk|2|0|0| +12745|M|D|Primary|1500|Low Risk|2|0|0| +12746|F|D|Primary|1500|Low Risk|2|0|0| +12747|M|W|Primary|1500|Low Risk|2|0|0| +12748|F|W|Primary|1500|Low Risk|2|0|0| +12749|M|U|Primary|1500|Low Risk|2|0|0| +12750|F|U|Primary|1500|Low Risk|2|0|0| +12751|M|M|Secondary|1500|Low Risk|2|0|0| +12752|F|M|Secondary|1500|Low Risk|2|0|0| +12753|M|S|Secondary|1500|Low Risk|2|0|0| +12754|F|S|Secondary|1500|Low Risk|2|0|0| +12755|M|D|Secondary|1500|Low Risk|2|0|0| +12756|F|D|Secondary|1500|Low Risk|2|0|0| +12757|M|W|Secondary|1500|Low Risk|2|0|0| +12758|F|W|Secondary|1500|Low Risk|2|0|0| +12759|M|U|Secondary|1500|Low Risk|2|0|0| +12760|F|U|Secondary|1500|Low Risk|2|0|0| +12761|M|M|College|1500|Low Risk|2|0|0| +12762|F|M|College|1500|Low Risk|2|0|0| +12763|M|S|College|1500|Low Risk|2|0|0| +12764|F|S|College|1500|Low Risk|2|0|0| +12765|M|D|College|1500|Low Risk|2|0|0| +12766|F|D|College|1500|Low Risk|2|0|0| +12767|M|W|College|1500|Low Risk|2|0|0| +12768|F|W|College|1500|Low Risk|2|0|0| +12769|M|U|College|1500|Low Risk|2|0|0| +12770|F|U|College|1500|Low Risk|2|0|0| +12771|M|M|2 yr Degree|1500|Low Risk|2|0|0| +12772|F|M|2 yr Degree|1500|Low Risk|2|0|0| +12773|M|S|2 yr Degree|1500|Low Risk|2|0|0| +12774|F|S|2 yr Degree|1500|Low Risk|2|0|0| +12775|M|D|2 yr Degree|1500|Low Risk|2|0|0| +12776|F|D|2 yr Degree|1500|Low Risk|2|0|0| +12777|M|W|2 yr Degree|1500|Low Risk|2|0|0| +12778|F|W|2 yr Degree|1500|Low Risk|2|0|0| +12779|M|U|2 yr Degree|1500|Low Risk|2|0|0| +12780|F|U|2 yr Degree|1500|Low Risk|2|0|0| +12781|M|M|4 yr Degree|1500|Low Risk|2|0|0| +12782|F|M|4 yr Degree|1500|Low Risk|2|0|0| +12783|M|S|4 yr Degree|1500|Low Risk|2|0|0| +12784|F|S|4 yr Degree|1500|Low Risk|2|0|0| +12785|M|D|4 yr Degree|1500|Low Risk|2|0|0| +12786|F|D|4 yr Degree|1500|Low Risk|2|0|0| +12787|M|W|4 yr Degree|1500|Low Risk|2|0|0| +12788|F|W|4 yr Degree|1500|Low Risk|2|0|0| +12789|M|U|4 yr Degree|1500|Low Risk|2|0|0| +12790|F|U|4 yr Degree|1500|Low Risk|2|0|0| +12791|M|M|Advanced Degree|1500|Low Risk|2|0|0| +12792|F|M|Advanced Degree|1500|Low Risk|2|0|0| +12793|M|S|Advanced Degree|1500|Low Risk|2|0|0| +12794|F|S|Advanced Degree|1500|Low Risk|2|0|0| +12795|M|D|Advanced Degree|1500|Low Risk|2|0|0| +12796|F|D|Advanced Degree|1500|Low Risk|2|0|0| +12797|M|W|Advanced Degree|1500|Low Risk|2|0|0| +12798|F|W|Advanced Degree|1500|Low Risk|2|0|0| +12799|M|U|Advanced Degree|1500|Low Risk|2|0|0| +12800|F|U|Advanced Degree|1500|Low Risk|2|0|0| +12801|M|M|Unknown|1500|Low Risk|2|0|0| +12802|F|M|Unknown|1500|Low Risk|2|0|0| +12803|M|S|Unknown|1500|Low Risk|2|0|0| +12804|F|S|Unknown|1500|Low Risk|2|0|0| +12805|M|D|Unknown|1500|Low Risk|2|0|0| +12806|F|D|Unknown|1500|Low Risk|2|0|0| +12807|M|W|Unknown|1500|Low Risk|2|0|0| +12808|F|W|Unknown|1500|Low Risk|2|0|0| +12809|M|U|Unknown|1500|Low Risk|2|0|0| +12810|F|U|Unknown|1500|Low Risk|2|0|0| +12811|M|M|Primary|2000|Low Risk|2|0|0| +12812|F|M|Primary|2000|Low Risk|2|0|0| +12813|M|S|Primary|2000|Low Risk|2|0|0| +12814|F|S|Primary|2000|Low Risk|2|0|0| +12815|M|D|Primary|2000|Low Risk|2|0|0| +12816|F|D|Primary|2000|Low Risk|2|0|0| +12817|M|W|Primary|2000|Low Risk|2|0|0| +12818|F|W|Primary|2000|Low Risk|2|0|0| +12819|M|U|Primary|2000|Low Risk|2|0|0| +12820|F|U|Primary|2000|Low Risk|2|0|0| +12821|M|M|Secondary|2000|Low Risk|2|0|0| +12822|F|M|Secondary|2000|Low Risk|2|0|0| +12823|M|S|Secondary|2000|Low Risk|2|0|0| +12824|F|S|Secondary|2000|Low Risk|2|0|0| +12825|M|D|Secondary|2000|Low Risk|2|0|0| +12826|F|D|Secondary|2000|Low Risk|2|0|0| +12827|M|W|Secondary|2000|Low Risk|2|0|0| +12828|F|W|Secondary|2000|Low Risk|2|0|0| +12829|M|U|Secondary|2000|Low Risk|2|0|0| +12830|F|U|Secondary|2000|Low Risk|2|0|0| +12831|M|M|College|2000|Low Risk|2|0|0| +12832|F|M|College|2000|Low Risk|2|0|0| +12833|M|S|College|2000|Low Risk|2|0|0| +12834|F|S|College|2000|Low Risk|2|0|0| +12835|M|D|College|2000|Low Risk|2|0|0| +12836|F|D|College|2000|Low Risk|2|0|0| +12837|M|W|College|2000|Low Risk|2|0|0| +12838|F|W|College|2000|Low Risk|2|0|0| +12839|M|U|College|2000|Low Risk|2|0|0| +12840|F|U|College|2000|Low Risk|2|0|0| +12841|M|M|2 yr Degree|2000|Low Risk|2|0|0| +12842|F|M|2 yr Degree|2000|Low Risk|2|0|0| +12843|M|S|2 yr Degree|2000|Low Risk|2|0|0| +12844|F|S|2 yr Degree|2000|Low Risk|2|0|0| +12845|M|D|2 yr Degree|2000|Low Risk|2|0|0| +12846|F|D|2 yr Degree|2000|Low Risk|2|0|0| +12847|M|W|2 yr Degree|2000|Low Risk|2|0|0| +12848|F|W|2 yr Degree|2000|Low Risk|2|0|0| +12849|M|U|2 yr Degree|2000|Low Risk|2|0|0| +12850|F|U|2 yr Degree|2000|Low Risk|2|0|0| +12851|M|M|4 yr Degree|2000|Low Risk|2|0|0| +12852|F|M|4 yr Degree|2000|Low Risk|2|0|0| +12853|M|S|4 yr Degree|2000|Low Risk|2|0|0| +12854|F|S|4 yr Degree|2000|Low Risk|2|0|0| +12855|M|D|4 yr Degree|2000|Low Risk|2|0|0| +12856|F|D|4 yr Degree|2000|Low Risk|2|0|0| +12857|M|W|4 yr Degree|2000|Low Risk|2|0|0| +12858|F|W|4 yr Degree|2000|Low Risk|2|0|0| +12859|M|U|4 yr Degree|2000|Low Risk|2|0|0| +12860|F|U|4 yr Degree|2000|Low Risk|2|0|0| +12861|M|M|Advanced Degree|2000|Low Risk|2|0|0| +12862|F|M|Advanced Degree|2000|Low Risk|2|0|0| +12863|M|S|Advanced Degree|2000|Low Risk|2|0|0| +12864|F|S|Advanced Degree|2000|Low Risk|2|0|0| +12865|M|D|Advanced Degree|2000|Low Risk|2|0|0| +12866|F|D|Advanced Degree|2000|Low Risk|2|0|0| +12867|M|W|Advanced Degree|2000|Low Risk|2|0|0| +12868|F|W|Advanced Degree|2000|Low Risk|2|0|0| +12869|M|U|Advanced Degree|2000|Low Risk|2|0|0| +12870|F|U|Advanced Degree|2000|Low Risk|2|0|0| +12871|M|M|Unknown|2000|Low Risk|2|0|0| +12872|F|M|Unknown|2000|Low Risk|2|0|0| +12873|M|S|Unknown|2000|Low Risk|2|0|0| +12874|F|S|Unknown|2000|Low Risk|2|0|0| +12875|M|D|Unknown|2000|Low Risk|2|0|0| +12876|F|D|Unknown|2000|Low Risk|2|0|0| +12877|M|W|Unknown|2000|Low Risk|2|0|0| +12878|F|W|Unknown|2000|Low Risk|2|0|0| +12879|M|U|Unknown|2000|Low Risk|2|0|0| +12880|F|U|Unknown|2000|Low Risk|2|0|0| +12881|M|M|Primary|2500|Low Risk|2|0|0| +12882|F|M|Primary|2500|Low Risk|2|0|0| +12883|M|S|Primary|2500|Low Risk|2|0|0| +12884|F|S|Primary|2500|Low Risk|2|0|0| +12885|M|D|Primary|2500|Low Risk|2|0|0| +12886|F|D|Primary|2500|Low Risk|2|0|0| +12887|M|W|Primary|2500|Low Risk|2|0|0| +12888|F|W|Primary|2500|Low Risk|2|0|0| +12889|M|U|Primary|2500|Low Risk|2|0|0| +12890|F|U|Primary|2500|Low Risk|2|0|0| +12891|M|M|Secondary|2500|Low Risk|2|0|0| +12892|F|M|Secondary|2500|Low Risk|2|0|0| +12893|M|S|Secondary|2500|Low Risk|2|0|0| +12894|F|S|Secondary|2500|Low Risk|2|0|0| +12895|M|D|Secondary|2500|Low Risk|2|0|0| +12896|F|D|Secondary|2500|Low Risk|2|0|0| +12897|M|W|Secondary|2500|Low Risk|2|0|0| +12898|F|W|Secondary|2500|Low Risk|2|0|0| +12899|M|U|Secondary|2500|Low Risk|2|0|0| +12900|F|U|Secondary|2500|Low Risk|2|0|0| +12901|M|M|College|2500|Low Risk|2|0|0| +12902|F|M|College|2500|Low Risk|2|0|0| +12903|M|S|College|2500|Low Risk|2|0|0| +12904|F|S|College|2500|Low Risk|2|0|0| +12905|M|D|College|2500|Low Risk|2|0|0| +12906|F|D|College|2500|Low Risk|2|0|0| +12907|M|W|College|2500|Low Risk|2|0|0| +12908|F|W|College|2500|Low Risk|2|0|0| +12909|M|U|College|2500|Low Risk|2|0|0| +12910|F|U|College|2500|Low Risk|2|0|0| +12911|M|M|2 yr Degree|2500|Low Risk|2|0|0| +12912|F|M|2 yr Degree|2500|Low Risk|2|0|0| +12913|M|S|2 yr Degree|2500|Low Risk|2|0|0| +12914|F|S|2 yr Degree|2500|Low Risk|2|0|0| +12915|M|D|2 yr Degree|2500|Low Risk|2|0|0| +12916|F|D|2 yr Degree|2500|Low Risk|2|0|0| +12917|M|W|2 yr Degree|2500|Low Risk|2|0|0| +12918|F|W|2 yr Degree|2500|Low Risk|2|0|0| +12919|M|U|2 yr Degree|2500|Low Risk|2|0|0| +12920|F|U|2 yr Degree|2500|Low Risk|2|0|0| +12921|M|M|4 yr Degree|2500|Low Risk|2|0|0| +12922|F|M|4 yr Degree|2500|Low Risk|2|0|0| +12923|M|S|4 yr Degree|2500|Low Risk|2|0|0| +12924|F|S|4 yr Degree|2500|Low Risk|2|0|0| +12925|M|D|4 yr Degree|2500|Low Risk|2|0|0| +12926|F|D|4 yr Degree|2500|Low Risk|2|0|0| +12927|M|W|4 yr Degree|2500|Low Risk|2|0|0| +12928|F|W|4 yr Degree|2500|Low Risk|2|0|0| +12929|M|U|4 yr Degree|2500|Low Risk|2|0|0| +12930|F|U|4 yr Degree|2500|Low Risk|2|0|0| +12931|M|M|Advanced Degree|2500|Low Risk|2|0|0| +12932|F|M|Advanced Degree|2500|Low Risk|2|0|0| +12933|M|S|Advanced Degree|2500|Low Risk|2|0|0| +12934|F|S|Advanced Degree|2500|Low Risk|2|0|0| +12935|M|D|Advanced Degree|2500|Low Risk|2|0|0| +12936|F|D|Advanced Degree|2500|Low Risk|2|0|0| +12937|M|W|Advanced Degree|2500|Low Risk|2|0|0| +12938|F|W|Advanced Degree|2500|Low Risk|2|0|0| +12939|M|U|Advanced Degree|2500|Low Risk|2|0|0| +12940|F|U|Advanced Degree|2500|Low Risk|2|0|0| +12941|M|M|Unknown|2500|Low Risk|2|0|0| +12942|F|M|Unknown|2500|Low Risk|2|0|0| +12943|M|S|Unknown|2500|Low Risk|2|0|0| +12944|F|S|Unknown|2500|Low Risk|2|0|0| +12945|M|D|Unknown|2500|Low Risk|2|0|0| +12946|F|D|Unknown|2500|Low Risk|2|0|0| +12947|M|W|Unknown|2500|Low Risk|2|0|0| +12948|F|W|Unknown|2500|Low Risk|2|0|0| +12949|M|U|Unknown|2500|Low Risk|2|0|0| +12950|F|U|Unknown|2500|Low Risk|2|0|0| +12951|M|M|Primary|3000|Low Risk|2|0|0| +12952|F|M|Primary|3000|Low Risk|2|0|0| +12953|M|S|Primary|3000|Low Risk|2|0|0| +12954|F|S|Primary|3000|Low Risk|2|0|0| +12955|M|D|Primary|3000|Low Risk|2|0|0| +12956|F|D|Primary|3000|Low Risk|2|0|0| +12957|M|W|Primary|3000|Low Risk|2|0|0| +12958|F|W|Primary|3000|Low Risk|2|0|0| +12959|M|U|Primary|3000|Low Risk|2|0|0| +12960|F|U|Primary|3000|Low Risk|2|0|0| +12961|M|M|Secondary|3000|Low Risk|2|0|0| +12962|F|M|Secondary|3000|Low Risk|2|0|0| +12963|M|S|Secondary|3000|Low Risk|2|0|0| +12964|F|S|Secondary|3000|Low Risk|2|0|0| +12965|M|D|Secondary|3000|Low Risk|2|0|0| +12966|F|D|Secondary|3000|Low Risk|2|0|0| +12967|M|W|Secondary|3000|Low Risk|2|0|0| +12968|F|W|Secondary|3000|Low Risk|2|0|0| +12969|M|U|Secondary|3000|Low Risk|2|0|0| +12970|F|U|Secondary|3000|Low Risk|2|0|0| +12971|M|M|College|3000|Low Risk|2|0|0| +12972|F|M|College|3000|Low Risk|2|0|0| +12973|M|S|College|3000|Low Risk|2|0|0| +12974|F|S|College|3000|Low Risk|2|0|0| +12975|M|D|College|3000|Low Risk|2|0|0| +12976|F|D|College|3000|Low Risk|2|0|0| +12977|M|W|College|3000|Low Risk|2|0|0| +12978|F|W|College|3000|Low Risk|2|0|0| +12979|M|U|College|3000|Low Risk|2|0|0| +12980|F|U|College|3000|Low Risk|2|0|0| +12981|M|M|2 yr Degree|3000|Low Risk|2|0|0| +12982|F|M|2 yr Degree|3000|Low Risk|2|0|0| +12983|M|S|2 yr Degree|3000|Low Risk|2|0|0| +12984|F|S|2 yr Degree|3000|Low Risk|2|0|0| +12985|M|D|2 yr Degree|3000|Low Risk|2|0|0| +12986|F|D|2 yr Degree|3000|Low Risk|2|0|0| +12987|M|W|2 yr Degree|3000|Low Risk|2|0|0| +12988|F|W|2 yr Degree|3000|Low Risk|2|0|0| +12989|M|U|2 yr Degree|3000|Low Risk|2|0|0| +12990|F|U|2 yr Degree|3000|Low Risk|2|0|0| +12991|M|M|4 yr Degree|3000|Low Risk|2|0|0| +12992|F|M|4 yr Degree|3000|Low Risk|2|0|0| +12993|M|S|4 yr Degree|3000|Low Risk|2|0|0| +12994|F|S|4 yr Degree|3000|Low Risk|2|0|0| +12995|M|D|4 yr Degree|3000|Low Risk|2|0|0| +12996|F|D|4 yr Degree|3000|Low Risk|2|0|0| +12997|M|W|4 yr Degree|3000|Low Risk|2|0|0| +12998|F|W|4 yr Degree|3000|Low Risk|2|0|0| +12999|M|U|4 yr Degree|3000|Low Risk|2|0|0| +13000|F|U|4 yr Degree|3000|Low Risk|2|0|0| +13001|M|M|Advanced Degree|3000|Low Risk|2|0|0| +13002|F|M|Advanced Degree|3000|Low Risk|2|0|0| +13003|M|S|Advanced Degree|3000|Low Risk|2|0|0| +13004|F|S|Advanced Degree|3000|Low Risk|2|0|0| +13005|M|D|Advanced Degree|3000|Low Risk|2|0|0| +13006|F|D|Advanced Degree|3000|Low Risk|2|0|0| +13007|M|W|Advanced Degree|3000|Low Risk|2|0|0| +13008|F|W|Advanced Degree|3000|Low Risk|2|0|0| +13009|M|U|Advanced Degree|3000|Low Risk|2|0|0| +13010|F|U|Advanced Degree|3000|Low Risk|2|0|0| +13011|M|M|Unknown|3000|Low Risk|2|0|0| +13012|F|M|Unknown|3000|Low Risk|2|0|0| +13013|M|S|Unknown|3000|Low Risk|2|0|0| +13014|F|S|Unknown|3000|Low Risk|2|0|0| +13015|M|D|Unknown|3000|Low Risk|2|0|0| +13016|F|D|Unknown|3000|Low Risk|2|0|0| +13017|M|W|Unknown|3000|Low Risk|2|0|0| +13018|F|W|Unknown|3000|Low Risk|2|0|0| +13019|M|U|Unknown|3000|Low Risk|2|0|0| +13020|F|U|Unknown|3000|Low Risk|2|0|0| +13021|M|M|Primary|3500|Low Risk|2|0|0| +13022|F|M|Primary|3500|Low Risk|2|0|0| +13023|M|S|Primary|3500|Low Risk|2|0|0| +13024|F|S|Primary|3500|Low Risk|2|0|0| +13025|M|D|Primary|3500|Low Risk|2|0|0| +13026|F|D|Primary|3500|Low Risk|2|0|0| +13027|M|W|Primary|3500|Low Risk|2|0|0| +13028|F|W|Primary|3500|Low Risk|2|0|0| +13029|M|U|Primary|3500|Low Risk|2|0|0| +13030|F|U|Primary|3500|Low Risk|2|0|0| +13031|M|M|Secondary|3500|Low Risk|2|0|0| +13032|F|M|Secondary|3500|Low Risk|2|0|0| +13033|M|S|Secondary|3500|Low Risk|2|0|0| +13034|F|S|Secondary|3500|Low Risk|2|0|0| +13035|M|D|Secondary|3500|Low Risk|2|0|0| +13036|F|D|Secondary|3500|Low Risk|2|0|0| +13037|M|W|Secondary|3500|Low Risk|2|0|0| +13038|F|W|Secondary|3500|Low Risk|2|0|0| +13039|M|U|Secondary|3500|Low Risk|2|0|0| +13040|F|U|Secondary|3500|Low Risk|2|0|0| +13041|M|M|College|3500|Low Risk|2|0|0| +13042|F|M|College|3500|Low Risk|2|0|0| +13043|M|S|College|3500|Low Risk|2|0|0| +13044|F|S|College|3500|Low Risk|2|0|0| +13045|M|D|College|3500|Low Risk|2|0|0| +13046|F|D|College|3500|Low Risk|2|0|0| +13047|M|W|College|3500|Low Risk|2|0|0| +13048|F|W|College|3500|Low Risk|2|0|0| +13049|M|U|College|3500|Low Risk|2|0|0| +13050|F|U|College|3500|Low Risk|2|0|0| +13051|M|M|2 yr Degree|3500|Low Risk|2|0|0| +13052|F|M|2 yr Degree|3500|Low Risk|2|0|0| +13053|M|S|2 yr Degree|3500|Low Risk|2|0|0| +13054|F|S|2 yr Degree|3500|Low Risk|2|0|0| +13055|M|D|2 yr Degree|3500|Low Risk|2|0|0| +13056|F|D|2 yr Degree|3500|Low Risk|2|0|0| +13057|M|W|2 yr Degree|3500|Low Risk|2|0|0| +13058|F|W|2 yr Degree|3500|Low Risk|2|0|0| +13059|M|U|2 yr Degree|3500|Low Risk|2|0|0| +13060|F|U|2 yr Degree|3500|Low Risk|2|0|0| +13061|M|M|4 yr Degree|3500|Low Risk|2|0|0| +13062|F|M|4 yr Degree|3500|Low Risk|2|0|0| +13063|M|S|4 yr Degree|3500|Low Risk|2|0|0| +13064|F|S|4 yr Degree|3500|Low Risk|2|0|0| +13065|M|D|4 yr Degree|3500|Low Risk|2|0|0| +13066|F|D|4 yr Degree|3500|Low Risk|2|0|0| +13067|M|W|4 yr Degree|3500|Low Risk|2|0|0| +13068|F|W|4 yr Degree|3500|Low Risk|2|0|0| +13069|M|U|4 yr Degree|3500|Low Risk|2|0|0| +13070|F|U|4 yr Degree|3500|Low Risk|2|0|0| +13071|M|M|Advanced Degree|3500|Low Risk|2|0|0| +13072|F|M|Advanced Degree|3500|Low Risk|2|0|0| +13073|M|S|Advanced Degree|3500|Low Risk|2|0|0| +13074|F|S|Advanced Degree|3500|Low Risk|2|0|0| +13075|M|D|Advanced Degree|3500|Low Risk|2|0|0| +13076|F|D|Advanced Degree|3500|Low Risk|2|0|0| +13077|M|W|Advanced Degree|3500|Low Risk|2|0|0| +13078|F|W|Advanced Degree|3500|Low Risk|2|0|0| +13079|M|U|Advanced Degree|3500|Low Risk|2|0|0| +13080|F|U|Advanced Degree|3500|Low Risk|2|0|0| +13081|M|M|Unknown|3500|Low Risk|2|0|0| +13082|F|M|Unknown|3500|Low Risk|2|0|0| +13083|M|S|Unknown|3500|Low Risk|2|0|0| +13084|F|S|Unknown|3500|Low Risk|2|0|0| +13085|M|D|Unknown|3500|Low Risk|2|0|0| +13086|F|D|Unknown|3500|Low Risk|2|0|0| +13087|M|W|Unknown|3500|Low Risk|2|0|0| +13088|F|W|Unknown|3500|Low Risk|2|0|0| +13089|M|U|Unknown|3500|Low Risk|2|0|0| +13090|F|U|Unknown|3500|Low Risk|2|0|0| +13091|M|M|Primary|4000|Low Risk|2|0|0| +13092|F|M|Primary|4000|Low Risk|2|0|0| +13093|M|S|Primary|4000|Low Risk|2|0|0| +13094|F|S|Primary|4000|Low Risk|2|0|0| +13095|M|D|Primary|4000|Low Risk|2|0|0| +13096|F|D|Primary|4000|Low Risk|2|0|0| +13097|M|W|Primary|4000|Low Risk|2|0|0| +13098|F|W|Primary|4000|Low Risk|2|0|0| +13099|M|U|Primary|4000|Low Risk|2|0|0| +13100|F|U|Primary|4000|Low Risk|2|0|0| +13101|M|M|Secondary|4000|Low Risk|2|0|0| +13102|F|M|Secondary|4000|Low Risk|2|0|0| +13103|M|S|Secondary|4000|Low Risk|2|0|0| +13104|F|S|Secondary|4000|Low Risk|2|0|0| +13105|M|D|Secondary|4000|Low Risk|2|0|0| +13106|F|D|Secondary|4000|Low Risk|2|0|0| +13107|M|W|Secondary|4000|Low Risk|2|0|0| +13108|F|W|Secondary|4000|Low Risk|2|0|0| +13109|M|U|Secondary|4000|Low Risk|2|0|0| +13110|F|U|Secondary|4000|Low Risk|2|0|0| +13111|M|M|College|4000|Low Risk|2|0|0| +13112|F|M|College|4000|Low Risk|2|0|0| +13113|M|S|College|4000|Low Risk|2|0|0| +13114|F|S|College|4000|Low Risk|2|0|0| +13115|M|D|College|4000|Low Risk|2|0|0| +13116|F|D|College|4000|Low Risk|2|0|0| +13117|M|W|College|4000|Low Risk|2|0|0| +13118|F|W|College|4000|Low Risk|2|0|0| +13119|M|U|College|4000|Low Risk|2|0|0| +13120|F|U|College|4000|Low Risk|2|0|0| +13121|M|M|2 yr Degree|4000|Low Risk|2|0|0| +13122|F|M|2 yr Degree|4000|Low Risk|2|0|0| +13123|M|S|2 yr Degree|4000|Low Risk|2|0|0| +13124|F|S|2 yr Degree|4000|Low Risk|2|0|0| +13125|M|D|2 yr Degree|4000|Low Risk|2|0|0| +13126|F|D|2 yr Degree|4000|Low Risk|2|0|0| +13127|M|W|2 yr Degree|4000|Low Risk|2|0|0| +13128|F|W|2 yr Degree|4000|Low Risk|2|0|0| +13129|M|U|2 yr Degree|4000|Low Risk|2|0|0| +13130|F|U|2 yr Degree|4000|Low Risk|2|0|0| +13131|M|M|4 yr Degree|4000|Low Risk|2|0|0| +13132|F|M|4 yr Degree|4000|Low Risk|2|0|0| +13133|M|S|4 yr Degree|4000|Low Risk|2|0|0| +13134|F|S|4 yr Degree|4000|Low Risk|2|0|0| +13135|M|D|4 yr Degree|4000|Low Risk|2|0|0| +13136|F|D|4 yr Degree|4000|Low Risk|2|0|0| +13137|M|W|4 yr Degree|4000|Low Risk|2|0|0| +13138|F|W|4 yr Degree|4000|Low Risk|2|0|0| +13139|M|U|4 yr Degree|4000|Low Risk|2|0|0| +13140|F|U|4 yr Degree|4000|Low Risk|2|0|0| +13141|M|M|Advanced Degree|4000|Low Risk|2|0|0| +13142|F|M|Advanced Degree|4000|Low Risk|2|0|0| +13143|M|S|Advanced Degree|4000|Low Risk|2|0|0| +13144|F|S|Advanced Degree|4000|Low Risk|2|0|0| +13145|M|D|Advanced Degree|4000|Low Risk|2|0|0| +13146|F|D|Advanced Degree|4000|Low Risk|2|0|0| +13147|M|W|Advanced Degree|4000|Low Risk|2|0|0| +13148|F|W|Advanced Degree|4000|Low Risk|2|0|0| +13149|M|U|Advanced Degree|4000|Low Risk|2|0|0| +13150|F|U|Advanced Degree|4000|Low Risk|2|0|0| +13151|M|M|Unknown|4000|Low Risk|2|0|0| +13152|F|M|Unknown|4000|Low Risk|2|0|0| +13153|M|S|Unknown|4000|Low Risk|2|0|0| +13154|F|S|Unknown|4000|Low Risk|2|0|0| +13155|M|D|Unknown|4000|Low Risk|2|0|0| +13156|F|D|Unknown|4000|Low Risk|2|0|0| +13157|M|W|Unknown|4000|Low Risk|2|0|0| +13158|F|W|Unknown|4000|Low Risk|2|0|0| +13159|M|U|Unknown|4000|Low Risk|2|0|0| +13160|F|U|Unknown|4000|Low Risk|2|0|0| +13161|M|M|Primary|4500|Low Risk|2|0|0| +13162|F|M|Primary|4500|Low Risk|2|0|0| +13163|M|S|Primary|4500|Low Risk|2|0|0| +13164|F|S|Primary|4500|Low Risk|2|0|0| +13165|M|D|Primary|4500|Low Risk|2|0|0| +13166|F|D|Primary|4500|Low Risk|2|0|0| +13167|M|W|Primary|4500|Low Risk|2|0|0| +13168|F|W|Primary|4500|Low Risk|2|0|0| +13169|M|U|Primary|4500|Low Risk|2|0|0| +13170|F|U|Primary|4500|Low Risk|2|0|0| +13171|M|M|Secondary|4500|Low Risk|2|0|0| +13172|F|M|Secondary|4500|Low Risk|2|0|0| +13173|M|S|Secondary|4500|Low Risk|2|0|0| +13174|F|S|Secondary|4500|Low Risk|2|0|0| +13175|M|D|Secondary|4500|Low Risk|2|0|0| +13176|F|D|Secondary|4500|Low Risk|2|0|0| +13177|M|W|Secondary|4500|Low Risk|2|0|0| +13178|F|W|Secondary|4500|Low Risk|2|0|0| +13179|M|U|Secondary|4500|Low Risk|2|0|0| +13180|F|U|Secondary|4500|Low Risk|2|0|0| +13181|M|M|College|4500|Low Risk|2|0|0| +13182|F|M|College|4500|Low Risk|2|0|0| +13183|M|S|College|4500|Low Risk|2|0|0| +13184|F|S|College|4500|Low Risk|2|0|0| +13185|M|D|College|4500|Low Risk|2|0|0| +13186|F|D|College|4500|Low Risk|2|0|0| +13187|M|W|College|4500|Low Risk|2|0|0| +13188|F|W|College|4500|Low Risk|2|0|0| +13189|M|U|College|4500|Low Risk|2|0|0| +13190|F|U|College|4500|Low Risk|2|0|0| +13191|M|M|2 yr Degree|4500|Low Risk|2|0|0| +13192|F|M|2 yr Degree|4500|Low Risk|2|0|0| +13193|M|S|2 yr Degree|4500|Low Risk|2|0|0| +13194|F|S|2 yr Degree|4500|Low Risk|2|0|0| +13195|M|D|2 yr Degree|4500|Low Risk|2|0|0| +13196|F|D|2 yr Degree|4500|Low Risk|2|0|0| +13197|M|W|2 yr Degree|4500|Low Risk|2|0|0| +13198|F|W|2 yr Degree|4500|Low Risk|2|0|0| +13199|M|U|2 yr Degree|4500|Low Risk|2|0|0| +13200|F|U|2 yr Degree|4500|Low Risk|2|0|0| +13201|M|M|4 yr Degree|4500|Low Risk|2|0|0| +13202|F|M|4 yr Degree|4500|Low Risk|2|0|0| +13203|M|S|4 yr Degree|4500|Low Risk|2|0|0| +13204|F|S|4 yr Degree|4500|Low Risk|2|0|0| +13205|M|D|4 yr Degree|4500|Low Risk|2|0|0| +13206|F|D|4 yr Degree|4500|Low Risk|2|0|0| +13207|M|W|4 yr Degree|4500|Low Risk|2|0|0| +13208|F|W|4 yr Degree|4500|Low Risk|2|0|0| +13209|M|U|4 yr Degree|4500|Low Risk|2|0|0| +13210|F|U|4 yr Degree|4500|Low Risk|2|0|0| +13211|M|M|Advanced Degree|4500|Low Risk|2|0|0| +13212|F|M|Advanced Degree|4500|Low Risk|2|0|0| +13213|M|S|Advanced Degree|4500|Low Risk|2|0|0| +13214|F|S|Advanced Degree|4500|Low Risk|2|0|0| +13215|M|D|Advanced Degree|4500|Low Risk|2|0|0| +13216|F|D|Advanced Degree|4500|Low Risk|2|0|0| +13217|M|W|Advanced Degree|4500|Low Risk|2|0|0| +13218|F|W|Advanced Degree|4500|Low Risk|2|0|0| +13219|M|U|Advanced Degree|4500|Low Risk|2|0|0| +13220|F|U|Advanced Degree|4500|Low Risk|2|0|0| +13221|M|M|Unknown|4500|Low Risk|2|0|0| +13222|F|M|Unknown|4500|Low Risk|2|0|0| +13223|M|S|Unknown|4500|Low Risk|2|0|0| +13224|F|S|Unknown|4500|Low Risk|2|0|0| +13225|M|D|Unknown|4500|Low Risk|2|0|0| +13226|F|D|Unknown|4500|Low Risk|2|0|0| +13227|M|W|Unknown|4500|Low Risk|2|0|0| +13228|F|W|Unknown|4500|Low Risk|2|0|0| +13229|M|U|Unknown|4500|Low Risk|2|0|0| +13230|F|U|Unknown|4500|Low Risk|2|0|0| +13231|M|M|Primary|5000|Low Risk|2|0|0| +13232|F|M|Primary|5000|Low Risk|2|0|0| +13233|M|S|Primary|5000|Low Risk|2|0|0| +13234|F|S|Primary|5000|Low Risk|2|0|0| +13235|M|D|Primary|5000|Low Risk|2|0|0| +13236|F|D|Primary|5000|Low Risk|2|0|0| +13237|M|W|Primary|5000|Low Risk|2|0|0| +13238|F|W|Primary|5000|Low Risk|2|0|0| +13239|M|U|Primary|5000|Low Risk|2|0|0| +13240|F|U|Primary|5000|Low Risk|2|0|0| +13241|M|M|Secondary|5000|Low Risk|2|0|0| +13242|F|M|Secondary|5000|Low Risk|2|0|0| +13243|M|S|Secondary|5000|Low Risk|2|0|0| +13244|F|S|Secondary|5000|Low Risk|2|0|0| +13245|M|D|Secondary|5000|Low Risk|2|0|0| +13246|F|D|Secondary|5000|Low Risk|2|0|0| +13247|M|W|Secondary|5000|Low Risk|2|0|0| +13248|F|W|Secondary|5000|Low Risk|2|0|0| +13249|M|U|Secondary|5000|Low Risk|2|0|0| +13250|F|U|Secondary|5000|Low Risk|2|0|0| +13251|M|M|College|5000|Low Risk|2|0|0| +13252|F|M|College|5000|Low Risk|2|0|0| +13253|M|S|College|5000|Low Risk|2|0|0| +13254|F|S|College|5000|Low Risk|2|0|0| +13255|M|D|College|5000|Low Risk|2|0|0| +13256|F|D|College|5000|Low Risk|2|0|0| +13257|M|W|College|5000|Low Risk|2|0|0| +13258|F|W|College|5000|Low Risk|2|0|0| +13259|M|U|College|5000|Low Risk|2|0|0| +13260|F|U|College|5000|Low Risk|2|0|0| +13261|M|M|2 yr Degree|5000|Low Risk|2|0|0| +13262|F|M|2 yr Degree|5000|Low Risk|2|0|0| +13263|M|S|2 yr Degree|5000|Low Risk|2|0|0| +13264|F|S|2 yr Degree|5000|Low Risk|2|0|0| +13265|M|D|2 yr Degree|5000|Low Risk|2|0|0| +13266|F|D|2 yr Degree|5000|Low Risk|2|0|0| +13267|M|W|2 yr Degree|5000|Low Risk|2|0|0| +13268|F|W|2 yr Degree|5000|Low Risk|2|0|0| +13269|M|U|2 yr Degree|5000|Low Risk|2|0|0| +13270|F|U|2 yr Degree|5000|Low Risk|2|0|0| +13271|M|M|4 yr Degree|5000|Low Risk|2|0|0| +13272|F|M|4 yr Degree|5000|Low Risk|2|0|0| +13273|M|S|4 yr Degree|5000|Low Risk|2|0|0| +13274|F|S|4 yr Degree|5000|Low Risk|2|0|0| +13275|M|D|4 yr Degree|5000|Low Risk|2|0|0| +13276|F|D|4 yr Degree|5000|Low Risk|2|0|0| +13277|M|W|4 yr Degree|5000|Low Risk|2|0|0| +13278|F|W|4 yr Degree|5000|Low Risk|2|0|0| +13279|M|U|4 yr Degree|5000|Low Risk|2|0|0| +13280|F|U|4 yr Degree|5000|Low Risk|2|0|0| +13281|M|M|Advanced Degree|5000|Low Risk|2|0|0| +13282|F|M|Advanced Degree|5000|Low Risk|2|0|0| +13283|M|S|Advanced Degree|5000|Low Risk|2|0|0| +13284|F|S|Advanced Degree|5000|Low Risk|2|0|0| +13285|M|D|Advanced Degree|5000|Low Risk|2|0|0| +13286|F|D|Advanced Degree|5000|Low Risk|2|0|0| +13287|M|W|Advanced Degree|5000|Low Risk|2|0|0| +13288|F|W|Advanced Degree|5000|Low Risk|2|0|0| +13289|M|U|Advanced Degree|5000|Low Risk|2|0|0| +13290|F|U|Advanced Degree|5000|Low Risk|2|0|0| +13291|M|M|Unknown|5000|Low Risk|2|0|0| +13292|F|M|Unknown|5000|Low Risk|2|0|0| +13293|M|S|Unknown|5000|Low Risk|2|0|0| +13294|F|S|Unknown|5000|Low Risk|2|0|0| +13295|M|D|Unknown|5000|Low Risk|2|0|0| +13296|F|D|Unknown|5000|Low Risk|2|0|0| +13297|M|W|Unknown|5000|Low Risk|2|0|0| +13298|F|W|Unknown|5000|Low Risk|2|0|0| +13299|M|U|Unknown|5000|Low Risk|2|0|0| +13300|F|U|Unknown|5000|Low Risk|2|0|0| +13301|M|M|Primary|5500|Low Risk|2|0|0| +13302|F|M|Primary|5500|Low Risk|2|0|0| +13303|M|S|Primary|5500|Low Risk|2|0|0| +13304|F|S|Primary|5500|Low Risk|2|0|0| +13305|M|D|Primary|5500|Low Risk|2|0|0| +13306|F|D|Primary|5500|Low Risk|2|0|0| +13307|M|W|Primary|5500|Low Risk|2|0|0| +13308|F|W|Primary|5500|Low Risk|2|0|0| +13309|M|U|Primary|5500|Low Risk|2|0|0| +13310|F|U|Primary|5500|Low Risk|2|0|0| +13311|M|M|Secondary|5500|Low Risk|2|0|0| +13312|F|M|Secondary|5500|Low Risk|2|0|0| +13313|M|S|Secondary|5500|Low Risk|2|0|0| +13314|F|S|Secondary|5500|Low Risk|2|0|0| +13315|M|D|Secondary|5500|Low Risk|2|0|0| +13316|F|D|Secondary|5500|Low Risk|2|0|0| +13317|M|W|Secondary|5500|Low Risk|2|0|0| +13318|F|W|Secondary|5500|Low Risk|2|0|0| +13319|M|U|Secondary|5500|Low Risk|2|0|0| +13320|F|U|Secondary|5500|Low Risk|2|0|0| +13321|M|M|College|5500|Low Risk|2|0|0| +13322|F|M|College|5500|Low Risk|2|0|0| +13323|M|S|College|5500|Low Risk|2|0|0| +13324|F|S|College|5500|Low Risk|2|0|0| +13325|M|D|College|5500|Low Risk|2|0|0| +13326|F|D|College|5500|Low Risk|2|0|0| +13327|M|W|College|5500|Low Risk|2|0|0| +13328|F|W|College|5500|Low Risk|2|0|0| +13329|M|U|College|5500|Low Risk|2|0|0| +13330|F|U|College|5500|Low Risk|2|0|0| +13331|M|M|2 yr Degree|5500|Low Risk|2|0|0| +13332|F|M|2 yr Degree|5500|Low Risk|2|0|0| +13333|M|S|2 yr Degree|5500|Low Risk|2|0|0| +13334|F|S|2 yr Degree|5500|Low Risk|2|0|0| +13335|M|D|2 yr Degree|5500|Low Risk|2|0|0| +13336|F|D|2 yr Degree|5500|Low Risk|2|0|0| +13337|M|W|2 yr Degree|5500|Low Risk|2|0|0| +13338|F|W|2 yr Degree|5500|Low Risk|2|0|0| +13339|M|U|2 yr Degree|5500|Low Risk|2|0|0| +13340|F|U|2 yr Degree|5500|Low Risk|2|0|0| +13341|M|M|4 yr Degree|5500|Low Risk|2|0|0| +13342|F|M|4 yr Degree|5500|Low Risk|2|0|0| +13343|M|S|4 yr Degree|5500|Low Risk|2|0|0| +13344|F|S|4 yr Degree|5500|Low Risk|2|0|0| +13345|M|D|4 yr Degree|5500|Low Risk|2|0|0| +13346|F|D|4 yr Degree|5500|Low Risk|2|0|0| +13347|M|W|4 yr Degree|5500|Low Risk|2|0|0| +13348|F|W|4 yr Degree|5500|Low Risk|2|0|0| +13349|M|U|4 yr Degree|5500|Low Risk|2|0|0| +13350|F|U|4 yr Degree|5500|Low Risk|2|0|0| +13351|M|M|Advanced Degree|5500|Low Risk|2|0|0| +13352|F|M|Advanced Degree|5500|Low Risk|2|0|0| +13353|M|S|Advanced Degree|5500|Low Risk|2|0|0| +13354|F|S|Advanced Degree|5500|Low Risk|2|0|0| +13355|M|D|Advanced Degree|5500|Low Risk|2|0|0| +13356|F|D|Advanced Degree|5500|Low Risk|2|0|0| +13357|M|W|Advanced Degree|5500|Low Risk|2|0|0| +13358|F|W|Advanced Degree|5500|Low Risk|2|0|0| +13359|M|U|Advanced Degree|5500|Low Risk|2|0|0| +13360|F|U|Advanced Degree|5500|Low Risk|2|0|0| +13361|M|M|Unknown|5500|Low Risk|2|0|0| +13362|F|M|Unknown|5500|Low Risk|2|0|0| +13363|M|S|Unknown|5500|Low Risk|2|0|0| +13364|F|S|Unknown|5500|Low Risk|2|0|0| +13365|M|D|Unknown|5500|Low Risk|2|0|0| +13366|F|D|Unknown|5500|Low Risk|2|0|0| +13367|M|W|Unknown|5500|Low Risk|2|0|0| +13368|F|W|Unknown|5500|Low Risk|2|0|0| +13369|M|U|Unknown|5500|Low Risk|2|0|0| +13370|F|U|Unknown|5500|Low Risk|2|0|0| +13371|M|M|Primary|6000|Low Risk|2|0|0| +13372|F|M|Primary|6000|Low Risk|2|0|0| +13373|M|S|Primary|6000|Low Risk|2|0|0| +13374|F|S|Primary|6000|Low Risk|2|0|0| +13375|M|D|Primary|6000|Low Risk|2|0|0| +13376|F|D|Primary|6000|Low Risk|2|0|0| +13377|M|W|Primary|6000|Low Risk|2|0|0| +13378|F|W|Primary|6000|Low Risk|2|0|0| +13379|M|U|Primary|6000|Low Risk|2|0|0| +13380|F|U|Primary|6000|Low Risk|2|0|0| +13381|M|M|Secondary|6000|Low Risk|2|0|0| +13382|F|M|Secondary|6000|Low Risk|2|0|0| +13383|M|S|Secondary|6000|Low Risk|2|0|0| +13384|F|S|Secondary|6000|Low Risk|2|0|0| +13385|M|D|Secondary|6000|Low Risk|2|0|0| +13386|F|D|Secondary|6000|Low Risk|2|0|0| +13387|M|W|Secondary|6000|Low Risk|2|0|0| +13388|F|W|Secondary|6000|Low Risk|2|0|0| +13389|M|U|Secondary|6000|Low Risk|2|0|0| +13390|F|U|Secondary|6000|Low Risk|2|0|0| +13391|M|M|College|6000|Low Risk|2|0|0| +13392|F|M|College|6000|Low Risk|2|0|0| +13393|M|S|College|6000|Low Risk|2|0|0| +13394|F|S|College|6000|Low Risk|2|0|0| +13395|M|D|College|6000|Low Risk|2|0|0| +13396|F|D|College|6000|Low Risk|2|0|0| +13397|M|W|College|6000|Low Risk|2|0|0| +13398|F|W|College|6000|Low Risk|2|0|0| +13399|M|U|College|6000|Low Risk|2|0|0| +13400|F|U|College|6000|Low Risk|2|0|0| +13401|M|M|2 yr Degree|6000|Low Risk|2|0|0| +13402|F|M|2 yr Degree|6000|Low Risk|2|0|0| +13403|M|S|2 yr Degree|6000|Low Risk|2|0|0| +13404|F|S|2 yr Degree|6000|Low Risk|2|0|0| +13405|M|D|2 yr Degree|6000|Low Risk|2|0|0| +13406|F|D|2 yr Degree|6000|Low Risk|2|0|0| +13407|M|W|2 yr Degree|6000|Low Risk|2|0|0| +13408|F|W|2 yr Degree|6000|Low Risk|2|0|0| +13409|M|U|2 yr Degree|6000|Low Risk|2|0|0| +13410|F|U|2 yr Degree|6000|Low Risk|2|0|0| +13411|M|M|4 yr Degree|6000|Low Risk|2|0|0| +13412|F|M|4 yr Degree|6000|Low Risk|2|0|0| +13413|M|S|4 yr Degree|6000|Low Risk|2|0|0| +13414|F|S|4 yr Degree|6000|Low Risk|2|0|0| +13415|M|D|4 yr Degree|6000|Low Risk|2|0|0| +13416|F|D|4 yr Degree|6000|Low Risk|2|0|0| +13417|M|W|4 yr Degree|6000|Low Risk|2|0|0| +13418|F|W|4 yr Degree|6000|Low Risk|2|0|0| +13419|M|U|4 yr Degree|6000|Low Risk|2|0|0| +13420|F|U|4 yr Degree|6000|Low Risk|2|0|0| +13421|M|M|Advanced Degree|6000|Low Risk|2|0|0| +13422|F|M|Advanced Degree|6000|Low Risk|2|0|0| +13423|M|S|Advanced Degree|6000|Low Risk|2|0|0| +13424|F|S|Advanced Degree|6000|Low Risk|2|0|0| +13425|M|D|Advanced Degree|6000|Low Risk|2|0|0| +13426|F|D|Advanced Degree|6000|Low Risk|2|0|0| +13427|M|W|Advanced Degree|6000|Low Risk|2|0|0| +13428|F|W|Advanced Degree|6000|Low Risk|2|0|0| +13429|M|U|Advanced Degree|6000|Low Risk|2|0|0| +13430|F|U|Advanced Degree|6000|Low Risk|2|0|0| +13431|M|M|Unknown|6000|Low Risk|2|0|0| +13432|F|M|Unknown|6000|Low Risk|2|0|0| +13433|M|S|Unknown|6000|Low Risk|2|0|0| +13434|F|S|Unknown|6000|Low Risk|2|0|0| +13435|M|D|Unknown|6000|Low Risk|2|0|0| +13436|F|D|Unknown|6000|Low Risk|2|0|0| +13437|M|W|Unknown|6000|Low Risk|2|0|0| +13438|F|W|Unknown|6000|Low Risk|2|0|0| +13439|M|U|Unknown|6000|Low Risk|2|0|0| +13440|F|U|Unknown|6000|Low Risk|2|0|0| +13441|M|M|Primary|6500|Low Risk|2|0|0| +13442|F|M|Primary|6500|Low Risk|2|0|0| +13443|M|S|Primary|6500|Low Risk|2|0|0| +13444|F|S|Primary|6500|Low Risk|2|0|0| +13445|M|D|Primary|6500|Low Risk|2|0|0| +13446|F|D|Primary|6500|Low Risk|2|0|0| +13447|M|W|Primary|6500|Low Risk|2|0|0| +13448|F|W|Primary|6500|Low Risk|2|0|0| +13449|M|U|Primary|6500|Low Risk|2|0|0| +13450|F|U|Primary|6500|Low Risk|2|0|0| +13451|M|M|Secondary|6500|Low Risk|2|0|0| +13452|F|M|Secondary|6500|Low Risk|2|0|0| +13453|M|S|Secondary|6500|Low Risk|2|0|0| +13454|F|S|Secondary|6500|Low Risk|2|0|0| +13455|M|D|Secondary|6500|Low Risk|2|0|0| +13456|F|D|Secondary|6500|Low Risk|2|0|0| +13457|M|W|Secondary|6500|Low Risk|2|0|0| +13458|F|W|Secondary|6500|Low Risk|2|0|0| +13459|M|U|Secondary|6500|Low Risk|2|0|0| +13460|F|U|Secondary|6500|Low Risk|2|0|0| +13461|M|M|College|6500|Low Risk|2|0|0| +13462|F|M|College|6500|Low Risk|2|0|0| +13463|M|S|College|6500|Low Risk|2|0|0| +13464|F|S|College|6500|Low Risk|2|0|0| +13465|M|D|College|6500|Low Risk|2|0|0| +13466|F|D|College|6500|Low Risk|2|0|0| +13467|M|W|College|6500|Low Risk|2|0|0| +13468|F|W|College|6500|Low Risk|2|0|0| +13469|M|U|College|6500|Low Risk|2|0|0| +13470|F|U|College|6500|Low Risk|2|0|0| +13471|M|M|2 yr Degree|6500|Low Risk|2|0|0| +13472|F|M|2 yr Degree|6500|Low Risk|2|0|0| +13473|M|S|2 yr Degree|6500|Low Risk|2|0|0| +13474|F|S|2 yr Degree|6500|Low Risk|2|0|0| +13475|M|D|2 yr Degree|6500|Low Risk|2|0|0| +13476|F|D|2 yr Degree|6500|Low Risk|2|0|0| +13477|M|W|2 yr Degree|6500|Low Risk|2|0|0| +13478|F|W|2 yr Degree|6500|Low Risk|2|0|0| +13479|M|U|2 yr Degree|6500|Low Risk|2|0|0| +13480|F|U|2 yr Degree|6500|Low Risk|2|0|0| +13481|M|M|4 yr Degree|6500|Low Risk|2|0|0| +13482|F|M|4 yr Degree|6500|Low Risk|2|0|0| +13483|M|S|4 yr Degree|6500|Low Risk|2|0|0| +13484|F|S|4 yr Degree|6500|Low Risk|2|0|0| +13485|M|D|4 yr Degree|6500|Low Risk|2|0|0| +13486|F|D|4 yr Degree|6500|Low Risk|2|0|0| +13487|M|W|4 yr Degree|6500|Low Risk|2|0|0| +13488|F|W|4 yr Degree|6500|Low Risk|2|0|0| +13489|M|U|4 yr Degree|6500|Low Risk|2|0|0| +13490|F|U|4 yr Degree|6500|Low Risk|2|0|0| +13491|M|M|Advanced Degree|6500|Low Risk|2|0|0| +13492|F|M|Advanced Degree|6500|Low Risk|2|0|0| +13493|M|S|Advanced Degree|6500|Low Risk|2|0|0| +13494|F|S|Advanced Degree|6500|Low Risk|2|0|0| +13495|M|D|Advanced Degree|6500|Low Risk|2|0|0| +13496|F|D|Advanced Degree|6500|Low Risk|2|0|0| +13497|M|W|Advanced Degree|6500|Low Risk|2|0|0| +13498|F|W|Advanced Degree|6500|Low Risk|2|0|0| +13499|M|U|Advanced Degree|6500|Low Risk|2|0|0| +13500|F|U|Advanced Degree|6500|Low Risk|2|0|0| +13501|M|M|Unknown|6500|Low Risk|2|0|0| +13502|F|M|Unknown|6500|Low Risk|2|0|0| +13503|M|S|Unknown|6500|Low Risk|2|0|0| +13504|F|S|Unknown|6500|Low Risk|2|0|0| +13505|M|D|Unknown|6500|Low Risk|2|0|0| +13506|F|D|Unknown|6500|Low Risk|2|0|0| +13507|M|W|Unknown|6500|Low Risk|2|0|0| +13508|F|W|Unknown|6500|Low Risk|2|0|0| +13509|M|U|Unknown|6500|Low Risk|2|0|0| +13510|F|U|Unknown|6500|Low Risk|2|0|0| +13511|M|M|Primary|7000|Low Risk|2|0|0| +13512|F|M|Primary|7000|Low Risk|2|0|0| +13513|M|S|Primary|7000|Low Risk|2|0|0| +13514|F|S|Primary|7000|Low Risk|2|0|0| +13515|M|D|Primary|7000|Low Risk|2|0|0| +13516|F|D|Primary|7000|Low Risk|2|0|0| +13517|M|W|Primary|7000|Low Risk|2|0|0| +13518|F|W|Primary|7000|Low Risk|2|0|0| +13519|M|U|Primary|7000|Low Risk|2|0|0| +13520|F|U|Primary|7000|Low Risk|2|0|0| +13521|M|M|Secondary|7000|Low Risk|2|0|0| +13522|F|M|Secondary|7000|Low Risk|2|0|0| +13523|M|S|Secondary|7000|Low Risk|2|0|0| +13524|F|S|Secondary|7000|Low Risk|2|0|0| +13525|M|D|Secondary|7000|Low Risk|2|0|0| +13526|F|D|Secondary|7000|Low Risk|2|0|0| +13527|M|W|Secondary|7000|Low Risk|2|0|0| +13528|F|W|Secondary|7000|Low Risk|2|0|0| +13529|M|U|Secondary|7000|Low Risk|2|0|0| +13530|F|U|Secondary|7000|Low Risk|2|0|0| +13531|M|M|College|7000|Low Risk|2|0|0| +13532|F|M|College|7000|Low Risk|2|0|0| +13533|M|S|College|7000|Low Risk|2|0|0| +13534|F|S|College|7000|Low Risk|2|0|0| +13535|M|D|College|7000|Low Risk|2|0|0| +13536|F|D|College|7000|Low Risk|2|0|0| +13537|M|W|College|7000|Low Risk|2|0|0| +13538|F|W|College|7000|Low Risk|2|0|0| +13539|M|U|College|7000|Low Risk|2|0|0| +13540|F|U|College|7000|Low Risk|2|0|0| +13541|M|M|2 yr Degree|7000|Low Risk|2|0|0| +13542|F|M|2 yr Degree|7000|Low Risk|2|0|0| +13543|M|S|2 yr Degree|7000|Low Risk|2|0|0| +13544|F|S|2 yr Degree|7000|Low Risk|2|0|0| +13545|M|D|2 yr Degree|7000|Low Risk|2|0|0| +13546|F|D|2 yr Degree|7000|Low Risk|2|0|0| +13547|M|W|2 yr Degree|7000|Low Risk|2|0|0| +13548|F|W|2 yr Degree|7000|Low Risk|2|0|0| +13549|M|U|2 yr Degree|7000|Low Risk|2|0|0| +13550|F|U|2 yr Degree|7000|Low Risk|2|0|0| +13551|M|M|4 yr Degree|7000|Low Risk|2|0|0| +13552|F|M|4 yr Degree|7000|Low Risk|2|0|0| +13553|M|S|4 yr Degree|7000|Low Risk|2|0|0| +13554|F|S|4 yr Degree|7000|Low Risk|2|0|0| +13555|M|D|4 yr Degree|7000|Low Risk|2|0|0| +13556|F|D|4 yr Degree|7000|Low Risk|2|0|0| +13557|M|W|4 yr Degree|7000|Low Risk|2|0|0| +13558|F|W|4 yr Degree|7000|Low Risk|2|0|0| +13559|M|U|4 yr Degree|7000|Low Risk|2|0|0| +13560|F|U|4 yr Degree|7000|Low Risk|2|0|0| +13561|M|M|Advanced Degree|7000|Low Risk|2|0|0| +13562|F|M|Advanced Degree|7000|Low Risk|2|0|0| +13563|M|S|Advanced Degree|7000|Low Risk|2|0|0| +13564|F|S|Advanced Degree|7000|Low Risk|2|0|0| +13565|M|D|Advanced Degree|7000|Low Risk|2|0|0| +13566|F|D|Advanced Degree|7000|Low Risk|2|0|0| +13567|M|W|Advanced Degree|7000|Low Risk|2|0|0| +13568|F|W|Advanced Degree|7000|Low Risk|2|0|0| +13569|M|U|Advanced Degree|7000|Low Risk|2|0|0| +13570|F|U|Advanced Degree|7000|Low Risk|2|0|0| +13571|M|M|Unknown|7000|Low Risk|2|0|0| +13572|F|M|Unknown|7000|Low Risk|2|0|0| +13573|M|S|Unknown|7000|Low Risk|2|0|0| +13574|F|S|Unknown|7000|Low Risk|2|0|0| +13575|M|D|Unknown|7000|Low Risk|2|0|0| +13576|F|D|Unknown|7000|Low Risk|2|0|0| +13577|M|W|Unknown|7000|Low Risk|2|0|0| +13578|F|W|Unknown|7000|Low Risk|2|0|0| +13579|M|U|Unknown|7000|Low Risk|2|0|0| +13580|F|U|Unknown|7000|Low Risk|2|0|0| +13581|M|M|Primary|7500|Low Risk|2|0|0| +13582|F|M|Primary|7500|Low Risk|2|0|0| +13583|M|S|Primary|7500|Low Risk|2|0|0| +13584|F|S|Primary|7500|Low Risk|2|0|0| +13585|M|D|Primary|7500|Low Risk|2|0|0| +13586|F|D|Primary|7500|Low Risk|2|0|0| +13587|M|W|Primary|7500|Low Risk|2|0|0| +13588|F|W|Primary|7500|Low Risk|2|0|0| +13589|M|U|Primary|7500|Low Risk|2|0|0| +13590|F|U|Primary|7500|Low Risk|2|0|0| +13591|M|M|Secondary|7500|Low Risk|2|0|0| +13592|F|M|Secondary|7500|Low Risk|2|0|0| +13593|M|S|Secondary|7500|Low Risk|2|0|0| +13594|F|S|Secondary|7500|Low Risk|2|0|0| +13595|M|D|Secondary|7500|Low Risk|2|0|0| +13596|F|D|Secondary|7500|Low Risk|2|0|0| +13597|M|W|Secondary|7500|Low Risk|2|0|0| +13598|F|W|Secondary|7500|Low Risk|2|0|0| +13599|M|U|Secondary|7500|Low Risk|2|0|0| +13600|F|U|Secondary|7500|Low Risk|2|0|0| +13601|M|M|College|7500|Low Risk|2|0|0| +13602|F|M|College|7500|Low Risk|2|0|0| +13603|M|S|College|7500|Low Risk|2|0|0| +13604|F|S|College|7500|Low Risk|2|0|0| +13605|M|D|College|7500|Low Risk|2|0|0| +13606|F|D|College|7500|Low Risk|2|0|0| +13607|M|W|College|7500|Low Risk|2|0|0| +13608|F|W|College|7500|Low Risk|2|0|0| +13609|M|U|College|7500|Low Risk|2|0|0| +13610|F|U|College|7500|Low Risk|2|0|0| +13611|M|M|2 yr Degree|7500|Low Risk|2|0|0| +13612|F|M|2 yr Degree|7500|Low Risk|2|0|0| +13613|M|S|2 yr Degree|7500|Low Risk|2|0|0| +13614|F|S|2 yr Degree|7500|Low Risk|2|0|0| +13615|M|D|2 yr Degree|7500|Low Risk|2|0|0| +13616|F|D|2 yr Degree|7500|Low Risk|2|0|0| +13617|M|W|2 yr Degree|7500|Low Risk|2|0|0| +13618|F|W|2 yr Degree|7500|Low Risk|2|0|0| +13619|M|U|2 yr Degree|7500|Low Risk|2|0|0| +13620|F|U|2 yr Degree|7500|Low Risk|2|0|0| +13621|M|M|4 yr Degree|7500|Low Risk|2|0|0| +13622|F|M|4 yr Degree|7500|Low Risk|2|0|0| +13623|M|S|4 yr Degree|7500|Low Risk|2|0|0| +13624|F|S|4 yr Degree|7500|Low Risk|2|0|0| +13625|M|D|4 yr Degree|7500|Low Risk|2|0|0| +13626|F|D|4 yr Degree|7500|Low Risk|2|0|0| +13627|M|W|4 yr Degree|7500|Low Risk|2|0|0| +13628|F|W|4 yr Degree|7500|Low Risk|2|0|0| +13629|M|U|4 yr Degree|7500|Low Risk|2|0|0| +13630|F|U|4 yr Degree|7500|Low Risk|2|0|0| +13631|M|M|Advanced Degree|7500|Low Risk|2|0|0| +13632|F|M|Advanced Degree|7500|Low Risk|2|0|0| +13633|M|S|Advanced Degree|7500|Low Risk|2|0|0| +13634|F|S|Advanced Degree|7500|Low Risk|2|0|0| +13635|M|D|Advanced Degree|7500|Low Risk|2|0|0| +13636|F|D|Advanced Degree|7500|Low Risk|2|0|0| +13637|M|W|Advanced Degree|7500|Low Risk|2|0|0| +13638|F|W|Advanced Degree|7500|Low Risk|2|0|0| +13639|M|U|Advanced Degree|7500|Low Risk|2|0|0| +13640|F|U|Advanced Degree|7500|Low Risk|2|0|0| +13641|M|M|Unknown|7500|Low Risk|2|0|0| +13642|F|M|Unknown|7500|Low Risk|2|0|0| +13643|M|S|Unknown|7500|Low Risk|2|0|0| +13644|F|S|Unknown|7500|Low Risk|2|0|0| +13645|M|D|Unknown|7500|Low Risk|2|0|0| +13646|F|D|Unknown|7500|Low Risk|2|0|0| +13647|M|W|Unknown|7500|Low Risk|2|0|0| +13648|F|W|Unknown|7500|Low Risk|2|0|0| +13649|M|U|Unknown|7500|Low Risk|2|0|0| +13650|F|U|Unknown|7500|Low Risk|2|0|0| +13651|M|M|Primary|8000|Low Risk|2|0|0| +13652|F|M|Primary|8000|Low Risk|2|0|0| +13653|M|S|Primary|8000|Low Risk|2|0|0| +13654|F|S|Primary|8000|Low Risk|2|0|0| +13655|M|D|Primary|8000|Low Risk|2|0|0| +13656|F|D|Primary|8000|Low Risk|2|0|0| +13657|M|W|Primary|8000|Low Risk|2|0|0| +13658|F|W|Primary|8000|Low Risk|2|0|0| +13659|M|U|Primary|8000|Low Risk|2|0|0| +13660|F|U|Primary|8000|Low Risk|2|0|0| +13661|M|M|Secondary|8000|Low Risk|2|0|0| +13662|F|M|Secondary|8000|Low Risk|2|0|0| +13663|M|S|Secondary|8000|Low Risk|2|0|0| +13664|F|S|Secondary|8000|Low Risk|2|0|0| +13665|M|D|Secondary|8000|Low Risk|2|0|0| +13666|F|D|Secondary|8000|Low Risk|2|0|0| +13667|M|W|Secondary|8000|Low Risk|2|0|0| +13668|F|W|Secondary|8000|Low Risk|2|0|0| +13669|M|U|Secondary|8000|Low Risk|2|0|0| +13670|F|U|Secondary|8000|Low Risk|2|0|0| +13671|M|M|College|8000|Low Risk|2|0|0| +13672|F|M|College|8000|Low Risk|2|0|0| +13673|M|S|College|8000|Low Risk|2|0|0| +13674|F|S|College|8000|Low Risk|2|0|0| +13675|M|D|College|8000|Low Risk|2|0|0| +13676|F|D|College|8000|Low Risk|2|0|0| +13677|M|W|College|8000|Low Risk|2|0|0| +13678|F|W|College|8000|Low Risk|2|0|0| +13679|M|U|College|8000|Low Risk|2|0|0| +13680|F|U|College|8000|Low Risk|2|0|0| +13681|M|M|2 yr Degree|8000|Low Risk|2|0|0| +13682|F|M|2 yr Degree|8000|Low Risk|2|0|0| +13683|M|S|2 yr Degree|8000|Low Risk|2|0|0| +13684|F|S|2 yr Degree|8000|Low Risk|2|0|0| +13685|M|D|2 yr Degree|8000|Low Risk|2|0|0| +13686|F|D|2 yr Degree|8000|Low Risk|2|0|0| +13687|M|W|2 yr Degree|8000|Low Risk|2|0|0| +13688|F|W|2 yr Degree|8000|Low Risk|2|0|0| +13689|M|U|2 yr Degree|8000|Low Risk|2|0|0| +13690|F|U|2 yr Degree|8000|Low Risk|2|0|0| +13691|M|M|4 yr Degree|8000|Low Risk|2|0|0| +13692|F|M|4 yr Degree|8000|Low Risk|2|0|0| +13693|M|S|4 yr Degree|8000|Low Risk|2|0|0| +13694|F|S|4 yr Degree|8000|Low Risk|2|0|0| +13695|M|D|4 yr Degree|8000|Low Risk|2|0|0| +13696|F|D|4 yr Degree|8000|Low Risk|2|0|0| +13697|M|W|4 yr Degree|8000|Low Risk|2|0|0| +13698|F|W|4 yr Degree|8000|Low Risk|2|0|0| +13699|M|U|4 yr Degree|8000|Low Risk|2|0|0| +13700|F|U|4 yr Degree|8000|Low Risk|2|0|0| +13701|M|M|Advanced Degree|8000|Low Risk|2|0|0| +13702|F|M|Advanced Degree|8000|Low Risk|2|0|0| +13703|M|S|Advanced Degree|8000|Low Risk|2|0|0| +13704|F|S|Advanced Degree|8000|Low Risk|2|0|0| +13705|M|D|Advanced Degree|8000|Low Risk|2|0|0| +13706|F|D|Advanced Degree|8000|Low Risk|2|0|0| +13707|M|W|Advanced Degree|8000|Low Risk|2|0|0| +13708|F|W|Advanced Degree|8000|Low Risk|2|0|0| +13709|M|U|Advanced Degree|8000|Low Risk|2|0|0| +13710|F|U|Advanced Degree|8000|Low Risk|2|0|0| +13711|M|M|Unknown|8000|Low Risk|2|0|0| +13712|F|M|Unknown|8000|Low Risk|2|0|0| +13713|M|S|Unknown|8000|Low Risk|2|0|0| +13714|F|S|Unknown|8000|Low Risk|2|0|0| +13715|M|D|Unknown|8000|Low Risk|2|0|0| +13716|F|D|Unknown|8000|Low Risk|2|0|0| +13717|M|W|Unknown|8000|Low Risk|2|0|0| +13718|F|W|Unknown|8000|Low Risk|2|0|0| +13719|M|U|Unknown|8000|Low Risk|2|0|0| +13720|F|U|Unknown|8000|Low Risk|2|0|0| +13721|M|M|Primary|8500|Low Risk|2|0|0| +13722|F|M|Primary|8500|Low Risk|2|0|0| +13723|M|S|Primary|8500|Low Risk|2|0|0| +13724|F|S|Primary|8500|Low Risk|2|0|0| +13725|M|D|Primary|8500|Low Risk|2|0|0| +13726|F|D|Primary|8500|Low Risk|2|0|0| +13727|M|W|Primary|8500|Low Risk|2|0|0| +13728|F|W|Primary|8500|Low Risk|2|0|0| +13729|M|U|Primary|8500|Low Risk|2|0|0| +13730|F|U|Primary|8500|Low Risk|2|0|0| +13731|M|M|Secondary|8500|Low Risk|2|0|0| +13732|F|M|Secondary|8500|Low Risk|2|0|0| +13733|M|S|Secondary|8500|Low Risk|2|0|0| +13734|F|S|Secondary|8500|Low Risk|2|0|0| +13735|M|D|Secondary|8500|Low Risk|2|0|0| +13736|F|D|Secondary|8500|Low Risk|2|0|0| +13737|M|W|Secondary|8500|Low Risk|2|0|0| +13738|F|W|Secondary|8500|Low Risk|2|0|0| +13739|M|U|Secondary|8500|Low Risk|2|0|0| +13740|F|U|Secondary|8500|Low Risk|2|0|0| +13741|M|M|College|8500|Low Risk|2|0|0| +13742|F|M|College|8500|Low Risk|2|0|0| +13743|M|S|College|8500|Low Risk|2|0|0| +13744|F|S|College|8500|Low Risk|2|0|0| +13745|M|D|College|8500|Low Risk|2|0|0| +13746|F|D|College|8500|Low Risk|2|0|0| +13747|M|W|College|8500|Low Risk|2|0|0| +13748|F|W|College|8500|Low Risk|2|0|0| +13749|M|U|College|8500|Low Risk|2|0|0| +13750|F|U|College|8500|Low Risk|2|0|0| +13751|M|M|2 yr Degree|8500|Low Risk|2|0|0| +13752|F|M|2 yr Degree|8500|Low Risk|2|0|0| +13753|M|S|2 yr Degree|8500|Low Risk|2|0|0| +13754|F|S|2 yr Degree|8500|Low Risk|2|0|0| +13755|M|D|2 yr Degree|8500|Low Risk|2|0|0| +13756|F|D|2 yr Degree|8500|Low Risk|2|0|0| +13757|M|W|2 yr Degree|8500|Low Risk|2|0|0| +13758|F|W|2 yr Degree|8500|Low Risk|2|0|0| +13759|M|U|2 yr Degree|8500|Low Risk|2|0|0| +13760|F|U|2 yr Degree|8500|Low Risk|2|0|0| +13761|M|M|4 yr Degree|8500|Low Risk|2|0|0| +13762|F|M|4 yr Degree|8500|Low Risk|2|0|0| +13763|M|S|4 yr Degree|8500|Low Risk|2|0|0| +13764|F|S|4 yr Degree|8500|Low Risk|2|0|0| +13765|M|D|4 yr Degree|8500|Low Risk|2|0|0| +13766|F|D|4 yr Degree|8500|Low Risk|2|0|0| +13767|M|W|4 yr Degree|8500|Low Risk|2|0|0| +13768|F|W|4 yr Degree|8500|Low Risk|2|0|0| +13769|M|U|4 yr Degree|8500|Low Risk|2|0|0| +13770|F|U|4 yr Degree|8500|Low Risk|2|0|0| +13771|M|M|Advanced Degree|8500|Low Risk|2|0|0| +13772|F|M|Advanced Degree|8500|Low Risk|2|0|0| +13773|M|S|Advanced Degree|8500|Low Risk|2|0|0| +13774|F|S|Advanced Degree|8500|Low Risk|2|0|0| +13775|M|D|Advanced Degree|8500|Low Risk|2|0|0| +13776|F|D|Advanced Degree|8500|Low Risk|2|0|0| +13777|M|W|Advanced Degree|8500|Low Risk|2|0|0| +13778|F|W|Advanced Degree|8500|Low Risk|2|0|0| +13779|M|U|Advanced Degree|8500|Low Risk|2|0|0| +13780|F|U|Advanced Degree|8500|Low Risk|2|0|0| +13781|M|M|Unknown|8500|Low Risk|2|0|0| +13782|F|M|Unknown|8500|Low Risk|2|0|0| +13783|M|S|Unknown|8500|Low Risk|2|0|0| +13784|F|S|Unknown|8500|Low Risk|2|0|0| +13785|M|D|Unknown|8500|Low Risk|2|0|0| +13786|F|D|Unknown|8500|Low Risk|2|0|0| +13787|M|W|Unknown|8500|Low Risk|2|0|0| +13788|F|W|Unknown|8500|Low Risk|2|0|0| +13789|M|U|Unknown|8500|Low Risk|2|0|0| +13790|F|U|Unknown|8500|Low Risk|2|0|0| +13791|M|M|Primary|9000|Low Risk|2|0|0| +13792|F|M|Primary|9000|Low Risk|2|0|0| +13793|M|S|Primary|9000|Low Risk|2|0|0| +13794|F|S|Primary|9000|Low Risk|2|0|0| +13795|M|D|Primary|9000|Low Risk|2|0|0| +13796|F|D|Primary|9000|Low Risk|2|0|0| +13797|M|W|Primary|9000|Low Risk|2|0|0| +13798|F|W|Primary|9000|Low Risk|2|0|0| +13799|M|U|Primary|9000|Low Risk|2|0|0| +13800|F|U|Primary|9000|Low Risk|2|0|0| +13801|M|M|Secondary|9000|Low Risk|2|0|0| +13802|F|M|Secondary|9000|Low Risk|2|0|0| +13803|M|S|Secondary|9000|Low Risk|2|0|0| +13804|F|S|Secondary|9000|Low Risk|2|0|0| +13805|M|D|Secondary|9000|Low Risk|2|0|0| +13806|F|D|Secondary|9000|Low Risk|2|0|0| +13807|M|W|Secondary|9000|Low Risk|2|0|0| +13808|F|W|Secondary|9000|Low Risk|2|0|0| +13809|M|U|Secondary|9000|Low Risk|2|0|0| +13810|F|U|Secondary|9000|Low Risk|2|0|0| +13811|M|M|College|9000|Low Risk|2|0|0| +13812|F|M|College|9000|Low Risk|2|0|0| +13813|M|S|College|9000|Low Risk|2|0|0| +13814|F|S|College|9000|Low Risk|2|0|0| +13815|M|D|College|9000|Low Risk|2|0|0| +13816|F|D|College|9000|Low Risk|2|0|0| +13817|M|W|College|9000|Low Risk|2|0|0| +13818|F|W|College|9000|Low Risk|2|0|0| +13819|M|U|College|9000|Low Risk|2|0|0| +13820|F|U|College|9000|Low Risk|2|0|0| +13821|M|M|2 yr Degree|9000|Low Risk|2|0|0| +13822|F|M|2 yr Degree|9000|Low Risk|2|0|0| +13823|M|S|2 yr Degree|9000|Low Risk|2|0|0| +13824|F|S|2 yr Degree|9000|Low Risk|2|0|0| +13825|M|D|2 yr Degree|9000|Low Risk|2|0|0| +13826|F|D|2 yr Degree|9000|Low Risk|2|0|0| +13827|M|W|2 yr Degree|9000|Low Risk|2|0|0| +13828|F|W|2 yr Degree|9000|Low Risk|2|0|0| +13829|M|U|2 yr Degree|9000|Low Risk|2|0|0| +13830|F|U|2 yr Degree|9000|Low Risk|2|0|0| +13831|M|M|4 yr Degree|9000|Low Risk|2|0|0| +13832|F|M|4 yr Degree|9000|Low Risk|2|0|0| +13833|M|S|4 yr Degree|9000|Low Risk|2|0|0| +13834|F|S|4 yr Degree|9000|Low Risk|2|0|0| +13835|M|D|4 yr Degree|9000|Low Risk|2|0|0| +13836|F|D|4 yr Degree|9000|Low Risk|2|0|0| +13837|M|W|4 yr Degree|9000|Low Risk|2|0|0| +13838|F|W|4 yr Degree|9000|Low Risk|2|0|0| +13839|M|U|4 yr Degree|9000|Low Risk|2|0|0| +13840|F|U|4 yr Degree|9000|Low Risk|2|0|0| +13841|M|M|Advanced Degree|9000|Low Risk|2|0|0| +13842|F|M|Advanced Degree|9000|Low Risk|2|0|0| +13843|M|S|Advanced Degree|9000|Low Risk|2|0|0| +13844|F|S|Advanced Degree|9000|Low Risk|2|0|0| +13845|M|D|Advanced Degree|9000|Low Risk|2|0|0| +13846|F|D|Advanced Degree|9000|Low Risk|2|0|0| +13847|M|W|Advanced Degree|9000|Low Risk|2|0|0| +13848|F|W|Advanced Degree|9000|Low Risk|2|0|0| +13849|M|U|Advanced Degree|9000|Low Risk|2|0|0| +13850|F|U|Advanced Degree|9000|Low Risk|2|0|0| +13851|M|M|Unknown|9000|Low Risk|2|0|0| +13852|F|M|Unknown|9000|Low Risk|2|0|0| +13853|M|S|Unknown|9000|Low Risk|2|0|0| +13854|F|S|Unknown|9000|Low Risk|2|0|0| +13855|M|D|Unknown|9000|Low Risk|2|0|0| +13856|F|D|Unknown|9000|Low Risk|2|0|0| +13857|M|W|Unknown|9000|Low Risk|2|0|0| +13858|F|W|Unknown|9000|Low Risk|2|0|0| +13859|M|U|Unknown|9000|Low Risk|2|0|0| +13860|F|U|Unknown|9000|Low Risk|2|0|0| +13861|M|M|Primary|9500|Low Risk|2|0|0| +13862|F|M|Primary|9500|Low Risk|2|0|0| +13863|M|S|Primary|9500|Low Risk|2|0|0| +13864|F|S|Primary|9500|Low Risk|2|0|0| +13865|M|D|Primary|9500|Low Risk|2|0|0| +13866|F|D|Primary|9500|Low Risk|2|0|0| +13867|M|W|Primary|9500|Low Risk|2|0|0| +13868|F|W|Primary|9500|Low Risk|2|0|0| +13869|M|U|Primary|9500|Low Risk|2|0|0| +13870|F|U|Primary|9500|Low Risk|2|0|0| +13871|M|M|Secondary|9500|Low Risk|2|0|0| +13872|F|M|Secondary|9500|Low Risk|2|0|0| +13873|M|S|Secondary|9500|Low Risk|2|0|0| +13874|F|S|Secondary|9500|Low Risk|2|0|0| +13875|M|D|Secondary|9500|Low Risk|2|0|0| +13876|F|D|Secondary|9500|Low Risk|2|0|0| +13877|M|W|Secondary|9500|Low Risk|2|0|0| +13878|F|W|Secondary|9500|Low Risk|2|0|0| +13879|M|U|Secondary|9500|Low Risk|2|0|0| +13880|F|U|Secondary|9500|Low Risk|2|0|0| +13881|M|M|College|9500|Low Risk|2|0|0| +13882|F|M|College|9500|Low Risk|2|0|0| +13883|M|S|College|9500|Low Risk|2|0|0| +13884|F|S|College|9500|Low Risk|2|0|0| +13885|M|D|College|9500|Low Risk|2|0|0| +13886|F|D|College|9500|Low Risk|2|0|0| +13887|M|W|College|9500|Low Risk|2|0|0| +13888|F|W|College|9500|Low Risk|2|0|0| +13889|M|U|College|9500|Low Risk|2|0|0| +13890|F|U|College|9500|Low Risk|2|0|0| +13891|M|M|2 yr Degree|9500|Low Risk|2|0|0| +13892|F|M|2 yr Degree|9500|Low Risk|2|0|0| +13893|M|S|2 yr Degree|9500|Low Risk|2|0|0| +13894|F|S|2 yr Degree|9500|Low Risk|2|0|0| +13895|M|D|2 yr Degree|9500|Low Risk|2|0|0| +13896|F|D|2 yr Degree|9500|Low Risk|2|0|0| +13897|M|W|2 yr Degree|9500|Low Risk|2|0|0| +13898|F|W|2 yr Degree|9500|Low Risk|2|0|0| +13899|M|U|2 yr Degree|9500|Low Risk|2|0|0| +13900|F|U|2 yr Degree|9500|Low Risk|2|0|0| +13901|M|M|4 yr Degree|9500|Low Risk|2|0|0| +13902|F|M|4 yr Degree|9500|Low Risk|2|0|0| +13903|M|S|4 yr Degree|9500|Low Risk|2|0|0| +13904|F|S|4 yr Degree|9500|Low Risk|2|0|0| +13905|M|D|4 yr Degree|9500|Low Risk|2|0|0| +13906|F|D|4 yr Degree|9500|Low Risk|2|0|0| +13907|M|W|4 yr Degree|9500|Low Risk|2|0|0| +13908|F|W|4 yr Degree|9500|Low Risk|2|0|0| +13909|M|U|4 yr Degree|9500|Low Risk|2|0|0| +13910|F|U|4 yr Degree|9500|Low Risk|2|0|0| +13911|M|M|Advanced Degree|9500|Low Risk|2|0|0| +13912|F|M|Advanced Degree|9500|Low Risk|2|0|0| +13913|M|S|Advanced Degree|9500|Low Risk|2|0|0| +13914|F|S|Advanced Degree|9500|Low Risk|2|0|0| +13915|M|D|Advanced Degree|9500|Low Risk|2|0|0| +13916|F|D|Advanced Degree|9500|Low Risk|2|0|0| +13917|M|W|Advanced Degree|9500|Low Risk|2|0|0| +13918|F|W|Advanced Degree|9500|Low Risk|2|0|0| +13919|M|U|Advanced Degree|9500|Low Risk|2|0|0| +13920|F|U|Advanced Degree|9500|Low Risk|2|0|0| +13921|M|M|Unknown|9500|Low Risk|2|0|0| +13922|F|M|Unknown|9500|Low Risk|2|0|0| +13923|M|S|Unknown|9500|Low Risk|2|0|0| +13924|F|S|Unknown|9500|Low Risk|2|0|0| +13925|M|D|Unknown|9500|Low Risk|2|0|0| +13926|F|D|Unknown|9500|Low Risk|2|0|0| +13927|M|W|Unknown|9500|Low Risk|2|0|0| +13928|F|W|Unknown|9500|Low Risk|2|0|0| +13929|M|U|Unknown|9500|Low Risk|2|0|0| +13930|F|U|Unknown|9500|Low Risk|2|0|0| +13931|M|M|Primary|10000|Low Risk|2|0|0| +13932|F|M|Primary|10000|Low Risk|2|0|0| +13933|M|S|Primary|10000|Low Risk|2|0|0| +13934|F|S|Primary|10000|Low Risk|2|0|0| +13935|M|D|Primary|10000|Low Risk|2|0|0| +13936|F|D|Primary|10000|Low Risk|2|0|0| +13937|M|W|Primary|10000|Low Risk|2|0|0| +13938|F|W|Primary|10000|Low Risk|2|0|0| +13939|M|U|Primary|10000|Low Risk|2|0|0| +13940|F|U|Primary|10000|Low Risk|2|0|0| +13941|M|M|Secondary|10000|Low Risk|2|0|0| +13942|F|M|Secondary|10000|Low Risk|2|0|0| +13943|M|S|Secondary|10000|Low Risk|2|0|0| +13944|F|S|Secondary|10000|Low Risk|2|0|0| +13945|M|D|Secondary|10000|Low Risk|2|0|0| +13946|F|D|Secondary|10000|Low Risk|2|0|0| +13947|M|W|Secondary|10000|Low Risk|2|0|0| +13948|F|W|Secondary|10000|Low Risk|2|0|0| +13949|M|U|Secondary|10000|Low Risk|2|0|0| +13950|F|U|Secondary|10000|Low Risk|2|0|0| +13951|M|M|College|10000|Low Risk|2|0|0| +13952|F|M|College|10000|Low Risk|2|0|0| +13953|M|S|College|10000|Low Risk|2|0|0| +13954|F|S|College|10000|Low Risk|2|0|0| +13955|M|D|College|10000|Low Risk|2|0|0| +13956|F|D|College|10000|Low Risk|2|0|0| +13957|M|W|College|10000|Low Risk|2|0|0| +13958|F|W|College|10000|Low Risk|2|0|0| +13959|M|U|College|10000|Low Risk|2|0|0| +13960|F|U|College|10000|Low Risk|2|0|0| +13961|M|M|2 yr Degree|10000|Low Risk|2|0|0| +13962|F|M|2 yr Degree|10000|Low Risk|2|0|0| +13963|M|S|2 yr Degree|10000|Low Risk|2|0|0| +13964|F|S|2 yr Degree|10000|Low Risk|2|0|0| +13965|M|D|2 yr Degree|10000|Low Risk|2|0|0| +13966|F|D|2 yr Degree|10000|Low Risk|2|0|0| +13967|M|W|2 yr Degree|10000|Low Risk|2|0|0| +13968|F|W|2 yr Degree|10000|Low Risk|2|0|0| +13969|M|U|2 yr Degree|10000|Low Risk|2|0|0| +13970|F|U|2 yr Degree|10000|Low Risk|2|0|0| +13971|M|M|4 yr Degree|10000|Low Risk|2|0|0| +13972|F|M|4 yr Degree|10000|Low Risk|2|0|0| +13973|M|S|4 yr Degree|10000|Low Risk|2|0|0| +13974|F|S|4 yr Degree|10000|Low Risk|2|0|0| +13975|M|D|4 yr Degree|10000|Low Risk|2|0|0| +13976|F|D|4 yr Degree|10000|Low Risk|2|0|0| +13977|M|W|4 yr Degree|10000|Low Risk|2|0|0| +13978|F|W|4 yr Degree|10000|Low Risk|2|0|0| +13979|M|U|4 yr Degree|10000|Low Risk|2|0|0| +13980|F|U|4 yr Degree|10000|Low Risk|2|0|0| +13981|M|M|Advanced Degree|10000|Low Risk|2|0|0| +13982|F|M|Advanced Degree|10000|Low Risk|2|0|0| +13983|M|S|Advanced Degree|10000|Low Risk|2|0|0| +13984|F|S|Advanced Degree|10000|Low Risk|2|0|0| +13985|M|D|Advanced Degree|10000|Low Risk|2|0|0| +13986|F|D|Advanced Degree|10000|Low Risk|2|0|0| +13987|M|W|Advanced Degree|10000|Low Risk|2|0|0| +13988|F|W|Advanced Degree|10000|Low Risk|2|0|0| +13989|M|U|Advanced Degree|10000|Low Risk|2|0|0| +13990|F|U|Advanced Degree|10000|Low Risk|2|0|0| +13991|M|M|Unknown|10000|Low Risk|2|0|0| +13992|F|M|Unknown|10000|Low Risk|2|0|0| +13993|M|S|Unknown|10000|Low Risk|2|0|0| +13994|F|S|Unknown|10000|Low Risk|2|0|0| +13995|M|D|Unknown|10000|Low Risk|2|0|0| +13996|F|D|Unknown|10000|Low Risk|2|0|0| +13997|M|W|Unknown|10000|Low Risk|2|0|0| +13998|F|W|Unknown|10000|Low Risk|2|0|0| +13999|M|U|Unknown|10000|Low Risk|2|0|0| +14000|F|U|Unknown|10000|Low Risk|2|0|0| +14001|M|M|Primary|500|High Risk|2|0|0| +14002|F|M|Primary|500|High Risk|2|0|0| +14003|M|S|Primary|500|High Risk|2|0|0| +14004|F|S|Primary|500|High Risk|2|0|0| +14005|M|D|Primary|500|High Risk|2|0|0| +14006|F|D|Primary|500|High Risk|2|0|0| +14007|M|W|Primary|500|High Risk|2|0|0| +14008|F|W|Primary|500|High Risk|2|0|0| +14009|M|U|Primary|500|High Risk|2|0|0| +14010|F|U|Primary|500|High Risk|2|0|0| +14011|M|M|Secondary|500|High Risk|2|0|0| +14012|F|M|Secondary|500|High Risk|2|0|0| +14013|M|S|Secondary|500|High Risk|2|0|0| +14014|F|S|Secondary|500|High Risk|2|0|0| +14015|M|D|Secondary|500|High Risk|2|0|0| +14016|F|D|Secondary|500|High Risk|2|0|0| +14017|M|W|Secondary|500|High Risk|2|0|0| +14018|F|W|Secondary|500|High Risk|2|0|0| +14019|M|U|Secondary|500|High Risk|2|0|0| +14020|F|U|Secondary|500|High Risk|2|0|0| +14021|M|M|College|500|High Risk|2|0|0| +14022|F|M|College|500|High Risk|2|0|0| +14023|M|S|College|500|High Risk|2|0|0| +14024|F|S|College|500|High Risk|2|0|0| +14025|M|D|College|500|High Risk|2|0|0| +14026|F|D|College|500|High Risk|2|0|0| +14027|M|W|College|500|High Risk|2|0|0| +14028|F|W|College|500|High Risk|2|0|0| +14029|M|U|College|500|High Risk|2|0|0| +14030|F|U|College|500|High Risk|2|0|0| +14031|M|M|2 yr Degree|500|High Risk|2|0|0| +14032|F|M|2 yr Degree|500|High Risk|2|0|0| +14033|M|S|2 yr Degree|500|High Risk|2|0|0| +14034|F|S|2 yr Degree|500|High Risk|2|0|0| +14035|M|D|2 yr Degree|500|High Risk|2|0|0| +14036|F|D|2 yr Degree|500|High Risk|2|0|0| +14037|M|W|2 yr Degree|500|High Risk|2|0|0| +14038|F|W|2 yr Degree|500|High Risk|2|0|0| +14039|M|U|2 yr Degree|500|High Risk|2|0|0| +14040|F|U|2 yr Degree|500|High Risk|2|0|0| +14041|M|M|4 yr Degree|500|High Risk|2|0|0| +14042|F|M|4 yr Degree|500|High Risk|2|0|0| +14043|M|S|4 yr Degree|500|High Risk|2|0|0| +14044|F|S|4 yr Degree|500|High Risk|2|0|0| +14045|M|D|4 yr Degree|500|High Risk|2|0|0| +14046|F|D|4 yr Degree|500|High Risk|2|0|0| +14047|M|W|4 yr Degree|500|High Risk|2|0|0| +14048|F|W|4 yr Degree|500|High Risk|2|0|0| +14049|M|U|4 yr Degree|500|High Risk|2|0|0| +14050|F|U|4 yr Degree|500|High Risk|2|0|0| +14051|M|M|Advanced Degree|500|High Risk|2|0|0| +14052|F|M|Advanced Degree|500|High Risk|2|0|0| +14053|M|S|Advanced Degree|500|High Risk|2|0|0| +14054|F|S|Advanced Degree|500|High Risk|2|0|0| +14055|M|D|Advanced Degree|500|High Risk|2|0|0| +14056|F|D|Advanced Degree|500|High Risk|2|0|0| +14057|M|W|Advanced Degree|500|High Risk|2|0|0| +14058|F|W|Advanced Degree|500|High Risk|2|0|0| +14059|M|U|Advanced Degree|500|High Risk|2|0|0| +14060|F|U|Advanced Degree|500|High Risk|2|0|0| +14061|M|M|Unknown|500|High Risk|2|0|0| +14062|F|M|Unknown|500|High Risk|2|0|0| +14063|M|S|Unknown|500|High Risk|2|0|0| +14064|F|S|Unknown|500|High Risk|2|0|0| +14065|M|D|Unknown|500|High Risk|2|0|0| +14066|F|D|Unknown|500|High Risk|2|0|0| +14067|M|W|Unknown|500|High Risk|2|0|0| +14068|F|W|Unknown|500|High Risk|2|0|0| +14069|M|U|Unknown|500|High Risk|2|0|0| +14070|F|U|Unknown|500|High Risk|2|0|0| +14071|M|M|Primary|1000|High Risk|2|0|0| +14072|F|M|Primary|1000|High Risk|2|0|0| +14073|M|S|Primary|1000|High Risk|2|0|0| +14074|F|S|Primary|1000|High Risk|2|0|0| +14075|M|D|Primary|1000|High Risk|2|0|0| +14076|F|D|Primary|1000|High Risk|2|0|0| +14077|M|W|Primary|1000|High Risk|2|0|0| +14078|F|W|Primary|1000|High Risk|2|0|0| +14079|M|U|Primary|1000|High Risk|2|0|0| +14080|F|U|Primary|1000|High Risk|2|0|0| +14081|M|M|Secondary|1000|High Risk|2|0|0| +14082|F|M|Secondary|1000|High Risk|2|0|0| +14083|M|S|Secondary|1000|High Risk|2|0|0| +14084|F|S|Secondary|1000|High Risk|2|0|0| +14085|M|D|Secondary|1000|High Risk|2|0|0| +14086|F|D|Secondary|1000|High Risk|2|0|0| +14087|M|W|Secondary|1000|High Risk|2|0|0| +14088|F|W|Secondary|1000|High Risk|2|0|0| +14089|M|U|Secondary|1000|High Risk|2|0|0| +14090|F|U|Secondary|1000|High Risk|2|0|0| +14091|M|M|College|1000|High Risk|2|0|0| +14092|F|M|College|1000|High Risk|2|0|0| +14093|M|S|College|1000|High Risk|2|0|0| +14094|F|S|College|1000|High Risk|2|0|0| +14095|M|D|College|1000|High Risk|2|0|0| +14096|F|D|College|1000|High Risk|2|0|0| +14097|M|W|College|1000|High Risk|2|0|0| +14098|F|W|College|1000|High Risk|2|0|0| +14099|M|U|College|1000|High Risk|2|0|0| +14100|F|U|College|1000|High Risk|2|0|0| +14101|M|M|2 yr Degree|1000|High Risk|2|0|0| +14102|F|M|2 yr Degree|1000|High Risk|2|0|0| +14103|M|S|2 yr Degree|1000|High Risk|2|0|0| +14104|F|S|2 yr Degree|1000|High Risk|2|0|0| +14105|M|D|2 yr Degree|1000|High Risk|2|0|0| +14106|F|D|2 yr Degree|1000|High Risk|2|0|0| +14107|M|W|2 yr Degree|1000|High Risk|2|0|0| +14108|F|W|2 yr Degree|1000|High Risk|2|0|0| +14109|M|U|2 yr Degree|1000|High Risk|2|0|0| +14110|F|U|2 yr Degree|1000|High Risk|2|0|0| +14111|M|M|4 yr Degree|1000|High Risk|2|0|0| +14112|F|M|4 yr Degree|1000|High Risk|2|0|0| +14113|M|S|4 yr Degree|1000|High Risk|2|0|0| +14114|F|S|4 yr Degree|1000|High Risk|2|0|0| +14115|M|D|4 yr Degree|1000|High Risk|2|0|0| +14116|F|D|4 yr Degree|1000|High Risk|2|0|0| +14117|M|W|4 yr Degree|1000|High Risk|2|0|0| +14118|F|W|4 yr Degree|1000|High Risk|2|0|0| +14119|M|U|4 yr Degree|1000|High Risk|2|0|0| +14120|F|U|4 yr Degree|1000|High Risk|2|0|0| +14121|M|M|Advanced Degree|1000|High Risk|2|0|0| +14122|F|M|Advanced Degree|1000|High Risk|2|0|0| +14123|M|S|Advanced Degree|1000|High Risk|2|0|0| +14124|F|S|Advanced Degree|1000|High Risk|2|0|0| +14125|M|D|Advanced Degree|1000|High Risk|2|0|0| +14126|F|D|Advanced Degree|1000|High Risk|2|0|0| +14127|M|W|Advanced Degree|1000|High Risk|2|0|0| +14128|F|W|Advanced Degree|1000|High Risk|2|0|0| +14129|M|U|Advanced Degree|1000|High Risk|2|0|0| +14130|F|U|Advanced Degree|1000|High Risk|2|0|0| +14131|M|M|Unknown|1000|High Risk|2|0|0| +14132|F|M|Unknown|1000|High Risk|2|0|0| +14133|M|S|Unknown|1000|High Risk|2|0|0| +14134|F|S|Unknown|1000|High Risk|2|0|0| +14135|M|D|Unknown|1000|High Risk|2|0|0| +14136|F|D|Unknown|1000|High Risk|2|0|0| +14137|M|W|Unknown|1000|High Risk|2|0|0| +14138|F|W|Unknown|1000|High Risk|2|0|0| +14139|M|U|Unknown|1000|High Risk|2|0|0| +14140|F|U|Unknown|1000|High Risk|2|0|0| +14141|M|M|Primary|1500|High Risk|2|0|0| +14142|F|M|Primary|1500|High Risk|2|0|0| +14143|M|S|Primary|1500|High Risk|2|0|0| +14144|F|S|Primary|1500|High Risk|2|0|0| +14145|M|D|Primary|1500|High Risk|2|0|0| +14146|F|D|Primary|1500|High Risk|2|0|0| +14147|M|W|Primary|1500|High Risk|2|0|0| +14148|F|W|Primary|1500|High Risk|2|0|0| +14149|M|U|Primary|1500|High Risk|2|0|0| +14150|F|U|Primary|1500|High Risk|2|0|0| +14151|M|M|Secondary|1500|High Risk|2|0|0| +14152|F|M|Secondary|1500|High Risk|2|0|0| +14153|M|S|Secondary|1500|High Risk|2|0|0| +14154|F|S|Secondary|1500|High Risk|2|0|0| +14155|M|D|Secondary|1500|High Risk|2|0|0| +14156|F|D|Secondary|1500|High Risk|2|0|0| +14157|M|W|Secondary|1500|High Risk|2|0|0| +14158|F|W|Secondary|1500|High Risk|2|0|0| +14159|M|U|Secondary|1500|High Risk|2|0|0| +14160|F|U|Secondary|1500|High Risk|2|0|0| +14161|M|M|College|1500|High Risk|2|0|0| +14162|F|M|College|1500|High Risk|2|0|0| +14163|M|S|College|1500|High Risk|2|0|0| +14164|F|S|College|1500|High Risk|2|0|0| +14165|M|D|College|1500|High Risk|2|0|0| +14166|F|D|College|1500|High Risk|2|0|0| +14167|M|W|College|1500|High Risk|2|0|0| +14168|F|W|College|1500|High Risk|2|0|0| +14169|M|U|College|1500|High Risk|2|0|0| +14170|F|U|College|1500|High Risk|2|0|0| +14171|M|M|2 yr Degree|1500|High Risk|2|0|0| +14172|F|M|2 yr Degree|1500|High Risk|2|0|0| +14173|M|S|2 yr Degree|1500|High Risk|2|0|0| +14174|F|S|2 yr Degree|1500|High Risk|2|0|0| +14175|M|D|2 yr Degree|1500|High Risk|2|0|0| +14176|F|D|2 yr Degree|1500|High Risk|2|0|0| +14177|M|W|2 yr Degree|1500|High Risk|2|0|0| +14178|F|W|2 yr Degree|1500|High Risk|2|0|0| +14179|M|U|2 yr Degree|1500|High Risk|2|0|0| +14180|F|U|2 yr Degree|1500|High Risk|2|0|0| +14181|M|M|4 yr Degree|1500|High Risk|2|0|0| +14182|F|M|4 yr Degree|1500|High Risk|2|0|0| +14183|M|S|4 yr Degree|1500|High Risk|2|0|0| +14184|F|S|4 yr Degree|1500|High Risk|2|0|0| +14185|M|D|4 yr Degree|1500|High Risk|2|0|0| +14186|F|D|4 yr Degree|1500|High Risk|2|0|0| +14187|M|W|4 yr Degree|1500|High Risk|2|0|0| +14188|F|W|4 yr Degree|1500|High Risk|2|0|0| +14189|M|U|4 yr Degree|1500|High Risk|2|0|0| +14190|F|U|4 yr Degree|1500|High Risk|2|0|0| +14191|M|M|Advanced Degree|1500|High Risk|2|0|0| +14192|F|M|Advanced Degree|1500|High Risk|2|0|0| +14193|M|S|Advanced Degree|1500|High Risk|2|0|0| +14194|F|S|Advanced Degree|1500|High Risk|2|0|0| +14195|M|D|Advanced Degree|1500|High Risk|2|0|0| +14196|F|D|Advanced Degree|1500|High Risk|2|0|0| +14197|M|W|Advanced Degree|1500|High Risk|2|0|0| +14198|F|W|Advanced Degree|1500|High Risk|2|0|0| +14199|M|U|Advanced Degree|1500|High Risk|2|0|0| +14200|F|U|Advanced Degree|1500|High Risk|2|0|0| +14201|M|M|Unknown|1500|High Risk|2|0|0| +14202|F|M|Unknown|1500|High Risk|2|0|0| +14203|M|S|Unknown|1500|High Risk|2|0|0| +14204|F|S|Unknown|1500|High Risk|2|0|0| +14205|M|D|Unknown|1500|High Risk|2|0|0| +14206|F|D|Unknown|1500|High Risk|2|0|0| +14207|M|W|Unknown|1500|High Risk|2|0|0| +14208|F|W|Unknown|1500|High Risk|2|0|0| +14209|M|U|Unknown|1500|High Risk|2|0|0| +14210|F|U|Unknown|1500|High Risk|2|0|0| +14211|M|M|Primary|2000|High Risk|2|0|0| +14212|F|M|Primary|2000|High Risk|2|0|0| +14213|M|S|Primary|2000|High Risk|2|0|0| +14214|F|S|Primary|2000|High Risk|2|0|0| +14215|M|D|Primary|2000|High Risk|2|0|0| +14216|F|D|Primary|2000|High Risk|2|0|0| +14217|M|W|Primary|2000|High Risk|2|0|0| +14218|F|W|Primary|2000|High Risk|2|0|0| +14219|M|U|Primary|2000|High Risk|2|0|0| +14220|F|U|Primary|2000|High Risk|2|0|0| +14221|M|M|Secondary|2000|High Risk|2|0|0| +14222|F|M|Secondary|2000|High Risk|2|0|0| +14223|M|S|Secondary|2000|High Risk|2|0|0| +14224|F|S|Secondary|2000|High Risk|2|0|0| +14225|M|D|Secondary|2000|High Risk|2|0|0| +14226|F|D|Secondary|2000|High Risk|2|0|0| +14227|M|W|Secondary|2000|High Risk|2|0|0| +14228|F|W|Secondary|2000|High Risk|2|0|0| +14229|M|U|Secondary|2000|High Risk|2|0|0| +14230|F|U|Secondary|2000|High Risk|2|0|0| +14231|M|M|College|2000|High Risk|2|0|0| +14232|F|M|College|2000|High Risk|2|0|0| +14233|M|S|College|2000|High Risk|2|0|0| +14234|F|S|College|2000|High Risk|2|0|0| +14235|M|D|College|2000|High Risk|2|0|0| +14236|F|D|College|2000|High Risk|2|0|0| +14237|M|W|College|2000|High Risk|2|0|0| +14238|F|W|College|2000|High Risk|2|0|0| +14239|M|U|College|2000|High Risk|2|0|0| +14240|F|U|College|2000|High Risk|2|0|0| +14241|M|M|2 yr Degree|2000|High Risk|2|0|0| +14242|F|M|2 yr Degree|2000|High Risk|2|0|0| +14243|M|S|2 yr Degree|2000|High Risk|2|0|0| +14244|F|S|2 yr Degree|2000|High Risk|2|0|0| +14245|M|D|2 yr Degree|2000|High Risk|2|0|0| +14246|F|D|2 yr Degree|2000|High Risk|2|0|0| +14247|M|W|2 yr Degree|2000|High Risk|2|0|0| +14248|F|W|2 yr Degree|2000|High Risk|2|0|0| +14249|M|U|2 yr Degree|2000|High Risk|2|0|0| +14250|F|U|2 yr Degree|2000|High Risk|2|0|0| +14251|M|M|4 yr Degree|2000|High Risk|2|0|0| +14252|F|M|4 yr Degree|2000|High Risk|2|0|0| +14253|M|S|4 yr Degree|2000|High Risk|2|0|0| +14254|F|S|4 yr Degree|2000|High Risk|2|0|0| +14255|M|D|4 yr Degree|2000|High Risk|2|0|0| +14256|F|D|4 yr Degree|2000|High Risk|2|0|0| +14257|M|W|4 yr Degree|2000|High Risk|2|0|0| +14258|F|W|4 yr Degree|2000|High Risk|2|0|0| +14259|M|U|4 yr Degree|2000|High Risk|2|0|0| +14260|F|U|4 yr Degree|2000|High Risk|2|0|0| +14261|M|M|Advanced Degree|2000|High Risk|2|0|0| +14262|F|M|Advanced Degree|2000|High Risk|2|0|0| +14263|M|S|Advanced Degree|2000|High Risk|2|0|0| +14264|F|S|Advanced Degree|2000|High Risk|2|0|0| +14265|M|D|Advanced Degree|2000|High Risk|2|0|0| +14266|F|D|Advanced Degree|2000|High Risk|2|0|0| +14267|M|W|Advanced Degree|2000|High Risk|2|0|0| +14268|F|W|Advanced Degree|2000|High Risk|2|0|0| +14269|M|U|Advanced Degree|2000|High Risk|2|0|0| +14270|F|U|Advanced Degree|2000|High Risk|2|0|0| +14271|M|M|Unknown|2000|High Risk|2|0|0| +14272|F|M|Unknown|2000|High Risk|2|0|0| +14273|M|S|Unknown|2000|High Risk|2|0|0| +14274|F|S|Unknown|2000|High Risk|2|0|0| +14275|M|D|Unknown|2000|High Risk|2|0|0| +14276|F|D|Unknown|2000|High Risk|2|0|0| +14277|M|W|Unknown|2000|High Risk|2|0|0| +14278|F|W|Unknown|2000|High Risk|2|0|0| +14279|M|U|Unknown|2000|High Risk|2|0|0| +14280|F|U|Unknown|2000|High Risk|2|0|0| +14281|M|M|Primary|2500|High Risk|2|0|0| +14282|F|M|Primary|2500|High Risk|2|0|0| +14283|M|S|Primary|2500|High Risk|2|0|0| +14284|F|S|Primary|2500|High Risk|2|0|0| +14285|M|D|Primary|2500|High Risk|2|0|0| +14286|F|D|Primary|2500|High Risk|2|0|0| +14287|M|W|Primary|2500|High Risk|2|0|0| +14288|F|W|Primary|2500|High Risk|2|0|0| +14289|M|U|Primary|2500|High Risk|2|0|0| +14290|F|U|Primary|2500|High Risk|2|0|0| +14291|M|M|Secondary|2500|High Risk|2|0|0| +14292|F|M|Secondary|2500|High Risk|2|0|0| +14293|M|S|Secondary|2500|High Risk|2|0|0| +14294|F|S|Secondary|2500|High Risk|2|0|0| +14295|M|D|Secondary|2500|High Risk|2|0|0| +14296|F|D|Secondary|2500|High Risk|2|0|0| +14297|M|W|Secondary|2500|High Risk|2|0|0| +14298|F|W|Secondary|2500|High Risk|2|0|0| +14299|M|U|Secondary|2500|High Risk|2|0|0| +14300|F|U|Secondary|2500|High Risk|2|0|0| +14301|M|M|College|2500|High Risk|2|0|0| +14302|F|M|College|2500|High Risk|2|0|0| +14303|M|S|College|2500|High Risk|2|0|0| +14304|F|S|College|2500|High Risk|2|0|0| +14305|M|D|College|2500|High Risk|2|0|0| +14306|F|D|College|2500|High Risk|2|0|0| +14307|M|W|College|2500|High Risk|2|0|0| +14308|F|W|College|2500|High Risk|2|0|0| +14309|M|U|College|2500|High Risk|2|0|0| +14310|F|U|College|2500|High Risk|2|0|0| +14311|M|M|2 yr Degree|2500|High Risk|2|0|0| +14312|F|M|2 yr Degree|2500|High Risk|2|0|0| +14313|M|S|2 yr Degree|2500|High Risk|2|0|0| +14314|F|S|2 yr Degree|2500|High Risk|2|0|0| +14315|M|D|2 yr Degree|2500|High Risk|2|0|0| +14316|F|D|2 yr Degree|2500|High Risk|2|0|0| +14317|M|W|2 yr Degree|2500|High Risk|2|0|0| +14318|F|W|2 yr Degree|2500|High Risk|2|0|0| +14319|M|U|2 yr Degree|2500|High Risk|2|0|0| +14320|F|U|2 yr Degree|2500|High Risk|2|0|0| +14321|M|M|4 yr Degree|2500|High Risk|2|0|0| +14322|F|M|4 yr Degree|2500|High Risk|2|0|0| +14323|M|S|4 yr Degree|2500|High Risk|2|0|0| +14324|F|S|4 yr Degree|2500|High Risk|2|0|0| +14325|M|D|4 yr Degree|2500|High Risk|2|0|0| +14326|F|D|4 yr Degree|2500|High Risk|2|0|0| +14327|M|W|4 yr Degree|2500|High Risk|2|0|0| +14328|F|W|4 yr Degree|2500|High Risk|2|0|0| +14329|M|U|4 yr Degree|2500|High Risk|2|0|0| +14330|F|U|4 yr Degree|2500|High Risk|2|0|0| +14331|M|M|Advanced Degree|2500|High Risk|2|0|0| +14332|F|M|Advanced Degree|2500|High Risk|2|0|0| +14333|M|S|Advanced Degree|2500|High Risk|2|0|0| +14334|F|S|Advanced Degree|2500|High Risk|2|0|0| +14335|M|D|Advanced Degree|2500|High Risk|2|0|0| +14336|F|D|Advanced Degree|2500|High Risk|2|0|0| +14337|M|W|Advanced Degree|2500|High Risk|2|0|0| +14338|F|W|Advanced Degree|2500|High Risk|2|0|0| +14339|M|U|Advanced Degree|2500|High Risk|2|0|0| +14340|F|U|Advanced Degree|2500|High Risk|2|0|0| +14341|M|M|Unknown|2500|High Risk|2|0|0| +14342|F|M|Unknown|2500|High Risk|2|0|0| +14343|M|S|Unknown|2500|High Risk|2|0|0| +14344|F|S|Unknown|2500|High Risk|2|0|0| +14345|M|D|Unknown|2500|High Risk|2|0|0| +14346|F|D|Unknown|2500|High Risk|2|0|0| +14347|M|W|Unknown|2500|High Risk|2|0|0| +14348|F|W|Unknown|2500|High Risk|2|0|0| +14349|M|U|Unknown|2500|High Risk|2|0|0| +14350|F|U|Unknown|2500|High Risk|2|0|0| +14351|M|M|Primary|3000|High Risk|2|0|0| +14352|F|M|Primary|3000|High Risk|2|0|0| +14353|M|S|Primary|3000|High Risk|2|0|0| +14354|F|S|Primary|3000|High Risk|2|0|0| +14355|M|D|Primary|3000|High Risk|2|0|0| +14356|F|D|Primary|3000|High Risk|2|0|0| +14357|M|W|Primary|3000|High Risk|2|0|0| +14358|F|W|Primary|3000|High Risk|2|0|0| +14359|M|U|Primary|3000|High Risk|2|0|0| +14360|F|U|Primary|3000|High Risk|2|0|0| +14361|M|M|Secondary|3000|High Risk|2|0|0| +14362|F|M|Secondary|3000|High Risk|2|0|0| +14363|M|S|Secondary|3000|High Risk|2|0|0| +14364|F|S|Secondary|3000|High Risk|2|0|0| +14365|M|D|Secondary|3000|High Risk|2|0|0| +14366|F|D|Secondary|3000|High Risk|2|0|0| +14367|M|W|Secondary|3000|High Risk|2|0|0| +14368|F|W|Secondary|3000|High Risk|2|0|0| +14369|M|U|Secondary|3000|High Risk|2|0|0| +14370|F|U|Secondary|3000|High Risk|2|0|0| +14371|M|M|College|3000|High Risk|2|0|0| +14372|F|M|College|3000|High Risk|2|0|0| +14373|M|S|College|3000|High Risk|2|0|0| +14374|F|S|College|3000|High Risk|2|0|0| +14375|M|D|College|3000|High Risk|2|0|0| +14376|F|D|College|3000|High Risk|2|0|0| +14377|M|W|College|3000|High Risk|2|0|0| +14378|F|W|College|3000|High Risk|2|0|0| +14379|M|U|College|3000|High Risk|2|0|0| +14380|F|U|College|3000|High Risk|2|0|0| +14381|M|M|2 yr Degree|3000|High Risk|2|0|0| +14382|F|M|2 yr Degree|3000|High Risk|2|0|0| +14383|M|S|2 yr Degree|3000|High Risk|2|0|0| +14384|F|S|2 yr Degree|3000|High Risk|2|0|0| +14385|M|D|2 yr Degree|3000|High Risk|2|0|0| +14386|F|D|2 yr Degree|3000|High Risk|2|0|0| +14387|M|W|2 yr Degree|3000|High Risk|2|0|0| +14388|F|W|2 yr Degree|3000|High Risk|2|0|0| +14389|M|U|2 yr Degree|3000|High Risk|2|0|0| +14390|F|U|2 yr Degree|3000|High Risk|2|0|0| +14391|M|M|4 yr Degree|3000|High Risk|2|0|0| +14392|F|M|4 yr Degree|3000|High Risk|2|0|0| +14393|M|S|4 yr Degree|3000|High Risk|2|0|0| +14394|F|S|4 yr Degree|3000|High Risk|2|0|0| +14395|M|D|4 yr Degree|3000|High Risk|2|0|0| +14396|F|D|4 yr Degree|3000|High Risk|2|0|0| +14397|M|W|4 yr Degree|3000|High Risk|2|0|0| +14398|F|W|4 yr Degree|3000|High Risk|2|0|0| +14399|M|U|4 yr Degree|3000|High Risk|2|0|0| +14400|F|U|4 yr Degree|3000|High Risk|2|0|0| +14401|M|M|Advanced Degree|3000|High Risk|2|0|0| +14402|F|M|Advanced Degree|3000|High Risk|2|0|0| +14403|M|S|Advanced Degree|3000|High Risk|2|0|0| +14404|F|S|Advanced Degree|3000|High Risk|2|0|0| +14405|M|D|Advanced Degree|3000|High Risk|2|0|0| +14406|F|D|Advanced Degree|3000|High Risk|2|0|0| +14407|M|W|Advanced Degree|3000|High Risk|2|0|0| +14408|F|W|Advanced Degree|3000|High Risk|2|0|0| +14409|M|U|Advanced Degree|3000|High Risk|2|0|0| +14410|F|U|Advanced Degree|3000|High Risk|2|0|0| +14411|M|M|Unknown|3000|High Risk|2|0|0| +14412|F|M|Unknown|3000|High Risk|2|0|0| +14413|M|S|Unknown|3000|High Risk|2|0|0| +14414|F|S|Unknown|3000|High Risk|2|0|0| +14415|M|D|Unknown|3000|High Risk|2|0|0| +14416|F|D|Unknown|3000|High Risk|2|0|0| +14417|M|W|Unknown|3000|High Risk|2|0|0| +14418|F|W|Unknown|3000|High Risk|2|0|0| +14419|M|U|Unknown|3000|High Risk|2|0|0| +14420|F|U|Unknown|3000|High Risk|2|0|0| +14421|M|M|Primary|3500|High Risk|2|0|0| +14422|F|M|Primary|3500|High Risk|2|0|0| +14423|M|S|Primary|3500|High Risk|2|0|0| +14424|F|S|Primary|3500|High Risk|2|0|0| +14425|M|D|Primary|3500|High Risk|2|0|0| +14426|F|D|Primary|3500|High Risk|2|0|0| +14427|M|W|Primary|3500|High Risk|2|0|0| +14428|F|W|Primary|3500|High Risk|2|0|0| +14429|M|U|Primary|3500|High Risk|2|0|0| +14430|F|U|Primary|3500|High Risk|2|0|0| +14431|M|M|Secondary|3500|High Risk|2|0|0| +14432|F|M|Secondary|3500|High Risk|2|0|0| +14433|M|S|Secondary|3500|High Risk|2|0|0| +14434|F|S|Secondary|3500|High Risk|2|0|0| +14435|M|D|Secondary|3500|High Risk|2|0|0| +14436|F|D|Secondary|3500|High Risk|2|0|0| +14437|M|W|Secondary|3500|High Risk|2|0|0| +14438|F|W|Secondary|3500|High Risk|2|0|0| +14439|M|U|Secondary|3500|High Risk|2|0|0| +14440|F|U|Secondary|3500|High Risk|2|0|0| +14441|M|M|College|3500|High Risk|2|0|0| +14442|F|M|College|3500|High Risk|2|0|0| +14443|M|S|College|3500|High Risk|2|0|0| +14444|F|S|College|3500|High Risk|2|0|0| +14445|M|D|College|3500|High Risk|2|0|0| +14446|F|D|College|3500|High Risk|2|0|0| +14447|M|W|College|3500|High Risk|2|0|0| +14448|F|W|College|3500|High Risk|2|0|0| +14449|M|U|College|3500|High Risk|2|0|0| +14450|F|U|College|3500|High Risk|2|0|0| +14451|M|M|2 yr Degree|3500|High Risk|2|0|0| +14452|F|M|2 yr Degree|3500|High Risk|2|0|0| +14453|M|S|2 yr Degree|3500|High Risk|2|0|0| +14454|F|S|2 yr Degree|3500|High Risk|2|0|0| +14455|M|D|2 yr Degree|3500|High Risk|2|0|0| +14456|F|D|2 yr Degree|3500|High Risk|2|0|0| +14457|M|W|2 yr Degree|3500|High Risk|2|0|0| +14458|F|W|2 yr Degree|3500|High Risk|2|0|0| +14459|M|U|2 yr Degree|3500|High Risk|2|0|0| +14460|F|U|2 yr Degree|3500|High Risk|2|0|0| +14461|M|M|4 yr Degree|3500|High Risk|2|0|0| +14462|F|M|4 yr Degree|3500|High Risk|2|0|0| +14463|M|S|4 yr Degree|3500|High Risk|2|0|0| +14464|F|S|4 yr Degree|3500|High Risk|2|0|0| +14465|M|D|4 yr Degree|3500|High Risk|2|0|0| +14466|F|D|4 yr Degree|3500|High Risk|2|0|0| +14467|M|W|4 yr Degree|3500|High Risk|2|0|0| +14468|F|W|4 yr Degree|3500|High Risk|2|0|0| +14469|M|U|4 yr Degree|3500|High Risk|2|0|0| +14470|F|U|4 yr Degree|3500|High Risk|2|0|0| +14471|M|M|Advanced Degree|3500|High Risk|2|0|0| +14472|F|M|Advanced Degree|3500|High Risk|2|0|0| +14473|M|S|Advanced Degree|3500|High Risk|2|0|0| +14474|F|S|Advanced Degree|3500|High Risk|2|0|0| +14475|M|D|Advanced Degree|3500|High Risk|2|0|0| +14476|F|D|Advanced Degree|3500|High Risk|2|0|0| +14477|M|W|Advanced Degree|3500|High Risk|2|0|0| +14478|F|W|Advanced Degree|3500|High Risk|2|0|0| +14479|M|U|Advanced Degree|3500|High Risk|2|0|0| +14480|F|U|Advanced Degree|3500|High Risk|2|0|0| +14481|M|M|Unknown|3500|High Risk|2|0|0| +14482|F|M|Unknown|3500|High Risk|2|0|0| +14483|M|S|Unknown|3500|High Risk|2|0|0| +14484|F|S|Unknown|3500|High Risk|2|0|0| +14485|M|D|Unknown|3500|High Risk|2|0|0| +14486|F|D|Unknown|3500|High Risk|2|0|0| +14487|M|W|Unknown|3500|High Risk|2|0|0| +14488|F|W|Unknown|3500|High Risk|2|0|0| +14489|M|U|Unknown|3500|High Risk|2|0|0| +14490|F|U|Unknown|3500|High Risk|2|0|0| +14491|M|M|Primary|4000|High Risk|2|0|0| +14492|F|M|Primary|4000|High Risk|2|0|0| +14493|M|S|Primary|4000|High Risk|2|0|0| +14494|F|S|Primary|4000|High Risk|2|0|0| +14495|M|D|Primary|4000|High Risk|2|0|0| +14496|F|D|Primary|4000|High Risk|2|0|0| +14497|M|W|Primary|4000|High Risk|2|0|0| +14498|F|W|Primary|4000|High Risk|2|0|0| +14499|M|U|Primary|4000|High Risk|2|0|0| +14500|F|U|Primary|4000|High Risk|2|0|0| +14501|M|M|Secondary|4000|High Risk|2|0|0| +14502|F|M|Secondary|4000|High Risk|2|0|0| +14503|M|S|Secondary|4000|High Risk|2|0|0| +14504|F|S|Secondary|4000|High Risk|2|0|0| +14505|M|D|Secondary|4000|High Risk|2|0|0| +14506|F|D|Secondary|4000|High Risk|2|0|0| +14507|M|W|Secondary|4000|High Risk|2|0|0| +14508|F|W|Secondary|4000|High Risk|2|0|0| +14509|M|U|Secondary|4000|High Risk|2|0|0| +14510|F|U|Secondary|4000|High Risk|2|0|0| +14511|M|M|College|4000|High Risk|2|0|0| +14512|F|M|College|4000|High Risk|2|0|0| +14513|M|S|College|4000|High Risk|2|0|0| +14514|F|S|College|4000|High Risk|2|0|0| +14515|M|D|College|4000|High Risk|2|0|0| +14516|F|D|College|4000|High Risk|2|0|0| +14517|M|W|College|4000|High Risk|2|0|0| +14518|F|W|College|4000|High Risk|2|0|0| +14519|M|U|College|4000|High Risk|2|0|0| +14520|F|U|College|4000|High Risk|2|0|0| +14521|M|M|2 yr Degree|4000|High Risk|2|0|0| +14522|F|M|2 yr Degree|4000|High Risk|2|0|0| +14523|M|S|2 yr Degree|4000|High Risk|2|0|0| +14524|F|S|2 yr Degree|4000|High Risk|2|0|0| +14525|M|D|2 yr Degree|4000|High Risk|2|0|0| +14526|F|D|2 yr Degree|4000|High Risk|2|0|0| +14527|M|W|2 yr Degree|4000|High Risk|2|0|0| +14528|F|W|2 yr Degree|4000|High Risk|2|0|0| +14529|M|U|2 yr Degree|4000|High Risk|2|0|0| +14530|F|U|2 yr Degree|4000|High Risk|2|0|0| +14531|M|M|4 yr Degree|4000|High Risk|2|0|0| +14532|F|M|4 yr Degree|4000|High Risk|2|0|0| +14533|M|S|4 yr Degree|4000|High Risk|2|0|0| +14534|F|S|4 yr Degree|4000|High Risk|2|0|0| +14535|M|D|4 yr Degree|4000|High Risk|2|0|0| +14536|F|D|4 yr Degree|4000|High Risk|2|0|0| +14537|M|W|4 yr Degree|4000|High Risk|2|0|0| +14538|F|W|4 yr Degree|4000|High Risk|2|0|0| +14539|M|U|4 yr Degree|4000|High Risk|2|0|0| +14540|F|U|4 yr Degree|4000|High Risk|2|0|0| +14541|M|M|Advanced Degree|4000|High Risk|2|0|0| +14542|F|M|Advanced Degree|4000|High Risk|2|0|0| +14543|M|S|Advanced Degree|4000|High Risk|2|0|0| +14544|F|S|Advanced Degree|4000|High Risk|2|0|0| +14545|M|D|Advanced Degree|4000|High Risk|2|0|0| +14546|F|D|Advanced Degree|4000|High Risk|2|0|0| +14547|M|W|Advanced Degree|4000|High Risk|2|0|0| +14548|F|W|Advanced Degree|4000|High Risk|2|0|0| +14549|M|U|Advanced Degree|4000|High Risk|2|0|0| +14550|F|U|Advanced Degree|4000|High Risk|2|0|0| +14551|M|M|Unknown|4000|High Risk|2|0|0| +14552|F|M|Unknown|4000|High Risk|2|0|0| +14553|M|S|Unknown|4000|High Risk|2|0|0| +14554|F|S|Unknown|4000|High Risk|2|0|0| +14555|M|D|Unknown|4000|High Risk|2|0|0| +14556|F|D|Unknown|4000|High Risk|2|0|0| +14557|M|W|Unknown|4000|High Risk|2|0|0| +14558|F|W|Unknown|4000|High Risk|2|0|0| +14559|M|U|Unknown|4000|High Risk|2|0|0| +14560|F|U|Unknown|4000|High Risk|2|0|0| +14561|M|M|Primary|4500|High Risk|2|0|0| +14562|F|M|Primary|4500|High Risk|2|0|0| +14563|M|S|Primary|4500|High Risk|2|0|0| +14564|F|S|Primary|4500|High Risk|2|0|0| +14565|M|D|Primary|4500|High Risk|2|0|0| +14566|F|D|Primary|4500|High Risk|2|0|0| +14567|M|W|Primary|4500|High Risk|2|0|0| +14568|F|W|Primary|4500|High Risk|2|0|0| +14569|M|U|Primary|4500|High Risk|2|0|0| +14570|F|U|Primary|4500|High Risk|2|0|0| +14571|M|M|Secondary|4500|High Risk|2|0|0| +14572|F|M|Secondary|4500|High Risk|2|0|0| +14573|M|S|Secondary|4500|High Risk|2|0|0| +14574|F|S|Secondary|4500|High Risk|2|0|0| +14575|M|D|Secondary|4500|High Risk|2|0|0| +14576|F|D|Secondary|4500|High Risk|2|0|0| +14577|M|W|Secondary|4500|High Risk|2|0|0| +14578|F|W|Secondary|4500|High Risk|2|0|0| +14579|M|U|Secondary|4500|High Risk|2|0|0| +14580|F|U|Secondary|4500|High Risk|2|0|0| +14581|M|M|College|4500|High Risk|2|0|0| +14582|F|M|College|4500|High Risk|2|0|0| +14583|M|S|College|4500|High Risk|2|0|0| +14584|F|S|College|4500|High Risk|2|0|0| +14585|M|D|College|4500|High Risk|2|0|0| +14586|F|D|College|4500|High Risk|2|0|0| +14587|M|W|College|4500|High Risk|2|0|0| +14588|F|W|College|4500|High Risk|2|0|0| +14589|M|U|College|4500|High Risk|2|0|0| +14590|F|U|College|4500|High Risk|2|0|0| +14591|M|M|2 yr Degree|4500|High Risk|2|0|0| +14592|F|M|2 yr Degree|4500|High Risk|2|0|0| +14593|M|S|2 yr Degree|4500|High Risk|2|0|0| +14594|F|S|2 yr Degree|4500|High Risk|2|0|0| +14595|M|D|2 yr Degree|4500|High Risk|2|0|0| +14596|F|D|2 yr Degree|4500|High Risk|2|0|0| +14597|M|W|2 yr Degree|4500|High Risk|2|0|0| +14598|F|W|2 yr Degree|4500|High Risk|2|0|0| +14599|M|U|2 yr Degree|4500|High Risk|2|0|0| +14600|F|U|2 yr Degree|4500|High Risk|2|0|0| +14601|M|M|4 yr Degree|4500|High Risk|2|0|0| +14602|F|M|4 yr Degree|4500|High Risk|2|0|0| +14603|M|S|4 yr Degree|4500|High Risk|2|0|0| +14604|F|S|4 yr Degree|4500|High Risk|2|0|0| +14605|M|D|4 yr Degree|4500|High Risk|2|0|0| +14606|F|D|4 yr Degree|4500|High Risk|2|0|0| +14607|M|W|4 yr Degree|4500|High Risk|2|0|0| +14608|F|W|4 yr Degree|4500|High Risk|2|0|0| +14609|M|U|4 yr Degree|4500|High Risk|2|0|0| +14610|F|U|4 yr Degree|4500|High Risk|2|0|0| +14611|M|M|Advanced Degree|4500|High Risk|2|0|0| +14612|F|M|Advanced Degree|4500|High Risk|2|0|0| +14613|M|S|Advanced Degree|4500|High Risk|2|0|0| +14614|F|S|Advanced Degree|4500|High Risk|2|0|0| +14615|M|D|Advanced Degree|4500|High Risk|2|0|0| +14616|F|D|Advanced Degree|4500|High Risk|2|0|0| +14617|M|W|Advanced Degree|4500|High Risk|2|0|0| +14618|F|W|Advanced Degree|4500|High Risk|2|0|0| +14619|M|U|Advanced Degree|4500|High Risk|2|0|0| +14620|F|U|Advanced Degree|4500|High Risk|2|0|0| +14621|M|M|Unknown|4500|High Risk|2|0|0| +14622|F|M|Unknown|4500|High Risk|2|0|0| +14623|M|S|Unknown|4500|High Risk|2|0|0| +14624|F|S|Unknown|4500|High Risk|2|0|0| +14625|M|D|Unknown|4500|High Risk|2|0|0| +14626|F|D|Unknown|4500|High Risk|2|0|0| +14627|M|W|Unknown|4500|High Risk|2|0|0| +14628|F|W|Unknown|4500|High Risk|2|0|0| +14629|M|U|Unknown|4500|High Risk|2|0|0| +14630|F|U|Unknown|4500|High Risk|2|0|0| +14631|M|M|Primary|5000|High Risk|2|0|0| +14632|F|M|Primary|5000|High Risk|2|0|0| +14633|M|S|Primary|5000|High Risk|2|0|0| +14634|F|S|Primary|5000|High Risk|2|0|0| +14635|M|D|Primary|5000|High Risk|2|0|0| +14636|F|D|Primary|5000|High Risk|2|0|0| +14637|M|W|Primary|5000|High Risk|2|0|0| +14638|F|W|Primary|5000|High Risk|2|0|0| +14639|M|U|Primary|5000|High Risk|2|0|0| +14640|F|U|Primary|5000|High Risk|2|0|0| +14641|M|M|Secondary|5000|High Risk|2|0|0| +14642|F|M|Secondary|5000|High Risk|2|0|0| +14643|M|S|Secondary|5000|High Risk|2|0|0| +14644|F|S|Secondary|5000|High Risk|2|0|0| +14645|M|D|Secondary|5000|High Risk|2|0|0| +14646|F|D|Secondary|5000|High Risk|2|0|0| +14647|M|W|Secondary|5000|High Risk|2|0|0| +14648|F|W|Secondary|5000|High Risk|2|0|0| +14649|M|U|Secondary|5000|High Risk|2|0|0| +14650|F|U|Secondary|5000|High Risk|2|0|0| +14651|M|M|College|5000|High Risk|2|0|0| +14652|F|M|College|5000|High Risk|2|0|0| +14653|M|S|College|5000|High Risk|2|0|0| +14654|F|S|College|5000|High Risk|2|0|0| +14655|M|D|College|5000|High Risk|2|0|0| +14656|F|D|College|5000|High Risk|2|0|0| +14657|M|W|College|5000|High Risk|2|0|0| +14658|F|W|College|5000|High Risk|2|0|0| +14659|M|U|College|5000|High Risk|2|0|0| +14660|F|U|College|5000|High Risk|2|0|0| +14661|M|M|2 yr Degree|5000|High Risk|2|0|0| +14662|F|M|2 yr Degree|5000|High Risk|2|0|0| +14663|M|S|2 yr Degree|5000|High Risk|2|0|0| +14664|F|S|2 yr Degree|5000|High Risk|2|0|0| +14665|M|D|2 yr Degree|5000|High Risk|2|0|0| +14666|F|D|2 yr Degree|5000|High Risk|2|0|0| +14667|M|W|2 yr Degree|5000|High Risk|2|0|0| +14668|F|W|2 yr Degree|5000|High Risk|2|0|0| +14669|M|U|2 yr Degree|5000|High Risk|2|0|0| +14670|F|U|2 yr Degree|5000|High Risk|2|0|0| +14671|M|M|4 yr Degree|5000|High Risk|2|0|0| +14672|F|M|4 yr Degree|5000|High Risk|2|0|0| +14673|M|S|4 yr Degree|5000|High Risk|2|0|0| +14674|F|S|4 yr Degree|5000|High Risk|2|0|0| +14675|M|D|4 yr Degree|5000|High Risk|2|0|0| +14676|F|D|4 yr Degree|5000|High Risk|2|0|0| +14677|M|W|4 yr Degree|5000|High Risk|2|0|0| +14678|F|W|4 yr Degree|5000|High Risk|2|0|0| +14679|M|U|4 yr Degree|5000|High Risk|2|0|0| +14680|F|U|4 yr Degree|5000|High Risk|2|0|0| +14681|M|M|Advanced Degree|5000|High Risk|2|0|0| +14682|F|M|Advanced Degree|5000|High Risk|2|0|0| +14683|M|S|Advanced Degree|5000|High Risk|2|0|0| +14684|F|S|Advanced Degree|5000|High Risk|2|0|0| +14685|M|D|Advanced Degree|5000|High Risk|2|0|0| +14686|F|D|Advanced Degree|5000|High Risk|2|0|0| +14687|M|W|Advanced Degree|5000|High Risk|2|0|0| +14688|F|W|Advanced Degree|5000|High Risk|2|0|0| +14689|M|U|Advanced Degree|5000|High Risk|2|0|0| +14690|F|U|Advanced Degree|5000|High Risk|2|0|0| +14691|M|M|Unknown|5000|High Risk|2|0|0| +14692|F|M|Unknown|5000|High Risk|2|0|0| +14693|M|S|Unknown|5000|High Risk|2|0|0| +14694|F|S|Unknown|5000|High Risk|2|0|0| +14695|M|D|Unknown|5000|High Risk|2|0|0| +14696|F|D|Unknown|5000|High Risk|2|0|0| +14697|M|W|Unknown|5000|High Risk|2|0|0| +14698|F|W|Unknown|5000|High Risk|2|0|0| +14699|M|U|Unknown|5000|High Risk|2|0|0| +14700|F|U|Unknown|5000|High Risk|2|0|0| +14701|M|M|Primary|5500|High Risk|2|0|0| +14702|F|M|Primary|5500|High Risk|2|0|0| +14703|M|S|Primary|5500|High Risk|2|0|0| +14704|F|S|Primary|5500|High Risk|2|0|0| +14705|M|D|Primary|5500|High Risk|2|0|0| +14706|F|D|Primary|5500|High Risk|2|0|0| +14707|M|W|Primary|5500|High Risk|2|0|0| +14708|F|W|Primary|5500|High Risk|2|0|0| +14709|M|U|Primary|5500|High Risk|2|0|0| +14710|F|U|Primary|5500|High Risk|2|0|0| +14711|M|M|Secondary|5500|High Risk|2|0|0| +14712|F|M|Secondary|5500|High Risk|2|0|0| +14713|M|S|Secondary|5500|High Risk|2|0|0| +14714|F|S|Secondary|5500|High Risk|2|0|0| +14715|M|D|Secondary|5500|High Risk|2|0|0| +14716|F|D|Secondary|5500|High Risk|2|0|0| +14717|M|W|Secondary|5500|High Risk|2|0|0| +14718|F|W|Secondary|5500|High Risk|2|0|0| +14719|M|U|Secondary|5500|High Risk|2|0|0| +14720|F|U|Secondary|5500|High Risk|2|0|0| +14721|M|M|College|5500|High Risk|2|0|0| +14722|F|M|College|5500|High Risk|2|0|0| +14723|M|S|College|5500|High Risk|2|0|0| +14724|F|S|College|5500|High Risk|2|0|0| +14725|M|D|College|5500|High Risk|2|0|0| +14726|F|D|College|5500|High Risk|2|0|0| +14727|M|W|College|5500|High Risk|2|0|0| +14728|F|W|College|5500|High Risk|2|0|0| +14729|M|U|College|5500|High Risk|2|0|0| +14730|F|U|College|5500|High Risk|2|0|0| +14731|M|M|2 yr Degree|5500|High Risk|2|0|0| +14732|F|M|2 yr Degree|5500|High Risk|2|0|0| +14733|M|S|2 yr Degree|5500|High Risk|2|0|0| +14734|F|S|2 yr Degree|5500|High Risk|2|0|0| +14735|M|D|2 yr Degree|5500|High Risk|2|0|0| +14736|F|D|2 yr Degree|5500|High Risk|2|0|0| +14737|M|W|2 yr Degree|5500|High Risk|2|0|0| +14738|F|W|2 yr Degree|5500|High Risk|2|0|0| +14739|M|U|2 yr Degree|5500|High Risk|2|0|0| +14740|F|U|2 yr Degree|5500|High Risk|2|0|0| +14741|M|M|4 yr Degree|5500|High Risk|2|0|0| +14742|F|M|4 yr Degree|5500|High Risk|2|0|0| +14743|M|S|4 yr Degree|5500|High Risk|2|0|0| +14744|F|S|4 yr Degree|5500|High Risk|2|0|0| +14745|M|D|4 yr Degree|5500|High Risk|2|0|0| +14746|F|D|4 yr Degree|5500|High Risk|2|0|0| +14747|M|W|4 yr Degree|5500|High Risk|2|0|0| +14748|F|W|4 yr Degree|5500|High Risk|2|0|0| +14749|M|U|4 yr Degree|5500|High Risk|2|0|0| +14750|F|U|4 yr Degree|5500|High Risk|2|0|0| +14751|M|M|Advanced Degree|5500|High Risk|2|0|0| +14752|F|M|Advanced Degree|5500|High Risk|2|0|0| +14753|M|S|Advanced Degree|5500|High Risk|2|0|0| +14754|F|S|Advanced Degree|5500|High Risk|2|0|0| +14755|M|D|Advanced Degree|5500|High Risk|2|0|0| +14756|F|D|Advanced Degree|5500|High Risk|2|0|0| +14757|M|W|Advanced Degree|5500|High Risk|2|0|0| +14758|F|W|Advanced Degree|5500|High Risk|2|0|0| +14759|M|U|Advanced Degree|5500|High Risk|2|0|0| +14760|F|U|Advanced Degree|5500|High Risk|2|0|0| +14761|M|M|Unknown|5500|High Risk|2|0|0| +14762|F|M|Unknown|5500|High Risk|2|0|0| +14763|M|S|Unknown|5500|High Risk|2|0|0| +14764|F|S|Unknown|5500|High Risk|2|0|0| +14765|M|D|Unknown|5500|High Risk|2|0|0| +14766|F|D|Unknown|5500|High Risk|2|0|0| +14767|M|W|Unknown|5500|High Risk|2|0|0| +14768|F|W|Unknown|5500|High Risk|2|0|0| +14769|M|U|Unknown|5500|High Risk|2|0|0| +14770|F|U|Unknown|5500|High Risk|2|0|0| +14771|M|M|Primary|6000|High Risk|2|0|0| +14772|F|M|Primary|6000|High Risk|2|0|0| +14773|M|S|Primary|6000|High Risk|2|0|0| +14774|F|S|Primary|6000|High Risk|2|0|0| +14775|M|D|Primary|6000|High Risk|2|0|0| +14776|F|D|Primary|6000|High Risk|2|0|0| +14777|M|W|Primary|6000|High Risk|2|0|0| +14778|F|W|Primary|6000|High Risk|2|0|0| +14779|M|U|Primary|6000|High Risk|2|0|0| +14780|F|U|Primary|6000|High Risk|2|0|0| +14781|M|M|Secondary|6000|High Risk|2|0|0| +14782|F|M|Secondary|6000|High Risk|2|0|0| +14783|M|S|Secondary|6000|High Risk|2|0|0| +14784|F|S|Secondary|6000|High Risk|2|0|0| +14785|M|D|Secondary|6000|High Risk|2|0|0| +14786|F|D|Secondary|6000|High Risk|2|0|0| +14787|M|W|Secondary|6000|High Risk|2|0|0| +14788|F|W|Secondary|6000|High Risk|2|0|0| +14789|M|U|Secondary|6000|High Risk|2|0|0| +14790|F|U|Secondary|6000|High Risk|2|0|0| +14791|M|M|College|6000|High Risk|2|0|0| +14792|F|M|College|6000|High Risk|2|0|0| +14793|M|S|College|6000|High Risk|2|0|0| +14794|F|S|College|6000|High Risk|2|0|0| +14795|M|D|College|6000|High Risk|2|0|0| +14796|F|D|College|6000|High Risk|2|0|0| +14797|M|W|College|6000|High Risk|2|0|0| +14798|F|W|College|6000|High Risk|2|0|0| +14799|M|U|College|6000|High Risk|2|0|0| +14800|F|U|College|6000|High Risk|2|0|0| +14801|M|M|2 yr Degree|6000|High Risk|2|0|0| +14802|F|M|2 yr Degree|6000|High Risk|2|0|0| +14803|M|S|2 yr Degree|6000|High Risk|2|0|0| +14804|F|S|2 yr Degree|6000|High Risk|2|0|0| +14805|M|D|2 yr Degree|6000|High Risk|2|0|0| +14806|F|D|2 yr Degree|6000|High Risk|2|0|0| +14807|M|W|2 yr Degree|6000|High Risk|2|0|0| +14808|F|W|2 yr Degree|6000|High Risk|2|0|0| +14809|M|U|2 yr Degree|6000|High Risk|2|0|0| +14810|F|U|2 yr Degree|6000|High Risk|2|0|0| +14811|M|M|4 yr Degree|6000|High Risk|2|0|0| +14812|F|M|4 yr Degree|6000|High Risk|2|0|0| +14813|M|S|4 yr Degree|6000|High Risk|2|0|0| +14814|F|S|4 yr Degree|6000|High Risk|2|0|0| +14815|M|D|4 yr Degree|6000|High Risk|2|0|0| +14816|F|D|4 yr Degree|6000|High Risk|2|0|0| +14817|M|W|4 yr Degree|6000|High Risk|2|0|0| +14818|F|W|4 yr Degree|6000|High Risk|2|0|0| +14819|M|U|4 yr Degree|6000|High Risk|2|0|0| +14820|F|U|4 yr Degree|6000|High Risk|2|0|0| +14821|M|M|Advanced Degree|6000|High Risk|2|0|0| +14822|F|M|Advanced Degree|6000|High Risk|2|0|0| +14823|M|S|Advanced Degree|6000|High Risk|2|0|0| +14824|F|S|Advanced Degree|6000|High Risk|2|0|0| +14825|M|D|Advanced Degree|6000|High Risk|2|0|0| +14826|F|D|Advanced Degree|6000|High Risk|2|0|0| +14827|M|W|Advanced Degree|6000|High Risk|2|0|0| +14828|F|W|Advanced Degree|6000|High Risk|2|0|0| +14829|M|U|Advanced Degree|6000|High Risk|2|0|0| +14830|F|U|Advanced Degree|6000|High Risk|2|0|0| +14831|M|M|Unknown|6000|High Risk|2|0|0| +14832|F|M|Unknown|6000|High Risk|2|0|0| +14833|M|S|Unknown|6000|High Risk|2|0|0| +14834|F|S|Unknown|6000|High Risk|2|0|0| +14835|M|D|Unknown|6000|High Risk|2|0|0| +14836|F|D|Unknown|6000|High Risk|2|0|0| +14837|M|W|Unknown|6000|High Risk|2|0|0| +14838|F|W|Unknown|6000|High Risk|2|0|0| +14839|M|U|Unknown|6000|High Risk|2|0|0| +14840|F|U|Unknown|6000|High Risk|2|0|0| +14841|M|M|Primary|6500|High Risk|2|0|0| +14842|F|M|Primary|6500|High Risk|2|0|0| +14843|M|S|Primary|6500|High Risk|2|0|0| +14844|F|S|Primary|6500|High Risk|2|0|0| +14845|M|D|Primary|6500|High Risk|2|0|0| +14846|F|D|Primary|6500|High Risk|2|0|0| +14847|M|W|Primary|6500|High Risk|2|0|0| +14848|F|W|Primary|6500|High Risk|2|0|0| +14849|M|U|Primary|6500|High Risk|2|0|0| +14850|F|U|Primary|6500|High Risk|2|0|0| +14851|M|M|Secondary|6500|High Risk|2|0|0| +14852|F|M|Secondary|6500|High Risk|2|0|0| +14853|M|S|Secondary|6500|High Risk|2|0|0| +14854|F|S|Secondary|6500|High Risk|2|0|0| +14855|M|D|Secondary|6500|High Risk|2|0|0| +14856|F|D|Secondary|6500|High Risk|2|0|0| +14857|M|W|Secondary|6500|High Risk|2|0|0| +14858|F|W|Secondary|6500|High Risk|2|0|0| +14859|M|U|Secondary|6500|High Risk|2|0|0| +14860|F|U|Secondary|6500|High Risk|2|0|0| +14861|M|M|College|6500|High Risk|2|0|0| +14862|F|M|College|6500|High Risk|2|0|0| +14863|M|S|College|6500|High Risk|2|0|0| +14864|F|S|College|6500|High Risk|2|0|0| +14865|M|D|College|6500|High Risk|2|0|0| +14866|F|D|College|6500|High Risk|2|0|0| +14867|M|W|College|6500|High Risk|2|0|0| +14868|F|W|College|6500|High Risk|2|0|0| +14869|M|U|College|6500|High Risk|2|0|0| +14870|F|U|College|6500|High Risk|2|0|0| +14871|M|M|2 yr Degree|6500|High Risk|2|0|0| +14872|F|M|2 yr Degree|6500|High Risk|2|0|0| +14873|M|S|2 yr Degree|6500|High Risk|2|0|0| +14874|F|S|2 yr Degree|6500|High Risk|2|0|0| +14875|M|D|2 yr Degree|6500|High Risk|2|0|0| +14876|F|D|2 yr Degree|6500|High Risk|2|0|0| +14877|M|W|2 yr Degree|6500|High Risk|2|0|0| +14878|F|W|2 yr Degree|6500|High Risk|2|0|0| +14879|M|U|2 yr Degree|6500|High Risk|2|0|0| +14880|F|U|2 yr Degree|6500|High Risk|2|0|0| +14881|M|M|4 yr Degree|6500|High Risk|2|0|0| +14882|F|M|4 yr Degree|6500|High Risk|2|0|0| +14883|M|S|4 yr Degree|6500|High Risk|2|0|0| +14884|F|S|4 yr Degree|6500|High Risk|2|0|0| +14885|M|D|4 yr Degree|6500|High Risk|2|0|0| +14886|F|D|4 yr Degree|6500|High Risk|2|0|0| +14887|M|W|4 yr Degree|6500|High Risk|2|0|0| +14888|F|W|4 yr Degree|6500|High Risk|2|0|0| +14889|M|U|4 yr Degree|6500|High Risk|2|0|0| +14890|F|U|4 yr Degree|6500|High Risk|2|0|0| +14891|M|M|Advanced Degree|6500|High Risk|2|0|0| +14892|F|M|Advanced Degree|6500|High Risk|2|0|0| +14893|M|S|Advanced Degree|6500|High Risk|2|0|0| +14894|F|S|Advanced Degree|6500|High Risk|2|0|0| +14895|M|D|Advanced Degree|6500|High Risk|2|0|0| +14896|F|D|Advanced Degree|6500|High Risk|2|0|0| +14897|M|W|Advanced Degree|6500|High Risk|2|0|0| +14898|F|W|Advanced Degree|6500|High Risk|2|0|0| +14899|M|U|Advanced Degree|6500|High Risk|2|0|0| +14900|F|U|Advanced Degree|6500|High Risk|2|0|0| +14901|M|M|Unknown|6500|High Risk|2|0|0| +14902|F|M|Unknown|6500|High Risk|2|0|0| +14903|M|S|Unknown|6500|High Risk|2|0|0| +14904|F|S|Unknown|6500|High Risk|2|0|0| +14905|M|D|Unknown|6500|High Risk|2|0|0| +14906|F|D|Unknown|6500|High Risk|2|0|0| +14907|M|W|Unknown|6500|High Risk|2|0|0| +14908|F|W|Unknown|6500|High Risk|2|0|0| +14909|M|U|Unknown|6500|High Risk|2|0|0| +14910|F|U|Unknown|6500|High Risk|2|0|0| +14911|M|M|Primary|7000|High Risk|2|0|0| +14912|F|M|Primary|7000|High Risk|2|0|0| +14913|M|S|Primary|7000|High Risk|2|0|0| +14914|F|S|Primary|7000|High Risk|2|0|0| +14915|M|D|Primary|7000|High Risk|2|0|0| +14916|F|D|Primary|7000|High Risk|2|0|0| +14917|M|W|Primary|7000|High Risk|2|0|0| +14918|F|W|Primary|7000|High Risk|2|0|0| +14919|M|U|Primary|7000|High Risk|2|0|0| +14920|F|U|Primary|7000|High Risk|2|0|0| +14921|M|M|Secondary|7000|High Risk|2|0|0| +14922|F|M|Secondary|7000|High Risk|2|0|0| +14923|M|S|Secondary|7000|High Risk|2|0|0| +14924|F|S|Secondary|7000|High Risk|2|0|0| +14925|M|D|Secondary|7000|High Risk|2|0|0| +14926|F|D|Secondary|7000|High Risk|2|0|0| +14927|M|W|Secondary|7000|High Risk|2|0|0| +14928|F|W|Secondary|7000|High Risk|2|0|0| +14929|M|U|Secondary|7000|High Risk|2|0|0| +14930|F|U|Secondary|7000|High Risk|2|0|0| +14931|M|M|College|7000|High Risk|2|0|0| +14932|F|M|College|7000|High Risk|2|0|0| +14933|M|S|College|7000|High Risk|2|0|0| +14934|F|S|College|7000|High Risk|2|0|0| +14935|M|D|College|7000|High Risk|2|0|0| +14936|F|D|College|7000|High Risk|2|0|0| +14937|M|W|College|7000|High Risk|2|0|0| +14938|F|W|College|7000|High Risk|2|0|0| +14939|M|U|College|7000|High Risk|2|0|0| +14940|F|U|College|7000|High Risk|2|0|0| +14941|M|M|2 yr Degree|7000|High Risk|2|0|0| +14942|F|M|2 yr Degree|7000|High Risk|2|0|0| +14943|M|S|2 yr Degree|7000|High Risk|2|0|0| +14944|F|S|2 yr Degree|7000|High Risk|2|0|0| +14945|M|D|2 yr Degree|7000|High Risk|2|0|0| +14946|F|D|2 yr Degree|7000|High Risk|2|0|0| +14947|M|W|2 yr Degree|7000|High Risk|2|0|0| +14948|F|W|2 yr Degree|7000|High Risk|2|0|0| +14949|M|U|2 yr Degree|7000|High Risk|2|0|0| +14950|F|U|2 yr Degree|7000|High Risk|2|0|0| +14951|M|M|4 yr Degree|7000|High Risk|2|0|0| +14952|F|M|4 yr Degree|7000|High Risk|2|0|0| +14953|M|S|4 yr Degree|7000|High Risk|2|0|0| +14954|F|S|4 yr Degree|7000|High Risk|2|0|0| +14955|M|D|4 yr Degree|7000|High Risk|2|0|0| +14956|F|D|4 yr Degree|7000|High Risk|2|0|0| +14957|M|W|4 yr Degree|7000|High Risk|2|0|0| +14958|F|W|4 yr Degree|7000|High Risk|2|0|0| +14959|M|U|4 yr Degree|7000|High Risk|2|0|0| +14960|F|U|4 yr Degree|7000|High Risk|2|0|0| +14961|M|M|Advanced Degree|7000|High Risk|2|0|0| +14962|F|M|Advanced Degree|7000|High Risk|2|0|0| +14963|M|S|Advanced Degree|7000|High Risk|2|0|0| +14964|F|S|Advanced Degree|7000|High Risk|2|0|0| +14965|M|D|Advanced Degree|7000|High Risk|2|0|0| +14966|F|D|Advanced Degree|7000|High Risk|2|0|0| +14967|M|W|Advanced Degree|7000|High Risk|2|0|0| +14968|F|W|Advanced Degree|7000|High Risk|2|0|0| +14969|M|U|Advanced Degree|7000|High Risk|2|0|0| +14970|F|U|Advanced Degree|7000|High Risk|2|0|0| +14971|M|M|Unknown|7000|High Risk|2|0|0| +14972|F|M|Unknown|7000|High Risk|2|0|0| +14973|M|S|Unknown|7000|High Risk|2|0|0| +14974|F|S|Unknown|7000|High Risk|2|0|0| +14975|M|D|Unknown|7000|High Risk|2|0|0| +14976|F|D|Unknown|7000|High Risk|2|0|0| +14977|M|W|Unknown|7000|High Risk|2|0|0| +14978|F|W|Unknown|7000|High Risk|2|0|0| +14979|M|U|Unknown|7000|High Risk|2|0|0| +14980|F|U|Unknown|7000|High Risk|2|0|0| +14981|M|M|Primary|7500|High Risk|2|0|0| +14982|F|M|Primary|7500|High Risk|2|0|0| +14983|M|S|Primary|7500|High Risk|2|0|0| +14984|F|S|Primary|7500|High Risk|2|0|0| +14985|M|D|Primary|7500|High Risk|2|0|0| +14986|F|D|Primary|7500|High Risk|2|0|0| +14987|M|W|Primary|7500|High Risk|2|0|0| +14988|F|W|Primary|7500|High Risk|2|0|0| +14989|M|U|Primary|7500|High Risk|2|0|0| +14990|F|U|Primary|7500|High Risk|2|0|0| +14991|M|M|Secondary|7500|High Risk|2|0|0| +14992|F|M|Secondary|7500|High Risk|2|0|0| +14993|M|S|Secondary|7500|High Risk|2|0|0| +14994|F|S|Secondary|7500|High Risk|2|0|0| +14995|M|D|Secondary|7500|High Risk|2|0|0| +14996|F|D|Secondary|7500|High Risk|2|0|0| +14997|M|W|Secondary|7500|High Risk|2|0|0| +14998|F|W|Secondary|7500|High Risk|2|0|0| +14999|M|U|Secondary|7500|High Risk|2|0|0| +15000|F|U|Secondary|7500|High Risk|2|0|0| +15001|M|M|College|7500|High Risk|2|0|0| +15002|F|M|College|7500|High Risk|2|0|0| +15003|M|S|College|7500|High Risk|2|0|0| +15004|F|S|College|7500|High Risk|2|0|0| +15005|M|D|College|7500|High Risk|2|0|0| +15006|F|D|College|7500|High Risk|2|0|0| +15007|M|W|College|7500|High Risk|2|0|0| +15008|F|W|College|7500|High Risk|2|0|0| +15009|M|U|College|7500|High Risk|2|0|0| +15010|F|U|College|7500|High Risk|2|0|0| +15011|M|M|2 yr Degree|7500|High Risk|2|0|0| +15012|F|M|2 yr Degree|7500|High Risk|2|0|0| +15013|M|S|2 yr Degree|7500|High Risk|2|0|0| +15014|F|S|2 yr Degree|7500|High Risk|2|0|0| +15015|M|D|2 yr Degree|7500|High Risk|2|0|0| +15016|F|D|2 yr Degree|7500|High Risk|2|0|0| +15017|M|W|2 yr Degree|7500|High Risk|2|0|0| +15018|F|W|2 yr Degree|7500|High Risk|2|0|0| +15019|M|U|2 yr Degree|7500|High Risk|2|0|0| +15020|F|U|2 yr Degree|7500|High Risk|2|0|0| +15021|M|M|4 yr Degree|7500|High Risk|2|0|0| +15022|F|M|4 yr Degree|7500|High Risk|2|0|0| +15023|M|S|4 yr Degree|7500|High Risk|2|0|0| +15024|F|S|4 yr Degree|7500|High Risk|2|0|0| +15025|M|D|4 yr Degree|7500|High Risk|2|0|0| +15026|F|D|4 yr Degree|7500|High Risk|2|0|0| +15027|M|W|4 yr Degree|7500|High Risk|2|0|0| +15028|F|W|4 yr Degree|7500|High Risk|2|0|0| +15029|M|U|4 yr Degree|7500|High Risk|2|0|0| +15030|F|U|4 yr Degree|7500|High Risk|2|0|0| +15031|M|M|Advanced Degree|7500|High Risk|2|0|0| +15032|F|M|Advanced Degree|7500|High Risk|2|0|0| +15033|M|S|Advanced Degree|7500|High Risk|2|0|0| +15034|F|S|Advanced Degree|7500|High Risk|2|0|0| +15035|M|D|Advanced Degree|7500|High Risk|2|0|0| +15036|F|D|Advanced Degree|7500|High Risk|2|0|0| +15037|M|W|Advanced Degree|7500|High Risk|2|0|0| +15038|F|W|Advanced Degree|7500|High Risk|2|0|0| +15039|M|U|Advanced Degree|7500|High Risk|2|0|0| +15040|F|U|Advanced Degree|7500|High Risk|2|0|0| +15041|M|M|Unknown|7500|High Risk|2|0|0| +15042|F|M|Unknown|7500|High Risk|2|0|0| +15043|M|S|Unknown|7500|High Risk|2|0|0| +15044|F|S|Unknown|7500|High Risk|2|0|0| +15045|M|D|Unknown|7500|High Risk|2|0|0| +15046|F|D|Unknown|7500|High Risk|2|0|0| +15047|M|W|Unknown|7500|High Risk|2|0|0| +15048|F|W|Unknown|7500|High Risk|2|0|0| +15049|M|U|Unknown|7500|High Risk|2|0|0| +15050|F|U|Unknown|7500|High Risk|2|0|0| +15051|M|M|Primary|8000|High Risk|2|0|0| +15052|F|M|Primary|8000|High Risk|2|0|0| +15053|M|S|Primary|8000|High Risk|2|0|0| +15054|F|S|Primary|8000|High Risk|2|0|0| +15055|M|D|Primary|8000|High Risk|2|0|0| +15056|F|D|Primary|8000|High Risk|2|0|0| +15057|M|W|Primary|8000|High Risk|2|0|0| +15058|F|W|Primary|8000|High Risk|2|0|0| +15059|M|U|Primary|8000|High Risk|2|0|0| +15060|F|U|Primary|8000|High Risk|2|0|0| +15061|M|M|Secondary|8000|High Risk|2|0|0| +15062|F|M|Secondary|8000|High Risk|2|0|0| +15063|M|S|Secondary|8000|High Risk|2|0|0| +15064|F|S|Secondary|8000|High Risk|2|0|0| +15065|M|D|Secondary|8000|High Risk|2|0|0| +15066|F|D|Secondary|8000|High Risk|2|0|0| +15067|M|W|Secondary|8000|High Risk|2|0|0| +15068|F|W|Secondary|8000|High Risk|2|0|0| +15069|M|U|Secondary|8000|High Risk|2|0|0| +15070|F|U|Secondary|8000|High Risk|2|0|0| +15071|M|M|College|8000|High Risk|2|0|0| +15072|F|M|College|8000|High Risk|2|0|0| +15073|M|S|College|8000|High Risk|2|0|0| +15074|F|S|College|8000|High Risk|2|0|0| +15075|M|D|College|8000|High Risk|2|0|0| +15076|F|D|College|8000|High Risk|2|0|0| +15077|M|W|College|8000|High Risk|2|0|0| +15078|F|W|College|8000|High Risk|2|0|0| +15079|M|U|College|8000|High Risk|2|0|0| +15080|F|U|College|8000|High Risk|2|0|0| +15081|M|M|2 yr Degree|8000|High Risk|2|0|0| +15082|F|M|2 yr Degree|8000|High Risk|2|0|0| +15083|M|S|2 yr Degree|8000|High Risk|2|0|0| +15084|F|S|2 yr Degree|8000|High Risk|2|0|0| +15085|M|D|2 yr Degree|8000|High Risk|2|0|0| +15086|F|D|2 yr Degree|8000|High Risk|2|0|0| +15087|M|W|2 yr Degree|8000|High Risk|2|0|0| +15088|F|W|2 yr Degree|8000|High Risk|2|0|0| +15089|M|U|2 yr Degree|8000|High Risk|2|0|0| +15090|F|U|2 yr Degree|8000|High Risk|2|0|0| +15091|M|M|4 yr Degree|8000|High Risk|2|0|0| +15092|F|M|4 yr Degree|8000|High Risk|2|0|0| +15093|M|S|4 yr Degree|8000|High Risk|2|0|0| +15094|F|S|4 yr Degree|8000|High Risk|2|0|0| +15095|M|D|4 yr Degree|8000|High Risk|2|0|0| +15096|F|D|4 yr Degree|8000|High Risk|2|0|0| +15097|M|W|4 yr Degree|8000|High Risk|2|0|0| +15098|F|W|4 yr Degree|8000|High Risk|2|0|0| +15099|M|U|4 yr Degree|8000|High Risk|2|0|0| +15100|F|U|4 yr Degree|8000|High Risk|2|0|0| +15101|M|M|Advanced Degree|8000|High Risk|2|0|0| +15102|F|M|Advanced Degree|8000|High Risk|2|0|0| +15103|M|S|Advanced Degree|8000|High Risk|2|0|0| +15104|F|S|Advanced Degree|8000|High Risk|2|0|0| +15105|M|D|Advanced Degree|8000|High Risk|2|0|0| +15106|F|D|Advanced Degree|8000|High Risk|2|0|0| +15107|M|W|Advanced Degree|8000|High Risk|2|0|0| +15108|F|W|Advanced Degree|8000|High Risk|2|0|0| +15109|M|U|Advanced Degree|8000|High Risk|2|0|0| +15110|F|U|Advanced Degree|8000|High Risk|2|0|0| +15111|M|M|Unknown|8000|High Risk|2|0|0| +15112|F|M|Unknown|8000|High Risk|2|0|0| +15113|M|S|Unknown|8000|High Risk|2|0|0| +15114|F|S|Unknown|8000|High Risk|2|0|0| +15115|M|D|Unknown|8000|High Risk|2|0|0| +15116|F|D|Unknown|8000|High Risk|2|0|0| +15117|M|W|Unknown|8000|High Risk|2|0|0| +15118|F|W|Unknown|8000|High Risk|2|0|0| +15119|M|U|Unknown|8000|High Risk|2|0|0| +15120|F|U|Unknown|8000|High Risk|2|0|0| +15121|M|M|Primary|8500|High Risk|2|0|0| +15122|F|M|Primary|8500|High Risk|2|0|0| +15123|M|S|Primary|8500|High Risk|2|0|0| +15124|F|S|Primary|8500|High Risk|2|0|0| +15125|M|D|Primary|8500|High Risk|2|0|0| +15126|F|D|Primary|8500|High Risk|2|0|0| +15127|M|W|Primary|8500|High Risk|2|0|0| +15128|F|W|Primary|8500|High Risk|2|0|0| +15129|M|U|Primary|8500|High Risk|2|0|0| +15130|F|U|Primary|8500|High Risk|2|0|0| +15131|M|M|Secondary|8500|High Risk|2|0|0| +15132|F|M|Secondary|8500|High Risk|2|0|0| +15133|M|S|Secondary|8500|High Risk|2|0|0| +15134|F|S|Secondary|8500|High Risk|2|0|0| +15135|M|D|Secondary|8500|High Risk|2|0|0| +15136|F|D|Secondary|8500|High Risk|2|0|0| +15137|M|W|Secondary|8500|High Risk|2|0|0| +15138|F|W|Secondary|8500|High Risk|2|0|0| +15139|M|U|Secondary|8500|High Risk|2|0|0| +15140|F|U|Secondary|8500|High Risk|2|0|0| +15141|M|M|College|8500|High Risk|2|0|0| +15142|F|M|College|8500|High Risk|2|0|0| +15143|M|S|College|8500|High Risk|2|0|0| +15144|F|S|College|8500|High Risk|2|0|0| +15145|M|D|College|8500|High Risk|2|0|0| +15146|F|D|College|8500|High Risk|2|0|0| +15147|M|W|College|8500|High Risk|2|0|0| +15148|F|W|College|8500|High Risk|2|0|0| +15149|M|U|College|8500|High Risk|2|0|0| +15150|F|U|College|8500|High Risk|2|0|0| +15151|M|M|2 yr Degree|8500|High Risk|2|0|0| +15152|F|M|2 yr Degree|8500|High Risk|2|0|0| +15153|M|S|2 yr Degree|8500|High Risk|2|0|0| +15154|F|S|2 yr Degree|8500|High Risk|2|0|0| +15155|M|D|2 yr Degree|8500|High Risk|2|0|0| +15156|F|D|2 yr Degree|8500|High Risk|2|0|0| +15157|M|W|2 yr Degree|8500|High Risk|2|0|0| +15158|F|W|2 yr Degree|8500|High Risk|2|0|0| +15159|M|U|2 yr Degree|8500|High Risk|2|0|0| +15160|F|U|2 yr Degree|8500|High Risk|2|0|0| +15161|M|M|4 yr Degree|8500|High Risk|2|0|0| +15162|F|M|4 yr Degree|8500|High Risk|2|0|0| +15163|M|S|4 yr Degree|8500|High Risk|2|0|0| +15164|F|S|4 yr Degree|8500|High Risk|2|0|0| +15165|M|D|4 yr Degree|8500|High Risk|2|0|0| +15166|F|D|4 yr Degree|8500|High Risk|2|0|0| +15167|M|W|4 yr Degree|8500|High Risk|2|0|0| +15168|F|W|4 yr Degree|8500|High Risk|2|0|0| +15169|M|U|4 yr Degree|8500|High Risk|2|0|0| +15170|F|U|4 yr Degree|8500|High Risk|2|0|0| +15171|M|M|Advanced Degree|8500|High Risk|2|0|0| +15172|F|M|Advanced Degree|8500|High Risk|2|0|0| +15173|M|S|Advanced Degree|8500|High Risk|2|0|0| +15174|F|S|Advanced Degree|8500|High Risk|2|0|0| +15175|M|D|Advanced Degree|8500|High Risk|2|0|0| +15176|F|D|Advanced Degree|8500|High Risk|2|0|0| +15177|M|W|Advanced Degree|8500|High Risk|2|0|0| +15178|F|W|Advanced Degree|8500|High Risk|2|0|0| +15179|M|U|Advanced Degree|8500|High Risk|2|0|0| +15180|F|U|Advanced Degree|8500|High Risk|2|0|0| +15181|M|M|Unknown|8500|High Risk|2|0|0| +15182|F|M|Unknown|8500|High Risk|2|0|0| +15183|M|S|Unknown|8500|High Risk|2|0|0| +15184|F|S|Unknown|8500|High Risk|2|0|0| +15185|M|D|Unknown|8500|High Risk|2|0|0| +15186|F|D|Unknown|8500|High Risk|2|0|0| +15187|M|W|Unknown|8500|High Risk|2|0|0| +15188|F|W|Unknown|8500|High Risk|2|0|0| +15189|M|U|Unknown|8500|High Risk|2|0|0| +15190|F|U|Unknown|8500|High Risk|2|0|0| +15191|M|M|Primary|9000|High Risk|2|0|0| +15192|F|M|Primary|9000|High Risk|2|0|0| +15193|M|S|Primary|9000|High Risk|2|0|0| +15194|F|S|Primary|9000|High Risk|2|0|0| +15195|M|D|Primary|9000|High Risk|2|0|0| +15196|F|D|Primary|9000|High Risk|2|0|0| +15197|M|W|Primary|9000|High Risk|2|0|0| +15198|F|W|Primary|9000|High Risk|2|0|0| +15199|M|U|Primary|9000|High Risk|2|0|0| +15200|F|U|Primary|9000|High Risk|2|0|0| +15201|M|M|Secondary|9000|High Risk|2|0|0| +15202|F|M|Secondary|9000|High Risk|2|0|0| +15203|M|S|Secondary|9000|High Risk|2|0|0| +15204|F|S|Secondary|9000|High Risk|2|0|0| +15205|M|D|Secondary|9000|High Risk|2|0|0| +15206|F|D|Secondary|9000|High Risk|2|0|0| +15207|M|W|Secondary|9000|High Risk|2|0|0| +15208|F|W|Secondary|9000|High Risk|2|0|0| +15209|M|U|Secondary|9000|High Risk|2|0|0| +15210|F|U|Secondary|9000|High Risk|2|0|0| +15211|M|M|College|9000|High Risk|2|0|0| +15212|F|M|College|9000|High Risk|2|0|0| +15213|M|S|College|9000|High Risk|2|0|0| +15214|F|S|College|9000|High Risk|2|0|0| +15215|M|D|College|9000|High Risk|2|0|0| +15216|F|D|College|9000|High Risk|2|0|0| +15217|M|W|College|9000|High Risk|2|0|0| +15218|F|W|College|9000|High Risk|2|0|0| +15219|M|U|College|9000|High Risk|2|0|0| +15220|F|U|College|9000|High Risk|2|0|0| +15221|M|M|2 yr Degree|9000|High Risk|2|0|0| +15222|F|M|2 yr Degree|9000|High Risk|2|0|0| +15223|M|S|2 yr Degree|9000|High Risk|2|0|0| +15224|F|S|2 yr Degree|9000|High Risk|2|0|0| +15225|M|D|2 yr Degree|9000|High Risk|2|0|0| +15226|F|D|2 yr Degree|9000|High Risk|2|0|0| +15227|M|W|2 yr Degree|9000|High Risk|2|0|0| +15228|F|W|2 yr Degree|9000|High Risk|2|0|0| +15229|M|U|2 yr Degree|9000|High Risk|2|0|0| +15230|F|U|2 yr Degree|9000|High Risk|2|0|0| +15231|M|M|4 yr Degree|9000|High Risk|2|0|0| +15232|F|M|4 yr Degree|9000|High Risk|2|0|0| +15233|M|S|4 yr Degree|9000|High Risk|2|0|0| +15234|F|S|4 yr Degree|9000|High Risk|2|0|0| +15235|M|D|4 yr Degree|9000|High Risk|2|0|0| +15236|F|D|4 yr Degree|9000|High Risk|2|0|0| +15237|M|W|4 yr Degree|9000|High Risk|2|0|0| +15238|F|W|4 yr Degree|9000|High Risk|2|0|0| +15239|M|U|4 yr Degree|9000|High Risk|2|0|0| +15240|F|U|4 yr Degree|9000|High Risk|2|0|0| +15241|M|M|Advanced Degree|9000|High Risk|2|0|0| +15242|F|M|Advanced Degree|9000|High Risk|2|0|0| +15243|M|S|Advanced Degree|9000|High Risk|2|0|0| +15244|F|S|Advanced Degree|9000|High Risk|2|0|0| +15245|M|D|Advanced Degree|9000|High Risk|2|0|0| +15246|F|D|Advanced Degree|9000|High Risk|2|0|0| +15247|M|W|Advanced Degree|9000|High Risk|2|0|0| +15248|F|W|Advanced Degree|9000|High Risk|2|0|0| +15249|M|U|Advanced Degree|9000|High Risk|2|0|0| +15250|F|U|Advanced Degree|9000|High Risk|2|0|0| +15251|M|M|Unknown|9000|High Risk|2|0|0| +15252|F|M|Unknown|9000|High Risk|2|0|0| +15253|M|S|Unknown|9000|High Risk|2|0|0| +15254|F|S|Unknown|9000|High Risk|2|0|0| +15255|M|D|Unknown|9000|High Risk|2|0|0| +15256|F|D|Unknown|9000|High Risk|2|0|0| +15257|M|W|Unknown|9000|High Risk|2|0|0| +15258|F|W|Unknown|9000|High Risk|2|0|0| +15259|M|U|Unknown|9000|High Risk|2|0|0| +15260|F|U|Unknown|9000|High Risk|2|0|0| +15261|M|M|Primary|9500|High Risk|2|0|0| +15262|F|M|Primary|9500|High Risk|2|0|0| +15263|M|S|Primary|9500|High Risk|2|0|0| +15264|F|S|Primary|9500|High Risk|2|0|0| +15265|M|D|Primary|9500|High Risk|2|0|0| +15266|F|D|Primary|9500|High Risk|2|0|0| +15267|M|W|Primary|9500|High Risk|2|0|0| +15268|F|W|Primary|9500|High Risk|2|0|0| +15269|M|U|Primary|9500|High Risk|2|0|0| +15270|F|U|Primary|9500|High Risk|2|0|0| +15271|M|M|Secondary|9500|High Risk|2|0|0| +15272|F|M|Secondary|9500|High Risk|2|0|0| +15273|M|S|Secondary|9500|High Risk|2|0|0| +15274|F|S|Secondary|9500|High Risk|2|0|0| +15275|M|D|Secondary|9500|High Risk|2|0|0| +15276|F|D|Secondary|9500|High Risk|2|0|0| +15277|M|W|Secondary|9500|High Risk|2|0|0| +15278|F|W|Secondary|9500|High Risk|2|0|0| +15279|M|U|Secondary|9500|High Risk|2|0|0| +15280|F|U|Secondary|9500|High Risk|2|0|0| +15281|M|M|College|9500|High Risk|2|0|0| +15282|F|M|College|9500|High Risk|2|0|0| +15283|M|S|College|9500|High Risk|2|0|0| +15284|F|S|College|9500|High Risk|2|0|0| +15285|M|D|College|9500|High Risk|2|0|0| +15286|F|D|College|9500|High Risk|2|0|0| +15287|M|W|College|9500|High Risk|2|0|0| +15288|F|W|College|9500|High Risk|2|0|0| +15289|M|U|College|9500|High Risk|2|0|0| +15290|F|U|College|9500|High Risk|2|0|0| +15291|M|M|2 yr Degree|9500|High Risk|2|0|0| +15292|F|M|2 yr Degree|9500|High Risk|2|0|0| +15293|M|S|2 yr Degree|9500|High Risk|2|0|0| +15294|F|S|2 yr Degree|9500|High Risk|2|0|0| +15295|M|D|2 yr Degree|9500|High Risk|2|0|0| +15296|F|D|2 yr Degree|9500|High Risk|2|0|0| +15297|M|W|2 yr Degree|9500|High Risk|2|0|0| +15298|F|W|2 yr Degree|9500|High Risk|2|0|0| +15299|M|U|2 yr Degree|9500|High Risk|2|0|0| +15300|F|U|2 yr Degree|9500|High Risk|2|0|0| +15301|M|M|4 yr Degree|9500|High Risk|2|0|0| +15302|F|M|4 yr Degree|9500|High Risk|2|0|0| +15303|M|S|4 yr Degree|9500|High Risk|2|0|0| +15304|F|S|4 yr Degree|9500|High Risk|2|0|0| +15305|M|D|4 yr Degree|9500|High Risk|2|0|0| +15306|F|D|4 yr Degree|9500|High Risk|2|0|0| +15307|M|W|4 yr Degree|9500|High Risk|2|0|0| +15308|F|W|4 yr Degree|9500|High Risk|2|0|0| +15309|M|U|4 yr Degree|9500|High Risk|2|0|0| +15310|F|U|4 yr Degree|9500|High Risk|2|0|0| +15311|M|M|Advanced Degree|9500|High Risk|2|0|0| +15312|F|M|Advanced Degree|9500|High Risk|2|0|0| +15313|M|S|Advanced Degree|9500|High Risk|2|0|0| +15314|F|S|Advanced Degree|9500|High Risk|2|0|0| +15315|M|D|Advanced Degree|9500|High Risk|2|0|0| +15316|F|D|Advanced Degree|9500|High Risk|2|0|0| +15317|M|W|Advanced Degree|9500|High Risk|2|0|0| +15318|F|W|Advanced Degree|9500|High Risk|2|0|0| +15319|M|U|Advanced Degree|9500|High Risk|2|0|0| +15320|F|U|Advanced Degree|9500|High Risk|2|0|0| +15321|M|M|Unknown|9500|High Risk|2|0|0| +15322|F|M|Unknown|9500|High Risk|2|0|0| +15323|M|S|Unknown|9500|High Risk|2|0|0| +15324|F|S|Unknown|9500|High Risk|2|0|0| +15325|M|D|Unknown|9500|High Risk|2|0|0| +15326|F|D|Unknown|9500|High Risk|2|0|0| +15327|M|W|Unknown|9500|High Risk|2|0|0| +15328|F|W|Unknown|9500|High Risk|2|0|0| +15329|M|U|Unknown|9500|High Risk|2|0|0| +15330|F|U|Unknown|9500|High Risk|2|0|0| +15331|M|M|Primary|10000|High Risk|2|0|0| +15332|F|M|Primary|10000|High Risk|2|0|0| +15333|M|S|Primary|10000|High Risk|2|0|0| +15334|F|S|Primary|10000|High Risk|2|0|0| +15335|M|D|Primary|10000|High Risk|2|0|0| +15336|F|D|Primary|10000|High Risk|2|0|0| +15337|M|W|Primary|10000|High Risk|2|0|0| +15338|F|W|Primary|10000|High Risk|2|0|0| +15339|M|U|Primary|10000|High Risk|2|0|0| +15340|F|U|Primary|10000|High Risk|2|0|0| +15341|M|M|Secondary|10000|High Risk|2|0|0| +15342|F|M|Secondary|10000|High Risk|2|0|0| +15343|M|S|Secondary|10000|High Risk|2|0|0| +15344|F|S|Secondary|10000|High Risk|2|0|0| +15345|M|D|Secondary|10000|High Risk|2|0|0| +15346|F|D|Secondary|10000|High Risk|2|0|0| +15347|M|W|Secondary|10000|High Risk|2|0|0| +15348|F|W|Secondary|10000|High Risk|2|0|0| +15349|M|U|Secondary|10000|High Risk|2|0|0| +15350|F|U|Secondary|10000|High Risk|2|0|0| +15351|M|M|College|10000|High Risk|2|0|0| +15352|F|M|College|10000|High Risk|2|0|0| +15353|M|S|College|10000|High Risk|2|0|0| +15354|F|S|College|10000|High Risk|2|0|0| +15355|M|D|College|10000|High Risk|2|0|0| +15356|F|D|College|10000|High Risk|2|0|0| +15357|M|W|College|10000|High Risk|2|0|0| +15358|F|W|College|10000|High Risk|2|0|0| +15359|M|U|College|10000|High Risk|2|0|0| +15360|F|U|College|10000|High Risk|2|0|0| +15361|M|M|2 yr Degree|10000|High Risk|2|0|0| +15362|F|M|2 yr Degree|10000|High Risk|2|0|0| +15363|M|S|2 yr Degree|10000|High Risk|2|0|0| +15364|F|S|2 yr Degree|10000|High Risk|2|0|0| +15365|M|D|2 yr Degree|10000|High Risk|2|0|0| +15366|F|D|2 yr Degree|10000|High Risk|2|0|0| +15367|M|W|2 yr Degree|10000|High Risk|2|0|0| +15368|F|W|2 yr Degree|10000|High Risk|2|0|0| +15369|M|U|2 yr Degree|10000|High Risk|2|0|0| +15370|F|U|2 yr Degree|10000|High Risk|2|0|0| +15371|M|M|4 yr Degree|10000|High Risk|2|0|0| +15372|F|M|4 yr Degree|10000|High Risk|2|0|0| +15373|M|S|4 yr Degree|10000|High Risk|2|0|0| +15374|F|S|4 yr Degree|10000|High Risk|2|0|0| +15375|M|D|4 yr Degree|10000|High Risk|2|0|0| +15376|F|D|4 yr Degree|10000|High Risk|2|0|0| +15377|M|W|4 yr Degree|10000|High Risk|2|0|0| +15378|F|W|4 yr Degree|10000|High Risk|2|0|0| +15379|M|U|4 yr Degree|10000|High Risk|2|0|0| +15380|F|U|4 yr Degree|10000|High Risk|2|0|0| +15381|M|M|Advanced Degree|10000|High Risk|2|0|0| +15382|F|M|Advanced Degree|10000|High Risk|2|0|0| +15383|M|S|Advanced Degree|10000|High Risk|2|0|0| +15384|F|S|Advanced Degree|10000|High Risk|2|0|0| +15385|M|D|Advanced Degree|10000|High Risk|2|0|0| +15386|F|D|Advanced Degree|10000|High Risk|2|0|0| +15387|M|W|Advanced Degree|10000|High Risk|2|0|0| +15388|F|W|Advanced Degree|10000|High Risk|2|0|0| +15389|M|U|Advanced Degree|10000|High Risk|2|0|0| +15390|F|U|Advanced Degree|10000|High Risk|2|0|0| +15391|M|M|Unknown|10000|High Risk|2|0|0| +15392|F|M|Unknown|10000|High Risk|2|0|0| +15393|M|S|Unknown|10000|High Risk|2|0|0| +15394|F|S|Unknown|10000|High Risk|2|0|0| +15395|M|D|Unknown|10000|High Risk|2|0|0| +15396|F|D|Unknown|10000|High Risk|2|0|0| +15397|M|W|Unknown|10000|High Risk|2|0|0| +15398|F|W|Unknown|10000|High Risk|2|0|0| +15399|M|U|Unknown|10000|High Risk|2|0|0| +15400|F|U|Unknown|10000|High Risk|2|0|0| +15401|M|M|Primary|500|Unknown|2|0|0| +15402|F|M|Primary|500|Unknown|2|0|0| +15403|M|S|Primary|500|Unknown|2|0|0| +15404|F|S|Primary|500|Unknown|2|0|0| +15405|M|D|Primary|500|Unknown|2|0|0| +15406|F|D|Primary|500|Unknown|2|0|0| +15407|M|W|Primary|500|Unknown|2|0|0| +15408|F|W|Primary|500|Unknown|2|0|0| +15409|M|U|Primary|500|Unknown|2|0|0| +15410|F|U|Primary|500|Unknown|2|0|0| +15411|M|M|Secondary|500|Unknown|2|0|0| +15412|F|M|Secondary|500|Unknown|2|0|0| +15413|M|S|Secondary|500|Unknown|2|0|0| +15414|F|S|Secondary|500|Unknown|2|0|0| +15415|M|D|Secondary|500|Unknown|2|0|0| +15416|F|D|Secondary|500|Unknown|2|0|0| +15417|M|W|Secondary|500|Unknown|2|0|0| +15418|F|W|Secondary|500|Unknown|2|0|0| +15419|M|U|Secondary|500|Unknown|2|0|0| +15420|F|U|Secondary|500|Unknown|2|0|0| +15421|M|M|College|500|Unknown|2|0|0| +15422|F|M|College|500|Unknown|2|0|0| +15423|M|S|College|500|Unknown|2|0|0| +15424|F|S|College|500|Unknown|2|0|0| +15425|M|D|College|500|Unknown|2|0|0| +15426|F|D|College|500|Unknown|2|0|0| +15427|M|W|College|500|Unknown|2|0|0| +15428|F|W|College|500|Unknown|2|0|0| +15429|M|U|College|500|Unknown|2|0|0| +15430|F|U|College|500|Unknown|2|0|0| +15431|M|M|2 yr Degree|500|Unknown|2|0|0| +15432|F|M|2 yr Degree|500|Unknown|2|0|0| +15433|M|S|2 yr Degree|500|Unknown|2|0|0| +15434|F|S|2 yr Degree|500|Unknown|2|0|0| +15435|M|D|2 yr Degree|500|Unknown|2|0|0| +15436|F|D|2 yr Degree|500|Unknown|2|0|0| +15437|M|W|2 yr Degree|500|Unknown|2|0|0| +15438|F|W|2 yr Degree|500|Unknown|2|0|0| +15439|M|U|2 yr Degree|500|Unknown|2|0|0| +15440|F|U|2 yr Degree|500|Unknown|2|0|0| +15441|M|M|4 yr Degree|500|Unknown|2|0|0| +15442|F|M|4 yr Degree|500|Unknown|2|0|0| +15443|M|S|4 yr Degree|500|Unknown|2|0|0| +15444|F|S|4 yr Degree|500|Unknown|2|0|0| +15445|M|D|4 yr Degree|500|Unknown|2|0|0| +15446|F|D|4 yr Degree|500|Unknown|2|0|0| +15447|M|W|4 yr Degree|500|Unknown|2|0|0| +15448|F|W|4 yr Degree|500|Unknown|2|0|0| +15449|M|U|4 yr Degree|500|Unknown|2|0|0| +15450|F|U|4 yr Degree|500|Unknown|2|0|0| +15451|M|M|Advanced Degree|500|Unknown|2|0|0| +15452|F|M|Advanced Degree|500|Unknown|2|0|0| +15453|M|S|Advanced Degree|500|Unknown|2|0|0| +15454|F|S|Advanced Degree|500|Unknown|2|0|0| +15455|M|D|Advanced Degree|500|Unknown|2|0|0| +15456|F|D|Advanced Degree|500|Unknown|2|0|0| +15457|M|W|Advanced Degree|500|Unknown|2|0|0| +15458|F|W|Advanced Degree|500|Unknown|2|0|0| +15459|M|U|Advanced Degree|500|Unknown|2|0|0| +15460|F|U|Advanced Degree|500|Unknown|2|0|0| +15461|M|M|Unknown|500|Unknown|2|0|0| +15462|F|M|Unknown|500|Unknown|2|0|0| +15463|M|S|Unknown|500|Unknown|2|0|0| +15464|F|S|Unknown|500|Unknown|2|0|0| +15465|M|D|Unknown|500|Unknown|2|0|0| +15466|F|D|Unknown|500|Unknown|2|0|0| +15467|M|W|Unknown|500|Unknown|2|0|0| +15468|F|W|Unknown|500|Unknown|2|0|0| +15469|M|U|Unknown|500|Unknown|2|0|0| +15470|F|U|Unknown|500|Unknown|2|0|0| +15471|M|M|Primary|1000|Unknown|2|0|0| +15472|F|M|Primary|1000|Unknown|2|0|0| +15473|M|S|Primary|1000|Unknown|2|0|0| +15474|F|S|Primary|1000|Unknown|2|0|0| +15475|M|D|Primary|1000|Unknown|2|0|0| +15476|F|D|Primary|1000|Unknown|2|0|0| +15477|M|W|Primary|1000|Unknown|2|0|0| +15478|F|W|Primary|1000|Unknown|2|0|0| +15479|M|U|Primary|1000|Unknown|2|0|0| +15480|F|U|Primary|1000|Unknown|2|0|0| +15481|M|M|Secondary|1000|Unknown|2|0|0| +15482|F|M|Secondary|1000|Unknown|2|0|0| +15483|M|S|Secondary|1000|Unknown|2|0|0| +15484|F|S|Secondary|1000|Unknown|2|0|0| +15485|M|D|Secondary|1000|Unknown|2|0|0| +15486|F|D|Secondary|1000|Unknown|2|0|0| +15487|M|W|Secondary|1000|Unknown|2|0|0| +15488|F|W|Secondary|1000|Unknown|2|0|0| +15489|M|U|Secondary|1000|Unknown|2|0|0| +15490|F|U|Secondary|1000|Unknown|2|0|0| +15491|M|M|College|1000|Unknown|2|0|0| +15492|F|M|College|1000|Unknown|2|0|0| +15493|M|S|College|1000|Unknown|2|0|0| +15494|F|S|College|1000|Unknown|2|0|0| +15495|M|D|College|1000|Unknown|2|0|0| +15496|F|D|College|1000|Unknown|2|0|0| +15497|M|W|College|1000|Unknown|2|0|0| +15498|F|W|College|1000|Unknown|2|0|0| +15499|M|U|College|1000|Unknown|2|0|0| +15500|F|U|College|1000|Unknown|2|0|0| +15501|M|M|2 yr Degree|1000|Unknown|2|0|0| +15502|F|M|2 yr Degree|1000|Unknown|2|0|0| +15503|M|S|2 yr Degree|1000|Unknown|2|0|0| +15504|F|S|2 yr Degree|1000|Unknown|2|0|0| +15505|M|D|2 yr Degree|1000|Unknown|2|0|0| +15506|F|D|2 yr Degree|1000|Unknown|2|0|0| +15507|M|W|2 yr Degree|1000|Unknown|2|0|0| +15508|F|W|2 yr Degree|1000|Unknown|2|0|0| +15509|M|U|2 yr Degree|1000|Unknown|2|0|0| +15510|F|U|2 yr Degree|1000|Unknown|2|0|0| +15511|M|M|4 yr Degree|1000|Unknown|2|0|0| +15512|F|M|4 yr Degree|1000|Unknown|2|0|0| +15513|M|S|4 yr Degree|1000|Unknown|2|0|0| +15514|F|S|4 yr Degree|1000|Unknown|2|0|0| +15515|M|D|4 yr Degree|1000|Unknown|2|0|0| +15516|F|D|4 yr Degree|1000|Unknown|2|0|0| +15517|M|W|4 yr Degree|1000|Unknown|2|0|0| +15518|F|W|4 yr Degree|1000|Unknown|2|0|0| +15519|M|U|4 yr Degree|1000|Unknown|2|0|0| +15520|F|U|4 yr Degree|1000|Unknown|2|0|0| +15521|M|M|Advanced Degree|1000|Unknown|2|0|0| +15522|F|M|Advanced Degree|1000|Unknown|2|0|0| +15523|M|S|Advanced Degree|1000|Unknown|2|0|0| +15524|F|S|Advanced Degree|1000|Unknown|2|0|0| +15525|M|D|Advanced Degree|1000|Unknown|2|0|0| +15526|F|D|Advanced Degree|1000|Unknown|2|0|0| +15527|M|W|Advanced Degree|1000|Unknown|2|0|0| +15528|F|W|Advanced Degree|1000|Unknown|2|0|0| +15529|M|U|Advanced Degree|1000|Unknown|2|0|0| +15530|F|U|Advanced Degree|1000|Unknown|2|0|0| +15531|M|M|Unknown|1000|Unknown|2|0|0| +15532|F|M|Unknown|1000|Unknown|2|0|0| +15533|M|S|Unknown|1000|Unknown|2|0|0| +15534|F|S|Unknown|1000|Unknown|2|0|0| +15535|M|D|Unknown|1000|Unknown|2|0|0| +15536|F|D|Unknown|1000|Unknown|2|0|0| +15537|M|W|Unknown|1000|Unknown|2|0|0| +15538|F|W|Unknown|1000|Unknown|2|0|0| +15539|M|U|Unknown|1000|Unknown|2|0|0| +15540|F|U|Unknown|1000|Unknown|2|0|0| +15541|M|M|Primary|1500|Unknown|2|0|0| +15542|F|M|Primary|1500|Unknown|2|0|0| +15543|M|S|Primary|1500|Unknown|2|0|0| +15544|F|S|Primary|1500|Unknown|2|0|0| +15545|M|D|Primary|1500|Unknown|2|0|0| +15546|F|D|Primary|1500|Unknown|2|0|0| +15547|M|W|Primary|1500|Unknown|2|0|0| +15548|F|W|Primary|1500|Unknown|2|0|0| +15549|M|U|Primary|1500|Unknown|2|0|0| +15550|F|U|Primary|1500|Unknown|2|0|0| +15551|M|M|Secondary|1500|Unknown|2|0|0| +15552|F|M|Secondary|1500|Unknown|2|0|0| +15553|M|S|Secondary|1500|Unknown|2|0|0| +15554|F|S|Secondary|1500|Unknown|2|0|0| +15555|M|D|Secondary|1500|Unknown|2|0|0| +15556|F|D|Secondary|1500|Unknown|2|0|0| +15557|M|W|Secondary|1500|Unknown|2|0|0| +15558|F|W|Secondary|1500|Unknown|2|0|0| +15559|M|U|Secondary|1500|Unknown|2|0|0| +15560|F|U|Secondary|1500|Unknown|2|0|0| +15561|M|M|College|1500|Unknown|2|0|0| +15562|F|M|College|1500|Unknown|2|0|0| +15563|M|S|College|1500|Unknown|2|0|0| +15564|F|S|College|1500|Unknown|2|0|0| +15565|M|D|College|1500|Unknown|2|0|0| +15566|F|D|College|1500|Unknown|2|0|0| +15567|M|W|College|1500|Unknown|2|0|0| +15568|F|W|College|1500|Unknown|2|0|0| +15569|M|U|College|1500|Unknown|2|0|0| +15570|F|U|College|1500|Unknown|2|0|0| +15571|M|M|2 yr Degree|1500|Unknown|2|0|0| +15572|F|M|2 yr Degree|1500|Unknown|2|0|0| +15573|M|S|2 yr Degree|1500|Unknown|2|0|0| +15574|F|S|2 yr Degree|1500|Unknown|2|0|0| +15575|M|D|2 yr Degree|1500|Unknown|2|0|0| +15576|F|D|2 yr Degree|1500|Unknown|2|0|0| +15577|M|W|2 yr Degree|1500|Unknown|2|0|0| +15578|F|W|2 yr Degree|1500|Unknown|2|0|0| +15579|M|U|2 yr Degree|1500|Unknown|2|0|0| +15580|F|U|2 yr Degree|1500|Unknown|2|0|0| +15581|M|M|4 yr Degree|1500|Unknown|2|0|0| +15582|F|M|4 yr Degree|1500|Unknown|2|0|0| +15583|M|S|4 yr Degree|1500|Unknown|2|0|0| +15584|F|S|4 yr Degree|1500|Unknown|2|0|0| +15585|M|D|4 yr Degree|1500|Unknown|2|0|0| +15586|F|D|4 yr Degree|1500|Unknown|2|0|0| +15587|M|W|4 yr Degree|1500|Unknown|2|0|0| +15588|F|W|4 yr Degree|1500|Unknown|2|0|0| +15589|M|U|4 yr Degree|1500|Unknown|2|0|0| +15590|F|U|4 yr Degree|1500|Unknown|2|0|0| +15591|M|M|Advanced Degree|1500|Unknown|2|0|0| +15592|F|M|Advanced Degree|1500|Unknown|2|0|0| +15593|M|S|Advanced Degree|1500|Unknown|2|0|0| +15594|F|S|Advanced Degree|1500|Unknown|2|0|0| +15595|M|D|Advanced Degree|1500|Unknown|2|0|0| +15596|F|D|Advanced Degree|1500|Unknown|2|0|0| +15597|M|W|Advanced Degree|1500|Unknown|2|0|0| +15598|F|W|Advanced Degree|1500|Unknown|2|0|0| +15599|M|U|Advanced Degree|1500|Unknown|2|0|0| +15600|F|U|Advanced Degree|1500|Unknown|2|0|0| +15601|M|M|Unknown|1500|Unknown|2|0|0| +15602|F|M|Unknown|1500|Unknown|2|0|0| +15603|M|S|Unknown|1500|Unknown|2|0|0| +15604|F|S|Unknown|1500|Unknown|2|0|0| +15605|M|D|Unknown|1500|Unknown|2|0|0| +15606|F|D|Unknown|1500|Unknown|2|0|0| +15607|M|W|Unknown|1500|Unknown|2|0|0| +15608|F|W|Unknown|1500|Unknown|2|0|0| +15609|M|U|Unknown|1500|Unknown|2|0|0| +15610|F|U|Unknown|1500|Unknown|2|0|0| +15611|M|M|Primary|2000|Unknown|2|0|0| +15612|F|M|Primary|2000|Unknown|2|0|0| +15613|M|S|Primary|2000|Unknown|2|0|0| +15614|F|S|Primary|2000|Unknown|2|0|0| +15615|M|D|Primary|2000|Unknown|2|0|0| +15616|F|D|Primary|2000|Unknown|2|0|0| +15617|M|W|Primary|2000|Unknown|2|0|0| +15618|F|W|Primary|2000|Unknown|2|0|0| +15619|M|U|Primary|2000|Unknown|2|0|0| +15620|F|U|Primary|2000|Unknown|2|0|0| +15621|M|M|Secondary|2000|Unknown|2|0|0| +15622|F|M|Secondary|2000|Unknown|2|0|0| +15623|M|S|Secondary|2000|Unknown|2|0|0| +15624|F|S|Secondary|2000|Unknown|2|0|0| +15625|M|D|Secondary|2000|Unknown|2|0|0| +15626|F|D|Secondary|2000|Unknown|2|0|0| +15627|M|W|Secondary|2000|Unknown|2|0|0| +15628|F|W|Secondary|2000|Unknown|2|0|0| +15629|M|U|Secondary|2000|Unknown|2|0|0| +15630|F|U|Secondary|2000|Unknown|2|0|0| +15631|M|M|College|2000|Unknown|2|0|0| +15632|F|M|College|2000|Unknown|2|0|0| +15633|M|S|College|2000|Unknown|2|0|0| +15634|F|S|College|2000|Unknown|2|0|0| +15635|M|D|College|2000|Unknown|2|0|0| +15636|F|D|College|2000|Unknown|2|0|0| +15637|M|W|College|2000|Unknown|2|0|0| +15638|F|W|College|2000|Unknown|2|0|0| +15639|M|U|College|2000|Unknown|2|0|0| +15640|F|U|College|2000|Unknown|2|0|0| +15641|M|M|2 yr Degree|2000|Unknown|2|0|0| +15642|F|M|2 yr Degree|2000|Unknown|2|0|0| +15643|M|S|2 yr Degree|2000|Unknown|2|0|0| +15644|F|S|2 yr Degree|2000|Unknown|2|0|0| +15645|M|D|2 yr Degree|2000|Unknown|2|0|0| +15646|F|D|2 yr Degree|2000|Unknown|2|0|0| +15647|M|W|2 yr Degree|2000|Unknown|2|0|0| +15648|F|W|2 yr Degree|2000|Unknown|2|0|0| +15649|M|U|2 yr Degree|2000|Unknown|2|0|0| +15650|F|U|2 yr Degree|2000|Unknown|2|0|0| +15651|M|M|4 yr Degree|2000|Unknown|2|0|0| +15652|F|M|4 yr Degree|2000|Unknown|2|0|0| +15653|M|S|4 yr Degree|2000|Unknown|2|0|0| +15654|F|S|4 yr Degree|2000|Unknown|2|0|0| +15655|M|D|4 yr Degree|2000|Unknown|2|0|0| +15656|F|D|4 yr Degree|2000|Unknown|2|0|0| +15657|M|W|4 yr Degree|2000|Unknown|2|0|0| +15658|F|W|4 yr Degree|2000|Unknown|2|0|0| +15659|M|U|4 yr Degree|2000|Unknown|2|0|0| +15660|F|U|4 yr Degree|2000|Unknown|2|0|0| +15661|M|M|Advanced Degree|2000|Unknown|2|0|0| +15662|F|M|Advanced Degree|2000|Unknown|2|0|0| +15663|M|S|Advanced Degree|2000|Unknown|2|0|0| +15664|F|S|Advanced Degree|2000|Unknown|2|0|0| +15665|M|D|Advanced Degree|2000|Unknown|2|0|0| +15666|F|D|Advanced Degree|2000|Unknown|2|0|0| +15667|M|W|Advanced Degree|2000|Unknown|2|0|0| +15668|F|W|Advanced Degree|2000|Unknown|2|0|0| +15669|M|U|Advanced Degree|2000|Unknown|2|0|0| +15670|F|U|Advanced Degree|2000|Unknown|2|0|0| +15671|M|M|Unknown|2000|Unknown|2|0|0| +15672|F|M|Unknown|2000|Unknown|2|0|0| +15673|M|S|Unknown|2000|Unknown|2|0|0| +15674|F|S|Unknown|2000|Unknown|2|0|0| +15675|M|D|Unknown|2000|Unknown|2|0|0| +15676|F|D|Unknown|2000|Unknown|2|0|0| +15677|M|W|Unknown|2000|Unknown|2|0|0| +15678|F|W|Unknown|2000|Unknown|2|0|0| +15679|M|U|Unknown|2000|Unknown|2|0|0| +15680|F|U|Unknown|2000|Unknown|2|0|0| +15681|M|M|Primary|2500|Unknown|2|0|0| +15682|F|M|Primary|2500|Unknown|2|0|0| +15683|M|S|Primary|2500|Unknown|2|0|0| +15684|F|S|Primary|2500|Unknown|2|0|0| +15685|M|D|Primary|2500|Unknown|2|0|0| +15686|F|D|Primary|2500|Unknown|2|0|0| +15687|M|W|Primary|2500|Unknown|2|0|0| +15688|F|W|Primary|2500|Unknown|2|0|0| +15689|M|U|Primary|2500|Unknown|2|0|0| +15690|F|U|Primary|2500|Unknown|2|0|0| +15691|M|M|Secondary|2500|Unknown|2|0|0| +15692|F|M|Secondary|2500|Unknown|2|0|0| +15693|M|S|Secondary|2500|Unknown|2|0|0| +15694|F|S|Secondary|2500|Unknown|2|0|0| +15695|M|D|Secondary|2500|Unknown|2|0|0| +15696|F|D|Secondary|2500|Unknown|2|0|0| +15697|M|W|Secondary|2500|Unknown|2|0|0| +15698|F|W|Secondary|2500|Unknown|2|0|0| +15699|M|U|Secondary|2500|Unknown|2|0|0| +15700|F|U|Secondary|2500|Unknown|2|0|0| +15701|M|M|College|2500|Unknown|2|0|0| +15702|F|M|College|2500|Unknown|2|0|0| +15703|M|S|College|2500|Unknown|2|0|0| +15704|F|S|College|2500|Unknown|2|0|0| +15705|M|D|College|2500|Unknown|2|0|0| +15706|F|D|College|2500|Unknown|2|0|0| +15707|M|W|College|2500|Unknown|2|0|0| +15708|F|W|College|2500|Unknown|2|0|0| +15709|M|U|College|2500|Unknown|2|0|0| +15710|F|U|College|2500|Unknown|2|0|0| +15711|M|M|2 yr Degree|2500|Unknown|2|0|0| +15712|F|M|2 yr Degree|2500|Unknown|2|0|0| +15713|M|S|2 yr Degree|2500|Unknown|2|0|0| +15714|F|S|2 yr Degree|2500|Unknown|2|0|0| +15715|M|D|2 yr Degree|2500|Unknown|2|0|0| +15716|F|D|2 yr Degree|2500|Unknown|2|0|0| +15717|M|W|2 yr Degree|2500|Unknown|2|0|0| +15718|F|W|2 yr Degree|2500|Unknown|2|0|0| +15719|M|U|2 yr Degree|2500|Unknown|2|0|0| +15720|F|U|2 yr Degree|2500|Unknown|2|0|0| +15721|M|M|4 yr Degree|2500|Unknown|2|0|0| +15722|F|M|4 yr Degree|2500|Unknown|2|0|0| +15723|M|S|4 yr Degree|2500|Unknown|2|0|0| +15724|F|S|4 yr Degree|2500|Unknown|2|0|0| +15725|M|D|4 yr Degree|2500|Unknown|2|0|0| +15726|F|D|4 yr Degree|2500|Unknown|2|0|0| +15727|M|W|4 yr Degree|2500|Unknown|2|0|0| +15728|F|W|4 yr Degree|2500|Unknown|2|0|0| +15729|M|U|4 yr Degree|2500|Unknown|2|0|0| +15730|F|U|4 yr Degree|2500|Unknown|2|0|0| +15731|M|M|Advanced Degree|2500|Unknown|2|0|0| +15732|F|M|Advanced Degree|2500|Unknown|2|0|0| +15733|M|S|Advanced Degree|2500|Unknown|2|0|0| +15734|F|S|Advanced Degree|2500|Unknown|2|0|0| +15735|M|D|Advanced Degree|2500|Unknown|2|0|0| +15736|F|D|Advanced Degree|2500|Unknown|2|0|0| +15737|M|W|Advanced Degree|2500|Unknown|2|0|0| +15738|F|W|Advanced Degree|2500|Unknown|2|0|0| +15739|M|U|Advanced Degree|2500|Unknown|2|0|0| +15740|F|U|Advanced Degree|2500|Unknown|2|0|0| +15741|M|M|Unknown|2500|Unknown|2|0|0| +15742|F|M|Unknown|2500|Unknown|2|0|0| +15743|M|S|Unknown|2500|Unknown|2|0|0| +15744|F|S|Unknown|2500|Unknown|2|0|0| +15745|M|D|Unknown|2500|Unknown|2|0|0| +15746|F|D|Unknown|2500|Unknown|2|0|0| +15747|M|W|Unknown|2500|Unknown|2|0|0| +15748|F|W|Unknown|2500|Unknown|2|0|0| +15749|M|U|Unknown|2500|Unknown|2|0|0| +15750|F|U|Unknown|2500|Unknown|2|0|0| +15751|M|M|Primary|3000|Unknown|2|0|0| +15752|F|M|Primary|3000|Unknown|2|0|0| +15753|M|S|Primary|3000|Unknown|2|0|0| +15754|F|S|Primary|3000|Unknown|2|0|0| +15755|M|D|Primary|3000|Unknown|2|0|0| +15756|F|D|Primary|3000|Unknown|2|0|0| +15757|M|W|Primary|3000|Unknown|2|0|0| +15758|F|W|Primary|3000|Unknown|2|0|0| +15759|M|U|Primary|3000|Unknown|2|0|0| +15760|F|U|Primary|3000|Unknown|2|0|0| +15761|M|M|Secondary|3000|Unknown|2|0|0| +15762|F|M|Secondary|3000|Unknown|2|0|0| +15763|M|S|Secondary|3000|Unknown|2|0|0| +15764|F|S|Secondary|3000|Unknown|2|0|0| +15765|M|D|Secondary|3000|Unknown|2|0|0| +15766|F|D|Secondary|3000|Unknown|2|0|0| +15767|M|W|Secondary|3000|Unknown|2|0|0| +15768|F|W|Secondary|3000|Unknown|2|0|0| +15769|M|U|Secondary|3000|Unknown|2|0|0| +15770|F|U|Secondary|3000|Unknown|2|0|0| +15771|M|M|College|3000|Unknown|2|0|0| +15772|F|M|College|3000|Unknown|2|0|0| +15773|M|S|College|3000|Unknown|2|0|0| +15774|F|S|College|3000|Unknown|2|0|0| +15775|M|D|College|3000|Unknown|2|0|0| +15776|F|D|College|3000|Unknown|2|0|0| +15777|M|W|College|3000|Unknown|2|0|0| +15778|F|W|College|3000|Unknown|2|0|0| +15779|M|U|College|3000|Unknown|2|0|0| +15780|F|U|College|3000|Unknown|2|0|0| +15781|M|M|2 yr Degree|3000|Unknown|2|0|0| +15782|F|M|2 yr Degree|3000|Unknown|2|0|0| +15783|M|S|2 yr Degree|3000|Unknown|2|0|0| +15784|F|S|2 yr Degree|3000|Unknown|2|0|0| +15785|M|D|2 yr Degree|3000|Unknown|2|0|0| +15786|F|D|2 yr Degree|3000|Unknown|2|0|0| +15787|M|W|2 yr Degree|3000|Unknown|2|0|0| +15788|F|W|2 yr Degree|3000|Unknown|2|0|0| +15789|M|U|2 yr Degree|3000|Unknown|2|0|0| +15790|F|U|2 yr Degree|3000|Unknown|2|0|0| +15791|M|M|4 yr Degree|3000|Unknown|2|0|0| +15792|F|M|4 yr Degree|3000|Unknown|2|0|0| +15793|M|S|4 yr Degree|3000|Unknown|2|0|0| +15794|F|S|4 yr Degree|3000|Unknown|2|0|0| +15795|M|D|4 yr Degree|3000|Unknown|2|0|0| +15796|F|D|4 yr Degree|3000|Unknown|2|0|0| +15797|M|W|4 yr Degree|3000|Unknown|2|0|0| +15798|F|W|4 yr Degree|3000|Unknown|2|0|0| +15799|M|U|4 yr Degree|3000|Unknown|2|0|0| +15800|F|U|4 yr Degree|3000|Unknown|2|0|0| +15801|M|M|Advanced Degree|3000|Unknown|2|0|0| +15802|F|M|Advanced Degree|3000|Unknown|2|0|0| +15803|M|S|Advanced Degree|3000|Unknown|2|0|0| +15804|F|S|Advanced Degree|3000|Unknown|2|0|0| +15805|M|D|Advanced Degree|3000|Unknown|2|0|0| +15806|F|D|Advanced Degree|3000|Unknown|2|0|0| +15807|M|W|Advanced Degree|3000|Unknown|2|0|0| +15808|F|W|Advanced Degree|3000|Unknown|2|0|0| +15809|M|U|Advanced Degree|3000|Unknown|2|0|0| +15810|F|U|Advanced Degree|3000|Unknown|2|0|0| +15811|M|M|Unknown|3000|Unknown|2|0|0| +15812|F|M|Unknown|3000|Unknown|2|0|0| +15813|M|S|Unknown|3000|Unknown|2|0|0| +15814|F|S|Unknown|3000|Unknown|2|0|0| +15815|M|D|Unknown|3000|Unknown|2|0|0| +15816|F|D|Unknown|3000|Unknown|2|0|0| +15817|M|W|Unknown|3000|Unknown|2|0|0| +15818|F|W|Unknown|3000|Unknown|2|0|0| +15819|M|U|Unknown|3000|Unknown|2|0|0| +15820|F|U|Unknown|3000|Unknown|2|0|0| +15821|M|M|Primary|3500|Unknown|2|0|0| +15822|F|M|Primary|3500|Unknown|2|0|0| +15823|M|S|Primary|3500|Unknown|2|0|0| +15824|F|S|Primary|3500|Unknown|2|0|0| +15825|M|D|Primary|3500|Unknown|2|0|0| +15826|F|D|Primary|3500|Unknown|2|0|0| +15827|M|W|Primary|3500|Unknown|2|0|0| +15828|F|W|Primary|3500|Unknown|2|0|0| +15829|M|U|Primary|3500|Unknown|2|0|0| +15830|F|U|Primary|3500|Unknown|2|0|0| +15831|M|M|Secondary|3500|Unknown|2|0|0| +15832|F|M|Secondary|3500|Unknown|2|0|0| +15833|M|S|Secondary|3500|Unknown|2|0|0| +15834|F|S|Secondary|3500|Unknown|2|0|0| +15835|M|D|Secondary|3500|Unknown|2|0|0| +15836|F|D|Secondary|3500|Unknown|2|0|0| +15837|M|W|Secondary|3500|Unknown|2|0|0| +15838|F|W|Secondary|3500|Unknown|2|0|0| +15839|M|U|Secondary|3500|Unknown|2|0|0| +15840|F|U|Secondary|3500|Unknown|2|0|0| +15841|M|M|College|3500|Unknown|2|0|0| +15842|F|M|College|3500|Unknown|2|0|0| +15843|M|S|College|3500|Unknown|2|0|0| +15844|F|S|College|3500|Unknown|2|0|0| +15845|M|D|College|3500|Unknown|2|0|0| +15846|F|D|College|3500|Unknown|2|0|0| +15847|M|W|College|3500|Unknown|2|0|0| +15848|F|W|College|3500|Unknown|2|0|0| +15849|M|U|College|3500|Unknown|2|0|0| +15850|F|U|College|3500|Unknown|2|0|0| +15851|M|M|2 yr Degree|3500|Unknown|2|0|0| +15852|F|M|2 yr Degree|3500|Unknown|2|0|0| +15853|M|S|2 yr Degree|3500|Unknown|2|0|0| +15854|F|S|2 yr Degree|3500|Unknown|2|0|0| +15855|M|D|2 yr Degree|3500|Unknown|2|0|0| +15856|F|D|2 yr Degree|3500|Unknown|2|0|0| +15857|M|W|2 yr Degree|3500|Unknown|2|0|0| +15858|F|W|2 yr Degree|3500|Unknown|2|0|0| +15859|M|U|2 yr Degree|3500|Unknown|2|0|0| +15860|F|U|2 yr Degree|3500|Unknown|2|0|0| +15861|M|M|4 yr Degree|3500|Unknown|2|0|0| +15862|F|M|4 yr Degree|3500|Unknown|2|0|0| +15863|M|S|4 yr Degree|3500|Unknown|2|0|0| +15864|F|S|4 yr Degree|3500|Unknown|2|0|0| +15865|M|D|4 yr Degree|3500|Unknown|2|0|0| +15866|F|D|4 yr Degree|3500|Unknown|2|0|0| +15867|M|W|4 yr Degree|3500|Unknown|2|0|0| +15868|F|W|4 yr Degree|3500|Unknown|2|0|0| +15869|M|U|4 yr Degree|3500|Unknown|2|0|0| +15870|F|U|4 yr Degree|3500|Unknown|2|0|0| +15871|M|M|Advanced Degree|3500|Unknown|2|0|0| +15872|F|M|Advanced Degree|3500|Unknown|2|0|0| +15873|M|S|Advanced Degree|3500|Unknown|2|0|0| +15874|F|S|Advanced Degree|3500|Unknown|2|0|0| +15875|M|D|Advanced Degree|3500|Unknown|2|0|0| +15876|F|D|Advanced Degree|3500|Unknown|2|0|0| +15877|M|W|Advanced Degree|3500|Unknown|2|0|0| +15878|F|W|Advanced Degree|3500|Unknown|2|0|0| +15879|M|U|Advanced Degree|3500|Unknown|2|0|0| +15880|F|U|Advanced Degree|3500|Unknown|2|0|0| +15881|M|M|Unknown|3500|Unknown|2|0|0| +15882|F|M|Unknown|3500|Unknown|2|0|0| +15883|M|S|Unknown|3500|Unknown|2|0|0| +15884|F|S|Unknown|3500|Unknown|2|0|0| +15885|M|D|Unknown|3500|Unknown|2|0|0| +15886|F|D|Unknown|3500|Unknown|2|0|0| +15887|M|W|Unknown|3500|Unknown|2|0|0| +15888|F|W|Unknown|3500|Unknown|2|0|0| +15889|M|U|Unknown|3500|Unknown|2|0|0| +15890|F|U|Unknown|3500|Unknown|2|0|0| +15891|M|M|Primary|4000|Unknown|2|0|0| +15892|F|M|Primary|4000|Unknown|2|0|0| +15893|M|S|Primary|4000|Unknown|2|0|0| +15894|F|S|Primary|4000|Unknown|2|0|0| +15895|M|D|Primary|4000|Unknown|2|0|0| +15896|F|D|Primary|4000|Unknown|2|0|0| +15897|M|W|Primary|4000|Unknown|2|0|0| +15898|F|W|Primary|4000|Unknown|2|0|0| +15899|M|U|Primary|4000|Unknown|2|0|0| +15900|F|U|Primary|4000|Unknown|2|0|0| +15901|M|M|Secondary|4000|Unknown|2|0|0| +15902|F|M|Secondary|4000|Unknown|2|0|0| +15903|M|S|Secondary|4000|Unknown|2|0|0| +15904|F|S|Secondary|4000|Unknown|2|0|0| +15905|M|D|Secondary|4000|Unknown|2|0|0| +15906|F|D|Secondary|4000|Unknown|2|0|0| +15907|M|W|Secondary|4000|Unknown|2|0|0| +15908|F|W|Secondary|4000|Unknown|2|0|0| +15909|M|U|Secondary|4000|Unknown|2|0|0| +15910|F|U|Secondary|4000|Unknown|2|0|0| +15911|M|M|College|4000|Unknown|2|0|0| +15912|F|M|College|4000|Unknown|2|0|0| +15913|M|S|College|4000|Unknown|2|0|0| +15914|F|S|College|4000|Unknown|2|0|0| +15915|M|D|College|4000|Unknown|2|0|0| +15916|F|D|College|4000|Unknown|2|0|0| +15917|M|W|College|4000|Unknown|2|0|0| +15918|F|W|College|4000|Unknown|2|0|0| +15919|M|U|College|4000|Unknown|2|0|0| +15920|F|U|College|4000|Unknown|2|0|0| +15921|M|M|2 yr Degree|4000|Unknown|2|0|0| +15922|F|M|2 yr Degree|4000|Unknown|2|0|0| +15923|M|S|2 yr Degree|4000|Unknown|2|0|0| +15924|F|S|2 yr Degree|4000|Unknown|2|0|0| +15925|M|D|2 yr Degree|4000|Unknown|2|0|0| +15926|F|D|2 yr Degree|4000|Unknown|2|0|0| +15927|M|W|2 yr Degree|4000|Unknown|2|0|0| +15928|F|W|2 yr Degree|4000|Unknown|2|0|0| +15929|M|U|2 yr Degree|4000|Unknown|2|0|0| +15930|F|U|2 yr Degree|4000|Unknown|2|0|0| +15931|M|M|4 yr Degree|4000|Unknown|2|0|0| +15932|F|M|4 yr Degree|4000|Unknown|2|0|0| +15933|M|S|4 yr Degree|4000|Unknown|2|0|0| +15934|F|S|4 yr Degree|4000|Unknown|2|0|0| +15935|M|D|4 yr Degree|4000|Unknown|2|0|0| +15936|F|D|4 yr Degree|4000|Unknown|2|0|0| +15937|M|W|4 yr Degree|4000|Unknown|2|0|0| +15938|F|W|4 yr Degree|4000|Unknown|2|0|0| +15939|M|U|4 yr Degree|4000|Unknown|2|0|0| +15940|F|U|4 yr Degree|4000|Unknown|2|0|0| +15941|M|M|Advanced Degree|4000|Unknown|2|0|0| +15942|F|M|Advanced Degree|4000|Unknown|2|0|0| +15943|M|S|Advanced Degree|4000|Unknown|2|0|0| +15944|F|S|Advanced Degree|4000|Unknown|2|0|0| +15945|M|D|Advanced Degree|4000|Unknown|2|0|0| +15946|F|D|Advanced Degree|4000|Unknown|2|0|0| +15947|M|W|Advanced Degree|4000|Unknown|2|0|0| +15948|F|W|Advanced Degree|4000|Unknown|2|0|0| +15949|M|U|Advanced Degree|4000|Unknown|2|0|0| +15950|F|U|Advanced Degree|4000|Unknown|2|0|0| +15951|M|M|Unknown|4000|Unknown|2|0|0| +15952|F|M|Unknown|4000|Unknown|2|0|0| +15953|M|S|Unknown|4000|Unknown|2|0|0| +15954|F|S|Unknown|4000|Unknown|2|0|0| +15955|M|D|Unknown|4000|Unknown|2|0|0| +15956|F|D|Unknown|4000|Unknown|2|0|0| +15957|M|W|Unknown|4000|Unknown|2|0|0| +15958|F|W|Unknown|4000|Unknown|2|0|0| +15959|M|U|Unknown|4000|Unknown|2|0|0| +15960|F|U|Unknown|4000|Unknown|2|0|0| +15961|M|M|Primary|4500|Unknown|2|0|0| +15962|F|M|Primary|4500|Unknown|2|0|0| +15963|M|S|Primary|4500|Unknown|2|0|0| +15964|F|S|Primary|4500|Unknown|2|0|0| +15965|M|D|Primary|4500|Unknown|2|0|0| +15966|F|D|Primary|4500|Unknown|2|0|0| +15967|M|W|Primary|4500|Unknown|2|0|0| +15968|F|W|Primary|4500|Unknown|2|0|0| +15969|M|U|Primary|4500|Unknown|2|0|0| +15970|F|U|Primary|4500|Unknown|2|0|0| +15971|M|M|Secondary|4500|Unknown|2|0|0| +15972|F|M|Secondary|4500|Unknown|2|0|0| +15973|M|S|Secondary|4500|Unknown|2|0|0| +15974|F|S|Secondary|4500|Unknown|2|0|0| +15975|M|D|Secondary|4500|Unknown|2|0|0| +15976|F|D|Secondary|4500|Unknown|2|0|0| +15977|M|W|Secondary|4500|Unknown|2|0|0| +15978|F|W|Secondary|4500|Unknown|2|0|0| +15979|M|U|Secondary|4500|Unknown|2|0|0| +15980|F|U|Secondary|4500|Unknown|2|0|0| +15981|M|M|College|4500|Unknown|2|0|0| +15982|F|M|College|4500|Unknown|2|0|0| +15983|M|S|College|4500|Unknown|2|0|0| +15984|F|S|College|4500|Unknown|2|0|0| +15985|M|D|College|4500|Unknown|2|0|0| +15986|F|D|College|4500|Unknown|2|0|0| +15987|M|W|College|4500|Unknown|2|0|0| +15988|F|W|College|4500|Unknown|2|0|0| +15989|M|U|College|4500|Unknown|2|0|0| +15990|F|U|College|4500|Unknown|2|0|0| +15991|M|M|2 yr Degree|4500|Unknown|2|0|0| +15992|F|M|2 yr Degree|4500|Unknown|2|0|0| +15993|M|S|2 yr Degree|4500|Unknown|2|0|0| +15994|F|S|2 yr Degree|4500|Unknown|2|0|0| +15995|M|D|2 yr Degree|4500|Unknown|2|0|0| +15996|F|D|2 yr Degree|4500|Unknown|2|0|0| +15997|M|W|2 yr Degree|4500|Unknown|2|0|0| +15998|F|W|2 yr Degree|4500|Unknown|2|0|0| +15999|M|U|2 yr Degree|4500|Unknown|2|0|0| +16000|F|U|2 yr Degree|4500|Unknown|2|0|0| +16001|M|M|4 yr Degree|4500|Unknown|2|0|0| +16002|F|M|4 yr Degree|4500|Unknown|2|0|0| +16003|M|S|4 yr Degree|4500|Unknown|2|0|0| +16004|F|S|4 yr Degree|4500|Unknown|2|0|0| +16005|M|D|4 yr Degree|4500|Unknown|2|0|0| +16006|F|D|4 yr Degree|4500|Unknown|2|0|0| +16007|M|W|4 yr Degree|4500|Unknown|2|0|0| +16008|F|W|4 yr Degree|4500|Unknown|2|0|0| +16009|M|U|4 yr Degree|4500|Unknown|2|0|0| +16010|F|U|4 yr Degree|4500|Unknown|2|0|0| +16011|M|M|Advanced Degree|4500|Unknown|2|0|0| +16012|F|M|Advanced Degree|4500|Unknown|2|0|0| +16013|M|S|Advanced Degree|4500|Unknown|2|0|0| +16014|F|S|Advanced Degree|4500|Unknown|2|0|0| +16015|M|D|Advanced Degree|4500|Unknown|2|0|0| +16016|F|D|Advanced Degree|4500|Unknown|2|0|0| +16017|M|W|Advanced Degree|4500|Unknown|2|0|0| +16018|F|W|Advanced Degree|4500|Unknown|2|0|0| +16019|M|U|Advanced Degree|4500|Unknown|2|0|0| +16020|F|U|Advanced Degree|4500|Unknown|2|0|0| +16021|M|M|Unknown|4500|Unknown|2|0|0| +16022|F|M|Unknown|4500|Unknown|2|0|0| +16023|M|S|Unknown|4500|Unknown|2|0|0| +16024|F|S|Unknown|4500|Unknown|2|0|0| +16025|M|D|Unknown|4500|Unknown|2|0|0| +16026|F|D|Unknown|4500|Unknown|2|0|0| +16027|M|W|Unknown|4500|Unknown|2|0|0| +16028|F|W|Unknown|4500|Unknown|2|0|0| +16029|M|U|Unknown|4500|Unknown|2|0|0| +16030|F|U|Unknown|4500|Unknown|2|0|0| +16031|M|M|Primary|5000|Unknown|2|0|0| +16032|F|M|Primary|5000|Unknown|2|0|0| +16033|M|S|Primary|5000|Unknown|2|0|0| +16034|F|S|Primary|5000|Unknown|2|0|0| +16035|M|D|Primary|5000|Unknown|2|0|0| +16036|F|D|Primary|5000|Unknown|2|0|0| +16037|M|W|Primary|5000|Unknown|2|0|0| +16038|F|W|Primary|5000|Unknown|2|0|0| +16039|M|U|Primary|5000|Unknown|2|0|0| +16040|F|U|Primary|5000|Unknown|2|0|0| +16041|M|M|Secondary|5000|Unknown|2|0|0| +16042|F|M|Secondary|5000|Unknown|2|0|0| +16043|M|S|Secondary|5000|Unknown|2|0|0| +16044|F|S|Secondary|5000|Unknown|2|0|0| +16045|M|D|Secondary|5000|Unknown|2|0|0| +16046|F|D|Secondary|5000|Unknown|2|0|0| +16047|M|W|Secondary|5000|Unknown|2|0|0| +16048|F|W|Secondary|5000|Unknown|2|0|0| +16049|M|U|Secondary|5000|Unknown|2|0|0| +16050|F|U|Secondary|5000|Unknown|2|0|0| +16051|M|M|College|5000|Unknown|2|0|0| +16052|F|M|College|5000|Unknown|2|0|0| +16053|M|S|College|5000|Unknown|2|0|0| +16054|F|S|College|5000|Unknown|2|0|0| +16055|M|D|College|5000|Unknown|2|0|0| +16056|F|D|College|5000|Unknown|2|0|0| +16057|M|W|College|5000|Unknown|2|0|0| +16058|F|W|College|5000|Unknown|2|0|0| +16059|M|U|College|5000|Unknown|2|0|0| +16060|F|U|College|5000|Unknown|2|0|0| +16061|M|M|2 yr Degree|5000|Unknown|2|0|0| +16062|F|M|2 yr Degree|5000|Unknown|2|0|0| +16063|M|S|2 yr Degree|5000|Unknown|2|0|0| +16064|F|S|2 yr Degree|5000|Unknown|2|0|0| +16065|M|D|2 yr Degree|5000|Unknown|2|0|0| +16066|F|D|2 yr Degree|5000|Unknown|2|0|0| +16067|M|W|2 yr Degree|5000|Unknown|2|0|0| +16068|F|W|2 yr Degree|5000|Unknown|2|0|0| +16069|M|U|2 yr Degree|5000|Unknown|2|0|0| +16070|F|U|2 yr Degree|5000|Unknown|2|0|0| +16071|M|M|4 yr Degree|5000|Unknown|2|0|0| +16072|F|M|4 yr Degree|5000|Unknown|2|0|0| +16073|M|S|4 yr Degree|5000|Unknown|2|0|0| +16074|F|S|4 yr Degree|5000|Unknown|2|0|0| +16075|M|D|4 yr Degree|5000|Unknown|2|0|0| +16076|F|D|4 yr Degree|5000|Unknown|2|0|0| +16077|M|W|4 yr Degree|5000|Unknown|2|0|0| +16078|F|W|4 yr Degree|5000|Unknown|2|0|0| +16079|M|U|4 yr Degree|5000|Unknown|2|0|0| +16080|F|U|4 yr Degree|5000|Unknown|2|0|0| +16081|M|M|Advanced Degree|5000|Unknown|2|0|0| +16082|F|M|Advanced Degree|5000|Unknown|2|0|0| +16083|M|S|Advanced Degree|5000|Unknown|2|0|0| +16084|F|S|Advanced Degree|5000|Unknown|2|0|0| +16085|M|D|Advanced Degree|5000|Unknown|2|0|0| +16086|F|D|Advanced Degree|5000|Unknown|2|0|0| +16087|M|W|Advanced Degree|5000|Unknown|2|0|0| +16088|F|W|Advanced Degree|5000|Unknown|2|0|0| +16089|M|U|Advanced Degree|5000|Unknown|2|0|0| +16090|F|U|Advanced Degree|5000|Unknown|2|0|0| +16091|M|M|Unknown|5000|Unknown|2|0|0| +16092|F|M|Unknown|5000|Unknown|2|0|0| +16093|M|S|Unknown|5000|Unknown|2|0|0| +16094|F|S|Unknown|5000|Unknown|2|0|0| +16095|M|D|Unknown|5000|Unknown|2|0|0| +16096|F|D|Unknown|5000|Unknown|2|0|0| +16097|M|W|Unknown|5000|Unknown|2|0|0| +16098|F|W|Unknown|5000|Unknown|2|0|0| +16099|M|U|Unknown|5000|Unknown|2|0|0| +16100|F|U|Unknown|5000|Unknown|2|0|0| +16101|M|M|Primary|5500|Unknown|2|0|0| +16102|F|M|Primary|5500|Unknown|2|0|0| +16103|M|S|Primary|5500|Unknown|2|0|0| +16104|F|S|Primary|5500|Unknown|2|0|0| +16105|M|D|Primary|5500|Unknown|2|0|0| +16106|F|D|Primary|5500|Unknown|2|0|0| +16107|M|W|Primary|5500|Unknown|2|0|0| +16108|F|W|Primary|5500|Unknown|2|0|0| +16109|M|U|Primary|5500|Unknown|2|0|0| +16110|F|U|Primary|5500|Unknown|2|0|0| +16111|M|M|Secondary|5500|Unknown|2|0|0| +16112|F|M|Secondary|5500|Unknown|2|0|0| +16113|M|S|Secondary|5500|Unknown|2|0|0| +16114|F|S|Secondary|5500|Unknown|2|0|0| +16115|M|D|Secondary|5500|Unknown|2|0|0| +16116|F|D|Secondary|5500|Unknown|2|0|0| +16117|M|W|Secondary|5500|Unknown|2|0|0| +16118|F|W|Secondary|5500|Unknown|2|0|0| +16119|M|U|Secondary|5500|Unknown|2|0|0| +16120|F|U|Secondary|5500|Unknown|2|0|0| +16121|M|M|College|5500|Unknown|2|0|0| +16122|F|M|College|5500|Unknown|2|0|0| +16123|M|S|College|5500|Unknown|2|0|0| +16124|F|S|College|5500|Unknown|2|0|0| +16125|M|D|College|5500|Unknown|2|0|0| +16126|F|D|College|5500|Unknown|2|0|0| +16127|M|W|College|5500|Unknown|2|0|0| +16128|F|W|College|5500|Unknown|2|0|0| +16129|M|U|College|5500|Unknown|2|0|0| +16130|F|U|College|5500|Unknown|2|0|0| +16131|M|M|2 yr Degree|5500|Unknown|2|0|0| +16132|F|M|2 yr Degree|5500|Unknown|2|0|0| +16133|M|S|2 yr Degree|5500|Unknown|2|0|0| +16134|F|S|2 yr Degree|5500|Unknown|2|0|0| +16135|M|D|2 yr Degree|5500|Unknown|2|0|0| +16136|F|D|2 yr Degree|5500|Unknown|2|0|0| +16137|M|W|2 yr Degree|5500|Unknown|2|0|0| +16138|F|W|2 yr Degree|5500|Unknown|2|0|0| +16139|M|U|2 yr Degree|5500|Unknown|2|0|0| +16140|F|U|2 yr Degree|5500|Unknown|2|0|0| +16141|M|M|4 yr Degree|5500|Unknown|2|0|0| +16142|F|M|4 yr Degree|5500|Unknown|2|0|0| +16143|M|S|4 yr Degree|5500|Unknown|2|0|0| +16144|F|S|4 yr Degree|5500|Unknown|2|0|0| +16145|M|D|4 yr Degree|5500|Unknown|2|0|0| +16146|F|D|4 yr Degree|5500|Unknown|2|0|0| +16147|M|W|4 yr Degree|5500|Unknown|2|0|0| +16148|F|W|4 yr Degree|5500|Unknown|2|0|0| +16149|M|U|4 yr Degree|5500|Unknown|2|0|0| +16150|F|U|4 yr Degree|5500|Unknown|2|0|0| +16151|M|M|Advanced Degree|5500|Unknown|2|0|0| +16152|F|M|Advanced Degree|5500|Unknown|2|0|0| +16153|M|S|Advanced Degree|5500|Unknown|2|0|0| +16154|F|S|Advanced Degree|5500|Unknown|2|0|0| +16155|M|D|Advanced Degree|5500|Unknown|2|0|0| +16156|F|D|Advanced Degree|5500|Unknown|2|0|0| +16157|M|W|Advanced Degree|5500|Unknown|2|0|0| +16158|F|W|Advanced Degree|5500|Unknown|2|0|0| +16159|M|U|Advanced Degree|5500|Unknown|2|0|0| +16160|F|U|Advanced Degree|5500|Unknown|2|0|0| +16161|M|M|Unknown|5500|Unknown|2|0|0| +16162|F|M|Unknown|5500|Unknown|2|0|0| +16163|M|S|Unknown|5500|Unknown|2|0|0| +16164|F|S|Unknown|5500|Unknown|2|0|0| +16165|M|D|Unknown|5500|Unknown|2|0|0| +16166|F|D|Unknown|5500|Unknown|2|0|0| +16167|M|W|Unknown|5500|Unknown|2|0|0| +16168|F|W|Unknown|5500|Unknown|2|0|0| +16169|M|U|Unknown|5500|Unknown|2|0|0| +16170|F|U|Unknown|5500|Unknown|2|0|0| +16171|M|M|Primary|6000|Unknown|2|0|0| +16172|F|M|Primary|6000|Unknown|2|0|0| +16173|M|S|Primary|6000|Unknown|2|0|0| +16174|F|S|Primary|6000|Unknown|2|0|0| +16175|M|D|Primary|6000|Unknown|2|0|0| +16176|F|D|Primary|6000|Unknown|2|0|0| +16177|M|W|Primary|6000|Unknown|2|0|0| +16178|F|W|Primary|6000|Unknown|2|0|0| +16179|M|U|Primary|6000|Unknown|2|0|0| +16180|F|U|Primary|6000|Unknown|2|0|0| +16181|M|M|Secondary|6000|Unknown|2|0|0| +16182|F|M|Secondary|6000|Unknown|2|0|0| +16183|M|S|Secondary|6000|Unknown|2|0|0| +16184|F|S|Secondary|6000|Unknown|2|0|0| +16185|M|D|Secondary|6000|Unknown|2|0|0| +16186|F|D|Secondary|6000|Unknown|2|0|0| +16187|M|W|Secondary|6000|Unknown|2|0|0| +16188|F|W|Secondary|6000|Unknown|2|0|0| +16189|M|U|Secondary|6000|Unknown|2|0|0| +16190|F|U|Secondary|6000|Unknown|2|0|0| +16191|M|M|College|6000|Unknown|2|0|0| +16192|F|M|College|6000|Unknown|2|0|0| +16193|M|S|College|6000|Unknown|2|0|0| +16194|F|S|College|6000|Unknown|2|0|0| +16195|M|D|College|6000|Unknown|2|0|0| +16196|F|D|College|6000|Unknown|2|0|0| +16197|M|W|College|6000|Unknown|2|0|0| +16198|F|W|College|6000|Unknown|2|0|0| +16199|M|U|College|6000|Unknown|2|0|0| +16200|F|U|College|6000|Unknown|2|0|0| +16201|M|M|2 yr Degree|6000|Unknown|2|0|0| +16202|F|M|2 yr Degree|6000|Unknown|2|0|0| +16203|M|S|2 yr Degree|6000|Unknown|2|0|0| +16204|F|S|2 yr Degree|6000|Unknown|2|0|0| +16205|M|D|2 yr Degree|6000|Unknown|2|0|0| +16206|F|D|2 yr Degree|6000|Unknown|2|0|0| +16207|M|W|2 yr Degree|6000|Unknown|2|0|0| +16208|F|W|2 yr Degree|6000|Unknown|2|0|0| +16209|M|U|2 yr Degree|6000|Unknown|2|0|0| +16210|F|U|2 yr Degree|6000|Unknown|2|0|0| +16211|M|M|4 yr Degree|6000|Unknown|2|0|0| +16212|F|M|4 yr Degree|6000|Unknown|2|0|0| +16213|M|S|4 yr Degree|6000|Unknown|2|0|0| +16214|F|S|4 yr Degree|6000|Unknown|2|0|0| +16215|M|D|4 yr Degree|6000|Unknown|2|0|0| +16216|F|D|4 yr Degree|6000|Unknown|2|0|0| +16217|M|W|4 yr Degree|6000|Unknown|2|0|0| +16218|F|W|4 yr Degree|6000|Unknown|2|0|0| +16219|M|U|4 yr Degree|6000|Unknown|2|0|0| +16220|F|U|4 yr Degree|6000|Unknown|2|0|0| +16221|M|M|Advanced Degree|6000|Unknown|2|0|0| +16222|F|M|Advanced Degree|6000|Unknown|2|0|0| +16223|M|S|Advanced Degree|6000|Unknown|2|0|0| +16224|F|S|Advanced Degree|6000|Unknown|2|0|0| +16225|M|D|Advanced Degree|6000|Unknown|2|0|0| +16226|F|D|Advanced Degree|6000|Unknown|2|0|0| +16227|M|W|Advanced Degree|6000|Unknown|2|0|0| +16228|F|W|Advanced Degree|6000|Unknown|2|0|0| +16229|M|U|Advanced Degree|6000|Unknown|2|0|0| +16230|F|U|Advanced Degree|6000|Unknown|2|0|0| +16231|M|M|Unknown|6000|Unknown|2|0|0| +16232|F|M|Unknown|6000|Unknown|2|0|0| +16233|M|S|Unknown|6000|Unknown|2|0|0| +16234|F|S|Unknown|6000|Unknown|2|0|0| +16235|M|D|Unknown|6000|Unknown|2|0|0| +16236|F|D|Unknown|6000|Unknown|2|0|0| +16237|M|W|Unknown|6000|Unknown|2|0|0| +16238|F|W|Unknown|6000|Unknown|2|0|0| +16239|M|U|Unknown|6000|Unknown|2|0|0| +16240|F|U|Unknown|6000|Unknown|2|0|0| +16241|M|M|Primary|6500|Unknown|2|0|0| +16242|F|M|Primary|6500|Unknown|2|0|0| +16243|M|S|Primary|6500|Unknown|2|0|0| +16244|F|S|Primary|6500|Unknown|2|0|0| +16245|M|D|Primary|6500|Unknown|2|0|0| +16246|F|D|Primary|6500|Unknown|2|0|0| +16247|M|W|Primary|6500|Unknown|2|0|0| +16248|F|W|Primary|6500|Unknown|2|0|0| +16249|M|U|Primary|6500|Unknown|2|0|0| +16250|F|U|Primary|6500|Unknown|2|0|0| +16251|M|M|Secondary|6500|Unknown|2|0|0| +16252|F|M|Secondary|6500|Unknown|2|0|0| +16253|M|S|Secondary|6500|Unknown|2|0|0| +16254|F|S|Secondary|6500|Unknown|2|0|0| +16255|M|D|Secondary|6500|Unknown|2|0|0| +16256|F|D|Secondary|6500|Unknown|2|0|0| +16257|M|W|Secondary|6500|Unknown|2|0|0| +16258|F|W|Secondary|6500|Unknown|2|0|0| +16259|M|U|Secondary|6500|Unknown|2|0|0| +16260|F|U|Secondary|6500|Unknown|2|0|0| +16261|M|M|College|6500|Unknown|2|0|0| +16262|F|M|College|6500|Unknown|2|0|0| +16263|M|S|College|6500|Unknown|2|0|0| +16264|F|S|College|6500|Unknown|2|0|0| +16265|M|D|College|6500|Unknown|2|0|0| +16266|F|D|College|6500|Unknown|2|0|0| +16267|M|W|College|6500|Unknown|2|0|0| +16268|F|W|College|6500|Unknown|2|0|0| +16269|M|U|College|6500|Unknown|2|0|0| +16270|F|U|College|6500|Unknown|2|0|0| +16271|M|M|2 yr Degree|6500|Unknown|2|0|0| +16272|F|M|2 yr Degree|6500|Unknown|2|0|0| +16273|M|S|2 yr Degree|6500|Unknown|2|0|0| +16274|F|S|2 yr Degree|6500|Unknown|2|0|0| +16275|M|D|2 yr Degree|6500|Unknown|2|0|0| +16276|F|D|2 yr Degree|6500|Unknown|2|0|0| +16277|M|W|2 yr Degree|6500|Unknown|2|0|0| +16278|F|W|2 yr Degree|6500|Unknown|2|0|0| +16279|M|U|2 yr Degree|6500|Unknown|2|0|0| +16280|F|U|2 yr Degree|6500|Unknown|2|0|0| +16281|M|M|4 yr Degree|6500|Unknown|2|0|0| +16282|F|M|4 yr Degree|6500|Unknown|2|0|0| +16283|M|S|4 yr Degree|6500|Unknown|2|0|0| +16284|F|S|4 yr Degree|6500|Unknown|2|0|0| +16285|M|D|4 yr Degree|6500|Unknown|2|0|0| +16286|F|D|4 yr Degree|6500|Unknown|2|0|0| +16287|M|W|4 yr Degree|6500|Unknown|2|0|0| +16288|F|W|4 yr Degree|6500|Unknown|2|0|0| +16289|M|U|4 yr Degree|6500|Unknown|2|0|0| +16290|F|U|4 yr Degree|6500|Unknown|2|0|0| +16291|M|M|Advanced Degree|6500|Unknown|2|0|0| +16292|F|M|Advanced Degree|6500|Unknown|2|0|0| +16293|M|S|Advanced Degree|6500|Unknown|2|0|0| +16294|F|S|Advanced Degree|6500|Unknown|2|0|0| +16295|M|D|Advanced Degree|6500|Unknown|2|0|0| +16296|F|D|Advanced Degree|6500|Unknown|2|0|0| +16297|M|W|Advanced Degree|6500|Unknown|2|0|0| +16298|F|W|Advanced Degree|6500|Unknown|2|0|0| +16299|M|U|Advanced Degree|6500|Unknown|2|0|0| +16300|F|U|Advanced Degree|6500|Unknown|2|0|0| +16301|M|M|Unknown|6500|Unknown|2|0|0| +16302|F|M|Unknown|6500|Unknown|2|0|0| +16303|M|S|Unknown|6500|Unknown|2|0|0| +16304|F|S|Unknown|6500|Unknown|2|0|0| +16305|M|D|Unknown|6500|Unknown|2|0|0| +16306|F|D|Unknown|6500|Unknown|2|0|0| +16307|M|W|Unknown|6500|Unknown|2|0|0| +16308|F|W|Unknown|6500|Unknown|2|0|0| +16309|M|U|Unknown|6500|Unknown|2|0|0| +16310|F|U|Unknown|6500|Unknown|2|0|0| +16311|M|M|Primary|7000|Unknown|2|0|0| +16312|F|M|Primary|7000|Unknown|2|0|0| +16313|M|S|Primary|7000|Unknown|2|0|0| +16314|F|S|Primary|7000|Unknown|2|0|0| +16315|M|D|Primary|7000|Unknown|2|0|0| +16316|F|D|Primary|7000|Unknown|2|0|0| +16317|M|W|Primary|7000|Unknown|2|0|0| +16318|F|W|Primary|7000|Unknown|2|0|0| +16319|M|U|Primary|7000|Unknown|2|0|0| +16320|F|U|Primary|7000|Unknown|2|0|0| +16321|M|M|Secondary|7000|Unknown|2|0|0| +16322|F|M|Secondary|7000|Unknown|2|0|0| +16323|M|S|Secondary|7000|Unknown|2|0|0| +16324|F|S|Secondary|7000|Unknown|2|0|0| +16325|M|D|Secondary|7000|Unknown|2|0|0| +16326|F|D|Secondary|7000|Unknown|2|0|0| +16327|M|W|Secondary|7000|Unknown|2|0|0| +16328|F|W|Secondary|7000|Unknown|2|0|0| +16329|M|U|Secondary|7000|Unknown|2|0|0| +16330|F|U|Secondary|7000|Unknown|2|0|0| +16331|M|M|College|7000|Unknown|2|0|0| +16332|F|M|College|7000|Unknown|2|0|0| +16333|M|S|College|7000|Unknown|2|0|0| +16334|F|S|College|7000|Unknown|2|0|0| +16335|M|D|College|7000|Unknown|2|0|0| +16336|F|D|College|7000|Unknown|2|0|0| +16337|M|W|College|7000|Unknown|2|0|0| +16338|F|W|College|7000|Unknown|2|0|0| +16339|M|U|College|7000|Unknown|2|0|0| +16340|F|U|College|7000|Unknown|2|0|0| +16341|M|M|2 yr Degree|7000|Unknown|2|0|0| +16342|F|M|2 yr Degree|7000|Unknown|2|0|0| +16343|M|S|2 yr Degree|7000|Unknown|2|0|0| +16344|F|S|2 yr Degree|7000|Unknown|2|0|0| +16345|M|D|2 yr Degree|7000|Unknown|2|0|0| +16346|F|D|2 yr Degree|7000|Unknown|2|0|0| +16347|M|W|2 yr Degree|7000|Unknown|2|0|0| +16348|F|W|2 yr Degree|7000|Unknown|2|0|0| +16349|M|U|2 yr Degree|7000|Unknown|2|0|0| +16350|F|U|2 yr Degree|7000|Unknown|2|0|0| +16351|M|M|4 yr Degree|7000|Unknown|2|0|0| +16352|F|M|4 yr Degree|7000|Unknown|2|0|0| +16353|M|S|4 yr Degree|7000|Unknown|2|0|0| +16354|F|S|4 yr Degree|7000|Unknown|2|0|0| +16355|M|D|4 yr Degree|7000|Unknown|2|0|0| +16356|F|D|4 yr Degree|7000|Unknown|2|0|0| +16357|M|W|4 yr Degree|7000|Unknown|2|0|0| +16358|F|W|4 yr Degree|7000|Unknown|2|0|0| +16359|M|U|4 yr Degree|7000|Unknown|2|0|0| +16360|F|U|4 yr Degree|7000|Unknown|2|0|0| +16361|M|M|Advanced Degree|7000|Unknown|2|0|0| +16362|F|M|Advanced Degree|7000|Unknown|2|0|0| +16363|M|S|Advanced Degree|7000|Unknown|2|0|0| +16364|F|S|Advanced Degree|7000|Unknown|2|0|0| +16365|M|D|Advanced Degree|7000|Unknown|2|0|0| +16366|F|D|Advanced Degree|7000|Unknown|2|0|0| +16367|M|W|Advanced Degree|7000|Unknown|2|0|0| +16368|F|W|Advanced Degree|7000|Unknown|2|0|0| +16369|M|U|Advanced Degree|7000|Unknown|2|0|0| +16370|F|U|Advanced Degree|7000|Unknown|2|0|0| +16371|M|M|Unknown|7000|Unknown|2|0|0| +16372|F|M|Unknown|7000|Unknown|2|0|0| +16373|M|S|Unknown|7000|Unknown|2|0|0| +16374|F|S|Unknown|7000|Unknown|2|0|0| +16375|M|D|Unknown|7000|Unknown|2|0|0| +16376|F|D|Unknown|7000|Unknown|2|0|0| +16377|M|W|Unknown|7000|Unknown|2|0|0| +16378|F|W|Unknown|7000|Unknown|2|0|0| +16379|M|U|Unknown|7000|Unknown|2|0|0| +16380|F|U|Unknown|7000|Unknown|2|0|0| +16381|M|M|Primary|7500|Unknown|2|0|0| +16382|F|M|Primary|7500|Unknown|2|0|0| +16383|M|S|Primary|7500|Unknown|2|0|0| +16384|F|S|Primary|7500|Unknown|2|0|0| +16385|M|D|Primary|7500|Unknown|2|0|0| +16386|F|D|Primary|7500|Unknown|2|0|0| +16387|M|W|Primary|7500|Unknown|2|0|0| +16388|F|W|Primary|7500|Unknown|2|0|0| +16389|M|U|Primary|7500|Unknown|2|0|0| +16390|F|U|Primary|7500|Unknown|2|0|0| +16391|M|M|Secondary|7500|Unknown|2|0|0| +16392|F|M|Secondary|7500|Unknown|2|0|0| +16393|M|S|Secondary|7500|Unknown|2|0|0| +16394|F|S|Secondary|7500|Unknown|2|0|0| +16395|M|D|Secondary|7500|Unknown|2|0|0| +16396|F|D|Secondary|7500|Unknown|2|0|0| +16397|M|W|Secondary|7500|Unknown|2|0|0| +16398|F|W|Secondary|7500|Unknown|2|0|0| +16399|M|U|Secondary|7500|Unknown|2|0|0| +16400|F|U|Secondary|7500|Unknown|2|0|0| +16401|M|M|College|7500|Unknown|2|0|0| +16402|F|M|College|7500|Unknown|2|0|0| +16403|M|S|College|7500|Unknown|2|0|0| +16404|F|S|College|7500|Unknown|2|0|0| +16405|M|D|College|7500|Unknown|2|0|0| +16406|F|D|College|7500|Unknown|2|0|0| +16407|M|W|College|7500|Unknown|2|0|0| +16408|F|W|College|7500|Unknown|2|0|0| +16409|M|U|College|7500|Unknown|2|0|0| +16410|F|U|College|7500|Unknown|2|0|0| +16411|M|M|2 yr Degree|7500|Unknown|2|0|0| +16412|F|M|2 yr Degree|7500|Unknown|2|0|0| +16413|M|S|2 yr Degree|7500|Unknown|2|0|0| +16414|F|S|2 yr Degree|7500|Unknown|2|0|0| +16415|M|D|2 yr Degree|7500|Unknown|2|0|0| +16416|F|D|2 yr Degree|7500|Unknown|2|0|0| +16417|M|W|2 yr Degree|7500|Unknown|2|0|0| +16418|F|W|2 yr Degree|7500|Unknown|2|0|0| +16419|M|U|2 yr Degree|7500|Unknown|2|0|0| +16420|F|U|2 yr Degree|7500|Unknown|2|0|0| +16421|M|M|4 yr Degree|7500|Unknown|2|0|0| +16422|F|M|4 yr Degree|7500|Unknown|2|0|0| +16423|M|S|4 yr Degree|7500|Unknown|2|0|0| +16424|F|S|4 yr Degree|7500|Unknown|2|0|0| +16425|M|D|4 yr Degree|7500|Unknown|2|0|0| +16426|F|D|4 yr Degree|7500|Unknown|2|0|0| +16427|M|W|4 yr Degree|7500|Unknown|2|0|0| +16428|F|W|4 yr Degree|7500|Unknown|2|0|0| +16429|M|U|4 yr Degree|7500|Unknown|2|0|0| +16430|F|U|4 yr Degree|7500|Unknown|2|0|0| +16431|M|M|Advanced Degree|7500|Unknown|2|0|0| +16432|F|M|Advanced Degree|7500|Unknown|2|0|0| +16433|M|S|Advanced Degree|7500|Unknown|2|0|0| +16434|F|S|Advanced Degree|7500|Unknown|2|0|0| +16435|M|D|Advanced Degree|7500|Unknown|2|0|0| +16436|F|D|Advanced Degree|7500|Unknown|2|0|0| +16437|M|W|Advanced Degree|7500|Unknown|2|0|0| +16438|F|W|Advanced Degree|7500|Unknown|2|0|0| +16439|M|U|Advanced Degree|7500|Unknown|2|0|0| +16440|F|U|Advanced Degree|7500|Unknown|2|0|0| +16441|M|M|Unknown|7500|Unknown|2|0|0| +16442|F|M|Unknown|7500|Unknown|2|0|0| +16443|M|S|Unknown|7500|Unknown|2|0|0| +16444|F|S|Unknown|7500|Unknown|2|0|0| +16445|M|D|Unknown|7500|Unknown|2|0|0| +16446|F|D|Unknown|7500|Unknown|2|0|0| +16447|M|W|Unknown|7500|Unknown|2|0|0| +16448|F|W|Unknown|7500|Unknown|2|0|0| +16449|M|U|Unknown|7500|Unknown|2|0|0| +16450|F|U|Unknown|7500|Unknown|2|0|0| +16451|M|M|Primary|8000|Unknown|2|0|0| +16452|F|M|Primary|8000|Unknown|2|0|0| +16453|M|S|Primary|8000|Unknown|2|0|0| +16454|F|S|Primary|8000|Unknown|2|0|0| +16455|M|D|Primary|8000|Unknown|2|0|0| +16456|F|D|Primary|8000|Unknown|2|0|0| +16457|M|W|Primary|8000|Unknown|2|0|0| +16458|F|W|Primary|8000|Unknown|2|0|0| +16459|M|U|Primary|8000|Unknown|2|0|0| +16460|F|U|Primary|8000|Unknown|2|0|0| +16461|M|M|Secondary|8000|Unknown|2|0|0| +16462|F|M|Secondary|8000|Unknown|2|0|0| +16463|M|S|Secondary|8000|Unknown|2|0|0| +16464|F|S|Secondary|8000|Unknown|2|0|0| +16465|M|D|Secondary|8000|Unknown|2|0|0| +16466|F|D|Secondary|8000|Unknown|2|0|0| +16467|M|W|Secondary|8000|Unknown|2|0|0| +16468|F|W|Secondary|8000|Unknown|2|0|0| +16469|M|U|Secondary|8000|Unknown|2|0|0| +16470|F|U|Secondary|8000|Unknown|2|0|0| +16471|M|M|College|8000|Unknown|2|0|0| +16472|F|M|College|8000|Unknown|2|0|0| +16473|M|S|College|8000|Unknown|2|0|0| +16474|F|S|College|8000|Unknown|2|0|0| +16475|M|D|College|8000|Unknown|2|0|0| +16476|F|D|College|8000|Unknown|2|0|0| +16477|M|W|College|8000|Unknown|2|0|0| +16478|F|W|College|8000|Unknown|2|0|0| +16479|M|U|College|8000|Unknown|2|0|0| +16480|F|U|College|8000|Unknown|2|0|0| +16481|M|M|2 yr Degree|8000|Unknown|2|0|0| +16482|F|M|2 yr Degree|8000|Unknown|2|0|0| +16483|M|S|2 yr Degree|8000|Unknown|2|0|0| +16484|F|S|2 yr Degree|8000|Unknown|2|0|0| +16485|M|D|2 yr Degree|8000|Unknown|2|0|0| +16486|F|D|2 yr Degree|8000|Unknown|2|0|0| +16487|M|W|2 yr Degree|8000|Unknown|2|0|0| +16488|F|W|2 yr Degree|8000|Unknown|2|0|0| +16489|M|U|2 yr Degree|8000|Unknown|2|0|0| +16490|F|U|2 yr Degree|8000|Unknown|2|0|0| +16491|M|M|4 yr Degree|8000|Unknown|2|0|0| +16492|F|M|4 yr Degree|8000|Unknown|2|0|0| +16493|M|S|4 yr Degree|8000|Unknown|2|0|0| +16494|F|S|4 yr Degree|8000|Unknown|2|0|0| +16495|M|D|4 yr Degree|8000|Unknown|2|0|0| +16496|F|D|4 yr Degree|8000|Unknown|2|0|0| +16497|M|W|4 yr Degree|8000|Unknown|2|0|0| +16498|F|W|4 yr Degree|8000|Unknown|2|0|0| +16499|M|U|4 yr Degree|8000|Unknown|2|0|0| +16500|F|U|4 yr Degree|8000|Unknown|2|0|0| +16501|M|M|Advanced Degree|8000|Unknown|2|0|0| +16502|F|M|Advanced Degree|8000|Unknown|2|0|0| +16503|M|S|Advanced Degree|8000|Unknown|2|0|0| +16504|F|S|Advanced Degree|8000|Unknown|2|0|0| +16505|M|D|Advanced Degree|8000|Unknown|2|0|0| +16506|F|D|Advanced Degree|8000|Unknown|2|0|0| +16507|M|W|Advanced Degree|8000|Unknown|2|0|0| +16508|F|W|Advanced Degree|8000|Unknown|2|0|0| +16509|M|U|Advanced Degree|8000|Unknown|2|0|0| +16510|F|U|Advanced Degree|8000|Unknown|2|0|0| +16511|M|M|Unknown|8000|Unknown|2|0|0| +16512|F|M|Unknown|8000|Unknown|2|0|0| +16513|M|S|Unknown|8000|Unknown|2|0|0| +16514|F|S|Unknown|8000|Unknown|2|0|0| +16515|M|D|Unknown|8000|Unknown|2|0|0| +16516|F|D|Unknown|8000|Unknown|2|0|0| +16517|M|W|Unknown|8000|Unknown|2|0|0| +16518|F|W|Unknown|8000|Unknown|2|0|0| +16519|M|U|Unknown|8000|Unknown|2|0|0| +16520|F|U|Unknown|8000|Unknown|2|0|0| +16521|M|M|Primary|8500|Unknown|2|0|0| +16522|F|M|Primary|8500|Unknown|2|0|0| +16523|M|S|Primary|8500|Unknown|2|0|0| +16524|F|S|Primary|8500|Unknown|2|0|0| +16525|M|D|Primary|8500|Unknown|2|0|0| +16526|F|D|Primary|8500|Unknown|2|0|0| +16527|M|W|Primary|8500|Unknown|2|0|0| +16528|F|W|Primary|8500|Unknown|2|0|0| +16529|M|U|Primary|8500|Unknown|2|0|0| +16530|F|U|Primary|8500|Unknown|2|0|0| +16531|M|M|Secondary|8500|Unknown|2|0|0| +16532|F|M|Secondary|8500|Unknown|2|0|0| +16533|M|S|Secondary|8500|Unknown|2|0|0| +16534|F|S|Secondary|8500|Unknown|2|0|0| +16535|M|D|Secondary|8500|Unknown|2|0|0| +16536|F|D|Secondary|8500|Unknown|2|0|0| +16537|M|W|Secondary|8500|Unknown|2|0|0| +16538|F|W|Secondary|8500|Unknown|2|0|0| +16539|M|U|Secondary|8500|Unknown|2|0|0| +16540|F|U|Secondary|8500|Unknown|2|0|0| +16541|M|M|College|8500|Unknown|2|0|0| +16542|F|M|College|8500|Unknown|2|0|0| +16543|M|S|College|8500|Unknown|2|0|0| +16544|F|S|College|8500|Unknown|2|0|0| +16545|M|D|College|8500|Unknown|2|0|0| +16546|F|D|College|8500|Unknown|2|0|0| +16547|M|W|College|8500|Unknown|2|0|0| +16548|F|W|College|8500|Unknown|2|0|0| +16549|M|U|College|8500|Unknown|2|0|0| +16550|F|U|College|8500|Unknown|2|0|0| +16551|M|M|2 yr Degree|8500|Unknown|2|0|0| +16552|F|M|2 yr Degree|8500|Unknown|2|0|0| +16553|M|S|2 yr Degree|8500|Unknown|2|0|0| +16554|F|S|2 yr Degree|8500|Unknown|2|0|0| +16555|M|D|2 yr Degree|8500|Unknown|2|0|0| +16556|F|D|2 yr Degree|8500|Unknown|2|0|0| +16557|M|W|2 yr Degree|8500|Unknown|2|0|0| +16558|F|W|2 yr Degree|8500|Unknown|2|0|0| +16559|M|U|2 yr Degree|8500|Unknown|2|0|0| +16560|F|U|2 yr Degree|8500|Unknown|2|0|0| +16561|M|M|4 yr Degree|8500|Unknown|2|0|0| +16562|F|M|4 yr Degree|8500|Unknown|2|0|0| +16563|M|S|4 yr Degree|8500|Unknown|2|0|0| +16564|F|S|4 yr Degree|8500|Unknown|2|0|0| +16565|M|D|4 yr Degree|8500|Unknown|2|0|0| +16566|F|D|4 yr Degree|8500|Unknown|2|0|0| +16567|M|W|4 yr Degree|8500|Unknown|2|0|0| +16568|F|W|4 yr Degree|8500|Unknown|2|0|0| +16569|M|U|4 yr Degree|8500|Unknown|2|0|0| +16570|F|U|4 yr Degree|8500|Unknown|2|0|0| +16571|M|M|Advanced Degree|8500|Unknown|2|0|0| +16572|F|M|Advanced Degree|8500|Unknown|2|0|0| +16573|M|S|Advanced Degree|8500|Unknown|2|0|0| +16574|F|S|Advanced Degree|8500|Unknown|2|0|0| +16575|M|D|Advanced Degree|8500|Unknown|2|0|0| +16576|F|D|Advanced Degree|8500|Unknown|2|0|0| +16577|M|W|Advanced Degree|8500|Unknown|2|0|0| +16578|F|W|Advanced Degree|8500|Unknown|2|0|0| +16579|M|U|Advanced Degree|8500|Unknown|2|0|0| +16580|F|U|Advanced Degree|8500|Unknown|2|0|0| +16581|M|M|Unknown|8500|Unknown|2|0|0| +16582|F|M|Unknown|8500|Unknown|2|0|0| +16583|M|S|Unknown|8500|Unknown|2|0|0| +16584|F|S|Unknown|8500|Unknown|2|0|0| +16585|M|D|Unknown|8500|Unknown|2|0|0| +16586|F|D|Unknown|8500|Unknown|2|0|0| +16587|M|W|Unknown|8500|Unknown|2|0|0| +16588|F|W|Unknown|8500|Unknown|2|0|0| +16589|M|U|Unknown|8500|Unknown|2|0|0| +16590|F|U|Unknown|8500|Unknown|2|0|0| +16591|M|M|Primary|9000|Unknown|2|0|0| +16592|F|M|Primary|9000|Unknown|2|0|0| +16593|M|S|Primary|9000|Unknown|2|0|0| +16594|F|S|Primary|9000|Unknown|2|0|0| +16595|M|D|Primary|9000|Unknown|2|0|0| +16596|F|D|Primary|9000|Unknown|2|0|0| +16597|M|W|Primary|9000|Unknown|2|0|0| +16598|F|W|Primary|9000|Unknown|2|0|0| +16599|M|U|Primary|9000|Unknown|2|0|0| +16600|F|U|Primary|9000|Unknown|2|0|0| +16601|M|M|Secondary|9000|Unknown|2|0|0| +16602|F|M|Secondary|9000|Unknown|2|0|0| +16603|M|S|Secondary|9000|Unknown|2|0|0| +16604|F|S|Secondary|9000|Unknown|2|0|0| +16605|M|D|Secondary|9000|Unknown|2|0|0| +16606|F|D|Secondary|9000|Unknown|2|0|0| +16607|M|W|Secondary|9000|Unknown|2|0|0| +16608|F|W|Secondary|9000|Unknown|2|0|0| +16609|M|U|Secondary|9000|Unknown|2|0|0| +16610|F|U|Secondary|9000|Unknown|2|0|0| +16611|M|M|College|9000|Unknown|2|0|0| +16612|F|M|College|9000|Unknown|2|0|0| +16613|M|S|College|9000|Unknown|2|0|0| +16614|F|S|College|9000|Unknown|2|0|0| +16615|M|D|College|9000|Unknown|2|0|0| +16616|F|D|College|9000|Unknown|2|0|0| +16617|M|W|College|9000|Unknown|2|0|0| +16618|F|W|College|9000|Unknown|2|0|0| +16619|M|U|College|9000|Unknown|2|0|0| +16620|F|U|College|9000|Unknown|2|0|0| +16621|M|M|2 yr Degree|9000|Unknown|2|0|0| +16622|F|M|2 yr Degree|9000|Unknown|2|0|0| +16623|M|S|2 yr Degree|9000|Unknown|2|0|0| +16624|F|S|2 yr Degree|9000|Unknown|2|0|0| +16625|M|D|2 yr Degree|9000|Unknown|2|0|0| +16626|F|D|2 yr Degree|9000|Unknown|2|0|0| +16627|M|W|2 yr Degree|9000|Unknown|2|0|0| +16628|F|W|2 yr Degree|9000|Unknown|2|0|0| +16629|M|U|2 yr Degree|9000|Unknown|2|0|0| +16630|F|U|2 yr Degree|9000|Unknown|2|0|0| +16631|M|M|4 yr Degree|9000|Unknown|2|0|0| +16632|F|M|4 yr Degree|9000|Unknown|2|0|0| +16633|M|S|4 yr Degree|9000|Unknown|2|0|0| +16634|F|S|4 yr Degree|9000|Unknown|2|0|0| +16635|M|D|4 yr Degree|9000|Unknown|2|0|0| +16636|F|D|4 yr Degree|9000|Unknown|2|0|0| +16637|M|W|4 yr Degree|9000|Unknown|2|0|0| +16638|F|W|4 yr Degree|9000|Unknown|2|0|0| +16639|M|U|4 yr Degree|9000|Unknown|2|0|0| +16640|F|U|4 yr Degree|9000|Unknown|2|0|0| +16641|M|M|Advanced Degree|9000|Unknown|2|0|0| +16642|F|M|Advanced Degree|9000|Unknown|2|0|0| +16643|M|S|Advanced Degree|9000|Unknown|2|0|0| +16644|F|S|Advanced Degree|9000|Unknown|2|0|0| +16645|M|D|Advanced Degree|9000|Unknown|2|0|0| +16646|F|D|Advanced Degree|9000|Unknown|2|0|0| +16647|M|W|Advanced Degree|9000|Unknown|2|0|0| +16648|F|W|Advanced Degree|9000|Unknown|2|0|0| +16649|M|U|Advanced Degree|9000|Unknown|2|0|0| +16650|F|U|Advanced Degree|9000|Unknown|2|0|0| +16651|M|M|Unknown|9000|Unknown|2|0|0| +16652|F|M|Unknown|9000|Unknown|2|0|0| +16653|M|S|Unknown|9000|Unknown|2|0|0| +16654|F|S|Unknown|9000|Unknown|2|0|0| +16655|M|D|Unknown|9000|Unknown|2|0|0| +16656|F|D|Unknown|9000|Unknown|2|0|0| +16657|M|W|Unknown|9000|Unknown|2|0|0| +16658|F|W|Unknown|9000|Unknown|2|0|0| +16659|M|U|Unknown|9000|Unknown|2|0|0| +16660|F|U|Unknown|9000|Unknown|2|0|0| +16661|M|M|Primary|9500|Unknown|2|0|0| +16662|F|M|Primary|9500|Unknown|2|0|0| +16663|M|S|Primary|9500|Unknown|2|0|0| +16664|F|S|Primary|9500|Unknown|2|0|0| +16665|M|D|Primary|9500|Unknown|2|0|0| +16666|F|D|Primary|9500|Unknown|2|0|0| +16667|M|W|Primary|9500|Unknown|2|0|0| +16668|F|W|Primary|9500|Unknown|2|0|0| +16669|M|U|Primary|9500|Unknown|2|0|0| +16670|F|U|Primary|9500|Unknown|2|0|0| +16671|M|M|Secondary|9500|Unknown|2|0|0| +16672|F|M|Secondary|9500|Unknown|2|0|0| +16673|M|S|Secondary|9500|Unknown|2|0|0| +16674|F|S|Secondary|9500|Unknown|2|0|0| +16675|M|D|Secondary|9500|Unknown|2|0|0| +16676|F|D|Secondary|9500|Unknown|2|0|0| +16677|M|W|Secondary|9500|Unknown|2|0|0| +16678|F|W|Secondary|9500|Unknown|2|0|0| +16679|M|U|Secondary|9500|Unknown|2|0|0| +16680|F|U|Secondary|9500|Unknown|2|0|0| +16681|M|M|College|9500|Unknown|2|0|0| +16682|F|M|College|9500|Unknown|2|0|0| +16683|M|S|College|9500|Unknown|2|0|0| +16684|F|S|College|9500|Unknown|2|0|0| +16685|M|D|College|9500|Unknown|2|0|0| +16686|F|D|College|9500|Unknown|2|0|0| +16687|M|W|College|9500|Unknown|2|0|0| +16688|F|W|College|9500|Unknown|2|0|0| +16689|M|U|College|9500|Unknown|2|0|0| +16690|F|U|College|9500|Unknown|2|0|0| +16691|M|M|2 yr Degree|9500|Unknown|2|0|0| +16692|F|M|2 yr Degree|9500|Unknown|2|0|0| +16693|M|S|2 yr Degree|9500|Unknown|2|0|0| +16694|F|S|2 yr Degree|9500|Unknown|2|0|0| +16695|M|D|2 yr Degree|9500|Unknown|2|0|0| +16696|F|D|2 yr Degree|9500|Unknown|2|0|0| +16697|M|W|2 yr Degree|9500|Unknown|2|0|0| +16698|F|W|2 yr Degree|9500|Unknown|2|0|0| +16699|M|U|2 yr Degree|9500|Unknown|2|0|0| +16700|F|U|2 yr Degree|9500|Unknown|2|0|0| +16701|M|M|4 yr Degree|9500|Unknown|2|0|0| +16702|F|M|4 yr Degree|9500|Unknown|2|0|0| +16703|M|S|4 yr Degree|9500|Unknown|2|0|0| +16704|F|S|4 yr Degree|9500|Unknown|2|0|0| +16705|M|D|4 yr Degree|9500|Unknown|2|0|0| +16706|F|D|4 yr Degree|9500|Unknown|2|0|0| +16707|M|W|4 yr Degree|9500|Unknown|2|0|0| +16708|F|W|4 yr Degree|9500|Unknown|2|0|0| +16709|M|U|4 yr Degree|9500|Unknown|2|0|0| +16710|F|U|4 yr Degree|9500|Unknown|2|0|0| +16711|M|M|Advanced Degree|9500|Unknown|2|0|0| +16712|F|M|Advanced Degree|9500|Unknown|2|0|0| +16713|M|S|Advanced Degree|9500|Unknown|2|0|0| +16714|F|S|Advanced Degree|9500|Unknown|2|0|0| +16715|M|D|Advanced Degree|9500|Unknown|2|0|0| +16716|F|D|Advanced Degree|9500|Unknown|2|0|0| +16717|M|W|Advanced Degree|9500|Unknown|2|0|0| +16718|F|W|Advanced Degree|9500|Unknown|2|0|0| +16719|M|U|Advanced Degree|9500|Unknown|2|0|0| +16720|F|U|Advanced Degree|9500|Unknown|2|0|0| +16721|M|M|Unknown|9500|Unknown|2|0|0| +16722|F|M|Unknown|9500|Unknown|2|0|0| +16723|M|S|Unknown|9500|Unknown|2|0|0| +16724|F|S|Unknown|9500|Unknown|2|0|0| +16725|M|D|Unknown|9500|Unknown|2|0|0| +16726|F|D|Unknown|9500|Unknown|2|0|0| +16727|M|W|Unknown|9500|Unknown|2|0|0| +16728|F|W|Unknown|9500|Unknown|2|0|0| +16729|M|U|Unknown|9500|Unknown|2|0|0| +16730|F|U|Unknown|9500|Unknown|2|0|0| +16731|M|M|Primary|10000|Unknown|2|0|0| +16732|F|M|Primary|10000|Unknown|2|0|0| +16733|M|S|Primary|10000|Unknown|2|0|0| +16734|F|S|Primary|10000|Unknown|2|0|0| +16735|M|D|Primary|10000|Unknown|2|0|0| +16736|F|D|Primary|10000|Unknown|2|0|0| +16737|M|W|Primary|10000|Unknown|2|0|0| +16738|F|W|Primary|10000|Unknown|2|0|0| +16739|M|U|Primary|10000|Unknown|2|0|0| +16740|F|U|Primary|10000|Unknown|2|0|0| +16741|M|M|Secondary|10000|Unknown|2|0|0| +16742|F|M|Secondary|10000|Unknown|2|0|0| +16743|M|S|Secondary|10000|Unknown|2|0|0| +16744|F|S|Secondary|10000|Unknown|2|0|0| +16745|M|D|Secondary|10000|Unknown|2|0|0| +16746|F|D|Secondary|10000|Unknown|2|0|0| +16747|M|W|Secondary|10000|Unknown|2|0|0| +16748|F|W|Secondary|10000|Unknown|2|0|0| +16749|M|U|Secondary|10000|Unknown|2|0|0| +16750|F|U|Secondary|10000|Unknown|2|0|0| +16751|M|M|College|10000|Unknown|2|0|0| +16752|F|M|College|10000|Unknown|2|0|0| +16753|M|S|College|10000|Unknown|2|0|0| +16754|F|S|College|10000|Unknown|2|0|0| +16755|M|D|College|10000|Unknown|2|0|0| +16756|F|D|College|10000|Unknown|2|0|0| +16757|M|W|College|10000|Unknown|2|0|0| +16758|F|W|College|10000|Unknown|2|0|0| +16759|M|U|College|10000|Unknown|2|0|0| +16760|F|U|College|10000|Unknown|2|0|0| +16761|M|M|2 yr Degree|10000|Unknown|2|0|0| +16762|F|M|2 yr Degree|10000|Unknown|2|0|0| +16763|M|S|2 yr Degree|10000|Unknown|2|0|0| +16764|F|S|2 yr Degree|10000|Unknown|2|0|0| +16765|M|D|2 yr Degree|10000|Unknown|2|0|0| +16766|F|D|2 yr Degree|10000|Unknown|2|0|0| +16767|M|W|2 yr Degree|10000|Unknown|2|0|0| +16768|F|W|2 yr Degree|10000|Unknown|2|0|0| +16769|M|U|2 yr Degree|10000|Unknown|2|0|0| +16770|F|U|2 yr Degree|10000|Unknown|2|0|0| +16771|M|M|4 yr Degree|10000|Unknown|2|0|0| +16772|F|M|4 yr Degree|10000|Unknown|2|0|0| +16773|M|S|4 yr Degree|10000|Unknown|2|0|0| +16774|F|S|4 yr Degree|10000|Unknown|2|0|0| +16775|M|D|4 yr Degree|10000|Unknown|2|0|0| +16776|F|D|4 yr Degree|10000|Unknown|2|0|0| +16777|M|W|4 yr Degree|10000|Unknown|2|0|0| +16778|F|W|4 yr Degree|10000|Unknown|2|0|0| +16779|M|U|4 yr Degree|10000|Unknown|2|0|0| +16780|F|U|4 yr Degree|10000|Unknown|2|0|0| +16781|M|M|Advanced Degree|10000|Unknown|2|0|0| +16782|F|M|Advanced Degree|10000|Unknown|2|0|0| +16783|M|S|Advanced Degree|10000|Unknown|2|0|0| +16784|F|S|Advanced Degree|10000|Unknown|2|0|0| +16785|M|D|Advanced Degree|10000|Unknown|2|0|0| +16786|F|D|Advanced Degree|10000|Unknown|2|0|0| +16787|M|W|Advanced Degree|10000|Unknown|2|0|0| +16788|F|W|Advanced Degree|10000|Unknown|2|0|0| +16789|M|U|Advanced Degree|10000|Unknown|2|0|0| +16790|F|U|Advanced Degree|10000|Unknown|2|0|0| +16791|M|M|Unknown|10000|Unknown|2|0|0| +16792|F|M|Unknown|10000|Unknown|2|0|0| +16793|M|S|Unknown|10000|Unknown|2|0|0| +16794|F|S|Unknown|10000|Unknown|2|0|0| +16795|M|D|Unknown|10000|Unknown|2|0|0| +16796|F|D|Unknown|10000|Unknown|2|0|0| +16797|M|W|Unknown|10000|Unknown|2|0|0| +16798|F|W|Unknown|10000|Unknown|2|0|0| +16799|M|U|Unknown|10000|Unknown|2|0|0| +16800|F|U|Unknown|10000|Unknown|2|0|0| +16801|M|M|Primary|500|Good|3|0|0| +16802|F|M|Primary|500|Good|3|0|0| +16803|M|S|Primary|500|Good|3|0|0| +16804|F|S|Primary|500|Good|3|0|0| +16805|M|D|Primary|500|Good|3|0|0| +16806|F|D|Primary|500|Good|3|0|0| +16807|M|W|Primary|500|Good|3|0|0| +16808|F|W|Primary|500|Good|3|0|0| +16809|M|U|Primary|500|Good|3|0|0| +16810|F|U|Primary|500|Good|3|0|0| +16811|M|M|Secondary|500|Good|3|0|0| +16812|F|M|Secondary|500|Good|3|0|0| +16813|M|S|Secondary|500|Good|3|0|0| +16814|F|S|Secondary|500|Good|3|0|0| +16815|M|D|Secondary|500|Good|3|0|0| +16816|F|D|Secondary|500|Good|3|0|0| +16817|M|W|Secondary|500|Good|3|0|0| +16818|F|W|Secondary|500|Good|3|0|0| +16819|M|U|Secondary|500|Good|3|0|0| +16820|F|U|Secondary|500|Good|3|0|0| +16821|M|M|College|500|Good|3|0|0| +16822|F|M|College|500|Good|3|0|0| +16823|M|S|College|500|Good|3|0|0| +16824|F|S|College|500|Good|3|0|0| +16825|M|D|College|500|Good|3|0|0| +16826|F|D|College|500|Good|3|0|0| +16827|M|W|College|500|Good|3|0|0| +16828|F|W|College|500|Good|3|0|0| +16829|M|U|College|500|Good|3|0|0| +16830|F|U|College|500|Good|3|0|0| +16831|M|M|2 yr Degree|500|Good|3|0|0| +16832|F|M|2 yr Degree|500|Good|3|0|0| +16833|M|S|2 yr Degree|500|Good|3|0|0| +16834|F|S|2 yr Degree|500|Good|3|0|0| +16835|M|D|2 yr Degree|500|Good|3|0|0| +16836|F|D|2 yr Degree|500|Good|3|0|0| +16837|M|W|2 yr Degree|500|Good|3|0|0| +16838|F|W|2 yr Degree|500|Good|3|0|0| +16839|M|U|2 yr Degree|500|Good|3|0|0| +16840|F|U|2 yr Degree|500|Good|3|0|0| +16841|M|M|4 yr Degree|500|Good|3|0|0| +16842|F|M|4 yr Degree|500|Good|3|0|0| +16843|M|S|4 yr Degree|500|Good|3|0|0| +16844|F|S|4 yr Degree|500|Good|3|0|0| +16845|M|D|4 yr Degree|500|Good|3|0|0| +16846|F|D|4 yr Degree|500|Good|3|0|0| +16847|M|W|4 yr Degree|500|Good|3|0|0| +16848|F|W|4 yr Degree|500|Good|3|0|0| +16849|M|U|4 yr Degree|500|Good|3|0|0| +16850|F|U|4 yr Degree|500|Good|3|0|0| +16851|M|M|Advanced Degree|500|Good|3|0|0| +16852|F|M|Advanced Degree|500|Good|3|0|0| +16853|M|S|Advanced Degree|500|Good|3|0|0| +16854|F|S|Advanced Degree|500|Good|3|0|0| +16855|M|D|Advanced Degree|500|Good|3|0|0| +16856|F|D|Advanced Degree|500|Good|3|0|0| +16857|M|W|Advanced Degree|500|Good|3|0|0| +16858|F|W|Advanced Degree|500|Good|3|0|0| +16859|M|U|Advanced Degree|500|Good|3|0|0| +16860|F|U|Advanced Degree|500|Good|3|0|0| +16861|M|M|Unknown|500|Good|3|0|0| +16862|F|M|Unknown|500|Good|3|0|0| +16863|M|S|Unknown|500|Good|3|0|0| +16864|F|S|Unknown|500|Good|3|0|0| +16865|M|D|Unknown|500|Good|3|0|0| +16866|F|D|Unknown|500|Good|3|0|0| +16867|M|W|Unknown|500|Good|3|0|0| +16868|F|W|Unknown|500|Good|3|0|0| +16869|M|U|Unknown|500|Good|3|0|0| +16870|F|U|Unknown|500|Good|3|0|0| +16871|M|M|Primary|1000|Good|3|0|0| +16872|F|M|Primary|1000|Good|3|0|0| +16873|M|S|Primary|1000|Good|3|0|0| +16874|F|S|Primary|1000|Good|3|0|0| +16875|M|D|Primary|1000|Good|3|0|0| +16876|F|D|Primary|1000|Good|3|0|0| +16877|M|W|Primary|1000|Good|3|0|0| +16878|F|W|Primary|1000|Good|3|0|0| +16879|M|U|Primary|1000|Good|3|0|0| +16880|F|U|Primary|1000|Good|3|0|0| +16881|M|M|Secondary|1000|Good|3|0|0| +16882|F|M|Secondary|1000|Good|3|0|0| +16883|M|S|Secondary|1000|Good|3|0|0| +16884|F|S|Secondary|1000|Good|3|0|0| +16885|M|D|Secondary|1000|Good|3|0|0| +16886|F|D|Secondary|1000|Good|3|0|0| +16887|M|W|Secondary|1000|Good|3|0|0| +16888|F|W|Secondary|1000|Good|3|0|0| +16889|M|U|Secondary|1000|Good|3|0|0| +16890|F|U|Secondary|1000|Good|3|0|0| +16891|M|M|College|1000|Good|3|0|0| +16892|F|M|College|1000|Good|3|0|0| +16893|M|S|College|1000|Good|3|0|0| +16894|F|S|College|1000|Good|3|0|0| +16895|M|D|College|1000|Good|3|0|0| +16896|F|D|College|1000|Good|3|0|0| +16897|M|W|College|1000|Good|3|0|0| +16898|F|W|College|1000|Good|3|0|0| +16899|M|U|College|1000|Good|3|0|0| +16900|F|U|College|1000|Good|3|0|0| +16901|M|M|2 yr Degree|1000|Good|3|0|0| +16902|F|M|2 yr Degree|1000|Good|3|0|0| +16903|M|S|2 yr Degree|1000|Good|3|0|0| +16904|F|S|2 yr Degree|1000|Good|3|0|0| +16905|M|D|2 yr Degree|1000|Good|3|0|0| +16906|F|D|2 yr Degree|1000|Good|3|0|0| +16907|M|W|2 yr Degree|1000|Good|3|0|0| +16908|F|W|2 yr Degree|1000|Good|3|0|0| +16909|M|U|2 yr Degree|1000|Good|3|0|0| +16910|F|U|2 yr Degree|1000|Good|3|0|0| +16911|M|M|4 yr Degree|1000|Good|3|0|0| +16912|F|M|4 yr Degree|1000|Good|3|0|0| +16913|M|S|4 yr Degree|1000|Good|3|0|0| +16914|F|S|4 yr Degree|1000|Good|3|0|0| +16915|M|D|4 yr Degree|1000|Good|3|0|0| +16916|F|D|4 yr Degree|1000|Good|3|0|0| +16917|M|W|4 yr Degree|1000|Good|3|0|0| +16918|F|W|4 yr Degree|1000|Good|3|0|0| +16919|M|U|4 yr Degree|1000|Good|3|0|0| +16920|F|U|4 yr Degree|1000|Good|3|0|0| +16921|M|M|Advanced Degree|1000|Good|3|0|0| +16922|F|M|Advanced Degree|1000|Good|3|0|0| +16923|M|S|Advanced Degree|1000|Good|3|0|0| +16924|F|S|Advanced Degree|1000|Good|3|0|0| +16925|M|D|Advanced Degree|1000|Good|3|0|0| +16926|F|D|Advanced Degree|1000|Good|3|0|0| +16927|M|W|Advanced Degree|1000|Good|3|0|0| +16928|F|W|Advanced Degree|1000|Good|3|0|0| +16929|M|U|Advanced Degree|1000|Good|3|0|0| +16930|F|U|Advanced Degree|1000|Good|3|0|0| +16931|M|M|Unknown|1000|Good|3|0|0| +16932|F|M|Unknown|1000|Good|3|0|0| +16933|M|S|Unknown|1000|Good|3|0|0| +16934|F|S|Unknown|1000|Good|3|0|0| +16935|M|D|Unknown|1000|Good|3|0|0| +16936|F|D|Unknown|1000|Good|3|0|0| +16937|M|W|Unknown|1000|Good|3|0|0| +16938|F|W|Unknown|1000|Good|3|0|0| +16939|M|U|Unknown|1000|Good|3|0|0| +16940|F|U|Unknown|1000|Good|3|0|0| +16941|M|M|Primary|1500|Good|3|0|0| +16942|F|M|Primary|1500|Good|3|0|0| +16943|M|S|Primary|1500|Good|3|0|0| +16944|F|S|Primary|1500|Good|3|0|0| +16945|M|D|Primary|1500|Good|3|0|0| +16946|F|D|Primary|1500|Good|3|0|0| +16947|M|W|Primary|1500|Good|3|0|0| +16948|F|W|Primary|1500|Good|3|0|0| +16949|M|U|Primary|1500|Good|3|0|0| +16950|F|U|Primary|1500|Good|3|0|0| +16951|M|M|Secondary|1500|Good|3|0|0| +16952|F|M|Secondary|1500|Good|3|0|0| +16953|M|S|Secondary|1500|Good|3|0|0| +16954|F|S|Secondary|1500|Good|3|0|0| +16955|M|D|Secondary|1500|Good|3|0|0| +16956|F|D|Secondary|1500|Good|3|0|0| +16957|M|W|Secondary|1500|Good|3|0|0| +16958|F|W|Secondary|1500|Good|3|0|0| +16959|M|U|Secondary|1500|Good|3|0|0| +16960|F|U|Secondary|1500|Good|3|0|0| +16961|M|M|College|1500|Good|3|0|0| +16962|F|M|College|1500|Good|3|0|0| +16963|M|S|College|1500|Good|3|0|0| +16964|F|S|College|1500|Good|3|0|0| +16965|M|D|College|1500|Good|3|0|0| +16966|F|D|College|1500|Good|3|0|0| +16967|M|W|College|1500|Good|3|0|0| +16968|F|W|College|1500|Good|3|0|0| +16969|M|U|College|1500|Good|3|0|0| +16970|F|U|College|1500|Good|3|0|0| +16971|M|M|2 yr Degree|1500|Good|3|0|0| +16972|F|M|2 yr Degree|1500|Good|3|0|0| +16973|M|S|2 yr Degree|1500|Good|3|0|0| +16974|F|S|2 yr Degree|1500|Good|3|0|0| +16975|M|D|2 yr Degree|1500|Good|3|0|0| +16976|F|D|2 yr Degree|1500|Good|3|0|0| +16977|M|W|2 yr Degree|1500|Good|3|0|0| +16978|F|W|2 yr Degree|1500|Good|3|0|0| +16979|M|U|2 yr Degree|1500|Good|3|0|0| +16980|F|U|2 yr Degree|1500|Good|3|0|0| +16981|M|M|4 yr Degree|1500|Good|3|0|0| +16982|F|M|4 yr Degree|1500|Good|3|0|0| +16983|M|S|4 yr Degree|1500|Good|3|0|0| +16984|F|S|4 yr Degree|1500|Good|3|0|0| +16985|M|D|4 yr Degree|1500|Good|3|0|0| +16986|F|D|4 yr Degree|1500|Good|3|0|0| +16987|M|W|4 yr Degree|1500|Good|3|0|0| +16988|F|W|4 yr Degree|1500|Good|3|0|0| +16989|M|U|4 yr Degree|1500|Good|3|0|0| +16990|F|U|4 yr Degree|1500|Good|3|0|0| +16991|M|M|Advanced Degree|1500|Good|3|0|0| +16992|F|M|Advanced Degree|1500|Good|3|0|0| +16993|M|S|Advanced Degree|1500|Good|3|0|0| +16994|F|S|Advanced Degree|1500|Good|3|0|0| +16995|M|D|Advanced Degree|1500|Good|3|0|0| +16996|F|D|Advanced Degree|1500|Good|3|0|0| +16997|M|W|Advanced Degree|1500|Good|3|0|0| +16998|F|W|Advanced Degree|1500|Good|3|0|0| +16999|M|U|Advanced Degree|1500|Good|3|0|0| +17000|F|U|Advanced Degree|1500|Good|3|0|0| +17001|M|M|Unknown|1500|Good|3|0|0| +17002|F|M|Unknown|1500|Good|3|0|0| +17003|M|S|Unknown|1500|Good|3|0|0| +17004|F|S|Unknown|1500|Good|3|0|0| +17005|M|D|Unknown|1500|Good|3|0|0| +17006|F|D|Unknown|1500|Good|3|0|0| +17007|M|W|Unknown|1500|Good|3|0|0| +17008|F|W|Unknown|1500|Good|3|0|0| +17009|M|U|Unknown|1500|Good|3|0|0| +17010|F|U|Unknown|1500|Good|3|0|0| +17011|M|M|Primary|2000|Good|3|0|0| +17012|F|M|Primary|2000|Good|3|0|0| +17013|M|S|Primary|2000|Good|3|0|0| +17014|F|S|Primary|2000|Good|3|0|0| +17015|M|D|Primary|2000|Good|3|0|0| +17016|F|D|Primary|2000|Good|3|0|0| +17017|M|W|Primary|2000|Good|3|0|0| +17018|F|W|Primary|2000|Good|3|0|0| +17019|M|U|Primary|2000|Good|3|0|0| +17020|F|U|Primary|2000|Good|3|0|0| +17021|M|M|Secondary|2000|Good|3|0|0| +17022|F|M|Secondary|2000|Good|3|0|0| +17023|M|S|Secondary|2000|Good|3|0|0| +17024|F|S|Secondary|2000|Good|3|0|0| +17025|M|D|Secondary|2000|Good|3|0|0| +17026|F|D|Secondary|2000|Good|3|0|0| +17027|M|W|Secondary|2000|Good|3|0|0| +17028|F|W|Secondary|2000|Good|3|0|0| +17029|M|U|Secondary|2000|Good|3|0|0| +17030|F|U|Secondary|2000|Good|3|0|0| +17031|M|M|College|2000|Good|3|0|0| +17032|F|M|College|2000|Good|3|0|0| +17033|M|S|College|2000|Good|3|0|0| +17034|F|S|College|2000|Good|3|0|0| +17035|M|D|College|2000|Good|3|0|0| +17036|F|D|College|2000|Good|3|0|0| +17037|M|W|College|2000|Good|3|0|0| +17038|F|W|College|2000|Good|3|0|0| +17039|M|U|College|2000|Good|3|0|0| +17040|F|U|College|2000|Good|3|0|0| +17041|M|M|2 yr Degree|2000|Good|3|0|0| +17042|F|M|2 yr Degree|2000|Good|3|0|0| +17043|M|S|2 yr Degree|2000|Good|3|0|0| +17044|F|S|2 yr Degree|2000|Good|3|0|0| +17045|M|D|2 yr Degree|2000|Good|3|0|0| +17046|F|D|2 yr Degree|2000|Good|3|0|0| +17047|M|W|2 yr Degree|2000|Good|3|0|0| +17048|F|W|2 yr Degree|2000|Good|3|0|0| +17049|M|U|2 yr Degree|2000|Good|3|0|0| +17050|F|U|2 yr Degree|2000|Good|3|0|0| +17051|M|M|4 yr Degree|2000|Good|3|0|0| +17052|F|M|4 yr Degree|2000|Good|3|0|0| +17053|M|S|4 yr Degree|2000|Good|3|0|0| +17054|F|S|4 yr Degree|2000|Good|3|0|0| +17055|M|D|4 yr Degree|2000|Good|3|0|0| +17056|F|D|4 yr Degree|2000|Good|3|0|0| +17057|M|W|4 yr Degree|2000|Good|3|0|0| +17058|F|W|4 yr Degree|2000|Good|3|0|0| +17059|M|U|4 yr Degree|2000|Good|3|0|0| +17060|F|U|4 yr Degree|2000|Good|3|0|0| +17061|M|M|Advanced Degree|2000|Good|3|0|0| +17062|F|M|Advanced Degree|2000|Good|3|0|0| +17063|M|S|Advanced Degree|2000|Good|3|0|0| +17064|F|S|Advanced Degree|2000|Good|3|0|0| +17065|M|D|Advanced Degree|2000|Good|3|0|0| +17066|F|D|Advanced Degree|2000|Good|3|0|0| +17067|M|W|Advanced Degree|2000|Good|3|0|0| +17068|F|W|Advanced Degree|2000|Good|3|0|0| +17069|M|U|Advanced Degree|2000|Good|3|0|0| +17070|F|U|Advanced Degree|2000|Good|3|0|0| +17071|M|M|Unknown|2000|Good|3|0|0| +17072|F|M|Unknown|2000|Good|3|0|0| +17073|M|S|Unknown|2000|Good|3|0|0| +17074|F|S|Unknown|2000|Good|3|0|0| +17075|M|D|Unknown|2000|Good|3|0|0| +17076|F|D|Unknown|2000|Good|3|0|0| +17077|M|W|Unknown|2000|Good|3|0|0| +17078|F|W|Unknown|2000|Good|3|0|0| +17079|M|U|Unknown|2000|Good|3|0|0| +17080|F|U|Unknown|2000|Good|3|0|0| +17081|M|M|Primary|2500|Good|3|0|0| +17082|F|M|Primary|2500|Good|3|0|0| +17083|M|S|Primary|2500|Good|3|0|0| +17084|F|S|Primary|2500|Good|3|0|0| +17085|M|D|Primary|2500|Good|3|0|0| +17086|F|D|Primary|2500|Good|3|0|0| +17087|M|W|Primary|2500|Good|3|0|0| +17088|F|W|Primary|2500|Good|3|0|0| +17089|M|U|Primary|2500|Good|3|0|0| +17090|F|U|Primary|2500|Good|3|0|0| +17091|M|M|Secondary|2500|Good|3|0|0| +17092|F|M|Secondary|2500|Good|3|0|0| +17093|M|S|Secondary|2500|Good|3|0|0| +17094|F|S|Secondary|2500|Good|3|0|0| +17095|M|D|Secondary|2500|Good|3|0|0| +17096|F|D|Secondary|2500|Good|3|0|0| +17097|M|W|Secondary|2500|Good|3|0|0| +17098|F|W|Secondary|2500|Good|3|0|0| +17099|M|U|Secondary|2500|Good|3|0|0| +17100|F|U|Secondary|2500|Good|3|0|0| +17101|M|M|College|2500|Good|3|0|0| +17102|F|M|College|2500|Good|3|0|0| +17103|M|S|College|2500|Good|3|0|0| +17104|F|S|College|2500|Good|3|0|0| +17105|M|D|College|2500|Good|3|0|0| +17106|F|D|College|2500|Good|3|0|0| +17107|M|W|College|2500|Good|3|0|0| +17108|F|W|College|2500|Good|3|0|0| +17109|M|U|College|2500|Good|3|0|0| +17110|F|U|College|2500|Good|3|0|0| +17111|M|M|2 yr Degree|2500|Good|3|0|0| +17112|F|M|2 yr Degree|2500|Good|3|0|0| +17113|M|S|2 yr Degree|2500|Good|3|0|0| +17114|F|S|2 yr Degree|2500|Good|3|0|0| +17115|M|D|2 yr Degree|2500|Good|3|0|0| +17116|F|D|2 yr Degree|2500|Good|3|0|0| +17117|M|W|2 yr Degree|2500|Good|3|0|0| +17118|F|W|2 yr Degree|2500|Good|3|0|0| +17119|M|U|2 yr Degree|2500|Good|3|0|0| +17120|F|U|2 yr Degree|2500|Good|3|0|0| +17121|M|M|4 yr Degree|2500|Good|3|0|0| +17122|F|M|4 yr Degree|2500|Good|3|0|0| +17123|M|S|4 yr Degree|2500|Good|3|0|0| +17124|F|S|4 yr Degree|2500|Good|3|0|0| +17125|M|D|4 yr Degree|2500|Good|3|0|0| +17126|F|D|4 yr Degree|2500|Good|3|0|0| +17127|M|W|4 yr Degree|2500|Good|3|0|0| +17128|F|W|4 yr Degree|2500|Good|3|0|0| +17129|M|U|4 yr Degree|2500|Good|3|0|0| +17130|F|U|4 yr Degree|2500|Good|3|0|0| +17131|M|M|Advanced Degree|2500|Good|3|0|0| +17132|F|M|Advanced Degree|2500|Good|3|0|0| +17133|M|S|Advanced Degree|2500|Good|3|0|0| +17134|F|S|Advanced Degree|2500|Good|3|0|0| +17135|M|D|Advanced Degree|2500|Good|3|0|0| +17136|F|D|Advanced Degree|2500|Good|3|0|0| +17137|M|W|Advanced Degree|2500|Good|3|0|0| +17138|F|W|Advanced Degree|2500|Good|3|0|0| +17139|M|U|Advanced Degree|2500|Good|3|0|0| +17140|F|U|Advanced Degree|2500|Good|3|0|0| +17141|M|M|Unknown|2500|Good|3|0|0| +17142|F|M|Unknown|2500|Good|3|0|0| +17143|M|S|Unknown|2500|Good|3|0|0| +17144|F|S|Unknown|2500|Good|3|0|0| +17145|M|D|Unknown|2500|Good|3|0|0| +17146|F|D|Unknown|2500|Good|3|0|0| +17147|M|W|Unknown|2500|Good|3|0|0| +17148|F|W|Unknown|2500|Good|3|0|0| +17149|M|U|Unknown|2500|Good|3|0|0| +17150|F|U|Unknown|2500|Good|3|0|0| +17151|M|M|Primary|3000|Good|3|0|0| +17152|F|M|Primary|3000|Good|3|0|0| +17153|M|S|Primary|3000|Good|3|0|0| +17154|F|S|Primary|3000|Good|3|0|0| +17155|M|D|Primary|3000|Good|3|0|0| +17156|F|D|Primary|3000|Good|3|0|0| +17157|M|W|Primary|3000|Good|3|0|0| +17158|F|W|Primary|3000|Good|3|0|0| +17159|M|U|Primary|3000|Good|3|0|0| +17160|F|U|Primary|3000|Good|3|0|0| +17161|M|M|Secondary|3000|Good|3|0|0| +17162|F|M|Secondary|3000|Good|3|0|0| +17163|M|S|Secondary|3000|Good|3|0|0| +17164|F|S|Secondary|3000|Good|3|0|0| +17165|M|D|Secondary|3000|Good|3|0|0| +17166|F|D|Secondary|3000|Good|3|0|0| +17167|M|W|Secondary|3000|Good|3|0|0| +17168|F|W|Secondary|3000|Good|3|0|0| +17169|M|U|Secondary|3000|Good|3|0|0| +17170|F|U|Secondary|3000|Good|3|0|0| +17171|M|M|College|3000|Good|3|0|0| +17172|F|M|College|3000|Good|3|0|0| +17173|M|S|College|3000|Good|3|0|0| +17174|F|S|College|3000|Good|3|0|0| +17175|M|D|College|3000|Good|3|0|0| +17176|F|D|College|3000|Good|3|0|0| +17177|M|W|College|3000|Good|3|0|0| +17178|F|W|College|3000|Good|3|0|0| +17179|M|U|College|3000|Good|3|0|0| +17180|F|U|College|3000|Good|3|0|0| +17181|M|M|2 yr Degree|3000|Good|3|0|0| +17182|F|M|2 yr Degree|3000|Good|3|0|0| +17183|M|S|2 yr Degree|3000|Good|3|0|0| +17184|F|S|2 yr Degree|3000|Good|3|0|0| +17185|M|D|2 yr Degree|3000|Good|3|0|0| +17186|F|D|2 yr Degree|3000|Good|3|0|0| +17187|M|W|2 yr Degree|3000|Good|3|0|0| +17188|F|W|2 yr Degree|3000|Good|3|0|0| +17189|M|U|2 yr Degree|3000|Good|3|0|0| +17190|F|U|2 yr Degree|3000|Good|3|0|0| +17191|M|M|4 yr Degree|3000|Good|3|0|0| +17192|F|M|4 yr Degree|3000|Good|3|0|0| +17193|M|S|4 yr Degree|3000|Good|3|0|0| +17194|F|S|4 yr Degree|3000|Good|3|0|0| +17195|M|D|4 yr Degree|3000|Good|3|0|0| +17196|F|D|4 yr Degree|3000|Good|3|0|0| +17197|M|W|4 yr Degree|3000|Good|3|0|0| +17198|F|W|4 yr Degree|3000|Good|3|0|0| +17199|M|U|4 yr Degree|3000|Good|3|0|0| +17200|F|U|4 yr Degree|3000|Good|3|0|0| +17201|M|M|Advanced Degree|3000|Good|3|0|0| +17202|F|M|Advanced Degree|3000|Good|3|0|0| +17203|M|S|Advanced Degree|3000|Good|3|0|0| +17204|F|S|Advanced Degree|3000|Good|3|0|0| +17205|M|D|Advanced Degree|3000|Good|3|0|0| +17206|F|D|Advanced Degree|3000|Good|3|0|0| +17207|M|W|Advanced Degree|3000|Good|3|0|0| +17208|F|W|Advanced Degree|3000|Good|3|0|0| +17209|M|U|Advanced Degree|3000|Good|3|0|0| +17210|F|U|Advanced Degree|3000|Good|3|0|0| +17211|M|M|Unknown|3000|Good|3|0|0| +17212|F|M|Unknown|3000|Good|3|0|0| +17213|M|S|Unknown|3000|Good|3|0|0| +17214|F|S|Unknown|3000|Good|3|0|0| +17215|M|D|Unknown|3000|Good|3|0|0| +17216|F|D|Unknown|3000|Good|3|0|0| +17217|M|W|Unknown|3000|Good|3|0|0| +17218|F|W|Unknown|3000|Good|3|0|0| +17219|M|U|Unknown|3000|Good|3|0|0| +17220|F|U|Unknown|3000|Good|3|0|0| +17221|M|M|Primary|3500|Good|3|0|0| +17222|F|M|Primary|3500|Good|3|0|0| +17223|M|S|Primary|3500|Good|3|0|0| +17224|F|S|Primary|3500|Good|3|0|0| +17225|M|D|Primary|3500|Good|3|0|0| +17226|F|D|Primary|3500|Good|3|0|0| +17227|M|W|Primary|3500|Good|3|0|0| +17228|F|W|Primary|3500|Good|3|0|0| +17229|M|U|Primary|3500|Good|3|0|0| +17230|F|U|Primary|3500|Good|3|0|0| +17231|M|M|Secondary|3500|Good|3|0|0| +17232|F|M|Secondary|3500|Good|3|0|0| +17233|M|S|Secondary|3500|Good|3|0|0| +17234|F|S|Secondary|3500|Good|3|0|0| +17235|M|D|Secondary|3500|Good|3|0|0| +17236|F|D|Secondary|3500|Good|3|0|0| +17237|M|W|Secondary|3500|Good|3|0|0| +17238|F|W|Secondary|3500|Good|3|0|0| +17239|M|U|Secondary|3500|Good|3|0|0| +17240|F|U|Secondary|3500|Good|3|0|0| +17241|M|M|College|3500|Good|3|0|0| +17242|F|M|College|3500|Good|3|0|0| +17243|M|S|College|3500|Good|3|0|0| +17244|F|S|College|3500|Good|3|0|0| +17245|M|D|College|3500|Good|3|0|0| +17246|F|D|College|3500|Good|3|0|0| +17247|M|W|College|3500|Good|3|0|0| +17248|F|W|College|3500|Good|3|0|0| +17249|M|U|College|3500|Good|3|0|0| +17250|F|U|College|3500|Good|3|0|0| +17251|M|M|2 yr Degree|3500|Good|3|0|0| +17252|F|M|2 yr Degree|3500|Good|3|0|0| +17253|M|S|2 yr Degree|3500|Good|3|0|0| +17254|F|S|2 yr Degree|3500|Good|3|0|0| +17255|M|D|2 yr Degree|3500|Good|3|0|0| +17256|F|D|2 yr Degree|3500|Good|3|0|0| +17257|M|W|2 yr Degree|3500|Good|3|0|0| +17258|F|W|2 yr Degree|3500|Good|3|0|0| +17259|M|U|2 yr Degree|3500|Good|3|0|0| +17260|F|U|2 yr Degree|3500|Good|3|0|0| +17261|M|M|4 yr Degree|3500|Good|3|0|0| +17262|F|M|4 yr Degree|3500|Good|3|0|0| +17263|M|S|4 yr Degree|3500|Good|3|0|0| +17264|F|S|4 yr Degree|3500|Good|3|0|0| +17265|M|D|4 yr Degree|3500|Good|3|0|0| +17266|F|D|4 yr Degree|3500|Good|3|0|0| +17267|M|W|4 yr Degree|3500|Good|3|0|0| +17268|F|W|4 yr Degree|3500|Good|3|0|0| +17269|M|U|4 yr Degree|3500|Good|3|0|0| +17270|F|U|4 yr Degree|3500|Good|3|0|0| +17271|M|M|Advanced Degree|3500|Good|3|0|0| +17272|F|M|Advanced Degree|3500|Good|3|0|0| +17273|M|S|Advanced Degree|3500|Good|3|0|0| +17274|F|S|Advanced Degree|3500|Good|3|0|0| +17275|M|D|Advanced Degree|3500|Good|3|0|0| +17276|F|D|Advanced Degree|3500|Good|3|0|0| +17277|M|W|Advanced Degree|3500|Good|3|0|0| +17278|F|W|Advanced Degree|3500|Good|3|0|0| +17279|M|U|Advanced Degree|3500|Good|3|0|0| +17280|F|U|Advanced Degree|3500|Good|3|0|0| +17281|M|M|Unknown|3500|Good|3|0|0| +17282|F|M|Unknown|3500|Good|3|0|0| +17283|M|S|Unknown|3500|Good|3|0|0| +17284|F|S|Unknown|3500|Good|3|0|0| +17285|M|D|Unknown|3500|Good|3|0|0| +17286|F|D|Unknown|3500|Good|3|0|0| +17287|M|W|Unknown|3500|Good|3|0|0| +17288|F|W|Unknown|3500|Good|3|0|0| +17289|M|U|Unknown|3500|Good|3|0|0| +17290|F|U|Unknown|3500|Good|3|0|0| +17291|M|M|Primary|4000|Good|3|0|0| +17292|F|M|Primary|4000|Good|3|0|0| +17293|M|S|Primary|4000|Good|3|0|0| +17294|F|S|Primary|4000|Good|3|0|0| +17295|M|D|Primary|4000|Good|3|0|0| +17296|F|D|Primary|4000|Good|3|0|0| +17297|M|W|Primary|4000|Good|3|0|0| +17298|F|W|Primary|4000|Good|3|0|0| +17299|M|U|Primary|4000|Good|3|0|0| +17300|F|U|Primary|4000|Good|3|0|0| +17301|M|M|Secondary|4000|Good|3|0|0| +17302|F|M|Secondary|4000|Good|3|0|0| +17303|M|S|Secondary|4000|Good|3|0|0| +17304|F|S|Secondary|4000|Good|3|0|0| +17305|M|D|Secondary|4000|Good|3|0|0| +17306|F|D|Secondary|4000|Good|3|0|0| +17307|M|W|Secondary|4000|Good|3|0|0| +17308|F|W|Secondary|4000|Good|3|0|0| +17309|M|U|Secondary|4000|Good|3|0|0| +17310|F|U|Secondary|4000|Good|3|0|0| +17311|M|M|College|4000|Good|3|0|0| +17312|F|M|College|4000|Good|3|0|0| +17313|M|S|College|4000|Good|3|0|0| +17314|F|S|College|4000|Good|3|0|0| +17315|M|D|College|4000|Good|3|0|0| +17316|F|D|College|4000|Good|3|0|0| +17317|M|W|College|4000|Good|3|0|0| +17318|F|W|College|4000|Good|3|0|0| +17319|M|U|College|4000|Good|3|0|0| +17320|F|U|College|4000|Good|3|0|0| +17321|M|M|2 yr Degree|4000|Good|3|0|0| +17322|F|M|2 yr Degree|4000|Good|3|0|0| +17323|M|S|2 yr Degree|4000|Good|3|0|0| +17324|F|S|2 yr Degree|4000|Good|3|0|0| +17325|M|D|2 yr Degree|4000|Good|3|0|0| +17326|F|D|2 yr Degree|4000|Good|3|0|0| +17327|M|W|2 yr Degree|4000|Good|3|0|0| +17328|F|W|2 yr Degree|4000|Good|3|0|0| +17329|M|U|2 yr Degree|4000|Good|3|0|0| +17330|F|U|2 yr Degree|4000|Good|3|0|0| +17331|M|M|4 yr Degree|4000|Good|3|0|0| +17332|F|M|4 yr Degree|4000|Good|3|0|0| +17333|M|S|4 yr Degree|4000|Good|3|0|0| +17334|F|S|4 yr Degree|4000|Good|3|0|0| +17335|M|D|4 yr Degree|4000|Good|3|0|0| +17336|F|D|4 yr Degree|4000|Good|3|0|0| +17337|M|W|4 yr Degree|4000|Good|3|0|0| +17338|F|W|4 yr Degree|4000|Good|3|0|0| +17339|M|U|4 yr Degree|4000|Good|3|0|0| +17340|F|U|4 yr Degree|4000|Good|3|0|0| +17341|M|M|Advanced Degree|4000|Good|3|0|0| +17342|F|M|Advanced Degree|4000|Good|3|0|0| +17343|M|S|Advanced Degree|4000|Good|3|0|0| +17344|F|S|Advanced Degree|4000|Good|3|0|0| +17345|M|D|Advanced Degree|4000|Good|3|0|0| +17346|F|D|Advanced Degree|4000|Good|3|0|0| +17347|M|W|Advanced Degree|4000|Good|3|0|0| +17348|F|W|Advanced Degree|4000|Good|3|0|0| +17349|M|U|Advanced Degree|4000|Good|3|0|0| +17350|F|U|Advanced Degree|4000|Good|3|0|0| +17351|M|M|Unknown|4000|Good|3|0|0| +17352|F|M|Unknown|4000|Good|3|0|0| +17353|M|S|Unknown|4000|Good|3|0|0| +17354|F|S|Unknown|4000|Good|3|0|0| +17355|M|D|Unknown|4000|Good|3|0|0| +17356|F|D|Unknown|4000|Good|3|0|0| +17357|M|W|Unknown|4000|Good|3|0|0| +17358|F|W|Unknown|4000|Good|3|0|0| +17359|M|U|Unknown|4000|Good|3|0|0| +17360|F|U|Unknown|4000|Good|3|0|0| +17361|M|M|Primary|4500|Good|3|0|0| +17362|F|M|Primary|4500|Good|3|0|0| +17363|M|S|Primary|4500|Good|3|0|0| +17364|F|S|Primary|4500|Good|3|0|0| +17365|M|D|Primary|4500|Good|3|0|0| +17366|F|D|Primary|4500|Good|3|0|0| +17367|M|W|Primary|4500|Good|3|0|0| +17368|F|W|Primary|4500|Good|3|0|0| +17369|M|U|Primary|4500|Good|3|0|0| +17370|F|U|Primary|4500|Good|3|0|0| +17371|M|M|Secondary|4500|Good|3|0|0| +17372|F|M|Secondary|4500|Good|3|0|0| +17373|M|S|Secondary|4500|Good|3|0|0| +17374|F|S|Secondary|4500|Good|3|0|0| +17375|M|D|Secondary|4500|Good|3|0|0| +17376|F|D|Secondary|4500|Good|3|0|0| +17377|M|W|Secondary|4500|Good|3|0|0| +17378|F|W|Secondary|4500|Good|3|0|0| +17379|M|U|Secondary|4500|Good|3|0|0| +17380|F|U|Secondary|4500|Good|3|0|0| +17381|M|M|College|4500|Good|3|0|0| +17382|F|M|College|4500|Good|3|0|0| +17383|M|S|College|4500|Good|3|0|0| +17384|F|S|College|4500|Good|3|0|0| +17385|M|D|College|4500|Good|3|0|0| +17386|F|D|College|4500|Good|3|0|0| +17387|M|W|College|4500|Good|3|0|0| +17388|F|W|College|4500|Good|3|0|0| +17389|M|U|College|4500|Good|3|0|0| +17390|F|U|College|4500|Good|3|0|0| +17391|M|M|2 yr Degree|4500|Good|3|0|0| +17392|F|M|2 yr Degree|4500|Good|3|0|0| +17393|M|S|2 yr Degree|4500|Good|3|0|0| +17394|F|S|2 yr Degree|4500|Good|3|0|0| +17395|M|D|2 yr Degree|4500|Good|3|0|0| +17396|F|D|2 yr Degree|4500|Good|3|0|0| +17397|M|W|2 yr Degree|4500|Good|3|0|0| +17398|F|W|2 yr Degree|4500|Good|3|0|0| +17399|M|U|2 yr Degree|4500|Good|3|0|0| +17400|F|U|2 yr Degree|4500|Good|3|0|0| +17401|M|M|4 yr Degree|4500|Good|3|0|0| +17402|F|M|4 yr Degree|4500|Good|3|0|0| +17403|M|S|4 yr Degree|4500|Good|3|0|0| +17404|F|S|4 yr Degree|4500|Good|3|0|0| +17405|M|D|4 yr Degree|4500|Good|3|0|0| +17406|F|D|4 yr Degree|4500|Good|3|0|0| +17407|M|W|4 yr Degree|4500|Good|3|0|0| +17408|F|W|4 yr Degree|4500|Good|3|0|0| +17409|M|U|4 yr Degree|4500|Good|3|0|0| +17410|F|U|4 yr Degree|4500|Good|3|0|0| +17411|M|M|Advanced Degree|4500|Good|3|0|0| +17412|F|M|Advanced Degree|4500|Good|3|0|0| +17413|M|S|Advanced Degree|4500|Good|3|0|0| +17414|F|S|Advanced Degree|4500|Good|3|0|0| +17415|M|D|Advanced Degree|4500|Good|3|0|0| +17416|F|D|Advanced Degree|4500|Good|3|0|0| +17417|M|W|Advanced Degree|4500|Good|3|0|0| +17418|F|W|Advanced Degree|4500|Good|3|0|0| +17419|M|U|Advanced Degree|4500|Good|3|0|0| +17420|F|U|Advanced Degree|4500|Good|3|0|0| +17421|M|M|Unknown|4500|Good|3|0|0| +17422|F|M|Unknown|4500|Good|3|0|0| +17423|M|S|Unknown|4500|Good|3|0|0| +17424|F|S|Unknown|4500|Good|3|0|0| +17425|M|D|Unknown|4500|Good|3|0|0| +17426|F|D|Unknown|4500|Good|3|0|0| +17427|M|W|Unknown|4500|Good|3|0|0| +17428|F|W|Unknown|4500|Good|3|0|0| +17429|M|U|Unknown|4500|Good|3|0|0| +17430|F|U|Unknown|4500|Good|3|0|0| +17431|M|M|Primary|5000|Good|3|0|0| +17432|F|M|Primary|5000|Good|3|0|0| +17433|M|S|Primary|5000|Good|3|0|0| +17434|F|S|Primary|5000|Good|3|0|0| +17435|M|D|Primary|5000|Good|3|0|0| +17436|F|D|Primary|5000|Good|3|0|0| +17437|M|W|Primary|5000|Good|3|0|0| +17438|F|W|Primary|5000|Good|3|0|0| +17439|M|U|Primary|5000|Good|3|0|0| +17440|F|U|Primary|5000|Good|3|0|0| +17441|M|M|Secondary|5000|Good|3|0|0| +17442|F|M|Secondary|5000|Good|3|0|0| +17443|M|S|Secondary|5000|Good|3|0|0| +17444|F|S|Secondary|5000|Good|3|0|0| +17445|M|D|Secondary|5000|Good|3|0|0| +17446|F|D|Secondary|5000|Good|3|0|0| +17447|M|W|Secondary|5000|Good|3|0|0| +17448|F|W|Secondary|5000|Good|3|0|0| +17449|M|U|Secondary|5000|Good|3|0|0| +17450|F|U|Secondary|5000|Good|3|0|0| +17451|M|M|College|5000|Good|3|0|0| +17452|F|M|College|5000|Good|3|0|0| +17453|M|S|College|5000|Good|3|0|0| +17454|F|S|College|5000|Good|3|0|0| +17455|M|D|College|5000|Good|3|0|0| +17456|F|D|College|5000|Good|3|0|0| +17457|M|W|College|5000|Good|3|0|0| +17458|F|W|College|5000|Good|3|0|0| +17459|M|U|College|5000|Good|3|0|0| +17460|F|U|College|5000|Good|3|0|0| +17461|M|M|2 yr Degree|5000|Good|3|0|0| +17462|F|M|2 yr Degree|5000|Good|3|0|0| +17463|M|S|2 yr Degree|5000|Good|3|0|0| +17464|F|S|2 yr Degree|5000|Good|3|0|0| +17465|M|D|2 yr Degree|5000|Good|3|0|0| +17466|F|D|2 yr Degree|5000|Good|3|0|0| +17467|M|W|2 yr Degree|5000|Good|3|0|0| +17468|F|W|2 yr Degree|5000|Good|3|0|0| +17469|M|U|2 yr Degree|5000|Good|3|0|0| +17470|F|U|2 yr Degree|5000|Good|3|0|0| +17471|M|M|4 yr Degree|5000|Good|3|0|0| +17472|F|M|4 yr Degree|5000|Good|3|0|0| +17473|M|S|4 yr Degree|5000|Good|3|0|0| +17474|F|S|4 yr Degree|5000|Good|3|0|0| +17475|M|D|4 yr Degree|5000|Good|3|0|0| +17476|F|D|4 yr Degree|5000|Good|3|0|0| +17477|M|W|4 yr Degree|5000|Good|3|0|0| +17478|F|W|4 yr Degree|5000|Good|3|0|0| +17479|M|U|4 yr Degree|5000|Good|3|0|0| +17480|F|U|4 yr Degree|5000|Good|3|0|0| +17481|M|M|Advanced Degree|5000|Good|3|0|0| +17482|F|M|Advanced Degree|5000|Good|3|0|0| +17483|M|S|Advanced Degree|5000|Good|3|0|0| +17484|F|S|Advanced Degree|5000|Good|3|0|0| +17485|M|D|Advanced Degree|5000|Good|3|0|0| +17486|F|D|Advanced Degree|5000|Good|3|0|0| +17487|M|W|Advanced Degree|5000|Good|3|0|0| +17488|F|W|Advanced Degree|5000|Good|3|0|0| +17489|M|U|Advanced Degree|5000|Good|3|0|0| +17490|F|U|Advanced Degree|5000|Good|3|0|0| +17491|M|M|Unknown|5000|Good|3|0|0| +17492|F|M|Unknown|5000|Good|3|0|0| +17493|M|S|Unknown|5000|Good|3|0|0| +17494|F|S|Unknown|5000|Good|3|0|0| +17495|M|D|Unknown|5000|Good|3|0|0| +17496|F|D|Unknown|5000|Good|3|0|0| +17497|M|W|Unknown|5000|Good|3|0|0| +17498|F|W|Unknown|5000|Good|3|0|0| +17499|M|U|Unknown|5000|Good|3|0|0| +17500|F|U|Unknown|5000|Good|3|0|0| +17501|M|M|Primary|5500|Good|3|0|0| +17502|F|M|Primary|5500|Good|3|0|0| +17503|M|S|Primary|5500|Good|3|0|0| +17504|F|S|Primary|5500|Good|3|0|0| +17505|M|D|Primary|5500|Good|3|0|0| +17506|F|D|Primary|5500|Good|3|0|0| +17507|M|W|Primary|5500|Good|3|0|0| +17508|F|W|Primary|5500|Good|3|0|0| +17509|M|U|Primary|5500|Good|3|0|0| +17510|F|U|Primary|5500|Good|3|0|0| +17511|M|M|Secondary|5500|Good|3|0|0| +17512|F|M|Secondary|5500|Good|3|0|0| +17513|M|S|Secondary|5500|Good|3|0|0| +17514|F|S|Secondary|5500|Good|3|0|0| +17515|M|D|Secondary|5500|Good|3|0|0| +17516|F|D|Secondary|5500|Good|3|0|0| +17517|M|W|Secondary|5500|Good|3|0|0| +17518|F|W|Secondary|5500|Good|3|0|0| +17519|M|U|Secondary|5500|Good|3|0|0| +17520|F|U|Secondary|5500|Good|3|0|0| +17521|M|M|College|5500|Good|3|0|0| +17522|F|M|College|5500|Good|3|0|0| +17523|M|S|College|5500|Good|3|0|0| +17524|F|S|College|5500|Good|3|0|0| +17525|M|D|College|5500|Good|3|0|0| +17526|F|D|College|5500|Good|3|0|0| +17527|M|W|College|5500|Good|3|0|0| +17528|F|W|College|5500|Good|3|0|0| +17529|M|U|College|5500|Good|3|0|0| +17530|F|U|College|5500|Good|3|0|0| +17531|M|M|2 yr Degree|5500|Good|3|0|0| +17532|F|M|2 yr Degree|5500|Good|3|0|0| +17533|M|S|2 yr Degree|5500|Good|3|0|0| +17534|F|S|2 yr Degree|5500|Good|3|0|0| +17535|M|D|2 yr Degree|5500|Good|3|0|0| +17536|F|D|2 yr Degree|5500|Good|3|0|0| +17537|M|W|2 yr Degree|5500|Good|3|0|0| +17538|F|W|2 yr Degree|5500|Good|3|0|0| +17539|M|U|2 yr Degree|5500|Good|3|0|0| +17540|F|U|2 yr Degree|5500|Good|3|0|0| +17541|M|M|4 yr Degree|5500|Good|3|0|0| +17542|F|M|4 yr Degree|5500|Good|3|0|0| +17543|M|S|4 yr Degree|5500|Good|3|0|0| +17544|F|S|4 yr Degree|5500|Good|3|0|0| +17545|M|D|4 yr Degree|5500|Good|3|0|0| +17546|F|D|4 yr Degree|5500|Good|3|0|0| +17547|M|W|4 yr Degree|5500|Good|3|0|0| +17548|F|W|4 yr Degree|5500|Good|3|0|0| +17549|M|U|4 yr Degree|5500|Good|3|0|0| +17550|F|U|4 yr Degree|5500|Good|3|0|0| +17551|M|M|Advanced Degree|5500|Good|3|0|0| +17552|F|M|Advanced Degree|5500|Good|3|0|0| +17553|M|S|Advanced Degree|5500|Good|3|0|0| +17554|F|S|Advanced Degree|5500|Good|3|0|0| +17555|M|D|Advanced Degree|5500|Good|3|0|0| +17556|F|D|Advanced Degree|5500|Good|3|0|0| +17557|M|W|Advanced Degree|5500|Good|3|0|0| +17558|F|W|Advanced Degree|5500|Good|3|0|0| +17559|M|U|Advanced Degree|5500|Good|3|0|0| +17560|F|U|Advanced Degree|5500|Good|3|0|0| +17561|M|M|Unknown|5500|Good|3|0|0| +17562|F|M|Unknown|5500|Good|3|0|0| +17563|M|S|Unknown|5500|Good|3|0|0| +17564|F|S|Unknown|5500|Good|3|0|0| +17565|M|D|Unknown|5500|Good|3|0|0| +17566|F|D|Unknown|5500|Good|3|0|0| +17567|M|W|Unknown|5500|Good|3|0|0| +17568|F|W|Unknown|5500|Good|3|0|0| +17569|M|U|Unknown|5500|Good|3|0|0| +17570|F|U|Unknown|5500|Good|3|0|0| +17571|M|M|Primary|6000|Good|3|0|0| +17572|F|M|Primary|6000|Good|3|0|0| +17573|M|S|Primary|6000|Good|3|0|0| +17574|F|S|Primary|6000|Good|3|0|0| +17575|M|D|Primary|6000|Good|3|0|0| +17576|F|D|Primary|6000|Good|3|0|0| +17577|M|W|Primary|6000|Good|3|0|0| +17578|F|W|Primary|6000|Good|3|0|0| +17579|M|U|Primary|6000|Good|3|0|0| +17580|F|U|Primary|6000|Good|3|0|0| +17581|M|M|Secondary|6000|Good|3|0|0| +17582|F|M|Secondary|6000|Good|3|0|0| +17583|M|S|Secondary|6000|Good|3|0|0| +17584|F|S|Secondary|6000|Good|3|0|0| +17585|M|D|Secondary|6000|Good|3|0|0| +17586|F|D|Secondary|6000|Good|3|0|0| +17587|M|W|Secondary|6000|Good|3|0|0| +17588|F|W|Secondary|6000|Good|3|0|0| +17589|M|U|Secondary|6000|Good|3|0|0| +17590|F|U|Secondary|6000|Good|3|0|0| +17591|M|M|College|6000|Good|3|0|0| +17592|F|M|College|6000|Good|3|0|0| +17593|M|S|College|6000|Good|3|0|0| +17594|F|S|College|6000|Good|3|0|0| +17595|M|D|College|6000|Good|3|0|0| +17596|F|D|College|6000|Good|3|0|0| +17597|M|W|College|6000|Good|3|0|0| +17598|F|W|College|6000|Good|3|0|0| +17599|M|U|College|6000|Good|3|0|0| +17600|F|U|College|6000|Good|3|0|0| +17601|M|M|2 yr Degree|6000|Good|3|0|0| +17602|F|M|2 yr Degree|6000|Good|3|0|0| +17603|M|S|2 yr Degree|6000|Good|3|0|0| +17604|F|S|2 yr Degree|6000|Good|3|0|0| +17605|M|D|2 yr Degree|6000|Good|3|0|0| +17606|F|D|2 yr Degree|6000|Good|3|0|0| +17607|M|W|2 yr Degree|6000|Good|3|0|0| +17608|F|W|2 yr Degree|6000|Good|3|0|0| +17609|M|U|2 yr Degree|6000|Good|3|0|0| +17610|F|U|2 yr Degree|6000|Good|3|0|0| +17611|M|M|4 yr Degree|6000|Good|3|0|0| +17612|F|M|4 yr Degree|6000|Good|3|0|0| +17613|M|S|4 yr Degree|6000|Good|3|0|0| +17614|F|S|4 yr Degree|6000|Good|3|0|0| +17615|M|D|4 yr Degree|6000|Good|3|0|0| +17616|F|D|4 yr Degree|6000|Good|3|0|0| +17617|M|W|4 yr Degree|6000|Good|3|0|0| +17618|F|W|4 yr Degree|6000|Good|3|0|0| +17619|M|U|4 yr Degree|6000|Good|3|0|0| +17620|F|U|4 yr Degree|6000|Good|3|0|0| +17621|M|M|Advanced Degree|6000|Good|3|0|0| +17622|F|M|Advanced Degree|6000|Good|3|0|0| +17623|M|S|Advanced Degree|6000|Good|3|0|0| +17624|F|S|Advanced Degree|6000|Good|3|0|0| +17625|M|D|Advanced Degree|6000|Good|3|0|0| +17626|F|D|Advanced Degree|6000|Good|3|0|0| +17627|M|W|Advanced Degree|6000|Good|3|0|0| +17628|F|W|Advanced Degree|6000|Good|3|0|0| +17629|M|U|Advanced Degree|6000|Good|3|0|0| +17630|F|U|Advanced Degree|6000|Good|3|0|0| +17631|M|M|Unknown|6000|Good|3|0|0| +17632|F|M|Unknown|6000|Good|3|0|0| +17633|M|S|Unknown|6000|Good|3|0|0| +17634|F|S|Unknown|6000|Good|3|0|0| +17635|M|D|Unknown|6000|Good|3|0|0| +17636|F|D|Unknown|6000|Good|3|0|0| +17637|M|W|Unknown|6000|Good|3|0|0| +17638|F|W|Unknown|6000|Good|3|0|0| +17639|M|U|Unknown|6000|Good|3|0|0| +17640|F|U|Unknown|6000|Good|3|0|0| +17641|M|M|Primary|6500|Good|3|0|0| +17642|F|M|Primary|6500|Good|3|0|0| +17643|M|S|Primary|6500|Good|3|0|0| +17644|F|S|Primary|6500|Good|3|0|0| +17645|M|D|Primary|6500|Good|3|0|0| +17646|F|D|Primary|6500|Good|3|0|0| +17647|M|W|Primary|6500|Good|3|0|0| +17648|F|W|Primary|6500|Good|3|0|0| +17649|M|U|Primary|6500|Good|3|0|0| +17650|F|U|Primary|6500|Good|3|0|0| +17651|M|M|Secondary|6500|Good|3|0|0| +17652|F|M|Secondary|6500|Good|3|0|0| +17653|M|S|Secondary|6500|Good|3|0|0| +17654|F|S|Secondary|6500|Good|3|0|0| +17655|M|D|Secondary|6500|Good|3|0|0| +17656|F|D|Secondary|6500|Good|3|0|0| +17657|M|W|Secondary|6500|Good|3|0|0| +17658|F|W|Secondary|6500|Good|3|0|0| +17659|M|U|Secondary|6500|Good|3|0|0| +17660|F|U|Secondary|6500|Good|3|0|0| +17661|M|M|College|6500|Good|3|0|0| +17662|F|M|College|6500|Good|3|0|0| +17663|M|S|College|6500|Good|3|0|0| +17664|F|S|College|6500|Good|3|0|0| +17665|M|D|College|6500|Good|3|0|0| +17666|F|D|College|6500|Good|3|0|0| +17667|M|W|College|6500|Good|3|0|0| +17668|F|W|College|6500|Good|3|0|0| +17669|M|U|College|6500|Good|3|0|0| +17670|F|U|College|6500|Good|3|0|0| +17671|M|M|2 yr Degree|6500|Good|3|0|0| +17672|F|M|2 yr Degree|6500|Good|3|0|0| +17673|M|S|2 yr Degree|6500|Good|3|0|0| +17674|F|S|2 yr Degree|6500|Good|3|0|0| +17675|M|D|2 yr Degree|6500|Good|3|0|0| +17676|F|D|2 yr Degree|6500|Good|3|0|0| +17677|M|W|2 yr Degree|6500|Good|3|0|0| +17678|F|W|2 yr Degree|6500|Good|3|0|0| +17679|M|U|2 yr Degree|6500|Good|3|0|0| +17680|F|U|2 yr Degree|6500|Good|3|0|0| +17681|M|M|4 yr Degree|6500|Good|3|0|0| +17682|F|M|4 yr Degree|6500|Good|3|0|0| +17683|M|S|4 yr Degree|6500|Good|3|0|0| +17684|F|S|4 yr Degree|6500|Good|3|0|0| +17685|M|D|4 yr Degree|6500|Good|3|0|0| +17686|F|D|4 yr Degree|6500|Good|3|0|0| +17687|M|W|4 yr Degree|6500|Good|3|0|0| +17688|F|W|4 yr Degree|6500|Good|3|0|0| +17689|M|U|4 yr Degree|6500|Good|3|0|0| +17690|F|U|4 yr Degree|6500|Good|3|0|0| +17691|M|M|Advanced Degree|6500|Good|3|0|0| +17692|F|M|Advanced Degree|6500|Good|3|0|0| +17693|M|S|Advanced Degree|6500|Good|3|0|0| +17694|F|S|Advanced Degree|6500|Good|3|0|0| +17695|M|D|Advanced Degree|6500|Good|3|0|0| +17696|F|D|Advanced Degree|6500|Good|3|0|0| +17697|M|W|Advanced Degree|6500|Good|3|0|0| +17698|F|W|Advanced Degree|6500|Good|3|0|0| +17699|M|U|Advanced Degree|6500|Good|3|0|0| +17700|F|U|Advanced Degree|6500|Good|3|0|0| +17701|M|M|Unknown|6500|Good|3|0|0| +17702|F|M|Unknown|6500|Good|3|0|0| +17703|M|S|Unknown|6500|Good|3|0|0| +17704|F|S|Unknown|6500|Good|3|0|0| +17705|M|D|Unknown|6500|Good|3|0|0| +17706|F|D|Unknown|6500|Good|3|0|0| +17707|M|W|Unknown|6500|Good|3|0|0| +17708|F|W|Unknown|6500|Good|3|0|0| +17709|M|U|Unknown|6500|Good|3|0|0| +17710|F|U|Unknown|6500|Good|3|0|0| +17711|M|M|Primary|7000|Good|3|0|0| +17712|F|M|Primary|7000|Good|3|0|0| +17713|M|S|Primary|7000|Good|3|0|0| +17714|F|S|Primary|7000|Good|3|0|0| +17715|M|D|Primary|7000|Good|3|0|0| +17716|F|D|Primary|7000|Good|3|0|0| +17717|M|W|Primary|7000|Good|3|0|0| +17718|F|W|Primary|7000|Good|3|0|0| +17719|M|U|Primary|7000|Good|3|0|0| +17720|F|U|Primary|7000|Good|3|0|0| +17721|M|M|Secondary|7000|Good|3|0|0| +17722|F|M|Secondary|7000|Good|3|0|0| +17723|M|S|Secondary|7000|Good|3|0|0| +17724|F|S|Secondary|7000|Good|3|0|0| +17725|M|D|Secondary|7000|Good|3|0|0| +17726|F|D|Secondary|7000|Good|3|0|0| +17727|M|W|Secondary|7000|Good|3|0|0| +17728|F|W|Secondary|7000|Good|3|0|0| +17729|M|U|Secondary|7000|Good|3|0|0| +17730|F|U|Secondary|7000|Good|3|0|0| +17731|M|M|College|7000|Good|3|0|0| +17732|F|M|College|7000|Good|3|0|0| +17733|M|S|College|7000|Good|3|0|0| +17734|F|S|College|7000|Good|3|0|0| +17735|M|D|College|7000|Good|3|0|0| +17736|F|D|College|7000|Good|3|0|0| +17737|M|W|College|7000|Good|3|0|0| +17738|F|W|College|7000|Good|3|0|0| +17739|M|U|College|7000|Good|3|0|0| +17740|F|U|College|7000|Good|3|0|0| +17741|M|M|2 yr Degree|7000|Good|3|0|0| +17742|F|M|2 yr Degree|7000|Good|3|0|0| +17743|M|S|2 yr Degree|7000|Good|3|0|0| +17744|F|S|2 yr Degree|7000|Good|3|0|0| +17745|M|D|2 yr Degree|7000|Good|3|0|0| +17746|F|D|2 yr Degree|7000|Good|3|0|0| +17747|M|W|2 yr Degree|7000|Good|3|0|0| +17748|F|W|2 yr Degree|7000|Good|3|0|0| +17749|M|U|2 yr Degree|7000|Good|3|0|0| +17750|F|U|2 yr Degree|7000|Good|3|0|0| +17751|M|M|4 yr Degree|7000|Good|3|0|0| +17752|F|M|4 yr Degree|7000|Good|3|0|0| +17753|M|S|4 yr Degree|7000|Good|3|0|0| +17754|F|S|4 yr Degree|7000|Good|3|0|0| +17755|M|D|4 yr Degree|7000|Good|3|0|0| +17756|F|D|4 yr Degree|7000|Good|3|0|0| +17757|M|W|4 yr Degree|7000|Good|3|0|0| +17758|F|W|4 yr Degree|7000|Good|3|0|0| +17759|M|U|4 yr Degree|7000|Good|3|0|0| +17760|F|U|4 yr Degree|7000|Good|3|0|0| +17761|M|M|Advanced Degree|7000|Good|3|0|0| +17762|F|M|Advanced Degree|7000|Good|3|0|0| +17763|M|S|Advanced Degree|7000|Good|3|0|0| +17764|F|S|Advanced Degree|7000|Good|3|0|0| +17765|M|D|Advanced Degree|7000|Good|3|0|0| +17766|F|D|Advanced Degree|7000|Good|3|0|0| +17767|M|W|Advanced Degree|7000|Good|3|0|0| +17768|F|W|Advanced Degree|7000|Good|3|0|0| +17769|M|U|Advanced Degree|7000|Good|3|0|0| +17770|F|U|Advanced Degree|7000|Good|3|0|0| +17771|M|M|Unknown|7000|Good|3|0|0| +17772|F|M|Unknown|7000|Good|3|0|0| +17773|M|S|Unknown|7000|Good|3|0|0| +17774|F|S|Unknown|7000|Good|3|0|0| +17775|M|D|Unknown|7000|Good|3|0|0| +17776|F|D|Unknown|7000|Good|3|0|0| +17777|M|W|Unknown|7000|Good|3|0|0| +17778|F|W|Unknown|7000|Good|3|0|0| +17779|M|U|Unknown|7000|Good|3|0|0| +17780|F|U|Unknown|7000|Good|3|0|0| +17781|M|M|Primary|7500|Good|3|0|0| +17782|F|M|Primary|7500|Good|3|0|0| +17783|M|S|Primary|7500|Good|3|0|0| +17784|F|S|Primary|7500|Good|3|0|0| +17785|M|D|Primary|7500|Good|3|0|0| +17786|F|D|Primary|7500|Good|3|0|0| +17787|M|W|Primary|7500|Good|3|0|0| +17788|F|W|Primary|7500|Good|3|0|0| +17789|M|U|Primary|7500|Good|3|0|0| +17790|F|U|Primary|7500|Good|3|0|0| +17791|M|M|Secondary|7500|Good|3|0|0| +17792|F|M|Secondary|7500|Good|3|0|0| +17793|M|S|Secondary|7500|Good|3|0|0| +17794|F|S|Secondary|7500|Good|3|0|0| +17795|M|D|Secondary|7500|Good|3|0|0| +17796|F|D|Secondary|7500|Good|3|0|0| +17797|M|W|Secondary|7500|Good|3|0|0| +17798|F|W|Secondary|7500|Good|3|0|0| +17799|M|U|Secondary|7500|Good|3|0|0| +17800|F|U|Secondary|7500|Good|3|0|0| +17801|M|M|College|7500|Good|3|0|0| +17802|F|M|College|7500|Good|3|0|0| +17803|M|S|College|7500|Good|3|0|0| +17804|F|S|College|7500|Good|3|0|0| +17805|M|D|College|7500|Good|3|0|0| +17806|F|D|College|7500|Good|3|0|0| +17807|M|W|College|7500|Good|3|0|0| +17808|F|W|College|7500|Good|3|0|0| +17809|M|U|College|7500|Good|3|0|0| +17810|F|U|College|7500|Good|3|0|0| +17811|M|M|2 yr Degree|7500|Good|3|0|0| +17812|F|M|2 yr Degree|7500|Good|3|0|0| +17813|M|S|2 yr Degree|7500|Good|3|0|0| +17814|F|S|2 yr Degree|7500|Good|3|0|0| +17815|M|D|2 yr Degree|7500|Good|3|0|0| +17816|F|D|2 yr Degree|7500|Good|3|0|0| +17817|M|W|2 yr Degree|7500|Good|3|0|0| +17818|F|W|2 yr Degree|7500|Good|3|0|0| +17819|M|U|2 yr Degree|7500|Good|3|0|0| +17820|F|U|2 yr Degree|7500|Good|3|0|0| +17821|M|M|4 yr Degree|7500|Good|3|0|0| +17822|F|M|4 yr Degree|7500|Good|3|0|0| +17823|M|S|4 yr Degree|7500|Good|3|0|0| +17824|F|S|4 yr Degree|7500|Good|3|0|0| +17825|M|D|4 yr Degree|7500|Good|3|0|0| +17826|F|D|4 yr Degree|7500|Good|3|0|0| +17827|M|W|4 yr Degree|7500|Good|3|0|0| +17828|F|W|4 yr Degree|7500|Good|3|0|0| +17829|M|U|4 yr Degree|7500|Good|3|0|0| +17830|F|U|4 yr Degree|7500|Good|3|0|0| +17831|M|M|Advanced Degree|7500|Good|3|0|0| +17832|F|M|Advanced Degree|7500|Good|3|0|0| +17833|M|S|Advanced Degree|7500|Good|3|0|0| +17834|F|S|Advanced Degree|7500|Good|3|0|0| +17835|M|D|Advanced Degree|7500|Good|3|0|0| +17836|F|D|Advanced Degree|7500|Good|3|0|0| +17837|M|W|Advanced Degree|7500|Good|3|0|0| +17838|F|W|Advanced Degree|7500|Good|3|0|0| +17839|M|U|Advanced Degree|7500|Good|3|0|0| +17840|F|U|Advanced Degree|7500|Good|3|0|0| +17841|M|M|Unknown|7500|Good|3|0|0| +17842|F|M|Unknown|7500|Good|3|0|0| +17843|M|S|Unknown|7500|Good|3|0|0| +17844|F|S|Unknown|7500|Good|3|0|0| +17845|M|D|Unknown|7500|Good|3|0|0| +17846|F|D|Unknown|7500|Good|3|0|0| +17847|M|W|Unknown|7500|Good|3|0|0| +17848|F|W|Unknown|7500|Good|3|0|0| +17849|M|U|Unknown|7500|Good|3|0|0| +17850|F|U|Unknown|7500|Good|3|0|0| +17851|M|M|Primary|8000|Good|3|0|0| +17852|F|M|Primary|8000|Good|3|0|0| +17853|M|S|Primary|8000|Good|3|0|0| +17854|F|S|Primary|8000|Good|3|0|0| +17855|M|D|Primary|8000|Good|3|0|0| +17856|F|D|Primary|8000|Good|3|0|0| +17857|M|W|Primary|8000|Good|3|0|0| +17858|F|W|Primary|8000|Good|3|0|0| +17859|M|U|Primary|8000|Good|3|0|0| +17860|F|U|Primary|8000|Good|3|0|0| +17861|M|M|Secondary|8000|Good|3|0|0| +17862|F|M|Secondary|8000|Good|3|0|0| +17863|M|S|Secondary|8000|Good|3|0|0| +17864|F|S|Secondary|8000|Good|3|0|0| +17865|M|D|Secondary|8000|Good|3|0|0| +17866|F|D|Secondary|8000|Good|3|0|0| +17867|M|W|Secondary|8000|Good|3|0|0| +17868|F|W|Secondary|8000|Good|3|0|0| +17869|M|U|Secondary|8000|Good|3|0|0| +17870|F|U|Secondary|8000|Good|3|0|0| +17871|M|M|College|8000|Good|3|0|0| +17872|F|M|College|8000|Good|3|0|0| +17873|M|S|College|8000|Good|3|0|0| +17874|F|S|College|8000|Good|3|0|0| +17875|M|D|College|8000|Good|3|0|0| +17876|F|D|College|8000|Good|3|0|0| +17877|M|W|College|8000|Good|3|0|0| +17878|F|W|College|8000|Good|3|0|0| +17879|M|U|College|8000|Good|3|0|0| +17880|F|U|College|8000|Good|3|0|0| +17881|M|M|2 yr Degree|8000|Good|3|0|0| +17882|F|M|2 yr Degree|8000|Good|3|0|0| +17883|M|S|2 yr Degree|8000|Good|3|0|0| +17884|F|S|2 yr Degree|8000|Good|3|0|0| +17885|M|D|2 yr Degree|8000|Good|3|0|0| +17886|F|D|2 yr Degree|8000|Good|3|0|0| +17887|M|W|2 yr Degree|8000|Good|3|0|0| +17888|F|W|2 yr Degree|8000|Good|3|0|0| +17889|M|U|2 yr Degree|8000|Good|3|0|0| +17890|F|U|2 yr Degree|8000|Good|3|0|0| +17891|M|M|4 yr Degree|8000|Good|3|0|0| +17892|F|M|4 yr Degree|8000|Good|3|0|0| +17893|M|S|4 yr Degree|8000|Good|3|0|0| +17894|F|S|4 yr Degree|8000|Good|3|0|0| +17895|M|D|4 yr Degree|8000|Good|3|0|0| +17896|F|D|4 yr Degree|8000|Good|3|0|0| +17897|M|W|4 yr Degree|8000|Good|3|0|0| +17898|F|W|4 yr Degree|8000|Good|3|0|0| +17899|M|U|4 yr Degree|8000|Good|3|0|0| +17900|F|U|4 yr Degree|8000|Good|3|0|0| +17901|M|M|Advanced Degree|8000|Good|3|0|0| +17902|F|M|Advanced Degree|8000|Good|3|0|0| +17903|M|S|Advanced Degree|8000|Good|3|0|0| +17904|F|S|Advanced Degree|8000|Good|3|0|0| +17905|M|D|Advanced Degree|8000|Good|3|0|0| +17906|F|D|Advanced Degree|8000|Good|3|0|0| +17907|M|W|Advanced Degree|8000|Good|3|0|0| +17908|F|W|Advanced Degree|8000|Good|3|0|0| +17909|M|U|Advanced Degree|8000|Good|3|0|0| +17910|F|U|Advanced Degree|8000|Good|3|0|0| +17911|M|M|Unknown|8000|Good|3|0|0| +17912|F|M|Unknown|8000|Good|3|0|0| +17913|M|S|Unknown|8000|Good|3|0|0| +17914|F|S|Unknown|8000|Good|3|0|0| +17915|M|D|Unknown|8000|Good|3|0|0| +17916|F|D|Unknown|8000|Good|3|0|0| +17917|M|W|Unknown|8000|Good|3|0|0| +17918|F|W|Unknown|8000|Good|3|0|0| +17919|M|U|Unknown|8000|Good|3|0|0| +17920|F|U|Unknown|8000|Good|3|0|0| +17921|M|M|Primary|8500|Good|3|0|0| +17922|F|M|Primary|8500|Good|3|0|0| +17923|M|S|Primary|8500|Good|3|0|0| +17924|F|S|Primary|8500|Good|3|0|0| +17925|M|D|Primary|8500|Good|3|0|0| +17926|F|D|Primary|8500|Good|3|0|0| +17927|M|W|Primary|8500|Good|3|0|0| +17928|F|W|Primary|8500|Good|3|0|0| +17929|M|U|Primary|8500|Good|3|0|0| +17930|F|U|Primary|8500|Good|3|0|0| +17931|M|M|Secondary|8500|Good|3|0|0| +17932|F|M|Secondary|8500|Good|3|0|0| +17933|M|S|Secondary|8500|Good|3|0|0| +17934|F|S|Secondary|8500|Good|3|0|0| +17935|M|D|Secondary|8500|Good|3|0|0| +17936|F|D|Secondary|8500|Good|3|0|0| +17937|M|W|Secondary|8500|Good|3|0|0| +17938|F|W|Secondary|8500|Good|3|0|0| +17939|M|U|Secondary|8500|Good|3|0|0| +17940|F|U|Secondary|8500|Good|3|0|0| +17941|M|M|College|8500|Good|3|0|0| +17942|F|M|College|8500|Good|3|0|0| +17943|M|S|College|8500|Good|3|0|0| +17944|F|S|College|8500|Good|3|0|0| +17945|M|D|College|8500|Good|3|0|0| +17946|F|D|College|8500|Good|3|0|0| +17947|M|W|College|8500|Good|3|0|0| +17948|F|W|College|8500|Good|3|0|0| +17949|M|U|College|8500|Good|3|0|0| +17950|F|U|College|8500|Good|3|0|0| +17951|M|M|2 yr Degree|8500|Good|3|0|0| +17952|F|M|2 yr Degree|8500|Good|3|0|0| +17953|M|S|2 yr Degree|8500|Good|3|0|0| +17954|F|S|2 yr Degree|8500|Good|3|0|0| +17955|M|D|2 yr Degree|8500|Good|3|0|0| +17956|F|D|2 yr Degree|8500|Good|3|0|0| +17957|M|W|2 yr Degree|8500|Good|3|0|0| +17958|F|W|2 yr Degree|8500|Good|3|0|0| +17959|M|U|2 yr Degree|8500|Good|3|0|0| +17960|F|U|2 yr Degree|8500|Good|3|0|0| +17961|M|M|4 yr Degree|8500|Good|3|0|0| +17962|F|M|4 yr Degree|8500|Good|3|0|0| +17963|M|S|4 yr Degree|8500|Good|3|0|0| +17964|F|S|4 yr Degree|8500|Good|3|0|0| +17965|M|D|4 yr Degree|8500|Good|3|0|0| +17966|F|D|4 yr Degree|8500|Good|3|0|0| +17967|M|W|4 yr Degree|8500|Good|3|0|0| +17968|F|W|4 yr Degree|8500|Good|3|0|0| +17969|M|U|4 yr Degree|8500|Good|3|0|0| +17970|F|U|4 yr Degree|8500|Good|3|0|0| +17971|M|M|Advanced Degree|8500|Good|3|0|0| +17972|F|M|Advanced Degree|8500|Good|3|0|0| +17973|M|S|Advanced Degree|8500|Good|3|0|0| +17974|F|S|Advanced Degree|8500|Good|3|0|0| +17975|M|D|Advanced Degree|8500|Good|3|0|0| +17976|F|D|Advanced Degree|8500|Good|3|0|0| +17977|M|W|Advanced Degree|8500|Good|3|0|0| +17978|F|W|Advanced Degree|8500|Good|3|0|0| +17979|M|U|Advanced Degree|8500|Good|3|0|0| +17980|F|U|Advanced Degree|8500|Good|3|0|0| +17981|M|M|Unknown|8500|Good|3|0|0| +17982|F|M|Unknown|8500|Good|3|0|0| +17983|M|S|Unknown|8500|Good|3|0|0| +17984|F|S|Unknown|8500|Good|3|0|0| +17985|M|D|Unknown|8500|Good|3|0|0| +17986|F|D|Unknown|8500|Good|3|0|0| +17987|M|W|Unknown|8500|Good|3|0|0| +17988|F|W|Unknown|8500|Good|3|0|0| +17989|M|U|Unknown|8500|Good|3|0|0| +17990|F|U|Unknown|8500|Good|3|0|0| +17991|M|M|Primary|9000|Good|3|0|0| +17992|F|M|Primary|9000|Good|3|0|0| +17993|M|S|Primary|9000|Good|3|0|0| +17994|F|S|Primary|9000|Good|3|0|0| +17995|M|D|Primary|9000|Good|3|0|0| +17996|F|D|Primary|9000|Good|3|0|0| +17997|M|W|Primary|9000|Good|3|0|0| +17998|F|W|Primary|9000|Good|3|0|0| +17999|M|U|Primary|9000|Good|3|0|0| +18000|F|U|Primary|9000|Good|3|0|0| +18001|M|M|Secondary|9000|Good|3|0|0| +18002|F|M|Secondary|9000|Good|3|0|0| +18003|M|S|Secondary|9000|Good|3|0|0| +18004|F|S|Secondary|9000|Good|3|0|0| +18005|M|D|Secondary|9000|Good|3|0|0| +18006|F|D|Secondary|9000|Good|3|0|0| +18007|M|W|Secondary|9000|Good|3|0|0| +18008|F|W|Secondary|9000|Good|3|0|0| +18009|M|U|Secondary|9000|Good|3|0|0| +18010|F|U|Secondary|9000|Good|3|0|0| +18011|M|M|College|9000|Good|3|0|0| +18012|F|M|College|9000|Good|3|0|0| +18013|M|S|College|9000|Good|3|0|0| +18014|F|S|College|9000|Good|3|0|0| +18015|M|D|College|9000|Good|3|0|0| +18016|F|D|College|9000|Good|3|0|0| +18017|M|W|College|9000|Good|3|0|0| +18018|F|W|College|9000|Good|3|0|0| +18019|M|U|College|9000|Good|3|0|0| +18020|F|U|College|9000|Good|3|0|0| +18021|M|M|2 yr Degree|9000|Good|3|0|0| +18022|F|M|2 yr Degree|9000|Good|3|0|0| +18023|M|S|2 yr Degree|9000|Good|3|0|0| +18024|F|S|2 yr Degree|9000|Good|3|0|0| +18025|M|D|2 yr Degree|9000|Good|3|0|0| +18026|F|D|2 yr Degree|9000|Good|3|0|0| +18027|M|W|2 yr Degree|9000|Good|3|0|0| +18028|F|W|2 yr Degree|9000|Good|3|0|0| +18029|M|U|2 yr Degree|9000|Good|3|0|0| +18030|F|U|2 yr Degree|9000|Good|3|0|0| +18031|M|M|4 yr Degree|9000|Good|3|0|0| +18032|F|M|4 yr Degree|9000|Good|3|0|0| +18033|M|S|4 yr Degree|9000|Good|3|0|0| +18034|F|S|4 yr Degree|9000|Good|3|0|0| +18035|M|D|4 yr Degree|9000|Good|3|0|0| +18036|F|D|4 yr Degree|9000|Good|3|0|0| +18037|M|W|4 yr Degree|9000|Good|3|0|0| +18038|F|W|4 yr Degree|9000|Good|3|0|0| +18039|M|U|4 yr Degree|9000|Good|3|0|0| +18040|F|U|4 yr Degree|9000|Good|3|0|0| +18041|M|M|Advanced Degree|9000|Good|3|0|0| +18042|F|M|Advanced Degree|9000|Good|3|0|0| +18043|M|S|Advanced Degree|9000|Good|3|0|0| +18044|F|S|Advanced Degree|9000|Good|3|0|0| +18045|M|D|Advanced Degree|9000|Good|3|0|0| +18046|F|D|Advanced Degree|9000|Good|3|0|0| +18047|M|W|Advanced Degree|9000|Good|3|0|0| +18048|F|W|Advanced Degree|9000|Good|3|0|0| +18049|M|U|Advanced Degree|9000|Good|3|0|0| +18050|F|U|Advanced Degree|9000|Good|3|0|0| +18051|M|M|Unknown|9000|Good|3|0|0| +18052|F|M|Unknown|9000|Good|3|0|0| +18053|M|S|Unknown|9000|Good|3|0|0| +18054|F|S|Unknown|9000|Good|3|0|0| +18055|M|D|Unknown|9000|Good|3|0|0| +18056|F|D|Unknown|9000|Good|3|0|0| +18057|M|W|Unknown|9000|Good|3|0|0| +18058|F|W|Unknown|9000|Good|3|0|0| +18059|M|U|Unknown|9000|Good|3|0|0| +18060|F|U|Unknown|9000|Good|3|0|0| +18061|M|M|Primary|9500|Good|3|0|0| +18062|F|M|Primary|9500|Good|3|0|0| +18063|M|S|Primary|9500|Good|3|0|0| +18064|F|S|Primary|9500|Good|3|0|0| +18065|M|D|Primary|9500|Good|3|0|0| +18066|F|D|Primary|9500|Good|3|0|0| +18067|M|W|Primary|9500|Good|3|0|0| +18068|F|W|Primary|9500|Good|3|0|0| +18069|M|U|Primary|9500|Good|3|0|0| +18070|F|U|Primary|9500|Good|3|0|0| +18071|M|M|Secondary|9500|Good|3|0|0| +18072|F|M|Secondary|9500|Good|3|0|0| +18073|M|S|Secondary|9500|Good|3|0|0| +18074|F|S|Secondary|9500|Good|3|0|0| +18075|M|D|Secondary|9500|Good|3|0|0| +18076|F|D|Secondary|9500|Good|3|0|0| +18077|M|W|Secondary|9500|Good|3|0|0| +18078|F|W|Secondary|9500|Good|3|0|0| +18079|M|U|Secondary|9500|Good|3|0|0| +18080|F|U|Secondary|9500|Good|3|0|0| +18081|M|M|College|9500|Good|3|0|0| +18082|F|M|College|9500|Good|3|0|0| +18083|M|S|College|9500|Good|3|0|0| +18084|F|S|College|9500|Good|3|0|0| +18085|M|D|College|9500|Good|3|0|0| +18086|F|D|College|9500|Good|3|0|0| +18087|M|W|College|9500|Good|3|0|0| +18088|F|W|College|9500|Good|3|0|0| +18089|M|U|College|9500|Good|3|0|0| +18090|F|U|College|9500|Good|3|0|0| +18091|M|M|2 yr Degree|9500|Good|3|0|0| +18092|F|M|2 yr Degree|9500|Good|3|0|0| +18093|M|S|2 yr Degree|9500|Good|3|0|0| +18094|F|S|2 yr Degree|9500|Good|3|0|0| +18095|M|D|2 yr Degree|9500|Good|3|0|0| +18096|F|D|2 yr Degree|9500|Good|3|0|0| +18097|M|W|2 yr Degree|9500|Good|3|0|0| +18098|F|W|2 yr Degree|9500|Good|3|0|0| +18099|M|U|2 yr Degree|9500|Good|3|0|0| +18100|F|U|2 yr Degree|9500|Good|3|0|0| +18101|M|M|4 yr Degree|9500|Good|3|0|0| +18102|F|M|4 yr Degree|9500|Good|3|0|0| +18103|M|S|4 yr Degree|9500|Good|3|0|0| +18104|F|S|4 yr Degree|9500|Good|3|0|0| +18105|M|D|4 yr Degree|9500|Good|3|0|0| +18106|F|D|4 yr Degree|9500|Good|3|0|0| +18107|M|W|4 yr Degree|9500|Good|3|0|0| +18108|F|W|4 yr Degree|9500|Good|3|0|0| +18109|M|U|4 yr Degree|9500|Good|3|0|0| +18110|F|U|4 yr Degree|9500|Good|3|0|0| +18111|M|M|Advanced Degree|9500|Good|3|0|0| +18112|F|M|Advanced Degree|9500|Good|3|0|0| +18113|M|S|Advanced Degree|9500|Good|3|0|0| +18114|F|S|Advanced Degree|9500|Good|3|0|0| +18115|M|D|Advanced Degree|9500|Good|3|0|0| +18116|F|D|Advanced Degree|9500|Good|3|0|0| +18117|M|W|Advanced Degree|9500|Good|3|0|0| +18118|F|W|Advanced Degree|9500|Good|3|0|0| +18119|M|U|Advanced Degree|9500|Good|3|0|0| +18120|F|U|Advanced Degree|9500|Good|3|0|0| +18121|M|M|Unknown|9500|Good|3|0|0| +18122|F|M|Unknown|9500|Good|3|0|0| +18123|M|S|Unknown|9500|Good|3|0|0| +18124|F|S|Unknown|9500|Good|3|0|0| +18125|M|D|Unknown|9500|Good|3|0|0| +18126|F|D|Unknown|9500|Good|3|0|0| +18127|M|W|Unknown|9500|Good|3|0|0| +18128|F|W|Unknown|9500|Good|3|0|0| +18129|M|U|Unknown|9500|Good|3|0|0| +18130|F|U|Unknown|9500|Good|3|0|0| +18131|M|M|Primary|10000|Good|3|0|0| +18132|F|M|Primary|10000|Good|3|0|0| +18133|M|S|Primary|10000|Good|3|0|0| +18134|F|S|Primary|10000|Good|3|0|0| +18135|M|D|Primary|10000|Good|3|0|0| +18136|F|D|Primary|10000|Good|3|0|0| +18137|M|W|Primary|10000|Good|3|0|0| +18138|F|W|Primary|10000|Good|3|0|0| +18139|M|U|Primary|10000|Good|3|0|0| +18140|F|U|Primary|10000|Good|3|0|0| +18141|M|M|Secondary|10000|Good|3|0|0| +18142|F|M|Secondary|10000|Good|3|0|0| +18143|M|S|Secondary|10000|Good|3|0|0| +18144|F|S|Secondary|10000|Good|3|0|0| +18145|M|D|Secondary|10000|Good|3|0|0| +18146|F|D|Secondary|10000|Good|3|0|0| +18147|M|W|Secondary|10000|Good|3|0|0| +18148|F|W|Secondary|10000|Good|3|0|0| +18149|M|U|Secondary|10000|Good|3|0|0| +18150|F|U|Secondary|10000|Good|3|0|0| +18151|M|M|College|10000|Good|3|0|0| +18152|F|M|College|10000|Good|3|0|0| +18153|M|S|College|10000|Good|3|0|0| +18154|F|S|College|10000|Good|3|0|0| +18155|M|D|College|10000|Good|3|0|0| +18156|F|D|College|10000|Good|3|0|0| +18157|M|W|College|10000|Good|3|0|0| +18158|F|W|College|10000|Good|3|0|0| +18159|M|U|College|10000|Good|3|0|0| +18160|F|U|College|10000|Good|3|0|0| +18161|M|M|2 yr Degree|10000|Good|3|0|0| +18162|F|M|2 yr Degree|10000|Good|3|0|0| +18163|M|S|2 yr Degree|10000|Good|3|0|0| +18164|F|S|2 yr Degree|10000|Good|3|0|0| +18165|M|D|2 yr Degree|10000|Good|3|0|0| +18166|F|D|2 yr Degree|10000|Good|3|0|0| +18167|M|W|2 yr Degree|10000|Good|3|0|0| +18168|F|W|2 yr Degree|10000|Good|3|0|0| +18169|M|U|2 yr Degree|10000|Good|3|0|0| +18170|F|U|2 yr Degree|10000|Good|3|0|0| +18171|M|M|4 yr Degree|10000|Good|3|0|0| +18172|F|M|4 yr Degree|10000|Good|3|0|0| +18173|M|S|4 yr Degree|10000|Good|3|0|0| +18174|F|S|4 yr Degree|10000|Good|3|0|0| +18175|M|D|4 yr Degree|10000|Good|3|0|0| +18176|F|D|4 yr Degree|10000|Good|3|0|0| +18177|M|W|4 yr Degree|10000|Good|3|0|0| +18178|F|W|4 yr Degree|10000|Good|3|0|0| +18179|M|U|4 yr Degree|10000|Good|3|0|0| +18180|F|U|4 yr Degree|10000|Good|3|0|0| +18181|M|M|Advanced Degree|10000|Good|3|0|0| +18182|F|M|Advanced Degree|10000|Good|3|0|0| +18183|M|S|Advanced Degree|10000|Good|3|0|0| +18184|F|S|Advanced Degree|10000|Good|3|0|0| +18185|M|D|Advanced Degree|10000|Good|3|0|0| +18186|F|D|Advanced Degree|10000|Good|3|0|0| +18187|M|W|Advanced Degree|10000|Good|3|0|0| +18188|F|W|Advanced Degree|10000|Good|3|0|0| +18189|M|U|Advanced Degree|10000|Good|3|0|0| +18190|F|U|Advanced Degree|10000|Good|3|0|0| +18191|M|M|Unknown|10000|Good|3|0|0| +18192|F|M|Unknown|10000|Good|3|0|0| +18193|M|S|Unknown|10000|Good|3|0|0| +18194|F|S|Unknown|10000|Good|3|0|0| +18195|M|D|Unknown|10000|Good|3|0|0| +18196|F|D|Unknown|10000|Good|3|0|0| +18197|M|W|Unknown|10000|Good|3|0|0| +18198|F|W|Unknown|10000|Good|3|0|0| +18199|M|U|Unknown|10000|Good|3|0|0| +18200|F|U|Unknown|10000|Good|3|0|0| +18201|M|M|Primary|500|Low Risk|3|0|0| +18202|F|M|Primary|500|Low Risk|3|0|0| +18203|M|S|Primary|500|Low Risk|3|0|0| +18204|F|S|Primary|500|Low Risk|3|0|0| +18205|M|D|Primary|500|Low Risk|3|0|0| +18206|F|D|Primary|500|Low Risk|3|0|0| +18207|M|W|Primary|500|Low Risk|3|0|0| +18208|F|W|Primary|500|Low Risk|3|0|0| +18209|M|U|Primary|500|Low Risk|3|0|0| +18210|F|U|Primary|500|Low Risk|3|0|0| +18211|M|M|Secondary|500|Low Risk|3|0|0| +18212|F|M|Secondary|500|Low Risk|3|0|0| +18213|M|S|Secondary|500|Low Risk|3|0|0| +18214|F|S|Secondary|500|Low Risk|3|0|0| +18215|M|D|Secondary|500|Low Risk|3|0|0| +18216|F|D|Secondary|500|Low Risk|3|0|0| +18217|M|W|Secondary|500|Low Risk|3|0|0| +18218|F|W|Secondary|500|Low Risk|3|0|0| +18219|M|U|Secondary|500|Low Risk|3|0|0| +18220|F|U|Secondary|500|Low Risk|3|0|0| +18221|M|M|College|500|Low Risk|3|0|0| +18222|F|M|College|500|Low Risk|3|0|0| +18223|M|S|College|500|Low Risk|3|0|0| +18224|F|S|College|500|Low Risk|3|0|0| +18225|M|D|College|500|Low Risk|3|0|0| +18226|F|D|College|500|Low Risk|3|0|0| +18227|M|W|College|500|Low Risk|3|0|0| +18228|F|W|College|500|Low Risk|3|0|0| +18229|M|U|College|500|Low Risk|3|0|0| +18230|F|U|College|500|Low Risk|3|0|0| +18231|M|M|2 yr Degree|500|Low Risk|3|0|0| +18232|F|M|2 yr Degree|500|Low Risk|3|0|0| +18233|M|S|2 yr Degree|500|Low Risk|3|0|0| +18234|F|S|2 yr Degree|500|Low Risk|3|0|0| +18235|M|D|2 yr Degree|500|Low Risk|3|0|0| +18236|F|D|2 yr Degree|500|Low Risk|3|0|0| +18237|M|W|2 yr Degree|500|Low Risk|3|0|0| +18238|F|W|2 yr Degree|500|Low Risk|3|0|0| +18239|M|U|2 yr Degree|500|Low Risk|3|0|0| +18240|F|U|2 yr Degree|500|Low Risk|3|0|0| +18241|M|M|4 yr Degree|500|Low Risk|3|0|0| +18242|F|M|4 yr Degree|500|Low Risk|3|0|0| +18243|M|S|4 yr Degree|500|Low Risk|3|0|0| +18244|F|S|4 yr Degree|500|Low Risk|3|0|0| +18245|M|D|4 yr Degree|500|Low Risk|3|0|0| +18246|F|D|4 yr Degree|500|Low Risk|3|0|0| +18247|M|W|4 yr Degree|500|Low Risk|3|0|0| +18248|F|W|4 yr Degree|500|Low Risk|3|0|0| +18249|M|U|4 yr Degree|500|Low Risk|3|0|0| +18250|F|U|4 yr Degree|500|Low Risk|3|0|0| +18251|M|M|Advanced Degree|500|Low Risk|3|0|0| +18252|F|M|Advanced Degree|500|Low Risk|3|0|0| +18253|M|S|Advanced Degree|500|Low Risk|3|0|0| +18254|F|S|Advanced Degree|500|Low Risk|3|0|0| +18255|M|D|Advanced Degree|500|Low Risk|3|0|0| +18256|F|D|Advanced Degree|500|Low Risk|3|0|0| +18257|M|W|Advanced Degree|500|Low Risk|3|0|0| +18258|F|W|Advanced Degree|500|Low Risk|3|0|0| +18259|M|U|Advanced Degree|500|Low Risk|3|0|0| +18260|F|U|Advanced Degree|500|Low Risk|3|0|0| +18261|M|M|Unknown|500|Low Risk|3|0|0| +18262|F|M|Unknown|500|Low Risk|3|0|0| +18263|M|S|Unknown|500|Low Risk|3|0|0| +18264|F|S|Unknown|500|Low Risk|3|0|0| +18265|M|D|Unknown|500|Low Risk|3|0|0| +18266|F|D|Unknown|500|Low Risk|3|0|0| +18267|M|W|Unknown|500|Low Risk|3|0|0| +18268|F|W|Unknown|500|Low Risk|3|0|0| +18269|M|U|Unknown|500|Low Risk|3|0|0| +18270|F|U|Unknown|500|Low Risk|3|0|0| +18271|M|M|Primary|1000|Low Risk|3|0|0| +18272|F|M|Primary|1000|Low Risk|3|0|0| +18273|M|S|Primary|1000|Low Risk|3|0|0| +18274|F|S|Primary|1000|Low Risk|3|0|0| +18275|M|D|Primary|1000|Low Risk|3|0|0| +18276|F|D|Primary|1000|Low Risk|3|0|0| +18277|M|W|Primary|1000|Low Risk|3|0|0| +18278|F|W|Primary|1000|Low Risk|3|0|0| +18279|M|U|Primary|1000|Low Risk|3|0|0| +18280|F|U|Primary|1000|Low Risk|3|0|0| +18281|M|M|Secondary|1000|Low Risk|3|0|0| +18282|F|M|Secondary|1000|Low Risk|3|0|0| +18283|M|S|Secondary|1000|Low Risk|3|0|0| +18284|F|S|Secondary|1000|Low Risk|3|0|0| +18285|M|D|Secondary|1000|Low Risk|3|0|0| +18286|F|D|Secondary|1000|Low Risk|3|0|0| +18287|M|W|Secondary|1000|Low Risk|3|0|0| +18288|F|W|Secondary|1000|Low Risk|3|0|0| +18289|M|U|Secondary|1000|Low Risk|3|0|0| +18290|F|U|Secondary|1000|Low Risk|3|0|0| +18291|M|M|College|1000|Low Risk|3|0|0| +18292|F|M|College|1000|Low Risk|3|0|0| +18293|M|S|College|1000|Low Risk|3|0|0| +18294|F|S|College|1000|Low Risk|3|0|0| +18295|M|D|College|1000|Low Risk|3|0|0| +18296|F|D|College|1000|Low Risk|3|0|0| +18297|M|W|College|1000|Low Risk|3|0|0| +18298|F|W|College|1000|Low Risk|3|0|0| +18299|M|U|College|1000|Low Risk|3|0|0| +18300|F|U|College|1000|Low Risk|3|0|0| +18301|M|M|2 yr Degree|1000|Low Risk|3|0|0| +18302|F|M|2 yr Degree|1000|Low Risk|3|0|0| +18303|M|S|2 yr Degree|1000|Low Risk|3|0|0| +18304|F|S|2 yr Degree|1000|Low Risk|3|0|0| +18305|M|D|2 yr Degree|1000|Low Risk|3|0|0| +18306|F|D|2 yr Degree|1000|Low Risk|3|0|0| +18307|M|W|2 yr Degree|1000|Low Risk|3|0|0| +18308|F|W|2 yr Degree|1000|Low Risk|3|0|0| +18309|M|U|2 yr Degree|1000|Low Risk|3|0|0| +18310|F|U|2 yr Degree|1000|Low Risk|3|0|0| +18311|M|M|4 yr Degree|1000|Low Risk|3|0|0| +18312|F|M|4 yr Degree|1000|Low Risk|3|0|0| +18313|M|S|4 yr Degree|1000|Low Risk|3|0|0| +18314|F|S|4 yr Degree|1000|Low Risk|3|0|0| +18315|M|D|4 yr Degree|1000|Low Risk|3|0|0| +18316|F|D|4 yr Degree|1000|Low Risk|3|0|0| +18317|M|W|4 yr Degree|1000|Low Risk|3|0|0| +18318|F|W|4 yr Degree|1000|Low Risk|3|0|0| +18319|M|U|4 yr Degree|1000|Low Risk|3|0|0| +18320|F|U|4 yr Degree|1000|Low Risk|3|0|0| +18321|M|M|Advanced Degree|1000|Low Risk|3|0|0| +18322|F|M|Advanced Degree|1000|Low Risk|3|0|0| +18323|M|S|Advanced Degree|1000|Low Risk|3|0|0| +18324|F|S|Advanced Degree|1000|Low Risk|3|0|0| +18325|M|D|Advanced Degree|1000|Low Risk|3|0|0| +18326|F|D|Advanced Degree|1000|Low Risk|3|0|0| +18327|M|W|Advanced Degree|1000|Low Risk|3|0|0| +18328|F|W|Advanced Degree|1000|Low Risk|3|0|0| +18329|M|U|Advanced Degree|1000|Low Risk|3|0|0| +18330|F|U|Advanced Degree|1000|Low Risk|3|0|0| +18331|M|M|Unknown|1000|Low Risk|3|0|0| +18332|F|M|Unknown|1000|Low Risk|3|0|0| +18333|M|S|Unknown|1000|Low Risk|3|0|0| +18334|F|S|Unknown|1000|Low Risk|3|0|0| +18335|M|D|Unknown|1000|Low Risk|3|0|0| +18336|F|D|Unknown|1000|Low Risk|3|0|0| +18337|M|W|Unknown|1000|Low Risk|3|0|0| +18338|F|W|Unknown|1000|Low Risk|3|0|0| +18339|M|U|Unknown|1000|Low Risk|3|0|0| +18340|F|U|Unknown|1000|Low Risk|3|0|0| +18341|M|M|Primary|1500|Low Risk|3|0|0| +18342|F|M|Primary|1500|Low Risk|3|0|0| +18343|M|S|Primary|1500|Low Risk|3|0|0| +18344|F|S|Primary|1500|Low Risk|3|0|0| +18345|M|D|Primary|1500|Low Risk|3|0|0| +18346|F|D|Primary|1500|Low Risk|3|0|0| +18347|M|W|Primary|1500|Low Risk|3|0|0| +18348|F|W|Primary|1500|Low Risk|3|0|0| +18349|M|U|Primary|1500|Low Risk|3|0|0| +18350|F|U|Primary|1500|Low Risk|3|0|0| +18351|M|M|Secondary|1500|Low Risk|3|0|0| +18352|F|M|Secondary|1500|Low Risk|3|0|0| +18353|M|S|Secondary|1500|Low Risk|3|0|0| +18354|F|S|Secondary|1500|Low Risk|3|0|0| +18355|M|D|Secondary|1500|Low Risk|3|0|0| +18356|F|D|Secondary|1500|Low Risk|3|0|0| +18357|M|W|Secondary|1500|Low Risk|3|0|0| +18358|F|W|Secondary|1500|Low Risk|3|0|0| +18359|M|U|Secondary|1500|Low Risk|3|0|0| +18360|F|U|Secondary|1500|Low Risk|3|0|0| +18361|M|M|College|1500|Low Risk|3|0|0| +18362|F|M|College|1500|Low Risk|3|0|0| +18363|M|S|College|1500|Low Risk|3|0|0| +18364|F|S|College|1500|Low Risk|3|0|0| +18365|M|D|College|1500|Low Risk|3|0|0| +18366|F|D|College|1500|Low Risk|3|0|0| +18367|M|W|College|1500|Low Risk|3|0|0| +18368|F|W|College|1500|Low Risk|3|0|0| +18369|M|U|College|1500|Low Risk|3|0|0| +18370|F|U|College|1500|Low Risk|3|0|0| +18371|M|M|2 yr Degree|1500|Low Risk|3|0|0| +18372|F|M|2 yr Degree|1500|Low Risk|3|0|0| +18373|M|S|2 yr Degree|1500|Low Risk|3|0|0| +18374|F|S|2 yr Degree|1500|Low Risk|3|0|0| +18375|M|D|2 yr Degree|1500|Low Risk|3|0|0| +18376|F|D|2 yr Degree|1500|Low Risk|3|0|0| +18377|M|W|2 yr Degree|1500|Low Risk|3|0|0| +18378|F|W|2 yr Degree|1500|Low Risk|3|0|0| +18379|M|U|2 yr Degree|1500|Low Risk|3|0|0| +18380|F|U|2 yr Degree|1500|Low Risk|3|0|0| +18381|M|M|4 yr Degree|1500|Low Risk|3|0|0| +18382|F|M|4 yr Degree|1500|Low Risk|3|0|0| +18383|M|S|4 yr Degree|1500|Low Risk|3|0|0| +18384|F|S|4 yr Degree|1500|Low Risk|3|0|0| +18385|M|D|4 yr Degree|1500|Low Risk|3|0|0| +18386|F|D|4 yr Degree|1500|Low Risk|3|0|0| +18387|M|W|4 yr Degree|1500|Low Risk|3|0|0| +18388|F|W|4 yr Degree|1500|Low Risk|3|0|0| +18389|M|U|4 yr Degree|1500|Low Risk|3|0|0| +18390|F|U|4 yr Degree|1500|Low Risk|3|0|0| +18391|M|M|Advanced Degree|1500|Low Risk|3|0|0| +18392|F|M|Advanced Degree|1500|Low Risk|3|0|0| +18393|M|S|Advanced Degree|1500|Low Risk|3|0|0| +18394|F|S|Advanced Degree|1500|Low Risk|3|0|0| +18395|M|D|Advanced Degree|1500|Low Risk|3|0|0| +18396|F|D|Advanced Degree|1500|Low Risk|3|0|0| +18397|M|W|Advanced Degree|1500|Low Risk|3|0|0| +18398|F|W|Advanced Degree|1500|Low Risk|3|0|0| +18399|M|U|Advanced Degree|1500|Low Risk|3|0|0| +18400|F|U|Advanced Degree|1500|Low Risk|3|0|0| +18401|M|M|Unknown|1500|Low Risk|3|0|0| +18402|F|M|Unknown|1500|Low Risk|3|0|0| +18403|M|S|Unknown|1500|Low Risk|3|0|0| +18404|F|S|Unknown|1500|Low Risk|3|0|0| +18405|M|D|Unknown|1500|Low Risk|3|0|0| +18406|F|D|Unknown|1500|Low Risk|3|0|0| +18407|M|W|Unknown|1500|Low Risk|3|0|0| +18408|F|W|Unknown|1500|Low Risk|3|0|0| +18409|M|U|Unknown|1500|Low Risk|3|0|0| +18410|F|U|Unknown|1500|Low Risk|3|0|0| +18411|M|M|Primary|2000|Low Risk|3|0|0| +18412|F|M|Primary|2000|Low Risk|3|0|0| +18413|M|S|Primary|2000|Low Risk|3|0|0| +18414|F|S|Primary|2000|Low Risk|3|0|0| +18415|M|D|Primary|2000|Low Risk|3|0|0| +18416|F|D|Primary|2000|Low Risk|3|0|0| +18417|M|W|Primary|2000|Low Risk|3|0|0| +18418|F|W|Primary|2000|Low Risk|3|0|0| +18419|M|U|Primary|2000|Low Risk|3|0|0| +18420|F|U|Primary|2000|Low Risk|3|0|0| +18421|M|M|Secondary|2000|Low Risk|3|0|0| +18422|F|M|Secondary|2000|Low Risk|3|0|0| +18423|M|S|Secondary|2000|Low Risk|3|0|0| +18424|F|S|Secondary|2000|Low Risk|3|0|0| +18425|M|D|Secondary|2000|Low Risk|3|0|0| +18426|F|D|Secondary|2000|Low Risk|3|0|0| +18427|M|W|Secondary|2000|Low Risk|3|0|0| +18428|F|W|Secondary|2000|Low Risk|3|0|0| +18429|M|U|Secondary|2000|Low Risk|3|0|0| +18430|F|U|Secondary|2000|Low Risk|3|0|0| +18431|M|M|College|2000|Low Risk|3|0|0| +18432|F|M|College|2000|Low Risk|3|0|0| +18433|M|S|College|2000|Low Risk|3|0|0| +18434|F|S|College|2000|Low Risk|3|0|0| +18435|M|D|College|2000|Low Risk|3|0|0| +18436|F|D|College|2000|Low Risk|3|0|0| +18437|M|W|College|2000|Low Risk|3|0|0| +18438|F|W|College|2000|Low Risk|3|0|0| +18439|M|U|College|2000|Low Risk|3|0|0| +18440|F|U|College|2000|Low Risk|3|0|0| +18441|M|M|2 yr Degree|2000|Low Risk|3|0|0| +18442|F|M|2 yr Degree|2000|Low Risk|3|0|0| +18443|M|S|2 yr Degree|2000|Low Risk|3|0|0| +18444|F|S|2 yr Degree|2000|Low Risk|3|0|0| +18445|M|D|2 yr Degree|2000|Low Risk|3|0|0| +18446|F|D|2 yr Degree|2000|Low Risk|3|0|0| +18447|M|W|2 yr Degree|2000|Low Risk|3|0|0| +18448|F|W|2 yr Degree|2000|Low Risk|3|0|0| +18449|M|U|2 yr Degree|2000|Low Risk|3|0|0| +18450|F|U|2 yr Degree|2000|Low Risk|3|0|0| +18451|M|M|4 yr Degree|2000|Low Risk|3|0|0| +18452|F|M|4 yr Degree|2000|Low Risk|3|0|0| +18453|M|S|4 yr Degree|2000|Low Risk|3|0|0| +18454|F|S|4 yr Degree|2000|Low Risk|3|0|0| +18455|M|D|4 yr Degree|2000|Low Risk|3|0|0| +18456|F|D|4 yr Degree|2000|Low Risk|3|0|0| +18457|M|W|4 yr Degree|2000|Low Risk|3|0|0| +18458|F|W|4 yr Degree|2000|Low Risk|3|0|0| +18459|M|U|4 yr Degree|2000|Low Risk|3|0|0| +18460|F|U|4 yr Degree|2000|Low Risk|3|0|0| +18461|M|M|Advanced Degree|2000|Low Risk|3|0|0| +18462|F|M|Advanced Degree|2000|Low Risk|3|0|0| +18463|M|S|Advanced Degree|2000|Low Risk|3|0|0| +18464|F|S|Advanced Degree|2000|Low Risk|3|0|0| +18465|M|D|Advanced Degree|2000|Low Risk|3|0|0| +18466|F|D|Advanced Degree|2000|Low Risk|3|0|0| +18467|M|W|Advanced Degree|2000|Low Risk|3|0|0| +18468|F|W|Advanced Degree|2000|Low Risk|3|0|0| +18469|M|U|Advanced Degree|2000|Low Risk|3|0|0| +18470|F|U|Advanced Degree|2000|Low Risk|3|0|0| +18471|M|M|Unknown|2000|Low Risk|3|0|0| +18472|F|M|Unknown|2000|Low Risk|3|0|0| +18473|M|S|Unknown|2000|Low Risk|3|0|0| +18474|F|S|Unknown|2000|Low Risk|3|0|0| +18475|M|D|Unknown|2000|Low Risk|3|0|0| +18476|F|D|Unknown|2000|Low Risk|3|0|0| +18477|M|W|Unknown|2000|Low Risk|3|0|0| +18478|F|W|Unknown|2000|Low Risk|3|0|0| +18479|M|U|Unknown|2000|Low Risk|3|0|0| +18480|F|U|Unknown|2000|Low Risk|3|0|0| +18481|M|M|Primary|2500|Low Risk|3|0|0| +18482|F|M|Primary|2500|Low Risk|3|0|0| +18483|M|S|Primary|2500|Low Risk|3|0|0| +18484|F|S|Primary|2500|Low Risk|3|0|0| +18485|M|D|Primary|2500|Low Risk|3|0|0| +18486|F|D|Primary|2500|Low Risk|3|0|0| +18487|M|W|Primary|2500|Low Risk|3|0|0| +18488|F|W|Primary|2500|Low Risk|3|0|0| +18489|M|U|Primary|2500|Low Risk|3|0|0| +18490|F|U|Primary|2500|Low Risk|3|0|0| +18491|M|M|Secondary|2500|Low Risk|3|0|0| +18492|F|M|Secondary|2500|Low Risk|3|0|0| +18493|M|S|Secondary|2500|Low Risk|3|0|0| +18494|F|S|Secondary|2500|Low Risk|3|0|0| +18495|M|D|Secondary|2500|Low Risk|3|0|0| +18496|F|D|Secondary|2500|Low Risk|3|0|0| +18497|M|W|Secondary|2500|Low Risk|3|0|0| +18498|F|W|Secondary|2500|Low Risk|3|0|0| +18499|M|U|Secondary|2500|Low Risk|3|0|0| +18500|F|U|Secondary|2500|Low Risk|3|0|0| +18501|M|M|College|2500|Low Risk|3|0|0| +18502|F|M|College|2500|Low Risk|3|0|0| +18503|M|S|College|2500|Low Risk|3|0|0| +18504|F|S|College|2500|Low Risk|3|0|0| +18505|M|D|College|2500|Low Risk|3|0|0| +18506|F|D|College|2500|Low Risk|3|0|0| +18507|M|W|College|2500|Low Risk|3|0|0| +18508|F|W|College|2500|Low Risk|3|0|0| +18509|M|U|College|2500|Low Risk|3|0|0| +18510|F|U|College|2500|Low Risk|3|0|0| +18511|M|M|2 yr Degree|2500|Low Risk|3|0|0| +18512|F|M|2 yr Degree|2500|Low Risk|3|0|0| +18513|M|S|2 yr Degree|2500|Low Risk|3|0|0| +18514|F|S|2 yr Degree|2500|Low Risk|3|0|0| +18515|M|D|2 yr Degree|2500|Low Risk|3|0|0| +18516|F|D|2 yr Degree|2500|Low Risk|3|0|0| +18517|M|W|2 yr Degree|2500|Low Risk|3|0|0| +18518|F|W|2 yr Degree|2500|Low Risk|3|0|0| +18519|M|U|2 yr Degree|2500|Low Risk|3|0|0| +18520|F|U|2 yr Degree|2500|Low Risk|3|0|0| +18521|M|M|4 yr Degree|2500|Low Risk|3|0|0| +18522|F|M|4 yr Degree|2500|Low Risk|3|0|0| +18523|M|S|4 yr Degree|2500|Low Risk|3|0|0| +18524|F|S|4 yr Degree|2500|Low Risk|3|0|0| +18525|M|D|4 yr Degree|2500|Low Risk|3|0|0| +18526|F|D|4 yr Degree|2500|Low Risk|3|0|0| +18527|M|W|4 yr Degree|2500|Low Risk|3|0|0| +18528|F|W|4 yr Degree|2500|Low Risk|3|0|0| +18529|M|U|4 yr Degree|2500|Low Risk|3|0|0| +18530|F|U|4 yr Degree|2500|Low Risk|3|0|0| +18531|M|M|Advanced Degree|2500|Low Risk|3|0|0| +18532|F|M|Advanced Degree|2500|Low Risk|3|0|0| +18533|M|S|Advanced Degree|2500|Low Risk|3|0|0| +18534|F|S|Advanced Degree|2500|Low Risk|3|0|0| +18535|M|D|Advanced Degree|2500|Low Risk|3|0|0| +18536|F|D|Advanced Degree|2500|Low Risk|3|0|0| +18537|M|W|Advanced Degree|2500|Low Risk|3|0|0| +18538|F|W|Advanced Degree|2500|Low Risk|3|0|0| +18539|M|U|Advanced Degree|2500|Low Risk|3|0|0| +18540|F|U|Advanced Degree|2500|Low Risk|3|0|0| +18541|M|M|Unknown|2500|Low Risk|3|0|0| +18542|F|M|Unknown|2500|Low Risk|3|0|0| +18543|M|S|Unknown|2500|Low Risk|3|0|0| +18544|F|S|Unknown|2500|Low Risk|3|0|0| +18545|M|D|Unknown|2500|Low Risk|3|0|0| +18546|F|D|Unknown|2500|Low Risk|3|0|0| +18547|M|W|Unknown|2500|Low Risk|3|0|0| +18548|F|W|Unknown|2500|Low Risk|3|0|0| +18549|M|U|Unknown|2500|Low Risk|3|0|0| +18550|F|U|Unknown|2500|Low Risk|3|0|0| +18551|M|M|Primary|3000|Low Risk|3|0|0| +18552|F|M|Primary|3000|Low Risk|3|0|0| +18553|M|S|Primary|3000|Low Risk|3|0|0| +18554|F|S|Primary|3000|Low Risk|3|0|0| +18555|M|D|Primary|3000|Low Risk|3|0|0| +18556|F|D|Primary|3000|Low Risk|3|0|0| +18557|M|W|Primary|3000|Low Risk|3|0|0| +18558|F|W|Primary|3000|Low Risk|3|0|0| +18559|M|U|Primary|3000|Low Risk|3|0|0| +18560|F|U|Primary|3000|Low Risk|3|0|0| +18561|M|M|Secondary|3000|Low Risk|3|0|0| +18562|F|M|Secondary|3000|Low Risk|3|0|0| +18563|M|S|Secondary|3000|Low Risk|3|0|0| +18564|F|S|Secondary|3000|Low Risk|3|0|0| +18565|M|D|Secondary|3000|Low Risk|3|0|0| +18566|F|D|Secondary|3000|Low Risk|3|0|0| +18567|M|W|Secondary|3000|Low Risk|3|0|0| +18568|F|W|Secondary|3000|Low Risk|3|0|0| +18569|M|U|Secondary|3000|Low Risk|3|0|0| +18570|F|U|Secondary|3000|Low Risk|3|0|0| +18571|M|M|College|3000|Low Risk|3|0|0| +18572|F|M|College|3000|Low Risk|3|0|0| +18573|M|S|College|3000|Low Risk|3|0|0| +18574|F|S|College|3000|Low Risk|3|0|0| +18575|M|D|College|3000|Low Risk|3|0|0| +18576|F|D|College|3000|Low Risk|3|0|0| +18577|M|W|College|3000|Low Risk|3|0|0| +18578|F|W|College|3000|Low Risk|3|0|0| +18579|M|U|College|3000|Low Risk|3|0|0| +18580|F|U|College|3000|Low Risk|3|0|0| +18581|M|M|2 yr Degree|3000|Low Risk|3|0|0| +18582|F|M|2 yr Degree|3000|Low Risk|3|0|0| +18583|M|S|2 yr Degree|3000|Low Risk|3|0|0| +18584|F|S|2 yr Degree|3000|Low Risk|3|0|0| +18585|M|D|2 yr Degree|3000|Low Risk|3|0|0| +18586|F|D|2 yr Degree|3000|Low Risk|3|0|0| +18587|M|W|2 yr Degree|3000|Low Risk|3|0|0| +18588|F|W|2 yr Degree|3000|Low Risk|3|0|0| +18589|M|U|2 yr Degree|3000|Low Risk|3|0|0| +18590|F|U|2 yr Degree|3000|Low Risk|3|0|0| +18591|M|M|4 yr Degree|3000|Low Risk|3|0|0| +18592|F|M|4 yr Degree|3000|Low Risk|3|0|0| +18593|M|S|4 yr Degree|3000|Low Risk|3|0|0| +18594|F|S|4 yr Degree|3000|Low Risk|3|0|0| +18595|M|D|4 yr Degree|3000|Low Risk|3|0|0| +18596|F|D|4 yr Degree|3000|Low Risk|3|0|0| +18597|M|W|4 yr Degree|3000|Low Risk|3|0|0| +18598|F|W|4 yr Degree|3000|Low Risk|3|0|0| +18599|M|U|4 yr Degree|3000|Low Risk|3|0|0| +18600|F|U|4 yr Degree|3000|Low Risk|3|0|0| +18601|M|M|Advanced Degree|3000|Low Risk|3|0|0| +18602|F|M|Advanced Degree|3000|Low Risk|3|0|0| +18603|M|S|Advanced Degree|3000|Low Risk|3|0|0| +18604|F|S|Advanced Degree|3000|Low Risk|3|0|0| +18605|M|D|Advanced Degree|3000|Low Risk|3|0|0| +18606|F|D|Advanced Degree|3000|Low Risk|3|0|0| +18607|M|W|Advanced Degree|3000|Low Risk|3|0|0| +18608|F|W|Advanced Degree|3000|Low Risk|3|0|0| +18609|M|U|Advanced Degree|3000|Low Risk|3|0|0| +18610|F|U|Advanced Degree|3000|Low Risk|3|0|0| +18611|M|M|Unknown|3000|Low Risk|3|0|0| +18612|F|M|Unknown|3000|Low Risk|3|0|0| +18613|M|S|Unknown|3000|Low Risk|3|0|0| +18614|F|S|Unknown|3000|Low Risk|3|0|0| +18615|M|D|Unknown|3000|Low Risk|3|0|0| +18616|F|D|Unknown|3000|Low Risk|3|0|0| +18617|M|W|Unknown|3000|Low Risk|3|0|0| +18618|F|W|Unknown|3000|Low Risk|3|0|0| +18619|M|U|Unknown|3000|Low Risk|3|0|0| +18620|F|U|Unknown|3000|Low Risk|3|0|0| +18621|M|M|Primary|3500|Low Risk|3|0|0| +18622|F|M|Primary|3500|Low Risk|3|0|0| +18623|M|S|Primary|3500|Low Risk|3|0|0| +18624|F|S|Primary|3500|Low Risk|3|0|0| +18625|M|D|Primary|3500|Low Risk|3|0|0| +18626|F|D|Primary|3500|Low Risk|3|0|0| +18627|M|W|Primary|3500|Low Risk|3|0|0| +18628|F|W|Primary|3500|Low Risk|3|0|0| +18629|M|U|Primary|3500|Low Risk|3|0|0| +18630|F|U|Primary|3500|Low Risk|3|0|0| +18631|M|M|Secondary|3500|Low Risk|3|0|0| +18632|F|M|Secondary|3500|Low Risk|3|0|0| +18633|M|S|Secondary|3500|Low Risk|3|0|0| +18634|F|S|Secondary|3500|Low Risk|3|0|0| +18635|M|D|Secondary|3500|Low Risk|3|0|0| +18636|F|D|Secondary|3500|Low Risk|3|0|0| +18637|M|W|Secondary|3500|Low Risk|3|0|0| +18638|F|W|Secondary|3500|Low Risk|3|0|0| +18639|M|U|Secondary|3500|Low Risk|3|0|0| +18640|F|U|Secondary|3500|Low Risk|3|0|0| +18641|M|M|College|3500|Low Risk|3|0|0| +18642|F|M|College|3500|Low Risk|3|0|0| +18643|M|S|College|3500|Low Risk|3|0|0| +18644|F|S|College|3500|Low Risk|3|0|0| +18645|M|D|College|3500|Low Risk|3|0|0| +18646|F|D|College|3500|Low Risk|3|0|0| +18647|M|W|College|3500|Low Risk|3|0|0| +18648|F|W|College|3500|Low Risk|3|0|0| +18649|M|U|College|3500|Low Risk|3|0|0| +18650|F|U|College|3500|Low Risk|3|0|0| +18651|M|M|2 yr Degree|3500|Low Risk|3|0|0| +18652|F|M|2 yr Degree|3500|Low Risk|3|0|0| +18653|M|S|2 yr Degree|3500|Low Risk|3|0|0| +18654|F|S|2 yr Degree|3500|Low Risk|3|0|0| +18655|M|D|2 yr Degree|3500|Low Risk|3|0|0| +18656|F|D|2 yr Degree|3500|Low Risk|3|0|0| +18657|M|W|2 yr Degree|3500|Low Risk|3|0|0| +18658|F|W|2 yr Degree|3500|Low Risk|3|0|0| +18659|M|U|2 yr Degree|3500|Low Risk|3|0|0| +18660|F|U|2 yr Degree|3500|Low Risk|3|0|0| +18661|M|M|4 yr Degree|3500|Low Risk|3|0|0| +18662|F|M|4 yr Degree|3500|Low Risk|3|0|0| +18663|M|S|4 yr Degree|3500|Low Risk|3|0|0| +18664|F|S|4 yr Degree|3500|Low Risk|3|0|0| +18665|M|D|4 yr Degree|3500|Low Risk|3|0|0| +18666|F|D|4 yr Degree|3500|Low Risk|3|0|0| +18667|M|W|4 yr Degree|3500|Low Risk|3|0|0| +18668|F|W|4 yr Degree|3500|Low Risk|3|0|0| +18669|M|U|4 yr Degree|3500|Low Risk|3|0|0| +18670|F|U|4 yr Degree|3500|Low Risk|3|0|0| +18671|M|M|Advanced Degree|3500|Low Risk|3|0|0| +18672|F|M|Advanced Degree|3500|Low Risk|3|0|0| +18673|M|S|Advanced Degree|3500|Low Risk|3|0|0| +18674|F|S|Advanced Degree|3500|Low Risk|3|0|0| +18675|M|D|Advanced Degree|3500|Low Risk|3|0|0| +18676|F|D|Advanced Degree|3500|Low Risk|3|0|0| +18677|M|W|Advanced Degree|3500|Low Risk|3|0|0| +18678|F|W|Advanced Degree|3500|Low Risk|3|0|0| +18679|M|U|Advanced Degree|3500|Low Risk|3|0|0| +18680|F|U|Advanced Degree|3500|Low Risk|3|0|0| +18681|M|M|Unknown|3500|Low Risk|3|0|0| +18682|F|M|Unknown|3500|Low Risk|3|0|0| +18683|M|S|Unknown|3500|Low Risk|3|0|0| +18684|F|S|Unknown|3500|Low Risk|3|0|0| +18685|M|D|Unknown|3500|Low Risk|3|0|0| +18686|F|D|Unknown|3500|Low Risk|3|0|0| +18687|M|W|Unknown|3500|Low Risk|3|0|0| +18688|F|W|Unknown|3500|Low Risk|3|0|0| +18689|M|U|Unknown|3500|Low Risk|3|0|0| +18690|F|U|Unknown|3500|Low Risk|3|0|0| +18691|M|M|Primary|4000|Low Risk|3|0|0| +18692|F|M|Primary|4000|Low Risk|3|0|0| +18693|M|S|Primary|4000|Low Risk|3|0|0| +18694|F|S|Primary|4000|Low Risk|3|0|0| +18695|M|D|Primary|4000|Low Risk|3|0|0| +18696|F|D|Primary|4000|Low Risk|3|0|0| +18697|M|W|Primary|4000|Low Risk|3|0|0| +18698|F|W|Primary|4000|Low Risk|3|0|0| +18699|M|U|Primary|4000|Low Risk|3|0|0| +18700|F|U|Primary|4000|Low Risk|3|0|0| +18701|M|M|Secondary|4000|Low Risk|3|0|0| +18702|F|M|Secondary|4000|Low Risk|3|0|0| +18703|M|S|Secondary|4000|Low Risk|3|0|0| +18704|F|S|Secondary|4000|Low Risk|3|0|0| +18705|M|D|Secondary|4000|Low Risk|3|0|0| +18706|F|D|Secondary|4000|Low Risk|3|0|0| +18707|M|W|Secondary|4000|Low Risk|3|0|0| +18708|F|W|Secondary|4000|Low Risk|3|0|0| +18709|M|U|Secondary|4000|Low Risk|3|0|0| +18710|F|U|Secondary|4000|Low Risk|3|0|0| +18711|M|M|College|4000|Low Risk|3|0|0| +18712|F|M|College|4000|Low Risk|3|0|0| +18713|M|S|College|4000|Low Risk|3|0|0| +18714|F|S|College|4000|Low Risk|3|0|0| +18715|M|D|College|4000|Low Risk|3|0|0| +18716|F|D|College|4000|Low Risk|3|0|0| +18717|M|W|College|4000|Low Risk|3|0|0| +18718|F|W|College|4000|Low Risk|3|0|0| +18719|M|U|College|4000|Low Risk|3|0|0| +18720|F|U|College|4000|Low Risk|3|0|0| +18721|M|M|2 yr Degree|4000|Low Risk|3|0|0| +18722|F|M|2 yr Degree|4000|Low Risk|3|0|0| +18723|M|S|2 yr Degree|4000|Low Risk|3|0|0| +18724|F|S|2 yr Degree|4000|Low Risk|3|0|0| +18725|M|D|2 yr Degree|4000|Low Risk|3|0|0| +18726|F|D|2 yr Degree|4000|Low Risk|3|0|0| +18727|M|W|2 yr Degree|4000|Low Risk|3|0|0| +18728|F|W|2 yr Degree|4000|Low Risk|3|0|0| +18729|M|U|2 yr Degree|4000|Low Risk|3|0|0| +18730|F|U|2 yr Degree|4000|Low Risk|3|0|0| +18731|M|M|4 yr Degree|4000|Low Risk|3|0|0| +18732|F|M|4 yr Degree|4000|Low Risk|3|0|0| +18733|M|S|4 yr Degree|4000|Low Risk|3|0|0| +18734|F|S|4 yr Degree|4000|Low Risk|3|0|0| +18735|M|D|4 yr Degree|4000|Low Risk|3|0|0| +18736|F|D|4 yr Degree|4000|Low Risk|3|0|0| +18737|M|W|4 yr Degree|4000|Low Risk|3|0|0| +18738|F|W|4 yr Degree|4000|Low Risk|3|0|0| +18739|M|U|4 yr Degree|4000|Low Risk|3|0|0| +18740|F|U|4 yr Degree|4000|Low Risk|3|0|0| +18741|M|M|Advanced Degree|4000|Low Risk|3|0|0| +18742|F|M|Advanced Degree|4000|Low Risk|3|0|0| +18743|M|S|Advanced Degree|4000|Low Risk|3|0|0| +18744|F|S|Advanced Degree|4000|Low Risk|3|0|0| +18745|M|D|Advanced Degree|4000|Low Risk|3|0|0| +18746|F|D|Advanced Degree|4000|Low Risk|3|0|0| +18747|M|W|Advanced Degree|4000|Low Risk|3|0|0| +18748|F|W|Advanced Degree|4000|Low Risk|3|0|0| +18749|M|U|Advanced Degree|4000|Low Risk|3|0|0| +18750|F|U|Advanced Degree|4000|Low Risk|3|0|0| +18751|M|M|Unknown|4000|Low Risk|3|0|0| +18752|F|M|Unknown|4000|Low Risk|3|0|0| +18753|M|S|Unknown|4000|Low Risk|3|0|0| +18754|F|S|Unknown|4000|Low Risk|3|0|0| +18755|M|D|Unknown|4000|Low Risk|3|0|0| +18756|F|D|Unknown|4000|Low Risk|3|0|0| +18757|M|W|Unknown|4000|Low Risk|3|0|0| +18758|F|W|Unknown|4000|Low Risk|3|0|0| +18759|M|U|Unknown|4000|Low Risk|3|0|0| +18760|F|U|Unknown|4000|Low Risk|3|0|0| +18761|M|M|Primary|4500|Low Risk|3|0|0| +18762|F|M|Primary|4500|Low Risk|3|0|0| +18763|M|S|Primary|4500|Low Risk|3|0|0| +18764|F|S|Primary|4500|Low Risk|3|0|0| +18765|M|D|Primary|4500|Low Risk|3|0|0| +18766|F|D|Primary|4500|Low Risk|3|0|0| +18767|M|W|Primary|4500|Low Risk|3|0|0| +18768|F|W|Primary|4500|Low Risk|3|0|0| +18769|M|U|Primary|4500|Low Risk|3|0|0| +18770|F|U|Primary|4500|Low Risk|3|0|0| +18771|M|M|Secondary|4500|Low Risk|3|0|0| +18772|F|M|Secondary|4500|Low Risk|3|0|0| +18773|M|S|Secondary|4500|Low Risk|3|0|0| +18774|F|S|Secondary|4500|Low Risk|3|0|0| +18775|M|D|Secondary|4500|Low Risk|3|0|0| +18776|F|D|Secondary|4500|Low Risk|3|0|0| +18777|M|W|Secondary|4500|Low Risk|3|0|0| +18778|F|W|Secondary|4500|Low Risk|3|0|0| +18779|M|U|Secondary|4500|Low Risk|3|0|0| +18780|F|U|Secondary|4500|Low Risk|3|0|0| +18781|M|M|College|4500|Low Risk|3|0|0| +18782|F|M|College|4500|Low Risk|3|0|0| +18783|M|S|College|4500|Low Risk|3|0|0| +18784|F|S|College|4500|Low Risk|3|0|0| +18785|M|D|College|4500|Low Risk|3|0|0| +18786|F|D|College|4500|Low Risk|3|0|0| +18787|M|W|College|4500|Low Risk|3|0|0| +18788|F|W|College|4500|Low Risk|3|0|0| +18789|M|U|College|4500|Low Risk|3|0|0| +18790|F|U|College|4500|Low Risk|3|0|0| +18791|M|M|2 yr Degree|4500|Low Risk|3|0|0| +18792|F|M|2 yr Degree|4500|Low Risk|3|0|0| +18793|M|S|2 yr Degree|4500|Low Risk|3|0|0| +18794|F|S|2 yr Degree|4500|Low Risk|3|0|0| +18795|M|D|2 yr Degree|4500|Low Risk|3|0|0| +18796|F|D|2 yr Degree|4500|Low Risk|3|0|0| +18797|M|W|2 yr Degree|4500|Low Risk|3|0|0| +18798|F|W|2 yr Degree|4500|Low Risk|3|0|0| +18799|M|U|2 yr Degree|4500|Low Risk|3|0|0| +18800|F|U|2 yr Degree|4500|Low Risk|3|0|0| +18801|M|M|4 yr Degree|4500|Low Risk|3|0|0| +18802|F|M|4 yr Degree|4500|Low Risk|3|0|0| +18803|M|S|4 yr Degree|4500|Low Risk|3|0|0| +18804|F|S|4 yr Degree|4500|Low Risk|3|0|0| +18805|M|D|4 yr Degree|4500|Low Risk|3|0|0| +18806|F|D|4 yr Degree|4500|Low Risk|3|0|0| +18807|M|W|4 yr Degree|4500|Low Risk|3|0|0| +18808|F|W|4 yr Degree|4500|Low Risk|3|0|0| +18809|M|U|4 yr Degree|4500|Low Risk|3|0|0| +18810|F|U|4 yr Degree|4500|Low Risk|3|0|0| +18811|M|M|Advanced Degree|4500|Low Risk|3|0|0| +18812|F|M|Advanced Degree|4500|Low Risk|3|0|0| +18813|M|S|Advanced Degree|4500|Low Risk|3|0|0| +18814|F|S|Advanced Degree|4500|Low Risk|3|0|0| +18815|M|D|Advanced Degree|4500|Low Risk|3|0|0| +18816|F|D|Advanced Degree|4500|Low Risk|3|0|0| +18817|M|W|Advanced Degree|4500|Low Risk|3|0|0| +18818|F|W|Advanced Degree|4500|Low Risk|3|0|0| +18819|M|U|Advanced Degree|4500|Low Risk|3|0|0| +18820|F|U|Advanced Degree|4500|Low Risk|3|0|0| +18821|M|M|Unknown|4500|Low Risk|3|0|0| +18822|F|M|Unknown|4500|Low Risk|3|0|0| +18823|M|S|Unknown|4500|Low Risk|3|0|0| +18824|F|S|Unknown|4500|Low Risk|3|0|0| +18825|M|D|Unknown|4500|Low Risk|3|0|0| +18826|F|D|Unknown|4500|Low Risk|3|0|0| +18827|M|W|Unknown|4500|Low Risk|3|0|0| +18828|F|W|Unknown|4500|Low Risk|3|0|0| +18829|M|U|Unknown|4500|Low Risk|3|0|0| +18830|F|U|Unknown|4500|Low Risk|3|0|0| +18831|M|M|Primary|5000|Low Risk|3|0|0| +18832|F|M|Primary|5000|Low Risk|3|0|0| +18833|M|S|Primary|5000|Low Risk|3|0|0| +18834|F|S|Primary|5000|Low Risk|3|0|0| +18835|M|D|Primary|5000|Low Risk|3|0|0| +18836|F|D|Primary|5000|Low Risk|3|0|0| +18837|M|W|Primary|5000|Low Risk|3|0|0| +18838|F|W|Primary|5000|Low Risk|3|0|0| +18839|M|U|Primary|5000|Low Risk|3|0|0| +18840|F|U|Primary|5000|Low Risk|3|0|0| +18841|M|M|Secondary|5000|Low Risk|3|0|0| +18842|F|M|Secondary|5000|Low Risk|3|0|0| +18843|M|S|Secondary|5000|Low Risk|3|0|0| +18844|F|S|Secondary|5000|Low Risk|3|0|0| +18845|M|D|Secondary|5000|Low Risk|3|0|0| +18846|F|D|Secondary|5000|Low Risk|3|0|0| +18847|M|W|Secondary|5000|Low Risk|3|0|0| +18848|F|W|Secondary|5000|Low Risk|3|0|0| +18849|M|U|Secondary|5000|Low Risk|3|0|0| +18850|F|U|Secondary|5000|Low Risk|3|0|0| +18851|M|M|College|5000|Low Risk|3|0|0| +18852|F|M|College|5000|Low Risk|3|0|0| +18853|M|S|College|5000|Low Risk|3|0|0| +18854|F|S|College|5000|Low Risk|3|0|0| +18855|M|D|College|5000|Low Risk|3|0|0| +18856|F|D|College|5000|Low Risk|3|0|0| +18857|M|W|College|5000|Low Risk|3|0|0| +18858|F|W|College|5000|Low Risk|3|0|0| +18859|M|U|College|5000|Low Risk|3|0|0| +18860|F|U|College|5000|Low Risk|3|0|0| +18861|M|M|2 yr Degree|5000|Low Risk|3|0|0| +18862|F|M|2 yr Degree|5000|Low Risk|3|0|0| +18863|M|S|2 yr Degree|5000|Low Risk|3|0|0| +18864|F|S|2 yr Degree|5000|Low Risk|3|0|0| +18865|M|D|2 yr Degree|5000|Low Risk|3|0|0| +18866|F|D|2 yr Degree|5000|Low Risk|3|0|0| +18867|M|W|2 yr Degree|5000|Low Risk|3|0|0| +18868|F|W|2 yr Degree|5000|Low Risk|3|0|0| +18869|M|U|2 yr Degree|5000|Low Risk|3|0|0| +18870|F|U|2 yr Degree|5000|Low Risk|3|0|0| +18871|M|M|4 yr Degree|5000|Low Risk|3|0|0| +18872|F|M|4 yr Degree|5000|Low Risk|3|0|0| +18873|M|S|4 yr Degree|5000|Low Risk|3|0|0| +18874|F|S|4 yr Degree|5000|Low Risk|3|0|0| +18875|M|D|4 yr Degree|5000|Low Risk|3|0|0| +18876|F|D|4 yr Degree|5000|Low Risk|3|0|0| +18877|M|W|4 yr Degree|5000|Low Risk|3|0|0| +18878|F|W|4 yr Degree|5000|Low Risk|3|0|0| +18879|M|U|4 yr Degree|5000|Low Risk|3|0|0| +18880|F|U|4 yr Degree|5000|Low Risk|3|0|0| +18881|M|M|Advanced Degree|5000|Low Risk|3|0|0| +18882|F|M|Advanced Degree|5000|Low Risk|3|0|0| +18883|M|S|Advanced Degree|5000|Low Risk|3|0|0| +18884|F|S|Advanced Degree|5000|Low Risk|3|0|0| +18885|M|D|Advanced Degree|5000|Low Risk|3|0|0| +18886|F|D|Advanced Degree|5000|Low Risk|3|0|0| +18887|M|W|Advanced Degree|5000|Low Risk|3|0|0| +18888|F|W|Advanced Degree|5000|Low Risk|3|0|0| +18889|M|U|Advanced Degree|5000|Low Risk|3|0|0| +18890|F|U|Advanced Degree|5000|Low Risk|3|0|0| +18891|M|M|Unknown|5000|Low Risk|3|0|0| +18892|F|M|Unknown|5000|Low Risk|3|0|0| +18893|M|S|Unknown|5000|Low Risk|3|0|0| +18894|F|S|Unknown|5000|Low Risk|3|0|0| +18895|M|D|Unknown|5000|Low Risk|3|0|0| +18896|F|D|Unknown|5000|Low Risk|3|0|0| +18897|M|W|Unknown|5000|Low Risk|3|0|0| +18898|F|W|Unknown|5000|Low Risk|3|0|0| +18899|M|U|Unknown|5000|Low Risk|3|0|0| +18900|F|U|Unknown|5000|Low Risk|3|0|0| +18901|M|M|Primary|5500|Low Risk|3|0|0| +18902|F|M|Primary|5500|Low Risk|3|0|0| +18903|M|S|Primary|5500|Low Risk|3|0|0| +18904|F|S|Primary|5500|Low Risk|3|0|0| +18905|M|D|Primary|5500|Low Risk|3|0|0| +18906|F|D|Primary|5500|Low Risk|3|0|0| +18907|M|W|Primary|5500|Low Risk|3|0|0| +18908|F|W|Primary|5500|Low Risk|3|0|0| +18909|M|U|Primary|5500|Low Risk|3|0|0| +18910|F|U|Primary|5500|Low Risk|3|0|0| +18911|M|M|Secondary|5500|Low Risk|3|0|0| +18912|F|M|Secondary|5500|Low Risk|3|0|0| +18913|M|S|Secondary|5500|Low Risk|3|0|0| +18914|F|S|Secondary|5500|Low Risk|3|0|0| +18915|M|D|Secondary|5500|Low Risk|3|0|0| +18916|F|D|Secondary|5500|Low Risk|3|0|0| +18917|M|W|Secondary|5500|Low Risk|3|0|0| +18918|F|W|Secondary|5500|Low Risk|3|0|0| +18919|M|U|Secondary|5500|Low Risk|3|0|0| +18920|F|U|Secondary|5500|Low Risk|3|0|0| +18921|M|M|College|5500|Low Risk|3|0|0| +18922|F|M|College|5500|Low Risk|3|0|0| +18923|M|S|College|5500|Low Risk|3|0|0| +18924|F|S|College|5500|Low Risk|3|0|0| +18925|M|D|College|5500|Low Risk|3|0|0| +18926|F|D|College|5500|Low Risk|3|0|0| +18927|M|W|College|5500|Low Risk|3|0|0| +18928|F|W|College|5500|Low Risk|3|0|0| +18929|M|U|College|5500|Low Risk|3|0|0| +18930|F|U|College|5500|Low Risk|3|0|0| +18931|M|M|2 yr Degree|5500|Low Risk|3|0|0| +18932|F|M|2 yr Degree|5500|Low Risk|3|0|0| +18933|M|S|2 yr Degree|5500|Low Risk|3|0|0| +18934|F|S|2 yr Degree|5500|Low Risk|3|0|0| +18935|M|D|2 yr Degree|5500|Low Risk|3|0|0| +18936|F|D|2 yr Degree|5500|Low Risk|3|0|0| +18937|M|W|2 yr Degree|5500|Low Risk|3|0|0| +18938|F|W|2 yr Degree|5500|Low Risk|3|0|0| +18939|M|U|2 yr Degree|5500|Low Risk|3|0|0| +18940|F|U|2 yr Degree|5500|Low Risk|3|0|0| +18941|M|M|4 yr Degree|5500|Low Risk|3|0|0| +18942|F|M|4 yr Degree|5500|Low Risk|3|0|0| +18943|M|S|4 yr Degree|5500|Low Risk|3|0|0| +18944|F|S|4 yr Degree|5500|Low Risk|3|0|0| +18945|M|D|4 yr Degree|5500|Low Risk|3|0|0| +18946|F|D|4 yr Degree|5500|Low Risk|3|0|0| +18947|M|W|4 yr Degree|5500|Low Risk|3|0|0| +18948|F|W|4 yr Degree|5500|Low Risk|3|0|0| +18949|M|U|4 yr Degree|5500|Low Risk|3|0|0| +18950|F|U|4 yr Degree|5500|Low Risk|3|0|0| +18951|M|M|Advanced Degree|5500|Low Risk|3|0|0| +18952|F|M|Advanced Degree|5500|Low Risk|3|0|0| +18953|M|S|Advanced Degree|5500|Low Risk|3|0|0| +18954|F|S|Advanced Degree|5500|Low Risk|3|0|0| +18955|M|D|Advanced Degree|5500|Low Risk|3|0|0| +18956|F|D|Advanced Degree|5500|Low Risk|3|0|0| +18957|M|W|Advanced Degree|5500|Low Risk|3|0|0| +18958|F|W|Advanced Degree|5500|Low Risk|3|0|0| +18959|M|U|Advanced Degree|5500|Low Risk|3|0|0| +18960|F|U|Advanced Degree|5500|Low Risk|3|0|0| +18961|M|M|Unknown|5500|Low Risk|3|0|0| +18962|F|M|Unknown|5500|Low Risk|3|0|0| +18963|M|S|Unknown|5500|Low Risk|3|0|0| +18964|F|S|Unknown|5500|Low Risk|3|0|0| +18965|M|D|Unknown|5500|Low Risk|3|0|0| +18966|F|D|Unknown|5500|Low Risk|3|0|0| +18967|M|W|Unknown|5500|Low Risk|3|0|0| +18968|F|W|Unknown|5500|Low Risk|3|0|0| +18969|M|U|Unknown|5500|Low Risk|3|0|0| +18970|F|U|Unknown|5500|Low Risk|3|0|0| +18971|M|M|Primary|6000|Low Risk|3|0|0| +18972|F|M|Primary|6000|Low Risk|3|0|0| +18973|M|S|Primary|6000|Low Risk|3|0|0| +18974|F|S|Primary|6000|Low Risk|3|0|0| +18975|M|D|Primary|6000|Low Risk|3|0|0| +18976|F|D|Primary|6000|Low Risk|3|0|0| +18977|M|W|Primary|6000|Low Risk|3|0|0| +18978|F|W|Primary|6000|Low Risk|3|0|0| +18979|M|U|Primary|6000|Low Risk|3|0|0| +18980|F|U|Primary|6000|Low Risk|3|0|0| +18981|M|M|Secondary|6000|Low Risk|3|0|0| +18982|F|M|Secondary|6000|Low Risk|3|0|0| +18983|M|S|Secondary|6000|Low Risk|3|0|0| +18984|F|S|Secondary|6000|Low Risk|3|0|0| +18985|M|D|Secondary|6000|Low Risk|3|0|0| +18986|F|D|Secondary|6000|Low Risk|3|0|0| +18987|M|W|Secondary|6000|Low Risk|3|0|0| +18988|F|W|Secondary|6000|Low Risk|3|0|0| +18989|M|U|Secondary|6000|Low Risk|3|0|0| +18990|F|U|Secondary|6000|Low Risk|3|0|0| +18991|M|M|College|6000|Low Risk|3|0|0| +18992|F|M|College|6000|Low Risk|3|0|0| +18993|M|S|College|6000|Low Risk|3|0|0| +18994|F|S|College|6000|Low Risk|3|0|0| +18995|M|D|College|6000|Low Risk|3|0|0| +18996|F|D|College|6000|Low Risk|3|0|0| +18997|M|W|College|6000|Low Risk|3|0|0| +18998|F|W|College|6000|Low Risk|3|0|0| +18999|M|U|College|6000|Low Risk|3|0|0| +19000|F|U|College|6000|Low Risk|3|0|0| +19001|M|M|2 yr Degree|6000|Low Risk|3|0|0| +19002|F|M|2 yr Degree|6000|Low Risk|3|0|0| +19003|M|S|2 yr Degree|6000|Low Risk|3|0|0| +19004|F|S|2 yr Degree|6000|Low Risk|3|0|0| +19005|M|D|2 yr Degree|6000|Low Risk|3|0|0| +19006|F|D|2 yr Degree|6000|Low Risk|3|0|0| +19007|M|W|2 yr Degree|6000|Low Risk|3|0|0| +19008|F|W|2 yr Degree|6000|Low Risk|3|0|0| +19009|M|U|2 yr Degree|6000|Low Risk|3|0|0| +19010|F|U|2 yr Degree|6000|Low Risk|3|0|0| +19011|M|M|4 yr Degree|6000|Low Risk|3|0|0| +19012|F|M|4 yr Degree|6000|Low Risk|3|0|0| +19013|M|S|4 yr Degree|6000|Low Risk|3|0|0| +19014|F|S|4 yr Degree|6000|Low Risk|3|0|0| +19015|M|D|4 yr Degree|6000|Low Risk|3|0|0| +19016|F|D|4 yr Degree|6000|Low Risk|3|0|0| +19017|M|W|4 yr Degree|6000|Low Risk|3|0|0| +19018|F|W|4 yr Degree|6000|Low Risk|3|0|0| +19019|M|U|4 yr Degree|6000|Low Risk|3|0|0| +19020|F|U|4 yr Degree|6000|Low Risk|3|0|0| +19021|M|M|Advanced Degree|6000|Low Risk|3|0|0| +19022|F|M|Advanced Degree|6000|Low Risk|3|0|0| +19023|M|S|Advanced Degree|6000|Low Risk|3|0|0| +19024|F|S|Advanced Degree|6000|Low Risk|3|0|0| +19025|M|D|Advanced Degree|6000|Low Risk|3|0|0| +19026|F|D|Advanced Degree|6000|Low Risk|3|0|0| +19027|M|W|Advanced Degree|6000|Low Risk|3|0|0| +19028|F|W|Advanced Degree|6000|Low Risk|3|0|0| +19029|M|U|Advanced Degree|6000|Low Risk|3|0|0| +19030|F|U|Advanced Degree|6000|Low Risk|3|0|0| +19031|M|M|Unknown|6000|Low Risk|3|0|0| +19032|F|M|Unknown|6000|Low Risk|3|0|0| +19033|M|S|Unknown|6000|Low Risk|3|0|0| +19034|F|S|Unknown|6000|Low Risk|3|0|0| +19035|M|D|Unknown|6000|Low Risk|3|0|0| +19036|F|D|Unknown|6000|Low Risk|3|0|0| +19037|M|W|Unknown|6000|Low Risk|3|0|0| +19038|F|W|Unknown|6000|Low Risk|3|0|0| +19039|M|U|Unknown|6000|Low Risk|3|0|0| +19040|F|U|Unknown|6000|Low Risk|3|0|0| +19041|M|M|Primary|6500|Low Risk|3|0|0| +19042|F|M|Primary|6500|Low Risk|3|0|0| +19043|M|S|Primary|6500|Low Risk|3|0|0| +19044|F|S|Primary|6500|Low Risk|3|0|0| +19045|M|D|Primary|6500|Low Risk|3|0|0| +19046|F|D|Primary|6500|Low Risk|3|0|0| +19047|M|W|Primary|6500|Low Risk|3|0|0| +19048|F|W|Primary|6500|Low Risk|3|0|0| +19049|M|U|Primary|6500|Low Risk|3|0|0| +19050|F|U|Primary|6500|Low Risk|3|0|0| +19051|M|M|Secondary|6500|Low Risk|3|0|0| +19052|F|M|Secondary|6500|Low Risk|3|0|0| +19053|M|S|Secondary|6500|Low Risk|3|0|0| +19054|F|S|Secondary|6500|Low Risk|3|0|0| +19055|M|D|Secondary|6500|Low Risk|3|0|0| +19056|F|D|Secondary|6500|Low Risk|3|0|0| +19057|M|W|Secondary|6500|Low Risk|3|0|0| +19058|F|W|Secondary|6500|Low Risk|3|0|0| +19059|M|U|Secondary|6500|Low Risk|3|0|0| +19060|F|U|Secondary|6500|Low Risk|3|0|0| +19061|M|M|College|6500|Low Risk|3|0|0| +19062|F|M|College|6500|Low Risk|3|0|0| +19063|M|S|College|6500|Low Risk|3|0|0| +19064|F|S|College|6500|Low Risk|3|0|0| +19065|M|D|College|6500|Low Risk|3|0|0| +19066|F|D|College|6500|Low Risk|3|0|0| +19067|M|W|College|6500|Low Risk|3|0|0| +19068|F|W|College|6500|Low Risk|3|0|0| +19069|M|U|College|6500|Low Risk|3|0|0| +19070|F|U|College|6500|Low Risk|3|0|0| +19071|M|M|2 yr Degree|6500|Low Risk|3|0|0| +19072|F|M|2 yr Degree|6500|Low Risk|3|0|0| +19073|M|S|2 yr Degree|6500|Low Risk|3|0|0| +19074|F|S|2 yr Degree|6500|Low Risk|3|0|0| +19075|M|D|2 yr Degree|6500|Low Risk|3|0|0| +19076|F|D|2 yr Degree|6500|Low Risk|3|0|0| +19077|M|W|2 yr Degree|6500|Low Risk|3|0|0| +19078|F|W|2 yr Degree|6500|Low Risk|3|0|0| +19079|M|U|2 yr Degree|6500|Low Risk|3|0|0| +19080|F|U|2 yr Degree|6500|Low Risk|3|0|0| +19081|M|M|4 yr Degree|6500|Low Risk|3|0|0| +19082|F|M|4 yr Degree|6500|Low Risk|3|0|0| +19083|M|S|4 yr Degree|6500|Low Risk|3|0|0| +19084|F|S|4 yr Degree|6500|Low Risk|3|0|0| +19085|M|D|4 yr Degree|6500|Low Risk|3|0|0| +19086|F|D|4 yr Degree|6500|Low Risk|3|0|0| +19087|M|W|4 yr Degree|6500|Low Risk|3|0|0| +19088|F|W|4 yr Degree|6500|Low Risk|3|0|0| +19089|M|U|4 yr Degree|6500|Low Risk|3|0|0| +19090|F|U|4 yr Degree|6500|Low Risk|3|0|0| +19091|M|M|Advanced Degree|6500|Low Risk|3|0|0| +19092|F|M|Advanced Degree|6500|Low Risk|3|0|0| +19093|M|S|Advanced Degree|6500|Low Risk|3|0|0| +19094|F|S|Advanced Degree|6500|Low Risk|3|0|0| +19095|M|D|Advanced Degree|6500|Low Risk|3|0|0| +19096|F|D|Advanced Degree|6500|Low Risk|3|0|0| +19097|M|W|Advanced Degree|6500|Low Risk|3|0|0| +19098|F|W|Advanced Degree|6500|Low Risk|3|0|0| +19099|M|U|Advanced Degree|6500|Low Risk|3|0|0| +19100|F|U|Advanced Degree|6500|Low Risk|3|0|0| +19101|M|M|Unknown|6500|Low Risk|3|0|0| +19102|F|M|Unknown|6500|Low Risk|3|0|0| +19103|M|S|Unknown|6500|Low Risk|3|0|0| +19104|F|S|Unknown|6500|Low Risk|3|0|0| +19105|M|D|Unknown|6500|Low Risk|3|0|0| +19106|F|D|Unknown|6500|Low Risk|3|0|0| +19107|M|W|Unknown|6500|Low Risk|3|0|0| +19108|F|W|Unknown|6500|Low Risk|3|0|0| +19109|M|U|Unknown|6500|Low Risk|3|0|0| +19110|F|U|Unknown|6500|Low Risk|3|0|0| +19111|M|M|Primary|7000|Low Risk|3|0|0| +19112|F|M|Primary|7000|Low Risk|3|0|0| +19113|M|S|Primary|7000|Low Risk|3|0|0| +19114|F|S|Primary|7000|Low Risk|3|0|0| +19115|M|D|Primary|7000|Low Risk|3|0|0| +19116|F|D|Primary|7000|Low Risk|3|0|0| +19117|M|W|Primary|7000|Low Risk|3|0|0| +19118|F|W|Primary|7000|Low Risk|3|0|0| +19119|M|U|Primary|7000|Low Risk|3|0|0| +19120|F|U|Primary|7000|Low Risk|3|0|0| +19121|M|M|Secondary|7000|Low Risk|3|0|0| +19122|F|M|Secondary|7000|Low Risk|3|0|0| +19123|M|S|Secondary|7000|Low Risk|3|0|0| +19124|F|S|Secondary|7000|Low Risk|3|0|0| +19125|M|D|Secondary|7000|Low Risk|3|0|0| +19126|F|D|Secondary|7000|Low Risk|3|0|0| +19127|M|W|Secondary|7000|Low Risk|3|0|0| +19128|F|W|Secondary|7000|Low Risk|3|0|0| +19129|M|U|Secondary|7000|Low Risk|3|0|0| +19130|F|U|Secondary|7000|Low Risk|3|0|0| +19131|M|M|College|7000|Low Risk|3|0|0| +19132|F|M|College|7000|Low Risk|3|0|0| +19133|M|S|College|7000|Low Risk|3|0|0| +19134|F|S|College|7000|Low Risk|3|0|0| +19135|M|D|College|7000|Low Risk|3|0|0| +19136|F|D|College|7000|Low Risk|3|0|0| +19137|M|W|College|7000|Low Risk|3|0|0| +19138|F|W|College|7000|Low Risk|3|0|0| +19139|M|U|College|7000|Low Risk|3|0|0| +19140|F|U|College|7000|Low Risk|3|0|0| +19141|M|M|2 yr Degree|7000|Low Risk|3|0|0| +19142|F|M|2 yr Degree|7000|Low Risk|3|0|0| +19143|M|S|2 yr Degree|7000|Low Risk|3|0|0| +19144|F|S|2 yr Degree|7000|Low Risk|3|0|0| +19145|M|D|2 yr Degree|7000|Low Risk|3|0|0| +19146|F|D|2 yr Degree|7000|Low Risk|3|0|0| +19147|M|W|2 yr Degree|7000|Low Risk|3|0|0| +19148|F|W|2 yr Degree|7000|Low Risk|3|0|0| +19149|M|U|2 yr Degree|7000|Low Risk|3|0|0| +19150|F|U|2 yr Degree|7000|Low Risk|3|0|0| +19151|M|M|4 yr Degree|7000|Low Risk|3|0|0| +19152|F|M|4 yr Degree|7000|Low Risk|3|0|0| +19153|M|S|4 yr Degree|7000|Low Risk|3|0|0| +19154|F|S|4 yr Degree|7000|Low Risk|3|0|0| +19155|M|D|4 yr Degree|7000|Low Risk|3|0|0| +19156|F|D|4 yr Degree|7000|Low Risk|3|0|0| +19157|M|W|4 yr Degree|7000|Low Risk|3|0|0| +19158|F|W|4 yr Degree|7000|Low Risk|3|0|0| +19159|M|U|4 yr Degree|7000|Low Risk|3|0|0| +19160|F|U|4 yr Degree|7000|Low Risk|3|0|0| +19161|M|M|Advanced Degree|7000|Low Risk|3|0|0| +19162|F|M|Advanced Degree|7000|Low Risk|3|0|0| +19163|M|S|Advanced Degree|7000|Low Risk|3|0|0| +19164|F|S|Advanced Degree|7000|Low Risk|3|0|0| +19165|M|D|Advanced Degree|7000|Low Risk|3|0|0| +19166|F|D|Advanced Degree|7000|Low Risk|3|0|0| +19167|M|W|Advanced Degree|7000|Low Risk|3|0|0| +19168|F|W|Advanced Degree|7000|Low Risk|3|0|0| +19169|M|U|Advanced Degree|7000|Low Risk|3|0|0| +19170|F|U|Advanced Degree|7000|Low Risk|3|0|0| +19171|M|M|Unknown|7000|Low Risk|3|0|0| +19172|F|M|Unknown|7000|Low Risk|3|0|0| +19173|M|S|Unknown|7000|Low Risk|3|0|0| +19174|F|S|Unknown|7000|Low Risk|3|0|0| +19175|M|D|Unknown|7000|Low Risk|3|0|0| +19176|F|D|Unknown|7000|Low Risk|3|0|0| +19177|M|W|Unknown|7000|Low Risk|3|0|0| +19178|F|W|Unknown|7000|Low Risk|3|0|0| +19179|M|U|Unknown|7000|Low Risk|3|0|0| +19180|F|U|Unknown|7000|Low Risk|3|0|0| +19181|M|M|Primary|7500|Low Risk|3|0|0| +19182|F|M|Primary|7500|Low Risk|3|0|0| +19183|M|S|Primary|7500|Low Risk|3|0|0| +19184|F|S|Primary|7500|Low Risk|3|0|0| +19185|M|D|Primary|7500|Low Risk|3|0|0| +19186|F|D|Primary|7500|Low Risk|3|0|0| +19187|M|W|Primary|7500|Low Risk|3|0|0| +19188|F|W|Primary|7500|Low Risk|3|0|0| +19189|M|U|Primary|7500|Low Risk|3|0|0| +19190|F|U|Primary|7500|Low Risk|3|0|0| +19191|M|M|Secondary|7500|Low Risk|3|0|0| +19192|F|M|Secondary|7500|Low Risk|3|0|0| +19193|M|S|Secondary|7500|Low Risk|3|0|0| +19194|F|S|Secondary|7500|Low Risk|3|0|0| +19195|M|D|Secondary|7500|Low Risk|3|0|0| +19196|F|D|Secondary|7500|Low Risk|3|0|0| +19197|M|W|Secondary|7500|Low Risk|3|0|0| +19198|F|W|Secondary|7500|Low Risk|3|0|0| +19199|M|U|Secondary|7500|Low Risk|3|0|0| +19200|F|U|Secondary|7500|Low Risk|3|0|0| +19201|M|M|College|7500|Low Risk|3|0|0| +19202|F|M|College|7500|Low Risk|3|0|0| +19203|M|S|College|7500|Low Risk|3|0|0| +19204|F|S|College|7500|Low Risk|3|0|0| +19205|M|D|College|7500|Low Risk|3|0|0| +19206|F|D|College|7500|Low Risk|3|0|0| +19207|M|W|College|7500|Low Risk|3|0|0| +19208|F|W|College|7500|Low Risk|3|0|0| diff --git a/spark-connector/spark-common/src/test/resources/data/tpcds/date_dim/date_dim.dat b/spark-connector/spark-common/src/test/resources/data/tpcds/date_dim/date_dim.dat new file mode 100644 index 00000000000..123acaa1cc9 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/data/tpcds/date_dim/date_dim.dat @@ -0,0 +1,73049 @@ +2415022|AAAAAAAAOKJNECAA|1900-01-02|0|1|1|1900|1|1|2|1|1900|1|1|Monday|1900Q1|N|N|Y|2415021|2415020|2414657|2414930|N|N|N|N|N| +2415023|AAAAAAAAPKJNECAA|1900-01-03|0|1|1|1900|2|1|3|1|1900|1|1|Tuesday|1900Q1|N|N|N|2415021|2415020|2414658|2414931|N|N|N|N|N| +2415024|AAAAAAAAALJNECAA|1900-01-04|0|1|1|1900|3|1|4|1|1900|1|1|Wednesday|1900Q1|N|N|N|2415021|2415020|2414659|2414932|N|N|N|N|N| +2415025|AAAAAAAABLJNECAA|1900-01-05|0|1|1|1900|4|1|5|1|1900|1|1|Thursday|1900Q1|N|N|N|2415021|2415020|2414660|2414933|N|N|N|N|N| +2415026|AAAAAAAACLJNECAA|1900-01-06|0|1|1|1900|5|1|6|1|1900|1|1|Friday|1900Q1|N|Y|N|2415021|2415020|2414661|2414934|N|N|N|N|N| +2415027|AAAAAAAADLJNECAA|1900-01-07|0|1|1|1900|6|1|7|1|1900|1|1|Saturday|1900Q1|N|Y|N|2415021|2415020|2414662|2414935|N|N|N|N|N| +2415028|AAAAAAAAELJNECAA|1900-01-08|0|1|1|1900|0|1|8|1|1900|1|1|Sunday|1900Q1|N|N|N|2415021|2415020|2414663|2414936|N|N|N|N|N| +2415029|AAAAAAAAFLJNECAA|1900-01-09|0|2|1|1900|1|1|9|1|1900|1|2|Monday|1900Q1|N|N|N|2415021|2415020|2414664|2414937|N|N|N|N|N| +2415030|AAAAAAAAGLJNECAA|1900-01-10|0|2|1|1900|2|1|10|1|1900|1|2|Tuesday|1900Q1|N|N|N|2415021|2415020|2414665|2414938|N|N|N|N|N| +2415031|AAAAAAAAHLJNECAA|1900-01-11|0|2|1|1900|3|1|11|1|1900|1|2|Wednesday|1900Q1|N|N|N|2415021|2415020|2414666|2414939|N|N|N|N|N| +2415032|AAAAAAAAILJNECAA|1900-01-12|0|2|1|1900|4|1|12|1|1900|1|2|Thursday|1900Q1|N|N|N|2415021|2415020|2414667|2414940|N|N|N|N|N| +2415033|AAAAAAAAJLJNECAA|1900-01-13|0|2|1|1900|5|1|13|1|1900|1|2|Friday|1900Q1|N|Y|N|2415021|2415020|2414668|2414941|N|N|N|N|N| +2415034|AAAAAAAAKLJNECAA|1900-01-14|0|2|1|1900|6|1|14|1|1900|1|2|Saturday|1900Q1|N|Y|N|2415021|2415020|2414669|2414942|N|N|N|N|N| +2415035|AAAAAAAALLJNECAA|1900-01-15|0|2|1|1900|0|1|15|1|1900|1|2|Sunday|1900Q1|N|N|N|2415021|2415020|2414670|2414943|N|N|N|N|N| +2415036|AAAAAAAAMLJNECAA|1900-01-16|0|3|1|1900|1|1|16|1|1900|1|3|Monday|1900Q1|N|N|N|2415021|2415020|2414671|2414944|N|N|N|N|N| +2415037|AAAAAAAANLJNECAA|1900-01-17|0|3|1|1900|2|1|17|1|1900|1|3|Tuesday|1900Q1|N|N|N|2415021|2415020|2414672|2414945|N|N|N|N|N| +2415038|AAAAAAAAOLJNECAA|1900-01-18|0|3|1|1900|3|1|18|1|1900|1|3|Wednesday|1900Q1|N|N|N|2415021|2415020|2414673|2414946|N|N|N|N|N| +2415039|AAAAAAAAPLJNECAA|1900-01-19|0|3|1|1900|4|1|19|1|1900|1|3|Thursday|1900Q1|N|N|N|2415021|2415020|2414674|2414947|N|N|N|N|N| +2415040|AAAAAAAAAMJNECAA|1900-01-20|0|3|1|1900|5|1|20|1|1900|1|3|Friday|1900Q1|N|Y|N|2415021|2415020|2414675|2414948|N|N|N|N|N| +2415041|AAAAAAAABMJNECAA|1900-01-21|0|3|1|1900|6|1|21|1|1900|1|3|Saturday|1900Q1|N|Y|N|2415021|2415020|2414676|2414949|N|N|N|N|N| +2415042|AAAAAAAACMJNECAA|1900-01-22|0|3|1|1900|0|1|22|1|1900|1|3|Sunday|1900Q1|N|N|N|2415021|2415020|2414677|2414950|N|N|N|N|N| +2415043|AAAAAAAADMJNECAA|1900-01-23|0|4|1|1900|1|1|23|1|1900|1|4|Monday|1900Q1|N|N|N|2415021|2415020|2414678|2414951|N|N|N|N|N| +2415044|AAAAAAAAEMJNECAA|1900-01-24|0|4|1|1900|2|1|24|1|1900|1|4|Tuesday|1900Q1|N|N|N|2415021|2415020|2414679|2414952|N|N|N|N|N| +2415045|AAAAAAAAFMJNECAA|1900-01-25|0|4|1|1900|3|1|25|1|1900|1|4|Wednesday|1900Q1|N|N|N|2415021|2415020|2414680|2414953|N|N|N|N|N| +2415046|AAAAAAAAGMJNECAA|1900-01-26|0|4|1|1900|4|1|26|1|1900|1|4|Thursday|1900Q1|N|N|N|2415021|2415020|2414681|2414954|N|N|N|N|N| +2415047|AAAAAAAAHMJNECAA|1900-01-27|0|4|1|1900|5|1|27|1|1900|1|4|Friday|1900Q1|N|Y|N|2415021|2415020|2414682|2414955|N|N|N|N|N| +2415048|AAAAAAAAIMJNECAA|1900-01-28|0|4|1|1900|6|1|28|1|1900|1|4|Saturday|1900Q1|N|Y|N|2415021|2415020|2414683|2414956|N|N|N|N|N| +2415049|AAAAAAAAJMJNECAA|1900-01-29|0|4|1|1900|0|1|29|1|1900|1|4|Sunday|1900Q1|N|N|N|2415021|2415020|2414684|2414957|N|N|N|N|N| +2415050|AAAAAAAAKMJNECAA|1900-01-30|0|5|1|1900|1|1|30|1|1900|1|5|Monday|1900Q1|N|N|N|2415021|2415020|2414685|2414958|N|N|N|N|N| +2415051|AAAAAAAALMJNECAA|1900-01-31|0|5|1|1900|2|1|31|1|1900|1|5|Tuesday|1900Q1|N|N|N|2415021|2415020|2414686|2414959|N|N|N|N|N| +2415052|AAAAAAAAMMJNECAA|1900-02-01|1|5|1|1900|3|2|1|1|1900|1|5|Wednesday|1900Q1|N|N|N|2415052|2415082|2414687|2414960|N|N|N|N|N| +2415053|AAAAAAAANMJNECAA|1900-02-02|1|5|1|1900|4|2|2|1|1900|1|5|Thursday|1900Q1|N|N|N|2415052|2415082|2414688|2414961|N|N|N|N|N| +2415054|AAAAAAAAOMJNECAA|1900-02-03|1|5|1|1900|5|2|3|1|1900|1|5|Friday|1900Q1|N|Y|N|2415052|2415082|2414689|2414962|N|N|N|N|N| +2415055|AAAAAAAAPMJNECAA|1900-02-04|1|5|1|1900|6|2|4|1|1900|1|5|Saturday|1900Q1|N|Y|N|2415052|2415082|2414690|2414963|N|N|N|N|N| +2415056|AAAAAAAAANJNECAA|1900-02-05|1|5|1|1900|0|2|5|1|1900|1|5|Sunday|1900Q1|N|N|N|2415052|2415082|2414691|2414964|N|N|N|N|N| +2415057|AAAAAAAABNJNECAA|1900-02-06|1|6|1|1900|1|2|6|1|1900|1|6|Monday|1900Q1|N|N|N|2415052|2415082|2414692|2414965|N|N|N|N|N| +2415058|AAAAAAAACNJNECAA|1900-02-07|1|6|1|1900|2|2|7|1|1900|1|6|Tuesday|1900Q1|N|N|N|2415052|2415082|2414693|2414966|N|N|N|N|N| +2415059|AAAAAAAADNJNECAA|1900-02-08|1|6|1|1900|3|2|8|1|1900|1|6|Wednesday|1900Q1|N|N|N|2415052|2415082|2414694|2414967|N|N|N|N|N| +2415060|AAAAAAAAENJNECAA|1900-02-09|1|6|1|1900|4|2|9|1|1900|1|6|Thursday|1900Q1|N|N|N|2415052|2415082|2414695|2414968|N|N|N|N|N| +2415061|AAAAAAAAFNJNECAA|1900-02-10|1|6|1|1900|5|2|10|1|1900|1|6|Friday|1900Q1|N|Y|N|2415052|2415082|2414696|2414969|N|N|N|N|N| +2415062|AAAAAAAAGNJNECAA|1900-02-11|1|6|1|1900|6|2|11|1|1900|1|6|Saturday|1900Q1|N|Y|N|2415052|2415082|2414697|2414970|N|N|N|N|N| +2415063|AAAAAAAAHNJNECAA|1900-02-12|1|6|1|1900|0|2|12|1|1900|1|6|Sunday|1900Q1|N|N|N|2415052|2415082|2414698|2414971|N|N|N|N|N| +2415064|AAAAAAAAINJNECAA|1900-02-13|1|7|1|1900|1|2|13|1|1900|1|7|Monday|1900Q1|N|N|N|2415052|2415082|2414699|2414972|N|N|N|N|N| +2415065|AAAAAAAAJNJNECAA|1900-02-14|1|7|1|1900|2|2|14|1|1900|1|7|Tuesday|1900Q1|N|N|N|2415052|2415082|2414700|2414973|N|N|N|N|N| +2415066|AAAAAAAAKNJNECAA|1900-02-15|1|7|1|1900|3|2|15|1|1900|1|7|Wednesday|1900Q1|N|N|N|2415052|2415082|2414701|2414974|N|N|N|N|N| +2415067|AAAAAAAALNJNECAA|1900-02-16|1|7|1|1900|4|2|16|1|1900|1|7|Thursday|1900Q1|N|N|N|2415052|2415082|2414702|2414975|N|N|N|N|N| +2415068|AAAAAAAAMNJNECAA|1900-02-17|1|7|1|1900|5|2|17|1|1900|1|7|Friday|1900Q1|N|Y|N|2415052|2415082|2414703|2414976|N|N|N|N|N| +2415069|AAAAAAAANNJNECAA|1900-02-18|1|7|1|1900|6|2|18|1|1900|1|7|Saturday|1900Q1|N|Y|N|2415052|2415082|2414704|2414977|N|N|N|N|N| +2415070|AAAAAAAAONJNECAA|1900-02-19|1|7|1|1900|0|2|19|1|1900|1|7|Sunday|1900Q1|N|N|N|2415052|2415082|2414705|2414978|N|N|N|N|N| +2415071|AAAAAAAAPNJNECAA|1900-02-20|1|8|1|1900|1|2|20|1|1900|1|8|Monday|1900Q1|N|N|N|2415052|2415082|2414706|2414979|N|N|N|N|N| +2415072|AAAAAAAAAOJNECAA|1900-02-21|1|8|1|1900|2|2|21|1|1900|1|8|Tuesday|1900Q1|N|N|N|2415052|2415082|2414707|2414980|N|N|N|N|N| +2415073|AAAAAAAABOJNECAA|1900-02-22|1|8|1|1900|3|2|22|1|1900|1|8|Wednesday|1900Q1|N|N|N|2415052|2415082|2414708|2414981|N|N|N|N|N| +2415074|AAAAAAAACOJNECAA|1900-02-23|1|8|1|1900|4|2|23|1|1900|1|8|Thursday|1900Q1|N|N|N|2415052|2415082|2414709|2414982|N|N|N|N|N| +2415075|AAAAAAAADOJNECAA|1900-02-24|1|8|1|1900|5|2|24|1|1900|1|8|Friday|1900Q1|N|Y|N|2415052|2415082|2414710|2414983|N|N|N|N|N| +2415076|AAAAAAAAEOJNECAA|1900-02-25|1|8|1|1900|6|2|25|1|1900|1|8|Saturday|1900Q1|N|Y|N|2415052|2415082|2414711|2414984|N|N|N|N|N| +2415077|AAAAAAAAFOJNECAA|1900-02-26|1|8|1|1900|0|2|26|1|1900|1|8|Sunday|1900Q1|N|N|N|2415052|2415082|2414712|2414985|N|N|N|N|N| +2415078|AAAAAAAAGOJNECAA|1900-02-27|1|9|1|1900|1|2|27|1|1900|1|9|Monday|1900Q1|N|N|N|2415052|2415082|2414713|2414986|N|N|N|N|N| +2415079|AAAAAAAAHOJNECAA|1900-02-28|1|9|1|1900|2|2|28|1|1900|1|9|Tuesday|1900Q1|N|N|N|2415052|2415082|2414714|2414987|N|N|N|N|N| +2415080|AAAAAAAAIOJNECAA|1900-03-01|2|9|2|1900|4|3|1|1|1900|2|9|Thursday|1900Q1|N|N|N|2415080|2415139|2414715|2414988|N|N|N|N|N| +2415081|AAAAAAAAJOJNECAA|1900-03-02|2|9|2|1900|5|3|2|1|1900|2|9|Friday|1900Q1|N|Y|N|2415080|2415139|2414716|2414989|N|N|N|N|N| +2415082|AAAAAAAAKOJNECAA|1900-03-03|2|9|2|1900|6|3|3|1|1900|2|9|Saturday|1900Q1|N|Y|N|2415080|2415139|2414717|2414990|N|N|N|N|N| +2415083|AAAAAAAALOJNECAA|1900-03-04|2|9|2|1900|0|3|4|1|1900|2|9|Sunday|1900Q1|N|N|N|2415080|2415139|2414718|2414991|N|N|N|N|N| +2415084|AAAAAAAAMOJNECAA|1900-03-05|2|9|2|1900|1|3|5|1|1900|2|9|Monday|1900Q1|N|N|N|2415080|2415139|2414719|2414992|N|N|N|N|N| +2415085|AAAAAAAANOJNECAA|1900-03-06|2|10|2|1900|2|3|6|1|1900|2|10|Tuesday|1900Q1|N|N|N|2415080|2415139|2414720|2414993|N|N|N|N|N| +2415086|AAAAAAAAOOJNECAA|1900-03-07|2|10|2|1900|3|3|7|1|1900|2|10|Wednesday|1900Q1|N|N|N|2415080|2415139|2414721|2414994|N|N|N|N|N| +2415087|AAAAAAAAPOJNECAA|1900-03-08|2|10|2|1900|4|3|8|1|1900|2|10|Thursday|1900Q1|N|N|N|2415080|2415139|2414722|2414995|N|N|N|N|N| +2415088|AAAAAAAAAPJNECAA|1900-03-09|2|10|2|1900|5|3|9|1|1900|2|10|Friday|1900Q1|N|Y|N|2415080|2415139|2414723|2414996|N|N|N|N|N| +2415089|AAAAAAAABPJNECAA|1900-03-10|2|10|2|1900|6|3|10|1|1900|2|10|Saturday|1900Q1|N|Y|N|2415080|2415139|2414724|2414997|N|N|N|N|N| +2415090|AAAAAAAACPJNECAA|1900-03-11|2|10|2|1900|0|3|11|1|1900|2|10|Sunday|1900Q1|N|N|N|2415080|2415139|2414725|2414998|N|N|N|N|N| +2415091|AAAAAAAADPJNECAA|1900-03-12|2|10|2|1900|1|3|12|1|1900|2|10|Monday|1900Q1|N|N|N|2415080|2415139|2414726|2414999|N|N|N|N|N| +2415092|AAAAAAAAEPJNECAA|1900-03-13|2|11|2|1900|2|3|13|1|1900|2|11|Tuesday|1900Q1|N|N|N|2415080|2415139|2414727|2415000|N|N|N|N|N| +2415093|AAAAAAAAFPJNECAA|1900-03-14|2|11|2|1900|3|3|14|1|1900|2|11|Wednesday|1900Q1|N|N|N|2415080|2415139|2414728|2415001|N|N|N|N|N| +2415094|AAAAAAAAGPJNECAA|1900-03-15|2|11|2|1900|4|3|15|1|1900|2|11|Thursday|1900Q1|N|N|N|2415080|2415139|2414729|2415002|N|N|N|N|N| +2415095|AAAAAAAAHPJNECAA|1900-03-16|2|11|2|1900|5|3|16|1|1900|2|11|Friday|1900Q1|N|Y|N|2415080|2415139|2414730|2415003|N|N|N|N|N| +2415096|AAAAAAAAIPJNECAA|1900-03-17|2|11|2|1900|6|3|17|1|1900|2|11|Saturday|1900Q1|N|Y|N|2415080|2415139|2414731|2415004|N|N|N|N|N| +2415097|AAAAAAAAJPJNECAA|1900-03-18|2|11|2|1900|0|3|18|1|1900|2|11|Sunday|1900Q1|N|N|N|2415080|2415139|2414732|2415005|N|N|N|N|N| +2415098|AAAAAAAAKPJNECAA|1900-03-19|2|11|2|1900|1|3|19|1|1900|2|11|Monday|1900Q1|N|N|N|2415080|2415139|2414733|2415006|N|N|N|N|N| +2415099|AAAAAAAALPJNECAA|1900-03-20|2|12|2|1900|2|3|20|1|1900|2|12|Tuesday|1900Q1|N|N|N|2415080|2415139|2414734|2415007|N|N|N|N|N| +2415100|AAAAAAAAMPJNECAA|1900-03-21|2|12|2|1900|3|3|21|1|1900|2|12|Wednesday|1900Q1|N|N|N|2415080|2415139|2414735|2415008|N|N|N|N|N| +2415101|AAAAAAAANPJNECAA|1900-03-22|2|12|2|1900|4|3|22|1|1900|2|12|Thursday|1900Q1|N|N|N|2415080|2415139|2414736|2415009|N|N|N|N|N| +2415102|AAAAAAAAOPJNECAA|1900-03-23|2|12|2|1900|5|3|23|1|1900|2|12|Friday|1900Q1|N|Y|N|2415080|2415139|2414737|2415010|N|N|N|N|N| +2415103|AAAAAAAAPPJNECAA|1900-03-24|2|12|2|1900|6|3|24|1|1900|2|12|Saturday|1900Q1|N|Y|N|2415080|2415139|2414738|2415011|N|N|N|N|N| +2415104|AAAAAAAAAAKNECAA|1900-03-25|2|12|2|1900|0|3|25|1|1900|2|12|Sunday|1900Q1|N|N|N|2415080|2415139|2414739|2415012|N|N|N|N|N| +2415105|AAAAAAAABAKNECAA|1900-03-26|2|12|2|1900|1|3|26|1|1900|2|12|Monday|1900Q1|N|N|N|2415080|2415139|2414740|2415013|N|N|N|N|N| +2415106|AAAAAAAACAKNECAA|1900-03-27|2|13|2|1900|2|3|27|1|1900|2|13|Tuesday|1900Q1|N|N|N|2415080|2415139|2414741|2415014|N|N|N|N|N| +2415107|AAAAAAAADAKNECAA|1900-03-28|2|13|2|1900|3|3|28|1|1900|2|13|Wednesday|1900Q1|N|N|N|2415080|2415139|2414742|2415015|N|N|N|N|N| +2415108|AAAAAAAAEAKNECAA|1900-03-29|2|13|2|1900|4|3|29|1|1900|2|13|Thursday|1900Q1|N|N|N|2415080|2415139|2414743|2415016|N|N|N|N|N| +2415109|AAAAAAAAFAKNECAA|1900-03-30|2|13|2|1900|5|3|30|1|1900|2|13|Friday|1900Q1|N|Y|N|2415080|2415139|2414744|2415017|N|N|N|N|N| +2415110|AAAAAAAAGAKNECAA|1900-03-31|2|13|2|1900|6|3|31|1|1900|2|13|Saturday|1900Q1|N|Y|N|2415080|2415139|2414745|2415018|N|N|N|N|N| +2415111|AAAAAAAAHAKNECAA|1900-04-01|3|13|2|1900|0|4|1|2|1900|2|13|Sunday|1900Q2|N|N|N|2415111|2415201|2414746|2415021|N|N|N|N|N| +2415112|AAAAAAAAIAKNECAA|1900-04-02|3|13|2|1900|1|4|2|2|1900|2|13|Monday|1900Q2|N|N|N|2415111|2415201|2414747|2415022|N|N|N|N|N| +2415113|AAAAAAAAJAKNECAA|1900-04-03|3|14|2|1900|2|4|3|2|1900|2|14|Tuesday|1900Q2|N|N|N|2415111|2415201|2414748|2415023|N|N|N|N|N| +2415114|AAAAAAAAKAKNECAA|1900-04-04|3|14|2|1900|3|4|4|2|1900|2|14|Wednesday|1900Q2|N|N|N|2415111|2415201|2414749|2415024|N|N|N|N|N| +2415115|AAAAAAAALAKNECAA|1900-04-05|3|14|2|1900|4|4|5|2|1900|2|14|Thursday|1900Q2|N|N|N|2415111|2415201|2414750|2415025|N|N|N|N|N| +2415116|AAAAAAAAMAKNECAA|1900-04-06|3|14|2|1900|5|4|6|2|1900|2|14|Friday|1900Q2|N|Y|N|2415111|2415201|2414751|2415026|N|N|N|N|N| +2415117|AAAAAAAANAKNECAA|1900-04-07|3|14|2|1900|6|4|7|2|1900|2|14|Saturday|1900Q2|N|Y|N|2415111|2415201|2414752|2415027|N|N|N|N|N| +2415118|AAAAAAAAOAKNECAA|1900-04-08|3|14|2|1900|0|4|8|2|1900|2|14|Sunday|1900Q2|N|N|N|2415111|2415201|2414753|2415028|N|N|N|N|N| +2415119|AAAAAAAAPAKNECAA|1900-04-09|3|14|2|1900|1|4|9|2|1900|2|14|Monday|1900Q2|N|N|N|2415111|2415201|2414754|2415029|N|N|N|N|N| +2415120|AAAAAAAAABKNECAA|1900-04-10|3|15|2|1900|2|4|10|2|1900|2|15|Tuesday|1900Q2|N|N|N|2415111|2415201|2414755|2415030|N|N|N|N|N| +2415121|AAAAAAAABBKNECAA|1900-04-11|3|15|2|1900|3|4|11|2|1900|2|15|Wednesday|1900Q2|N|N|N|2415111|2415201|2414756|2415031|N|N|N|N|N| +2415122|AAAAAAAACBKNECAA|1900-04-12|3|15|2|1900|4|4|12|2|1900|2|15|Thursday|1900Q2|N|N|N|2415111|2415201|2414757|2415032|N|N|N|N|N| +2415123|AAAAAAAADBKNECAA|1900-04-13|3|15|2|1900|5|4|13|2|1900|2|15|Friday|1900Q2|N|Y|N|2415111|2415201|2414758|2415033|N|N|N|N|N| +2415124|AAAAAAAAEBKNECAA|1900-04-14|3|15|2|1900|6|4|14|2|1900|2|15|Saturday|1900Q2|N|Y|N|2415111|2415201|2414759|2415034|N|N|N|N|N| +2415125|AAAAAAAAFBKNECAA|1900-04-15|3|15|2|1900|0|4|15|2|1900|2|15|Sunday|1900Q2|N|N|N|2415111|2415201|2414760|2415035|N|N|N|N|N| +2415126|AAAAAAAAGBKNECAA|1900-04-16|3|15|2|1900|1|4|16|2|1900|2|15|Monday|1900Q2|N|N|N|2415111|2415201|2414761|2415036|N|N|N|N|N| +2415127|AAAAAAAAHBKNECAA|1900-04-17|3|16|2|1900|2|4|17|2|1900|2|16|Tuesday|1900Q2|N|N|N|2415111|2415201|2414762|2415037|N|N|N|N|N| +2415128|AAAAAAAAIBKNECAA|1900-04-18|3|16|2|1900|3|4|18|2|1900|2|16|Wednesday|1900Q2|N|N|N|2415111|2415201|2414763|2415038|N|N|N|N|N| +2415129|AAAAAAAAJBKNECAA|1900-04-19|3|16|2|1900|4|4|19|2|1900|2|16|Thursday|1900Q2|N|N|N|2415111|2415201|2414764|2415039|N|N|N|N|N| +2415130|AAAAAAAAKBKNECAA|1900-04-20|3|16|2|1900|5|4|20|2|1900|2|16|Friday|1900Q2|N|Y|N|2415111|2415201|2414765|2415040|N|N|N|N|N| +2415131|AAAAAAAALBKNECAA|1900-04-21|3|16|2|1900|6|4|21|2|1900|2|16|Saturday|1900Q2|N|Y|N|2415111|2415201|2414766|2415041|N|N|N|N|N| +2415132|AAAAAAAAMBKNECAA|1900-04-22|3|16|2|1900|0|4|22|2|1900|2|16|Sunday|1900Q2|N|N|N|2415111|2415201|2414767|2415042|N|N|N|N|N| +2415133|AAAAAAAANBKNECAA|1900-04-23|3|16|2|1900|1|4|23|2|1900|2|16|Monday|1900Q2|N|N|N|2415111|2415201|2414768|2415043|N|N|N|N|N| +2415134|AAAAAAAAOBKNECAA|1900-04-24|3|17|2|1900|2|4|24|2|1900|2|17|Tuesday|1900Q2|N|N|N|2415111|2415201|2414769|2415044|N|N|N|N|N| +2415135|AAAAAAAAPBKNECAA|1900-04-25|3|17|2|1900|3|4|25|2|1900|2|17|Wednesday|1900Q2|N|N|N|2415111|2415201|2414770|2415045|N|N|N|N|N| +2415136|AAAAAAAAACKNECAA|1900-04-26|3|17|2|1900|4|4|26|2|1900|2|17|Thursday|1900Q2|N|N|N|2415111|2415201|2414771|2415046|N|N|N|N|N| +2415137|AAAAAAAABCKNECAA|1900-04-27|3|17|2|1900|5|4|27|2|1900|2|17|Friday|1900Q2|N|Y|N|2415111|2415201|2414772|2415047|N|N|N|N|N| +2415138|AAAAAAAACCKNECAA|1900-04-28|3|17|2|1900|6|4|28|2|1900|2|17|Saturday|1900Q2|N|Y|N|2415111|2415201|2414773|2415048|N|N|N|N|N| +2415139|AAAAAAAADCKNECAA|1900-04-29|3|17|2|1900|0|4|29|2|1900|2|17|Sunday|1900Q2|N|N|N|2415111|2415201|2414774|2415049|N|N|N|N|N| +2415140|AAAAAAAAECKNECAA|1900-04-30|3|17|2|1900|1|4|30|2|1900|2|17|Monday|1900Q2|N|N|N|2415111|2415201|2414775|2415050|N|N|N|N|N| +2415141|AAAAAAAAFCKNECAA|1900-05-01|4|18|2|1900|2|5|1|2|1900|2|18|Tuesday|1900Q2|N|N|N|2415141|2415261|2414776|2415051|N|N|N|N|N| +2415142|AAAAAAAAGCKNECAA|1900-05-02|4|18|2|1900|3|5|2|2|1900|2|18|Wednesday|1900Q2|N|N|N|2415141|2415261|2414777|2415052|N|N|N|N|N| +2415143|AAAAAAAAHCKNECAA|1900-05-03|4|18|2|1900|4|5|3|2|1900|2|18|Thursday|1900Q2|N|N|N|2415141|2415261|2414778|2415053|N|N|N|N|N| +2415144|AAAAAAAAICKNECAA|1900-05-04|4|18|2|1900|5|5|4|2|1900|2|18|Friday|1900Q2|N|Y|N|2415141|2415261|2414779|2415054|N|N|N|N|N| +2415145|AAAAAAAAJCKNECAA|1900-05-05|4|18|2|1900|6|5|5|2|1900|2|18|Saturday|1900Q2|N|Y|N|2415141|2415261|2414780|2415055|N|N|N|N|N| +2415146|AAAAAAAAKCKNECAA|1900-05-06|4|18|2|1900|0|5|6|2|1900|2|18|Sunday|1900Q2|N|N|N|2415141|2415261|2414781|2415056|N|N|N|N|N| +2415147|AAAAAAAALCKNECAA|1900-05-07|4|18|2|1900|1|5|7|2|1900|2|18|Monday|1900Q2|N|N|N|2415141|2415261|2414782|2415057|N|N|N|N|N| +2415148|AAAAAAAAMCKNECAA|1900-05-08|4|19|2|1900|2|5|8|2|1900|2|19|Tuesday|1900Q2|N|N|N|2415141|2415261|2414783|2415058|N|N|N|N|N| +2415149|AAAAAAAANCKNECAA|1900-05-09|4|19|2|1900|3|5|9|2|1900|2|19|Wednesday|1900Q2|N|N|N|2415141|2415261|2414784|2415059|N|N|N|N|N| +2415150|AAAAAAAAOCKNECAA|1900-05-10|4|19|2|1900|4|5|10|2|1900|2|19|Thursday|1900Q2|N|N|N|2415141|2415261|2414785|2415060|N|N|N|N|N| +2415151|AAAAAAAAPCKNECAA|1900-05-11|4|19|2|1900|5|5|11|2|1900|2|19|Friday|1900Q2|N|Y|N|2415141|2415261|2414786|2415061|N|N|N|N|N| +2415152|AAAAAAAAADKNECAA|1900-05-12|4|19|2|1900|6|5|12|2|1900|2|19|Saturday|1900Q2|N|Y|N|2415141|2415261|2414787|2415062|N|N|N|N|N| +2415153|AAAAAAAABDKNECAA|1900-05-13|4|19|2|1900|0|5|13|2|1900|2|19|Sunday|1900Q2|N|N|N|2415141|2415261|2414788|2415063|N|N|N|N|N| +2415154|AAAAAAAACDKNECAA|1900-05-14|4|19|2|1900|1|5|14|2|1900|2|19|Monday|1900Q2|N|N|N|2415141|2415261|2414789|2415064|N|N|N|N|N| +2415155|AAAAAAAADDKNECAA|1900-05-15|4|20|2|1900|2|5|15|2|1900|2|20|Tuesday|1900Q2|N|N|N|2415141|2415261|2414790|2415065|N|N|N|N|N| +2415156|AAAAAAAAEDKNECAA|1900-05-16|4|20|2|1900|3|5|16|2|1900|2|20|Wednesday|1900Q2|N|N|N|2415141|2415261|2414791|2415066|N|N|N|N|N| +2415157|AAAAAAAAFDKNECAA|1900-05-17|4|20|2|1900|4|5|17|2|1900|2|20|Thursday|1900Q2|N|N|N|2415141|2415261|2414792|2415067|N|N|N|N|N| +2415158|AAAAAAAAGDKNECAA|1900-05-18|4|20|2|1900|5|5|18|2|1900|2|20|Friday|1900Q2|N|Y|N|2415141|2415261|2414793|2415068|N|N|N|N|N| +2415159|AAAAAAAAHDKNECAA|1900-05-19|4|20|2|1900|6|5|19|2|1900|2|20|Saturday|1900Q2|N|Y|N|2415141|2415261|2414794|2415069|N|N|N|N|N| +2415160|AAAAAAAAIDKNECAA|1900-05-20|4|20|2|1900|0|5|20|2|1900|2|20|Sunday|1900Q2|N|N|N|2415141|2415261|2414795|2415070|N|N|N|N|N| +2415161|AAAAAAAAJDKNECAA|1900-05-21|4|20|2|1900|1|5|21|2|1900|2|20|Monday|1900Q2|N|N|N|2415141|2415261|2414796|2415071|N|N|N|N|N| +2415162|AAAAAAAAKDKNECAA|1900-05-22|4|21|2|1900|2|5|22|2|1900|2|21|Tuesday|1900Q2|N|N|N|2415141|2415261|2414797|2415072|N|N|N|N|N| +2415163|AAAAAAAALDKNECAA|1900-05-23|4|21|2|1900|3|5|23|2|1900|2|21|Wednesday|1900Q2|N|N|N|2415141|2415261|2414798|2415073|N|N|N|N|N| +2415164|AAAAAAAAMDKNECAA|1900-05-24|4|21|2|1900|4|5|24|2|1900|2|21|Thursday|1900Q2|N|N|N|2415141|2415261|2414799|2415074|N|N|N|N|N| +2415165|AAAAAAAANDKNECAA|1900-05-25|4|21|2|1900|5|5|25|2|1900|2|21|Friday|1900Q2|N|Y|N|2415141|2415261|2414800|2415075|N|N|N|N|N| +2415166|AAAAAAAAODKNECAA|1900-05-26|4|21|2|1900|6|5|26|2|1900|2|21|Saturday|1900Q2|N|Y|N|2415141|2415261|2414801|2415076|N|N|N|N|N| +2415167|AAAAAAAAPDKNECAA|1900-05-27|4|21|2|1900|0|5|27|2|1900|2|21|Sunday|1900Q2|N|N|N|2415141|2415261|2414802|2415077|N|N|N|N|N| +2415168|AAAAAAAAAEKNECAA|1900-05-28|4|21|2|1900|1|5|28|2|1900|2|21|Monday|1900Q2|N|N|N|2415141|2415261|2414803|2415078|N|N|N|N|N| +2415169|AAAAAAAABEKNECAA|1900-05-29|4|22|2|1900|2|5|29|2|1900|2|22|Tuesday|1900Q2|N|N|N|2415141|2415261|2414804|2415079|N|N|N|N|N| +2415170|AAAAAAAACEKNECAA|1900-05-30|4|22|2|1900|3|5|30|2|1900|2|22|Wednesday|1900Q2|N|N|N|2415141|2415261|2414805|2415080|N|N|N|N|N| +2415171|AAAAAAAADEKNECAA|1900-05-31|4|22|2|1900|4|5|31|2|1900|2|22|Thursday|1900Q2|N|N|N|2415141|2415261|2414806|2415081|N|N|N|N|N| +2415172|AAAAAAAAEEKNECAA|1900-06-01|5|22|3|1900|5|6|1|2|1900|3|22|Friday|1900Q2|N|Y|N|2415172|2415323|2414807|2415082|N|N|N|N|N| +2415173|AAAAAAAAFEKNECAA|1900-06-02|5|22|3|1900|6|6|2|2|1900|3|22|Saturday|1900Q2|N|Y|N|2415172|2415323|2414808|2415083|N|N|N|N|N| +2415174|AAAAAAAAGEKNECAA|1900-06-03|5|22|3|1900|0|6|3|2|1900|3|22|Sunday|1900Q2|N|N|N|2415172|2415323|2414809|2415084|N|N|N|N|N| +2415175|AAAAAAAAHEKNECAA|1900-06-04|5|22|3|1900|1|6|4|2|1900|3|22|Monday|1900Q2|N|N|N|2415172|2415323|2414810|2415085|N|N|N|N|N| +2415176|AAAAAAAAIEKNECAA|1900-06-05|5|23|3|1900|2|6|5|2|1900|3|23|Tuesday|1900Q2|N|N|N|2415172|2415323|2414811|2415086|N|N|N|N|N| +2415177|AAAAAAAAJEKNECAA|1900-06-06|5|23|3|1900|3|6|6|2|1900|3|23|Wednesday|1900Q2|N|N|N|2415172|2415323|2414812|2415087|N|N|N|N|N| +2415178|AAAAAAAAKEKNECAA|1900-06-07|5|23|3|1900|4|6|7|2|1900|3|23|Thursday|1900Q2|N|N|N|2415172|2415323|2414813|2415088|N|N|N|N|N| +2415179|AAAAAAAALEKNECAA|1900-06-08|5|23|3|1900|5|6|8|2|1900|3|23|Friday|1900Q2|N|Y|N|2415172|2415323|2414814|2415089|N|N|N|N|N| +2415180|AAAAAAAAMEKNECAA|1900-06-09|5|23|3|1900|6|6|9|2|1900|3|23|Saturday|1900Q2|N|Y|N|2415172|2415323|2414815|2415090|N|N|N|N|N| +2415181|AAAAAAAANEKNECAA|1900-06-10|5|23|3|1900|0|6|10|2|1900|3|23|Sunday|1900Q2|N|N|N|2415172|2415323|2414816|2415091|N|N|N|N|N| +2415182|AAAAAAAAOEKNECAA|1900-06-11|5|23|3|1900|1|6|11|2|1900|3|23|Monday|1900Q2|N|N|N|2415172|2415323|2414817|2415092|N|N|N|N|N| +2415183|AAAAAAAAPEKNECAA|1900-06-12|5|24|3|1900|2|6|12|2|1900|3|24|Tuesday|1900Q2|N|N|N|2415172|2415323|2414818|2415093|N|N|N|N|N| +2415184|AAAAAAAAAFKNECAA|1900-06-13|5|24|3|1900|3|6|13|2|1900|3|24|Wednesday|1900Q2|N|N|N|2415172|2415323|2414819|2415094|N|N|N|N|N| +2415185|AAAAAAAABFKNECAA|1900-06-14|5|24|3|1900|4|6|14|2|1900|3|24|Thursday|1900Q2|N|N|N|2415172|2415323|2414820|2415095|N|N|N|N|N| +2415186|AAAAAAAACFKNECAA|1900-06-15|5|24|3|1900|5|6|15|2|1900|3|24|Friday|1900Q2|N|Y|N|2415172|2415323|2414821|2415096|N|N|N|N|N| +2415187|AAAAAAAADFKNECAA|1900-06-16|5|24|3|1900|6|6|16|2|1900|3|24|Saturday|1900Q2|N|Y|N|2415172|2415323|2414822|2415097|N|N|N|N|N| +2415188|AAAAAAAAEFKNECAA|1900-06-17|5|24|3|1900|0|6|17|2|1900|3|24|Sunday|1900Q2|N|N|N|2415172|2415323|2414823|2415098|N|N|N|N|N| +2415189|AAAAAAAAFFKNECAA|1900-06-18|5|24|3|1900|1|6|18|2|1900|3|24|Monday|1900Q2|N|N|N|2415172|2415323|2414824|2415099|N|N|N|N|N| +2415190|AAAAAAAAGFKNECAA|1900-06-19|5|25|3|1900|2|6|19|2|1900|3|25|Tuesday|1900Q2|N|N|N|2415172|2415323|2414825|2415100|N|N|N|N|N| +2415191|AAAAAAAAHFKNECAA|1900-06-20|5|25|3|1900|3|6|20|2|1900|3|25|Wednesday|1900Q2|N|N|N|2415172|2415323|2414826|2415101|N|N|N|N|N| +2415192|AAAAAAAAIFKNECAA|1900-06-21|5|25|3|1900|4|6|21|2|1900|3|25|Thursday|1900Q2|N|N|N|2415172|2415323|2414827|2415102|N|N|N|N|N| +2415193|AAAAAAAAJFKNECAA|1900-06-22|5|25|3|1900|5|6|22|2|1900|3|25|Friday|1900Q2|N|Y|N|2415172|2415323|2414828|2415103|N|N|N|N|N| +2415194|AAAAAAAAKFKNECAA|1900-06-23|5|25|3|1900|6|6|23|2|1900|3|25|Saturday|1900Q2|N|Y|N|2415172|2415323|2414829|2415104|N|N|N|N|N| +2415195|AAAAAAAALFKNECAA|1900-06-24|5|25|3|1900|0|6|24|2|1900|3|25|Sunday|1900Q2|N|N|N|2415172|2415323|2414830|2415105|N|N|N|N|N| +2415196|AAAAAAAAMFKNECAA|1900-06-25|5|25|3|1900|1|6|25|2|1900|3|25|Monday|1900Q2|N|N|N|2415172|2415323|2414831|2415106|N|N|N|N|N| +2415197|AAAAAAAANFKNECAA|1900-06-26|5|26|3|1900|2|6|26|2|1900|3|26|Tuesday|1900Q2|N|N|N|2415172|2415323|2414832|2415107|N|N|N|N|N| +2415198|AAAAAAAAOFKNECAA|1900-06-27|5|26|3|1900|3|6|27|2|1900|3|26|Wednesday|1900Q2|N|N|N|2415172|2415323|2414833|2415108|N|N|N|N|N| +2415199|AAAAAAAAPFKNECAA|1900-06-28|5|26|3|1900|4|6|28|2|1900|3|26|Thursday|1900Q2|N|N|N|2415172|2415323|2414834|2415109|N|N|N|N|N| +2415200|AAAAAAAAAGKNECAA|1900-06-29|5|26|3|1900|5|6|29|2|1900|3|26|Friday|1900Q2|N|Y|N|2415172|2415323|2414835|2415110|N|N|N|N|N| +2415201|AAAAAAAABGKNECAA|1900-06-30|5|26|3|1900|6|6|30|2|1900|3|26|Saturday|1900Q2|N|Y|N|2415172|2415323|2414836|2415111|N|N|N|N|N| +2415202|AAAAAAAACGKNECAA|1900-07-01|6|26|3|1900|0|7|1|3|1900|3|26|Sunday|1900Q3|N|N|N|2415202|2415383|2414837|2415111|N|N|N|N|N| +2415203|AAAAAAAADGKNECAA|1900-07-02|6|26|3|1900|1|7|2|3|1900|3|26|Monday|1900Q3|N|N|N|2415202|2415383|2414838|2415112|N|N|N|N|N| +2415204|AAAAAAAAEGKNECAA|1900-07-03|6|27|3|1900|2|7|3|3|1900|3|27|Tuesday|1900Q3|N|N|N|2415202|2415383|2414839|2415113|N|N|N|N|N| +2415205|AAAAAAAAFGKNECAA|1900-07-04|6|27|3|1900|3|7|4|3|1900|3|27|Wednesday|1900Q3|Y|N|N|2415202|2415383|2414840|2415114|N|N|N|N|N| +2415206|AAAAAAAAGGKNECAA|1900-07-05|6|27|3|1900|4|7|5|3|1900|3|27|Thursday|1900Q3|N|N|Y|2415202|2415383|2414841|2415115|N|N|N|N|N| +2415207|AAAAAAAAHGKNECAA|1900-07-06|6|27|3|1900|5|7|6|3|1900|3|27|Friday|1900Q3|N|Y|N|2415202|2415383|2414842|2415116|N|N|N|N|N| +2415208|AAAAAAAAIGKNECAA|1900-07-07|6|27|3|1900|6|7|7|3|1900|3|27|Saturday|1900Q3|N|Y|N|2415202|2415383|2414843|2415117|N|N|N|N|N| +2415209|AAAAAAAAJGKNECAA|1900-07-08|6|27|3|1900|0|7|8|3|1900|3|27|Sunday|1900Q3|N|N|N|2415202|2415383|2414844|2415118|N|N|N|N|N| +2415210|AAAAAAAAKGKNECAA|1900-07-09|6|27|3|1900|1|7|9|3|1900|3|27|Monday|1900Q3|N|N|N|2415202|2415383|2414845|2415119|N|N|N|N|N| +2415211|AAAAAAAALGKNECAA|1900-07-10|6|28|3|1900|2|7|10|3|1900|3|28|Tuesday|1900Q3|N|N|N|2415202|2415383|2414846|2415120|N|N|N|N|N| +2415212|AAAAAAAAMGKNECAA|1900-07-11|6|28|3|1900|3|7|11|3|1900|3|28|Wednesday|1900Q3|N|N|N|2415202|2415383|2414847|2415121|N|N|N|N|N| +2415213|AAAAAAAANGKNECAA|1900-07-12|6|28|3|1900|4|7|12|3|1900|3|28|Thursday|1900Q3|N|N|N|2415202|2415383|2414848|2415122|N|N|N|N|N| +2415214|AAAAAAAAOGKNECAA|1900-07-13|6|28|3|1900|5|7|13|3|1900|3|28|Friday|1900Q3|N|Y|N|2415202|2415383|2414849|2415123|N|N|N|N|N| +2415215|AAAAAAAAPGKNECAA|1900-07-14|6|28|3|1900|6|7|14|3|1900|3|28|Saturday|1900Q3|N|Y|N|2415202|2415383|2414850|2415124|N|N|N|N|N| +2415216|AAAAAAAAAHKNECAA|1900-07-15|6|28|3|1900|0|7|15|3|1900|3|28|Sunday|1900Q3|N|N|N|2415202|2415383|2414851|2415125|N|N|N|N|N| +2415217|AAAAAAAABHKNECAA|1900-07-16|6|28|3|1900|1|7|16|3|1900|3|28|Monday|1900Q3|N|N|N|2415202|2415383|2414852|2415126|N|N|N|N|N| +2415218|AAAAAAAACHKNECAA|1900-07-17|6|29|3|1900|2|7|17|3|1900|3|29|Tuesday|1900Q3|N|N|N|2415202|2415383|2414853|2415127|N|N|N|N|N| +2415219|AAAAAAAADHKNECAA|1900-07-18|6|29|3|1900|3|7|18|3|1900|3|29|Wednesday|1900Q3|N|N|N|2415202|2415383|2414854|2415128|N|N|N|N|N| +2415220|AAAAAAAAEHKNECAA|1900-07-19|6|29|3|1900|4|7|19|3|1900|3|29|Thursday|1900Q3|N|N|N|2415202|2415383|2414855|2415129|N|N|N|N|N| +2415221|AAAAAAAAFHKNECAA|1900-07-20|6|29|3|1900|5|7|20|3|1900|3|29|Friday|1900Q3|N|Y|N|2415202|2415383|2414856|2415130|N|N|N|N|N| +2415222|AAAAAAAAGHKNECAA|1900-07-21|6|29|3|1900|6|7|21|3|1900|3|29|Saturday|1900Q3|N|Y|N|2415202|2415383|2414857|2415131|N|N|N|N|N| +2415223|AAAAAAAAHHKNECAA|1900-07-22|6|29|3|1900|0|7|22|3|1900|3|29|Sunday|1900Q3|N|N|N|2415202|2415383|2414858|2415132|N|N|N|N|N| +2415224|AAAAAAAAIHKNECAA|1900-07-23|6|29|3|1900|1|7|23|3|1900|3|29|Monday|1900Q3|N|N|N|2415202|2415383|2414859|2415133|N|N|N|N|N| +2415225|AAAAAAAAJHKNECAA|1900-07-24|6|30|3|1900|2|7|24|3|1900|3|30|Tuesday|1900Q3|N|N|N|2415202|2415383|2414860|2415134|N|N|N|N|N| +2415226|AAAAAAAAKHKNECAA|1900-07-25|6|30|3|1900|3|7|25|3|1900|3|30|Wednesday|1900Q3|N|N|N|2415202|2415383|2414861|2415135|N|N|N|N|N| +2415227|AAAAAAAALHKNECAA|1900-07-26|6|30|3|1900|4|7|26|3|1900|3|30|Thursday|1900Q3|N|N|N|2415202|2415383|2414862|2415136|N|N|N|N|N| +2415228|AAAAAAAAMHKNECAA|1900-07-27|6|30|3|1900|5|7|27|3|1900|3|30|Friday|1900Q3|N|Y|N|2415202|2415383|2414863|2415137|N|N|N|N|N| +2415229|AAAAAAAANHKNECAA|1900-07-28|6|30|3|1900|6|7|28|3|1900|3|30|Saturday|1900Q3|N|Y|N|2415202|2415383|2414864|2415138|N|N|N|N|N| +2415230|AAAAAAAAOHKNECAA|1900-07-29|6|30|3|1900|0|7|29|3|1900|3|30|Sunday|1900Q3|N|N|N|2415202|2415383|2414865|2415139|N|N|N|N|N| +2415231|AAAAAAAAPHKNECAA|1900-07-30|6|30|3|1900|1|7|30|3|1900|3|30|Monday|1900Q3|N|N|N|2415202|2415383|2414866|2415140|N|N|N|N|N| +2415232|AAAAAAAAAIKNECAA|1900-07-31|6|31|3|1900|2|7|31|3|1900|3|31|Tuesday|1900Q3|N|N|N|2415202|2415383|2414867|2415141|N|N|N|N|N| +2415233|AAAAAAAABIKNECAA|1900-08-01|7|31|3|1900|3|8|1|3|1900|3|31|Wednesday|1900Q3|N|N|N|2415233|2415445|2414868|2415142|N|N|N|N|N| +2415234|AAAAAAAACIKNECAA|1900-08-02|7|31|3|1900|4|8|2|3|1900|3|31|Thursday|1900Q3|N|N|N|2415233|2415445|2414869|2415143|N|N|N|N|N| +2415235|AAAAAAAADIKNECAA|1900-08-03|7|31|3|1900|5|8|3|3|1900|3|31|Friday|1900Q3|N|Y|N|2415233|2415445|2414870|2415144|N|N|N|N|N| +2415236|AAAAAAAAEIKNECAA|1900-08-04|7|31|3|1900|6|8|4|3|1900|3|31|Saturday|1900Q3|N|Y|N|2415233|2415445|2414871|2415145|N|N|N|N|N| +2415237|AAAAAAAAFIKNECAA|1900-08-05|7|31|3|1900|0|8|5|3|1900|3|31|Sunday|1900Q3|N|N|N|2415233|2415445|2414872|2415146|N|N|N|N|N| +2415238|AAAAAAAAGIKNECAA|1900-08-06|7|31|3|1900|1|8|6|3|1900|3|31|Monday|1900Q3|N|N|N|2415233|2415445|2414873|2415147|N|N|N|N|N| +2415239|AAAAAAAAHIKNECAA|1900-08-07|7|32|3|1900|2|8|7|3|1900|3|32|Tuesday|1900Q3|N|N|N|2415233|2415445|2414874|2415148|N|N|N|N|N| +2415240|AAAAAAAAIIKNECAA|1900-08-08|7|32|3|1900|3|8|8|3|1900|3|32|Wednesday|1900Q3|N|N|N|2415233|2415445|2414875|2415149|N|N|N|N|N| +2415241|AAAAAAAAJIKNECAA|1900-08-09|7|32|3|1900|4|8|9|3|1900|3|32|Thursday|1900Q3|N|N|N|2415233|2415445|2414876|2415150|N|N|N|N|N| +2415242|AAAAAAAAKIKNECAA|1900-08-10|7|32|3|1900|5|8|10|3|1900|3|32|Friday|1900Q3|N|Y|N|2415233|2415445|2414877|2415151|N|N|N|N|N| +2415243|AAAAAAAALIKNECAA|1900-08-11|7|32|3|1900|6|8|11|3|1900|3|32|Saturday|1900Q3|N|Y|N|2415233|2415445|2414878|2415152|N|N|N|N|N| +2415244|AAAAAAAAMIKNECAA|1900-08-12|7|32|3|1900|0|8|12|3|1900|3|32|Sunday|1900Q3|N|N|N|2415233|2415445|2414879|2415153|N|N|N|N|N| +2415245|AAAAAAAANIKNECAA|1900-08-13|7|32|3|1900|1|8|13|3|1900|3|32|Monday|1900Q3|N|N|N|2415233|2415445|2414880|2415154|N|N|N|N|N| +2415246|AAAAAAAAOIKNECAA|1900-08-14|7|33|3|1900|2|8|14|3|1900|3|33|Tuesday|1900Q3|N|N|N|2415233|2415445|2414881|2415155|N|N|N|N|N| +2415247|AAAAAAAAPIKNECAA|1900-08-15|7|33|3|1900|3|8|15|3|1900|3|33|Wednesday|1900Q3|N|N|N|2415233|2415445|2414882|2415156|N|N|N|N|N| +2415248|AAAAAAAAAJKNECAA|1900-08-16|7|33|3|1900|4|8|16|3|1900|3|33|Thursday|1900Q3|N|N|N|2415233|2415445|2414883|2415157|N|N|N|N|N| +2415249|AAAAAAAABJKNECAA|1900-08-17|7|33|3|1900|5|8|17|3|1900|3|33|Friday|1900Q3|N|Y|N|2415233|2415445|2414884|2415158|N|N|N|N|N| +2415250|AAAAAAAACJKNECAA|1900-08-18|7|33|3|1900|6|8|18|3|1900|3|33|Saturday|1900Q3|N|Y|N|2415233|2415445|2414885|2415159|N|N|N|N|N| +2415251|AAAAAAAADJKNECAA|1900-08-19|7|33|3|1900|0|8|19|3|1900|3|33|Sunday|1900Q3|N|N|N|2415233|2415445|2414886|2415160|N|N|N|N|N| +2415252|AAAAAAAAEJKNECAA|1900-08-20|7|33|3|1900|1|8|20|3|1900|3|33|Monday|1900Q3|N|N|N|2415233|2415445|2414887|2415161|N|N|N|N|N| +2415253|AAAAAAAAFJKNECAA|1900-08-21|7|34|3|1900|2|8|21|3|1900|3|34|Tuesday|1900Q3|N|N|N|2415233|2415445|2414888|2415162|N|N|N|N|N| +2415254|AAAAAAAAGJKNECAA|1900-08-22|7|34|3|1900|3|8|22|3|1900|3|34|Wednesday|1900Q3|N|N|N|2415233|2415445|2414889|2415163|N|N|N|N|N| +2415255|AAAAAAAAHJKNECAA|1900-08-23|7|34|3|1900|4|8|23|3|1900|3|34|Thursday|1900Q3|N|N|N|2415233|2415445|2414890|2415164|N|N|N|N|N| +2415256|AAAAAAAAIJKNECAA|1900-08-24|7|34|3|1900|5|8|24|3|1900|3|34|Friday|1900Q3|N|Y|N|2415233|2415445|2414891|2415165|N|N|N|N|N| +2415257|AAAAAAAAJJKNECAA|1900-08-25|7|34|3|1900|6|8|25|3|1900|3|34|Saturday|1900Q3|N|Y|N|2415233|2415445|2414892|2415166|N|N|N|N|N| +2415258|AAAAAAAAKJKNECAA|1900-08-26|7|34|3|1900|0|8|26|3|1900|3|34|Sunday|1900Q3|N|N|N|2415233|2415445|2414893|2415167|N|N|N|N|N| +2415259|AAAAAAAALJKNECAA|1900-08-27|7|34|3|1900|1|8|27|3|1900|3|34|Monday|1900Q3|N|N|N|2415233|2415445|2414894|2415168|N|N|N|N|N| +2415260|AAAAAAAAMJKNECAA|1900-08-28|7|35|3|1900|2|8|28|3|1900|3|35|Tuesday|1900Q3|N|N|N|2415233|2415445|2414895|2415169|N|N|N|N|N| +2415261|AAAAAAAANJKNECAA|1900-08-29|7|35|3|1900|3|8|29|3|1900|3|35|Wednesday|1900Q3|N|N|N|2415233|2415445|2414896|2415170|N|N|N|N|N| +2415262|AAAAAAAAOJKNECAA|1900-08-30|7|35|3|1900|4|8|30|3|1900|3|35|Thursday|1900Q3|N|N|N|2415233|2415445|2414897|2415171|N|N|N|N|N| +2415263|AAAAAAAAPJKNECAA|1900-08-31|7|35|3|1900|5|8|31|3|1900|3|35|Friday|1900Q3|N|Y|N|2415233|2415445|2414898|2415172|N|N|N|N|N| +2415264|AAAAAAAAAKKNECAA|1900-09-01|8|35|4|1900|6|9|1|3|1900|4|35|Saturday|1900Q3|N|Y|N|2415264|2415507|2414899|2415173|N|N|N|N|N| +2415265|AAAAAAAABKKNECAA|1900-09-02|8|35|4|1900|0|9|2|3|1900|4|35|Sunday|1900Q3|N|N|N|2415264|2415507|2414900|2415174|N|N|N|N|N| +2415266|AAAAAAAACKKNECAA|1900-09-03|8|35|4|1900|1|9|3|3|1900|4|35|Monday|1900Q3|N|N|N|2415264|2415507|2414901|2415175|N|N|N|N|N| +2415267|AAAAAAAADKKNECAA|1900-09-04|8|36|4|1900|2|9|4|3|1900|4|36|Tuesday|1900Q3|N|N|N|2415264|2415507|2414902|2415176|N|N|N|N|N| +2415268|AAAAAAAAEKKNECAA|1900-09-05|8|36|4|1900|3|9|5|3|1900|4|36|Wednesday|1900Q3|N|N|N|2415264|2415507|2414903|2415177|N|N|N|N|N| +2415269|AAAAAAAAFKKNECAA|1900-09-06|8|36|4|1900|4|9|6|3|1900|4|36|Thursday|1900Q3|N|N|N|2415264|2415507|2414904|2415178|N|N|N|N|N| +2415270|AAAAAAAAGKKNECAA|1900-09-07|8|36|4|1900|5|9|7|3|1900|4|36|Friday|1900Q3|N|Y|N|2415264|2415507|2414905|2415179|N|N|N|N|N| +2415271|AAAAAAAAHKKNECAA|1900-09-08|8|36|4|1900|6|9|8|3|1900|4|36|Saturday|1900Q3|N|Y|N|2415264|2415507|2414906|2415180|N|N|N|N|N| +2415272|AAAAAAAAIKKNECAA|1900-09-09|8|36|4|1900|0|9|9|3|1900|4|36|Sunday|1900Q3|N|N|N|2415264|2415507|2414907|2415181|N|N|N|N|N| +2415273|AAAAAAAAJKKNECAA|1900-09-10|8|36|4|1900|1|9|10|3|1900|4|36|Monday|1900Q3|N|N|N|2415264|2415507|2414908|2415182|N|N|N|N|N| +2415274|AAAAAAAAKKKNECAA|1900-09-11|8|37|4|1900|2|9|11|3|1900|4|37|Tuesday|1900Q3|N|N|N|2415264|2415507|2414909|2415183|N|N|N|N|N| +2415275|AAAAAAAALKKNECAA|1900-09-12|8|37|4|1900|3|9|12|3|1900|4|37|Wednesday|1900Q3|N|N|N|2415264|2415507|2414910|2415184|N|N|N|N|N| +2415276|AAAAAAAAMKKNECAA|1900-09-13|8|37|4|1900|4|9|13|3|1900|4|37|Thursday|1900Q3|N|N|N|2415264|2415507|2414911|2415185|N|N|N|N|N| +2415277|AAAAAAAANKKNECAA|1900-09-14|8|37|4|1900|5|9|14|3|1900|4|37|Friday|1900Q3|N|Y|N|2415264|2415507|2414912|2415186|N|N|N|N|N| +2415278|AAAAAAAAOKKNECAA|1900-09-15|8|37|4|1900|6|9|15|3|1900|4|37|Saturday|1900Q3|N|Y|N|2415264|2415507|2414913|2415187|N|N|N|N|N| +2415279|AAAAAAAAPKKNECAA|1900-09-16|8|37|4|1900|0|9|16|3|1900|4|37|Sunday|1900Q3|N|N|N|2415264|2415507|2414914|2415188|N|N|N|N|N| +2415280|AAAAAAAAALKNECAA|1900-09-17|8|37|4|1900|1|9|17|3|1900|4|37|Monday|1900Q3|N|N|N|2415264|2415507|2414915|2415189|N|N|N|N|N| +2415281|AAAAAAAABLKNECAA|1900-09-18|8|38|4|1900|2|9|18|3|1900|4|38|Tuesday|1900Q3|N|N|N|2415264|2415507|2414916|2415190|N|N|N|N|N| +2415282|AAAAAAAACLKNECAA|1900-09-19|8|38|4|1900|3|9|19|3|1900|4|38|Wednesday|1900Q3|N|N|N|2415264|2415507|2414917|2415191|N|N|N|N|N| +2415283|AAAAAAAADLKNECAA|1900-09-20|8|38|4|1900|4|9|20|3|1900|4|38|Thursday|1900Q3|N|N|N|2415264|2415507|2414918|2415192|N|N|N|N|N| +2415284|AAAAAAAAELKNECAA|1900-09-21|8|38|4|1900|5|9|21|3|1900|4|38|Friday|1900Q3|N|Y|N|2415264|2415507|2414919|2415193|N|N|N|N|N| +2415285|AAAAAAAAFLKNECAA|1900-09-22|8|38|4|1900|6|9|22|3|1900|4|38|Saturday|1900Q3|N|Y|N|2415264|2415507|2414920|2415194|N|N|N|N|N| +2415286|AAAAAAAAGLKNECAA|1900-09-23|8|38|4|1900|0|9|23|3|1900|4|38|Sunday|1900Q3|N|N|N|2415264|2415507|2414921|2415195|N|N|N|N|N| +2415287|AAAAAAAAHLKNECAA|1900-09-24|8|38|4|1900|1|9|24|3|1900|4|38|Monday|1900Q3|N|N|N|2415264|2415507|2414922|2415196|N|N|N|N|N| +2415288|AAAAAAAAILKNECAA|1900-09-25|8|39|4|1900|2|9|25|3|1900|4|39|Tuesday|1900Q3|N|N|N|2415264|2415507|2414923|2415197|N|N|N|N|N| +2415289|AAAAAAAAJLKNECAA|1900-09-26|8|39|4|1900|3|9|26|3|1900|4|39|Wednesday|1900Q3|N|N|N|2415264|2415507|2414924|2415198|N|N|N|N|N| +2415290|AAAAAAAAKLKNECAA|1900-09-27|8|39|4|1900|4|9|27|3|1900|4|39|Thursday|1900Q3|N|N|N|2415264|2415507|2414925|2415199|N|N|N|N|N| +2415291|AAAAAAAALLKNECAA|1900-09-28|8|39|4|1900|5|9|28|3|1900|4|39|Friday|1900Q3|N|Y|N|2415264|2415507|2414926|2415200|N|N|N|N|N| +2415292|AAAAAAAAMLKNECAA|1900-09-29|8|39|4|1900|6|9|29|3|1900|4|39|Saturday|1900Q3|N|Y|N|2415264|2415507|2414927|2415201|N|N|N|N|N| +2415293|AAAAAAAANLKNECAA|1900-09-30|8|39|4|1900|0|9|30|3|1900|4|39|Sunday|1900Q3|N|N|N|2415264|2415507|2414928|2415202|N|N|N|N|N| +2415294|AAAAAAAAOLKNECAA|1900-10-01|9|39|4|1900|1|10|1|4|1900|4|39|Monday|1900Q4|N|N|N|2415294|2415567|2414929|2415202|N|N|N|N|N| +2415295|AAAAAAAAPLKNECAA|1900-10-02|9|40|4|1900|2|10|2|4|1900|4|40|Tuesday|1900Q4|N|N|N|2415294|2415567|2414930|2415203|N|N|N|N|N| +2415296|AAAAAAAAAMKNECAA|1900-10-03|9|40|4|1900|3|10|3|4|1900|4|40|Wednesday|1900Q4|N|N|N|2415294|2415567|2414931|2415204|N|N|N|N|N| +2415297|AAAAAAAABMKNECAA|1900-10-04|9|40|4|1900|4|10|4|4|1900|4|40|Thursday|1900Q4|N|N|N|2415294|2415567|2414932|2415205|N|N|N|N|N| +2415298|AAAAAAAACMKNECAA|1900-10-05|9|40|4|1900|5|10|5|4|1900|4|40|Friday|1900Q4|N|Y|N|2415294|2415567|2414933|2415206|N|N|N|N|N| +2415299|AAAAAAAADMKNECAA|1900-10-06|9|40|4|1900|6|10|6|4|1900|4|40|Saturday|1900Q4|N|Y|N|2415294|2415567|2414934|2415207|N|N|N|N|N| +2415300|AAAAAAAAEMKNECAA|1900-10-07|9|40|4|1900|0|10|7|4|1900|4|40|Sunday|1900Q4|N|N|N|2415294|2415567|2414935|2415208|N|N|N|N|N| +2415301|AAAAAAAAFMKNECAA|1900-10-08|9|40|4|1900|1|10|8|4|1900|4|40|Monday|1900Q4|N|N|N|2415294|2415567|2414936|2415209|N|N|N|N|N| +2415302|AAAAAAAAGMKNECAA|1900-10-09|9|41|4|1900|2|10|9|4|1900|4|41|Tuesday|1900Q4|N|N|N|2415294|2415567|2414937|2415210|N|N|N|N|N| +2415303|AAAAAAAAHMKNECAA|1900-10-10|9|41|4|1900|3|10|10|4|1900|4|41|Wednesday|1900Q4|N|N|N|2415294|2415567|2414938|2415211|N|N|N|N|N| +2415304|AAAAAAAAIMKNECAA|1900-10-11|9|41|4|1900|4|10|11|4|1900|4|41|Thursday|1900Q4|N|N|N|2415294|2415567|2414939|2415212|N|N|N|N|N| +2415305|AAAAAAAAJMKNECAA|1900-10-12|9|41|4|1900|5|10|12|4|1900|4|41|Friday|1900Q4|N|Y|N|2415294|2415567|2414940|2415213|N|N|N|N|N| +2415306|AAAAAAAAKMKNECAA|1900-10-13|9|41|4|1900|6|10|13|4|1900|4|41|Saturday|1900Q4|N|Y|N|2415294|2415567|2414941|2415214|N|N|N|N|N| +2415307|AAAAAAAALMKNECAA|1900-10-14|9|41|4|1900|0|10|14|4|1900|4|41|Sunday|1900Q4|N|N|N|2415294|2415567|2414942|2415215|N|N|N|N|N| +2415308|AAAAAAAAMMKNECAA|1900-10-15|9|41|4|1900|1|10|15|4|1900|4|41|Monday|1900Q4|N|N|N|2415294|2415567|2414943|2415216|N|N|N|N|N| +2415309|AAAAAAAANMKNECAA|1900-10-16|9|42|4|1900|2|10|16|4|1900|4|42|Tuesday|1900Q4|N|N|N|2415294|2415567|2414944|2415217|N|N|N|N|N| +2415310|AAAAAAAAOMKNECAA|1900-10-17|9|42|4|1900|3|10|17|4|1900|4|42|Wednesday|1900Q4|N|N|N|2415294|2415567|2414945|2415218|N|N|N|N|N| +2415311|AAAAAAAAPMKNECAA|1900-10-18|9|42|4|1900|4|10|18|4|1900|4|42|Thursday|1900Q4|N|N|N|2415294|2415567|2414946|2415219|N|N|N|N|N| +2415312|AAAAAAAAANKNECAA|1900-10-19|9|42|4|1900|5|10|19|4|1900|4|42|Friday|1900Q4|N|Y|N|2415294|2415567|2414947|2415220|N|N|N|N|N| +2415313|AAAAAAAABNKNECAA|1900-10-20|9|42|4|1900|6|10|20|4|1900|4|42|Saturday|1900Q4|N|Y|N|2415294|2415567|2414948|2415221|N|N|N|N|N| +2415314|AAAAAAAACNKNECAA|1900-10-21|9|42|4|1900|0|10|21|4|1900|4|42|Sunday|1900Q4|N|N|N|2415294|2415567|2414949|2415222|N|N|N|N|N| +2415315|AAAAAAAADNKNECAA|1900-10-22|9|42|4|1900|1|10|22|4|1900|4|42|Monday|1900Q4|N|N|N|2415294|2415567|2414950|2415223|N|N|N|N|N| +2415316|AAAAAAAAENKNECAA|1900-10-23|9|43|4|1900|2|10|23|4|1900|4|43|Tuesday|1900Q4|N|N|N|2415294|2415567|2414951|2415224|N|N|N|N|N| +2415317|AAAAAAAAFNKNECAA|1900-10-24|9|43|4|1900|3|10|24|4|1900|4|43|Wednesday|1900Q4|N|N|N|2415294|2415567|2414952|2415225|N|N|N|N|N| +2415318|AAAAAAAAGNKNECAA|1900-10-25|9|43|4|1900|4|10|25|4|1900|4|43|Thursday|1900Q4|N|N|N|2415294|2415567|2414953|2415226|N|N|N|N|N| +2415319|AAAAAAAAHNKNECAA|1900-10-26|9|43|4|1900|5|10|26|4|1900|4|43|Friday|1900Q4|N|Y|N|2415294|2415567|2414954|2415227|N|N|N|N|N| +2415320|AAAAAAAAINKNECAA|1900-10-27|9|43|4|1900|6|10|27|4|1900|4|43|Saturday|1900Q4|N|Y|N|2415294|2415567|2414955|2415228|N|N|N|N|N| +2415321|AAAAAAAAJNKNECAA|1900-10-28|9|43|4|1900|0|10|28|4|1900|4|43|Sunday|1900Q4|N|N|N|2415294|2415567|2414956|2415229|N|N|N|N|N| +2415322|AAAAAAAAKNKNECAA|1900-10-29|9|43|4|1900|1|10|29|4|1900|4|43|Monday|1900Q4|N|N|N|2415294|2415567|2414957|2415230|N|N|N|N|N| +2415323|AAAAAAAALNKNECAA|1900-10-30|9|44|4|1900|2|10|30|4|1900|4|44|Tuesday|1900Q4|N|N|N|2415294|2415567|2414958|2415231|N|N|N|N|N| +2415324|AAAAAAAAMNKNECAA|1900-10-31|9|44|4|1900|3|10|31|4|1900|4|44|Wednesday|1900Q4|N|N|N|2415294|2415567|2414959|2415232|N|N|N|N|N| +2415325|AAAAAAAANNKNECAA|1900-11-01|10|44|4|1900|4|11|1|4|1900|4|44|Thursday|1900Q4|N|N|N|2415325|2415629|2414960|2415233|N|N|N|N|N| +2415326|AAAAAAAAONKNECAA|1900-11-02|10|44|4|1900|5|11|2|4|1900|4|44|Friday|1900Q4|N|Y|N|2415325|2415629|2414961|2415234|N|N|N|N|N| +2415327|AAAAAAAAPNKNECAA|1900-11-03|10|44|4|1900|6|11|3|4|1900|4|44|Saturday|1900Q4|N|Y|N|2415325|2415629|2414962|2415235|N|N|N|N|N| +2415328|AAAAAAAAAOKNECAA|1900-11-04|10|44|4|1900|0|11|4|4|1900|4|44|Sunday|1900Q4|N|N|N|2415325|2415629|2414963|2415236|N|N|N|N|N| +2415329|AAAAAAAABOKNECAA|1900-11-05|10|44|4|1900|1|11|5|4|1900|4|44|Monday|1900Q4|N|N|N|2415325|2415629|2414964|2415237|N|N|N|N|N| +2415330|AAAAAAAACOKNECAA|1900-11-06|10|45|4|1900|2|11|6|4|1900|4|45|Tuesday|1900Q4|N|N|N|2415325|2415629|2414965|2415238|N|N|N|N|N| +2415331|AAAAAAAADOKNECAA|1900-11-07|10|45|4|1900|3|11|7|4|1900|4|45|Wednesday|1900Q4|N|N|N|2415325|2415629|2414966|2415239|N|N|N|N|N| +2415332|AAAAAAAAEOKNECAA|1900-11-08|10|45|4|1900|4|11|8|4|1900|4|45|Thursday|1900Q4|N|N|N|2415325|2415629|2414967|2415240|N|N|N|N|N| +2415333|AAAAAAAAFOKNECAA|1900-11-09|10|45|4|1900|5|11|9|4|1900|4|45|Friday|1900Q4|N|Y|N|2415325|2415629|2414968|2415241|N|N|N|N|N| +2415334|AAAAAAAAGOKNECAA|1900-11-10|10|45|4|1900|6|11|10|4|1900|4|45|Saturday|1900Q4|N|Y|N|2415325|2415629|2414969|2415242|N|N|N|N|N| +2415335|AAAAAAAAHOKNECAA|1900-11-11|10|45|4|1900|0|11|11|4|1900|4|45|Sunday|1900Q4|N|N|N|2415325|2415629|2414970|2415243|N|N|N|N|N| +2415336|AAAAAAAAIOKNECAA|1900-11-12|10|45|4|1900|1|11|12|4|1900|4|45|Monday|1900Q4|N|N|N|2415325|2415629|2414971|2415244|N|N|N|N|N| +2415337|AAAAAAAAJOKNECAA|1900-11-13|10|46|4|1900|2|11|13|4|1900|4|46|Tuesday|1900Q4|N|N|N|2415325|2415629|2414972|2415245|N|N|N|N|N| +2415338|AAAAAAAAKOKNECAA|1900-11-14|10|46|4|1900|3|11|14|4|1900|4|46|Wednesday|1900Q4|N|N|N|2415325|2415629|2414973|2415246|N|N|N|N|N| +2415339|AAAAAAAALOKNECAA|1900-11-15|10|46|4|1900|4|11|15|4|1900|4|46|Thursday|1900Q4|N|N|N|2415325|2415629|2414974|2415247|N|N|N|N|N| +2415340|AAAAAAAAMOKNECAA|1900-11-16|10|46|4|1900|5|11|16|4|1900|4|46|Friday|1900Q4|N|Y|N|2415325|2415629|2414975|2415248|N|N|N|N|N| +2415341|AAAAAAAANOKNECAA|1900-11-17|10|46|4|1900|6|11|17|4|1900|4|46|Saturday|1900Q4|N|Y|N|2415325|2415629|2414976|2415249|N|N|N|N|N| +2415342|AAAAAAAAOOKNECAA|1900-11-18|10|46|4|1900|0|11|18|4|1900|4|46|Sunday|1900Q4|N|N|N|2415325|2415629|2414977|2415250|N|N|N|N|N| +2415343|AAAAAAAAPOKNECAA|1900-11-19|10|46|4|1900|1|11|19|4|1900|4|46|Monday|1900Q4|N|N|N|2415325|2415629|2414978|2415251|N|N|N|N|N| +2415344|AAAAAAAAAPKNECAA|1900-11-20|10|47|4|1900|2|11|20|4|1900|4|47|Tuesday|1900Q4|N|N|N|2415325|2415629|2414979|2415252|N|N|N|N|N| +2415345|AAAAAAAABPKNECAA|1900-11-21|10|47|4|1900|3|11|21|4|1900|4|47|Wednesday|1900Q4|N|N|N|2415325|2415629|2414980|2415253|N|N|N|N|N| +2415346|AAAAAAAACPKNECAA|1900-11-22|10|47|4|1900|4|11|22|4|1900|4|47|Thursday|1900Q4|N|N|N|2415325|2415629|2414981|2415254|N|N|N|N|N| +2415347|AAAAAAAADPKNECAA|1900-11-23|10|47|4|1900|5|11|23|4|1900|4|47|Friday|1900Q4|N|Y|N|2415325|2415629|2414982|2415255|N|N|N|N|N| +2415348|AAAAAAAAEPKNECAA|1900-11-24|10|47|4|1900|6|11|24|4|1900|4|47|Saturday|1900Q4|N|Y|N|2415325|2415629|2414983|2415256|N|N|N|N|N| +2415349|AAAAAAAAFPKNECAA|1900-11-25|10|47|4|1900|0|11|25|4|1900|4|47|Sunday|1900Q4|N|N|N|2415325|2415629|2414984|2415257|N|N|N|N|N| +2415350|AAAAAAAAGPKNECAA|1900-11-26|10|47|4|1900|1|11|26|4|1900|4|47|Monday|1900Q4|N|N|N|2415325|2415629|2414985|2415258|N|N|N|N|N| +2415351|AAAAAAAAHPKNECAA|1900-11-27|10|48|4|1900|2|11|27|4|1900|4|48|Tuesday|1900Q4|N|N|N|2415325|2415629|2414986|2415259|N|N|N|N|N| +2415352|AAAAAAAAIPKNECAA|1900-11-28|10|48|4|1900|3|11|28|4|1900|4|48|Wednesday|1900Q4|N|N|N|2415325|2415629|2414987|2415260|N|N|N|N|N| +2415353|AAAAAAAAJPKNECAA|1900-11-29|10|48|4|1900|4|11|29|4|1900|4|48|Thursday|1900Q4|N|N|N|2415325|2415629|2414988|2415261|N|N|N|N|N| +2415354|AAAAAAAAKPKNECAA|1900-11-30|10|48|4|1900|5|11|30|4|1900|4|48|Friday|1900Q4|N|Y|N|2415325|2415629|2414989|2415262|N|N|N|N|N| +2415355|AAAAAAAALPKNECAA|1900-12-01|11|48|5|1900|6|12|1|4|1900|5|48|Saturday|1900Q4|N|Y|N|2415355|2415689|2414990|2415263|N|N|N|N|N| +2415356|AAAAAAAAMPKNECAA|1900-12-02|11|48|5|1900|0|12|2|4|1900|5|48|Sunday|1900Q4|N|N|N|2415355|2415689|2414991|2415264|N|N|N|N|N| +2415357|AAAAAAAANPKNECAA|1900-12-03|11|48|5|1900|1|12|3|4|1900|5|48|Monday|1900Q4|N|N|N|2415355|2415689|2414992|2415265|N|N|N|N|N| +2415358|AAAAAAAAOPKNECAA|1900-12-04|11|49|5|1900|2|12|4|4|1900|5|49|Tuesday|1900Q4|N|N|N|2415355|2415689|2414993|2415266|N|N|N|N|N| +2415359|AAAAAAAAPPKNECAA|1900-12-05|11|49|5|1900|3|12|5|4|1900|5|49|Wednesday|1900Q4|N|N|N|2415355|2415689|2414994|2415267|N|N|N|N|N| +2415360|AAAAAAAAAALNECAA|1900-12-06|11|49|5|1900|4|12|6|4|1900|5|49|Thursday|1900Q4|N|N|N|2415355|2415689|2414995|2415268|N|N|N|N|N| +2415361|AAAAAAAABALNECAA|1900-12-07|11|49|5|1900|5|12|7|4|1900|5|49|Friday|1900Q4|N|Y|N|2415355|2415689|2414996|2415269|N|N|N|N|N| +2415362|AAAAAAAACALNECAA|1900-12-08|11|49|5|1900|6|12|8|4|1900|5|49|Saturday|1900Q4|N|Y|N|2415355|2415689|2414997|2415270|N|N|N|N|N| +2415363|AAAAAAAADALNECAA|1900-12-09|11|49|5|1900|0|12|9|4|1900|5|49|Sunday|1900Q4|N|N|N|2415355|2415689|2414998|2415271|N|N|N|N|N| +2415364|AAAAAAAAEALNECAA|1900-12-10|11|49|5|1900|1|12|10|4|1900|5|49|Monday|1900Q4|N|N|N|2415355|2415689|2414999|2415272|N|N|N|N|N| +2415365|AAAAAAAAFALNECAA|1900-12-11|11|50|5|1900|2|12|11|4|1900|5|50|Tuesday|1900Q4|N|N|N|2415355|2415689|2415000|2415273|N|N|N|N|N| +2415366|AAAAAAAAGALNECAA|1900-12-12|11|50|5|1900|3|12|12|4|1900|5|50|Wednesday|1900Q4|N|N|N|2415355|2415689|2415001|2415274|N|N|N|N|N| +2415367|AAAAAAAAHALNECAA|1900-12-13|11|50|5|1900|4|12|13|4|1900|5|50|Thursday|1900Q4|N|N|N|2415355|2415689|2415002|2415275|N|N|N|N|N| +2415368|AAAAAAAAIALNECAA|1900-12-14|11|50|5|1900|5|12|14|4|1900|5|50|Friday|1900Q4|N|Y|N|2415355|2415689|2415003|2415276|N|N|N|N|N| +2415369|AAAAAAAAJALNECAA|1900-12-15|11|50|5|1900|6|12|15|4|1900|5|50|Saturday|1900Q4|N|Y|N|2415355|2415689|2415004|2415277|N|N|N|N|N| +2415370|AAAAAAAAKALNECAA|1900-12-16|11|50|5|1900|0|12|16|4|1900|5|50|Sunday|1900Q4|N|N|N|2415355|2415689|2415005|2415278|N|N|N|N|N| +2415371|AAAAAAAALALNECAA|1900-12-17|11|50|5|1900|1|12|17|4|1900|5|50|Monday|1900Q4|N|N|N|2415355|2415689|2415006|2415279|N|N|N|N|N| +2415372|AAAAAAAAMALNECAA|1900-12-18|11|51|5|1900|2|12|18|4|1900|5|51|Tuesday|1900Q4|N|N|N|2415355|2415689|2415007|2415280|N|N|N|N|N| +2415373|AAAAAAAANALNECAA|1900-12-19|11|51|5|1900|3|12|19|4|1900|5|51|Wednesday|1900Q4|N|N|N|2415355|2415689|2415008|2415281|N|N|N|N|N| +2415374|AAAAAAAAOALNECAA|1900-12-20|11|51|5|1900|4|12|20|4|1900|5|51|Thursday|1900Q4|N|N|N|2415355|2415689|2415009|2415282|N|N|N|N|N| +2415375|AAAAAAAAPALNECAA|1900-12-21|11|51|5|1900|5|12|21|4|1900|5|51|Friday|1900Q4|N|Y|N|2415355|2415689|2415010|2415283|N|N|N|N|N| +2415376|AAAAAAAAABLNECAA|1900-12-22|11|51|5|1900|6|12|22|4|1900|5|51|Saturday|1900Q4|N|Y|N|2415355|2415689|2415011|2415284|N|N|N|N|N| +2415377|AAAAAAAABBLNECAA|1900-12-23|11|51|5|1900|0|12|23|4|1900|5|51|Sunday|1900Q4|N|N|N|2415355|2415689|2415012|2415285|N|N|N|N|N| +2415378|AAAAAAAACBLNECAA|1900-12-24|11|51|5|1900|1|12|24|4|1900|5|51|Monday|1900Q4|N|N|N|2415355|2415689|2415013|2415286|N|N|N|N|N| +2415379|AAAAAAAADBLNECAA|1900-12-25|11|52|5|1900|2|12|25|4|1900|5|52|Tuesday|1900Q4|Y|N|N|2415355|2415689|2415014|2415287|N|N|N|N|N| +2415380|AAAAAAAAEBLNECAA|1900-12-26|11|52|5|1900|3|12|26|4|1900|5|52|Wednesday|1900Q4|N|N|Y|2415355|2415689|2415015|2415288|N|N|N|N|N| +2415381|AAAAAAAAFBLNECAA|1900-12-27|11|52|5|1900|4|12|27|4|1900|5|52|Thursday|1900Q4|N|N|N|2415355|2415689|2415016|2415289|N|N|N|N|N| +2415382|AAAAAAAAGBLNECAA|1900-12-28|11|52|5|1900|5|12|28|4|1900|5|52|Friday|1900Q4|N|Y|N|2415355|2415689|2415017|2415290|N|N|N|N|N| +2415383|AAAAAAAAHBLNECAA|1900-12-29|11|52|5|1900|6|12|29|4|1900|5|52|Saturday|1900Q4|N|Y|N|2415355|2415689|2415018|2415291|N|N|N|N|N| +2415384|AAAAAAAAIBLNECAA|1900-12-30|11|52|5|1900|0|12|30|4|1900|5|52|Sunday|1900Q4|N|N|N|2415355|2415689|2415019|2415292|N|N|N|N|N| +2415385|AAAAAAAAJBLNECAA|1900-12-31|11|52|5|1900|1|12|31|4|1900|5|52|Monday|1900Q4|Y|N|N|2415355|2415689|2415020|2415293|N|N|N|N|N| +2415386|AAAAAAAAKBLNECAA|1901-01-01|12|53|5|1901|2|1|1|1|1901|5|53|Tuesday|1901Q1|Y|N|Y|2415386|2415385|2415021|2415294|N|N|N|N|N| +2415387|AAAAAAAALBLNECAA|1901-01-02|12|53|5|1901|3|1|2|1|1901|5|53|Wednesday|1901Q1|N|N|Y|2415386|2415385|2415022|2415295|N|N|N|N|N| +2415388|AAAAAAAAMBLNECAA|1901-01-03|12|53|5|1901|4|1|3|1|1901|5|53|Thursday|1901Q1|N|N|N|2415386|2415385|2415023|2415296|N|N|N|N|N| +2415389|AAAAAAAANBLNECAA|1901-01-04|12|53|5|1901|5|1|4|1|1901|5|53|Friday|1901Q1|N|Y|N|2415386|2415385|2415024|2415297|N|N|N|N|N| +2415390|AAAAAAAAOBLNECAA|1901-01-05|12|53|5|1901|6|1|5|1|1901|5|53|Saturday|1901Q1|N|Y|N|2415386|2415385|2415025|2415298|N|N|N|N|N| +2415391|AAAAAAAAPBLNECAA|1901-01-06|12|53|5|1901|0|1|6|1|1901|5|53|Sunday|1901Q1|N|N|N|2415386|2415385|2415026|2415299|N|N|N|N|N| +2415392|AAAAAAAAACLNECAA|1901-01-07|12|53|5|1901|1|1|7|1|1901|5|53|Monday|1901Q1|N|N|N|2415386|2415385|2415027|2415300|N|N|N|N|N| +2415393|AAAAAAAABCLNECAA|1901-01-08|12|54|5|1901|2|1|8|1|1901|5|54|Tuesday|1901Q1|N|N|N|2415386|2415385|2415028|2415301|N|N|N|N|N| +2415394|AAAAAAAACCLNECAA|1901-01-09|12|54|5|1901|3|1|9|1|1901|5|54|Wednesday|1901Q1|N|N|N|2415386|2415385|2415029|2415302|N|N|N|N|N| +2415395|AAAAAAAADCLNECAA|1901-01-10|12|54|5|1901|4|1|10|1|1901|5|54|Thursday|1901Q1|N|N|N|2415386|2415385|2415030|2415303|N|N|N|N|N| +2415396|AAAAAAAAECLNECAA|1901-01-11|12|54|5|1901|5|1|11|1|1901|5|54|Friday|1901Q1|N|Y|N|2415386|2415385|2415031|2415304|N|N|N|N|N| +2415397|AAAAAAAAFCLNECAA|1901-01-12|12|54|5|1901|6|1|12|1|1901|5|54|Saturday|1901Q1|N|Y|N|2415386|2415385|2415032|2415305|N|N|N|N|N| +2415398|AAAAAAAAGCLNECAA|1901-01-13|12|54|5|1901|0|1|13|1|1901|5|54|Sunday|1901Q1|N|N|N|2415386|2415385|2415033|2415306|N|N|N|N|N| +2415399|AAAAAAAAHCLNECAA|1901-01-14|12|54|5|1901|1|1|14|1|1901|5|54|Monday|1901Q1|N|N|N|2415386|2415385|2415034|2415307|N|N|N|N|N| +2415400|AAAAAAAAICLNECAA|1901-01-15|12|55|5|1901|2|1|15|1|1901|5|55|Tuesday|1901Q1|N|N|N|2415386|2415385|2415035|2415308|N|N|N|N|N| +2415401|AAAAAAAAJCLNECAA|1901-01-16|12|55|5|1901|3|1|16|1|1901|5|55|Wednesday|1901Q1|N|N|N|2415386|2415385|2415036|2415309|N|N|N|N|N| +2415402|AAAAAAAAKCLNECAA|1901-01-17|12|55|5|1901|4|1|17|1|1901|5|55|Thursday|1901Q1|N|N|N|2415386|2415385|2415037|2415310|N|N|N|N|N| +2415403|AAAAAAAALCLNECAA|1901-01-18|12|55|5|1901|5|1|18|1|1901|5|55|Friday|1901Q1|N|Y|N|2415386|2415385|2415038|2415311|N|N|N|N|N| +2415404|AAAAAAAAMCLNECAA|1901-01-19|12|55|5|1901|6|1|19|1|1901|5|55|Saturday|1901Q1|N|Y|N|2415386|2415385|2415039|2415312|N|N|N|N|N| +2415405|AAAAAAAANCLNECAA|1901-01-20|12|55|5|1901|0|1|20|1|1901|5|55|Sunday|1901Q1|N|N|N|2415386|2415385|2415040|2415313|N|N|N|N|N| +2415406|AAAAAAAAOCLNECAA|1901-01-21|12|55|5|1901|1|1|21|1|1901|5|55|Monday|1901Q1|N|N|N|2415386|2415385|2415041|2415314|N|N|N|N|N| +2415407|AAAAAAAAPCLNECAA|1901-01-22|12|56|5|1901|2|1|22|1|1901|5|56|Tuesday|1901Q1|N|N|N|2415386|2415385|2415042|2415315|N|N|N|N|N| +2415408|AAAAAAAAADLNECAA|1901-01-23|12|56|5|1901|3|1|23|1|1901|5|56|Wednesday|1901Q1|N|N|N|2415386|2415385|2415043|2415316|N|N|N|N|N| +2415409|AAAAAAAABDLNECAA|1901-01-24|12|56|5|1901|4|1|24|1|1901|5|56|Thursday|1901Q1|N|N|N|2415386|2415385|2415044|2415317|N|N|N|N|N| +2415410|AAAAAAAACDLNECAA|1901-01-25|12|56|5|1901|5|1|25|1|1901|5|56|Friday|1901Q1|N|Y|N|2415386|2415385|2415045|2415318|N|N|N|N|N| +2415411|AAAAAAAADDLNECAA|1901-01-26|12|56|5|1901|6|1|26|1|1901|5|56|Saturday|1901Q1|N|Y|N|2415386|2415385|2415046|2415319|N|N|N|N|N| +2415412|AAAAAAAAEDLNECAA|1901-01-27|12|56|5|1901|0|1|27|1|1901|5|56|Sunday|1901Q1|N|N|N|2415386|2415385|2415047|2415320|N|N|N|N|N| +2415413|AAAAAAAAFDLNECAA|1901-01-28|12|56|5|1901|1|1|28|1|1901|5|56|Monday|1901Q1|N|N|N|2415386|2415385|2415048|2415321|N|N|N|N|N| +2415414|AAAAAAAAGDLNECAA|1901-01-29|12|57|5|1901|2|1|29|1|1901|5|57|Tuesday|1901Q1|N|N|N|2415386|2415385|2415049|2415322|N|N|N|N|N| +2415415|AAAAAAAAHDLNECAA|1901-01-30|12|57|5|1901|3|1|30|1|1901|5|57|Wednesday|1901Q1|N|N|N|2415386|2415385|2415050|2415323|N|N|N|N|N| +2415416|AAAAAAAAIDLNECAA|1901-01-31|12|57|5|1901|4|1|31|1|1901|5|57|Thursday|1901Q1|N|N|N|2415386|2415385|2415051|2415324|N|N|N|N|N| +2415417|AAAAAAAAJDLNECAA|1901-02-01|13|57|5|1901|5|2|1|1|1901|5|57|Friday|1901Q1|N|Y|N|2415417|2415447|2415052|2415325|N|N|N|N|N| +2415418|AAAAAAAAKDLNECAA|1901-02-02|13|57|5|1901|6|2|2|1|1901|5|57|Saturday|1901Q1|N|Y|N|2415417|2415447|2415053|2415326|N|N|N|N|N| +2415419|AAAAAAAALDLNECAA|1901-02-03|13|57|5|1901|0|2|3|1|1901|5|57|Sunday|1901Q1|N|N|N|2415417|2415447|2415054|2415327|N|N|N|N|N| +2415420|AAAAAAAAMDLNECAA|1901-02-04|13|57|5|1901|1|2|4|1|1901|5|57|Monday|1901Q1|N|N|N|2415417|2415447|2415055|2415328|N|N|N|N|N| +2415421|AAAAAAAANDLNECAA|1901-02-05|13|58|5|1901|2|2|5|1|1901|5|58|Tuesday|1901Q1|N|N|N|2415417|2415447|2415056|2415329|N|N|N|N|N| +2415422|AAAAAAAAODLNECAA|1901-02-06|13|58|5|1901|3|2|6|1|1901|5|58|Wednesday|1901Q1|N|N|N|2415417|2415447|2415057|2415330|N|N|N|N|N| +2415423|AAAAAAAAPDLNECAA|1901-02-07|13|58|5|1901|4|2|7|1|1901|5|58|Thursday|1901Q1|N|N|N|2415417|2415447|2415058|2415331|N|N|N|N|N| +2415424|AAAAAAAAAELNECAA|1901-02-08|13|58|5|1901|5|2|8|1|1901|5|58|Friday|1901Q1|N|Y|N|2415417|2415447|2415059|2415332|N|N|N|N|N| +2415425|AAAAAAAABELNECAA|1901-02-09|13|58|5|1901|6|2|9|1|1901|5|58|Saturday|1901Q1|N|Y|N|2415417|2415447|2415060|2415333|N|N|N|N|N| +2415426|AAAAAAAACELNECAA|1901-02-10|13|58|5|1901|0|2|10|1|1901|5|58|Sunday|1901Q1|N|N|N|2415417|2415447|2415061|2415334|N|N|N|N|N| +2415427|AAAAAAAADELNECAA|1901-02-11|13|58|5|1901|1|2|11|1|1901|5|58|Monday|1901Q1|N|N|N|2415417|2415447|2415062|2415335|N|N|N|N|N| +2415428|AAAAAAAAEELNECAA|1901-02-12|13|59|5|1901|2|2|12|1|1901|5|59|Tuesday|1901Q1|N|N|N|2415417|2415447|2415063|2415336|N|N|N|N|N| +2415429|AAAAAAAAFELNECAA|1901-02-13|13|59|5|1901|3|2|13|1|1901|5|59|Wednesday|1901Q1|N|N|N|2415417|2415447|2415064|2415337|N|N|N|N|N| +2415430|AAAAAAAAGELNECAA|1901-02-14|13|59|5|1901|4|2|14|1|1901|5|59|Thursday|1901Q1|N|N|N|2415417|2415447|2415065|2415338|N|N|N|N|N| +2415431|AAAAAAAAHELNECAA|1901-02-15|13|59|5|1901|5|2|15|1|1901|5|59|Friday|1901Q1|N|Y|N|2415417|2415447|2415066|2415339|N|N|N|N|N| +2415432|AAAAAAAAIELNECAA|1901-02-16|13|59|5|1901|6|2|16|1|1901|5|59|Saturday|1901Q1|N|Y|N|2415417|2415447|2415067|2415340|N|N|N|N|N| +2415433|AAAAAAAAJELNECAA|1901-02-17|13|59|5|1901|0|2|17|1|1901|5|59|Sunday|1901Q1|N|N|N|2415417|2415447|2415068|2415341|N|N|N|N|N| +2415434|AAAAAAAAKELNECAA|1901-02-18|13|59|5|1901|1|2|18|1|1901|5|59|Monday|1901Q1|N|N|N|2415417|2415447|2415069|2415342|N|N|N|N|N| +2415435|AAAAAAAALELNECAA|1901-02-19|13|60|5|1901|2|2|19|1|1901|5|60|Tuesday|1901Q1|N|N|N|2415417|2415447|2415070|2415343|N|N|N|N|N| +2415436|AAAAAAAAMELNECAA|1901-02-20|13|60|5|1901|3|2|20|1|1901|5|60|Wednesday|1901Q1|N|N|N|2415417|2415447|2415071|2415344|N|N|N|N|N| +2415437|AAAAAAAANELNECAA|1901-02-21|13|60|5|1901|4|2|21|1|1901|5|60|Thursday|1901Q1|N|N|N|2415417|2415447|2415072|2415345|N|N|N|N|N| +2415438|AAAAAAAAOELNECAA|1901-02-22|13|60|5|1901|5|2|22|1|1901|5|60|Friday|1901Q1|N|Y|N|2415417|2415447|2415073|2415346|N|N|N|N|N| +2415439|AAAAAAAAPELNECAA|1901-02-23|13|60|5|1901|6|2|23|1|1901|5|60|Saturday|1901Q1|N|Y|N|2415417|2415447|2415074|2415347|N|N|N|N|N| +2415440|AAAAAAAAAFLNECAA|1901-02-24|13|60|5|1901|0|2|24|1|1901|5|60|Sunday|1901Q1|N|N|N|2415417|2415447|2415075|2415348|N|N|N|N|N| +2415441|AAAAAAAABFLNECAA|1901-02-25|13|60|5|1901|1|2|25|1|1901|5|60|Monday|1901Q1|N|N|N|2415417|2415447|2415076|2415349|N|N|N|N|N| +2415442|AAAAAAAACFLNECAA|1901-02-26|13|61|5|1901|2|2|26|1|1901|5|61|Tuesday|1901Q1|N|N|N|2415417|2415447|2415077|2415350|N|N|N|N|N| +2415443|AAAAAAAADFLNECAA|1901-02-27|13|61|5|1901|3|2|27|1|1901|5|61|Wednesday|1901Q1|N|N|N|2415417|2415447|2415078|2415351|N|N|N|N|N| +2415444|AAAAAAAAEFLNECAA|1901-02-28|13|61|5|1901|4|2|28|1|1901|5|61|Thursday|1901Q1|N|N|N|2415417|2415447|2415079|2415352|N|N|N|N|N| +2415445|AAAAAAAAFFLNECAA|1901-03-01|14|61|6|1901|5|3|1|1|1901|6|61|Friday|1901Q1|N|Y|N|2415445|2415503|2415080|2415353|N|N|N|N|N| +2415446|AAAAAAAAGFLNECAA|1901-03-02|14|61|6|1901|6|3|2|1|1901|6|61|Saturday|1901Q1|N|Y|N|2415445|2415503|2415081|2415354|N|N|N|N|N| +2415447|AAAAAAAAHFLNECAA|1901-03-03|14|61|6|1901|0|3|3|1|1901|6|61|Sunday|1901Q1|N|N|N|2415445|2415503|2415082|2415355|N|N|N|N|N| +2415448|AAAAAAAAIFLNECAA|1901-03-04|14|61|6|1901|1|3|4|1|1901|6|61|Monday|1901Q1|N|N|N|2415445|2415503|2415083|2415356|N|N|N|N|N| +2415449|AAAAAAAAJFLNECAA|1901-03-05|14|62|6|1901|2|3|5|1|1901|6|62|Tuesday|1901Q1|N|N|N|2415445|2415503|2415084|2415357|N|N|N|N|N| +2415450|AAAAAAAAKFLNECAA|1901-03-06|14|62|6|1901|3|3|6|1|1901|6|62|Wednesday|1901Q1|N|N|N|2415445|2415503|2415085|2415358|N|N|N|N|N| +2415451|AAAAAAAALFLNECAA|1901-03-07|14|62|6|1901|4|3|7|1|1901|6|62|Thursday|1901Q1|N|N|N|2415445|2415503|2415086|2415359|N|N|N|N|N| +2415452|AAAAAAAAMFLNECAA|1901-03-08|14|62|6|1901|5|3|8|1|1901|6|62|Friday|1901Q1|N|Y|N|2415445|2415503|2415087|2415360|N|N|N|N|N| +2415453|AAAAAAAANFLNECAA|1901-03-09|14|62|6|1901|6|3|9|1|1901|6|62|Saturday|1901Q1|N|Y|N|2415445|2415503|2415088|2415361|N|N|N|N|N| +2415454|AAAAAAAAOFLNECAA|1901-03-10|14|62|6|1901|0|3|10|1|1901|6|62|Sunday|1901Q1|N|N|N|2415445|2415503|2415089|2415362|N|N|N|N|N| +2415455|AAAAAAAAPFLNECAA|1901-03-11|14|62|6|1901|1|3|11|1|1901|6|62|Monday|1901Q1|N|N|N|2415445|2415503|2415090|2415363|N|N|N|N|N| +2415456|AAAAAAAAAGLNECAA|1901-03-12|14|63|6|1901|2|3|12|1|1901|6|63|Tuesday|1901Q1|N|N|N|2415445|2415503|2415091|2415364|N|N|N|N|N| +2415457|AAAAAAAABGLNECAA|1901-03-13|14|63|6|1901|3|3|13|1|1901|6|63|Wednesday|1901Q1|N|N|N|2415445|2415503|2415092|2415365|N|N|N|N|N| +2415458|AAAAAAAACGLNECAA|1901-03-14|14|63|6|1901|4|3|14|1|1901|6|63|Thursday|1901Q1|N|N|N|2415445|2415503|2415093|2415366|N|N|N|N|N| +2415459|AAAAAAAADGLNECAA|1901-03-15|14|63|6|1901|5|3|15|1|1901|6|63|Friday|1901Q1|N|Y|N|2415445|2415503|2415094|2415367|N|N|N|N|N| +2415460|AAAAAAAAEGLNECAA|1901-03-16|14|63|6|1901|6|3|16|1|1901|6|63|Saturday|1901Q1|N|Y|N|2415445|2415503|2415095|2415368|N|N|N|N|N| +2415461|AAAAAAAAFGLNECAA|1901-03-17|14|63|6|1901|0|3|17|1|1901|6|63|Sunday|1901Q1|N|N|N|2415445|2415503|2415096|2415369|N|N|N|N|N| +2415462|AAAAAAAAGGLNECAA|1901-03-18|14|63|6|1901|1|3|18|1|1901|6|63|Monday|1901Q1|N|N|N|2415445|2415503|2415097|2415370|N|N|N|N|N| +2415463|AAAAAAAAHGLNECAA|1901-03-19|14|64|6|1901|2|3|19|1|1901|6|64|Tuesday|1901Q1|N|N|N|2415445|2415503|2415098|2415371|N|N|N|N|N| +2415464|AAAAAAAAIGLNECAA|1901-03-20|14|64|6|1901|3|3|20|1|1901|6|64|Wednesday|1901Q1|N|N|N|2415445|2415503|2415099|2415372|N|N|N|N|N| +2415465|AAAAAAAAJGLNECAA|1901-03-21|14|64|6|1901|4|3|21|1|1901|6|64|Thursday|1901Q1|N|N|N|2415445|2415503|2415100|2415373|N|N|N|N|N| +2415466|AAAAAAAAKGLNECAA|1901-03-22|14|64|6|1901|5|3|22|1|1901|6|64|Friday|1901Q1|N|Y|N|2415445|2415503|2415101|2415374|N|N|N|N|N| +2415467|AAAAAAAALGLNECAA|1901-03-23|14|64|6|1901|6|3|23|1|1901|6|64|Saturday|1901Q1|N|Y|N|2415445|2415503|2415102|2415375|N|N|N|N|N| +2415468|AAAAAAAAMGLNECAA|1901-03-24|14|64|6|1901|0|3|24|1|1901|6|64|Sunday|1901Q1|N|N|N|2415445|2415503|2415103|2415376|N|N|N|N|N| +2415469|AAAAAAAANGLNECAA|1901-03-25|14|64|6|1901|1|3|25|1|1901|6|64|Monday|1901Q1|N|N|N|2415445|2415503|2415104|2415377|N|N|N|N|N| +2415470|AAAAAAAAOGLNECAA|1901-03-26|14|65|6|1901|2|3|26|1|1901|6|65|Tuesday|1901Q1|N|N|N|2415445|2415503|2415105|2415378|N|N|N|N|N| +2415471|AAAAAAAAPGLNECAA|1901-03-27|14|65|6|1901|3|3|27|1|1901|6|65|Wednesday|1901Q1|N|N|N|2415445|2415503|2415106|2415379|N|N|N|N|N| +2415472|AAAAAAAAAHLNECAA|1901-03-28|14|65|6|1901|4|3|28|1|1901|6|65|Thursday|1901Q1|N|N|N|2415445|2415503|2415107|2415380|N|N|N|N|N| +2415473|AAAAAAAABHLNECAA|1901-03-29|14|65|6|1901|5|3|29|1|1901|6|65|Friday|1901Q1|N|Y|N|2415445|2415503|2415108|2415381|N|N|N|N|N| +2415474|AAAAAAAACHLNECAA|1901-03-30|14|65|6|1901|6|3|30|1|1901|6|65|Saturday|1901Q1|N|Y|N|2415445|2415503|2415109|2415382|N|N|N|N|N| +2415475|AAAAAAAADHLNECAA|1901-03-31|14|65|6|1901|0|3|31|1|1901|6|65|Sunday|1901Q1|N|N|N|2415445|2415503|2415110|2415383|N|N|N|N|N| +2415476|AAAAAAAAEHLNECAA|1901-04-01|15|65|6|1901|1|4|1|1|1901|6|65|Monday|1901Q1|N|N|N|2415476|2415565|2415111|2415386|N|N|N|N|N| +2415477|AAAAAAAAFHLNECAA|1901-04-02|15|66|6|1901|2|4|2|2|1901|6|66|Tuesday|1901Q2|N|N|N|2415476|2415565|2415112|2415387|N|N|N|N|N| +2415478|AAAAAAAAGHLNECAA|1901-04-03|15|66|6|1901|3|4|3|2|1901|6|66|Wednesday|1901Q2|N|N|N|2415476|2415565|2415113|2415388|N|N|N|N|N| +2415479|AAAAAAAAHHLNECAA|1901-04-04|15|66|6|1901|4|4|4|2|1901|6|66|Thursday|1901Q2|N|N|N|2415476|2415565|2415114|2415389|N|N|N|N|N| +2415480|AAAAAAAAIHLNECAA|1901-04-05|15|66|6|1901|5|4|5|2|1901|6|66|Friday|1901Q2|N|Y|N|2415476|2415565|2415115|2415390|N|N|N|N|N| +2415481|AAAAAAAAJHLNECAA|1901-04-06|15|66|6|1901|6|4|6|2|1901|6|66|Saturday|1901Q2|N|Y|N|2415476|2415565|2415116|2415391|N|N|N|N|N| +2415482|AAAAAAAAKHLNECAA|1901-04-07|15|66|6|1901|0|4|7|2|1901|6|66|Sunday|1901Q2|N|N|N|2415476|2415565|2415117|2415392|N|N|N|N|N| +2415483|AAAAAAAALHLNECAA|1901-04-08|15|66|6|1901|1|4|8|2|1901|6|66|Monday|1901Q2|N|N|N|2415476|2415565|2415118|2415393|N|N|N|N|N| +2415484|AAAAAAAAMHLNECAA|1901-04-09|15|67|6|1901|2|4|9|2|1901|6|67|Tuesday|1901Q2|N|N|N|2415476|2415565|2415119|2415394|N|N|N|N|N| +2415485|AAAAAAAANHLNECAA|1901-04-10|15|67|6|1901|3|4|10|2|1901|6|67|Wednesday|1901Q2|N|N|N|2415476|2415565|2415120|2415395|N|N|N|N|N| +2415486|AAAAAAAAOHLNECAA|1901-04-11|15|67|6|1901|4|4|11|2|1901|6|67|Thursday|1901Q2|N|N|N|2415476|2415565|2415121|2415396|N|N|N|N|N| +2415487|AAAAAAAAPHLNECAA|1901-04-12|15|67|6|1901|5|4|12|2|1901|6|67|Friday|1901Q2|N|Y|N|2415476|2415565|2415122|2415397|N|N|N|N|N| +2415488|AAAAAAAAAILNECAA|1901-04-13|15|67|6|1901|6|4|13|2|1901|6|67|Saturday|1901Q2|N|Y|N|2415476|2415565|2415123|2415398|N|N|N|N|N| +2415489|AAAAAAAABILNECAA|1901-04-14|15|67|6|1901|0|4|14|2|1901|6|67|Sunday|1901Q2|N|N|N|2415476|2415565|2415124|2415399|N|N|N|N|N| +2415490|AAAAAAAACILNECAA|1901-04-15|15|67|6|1901|1|4|15|2|1901|6|67|Monday|1901Q2|N|N|N|2415476|2415565|2415125|2415400|N|N|N|N|N| +2415491|AAAAAAAADILNECAA|1901-04-16|15|68|6|1901|2|4|16|2|1901|6|68|Tuesday|1901Q2|N|N|N|2415476|2415565|2415126|2415401|N|N|N|N|N| +2415492|AAAAAAAAEILNECAA|1901-04-17|15|68|6|1901|3|4|17|2|1901|6|68|Wednesday|1901Q2|N|N|N|2415476|2415565|2415127|2415402|N|N|N|N|N| +2415493|AAAAAAAAFILNECAA|1901-04-18|15|68|6|1901|4|4|18|2|1901|6|68|Thursday|1901Q2|N|N|N|2415476|2415565|2415128|2415403|N|N|N|N|N| +2415494|AAAAAAAAGILNECAA|1901-04-19|15|68|6|1901|5|4|19|2|1901|6|68|Friday|1901Q2|N|Y|N|2415476|2415565|2415129|2415404|N|N|N|N|N| +2415495|AAAAAAAAHILNECAA|1901-04-20|15|68|6|1901|6|4|20|2|1901|6|68|Saturday|1901Q2|N|Y|N|2415476|2415565|2415130|2415405|N|N|N|N|N| +2415496|AAAAAAAAIILNECAA|1901-04-21|15|68|6|1901|0|4|21|2|1901|6|68|Sunday|1901Q2|N|N|N|2415476|2415565|2415131|2415406|N|N|N|N|N| +2415497|AAAAAAAAJILNECAA|1901-04-22|15|68|6|1901|1|4|22|2|1901|6|68|Monday|1901Q2|N|N|N|2415476|2415565|2415132|2415407|N|N|N|N|N| +2415498|AAAAAAAAKILNECAA|1901-04-23|15|69|6|1901|2|4|23|2|1901|6|69|Tuesday|1901Q2|N|N|N|2415476|2415565|2415133|2415408|N|N|N|N|N| +2415499|AAAAAAAALILNECAA|1901-04-24|15|69|6|1901|3|4|24|2|1901|6|69|Wednesday|1901Q2|N|N|N|2415476|2415565|2415134|2415409|N|N|N|N|N| +2415500|AAAAAAAAMILNECAA|1901-04-25|15|69|6|1901|4|4|25|2|1901|6|69|Thursday|1901Q2|N|N|N|2415476|2415565|2415135|2415410|N|N|N|N|N| +2415501|AAAAAAAANILNECAA|1901-04-26|15|69|6|1901|5|4|26|2|1901|6|69|Friday|1901Q2|N|Y|N|2415476|2415565|2415136|2415411|N|N|N|N|N| +2415502|AAAAAAAAOILNECAA|1901-04-27|15|69|6|1901|6|4|27|2|1901|6|69|Saturday|1901Q2|N|Y|N|2415476|2415565|2415137|2415412|N|N|N|N|N| +2415503|AAAAAAAAPILNECAA|1901-04-28|15|69|6|1901|0|4|28|2|1901|6|69|Sunday|1901Q2|N|N|N|2415476|2415565|2415138|2415413|N|N|N|N|N| +2415504|AAAAAAAAAJLNECAA|1901-04-29|15|69|6|1901|1|4|29|2|1901|6|69|Monday|1901Q2|N|N|N|2415476|2415565|2415139|2415414|N|N|N|N|N| +2415505|AAAAAAAABJLNECAA|1901-04-30|15|70|6|1901|2|4|30|2|1901|6|70|Tuesday|1901Q2|N|N|N|2415476|2415565|2415140|2415415|N|N|N|N|N| +2415506|AAAAAAAACJLNECAA|1901-05-01|16|70|6|1901|3|5|1|2|1901|6|70|Wednesday|1901Q2|N|N|N|2415506|2415625|2415141|2415416|N|N|N|N|N| +2415507|AAAAAAAADJLNECAA|1901-05-02|16|70|6|1901|4|5|2|2|1901|6|70|Thursday|1901Q2|N|N|N|2415506|2415625|2415142|2415417|N|N|N|N|N| +2415508|AAAAAAAAEJLNECAA|1901-05-03|16|70|6|1901|5|5|3|2|1901|6|70|Friday|1901Q2|N|Y|N|2415506|2415625|2415143|2415418|N|N|N|N|N| +2415509|AAAAAAAAFJLNECAA|1901-05-04|16|70|6|1901|6|5|4|2|1901|6|70|Saturday|1901Q2|N|Y|N|2415506|2415625|2415144|2415419|N|N|N|N|N| +2415510|AAAAAAAAGJLNECAA|1901-05-05|16|70|6|1901|0|5|5|2|1901|6|70|Sunday|1901Q2|N|N|N|2415506|2415625|2415145|2415420|N|N|N|N|N| +2415511|AAAAAAAAHJLNECAA|1901-05-06|16|70|6|1901|1|5|6|2|1901|6|70|Monday|1901Q2|N|N|N|2415506|2415625|2415146|2415421|N|N|N|N|N| +2415512|AAAAAAAAIJLNECAA|1901-05-07|16|71|6|1901|2|5|7|2|1901|6|71|Tuesday|1901Q2|N|N|N|2415506|2415625|2415147|2415422|N|N|N|N|N| +2415513|AAAAAAAAJJLNECAA|1901-05-08|16|71|6|1901|3|5|8|2|1901|6|71|Wednesday|1901Q2|N|N|N|2415506|2415625|2415148|2415423|N|N|N|N|N| +2415514|AAAAAAAAKJLNECAA|1901-05-09|16|71|6|1901|4|5|9|2|1901|6|71|Thursday|1901Q2|N|N|N|2415506|2415625|2415149|2415424|N|N|N|N|N| +2415515|AAAAAAAALJLNECAA|1901-05-10|16|71|6|1901|5|5|10|2|1901|6|71|Friday|1901Q2|N|Y|N|2415506|2415625|2415150|2415425|N|N|N|N|N| +2415516|AAAAAAAAMJLNECAA|1901-05-11|16|71|6|1901|6|5|11|2|1901|6|71|Saturday|1901Q2|N|Y|N|2415506|2415625|2415151|2415426|N|N|N|N|N| +2415517|AAAAAAAANJLNECAA|1901-05-12|16|71|6|1901|0|5|12|2|1901|6|71|Sunday|1901Q2|N|N|N|2415506|2415625|2415152|2415427|N|N|N|N|N| +2415518|AAAAAAAAOJLNECAA|1901-05-13|16|71|6|1901|1|5|13|2|1901|6|71|Monday|1901Q2|N|N|N|2415506|2415625|2415153|2415428|N|N|N|N|N| +2415519|AAAAAAAAPJLNECAA|1901-05-14|16|72|6|1901|2|5|14|2|1901|6|72|Tuesday|1901Q2|N|N|N|2415506|2415625|2415154|2415429|N|N|N|N|N| +2415520|AAAAAAAAAKLNECAA|1901-05-15|16|72|6|1901|3|5|15|2|1901|6|72|Wednesday|1901Q2|N|N|N|2415506|2415625|2415155|2415430|N|N|N|N|N| +2415521|AAAAAAAABKLNECAA|1901-05-16|16|72|6|1901|4|5|16|2|1901|6|72|Thursday|1901Q2|N|N|N|2415506|2415625|2415156|2415431|N|N|N|N|N| +2415522|AAAAAAAACKLNECAA|1901-05-17|16|72|6|1901|5|5|17|2|1901|6|72|Friday|1901Q2|N|Y|N|2415506|2415625|2415157|2415432|N|N|N|N|N| +2415523|AAAAAAAADKLNECAA|1901-05-18|16|72|6|1901|6|5|18|2|1901|6|72|Saturday|1901Q2|N|Y|N|2415506|2415625|2415158|2415433|N|N|N|N|N| +2415524|AAAAAAAAEKLNECAA|1901-05-19|16|72|6|1901|0|5|19|2|1901|6|72|Sunday|1901Q2|N|N|N|2415506|2415625|2415159|2415434|N|N|N|N|N| +2415525|AAAAAAAAFKLNECAA|1901-05-20|16|72|6|1901|1|5|20|2|1901|6|72|Monday|1901Q2|N|N|N|2415506|2415625|2415160|2415435|N|N|N|N|N| +2415526|AAAAAAAAGKLNECAA|1901-05-21|16|73|6|1901|2|5|21|2|1901|6|73|Tuesday|1901Q2|N|N|N|2415506|2415625|2415161|2415436|N|N|N|N|N| +2415527|AAAAAAAAHKLNECAA|1901-05-22|16|73|6|1901|3|5|22|2|1901|6|73|Wednesday|1901Q2|N|N|N|2415506|2415625|2415162|2415437|N|N|N|N|N| +2415528|AAAAAAAAIKLNECAA|1901-05-23|16|73|6|1901|4|5|23|2|1901|6|73|Thursday|1901Q2|N|N|N|2415506|2415625|2415163|2415438|N|N|N|N|N| +2415529|AAAAAAAAJKLNECAA|1901-05-24|16|73|6|1901|5|5|24|2|1901|6|73|Friday|1901Q2|N|Y|N|2415506|2415625|2415164|2415439|N|N|N|N|N| +2415530|AAAAAAAAKKLNECAA|1901-05-25|16|73|6|1901|6|5|25|2|1901|6|73|Saturday|1901Q2|N|Y|N|2415506|2415625|2415165|2415440|N|N|N|N|N| +2415531|AAAAAAAALKLNECAA|1901-05-26|16|73|6|1901|0|5|26|2|1901|6|73|Sunday|1901Q2|N|N|N|2415506|2415625|2415166|2415441|N|N|N|N|N| +2415532|AAAAAAAAMKLNECAA|1901-05-27|16|73|6|1901|1|5|27|2|1901|6|73|Monday|1901Q2|N|N|N|2415506|2415625|2415167|2415442|N|N|N|N|N| +2415533|AAAAAAAANKLNECAA|1901-05-28|16|74|6|1901|2|5|28|2|1901|6|74|Tuesday|1901Q2|N|N|N|2415506|2415625|2415168|2415443|N|N|N|N|N| +2415534|AAAAAAAAOKLNECAA|1901-05-29|16|74|6|1901|3|5|29|2|1901|6|74|Wednesday|1901Q2|N|N|N|2415506|2415625|2415169|2415444|N|N|N|N|N| +2415535|AAAAAAAAPKLNECAA|1901-05-30|16|74|6|1901|4|5|30|2|1901|6|74|Thursday|1901Q2|N|N|N|2415506|2415625|2415170|2415445|N|N|N|N|N| +2415536|AAAAAAAAALLNECAA|1901-05-31|16|74|6|1901|5|5|31|2|1901|6|74|Friday|1901Q2|N|Y|N|2415506|2415625|2415171|2415446|N|N|N|N|N| +2415537|AAAAAAAABLLNECAA|1901-06-01|17|74|7|1901|6|6|1|2|1901|7|74|Saturday|1901Q2|N|Y|N|2415537|2415687|2415172|2415447|N|N|N|N|N| +2415538|AAAAAAAACLLNECAA|1901-06-02|17|74|7|1901|0|6|2|2|1901|7|74|Sunday|1901Q2|N|N|N|2415537|2415687|2415173|2415448|N|N|N|N|N| +2415539|AAAAAAAADLLNECAA|1901-06-03|17|74|7|1901|1|6|3|2|1901|7|74|Monday|1901Q2|N|N|N|2415537|2415687|2415174|2415449|N|N|N|N|N| +2415540|AAAAAAAAELLNECAA|1901-06-04|17|75|7|1901|2|6|4|2|1901|7|75|Tuesday|1901Q2|N|N|N|2415537|2415687|2415175|2415450|N|N|N|N|N| +2415541|AAAAAAAAFLLNECAA|1901-06-05|17|75|7|1901|3|6|5|2|1901|7|75|Wednesday|1901Q2|N|N|N|2415537|2415687|2415176|2415451|N|N|N|N|N| +2415542|AAAAAAAAGLLNECAA|1901-06-06|17|75|7|1901|4|6|6|2|1901|7|75|Thursday|1901Q2|N|N|N|2415537|2415687|2415177|2415452|N|N|N|N|N| +2415543|AAAAAAAAHLLNECAA|1901-06-07|17|75|7|1901|5|6|7|2|1901|7|75|Friday|1901Q2|N|Y|N|2415537|2415687|2415178|2415453|N|N|N|N|N| +2415544|AAAAAAAAILLNECAA|1901-06-08|17|75|7|1901|6|6|8|2|1901|7|75|Saturday|1901Q2|N|Y|N|2415537|2415687|2415179|2415454|N|N|N|N|N| +2415545|AAAAAAAAJLLNECAA|1901-06-09|17|75|7|1901|0|6|9|2|1901|7|75|Sunday|1901Q2|N|N|N|2415537|2415687|2415180|2415455|N|N|N|N|N| +2415546|AAAAAAAAKLLNECAA|1901-06-10|17|75|7|1901|1|6|10|2|1901|7|75|Monday|1901Q2|N|N|N|2415537|2415687|2415181|2415456|N|N|N|N|N| +2415547|AAAAAAAALLLNECAA|1901-06-11|17|76|7|1901|2|6|11|2|1901|7|76|Tuesday|1901Q2|N|N|N|2415537|2415687|2415182|2415457|N|N|N|N|N| +2415548|AAAAAAAAMLLNECAA|1901-06-12|17|76|7|1901|3|6|12|2|1901|7|76|Wednesday|1901Q2|N|N|N|2415537|2415687|2415183|2415458|N|N|N|N|N| +2415549|AAAAAAAANLLNECAA|1901-06-13|17|76|7|1901|4|6|13|2|1901|7|76|Thursday|1901Q2|N|N|N|2415537|2415687|2415184|2415459|N|N|N|N|N| +2415550|AAAAAAAAOLLNECAA|1901-06-14|17|76|7|1901|5|6|14|2|1901|7|76|Friday|1901Q2|N|Y|N|2415537|2415687|2415185|2415460|N|N|N|N|N| +2415551|AAAAAAAAPLLNECAA|1901-06-15|17|76|7|1901|6|6|15|2|1901|7|76|Saturday|1901Q2|N|Y|N|2415537|2415687|2415186|2415461|N|N|N|N|N| +2415552|AAAAAAAAAMLNECAA|1901-06-16|17|76|7|1901|0|6|16|2|1901|7|76|Sunday|1901Q2|N|N|N|2415537|2415687|2415187|2415462|N|N|N|N|N| +2415553|AAAAAAAABMLNECAA|1901-06-17|17|76|7|1901|1|6|17|2|1901|7|76|Monday|1901Q2|N|N|N|2415537|2415687|2415188|2415463|N|N|N|N|N| +2415554|AAAAAAAACMLNECAA|1901-06-18|17|77|7|1901|2|6|18|2|1901|7|77|Tuesday|1901Q2|N|N|N|2415537|2415687|2415189|2415464|N|N|N|N|N| +2415555|AAAAAAAADMLNECAA|1901-06-19|17|77|7|1901|3|6|19|2|1901|7|77|Wednesday|1901Q2|N|N|N|2415537|2415687|2415190|2415465|N|N|N|N|N| +2415556|AAAAAAAAEMLNECAA|1901-06-20|17|77|7|1901|4|6|20|2|1901|7|77|Thursday|1901Q2|N|N|N|2415537|2415687|2415191|2415466|N|N|N|N|N| +2415557|AAAAAAAAFMLNECAA|1901-06-21|17|77|7|1901|5|6|21|2|1901|7|77|Friday|1901Q2|N|Y|N|2415537|2415687|2415192|2415467|N|N|N|N|N| +2415558|AAAAAAAAGMLNECAA|1901-06-22|17|77|7|1901|6|6|22|2|1901|7|77|Saturday|1901Q2|N|Y|N|2415537|2415687|2415193|2415468|N|N|N|N|N| +2415559|AAAAAAAAHMLNECAA|1901-06-23|17|77|7|1901|0|6|23|2|1901|7|77|Sunday|1901Q2|N|N|N|2415537|2415687|2415194|2415469|N|N|N|N|N| +2415560|AAAAAAAAIMLNECAA|1901-06-24|17|77|7|1901|1|6|24|2|1901|7|77|Monday|1901Q2|N|N|N|2415537|2415687|2415195|2415470|N|N|N|N|N| +2415561|AAAAAAAAJMLNECAA|1901-06-25|17|78|7|1901|2|6|25|2|1901|7|78|Tuesday|1901Q2|N|N|N|2415537|2415687|2415196|2415471|N|N|N|N|N| +2415562|AAAAAAAAKMLNECAA|1901-06-26|17|78|7|1901|3|6|26|2|1901|7|78|Wednesday|1901Q2|N|N|N|2415537|2415687|2415197|2415472|N|N|N|N|N| +2415563|AAAAAAAALMLNECAA|1901-06-27|17|78|7|1901|4|6|27|2|1901|7|78|Thursday|1901Q2|N|N|N|2415537|2415687|2415198|2415473|N|N|N|N|N| +2415564|AAAAAAAAMMLNECAA|1901-06-28|17|78|7|1901|5|6|28|2|1901|7|78|Friday|1901Q2|N|Y|N|2415537|2415687|2415199|2415474|N|N|N|N|N| +2415565|AAAAAAAANMLNECAA|1901-06-29|17|78|7|1901|6|6|29|2|1901|7|78|Saturday|1901Q2|N|Y|N|2415537|2415687|2415200|2415475|N|N|N|N|N| +2415566|AAAAAAAAOMLNECAA|1901-06-30|17|78|7|1901|0|6|30|2|1901|7|78|Sunday|1901Q2|N|N|N|2415537|2415687|2415201|2415476|N|N|N|N|N| +2415567|AAAAAAAAPMLNECAA|1901-07-01|18|78|7|1901|1|7|1|2|1901|7|78|Monday|1901Q2|N|N|N|2415567|2415747|2415202|2415476|N|N|N|N|N| +2415568|AAAAAAAAANLNECAA|1901-07-02|18|79|7|1901|2|7|2|3|1901|7|79|Tuesday|1901Q3|N|N|N|2415567|2415747|2415203|2415477|N|N|N|N|N| +2415569|AAAAAAAABNLNECAA|1901-07-03|18|79|7|1901|3|7|3|3|1901|7|79|Wednesday|1901Q3|N|N|N|2415567|2415747|2415204|2415478|N|N|N|N|N| +2415570|AAAAAAAACNLNECAA|1901-07-04|18|79|7|1901|4|7|4|3|1901|7|79|Thursday|1901Q3|N|N|N|2415567|2415747|2415205|2415479|N|N|N|N|N| +2415571|AAAAAAAADNLNECAA|1901-07-05|18|79|7|1901|5|7|5|3|1901|7|79|Friday|1901Q3|Y|Y|N|2415567|2415747|2415206|2415480|N|N|N|N|N| +2415572|AAAAAAAAENLNECAA|1901-07-06|18|79|7|1901|6|7|6|3|1901|7|79|Saturday|1901Q3|N|Y|Y|2415567|2415747|2415207|2415481|N|N|N|N|N| +2415573|AAAAAAAAFNLNECAA|1901-07-07|18|79|7|1901|0|7|7|3|1901|7|79|Sunday|1901Q3|N|N|N|2415567|2415747|2415208|2415482|N|N|N|N|N| +2415574|AAAAAAAAGNLNECAA|1901-07-08|18|79|7|1901|1|7|8|3|1901|7|79|Monday|1901Q3|N|N|N|2415567|2415747|2415209|2415483|N|N|N|N|N| +2415575|AAAAAAAAHNLNECAA|1901-07-09|18|80|7|1901|2|7|9|3|1901|7|80|Tuesday|1901Q3|N|N|N|2415567|2415747|2415210|2415484|N|N|N|N|N| +2415576|AAAAAAAAINLNECAA|1901-07-10|18|80|7|1901|3|7|10|3|1901|7|80|Wednesday|1901Q3|N|N|N|2415567|2415747|2415211|2415485|N|N|N|N|N| +2415577|AAAAAAAAJNLNECAA|1901-07-11|18|80|7|1901|4|7|11|3|1901|7|80|Thursday|1901Q3|N|N|N|2415567|2415747|2415212|2415486|N|N|N|N|N| +2415578|AAAAAAAAKNLNECAA|1901-07-12|18|80|7|1901|5|7|12|3|1901|7|80|Friday|1901Q3|N|Y|N|2415567|2415747|2415213|2415487|N|N|N|N|N| +2415579|AAAAAAAALNLNECAA|1901-07-13|18|80|7|1901|6|7|13|3|1901|7|80|Saturday|1901Q3|N|Y|N|2415567|2415747|2415214|2415488|N|N|N|N|N| +2415580|AAAAAAAAMNLNECAA|1901-07-14|18|80|7|1901|0|7|14|3|1901|7|80|Sunday|1901Q3|N|N|N|2415567|2415747|2415215|2415489|N|N|N|N|N| +2415581|AAAAAAAANNLNECAA|1901-07-15|18|80|7|1901|1|7|15|3|1901|7|80|Monday|1901Q3|N|N|N|2415567|2415747|2415216|2415490|N|N|N|N|N| +2415582|AAAAAAAAONLNECAA|1901-07-16|18|81|7|1901|2|7|16|3|1901|7|81|Tuesday|1901Q3|N|N|N|2415567|2415747|2415217|2415491|N|N|N|N|N| +2415583|AAAAAAAAPNLNECAA|1901-07-17|18|81|7|1901|3|7|17|3|1901|7|81|Wednesday|1901Q3|N|N|N|2415567|2415747|2415218|2415492|N|N|N|N|N| +2415584|AAAAAAAAAOLNECAA|1901-07-18|18|81|7|1901|4|7|18|3|1901|7|81|Thursday|1901Q3|N|N|N|2415567|2415747|2415219|2415493|N|N|N|N|N| +2415585|AAAAAAAABOLNECAA|1901-07-19|18|81|7|1901|5|7|19|3|1901|7|81|Friday|1901Q3|N|Y|N|2415567|2415747|2415220|2415494|N|N|N|N|N| +2415586|AAAAAAAACOLNECAA|1901-07-20|18|81|7|1901|6|7|20|3|1901|7|81|Saturday|1901Q3|N|Y|N|2415567|2415747|2415221|2415495|N|N|N|N|N| +2415587|AAAAAAAADOLNECAA|1901-07-21|18|81|7|1901|0|7|21|3|1901|7|81|Sunday|1901Q3|N|N|N|2415567|2415747|2415222|2415496|N|N|N|N|N| +2415588|AAAAAAAAEOLNECAA|1901-07-22|18|81|7|1901|1|7|22|3|1901|7|81|Monday|1901Q3|N|N|N|2415567|2415747|2415223|2415497|N|N|N|N|N| +2415589|AAAAAAAAFOLNECAA|1901-07-23|18|82|7|1901|2|7|23|3|1901|7|82|Tuesday|1901Q3|N|N|N|2415567|2415747|2415224|2415498|N|N|N|N|N| +2415590|AAAAAAAAGOLNECAA|1901-07-24|18|82|7|1901|3|7|24|3|1901|7|82|Wednesday|1901Q3|N|N|N|2415567|2415747|2415225|2415499|N|N|N|N|N| +2415591|AAAAAAAAHOLNECAA|1901-07-25|18|82|7|1901|4|7|25|3|1901|7|82|Thursday|1901Q3|N|N|N|2415567|2415747|2415226|2415500|N|N|N|N|N| +2415592|AAAAAAAAIOLNECAA|1901-07-26|18|82|7|1901|5|7|26|3|1901|7|82|Friday|1901Q3|N|Y|N|2415567|2415747|2415227|2415501|N|N|N|N|N| +2415593|AAAAAAAAJOLNECAA|1901-07-27|18|82|7|1901|6|7|27|3|1901|7|82|Saturday|1901Q3|N|Y|N|2415567|2415747|2415228|2415502|N|N|N|N|N| +2415594|AAAAAAAAKOLNECAA|1901-07-28|18|82|7|1901|0|7|28|3|1901|7|82|Sunday|1901Q3|N|N|N|2415567|2415747|2415229|2415503|N|N|N|N|N| +2415595|AAAAAAAALOLNECAA|1901-07-29|18|82|7|1901|1|7|29|3|1901|7|82|Monday|1901Q3|N|N|N|2415567|2415747|2415230|2415504|N|N|N|N|N| +2415596|AAAAAAAAMOLNECAA|1901-07-30|18|83|7|1901|2|7|30|3|1901|7|83|Tuesday|1901Q3|N|N|N|2415567|2415747|2415231|2415505|N|N|N|N|N| +2415597|AAAAAAAANOLNECAA|1901-07-31|18|83|7|1901|3|7|31|3|1901|7|83|Wednesday|1901Q3|N|N|N|2415567|2415747|2415232|2415506|N|N|N|N|N| +2415598|AAAAAAAAOOLNECAA|1901-08-01|19|83|7|1901|4|8|1|3|1901|7|83|Thursday|1901Q3|N|N|N|2415598|2415809|2415233|2415507|N|N|N|N|N| +2415599|AAAAAAAAPOLNECAA|1901-08-02|19|83|7|1901|5|8|2|3|1901|7|83|Friday|1901Q3|N|Y|N|2415598|2415809|2415234|2415508|N|N|N|N|N| +2415600|AAAAAAAAAPLNECAA|1901-08-03|19|83|7|1901|6|8|3|3|1901|7|83|Saturday|1901Q3|N|Y|N|2415598|2415809|2415235|2415509|N|N|N|N|N| +2415601|AAAAAAAABPLNECAA|1901-08-04|19|83|7|1901|0|8|4|3|1901|7|83|Sunday|1901Q3|N|N|N|2415598|2415809|2415236|2415510|N|N|N|N|N| +2415602|AAAAAAAACPLNECAA|1901-08-05|19|83|7|1901|1|8|5|3|1901|7|83|Monday|1901Q3|N|N|N|2415598|2415809|2415237|2415511|N|N|N|N|N| +2415603|AAAAAAAADPLNECAA|1901-08-06|19|84|7|1901|2|8|6|3|1901|7|84|Tuesday|1901Q3|N|N|N|2415598|2415809|2415238|2415512|N|N|N|N|N| +2415604|AAAAAAAAEPLNECAA|1901-08-07|19|84|7|1901|3|8|7|3|1901|7|84|Wednesday|1901Q3|N|N|N|2415598|2415809|2415239|2415513|N|N|N|N|N| +2415605|AAAAAAAAFPLNECAA|1901-08-08|19|84|7|1901|4|8|8|3|1901|7|84|Thursday|1901Q3|N|N|N|2415598|2415809|2415240|2415514|N|N|N|N|N| +2415606|AAAAAAAAGPLNECAA|1901-08-09|19|84|7|1901|5|8|9|3|1901|7|84|Friday|1901Q3|N|Y|N|2415598|2415809|2415241|2415515|N|N|N|N|N| +2415607|AAAAAAAAHPLNECAA|1901-08-10|19|84|7|1901|6|8|10|3|1901|7|84|Saturday|1901Q3|N|Y|N|2415598|2415809|2415242|2415516|N|N|N|N|N| +2415608|AAAAAAAAIPLNECAA|1901-08-11|19|84|7|1901|0|8|11|3|1901|7|84|Sunday|1901Q3|N|N|N|2415598|2415809|2415243|2415517|N|N|N|N|N| +2415609|AAAAAAAAJPLNECAA|1901-08-12|19|84|7|1901|1|8|12|3|1901|7|84|Monday|1901Q3|N|N|N|2415598|2415809|2415244|2415518|N|N|N|N|N| +2415610|AAAAAAAAKPLNECAA|1901-08-13|19|85|7|1901|2|8|13|3|1901|7|85|Tuesday|1901Q3|N|N|N|2415598|2415809|2415245|2415519|N|N|N|N|N| +2415611|AAAAAAAALPLNECAA|1901-08-14|19|85|7|1901|3|8|14|3|1901|7|85|Wednesday|1901Q3|N|N|N|2415598|2415809|2415246|2415520|N|N|N|N|N| +2415612|AAAAAAAAMPLNECAA|1901-08-15|19|85|7|1901|4|8|15|3|1901|7|85|Thursday|1901Q3|N|N|N|2415598|2415809|2415247|2415521|N|N|N|N|N| +2415613|AAAAAAAANPLNECAA|1901-08-16|19|85|7|1901|5|8|16|3|1901|7|85|Friday|1901Q3|N|Y|N|2415598|2415809|2415248|2415522|N|N|N|N|N| +2415614|AAAAAAAAOPLNECAA|1901-08-17|19|85|7|1901|6|8|17|3|1901|7|85|Saturday|1901Q3|N|Y|N|2415598|2415809|2415249|2415523|N|N|N|N|N| +2415615|AAAAAAAAPPLNECAA|1901-08-18|19|85|7|1901|0|8|18|3|1901|7|85|Sunday|1901Q3|N|N|N|2415598|2415809|2415250|2415524|N|N|N|N|N| +2415616|AAAAAAAAAAMNECAA|1901-08-19|19|85|7|1901|1|8|19|3|1901|7|85|Monday|1901Q3|N|N|N|2415598|2415809|2415251|2415525|N|N|N|N|N| +2415617|AAAAAAAABAMNECAA|1901-08-20|19|86|7|1901|2|8|20|3|1901|7|86|Tuesday|1901Q3|N|N|N|2415598|2415809|2415252|2415526|N|N|N|N|N| +2415618|AAAAAAAACAMNECAA|1901-08-21|19|86|7|1901|3|8|21|3|1901|7|86|Wednesday|1901Q3|N|N|N|2415598|2415809|2415253|2415527|N|N|N|N|N| +2415619|AAAAAAAADAMNECAA|1901-08-22|19|86|7|1901|4|8|22|3|1901|7|86|Thursday|1901Q3|N|N|N|2415598|2415809|2415254|2415528|N|N|N|N|N| +2415620|AAAAAAAAEAMNECAA|1901-08-23|19|86|7|1901|5|8|23|3|1901|7|86|Friday|1901Q3|N|Y|N|2415598|2415809|2415255|2415529|N|N|N|N|N| +2415621|AAAAAAAAFAMNECAA|1901-08-24|19|86|7|1901|6|8|24|3|1901|7|86|Saturday|1901Q3|N|Y|N|2415598|2415809|2415256|2415530|N|N|N|N|N| +2415622|AAAAAAAAGAMNECAA|1901-08-25|19|86|7|1901|0|8|25|3|1901|7|86|Sunday|1901Q3|N|N|N|2415598|2415809|2415257|2415531|N|N|N|N|N| +2415623|AAAAAAAAHAMNECAA|1901-08-26|19|86|7|1901|1|8|26|3|1901|7|86|Monday|1901Q3|N|N|N|2415598|2415809|2415258|2415532|N|N|N|N|N| +2415624|AAAAAAAAIAMNECAA|1901-08-27|19|87|7|1901|2|8|27|3|1901|7|87|Tuesday|1901Q3|N|N|N|2415598|2415809|2415259|2415533|N|N|N|N|N| +2415625|AAAAAAAAJAMNECAA|1901-08-28|19|87|7|1901|3|8|28|3|1901|7|87|Wednesday|1901Q3|N|N|N|2415598|2415809|2415260|2415534|N|N|N|N|N| +2415626|AAAAAAAAKAMNECAA|1901-08-29|19|87|7|1901|4|8|29|3|1901|7|87|Thursday|1901Q3|N|N|N|2415598|2415809|2415261|2415535|N|N|N|N|N| +2415627|AAAAAAAALAMNECAA|1901-08-30|19|87|7|1901|5|8|30|3|1901|7|87|Friday|1901Q3|N|Y|N|2415598|2415809|2415262|2415536|N|N|N|N|N| +2415628|AAAAAAAAMAMNECAA|1901-08-31|19|87|7|1901|6|8|31|3|1901|7|87|Saturday|1901Q3|N|Y|N|2415598|2415809|2415263|2415537|N|N|N|N|N| +2415629|AAAAAAAANAMNECAA|1901-09-01|20|87|8|1901|0|9|1|3|1901|8|87|Sunday|1901Q3|N|N|N|2415629|2415871|2415264|2415538|N|N|N|N|N| +2415630|AAAAAAAAOAMNECAA|1901-09-02|20|87|8|1901|1|9|2|3|1901|8|87|Monday|1901Q3|N|N|N|2415629|2415871|2415265|2415539|N|N|N|N|N| +2415631|AAAAAAAAPAMNECAA|1901-09-03|20|88|8|1901|2|9|3|3|1901|8|88|Tuesday|1901Q3|N|N|N|2415629|2415871|2415266|2415540|N|N|N|N|N| +2415632|AAAAAAAAABMNECAA|1901-09-04|20|88|8|1901|3|9|4|3|1901|8|88|Wednesday|1901Q3|N|N|N|2415629|2415871|2415267|2415541|N|N|N|N|N| +2415633|AAAAAAAABBMNECAA|1901-09-05|20|88|8|1901|4|9|5|3|1901|8|88|Thursday|1901Q3|N|N|N|2415629|2415871|2415268|2415542|N|N|N|N|N| +2415634|AAAAAAAACBMNECAA|1901-09-06|20|88|8|1901|5|9|6|3|1901|8|88|Friday|1901Q3|N|Y|N|2415629|2415871|2415269|2415543|N|N|N|N|N| +2415635|AAAAAAAADBMNECAA|1901-09-07|20|88|8|1901|6|9|7|3|1901|8|88|Saturday|1901Q3|N|Y|N|2415629|2415871|2415270|2415544|N|N|N|N|N| +2415636|AAAAAAAAEBMNECAA|1901-09-08|20|88|8|1901|0|9|8|3|1901|8|88|Sunday|1901Q3|N|N|N|2415629|2415871|2415271|2415545|N|N|N|N|N| +2415637|AAAAAAAAFBMNECAA|1901-09-09|20|88|8|1901|1|9|9|3|1901|8|88|Monday|1901Q3|N|N|N|2415629|2415871|2415272|2415546|N|N|N|N|N| +2415638|AAAAAAAAGBMNECAA|1901-09-10|20|89|8|1901|2|9|10|3|1901|8|89|Tuesday|1901Q3|N|N|N|2415629|2415871|2415273|2415547|N|N|N|N|N| +2415639|AAAAAAAAHBMNECAA|1901-09-11|20|89|8|1901|3|9|11|3|1901|8|89|Wednesday|1901Q3|N|N|N|2415629|2415871|2415274|2415548|N|N|N|N|N| +2415640|AAAAAAAAIBMNECAA|1901-09-12|20|89|8|1901|4|9|12|3|1901|8|89|Thursday|1901Q3|N|N|N|2415629|2415871|2415275|2415549|N|N|N|N|N| +2415641|AAAAAAAAJBMNECAA|1901-09-13|20|89|8|1901|5|9|13|3|1901|8|89|Friday|1901Q3|N|Y|N|2415629|2415871|2415276|2415550|N|N|N|N|N| +2415642|AAAAAAAAKBMNECAA|1901-09-14|20|89|8|1901|6|9|14|3|1901|8|89|Saturday|1901Q3|N|Y|N|2415629|2415871|2415277|2415551|N|N|N|N|N| +2415643|AAAAAAAALBMNECAA|1901-09-15|20|89|8|1901|0|9|15|3|1901|8|89|Sunday|1901Q3|N|N|N|2415629|2415871|2415278|2415552|N|N|N|N|N| +2415644|AAAAAAAAMBMNECAA|1901-09-16|20|89|8|1901|1|9|16|3|1901|8|89|Monday|1901Q3|N|N|N|2415629|2415871|2415279|2415553|N|N|N|N|N| +2415645|AAAAAAAANBMNECAA|1901-09-17|20|90|8|1901|2|9|17|3|1901|8|90|Tuesday|1901Q3|N|N|N|2415629|2415871|2415280|2415554|N|N|N|N|N| +2415646|AAAAAAAAOBMNECAA|1901-09-18|20|90|8|1901|3|9|18|3|1901|8|90|Wednesday|1901Q3|N|N|N|2415629|2415871|2415281|2415555|N|N|N|N|N| +2415647|AAAAAAAAPBMNECAA|1901-09-19|20|90|8|1901|4|9|19|3|1901|8|90|Thursday|1901Q3|N|N|N|2415629|2415871|2415282|2415556|N|N|N|N|N| +2415648|AAAAAAAAACMNECAA|1901-09-20|20|90|8|1901|5|9|20|3|1901|8|90|Friday|1901Q3|N|Y|N|2415629|2415871|2415283|2415557|N|N|N|N|N| +2415649|AAAAAAAABCMNECAA|1901-09-21|20|90|8|1901|6|9|21|3|1901|8|90|Saturday|1901Q3|N|Y|N|2415629|2415871|2415284|2415558|N|N|N|N|N| +2415650|AAAAAAAACCMNECAA|1901-09-22|20|90|8|1901|0|9|22|3|1901|8|90|Sunday|1901Q3|N|N|N|2415629|2415871|2415285|2415559|N|N|N|N|N| +2415651|AAAAAAAADCMNECAA|1901-09-23|20|90|8|1901|1|9|23|3|1901|8|90|Monday|1901Q3|N|N|N|2415629|2415871|2415286|2415560|N|N|N|N|N| +2415652|AAAAAAAAECMNECAA|1901-09-24|20|91|8|1901|2|9|24|3|1901|8|91|Tuesday|1901Q3|N|N|N|2415629|2415871|2415287|2415561|N|N|N|N|N| +2415653|AAAAAAAAFCMNECAA|1901-09-25|20|91|8|1901|3|9|25|3|1901|8|91|Wednesday|1901Q3|N|N|N|2415629|2415871|2415288|2415562|N|N|N|N|N| +2415654|AAAAAAAAGCMNECAA|1901-09-26|20|91|8|1901|4|9|26|3|1901|8|91|Thursday|1901Q3|N|N|N|2415629|2415871|2415289|2415563|N|N|N|N|N| +2415655|AAAAAAAAHCMNECAA|1901-09-27|20|91|8|1901|5|9|27|3|1901|8|91|Friday|1901Q3|N|Y|N|2415629|2415871|2415290|2415564|N|N|N|N|N| +2415656|AAAAAAAAICMNECAA|1901-09-28|20|91|8|1901|6|9|28|3|1901|8|91|Saturday|1901Q3|N|Y|N|2415629|2415871|2415291|2415565|N|N|N|N|N| +2415657|AAAAAAAAJCMNECAA|1901-09-29|20|91|8|1901|0|9|29|3|1901|8|91|Sunday|1901Q3|N|N|N|2415629|2415871|2415292|2415566|N|N|N|N|N| +2415658|AAAAAAAAKCMNECAA|1901-09-30|20|91|8|1901|1|9|30|3|1901|8|91|Monday|1901Q3|N|N|N|2415629|2415871|2415293|2415567|N|N|N|N|N| +2415659|AAAAAAAALCMNECAA|1901-10-01|21|92|8|1901|2|10|1|3|1901|8|92|Tuesday|1901Q3|N|N|N|2415659|2415931|2415294|2415567|N|N|N|N|N| +2415660|AAAAAAAAMCMNECAA|1901-10-02|21|92|8|1901|3|10|2|4|1901|8|92|Wednesday|1901Q4|N|N|N|2415659|2415931|2415295|2415568|N|N|N|N|N| +2415661|AAAAAAAANCMNECAA|1901-10-03|21|92|8|1901|4|10|3|4|1901|8|92|Thursday|1901Q4|N|N|N|2415659|2415931|2415296|2415569|N|N|N|N|N| +2415662|AAAAAAAAOCMNECAA|1901-10-04|21|92|8|1901|5|10|4|4|1901|8|92|Friday|1901Q4|N|Y|N|2415659|2415931|2415297|2415570|N|N|N|N|N| +2415663|AAAAAAAAPCMNECAA|1901-10-05|21|92|8|1901|6|10|5|4|1901|8|92|Saturday|1901Q4|N|Y|N|2415659|2415931|2415298|2415571|N|N|N|N|N| +2415664|AAAAAAAAADMNECAA|1901-10-06|21|92|8|1901|0|10|6|4|1901|8|92|Sunday|1901Q4|N|N|N|2415659|2415931|2415299|2415572|N|N|N|N|N| +2415665|AAAAAAAABDMNECAA|1901-10-07|21|92|8|1901|1|10|7|4|1901|8|92|Monday|1901Q4|N|N|N|2415659|2415931|2415300|2415573|N|N|N|N|N| +2415666|AAAAAAAACDMNECAA|1901-10-08|21|93|8|1901|2|10|8|4|1901|8|93|Tuesday|1901Q4|N|N|N|2415659|2415931|2415301|2415574|N|N|N|N|N| +2415667|AAAAAAAADDMNECAA|1901-10-09|21|93|8|1901|3|10|9|4|1901|8|93|Wednesday|1901Q4|N|N|N|2415659|2415931|2415302|2415575|N|N|N|N|N| +2415668|AAAAAAAAEDMNECAA|1901-10-10|21|93|8|1901|4|10|10|4|1901|8|93|Thursday|1901Q4|N|N|N|2415659|2415931|2415303|2415576|N|N|N|N|N| +2415669|AAAAAAAAFDMNECAA|1901-10-11|21|93|8|1901|5|10|11|4|1901|8|93|Friday|1901Q4|N|Y|N|2415659|2415931|2415304|2415577|N|N|N|N|N| +2415670|AAAAAAAAGDMNECAA|1901-10-12|21|93|8|1901|6|10|12|4|1901|8|93|Saturday|1901Q4|N|Y|N|2415659|2415931|2415305|2415578|N|N|N|N|N| +2415671|AAAAAAAAHDMNECAA|1901-10-13|21|93|8|1901|0|10|13|4|1901|8|93|Sunday|1901Q4|N|N|N|2415659|2415931|2415306|2415579|N|N|N|N|N| +2415672|AAAAAAAAIDMNECAA|1901-10-14|21|93|8|1901|1|10|14|4|1901|8|93|Monday|1901Q4|N|N|N|2415659|2415931|2415307|2415580|N|N|N|N|N| +2415673|AAAAAAAAJDMNECAA|1901-10-15|21|94|8|1901|2|10|15|4|1901|8|94|Tuesday|1901Q4|N|N|N|2415659|2415931|2415308|2415581|N|N|N|N|N| +2415674|AAAAAAAAKDMNECAA|1901-10-16|21|94|8|1901|3|10|16|4|1901|8|94|Wednesday|1901Q4|N|N|N|2415659|2415931|2415309|2415582|N|N|N|N|N| +2415675|AAAAAAAALDMNECAA|1901-10-17|21|94|8|1901|4|10|17|4|1901|8|94|Thursday|1901Q4|N|N|N|2415659|2415931|2415310|2415583|N|N|N|N|N| +2415676|AAAAAAAAMDMNECAA|1901-10-18|21|94|8|1901|5|10|18|4|1901|8|94|Friday|1901Q4|N|Y|N|2415659|2415931|2415311|2415584|N|N|N|N|N| +2415677|AAAAAAAANDMNECAA|1901-10-19|21|94|8|1901|6|10|19|4|1901|8|94|Saturday|1901Q4|N|Y|N|2415659|2415931|2415312|2415585|N|N|N|N|N| +2415678|AAAAAAAAODMNECAA|1901-10-20|21|94|8|1901|0|10|20|4|1901|8|94|Sunday|1901Q4|N|N|N|2415659|2415931|2415313|2415586|N|N|N|N|N| +2415679|AAAAAAAAPDMNECAA|1901-10-21|21|94|8|1901|1|10|21|4|1901|8|94|Monday|1901Q4|N|N|N|2415659|2415931|2415314|2415587|N|N|N|N|N| +2415680|AAAAAAAAAEMNECAA|1901-10-22|21|95|8|1901|2|10|22|4|1901|8|95|Tuesday|1901Q4|N|N|N|2415659|2415931|2415315|2415588|N|N|N|N|N| +2415681|AAAAAAAABEMNECAA|1901-10-23|21|95|8|1901|3|10|23|4|1901|8|95|Wednesday|1901Q4|N|N|N|2415659|2415931|2415316|2415589|N|N|N|N|N| +2415682|AAAAAAAACEMNECAA|1901-10-24|21|95|8|1901|4|10|24|4|1901|8|95|Thursday|1901Q4|N|N|N|2415659|2415931|2415317|2415590|N|N|N|N|N| +2415683|AAAAAAAADEMNECAA|1901-10-25|21|95|8|1901|5|10|25|4|1901|8|95|Friday|1901Q4|N|Y|N|2415659|2415931|2415318|2415591|N|N|N|N|N| +2415684|AAAAAAAAEEMNECAA|1901-10-26|21|95|8|1901|6|10|26|4|1901|8|95|Saturday|1901Q4|N|Y|N|2415659|2415931|2415319|2415592|N|N|N|N|N| +2415685|AAAAAAAAFEMNECAA|1901-10-27|21|95|8|1901|0|10|27|4|1901|8|95|Sunday|1901Q4|N|N|N|2415659|2415931|2415320|2415593|N|N|N|N|N| +2415686|AAAAAAAAGEMNECAA|1901-10-28|21|95|8|1901|1|10|28|4|1901|8|95|Monday|1901Q4|N|N|N|2415659|2415931|2415321|2415594|N|N|N|N|N| +2415687|AAAAAAAAHEMNECAA|1901-10-29|21|96|8|1901|2|10|29|4|1901|8|96|Tuesday|1901Q4|N|N|N|2415659|2415931|2415322|2415595|N|N|N|N|N| +2415688|AAAAAAAAIEMNECAA|1901-10-30|21|96|8|1901|3|10|30|4|1901|8|96|Wednesday|1901Q4|N|N|N|2415659|2415931|2415323|2415596|N|N|N|N|N| +2415689|AAAAAAAAJEMNECAA|1901-10-31|21|96|8|1901|4|10|31|4|1901|8|96|Thursday|1901Q4|N|N|N|2415659|2415931|2415324|2415597|N|N|N|N|N| +2415690|AAAAAAAAKEMNECAA|1901-11-01|22|96|8|1901|5|11|1|4|1901|8|96|Friday|1901Q4|N|Y|N|2415690|2415993|2415325|2415598|N|N|N|N|N| +2415691|AAAAAAAALEMNECAA|1901-11-02|22|96|8|1901|6|11|2|4|1901|8|96|Saturday|1901Q4|N|Y|N|2415690|2415993|2415326|2415599|N|N|N|N|N| +2415692|AAAAAAAAMEMNECAA|1901-11-03|22|96|8|1901|0|11|3|4|1901|8|96|Sunday|1901Q4|N|N|N|2415690|2415993|2415327|2415600|N|N|N|N|N| +2415693|AAAAAAAANEMNECAA|1901-11-04|22|96|8|1901|1|11|4|4|1901|8|96|Monday|1901Q4|N|N|N|2415690|2415993|2415328|2415601|N|N|N|N|N| +2415694|AAAAAAAAOEMNECAA|1901-11-05|22|97|8|1901|2|11|5|4|1901|8|97|Tuesday|1901Q4|N|N|N|2415690|2415993|2415329|2415602|N|N|N|N|N| +2415695|AAAAAAAAPEMNECAA|1901-11-06|22|97|8|1901|3|11|6|4|1901|8|97|Wednesday|1901Q4|N|N|N|2415690|2415993|2415330|2415603|N|N|N|N|N| +2415696|AAAAAAAAAFMNECAA|1901-11-07|22|97|8|1901|4|11|7|4|1901|8|97|Thursday|1901Q4|N|N|N|2415690|2415993|2415331|2415604|N|N|N|N|N| +2415697|AAAAAAAABFMNECAA|1901-11-08|22|97|8|1901|5|11|8|4|1901|8|97|Friday|1901Q4|N|Y|N|2415690|2415993|2415332|2415605|N|N|N|N|N| +2415698|AAAAAAAACFMNECAA|1901-11-09|22|97|8|1901|6|11|9|4|1901|8|97|Saturday|1901Q4|N|Y|N|2415690|2415993|2415333|2415606|N|N|N|N|N| +2415699|AAAAAAAADFMNECAA|1901-11-10|22|97|8|1901|0|11|10|4|1901|8|97|Sunday|1901Q4|N|N|N|2415690|2415993|2415334|2415607|N|N|N|N|N| +2415700|AAAAAAAAEFMNECAA|1901-11-11|22|97|8|1901|1|11|11|4|1901|8|97|Monday|1901Q4|N|N|N|2415690|2415993|2415335|2415608|N|N|N|N|N| +2415701|AAAAAAAAFFMNECAA|1901-11-12|22|98|8|1901|2|11|12|4|1901|8|98|Tuesday|1901Q4|N|N|N|2415690|2415993|2415336|2415609|N|N|N|N|N| +2415702|AAAAAAAAGFMNECAA|1901-11-13|22|98|8|1901|3|11|13|4|1901|8|98|Wednesday|1901Q4|N|N|N|2415690|2415993|2415337|2415610|N|N|N|N|N| +2415703|AAAAAAAAHFMNECAA|1901-11-14|22|98|8|1901|4|11|14|4|1901|8|98|Thursday|1901Q4|N|N|N|2415690|2415993|2415338|2415611|N|N|N|N|N| +2415704|AAAAAAAAIFMNECAA|1901-11-15|22|98|8|1901|5|11|15|4|1901|8|98|Friday|1901Q4|N|Y|N|2415690|2415993|2415339|2415612|N|N|N|N|N| +2415705|AAAAAAAAJFMNECAA|1901-11-16|22|98|8|1901|6|11|16|4|1901|8|98|Saturday|1901Q4|N|Y|N|2415690|2415993|2415340|2415613|N|N|N|N|N| +2415706|AAAAAAAAKFMNECAA|1901-11-17|22|98|8|1901|0|11|17|4|1901|8|98|Sunday|1901Q4|N|N|N|2415690|2415993|2415341|2415614|N|N|N|N|N| +2415707|AAAAAAAALFMNECAA|1901-11-18|22|98|8|1901|1|11|18|4|1901|8|98|Monday|1901Q4|N|N|N|2415690|2415993|2415342|2415615|N|N|N|N|N| +2415708|AAAAAAAAMFMNECAA|1901-11-19|22|99|8|1901|2|11|19|4|1901|8|99|Tuesday|1901Q4|N|N|N|2415690|2415993|2415343|2415616|N|N|N|N|N| +2415709|AAAAAAAANFMNECAA|1901-11-20|22|99|8|1901|3|11|20|4|1901|8|99|Wednesday|1901Q4|N|N|N|2415690|2415993|2415344|2415617|N|N|N|N|N| +2415710|AAAAAAAAOFMNECAA|1901-11-21|22|99|8|1901|4|11|21|4|1901|8|99|Thursday|1901Q4|N|N|N|2415690|2415993|2415345|2415618|N|N|N|N|N| +2415711|AAAAAAAAPFMNECAA|1901-11-22|22|99|8|1901|5|11|22|4|1901|8|99|Friday|1901Q4|N|Y|N|2415690|2415993|2415346|2415619|N|N|N|N|N| +2415712|AAAAAAAAAGMNECAA|1901-11-23|22|99|8|1901|6|11|23|4|1901|8|99|Saturday|1901Q4|N|Y|N|2415690|2415993|2415347|2415620|N|N|N|N|N| +2415713|AAAAAAAABGMNECAA|1901-11-24|22|99|8|1901|0|11|24|4|1901|8|99|Sunday|1901Q4|N|N|N|2415690|2415993|2415348|2415621|N|N|N|N|N| +2415714|AAAAAAAACGMNECAA|1901-11-25|22|99|8|1901|1|11|25|4|1901|8|99|Monday|1901Q4|N|N|N|2415690|2415993|2415349|2415622|N|N|N|N|N| +2415715|AAAAAAAADGMNECAA|1901-11-26|22|100|8|1901|2|11|26|4|1901|8|100|Tuesday|1901Q4|N|N|N|2415690|2415993|2415350|2415623|N|N|N|N|N| +2415716|AAAAAAAAEGMNECAA|1901-11-27|22|100|8|1901|3|11|27|4|1901|8|100|Wednesday|1901Q4|N|N|N|2415690|2415993|2415351|2415624|N|N|N|N|N| +2415717|AAAAAAAAFGMNECAA|1901-11-28|22|100|8|1901|4|11|28|4|1901|8|100|Thursday|1901Q4|N|N|N|2415690|2415993|2415352|2415625|N|N|N|N|N| +2415718|AAAAAAAAGGMNECAA|1901-11-29|22|100|8|1901|5|11|29|4|1901|8|100|Friday|1901Q4|N|Y|N|2415690|2415993|2415353|2415626|N|N|N|N|N| +2415719|AAAAAAAAHGMNECAA|1901-11-30|22|100|8|1901|6|11|30|4|1901|8|100|Saturday|1901Q4|N|Y|N|2415690|2415993|2415354|2415627|N|N|N|N|N| +2415720|AAAAAAAAIGMNECAA|1901-12-01|23|100|9|1901|0|12|1|4|1901|9|100|Sunday|1901Q4|N|N|N|2415720|2416053|2415355|2415628|N|N|N|N|N| +2415721|AAAAAAAAJGMNECAA|1901-12-02|23|100|9|1901|1|12|2|4|1901|9|100|Monday|1901Q4|N|N|N|2415720|2416053|2415356|2415629|N|N|N|N|N| +2415722|AAAAAAAAKGMNECAA|1901-12-03|23|101|9|1901|2|12|3|4|1901|9|101|Tuesday|1901Q4|N|N|N|2415720|2416053|2415357|2415630|N|N|N|N|N| +2415723|AAAAAAAALGMNECAA|1901-12-04|23|101|9|1901|3|12|4|4|1901|9|101|Wednesday|1901Q4|N|N|N|2415720|2416053|2415358|2415631|N|N|N|N|N| +2415724|AAAAAAAAMGMNECAA|1901-12-05|23|101|9|1901|4|12|5|4|1901|9|101|Thursday|1901Q4|N|N|N|2415720|2416053|2415359|2415632|N|N|N|N|N| +2415725|AAAAAAAANGMNECAA|1901-12-06|23|101|9|1901|5|12|6|4|1901|9|101|Friday|1901Q4|N|Y|N|2415720|2416053|2415360|2415633|N|N|N|N|N| +2415726|AAAAAAAAOGMNECAA|1901-12-07|23|101|9|1901|6|12|7|4|1901|9|101|Saturday|1901Q4|N|Y|N|2415720|2416053|2415361|2415634|N|N|N|N|N| +2415727|AAAAAAAAPGMNECAA|1901-12-08|23|101|9|1901|0|12|8|4|1901|9|101|Sunday|1901Q4|N|N|N|2415720|2416053|2415362|2415635|N|N|N|N|N| +2415728|AAAAAAAAAHMNECAA|1901-12-09|23|101|9|1901|1|12|9|4|1901|9|101|Monday|1901Q4|N|N|N|2415720|2416053|2415363|2415636|N|N|N|N|N| +2415729|AAAAAAAABHMNECAA|1901-12-10|23|102|9|1901|2|12|10|4|1901|9|102|Tuesday|1901Q4|N|N|N|2415720|2416053|2415364|2415637|N|N|N|N|N| +2415730|AAAAAAAACHMNECAA|1901-12-11|23|102|9|1901|3|12|11|4|1901|9|102|Wednesday|1901Q4|N|N|N|2415720|2416053|2415365|2415638|N|N|N|N|N| +2415731|AAAAAAAADHMNECAA|1901-12-12|23|102|9|1901|4|12|12|4|1901|9|102|Thursday|1901Q4|N|N|N|2415720|2416053|2415366|2415639|N|N|N|N|N| +2415732|AAAAAAAAEHMNECAA|1901-12-13|23|102|9|1901|5|12|13|4|1901|9|102|Friday|1901Q4|N|Y|N|2415720|2416053|2415367|2415640|N|N|N|N|N| +2415733|AAAAAAAAFHMNECAA|1901-12-14|23|102|9|1901|6|12|14|4|1901|9|102|Saturday|1901Q4|N|Y|N|2415720|2416053|2415368|2415641|N|N|N|N|N| +2415734|AAAAAAAAGHMNECAA|1901-12-15|23|102|9|1901|0|12|15|4|1901|9|102|Sunday|1901Q4|N|N|N|2415720|2416053|2415369|2415642|N|N|N|N|N| +2415735|AAAAAAAAHHMNECAA|1901-12-16|23|102|9|1901|1|12|16|4|1901|9|102|Monday|1901Q4|N|N|N|2415720|2416053|2415370|2415643|N|N|N|N|N| +2415736|AAAAAAAAIHMNECAA|1901-12-17|23|103|9|1901|2|12|17|4|1901|9|103|Tuesday|1901Q4|N|N|N|2415720|2416053|2415371|2415644|N|N|N|N|N| +2415737|AAAAAAAAJHMNECAA|1901-12-18|23|103|9|1901|3|12|18|4|1901|9|103|Wednesday|1901Q4|N|N|N|2415720|2416053|2415372|2415645|N|N|N|N|N| +2415738|AAAAAAAAKHMNECAA|1901-12-19|23|103|9|1901|4|12|19|4|1901|9|103|Thursday|1901Q4|N|N|N|2415720|2416053|2415373|2415646|N|N|N|N|N| +2415739|AAAAAAAALHMNECAA|1901-12-20|23|103|9|1901|5|12|20|4|1901|9|103|Friday|1901Q4|N|Y|N|2415720|2416053|2415374|2415647|N|N|N|N|N| +2415740|AAAAAAAAMHMNECAA|1901-12-21|23|103|9|1901|6|12|21|4|1901|9|103|Saturday|1901Q4|N|Y|N|2415720|2416053|2415375|2415648|N|N|N|N|N| +2415741|AAAAAAAANHMNECAA|1901-12-22|23|103|9|1901|0|12|22|4|1901|9|103|Sunday|1901Q4|N|N|N|2415720|2416053|2415376|2415649|N|N|N|N|N| +2415742|AAAAAAAAOHMNECAA|1901-12-23|23|103|9|1901|1|12|23|4|1901|9|103|Monday|1901Q4|N|N|N|2415720|2416053|2415377|2415650|N|N|N|N|N| +2415743|AAAAAAAAPHMNECAA|1901-12-24|23|104|9|1901|2|12|24|4|1901|9|104|Tuesday|1901Q4|N|N|N|2415720|2416053|2415378|2415651|N|N|N|N|N| +2415744|AAAAAAAAAIMNECAA|1901-12-25|23|104|9|1901|3|12|25|4|1901|9|104|Wednesday|1901Q4|N|N|N|2415720|2416053|2415379|2415652|N|N|N|N|N| +2415745|AAAAAAAABIMNECAA|1901-12-26|23|104|9|1901|4|12|26|4|1901|9|104|Thursday|1901Q4|Y|N|N|2415720|2416053|2415380|2415653|N|N|N|N|N| +2415746|AAAAAAAACIMNECAA|1901-12-27|23|104|9|1901|5|12|27|4|1901|9|104|Friday|1901Q4|N|Y|Y|2415720|2416053|2415381|2415654|N|N|N|N|N| +2415747|AAAAAAAADIMNECAA|1901-12-28|23|104|9|1901|6|12|28|4|1901|9|104|Saturday|1901Q4|N|Y|N|2415720|2416053|2415382|2415655|N|N|N|N|N| +2415748|AAAAAAAAEIMNECAA|1901-12-29|23|104|9|1901|0|12|29|4|1901|9|104|Sunday|1901Q4|N|N|N|2415720|2416053|2415383|2415656|N|N|N|N|N| +2415749|AAAAAAAAFIMNECAA|1901-12-30|23|104|9|1901|1|12|30|4|1901|9|104|Monday|1901Q4|N|N|N|2415720|2416053|2415384|2415657|N|N|N|N|N| +2415750|AAAAAAAAGIMNECAA|1901-12-31|23|105|9|1901|2|12|31|4|1901|9|105|Tuesday|1901Q4|N|N|N|2415720|2416053|2415385|2415658|N|N|N|N|N| +2415751|AAAAAAAAHIMNECAA|1902-01-01|24|105|9|1902|3|1|1|1|1902|9|105|Wednesday|1902Q1|Y|N|N|2415751|2415750|2415386|2415659|N|N|N|N|N| +2415752|AAAAAAAAIIMNECAA|1902-01-02|24|105|9|1902|4|1|2|1|1902|9|105|Thursday|1902Q1|N|N|Y|2415751|2415750|2415387|2415660|N|N|N|N|N| +2415753|AAAAAAAAJIMNECAA|1902-01-03|24|105|9|1902|5|1|3|1|1902|9|105|Friday|1902Q1|N|Y|N|2415751|2415750|2415388|2415661|N|N|N|N|N| +2415754|AAAAAAAAKIMNECAA|1902-01-04|24|105|9|1902|6|1|4|1|1902|9|105|Saturday|1902Q1|N|Y|N|2415751|2415750|2415389|2415662|N|N|N|N|N| +2415755|AAAAAAAALIMNECAA|1902-01-05|24|105|9|1902|0|1|5|1|1902|9|105|Sunday|1902Q1|N|N|N|2415751|2415750|2415390|2415663|N|N|N|N|N| +2415756|AAAAAAAAMIMNECAA|1902-01-06|24|105|9|1902|1|1|6|1|1902|9|105|Monday|1902Q1|N|N|N|2415751|2415750|2415391|2415664|N|N|N|N|N| +2415757|AAAAAAAANIMNECAA|1902-01-07|24|106|9|1902|2|1|7|1|1902|9|106|Tuesday|1902Q1|N|N|N|2415751|2415750|2415392|2415665|N|N|N|N|N| +2415758|AAAAAAAAOIMNECAA|1902-01-08|24|106|9|1902|3|1|8|1|1902|9|106|Wednesday|1902Q1|N|N|N|2415751|2415750|2415393|2415666|N|N|N|N|N| +2415759|AAAAAAAAPIMNECAA|1902-01-09|24|106|9|1902|4|1|9|1|1902|9|106|Thursday|1902Q1|N|N|N|2415751|2415750|2415394|2415667|N|N|N|N|N| +2415760|AAAAAAAAAJMNECAA|1902-01-10|24|106|9|1902|5|1|10|1|1902|9|106|Friday|1902Q1|N|Y|N|2415751|2415750|2415395|2415668|N|N|N|N|N| +2415761|AAAAAAAABJMNECAA|1902-01-11|24|106|9|1902|6|1|11|1|1902|9|106|Saturday|1902Q1|N|Y|N|2415751|2415750|2415396|2415669|N|N|N|N|N| +2415762|AAAAAAAACJMNECAA|1902-01-12|24|106|9|1902|0|1|12|1|1902|9|106|Sunday|1902Q1|N|N|N|2415751|2415750|2415397|2415670|N|N|N|N|N| +2415763|AAAAAAAADJMNECAA|1902-01-13|24|106|9|1902|1|1|13|1|1902|9|106|Monday|1902Q1|N|N|N|2415751|2415750|2415398|2415671|N|N|N|N|N| +2415764|AAAAAAAAEJMNECAA|1902-01-14|24|107|9|1902|2|1|14|1|1902|9|107|Tuesday|1902Q1|N|N|N|2415751|2415750|2415399|2415672|N|N|N|N|N| +2415765|AAAAAAAAFJMNECAA|1902-01-15|24|107|9|1902|3|1|15|1|1902|9|107|Wednesday|1902Q1|N|N|N|2415751|2415750|2415400|2415673|N|N|N|N|N| +2415766|AAAAAAAAGJMNECAA|1902-01-16|24|107|9|1902|4|1|16|1|1902|9|107|Thursday|1902Q1|N|N|N|2415751|2415750|2415401|2415674|N|N|N|N|N| +2415767|AAAAAAAAHJMNECAA|1902-01-17|24|107|9|1902|5|1|17|1|1902|9|107|Friday|1902Q1|N|Y|N|2415751|2415750|2415402|2415675|N|N|N|N|N| +2415768|AAAAAAAAIJMNECAA|1902-01-18|24|107|9|1902|6|1|18|1|1902|9|107|Saturday|1902Q1|N|Y|N|2415751|2415750|2415403|2415676|N|N|N|N|N| +2415769|AAAAAAAAJJMNECAA|1902-01-19|24|107|9|1902|0|1|19|1|1902|9|107|Sunday|1902Q1|N|N|N|2415751|2415750|2415404|2415677|N|N|N|N|N| +2415770|AAAAAAAAKJMNECAA|1902-01-20|24|107|9|1902|1|1|20|1|1902|9|107|Monday|1902Q1|N|N|N|2415751|2415750|2415405|2415678|N|N|N|N|N| +2415771|AAAAAAAALJMNECAA|1902-01-21|24|108|9|1902|2|1|21|1|1902|9|108|Tuesday|1902Q1|N|N|N|2415751|2415750|2415406|2415679|N|N|N|N|N| +2415772|AAAAAAAAMJMNECAA|1902-01-22|24|108|9|1902|3|1|22|1|1902|9|108|Wednesday|1902Q1|N|N|N|2415751|2415750|2415407|2415680|N|N|N|N|N| +2415773|AAAAAAAANJMNECAA|1902-01-23|24|108|9|1902|4|1|23|1|1902|9|108|Thursday|1902Q1|N|N|N|2415751|2415750|2415408|2415681|N|N|N|N|N| +2415774|AAAAAAAAOJMNECAA|1902-01-24|24|108|9|1902|5|1|24|1|1902|9|108|Friday|1902Q1|N|Y|N|2415751|2415750|2415409|2415682|N|N|N|N|N| +2415775|AAAAAAAAPJMNECAA|1902-01-25|24|108|9|1902|6|1|25|1|1902|9|108|Saturday|1902Q1|N|Y|N|2415751|2415750|2415410|2415683|N|N|N|N|N| +2415776|AAAAAAAAAKMNECAA|1902-01-26|24|108|9|1902|0|1|26|1|1902|9|108|Sunday|1902Q1|N|N|N|2415751|2415750|2415411|2415684|N|N|N|N|N| +2415777|AAAAAAAABKMNECAA|1902-01-27|24|108|9|1902|1|1|27|1|1902|9|108|Monday|1902Q1|N|N|N|2415751|2415750|2415412|2415685|N|N|N|N|N| +2415778|AAAAAAAACKMNECAA|1902-01-28|24|109|9|1902|2|1|28|1|1902|9|109|Tuesday|1902Q1|N|N|N|2415751|2415750|2415413|2415686|N|N|N|N|N| +2415779|AAAAAAAADKMNECAA|1902-01-29|24|109|9|1902|3|1|29|1|1902|9|109|Wednesday|1902Q1|N|N|N|2415751|2415750|2415414|2415687|N|N|N|N|N| +2415780|AAAAAAAAEKMNECAA|1902-01-30|24|109|9|1902|4|1|30|1|1902|9|109|Thursday|1902Q1|N|N|N|2415751|2415750|2415415|2415688|N|N|N|N|N| +2415781|AAAAAAAAFKMNECAA|1902-01-31|24|109|9|1902|5|1|31|1|1902|9|109|Friday|1902Q1|N|Y|N|2415751|2415750|2415416|2415689|N|N|N|N|N| +2415782|AAAAAAAAGKMNECAA|1902-02-01|25|109|9|1902|6|2|1|1|1902|9|109|Saturday|1902Q1|N|Y|N|2415782|2415812|2415417|2415690|N|N|N|N|N| +2415783|AAAAAAAAHKMNECAA|1902-02-02|25|109|9|1902|0|2|2|1|1902|9|109|Sunday|1902Q1|N|N|N|2415782|2415812|2415418|2415691|N|N|N|N|N| +2415784|AAAAAAAAIKMNECAA|1902-02-03|25|109|9|1902|1|2|3|1|1902|9|109|Monday|1902Q1|N|N|N|2415782|2415812|2415419|2415692|N|N|N|N|N| +2415785|AAAAAAAAJKMNECAA|1902-02-04|25|110|9|1902|2|2|4|1|1902|9|110|Tuesday|1902Q1|N|N|N|2415782|2415812|2415420|2415693|N|N|N|N|N| +2415786|AAAAAAAAKKMNECAA|1902-02-05|25|110|9|1902|3|2|5|1|1902|9|110|Wednesday|1902Q1|N|N|N|2415782|2415812|2415421|2415694|N|N|N|N|N| +2415787|AAAAAAAALKMNECAA|1902-02-06|25|110|9|1902|4|2|6|1|1902|9|110|Thursday|1902Q1|N|N|N|2415782|2415812|2415422|2415695|N|N|N|N|N| +2415788|AAAAAAAAMKMNECAA|1902-02-07|25|110|9|1902|5|2|7|1|1902|9|110|Friday|1902Q1|N|Y|N|2415782|2415812|2415423|2415696|N|N|N|N|N| +2415789|AAAAAAAANKMNECAA|1902-02-08|25|110|9|1902|6|2|8|1|1902|9|110|Saturday|1902Q1|N|Y|N|2415782|2415812|2415424|2415697|N|N|N|N|N| +2415790|AAAAAAAAOKMNECAA|1902-02-09|25|110|9|1902|0|2|9|1|1902|9|110|Sunday|1902Q1|N|N|N|2415782|2415812|2415425|2415698|N|N|N|N|N| +2415791|AAAAAAAAPKMNECAA|1902-02-10|25|110|9|1902|1|2|10|1|1902|9|110|Monday|1902Q1|N|N|N|2415782|2415812|2415426|2415699|N|N|N|N|N| +2415792|AAAAAAAAALMNECAA|1902-02-11|25|111|9|1902|2|2|11|1|1902|9|111|Tuesday|1902Q1|N|N|N|2415782|2415812|2415427|2415700|N|N|N|N|N| +2415793|AAAAAAAABLMNECAA|1902-02-12|25|111|9|1902|3|2|12|1|1902|9|111|Wednesday|1902Q1|N|N|N|2415782|2415812|2415428|2415701|N|N|N|N|N| +2415794|AAAAAAAACLMNECAA|1902-02-13|25|111|9|1902|4|2|13|1|1902|9|111|Thursday|1902Q1|N|N|N|2415782|2415812|2415429|2415702|N|N|N|N|N| +2415795|AAAAAAAADLMNECAA|1902-02-14|25|111|9|1902|5|2|14|1|1902|9|111|Friday|1902Q1|N|Y|N|2415782|2415812|2415430|2415703|N|N|N|N|N| +2415796|AAAAAAAAELMNECAA|1902-02-15|25|111|9|1902|6|2|15|1|1902|9|111|Saturday|1902Q1|N|Y|N|2415782|2415812|2415431|2415704|N|N|N|N|N| +2415797|AAAAAAAAFLMNECAA|1902-02-16|25|111|9|1902|0|2|16|1|1902|9|111|Sunday|1902Q1|N|N|N|2415782|2415812|2415432|2415705|N|N|N|N|N| +2415798|AAAAAAAAGLMNECAA|1902-02-17|25|111|9|1902|1|2|17|1|1902|9|111|Monday|1902Q1|N|N|N|2415782|2415812|2415433|2415706|N|N|N|N|N| +2415799|AAAAAAAAHLMNECAA|1902-02-18|25|112|9|1902|2|2|18|1|1902|9|112|Tuesday|1902Q1|N|N|N|2415782|2415812|2415434|2415707|N|N|N|N|N| +2415800|AAAAAAAAILMNECAA|1902-02-19|25|112|9|1902|3|2|19|1|1902|9|112|Wednesday|1902Q1|N|N|N|2415782|2415812|2415435|2415708|N|N|N|N|N| +2415801|AAAAAAAAJLMNECAA|1902-02-20|25|112|9|1902|4|2|20|1|1902|9|112|Thursday|1902Q1|N|N|N|2415782|2415812|2415436|2415709|N|N|N|N|N| +2415802|AAAAAAAAKLMNECAA|1902-02-21|25|112|9|1902|5|2|21|1|1902|9|112|Friday|1902Q1|N|Y|N|2415782|2415812|2415437|2415710|N|N|N|N|N| +2415803|AAAAAAAALLMNECAA|1902-02-22|25|112|9|1902|6|2|22|1|1902|9|112|Saturday|1902Q1|N|Y|N|2415782|2415812|2415438|2415711|N|N|N|N|N| +2415804|AAAAAAAAMLMNECAA|1902-02-23|25|112|9|1902|0|2|23|1|1902|9|112|Sunday|1902Q1|N|N|N|2415782|2415812|2415439|2415712|N|N|N|N|N| +2415805|AAAAAAAANLMNECAA|1902-02-24|25|112|9|1902|1|2|24|1|1902|9|112|Monday|1902Q1|N|N|N|2415782|2415812|2415440|2415713|N|N|N|N|N| +2415806|AAAAAAAAOLMNECAA|1902-02-25|25|113|9|1902|2|2|25|1|1902|9|113|Tuesday|1902Q1|N|N|N|2415782|2415812|2415441|2415714|N|N|N|N|N| +2415807|AAAAAAAAPLMNECAA|1902-02-26|25|113|9|1902|3|2|26|1|1902|9|113|Wednesday|1902Q1|N|N|N|2415782|2415812|2415442|2415715|N|N|N|N|N| +2415808|AAAAAAAAAMMNECAA|1902-02-27|25|113|9|1902|4|2|27|1|1902|9|113|Thursday|1902Q1|N|N|N|2415782|2415812|2415443|2415716|N|N|N|N|N| +2415809|AAAAAAAABMMNECAA|1902-02-28|25|113|9|1902|5|2|28|1|1902|9|113|Friday|1902Q1|N|Y|N|2415782|2415812|2415444|2415717|N|N|N|N|N| +2415810|AAAAAAAACMMNECAA|1902-03-01|26|113|10|1902|6|3|1|1|1902|10|113|Saturday|1902Q1|N|Y|N|2415810|2415868|2415445|2415718|N|N|N|N|N| +2415811|AAAAAAAADMMNECAA|1902-03-02|26|113|10|1902|0|3|2|1|1902|10|113|Sunday|1902Q1|N|N|N|2415810|2415868|2415446|2415719|N|N|N|N|N| +2415812|AAAAAAAAEMMNECAA|1902-03-03|26|113|10|1902|1|3|3|1|1902|10|113|Monday|1902Q1|N|N|N|2415810|2415868|2415447|2415720|N|N|N|N|N| +2415813|AAAAAAAAFMMNECAA|1902-03-04|26|114|10|1902|2|3|4|1|1902|10|114|Tuesday|1902Q1|N|N|N|2415810|2415868|2415448|2415721|N|N|N|N|N| +2415814|AAAAAAAAGMMNECAA|1902-03-05|26|114|10|1902|3|3|5|1|1902|10|114|Wednesday|1902Q1|N|N|N|2415810|2415868|2415449|2415722|N|N|N|N|N| +2415815|AAAAAAAAHMMNECAA|1902-03-06|26|114|10|1902|4|3|6|1|1902|10|114|Thursday|1902Q1|N|N|N|2415810|2415868|2415450|2415723|N|N|N|N|N| +2415816|AAAAAAAAIMMNECAA|1902-03-07|26|114|10|1902|5|3|7|1|1902|10|114|Friday|1902Q1|N|Y|N|2415810|2415868|2415451|2415724|N|N|N|N|N| +2415817|AAAAAAAAJMMNECAA|1902-03-08|26|114|10|1902|6|3|8|1|1902|10|114|Saturday|1902Q1|N|Y|N|2415810|2415868|2415452|2415725|N|N|N|N|N| +2415818|AAAAAAAAKMMNECAA|1902-03-09|26|114|10|1902|0|3|9|1|1902|10|114|Sunday|1902Q1|N|N|N|2415810|2415868|2415453|2415726|N|N|N|N|N| +2415819|AAAAAAAALMMNECAA|1902-03-10|26|114|10|1902|1|3|10|1|1902|10|114|Monday|1902Q1|N|N|N|2415810|2415868|2415454|2415727|N|N|N|N|N| +2415820|AAAAAAAAMMMNECAA|1902-03-11|26|115|10|1902|2|3|11|1|1902|10|115|Tuesday|1902Q1|N|N|N|2415810|2415868|2415455|2415728|N|N|N|N|N| +2415821|AAAAAAAANMMNECAA|1902-03-12|26|115|10|1902|3|3|12|1|1902|10|115|Wednesday|1902Q1|N|N|N|2415810|2415868|2415456|2415729|N|N|N|N|N| +2415822|AAAAAAAAOMMNECAA|1902-03-13|26|115|10|1902|4|3|13|1|1902|10|115|Thursday|1902Q1|N|N|N|2415810|2415868|2415457|2415730|N|N|N|N|N| +2415823|AAAAAAAAPMMNECAA|1902-03-14|26|115|10|1902|5|3|14|1|1902|10|115|Friday|1902Q1|N|Y|N|2415810|2415868|2415458|2415731|N|N|N|N|N| +2415824|AAAAAAAAANMNECAA|1902-03-15|26|115|10|1902|6|3|15|1|1902|10|115|Saturday|1902Q1|N|Y|N|2415810|2415868|2415459|2415732|N|N|N|N|N| +2415825|AAAAAAAABNMNECAA|1902-03-16|26|115|10|1902|0|3|16|1|1902|10|115|Sunday|1902Q1|N|N|N|2415810|2415868|2415460|2415733|N|N|N|N|N| +2415826|AAAAAAAACNMNECAA|1902-03-17|26|115|10|1902|1|3|17|1|1902|10|115|Monday|1902Q1|N|N|N|2415810|2415868|2415461|2415734|N|N|N|N|N| +2415827|AAAAAAAADNMNECAA|1902-03-18|26|116|10|1902|2|3|18|1|1902|10|116|Tuesday|1902Q1|N|N|N|2415810|2415868|2415462|2415735|N|N|N|N|N| +2415828|AAAAAAAAENMNECAA|1902-03-19|26|116|10|1902|3|3|19|1|1902|10|116|Wednesday|1902Q1|N|N|N|2415810|2415868|2415463|2415736|N|N|N|N|N| +2415829|AAAAAAAAFNMNECAA|1902-03-20|26|116|10|1902|4|3|20|1|1902|10|116|Thursday|1902Q1|N|N|N|2415810|2415868|2415464|2415737|N|N|N|N|N| +2415830|AAAAAAAAGNMNECAA|1902-03-21|26|116|10|1902|5|3|21|1|1902|10|116|Friday|1902Q1|N|Y|N|2415810|2415868|2415465|2415738|N|N|N|N|N| +2415831|AAAAAAAAHNMNECAA|1902-03-22|26|116|10|1902|6|3|22|1|1902|10|116|Saturday|1902Q1|N|Y|N|2415810|2415868|2415466|2415739|N|N|N|N|N| +2415832|AAAAAAAAINMNECAA|1902-03-23|26|116|10|1902|0|3|23|1|1902|10|116|Sunday|1902Q1|N|N|N|2415810|2415868|2415467|2415740|N|N|N|N|N| +2415833|AAAAAAAAJNMNECAA|1902-03-24|26|116|10|1902|1|3|24|1|1902|10|116|Monday|1902Q1|N|N|N|2415810|2415868|2415468|2415741|N|N|N|N|N| +2415834|AAAAAAAAKNMNECAA|1902-03-25|26|117|10|1902|2|3|25|1|1902|10|117|Tuesday|1902Q1|N|N|N|2415810|2415868|2415469|2415742|N|N|N|N|N| +2415835|AAAAAAAALNMNECAA|1902-03-26|26|117|10|1902|3|3|26|1|1902|10|117|Wednesday|1902Q1|N|N|N|2415810|2415868|2415470|2415743|N|N|N|N|N| +2415836|AAAAAAAAMNMNECAA|1902-03-27|26|117|10|1902|4|3|27|1|1902|10|117|Thursday|1902Q1|N|N|N|2415810|2415868|2415471|2415744|N|N|N|N|N| +2415837|AAAAAAAANNMNECAA|1902-03-28|26|117|10|1902|5|3|28|1|1902|10|117|Friday|1902Q1|N|Y|N|2415810|2415868|2415472|2415745|N|N|N|N|N| +2415838|AAAAAAAAONMNECAA|1902-03-29|26|117|10|1902|6|3|29|1|1902|10|117|Saturday|1902Q1|N|Y|N|2415810|2415868|2415473|2415746|N|N|N|N|N| +2415839|AAAAAAAAPNMNECAA|1902-03-30|26|117|10|1902|0|3|30|1|1902|10|117|Sunday|1902Q1|N|N|N|2415810|2415868|2415474|2415747|N|N|N|N|N| +2415840|AAAAAAAAAOMNECAA|1902-03-31|26|117|10|1902|1|3|31|1|1902|10|117|Monday|1902Q1|N|N|N|2415810|2415868|2415475|2415748|N|N|N|N|N| +2415841|AAAAAAAABOMNECAA|1902-04-01|27|118|10|1902|2|4|1|1|1902|10|118|Tuesday|1902Q1|N|N|N|2415841|2415930|2415476|2415751|N|N|N|N|N| +2415842|AAAAAAAACOMNECAA|1902-04-02|27|118|10|1902|3|4|2|2|1902|10|118|Wednesday|1902Q2|N|N|N|2415841|2415930|2415477|2415752|N|N|N|N|N| +2415843|AAAAAAAADOMNECAA|1902-04-03|27|118|10|1902|4|4|3|2|1902|10|118|Thursday|1902Q2|N|N|N|2415841|2415930|2415478|2415753|N|N|N|N|N| +2415844|AAAAAAAAEOMNECAA|1902-04-04|27|118|10|1902|5|4|4|2|1902|10|118|Friday|1902Q2|N|Y|N|2415841|2415930|2415479|2415754|N|N|N|N|N| +2415845|AAAAAAAAFOMNECAA|1902-04-05|27|118|10|1902|6|4|5|2|1902|10|118|Saturday|1902Q2|N|Y|N|2415841|2415930|2415480|2415755|N|N|N|N|N| +2415846|AAAAAAAAGOMNECAA|1902-04-06|27|118|10|1902|0|4|6|2|1902|10|118|Sunday|1902Q2|N|N|N|2415841|2415930|2415481|2415756|N|N|N|N|N| +2415847|AAAAAAAAHOMNECAA|1902-04-07|27|118|10|1902|1|4|7|2|1902|10|118|Monday|1902Q2|N|N|N|2415841|2415930|2415482|2415757|N|N|N|N|N| +2415848|AAAAAAAAIOMNECAA|1902-04-08|27|119|10|1902|2|4|8|2|1902|10|119|Tuesday|1902Q2|N|N|N|2415841|2415930|2415483|2415758|N|N|N|N|N| +2415849|AAAAAAAAJOMNECAA|1902-04-09|27|119|10|1902|3|4|9|2|1902|10|119|Wednesday|1902Q2|N|N|N|2415841|2415930|2415484|2415759|N|N|N|N|N| +2415850|AAAAAAAAKOMNECAA|1902-04-10|27|119|10|1902|4|4|10|2|1902|10|119|Thursday|1902Q2|N|N|N|2415841|2415930|2415485|2415760|N|N|N|N|N| +2415851|AAAAAAAALOMNECAA|1902-04-11|27|119|10|1902|5|4|11|2|1902|10|119|Friday|1902Q2|N|Y|N|2415841|2415930|2415486|2415761|N|N|N|N|N| +2415852|AAAAAAAAMOMNECAA|1902-04-12|27|119|10|1902|6|4|12|2|1902|10|119|Saturday|1902Q2|N|Y|N|2415841|2415930|2415487|2415762|N|N|N|N|N| +2415853|AAAAAAAANOMNECAA|1902-04-13|27|119|10|1902|0|4|13|2|1902|10|119|Sunday|1902Q2|N|N|N|2415841|2415930|2415488|2415763|N|N|N|N|N| +2415854|AAAAAAAAOOMNECAA|1902-04-14|27|119|10|1902|1|4|14|2|1902|10|119|Monday|1902Q2|N|N|N|2415841|2415930|2415489|2415764|N|N|N|N|N| +2415855|AAAAAAAAPOMNECAA|1902-04-15|27|120|10|1902|2|4|15|2|1902|10|120|Tuesday|1902Q2|N|N|N|2415841|2415930|2415490|2415765|N|N|N|N|N| +2415856|AAAAAAAAAPMNECAA|1902-04-16|27|120|10|1902|3|4|16|2|1902|10|120|Wednesday|1902Q2|N|N|N|2415841|2415930|2415491|2415766|N|N|N|N|N| +2415857|AAAAAAAABPMNECAA|1902-04-17|27|120|10|1902|4|4|17|2|1902|10|120|Thursday|1902Q2|N|N|N|2415841|2415930|2415492|2415767|N|N|N|N|N| +2415858|AAAAAAAACPMNECAA|1902-04-18|27|120|10|1902|5|4|18|2|1902|10|120|Friday|1902Q2|N|Y|N|2415841|2415930|2415493|2415768|N|N|N|N|N| +2415859|AAAAAAAADPMNECAA|1902-04-19|27|120|10|1902|6|4|19|2|1902|10|120|Saturday|1902Q2|N|Y|N|2415841|2415930|2415494|2415769|N|N|N|N|N| +2415860|AAAAAAAAEPMNECAA|1902-04-20|27|120|10|1902|0|4|20|2|1902|10|120|Sunday|1902Q2|N|N|N|2415841|2415930|2415495|2415770|N|N|N|N|N| +2415861|AAAAAAAAFPMNECAA|1902-04-21|27|120|10|1902|1|4|21|2|1902|10|120|Monday|1902Q2|N|N|N|2415841|2415930|2415496|2415771|N|N|N|N|N| +2415862|AAAAAAAAGPMNECAA|1902-04-22|27|121|10|1902|2|4|22|2|1902|10|121|Tuesday|1902Q2|N|N|N|2415841|2415930|2415497|2415772|N|N|N|N|N| +2415863|AAAAAAAAHPMNECAA|1902-04-23|27|121|10|1902|3|4|23|2|1902|10|121|Wednesday|1902Q2|N|N|N|2415841|2415930|2415498|2415773|N|N|N|N|N| +2415864|AAAAAAAAIPMNECAA|1902-04-24|27|121|10|1902|4|4|24|2|1902|10|121|Thursday|1902Q2|N|N|N|2415841|2415930|2415499|2415774|N|N|N|N|N| +2415865|AAAAAAAAJPMNECAA|1902-04-25|27|121|10|1902|5|4|25|2|1902|10|121|Friday|1902Q2|N|Y|N|2415841|2415930|2415500|2415775|N|N|N|N|N| +2415866|AAAAAAAAKPMNECAA|1902-04-26|27|121|10|1902|6|4|26|2|1902|10|121|Saturday|1902Q2|N|Y|N|2415841|2415930|2415501|2415776|N|N|N|N|N| +2415867|AAAAAAAALPMNECAA|1902-04-27|27|121|10|1902|0|4|27|2|1902|10|121|Sunday|1902Q2|N|N|N|2415841|2415930|2415502|2415777|N|N|N|N|N| +2415868|AAAAAAAAMPMNECAA|1902-04-28|27|121|10|1902|1|4|28|2|1902|10|121|Monday|1902Q2|N|N|N|2415841|2415930|2415503|2415778|N|N|N|N|N| +2415869|AAAAAAAANPMNECAA|1902-04-29|27|122|10|1902|2|4|29|2|1902|10|122|Tuesday|1902Q2|N|N|N|2415841|2415930|2415504|2415779|N|N|N|N|N| +2415870|AAAAAAAAOPMNECAA|1902-04-30|27|122|10|1902|3|4|30|2|1902|10|122|Wednesday|1902Q2|N|N|N|2415841|2415930|2415505|2415780|N|N|N|N|N| +2415871|AAAAAAAAPPMNECAA|1902-05-01|28|122|10|1902|4|5|1|2|1902|10|122|Thursday|1902Q2|N|N|N|2415871|2415990|2415506|2415781|N|N|N|N|N| +2415872|AAAAAAAAAANNECAA|1902-05-02|28|122|10|1902|5|5|2|2|1902|10|122|Friday|1902Q2|N|Y|N|2415871|2415990|2415507|2415782|N|N|N|N|N| +2415873|AAAAAAAABANNECAA|1902-05-03|28|122|10|1902|6|5|3|2|1902|10|122|Saturday|1902Q2|N|Y|N|2415871|2415990|2415508|2415783|N|N|N|N|N| +2415874|AAAAAAAACANNECAA|1902-05-04|28|122|10|1902|0|5|4|2|1902|10|122|Sunday|1902Q2|N|N|N|2415871|2415990|2415509|2415784|N|N|N|N|N| +2415875|AAAAAAAADANNECAA|1902-05-05|28|122|10|1902|1|5|5|2|1902|10|122|Monday|1902Q2|N|N|N|2415871|2415990|2415510|2415785|N|N|N|N|N| +2415876|AAAAAAAAEANNECAA|1902-05-06|28|123|10|1902|2|5|6|2|1902|10|123|Tuesday|1902Q2|N|N|N|2415871|2415990|2415511|2415786|N|N|N|N|N| +2415877|AAAAAAAAFANNECAA|1902-05-07|28|123|10|1902|3|5|7|2|1902|10|123|Wednesday|1902Q2|N|N|N|2415871|2415990|2415512|2415787|N|N|N|N|N| +2415878|AAAAAAAAGANNECAA|1902-05-08|28|123|10|1902|4|5|8|2|1902|10|123|Thursday|1902Q2|N|N|N|2415871|2415990|2415513|2415788|N|N|N|N|N| +2415879|AAAAAAAAHANNECAA|1902-05-09|28|123|10|1902|5|5|9|2|1902|10|123|Friday|1902Q2|N|Y|N|2415871|2415990|2415514|2415789|N|N|N|N|N| +2415880|AAAAAAAAIANNECAA|1902-05-10|28|123|10|1902|6|5|10|2|1902|10|123|Saturday|1902Q2|N|Y|N|2415871|2415990|2415515|2415790|N|N|N|N|N| +2415881|AAAAAAAAJANNECAA|1902-05-11|28|123|10|1902|0|5|11|2|1902|10|123|Sunday|1902Q2|N|N|N|2415871|2415990|2415516|2415791|N|N|N|N|N| +2415882|AAAAAAAAKANNECAA|1902-05-12|28|123|10|1902|1|5|12|2|1902|10|123|Monday|1902Q2|N|N|N|2415871|2415990|2415517|2415792|N|N|N|N|N| +2415883|AAAAAAAALANNECAA|1902-05-13|28|124|10|1902|2|5|13|2|1902|10|124|Tuesday|1902Q2|N|N|N|2415871|2415990|2415518|2415793|N|N|N|N|N| +2415884|AAAAAAAAMANNECAA|1902-05-14|28|124|10|1902|3|5|14|2|1902|10|124|Wednesday|1902Q2|N|N|N|2415871|2415990|2415519|2415794|N|N|N|N|N| +2415885|AAAAAAAANANNECAA|1902-05-15|28|124|10|1902|4|5|15|2|1902|10|124|Thursday|1902Q2|N|N|N|2415871|2415990|2415520|2415795|N|N|N|N|N| +2415886|AAAAAAAAOANNECAA|1902-05-16|28|124|10|1902|5|5|16|2|1902|10|124|Friday|1902Q2|N|Y|N|2415871|2415990|2415521|2415796|N|N|N|N|N| +2415887|AAAAAAAAPANNECAA|1902-05-17|28|124|10|1902|6|5|17|2|1902|10|124|Saturday|1902Q2|N|Y|N|2415871|2415990|2415522|2415797|N|N|N|N|N| +2415888|AAAAAAAAABNNECAA|1902-05-18|28|124|10|1902|0|5|18|2|1902|10|124|Sunday|1902Q2|N|N|N|2415871|2415990|2415523|2415798|N|N|N|N|N| +2415889|AAAAAAAABBNNECAA|1902-05-19|28|124|10|1902|1|5|19|2|1902|10|124|Monday|1902Q2|N|N|N|2415871|2415990|2415524|2415799|N|N|N|N|N| +2415890|AAAAAAAACBNNECAA|1902-05-20|28|125|10|1902|2|5|20|2|1902|10|125|Tuesday|1902Q2|N|N|N|2415871|2415990|2415525|2415800|N|N|N|N|N| +2415891|AAAAAAAADBNNECAA|1902-05-21|28|125|10|1902|3|5|21|2|1902|10|125|Wednesday|1902Q2|N|N|N|2415871|2415990|2415526|2415801|N|N|N|N|N| +2415892|AAAAAAAAEBNNECAA|1902-05-22|28|125|10|1902|4|5|22|2|1902|10|125|Thursday|1902Q2|N|N|N|2415871|2415990|2415527|2415802|N|N|N|N|N| +2415893|AAAAAAAAFBNNECAA|1902-05-23|28|125|10|1902|5|5|23|2|1902|10|125|Friday|1902Q2|N|Y|N|2415871|2415990|2415528|2415803|N|N|N|N|N| +2415894|AAAAAAAAGBNNECAA|1902-05-24|28|125|10|1902|6|5|24|2|1902|10|125|Saturday|1902Q2|N|Y|N|2415871|2415990|2415529|2415804|N|N|N|N|N| +2415895|AAAAAAAAHBNNECAA|1902-05-25|28|125|10|1902|0|5|25|2|1902|10|125|Sunday|1902Q2|N|N|N|2415871|2415990|2415530|2415805|N|N|N|N|N| +2415896|AAAAAAAAIBNNECAA|1902-05-26|28|125|10|1902|1|5|26|2|1902|10|125|Monday|1902Q2|N|N|N|2415871|2415990|2415531|2415806|N|N|N|N|N| +2415897|AAAAAAAAJBNNECAA|1902-05-27|28|126|10|1902|2|5|27|2|1902|10|126|Tuesday|1902Q2|N|N|N|2415871|2415990|2415532|2415807|N|N|N|N|N| +2415898|AAAAAAAAKBNNECAA|1902-05-28|28|126|10|1902|3|5|28|2|1902|10|126|Wednesday|1902Q2|N|N|N|2415871|2415990|2415533|2415808|N|N|N|N|N| +2415899|AAAAAAAALBNNECAA|1902-05-29|28|126|10|1902|4|5|29|2|1902|10|126|Thursday|1902Q2|N|N|N|2415871|2415990|2415534|2415809|N|N|N|N|N| +2415900|AAAAAAAAMBNNECAA|1902-05-30|28|126|10|1902|5|5|30|2|1902|10|126|Friday|1902Q2|N|Y|N|2415871|2415990|2415535|2415810|N|N|N|N|N| +2415901|AAAAAAAANBNNECAA|1902-05-31|28|126|10|1902|6|5|31|2|1902|10|126|Saturday|1902Q2|N|Y|N|2415871|2415990|2415536|2415811|N|N|N|N|N| +2415902|AAAAAAAAOBNNECAA|1902-06-01|29|126|11|1902|0|6|1|2|1902|11|126|Sunday|1902Q2|N|N|N|2415902|2416052|2415537|2415812|N|N|N|N|N| +2415903|AAAAAAAAPBNNECAA|1902-06-02|29|126|11|1902|1|6|2|2|1902|11|126|Monday|1902Q2|N|N|N|2415902|2416052|2415538|2415813|N|N|N|N|N| +2415904|AAAAAAAAACNNECAA|1902-06-03|29|127|11|1902|2|6|3|2|1902|11|127|Tuesday|1902Q2|N|N|N|2415902|2416052|2415539|2415814|N|N|N|N|N| +2415905|AAAAAAAABCNNECAA|1902-06-04|29|127|11|1902|3|6|4|2|1902|11|127|Wednesday|1902Q2|N|N|N|2415902|2416052|2415540|2415815|N|N|N|N|N| +2415906|AAAAAAAACCNNECAA|1902-06-05|29|127|11|1902|4|6|5|2|1902|11|127|Thursday|1902Q2|N|N|N|2415902|2416052|2415541|2415816|N|N|N|N|N| +2415907|AAAAAAAADCNNECAA|1902-06-06|29|127|11|1902|5|6|6|2|1902|11|127|Friday|1902Q2|N|Y|N|2415902|2416052|2415542|2415817|N|N|N|N|N| +2415908|AAAAAAAAECNNECAA|1902-06-07|29|127|11|1902|6|6|7|2|1902|11|127|Saturday|1902Q2|N|Y|N|2415902|2416052|2415543|2415818|N|N|N|N|N| +2415909|AAAAAAAAFCNNECAA|1902-06-08|29|127|11|1902|0|6|8|2|1902|11|127|Sunday|1902Q2|N|N|N|2415902|2416052|2415544|2415819|N|N|N|N|N| +2415910|AAAAAAAAGCNNECAA|1902-06-09|29|127|11|1902|1|6|9|2|1902|11|127|Monday|1902Q2|N|N|N|2415902|2416052|2415545|2415820|N|N|N|N|N| +2415911|AAAAAAAAHCNNECAA|1902-06-10|29|128|11|1902|2|6|10|2|1902|11|128|Tuesday|1902Q2|N|N|N|2415902|2416052|2415546|2415821|N|N|N|N|N| +2415912|AAAAAAAAICNNECAA|1902-06-11|29|128|11|1902|3|6|11|2|1902|11|128|Wednesday|1902Q2|N|N|N|2415902|2416052|2415547|2415822|N|N|N|N|N| +2415913|AAAAAAAAJCNNECAA|1902-06-12|29|128|11|1902|4|6|12|2|1902|11|128|Thursday|1902Q2|N|N|N|2415902|2416052|2415548|2415823|N|N|N|N|N| +2415914|AAAAAAAAKCNNECAA|1902-06-13|29|128|11|1902|5|6|13|2|1902|11|128|Friday|1902Q2|N|Y|N|2415902|2416052|2415549|2415824|N|N|N|N|N| +2415915|AAAAAAAALCNNECAA|1902-06-14|29|128|11|1902|6|6|14|2|1902|11|128|Saturday|1902Q2|N|Y|N|2415902|2416052|2415550|2415825|N|N|N|N|N| +2415916|AAAAAAAAMCNNECAA|1902-06-15|29|128|11|1902|0|6|15|2|1902|11|128|Sunday|1902Q2|N|N|N|2415902|2416052|2415551|2415826|N|N|N|N|N| +2415917|AAAAAAAANCNNECAA|1902-06-16|29|128|11|1902|1|6|16|2|1902|11|128|Monday|1902Q2|N|N|N|2415902|2416052|2415552|2415827|N|N|N|N|N| +2415918|AAAAAAAAOCNNECAA|1902-06-17|29|129|11|1902|2|6|17|2|1902|11|129|Tuesday|1902Q2|N|N|N|2415902|2416052|2415553|2415828|N|N|N|N|N| +2415919|AAAAAAAAPCNNECAA|1902-06-18|29|129|11|1902|3|6|18|2|1902|11|129|Wednesday|1902Q2|N|N|N|2415902|2416052|2415554|2415829|N|N|N|N|N| +2415920|AAAAAAAAADNNECAA|1902-06-19|29|129|11|1902|4|6|19|2|1902|11|129|Thursday|1902Q2|N|N|N|2415902|2416052|2415555|2415830|N|N|N|N|N| +2415921|AAAAAAAABDNNECAA|1902-06-20|29|129|11|1902|5|6|20|2|1902|11|129|Friday|1902Q2|N|Y|N|2415902|2416052|2415556|2415831|N|N|N|N|N| +2415922|AAAAAAAACDNNECAA|1902-06-21|29|129|11|1902|6|6|21|2|1902|11|129|Saturday|1902Q2|N|Y|N|2415902|2416052|2415557|2415832|N|N|N|N|N| +2415923|AAAAAAAADDNNECAA|1902-06-22|29|129|11|1902|0|6|22|2|1902|11|129|Sunday|1902Q2|N|N|N|2415902|2416052|2415558|2415833|N|N|N|N|N| +2415924|AAAAAAAAEDNNECAA|1902-06-23|29|129|11|1902|1|6|23|2|1902|11|129|Monday|1902Q2|N|N|N|2415902|2416052|2415559|2415834|N|N|N|N|N| +2415925|AAAAAAAAFDNNECAA|1902-06-24|29|130|11|1902|2|6|24|2|1902|11|130|Tuesday|1902Q2|N|N|N|2415902|2416052|2415560|2415835|N|N|N|N|N| +2415926|AAAAAAAAGDNNECAA|1902-06-25|29|130|11|1902|3|6|25|2|1902|11|130|Wednesday|1902Q2|N|N|N|2415902|2416052|2415561|2415836|N|N|N|N|N| +2415927|AAAAAAAAHDNNECAA|1902-06-26|29|130|11|1902|4|6|26|2|1902|11|130|Thursday|1902Q2|N|N|N|2415902|2416052|2415562|2415837|N|N|N|N|N| +2415928|AAAAAAAAIDNNECAA|1902-06-27|29|130|11|1902|5|6|27|2|1902|11|130|Friday|1902Q2|N|Y|N|2415902|2416052|2415563|2415838|N|N|N|N|N| +2415929|AAAAAAAAJDNNECAA|1902-06-28|29|130|11|1902|6|6|28|2|1902|11|130|Saturday|1902Q2|N|Y|N|2415902|2416052|2415564|2415839|N|N|N|N|N| +2415930|AAAAAAAAKDNNECAA|1902-06-29|29|130|11|1902|0|6|29|2|1902|11|130|Sunday|1902Q2|N|N|N|2415902|2416052|2415565|2415840|N|N|N|N|N| +2415931|AAAAAAAALDNNECAA|1902-06-30|29|130|11|1902|1|6|30|2|1902|11|130|Monday|1902Q2|N|N|N|2415902|2416052|2415566|2415841|N|N|N|N|N| +2415932|AAAAAAAAMDNNECAA|1902-07-01|30|131|11|1902|2|7|1|2|1902|11|131|Tuesday|1902Q2|N|N|N|2415932|2416112|2415567|2415841|N|N|N|N|N| +2415933|AAAAAAAANDNNECAA|1902-07-02|30|131|11|1902|3|7|2|3|1902|11|131|Wednesday|1902Q3|N|N|N|2415932|2416112|2415568|2415842|N|N|N|N|N| +2415934|AAAAAAAAODNNECAA|1902-07-03|30|131|11|1902|4|7|3|3|1902|11|131|Thursday|1902Q3|N|N|N|2415932|2416112|2415569|2415843|N|N|N|N|N| +2415935|AAAAAAAAPDNNECAA|1902-07-04|30|131|11|1902|5|7|4|3|1902|11|131|Friday|1902Q3|N|Y|N|2415932|2416112|2415570|2415844|N|N|N|N|N| +2415936|AAAAAAAAAENNECAA|1902-07-05|30|131|11|1902|6|7|5|3|1902|11|131|Saturday|1902Q3|Y|Y|N|2415932|2416112|2415571|2415845|N|N|N|N|N| +2415937|AAAAAAAABENNECAA|1902-07-06|30|131|11|1902|0|7|6|3|1902|11|131|Sunday|1902Q3|N|N|Y|2415932|2416112|2415572|2415846|N|N|N|N|N| +2415938|AAAAAAAACENNECAA|1902-07-07|30|131|11|1902|1|7|7|3|1902|11|131|Monday|1902Q3|N|N|N|2415932|2416112|2415573|2415847|N|N|N|N|N| +2415939|AAAAAAAADENNECAA|1902-07-08|30|132|11|1902|2|7|8|3|1902|11|132|Tuesday|1902Q3|N|N|N|2415932|2416112|2415574|2415848|N|N|N|N|N| +2415940|AAAAAAAAEENNECAA|1902-07-09|30|132|11|1902|3|7|9|3|1902|11|132|Wednesday|1902Q3|N|N|N|2415932|2416112|2415575|2415849|N|N|N|N|N| +2415941|AAAAAAAAFENNECAA|1902-07-10|30|132|11|1902|4|7|10|3|1902|11|132|Thursday|1902Q3|N|N|N|2415932|2416112|2415576|2415850|N|N|N|N|N| +2415942|AAAAAAAAGENNECAA|1902-07-11|30|132|11|1902|5|7|11|3|1902|11|132|Friday|1902Q3|N|Y|N|2415932|2416112|2415577|2415851|N|N|N|N|N| +2415943|AAAAAAAAHENNECAA|1902-07-12|30|132|11|1902|6|7|12|3|1902|11|132|Saturday|1902Q3|N|Y|N|2415932|2416112|2415578|2415852|N|N|N|N|N| +2415944|AAAAAAAAIENNECAA|1902-07-13|30|132|11|1902|0|7|13|3|1902|11|132|Sunday|1902Q3|N|N|N|2415932|2416112|2415579|2415853|N|N|N|N|N| +2415945|AAAAAAAAJENNECAA|1902-07-14|30|132|11|1902|1|7|14|3|1902|11|132|Monday|1902Q3|N|N|N|2415932|2416112|2415580|2415854|N|N|N|N|N| +2415946|AAAAAAAAKENNECAA|1902-07-15|30|133|11|1902|2|7|15|3|1902|11|133|Tuesday|1902Q3|N|N|N|2415932|2416112|2415581|2415855|N|N|N|N|N| +2415947|AAAAAAAALENNECAA|1902-07-16|30|133|11|1902|3|7|16|3|1902|11|133|Wednesday|1902Q3|N|N|N|2415932|2416112|2415582|2415856|N|N|N|N|N| +2415948|AAAAAAAAMENNECAA|1902-07-17|30|133|11|1902|4|7|17|3|1902|11|133|Thursday|1902Q3|N|N|N|2415932|2416112|2415583|2415857|N|N|N|N|N| +2415949|AAAAAAAANENNECAA|1902-07-18|30|133|11|1902|5|7|18|3|1902|11|133|Friday|1902Q3|N|Y|N|2415932|2416112|2415584|2415858|N|N|N|N|N| +2415950|AAAAAAAAOENNECAA|1902-07-19|30|133|11|1902|6|7|19|3|1902|11|133|Saturday|1902Q3|N|Y|N|2415932|2416112|2415585|2415859|N|N|N|N|N| +2415951|AAAAAAAAPENNECAA|1902-07-20|30|133|11|1902|0|7|20|3|1902|11|133|Sunday|1902Q3|N|N|N|2415932|2416112|2415586|2415860|N|N|N|N|N| +2415952|AAAAAAAAAFNNECAA|1902-07-21|30|133|11|1902|1|7|21|3|1902|11|133|Monday|1902Q3|N|N|N|2415932|2416112|2415587|2415861|N|N|N|N|N| +2415953|AAAAAAAABFNNECAA|1902-07-22|30|134|11|1902|2|7|22|3|1902|11|134|Tuesday|1902Q3|N|N|N|2415932|2416112|2415588|2415862|N|N|N|N|N| +2415954|AAAAAAAACFNNECAA|1902-07-23|30|134|11|1902|3|7|23|3|1902|11|134|Wednesday|1902Q3|N|N|N|2415932|2416112|2415589|2415863|N|N|N|N|N| +2415955|AAAAAAAADFNNECAA|1902-07-24|30|134|11|1902|4|7|24|3|1902|11|134|Thursday|1902Q3|N|N|N|2415932|2416112|2415590|2415864|N|N|N|N|N| +2415956|AAAAAAAAEFNNECAA|1902-07-25|30|134|11|1902|5|7|25|3|1902|11|134|Friday|1902Q3|N|Y|N|2415932|2416112|2415591|2415865|N|N|N|N|N| +2415957|AAAAAAAAFFNNECAA|1902-07-26|30|134|11|1902|6|7|26|3|1902|11|134|Saturday|1902Q3|N|Y|N|2415932|2416112|2415592|2415866|N|N|N|N|N| +2415958|AAAAAAAAGFNNECAA|1902-07-27|30|134|11|1902|0|7|27|3|1902|11|134|Sunday|1902Q3|N|N|N|2415932|2416112|2415593|2415867|N|N|N|N|N| +2415959|AAAAAAAAHFNNECAA|1902-07-28|30|134|11|1902|1|7|28|3|1902|11|134|Monday|1902Q3|N|N|N|2415932|2416112|2415594|2415868|N|N|N|N|N| +2415960|AAAAAAAAIFNNECAA|1902-07-29|30|135|11|1902|2|7|29|3|1902|11|135|Tuesday|1902Q3|N|N|N|2415932|2416112|2415595|2415869|N|N|N|N|N| +2415961|AAAAAAAAJFNNECAA|1902-07-30|30|135|11|1902|3|7|30|3|1902|11|135|Wednesday|1902Q3|N|N|N|2415932|2416112|2415596|2415870|N|N|N|N|N| +2415962|AAAAAAAAKFNNECAA|1902-07-31|30|135|11|1902|4|7|31|3|1902|11|135|Thursday|1902Q3|N|N|N|2415932|2416112|2415597|2415871|N|N|N|N|N| +2415963|AAAAAAAALFNNECAA|1902-08-01|31|135|11|1902|5|8|1|3|1902|11|135|Friday|1902Q3|N|Y|N|2415963|2416174|2415598|2415872|N|N|N|N|N| +2415964|AAAAAAAAMFNNECAA|1902-08-02|31|135|11|1902|6|8|2|3|1902|11|135|Saturday|1902Q3|N|Y|N|2415963|2416174|2415599|2415873|N|N|N|N|N| +2415965|AAAAAAAANFNNECAA|1902-08-03|31|135|11|1902|0|8|3|3|1902|11|135|Sunday|1902Q3|N|N|N|2415963|2416174|2415600|2415874|N|N|N|N|N| +2415966|AAAAAAAAOFNNECAA|1902-08-04|31|135|11|1902|1|8|4|3|1902|11|135|Monday|1902Q3|N|N|N|2415963|2416174|2415601|2415875|N|N|N|N|N| +2415967|AAAAAAAAPFNNECAA|1902-08-05|31|136|11|1902|2|8|5|3|1902|11|136|Tuesday|1902Q3|N|N|N|2415963|2416174|2415602|2415876|N|N|N|N|N| +2415968|AAAAAAAAAGNNECAA|1902-08-06|31|136|11|1902|3|8|6|3|1902|11|136|Wednesday|1902Q3|N|N|N|2415963|2416174|2415603|2415877|N|N|N|N|N| +2415969|AAAAAAAABGNNECAA|1902-08-07|31|136|11|1902|4|8|7|3|1902|11|136|Thursday|1902Q3|N|N|N|2415963|2416174|2415604|2415878|N|N|N|N|N| +2415970|AAAAAAAACGNNECAA|1902-08-08|31|136|11|1902|5|8|8|3|1902|11|136|Friday|1902Q3|N|Y|N|2415963|2416174|2415605|2415879|N|N|N|N|N| +2415971|AAAAAAAADGNNECAA|1902-08-09|31|136|11|1902|6|8|9|3|1902|11|136|Saturday|1902Q3|N|Y|N|2415963|2416174|2415606|2415880|N|N|N|N|N| +2415972|AAAAAAAAEGNNECAA|1902-08-10|31|136|11|1902|0|8|10|3|1902|11|136|Sunday|1902Q3|N|N|N|2415963|2416174|2415607|2415881|N|N|N|N|N| +2415973|AAAAAAAAFGNNECAA|1902-08-11|31|136|11|1902|1|8|11|3|1902|11|136|Monday|1902Q3|N|N|N|2415963|2416174|2415608|2415882|N|N|N|N|N| +2415974|AAAAAAAAGGNNECAA|1902-08-12|31|137|11|1902|2|8|12|3|1902|11|137|Tuesday|1902Q3|N|N|N|2415963|2416174|2415609|2415883|N|N|N|N|N| +2415975|AAAAAAAAHGNNECAA|1902-08-13|31|137|11|1902|3|8|13|3|1902|11|137|Wednesday|1902Q3|N|N|N|2415963|2416174|2415610|2415884|N|N|N|N|N| +2415976|AAAAAAAAIGNNECAA|1902-08-14|31|137|11|1902|4|8|14|3|1902|11|137|Thursday|1902Q3|N|N|N|2415963|2416174|2415611|2415885|N|N|N|N|N| +2415977|AAAAAAAAJGNNECAA|1902-08-15|31|137|11|1902|5|8|15|3|1902|11|137|Friday|1902Q3|N|Y|N|2415963|2416174|2415612|2415886|N|N|N|N|N| +2415978|AAAAAAAAKGNNECAA|1902-08-16|31|137|11|1902|6|8|16|3|1902|11|137|Saturday|1902Q3|N|Y|N|2415963|2416174|2415613|2415887|N|N|N|N|N| +2415979|AAAAAAAALGNNECAA|1902-08-17|31|137|11|1902|0|8|17|3|1902|11|137|Sunday|1902Q3|N|N|N|2415963|2416174|2415614|2415888|N|N|N|N|N| +2415980|AAAAAAAAMGNNECAA|1902-08-18|31|137|11|1902|1|8|18|3|1902|11|137|Monday|1902Q3|N|N|N|2415963|2416174|2415615|2415889|N|N|N|N|N| +2415981|AAAAAAAANGNNECAA|1902-08-19|31|138|11|1902|2|8|19|3|1902|11|138|Tuesday|1902Q3|N|N|N|2415963|2416174|2415616|2415890|N|N|N|N|N| +2415982|AAAAAAAAOGNNECAA|1902-08-20|31|138|11|1902|3|8|20|3|1902|11|138|Wednesday|1902Q3|N|N|N|2415963|2416174|2415617|2415891|N|N|N|N|N| +2415983|AAAAAAAAPGNNECAA|1902-08-21|31|138|11|1902|4|8|21|3|1902|11|138|Thursday|1902Q3|N|N|N|2415963|2416174|2415618|2415892|N|N|N|N|N| +2415984|AAAAAAAAAHNNECAA|1902-08-22|31|138|11|1902|5|8|22|3|1902|11|138|Friday|1902Q3|N|Y|N|2415963|2416174|2415619|2415893|N|N|N|N|N| +2415985|AAAAAAAABHNNECAA|1902-08-23|31|138|11|1902|6|8|23|3|1902|11|138|Saturday|1902Q3|N|Y|N|2415963|2416174|2415620|2415894|N|N|N|N|N| +2415986|AAAAAAAACHNNECAA|1902-08-24|31|138|11|1902|0|8|24|3|1902|11|138|Sunday|1902Q3|N|N|N|2415963|2416174|2415621|2415895|N|N|N|N|N| +2415987|AAAAAAAADHNNECAA|1902-08-25|31|138|11|1902|1|8|25|3|1902|11|138|Monday|1902Q3|N|N|N|2415963|2416174|2415622|2415896|N|N|N|N|N| +2415988|AAAAAAAAEHNNECAA|1902-08-26|31|139|11|1902|2|8|26|3|1902|11|139|Tuesday|1902Q3|N|N|N|2415963|2416174|2415623|2415897|N|N|N|N|N| +2415989|AAAAAAAAFHNNECAA|1902-08-27|31|139|11|1902|3|8|27|3|1902|11|139|Wednesday|1902Q3|N|N|N|2415963|2416174|2415624|2415898|N|N|N|N|N| +2415990|AAAAAAAAGHNNECAA|1902-08-28|31|139|11|1902|4|8|28|3|1902|11|139|Thursday|1902Q3|N|N|N|2415963|2416174|2415625|2415899|N|N|N|N|N| +2415991|AAAAAAAAHHNNECAA|1902-08-29|31|139|11|1902|5|8|29|3|1902|11|139|Friday|1902Q3|N|Y|N|2415963|2416174|2415626|2415900|N|N|N|N|N| +2415992|AAAAAAAAIHNNECAA|1902-08-30|31|139|11|1902|6|8|30|3|1902|11|139|Saturday|1902Q3|N|Y|N|2415963|2416174|2415627|2415901|N|N|N|N|N| +2415993|AAAAAAAAJHNNECAA|1902-08-31|31|139|11|1902|0|8|31|3|1902|11|139|Sunday|1902Q3|N|N|N|2415963|2416174|2415628|2415902|N|N|N|N|N| +2415994|AAAAAAAAKHNNECAA|1902-09-01|32|139|12|1902|1|9|1|3|1902|12|139|Monday|1902Q3|N|N|N|2415994|2416236|2415629|2415903|N|N|N|N|N| +2415995|AAAAAAAALHNNECAA|1902-09-02|32|140|12|1902|2|9|2|3|1902|12|140|Tuesday|1902Q3|N|N|N|2415994|2416236|2415630|2415904|N|N|N|N|N| +2415996|AAAAAAAAMHNNECAA|1902-09-03|32|140|12|1902|3|9|3|3|1902|12|140|Wednesday|1902Q3|N|N|N|2415994|2416236|2415631|2415905|N|N|N|N|N| +2415997|AAAAAAAANHNNECAA|1902-09-04|32|140|12|1902|4|9|4|3|1902|12|140|Thursday|1902Q3|N|N|N|2415994|2416236|2415632|2415906|N|N|N|N|N| +2415998|AAAAAAAAOHNNECAA|1902-09-05|32|140|12|1902|5|9|5|3|1902|12|140|Friday|1902Q3|N|Y|N|2415994|2416236|2415633|2415907|N|N|N|N|N| +2415999|AAAAAAAAPHNNECAA|1902-09-06|32|140|12|1902|6|9|6|3|1902|12|140|Saturday|1902Q3|N|Y|N|2415994|2416236|2415634|2415908|N|N|N|N|N| +2416000|AAAAAAAAAINNECAA|1902-09-07|32|140|12|1902|0|9|7|3|1902|12|140|Sunday|1902Q3|N|N|N|2415994|2416236|2415635|2415909|N|N|N|N|N| +2416001|AAAAAAAABINNECAA|1902-09-08|32|140|12|1902|1|9|8|3|1902|12|140|Monday|1902Q3|N|N|N|2415994|2416236|2415636|2415910|N|N|N|N|N| +2416002|AAAAAAAACINNECAA|1902-09-09|32|141|12|1902|2|9|9|3|1902|12|141|Tuesday|1902Q3|N|N|N|2415994|2416236|2415637|2415911|N|N|N|N|N| +2416003|AAAAAAAADINNECAA|1902-09-10|32|141|12|1902|3|9|10|3|1902|12|141|Wednesday|1902Q3|N|N|N|2415994|2416236|2415638|2415912|N|N|N|N|N| +2416004|AAAAAAAAEINNECAA|1902-09-11|32|141|12|1902|4|9|11|3|1902|12|141|Thursday|1902Q3|N|N|N|2415994|2416236|2415639|2415913|N|N|N|N|N| +2416005|AAAAAAAAFINNECAA|1902-09-12|32|141|12|1902|5|9|12|3|1902|12|141|Friday|1902Q3|N|Y|N|2415994|2416236|2415640|2415914|N|N|N|N|N| +2416006|AAAAAAAAGINNECAA|1902-09-13|32|141|12|1902|6|9|13|3|1902|12|141|Saturday|1902Q3|N|Y|N|2415994|2416236|2415641|2415915|N|N|N|N|N| +2416007|AAAAAAAAHINNECAA|1902-09-14|32|141|12|1902|0|9|14|3|1902|12|141|Sunday|1902Q3|N|N|N|2415994|2416236|2415642|2415916|N|N|N|N|N| +2416008|AAAAAAAAIINNECAA|1902-09-15|32|141|12|1902|1|9|15|3|1902|12|141|Monday|1902Q3|N|N|N|2415994|2416236|2415643|2415917|N|N|N|N|N| +2416009|AAAAAAAAJINNECAA|1902-09-16|32|142|12|1902|2|9|16|3|1902|12|142|Tuesday|1902Q3|N|N|N|2415994|2416236|2415644|2415918|N|N|N|N|N| +2416010|AAAAAAAAKINNECAA|1902-09-17|32|142|12|1902|3|9|17|3|1902|12|142|Wednesday|1902Q3|N|N|N|2415994|2416236|2415645|2415919|N|N|N|N|N| +2416011|AAAAAAAALINNECAA|1902-09-18|32|142|12|1902|4|9|18|3|1902|12|142|Thursday|1902Q3|N|N|N|2415994|2416236|2415646|2415920|N|N|N|N|N| +2416012|AAAAAAAAMINNECAA|1902-09-19|32|142|12|1902|5|9|19|3|1902|12|142|Friday|1902Q3|N|Y|N|2415994|2416236|2415647|2415921|N|N|N|N|N| +2416013|AAAAAAAANINNECAA|1902-09-20|32|142|12|1902|6|9|20|3|1902|12|142|Saturday|1902Q3|N|Y|N|2415994|2416236|2415648|2415922|N|N|N|N|N| +2416014|AAAAAAAAOINNECAA|1902-09-21|32|142|12|1902|0|9|21|3|1902|12|142|Sunday|1902Q3|N|N|N|2415994|2416236|2415649|2415923|N|N|N|N|N| +2416015|AAAAAAAAPINNECAA|1902-09-22|32|142|12|1902|1|9|22|3|1902|12|142|Monday|1902Q3|N|N|N|2415994|2416236|2415650|2415924|N|N|N|N|N| +2416016|AAAAAAAAAJNNECAA|1902-09-23|32|143|12|1902|2|9|23|3|1902|12|143|Tuesday|1902Q3|N|N|N|2415994|2416236|2415651|2415925|N|N|N|N|N| +2416017|AAAAAAAABJNNECAA|1902-09-24|32|143|12|1902|3|9|24|3|1902|12|143|Wednesday|1902Q3|N|N|N|2415994|2416236|2415652|2415926|N|N|N|N|N| +2416018|AAAAAAAACJNNECAA|1902-09-25|32|143|12|1902|4|9|25|3|1902|12|143|Thursday|1902Q3|N|N|N|2415994|2416236|2415653|2415927|N|N|N|N|N| +2416019|AAAAAAAADJNNECAA|1902-09-26|32|143|12|1902|5|9|26|3|1902|12|143|Friday|1902Q3|N|Y|N|2415994|2416236|2415654|2415928|N|N|N|N|N| +2416020|AAAAAAAAEJNNECAA|1902-09-27|32|143|12|1902|6|9|27|3|1902|12|143|Saturday|1902Q3|N|Y|N|2415994|2416236|2415655|2415929|N|N|N|N|N| +2416021|AAAAAAAAFJNNECAA|1902-09-28|32|143|12|1902|0|9|28|3|1902|12|143|Sunday|1902Q3|N|N|N|2415994|2416236|2415656|2415930|N|N|N|N|N| +2416022|AAAAAAAAGJNNECAA|1902-09-29|32|143|12|1902|1|9|29|3|1902|12|143|Monday|1902Q3|N|N|N|2415994|2416236|2415657|2415931|N|N|N|N|N| +2416023|AAAAAAAAHJNNECAA|1902-09-30|32|144|12|1902|2|9|30|3|1902|12|144|Tuesday|1902Q3|N|N|N|2415994|2416236|2415658|2415932|N|N|N|N|N| +2416024|AAAAAAAAIJNNECAA|1902-10-01|33|144|12|1902|3|10|1|3|1902|12|144|Wednesday|1902Q3|N|N|N|2416024|2416296|2415659|2415932|N|N|N|N|N| +2416025|AAAAAAAAJJNNECAA|1902-10-02|33|144|12|1902|4|10|2|4|1902|12|144|Thursday|1902Q4|N|N|N|2416024|2416296|2415660|2415933|N|N|N|N|N| +2416026|AAAAAAAAKJNNECAA|1902-10-03|33|144|12|1902|5|10|3|4|1902|12|144|Friday|1902Q4|N|Y|N|2416024|2416296|2415661|2415934|N|N|N|N|N| +2416027|AAAAAAAALJNNECAA|1902-10-04|33|144|12|1902|6|10|4|4|1902|12|144|Saturday|1902Q4|N|Y|N|2416024|2416296|2415662|2415935|N|N|N|N|N| +2416028|AAAAAAAAMJNNECAA|1902-10-05|33|144|12|1902|0|10|5|4|1902|12|144|Sunday|1902Q4|N|N|N|2416024|2416296|2415663|2415936|N|N|N|N|N| +2416029|AAAAAAAANJNNECAA|1902-10-06|33|144|12|1902|1|10|6|4|1902|12|144|Monday|1902Q4|N|N|N|2416024|2416296|2415664|2415937|N|N|N|N|N| +2416030|AAAAAAAAOJNNECAA|1902-10-07|33|145|12|1902|2|10|7|4|1902|12|145|Tuesday|1902Q4|N|N|N|2416024|2416296|2415665|2415938|N|N|N|N|N| +2416031|AAAAAAAAPJNNECAA|1902-10-08|33|145|12|1902|3|10|8|4|1902|12|145|Wednesday|1902Q4|N|N|N|2416024|2416296|2415666|2415939|N|N|N|N|N| +2416032|AAAAAAAAAKNNECAA|1902-10-09|33|145|12|1902|4|10|9|4|1902|12|145|Thursday|1902Q4|N|N|N|2416024|2416296|2415667|2415940|N|N|N|N|N| +2416033|AAAAAAAABKNNECAA|1902-10-10|33|145|12|1902|5|10|10|4|1902|12|145|Friday|1902Q4|N|Y|N|2416024|2416296|2415668|2415941|N|N|N|N|N| +2416034|AAAAAAAACKNNECAA|1902-10-11|33|145|12|1902|6|10|11|4|1902|12|145|Saturday|1902Q4|N|Y|N|2416024|2416296|2415669|2415942|N|N|N|N|N| +2416035|AAAAAAAADKNNECAA|1902-10-12|33|145|12|1902|0|10|12|4|1902|12|145|Sunday|1902Q4|N|N|N|2416024|2416296|2415670|2415943|N|N|N|N|N| +2416036|AAAAAAAAEKNNECAA|1902-10-13|33|145|12|1902|1|10|13|4|1902|12|145|Monday|1902Q4|N|N|N|2416024|2416296|2415671|2415944|N|N|N|N|N| +2416037|AAAAAAAAFKNNECAA|1902-10-14|33|146|12|1902|2|10|14|4|1902|12|146|Tuesday|1902Q4|N|N|N|2416024|2416296|2415672|2415945|N|N|N|N|N| +2416038|AAAAAAAAGKNNECAA|1902-10-15|33|146|12|1902|3|10|15|4|1902|12|146|Wednesday|1902Q4|N|N|N|2416024|2416296|2415673|2415946|N|N|N|N|N| +2416039|AAAAAAAAHKNNECAA|1902-10-16|33|146|12|1902|4|10|16|4|1902|12|146|Thursday|1902Q4|N|N|N|2416024|2416296|2415674|2415947|N|N|N|N|N| +2416040|AAAAAAAAIKNNECAA|1902-10-17|33|146|12|1902|5|10|17|4|1902|12|146|Friday|1902Q4|N|Y|N|2416024|2416296|2415675|2415948|N|N|N|N|N| +2416041|AAAAAAAAJKNNECAA|1902-10-18|33|146|12|1902|6|10|18|4|1902|12|146|Saturday|1902Q4|N|Y|N|2416024|2416296|2415676|2415949|N|N|N|N|N| +2416042|AAAAAAAAKKNNECAA|1902-10-19|33|146|12|1902|0|10|19|4|1902|12|146|Sunday|1902Q4|N|N|N|2416024|2416296|2415677|2415950|N|N|N|N|N| +2416043|AAAAAAAALKNNECAA|1902-10-20|33|146|12|1902|1|10|20|4|1902|12|146|Monday|1902Q4|N|N|N|2416024|2416296|2415678|2415951|N|N|N|N|N| +2416044|AAAAAAAAMKNNECAA|1902-10-21|33|147|12|1902|2|10|21|4|1902|12|147|Tuesday|1902Q4|N|N|N|2416024|2416296|2415679|2415952|N|N|N|N|N| +2416045|AAAAAAAANKNNECAA|1902-10-22|33|147|12|1902|3|10|22|4|1902|12|147|Wednesday|1902Q4|N|N|N|2416024|2416296|2415680|2415953|N|N|N|N|N| +2416046|AAAAAAAAOKNNECAA|1902-10-23|33|147|12|1902|4|10|23|4|1902|12|147|Thursday|1902Q4|N|N|N|2416024|2416296|2415681|2415954|N|N|N|N|N| +2416047|AAAAAAAAPKNNECAA|1902-10-24|33|147|12|1902|5|10|24|4|1902|12|147|Friday|1902Q4|N|Y|N|2416024|2416296|2415682|2415955|N|N|N|N|N| +2416048|AAAAAAAAALNNECAA|1902-10-25|33|147|12|1902|6|10|25|4|1902|12|147|Saturday|1902Q4|N|Y|N|2416024|2416296|2415683|2415956|N|N|N|N|N| +2416049|AAAAAAAABLNNECAA|1902-10-26|33|147|12|1902|0|10|26|4|1902|12|147|Sunday|1902Q4|N|N|N|2416024|2416296|2415684|2415957|N|N|N|N|N| +2416050|AAAAAAAACLNNECAA|1902-10-27|33|147|12|1902|1|10|27|4|1902|12|147|Monday|1902Q4|N|N|N|2416024|2416296|2415685|2415958|N|N|N|N|N| +2416051|AAAAAAAADLNNECAA|1902-10-28|33|148|12|1902|2|10|28|4|1902|12|148|Tuesday|1902Q4|N|N|N|2416024|2416296|2415686|2415959|N|N|N|N|N| +2416052|AAAAAAAAELNNECAA|1902-10-29|33|148|12|1902|3|10|29|4|1902|12|148|Wednesday|1902Q4|N|N|N|2416024|2416296|2415687|2415960|N|N|N|N|N| +2416053|AAAAAAAAFLNNECAA|1902-10-30|33|148|12|1902|4|10|30|4|1902|12|148|Thursday|1902Q4|N|N|N|2416024|2416296|2415688|2415961|N|N|N|N|N| +2416054|AAAAAAAAGLNNECAA|1902-10-31|33|148|12|1902|5|10|31|4|1902|12|148|Friday|1902Q4|N|Y|N|2416024|2416296|2415689|2415962|N|N|N|N|N| +2416055|AAAAAAAAHLNNECAA|1902-11-01|34|148|12|1902|6|11|1|4|1902|12|148|Saturday|1902Q4|N|Y|N|2416055|2416358|2415690|2415963|N|N|N|N|N| +2416056|AAAAAAAAILNNECAA|1902-11-02|34|148|12|1902|0|11|2|4|1902|12|148|Sunday|1902Q4|N|N|N|2416055|2416358|2415691|2415964|N|N|N|N|N| +2416057|AAAAAAAAJLNNECAA|1902-11-03|34|148|12|1902|1|11|3|4|1902|12|148|Monday|1902Q4|N|N|N|2416055|2416358|2415692|2415965|N|N|N|N|N| +2416058|AAAAAAAAKLNNECAA|1902-11-04|34|149|12|1902|2|11|4|4|1902|12|149|Tuesday|1902Q4|N|N|N|2416055|2416358|2415693|2415966|N|N|N|N|N| +2416059|AAAAAAAALLNNECAA|1902-11-05|34|149|12|1902|3|11|5|4|1902|12|149|Wednesday|1902Q4|N|N|N|2416055|2416358|2415694|2415967|N|N|N|N|N| +2416060|AAAAAAAAMLNNECAA|1902-11-06|34|149|12|1902|4|11|6|4|1902|12|149|Thursday|1902Q4|N|N|N|2416055|2416358|2415695|2415968|N|N|N|N|N| +2416061|AAAAAAAANLNNECAA|1902-11-07|34|149|12|1902|5|11|7|4|1902|12|149|Friday|1902Q4|N|Y|N|2416055|2416358|2415696|2415969|N|N|N|N|N| +2416062|AAAAAAAAOLNNECAA|1902-11-08|34|149|12|1902|6|11|8|4|1902|12|149|Saturday|1902Q4|N|Y|N|2416055|2416358|2415697|2415970|N|N|N|N|N| +2416063|AAAAAAAAPLNNECAA|1902-11-09|34|149|12|1902|0|11|9|4|1902|12|149|Sunday|1902Q4|N|N|N|2416055|2416358|2415698|2415971|N|N|N|N|N| +2416064|AAAAAAAAAMNNECAA|1902-11-10|34|149|12|1902|1|11|10|4|1902|12|149|Monday|1902Q4|N|N|N|2416055|2416358|2415699|2415972|N|N|N|N|N| +2416065|AAAAAAAABMNNECAA|1902-11-11|34|150|12|1902|2|11|11|4|1902|12|150|Tuesday|1902Q4|N|N|N|2416055|2416358|2415700|2415973|N|N|N|N|N| +2416066|AAAAAAAACMNNECAA|1902-11-12|34|150|12|1902|3|11|12|4|1902|12|150|Wednesday|1902Q4|N|N|N|2416055|2416358|2415701|2415974|N|N|N|N|N| +2416067|AAAAAAAADMNNECAA|1902-11-13|34|150|12|1902|4|11|13|4|1902|12|150|Thursday|1902Q4|N|N|N|2416055|2416358|2415702|2415975|N|N|N|N|N| +2416068|AAAAAAAAEMNNECAA|1902-11-14|34|150|12|1902|5|11|14|4|1902|12|150|Friday|1902Q4|N|Y|N|2416055|2416358|2415703|2415976|N|N|N|N|N| +2416069|AAAAAAAAFMNNECAA|1902-11-15|34|150|12|1902|6|11|15|4|1902|12|150|Saturday|1902Q4|N|Y|N|2416055|2416358|2415704|2415977|N|N|N|N|N| +2416070|AAAAAAAAGMNNECAA|1902-11-16|34|150|12|1902|0|11|16|4|1902|12|150|Sunday|1902Q4|N|N|N|2416055|2416358|2415705|2415978|N|N|N|N|N| +2416071|AAAAAAAAHMNNECAA|1902-11-17|34|150|12|1902|1|11|17|4|1902|12|150|Monday|1902Q4|N|N|N|2416055|2416358|2415706|2415979|N|N|N|N|N| +2416072|AAAAAAAAIMNNECAA|1902-11-18|34|151|12|1902|2|11|18|4|1902|12|151|Tuesday|1902Q4|N|N|N|2416055|2416358|2415707|2415980|N|N|N|N|N| +2416073|AAAAAAAAJMNNECAA|1902-11-19|34|151|12|1902|3|11|19|4|1902|12|151|Wednesday|1902Q4|N|N|N|2416055|2416358|2415708|2415981|N|N|N|N|N| +2416074|AAAAAAAAKMNNECAA|1902-11-20|34|151|12|1902|4|11|20|4|1902|12|151|Thursday|1902Q4|N|N|N|2416055|2416358|2415709|2415982|N|N|N|N|N| +2416075|AAAAAAAALMNNECAA|1902-11-21|34|151|12|1902|5|11|21|4|1902|12|151|Friday|1902Q4|N|Y|N|2416055|2416358|2415710|2415983|N|N|N|N|N| +2416076|AAAAAAAAMMNNECAA|1902-11-22|34|151|12|1902|6|11|22|4|1902|12|151|Saturday|1902Q4|N|Y|N|2416055|2416358|2415711|2415984|N|N|N|N|N| +2416077|AAAAAAAANMNNECAA|1902-11-23|34|151|12|1902|0|11|23|4|1902|12|151|Sunday|1902Q4|N|N|N|2416055|2416358|2415712|2415985|N|N|N|N|N| +2416078|AAAAAAAAOMNNECAA|1902-11-24|34|151|12|1902|1|11|24|4|1902|12|151|Monday|1902Q4|N|N|N|2416055|2416358|2415713|2415986|N|N|N|N|N| +2416079|AAAAAAAAPMNNECAA|1902-11-25|34|152|12|1902|2|11|25|4|1902|12|152|Tuesday|1902Q4|N|N|N|2416055|2416358|2415714|2415987|N|N|N|N|N| +2416080|AAAAAAAAANNNECAA|1902-11-26|34|152|12|1902|3|11|26|4|1902|12|152|Wednesday|1902Q4|N|N|N|2416055|2416358|2415715|2415988|N|N|N|N|N| +2416081|AAAAAAAABNNNECAA|1902-11-27|34|152|12|1902|4|11|27|4|1902|12|152|Thursday|1902Q4|N|N|N|2416055|2416358|2415716|2415989|N|N|N|N|N| +2416082|AAAAAAAACNNNECAA|1902-11-28|34|152|12|1902|5|11|28|4|1902|12|152|Friday|1902Q4|N|Y|N|2416055|2416358|2415717|2415990|N|N|N|N|N| +2416083|AAAAAAAADNNNECAA|1902-11-29|34|152|12|1902|6|11|29|4|1902|12|152|Saturday|1902Q4|N|Y|N|2416055|2416358|2415718|2415991|N|N|N|N|N| +2416084|AAAAAAAAENNNECAA|1902-11-30|34|152|12|1902|0|11|30|4|1902|12|152|Sunday|1902Q4|N|N|N|2416055|2416358|2415719|2415992|N|N|N|N|N| +2416085|AAAAAAAAFNNNECAA|1902-12-01|35|152|13|1902|1|12|1|4|1902|13|152|Monday|1902Q4|N|N|N|2416085|2416418|2415720|2415993|N|N|N|N|N| +2416086|AAAAAAAAGNNNECAA|1902-12-02|35|153|13|1902|2|12|2|4|1902|13|153|Tuesday|1902Q4|N|N|N|2416085|2416418|2415721|2415994|N|N|N|N|N| +2416087|AAAAAAAAHNNNECAA|1902-12-03|35|153|13|1902|3|12|3|4|1902|13|153|Wednesday|1902Q4|N|N|N|2416085|2416418|2415722|2415995|N|N|N|N|N| +2416088|AAAAAAAAINNNECAA|1902-12-04|35|153|13|1902|4|12|4|4|1902|13|153|Thursday|1902Q4|N|N|N|2416085|2416418|2415723|2415996|N|N|N|N|N| +2416089|AAAAAAAAJNNNECAA|1902-12-05|35|153|13|1902|5|12|5|4|1902|13|153|Friday|1902Q4|N|Y|N|2416085|2416418|2415724|2415997|N|N|N|N|N| +2416090|AAAAAAAAKNNNECAA|1902-12-06|35|153|13|1902|6|12|6|4|1902|13|153|Saturday|1902Q4|N|Y|N|2416085|2416418|2415725|2415998|N|N|N|N|N| +2416091|AAAAAAAALNNNECAA|1902-12-07|35|153|13|1902|0|12|7|4|1902|13|153|Sunday|1902Q4|N|N|N|2416085|2416418|2415726|2415999|N|N|N|N|N| +2416092|AAAAAAAAMNNNECAA|1902-12-08|35|153|13|1902|1|12|8|4|1902|13|153|Monday|1902Q4|N|N|N|2416085|2416418|2415727|2416000|N|N|N|N|N| +2416093|AAAAAAAANNNNECAA|1902-12-09|35|154|13|1902|2|12|9|4|1902|13|154|Tuesday|1902Q4|N|N|N|2416085|2416418|2415728|2416001|N|N|N|N|N| +2416094|AAAAAAAAONNNECAA|1902-12-10|35|154|13|1902|3|12|10|4|1902|13|154|Wednesday|1902Q4|N|N|N|2416085|2416418|2415729|2416002|N|N|N|N|N| +2416095|AAAAAAAAPNNNECAA|1902-12-11|35|154|13|1902|4|12|11|4|1902|13|154|Thursday|1902Q4|N|N|N|2416085|2416418|2415730|2416003|N|N|N|N|N| +2416096|AAAAAAAAAONNECAA|1902-12-12|35|154|13|1902|5|12|12|4|1902|13|154|Friday|1902Q4|N|Y|N|2416085|2416418|2415731|2416004|N|N|N|N|N| +2416097|AAAAAAAABONNECAA|1902-12-13|35|154|13|1902|6|12|13|4|1902|13|154|Saturday|1902Q4|N|Y|N|2416085|2416418|2415732|2416005|N|N|N|N|N| +2416098|AAAAAAAACONNECAA|1902-12-14|35|154|13|1902|0|12|14|4|1902|13|154|Sunday|1902Q4|N|N|N|2416085|2416418|2415733|2416006|N|N|N|N|N| +2416099|AAAAAAAADONNECAA|1902-12-15|35|154|13|1902|1|12|15|4|1902|13|154|Monday|1902Q4|N|N|N|2416085|2416418|2415734|2416007|N|N|N|N|N| +2416100|AAAAAAAAEONNECAA|1902-12-16|35|155|13|1902|2|12|16|4|1902|13|155|Tuesday|1902Q4|N|N|N|2416085|2416418|2415735|2416008|N|N|N|N|N| +2416101|AAAAAAAAFONNECAA|1902-12-17|35|155|13|1902|3|12|17|4|1902|13|155|Wednesday|1902Q4|N|N|N|2416085|2416418|2415736|2416009|N|N|N|N|N| +2416102|AAAAAAAAGONNECAA|1902-12-18|35|155|13|1902|4|12|18|4|1902|13|155|Thursday|1902Q4|N|N|N|2416085|2416418|2415737|2416010|N|N|N|N|N| +2416103|AAAAAAAAHONNECAA|1902-12-19|35|155|13|1902|5|12|19|4|1902|13|155|Friday|1902Q4|N|Y|N|2416085|2416418|2415738|2416011|N|N|N|N|N| +2416104|AAAAAAAAIONNECAA|1902-12-20|35|155|13|1902|6|12|20|4|1902|13|155|Saturday|1902Q4|N|Y|N|2416085|2416418|2415739|2416012|N|N|N|N|N| +2416105|AAAAAAAAJONNECAA|1902-12-21|35|155|13|1902|0|12|21|4|1902|13|155|Sunday|1902Q4|N|N|N|2416085|2416418|2415740|2416013|N|N|N|N|N| +2416106|AAAAAAAAKONNECAA|1902-12-22|35|155|13|1902|1|12|22|4|1902|13|155|Monday|1902Q4|N|N|N|2416085|2416418|2415741|2416014|N|N|N|N|N| +2416107|AAAAAAAALONNECAA|1902-12-23|35|156|13|1902|2|12|23|4|1902|13|156|Tuesday|1902Q4|N|N|N|2416085|2416418|2415742|2416015|N|N|N|N|N| +2416108|AAAAAAAAMONNECAA|1902-12-24|35|156|13|1902|3|12|24|4|1902|13|156|Wednesday|1902Q4|N|N|N|2416085|2416418|2415743|2416016|N|N|N|N|N| +2416109|AAAAAAAANONNECAA|1902-12-25|35|156|13|1902|4|12|25|4|1902|13|156|Thursday|1902Q4|N|N|N|2416085|2416418|2415744|2416017|N|N|N|N|N| +2416110|AAAAAAAAOONNECAA|1902-12-26|35|156|13|1902|5|12|26|4|1902|13|156|Friday|1902Q4|Y|Y|N|2416085|2416418|2415745|2416018|N|N|N|N|N| +2416111|AAAAAAAAPONNECAA|1902-12-27|35|156|13|1902|6|12|27|4|1902|13|156|Saturday|1902Q4|N|Y|Y|2416085|2416418|2415746|2416019|N|N|N|N|N| +2416112|AAAAAAAAAPNNECAA|1902-12-28|35|156|13|1902|0|12|28|4|1902|13|156|Sunday|1902Q4|N|N|N|2416085|2416418|2415747|2416020|N|N|N|N|N| +2416113|AAAAAAAABPNNECAA|1902-12-29|35|156|13|1902|1|12|29|4|1902|13|156|Monday|1902Q4|N|N|N|2416085|2416418|2415748|2416021|N|N|N|N|N| +2416114|AAAAAAAACPNNECAA|1902-12-30|35|157|13|1902|2|12|30|4|1902|13|157|Tuesday|1902Q4|N|N|N|2416085|2416418|2415749|2416022|N|N|N|N|N| +2416115|AAAAAAAADPNNECAA|1902-12-31|35|157|13|1902|3|12|31|4|1902|13|157|Wednesday|1902Q4|N|N|N|2416085|2416418|2415750|2416023|N|N|N|N|N| +2416116|AAAAAAAAEPNNECAA|1903-01-01|36|157|13|1903|4|1|1|1|1903|13|157|Thursday|1903Q1|Y|N|N|2416116|2416115|2415751|2416024|N|N|N|N|N| +2416117|AAAAAAAAFPNNECAA|1903-01-02|36|157|13|1903|5|1|2|1|1903|13|157|Friday|1903Q1|N|Y|Y|2416116|2416115|2415752|2416025|N|N|N|N|N| +2416118|AAAAAAAAGPNNECAA|1903-01-03|36|157|13|1903|6|1|3|1|1903|13|157|Saturday|1903Q1|N|Y|N|2416116|2416115|2415753|2416026|N|N|N|N|N| +2416119|AAAAAAAAHPNNECAA|1903-01-04|36|157|13|1903|0|1|4|1|1903|13|157|Sunday|1903Q1|N|N|N|2416116|2416115|2415754|2416027|N|N|N|N|N| +2416120|AAAAAAAAIPNNECAA|1903-01-05|36|157|13|1903|1|1|5|1|1903|13|157|Monday|1903Q1|N|N|N|2416116|2416115|2415755|2416028|N|N|N|N|N| +2416121|AAAAAAAAJPNNECAA|1903-01-06|36|158|13|1903|2|1|6|1|1903|13|158|Tuesday|1903Q1|N|N|N|2416116|2416115|2415756|2416029|N|N|N|N|N| +2416122|AAAAAAAAKPNNECAA|1903-01-07|36|158|13|1903|3|1|7|1|1903|13|158|Wednesday|1903Q1|N|N|N|2416116|2416115|2415757|2416030|N|N|N|N|N| +2416123|AAAAAAAALPNNECAA|1903-01-08|36|158|13|1903|4|1|8|1|1903|13|158|Thursday|1903Q1|N|N|N|2416116|2416115|2415758|2416031|N|N|N|N|N| +2416124|AAAAAAAAMPNNECAA|1903-01-09|36|158|13|1903|5|1|9|1|1903|13|158|Friday|1903Q1|N|Y|N|2416116|2416115|2415759|2416032|N|N|N|N|N| +2416125|AAAAAAAANPNNECAA|1903-01-10|36|158|13|1903|6|1|10|1|1903|13|158|Saturday|1903Q1|N|Y|N|2416116|2416115|2415760|2416033|N|N|N|N|N| +2416126|AAAAAAAAOPNNECAA|1903-01-11|36|158|13|1903|0|1|11|1|1903|13|158|Sunday|1903Q1|N|N|N|2416116|2416115|2415761|2416034|N|N|N|N|N| +2416127|AAAAAAAAPPNNECAA|1903-01-12|36|158|13|1903|1|1|12|1|1903|13|158|Monday|1903Q1|N|N|N|2416116|2416115|2415762|2416035|N|N|N|N|N| +2416128|AAAAAAAAAAONECAA|1903-01-13|36|159|13|1903|2|1|13|1|1903|13|159|Tuesday|1903Q1|N|N|N|2416116|2416115|2415763|2416036|N|N|N|N|N| +2416129|AAAAAAAABAONECAA|1903-01-14|36|159|13|1903|3|1|14|1|1903|13|159|Wednesday|1903Q1|N|N|N|2416116|2416115|2415764|2416037|N|N|N|N|N| +2416130|AAAAAAAACAONECAA|1903-01-15|36|159|13|1903|4|1|15|1|1903|13|159|Thursday|1903Q1|N|N|N|2416116|2416115|2415765|2416038|N|N|N|N|N| +2416131|AAAAAAAADAONECAA|1903-01-16|36|159|13|1903|5|1|16|1|1903|13|159|Friday|1903Q1|N|Y|N|2416116|2416115|2415766|2416039|N|N|N|N|N| +2416132|AAAAAAAAEAONECAA|1903-01-17|36|159|13|1903|6|1|17|1|1903|13|159|Saturday|1903Q1|N|Y|N|2416116|2416115|2415767|2416040|N|N|N|N|N| +2416133|AAAAAAAAFAONECAA|1903-01-18|36|159|13|1903|0|1|18|1|1903|13|159|Sunday|1903Q1|N|N|N|2416116|2416115|2415768|2416041|N|N|N|N|N| +2416134|AAAAAAAAGAONECAA|1903-01-19|36|159|13|1903|1|1|19|1|1903|13|159|Monday|1903Q1|N|N|N|2416116|2416115|2415769|2416042|N|N|N|N|N| +2416135|AAAAAAAAHAONECAA|1903-01-20|36|160|13|1903|2|1|20|1|1903|13|160|Tuesday|1903Q1|N|N|N|2416116|2416115|2415770|2416043|N|N|N|N|N| +2416136|AAAAAAAAIAONECAA|1903-01-21|36|160|13|1903|3|1|21|1|1903|13|160|Wednesday|1903Q1|N|N|N|2416116|2416115|2415771|2416044|N|N|N|N|N| +2416137|AAAAAAAAJAONECAA|1903-01-22|36|160|13|1903|4|1|22|1|1903|13|160|Thursday|1903Q1|N|N|N|2416116|2416115|2415772|2416045|N|N|N|N|N| +2416138|AAAAAAAAKAONECAA|1903-01-23|36|160|13|1903|5|1|23|1|1903|13|160|Friday|1903Q1|N|Y|N|2416116|2416115|2415773|2416046|N|N|N|N|N| +2416139|AAAAAAAALAONECAA|1903-01-24|36|160|13|1903|6|1|24|1|1903|13|160|Saturday|1903Q1|N|Y|N|2416116|2416115|2415774|2416047|N|N|N|N|N| +2416140|AAAAAAAAMAONECAA|1903-01-25|36|160|13|1903|0|1|25|1|1903|13|160|Sunday|1903Q1|N|N|N|2416116|2416115|2415775|2416048|N|N|N|N|N| +2416141|AAAAAAAANAONECAA|1903-01-26|36|160|13|1903|1|1|26|1|1903|13|160|Monday|1903Q1|N|N|N|2416116|2416115|2415776|2416049|N|N|N|N|N| +2416142|AAAAAAAAOAONECAA|1903-01-27|36|161|13|1903|2|1|27|1|1903|13|161|Tuesday|1903Q1|N|N|N|2416116|2416115|2415777|2416050|N|N|N|N|N| +2416143|AAAAAAAAPAONECAA|1903-01-28|36|161|13|1903|3|1|28|1|1903|13|161|Wednesday|1903Q1|N|N|N|2416116|2416115|2415778|2416051|N|N|N|N|N| +2416144|AAAAAAAAABONECAA|1903-01-29|36|161|13|1903|4|1|29|1|1903|13|161|Thursday|1903Q1|N|N|N|2416116|2416115|2415779|2416052|N|N|N|N|N| +2416145|AAAAAAAABBONECAA|1903-01-30|36|161|13|1903|5|1|30|1|1903|13|161|Friday|1903Q1|N|Y|N|2416116|2416115|2415780|2416053|N|N|N|N|N| +2416146|AAAAAAAACBONECAA|1903-01-31|36|161|13|1903|6|1|31|1|1903|13|161|Saturday|1903Q1|N|Y|N|2416116|2416115|2415781|2416054|N|N|N|N|N| +2416147|AAAAAAAADBONECAA|1903-02-01|37|161|13|1903|0|2|1|1|1903|13|161|Sunday|1903Q1|N|N|N|2416147|2416177|2415782|2416055|N|N|N|N|N| +2416148|AAAAAAAAEBONECAA|1903-02-02|37|161|13|1903|1|2|2|1|1903|13|161|Monday|1903Q1|N|N|N|2416147|2416177|2415783|2416056|N|N|N|N|N| +2416149|AAAAAAAAFBONECAA|1903-02-03|37|162|13|1903|2|2|3|1|1903|13|162|Tuesday|1903Q1|N|N|N|2416147|2416177|2415784|2416057|N|N|N|N|N| +2416150|AAAAAAAAGBONECAA|1903-02-04|37|162|13|1903|3|2|4|1|1903|13|162|Wednesday|1903Q1|N|N|N|2416147|2416177|2415785|2416058|N|N|N|N|N| +2416151|AAAAAAAAHBONECAA|1903-02-05|37|162|13|1903|4|2|5|1|1903|13|162|Thursday|1903Q1|N|N|N|2416147|2416177|2415786|2416059|N|N|N|N|N| +2416152|AAAAAAAAIBONECAA|1903-02-06|37|162|13|1903|5|2|6|1|1903|13|162|Friday|1903Q1|N|Y|N|2416147|2416177|2415787|2416060|N|N|N|N|N| +2416153|AAAAAAAAJBONECAA|1903-02-07|37|162|13|1903|6|2|7|1|1903|13|162|Saturday|1903Q1|N|Y|N|2416147|2416177|2415788|2416061|N|N|N|N|N| +2416154|AAAAAAAAKBONECAA|1903-02-08|37|162|13|1903|0|2|8|1|1903|13|162|Sunday|1903Q1|N|N|N|2416147|2416177|2415789|2416062|N|N|N|N|N| +2416155|AAAAAAAALBONECAA|1903-02-09|37|162|13|1903|1|2|9|1|1903|13|162|Monday|1903Q1|N|N|N|2416147|2416177|2415790|2416063|N|N|N|N|N| +2416156|AAAAAAAAMBONECAA|1903-02-10|37|163|13|1903|2|2|10|1|1903|13|163|Tuesday|1903Q1|N|N|N|2416147|2416177|2415791|2416064|N|N|N|N|N| +2416157|AAAAAAAANBONECAA|1903-02-11|37|163|13|1903|3|2|11|1|1903|13|163|Wednesday|1903Q1|N|N|N|2416147|2416177|2415792|2416065|N|N|N|N|N| +2416158|AAAAAAAAOBONECAA|1903-02-12|37|163|13|1903|4|2|12|1|1903|13|163|Thursday|1903Q1|N|N|N|2416147|2416177|2415793|2416066|N|N|N|N|N| +2416159|AAAAAAAAPBONECAA|1903-02-13|37|163|13|1903|5|2|13|1|1903|13|163|Friday|1903Q1|N|Y|N|2416147|2416177|2415794|2416067|N|N|N|N|N| +2416160|AAAAAAAAACONECAA|1903-02-14|37|163|13|1903|6|2|14|1|1903|13|163|Saturday|1903Q1|N|Y|N|2416147|2416177|2415795|2416068|N|N|N|N|N| +2416161|AAAAAAAABCONECAA|1903-02-15|37|163|13|1903|0|2|15|1|1903|13|163|Sunday|1903Q1|N|N|N|2416147|2416177|2415796|2416069|N|N|N|N|N| +2416162|AAAAAAAACCONECAA|1903-02-16|37|163|13|1903|1|2|16|1|1903|13|163|Monday|1903Q1|N|N|N|2416147|2416177|2415797|2416070|N|N|N|N|N| +2416163|AAAAAAAADCONECAA|1903-02-17|37|164|13|1903|2|2|17|1|1903|13|164|Tuesday|1903Q1|N|N|N|2416147|2416177|2415798|2416071|N|N|N|N|N| +2416164|AAAAAAAAECONECAA|1903-02-18|37|164|13|1903|3|2|18|1|1903|13|164|Wednesday|1903Q1|N|N|N|2416147|2416177|2415799|2416072|N|N|N|N|N| +2416165|AAAAAAAAFCONECAA|1903-02-19|37|164|13|1903|4|2|19|1|1903|13|164|Thursday|1903Q1|N|N|N|2416147|2416177|2415800|2416073|N|N|N|N|N| +2416166|AAAAAAAAGCONECAA|1903-02-20|37|164|13|1903|5|2|20|1|1903|13|164|Friday|1903Q1|N|Y|N|2416147|2416177|2415801|2416074|N|N|N|N|N| +2416167|AAAAAAAAHCONECAA|1903-02-21|37|164|13|1903|6|2|21|1|1903|13|164|Saturday|1903Q1|N|Y|N|2416147|2416177|2415802|2416075|N|N|N|N|N| +2416168|AAAAAAAAICONECAA|1903-02-22|37|164|13|1903|0|2|22|1|1903|13|164|Sunday|1903Q1|N|N|N|2416147|2416177|2415803|2416076|N|N|N|N|N| +2416169|AAAAAAAAJCONECAA|1903-02-23|37|164|13|1903|1|2|23|1|1903|13|164|Monday|1903Q1|N|N|N|2416147|2416177|2415804|2416077|N|N|N|N|N| +2416170|AAAAAAAAKCONECAA|1903-02-24|37|165|13|1903|2|2|24|1|1903|13|165|Tuesday|1903Q1|N|N|N|2416147|2416177|2415805|2416078|N|N|N|N|N| +2416171|AAAAAAAALCONECAA|1903-02-25|37|165|13|1903|3|2|25|1|1903|13|165|Wednesday|1903Q1|N|N|N|2416147|2416177|2415806|2416079|N|N|N|N|N| +2416172|AAAAAAAAMCONECAA|1903-02-26|37|165|13|1903|4|2|26|1|1903|13|165|Thursday|1903Q1|N|N|N|2416147|2416177|2415807|2416080|N|N|N|N|N| +2416173|AAAAAAAANCONECAA|1903-02-27|37|165|13|1903|5|2|27|1|1903|13|165|Friday|1903Q1|N|Y|N|2416147|2416177|2415808|2416081|N|N|N|N|N| +2416174|AAAAAAAAOCONECAA|1903-02-28|37|165|13|1903|6|2|28|1|1903|13|165|Saturday|1903Q1|N|Y|N|2416147|2416177|2415809|2416082|N|N|N|N|N| +2416175|AAAAAAAAPCONECAA|1903-03-01|38|165|14|1903|0|3|1|1|1903|14|165|Sunday|1903Q1|N|N|N|2416175|2416233|2415810|2416083|N|N|N|N|N| +2416176|AAAAAAAAADONECAA|1903-03-02|38|165|14|1903|1|3|2|1|1903|14|165|Monday|1903Q1|N|N|N|2416175|2416233|2415811|2416084|N|N|N|N|N| +2416177|AAAAAAAABDONECAA|1903-03-03|38|166|14|1903|2|3|3|1|1903|14|166|Tuesday|1903Q1|N|N|N|2416175|2416233|2415812|2416085|N|N|N|N|N| +2416178|AAAAAAAACDONECAA|1903-03-04|38|166|14|1903|3|3|4|1|1903|14|166|Wednesday|1903Q1|N|N|N|2416175|2416233|2415813|2416086|N|N|N|N|N| +2416179|AAAAAAAADDONECAA|1903-03-05|38|166|14|1903|4|3|5|1|1903|14|166|Thursday|1903Q1|N|N|N|2416175|2416233|2415814|2416087|N|N|N|N|N| +2416180|AAAAAAAAEDONECAA|1903-03-06|38|166|14|1903|5|3|6|1|1903|14|166|Friday|1903Q1|N|Y|N|2416175|2416233|2415815|2416088|N|N|N|N|N| +2416181|AAAAAAAAFDONECAA|1903-03-07|38|166|14|1903|6|3|7|1|1903|14|166|Saturday|1903Q1|N|Y|N|2416175|2416233|2415816|2416089|N|N|N|N|N| +2416182|AAAAAAAAGDONECAA|1903-03-08|38|166|14|1903|0|3|8|1|1903|14|166|Sunday|1903Q1|N|N|N|2416175|2416233|2415817|2416090|N|N|N|N|N| +2416183|AAAAAAAAHDONECAA|1903-03-09|38|166|14|1903|1|3|9|1|1903|14|166|Monday|1903Q1|N|N|N|2416175|2416233|2415818|2416091|N|N|N|N|N| +2416184|AAAAAAAAIDONECAA|1903-03-10|38|167|14|1903|2|3|10|1|1903|14|167|Tuesday|1903Q1|N|N|N|2416175|2416233|2415819|2416092|N|N|N|N|N| +2416185|AAAAAAAAJDONECAA|1903-03-11|38|167|14|1903|3|3|11|1|1903|14|167|Wednesday|1903Q1|N|N|N|2416175|2416233|2415820|2416093|N|N|N|N|N| +2416186|AAAAAAAAKDONECAA|1903-03-12|38|167|14|1903|4|3|12|1|1903|14|167|Thursday|1903Q1|N|N|N|2416175|2416233|2415821|2416094|N|N|N|N|N| +2416187|AAAAAAAALDONECAA|1903-03-13|38|167|14|1903|5|3|13|1|1903|14|167|Friday|1903Q1|N|Y|N|2416175|2416233|2415822|2416095|N|N|N|N|N| +2416188|AAAAAAAAMDONECAA|1903-03-14|38|167|14|1903|6|3|14|1|1903|14|167|Saturday|1903Q1|N|Y|N|2416175|2416233|2415823|2416096|N|N|N|N|N| +2416189|AAAAAAAANDONECAA|1903-03-15|38|167|14|1903|0|3|15|1|1903|14|167|Sunday|1903Q1|N|N|N|2416175|2416233|2415824|2416097|N|N|N|N|N| +2416190|AAAAAAAAODONECAA|1903-03-16|38|167|14|1903|1|3|16|1|1903|14|167|Monday|1903Q1|N|N|N|2416175|2416233|2415825|2416098|N|N|N|N|N| +2416191|AAAAAAAAPDONECAA|1903-03-17|38|168|14|1903|2|3|17|1|1903|14|168|Tuesday|1903Q1|N|N|N|2416175|2416233|2415826|2416099|N|N|N|N|N| +2416192|AAAAAAAAAEONECAA|1903-03-18|38|168|14|1903|3|3|18|1|1903|14|168|Wednesday|1903Q1|N|N|N|2416175|2416233|2415827|2416100|N|N|N|N|N| +2416193|AAAAAAAABEONECAA|1903-03-19|38|168|14|1903|4|3|19|1|1903|14|168|Thursday|1903Q1|N|N|N|2416175|2416233|2415828|2416101|N|N|N|N|N| +2416194|AAAAAAAACEONECAA|1903-03-20|38|168|14|1903|5|3|20|1|1903|14|168|Friday|1903Q1|N|Y|N|2416175|2416233|2415829|2416102|N|N|N|N|N| +2416195|AAAAAAAADEONECAA|1903-03-21|38|168|14|1903|6|3|21|1|1903|14|168|Saturday|1903Q1|N|Y|N|2416175|2416233|2415830|2416103|N|N|N|N|N| +2416196|AAAAAAAAEEONECAA|1903-03-22|38|168|14|1903|0|3|22|1|1903|14|168|Sunday|1903Q1|N|N|N|2416175|2416233|2415831|2416104|N|N|N|N|N| +2416197|AAAAAAAAFEONECAA|1903-03-23|38|168|14|1903|1|3|23|1|1903|14|168|Monday|1903Q1|N|N|N|2416175|2416233|2415832|2416105|N|N|N|N|N| +2416198|AAAAAAAAGEONECAA|1903-03-24|38|169|14|1903|2|3|24|1|1903|14|169|Tuesday|1903Q1|N|N|N|2416175|2416233|2415833|2416106|N|N|N|N|N| +2416199|AAAAAAAAHEONECAA|1903-03-25|38|169|14|1903|3|3|25|1|1903|14|169|Wednesday|1903Q1|N|N|N|2416175|2416233|2415834|2416107|N|N|N|N|N| +2416200|AAAAAAAAIEONECAA|1903-03-26|38|169|14|1903|4|3|26|1|1903|14|169|Thursday|1903Q1|N|N|N|2416175|2416233|2415835|2416108|N|N|N|N|N| +2416201|AAAAAAAAJEONECAA|1903-03-27|38|169|14|1903|5|3|27|1|1903|14|169|Friday|1903Q1|N|Y|N|2416175|2416233|2415836|2416109|N|N|N|N|N| +2416202|AAAAAAAAKEONECAA|1903-03-28|38|169|14|1903|6|3|28|1|1903|14|169|Saturday|1903Q1|N|Y|N|2416175|2416233|2415837|2416110|N|N|N|N|N| +2416203|AAAAAAAALEONECAA|1903-03-29|38|169|14|1903|0|3|29|1|1903|14|169|Sunday|1903Q1|N|N|N|2416175|2416233|2415838|2416111|N|N|N|N|N| +2416204|AAAAAAAAMEONECAA|1903-03-30|38|169|14|1903|1|3|30|1|1903|14|169|Monday|1903Q1|N|N|N|2416175|2416233|2415839|2416112|N|N|N|N|N| +2416205|AAAAAAAANEONECAA|1903-03-31|38|170|14|1903|2|3|31|1|1903|14|170|Tuesday|1903Q1|N|N|N|2416175|2416233|2415840|2416113|N|N|N|N|N| +2416206|AAAAAAAAOEONECAA|1903-04-01|39|170|14|1903|3|4|1|1|1903|14|170|Wednesday|1903Q1|N|N|N|2416206|2416295|2415841|2416116|N|N|N|N|N| +2416207|AAAAAAAAPEONECAA|1903-04-02|39|170|14|1903|4|4|2|2|1903|14|170|Thursday|1903Q2|N|N|N|2416206|2416295|2415842|2416117|N|N|N|N|N| +2416208|AAAAAAAAAFONECAA|1903-04-03|39|170|14|1903|5|4|3|2|1903|14|170|Friday|1903Q2|N|Y|N|2416206|2416295|2415843|2416118|N|N|N|N|N| +2416209|AAAAAAAABFONECAA|1903-04-04|39|170|14|1903|6|4|4|2|1903|14|170|Saturday|1903Q2|N|Y|N|2416206|2416295|2415844|2416119|N|N|N|N|N| +2416210|AAAAAAAACFONECAA|1903-04-05|39|170|14|1903|0|4|5|2|1903|14|170|Sunday|1903Q2|N|N|N|2416206|2416295|2415845|2416120|N|N|N|N|N| +2416211|AAAAAAAADFONECAA|1903-04-06|39|170|14|1903|1|4|6|2|1903|14|170|Monday|1903Q2|N|N|N|2416206|2416295|2415846|2416121|N|N|N|N|N| +2416212|AAAAAAAAEFONECAA|1903-04-07|39|171|14|1903|2|4|7|2|1903|14|171|Tuesday|1903Q2|N|N|N|2416206|2416295|2415847|2416122|N|N|N|N|N| +2416213|AAAAAAAAFFONECAA|1903-04-08|39|171|14|1903|3|4|8|2|1903|14|171|Wednesday|1903Q2|N|N|N|2416206|2416295|2415848|2416123|N|N|N|N|N| +2416214|AAAAAAAAGFONECAA|1903-04-09|39|171|14|1903|4|4|9|2|1903|14|171|Thursday|1903Q2|N|N|N|2416206|2416295|2415849|2416124|N|N|N|N|N| +2416215|AAAAAAAAHFONECAA|1903-04-10|39|171|14|1903|5|4|10|2|1903|14|171|Friday|1903Q2|N|Y|N|2416206|2416295|2415850|2416125|N|N|N|N|N| +2416216|AAAAAAAAIFONECAA|1903-04-11|39|171|14|1903|6|4|11|2|1903|14|171|Saturday|1903Q2|N|Y|N|2416206|2416295|2415851|2416126|N|N|N|N|N| +2416217|AAAAAAAAJFONECAA|1903-04-12|39|171|14|1903|0|4|12|2|1903|14|171|Sunday|1903Q2|N|N|N|2416206|2416295|2415852|2416127|N|N|N|N|N| +2416218|AAAAAAAAKFONECAA|1903-04-13|39|171|14|1903|1|4|13|2|1903|14|171|Monday|1903Q2|N|N|N|2416206|2416295|2415853|2416128|N|N|N|N|N| +2416219|AAAAAAAALFONECAA|1903-04-14|39|172|14|1903|2|4|14|2|1903|14|172|Tuesday|1903Q2|N|N|N|2416206|2416295|2415854|2416129|N|N|N|N|N| +2416220|AAAAAAAAMFONECAA|1903-04-15|39|172|14|1903|3|4|15|2|1903|14|172|Wednesday|1903Q2|N|N|N|2416206|2416295|2415855|2416130|N|N|N|N|N| +2416221|AAAAAAAANFONECAA|1903-04-16|39|172|14|1903|4|4|16|2|1903|14|172|Thursday|1903Q2|N|N|N|2416206|2416295|2415856|2416131|N|N|N|N|N| +2416222|AAAAAAAAOFONECAA|1903-04-17|39|172|14|1903|5|4|17|2|1903|14|172|Friday|1903Q2|N|Y|N|2416206|2416295|2415857|2416132|N|N|N|N|N| +2416223|AAAAAAAAPFONECAA|1903-04-18|39|172|14|1903|6|4|18|2|1903|14|172|Saturday|1903Q2|N|Y|N|2416206|2416295|2415858|2416133|N|N|N|N|N| +2416224|AAAAAAAAAGONECAA|1903-04-19|39|172|14|1903|0|4|19|2|1903|14|172|Sunday|1903Q2|N|N|N|2416206|2416295|2415859|2416134|N|N|N|N|N| +2416225|AAAAAAAABGONECAA|1903-04-20|39|172|14|1903|1|4|20|2|1903|14|172|Monday|1903Q2|N|N|N|2416206|2416295|2415860|2416135|N|N|N|N|N| +2416226|AAAAAAAACGONECAA|1903-04-21|39|173|14|1903|2|4|21|2|1903|14|173|Tuesday|1903Q2|N|N|N|2416206|2416295|2415861|2416136|N|N|N|N|N| +2416227|AAAAAAAADGONECAA|1903-04-22|39|173|14|1903|3|4|22|2|1903|14|173|Wednesday|1903Q2|N|N|N|2416206|2416295|2415862|2416137|N|N|N|N|N| +2416228|AAAAAAAAEGONECAA|1903-04-23|39|173|14|1903|4|4|23|2|1903|14|173|Thursday|1903Q2|N|N|N|2416206|2416295|2415863|2416138|N|N|N|N|N| +2416229|AAAAAAAAFGONECAA|1903-04-24|39|173|14|1903|5|4|24|2|1903|14|173|Friday|1903Q2|N|Y|N|2416206|2416295|2415864|2416139|N|N|N|N|N| +2416230|AAAAAAAAGGONECAA|1903-04-25|39|173|14|1903|6|4|25|2|1903|14|173|Saturday|1903Q2|N|Y|N|2416206|2416295|2415865|2416140|N|N|N|N|N| +2416231|AAAAAAAAHGONECAA|1903-04-26|39|173|14|1903|0|4|26|2|1903|14|173|Sunday|1903Q2|N|N|N|2416206|2416295|2415866|2416141|N|N|N|N|N| +2416232|AAAAAAAAIGONECAA|1903-04-27|39|173|14|1903|1|4|27|2|1903|14|173|Monday|1903Q2|N|N|N|2416206|2416295|2415867|2416142|N|N|N|N|N| +2416233|AAAAAAAAJGONECAA|1903-04-28|39|174|14|1903|2|4|28|2|1903|14|174|Tuesday|1903Q2|N|N|N|2416206|2416295|2415868|2416143|N|N|N|N|N| +2416234|AAAAAAAAKGONECAA|1903-04-29|39|174|14|1903|3|4|29|2|1903|14|174|Wednesday|1903Q2|N|N|N|2416206|2416295|2415869|2416144|N|N|N|N|N| +2416235|AAAAAAAALGONECAA|1903-04-30|39|174|14|1903|4|4|30|2|1903|14|174|Thursday|1903Q2|N|N|N|2416206|2416295|2415870|2416145|N|N|N|N|N| +2416236|AAAAAAAAMGONECAA|1903-05-01|40|174|14|1903|5|5|1|2|1903|14|174|Friday|1903Q2|N|Y|N|2416236|2416355|2415871|2416146|N|N|N|N|N| +2416237|AAAAAAAANGONECAA|1903-05-02|40|174|14|1903|6|5|2|2|1903|14|174|Saturday|1903Q2|N|Y|N|2416236|2416355|2415872|2416147|N|N|N|N|N| +2416238|AAAAAAAAOGONECAA|1903-05-03|40|174|14|1903|0|5|3|2|1903|14|174|Sunday|1903Q2|N|N|N|2416236|2416355|2415873|2416148|N|N|N|N|N| +2416239|AAAAAAAAPGONECAA|1903-05-04|40|174|14|1903|1|5|4|2|1903|14|174|Monday|1903Q2|N|N|N|2416236|2416355|2415874|2416149|N|N|N|N|N| +2416240|AAAAAAAAAHONECAA|1903-05-05|40|175|14|1903|2|5|5|2|1903|14|175|Tuesday|1903Q2|N|N|N|2416236|2416355|2415875|2416150|N|N|N|N|N| +2416241|AAAAAAAABHONECAA|1903-05-06|40|175|14|1903|3|5|6|2|1903|14|175|Wednesday|1903Q2|N|N|N|2416236|2416355|2415876|2416151|N|N|N|N|N| +2416242|AAAAAAAACHONECAA|1903-05-07|40|175|14|1903|4|5|7|2|1903|14|175|Thursday|1903Q2|N|N|N|2416236|2416355|2415877|2416152|N|N|N|N|N| +2416243|AAAAAAAADHONECAA|1903-05-08|40|175|14|1903|5|5|8|2|1903|14|175|Friday|1903Q2|N|Y|N|2416236|2416355|2415878|2416153|N|N|N|N|N| +2416244|AAAAAAAAEHONECAA|1903-05-09|40|175|14|1903|6|5|9|2|1903|14|175|Saturday|1903Q2|N|Y|N|2416236|2416355|2415879|2416154|N|N|N|N|N| +2416245|AAAAAAAAFHONECAA|1903-05-10|40|175|14|1903|0|5|10|2|1903|14|175|Sunday|1903Q2|N|N|N|2416236|2416355|2415880|2416155|N|N|N|N|N| +2416246|AAAAAAAAGHONECAA|1903-05-11|40|175|14|1903|1|5|11|2|1903|14|175|Monday|1903Q2|N|N|N|2416236|2416355|2415881|2416156|N|N|N|N|N| +2416247|AAAAAAAAHHONECAA|1903-05-12|40|176|14|1903|2|5|12|2|1903|14|176|Tuesday|1903Q2|N|N|N|2416236|2416355|2415882|2416157|N|N|N|N|N| +2416248|AAAAAAAAIHONECAA|1903-05-13|40|176|14|1903|3|5|13|2|1903|14|176|Wednesday|1903Q2|N|N|N|2416236|2416355|2415883|2416158|N|N|N|N|N| +2416249|AAAAAAAAJHONECAA|1903-05-14|40|176|14|1903|4|5|14|2|1903|14|176|Thursday|1903Q2|N|N|N|2416236|2416355|2415884|2416159|N|N|N|N|N| +2416250|AAAAAAAAKHONECAA|1903-05-15|40|176|14|1903|5|5|15|2|1903|14|176|Friday|1903Q2|N|Y|N|2416236|2416355|2415885|2416160|N|N|N|N|N| +2416251|AAAAAAAALHONECAA|1903-05-16|40|176|14|1903|6|5|16|2|1903|14|176|Saturday|1903Q2|N|Y|N|2416236|2416355|2415886|2416161|N|N|N|N|N| +2416252|AAAAAAAAMHONECAA|1903-05-17|40|176|14|1903|0|5|17|2|1903|14|176|Sunday|1903Q2|N|N|N|2416236|2416355|2415887|2416162|N|N|N|N|N| +2416253|AAAAAAAANHONECAA|1903-05-18|40|176|14|1903|1|5|18|2|1903|14|176|Monday|1903Q2|N|N|N|2416236|2416355|2415888|2416163|N|N|N|N|N| +2416254|AAAAAAAAOHONECAA|1903-05-19|40|177|14|1903|2|5|19|2|1903|14|177|Tuesday|1903Q2|N|N|N|2416236|2416355|2415889|2416164|N|N|N|N|N| +2416255|AAAAAAAAPHONECAA|1903-05-20|40|177|14|1903|3|5|20|2|1903|14|177|Wednesday|1903Q2|N|N|N|2416236|2416355|2415890|2416165|N|N|N|N|N| +2416256|AAAAAAAAAIONECAA|1903-05-21|40|177|14|1903|4|5|21|2|1903|14|177|Thursday|1903Q2|N|N|N|2416236|2416355|2415891|2416166|N|N|N|N|N| +2416257|AAAAAAAABIONECAA|1903-05-22|40|177|14|1903|5|5|22|2|1903|14|177|Friday|1903Q2|N|Y|N|2416236|2416355|2415892|2416167|N|N|N|N|N| +2416258|AAAAAAAACIONECAA|1903-05-23|40|177|14|1903|6|5|23|2|1903|14|177|Saturday|1903Q2|N|Y|N|2416236|2416355|2415893|2416168|N|N|N|N|N| +2416259|AAAAAAAADIONECAA|1903-05-24|40|177|14|1903|0|5|24|2|1903|14|177|Sunday|1903Q2|N|N|N|2416236|2416355|2415894|2416169|N|N|N|N|N| +2416260|AAAAAAAAEIONECAA|1903-05-25|40|177|14|1903|1|5|25|2|1903|14|177|Monday|1903Q2|N|N|N|2416236|2416355|2415895|2416170|N|N|N|N|N| +2416261|AAAAAAAAFIONECAA|1903-05-26|40|178|14|1903|2|5|26|2|1903|14|178|Tuesday|1903Q2|N|N|N|2416236|2416355|2415896|2416171|N|N|N|N|N| +2416262|AAAAAAAAGIONECAA|1903-05-27|40|178|14|1903|3|5|27|2|1903|14|178|Wednesday|1903Q2|N|N|N|2416236|2416355|2415897|2416172|N|N|N|N|N| +2416263|AAAAAAAAHIONECAA|1903-05-28|40|178|14|1903|4|5|28|2|1903|14|178|Thursday|1903Q2|N|N|N|2416236|2416355|2415898|2416173|N|N|N|N|N| +2416264|AAAAAAAAIIONECAA|1903-05-29|40|178|14|1903|5|5|29|2|1903|14|178|Friday|1903Q2|N|Y|N|2416236|2416355|2415899|2416174|N|N|N|N|N| +2416265|AAAAAAAAJIONECAA|1903-05-30|40|178|14|1903|6|5|30|2|1903|14|178|Saturday|1903Q2|N|Y|N|2416236|2416355|2415900|2416175|N|N|N|N|N| +2416266|AAAAAAAAKIONECAA|1903-05-31|40|178|14|1903|0|5|31|2|1903|14|178|Sunday|1903Q2|N|N|N|2416236|2416355|2415901|2416176|N|N|N|N|N| +2416267|AAAAAAAALIONECAA|1903-06-01|41|178|15|1903|1|6|1|2|1903|15|178|Monday|1903Q2|N|N|N|2416267|2416417|2415902|2416177|N|N|N|N|N| +2416268|AAAAAAAAMIONECAA|1903-06-02|41|179|15|1903|2|6|2|2|1903|15|179|Tuesday|1903Q2|N|N|N|2416267|2416417|2415903|2416178|N|N|N|N|N| +2416269|AAAAAAAANIONECAA|1903-06-03|41|179|15|1903|3|6|3|2|1903|15|179|Wednesday|1903Q2|N|N|N|2416267|2416417|2415904|2416179|N|N|N|N|N| +2416270|AAAAAAAAOIONECAA|1903-06-04|41|179|15|1903|4|6|4|2|1903|15|179|Thursday|1903Q2|N|N|N|2416267|2416417|2415905|2416180|N|N|N|N|N| +2416271|AAAAAAAAPIONECAA|1903-06-05|41|179|15|1903|5|6|5|2|1903|15|179|Friday|1903Q2|N|Y|N|2416267|2416417|2415906|2416181|N|N|N|N|N| +2416272|AAAAAAAAAJONECAA|1903-06-06|41|179|15|1903|6|6|6|2|1903|15|179|Saturday|1903Q2|N|Y|N|2416267|2416417|2415907|2416182|N|N|N|N|N| +2416273|AAAAAAAABJONECAA|1903-06-07|41|179|15|1903|0|6|7|2|1903|15|179|Sunday|1903Q2|N|N|N|2416267|2416417|2415908|2416183|N|N|N|N|N| +2416274|AAAAAAAACJONECAA|1903-06-08|41|179|15|1903|1|6|8|2|1903|15|179|Monday|1903Q2|N|N|N|2416267|2416417|2415909|2416184|N|N|N|N|N| +2416275|AAAAAAAADJONECAA|1903-06-09|41|180|15|1903|2|6|9|2|1903|15|180|Tuesday|1903Q2|N|N|N|2416267|2416417|2415910|2416185|N|N|N|N|N| +2416276|AAAAAAAAEJONECAA|1903-06-10|41|180|15|1903|3|6|10|2|1903|15|180|Wednesday|1903Q2|N|N|N|2416267|2416417|2415911|2416186|N|N|N|N|N| +2416277|AAAAAAAAFJONECAA|1903-06-11|41|180|15|1903|4|6|11|2|1903|15|180|Thursday|1903Q2|N|N|N|2416267|2416417|2415912|2416187|N|N|N|N|N| +2416278|AAAAAAAAGJONECAA|1903-06-12|41|180|15|1903|5|6|12|2|1903|15|180|Friday|1903Q2|N|Y|N|2416267|2416417|2415913|2416188|N|N|N|N|N| +2416279|AAAAAAAAHJONECAA|1903-06-13|41|180|15|1903|6|6|13|2|1903|15|180|Saturday|1903Q2|N|Y|N|2416267|2416417|2415914|2416189|N|N|N|N|N| +2416280|AAAAAAAAIJONECAA|1903-06-14|41|180|15|1903|0|6|14|2|1903|15|180|Sunday|1903Q2|N|N|N|2416267|2416417|2415915|2416190|N|N|N|N|N| +2416281|AAAAAAAAJJONECAA|1903-06-15|41|180|15|1903|1|6|15|2|1903|15|180|Monday|1903Q2|N|N|N|2416267|2416417|2415916|2416191|N|N|N|N|N| +2416282|AAAAAAAAKJONECAA|1903-06-16|41|181|15|1903|2|6|16|2|1903|15|181|Tuesday|1903Q2|N|N|N|2416267|2416417|2415917|2416192|N|N|N|N|N| +2416283|AAAAAAAALJONECAA|1903-06-17|41|181|15|1903|3|6|17|2|1903|15|181|Wednesday|1903Q2|N|N|N|2416267|2416417|2415918|2416193|N|N|N|N|N| +2416284|AAAAAAAAMJONECAA|1903-06-18|41|181|15|1903|4|6|18|2|1903|15|181|Thursday|1903Q2|N|N|N|2416267|2416417|2415919|2416194|N|N|N|N|N| +2416285|AAAAAAAANJONECAA|1903-06-19|41|181|15|1903|5|6|19|2|1903|15|181|Friday|1903Q2|N|Y|N|2416267|2416417|2415920|2416195|N|N|N|N|N| +2416286|AAAAAAAAOJONECAA|1903-06-20|41|181|15|1903|6|6|20|2|1903|15|181|Saturday|1903Q2|N|Y|N|2416267|2416417|2415921|2416196|N|N|N|N|N| +2416287|AAAAAAAAPJONECAA|1903-06-21|41|181|15|1903|0|6|21|2|1903|15|181|Sunday|1903Q2|N|N|N|2416267|2416417|2415922|2416197|N|N|N|N|N| +2416288|AAAAAAAAAKONECAA|1903-06-22|41|181|15|1903|1|6|22|2|1903|15|181|Monday|1903Q2|N|N|N|2416267|2416417|2415923|2416198|N|N|N|N|N| +2416289|AAAAAAAABKONECAA|1903-06-23|41|182|15|1903|2|6|23|2|1903|15|182|Tuesday|1903Q2|N|N|N|2416267|2416417|2415924|2416199|N|N|N|N|N| +2416290|AAAAAAAACKONECAA|1903-06-24|41|182|15|1903|3|6|24|2|1903|15|182|Wednesday|1903Q2|N|N|N|2416267|2416417|2415925|2416200|N|N|N|N|N| +2416291|AAAAAAAADKONECAA|1903-06-25|41|182|15|1903|4|6|25|2|1903|15|182|Thursday|1903Q2|N|N|N|2416267|2416417|2415926|2416201|N|N|N|N|N| +2416292|AAAAAAAAEKONECAA|1903-06-26|41|182|15|1903|5|6|26|2|1903|15|182|Friday|1903Q2|N|Y|N|2416267|2416417|2415927|2416202|N|N|N|N|N| +2416293|AAAAAAAAFKONECAA|1903-06-27|41|182|15|1903|6|6|27|2|1903|15|182|Saturday|1903Q2|N|Y|N|2416267|2416417|2415928|2416203|N|N|N|N|N| +2416294|AAAAAAAAGKONECAA|1903-06-28|41|182|15|1903|0|6|28|2|1903|15|182|Sunday|1903Q2|N|N|N|2416267|2416417|2415929|2416204|N|N|N|N|N| +2416295|AAAAAAAAHKONECAA|1903-06-29|41|182|15|1903|1|6|29|2|1903|15|182|Monday|1903Q2|N|N|N|2416267|2416417|2415930|2416205|N|N|N|N|N| +2416296|AAAAAAAAIKONECAA|1903-06-30|41|183|15|1903|2|6|30|2|1903|15|183|Tuesday|1903Q2|N|N|N|2416267|2416417|2415931|2416206|N|N|N|N|N| +2416297|AAAAAAAAJKONECAA|1903-07-01|42|183|15|1903|3|7|1|2|1903|15|183|Wednesday|1903Q2|N|N|N|2416297|2416477|2415932|2416206|N|N|N|N|N| +2416298|AAAAAAAAKKONECAA|1903-07-02|42|183|15|1903|4|7|2|3|1903|15|183|Thursday|1903Q3|N|N|N|2416297|2416477|2415933|2416207|N|N|N|N|N| +2416299|AAAAAAAALKONECAA|1903-07-03|42|183|15|1903|5|7|3|3|1903|15|183|Friday|1903Q3|N|Y|N|2416297|2416477|2415934|2416208|N|N|N|N|N| +2416300|AAAAAAAAMKONECAA|1903-07-04|42|183|15|1903|6|7|4|3|1903|15|183|Saturday|1903Q3|N|Y|N|2416297|2416477|2415935|2416209|N|N|N|N|N| +2416301|AAAAAAAANKONECAA|1903-07-05|42|183|15|1903|0|7|5|3|1903|15|183|Sunday|1903Q3|Y|N|N|2416297|2416477|2415936|2416210|N|N|N|N|N| +2416302|AAAAAAAAOKONECAA|1903-07-06|42|183|15|1903|1|7|6|3|1903|15|183|Monday|1903Q3|N|N|Y|2416297|2416477|2415937|2416211|N|N|N|N|N| +2416303|AAAAAAAAPKONECAA|1903-07-07|42|184|15|1903|2|7|7|3|1903|15|184|Tuesday|1903Q3|N|N|N|2416297|2416477|2415938|2416212|N|N|N|N|N| +2416304|AAAAAAAAALONECAA|1903-07-08|42|184|15|1903|3|7|8|3|1903|15|184|Wednesday|1903Q3|N|N|N|2416297|2416477|2415939|2416213|N|N|N|N|N| +2416305|AAAAAAAABLONECAA|1903-07-09|42|184|15|1903|4|7|9|3|1903|15|184|Thursday|1903Q3|N|N|N|2416297|2416477|2415940|2416214|N|N|N|N|N| +2416306|AAAAAAAACLONECAA|1903-07-10|42|184|15|1903|5|7|10|3|1903|15|184|Friday|1903Q3|N|Y|N|2416297|2416477|2415941|2416215|N|N|N|N|N| +2416307|AAAAAAAADLONECAA|1903-07-11|42|184|15|1903|6|7|11|3|1903|15|184|Saturday|1903Q3|N|Y|N|2416297|2416477|2415942|2416216|N|N|N|N|N| +2416308|AAAAAAAAELONECAA|1903-07-12|42|184|15|1903|0|7|12|3|1903|15|184|Sunday|1903Q3|N|N|N|2416297|2416477|2415943|2416217|N|N|N|N|N| +2416309|AAAAAAAAFLONECAA|1903-07-13|42|184|15|1903|1|7|13|3|1903|15|184|Monday|1903Q3|N|N|N|2416297|2416477|2415944|2416218|N|N|N|N|N| +2416310|AAAAAAAAGLONECAA|1903-07-14|42|185|15|1903|2|7|14|3|1903|15|185|Tuesday|1903Q3|N|N|N|2416297|2416477|2415945|2416219|N|N|N|N|N| +2416311|AAAAAAAAHLONECAA|1903-07-15|42|185|15|1903|3|7|15|3|1903|15|185|Wednesday|1903Q3|N|N|N|2416297|2416477|2415946|2416220|N|N|N|N|N| +2416312|AAAAAAAAILONECAA|1903-07-16|42|185|15|1903|4|7|16|3|1903|15|185|Thursday|1903Q3|N|N|N|2416297|2416477|2415947|2416221|N|N|N|N|N| +2416313|AAAAAAAAJLONECAA|1903-07-17|42|185|15|1903|5|7|17|3|1903|15|185|Friday|1903Q3|N|Y|N|2416297|2416477|2415948|2416222|N|N|N|N|N| +2416314|AAAAAAAAKLONECAA|1903-07-18|42|185|15|1903|6|7|18|3|1903|15|185|Saturday|1903Q3|N|Y|N|2416297|2416477|2415949|2416223|N|N|N|N|N| +2416315|AAAAAAAALLONECAA|1903-07-19|42|185|15|1903|0|7|19|3|1903|15|185|Sunday|1903Q3|N|N|N|2416297|2416477|2415950|2416224|N|N|N|N|N| +2416316|AAAAAAAAMLONECAA|1903-07-20|42|185|15|1903|1|7|20|3|1903|15|185|Monday|1903Q3|N|N|N|2416297|2416477|2415951|2416225|N|N|N|N|N| +2416317|AAAAAAAANLONECAA|1903-07-21|42|186|15|1903|2|7|21|3|1903|15|186|Tuesday|1903Q3|N|N|N|2416297|2416477|2415952|2416226|N|N|N|N|N| +2416318|AAAAAAAAOLONECAA|1903-07-22|42|186|15|1903|3|7|22|3|1903|15|186|Wednesday|1903Q3|N|N|N|2416297|2416477|2415953|2416227|N|N|N|N|N| +2416319|AAAAAAAAPLONECAA|1903-07-23|42|186|15|1903|4|7|23|3|1903|15|186|Thursday|1903Q3|N|N|N|2416297|2416477|2415954|2416228|N|N|N|N|N| +2416320|AAAAAAAAAMONECAA|1903-07-24|42|186|15|1903|5|7|24|3|1903|15|186|Friday|1903Q3|N|Y|N|2416297|2416477|2415955|2416229|N|N|N|N|N| +2416321|AAAAAAAABMONECAA|1903-07-25|42|186|15|1903|6|7|25|3|1903|15|186|Saturday|1903Q3|N|Y|N|2416297|2416477|2415956|2416230|N|N|N|N|N| +2416322|AAAAAAAACMONECAA|1903-07-26|42|186|15|1903|0|7|26|3|1903|15|186|Sunday|1903Q3|N|N|N|2416297|2416477|2415957|2416231|N|N|N|N|N| +2416323|AAAAAAAADMONECAA|1903-07-27|42|186|15|1903|1|7|27|3|1903|15|186|Monday|1903Q3|N|N|N|2416297|2416477|2415958|2416232|N|N|N|N|N| +2416324|AAAAAAAAEMONECAA|1903-07-28|42|187|15|1903|2|7|28|3|1903|15|187|Tuesday|1903Q3|N|N|N|2416297|2416477|2415959|2416233|N|N|N|N|N| +2416325|AAAAAAAAFMONECAA|1903-07-29|42|187|15|1903|3|7|29|3|1903|15|187|Wednesday|1903Q3|N|N|N|2416297|2416477|2415960|2416234|N|N|N|N|N| +2416326|AAAAAAAAGMONECAA|1903-07-30|42|187|15|1903|4|7|30|3|1903|15|187|Thursday|1903Q3|N|N|N|2416297|2416477|2415961|2416235|N|N|N|N|N| +2416327|AAAAAAAAHMONECAA|1903-07-31|42|187|15|1903|5|7|31|3|1903|15|187|Friday|1903Q3|N|Y|N|2416297|2416477|2415962|2416236|N|N|N|N|N| +2416328|AAAAAAAAIMONECAA|1903-08-01|43|187|15|1903|6|8|1|3|1903|15|187|Saturday|1903Q3|N|Y|N|2416328|2416539|2415963|2416237|N|N|N|N|N| +2416329|AAAAAAAAJMONECAA|1903-08-02|43|187|15|1903|0|8|2|3|1903|15|187|Sunday|1903Q3|N|N|N|2416328|2416539|2415964|2416238|N|N|N|N|N| +2416330|AAAAAAAAKMONECAA|1903-08-03|43|187|15|1903|1|8|3|3|1903|15|187|Monday|1903Q3|N|N|N|2416328|2416539|2415965|2416239|N|N|N|N|N| +2416331|AAAAAAAALMONECAA|1903-08-04|43|188|15|1903|2|8|4|3|1903|15|188|Tuesday|1903Q3|N|N|N|2416328|2416539|2415966|2416240|N|N|N|N|N| +2416332|AAAAAAAAMMONECAA|1903-08-05|43|188|15|1903|3|8|5|3|1903|15|188|Wednesday|1903Q3|N|N|N|2416328|2416539|2415967|2416241|N|N|N|N|N| +2416333|AAAAAAAANMONECAA|1903-08-06|43|188|15|1903|4|8|6|3|1903|15|188|Thursday|1903Q3|N|N|N|2416328|2416539|2415968|2416242|N|N|N|N|N| +2416334|AAAAAAAAOMONECAA|1903-08-07|43|188|15|1903|5|8|7|3|1903|15|188|Friday|1903Q3|N|Y|N|2416328|2416539|2415969|2416243|N|N|N|N|N| +2416335|AAAAAAAAPMONECAA|1903-08-08|43|188|15|1903|6|8|8|3|1903|15|188|Saturday|1903Q3|N|Y|N|2416328|2416539|2415970|2416244|N|N|N|N|N| +2416336|AAAAAAAAANONECAA|1903-08-09|43|188|15|1903|0|8|9|3|1903|15|188|Sunday|1903Q3|N|N|N|2416328|2416539|2415971|2416245|N|N|N|N|N| +2416337|AAAAAAAABNONECAA|1903-08-10|43|188|15|1903|1|8|10|3|1903|15|188|Monday|1903Q3|N|N|N|2416328|2416539|2415972|2416246|N|N|N|N|N| +2416338|AAAAAAAACNONECAA|1903-08-11|43|189|15|1903|2|8|11|3|1903|15|189|Tuesday|1903Q3|N|N|N|2416328|2416539|2415973|2416247|N|N|N|N|N| +2416339|AAAAAAAADNONECAA|1903-08-12|43|189|15|1903|3|8|12|3|1903|15|189|Wednesday|1903Q3|N|N|N|2416328|2416539|2415974|2416248|N|N|N|N|N| +2416340|AAAAAAAAENONECAA|1903-08-13|43|189|15|1903|4|8|13|3|1903|15|189|Thursday|1903Q3|N|N|N|2416328|2416539|2415975|2416249|N|N|N|N|N| +2416341|AAAAAAAAFNONECAA|1903-08-14|43|189|15|1903|5|8|14|3|1903|15|189|Friday|1903Q3|N|Y|N|2416328|2416539|2415976|2416250|N|N|N|N|N| +2416342|AAAAAAAAGNONECAA|1903-08-15|43|189|15|1903|6|8|15|3|1903|15|189|Saturday|1903Q3|N|Y|N|2416328|2416539|2415977|2416251|N|N|N|N|N| +2416343|AAAAAAAAHNONECAA|1903-08-16|43|189|15|1903|0|8|16|3|1903|15|189|Sunday|1903Q3|N|N|N|2416328|2416539|2415978|2416252|N|N|N|N|N| +2416344|AAAAAAAAINONECAA|1903-08-17|43|189|15|1903|1|8|17|3|1903|15|189|Monday|1903Q3|N|N|N|2416328|2416539|2415979|2416253|N|N|N|N|N| +2416345|AAAAAAAAJNONECAA|1903-08-18|43|190|15|1903|2|8|18|3|1903|15|190|Tuesday|1903Q3|N|N|N|2416328|2416539|2415980|2416254|N|N|N|N|N| +2416346|AAAAAAAAKNONECAA|1903-08-19|43|190|15|1903|3|8|19|3|1903|15|190|Wednesday|1903Q3|N|N|N|2416328|2416539|2415981|2416255|N|N|N|N|N| +2416347|AAAAAAAALNONECAA|1903-08-20|43|190|15|1903|4|8|20|3|1903|15|190|Thursday|1903Q3|N|N|N|2416328|2416539|2415982|2416256|N|N|N|N|N| +2416348|AAAAAAAAMNONECAA|1903-08-21|43|190|15|1903|5|8|21|3|1903|15|190|Friday|1903Q3|N|Y|N|2416328|2416539|2415983|2416257|N|N|N|N|N| +2416349|AAAAAAAANNONECAA|1903-08-22|43|190|15|1903|6|8|22|3|1903|15|190|Saturday|1903Q3|N|Y|N|2416328|2416539|2415984|2416258|N|N|N|N|N| +2416350|AAAAAAAAONONECAA|1903-08-23|43|190|15|1903|0|8|23|3|1903|15|190|Sunday|1903Q3|N|N|N|2416328|2416539|2415985|2416259|N|N|N|N|N| +2416351|AAAAAAAAPNONECAA|1903-08-24|43|190|15|1903|1|8|24|3|1903|15|190|Monday|1903Q3|N|N|N|2416328|2416539|2415986|2416260|N|N|N|N|N| +2416352|AAAAAAAAAOONECAA|1903-08-25|43|191|15|1903|2|8|25|3|1903|15|191|Tuesday|1903Q3|N|N|N|2416328|2416539|2415987|2416261|N|N|N|N|N| +2416353|AAAAAAAABOONECAA|1903-08-26|43|191|15|1903|3|8|26|3|1903|15|191|Wednesday|1903Q3|N|N|N|2416328|2416539|2415988|2416262|N|N|N|N|N| +2416354|AAAAAAAACOONECAA|1903-08-27|43|191|15|1903|4|8|27|3|1903|15|191|Thursday|1903Q3|N|N|N|2416328|2416539|2415989|2416263|N|N|N|N|N| +2416355|AAAAAAAADOONECAA|1903-08-28|43|191|15|1903|5|8|28|3|1903|15|191|Friday|1903Q3|N|Y|N|2416328|2416539|2415990|2416264|N|N|N|N|N| +2416356|AAAAAAAAEOONECAA|1903-08-29|43|191|15|1903|6|8|29|3|1903|15|191|Saturday|1903Q3|N|Y|N|2416328|2416539|2415991|2416265|N|N|N|N|N| +2416357|AAAAAAAAFOONECAA|1903-08-30|43|191|15|1903|0|8|30|3|1903|15|191|Sunday|1903Q3|N|N|N|2416328|2416539|2415992|2416266|N|N|N|N|N| +2416358|AAAAAAAAGOONECAA|1903-08-31|43|191|15|1903|1|8|31|3|1903|15|191|Monday|1903Q3|N|N|N|2416328|2416539|2415993|2416267|N|N|N|N|N| +2416359|AAAAAAAAHOONECAA|1903-09-01|44|192|16|1903|2|9|1|3|1903|16|192|Tuesday|1903Q3|N|N|N|2416359|2416601|2415994|2416268|N|N|N|N|N| +2416360|AAAAAAAAIOONECAA|1903-09-02|44|192|16|1903|3|9|2|3|1903|16|192|Wednesday|1903Q3|N|N|N|2416359|2416601|2415995|2416269|N|N|N|N|N| +2416361|AAAAAAAAJOONECAA|1903-09-03|44|192|16|1903|4|9|3|3|1903|16|192|Thursday|1903Q3|N|N|N|2416359|2416601|2415996|2416270|N|N|N|N|N| +2416362|AAAAAAAAKOONECAA|1903-09-04|44|192|16|1903|5|9|4|3|1903|16|192|Friday|1903Q3|N|Y|N|2416359|2416601|2415997|2416271|N|N|N|N|N| +2416363|AAAAAAAALOONECAA|1903-09-05|44|192|16|1903|6|9|5|3|1903|16|192|Saturday|1903Q3|N|Y|N|2416359|2416601|2415998|2416272|N|N|N|N|N| +2416364|AAAAAAAAMOONECAA|1903-09-06|44|192|16|1903|0|9|6|3|1903|16|192|Sunday|1903Q3|N|N|N|2416359|2416601|2415999|2416273|N|N|N|N|N| +2416365|AAAAAAAANOONECAA|1903-09-07|44|192|16|1903|1|9|7|3|1903|16|192|Monday|1903Q3|N|N|N|2416359|2416601|2416000|2416274|N|N|N|N|N| +2416366|AAAAAAAAOOONECAA|1903-09-08|44|193|16|1903|2|9|8|3|1903|16|193|Tuesday|1903Q3|N|N|N|2416359|2416601|2416001|2416275|N|N|N|N|N| +2416367|AAAAAAAAPOONECAA|1903-09-09|44|193|16|1903|3|9|9|3|1903|16|193|Wednesday|1903Q3|N|N|N|2416359|2416601|2416002|2416276|N|N|N|N|N| +2416368|AAAAAAAAAPONECAA|1903-09-10|44|193|16|1903|4|9|10|3|1903|16|193|Thursday|1903Q3|N|N|N|2416359|2416601|2416003|2416277|N|N|N|N|N| +2416369|AAAAAAAABPONECAA|1903-09-11|44|193|16|1903|5|9|11|3|1903|16|193|Friday|1903Q3|N|Y|N|2416359|2416601|2416004|2416278|N|N|N|N|N| +2416370|AAAAAAAACPONECAA|1903-09-12|44|193|16|1903|6|9|12|3|1903|16|193|Saturday|1903Q3|N|Y|N|2416359|2416601|2416005|2416279|N|N|N|N|N| +2416371|AAAAAAAADPONECAA|1903-09-13|44|193|16|1903|0|9|13|3|1903|16|193|Sunday|1903Q3|N|N|N|2416359|2416601|2416006|2416280|N|N|N|N|N| +2416372|AAAAAAAAEPONECAA|1903-09-14|44|193|16|1903|1|9|14|3|1903|16|193|Monday|1903Q3|N|N|N|2416359|2416601|2416007|2416281|N|N|N|N|N| +2416373|AAAAAAAAFPONECAA|1903-09-15|44|194|16|1903|2|9|15|3|1903|16|194|Tuesday|1903Q3|N|N|N|2416359|2416601|2416008|2416282|N|N|N|N|N| +2416374|AAAAAAAAGPONECAA|1903-09-16|44|194|16|1903|3|9|16|3|1903|16|194|Wednesday|1903Q3|N|N|N|2416359|2416601|2416009|2416283|N|N|N|N|N| +2416375|AAAAAAAAHPONECAA|1903-09-17|44|194|16|1903|4|9|17|3|1903|16|194|Thursday|1903Q3|N|N|N|2416359|2416601|2416010|2416284|N|N|N|N|N| +2416376|AAAAAAAAIPONECAA|1903-09-18|44|194|16|1903|5|9|18|3|1903|16|194|Friday|1903Q3|N|Y|N|2416359|2416601|2416011|2416285|N|N|N|N|N| +2416377|AAAAAAAAJPONECAA|1903-09-19|44|194|16|1903|6|9|19|3|1903|16|194|Saturday|1903Q3|N|Y|N|2416359|2416601|2416012|2416286|N|N|N|N|N| +2416378|AAAAAAAAKPONECAA|1903-09-20|44|194|16|1903|0|9|20|3|1903|16|194|Sunday|1903Q3|N|N|N|2416359|2416601|2416013|2416287|N|N|N|N|N| +2416379|AAAAAAAALPONECAA|1903-09-21|44|194|16|1903|1|9|21|3|1903|16|194|Monday|1903Q3|N|N|N|2416359|2416601|2416014|2416288|N|N|N|N|N| +2416380|AAAAAAAAMPONECAA|1903-09-22|44|195|16|1903|2|9|22|3|1903|16|195|Tuesday|1903Q3|N|N|N|2416359|2416601|2416015|2416289|N|N|N|N|N| +2416381|AAAAAAAANPONECAA|1903-09-23|44|195|16|1903|3|9|23|3|1903|16|195|Wednesday|1903Q3|N|N|N|2416359|2416601|2416016|2416290|N|N|N|N|N| +2416382|AAAAAAAAOPONECAA|1903-09-24|44|195|16|1903|4|9|24|3|1903|16|195|Thursday|1903Q3|N|N|N|2416359|2416601|2416017|2416291|N|N|N|N|N| +2416383|AAAAAAAAPPONECAA|1903-09-25|44|195|16|1903|5|9|25|3|1903|16|195|Friday|1903Q3|N|Y|N|2416359|2416601|2416018|2416292|N|N|N|N|N| +2416384|AAAAAAAAAAPNECAA|1903-09-26|44|195|16|1903|6|9|26|3|1903|16|195|Saturday|1903Q3|N|Y|N|2416359|2416601|2416019|2416293|N|N|N|N|N| +2416385|AAAAAAAABAPNECAA|1903-09-27|44|195|16|1903|0|9|27|3|1903|16|195|Sunday|1903Q3|N|N|N|2416359|2416601|2416020|2416294|N|N|N|N|N| +2416386|AAAAAAAACAPNECAA|1903-09-28|44|195|16|1903|1|9|28|3|1903|16|195|Monday|1903Q3|N|N|N|2416359|2416601|2416021|2416295|N|N|N|N|N| +2416387|AAAAAAAADAPNECAA|1903-09-29|44|196|16|1903|2|9|29|3|1903|16|196|Tuesday|1903Q3|N|N|N|2416359|2416601|2416022|2416296|N|N|N|N|N| +2416388|AAAAAAAAEAPNECAA|1903-09-30|44|196|16|1903|3|9|30|3|1903|16|196|Wednesday|1903Q3|N|N|N|2416359|2416601|2416023|2416297|N|N|N|N|N| +2416389|AAAAAAAAFAPNECAA|1903-10-01|45|196|16|1903|4|10|1|3|1903|16|196|Thursday|1903Q3|N|N|N|2416389|2416661|2416024|2416297|N|N|N|N|N| +2416390|AAAAAAAAGAPNECAA|1903-10-02|45|196|16|1903|5|10|2|4|1903|16|196|Friday|1903Q4|N|Y|N|2416389|2416661|2416025|2416298|N|N|N|N|N| +2416391|AAAAAAAAHAPNECAA|1903-10-03|45|196|16|1903|6|10|3|4|1903|16|196|Saturday|1903Q4|N|Y|N|2416389|2416661|2416026|2416299|N|N|N|N|N| +2416392|AAAAAAAAIAPNECAA|1903-10-04|45|196|16|1903|0|10|4|4|1903|16|196|Sunday|1903Q4|N|N|N|2416389|2416661|2416027|2416300|N|N|N|N|N| +2416393|AAAAAAAAJAPNECAA|1903-10-05|45|196|16|1903|1|10|5|4|1903|16|196|Monday|1903Q4|N|N|N|2416389|2416661|2416028|2416301|N|N|N|N|N| +2416394|AAAAAAAAKAPNECAA|1903-10-06|45|197|16|1903|2|10|6|4|1903|16|197|Tuesday|1903Q4|N|N|N|2416389|2416661|2416029|2416302|N|N|N|N|N| +2416395|AAAAAAAALAPNECAA|1903-10-07|45|197|16|1903|3|10|7|4|1903|16|197|Wednesday|1903Q4|N|N|N|2416389|2416661|2416030|2416303|N|N|N|N|N| +2416396|AAAAAAAAMAPNECAA|1903-10-08|45|197|16|1903|4|10|8|4|1903|16|197|Thursday|1903Q4|N|N|N|2416389|2416661|2416031|2416304|N|N|N|N|N| +2416397|AAAAAAAANAPNECAA|1903-10-09|45|197|16|1903|5|10|9|4|1903|16|197|Friday|1903Q4|N|Y|N|2416389|2416661|2416032|2416305|N|N|N|N|N| +2416398|AAAAAAAAOAPNECAA|1903-10-10|45|197|16|1903|6|10|10|4|1903|16|197|Saturday|1903Q4|N|Y|N|2416389|2416661|2416033|2416306|N|N|N|N|N| +2416399|AAAAAAAAPAPNECAA|1903-10-11|45|197|16|1903|0|10|11|4|1903|16|197|Sunday|1903Q4|N|N|N|2416389|2416661|2416034|2416307|N|N|N|N|N| +2416400|AAAAAAAAABPNECAA|1903-10-12|45|197|16|1903|1|10|12|4|1903|16|197|Monday|1903Q4|N|N|N|2416389|2416661|2416035|2416308|N|N|N|N|N| +2416401|AAAAAAAABBPNECAA|1903-10-13|45|198|16|1903|2|10|13|4|1903|16|198|Tuesday|1903Q4|N|N|N|2416389|2416661|2416036|2416309|N|N|N|N|N| +2416402|AAAAAAAACBPNECAA|1903-10-14|45|198|16|1903|3|10|14|4|1903|16|198|Wednesday|1903Q4|N|N|N|2416389|2416661|2416037|2416310|N|N|N|N|N| +2416403|AAAAAAAADBPNECAA|1903-10-15|45|198|16|1903|4|10|15|4|1903|16|198|Thursday|1903Q4|N|N|N|2416389|2416661|2416038|2416311|N|N|N|N|N| +2416404|AAAAAAAAEBPNECAA|1903-10-16|45|198|16|1903|5|10|16|4|1903|16|198|Friday|1903Q4|N|Y|N|2416389|2416661|2416039|2416312|N|N|N|N|N| +2416405|AAAAAAAAFBPNECAA|1903-10-17|45|198|16|1903|6|10|17|4|1903|16|198|Saturday|1903Q4|N|Y|N|2416389|2416661|2416040|2416313|N|N|N|N|N| +2416406|AAAAAAAAGBPNECAA|1903-10-18|45|198|16|1903|0|10|18|4|1903|16|198|Sunday|1903Q4|N|N|N|2416389|2416661|2416041|2416314|N|N|N|N|N| +2416407|AAAAAAAAHBPNECAA|1903-10-19|45|198|16|1903|1|10|19|4|1903|16|198|Monday|1903Q4|N|N|N|2416389|2416661|2416042|2416315|N|N|N|N|N| +2416408|AAAAAAAAIBPNECAA|1903-10-20|45|199|16|1903|2|10|20|4|1903|16|199|Tuesday|1903Q4|N|N|N|2416389|2416661|2416043|2416316|N|N|N|N|N| +2416409|AAAAAAAAJBPNECAA|1903-10-21|45|199|16|1903|3|10|21|4|1903|16|199|Wednesday|1903Q4|N|N|N|2416389|2416661|2416044|2416317|N|N|N|N|N| +2416410|AAAAAAAAKBPNECAA|1903-10-22|45|199|16|1903|4|10|22|4|1903|16|199|Thursday|1903Q4|N|N|N|2416389|2416661|2416045|2416318|N|N|N|N|N| +2416411|AAAAAAAALBPNECAA|1903-10-23|45|199|16|1903|5|10|23|4|1903|16|199|Friday|1903Q4|N|Y|N|2416389|2416661|2416046|2416319|N|N|N|N|N| +2416412|AAAAAAAAMBPNECAA|1903-10-24|45|199|16|1903|6|10|24|4|1903|16|199|Saturday|1903Q4|N|Y|N|2416389|2416661|2416047|2416320|N|N|N|N|N| +2416413|AAAAAAAANBPNECAA|1903-10-25|45|199|16|1903|0|10|25|4|1903|16|199|Sunday|1903Q4|N|N|N|2416389|2416661|2416048|2416321|N|N|N|N|N| +2416414|AAAAAAAAOBPNECAA|1903-10-26|45|199|16|1903|1|10|26|4|1903|16|199|Monday|1903Q4|N|N|N|2416389|2416661|2416049|2416322|N|N|N|N|N| +2416415|AAAAAAAAPBPNECAA|1903-10-27|45|200|16|1903|2|10|27|4|1903|16|200|Tuesday|1903Q4|N|N|N|2416389|2416661|2416050|2416323|N|N|N|N|N| +2416416|AAAAAAAAACPNECAA|1903-10-28|45|200|16|1903|3|10|28|4|1903|16|200|Wednesday|1903Q4|N|N|N|2416389|2416661|2416051|2416324|N|N|N|N|N| +2416417|AAAAAAAABCPNECAA|1903-10-29|45|200|16|1903|4|10|29|4|1903|16|200|Thursday|1903Q4|N|N|N|2416389|2416661|2416052|2416325|N|N|N|N|N| +2416418|AAAAAAAACCPNECAA|1903-10-30|45|200|16|1903|5|10|30|4|1903|16|200|Friday|1903Q4|N|Y|N|2416389|2416661|2416053|2416326|N|N|N|N|N| +2416419|AAAAAAAADCPNECAA|1903-10-31|45|200|16|1903|6|10|31|4|1903|16|200|Saturday|1903Q4|N|Y|N|2416389|2416661|2416054|2416327|N|N|N|N|N| +2416420|AAAAAAAAECPNECAA|1903-11-01|46|200|16|1903|0|11|1|4|1903|16|200|Sunday|1903Q4|N|N|N|2416420|2416723|2416055|2416328|N|N|N|N|N| +2416421|AAAAAAAAFCPNECAA|1903-11-02|46|200|16|1903|1|11|2|4|1903|16|200|Monday|1903Q4|N|N|N|2416420|2416723|2416056|2416329|N|N|N|N|N| +2416422|AAAAAAAAGCPNECAA|1903-11-03|46|201|16|1903|2|11|3|4|1903|16|201|Tuesday|1903Q4|N|N|N|2416420|2416723|2416057|2416330|N|N|N|N|N| +2416423|AAAAAAAAHCPNECAA|1903-11-04|46|201|16|1903|3|11|4|4|1903|16|201|Wednesday|1903Q4|N|N|N|2416420|2416723|2416058|2416331|N|N|N|N|N| +2416424|AAAAAAAAICPNECAA|1903-11-05|46|201|16|1903|4|11|5|4|1903|16|201|Thursday|1903Q4|N|N|N|2416420|2416723|2416059|2416332|N|N|N|N|N| +2416425|AAAAAAAAJCPNECAA|1903-11-06|46|201|16|1903|5|11|6|4|1903|16|201|Friday|1903Q4|N|Y|N|2416420|2416723|2416060|2416333|N|N|N|N|N| +2416426|AAAAAAAAKCPNECAA|1903-11-07|46|201|16|1903|6|11|7|4|1903|16|201|Saturday|1903Q4|N|Y|N|2416420|2416723|2416061|2416334|N|N|N|N|N| +2416427|AAAAAAAALCPNECAA|1903-11-08|46|201|16|1903|0|11|8|4|1903|16|201|Sunday|1903Q4|N|N|N|2416420|2416723|2416062|2416335|N|N|N|N|N| +2416428|AAAAAAAAMCPNECAA|1903-11-09|46|201|16|1903|1|11|9|4|1903|16|201|Monday|1903Q4|N|N|N|2416420|2416723|2416063|2416336|N|N|N|N|N| +2416429|AAAAAAAANCPNECAA|1903-11-10|46|202|16|1903|2|11|10|4|1903|16|202|Tuesday|1903Q4|N|N|N|2416420|2416723|2416064|2416337|N|N|N|N|N| +2416430|AAAAAAAAOCPNECAA|1903-11-11|46|202|16|1903|3|11|11|4|1903|16|202|Wednesday|1903Q4|N|N|N|2416420|2416723|2416065|2416338|N|N|N|N|N| +2416431|AAAAAAAAPCPNECAA|1903-11-12|46|202|16|1903|4|11|12|4|1903|16|202|Thursday|1903Q4|N|N|N|2416420|2416723|2416066|2416339|N|N|N|N|N| +2416432|AAAAAAAAADPNECAA|1903-11-13|46|202|16|1903|5|11|13|4|1903|16|202|Friday|1903Q4|N|Y|N|2416420|2416723|2416067|2416340|N|N|N|N|N| +2416433|AAAAAAAABDPNECAA|1903-11-14|46|202|16|1903|6|11|14|4|1903|16|202|Saturday|1903Q4|N|Y|N|2416420|2416723|2416068|2416341|N|N|N|N|N| +2416434|AAAAAAAACDPNECAA|1903-11-15|46|202|16|1903|0|11|15|4|1903|16|202|Sunday|1903Q4|N|N|N|2416420|2416723|2416069|2416342|N|N|N|N|N| +2416435|AAAAAAAADDPNECAA|1903-11-16|46|202|16|1903|1|11|16|4|1903|16|202|Monday|1903Q4|N|N|N|2416420|2416723|2416070|2416343|N|N|N|N|N| +2416436|AAAAAAAAEDPNECAA|1903-11-17|46|203|16|1903|2|11|17|4|1903|16|203|Tuesday|1903Q4|N|N|N|2416420|2416723|2416071|2416344|N|N|N|N|N| +2416437|AAAAAAAAFDPNECAA|1903-11-18|46|203|16|1903|3|11|18|4|1903|16|203|Wednesday|1903Q4|N|N|N|2416420|2416723|2416072|2416345|N|N|N|N|N| +2416438|AAAAAAAAGDPNECAA|1903-11-19|46|203|16|1903|4|11|19|4|1903|16|203|Thursday|1903Q4|N|N|N|2416420|2416723|2416073|2416346|N|N|N|N|N| +2416439|AAAAAAAAHDPNECAA|1903-11-20|46|203|16|1903|5|11|20|4|1903|16|203|Friday|1903Q4|N|Y|N|2416420|2416723|2416074|2416347|N|N|N|N|N| +2416440|AAAAAAAAIDPNECAA|1903-11-21|46|203|16|1903|6|11|21|4|1903|16|203|Saturday|1903Q4|N|Y|N|2416420|2416723|2416075|2416348|N|N|N|N|N| +2416441|AAAAAAAAJDPNECAA|1903-11-22|46|203|16|1903|0|11|22|4|1903|16|203|Sunday|1903Q4|N|N|N|2416420|2416723|2416076|2416349|N|N|N|N|N| +2416442|AAAAAAAAKDPNECAA|1903-11-23|46|203|16|1903|1|11|23|4|1903|16|203|Monday|1903Q4|N|N|N|2416420|2416723|2416077|2416350|N|N|N|N|N| +2416443|AAAAAAAALDPNECAA|1903-11-24|46|204|16|1903|2|11|24|4|1903|16|204|Tuesday|1903Q4|N|N|N|2416420|2416723|2416078|2416351|N|N|N|N|N| +2416444|AAAAAAAAMDPNECAA|1903-11-25|46|204|16|1903|3|11|25|4|1903|16|204|Wednesday|1903Q4|N|N|N|2416420|2416723|2416079|2416352|N|N|N|N|N| +2416445|AAAAAAAANDPNECAA|1903-11-26|46|204|16|1903|4|11|26|4|1903|16|204|Thursday|1903Q4|N|N|N|2416420|2416723|2416080|2416353|N|N|N|N|N| +2416446|AAAAAAAAODPNECAA|1903-11-27|46|204|16|1903|5|11|27|4|1903|16|204|Friday|1903Q4|N|Y|N|2416420|2416723|2416081|2416354|N|N|N|N|N| +2416447|AAAAAAAAPDPNECAA|1903-11-28|46|204|16|1903|6|11|28|4|1903|16|204|Saturday|1903Q4|N|Y|N|2416420|2416723|2416082|2416355|N|N|N|N|N| +2416448|AAAAAAAAAEPNECAA|1903-11-29|46|204|16|1903|0|11|29|4|1903|16|204|Sunday|1903Q4|N|N|N|2416420|2416723|2416083|2416356|N|N|N|N|N| +2416449|AAAAAAAABEPNECAA|1903-11-30|46|204|16|1903|1|11|30|4|1903|16|204|Monday|1903Q4|N|N|N|2416420|2416723|2416084|2416357|N|N|N|N|N| +2416450|AAAAAAAACEPNECAA|1903-12-01|47|205|17|1903|2|12|1|4|1903|17|205|Tuesday|1903Q4|N|N|N|2416450|2416783|2416085|2416358|N|N|N|N|N| +2416451|AAAAAAAADEPNECAA|1903-12-02|47|205|17|1903|3|12|2|4|1903|17|205|Wednesday|1903Q4|N|N|N|2416450|2416783|2416086|2416359|N|N|N|N|N| +2416452|AAAAAAAAEEPNECAA|1903-12-03|47|205|17|1903|4|12|3|4|1903|17|205|Thursday|1903Q4|N|N|N|2416450|2416783|2416087|2416360|N|N|N|N|N| +2416453|AAAAAAAAFEPNECAA|1903-12-04|47|205|17|1903|5|12|4|4|1903|17|205|Friday|1903Q4|N|Y|N|2416450|2416783|2416088|2416361|N|N|N|N|N| +2416454|AAAAAAAAGEPNECAA|1903-12-05|47|205|17|1903|6|12|5|4|1903|17|205|Saturday|1903Q4|N|Y|N|2416450|2416783|2416089|2416362|N|N|N|N|N| +2416455|AAAAAAAAHEPNECAA|1903-12-06|47|205|17|1903|0|12|6|4|1903|17|205|Sunday|1903Q4|N|N|N|2416450|2416783|2416090|2416363|N|N|N|N|N| +2416456|AAAAAAAAIEPNECAA|1903-12-07|47|205|17|1903|1|12|7|4|1903|17|205|Monday|1903Q4|N|N|N|2416450|2416783|2416091|2416364|N|N|N|N|N| +2416457|AAAAAAAAJEPNECAA|1903-12-08|47|206|17|1903|2|12|8|4|1903|17|206|Tuesday|1903Q4|N|N|N|2416450|2416783|2416092|2416365|N|N|N|N|N| +2416458|AAAAAAAAKEPNECAA|1903-12-09|47|206|17|1903|3|12|9|4|1903|17|206|Wednesday|1903Q4|N|N|N|2416450|2416783|2416093|2416366|N|N|N|N|N| +2416459|AAAAAAAALEPNECAA|1903-12-10|47|206|17|1903|4|12|10|4|1903|17|206|Thursday|1903Q4|N|N|N|2416450|2416783|2416094|2416367|N|N|N|N|N| +2416460|AAAAAAAAMEPNECAA|1903-12-11|47|206|17|1903|5|12|11|4|1903|17|206|Friday|1903Q4|N|Y|N|2416450|2416783|2416095|2416368|N|N|N|N|N| +2416461|AAAAAAAANEPNECAA|1903-12-12|47|206|17|1903|6|12|12|4|1903|17|206|Saturday|1903Q4|N|Y|N|2416450|2416783|2416096|2416369|N|N|N|N|N| +2416462|AAAAAAAAOEPNECAA|1903-12-13|47|206|17|1903|0|12|13|4|1903|17|206|Sunday|1903Q4|N|N|N|2416450|2416783|2416097|2416370|N|N|N|N|N| +2416463|AAAAAAAAPEPNECAA|1903-12-14|47|206|17|1903|1|12|14|4|1903|17|206|Monday|1903Q4|N|N|N|2416450|2416783|2416098|2416371|N|N|N|N|N| +2416464|AAAAAAAAAFPNECAA|1903-12-15|47|207|17|1903|2|12|15|4|1903|17|207|Tuesday|1903Q4|N|N|N|2416450|2416783|2416099|2416372|N|N|N|N|N| +2416465|AAAAAAAABFPNECAA|1903-12-16|47|207|17|1903|3|12|16|4|1903|17|207|Wednesday|1903Q4|N|N|N|2416450|2416783|2416100|2416373|N|N|N|N|N| +2416466|AAAAAAAACFPNECAA|1903-12-17|47|207|17|1903|4|12|17|4|1903|17|207|Thursday|1903Q4|N|N|N|2416450|2416783|2416101|2416374|N|N|N|N|N| +2416467|AAAAAAAADFPNECAA|1903-12-18|47|207|17|1903|5|12|18|4|1903|17|207|Friday|1903Q4|N|Y|N|2416450|2416783|2416102|2416375|N|N|N|N|N| +2416468|AAAAAAAAEFPNECAA|1903-12-19|47|207|17|1903|6|12|19|4|1903|17|207|Saturday|1903Q4|N|Y|N|2416450|2416783|2416103|2416376|N|N|N|N|N| +2416469|AAAAAAAAFFPNECAA|1903-12-20|47|207|17|1903|0|12|20|4|1903|17|207|Sunday|1903Q4|N|N|N|2416450|2416783|2416104|2416377|N|N|N|N|N| +2416470|AAAAAAAAGFPNECAA|1903-12-21|47|207|17|1903|1|12|21|4|1903|17|207|Monday|1903Q4|N|N|N|2416450|2416783|2416105|2416378|N|N|N|N|N| +2416471|AAAAAAAAHFPNECAA|1903-12-22|47|208|17|1903|2|12|22|4|1903|17|208|Tuesday|1903Q4|N|N|N|2416450|2416783|2416106|2416379|N|N|N|N|N| +2416472|AAAAAAAAIFPNECAA|1903-12-23|47|208|17|1903|3|12|23|4|1903|17|208|Wednesday|1903Q4|N|N|N|2416450|2416783|2416107|2416380|N|N|N|N|N| +2416473|AAAAAAAAJFPNECAA|1903-12-24|47|208|17|1903|4|12|24|4|1903|17|208|Thursday|1903Q4|N|N|N|2416450|2416783|2416108|2416381|N|N|N|N|N| +2416474|AAAAAAAAKFPNECAA|1903-12-25|47|208|17|1903|5|12|25|4|1903|17|208|Friday|1903Q4|N|Y|N|2416450|2416783|2416109|2416382|N|N|N|N|N| +2416475|AAAAAAAALFPNECAA|1903-12-26|47|208|17|1903|6|12|26|4|1903|17|208|Saturday|1903Q4|Y|Y|N|2416450|2416783|2416110|2416383|N|N|N|N|N| +2416476|AAAAAAAAMFPNECAA|1903-12-27|47|208|17|1903|0|12|27|4|1903|17|208|Sunday|1903Q4|N|N|Y|2416450|2416783|2416111|2416384|N|N|N|N|N| +2416477|AAAAAAAANFPNECAA|1903-12-28|47|208|17|1903|1|12|28|4|1903|17|208|Monday|1903Q4|N|N|N|2416450|2416783|2416112|2416385|N|N|N|N|N| +2416478|AAAAAAAAOFPNECAA|1903-12-29|47|209|17|1903|2|12|29|4|1903|17|209|Tuesday|1903Q4|N|N|N|2416450|2416783|2416113|2416386|N|N|N|N|N| +2416479|AAAAAAAAPFPNECAA|1903-12-30|47|209|17|1903|3|12|30|4|1903|17|209|Wednesday|1903Q4|N|N|N|2416450|2416783|2416114|2416387|N|N|N|N|N| +2416480|AAAAAAAAAGPNECAA|1903-12-31|47|209|17|1903|4|12|31|4|1903|17|209|Thursday|1903Q4|N|N|N|2416450|2416783|2416115|2416388|N|N|N|N|N| +2416481|AAAAAAAABGPNECAA|1904-01-01|48|209|17|1904|5|1|1|1|1904|17|209|Friday|1904Q1|Y|Y|N|2416481|2416480|2416116|2416389|N|N|N|N|N| +2416482|AAAAAAAACGPNECAA|1904-01-02|48|209|17|1904|6|1|2|1|1904|17|209|Saturday|1904Q1|N|Y|Y|2416481|2416480|2416117|2416390|N|N|N|N|N| +2416483|AAAAAAAADGPNECAA|1904-01-03|48|209|17|1904|0|1|3|1|1904|17|209|Sunday|1904Q1|N|N|N|2416481|2416480|2416118|2416391|N|N|N|N|N| +2416484|AAAAAAAAEGPNECAA|1904-01-04|48|209|17|1904|1|1|4|1|1904|17|209|Monday|1904Q1|N|N|N|2416481|2416480|2416119|2416392|N|N|N|N|N| +2416485|AAAAAAAAFGPNECAA|1904-01-05|48|210|17|1904|2|1|5|1|1904|17|210|Tuesday|1904Q1|N|N|N|2416481|2416480|2416120|2416393|N|N|N|N|N| +2416486|AAAAAAAAGGPNECAA|1904-01-06|48|210|17|1904|3|1|6|1|1904|17|210|Wednesday|1904Q1|N|N|N|2416481|2416480|2416121|2416394|N|N|N|N|N| +2416487|AAAAAAAAHGPNECAA|1904-01-07|48|210|17|1904|4|1|7|1|1904|17|210|Thursday|1904Q1|N|N|N|2416481|2416480|2416122|2416395|N|N|N|N|N| +2416488|AAAAAAAAIGPNECAA|1904-01-08|48|210|17|1904|5|1|8|1|1904|17|210|Friday|1904Q1|N|Y|N|2416481|2416480|2416123|2416396|N|N|N|N|N| +2416489|AAAAAAAAJGPNECAA|1904-01-09|48|210|17|1904|6|1|9|1|1904|17|210|Saturday|1904Q1|N|Y|N|2416481|2416480|2416124|2416397|N|N|N|N|N| +2416490|AAAAAAAAKGPNECAA|1904-01-10|48|210|17|1904|0|1|10|1|1904|17|210|Sunday|1904Q1|N|N|N|2416481|2416480|2416125|2416398|N|N|N|N|N| +2416491|AAAAAAAALGPNECAA|1904-01-11|48|210|17|1904|1|1|11|1|1904|17|210|Monday|1904Q1|N|N|N|2416481|2416480|2416126|2416399|N|N|N|N|N| +2416492|AAAAAAAAMGPNECAA|1904-01-12|48|211|17|1904|2|1|12|1|1904|17|211|Tuesday|1904Q1|N|N|N|2416481|2416480|2416127|2416400|N|N|N|N|N| +2416493|AAAAAAAANGPNECAA|1904-01-13|48|211|17|1904|3|1|13|1|1904|17|211|Wednesday|1904Q1|N|N|N|2416481|2416480|2416128|2416401|N|N|N|N|N| +2416494|AAAAAAAAOGPNECAA|1904-01-14|48|211|17|1904|4|1|14|1|1904|17|211|Thursday|1904Q1|N|N|N|2416481|2416480|2416129|2416402|N|N|N|N|N| +2416495|AAAAAAAAPGPNECAA|1904-01-15|48|211|17|1904|5|1|15|1|1904|17|211|Friday|1904Q1|N|Y|N|2416481|2416480|2416130|2416403|N|N|N|N|N| +2416496|AAAAAAAAAHPNECAA|1904-01-16|48|211|17|1904|6|1|16|1|1904|17|211|Saturday|1904Q1|N|Y|N|2416481|2416480|2416131|2416404|N|N|N|N|N| +2416497|AAAAAAAABHPNECAA|1904-01-17|48|211|17|1904|0|1|17|1|1904|17|211|Sunday|1904Q1|N|N|N|2416481|2416480|2416132|2416405|N|N|N|N|N| +2416498|AAAAAAAACHPNECAA|1904-01-18|48|211|17|1904|1|1|18|1|1904|17|211|Monday|1904Q1|N|N|N|2416481|2416480|2416133|2416406|N|N|N|N|N| +2416499|AAAAAAAADHPNECAA|1904-01-19|48|212|17|1904|2|1|19|1|1904|17|212|Tuesday|1904Q1|N|N|N|2416481|2416480|2416134|2416407|N|N|N|N|N| +2416500|AAAAAAAAEHPNECAA|1904-01-20|48|212|17|1904|3|1|20|1|1904|17|212|Wednesday|1904Q1|N|N|N|2416481|2416480|2416135|2416408|N|N|N|N|N| +2416501|AAAAAAAAFHPNECAA|1904-01-21|48|212|17|1904|4|1|21|1|1904|17|212|Thursday|1904Q1|N|N|N|2416481|2416480|2416136|2416409|N|N|N|N|N| +2416502|AAAAAAAAGHPNECAA|1904-01-22|48|212|17|1904|5|1|22|1|1904|17|212|Friday|1904Q1|N|Y|N|2416481|2416480|2416137|2416410|N|N|N|N|N| +2416503|AAAAAAAAHHPNECAA|1904-01-23|48|212|17|1904|6|1|23|1|1904|17|212|Saturday|1904Q1|N|Y|N|2416481|2416480|2416138|2416411|N|N|N|N|N| +2416504|AAAAAAAAIHPNECAA|1904-01-24|48|212|17|1904|0|1|24|1|1904|17|212|Sunday|1904Q1|N|N|N|2416481|2416480|2416139|2416412|N|N|N|N|N| +2416505|AAAAAAAAJHPNECAA|1904-01-25|48|212|17|1904|1|1|25|1|1904|17|212|Monday|1904Q1|N|N|N|2416481|2416480|2416140|2416413|N|N|N|N|N| +2416506|AAAAAAAAKHPNECAA|1904-01-26|48|213|17|1904|2|1|26|1|1904|17|213|Tuesday|1904Q1|N|N|N|2416481|2416480|2416141|2416414|N|N|N|N|N| +2416507|AAAAAAAALHPNECAA|1904-01-27|48|213|17|1904|3|1|27|1|1904|17|213|Wednesday|1904Q1|N|N|N|2416481|2416480|2416142|2416415|N|N|N|N|N| +2416508|AAAAAAAAMHPNECAA|1904-01-28|48|213|17|1904|4|1|28|1|1904|17|213|Thursday|1904Q1|N|N|N|2416481|2416480|2416143|2416416|N|N|N|N|N| +2416509|AAAAAAAANHPNECAA|1904-01-29|48|213|17|1904|5|1|29|1|1904|17|213|Friday|1904Q1|N|Y|N|2416481|2416480|2416144|2416417|N|N|N|N|N| +2416510|AAAAAAAAOHPNECAA|1904-01-30|48|213|17|1904|6|1|30|1|1904|17|213|Saturday|1904Q1|N|Y|N|2416481|2416480|2416145|2416418|N|N|N|N|N| +2416511|AAAAAAAAPHPNECAA|1904-01-31|48|213|17|1904|0|1|31|1|1904|17|213|Sunday|1904Q1|N|N|N|2416481|2416480|2416146|2416419|N|N|N|N|N| +2416512|AAAAAAAAAIPNECAA|1904-02-01|49|213|17|1904|1|2|1|1|1904|17|213|Monday|1904Q1|N|N|N|2416512|2416542|2416147|2416420|N|N|N|N|N| +2416513|AAAAAAAABIPNECAA|1904-02-02|49|214|17|1904|2|2|2|1|1904|17|214|Tuesday|1904Q1|N|N|N|2416512|2416542|2416148|2416421|N|N|N|N|N| +2416514|AAAAAAAACIPNECAA|1904-02-03|49|214|17|1904|3|2|3|1|1904|17|214|Wednesday|1904Q1|N|N|N|2416512|2416542|2416149|2416422|N|N|N|N|N| +2416515|AAAAAAAADIPNECAA|1904-02-04|49|214|17|1904|4|2|4|1|1904|17|214|Thursday|1904Q1|N|N|N|2416512|2416542|2416150|2416423|N|N|N|N|N| +2416516|AAAAAAAAEIPNECAA|1904-02-05|49|214|17|1904|5|2|5|1|1904|17|214|Friday|1904Q1|N|Y|N|2416512|2416542|2416151|2416424|N|N|N|N|N| +2416517|AAAAAAAAFIPNECAA|1904-02-06|49|214|17|1904|6|2|6|1|1904|17|214|Saturday|1904Q1|N|Y|N|2416512|2416542|2416152|2416425|N|N|N|N|N| +2416518|AAAAAAAAGIPNECAA|1904-02-07|49|214|17|1904|0|2|7|1|1904|17|214|Sunday|1904Q1|N|N|N|2416512|2416542|2416153|2416426|N|N|N|N|N| +2416519|AAAAAAAAHIPNECAA|1904-02-08|49|214|17|1904|1|2|8|1|1904|17|214|Monday|1904Q1|N|N|N|2416512|2416542|2416154|2416427|N|N|N|N|N| +2416520|AAAAAAAAIIPNECAA|1904-02-09|49|215|17|1904|2|2|9|1|1904|17|215|Tuesday|1904Q1|N|N|N|2416512|2416542|2416155|2416428|N|N|N|N|N| +2416521|AAAAAAAAJIPNECAA|1904-02-10|49|215|17|1904|3|2|10|1|1904|17|215|Wednesday|1904Q1|N|N|N|2416512|2416542|2416156|2416429|N|N|N|N|N| +2416522|AAAAAAAAKIPNECAA|1904-02-11|49|215|17|1904|4|2|11|1|1904|17|215|Thursday|1904Q1|N|N|N|2416512|2416542|2416157|2416430|N|N|N|N|N| +2416523|AAAAAAAALIPNECAA|1904-02-12|49|215|17|1904|5|2|12|1|1904|17|215|Friday|1904Q1|N|Y|N|2416512|2416542|2416158|2416431|N|N|N|N|N| +2416524|AAAAAAAAMIPNECAA|1904-02-13|49|215|17|1904|6|2|13|1|1904|17|215|Saturday|1904Q1|N|Y|N|2416512|2416542|2416159|2416432|N|N|N|N|N| +2416525|AAAAAAAANIPNECAA|1904-02-14|49|215|17|1904|0|2|14|1|1904|17|215|Sunday|1904Q1|N|N|N|2416512|2416542|2416160|2416433|N|N|N|N|N| +2416526|AAAAAAAAOIPNECAA|1904-02-15|49|215|17|1904|1|2|15|1|1904|17|215|Monday|1904Q1|N|N|N|2416512|2416542|2416161|2416434|N|N|N|N|N| +2416527|AAAAAAAAPIPNECAA|1904-02-16|49|216|17|1904|2|2|16|1|1904|17|216|Tuesday|1904Q1|N|N|N|2416512|2416542|2416162|2416435|N|N|N|N|N| +2416528|AAAAAAAAAJPNECAA|1904-02-17|49|216|17|1904|3|2|17|1|1904|17|216|Wednesday|1904Q1|N|N|N|2416512|2416542|2416163|2416436|N|N|N|N|N| +2416529|AAAAAAAABJPNECAA|1904-02-18|49|216|17|1904|4|2|18|1|1904|17|216|Thursday|1904Q1|N|N|N|2416512|2416542|2416164|2416437|N|N|N|N|N| +2416530|AAAAAAAACJPNECAA|1904-02-19|49|216|17|1904|5|2|19|1|1904|17|216|Friday|1904Q1|N|Y|N|2416512|2416542|2416165|2416438|N|N|N|N|N| +2416531|AAAAAAAADJPNECAA|1904-02-20|49|216|17|1904|6|2|20|1|1904|17|216|Saturday|1904Q1|N|Y|N|2416512|2416542|2416166|2416439|N|N|N|N|N| +2416532|AAAAAAAAEJPNECAA|1904-02-21|49|216|17|1904|0|2|21|1|1904|17|216|Sunday|1904Q1|N|N|N|2416512|2416542|2416167|2416440|N|N|N|N|N| +2416533|AAAAAAAAFJPNECAA|1904-02-22|49|216|17|1904|1|2|22|1|1904|17|216|Monday|1904Q1|N|N|N|2416512|2416542|2416168|2416441|N|N|N|N|N| +2416534|AAAAAAAAGJPNECAA|1904-02-23|49|217|17|1904|2|2|23|1|1904|17|217|Tuesday|1904Q1|N|N|N|2416512|2416542|2416169|2416442|N|N|N|N|N| +2416535|AAAAAAAAHJPNECAA|1904-02-24|49|217|17|1904|3|2|24|1|1904|17|217|Wednesday|1904Q1|N|N|N|2416512|2416542|2416170|2416443|N|N|N|N|N| +2416536|AAAAAAAAIJPNECAA|1904-02-25|49|217|17|1904|4|2|25|1|1904|17|217|Thursday|1904Q1|N|N|N|2416512|2416542|2416171|2416444|N|N|N|N|N| +2416537|AAAAAAAAJJPNECAA|1904-02-26|49|217|17|1904|5|2|26|1|1904|17|217|Friday|1904Q1|N|Y|N|2416512|2416542|2416172|2416445|N|N|N|N|N| +2416538|AAAAAAAAKJPNECAA|1904-02-27|49|217|17|1904|6|2|27|1|1904|17|217|Saturday|1904Q1|N|Y|N|2416512|2416542|2416173|2416446|N|N|N|N|N| +2416539|AAAAAAAALJPNECAA|1904-02-28|49|217|17|1904|0|2|28|1|1904|17|217|Sunday|1904Q1|N|N|N|2416512|2416542|2416174|2416447|N|N|N|N|N| +2416540|AAAAAAAAMJPNECAA|1904-02-29|49|217|17|1904|1|2|29|1|1904|17|217|Monday|1904Q1|N|N|N|2416512|2416542|2416174|2416448|N|N|N|N|N| +2416541|AAAAAAAANJPNECAA|1904-03-01|50|218|18|1904|2|3|1|1|1904|18|218|Tuesday|1904Q1|N|N|N|2416541|2416600|2416175|2416449|N|N|N|N|N| +2416542|AAAAAAAAOJPNECAA|1904-03-02|50|218|18|1904|3|3|2|1|1904|18|218|Wednesday|1904Q1|N|N|N|2416541|2416600|2416176|2416450|N|N|N|N|N| +2416543|AAAAAAAAPJPNECAA|1904-03-03|50|218|18|1904|4|3|3|1|1904|18|218|Thursday|1904Q1|N|N|N|2416541|2416600|2416177|2416451|N|N|N|N|N| +2416544|AAAAAAAAAKPNECAA|1904-03-04|50|218|18|1904|5|3|4|1|1904|18|218|Friday|1904Q1|N|Y|N|2416541|2416600|2416178|2416452|N|N|N|N|N| +2416545|AAAAAAAABKPNECAA|1904-03-05|50|218|18|1904|6|3|5|1|1904|18|218|Saturday|1904Q1|N|Y|N|2416541|2416600|2416179|2416453|N|N|N|N|N| +2416546|AAAAAAAACKPNECAA|1904-03-06|50|218|18|1904|0|3|6|1|1904|18|218|Sunday|1904Q1|N|N|N|2416541|2416600|2416180|2416454|N|N|N|N|N| +2416547|AAAAAAAADKPNECAA|1904-03-07|50|218|18|1904|1|3|7|1|1904|18|218|Monday|1904Q1|N|N|N|2416541|2416600|2416181|2416455|N|N|N|N|N| +2416548|AAAAAAAAEKPNECAA|1904-03-08|50|219|18|1904|2|3|8|1|1904|18|219|Tuesday|1904Q1|N|N|N|2416541|2416600|2416182|2416456|N|N|N|N|N| +2416549|AAAAAAAAFKPNECAA|1904-03-09|50|219|18|1904|3|3|9|1|1904|18|219|Wednesday|1904Q1|N|N|N|2416541|2416600|2416183|2416457|N|N|N|N|N| +2416550|AAAAAAAAGKPNECAA|1904-03-10|50|219|18|1904|4|3|10|1|1904|18|219|Thursday|1904Q1|N|N|N|2416541|2416600|2416184|2416458|N|N|N|N|N| +2416551|AAAAAAAAHKPNECAA|1904-03-11|50|219|18|1904|5|3|11|1|1904|18|219|Friday|1904Q1|N|Y|N|2416541|2416600|2416185|2416459|N|N|N|N|N| +2416552|AAAAAAAAIKPNECAA|1904-03-12|50|219|18|1904|6|3|12|1|1904|18|219|Saturday|1904Q1|N|Y|N|2416541|2416600|2416186|2416460|N|N|N|N|N| +2416553|AAAAAAAAJKPNECAA|1904-03-13|50|219|18|1904|0|3|13|1|1904|18|219|Sunday|1904Q1|N|N|N|2416541|2416600|2416187|2416461|N|N|N|N|N| +2416554|AAAAAAAAKKPNECAA|1904-03-14|50|219|18|1904|1|3|14|1|1904|18|219|Monday|1904Q1|N|N|N|2416541|2416600|2416188|2416462|N|N|N|N|N| +2416555|AAAAAAAALKPNECAA|1904-03-15|50|220|18|1904|2|3|15|1|1904|18|220|Tuesday|1904Q1|N|N|N|2416541|2416600|2416189|2416463|N|N|N|N|N| +2416556|AAAAAAAAMKPNECAA|1904-03-16|50|220|18|1904|3|3|16|1|1904|18|220|Wednesday|1904Q1|N|N|N|2416541|2416600|2416190|2416464|N|N|N|N|N| +2416557|AAAAAAAANKPNECAA|1904-03-17|50|220|18|1904|4|3|17|1|1904|18|220|Thursday|1904Q1|N|N|N|2416541|2416600|2416191|2416465|N|N|N|N|N| +2416558|AAAAAAAAOKPNECAA|1904-03-18|50|220|18|1904|5|3|18|1|1904|18|220|Friday|1904Q1|N|Y|N|2416541|2416600|2416192|2416466|N|N|N|N|N| +2416559|AAAAAAAAPKPNECAA|1904-03-19|50|220|18|1904|6|3|19|1|1904|18|220|Saturday|1904Q1|N|Y|N|2416541|2416600|2416193|2416467|N|N|N|N|N| +2416560|AAAAAAAAALPNECAA|1904-03-20|50|220|18|1904|0|3|20|1|1904|18|220|Sunday|1904Q1|N|N|N|2416541|2416600|2416194|2416468|N|N|N|N|N| +2416561|AAAAAAAABLPNECAA|1904-03-21|50|220|18|1904|1|3|21|1|1904|18|220|Monday|1904Q1|N|N|N|2416541|2416600|2416195|2416469|N|N|N|N|N| +2416562|AAAAAAAACLPNECAA|1904-03-22|50|221|18|1904|2|3|22|1|1904|18|221|Tuesday|1904Q1|N|N|N|2416541|2416600|2416196|2416470|N|N|N|N|N| +2416563|AAAAAAAADLPNECAA|1904-03-23|50|221|18|1904|3|3|23|1|1904|18|221|Wednesday|1904Q1|N|N|N|2416541|2416600|2416197|2416471|N|N|N|N|N| +2416564|AAAAAAAAELPNECAA|1904-03-24|50|221|18|1904|4|3|24|1|1904|18|221|Thursday|1904Q1|N|N|N|2416541|2416600|2416198|2416472|N|N|N|N|N| +2416565|AAAAAAAAFLPNECAA|1904-03-25|50|221|18|1904|5|3|25|1|1904|18|221|Friday|1904Q1|N|Y|N|2416541|2416600|2416199|2416473|N|N|N|N|N| +2416566|AAAAAAAAGLPNECAA|1904-03-26|50|221|18|1904|6|3|26|1|1904|18|221|Saturday|1904Q1|N|Y|N|2416541|2416600|2416200|2416474|N|N|N|N|N| +2416567|AAAAAAAAHLPNECAA|1904-03-27|50|221|18|1904|0|3|27|1|1904|18|221|Sunday|1904Q1|N|N|N|2416541|2416600|2416201|2416475|N|N|N|N|N| +2416568|AAAAAAAAILPNECAA|1904-03-28|50|221|18|1904|1|3|28|1|1904|18|221|Monday|1904Q1|N|N|N|2416541|2416600|2416202|2416476|N|N|N|N|N| +2416569|AAAAAAAAJLPNECAA|1904-03-29|50|222|18|1904|2|3|29|1|1904|18|222|Tuesday|1904Q1|N|N|N|2416541|2416600|2416203|2416477|N|N|N|N|N| +2416570|AAAAAAAAKLPNECAA|1904-03-30|50|222|18|1904|3|3|30|1|1904|18|222|Wednesday|1904Q1|N|N|N|2416541|2416600|2416204|2416478|N|N|N|N|N| +2416571|AAAAAAAALLPNECAA|1904-03-31|50|222|18|1904|4|3|31|1|1904|18|222|Thursday|1904Q1|N|N|N|2416541|2416600|2416205|2416479|N|N|N|N|N| +2416572|AAAAAAAAMLPNECAA|1904-04-01|51|222|18|1904|5|4|1|2|1904|18|222|Friday|1904Q2|N|Y|N|2416572|2416662|2416206|2416481|N|N|N|N|N| +2416573|AAAAAAAANLPNECAA|1904-04-02|51|222|18|1904|6|4|2|2|1904|18|222|Saturday|1904Q2|N|Y|N|2416572|2416662|2416207|2416482|N|N|N|N|N| +2416574|AAAAAAAAOLPNECAA|1904-04-03|51|222|18|1904|0|4|3|2|1904|18|222|Sunday|1904Q2|N|N|N|2416572|2416662|2416208|2416483|N|N|N|N|N| +2416575|AAAAAAAAPLPNECAA|1904-04-04|51|222|18|1904|1|4|4|2|1904|18|222|Monday|1904Q2|N|N|N|2416572|2416662|2416209|2416484|N|N|N|N|N| +2416576|AAAAAAAAAMPNECAA|1904-04-05|51|223|18|1904|2|4|5|2|1904|18|223|Tuesday|1904Q2|N|N|N|2416572|2416662|2416210|2416485|N|N|N|N|N| +2416577|AAAAAAAABMPNECAA|1904-04-06|51|223|18|1904|3|4|6|2|1904|18|223|Wednesday|1904Q2|N|N|N|2416572|2416662|2416211|2416486|N|N|N|N|N| +2416578|AAAAAAAACMPNECAA|1904-04-07|51|223|18|1904|4|4|7|2|1904|18|223|Thursday|1904Q2|N|N|N|2416572|2416662|2416212|2416487|N|N|N|N|N| +2416579|AAAAAAAADMPNECAA|1904-04-08|51|223|18|1904|5|4|8|2|1904|18|223|Friday|1904Q2|N|Y|N|2416572|2416662|2416213|2416488|N|N|N|N|N| +2416580|AAAAAAAAEMPNECAA|1904-04-09|51|223|18|1904|6|4|9|2|1904|18|223|Saturday|1904Q2|N|Y|N|2416572|2416662|2416214|2416489|N|N|N|N|N| +2416581|AAAAAAAAFMPNECAA|1904-04-10|51|223|18|1904|0|4|10|2|1904|18|223|Sunday|1904Q2|N|N|N|2416572|2416662|2416215|2416490|N|N|N|N|N| +2416582|AAAAAAAAGMPNECAA|1904-04-11|51|223|18|1904|1|4|11|2|1904|18|223|Monday|1904Q2|N|N|N|2416572|2416662|2416216|2416491|N|N|N|N|N| +2416583|AAAAAAAAHMPNECAA|1904-04-12|51|224|18|1904|2|4|12|2|1904|18|224|Tuesday|1904Q2|N|N|N|2416572|2416662|2416217|2416492|N|N|N|N|N| +2416584|AAAAAAAAIMPNECAA|1904-04-13|51|224|18|1904|3|4|13|2|1904|18|224|Wednesday|1904Q2|N|N|N|2416572|2416662|2416218|2416493|N|N|N|N|N| +2416585|AAAAAAAAJMPNECAA|1904-04-14|51|224|18|1904|4|4|14|2|1904|18|224|Thursday|1904Q2|N|N|N|2416572|2416662|2416219|2416494|N|N|N|N|N| +2416586|AAAAAAAAKMPNECAA|1904-04-15|51|224|18|1904|5|4|15|2|1904|18|224|Friday|1904Q2|N|Y|N|2416572|2416662|2416220|2416495|N|N|N|N|N| +2416587|AAAAAAAALMPNECAA|1904-04-16|51|224|18|1904|6|4|16|2|1904|18|224|Saturday|1904Q2|N|Y|N|2416572|2416662|2416221|2416496|N|N|N|N|N| +2416588|AAAAAAAAMMPNECAA|1904-04-17|51|224|18|1904|0|4|17|2|1904|18|224|Sunday|1904Q2|N|N|N|2416572|2416662|2416222|2416497|N|N|N|N|N| +2416589|AAAAAAAANMPNECAA|1904-04-18|51|224|18|1904|1|4|18|2|1904|18|224|Monday|1904Q2|N|N|N|2416572|2416662|2416223|2416498|N|N|N|N|N| +2416590|AAAAAAAAOMPNECAA|1904-04-19|51|225|18|1904|2|4|19|2|1904|18|225|Tuesday|1904Q2|N|N|N|2416572|2416662|2416224|2416499|N|N|N|N|N| +2416591|AAAAAAAAPMPNECAA|1904-04-20|51|225|18|1904|3|4|20|2|1904|18|225|Wednesday|1904Q2|N|N|N|2416572|2416662|2416225|2416500|N|N|N|N|N| +2416592|AAAAAAAAANPNECAA|1904-04-21|51|225|18|1904|4|4|21|2|1904|18|225|Thursday|1904Q2|N|N|N|2416572|2416662|2416226|2416501|N|N|N|N|N| +2416593|AAAAAAAABNPNECAA|1904-04-22|51|225|18|1904|5|4|22|2|1904|18|225|Friday|1904Q2|N|Y|N|2416572|2416662|2416227|2416502|N|N|N|N|N| +2416594|AAAAAAAACNPNECAA|1904-04-23|51|225|18|1904|6|4|23|2|1904|18|225|Saturday|1904Q2|N|Y|N|2416572|2416662|2416228|2416503|N|N|N|N|N| +2416595|AAAAAAAADNPNECAA|1904-04-24|51|225|18|1904|0|4|24|2|1904|18|225|Sunday|1904Q2|N|N|N|2416572|2416662|2416229|2416504|N|N|N|N|N| +2416596|AAAAAAAAENPNECAA|1904-04-25|51|225|18|1904|1|4|25|2|1904|18|225|Monday|1904Q2|N|N|N|2416572|2416662|2416230|2416505|N|N|N|N|N| +2416597|AAAAAAAAFNPNECAA|1904-04-26|51|226|18|1904|2|4|26|2|1904|18|226|Tuesday|1904Q2|N|N|N|2416572|2416662|2416231|2416506|N|N|N|N|N| +2416598|AAAAAAAAGNPNECAA|1904-04-27|51|226|18|1904|3|4|27|2|1904|18|226|Wednesday|1904Q2|N|N|N|2416572|2416662|2416232|2416507|N|N|N|N|N| +2416599|AAAAAAAAHNPNECAA|1904-04-28|51|226|18|1904|4|4|28|2|1904|18|226|Thursday|1904Q2|N|N|N|2416572|2416662|2416233|2416508|N|N|N|N|N| +2416600|AAAAAAAAINPNECAA|1904-04-29|51|226|18|1904|5|4|29|2|1904|18|226|Friday|1904Q2|N|Y|N|2416572|2416662|2416234|2416509|N|N|N|N|N| +2416601|AAAAAAAAJNPNECAA|1904-04-30|51|226|18|1904|6|4|30|2|1904|18|226|Saturday|1904Q2|N|Y|N|2416572|2416662|2416235|2416510|N|N|N|N|N| +2416602|AAAAAAAAKNPNECAA|1904-05-01|52|226|18|1904|0|5|1|2|1904|18|226|Sunday|1904Q2|N|N|N|2416602|2416722|2416236|2416511|N|N|N|N|N| +2416603|AAAAAAAALNPNECAA|1904-05-02|52|226|18|1904|1|5|2|2|1904|18|226|Monday|1904Q2|N|N|N|2416602|2416722|2416237|2416512|N|N|N|N|N| +2416604|AAAAAAAAMNPNECAA|1904-05-03|52|227|18|1904|2|5|3|2|1904|18|227|Tuesday|1904Q2|N|N|N|2416602|2416722|2416238|2416513|N|N|N|N|N| +2416605|AAAAAAAANNPNECAA|1904-05-04|52|227|18|1904|3|5|4|2|1904|18|227|Wednesday|1904Q2|N|N|N|2416602|2416722|2416239|2416514|N|N|N|N|N| +2416606|AAAAAAAAONPNECAA|1904-05-05|52|227|18|1904|4|5|5|2|1904|18|227|Thursday|1904Q2|N|N|N|2416602|2416722|2416240|2416515|N|N|N|N|N| +2416607|AAAAAAAAPNPNECAA|1904-05-06|52|227|18|1904|5|5|6|2|1904|18|227|Friday|1904Q2|N|Y|N|2416602|2416722|2416241|2416516|N|N|N|N|N| +2416608|AAAAAAAAAOPNECAA|1904-05-07|52|227|18|1904|6|5|7|2|1904|18|227|Saturday|1904Q2|N|Y|N|2416602|2416722|2416242|2416517|N|N|N|N|N| +2416609|AAAAAAAABOPNECAA|1904-05-08|52|227|18|1904|0|5|8|2|1904|18|227|Sunday|1904Q2|N|N|N|2416602|2416722|2416243|2416518|N|N|N|N|N| +2416610|AAAAAAAACOPNECAA|1904-05-09|52|227|18|1904|1|5|9|2|1904|18|227|Monday|1904Q2|N|N|N|2416602|2416722|2416244|2416519|N|N|N|N|N| +2416611|AAAAAAAADOPNECAA|1904-05-10|52|228|18|1904|2|5|10|2|1904|18|228|Tuesday|1904Q2|N|N|N|2416602|2416722|2416245|2416520|N|N|N|N|N| +2416612|AAAAAAAAEOPNECAA|1904-05-11|52|228|18|1904|3|5|11|2|1904|18|228|Wednesday|1904Q2|N|N|N|2416602|2416722|2416246|2416521|N|N|N|N|N| +2416613|AAAAAAAAFOPNECAA|1904-05-12|52|228|18|1904|4|5|12|2|1904|18|228|Thursday|1904Q2|N|N|N|2416602|2416722|2416247|2416522|N|N|N|N|N| +2416614|AAAAAAAAGOPNECAA|1904-05-13|52|228|18|1904|5|5|13|2|1904|18|228|Friday|1904Q2|N|Y|N|2416602|2416722|2416248|2416523|N|N|N|N|N| +2416615|AAAAAAAAHOPNECAA|1904-05-14|52|228|18|1904|6|5|14|2|1904|18|228|Saturday|1904Q2|N|Y|N|2416602|2416722|2416249|2416524|N|N|N|N|N| +2416616|AAAAAAAAIOPNECAA|1904-05-15|52|228|18|1904|0|5|15|2|1904|18|228|Sunday|1904Q2|N|N|N|2416602|2416722|2416250|2416525|N|N|N|N|N| +2416617|AAAAAAAAJOPNECAA|1904-05-16|52|228|18|1904|1|5|16|2|1904|18|228|Monday|1904Q2|N|N|N|2416602|2416722|2416251|2416526|N|N|N|N|N| +2416618|AAAAAAAAKOPNECAA|1904-05-17|52|229|18|1904|2|5|17|2|1904|18|229|Tuesday|1904Q2|N|N|N|2416602|2416722|2416252|2416527|N|N|N|N|N| +2416619|AAAAAAAALOPNECAA|1904-05-18|52|229|18|1904|3|5|18|2|1904|18|229|Wednesday|1904Q2|N|N|N|2416602|2416722|2416253|2416528|N|N|N|N|N| +2416620|AAAAAAAAMOPNECAA|1904-05-19|52|229|18|1904|4|5|19|2|1904|18|229|Thursday|1904Q2|N|N|N|2416602|2416722|2416254|2416529|N|N|N|N|N| +2416621|AAAAAAAANOPNECAA|1904-05-20|52|229|18|1904|5|5|20|2|1904|18|229|Friday|1904Q2|N|Y|N|2416602|2416722|2416255|2416530|N|N|N|N|N| +2416622|AAAAAAAAOOPNECAA|1904-05-21|52|229|18|1904|6|5|21|2|1904|18|229|Saturday|1904Q2|N|Y|N|2416602|2416722|2416256|2416531|N|N|N|N|N| +2416623|AAAAAAAAPOPNECAA|1904-05-22|52|229|18|1904|0|5|22|2|1904|18|229|Sunday|1904Q2|N|N|N|2416602|2416722|2416257|2416532|N|N|N|N|N| +2416624|AAAAAAAAAPPNECAA|1904-05-23|52|229|18|1904|1|5|23|2|1904|18|229|Monday|1904Q2|N|N|N|2416602|2416722|2416258|2416533|N|N|N|N|N| +2416625|AAAAAAAABPPNECAA|1904-05-24|52|230|18|1904|2|5|24|2|1904|18|230|Tuesday|1904Q2|N|N|N|2416602|2416722|2416259|2416534|N|N|N|N|N| +2416626|AAAAAAAACPPNECAA|1904-05-25|52|230|18|1904|3|5|25|2|1904|18|230|Wednesday|1904Q2|N|N|N|2416602|2416722|2416260|2416535|N|N|N|N|N| +2416627|AAAAAAAADPPNECAA|1904-05-26|52|230|18|1904|4|5|26|2|1904|18|230|Thursday|1904Q2|N|N|N|2416602|2416722|2416261|2416536|N|N|N|N|N| +2416628|AAAAAAAAEPPNECAA|1904-05-27|52|230|18|1904|5|5|27|2|1904|18|230|Friday|1904Q2|N|Y|N|2416602|2416722|2416262|2416537|N|N|N|N|N| +2416629|AAAAAAAAFPPNECAA|1904-05-28|52|230|18|1904|6|5|28|2|1904|18|230|Saturday|1904Q2|N|Y|N|2416602|2416722|2416263|2416538|N|N|N|N|N| +2416630|AAAAAAAAGPPNECAA|1904-05-29|52|230|18|1904|0|5|29|2|1904|18|230|Sunday|1904Q2|N|N|N|2416602|2416722|2416264|2416539|N|N|N|N|N| +2416631|AAAAAAAAHPPNECAA|1904-05-30|52|230|18|1904|1|5|30|2|1904|18|230|Monday|1904Q2|N|N|N|2416602|2416722|2416265|2416540|N|N|N|N|N| +2416632|AAAAAAAAIPPNECAA|1904-05-31|52|231|18|1904|2|5|31|2|1904|18|231|Tuesday|1904Q2|N|N|N|2416602|2416722|2416266|2416541|N|N|N|N|N| +2416633|AAAAAAAAJPPNECAA|1904-06-01|53|231|19|1904|3|6|1|2|1904|19|231|Wednesday|1904Q2|N|N|N|2416633|2416784|2416267|2416542|N|N|N|N|N| +2416634|AAAAAAAAKPPNECAA|1904-06-02|53|231|19|1904|4|6|2|2|1904|19|231|Thursday|1904Q2|N|N|N|2416633|2416784|2416268|2416543|N|N|N|N|N| +2416635|AAAAAAAALPPNECAA|1904-06-03|53|231|19|1904|5|6|3|2|1904|19|231|Friday|1904Q2|N|Y|N|2416633|2416784|2416269|2416544|N|N|N|N|N| +2416636|AAAAAAAAMPPNECAA|1904-06-04|53|231|19|1904|6|6|4|2|1904|19|231|Saturday|1904Q2|N|Y|N|2416633|2416784|2416270|2416545|N|N|N|N|N| +2416637|AAAAAAAANPPNECAA|1904-06-05|53|231|19|1904|0|6|5|2|1904|19|231|Sunday|1904Q2|N|N|N|2416633|2416784|2416271|2416546|N|N|N|N|N| +2416638|AAAAAAAAOPPNECAA|1904-06-06|53|231|19|1904|1|6|6|2|1904|19|231|Monday|1904Q2|N|N|N|2416633|2416784|2416272|2416547|N|N|N|N|N| +2416639|AAAAAAAAPPPNECAA|1904-06-07|53|232|19|1904|2|6|7|2|1904|19|232|Tuesday|1904Q2|N|N|N|2416633|2416784|2416273|2416548|N|N|N|N|N| +2416640|AAAAAAAAAAAOECAA|1904-06-08|53|232|19|1904|3|6|8|2|1904|19|232|Wednesday|1904Q2|N|N|N|2416633|2416784|2416274|2416549|N|N|N|N|N| +2416641|AAAAAAAABAAOECAA|1904-06-09|53|232|19|1904|4|6|9|2|1904|19|232|Thursday|1904Q2|N|N|N|2416633|2416784|2416275|2416550|N|N|N|N|N| +2416642|AAAAAAAACAAOECAA|1904-06-10|53|232|19|1904|5|6|10|2|1904|19|232|Friday|1904Q2|N|Y|N|2416633|2416784|2416276|2416551|N|N|N|N|N| +2416643|AAAAAAAADAAOECAA|1904-06-11|53|232|19|1904|6|6|11|2|1904|19|232|Saturday|1904Q2|N|Y|N|2416633|2416784|2416277|2416552|N|N|N|N|N| +2416644|AAAAAAAAEAAOECAA|1904-06-12|53|232|19|1904|0|6|12|2|1904|19|232|Sunday|1904Q2|N|N|N|2416633|2416784|2416278|2416553|N|N|N|N|N| +2416645|AAAAAAAAFAAOECAA|1904-06-13|53|232|19|1904|1|6|13|2|1904|19|232|Monday|1904Q2|N|N|N|2416633|2416784|2416279|2416554|N|N|N|N|N| +2416646|AAAAAAAAGAAOECAA|1904-06-14|53|233|19|1904|2|6|14|2|1904|19|233|Tuesday|1904Q2|N|N|N|2416633|2416784|2416280|2416555|N|N|N|N|N| +2416647|AAAAAAAAHAAOECAA|1904-06-15|53|233|19|1904|3|6|15|2|1904|19|233|Wednesday|1904Q2|N|N|N|2416633|2416784|2416281|2416556|N|N|N|N|N| +2416648|AAAAAAAAIAAOECAA|1904-06-16|53|233|19|1904|4|6|16|2|1904|19|233|Thursday|1904Q2|N|N|N|2416633|2416784|2416282|2416557|N|N|N|N|N| +2416649|AAAAAAAAJAAOECAA|1904-06-17|53|233|19|1904|5|6|17|2|1904|19|233|Friday|1904Q2|N|Y|N|2416633|2416784|2416283|2416558|N|N|N|N|N| +2416650|AAAAAAAAKAAOECAA|1904-06-18|53|233|19|1904|6|6|18|2|1904|19|233|Saturday|1904Q2|N|Y|N|2416633|2416784|2416284|2416559|N|N|N|N|N| +2416651|AAAAAAAALAAOECAA|1904-06-19|53|233|19|1904|0|6|19|2|1904|19|233|Sunday|1904Q2|N|N|N|2416633|2416784|2416285|2416560|N|N|N|N|N| +2416652|AAAAAAAAMAAOECAA|1904-06-20|53|233|19|1904|1|6|20|2|1904|19|233|Monday|1904Q2|N|N|N|2416633|2416784|2416286|2416561|N|N|N|N|N| +2416653|AAAAAAAANAAOECAA|1904-06-21|53|234|19|1904|2|6|21|2|1904|19|234|Tuesday|1904Q2|N|N|N|2416633|2416784|2416287|2416562|N|N|N|N|N| +2416654|AAAAAAAAOAAOECAA|1904-06-22|53|234|19|1904|3|6|22|2|1904|19|234|Wednesday|1904Q2|N|N|N|2416633|2416784|2416288|2416563|N|N|N|N|N| +2416655|AAAAAAAAPAAOECAA|1904-06-23|53|234|19|1904|4|6|23|2|1904|19|234|Thursday|1904Q2|N|N|N|2416633|2416784|2416289|2416564|N|N|N|N|N| +2416656|AAAAAAAAABAOECAA|1904-06-24|53|234|19|1904|5|6|24|2|1904|19|234|Friday|1904Q2|N|Y|N|2416633|2416784|2416290|2416565|N|N|N|N|N| +2416657|AAAAAAAABBAOECAA|1904-06-25|53|234|19|1904|6|6|25|2|1904|19|234|Saturday|1904Q2|N|Y|N|2416633|2416784|2416291|2416566|N|N|N|N|N| +2416658|AAAAAAAACBAOECAA|1904-06-26|53|234|19|1904|0|6|26|2|1904|19|234|Sunday|1904Q2|N|N|N|2416633|2416784|2416292|2416567|N|N|N|N|N| +2416659|AAAAAAAADBAOECAA|1904-06-27|53|234|19|1904|1|6|27|2|1904|19|234|Monday|1904Q2|N|N|N|2416633|2416784|2416293|2416568|N|N|N|N|N| +2416660|AAAAAAAAEBAOECAA|1904-06-28|53|235|19|1904|2|6|28|2|1904|19|235|Tuesday|1904Q2|N|N|N|2416633|2416784|2416294|2416569|N|N|N|N|N| +2416661|AAAAAAAAFBAOECAA|1904-06-29|53|235|19|1904|3|6|29|2|1904|19|235|Wednesday|1904Q2|N|N|N|2416633|2416784|2416295|2416570|N|N|N|N|N| +2416662|AAAAAAAAGBAOECAA|1904-06-30|53|235|19|1904|4|6|30|2|1904|19|235|Thursday|1904Q2|N|N|N|2416633|2416784|2416296|2416571|N|N|N|N|N| +2416663|AAAAAAAAHBAOECAA|1904-07-01|54|235|19|1904|5|7|1|3|1904|19|235|Friday|1904Q3|N|Y|N|2416663|2416844|2416297|2416572|N|N|N|N|N| +2416664|AAAAAAAAIBAOECAA|1904-07-02|54|235|19|1904|6|7|2|3|1904|19|235|Saturday|1904Q3|N|Y|N|2416663|2416844|2416298|2416573|N|N|N|N|N| +2416665|AAAAAAAAJBAOECAA|1904-07-03|54|235|19|1904|0|7|3|3|1904|19|235|Sunday|1904Q3|N|N|N|2416663|2416844|2416299|2416574|N|N|N|N|N| +2416666|AAAAAAAAKBAOECAA|1904-07-04|54|235|19|1904|1|7|4|3|1904|19|235|Monday|1904Q3|Y|N|N|2416663|2416844|2416300|2416575|N|N|N|N|N| +2416667|AAAAAAAALBAOECAA|1904-07-05|54|236|19|1904|2|7|5|3|1904|19|236|Tuesday|1904Q3|N|N|Y|2416663|2416844|2416301|2416576|N|N|N|N|N| +2416668|AAAAAAAAMBAOECAA|1904-07-06|54|236|19|1904|3|7|6|3|1904|19|236|Wednesday|1904Q3|N|N|N|2416663|2416844|2416302|2416577|N|N|N|N|N| +2416669|AAAAAAAANBAOECAA|1904-07-07|54|236|19|1904|4|7|7|3|1904|19|236|Thursday|1904Q3|N|N|N|2416663|2416844|2416303|2416578|N|N|N|N|N| +2416670|AAAAAAAAOBAOECAA|1904-07-08|54|236|19|1904|5|7|8|3|1904|19|236|Friday|1904Q3|N|Y|N|2416663|2416844|2416304|2416579|N|N|N|N|N| +2416671|AAAAAAAAPBAOECAA|1904-07-09|54|236|19|1904|6|7|9|3|1904|19|236|Saturday|1904Q3|N|Y|N|2416663|2416844|2416305|2416580|N|N|N|N|N| +2416672|AAAAAAAAACAOECAA|1904-07-10|54|236|19|1904|0|7|10|3|1904|19|236|Sunday|1904Q3|N|N|N|2416663|2416844|2416306|2416581|N|N|N|N|N| +2416673|AAAAAAAABCAOECAA|1904-07-11|54|236|19|1904|1|7|11|3|1904|19|236|Monday|1904Q3|N|N|N|2416663|2416844|2416307|2416582|N|N|N|N|N| +2416674|AAAAAAAACCAOECAA|1904-07-12|54|237|19|1904|2|7|12|3|1904|19|237|Tuesday|1904Q3|N|N|N|2416663|2416844|2416308|2416583|N|N|N|N|N| +2416675|AAAAAAAADCAOECAA|1904-07-13|54|237|19|1904|3|7|13|3|1904|19|237|Wednesday|1904Q3|N|N|N|2416663|2416844|2416309|2416584|N|N|N|N|N| +2416676|AAAAAAAAECAOECAA|1904-07-14|54|237|19|1904|4|7|14|3|1904|19|237|Thursday|1904Q3|N|N|N|2416663|2416844|2416310|2416585|N|N|N|N|N| +2416677|AAAAAAAAFCAOECAA|1904-07-15|54|237|19|1904|5|7|15|3|1904|19|237|Friday|1904Q3|N|Y|N|2416663|2416844|2416311|2416586|N|N|N|N|N| +2416678|AAAAAAAAGCAOECAA|1904-07-16|54|237|19|1904|6|7|16|3|1904|19|237|Saturday|1904Q3|N|Y|N|2416663|2416844|2416312|2416587|N|N|N|N|N| +2416679|AAAAAAAAHCAOECAA|1904-07-17|54|237|19|1904|0|7|17|3|1904|19|237|Sunday|1904Q3|N|N|N|2416663|2416844|2416313|2416588|N|N|N|N|N| +2416680|AAAAAAAAICAOECAA|1904-07-18|54|237|19|1904|1|7|18|3|1904|19|237|Monday|1904Q3|N|N|N|2416663|2416844|2416314|2416589|N|N|N|N|N| +2416681|AAAAAAAAJCAOECAA|1904-07-19|54|238|19|1904|2|7|19|3|1904|19|238|Tuesday|1904Q3|N|N|N|2416663|2416844|2416315|2416590|N|N|N|N|N| +2416682|AAAAAAAAKCAOECAA|1904-07-20|54|238|19|1904|3|7|20|3|1904|19|238|Wednesday|1904Q3|N|N|N|2416663|2416844|2416316|2416591|N|N|N|N|N| +2416683|AAAAAAAALCAOECAA|1904-07-21|54|238|19|1904|4|7|21|3|1904|19|238|Thursday|1904Q3|N|N|N|2416663|2416844|2416317|2416592|N|N|N|N|N| +2416684|AAAAAAAAMCAOECAA|1904-07-22|54|238|19|1904|5|7|22|3|1904|19|238|Friday|1904Q3|N|Y|N|2416663|2416844|2416318|2416593|N|N|N|N|N| +2416685|AAAAAAAANCAOECAA|1904-07-23|54|238|19|1904|6|7|23|3|1904|19|238|Saturday|1904Q3|N|Y|N|2416663|2416844|2416319|2416594|N|N|N|N|N| +2416686|AAAAAAAAOCAOECAA|1904-07-24|54|238|19|1904|0|7|24|3|1904|19|238|Sunday|1904Q3|N|N|N|2416663|2416844|2416320|2416595|N|N|N|N|N| +2416687|AAAAAAAAPCAOECAA|1904-07-25|54|238|19|1904|1|7|25|3|1904|19|238|Monday|1904Q3|N|N|N|2416663|2416844|2416321|2416596|N|N|N|N|N| +2416688|AAAAAAAAADAOECAA|1904-07-26|54|239|19|1904|2|7|26|3|1904|19|239|Tuesday|1904Q3|N|N|N|2416663|2416844|2416322|2416597|N|N|N|N|N| +2416689|AAAAAAAABDAOECAA|1904-07-27|54|239|19|1904|3|7|27|3|1904|19|239|Wednesday|1904Q3|N|N|N|2416663|2416844|2416323|2416598|N|N|N|N|N| +2416690|AAAAAAAACDAOECAA|1904-07-28|54|239|19|1904|4|7|28|3|1904|19|239|Thursday|1904Q3|N|N|N|2416663|2416844|2416324|2416599|N|N|N|N|N| +2416691|AAAAAAAADDAOECAA|1904-07-29|54|239|19|1904|5|7|29|3|1904|19|239|Friday|1904Q3|N|Y|N|2416663|2416844|2416325|2416600|N|N|N|N|N| +2416692|AAAAAAAAEDAOECAA|1904-07-30|54|239|19|1904|6|7|30|3|1904|19|239|Saturday|1904Q3|N|Y|N|2416663|2416844|2416326|2416601|N|N|N|N|N| +2416693|AAAAAAAAFDAOECAA|1904-07-31|54|239|19|1904|0|7|31|3|1904|19|239|Sunday|1904Q3|N|N|N|2416663|2416844|2416327|2416602|N|N|N|N|N| +2416694|AAAAAAAAGDAOECAA|1904-08-01|55|239|19|1904|1|8|1|3|1904|19|239|Monday|1904Q3|N|N|N|2416694|2416906|2416328|2416603|N|N|N|N|N| +2416695|AAAAAAAAHDAOECAA|1904-08-02|55|240|19|1904|2|8|2|3|1904|19|240|Tuesday|1904Q3|N|N|N|2416694|2416906|2416329|2416604|N|N|N|N|N| +2416696|AAAAAAAAIDAOECAA|1904-08-03|55|240|19|1904|3|8|3|3|1904|19|240|Wednesday|1904Q3|N|N|N|2416694|2416906|2416330|2416605|N|N|N|N|N| +2416697|AAAAAAAAJDAOECAA|1904-08-04|55|240|19|1904|4|8|4|3|1904|19|240|Thursday|1904Q3|N|N|N|2416694|2416906|2416331|2416606|N|N|N|N|N| +2416698|AAAAAAAAKDAOECAA|1904-08-05|55|240|19|1904|5|8|5|3|1904|19|240|Friday|1904Q3|N|Y|N|2416694|2416906|2416332|2416607|N|N|N|N|N| +2416699|AAAAAAAALDAOECAA|1904-08-06|55|240|19|1904|6|8|6|3|1904|19|240|Saturday|1904Q3|N|Y|N|2416694|2416906|2416333|2416608|N|N|N|N|N| +2416700|AAAAAAAAMDAOECAA|1904-08-07|55|240|19|1904|0|8|7|3|1904|19|240|Sunday|1904Q3|N|N|N|2416694|2416906|2416334|2416609|N|N|N|N|N| +2416701|AAAAAAAANDAOECAA|1904-08-08|55|240|19|1904|1|8|8|3|1904|19|240|Monday|1904Q3|N|N|N|2416694|2416906|2416335|2416610|N|N|N|N|N| +2416702|AAAAAAAAODAOECAA|1904-08-09|55|241|19|1904|2|8|9|3|1904|19|241|Tuesday|1904Q3|N|N|N|2416694|2416906|2416336|2416611|N|N|N|N|N| +2416703|AAAAAAAAPDAOECAA|1904-08-10|55|241|19|1904|3|8|10|3|1904|19|241|Wednesday|1904Q3|N|N|N|2416694|2416906|2416337|2416612|N|N|N|N|N| +2416704|AAAAAAAAAEAOECAA|1904-08-11|55|241|19|1904|4|8|11|3|1904|19|241|Thursday|1904Q3|N|N|N|2416694|2416906|2416338|2416613|N|N|N|N|N| +2416705|AAAAAAAABEAOECAA|1904-08-12|55|241|19|1904|5|8|12|3|1904|19|241|Friday|1904Q3|N|Y|N|2416694|2416906|2416339|2416614|N|N|N|N|N| +2416706|AAAAAAAACEAOECAA|1904-08-13|55|241|19|1904|6|8|13|3|1904|19|241|Saturday|1904Q3|N|Y|N|2416694|2416906|2416340|2416615|N|N|N|N|N| +2416707|AAAAAAAADEAOECAA|1904-08-14|55|241|19|1904|0|8|14|3|1904|19|241|Sunday|1904Q3|N|N|N|2416694|2416906|2416341|2416616|N|N|N|N|N| +2416708|AAAAAAAAEEAOECAA|1904-08-15|55|241|19|1904|1|8|15|3|1904|19|241|Monday|1904Q3|N|N|N|2416694|2416906|2416342|2416617|N|N|N|N|N| +2416709|AAAAAAAAFEAOECAA|1904-08-16|55|242|19|1904|2|8|16|3|1904|19|242|Tuesday|1904Q3|N|N|N|2416694|2416906|2416343|2416618|N|N|N|N|N| +2416710|AAAAAAAAGEAOECAA|1904-08-17|55|242|19|1904|3|8|17|3|1904|19|242|Wednesday|1904Q3|N|N|N|2416694|2416906|2416344|2416619|N|N|N|N|N| +2416711|AAAAAAAAHEAOECAA|1904-08-18|55|242|19|1904|4|8|18|3|1904|19|242|Thursday|1904Q3|N|N|N|2416694|2416906|2416345|2416620|N|N|N|N|N| +2416712|AAAAAAAAIEAOECAA|1904-08-19|55|242|19|1904|5|8|19|3|1904|19|242|Friday|1904Q3|N|Y|N|2416694|2416906|2416346|2416621|N|N|N|N|N| +2416713|AAAAAAAAJEAOECAA|1904-08-20|55|242|19|1904|6|8|20|3|1904|19|242|Saturday|1904Q3|N|Y|N|2416694|2416906|2416347|2416622|N|N|N|N|N| +2416714|AAAAAAAAKEAOECAA|1904-08-21|55|242|19|1904|0|8|21|3|1904|19|242|Sunday|1904Q3|N|N|N|2416694|2416906|2416348|2416623|N|N|N|N|N| +2416715|AAAAAAAALEAOECAA|1904-08-22|55|242|19|1904|1|8|22|3|1904|19|242|Monday|1904Q3|N|N|N|2416694|2416906|2416349|2416624|N|N|N|N|N| +2416716|AAAAAAAAMEAOECAA|1904-08-23|55|243|19|1904|2|8|23|3|1904|19|243|Tuesday|1904Q3|N|N|N|2416694|2416906|2416350|2416625|N|N|N|N|N| +2416717|AAAAAAAANEAOECAA|1904-08-24|55|243|19|1904|3|8|24|3|1904|19|243|Wednesday|1904Q3|N|N|N|2416694|2416906|2416351|2416626|N|N|N|N|N| +2416718|AAAAAAAAOEAOECAA|1904-08-25|55|243|19|1904|4|8|25|3|1904|19|243|Thursday|1904Q3|N|N|N|2416694|2416906|2416352|2416627|N|N|N|N|N| +2416719|AAAAAAAAPEAOECAA|1904-08-26|55|243|19|1904|5|8|26|3|1904|19|243|Friday|1904Q3|N|Y|N|2416694|2416906|2416353|2416628|N|N|N|N|N| +2416720|AAAAAAAAAFAOECAA|1904-08-27|55|243|19|1904|6|8|27|3|1904|19|243|Saturday|1904Q3|N|Y|N|2416694|2416906|2416354|2416629|N|N|N|N|N| +2416721|AAAAAAAABFAOECAA|1904-08-28|55|243|19|1904|0|8|28|3|1904|19|243|Sunday|1904Q3|N|N|N|2416694|2416906|2416355|2416630|N|N|N|N|N| +2416722|AAAAAAAACFAOECAA|1904-08-29|55|243|19|1904|1|8|29|3|1904|19|243|Monday|1904Q3|N|N|N|2416694|2416906|2416356|2416631|N|N|N|N|N| +2416723|AAAAAAAADFAOECAA|1904-08-30|55|244|19|1904|2|8|30|3|1904|19|244|Tuesday|1904Q3|N|N|N|2416694|2416906|2416357|2416632|N|N|N|N|N| +2416724|AAAAAAAAEFAOECAA|1904-08-31|55|244|19|1904|3|8|31|3|1904|19|244|Wednesday|1904Q3|N|N|N|2416694|2416906|2416358|2416633|N|N|N|N|N| +2416725|AAAAAAAAFFAOECAA|1904-09-01|56|244|20|1904|4|9|1|3|1904|20|244|Thursday|1904Q3|N|N|N|2416725|2416968|2416359|2416634|N|N|N|N|N| +2416726|AAAAAAAAGFAOECAA|1904-09-02|56|244|20|1904|5|9|2|3|1904|20|244|Friday|1904Q3|N|Y|N|2416725|2416968|2416360|2416635|N|N|N|N|N| +2416727|AAAAAAAAHFAOECAA|1904-09-03|56|244|20|1904|6|9|3|3|1904|20|244|Saturday|1904Q3|N|Y|N|2416725|2416968|2416361|2416636|N|N|N|N|N| +2416728|AAAAAAAAIFAOECAA|1904-09-04|56|244|20|1904|0|9|4|3|1904|20|244|Sunday|1904Q3|N|N|N|2416725|2416968|2416362|2416637|N|N|N|N|N| +2416729|AAAAAAAAJFAOECAA|1904-09-05|56|244|20|1904|1|9|5|3|1904|20|244|Monday|1904Q3|N|N|N|2416725|2416968|2416363|2416638|N|N|N|N|N| +2416730|AAAAAAAAKFAOECAA|1904-09-06|56|245|20|1904|2|9|6|3|1904|20|245|Tuesday|1904Q3|N|N|N|2416725|2416968|2416364|2416639|N|N|N|N|N| +2416731|AAAAAAAALFAOECAA|1904-09-07|56|245|20|1904|3|9|7|3|1904|20|245|Wednesday|1904Q3|N|N|N|2416725|2416968|2416365|2416640|N|N|N|N|N| +2416732|AAAAAAAAMFAOECAA|1904-09-08|56|245|20|1904|4|9|8|3|1904|20|245|Thursday|1904Q3|N|N|N|2416725|2416968|2416366|2416641|N|N|N|N|N| +2416733|AAAAAAAANFAOECAA|1904-09-09|56|245|20|1904|5|9|9|3|1904|20|245|Friday|1904Q3|N|Y|N|2416725|2416968|2416367|2416642|N|N|N|N|N| +2416734|AAAAAAAAOFAOECAA|1904-09-10|56|245|20|1904|6|9|10|3|1904|20|245|Saturday|1904Q3|N|Y|N|2416725|2416968|2416368|2416643|N|N|N|N|N| +2416735|AAAAAAAAPFAOECAA|1904-09-11|56|245|20|1904|0|9|11|3|1904|20|245|Sunday|1904Q3|N|N|N|2416725|2416968|2416369|2416644|N|N|N|N|N| +2416736|AAAAAAAAAGAOECAA|1904-09-12|56|245|20|1904|1|9|12|3|1904|20|245|Monday|1904Q3|N|N|N|2416725|2416968|2416370|2416645|N|N|N|N|N| +2416737|AAAAAAAABGAOECAA|1904-09-13|56|246|20|1904|2|9|13|3|1904|20|246|Tuesday|1904Q3|N|N|N|2416725|2416968|2416371|2416646|N|N|N|N|N| +2416738|AAAAAAAACGAOECAA|1904-09-14|56|246|20|1904|3|9|14|3|1904|20|246|Wednesday|1904Q3|N|N|N|2416725|2416968|2416372|2416647|N|N|N|N|N| +2416739|AAAAAAAADGAOECAA|1904-09-15|56|246|20|1904|4|9|15|3|1904|20|246|Thursday|1904Q3|N|N|N|2416725|2416968|2416373|2416648|N|N|N|N|N| +2416740|AAAAAAAAEGAOECAA|1904-09-16|56|246|20|1904|5|9|16|3|1904|20|246|Friday|1904Q3|N|Y|N|2416725|2416968|2416374|2416649|N|N|N|N|N| +2416741|AAAAAAAAFGAOECAA|1904-09-17|56|246|20|1904|6|9|17|3|1904|20|246|Saturday|1904Q3|N|Y|N|2416725|2416968|2416375|2416650|N|N|N|N|N| +2416742|AAAAAAAAGGAOECAA|1904-09-18|56|246|20|1904|0|9|18|3|1904|20|246|Sunday|1904Q3|N|N|N|2416725|2416968|2416376|2416651|N|N|N|N|N| +2416743|AAAAAAAAHGAOECAA|1904-09-19|56|246|20|1904|1|9|19|3|1904|20|246|Monday|1904Q3|N|N|N|2416725|2416968|2416377|2416652|N|N|N|N|N| +2416744|AAAAAAAAIGAOECAA|1904-09-20|56|247|20|1904|2|9|20|3|1904|20|247|Tuesday|1904Q3|N|N|N|2416725|2416968|2416378|2416653|N|N|N|N|N| +2416745|AAAAAAAAJGAOECAA|1904-09-21|56|247|20|1904|3|9|21|3|1904|20|247|Wednesday|1904Q3|N|N|N|2416725|2416968|2416379|2416654|N|N|N|N|N| +2416746|AAAAAAAAKGAOECAA|1904-09-22|56|247|20|1904|4|9|22|3|1904|20|247|Thursday|1904Q3|N|N|N|2416725|2416968|2416380|2416655|N|N|N|N|N| +2416747|AAAAAAAALGAOECAA|1904-09-23|56|247|20|1904|5|9|23|3|1904|20|247|Friday|1904Q3|N|Y|N|2416725|2416968|2416381|2416656|N|N|N|N|N| +2416748|AAAAAAAAMGAOECAA|1904-09-24|56|247|20|1904|6|9|24|3|1904|20|247|Saturday|1904Q3|N|Y|N|2416725|2416968|2416382|2416657|N|N|N|N|N| +2416749|AAAAAAAANGAOECAA|1904-09-25|56|247|20|1904|0|9|25|3|1904|20|247|Sunday|1904Q3|N|N|N|2416725|2416968|2416383|2416658|N|N|N|N|N| +2416750|AAAAAAAAOGAOECAA|1904-09-26|56|247|20|1904|1|9|26|3|1904|20|247|Monday|1904Q3|N|N|N|2416725|2416968|2416384|2416659|N|N|N|N|N| +2416751|AAAAAAAAPGAOECAA|1904-09-27|56|248|20|1904|2|9|27|3|1904|20|248|Tuesday|1904Q3|N|N|N|2416725|2416968|2416385|2416660|N|N|N|N|N| +2416752|AAAAAAAAAHAOECAA|1904-09-28|56|248|20|1904|3|9|28|3|1904|20|248|Wednesday|1904Q3|N|N|N|2416725|2416968|2416386|2416661|N|N|N|N|N| +2416753|AAAAAAAABHAOECAA|1904-09-29|56|248|20|1904|4|9|29|3|1904|20|248|Thursday|1904Q3|N|N|N|2416725|2416968|2416387|2416662|N|N|N|N|N| +2416754|AAAAAAAACHAOECAA|1904-09-30|56|248|20|1904|5|9|30|3|1904|20|248|Friday|1904Q3|N|Y|N|2416725|2416968|2416388|2416663|N|N|N|N|N| +2416755|AAAAAAAADHAOECAA|1904-10-01|57|248|20|1904|6|10|1|4|1904|20|248|Saturday|1904Q4|N|Y|N|2416755|2417028|2416389|2416663|N|N|N|N|N| +2416756|AAAAAAAAEHAOECAA|1904-10-02|57|248|20|1904|0|10|2|4|1904|20|248|Sunday|1904Q4|N|N|N|2416755|2417028|2416390|2416664|N|N|N|N|N| +2416757|AAAAAAAAFHAOECAA|1904-10-03|57|248|20|1904|1|10|3|4|1904|20|248|Monday|1904Q4|N|N|N|2416755|2417028|2416391|2416665|N|N|N|N|N| +2416758|AAAAAAAAGHAOECAA|1904-10-04|57|249|20|1904|2|10|4|4|1904|20|249|Tuesday|1904Q4|N|N|N|2416755|2417028|2416392|2416666|N|N|N|N|N| +2416759|AAAAAAAAHHAOECAA|1904-10-05|57|249|20|1904|3|10|5|4|1904|20|249|Wednesday|1904Q4|N|N|N|2416755|2417028|2416393|2416667|N|N|N|N|N| +2416760|AAAAAAAAIHAOECAA|1904-10-06|57|249|20|1904|4|10|6|4|1904|20|249|Thursday|1904Q4|N|N|N|2416755|2417028|2416394|2416668|N|N|N|N|N| +2416761|AAAAAAAAJHAOECAA|1904-10-07|57|249|20|1904|5|10|7|4|1904|20|249|Friday|1904Q4|N|Y|N|2416755|2417028|2416395|2416669|N|N|N|N|N| +2416762|AAAAAAAAKHAOECAA|1904-10-08|57|249|20|1904|6|10|8|4|1904|20|249|Saturday|1904Q4|N|Y|N|2416755|2417028|2416396|2416670|N|N|N|N|N| +2416763|AAAAAAAALHAOECAA|1904-10-09|57|249|20|1904|0|10|9|4|1904|20|249|Sunday|1904Q4|N|N|N|2416755|2417028|2416397|2416671|N|N|N|N|N| +2416764|AAAAAAAAMHAOECAA|1904-10-10|57|249|20|1904|1|10|10|4|1904|20|249|Monday|1904Q4|N|N|N|2416755|2417028|2416398|2416672|N|N|N|N|N| +2416765|AAAAAAAANHAOECAA|1904-10-11|57|250|20|1904|2|10|11|4|1904|20|250|Tuesday|1904Q4|N|N|N|2416755|2417028|2416399|2416673|N|N|N|N|N| +2416766|AAAAAAAAOHAOECAA|1904-10-12|57|250|20|1904|3|10|12|4|1904|20|250|Wednesday|1904Q4|N|N|N|2416755|2417028|2416400|2416674|N|N|N|N|N| +2416767|AAAAAAAAPHAOECAA|1904-10-13|57|250|20|1904|4|10|13|4|1904|20|250|Thursday|1904Q4|N|N|N|2416755|2417028|2416401|2416675|N|N|N|N|N| +2416768|AAAAAAAAAIAOECAA|1904-10-14|57|250|20|1904|5|10|14|4|1904|20|250|Friday|1904Q4|N|Y|N|2416755|2417028|2416402|2416676|N|N|N|N|N| +2416769|AAAAAAAABIAOECAA|1904-10-15|57|250|20|1904|6|10|15|4|1904|20|250|Saturday|1904Q4|N|Y|N|2416755|2417028|2416403|2416677|N|N|N|N|N| +2416770|AAAAAAAACIAOECAA|1904-10-16|57|250|20|1904|0|10|16|4|1904|20|250|Sunday|1904Q4|N|N|N|2416755|2417028|2416404|2416678|N|N|N|N|N| +2416771|AAAAAAAADIAOECAA|1904-10-17|57|250|20|1904|1|10|17|4|1904|20|250|Monday|1904Q4|N|N|N|2416755|2417028|2416405|2416679|N|N|N|N|N| +2416772|AAAAAAAAEIAOECAA|1904-10-18|57|251|20|1904|2|10|18|4|1904|20|251|Tuesday|1904Q4|N|N|N|2416755|2417028|2416406|2416680|N|N|N|N|N| +2416773|AAAAAAAAFIAOECAA|1904-10-19|57|251|20|1904|3|10|19|4|1904|20|251|Wednesday|1904Q4|N|N|N|2416755|2417028|2416407|2416681|N|N|N|N|N| +2416774|AAAAAAAAGIAOECAA|1904-10-20|57|251|20|1904|4|10|20|4|1904|20|251|Thursday|1904Q4|N|N|N|2416755|2417028|2416408|2416682|N|N|N|N|N| +2416775|AAAAAAAAHIAOECAA|1904-10-21|57|251|20|1904|5|10|21|4|1904|20|251|Friday|1904Q4|N|Y|N|2416755|2417028|2416409|2416683|N|N|N|N|N| +2416776|AAAAAAAAIIAOECAA|1904-10-22|57|251|20|1904|6|10|22|4|1904|20|251|Saturday|1904Q4|N|Y|N|2416755|2417028|2416410|2416684|N|N|N|N|N| +2416777|AAAAAAAAJIAOECAA|1904-10-23|57|251|20|1904|0|10|23|4|1904|20|251|Sunday|1904Q4|N|N|N|2416755|2417028|2416411|2416685|N|N|N|N|N| +2416778|AAAAAAAAKIAOECAA|1904-10-24|57|251|20|1904|1|10|24|4|1904|20|251|Monday|1904Q4|N|N|N|2416755|2417028|2416412|2416686|N|N|N|N|N| +2416779|AAAAAAAALIAOECAA|1904-10-25|57|252|20|1904|2|10|25|4|1904|20|252|Tuesday|1904Q4|N|N|N|2416755|2417028|2416413|2416687|N|N|N|N|N| +2416780|AAAAAAAAMIAOECAA|1904-10-26|57|252|20|1904|3|10|26|4|1904|20|252|Wednesday|1904Q4|N|N|N|2416755|2417028|2416414|2416688|N|N|N|N|N| +2416781|AAAAAAAANIAOECAA|1904-10-27|57|252|20|1904|4|10|27|4|1904|20|252|Thursday|1904Q4|N|N|N|2416755|2417028|2416415|2416689|N|N|N|N|N| +2416782|AAAAAAAAOIAOECAA|1904-10-28|57|252|20|1904|5|10|28|4|1904|20|252|Friday|1904Q4|N|Y|N|2416755|2417028|2416416|2416690|N|N|N|N|N| +2416783|AAAAAAAAPIAOECAA|1904-10-29|57|252|20|1904|6|10|29|4|1904|20|252|Saturday|1904Q4|N|Y|N|2416755|2417028|2416417|2416691|N|N|N|N|N| +2416784|AAAAAAAAAJAOECAA|1904-10-30|57|252|20|1904|0|10|30|4|1904|20|252|Sunday|1904Q4|N|N|N|2416755|2417028|2416418|2416692|N|N|N|N|N| +2416785|AAAAAAAABJAOECAA|1904-10-31|57|252|20|1904|1|10|31|4|1904|20|252|Monday|1904Q4|N|N|N|2416755|2417028|2416419|2416693|N|N|N|N|N| +2416786|AAAAAAAACJAOECAA|1904-11-01|58|253|20|1904|2|11|1|4|1904|20|253|Tuesday|1904Q4|N|N|N|2416786|2417090|2416420|2416694|N|N|N|N|N| +2416787|AAAAAAAADJAOECAA|1904-11-02|58|253|20|1904|3|11|2|4|1904|20|253|Wednesday|1904Q4|N|N|N|2416786|2417090|2416421|2416695|N|N|N|N|N| +2416788|AAAAAAAAEJAOECAA|1904-11-03|58|253|20|1904|4|11|3|4|1904|20|253|Thursday|1904Q4|N|N|N|2416786|2417090|2416422|2416696|N|N|N|N|N| +2416789|AAAAAAAAFJAOECAA|1904-11-04|58|253|20|1904|5|11|4|4|1904|20|253|Friday|1904Q4|N|Y|N|2416786|2417090|2416423|2416697|N|N|N|N|N| +2416790|AAAAAAAAGJAOECAA|1904-11-05|58|253|20|1904|6|11|5|4|1904|20|253|Saturday|1904Q4|N|Y|N|2416786|2417090|2416424|2416698|N|N|N|N|N| +2416791|AAAAAAAAHJAOECAA|1904-11-06|58|253|20|1904|0|11|6|4|1904|20|253|Sunday|1904Q4|N|N|N|2416786|2417090|2416425|2416699|N|N|N|N|N| +2416792|AAAAAAAAIJAOECAA|1904-11-07|58|253|20|1904|1|11|7|4|1904|20|253|Monday|1904Q4|N|N|N|2416786|2417090|2416426|2416700|N|N|N|N|N| +2416793|AAAAAAAAJJAOECAA|1904-11-08|58|254|20|1904|2|11|8|4|1904|20|254|Tuesday|1904Q4|N|N|N|2416786|2417090|2416427|2416701|N|N|N|N|N| +2416794|AAAAAAAAKJAOECAA|1904-11-09|58|254|20|1904|3|11|9|4|1904|20|254|Wednesday|1904Q4|N|N|N|2416786|2417090|2416428|2416702|N|N|N|N|N| +2416795|AAAAAAAALJAOECAA|1904-11-10|58|254|20|1904|4|11|10|4|1904|20|254|Thursday|1904Q4|N|N|N|2416786|2417090|2416429|2416703|N|N|N|N|N| +2416796|AAAAAAAAMJAOECAA|1904-11-11|58|254|20|1904|5|11|11|4|1904|20|254|Friday|1904Q4|N|Y|N|2416786|2417090|2416430|2416704|N|N|N|N|N| +2416797|AAAAAAAANJAOECAA|1904-11-12|58|254|20|1904|6|11|12|4|1904|20|254|Saturday|1904Q4|N|Y|N|2416786|2417090|2416431|2416705|N|N|N|N|N| +2416798|AAAAAAAAOJAOECAA|1904-11-13|58|254|20|1904|0|11|13|4|1904|20|254|Sunday|1904Q4|N|N|N|2416786|2417090|2416432|2416706|N|N|N|N|N| +2416799|AAAAAAAAPJAOECAA|1904-11-14|58|254|20|1904|1|11|14|4|1904|20|254|Monday|1904Q4|N|N|N|2416786|2417090|2416433|2416707|N|N|N|N|N| +2416800|AAAAAAAAAKAOECAA|1904-11-15|58|255|20|1904|2|11|15|4|1904|20|255|Tuesday|1904Q4|N|N|N|2416786|2417090|2416434|2416708|N|N|N|N|N| +2416801|AAAAAAAABKAOECAA|1904-11-16|58|255|20|1904|3|11|16|4|1904|20|255|Wednesday|1904Q4|N|N|N|2416786|2417090|2416435|2416709|N|N|N|N|N| +2416802|AAAAAAAACKAOECAA|1904-11-17|58|255|20|1904|4|11|17|4|1904|20|255|Thursday|1904Q4|N|N|N|2416786|2417090|2416436|2416710|N|N|N|N|N| +2416803|AAAAAAAADKAOECAA|1904-11-18|58|255|20|1904|5|11|18|4|1904|20|255|Friday|1904Q4|N|Y|N|2416786|2417090|2416437|2416711|N|N|N|N|N| +2416804|AAAAAAAAEKAOECAA|1904-11-19|58|255|20|1904|6|11|19|4|1904|20|255|Saturday|1904Q4|N|Y|N|2416786|2417090|2416438|2416712|N|N|N|N|N| +2416805|AAAAAAAAFKAOECAA|1904-11-20|58|255|20|1904|0|11|20|4|1904|20|255|Sunday|1904Q4|N|N|N|2416786|2417090|2416439|2416713|N|N|N|N|N| +2416806|AAAAAAAAGKAOECAA|1904-11-21|58|255|20|1904|1|11|21|4|1904|20|255|Monday|1904Q4|N|N|N|2416786|2417090|2416440|2416714|N|N|N|N|N| +2416807|AAAAAAAAHKAOECAA|1904-11-22|58|256|20|1904|2|11|22|4|1904|20|256|Tuesday|1904Q4|N|N|N|2416786|2417090|2416441|2416715|N|N|N|N|N| +2416808|AAAAAAAAIKAOECAA|1904-11-23|58|256|20|1904|3|11|23|4|1904|20|256|Wednesday|1904Q4|N|N|N|2416786|2417090|2416442|2416716|N|N|N|N|N| +2416809|AAAAAAAAJKAOECAA|1904-11-24|58|256|20|1904|4|11|24|4|1904|20|256|Thursday|1904Q4|N|N|N|2416786|2417090|2416443|2416717|N|N|N|N|N| +2416810|AAAAAAAAKKAOECAA|1904-11-25|58|256|20|1904|5|11|25|4|1904|20|256|Friday|1904Q4|N|Y|N|2416786|2417090|2416444|2416718|N|N|N|N|N| +2416811|AAAAAAAALKAOECAA|1904-11-26|58|256|20|1904|6|11|26|4|1904|20|256|Saturday|1904Q4|N|Y|N|2416786|2417090|2416445|2416719|N|N|N|N|N| +2416812|AAAAAAAAMKAOECAA|1904-11-27|58|256|20|1904|0|11|27|4|1904|20|256|Sunday|1904Q4|N|N|N|2416786|2417090|2416446|2416720|N|N|N|N|N| +2416813|AAAAAAAANKAOECAA|1904-11-28|58|256|20|1904|1|11|28|4|1904|20|256|Monday|1904Q4|N|N|N|2416786|2417090|2416447|2416721|N|N|N|N|N| +2416814|AAAAAAAAOKAOECAA|1904-11-29|58|257|20|1904|2|11|29|4|1904|20|257|Tuesday|1904Q4|N|N|N|2416786|2417090|2416448|2416722|N|N|N|N|N| +2416815|AAAAAAAAPKAOECAA|1904-11-30|58|257|20|1904|3|11|30|4|1904|20|257|Wednesday|1904Q4|N|N|N|2416786|2417090|2416449|2416723|N|N|N|N|N| +2416816|AAAAAAAAALAOECAA|1904-12-01|59|257|21|1904|4|12|1|4|1904|21|257|Thursday|1904Q4|N|N|N|2416816|2417150|2416450|2416724|N|N|N|N|N| +2416817|AAAAAAAABLAOECAA|1904-12-02|59|257|21|1904|5|12|2|4|1904|21|257|Friday|1904Q4|N|Y|N|2416816|2417150|2416451|2416725|N|N|N|N|N| +2416818|AAAAAAAACLAOECAA|1904-12-03|59|257|21|1904|6|12|3|4|1904|21|257|Saturday|1904Q4|N|Y|N|2416816|2417150|2416452|2416726|N|N|N|N|N| +2416819|AAAAAAAADLAOECAA|1904-12-04|59|257|21|1904|0|12|4|4|1904|21|257|Sunday|1904Q4|N|N|N|2416816|2417150|2416453|2416727|N|N|N|N|N| +2416820|AAAAAAAAELAOECAA|1904-12-05|59|257|21|1904|1|12|5|4|1904|21|257|Monday|1904Q4|N|N|N|2416816|2417150|2416454|2416728|N|N|N|N|N| +2416821|AAAAAAAAFLAOECAA|1904-12-06|59|258|21|1904|2|12|6|4|1904|21|258|Tuesday|1904Q4|N|N|N|2416816|2417150|2416455|2416729|N|N|N|N|N| +2416822|AAAAAAAAGLAOECAA|1904-12-07|59|258|21|1904|3|12|7|4|1904|21|258|Wednesday|1904Q4|N|N|N|2416816|2417150|2416456|2416730|N|N|N|N|N| +2416823|AAAAAAAAHLAOECAA|1904-12-08|59|258|21|1904|4|12|8|4|1904|21|258|Thursday|1904Q4|N|N|N|2416816|2417150|2416457|2416731|N|N|N|N|N| +2416824|AAAAAAAAILAOECAA|1904-12-09|59|258|21|1904|5|12|9|4|1904|21|258|Friday|1904Q4|N|Y|N|2416816|2417150|2416458|2416732|N|N|N|N|N| +2416825|AAAAAAAAJLAOECAA|1904-12-10|59|258|21|1904|6|12|10|4|1904|21|258|Saturday|1904Q4|N|Y|N|2416816|2417150|2416459|2416733|N|N|N|N|N| +2416826|AAAAAAAAKLAOECAA|1904-12-11|59|258|21|1904|0|12|11|4|1904|21|258|Sunday|1904Q4|N|N|N|2416816|2417150|2416460|2416734|N|N|N|N|N| +2416827|AAAAAAAALLAOECAA|1904-12-12|59|258|21|1904|1|12|12|4|1904|21|258|Monday|1904Q4|N|N|N|2416816|2417150|2416461|2416735|N|N|N|N|N| +2416828|AAAAAAAAMLAOECAA|1904-12-13|59|259|21|1904|2|12|13|4|1904|21|259|Tuesday|1904Q4|N|N|N|2416816|2417150|2416462|2416736|N|N|N|N|N| +2416829|AAAAAAAANLAOECAA|1904-12-14|59|259|21|1904|3|12|14|4|1904|21|259|Wednesday|1904Q4|N|N|N|2416816|2417150|2416463|2416737|N|N|N|N|N| +2416830|AAAAAAAAOLAOECAA|1904-12-15|59|259|21|1904|4|12|15|4|1904|21|259|Thursday|1904Q4|N|N|N|2416816|2417150|2416464|2416738|N|N|N|N|N| +2416831|AAAAAAAAPLAOECAA|1904-12-16|59|259|21|1904|5|12|16|4|1904|21|259|Friday|1904Q4|N|Y|N|2416816|2417150|2416465|2416739|N|N|N|N|N| +2416832|AAAAAAAAAMAOECAA|1904-12-17|59|259|21|1904|6|12|17|4|1904|21|259|Saturday|1904Q4|N|Y|N|2416816|2417150|2416466|2416740|N|N|N|N|N| +2416833|AAAAAAAABMAOECAA|1904-12-18|59|259|21|1904|0|12|18|4|1904|21|259|Sunday|1904Q4|N|N|N|2416816|2417150|2416467|2416741|N|N|N|N|N| +2416834|AAAAAAAACMAOECAA|1904-12-19|59|259|21|1904|1|12|19|4|1904|21|259|Monday|1904Q4|N|N|N|2416816|2417150|2416468|2416742|N|N|N|N|N| +2416835|AAAAAAAADMAOECAA|1904-12-20|59|260|21|1904|2|12|20|4|1904|21|260|Tuesday|1904Q4|N|N|N|2416816|2417150|2416469|2416743|N|N|N|N|N| +2416836|AAAAAAAAEMAOECAA|1904-12-21|59|260|21|1904|3|12|21|4|1904|21|260|Wednesday|1904Q4|N|N|N|2416816|2417150|2416470|2416744|N|N|N|N|N| +2416837|AAAAAAAAFMAOECAA|1904-12-22|59|260|21|1904|4|12|22|4|1904|21|260|Thursday|1904Q4|N|N|N|2416816|2417150|2416471|2416745|N|N|N|N|N| +2416838|AAAAAAAAGMAOECAA|1904-12-23|59|260|21|1904|5|12|23|4|1904|21|260|Friday|1904Q4|N|Y|N|2416816|2417150|2416472|2416746|N|N|N|N|N| +2416839|AAAAAAAAHMAOECAA|1904-12-24|59|260|21|1904|6|12|24|4|1904|21|260|Saturday|1904Q4|N|Y|N|2416816|2417150|2416473|2416747|N|N|N|N|N| +2416840|AAAAAAAAIMAOECAA|1904-12-25|59|260|21|1904|0|12|25|4|1904|21|260|Sunday|1904Q4|Y|N|N|2416816|2417150|2416474|2416748|N|N|N|N|N| +2416841|AAAAAAAAJMAOECAA|1904-12-26|59|260|21|1904|1|12|26|4|1904|21|260|Monday|1904Q4|N|N|Y|2416816|2417150|2416475|2416749|N|N|N|N|N| +2416842|AAAAAAAAKMAOECAA|1904-12-27|59|261|21|1904|2|12|27|4|1904|21|261|Tuesday|1904Q4|N|N|N|2416816|2417150|2416476|2416750|N|N|N|N|N| +2416843|AAAAAAAALMAOECAA|1904-12-28|59|261|21|1904|3|12|28|4|1904|21|261|Wednesday|1904Q4|N|N|N|2416816|2417150|2416477|2416751|N|N|N|N|N| +2416844|AAAAAAAAMMAOECAA|1904-12-29|59|261|21|1904|4|12|29|4|1904|21|261|Thursday|1904Q4|N|N|N|2416816|2417150|2416478|2416752|N|N|N|N|N| +2416845|AAAAAAAANMAOECAA|1904-12-30|59|261|21|1904|5|12|30|4|1904|21|261|Friday|1904Q4|N|Y|N|2416816|2417150|2416479|2416753|N|N|N|N|N| +2416846|AAAAAAAAOMAOECAA|1904-12-31|59|261|21|1904|6|12|31|4|1904|21|261|Saturday|1904Q4|Y|Y|N|2416816|2417150|2416480|2416754|N|N|N|N|N| +2416847|AAAAAAAAPMAOECAA|1905-01-01|60|261|21|1905|0|1|1|1|1905|21|261|Sunday|1905Q1|Y|N|Y|2416847|2416846|2416481|2416755|N|N|N|N|N| +2416848|AAAAAAAAANAOECAA|1905-01-02|60|261|21|1905|1|1|2|1|1905|21|261|Monday|1905Q1|N|N|Y|2416847|2416846|2416482|2416756|N|N|N|N|N| +2416849|AAAAAAAABNAOECAA|1905-01-03|60|262|21|1905|2|1|3|1|1905|21|262|Tuesday|1905Q1|N|N|N|2416847|2416846|2416483|2416757|N|N|N|N|N| +2416850|AAAAAAAACNAOECAA|1905-01-04|60|262|21|1905|3|1|4|1|1905|21|262|Wednesday|1905Q1|N|N|N|2416847|2416846|2416484|2416758|N|N|N|N|N| +2416851|AAAAAAAADNAOECAA|1905-01-05|60|262|21|1905|4|1|5|1|1905|21|262|Thursday|1905Q1|N|N|N|2416847|2416846|2416485|2416759|N|N|N|N|N| +2416852|AAAAAAAAENAOECAA|1905-01-06|60|262|21|1905|5|1|6|1|1905|21|262|Friday|1905Q1|N|Y|N|2416847|2416846|2416486|2416760|N|N|N|N|N| +2416853|AAAAAAAAFNAOECAA|1905-01-07|60|262|21|1905|6|1|7|1|1905|21|262|Saturday|1905Q1|N|Y|N|2416847|2416846|2416487|2416761|N|N|N|N|N| +2416854|AAAAAAAAGNAOECAA|1905-01-08|60|262|21|1905|0|1|8|1|1905|21|262|Sunday|1905Q1|N|N|N|2416847|2416846|2416488|2416762|N|N|N|N|N| +2416855|AAAAAAAAHNAOECAA|1905-01-09|60|262|21|1905|1|1|9|1|1905|21|262|Monday|1905Q1|N|N|N|2416847|2416846|2416489|2416763|N|N|N|N|N| +2416856|AAAAAAAAINAOECAA|1905-01-10|60|263|21|1905|2|1|10|1|1905|21|263|Tuesday|1905Q1|N|N|N|2416847|2416846|2416490|2416764|N|N|N|N|N| +2416857|AAAAAAAAJNAOECAA|1905-01-11|60|263|21|1905|3|1|11|1|1905|21|263|Wednesday|1905Q1|N|N|N|2416847|2416846|2416491|2416765|N|N|N|N|N| +2416858|AAAAAAAAKNAOECAA|1905-01-12|60|263|21|1905|4|1|12|1|1905|21|263|Thursday|1905Q1|N|N|N|2416847|2416846|2416492|2416766|N|N|N|N|N| +2416859|AAAAAAAALNAOECAA|1905-01-13|60|263|21|1905|5|1|13|1|1905|21|263|Friday|1905Q1|N|Y|N|2416847|2416846|2416493|2416767|N|N|N|N|N| +2416860|AAAAAAAAMNAOECAA|1905-01-14|60|263|21|1905|6|1|14|1|1905|21|263|Saturday|1905Q1|N|Y|N|2416847|2416846|2416494|2416768|N|N|N|N|N| +2416861|AAAAAAAANNAOECAA|1905-01-15|60|263|21|1905|0|1|15|1|1905|21|263|Sunday|1905Q1|N|N|N|2416847|2416846|2416495|2416769|N|N|N|N|N| +2416862|AAAAAAAAONAOECAA|1905-01-16|60|263|21|1905|1|1|16|1|1905|21|263|Monday|1905Q1|N|N|N|2416847|2416846|2416496|2416770|N|N|N|N|N| +2416863|AAAAAAAAPNAOECAA|1905-01-17|60|264|21|1905|2|1|17|1|1905|21|264|Tuesday|1905Q1|N|N|N|2416847|2416846|2416497|2416771|N|N|N|N|N| +2416864|AAAAAAAAAOAOECAA|1905-01-18|60|264|21|1905|3|1|18|1|1905|21|264|Wednesday|1905Q1|N|N|N|2416847|2416846|2416498|2416772|N|N|N|N|N| +2416865|AAAAAAAABOAOECAA|1905-01-19|60|264|21|1905|4|1|19|1|1905|21|264|Thursday|1905Q1|N|N|N|2416847|2416846|2416499|2416773|N|N|N|N|N| +2416866|AAAAAAAACOAOECAA|1905-01-20|60|264|21|1905|5|1|20|1|1905|21|264|Friday|1905Q1|N|Y|N|2416847|2416846|2416500|2416774|N|N|N|N|N| +2416867|AAAAAAAADOAOECAA|1905-01-21|60|264|21|1905|6|1|21|1|1905|21|264|Saturday|1905Q1|N|Y|N|2416847|2416846|2416501|2416775|N|N|N|N|N| +2416868|AAAAAAAAEOAOECAA|1905-01-22|60|264|21|1905|0|1|22|1|1905|21|264|Sunday|1905Q1|N|N|N|2416847|2416846|2416502|2416776|N|N|N|N|N| +2416869|AAAAAAAAFOAOECAA|1905-01-23|60|264|21|1905|1|1|23|1|1905|21|264|Monday|1905Q1|N|N|N|2416847|2416846|2416503|2416777|N|N|N|N|N| +2416870|AAAAAAAAGOAOECAA|1905-01-24|60|265|21|1905|2|1|24|1|1905|21|265|Tuesday|1905Q1|N|N|N|2416847|2416846|2416504|2416778|N|N|N|N|N| +2416871|AAAAAAAAHOAOECAA|1905-01-25|60|265|21|1905|3|1|25|1|1905|21|265|Wednesday|1905Q1|N|N|N|2416847|2416846|2416505|2416779|N|N|N|N|N| +2416872|AAAAAAAAIOAOECAA|1905-01-26|60|265|21|1905|4|1|26|1|1905|21|265|Thursday|1905Q1|N|N|N|2416847|2416846|2416506|2416780|N|N|N|N|N| +2416873|AAAAAAAAJOAOECAA|1905-01-27|60|265|21|1905|5|1|27|1|1905|21|265|Friday|1905Q1|N|Y|N|2416847|2416846|2416507|2416781|N|N|N|N|N| +2416874|AAAAAAAAKOAOECAA|1905-01-28|60|265|21|1905|6|1|28|1|1905|21|265|Saturday|1905Q1|N|Y|N|2416847|2416846|2416508|2416782|N|N|N|N|N| +2416875|AAAAAAAALOAOECAA|1905-01-29|60|265|21|1905|0|1|29|1|1905|21|265|Sunday|1905Q1|N|N|N|2416847|2416846|2416509|2416783|N|N|N|N|N| +2416876|AAAAAAAAMOAOECAA|1905-01-30|60|265|21|1905|1|1|30|1|1905|21|265|Monday|1905Q1|N|N|N|2416847|2416846|2416510|2416784|N|N|N|N|N| +2416877|AAAAAAAANOAOECAA|1905-01-31|60|266|21|1905|2|1|31|1|1905|21|266|Tuesday|1905Q1|N|N|N|2416847|2416846|2416511|2416785|N|N|N|N|N| +2416878|AAAAAAAAOOAOECAA|1905-02-01|61|266|21|1905|3|2|1|1|1905|21|266|Wednesday|1905Q1|N|N|N|2416878|2416908|2416512|2416786|N|N|N|N|N| +2416879|AAAAAAAAPOAOECAA|1905-02-02|61|266|21|1905|4|2|2|1|1905|21|266|Thursday|1905Q1|N|N|N|2416878|2416908|2416513|2416787|N|N|N|N|N| +2416880|AAAAAAAAAPAOECAA|1905-02-03|61|266|21|1905|5|2|3|1|1905|21|266|Friday|1905Q1|N|Y|N|2416878|2416908|2416514|2416788|N|N|N|N|N| +2416881|AAAAAAAABPAOECAA|1905-02-04|61|266|21|1905|6|2|4|1|1905|21|266|Saturday|1905Q1|N|Y|N|2416878|2416908|2416515|2416789|N|N|N|N|N| +2416882|AAAAAAAACPAOECAA|1905-02-05|61|266|21|1905|0|2|5|1|1905|21|266|Sunday|1905Q1|N|N|N|2416878|2416908|2416516|2416790|N|N|N|N|N| +2416883|AAAAAAAADPAOECAA|1905-02-06|61|266|21|1905|1|2|6|1|1905|21|266|Monday|1905Q1|N|N|N|2416878|2416908|2416517|2416791|N|N|N|N|N| +2416884|AAAAAAAAEPAOECAA|1905-02-07|61|267|21|1905|2|2|7|1|1905|21|267|Tuesday|1905Q1|N|N|N|2416878|2416908|2416518|2416792|N|N|N|N|N| +2416885|AAAAAAAAFPAOECAA|1905-02-08|61|267|21|1905|3|2|8|1|1905|21|267|Wednesday|1905Q1|N|N|N|2416878|2416908|2416519|2416793|N|N|N|N|N| +2416886|AAAAAAAAGPAOECAA|1905-02-09|61|267|21|1905|4|2|9|1|1905|21|267|Thursday|1905Q1|N|N|N|2416878|2416908|2416520|2416794|N|N|N|N|N| +2416887|AAAAAAAAHPAOECAA|1905-02-10|61|267|21|1905|5|2|10|1|1905|21|267|Friday|1905Q1|N|Y|N|2416878|2416908|2416521|2416795|N|N|N|N|N| +2416888|AAAAAAAAIPAOECAA|1905-02-11|61|267|21|1905|6|2|11|1|1905|21|267|Saturday|1905Q1|N|Y|N|2416878|2416908|2416522|2416796|N|N|N|N|N| +2416889|AAAAAAAAJPAOECAA|1905-02-12|61|267|21|1905|0|2|12|1|1905|21|267|Sunday|1905Q1|N|N|N|2416878|2416908|2416523|2416797|N|N|N|N|N| +2416890|AAAAAAAAKPAOECAA|1905-02-13|61|267|21|1905|1|2|13|1|1905|21|267|Monday|1905Q1|N|N|N|2416878|2416908|2416524|2416798|N|N|N|N|N| +2416891|AAAAAAAALPAOECAA|1905-02-14|61|268|21|1905|2|2|14|1|1905|21|268|Tuesday|1905Q1|N|N|N|2416878|2416908|2416525|2416799|N|N|N|N|N| +2416892|AAAAAAAAMPAOECAA|1905-02-15|61|268|21|1905|3|2|15|1|1905|21|268|Wednesday|1905Q1|N|N|N|2416878|2416908|2416526|2416800|N|N|N|N|N| +2416893|AAAAAAAANPAOECAA|1905-02-16|61|268|21|1905|4|2|16|1|1905|21|268|Thursday|1905Q1|N|N|N|2416878|2416908|2416527|2416801|N|N|N|N|N| +2416894|AAAAAAAAOPAOECAA|1905-02-17|61|268|21|1905|5|2|17|1|1905|21|268|Friday|1905Q1|N|Y|N|2416878|2416908|2416528|2416802|N|N|N|N|N| +2416895|AAAAAAAAPPAOECAA|1905-02-18|61|268|21|1905|6|2|18|1|1905|21|268|Saturday|1905Q1|N|Y|N|2416878|2416908|2416529|2416803|N|N|N|N|N| +2416896|AAAAAAAAAABOECAA|1905-02-19|61|268|21|1905|0|2|19|1|1905|21|268|Sunday|1905Q1|N|N|N|2416878|2416908|2416530|2416804|N|N|N|N|N| +2416897|AAAAAAAABABOECAA|1905-02-20|61|268|21|1905|1|2|20|1|1905|21|268|Monday|1905Q1|N|N|N|2416878|2416908|2416531|2416805|N|N|N|N|N| +2416898|AAAAAAAACABOECAA|1905-02-21|61|269|21|1905|2|2|21|1|1905|21|269|Tuesday|1905Q1|N|N|N|2416878|2416908|2416532|2416806|N|N|N|N|N| +2416899|AAAAAAAADABOECAA|1905-02-22|61|269|21|1905|3|2|22|1|1905|21|269|Wednesday|1905Q1|N|N|N|2416878|2416908|2416533|2416807|N|N|N|N|N| +2416900|AAAAAAAAEABOECAA|1905-02-23|61|269|21|1905|4|2|23|1|1905|21|269|Thursday|1905Q1|N|N|N|2416878|2416908|2416534|2416808|N|N|N|N|N| +2416901|AAAAAAAAFABOECAA|1905-02-24|61|269|21|1905|5|2|24|1|1905|21|269|Friday|1905Q1|N|Y|N|2416878|2416908|2416535|2416809|N|N|N|N|N| +2416902|AAAAAAAAGABOECAA|1905-02-25|61|269|21|1905|6|2|25|1|1905|21|269|Saturday|1905Q1|N|Y|N|2416878|2416908|2416536|2416810|N|N|N|N|N| +2416903|AAAAAAAAHABOECAA|1905-02-26|61|269|21|1905|0|2|26|1|1905|21|269|Sunday|1905Q1|N|N|N|2416878|2416908|2416537|2416811|N|N|N|N|N| +2416904|AAAAAAAAIABOECAA|1905-02-27|61|269|21|1905|1|2|27|1|1905|21|269|Monday|1905Q1|N|N|N|2416878|2416908|2416538|2416812|N|N|N|N|N| +2416905|AAAAAAAAJABOECAA|1905-02-28|61|270|21|1905|2|2|28|1|1905|21|270|Tuesday|1905Q1|N|N|N|2416878|2416908|2416539|2416813|N|N|N|N|N| +2416906|AAAAAAAAKABOECAA|1905-03-01|62|270|22|1905|3|3|1|1|1905|22|270|Wednesday|1905Q1|N|N|N|2416906|2416964|2416541|2416814|N|N|N|N|N| +2416907|AAAAAAAALABOECAA|1905-03-02|62|270|22|1905|4|3|2|1|1905|22|270|Thursday|1905Q1|N|N|N|2416906|2416964|2416542|2416815|N|N|N|N|N| +2416908|AAAAAAAAMABOECAA|1905-03-03|62|270|22|1905|5|3|3|1|1905|22|270|Friday|1905Q1|N|Y|N|2416906|2416964|2416543|2416816|N|N|N|N|N| +2416909|AAAAAAAANABOECAA|1905-03-04|62|270|22|1905|6|3|4|1|1905|22|270|Saturday|1905Q1|N|Y|N|2416906|2416964|2416544|2416817|N|N|N|N|N| +2416910|AAAAAAAAOABOECAA|1905-03-05|62|270|22|1905|0|3|5|1|1905|22|270|Sunday|1905Q1|N|N|N|2416906|2416964|2416545|2416818|N|N|N|N|N| +2416911|AAAAAAAAPABOECAA|1905-03-06|62|270|22|1905|1|3|6|1|1905|22|270|Monday|1905Q1|N|N|N|2416906|2416964|2416546|2416819|N|N|N|N|N| +2416912|AAAAAAAAABBOECAA|1905-03-07|62|271|22|1905|2|3|7|1|1905|22|271|Tuesday|1905Q1|N|N|N|2416906|2416964|2416547|2416820|N|N|N|N|N| +2416913|AAAAAAAABBBOECAA|1905-03-08|62|271|22|1905|3|3|8|1|1905|22|271|Wednesday|1905Q1|N|N|N|2416906|2416964|2416548|2416821|N|N|N|N|N| +2416914|AAAAAAAACBBOECAA|1905-03-09|62|271|22|1905|4|3|9|1|1905|22|271|Thursday|1905Q1|N|N|N|2416906|2416964|2416549|2416822|N|N|N|N|N| +2416915|AAAAAAAADBBOECAA|1905-03-10|62|271|22|1905|5|3|10|1|1905|22|271|Friday|1905Q1|N|Y|N|2416906|2416964|2416550|2416823|N|N|N|N|N| +2416916|AAAAAAAAEBBOECAA|1905-03-11|62|271|22|1905|6|3|11|1|1905|22|271|Saturday|1905Q1|N|Y|N|2416906|2416964|2416551|2416824|N|N|N|N|N| +2416917|AAAAAAAAFBBOECAA|1905-03-12|62|271|22|1905|0|3|12|1|1905|22|271|Sunday|1905Q1|N|N|N|2416906|2416964|2416552|2416825|N|N|N|N|N| +2416918|AAAAAAAAGBBOECAA|1905-03-13|62|271|22|1905|1|3|13|1|1905|22|271|Monday|1905Q1|N|N|N|2416906|2416964|2416553|2416826|N|N|N|N|N| +2416919|AAAAAAAAHBBOECAA|1905-03-14|62|272|22|1905|2|3|14|1|1905|22|272|Tuesday|1905Q1|N|N|N|2416906|2416964|2416554|2416827|N|N|N|N|N| +2416920|AAAAAAAAIBBOECAA|1905-03-15|62|272|22|1905|3|3|15|1|1905|22|272|Wednesday|1905Q1|N|N|N|2416906|2416964|2416555|2416828|N|N|N|N|N| +2416921|AAAAAAAAJBBOECAA|1905-03-16|62|272|22|1905|4|3|16|1|1905|22|272|Thursday|1905Q1|N|N|N|2416906|2416964|2416556|2416829|N|N|N|N|N| +2416922|AAAAAAAAKBBOECAA|1905-03-17|62|272|22|1905|5|3|17|1|1905|22|272|Friday|1905Q1|N|Y|N|2416906|2416964|2416557|2416830|N|N|N|N|N| +2416923|AAAAAAAALBBOECAA|1905-03-18|62|272|22|1905|6|3|18|1|1905|22|272|Saturday|1905Q1|N|Y|N|2416906|2416964|2416558|2416831|N|N|N|N|N| +2416924|AAAAAAAAMBBOECAA|1905-03-19|62|272|22|1905|0|3|19|1|1905|22|272|Sunday|1905Q1|N|N|N|2416906|2416964|2416559|2416832|N|N|N|N|N| +2416925|AAAAAAAANBBOECAA|1905-03-20|62|272|22|1905|1|3|20|1|1905|22|272|Monday|1905Q1|N|N|N|2416906|2416964|2416560|2416833|N|N|N|N|N| +2416926|AAAAAAAAOBBOECAA|1905-03-21|62|273|22|1905|2|3|21|1|1905|22|273|Tuesday|1905Q1|N|N|N|2416906|2416964|2416561|2416834|N|N|N|N|N| +2416927|AAAAAAAAPBBOECAA|1905-03-22|62|273|22|1905|3|3|22|1|1905|22|273|Wednesday|1905Q1|N|N|N|2416906|2416964|2416562|2416835|N|N|N|N|N| +2416928|AAAAAAAAACBOECAA|1905-03-23|62|273|22|1905|4|3|23|1|1905|22|273|Thursday|1905Q1|N|N|N|2416906|2416964|2416563|2416836|N|N|N|N|N| +2416929|AAAAAAAABCBOECAA|1905-03-24|62|273|22|1905|5|3|24|1|1905|22|273|Friday|1905Q1|N|Y|N|2416906|2416964|2416564|2416837|N|N|N|N|N| +2416930|AAAAAAAACCBOECAA|1905-03-25|62|273|22|1905|6|3|25|1|1905|22|273|Saturday|1905Q1|N|Y|N|2416906|2416964|2416565|2416838|N|N|N|N|N| +2416931|AAAAAAAADCBOECAA|1905-03-26|62|273|22|1905|0|3|26|1|1905|22|273|Sunday|1905Q1|N|N|N|2416906|2416964|2416566|2416839|N|N|N|N|N| +2416932|AAAAAAAAECBOECAA|1905-03-27|62|273|22|1905|1|3|27|1|1905|22|273|Monday|1905Q1|N|N|N|2416906|2416964|2416567|2416840|N|N|N|N|N| +2416933|AAAAAAAAFCBOECAA|1905-03-28|62|274|22|1905|2|3|28|1|1905|22|274|Tuesday|1905Q1|N|N|N|2416906|2416964|2416568|2416841|N|N|N|N|N| +2416934|AAAAAAAAGCBOECAA|1905-03-29|62|274|22|1905|3|3|29|1|1905|22|274|Wednesday|1905Q1|N|N|N|2416906|2416964|2416569|2416842|N|N|N|N|N| +2416935|AAAAAAAAHCBOECAA|1905-03-30|62|274|22|1905|4|3|30|1|1905|22|274|Thursday|1905Q1|N|N|N|2416906|2416964|2416570|2416843|N|N|N|N|N| +2416936|AAAAAAAAICBOECAA|1905-03-31|62|274|22|1905|5|3|31|1|1905|22|274|Friday|1905Q1|N|Y|N|2416906|2416964|2416571|2416844|N|N|N|N|N| +2416937|AAAAAAAAJCBOECAA|1905-04-01|63|274|22|1905|6|4|1|1|1905|22|274|Saturday|1905Q1|N|Y|N|2416937|2417026|2416572|2416847|N|N|N|N|N| +2416938|AAAAAAAAKCBOECAA|1905-04-02|63|274|22|1905|0|4|2|2|1905|22|274|Sunday|1905Q2|N|N|N|2416937|2417026|2416573|2416848|N|N|N|N|N| +2416939|AAAAAAAALCBOECAA|1905-04-03|63|274|22|1905|1|4|3|2|1905|22|274|Monday|1905Q2|N|N|N|2416937|2417026|2416574|2416849|N|N|N|N|N| +2416940|AAAAAAAAMCBOECAA|1905-04-04|63|275|22|1905|2|4|4|2|1905|22|275|Tuesday|1905Q2|N|N|N|2416937|2417026|2416575|2416850|N|N|N|N|N| +2416941|AAAAAAAANCBOECAA|1905-04-05|63|275|22|1905|3|4|5|2|1905|22|275|Wednesday|1905Q2|N|N|N|2416937|2417026|2416576|2416851|N|N|N|N|N| +2416942|AAAAAAAAOCBOECAA|1905-04-06|63|275|22|1905|4|4|6|2|1905|22|275|Thursday|1905Q2|N|N|N|2416937|2417026|2416577|2416852|N|N|N|N|N| +2416943|AAAAAAAAPCBOECAA|1905-04-07|63|275|22|1905|5|4|7|2|1905|22|275|Friday|1905Q2|N|Y|N|2416937|2417026|2416578|2416853|N|N|N|N|N| +2416944|AAAAAAAAADBOECAA|1905-04-08|63|275|22|1905|6|4|8|2|1905|22|275|Saturday|1905Q2|N|Y|N|2416937|2417026|2416579|2416854|N|N|N|N|N| +2416945|AAAAAAAABDBOECAA|1905-04-09|63|275|22|1905|0|4|9|2|1905|22|275|Sunday|1905Q2|N|N|N|2416937|2417026|2416580|2416855|N|N|N|N|N| +2416946|AAAAAAAACDBOECAA|1905-04-10|63|275|22|1905|1|4|10|2|1905|22|275|Monday|1905Q2|N|N|N|2416937|2417026|2416581|2416856|N|N|N|N|N| +2416947|AAAAAAAADDBOECAA|1905-04-11|63|276|22|1905|2|4|11|2|1905|22|276|Tuesday|1905Q2|N|N|N|2416937|2417026|2416582|2416857|N|N|N|N|N| +2416948|AAAAAAAAEDBOECAA|1905-04-12|63|276|22|1905|3|4|12|2|1905|22|276|Wednesday|1905Q2|N|N|N|2416937|2417026|2416583|2416858|N|N|N|N|N| +2416949|AAAAAAAAFDBOECAA|1905-04-13|63|276|22|1905|4|4|13|2|1905|22|276|Thursday|1905Q2|N|N|N|2416937|2417026|2416584|2416859|N|N|N|N|N| +2416950|AAAAAAAAGDBOECAA|1905-04-14|63|276|22|1905|5|4|14|2|1905|22|276|Friday|1905Q2|N|Y|N|2416937|2417026|2416585|2416860|N|N|N|N|N| +2416951|AAAAAAAAHDBOECAA|1905-04-15|63|276|22|1905|6|4|15|2|1905|22|276|Saturday|1905Q2|N|Y|N|2416937|2417026|2416586|2416861|N|N|N|N|N| +2416952|AAAAAAAAIDBOECAA|1905-04-16|63|276|22|1905|0|4|16|2|1905|22|276|Sunday|1905Q2|N|N|N|2416937|2417026|2416587|2416862|N|N|N|N|N| +2416953|AAAAAAAAJDBOECAA|1905-04-17|63|276|22|1905|1|4|17|2|1905|22|276|Monday|1905Q2|N|N|N|2416937|2417026|2416588|2416863|N|N|N|N|N| +2416954|AAAAAAAAKDBOECAA|1905-04-18|63|277|22|1905|2|4|18|2|1905|22|277|Tuesday|1905Q2|N|N|N|2416937|2417026|2416589|2416864|N|N|N|N|N| +2416955|AAAAAAAALDBOECAA|1905-04-19|63|277|22|1905|3|4|19|2|1905|22|277|Wednesday|1905Q2|N|N|N|2416937|2417026|2416590|2416865|N|N|N|N|N| +2416956|AAAAAAAAMDBOECAA|1905-04-20|63|277|22|1905|4|4|20|2|1905|22|277|Thursday|1905Q2|N|N|N|2416937|2417026|2416591|2416866|N|N|N|N|N| +2416957|AAAAAAAANDBOECAA|1905-04-21|63|277|22|1905|5|4|21|2|1905|22|277|Friday|1905Q2|N|Y|N|2416937|2417026|2416592|2416867|N|N|N|N|N| +2416958|AAAAAAAAODBOECAA|1905-04-22|63|277|22|1905|6|4|22|2|1905|22|277|Saturday|1905Q2|N|Y|N|2416937|2417026|2416593|2416868|N|N|N|N|N| +2416959|AAAAAAAAPDBOECAA|1905-04-23|63|277|22|1905|0|4|23|2|1905|22|277|Sunday|1905Q2|N|N|N|2416937|2417026|2416594|2416869|N|N|N|N|N| +2416960|AAAAAAAAAEBOECAA|1905-04-24|63|277|22|1905|1|4|24|2|1905|22|277|Monday|1905Q2|N|N|N|2416937|2417026|2416595|2416870|N|N|N|N|N| +2416961|AAAAAAAABEBOECAA|1905-04-25|63|278|22|1905|2|4|25|2|1905|22|278|Tuesday|1905Q2|N|N|N|2416937|2417026|2416596|2416871|N|N|N|N|N| +2416962|AAAAAAAACEBOECAA|1905-04-26|63|278|22|1905|3|4|26|2|1905|22|278|Wednesday|1905Q2|N|N|N|2416937|2417026|2416597|2416872|N|N|N|N|N| +2416963|AAAAAAAADEBOECAA|1905-04-27|63|278|22|1905|4|4|27|2|1905|22|278|Thursday|1905Q2|N|N|N|2416937|2417026|2416598|2416873|N|N|N|N|N| +2416964|AAAAAAAAEEBOECAA|1905-04-28|63|278|22|1905|5|4|28|2|1905|22|278|Friday|1905Q2|N|Y|N|2416937|2417026|2416599|2416874|N|N|N|N|N| +2416965|AAAAAAAAFEBOECAA|1905-04-29|63|278|22|1905|6|4|29|2|1905|22|278|Saturday|1905Q2|N|Y|N|2416937|2417026|2416600|2416875|N|N|N|N|N| +2416966|AAAAAAAAGEBOECAA|1905-04-30|63|278|22|1905|0|4|30|2|1905|22|278|Sunday|1905Q2|N|N|N|2416937|2417026|2416601|2416876|N|N|N|N|N| +2416967|AAAAAAAAHEBOECAA|1905-05-01|64|278|22|1905|1|5|1|2|1905|22|278|Monday|1905Q2|N|N|N|2416967|2417086|2416602|2416877|N|N|N|N|N| +2416968|AAAAAAAAIEBOECAA|1905-05-02|64|279|22|1905|2|5|2|2|1905|22|279|Tuesday|1905Q2|N|N|N|2416967|2417086|2416603|2416878|N|N|N|N|N| +2416969|AAAAAAAAJEBOECAA|1905-05-03|64|279|22|1905|3|5|3|2|1905|22|279|Wednesday|1905Q2|N|N|N|2416967|2417086|2416604|2416879|N|N|N|N|N| +2416970|AAAAAAAAKEBOECAA|1905-05-04|64|279|22|1905|4|5|4|2|1905|22|279|Thursday|1905Q2|N|N|N|2416967|2417086|2416605|2416880|N|N|N|N|N| +2416971|AAAAAAAALEBOECAA|1905-05-05|64|279|22|1905|5|5|5|2|1905|22|279|Friday|1905Q2|N|Y|N|2416967|2417086|2416606|2416881|N|N|N|N|N| +2416972|AAAAAAAAMEBOECAA|1905-05-06|64|279|22|1905|6|5|6|2|1905|22|279|Saturday|1905Q2|N|Y|N|2416967|2417086|2416607|2416882|N|N|N|N|N| +2416973|AAAAAAAANEBOECAA|1905-05-07|64|279|22|1905|0|5|7|2|1905|22|279|Sunday|1905Q2|N|N|N|2416967|2417086|2416608|2416883|N|N|N|N|N| +2416974|AAAAAAAAOEBOECAA|1905-05-08|64|279|22|1905|1|5|8|2|1905|22|279|Monday|1905Q2|N|N|N|2416967|2417086|2416609|2416884|N|N|N|N|N| +2416975|AAAAAAAAPEBOECAA|1905-05-09|64|280|22|1905|2|5|9|2|1905|22|280|Tuesday|1905Q2|N|N|N|2416967|2417086|2416610|2416885|N|N|N|N|N| +2416976|AAAAAAAAAFBOECAA|1905-05-10|64|280|22|1905|3|5|10|2|1905|22|280|Wednesday|1905Q2|N|N|N|2416967|2417086|2416611|2416886|N|N|N|N|N| +2416977|AAAAAAAABFBOECAA|1905-05-11|64|280|22|1905|4|5|11|2|1905|22|280|Thursday|1905Q2|N|N|N|2416967|2417086|2416612|2416887|N|N|N|N|N| +2416978|AAAAAAAACFBOECAA|1905-05-12|64|280|22|1905|5|5|12|2|1905|22|280|Friday|1905Q2|N|Y|N|2416967|2417086|2416613|2416888|N|N|N|N|N| +2416979|AAAAAAAADFBOECAA|1905-05-13|64|280|22|1905|6|5|13|2|1905|22|280|Saturday|1905Q2|N|Y|N|2416967|2417086|2416614|2416889|N|N|N|N|N| +2416980|AAAAAAAAEFBOECAA|1905-05-14|64|280|22|1905|0|5|14|2|1905|22|280|Sunday|1905Q2|N|N|N|2416967|2417086|2416615|2416890|N|N|N|N|N| +2416981|AAAAAAAAFFBOECAA|1905-05-15|64|280|22|1905|1|5|15|2|1905|22|280|Monday|1905Q2|N|N|N|2416967|2417086|2416616|2416891|N|N|N|N|N| +2416982|AAAAAAAAGFBOECAA|1905-05-16|64|281|22|1905|2|5|16|2|1905|22|281|Tuesday|1905Q2|N|N|N|2416967|2417086|2416617|2416892|N|N|N|N|N| +2416983|AAAAAAAAHFBOECAA|1905-05-17|64|281|22|1905|3|5|17|2|1905|22|281|Wednesday|1905Q2|N|N|N|2416967|2417086|2416618|2416893|N|N|N|N|N| +2416984|AAAAAAAAIFBOECAA|1905-05-18|64|281|22|1905|4|5|18|2|1905|22|281|Thursday|1905Q2|N|N|N|2416967|2417086|2416619|2416894|N|N|N|N|N| +2416985|AAAAAAAAJFBOECAA|1905-05-19|64|281|22|1905|5|5|19|2|1905|22|281|Friday|1905Q2|N|Y|N|2416967|2417086|2416620|2416895|N|N|N|N|N| +2416986|AAAAAAAAKFBOECAA|1905-05-20|64|281|22|1905|6|5|20|2|1905|22|281|Saturday|1905Q2|N|Y|N|2416967|2417086|2416621|2416896|N|N|N|N|N| +2416987|AAAAAAAALFBOECAA|1905-05-21|64|281|22|1905|0|5|21|2|1905|22|281|Sunday|1905Q2|N|N|N|2416967|2417086|2416622|2416897|N|N|N|N|N| +2416988|AAAAAAAAMFBOECAA|1905-05-22|64|281|22|1905|1|5|22|2|1905|22|281|Monday|1905Q2|N|N|N|2416967|2417086|2416623|2416898|N|N|N|N|N| +2416989|AAAAAAAANFBOECAA|1905-05-23|64|282|22|1905|2|5|23|2|1905|22|282|Tuesday|1905Q2|N|N|N|2416967|2417086|2416624|2416899|N|N|N|N|N| +2416990|AAAAAAAAOFBOECAA|1905-05-24|64|282|22|1905|3|5|24|2|1905|22|282|Wednesday|1905Q2|N|N|N|2416967|2417086|2416625|2416900|N|N|N|N|N| +2416991|AAAAAAAAPFBOECAA|1905-05-25|64|282|22|1905|4|5|25|2|1905|22|282|Thursday|1905Q2|N|N|N|2416967|2417086|2416626|2416901|N|N|N|N|N| +2416992|AAAAAAAAAGBOECAA|1905-05-26|64|282|22|1905|5|5|26|2|1905|22|282|Friday|1905Q2|N|Y|N|2416967|2417086|2416627|2416902|N|N|N|N|N| +2416993|AAAAAAAABGBOECAA|1905-05-27|64|282|22|1905|6|5|27|2|1905|22|282|Saturday|1905Q2|N|Y|N|2416967|2417086|2416628|2416903|N|N|N|N|N| +2416994|AAAAAAAACGBOECAA|1905-05-28|64|282|22|1905|0|5|28|2|1905|22|282|Sunday|1905Q2|N|N|N|2416967|2417086|2416629|2416904|N|N|N|N|N| +2416995|AAAAAAAADGBOECAA|1905-05-29|64|282|22|1905|1|5|29|2|1905|22|282|Monday|1905Q2|N|N|N|2416967|2417086|2416630|2416905|N|N|N|N|N| +2416996|AAAAAAAAEGBOECAA|1905-05-30|64|283|22|1905|2|5|30|2|1905|22|283|Tuesday|1905Q2|N|N|N|2416967|2417086|2416631|2416906|N|N|N|N|N| +2416997|AAAAAAAAFGBOECAA|1905-05-31|64|283|22|1905|3|5|31|2|1905|22|283|Wednesday|1905Q2|N|N|N|2416967|2417086|2416632|2416907|N|N|N|N|N| +2416998|AAAAAAAAGGBOECAA|1905-06-01|65|283|23|1905|4|6|1|2|1905|23|283|Thursday|1905Q2|N|N|N|2416998|2417148|2416633|2416908|N|N|N|N|N| +2416999|AAAAAAAAHGBOECAA|1905-06-02|65|283|23|1905|5|6|2|2|1905|23|283|Friday|1905Q2|N|Y|N|2416998|2417148|2416634|2416909|N|N|N|N|N| +2417000|AAAAAAAAIGBOECAA|1905-06-03|65|283|23|1905|6|6|3|2|1905|23|283|Saturday|1905Q2|N|Y|N|2416998|2417148|2416635|2416910|N|N|N|N|N| +2417001|AAAAAAAAJGBOECAA|1905-06-04|65|283|23|1905|0|6|4|2|1905|23|283|Sunday|1905Q2|N|N|N|2416998|2417148|2416636|2416911|N|N|N|N|N| +2417002|AAAAAAAAKGBOECAA|1905-06-05|65|283|23|1905|1|6|5|2|1905|23|283|Monday|1905Q2|N|N|N|2416998|2417148|2416637|2416912|N|N|N|N|N| +2417003|AAAAAAAALGBOECAA|1905-06-06|65|284|23|1905|2|6|6|2|1905|23|284|Tuesday|1905Q2|N|N|N|2416998|2417148|2416638|2416913|N|N|N|N|N| +2417004|AAAAAAAAMGBOECAA|1905-06-07|65|284|23|1905|3|6|7|2|1905|23|284|Wednesday|1905Q2|N|N|N|2416998|2417148|2416639|2416914|N|N|N|N|N| +2417005|AAAAAAAANGBOECAA|1905-06-08|65|284|23|1905|4|6|8|2|1905|23|284|Thursday|1905Q2|N|N|N|2416998|2417148|2416640|2416915|N|N|N|N|N| +2417006|AAAAAAAAOGBOECAA|1905-06-09|65|284|23|1905|5|6|9|2|1905|23|284|Friday|1905Q2|N|Y|N|2416998|2417148|2416641|2416916|N|N|N|N|N| +2417007|AAAAAAAAPGBOECAA|1905-06-10|65|284|23|1905|6|6|10|2|1905|23|284|Saturday|1905Q2|N|Y|N|2416998|2417148|2416642|2416917|N|N|N|N|N| +2417008|AAAAAAAAAHBOECAA|1905-06-11|65|284|23|1905|0|6|11|2|1905|23|284|Sunday|1905Q2|N|N|N|2416998|2417148|2416643|2416918|N|N|N|N|N| +2417009|AAAAAAAABHBOECAA|1905-06-12|65|284|23|1905|1|6|12|2|1905|23|284|Monday|1905Q2|N|N|N|2416998|2417148|2416644|2416919|N|N|N|N|N| +2417010|AAAAAAAACHBOECAA|1905-06-13|65|285|23|1905|2|6|13|2|1905|23|285|Tuesday|1905Q2|N|N|N|2416998|2417148|2416645|2416920|N|N|N|N|N| +2417011|AAAAAAAADHBOECAA|1905-06-14|65|285|23|1905|3|6|14|2|1905|23|285|Wednesday|1905Q2|N|N|N|2416998|2417148|2416646|2416921|N|N|N|N|N| +2417012|AAAAAAAAEHBOECAA|1905-06-15|65|285|23|1905|4|6|15|2|1905|23|285|Thursday|1905Q2|N|N|N|2416998|2417148|2416647|2416922|N|N|N|N|N| +2417013|AAAAAAAAFHBOECAA|1905-06-16|65|285|23|1905|5|6|16|2|1905|23|285|Friday|1905Q2|N|Y|N|2416998|2417148|2416648|2416923|N|N|N|N|N| +2417014|AAAAAAAAGHBOECAA|1905-06-17|65|285|23|1905|6|6|17|2|1905|23|285|Saturday|1905Q2|N|Y|N|2416998|2417148|2416649|2416924|N|N|N|N|N| +2417015|AAAAAAAAHHBOECAA|1905-06-18|65|285|23|1905|0|6|18|2|1905|23|285|Sunday|1905Q2|N|N|N|2416998|2417148|2416650|2416925|N|N|N|N|N| +2417016|AAAAAAAAIHBOECAA|1905-06-19|65|285|23|1905|1|6|19|2|1905|23|285|Monday|1905Q2|N|N|N|2416998|2417148|2416651|2416926|N|N|N|N|N| +2417017|AAAAAAAAJHBOECAA|1905-06-20|65|286|23|1905|2|6|20|2|1905|23|286|Tuesday|1905Q2|N|N|N|2416998|2417148|2416652|2416927|N|N|N|N|N| +2417018|AAAAAAAAKHBOECAA|1905-06-21|65|286|23|1905|3|6|21|2|1905|23|286|Wednesday|1905Q2|N|N|N|2416998|2417148|2416653|2416928|N|N|N|N|N| +2417019|AAAAAAAALHBOECAA|1905-06-22|65|286|23|1905|4|6|22|2|1905|23|286|Thursday|1905Q2|N|N|N|2416998|2417148|2416654|2416929|N|N|N|N|N| +2417020|AAAAAAAAMHBOECAA|1905-06-23|65|286|23|1905|5|6|23|2|1905|23|286|Friday|1905Q2|N|Y|N|2416998|2417148|2416655|2416930|N|N|N|N|N| +2417021|AAAAAAAANHBOECAA|1905-06-24|65|286|23|1905|6|6|24|2|1905|23|286|Saturday|1905Q2|N|Y|N|2416998|2417148|2416656|2416931|N|N|N|N|N| +2417022|AAAAAAAAOHBOECAA|1905-06-25|65|286|23|1905|0|6|25|2|1905|23|286|Sunday|1905Q2|N|N|N|2416998|2417148|2416657|2416932|N|N|N|N|N| +2417023|AAAAAAAAPHBOECAA|1905-06-26|65|286|23|1905|1|6|26|2|1905|23|286|Monday|1905Q2|N|N|N|2416998|2417148|2416658|2416933|N|N|N|N|N| +2417024|AAAAAAAAAIBOECAA|1905-06-27|65|287|23|1905|2|6|27|2|1905|23|287|Tuesday|1905Q2|N|N|N|2416998|2417148|2416659|2416934|N|N|N|N|N| +2417025|AAAAAAAABIBOECAA|1905-06-28|65|287|23|1905|3|6|28|2|1905|23|287|Wednesday|1905Q2|N|N|N|2416998|2417148|2416660|2416935|N|N|N|N|N| +2417026|AAAAAAAACIBOECAA|1905-06-29|65|287|23|1905|4|6|29|2|1905|23|287|Thursday|1905Q2|N|N|N|2416998|2417148|2416661|2416936|N|N|N|N|N| +2417027|AAAAAAAADIBOECAA|1905-06-30|65|287|23|1905|5|6|30|2|1905|23|287|Friday|1905Q2|N|Y|N|2416998|2417148|2416662|2416937|N|N|N|N|N| +2417028|AAAAAAAAEIBOECAA|1905-07-01|66|287|23|1905|6|7|1|2|1905|23|287|Saturday|1905Q2|N|Y|N|2417028|2417208|2416663|2416937|N|N|N|N|N| +2417029|AAAAAAAAFIBOECAA|1905-07-02|66|287|23|1905|0|7|2|3|1905|23|287|Sunday|1905Q3|N|N|N|2417028|2417208|2416664|2416938|N|N|N|N|N| +2417030|AAAAAAAAGIBOECAA|1905-07-03|66|287|23|1905|1|7|3|3|1905|23|287|Monday|1905Q3|N|N|N|2417028|2417208|2416665|2416939|N|N|N|N|N| +2417031|AAAAAAAAHIBOECAA|1905-07-04|66|288|23|1905|2|7|4|3|1905|23|288|Tuesday|1905Q3|N|N|N|2417028|2417208|2416666|2416940|N|N|N|N|N| +2417032|AAAAAAAAIIBOECAA|1905-07-05|66|288|23|1905|3|7|5|3|1905|23|288|Wednesday|1905Q3|Y|N|N|2417028|2417208|2416667|2416941|N|N|N|N|N| +2417033|AAAAAAAAJIBOECAA|1905-07-06|66|288|23|1905|4|7|6|3|1905|23|288|Thursday|1905Q3|N|N|Y|2417028|2417208|2416668|2416942|N|N|N|N|N| +2417034|AAAAAAAAKIBOECAA|1905-07-07|66|288|23|1905|5|7|7|3|1905|23|288|Friday|1905Q3|N|Y|N|2417028|2417208|2416669|2416943|N|N|N|N|N| +2417035|AAAAAAAALIBOECAA|1905-07-08|66|288|23|1905|6|7|8|3|1905|23|288|Saturday|1905Q3|N|Y|N|2417028|2417208|2416670|2416944|N|N|N|N|N| +2417036|AAAAAAAAMIBOECAA|1905-07-09|66|288|23|1905|0|7|9|3|1905|23|288|Sunday|1905Q3|N|N|N|2417028|2417208|2416671|2416945|N|N|N|N|N| +2417037|AAAAAAAANIBOECAA|1905-07-10|66|288|23|1905|1|7|10|3|1905|23|288|Monday|1905Q3|N|N|N|2417028|2417208|2416672|2416946|N|N|N|N|N| +2417038|AAAAAAAAOIBOECAA|1905-07-11|66|289|23|1905|2|7|11|3|1905|23|289|Tuesday|1905Q3|N|N|N|2417028|2417208|2416673|2416947|N|N|N|N|N| +2417039|AAAAAAAAPIBOECAA|1905-07-12|66|289|23|1905|3|7|12|3|1905|23|289|Wednesday|1905Q3|N|N|N|2417028|2417208|2416674|2416948|N|N|N|N|N| +2417040|AAAAAAAAAJBOECAA|1905-07-13|66|289|23|1905|4|7|13|3|1905|23|289|Thursday|1905Q3|N|N|N|2417028|2417208|2416675|2416949|N|N|N|N|N| +2417041|AAAAAAAABJBOECAA|1905-07-14|66|289|23|1905|5|7|14|3|1905|23|289|Friday|1905Q3|N|Y|N|2417028|2417208|2416676|2416950|N|N|N|N|N| +2417042|AAAAAAAACJBOECAA|1905-07-15|66|289|23|1905|6|7|15|3|1905|23|289|Saturday|1905Q3|N|Y|N|2417028|2417208|2416677|2416951|N|N|N|N|N| +2417043|AAAAAAAADJBOECAA|1905-07-16|66|289|23|1905|0|7|16|3|1905|23|289|Sunday|1905Q3|N|N|N|2417028|2417208|2416678|2416952|N|N|N|N|N| +2417044|AAAAAAAAEJBOECAA|1905-07-17|66|289|23|1905|1|7|17|3|1905|23|289|Monday|1905Q3|N|N|N|2417028|2417208|2416679|2416953|N|N|N|N|N| +2417045|AAAAAAAAFJBOECAA|1905-07-18|66|290|23|1905|2|7|18|3|1905|23|290|Tuesday|1905Q3|N|N|N|2417028|2417208|2416680|2416954|N|N|N|N|N| +2417046|AAAAAAAAGJBOECAA|1905-07-19|66|290|23|1905|3|7|19|3|1905|23|290|Wednesday|1905Q3|N|N|N|2417028|2417208|2416681|2416955|N|N|N|N|N| +2417047|AAAAAAAAHJBOECAA|1905-07-20|66|290|23|1905|4|7|20|3|1905|23|290|Thursday|1905Q3|N|N|N|2417028|2417208|2416682|2416956|N|N|N|N|N| +2417048|AAAAAAAAIJBOECAA|1905-07-21|66|290|23|1905|5|7|21|3|1905|23|290|Friday|1905Q3|N|Y|N|2417028|2417208|2416683|2416957|N|N|N|N|N| +2417049|AAAAAAAAJJBOECAA|1905-07-22|66|290|23|1905|6|7|22|3|1905|23|290|Saturday|1905Q3|N|Y|N|2417028|2417208|2416684|2416958|N|N|N|N|N| +2417050|AAAAAAAAKJBOECAA|1905-07-23|66|290|23|1905|0|7|23|3|1905|23|290|Sunday|1905Q3|N|N|N|2417028|2417208|2416685|2416959|N|N|N|N|N| +2417051|AAAAAAAALJBOECAA|1905-07-24|66|290|23|1905|1|7|24|3|1905|23|290|Monday|1905Q3|N|N|N|2417028|2417208|2416686|2416960|N|N|N|N|N| +2417052|AAAAAAAAMJBOECAA|1905-07-25|66|291|23|1905|2|7|25|3|1905|23|291|Tuesday|1905Q3|N|N|N|2417028|2417208|2416687|2416961|N|N|N|N|N| +2417053|AAAAAAAANJBOECAA|1905-07-26|66|291|23|1905|3|7|26|3|1905|23|291|Wednesday|1905Q3|N|N|N|2417028|2417208|2416688|2416962|N|N|N|N|N| +2417054|AAAAAAAAOJBOECAA|1905-07-27|66|291|23|1905|4|7|27|3|1905|23|291|Thursday|1905Q3|N|N|N|2417028|2417208|2416689|2416963|N|N|N|N|N| +2417055|AAAAAAAAPJBOECAA|1905-07-28|66|291|23|1905|5|7|28|3|1905|23|291|Friday|1905Q3|N|Y|N|2417028|2417208|2416690|2416964|N|N|N|N|N| +2417056|AAAAAAAAAKBOECAA|1905-07-29|66|291|23|1905|6|7|29|3|1905|23|291|Saturday|1905Q3|N|Y|N|2417028|2417208|2416691|2416965|N|N|N|N|N| +2417057|AAAAAAAABKBOECAA|1905-07-30|66|291|23|1905|0|7|30|3|1905|23|291|Sunday|1905Q3|N|N|N|2417028|2417208|2416692|2416966|N|N|N|N|N| +2417058|AAAAAAAACKBOECAA|1905-07-31|66|291|23|1905|1|7|31|3|1905|23|291|Monday|1905Q3|N|N|N|2417028|2417208|2416693|2416967|N|N|N|N|N| +2417059|AAAAAAAADKBOECAA|1905-08-01|67|292|23|1905|2|8|1|3|1905|23|292|Tuesday|1905Q3|N|N|N|2417059|2417270|2416694|2416968|N|N|N|N|N| +2417060|AAAAAAAAEKBOECAA|1905-08-02|67|292|23|1905|3|8|2|3|1905|23|292|Wednesday|1905Q3|N|N|N|2417059|2417270|2416695|2416969|N|N|N|N|N| +2417061|AAAAAAAAFKBOECAA|1905-08-03|67|292|23|1905|4|8|3|3|1905|23|292|Thursday|1905Q3|N|N|N|2417059|2417270|2416696|2416970|N|N|N|N|N| +2417062|AAAAAAAAGKBOECAA|1905-08-04|67|292|23|1905|5|8|4|3|1905|23|292|Friday|1905Q3|N|Y|N|2417059|2417270|2416697|2416971|N|N|N|N|N| +2417063|AAAAAAAAHKBOECAA|1905-08-05|67|292|23|1905|6|8|5|3|1905|23|292|Saturday|1905Q3|N|Y|N|2417059|2417270|2416698|2416972|N|N|N|N|N| +2417064|AAAAAAAAIKBOECAA|1905-08-06|67|292|23|1905|0|8|6|3|1905|23|292|Sunday|1905Q3|N|N|N|2417059|2417270|2416699|2416973|N|N|N|N|N| +2417065|AAAAAAAAJKBOECAA|1905-08-07|67|292|23|1905|1|8|7|3|1905|23|292|Monday|1905Q3|N|N|N|2417059|2417270|2416700|2416974|N|N|N|N|N| +2417066|AAAAAAAAKKBOECAA|1905-08-08|67|293|23|1905|2|8|8|3|1905|23|293|Tuesday|1905Q3|N|N|N|2417059|2417270|2416701|2416975|N|N|N|N|N| +2417067|AAAAAAAALKBOECAA|1905-08-09|67|293|23|1905|3|8|9|3|1905|23|293|Wednesday|1905Q3|N|N|N|2417059|2417270|2416702|2416976|N|N|N|N|N| +2417068|AAAAAAAAMKBOECAA|1905-08-10|67|293|23|1905|4|8|10|3|1905|23|293|Thursday|1905Q3|N|N|N|2417059|2417270|2416703|2416977|N|N|N|N|N| +2417069|AAAAAAAANKBOECAA|1905-08-11|67|293|23|1905|5|8|11|3|1905|23|293|Friday|1905Q3|N|Y|N|2417059|2417270|2416704|2416978|N|N|N|N|N| +2417070|AAAAAAAAOKBOECAA|1905-08-12|67|293|23|1905|6|8|12|3|1905|23|293|Saturday|1905Q3|N|Y|N|2417059|2417270|2416705|2416979|N|N|N|N|N| +2417071|AAAAAAAAPKBOECAA|1905-08-13|67|293|23|1905|0|8|13|3|1905|23|293|Sunday|1905Q3|N|N|N|2417059|2417270|2416706|2416980|N|N|N|N|N| +2417072|AAAAAAAAALBOECAA|1905-08-14|67|293|23|1905|1|8|14|3|1905|23|293|Monday|1905Q3|N|N|N|2417059|2417270|2416707|2416981|N|N|N|N|N| +2417073|AAAAAAAABLBOECAA|1905-08-15|67|294|23|1905|2|8|15|3|1905|23|294|Tuesday|1905Q3|N|N|N|2417059|2417270|2416708|2416982|N|N|N|N|N| +2417074|AAAAAAAACLBOECAA|1905-08-16|67|294|23|1905|3|8|16|3|1905|23|294|Wednesday|1905Q3|N|N|N|2417059|2417270|2416709|2416983|N|N|N|N|N| +2417075|AAAAAAAADLBOECAA|1905-08-17|67|294|23|1905|4|8|17|3|1905|23|294|Thursday|1905Q3|N|N|N|2417059|2417270|2416710|2416984|N|N|N|N|N| +2417076|AAAAAAAAELBOECAA|1905-08-18|67|294|23|1905|5|8|18|3|1905|23|294|Friday|1905Q3|N|Y|N|2417059|2417270|2416711|2416985|N|N|N|N|N| +2417077|AAAAAAAAFLBOECAA|1905-08-19|67|294|23|1905|6|8|19|3|1905|23|294|Saturday|1905Q3|N|Y|N|2417059|2417270|2416712|2416986|N|N|N|N|N| +2417078|AAAAAAAAGLBOECAA|1905-08-20|67|294|23|1905|0|8|20|3|1905|23|294|Sunday|1905Q3|N|N|N|2417059|2417270|2416713|2416987|N|N|N|N|N| +2417079|AAAAAAAAHLBOECAA|1905-08-21|67|294|23|1905|1|8|21|3|1905|23|294|Monday|1905Q3|N|N|N|2417059|2417270|2416714|2416988|N|N|N|N|N| +2417080|AAAAAAAAILBOECAA|1905-08-22|67|295|23|1905|2|8|22|3|1905|23|295|Tuesday|1905Q3|N|N|N|2417059|2417270|2416715|2416989|N|N|N|N|N| +2417081|AAAAAAAAJLBOECAA|1905-08-23|67|295|23|1905|3|8|23|3|1905|23|295|Wednesday|1905Q3|N|N|N|2417059|2417270|2416716|2416990|N|N|N|N|N| +2417082|AAAAAAAAKLBOECAA|1905-08-24|67|295|23|1905|4|8|24|3|1905|23|295|Thursday|1905Q3|N|N|N|2417059|2417270|2416717|2416991|N|N|N|N|N| +2417083|AAAAAAAALLBOECAA|1905-08-25|67|295|23|1905|5|8|25|3|1905|23|295|Friday|1905Q3|N|Y|N|2417059|2417270|2416718|2416992|N|N|N|N|N| +2417084|AAAAAAAAMLBOECAA|1905-08-26|67|295|23|1905|6|8|26|3|1905|23|295|Saturday|1905Q3|N|Y|N|2417059|2417270|2416719|2416993|N|N|N|N|N| +2417085|AAAAAAAANLBOECAA|1905-08-27|67|295|23|1905|0|8|27|3|1905|23|295|Sunday|1905Q3|N|N|N|2417059|2417270|2416720|2416994|N|N|N|N|N| +2417086|AAAAAAAAOLBOECAA|1905-08-28|67|295|23|1905|1|8|28|3|1905|23|295|Monday|1905Q3|N|N|N|2417059|2417270|2416721|2416995|N|N|N|N|N| +2417087|AAAAAAAAPLBOECAA|1905-08-29|67|296|23|1905|2|8|29|3|1905|23|296|Tuesday|1905Q3|N|N|N|2417059|2417270|2416722|2416996|N|N|N|N|N| +2417088|AAAAAAAAAMBOECAA|1905-08-30|67|296|23|1905|3|8|30|3|1905|23|296|Wednesday|1905Q3|N|N|N|2417059|2417270|2416723|2416997|N|N|N|N|N| +2417089|AAAAAAAABMBOECAA|1905-08-31|67|296|23|1905|4|8|31|3|1905|23|296|Thursday|1905Q3|N|N|N|2417059|2417270|2416724|2416998|N|N|N|N|N| +2417090|AAAAAAAACMBOECAA|1905-09-01|68|296|24|1905|5|9|1|3|1905|24|296|Friday|1905Q3|N|Y|N|2417090|2417332|2416725|2416999|N|N|N|N|N| +2417091|AAAAAAAADMBOECAA|1905-09-02|68|296|24|1905|6|9|2|3|1905|24|296|Saturday|1905Q3|N|Y|N|2417090|2417332|2416726|2417000|N|N|N|N|N| +2417092|AAAAAAAAEMBOECAA|1905-09-03|68|296|24|1905|0|9|3|3|1905|24|296|Sunday|1905Q3|N|N|N|2417090|2417332|2416727|2417001|N|N|N|N|N| +2417093|AAAAAAAAFMBOECAA|1905-09-04|68|296|24|1905|1|9|4|3|1905|24|296|Monday|1905Q3|N|N|N|2417090|2417332|2416728|2417002|N|N|N|N|N| +2417094|AAAAAAAAGMBOECAA|1905-09-05|68|297|24|1905|2|9|5|3|1905|24|297|Tuesday|1905Q3|N|N|N|2417090|2417332|2416729|2417003|N|N|N|N|N| +2417095|AAAAAAAAHMBOECAA|1905-09-06|68|297|24|1905|3|9|6|3|1905|24|297|Wednesday|1905Q3|N|N|N|2417090|2417332|2416730|2417004|N|N|N|N|N| +2417096|AAAAAAAAIMBOECAA|1905-09-07|68|297|24|1905|4|9|7|3|1905|24|297|Thursday|1905Q3|N|N|N|2417090|2417332|2416731|2417005|N|N|N|N|N| +2417097|AAAAAAAAJMBOECAA|1905-09-08|68|297|24|1905|5|9|8|3|1905|24|297|Friday|1905Q3|N|Y|N|2417090|2417332|2416732|2417006|N|N|N|N|N| +2417098|AAAAAAAAKMBOECAA|1905-09-09|68|297|24|1905|6|9|9|3|1905|24|297|Saturday|1905Q3|N|Y|N|2417090|2417332|2416733|2417007|N|N|N|N|N| +2417099|AAAAAAAALMBOECAA|1905-09-10|68|297|24|1905|0|9|10|3|1905|24|297|Sunday|1905Q3|N|N|N|2417090|2417332|2416734|2417008|N|N|N|N|N| +2417100|AAAAAAAAMMBOECAA|1905-09-11|68|297|24|1905|1|9|11|3|1905|24|297|Monday|1905Q3|N|N|N|2417090|2417332|2416735|2417009|N|N|N|N|N| +2417101|AAAAAAAANMBOECAA|1905-09-12|68|298|24|1905|2|9|12|3|1905|24|298|Tuesday|1905Q3|N|N|N|2417090|2417332|2416736|2417010|N|N|N|N|N| +2417102|AAAAAAAAOMBOECAA|1905-09-13|68|298|24|1905|3|9|13|3|1905|24|298|Wednesday|1905Q3|N|N|N|2417090|2417332|2416737|2417011|N|N|N|N|N| +2417103|AAAAAAAAPMBOECAA|1905-09-14|68|298|24|1905|4|9|14|3|1905|24|298|Thursday|1905Q3|N|N|N|2417090|2417332|2416738|2417012|N|N|N|N|N| +2417104|AAAAAAAAANBOECAA|1905-09-15|68|298|24|1905|5|9|15|3|1905|24|298|Friday|1905Q3|N|Y|N|2417090|2417332|2416739|2417013|N|N|N|N|N| +2417105|AAAAAAAABNBOECAA|1905-09-16|68|298|24|1905|6|9|16|3|1905|24|298|Saturday|1905Q3|N|Y|N|2417090|2417332|2416740|2417014|N|N|N|N|N| +2417106|AAAAAAAACNBOECAA|1905-09-17|68|298|24|1905|0|9|17|3|1905|24|298|Sunday|1905Q3|N|N|N|2417090|2417332|2416741|2417015|N|N|N|N|N| +2417107|AAAAAAAADNBOECAA|1905-09-18|68|298|24|1905|1|9|18|3|1905|24|298|Monday|1905Q3|N|N|N|2417090|2417332|2416742|2417016|N|N|N|N|N| +2417108|AAAAAAAAENBOECAA|1905-09-19|68|299|24|1905|2|9|19|3|1905|24|299|Tuesday|1905Q3|N|N|N|2417090|2417332|2416743|2417017|N|N|N|N|N| +2417109|AAAAAAAAFNBOECAA|1905-09-20|68|299|24|1905|3|9|20|3|1905|24|299|Wednesday|1905Q3|N|N|N|2417090|2417332|2416744|2417018|N|N|N|N|N| +2417110|AAAAAAAAGNBOECAA|1905-09-21|68|299|24|1905|4|9|21|3|1905|24|299|Thursday|1905Q3|N|N|N|2417090|2417332|2416745|2417019|N|N|N|N|N| +2417111|AAAAAAAAHNBOECAA|1905-09-22|68|299|24|1905|5|9|22|3|1905|24|299|Friday|1905Q3|N|Y|N|2417090|2417332|2416746|2417020|N|N|N|N|N| +2417112|AAAAAAAAINBOECAA|1905-09-23|68|299|24|1905|6|9|23|3|1905|24|299|Saturday|1905Q3|N|Y|N|2417090|2417332|2416747|2417021|N|N|N|N|N| +2417113|AAAAAAAAJNBOECAA|1905-09-24|68|299|24|1905|0|9|24|3|1905|24|299|Sunday|1905Q3|N|N|N|2417090|2417332|2416748|2417022|N|N|N|N|N| +2417114|AAAAAAAAKNBOECAA|1905-09-25|68|299|24|1905|1|9|25|3|1905|24|299|Monday|1905Q3|N|N|N|2417090|2417332|2416749|2417023|N|N|N|N|N| +2417115|AAAAAAAALNBOECAA|1905-09-26|68|300|24|1905|2|9|26|3|1905|24|300|Tuesday|1905Q3|N|N|N|2417090|2417332|2416750|2417024|N|N|N|N|N| +2417116|AAAAAAAAMNBOECAA|1905-09-27|68|300|24|1905|3|9|27|3|1905|24|300|Wednesday|1905Q3|N|N|N|2417090|2417332|2416751|2417025|N|N|N|N|N| +2417117|AAAAAAAANNBOECAA|1905-09-28|68|300|24|1905|4|9|28|3|1905|24|300|Thursday|1905Q3|N|N|N|2417090|2417332|2416752|2417026|N|N|N|N|N| +2417118|AAAAAAAAONBOECAA|1905-09-29|68|300|24|1905|5|9|29|3|1905|24|300|Friday|1905Q3|N|Y|N|2417090|2417332|2416753|2417027|N|N|N|N|N| +2417119|AAAAAAAAPNBOECAA|1905-09-30|68|300|24|1905|6|9|30|3|1905|24|300|Saturday|1905Q3|N|Y|N|2417090|2417332|2416754|2417028|N|N|N|N|N| +2417120|AAAAAAAAAOBOECAA|1905-10-01|69|300|24|1905|0|10|1|3|1905|24|300|Sunday|1905Q3|N|N|N|2417120|2417392|2416755|2417028|N|N|N|N|N| +2417121|AAAAAAAABOBOECAA|1905-10-02|69|300|24|1905|1|10|2|4|1905|24|300|Monday|1905Q4|N|N|N|2417120|2417392|2416756|2417029|N|N|N|N|N| +2417122|AAAAAAAACOBOECAA|1905-10-03|69|301|24|1905|2|10|3|4|1905|24|301|Tuesday|1905Q4|N|N|N|2417120|2417392|2416757|2417030|N|N|N|N|N| +2417123|AAAAAAAADOBOECAA|1905-10-04|69|301|24|1905|3|10|4|4|1905|24|301|Wednesday|1905Q4|N|N|N|2417120|2417392|2416758|2417031|N|N|N|N|N| +2417124|AAAAAAAAEOBOECAA|1905-10-05|69|301|24|1905|4|10|5|4|1905|24|301|Thursday|1905Q4|N|N|N|2417120|2417392|2416759|2417032|N|N|N|N|N| +2417125|AAAAAAAAFOBOECAA|1905-10-06|69|301|24|1905|5|10|6|4|1905|24|301|Friday|1905Q4|N|Y|N|2417120|2417392|2416760|2417033|N|N|N|N|N| +2417126|AAAAAAAAGOBOECAA|1905-10-07|69|301|24|1905|6|10|7|4|1905|24|301|Saturday|1905Q4|N|Y|N|2417120|2417392|2416761|2417034|N|N|N|N|N| +2417127|AAAAAAAAHOBOECAA|1905-10-08|69|301|24|1905|0|10|8|4|1905|24|301|Sunday|1905Q4|N|N|N|2417120|2417392|2416762|2417035|N|N|N|N|N| +2417128|AAAAAAAAIOBOECAA|1905-10-09|69|301|24|1905|1|10|9|4|1905|24|301|Monday|1905Q4|N|N|N|2417120|2417392|2416763|2417036|N|N|N|N|N| +2417129|AAAAAAAAJOBOECAA|1905-10-10|69|302|24|1905|2|10|10|4|1905|24|302|Tuesday|1905Q4|N|N|N|2417120|2417392|2416764|2417037|N|N|N|N|N| +2417130|AAAAAAAAKOBOECAA|1905-10-11|69|302|24|1905|3|10|11|4|1905|24|302|Wednesday|1905Q4|N|N|N|2417120|2417392|2416765|2417038|N|N|N|N|N| +2417131|AAAAAAAALOBOECAA|1905-10-12|69|302|24|1905|4|10|12|4|1905|24|302|Thursday|1905Q4|N|N|N|2417120|2417392|2416766|2417039|N|N|N|N|N| +2417132|AAAAAAAAMOBOECAA|1905-10-13|69|302|24|1905|5|10|13|4|1905|24|302|Friday|1905Q4|N|Y|N|2417120|2417392|2416767|2417040|N|N|N|N|N| +2417133|AAAAAAAANOBOECAA|1905-10-14|69|302|24|1905|6|10|14|4|1905|24|302|Saturday|1905Q4|N|Y|N|2417120|2417392|2416768|2417041|N|N|N|N|N| +2417134|AAAAAAAAOOBOECAA|1905-10-15|69|302|24|1905|0|10|15|4|1905|24|302|Sunday|1905Q4|N|N|N|2417120|2417392|2416769|2417042|N|N|N|N|N| +2417135|AAAAAAAAPOBOECAA|1905-10-16|69|302|24|1905|1|10|16|4|1905|24|302|Monday|1905Q4|N|N|N|2417120|2417392|2416770|2417043|N|N|N|N|N| +2417136|AAAAAAAAAPBOECAA|1905-10-17|69|303|24|1905|2|10|17|4|1905|24|303|Tuesday|1905Q4|N|N|N|2417120|2417392|2416771|2417044|N|N|N|N|N| +2417137|AAAAAAAABPBOECAA|1905-10-18|69|303|24|1905|3|10|18|4|1905|24|303|Wednesday|1905Q4|N|N|N|2417120|2417392|2416772|2417045|N|N|N|N|N| +2417138|AAAAAAAACPBOECAA|1905-10-19|69|303|24|1905|4|10|19|4|1905|24|303|Thursday|1905Q4|N|N|N|2417120|2417392|2416773|2417046|N|N|N|N|N| +2417139|AAAAAAAADPBOECAA|1905-10-20|69|303|24|1905|5|10|20|4|1905|24|303|Friday|1905Q4|N|Y|N|2417120|2417392|2416774|2417047|N|N|N|N|N| +2417140|AAAAAAAAEPBOECAA|1905-10-21|69|303|24|1905|6|10|21|4|1905|24|303|Saturday|1905Q4|N|Y|N|2417120|2417392|2416775|2417048|N|N|N|N|N| +2417141|AAAAAAAAFPBOECAA|1905-10-22|69|303|24|1905|0|10|22|4|1905|24|303|Sunday|1905Q4|N|N|N|2417120|2417392|2416776|2417049|N|N|N|N|N| +2417142|AAAAAAAAGPBOECAA|1905-10-23|69|303|24|1905|1|10|23|4|1905|24|303|Monday|1905Q4|N|N|N|2417120|2417392|2416777|2417050|N|N|N|N|N| +2417143|AAAAAAAAHPBOECAA|1905-10-24|69|304|24|1905|2|10|24|4|1905|24|304|Tuesday|1905Q4|N|N|N|2417120|2417392|2416778|2417051|N|N|N|N|N| +2417144|AAAAAAAAIPBOECAA|1905-10-25|69|304|24|1905|3|10|25|4|1905|24|304|Wednesday|1905Q4|N|N|N|2417120|2417392|2416779|2417052|N|N|N|N|N| +2417145|AAAAAAAAJPBOECAA|1905-10-26|69|304|24|1905|4|10|26|4|1905|24|304|Thursday|1905Q4|N|N|N|2417120|2417392|2416780|2417053|N|N|N|N|N| +2417146|AAAAAAAAKPBOECAA|1905-10-27|69|304|24|1905|5|10|27|4|1905|24|304|Friday|1905Q4|N|Y|N|2417120|2417392|2416781|2417054|N|N|N|N|N| +2417147|AAAAAAAALPBOECAA|1905-10-28|69|304|24|1905|6|10|28|4|1905|24|304|Saturday|1905Q4|N|Y|N|2417120|2417392|2416782|2417055|N|N|N|N|N| +2417148|AAAAAAAAMPBOECAA|1905-10-29|69|304|24|1905|0|10|29|4|1905|24|304|Sunday|1905Q4|N|N|N|2417120|2417392|2416783|2417056|N|N|N|N|N| +2417149|AAAAAAAANPBOECAA|1905-10-30|69|304|24|1905|1|10|30|4|1905|24|304|Monday|1905Q4|N|N|N|2417120|2417392|2416784|2417057|N|N|N|N|N| +2417150|AAAAAAAAOPBOECAA|1905-10-31|69|305|24|1905|2|10|31|4|1905|24|305|Tuesday|1905Q4|N|N|N|2417120|2417392|2416785|2417058|N|N|N|N|N| +2417151|AAAAAAAAPPBOECAA|1905-11-01|70|305|24|1905|3|11|1|4|1905|24|305|Wednesday|1905Q4|N|N|N|2417151|2417454|2416786|2417059|N|N|N|N|N| +2417152|AAAAAAAAAACOECAA|1905-11-02|70|305|24|1905|4|11|2|4|1905|24|305|Thursday|1905Q4|N|N|N|2417151|2417454|2416787|2417060|N|N|N|N|N| +2417153|AAAAAAAABACOECAA|1905-11-03|70|305|24|1905|5|11|3|4|1905|24|305|Friday|1905Q4|N|Y|N|2417151|2417454|2416788|2417061|N|N|N|N|N| +2417154|AAAAAAAACACOECAA|1905-11-04|70|305|24|1905|6|11|4|4|1905|24|305|Saturday|1905Q4|N|Y|N|2417151|2417454|2416789|2417062|N|N|N|N|N| +2417155|AAAAAAAADACOECAA|1905-11-05|70|305|24|1905|0|11|5|4|1905|24|305|Sunday|1905Q4|N|N|N|2417151|2417454|2416790|2417063|N|N|N|N|N| +2417156|AAAAAAAAEACOECAA|1905-11-06|70|305|24|1905|1|11|6|4|1905|24|305|Monday|1905Q4|N|N|N|2417151|2417454|2416791|2417064|N|N|N|N|N| +2417157|AAAAAAAAFACOECAA|1905-11-07|70|306|24|1905|2|11|7|4|1905|24|306|Tuesday|1905Q4|N|N|N|2417151|2417454|2416792|2417065|N|N|N|N|N| +2417158|AAAAAAAAGACOECAA|1905-11-08|70|306|24|1905|3|11|8|4|1905|24|306|Wednesday|1905Q4|N|N|N|2417151|2417454|2416793|2417066|N|N|N|N|N| +2417159|AAAAAAAAHACOECAA|1905-11-09|70|306|24|1905|4|11|9|4|1905|24|306|Thursday|1905Q4|N|N|N|2417151|2417454|2416794|2417067|N|N|N|N|N| +2417160|AAAAAAAAIACOECAA|1905-11-10|70|306|24|1905|5|11|10|4|1905|24|306|Friday|1905Q4|N|Y|N|2417151|2417454|2416795|2417068|N|N|N|N|N| +2417161|AAAAAAAAJACOECAA|1905-11-11|70|306|24|1905|6|11|11|4|1905|24|306|Saturday|1905Q4|N|Y|N|2417151|2417454|2416796|2417069|N|N|N|N|N| +2417162|AAAAAAAAKACOECAA|1905-11-12|70|306|24|1905|0|11|12|4|1905|24|306|Sunday|1905Q4|N|N|N|2417151|2417454|2416797|2417070|N|N|N|N|N| +2417163|AAAAAAAALACOECAA|1905-11-13|70|306|24|1905|1|11|13|4|1905|24|306|Monday|1905Q4|N|N|N|2417151|2417454|2416798|2417071|N|N|N|N|N| +2417164|AAAAAAAAMACOECAA|1905-11-14|70|307|24|1905|2|11|14|4|1905|24|307|Tuesday|1905Q4|N|N|N|2417151|2417454|2416799|2417072|N|N|N|N|N| +2417165|AAAAAAAANACOECAA|1905-11-15|70|307|24|1905|3|11|15|4|1905|24|307|Wednesday|1905Q4|N|N|N|2417151|2417454|2416800|2417073|N|N|N|N|N| +2417166|AAAAAAAAOACOECAA|1905-11-16|70|307|24|1905|4|11|16|4|1905|24|307|Thursday|1905Q4|N|N|N|2417151|2417454|2416801|2417074|N|N|N|N|N| +2417167|AAAAAAAAPACOECAA|1905-11-17|70|307|24|1905|5|11|17|4|1905|24|307|Friday|1905Q4|N|Y|N|2417151|2417454|2416802|2417075|N|N|N|N|N| +2417168|AAAAAAAAABCOECAA|1905-11-18|70|307|24|1905|6|11|18|4|1905|24|307|Saturday|1905Q4|N|Y|N|2417151|2417454|2416803|2417076|N|N|N|N|N| +2417169|AAAAAAAABBCOECAA|1905-11-19|70|307|24|1905|0|11|19|4|1905|24|307|Sunday|1905Q4|N|N|N|2417151|2417454|2416804|2417077|N|N|N|N|N| +2417170|AAAAAAAACBCOECAA|1905-11-20|70|307|24|1905|1|11|20|4|1905|24|307|Monday|1905Q4|N|N|N|2417151|2417454|2416805|2417078|N|N|N|N|N| +2417171|AAAAAAAADBCOECAA|1905-11-21|70|308|24|1905|2|11|21|4|1905|24|308|Tuesday|1905Q4|N|N|N|2417151|2417454|2416806|2417079|N|N|N|N|N| +2417172|AAAAAAAAEBCOECAA|1905-11-22|70|308|24|1905|3|11|22|4|1905|24|308|Wednesday|1905Q4|N|N|N|2417151|2417454|2416807|2417080|N|N|N|N|N| +2417173|AAAAAAAAFBCOECAA|1905-11-23|70|308|24|1905|4|11|23|4|1905|24|308|Thursday|1905Q4|N|N|N|2417151|2417454|2416808|2417081|N|N|N|N|N| +2417174|AAAAAAAAGBCOECAA|1905-11-24|70|308|24|1905|5|11|24|4|1905|24|308|Friday|1905Q4|N|Y|N|2417151|2417454|2416809|2417082|N|N|N|N|N| +2417175|AAAAAAAAHBCOECAA|1905-11-25|70|308|24|1905|6|11|25|4|1905|24|308|Saturday|1905Q4|N|Y|N|2417151|2417454|2416810|2417083|N|N|N|N|N| +2417176|AAAAAAAAIBCOECAA|1905-11-26|70|308|24|1905|0|11|26|4|1905|24|308|Sunday|1905Q4|N|N|N|2417151|2417454|2416811|2417084|N|N|N|N|N| +2417177|AAAAAAAAJBCOECAA|1905-11-27|70|308|24|1905|1|11|27|4|1905|24|308|Monday|1905Q4|N|N|N|2417151|2417454|2416812|2417085|N|N|N|N|N| +2417178|AAAAAAAAKBCOECAA|1905-11-28|70|309|24|1905|2|11|28|4|1905|24|309|Tuesday|1905Q4|N|N|N|2417151|2417454|2416813|2417086|N|N|N|N|N| +2417179|AAAAAAAALBCOECAA|1905-11-29|70|309|24|1905|3|11|29|4|1905|24|309|Wednesday|1905Q4|N|N|N|2417151|2417454|2416814|2417087|N|N|N|N|N| +2417180|AAAAAAAAMBCOECAA|1905-11-30|70|309|24|1905|4|11|30|4|1905|24|309|Thursday|1905Q4|N|N|N|2417151|2417454|2416815|2417088|N|N|N|N|N| +2417181|AAAAAAAANBCOECAA|1905-12-01|71|309|25|1905|5|12|1|4|1905|25|309|Friday|1905Q4|N|Y|N|2417181|2417514|2416816|2417089|N|N|N|N|N| +2417182|AAAAAAAAOBCOECAA|1905-12-02|71|309|25|1905|6|12|2|4|1905|25|309|Saturday|1905Q4|N|Y|N|2417181|2417514|2416817|2417090|N|N|N|N|N| +2417183|AAAAAAAAPBCOECAA|1905-12-03|71|309|25|1905|0|12|3|4|1905|25|309|Sunday|1905Q4|N|N|N|2417181|2417514|2416818|2417091|N|N|N|N|N| +2417184|AAAAAAAAACCOECAA|1905-12-04|71|309|25|1905|1|12|4|4|1905|25|309|Monday|1905Q4|N|N|N|2417181|2417514|2416819|2417092|N|N|N|N|N| +2417185|AAAAAAAABCCOECAA|1905-12-05|71|310|25|1905|2|12|5|4|1905|25|310|Tuesday|1905Q4|N|N|N|2417181|2417514|2416820|2417093|N|N|N|N|N| +2417186|AAAAAAAACCCOECAA|1905-12-06|71|310|25|1905|3|12|6|4|1905|25|310|Wednesday|1905Q4|N|N|N|2417181|2417514|2416821|2417094|N|N|N|N|N| +2417187|AAAAAAAADCCOECAA|1905-12-07|71|310|25|1905|4|12|7|4|1905|25|310|Thursday|1905Q4|N|N|N|2417181|2417514|2416822|2417095|N|N|N|N|N| +2417188|AAAAAAAAECCOECAA|1905-12-08|71|310|25|1905|5|12|8|4|1905|25|310|Friday|1905Q4|N|Y|N|2417181|2417514|2416823|2417096|N|N|N|N|N| +2417189|AAAAAAAAFCCOECAA|1905-12-09|71|310|25|1905|6|12|9|4|1905|25|310|Saturday|1905Q4|N|Y|N|2417181|2417514|2416824|2417097|N|N|N|N|N| +2417190|AAAAAAAAGCCOECAA|1905-12-10|71|310|25|1905|0|12|10|4|1905|25|310|Sunday|1905Q4|N|N|N|2417181|2417514|2416825|2417098|N|N|N|N|N| +2417191|AAAAAAAAHCCOECAA|1905-12-11|71|310|25|1905|1|12|11|4|1905|25|310|Monday|1905Q4|N|N|N|2417181|2417514|2416826|2417099|N|N|N|N|N| +2417192|AAAAAAAAICCOECAA|1905-12-12|71|311|25|1905|2|12|12|4|1905|25|311|Tuesday|1905Q4|N|N|N|2417181|2417514|2416827|2417100|N|N|N|N|N| +2417193|AAAAAAAAJCCOECAA|1905-12-13|71|311|25|1905|3|12|13|4|1905|25|311|Wednesday|1905Q4|N|N|N|2417181|2417514|2416828|2417101|N|N|N|N|N| +2417194|AAAAAAAAKCCOECAA|1905-12-14|71|311|25|1905|4|12|14|4|1905|25|311|Thursday|1905Q4|N|N|N|2417181|2417514|2416829|2417102|N|N|N|N|N| +2417195|AAAAAAAALCCOECAA|1905-12-15|71|311|25|1905|5|12|15|4|1905|25|311|Friday|1905Q4|N|Y|N|2417181|2417514|2416830|2417103|N|N|N|N|N| +2417196|AAAAAAAAMCCOECAA|1905-12-16|71|311|25|1905|6|12|16|4|1905|25|311|Saturday|1905Q4|N|Y|N|2417181|2417514|2416831|2417104|N|N|N|N|N| +2417197|AAAAAAAANCCOECAA|1905-12-17|71|311|25|1905|0|12|17|4|1905|25|311|Sunday|1905Q4|N|N|N|2417181|2417514|2416832|2417105|N|N|N|N|N| +2417198|AAAAAAAAOCCOECAA|1905-12-18|71|311|25|1905|1|12|18|4|1905|25|311|Monday|1905Q4|N|N|N|2417181|2417514|2416833|2417106|N|N|N|N|N| +2417199|AAAAAAAAPCCOECAA|1905-12-19|71|312|25|1905|2|12|19|4|1905|25|312|Tuesday|1905Q4|N|N|N|2417181|2417514|2416834|2417107|N|N|N|N|N| +2417200|AAAAAAAAADCOECAA|1905-12-20|71|312|25|1905|3|12|20|4|1905|25|312|Wednesday|1905Q4|N|N|N|2417181|2417514|2416835|2417108|N|N|N|N|N| +2417201|AAAAAAAABDCOECAA|1905-12-21|71|312|25|1905|4|12|21|4|1905|25|312|Thursday|1905Q4|N|N|N|2417181|2417514|2416836|2417109|N|N|N|N|N| +2417202|AAAAAAAACDCOECAA|1905-12-22|71|312|25|1905|5|12|22|4|1905|25|312|Friday|1905Q4|N|Y|N|2417181|2417514|2416837|2417110|N|N|N|N|N| +2417203|AAAAAAAADDCOECAA|1905-12-23|71|312|25|1905|6|12|23|4|1905|25|312|Saturday|1905Q4|N|Y|N|2417181|2417514|2416838|2417111|N|N|N|N|N| +2417204|AAAAAAAAEDCOECAA|1905-12-24|71|312|25|1905|0|12|24|4|1905|25|312|Sunday|1905Q4|N|N|N|2417181|2417514|2416839|2417112|N|N|N|N|N| +2417205|AAAAAAAAFDCOECAA|1905-12-25|71|312|25|1905|1|12|25|4|1905|25|312|Monday|1905Q4|N|N|N|2417181|2417514|2416840|2417113|N|N|N|N|N| +2417206|AAAAAAAAGDCOECAA|1905-12-26|71|313|25|1905|2|12|26|4|1905|25|313|Tuesday|1905Q4|Y|N|N|2417181|2417514|2416841|2417114|N|N|N|N|N| +2417207|AAAAAAAAHDCOECAA|1905-12-27|71|313|25|1905|3|12|27|4|1905|25|313|Wednesday|1905Q4|N|N|Y|2417181|2417514|2416842|2417115|N|N|N|N|N| +2417208|AAAAAAAAIDCOECAA|1905-12-28|71|313|25|1905|4|12|28|4|1905|25|313|Thursday|1905Q4|N|N|N|2417181|2417514|2416843|2417116|N|N|N|N|N| +2417209|AAAAAAAAJDCOECAA|1905-12-29|71|313|25|1905|5|12|29|4|1905|25|313|Friday|1905Q4|N|Y|N|2417181|2417514|2416844|2417117|N|N|N|N|N| +2417210|AAAAAAAAKDCOECAA|1905-12-30|71|313|25|1905|6|12|30|4|1905|25|313|Saturday|1905Q4|N|Y|N|2417181|2417514|2416845|2417118|N|N|N|N|N| +2417211|AAAAAAAALDCOECAA|1905-12-31|71|313|25|1905|0|12|31|4|1905|25|313|Sunday|1905Q4|N|N|N|2417181|2417514|2416846|2417119|N|N|N|N|N| +2417212|AAAAAAAAMDCOECAA|1906-01-01|72|313|25|1906|1|1|1|1|1906|25|313|Monday|1906Q1|Y|N|N|2417212|2417211|2416847|2417120|N|N|N|N|N| +2417213|AAAAAAAANDCOECAA|1906-01-02|72|314|25|1906|2|1|2|1|1906|25|314|Tuesday|1906Q1|N|N|Y|2417212|2417211|2416848|2417121|N|N|N|N|N| +2417214|AAAAAAAAODCOECAA|1906-01-03|72|314|25|1906|3|1|3|1|1906|25|314|Wednesday|1906Q1|N|N|N|2417212|2417211|2416849|2417122|N|N|N|N|N| +2417215|AAAAAAAAPDCOECAA|1906-01-04|72|314|25|1906|4|1|4|1|1906|25|314|Thursday|1906Q1|N|N|N|2417212|2417211|2416850|2417123|N|N|N|N|N| +2417216|AAAAAAAAAECOECAA|1906-01-05|72|314|25|1906|5|1|5|1|1906|25|314|Friday|1906Q1|N|Y|N|2417212|2417211|2416851|2417124|N|N|N|N|N| +2417217|AAAAAAAABECOECAA|1906-01-06|72|314|25|1906|6|1|6|1|1906|25|314|Saturday|1906Q1|N|Y|N|2417212|2417211|2416852|2417125|N|N|N|N|N| +2417218|AAAAAAAACECOECAA|1906-01-07|72|314|25|1906|0|1|7|1|1906|25|314|Sunday|1906Q1|N|N|N|2417212|2417211|2416853|2417126|N|N|N|N|N| +2417219|AAAAAAAADECOECAA|1906-01-08|72|314|25|1906|1|1|8|1|1906|25|314|Monday|1906Q1|N|N|N|2417212|2417211|2416854|2417127|N|N|N|N|N| +2417220|AAAAAAAAEECOECAA|1906-01-09|72|315|25|1906|2|1|9|1|1906|25|315|Tuesday|1906Q1|N|N|N|2417212|2417211|2416855|2417128|N|N|N|N|N| +2417221|AAAAAAAAFECOECAA|1906-01-10|72|315|25|1906|3|1|10|1|1906|25|315|Wednesday|1906Q1|N|N|N|2417212|2417211|2416856|2417129|N|N|N|N|N| +2417222|AAAAAAAAGECOECAA|1906-01-11|72|315|25|1906|4|1|11|1|1906|25|315|Thursday|1906Q1|N|N|N|2417212|2417211|2416857|2417130|N|N|N|N|N| +2417223|AAAAAAAAHECOECAA|1906-01-12|72|315|25|1906|5|1|12|1|1906|25|315|Friday|1906Q1|N|Y|N|2417212|2417211|2416858|2417131|N|N|N|N|N| +2417224|AAAAAAAAIECOECAA|1906-01-13|72|315|25|1906|6|1|13|1|1906|25|315|Saturday|1906Q1|N|Y|N|2417212|2417211|2416859|2417132|N|N|N|N|N| +2417225|AAAAAAAAJECOECAA|1906-01-14|72|315|25|1906|0|1|14|1|1906|25|315|Sunday|1906Q1|N|N|N|2417212|2417211|2416860|2417133|N|N|N|N|N| +2417226|AAAAAAAAKECOECAA|1906-01-15|72|315|25|1906|1|1|15|1|1906|25|315|Monday|1906Q1|N|N|N|2417212|2417211|2416861|2417134|N|N|N|N|N| +2417227|AAAAAAAALECOECAA|1906-01-16|72|316|25|1906|2|1|16|1|1906|25|316|Tuesday|1906Q1|N|N|N|2417212|2417211|2416862|2417135|N|N|N|N|N| +2417228|AAAAAAAAMECOECAA|1906-01-17|72|316|25|1906|3|1|17|1|1906|25|316|Wednesday|1906Q1|N|N|N|2417212|2417211|2416863|2417136|N|N|N|N|N| +2417229|AAAAAAAANECOECAA|1906-01-18|72|316|25|1906|4|1|18|1|1906|25|316|Thursday|1906Q1|N|N|N|2417212|2417211|2416864|2417137|N|N|N|N|N| +2417230|AAAAAAAAOECOECAA|1906-01-19|72|316|25|1906|5|1|19|1|1906|25|316|Friday|1906Q1|N|Y|N|2417212|2417211|2416865|2417138|N|N|N|N|N| +2417231|AAAAAAAAPECOECAA|1906-01-20|72|316|25|1906|6|1|20|1|1906|25|316|Saturday|1906Q1|N|Y|N|2417212|2417211|2416866|2417139|N|N|N|N|N| +2417232|AAAAAAAAAFCOECAA|1906-01-21|72|316|25|1906|0|1|21|1|1906|25|316|Sunday|1906Q1|N|N|N|2417212|2417211|2416867|2417140|N|N|N|N|N| +2417233|AAAAAAAABFCOECAA|1906-01-22|72|316|25|1906|1|1|22|1|1906|25|316|Monday|1906Q1|N|N|N|2417212|2417211|2416868|2417141|N|N|N|N|N| +2417234|AAAAAAAACFCOECAA|1906-01-23|72|317|25|1906|2|1|23|1|1906|25|317|Tuesday|1906Q1|N|N|N|2417212|2417211|2416869|2417142|N|N|N|N|N| +2417235|AAAAAAAADFCOECAA|1906-01-24|72|317|25|1906|3|1|24|1|1906|25|317|Wednesday|1906Q1|N|N|N|2417212|2417211|2416870|2417143|N|N|N|N|N| +2417236|AAAAAAAAEFCOECAA|1906-01-25|72|317|25|1906|4|1|25|1|1906|25|317|Thursday|1906Q1|N|N|N|2417212|2417211|2416871|2417144|N|N|N|N|N| +2417237|AAAAAAAAFFCOECAA|1906-01-26|72|317|25|1906|5|1|26|1|1906|25|317|Friday|1906Q1|N|Y|N|2417212|2417211|2416872|2417145|N|N|N|N|N| +2417238|AAAAAAAAGFCOECAA|1906-01-27|72|317|25|1906|6|1|27|1|1906|25|317|Saturday|1906Q1|N|Y|N|2417212|2417211|2416873|2417146|N|N|N|N|N| +2417239|AAAAAAAAHFCOECAA|1906-01-28|72|317|25|1906|0|1|28|1|1906|25|317|Sunday|1906Q1|N|N|N|2417212|2417211|2416874|2417147|N|N|N|N|N| +2417240|AAAAAAAAIFCOECAA|1906-01-29|72|317|25|1906|1|1|29|1|1906|25|317|Monday|1906Q1|N|N|N|2417212|2417211|2416875|2417148|N|N|N|N|N| +2417241|AAAAAAAAJFCOECAA|1906-01-30|72|318|25|1906|2|1|30|1|1906|25|318|Tuesday|1906Q1|N|N|N|2417212|2417211|2416876|2417149|N|N|N|N|N| +2417242|AAAAAAAAKFCOECAA|1906-01-31|72|318|25|1906|3|1|31|1|1906|25|318|Wednesday|1906Q1|N|N|N|2417212|2417211|2416877|2417150|N|N|N|N|N| +2417243|AAAAAAAALFCOECAA|1906-02-01|73|318|25|1906|4|2|1|1|1906|25|318|Thursday|1906Q1|N|N|N|2417243|2417273|2416878|2417151|N|N|N|N|N| +2417244|AAAAAAAAMFCOECAA|1906-02-02|73|318|25|1906|5|2|2|1|1906|25|318|Friday|1906Q1|N|Y|N|2417243|2417273|2416879|2417152|N|N|N|N|N| +2417245|AAAAAAAANFCOECAA|1906-02-03|73|318|25|1906|6|2|3|1|1906|25|318|Saturday|1906Q1|N|Y|N|2417243|2417273|2416880|2417153|N|N|N|N|N| +2417246|AAAAAAAAOFCOECAA|1906-02-04|73|318|25|1906|0|2|4|1|1906|25|318|Sunday|1906Q1|N|N|N|2417243|2417273|2416881|2417154|N|N|N|N|N| +2417247|AAAAAAAAPFCOECAA|1906-02-05|73|318|25|1906|1|2|5|1|1906|25|318|Monday|1906Q1|N|N|N|2417243|2417273|2416882|2417155|N|N|N|N|N| +2417248|AAAAAAAAAGCOECAA|1906-02-06|73|319|25|1906|2|2|6|1|1906|25|319|Tuesday|1906Q1|N|N|N|2417243|2417273|2416883|2417156|N|N|N|N|N| +2417249|AAAAAAAABGCOECAA|1906-02-07|73|319|25|1906|3|2|7|1|1906|25|319|Wednesday|1906Q1|N|N|N|2417243|2417273|2416884|2417157|N|N|N|N|N| +2417250|AAAAAAAACGCOECAA|1906-02-08|73|319|25|1906|4|2|8|1|1906|25|319|Thursday|1906Q1|N|N|N|2417243|2417273|2416885|2417158|N|N|N|N|N| +2417251|AAAAAAAADGCOECAA|1906-02-09|73|319|25|1906|5|2|9|1|1906|25|319|Friday|1906Q1|N|Y|N|2417243|2417273|2416886|2417159|N|N|N|N|N| +2417252|AAAAAAAAEGCOECAA|1906-02-10|73|319|25|1906|6|2|10|1|1906|25|319|Saturday|1906Q1|N|Y|N|2417243|2417273|2416887|2417160|N|N|N|N|N| +2417253|AAAAAAAAFGCOECAA|1906-02-11|73|319|25|1906|0|2|11|1|1906|25|319|Sunday|1906Q1|N|N|N|2417243|2417273|2416888|2417161|N|N|N|N|N| +2417254|AAAAAAAAGGCOECAA|1906-02-12|73|319|25|1906|1|2|12|1|1906|25|319|Monday|1906Q1|N|N|N|2417243|2417273|2416889|2417162|N|N|N|N|N| +2417255|AAAAAAAAHGCOECAA|1906-02-13|73|320|25|1906|2|2|13|1|1906|25|320|Tuesday|1906Q1|N|N|N|2417243|2417273|2416890|2417163|N|N|N|N|N| +2417256|AAAAAAAAIGCOECAA|1906-02-14|73|320|25|1906|3|2|14|1|1906|25|320|Wednesday|1906Q1|N|N|N|2417243|2417273|2416891|2417164|N|N|N|N|N| +2417257|AAAAAAAAJGCOECAA|1906-02-15|73|320|25|1906|4|2|15|1|1906|25|320|Thursday|1906Q1|N|N|N|2417243|2417273|2416892|2417165|N|N|N|N|N| +2417258|AAAAAAAAKGCOECAA|1906-02-16|73|320|25|1906|5|2|16|1|1906|25|320|Friday|1906Q1|N|Y|N|2417243|2417273|2416893|2417166|N|N|N|N|N| +2417259|AAAAAAAALGCOECAA|1906-02-17|73|320|25|1906|6|2|17|1|1906|25|320|Saturday|1906Q1|N|Y|N|2417243|2417273|2416894|2417167|N|N|N|N|N| +2417260|AAAAAAAAMGCOECAA|1906-02-18|73|320|25|1906|0|2|18|1|1906|25|320|Sunday|1906Q1|N|N|N|2417243|2417273|2416895|2417168|N|N|N|N|N| +2417261|AAAAAAAANGCOECAA|1906-02-19|73|320|25|1906|1|2|19|1|1906|25|320|Monday|1906Q1|N|N|N|2417243|2417273|2416896|2417169|N|N|N|N|N| +2417262|AAAAAAAAOGCOECAA|1906-02-20|73|321|25|1906|2|2|20|1|1906|25|321|Tuesday|1906Q1|N|N|N|2417243|2417273|2416897|2417170|N|N|N|N|N| +2417263|AAAAAAAAPGCOECAA|1906-02-21|73|321|25|1906|3|2|21|1|1906|25|321|Wednesday|1906Q1|N|N|N|2417243|2417273|2416898|2417171|N|N|N|N|N| +2417264|AAAAAAAAAHCOECAA|1906-02-22|73|321|25|1906|4|2|22|1|1906|25|321|Thursday|1906Q1|N|N|N|2417243|2417273|2416899|2417172|N|N|N|N|N| +2417265|AAAAAAAABHCOECAA|1906-02-23|73|321|25|1906|5|2|23|1|1906|25|321|Friday|1906Q1|N|Y|N|2417243|2417273|2416900|2417173|N|N|N|N|N| +2417266|AAAAAAAACHCOECAA|1906-02-24|73|321|25|1906|6|2|24|1|1906|25|321|Saturday|1906Q1|N|Y|N|2417243|2417273|2416901|2417174|N|N|N|N|N| +2417267|AAAAAAAADHCOECAA|1906-02-25|73|321|25|1906|0|2|25|1|1906|25|321|Sunday|1906Q1|N|N|N|2417243|2417273|2416902|2417175|N|N|N|N|N| +2417268|AAAAAAAAEHCOECAA|1906-02-26|73|321|25|1906|1|2|26|1|1906|25|321|Monday|1906Q1|N|N|N|2417243|2417273|2416903|2417176|N|N|N|N|N| +2417269|AAAAAAAAFHCOECAA|1906-02-27|73|322|25|1906|2|2|27|1|1906|25|322|Tuesday|1906Q1|N|N|N|2417243|2417273|2416904|2417177|N|N|N|N|N| +2417270|AAAAAAAAGHCOECAA|1906-02-28|73|322|25|1906|3|2|28|1|1906|25|322|Wednesday|1906Q1|N|N|N|2417243|2417273|2416905|2417178|N|N|N|N|N| +2417271|AAAAAAAAHHCOECAA|1906-03-01|74|322|26|1906|4|3|1|1|1906|26|322|Thursday|1906Q1|N|N|N|2417271|2417329|2416906|2417179|N|N|N|N|N| +2417272|AAAAAAAAIHCOECAA|1906-03-02|74|322|26|1906|5|3|2|1|1906|26|322|Friday|1906Q1|N|Y|N|2417271|2417329|2416907|2417180|N|N|N|N|N| +2417273|AAAAAAAAJHCOECAA|1906-03-03|74|322|26|1906|6|3|3|1|1906|26|322|Saturday|1906Q1|N|Y|N|2417271|2417329|2416908|2417181|N|N|N|N|N| +2417274|AAAAAAAAKHCOECAA|1906-03-04|74|322|26|1906|0|3|4|1|1906|26|322|Sunday|1906Q1|N|N|N|2417271|2417329|2416909|2417182|N|N|N|N|N| +2417275|AAAAAAAALHCOECAA|1906-03-05|74|322|26|1906|1|3|5|1|1906|26|322|Monday|1906Q1|N|N|N|2417271|2417329|2416910|2417183|N|N|N|N|N| +2417276|AAAAAAAAMHCOECAA|1906-03-06|74|323|26|1906|2|3|6|1|1906|26|323|Tuesday|1906Q1|N|N|N|2417271|2417329|2416911|2417184|N|N|N|N|N| +2417277|AAAAAAAANHCOECAA|1906-03-07|74|323|26|1906|3|3|7|1|1906|26|323|Wednesday|1906Q1|N|N|N|2417271|2417329|2416912|2417185|N|N|N|N|N| +2417278|AAAAAAAAOHCOECAA|1906-03-08|74|323|26|1906|4|3|8|1|1906|26|323|Thursday|1906Q1|N|N|N|2417271|2417329|2416913|2417186|N|N|N|N|N| +2417279|AAAAAAAAPHCOECAA|1906-03-09|74|323|26|1906|5|3|9|1|1906|26|323|Friday|1906Q1|N|Y|N|2417271|2417329|2416914|2417187|N|N|N|N|N| +2417280|AAAAAAAAAICOECAA|1906-03-10|74|323|26|1906|6|3|10|1|1906|26|323|Saturday|1906Q1|N|Y|N|2417271|2417329|2416915|2417188|N|N|N|N|N| +2417281|AAAAAAAABICOECAA|1906-03-11|74|323|26|1906|0|3|11|1|1906|26|323|Sunday|1906Q1|N|N|N|2417271|2417329|2416916|2417189|N|N|N|N|N| +2417282|AAAAAAAACICOECAA|1906-03-12|74|323|26|1906|1|3|12|1|1906|26|323|Monday|1906Q1|N|N|N|2417271|2417329|2416917|2417190|N|N|N|N|N| +2417283|AAAAAAAADICOECAA|1906-03-13|74|324|26|1906|2|3|13|1|1906|26|324|Tuesday|1906Q1|N|N|N|2417271|2417329|2416918|2417191|N|N|N|N|N| +2417284|AAAAAAAAEICOECAA|1906-03-14|74|324|26|1906|3|3|14|1|1906|26|324|Wednesday|1906Q1|N|N|N|2417271|2417329|2416919|2417192|N|N|N|N|N| +2417285|AAAAAAAAFICOECAA|1906-03-15|74|324|26|1906|4|3|15|1|1906|26|324|Thursday|1906Q1|N|N|N|2417271|2417329|2416920|2417193|N|N|N|N|N| +2417286|AAAAAAAAGICOECAA|1906-03-16|74|324|26|1906|5|3|16|1|1906|26|324|Friday|1906Q1|N|Y|N|2417271|2417329|2416921|2417194|N|N|N|N|N| +2417287|AAAAAAAAHICOECAA|1906-03-17|74|324|26|1906|6|3|17|1|1906|26|324|Saturday|1906Q1|N|Y|N|2417271|2417329|2416922|2417195|N|N|N|N|N| +2417288|AAAAAAAAIICOECAA|1906-03-18|74|324|26|1906|0|3|18|1|1906|26|324|Sunday|1906Q1|N|N|N|2417271|2417329|2416923|2417196|N|N|N|N|N| +2417289|AAAAAAAAJICOECAA|1906-03-19|74|324|26|1906|1|3|19|1|1906|26|324|Monday|1906Q1|N|N|N|2417271|2417329|2416924|2417197|N|N|N|N|N| +2417290|AAAAAAAAKICOECAA|1906-03-20|74|325|26|1906|2|3|20|1|1906|26|325|Tuesday|1906Q1|N|N|N|2417271|2417329|2416925|2417198|N|N|N|N|N| +2417291|AAAAAAAALICOECAA|1906-03-21|74|325|26|1906|3|3|21|1|1906|26|325|Wednesday|1906Q1|N|N|N|2417271|2417329|2416926|2417199|N|N|N|N|N| +2417292|AAAAAAAAMICOECAA|1906-03-22|74|325|26|1906|4|3|22|1|1906|26|325|Thursday|1906Q1|N|N|N|2417271|2417329|2416927|2417200|N|N|N|N|N| +2417293|AAAAAAAANICOECAA|1906-03-23|74|325|26|1906|5|3|23|1|1906|26|325|Friday|1906Q1|N|Y|N|2417271|2417329|2416928|2417201|N|N|N|N|N| +2417294|AAAAAAAAOICOECAA|1906-03-24|74|325|26|1906|6|3|24|1|1906|26|325|Saturday|1906Q1|N|Y|N|2417271|2417329|2416929|2417202|N|N|N|N|N| +2417295|AAAAAAAAPICOECAA|1906-03-25|74|325|26|1906|0|3|25|1|1906|26|325|Sunday|1906Q1|N|N|N|2417271|2417329|2416930|2417203|N|N|N|N|N| +2417296|AAAAAAAAAJCOECAA|1906-03-26|74|325|26|1906|1|3|26|1|1906|26|325|Monday|1906Q1|N|N|N|2417271|2417329|2416931|2417204|N|N|N|N|N| +2417297|AAAAAAAABJCOECAA|1906-03-27|74|326|26|1906|2|3|27|1|1906|26|326|Tuesday|1906Q1|N|N|N|2417271|2417329|2416932|2417205|N|N|N|N|N| +2417298|AAAAAAAACJCOECAA|1906-03-28|74|326|26|1906|3|3|28|1|1906|26|326|Wednesday|1906Q1|N|N|N|2417271|2417329|2416933|2417206|N|N|N|N|N| +2417299|AAAAAAAADJCOECAA|1906-03-29|74|326|26|1906|4|3|29|1|1906|26|326|Thursday|1906Q1|N|N|N|2417271|2417329|2416934|2417207|N|N|N|N|N| +2417300|AAAAAAAAEJCOECAA|1906-03-30|74|326|26|1906|5|3|30|1|1906|26|326|Friday|1906Q1|N|Y|N|2417271|2417329|2416935|2417208|N|N|N|N|N| +2417301|AAAAAAAAFJCOECAA|1906-03-31|74|326|26|1906|6|3|31|1|1906|26|326|Saturday|1906Q1|N|Y|N|2417271|2417329|2416936|2417209|N|N|N|N|N| +2417302|AAAAAAAAGJCOECAA|1906-04-01|75|326|26|1906|0|4|1|1|1906|26|326|Sunday|1906Q1|N|N|N|2417302|2417391|2416937|2417212|N|N|N|N|N| +2417303|AAAAAAAAHJCOECAA|1906-04-02|75|326|26|1906|1|4|2|2|1906|26|326|Monday|1906Q2|N|N|N|2417302|2417391|2416938|2417213|N|N|N|N|N| +2417304|AAAAAAAAIJCOECAA|1906-04-03|75|327|26|1906|2|4|3|2|1906|26|327|Tuesday|1906Q2|N|N|N|2417302|2417391|2416939|2417214|N|N|N|N|N| +2417305|AAAAAAAAJJCOECAA|1906-04-04|75|327|26|1906|3|4|4|2|1906|26|327|Wednesday|1906Q2|N|N|N|2417302|2417391|2416940|2417215|N|N|N|N|N| +2417306|AAAAAAAAKJCOECAA|1906-04-05|75|327|26|1906|4|4|5|2|1906|26|327|Thursday|1906Q2|N|N|N|2417302|2417391|2416941|2417216|N|N|N|N|N| +2417307|AAAAAAAALJCOECAA|1906-04-06|75|327|26|1906|5|4|6|2|1906|26|327|Friday|1906Q2|N|Y|N|2417302|2417391|2416942|2417217|N|N|N|N|N| +2417308|AAAAAAAAMJCOECAA|1906-04-07|75|327|26|1906|6|4|7|2|1906|26|327|Saturday|1906Q2|N|Y|N|2417302|2417391|2416943|2417218|N|N|N|N|N| +2417309|AAAAAAAANJCOECAA|1906-04-08|75|327|26|1906|0|4|8|2|1906|26|327|Sunday|1906Q2|N|N|N|2417302|2417391|2416944|2417219|N|N|N|N|N| +2417310|AAAAAAAAOJCOECAA|1906-04-09|75|327|26|1906|1|4|9|2|1906|26|327|Monday|1906Q2|N|N|N|2417302|2417391|2416945|2417220|N|N|N|N|N| +2417311|AAAAAAAAPJCOECAA|1906-04-10|75|328|26|1906|2|4|10|2|1906|26|328|Tuesday|1906Q2|N|N|N|2417302|2417391|2416946|2417221|N|N|N|N|N| +2417312|AAAAAAAAAKCOECAA|1906-04-11|75|328|26|1906|3|4|11|2|1906|26|328|Wednesday|1906Q2|N|N|N|2417302|2417391|2416947|2417222|N|N|N|N|N| +2417313|AAAAAAAABKCOECAA|1906-04-12|75|328|26|1906|4|4|12|2|1906|26|328|Thursday|1906Q2|N|N|N|2417302|2417391|2416948|2417223|N|N|N|N|N| +2417314|AAAAAAAACKCOECAA|1906-04-13|75|328|26|1906|5|4|13|2|1906|26|328|Friday|1906Q2|N|Y|N|2417302|2417391|2416949|2417224|N|N|N|N|N| +2417315|AAAAAAAADKCOECAA|1906-04-14|75|328|26|1906|6|4|14|2|1906|26|328|Saturday|1906Q2|N|Y|N|2417302|2417391|2416950|2417225|N|N|N|N|N| +2417316|AAAAAAAAEKCOECAA|1906-04-15|75|328|26|1906|0|4|15|2|1906|26|328|Sunday|1906Q2|N|N|N|2417302|2417391|2416951|2417226|N|N|N|N|N| +2417317|AAAAAAAAFKCOECAA|1906-04-16|75|328|26|1906|1|4|16|2|1906|26|328|Monday|1906Q2|N|N|N|2417302|2417391|2416952|2417227|N|N|N|N|N| +2417318|AAAAAAAAGKCOECAA|1906-04-17|75|329|26|1906|2|4|17|2|1906|26|329|Tuesday|1906Q2|N|N|N|2417302|2417391|2416953|2417228|N|N|N|N|N| +2417319|AAAAAAAAHKCOECAA|1906-04-18|75|329|26|1906|3|4|18|2|1906|26|329|Wednesday|1906Q2|N|N|N|2417302|2417391|2416954|2417229|N|N|N|N|N| +2417320|AAAAAAAAIKCOECAA|1906-04-19|75|329|26|1906|4|4|19|2|1906|26|329|Thursday|1906Q2|N|N|N|2417302|2417391|2416955|2417230|N|N|N|N|N| +2417321|AAAAAAAAJKCOECAA|1906-04-20|75|329|26|1906|5|4|20|2|1906|26|329|Friday|1906Q2|N|Y|N|2417302|2417391|2416956|2417231|N|N|N|N|N| +2417322|AAAAAAAAKKCOECAA|1906-04-21|75|329|26|1906|6|4|21|2|1906|26|329|Saturday|1906Q2|N|Y|N|2417302|2417391|2416957|2417232|N|N|N|N|N| +2417323|AAAAAAAALKCOECAA|1906-04-22|75|329|26|1906|0|4|22|2|1906|26|329|Sunday|1906Q2|N|N|N|2417302|2417391|2416958|2417233|N|N|N|N|N| +2417324|AAAAAAAAMKCOECAA|1906-04-23|75|329|26|1906|1|4|23|2|1906|26|329|Monday|1906Q2|N|N|N|2417302|2417391|2416959|2417234|N|N|N|N|N| +2417325|AAAAAAAANKCOECAA|1906-04-24|75|330|26|1906|2|4|24|2|1906|26|330|Tuesday|1906Q2|N|N|N|2417302|2417391|2416960|2417235|N|N|N|N|N| +2417326|AAAAAAAAOKCOECAA|1906-04-25|75|330|26|1906|3|4|25|2|1906|26|330|Wednesday|1906Q2|N|N|N|2417302|2417391|2416961|2417236|N|N|N|N|N| +2417327|AAAAAAAAPKCOECAA|1906-04-26|75|330|26|1906|4|4|26|2|1906|26|330|Thursday|1906Q2|N|N|N|2417302|2417391|2416962|2417237|N|N|N|N|N| +2417328|AAAAAAAAALCOECAA|1906-04-27|75|330|26|1906|5|4|27|2|1906|26|330|Friday|1906Q2|N|Y|N|2417302|2417391|2416963|2417238|N|N|N|N|N| +2417329|AAAAAAAABLCOECAA|1906-04-28|75|330|26|1906|6|4|28|2|1906|26|330|Saturday|1906Q2|N|Y|N|2417302|2417391|2416964|2417239|N|N|N|N|N| +2417330|AAAAAAAACLCOECAA|1906-04-29|75|330|26|1906|0|4|29|2|1906|26|330|Sunday|1906Q2|N|N|N|2417302|2417391|2416965|2417240|N|N|N|N|N| +2417331|AAAAAAAADLCOECAA|1906-04-30|75|330|26|1906|1|4|30|2|1906|26|330|Monday|1906Q2|N|N|N|2417302|2417391|2416966|2417241|N|N|N|N|N| +2417332|AAAAAAAAELCOECAA|1906-05-01|76|331|26|1906|2|5|1|2|1906|26|331|Tuesday|1906Q2|N|N|N|2417332|2417451|2416967|2417242|N|N|N|N|N| +2417333|AAAAAAAAFLCOECAA|1906-05-02|76|331|26|1906|3|5|2|2|1906|26|331|Wednesday|1906Q2|N|N|N|2417332|2417451|2416968|2417243|N|N|N|N|N| +2417334|AAAAAAAAGLCOECAA|1906-05-03|76|331|26|1906|4|5|3|2|1906|26|331|Thursday|1906Q2|N|N|N|2417332|2417451|2416969|2417244|N|N|N|N|N| +2417335|AAAAAAAAHLCOECAA|1906-05-04|76|331|26|1906|5|5|4|2|1906|26|331|Friday|1906Q2|N|Y|N|2417332|2417451|2416970|2417245|N|N|N|N|N| +2417336|AAAAAAAAILCOECAA|1906-05-05|76|331|26|1906|6|5|5|2|1906|26|331|Saturday|1906Q2|N|Y|N|2417332|2417451|2416971|2417246|N|N|N|N|N| +2417337|AAAAAAAAJLCOECAA|1906-05-06|76|331|26|1906|0|5|6|2|1906|26|331|Sunday|1906Q2|N|N|N|2417332|2417451|2416972|2417247|N|N|N|N|N| +2417338|AAAAAAAAKLCOECAA|1906-05-07|76|331|26|1906|1|5|7|2|1906|26|331|Monday|1906Q2|N|N|N|2417332|2417451|2416973|2417248|N|N|N|N|N| +2417339|AAAAAAAALLCOECAA|1906-05-08|76|332|26|1906|2|5|8|2|1906|26|332|Tuesday|1906Q2|N|N|N|2417332|2417451|2416974|2417249|N|N|N|N|N| +2417340|AAAAAAAAMLCOECAA|1906-05-09|76|332|26|1906|3|5|9|2|1906|26|332|Wednesday|1906Q2|N|N|N|2417332|2417451|2416975|2417250|N|N|N|N|N| +2417341|AAAAAAAANLCOECAA|1906-05-10|76|332|26|1906|4|5|10|2|1906|26|332|Thursday|1906Q2|N|N|N|2417332|2417451|2416976|2417251|N|N|N|N|N| +2417342|AAAAAAAAOLCOECAA|1906-05-11|76|332|26|1906|5|5|11|2|1906|26|332|Friday|1906Q2|N|Y|N|2417332|2417451|2416977|2417252|N|N|N|N|N| +2417343|AAAAAAAAPLCOECAA|1906-05-12|76|332|26|1906|6|5|12|2|1906|26|332|Saturday|1906Q2|N|Y|N|2417332|2417451|2416978|2417253|N|N|N|N|N| +2417344|AAAAAAAAAMCOECAA|1906-05-13|76|332|26|1906|0|5|13|2|1906|26|332|Sunday|1906Q2|N|N|N|2417332|2417451|2416979|2417254|N|N|N|N|N| +2417345|AAAAAAAABMCOECAA|1906-05-14|76|332|26|1906|1|5|14|2|1906|26|332|Monday|1906Q2|N|N|N|2417332|2417451|2416980|2417255|N|N|N|N|N| +2417346|AAAAAAAACMCOECAA|1906-05-15|76|333|26|1906|2|5|15|2|1906|26|333|Tuesday|1906Q2|N|N|N|2417332|2417451|2416981|2417256|N|N|N|N|N| +2417347|AAAAAAAADMCOECAA|1906-05-16|76|333|26|1906|3|5|16|2|1906|26|333|Wednesday|1906Q2|N|N|N|2417332|2417451|2416982|2417257|N|N|N|N|N| +2417348|AAAAAAAAEMCOECAA|1906-05-17|76|333|26|1906|4|5|17|2|1906|26|333|Thursday|1906Q2|N|N|N|2417332|2417451|2416983|2417258|N|N|N|N|N| +2417349|AAAAAAAAFMCOECAA|1906-05-18|76|333|26|1906|5|5|18|2|1906|26|333|Friday|1906Q2|N|Y|N|2417332|2417451|2416984|2417259|N|N|N|N|N| +2417350|AAAAAAAAGMCOECAA|1906-05-19|76|333|26|1906|6|5|19|2|1906|26|333|Saturday|1906Q2|N|Y|N|2417332|2417451|2416985|2417260|N|N|N|N|N| +2417351|AAAAAAAAHMCOECAA|1906-05-20|76|333|26|1906|0|5|20|2|1906|26|333|Sunday|1906Q2|N|N|N|2417332|2417451|2416986|2417261|N|N|N|N|N| +2417352|AAAAAAAAIMCOECAA|1906-05-21|76|333|26|1906|1|5|21|2|1906|26|333|Monday|1906Q2|N|N|N|2417332|2417451|2416987|2417262|N|N|N|N|N| +2417353|AAAAAAAAJMCOECAA|1906-05-22|76|334|26|1906|2|5|22|2|1906|26|334|Tuesday|1906Q2|N|N|N|2417332|2417451|2416988|2417263|N|N|N|N|N| +2417354|AAAAAAAAKMCOECAA|1906-05-23|76|334|26|1906|3|5|23|2|1906|26|334|Wednesday|1906Q2|N|N|N|2417332|2417451|2416989|2417264|N|N|N|N|N| +2417355|AAAAAAAALMCOECAA|1906-05-24|76|334|26|1906|4|5|24|2|1906|26|334|Thursday|1906Q2|N|N|N|2417332|2417451|2416990|2417265|N|N|N|N|N| +2417356|AAAAAAAAMMCOECAA|1906-05-25|76|334|26|1906|5|5|25|2|1906|26|334|Friday|1906Q2|N|Y|N|2417332|2417451|2416991|2417266|N|N|N|N|N| +2417357|AAAAAAAANMCOECAA|1906-05-26|76|334|26|1906|6|5|26|2|1906|26|334|Saturday|1906Q2|N|Y|N|2417332|2417451|2416992|2417267|N|N|N|N|N| +2417358|AAAAAAAAOMCOECAA|1906-05-27|76|334|26|1906|0|5|27|2|1906|26|334|Sunday|1906Q2|N|N|N|2417332|2417451|2416993|2417268|N|N|N|N|N| +2417359|AAAAAAAAPMCOECAA|1906-05-28|76|334|26|1906|1|5|28|2|1906|26|334|Monday|1906Q2|N|N|N|2417332|2417451|2416994|2417269|N|N|N|N|N| +2417360|AAAAAAAAANCOECAA|1906-05-29|76|335|26|1906|2|5|29|2|1906|26|335|Tuesday|1906Q2|N|N|N|2417332|2417451|2416995|2417270|N|N|N|N|N| +2417361|AAAAAAAABNCOECAA|1906-05-30|76|335|26|1906|3|5|30|2|1906|26|335|Wednesday|1906Q2|N|N|N|2417332|2417451|2416996|2417271|N|N|N|N|N| +2417362|AAAAAAAACNCOECAA|1906-05-31|76|335|26|1906|4|5|31|2|1906|26|335|Thursday|1906Q2|N|N|N|2417332|2417451|2416997|2417272|N|N|N|N|N| +2417363|AAAAAAAADNCOECAA|1906-06-01|77|335|27|1906|5|6|1|2|1906|27|335|Friday|1906Q2|N|Y|N|2417363|2417513|2416998|2417273|N|N|N|N|N| +2417364|AAAAAAAAENCOECAA|1906-06-02|77|335|27|1906|6|6|2|2|1906|27|335|Saturday|1906Q2|N|Y|N|2417363|2417513|2416999|2417274|N|N|N|N|N| +2417365|AAAAAAAAFNCOECAA|1906-06-03|77|335|27|1906|0|6|3|2|1906|27|335|Sunday|1906Q2|N|N|N|2417363|2417513|2417000|2417275|N|N|N|N|N| +2417366|AAAAAAAAGNCOECAA|1906-06-04|77|335|27|1906|1|6|4|2|1906|27|335|Monday|1906Q2|N|N|N|2417363|2417513|2417001|2417276|N|N|N|N|N| +2417367|AAAAAAAAHNCOECAA|1906-06-05|77|336|27|1906|2|6|5|2|1906|27|336|Tuesday|1906Q2|N|N|N|2417363|2417513|2417002|2417277|N|N|N|N|N| +2417368|AAAAAAAAINCOECAA|1906-06-06|77|336|27|1906|3|6|6|2|1906|27|336|Wednesday|1906Q2|N|N|N|2417363|2417513|2417003|2417278|N|N|N|N|N| +2417369|AAAAAAAAJNCOECAA|1906-06-07|77|336|27|1906|4|6|7|2|1906|27|336|Thursday|1906Q2|N|N|N|2417363|2417513|2417004|2417279|N|N|N|N|N| +2417370|AAAAAAAAKNCOECAA|1906-06-08|77|336|27|1906|5|6|8|2|1906|27|336|Friday|1906Q2|N|Y|N|2417363|2417513|2417005|2417280|N|N|N|N|N| +2417371|AAAAAAAALNCOECAA|1906-06-09|77|336|27|1906|6|6|9|2|1906|27|336|Saturday|1906Q2|N|Y|N|2417363|2417513|2417006|2417281|N|N|N|N|N| +2417372|AAAAAAAAMNCOECAA|1906-06-10|77|336|27|1906|0|6|10|2|1906|27|336|Sunday|1906Q2|N|N|N|2417363|2417513|2417007|2417282|N|N|N|N|N| +2417373|AAAAAAAANNCOECAA|1906-06-11|77|336|27|1906|1|6|11|2|1906|27|336|Monday|1906Q2|N|N|N|2417363|2417513|2417008|2417283|N|N|N|N|N| +2417374|AAAAAAAAONCOECAA|1906-06-12|77|337|27|1906|2|6|12|2|1906|27|337|Tuesday|1906Q2|N|N|N|2417363|2417513|2417009|2417284|N|N|N|N|N| +2417375|AAAAAAAAPNCOECAA|1906-06-13|77|337|27|1906|3|6|13|2|1906|27|337|Wednesday|1906Q2|N|N|N|2417363|2417513|2417010|2417285|N|N|N|N|N| +2417376|AAAAAAAAAOCOECAA|1906-06-14|77|337|27|1906|4|6|14|2|1906|27|337|Thursday|1906Q2|N|N|N|2417363|2417513|2417011|2417286|N|N|N|N|N| +2417377|AAAAAAAABOCOECAA|1906-06-15|77|337|27|1906|5|6|15|2|1906|27|337|Friday|1906Q2|N|Y|N|2417363|2417513|2417012|2417287|N|N|N|N|N| +2417378|AAAAAAAACOCOECAA|1906-06-16|77|337|27|1906|6|6|16|2|1906|27|337|Saturday|1906Q2|N|Y|N|2417363|2417513|2417013|2417288|N|N|N|N|N| +2417379|AAAAAAAADOCOECAA|1906-06-17|77|337|27|1906|0|6|17|2|1906|27|337|Sunday|1906Q2|N|N|N|2417363|2417513|2417014|2417289|N|N|N|N|N| +2417380|AAAAAAAAEOCOECAA|1906-06-18|77|337|27|1906|1|6|18|2|1906|27|337|Monday|1906Q2|N|N|N|2417363|2417513|2417015|2417290|N|N|N|N|N| +2417381|AAAAAAAAFOCOECAA|1906-06-19|77|338|27|1906|2|6|19|2|1906|27|338|Tuesday|1906Q2|N|N|N|2417363|2417513|2417016|2417291|N|N|N|N|N| +2417382|AAAAAAAAGOCOECAA|1906-06-20|77|338|27|1906|3|6|20|2|1906|27|338|Wednesday|1906Q2|N|N|N|2417363|2417513|2417017|2417292|N|N|N|N|N| +2417383|AAAAAAAAHOCOECAA|1906-06-21|77|338|27|1906|4|6|21|2|1906|27|338|Thursday|1906Q2|N|N|N|2417363|2417513|2417018|2417293|N|N|N|N|N| +2417384|AAAAAAAAIOCOECAA|1906-06-22|77|338|27|1906|5|6|22|2|1906|27|338|Friday|1906Q2|N|Y|N|2417363|2417513|2417019|2417294|N|N|N|N|N| +2417385|AAAAAAAAJOCOECAA|1906-06-23|77|338|27|1906|6|6|23|2|1906|27|338|Saturday|1906Q2|N|Y|N|2417363|2417513|2417020|2417295|N|N|N|N|N| +2417386|AAAAAAAAKOCOECAA|1906-06-24|77|338|27|1906|0|6|24|2|1906|27|338|Sunday|1906Q2|N|N|N|2417363|2417513|2417021|2417296|N|N|N|N|N| +2417387|AAAAAAAALOCOECAA|1906-06-25|77|338|27|1906|1|6|25|2|1906|27|338|Monday|1906Q2|N|N|N|2417363|2417513|2417022|2417297|N|N|N|N|N| +2417388|AAAAAAAAMOCOECAA|1906-06-26|77|339|27|1906|2|6|26|2|1906|27|339|Tuesday|1906Q2|N|N|N|2417363|2417513|2417023|2417298|N|N|N|N|N| +2417389|AAAAAAAANOCOECAA|1906-06-27|77|339|27|1906|3|6|27|2|1906|27|339|Wednesday|1906Q2|N|N|N|2417363|2417513|2417024|2417299|N|N|N|N|N| +2417390|AAAAAAAAOOCOECAA|1906-06-28|77|339|27|1906|4|6|28|2|1906|27|339|Thursday|1906Q2|N|N|N|2417363|2417513|2417025|2417300|N|N|N|N|N| +2417391|AAAAAAAAPOCOECAA|1906-06-29|77|339|27|1906|5|6|29|2|1906|27|339|Friday|1906Q2|N|Y|N|2417363|2417513|2417026|2417301|N|N|N|N|N| +2417392|AAAAAAAAAPCOECAA|1906-06-30|77|339|27|1906|6|6|30|2|1906|27|339|Saturday|1906Q2|N|Y|N|2417363|2417513|2417027|2417302|N|N|N|N|N| +2417393|AAAAAAAABPCOECAA|1906-07-01|78|339|27|1906|0|7|1|2|1906|27|339|Sunday|1906Q2|N|N|N|2417393|2417573|2417028|2417302|N|N|N|N|N| +2417394|AAAAAAAACPCOECAA|1906-07-02|78|339|27|1906|1|7|2|3|1906|27|339|Monday|1906Q3|N|N|N|2417393|2417573|2417029|2417303|N|N|N|N|N| +2417395|AAAAAAAADPCOECAA|1906-07-03|78|340|27|1906|2|7|3|3|1906|27|340|Tuesday|1906Q3|N|N|N|2417393|2417573|2417030|2417304|N|N|N|N|N| +2417396|AAAAAAAAEPCOECAA|1906-07-04|78|340|27|1906|3|7|4|3|1906|27|340|Wednesday|1906Q3|N|N|N|2417393|2417573|2417031|2417305|N|N|N|N|N| +2417397|AAAAAAAAFPCOECAA|1906-07-05|78|340|27|1906|4|7|5|3|1906|27|340|Thursday|1906Q3|Y|N|N|2417393|2417573|2417032|2417306|N|N|N|N|N| +2417398|AAAAAAAAGPCOECAA|1906-07-06|78|340|27|1906|5|7|6|3|1906|27|340|Friday|1906Q3|N|Y|Y|2417393|2417573|2417033|2417307|N|N|N|N|N| +2417399|AAAAAAAAHPCOECAA|1906-07-07|78|340|27|1906|6|7|7|3|1906|27|340|Saturday|1906Q3|N|Y|N|2417393|2417573|2417034|2417308|N|N|N|N|N| +2417400|AAAAAAAAIPCOECAA|1906-07-08|78|340|27|1906|0|7|8|3|1906|27|340|Sunday|1906Q3|N|N|N|2417393|2417573|2417035|2417309|N|N|N|N|N| +2417401|AAAAAAAAJPCOECAA|1906-07-09|78|340|27|1906|1|7|9|3|1906|27|340|Monday|1906Q3|N|N|N|2417393|2417573|2417036|2417310|N|N|N|N|N| +2417402|AAAAAAAAKPCOECAA|1906-07-10|78|341|27|1906|2|7|10|3|1906|27|341|Tuesday|1906Q3|N|N|N|2417393|2417573|2417037|2417311|N|N|N|N|N| +2417403|AAAAAAAALPCOECAA|1906-07-11|78|341|27|1906|3|7|11|3|1906|27|341|Wednesday|1906Q3|N|N|N|2417393|2417573|2417038|2417312|N|N|N|N|N| +2417404|AAAAAAAAMPCOECAA|1906-07-12|78|341|27|1906|4|7|12|3|1906|27|341|Thursday|1906Q3|N|N|N|2417393|2417573|2417039|2417313|N|N|N|N|N| +2417405|AAAAAAAANPCOECAA|1906-07-13|78|341|27|1906|5|7|13|3|1906|27|341|Friday|1906Q3|N|Y|N|2417393|2417573|2417040|2417314|N|N|N|N|N| +2417406|AAAAAAAAOPCOECAA|1906-07-14|78|341|27|1906|6|7|14|3|1906|27|341|Saturday|1906Q3|N|Y|N|2417393|2417573|2417041|2417315|N|N|N|N|N| +2417407|AAAAAAAAPPCOECAA|1906-07-15|78|341|27|1906|0|7|15|3|1906|27|341|Sunday|1906Q3|N|N|N|2417393|2417573|2417042|2417316|N|N|N|N|N| +2417408|AAAAAAAAAADOECAA|1906-07-16|78|341|27|1906|1|7|16|3|1906|27|341|Monday|1906Q3|N|N|N|2417393|2417573|2417043|2417317|N|N|N|N|N| +2417409|AAAAAAAABADOECAA|1906-07-17|78|342|27|1906|2|7|17|3|1906|27|342|Tuesday|1906Q3|N|N|N|2417393|2417573|2417044|2417318|N|N|N|N|N| +2417410|AAAAAAAACADOECAA|1906-07-18|78|342|27|1906|3|7|18|3|1906|27|342|Wednesday|1906Q3|N|N|N|2417393|2417573|2417045|2417319|N|N|N|N|N| +2417411|AAAAAAAADADOECAA|1906-07-19|78|342|27|1906|4|7|19|3|1906|27|342|Thursday|1906Q3|N|N|N|2417393|2417573|2417046|2417320|N|N|N|N|N| +2417412|AAAAAAAAEADOECAA|1906-07-20|78|342|27|1906|5|7|20|3|1906|27|342|Friday|1906Q3|N|Y|N|2417393|2417573|2417047|2417321|N|N|N|N|N| +2417413|AAAAAAAAFADOECAA|1906-07-21|78|342|27|1906|6|7|21|3|1906|27|342|Saturday|1906Q3|N|Y|N|2417393|2417573|2417048|2417322|N|N|N|N|N| +2417414|AAAAAAAAGADOECAA|1906-07-22|78|342|27|1906|0|7|22|3|1906|27|342|Sunday|1906Q3|N|N|N|2417393|2417573|2417049|2417323|N|N|N|N|N| +2417415|AAAAAAAAHADOECAA|1906-07-23|78|342|27|1906|1|7|23|3|1906|27|342|Monday|1906Q3|N|N|N|2417393|2417573|2417050|2417324|N|N|N|N|N| +2417416|AAAAAAAAIADOECAA|1906-07-24|78|343|27|1906|2|7|24|3|1906|27|343|Tuesday|1906Q3|N|N|N|2417393|2417573|2417051|2417325|N|N|N|N|N| +2417417|AAAAAAAAJADOECAA|1906-07-25|78|343|27|1906|3|7|25|3|1906|27|343|Wednesday|1906Q3|N|N|N|2417393|2417573|2417052|2417326|N|N|N|N|N| +2417418|AAAAAAAAKADOECAA|1906-07-26|78|343|27|1906|4|7|26|3|1906|27|343|Thursday|1906Q3|N|N|N|2417393|2417573|2417053|2417327|N|N|N|N|N| +2417419|AAAAAAAALADOECAA|1906-07-27|78|343|27|1906|5|7|27|3|1906|27|343|Friday|1906Q3|N|Y|N|2417393|2417573|2417054|2417328|N|N|N|N|N| +2417420|AAAAAAAAMADOECAA|1906-07-28|78|343|27|1906|6|7|28|3|1906|27|343|Saturday|1906Q3|N|Y|N|2417393|2417573|2417055|2417329|N|N|N|N|N| +2417421|AAAAAAAANADOECAA|1906-07-29|78|343|27|1906|0|7|29|3|1906|27|343|Sunday|1906Q3|N|N|N|2417393|2417573|2417056|2417330|N|N|N|N|N| +2417422|AAAAAAAAOADOECAA|1906-07-30|78|343|27|1906|1|7|30|3|1906|27|343|Monday|1906Q3|N|N|N|2417393|2417573|2417057|2417331|N|N|N|N|N| +2417423|AAAAAAAAPADOECAA|1906-07-31|78|344|27|1906|2|7|31|3|1906|27|344|Tuesday|1906Q3|N|N|N|2417393|2417573|2417058|2417332|N|N|N|N|N| +2417424|AAAAAAAAABDOECAA|1906-08-01|79|344|27|1906|3|8|1|3|1906|27|344|Wednesday|1906Q3|N|N|N|2417424|2417635|2417059|2417333|N|N|N|N|N| +2417425|AAAAAAAABBDOECAA|1906-08-02|79|344|27|1906|4|8|2|3|1906|27|344|Thursday|1906Q3|N|N|N|2417424|2417635|2417060|2417334|N|N|N|N|N| +2417426|AAAAAAAACBDOECAA|1906-08-03|79|344|27|1906|5|8|3|3|1906|27|344|Friday|1906Q3|N|Y|N|2417424|2417635|2417061|2417335|N|N|N|N|N| +2417427|AAAAAAAADBDOECAA|1906-08-04|79|344|27|1906|6|8|4|3|1906|27|344|Saturday|1906Q3|N|Y|N|2417424|2417635|2417062|2417336|N|N|N|N|N| +2417428|AAAAAAAAEBDOECAA|1906-08-05|79|344|27|1906|0|8|5|3|1906|27|344|Sunday|1906Q3|N|N|N|2417424|2417635|2417063|2417337|N|N|N|N|N| +2417429|AAAAAAAAFBDOECAA|1906-08-06|79|344|27|1906|1|8|6|3|1906|27|344|Monday|1906Q3|N|N|N|2417424|2417635|2417064|2417338|N|N|N|N|N| +2417430|AAAAAAAAGBDOECAA|1906-08-07|79|345|27|1906|2|8|7|3|1906|27|345|Tuesday|1906Q3|N|N|N|2417424|2417635|2417065|2417339|N|N|N|N|N| +2417431|AAAAAAAAHBDOECAA|1906-08-08|79|345|27|1906|3|8|8|3|1906|27|345|Wednesday|1906Q3|N|N|N|2417424|2417635|2417066|2417340|N|N|N|N|N| +2417432|AAAAAAAAIBDOECAA|1906-08-09|79|345|27|1906|4|8|9|3|1906|27|345|Thursday|1906Q3|N|N|N|2417424|2417635|2417067|2417341|N|N|N|N|N| +2417433|AAAAAAAAJBDOECAA|1906-08-10|79|345|27|1906|5|8|10|3|1906|27|345|Friday|1906Q3|N|Y|N|2417424|2417635|2417068|2417342|N|N|N|N|N| +2417434|AAAAAAAAKBDOECAA|1906-08-11|79|345|27|1906|6|8|11|3|1906|27|345|Saturday|1906Q3|N|Y|N|2417424|2417635|2417069|2417343|N|N|N|N|N| +2417435|AAAAAAAALBDOECAA|1906-08-12|79|345|27|1906|0|8|12|3|1906|27|345|Sunday|1906Q3|N|N|N|2417424|2417635|2417070|2417344|N|N|N|N|N| +2417436|AAAAAAAAMBDOECAA|1906-08-13|79|345|27|1906|1|8|13|3|1906|27|345|Monday|1906Q3|N|N|N|2417424|2417635|2417071|2417345|N|N|N|N|N| +2417437|AAAAAAAANBDOECAA|1906-08-14|79|346|27|1906|2|8|14|3|1906|27|346|Tuesday|1906Q3|N|N|N|2417424|2417635|2417072|2417346|N|N|N|N|N| +2417438|AAAAAAAAOBDOECAA|1906-08-15|79|346|27|1906|3|8|15|3|1906|27|346|Wednesday|1906Q3|N|N|N|2417424|2417635|2417073|2417347|N|N|N|N|N| +2417439|AAAAAAAAPBDOECAA|1906-08-16|79|346|27|1906|4|8|16|3|1906|27|346|Thursday|1906Q3|N|N|N|2417424|2417635|2417074|2417348|N|N|N|N|N| +2417440|AAAAAAAAACDOECAA|1906-08-17|79|346|27|1906|5|8|17|3|1906|27|346|Friday|1906Q3|N|Y|N|2417424|2417635|2417075|2417349|N|N|N|N|N| +2417441|AAAAAAAABCDOECAA|1906-08-18|79|346|27|1906|6|8|18|3|1906|27|346|Saturday|1906Q3|N|Y|N|2417424|2417635|2417076|2417350|N|N|N|N|N| +2417442|AAAAAAAACCDOECAA|1906-08-19|79|346|27|1906|0|8|19|3|1906|27|346|Sunday|1906Q3|N|N|N|2417424|2417635|2417077|2417351|N|N|N|N|N| +2417443|AAAAAAAADCDOECAA|1906-08-20|79|346|27|1906|1|8|20|3|1906|27|346|Monday|1906Q3|N|N|N|2417424|2417635|2417078|2417352|N|N|N|N|N| +2417444|AAAAAAAAECDOECAA|1906-08-21|79|347|27|1906|2|8|21|3|1906|27|347|Tuesday|1906Q3|N|N|N|2417424|2417635|2417079|2417353|N|N|N|N|N| +2417445|AAAAAAAAFCDOECAA|1906-08-22|79|347|27|1906|3|8|22|3|1906|27|347|Wednesday|1906Q3|N|N|N|2417424|2417635|2417080|2417354|N|N|N|N|N| +2417446|AAAAAAAAGCDOECAA|1906-08-23|79|347|27|1906|4|8|23|3|1906|27|347|Thursday|1906Q3|N|N|N|2417424|2417635|2417081|2417355|N|N|N|N|N| +2417447|AAAAAAAAHCDOECAA|1906-08-24|79|347|27|1906|5|8|24|3|1906|27|347|Friday|1906Q3|N|Y|N|2417424|2417635|2417082|2417356|N|N|N|N|N| +2417448|AAAAAAAAICDOECAA|1906-08-25|79|347|27|1906|6|8|25|3|1906|27|347|Saturday|1906Q3|N|Y|N|2417424|2417635|2417083|2417357|N|N|N|N|N| +2417449|AAAAAAAAJCDOECAA|1906-08-26|79|347|27|1906|0|8|26|3|1906|27|347|Sunday|1906Q3|N|N|N|2417424|2417635|2417084|2417358|N|N|N|N|N| +2417450|AAAAAAAAKCDOECAA|1906-08-27|79|347|27|1906|1|8|27|3|1906|27|347|Monday|1906Q3|N|N|N|2417424|2417635|2417085|2417359|N|N|N|N|N| +2417451|AAAAAAAALCDOECAA|1906-08-28|79|348|27|1906|2|8|28|3|1906|27|348|Tuesday|1906Q3|N|N|N|2417424|2417635|2417086|2417360|N|N|N|N|N| +2417452|AAAAAAAAMCDOECAA|1906-08-29|79|348|27|1906|3|8|29|3|1906|27|348|Wednesday|1906Q3|N|N|N|2417424|2417635|2417087|2417361|N|N|N|N|N| +2417453|AAAAAAAANCDOECAA|1906-08-30|79|348|27|1906|4|8|30|3|1906|27|348|Thursday|1906Q3|N|N|N|2417424|2417635|2417088|2417362|N|N|N|N|N| +2417454|AAAAAAAAOCDOECAA|1906-08-31|79|348|27|1906|5|8|31|3|1906|27|348|Friday|1906Q3|N|Y|N|2417424|2417635|2417089|2417363|N|N|N|N|N| +2417455|AAAAAAAAPCDOECAA|1906-09-01|80|348|28|1906|6|9|1|3|1906|28|348|Saturday|1906Q3|N|Y|N|2417455|2417697|2417090|2417364|N|N|N|N|N| +2417456|AAAAAAAAADDOECAA|1906-09-02|80|348|28|1906|0|9|2|3|1906|28|348|Sunday|1906Q3|N|N|N|2417455|2417697|2417091|2417365|N|N|N|N|N| +2417457|AAAAAAAABDDOECAA|1906-09-03|80|348|28|1906|1|9|3|3|1906|28|348|Monday|1906Q3|N|N|N|2417455|2417697|2417092|2417366|N|N|N|N|N| +2417458|AAAAAAAACDDOECAA|1906-09-04|80|349|28|1906|2|9|4|3|1906|28|349|Tuesday|1906Q3|N|N|N|2417455|2417697|2417093|2417367|N|N|N|N|N| +2417459|AAAAAAAADDDOECAA|1906-09-05|80|349|28|1906|3|9|5|3|1906|28|349|Wednesday|1906Q3|N|N|N|2417455|2417697|2417094|2417368|N|N|N|N|N| +2417460|AAAAAAAAEDDOECAA|1906-09-06|80|349|28|1906|4|9|6|3|1906|28|349|Thursday|1906Q3|N|N|N|2417455|2417697|2417095|2417369|N|N|N|N|N| +2417461|AAAAAAAAFDDOECAA|1906-09-07|80|349|28|1906|5|9|7|3|1906|28|349|Friday|1906Q3|N|Y|N|2417455|2417697|2417096|2417370|N|N|N|N|N| +2417462|AAAAAAAAGDDOECAA|1906-09-08|80|349|28|1906|6|9|8|3|1906|28|349|Saturday|1906Q3|N|Y|N|2417455|2417697|2417097|2417371|N|N|N|N|N| +2417463|AAAAAAAAHDDOECAA|1906-09-09|80|349|28|1906|0|9|9|3|1906|28|349|Sunday|1906Q3|N|N|N|2417455|2417697|2417098|2417372|N|N|N|N|N| +2417464|AAAAAAAAIDDOECAA|1906-09-10|80|349|28|1906|1|9|10|3|1906|28|349|Monday|1906Q3|N|N|N|2417455|2417697|2417099|2417373|N|N|N|N|N| +2417465|AAAAAAAAJDDOECAA|1906-09-11|80|350|28|1906|2|9|11|3|1906|28|350|Tuesday|1906Q3|N|N|N|2417455|2417697|2417100|2417374|N|N|N|N|N| +2417466|AAAAAAAAKDDOECAA|1906-09-12|80|350|28|1906|3|9|12|3|1906|28|350|Wednesday|1906Q3|N|N|N|2417455|2417697|2417101|2417375|N|N|N|N|N| +2417467|AAAAAAAALDDOECAA|1906-09-13|80|350|28|1906|4|9|13|3|1906|28|350|Thursday|1906Q3|N|N|N|2417455|2417697|2417102|2417376|N|N|N|N|N| +2417468|AAAAAAAAMDDOECAA|1906-09-14|80|350|28|1906|5|9|14|3|1906|28|350|Friday|1906Q3|N|Y|N|2417455|2417697|2417103|2417377|N|N|N|N|N| +2417469|AAAAAAAANDDOECAA|1906-09-15|80|350|28|1906|6|9|15|3|1906|28|350|Saturday|1906Q3|N|Y|N|2417455|2417697|2417104|2417378|N|N|N|N|N| +2417470|AAAAAAAAODDOECAA|1906-09-16|80|350|28|1906|0|9|16|3|1906|28|350|Sunday|1906Q3|N|N|N|2417455|2417697|2417105|2417379|N|N|N|N|N| +2417471|AAAAAAAAPDDOECAA|1906-09-17|80|350|28|1906|1|9|17|3|1906|28|350|Monday|1906Q3|N|N|N|2417455|2417697|2417106|2417380|N|N|N|N|N| +2417472|AAAAAAAAAEDOECAA|1906-09-18|80|351|28|1906|2|9|18|3|1906|28|351|Tuesday|1906Q3|N|N|N|2417455|2417697|2417107|2417381|N|N|N|N|N| +2417473|AAAAAAAABEDOECAA|1906-09-19|80|351|28|1906|3|9|19|3|1906|28|351|Wednesday|1906Q3|N|N|N|2417455|2417697|2417108|2417382|N|N|N|N|N| +2417474|AAAAAAAACEDOECAA|1906-09-20|80|351|28|1906|4|9|20|3|1906|28|351|Thursday|1906Q3|N|N|N|2417455|2417697|2417109|2417383|N|N|N|N|N| +2417475|AAAAAAAADEDOECAA|1906-09-21|80|351|28|1906|5|9|21|3|1906|28|351|Friday|1906Q3|N|Y|N|2417455|2417697|2417110|2417384|N|N|N|N|N| +2417476|AAAAAAAAEEDOECAA|1906-09-22|80|351|28|1906|6|9|22|3|1906|28|351|Saturday|1906Q3|N|Y|N|2417455|2417697|2417111|2417385|N|N|N|N|N| +2417477|AAAAAAAAFEDOECAA|1906-09-23|80|351|28|1906|0|9|23|3|1906|28|351|Sunday|1906Q3|N|N|N|2417455|2417697|2417112|2417386|N|N|N|N|N| +2417478|AAAAAAAAGEDOECAA|1906-09-24|80|351|28|1906|1|9|24|3|1906|28|351|Monday|1906Q3|N|N|N|2417455|2417697|2417113|2417387|N|N|N|N|N| +2417479|AAAAAAAAHEDOECAA|1906-09-25|80|352|28|1906|2|9|25|3|1906|28|352|Tuesday|1906Q3|N|N|N|2417455|2417697|2417114|2417388|N|N|N|N|N| +2417480|AAAAAAAAIEDOECAA|1906-09-26|80|352|28|1906|3|9|26|3|1906|28|352|Wednesday|1906Q3|N|N|N|2417455|2417697|2417115|2417389|N|N|N|N|N| +2417481|AAAAAAAAJEDOECAA|1906-09-27|80|352|28|1906|4|9|27|3|1906|28|352|Thursday|1906Q3|N|N|N|2417455|2417697|2417116|2417390|N|N|N|N|N| +2417482|AAAAAAAAKEDOECAA|1906-09-28|80|352|28|1906|5|9|28|3|1906|28|352|Friday|1906Q3|N|Y|N|2417455|2417697|2417117|2417391|N|N|N|N|N| +2417483|AAAAAAAALEDOECAA|1906-09-29|80|352|28|1906|6|9|29|3|1906|28|352|Saturday|1906Q3|N|Y|N|2417455|2417697|2417118|2417392|N|N|N|N|N| +2417484|AAAAAAAAMEDOECAA|1906-09-30|80|352|28|1906|0|9|30|3|1906|28|352|Sunday|1906Q3|N|N|N|2417455|2417697|2417119|2417393|N|N|N|N|N| +2417485|AAAAAAAANEDOECAA|1906-10-01|81|352|28|1906|1|10|1|3|1906|28|352|Monday|1906Q3|N|N|N|2417485|2417757|2417120|2417393|N|N|N|N|N| +2417486|AAAAAAAAOEDOECAA|1906-10-02|81|353|28|1906|2|10|2|4|1906|28|353|Tuesday|1906Q4|N|N|N|2417485|2417757|2417121|2417394|N|N|N|N|N| +2417487|AAAAAAAAPEDOECAA|1906-10-03|81|353|28|1906|3|10|3|4|1906|28|353|Wednesday|1906Q4|N|N|N|2417485|2417757|2417122|2417395|N|N|N|N|N| +2417488|AAAAAAAAAFDOECAA|1906-10-04|81|353|28|1906|4|10|4|4|1906|28|353|Thursday|1906Q4|N|N|N|2417485|2417757|2417123|2417396|N|N|N|N|N| +2417489|AAAAAAAABFDOECAA|1906-10-05|81|353|28|1906|5|10|5|4|1906|28|353|Friday|1906Q4|N|Y|N|2417485|2417757|2417124|2417397|N|N|N|N|N| +2417490|AAAAAAAACFDOECAA|1906-10-06|81|353|28|1906|6|10|6|4|1906|28|353|Saturday|1906Q4|N|Y|N|2417485|2417757|2417125|2417398|N|N|N|N|N| +2417491|AAAAAAAADFDOECAA|1906-10-07|81|353|28|1906|0|10|7|4|1906|28|353|Sunday|1906Q4|N|N|N|2417485|2417757|2417126|2417399|N|N|N|N|N| +2417492|AAAAAAAAEFDOECAA|1906-10-08|81|353|28|1906|1|10|8|4|1906|28|353|Monday|1906Q4|N|N|N|2417485|2417757|2417127|2417400|N|N|N|N|N| +2417493|AAAAAAAAFFDOECAA|1906-10-09|81|354|28|1906|2|10|9|4|1906|28|354|Tuesday|1906Q4|N|N|N|2417485|2417757|2417128|2417401|N|N|N|N|N| +2417494|AAAAAAAAGFDOECAA|1906-10-10|81|354|28|1906|3|10|10|4|1906|28|354|Wednesday|1906Q4|N|N|N|2417485|2417757|2417129|2417402|N|N|N|N|N| +2417495|AAAAAAAAHFDOECAA|1906-10-11|81|354|28|1906|4|10|11|4|1906|28|354|Thursday|1906Q4|N|N|N|2417485|2417757|2417130|2417403|N|N|N|N|N| +2417496|AAAAAAAAIFDOECAA|1906-10-12|81|354|28|1906|5|10|12|4|1906|28|354|Friday|1906Q4|N|Y|N|2417485|2417757|2417131|2417404|N|N|N|N|N| +2417497|AAAAAAAAJFDOECAA|1906-10-13|81|354|28|1906|6|10|13|4|1906|28|354|Saturday|1906Q4|N|Y|N|2417485|2417757|2417132|2417405|N|N|N|N|N| +2417498|AAAAAAAAKFDOECAA|1906-10-14|81|354|28|1906|0|10|14|4|1906|28|354|Sunday|1906Q4|N|N|N|2417485|2417757|2417133|2417406|N|N|N|N|N| +2417499|AAAAAAAALFDOECAA|1906-10-15|81|354|28|1906|1|10|15|4|1906|28|354|Monday|1906Q4|N|N|N|2417485|2417757|2417134|2417407|N|N|N|N|N| +2417500|AAAAAAAAMFDOECAA|1906-10-16|81|355|28|1906|2|10|16|4|1906|28|355|Tuesday|1906Q4|N|N|N|2417485|2417757|2417135|2417408|N|N|N|N|N| +2417501|AAAAAAAANFDOECAA|1906-10-17|81|355|28|1906|3|10|17|4|1906|28|355|Wednesday|1906Q4|N|N|N|2417485|2417757|2417136|2417409|N|N|N|N|N| +2417502|AAAAAAAAOFDOECAA|1906-10-18|81|355|28|1906|4|10|18|4|1906|28|355|Thursday|1906Q4|N|N|N|2417485|2417757|2417137|2417410|N|N|N|N|N| +2417503|AAAAAAAAPFDOECAA|1906-10-19|81|355|28|1906|5|10|19|4|1906|28|355|Friday|1906Q4|N|Y|N|2417485|2417757|2417138|2417411|N|N|N|N|N| +2417504|AAAAAAAAAGDOECAA|1906-10-20|81|355|28|1906|6|10|20|4|1906|28|355|Saturday|1906Q4|N|Y|N|2417485|2417757|2417139|2417412|N|N|N|N|N| +2417505|AAAAAAAABGDOECAA|1906-10-21|81|355|28|1906|0|10|21|4|1906|28|355|Sunday|1906Q4|N|N|N|2417485|2417757|2417140|2417413|N|N|N|N|N| +2417506|AAAAAAAACGDOECAA|1906-10-22|81|355|28|1906|1|10|22|4|1906|28|355|Monday|1906Q4|N|N|N|2417485|2417757|2417141|2417414|N|N|N|N|N| +2417507|AAAAAAAADGDOECAA|1906-10-23|81|356|28|1906|2|10|23|4|1906|28|356|Tuesday|1906Q4|N|N|N|2417485|2417757|2417142|2417415|N|N|N|N|N| +2417508|AAAAAAAAEGDOECAA|1906-10-24|81|356|28|1906|3|10|24|4|1906|28|356|Wednesday|1906Q4|N|N|N|2417485|2417757|2417143|2417416|N|N|N|N|N| +2417509|AAAAAAAAFGDOECAA|1906-10-25|81|356|28|1906|4|10|25|4|1906|28|356|Thursday|1906Q4|N|N|N|2417485|2417757|2417144|2417417|N|N|N|N|N| +2417510|AAAAAAAAGGDOECAA|1906-10-26|81|356|28|1906|5|10|26|4|1906|28|356|Friday|1906Q4|N|Y|N|2417485|2417757|2417145|2417418|N|N|N|N|N| +2417511|AAAAAAAAHGDOECAA|1906-10-27|81|356|28|1906|6|10|27|4|1906|28|356|Saturday|1906Q4|N|Y|N|2417485|2417757|2417146|2417419|N|N|N|N|N| +2417512|AAAAAAAAIGDOECAA|1906-10-28|81|356|28|1906|0|10|28|4|1906|28|356|Sunday|1906Q4|N|N|N|2417485|2417757|2417147|2417420|N|N|N|N|N| +2417513|AAAAAAAAJGDOECAA|1906-10-29|81|356|28|1906|1|10|29|4|1906|28|356|Monday|1906Q4|N|N|N|2417485|2417757|2417148|2417421|N|N|N|N|N| +2417514|AAAAAAAAKGDOECAA|1906-10-30|81|357|28|1906|2|10|30|4|1906|28|357|Tuesday|1906Q4|N|N|N|2417485|2417757|2417149|2417422|N|N|N|N|N| +2417515|AAAAAAAALGDOECAA|1906-10-31|81|357|28|1906|3|10|31|4|1906|28|357|Wednesday|1906Q4|N|N|N|2417485|2417757|2417150|2417423|N|N|N|N|N| +2417516|AAAAAAAAMGDOECAA|1906-11-01|82|357|28|1906|4|11|1|4|1906|28|357|Thursday|1906Q4|N|N|N|2417516|2417819|2417151|2417424|N|N|N|N|N| +2417517|AAAAAAAANGDOECAA|1906-11-02|82|357|28|1906|5|11|2|4|1906|28|357|Friday|1906Q4|N|Y|N|2417516|2417819|2417152|2417425|N|N|N|N|N| +2417518|AAAAAAAAOGDOECAA|1906-11-03|82|357|28|1906|6|11|3|4|1906|28|357|Saturday|1906Q4|N|Y|N|2417516|2417819|2417153|2417426|N|N|N|N|N| +2417519|AAAAAAAAPGDOECAA|1906-11-04|82|357|28|1906|0|11|4|4|1906|28|357|Sunday|1906Q4|N|N|N|2417516|2417819|2417154|2417427|N|N|N|N|N| +2417520|AAAAAAAAAHDOECAA|1906-11-05|82|357|28|1906|1|11|5|4|1906|28|357|Monday|1906Q4|N|N|N|2417516|2417819|2417155|2417428|N|N|N|N|N| +2417521|AAAAAAAABHDOECAA|1906-11-06|82|358|28|1906|2|11|6|4|1906|28|358|Tuesday|1906Q4|N|N|N|2417516|2417819|2417156|2417429|N|N|N|N|N| +2417522|AAAAAAAACHDOECAA|1906-11-07|82|358|28|1906|3|11|7|4|1906|28|358|Wednesday|1906Q4|N|N|N|2417516|2417819|2417157|2417430|N|N|N|N|N| +2417523|AAAAAAAADHDOECAA|1906-11-08|82|358|28|1906|4|11|8|4|1906|28|358|Thursday|1906Q4|N|N|N|2417516|2417819|2417158|2417431|N|N|N|N|N| +2417524|AAAAAAAAEHDOECAA|1906-11-09|82|358|28|1906|5|11|9|4|1906|28|358|Friday|1906Q4|N|Y|N|2417516|2417819|2417159|2417432|N|N|N|N|N| +2417525|AAAAAAAAFHDOECAA|1906-11-10|82|358|28|1906|6|11|10|4|1906|28|358|Saturday|1906Q4|N|Y|N|2417516|2417819|2417160|2417433|N|N|N|N|N| +2417526|AAAAAAAAGHDOECAA|1906-11-11|82|358|28|1906|0|11|11|4|1906|28|358|Sunday|1906Q4|N|N|N|2417516|2417819|2417161|2417434|N|N|N|N|N| +2417527|AAAAAAAAHHDOECAA|1906-11-12|82|358|28|1906|1|11|12|4|1906|28|358|Monday|1906Q4|N|N|N|2417516|2417819|2417162|2417435|N|N|N|N|N| +2417528|AAAAAAAAIHDOECAA|1906-11-13|82|359|28|1906|2|11|13|4|1906|28|359|Tuesday|1906Q4|N|N|N|2417516|2417819|2417163|2417436|N|N|N|N|N| +2417529|AAAAAAAAJHDOECAA|1906-11-14|82|359|28|1906|3|11|14|4|1906|28|359|Wednesday|1906Q4|N|N|N|2417516|2417819|2417164|2417437|N|N|N|N|N| +2417530|AAAAAAAAKHDOECAA|1906-11-15|82|359|28|1906|4|11|15|4|1906|28|359|Thursday|1906Q4|N|N|N|2417516|2417819|2417165|2417438|N|N|N|N|N| +2417531|AAAAAAAALHDOECAA|1906-11-16|82|359|28|1906|5|11|16|4|1906|28|359|Friday|1906Q4|N|Y|N|2417516|2417819|2417166|2417439|N|N|N|N|N| +2417532|AAAAAAAAMHDOECAA|1906-11-17|82|359|28|1906|6|11|17|4|1906|28|359|Saturday|1906Q4|N|Y|N|2417516|2417819|2417167|2417440|N|N|N|N|N| +2417533|AAAAAAAANHDOECAA|1906-11-18|82|359|28|1906|0|11|18|4|1906|28|359|Sunday|1906Q4|N|N|N|2417516|2417819|2417168|2417441|N|N|N|N|N| +2417534|AAAAAAAAOHDOECAA|1906-11-19|82|359|28|1906|1|11|19|4|1906|28|359|Monday|1906Q4|N|N|N|2417516|2417819|2417169|2417442|N|N|N|N|N| +2417535|AAAAAAAAPHDOECAA|1906-11-20|82|360|28|1906|2|11|20|4|1906|28|360|Tuesday|1906Q4|N|N|N|2417516|2417819|2417170|2417443|N|N|N|N|N| +2417536|AAAAAAAAAIDOECAA|1906-11-21|82|360|28|1906|3|11|21|4|1906|28|360|Wednesday|1906Q4|N|N|N|2417516|2417819|2417171|2417444|N|N|N|N|N| +2417537|AAAAAAAABIDOECAA|1906-11-22|82|360|28|1906|4|11|22|4|1906|28|360|Thursday|1906Q4|N|N|N|2417516|2417819|2417172|2417445|N|N|N|N|N| +2417538|AAAAAAAACIDOECAA|1906-11-23|82|360|28|1906|5|11|23|4|1906|28|360|Friday|1906Q4|N|Y|N|2417516|2417819|2417173|2417446|N|N|N|N|N| +2417539|AAAAAAAADIDOECAA|1906-11-24|82|360|28|1906|6|11|24|4|1906|28|360|Saturday|1906Q4|N|Y|N|2417516|2417819|2417174|2417447|N|N|N|N|N| +2417540|AAAAAAAAEIDOECAA|1906-11-25|82|360|28|1906|0|11|25|4|1906|28|360|Sunday|1906Q4|N|N|N|2417516|2417819|2417175|2417448|N|N|N|N|N| +2417541|AAAAAAAAFIDOECAA|1906-11-26|82|360|28|1906|1|11|26|4|1906|28|360|Monday|1906Q4|N|N|N|2417516|2417819|2417176|2417449|N|N|N|N|N| +2417542|AAAAAAAAGIDOECAA|1906-11-27|82|361|28|1906|2|11|27|4|1906|28|361|Tuesday|1906Q4|N|N|N|2417516|2417819|2417177|2417450|N|N|N|N|N| +2417543|AAAAAAAAHIDOECAA|1906-11-28|82|361|28|1906|3|11|28|4|1906|28|361|Wednesday|1906Q4|N|N|N|2417516|2417819|2417178|2417451|N|N|N|N|N| +2417544|AAAAAAAAIIDOECAA|1906-11-29|82|361|28|1906|4|11|29|4|1906|28|361|Thursday|1906Q4|N|N|N|2417516|2417819|2417179|2417452|N|N|N|N|N| +2417545|AAAAAAAAJIDOECAA|1906-11-30|82|361|28|1906|5|11|30|4|1906|28|361|Friday|1906Q4|N|Y|N|2417516|2417819|2417180|2417453|N|N|N|N|N| +2417546|AAAAAAAAKIDOECAA|1906-12-01|83|361|29|1906|6|12|1|4|1906|29|361|Saturday|1906Q4|N|Y|N|2417546|2417879|2417181|2417454|N|N|N|N|N| +2417547|AAAAAAAALIDOECAA|1906-12-02|83|361|29|1906|0|12|2|4|1906|29|361|Sunday|1906Q4|N|N|N|2417546|2417879|2417182|2417455|N|N|N|N|N| +2417548|AAAAAAAAMIDOECAA|1906-12-03|83|361|29|1906|1|12|3|4|1906|29|361|Monday|1906Q4|N|N|N|2417546|2417879|2417183|2417456|N|N|N|N|N| +2417549|AAAAAAAANIDOECAA|1906-12-04|83|362|29|1906|2|12|4|4|1906|29|362|Tuesday|1906Q4|N|N|N|2417546|2417879|2417184|2417457|N|N|N|N|N| +2417550|AAAAAAAAOIDOECAA|1906-12-05|83|362|29|1906|3|12|5|4|1906|29|362|Wednesday|1906Q4|N|N|N|2417546|2417879|2417185|2417458|N|N|N|N|N| +2417551|AAAAAAAAPIDOECAA|1906-12-06|83|362|29|1906|4|12|6|4|1906|29|362|Thursday|1906Q4|N|N|N|2417546|2417879|2417186|2417459|N|N|N|N|N| +2417552|AAAAAAAAAJDOECAA|1906-12-07|83|362|29|1906|5|12|7|4|1906|29|362|Friday|1906Q4|N|Y|N|2417546|2417879|2417187|2417460|N|N|N|N|N| +2417553|AAAAAAAABJDOECAA|1906-12-08|83|362|29|1906|6|12|8|4|1906|29|362|Saturday|1906Q4|N|Y|N|2417546|2417879|2417188|2417461|N|N|N|N|N| +2417554|AAAAAAAACJDOECAA|1906-12-09|83|362|29|1906|0|12|9|4|1906|29|362|Sunday|1906Q4|N|N|N|2417546|2417879|2417189|2417462|N|N|N|N|N| +2417555|AAAAAAAADJDOECAA|1906-12-10|83|362|29|1906|1|12|10|4|1906|29|362|Monday|1906Q4|N|N|N|2417546|2417879|2417190|2417463|N|N|N|N|N| +2417556|AAAAAAAAEJDOECAA|1906-12-11|83|363|29|1906|2|12|11|4|1906|29|363|Tuesday|1906Q4|N|N|N|2417546|2417879|2417191|2417464|N|N|N|N|N| +2417557|AAAAAAAAFJDOECAA|1906-12-12|83|363|29|1906|3|12|12|4|1906|29|363|Wednesday|1906Q4|N|N|N|2417546|2417879|2417192|2417465|N|N|N|N|N| +2417558|AAAAAAAAGJDOECAA|1906-12-13|83|363|29|1906|4|12|13|4|1906|29|363|Thursday|1906Q4|N|N|N|2417546|2417879|2417193|2417466|N|N|N|N|N| +2417559|AAAAAAAAHJDOECAA|1906-12-14|83|363|29|1906|5|12|14|4|1906|29|363|Friday|1906Q4|N|Y|N|2417546|2417879|2417194|2417467|N|N|N|N|N| +2417560|AAAAAAAAIJDOECAA|1906-12-15|83|363|29|1906|6|12|15|4|1906|29|363|Saturday|1906Q4|N|Y|N|2417546|2417879|2417195|2417468|N|N|N|N|N| +2417561|AAAAAAAAJJDOECAA|1906-12-16|83|363|29|1906|0|12|16|4|1906|29|363|Sunday|1906Q4|N|N|N|2417546|2417879|2417196|2417469|N|N|N|N|N| +2417562|AAAAAAAAKJDOECAA|1906-12-17|83|363|29|1906|1|12|17|4|1906|29|363|Monday|1906Q4|N|N|N|2417546|2417879|2417197|2417470|N|N|N|N|N| +2417563|AAAAAAAALJDOECAA|1906-12-18|83|364|29|1906|2|12|18|4|1906|29|364|Tuesday|1906Q4|N|N|N|2417546|2417879|2417198|2417471|N|N|N|N|N| +2417564|AAAAAAAAMJDOECAA|1906-12-19|83|364|29|1906|3|12|19|4|1906|29|364|Wednesday|1906Q4|N|N|N|2417546|2417879|2417199|2417472|N|N|N|N|N| +2417565|AAAAAAAANJDOECAA|1906-12-20|83|364|29|1906|4|12|20|4|1906|29|364|Thursday|1906Q4|N|N|N|2417546|2417879|2417200|2417473|N|N|N|N|N| +2417566|AAAAAAAAOJDOECAA|1906-12-21|83|364|29|1906|5|12|21|4|1906|29|364|Friday|1906Q4|N|Y|N|2417546|2417879|2417201|2417474|N|N|N|N|N| +2417567|AAAAAAAAPJDOECAA|1906-12-22|83|364|29|1906|6|12|22|4|1906|29|364|Saturday|1906Q4|N|Y|N|2417546|2417879|2417202|2417475|N|N|N|N|N| +2417568|AAAAAAAAAKDOECAA|1906-12-23|83|364|29|1906|0|12|23|4|1906|29|364|Sunday|1906Q4|N|N|N|2417546|2417879|2417203|2417476|N|N|N|N|N| +2417569|AAAAAAAABKDOECAA|1906-12-24|83|364|29|1906|1|12|24|4|1906|29|364|Monday|1906Q4|N|N|N|2417546|2417879|2417204|2417477|N|N|N|N|N| +2417570|AAAAAAAACKDOECAA|1906-12-25|83|365|29|1906|2|12|25|4|1906|29|365|Tuesday|1906Q4|N|N|N|2417546|2417879|2417205|2417478|N|N|N|N|N| +2417571|AAAAAAAADKDOECAA|1906-12-26|83|365|29|1906|3|12|26|4|1906|29|365|Wednesday|1906Q4|Y|N|N|2417546|2417879|2417206|2417479|N|N|N|N|N| +2417572|AAAAAAAAEKDOECAA|1906-12-27|83|365|29|1906|4|12|27|4|1906|29|365|Thursday|1906Q4|N|N|Y|2417546|2417879|2417207|2417480|N|N|N|N|N| +2417573|AAAAAAAAFKDOECAA|1906-12-28|83|365|29|1906|5|12|28|4|1906|29|365|Friday|1906Q4|N|Y|N|2417546|2417879|2417208|2417481|N|N|N|N|N| +2417574|AAAAAAAAGKDOECAA|1906-12-29|83|365|29|1906|6|12|29|4|1906|29|365|Saturday|1906Q4|N|Y|N|2417546|2417879|2417209|2417482|N|N|N|N|N| +2417575|AAAAAAAAHKDOECAA|1906-12-30|83|365|29|1906|0|12|30|4|1906|29|365|Sunday|1906Q4|N|N|N|2417546|2417879|2417210|2417483|N|N|N|N|N| +2417576|AAAAAAAAIKDOECAA|1906-12-31|83|365|29|1906|1|12|31|4|1906|29|365|Monday|1906Q4|N|N|N|2417546|2417879|2417211|2417484|N|N|N|N|N| +2417577|AAAAAAAAJKDOECAA|1907-01-01|84|366|29|1907|2|1|1|1|1907|29|366|Tuesday|1907Q1|Y|N|N|2417577|2417576|2417212|2417485|N|N|N|N|N| +2417578|AAAAAAAAKKDOECAA|1907-01-02|84|366|29|1907|3|1|2|1|1907|29|366|Wednesday|1907Q1|N|N|Y|2417577|2417576|2417213|2417486|N|N|N|N|N| +2417579|AAAAAAAALKDOECAA|1907-01-03|84|366|29|1907|4|1|3|1|1907|29|366|Thursday|1907Q1|N|N|N|2417577|2417576|2417214|2417487|N|N|N|N|N| +2417580|AAAAAAAAMKDOECAA|1907-01-04|84|366|29|1907|5|1|4|1|1907|29|366|Friday|1907Q1|N|Y|N|2417577|2417576|2417215|2417488|N|N|N|N|N| +2417581|AAAAAAAANKDOECAA|1907-01-05|84|366|29|1907|6|1|5|1|1907|29|366|Saturday|1907Q1|N|Y|N|2417577|2417576|2417216|2417489|N|N|N|N|N| +2417582|AAAAAAAAOKDOECAA|1907-01-06|84|366|29|1907|0|1|6|1|1907|29|366|Sunday|1907Q1|N|N|N|2417577|2417576|2417217|2417490|N|N|N|N|N| +2417583|AAAAAAAAPKDOECAA|1907-01-07|84|366|29|1907|1|1|7|1|1907|29|366|Monday|1907Q1|N|N|N|2417577|2417576|2417218|2417491|N|N|N|N|N| +2417584|AAAAAAAAALDOECAA|1907-01-08|84|367|29|1907|2|1|8|1|1907|29|367|Tuesday|1907Q1|N|N|N|2417577|2417576|2417219|2417492|N|N|N|N|N| +2417585|AAAAAAAABLDOECAA|1907-01-09|84|367|29|1907|3|1|9|1|1907|29|367|Wednesday|1907Q1|N|N|N|2417577|2417576|2417220|2417493|N|N|N|N|N| +2417586|AAAAAAAACLDOECAA|1907-01-10|84|367|29|1907|4|1|10|1|1907|29|367|Thursday|1907Q1|N|N|N|2417577|2417576|2417221|2417494|N|N|N|N|N| +2417587|AAAAAAAADLDOECAA|1907-01-11|84|367|29|1907|5|1|11|1|1907|29|367|Friday|1907Q1|N|Y|N|2417577|2417576|2417222|2417495|N|N|N|N|N| +2417588|AAAAAAAAELDOECAA|1907-01-12|84|367|29|1907|6|1|12|1|1907|29|367|Saturday|1907Q1|N|Y|N|2417577|2417576|2417223|2417496|N|N|N|N|N| +2417589|AAAAAAAAFLDOECAA|1907-01-13|84|367|29|1907|0|1|13|1|1907|29|367|Sunday|1907Q1|N|N|N|2417577|2417576|2417224|2417497|N|N|N|N|N| +2417590|AAAAAAAAGLDOECAA|1907-01-14|84|367|29|1907|1|1|14|1|1907|29|367|Monday|1907Q1|N|N|N|2417577|2417576|2417225|2417498|N|N|N|N|N| +2417591|AAAAAAAAHLDOECAA|1907-01-15|84|368|29|1907|2|1|15|1|1907|29|368|Tuesday|1907Q1|N|N|N|2417577|2417576|2417226|2417499|N|N|N|N|N| +2417592|AAAAAAAAILDOECAA|1907-01-16|84|368|29|1907|3|1|16|1|1907|29|368|Wednesday|1907Q1|N|N|N|2417577|2417576|2417227|2417500|N|N|N|N|N| +2417593|AAAAAAAAJLDOECAA|1907-01-17|84|368|29|1907|4|1|17|1|1907|29|368|Thursday|1907Q1|N|N|N|2417577|2417576|2417228|2417501|N|N|N|N|N| +2417594|AAAAAAAAKLDOECAA|1907-01-18|84|368|29|1907|5|1|18|1|1907|29|368|Friday|1907Q1|N|Y|N|2417577|2417576|2417229|2417502|N|N|N|N|N| +2417595|AAAAAAAALLDOECAA|1907-01-19|84|368|29|1907|6|1|19|1|1907|29|368|Saturday|1907Q1|N|Y|N|2417577|2417576|2417230|2417503|N|N|N|N|N| +2417596|AAAAAAAAMLDOECAA|1907-01-20|84|368|29|1907|0|1|20|1|1907|29|368|Sunday|1907Q1|N|N|N|2417577|2417576|2417231|2417504|N|N|N|N|N| +2417597|AAAAAAAANLDOECAA|1907-01-21|84|368|29|1907|1|1|21|1|1907|29|368|Monday|1907Q1|N|N|N|2417577|2417576|2417232|2417505|N|N|N|N|N| +2417598|AAAAAAAAOLDOECAA|1907-01-22|84|369|29|1907|2|1|22|1|1907|29|369|Tuesday|1907Q1|N|N|N|2417577|2417576|2417233|2417506|N|N|N|N|N| +2417599|AAAAAAAAPLDOECAA|1907-01-23|84|369|29|1907|3|1|23|1|1907|29|369|Wednesday|1907Q1|N|N|N|2417577|2417576|2417234|2417507|N|N|N|N|N| +2417600|AAAAAAAAAMDOECAA|1907-01-24|84|369|29|1907|4|1|24|1|1907|29|369|Thursday|1907Q1|N|N|N|2417577|2417576|2417235|2417508|N|N|N|N|N| +2417601|AAAAAAAABMDOECAA|1907-01-25|84|369|29|1907|5|1|25|1|1907|29|369|Friday|1907Q1|N|Y|N|2417577|2417576|2417236|2417509|N|N|N|N|N| +2417602|AAAAAAAACMDOECAA|1907-01-26|84|369|29|1907|6|1|26|1|1907|29|369|Saturday|1907Q1|N|Y|N|2417577|2417576|2417237|2417510|N|N|N|N|N| +2417603|AAAAAAAADMDOECAA|1907-01-27|84|369|29|1907|0|1|27|1|1907|29|369|Sunday|1907Q1|N|N|N|2417577|2417576|2417238|2417511|N|N|N|N|N| +2417604|AAAAAAAAEMDOECAA|1907-01-28|84|369|29|1907|1|1|28|1|1907|29|369|Monday|1907Q1|N|N|N|2417577|2417576|2417239|2417512|N|N|N|N|N| +2417605|AAAAAAAAFMDOECAA|1907-01-29|84|370|29|1907|2|1|29|1|1907|29|370|Tuesday|1907Q1|N|N|N|2417577|2417576|2417240|2417513|N|N|N|N|N| +2417606|AAAAAAAAGMDOECAA|1907-01-30|84|370|29|1907|3|1|30|1|1907|29|370|Wednesday|1907Q1|N|N|N|2417577|2417576|2417241|2417514|N|N|N|N|N| +2417607|AAAAAAAAHMDOECAA|1907-01-31|84|370|29|1907|4|1|31|1|1907|29|370|Thursday|1907Q1|N|N|N|2417577|2417576|2417242|2417515|N|N|N|N|N| +2417608|AAAAAAAAIMDOECAA|1907-02-01|85|370|29|1907|5|2|1|1|1907|29|370|Friday|1907Q1|N|Y|N|2417608|2417638|2417243|2417516|N|N|N|N|N| +2417609|AAAAAAAAJMDOECAA|1907-02-02|85|370|29|1907|6|2|2|1|1907|29|370|Saturday|1907Q1|N|Y|N|2417608|2417638|2417244|2417517|N|N|N|N|N| +2417610|AAAAAAAAKMDOECAA|1907-02-03|85|370|29|1907|0|2|3|1|1907|29|370|Sunday|1907Q1|N|N|N|2417608|2417638|2417245|2417518|N|N|N|N|N| +2417611|AAAAAAAALMDOECAA|1907-02-04|85|370|29|1907|1|2|4|1|1907|29|370|Monday|1907Q1|N|N|N|2417608|2417638|2417246|2417519|N|N|N|N|N| +2417612|AAAAAAAAMMDOECAA|1907-02-05|85|371|29|1907|2|2|5|1|1907|29|371|Tuesday|1907Q1|N|N|N|2417608|2417638|2417247|2417520|N|N|N|N|N| +2417613|AAAAAAAANMDOECAA|1907-02-06|85|371|29|1907|3|2|6|1|1907|29|371|Wednesday|1907Q1|N|N|N|2417608|2417638|2417248|2417521|N|N|N|N|N| +2417614|AAAAAAAAOMDOECAA|1907-02-07|85|371|29|1907|4|2|7|1|1907|29|371|Thursday|1907Q1|N|N|N|2417608|2417638|2417249|2417522|N|N|N|N|N| +2417615|AAAAAAAAPMDOECAA|1907-02-08|85|371|29|1907|5|2|8|1|1907|29|371|Friday|1907Q1|N|Y|N|2417608|2417638|2417250|2417523|N|N|N|N|N| +2417616|AAAAAAAAANDOECAA|1907-02-09|85|371|29|1907|6|2|9|1|1907|29|371|Saturday|1907Q1|N|Y|N|2417608|2417638|2417251|2417524|N|N|N|N|N| +2417617|AAAAAAAABNDOECAA|1907-02-10|85|371|29|1907|0|2|10|1|1907|29|371|Sunday|1907Q1|N|N|N|2417608|2417638|2417252|2417525|N|N|N|N|N| +2417618|AAAAAAAACNDOECAA|1907-02-11|85|371|29|1907|1|2|11|1|1907|29|371|Monday|1907Q1|N|N|N|2417608|2417638|2417253|2417526|N|N|N|N|N| +2417619|AAAAAAAADNDOECAA|1907-02-12|85|372|29|1907|2|2|12|1|1907|29|372|Tuesday|1907Q1|N|N|N|2417608|2417638|2417254|2417527|N|N|N|N|N| +2417620|AAAAAAAAENDOECAA|1907-02-13|85|372|29|1907|3|2|13|1|1907|29|372|Wednesday|1907Q1|N|N|N|2417608|2417638|2417255|2417528|N|N|N|N|N| +2417621|AAAAAAAAFNDOECAA|1907-02-14|85|372|29|1907|4|2|14|1|1907|29|372|Thursday|1907Q1|N|N|N|2417608|2417638|2417256|2417529|N|N|N|N|N| +2417622|AAAAAAAAGNDOECAA|1907-02-15|85|372|29|1907|5|2|15|1|1907|29|372|Friday|1907Q1|N|Y|N|2417608|2417638|2417257|2417530|N|N|N|N|N| +2417623|AAAAAAAAHNDOECAA|1907-02-16|85|372|29|1907|6|2|16|1|1907|29|372|Saturday|1907Q1|N|Y|N|2417608|2417638|2417258|2417531|N|N|N|N|N| +2417624|AAAAAAAAINDOECAA|1907-02-17|85|372|29|1907|0|2|17|1|1907|29|372|Sunday|1907Q1|N|N|N|2417608|2417638|2417259|2417532|N|N|N|N|N| +2417625|AAAAAAAAJNDOECAA|1907-02-18|85|372|29|1907|1|2|18|1|1907|29|372|Monday|1907Q1|N|N|N|2417608|2417638|2417260|2417533|N|N|N|N|N| +2417626|AAAAAAAAKNDOECAA|1907-02-19|85|373|29|1907|2|2|19|1|1907|29|373|Tuesday|1907Q1|N|N|N|2417608|2417638|2417261|2417534|N|N|N|N|N| +2417627|AAAAAAAALNDOECAA|1907-02-20|85|373|29|1907|3|2|20|1|1907|29|373|Wednesday|1907Q1|N|N|N|2417608|2417638|2417262|2417535|N|N|N|N|N| +2417628|AAAAAAAAMNDOECAA|1907-02-21|85|373|29|1907|4|2|21|1|1907|29|373|Thursday|1907Q1|N|N|N|2417608|2417638|2417263|2417536|N|N|N|N|N| +2417629|AAAAAAAANNDOECAA|1907-02-22|85|373|29|1907|5|2|22|1|1907|29|373|Friday|1907Q1|N|Y|N|2417608|2417638|2417264|2417537|N|N|N|N|N| +2417630|AAAAAAAAONDOECAA|1907-02-23|85|373|29|1907|6|2|23|1|1907|29|373|Saturday|1907Q1|N|Y|N|2417608|2417638|2417265|2417538|N|N|N|N|N| +2417631|AAAAAAAAPNDOECAA|1907-02-24|85|373|29|1907|0|2|24|1|1907|29|373|Sunday|1907Q1|N|N|N|2417608|2417638|2417266|2417539|N|N|N|N|N| +2417632|AAAAAAAAAODOECAA|1907-02-25|85|373|29|1907|1|2|25|1|1907|29|373|Monday|1907Q1|N|N|N|2417608|2417638|2417267|2417540|N|N|N|N|N| +2417633|AAAAAAAABODOECAA|1907-02-26|85|374|29|1907|2|2|26|1|1907|29|374|Tuesday|1907Q1|N|N|N|2417608|2417638|2417268|2417541|N|N|N|N|N| +2417634|AAAAAAAACODOECAA|1907-02-27|85|374|29|1907|3|2|27|1|1907|29|374|Wednesday|1907Q1|N|N|N|2417608|2417638|2417269|2417542|N|N|N|N|N| +2417635|AAAAAAAADODOECAA|1907-02-28|85|374|29|1907|4|2|28|1|1907|29|374|Thursday|1907Q1|N|N|N|2417608|2417638|2417270|2417543|N|N|N|N|N| +2417636|AAAAAAAAEODOECAA|1907-03-01|86|374|30|1907|5|3|1|1|1907|30|374|Friday|1907Q1|N|Y|N|2417636|2417694|2417271|2417544|N|N|N|N|N| +2417637|AAAAAAAAFODOECAA|1907-03-02|86|374|30|1907|6|3|2|1|1907|30|374|Saturday|1907Q1|N|Y|N|2417636|2417694|2417272|2417545|N|N|N|N|N| +2417638|AAAAAAAAGODOECAA|1907-03-03|86|374|30|1907|0|3|3|1|1907|30|374|Sunday|1907Q1|N|N|N|2417636|2417694|2417273|2417546|N|N|N|N|N| +2417639|AAAAAAAAHODOECAA|1907-03-04|86|374|30|1907|1|3|4|1|1907|30|374|Monday|1907Q1|N|N|N|2417636|2417694|2417274|2417547|N|N|N|N|N| +2417640|AAAAAAAAIODOECAA|1907-03-05|86|375|30|1907|2|3|5|1|1907|30|375|Tuesday|1907Q1|N|N|N|2417636|2417694|2417275|2417548|N|N|N|N|N| +2417641|AAAAAAAAJODOECAA|1907-03-06|86|375|30|1907|3|3|6|1|1907|30|375|Wednesday|1907Q1|N|N|N|2417636|2417694|2417276|2417549|N|N|N|N|N| +2417642|AAAAAAAAKODOECAA|1907-03-07|86|375|30|1907|4|3|7|1|1907|30|375|Thursday|1907Q1|N|N|N|2417636|2417694|2417277|2417550|N|N|N|N|N| +2417643|AAAAAAAALODOECAA|1907-03-08|86|375|30|1907|5|3|8|1|1907|30|375|Friday|1907Q1|N|Y|N|2417636|2417694|2417278|2417551|N|N|N|N|N| +2417644|AAAAAAAAMODOECAA|1907-03-09|86|375|30|1907|6|3|9|1|1907|30|375|Saturday|1907Q1|N|Y|N|2417636|2417694|2417279|2417552|N|N|N|N|N| +2417645|AAAAAAAANODOECAA|1907-03-10|86|375|30|1907|0|3|10|1|1907|30|375|Sunday|1907Q1|N|N|N|2417636|2417694|2417280|2417553|N|N|N|N|N| +2417646|AAAAAAAAOODOECAA|1907-03-11|86|375|30|1907|1|3|11|1|1907|30|375|Monday|1907Q1|N|N|N|2417636|2417694|2417281|2417554|N|N|N|N|N| +2417647|AAAAAAAAPODOECAA|1907-03-12|86|376|30|1907|2|3|12|1|1907|30|376|Tuesday|1907Q1|N|N|N|2417636|2417694|2417282|2417555|N|N|N|N|N| +2417648|AAAAAAAAAPDOECAA|1907-03-13|86|376|30|1907|3|3|13|1|1907|30|376|Wednesday|1907Q1|N|N|N|2417636|2417694|2417283|2417556|N|N|N|N|N| +2417649|AAAAAAAABPDOECAA|1907-03-14|86|376|30|1907|4|3|14|1|1907|30|376|Thursday|1907Q1|N|N|N|2417636|2417694|2417284|2417557|N|N|N|N|N| +2417650|AAAAAAAACPDOECAA|1907-03-15|86|376|30|1907|5|3|15|1|1907|30|376|Friday|1907Q1|N|Y|N|2417636|2417694|2417285|2417558|N|N|N|N|N| +2417651|AAAAAAAADPDOECAA|1907-03-16|86|376|30|1907|6|3|16|1|1907|30|376|Saturday|1907Q1|N|Y|N|2417636|2417694|2417286|2417559|N|N|N|N|N| +2417652|AAAAAAAAEPDOECAA|1907-03-17|86|376|30|1907|0|3|17|1|1907|30|376|Sunday|1907Q1|N|N|N|2417636|2417694|2417287|2417560|N|N|N|N|N| +2417653|AAAAAAAAFPDOECAA|1907-03-18|86|376|30|1907|1|3|18|1|1907|30|376|Monday|1907Q1|N|N|N|2417636|2417694|2417288|2417561|N|N|N|N|N| +2417654|AAAAAAAAGPDOECAA|1907-03-19|86|377|30|1907|2|3|19|1|1907|30|377|Tuesday|1907Q1|N|N|N|2417636|2417694|2417289|2417562|N|N|N|N|N| +2417655|AAAAAAAAHPDOECAA|1907-03-20|86|377|30|1907|3|3|20|1|1907|30|377|Wednesday|1907Q1|N|N|N|2417636|2417694|2417290|2417563|N|N|N|N|N| +2417656|AAAAAAAAIPDOECAA|1907-03-21|86|377|30|1907|4|3|21|1|1907|30|377|Thursday|1907Q1|N|N|N|2417636|2417694|2417291|2417564|N|N|N|N|N| +2417657|AAAAAAAAJPDOECAA|1907-03-22|86|377|30|1907|5|3|22|1|1907|30|377|Friday|1907Q1|N|Y|N|2417636|2417694|2417292|2417565|N|N|N|N|N| +2417658|AAAAAAAAKPDOECAA|1907-03-23|86|377|30|1907|6|3|23|1|1907|30|377|Saturday|1907Q1|N|Y|N|2417636|2417694|2417293|2417566|N|N|N|N|N| +2417659|AAAAAAAALPDOECAA|1907-03-24|86|377|30|1907|0|3|24|1|1907|30|377|Sunday|1907Q1|N|N|N|2417636|2417694|2417294|2417567|N|N|N|N|N| +2417660|AAAAAAAAMPDOECAA|1907-03-25|86|377|30|1907|1|3|25|1|1907|30|377|Monday|1907Q1|N|N|N|2417636|2417694|2417295|2417568|N|N|N|N|N| +2417661|AAAAAAAANPDOECAA|1907-03-26|86|378|30|1907|2|3|26|1|1907|30|378|Tuesday|1907Q1|N|N|N|2417636|2417694|2417296|2417569|N|N|N|N|N| +2417662|AAAAAAAAOPDOECAA|1907-03-27|86|378|30|1907|3|3|27|1|1907|30|378|Wednesday|1907Q1|N|N|N|2417636|2417694|2417297|2417570|N|N|N|N|N| +2417663|AAAAAAAAPPDOECAA|1907-03-28|86|378|30|1907|4|3|28|1|1907|30|378|Thursday|1907Q1|N|N|N|2417636|2417694|2417298|2417571|N|N|N|N|N| +2417664|AAAAAAAAAAEOECAA|1907-03-29|86|378|30|1907|5|3|29|1|1907|30|378|Friday|1907Q1|N|Y|N|2417636|2417694|2417299|2417572|N|N|N|N|N| +2417665|AAAAAAAABAEOECAA|1907-03-30|86|378|30|1907|6|3|30|1|1907|30|378|Saturday|1907Q1|N|Y|N|2417636|2417694|2417300|2417573|N|N|N|N|N| +2417666|AAAAAAAACAEOECAA|1907-03-31|86|378|30|1907|0|3|31|1|1907|30|378|Sunday|1907Q1|N|N|N|2417636|2417694|2417301|2417574|N|N|N|N|N| +2417667|AAAAAAAADAEOECAA|1907-04-01|87|378|30|1907|1|4|1|1|1907|30|378|Monday|1907Q1|N|N|N|2417667|2417756|2417302|2417577|N|N|N|N|N| +2417668|AAAAAAAAEAEOECAA|1907-04-02|87|379|30|1907|2|4|2|2|1907|30|379|Tuesday|1907Q2|N|N|N|2417667|2417756|2417303|2417578|N|N|N|N|N| +2417669|AAAAAAAAFAEOECAA|1907-04-03|87|379|30|1907|3|4|3|2|1907|30|379|Wednesday|1907Q2|N|N|N|2417667|2417756|2417304|2417579|N|N|N|N|N| +2417670|AAAAAAAAGAEOECAA|1907-04-04|87|379|30|1907|4|4|4|2|1907|30|379|Thursday|1907Q2|N|N|N|2417667|2417756|2417305|2417580|N|N|N|N|N| +2417671|AAAAAAAAHAEOECAA|1907-04-05|87|379|30|1907|5|4|5|2|1907|30|379|Friday|1907Q2|N|Y|N|2417667|2417756|2417306|2417581|N|N|N|N|N| +2417672|AAAAAAAAIAEOECAA|1907-04-06|87|379|30|1907|6|4|6|2|1907|30|379|Saturday|1907Q2|N|Y|N|2417667|2417756|2417307|2417582|N|N|N|N|N| +2417673|AAAAAAAAJAEOECAA|1907-04-07|87|379|30|1907|0|4|7|2|1907|30|379|Sunday|1907Q2|N|N|N|2417667|2417756|2417308|2417583|N|N|N|N|N| +2417674|AAAAAAAAKAEOECAA|1907-04-08|87|379|30|1907|1|4|8|2|1907|30|379|Monday|1907Q2|N|N|N|2417667|2417756|2417309|2417584|N|N|N|N|N| +2417675|AAAAAAAALAEOECAA|1907-04-09|87|380|30|1907|2|4|9|2|1907|30|380|Tuesday|1907Q2|N|N|N|2417667|2417756|2417310|2417585|N|N|N|N|N| +2417676|AAAAAAAAMAEOECAA|1907-04-10|87|380|30|1907|3|4|10|2|1907|30|380|Wednesday|1907Q2|N|N|N|2417667|2417756|2417311|2417586|N|N|N|N|N| +2417677|AAAAAAAANAEOECAA|1907-04-11|87|380|30|1907|4|4|11|2|1907|30|380|Thursday|1907Q2|N|N|N|2417667|2417756|2417312|2417587|N|N|N|N|N| +2417678|AAAAAAAAOAEOECAA|1907-04-12|87|380|30|1907|5|4|12|2|1907|30|380|Friday|1907Q2|N|Y|N|2417667|2417756|2417313|2417588|N|N|N|N|N| +2417679|AAAAAAAAPAEOECAA|1907-04-13|87|380|30|1907|6|4|13|2|1907|30|380|Saturday|1907Q2|N|Y|N|2417667|2417756|2417314|2417589|N|N|N|N|N| +2417680|AAAAAAAAABEOECAA|1907-04-14|87|380|30|1907|0|4|14|2|1907|30|380|Sunday|1907Q2|N|N|N|2417667|2417756|2417315|2417590|N|N|N|N|N| +2417681|AAAAAAAABBEOECAA|1907-04-15|87|380|30|1907|1|4|15|2|1907|30|380|Monday|1907Q2|N|N|N|2417667|2417756|2417316|2417591|N|N|N|N|N| +2417682|AAAAAAAACBEOECAA|1907-04-16|87|381|30|1907|2|4|16|2|1907|30|381|Tuesday|1907Q2|N|N|N|2417667|2417756|2417317|2417592|N|N|N|N|N| +2417683|AAAAAAAADBEOECAA|1907-04-17|87|381|30|1907|3|4|17|2|1907|30|381|Wednesday|1907Q2|N|N|N|2417667|2417756|2417318|2417593|N|N|N|N|N| +2417684|AAAAAAAAEBEOECAA|1907-04-18|87|381|30|1907|4|4|18|2|1907|30|381|Thursday|1907Q2|N|N|N|2417667|2417756|2417319|2417594|N|N|N|N|N| +2417685|AAAAAAAAFBEOECAA|1907-04-19|87|381|30|1907|5|4|19|2|1907|30|381|Friday|1907Q2|N|Y|N|2417667|2417756|2417320|2417595|N|N|N|N|N| +2417686|AAAAAAAAGBEOECAA|1907-04-20|87|381|30|1907|6|4|20|2|1907|30|381|Saturday|1907Q2|N|Y|N|2417667|2417756|2417321|2417596|N|N|N|N|N| +2417687|AAAAAAAAHBEOECAA|1907-04-21|87|381|30|1907|0|4|21|2|1907|30|381|Sunday|1907Q2|N|N|N|2417667|2417756|2417322|2417597|N|N|N|N|N| +2417688|AAAAAAAAIBEOECAA|1907-04-22|87|381|30|1907|1|4|22|2|1907|30|381|Monday|1907Q2|N|N|N|2417667|2417756|2417323|2417598|N|N|N|N|N| +2417689|AAAAAAAAJBEOECAA|1907-04-23|87|382|30|1907|2|4|23|2|1907|30|382|Tuesday|1907Q2|N|N|N|2417667|2417756|2417324|2417599|N|N|N|N|N| +2417690|AAAAAAAAKBEOECAA|1907-04-24|87|382|30|1907|3|4|24|2|1907|30|382|Wednesday|1907Q2|N|N|N|2417667|2417756|2417325|2417600|N|N|N|N|N| +2417691|AAAAAAAALBEOECAA|1907-04-25|87|382|30|1907|4|4|25|2|1907|30|382|Thursday|1907Q2|N|N|N|2417667|2417756|2417326|2417601|N|N|N|N|N| +2417692|AAAAAAAAMBEOECAA|1907-04-26|87|382|30|1907|5|4|26|2|1907|30|382|Friday|1907Q2|N|Y|N|2417667|2417756|2417327|2417602|N|N|N|N|N| +2417693|AAAAAAAANBEOECAA|1907-04-27|87|382|30|1907|6|4|27|2|1907|30|382|Saturday|1907Q2|N|Y|N|2417667|2417756|2417328|2417603|N|N|N|N|N| +2417694|AAAAAAAAOBEOECAA|1907-04-28|87|382|30|1907|0|4|28|2|1907|30|382|Sunday|1907Q2|N|N|N|2417667|2417756|2417329|2417604|N|N|N|N|N| +2417695|AAAAAAAAPBEOECAA|1907-04-29|87|382|30|1907|1|4|29|2|1907|30|382|Monday|1907Q2|N|N|N|2417667|2417756|2417330|2417605|N|N|N|N|N| +2417696|AAAAAAAAACEOECAA|1907-04-30|87|383|30|1907|2|4|30|2|1907|30|383|Tuesday|1907Q2|N|N|N|2417667|2417756|2417331|2417606|N|N|N|N|N| +2417697|AAAAAAAABCEOECAA|1907-05-01|88|383|30|1907|3|5|1|2|1907|30|383|Wednesday|1907Q2|N|N|N|2417697|2417816|2417332|2417607|N|N|N|N|N| +2417698|AAAAAAAACCEOECAA|1907-05-02|88|383|30|1907|4|5|2|2|1907|30|383|Thursday|1907Q2|N|N|N|2417697|2417816|2417333|2417608|N|N|N|N|N| +2417699|AAAAAAAADCEOECAA|1907-05-03|88|383|30|1907|5|5|3|2|1907|30|383|Friday|1907Q2|N|Y|N|2417697|2417816|2417334|2417609|N|N|N|N|N| +2417700|AAAAAAAAECEOECAA|1907-05-04|88|383|30|1907|6|5|4|2|1907|30|383|Saturday|1907Q2|N|Y|N|2417697|2417816|2417335|2417610|N|N|N|N|N| +2417701|AAAAAAAAFCEOECAA|1907-05-05|88|383|30|1907|0|5|5|2|1907|30|383|Sunday|1907Q2|N|N|N|2417697|2417816|2417336|2417611|N|N|N|N|N| +2417702|AAAAAAAAGCEOECAA|1907-05-06|88|383|30|1907|1|5|6|2|1907|30|383|Monday|1907Q2|N|N|N|2417697|2417816|2417337|2417612|N|N|N|N|N| +2417703|AAAAAAAAHCEOECAA|1907-05-07|88|384|30|1907|2|5|7|2|1907|30|384|Tuesday|1907Q2|N|N|N|2417697|2417816|2417338|2417613|N|N|N|N|N| +2417704|AAAAAAAAICEOECAA|1907-05-08|88|384|30|1907|3|5|8|2|1907|30|384|Wednesday|1907Q2|N|N|N|2417697|2417816|2417339|2417614|N|N|N|N|N| +2417705|AAAAAAAAJCEOECAA|1907-05-09|88|384|30|1907|4|5|9|2|1907|30|384|Thursday|1907Q2|N|N|N|2417697|2417816|2417340|2417615|N|N|N|N|N| +2417706|AAAAAAAAKCEOECAA|1907-05-10|88|384|30|1907|5|5|10|2|1907|30|384|Friday|1907Q2|N|Y|N|2417697|2417816|2417341|2417616|N|N|N|N|N| +2417707|AAAAAAAALCEOECAA|1907-05-11|88|384|30|1907|6|5|11|2|1907|30|384|Saturday|1907Q2|N|Y|N|2417697|2417816|2417342|2417617|N|N|N|N|N| +2417708|AAAAAAAAMCEOECAA|1907-05-12|88|384|30|1907|0|5|12|2|1907|30|384|Sunday|1907Q2|N|N|N|2417697|2417816|2417343|2417618|N|N|N|N|N| +2417709|AAAAAAAANCEOECAA|1907-05-13|88|384|30|1907|1|5|13|2|1907|30|384|Monday|1907Q2|N|N|N|2417697|2417816|2417344|2417619|N|N|N|N|N| +2417710|AAAAAAAAOCEOECAA|1907-05-14|88|385|30|1907|2|5|14|2|1907|30|385|Tuesday|1907Q2|N|N|N|2417697|2417816|2417345|2417620|N|N|N|N|N| +2417711|AAAAAAAAPCEOECAA|1907-05-15|88|385|30|1907|3|5|15|2|1907|30|385|Wednesday|1907Q2|N|N|N|2417697|2417816|2417346|2417621|N|N|N|N|N| +2417712|AAAAAAAAADEOECAA|1907-05-16|88|385|30|1907|4|5|16|2|1907|30|385|Thursday|1907Q2|N|N|N|2417697|2417816|2417347|2417622|N|N|N|N|N| +2417713|AAAAAAAABDEOECAA|1907-05-17|88|385|30|1907|5|5|17|2|1907|30|385|Friday|1907Q2|N|Y|N|2417697|2417816|2417348|2417623|N|N|N|N|N| +2417714|AAAAAAAACDEOECAA|1907-05-18|88|385|30|1907|6|5|18|2|1907|30|385|Saturday|1907Q2|N|Y|N|2417697|2417816|2417349|2417624|N|N|N|N|N| +2417715|AAAAAAAADDEOECAA|1907-05-19|88|385|30|1907|0|5|19|2|1907|30|385|Sunday|1907Q2|N|N|N|2417697|2417816|2417350|2417625|N|N|N|N|N| +2417716|AAAAAAAAEDEOECAA|1907-05-20|88|385|30|1907|1|5|20|2|1907|30|385|Monday|1907Q2|N|N|N|2417697|2417816|2417351|2417626|N|N|N|N|N| +2417717|AAAAAAAAFDEOECAA|1907-05-21|88|386|30|1907|2|5|21|2|1907|30|386|Tuesday|1907Q2|N|N|N|2417697|2417816|2417352|2417627|N|N|N|N|N| +2417718|AAAAAAAAGDEOECAA|1907-05-22|88|386|30|1907|3|5|22|2|1907|30|386|Wednesday|1907Q2|N|N|N|2417697|2417816|2417353|2417628|N|N|N|N|N| +2417719|AAAAAAAAHDEOECAA|1907-05-23|88|386|30|1907|4|5|23|2|1907|30|386|Thursday|1907Q2|N|N|N|2417697|2417816|2417354|2417629|N|N|N|N|N| +2417720|AAAAAAAAIDEOECAA|1907-05-24|88|386|30|1907|5|5|24|2|1907|30|386|Friday|1907Q2|N|Y|N|2417697|2417816|2417355|2417630|N|N|N|N|N| +2417721|AAAAAAAAJDEOECAA|1907-05-25|88|386|30|1907|6|5|25|2|1907|30|386|Saturday|1907Q2|N|Y|N|2417697|2417816|2417356|2417631|N|N|N|N|N| +2417722|AAAAAAAAKDEOECAA|1907-05-26|88|386|30|1907|0|5|26|2|1907|30|386|Sunday|1907Q2|N|N|N|2417697|2417816|2417357|2417632|N|N|N|N|N| +2417723|AAAAAAAALDEOECAA|1907-05-27|88|386|30|1907|1|5|27|2|1907|30|386|Monday|1907Q2|N|N|N|2417697|2417816|2417358|2417633|N|N|N|N|N| +2417724|AAAAAAAAMDEOECAA|1907-05-28|88|387|30|1907|2|5|28|2|1907|30|387|Tuesday|1907Q2|N|N|N|2417697|2417816|2417359|2417634|N|N|N|N|N| +2417725|AAAAAAAANDEOECAA|1907-05-29|88|387|30|1907|3|5|29|2|1907|30|387|Wednesday|1907Q2|N|N|N|2417697|2417816|2417360|2417635|N|N|N|N|N| +2417726|AAAAAAAAODEOECAA|1907-05-30|88|387|30|1907|4|5|30|2|1907|30|387|Thursday|1907Q2|N|N|N|2417697|2417816|2417361|2417636|N|N|N|N|N| +2417727|AAAAAAAAPDEOECAA|1907-05-31|88|387|30|1907|5|5|31|2|1907|30|387|Friday|1907Q2|N|Y|N|2417697|2417816|2417362|2417637|N|N|N|N|N| +2417728|AAAAAAAAAEEOECAA|1907-06-01|89|387|31|1907|6|6|1|2|1907|31|387|Saturday|1907Q2|N|Y|N|2417728|2417878|2417363|2417638|N|N|N|N|N| +2417729|AAAAAAAABEEOECAA|1907-06-02|89|387|31|1907|0|6|2|2|1907|31|387|Sunday|1907Q2|N|N|N|2417728|2417878|2417364|2417639|N|N|N|N|N| +2417730|AAAAAAAACEEOECAA|1907-06-03|89|387|31|1907|1|6|3|2|1907|31|387|Monday|1907Q2|N|N|N|2417728|2417878|2417365|2417640|N|N|N|N|N| +2417731|AAAAAAAADEEOECAA|1907-06-04|89|388|31|1907|2|6|4|2|1907|31|388|Tuesday|1907Q2|N|N|N|2417728|2417878|2417366|2417641|N|N|N|N|N| +2417732|AAAAAAAAEEEOECAA|1907-06-05|89|388|31|1907|3|6|5|2|1907|31|388|Wednesday|1907Q2|N|N|N|2417728|2417878|2417367|2417642|N|N|N|N|N| +2417733|AAAAAAAAFEEOECAA|1907-06-06|89|388|31|1907|4|6|6|2|1907|31|388|Thursday|1907Q2|N|N|N|2417728|2417878|2417368|2417643|N|N|N|N|N| +2417734|AAAAAAAAGEEOECAA|1907-06-07|89|388|31|1907|5|6|7|2|1907|31|388|Friday|1907Q2|N|Y|N|2417728|2417878|2417369|2417644|N|N|N|N|N| +2417735|AAAAAAAAHEEOECAA|1907-06-08|89|388|31|1907|6|6|8|2|1907|31|388|Saturday|1907Q2|N|Y|N|2417728|2417878|2417370|2417645|N|N|N|N|N| +2417736|AAAAAAAAIEEOECAA|1907-06-09|89|388|31|1907|0|6|9|2|1907|31|388|Sunday|1907Q2|N|N|N|2417728|2417878|2417371|2417646|N|N|N|N|N| +2417737|AAAAAAAAJEEOECAA|1907-06-10|89|388|31|1907|1|6|10|2|1907|31|388|Monday|1907Q2|N|N|N|2417728|2417878|2417372|2417647|N|N|N|N|N| +2417738|AAAAAAAAKEEOECAA|1907-06-11|89|389|31|1907|2|6|11|2|1907|31|389|Tuesday|1907Q2|N|N|N|2417728|2417878|2417373|2417648|N|N|N|N|N| +2417739|AAAAAAAALEEOECAA|1907-06-12|89|389|31|1907|3|6|12|2|1907|31|389|Wednesday|1907Q2|N|N|N|2417728|2417878|2417374|2417649|N|N|N|N|N| +2417740|AAAAAAAAMEEOECAA|1907-06-13|89|389|31|1907|4|6|13|2|1907|31|389|Thursday|1907Q2|N|N|N|2417728|2417878|2417375|2417650|N|N|N|N|N| +2417741|AAAAAAAANEEOECAA|1907-06-14|89|389|31|1907|5|6|14|2|1907|31|389|Friday|1907Q2|N|Y|N|2417728|2417878|2417376|2417651|N|N|N|N|N| +2417742|AAAAAAAAOEEOECAA|1907-06-15|89|389|31|1907|6|6|15|2|1907|31|389|Saturday|1907Q2|N|Y|N|2417728|2417878|2417377|2417652|N|N|N|N|N| +2417743|AAAAAAAAPEEOECAA|1907-06-16|89|389|31|1907|0|6|16|2|1907|31|389|Sunday|1907Q2|N|N|N|2417728|2417878|2417378|2417653|N|N|N|N|N| +2417744|AAAAAAAAAFEOECAA|1907-06-17|89|389|31|1907|1|6|17|2|1907|31|389|Monday|1907Q2|N|N|N|2417728|2417878|2417379|2417654|N|N|N|N|N| +2417745|AAAAAAAABFEOECAA|1907-06-18|89|390|31|1907|2|6|18|2|1907|31|390|Tuesday|1907Q2|N|N|N|2417728|2417878|2417380|2417655|N|N|N|N|N| +2417746|AAAAAAAACFEOECAA|1907-06-19|89|390|31|1907|3|6|19|2|1907|31|390|Wednesday|1907Q2|N|N|N|2417728|2417878|2417381|2417656|N|N|N|N|N| +2417747|AAAAAAAADFEOECAA|1907-06-20|89|390|31|1907|4|6|20|2|1907|31|390|Thursday|1907Q2|N|N|N|2417728|2417878|2417382|2417657|N|N|N|N|N| +2417748|AAAAAAAAEFEOECAA|1907-06-21|89|390|31|1907|5|6|21|2|1907|31|390|Friday|1907Q2|N|Y|N|2417728|2417878|2417383|2417658|N|N|N|N|N| +2417749|AAAAAAAAFFEOECAA|1907-06-22|89|390|31|1907|6|6|22|2|1907|31|390|Saturday|1907Q2|N|Y|N|2417728|2417878|2417384|2417659|N|N|N|N|N| +2417750|AAAAAAAAGFEOECAA|1907-06-23|89|390|31|1907|0|6|23|2|1907|31|390|Sunday|1907Q2|N|N|N|2417728|2417878|2417385|2417660|N|N|N|N|N| +2417751|AAAAAAAAHFEOECAA|1907-06-24|89|390|31|1907|1|6|24|2|1907|31|390|Monday|1907Q2|N|N|N|2417728|2417878|2417386|2417661|N|N|N|N|N| +2417752|AAAAAAAAIFEOECAA|1907-06-25|89|391|31|1907|2|6|25|2|1907|31|391|Tuesday|1907Q2|N|N|N|2417728|2417878|2417387|2417662|N|N|N|N|N| +2417753|AAAAAAAAJFEOECAA|1907-06-26|89|391|31|1907|3|6|26|2|1907|31|391|Wednesday|1907Q2|N|N|N|2417728|2417878|2417388|2417663|N|N|N|N|N| +2417754|AAAAAAAAKFEOECAA|1907-06-27|89|391|31|1907|4|6|27|2|1907|31|391|Thursday|1907Q2|N|N|N|2417728|2417878|2417389|2417664|N|N|N|N|N| +2417755|AAAAAAAALFEOECAA|1907-06-28|89|391|31|1907|5|6|28|2|1907|31|391|Friday|1907Q2|N|Y|N|2417728|2417878|2417390|2417665|N|N|N|N|N| +2417756|AAAAAAAAMFEOECAA|1907-06-29|89|391|31|1907|6|6|29|2|1907|31|391|Saturday|1907Q2|N|Y|N|2417728|2417878|2417391|2417666|N|N|N|N|N| +2417757|AAAAAAAANFEOECAA|1907-06-30|89|391|31|1907|0|6|30|2|1907|31|391|Sunday|1907Q2|N|N|N|2417728|2417878|2417392|2417667|N|N|N|N|N| +2417758|AAAAAAAAOFEOECAA|1907-07-01|90|391|31|1907|1|7|1|2|1907|31|391|Monday|1907Q2|N|N|N|2417758|2417938|2417393|2417667|N|N|N|N|N| +2417759|AAAAAAAAPFEOECAA|1907-07-02|90|392|31|1907|2|7|2|3|1907|31|392|Tuesday|1907Q3|N|N|N|2417758|2417938|2417394|2417668|N|N|N|N|N| +2417760|AAAAAAAAAGEOECAA|1907-07-03|90|392|31|1907|3|7|3|3|1907|31|392|Wednesday|1907Q3|N|N|N|2417758|2417938|2417395|2417669|N|N|N|N|N| +2417761|AAAAAAAABGEOECAA|1907-07-04|90|392|31|1907|4|7|4|3|1907|31|392|Thursday|1907Q3|N|N|N|2417758|2417938|2417396|2417670|N|N|N|N|N| +2417762|AAAAAAAACGEOECAA|1907-07-05|90|392|31|1907|5|7|5|3|1907|31|392|Friday|1907Q3|Y|Y|N|2417758|2417938|2417397|2417671|N|N|N|N|N| +2417763|AAAAAAAADGEOECAA|1907-07-06|90|392|31|1907|6|7|6|3|1907|31|392|Saturday|1907Q3|N|Y|Y|2417758|2417938|2417398|2417672|N|N|N|N|N| +2417764|AAAAAAAAEGEOECAA|1907-07-07|90|392|31|1907|0|7|7|3|1907|31|392|Sunday|1907Q3|N|N|N|2417758|2417938|2417399|2417673|N|N|N|N|N| +2417765|AAAAAAAAFGEOECAA|1907-07-08|90|392|31|1907|1|7|8|3|1907|31|392|Monday|1907Q3|N|N|N|2417758|2417938|2417400|2417674|N|N|N|N|N| +2417766|AAAAAAAAGGEOECAA|1907-07-09|90|393|31|1907|2|7|9|3|1907|31|393|Tuesday|1907Q3|N|N|N|2417758|2417938|2417401|2417675|N|N|N|N|N| +2417767|AAAAAAAAHGEOECAA|1907-07-10|90|393|31|1907|3|7|10|3|1907|31|393|Wednesday|1907Q3|N|N|N|2417758|2417938|2417402|2417676|N|N|N|N|N| +2417768|AAAAAAAAIGEOECAA|1907-07-11|90|393|31|1907|4|7|11|3|1907|31|393|Thursday|1907Q3|N|N|N|2417758|2417938|2417403|2417677|N|N|N|N|N| +2417769|AAAAAAAAJGEOECAA|1907-07-12|90|393|31|1907|5|7|12|3|1907|31|393|Friday|1907Q3|N|Y|N|2417758|2417938|2417404|2417678|N|N|N|N|N| +2417770|AAAAAAAAKGEOECAA|1907-07-13|90|393|31|1907|6|7|13|3|1907|31|393|Saturday|1907Q3|N|Y|N|2417758|2417938|2417405|2417679|N|N|N|N|N| +2417771|AAAAAAAALGEOECAA|1907-07-14|90|393|31|1907|0|7|14|3|1907|31|393|Sunday|1907Q3|N|N|N|2417758|2417938|2417406|2417680|N|N|N|N|N| +2417772|AAAAAAAAMGEOECAA|1907-07-15|90|393|31|1907|1|7|15|3|1907|31|393|Monday|1907Q3|N|N|N|2417758|2417938|2417407|2417681|N|N|N|N|N| +2417773|AAAAAAAANGEOECAA|1907-07-16|90|394|31|1907|2|7|16|3|1907|31|394|Tuesday|1907Q3|N|N|N|2417758|2417938|2417408|2417682|N|N|N|N|N| +2417774|AAAAAAAAOGEOECAA|1907-07-17|90|394|31|1907|3|7|17|3|1907|31|394|Wednesday|1907Q3|N|N|N|2417758|2417938|2417409|2417683|N|N|N|N|N| +2417775|AAAAAAAAPGEOECAA|1907-07-18|90|394|31|1907|4|7|18|3|1907|31|394|Thursday|1907Q3|N|N|N|2417758|2417938|2417410|2417684|N|N|N|N|N| +2417776|AAAAAAAAAHEOECAA|1907-07-19|90|394|31|1907|5|7|19|3|1907|31|394|Friday|1907Q3|N|Y|N|2417758|2417938|2417411|2417685|N|N|N|N|N| +2417777|AAAAAAAABHEOECAA|1907-07-20|90|394|31|1907|6|7|20|3|1907|31|394|Saturday|1907Q3|N|Y|N|2417758|2417938|2417412|2417686|N|N|N|N|N| +2417778|AAAAAAAACHEOECAA|1907-07-21|90|394|31|1907|0|7|21|3|1907|31|394|Sunday|1907Q3|N|N|N|2417758|2417938|2417413|2417687|N|N|N|N|N| +2417779|AAAAAAAADHEOECAA|1907-07-22|90|394|31|1907|1|7|22|3|1907|31|394|Monday|1907Q3|N|N|N|2417758|2417938|2417414|2417688|N|N|N|N|N| +2417780|AAAAAAAAEHEOECAA|1907-07-23|90|395|31|1907|2|7|23|3|1907|31|395|Tuesday|1907Q3|N|N|N|2417758|2417938|2417415|2417689|N|N|N|N|N| +2417781|AAAAAAAAFHEOECAA|1907-07-24|90|395|31|1907|3|7|24|3|1907|31|395|Wednesday|1907Q3|N|N|N|2417758|2417938|2417416|2417690|N|N|N|N|N| +2417782|AAAAAAAAGHEOECAA|1907-07-25|90|395|31|1907|4|7|25|3|1907|31|395|Thursday|1907Q3|N|N|N|2417758|2417938|2417417|2417691|N|N|N|N|N| +2417783|AAAAAAAAHHEOECAA|1907-07-26|90|395|31|1907|5|7|26|3|1907|31|395|Friday|1907Q3|N|Y|N|2417758|2417938|2417418|2417692|N|N|N|N|N| +2417784|AAAAAAAAIHEOECAA|1907-07-27|90|395|31|1907|6|7|27|3|1907|31|395|Saturday|1907Q3|N|Y|N|2417758|2417938|2417419|2417693|N|N|N|N|N| +2417785|AAAAAAAAJHEOECAA|1907-07-28|90|395|31|1907|0|7|28|3|1907|31|395|Sunday|1907Q3|N|N|N|2417758|2417938|2417420|2417694|N|N|N|N|N| +2417786|AAAAAAAAKHEOECAA|1907-07-29|90|395|31|1907|1|7|29|3|1907|31|395|Monday|1907Q3|N|N|N|2417758|2417938|2417421|2417695|N|N|N|N|N| +2417787|AAAAAAAALHEOECAA|1907-07-30|90|396|31|1907|2|7|30|3|1907|31|396|Tuesday|1907Q3|N|N|N|2417758|2417938|2417422|2417696|N|N|N|N|N| +2417788|AAAAAAAAMHEOECAA|1907-07-31|90|396|31|1907|3|7|31|3|1907|31|396|Wednesday|1907Q3|N|N|N|2417758|2417938|2417423|2417697|N|N|N|N|N| +2417789|AAAAAAAANHEOECAA|1907-08-01|91|396|31|1907|4|8|1|3|1907|31|396|Thursday|1907Q3|N|N|N|2417789|2418000|2417424|2417698|N|N|N|N|N| +2417790|AAAAAAAAOHEOECAA|1907-08-02|91|396|31|1907|5|8|2|3|1907|31|396|Friday|1907Q3|N|Y|N|2417789|2418000|2417425|2417699|N|N|N|N|N| +2417791|AAAAAAAAPHEOECAA|1907-08-03|91|396|31|1907|6|8|3|3|1907|31|396|Saturday|1907Q3|N|Y|N|2417789|2418000|2417426|2417700|N|N|N|N|N| +2417792|AAAAAAAAAIEOECAA|1907-08-04|91|396|31|1907|0|8|4|3|1907|31|396|Sunday|1907Q3|N|N|N|2417789|2418000|2417427|2417701|N|N|N|N|N| +2417793|AAAAAAAABIEOECAA|1907-08-05|91|396|31|1907|1|8|5|3|1907|31|396|Monday|1907Q3|N|N|N|2417789|2418000|2417428|2417702|N|N|N|N|N| +2417794|AAAAAAAACIEOECAA|1907-08-06|91|397|31|1907|2|8|6|3|1907|31|397|Tuesday|1907Q3|N|N|N|2417789|2418000|2417429|2417703|N|N|N|N|N| +2417795|AAAAAAAADIEOECAA|1907-08-07|91|397|31|1907|3|8|7|3|1907|31|397|Wednesday|1907Q3|N|N|N|2417789|2418000|2417430|2417704|N|N|N|N|N| +2417796|AAAAAAAAEIEOECAA|1907-08-08|91|397|31|1907|4|8|8|3|1907|31|397|Thursday|1907Q3|N|N|N|2417789|2418000|2417431|2417705|N|N|N|N|N| +2417797|AAAAAAAAFIEOECAA|1907-08-09|91|397|31|1907|5|8|9|3|1907|31|397|Friday|1907Q3|N|Y|N|2417789|2418000|2417432|2417706|N|N|N|N|N| +2417798|AAAAAAAAGIEOECAA|1907-08-10|91|397|31|1907|6|8|10|3|1907|31|397|Saturday|1907Q3|N|Y|N|2417789|2418000|2417433|2417707|N|N|N|N|N| +2417799|AAAAAAAAHIEOECAA|1907-08-11|91|397|31|1907|0|8|11|3|1907|31|397|Sunday|1907Q3|N|N|N|2417789|2418000|2417434|2417708|N|N|N|N|N| +2417800|AAAAAAAAIIEOECAA|1907-08-12|91|397|31|1907|1|8|12|3|1907|31|397|Monday|1907Q3|N|N|N|2417789|2418000|2417435|2417709|N|N|N|N|N| +2417801|AAAAAAAAJIEOECAA|1907-08-13|91|398|31|1907|2|8|13|3|1907|31|398|Tuesday|1907Q3|N|N|N|2417789|2418000|2417436|2417710|N|N|N|N|N| +2417802|AAAAAAAAKIEOECAA|1907-08-14|91|398|31|1907|3|8|14|3|1907|31|398|Wednesday|1907Q3|N|N|N|2417789|2418000|2417437|2417711|N|N|N|N|N| +2417803|AAAAAAAALIEOECAA|1907-08-15|91|398|31|1907|4|8|15|3|1907|31|398|Thursday|1907Q3|N|N|N|2417789|2418000|2417438|2417712|N|N|N|N|N| +2417804|AAAAAAAAMIEOECAA|1907-08-16|91|398|31|1907|5|8|16|3|1907|31|398|Friday|1907Q3|N|Y|N|2417789|2418000|2417439|2417713|N|N|N|N|N| +2417805|AAAAAAAANIEOECAA|1907-08-17|91|398|31|1907|6|8|17|3|1907|31|398|Saturday|1907Q3|N|Y|N|2417789|2418000|2417440|2417714|N|N|N|N|N| +2417806|AAAAAAAAOIEOECAA|1907-08-18|91|398|31|1907|0|8|18|3|1907|31|398|Sunday|1907Q3|N|N|N|2417789|2418000|2417441|2417715|N|N|N|N|N| +2417807|AAAAAAAAPIEOECAA|1907-08-19|91|398|31|1907|1|8|19|3|1907|31|398|Monday|1907Q3|N|N|N|2417789|2418000|2417442|2417716|N|N|N|N|N| +2417808|AAAAAAAAAJEOECAA|1907-08-20|91|399|31|1907|2|8|20|3|1907|31|399|Tuesday|1907Q3|N|N|N|2417789|2418000|2417443|2417717|N|N|N|N|N| +2417809|AAAAAAAABJEOECAA|1907-08-21|91|399|31|1907|3|8|21|3|1907|31|399|Wednesday|1907Q3|N|N|N|2417789|2418000|2417444|2417718|N|N|N|N|N| +2417810|AAAAAAAACJEOECAA|1907-08-22|91|399|31|1907|4|8|22|3|1907|31|399|Thursday|1907Q3|N|N|N|2417789|2418000|2417445|2417719|N|N|N|N|N| +2417811|AAAAAAAADJEOECAA|1907-08-23|91|399|31|1907|5|8|23|3|1907|31|399|Friday|1907Q3|N|Y|N|2417789|2418000|2417446|2417720|N|N|N|N|N| +2417812|AAAAAAAAEJEOECAA|1907-08-24|91|399|31|1907|6|8|24|3|1907|31|399|Saturday|1907Q3|N|Y|N|2417789|2418000|2417447|2417721|N|N|N|N|N| +2417813|AAAAAAAAFJEOECAA|1907-08-25|91|399|31|1907|0|8|25|3|1907|31|399|Sunday|1907Q3|N|N|N|2417789|2418000|2417448|2417722|N|N|N|N|N| +2417814|AAAAAAAAGJEOECAA|1907-08-26|91|399|31|1907|1|8|26|3|1907|31|399|Monday|1907Q3|N|N|N|2417789|2418000|2417449|2417723|N|N|N|N|N| +2417815|AAAAAAAAHJEOECAA|1907-08-27|91|400|31|1907|2|8|27|3|1907|31|400|Tuesday|1907Q3|N|N|N|2417789|2418000|2417450|2417724|N|N|N|N|N| +2417816|AAAAAAAAIJEOECAA|1907-08-28|91|400|31|1907|3|8|28|3|1907|31|400|Wednesday|1907Q3|N|N|N|2417789|2418000|2417451|2417725|N|N|N|N|N| +2417817|AAAAAAAAJJEOECAA|1907-08-29|91|400|31|1907|4|8|29|3|1907|31|400|Thursday|1907Q3|N|N|N|2417789|2418000|2417452|2417726|N|N|N|N|N| +2417818|AAAAAAAAKJEOECAA|1907-08-30|91|400|31|1907|5|8|30|3|1907|31|400|Friday|1907Q3|N|Y|N|2417789|2418000|2417453|2417727|N|N|N|N|N| +2417819|AAAAAAAALJEOECAA|1907-08-31|91|400|31|1907|6|8|31|3|1907|31|400|Saturday|1907Q3|N|Y|N|2417789|2418000|2417454|2417728|N|N|N|N|N| +2417820|AAAAAAAAMJEOECAA|1907-09-01|92|400|32|1907|0|9|1|3|1907|32|400|Sunday|1907Q3|N|N|N|2417820|2418062|2417455|2417729|N|N|N|N|N| +2417821|AAAAAAAANJEOECAA|1907-09-02|92|400|32|1907|1|9|2|3|1907|32|400|Monday|1907Q3|N|N|N|2417820|2418062|2417456|2417730|N|N|N|N|N| +2417822|AAAAAAAAOJEOECAA|1907-09-03|92|401|32|1907|2|9|3|3|1907|32|401|Tuesday|1907Q3|N|N|N|2417820|2418062|2417457|2417731|N|N|N|N|N| +2417823|AAAAAAAAPJEOECAA|1907-09-04|92|401|32|1907|3|9|4|3|1907|32|401|Wednesday|1907Q3|N|N|N|2417820|2418062|2417458|2417732|N|N|N|N|N| +2417824|AAAAAAAAAKEOECAA|1907-09-05|92|401|32|1907|4|9|5|3|1907|32|401|Thursday|1907Q3|N|N|N|2417820|2418062|2417459|2417733|N|N|N|N|N| +2417825|AAAAAAAABKEOECAA|1907-09-06|92|401|32|1907|5|9|6|3|1907|32|401|Friday|1907Q3|N|Y|N|2417820|2418062|2417460|2417734|N|N|N|N|N| +2417826|AAAAAAAACKEOECAA|1907-09-07|92|401|32|1907|6|9|7|3|1907|32|401|Saturday|1907Q3|N|Y|N|2417820|2418062|2417461|2417735|N|N|N|N|N| +2417827|AAAAAAAADKEOECAA|1907-09-08|92|401|32|1907|0|9|8|3|1907|32|401|Sunday|1907Q3|N|N|N|2417820|2418062|2417462|2417736|N|N|N|N|N| +2417828|AAAAAAAAEKEOECAA|1907-09-09|92|401|32|1907|1|9|9|3|1907|32|401|Monday|1907Q3|N|N|N|2417820|2418062|2417463|2417737|N|N|N|N|N| +2417829|AAAAAAAAFKEOECAA|1907-09-10|92|402|32|1907|2|9|10|3|1907|32|402|Tuesday|1907Q3|N|N|N|2417820|2418062|2417464|2417738|N|N|N|N|N| +2417830|AAAAAAAAGKEOECAA|1907-09-11|92|402|32|1907|3|9|11|3|1907|32|402|Wednesday|1907Q3|N|N|N|2417820|2418062|2417465|2417739|N|N|N|N|N| +2417831|AAAAAAAAHKEOECAA|1907-09-12|92|402|32|1907|4|9|12|3|1907|32|402|Thursday|1907Q3|N|N|N|2417820|2418062|2417466|2417740|N|N|N|N|N| +2417832|AAAAAAAAIKEOECAA|1907-09-13|92|402|32|1907|5|9|13|3|1907|32|402|Friday|1907Q3|N|Y|N|2417820|2418062|2417467|2417741|N|N|N|N|N| +2417833|AAAAAAAAJKEOECAA|1907-09-14|92|402|32|1907|6|9|14|3|1907|32|402|Saturday|1907Q3|N|Y|N|2417820|2418062|2417468|2417742|N|N|N|N|N| +2417834|AAAAAAAAKKEOECAA|1907-09-15|92|402|32|1907|0|9|15|3|1907|32|402|Sunday|1907Q3|N|N|N|2417820|2418062|2417469|2417743|N|N|N|N|N| +2417835|AAAAAAAALKEOECAA|1907-09-16|92|402|32|1907|1|9|16|3|1907|32|402|Monday|1907Q3|N|N|N|2417820|2418062|2417470|2417744|N|N|N|N|N| +2417836|AAAAAAAAMKEOECAA|1907-09-17|92|403|32|1907|2|9|17|3|1907|32|403|Tuesday|1907Q3|N|N|N|2417820|2418062|2417471|2417745|N|N|N|N|N| +2417837|AAAAAAAANKEOECAA|1907-09-18|92|403|32|1907|3|9|18|3|1907|32|403|Wednesday|1907Q3|N|N|N|2417820|2418062|2417472|2417746|N|N|N|N|N| +2417838|AAAAAAAAOKEOECAA|1907-09-19|92|403|32|1907|4|9|19|3|1907|32|403|Thursday|1907Q3|N|N|N|2417820|2418062|2417473|2417747|N|N|N|N|N| +2417839|AAAAAAAAPKEOECAA|1907-09-20|92|403|32|1907|5|9|20|3|1907|32|403|Friday|1907Q3|N|Y|N|2417820|2418062|2417474|2417748|N|N|N|N|N| +2417840|AAAAAAAAALEOECAA|1907-09-21|92|403|32|1907|6|9|21|3|1907|32|403|Saturday|1907Q3|N|Y|N|2417820|2418062|2417475|2417749|N|N|N|N|N| +2417841|AAAAAAAABLEOECAA|1907-09-22|92|403|32|1907|0|9|22|3|1907|32|403|Sunday|1907Q3|N|N|N|2417820|2418062|2417476|2417750|N|N|N|N|N| +2417842|AAAAAAAACLEOECAA|1907-09-23|92|403|32|1907|1|9|23|3|1907|32|403|Monday|1907Q3|N|N|N|2417820|2418062|2417477|2417751|N|N|N|N|N| +2417843|AAAAAAAADLEOECAA|1907-09-24|92|404|32|1907|2|9|24|3|1907|32|404|Tuesday|1907Q3|N|N|N|2417820|2418062|2417478|2417752|N|N|N|N|N| +2417844|AAAAAAAAELEOECAA|1907-09-25|92|404|32|1907|3|9|25|3|1907|32|404|Wednesday|1907Q3|N|N|N|2417820|2418062|2417479|2417753|N|N|N|N|N| +2417845|AAAAAAAAFLEOECAA|1907-09-26|92|404|32|1907|4|9|26|3|1907|32|404|Thursday|1907Q3|N|N|N|2417820|2418062|2417480|2417754|N|N|N|N|N| +2417846|AAAAAAAAGLEOECAA|1907-09-27|92|404|32|1907|5|9|27|3|1907|32|404|Friday|1907Q3|N|Y|N|2417820|2418062|2417481|2417755|N|N|N|N|N| +2417847|AAAAAAAAHLEOECAA|1907-09-28|92|404|32|1907|6|9|28|3|1907|32|404|Saturday|1907Q3|N|Y|N|2417820|2418062|2417482|2417756|N|N|N|N|N| +2417848|AAAAAAAAILEOECAA|1907-09-29|92|404|32|1907|0|9|29|3|1907|32|404|Sunday|1907Q3|N|N|N|2417820|2418062|2417483|2417757|N|N|N|N|N| +2417849|AAAAAAAAJLEOECAA|1907-09-30|92|404|32|1907|1|9|30|3|1907|32|404|Monday|1907Q3|N|N|N|2417820|2418062|2417484|2417758|N|N|N|N|N| +2417850|AAAAAAAAKLEOECAA|1907-10-01|93|405|32|1907|2|10|1|3|1907|32|405|Tuesday|1907Q3|N|N|N|2417850|2418122|2417485|2417758|N|N|N|N|N| +2417851|AAAAAAAALLEOECAA|1907-10-02|93|405|32|1907|3|10|2|4|1907|32|405|Wednesday|1907Q4|N|N|N|2417850|2418122|2417486|2417759|N|N|N|N|N| +2417852|AAAAAAAAMLEOECAA|1907-10-03|93|405|32|1907|4|10|3|4|1907|32|405|Thursday|1907Q4|N|N|N|2417850|2418122|2417487|2417760|N|N|N|N|N| +2417853|AAAAAAAANLEOECAA|1907-10-04|93|405|32|1907|5|10|4|4|1907|32|405|Friday|1907Q4|N|Y|N|2417850|2418122|2417488|2417761|N|N|N|N|N| +2417854|AAAAAAAAOLEOECAA|1907-10-05|93|405|32|1907|6|10|5|4|1907|32|405|Saturday|1907Q4|N|Y|N|2417850|2418122|2417489|2417762|N|N|N|N|N| +2417855|AAAAAAAAPLEOECAA|1907-10-06|93|405|32|1907|0|10|6|4|1907|32|405|Sunday|1907Q4|N|N|N|2417850|2418122|2417490|2417763|N|N|N|N|N| +2417856|AAAAAAAAAMEOECAA|1907-10-07|93|405|32|1907|1|10|7|4|1907|32|405|Monday|1907Q4|N|N|N|2417850|2418122|2417491|2417764|N|N|N|N|N| +2417857|AAAAAAAABMEOECAA|1907-10-08|93|406|32|1907|2|10|8|4|1907|32|406|Tuesday|1907Q4|N|N|N|2417850|2418122|2417492|2417765|N|N|N|N|N| +2417858|AAAAAAAACMEOECAA|1907-10-09|93|406|32|1907|3|10|9|4|1907|32|406|Wednesday|1907Q4|N|N|N|2417850|2418122|2417493|2417766|N|N|N|N|N| +2417859|AAAAAAAADMEOECAA|1907-10-10|93|406|32|1907|4|10|10|4|1907|32|406|Thursday|1907Q4|N|N|N|2417850|2418122|2417494|2417767|N|N|N|N|N| +2417860|AAAAAAAAEMEOECAA|1907-10-11|93|406|32|1907|5|10|11|4|1907|32|406|Friday|1907Q4|N|Y|N|2417850|2418122|2417495|2417768|N|N|N|N|N| +2417861|AAAAAAAAFMEOECAA|1907-10-12|93|406|32|1907|6|10|12|4|1907|32|406|Saturday|1907Q4|N|Y|N|2417850|2418122|2417496|2417769|N|N|N|N|N| +2417862|AAAAAAAAGMEOECAA|1907-10-13|93|406|32|1907|0|10|13|4|1907|32|406|Sunday|1907Q4|N|N|N|2417850|2418122|2417497|2417770|N|N|N|N|N| +2417863|AAAAAAAAHMEOECAA|1907-10-14|93|406|32|1907|1|10|14|4|1907|32|406|Monday|1907Q4|N|N|N|2417850|2418122|2417498|2417771|N|N|N|N|N| +2417864|AAAAAAAAIMEOECAA|1907-10-15|93|407|32|1907|2|10|15|4|1907|32|407|Tuesday|1907Q4|N|N|N|2417850|2418122|2417499|2417772|N|N|N|N|N| +2417865|AAAAAAAAJMEOECAA|1907-10-16|93|407|32|1907|3|10|16|4|1907|32|407|Wednesday|1907Q4|N|N|N|2417850|2418122|2417500|2417773|N|N|N|N|N| +2417866|AAAAAAAAKMEOECAA|1907-10-17|93|407|32|1907|4|10|17|4|1907|32|407|Thursday|1907Q4|N|N|N|2417850|2418122|2417501|2417774|N|N|N|N|N| +2417867|AAAAAAAALMEOECAA|1907-10-18|93|407|32|1907|5|10|18|4|1907|32|407|Friday|1907Q4|N|Y|N|2417850|2418122|2417502|2417775|N|N|N|N|N| +2417868|AAAAAAAAMMEOECAA|1907-10-19|93|407|32|1907|6|10|19|4|1907|32|407|Saturday|1907Q4|N|Y|N|2417850|2418122|2417503|2417776|N|N|N|N|N| +2417869|AAAAAAAANMEOECAA|1907-10-20|93|407|32|1907|0|10|20|4|1907|32|407|Sunday|1907Q4|N|N|N|2417850|2418122|2417504|2417777|N|N|N|N|N| +2417870|AAAAAAAAOMEOECAA|1907-10-21|93|407|32|1907|1|10|21|4|1907|32|407|Monday|1907Q4|N|N|N|2417850|2418122|2417505|2417778|N|N|N|N|N| +2417871|AAAAAAAAPMEOECAA|1907-10-22|93|408|32|1907|2|10|22|4|1907|32|408|Tuesday|1907Q4|N|N|N|2417850|2418122|2417506|2417779|N|N|N|N|N| +2417872|AAAAAAAAANEOECAA|1907-10-23|93|408|32|1907|3|10|23|4|1907|32|408|Wednesday|1907Q4|N|N|N|2417850|2418122|2417507|2417780|N|N|N|N|N| +2417873|AAAAAAAABNEOECAA|1907-10-24|93|408|32|1907|4|10|24|4|1907|32|408|Thursday|1907Q4|N|N|N|2417850|2418122|2417508|2417781|N|N|N|N|N| +2417874|AAAAAAAACNEOECAA|1907-10-25|93|408|32|1907|5|10|25|4|1907|32|408|Friday|1907Q4|N|Y|N|2417850|2418122|2417509|2417782|N|N|N|N|N| +2417875|AAAAAAAADNEOECAA|1907-10-26|93|408|32|1907|6|10|26|4|1907|32|408|Saturday|1907Q4|N|Y|N|2417850|2418122|2417510|2417783|N|N|N|N|N| +2417876|AAAAAAAAENEOECAA|1907-10-27|93|408|32|1907|0|10|27|4|1907|32|408|Sunday|1907Q4|N|N|N|2417850|2418122|2417511|2417784|N|N|N|N|N| +2417877|AAAAAAAAFNEOECAA|1907-10-28|93|408|32|1907|1|10|28|4|1907|32|408|Monday|1907Q4|N|N|N|2417850|2418122|2417512|2417785|N|N|N|N|N| +2417878|AAAAAAAAGNEOECAA|1907-10-29|93|409|32|1907|2|10|29|4|1907|32|409|Tuesday|1907Q4|N|N|N|2417850|2418122|2417513|2417786|N|N|N|N|N| +2417879|AAAAAAAAHNEOECAA|1907-10-30|93|409|32|1907|3|10|30|4|1907|32|409|Wednesday|1907Q4|N|N|N|2417850|2418122|2417514|2417787|N|N|N|N|N| +2417880|AAAAAAAAINEOECAA|1907-10-31|93|409|32|1907|4|10|31|4|1907|32|409|Thursday|1907Q4|N|N|N|2417850|2418122|2417515|2417788|N|N|N|N|N| +2417881|AAAAAAAAJNEOECAA|1907-11-01|94|409|32|1907|5|11|1|4|1907|32|409|Friday|1907Q4|N|Y|N|2417881|2418184|2417516|2417789|N|N|N|N|N| +2417882|AAAAAAAAKNEOECAA|1907-11-02|94|409|32|1907|6|11|2|4|1907|32|409|Saturday|1907Q4|N|Y|N|2417881|2418184|2417517|2417790|N|N|N|N|N| +2417883|AAAAAAAALNEOECAA|1907-11-03|94|409|32|1907|0|11|3|4|1907|32|409|Sunday|1907Q4|N|N|N|2417881|2418184|2417518|2417791|N|N|N|N|N| +2417884|AAAAAAAAMNEOECAA|1907-11-04|94|409|32|1907|1|11|4|4|1907|32|409|Monday|1907Q4|N|N|N|2417881|2418184|2417519|2417792|N|N|N|N|N| +2417885|AAAAAAAANNEOECAA|1907-11-05|94|410|32|1907|2|11|5|4|1907|32|410|Tuesday|1907Q4|N|N|N|2417881|2418184|2417520|2417793|N|N|N|N|N| +2417886|AAAAAAAAONEOECAA|1907-11-06|94|410|32|1907|3|11|6|4|1907|32|410|Wednesday|1907Q4|N|N|N|2417881|2418184|2417521|2417794|N|N|N|N|N| +2417887|AAAAAAAAPNEOECAA|1907-11-07|94|410|32|1907|4|11|7|4|1907|32|410|Thursday|1907Q4|N|N|N|2417881|2418184|2417522|2417795|N|N|N|N|N| +2417888|AAAAAAAAAOEOECAA|1907-11-08|94|410|32|1907|5|11|8|4|1907|32|410|Friday|1907Q4|N|Y|N|2417881|2418184|2417523|2417796|N|N|N|N|N| +2417889|AAAAAAAABOEOECAA|1907-11-09|94|410|32|1907|6|11|9|4|1907|32|410|Saturday|1907Q4|N|Y|N|2417881|2418184|2417524|2417797|N|N|N|N|N| +2417890|AAAAAAAACOEOECAA|1907-11-10|94|410|32|1907|0|11|10|4|1907|32|410|Sunday|1907Q4|N|N|N|2417881|2418184|2417525|2417798|N|N|N|N|N| +2417891|AAAAAAAADOEOECAA|1907-11-11|94|410|32|1907|1|11|11|4|1907|32|410|Monday|1907Q4|N|N|N|2417881|2418184|2417526|2417799|N|N|N|N|N| +2417892|AAAAAAAAEOEOECAA|1907-11-12|94|411|32|1907|2|11|12|4|1907|32|411|Tuesday|1907Q4|N|N|N|2417881|2418184|2417527|2417800|N|N|N|N|N| +2417893|AAAAAAAAFOEOECAA|1907-11-13|94|411|32|1907|3|11|13|4|1907|32|411|Wednesday|1907Q4|N|N|N|2417881|2418184|2417528|2417801|N|N|N|N|N| +2417894|AAAAAAAAGOEOECAA|1907-11-14|94|411|32|1907|4|11|14|4|1907|32|411|Thursday|1907Q4|N|N|N|2417881|2418184|2417529|2417802|N|N|N|N|N| +2417895|AAAAAAAAHOEOECAA|1907-11-15|94|411|32|1907|5|11|15|4|1907|32|411|Friday|1907Q4|N|Y|N|2417881|2418184|2417530|2417803|N|N|N|N|N| +2417896|AAAAAAAAIOEOECAA|1907-11-16|94|411|32|1907|6|11|16|4|1907|32|411|Saturday|1907Q4|N|Y|N|2417881|2418184|2417531|2417804|N|N|N|N|N| +2417897|AAAAAAAAJOEOECAA|1907-11-17|94|411|32|1907|0|11|17|4|1907|32|411|Sunday|1907Q4|N|N|N|2417881|2418184|2417532|2417805|N|N|N|N|N| +2417898|AAAAAAAAKOEOECAA|1907-11-18|94|411|32|1907|1|11|18|4|1907|32|411|Monday|1907Q4|N|N|N|2417881|2418184|2417533|2417806|N|N|N|N|N| +2417899|AAAAAAAALOEOECAA|1907-11-19|94|412|32|1907|2|11|19|4|1907|32|412|Tuesday|1907Q4|N|N|N|2417881|2418184|2417534|2417807|N|N|N|N|N| +2417900|AAAAAAAAMOEOECAA|1907-11-20|94|412|32|1907|3|11|20|4|1907|32|412|Wednesday|1907Q4|N|N|N|2417881|2418184|2417535|2417808|N|N|N|N|N| +2417901|AAAAAAAANOEOECAA|1907-11-21|94|412|32|1907|4|11|21|4|1907|32|412|Thursday|1907Q4|N|N|N|2417881|2418184|2417536|2417809|N|N|N|N|N| +2417902|AAAAAAAAOOEOECAA|1907-11-22|94|412|32|1907|5|11|22|4|1907|32|412|Friday|1907Q4|N|Y|N|2417881|2418184|2417537|2417810|N|N|N|N|N| +2417903|AAAAAAAAPOEOECAA|1907-11-23|94|412|32|1907|6|11|23|4|1907|32|412|Saturday|1907Q4|N|Y|N|2417881|2418184|2417538|2417811|N|N|N|N|N| +2417904|AAAAAAAAAPEOECAA|1907-11-24|94|412|32|1907|0|11|24|4|1907|32|412|Sunday|1907Q4|N|N|N|2417881|2418184|2417539|2417812|N|N|N|N|N| +2417905|AAAAAAAABPEOECAA|1907-11-25|94|412|32|1907|1|11|25|4|1907|32|412|Monday|1907Q4|N|N|N|2417881|2418184|2417540|2417813|N|N|N|N|N| +2417906|AAAAAAAACPEOECAA|1907-11-26|94|413|32|1907|2|11|26|4|1907|32|413|Tuesday|1907Q4|N|N|N|2417881|2418184|2417541|2417814|N|N|N|N|N| +2417907|AAAAAAAADPEOECAA|1907-11-27|94|413|32|1907|3|11|27|4|1907|32|413|Wednesday|1907Q4|N|N|N|2417881|2418184|2417542|2417815|N|N|N|N|N| +2417908|AAAAAAAAEPEOECAA|1907-11-28|94|413|32|1907|4|11|28|4|1907|32|413|Thursday|1907Q4|N|N|N|2417881|2418184|2417543|2417816|N|N|N|N|N| +2417909|AAAAAAAAFPEOECAA|1907-11-29|94|413|32|1907|5|11|29|4|1907|32|413|Friday|1907Q4|N|Y|N|2417881|2418184|2417544|2417817|N|N|N|N|N| +2417910|AAAAAAAAGPEOECAA|1907-11-30|94|413|32|1907|6|11|30|4|1907|32|413|Saturday|1907Q4|N|Y|N|2417881|2418184|2417545|2417818|N|N|N|N|N| +2417911|AAAAAAAAHPEOECAA|1907-12-01|95|413|33|1907|0|12|1|4|1907|33|413|Sunday|1907Q4|N|N|N|2417911|2418244|2417546|2417819|N|N|N|N|N| +2417912|AAAAAAAAIPEOECAA|1907-12-02|95|413|33|1907|1|12|2|4|1907|33|413|Monday|1907Q4|N|N|N|2417911|2418244|2417547|2417820|N|N|N|N|N| +2417913|AAAAAAAAJPEOECAA|1907-12-03|95|414|33|1907|2|12|3|4|1907|33|414|Tuesday|1907Q4|N|N|N|2417911|2418244|2417548|2417821|N|N|N|N|N| +2417914|AAAAAAAAKPEOECAA|1907-12-04|95|414|33|1907|3|12|4|4|1907|33|414|Wednesday|1907Q4|N|N|N|2417911|2418244|2417549|2417822|N|N|N|N|N| +2417915|AAAAAAAALPEOECAA|1907-12-05|95|414|33|1907|4|12|5|4|1907|33|414|Thursday|1907Q4|N|N|N|2417911|2418244|2417550|2417823|N|N|N|N|N| +2417916|AAAAAAAAMPEOECAA|1907-12-06|95|414|33|1907|5|12|6|4|1907|33|414|Friday|1907Q4|N|Y|N|2417911|2418244|2417551|2417824|N|N|N|N|N| +2417917|AAAAAAAANPEOECAA|1907-12-07|95|414|33|1907|6|12|7|4|1907|33|414|Saturday|1907Q4|N|Y|N|2417911|2418244|2417552|2417825|N|N|N|N|N| +2417918|AAAAAAAAOPEOECAA|1907-12-08|95|414|33|1907|0|12|8|4|1907|33|414|Sunday|1907Q4|N|N|N|2417911|2418244|2417553|2417826|N|N|N|N|N| +2417919|AAAAAAAAPPEOECAA|1907-12-09|95|414|33|1907|1|12|9|4|1907|33|414|Monday|1907Q4|N|N|N|2417911|2418244|2417554|2417827|N|N|N|N|N| +2417920|AAAAAAAAAAFOECAA|1907-12-10|95|415|33|1907|2|12|10|4|1907|33|415|Tuesday|1907Q4|N|N|N|2417911|2418244|2417555|2417828|N|N|N|N|N| +2417921|AAAAAAAABAFOECAA|1907-12-11|95|415|33|1907|3|12|11|4|1907|33|415|Wednesday|1907Q4|N|N|N|2417911|2418244|2417556|2417829|N|N|N|N|N| +2417922|AAAAAAAACAFOECAA|1907-12-12|95|415|33|1907|4|12|12|4|1907|33|415|Thursday|1907Q4|N|N|N|2417911|2418244|2417557|2417830|N|N|N|N|N| +2417923|AAAAAAAADAFOECAA|1907-12-13|95|415|33|1907|5|12|13|4|1907|33|415|Friday|1907Q4|N|Y|N|2417911|2418244|2417558|2417831|N|N|N|N|N| +2417924|AAAAAAAAEAFOECAA|1907-12-14|95|415|33|1907|6|12|14|4|1907|33|415|Saturday|1907Q4|N|Y|N|2417911|2418244|2417559|2417832|N|N|N|N|N| +2417925|AAAAAAAAFAFOECAA|1907-12-15|95|415|33|1907|0|12|15|4|1907|33|415|Sunday|1907Q4|N|N|N|2417911|2418244|2417560|2417833|N|N|N|N|N| +2417926|AAAAAAAAGAFOECAA|1907-12-16|95|415|33|1907|1|12|16|4|1907|33|415|Monday|1907Q4|N|N|N|2417911|2418244|2417561|2417834|N|N|N|N|N| +2417927|AAAAAAAAHAFOECAA|1907-12-17|95|416|33|1907|2|12|17|4|1907|33|416|Tuesday|1907Q4|N|N|N|2417911|2418244|2417562|2417835|N|N|N|N|N| +2417928|AAAAAAAAIAFOECAA|1907-12-18|95|416|33|1907|3|12|18|4|1907|33|416|Wednesday|1907Q4|N|N|N|2417911|2418244|2417563|2417836|N|N|N|N|N| +2417929|AAAAAAAAJAFOECAA|1907-12-19|95|416|33|1907|4|12|19|4|1907|33|416|Thursday|1907Q4|N|N|N|2417911|2418244|2417564|2417837|N|N|N|N|N| +2417930|AAAAAAAAKAFOECAA|1907-12-20|95|416|33|1907|5|12|20|4|1907|33|416|Friday|1907Q4|N|Y|N|2417911|2418244|2417565|2417838|N|N|N|N|N| +2417931|AAAAAAAALAFOECAA|1907-12-21|95|416|33|1907|6|12|21|4|1907|33|416|Saturday|1907Q4|N|Y|N|2417911|2418244|2417566|2417839|N|N|N|N|N| +2417932|AAAAAAAAMAFOECAA|1907-12-22|95|416|33|1907|0|12|22|4|1907|33|416|Sunday|1907Q4|N|N|N|2417911|2418244|2417567|2417840|N|N|N|N|N| +2417933|AAAAAAAANAFOECAA|1907-12-23|95|416|33|1907|1|12|23|4|1907|33|416|Monday|1907Q4|N|N|N|2417911|2418244|2417568|2417841|N|N|N|N|N| +2417934|AAAAAAAAOAFOECAA|1907-12-24|95|417|33|1907|2|12|24|4|1907|33|417|Tuesday|1907Q4|N|N|N|2417911|2418244|2417569|2417842|N|N|N|N|N| +2417935|AAAAAAAAPAFOECAA|1907-12-25|95|417|33|1907|3|12|25|4|1907|33|417|Wednesday|1907Q4|N|N|N|2417911|2418244|2417570|2417843|N|N|N|N|N| +2417936|AAAAAAAAABFOECAA|1907-12-26|95|417|33|1907|4|12|26|4|1907|33|417|Thursday|1907Q4|Y|N|N|2417911|2418244|2417571|2417844|N|N|N|N|N| +2417937|AAAAAAAABBFOECAA|1907-12-27|95|417|33|1907|5|12|27|4|1907|33|417|Friday|1907Q4|N|Y|Y|2417911|2418244|2417572|2417845|N|N|N|N|N| +2417938|AAAAAAAACBFOECAA|1907-12-28|95|417|33|1907|6|12|28|4|1907|33|417|Saturday|1907Q4|N|Y|N|2417911|2418244|2417573|2417846|N|N|N|N|N| +2417939|AAAAAAAADBFOECAA|1907-12-29|95|417|33|1907|0|12|29|4|1907|33|417|Sunday|1907Q4|N|N|N|2417911|2418244|2417574|2417847|N|N|N|N|N| +2417940|AAAAAAAAEBFOECAA|1907-12-30|95|417|33|1907|1|12|30|4|1907|33|417|Monday|1907Q4|N|N|N|2417911|2418244|2417575|2417848|N|N|N|N|N| +2417941|AAAAAAAAFBFOECAA|1907-12-31|95|418|33|1907|2|12|31|4|1907|33|418|Tuesday|1907Q4|N|N|N|2417911|2418244|2417576|2417849|N|N|N|N|N| +2417942|AAAAAAAAGBFOECAA|1908-01-01|96|418|33|1908|3|1|1|1|1908|33|418|Wednesday|1908Q1|Y|N|N|2417942|2417941|2417577|2417850|N|N|N|N|N| +2417943|AAAAAAAAHBFOECAA|1908-01-02|96|418|33|1908|4|1|2|1|1908|33|418|Thursday|1908Q1|N|N|Y|2417942|2417941|2417578|2417851|N|N|N|N|N| +2417944|AAAAAAAAIBFOECAA|1908-01-03|96|418|33|1908|5|1|3|1|1908|33|418|Friday|1908Q1|N|Y|N|2417942|2417941|2417579|2417852|N|N|N|N|N| +2417945|AAAAAAAAJBFOECAA|1908-01-04|96|418|33|1908|6|1|4|1|1908|33|418|Saturday|1908Q1|N|Y|N|2417942|2417941|2417580|2417853|N|N|N|N|N| +2417946|AAAAAAAAKBFOECAA|1908-01-05|96|418|33|1908|0|1|5|1|1908|33|418|Sunday|1908Q1|N|N|N|2417942|2417941|2417581|2417854|N|N|N|N|N| +2417947|AAAAAAAALBFOECAA|1908-01-06|96|418|33|1908|1|1|6|1|1908|33|418|Monday|1908Q1|N|N|N|2417942|2417941|2417582|2417855|N|N|N|N|N| +2417948|AAAAAAAAMBFOECAA|1908-01-07|96|419|33|1908|2|1|7|1|1908|33|419|Tuesday|1908Q1|N|N|N|2417942|2417941|2417583|2417856|N|N|N|N|N| +2417949|AAAAAAAANBFOECAA|1908-01-08|96|419|33|1908|3|1|8|1|1908|33|419|Wednesday|1908Q1|N|N|N|2417942|2417941|2417584|2417857|N|N|N|N|N| +2417950|AAAAAAAAOBFOECAA|1908-01-09|96|419|33|1908|4|1|9|1|1908|33|419|Thursday|1908Q1|N|N|N|2417942|2417941|2417585|2417858|N|N|N|N|N| +2417951|AAAAAAAAPBFOECAA|1908-01-10|96|419|33|1908|5|1|10|1|1908|33|419|Friday|1908Q1|N|Y|N|2417942|2417941|2417586|2417859|N|N|N|N|N| +2417952|AAAAAAAAACFOECAA|1908-01-11|96|419|33|1908|6|1|11|1|1908|33|419|Saturday|1908Q1|N|Y|N|2417942|2417941|2417587|2417860|N|N|N|N|N| +2417953|AAAAAAAABCFOECAA|1908-01-12|96|419|33|1908|0|1|12|1|1908|33|419|Sunday|1908Q1|N|N|N|2417942|2417941|2417588|2417861|N|N|N|N|N| +2417954|AAAAAAAACCFOECAA|1908-01-13|96|419|33|1908|1|1|13|1|1908|33|419|Monday|1908Q1|N|N|N|2417942|2417941|2417589|2417862|N|N|N|N|N| +2417955|AAAAAAAADCFOECAA|1908-01-14|96|420|33|1908|2|1|14|1|1908|33|420|Tuesday|1908Q1|N|N|N|2417942|2417941|2417590|2417863|N|N|N|N|N| +2417956|AAAAAAAAECFOECAA|1908-01-15|96|420|33|1908|3|1|15|1|1908|33|420|Wednesday|1908Q1|N|N|N|2417942|2417941|2417591|2417864|N|N|N|N|N| +2417957|AAAAAAAAFCFOECAA|1908-01-16|96|420|33|1908|4|1|16|1|1908|33|420|Thursday|1908Q1|N|N|N|2417942|2417941|2417592|2417865|N|N|N|N|N| +2417958|AAAAAAAAGCFOECAA|1908-01-17|96|420|33|1908|5|1|17|1|1908|33|420|Friday|1908Q1|N|Y|N|2417942|2417941|2417593|2417866|N|N|N|N|N| +2417959|AAAAAAAAHCFOECAA|1908-01-18|96|420|33|1908|6|1|18|1|1908|33|420|Saturday|1908Q1|N|Y|N|2417942|2417941|2417594|2417867|N|N|N|N|N| +2417960|AAAAAAAAICFOECAA|1908-01-19|96|420|33|1908|0|1|19|1|1908|33|420|Sunday|1908Q1|N|N|N|2417942|2417941|2417595|2417868|N|N|N|N|N| +2417961|AAAAAAAAJCFOECAA|1908-01-20|96|420|33|1908|1|1|20|1|1908|33|420|Monday|1908Q1|N|N|N|2417942|2417941|2417596|2417869|N|N|N|N|N| +2417962|AAAAAAAAKCFOECAA|1908-01-21|96|421|33|1908|2|1|21|1|1908|33|421|Tuesday|1908Q1|N|N|N|2417942|2417941|2417597|2417870|N|N|N|N|N| +2417963|AAAAAAAALCFOECAA|1908-01-22|96|421|33|1908|3|1|22|1|1908|33|421|Wednesday|1908Q1|N|N|N|2417942|2417941|2417598|2417871|N|N|N|N|N| +2417964|AAAAAAAAMCFOECAA|1908-01-23|96|421|33|1908|4|1|23|1|1908|33|421|Thursday|1908Q1|N|N|N|2417942|2417941|2417599|2417872|N|N|N|N|N| +2417965|AAAAAAAANCFOECAA|1908-01-24|96|421|33|1908|5|1|24|1|1908|33|421|Friday|1908Q1|N|Y|N|2417942|2417941|2417600|2417873|N|N|N|N|N| +2417966|AAAAAAAAOCFOECAA|1908-01-25|96|421|33|1908|6|1|25|1|1908|33|421|Saturday|1908Q1|N|Y|N|2417942|2417941|2417601|2417874|N|N|N|N|N| +2417967|AAAAAAAAPCFOECAA|1908-01-26|96|421|33|1908|0|1|26|1|1908|33|421|Sunday|1908Q1|N|N|N|2417942|2417941|2417602|2417875|N|N|N|N|N| +2417968|AAAAAAAAADFOECAA|1908-01-27|96|421|33|1908|1|1|27|1|1908|33|421|Monday|1908Q1|N|N|N|2417942|2417941|2417603|2417876|N|N|N|N|N| +2417969|AAAAAAAABDFOECAA|1908-01-28|96|422|33|1908|2|1|28|1|1908|33|422|Tuesday|1908Q1|N|N|N|2417942|2417941|2417604|2417877|N|N|N|N|N| +2417970|AAAAAAAACDFOECAA|1908-01-29|96|422|33|1908|3|1|29|1|1908|33|422|Wednesday|1908Q1|N|N|N|2417942|2417941|2417605|2417878|N|N|N|N|N| +2417971|AAAAAAAADDFOECAA|1908-01-30|96|422|33|1908|4|1|30|1|1908|33|422|Thursday|1908Q1|N|N|N|2417942|2417941|2417606|2417879|N|N|N|N|N| +2417972|AAAAAAAAEDFOECAA|1908-01-31|96|422|33|1908|5|1|31|1|1908|33|422|Friday|1908Q1|N|Y|N|2417942|2417941|2417607|2417880|N|N|N|N|N| +2417973|AAAAAAAAFDFOECAA|1908-02-01|97|422|33|1908|6|2|1|1|1908|33|422|Saturday|1908Q1|N|Y|N|2417973|2418003|2417608|2417881|N|N|N|N|N| +2417974|AAAAAAAAGDFOECAA|1908-02-02|97|422|33|1908|0|2|2|1|1908|33|422|Sunday|1908Q1|N|N|N|2417973|2418003|2417609|2417882|N|N|N|N|N| +2417975|AAAAAAAAHDFOECAA|1908-02-03|97|422|33|1908|1|2|3|1|1908|33|422|Monday|1908Q1|N|N|N|2417973|2418003|2417610|2417883|N|N|N|N|N| +2417976|AAAAAAAAIDFOECAA|1908-02-04|97|423|33|1908|2|2|4|1|1908|33|423|Tuesday|1908Q1|N|N|N|2417973|2418003|2417611|2417884|N|N|N|N|N| +2417977|AAAAAAAAJDFOECAA|1908-02-05|97|423|33|1908|3|2|5|1|1908|33|423|Wednesday|1908Q1|N|N|N|2417973|2418003|2417612|2417885|N|N|N|N|N| +2417978|AAAAAAAAKDFOECAA|1908-02-06|97|423|33|1908|4|2|6|1|1908|33|423|Thursday|1908Q1|N|N|N|2417973|2418003|2417613|2417886|N|N|N|N|N| +2417979|AAAAAAAALDFOECAA|1908-02-07|97|423|33|1908|5|2|7|1|1908|33|423|Friday|1908Q1|N|Y|N|2417973|2418003|2417614|2417887|N|N|N|N|N| +2417980|AAAAAAAAMDFOECAA|1908-02-08|97|423|33|1908|6|2|8|1|1908|33|423|Saturday|1908Q1|N|Y|N|2417973|2418003|2417615|2417888|N|N|N|N|N| +2417981|AAAAAAAANDFOECAA|1908-02-09|97|423|33|1908|0|2|9|1|1908|33|423|Sunday|1908Q1|N|N|N|2417973|2418003|2417616|2417889|N|N|N|N|N| +2417982|AAAAAAAAODFOECAA|1908-02-10|97|423|33|1908|1|2|10|1|1908|33|423|Monday|1908Q1|N|N|N|2417973|2418003|2417617|2417890|N|N|N|N|N| +2417983|AAAAAAAAPDFOECAA|1908-02-11|97|424|33|1908|2|2|11|1|1908|33|424|Tuesday|1908Q1|N|N|N|2417973|2418003|2417618|2417891|N|N|N|N|N| +2417984|AAAAAAAAAEFOECAA|1908-02-12|97|424|33|1908|3|2|12|1|1908|33|424|Wednesday|1908Q1|N|N|N|2417973|2418003|2417619|2417892|N|N|N|N|N| +2417985|AAAAAAAABEFOECAA|1908-02-13|97|424|33|1908|4|2|13|1|1908|33|424|Thursday|1908Q1|N|N|N|2417973|2418003|2417620|2417893|N|N|N|N|N| +2417986|AAAAAAAACEFOECAA|1908-02-14|97|424|33|1908|5|2|14|1|1908|33|424|Friday|1908Q1|N|Y|N|2417973|2418003|2417621|2417894|N|N|N|N|N| +2417987|AAAAAAAADEFOECAA|1908-02-15|97|424|33|1908|6|2|15|1|1908|33|424|Saturday|1908Q1|N|Y|N|2417973|2418003|2417622|2417895|N|N|N|N|N| +2417988|AAAAAAAAEEFOECAA|1908-02-16|97|424|33|1908|0|2|16|1|1908|33|424|Sunday|1908Q1|N|N|N|2417973|2418003|2417623|2417896|N|N|N|N|N| +2417989|AAAAAAAAFEFOECAA|1908-02-17|97|424|33|1908|1|2|17|1|1908|33|424|Monday|1908Q1|N|N|N|2417973|2418003|2417624|2417897|N|N|N|N|N| +2417990|AAAAAAAAGEFOECAA|1908-02-18|97|425|33|1908|2|2|18|1|1908|33|425|Tuesday|1908Q1|N|N|N|2417973|2418003|2417625|2417898|N|N|N|N|N| +2417991|AAAAAAAAHEFOECAA|1908-02-19|97|425|33|1908|3|2|19|1|1908|33|425|Wednesday|1908Q1|N|N|N|2417973|2418003|2417626|2417899|N|N|N|N|N| +2417992|AAAAAAAAIEFOECAA|1908-02-20|97|425|33|1908|4|2|20|1|1908|33|425|Thursday|1908Q1|N|N|N|2417973|2418003|2417627|2417900|N|N|N|N|N| +2417993|AAAAAAAAJEFOECAA|1908-02-21|97|425|33|1908|5|2|21|1|1908|33|425|Friday|1908Q1|N|Y|N|2417973|2418003|2417628|2417901|N|N|N|N|N| +2417994|AAAAAAAAKEFOECAA|1908-02-22|97|425|33|1908|6|2|22|1|1908|33|425|Saturday|1908Q1|N|Y|N|2417973|2418003|2417629|2417902|N|N|N|N|N| +2417995|AAAAAAAALEFOECAA|1908-02-23|97|425|33|1908|0|2|23|1|1908|33|425|Sunday|1908Q1|N|N|N|2417973|2418003|2417630|2417903|N|N|N|N|N| +2417996|AAAAAAAAMEFOECAA|1908-02-24|97|425|33|1908|1|2|24|1|1908|33|425|Monday|1908Q1|N|N|N|2417973|2418003|2417631|2417904|N|N|N|N|N| +2417997|AAAAAAAANEFOECAA|1908-02-25|97|426|33|1908|2|2|25|1|1908|33|426|Tuesday|1908Q1|N|N|N|2417973|2418003|2417632|2417905|N|N|N|N|N| +2417998|AAAAAAAAOEFOECAA|1908-02-26|97|426|33|1908|3|2|26|1|1908|33|426|Wednesday|1908Q1|N|N|N|2417973|2418003|2417633|2417906|N|N|N|N|N| +2417999|AAAAAAAAPEFOECAA|1908-02-27|97|426|33|1908|4|2|27|1|1908|33|426|Thursday|1908Q1|N|N|N|2417973|2418003|2417634|2417907|N|N|N|N|N| +2418000|AAAAAAAAAFFOECAA|1908-02-28|97|426|33|1908|5|2|28|1|1908|33|426|Friday|1908Q1|N|Y|N|2417973|2418003|2417635|2417908|N|N|N|N|N| +2418001|AAAAAAAABFFOECAA|1908-02-29|97|426|33|1908|6|2|29|1|1908|33|426|Saturday|1908Q1|N|Y|N|2417973|2418003|2417635|2417909|N|N|N|N|N| +2418002|AAAAAAAACFFOECAA|1908-03-01|98|426|34|1908|0|3|1|1|1908|34|426|Sunday|1908Q1|N|N|N|2418002|2418061|2417636|2417910|N|N|N|N|N| +2418003|AAAAAAAADFFOECAA|1908-03-02|98|426|34|1908|1|3|2|1|1908|34|426|Monday|1908Q1|N|N|N|2418002|2418061|2417637|2417911|N|N|N|N|N| +2418004|AAAAAAAAEFFOECAA|1908-03-03|98|427|34|1908|2|3|3|1|1908|34|427|Tuesday|1908Q1|N|N|N|2418002|2418061|2417638|2417912|N|N|N|N|N| +2418005|AAAAAAAAFFFOECAA|1908-03-04|98|427|34|1908|3|3|4|1|1908|34|427|Wednesday|1908Q1|N|N|N|2418002|2418061|2417639|2417913|N|N|N|N|N| +2418006|AAAAAAAAGFFOECAA|1908-03-05|98|427|34|1908|4|3|5|1|1908|34|427|Thursday|1908Q1|N|N|N|2418002|2418061|2417640|2417914|N|N|N|N|N| +2418007|AAAAAAAAHFFOECAA|1908-03-06|98|427|34|1908|5|3|6|1|1908|34|427|Friday|1908Q1|N|Y|N|2418002|2418061|2417641|2417915|N|N|N|N|N| +2418008|AAAAAAAAIFFOECAA|1908-03-07|98|427|34|1908|6|3|7|1|1908|34|427|Saturday|1908Q1|N|Y|N|2418002|2418061|2417642|2417916|N|N|N|N|N| +2418009|AAAAAAAAJFFOECAA|1908-03-08|98|427|34|1908|0|3|8|1|1908|34|427|Sunday|1908Q1|N|N|N|2418002|2418061|2417643|2417917|N|N|N|N|N| +2418010|AAAAAAAAKFFOECAA|1908-03-09|98|427|34|1908|1|3|9|1|1908|34|427|Monday|1908Q1|N|N|N|2418002|2418061|2417644|2417918|N|N|N|N|N| +2418011|AAAAAAAALFFOECAA|1908-03-10|98|428|34|1908|2|3|10|1|1908|34|428|Tuesday|1908Q1|N|N|N|2418002|2418061|2417645|2417919|N|N|N|N|N| +2418012|AAAAAAAAMFFOECAA|1908-03-11|98|428|34|1908|3|3|11|1|1908|34|428|Wednesday|1908Q1|N|N|N|2418002|2418061|2417646|2417920|N|N|N|N|N| +2418013|AAAAAAAANFFOECAA|1908-03-12|98|428|34|1908|4|3|12|1|1908|34|428|Thursday|1908Q1|N|N|N|2418002|2418061|2417647|2417921|N|N|N|N|N| +2418014|AAAAAAAAOFFOECAA|1908-03-13|98|428|34|1908|5|3|13|1|1908|34|428|Friday|1908Q1|N|Y|N|2418002|2418061|2417648|2417922|N|N|N|N|N| +2418015|AAAAAAAAPFFOECAA|1908-03-14|98|428|34|1908|6|3|14|1|1908|34|428|Saturday|1908Q1|N|Y|N|2418002|2418061|2417649|2417923|N|N|N|N|N| +2418016|AAAAAAAAAGFOECAA|1908-03-15|98|428|34|1908|0|3|15|1|1908|34|428|Sunday|1908Q1|N|N|N|2418002|2418061|2417650|2417924|N|N|N|N|N| +2418017|AAAAAAAABGFOECAA|1908-03-16|98|428|34|1908|1|3|16|1|1908|34|428|Monday|1908Q1|N|N|N|2418002|2418061|2417651|2417925|N|N|N|N|N| +2418018|AAAAAAAACGFOECAA|1908-03-17|98|429|34|1908|2|3|17|1|1908|34|429|Tuesday|1908Q1|N|N|N|2418002|2418061|2417652|2417926|N|N|N|N|N| +2418019|AAAAAAAADGFOECAA|1908-03-18|98|429|34|1908|3|3|18|1|1908|34|429|Wednesday|1908Q1|N|N|N|2418002|2418061|2417653|2417927|N|N|N|N|N| +2418020|AAAAAAAAEGFOECAA|1908-03-19|98|429|34|1908|4|3|19|1|1908|34|429|Thursday|1908Q1|N|N|N|2418002|2418061|2417654|2417928|N|N|N|N|N| +2418021|AAAAAAAAFGFOECAA|1908-03-20|98|429|34|1908|5|3|20|1|1908|34|429|Friday|1908Q1|N|Y|N|2418002|2418061|2417655|2417929|N|N|N|N|N| +2418022|AAAAAAAAGGFOECAA|1908-03-21|98|429|34|1908|6|3|21|1|1908|34|429|Saturday|1908Q1|N|Y|N|2418002|2418061|2417656|2417930|N|N|N|N|N| +2418023|AAAAAAAAHGFOECAA|1908-03-22|98|429|34|1908|0|3|22|1|1908|34|429|Sunday|1908Q1|N|N|N|2418002|2418061|2417657|2417931|N|N|N|N|N| +2418024|AAAAAAAAIGFOECAA|1908-03-23|98|429|34|1908|1|3|23|1|1908|34|429|Monday|1908Q1|N|N|N|2418002|2418061|2417658|2417932|N|N|N|N|N| +2418025|AAAAAAAAJGFOECAA|1908-03-24|98|430|34|1908|2|3|24|1|1908|34|430|Tuesday|1908Q1|N|N|N|2418002|2418061|2417659|2417933|N|N|N|N|N| +2418026|AAAAAAAAKGFOECAA|1908-03-25|98|430|34|1908|3|3|25|1|1908|34|430|Wednesday|1908Q1|N|N|N|2418002|2418061|2417660|2417934|N|N|N|N|N| +2418027|AAAAAAAALGFOECAA|1908-03-26|98|430|34|1908|4|3|26|1|1908|34|430|Thursday|1908Q1|N|N|N|2418002|2418061|2417661|2417935|N|N|N|N|N| +2418028|AAAAAAAAMGFOECAA|1908-03-27|98|430|34|1908|5|3|27|1|1908|34|430|Friday|1908Q1|N|Y|N|2418002|2418061|2417662|2417936|N|N|N|N|N| +2418029|AAAAAAAANGFOECAA|1908-03-28|98|430|34|1908|6|3|28|1|1908|34|430|Saturday|1908Q1|N|Y|N|2418002|2418061|2417663|2417937|N|N|N|N|N| +2418030|AAAAAAAAOGFOECAA|1908-03-29|98|430|34|1908|0|3|29|1|1908|34|430|Sunday|1908Q1|N|N|N|2418002|2418061|2417664|2417938|N|N|N|N|N| +2418031|AAAAAAAAPGFOECAA|1908-03-30|98|430|34|1908|1|3|30|1|1908|34|430|Monday|1908Q1|N|N|N|2418002|2418061|2417665|2417939|N|N|N|N|N| +2418032|AAAAAAAAAHFOECAA|1908-03-31|98|431|34|1908|2|3|31|1|1908|34|431|Tuesday|1908Q1|N|N|N|2418002|2418061|2417666|2417940|N|N|N|N|N| +2418033|AAAAAAAABHFOECAA|1908-04-01|99|431|34|1908|3|4|1|2|1908|34|431|Wednesday|1908Q2|N|N|N|2418033|2418123|2417667|2417942|N|N|N|N|N| +2418034|AAAAAAAACHFOECAA|1908-04-02|99|431|34|1908|4|4|2|2|1908|34|431|Thursday|1908Q2|N|N|N|2418033|2418123|2417668|2417943|N|N|N|N|N| +2418035|AAAAAAAADHFOECAA|1908-04-03|99|431|34|1908|5|4|3|2|1908|34|431|Friday|1908Q2|N|Y|N|2418033|2418123|2417669|2417944|N|N|N|N|N| +2418036|AAAAAAAAEHFOECAA|1908-04-04|99|431|34|1908|6|4|4|2|1908|34|431|Saturday|1908Q2|N|Y|N|2418033|2418123|2417670|2417945|N|N|N|N|N| +2418037|AAAAAAAAFHFOECAA|1908-04-05|99|431|34|1908|0|4|5|2|1908|34|431|Sunday|1908Q2|N|N|N|2418033|2418123|2417671|2417946|N|N|N|N|N| +2418038|AAAAAAAAGHFOECAA|1908-04-06|99|431|34|1908|1|4|6|2|1908|34|431|Monday|1908Q2|N|N|N|2418033|2418123|2417672|2417947|N|N|N|N|N| +2418039|AAAAAAAAHHFOECAA|1908-04-07|99|432|34|1908|2|4|7|2|1908|34|432|Tuesday|1908Q2|N|N|N|2418033|2418123|2417673|2417948|N|N|N|N|N| +2418040|AAAAAAAAIHFOECAA|1908-04-08|99|432|34|1908|3|4|8|2|1908|34|432|Wednesday|1908Q2|N|N|N|2418033|2418123|2417674|2417949|N|N|N|N|N| +2418041|AAAAAAAAJHFOECAA|1908-04-09|99|432|34|1908|4|4|9|2|1908|34|432|Thursday|1908Q2|N|N|N|2418033|2418123|2417675|2417950|N|N|N|N|N| +2418042|AAAAAAAAKHFOECAA|1908-04-10|99|432|34|1908|5|4|10|2|1908|34|432|Friday|1908Q2|N|Y|N|2418033|2418123|2417676|2417951|N|N|N|N|N| +2418043|AAAAAAAALHFOECAA|1908-04-11|99|432|34|1908|6|4|11|2|1908|34|432|Saturday|1908Q2|N|Y|N|2418033|2418123|2417677|2417952|N|N|N|N|N| +2418044|AAAAAAAAMHFOECAA|1908-04-12|99|432|34|1908|0|4|12|2|1908|34|432|Sunday|1908Q2|N|N|N|2418033|2418123|2417678|2417953|N|N|N|N|N| +2418045|AAAAAAAANHFOECAA|1908-04-13|99|432|34|1908|1|4|13|2|1908|34|432|Monday|1908Q2|N|N|N|2418033|2418123|2417679|2417954|N|N|N|N|N| +2418046|AAAAAAAAOHFOECAA|1908-04-14|99|433|34|1908|2|4|14|2|1908|34|433|Tuesday|1908Q2|N|N|N|2418033|2418123|2417680|2417955|N|N|N|N|N| +2418047|AAAAAAAAPHFOECAA|1908-04-15|99|433|34|1908|3|4|15|2|1908|34|433|Wednesday|1908Q2|N|N|N|2418033|2418123|2417681|2417956|N|N|N|N|N| +2418048|AAAAAAAAAIFOECAA|1908-04-16|99|433|34|1908|4|4|16|2|1908|34|433|Thursday|1908Q2|N|N|N|2418033|2418123|2417682|2417957|N|N|N|N|N| +2418049|AAAAAAAABIFOECAA|1908-04-17|99|433|34|1908|5|4|17|2|1908|34|433|Friday|1908Q2|N|Y|N|2418033|2418123|2417683|2417958|N|N|N|N|N| +2418050|AAAAAAAACIFOECAA|1908-04-18|99|433|34|1908|6|4|18|2|1908|34|433|Saturday|1908Q2|N|Y|N|2418033|2418123|2417684|2417959|N|N|N|N|N| +2418051|AAAAAAAADIFOECAA|1908-04-19|99|433|34|1908|0|4|19|2|1908|34|433|Sunday|1908Q2|N|N|N|2418033|2418123|2417685|2417960|N|N|N|N|N| +2418052|AAAAAAAAEIFOECAA|1908-04-20|99|433|34|1908|1|4|20|2|1908|34|433|Monday|1908Q2|N|N|N|2418033|2418123|2417686|2417961|N|N|N|N|N| +2418053|AAAAAAAAFIFOECAA|1908-04-21|99|434|34|1908|2|4|21|2|1908|34|434|Tuesday|1908Q2|N|N|N|2418033|2418123|2417687|2417962|N|N|N|N|N| +2418054|AAAAAAAAGIFOECAA|1908-04-22|99|434|34|1908|3|4|22|2|1908|34|434|Wednesday|1908Q2|N|N|N|2418033|2418123|2417688|2417963|N|N|N|N|N| +2418055|AAAAAAAAHIFOECAA|1908-04-23|99|434|34|1908|4|4|23|2|1908|34|434|Thursday|1908Q2|N|N|N|2418033|2418123|2417689|2417964|N|N|N|N|N| +2418056|AAAAAAAAIIFOECAA|1908-04-24|99|434|34|1908|5|4|24|2|1908|34|434|Friday|1908Q2|N|Y|N|2418033|2418123|2417690|2417965|N|N|N|N|N| +2418057|AAAAAAAAJIFOECAA|1908-04-25|99|434|34|1908|6|4|25|2|1908|34|434|Saturday|1908Q2|N|Y|N|2418033|2418123|2417691|2417966|N|N|N|N|N| +2418058|AAAAAAAAKIFOECAA|1908-04-26|99|434|34|1908|0|4|26|2|1908|34|434|Sunday|1908Q2|N|N|N|2418033|2418123|2417692|2417967|N|N|N|N|N| +2418059|AAAAAAAALIFOECAA|1908-04-27|99|434|34|1908|1|4|27|2|1908|34|434|Monday|1908Q2|N|N|N|2418033|2418123|2417693|2417968|N|N|N|N|N| +2418060|AAAAAAAAMIFOECAA|1908-04-28|99|435|34|1908|2|4|28|2|1908|34|435|Tuesday|1908Q2|N|N|N|2418033|2418123|2417694|2417969|N|N|N|N|N| +2418061|AAAAAAAANIFOECAA|1908-04-29|99|435|34|1908|3|4|29|2|1908|34|435|Wednesday|1908Q2|N|N|N|2418033|2418123|2417695|2417970|N|N|N|N|N| +2418062|AAAAAAAAOIFOECAA|1908-04-30|99|435|34|1908|4|4|30|2|1908|34|435|Thursday|1908Q2|N|N|N|2418033|2418123|2417696|2417971|N|N|N|N|N| +2418063|AAAAAAAAPIFOECAA|1908-05-01|100|435|34|1908|5|5|1|2|1908|34|435|Friday|1908Q2|N|Y|N|2418063|2418183|2417697|2417972|N|N|N|N|N| +2418064|AAAAAAAAAJFOECAA|1908-05-02|100|435|34|1908|6|5|2|2|1908|34|435|Saturday|1908Q2|N|Y|N|2418063|2418183|2417698|2417973|N|N|N|N|N| +2418065|AAAAAAAABJFOECAA|1908-05-03|100|435|34|1908|0|5|3|2|1908|34|435|Sunday|1908Q2|N|N|N|2418063|2418183|2417699|2417974|N|N|N|N|N| +2418066|AAAAAAAACJFOECAA|1908-05-04|100|435|34|1908|1|5|4|2|1908|34|435|Monday|1908Q2|N|N|N|2418063|2418183|2417700|2417975|N|N|N|N|N| +2418067|AAAAAAAADJFOECAA|1908-05-05|100|436|34|1908|2|5|5|2|1908|34|436|Tuesday|1908Q2|N|N|N|2418063|2418183|2417701|2417976|N|N|N|N|N| +2418068|AAAAAAAAEJFOECAA|1908-05-06|100|436|34|1908|3|5|6|2|1908|34|436|Wednesday|1908Q2|N|N|N|2418063|2418183|2417702|2417977|N|N|N|N|N| +2418069|AAAAAAAAFJFOECAA|1908-05-07|100|436|34|1908|4|5|7|2|1908|34|436|Thursday|1908Q2|N|N|N|2418063|2418183|2417703|2417978|N|N|N|N|N| +2418070|AAAAAAAAGJFOECAA|1908-05-08|100|436|34|1908|5|5|8|2|1908|34|436|Friday|1908Q2|N|Y|N|2418063|2418183|2417704|2417979|N|N|N|N|N| +2418071|AAAAAAAAHJFOECAA|1908-05-09|100|436|34|1908|6|5|9|2|1908|34|436|Saturday|1908Q2|N|Y|N|2418063|2418183|2417705|2417980|N|N|N|N|N| +2418072|AAAAAAAAIJFOECAA|1908-05-10|100|436|34|1908|0|5|10|2|1908|34|436|Sunday|1908Q2|N|N|N|2418063|2418183|2417706|2417981|N|N|N|N|N| +2418073|AAAAAAAAJJFOECAA|1908-05-11|100|436|34|1908|1|5|11|2|1908|34|436|Monday|1908Q2|N|N|N|2418063|2418183|2417707|2417982|N|N|N|N|N| +2418074|AAAAAAAAKJFOECAA|1908-05-12|100|437|34|1908|2|5|12|2|1908|34|437|Tuesday|1908Q2|N|N|N|2418063|2418183|2417708|2417983|N|N|N|N|N| +2418075|AAAAAAAALJFOECAA|1908-05-13|100|437|34|1908|3|5|13|2|1908|34|437|Wednesday|1908Q2|N|N|N|2418063|2418183|2417709|2417984|N|N|N|N|N| +2418076|AAAAAAAAMJFOECAA|1908-05-14|100|437|34|1908|4|5|14|2|1908|34|437|Thursday|1908Q2|N|N|N|2418063|2418183|2417710|2417985|N|N|N|N|N| +2418077|AAAAAAAANJFOECAA|1908-05-15|100|437|34|1908|5|5|15|2|1908|34|437|Friday|1908Q2|N|Y|N|2418063|2418183|2417711|2417986|N|N|N|N|N| +2418078|AAAAAAAAOJFOECAA|1908-05-16|100|437|34|1908|6|5|16|2|1908|34|437|Saturday|1908Q2|N|Y|N|2418063|2418183|2417712|2417987|N|N|N|N|N| +2418079|AAAAAAAAPJFOECAA|1908-05-17|100|437|34|1908|0|5|17|2|1908|34|437|Sunday|1908Q2|N|N|N|2418063|2418183|2417713|2417988|N|N|N|N|N| +2418080|AAAAAAAAAKFOECAA|1908-05-18|100|437|34|1908|1|5|18|2|1908|34|437|Monday|1908Q2|N|N|N|2418063|2418183|2417714|2417989|N|N|N|N|N| +2418081|AAAAAAAABKFOECAA|1908-05-19|100|438|34|1908|2|5|19|2|1908|34|438|Tuesday|1908Q2|N|N|N|2418063|2418183|2417715|2417990|N|N|N|N|N| +2418082|AAAAAAAACKFOECAA|1908-05-20|100|438|34|1908|3|5|20|2|1908|34|438|Wednesday|1908Q2|N|N|N|2418063|2418183|2417716|2417991|N|N|N|N|N| +2418083|AAAAAAAADKFOECAA|1908-05-21|100|438|34|1908|4|5|21|2|1908|34|438|Thursday|1908Q2|N|N|N|2418063|2418183|2417717|2417992|N|N|N|N|N| +2418084|AAAAAAAAEKFOECAA|1908-05-22|100|438|34|1908|5|5|22|2|1908|34|438|Friday|1908Q2|N|Y|N|2418063|2418183|2417718|2417993|N|N|N|N|N| +2418085|AAAAAAAAFKFOECAA|1908-05-23|100|438|34|1908|6|5|23|2|1908|34|438|Saturday|1908Q2|N|Y|N|2418063|2418183|2417719|2417994|N|N|N|N|N| +2418086|AAAAAAAAGKFOECAA|1908-05-24|100|438|34|1908|0|5|24|2|1908|34|438|Sunday|1908Q2|N|N|N|2418063|2418183|2417720|2417995|N|N|N|N|N| +2418087|AAAAAAAAHKFOECAA|1908-05-25|100|438|34|1908|1|5|25|2|1908|34|438|Monday|1908Q2|N|N|N|2418063|2418183|2417721|2417996|N|N|N|N|N| +2418088|AAAAAAAAIKFOECAA|1908-05-26|100|439|34|1908|2|5|26|2|1908|34|439|Tuesday|1908Q2|N|N|N|2418063|2418183|2417722|2417997|N|N|N|N|N| +2418089|AAAAAAAAJKFOECAA|1908-05-27|100|439|34|1908|3|5|27|2|1908|34|439|Wednesday|1908Q2|N|N|N|2418063|2418183|2417723|2417998|N|N|N|N|N| +2418090|AAAAAAAAKKFOECAA|1908-05-28|100|439|34|1908|4|5|28|2|1908|34|439|Thursday|1908Q2|N|N|N|2418063|2418183|2417724|2417999|N|N|N|N|N| +2418091|AAAAAAAALKFOECAA|1908-05-29|100|439|34|1908|5|5|29|2|1908|34|439|Friday|1908Q2|N|Y|N|2418063|2418183|2417725|2418000|N|N|N|N|N| +2418092|AAAAAAAAMKFOECAA|1908-05-30|100|439|34|1908|6|5|30|2|1908|34|439|Saturday|1908Q2|N|Y|N|2418063|2418183|2417726|2418001|N|N|N|N|N| +2418093|AAAAAAAANKFOECAA|1908-05-31|100|439|34|1908|0|5|31|2|1908|34|439|Sunday|1908Q2|N|N|N|2418063|2418183|2417727|2418002|N|N|N|N|N| +2418094|AAAAAAAAOKFOECAA|1908-06-01|101|439|35|1908|1|6|1|2|1908|35|439|Monday|1908Q2|N|N|N|2418094|2418245|2417728|2418003|N|N|N|N|N| +2418095|AAAAAAAAPKFOECAA|1908-06-02|101|440|35|1908|2|6|2|2|1908|35|440|Tuesday|1908Q2|N|N|N|2418094|2418245|2417729|2418004|N|N|N|N|N| +2418096|AAAAAAAAALFOECAA|1908-06-03|101|440|35|1908|3|6|3|2|1908|35|440|Wednesday|1908Q2|N|N|N|2418094|2418245|2417730|2418005|N|N|N|N|N| +2418097|AAAAAAAABLFOECAA|1908-06-04|101|440|35|1908|4|6|4|2|1908|35|440|Thursday|1908Q2|N|N|N|2418094|2418245|2417731|2418006|N|N|N|N|N| +2418098|AAAAAAAACLFOECAA|1908-06-05|101|440|35|1908|5|6|5|2|1908|35|440|Friday|1908Q2|N|Y|N|2418094|2418245|2417732|2418007|N|N|N|N|N| +2418099|AAAAAAAADLFOECAA|1908-06-06|101|440|35|1908|6|6|6|2|1908|35|440|Saturday|1908Q2|N|Y|N|2418094|2418245|2417733|2418008|N|N|N|N|N| +2418100|AAAAAAAAELFOECAA|1908-06-07|101|440|35|1908|0|6|7|2|1908|35|440|Sunday|1908Q2|N|N|N|2418094|2418245|2417734|2418009|N|N|N|N|N| +2418101|AAAAAAAAFLFOECAA|1908-06-08|101|440|35|1908|1|6|8|2|1908|35|440|Monday|1908Q2|N|N|N|2418094|2418245|2417735|2418010|N|N|N|N|N| +2418102|AAAAAAAAGLFOECAA|1908-06-09|101|441|35|1908|2|6|9|2|1908|35|441|Tuesday|1908Q2|N|N|N|2418094|2418245|2417736|2418011|N|N|N|N|N| +2418103|AAAAAAAAHLFOECAA|1908-06-10|101|441|35|1908|3|6|10|2|1908|35|441|Wednesday|1908Q2|N|N|N|2418094|2418245|2417737|2418012|N|N|N|N|N| +2418104|AAAAAAAAILFOECAA|1908-06-11|101|441|35|1908|4|6|11|2|1908|35|441|Thursday|1908Q2|N|N|N|2418094|2418245|2417738|2418013|N|N|N|N|N| +2418105|AAAAAAAAJLFOECAA|1908-06-12|101|441|35|1908|5|6|12|2|1908|35|441|Friday|1908Q2|N|Y|N|2418094|2418245|2417739|2418014|N|N|N|N|N| +2418106|AAAAAAAAKLFOECAA|1908-06-13|101|441|35|1908|6|6|13|2|1908|35|441|Saturday|1908Q2|N|Y|N|2418094|2418245|2417740|2418015|N|N|N|N|N| +2418107|AAAAAAAALLFOECAA|1908-06-14|101|441|35|1908|0|6|14|2|1908|35|441|Sunday|1908Q2|N|N|N|2418094|2418245|2417741|2418016|N|N|N|N|N| +2418108|AAAAAAAAMLFOECAA|1908-06-15|101|441|35|1908|1|6|15|2|1908|35|441|Monday|1908Q2|N|N|N|2418094|2418245|2417742|2418017|N|N|N|N|N| +2418109|AAAAAAAANLFOECAA|1908-06-16|101|442|35|1908|2|6|16|2|1908|35|442|Tuesday|1908Q2|N|N|N|2418094|2418245|2417743|2418018|N|N|N|N|N| +2418110|AAAAAAAAOLFOECAA|1908-06-17|101|442|35|1908|3|6|17|2|1908|35|442|Wednesday|1908Q2|N|N|N|2418094|2418245|2417744|2418019|N|N|N|N|N| +2418111|AAAAAAAAPLFOECAA|1908-06-18|101|442|35|1908|4|6|18|2|1908|35|442|Thursday|1908Q2|N|N|N|2418094|2418245|2417745|2418020|N|N|N|N|N| +2418112|AAAAAAAAAMFOECAA|1908-06-19|101|442|35|1908|5|6|19|2|1908|35|442|Friday|1908Q2|N|Y|N|2418094|2418245|2417746|2418021|N|N|N|N|N| +2418113|AAAAAAAABMFOECAA|1908-06-20|101|442|35|1908|6|6|20|2|1908|35|442|Saturday|1908Q2|N|Y|N|2418094|2418245|2417747|2418022|N|N|N|N|N| +2418114|AAAAAAAACMFOECAA|1908-06-21|101|442|35|1908|0|6|21|2|1908|35|442|Sunday|1908Q2|N|N|N|2418094|2418245|2417748|2418023|N|N|N|N|N| +2418115|AAAAAAAADMFOECAA|1908-06-22|101|442|35|1908|1|6|22|2|1908|35|442|Monday|1908Q2|N|N|N|2418094|2418245|2417749|2418024|N|N|N|N|N| +2418116|AAAAAAAAEMFOECAA|1908-06-23|101|443|35|1908|2|6|23|2|1908|35|443|Tuesday|1908Q2|N|N|N|2418094|2418245|2417750|2418025|N|N|N|N|N| +2418117|AAAAAAAAFMFOECAA|1908-06-24|101|443|35|1908|3|6|24|2|1908|35|443|Wednesday|1908Q2|N|N|N|2418094|2418245|2417751|2418026|N|N|N|N|N| +2418118|AAAAAAAAGMFOECAA|1908-06-25|101|443|35|1908|4|6|25|2|1908|35|443|Thursday|1908Q2|N|N|N|2418094|2418245|2417752|2418027|N|N|N|N|N| +2418119|AAAAAAAAHMFOECAA|1908-06-26|101|443|35|1908|5|6|26|2|1908|35|443|Friday|1908Q2|N|Y|N|2418094|2418245|2417753|2418028|N|N|N|N|N| +2418120|AAAAAAAAIMFOECAA|1908-06-27|101|443|35|1908|6|6|27|2|1908|35|443|Saturday|1908Q2|N|Y|N|2418094|2418245|2417754|2418029|N|N|N|N|N| +2418121|AAAAAAAAJMFOECAA|1908-06-28|101|443|35|1908|0|6|28|2|1908|35|443|Sunday|1908Q2|N|N|N|2418094|2418245|2417755|2418030|N|N|N|N|N| +2418122|AAAAAAAAKMFOECAA|1908-06-29|101|443|35|1908|1|6|29|2|1908|35|443|Monday|1908Q2|N|N|N|2418094|2418245|2417756|2418031|N|N|N|N|N| +2418123|AAAAAAAALMFOECAA|1908-06-30|101|444|35|1908|2|6|30|2|1908|35|444|Tuesday|1908Q2|N|N|N|2418094|2418245|2417757|2418032|N|N|N|N|N| +2418124|AAAAAAAAMMFOECAA|1908-07-01|102|444|35|1908|3|7|1|3|1908|35|444|Wednesday|1908Q3|N|N|N|2418124|2418305|2417758|2418033|N|N|N|N|N| +2418125|AAAAAAAANMFOECAA|1908-07-02|102|444|35|1908|4|7|2|3|1908|35|444|Thursday|1908Q3|N|N|N|2418124|2418305|2417759|2418034|N|N|N|N|N| +2418126|AAAAAAAAOMFOECAA|1908-07-03|102|444|35|1908|5|7|3|3|1908|35|444|Friday|1908Q3|N|Y|N|2418124|2418305|2417760|2418035|N|N|N|N|N| +2418127|AAAAAAAAPMFOECAA|1908-07-04|102|444|35|1908|6|7|4|3|1908|35|444|Saturday|1908Q3|Y|Y|N|2418124|2418305|2417761|2418036|N|N|N|N|N| +2418128|AAAAAAAAANFOECAA|1908-07-05|102|444|35|1908|0|7|5|3|1908|35|444|Sunday|1908Q3|N|N|Y|2418124|2418305|2417762|2418037|N|N|N|N|N| +2418129|AAAAAAAABNFOECAA|1908-07-06|102|444|35|1908|1|7|6|3|1908|35|444|Monday|1908Q3|N|N|N|2418124|2418305|2417763|2418038|N|N|N|N|N| +2418130|AAAAAAAACNFOECAA|1908-07-07|102|445|35|1908|2|7|7|3|1908|35|445|Tuesday|1908Q3|N|N|N|2418124|2418305|2417764|2418039|N|N|N|N|N| +2418131|AAAAAAAADNFOECAA|1908-07-08|102|445|35|1908|3|7|8|3|1908|35|445|Wednesday|1908Q3|N|N|N|2418124|2418305|2417765|2418040|N|N|N|N|N| +2418132|AAAAAAAAENFOECAA|1908-07-09|102|445|35|1908|4|7|9|3|1908|35|445|Thursday|1908Q3|N|N|N|2418124|2418305|2417766|2418041|N|N|N|N|N| +2418133|AAAAAAAAFNFOECAA|1908-07-10|102|445|35|1908|5|7|10|3|1908|35|445|Friday|1908Q3|N|Y|N|2418124|2418305|2417767|2418042|N|N|N|N|N| +2418134|AAAAAAAAGNFOECAA|1908-07-11|102|445|35|1908|6|7|11|3|1908|35|445|Saturday|1908Q3|N|Y|N|2418124|2418305|2417768|2418043|N|N|N|N|N| +2418135|AAAAAAAAHNFOECAA|1908-07-12|102|445|35|1908|0|7|12|3|1908|35|445|Sunday|1908Q3|N|N|N|2418124|2418305|2417769|2418044|N|N|N|N|N| +2418136|AAAAAAAAINFOECAA|1908-07-13|102|445|35|1908|1|7|13|3|1908|35|445|Monday|1908Q3|N|N|N|2418124|2418305|2417770|2418045|N|N|N|N|N| +2418137|AAAAAAAAJNFOECAA|1908-07-14|102|446|35|1908|2|7|14|3|1908|35|446|Tuesday|1908Q3|N|N|N|2418124|2418305|2417771|2418046|N|N|N|N|N| +2418138|AAAAAAAAKNFOECAA|1908-07-15|102|446|35|1908|3|7|15|3|1908|35|446|Wednesday|1908Q3|N|N|N|2418124|2418305|2417772|2418047|N|N|N|N|N| +2418139|AAAAAAAALNFOECAA|1908-07-16|102|446|35|1908|4|7|16|3|1908|35|446|Thursday|1908Q3|N|N|N|2418124|2418305|2417773|2418048|N|N|N|N|N| +2418140|AAAAAAAAMNFOECAA|1908-07-17|102|446|35|1908|5|7|17|3|1908|35|446|Friday|1908Q3|N|Y|N|2418124|2418305|2417774|2418049|N|N|N|N|N| +2418141|AAAAAAAANNFOECAA|1908-07-18|102|446|35|1908|6|7|18|3|1908|35|446|Saturday|1908Q3|N|Y|N|2418124|2418305|2417775|2418050|N|N|N|N|N| +2418142|AAAAAAAAONFOECAA|1908-07-19|102|446|35|1908|0|7|19|3|1908|35|446|Sunday|1908Q3|N|N|N|2418124|2418305|2417776|2418051|N|N|N|N|N| +2418143|AAAAAAAAPNFOECAA|1908-07-20|102|446|35|1908|1|7|20|3|1908|35|446|Monday|1908Q3|N|N|N|2418124|2418305|2417777|2418052|N|N|N|N|N| +2418144|AAAAAAAAAOFOECAA|1908-07-21|102|447|35|1908|2|7|21|3|1908|35|447|Tuesday|1908Q3|N|N|N|2418124|2418305|2417778|2418053|N|N|N|N|N| +2418145|AAAAAAAABOFOECAA|1908-07-22|102|447|35|1908|3|7|22|3|1908|35|447|Wednesday|1908Q3|N|N|N|2418124|2418305|2417779|2418054|N|N|N|N|N| +2418146|AAAAAAAACOFOECAA|1908-07-23|102|447|35|1908|4|7|23|3|1908|35|447|Thursday|1908Q3|N|N|N|2418124|2418305|2417780|2418055|N|N|N|N|N| +2418147|AAAAAAAADOFOECAA|1908-07-24|102|447|35|1908|5|7|24|3|1908|35|447|Friday|1908Q3|N|Y|N|2418124|2418305|2417781|2418056|N|N|N|N|N| +2418148|AAAAAAAAEOFOECAA|1908-07-25|102|447|35|1908|6|7|25|3|1908|35|447|Saturday|1908Q3|N|Y|N|2418124|2418305|2417782|2418057|N|N|N|N|N| +2418149|AAAAAAAAFOFOECAA|1908-07-26|102|447|35|1908|0|7|26|3|1908|35|447|Sunday|1908Q3|N|N|N|2418124|2418305|2417783|2418058|N|N|N|N|N| +2418150|AAAAAAAAGOFOECAA|1908-07-27|102|447|35|1908|1|7|27|3|1908|35|447|Monday|1908Q3|N|N|N|2418124|2418305|2417784|2418059|N|N|N|N|N| +2418151|AAAAAAAAHOFOECAA|1908-07-28|102|448|35|1908|2|7|28|3|1908|35|448|Tuesday|1908Q3|N|N|N|2418124|2418305|2417785|2418060|N|N|N|N|N| +2418152|AAAAAAAAIOFOECAA|1908-07-29|102|448|35|1908|3|7|29|3|1908|35|448|Wednesday|1908Q3|N|N|N|2418124|2418305|2417786|2418061|N|N|N|N|N| +2418153|AAAAAAAAJOFOECAA|1908-07-30|102|448|35|1908|4|7|30|3|1908|35|448|Thursday|1908Q3|N|N|N|2418124|2418305|2417787|2418062|N|N|N|N|N| +2418154|AAAAAAAAKOFOECAA|1908-07-31|102|448|35|1908|5|7|31|3|1908|35|448|Friday|1908Q3|N|Y|N|2418124|2418305|2417788|2418063|N|N|N|N|N| +2418155|AAAAAAAALOFOECAA|1908-08-01|103|448|35|1908|6|8|1|3|1908|35|448|Saturday|1908Q3|N|Y|N|2418155|2418367|2417789|2418064|N|N|N|N|N| +2418156|AAAAAAAAMOFOECAA|1908-08-02|103|448|35|1908|0|8|2|3|1908|35|448|Sunday|1908Q3|N|N|N|2418155|2418367|2417790|2418065|N|N|N|N|N| +2418157|AAAAAAAANOFOECAA|1908-08-03|103|448|35|1908|1|8|3|3|1908|35|448|Monday|1908Q3|N|N|N|2418155|2418367|2417791|2418066|N|N|N|N|N| +2418158|AAAAAAAAOOFOECAA|1908-08-04|103|449|35|1908|2|8|4|3|1908|35|449|Tuesday|1908Q3|N|N|N|2418155|2418367|2417792|2418067|N|N|N|N|N| +2418159|AAAAAAAAPOFOECAA|1908-08-05|103|449|35|1908|3|8|5|3|1908|35|449|Wednesday|1908Q3|N|N|N|2418155|2418367|2417793|2418068|N|N|N|N|N| +2418160|AAAAAAAAAPFOECAA|1908-08-06|103|449|35|1908|4|8|6|3|1908|35|449|Thursday|1908Q3|N|N|N|2418155|2418367|2417794|2418069|N|N|N|N|N| +2418161|AAAAAAAABPFOECAA|1908-08-07|103|449|35|1908|5|8|7|3|1908|35|449|Friday|1908Q3|N|Y|N|2418155|2418367|2417795|2418070|N|N|N|N|N| +2418162|AAAAAAAACPFOECAA|1908-08-08|103|449|35|1908|6|8|8|3|1908|35|449|Saturday|1908Q3|N|Y|N|2418155|2418367|2417796|2418071|N|N|N|N|N| +2418163|AAAAAAAADPFOECAA|1908-08-09|103|449|35|1908|0|8|9|3|1908|35|449|Sunday|1908Q3|N|N|N|2418155|2418367|2417797|2418072|N|N|N|N|N| +2418164|AAAAAAAAEPFOECAA|1908-08-10|103|449|35|1908|1|8|10|3|1908|35|449|Monday|1908Q3|N|N|N|2418155|2418367|2417798|2418073|N|N|N|N|N| +2418165|AAAAAAAAFPFOECAA|1908-08-11|103|450|35|1908|2|8|11|3|1908|35|450|Tuesday|1908Q3|N|N|N|2418155|2418367|2417799|2418074|N|N|N|N|N| +2418166|AAAAAAAAGPFOECAA|1908-08-12|103|450|35|1908|3|8|12|3|1908|35|450|Wednesday|1908Q3|N|N|N|2418155|2418367|2417800|2418075|N|N|N|N|N| +2418167|AAAAAAAAHPFOECAA|1908-08-13|103|450|35|1908|4|8|13|3|1908|35|450|Thursday|1908Q3|N|N|N|2418155|2418367|2417801|2418076|N|N|N|N|N| +2418168|AAAAAAAAIPFOECAA|1908-08-14|103|450|35|1908|5|8|14|3|1908|35|450|Friday|1908Q3|N|Y|N|2418155|2418367|2417802|2418077|N|N|N|N|N| +2418169|AAAAAAAAJPFOECAA|1908-08-15|103|450|35|1908|6|8|15|3|1908|35|450|Saturday|1908Q3|N|Y|N|2418155|2418367|2417803|2418078|N|N|N|N|N| +2418170|AAAAAAAAKPFOECAA|1908-08-16|103|450|35|1908|0|8|16|3|1908|35|450|Sunday|1908Q3|N|N|N|2418155|2418367|2417804|2418079|N|N|N|N|N| +2418171|AAAAAAAALPFOECAA|1908-08-17|103|450|35|1908|1|8|17|3|1908|35|450|Monday|1908Q3|N|N|N|2418155|2418367|2417805|2418080|N|N|N|N|N| +2418172|AAAAAAAAMPFOECAA|1908-08-18|103|451|35|1908|2|8|18|3|1908|35|451|Tuesday|1908Q3|N|N|N|2418155|2418367|2417806|2418081|N|N|N|N|N| +2418173|AAAAAAAANPFOECAA|1908-08-19|103|451|35|1908|3|8|19|3|1908|35|451|Wednesday|1908Q3|N|N|N|2418155|2418367|2417807|2418082|N|N|N|N|N| +2418174|AAAAAAAAOPFOECAA|1908-08-20|103|451|35|1908|4|8|20|3|1908|35|451|Thursday|1908Q3|N|N|N|2418155|2418367|2417808|2418083|N|N|N|N|N| +2418175|AAAAAAAAPPFOECAA|1908-08-21|103|451|35|1908|5|8|21|3|1908|35|451|Friday|1908Q3|N|Y|N|2418155|2418367|2417809|2418084|N|N|N|N|N| +2418176|AAAAAAAAAAGOECAA|1908-08-22|103|451|35|1908|6|8|22|3|1908|35|451|Saturday|1908Q3|N|Y|N|2418155|2418367|2417810|2418085|N|N|N|N|N| +2418177|AAAAAAAABAGOECAA|1908-08-23|103|451|35|1908|0|8|23|3|1908|35|451|Sunday|1908Q3|N|N|N|2418155|2418367|2417811|2418086|N|N|N|N|N| +2418178|AAAAAAAACAGOECAA|1908-08-24|103|451|35|1908|1|8|24|3|1908|35|451|Monday|1908Q3|N|N|N|2418155|2418367|2417812|2418087|N|N|N|N|N| +2418179|AAAAAAAADAGOECAA|1908-08-25|103|452|35|1908|2|8|25|3|1908|35|452|Tuesday|1908Q3|N|N|N|2418155|2418367|2417813|2418088|N|N|N|N|N| +2418180|AAAAAAAAEAGOECAA|1908-08-26|103|452|35|1908|3|8|26|3|1908|35|452|Wednesday|1908Q3|N|N|N|2418155|2418367|2417814|2418089|N|N|N|N|N| +2418181|AAAAAAAAFAGOECAA|1908-08-27|103|452|35|1908|4|8|27|3|1908|35|452|Thursday|1908Q3|N|N|N|2418155|2418367|2417815|2418090|N|N|N|N|N| +2418182|AAAAAAAAGAGOECAA|1908-08-28|103|452|35|1908|5|8|28|3|1908|35|452|Friday|1908Q3|N|Y|N|2418155|2418367|2417816|2418091|N|N|N|N|N| +2418183|AAAAAAAAHAGOECAA|1908-08-29|103|452|35|1908|6|8|29|3|1908|35|452|Saturday|1908Q3|N|Y|N|2418155|2418367|2417817|2418092|N|N|N|N|N| +2418184|AAAAAAAAIAGOECAA|1908-08-30|103|452|35|1908|0|8|30|3|1908|35|452|Sunday|1908Q3|N|N|N|2418155|2418367|2417818|2418093|N|N|N|N|N| +2418185|AAAAAAAAJAGOECAA|1908-08-31|103|452|35|1908|1|8|31|3|1908|35|452|Monday|1908Q3|N|N|N|2418155|2418367|2417819|2418094|N|N|N|N|N| +2418186|AAAAAAAAKAGOECAA|1908-09-01|104|453|36|1908|2|9|1|3|1908|36|453|Tuesday|1908Q3|N|N|N|2418186|2418429|2417820|2418095|N|N|N|N|N| +2418187|AAAAAAAALAGOECAA|1908-09-02|104|453|36|1908|3|9|2|3|1908|36|453|Wednesday|1908Q3|N|N|N|2418186|2418429|2417821|2418096|N|N|N|N|N| +2418188|AAAAAAAAMAGOECAA|1908-09-03|104|453|36|1908|4|9|3|3|1908|36|453|Thursday|1908Q3|N|N|N|2418186|2418429|2417822|2418097|N|N|N|N|N| +2418189|AAAAAAAANAGOECAA|1908-09-04|104|453|36|1908|5|9|4|3|1908|36|453|Friday|1908Q3|N|Y|N|2418186|2418429|2417823|2418098|N|N|N|N|N| +2418190|AAAAAAAAOAGOECAA|1908-09-05|104|453|36|1908|6|9|5|3|1908|36|453|Saturday|1908Q3|N|Y|N|2418186|2418429|2417824|2418099|N|N|N|N|N| +2418191|AAAAAAAAPAGOECAA|1908-09-06|104|453|36|1908|0|9|6|3|1908|36|453|Sunday|1908Q3|N|N|N|2418186|2418429|2417825|2418100|N|N|N|N|N| +2418192|AAAAAAAAABGOECAA|1908-09-07|104|453|36|1908|1|9|7|3|1908|36|453|Monday|1908Q3|N|N|N|2418186|2418429|2417826|2418101|N|N|N|N|N| +2418193|AAAAAAAABBGOECAA|1908-09-08|104|454|36|1908|2|9|8|3|1908|36|454|Tuesday|1908Q3|N|N|N|2418186|2418429|2417827|2418102|N|N|N|N|N| +2418194|AAAAAAAACBGOECAA|1908-09-09|104|454|36|1908|3|9|9|3|1908|36|454|Wednesday|1908Q3|N|N|N|2418186|2418429|2417828|2418103|N|N|N|N|N| +2418195|AAAAAAAADBGOECAA|1908-09-10|104|454|36|1908|4|9|10|3|1908|36|454|Thursday|1908Q3|N|N|N|2418186|2418429|2417829|2418104|N|N|N|N|N| +2418196|AAAAAAAAEBGOECAA|1908-09-11|104|454|36|1908|5|9|11|3|1908|36|454|Friday|1908Q3|N|Y|N|2418186|2418429|2417830|2418105|N|N|N|N|N| +2418197|AAAAAAAAFBGOECAA|1908-09-12|104|454|36|1908|6|9|12|3|1908|36|454|Saturday|1908Q3|N|Y|N|2418186|2418429|2417831|2418106|N|N|N|N|N| +2418198|AAAAAAAAGBGOECAA|1908-09-13|104|454|36|1908|0|9|13|3|1908|36|454|Sunday|1908Q3|N|N|N|2418186|2418429|2417832|2418107|N|N|N|N|N| +2418199|AAAAAAAAHBGOECAA|1908-09-14|104|454|36|1908|1|9|14|3|1908|36|454|Monday|1908Q3|N|N|N|2418186|2418429|2417833|2418108|N|N|N|N|N| +2418200|AAAAAAAAIBGOECAA|1908-09-15|104|455|36|1908|2|9|15|3|1908|36|455|Tuesday|1908Q3|N|N|N|2418186|2418429|2417834|2418109|N|N|N|N|N| +2418201|AAAAAAAAJBGOECAA|1908-09-16|104|455|36|1908|3|9|16|3|1908|36|455|Wednesday|1908Q3|N|N|N|2418186|2418429|2417835|2418110|N|N|N|N|N| +2418202|AAAAAAAAKBGOECAA|1908-09-17|104|455|36|1908|4|9|17|3|1908|36|455|Thursday|1908Q3|N|N|N|2418186|2418429|2417836|2418111|N|N|N|N|N| +2418203|AAAAAAAALBGOECAA|1908-09-18|104|455|36|1908|5|9|18|3|1908|36|455|Friday|1908Q3|N|Y|N|2418186|2418429|2417837|2418112|N|N|N|N|N| +2418204|AAAAAAAAMBGOECAA|1908-09-19|104|455|36|1908|6|9|19|3|1908|36|455|Saturday|1908Q3|N|Y|N|2418186|2418429|2417838|2418113|N|N|N|N|N| +2418205|AAAAAAAANBGOECAA|1908-09-20|104|455|36|1908|0|9|20|3|1908|36|455|Sunday|1908Q3|N|N|N|2418186|2418429|2417839|2418114|N|N|N|N|N| +2418206|AAAAAAAAOBGOECAA|1908-09-21|104|455|36|1908|1|9|21|3|1908|36|455|Monday|1908Q3|N|N|N|2418186|2418429|2417840|2418115|N|N|N|N|N| +2418207|AAAAAAAAPBGOECAA|1908-09-22|104|456|36|1908|2|9|22|3|1908|36|456|Tuesday|1908Q3|N|N|N|2418186|2418429|2417841|2418116|N|N|N|N|N| +2418208|AAAAAAAAACGOECAA|1908-09-23|104|456|36|1908|3|9|23|3|1908|36|456|Wednesday|1908Q3|N|N|N|2418186|2418429|2417842|2418117|N|N|N|N|N| +2418209|AAAAAAAABCGOECAA|1908-09-24|104|456|36|1908|4|9|24|3|1908|36|456|Thursday|1908Q3|N|N|N|2418186|2418429|2417843|2418118|N|N|N|N|N| +2418210|AAAAAAAACCGOECAA|1908-09-25|104|456|36|1908|5|9|25|3|1908|36|456|Friday|1908Q3|N|Y|N|2418186|2418429|2417844|2418119|N|N|N|N|N| +2418211|AAAAAAAADCGOECAA|1908-09-26|104|456|36|1908|6|9|26|3|1908|36|456|Saturday|1908Q3|N|Y|N|2418186|2418429|2417845|2418120|N|N|N|N|N| +2418212|AAAAAAAAECGOECAA|1908-09-27|104|456|36|1908|0|9|27|3|1908|36|456|Sunday|1908Q3|N|N|N|2418186|2418429|2417846|2418121|N|N|N|N|N| +2418213|AAAAAAAAFCGOECAA|1908-09-28|104|456|36|1908|1|9|28|3|1908|36|456|Monday|1908Q3|N|N|N|2418186|2418429|2417847|2418122|N|N|N|N|N| +2418214|AAAAAAAAGCGOECAA|1908-09-29|104|457|36|1908|2|9|29|3|1908|36|457|Tuesday|1908Q3|N|N|N|2418186|2418429|2417848|2418123|N|N|N|N|N| +2418215|AAAAAAAAHCGOECAA|1908-09-30|104|457|36|1908|3|9|30|3|1908|36|457|Wednesday|1908Q3|N|N|N|2418186|2418429|2417849|2418124|N|N|N|N|N| +2418216|AAAAAAAAICGOECAA|1908-10-01|105|457|36|1908|4|10|1|4|1908|36|457|Thursday|1908Q4|N|N|N|2418216|2418489|2417850|2418124|N|N|N|N|N| +2418217|AAAAAAAAJCGOECAA|1908-10-02|105|457|36|1908|5|10|2|4|1908|36|457|Friday|1908Q4|N|Y|N|2418216|2418489|2417851|2418125|N|N|N|N|N| +2418218|AAAAAAAAKCGOECAA|1908-10-03|105|457|36|1908|6|10|3|4|1908|36|457|Saturday|1908Q4|N|Y|N|2418216|2418489|2417852|2418126|N|N|N|N|N| +2418219|AAAAAAAALCGOECAA|1908-10-04|105|457|36|1908|0|10|4|4|1908|36|457|Sunday|1908Q4|N|N|N|2418216|2418489|2417853|2418127|N|N|N|N|N| +2418220|AAAAAAAAMCGOECAA|1908-10-05|105|457|36|1908|1|10|5|4|1908|36|457|Monday|1908Q4|N|N|N|2418216|2418489|2417854|2418128|N|N|N|N|N| +2418221|AAAAAAAANCGOECAA|1908-10-06|105|458|36|1908|2|10|6|4|1908|36|458|Tuesday|1908Q4|N|N|N|2418216|2418489|2417855|2418129|N|N|N|N|N| +2418222|AAAAAAAAOCGOECAA|1908-10-07|105|458|36|1908|3|10|7|4|1908|36|458|Wednesday|1908Q4|N|N|N|2418216|2418489|2417856|2418130|N|N|N|N|N| +2418223|AAAAAAAAPCGOECAA|1908-10-08|105|458|36|1908|4|10|8|4|1908|36|458|Thursday|1908Q4|N|N|N|2418216|2418489|2417857|2418131|N|N|N|N|N| +2418224|AAAAAAAAADGOECAA|1908-10-09|105|458|36|1908|5|10|9|4|1908|36|458|Friday|1908Q4|N|Y|N|2418216|2418489|2417858|2418132|N|N|N|N|N| +2418225|AAAAAAAABDGOECAA|1908-10-10|105|458|36|1908|6|10|10|4|1908|36|458|Saturday|1908Q4|N|Y|N|2418216|2418489|2417859|2418133|N|N|N|N|N| +2418226|AAAAAAAACDGOECAA|1908-10-11|105|458|36|1908|0|10|11|4|1908|36|458|Sunday|1908Q4|N|N|N|2418216|2418489|2417860|2418134|N|N|N|N|N| +2418227|AAAAAAAADDGOECAA|1908-10-12|105|458|36|1908|1|10|12|4|1908|36|458|Monday|1908Q4|N|N|N|2418216|2418489|2417861|2418135|N|N|N|N|N| +2418228|AAAAAAAAEDGOECAA|1908-10-13|105|459|36|1908|2|10|13|4|1908|36|459|Tuesday|1908Q4|N|N|N|2418216|2418489|2417862|2418136|N|N|N|N|N| +2418229|AAAAAAAAFDGOECAA|1908-10-14|105|459|36|1908|3|10|14|4|1908|36|459|Wednesday|1908Q4|N|N|N|2418216|2418489|2417863|2418137|N|N|N|N|N| +2418230|AAAAAAAAGDGOECAA|1908-10-15|105|459|36|1908|4|10|15|4|1908|36|459|Thursday|1908Q4|N|N|N|2418216|2418489|2417864|2418138|N|N|N|N|N| +2418231|AAAAAAAAHDGOECAA|1908-10-16|105|459|36|1908|5|10|16|4|1908|36|459|Friday|1908Q4|N|Y|N|2418216|2418489|2417865|2418139|N|N|N|N|N| +2418232|AAAAAAAAIDGOECAA|1908-10-17|105|459|36|1908|6|10|17|4|1908|36|459|Saturday|1908Q4|N|Y|N|2418216|2418489|2417866|2418140|N|N|N|N|N| +2418233|AAAAAAAAJDGOECAA|1908-10-18|105|459|36|1908|0|10|18|4|1908|36|459|Sunday|1908Q4|N|N|N|2418216|2418489|2417867|2418141|N|N|N|N|N| +2418234|AAAAAAAAKDGOECAA|1908-10-19|105|459|36|1908|1|10|19|4|1908|36|459|Monday|1908Q4|N|N|N|2418216|2418489|2417868|2418142|N|N|N|N|N| +2418235|AAAAAAAALDGOECAA|1908-10-20|105|460|36|1908|2|10|20|4|1908|36|460|Tuesday|1908Q4|N|N|N|2418216|2418489|2417869|2418143|N|N|N|N|N| +2418236|AAAAAAAAMDGOECAA|1908-10-21|105|460|36|1908|3|10|21|4|1908|36|460|Wednesday|1908Q4|N|N|N|2418216|2418489|2417870|2418144|N|N|N|N|N| +2418237|AAAAAAAANDGOECAA|1908-10-22|105|460|36|1908|4|10|22|4|1908|36|460|Thursday|1908Q4|N|N|N|2418216|2418489|2417871|2418145|N|N|N|N|N| +2418238|AAAAAAAAODGOECAA|1908-10-23|105|460|36|1908|5|10|23|4|1908|36|460|Friday|1908Q4|N|Y|N|2418216|2418489|2417872|2418146|N|N|N|N|N| +2418239|AAAAAAAAPDGOECAA|1908-10-24|105|460|36|1908|6|10|24|4|1908|36|460|Saturday|1908Q4|N|Y|N|2418216|2418489|2417873|2418147|N|N|N|N|N| +2418240|AAAAAAAAAEGOECAA|1908-10-25|105|460|36|1908|0|10|25|4|1908|36|460|Sunday|1908Q4|N|N|N|2418216|2418489|2417874|2418148|N|N|N|N|N| +2418241|AAAAAAAABEGOECAA|1908-10-26|105|460|36|1908|1|10|26|4|1908|36|460|Monday|1908Q4|N|N|N|2418216|2418489|2417875|2418149|N|N|N|N|N| +2418242|AAAAAAAACEGOECAA|1908-10-27|105|461|36|1908|2|10|27|4|1908|36|461|Tuesday|1908Q4|N|N|N|2418216|2418489|2417876|2418150|N|N|N|N|N| +2418243|AAAAAAAADEGOECAA|1908-10-28|105|461|36|1908|3|10|28|4|1908|36|461|Wednesday|1908Q4|N|N|N|2418216|2418489|2417877|2418151|N|N|N|N|N| +2418244|AAAAAAAAEEGOECAA|1908-10-29|105|461|36|1908|4|10|29|4|1908|36|461|Thursday|1908Q4|N|N|N|2418216|2418489|2417878|2418152|N|N|N|N|N| +2418245|AAAAAAAAFEGOECAA|1908-10-30|105|461|36|1908|5|10|30|4|1908|36|461|Friday|1908Q4|N|Y|N|2418216|2418489|2417879|2418153|N|N|N|N|N| +2418246|AAAAAAAAGEGOECAA|1908-10-31|105|461|36|1908|6|10|31|4|1908|36|461|Saturday|1908Q4|N|Y|N|2418216|2418489|2417880|2418154|N|N|N|N|N| +2418247|AAAAAAAAHEGOECAA|1908-11-01|106|461|36|1908|0|11|1|4|1908|36|461|Sunday|1908Q4|N|N|N|2418247|2418551|2417881|2418155|N|N|N|N|N| +2418248|AAAAAAAAIEGOECAA|1908-11-02|106|461|36|1908|1|11|2|4|1908|36|461|Monday|1908Q4|N|N|N|2418247|2418551|2417882|2418156|N|N|N|N|N| +2418249|AAAAAAAAJEGOECAA|1908-11-03|106|462|36|1908|2|11|3|4|1908|36|462|Tuesday|1908Q4|N|N|N|2418247|2418551|2417883|2418157|N|N|N|N|N| +2418250|AAAAAAAAKEGOECAA|1908-11-04|106|462|36|1908|3|11|4|4|1908|36|462|Wednesday|1908Q4|N|N|N|2418247|2418551|2417884|2418158|N|N|N|N|N| +2418251|AAAAAAAALEGOECAA|1908-11-05|106|462|36|1908|4|11|5|4|1908|36|462|Thursday|1908Q4|N|N|N|2418247|2418551|2417885|2418159|N|N|N|N|N| +2418252|AAAAAAAAMEGOECAA|1908-11-06|106|462|36|1908|5|11|6|4|1908|36|462|Friday|1908Q4|N|Y|N|2418247|2418551|2417886|2418160|N|N|N|N|N| +2418253|AAAAAAAANEGOECAA|1908-11-07|106|462|36|1908|6|11|7|4|1908|36|462|Saturday|1908Q4|N|Y|N|2418247|2418551|2417887|2418161|N|N|N|N|N| +2418254|AAAAAAAAOEGOECAA|1908-11-08|106|462|36|1908|0|11|8|4|1908|36|462|Sunday|1908Q4|N|N|N|2418247|2418551|2417888|2418162|N|N|N|N|N| +2418255|AAAAAAAAPEGOECAA|1908-11-09|106|462|36|1908|1|11|9|4|1908|36|462|Monday|1908Q4|N|N|N|2418247|2418551|2417889|2418163|N|N|N|N|N| +2418256|AAAAAAAAAFGOECAA|1908-11-10|106|463|36|1908|2|11|10|4|1908|36|463|Tuesday|1908Q4|N|N|N|2418247|2418551|2417890|2418164|N|N|N|N|N| +2418257|AAAAAAAABFGOECAA|1908-11-11|106|463|36|1908|3|11|11|4|1908|36|463|Wednesday|1908Q4|N|N|N|2418247|2418551|2417891|2418165|N|N|N|N|N| +2418258|AAAAAAAACFGOECAA|1908-11-12|106|463|36|1908|4|11|12|4|1908|36|463|Thursday|1908Q4|N|N|N|2418247|2418551|2417892|2418166|N|N|N|N|N| +2418259|AAAAAAAADFGOECAA|1908-11-13|106|463|36|1908|5|11|13|4|1908|36|463|Friday|1908Q4|N|Y|N|2418247|2418551|2417893|2418167|N|N|N|N|N| +2418260|AAAAAAAAEFGOECAA|1908-11-14|106|463|36|1908|6|11|14|4|1908|36|463|Saturday|1908Q4|N|Y|N|2418247|2418551|2417894|2418168|N|N|N|N|N| +2418261|AAAAAAAAFFGOECAA|1908-11-15|106|463|36|1908|0|11|15|4|1908|36|463|Sunday|1908Q4|N|N|N|2418247|2418551|2417895|2418169|N|N|N|N|N| +2418262|AAAAAAAAGFGOECAA|1908-11-16|106|463|36|1908|1|11|16|4|1908|36|463|Monday|1908Q4|N|N|N|2418247|2418551|2417896|2418170|N|N|N|N|N| +2418263|AAAAAAAAHFGOECAA|1908-11-17|106|464|36|1908|2|11|17|4|1908|36|464|Tuesday|1908Q4|N|N|N|2418247|2418551|2417897|2418171|N|N|N|N|N| +2418264|AAAAAAAAIFGOECAA|1908-11-18|106|464|36|1908|3|11|18|4|1908|36|464|Wednesday|1908Q4|N|N|N|2418247|2418551|2417898|2418172|N|N|N|N|N| +2418265|AAAAAAAAJFGOECAA|1908-11-19|106|464|36|1908|4|11|19|4|1908|36|464|Thursday|1908Q4|N|N|N|2418247|2418551|2417899|2418173|N|N|N|N|N| +2418266|AAAAAAAAKFGOECAA|1908-11-20|106|464|36|1908|5|11|20|4|1908|36|464|Friday|1908Q4|N|Y|N|2418247|2418551|2417900|2418174|N|N|N|N|N| +2418267|AAAAAAAALFGOECAA|1908-11-21|106|464|36|1908|6|11|21|4|1908|36|464|Saturday|1908Q4|N|Y|N|2418247|2418551|2417901|2418175|N|N|N|N|N| +2418268|AAAAAAAAMFGOECAA|1908-11-22|106|464|36|1908|0|11|22|4|1908|36|464|Sunday|1908Q4|N|N|N|2418247|2418551|2417902|2418176|N|N|N|N|N| +2418269|AAAAAAAANFGOECAA|1908-11-23|106|464|36|1908|1|11|23|4|1908|36|464|Monday|1908Q4|N|N|N|2418247|2418551|2417903|2418177|N|N|N|N|N| +2418270|AAAAAAAAOFGOECAA|1908-11-24|106|465|36|1908|2|11|24|4|1908|36|465|Tuesday|1908Q4|N|N|N|2418247|2418551|2417904|2418178|N|N|N|N|N| +2418271|AAAAAAAAPFGOECAA|1908-11-25|106|465|36|1908|3|11|25|4|1908|36|465|Wednesday|1908Q4|N|N|N|2418247|2418551|2417905|2418179|N|N|N|N|N| +2418272|AAAAAAAAAGGOECAA|1908-11-26|106|465|36|1908|4|11|26|4|1908|36|465|Thursday|1908Q4|N|N|N|2418247|2418551|2417906|2418180|N|N|N|N|N| +2418273|AAAAAAAABGGOECAA|1908-11-27|106|465|36|1908|5|11|27|4|1908|36|465|Friday|1908Q4|N|Y|N|2418247|2418551|2417907|2418181|N|N|N|N|N| +2418274|AAAAAAAACGGOECAA|1908-11-28|106|465|36|1908|6|11|28|4|1908|36|465|Saturday|1908Q4|N|Y|N|2418247|2418551|2417908|2418182|N|N|N|N|N| +2418275|AAAAAAAADGGOECAA|1908-11-29|106|465|36|1908|0|11|29|4|1908|36|465|Sunday|1908Q4|N|N|N|2418247|2418551|2417909|2418183|N|N|N|N|N| +2418276|AAAAAAAAEGGOECAA|1908-11-30|106|465|36|1908|1|11|30|4|1908|36|465|Monday|1908Q4|N|N|N|2418247|2418551|2417910|2418184|N|N|N|N|N| +2418277|AAAAAAAAFGGOECAA|1908-12-01|107|466|37|1908|2|12|1|4|1908|37|466|Tuesday|1908Q4|N|N|N|2418277|2418611|2417911|2418185|N|N|N|N|N| +2418278|AAAAAAAAGGGOECAA|1908-12-02|107|466|37|1908|3|12|2|4|1908|37|466|Wednesday|1908Q4|N|N|N|2418277|2418611|2417912|2418186|N|N|N|N|N| +2418279|AAAAAAAAHGGOECAA|1908-12-03|107|466|37|1908|4|12|3|4|1908|37|466|Thursday|1908Q4|N|N|N|2418277|2418611|2417913|2418187|N|N|N|N|N| +2418280|AAAAAAAAIGGOECAA|1908-12-04|107|466|37|1908|5|12|4|4|1908|37|466|Friday|1908Q4|N|Y|N|2418277|2418611|2417914|2418188|N|N|N|N|N| +2418281|AAAAAAAAJGGOECAA|1908-12-05|107|466|37|1908|6|12|5|4|1908|37|466|Saturday|1908Q4|N|Y|N|2418277|2418611|2417915|2418189|N|N|N|N|N| +2418282|AAAAAAAAKGGOECAA|1908-12-06|107|466|37|1908|0|12|6|4|1908|37|466|Sunday|1908Q4|N|N|N|2418277|2418611|2417916|2418190|N|N|N|N|N| +2418283|AAAAAAAALGGOECAA|1908-12-07|107|466|37|1908|1|12|7|4|1908|37|466|Monday|1908Q4|N|N|N|2418277|2418611|2417917|2418191|N|N|N|N|N| +2418284|AAAAAAAAMGGOECAA|1908-12-08|107|467|37|1908|2|12|8|4|1908|37|467|Tuesday|1908Q4|N|N|N|2418277|2418611|2417918|2418192|N|N|N|N|N| +2418285|AAAAAAAANGGOECAA|1908-12-09|107|467|37|1908|3|12|9|4|1908|37|467|Wednesday|1908Q4|N|N|N|2418277|2418611|2417919|2418193|N|N|N|N|N| +2418286|AAAAAAAAOGGOECAA|1908-12-10|107|467|37|1908|4|12|10|4|1908|37|467|Thursday|1908Q4|N|N|N|2418277|2418611|2417920|2418194|N|N|N|N|N| +2418287|AAAAAAAAPGGOECAA|1908-12-11|107|467|37|1908|5|12|11|4|1908|37|467|Friday|1908Q4|N|Y|N|2418277|2418611|2417921|2418195|N|N|N|N|N| +2418288|AAAAAAAAAHGOECAA|1908-12-12|107|467|37|1908|6|12|12|4|1908|37|467|Saturday|1908Q4|N|Y|N|2418277|2418611|2417922|2418196|N|N|N|N|N| +2418289|AAAAAAAABHGOECAA|1908-12-13|107|467|37|1908|0|12|13|4|1908|37|467|Sunday|1908Q4|N|N|N|2418277|2418611|2417923|2418197|N|N|N|N|N| +2418290|AAAAAAAACHGOECAA|1908-12-14|107|467|37|1908|1|12|14|4|1908|37|467|Monday|1908Q4|N|N|N|2418277|2418611|2417924|2418198|N|N|N|N|N| +2418291|AAAAAAAADHGOECAA|1908-12-15|107|468|37|1908|2|12|15|4|1908|37|468|Tuesday|1908Q4|N|N|N|2418277|2418611|2417925|2418199|N|N|N|N|N| +2418292|AAAAAAAAEHGOECAA|1908-12-16|107|468|37|1908|3|12|16|4|1908|37|468|Wednesday|1908Q4|N|N|N|2418277|2418611|2417926|2418200|N|N|N|N|N| +2418293|AAAAAAAAFHGOECAA|1908-12-17|107|468|37|1908|4|12|17|4|1908|37|468|Thursday|1908Q4|N|N|N|2418277|2418611|2417927|2418201|N|N|N|N|N| +2418294|AAAAAAAAGHGOECAA|1908-12-18|107|468|37|1908|5|12|18|4|1908|37|468|Friday|1908Q4|N|Y|N|2418277|2418611|2417928|2418202|N|N|N|N|N| +2418295|AAAAAAAAHHGOECAA|1908-12-19|107|468|37|1908|6|12|19|4|1908|37|468|Saturday|1908Q4|N|Y|N|2418277|2418611|2417929|2418203|N|N|N|N|N| +2418296|AAAAAAAAIHGOECAA|1908-12-20|107|468|37|1908|0|12|20|4|1908|37|468|Sunday|1908Q4|N|N|N|2418277|2418611|2417930|2418204|N|N|N|N|N| +2418297|AAAAAAAAJHGOECAA|1908-12-21|107|468|37|1908|1|12|21|4|1908|37|468|Monday|1908Q4|N|N|N|2418277|2418611|2417931|2418205|N|N|N|N|N| +2418298|AAAAAAAAKHGOECAA|1908-12-22|107|469|37|1908|2|12|22|4|1908|37|469|Tuesday|1908Q4|N|N|N|2418277|2418611|2417932|2418206|N|N|N|N|N| +2418299|AAAAAAAALHGOECAA|1908-12-23|107|469|37|1908|3|12|23|4|1908|37|469|Wednesday|1908Q4|N|N|N|2418277|2418611|2417933|2418207|N|N|N|N|N| +2418300|AAAAAAAAMHGOECAA|1908-12-24|107|469|37|1908|4|12|24|4|1908|37|469|Thursday|1908Q4|N|N|N|2418277|2418611|2417934|2418208|N|N|N|N|N| +2418301|AAAAAAAANHGOECAA|1908-12-25|107|469|37|1908|5|12|25|4|1908|37|469|Friday|1908Q4|Y|Y|N|2418277|2418611|2417935|2418209|N|N|N|N|N| +2418302|AAAAAAAAOHGOECAA|1908-12-26|107|469|37|1908|6|12|26|4|1908|37|469|Saturday|1908Q4|N|Y|Y|2418277|2418611|2417936|2418210|N|N|N|N|N| +2418303|AAAAAAAAPHGOECAA|1908-12-27|107|469|37|1908|0|12|27|4|1908|37|469|Sunday|1908Q4|N|N|N|2418277|2418611|2417937|2418211|N|N|N|N|N| +2418304|AAAAAAAAAIGOECAA|1908-12-28|107|469|37|1908|1|12|28|4|1908|37|469|Monday|1908Q4|N|N|N|2418277|2418611|2417938|2418212|N|N|N|N|N| +2418305|AAAAAAAABIGOECAA|1908-12-29|107|470|37|1908|2|12|29|4|1908|37|470|Tuesday|1908Q4|N|N|N|2418277|2418611|2417939|2418213|N|N|N|N|N| +2418306|AAAAAAAACIGOECAA|1908-12-30|107|470|37|1908|3|12|30|4|1908|37|470|Wednesday|1908Q4|N|N|N|2418277|2418611|2417940|2418214|N|N|N|N|N| +2418307|AAAAAAAADIGOECAA|1908-12-31|107|470|37|1908|4|12|31|4|1908|37|470|Thursday|1908Q4|Y|N|N|2418277|2418611|2417941|2418215|N|N|N|N|N| +2418308|AAAAAAAAEIGOECAA|1909-01-01|108|470|37|1909|5|1|1|1|1909|37|470|Friday|1909Q1|Y|Y|Y|2418308|2418307|2417942|2418216|N|N|N|N|N| +2418309|AAAAAAAAFIGOECAA|1909-01-02|108|470|37|1909|6|1|2|1|1909|37|470|Saturday|1909Q1|N|Y|Y|2418308|2418307|2417943|2418217|N|N|N|N|N| +2418310|AAAAAAAAGIGOECAA|1909-01-03|108|470|37|1909|0|1|3|1|1909|37|470|Sunday|1909Q1|N|N|N|2418308|2418307|2417944|2418218|N|N|N|N|N| +2418311|AAAAAAAAHIGOECAA|1909-01-04|108|470|37|1909|1|1|4|1|1909|37|470|Monday|1909Q1|N|N|N|2418308|2418307|2417945|2418219|N|N|N|N|N| +2418312|AAAAAAAAIIGOECAA|1909-01-05|108|471|37|1909|2|1|5|1|1909|37|471|Tuesday|1909Q1|N|N|N|2418308|2418307|2417946|2418220|N|N|N|N|N| +2418313|AAAAAAAAJIGOECAA|1909-01-06|108|471|37|1909|3|1|6|1|1909|37|471|Wednesday|1909Q1|N|N|N|2418308|2418307|2417947|2418221|N|N|N|N|N| +2418314|AAAAAAAAKIGOECAA|1909-01-07|108|471|37|1909|4|1|7|1|1909|37|471|Thursday|1909Q1|N|N|N|2418308|2418307|2417948|2418222|N|N|N|N|N| +2418315|AAAAAAAALIGOECAA|1909-01-08|108|471|37|1909|5|1|8|1|1909|37|471|Friday|1909Q1|N|Y|N|2418308|2418307|2417949|2418223|N|N|N|N|N| +2418316|AAAAAAAAMIGOECAA|1909-01-09|108|471|37|1909|6|1|9|1|1909|37|471|Saturday|1909Q1|N|Y|N|2418308|2418307|2417950|2418224|N|N|N|N|N| +2418317|AAAAAAAANIGOECAA|1909-01-10|108|471|37|1909|0|1|10|1|1909|37|471|Sunday|1909Q1|N|N|N|2418308|2418307|2417951|2418225|N|N|N|N|N| +2418318|AAAAAAAAOIGOECAA|1909-01-11|108|471|37|1909|1|1|11|1|1909|37|471|Monday|1909Q1|N|N|N|2418308|2418307|2417952|2418226|N|N|N|N|N| +2418319|AAAAAAAAPIGOECAA|1909-01-12|108|472|37|1909|2|1|12|1|1909|37|472|Tuesday|1909Q1|N|N|N|2418308|2418307|2417953|2418227|N|N|N|N|N| +2418320|AAAAAAAAAJGOECAA|1909-01-13|108|472|37|1909|3|1|13|1|1909|37|472|Wednesday|1909Q1|N|N|N|2418308|2418307|2417954|2418228|N|N|N|N|N| +2418321|AAAAAAAABJGOECAA|1909-01-14|108|472|37|1909|4|1|14|1|1909|37|472|Thursday|1909Q1|N|N|N|2418308|2418307|2417955|2418229|N|N|N|N|N| +2418322|AAAAAAAACJGOECAA|1909-01-15|108|472|37|1909|5|1|15|1|1909|37|472|Friday|1909Q1|N|Y|N|2418308|2418307|2417956|2418230|N|N|N|N|N| +2418323|AAAAAAAADJGOECAA|1909-01-16|108|472|37|1909|6|1|16|1|1909|37|472|Saturday|1909Q1|N|Y|N|2418308|2418307|2417957|2418231|N|N|N|N|N| +2418324|AAAAAAAAEJGOECAA|1909-01-17|108|472|37|1909|0|1|17|1|1909|37|472|Sunday|1909Q1|N|N|N|2418308|2418307|2417958|2418232|N|N|N|N|N| +2418325|AAAAAAAAFJGOECAA|1909-01-18|108|472|37|1909|1|1|18|1|1909|37|472|Monday|1909Q1|N|N|N|2418308|2418307|2417959|2418233|N|N|N|N|N| +2418326|AAAAAAAAGJGOECAA|1909-01-19|108|473|37|1909|2|1|19|1|1909|37|473|Tuesday|1909Q1|N|N|N|2418308|2418307|2417960|2418234|N|N|N|N|N| +2418327|AAAAAAAAHJGOECAA|1909-01-20|108|473|37|1909|3|1|20|1|1909|37|473|Wednesday|1909Q1|N|N|N|2418308|2418307|2417961|2418235|N|N|N|N|N| +2418328|AAAAAAAAIJGOECAA|1909-01-21|108|473|37|1909|4|1|21|1|1909|37|473|Thursday|1909Q1|N|N|N|2418308|2418307|2417962|2418236|N|N|N|N|N| +2418329|AAAAAAAAJJGOECAA|1909-01-22|108|473|37|1909|5|1|22|1|1909|37|473|Friday|1909Q1|N|Y|N|2418308|2418307|2417963|2418237|N|N|N|N|N| +2418330|AAAAAAAAKJGOECAA|1909-01-23|108|473|37|1909|6|1|23|1|1909|37|473|Saturday|1909Q1|N|Y|N|2418308|2418307|2417964|2418238|N|N|N|N|N| +2418331|AAAAAAAALJGOECAA|1909-01-24|108|473|37|1909|0|1|24|1|1909|37|473|Sunday|1909Q1|N|N|N|2418308|2418307|2417965|2418239|N|N|N|N|N| +2418332|AAAAAAAAMJGOECAA|1909-01-25|108|473|37|1909|1|1|25|1|1909|37|473|Monday|1909Q1|N|N|N|2418308|2418307|2417966|2418240|N|N|N|N|N| +2418333|AAAAAAAANJGOECAA|1909-01-26|108|474|37|1909|2|1|26|1|1909|37|474|Tuesday|1909Q1|N|N|N|2418308|2418307|2417967|2418241|N|N|N|N|N| +2418334|AAAAAAAAOJGOECAA|1909-01-27|108|474|37|1909|3|1|27|1|1909|37|474|Wednesday|1909Q1|N|N|N|2418308|2418307|2417968|2418242|N|N|N|N|N| +2418335|AAAAAAAAPJGOECAA|1909-01-28|108|474|37|1909|4|1|28|1|1909|37|474|Thursday|1909Q1|N|N|N|2418308|2418307|2417969|2418243|N|N|N|N|N| +2418336|AAAAAAAAAKGOECAA|1909-01-29|108|474|37|1909|5|1|29|1|1909|37|474|Friday|1909Q1|N|Y|N|2418308|2418307|2417970|2418244|N|N|N|N|N| +2418337|AAAAAAAABKGOECAA|1909-01-30|108|474|37|1909|6|1|30|1|1909|37|474|Saturday|1909Q1|N|Y|N|2418308|2418307|2417971|2418245|N|N|N|N|N| +2418338|AAAAAAAACKGOECAA|1909-01-31|108|474|37|1909|0|1|31|1|1909|37|474|Sunday|1909Q1|N|N|N|2418308|2418307|2417972|2418246|N|N|N|N|N| +2418339|AAAAAAAADKGOECAA|1909-02-01|109|474|37|1909|1|2|1|1|1909|37|474|Monday|1909Q1|N|N|N|2418339|2418369|2417973|2418247|N|N|N|N|N| +2418340|AAAAAAAAEKGOECAA|1909-02-02|109|475|37|1909|2|2|2|1|1909|37|475|Tuesday|1909Q1|N|N|N|2418339|2418369|2417974|2418248|N|N|N|N|N| +2418341|AAAAAAAAFKGOECAA|1909-02-03|109|475|37|1909|3|2|3|1|1909|37|475|Wednesday|1909Q1|N|N|N|2418339|2418369|2417975|2418249|N|N|N|N|N| +2418342|AAAAAAAAGKGOECAA|1909-02-04|109|475|37|1909|4|2|4|1|1909|37|475|Thursday|1909Q1|N|N|N|2418339|2418369|2417976|2418250|N|N|N|N|N| +2418343|AAAAAAAAHKGOECAA|1909-02-05|109|475|37|1909|5|2|5|1|1909|37|475|Friday|1909Q1|N|Y|N|2418339|2418369|2417977|2418251|N|N|N|N|N| +2418344|AAAAAAAAIKGOECAA|1909-02-06|109|475|37|1909|6|2|6|1|1909|37|475|Saturday|1909Q1|N|Y|N|2418339|2418369|2417978|2418252|N|N|N|N|N| +2418345|AAAAAAAAJKGOECAA|1909-02-07|109|475|37|1909|0|2|7|1|1909|37|475|Sunday|1909Q1|N|N|N|2418339|2418369|2417979|2418253|N|N|N|N|N| +2418346|AAAAAAAAKKGOECAA|1909-02-08|109|475|37|1909|1|2|8|1|1909|37|475|Monday|1909Q1|N|N|N|2418339|2418369|2417980|2418254|N|N|N|N|N| +2418347|AAAAAAAALKGOECAA|1909-02-09|109|476|37|1909|2|2|9|1|1909|37|476|Tuesday|1909Q1|N|N|N|2418339|2418369|2417981|2418255|N|N|N|N|N| +2418348|AAAAAAAAMKGOECAA|1909-02-10|109|476|37|1909|3|2|10|1|1909|37|476|Wednesday|1909Q1|N|N|N|2418339|2418369|2417982|2418256|N|N|N|N|N| +2418349|AAAAAAAANKGOECAA|1909-02-11|109|476|37|1909|4|2|11|1|1909|37|476|Thursday|1909Q1|N|N|N|2418339|2418369|2417983|2418257|N|N|N|N|N| +2418350|AAAAAAAAOKGOECAA|1909-02-12|109|476|37|1909|5|2|12|1|1909|37|476|Friday|1909Q1|N|Y|N|2418339|2418369|2417984|2418258|N|N|N|N|N| +2418351|AAAAAAAAPKGOECAA|1909-02-13|109|476|37|1909|6|2|13|1|1909|37|476|Saturday|1909Q1|N|Y|N|2418339|2418369|2417985|2418259|N|N|N|N|N| +2418352|AAAAAAAAALGOECAA|1909-02-14|109|476|37|1909|0|2|14|1|1909|37|476|Sunday|1909Q1|N|N|N|2418339|2418369|2417986|2418260|N|N|N|N|N| +2418353|AAAAAAAABLGOECAA|1909-02-15|109|476|37|1909|1|2|15|1|1909|37|476|Monday|1909Q1|N|N|N|2418339|2418369|2417987|2418261|N|N|N|N|N| +2418354|AAAAAAAACLGOECAA|1909-02-16|109|477|37|1909|2|2|16|1|1909|37|477|Tuesday|1909Q1|N|N|N|2418339|2418369|2417988|2418262|N|N|N|N|N| +2418355|AAAAAAAADLGOECAA|1909-02-17|109|477|37|1909|3|2|17|1|1909|37|477|Wednesday|1909Q1|N|N|N|2418339|2418369|2417989|2418263|N|N|N|N|N| +2418356|AAAAAAAAELGOECAA|1909-02-18|109|477|37|1909|4|2|18|1|1909|37|477|Thursday|1909Q1|N|N|N|2418339|2418369|2417990|2418264|N|N|N|N|N| +2418357|AAAAAAAAFLGOECAA|1909-02-19|109|477|37|1909|5|2|19|1|1909|37|477|Friday|1909Q1|N|Y|N|2418339|2418369|2417991|2418265|N|N|N|N|N| +2418358|AAAAAAAAGLGOECAA|1909-02-20|109|477|37|1909|6|2|20|1|1909|37|477|Saturday|1909Q1|N|Y|N|2418339|2418369|2417992|2418266|N|N|N|N|N| +2418359|AAAAAAAAHLGOECAA|1909-02-21|109|477|37|1909|0|2|21|1|1909|37|477|Sunday|1909Q1|N|N|N|2418339|2418369|2417993|2418267|N|N|N|N|N| +2418360|AAAAAAAAILGOECAA|1909-02-22|109|477|37|1909|1|2|22|1|1909|37|477|Monday|1909Q1|N|N|N|2418339|2418369|2417994|2418268|N|N|N|N|N| +2418361|AAAAAAAAJLGOECAA|1909-02-23|109|478|37|1909|2|2|23|1|1909|37|478|Tuesday|1909Q1|N|N|N|2418339|2418369|2417995|2418269|N|N|N|N|N| +2418362|AAAAAAAAKLGOECAA|1909-02-24|109|478|37|1909|3|2|24|1|1909|37|478|Wednesday|1909Q1|N|N|N|2418339|2418369|2417996|2418270|N|N|N|N|N| +2418363|AAAAAAAALLGOECAA|1909-02-25|109|478|37|1909|4|2|25|1|1909|37|478|Thursday|1909Q1|N|N|N|2418339|2418369|2417997|2418271|N|N|N|N|N| +2418364|AAAAAAAAMLGOECAA|1909-02-26|109|478|37|1909|5|2|26|1|1909|37|478|Friday|1909Q1|N|Y|N|2418339|2418369|2417998|2418272|N|N|N|N|N| +2418365|AAAAAAAANLGOECAA|1909-02-27|109|478|37|1909|6|2|27|1|1909|37|478|Saturday|1909Q1|N|Y|N|2418339|2418369|2417999|2418273|N|N|N|N|N| +2418366|AAAAAAAAOLGOECAA|1909-02-28|109|478|37|1909|0|2|28|1|1909|37|478|Sunday|1909Q1|N|N|N|2418339|2418369|2418000|2418274|N|N|N|N|N| +2418367|AAAAAAAAPLGOECAA|1909-03-01|110|478|38|1909|1|3|1|1|1909|38|478|Monday|1909Q1|N|N|N|2418367|2418425|2418002|2418275|N|N|N|N|N| +2418368|AAAAAAAAAMGOECAA|1909-03-02|110|479|38|1909|2|3|2|1|1909|38|479|Tuesday|1909Q1|N|N|N|2418367|2418425|2418003|2418276|N|N|N|N|N| +2418369|AAAAAAAABMGOECAA|1909-03-03|110|479|38|1909|3|3|3|1|1909|38|479|Wednesday|1909Q1|N|N|N|2418367|2418425|2418004|2418277|N|N|N|N|N| +2418370|AAAAAAAACMGOECAA|1909-03-04|110|479|38|1909|4|3|4|1|1909|38|479|Thursday|1909Q1|N|N|N|2418367|2418425|2418005|2418278|N|N|N|N|N| +2418371|AAAAAAAADMGOECAA|1909-03-05|110|479|38|1909|5|3|5|1|1909|38|479|Friday|1909Q1|N|Y|N|2418367|2418425|2418006|2418279|N|N|N|N|N| +2418372|AAAAAAAAEMGOECAA|1909-03-06|110|479|38|1909|6|3|6|1|1909|38|479|Saturday|1909Q1|N|Y|N|2418367|2418425|2418007|2418280|N|N|N|N|N| +2418373|AAAAAAAAFMGOECAA|1909-03-07|110|479|38|1909|0|3|7|1|1909|38|479|Sunday|1909Q1|N|N|N|2418367|2418425|2418008|2418281|N|N|N|N|N| +2418374|AAAAAAAAGMGOECAA|1909-03-08|110|479|38|1909|1|3|8|1|1909|38|479|Monday|1909Q1|N|N|N|2418367|2418425|2418009|2418282|N|N|N|N|N| +2418375|AAAAAAAAHMGOECAA|1909-03-09|110|480|38|1909|2|3|9|1|1909|38|480|Tuesday|1909Q1|N|N|N|2418367|2418425|2418010|2418283|N|N|N|N|N| +2418376|AAAAAAAAIMGOECAA|1909-03-10|110|480|38|1909|3|3|10|1|1909|38|480|Wednesday|1909Q1|N|N|N|2418367|2418425|2418011|2418284|N|N|N|N|N| +2418377|AAAAAAAAJMGOECAA|1909-03-11|110|480|38|1909|4|3|11|1|1909|38|480|Thursday|1909Q1|N|N|N|2418367|2418425|2418012|2418285|N|N|N|N|N| +2418378|AAAAAAAAKMGOECAA|1909-03-12|110|480|38|1909|5|3|12|1|1909|38|480|Friday|1909Q1|N|Y|N|2418367|2418425|2418013|2418286|N|N|N|N|N| +2418379|AAAAAAAALMGOECAA|1909-03-13|110|480|38|1909|6|3|13|1|1909|38|480|Saturday|1909Q1|N|Y|N|2418367|2418425|2418014|2418287|N|N|N|N|N| +2418380|AAAAAAAAMMGOECAA|1909-03-14|110|480|38|1909|0|3|14|1|1909|38|480|Sunday|1909Q1|N|N|N|2418367|2418425|2418015|2418288|N|N|N|N|N| +2418381|AAAAAAAANMGOECAA|1909-03-15|110|480|38|1909|1|3|15|1|1909|38|480|Monday|1909Q1|N|N|N|2418367|2418425|2418016|2418289|N|N|N|N|N| +2418382|AAAAAAAAOMGOECAA|1909-03-16|110|481|38|1909|2|3|16|1|1909|38|481|Tuesday|1909Q1|N|N|N|2418367|2418425|2418017|2418290|N|N|N|N|N| +2418383|AAAAAAAAPMGOECAA|1909-03-17|110|481|38|1909|3|3|17|1|1909|38|481|Wednesday|1909Q1|N|N|N|2418367|2418425|2418018|2418291|N|N|N|N|N| +2418384|AAAAAAAAANGOECAA|1909-03-18|110|481|38|1909|4|3|18|1|1909|38|481|Thursday|1909Q1|N|N|N|2418367|2418425|2418019|2418292|N|N|N|N|N| +2418385|AAAAAAAABNGOECAA|1909-03-19|110|481|38|1909|5|3|19|1|1909|38|481|Friday|1909Q1|N|Y|N|2418367|2418425|2418020|2418293|N|N|N|N|N| +2418386|AAAAAAAACNGOECAA|1909-03-20|110|481|38|1909|6|3|20|1|1909|38|481|Saturday|1909Q1|N|Y|N|2418367|2418425|2418021|2418294|N|N|N|N|N| +2418387|AAAAAAAADNGOECAA|1909-03-21|110|481|38|1909|0|3|21|1|1909|38|481|Sunday|1909Q1|N|N|N|2418367|2418425|2418022|2418295|N|N|N|N|N| +2418388|AAAAAAAAENGOECAA|1909-03-22|110|481|38|1909|1|3|22|1|1909|38|481|Monday|1909Q1|N|N|N|2418367|2418425|2418023|2418296|N|N|N|N|N| +2418389|AAAAAAAAFNGOECAA|1909-03-23|110|482|38|1909|2|3|23|1|1909|38|482|Tuesday|1909Q1|N|N|N|2418367|2418425|2418024|2418297|N|N|N|N|N| +2418390|AAAAAAAAGNGOECAA|1909-03-24|110|482|38|1909|3|3|24|1|1909|38|482|Wednesday|1909Q1|N|N|N|2418367|2418425|2418025|2418298|N|N|N|N|N| +2418391|AAAAAAAAHNGOECAA|1909-03-25|110|482|38|1909|4|3|25|1|1909|38|482|Thursday|1909Q1|N|N|N|2418367|2418425|2418026|2418299|N|N|N|N|N| +2418392|AAAAAAAAINGOECAA|1909-03-26|110|482|38|1909|5|3|26|1|1909|38|482|Friday|1909Q1|N|Y|N|2418367|2418425|2418027|2418300|N|N|N|N|N| +2418393|AAAAAAAAJNGOECAA|1909-03-27|110|482|38|1909|6|3|27|1|1909|38|482|Saturday|1909Q1|N|Y|N|2418367|2418425|2418028|2418301|N|N|N|N|N| +2418394|AAAAAAAAKNGOECAA|1909-03-28|110|482|38|1909|0|3|28|1|1909|38|482|Sunday|1909Q1|N|N|N|2418367|2418425|2418029|2418302|N|N|N|N|N| +2418395|AAAAAAAALNGOECAA|1909-03-29|110|482|38|1909|1|3|29|1|1909|38|482|Monday|1909Q1|N|N|N|2418367|2418425|2418030|2418303|N|N|N|N|N| +2418396|AAAAAAAAMNGOECAA|1909-03-30|110|483|38|1909|2|3|30|1|1909|38|483|Tuesday|1909Q1|N|N|N|2418367|2418425|2418031|2418304|N|N|N|N|N| +2418397|AAAAAAAANNGOECAA|1909-03-31|110|483|38|1909|3|3|31|1|1909|38|483|Wednesday|1909Q1|N|N|N|2418367|2418425|2418032|2418305|N|N|N|N|N| +2418398|AAAAAAAAONGOECAA|1909-04-01|111|483|38|1909|4|4|1|1|1909|38|483|Thursday|1909Q1|N|N|N|2418398|2418487|2418033|2418308|N|N|N|N|N| +2418399|AAAAAAAAPNGOECAA|1909-04-02|111|483|38|1909|5|4|2|2|1909|38|483|Friday|1909Q2|N|Y|N|2418398|2418487|2418034|2418309|N|N|N|N|N| +2418400|AAAAAAAAAOGOECAA|1909-04-03|111|483|38|1909|6|4|3|2|1909|38|483|Saturday|1909Q2|N|Y|N|2418398|2418487|2418035|2418310|N|N|N|N|N| +2418401|AAAAAAAABOGOECAA|1909-04-04|111|483|38|1909|0|4|4|2|1909|38|483|Sunday|1909Q2|N|N|N|2418398|2418487|2418036|2418311|N|N|N|N|N| +2418402|AAAAAAAACOGOECAA|1909-04-05|111|483|38|1909|1|4|5|2|1909|38|483|Monday|1909Q2|N|N|N|2418398|2418487|2418037|2418312|N|N|N|N|N| +2418403|AAAAAAAADOGOECAA|1909-04-06|111|484|38|1909|2|4|6|2|1909|38|484|Tuesday|1909Q2|N|N|N|2418398|2418487|2418038|2418313|N|N|N|N|N| +2418404|AAAAAAAAEOGOECAA|1909-04-07|111|484|38|1909|3|4|7|2|1909|38|484|Wednesday|1909Q2|N|N|N|2418398|2418487|2418039|2418314|N|N|N|N|N| +2418405|AAAAAAAAFOGOECAA|1909-04-08|111|484|38|1909|4|4|8|2|1909|38|484|Thursday|1909Q2|N|N|N|2418398|2418487|2418040|2418315|N|N|N|N|N| +2418406|AAAAAAAAGOGOECAA|1909-04-09|111|484|38|1909|5|4|9|2|1909|38|484|Friday|1909Q2|N|Y|N|2418398|2418487|2418041|2418316|N|N|N|N|N| +2418407|AAAAAAAAHOGOECAA|1909-04-10|111|484|38|1909|6|4|10|2|1909|38|484|Saturday|1909Q2|N|Y|N|2418398|2418487|2418042|2418317|N|N|N|N|N| +2418408|AAAAAAAAIOGOECAA|1909-04-11|111|484|38|1909|0|4|11|2|1909|38|484|Sunday|1909Q2|N|N|N|2418398|2418487|2418043|2418318|N|N|N|N|N| +2418409|AAAAAAAAJOGOECAA|1909-04-12|111|484|38|1909|1|4|12|2|1909|38|484|Monday|1909Q2|N|N|N|2418398|2418487|2418044|2418319|N|N|N|N|N| +2418410|AAAAAAAAKOGOECAA|1909-04-13|111|485|38|1909|2|4|13|2|1909|38|485|Tuesday|1909Q2|N|N|N|2418398|2418487|2418045|2418320|N|N|N|N|N| +2418411|AAAAAAAALOGOECAA|1909-04-14|111|485|38|1909|3|4|14|2|1909|38|485|Wednesday|1909Q2|N|N|N|2418398|2418487|2418046|2418321|N|N|N|N|N| +2418412|AAAAAAAAMOGOECAA|1909-04-15|111|485|38|1909|4|4|15|2|1909|38|485|Thursday|1909Q2|N|N|N|2418398|2418487|2418047|2418322|N|N|N|N|N| +2418413|AAAAAAAANOGOECAA|1909-04-16|111|485|38|1909|5|4|16|2|1909|38|485|Friday|1909Q2|N|Y|N|2418398|2418487|2418048|2418323|N|N|N|N|N| +2418414|AAAAAAAAOOGOECAA|1909-04-17|111|485|38|1909|6|4|17|2|1909|38|485|Saturday|1909Q2|N|Y|N|2418398|2418487|2418049|2418324|N|N|N|N|N| +2418415|AAAAAAAAPOGOECAA|1909-04-18|111|485|38|1909|0|4|18|2|1909|38|485|Sunday|1909Q2|N|N|N|2418398|2418487|2418050|2418325|N|N|N|N|N| +2418416|AAAAAAAAAPGOECAA|1909-04-19|111|485|38|1909|1|4|19|2|1909|38|485|Monday|1909Q2|N|N|N|2418398|2418487|2418051|2418326|N|N|N|N|N| +2418417|AAAAAAAABPGOECAA|1909-04-20|111|486|38|1909|2|4|20|2|1909|38|486|Tuesday|1909Q2|N|N|N|2418398|2418487|2418052|2418327|N|N|N|N|N| +2418418|AAAAAAAACPGOECAA|1909-04-21|111|486|38|1909|3|4|21|2|1909|38|486|Wednesday|1909Q2|N|N|N|2418398|2418487|2418053|2418328|N|N|N|N|N| +2418419|AAAAAAAADPGOECAA|1909-04-22|111|486|38|1909|4|4|22|2|1909|38|486|Thursday|1909Q2|N|N|N|2418398|2418487|2418054|2418329|N|N|N|N|N| +2418420|AAAAAAAAEPGOECAA|1909-04-23|111|486|38|1909|5|4|23|2|1909|38|486|Friday|1909Q2|N|Y|N|2418398|2418487|2418055|2418330|N|N|N|N|N| +2418421|AAAAAAAAFPGOECAA|1909-04-24|111|486|38|1909|6|4|24|2|1909|38|486|Saturday|1909Q2|N|Y|N|2418398|2418487|2418056|2418331|N|N|N|N|N| +2418422|AAAAAAAAGPGOECAA|1909-04-25|111|486|38|1909|0|4|25|2|1909|38|486|Sunday|1909Q2|N|N|N|2418398|2418487|2418057|2418332|N|N|N|N|N| +2418423|AAAAAAAAHPGOECAA|1909-04-26|111|486|38|1909|1|4|26|2|1909|38|486|Monday|1909Q2|N|N|N|2418398|2418487|2418058|2418333|N|N|N|N|N| +2418424|AAAAAAAAIPGOECAA|1909-04-27|111|487|38|1909|2|4|27|2|1909|38|487|Tuesday|1909Q2|N|N|N|2418398|2418487|2418059|2418334|N|N|N|N|N| +2418425|AAAAAAAAJPGOECAA|1909-04-28|111|487|38|1909|3|4|28|2|1909|38|487|Wednesday|1909Q2|N|N|N|2418398|2418487|2418060|2418335|N|N|N|N|N| +2418426|AAAAAAAAKPGOECAA|1909-04-29|111|487|38|1909|4|4|29|2|1909|38|487|Thursday|1909Q2|N|N|N|2418398|2418487|2418061|2418336|N|N|N|N|N| +2418427|AAAAAAAALPGOECAA|1909-04-30|111|487|38|1909|5|4|30|2|1909|38|487|Friday|1909Q2|N|Y|N|2418398|2418487|2418062|2418337|N|N|N|N|N| +2418428|AAAAAAAAMPGOECAA|1909-05-01|112|487|38|1909|6|5|1|2|1909|38|487|Saturday|1909Q2|N|Y|N|2418428|2418547|2418063|2418338|N|N|N|N|N| +2418429|AAAAAAAANPGOECAA|1909-05-02|112|487|38|1909|0|5|2|2|1909|38|487|Sunday|1909Q2|N|N|N|2418428|2418547|2418064|2418339|N|N|N|N|N| +2418430|AAAAAAAAOPGOECAA|1909-05-03|112|487|38|1909|1|5|3|2|1909|38|487|Monday|1909Q2|N|N|N|2418428|2418547|2418065|2418340|N|N|N|N|N| +2418431|AAAAAAAAPPGOECAA|1909-05-04|112|488|38|1909|2|5|4|2|1909|38|488|Tuesday|1909Q2|N|N|N|2418428|2418547|2418066|2418341|N|N|N|N|N| +2418432|AAAAAAAAAAHOECAA|1909-05-05|112|488|38|1909|3|5|5|2|1909|38|488|Wednesday|1909Q2|N|N|N|2418428|2418547|2418067|2418342|N|N|N|N|N| +2418433|AAAAAAAABAHOECAA|1909-05-06|112|488|38|1909|4|5|6|2|1909|38|488|Thursday|1909Q2|N|N|N|2418428|2418547|2418068|2418343|N|N|N|N|N| +2418434|AAAAAAAACAHOECAA|1909-05-07|112|488|38|1909|5|5|7|2|1909|38|488|Friday|1909Q2|N|Y|N|2418428|2418547|2418069|2418344|N|N|N|N|N| +2418435|AAAAAAAADAHOECAA|1909-05-08|112|488|38|1909|6|5|8|2|1909|38|488|Saturday|1909Q2|N|Y|N|2418428|2418547|2418070|2418345|N|N|N|N|N| +2418436|AAAAAAAAEAHOECAA|1909-05-09|112|488|38|1909|0|5|9|2|1909|38|488|Sunday|1909Q2|N|N|N|2418428|2418547|2418071|2418346|N|N|N|N|N| +2418437|AAAAAAAAFAHOECAA|1909-05-10|112|488|38|1909|1|5|10|2|1909|38|488|Monday|1909Q2|N|N|N|2418428|2418547|2418072|2418347|N|N|N|N|N| +2418438|AAAAAAAAGAHOECAA|1909-05-11|112|489|38|1909|2|5|11|2|1909|38|489|Tuesday|1909Q2|N|N|N|2418428|2418547|2418073|2418348|N|N|N|N|N| +2418439|AAAAAAAAHAHOECAA|1909-05-12|112|489|38|1909|3|5|12|2|1909|38|489|Wednesday|1909Q2|N|N|N|2418428|2418547|2418074|2418349|N|N|N|N|N| +2418440|AAAAAAAAIAHOECAA|1909-05-13|112|489|38|1909|4|5|13|2|1909|38|489|Thursday|1909Q2|N|N|N|2418428|2418547|2418075|2418350|N|N|N|N|N| +2418441|AAAAAAAAJAHOECAA|1909-05-14|112|489|38|1909|5|5|14|2|1909|38|489|Friday|1909Q2|N|Y|N|2418428|2418547|2418076|2418351|N|N|N|N|N| +2418442|AAAAAAAAKAHOECAA|1909-05-15|112|489|38|1909|6|5|15|2|1909|38|489|Saturday|1909Q2|N|Y|N|2418428|2418547|2418077|2418352|N|N|N|N|N| +2418443|AAAAAAAALAHOECAA|1909-05-16|112|489|38|1909|0|5|16|2|1909|38|489|Sunday|1909Q2|N|N|N|2418428|2418547|2418078|2418353|N|N|N|N|N| +2418444|AAAAAAAAMAHOECAA|1909-05-17|112|489|38|1909|1|5|17|2|1909|38|489|Monday|1909Q2|N|N|N|2418428|2418547|2418079|2418354|N|N|N|N|N| +2418445|AAAAAAAANAHOECAA|1909-05-18|112|490|38|1909|2|5|18|2|1909|38|490|Tuesday|1909Q2|N|N|N|2418428|2418547|2418080|2418355|N|N|N|N|N| +2418446|AAAAAAAAOAHOECAA|1909-05-19|112|490|38|1909|3|5|19|2|1909|38|490|Wednesday|1909Q2|N|N|N|2418428|2418547|2418081|2418356|N|N|N|N|N| +2418447|AAAAAAAAPAHOECAA|1909-05-20|112|490|38|1909|4|5|20|2|1909|38|490|Thursday|1909Q2|N|N|N|2418428|2418547|2418082|2418357|N|N|N|N|N| +2418448|AAAAAAAAABHOECAA|1909-05-21|112|490|38|1909|5|5|21|2|1909|38|490|Friday|1909Q2|N|Y|N|2418428|2418547|2418083|2418358|N|N|N|N|N| +2418449|AAAAAAAABBHOECAA|1909-05-22|112|490|38|1909|6|5|22|2|1909|38|490|Saturday|1909Q2|N|Y|N|2418428|2418547|2418084|2418359|N|N|N|N|N| +2418450|AAAAAAAACBHOECAA|1909-05-23|112|490|38|1909|0|5|23|2|1909|38|490|Sunday|1909Q2|N|N|N|2418428|2418547|2418085|2418360|N|N|N|N|N| +2418451|AAAAAAAADBHOECAA|1909-05-24|112|490|38|1909|1|5|24|2|1909|38|490|Monday|1909Q2|N|N|N|2418428|2418547|2418086|2418361|N|N|N|N|N| +2418452|AAAAAAAAEBHOECAA|1909-05-25|112|491|38|1909|2|5|25|2|1909|38|491|Tuesday|1909Q2|N|N|N|2418428|2418547|2418087|2418362|N|N|N|N|N| +2418453|AAAAAAAAFBHOECAA|1909-05-26|112|491|38|1909|3|5|26|2|1909|38|491|Wednesday|1909Q2|N|N|N|2418428|2418547|2418088|2418363|N|N|N|N|N| +2418454|AAAAAAAAGBHOECAA|1909-05-27|112|491|38|1909|4|5|27|2|1909|38|491|Thursday|1909Q2|N|N|N|2418428|2418547|2418089|2418364|N|N|N|N|N| +2418455|AAAAAAAAHBHOECAA|1909-05-28|112|491|38|1909|5|5|28|2|1909|38|491|Friday|1909Q2|N|Y|N|2418428|2418547|2418090|2418365|N|N|N|N|N| +2418456|AAAAAAAAIBHOECAA|1909-05-29|112|491|38|1909|6|5|29|2|1909|38|491|Saturday|1909Q2|N|Y|N|2418428|2418547|2418091|2418366|N|N|N|N|N| +2418457|AAAAAAAAJBHOECAA|1909-05-30|112|491|38|1909|0|5|30|2|1909|38|491|Sunday|1909Q2|N|N|N|2418428|2418547|2418092|2418367|N|N|N|N|N| +2418458|AAAAAAAAKBHOECAA|1909-05-31|112|491|38|1909|1|5|31|2|1909|38|491|Monday|1909Q2|N|N|N|2418428|2418547|2418093|2418368|N|N|N|N|N| +2418459|AAAAAAAALBHOECAA|1909-06-01|113|492|39|1909|2|6|1|2|1909|39|492|Tuesday|1909Q2|N|N|N|2418459|2418609|2418094|2418369|N|N|N|N|N| +2418460|AAAAAAAAMBHOECAA|1909-06-02|113|492|39|1909|3|6|2|2|1909|39|492|Wednesday|1909Q2|N|N|N|2418459|2418609|2418095|2418370|N|N|N|N|N| +2418461|AAAAAAAANBHOECAA|1909-06-03|113|492|39|1909|4|6|3|2|1909|39|492|Thursday|1909Q2|N|N|N|2418459|2418609|2418096|2418371|N|N|N|N|N| +2418462|AAAAAAAAOBHOECAA|1909-06-04|113|492|39|1909|5|6|4|2|1909|39|492|Friday|1909Q2|N|Y|N|2418459|2418609|2418097|2418372|N|N|N|N|N| +2418463|AAAAAAAAPBHOECAA|1909-06-05|113|492|39|1909|6|6|5|2|1909|39|492|Saturday|1909Q2|N|Y|N|2418459|2418609|2418098|2418373|N|N|N|N|N| +2418464|AAAAAAAAACHOECAA|1909-06-06|113|492|39|1909|0|6|6|2|1909|39|492|Sunday|1909Q2|N|N|N|2418459|2418609|2418099|2418374|N|N|N|N|N| +2418465|AAAAAAAABCHOECAA|1909-06-07|113|492|39|1909|1|6|7|2|1909|39|492|Monday|1909Q2|N|N|N|2418459|2418609|2418100|2418375|N|N|N|N|N| +2418466|AAAAAAAACCHOECAA|1909-06-08|113|493|39|1909|2|6|8|2|1909|39|493|Tuesday|1909Q2|N|N|N|2418459|2418609|2418101|2418376|N|N|N|N|N| +2418467|AAAAAAAADCHOECAA|1909-06-09|113|493|39|1909|3|6|9|2|1909|39|493|Wednesday|1909Q2|N|N|N|2418459|2418609|2418102|2418377|N|N|N|N|N| +2418468|AAAAAAAAECHOECAA|1909-06-10|113|493|39|1909|4|6|10|2|1909|39|493|Thursday|1909Q2|N|N|N|2418459|2418609|2418103|2418378|N|N|N|N|N| +2418469|AAAAAAAAFCHOECAA|1909-06-11|113|493|39|1909|5|6|11|2|1909|39|493|Friday|1909Q2|N|Y|N|2418459|2418609|2418104|2418379|N|N|N|N|N| +2418470|AAAAAAAAGCHOECAA|1909-06-12|113|493|39|1909|6|6|12|2|1909|39|493|Saturday|1909Q2|N|Y|N|2418459|2418609|2418105|2418380|N|N|N|N|N| +2418471|AAAAAAAAHCHOECAA|1909-06-13|113|493|39|1909|0|6|13|2|1909|39|493|Sunday|1909Q2|N|N|N|2418459|2418609|2418106|2418381|N|N|N|N|N| +2418472|AAAAAAAAICHOECAA|1909-06-14|113|493|39|1909|1|6|14|2|1909|39|493|Monday|1909Q2|N|N|N|2418459|2418609|2418107|2418382|N|N|N|N|N| +2418473|AAAAAAAAJCHOECAA|1909-06-15|113|494|39|1909|2|6|15|2|1909|39|494|Tuesday|1909Q2|N|N|N|2418459|2418609|2418108|2418383|N|N|N|N|N| +2418474|AAAAAAAAKCHOECAA|1909-06-16|113|494|39|1909|3|6|16|2|1909|39|494|Wednesday|1909Q2|N|N|N|2418459|2418609|2418109|2418384|N|N|N|N|N| +2418475|AAAAAAAALCHOECAA|1909-06-17|113|494|39|1909|4|6|17|2|1909|39|494|Thursday|1909Q2|N|N|N|2418459|2418609|2418110|2418385|N|N|N|N|N| +2418476|AAAAAAAAMCHOECAA|1909-06-18|113|494|39|1909|5|6|18|2|1909|39|494|Friday|1909Q2|N|Y|N|2418459|2418609|2418111|2418386|N|N|N|N|N| +2418477|AAAAAAAANCHOECAA|1909-06-19|113|494|39|1909|6|6|19|2|1909|39|494|Saturday|1909Q2|N|Y|N|2418459|2418609|2418112|2418387|N|N|N|N|N| +2418478|AAAAAAAAOCHOECAA|1909-06-20|113|494|39|1909|0|6|20|2|1909|39|494|Sunday|1909Q2|N|N|N|2418459|2418609|2418113|2418388|N|N|N|N|N| +2418479|AAAAAAAAPCHOECAA|1909-06-21|113|494|39|1909|1|6|21|2|1909|39|494|Monday|1909Q2|N|N|N|2418459|2418609|2418114|2418389|N|N|N|N|N| +2418480|AAAAAAAAADHOECAA|1909-06-22|113|495|39|1909|2|6|22|2|1909|39|495|Tuesday|1909Q2|N|N|N|2418459|2418609|2418115|2418390|N|N|N|N|N| +2418481|AAAAAAAABDHOECAA|1909-06-23|113|495|39|1909|3|6|23|2|1909|39|495|Wednesday|1909Q2|N|N|N|2418459|2418609|2418116|2418391|N|N|N|N|N| +2418482|AAAAAAAACDHOECAA|1909-06-24|113|495|39|1909|4|6|24|2|1909|39|495|Thursday|1909Q2|N|N|N|2418459|2418609|2418117|2418392|N|N|N|N|N| +2418483|AAAAAAAADDHOECAA|1909-06-25|113|495|39|1909|5|6|25|2|1909|39|495|Friday|1909Q2|N|Y|N|2418459|2418609|2418118|2418393|N|N|N|N|N| +2418484|AAAAAAAAEDHOECAA|1909-06-26|113|495|39|1909|6|6|26|2|1909|39|495|Saturday|1909Q2|N|Y|N|2418459|2418609|2418119|2418394|N|N|N|N|N| +2418485|AAAAAAAAFDHOECAA|1909-06-27|113|495|39|1909|0|6|27|2|1909|39|495|Sunday|1909Q2|N|N|N|2418459|2418609|2418120|2418395|N|N|N|N|N| +2418486|AAAAAAAAGDHOECAA|1909-06-28|113|495|39|1909|1|6|28|2|1909|39|495|Monday|1909Q2|N|N|N|2418459|2418609|2418121|2418396|N|N|N|N|N| +2418487|AAAAAAAAHDHOECAA|1909-06-29|113|496|39|1909|2|6|29|2|1909|39|496|Tuesday|1909Q2|N|N|N|2418459|2418609|2418122|2418397|N|N|N|N|N| +2418488|AAAAAAAAIDHOECAA|1909-06-30|113|496|39|1909|3|6|30|2|1909|39|496|Wednesday|1909Q2|N|N|N|2418459|2418609|2418123|2418398|N|N|N|N|N| +2418489|AAAAAAAAJDHOECAA|1909-07-01|114|496|39|1909|4|7|1|2|1909|39|496|Thursday|1909Q2|N|N|N|2418489|2418669|2418124|2418398|N|N|N|N|N| +2418490|AAAAAAAAKDHOECAA|1909-07-02|114|496|39|1909|5|7|2|3|1909|39|496|Friday|1909Q3|N|Y|N|2418489|2418669|2418125|2418399|N|N|N|N|N| +2418491|AAAAAAAALDHOECAA|1909-07-03|114|496|39|1909|6|7|3|3|1909|39|496|Saturday|1909Q3|N|Y|N|2418489|2418669|2418126|2418400|N|N|N|N|N| +2418492|AAAAAAAAMDHOECAA|1909-07-04|114|496|39|1909|0|7|4|3|1909|39|496|Sunday|1909Q3|N|N|N|2418489|2418669|2418127|2418401|N|N|N|N|N| +2418493|AAAAAAAANDHOECAA|1909-07-05|114|496|39|1909|1|7|5|3|1909|39|496|Monday|1909Q3|Y|N|N|2418489|2418669|2418128|2418402|N|N|N|N|N| +2418494|AAAAAAAAODHOECAA|1909-07-06|114|497|39|1909|2|7|6|3|1909|39|497|Tuesday|1909Q3|N|N|Y|2418489|2418669|2418129|2418403|N|N|N|N|N| +2418495|AAAAAAAAPDHOECAA|1909-07-07|114|497|39|1909|3|7|7|3|1909|39|497|Wednesday|1909Q3|N|N|N|2418489|2418669|2418130|2418404|N|N|N|N|N| +2418496|AAAAAAAAAEHOECAA|1909-07-08|114|497|39|1909|4|7|8|3|1909|39|497|Thursday|1909Q3|N|N|N|2418489|2418669|2418131|2418405|N|N|N|N|N| +2418497|AAAAAAAABEHOECAA|1909-07-09|114|497|39|1909|5|7|9|3|1909|39|497|Friday|1909Q3|N|Y|N|2418489|2418669|2418132|2418406|N|N|N|N|N| +2418498|AAAAAAAACEHOECAA|1909-07-10|114|497|39|1909|6|7|10|3|1909|39|497|Saturday|1909Q3|N|Y|N|2418489|2418669|2418133|2418407|N|N|N|N|N| +2418499|AAAAAAAADEHOECAA|1909-07-11|114|497|39|1909|0|7|11|3|1909|39|497|Sunday|1909Q3|N|N|N|2418489|2418669|2418134|2418408|N|N|N|N|N| +2418500|AAAAAAAAEEHOECAA|1909-07-12|114|497|39|1909|1|7|12|3|1909|39|497|Monday|1909Q3|N|N|N|2418489|2418669|2418135|2418409|N|N|N|N|N| +2418501|AAAAAAAAFEHOECAA|1909-07-13|114|498|39|1909|2|7|13|3|1909|39|498|Tuesday|1909Q3|N|N|N|2418489|2418669|2418136|2418410|N|N|N|N|N| +2418502|AAAAAAAAGEHOECAA|1909-07-14|114|498|39|1909|3|7|14|3|1909|39|498|Wednesday|1909Q3|N|N|N|2418489|2418669|2418137|2418411|N|N|N|N|N| +2418503|AAAAAAAAHEHOECAA|1909-07-15|114|498|39|1909|4|7|15|3|1909|39|498|Thursday|1909Q3|N|N|N|2418489|2418669|2418138|2418412|N|N|N|N|N| +2418504|AAAAAAAAIEHOECAA|1909-07-16|114|498|39|1909|5|7|16|3|1909|39|498|Friday|1909Q3|N|Y|N|2418489|2418669|2418139|2418413|N|N|N|N|N| +2418505|AAAAAAAAJEHOECAA|1909-07-17|114|498|39|1909|6|7|17|3|1909|39|498|Saturday|1909Q3|N|Y|N|2418489|2418669|2418140|2418414|N|N|N|N|N| +2418506|AAAAAAAAKEHOECAA|1909-07-18|114|498|39|1909|0|7|18|3|1909|39|498|Sunday|1909Q3|N|N|N|2418489|2418669|2418141|2418415|N|N|N|N|N| +2418507|AAAAAAAALEHOECAA|1909-07-19|114|498|39|1909|1|7|19|3|1909|39|498|Monday|1909Q3|N|N|N|2418489|2418669|2418142|2418416|N|N|N|N|N| +2418508|AAAAAAAAMEHOECAA|1909-07-20|114|499|39|1909|2|7|20|3|1909|39|499|Tuesday|1909Q3|N|N|N|2418489|2418669|2418143|2418417|N|N|N|N|N| +2418509|AAAAAAAANEHOECAA|1909-07-21|114|499|39|1909|3|7|21|3|1909|39|499|Wednesday|1909Q3|N|N|N|2418489|2418669|2418144|2418418|N|N|N|N|N| +2418510|AAAAAAAAOEHOECAA|1909-07-22|114|499|39|1909|4|7|22|3|1909|39|499|Thursday|1909Q3|N|N|N|2418489|2418669|2418145|2418419|N|N|N|N|N| +2418511|AAAAAAAAPEHOECAA|1909-07-23|114|499|39|1909|5|7|23|3|1909|39|499|Friday|1909Q3|N|Y|N|2418489|2418669|2418146|2418420|N|N|N|N|N| +2418512|AAAAAAAAAFHOECAA|1909-07-24|114|499|39|1909|6|7|24|3|1909|39|499|Saturday|1909Q3|N|Y|N|2418489|2418669|2418147|2418421|N|N|N|N|N| +2418513|AAAAAAAABFHOECAA|1909-07-25|114|499|39|1909|0|7|25|3|1909|39|499|Sunday|1909Q3|N|N|N|2418489|2418669|2418148|2418422|N|N|N|N|N| +2418514|AAAAAAAACFHOECAA|1909-07-26|114|499|39|1909|1|7|26|3|1909|39|499|Monday|1909Q3|N|N|N|2418489|2418669|2418149|2418423|N|N|N|N|N| +2418515|AAAAAAAADFHOECAA|1909-07-27|114|500|39|1909|2|7|27|3|1909|39|500|Tuesday|1909Q3|N|N|N|2418489|2418669|2418150|2418424|N|N|N|N|N| +2418516|AAAAAAAAEFHOECAA|1909-07-28|114|500|39|1909|3|7|28|3|1909|39|500|Wednesday|1909Q3|N|N|N|2418489|2418669|2418151|2418425|N|N|N|N|N| +2418517|AAAAAAAAFFHOECAA|1909-07-29|114|500|39|1909|4|7|29|3|1909|39|500|Thursday|1909Q3|N|N|N|2418489|2418669|2418152|2418426|N|N|N|N|N| +2418518|AAAAAAAAGFHOECAA|1909-07-30|114|500|39|1909|5|7|30|3|1909|39|500|Friday|1909Q3|N|Y|N|2418489|2418669|2418153|2418427|N|N|N|N|N| +2418519|AAAAAAAAHFHOECAA|1909-07-31|114|500|39|1909|6|7|31|3|1909|39|500|Saturday|1909Q3|N|Y|N|2418489|2418669|2418154|2418428|N|N|N|N|N| +2418520|AAAAAAAAIFHOECAA|1909-08-01|115|500|39|1909|0|8|1|3|1909|39|500|Sunday|1909Q3|N|N|N|2418520|2418731|2418155|2418429|N|N|N|N|N| +2418521|AAAAAAAAJFHOECAA|1909-08-02|115|500|39|1909|1|8|2|3|1909|39|500|Monday|1909Q3|N|N|N|2418520|2418731|2418156|2418430|N|N|N|N|N| +2418522|AAAAAAAAKFHOECAA|1909-08-03|115|501|39|1909|2|8|3|3|1909|39|501|Tuesday|1909Q3|N|N|N|2418520|2418731|2418157|2418431|N|N|N|N|N| +2418523|AAAAAAAALFHOECAA|1909-08-04|115|501|39|1909|3|8|4|3|1909|39|501|Wednesday|1909Q3|N|N|N|2418520|2418731|2418158|2418432|N|N|N|N|N| +2418524|AAAAAAAAMFHOECAA|1909-08-05|115|501|39|1909|4|8|5|3|1909|39|501|Thursday|1909Q3|N|N|N|2418520|2418731|2418159|2418433|N|N|N|N|N| +2418525|AAAAAAAANFHOECAA|1909-08-06|115|501|39|1909|5|8|6|3|1909|39|501|Friday|1909Q3|N|Y|N|2418520|2418731|2418160|2418434|N|N|N|N|N| +2418526|AAAAAAAAOFHOECAA|1909-08-07|115|501|39|1909|6|8|7|3|1909|39|501|Saturday|1909Q3|N|Y|N|2418520|2418731|2418161|2418435|N|N|N|N|N| +2418527|AAAAAAAAPFHOECAA|1909-08-08|115|501|39|1909|0|8|8|3|1909|39|501|Sunday|1909Q3|N|N|N|2418520|2418731|2418162|2418436|N|N|N|N|N| +2418528|AAAAAAAAAGHOECAA|1909-08-09|115|501|39|1909|1|8|9|3|1909|39|501|Monday|1909Q3|N|N|N|2418520|2418731|2418163|2418437|N|N|N|N|N| +2418529|AAAAAAAABGHOECAA|1909-08-10|115|502|39|1909|2|8|10|3|1909|39|502|Tuesday|1909Q3|N|N|N|2418520|2418731|2418164|2418438|N|N|N|N|N| +2418530|AAAAAAAACGHOECAA|1909-08-11|115|502|39|1909|3|8|11|3|1909|39|502|Wednesday|1909Q3|N|N|N|2418520|2418731|2418165|2418439|N|N|N|N|N| +2418531|AAAAAAAADGHOECAA|1909-08-12|115|502|39|1909|4|8|12|3|1909|39|502|Thursday|1909Q3|N|N|N|2418520|2418731|2418166|2418440|N|N|N|N|N| +2418532|AAAAAAAAEGHOECAA|1909-08-13|115|502|39|1909|5|8|13|3|1909|39|502|Friday|1909Q3|N|Y|N|2418520|2418731|2418167|2418441|N|N|N|N|N| +2418533|AAAAAAAAFGHOECAA|1909-08-14|115|502|39|1909|6|8|14|3|1909|39|502|Saturday|1909Q3|N|Y|N|2418520|2418731|2418168|2418442|N|N|N|N|N| +2418534|AAAAAAAAGGHOECAA|1909-08-15|115|502|39|1909|0|8|15|3|1909|39|502|Sunday|1909Q3|N|N|N|2418520|2418731|2418169|2418443|N|N|N|N|N| +2418535|AAAAAAAAHGHOECAA|1909-08-16|115|502|39|1909|1|8|16|3|1909|39|502|Monday|1909Q3|N|N|N|2418520|2418731|2418170|2418444|N|N|N|N|N| +2418536|AAAAAAAAIGHOECAA|1909-08-17|115|503|39|1909|2|8|17|3|1909|39|503|Tuesday|1909Q3|N|N|N|2418520|2418731|2418171|2418445|N|N|N|N|N| +2418537|AAAAAAAAJGHOECAA|1909-08-18|115|503|39|1909|3|8|18|3|1909|39|503|Wednesday|1909Q3|N|N|N|2418520|2418731|2418172|2418446|N|N|N|N|N| +2418538|AAAAAAAAKGHOECAA|1909-08-19|115|503|39|1909|4|8|19|3|1909|39|503|Thursday|1909Q3|N|N|N|2418520|2418731|2418173|2418447|N|N|N|N|N| +2418539|AAAAAAAALGHOECAA|1909-08-20|115|503|39|1909|5|8|20|3|1909|39|503|Friday|1909Q3|N|Y|N|2418520|2418731|2418174|2418448|N|N|N|N|N| +2418540|AAAAAAAAMGHOECAA|1909-08-21|115|503|39|1909|6|8|21|3|1909|39|503|Saturday|1909Q3|N|Y|N|2418520|2418731|2418175|2418449|N|N|N|N|N| +2418541|AAAAAAAANGHOECAA|1909-08-22|115|503|39|1909|0|8|22|3|1909|39|503|Sunday|1909Q3|N|N|N|2418520|2418731|2418176|2418450|N|N|N|N|N| +2418542|AAAAAAAAOGHOECAA|1909-08-23|115|503|39|1909|1|8|23|3|1909|39|503|Monday|1909Q3|N|N|N|2418520|2418731|2418177|2418451|N|N|N|N|N| +2418543|AAAAAAAAPGHOECAA|1909-08-24|115|504|39|1909|2|8|24|3|1909|39|504|Tuesday|1909Q3|N|N|N|2418520|2418731|2418178|2418452|N|N|N|N|N| +2418544|AAAAAAAAAHHOECAA|1909-08-25|115|504|39|1909|3|8|25|3|1909|39|504|Wednesday|1909Q3|N|N|N|2418520|2418731|2418179|2418453|N|N|N|N|N| +2418545|AAAAAAAABHHOECAA|1909-08-26|115|504|39|1909|4|8|26|3|1909|39|504|Thursday|1909Q3|N|N|N|2418520|2418731|2418180|2418454|N|N|N|N|N| +2418546|AAAAAAAACHHOECAA|1909-08-27|115|504|39|1909|5|8|27|3|1909|39|504|Friday|1909Q3|N|Y|N|2418520|2418731|2418181|2418455|N|N|N|N|N| +2418547|AAAAAAAADHHOECAA|1909-08-28|115|504|39|1909|6|8|28|3|1909|39|504|Saturday|1909Q3|N|Y|N|2418520|2418731|2418182|2418456|N|N|N|N|N| +2418548|AAAAAAAAEHHOECAA|1909-08-29|115|504|39|1909|0|8|29|3|1909|39|504|Sunday|1909Q3|N|N|N|2418520|2418731|2418183|2418457|N|N|N|N|N| +2418549|AAAAAAAAFHHOECAA|1909-08-30|115|504|39|1909|1|8|30|3|1909|39|504|Monday|1909Q3|N|N|N|2418520|2418731|2418184|2418458|N|N|N|N|N| +2418550|AAAAAAAAGHHOECAA|1909-08-31|115|505|39|1909|2|8|31|3|1909|39|505|Tuesday|1909Q3|N|N|N|2418520|2418731|2418185|2418459|N|N|N|N|N| +2418551|AAAAAAAAHHHOECAA|1909-09-01|116|505|40|1909|3|9|1|3|1909|40|505|Wednesday|1909Q3|N|N|N|2418551|2418793|2418186|2418460|N|N|N|N|N| +2418552|AAAAAAAAIHHOECAA|1909-09-02|116|505|40|1909|4|9|2|3|1909|40|505|Thursday|1909Q3|N|N|N|2418551|2418793|2418187|2418461|N|N|N|N|N| +2418553|AAAAAAAAJHHOECAA|1909-09-03|116|505|40|1909|5|9|3|3|1909|40|505|Friday|1909Q3|N|Y|N|2418551|2418793|2418188|2418462|N|N|N|N|N| +2418554|AAAAAAAAKHHOECAA|1909-09-04|116|505|40|1909|6|9|4|3|1909|40|505|Saturday|1909Q3|N|Y|N|2418551|2418793|2418189|2418463|N|N|N|N|N| +2418555|AAAAAAAALHHOECAA|1909-09-05|116|505|40|1909|0|9|5|3|1909|40|505|Sunday|1909Q3|N|N|N|2418551|2418793|2418190|2418464|N|N|N|N|N| +2418556|AAAAAAAAMHHOECAA|1909-09-06|116|505|40|1909|1|9|6|3|1909|40|505|Monday|1909Q3|N|N|N|2418551|2418793|2418191|2418465|N|N|N|N|N| +2418557|AAAAAAAANHHOECAA|1909-09-07|116|506|40|1909|2|9|7|3|1909|40|506|Tuesday|1909Q3|N|N|N|2418551|2418793|2418192|2418466|N|N|N|N|N| +2418558|AAAAAAAAOHHOECAA|1909-09-08|116|506|40|1909|3|9|8|3|1909|40|506|Wednesday|1909Q3|N|N|N|2418551|2418793|2418193|2418467|N|N|N|N|N| +2418559|AAAAAAAAPHHOECAA|1909-09-09|116|506|40|1909|4|9|9|3|1909|40|506|Thursday|1909Q3|N|N|N|2418551|2418793|2418194|2418468|N|N|N|N|N| +2418560|AAAAAAAAAIHOECAA|1909-09-10|116|506|40|1909|5|9|10|3|1909|40|506|Friday|1909Q3|N|Y|N|2418551|2418793|2418195|2418469|N|N|N|N|N| +2418561|AAAAAAAABIHOECAA|1909-09-11|116|506|40|1909|6|9|11|3|1909|40|506|Saturday|1909Q3|N|Y|N|2418551|2418793|2418196|2418470|N|N|N|N|N| +2418562|AAAAAAAACIHOECAA|1909-09-12|116|506|40|1909|0|9|12|3|1909|40|506|Sunday|1909Q3|N|N|N|2418551|2418793|2418197|2418471|N|N|N|N|N| +2418563|AAAAAAAADIHOECAA|1909-09-13|116|506|40|1909|1|9|13|3|1909|40|506|Monday|1909Q3|N|N|N|2418551|2418793|2418198|2418472|N|N|N|N|N| +2418564|AAAAAAAAEIHOECAA|1909-09-14|116|507|40|1909|2|9|14|3|1909|40|507|Tuesday|1909Q3|N|N|N|2418551|2418793|2418199|2418473|N|N|N|N|N| +2418565|AAAAAAAAFIHOECAA|1909-09-15|116|507|40|1909|3|9|15|3|1909|40|507|Wednesday|1909Q3|N|N|N|2418551|2418793|2418200|2418474|N|N|N|N|N| +2418566|AAAAAAAAGIHOECAA|1909-09-16|116|507|40|1909|4|9|16|3|1909|40|507|Thursday|1909Q3|N|N|N|2418551|2418793|2418201|2418475|N|N|N|N|N| +2418567|AAAAAAAAHIHOECAA|1909-09-17|116|507|40|1909|5|9|17|3|1909|40|507|Friday|1909Q3|N|Y|N|2418551|2418793|2418202|2418476|N|N|N|N|N| +2418568|AAAAAAAAIIHOECAA|1909-09-18|116|507|40|1909|6|9|18|3|1909|40|507|Saturday|1909Q3|N|Y|N|2418551|2418793|2418203|2418477|N|N|N|N|N| +2418569|AAAAAAAAJIHOECAA|1909-09-19|116|507|40|1909|0|9|19|3|1909|40|507|Sunday|1909Q3|N|N|N|2418551|2418793|2418204|2418478|N|N|N|N|N| +2418570|AAAAAAAAKIHOECAA|1909-09-20|116|507|40|1909|1|9|20|3|1909|40|507|Monday|1909Q3|N|N|N|2418551|2418793|2418205|2418479|N|N|N|N|N| +2418571|AAAAAAAALIHOECAA|1909-09-21|116|508|40|1909|2|9|21|3|1909|40|508|Tuesday|1909Q3|N|N|N|2418551|2418793|2418206|2418480|N|N|N|N|N| +2418572|AAAAAAAAMIHOECAA|1909-09-22|116|508|40|1909|3|9|22|3|1909|40|508|Wednesday|1909Q3|N|N|N|2418551|2418793|2418207|2418481|N|N|N|N|N| +2418573|AAAAAAAANIHOECAA|1909-09-23|116|508|40|1909|4|9|23|3|1909|40|508|Thursday|1909Q3|N|N|N|2418551|2418793|2418208|2418482|N|N|N|N|N| +2418574|AAAAAAAAOIHOECAA|1909-09-24|116|508|40|1909|5|9|24|3|1909|40|508|Friday|1909Q3|N|Y|N|2418551|2418793|2418209|2418483|N|N|N|N|N| +2418575|AAAAAAAAPIHOECAA|1909-09-25|116|508|40|1909|6|9|25|3|1909|40|508|Saturday|1909Q3|N|Y|N|2418551|2418793|2418210|2418484|N|N|N|N|N| +2418576|AAAAAAAAAJHOECAA|1909-09-26|116|508|40|1909|0|9|26|3|1909|40|508|Sunday|1909Q3|N|N|N|2418551|2418793|2418211|2418485|N|N|N|N|N| +2418577|AAAAAAAABJHOECAA|1909-09-27|116|508|40|1909|1|9|27|3|1909|40|508|Monday|1909Q3|N|N|N|2418551|2418793|2418212|2418486|N|N|N|N|N| +2418578|AAAAAAAACJHOECAA|1909-09-28|116|509|40|1909|2|9|28|3|1909|40|509|Tuesday|1909Q3|N|N|N|2418551|2418793|2418213|2418487|N|N|N|N|N| +2418579|AAAAAAAADJHOECAA|1909-09-29|116|509|40|1909|3|9|29|3|1909|40|509|Wednesday|1909Q3|N|N|N|2418551|2418793|2418214|2418488|N|N|N|N|N| +2418580|AAAAAAAAEJHOECAA|1909-09-30|116|509|40|1909|4|9|30|3|1909|40|509|Thursday|1909Q3|N|N|N|2418551|2418793|2418215|2418489|N|N|N|N|N| +2418581|AAAAAAAAFJHOECAA|1909-10-01|117|509|40|1909|5|10|1|3|1909|40|509|Friday|1909Q3|N|Y|N|2418581|2418853|2418216|2418489|N|N|N|N|N| +2418582|AAAAAAAAGJHOECAA|1909-10-02|117|509|40|1909|6|10|2|4|1909|40|509|Saturday|1909Q4|N|Y|N|2418581|2418853|2418217|2418490|N|N|N|N|N| +2418583|AAAAAAAAHJHOECAA|1909-10-03|117|509|40|1909|0|10|3|4|1909|40|509|Sunday|1909Q4|N|N|N|2418581|2418853|2418218|2418491|N|N|N|N|N| +2418584|AAAAAAAAIJHOECAA|1909-10-04|117|509|40|1909|1|10|4|4|1909|40|509|Monday|1909Q4|N|N|N|2418581|2418853|2418219|2418492|N|N|N|N|N| +2418585|AAAAAAAAJJHOECAA|1909-10-05|117|510|40|1909|2|10|5|4|1909|40|510|Tuesday|1909Q4|N|N|N|2418581|2418853|2418220|2418493|N|N|N|N|N| +2418586|AAAAAAAAKJHOECAA|1909-10-06|117|510|40|1909|3|10|6|4|1909|40|510|Wednesday|1909Q4|N|N|N|2418581|2418853|2418221|2418494|N|N|N|N|N| +2418587|AAAAAAAALJHOECAA|1909-10-07|117|510|40|1909|4|10|7|4|1909|40|510|Thursday|1909Q4|N|N|N|2418581|2418853|2418222|2418495|N|N|N|N|N| +2418588|AAAAAAAAMJHOECAA|1909-10-08|117|510|40|1909|5|10|8|4|1909|40|510|Friday|1909Q4|N|Y|N|2418581|2418853|2418223|2418496|N|N|N|N|N| +2418589|AAAAAAAANJHOECAA|1909-10-09|117|510|40|1909|6|10|9|4|1909|40|510|Saturday|1909Q4|N|Y|N|2418581|2418853|2418224|2418497|N|N|N|N|N| +2418590|AAAAAAAAOJHOECAA|1909-10-10|117|510|40|1909|0|10|10|4|1909|40|510|Sunday|1909Q4|N|N|N|2418581|2418853|2418225|2418498|N|N|N|N|N| +2418591|AAAAAAAAPJHOECAA|1909-10-11|117|510|40|1909|1|10|11|4|1909|40|510|Monday|1909Q4|N|N|N|2418581|2418853|2418226|2418499|N|N|N|N|N| +2418592|AAAAAAAAAKHOECAA|1909-10-12|117|511|40|1909|2|10|12|4|1909|40|511|Tuesday|1909Q4|N|N|N|2418581|2418853|2418227|2418500|N|N|N|N|N| +2418593|AAAAAAAABKHOECAA|1909-10-13|117|511|40|1909|3|10|13|4|1909|40|511|Wednesday|1909Q4|N|N|N|2418581|2418853|2418228|2418501|N|N|N|N|N| +2418594|AAAAAAAACKHOECAA|1909-10-14|117|511|40|1909|4|10|14|4|1909|40|511|Thursday|1909Q4|N|N|N|2418581|2418853|2418229|2418502|N|N|N|N|N| +2418595|AAAAAAAADKHOECAA|1909-10-15|117|511|40|1909|5|10|15|4|1909|40|511|Friday|1909Q4|N|Y|N|2418581|2418853|2418230|2418503|N|N|N|N|N| +2418596|AAAAAAAAEKHOECAA|1909-10-16|117|511|40|1909|6|10|16|4|1909|40|511|Saturday|1909Q4|N|Y|N|2418581|2418853|2418231|2418504|N|N|N|N|N| +2418597|AAAAAAAAFKHOECAA|1909-10-17|117|511|40|1909|0|10|17|4|1909|40|511|Sunday|1909Q4|N|N|N|2418581|2418853|2418232|2418505|N|N|N|N|N| +2418598|AAAAAAAAGKHOECAA|1909-10-18|117|511|40|1909|1|10|18|4|1909|40|511|Monday|1909Q4|N|N|N|2418581|2418853|2418233|2418506|N|N|N|N|N| +2418599|AAAAAAAAHKHOECAA|1909-10-19|117|512|40|1909|2|10|19|4|1909|40|512|Tuesday|1909Q4|N|N|N|2418581|2418853|2418234|2418507|N|N|N|N|N| +2418600|AAAAAAAAIKHOECAA|1909-10-20|117|512|40|1909|3|10|20|4|1909|40|512|Wednesday|1909Q4|N|N|N|2418581|2418853|2418235|2418508|N|N|N|N|N| +2418601|AAAAAAAAJKHOECAA|1909-10-21|117|512|40|1909|4|10|21|4|1909|40|512|Thursday|1909Q4|N|N|N|2418581|2418853|2418236|2418509|N|N|N|N|N| +2418602|AAAAAAAAKKHOECAA|1909-10-22|117|512|40|1909|5|10|22|4|1909|40|512|Friday|1909Q4|N|Y|N|2418581|2418853|2418237|2418510|N|N|N|N|N| +2418603|AAAAAAAALKHOECAA|1909-10-23|117|512|40|1909|6|10|23|4|1909|40|512|Saturday|1909Q4|N|Y|N|2418581|2418853|2418238|2418511|N|N|N|N|N| +2418604|AAAAAAAAMKHOECAA|1909-10-24|117|512|40|1909|0|10|24|4|1909|40|512|Sunday|1909Q4|N|N|N|2418581|2418853|2418239|2418512|N|N|N|N|N| +2418605|AAAAAAAANKHOECAA|1909-10-25|117|512|40|1909|1|10|25|4|1909|40|512|Monday|1909Q4|N|N|N|2418581|2418853|2418240|2418513|N|N|N|N|N| +2418606|AAAAAAAAOKHOECAA|1909-10-26|117|513|40|1909|2|10|26|4|1909|40|513|Tuesday|1909Q4|N|N|N|2418581|2418853|2418241|2418514|N|N|N|N|N| +2418607|AAAAAAAAPKHOECAA|1909-10-27|117|513|40|1909|3|10|27|4|1909|40|513|Wednesday|1909Q4|N|N|N|2418581|2418853|2418242|2418515|N|N|N|N|N| +2418608|AAAAAAAAALHOECAA|1909-10-28|117|513|40|1909|4|10|28|4|1909|40|513|Thursday|1909Q4|N|N|N|2418581|2418853|2418243|2418516|N|N|N|N|N| +2418609|AAAAAAAABLHOECAA|1909-10-29|117|513|40|1909|5|10|29|4|1909|40|513|Friday|1909Q4|N|Y|N|2418581|2418853|2418244|2418517|N|N|N|N|N| +2418610|AAAAAAAACLHOECAA|1909-10-30|117|513|40|1909|6|10|30|4|1909|40|513|Saturday|1909Q4|N|Y|N|2418581|2418853|2418245|2418518|N|N|N|N|N| +2418611|AAAAAAAADLHOECAA|1909-10-31|117|513|40|1909|0|10|31|4|1909|40|513|Sunday|1909Q4|N|N|N|2418581|2418853|2418246|2418519|N|N|N|N|N| +2418612|AAAAAAAAELHOECAA|1909-11-01|118|513|40|1909|1|11|1|4|1909|40|513|Monday|1909Q4|N|N|N|2418612|2418915|2418247|2418520|N|N|N|N|N| +2418613|AAAAAAAAFLHOECAA|1909-11-02|118|514|40|1909|2|11|2|4|1909|40|514|Tuesday|1909Q4|N|N|N|2418612|2418915|2418248|2418521|N|N|N|N|N| +2418614|AAAAAAAAGLHOECAA|1909-11-03|118|514|40|1909|3|11|3|4|1909|40|514|Wednesday|1909Q4|N|N|N|2418612|2418915|2418249|2418522|N|N|N|N|N| +2418615|AAAAAAAAHLHOECAA|1909-11-04|118|514|40|1909|4|11|4|4|1909|40|514|Thursday|1909Q4|N|N|N|2418612|2418915|2418250|2418523|N|N|N|N|N| +2418616|AAAAAAAAILHOECAA|1909-11-05|118|514|40|1909|5|11|5|4|1909|40|514|Friday|1909Q4|N|Y|N|2418612|2418915|2418251|2418524|N|N|N|N|N| +2418617|AAAAAAAAJLHOECAA|1909-11-06|118|514|40|1909|6|11|6|4|1909|40|514|Saturday|1909Q4|N|Y|N|2418612|2418915|2418252|2418525|N|N|N|N|N| +2418618|AAAAAAAAKLHOECAA|1909-11-07|118|514|40|1909|0|11|7|4|1909|40|514|Sunday|1909Q4|N|N|N|2418612|2418915|2418253|2418526|N|N|N|N|N| +2418619|AAAAAAAALLHOECAA|1909-11-08|118|514|40|1909|1|11|8|4|1909|40|514|Monday|1909Q4|N|N|N|2418612|2418915|2418254|2418527|N|N|N|N|N| +2418620|AAAAAAAAMLHOECAA|1909-11-09|118|515|40|1909|2|11|9|4|1909|40|515|Tuesday|1909Q4|N|N|N|2418612|2418915|2418255|2418528|N|N|N|N|N| +2418621|AAAAAAAANLHOECAA|1909-11-10|118|515|40|1909|3|11|10|4|1909|40|515|Wednesday|1909Q4|N|N|N|2418612|2418915|2418256|2418529|N|N|N|N|N| +2418622|AAAAAAAAOLHOECAA|1909-11-11|118|515|40|1909|4|11|11|4|1909|40|515|Thursday|1909Q4|N|N|N|2418612|2418915|2418257|2418530|N|N|N|N|N| +2418623|AAAAAAAAPLHOECAA|1909-11-12|118|515|40|1909|5|11|12|4|1909|40|515|Friday|1909Q4|N|Y|N|2418612|2418915|2418258|2418531|N|N|N|N|N| +2418624|AAAAAAAAAMHOECAA|1909-11-13|118|515|40|1909|6|11|13|4|1909|40|515|Saturday|1909Q4|N|Y|N|2418612|2418915|2418259|2418532|N|N|N|N|N| +2418625|AAAAAAAABMHOECAA|1909-11-14|118|515|40|1909|0|11|14|4|1909|40|515|Sunday|1909Q4|N|N|N|2418612|2418915|2418260|2418533|N|N|N|N|N| +2418626|AAAAAAAACMHOECAA|1909-11-15|118|515|40|1909|1|11|15|4|1909|40|515|Monday|1909Q4|N|N|N|2418612|2418915|2418261|2418534|N|N|N|N|N| +2418627|AAAAAAAADMHOECAA|1909-11-16|118|516|40|1909|2|11|16|4|1909|40|516|Tuesday|1909Q4|N|N|N|2418612|2418915|2418262|2418535|N|N|N|N|N| +2418628|AAAAAAAAEMHOECAA|1909-11-17|118|516|40|1909|3|11|17|4|1909|40|516|Wednesday|1909Q4|N|N|N|2418612|2418915|2418263|2418536|N|N|N|N|N| +2418629|AAAAAAAAFMHOECAA|1909-11-18|118|516|40|1909|4|11|18|4|1909|40|516|Thursday|1909Q4|N|N|N|2418612|2418915|2418264|2418537|N|N|N|N|N| +2418630|AAAAAAAAGMHOECAA|1909-11-19|118|516|40|1909|5|11|19|4|1909|40|516|Friday|1909Q4|N|Y|N|2418612|2418915|2418265|2418538|N|N|N|N|N| +2418631|AAAAAAAAHMHOECAA|1909-11-20|118|516|40|1909|6|11|20|4|1909|40|516|Saturday|1909Q4|N|Y|N|2418612|2418915|2418266|2418539|N|N|N|N|N| +2418632|AAAAAAAAIMHOECAA|1909-11-21|118|516|40|1909|0|11|21|4|1909|40|516|Sunday|1909Q4|N|N|N|2418612|2418915|2418267|2418540|N|N|N|N|N| +2418633|AAAAAAAAJMHOECAA|1909-11-22|118|516|40|1909|1|11|22|4|1909|40|516|Monday|1909Q4|N|N|N|2418612|2418915|2418268|2418541|N|N|N|N|N| +2418634|AAAAAAAAKMHOECAA|1909-11-23|118|517|40|1909|2|11|23|4|1909|40|517|Tuesday|1909Q4|N|N|N|2418612|2418915|2418269|2418542|N|N|N|N|N| +2418635|AAAAAAAALMHOECAA|1909-11-24|118|517|40|1909|3|11|24|4|1909|40|517|Wednesday|1909Q4|N|N|N|2418612|2418915|2418270|2418543|N|N|N|N|N| +2418636|AAAAAAAAMMHOECAA|1909-11-25|118|517|40|1909|4|11|25|4|1909|40|517|Thursday|1909Q4|N|N|N|2418612|2418915|2418271|2418544|N|N|N|N|N| +2418637|AAAAAAAANMHOECAA|1909-11-26|118|517|40|1909|5|11|26|4|1909|40|517|Friday|1909Q4|N|Y|N|2418612|2418915|2418272|2418545|N|N|N|N|N| +2418638|AAAAAAAAOMHOECAA|1909-11-27|118|517|40|1909|6|11|27|4|1909|40|517|Saturday|1909Q4|N|Y|N|2418612|2418915|2418273|2418546|N|N|N|N|N| +2418639|AAAAAAAAPMHOECAA|1909-11-28|118|517|40|1909|0|11|28|4|1909|40|517|Sunday|1909Q4|N|N|N|2418612|2418915|2418274|2418547|N|N|N|N|N| +2418640|AAAAAAAAANHOECAA|1909-11-29|118|517|40|1909|1|11|29|4|1909|40|517|Monday|1909Q4|N|N|N|2418612|2418915|2418275|2418548|N|N|N|N|N| +2418641|AAAAAAAABNHOECAA|1909-11-30|118|518|40|1909|2|11|30|4|1909|40|518|Tuesday|1909Q4|N|N|N|2418612|2418915|2418276|2418549|N|N|N|N|N| +2418642|AAAAAAAACNHOECAA|1909-12-01|119|518|41|1909|3|12|1|4|1909|41|518|Wednesday|1909Q4|N|N|N|2418642|2418975|2418277|2418550|N|N|N|N|N| +2418643|AAAAAAAADNHOECAA|1909-12-02|119|518|41|1909|4|12|2|4|1909|41|518|Thursday|1909Q4|N|N|N|2418642|2418975|2418278|2418551|N|N|N|N|N| +2418644|AAAAAAAAENHOECAA|1909-12-03|119|518|41|1909|5|12|3|4|1909|41|518|Friday|1909Q4|N|Y|N|2418642|2418975|2418279|2418552|N|N|N|N|N| +2418645|AAAAAAAAFNHOECAA|1909-12-04|119|518|41|1909|6|12|4|4|1909|41|518|Saturday|1909Q4|N|Y|N|2418642|2418975|2418280|2418553|N|N|N|N|N| +2418646|AAAAAAAAGNHOECAA|1909-12-05|119|518|41|1909|0|12|5|4|1909|41|518|Sunday|1909Q4|N|N|N|2418642|2418975|2418281|2418554|N|N|N|N|N| +2418647|AAAAAAAAHNHOECAA|1909-12-06|119|518|41|1909|1|12|6|4|1909|41|518|Monday|1909Q4|N|N|N|2418642|2418975|2418282|2418555|N|N|N|N|N| +2418648|AAAAAAAAINHOECAA|1909-12-07|119|519|41|1909|2|12|7|4|1909|41|519|Tuesday|1909Q4|N|N|N|2418642|2418975|2418283|2418556|N|N|N|N|N| +2418649|AAAAAAAAJNHOECAA|1909-12-08|119|519|41|1909|3|12|8|4|1909|41|519|Wednesday|1909Q4|N|N|N|2418642|2418975|2418284|2418557|N|N|N|N|N| +2418650|AAAAAAAAKNHOECAA|1909-12-09|119|519|41|1909|4|12|9|4|1909|41|519|Thursday|1909Q4|N|N|N|2418642|2418975|2418285|2418558|N|N|N|N|N| +2418651|AAAAAAAALNHOECAA|1909-12-10|119|519|41|1909|5|12|10|4|1909|41|519|Friday|1909Q4|N|Y|N|2418642|2418975|2418286|2418559|N|N|N|N|N| +2418652|AAAAAAAAMNHOECAA|1909-12-11|119|519|41|1909|6|12|11|4|1909|41|519|Saturday|1909Q4|N|Y|N|2418642|2418975|2418287|2418560|N|N|N|N|N| +2418653|AAAAAAAANNHOECAA|1909-12-12|119|519|41|1909|0|12|12|4|1909|41|519|Sunday|1909Q4|N|N|N|2418642|2418975|2418288|2418561|N|N|N|N|N| +2418654|AAAAAAAAONHOECAA|1909-12-13|119|519|41|1909|1|12|13|4|1909|41|519|Monday|1909Q4|N|N|N|2418642|2418975|2418289|2418562|N|N|N|N|N| +2418655|AAAAAAAAPNHOECAA|1909-12-14|119|520|41|1909|2|12|14|4|1909|41|520|Tuesday|1909Q4|N|N|N|2418642|2418975|2418290|2418563|N|N|N|N|N| +2418656|AAAAAAAAAOHOECAA|1909-12-15|119|520|41|1909|3|12|15|4|1909|41|520|Wednesday|1909Q4|N|N|N|2418642|2418975|2418291|2418564|N|N|N|N|N| +2418657|AAAAAAAABOHOECAA|1909-12-16|119|520|41|1909|4|12|16|4|1909|41|520|Thursday|1909Q4|N|N|N|2418642|2418975|2418292|2418565|N|N|N|N|N| +2418658|AAAAAAAACOHOECAA|1909-12-17|119|520|41|1909|5|12|17|4|1909|41|520|Friday|1909Q4|N|Y|N|2418642|2418975|2418293|2418566|N|N|N|N|N| +2418659|AAAAAAAADOHOECAA|1909-12-18|119|520|41|1909|6|12|18|4|1909|41|520|Saturday|1909Q4|N|Y|N|2418642|2418975|2418294|2418567|N|N|N|N|N| +2418660|AAAAAAAAEOHOECAA|1909-12-19|119|520|41|1909|0|12|19|4|1909|41|520|Sunday|1909Q4|N|N|N|2418642|2418975|2418295|2418568|N|N|N|N|N| +2418661|AAAAAAAAFOHOECAA|1909-12-20|119|520|41|1909|1|12|20|4|1909|41|520|Monday|1909Q4|N|N|N|2418642|2418975|2418296|2418569|N|N|N|N|N| +2418662|AAAAAAAAGOHOECAA|1909-12-21|119|521|41|1909|2|12|21|4|1909|41|521|Tuesday|1909Q4|N|N|N|2418642|2418975|2418297|2418570|N|N|N|N|N| +2418663|AAAAAAAAHOHOECAA|1909-12-22|119|521|41|1909|3|12|22|4|1909|41|521|Wednesday|1909Q4|N|N|N|2418642|2418975|2418298|2418571|N|N|N|N|N| +2418664|AAAAAAAAIOHOECAA|1909-12-23|119|521|41|1909|4|12|23|4|1909|41|521|Thursday|1909Q4|N|N|N|2418642|2418975|2418299|2418572|N|N|N|N|N| +2418665|AAAAAAAAJOHOECAA|1909-12-24|119|521|41|1909|5|12|24|4|1909|41|521|Friday|1909Q4|N|Y|N|2418642|2418975|2418300|2418573|N|N|N|N|N| +2418666|AAAAAAAAKOHOECAA|1909-12-25|119|521|41|1909|6|12|25|4|1909|41|521|Saturday|1909Q4|N|Y|N|2418642|2418975|2418301|2418574|N|N|N|N|N| +2418667|AAAAAAAALOHOECAA|1909-12-26|119|521|41|1909|0|12|26|4|1909|41|521|Sunday|1909Q4|Y|N|N|2418642|2418975|2418302|2418575|N|N|N|N|N| +2418668|AAAAAAAAMOHOECAA|1909-12-27|119|521|41|1909|1|12|27|4|1909|41|521|Monday|1909Q4|N|N|Y|2418642|2418975|2418303|2418576|N|N|N|N|N| +2418669|AAAAAAAANOHOECAA|1909-12-28|119|522|41|1909|2|12|28|4|1909|41|522|Tuesday|1909Q4|N|N|N|2418642|2418975|2418304|2418577|N|N|N|N|N| +2418670|AAAAAAAAOOHOECAA|1909-12-29|119|522|41|1909|3|12|29|4|1909|41|522|Wednesday|1909Q4|N|N|N|2418642|2418975|2418305|2418578|N|N|N|N|N| +2418671|AAAAAAAAPOHOECAA|1909-12-30|119|522|41|1909|4|12|30|4|1909|41|522|Thursday|1909Q4|N|N|N|2418642|2418975|2418306|2418579|N|N|N|N|N| +2418672|AAAAAAAAAPHOECAA|1909-12-31|119|522|41|1909|5|12|31|4|1909|41|522|Friday|1909Q4|N|Y|N|2418642|2418975|2418307|2418580|N|N|N|N|N| +2418673|AAAAAAAABPHOECAA|1910-01-01|120|522|41|1910|6|1|1|1|1910|41|522|Saturday|1910Q1|Y|Y|N|2418673|2418672|2418308|2418581|N|N|N|N|N| +2418674|AAAAAAAACPHOECAA|1910-01-02|120|522|41|1910|0|1|2|1|1910|41|522|Sunday|1910Q1|N|N|Y|2418673|2418672|2418309|2418582|N|N|N|N|N| +2418675|AAAAAAAADPHOECAA|1910-01-03|120|522|41|1910|1|1|3|1|1910|41|522|Monday|1910Q1|N|N|N|2418673|2418672|2418310|2418583|N|N|N|N|N| +2418676|AAAAAAAAEPHOECAA|1910-01-04|120|523|41|1910|2|1|4|1|1910|41|523|Tuesday|1910Q1|N|N|N|2418673|2418672|2418311|2418584|N|N|N|N|N| +2418677|AAAAAAAAFPHOECAA|1910-01-05|120|523|41|1910|3|1|5|1|1910|41|523|Wednesday|1910Q1|N|N|N|2418673|2418672|2418312|2418585|N|N|N|N|N| +2418678|AAAAAAAAGPHOECAA|1910-01-06|120|523|41|1910|4|1|6|1|1910|41|523|Thursday|1910Q1|N|N|N|2418673|2418672|2418313|2418586|N|N|N|N|N| +2418679|AAAAAAAAHPHOECAA|1910-01-07|120|523|41|1910|5|1|7|1|1910|41|523|Friday|1910Q1|N|Y|N|2418673|2418672|2418314|2418587|N|N|N|N|N| +2418680|AAAAAAAAIPHOECAA|1910-01-08|120|523|41|1910|6|1|8|1|1910|41|523|Saturday|1910Q1|N|Y|N|2418673|2418672|2418315|2418588|N|N|N|N|N| +2418681|AAAAAAAAJPHOECAA|1910-01-09|120|523|41|1910|0|1|9|1|1910|41|523|Sunday|1910Q1|N|N|N|2418673|2418672|2418316|2418589|N|N|N|N|N| +2418682|AAAAAAAAKPHOECAA|1910-01-10|120|523|41|1910|1|1|10|1|1910|41|523|Monday|1910Q1|N|N|N|2418673|2418672|2418317|2418590|N|N|N|N|N| +2418683|AAAAAAAALPHOECAA|1910-01-11|120|524|41|1910|2|1|11|1|1910|41|524|Tuesday|1910Q1|N|N|N|2418673|2418672|2418318|2418591|N|N|N|N|N| +2418684|AAAAAAAAMPHOECAA|1910-01-12|120|524|41|1910|3|1|12|1|1910|41|524|Wednesday|1910Q1|N|N|N|2418673|2418672|2418319|2418592|N|N|N|N|N| +2418685|AAAAAAAANPHOECAA|1910-01-13|120|524|41|1910|4|1|13|1|1910|41|524|Thursday|1910Q1|N|N|N|2418673|2418672|2418320|2418593|N|N|N|N|N| +2418686|AAAAAAAAOPHOECAA|1910-01-14|120|524|41|1910|5|1|14|1|1910|41|524|Friday|1910Q1|N|Y|N|2418673|2418672|2418321|2418594|N|N|N|N|N| +2418687|AAAAAAAAPPHOECAA|1910-01-15|120|524|41|1910|6|1|15|1|1910|41|524|Saturday|1910Q1|N|Y|N|2418673|2418672|2418322|2418595|N|N|N|N|N| +2418688|AAAAAAAAAAIOECAA|1910-01-16|120|524|41|1910|0|1|16|1|1910|41|524|Sunday|1910Q1|N|N|N|2418673|2418672|2418323|2418596|N|N|N|N|N| +2418689|AAAAAAAABAIOECAA|1910-01-17|120|524|41|1910|1|1|17|1|1910|41|524|Monday|1910Q1|N|N|N|2418673|2418672|2418324|2418597|N|N|N|N|N| +2418690|AAAAAAAACAIOECAA|1910-01-18|120|525|41|1910|2|1|18|1|1910|41|525|Tuesday|1910Q1|N|N|N|2418673|2418672|2418325|2418598|N|N|N|N|N| +2418691|AAAAAAAADAIOECAA|1910-01-19|120|525|41|1910|3|1|19|1|1910|41|525|Wednesday|1910Q1|N|N|N|2418673|2418672|2418326|2418599|N|N|N|N|N| +2418692|AAAAAAAAEAIOECAA|1910-01-20|120|525|41|1910|4|1|20|1|1910|41|525|Thursday|1910Q1|N|N|N|2418673|2418672|2418327|2418600|N|N|N|N|N| +2418693|AAAAAAAAFAIOECAA|1910-01-21|120|525|41|1910|5|1|21|1|1910|41|525|Friday|1910Q1|N|Y|N|2418673|2418672|2418328|2418601|N|N|N|N|N| +2418694|AAAAAAAAGAIOECAA|1910-01-22|120|525|41|1910|6|1|22|1|1910|41|525|Saturday|1910Q1|N|Y|N|2418673|2418672|2418329|2418602|N|N|N|N|N| +2418695|AAAAAAAAHAIOECAA|1910-01-23|120|525|41|1910|0|1|23|1|1910|41|525|Sunday|1910Q1|N|N|N|2418673|2418672|2418330|2418603|N|N|N|N|N| +2418696|AAAAAAAAIAIOECAA|1910-01-24|120|525|41|1910|1|1|24|1|1910|41|525|Monday|1910Q1|N|N|N|2418673|2418672|2418331|2418604|N|N|N|N|N| +2418697|AAAAAAAAJAIOECAA|1910-01-25|120|526|41|1910|2|1|25|1|1910|41|526|Tuesday|1910Q1|N|N|N|2418673|2418672|2418332|2418605|N|N|N|N|N| +2418698|AAAAAAAAKAIOECAA|1910-01-26|120|526|41|1910|3|1|26|1|1910|41|526|Wednesday|1910Q1|N|N|N|2418673|2418672|2418333|2418606|N|N|N|N|N| +2418699|AAAAAAAALAIOECAA|1910-01-27|120|526|41|1910|4|1|27|1|1910|41|526|Thursday|1910Q1|N|N|N|2418673|2418672|2418334|2418607|N|N|N|N|N| +2418700|AAAAAAAAMAIOECAA|1910-01-28|120|526|41|1910|5|1|28|1|1910|41|526|Friday|1910Q1|N|Y|N|2418673|2418672|2418335|2418608|N|N|N|N|N| +2418701|AAAAAAAANAIOECAA|1910-01-29|120|526|41|1910|6|1|29|1|1910|41|526|Saturday|1910Q1|N|Y|N|2418673|2418672|2418336|2418609|N|N|N|N|N| +2418702|AAAAAAAAOAIOECAA|1910-01-30|120|526|41|1910|0|1|30|1|1910|41|526|Sunday|1910Q1|N|N|N|2418673|2418672|2418337|2418610|N|N|N|N|N| +2418703|AAAAAAAAPAIOECAA|1910-01-31|120|526|41|1910|1|1|31|1|1910|41|526|Monday|1910Q1|N|N|N|2418673|2418672|2418338|2418611|N|N|N|N|N| +2418704|AAAAAAAAABIOECAA|1910-02-01|121|527|41|1910|2|2|1|1|1910|41|527|Tuesday|1910Q1|N|N|N|2418704|2418734|2418339|2418612|N|N|N|N|N| +2418705|AAAAAAAABBIOECAA|1910-02-02|121|527|41|1910|3|2|2|1|1910|41|527|Wednesday|1910Q1|N|N|N|2418704|2418734|2418340|2418613|N|N|N|N|N| +2418706|AAAAAAAACBIOECAA|1910-02-03|121|527|41|1910|4|2|3|1|1910|41|527|Thursday|1910Q1|N|N|N|2418704|2418734|2418341|2418614|N|N|N|N|N| +2418707|AAAAAAAADBIOECAA|1910-02-04|121|527|41|1910|5|2|4|1|1910|41|527|Friday|1910Q1|N|Y|N|2418704|2418734|2418342|2418615|N|N|N|N|N| +2418708|AAAAAAAAEBIOECAA|1910-02-05|121|527|41|1910|6|2|5|1|1910|41|527|Saturday|1910Q1|N|Y|N|2418704|2418734|2418343|2418616|N|N|N|N|N| +2418709|AAAAAAAAFBIOECAA|1910-02-06|121|527|41|1910|0|2|6|1|1910|41|527|Sunday|1910Q1|N|N|N|2418704|2418734|2418344|2418617|N|N|N|N|N| +2418710|AAAAAAAAGBIOECAA|1910-02-07|121|527|41|1910|1|2|7|1|1910|41|527|Monday|1910Q1|N|N|N|2418704|2418734|2418345|2418618|N|N|N|N|N| +2418711|AAAAAAAAHBIOECAA|1910-02-08|121|528|41|1910|2|2|8|1|1910|41|528|Tuesday|1910Q1|N|N|N|2418704|2418734|2418346|2418619|N|N|N|N|N| +2418712|AAAAAAAAIBIOECAA|1910-02-09|121|528|41|1910|3|2|9|1|1910|41|528|Wednesday|1910Q1|N|N|N|2418704|2418734|2418347|2418620|N|N|N|N|N| +2418713|AAAAAAAAJBIOECAA|1910-02-10|121|528|41|1910|4|2|10|1|1910|41|528|Thursday|1910Q1|N|N|N|2418704|2418734|2418348|2418621|N|N|N|N|N| +2418714|AAAAAAAAKBIOECAA|1910-02-11|121|528|41|1910|5|2|11|1|1910|41|528|Friday|1910Q1|N|Y|N|2418704|2418734|2418349|2418622|N|N|N|N|N| +2418715|AAAAAAAALBIOECAA|1910-02-12|121|528|41|1910|6|2|12|1|1910|41|528|Saturday|1910Q1|N|Y|N|2418704|2418734|2418350|2418623|N|N|N|N|N| +2418716|AAAAAAAAMBIOECAA|1910-02-13|121|528|41|1910|0|2|13|1|1910|41|528|Sunday|1910Q1|N|N|N|2418704|2418734|2418351|2418624|N|N|N|N|N| +2418717|AAAAAAAANBIOECAA|1910-02-14|121|528|41|1910|1|2|14|1|1910|41|528|Monday|1910Q1|N|N|N|2418704|2418734|2418352|2418625|N|N|N|N|N| +2418718|AAAAAAAAOBIOECAA|1910-02-15|121|529|41|1910|2|2|15|1|1910|41|529|Tuesday|1910Q1|N|N|N|2418704|2418734|2418353|2418626|N|N|N|N|N| +2418719|AAAAAAAAPBIOECAA|1910-02-16|121|529|41|1910|3|2|16|1|1910|41|529|Wednesday|1910Q1|N|N|N|2418704|2418734|2418354|2418627|N|N|N|N|N| +2418720|AAAAAAAAACIOECAA|1910-02-17|121|529|41|1910|4|2|17|1|1910|41|529|Thursday|1910Q1|N|N|N|2418704|2418734|2418355|2418628|N|N|N|N|N| +2418721|AAAAAAAABCIOECAA|1910-02-18|121|529|41|1910|5|2|18|1|1910|41|529|Friday|1910Q1|N|Y|N|2418704|2418734|2418356|2418629|N|N|N|N|N| +2418722|AAAAAAAACCIOECAA|1910-02-19|121|529|41|1910|6|2|19|1|1910|41|529|Saturday|1910Q1|N|Y|N|2418704|2418734|2418357|2418630|N|N|N|N|N| +2418723|AAAAAAAADCIOECAA|1910-02-20|121|529|41|1910|0|2|20|1|1910|41|529|Sunday|1910Q1|N|N|N|2418704|2418734|2418358|2418631|N|N|N|N|N| +2418724|AAAAAAAAECIOECAA|1910-02-21|121|529|41|1910|1|2|21|1|1910|41|529|Monday|1910Q1|N|N|N|2418704|2418734|2418359|2418632|N|N|N|N|N| +2418725|AAAAAAAAFCIOECAA|1910-02-22|121|530|41|1910|2|2|22|1|1910|41|530|Tuesday|1910Q1|N|N|N|2418704|2418734|2418360|2418633|N|N|N|N|N| +2418726|AAAAAAAAGCIOECAA|1910-02-23|121|530|41|1910|3|2|23|1|1910|41|530|Wednesday|1910Q1|N|N|N|2418704|2418734|2418361|2418634|N|N|N|N|N| +2418727|AAAAAAAAHCIOECAA|1910-02-24|121|530|41|1910|4|2|24|1|1910|41|530|Thursday|1910Q1|N|N|N|2418704|2418734|2418362|2418635|N|N|N|N|N| +2418728|AAAAAAAAICIOECAA|1910-02-25|121|530|41|1910|5|2|25|1|1910|41|530|Friday|1910Q1|N|Y|N|2418704|2418734|2418363|2418636|N|N|N|N|N| +2418729|AAAAAAAAJCIOECAA|1910-02-26|121|530|41|1910|6|2|26|1|1910|41|530|Saturday|1910Q1|N|Y|N|2418704|2418734|2418364|2418637|N|N|N|N|N| +2418730|AAAAAAAAKCIOECAA|1910-02-27|121|530|41|1910|0|2|27|1|1910|41|530|Sunday|1910Q1|N|N|N|2418704|2418734|2418365|2418638|N|N|N|N|N| +2418731|AAAAAAAALCIOECAA|1910-02-28|121|530|41|1910|1|2|28|1|1910|41|530|Monday|1910Q1|N|N|N|2418704|2418734|2418366|2418639|N|N|N|N|N| +2418732|AAAAAAAAMCIOECAA|1910-03-01|122|531|42|1910|2|3|1|1|1910|42|531|Tuesday|1910Q1|N|N|N|2418732|2418790|2418367|2418640|N|N|N|N|N| +2418733|AAAAAAAANCIOECAA|1910-03-02|122|531|42|1910|3|3|2|1|1910|42|531|Wednesday|1910Q1|N|N|N|2418732|2418790|2418368|2418641|N|N|N|N|N| +2418734|AAAAAAAAOCIOECAA|1910-03-03|122|531|42|1910|4|3|3|1|1910|42|531|Thursday|1910Q1|N|N|N|2418732|2418790|2418369|2418642|N|N|N|N|N| +2418735|AAAAAAAAPCIOECAA|1910-03-04|122|531|42|1910|5|3|4|1|1910|42|531|Friday|1910Q1|N|Y|N|2418732|2418790|2418370|2418643|N|N|N|N|N| +2418736|AAAAAAAAADIOECAA|1910-03-05|122|531|42|1910|6|3|5|1|1910|42|531|Saturday|1910Q1|N|Y|N|2418732|2418790|2418371|2418644|N|N|N|N|N| +2418737|AAAAAAAABDIOECAA|1910-03-06|122|531|42|1910|0|3|6|1|1910|42|531|Sunday|1910Q1|N|N|N|2418732|2418790|2418372|2418645|N|N|N|N|N| +2418738|AAAAAAAACDIOECAA|1910-03-07|122|531|42|1910|1|3|7|1|1910|42|531|Monday|1910Q1|N|N|N|2418732|2418790|2418373|2418646|N|N|N|N|N| +2418739|AAAAAAAADDIOECAA|1910-03-08|122|532|42|1910|2|3|8|1|1910|42|532|Tuesday|1910Q1|N|N|N|2418732|2418790|2418374|2418647|N|N|N|N|N| +2418740|AAAAAAAAEDIOECAA|1910-03-09|122|532|42|1910|3|3|9|1|1910|42|532|Wednesday|1910Q1|N|N|N|2418732|2418790|2418375|2418648|N|N|N|N|N| +2418741|AAAAAAAAFDIOECAA|1910-03-10|122|532|42|1910|4|3|10|1|1910|42|532|Thursday|1910Q1|N|N|N|2418732|2418790|2418376|2418649|N|N|N|N|N| +2418742|AAAAAAAAGDIOECAA|1910-03-11|122|532|42|1910|5|3|11|1|1910|42|532|Friday|1910Q1|N|Y|N|2418732|2418790|2418377|2418650|N|N|N|N|N| +2418743|AAAAAAAAHDIOECAA|1910-03-12|122|532|42|1910|6|3|12|1|1910|42|532|Saturday|1910Q1|N|Y|N|2418732|2418790|2418378|2418651|N|N|N|N|N| +2418744|AAAAAAAAIDIOECAA|1910-03-13|122|532|42|1910|0|3|13|1|1910|42|532|Sunday|1910Q1|N|N|N|2418732|2418790|2418379|2418652|N|N|N|N|N| +2418745|AAAAAAAAJDIOECAA|1910-03-14|122|532|42|1910|1|3|14|1|1910|42|532|Monday|1910Q1|N|N|N|2418732|2418790|2418380|2418653|N|N|N|N|N| +2418746|AAAAAAAAKDIOECAA|1910-03-15|122|533|42|1910|2|3|15|1|1910|42|533|Tuesday|1910Q1|N|N|N|2418732|2418790|2418381|2418654|N|N|N|N|N| +2418747|AAAAAAAALDIOECAA|1910-03-16|122|533|42|1910|3|3|16|1|1910|42|533|Wednesday|1910Q1|N|N|N|2418732|2418790|2418382|2418655|N|N|N|N|N| +2418748|AAAAAAAAMDIOECAA|1910-03-17|122|533|42|1910|4|3|17|1|1910|42|533|Thursday|1910Q1|N|N|N|2418732|2418790|2418383|2418656|N|N|N|N|N| +2418749|AAAAAAAANDIOECAA|1910-03-18|122|533|42|1910|5|3|18|1|1910|42|533|Friday|1910Q1|N|Y|N|2418732|2418790|2418384|2418657|N|N|N|N|N| +2418750|AAAAAAAAODIOECAA|1910-03-19|122|533|42|1910|6|3|19|1|1910|42|533|Saturday|1910Q1|N|Y|N|2418732|2418790|2418385|2418658|N|N|N|N|N| +2418751|AAAAAAAAPDIOECAA|1910-03-20|122|533|42|1910|0|3|20|1|1910|42|533|Sunday|1910Q1|N|N|N|2418732|2418790|2418386|2418659|N|N|N|N|N| +2418752|AAAAAAAAAEIOECAA|1910-03-21|122|533|42|1910|1|3|21|1|1910|42|533|Monday|1910Q1|N|N|N|2418732|2418790|2418387|2418660|N|N|N|N|N| +2418753|AAAAAAAABEIOECAA|1910-03-22|122|534|42|1910|2|3|22|1|1910|42|534|Tuesday|1910Q1|N|N|N|2418732|2418790|2418388|2418661|N|N|N|N|N| +2418754|AAAAAAAACEIOECAA|1910-03-23|122|534|42|1910|3|3|23|1|1910|42|534|Wednesday|1910Q1|N|N|N|2418732|2418790|2418389|2418662|N|N|N|N|N| +2418755|AAAAAAAADEIOECAA|1910-03-24|122|534|42|1910|4|3|24|1|1910|42|534|Thursday|1910Q1|N|N|N|2418732|2418790|2418390|2418663|N|N|N|N|N| +2418756|AAAAAAAAEEIOECAA|1910-03-25|122|534|42|1910|5|3|25|1|1910|42|534|Friday|1910Q1|N|Y|N|2418732|2418790|2418391|2418664|N|N|N|N|N| +2418757|AAAAAAAAFEIOECAA|1910-03-26|122|534|42|1910|6|3|26|1|1910|42|534|Saturday|1910Q1|N|Y|N|2418732|2418790|2418392|2418665|N|N|N|N|N| +2418758|AAAAAAAAGEIOECAA|1910-03-27|122|534|42|1910|0|3|27|1|1910|42|534|Sunday|1910Q1|N|N|N|2418732|2418790|2418393|2418666|N|N|N|N|N| +2418759|AAAAAAAAHEIOECAA|1910-03-28|122|534|42|1910|1|3|28|1|1910|42|534|Monday|1910Q1|N|N|N|2418732|2418790|2418394|2418667|N|N|N|N|N| +2418760|AAAAAAAAIEIOECAA|1910-03-29|122|535|42|1910|2|3|29|1|1910|42|535|Tuesday|1910Q1|N|N|N|2418732|2418790|2418395|2418668|N|N|N|N|N| +2418761|AAAAAAAAJEIOECAA|1910-03-30|122|535|42|1910|3|3|30|1|1910|42|535|Wednesday|1910Q1|N|N|N|2418732|2418790|2418396|2418669|N|N|N|N|N| +2418762|AAAAAAAAKEIOECAA|1910-03-31|122|535|42|1910|4|3|31|1|1910|42|535|Thursday|1910Q1|N|N|N|2418732|2418790|2418397|2418670|N|N|N|N|N| +2418763|AAAAAAAALEIOECAA|1910-04-01|123|535|42|1910|5|4|1|1|1910|42|535|Friday|1910Q1|N|Y|N|2418763|2418852|2418398|2418673|N|N|N|N|N| +2418764|AAAAAAAAMEIOECAA|1910-04-02|123|535|42|1910|6|4|2|2|1910|42|535|Saturday|1910Q2|N|Y|N|2418763|2418852|2418399|2418674|N|N|N|N|N| +2418765|AAAAAAAANEIOECAA|1910-04-03|123|535|42|1910|0|4|3|2|1910|42|535|Sunday|1910Q2|N|N|N|2418763|2418852|2418400|2418675|N|N|N|N|N| +2418766|AAAAAAAAOEIOECAA|1910-04-04|123|535|42|1910|1|4|4|2|1910|42|535|Monday|1910Q2|N|N|N|2418763|2418852|2418401|2418676|N|N|N|N|N| +2418767|AAAAAAAAPEIOECAA|1910-04-05|123|536|42|1910|2|4|5|2|1910|42|536|Tuesday|1910Q2|N|N|N|2418763|2418852|2418402|2418677|N|N|N|N|N| +2418768|AAAAAAAAAFIOECAA|1910-04-06|123|536|42|1910|3|4|6|2|1910|42|536|Wednesday|1910Q2|N|N|N|2418763|2418852|2418403|2418678|N|N|N|N|N| +2418769|AAAAAAAABFIOECAA|1910-04-07|123|536|42|1910|4|4|7|2|1910|42|536|Thursday|1910Q2|N|N|N|2418763|2418852|2418404|2418679|N|N|N|N|N| +2418770|AAAAAAAACFIOECAA|1910-04-08|123|536|42|1910|5|4|8|2|1910|42|536|Friday|1910Q2|N|Y|N|2418763|2418852|2418405|2418680|N|N|N|N|N| +2418771|AAAAAAAADFIOECAA|1910-04-09|123|536|42|1910|6|4|9|2|1910|42|536|Saturday|1910Q2|N|Y|N|2418763|2418852|2418406|2418681|N|N|N|N|N| +2418772|AAAAAAAAEFIOECAA|1910-04-10|123|536|42|1910|0|4|10|2|1910|42|536|Sunday|1910Q2|N|N|N|2418763|2418852|2418407|2418682|N|N|N|N|N| +2418773|AAAAAAAAFFIOECAA|1910-04-11|123|536|42|1910|1|4|11|2|1910|42|536|Monday|1910Q2|N|N|N|2418763|2418852|2418408|2418683|N|N|N|N|N| +2418774|AAAAAAAAGFIOECAA|1910-04-12|123|537|42|1910|2|4|12|2|1910|42|537|Tuesday|1910Q2|N|N|N|2418763|2418852|2418409|2418684|N|N|N|N|N| +2418775|AAAAAAAAHFIOECAA|1910-04-13|123|537|42|1910|3|4|13|2|1910|42|537|Wednesday|1910Q2|N|N|N|2418763|2418852|2418410|2418685|N|N|N|N|N| +2418776|AAAAAAAAIFIOECAA|1910-04-14|123|537|42|1910|4|4|14|2|1910|42|537|Thursday|1910Q2|N|N|N|2418763|2418852|2418411|2418686|N|N|N|N|N| +2418777|AAAAAAAAJFIOECAA|1910-04-15|123|537|42|1910|5|4|15|2|1910|42|537|Friday|1910Q2|N|Y|N|2418763|2418852|2418412|2418687|N|N|N|N|N| +2418778|AAAAAAAAKFIOECAA|1910-04-16|123|537|42|1910|6|4|16|2|1910|42|537|Saturday|1910Q2|N|Y|N|2418763|2418852|2418413|2418688|N|N|N|N|N| +2418779|AAAAAAAALFIOECAA|1910-04-17|123|537|42|1910|0|4|17|2|1910|42|537|Sunday|1910Q2|N|N|N|2418763|2418852|2418414|2418689|N|N|N|N|N| +2418780|AAAAAAAAMFIOECAA|1910-04-18|123|537|42|1910|1|4|18|2|1910|42|537|Monday|1910Q2|N|N|N|2418763|2418852|2418415|2418690|N|N|N|N|N| +2418781|AAAAAAAANFIOECAA|1910-04-19|123|538|42|1910|2|4|19|2|1910|42|538|Tuesday|1910Q2|N|N|N|2418763|2418852|2418416|2418691|N|N|N|N|N| +2418782|AAAAAAAAOFIOECAA|1910-04-20|123|538|42|1910|3|4|20|2|1910|42|538|Wednesday|1910Q2|N|N|N|2418763|2418852|2418417|2418692|N|N|N|N|N| +2418783|AAAAAAAAPFIOECAA|1910-04-21|123|538|42|1910|4|4|21|2|1910|42|538|Thursday|1910Q2|N|N|N|2418763|2418852|2418418|2418693|N|N|N|N|N| +2418784|AAAAAAAAAGIOECAA|1910-04-22|123|538|42|1910|5|4|22|2|1910|42|538|Friday|1910Q2|N|Y|N|2418763|2418852|2418419|2418694|N|N|N|N|N| +2418785|AAAAAAAABGIOECAA|1910-04-23|123|538|42|1910|6|4|23|2|1910|42|538|Saturday|1910Q2|N|Y|N|2418763|2418852|2418420|2418695|N|N|N|N|N| +2418786|AAAAAAAACGIOECAA|1910-04-24|123|538|42|1910|0|4|24|2|1910|42|538|Sunday|1910Q2|N|N|N|2418763|2418852|2418421|2418696|N|N|N|N|N| +2418787|AAAAAAAADGIOECAA|1910-04-25|123|538|42|1910|1|4|25|2|1910|42|538|Monday|1910Q2|N|N|N|2418763|2418852|2418422|2418697|N|N|N|N|N| +2418788|AAAAAAAAEGIOECAA|1910-04-26|123|539|42|1910|2|4|26|2|1910|42|539|Tuesday|1910Q2|N|N|N|2418763|2418852|2418423|2418698|N|N|N|N|N| +2418789|AAAAAAAAFGIOECAA|1910-04-27|123|539|42|1910|3|4|27|2|1910|42|539|Wednesday|1910Q2|N|N|N|2418763|2418852|2418424|2418699|N|N|N|N|N| +2418790|AAAAAAAAGGIOECAA|1910-04-28|123|539|42|1910|4|4|28|2|1910|42|539|Thursday|1910Q2|N|N|N|2418763|2418852|2418425|2418700|N|N|N|N|N| +2418791|AAAAAAAAHGIOECAA|1910-04-29|123|539|42|1910|5|4|29|2|1910|42|539|Friday|1910Q2|N|Y|N|2418763|2418852|2418426|2418701|N|N|N|N|N| +2418792|AAAAAAAAIGIOECAA|1910-04-30|123|539|42|1910|6|4|30|2|1910|42|539|Saturday|1910Q2|N|Y|N|2418763|2418852|2418427|2418702|N|N|N|N|N| +2418793|AAAAAAAAJGIOECAA|1910-05-01|124|539|42|1910|0|5|1|2|1910|42|539|Sunday|1910Q2|N|N|N|2418793|2418912|2418428|2418703|N|N|N|N|N| +2418794|AAAAAAAAKGIOECAA|1910-05-02|124|539|42|1910|1|5|2|2|1910|42|539|Monday|1910Q2|N|N|N|2418793|2418912|2418429|2418704|N|N|N|N|N| +2418795|AAAAAAAALGIOECAA|1910-05-03|124|540|42|1910|2|5|3|2|1910|42|540|Tuesday|1910Q2|N|N|N|2418793|2418912|2418430|2418705|N|N|N|N|N| +2418796|AAAAAAAAMGIOECAA|1910-05-04|124|540|42|1910|3|5|4|2|1910|42|540|Wednesday|1910Q2|N|N|N|2418793|2418912|2418431|2418706|N|N|N|N|N| +2418797|AAAAAAAANGIOECAA|1910-05-05|124|540|42|1910|4|5|5|2|1910|42|540|Thursday|1910Q2|N|N|N|2418793|2418912|2418432|2418707|N|N|N|N|N| +2418798|AAAAAAAAOGIOECAA|1910-05-06|124|540|42|1910|5|5|6|2|1910|42|540|Friday|1910Q2|N|Y|N|2418793|2418912|2418433|2418708|N|N|N|N|N| +2418799|AAAAAAAAPGIOECAA|1910-05-07|124|540|42|1910|6|5|7|2|1910|42|540|Saturday|1910Q2|N|Y|N|2418793|2418912|2418434|2418709|N|N|N|N|N| +2418800|AAAAAAAAAHIOECAA|1910-05-08|124|540|42|1910|0|5|8|2|1910|42|540|Sunday|1910Q2|N|N|N|2418793|2418912|2418435|2418710|N|N|N|N|N| +2418801|AAAAAAAABHIOECAA|1910-05-09|124|540|42|1910|1|5|9|2|1910|42|540|Monday|1910Q2|N|N|N|2418793|2418912|2418436|2418711|N|N|N|N|N| +2418802|AAAAAAAACHIOECAA|1910-05-10|124|541|42|1910|2|5|10|2|1910|42|541|Tuesday|1910Q2|N|N|N|2418793|2418912|2418437|2418712|N|N|N|N|N| +2418803|AAAAAAAADHIOECAA|1910-05-11|124|541|42|1910|3|5|11|2|1910|42|541|Wednesday|1910Q2|N|N|N|2418793|2418912|2418438|2418713|N|N|N|N|N| +2418804|AAAAAAAAEHIOECAA|1910-05-12|124|541|42|1910|4|5|12|2|1910|42|541|Thursday|1910Q2|N|N|N|2418793|2418912|2418439|2418714|N|N|N|N|N| +2418805|AAAAAAAAFHIOECAA|1910-05-13|124|541|42|1910|5|5|13|2|1910|42|541|Friday|1910Q2|N|Y|N|2418793|2418912|2418440|2418715|N|N|N|N|N| +2418806|AAAAAAAAGHIOECAA|1910-05-14|124|541|42|1910|6|5|14|2|1910|42|541|Saturday|1910Q2|N|Y|N|2418793|2418912|2418441|2418716|N|N|N|N|N| +2418807|AAAAAAAAHHIOECAA|1910-05-15|124|541|42|1910|0|5|15|2|1910|42|541|Sunday|1910Q2|N|N|N|2418793|2418912|2418442|2418717|N|N|N|N|N| +2418808|AAAAAAAAIHIOECAA|1910-05-16|124|541|42|1910|1|5|16|2|1910|42|541|Monday|1910Q2|N|N|N|2418793|2418912|2418443|2418718|N|N|N|N|N| +2418809|AAAAAAAAJHIOECAA|1910-05-17|124|542|42|1910|2|5|17|2|1910|42|542|Tuesday|1910Q2|N|N|N|2418793|2418912|2418444|2418719|N|N|N|N|N| +2418810|AAAAAAAAKHIOECAA|1910-05-18|124|542|42|1910|3|5|18|2|1910|42|542|Wednesday|1910Q2|N|N|N|2418793|2418912|2418445|2418720|N|N|N|N|N| +2418811|AAAAAAAALHIOECAA|1910-05-19|124|542|42|1910|4|5|19|2|1910|42|542|Thursday|1910Q2|N|N|N|2418793|2418912|2418446|2418721|N|N|N|N|N| +2418812|AAAAAAAAMHIOECAA|1910-05-20|124|542|42|1910|5|5|20|2|1910|42|542|Friday|1910Q2|N|Y|N|2418793|2418912|2418447|2418722|N|N|N|N|N| +2418813|AAAAAAAANHIOECAA|1910-05-21|124|542|42|1910|6|5|21|2|1910|42|542|Saturday|1910Q2|N|Y|N|2418793|2418912|2418448|2418723|N|N|N|N|N| +2418814|AAAAAAAAOHIOECAA|1910-05-22|124|542|42|1910|0|5|22|2|1910|42|542|Sunday|1910Q2|N|N|N|2418793|2418912|2418449|2418724|N|N|N|N|N| +2418815|AAAAAAAAPHIOECAA|1910-05-23|124|542|42|1910|1|5|23|2|1910|42|542|Monday|1910Q2|N|N|N|2418793|2418912|2418450|2418725|N|N|N|N|N| +2418816|AAAAAAAAAIIOECAA|1910-05-24|124|543|42|1910|2|5|24|2|1910|42|543|Tuesday|1910Q2|N|N|N|2418793|2418912|2418451|2418726|N|N|N|N|N| +2418817|AAAAAAAABIIOECAA|1910-05-25|124|543|42|1910|3|5|25|2|1910|42|543|Wednesday|1910Q2|N|N|N|2418793|2418912|2418452|2418727|N|N|N|N|N| +2418818|AAAAAAAACIIOECAA|1910-05-26|124|543|42|1910|4|5|26|2|1910|42|543|Thursday|1910Q2|N|N|N|2418793|2418912|2418453|2418728|N|N|N|N|N| +2418819|AAAAAAAADIIOECAA|1910-05-27|124|543|42|1910|5|5|27|2|1910|42|543|Friday|1910Q2|N|Y|N|2418793|2418912|2418454|2418729|N|N|N|N|N| +2418820|AAAAAAAAEIIOECAA|1910-05-28|124|543|42|1910|6|5|28|2|1910|42|543|Saturday|1910Q2|N|Y|N|2418793|2418912|2418455|2418730|N|N|N|N|N| +2418821|AAAAAAAAFIIOECAA|1910-05-29|124|543|42|1910|0|5|29|2|1910|42|543|Sunday|1910Q2|N|N|N|2418793|2418912|2418456|2418731|N|N|N|N|N| +2418822|AAAAAAAAGIIOECAA|1910-05-30|124|543|42|1910|1|5|30|2|1910|42|543|Monday|1910Q2|N|N|N|2418793|2418912|2418457|2418732|N|N|N|N|N| +2418823|AAAAAAAAHIIOECAA|1910-05-31|124|544|42|1910|2|5|31|2|1910|42|544|Tuesday|1910Q2|N|N|N|2418793|2418912|2418458|2418733|N|N|N|N|N| +2418824|AAAAAAAAIIIOECAA|1910-06-01|125|544|43|1910|3|6|1|2|1910|43|544|Wednesday|1910Q2|N|N|N|2418824|2418974|2418459|2418734|N|N|N|N|N| +2418825|AAAAAAAAJIIOECAA|1910-06-02|125|544|43|1910|4|6|2|2|1910|43|544|Thursday|1910Q2|N|N|N|2418824|2418974|2418460|2418735|N|N|N|N|N| +2418826|AAAAAAAAKIIOECAA|1910-06-03|125|544|43|1910|5|6|3|2|1910|43|544|Friday|1910Q2|N|Y|N|2418824|2418974|2418461|2418736|N|N|N|N|N| +2418827|AAAAAAAALIIOECAA|1910-06-04|125|544|43|1910|6|6|4|2|1910|43|544|Saturday|1910Q2|N|Y|N|2418824|2418974|2418462|2418737|N|N|N|N|N| +2418828|AAAAAAAAMIIOECAA|1910-06-05|125|544|43|1910|0|6|5|2|1910|43|544|Sunday|1910Q2|N|N|N|2418824|2418974|2418463|2418738|N|N|N|N|N| +2418829|AAAAAAAANIIOECAA|1910-06-06|125|544|43|1910|1|6|6|2|1910|43|544|Monday|1910Q2|N|N|N|2418824|2418974|2418464|2418739|N|N|N|N|N| +2418830|AAAAAAAAOIIOECAA|1910-06-07|125|545|43|1910|2|6|7|2|1910|43|545|Tuesday|1910Q2|N|N|N|2418824|2418974|2418465|2418740|N|N|N|N|N| +2418831|AAAAAAAAPIIOECAA|1910-06-08|125|545|43|1910|3|6|8|2|1910|43|545|Wednesday|1910Q2|N|N|N|2418824|2418974|2418466|2418741|N|N|N|N|N| +2418832|AAAAAAAAAJIOECAA|1910-06-09|125|545|43|1910|4|6|9|2|1910|43|545|Thursday|1910Q2|N|N|N|2418824|2418974|2418467|2418742|N|N|N|N|N| +2418833|AAAAAAAABJIOECAA|1910-06-10|125|545|43|1910|5|6|10|2|1910|43|545|Friday|1910Q2|N|Y|N|2418824|2418974|2418468|2418743|N|N|N|N|N| +2418834|AAAAAAAACJIOECAA|1910-06-11|125|545|43|1910|6|6|11|2|1910|43|545|Saturday|1910Q2|N|Y|N|2418824|2418974|2418469|2418744|N|N|N|N|N| +2418835|AAAAAAAADJIOECAA|1910-06-12|125|545|43|1910|0|6|12|2|1910|43|545|Sunday|1910Q2|N|N|N|2418824|2418974|2418470|2418745|N|N|N|N|N| +2418836|AAAAAAAAEJIOECAA|1910-06-13|125|545|43|1910|1|6|13|2|1910|43|545|Monday|1910Q2|N|N|N|2418824|2418974|2418471|2418746|N|N|N|N|N| +2418837|AAAAAAAAFJIOECAA|1910-06-14|125|546|43|1910|2|6|14|2|1910|43|546|Tuesday|1910Q2|N|N|N|2418824|2418974|2418472|2418747|N|N|N|N|N| +2418838|AAAAAAAAGJIOECAA|1910-06-15|125|546|43|1910|3|6|15|2|1910|43|546|Wednesday|1910Q2|N|N|N|2418824|2418974|2418473|2418748|N|N|N|N|N| +2418839|AAAAAAAAHJIOECAA|1910-06-16|125|546|43|1910|4|6|16|2|1910|43|546|Thursday|1910Q2|N|N|N|2418824|2418974|2418474|2418749|N|N|N|N|N| +2418840|AAAAAAAAIJIOECAA|1910-06-17|125|546|43|1910|5|6|17|2|1910|43|546|Friday|1910Q2|N|Y|N|2418824|2418974|2418475|2418750|N|N|N|N|N| +2418841|AAAAAAAAJJIOECAA|1910-06-18|125|546|43|1910|6|6|18|2|1910|43|546|Saturday|1910Q2|N|Y|N|2418824|2418974|2418476|2418751|N|N|N|N|N| +2418842|AAAAAAAAKJIOECAA|1910-06-19|125|546|43|1910|0|6|19|2|1910|43|546|Sunday|1910Q2|N|N|N|2418824|2418974|2418477|2418752|N|N|N|N|N| +2418843|AAAAAAAALJIOECAA|1910-06-20|125|546|43|1910|1|6|20|2|1910|43|546|Monday|1910Q2|N|N|N|2418824|2418974|2418478|2418753|N|N|N|N|N| +2418844|AAAAAAAAMJIOECAA|1910-06-21|125|547|43|1910|2|6|21|2|1910|43|547|Tuesday|1910Q2|N|N|N|2418824|2418974|2418479|2418754|N|N|N|N|N| +2418845|AAAAAAAANJIOECAA|1910-06-22|125|547|43|1910|3|6|22|2|1910|43|547|Wednesday|1910Q2|N|N|N|2418824|2418974|2418480|2418755|N|N|N|N|N| +2418846|AAAAAAAAOJIOECAA|1910-06-23|125|547|43|1910|4|6|23|2|1910|43|547|Thursday|1910Q2|N|N|N|2418824|2418974|2418481|2418756|N|N|N|N|N| +2418847|AAAAAAAAPJIOECAA|1910-06-24|125|547|43|1910|5|6|24|2|1910|43|547|Friday|1910Q2|N|Y|N|2418824|2418974|2418482|2418757|N|N|N|N|N| +2418848|AAAAAAAAAKIOECAA|1910-06-25|125|547|43|1910|6|6|25|2|1910|43|547|Saturday|1910Q2|N|Y|N|2418824|2418974|2418483|2418758|N|N|N|N|N| +2418849|AAAAAAAABKIOECAA|1910-06-26|125|547|43|1910|0|6|26|2|1910|43|547|Sunday|1910Q2|N|N|N|2418824|2418974|2418484|2418759|N|N|N|N|N| +2418850|AAAAAAAACKIOECAA|1910-06-27|125|547|43|1910|1|6|27|2|1910|43|547|Monday|1910Q2|N|N|N|2418824|2418974|2418485|2418760|N|N|N|N|N| +2418851|AAAAAAAADKIOECAA|1910-06-28|125|548|43|1910|2|6|28|2|1910|43|548|Tuesday|1910Q2|N|N|N|2418824|2418974|2418486|2418761|N|N|N|N|N| +2418852|AAAAAAAAEKIOECAA|1910-06-29|125|548|43|1910|3|6|29|2|1910|43|548|Wednesday|1910Q2|N|N|N|2418824|2418974|2418487|2418762|N|N|N|N|N| +2418853|AAAAAAAAFKIOECAA|1910-06-30|125|548|43|1910|4|6|30|2|1910|43|548|Thursday|1910Q2|N|N|N|2418824|2418974|2418488|2418763|N|N|N|N|N| +2418854|AAAAAAAAGKIOECAA|1910-07-01|126|548|43|1910|5|7|1|2|1910|43|548|Friday|1910Q2|N|Y|N|2418854|2419034|2418489|2418763|N|N|N|N|N| +2418855|AAAAAAAAHKIOECAA|1910-07-02|126|548|43|1910|6|7|2|3|1910|43|548|Saturday|1910Q3|N|Y|N|2418854|2419034|2418490|2418764|N|N|N|N|N| +2418856|AAAAAAAAIKIOECAA|1910-07-03|126|548|43|1910|0|7|3|3|1910|43|548|Sunday|1910Q3|N|N|N|2418854|2419034|2418491|2418765|N|N|N|N|N| +2418857|AAAAAAAAJKIOECAA|1910-07-04|126|548|43|1910|1|7|4|3|1910|43|548|Monday|1910Q3|N|N|N|2418854|2419034|2418492|2418766|N|N|N|N|N| +2418858|AAAAAAAAKKIOECAA|1910-07-05|126|549|43|1910|2|7|5|3|1910|43|549|Tuesday|1910Q3|Y|N|N|2418854|2419034|2418493|2418767|N|N|N|N|N| +2418859|AAAAAAAALKIOECAA|1910-07-06|126|549|43|1910|3|7|6|3|1910|43|549|Wednesday|1910Q3|N|N|Y|2418854|2419034|2418494|2418768|N|N|N|N|N| +2418860|AAAAAAAAMKIOECAA|1910-07-07|126|549|43|1910|4|7|7|3|1910|43|549|Thursday|1910Q3|N|N|N|2418854|2419034|2418495|2418769|N|N|N|N|N| +2418861|AAAAAAAANKIOECAA|1910-07-08|126|549|43|1910|5|7|8|3|1910|43|549|Friday|1910Q3|N|Y|N|2418854|2419034|2418496|2418770|N|N|N|N|N| +2418862|AAAAAAAAOKIOECAA|1910-07-09|126|549|43|1910|6|7|9|3|1910|43|549|Saturday|1910Q3|N|Y|N|2418854|2419034|2418497|2418771|N|N|N|N|N| +2418863|AAAAAAAAPKIOECAA|1910-07-10|126|549|43|1910|0|7|10|3|1910|43|549|Sunday|1910Q3|N|N|N|2418854|2419034|2418498|2418772|N|N|N|N|N| +2418864|AAAAAAAAALIOECAA|1910-07-11|126|549|43|1910|1|7|11|3|1910|43|549|Monday|1910Q3|N|N|N|2418854|2419034|2418499|2418773|N|N|N|N|N| +2418865|AAAAAAAABLIOECAA|1910-07-12|126|550|43|1910|2|7|12|3|1910|43|550|Tuesday|1910Q3|N|N|N|2418854|2419034|2418500|2418774|N|N|N|N|N| +2418866|AAAAAAAACLIOECAA|1910-07-13|126|550|43|1910|3|7|13|3|1910|43|550|Wednesday|1910Q3|N|N|N|2418854|2419034|2418501|2418775|N|N|N|N|N| +2418867|AAAAAAAADLIOECAA|1910-07-14|126|550|43|1910|4|7|14|3|1910|43|550|Thursday|1910Q3|N|N|N|2418854|2419034|2418502|2418776|N|N|N|N|N| +2418868|AAAAAAAAELIOECAA|1910-07-15|126|550|43|1910|5|7|15|3|1910|43|550|Friday|1910Q3|N|Y|N|2418854|2419034|2418503|2418777|N|N|N|N|N| +2418869|AAAAAAAAFLIOECAA|1910-07-16|126|550|43|1910|6|7|16|3|1910|43|550|Saturday|1910Q3|N|Y|N|2418854|2419034|2418504|2418778|N|N|N|N|N| +2418870|AAAAAAAAGLIOECAA|1910-07-17|126|550|43|1910|0|7|17|3|1910|43|550|Sunday|1910Q3|N|N|N|2418854|2419034|2418505|2418779|N|N|N|N|N| +2418871|AAAAAAAAHLIOECAA|1910-07-18|126|550|43|1910|1|7|18|3|1910|43|550|Monday|1910Q3|N|N|N|2418854|2419034|2418506|2418780|N|N|N|N|N| +2418872|AAAAAAAAILIOECAA|1910-07-19|126|551|43|1910|2|7|19|3|1910|43|551|Tuesday|1910Q3|N|N|N|2418854|2419034|2418507|2418781|N|N|N|N|N| +2418873|AAAAAAAAJLIOECAA|1910-07-20|126|551|43|1910|3|7|20|3|1910|43|551|Wednesday|1910Q3|N|N|N|2418854|2419034|2418508|2418782|N|N|N|N|N| +2418874|AAAAAAAAKLIOECAA|1910-07-21|126|551|43|1910|4|7|21|3|1910|43|551|Thursday|1910Q3|N|N|N|2418854|2419034|2418509|2418783|N|N|N|N|N| +2418875|AAAAAAAALLIOECAA|1910-07-22|126|551|43|1910|5|7|22|3|1910|43|551|Friday|1910Q3|N|Y|N|2418854|2419034|2418510|2418784|N|N|N|N|N| +2418876|AAAAAAAAMLIOECAA|1910-07-23|126|551|43|1910|6|7|23|3|1910|43|551|Saturday|1910Q3|N|Y|N|2418854|2419034|2418511|2418785|N|N|N|N|N| +2418877|AAAAAAAANLIOECAA|1910-07-24|126|551|43|1910|0|7|24|3|1910|43|551|Sunday|1910Q3|N|N|N|2418854|2419034|2418512|2418786|N|N|N|N|N| +2418878|AAAAAAAAOLIOECAA|1910-07-25|126|551|43|1910|1|7|25|3|1910|43|551|Monday|1910Q3|N|N|N|2418854|2419034|2418513|2418787|N|N|N|N|N| +2418879|AAAAAAAAPLIOECAA|1910-07-26|126|552|43|1910|2|7|26|3|1910|43|552|Tuesday|1910Q3|N|N|N|2418854|2419034|2418514|2418788|N|N|N|N|N| +2418880|AAAAAAAAAMIOECAA|1910-07-27|126|552|43|1910|3|7|27|3|1910|43|552|Wednesday|1910Q3|N|N|N|2418854|2419034|2418515|2418789|N|N|N|N|N| +2418881|AAAAAAAABMIOECAA|1910-07-28|126|552|43|1910|4|7|28|3|1910|43|552|Thursday|1910Q3|N|N|N|2418854|2419034|2418516|2418790|N|N|N|N|N| +2418882|AAAAAAAACMIOECAA|1910-07-29|126|552|43|1910|5|7|29|3|1910|43|552|Friday|1910Q3|N|Y|N|2418854|2419034|2418517|2418791|N|N|N|N|N| +2418883|AAAAAAAADMIOECAA|1910-07-30|126|552|43|1910|6|7|30|3|1910|43|552|Saturday|1910Q3|N|Y|N|2418854|2419034|2418518|2418792|N|N|N|N|N| +2418884|AAAAAAAAEMIOECAA|1910-07-31|126|552|43|1910|0|7|31|3|1910|43|552|Sunday|1910Q3|N|N|N|2418854|2419034|2418519|2418793|N|N|N|N|N| +2418885|AAAAAAAAFMIOECAA|1910-08-01|127|552|43|1910|1|8|1|3|1910|43|552|Monday|1910Q3|N|N|N|2418885|2419096|2418520|2418794|N|N|N|N|N| +2418886|AAAAAAAAGMIOECAA|1910-08-02|127|553|43|1910|2|8|2|3|1910|43|553|Tuesday|1910Q3|N|N|N|2418885|2419096|2418521|2418795|N|N|N|N|N| +2418887|AAAAAAAAHMIOECAA|1910-08-03|127|553|43|1910|3|8|3|3|1910|43|553|Wednesday|1910Q3|N|N|N|2418885|2419096|2418522|2418796|N|N|N|N|N| +2418888|AAAAAAAAIMIOECAA|1910-08-04|127|553|43|1910|4|8|4|3|1910|43|553|Thursday|1910Q3|N|N|N|2418885|2419096|2418523|2418797|N|N|N|N|N| +2418889|AAAAAAAAJMIOECAA|1910-08-05|127|553|43|1910|5|8|5|3|1910|43|553|Friday|1910Q3|N|Y|N|2418885|2419096|2418524|2418798|N|N|N|N|N| +2418890|AAAAAAAAKMIOECAA|1910-08-06|127|553|43|1910|6|8|6|3|1910|43|553|Saturday|1910Q3|N|Y|N|2418885|2419096|2418525|2418799|N|N|N|N|N| +2418891|AAAAAAAALMIOECAA|1910-08-07|127|553|43|1910|0|8|7|3|1910|43|553|Sunday|1910Q3|N|N|N|2418885|2419096|2418526|2418800|N|N|N|N|N| +2418892|AAAAAAAAMMIOECAA|1910-08-08|127|553|43|1910|1|8|8|3|1910|43|553|Monday|1910Q3|N|N|N|2418885|2419096|2418527|2418801|N|N|N|N|N| +2418893|AAAAAAAANMIOECAA|1910-08-09|127|554|43|1910|2|8|9|3|1910|43|554|Tuesday|1910Q3|N|N|N|2418885|2419096|2418528|2418802|N|N|N|N|N| +2418894|AAAAAAAAOMIOECAA|1910-08-10|127|554|43|1910|3|8|10|3|1910|43|554|Wednesday|1910Q3|N|N|N|2418885|2419096|2418529|2418803|N|N|N|N|N| +2418895|AAAAAAAAPMIOECAA|1910-08-11|127|554|43|1910|4|8|11|3|1910|43|554|Thursday|1910Q3|N|N|N|2418885|2419096|2418530|2418804|N|N|N|N|N| +2418896|AAAAAAAAANIOECAA|1910-08-12|127|554|43|1910|5|8|12|3|1910|43|554|Friday|1910Q3|N|Y|N|2418885|2419096|2418531|2418805|N|N|N|N|N| +2418897|AAAAAAAABNIOECAA|1910-08-13|127|554|43|1910|6|8|13|3|1910|43|554|Saturday|1910Q3|N|Y|N|2418885|2419096|2418532|2418806|N|N|N|N|N| +2418898|AAAAAAAACNIOECAA|1910-08-14|127|554|43|1910|0|8|14|3|1910|43|554|Sunday|1910Q3|N|N|N|2418885|2419096|2418533|2418807|N|N|N|N|N| +2418899|AAAAAAAADNIOECAA|1910-08-15|127|554|43|1910|1|8|15|3|1910|43|554|Monday|1910Q3|N|N|N|2418885|2419096|2418534|2418808|N|N|N|N|N| +2418900|AAAAAAAAENIOECAA|1910-08-16|127|555|43|1910|2|8|16|3|1910|43|555|Tuesday|1910Q3|N|N|N|2418885|2419096|2418535|2418809|N|N|N|N|N| +2418901|AAAAAAAAFNIOECAA|1910-08-17|127|555|43|1910|3|8|17|3|1910|43|555|Wednesday|1910Q3|N|N|N|2418885|2419096|2418536|2418810|N|N|N|N|N| +2418902|AAAAAAAAGNIOECAA|1910-08-18|127|555|43|1910|4|8|18|3|1910|43|555|Thursday|1910Q3|N|N|N|2418885|2419096|2418537|2418811|N|N|N|N|N| +2418903|AAAAAAAAHNIOECAA|1910-08-19|127|555|43|1910|5|8|19|3|1910|43|555|Friday|1910Q3|N|Y|N|2418885|2419096|2418538|2418812|N|N|N|N|N| +2418904|AAAAAAAAINIOECAA|1910-08-20|127|555|43|1910|6|8|20|3|1910|43|555|Saturday|1910Q3|N|Y|N|2418885|2419096|2418539|2418813|N|N|N|N|N| +2418905|AAAAAAAAJNIOECAA|1910-08-21|127|555|43|1910|0|8|21|3|1910|43|555|Sunday|1910Q3|N|N|N|2418885|2419096|2418540|2418814|N|N|N|N|N| +2418906|AAAAAAAAKNIOECAA|1910-08-22|127|555|43|1910|1|8|22|3|1910|43|555|Monday|1910Q3|N|N|N|2418885|2419096|2418541|2418815|N|N|N|N|N| +2418907|AAAAAAAALNIOECAA|1910-08-23|127|556|43|1910|2|8|23|3|1910|43|556|Tuesday|1910Q3|N|N|N|2418885|2419096|2418542|2418816|N|N|N|N|N| +2418908|AAAAAAAAMNIOECAA|1910-08-24|127|556|43|1910|3|8|24|3|1910|43|556|Wednesday|1910Q3|N|N|N|2418885|2419096|2418543|2418817|N|N|N|N|N| +2418909|AAAAAAAANNIOECAA|1910-08-25|127|556|43|1910|4|8|25|3|1910|43|556|Thursday|1910Q3|N|N|N|2418885|2419096|2418544|2418818|N|N|N|N|N| +2418910|AAAAAAAAONIOECAA|1910-08-26|127|556|43|1910|5|8|26|3|1910|43|556|Friday|1910Q3|N|Y|N|2418885|2419096|2418545|2418819|N|N|N|N|N| +2418911|AAAAAAAAPNIOECAA|1910-08-27|127|556|43|1910|6|8|27|3|1910|43|556|Saturday|1910Q3|N|Y|N|2418885|2419096|2418546|2418820|N|N|N|N|N| +2418912|AAAAAAAAAOIOECAA|1910-08-28|127|556|43|1910|0|8|28|3|1910|43|556|Sunday|1910Q3|N|N|N|2418885|2419096|2418547|2418821|N|N|N|N|N| +2418913|AAAAAAAABOIOECAA|1910-08-29|127|556|43|1910|1|8|29|3|1910|43|556|Monday|1910Q3|N|N|N|2418885|2419096|2418548|2418822|N|N|N|N|N| +2418914|AAAAAAAACOIOECAA|1910-08-30|127|557|43|1910|2|8|30|3|1910|43|557|Tuesday|1910Q3|N|N|N|2418885|2419096|2418549|2418823|N|N|N|N|N| +2418915|AAAAAAAADOIOECAA|1910-08-31|127|557|43|1910|3|8|31|3|1910|43|557|Wednesday|1910Q3|N|N|N|2418885|2419096|2418550|2418824|N|N|N|N|N| +2418916|AAAAAAAAEOIOECAA|1910-09-01|128|557|44|1910|4|9|1|3|1910|44|557|Thursday|1910Q3|N|N|N|2418916|2419158|2418551|2418825|N|N|N|N|N| +2418917|AAAAAAAAFOIOECAA|1910-09-02|128|557|44|1910|5|9|2|3|1910|44|557|Friday|1910Q3|N|Y|N|2418916|2419158|2418552|2418826|N|N|N|N|N| +2418918|AAAAAAAAGOIOECAA|1910-09-03|128|557|44|1910|6|9|3|3|1910|44|557|Saturday|1910Q3|N|Y|N|2418916|2419158|2418553|2418827|N|N|N|N|N| +2418919|AAAAAAAAHOIOECAA|1910-09-04|128|557|44|1910|0|9|4|3|1910|44|557|Sunday|1910Q3|N|N|N|2418916|2419158|2418554|2418828|N|N|N|N|N| +2418920|AAAAAAAAIOIOECAA|1910-09-05|128|557|44|1910|1|9|5|3|1910|44|557|Monday|1910Q3|N|N|N|2418916|2419158|2418555|2418829|N|N|N|N|N| +2418921|AAAAAAAAJOIOECAA|1910-09-06|128|558|44|1910|2|9|6|3|1910|44|558|Tuesday|1910Q3|N|N|N|2418916|2419158|2418556|2418830|N|N|N|N|N| +2418922|AAAAAAAAKOIOECAA|1910-09-07|128|558|44|1910|3|9|7|3|1910|44|558|Wednesday|1910Q3|N|N|N|2418916|2419158|2418557|2418831|N|N|N|N|N| +2418923|AAAAAAAALOIOECAA|1910-09-08|128|558|44|1910|4|9|8|3|1910|44|558|Thursday|1910Q3|N|N|N|2418916|2419158|2418558|2418832|N|N|N|N|N| +2418924|AAAAAAAAMOIOECAA|1910-09-09|128|558|44|1910|5|9|9|3|1910|44|558|Friday|1910Q3|N|Y|N|2418916|2419158|2418559|2418833|N|N|N|N|N| +2418925|AAAAAAAANOIOECAA|1910-09-10|128|558|44|1910|6|9|10|3|1910|44|558|Saturday|1910Q3|N|Y|N|2418916|2419158|2418560|2418834|N|N|N|N|N| +2418926|AAAAAAAAOOIOECAA|1910-09-11|128|558|44|1910|0|9|11|3|1910|44|558|Sunday|1910Q3|N|N|N|2418916|2419158|2418561|2418835|N|N|N|N|N| +2418927|AAAAAAAAPOIOECAA|1910-09-12|128|558|44|1910|1|9|12|3|1910|44|558|Monday|1910Q3|N|N|N|2418916|2419158|2418562|2418836|N|N|N|N|N| +2418928|AAAAAAAAAPIOECAA|1910-09-13|128|559|44|1910|2|9|13|3|1910|44|559|Tuesday|1910Q3|N|N|N|2418916|2419158|2418563|2418837|N|N|N|N|N| +2418929|AAAAAAAABPIOECAA|1910-09-14|128|559|44|1910|3|9|14|3|1910|44|559|Wednesday|1910Q3|N|N|N|2418916|2419158|2418564|2418838|N|N|N|N|N| +2418930|AAAAAAAACPIOECAA|1910-09-15|128|559|44|1910|4|9|15|3|1910|44|559|Thursday|1910Q3|N|N|N|2418916|2419158|2418565|2418839|N|N|N|N|N| +2418931|AAAAAAAADPIOECAA|1910-09-16|128|559|44|1910|5|9|16|3|1910|44|559|Friday|1910Q3|N|Y|N|2418916|2419158|2418566|2418840|N|N|N|N|N| +2418932|AAAAAAAAEPIOECAA|1910-09-17|128|559|44|1910|6|9|17|3|1910|44|559|Saturday|1910Q3|N|Y|N|2418916|2419158|2418567|2418841|N|N|N|N|N| +2418933|AAAAAAAAFPIOECAA|1910-09-18|128|559|44|1910|0|9|18|3|1910|44|559|Sunday|1910Q3|N|N|N|2418916|2419158|2418568|2418842|N|N|N|N|N| +2418934|AAAAAAAAGPIOECAA|1910-09-19|128|559|44|1910|1|9|19|3|1910|44|559|Monday|1910Q3|N|N|N|2418916|2419158|2418569|2418843|N|N|N|N|N| +2418935|AAAAAAAAHPIOECAA|1910-09-20|128|560|44|1910|2|9|20|3|1910|44|560|Tuesday|1910Q3|N|N|N|2418916|2419158|2418570|2418844|N|N|N|N|N| +2418936|AAAAAAAAIPIOECAA|1910-09-21|128|560|44|1910|3|9|21|3|1910|44|560|Wednesday|1910Q3|N|N|N|2418916|2419158|2418571|2418845|N|N|N|N|N| +2418937|AAAAAAAAJPIOECAA|1910-09-22|128|560|44|1910|4|9|22|3|1910|44|560|Thursday|1910Q3|N|N|N|2418916|2419158|2418572|2418846|N|N|N|N|N| +2418938|AAAAAAAAKPIOECAA|1910-09-23|128|560|44|1910|5|9|23|3|1910|44|560|Friday|1910Q3|N|Y|N|2418916|2419158|2418573|2418847|N|N|N|N|N| +2418939|AAAAAAAALPIOECAA|1910-09-24|128|560|44|1910|6|9|24|3|1910|44|560|Saturday|1910Q3|N|Y|N|2418916|2419158|2418574|2418848|N|N|N|N|N| +2418940|AAAAAAAAMPIOECAA|1910-09-25|128|560|44|1910|0|9|25|3|1910|44|560|Sunday|1910Q3|N|N|N|2418916|2419158|2418575|2418849|N|N|N|N|N| +2418941|AAAAAAAANPIOECAA|1910-09-26|128|560|44|1910|1|9|26|3|1910|44|560|Monday|1910Q3|N|N|N|2418916|2419158|2418576|2418850|N|N|N|N|N| +2418942|AAAAAAAAOPIOECAA|1910-09-27|128|561|44|1910|2|9|27|3|1910|44|561|Tuesday|1910Q3|N|N|N|2418916|2419158|2418577|2418851|N|N|N|N|N| +2418943|AAAAAAAAPPIOECAA|1910-09-28|128|561|44|1910|3|9|28|3|1910|44|561|Wednesday|1910Q3|N|N|N|2418916|2419158|2418578|2418852|N|N|N|N|N| +2418944|AAAAAAAAAAJOECAA|1910-09-29|128|561|44|1910|4|9|29|3|1910|44|561|Thursday|1910Q3|N|N|N|2418916|2419158|2418579|2418853|N|N|N|N|N| +2418945|AAAAAAAABAJOECAA|1910-09-30|128|561|44|1910|5|9|30|3|1910|44|561|Friday|1910Q3|N|Y|N|2418916|2419158|2418580|2418854|N|N|N|N|N| +2418946|AAAAAAAACAJOECAA|1910-10-01|129|561|44|1910|6|10|1|3|1910|44|561|Saturday|1910Q3|N|Y|N|2418946|2419218|2418581|2418854|N|N|N|N|N| +2418947|AAAAAAAADAJOECAA|1910-10-02|129|561|44|1910|0|10|2|4|1910|44|561|Sunday|1910Q4|N|N|N|2418946|2419218|2418582|2418855|N|N|N|N|N| +2418948|AAAAAAAAEAJOECAA|1910-10-03|129|561|44|1910|1|10|3|4|1910|44|561|Monday|1910Q4|N|N|N|2418946|2419218|2418583|2418856|N|N|N|N|N| +2418949|AAAAAAAAFAJOECAA|1910-10-04|129|562|44|1910|2|10|4|4|1910|44|562|Tuesday|1910Q4|N|N|N|2418946|2419218|2418584|2418857|N|N|N|N|N| +2418950|AAAAAAAAGAJOECAA|1910-10-05|129|562|44|1910|3|10|5|4|1910|44|562|Wednesday|1910Q4|N|N|N|2418946|2419218|2418585|2418858|N|N|N|N|N| +2418951|AAAAAAAAHAJOECAA|1910-10-06|129|562|44|1910|4|10|6|4|1910|44|562|Thursday|1910Q4|N|N|N|2418946|2419218|2418586|2418859|N|N|N|N|N| +2418952|AAAAAAAAIAJOECAA|1910-10-07|129|562|44|1910|5|10|7|4|1910|44|562|Friday|1910Q4|N|Y|N|2418946|2419218|2418587|2418860|N|N|N|N|N| +2418953|AAAAAAAAJAJOECAA|1910-10-08|129|562|44|1910|6|10|8|4|1910|44|562|Saturday|1910Q4|N|Y|N|2418946|2419218|2418588|2418861|N|N|N|N|N| +2418954|AAAAAAAAKAJOECAA|1910-10-09|129|562|44|1910|0|10|9|4|1910|44|562|Sunday|1910Q4|N|N|N|2418946|2419218|2418589|2418862|N|N|N|N|N| +2418955|AAAAAAAALAJOECAA|1910-10-10|129|562|44|1910|1|10|10|4|1910|44|562|Monday|1910Q4|N|N|N|2418946|2419218|2418590|2418863|N|N|N|N|N| +2418956|AAAAAAAAMAJOECAA|1910-10-11|129|563|44|1910|2|10|11|4|1910|44|563|Tuesday|1910Q4|N|N|N|2418946|2419218|2418591|2418864|N|N|N|N|N| +2418957|AAAAAAAANAJOECAA|1910-10-12|129|563|44|1910|3|10|12|4|1910|44|563|Wednesday|1910Q4|N|N|N|2418946|2419218|2418592|2418865|N|N|N|N|N| +2418958|AAAAAAAAOAJOECAA|1910-10-13|129|563|44|1910|4|10|13|4|1910|44|563|Thursday|1910Q4|N|N|N|2418946|2419218|2418593|2418866|N|N|N|N|N| +2418959|AAAAAAAAPAJOECAA|1910-10-14|129|563|44|1910|5|10|14|4|1910|44|563|Friday|1910Q4|N|Y|N|2418946|2419218|2418594|2418867|N|N|N|N|N| +2418960|AAAAAAAAABJOECAA|1910-10-15|129|563|44|1910|6|10|15|4|1910|44|563|Saturday|1910Q4|N|Y|N|2418946|2419218|2418595|2418868|N|N|N|N|N| +2418961|AAAAAAAABBJOECAA|1910-10-16|129|563|44|1910|0|10|16|4|1910|44|563|Sunday|1910Q4|N|N|N|2418946|2419218|2418596|2418869|N|N|N|N|N| +2418962|AAAAAAAACBJOECAA|1910-10-17|129|563|44|1910|1|10|17|4|1910|44|563|Monday|1910Q4|N|N|N|2418946|2419218|2418597|2418870|N|N|N|N|N| +2418963|AAAAAAAADBJOECAA|1910-10-18|129|564|44|1910|2|10|18|4|1910|44|564|Tuesday|1910Q4|N|N|N|2418946|2419218|2418598|2418871|N|N|N|N|N| +2418964|AAAAAAAAEBJOECAA|1910-10-19|129|564|44|1910|3|10|19|4|1910|44|564|Wednesday|1910Q4|N|N|N|2418946|2419218|2418599|2418872|N|N|N|N|N| +2418965|AAAAAAAAFBJOECAA|1910-10-20|129|564|44|1910|4|10|20|4|1910|44|564|Thursday|1910Q4|N|N|N|2418946|2419218|2418600|2418873|N|N|N|N|N| +2418966|AAAAAAAAGBJOECAA|1910-10-21|129|564|44|1910|5|10|21|4|1910|44|564|Friday|1910Q4|N|Y|N|2418946|2419218|2418601|2418874|N|N|N|N|N| +2418967|AAAAAAAAHBJOECAA|1910-10-22|129|564|44|1910|6|10|22|4|1910|44|564|Saturday|1910Q4|N|Y|N|2418946|2419218|2418602|2418875|N|N|N|N|N| +2418968|AAAAAAAAIBJOECAA|1910-10-23|129|564|44|1910|0|10|23|4|1910|44|564|Sunday|1910Q4|N|N|N|2418946|2419218|2418603|2418876|N|N|N|N|N| +2418969|AAAAAAAAJBJOECAA|1910-10-24|129|564|44|1910|1|10|24|4|1910|44|564|Monday|1910Q4|N|N|N|2418946|2419218|2418604|2418877|N|N|N|N|N| +2418970|AAAAAAAAKBJOECAA|1910-10-25|129|565|44|1910|2|10|25|4|1910|44|565|Tuesday|1910Q4|N|N|N|2418946|2419218|2418605|2418878|N|N|N|N|N| +2418971|AAAAAAAALBJOECAA|1910-10-26|129|565|44|1910|3|10|26|4|1910|44|565|Wednesday|1910Q4|N|N|N|2418946|2419218|2418606|2418879|N|N|N|N|N| +2418972|AAAAAAAAMBJOECAA|1910-10-27|129|565|44|1910|4|10|27|4|1910|44|565|Thursday|1910Q4|N|N|N|2418946|2419218|2418607|2418880|N|N|N|N|N| +2418973|AAAAAAAANBJOECAA|1910-10-28|129|565|44|1910|5|10|28|4|1910|44|565|Friday|1910Q4|N|Y|N|2418946|2419218|2418608|2418881|N|N|N|N|N| +2418974|AAAAAAAAOBJOECAA|1910-10-29|129|565|44|1910|6|10|29|4|1910|44|565|Saturday|1910Q4|N|Y|N|2418946|2419218|2418609|2418882|N|N|N|N|N| +2418975|AAAAAAAAPBJOECAA|1910-10-30|129|565|44|1910|0|10|30|4|1910|44|565|Sunday|1910Q4|N|N|N|2418946|2419218|2418610|2418883|N|N|N|N|N| +2418976|AAAAAAAAACJOECAA|1910-10-31|129|565|44|1910|1|10|31|4|1910|44|565|Monday|1910Q4|N|N|N|2418946|2419218|2418611|2418884|N|N|N|N|N| +2418977|AAAAAAAABCJOECAA|1910-11-01|130|566|44|1910|2|11|1|4|1910|44|566|Tuesday|1910Q4|N|N|N|2418977|2419280|2418612|2418885|N|N|N|N|N| +2418978|AAAAAAAACCJOECAA|1910-11-02|130|566|44|1910|3|11|2|4|1910|44|566|Wednesday|1910Q4|N|N|N|2418977|2419280|2418613|2418886|N|N|N|N|N| +2418979|AAAAAAAADCJOECAA|1910-11-03|130|566|44|1910|4|11|3|4|1910|44|566|Thursday|1910Q4|N|N|N|2418977|2419280|2418614|2418887|N|N|N|N|N| +2418980|AAAAAAAAECJOECAA|1910-11-04|130|566|44|1910|5|11|4|4|1910|44|566|Friday|1910Q4|N|Y|N|2418977|2419280|2418615|2418888|N|N|N|N|N| +2418981|AAAAAAAAFCJOECAA|1910-11-05|130|566|44|1910|6|11|5|4|1910|44|566|Saturday|1910Q4|N|Y|N|2418977|2419280|2418616|2418889|N|N|N|N|N| +2418982|AAAAAAAAGCJOECAA|1910-11-06|130|566|44|1910|0|11|6|4|1910|44|566|Sunday|1910Q4|N|N|N|2418977|2419280|2418617|2418890|N|N|N|N|N| +2418983|AAAAAAAAHCJOECAA|1910-11-07|130|566|44|1910|1|11|7|4|1910|44|566|Monday|1910Q4|N|N|N|2418977|2419280|2418618|2418891|N|N|N|N|N| +2418984|AAAAAAAAICJOECAA|1910-11-08|130|567|44|1910|2|11|8|4|1910|44|567|Tuesday|1910Q4|N|N|N|2418977|2419280|2418619|2418892|N|N|N|N|N| +2418985|AAAAAAAAJCJOECAA|1910-11-09|130|567|44|1910|3|11|9|4|1910|44|567|Wednesday|1910Q4|N|N|N|2418977|2419280|2418620|2418893|N|N|N|N|N| +2418986|AAAAAAAAKCJOECAA|1910-11-10|130|567|44|1910|4|11|10|4|1910|44|567|Thursday|1910Q4|N|N|N|2418977|2419280|2418621|2418894|N|N|N|N|N| +2418987|AAAAAAAALCJOECAA|1910-11-11|130|567|44|1910|5|11|11|4|1910|44|567|Friday|1910Q4|N|Y|N|2418977|2419280|2418622|2418895|N|N|N|N|N| +2418988|AAAAAAAAMCJOECAA|1910-11-12|130|567|44|1910|6|11|12|4|1910|44|567|Saturday|1910Q4|N|Y|N|2418977|2419280|2418623|2418896|N|N|N|N|N| +2418989|AAAAAAAANCJOECAA|1910-11-13|130|567|44|1910|0|11|13|4|1910|44|567|Sunday|1910Q4|N|N|N|2418977|2419280|2418624|2418897|N|N|N|N|N| +2418990|AAAAAAAAOCJOECAA|1910-11-14|130|567|44|1910|1|11|14|4|1910|44|567|Monday|1910Q4|N|N|N|2418977|2419280|2418625|2418898|N|N|N|N|N| +2418991|AAAAAAAAPCJOECAA|1910-11-15|130|568|44|1910|2|11|15|4|1910|44|568|Tuesday|1910Q4|N|N|N|2418977|2419280|2418626|2418899|N|N|N|N|N| +2418992|AAAAAAAAADJOECAA|1910-11-16|130|568|44|1910|3|11|16|4|1910|44|568|Wednesday|1910Q4|N|N|N|2418977|2419280|2418627|2418900|N|N|N|N|N| +2418993|AAAAAAAABDJOECAA|1910-11-17|130|568|44|1910|4|11|17|4|1910|44|568|Thursday|1910Q4|N|N|N|2418977|2419280|2418628|2418901|N|N|N|N|N| +2418994|AAAAAAAACDJOECAA|1910-11-18|130|568|44|1910|5|11|18|4|1910|44|568|Friday|1910Q4|N|Y|N|2418977|2419280|2418629|2418902|N|N|N|N|N| +2418995|AAAAAAAADDJOECAA|1910-11-19|130|568|44|1910|6|11|19|4|1910|44|568|Saturday|1910Q4|N|Y|N|2418977|2419280|2418630|2418903|N|N|N|N|N| +2418996|AAAAAAAAEDJOECAA|1910-11-20|130|568|44|1910|0|11|20|4|1910|44|568|Sunday|1910Q4|N|N|N|2418977|2419280|2418631|2418904|N|N|N|N|N| +2418997|AAAAAAAAFDJOECAA|1910-11-21|130|568|44|1910|1|11|21|4|1910|44|568|Monday|1910Q4|N|N|N|2418977|2419280|2418632|2418905|N|N|N|N|N| +2418998|AAAAAAAAGDJOECAA|1910-11-22|130|569|44|1910|2|11|22|4|1910|44|569|Tuesday|1910Q4|N|N|N|2418977|2419280|2418633|2418906|N|N|N|N|N| +2418999|AAAAAAAAHDJOECAA|1910-11-23|130|569|44|1910|3|11|23|4|1910|44|569|Wednesday|1910Q4|N|N|N|2418977|2419280|2418634|2418907|N|N|N|N|N| +2419000|AAAAAAAAIDJOECAA|1910-11-24|130|569|44|1910|4|11|24|4|1910|44|569|Thursday|1910Q4|N|N|N|2418977|2419280|2418635|2418908|N|N|N|N|N| +2419001|AAAAAAAAJDJOECAA|1910-11-25|130|569|44|1910|5|11|25|4|1910|44|569|Friday|1910Q4|N|Y|N|2418977|2419280|2418636|2418909|N|N|N|N|N| +2419002|AAAAAAAAKDJOECAA|1910-11-26|130|569|44|1910|6|11|26|4|1910|44|569|Saturday|1910Q4|N|Y|N|2418977|2419280|2418637|2418910|N|N|N|N|N| +2419003|AAAAAAAALDJOECAA|1910-11-27|130|569|44|1910|0|11|27|4|1910|44|569|Sunday|1910Q4|N|N|N|2418977|2419280|2418638|2418911|N|N|N|N|N| +2419004|AAAAAAAAMDJOECAA|1910-11-28|130|569|44|1910|1|11|28|4|1910|44|569|Monday|1910Q4|N|N|N|2418977|2419280|2418639|2418912|N|N|N|N|N| +2419005|AAAAAAAANDJOECAA|1910-11-29|130|570|44|1910|2|11|29|4|1910|44|570|Tuesday|1910Q4|N|N|N|2418977|2419280|2418640|2418913|N|N|N|N|N| +2419006|AAAAAAAAODJOECAA|1910-11-30|130|570|44|1910|3|11|30|4|1910|44|570|Wednesday|1910Q4|N|N|N|2418977|2419280|2418641|2418914|N|N|N|N|N| +2419007|AAAAAAAAPDJOECAA|1910-12-01|131|570|45|1910|4|12|1|4|1910|45|570|Thursday|1910Q4|N|N|N|2419007|2419340|2418642|2418915|N|N|N|N|N| +2419008|AAAAAAAAAEJOECAA|1910-12-02|131|570|45|1910|5|12|2|4|1910|45|570|Friday|1910Q4|N|Y|N|2419007|2419340|2418643|2418916|N|N|N|N|N| +2419009|AAAAAAAABEJOECAA|1910-12-03|131|570|45|1910|6|12|3|4|1910|45|570|Saturday|1910Q4|N|Y|N|2419007|2419340|2418644|2418917|N|N|N|N|N| +2419010|AAAAAAAACEJOECAA|1910-12-04|131|570|45|1910|0|12|4|4|1910|45|570|Sunday|1910Q4|N|N|N|2419007|2419340|2418645|2418918|N|N|N|N|N| +2419011|AAAAAAAADEJOECAA|1910-12-05|131|570|45|1910|1|12|5|4|1910|45|570|Monday|1910Q4|N|N|N|2419007|2419340|2418646|2418919|N|N|N|N|N| +2419012|AAAAAAAAEEJOECAA|1910-12-06|131|571|45|1910|2|12|6|4|1910|45|571|Tuesday|1910Q4|N|N|N|2419007|2419340|2418647|2418920|N|N|N|N|N| +2419013|AAAAAAAAFEJOECAA|1910-12-07|131|571|45|1910|3|12|7|4|1910|45|571|Wednesday|1910Q4|N|N|N|2419007|2419340|2418648|2418921|N|N|N|N|N| +2419014|AAAAAAAAGEJOECAA|1910-12-08|131|571|45|1910|4|12|8|4|1910|45|571|Thursday|1910Q4|N|N|N|2419007|2419340|2418649|2418922|N|N|N|N|N| +2419015|AAAAAAAAHEJOECAA|1910-12-09|131|571|45|1910|5|12|9|4|1910|45|571|Friday|1910Q4|N|Y|N|2419007|2419340|2418650|2418923|N|N|N|N|N| +2419016|AAAAAAAAIEJOECAA|1910-12-10|131|571|45|1910|6|12|10|4|1910|45|571|Saturday|1910Q4|N|Y|N|2419007|2419340|2418651|2418924|N|N|N|N|N| +2419017|AAAAAAAAJEJOECAA|1910-12-11|131|571|45|1910|0|12|11|4|1910|45|571|Sunday|1910Q4|N|N|N|2419007|2419340|2418652|2418925|N|N|N|N|N| +2419018|AAAAAAAAKEJOECAA|1910-12-12|131|571|45|1910|1|12|12|4|1910|45|571|Monday|1910Q4|N|N|N|2419007|2419340|2418653|2418926|N|N|N|N|N| +2419019|AAAAAAAALEJOECAA|1910-12-13|131|572|45|1910|2|12|13|4|1910|45|572|Tuesday|1910Q4|N|N|N|2419007|2419340|2418654|2418927|N|N|N|N|N| +2419020|AAAAAAAAMEJOECAA|1910-12-14|131|572|45|1910|3|12|14|4|1910|45|572|Wednesday|1910Q4|N|N|N|2419007|2419340|2418655|2418928|N|N|N|N|N| +2419021|AAAAAAAANEJOECAA|1910-12-15|131|572|45|1910|4|12|15|4|1910|45|572|Thursday|1910Q4|N|N|N|2419007|2419340|2418656|2418929|N|N|N|N|N| +2419022|AAAAAAAAOEJOECAA|1910-12-16|131|572|45|1910|5|12|16|4|1910|45|572|Friday|1910Q4|N|Y|N|2419007|2419340|2418657|2418930|N|N|N|N|N| +2419023|AAAAAAAAPEJOECAA|1910-12-17|131|572|45|1910|6|12|17|4|1910|45|572|Saturday|1910Q4|N|Y|N|2419007|2419340|2418658|2418931|N|N|N|N|N| +2419024|AAAAAAAAAFJOECAA|1910-12-18|131|572|45|1910|0|12|18|4|1910|45|572|Sunday|1910Q4|N|N|N|2419007|2419340|2418659|2418932|N|N|N|N|N| +2419025|AAAAAAAABFJOECAA|1910-12-19|131|572|45|1910|1|12|19|4|1910|45|572|Monday|1910Q4|N|N|N|2419007|2419340|2418660|2418933|N|N|N|N|N| +2419026|AAAAAAAACFJOECAA|1910-12-20|131|573|45|1910|2|12|20|4|1910|45|573|Tuesday|1910Q4|N|N|N|2419007|2419340|2418661|2418934|N|N|N|N|N| +2419027|AAAAAAAADFJOECAA|1910-12-21|131|573|45|1910|3|12|21|4|1910|45|573|Wednesday|1910Q4|N|N|N|2419007|2419340|2418662|2418935|N|N|N|N|N| +2419028|AAAAAAAAEFJOECAA|1910-12-22|131|573|45|1910|4|12|22|4|1910|45|573|Thursday|1910Q4|N|N|N|2419007|2419340|2418663|2418936|N|N|N|N|N| +2419029|AAAAAAAAFFJOECAA|1910-12-23|131|573|45|1910|5|12|23|4|1910|45|573|Friday|1910Q4|N|Y|N|2419007|2419340|2418664|2418937|N|N|N|N|N| +2419030|AAAAAAAAGFJOECAA|1910-12-24|131|573|45|1910|6|12|24|4|1910|45|573|Saturday|1910Q4|N|Y|N|2419007|2419340|2418665|2418938|N|N|N|N|N| +2419031|AAAAAAAAHFJOECAA|1910-12-25|131|573|45|1910|0|12|25|4|1910|45|573|Sunday|1910Q4|N|N|N|2419007|2419340|2418666|2418939|N|N|N|N|N| +2419032|AAAAAAAAIFJOECAA|1910-12-26|131|573|45|1910|1|12|26|4|1910|45|573|Monday|1910Q4|Y|N|N|2419007|2419340|2418667|2418940|N|N|N|N|N| +2419033|AAAAAAAAJFJOECAA|1910-12-27|131|574|45|1910|2|12|27|4|1910|45|574|Tuesday|1910Q4|N|N|Y|2419007|2419340|2418668|2418941|N|N|N|N|N| +2419034|AAAAAAAAKFJOECAA|1910-12-28|131|574|45|1910|3|12|28|4|1910|45|574|Wednesday|1910Q4|N|N|N|2419007|2419340|2418669|2418942|N|N|N|N|N| +2419035|AAAAAAAALFJOECAA|1910-12-29|131|574|45|1910|4|12|29|4|1910|45|574|Thursday|1910Q4|N|N|N|2419007|2419340|2418670|2418943|N|N|N|N|N| +2419036|AAAAAAAAMFJOECAA|1910-12-30|131|574|45|1910|5|12|30|4|1910|45|574|Friday|1910Q4|N|Y|N|2419007|2419340|2418671|2418944|N|N|N|N|N| +2419037|AAAAAAAANFJOECAA|1910-12-31|131|574|45|1910|6|12|31|4|1910|45|574|Saturday|1910Q4|N|Y|N|2419007|2419340|2418672|2418945|N|N|N|N|N| +2419038|AAAAAAAAOFJOECAA|1911-01-01|132|574|45|1911|0|1|1|1|1911|45|574|Sunday|1911Q1|Y|N|N|2419038|2419037|2418673|2418946|N|N|N|N|N| +2419039|AAAAAAAAPFJOECAA|1911-01-02|132|574|45|1911|1|1|2|1|1911|45|574|Monday|1911Q1|N|N|Y|2419038|2419037|2418674|2418947|N|N|N|N|N| +2419040|AAAAAAAAAGJOECAA|1911-01-03|132|575|45|1911|2|1|3|1|1911|45|575|Tuesday|1911Q1|N|N|N|2419038|2419037|2418675|2418948|N|N|N|N|N| +2419041|AAAAAAAABGJOECAA|1911-01-04|132|575|45|1911|3|1|4|1|1911|45|575|Wednesday|1911Q1|N|N|N|2419038|2419037|2418676|2418949|N|N|N|N|N| +2419042|AAAAAAAACGJOECAA|1911-01-05|132|575|45|1911|4|1|5|1|1911|45|575|Thursday|1911Q1|N|N|N|2419038|2419037|2418677|2418950|N|N|N|N|N| +2419043|AAAAAAAADGJOECAA|1911-01-06|132|575|45|1911|5|1|6|1|1911|45|575|Friday|1911Q1|N|Y|N|2419038|2419037|2418678|2418951|N|N|N|N|N| +2419044|AAAAAAAAEGJOECAA|1911-01-07|132|575|45|1911|6|1|7|1|1911|45|575|Saturday|1911Q1|N|Y|N|2419038|2419037|2418679|2418952|N|N|N|N|N| +2419045|AAAAAAAAFGJOECAA|1911-01-08|132|575|45|1911|0|1|8|1|1911|45|575|Sunday|1911Q1|N|N|N|2419038|2419037|2418680|2418953|N|N|N|N|N| +2419046|AAAAAAAAGGJOECAA|1911-01-09|132|575|45|1911|1|1|9|1|1911|45|575|Monday|1911Q1|N|N|N|2419038|2419037|2418681|2418954|N|N|N|N|N| +2419047|AAAAAAAAHGJOECAA|1911-01-10|132|576|45|1911|2|1|10|1|1911|45|576|Tuesday|1911Q1|N|N|N|2419038|2419037|2418682|2418955|N|N|N|N|N| +2419048|AAAAAAAAIGJOECAA|1911-01-11|132|576|45|1911|3|1|11|1|1911|45|576|Wednesday|1911Q1|N|N|N|2419038|2419037|2418683|2418956|N|N|N|N|N| +2419049|AAAAAAAAJGJOECAA|1911-01-12|132|576|45|1911|4|1|12|1|1911|45|576|Thursday|1911Q1|N|N|N|2419038|2419037|2418684|2418957|N|N|N|N|N| +2419050|AAAAAAAAKGJOECAA|1911-01-13|132|576|45|1911|5|1|13|1|1911|45|576|Friday|1911Q1|N|Y|N|2419038|2419037|2418685|2418958|N|N|N|N|N| +2419051|AAAAAAAALGJOECAA|1911-01-14|132|576|45|1911|6|1|14|1|1911|45|576|Saturday|1911Q1|N|Y|N|2419038|2419037|2418686|2418959|N|N|N|N|N| +2419052|AAAAAAAAMGJOECAA|1911-01-15|132|576|45|1911|0|1|15|1|1911|45|576|Sunday|1911Q1|N|N|N|2419038|2419037|2418687|2418960|N|N|N|N|N| +2419053|AAAAAAAANGJOECAA|1911-01-16|132|576|45|1911|1|1|16|1|1911|45|576|Monday|1911Q1|N|N|N|2419038|2419037|2418688|2418961|N|N|N|N|N| +2419054|AAAAAAAAOGJOECAA|1911-01-17|132|577|45|1911|2|1|17|1|1911|45|577|Tuesday|1911Q1|N|N|N|2419038|2419037|2418689|2418962|N|N|N|N|N| +2419055|AAAAAAAAPGJOECAA|1911-01-18|132|577|45|1911|3|1|18|1|1911|45|577|Wednesday|1911Q1|N|N|N|2419038|2419037|2418690|2418963|N|N|N|N|N| +2419056|AAAAAAAAAHJOECAA|1911-01-19|132|577|45|1911|4|1|19|1|1911|45|577|Thursday|1911Q1|N|N|N|2419038|2419037|2418691|2418964|N|N|N|N|N| +2419057|AAAAAAAABHJOECAA|1911-01-20|132|577|45|1911|5|1|20|1|1911|45|577|Friday|1911Q1|N|Y|N|2419038|2419037|2418692|2418965|N|N|N|N|N| +2419058|AAAAAAAACHJOECAA|1911-01-21|132|577|45|1911|6|1|21|1|1911|45|577|Saturday|1911Q1|N|Y|N|2419038|2419037|2418693|2418966|N|N|N|N|N| +2419059|AAAAAAAADHJOECAA|1911-01-22|132|577|45|1911|0|1|22|1|1911|45|577|Sunday|1911Q1|N|N|N|2419038|2419037|2418694|2418967|N|N|N|N|N| +2419060|AAAAAAAAEHJOECAA|1911-01-23|132|577|45|1911|1|1|23|1|1911|45|577|Monday|1911Q1|N|N|N|2419038|2419037|2418695|2418968|N|N|N|N|N| +2419061|AAAAAAAAFHJOECAA|1911-01-24|132|578|45|1911|2|1|24|1|1911|45|578|Tuesday|1911Q1|N|N|N|2419038|2419037|2418696|2418969|N|N|N|N|N| +2419062|AAAAAAAAGHJOECAA|1911-01-25|132|578|45|1911|3|1|25|1|1911|45|578|Wednesday|1911Q1|N|N|N|2419038|2419037|2418697|2418970|N|N|N|N|N| +2419063|AAAAAAAAHHJOECAA|1911-01-26|132|578|45|1911|4|1|26|1|1911|45|578|Thursday|1911Q1|N|N|N|2419038|2419037|2418698|2418971|N|N|N|N|N| +2419064|AAAAAAAAIHJOECAA|1911-01-27|132|578|45|1911|5|1|27|1|1911|45|578|Friday|1911Q1|N|Y|N|2419038|2419037|2418699|2418972|N|N|N|N|N| +2419065|AAAAAAAAJHJOECAA|1911-01-28|132|578|45|1911|6|1|28|1|1911|45|578|Saturday|1911Q1|N|Y|N|2419038|2419037|2418700|2418973|N|N|N|N|N| +2419066|AAAAAAAAKHJOECAA|1911-01-29|132|578|45|1911|0|1|29|1|1911|45|578|Sunday|1911Q1|N|N|N|2419038|2419037|2418701|2418974|N|N|N|N|N| +2419067|AAAAAAAALHJOECAA|1911-01-30|132|578|45|1911|1|1|30|1|1911|45|578|Monday|1911Q1|N|N|N|2419038|2419037|2418702|2418975|N|N|N|N|N| +2419068|AAAAAAAAMHJOECAA|1911-01-31|132|579|45|1911|2|1|31|1|1911|45|579|Tuesday|1911Q1|N|N|N|2419038|2419037|2418703|2418976|N|N|N|N|N| +2419069|AAAAAAAANHJOECAA|1911-02-01|133|579|45|1911|3|2|1|1|1911|45|579|Wednesday|1911Q1|N|N|N|2419069|2419099|2418704|2418977|N|N|N|N|N| +2419070|AAAAAAAAOHJOECAA|1911-02-02|133|579|45|1911|4|2|2|1|1911|45|579|Thursday|1911Q1|N|N|N|2419069|2419099|2418705|2418978|N|N|N|N|N| +2419071|AAAAAAAAPHJOECAA|1911-02-03|133|579|45|1911|5|2|3|1|1911|45|579|Friday|1911Q1|N|Y|N|2419069|2419099|2418706|2418979|N|N|N|N|N| +2419072|AAAAAAAAAIJOECAA|1911-02-04|133|579|45|1911|6|2|4|1|1911|45|579|Saturday|1911Q1|N|Y|N|2419069|2419099|2418707|2418980|N|N|N|N|N| +2419073|AAAAAAAABIJOECAA|1911-02-05|133|579|45|1911|0|2|5|1|1911|45|579|Sunday|1911Q1|N|N|N|2419069|2419099|2418708|2418981|N|N|N|N|N| +2419074|AAAAAAAACIJOECAA|1911-02-06|133|579|45|1911|1|2|6|1|1911|45|579|Monday|1911Q1|N|N|N|2419069|2419099|2418709|2418982|N|N|N|N|N| +2419075|AAAAAAAADIJOECAA|1911-02-07|133|580|45|1911|2|2|7|1|1911|45|580|Tuesday|1911Q1|N|N|N|2419069|2419099|2418710|2418983|N|N|N|N|N| +2419076|AAAAAAAAEIJOECAA|1911-02-08|133|580|45|1911|3|2|8|1|1911|45|580|Wednesday|1911Q1|N|N|N|2419069|2419099|2418711|2418984|N|N|N|N|N| +2419077|AAAAAAAAFIJOECAA|1911-02-09|133|580|45|1911|4|2|9|1|1911|45|580|Thursday|1911Q1|N|N|N|2419069|2419099|2418712|2418985|N|N|N|N|N| +2419078|AAAAAAAAGIJOECAA|1911-02-10|133|580|45|1911|5|2|10|1|1911|45|580|Friday|1911Q1|N|Y|N|2419069|2419099|2418713|2418986|N|N|N|N|N| +2419079|AAAAAAAAHIJOECAA|1911-02-11|133|580|45|1911|6|2|11|1|1911|45|580|Saturday|1911Q1|N|Y|N|2419069|2419099|2418714|2418987|N|N|N|N|N| +2419080|AAAAAAAAIIJOECAA|1911-02-12|133|580|45|1911|0|2|12|1|1911|45|580|Sunday|1911Q1|N|N|N|2419069|2419099|2418715|2418988|N|N|N|N|N| +2419081|AAAAAAAAJIJOECAA|1911-02-13|133|580|45|1911|1|2|13|1|1911|45|580|Monday|1911Q1|N|N|N|2419069|2419099|2418716|2418989|N|N|N|N|N| +2419082|AAAAAAAAKIJOECAA|1911-02-14|133|581|45|1911|2|2|14|1|1911|45|581|Tuesday|1911Q1|N|N|N|2419069|2419099|2418717|2418990|N|N|N|N|N| +2419083|AAAAAAAALIJOECAA|1911-02-15|133|581|45|1911|3|2|15|1|1911|45|581|Wednesday|1911Q1|N|N|N|2419069|2419099|2418718|2418991|N|N|N|N|N| +2419084|AAAAAAAAMIJOECAA|1911-02-16|133|581|45|1911|4|2|16|1|1911|45|581|Thursday|1911Q1|N|N|N|2419069|2419099|2418719|2418992|N|N|N|N|N| +2419085|AAAAAAAANIJOECAA|1911-02-17|133|581|45|1911|5|2|17|1|1911|45|581|Friday|1911Q1|N|Y|N|2419069|2419099|2418720|2418993|N|N|N|N|N| +2419086|AAAAAAAAOIJOECAA|1911-02-18|133|581|45|1911|6|2|18|1|1911|45|581|Saturday|1911Q1|N|Y|N|2419069|2419099|2418721|2418994|N|N|N|N|N| +2419087|AAAAAAAAPIJOECAA|1911-02-19|133|581|45|1911|0|2|19|1|1911|45|581|Sunday|1911Q1|N|N|N|2419069|2419099|2418722|2418995|N|N|N|N|N| +2419088|AAAAAAAAAJJOECAA|1911-02-20|133|581|45|1911|1|2|20|1|1911|45|581|Monday|1911Q1|N|N|N|2419069|2419099|2418723|2418996|N|N|N|N|N| +2419089|AAAAAAAABJJOECAA|1911-02-21|133|582|45|1911|2|2|21|1|1911|45|582|Tuesday|1911Q1|N|N|N|2419069|2419099|2418724|2418997|N|N|N|N|N| +2419090|AAAAAAAACJJOECAA|1911-02-22|133|582|45|1911|3|2|22|1|1911|45|582|Wednesday|1911Q1|N|N|N|2419069|2419099|2418725|2418998|N|N|N|N|N| +2419091|AAAAAAAADJJOECAA|1911-02-23|133|582|45|1911|4|2|23|1|1911|45|582|Thursday|1911Q1|N|N|N|2419069|2419099|2418726|2418999|N|N|N|N|N| +2419092|AAAAAAAAEJJOECAA|1911-02-24|133|582|45|1911|5|2|24|1|1911|45|582|Friday|1911Q1|N|Y|N|2419069|2419099|2418727|2419000|N|N|N|N|N| +2419093|AAAAAAAAFJJOECAA|1911-02-25|133|582|45|1911|6|2|25|1|1911|45|582|Saturday|1911Q1|N|Y|N|2419069|2419099|2418728|2419001|N|N|N|N|N| +2419094|AAAAAAAAGJJOECAA|1911-02-26|133|582|45|1911|0|2|26|1|1911|45|582|Sunday|1911Q1|N|N|N|2419069|2419099|2418729|2419002|N|N|N|N|N| +2419095|AAAAAAAAHJJOECAA|1911-02-27|133|582|45|1911|1|2|27|1|1911|45|582|Monday|1911Q1|N|N|N|2419069|2419099|2418730|2419003|N|N|N|N|N| +2419096|AAAAAAAAIJJOECAA|1911-02-28|133|583|45|1911|2|2|28|1|1911|45|583|Tuesday|1911Q1|N|N|N|2419069|2419099|2418731|2419004|N|N|N|N|N| +2419097|AAAAAAAAJJJOECAA|1911-03-01|134|583|46|1911|3|3|1|1|1911|46|583|Wednesday|1911Q1|N|N|N|2419097|2419155|2418732|2419005|N|N|N|N|N| +2419098|AAAAAAAAKJJOECAA|1911-03-02|134|583|46|1911|4|3|2|1|1911|46|583|Thursday|1911Q1|N|N|N|2419097|2419155|2418733|2419006|N|N|N|N|N| +2419099|AAAAAAAALJJOECAA|1911-03-03|134|583|46|1911|5|3|3|1|1911|46|583|Friday|1911Q1|N|Y|N|2419097|2419155|2418734|2419007|N|N|N|N|N| +2419100|AAAAAAAAMJJOECAA|1911-03-04|134|583|46|1911|6|3|4|1|1911|46|583|Saturday|1911Q1|N|Y|N|2419097|2419155|2418735|2419008|N|N|N|N|N| +2419101|AAAAAAAANJJOECAA|1911-03-05|134|583|46|1911|0|3|5|1|1911|46|583|Sunday|1911Q1|N|N|N|2419097|2419155|2418736|2419009|N|N|N|N|N| +2419102|AAAAAAAAOJJOECAA|1911-03-06|134|583|46|1911|1|3|6|1|1911|46|583|Monday|1911Q1|N|N|N|2419097|2419155|2418737|2419010|N|N|N|N|N| +2419103|AAAAAAAAPJJOECAA|1911-03-07|134|584|46|1911|2|3|7|1|1911|46|584|Tuesday|1911Q1|N|N|N|2419097|2419155|2418738|2419011|N|N|N|N|N| +2419104|AAAAAAAAAKJOECAA|1911-03-08|134|584|46|1911|3|3|8|1|1911|46|584|Wednesday|1911Q1|N|N|N|2419097|2419155|2418739|2419012|N|N|N|N|N| +2419105|AAAAAAAABKJOECAA|1911-03-09|134|584|46|1911|4|3|9|1|1911|46|584|Thursday|1911Q1|N|N|N|2419097|2419155|2418740|2419013|N|N|N|N|N| +2419106|AAAAAAAACKJOECAA|1911-03-10|134|584|46|1911|5|3|10|1|1911|46|584|Friday|1911Q1|N|Y|N|2419097|2419155|2418741|2419014|N|N|N|N|N| +2419107|AAAAAAAADKJOECAA|1911-03-11|134|584|46|1911|6|3|11|1|1911|46|584|Saturday|1911Q1|N|Y|N|2419097|2419155|2418742|2419015|N|N|N|N|N| +2419108|AAAAAAAAEKJOECAA|1911-03-12|134|584|46|1911|0|3|12|1|1911|46|584|Sunday|1911Q1|N|N|N|2419097|2419155|2418743|2419016|N|N|N|N|N| +2419109|AAAAAAAAFKJOECAA|1911-03-13|134|584|46|1911|1|3|13|1|1911|46|584|Monday|1911Q1|N|N|N|2419097|2419155|2418744|2419017|N|N|N|N|N| +2419110|AAAAAAAAGKJOECAA|1911-03-14|134|585|46|1911|2|3|14|1|1911|46|585|Tuesday|1911Q1|N|N|N|2419097|2419155|2418745|2419018|N|N|N|N|N| +2419111|AAAAAAAAHKJOECAA|1911-03-15|134|585|46|1911|3|3|15|1|1911|46|585|Wednesday|1911Q1|N|N|N|2419097|2419155|2418746|2419019|N|N|N|N|N| +2419112|AAAAAAAAIKJOECAA|1911-03-16|134|585|46|1911|4|3|16|1|1911|46|585|Thursday|1911Q1|N|N|N|2419097|2419155|2418747|2419020|N|N|N|N|N| +2419113|AAAAAAAAJKJOECAA|1911-03-17|134|585|46|1911|5|3|17|1|1911|46|585|Friday|1911Q1|N|Y|N|2419097|2419155|2418748|2419021|N|N|N|N|N| +2419114|AAAAAAAAKKJOECAA|1911-03-18|134|585|46|1911|6|3|18|1|1911|46|585|Saturday|1911Q1|N|Y|N|2419097|2419155|2418749|2419022|N|N|N|N|N| +2419115|AAAAAAAALKJOECAA|1911-03-19|134|585|46|1911|0|3|19|1|1911|46|585|Sunday|1911Q1|N|N|N|2419097|2419155|2418750|2419023|N|N|N|N|N| +2419116|AAAAAAAAMKJOECAA|1911-03-20|134|585|46|1911|1|3|20|1|1911|46|585|Monday|1911Q1|N|N|N|2419097|2419155|2418751|2419024|N|N|N|N|N| +2419117|AAAAAAAANKJOECAA|1911-03-21|134|586|46|1911|2|3|21|1|1911|46|586|Tuesday|1911Q1|N|N|N|2419097|2419155|2418752|2419025|N|N|N|N|N| +2419118|AAAAAAAAOKJOECAA|1911-03-22|134|586|46|1911|3|3|22|1|1911|46|586|Wednesday|1911Q1|N|N|N|2419097|2419155|2418753|2419026|N|N|N|N|N| +2419119|AAAAAAAAPKJOECAA|1911-03-23|134|586|46|1911|4|3|23|1|1911|46|586|Thursday|1911Q1|N|N|N|2419097|2419155|2418754|2419027|N|N|N|N|N| +2419120|AAAAAAAAALJOECAA|1911-03-24|134|586|46|1911|5|3|24|1|1911|46|586|Friday|1911Q1|N|Y|N|2419097|2419155|2418755|2419028|N|N|N|N|N| +2419121|AAAAAAAABLJOECAA|1911-03-25|134|586|46|1911|6|3|25|1|1911|46|586|Saturday|1911Q1|N|Y|N|2419097|2419155|2418756|2419029|N|N|N|N|N| +2419122|AAAAAAAACLJOECAA|1911-03-26|134|586|46|1911|0|3|26|1|1911|46|586|Sunday|1911Q1|N|N|N|2419097|2419155|2418757|2419030|N|N|N|N|N| +2419123|AAAAAAAADLJOECAA|1911-03-27|134|586|46|1911|1|3|27|1|1911|46|586|Monday|1911Q1|N|N|N|2419097|2419155|2418758|2419031|N|N|N|N|N| +2419124|AAAAAAAAELJOECAA|1911-03-28|134|587|46|1911|2|3|28|1|1911|46|587|Tuesday|1911Q1|N|N|N|2419097|2419155|2418759|2419032|N|N|N|N|N| +2419125|AAAAAAAAFLJOECAA|1911-03-29|134|587|46|1911|3|3|29|1|1911|46|587|Wednesday|1911Q1|N|N|N|2419097|2419155|2418760|2419033|N|N|N|N|N| +2419126|AAAAAAAAGLJOECAA|1911-03-30|134|587|46|1911|4|3|30|1|1911|46|587|Thursday|1911Q1|N|N|N|2419097|2419155|2418761|2419034|N|N|N|N|N| +2419127|AAAAAAAAHLJOECAA|1911-03-31|134|587|46|1911|5|3|31|1|1911|46|587|Friday|1911Q1|N|Y|N|2419097|2419155|2418762|2419035|N|N|N|N|N| +2419128|AAAAAAAAILJOECAA|1911-04-01|135|587|46|1911|6|4|1|1|1911|46|587|Saturday|1911Q1|N|Y|N|2419128|2419217|2418763|2419038|N|N|N|N|N| +2419129|AAAAAAAAJLJOECAA|1911-04-02|135|587|46|1911|0|4|2|2|1911|46|587|Sunday|1911Q2|N|N|N|2419128|2419217|2418764|2419039|N|N|N|N|N| +2419130|AAAAAAAAKLJOECAA|1911-04-03|135|587|46|1911|1|4|3|2|1911|46|587|Monday|1911Q2|N|N|N|2419128|2419217|2418765|2419040|N|N|N|N|N| +2419131|AAAAAAAALLJOECAA|1911-04-04|135|588|46|1911|2|4|4|2|1911|46|588|Tuesday|1911Q2|N|N|N|2419128|2419217|2418766|2419041|N|N|N|N|N| +2419132|AAAAAAAAMLJOECAA|1911-04-05|135|588|46|1911|3|4|5|2|1911|46|588|Wednesday|1911Q2|N|N|N|2419128|2419217|2418767|2419042|N|N|N|N|N| +2419133|AAAAAAAANLJOECAA|1911-04-06|135|588|46|1911|4|4|6|2|1911|46|588|Thursday|1911Q2|N|N|N|2419128|2419217|2418768|2419043|N|N|N|N|N| +2419134|AAAAAAAAOLJOECAA|1911-04-07|135|588|46|1911|5|4|7|2|1911|46|588|Friday|1911Q2|N|Y|N|2419128|2419217|2418769|2419044|N|N|N|N|N| +2419135|AAAAAAAAPLJOECAA|1911-04-08|135|588|46|1911|6|4|8|2|1911|46|588|Saturday|1911Q2|N|Y|N|2419128|2419217|2418770|2419045|N|N|N|N|N| +2419136|AAAAAAAAAMJOECAA|1911-04-09|135|588|46|1911|0|4|9|2|1911|46|588|Sunday|1911Q2|N|N|N|2419128|2419217|2418771|2419046|N|N|N|N|N| +2419137|AAAAAAAABMJOECAA|1911-04-10|135|588|46|1911|1|4|10|2|1911|46|588|Monday|1911Q2|N|N|N|2419128|2419217|2418772|2419047|N|N|N|N|N| +2419138|AAAAAAAACMJOECAA|1911-04-11|135|589|46|1911|2|4|11|2|1911|46|589|Tuesday|1911Q2|N|N|N|2419128|2419217|2418773|2419048|N|N|N|N|N| +2419139|AAAAAAAADMJOECAA|1911-04-12|135|589|46|1911|3|4|12|2|1911|46|589|Wednesday|1911Q2|N|N|N|2419128|2419217|2418774|2419049|N|N|N|N|N| +2419140|AAAAAAAAEMJOECAA|1911-04-13|135|589|46|1911|4|4|13|2|1911|46|589|Thursday|1911Q2|N|N|N|2419128|2419217|2418775|2419050|N|N|N|N|N| +2419141|AAAAAAAAFMJOECAA|1911-04-14|135|589|46|1911|5|4|14|2|1911|46|589|Friday|1911Q2|N|Y|N|2419128|2419217|2418776|2419051|N|N|N|N|N| +2419142|AAAAAAAAGMJOECAA|1911-04-15|135|589|46|1911|6|4|15|2|1911|46|589|Saturday|1911Q2|N|Y|N|2419128|2419217|2418777|2419052|N|N|N|N|N| +2419143|AAAAAAAAHMJOECAA|1911-04-16|135|589|46|1911|0|4|16|2|1911|46|589|Sunday|1911Q2|N|N|N|2419128|2419217|2418778|2419053|N|N|N|N|N| +2419144|AAAAAAAAIMJOECAA|1911-04-17|135|589|46|1911|1|4|17|2|1911|46|589|Monday|1911Q2|N|N|N|2419128|2419217|2418779|2419054|N|N|N|N|N| +2419145|AAAAAAAAJMJOECAA|1911-04-18|135|590|46|1911|2|4|18|2|1911|46|590|Tuesday|1911Q2|N|N|N|2419128|2419217|2418780|2419055|N|N|N|N|N| +2419146|AAAAAAAAKMJOECAA|1911-04-19|135|590|46|1911|3|4|19|2|1911|46|590|Wednesday|1911Q2|N|N|N|2419128|2419217|2418781|2419056|N|N|N|N|N| +2419147|AAAAAAAALMJOECAA|1911-04-20|135|590|46|1911|4|4|20|2|1911|46|590|Thursday|1911Q2|N|N|N|2419128|2419217|2418782|2419057|N|N|N|N|N| +2419148|AAAAAAAAMMJOECAA|1911-04-21|135|590|46|1911|5|4|21|2|1911|46|590|Friday|1911Q2|N|Y|N|2419128|2419217|2418783|2419058|N|N|N|N|N| +2419149|AAAAAAAANMJOECAA|1911-04-22|135|590|46|1911|6|4|22|2|1911|46|590|Saturday|1911Q2|N|Y|N|2419128|2419217|2418784|2419059|N|N|N|N|N| +2419150|AAAAAAAAOMJOECAA|1911-04-23|135|590|46|1911|0|4|23|2|1911|46|590|Sunday|1911Q2|N|N|N|2419128|2419217|2418785|2419060|N|N|N|N|N| +2419151|AAAAAAAAPMJOECAA|1911-04-24|135|590|46|1911|1|4|24|2|1911|46|590|Monday|1911Q2|N|N|N|2419128|2419217|2418786|2419061|N|N|N|N|N| +2419152|AAAAAAAAANJOECAA|1911-04-25|135|591|46|1911|2|4|25|2|1911|46|591|Tuesday|1911Q2|N|N|N|2419128|2419217|2418787|2419062|N|N|N|N|N| +2419153|AAAAAAAABNJOECAA|1911-04-26|135|591|46|1911|3|4|26|2|1911|46|591|Wednesday|1911Q2|N|N|N|2419128|2419217|2418788|2419063|N|N|N|N|N| +2419154|AAAAAAAACNJOECAA|1911-04-27|135|591|46|1911|4|4|27|2|1911|46|591|Thursday|1911Q2|N|N|N|2419128|2419217|2418789|2419064|N|N|N|N|N| +2419155|AAAAAAAADNJOECAA|1911-04-28|135|591|46|1911|5|4|28|2|1911|46|591|Friday|1911Q2|N|Y|N|2419128|2419217|2418790|2419065|N|N|N|N|N| +2419156|AAAAAAAAENJOECAA|1911-04-29|135|591|46|1911|6|4|29|2|1911|46|591|Saturday|1911Q2|N|Y|N|2419128|2419217|2418791|2419066|N|N|N|N|N| +2419157|AAAAAAAAFNJOECAA|1911-04-30|135|591|46|1911|0|4|30|2|1911|46|591|Sunday|1911Q2|N|N|N|2419128|2419217|2418792|2419067|N|N|N|N|N| +2419158|AAAAAAAAGNJOECAA|1911-05-01|136|591|46|1911|1|5|1|2|1911|46|591|Monday|1911Q2|N|N|N|2419158|2419277|2418793|2419068|N|N|N|N|N| +2419159|AAAAAAAAHNJOECAA|1911-05-02|136|592|46|1911|2|5|2|2|1911|46|592|Tuesday|1911Q2|N|N|N|2419158|2419277|2418794|2419069|N|N|N|N|N| +2419160|AAAAAAAAINJOECAA|1911-05-03|136|592|46|1911|3|5|3|2|1911|46|592|Wednesday|1911Q2|N|N|N|2419158|2419277|2418795|2419070|N|N|N|N|N| +2419161|AAAAAAAAJNJOECAA|1911-05-04|136|592|46|1911|4|5|4|2|1911|46|592|Thursday|1911Q2|N|N|N|2419158|2419277|2418796|2419071|N|N|N|N|N| +2419162|AAAAAAAAKNJOECAA|1911-05-05|136|592|46|1911|5|5|5|2|1911|46|592|Friday|1911Q2|N|Y|N|2419158|2419277|2418797|2419072|N|N|N|N|N| +2419163|AAAAAAAALNJOECAA|1911-05-06|136|592|46|1911|6|5|6|2|1911|46|592|Saturday|1911Q2|N|Y|N|2419158|2419277|2418798|2419073|N|N|N|N|N| +2419164|AAAAAAAAMNJOECAA|1911-05-07|136|592|46|1911|0|5|7|2|1911|46|592|Sunday|1911Q2|N|N|N|2419158|2419277|2418799|2419074|N|N|N|N|N| +2419165|AAAAAAAANNJOECAA|1911-05-08|136|592|46|1911|1|5|8|2|1911|46|592|Monday|1911Q2|N|N|N|2419158|2419277|2418800|2419075|N|N|N|N|N| +2419166|AAAAAAAAONJOECAA|1911-05-09|136|593|46|1911|2|5|9|2|1911|46|593|Tuesday|1911Q2|N|N|N|2419158|2419277|2418801|2419076|N|N|N|N|N| +2419167|AAAAAAAAPNJOECAA|1911-05-10|136|593|46|1911|3|5|10|2|1911|46|593|Wednesday|1911Q2|N|N|N|2419158|2419277|2418802|2419077|N|N|N|N|N| +2419168|AAAAAAAAAOJOECAA|1911-05-11|136|593|46|1911|4|5|11|2|1911|46|593|Thursday|1911Q2|N|N|N|2419158|2419277|2418803|2419078|N|N|N|N|N| +2419169|AAAAAAAABOJOECAA|1911-05-12|136|593|46|1911|5|5|12|2|1911|46|593|Friday|1911Q2|N|Y|N|2419158|2419277|2418804|2419079|N|N|N|N|N| +2419170|AAAAAAAACOJOECAA|1911-05-13|136|593|46|1911|6|5|13|2|1911|46|593|Saturday|1911Q2|N|Y|N|2419158|2419277|2418805|2419080|N|N|N|N|N| +2419171|AAAAAAAADOJOECAA|1911-05-14|136|593|46|1911|0|5|14|2|1911|46|593|Sunday|1911Q2|N|N|N|2419158|2419277|2418806|2419081|N|N|N|N|N| +2419172|AAAAAAAAEOJOECAA|1911-05-15|136|593|46|1911|1|5|15|2|1911|46|593|Monday|1911Q2|N|N|N|2419158|2419277|2418807|2419082|N|N|N|N|N| +2419173|AAAAAAAAFOJOECAA|1911-05-16|136|594|46|1911|2|5|16|2|1911|46|594|Tuesday|1911Q2|N|N|N|2419158|2419277|2418808|2419083|N|N|N|N|N| +2419174|AAAAAAAAGOJOECAA|1911-05-17|136|594|46|1911|3|5|17|2|1911|46|594|Wednesday|1911Q2|N|N|N|2419158|2419277|2418809|2419084|N|N|N|N|N| +2419175|AAAAAAAAHOJOECAA|1911-05-18|136|594|46|1911|4|5|18|2|1911|46|594|Thursday|1911Q2|N|N|N|2419158|2419277|2418810|2419085|N|N|N|N|N| +2419176|AAAAAAAAIOJOECAA|1911-05-19|136|594|46|1911|5|5|19|2|1911|46|594|Friday|1911Q2|N|Y|N|2419158|2419277|2418811|2419086|N|N|N|N|N| +2419177|AAAAAAAAJOJOECAA|1911-05-20|136|594|46|1911|6|5|20|2|1911|46|594|Saturday|1911Q2|N|Y|N|2419158|2419277|2418812|2419087|N|N|N|N|N| +2419178|AAAAAAAAKOJOECAA|1911-05-21|136|594|46|1911|0|5|21|2|1911|46|594|Sunday|1911Q2|N|N|N|2419158|2419277|2418813|2419088|N|N|N|N|N| +2419179|AAAAAAAALOJOECAA|1911-05-22|136|594|46|1911|1|5|22|2|1911|46|594|Monday|1911Q2|N|N|N|2419158|2419277|2418814|2419089|N|N|N|N|N| +2419180|AAAAAAAAMOJOECAA|1911-05-23|136|595|46|1911|2|5|23|2|1911|46|595|Tuesday|1911Q2|N|N|N|2419158|2419277|2418815|2419090|N|N|N|N|N| +2419181|AAAAAAAANOJOECAA|1911-05-24|136|595|46|1911|3|5|24|2|1911|46|595|Wednesday|1911Q2|N|N|N|2419158|2419277|2418816|2419091|N|N|N|N|N| +2419182|AAAAAAAAOOJOECAA|1911-05-25|136|595|46|1911|4|5|25|2|1911|46|595|Thursday|1911Q2|N|N|N|2419158|2419277|2418817|2419092|N|N|N|N|N| +2419183|AAAAAAAAPOJOECAA|1911-05-26|136|595|46|1911|5|5|26|2|1911|46|595|Friday|1911Q2|N|Y|N|2419158|2419277|2418818|2419093|N|N|N|N|N| +2419184|AAAAAAAAAPJOECAA|1911-05-27|136|595|46|1911|6|5|27|2|1911|46|595|Saturday|1911Q2|N|Y|N|2419158|2419277|2418819|2419094|N|N|N|N|N| +2419185|AAAAAAAABPJOECAA|1911-05-28|136|595|46|1911|0|5|28|2|1911|46|595|Sunday|1911Q2|N|N|N|2419158|2419277|2418820|2419095|N|N|N|N|N| +2419186|AAAAAAAACPJOECAA|1911-05-29|136|595|46|1911|1|5|29|2|1911|46|595|Monday|1911Q2|N|N|N|2419158|2419277|2418821|2419096|N|N|N|N|N| +2419187|AAAAAAAADPJOECAA|1911-05-30|136|596|46|1911|2|5|30|2|1911|46|596|Tuesday|1911Q2|N|N|N|2419158|2419277|2418822|2419097|N|N|N|N|N| +2419188|AAAAAAAAEPJOECAA|1911-05-31|136|596|46|1911|3|5|31|2|1911|46|596|Wednesday|1911Q2|N|N|N|2419158|2419277|2418823|2419098|N|N|N|N|N| +2419189|AAAAAAAAFPJOECAA|1911-06-01|137|596|47|1911|4|6|1|2|1911|47|596|Thursday|1911Q2|N|N|N|2419189|2419339|2418824|2419099|N|N|N|N|N| +2419190|AAAAAAAAGPJOECAA|1911-06-02|137|596|47|1911|5|6|2|2|1911|47|596|Friday|1911Q2|N|Y|N|2419189|2419339|2418825|2419100|N|N|N|N|N| +2419191|AAAAAAAAHPJOECAA|1911-06-03|137|596|47|1911|6|6|3|2|1911|47|596|Saturday|1911Q2|N|Y|N|2419189|2419339|2418826|2419101|N|N|N|N|N| +2419192|AAAAAAAAIPJOECAA|1911-06-04|137|596|47|1911|0|6|4|2|1911|47|596|Sunday|1911Q2|N|N|N|2419189|2419339|2418827|2419102|N|N|N|N|N| +2419193|AAAAAAAAJPJOECAA|1911-06-05|137|596|47|1911|1|6|5|2|1911|47|596|Monday|1911Q2|N|N|N|2419189|2419339|2418828|2419103|N|N|N|N|N| +2419194|AAAAAAAAKPJOECAA|1911-06-06|137|597|47|1911|2|6|6|2|1911|47|597|Tuesday|1911Q2|N|N|N|2419189|2419339|2418829|2419104|N|N|N|N|N| +2419195|AAAAAAAALPJOECAA|1911-06-07|137|597|47|1911|3|6|7|2|1911|47|597|Wednesday|1911Q2|N|N|N|2419189|2419339|2418830|2419105|N|N|N|N|N| +2419196|AAAAAAAAMPJOECAA|1911-06-08|137|597|47|1911|4|6|8|2|1911|47|597|Thursday|1911Q2|N|N|N|2419189|2419339|2418831|2419106|N|N|N|N|N| +2419197|AAAAAAAANPJOECAA|1911-06-09|137|597|47|1911|5|6|9|2|1911|47|597|Friday|1911Q2|N|Y|N|2419189|2419339|2418832|2419107|N|N|N|N|N| +2419198|AAAAAAAAOPJOECAA|1911-06-10|137|597|47|1911|6|6|10|2|1911|47|597|Saturday|1911Q2|N|Y|N|2419189|2419339|2418833|2419108|N|N|N|N|N| +2419199|AAAAAAAAPPJOECAA|1911-06-11|137|597|47|1911|0|6|11|2|1911|47|597|Sunday|1911Q2|N|N|N|2419189|2419339|2418834|2419109|N|N|N|N|N| +2419200|AAAAAAAAAAKOECAA|1911-06-12|137|597|47|1911|1|6|12|2|1911|47|597|Monday|1911Q2|N|N|N|2419189|2419339|2418835|2419110|N|N|N|N|N| +2419201|AAAAAAAABAKOECAA|1911-06-13|137|598|47|1911|2|6|13|2|1911|47|598|Tuesday|1911Q2|N|N|N|2419189|2419339|2418836|2419111|N|N|N|N|N| +2419202|AAAAAAAACAKOECAA|1911-06-14|137|598|47|1911|3|6|14|2|1911|47|598|Wednesday|1911Q2|N|N|N|2419189|2419339|2418837|2419112|N|N|N|N|N| +2419203|AAAAAAAADAKOECAA|1911-06-15|137|598|47|1911|4|6|15|2|1911|47|598|Thursday|1911Q2|N|N|N|2419189|2419339|2418838|2419113|N|N|N|N|N| +2419204|AAAAAAAAEAKOECAA|1911-06-16|137|598|47|1911|5|6|16|2|1911|47|598|Friday|1911Q2|N|Y|N|2419189|2419339|2418839|2419114|N|N|N|N|N| +2419205|AAAAAAAAFAKOECAA|1911-06-17|137|598|47|1911|6|6|17|2|1911|47|598|Saturday|1911Q2|N|Y|N|2419189|2419339|2418840|2419115|N|N|N|N|N| +2419206|AAAAAAAAGAKOECAA|1911-06-18|137|598|47|1911|0|6|18|2|1911|47|598|Sunday|1911Q2|N|N|N|2419189|2419339|2418841|2419116|N|N|N|N|N| +2419207|AAAAAAAAHAKOECAA|1911-06-19|137|598|47|1911|1|6|19|2|1911|47|598|Monday|1911Q2|N|N|N|2419189|2419339|2418842|2419117|N|N|N|N|N| +2419208|AAAAAAAAIAKOECAA|1911-06-20|137|599|47|1911|2|6|20|2|1911|47|599|Tuesday|1911Q2|N|N|N|2419189|2419339|2418843|2419118|N|N|N|N|N| +2419209|AAAAAAAAJAKOECAA|1911-06-21|137|599|47|1911|3|6|21|2|1911|47|599|Wednesday|1911Q2|N|N|N|2419189|2419339|2418844|2419119|N|N|N|N|N| +2419210|AAAAAAAAKAKOECAA|1911-06-22|137|599|47|1911|4|6|22|2|1911|47|599|Thursday|1911Q2|N|N|N|2419189|2419339|2418845|2419120|N|N|N|N|N| +2419211|AAAAAAAALAKOECAA|1911-06-23|137|599|47|1911|5|6|23|2|1911|47|599|Friday|1911Q2|N|Y|N|2419189|2419339|2418846|2419121|N|N|N|N|N| +2419212|AAAAAAAAMAKOECAA|1911-06-24|137|599|47|1911|6|6|24|2|1911|47|599|Saturday|1911Q2|N|Y|N|2419189|2419339|2418847|2419122|N|N|N|N|N| +2419213|AAAAAAAANAKOECAA|1911-06-25|137|599|47|1911|0|6|25|2|1911|47|599|Sunday|1911Q2|N|N|N|2419189|2419339|2418848|2419123|N|N|N|N|N| +2419214|AAAAAAAAOAKOECAA|1911-06-26|137|599|47|1911|1|6|26|2|1911|47|599|Monday|1911Q2|N|N|N|2419189|2419339|2418849|2419124|N|N|N|N|N| +2419215|AAAAAAAAPAKOECAA|1911-06-27|137|600|47|1911|2|6|27|2|1911|47|600|Tuesday|1911Q2|N|N|N|2419189|2419339|2418850|2419125|N|N|N|N|N| +2419216|AAAAAAAAABKOECAA|1911-06-28|137|600|47|1911|3|6|28|2|1911|47|600|Wednesday|1911Q2|N|N|N|2419189|2419339|2418851|2419126|N|N|N|N|N| +2419217|AAAAAAAABBKOECAA|1911-06-29|137|600|47|1911|4|6|29|2|1911|47|600|Thursday|1911Q2|N|N|N|2419189|2419339|2418852|2419127|N|N|N|N|N| +2419218|AAAAAAAACBKOECAA|1911-06-30|137|600|47|1911|5|6|30|2|1911|47|600|Friday|1911Q2|N|Y|N|2419189|2419339|2418853|2419128|N|N|N|N|N| +2419219|AAAAAAAADBKOECAA|1911-07-01|138|600|47|1911|6|7|1|2|1911|47|600|Saturday|1911Q2|N|Y|N|2419219|2419399|2418854|2419128|N|N|N|N|N| +2419220|AAAAAAAAEBKOECAA|1911-07-02|138|600|47|1911|0|7|2|3|1911|47|600|Sunday|1911Q3|N|N|N|2419219|2419399|2418855|2419129|N|N|N|N|N| +2419221|AAAAAAAAFBKOECAA|1911-07-03|138|600|47|1911|1|7|3|3|1911|47|600|Monday|1911Q3|N|N|N|2419219|2419399|2418856|2419130|N|N|N|N|N| +2419222|AAAAAAAAGBKOECAA|1911-07-04|138|601|47|1911|2|7|4|3|1911|47|601|Tuesday|1911Q3|N|N|N|2419219|2419399|2418857|2419131|N|N|N|N|N| +2419223|AAAAAAAAHBKOECAA|1911-07-05|138|601|47|1911|3|7|5|3|1911|47|601|Wednesday|1911Q3|Y|N|N|2419219|2419399|2418858|2419132|N|N|N|N|N| +2419224|AAAAAAAAIBKOECAA|1911-07-06|138|601|47|1911|4|7|6|3|1911|47|601|Thursday|1911Q3|N|N|Y|2419219|2419399|2418859|2419133|N|N|N|N|N| +2419225|AAAAAAAAJBKOECAA|1911-07-07|138|601|47|1911|5|7|7|3|1911|47|601|Friday|1911Q3|N|Y|N|2419219|2419399|2418860|2419134|N|N|N|N|N| +2419226|AAAAAAAAKBKOECAA|1911-07-08|138|601|47|1911|6|7|8|3|1911|47|601|Saturday|1911Q3|N|Y|N|2419219|2419399|2418861|2419135|N|N|N|N|N| +2419227|AAAAAAAALBKOECAA|1911-07-09|138|601|47|1911|0|7|9|3|1911|47|601|Sunday|1911Q3|N|N|N|2419219|2419399|2418862|2419136|N|N|N|N|N| +2419228|AAAAAAAAMBKOECAA|1911-07-10|138|601|47|1911|1|7|10|3|1911|47|601|Monday|1911Q3|N|N|N|2419219|2419399|2418863|2419137|N|N|N|N|N| +2419229|AAAAAAAANBKOECAA|1911-07-11|138|602|47|1911|2|7|11|3|1911|47|602|Tuesday|1911Q3|N|N|N|2419219|2419399|2418864|2419138|N|N|N|N|N| +2419230|AAAAAAAAOBKOECAA|1911-07-12|138|602|47|1911|3|7|12|3|1911|47|602|Wednesday|1911Q3|N|N|N|2419219|2419399|2418865|2419139|N|N|N|N|N| +2419231|AAAAAAAAPBKOECAA|1911-07-13|138|602|47|1911|4|7|13|3|1911|47|602|Thursday|1911Q3|N|N|N|2419219|2419399|2418866|2419140|N|N|N|N|N| +2419232|AAAAAAAAACKOECAA|1911-07-14|138|602|47|1911|5|7|14|3|1911|47|602|Friday|1911Q3|N|Y|N|2419219|2419399|2418867|2419141|N|N|N|N|N| +2419233|AAAAAAAABCKOECAA|1911-07-15|138|602|47|1911|6|7|15|3|1911|47|602|Saturday|1911Q3|N|Y|N|2419219|2419399|2418868|2419142|N|N|N|N|N| +2419234|AAAAAAAACCKOECAA|1911-07-16|138|602|47|1911|0|7|16|3|1911|47|602|Sunday|1911Q3|N|N|N|2419219|2419399|2418869|2419143|N|N|N|N|N| +2419235|AAAAAAAADCKOECAA|1911-07-17|138|602|47|1911|1|7|17|3|1911|47|602|Monday|1911Q3|N|N|N|2419219|2419399|2418870|2419144|N|N|N|N|N| +2419236|AAAAAAAAECKOECAA|1911-07-18|138|603|47|1911|2|7|18|3|1911|47|603|Tuesday|1911Q3|N|N|N|2419219|2419399|2418871|2419145|N|N|N|N|N| +2419237|AAAAAAAAFCKOECAA|1911-07-19|138|603|47|1911|3|7|19|3|1911|47|603|Wednesday|1911Q3|N|N|N|2419219|2419399|2418872|2419146|N|N|N|N|N| +2419238|AAAAAAAAGCKOECAA|1911-07-20|138|603|47|1911|4|7|20|3|1911|47|603|Thursday|1911Q3|N|N|N|2419219|2419399|2418873|2419147|N|N|N|N|N| +2419239|AAAAAAAAHCKOECAA|1911-07-21|138|603|47|1911|5|7|21|3|1911|47|603|Friday|1911Q3|N|Y|N|2419219|2419399|2418874|2419148|N|N|N|N|N| +2419240|AAAAAAAAICKOECAA|1911-07-22|138|603|47|1911|6|7|22|3|1911|47|603|Saturday|1911Q3|N|Y|N|2419219|2419399|2418875|2419149|N|N|N|N|N| +2419241|AAAAAAAAJCKOECAA|1911-07-23|138|603|47|1911|0|7|23|3|1911|47|603|Sunday|1911Q3|N|N|N|2419219|2419399|2418876|2419150|N|N|N|N|N| +2419242|AAAAAAAAKCKOECAA|1911-07-24|138|603|47|1911|1|7|24|3|1911|47|603|Monday|1911Q3|N|N|N|2419219|2419399|2418877|2419151|N|N|N|N|N| +2419243|AAAAAAAALCKOECAA|1911-07-25|138|604|47|1911|2|7|25|3|1911|47|604|Tuesday|1911Q3|N|N|N|2419219|2419399|2418878|2419152|N|N|N|N|N| +2419244|AAAAAAAAMCKOECAA|1911-07-26|138|604|47|1911|3|7|26|3|1911|47|604|Wednesday|1911Q3|N|N|N|2419219|2419399|2418879|2419153|N|N|N|N|N| +2419245|AAAAAAAANCKOECAA|1911-07-27|138|604|47|1911|4|7|27|3|1911|47|604|Thursday|1911Q3|N|N|N|2419219|2419399|2418880|2419154|N|N|N|N|N| +2419246|AAAAAAAAOCKOECAA|1911-07-28|138|604|47|1911|5|7|28|3|1911|47|604|Friday|1911Q3|N|Y|N|2419219|2419399|2418881|2419155|N|N|N|N|N| +2419247|AAAAAAAAPCKOECAA|1911-07-29|138|604|47|1911|6|7|29|3|1911|47|604|Saturday|1911Q3|N|Y|N|2419219|2419399|2418882|2419156|N|N|N|N|N| +2419248|AAAAAAAAADKOECAA|1911-07-30|138|604|47|1911|0|7|30|3|1911|47|604|Sunday|1911Q3|N|N|N|2419219|2419399|2418883|2419157|N|N|N|N|N| +2419249|AAAAAAAABDKOECAA|1911-07-31|138|604|47|1911|1|7|31|3|1911|47|604|Monday|1911Q3|N|N|N|2419219|2419399|2418884|2419158|N|N|N|N|N| +2419250|AAAAAAAACDKOECAA|1911-08-01|139|605|47|1911|2|8|1|3|1911|47|605|Tuesday|1911Q3|N|N|N|2419250|2419461|2418885|2419159|N|N|N|N|N| +2419251|AAAAAAAADDKOECAA|1911-08-02|139|605|47|1911|3|8|2|3|1911|47|605|Wednesday|1911Q3|N|N|N|2419250|2419461|2418886|2419160|N|N|N|N|N| +2419252|AAAAAAAAEDKOECAA|1911-08-03|139|605|47|1911|4|8|3|3|1911|47|605|Thursday|1911Q3|N|N|N|2419250|2419461|2418887|2419161|N|N|N|N|N| +2419253|AAAAAAAAFDKOECAA|1911-08-04|139|605|47|1911|5|8|4|3|1911|47|605|Friday|1911Q3|N|Y|N|2419250|2419461|2418888|2419162|N|N|N|N|N| +2419254|AAAAAAAAGDKOECAA|1911-08-05|139|605|47|1911|6|8|5|3|1911|47|605|Saturday|1911Q3|N|Y|N|2419250|2419461|2418889|2419163|N|N|N|N|N| +2419255|AAAAAAAAHDKOECAA|1911-08-06|139|605|47|1911|0|8|6|3|1911|47|605|Sunday|1911Q3|N|N|N|2419250|2419461|2418890|2419164|N|N|N|N|N| +2419256|AAAAAAAAIDKOECAA|1911-08-07|139|605|47|1911|1|8|7|3|1911|47|605|Monday|1911Q3|N|N|N|2419250|2419461|2418891|2419165|N|N|N|N|N| +2419257|AAAAAAAAJDKOECAA|1911-08-08|139|606|47|1911|2|8|8|3|1911|47|606|Tuesday|1911Q3|N|N|N|2419250|2419461|2418892|2419166|N|N|N|N|N| +2419258|AAAAAAAAKDKOECAA|1911-08-09|139|606|47|1911|3|8|9|3|1911|47|606|Wednesday|1911Q3|N|N|N|2419250|2419461|2418893|2419167|N|N|N|N|N| +2419259|AAAAAAAALDKOECAA|1911-08-10|139|606|47|1911|4|8|10|3|1911|47|606|Thursday|1911Q3|N|N|N|2419250|2419461|2418894|2419168|N|N|N|N|N| +2419260|AAAAAAAAMDKOECAA|1911-08-11|139|606|47|1911|5|8|11|3|1911|47|606|Friday|1911Q3|N|Y|N|2419250|2419461|2418895|2419169|N|N|N|N|N| +2419261|AAAAAAAANDKOECAA|1911-08-12|139|606|47|1911|6|8|12|3|1911|47|606|Saturday|1911Q3|N|Y|N|2419250|2419461|2418896|2419170|N|N|N|N|N| +2419262|AAAAAAAAODKOECAA|1911-08-13|139|606|47|1911|0|8|13|3|1911|47|606|Sunday|1911Q3|N|N|N|2419250|2419461|2418897|2419171|N|N|N|N|N| +2419263|AAAAAAAAPDKOECAA|1911-08-14|139|606|47|1911|1|8|14|3|1911|47|606|Monday|1911Q3|N|N|N|2419250|2419461|2418898|2419172|N|N|N|N|N| +2419264|AAAAAAAAAEKOECAA|1911-08-15|139|607|47|1911|2|8|15|3|1911|47|607|Tuesday|1911Q3|N|N|N|2419250|2419461|2418899|2419173|N|N|N|N|N| +2419265|AAAAAAAABEKOECAA|1911-08-16|139|607|47|1911|3|8|16|3|1911|47|607|Wednesday|1911Q3|N|N|N|2419250|2419461|2418900|2419174|N|N|N|N|N| +2419266|AAAAAAAACEKOECAA|1911-08-17|139|607|47|1911|4|8|17|3|1911|47|607|Thursday|1911Q3|N|N|N|2419250|2419461|2418901|2419175|N|N|N|N|N| +2419267|AAAAAAAADEKOECAA|1911-08-18|139|607|47|1911|5|8|18|3|1911|47|607|Friday|1911Q3|N|Y|N|2419250|2419461|2418902|2419176|N|N|N|N|N| +2419268|AAAAAAAAEEKOECAA|1911-08-19|139|607|47|1911|6|8|19|3|1911|47|607|Saturday|1911Q3|N|Y|N|2419250|2419461|2418903|2419177|N|N|N|N|N| +2419269|AAAAAAAAFEKOECAA|1911-08-20|139|607|47|1911|0|8|20|3|1911|47|607|Sunday|1911Q3|N|N|N|2419250|2419461|2418904|2419178|N|N|N|N|N| +2419270|AAAAAAAAGEKOECAA|1911-08-21|139|607|47|1911|1|8|21|3|1911|47|607|Monday|1911Q3|N|N|N|2419250|2419461|2418905|2419179|N|N|N|N|N| +2419271|AAAAAAAAHEKOECAA|1911-08-22|139|608|47|1911|2|8|22|3|1911|47|608|Tuesday|1911Q3|N|N|N|2419250|2419461|2418906|2419180|N|N|N|N|N| +2419272|AAAAAAAAIEKOECAA|1911-08-23|139|608|47|1911|3|8|23|3|1911|47|608|Wednesday|1911Q3|N|N|N|2419250|2419461|2418907|2419181|N|N|N|N|N| +2419273|AAAAAAAAJEKOECAA|1911-08-24|139|608|47|1911|4|8|24|3|1911|47|608|Thursday|1911Q3|N|N|N|2419250|2419461|2418908|2419182|N|N|N|N|N| +2419274|AAAAAAAAKEKOECAA|1911-08-25|139|608|47|1911|5|8|25|3|1911|47|608|Friday|1911Q3|N|Y|N|2419250|2419461|2418909|2419183|N|N|N|N|N| +2419275|AAAAAAAALEKOECAA|1911-08-26|139|608|47|1911|6|8|26|3|1911|47|608|Saturday|1911Q3|N|Y|N|2419250|2419461|2418910|2419184|N|N|N|N|N| +2419276|AAAAAAAAMEKOECAA|1911-08-27|139|608|47|1911|0|8|27|3|1911|47|608|Sunday|1911Q3|N|N|N|2419250|2419461|2418911|2419185|N|N|N|N|N| +2419277|AAAAAAAANEKOECAA|1911-08-28|139|608|47|1911|1|8|28|3|1911|47|608|Monday|1911Q3|N|N|N|2419250|2419461|2418912|2419186|N|N|N|N|N| +2419278|AAAAAAAAOEKOECAA|1911-08-29|139|609|47|1911|2|8|29|3|1911|47|609|Tuesday|1911Q3|N|N|N|2419250|2419461|2418913|2419187|N|N|N|N|N| +2419279|AAAAAAAAPEKOECAA|1911-08-30|139|609|47|1911|3|8|30|3|1911|47|609|Wednesday|1911Q3|N|N|N|2419250|2419461|2418914|2419188|N|N|N|N|N| +2419280|AAAAAAAAAFKOECAA|1911-08-31|139|609|47|1911|4|8|31|3|1911|47|609|Thursday|1911Q3|N|N|N|2419250|2419461|2418915|2419189|N|N|N|N|N| +2419281|AAAAAAAABFKOECAA|1911-09-01|140|609|48|1911|5|9|1|3|1911|48|609|Friday|1911Q3|N|Y|N|2419281|2419523|2418916|2419190|N|N|N|N|N| +2419282|AAAAAAAACFKOECAA|1911-09-02|140|609|48|1911|6|9|2|3|1911|48|609|Saturday|1911Q3|N|Y|N|2419281|2419523|2418917|2419191|N|N|N|N|N| +2419283|AAAAAAAADFKOECAA|1911-09-03|140|609|48|1911|0|9|3|3|1911|48|609|Sunday|1911Q3|N|N|N|2419281|2419523|2418918|2419192|N|N|N|N|N| +2419284|AAAAAAAAEFKOECAA|1911-09-04|140|609|48|1911|1|9|4|3|1911|48|609|Monday|1911Q3|N|N|N|2419281|2419523|2418919|2419193|N|N|N|N|N| +2419285|AAAAAAAAFFKOECAA|1911-09-05|140|610|48|1911|2|9|5|3|1911|48|610|Tuesday|1911Q3|N|N|N|2419281|2419523|2418920|2419194|N|N|N|N|N| +2419286|AAAAAAAAGFKOECAA|1911-09-06|140|610|48|1911|3|9|6|3|1911|48|610|Wednesday|1911Q3|N|N|N|2419281|2419523|2418921|2419195|N|N|N|N|N| +2419287|AAAAAAAAHFKOECAA|1911-09-07|140|610|48|1911|4|9|7|3|1911|48|610|Thursday|1911Q3|N|N|N|2419281|2419523|2418922|2419196|N|N|N|N|N| +2419288|AAAAAAAAIFKOECAA|1911-09-08|140|610|48|1911|5|9|8|3|1911|48|610|Friday|1911Q3|N|Y|N|2419281|2419523|2418923|2419197|N|N|N|N|N| +2419289|AAAAAAAAJFKOECAA|1911-09-09|140|610|48|1911|6|9|9|3|1911|48|610|Saturday|1911Q3|N|Y|N|2419281|2419523|2418924|2419198|N|N|N|N|N| +2419290|AAAAAAAAKFKOECAA|1911-09-10|140|610|48|1911|0|9|10|3|1911|48|610|Sunday|1911Q3|N|N|N|2419281|2419523|2418925|2419199|N|N|N|N|N| +2419291|AAAAAAAALFKOECAA|1911-09-11|140|610|48|1911|1|9|11|3|1911|48|610|Monday|1911Q3|N|N|N|2419281|2419523|2418926|2419200|N|N|N|N|N| +2419292|AAAAAAAAMFKOECAA|1911-09-12|140|611|48|1911|2|9|12|3|1911|48|611|Tuesday|1911Q3|N|N|N|2419281|2419523|2418927|2419201|N|N|N|N|N| +2419293|AAAAAAAANFKOECAA|1911-09-13|140|611|48|1911|3|9|13|3|1911|48|611|Wednesday|1911Q3|N|N|N|2419281|2419523|2418928|2419202|N|N|N|N|N| +2419294|AAAAAAAAOFKOECAA|1911-09-14|140|611|48|1911|4|9|14|3|1911|48|611|Thursday|1911Q3|N|N|N|2419281|2419523|2418929|2419203|N|N|N|N|N| +2419295|AAAAAAAAPFKOECAA|1911-09-15|140|611|48|1911|5|9|15|3|1911|48|611|Friday|1911Q3|N|Y|N|2419281|2419523|2418930|2419204|N|N|N|N|N| +2419296|AAAAAAAAAGKOECAA|1911-09-16|140|611|48|1911|6|9|16|3|1911|48|611|Saturday|1911Q3|N|Y|N|2419281|2419523|2418931|2419205|N|N|N|N|N| +2419297|AAAAAAAABGKOECAA|1911-09-17|140|611|48|1911|0|9|17|3|1911|48|611|Sunday|1911Q3|N|N|N|2419281|2419523|2418932|2419206|N|N|N|N|N| +2419298|AAAAAAAACGKOECAA|1911-09-18|140|611|48|1911|1|9|18|3|1911|48|611|Monday|1911Q3|N|N|N|2419281|2419523|2418933|2419207|N|N|N|N|N| +2419299|AAAAAAAADGKOECAA|1911-09-19|140|612|48|1911|2|9|19|3|1911|48|612|Tuesday|1911Q3|N|N|N|2419281|2419523|2418934|2419208|N|N|N|N|N| +2419300|AAAAAAAAEGKOECAA|1911-09-20|140|612|48|1911|3|9|20|3|1911|48|612|Wednesday|1911Q3|N|N|N|2419281|2419523|2418935|2419209|N|N|N|N|N| +2419301|AAAAAAAAFGKOECAA|1911-09-21|140|612|48|1911|4|9|21|3|1911|48|612|Thursday|1911Q3|N|N|N|2419281|2419523|2418936|2419210|N|N|N|N|N| +2419302|AAAAAAAAGGKOECAA|1911-09-22|140|612|48|1911|5|9|22|3|1911|48|612|Friday|1911Q3|N|Y|N|2419281|2419523|2418937|2419211|N|N|N|N|N| +2419303|AAAAAAAAHGKOECAA|1911-09-23|140|612|48|1911|6|9|23|3|1911|48|612|Saturday|1911Q3|N|Y|N|2419281|2419523|2418938|2419212|N|N|N|N|N| +2419304|AAAAAAAAIGKOECAA|1911-09-24|140|612|48|1911|0|9|24|3|1911|48|612|Sunday|1911Q3|N|N|N|2419281|2419523|2418939|2419213|N|N|N|N|N| +2419305|AAAAAAAAJGKOECAA|1911-09-25|140|612|48|1911|1|9|25|3|1911|48|612|Monday|1911Q3|N|N|N|2419281|2419523|2418940|2419214|N|N|N|N|N| +2419306|AAAAAAAAKGKOECAA|1911-09-26|140|613|48|1911|2|9|26|3|1911|48|613|Tuesday|1911Q3|N|N|N|2419281|2419523|2418941|2419215|N|N|N|N|N| +2419307|AAAAAAAALGKOECAA|1911-09-27|140|613|48|1911|3|9|27|3|1911|48|613|Wednesday|1911Q3|N|N|N|2419281|2419523|2418942|2419216|N|N|N|N|N| +2419308|AAAAAAAAMGKOECAA|1911-09-28|140|613|48|1911|4|9|28|3|1911|48|613|Thursday|1911Q3|N|N|N|2419281|2419523|2418943|2419217|N|N|N|N|N| +2419309|AAAAAAAANGKOECAA|1911-09-29|140|613|48|1911|5|9|29|3|1911|48|613|Friday|1911Q3|N|Y|N|2419281|2419523|2418944|2419218|N|N|N|N|N| +2419310|AAAAAAAAOGKOECAA|1911-09-30|140|613|48|1911|6|9|30|3|1911|48|613|Saturday|1911Q3|N|Y|N|2419281|2419523|2418945|2419219|N|N|N|N|N| +2419311|AAAAAAAAPGKOECAA|1911-10-01|141|613|48|1911|0|10|1|3|1911|48|613|Sunday|1911Q3|N|N|N|2419311|2419583|2418946|2419219|N|N|N|N|N| +2419312|AAAAAAAAAHKOECAA|1911-10-02|141|613|48|1911|1|10|2|4|1911|48|613|Monday|1911Q4|N|N|N|2419311|2419583|2418947|2419220|N|N|N|N|N| +2419313|AAAAAAAABHKOECAA|1911-10-03|141|614|48|1911|2|10|3|4|1911|48|614|Tuesday|1911Q4|N|N|N|2419311|2419583|2418948|2419221|N|N|N|N|N| +2419314|AAAAAAAACHKOECAA|1911-10-04|141|614|48|1911|3|10|4|4|1911|48|614|Wednesday|1911Q4|N|N|N|2419311|2419583|2418949|2419222|N|N|N|N|N| +2419315|AAAAAAAADHKOECAA|1911-10-05|141|614|48|1911|4|10|5|4|1911|48|614|Thursday|1911Q4|N|N|N|2419311|2419583|2418950|2419223|N|N|N|N|N| +2419316|AAAAAAAAEHKOECAA|1911-10-06|141|614|48|1911|5|10|6|4|1911|48|614|Friday|1911Q4|N|Y|N|2419311|2419583|2418951|2419224|N|N|N|N|N| +2419317|AAAAAAAAFHKOECAA|1911-10-07|141|614|48|1911|6|10|7|4|1911|48|614|Saturday|1911Q4|N|Y|N|2419311|2419583|2418952|2419225|N|N|N|N|N| +2419318|AAAAAAAAGHKOECAA|1911-10-08|141|614|48|1911|0|10|8|4|1911|48|614|Sunday|1911Q4|N|N|N|2419311|2419583|2418953|2419226|N|N|N|N|N| +2419319|AAAAAAAAHHKOECAA|1911-10-09|141|614|48|1911|1|10|9|4|1911|48|614|Monday|1911Q4|N|N|N|2419311|2419583|2418954|2419227|N|N|N|N|N| +2419320|AAAAAAAAIHKOECAA|1911-10-10|141|615|48|1911|2|10|10|4|1911|48|615|Tuesday|1911Q4|N|N|N|2419311|2419583|2418955|2419228|N|N|N|N|N| +2419321|AAAAAAAAJHKOECAA|1911-10-11|141|615|48|1911|3|10|11|4|1911|48|615|Wednesday|1911Q4|N|N|N|2419311|2419583|2418956|2419229|N|N|N|N|N| +2419322|AAAAAAAAKHKOECAA|1911-10-12|141|615|48|1911|4|10|12|4|1911|48|615|Thursday|1911Q4|N|N|N|2419311|2419583|2418957|2419230|N|N|N|N|N| +2419323|AAAAAAAALHKOECAA|1911-10-13|141|615|48|1911|5|10|13|4|1911|48|615|Friday|1911Q4|N|Y|N|2419311|2419583|2418958|2419231|N|N|N|N|N| +2419324|AAAAAAAAMHKOECAA|1911-10-14|141|615|48|1911|6|10|14|4|1911|48|615|Saturday|1911Q4|N|Y|N|2419311|2419583|2418959|2419232|N|N|N|N|N| +2419325|AAAAAAAANHKOECAA|1911-10-15|141|615|48|1911|0|10|15|4|1911|48|615|Sunday|1911Q4|N|N|N|2419311|2419583|2418960|2419233|N|N|N|N|N| +2419326|AAAAAAAAOHKOECAA|1911-10-16|141|615|48|1911|1|10|16|4|1911|48|615|Monday|1911Q4|N|N|N|2419311|2419583|2418961|2419234|N|N|N|N|N| +2419327|AAAAAAAAPHKOECAA|1911-10-17|141|616|48|1911|2|10|17|4|1911|48|616|Tuesday|1911Q4|N|N|N|2419311|2419583|2418962|2419235|N|N|N|N|N| +2419328|AAAAAAAAAIKOECAA|1911-10-18|141|616|48|1911|3|10|18|4|1911|48|616|Wednesday|1911Q4|N|N|N|2419311|2419583|2418963|2419236|N|N|N|N|N| +2419329|AAAAAAAABIKOECAA|1911-10-19|141|616|48|1911|4|10|19|4|1911|48|616|Thursday|1911Q4|N|N|N|2419311|2419583|2418964|2419237|N|N|N|N|N| +2419330|AAAAAAAACIKOECAA|1911-10-20|141|616|48|1911|5|10|20|4|1911|48|616|Friday|1911Q4|N|Y|N|2419311|2419583|2418965|2419238|N|N|N|N|N| +2419331|AAAAAAAADIKOECAA|1911-10-21|141|616|48|1911|6|10|21|4|1911|48|616|Saturday|1911Q4|N|Y|N|2419311|2419583|2418966|2419239|N|N|N|N|N| +2419332|AAAAAAAAEIKOECAA|1911-10-22|141|616|48|1911|0|10|22|4|1911|48|616|Sunday|1911Q4|N|N|N|2419311|2419583|2418967|2419240|N|N|N|N|N| +2419333|AAAAAAAAFIKOECAA|1911-10-23|141|616|48|1911|1|10|23|4|1911|48|616|Monday|1911Q4|N|N|N|2419311|2419583|2418968|2419241|N|N|N|N|N| +2419334|AAAAAAAAGIKOECAA|1911-10-24|141|617|48|1911|2|10|24|4|1911|48|617|Tuesday|1911Q4|N|N|N|2419311|2419583|2418969|2419242|N|N|N|N|N| +2419335|AAAAAAAAHIKOECAA|1911-10-25|141|617|48|1911|3|10|25|4|1911|48|617|Wednesday|1911Q4|N|N|N|2419311|2419583|2418970|2419243|N|N|N|N|N| +2419336|AAAAAAAAIIKOECAA|1911-10-26|141|617|48|1911|4|10|26|4|1911|48|617|Thursday|1911Q4|N|N|N|2419311|2419583|2418971|2419244|N|N|N|N|N| +2419337|AAAAAAAAJIKOECAA|1911-10-27|141|617|48|1911|5|10|27|4|1911|48|617|Friday|1911Q4|N|Y|N|2419311|2419583|2418972|2419245|N|N|N|N|N| +2419338|AAAAAAAAKIKOECAA|1911-10-28|141|617|48|1911|6|10|28|4|1911|48|617|Saturday|1911Q4|N|Y|N|2419311|2419583|2418973|2419246|N|N|N|N|N| +2419339|AAAAAAAALIKOECAA|1911-10-29|141|617|48|1911|0|10|29|4|1911|48|617|Sunday|1911Q4|N|N|N|2419311|2419583|2418974|2419247|N|N|N|N|N| +2419340|AAAAAAAAMIKOECAA|1911-10-30|141|617|48|1911|1|10|30|4|1911|48|617|Monday|1911Q4|N|N|N|2419311|2419583|2418975|2419248|N|N|N|N|N| +2419341|AAAAAAAANIKOECAA|1911-10-31|141|618|48|1911|2|10|31|4|1911|48|618|Tuesday|1911Q4|N|N|N|2419311|2419583|2418976|2419249|N|N|N|N|N| +2419342|AAAAAAAAOIKOECAA|1911-11-01|142|618|48|1911|3|11|1|4|1911|48|618|Wednesday|1911Q4|N|N|N|2419342|2419645|2418977|2419250|N|N|N|N|N| +2419343|AAAAAAAAPIKOECAA|1911-11-02|142|618|48|1911|4|11|2|4|1911|48|618|Thursday|1911Q4|N|N|N|2419342|2419645|2418978|2419251|N|N|N|N|N| +2419344|AAAAAAAAAJKOECAA|1911-11-03|142|618|48|1911|5|11|3|4|1911|48|618|Friday|1911Q4|N|Y|N|2419342|2419645|2418979|2419252|N|N|N|N|N| +2419345|AAAAAAAABJKOECAA|1911-11-04|142|618|48|1911|6|11|4|4|1911|48|618|Saturday|1911Q4|N|Y|N|2419342|2419645|2418980|2419253|N|N|N|N|N| +2419346|AAAAAAAACJKOECAA|1911-11-05|142|618|48|1911|0|11|5|4|1911|48|618|Sunday|1911Q4|N|N|N|2419342|2419645|2418981|2419254|N|N|N|N|N| +2419347|AAAAAAAADJKOECAA|1911-11-06|142|618|48|1911|1|11|6|4|1911|48|618|Monday|1911Q4|N|N|N|2419342|2419645|2418982|2419255|N|N|N|N|N| +2419348|AAAAAAAAEJKOECAA|1911-11-07|142|619|48|1911|2|11|7|4|1911|48|619|Tuesday|1911Q4|N|N|N|2419342|2419645|2418983|2419256|N|N|N|N|N| +2419349|AAAAAAAAFJKOECAA|1911-11-08|142|619|48|1911|3|11|8|4|1911|48|619|Wednesday|1911Q4|N|N|N|2419342|2419645|2418984|2419257|N|N|N|N|N| +2419350|AAAAAAAAGJKOECAA|1911-11-09|142|619|48|1911|4|11|9|4|1911|48|619|Thursday|1911Q4|N|N|N|2419342|2419645|2418985|2419258|N|N|N|N|N| +2419351|AAAAAAAAHJKOECAA|1911-11-10|142|619|48|1911|5|11|10|4|1911|48|619|Friday|1911Q4|N|Y|N|2419342|2419645|2418986|2419259|N|N|N|N|N| +2419352|AAAAAAAAIJKOECAA|1911-11-11|142|619|48|1911|6|11|11|4|1911|48|619|Saturday|1911Q4|N|Y|N|2419342|2419645|2418987|2419260|N|N|N|N|N| +2419353|AAAAAAAAJJKOECAA|1911-11-12|142|619|48|1911|0|11|12|4|1911|48|619|Sunday|1911Q4|N|N|N|2419342|2419645|2418988|2419261|N|N|N|N|N| +2419354|AAAAAAAAKJKOECAA|1911-11-13|142|619|48|1911|1|11|13|4|1911|48|619|Monday|1911Q4|N|N|N|2419342|2419645|2418989|2419262|N|N|N|N|N| +2419355|AAAAAAAALJKOECAA|1911-11-14|142|620|48|1911|2|11|14|4|1911|48|620|Tuesday|1911Q4|N|N|N|2419342|2419645|2418990|2419263|N|N|N|N|N| +2419356|AAAAAAAAMJKOECAA|1911-11-15|142|620|48|1911|3|11|15|4|1911|48|620|Wednesday|1911Q4|N|N|N|2419342|2419645|2418991|2419264|N|N|N|N|N| +2419357|AAAAAAAANJKOECAA|1911-11-16|142|620|48|1911|4|11|16|4|1911|48|620|Thursday|1911Q4|N|N|N|2419342|2419645|2418992|2419265|N|N|N|N|N| +2419358|AAAAAAAAOJKOECAA|1911-11-17|142|620|48|1911|5|11|17|4|1911|48|620|Friday|1911Q4|N|Y|N|2419342|2419645|2418993|2419266|N|N|N|N|N| +2419359|AAAAAAAAPJKOECAA|1911-11-18|142|620|48|1911|6|11|18|4|1911|48|620|Saturday|1911Q4|N|Y|N|2419342|2419645|2418994|2419267|N|N|N|N|N| +2419360|AAAAAAAAAKKOECAA|1911-11-19|142|620|48|1911|0|11|19|4|1911|48|620|Sunday|1911Q4|N|N|N|2419342|2419645|2418995|2419268|N|N|N|N|N| +2419361|AAAAAAAABKKOECAA|1911-11-20|142|620|48|1911|1|11|20|4|1911|48|620|Monday|1911Q4|N|N|N|2419342|2419645|2418996|2419269|N|N|N|N|N| +2419362|AAAAAAAACKKOECAA|1911-11-21|142|621|48|1911|2|11|21|4|1911|48|621|Tuesday|1911Q4|N|N|N|2419342|2419645|2418997|2419270|N|N|N|N|N| +2419363|AAAAAAAADKKOECAA|1911-11-22|142|621|48|1911|3|11|22|4|1911|48|621|Wednesday|1911Q4|N|N|N|2419342|2419645|2418998|2419271|N|N|N|N|N| +2419364|AAAAAAAAEKKOECAA|1911-11-23|142|621|48|1911|4|11|23|4|1911|48|621|Thursday|1911Q4|N|N|N|2419342|2419645|2418999|2419272|N|N|N|N|N| +2419365|AAAAAAAAFKKOECAA|1911-11-24|142|621|48|1911|5|11|24|4|1911|48|621|Friday|1911Q4|N|Y|N|2419342|2419645|2419000|2419273|N|N|N|N|N| +2419366|AAAAAAAAGKKOECAA|1911-11-25|142|621|48|1911|6|11|25|4|1911|48|621|Saturday|1911Q4|N|Y|N|2419342|2419645|2419001|2419274|N|N|N|N|N| +2419367|AAAAAAAAHKKOECAA|1911-11-26|142|621|48|1911|0|11|26|4|1911|48|621|Sunday|1911Q4|N|N|N|2419342|2419645|2419002|2419275|N|N|N|N|N| +2419368|AAAAAAAAIKKOECAA|1911-11-27|142|621|48|1911|1|11|27|4|1911|48|621|Monday|1911Q4|N|N|N|2419342|2419645|2419003|2419276|N|N|N|N|N| +2419369|AAAAAAAAJKKOECAA|1911-11-28|142|622|48|1911|2|11|28|4|1911|48|622|Tuesday|1911Q4|N|N|N|2419342|2419645|2419004|2419277|N|N|N|N|N| +2419370|AAAAAAAAKKKOECAA|1911-11-29|142|622|48|1911|3|11|29|4|1911|48|622|Wednesday|1911Q4|N|N|N|2419342|2419645|2419005|2419278|N|N|N|N|N| +2419371|AAAAAAAALKKOECAA|1911-11-30|142|622|48|1911|4|11|30|4|1911|48|622|Thursday|1911Q4|N|N|N|2419342|2419645|2419006|2419279|N|N|N|N|N| +2419372|AAAAAAAAMKKOECAA|1911-12-01|143|622|49|1911|5|12|1|4|1911|49|622|Friday|1911Q4|N|Y|N|2419372|2419705|2419007|2419280|N|N|N|N|N| +2419373|AAAAAAAANKKOECAA|1911-12-02|143|622|49|1911|6|12|2|4|1911|49|622|Saturday|1911Q4|N|Y|N|2419372|2419705|2419008|2419281|N|N|N|N|N| +2419374|AAAAAAAAOKKOECAA|1911-12-03|143|622|49|1911|0|12|3|4|1911|49|622|Sunday|1911Q4|N|N|N|2419372|2419705|2419009|2419282|N|N|N|N|N| +2419375|AAAAAAAAPKKOECAA|1911-12-04|143|622|49|1911|1|12|4|4|1911|49|622|Monday|1911Q4|N|N|N|2419372|2419705|2419010|2419283|N|N|N|N|N| +2419376|AAAAAAAAALKOECAA|1911-12-05|143|623|49|1911|2|12|5|4|1911|49|623|Tuesday|1911Q4|N|N|N|2419372|2419705|2419011|2419284|N|N|N|N|N| +2419377|AAAAAAAABLKOECAA|1911-12-06|143|623|49|1911|3|12|6|4|1911|49|623|Wednesday|1911Q4|N|N|N|2419372|2419705|2419012|2419285|N|N|N|N|N| +2419378|AAAAAAAACLKOECAA|1911-12-07|143|623|49|1911|4|12|7|4|1911|49|623|Thursday|1911Q4|N|N|N|2419372|2419705|2419013|2419286|N|N|N|N|N| +2419379|AAAAAAAADLKOECAA|1911-12-08|143|623|49|1911|5|12|8|4|1911|49|623|Friday|1911Q4|N|Y|N|2419372|2419705|2419014|2419287|N|N|N|N|N| +2419380|AAAAAAAAELKOECAA|1911-12-09|143|623|49|1911|6|12|9|4|1911|49|623|Saturday|1911Q4|N|Y|N|2419372|2419705|2419015|2419288|N|N|N|N|N| +2419381|AAAAAAAAFLKOECAA|1911-12-10|143|623|49|1911|0|12|10|4|1911|49|623|Sunday|1911Q4|N|N|N|2419372|2419705|2419016|2419289|N|N|N|N|N| +2419382|AAAAAAAAGLKOECAA|1911-12-11|143|623|49|1911|1|12|11|4|1911|49|623|Monday|1911Q4|N|N|N|2419372|2419705|2419017|2419290|N|N|N|N|N| +2419383|AAAAAAAAHLKOECAA|1911-12-12|143|624|49|1911|2|12|12|4|1911|49|624|Tuesday|1911Q4|N|N|N|2419372|2419705|2419018|2419291|N|N|N|N|N| +2419384|AAAAAAAAILKOECAA|1911-12-13|143|624|49|1911|3|12|13|4|1911|49|624|Wednesday|1911Q4|N|N|N|2419372|2419705|2419019|2419292|N|N|N|N|N| +2419385|AAAAAAAAJLKOECAA|1911-12-14|143|624|49|1911|4|12|14|4|1911|49|624|Thursday|1911Q4|N|N|N|2419372|2419705|2419020|2419293|N|N|N|N|N| +2419386|AAAAAAAAKLKOECAA|1911-12-15|143|624|49|1911|5|12|15|4|1911|49|624|Friday|1911Q4|N|Y|N|2419372|2419705|2419021|2419294|N|N|N|N|N| +2419387|AAAAAAAALLKOECAA|1911-12-16|143|624|49|1911|6|12|16|4|1911|49|624|Saturday|1911Q4|N|Y|N|2419372|2419705|2419022|2419295|N|N|N|N|N| +2419388|AAAAAAAAMLKOECAA|1911-12-17|143|624|49|1911|0|12|17|4|1911|49|624|Sunday|1911Q4|N|N|N|2419372|2419705|2419023|2419296|N|N|N|N|N| +2419389|AAAAAAAANLKOECAA|1911-12-18|143|624|49|1911|1|12|18|4|1911|49|624|Monday|1911Q4|N|N|N|2419372|2419705|2419024|2419297|N|N|N|N|N| +2419390|AAAAAAAAOLKOECAA|1911-12-19|143|625|49|1911|2|12|19|4|1911|49|625|Tuesday|1911Q4|N|N|N|2419372|2419705|2419025|2419298|N|N|N|N|N| +2419391|AAAAAAAAPLKOECAA|1911-12-20|143|625|49|1911|3|12|20|4|1911|49|625|Wednesday|1911Q4|N|N|N|2419372|2419705|2419026|2419299|N|N|N|N|N| +2419392|AAAAAAAAAMKOECAA|1911-12-21|143|625|49|1911|4|12|21|4|1911|49|625|Thursday|1911Q4|N|N|N|2419372|2419705|2419027|2419300|N|N|N|N|N| +2419393|AAAAAAAABMKOECAA|1911-12-22|143|625|49|1911|5|12|22|4|1911|49|625|Friday|1911Q4|N|Y|N|2419372|2419705|2419028|2419301|N|N|N|N|N| +2419394|AAAAAAAACMKOECAA|1911-12-23|143|625|49|1911|6|12|23|4|1911|49|625|Saturday|1911Q4|N|Y|N|2419372|2419705|2419029|2419302|N|N|N|N|N| +2419395|AAAAAAAADMKOECAA|1911-12-24|143|625|49|1911|0|12|24|4|1911|49|625|Sunday|1911Q4|N|N|N|2419372|2419705|2419030|2419303|N|N|N|N|N| +2419396|AAAAAAAAEMKOECAA|1911-12-25|143|625|49|1911|1|12|25|4|1911|49|625|Monday|1911Q4|N|N|N|2419372|2419705|2419031|2419304|N|N|N|N|N| +2419397|AAAAAAAAFMKOECAA|1911-12-26|143|626|49|1911|2|12|26|4|1911|49|626|Tuesday|1911Q4|Y|N|N|2419372|2419705|2419032|2419305|N|N|N|N|N| +2419398|AAAAAAAAGMKOECAA|1911-12-27|143|626|49|1911|3|12|27|4|1911|49|626|Wednesday|1911Q4|N|N|Y|2419372|2419705|2419033|2419306|N|N|N|N|N| +2419399|AAAAAAAAHMKOECAA|1911-12-28|143|626|49|1911|4|12|28|4|1911|49|626|Thursday|1911Q4|N|N|N|2419372|2419705|2419034|2419307|N|N|N|N|N| +2419400|AAAAAAAAIMKOECAA|1911-12-29|143|626|49|1911|5|12|29|4|1911|49|626|Friday|1911Q4|N|Y|N|2419372|2419705|2419035|2419308|N|N|N|N|N| +2419401|AAAAAAAAJMKOECAA|1911-12-30|143|626|49|1911|6|12|30|4|1911|49|626|Saturday|1911Q4|N|Y|N|2419372|2419705|2419036|2419309|N|N|N|N|N| +2419402|AAAAAAAAKMKOECAA|1911-12-31|143|626|49|1911|0|12|31|4|1911|49|626|Sunday|1911Q4|N|N|N|2419372|2419705|2419037|2419310|N|N|N|N|N| +2419403|AAAAAAAALMKOECAA|1912-01-01|144|626|49|1912|1|1|1|1|1912|49|626|Monday|1912Q1|Y|N|N|2419403|2419402|2419038|2419311|N|N|N|N|N| +2419404|AAAAAAAAMMKOECAA|1912-01-02|144|627|49|1912|2|1|2|1|1912|49|627|Tuesday|1912Q1|N|N|Y|2419403|2419402|2419039|2419312|N|N|N|N|N| +2419405|AAAAAAAANMKOECAA|1912-01-03|144|627|49|1912|3|1|3|1|1912|49|627|Wednesday|1912Q1|N|N|N|2419403|2419402|2419040|2419313|N|N|N|N|N| +2419406|AAAAAAAAOMKOECAA|1912-01-04|144|627|49|1912|4|1|4|1|1912|49|627|Thursday|1912Q1|N|N|N|2419403|2419402|2419041|2419314|N|N|N|N|N| +2419407|AAAAAAAAPMKOECAA|1912-01-05|144|627|49|1912|5|1|5|1|1912|49|627|Friday|1912Q1|N|Y|N|2419403|2419402|2419042|2419315|N|N|N|N|N| +2419408|AAAAAAAAANKOECAA|1912-01-06|144|627|49|1912|6|1|6|1|1912|49|627|Saturday|1912Q1|N|Y|N|2419403|2419402|2419043|2419316|N|N|N|N|N| +2419409|AAAAAAAABNKOECAA|1912-01-07|144|627|49|1912|0|1|7|1|1912|49|627|Sunday|1912Q1|N|N|N|2419403|2419402|2419044|2419317|N|N|N|N|N| +2419410|AAAAAAAACNKOECAA|1912-01-08|144|627|49|1912|1|1|8|1|1912|49|627|Monday|1912Q1|N|N|N|2419403|2419402|2419045|2419318|N|N|N|N|N| +2419411|AAAAAAAADNKOECAA|1912-01-09|144|628|49|1912|2|1|9|1|1912|49|628|Tuesday|1912Q1|N|N|N|2419403|2419402|2419046|2419319|N|N|N|N|N| +2419412|AAAAAAAAENKOECAA|1912-01-10|144|628|49|1912|3|1|10|1|1912|49|628|Wednesday|1912Q1|N|N|N|2419403|2419402|2419047|2419320|N|N|N|N|N| +2419413|AAAAAAAAFNKOECAA|1912-01-11|144|628|49|1912|4|1|11|1|1912|49|628|Thursday|1912Q1|N|N|N|2419403|2419402|2419048|2419321|N|N|N|N|N| +2419414|AAAAAAAAGNKOECAA|1912-01-12|144|628|49|1912|5|1|12|1|1912|49|628|Friday|1912Q1|N|Y|N|2419403|2419402|2419049|2419322|N|N|N|N|N| +2419415|AAAAAAAAHNKOECAA|1912-01-13|144|628|49|1912|6|1|13|1|1912|49|628|Saturday|1912Q1|N|Y|N|2419403|2419402|2419050|2419323|N|N|N|N|N| +2419416|AAAAAAAAINKOECAA|1912-01-14|144|628|49|1912|0|1|14|1|1912|49|628|Sunday|1912Q1|N|N|N|2419403|2419402|2419051|2419324|N|N|N|N|N| +2419417|AAAAAAAAJNKOECAA|1912-01-15|144|628|49|1912|1|1|15|1|1912|49|628|Monday|1912Q1|N|N|N|2419403|2419402|2419052|2419325|N|N|N|N|N| +2419418|AAAAAAAAKNKOECAA|1912-01-16|144|629|49|1912|2|1|16|1|1912|49|629|Tuesday|1912Q1|N|N|N|2419403|2419402|2419053|2419326|N|N|N|N|N| +2419419|AAAAAAAALNKOECAA|1912-01-17|144|629|49|1912|3|1|17|1|1912|49|629|Wednesday|1912Q1|N|N|N|2419403|2419402|2419054|2419327|N|N|N|N|N| +2419420|AAAAAAAAMNKOECAA|1912-01-18|144|629|49|1912|4|1|18|1|1912|49|629|Thursday|1912Q1|N|N|N|2419403|2419402|2419055|2419328|N|N|N|N|N| +2419421|AAAAAAAANNKOECAA|1912-01-19|144|629|49|1912|5|1|19|1|1912|49|629|Friday|1912Q1|N|Y|N|2419403|2419402|2419056|2419329|N|N|N|N|N| +2419422|AAAAAAAAONKOECAA|1912-01-20|144|629|49|1912|6|1|20|1|1912|49|629|Saturday|1912Q1|N|Y|N|2419403|2419402|2419057|2419330|N|N|N|N|N| +2419423|AAAAAAAAPNKOECAA|1912-01-21|144|629|49|1912|0|1|21|1|1912|49|629|Sunday|1912Q1|N|N|N|2419403|2419402|2419058|2419331|N|N|N|N|N| +2419424|AAAAAAAAAOKOECAA|1912-01-22|144|629|49|1912|1|1|22|1|1912|49|629|Monday|1912Q1|N|N|N|2419403|2419402|2419059|2419332|N|N|N|N|N| +2419425|AAAAAAAABOKOECAA|1912-01-23|144|630|49|1912|2|1|23|1|1912|49|630|Tuesday|1912Q1|N|N|N|2419403|2419402|2419060|2419333|N|N|N|N|N| +2419426|AAAAAAAACOKOECAA|1912-01-24|144|630|49|1912|3|1|24|1|1912|49|630|Wednesday|1912Q1|N|N|N|2419403|2419402|2419061|2419334|N|N|N|N|N| +2419427|AAAAAAAADOKOECAA|1912-01-25|144|630|49|1912|4|1|25|1|1912|49|630|Thursday|1912Q1|N|N|N|2419403|2419402|2419062|2419335|N|N|N|N|N| +2419428|AAAAAAAAEOKOECAA|1912-01-26|144|630|49|1912|5|1|26|1|1912|49|630|Friday|1912Q1|N|Y|N|2419403|2419402|2419063|2419336|N|N|N|N|N| +2419429|AAAAAAAAFOKOECAA|1912-01-27|144|630|49|1912|6|1|27|1|1912|49|630|Saturday|1912Q1|N|Y|N|2419403|2419402|2419064|2419337|N|N|N|N|N| +2419430|AAAAAAAAGOKOECAA|1912-01-28|144|630|49|1912|0|1|28|1|1912|49|630|Sunday|1912Q1|N|N|N|2419403|2419402|2419065|2419338|N|N|N|N|N| +2419431|AAAAAAAAHOKOECAA|1912-01-29|144|630|49|1912|1|1|29|1|1912|49|630|Monday|1912Q1|N|N|N|2419403|2419402|2419066|2419339|N|N|N|N|N| +2419432|AAAAAAAAIOKOECAA|1912-01-30|144|631|49|1912|2|1|30|1|1912|49|631|Tuesday|1912Q1|N|N|N|2419403|2419402|2419067|2419340|N|N|N|N|N| +2419433|AAAAAAAAJOKOECAA|1912-01-31|144|631|49|1912|3|1|31|1|1912|49|631|Wednesday|1912Q1|N|N|N|2419403|2419402|2419068|2419341|N|N|N|N|N| +2419434|AAAAAAAAKOKOECAA|1912-02-01|145|631|49|1912|4|2|1|1|1912|49|631|Thursday|1912Q1|N|N|N|2419434|2419464|2419069|2419342|N|N|N|N|N| +2419435|AAAAAAAALOKOECAA|1912-02-02|145|631|49|1912|5|2|2|1|1912|49|631|Friday|1912Q1|N|Y|N|2419434|2419464|2419070|2419343|N|N|N|N|N| +2419436|AAAAAAAAMOKOECAA|1912-02-03|145|631|49|1912|6|2|3|1|1912|49|631|Saturday|1912Q1|N|Y|N|2419434|2419464|2419071|2419344|N|N|N|N|N| +2419437|AAAAAAAANOKOECAA|1912-02-04|145|631|49|1912|0|2|4|1|1912|49|631|Sunday|1912Q1|N|N|N|2419434|2419464|2419072|2419345|N|N|N|N|N| +2419438|AAAAAAAAOOKOECAA|1912-02-05|145|631|49|1912|1|2|5|1|1912|49|631|Monday|1912Q1|N|N|N|2419434|2419464|2419073|2419346|N|N|N|N|N| +2419439|AAAAAAAAPOKOECAA|1912-02-06|145|632|49|1912|2|2|6|1|1912|49|632|Tuesday|1912Q1|N|N|N|2419434|2419464|2419074|2419347|N|N|N|N|N| +2419440|AAAAAAAAAPKOECAA|1912-02-07|145|632|49|1912|3|2|7|1|1912|49|632|Wednesday|1912Q1|N|N|N|2419434|2419464|2419075|2419348|N|N|N|N|N| +2419441|AAAAAAAABPKOECAA|1912-02-08|145|632|49|1912|4|2|8|1|1912|49|632|Thursday|1912Q1|N|N|N|2419434|2419464|2419076|2419349|N|N|N|N|N| +2419442|AAAAAAAACPKOECAA|1912-02-09|145|632|49|1912|5|2|9|1|1912|49|632|Friday|1912Q1|N|Y|N|2419434|2419464|2419077|2419350|N|N|N|N|N| +2419443|AAAAAAAADPKOECAA|1912-02-10|145|632|49|1912|6|2|10|1|1912|49|632|Saturday|1912Q1|N|Y|N|2419434|2419464|2419078|2419351|N|N|N|N|N| +2419444|AAAAAAAAEPKOECAA|1912-02-11|145|632|49|1912|0|2|11|1|1912|49|632|Sunday|1912Q1|N|N|N|2419434|2419464|2419079|2419352|N|N|N|N|N| +2419445|AAAAAAAAFPKOECAA|1912-02-12|145|632|49|1912|1|2|12|1|1912|49|632|Monday|1912Q1|N|N|N|2419434|2419464|2419080|2419353|N|N|N|N|N| +2419446|AAAAAAAAGPKOECAA|1912-02-13|145|633|49|1912|2|2|13|1|1912|49|633|Tuesday|1912Q1|N|N|N|2419434|2419464|2419081|2419354|N|N|N|N|N| +2419447|AAAAAAAAHPKOECAA|1912-02-14|145|633|49|1912|3|2|14|1|1912|49|633|Wednesday|1912Q1|N|N|N|2419434|2419464|2419082|2419355|N|N|N|N|N| +2419448|AAAAAAAAIPKOECAA|1912-02-15|145|633|49|1912|4|2|15|1|1912|49|633|Thursday|1912Q1|N|N|N|2419434|2419464|2419083|2419356|N|N|N|N|N| +2419449|AAAAAAAAJPKOECAA|1912-02-16|145|633|49|1912|5|2|16|1|1912|49|633|Friday|1912Q1|N|Y|N|2419434|2419464|2419084|2419357|N|N|N|N|N| +2419450|AAAAAAAAKPKOECAA|1912-02-17|145|633|49|1912|6|2|17|1|1912|49|633|Saturday|1912Q1|N|Y|N|2419434|2419464|2419085|2419358|N|N|N|N|N| +2419451|AAAAAAAALPKOECAA|1912-02-18|145|633|49|1912|0|2|18|1|1912|49|633|Sunday|1912Q1|N|N|N|2419434|2419464|2419086|2419359|N|N|N|N|N| +2419452|AAAAAAAAMPKOECAA|1912-02-19|145|633|49|1912|1|2|19|1|1912|49|633|Monday|1912Q1|N|N|N|2419434|2419464|2419087|2419360|N|N|N|N|N| +2419453|AAAAAAAANPKOECAA|1912-02-20|145|634|49|1912|2|2|20|1|1912|49|634|Tuesday|1912Q1|N|N|N|2419434|2419464|2419088|2419361|N|N|N|N|N| +2419454|AAAAAAAAOPKOECAA|1912-02-21|145|634|49|1912|3|2|21|1|1912|49|634|Wednesday|1912Q1|N|N|N|2419434|2419464|2419089|2419362|N|N|N|N|N| +2419455|AAAAAAAAPPKOECAA|1912-02-22|145|634|49|1912|4|2|22|1|1912|49|634|Thursday|1912Q1|N|N|N|2419434|2419464|2419090|2419363|N|N|N|N|N| +2419456|AAAAAAAAAALOECAA|1912-02-23|145|634|49|1912|5|2|23|1|1912|49|634|Friday|1912Q1|N|Y|N|2419434|2419464|2419091|2419364|N|N|N|N|N| +2419457|AAAAAAAABALOECAA|1912-02-24|145|634|49|1912|6|2|24|1|1912|49|634|Saturday|1912Q1|N|Y|N|2419434|2419464|2419092|2419365|N|N|N|N|N| +2419458|AAAAAAAACALOECAA|1912-02-25|145|634|49|1912|0|2|25|1|1912|49|634|Sunday|1912Q1|N|N|N|2419434|2419464|2419093|2419366|N|N|N|N|N| +2419459|AAAAAAAADALOECAA|1912-02-26|145|634|49|1912|1|2|26|1|1912|49|634|Monday|1912Q1|N|N|N|2419434|2419464|2419094|2419367|N|N|N|N|N| +2419460|AAAAAAAAEALOECAA|1912-02-27|145|635|49|1912|2|2|27|1|1912|49|635|Tuesday|1912Q1|N|N|N|2419434|2419464|2419095|2419368|N|N|N|N|N| +2419461|AAAAAAAAFALOECAA|1912-02-28|145|635|49|1912|3|2|28|1|1912|49|635|Wednesday|1912Q1|N|N|N|2419434|2419464|2419096|2419369|N|N|N|N|N| +2419462|AAAAAAAAGALOECAA|1912-02-29|145|635|49|1912|4|2|29|1|1912|49|635|Thursday|1912Q1|N|N|N|2419434|2419464|2419096|2419370|N|N|N|N|N| +2419463|AAAAAAAAHALOECAA|1912-03-01|146|635|50|1912|5|3|1|1|1912|50|635|Friday|1912Q1|N|Y|N|2419463|2419522|2419097|2419371|N|N|N|N|N| +2419464|AAAAAAAAIALOECAA|1912-03-02|146|635|50|1912|6|3|2|1|1912|50|635|Saturday|1912Q1|N|Y|N|2419463|2419522|2419098|2419372|N|N|N|N|N| +2419465|AAAAAAAAJALOECAA|1912-03-03|146|635|50|1912|0|3|3|1|1912|50|635|Sunday|1912Q1|N|N|N|2419463|2419522|2419099|2419373|N|N|N|N|N| +2419466|AAAAAAAAKALOECAA|1912-03-04|146|635|50|1912|1|3|4|1|1912|50|635|Monday|1912Q1|N|N|N|2419463|2419522|2419100|2419374|N|N|N|N|N| +2419467|AAAAAAAALALOECAA|1912-03-05|146|636|50|1912|2|3|5|1|1912|50|636|Tuesday|1912Q1|N|N|N|2419463|2419522|2419101|2419375|N|N|N|N|N| +2419468|AAAAAAAAMALOECAA|1912-03-06|146|636|50|1912|3|3|6|1|1912|50|636|Wednesday|1912Q1|N|N|N|2419463|2419522|2419102|2419376|N|N|N|N|N| +2419469|AAAAAAAANALOECAA|1912-03-07|146|636|50|1912|4|3|7|1|1912|50|636|Thursday|1912Q1|N|N|N|2419463|2419522|2419103|2419377|N|N|N|N|N| +2419470|AAAAAAAAOALOECAA|1912-03-08|146|636|50|1912|5|3|8|1|1912|50|636|Friday|1912Q1|N|Y|N|2419463|2419522|2419104|2419378|N|N|N|N|N| +2419471|AAAAAAAAPALOECAA|1912-03-09|146|636|50|1912|6|3|9|1|1912|50|636|Saturday|1912Q1|N|Y|N|2419463|2419522|2419105|2419379|N|N|N|N|N| +2419472|AAAAAAAAABLOECAA|1912-03-10|146|636|50|1912|0|3|10|1|1912|50|636|Sunday|1912Q1|N|N|N|2419463|2419522|2419106|2419380|N|N|N|N|N| +2419473|AAAAAAAABBLOECAA|1912-03-11|146|636|50|1912|1|3|11|1|1912|50|636|Monday|1912Q1|N|N|N|2419463|2419522|2419107|2419381|N|N|N|N|N| +2419474|AAAAAAAACBLOECAA|1912-03-12|146|637|50|1912|2|3|12|1|1912|50|637|Tuesday|1912Q1|N|N|N|2419463|2419522|2419108|2419382|N|N|N|N|N| +2419475|AAAAAAAADBLOECAA|1912-03-13|146|637|50|1912|3|3|13|1|1912|50|637|Wednesday|1912Q1|N|N|N|2419463|2419522|2419109|2419383|N|N|N|N|N| +2419476|AAAAAAAAEBLOECAA|1912-03-14|146|637|50|1912|4|3|14|1|1912|50|637|Thursday|1912Q1|N|N|N|2419463|2419522|2419110|2419384|N|N|N|N|N| +2419477|AAAAAAAAFBLOECAA|1912-03-15|146|637|50|1912|5|3|15|1|1912|50|637|Friday|1912Q1|N|Y|N|2419463|2419522|2419111|2419385|N|N|N|N|N| +2419478|AAAAAAAAGBLOECAA|1912-03-16|146|637|50|1912|6|3|16|1|1912|50|637|Saturday|1912Q1|N|Y|N|2419463|2419522|2419112|2419386|N|N|N|N|N| +2419479|AAAAAAAAHBLOECAA|1912-03-17|146|637|50|1912|0|3|17|1|1912|50|637|Sunday|1912Q1|N|N|N|2419463|2419522|2419113|2419387|N|N|N|N|N| +2419480|AAAAAAAAIBLOECAA|1912-03-18|146|637|50|1912|1|3|18|1|1912|50|637|Monday|1912Q1|N|N|N|2419463|2419522|2419114|2419388|N|N|N|N|N| +2419481|AAAAAAAAJBLOECAA|1912-03-19|146|638|50|1912|2|3|19|1|1912|50|638|Tuesday|1912Q1|N|N|N|2419463|2419522|2419115|2419389|N|N|N|N|N| +2419482|AAAAAAAAKBLOECAA|1912-03-20|146|638|50|1912|3|3|20|1|1912|50|638|Wednesday|1912Q1|N|N|N|2419463|2419522|2419116|2419390|N|N|N|N|N| +2419483|AAAAAAAALBLOECAA|1912-03-21|146|638|50|1912|4|3|21|1|1912|50|638|Thursday|1912Q1|N|N|N|2419463|2419522|2419117|2419391|N|N|N|N|N| +2419484|AAAAAAAAMBLOECAA|1912-03-22|146|638|50|1912|5|3|22|1|1912|50|638|Friday|1912Q1|N|Y|N|2419463|2419522|2419118|2419392|N|N|N|N|N| +2419485|AAAAAAAANBLOECAA|1912-03-23|146|638|50|1912|6|3|23|1|1912|50|638|Saturday|1912Q1|N|Y|N|2419463|2419522|2419119|2419393|N|N|N|N|N| +2419486|AAAAAAAAOBLOECAA|1912-03-24|146|638|50|1912|0|3|24|1|1912|50|638|Sunday|1912Q1|N|N|N|2419463|2419522|2419120|2419394|N|N|N|N|N| +2419487|AAAAAAAAPBLOECAA|1912-03-25|146|638|50|1912|1|3|25|1|1912|50|638|Monday|1912Q1|N|N|N|2419463|2419522|2419121|2419395|N|N|N|N|N| +2419488|AAAAAAAAACLOECAA|1912-03-26|146|639|50|1912|2|3|26|1|1912|50|639|Tuesday|1912Q1|N|N|N|2419463|2419522|2419122|2419396|N|N|N|N|N| +2419489|AAAAAAAABCLOECAA|1912-03-27|146|639|50|1912|3|3|27|1|1912|50|639|Wednesday|1912Q1|N|N|N|2419463|2419522|2419123|2419397|N|N|N|N|N| +2419490|AAAAAAAACCLOECAA|1912-03-28|146|639|50|1912|4|3|28|1|1912|50|639|Thursday|1912Q1|N|N|N|2419463|2419522|2419124|2419398|N|N|N|N|N| +2419491|AAAAAAAADCLOECAA|1912-03-29|146|639|50|1912|5|3|29|1|1912|50|639|Friday|1912Q1|N|Y|N|2419463|2419522|2419125|2419399|N|N|N|N|N| +2419492|AAAAAAAAECLOECAA|1912-03-30|146|639|50|1912|6|3|30|1|1912|50|639|Saturday|1912Q1|N|Y|N|2419463|2419522|2419126|2419400|N|N|N|N|N| +2419493|AAAAAAAAFCLOECAA|1912-03-31|146|639|50|1912|0|3|31|1|1912|50|639|Sunday|1912Q1|N|N|N|2419463|2419522|2419127|2419401|N|N|N|N|N| +2419494|AAAAAAAAGCLOECAA|1912-04-01|147|639|50|1912|1|4|1|2|1912|50|639|Monday|1912Q2|N|N|N|2419494|2419584|2419128|2419403|N|N|N|N|N| +2419495|AAAAAAAAHCLOECAA|1912-04-02|147|640|50|1912|2|4|2|2|1912|50|640|Tuesday|1912Q2|N|N|N|2419494|2419584|2419129|2419404|N|N|N|N|N| +2419496|AAAAAAAAICLOECAA|1912-04-03|147|640|50|1912|3|4|3|2|1912|50|640|Wednesday|1912Q2|N|N|N|2419494|2419584|2419130|2419405|N|N|N|N|N| +2419497|AAAAAAAAJCLOECAA|1912-04-04|147|640|50|1912|4|4|4|2|1912|50|640|Thursday|1912Q2|N|N|N|2419494|2419584|2419131|2419406|N|N|N|N|N| +2419498|AAAAAAAAKCLOECAA|1912-04-05|147|640|50|1912|5|4|5|2|1912|50|640|Friday|1912Q2|N|Y|N|2419494|2419584|2419132|2419407|N|N|N|N|N| +2419499|AAAAAAAALCLOECAA|1912-04-06|147|640|50|1912|6|4|6|2|1912|50|640|Saturday|1912Q2|N|Y|N|2419494|2419584|2419133|2419408|N|N|N|N|N| +2419500|AAAAAAAAMCLOECAA|1912-04-07|147|640|50|1912|0|4|7|2|1912|50|640|Sunday|1912Q2|N|N|N|2419494|2419584|2419134|2419409|N|N|N|N|N| +2419501|AAAAAAAANCLOECAA|1912-04-08|147|640|50|1912|1|4|8|2|1912|50|640|Monday|1912Q2|N|N|N|2419494|2419584|2419135|2419410|N|N|N|N|N| +2419502|AAAAAAAAOCLOECAA|1912-04-09|147|641|50|1912|2|4|9|2|1912|50|641|Tuesday|1912Q2|N|N|N|2419494|2419584|2419136|2419411|N|N|N|N|N| +2419503|AAAAAAAAPCLOECAA|1912-04-10|147|641|50|1912|3|4|10|2|1912|50|641|Wednesday|1912Q2|N|N|N|2419494|2419584|2419137|2419412|N|N|N|N|N| +2419504|AAAAAAAAADLOECAA|1912-04-11|147|641|50|1912|4|4|11|2|1912|50|641|Thursday|1912Q2|N|N|N|2419494|2419584|2419138|2419413|N|N|N|N|N| +2419505|AAAAAAAABDLOECAA|1912-04-12|147|641|50|1912|5|4|12|2|1912|50|641|Friday|1912Q2|N|Y|N|2419494|2419584|2419139|2419414|N|N|N|N|N| +2419506|AAAAAAAACDLOECAA|1912-04-13|147|641|50|1912|6|4|13|2|1912|50|641|Saturday|1912Q2|N|Y|N|2419494|2419584|2419140|2419415|N|N|N|N|N| +2419507|AAAAAAAADDLOECAA|1912-04-14|147|641|50|1912|0|4|14|2|1912|50|641|Sunday|1912Q2|N|N|N|2419494|2419584|2419141|2419416|N|N|N|N|N| +2419508|AAAAAAAAEDLOECAA|1912-04-15|147|641|50|1912|1|4|15|2|1912|50|641|Monday|1912Q2|N|N|N|2419494|2419584|2419142|2419417|N|N|N|N|N| +2419509|AAAAAAAAFDLOECAA|1912-04-16|147|642|50|1912|2|4|16|2|1912|50|642|Tuesday|1912Q2|N|N|N|2419494|2419584|2419143|2419418|N|N|N|N|N| +2419510|AAAAAAAAGDLOECAA|1912-04-17|147|642|50|1912|3|4|17|2|1912|50|642|Wednesday|1912Q2|N|N|N|2419494|2419584|2419144|2419419|N|N|N|N|N| +2419511|AAAAAAAAHDLOECAA|1912-04-18|147|642|50|1912|4|4|18|2|1912|50|642|Thursday|1912Q2|N|N|N|2419494|2419584|2419145|2419420|N|N|N|N|N| +2419512|AAAAAAAAIDLOECAA|1912-04-19|147|642|50|1912|5|4|19|2|1912|50|642|Friday|1912Q2|N|Y|N|2419494|2419584|2419146|2419421|N|N|N|N|N| +2419513|AAAAAAAAJDLOECAA|1912-04-20|147|642|50|1912|6|4|20|2|1912|50|642|Saturday|1912Q2|N|Y|N|2419494|2419584|2419147|2419422|N|N|N|N|N| +2419514|AAAAAAAAKDLOECAA|1912-04-21|147|642|50|1912|0|4|21|2|1912|50|642|Sunday|1912Q2|N|N|N|2419494|2419584|2419148|2419423|N|N|N|N|N| +2419515|AAAAAAAALDLOECAA|1912-04-22|147|642|50|1912|1|4|22|2|1912|50|642|Monday|1912Q2|N|N|N|2419494|2419584|2419149|2419424|N|N|N|N|N| +2419516|AAAAAAAAMDLOECAA|1912-04-23|147|643|50|1912|2|4|23|2|1912|50|643|Tuesday|1912Q2|N|N|N|2419494|2419584|2419150|2419425|N|N|N|N|N| +2419517|AAAAAAAANDLOECAA|1912-04-24|147|643|50|1912|3|4|24|2|1912|50|643|Wednesday|1912Q2|N|N|N|2419494|2419584|2419151|2419426|N|N|N|N|N| +2419518|AAAAAAAAODLOECAA|1912-04-25|147|643|50|1912|4|4|25|2|1912|50|643|Thursday|1912Q2|N|N|N|2419494|2419584|2419152|2419427|N|N|N|N|N| +2419519|AAAAAAAAPDLOECAA|1912-04-26|147|643|50|1912|5|4|26|2|1912|50|643|Friday|1912Q2|N|Y|N|2419494|2419584|2419153|2419428|N|N|N|N|N| +2419520|AAAAAAAAAELOECAA|1912-04-27|147|643|50|1912|6|4|27|2|1912|50|643|Saturday|1912Q2|N|Y|N|2419494|2419584|2419154|2419429|N|N|N|N|N| +2419521|AAAAAAAABELOECAA|1912-04-28|147|643|50|1912|0|4|28|2|1912|50|643|Sunday|1912Q2|N|N|N|2419494|2419584|2419155|2419430|N|N|N|N|N| +2419522|AAAAAAAACELOECAA|1912-04-29|147|643|50|1912|1|4|29|2|1912|50|643|Monday|1912Q2|N|N|N|2419494|2419584|2419156|2419431|N|N|N|N|N| +2419523|AAAAAAAADELOECAA|1912-04-30|147|644|50|1912|2|4|30|2|1912|50|644|Tuesday|1912Q2|N|N|N|2419494|2419584|2419157|2419432|N|N|N|N|N| +2419524|AAAAAAAAEELOECAA|1912-05-01|148|644|50|1912|3|5|1|2|1912|50|644|Wednesday|1912Q2|N|N|N|2419524|2419644|2419158|2419433|N|N|N|N|N| +2419525|AAAAAAAAFELOECAA|1912-05-02|148|644|50|1912|4|5|2|2|1912|50|644|Thursday|1912Q2|N|N|N|2419524|2419644|2419159|2419434|N|N|N|N|N| +2419526|AAAAAAAAGELOECAA|1912-05-03|148|644|50|1912|5|5|3|2|1912|50|644|Friday|1912Q2|N|Y|N|2419524|2419644|2419160|2419435|N|N|N|N|N| +2419527|AAAAAAAAHELOECAA|1912-05-04|148|644|50|1912|6|5|4|2|1912|50|644|Saturday|1912Q2|N|Y|N|2419524|2419644|2419161|2419436|N|N|N|N|N| +2419528|AAAAAAAAIELOECAA|1912-05-05|148|644|50|1912|0|5|5|2|1912|50|644|Sunday|1912Q2|N|N|N|2419524|2419644|2419162|2419437|N|N|N|N|N| +2419529|AAAAAAAAJELOECAA|1912-05-06|148|644|50|1912|1|5|6|2|1912|50|644|Monday|1912Q2|N|N|N|2419524|2419644|2419163|2419438|N|N|N|N|N| +2419530|AAAAAAAAKELOECAA|1912-05-07|148|645|50|1912|2|5|7|2|1912|50|645|Tuesday|1912Q2|N|N|N|2419524|2419644|2419164|2419439|N|N|N|N|N| +2419531|AAAAAAAALELOECAA|1912-05-08|148|645|50|1912|3|5|8|2|1912|50|645|Wednesday|1912Q2|N|N|N|2419524|2419644|2419165|2419440|N|N|N|N|N| +2419532|AAAAAAAAMELOECAA|1912-05-09|148|645|50|1912|4|5|9|2|1912|50|645|Thursday|1912Q2|N|N|N|2419524|2419644|2419166|2419441|N|N|N|N|N| +2419533|AAAAAAAANELOECAA|1912-05-10|148|645|50|1912|5|5|10|2|1912|50|645|Friday|1912Q2|N|Y|N|2419524|2419644|2419167|2419442|N|N|N|N|N| +2419534|AAAAAAAAOELOECAA|1912-05-11|148|645|50|1912|6|5|11|2|1912|50|645|Saturday|1912Q2|N|Y|N|2419524|2419644|2419168|2419443|N|N|N|N|N| +2419535|AAAAAAAAPELOECAA|1912-05-12|148|645|50|1912|0|5|12|2|1912|50|645|Sunday|1912Q2|N|N|N|2419524|2419644|2419169|2419444|N|N|N|N|N| +2419536|AAAAAAAAAFLOECAA|1912-05-13|148|645|50|1912|1|5|13|2|1912|50|645|Monday|1912Q2|N|N|N|2419524|2419644|2419170|2419445|N|N|N|N|N| +2419537|AAAAAAAABFLOECAA|1912-05-14|148|646|50|1912|2|5|14|2|1912|50|646|Tuesday|1912Q2|N|N|N|2419524|2419644|2419171|2419446|N|N|N|N|N| +2419538|AAAAAAAACFLOECAA|1912-05-15|148|646|50|1912|3|5|15|2|1912|50|646|Wednesday|1912Q2|N|N|N|2419524|2419644|2419172|2419447|N|N|N|N|N| +2419539|AAAAAAAADFLOECAA|1912-05-16|148|646|50|1912|4|5|16|2|1912|50|646|Thursday|1912Q2|N|N|N|2419524|2419644|2419173|2419448|N|N|N|N|N| +2419540|AAAAAAAAEFLOECAA|1912-05-17|148|646|50|1912|5|5|17|2|1912|50|646|Friday|1912Q2|N|Y|N|2419524|2419644|2419174|2419449|N|N|N|N|N| +2419541|AAAAAAAAFFLOECAA|1912-05-18|148|646|50|1912|6|5|18|2|1912|50|646|Saturday|1912Q2|N|Y|N|2419524|2419644|2419175|2419450|N|N|N|N|N| +2419542|AAAAAAAAGFLOECAA|1912-05-19|148|646|50|1912|0|5|19|2|1912|50|646|Sunday|1912Q2|N|N|N|2419524|2419644|2419176|2419451|N|N|N|N|N| +2419543|AAAAAAAAHFLOECAA|1912-05-20|148|646|50|1912|1|5|20|2|1912|50|646|Monday|1912Q2|N|N|N|2419524|2419644|2419177|2419452|N|N|N|N|N| +2419544|AAAAAAAAIFLOECAA|1912-05-21|148|647|50|1912|2|5|21|2|1912|50|647|Tuesday|1912Q2|N|N|N|2419524|2419644|2419178|2419453|N|N|N|N|N| +2419545|AAAAAAAAJFLOECAA|1912-05-22|148|647|50|1912|3|5|22|2|1912|50|647|Wednesday|1912Q2|N|N|N|2419524|2419644|2419179|2419454|N|N|N|N|N| +2419546|AAAAAAAAKFLOECAA|1912-05-23|148|647|50|1912|4|5|23|2|1912|50|647|Thursday|1912Q2|N|N|N|2419524|2419644|2419180|2419455|N|N|N|N|N| +2419547|AAAAAAAALFLOECAA|1912-05-24|148|647|50|1912|5|5|24|2|1912|50|647|Friday|1912Q2|N|Y|N|2419524|2419644|2419181|2419456|N|N|N|N|N| +2419548|AAAAAAAAMFLOECAA|1912-05-25|148|647|50|1912|6|5|25|2|1912|50|647|Saturday|1912Q2|N|Y|N|2419524|2419644|2419182|2419457|N|N|N|N|N| +2419549|AAAAAAAANFLOECAA|1912-05-26|148|647|50|1912|0|5|26|2|1912|50|647|Sunday|1912Q2|N|N|N|2419524|2419644|2419183|2419458|N|N|N|N|N| +2419550|AAAAAAAAOFLOECAA|1912-05-27|148|647|50|1912|1|5|27|2|1912|50|647|Monday|1912Q2|N|N|N|2419524|2419644|2419184|2419459|N|N|N|N|N| +2419551|AAAAAAAAPFLOECAA|1912-05-28|148|648|50|1912|2|5|28|2|1912|50|648|Tuesday|1912Q2|N|N|N|2419524|2419644|2419185|2419460|N|N|N|N|N| +2419552|AAAAAAAAAGLOECAA|1912-05-29|148|648|50|1912|3|5|29|2|1912|50|648|Wednesday|1912Q2|N|N|N|2419524|2419644|2419186|2419461|N|N|N|N|N| +2419553|AAAAAAAABGLOECAA|1912-05-30|148|648|50|1912|4|5|30|2|1912|50|648|Thursday|1912Q2|N|N|N|2419524|2419644|2419187|2419462|N|N|N|N|N| +2419554|AAAAAAAACGLOECAA|1912-05-31|148|648|50|1912|5|5|31|2|1912|50|648|Friday|1912Q2|N|Y|N|2419524|2419644|2419188|2419463|N|N|N|N|N| +2419555|AAAAAAAADGLOECAA|1912-06-01|149|648|51|1912|6|6|1|2|1912|51|648|Saturday|1912Q2|N|Y|N|2419555|2419706|2419189|2419464|N|N|N|N|N| +2419556|AAAAAAAAEGLOECAA|1912-06-02|149|648|51|1912|0|6|2|2|1912|51|648|Sunday|1912Q2|N|N|N|2419555|2419706|2419190|2419465|N|N|N|N|N| +2419557|AAAAAAAAFGLOECAA|1912-06-03|149|648|51|1912|1|6|3|2|1912|51|648|Monday|1912Q2|N|N|N|2419555|2419706|2419191|2419466|N|N|N|N|N| +2419558|AAAAAAAAGGLOECAA|1912-06-04|149|649|51|1912|2|6|4|2|1912|51|649|Tuesday|1912Q2|N|N|N|2419555|2419706|2419192|2419467|N|N|N|N|N| +2419559|AAAAAAAAHGLOECAA|1912-06-05|149|649|51|1912|3|6|5|2|1912|51|649|Wednesday|1912Q2|N|N|N|2419555|2419706|2419193|2419468|N|N|N|N|N| +2419560|AAAAAAAAIGLOECAA|1912-06-06|149|649|51|1912|4|6|6|2|1912|51|649|Thursday|1912Q2|N|N|N|2419555|2419706|2419194|2419469|N|N|N|N|N| +2419561|AAAAAAAAJGLOECAA|1912-06-07|149|649|51|1912|5|6|7|2|1912|51|649|Friday|1912Q2|N|Y|N|2419555|2419706|2419195|2419470|N|N|N|N|N| +2419562|AAAAAAAAKGLOECAA|1912-06-08|149|649|51|1912|6|6|8|2|1912|51|649|Saturday|1912Q2|N|Y|N|2419555|2419706|2419196|2419471|N|N|N|N|N| +2419563|AAAAAAAALGLOECAA|1912-06-09|149|649|51|1912|0|6|9|2|1912|51|649|Sunday|1912Q2|N|N|N|2419555|2419706|2419197|2419472|N|N|N|N|N| +2419564|AAAAAAAAMGLOECAA|1912-06-10|149|649|51|1912|1|6|10|2|1912|51|649|Monday|1912Q2|N|N|N|2419555|2419706|2419198|2419473|N|N|N|N|N| +2419565|AAAAAAAANGLOECAA|1912-06-11|149|650|51|1912|2|6|11|2|1912|51|650|Tuesday|1912Q2|N|N|N|2419555|2419706|2419199|2419474|N|N|N|N|N| +2419566|AAAAAAAAOGLOECAA|1912-06-12|149|650|51|1912|3|6|12|2|1912|51|650|Wednesday|1912Q2|N|N|N|2419555|2419706|2419200|2419475|N|N|N|N|N| +2419567|AAAAAAAAPGLOECAA|1912-06-13|149|650|51|1912|4|6|13|2|1912|51|650|Thursday|1912Q2|N|N|N|2419555|2419706|2419201|2419476|N|N|N|N|N| +2419568|AAAAAAAAAHLOECAA|1912-06-14|149|650|51|1912|5|6|14|2|1912|51|650|Friday|1912Q2|N|Y|N|2419555|2419706|2419202|2419477|N|N|N|N|N| +2419569|AAAAAAAABHLOECAA|1912-06-15|149|650|51|1912|6|6|15|2|1912|51|650|Saturday|1912Q2|N|Y|N|2419555|2419706|2419203|2419478|N|N|N|N|N| +2419570|AAAAAAAACHLOECAA|1912-06-16|149|650|51|1912|0|6|16|2|1912|51|650|Sunday|1912Q2|N|N|N|2419555|2419706|2419204|2419479|N|N|N|N|N| +2419571|AAAAAAAADHLOECAA|1912-06-17|149|650|51|1912|1|6|17|2|1912|51|650|Monday|1912Q2|N|N|N|2419555|2419706|2419205|2419480|N|N|N|N|N| +2419572|AAAAAAAAEHLOECAA|1912-06-18|149|651|51|1912|2|6|18|2|1912|51|651|Tuesday|1912Q2|N|N|N|2419555|2419706|2419206|2419481|N|N|N|N|N| +2419573|AAAAAAAAFHLOECAA|1912-06-19|149|651|51|1912|3|6|19|2|1912|51|651|Wednesday|1912Q2|N|N|N|2419555|2419706|2419207|2419482|N|N|N|N|N| +2419574|AAAAAAAAGHLOECAA|1912-06-20|149|651|51|1912|4|6|20|2|1912|51|651|Thursday|1912Q2|N|N|N|2419555|2419706|2419208|2419483|N|N|N|N|N| +2419575|AAAAAAAAHHLOECAA|1912-06-21|149|651|51|1912|5|6|21|2|1912|51|651|Friday|1912Q2|N|Y|N|2419555|2419706|2419209|2419484|N|N|N|N|N| +2419576|AAAAAAAAIHLOECAA|1912-06-22|149|651|51|1912|6|6|22|2|1912|51|651|Saturday|1912Q2|N|Y|N|2419555|2419706|2419210|2419485|N|N|N|N|N| +2419577|AAAAAAAAJHLOECAA|1912-06-23|149|651|51|1912|0|6|23|2|1912|51|651|Sunday|1912Q2|N|N|N|2419555|2419706|2419211|2419486|N|N|N|N|N| +2419578|AAAAAAAAKHLOECAA|1912-06-24|149|651|51|1912|1|6|24|2|1912|51|651|Monday|1912Q2|N|N|N|2419555|2419706|2419212|2419487|N|N|N|N|N| +2419579|AAAAAAAALHLOECAA|1912-06-25|149|652|51|1912|2|6|25|2|1912|51|652|Tuesday|1912Q2|N|N|N|2419555|2419706|2419213|2419488|N|N|N|N|N| +2419580|AAAAAAAAMHLOECAA|1912-06-26|149|652|51|1912|3|6|26|2|1912|51|652|Wednesday|1912Q2|N|N|N|2419555|2419706|2419214|2419489|N|N|N|N|N| +2419581|AAAAAAAANHLOECAA|1912-06-27|149|652|51|1912|4|6|27|2|1912|51|652|Thursday|1912Q2|N|N|N|2419555|2419706|2419215|2419490|N|N|N|N|N| +2419582|AAAAAAAAOHLOECAA|1912-06-28|149|652|51|1912|5|6|28|2|1912|51|652|Friday|1912Q2|N|Y|N|2419555|2419706|2419216|2419491|N|N|N|N|N| +2419583|AAAAAAAAPHLOECAA|1912-06-29|149|652|51|1912|6|6|29|2|1912|51|652|Saturday|1912Q2|N|Y|N|2419555|2419706|2419217|2419492|N|N|N|N|N| +2419584|AAAAAAAAAILOECAA|1912-06-30|149|652|51|1912|0|6|30|2|1912|51|652|Sunday|1912Q2|N|N|N|2419555|2419706|2419218|2419493|N|N|N|N|N| +2419585|AAAAAAAABILOECAA|1912-07-01|150|652|51|1912|1|7|1|3|1912|51|652|Monday|1912Q3|N|N|N|2419585|2419766|2419219|2419494|N|N|N|N|N| +2419586|AAAAAAAACILOECAA|1912-07-02|150|653|51|1912|2|7|2|3|1912|51|653|Tuesday|1912Q3|N|N|N|2419585|2419766|2419220|2419495|N|N|N|N|N| +2419587|AAAAAAAADILOECAA|1912-07-03|150|653|51|1912|3|7|3|3|1912|51|653|Wednesday|1912Q3|N|N|N|2419585|2419766|2419221|2419496|N|N|N|N|N| +2419588|AAAAAAAAEILOECAA|1912-07-04|150|653|51|1912|4|7|4|3|1912|51|653|Thursday|1912Q3|Y|N|N|2419585|2419766|2419222|2419497|N|N|N|N|N| +2419589|AAAAAAAAFILOECAA|1912-07-05|150|653|51|1912|5|7|5|3|1912|51|653|Friday|1912Q3|N|Y|Y|2419585|2419766|2419223|2419498|N|N|N|N|N| +2419590|AAAAAAAAGILOECAA|1912-07-06|150|653|51|1912|6|7|6|3|1912|51|653|Saturday|1912Q3|N|Y|N|2419585|2419766|2419224|2419499|N|N|N|N|N| +2419591|AAAAAAAAHILOECAA|1912-07-07|150|653|51|1912|0|7|7|3|1912|51|653|Sunday|1912Q3|N|N|N|2419585|2419766|2419225|2419500|N|N|N|N|N| +2419592|AAAAAAAAIILOECAA|1912-07-08|150|653|51|1912|1|7|8|3|1912|51|653|Monday|1912Q3|N|N|N|2419585|2419766|2419226|2419501|N|N|N|N|N| +2419593|AAAAAAAAJILOECAA|1912-07-09|150|654|51|1912|2|7|9|3|1912|51|654|Tuesday|1912Q3|N|N|N|2419585|2419766|2419227|2419502|N|N|N|N|N| +2419594|AAAAAAAAKILOECAA|1912-07-10|150|654|51|1912|3|7|10|3|1912|51|654|Wednesday|1912Q3|N|N|N|2419585|2419766|2419228|2419503|N|N|N|N|N| +2419595|AAAAAAAALILOECAA|1912-07-11|150|654|51|1912|4|7|11|3|1912|51|654|Thursday|1912Q3|N|N|N|2419585|2419766|2419229|2419504|N|N|N|N|N| +2419596|AAAAAAAAMILOECAA|1912-07-12|150|654|51|1912|5|7|12|3|1912|51|654|Friday|1912Q3|N|Y|N|2419585|2419766|2419230|2419505|N|N|N|N|N| +2419597|AAAAAAAANILOECAA|1912-07-13|150|654|51|1912|6|7|13|3|1912|51|654|Saturday|1912Q3|N|Y|N|2419585|2419766|2419231|2419506|N|N|N|N|N| +2419598|AAAAAAAAOILOECAA|1912-07-14|150|654|51|1912|0|7|14|3|1912|51|654|Sunday|1912Q3|N|N|N|2419585|2419766|2419232|2419507|N|N|N|N|N| +2419599|AAAAAAAAPILOECAA|1912-07-15|150|654|51|1912|1|7|15|3|1912|51|654|Monday|1912Q3|N|N|N|2419585|2419766|2419233|2419508|N|N|N|N|N| +2419600|AAAAAAAAAJLOECAA|1912-07-16|150|655|51|1912|2|7|16|3|1912|51|655|Tuesday|1912Q3|N|N|N|2419585|2419766|2419234|2419509|N|N|N|N|N| +2419601|AAAAAAAABJLOECAA|1912-07-17|150|655|51|1912|3|7|17|3|1912|51|655|Wednesday|1912Q3|N|N|N|2419585|2419766|2419235|2419510|N|N|N|N|N| +2419602|AAAAAAAACJLOECAA|1912-07-18|150|655|51|1912|4|7|18|3|1912|51|655|Thursday|1912Q3|N|N|N|2419585|2419766|2419236|2419511|N|N|N|N|N| +2419603|AAAAAAAADJLOECAA|1912-07-19|150|655|51|1912|5|7|19|3|1912|51|655|Friday|1912Q3|N|Y|N|2419585|2419766|2419237|2419512|N|N|N|N|N| +2419604|AAAAAAAAEJLOECAA|1912-07-20|150|655|51|1912|6|7|20|3|1912|51|655|Saturday|1912Q3|N|Y|N|2419585|2419766|2419238|2419513|N|N|N|N|N| +2419605|AAAAAAAAFJLOECAA|1912-07-21|150|655|51|1912|0|7|21|3|1912|51|655|Sunday|1912Q3|N|N|N|2419585|2419766|2419239|2419514|N|N|N|N|N| +2419606|AAAAAAAAGJLOECAA|1912-07-22|150|655|51|1912|1|7|22|3|1912|51|655|Monday|1912Q3|N|N|N|2419585|2419766|2419240|2419515|N|N|N|N|N| +2419607|AAAAAAAAHJLOECAA|1912-07-23|150|656|51|1912|2|7|23|3|1912|51|656|Tuesday|1912Q3|N|N|N|2419585|2419766|2419241|2419516|N|N|N|N|N| +2419608|AAAAAAAAIJLOECAA|1912-07-24|150|656|51|1912|3|7|24|3|1912|51|656|Wednesday|1912Q3|N|N|N|2419585|2419766|2419242|2419517|N|N|N|N|N| +2419609|AAAAAAAAJJLOECAA|1912-07-25|150|656|51|1912|4|7|25|3|1912|51|656|Thursday|1912Q3|N|N|N|2419585|2419766|2419243|2419518|N|N|N|N|N| +2419610|AAAAAAAAKJLOECAA|1912-07-26|150|656|51|1912|5|7|26|3|1912|51|656|Friday|1912Q3|N|Y|N|2419585|2419766|2419244|2419519|N|N|N|N|N| +2419611|AAAAAAAALJLOECAA|1912-07-27|150|656|51|1912|6|7|27|3|1912|51|656|Saturday|1912Q3|N|Y|N|2419585|2419766|2419245|2419520|N|N|N|N|N| +2419612|AAAAAAAAMJLOECAA|1912-07-28|150|656|51|1912|0|7|28|3|1912|51|656|Sunday|1912Q3|N|N|N|2419585|2419766|2419246|2419521|N|N|N|N|N| +2419613|AAAAAAAANJLOECAA|1912-07-29|150|656|51|1912|1|7|29|3|1912|51|656|Monday|1912Q3|N|N|N|2419585|2419766|2419247|2419522|N|N|N|N|N| +2419614|AAAAAAAAOJLOECAA|1912-07-30|150|657|51|1912|2|7|30|3|1912|51|657|Tuesday|1912Q3|N|N|N|2419585|2419766|2419248|2419523|N|N|N|N|N| +2419615|AAAAAAAAPJLOECAA|1912-07-31|150|657|51|1912|3|7|31|3|1912|51|657|Wednesday|1912Q3|N|N|N|2419585|2419766|2419249|2419524|N|N|N|N|N| +2419616|AAAAAAAAAKLOECAA|1912-08-01|151|657|51|1912|4|8|1|3|1912|51|657|Thursday|1912Q3|N|N|N|2419616|2419828|2419250|2419525|N|N|N|N|N| +2419617|AAAAAAAABKLOECAA|1912-08-02|151|657|51|1912|5|8|2|3|1912|51|657|Friday|1912Q3|N|Y|N|2419616|2419828|2419251|2419526|N|N|N|N|N| +2419618|AAAAAAAACKLOECAA|1912-08-03|151|657|51|1912|6|8|3|3|1912|51|657|Saturday|1912Q3|N|Y|N|2419616|2419828|2419252|2419527|N|N|N|N|N| +2419619|AAAAAAAADKLOECAA|1912-08-04|151|657|51|1912|0|8|4|3|1912|51|657|Sunday|1912Q3|N|N|N|2419616|2419828|2419253|2419528|N|N|N|N|N| +2419620|AAAAAAAAEKLOECAA|1912-08-05|151|657|51|1912|1|8|5|3|1912|51|657|Monday|1912Q3|N|N|N|2419616|2419828|2419254|2419529|N|N|N|N|N| +2419621|AAAAAAAAFKLOECAA|1912-08-06|151|658|51|1912|2|8|6|3|1912|51|658|Tuesday|1912Q3|N|N|N|2419616|2419828|2419255|2419530|N|N|N|N|N| +2419622|AAAAAAAAGKLOECAA|1912-08-07|151|658|51|1912|3|8|7|3|1912|51|658|Wednesday|1912Q3|N|N|N|2419616|2419828|2419256|2419531|N|N|N|N|N| +2419623|AAAAAAAAHKLOECAA|1912-08-08|151|658|51|1912|4|8|8|3|1912|51|658|Thursday|1912Q3|N|N|N|2419616|2419828|2419257|2419532|N|N|N|N|N| +2419624|AAAAAAAAIKLOECAA|1912-08-09|151|658|51|1912|5|8|9|3|1912|51|658|Friday|1912Q3|N|Y|N|2419616|2419828|2419258|2419533|N|N|N|N|N| +2419625|AAAAAAAAJKLOECAA|1912-08-10|151|658|51|1912|6|8|10|3|1912|51|658|Saturday|1912Q3|N|Y|N|2419616|2419828|2419259|2419534|N|N|N|N|N| +2419626|AAAAAAAAKKLOECAA|1912-08-11|151|658|51|1912|0|8|11|3|1912|51|658|Sunday|1912Q3|N|N|N|2419616|2419828|2419260|2419535|N|N|N|N|N| +2419627|AAAAAAAALKLOECAA|1912-08-12|151|658|51|1912|1|8|12|3|1912|51|658|Monday|1912Q3|N|N|N|2419616|2419828|2419261|2419536|N|N|N|N|N| +2419628|AAAAAAAAMKLOECAA|1912-08-13|151|659|51|1912|2|8|13|3|1912|51|659|Tuesday|1912Q3|N|N|N|2419616|2419828|2419262|2419537|N|N|N|N|N| +2419629|AAAAAAAANKLOECAA|1912-08-14|151|659|51|1912|3|8|14|3|1912|51|659|Wednesday|1912Q3|N|N|N|2419616|2419828|2419263|2419538|N|N|N|N|N| +2419630|AAAAAAAAOKLOECAA|1912-08-15|151|659|51|1912|4|8|15|3|1912|51|659|Thursday|1912Q3|N|N|N|2419616|2419828|2419264|2419539|N|N|N|N|N| +2419631|AAAAAAAAPKLOECAA|1912-08-16|151|659|51|1912|5|8|16|3|1912|51|659|Friday|1912Q3|N|Y|N|2419616|2419828|2419265|2419540|N|N|N|N|N| +2419632|AAAAAAAAALLOECAA|1912-08-17|151|659|51|1912|6|8|17|3|1912|51|659|Saturday|1912Q3|N|Y|N|2419616|2419828|2419266|2419541|N|N|N|N|N| +2419633|AAAAAAAABLLOECAA|1912-08-18|151|659|51|1912|0|8|18|3|1912|51|659|Sunday|1912Q3|N|N|N|2419616|2419828|2419267|2419542|N|N|N|N|N| +2419634|AAAAAAAACLLOECAA|1912-08-19|151|659|51|1912|1|8|19|3|1912|51|659|Monday|1912Q3|N|N|N|2419616|2419828|2419268|2419543|N|N|N|N|N| +2419635|AAAAAAAADLLOECAA|1912-08-20|151|660|51|1912|2|8|20|3|1912|51|660|Tuesday|1912Q3|N|N|N|2419616|2419828|2419269|2419544|N|N|N|N|N| +2419636|AAAAAAAAELLOECAA|1912-08-21|151|660|51|1912|3|8|21|3|1912|51|660|Wednesday|1912Q3|N|N|N|2419616|2419828|2419270|2419545|N|N|N|N|N| +2419637|AAAAAAAAFLLOECAA|1912-08-22|151|660|51|1912|4|8|22|3|1912|51|660|Thursday|1912Q3|N|N|N|2419616|2419828|2419271|2419546|N|N|N|N|N| +2419638|AAAAAAAAGLLOECAA|1912-08-23|151|660|51|1912|5|8|23|3|1912|51|660|Friday|1912Q3|N|Y|N|2419616|2419828|2419272|2419547|N|N|N|N|N| +2419639|AAAAAAAAHLLOECAA|1912-08-24|151|660|51|1912|6|8|24|3|1912|51|660|Saturday|1912Q3|N|Y|N|2419616|2419828|2419273|2419548|N|N|N|N|N| +2419640|AAAAAAAAILLOECAA|1912-08-25|151|660|51|1912|0|8|25|3|1912|51|660|Sunday|1912Q3|N|N|N|2419616|2419828|2419274|2419549|N|N|N|N|N| +2419641|AAAAAAAAJLLOECAA|1912-08-26|151|660|51|1912|1|8|26|3|1912|51|660|Monday|1912Q3|N|N|N|2419616|2419828|2419275|2419550|N|N|N|N|N| +2419642|AAAAAAAAKLLOECAA|1912-08-27|151|661|51|1912|2|8|27|3|1912|51|661|Tuesday|1912Q3|N|N|N|2419616|2419828|2419276|2419551|N|N|N|N|N| +2419643|AAAAAAAALLLOECAA|1912-08-28|151|661|51|1912|3|8|28|3|1912|51|661|Wednesday|1912Q3|N|N|N|2419616|2419828|2419277|2419552|N|N|N|N|N| +2419644|AAAAAAAAMLLOECAA|1912-08-29|151|661|51|1912|4|8|29|3|1912|51|661|Thursday|1912Q3|N|N|N|2419616|2419828|2419278|2419553|N|N|N|N|N| +2419645|AAAAAAAANLLOECAA|1912-08-30|151|661|51|1912|5|8|30|3|1912|51|661|Friday|1912Q3|N|Y|N|2419616|2419828|2419279|2419554|N|N|N|N|N| +2419646|AAAAAAAAOLLOECAA|1912-08-31|151|661|51|1912|6|8|31|3|1912|51|661|Saturday|1912Q3|N|Y|N|2419616|2419828|2419280|2419555|N|N|N|N|N| +2419647|AAAAAAAAPLLOECAA|1912-09-01|152|661|52|1912|0|9|1|3|1912|52|661|Sunday|1912Q3|N|N|N|2419647|2419890|2419281|2419556|N|N|N|N|N| +2419648|AAAAAAAAAMLOECAA|1912-09-02|152|661|52|1912|1|9|2|3|1912|52|661|Monday|1912Q3|N|N|N|2419647|2419890|2419282|2419557|N|N|N|N|N| +2419649|AAAAAAAABMLOECAA|1912-09-03|152|662|52|1912|2|9|3|3|1912|52|662|Tuesday|1912Q3|N|N|N|2419647|2419890|2419283|2419558|N|N|N|N|N| +2419650|AAAAAAAACMLOECAA|1912-09-04|152|662|52|1912|3|9|4|3|1912|52|662|Wednesday|1912Q3|N|N|N|2419647|2419890|2419284|2419559|N|N|N|N|N| +2419651|AAAAAAAADMLOECAA|1912-09-05|152|662|52|1912|4|9|5|3|1912|52|662|Thursday|1912Q3|N|N|N|2419647|2419890|2419285|2419560|N|N|N|N|N| +2419652|AAAAAAAAEMLOECAA|1912-09-06|152|662|52|1912|5|9|6|3|1912|52|662|Friday|1912Q3|N|Y|N|2419647|2419890|2419286|2419561|N|N|N|N|N| +2419653|AAAAAAAAFMLOECAA|1912-09-07|152|662|52|1912|6|9|7|3|1912|52|662|Saturday|1912Q3|N|Y|N|2419647|2419890|2419287|2419562|N|N|N|N|N| +2419654|AAAAAAAAGMLOECAA|1912-09-08|152|662|52|1912|0|9|8|3|1912|52|662|Sunday|1912Q3|N|N|N|2419647|2419890|2419288|2419563|N|N|N|N|N| +2419655|AAAAAAAAHMLOECAA|1912-09-09|152|662|52|1912|1|9|9|3|1912|52|662|Monday|1912Q3|N|N|N|2419647|2419890|2419289|2419564|N|N|N|N|N| +2419656|AAAAAAAAIMLOECAA|1912-09-10|152|663|52|1912|2|9|10|3|1912|52|663|Tuesday|1912Q3|N|N|N|2419647|2419890|2419290|2419565|N|N|N|N|N| +2419657|AAAAAAAAJMLOECAA|1912-09-11|152|663|52|1912|3|9|11|3|1912|52|663|Wednesday|1912Q3|N|N|N|2419647|2419890|2419291|2419566|N|N|N|N|N| +2419658|AAAAAAAAKMLOECAA|1912-09-12|152|663|52|1912|4|9|12|3|1912|52|663|Thursday|1912Q3|N|N|N|2419647|2419890|2419292|2419567|N|N|N|N|N| +2419659|AAAAAAAALMLOECAA|1912-09-13|152|663|52|1912|5|9|13|3|1912|52|663|Friday|1912Q3|N|Y|N|2419647|2419890|2419293|2419568|N|N|N|N|N| +2419660|AAAAAAAAMMLOECAA|1912-09-14|152|663|52|1912|6|9|14|3|1912|52|663|Saturday|1912Q3|N|Y|N|2419647|2419890|2419294|2419569|N|N|N|N|N| +2419661|AAAAAAAANMLOECAA|1912-09-15|152|663|52|1912|0|9|15|3|1912|52|663|Sunday|1912Q3|N|N|N|2419647|2419890|2419295|2419570|N|N|N|N|N| +2419662|AAAAAAAAOMLOECAA|1912-09-16|152|663|52|1912|1|9|16|3|1912|52|663|Monday|1912Q3|N|N|N|2419647|2419890|2419296|2419571|N|N|N|N|N| +2419663|AAAAAAAAPMLOECAA|1912-09-17|152|664|52|1912|2|9|17|3|1912|52|664|Tuesday|1912Q3|N|N|N|2419647|2419890|2419297|2419572|N|N|N|N|N| +2419664|AAAAAAAAANLOECAA|1912-09-18|152|664|52|1912|3|9|18|3|1912|52|664|Wednesday|1912Q3|N|N|N|2419647|2419890|2419298|2419573|N|N|N|N|N| +2419665|AAAAAAAABNLOECAA|1912-09-19|152|664|52|1912|4|9|19|3|1912|52|664|Thursday|1912Q3|N|N|N|2419647|2419890|2419299|2419574|N|N|N|N|N| +2419666|AAAAAAAACNLOECAA|1912-09-20|152|664|52|1912|5|9|20|3|1912|52|664|Friday|1912Q3|N|Y|N|2419647|2419890|2419300|2419575|N|N|N|N|N| +2419667|AAAAAAAADNLOECAA|1912-09-21|152|664|52|1912|6|9|21|3|1912|52|664|Saturday|1912Q3|N|Y|N|2419647|2419890|2419301|2419576|N|N|N|N|N| +2419668|AAAAAAAAENLOECAA|1912-09-22|152|664|52|1912|0|9|22|3|1912|52|664|Sunday|1912Q3|N|N|N|2419647|2419890|2419302|2419577|N|N|N|N|N| +2419669|AAAAAAAAFNLOECAA|1912-09-23|152|664|52|1912|1|9|23|3|1912|52|664|Monday|1912Q3|N|N|N|2419647|2419890|2419303|2419578|N|N|N|N|N| +2419670|AAAAAAAAGNLOECAA|1912-09-24|152|665|52|1912|2|9|24|3|1912|52|665|Tuesday|1912Q3|N|N|N|2419647|2419890|2419304|2419579|N|N|N|N|N| +2419671|AAAAAAAAHNLOECAA|1912-09-25|152|665|52|1912|3|9|25|3|1912|52|665|Wednesday|1912Q3|N|N|N|2419647|2419890|2419305|2419580|N|N|N|N|N| +2419672|AAAAAAAAINLOECAA|1912-09-26|152|665|52|1912|4|9|26|3|1912|52|665|Thursday|1912Q3|N|N|N|2419647|2419890|2419306|2419581|N|N|N|N|N| +2419673|AAAAAAAAJNLOECAA|1912-09-27|152|665|52|1912|5|9|27|3|1912|52|665|Friday|1912Q3|N|Y|N|2419647|2419890|2419307|2419582|N|N|N|N|N| +2419674|AAAAAAAAKNLOECAA|1912-09-28|152|665|52|1912|6|9|28|3|1912|52|665|Saturday|1912Q3|N|Y|N|2419647|2419890|2419308|2419583|N|N|N|N|N| +2419675|AAAAAAAALNLOECAA|1912-09-29|152|665|52|1912|0|9|29|3|1912|52|665|Sunday|1912Q3|N|N|N|2419647|2419890|2419309|2419584|N|N|N|N|N| +2419676|AAAAAAAAMNLOECAA|1912-09-30|152|665|52|1912|1|9|30|3|1912|52|665|Monday|1912Q3|N|N|N|2419647|2419890|2419310|2419585|N|N|N|N|N| +2419677|AAAAAAAANNLOECAA|1912-10-01|153|666|52|1912|2|10|1|4|1912|52|666|Tuesday|1912Q4|N|N|N|2419677|2419950|2419311|2419585|N|N|N|N|N| +2419678|AAAAAAAAONLOECAA|1912-10-02|153|666|52|1912|3|10|2|4|1912|52|666|Wednesday|1912Q4|N|N|N|2419677|2419950|2419312|2419586|N|N|N|N|N| +2419679|AAAAAAAAPNLOECAA|1912-10-03|153|666|52|1912|4|10|3|4|1912|52|666|Thursday|1912Q4|N|N|N|2419677|2419950|2419313|2419587|N|N|N|N|N| +2419680|AAAAAAAAAOLOECAA|1912-10-04|153|666|52|1912|5|10|4|4|1912|52|666|Friday|1912Q4|N|Y|N|2419677|2419950|2419314|2419588|N|N|N|N|N| +2419681|AAAAAAAABOLOECAA|1912-10-05|153|666|52|1912|6|10|5|4|1912|52|666|Saturday|1912Q4|N|Y|N|2419677|2419950|2419315|2419589|N|N|N|N|N| +2419682|AAAAAAAACOLOECAA|1912-10-06|153|666|52|1912|0|10|6|4|1912|52|666|Sunday|1912Q4|N|N|N|2419677|2419950|2419316|2419590|N|N|N|N|N| +2419683|AAAAAAAADOLOECAA|1912-10-07|153|666|52|1912|1|10|7|4|1912|52|666|Monday|1912Q4|N|N|N|2419677|2419950|2419317|2419591|N|N|N|N|N| +2419684|AAAAAAAAEOLOECAA|1912-10-08|153|667|52|1912|2|10|8|4|1912|52|667|Tuesday|1912Q4|N|N|N|2419677|2419950|2419318|2419592|N|N|N|N|N| +2419685|AAAAAAAAFOLOECAA|1912-10-09|153|667|52|1912|3|10|9|4|1912|52|667|Wednesday|1912Q4|N|N|N|2419677|2419950|2419319|2419593|N|N|N|N|N| +2419686|AAAAAAAAGOLOECAA|1912-10-10|153|667|52|1912|4|10|10|4|1912|52|667|Thursday|1912Q4|N|N|N|2419677|2419950|2419320|2419594|N|N|N|N|N| +2419687|AAAAAAAAHOLOECAA|1912-10-11|153|667|52|1912|5|10|11|4|1912|52|667|Friday|1912Q4|N|Y|N|2419677|2419950|2419321|2419595|N|N|N|N|N| +2419688|AAAAAAAAIOLOECAA|1912-10-12|153|667|52|1912|6|10|12|4|1912|52|667|Saturday|1912Q4|N|Y|N|2419677|2419950|2419322|2419596|N|N|N|N|N| +2419689|AAAAAAAAJOLOECAA|1912-10-13|153|667|52|1912|0|10|13|4|1912|52|667|Sunday|1912Q4|N|N|N|2419677|2419950|2419323|2419597|N|N|N|N|N| +2419690|AAAAAAAAKOLOECAA|1912-10-14|153|667|52|1912|1|10|14|4|1912|52|667|Monday|1912Q4|N|N|N|2419677|2419950|2419324|2419598|N|N|N|N|N| +2419691|AAAAAAAALOLOECAA|1912-10-15|153|668|52|1912|2|10|15|4|1912|52|668|Tuesday|1912Q4|N|N|N|2419677|2419950|2419325|2419599|N|N|N|N|N| +2419692|AAAAAAAAMOLOECAA|1912-10-16|153|668|52|1912|3|10|16|4|1912|52|668|Wednesday|1912Q4|N|N|N|2419677|2419950|2419326|2419600|N|N|N|N|N| +2419693|AAAAAAAANOLOECAA|1912-10-17|153|668|52|1912|4|10|17|4|1912|52|668|Thursday|1912Q4|N|N|N|2419677|2419950|2419327|2419601|N|N|N|N|N| +2419694|AAAAAAAAOOLOECAA|1912-10-18|153|668|52|1912|5|10|18|4|1912|52|668|Friday|1912Q4|N|Y|N|2419677|2419950|2419328|2419602|N|N|N|N|N| +2419695|AAAAAAAAPOLOECAA|1912-10-19|153|668|52|1912|6|10|19|4|1912|52|668|Saturday|1912Q4|N|Y|N|2419677|2419950|2419329|2419603|N|N|N|N|N| +2419696|AAAAAAAAAPLOECAA|1912-10-20|153|668|52|1912|0|10|20|4|1912|52|668|Sunday|1912Q4|N|N|N|2419677|2419950|2419330|2419604|N|N|N|N|N| +2419697|AAAAAAAABPLOECAA|1912-10-21|153|668|52|1912|1|10|21|4|1912|52|668|Monday|1912Q4|N|N|N|2419677|2419950|2419331|2419605|N|N|N|N|N| +2419698|AAAAAAAACPLOECAA|1912-10-22|153|669|52|1912|2|10|22|4|1912|52|669|Tuesday|1912Q4|N|N|N|2419677|2419950|2419332|2419606|N|N|N|N|N| +2419699|AAAAAAAADPLOECAA|1912-10-23|153|669|52|1912|3|10|23|4|1912|52|669|Wednesday|1912Q4|N|N|N|2419677|2419950|2419333|2419607|N|N|N|N|N| +2419700|AAAAAAAAEPLOECAA|1912-10-24|153|669|52|1912|4|10|24|4|1912|52|669|Thursday|1912Q4|N|N|N|2419677|2419950|2419334|2419608|N|N|N|N|N| +2419701|AAAAAAAAFPLOECAA|1912-10-25|153|669|52|1912|5|10|25|4|1912|52|669|Friday|1912Q4|N|Y|N|2419677|2419950|2419335|2419609|N|N|N|N|N| +2419702|AAAAAAAAGPLOECAA|1912-10-26|153|669|52|1912|6|10|26|4|1912|52|669|Saturday|1912Q4|N|Y|N|2419677|2419950|2419336|2419610|N|N|N|N|N| +2419703|AAAAAAAAHPLOECAA|1912-10-27|153|669|52|1912|0|10|27|4|1912|52|669|Sunday|1912Q4|N|N|N|2419677|2419950|2419337|2419611|N|N|N|N|N| +2419704|AAAAAAAAIPLOECAA|1912-10-28|153|669|52|1912|1|10|28|4|1912|52|669|Monday|1912Q4|N|N|N|2419677|2419950|2419338|2419612|N|N|N|N|N| +2419705|AAAAAAAAJPLOECAA|1912-10-29|153|670|52|1912|2|10|29|4|1912|52|670|Tuesday|1912Q4|N|N|N|2419677|2419950|2419339|2419613|N|N|N|N|N| +2419706|AAAAAAAAKPLOECAA|1912-10-30|153|670|52|1912|3|10|30|4|1912|52|670|Wednesday|1912Q4|N|N|N|2419677|2419950|2419340|2419614|N|N|N|N|N| +2419707|AAAAAAAALPLOECAA|1912-10-31|153|670|52|1912|4|10|31|4|1912|52|670|Thursday|1912Q4|N|N|N|2419677|2419950|2419341|2419615|N|N|N|N|N| +2419708|AAAAAAAAMPLOECAA|1912-11-01|154|670|52|1912|5|11|1|4|1912|52|670|Friday|1912Q4|N|Y|N|2419708|2420012|2419342|2419616|N|N|N|N|N| +2419709|AAAAAAAANPLOECAA|1912-11-02|154|670|52|1912|6|11|2|4|1912|52|670|Saturday|1912Q4|N|Y|N|2419708|2420012|2419343|2419617|N|N|N|N|N| +2419710|AAAAAAAAOPLOECAA|1912-11-03|154|670|52|1912|0|11|3|4|1912|52|670|Sunday|1912Q4|N|N|N|2419708|2420012|2419344|2419618|N|N|N|N|N| +2419711|AAAAAAAAPPLOECAA|1912-11-04|154|670|52|1912|1|11|4|4|1912|52|670|Monday|1912Q4|N|N|N|2419708|2420012|2419345|2419619|N|N|N|N|N| +2419712|AAAAAAAAAAMOECAA|1912-11-05|154|671|52|1912|2|11|5|4|1912|52|671|Tuesday|1912Q4|N|N|N|2419708|2420012|2419346|2419620|N|N|N|N|N| +2419713|AAAAAAAABAMOECAA|1912-11-06|154|671|52|1912|3|11|6|4|1912|52|671|Wednesday|1912Q4|N|N|N|2419708|2420012|2419347|2419621|N|N|N|N|N| +2419714|AAAAAAAACAMOECAA|1912-11-07|154|671|52|1912|4|11|7|4|1912|52|671|Thursday|1912Q4|N|N|N|2419708|2420012|2419348|2419622|N|N|N|N|N| +2419715|AAAAAAAADAMOECAA|1912-11-08|154|671|52|1912|5|11|8|4|1912|52|671|Friday|1912Q4|N|Y|N|2419708|2420012|2419349|2419623|N|N|N|N|N| +2419716|AAAAAAAAEAMOECAA|1912-11-09|154|671|52|1912|6|11|9|4|1912|52|671|Saturday|1912Q4|N|Y|N|2419708|2420012|2419350|2419624|N|N|N|N|N| +2419717|AAAAAAAAFAMOECAA|1912-11-10|154|671|52|1912|0|11|10|4|1912|52|671|Sunday|1912Q4|N|N|N|2419708|2420012|2419351|2419625|N|N|N|N|N| +2419718|AAAAAAAAGAMOECAA|1912-11-11|154|671|52|1912|1|11|11|4|1912|52|671|Monday|1912Q4|N|N|N|2419708|2420012|2419352|2419626|N|N|N|N|N| +2419719|AAAAAAAAHAMOECAA|1912-11-12|154|672|52|1912|2|11|12|4|1912|52|672|Tuesday|1912Q4|N|N|N|2419708|2420012|2419353|2419627|N|N|N|N|N| +2419720|AAAAAAAAIAMOECAA|1912-11-13|154|672|52|1912|3|11|13|4|1912|52|672|Wednesday|1912Q4|N|N|N|2419708|2420012|2419354|2419628|N|N|N|N|N| +2419721|AAAAAAAAJAMOECAA|1912-11-14|154|672|52|1912|4|11|14|4|1912|52|672|Thursday|1912Q4|N|N|N|2419708|2420012|2419355|2419629|N|N|N|N|N| +2419722|AAAAAAAAKAMOECAA|1912-11-15|154|672|52|1912|5|11|15|4|1912|52|672|Friday|1912Q4|N|Y|N|2419708|2420012|2419356|2419630|N|N|N|N|N| +2419723|AAAAAAAALAMOECAA|1912-11-16|154|672|52|1912|6|11|16|4|1912|52|672|Saturday|1912Q4|N|Y|N|2419708|2420012|2419357|2419631|N|N|N|N|N| +2419724|AAAAAAAAMAMOECAA|1912-11-17|154|672|52|1912|0|11|17|4|1912|52|672|Sunday|1912Q4|N|N|N|2419708|2420012|2419358|2419632|N|N|N|N|N| +2419725|AAAAAAAANAMOECAA|1912-11-18|154|672|52|1912|1|11|18|4|1912|52|672|Monday|1912Q4|N|N|N|2419708|2420012|2419359|2419633|N|N|N|N|N| +2419726|AAAAAAAAOAMOECAA|1912-11-19|154|673|52|1912|2|11|19|4|1912|52|673|Tuesday|1912Q4|N|N|N|2419708|2420012|2419360|2419634|N|N|N|N|N| +2419727|AAAAAAAAPAMOECAA|1912-11-20|154|673|52|1912|3|11|20|4|1912|52|673|Wednesday|1912Q4|N|N|N|2419708|2420012|2419361|2419635|N|N|N|N|N| +2419728|AAAAAAAAABMOECAA|1912-11-21|154|673|52|1912|4|11|21|4|1912|52|673|Thursday|1912Q4|N|N|N|2419708|2420012|2419362|2419636|N|N|N|N|N| +2419729|AAAAAAAABBMOECAA|1912-11-22|154|673|52|1912|5|11|22|4|1912|52|673|Friday|1912Q4|N|Y|N|2419708|2420012|2419363|2419637|N|N|N|N|N| +2419730|AAAAAAAACBMOECAA|1912-11-23|154|673|52|1912|6|11|23|4|1912|52|673|Saturday|1912Q4|N|Y|N|2419708|2420012|2419364|2419638|N|N|N|N|N| +2419731|AAAAAAAADBMOECAA|1912-11-24|154|673|52|1912|0|11|24|4|1912|52|673|Sunday|1912Q4|N|N|N|2419708|2420012|2419365|2419639|N|N|N|N|N| +2419732|AAAAAAAAEBMOECAA|1912-11-25|154|673|52|1912|1|11|25|4|1912|52|673|Monday|1912Q4|N|N|N|2419708|2420012|2419366|2419640|N|N|N|N|N| +2419733|AAAAAAAAFBMOECAA|1912-11-26|154|674|52|1912|2|11|26|4|1912|52|674|Tuesday|1912Q4|N|N|N|2419708|2420012|2419367|2419641|N|N|N|N|N| +2419734|AAAAAAAAGBMOECAA|1912-11-27|154|674|52|1912|3|11|27|4|1912|52|674|Wednesday|1912Q4|N|N|N|2419708|2420012|2419368|2419642|N|N|N|N|N| +2419735|AAAAAAAAHBMOECAA|1912-11-28|154|674|52|1912|4|11|28|4|1912|52|674|Thursday|1912Q4|N|N|N|2419708|2420012|2419369|2419643|N|N|N|N|N| +2419736|AAAAAAAAIBMOECAA|1912-11-29|154|674|52|1912|5|11|29|4|1912|52|674|Friday|1912Q4|N|Y|N|2419708|2420012|2419370|2419644|N|N|N|N|N| +2419737|AAAAAAAAJBMOECAA|1912-11-30|154|674|52|1912|6|11|30|4|1912|52|674|Saturday|1912Q4|N|Y|N|2419708|2420012|2419371|2419645|N|N|N|N|N| +2419738|AAAAAAAAKBMOECAA|1912-12-01|155|674|53|1912|0|12|1|4|1912|53|674|Sunday|1912Q4|N|N|N|2419738|2420072|2419372|2419646|N|N|N|N|N| +2419739|AAAAAAAALBMOECAA|1912-12-02|155|674|53|1912|1|12|2|4|1912|53|674|Monday|1912Q4|N|N|N|2419738|2420072|2419373|2419647|N|N|N|N|N| +2419740|AAAAAAAAMBMOECAA|1912-12-03|155|675|53|1912|2|12|3|4|1912|53|675|Tuesday|1912Q4|N|N|N|2419738|2420072|2419374|2419648|N|N|N|N|N| +2419741|AAAAAAAANBMOECAA|1912-12-04|155|675|53|1912|3|12|4|4|1912|53|675|Wednesday|1912Q4|N|N|N|2419738|2420072|2419375|2419649|N|N|N|N|N| +2419742|AAAAAAAAOBMOECAA|1912-12-05|155|675|53|1912|4|12|5|4|1912|53|675|Thursday|1912Q4|N|N|N|2419738|2420072|2419376|2419650|N|N|N|N|N| +2419743|AAAAAAAAPBMOECAA|1912-12-06|155|675|53|1912|5|12|6|4|1912|53|675|Friday|1912Q4|N|Y|N|2419738|2420072|2419377|2419651|N|N|N|N|N| +2419744|AAAAAAAAACMOECAA|1912-12-07|155|675|53|1912|6|12|7|4|1912|53|675|Saturday|1912Q4|N|Y|N|2419738|2420072|2419378|2419652|N|N|N|N|N| +2419745|AAAAAAAABCMOECAA|1912-12-08|155|675|53|1912|0|12|8|4|1912|53|675|Sunday|1912Q4|N|N|N|2419738|2420072|2419379|2419653|N|N|N|N|N| +2419746|AAAAAAAACCMOECAA|1912-12-09|155|675|53|1912|1|12|9|4|1912|53|675|Monday|1912Q4|N|N|N|2419738|2420072|2419380|2419654|N|N|N|N|N| +2419747|AAAAAAAADCMOECAA|1912-12-10|155|676|53|1912|2|12|10|4|1912|53|676|Tuesday|1912Q4|N|N|N|2419738|2420072|2419381|2419655|N|N|N|N|N| +2419748|AAAAAAAAECMOECAA|1912-12-11|155|676|53|1912|3|12|11|4|1912|53|676|Wednesday|1912Q4|N|N|N|2419738|2420072|2419382|2419656|N|N|N|N|N| +2419749|AAAAAAAAFCMOECAA|1912-12-12|155|676|53|1912|4|12|12|4|1912|53|676|Thursday|1912Q4|N|N|N|2419738|2420072|2419383|2419657|N|N|N|N|N| +2419750|AAAAAAAAGCMOECAA|1912-12-13|155|676|53|1912|5|12|13|4|1912|53|676|Friday|1912Q4|N|Y|N|2419738|2420072|2419384|2419658|N|N|N|N|N| +2419751|AAAAAAAAHCMOECAA|1912-12-14|155|676|53|1912|6|12|14|4|1912|53|676|Saturday|1912Q4|N|Y|N|2419738|2420072|2419385|2419659|N|N|N|N|N| +2419752|AAAAAAAAICMOECAA|1912-12-15|155|676|53|1912|0|12|15|4|1912|53|676|Sunday|1912Q4|N|N|N|2419738|2420072|2419386|2419660|N|N|N|N|N| +2419753|AAAAAAAAJCMOECAA|1912-12-16|155|676|53|1912|1|12|16|4|1912|53|676|Monday|1912Q4|N|N|N|2419738|2420072|2419387|2419661|N|N|N|N|N| +2419754|AAAAAAAAKCMOECAA|1912-12-17|155|677|53|1912|2|12|17|4|1912|53|677|Tuesday|1912Q4|N|N|N|2419738|2420072|2419388|2419662|N|N|N|N|N| +2419755|AAAAAAAALCMOECAA|1912-12-18|155|677|53|1912|3|12|18|4|1912|53|677|Wednesday|1912Q4|N|N|N|2419738|2420072|2419389|2419663|N|N|N|N|N| +2419756|AAAAAAAAMCMOECAA|1912-12-19|155|677|53|1912|4|12|19|4|1912|53|677|Thursday|1912Q4|N|N|N|2419738|2420072|2419390|2419664|N|N|N|N|N| +2419757|AAAAAAAANCMOECAA|1912-12-20|155|677|53|1912|5|12|20|4|1912|53|677|Friday|1912Q4|N|Y|N|2419738|2420072|2419391|2419665|N|N|N|N|N| +2419758|AAAAAAAAOCMOECAA|1912-12-21|155|677|53|1912|6|12|21|4|1912|53|677|Saturday|1912Q4|N|Y|N|2419738|2420072|2419392|2419666|N|N|N|N|N| +2419759|AAAAAAAAPCMOECAA|1912-12-22|155|677|53|1912|0|12|22|4|1912|53|677|Sunday|1912Q4|N|N|N|2419738|2420072|2419393|2419667|N|N|N|N|N| +2419760|AAAAAAAAADMOECAA|1912-12-23|155|677|53|1912|1|12|23|4|1912|53|677|Monday|1912Q4|N|N|N|2419738|2420072|2419394|2419668|N|N|N|N|N| +2419761|AAAAAAAABDMOECAA|1912-12-24|155|678|53|1912|2|12|24|4|1912|53|678|Tuesday|1912Q4|N|N|N|2419738|2420072|2419395|2419669|N|N|N|N|N| +2419762|AAAAAAAACDMOECAA|1912-12-25|155|678|53|1912|3|12|25|4|1912|53|678|Wednesday|1912Q4|Y|N|N|2419738|2420072|2419396|2419670|N|N|N|N|N| +2419763|AAAAAAAADDMOECAA|1912-12-26|155|678|53|1912|4|12|26|4|1912|53|678|Thursday|1912Q4|N|N|Y|2419738|2420072|2419397|2419671|N|N|N|N|N| +2419764|AAAAAAAAEDMOECAA|1912-12-27|155|678|53|1912|5|12|27|4|1912|53|678|Friday|1912Q4|N|Y|N|2419738|2420072|2419398|2419672|N|N|N|N|N| +2419765|AAAAAAAAFDMOECAA|1912-12-28|155|678|53|1912|6|12|28|4|1912|53|678|Saturday|1912Q4|N|Y|N|2419738|2420072|2419399|2419673|N|N|N|N|N| +2419766|AAAAAAAAGDMOECAA|1912-12-29|155|678|53|1912|0|12|29|4|1912|53|678|Sunday|1912Q4|N|N|N|2419738|2420072|2419400|2419674|N|N|N|N|N| +2419767|AAAAAAAAHDMOECAA|1912-12-30|155|678|53|1912|1|12|30|4|1912|53|678|Monday|1912Q4|N|N|N|2419738|2420072|2419401|2419675|N|N|N|N|N| +2419768|AAAAAAAAIDMOECAA|1912-12-31|155|679|53|1912|2|12|31|4|1912|53|679|Tuesday|1912Q4|Y|N|N|2419738|2420072|2419402|2419676|N|N|N|N|N| +2419769|AAAAAAAAJDMOECAA|1913-01-01|156|679|53|1913|3|1|1|1|1913|53|679|Wednesday|1913Q1|Y|N|Y|2419769|2419768|2419403|2419677|N|N|N|N|N| +2419770|AAAAAAAAKDMOECAA|1913-01-02|156|679|53|1913|4|1|2|1|1913|53|679|Thursday|1913Q1|N|N|Y|2419769|2419768|2419404|2419678|N|N|N|N|N| +2419771|AAAAAAAALDMOECAA|1913-01-03|156|679|53|1913|5|1|3|1|1913|53|679|Friday|1913Q1|N|Y|N|2419769|2419768|2419405|2419679|N|N|N|N|N| +2419772|AAAAAAAAMDMOECAA|1913-01-04|156|679|53|1913|6|1|4|1|1913|53|679|Saturday|1913Q1|N|Y|N|2419769|2419768|2419406|2419680|N|N|N|N|N| +2419773|AAAAAAAANDMOECAA|1913-01-05|156|679|53|1913|0|1|5|1|1913|53|679|Sunday|1913Q1|N|N|N|2419769|2419768|2419407|2419681|N|N|N|N|N| +2419774|AAAAAAAAODMOECAA|1913-01-06|156|679|53|1913|1|1|6|1|1913|53|679|Monday|1913Q1|N|N|N|2419769|2419768|2419408|2419682|N|N|N|N|N| +2419775|AAAAAAAAPDMOECAA|1913-01-07|156|680|53|1913|2|1|7|1|1913|53|680|Tuesday|1913Q1|N|N|N|2419769|2419768|2419409|2419683|N|N|N|N|N| +2419776|AAAAAAAAAEMOECAA|1913-01-08|156|680|53|1913|3|1|8|1|1913|53|680|Wednesday|1913Q1|N|N|N|2419769|2419768|2419410|2419684|N|N|N|N|N| +2419777|AAAAAAAABEMOECAA|1913-01-09|156|680|53|1913|4|1|9|1|1913|53|680|Thursday|1913Q1|N|N|N|2419769|2419768|2419411|2419685|N|N|N|N|N| +2419778|AAAAAAAACEMOECAA|1913-01-10|156|680|53|1913|5|1|10|1|1913|53|680|Friday|1913Q1|N|Y|N|2419769|2419768|2419412|2419686|N|N|N|N|N| +2419779|AAAAAAAADEMOECAA|1913-01-11|156|680|53|1913|6|1|11|1|1913|53|680|Saturday|1913Q1|N|Y|N|2419769|2419768|2419413|2419687|N|N|N|N|N| +2419780|AAAAAAAAEEMOECAA|1913-01-12|156|680|53|1913|0|1|12|1|1913|53|680|Sunday|1913Q1|N|N|N|2419769|2419768|2419414|2419688|N|N|N|N|N| +2419781|AAAAAAAAFEMOECAA|1913-01-13|156|680|53|1913|1|1|13|1|1913|53|680|Monday|1913Q1|N|N|N|2419769|2419768|2419415|2419689|N|N|N|N|N| +2419782|AAAAAAAAGEMOECAA|1913-01-14|156|681|53|1913|2|1|14|1|1913|53|681|Tuesday|1913Q1|N|N|N|2419769|2419768|2419416|2419690|N|N|N|N|N| +2419783|AAAAAAAAHEMOECAA|1913-01-15|156|681|53|1913|3|1|15|1|1913|53|681|Wednesday|1913Q1|N|N|N|2419769|2419768|2419417|2419691|N|N|N|N|N| +2419784|AAAAAAAAIEMOECAA|1913-01-16|156|681|53|1913|4|1|16|1|1913|53|681|Thursday|1913Q1|N|N|N|2419769|2419768|2419418|2419692|N|N|N|N|N| +2419785|AAAAAAAAJEMOECAA|1913-01-17|156|681|53|1913|5|1|17|1|1913|53|681|Friday|1913Q1|N|Y|N|2419769|2419768|2419419|2419693|N|N|N|N|N| +2419786|AAAAAAAAKEMOECAA|1913-01-18|156|681|53|1913|6|1|18|1|1913|53|681|Saturday|1913Q1|N|Y|N|2419769|2419768|2419420|2419694|N|N|N|N|N| +2419787|AAAAAAAALEMOECAA|1913-01-19|156|681|53|1913|0|1|19|1|1913|53|681|Sunday|1913Q1|N|N|N|2419769|2419768|2419421|2419695|N|N|N|N|N| +2419788|AAAAAAAAMEMOECAA|1913-01-20|156|681|53|1913|1|1|20|1|1913|53|681|Monday|1913Q1|N|N|N|2419769|2419768|2419422|2419696|N|N|N|N|N| +2419789|AAAAAAAANEMOECAA|1913-01-21|156|682|53|1913|2|1|21|1|1913|53|682|Tuesday|1913Q1|N|N|N|2419769|2419768|2419423|2419697|N|N|N|N|N| +2419790|AAAAAAAAOEMOECAA|1913-01-22|156|682|53|1913|3|1|22|1|1913|53|682|Wednesday|1913Q1|N|N|N|2419769|2419768|2419424|2419698|N|N|N|N|N| +2419791|AAAAAAAAPEMOECAA|1913-01-23|156|682|53|1913|4|1|23|1|1913|53|682|Thursday|1913Q1|N|N|N|2419769|2419768|2419425|2419699|N|N|N|N|N| +2419792|AAAAAAAAAFMOECAA|1913-01-24|156|682|53|1913|5|1|24|1|1913|53|682|Friday|1913Q1|N|Y|N|2419769|2419768|2419426|2419700|N|N|N|N|N| +2419793|AAAAAAAABFMOECAA|1913-01-25|156|682|53|1913|6|1|25|1|1913|53|682|Saturday|1913Q1|N|Y|N|2419769|2419768|2419427|2419701|N|N|N|N|N| +2419794|AAAAAAAACFMOECAA|1913-01-26|156|682|53|1913|0|1|26|1|1913|53|682|Sunday|1913Q1|N|N|N|2419769|2419768|2419428|2419702|N|N|N|N|N| +2419795|AAAAAAAADFMOECAA|1913-01-27|156|682|53|1913|1|1|27|1|1913|53|682|Monday|1913Q1|N|N|N|2419769|2419768|2419429|2419703|N|N|N|N|N| +2419796|AAAAAAAAEFMOECAA|1913-01-28|156|683|53|1913|2|1|28|1|1913|53|683|Tuesday|1913Q1|N|N|N|2419769|2419768|2419430|2419704|N|N|N|N|N| +2419797|AAAAAAAAFFMOECAA|1913-01-29|156|683|53|1913|3|1|29|1|1913|53|683|Wednesday|1913Q1|N|N|N|2419769|2419768|2419431|2419705|N|N|N|N|N| +2419798|AAAAAAAAGFMOECAA|1913-01-30|156|683|53|1913|4|1|30|1|1913|53|683|Thursday|1913Q1|N|N|N|2419769|2419768|2419432|2419706|N|N|N|N|N| +2419799|AAAAAAAAHFMOECAA|1913-01-31|156|683|53|1913|5|1|31|1|1913|53|683|Friday|1913Q1|N|Y|N|2419769|2419768|2419433|2419707|N|N|N|N|N| +2419800|AAAAAAAAIFMOECAA|1913-02-01|157|683|53|1913|6|2|1|1|1913|53|683|Saturday|1913Q1|N|Y|N|2419800|2419830|2419434|2419708|N|N|N|N|N| +2419801|AAAAAAAAJFMOECAA|1913-02-02|157|683|53|1913|0|2|2|1|1913|53|683|Sunday|1913Q1|N|N|N|2419800|2419830|2419435|2419709|N|N|N|N|N| +2419802|AAAAAAAAKFMOECAA|1913-02-03|157|683|53|1913|1|2|3|1|1913|53|683|Monday|1913Q1|N|N|N|2419800|2419830|2419436|2419710|N|N|N|N|N| +2419803|AAAAAAAALFMOECAA|1913-02-04|157|684|53|1913|2|2|4|1|1913|53|684|Tuesday|1913Q1|N|N|N|2419800|2419830|2419437|2419711|N|N|N|N|N| +2419804|AAAAAAAAMFMOECAA|1913-02-05|157|684|53|1913|3|2|5|1|1913|53|684|Wednesday|1913Q1|N|N|N|2419800|2419830|2419438|2419712|N|N|N|N|N| +2419805|AAAAAAAANFMOECAA|1913-02-06|157|684|53|1913|4|2|6|1|1913|53|684|Thursday|1913Q1|N|N|N|2419800|2419830|2419439|2419713|N|N|N|N|N| +2419806|AAAAAAAAOFMOECAA|1913-02-07|157|684|53|1913|5|2|7|1|1913|53|684|Friday|1913Q1|N|Y|N|2419800|2419830|2419440|2419714|N|N|N|N|N| +2419807|AAAAAAAAPFMOECAA|1913-02-08|157|684|53|1913|6|2|8|1|1913|53|684|Saturday|1913Q1|N|Y|N|2419800|2419830|2419441|2419715|N|N|N|N|N| +2419808|AAAAAAAAAGMOECAA|1913-02-09|157|684|53|1913|0|2|9|1|1913|53|684|Sunday|1913Q1|N|N|N|2419800|2419830|2419442|2419716|N|N|N|N|N| +2419809|AAAAAAAABGMOECAA|1913-02-10|157|684|53|1913|1|2|10|1|1913|53|684|Monday|1913Q1|N|N|N|2419800|2419830|2419443|2419717|N|N|N|N|N| +2419810|AAAAAAAACGMOECAA|1913-02-11|157|685|53|1913|2|2|11|1|1913|53|685|Tuesday|1913Q1|N|N|N|2419800|2419830|2419444|2419718|N|N|N|N|N| +2419811|AAAAAAAADGMOECAA|1913-02-12|157|685|53|1913|3|2|12|1|1913|53|685|Wednesday|1913Q1|N|N|N|2419800|2419830|2419445|2419719|N|N|N|N|N| +2419812|AAAAAAAAEGMOECAA|1913-02-13|157|685|53|1913|4|2|13|1|1913|53|685|Thursday|1913Q1|N|N|N|2419800|2419830|2419446|2419720|N|N|N|N|N| +2419813|AAAAAAAAFGMOECAA|1913-02-14|157|685|53|1913|5|2|14|1|1913|53|685|Friday|1913Q1|N|Y|N|2419800|2419830|2419447|2419721|N|N|N|N|N| +2419814|AAAAAAAAGGMOECAA|1913-02-15|157|685|53|1913|6|2|15|1|1913|53|685|Saturday|1913Q1|N|Y|N|2419800|2419830|2419448|2419722|N|N|N|N|N| +2419815|AAAAAAAAHGMOECAA|1913-02-16|157|685|53|1913|0|2|16|1|1913|53|685|Sunday|1913Q1|N|N|N|2419800|2419830|2419449|2419723|N|N|N|N|N| +2419816|AAAAAAAAIGMOECAA|1913-02-17|157|685|53|1913|1|2|17|1|1913|53|685|Monday|1913Q1|N|N|N|2419800|2419830|2419450|2419724|N|N|N|N|N| +2419817|AAAAAAAAJGMOECAA|1913-02-18|157|686|53|1913|2|2|18|1|1913|53|686|Tuesday|1913Q1|N|N|N|2419800|2419830|2419451|2419725|N|N|N|N|N| +2419818|AAAAAAAAKGMOECAA|1913-02-19|157|686|53|1913|3|2|19|1|1913|53|686|Wednesday|1913Q1|N|N|N|2419800|2419830|2419452|2419726|N|N|N|N|N| +2419819|AAAAAAAALGMOECAA|1913-02-20|157|686|53|1913|4|2|20|1|1913|53|686|Thursday|1913Q1|N|N|N|2419800|2419830|2419453|2419727|N|N|N|N|N| +2419820|AAAAAAAAMGMOECAA|1913-02-21|157|686|53|1913|5|2|21|1|1913|53|686|Friday|1913Q1|N|Y|N|2419800|2419830|2419454|2419728|N|N|N|N|N| +2419821|AAAAAAAANGMOECAA|1913-02-22|157|686|53|1913|6|2|22|1|1913|53|686|Saturday|1913Q1|N|Y|N|2419800|2419830|2419455|2419729|N|N|N|N|N| +2419822|AAAAAAAAOGMOECAA|1913-02-23|157|686|53|1913|0|2|23|1|1913|53|686|Sunday|1913Q1|N|N|N|2419800|2419830|2419456|2419730|N|N|N|N|N| +2419823|AAAAAAAAPGMOECAA|1913-02-24|157|686|53|1913|1|2|24|1|1913|53|686|Monday|1913Q1|N|N|N|2419800|2419830|2419457|2419731|N|N|N|N|N| +2419824|AAAAAAAAAHMOECAA|1913-02-25|157|687|53|1913|2|2|25|1|1913|53|687|Tuesday|1913Q1|N|N|N|2419800|2419830|2419458|2419732|N|N|N|N|N| +2419825|AAAAAAAABHMOECAA|1913-02-26|157|687|53|1913|3|2|26|1|1913|53|687|Wednesday|1913Q1|N|N|N|2419800|2419830|2419459|2419733|N|N|N|N|N| +2419826|AAAAAAAACHMOECAA|1913-02-27|157|687|53|1913|4|2|27|1|1913|53|687|Thursday|1913Q1|N|N|N|2419800|2419830|2419460|2419734|N|N|N|N|N| +2419827|AAAAAAAADHMOECAA|1913-02-28|157|687|53|1913|5|2|28|1|1913|53|687|Friday|1913Q1|N|Y|N|2419800|2419830|2419461|2419735|N|N|N|N|N| +2419828|AAAAAAAAEHMOECAA|1913-03-01|158|687|54|1913|6|3|1|1|1913|54|687|Saturday|1913Q1|N|Y|N|2419828|2419886|2419463|2419736|N|N|N|N|N| +2419829|AAAAAAAAFHMOECAA|1913-03-02|158|687|54|1913|0|3|2|1|1913|54|687|Sunday|1913Q1|N|N|N|2419828|2419886|2419464|2419737|N|N|N|N|N| +2419830|AAAAAAAAGHMOECAA|1913-03-03|158|687|54|1913|1|3|3|1|1913|54|687|Monday|1913Q1|N|N|N|2419828|2419886|2419465|2419738|N|N|N|N|N| +2419831|AAAAAAAAHHMOECAA|1913-03-04|158|688|54|1913|2|3|4|1|1913|54|688|Tuesday|1913Q1|N|N|N|2419828|2419886|2419466|2419739|N|N|N|N|N| +2419832|AAAAAAAAIHMOECAA|1913-03-05|158|688|54|1913|3|3|5|1|1913|54|688|Wednesday|1913Q1|N|N|N|2419828|2419886|2419467|2419740|N|N|N|N|N| +2419833|AAAAAAAAJHMOECAA|1913-03-06|158|688|54|1913|4|3|6|1|1913|54|688|Thursday|1913Q1|N|N|N|2419828|2419886|2419468|2419741|N|N|N|N|N| +2419834|AAAAAAAAKHMOECAA|1913-03-07|158|688|54|1913|5|3|7|1|1913|54|688|Friday|1913Q1|N|Y|N|2419828|2419886|2419469|2419742|N|N|N|N|N| +2419835|AAAAAAAALHMOECAA|1913-03-08|158|688|54|1913|6|3|8|1|1913|54|688|Saturday|1913Q1|N|Y|N|2419828|2419886|2419470|2419743|N|N|N|N|N| +2419836|AAAAAAAAMHMOECAA|1913-03-09|158|688|54|1913|0|3|9|1|1913|54|688|Sunday|1913Q1|N|N|N|2419828|2419886|2419471|2419744|N|N|N|N|N| +2419837|AAAAAAAANHMOECAA|1913-03-10|158|688|54|1913|1|3|10|1|1913|54|688|Monday|1913Q1|N|N|N|2419828|2419886|2419472|2419745|N|N|N|N|N| +2419838|AAAAAAAAOHMOECAA|1913-03-11|158|689|54|1913|2|3|11|1|1913|54|689|Tuesday|1913Q1|N|N|N|2419828|2419886|2419473|2419746|N|N|N|N|N| +2419839|AAAAAAAAPHMOECAA|1913-03-12|158|689|54|1913|3|3|12|1|1913|54|689|Wednesday|1913Q1|N|N|N|2419828|2419886|2419474|2419747|N|N|N|N|N| +2419840|AAAAAAAAAIMOECAA|1913-03-13|158|689|54|1913|4|3|13|1|1913|54|689|Thursday|1913Q1|N|N|N|2419828|2419886|2419475|2419748|N|N|N|N|N| +2419841|AAAAAAAABIMOECAA|1913-03-14|158|689|54|1913|5|3|14|1|1913|54|689|Friday|1913Q1|N|Y|N|2419828|2419886|2419476|2419749|N|N|N|N|N| +2419842|AAAAAAAACIMOECAA|1913-03-15|158|689|54|1913|6|3|15|1|1913|54|689|Saturday|1913Q1|N|Y|N|2419828|2419886|2419477|2419750|N|N|N|N|N| +2419843|AAAAAAAADIMOECAA|1913-03-16|158|689|54|1913|0|3|16|1|1913|54|689|Sunday|1913Q1|N|N|N|2419828|2419886|2419478|2419751|N|N|N|N|N| +2419844|AAAAAAAAEIMOECAA|1913-03-17|158|689|54|1913|1|3|17|1|1913|54|689|Monday|1913Q1|N|N|N|2419828|2419886|2419479|2419752|N|N|N|N|N| +2419845|AAAAAAAAFIMOECAA|1913-03-18|158|690|54|1913|2|3|18|1|1913|54|690|Tuesday|1913Q1|N|N|N|2419828|2419886|2419480|2419753|N|N|N|N|N| +2419846|AAAAAAAAGIMOECAA|1913-03-19|158|690|54|1913|3|3|19|1|1913|54|690|Wednesday|1913Q1|N|N|N|2419828|2419886|2419481|2419754|N|N|N|N|N| +2419847|AAAAAAAAHIMOECAA|1913-03-20|158|690|54|1913|4|3|20|1|1913|54|690|Thursday|1913Q1|N|N|N|2419828|2419886|2419482|2419755|N|N|N|N|N| +2419848|AAAAAAAAIIMOECAA|1913-03-21|158|690|54|1913|5|3|21|1|1913|54|690|Friday|1913Q1|N|Y|N|2419828|2419886|2419483|2419756|N|N|N|N|N| +2419849|AAAAAAAAJIMOECAA|1913-03-22|158|690|54|1913|6|3|22|1|1913|54|690|Saturday|1913Q1|N|Y|N|2419828|2419886|2419484|2419757|N|N|N|N|N| +2419850|AAAAAAAAKIMOECAA|1913-03-23|158|690|54|1913|0|3|23|1|1913|54|690|Sunday|1913Q1|N|N|N|2419828|2419886|2419485|2419758|N|N|N|N|N| +2419851|AAAAAAAALIMOECAA|1913-03-24|158|690|54|1913|1|3|24|1|1913|54|690|Monday|1913Q1|N|N|N|2419828|2419886|2419486|2419759|N|N|N|N|N| +2419852|AAAAAAAAMIMOECAA|1913-03-25|158|691|54|1913|2|3|25|1|1913|54|691|Tuesday|1913Q1|N|N|N|2419828|2419886|2419487|2419760|N|N|N|N|N| +2419853|AAAAAAAANIMOECAA|1913-03-26|158|691|54|1913|3|3|26|1|1913|54|691|Wednesday|1913Q1|N|N|N|2419828|2419886|2419488|2419761|N|N|N|N|N| +2419854|AAAAAAAAOIMOECAA|1913-03-27|158|691|54|1913|4|3|27|1|1913|54|691|Thursday|1913Q1|N|N|N|2419828|2419886|2419489|2419762|N|N|N|N|N| +2419855|AAAAAAAAPIMOECAA|1913-03-28|158|691|54|1913|5|3|28|1|1913|54|691|Friday|1913Q1|N|Y|N|2419828|2419886|2419490|2419763|N|N|N|N|N| +2419856|AAAAAAAAAJMOECAA|1913-03-29|158|691|54|1913|6|3|29|1|1913|54|691|Saturday|1913Q1|N|Y|N|2419828|2419886|2419491|2419764|N|N|N|N|N| +2419857|AAAAAAAABJMOECAA|1913-03-30|158|691|54|1913|0|3|30|1|1913|54|691|Sunday|1913Q1|N|N|N|2419828|2419886|2419492|2419765|N|N|N|N|N| +2419858|AAAAAAAACJMOECAA|1913-03-31|158|691|54|1913|1|3|31|1|1913|54|691|Monday|1913Q1|N|N|N|2419828|2419886|2419493|2419766|N|N|N|N|N| +2419859|AAAAAAAADJMOECAA|1913-04-01|159|692|54|1913|2|4|1|1|1913|54|692|Tuesday|1913Q1|N|N|N|2419859|2419948|2419494|2419769|N|N|N|N|N| +2419860|AAAAAAAAEJMOECAA|1913-04-02|159|692|54|1913|3|4|2|2|1913|54|692|Wednesday|1913Q2|N|N|N|2419859|2419948|2419495|2419770|N|N|N|N|N| +2419861|AAAAAAAAFJMOECAA|1913-04-03|159|692|54|1913|4|4|3|2|1913|54|692|Thursday|1913Q2|N|N|N|2419859|2419948|2419496|2419771|N|N|N|N|N| +2419862|AAAAAAAAGJMOECAA|1913-04-04|159|692|54|1913|5|4|4|2|1913|54|692|Friday|1913Q2|N|Y|N|2419859|2419948|2419497|2419772|N|N|N|N|N| +2419863|AAAAAAAAHJMOECAA|1913-04-05|159|692|54|1913|6|4|5|2|1913|54|692|Saturday|1913Q2|N|Y|N|2419859|2419948|2419498|2419773|N|N|N|N|N| +2419864|AAAAAAAAIJMOECAA|1913-04-06|159|692|54|1913|0|4|6|2|1913|54|692|Sunday|1913Q2|N|N|N|2419859|2419948|2419499|2419774|N|N|N|N|N| +2419865|AAAAAAAAJJMOECAA|1913-04-07|159|692|54|1913|1|4|7|2|1913|54|692|Monday|1913Q2|N|N|N|2419859|2419948|2419500|2419775|N|N|N|N|N| +2419866|AAAAAAAAKJMOECAA|1913-04-08|159|693|54|1913|2|4|8|2|1913|54|693|Tuesday|1913Q2|N|N|N|2419859|2419948|2419501|2419776|N|N|N|N|N| +2419867|AAAAAAAALJMOECAA|1913-04-09|159|693|54|1913|3|4|9|2|1913|54|693|Wednesday|1913Q2|N|N|N|2419859|2419948|2419502|2419777|N|N|N|N|N| +2419868|AAAAAAAAMJMOECAA|1913-04-10|159|693|54|1913|4|4|10|2|1913|54|693|Thursday|1913Q2|N|N|N|2419859|2419948|2419503|2419778|N|N|N|N|N| +2419869|AAAAAAAANJMOECAA|1913-04-11|159|693|54|1913|5|4|11|2|1913|54|693|Friday|1913Q2|N|Y|N|2419859|2419948|2419504|2419779|N|N|N|N|N| +2419870|AAAAAAAAOJMOECAA|1913-04-12|159|693|54|1913|6|4|12|2|1913|54|693|Saturday|1913Q2|N|Y|N|2419859|2419948|2419505|2419780|N|N|N|N|N| +2419871|AAAAAAAAPJMOECAA|1913-04-13|159|693|54|1913|0|4|13|2|1913|54|693|Sunday|1913Q2|N|N|N|2419859|2419948|2419506|2419781|N|N|N|N|N| +2419872|AAAAAAAAAKMOECAA|1913-04-14|159|693|54|1913|1|4|14|2|1913|54|693|Monday|1913Q2|N|N|N|2419859|2419948|2419507|2419782|N|N|N|N|N| +2419873|AAAAAAAABKMOECAA|1913-04-15|159|694|54|1913|2|4|15|2|1913|54|694|Tuesday|1913Q2|N|N|N|2419859|2419948|2419508|2419783|N|N|N|N|N| +2419874|AAAAAAAACKMOECAA|1913-04-16|159|694|54|1913|3|4|16|2|1913|54|694|Wednesday|1913Q2|N|N|N|2419859|2419948|2419509|2419784|N|N|N|N|N| +2419875|AAAAAAAADKMOECAA|1913-04-17|159|694|54|1913|4|4|17|2|1913|54|694|Thursday|1913Q2|N|N|N|2419859|2419948|2419510|2419785|N|N|N|N|N| +2419876|AAAAAAAAEKMOECAA|1913-04-18|159|694|54|1913|5|4|18|2|1913|54|694|Friday|1913Q2|N|Y|N|2419859|2419948|2419511|2419786|N|N|N|N|N| +2419877|AAAAAAAAFKMOECAA|1913-04-19|159|694|54|1913|6|4|19|2|1913|54|694|Saturday|1913Q2|N|Y|N|2419859|2419948|2419512|2419787|N|N|N|N|N| +2419878|AAAAAAAAGKMOECAA|1913-04-20|159|694|54|1913|0|4|20|2|1913|54|694|Sunday|1913Q2|N|N|N|2419859|2419948|2419513|2419788|N|N|N|N|N| +2419879|AAAAAAAAHKMOECAA|1913-04-21|159|694|54|1913|1|4|21|2|1913|54|694|Monday|1913Q2|N|N|N|2419859|2419948|2419514|2419789|N|N|N|N|N| +2419880|AAAAAAAAIKMOECAA|1913-04-22|159|695|54|1913|2|4|22|2|1913|54|695|Tuesday|1913Q2|N|N|N|2419859|2419948|2419515|2419790|N|N|N|N|N| +2419881|AAAAAAAAJKMOECAA|1913-04-23|159|695|54|1913|3|4|23|2|1913|54|695|Wednesday|1913Q2|N|N|N|2419859|2419948|2419516|2419791|N|N|N|N|N| +2419882|AAAAAAAAKKMOECAA|1913-04-24|159|695|54|1913|4|4|24|2|1913|54|695|Thursday|1913Q2|N|N|N|2419859|2419948|2419517|2419792|N|N|N|N|N| +2419883|AAAAAAAALKMOECAA|1913-04-25|159|695|54|1913|5|4|25|2|1913|54|695|Friday|1913Q2|N|Y|N|2419859|2419948|2419518|2419793|N|N|N|N|N| +2419884|AAAAAAAAMKMOECAA|1913-04-26|159|695|54|1913|6|4|26|2|1913|54|695|Saturday|1913Q2|N|Y|N|2419859|2419948|2419519|2419794|N|N|N|N|N| +2419885|AAAAAAAANKMOECAA|1913-04-27|159|695|54|1913|0|4|27|2|1913|54|695|Sunday|1913Q2|N|N|N|2419859|2419948|2419520|2419795|N|N|N|N|N| +2419886|AAAAAAAAOKMOECAA|1913-04-28|159|695|54|1913|1|4|28|2|1913|54|695|Monday|1913Q2|N|N|N|2419859|2419948|2419521|2419796|N|N|N|N|N| +2419887|AAAAAAAAPKMOECAA|1913-04-29|159|696|54|1913|2|4|29|2|1913|54|696|Tuesday|1913Q2|N|N|N|2419859|2419948|2419522|2419797|N|N|N|N|N| +2419888|AAAAAAAAALMOECAA|1913-04-30|159|696|54|1913|3|4|30|2|1913|54|696|Wednesday|1913Q2|N|N|N|2419859|2419948|2419523|2419798|N|N|N|N|N| +2419889|AAAAAAAABLMOECAA|1913-05-01|160|696|54|1913|4|5|1|2|1913|54|696|Thursday|1913Q2|N|N|N|2419889|2420008|2419524|2419799|N|N|N|N|N| +2419890|AAAAAAAACLMOECAA|1913-05-02|160|696|54|1913|5|5|2|2|1913|54|696|Friday|1913Q2|N|Y|N|2419889|2420008|2419525|2419800|N|N|N|N|N| +2419891|AAAAAAAADLMOECAA|1913-05-03|160|696|54|1913|6|5|3|2|1913|54|696|Saturday|1913Q2|N|Y|N|2419889|2420008|2419526|2419801|N|N|N|N|N| +2419892|AAAAAAAAELMOECAA|1913-05-04|160|696|54|1913|0|5|4|2|1913|54|696|Sunday|1913Q2|N|N|N|2419889|2420008|2419527|2419802|N|N|N|N|N| +2419893|AAAAAAAAFLMOECAA|1913-05-05|160|696|54|1913|1|5|5|2|1913|54|696|Monday|1913Q2|N|N|N|2419889|2420008|2419528|2419803|N|N|N|N|N| +2419894|AAAAAAAAGLMOECAA|1913-05-06|160|697|54|1913|2|5|6|2|1913|54|697|Tuesday|1913Q2|N|N|N|2419889|2420008|2419529|2419804|N|N|N|N|N| +2419895|AAAAAAAAHLMOECAA|1913-05-07|160|697|54|1913|3|5|7|2|1913|54|697|Wednesday|1913Q2|N|N|N|2419889|2420008|2419530|2419805|N|N|N|N|N| +2419896|AAAAAAAAILMOECAA|1913-05-08|160|697|54|1913|4|5|8|2|1913|54|697|Thursday|1913Q2|N|N|N|2419889|2420008|2419531|2419806|N|N|N|N|N| +2419897|AAAAAAAAJLMOECAA|1913-05-09|160|697|54|1913|5|5|9|2|1913|54|697|Friday|1913Q2|N|Y|N|2419889|2420008|2419532|2419807|N|N|N|N|N| +2419898|AAAAAAAAKLMOECAA|1913-05-10|160|697|54|1913|6|5|10|2|1913|54|697|Saturday|1913Q2|N|Y|N|2419889|2420008|2419533|2419808|N|N|N|N|N| +2419899|AAAAAAAALLMOECAA|1913-05-11|160|697|54|1913|0|5|11|2|1913|54|697|Sunday|1913Q2|N|N|N|2419889|2420008|2419534|2419809|N|N|N|N|N| +2419900|AAAAAAAAMLMOECAA|1913-05-12|160|697|54|1913|1|5|12|2|1913|54|697|Monday|1913Q2|N|N|N|2419889|2420008|2419535|2419810|N|N|N|N|N| +2419901|AAAAAAAANLMOECAA|1913-05-13|160|698|54|1913|2|5|13|2|1913|54|698|Tuesday|1913Q2|N|N|N|2419889|2420008|2419536|2419811|N|N|N|N|N| +2419902|AAAAAAAAOLMOECAA|1913-05-14|160|698|54|1913|3|5|14|2|1913|54|698|Wednesday|1913Q2|N|N|N|2419889|2420008|2419537|2419812|N|N|N|N|N| +2419903|AAAAAAAAPLMOECAA|1913-05-15|160|698|54|1913|4|5|15|2|1913|54|698|Thursday|1913Q2|N|N|N|2419889|2420008|2419538|2419813|N|N|N|N|N| +2419904|AAAAAAAAAMMOECAA|1913-05-16|160|698|54|1913|5|5|16|2|1913|54|698|Friday|1913Q2|N|Y|N|2419889|2420008|2419539|2419814|N|N|N|N|N| +2419905|AAAAAAAABMMOECAA|1913-05-17|160|698|54|1913|6|5|17|2|1913|54|698|Saturday|1913Q2|N|Y|N|2419889|2420008|2419540|2419815|N|N|N|N|N| +2419906|AAAAAAAACMMOECAA|1913-05-18|160|698|54|1913|0|5|18|2|1913|54|698|Sunday|1913Q2|N|N|N|2419889|2420008|2419541|2419816|N|N|N|N|N| +2419907|AAAAAAAADMMOECAA|1913-05-19|160|698|54|1913|1|5|19|2|1913|54|698|Monday|1913Q2|N|N|N|2419889|2420008|2419542|2419817|N|N|N|N|N| +2419908|AAAAAAAAEMMOECAA|1913-05-20|160|699|54|1913|2|5|20|2|1913|54|699|Tuesday|1913Q2|N|N|N|2419889|2420008|2419543|2419818|N|N|N|N|N| +2419909|AAAAAAAAFMMOECAA|1913-05-21|160|699|54|1913|3|5|21|2|1913|54|699|Wednesday|1913Q2|N|N|N|2419889|2420008|2419544|2419819|N|N|N|N|N| +2419910|AAAAAAAAGMMOECAA|1913-05-22|160|699|54|1913|4|5|22|2|1913|54|699|Thursday|1913Q2|N|N|N|2419889|2420008|2419545|2419820|N|N|N|N|N| +2419911|AAAAAAAAHMMOECAA|1913-05-23|160|699|54|1913|5|5|23|2|1913|54|699|Friday|1913Q2|N|Y|N|2419889|2420008|2419546|2419821|N|N|N|N|N| +2419912|AAAAAAAAIMMOECAA|1913-05-24|160|699|54|1913|6|5|24|2|1913|54|699|Saturday|1913Q2|N|Y|N|2419889|2420008|2419547|2419822|N|N|N|N|N| +2419913|AAAAAAAAJMMOECAA|1913-05-25|160|699|54|1913|0|5|25|2|1913|54|699|Sunday|1913Q2|N|N|N|2419889|2420008|2419548|2419823|N|N|N|N|N| +2419914|AAAAAAAAKMMOECAA|1913-05-26|160|699|54|1913|1|5|26|2|1913|54|699|Monday|1913Q2|N|N|N|2419889|2420008|2419549|2419824|N|N|N|N|N| +2419915|AAAAAAAALMMOECAA|1913-05-27|160|700|54|1913|2|5|27|2|1913|54|700|Tuesday|1913Q2|N|N|N|2419889|2420008|2419550|2419825|N|N|N|N|N| +2419916|AAAAAAAAMMMOECAA|1913-05-28|160|700|54|1913|3|5|28|2|1913|54|700|Wednesday|1913Q2|N|N|N|2419889|2420008|2419551|2419826|N|N|N|N|N| +2419917|AAAAAAAANMMOECAA|1913-05-29|160|700|54|1913|4|5|29|2|1913|54|700|Thursday|1913Q2|N|N|N|2419889|2420008|2419552|2419827|N|N|N|N|N| +2419918|AAAAAAAAOMMOECAA|1913-05-30|160|700|54|1913|5|5|30|2|1913|54|700|Friday|1913Q2|N|Y|N|2419889|2420008|2419553|2419828|N|N|N|N|N| +2419919|AAAAAAAAPMMOECAA|1913-05-31|160|700|54|1913|6|5|31|2|1913|54|700|Saturday|1913Q2|N|Y|N|2419889|2420008|2419554|2419829|N|N|N|N|N| +2419920|AAAAAAAAANMOECAA|1913-06-01|161|700|55|1913|0|6|1|2|1913|55|700|Sunday|1913Q2|N|N|N|2419920|2420070|2419555|2419830|N|N|N|N|N| +2419921|AAAAAAAABNMOECAA|1913-06-02|161|700|55|1913|1|6|2|2|1913|55|700|Monday|1913Q2|N|N|N|2419920|2420070|2419556|2419831|N|N|N|N|N| +2419922|AAAAAAAACNMOECAA|1913-06-03|161|701|55|1913|2|6|3|2|1913|55|701|Tuesday|1913Q2|N|N|N|2419920|2420070|2419557|2419832|N|N|N|N|N| +2419923|AAAAAAAADNMOECAA|1913-06-04|161|701|55|1913|3|6|4|2|1913|55|701|Wednesday|1913Q2|N|N|N|2419920|2420070|2419558|2419833|N|N|N|N|N| +2419924|AAAAAAAAENMOECAA|1913-06-05|161|701|55|1913|4|6|5|2|1913|55|701|Thursday|1913Q2|N|N|N|2419920|2420070|2419559|2419834|N|N|N|N|N| +2419925|AAAAAAAAFNMOECAA|1913-06-06|161|701|55|1913|5|6|6|2|1913|55|701|Friday|1913Q2|N|Y|N|2419920|2420070|2419560|2419835|N|N|N|N|N| +2419926|AAAAAAAAGNMOECAA|1913-06-07|161|701|55|1913|6|6|7|2|1913|55|701|Saturday|1913Q2|N|Y|N|2419920|2420070|2419561|2419836|N|N|N|N|N| +2419927|AAAAAAAAHNMOECAA|1913-06-08|161|701|55|1913|0|6|8|2|1913|55|701|Sunday|1913Q2|N|N|N|2419920|2420070|2419562|2419837|N|N|N|N|N| +2419928|AAAAAAAAINMOECAA|1913-06-09|161|701|55|1913|1|6|9|2|1913|55|701|Monday|1913Q2|N|N|N|2419920|2420070|2419563|2419838|N|N|N|N|N| +2419929|AAAAAAAAJNMOECAA|1913-06-10|161|702|55|1913|2|6|10|2|1913|55|702|Tuesday|1913Q2|N|N|N|2419920|2420070|2419564|2419839|N|N|N|N|N| +2419930|AAAAAAAAKNMOECAA|1913-06-11|161|702|55|1913|3|6|11|2|1913|55|702|Wednesday|1913Q2|N|N|N|2419920|2420070|2419565|2419840|N|N|N|N|N| +2419931|AAAAAAAALNMOECAA|1913-06-12|161|702|55|1913|4|6|12|2|1913|55|702|Thursday|1913Q2|N|N|N|2419920|2420070|2419566|2419841|N|N|N|N|N| +2419932|AAAAAAAAMNMOECAA|1913-06-13|161|702|55|1913|5|6|13|2|1913|55|702|Friday|1913Q2|N|Y|N|2419920|2420070|2419567|2419842|N|N|N|N|N| +2419933|AAAAAAAANNMOECAA|1913-06-14|161|702|55|1913|6|6|14|2|1913|55|702|Saturday|1913Q2|N|Y|N|2419920|2420070|2419568|2419843|N|N|N|N|N| +2419934|AAAAAAAAONMOECAA|1913-06-15|161|702|55|1913|0|6|15|2|1913|55|702|Sunday|1913Q2|N|N|N|2419920|2420070|2419569|2419844|N|N|N|N|N| +2419935|AAAAAAAAPNMOECAA|1913-06-16|161|702|55|1913|1|6|16|2|1913|55|702|Monday|1913Q2|N|N|N|2419920|2420070|2419570|2419845|N|N|N|N|N| +2419936|AAAAAAAAAOMOECAA|1913-06-17|161|703|55|1913|2|6|17|2|1913|55|703|Tuesday|1913Q2|N|N|N|2419920|2420070|2419571|2419846|N|N|N|N|N| +2419937|AAAAAAAABOMOECAA|1913-06-18|161|703|55|1913|3|6|18|2|1913|55|703|Wednesday|1913Q2|N|N|N|2419920|2420070|2419572|2419847|N|N|N|N|N| +2419938|AAAAAAAACOMOECAA|1913-06-19|161|703|55|1913|4|6|19|2|1913|55|703|Thursday|1913Q2|N|N|N|2419920|2420070|2419573|2419848|N|N|N|N|N| +2419939|AAAAAAAADOMOECAA|1913-06-20|161|703|55|1913|5|6|20|2|1913|55|703|Friday|1913Q2|N|Y|N|2419920|2420070|2419574|2419849|N|N|N|N|N| +2419940|AAAAAAAAEOMOECAA|1913-06-21|161|703|55|1913|6|6|21|2|1913|55|703|Saturday|1913Q2|N|Y|N|2419920|2420070|2419575|2419850|N|N|N|N|N| +2419941|AAAAAAAAFOMOECAA|1913-06-22|161|703|55|1913|0|6|22|2|1913|55|703|Sunday|1913Q2|N|N|N|2419920|2420070|2419576|2419851|N|N|N|N|N| +2419942|AAAAAAAAGOMOECAA|1913-06-23|161|703|55|1913|1|6|23|2|1913|55|703|Monday|1913Q2|N|N|N|2419920|2420070|2419577|2419852|N|N|N|N|N| +2419943|AAAAAAAAHOMOECAA|1913-06-24|161|704|55|1913|2|6|24|2|1913|55|704|Tuesday|1913Q2|N|N|N|2419920|2420070|2419578|2419853|N|N|N|N|N| +2419944|AAAAAAAAIOMOECAA|1913-06-25|161|704|55|1913|3|6|25|2|1913|55|704|Wednesday|1913Q2|N|N|N|2419920|2420070|2419579|2419854|N|N|N|N|N| +2419945|AAAAAAAAJOMOECAA|1913-06-26|161|704|55|1913|4|6|26|2|1913|55|704|Thursday|1913Q2|N|N|N|2419920|2420070|2419580|2419855|N|N|N|N|N| +2419946|AAAAAAAAKOMOECAA|1913-06-27|161|704|55|1913|5|6|27|2|1913|55|704|Friday|1913Q2|N|Y|N|2419920|2420070|2419581|2419856|N|N|N|N|N| +2419947|AAAAAAAALOMOECAA|1913-06-28|161|704|55|1913|6|6|28|2|1913|55|704|Saturday|1913Q2|N|Y|N|2419920|2420070|2419582|2419857|N|N|N|N|N| +2419948|AAAAAAAAMOMOECAA|1913-06-29|161|704|55|1913|0|6|29|2|1913|55|704|Sunday|1913Q2|N|N|N|2419920|2420070|2419583|2419858|N|N|N|N|N| +2419949|AAAAAAAANOMOECAA|1913-06-30|161|704|55|1913|1|6|30|2|1913|55|704|Monday|1913Q2|N|N|N|2419920|2420070|2419584|2419859|N|N|N|N|N| +2419950|AAAAAAAAOOMOECAA|1913-07-01|162|705|55|1913|2|7|1|2|1913|55|705|Tuesday|1913Q2|N|N|N|2419950|2420130|2419585|2419859|N|N|N|N|N| +2419951|AAAAAAAAPOMOECAA|1913-07-02|162|705|55|1913|3|7|2|3|1913|55|705|Wednesday|1913Q3|N|N|N|2419950|2420130|2419586|2419860|N|N|N|N|N| +2419952|AAAAAAAAAPMOECAA|1913-07-03|162|705|55|1913|4|7|3|3|1913|55|705|Thursday|1913Q3|N|N|N|2419950|2420130|2419587|2419861|N|N|N|N|N| +2419953|AAAAAAAABPMOECAA|1913-07-04|162|705|55|1913|5|7|4|3|1913|55|705|Friday|1913Q3|N|Y|N|2419950|2420130|2419588|2419862|N|N|N|N|N| +2419954|AAAAAAAACPMOECAA|1913-07-05|162|705|55|1913|6|7|5|3|1913|55|705|Saturday|1913Q3|Y|Y|N|2419950|2420130|2419589|2419863|N|N|N|N|N| +2419955|AAAAAAAADPMOECAA|1913-07-06|162|705|55|1913|0|7|6|3|1913|55|705|Sunday|1913Q3|N|N|Y|2419950|2420130|2419590|2419864|N|N|N|N|N| +2419956|AAAAAAAAEPMOECAA|1913-07-07|162|705|55|1913|1|7|7|3|1913|55|705|Monday|1913Q3|N|N|N|2419950|2420130|2419591|2419865|N|N|N|N|N| +2419957|AAAAAAAAFPMOECAA|1913-07-08|162|706|55|1913|2|7|8|3|1913|55|706|Tuesday|1913Q3|N|N|N|2419950|2420130|2419592|2419866|N|N|N|N|N| +2419958|AAAAAAAAGPMOECAA|1913-07-09|162|706|55|1913|3|7|9|3|1913|55|706|Wednesday|1913Q3|N|N|N|2419950|2420130|2419593|2419867|N|N|N|N|N| +2419959|AAAAAAAAHPMOECAA|1913-07-10|162|706|55|1913|4|7|10|3|1913|55|706|Thursday|1913Q3|N|N|N|2419950|2420130|2419594|2419868|N|N|N|N|N| +2419960|AAAAAAAAIPMOECAA|1913-07-11|162|706|55|1913|5|7|11|3|1913|55|706|Friday|1913Q3|N|Y|N|2419950|2420130|2419595|2419869|N|N|N|N|N| +2419961|AAAAAAAAJPMOECAA|1913-07-12|162|706|55|1913|6|7|12|3|1913|55|706|Saturday|1913Q3|N|Y|N|2419950|2420130|2419596|2419870|N|N|N|N|N| +2419962|AAAAAAAAKPMOECAA|1913-07-13|162|706|55|1913|0|7|13|3|1913|55|706|Sunday|1913Q3|N|N|N|2419950|2420130|2419597|2419871|N|N|N|N|N| +2419963|AAAAAAAALPMOECAA|1913-07-14|162|706|55|1913|1|7|14|3|1913|55|706|Monday|1913Q3|N|N|N|2419950|2420130|2419598|2419872|N|N|N|N|N| +2419964|AAAAAAAAMPMOECAA|1913-07-15|162|707|55|1913|2|7|15|3|1913|55|707|Tuesday|1913Q3|N|N|N|2419950|2420130|2419599|2419873|N|N|N|N|N| +2419965|AAAAAAAANPMOECAA|1913-07-16|162|707|55|1913|3|7|16|3|1913|55|707|Wednesday|1913Q3|N|N|N|2419950|2420130|2419600|2419874|N|N|N|N|N| +2419966|AAAAAAAAOPMOECAA|1913-07-17|162|707|55|1913|4|7|17|3|1913|55|707|Thursday|1913Q3|N|N|N|2419950|2420130|2419601|2419875|N|N|N|N|N| +2419967|AAAAAAAAPPMOECAA|1913-07-18|162|707|55|1913|5|7|18|3|1913|55|707|Friday|1913Q3|N|Y|N|2419950|2420130|2419602|2419876|N|N|N|N|N| +2419968|AAAAAAAAAANOECAA|1913-07-19|162|707|55|1913|6|7|19|3|1913|55|707|Saturday|1913Q3|N|Y|N|2419950|2420130|2419603|2419877|N|N|N|N|N| +2419969|AAAAAAAABANOECAA|1913-07-20|162|707|55|1913|0|7|20|3|1913|55|707|Sunday|1913Q3|N|N|N|2419950|2420130|2419604|2419878|N|N|N|N|N| +2419970|AAAAAAAACANOECAA|1913-07-21|162|707|55|1913|1|7|21|3|1913|55|707|Monday|1913Q3|N|N|N|2419950|2420130|2419605|2419879|N|N|N|N|N| +2419971|AAAAAAAADANOECAA|1913-07-22|162|708|55|1913|2|7|22|3|1913|55|708|Tuesday|1913Q3|N|N|N|2419950|2420130|2419606|2419880|N|N|N|N|N| +2419972|AAAAAAAAEANOECAA|1913-07-23|162|708|55|1913|3|7|23|3|1913|55|708|Wednesday|1913Q3|N|N|N|2419950|2420130|2419607|2419881|N|N|N|N|N| +2419973|AAAAAAAAFANOECAA|1913-07-24|162|708|55|1913|4|7|24|3|1913|55|708|Thursday|1913Q3|N|N|N|2419950|2420130|2419608|2419882|N|N|N|N|N| +2419974|AAAAAAAAGANOECAA|1913-07-25|162|708|55|1913|5|7|25|3|1913|55|708|Friday|1913Q3|N|Y|N|2419950|2420130|2419609|2419883|N|N|N|N|N| +2419975|AAAAAAAAHANOECAA|1913-07-26|162|708|55|1913|6|7|26|3|1913|55|708|Saturday|1913Q3|N|Y|N|2419950|2420130|2419610|2419884|N|N|N|N|N| +2419976|AAAAAAAAIANOECAA|1913-07-27|162|708|55|1913|0|7|27|3|1913|55|708|Sunday|1913Q3|N|N|N|2419950|2420130|2419611|2419885|N|N|N|N|N| +2419977|AAAAAAAAJANOECAA|1913-07-28|162|708|55|1913|1|7|28|3|1913|55|708|Monday|1913Q3|N|N|N|2419950|2420130|2419612|2419886|N|N|N|N|N| +2419978|AAAAAAAAKANOECAA|1913-07-29|162|709|55|1913|2|7|29|3|1913|55|709|Tuesday|1913Q3|N|N|N|2419950|2420130|2419613|2419887|N|N|N|N|N| +2419979|AAAAAAAALANOECAA|1913-07-30|162|709|55|1913|3|7|30|3|1913|55|709|Wednesday|1913Q3|N|N|N|2419950|2420130|2419614|2419888|N|N|N|N|N| +2419980|AAAAAAAAMANOECAA|1913-07-31|162|709|55|1913|4|7|31|3|1913|55|709|Thursday|1913Q3|N|N|N|2419950|2420130|2419615|2419889|N|N|N|N|N| +2419981|AAAAAAAANANOECAA|1913-08-01|163|709|55|1913|5|8|1|3|1913|55|709|Friday|1913Q3|N|Y|N|2419981|2420192|2419616|2419890|N|N|N|N|N| +2419982|AAAAAAAAOANOECAA|1913-08-02|163|709|55|1913|6|8|2|3|1913|55|709|Saturday|1913Q3|N|Y|N|2419981|2420192|2419617|2419891|N|N|N|N|N| +2419983|AAAAAAAAPANOECAA|1913-08-03|163|709|55|1913|0|8|3|3|1913|55|709|Sunday|1913Q3|N|N|N|2419981|2420192|2419618|2419892|N|N|N|N|N| +2419984|AAAAAAAAABNOECAA|1913-08-04|163|709|55|1913|1|8|4|3|1913|55|709|Monday|1913Q3|N|N|N|2419981|2420192|2419619|2419893|N|N|N|N|N| +2419985|AAAAAAAABBNOECAA|1913-08-05|163|710|55|1913|2|8|5|3|1913|55|710|Tuesday|1913Q3|N|N|N|2419981|2420192|2419620|2419894|N|N|N|N|N| +2419986|AAAAAAAACBNOECAA|1913-08-06|163|710|55|1913|3|8|6|3|1913|55|710|Wednesday|1913Q3|N|N|N|2419981|2420192|2419621|2419895|N|N|N|N|N| +2419987|AAAAAAAADBNOECAA|1913-08-07|163|710|55|1913|4|8|7|3|1913|55|710|Thursday|1913Q3|N|N|N|2419981|2420192|2419622|2419896|N|N|N|N|N| +2419988|AAAAAAAAEBNOECAA|1913-08-08|163|710|55|1913|5|8|8|3|1913|55|710|Friday|1913Q3|N|Y|N|2419981|2420192|2419623|2419897|N|N|N|N|N| +2419989|AAAAAAAAFBNOECAA|1913-08-09|163|710|55|1913|6|8|9|3|1913|55|710|Saturday|1913Q3|N|Y|N|2419981|2420192|2419624|2419898|N|N|N|N|N| +2419990|AAAAAAAAGBNOECAA|1913-08-10|163|710|55|1913|0|8|10|3|1913|55|710|Sunday|1913Q3|N|N|N|2419981|2420192|2419625|2419899|N|N|N|N|N| +2419991|AAAAAAAAHBNOECAA|1913-08-11|163|710|55|1913|1|8|11|3|1913|55|710|Monday|1913Q3|N|N|N|2419981|2420192|2419626|2419900|N|N|N|N|N| +2419992|AAAAAAAAIBNOECAA|1913-08-12|163|711|55|1913|2|8|12|3|1913|55|711|Tuesday|1913Q3|N|N|N|2419981|2420192|2419627|2419901|N|N|N|N|N| +2419993|AAAAAAAAJBNOECAA|1913-08-13|163|711|55|1913|3|8|13|3|1913|55|711|Wednesday|1913Q3|N|N|N|2419981|2420192|2419628|2419902|N|N|N|N|N| +2419994|AAAAAAAAKBNOECAA|1913-08-14|163|711|55|1913|4|8|14|3|1913|55|711|Thursday|1913Q3|N|N|N|2419981|2420192|2419629|2419903|N|N|N|N|N| +2419995|AAAAAAAALBNOECAA|1913-08-15|163|711|55|1913|5|8|15|3|1913|55|711|Friday|1913Q3|N|Y|N|2419981|2420192|2419630|2419904|N|N|N|N|N| +2419996|AAAAAAAAMBNOECAA|1913-08-16|163|711|55|1913|6|8|16|3|1913|55|711|Saturday|1913Q3|N|Y|N|2419981|2420192|2419631|2419905|N|N|N|N|N| +2419997|AAAAAAAANBNOECAA|1913-08-17|163|711|55|1913|0|8|17|3|1913|55|711|Sunday|1913Q3|N|N|N|2419981|2420192|2419632|2419906|N|N|N|N|N| +2419998|AAAAAAAAOBNOECAA|1913-08-18|163|711|55|1913|1|8|18|3|1913|55|711|Monday|1913Q3|N|N|N|2419981|2420192|2419633|2419907|N|N|N|N|N| +2419999|AAAAAAAAPBNOECAA|1913-08-19|163|712|55|1913|2|8|19|3|1913|55|712|Tuesday|1913Q3|N|N|N|2419981|2420192|2419634|2419908|N|N|N|N|N| +2420000|AAAAAAAAACNOECAA|1913-08-20|163|712|55|1913|3|8|20|3|1913|55|712|Wednesday|1913Q3|N|N|N|2419981|2420192|2419635|2419909|N|N|N|N|N| +2420001|AAAAAAAABCNOECAA|1913-08-21|163|712|55|1913|4|8|21|3|1913|55|712|Thursday|1913Q3|N|N|N|2419981|2420192|2419636|2419910|N|N|N|N|N| +2420002|AAAAAAAACCNOECAA|1913-08-22|163|712|55|1913|5|8|22|3|1913|55|712|Friday|1913Q3|N|Y|N|2419981|2420192|2419637|2419911|N|N|N|N|N| +2420003|AAAAAAAADCNOECAA|1913-08-23|163|712|55|1913|6|8|23|3|1913|55|712|Saturday|1913Q3|N|Y|N|2419981|2420192|2419638|2419912|N|N|N|N|N| +2420004|AAAAAAAAECNOECAA|1913-08-24|163|712|55|1913|0|8|24|3|1913|55|712|Sunday|1913Q3|N|N|N|2419981|2420192|2419639|2419913|N|N|N|N|N| +2420005|AAAAAAAAFCNOECAA|1913-08-25|163|712|55|1913|1|8|25|3|1913|55|712|Monday|1913Q3|N|N|N|2419981|2420192|2419640|2419914|N|N|N|N|N| +2420006|AAAAAAAAGCNOECAA|1913-08-26|163|713|55|1913|2|8|26|3|1913|55|713|Tuesday|1913Q3|N|N|N|2419981|2420192|2419641|2419915|N|N|N|N|N| +2420007|AAAAAAAAHCNOECAA|1913-08-27|163|713|55|1913|3|8|27|3|1913|55|713|Wednesday|1913Q3|N|N|N|2419981|2420192|2419642|2419916|N|N|N|N|N| +2420008|AAAAAAAAICNOECAA|1913-08-28|163|713|55|1913|4|8|28|3|1913|55|713|Thursday|1913Q3|N|N|N|2419981|2420192|2419643|2419917|N|N|N|N|N| +2420009|AAAAAAAAJCNOECAA|1913-08-29|163|713|55|1913|5|8|29|3|1913|55|713|Friday|1913Q3|N|Y|N|2419981|2420192|2419644|2419918|N|N|N|N|N| +2420010|AAAAAAAAKCNOECAA|1913-08-30|163|713|55|1913|6|8|30|3|1913|55|713|Saturday|1913Q3|N|Y|N|2419981|2420192|2419645|2419919|N|N|N|N|N| +2420011|AAAAAAAALCNOECAA|1913-08-31|163|713|55|1913|0|8|31|3|1913|55|713|Sunday|1913Q3|N|N|N|2419981|2420192|2419646|2419920|N|N|N|N|N| +2420012|AAAAAAAAMCNOECAA|1913-09-01|164|713|56|1913|1|9|1|3|1913|56|713|Monday|1913Q3|N|N|N|2420012|2420254|2419647|2419921|N|N|N|N|N| +2420013|AAAAAAAANCNOECAA|1913-09-02|164|714|56|1913|2|9|2|3|1913|56|714|Tuesday|1913Q3|N|N|N|2420012|2420254|2419648|2419922|N|N|N|N|N| +2420014|AAAAAAAAOCNOECAA|1913-09-03|164|714|56|1913|3|9|3|3|1913|56|714|Wednesday|1913Q3|N|N|N|2420012|2420254|2419649|2419923|N|N|N|N|N| +2420015|AAAAAAAAPCNOECAA|1913-09-04|164|714|56|1913|4|9|4|3|1913|56|714|Thursday|1913Q3|N|N|N|2420012|2420254|2419650|2419924|N|N|N|N|N| +2420016|AAAAAAAAADNOECAA|1913-09-05|164|714|56|1913|5|9|5|3|1913|56|714|Friday|1913Q3|N|Y|N|2420012|2420254|2419651|2419925|N|N|N|N|N| +2420017|AAAAAAAABDNOECAA|1913-09-06|164|714|56|1913|6|9|6|3|1913|56|714|Saturday|1913Q3|N|Y|N|2420012|2420254|2419652|2419926|N|N|N|N|N| +2420018|AAAAAAAACDNOECAA|1913-09-07|164|714|56|1913|0|9|7|3|1913|56|714|Sunday|1913Q3|N|N|N|2420012|2420254|2419653|2419927|N|N|N|N|N| +2420019|AAAAAAAADDNOECAA|1913-09-08|164|714|56|1913|1|9|8|3|1913|56|714|Monday|1913Q3|N|N|N|2420012|2420254|2419654|2419928|N|N|N|N|N| +2420020|AAAAAAAAEDNOECAA|1913-09-09|164|715|56|1913|2|9|9|3|1913|56|715|Tuesday|1913Q3|N|N|N|2420012|2420254|2419655|2419929|N|N|N|N|N| +2420021|AAAAAAAAFDNOECAA|1913-09-10|164|715|56|1913|3|9|10|3|1913|56|715|Wednesday|1913Q3|N|N|N|2420012|2420254|2419656|2419930|N|N|N|N|N| +2420022|AAAAAAAAGDNOECAA|1913-09-11|164|715|56|1913|4|9|11|3|1913|56|715|Thursday|1913Q3|N|N|N|2420012|2420254|2419657|2419931|N|N|N|N|N| +2420023|AAAAAAAAHDNOECAA|1913-09-12|164|715|56|1913|5|9|12|3|1913|56|715|Friday|1913Q3|N|Y|N|2420012|2420254|2419658|2419932|N|N|N|N|N| +2420024|AAAAAAAAIDNOECAA|1913-09-13|164|715|56|1913|6|9|13|3|1913|56|715|Saturday|1913Q3|N|Y|N|2420012|2420254|2419659|2419933|N|N|N|N|N| +2420025|AAAAAAAAJDNOECAA|1913-09-14|164|715|56|1913|0|9|14|3|1913|56|715|Sunday|1913Q3|N|N|N|2420012|2420254|2419660|2419934|N|N|N|N|N| +2420026|AAAAAAAAKDNOECAA|1913-09-15|164|715|56|1913|1|9|15|3|1913|56|715|Monday|1913Q3|N|N|N|2420012|2420254|2419661|2419935|N|N|N|N|N| +2420027|AAAAAAAALDNOECAA|1913-09-16|164|716|56|1913|2|9|16|3|1913|56|716|Tuesday|1913Q3|N|N|N|2420012|2420254|2419662|2419936|N|N|N|N|N| +2420028|AAAAAAAAMDNOECAA|1913-09-17|164|716|56|1913|3|9|17|3|1913|56|716|Wednesday|1913Q3|N|N|N|2420012|2420254|2419663|2419937|N|N|N|N|N| +2420029|AAAAAAAANDNOECAA|1913-09-18|164|716|56|1913|4|9|18|3|1913|56|716|Thursday|1913Q3|N|N|N|2420012|2420254|2419664|2419938|N|N|N|N|N| +2420030|AAAAAAAAODNOECAA|1913-09-19|164|716|56|1913|5|9|19|3|1913|56|716|Friday|1913Q3|N|Y|N|2420012|2420254|2419665|2419939|N|N|N|N|N| +2420031|AAAAAAAAPDNOECAA|1913-09-20|164|716|56|1913|6|9|20|3|1913|56|716|Saturday|1913Q3|N|Y|N|2420012|2420254|2419666|2419940|N|N|N|N|N| +2420032|AAAAAAAAAENOECAA|1913-09-21|164|716|56|1913|0|9|21|3|1913|56|716|Sunday|1913Q3|N|N|N|2420012|2420254|2419667|2419941|N|N|N|N|N| +2420033|AAAAAAAABENOECAA|1913-09-22|164|716|56|1913|1|9|22|3|1913|56|716|Monday|1913Q3|N|N|N|2420012|2420254|2419668|2419942|N|N|N|N|N| +2420034|AAAAAAAACENOECAA|1913-09-23|164|717|56|1913|2|9|23|3|1913|56|717|Tuesday|1913Q3|N|N|N|2420012|2420254|2419669|2419943|N|N|N|N|N| +2420035|AAAAAAAADENOECAA|1913-09-24|164|717|56|1913|3|9|24|3|1913|56|717|Wednesday|1913Q3|N|N|N|2420012|2420254|2419670|2419944|N|N|N|N|N| +2420036|AAAAAAAAEENOECAA|1913-09-25|164|717|56|1913|4|9|25|3|1913|56|717|Thursday|1913Q3|N|N|N|2420012|2420254|2419671|2419945|N|N|N|N|N| +2420037|AAAAAAAAFENOECAA|1913-09-26|164|717|56|1913|5|9|26|3|1913|56|717|Friday|1913Q3|N|Y|N|2420012|2420254|2419672|2419946|N|N|N|N|N| +2420038|AAAAAAAAGENOECAA|1913-09-27|164|717|56|1913|6|9|27|3|1913|56|717|Saturday|1913Q3|N|Y|N|2420012|2420254|2419673|2419947|N|N|N|N|N| +2420039|AAAAAAAAHENOECAA|1913-09-28|164|717|56|1913|0|9|28|3|1913|56|717|Sunday|1913Q3|N|N|N|2420012|2420254|2419674|2419948|N|N|N|N|N| +2420040|AAAAAAAAIENOECAA|1913-09-29|164|717|56|1913|1|9|29|3|1913|56|717|Monday|1913Q3|N|N|N|2420012|2420254|2419675|2419949|N|N|N|N|N| +2420041|AAAAAAAAJENOECAA|1913-09-30|164|718|56|1913|2|9|30|3|1913|56|718|Tuesday|1913Q3|N|N|N|2420012|2420254|2419676|2419950|N|N|N|N|N| +2420042|AAAAAAAAKENOECAA|1913-10-01|165|718|56|1913|3|10|1|3|1913|56|718|Wednesday|1913Q3|N|N|N|2420042|2420314|2419677|2419950|N|N|N|N|N| +2420043|AAAAAAAALENOECAA|1913-10-02|165|718|56|1913|4|10|2|4|1913|56|718|Thursday|1913Q4|N|N|N|2420042|2420314|2419678|2419951|N|N|N|N|N| +2420044|AAAAAAAAMENOECAA|1913-10-03|165|718|56|1913|5|10|3|4|1913|56|718|Friday|1913Q4|N|Y|N|2420042|2420314|2419679|2419952|N|N|N|N|N| +2420045|AAAAAAAANENOECAA|1913-10-04|165|718|56|1913|6|10|4|4|1913|56|718|Saturday|1913Q4|N|Y|N|2420042|2420314|2419680|2419953|N|N|N|N|N| +2420046|AAAAAAAAOENOECAA|1913-10-05|165|718|56|1913|0|10|5|4|1913|56|718|Sunday|1913Q4|N|N|N|2420042|2420314|2419681|2419954|N|N|N|N|N| +2420047|AAAAAAAAPENOECAA|1913-10-06|165|718|56|1913|1|10|6|4|1913|56|718|Monday|1913Q4|N|N|N|2420042|2420314|2419682|2419955|N|N|N|N|N| +2420048|AAAAAAAAAFNOECAA|1913-10-07|165|719|56|1913|2|10|7|4|1913|56|719|Tuesday|1913Q4|N|N|N|2420042|2420314|2419683|2419956|N|N|N|N|N| +2420049|AAAAAAAABFNOECAA|1913-10-08|165|719|56|1913|3|10|8|4|1913|56|719|Wednesday|1913Q4|N|N|N|2420042|2420314|2419684|2419957|N|N|N|N|N| +2420050|AAAAAAAACFNOECAA|1913-10-09|165|719|56|1913|4|10|9|4|1913|56|719|Thursday|1913Q4|N|N|N|2420042|2420314|2419685|2419958|N|N|N|N|N| +2420051|AAAAAAAADFNOECAA|1913-10-10|165|719|56|1913|5|10|10|4|1913|56|719|Friday|1913Q4|N|Y|N|2420042|2420314|2419686|2419959|N|N|N|N|N| +2420052|AAAAAAAAEFNOECAA|1913-10-11|165|719|56|1913|6|10|11|4|1913|56|719|Saturday|1913Q4|N|Y|N|2420042|2420314|2419687|2419960|N|N|N|N|N| +2420053|AAAAAAAAFFNOECAA|1913-10-12|165|719|56|1913|0|10|12|4|1913|56|719|Sunday|1913Q4|N|N|N|2420042|2420314|2419688|2419961|N|N|N|N|N| +2420054|AAAAAAAAGFNOECAA|1913-10-13|165|719|56|1913|1|10|13|4|1913|56|719|Monday|1913Q4|N|N|N|2420042|2420314|2419689|2419962|N|N|N|N|N| +2420055|AAAAAAAAHFNOECAA|1913-10-14|165|720|56|1913|2|10|14|4|1913|56|720|Tuesday|1913Q4|N|N|N|2420042|2420314|2419690|2419963|N|N|N|N|N| +2420056|AAAAAAAAIFNOECAA|1913-10-15|165|720|56|1913|3|10|15|4|1913|56|720|Wednesday|1913Q4|N|N|N|2420042|2420314|2419691|2419964|N|N|N|N|N| +2420057|AAAAAAAAJFNOECAA|1913-10-16|165|720|56|1913|4|10|16|4|1913|56|720|Thursday|1913Q4|N|N|N|2420042|2420314|2419692|2419965|N|N|N|N|N| +2420058|AAAAAAAAKFNOECAA|1913-10-17|165|720|56|1913|5|10|17|4|1913|56|720|Friday|1913Q4|N|Y|N|2420042|2420314|2419693|2419966|N|N|N|N|N| +2420059|AAAAAAAALFNOECAA|1913-10-18|165|720|56|1913|6|10|18|4|1913|56|720|Saturday|1913Q4|N|Y|N|2420042|2420314|2419694|2419967|N|N|N|N|N| +2420060|AAAAAAAAMFNOECAA|1913-10-19|165|720|56|1913|0|10|19|4|1913|56|720|Sunday|1913Q4|N|N|N|2420042|2420314|2419695|2419968|N|N|N|N|N| +2420061|AAAAAAAANFNOECAA|1913-10-20|165|720|56|1913|1|10|20|4|1913|56|720|Monday|1913Q4|N|N|N|2420042|2420314|2419696|2419969|N|N|N|N|N| +2420062|AAAAAAAAOFNOECAA|1913-10-21|165|721|56|1913|2|10|21|4|1913|56|721|Tuesday|1913Q4|N|N|N|2420042|2420314|2419697|2419970|N|N|N|N|N| +2420063|AAAAAAAAPFNOECAA|1913-10-22|165|721|56|1913|3|10|22|4|1913|56|721|Wednesday|1913Q4|N|N|N|2420042|2420314|2419698|2419971|N|N|N|N|N| +2420064|AAAAAAAAAGNOECAA|1913-10-23|165|721|56|1913|4|10|23|4|1913|56|721|Thursday|1913Q4|N|N|N|2420042|2420314|2419699|2419972|N|N|N|N|N| +2420065|AAAAAAAABGNOECAA|1913-10-24|165|721|56|1913|5|10|24|4|1913|56|721|Friday|1913Q4|N|Y|N|2420042|2420314|2419700|2419973|N|N|N|N|N| +2420066|AAAAAAAACGNOECAA|1913-10-25|165|721|56|1913|6|10|25|4|1913|56|721|Saturday|1913Q4|N|Y|N|2420042|2420314|2419701|2419974|N|N|N|N|N| +2420067|AAAAAAAADGNOECAA|1913-10-26|165|721|56|1913|0|10|26|4|1913|56|721|Sunday|1913Q4|N|N|N|2420042|2420314|2419702|2419975|N|N|N|N|N| +2420068|AAAAAAAAEGNOECAA|1913-10-27|165|721|56|1913|1|10|27|4|1913|56|721|Monday|1913Q4|N|N|N|2420042|2420314|2419703|2419976|N|N|N|N|N| +2420069|AAAAAAAAFGNOECAA|1913-10-28|165|722|56|1913|2|10|28|4|1913|56|722|Tuesday|1913Q4|N|N|N|2420042|2420314|2419704|2419977|N|N|N|N|N| +2420070|AAAAAAAAGGNOECAA|1913-10-29|165|722|56|1913|3|10|29|4|1913|56|722|Wednesday|1913Q4|N|N|N|2420042|2420314|2419705|2419978|N|N|N|N|N| +2420071|AAAAAAAAHGNOECAA|1913-10-30|165|722|56|1913|4|10|30|4|1913|56|722|Thursday|1913Q4|N|N|N|2420042|2420314|2419706|2419979|N|N|N|N|N| +2420072|AAAAAAAAIGNOECAA|1913-10-31|165|722|56|1913|5|10|31|4|1913|56|722|Friday|1913Q4|N|Y|N|2420042|2420314|2419707|2419980|N|N|N|N|N| +2420073|AAAAAAAAJGNOECAA|1913-11-01|166|722|56|1913|6|11|1|4|1913|56|722|Saturday|1913Q4|N|Y|N|2420073|2420376|2419708|2419981|N|N|N|N|N| +2420074|AAAAAAAAKGNOECAA|1913-11-02|166|722|56|1913|0|11|2|4|1913|56|722|Sunday|1913Q4|N|N|N|2420073|2420376|2419709|2419982|N|N|N|N|N| +2420075|AAAAAAAALGNOECAA|1913-11-03|166|722|56|1913|1|11|3|4|1913|56|722|Monday|1913Q4|N|N|N|2420073|2420376|2419710|2419983|N|N|N|N|N| +2420076|AAAAAAAAMGNOECAA|1913-11-04|166|723|56|1913|2|11|4|4|1913|56|723|Tuesday|1913Q4|N|N|N|2420073|2420376|2419711|2419984|N|N|N|N|N| +2420077|AAAAAAAANGNOECAA|1913-11-05|166|723|56|1913|3|11|5|4|1913|56|723|Wednesday|1913Q4|N|N|N|2420073|2420376|2419712|2419985|N|N|N|N|N| +2420078|AAAAAAAAOGNOECAA|1913-11-06|166|723|56|1913|4|11|6|4|1913|56|723|Thursday|1913Q4|N|N|N|2420073|2420376|2419713|2419986|N|N|N|N|N| +2420079|AAAAAAAAPGNOECAA|1913-11-07|166|723|56|1913|5|11|7|4|1913|56|723|Friday|1913Q4|N|Y|N|2420073|2420376|2419714|2419987|N|N|N|N|N| +2420080|AAAAAAAAAHNOECAA|1913-11-08|166|723|56|1913|6|11|8|4|1913|56|723|Saturday|1913Q4|N|Y|N|2420073|2420376|2419715|2419988|N|N|N|N|N| +2420081|AAAAAAAABHNOECAA|1913-11-09|166|723|56|1913|0|11|9|4|1913|56|723|Sunday|1913Q4|N|N|N|2420073|2420376|2419716|2419989|N|N|N|N|N| +2420082|AAAAAAAACHNOECAA|1913-11-10|166|723|56|1913|1|11|10|4|1913|56|723|Monday|1913Q4|N|N|N|2420073|2420376|2419717|2419990|N|N|N|N|N| +2420083|AAAAAAAADHNOECAA|1913-11-11|166|724|56|1913|2|11|11|4|1913|56|724|Tuesday|1913Q4|N|N|N|2420073|2420376|2419718|2419991|N|N|N|N|N| +2420084|AAAAAAAAEHNOECAA|1913-11-12|166|724|56|1913|3|11|12|4|1913|56|724|Wednesday|1913Q4|N|N|N|2420073|2420376|2419719|2419992|N|N|N|N|N| +2420085|AAAAAAAAFHNOECAA|1913-11-13|166|724|56|1913|4|11|13|4|1913|56|724|Thursday|1913Q4|N|N|N|2420073|2420376|2419720|2419993|N|N|N|N|N| +2420086|AAAAAAAAGHNOECAA|1913-11-14|166|724|56|1913|5|11|14|4|1913|56|724|Friday|1913Q4|N|Y|N|2420073|2420376|2419721|2419994|N|N|N|N|N| +2420087|AAAAAAAAHHNOECAA|1913-11-15|166|724|56|1913|6|11|15|4|1913|56|724|Saturday|1913Q4|N|Y|N|2420073|2420376|2419722|2419995|N|N|N|N|N| +2420088|AAAAAAAAIHNOECAA|1913-11-16|166|724|56|1913|0|11|16|4|1913|56|724|Sunday|1913Q4|N|N|N|2420073|2420376|2419723|2419996|N|N|N|N|N| +2420089|AAAAAAAAJHNOECAA|1913-11-17|166|724|56|1913|1|11|17|4|1913|56|724|Monday|1913Q4|N|N|N|2420073|2420376|2419724|2419997|N|N|N|N|N| +2420090|AAAAAAAAKHNOECAA|1913-11-18|166|725|56|1913|2|11|18|4|1913|56|725|Tuesday|1913Q4|N|N|N|2420073|2420376|2419725|2419998|N|N|N|N|N| +2420091|AAAAAAAALHNOECAA|1913-11-19|166|725|56|1913|3|11|19|4|1913|56|725|Wednesday|1913Q4|N|N|N|2420073|2420376|2419726|2419999|N|N|N|N|N| +2420092|AAAAAAAAMHNOECAA|1913-11-20|166|725|56|1913|4|11|20|4|1913|56|725|Thursday|1913Q4|N|N|N|2420073|2420376|2419727|2420000|N|N|N|N|N| +2420093|AAAAAAAANHNOECAA|1913-11-21|166|725|56|1913|5|11|21|4|1913|56|725|Friday|1913Q4|N|Y|N|2420073|2420376|2419728|2420001|N|N|N|N|N| +2420094|AAAAAAAAOHNOECAA|1913-11-22|166|725|56|1913|6|11|22|4|1913|56|725|Saturday|1913Q4|N|Y|N|2420073|2420376|2419729|2420002|N|N|N|N|N| +2420095|AAAAAAAAPHNOECAA|1913-11-23|166|725|56|1913|0|11|23|4|1913|56|725|Sunday|1913Q4|N|N|N|2420073|2420376|2419730|2420003|N|N|N|N|N| +2420096|AAAAAAAAAINOECAA|1913-11-24|166|725|56|1913|1|11|24|4|1913|56|725|Monday|1913Q4|N|N|N|2420073|2420376|2419731|2420004|N|N|N|N|N| +2420097|AAAAAAAABINOECAA|1913-11-25|166|726|56|1913|2|11|25|4|1913|56|726|Tuesday|1913Q4|N|N|N|2420073|2420376|2419732|2420005|N|N|N|N|N| +2420098|AAAAAAAACINOECAA|1913-11-26|166|726|56|1913|3|11|26|4|1913|56|726|Wednesday|1913Q4|N|N|N|2420073|2420376|2419733|2420006|N|N|N|N|N| +2420099|AAAAAAAADINOECAA|1913-11-27|166|726|56|1913|4|11|27|4|1913|56|726|Thursday|1913Q4|N|N|N|2420073|2420376|2419734|2420007|N|N|N|N|N| +2420100|AAAAAAAAEINOECAA|1913-11-28|166|726|56|1913|5|11|28|4|1913|56|726|Friday|1913Q4|N|Y|N|2420073|2420376|2419735|2420008|N|N|N|N|N| +2420101|AAAAAAAAFINOECAA|1913-11-29|166|726|56|1913|6|11|29|4|1913|56|726|Saturday|1913Q4|N|Y|N|2420073|2420376|2419736|2420009|N|N|N|N|N| +2420102|AAAAAAAAGINOECAA|1913-11-30|166|726|56|1913|0|11|30|4|1913|56|726|Sunday|1913Q4|N|N|N|2420073|2420376|2419737|2420010|N|N|N|N|N| +2420103|AAAAAAAAHINOECAA|1913-12-01|167|726|57|1913|1|12|1|4|1913|57|726|Monday|1913Q4|N|N|N|2420103|2420436|2419738|2420011|N|N|N|N|N| +2420104|AAAAAAAAIINOECAA|1913-12-02|167|727|57|1913|2|12|2|4|1913|57|727|Tuesday|1913Q4|N|N|N|2420103|2420436|2419739|2420012|N|N|N|N|N| +2420105|AAAAAAAAJINOECAA|1913-12-03|167|727|57|1913|3|12|3|4|1913|57|727|Wednesday|1913Q4|N|N|N|2420103|2420436|2419740|2420013|N|N|N|N|N| +2420106|AAAAAAAAKINOECAA|1913-12-04|167|727|57|1913|4|12|4|4|1913|57|727|Thursday|1913Q4|N|N|N|2420103|2420436|2419741|2420014|N|N|N|N|N| +2420107|AAAAAAAALINOECAA|1913-12-05|167|727|57|1913|5|12|5|4|1913|57|727|Friday|1913Q4|N|Y|N|2420103|2420436|2419742|2420015|N|N|N|N|N| +2420108|AAAAAAAAMINOECAA|1913-12-06|167|727|57|1913|6|12|6|4|1913|57|727|Saturday|1913Q4|N|Y|N|2420103|2420436|2419743|2420016|N|N|N|N|N| +2420109|AAAAAAAANINOECAA|1913-12-07|167|727|57|1913|0|12|7|4|1913|57|727|Sunday|1913Q4|N|N|N|2420103|2420436|2419744|2420017|N|N|N|N|N| +2420110|AAAAAAAAOINOECAA|1913-12-08|167|727|57|1913|1|12|8|4|1913|57|727|Monday|1913Q4|N|N|N|2420103|2420436|2419745|2420018|N|N|N|N|N| +2420111|AAAAAAAAPINOECAA|1913-12-09|167|728|57|1913|2|12|9|4|1913|57|728|Tuesday|1913Q4|N|N|N|2420103|2420436|2419746|2420019|N|N|N|N|N| +2420112|AAAAAAAAAJNOECAA|1913-12-10|167|728|57|1913|3|12|10|4|1913|57|728|Wednesday|1913Q4|N|N|N|2420103|2420436|2419747|2420020|N|N|N|N|N| +2420113|AAAAAAAABJNOECAA|1913-12-11|167|728|57|1913|4|12|11|4|1913|57|728|Thursday|1913Q4|N|N|N|2420103|2420436|2419748|2420021|N|N|N|N|N| +2420114|AAAAAAAACJNOECAA|1913-12-12|167|728|57|1913|5|12|12|4|1913|57|728|Friday|1913Q4|N|Y|N|2420103|2420436|2419749|2420022|N|N|N|N|N| +2420115|AAAAAAAADJNOECAA|1913-12-13|167|728|57|1913|6|12|13|4|1913|57|728|Saturday|1913Q4|N|Y|N|2420103|2420436|2419750|2420023|N|N|N|N|N| +2420116|AAAAAAAAEJNOECAA|1913-12-14|167|728|57|1913|0|12|14|4|1913|57|728|Sunday|1913Q4|N|N|N|2420103|2420436|2419751|2420024|N|N|N|N|N| +2420117|AAAAAAAAFJNOECAA|1913-12-15|167|728|57|1913|1|12|15|4|1913|57|728|Monday|1913Q4|N|N|N|2420103|2420436|2419752|2420025|N|N|N|N|N| +2420118|AAAAAAAAGJNOECAA|1913-12-16|167|729|57|1913|2|12|16|4|1913|57|729|Tuesday|1913Q4|N|N|N|2420103|2420436|2419753|2420026|N|N|N|N|N| +2420119|AAAAAAAAHJNOECAA|1913-12-17|167|729|57|1913|3|12|17|4|1913|57|729|Wednesday|1913Q4|N|N|N|2420103|2420436|2419754|2420027|N|N|N|N|N| +2420120|AAAAAAAAIJNOECAA|1913-12-18|167|729|57|1913|4|12|18|4|1913|57|729|Thursday|1913Q4|N|N|N|2420103|2420436|2419755|2420028|N|N|N|N|N| +2420121|AAAAAAAAJJNOECAA|1913-12-19|167|729|57|1913|5|12|19|4|1913|57|729|Friday|1913Q4|N|Y|N|2420103|2420436|2419756|2420029|N|N|N|N|N| +2420122|AAAAAAAAKJNOECAA|1913-12-20|167|729|57|1913|6|12|20|4|1913|57|729|Saturday|1913Q4|N|Y|N|2420103|2420436|2419757|2420030|N|N|N|N|N| +2420123|AAAAAAAALJNOECAA|1913-12-21|167|729|57|1913|0|12|21|4|1913|57|729|Sunday|1913Q4|N|N|N|2420103|2420436|2419758|2420031|N|N|N|N|N| +2420124|AAAAAAAAMJNOECAA|1913-12-22|167|729|57|1913|1|12|22|4|1913|57|729|Monday|1913Q4|N|N|N|2420103|2420436|2419759|2420032|N|N|N|N|N| +2420125|AAAAAAAANJNOECAA|1913-12-23|167|730|57|1913|2|12|23|4|1913|57|730|Tuesday|1913Q4|N|N|N|2420103|2420436|2419760|2420033|N|N|N|N|N| +2420126|AAAAAAAAOJNOECAA|1913-12-24|167|730|57|1913|3|12|24|4|1913|57|730|Wednesday|1913Q4|N|N|N|2420103|2420436|2419761|2420034|N|N|N|N|N| +2420127|AAAAAAAAPJNOECAA|1913-12-25|167|730|57|1913|4|12|25|4|1913|57|730|Thursday|1913Q4|N|N|N|2420103|2420436|2419762|2420035|N|N|N|N|N| +2420128|AAAAAAAAAKNOECAA|1913-12-26|167|730|57|1913|5|12|26|4|1913|57|730|Friday|1913Q4|Y|Y|N|2420103|2420436|2419763|2420036|N|N|N|N|N| +2420129|AAAAAAAABKNOECAA|1913-12-27|167|730|57|1913|6|12|27|4|1913|57|730|Saturday|1913Q4|N|Y|Y|2420103|2420436|2419764|2420037|N|N|N|N|N| +2420130|AAAAAAAACKNOECAA|1913-12-28|167|730|57|1913|0|12|28|4|1913|57|730|Sunday|1913Q4|N|N|N|2420103|2420436|2419765|2420038|N|N|N|N|N| +2420131|AAAAAAAADKNOECAA|1913-12-29|167|730|57|1913|1|12|29|4|1913|57|730|Monday|1913Q4|N|N|N|2420103|2420436|2419766|2420039|N|N|N|N|N| +2420132|AAAAAAAAEKNOECAA|1913-12-30|167|731|57|1913|2|12|30|4|1913|57|731|Tuesday|1913Q4|N|N|N|2420103|2420436|2419767|2420040|N|N|N|N|N| +2420133|AAAAAAAAFKNOECAA|1913-12-31|167|731|57|1913|3|12|31|4|1913|57|731|Wednesday|1913Q4|N|N|N|2420103|2420436|2419768|2420041|N|N|N|N|N| +2420134|AAAAAAAAGKNOECAA|1914-01-01|168|731|57|1914|4|1|1|1|1914|57|731|Thursday|1914Q1|Y|N|N|2420134|2420133|2419769|2420042|N|N|N|N|N| +2420135|AAAAAAAAHKNOECAA|1914-01-02|168|731|57|1914|5|1|2|1|1914|57|731|Friday|1914Q1|N|Y|Y|2420134|2420133|2419770|2420043|N|N|N|N|N| +2420136|AAAAAAAAIKNOECAA|1914-01-03|168|731|57|1914|6|1|3|1|1914|57|731|Saturday|1914Q1|N|Y|N|2420134|2420133|2419771|2420044|N|N|N|N|N| +2420137|AAAAAAAAJKNOECAA|1914-01-04|168|731|57|1914|0|1|4|1|1914|57|731|Sunday|1914Q1|N|N|N|2420134|2420133|2419772|2420045|N|N|N|N|N| +2420138|AAAAAAAAKKNOECAA|1914-01-05|168|731|57|1914|1|1|5|1|1914|57|731|Monday|1914Q1|N|N|N|2420134|2420133|2419773|2420046|N|N|N|N|N| +2420139|AAAAAAAALKNOECAA|1914-01-06|168|732|57|1914|2|1|6|1|1914|57|732|Tuesday|1914Q1|N|N|N|2420134|2420133|2419774|2420047|N|N|N|N|N| +2420140|AAAAAAAAMKNOECAA|1914-01-07|168|732|57|1914|3|1|7|1|1914|57|732|Wednesday|1914Q1|N|N|N|2420134|2420133|2419775|2420048|N|N|N|N|N| +2420141|AAAAAAAANKNOECAA|1914-01-08|168|732|57|1914|4|1|8|1|1914|57|732|Thursday|1914Q1|N|N|N|2420134|2420133|2419776|2420049|N|N|N|N|N| +2420142|AAAAAAAAOKNOECAA|1914-01-09|168|732|57|1914|5|1|9|1|1914|57|732|Friday|1914Q1|N|Y|N|2420134|2420133|2419777|2420050|N|N|N|N|N| +2420143|AAAAAAAAPKNOECAA|1914-01-10|168|732|57|1914|6|1|10|1|1914|57|732|Saturday|1914Q1|N|Y|N|2420134|2420133|2419778|2420051|N|N|N|N|N| +2420144|AAAAAAAAALNOECAA|1914-01-11|168|732|57|1914|0|1|11|1|1914|57|732|Sunday|1914Q1|N|N|N|2420134|2420133|2419779|2420052|N|N|N|N|N| +2420145|AAAAAAAABLNOECAA|1914-01-12|168|732|57|1914|1|1|12|1|1914|57|732|Monday|1914Q1|N|N|N|2420134|2420133|2419780|2420053|N|N|N|N|N| +2420146|AAAAAAAACLNOECAA|1914-01-13|168|733|57|1914|2|1|13|1|1914|57|733|Tuesday|1914Q1|N|N|N|2420134|2420133|2419781|2420054|N|N|N|N|N| +2420147|AAAAAAAADLNOECAA|1914-01-14|168|733|57|1914|3|1|14|1|1914|57|733|Wednesday|1914Q1|N|N|N|2420134|2420133|2419782|2420055|N|N|N|N|N| +2420148|AAAAAAAAELNOECAA|1914-01-15|168|733|57|1914|4|1|15|1|1914|57|733|Thursday|1914Q1|N|N|N|2420134|2420133|2419783|2420056|N|N|N|N|N| +2420149|AAAAAAAAFLNOECAA|1914-01-16|168|733|57|1914|5|1|16|1|1914|57|733|Friday|1914Q1|N|Y|N|2420134|2420133|2419784|2420057|N|N|N|N|N| +2420150|AAAAAAAAGLNOECAA|1914-01-17|168|733|57|1914|6|1|17|1|1914|57|733|Saturday|1914Q1|N|Y|N|2420134|2420133|2419785|2420058|N|N|N|N|N| +2420151|AAAAAAAAHLNOECAA|1914-01-18|168|733|57|1914|0|1|18|1|1914|57|733|Sunday|1914Q1|N|N|N|2420134|2420133|2419786|2420059|N|N|N|N|N| +2420152|AAAAAAAAILNOECAA|1914-01-19|168|733|57|1914|1|1|19|1|1914|57|733|Monday|1914Q1|N|N|N|2420134|2420133|2419787|2420060|N|N|N|N|N| +2420153|AAAAAAAAJLNOECAA|1914-01-20|168|734|57|1914|2|1|20|1|1914|57|734|Tuesday|1914Q1|N|N|N|2420134|2420133|2419788|2420061|N|N|N|N|N| +2420154|AAAAAAAAKLNOECAA|1914-01-21|168|734|57|1914|3|1|21|1|1914|57|734|Wednesday|1914Q1|N|N|N|2420134|2420133|2419789|2420062|N|N|N|N|N| +2420155|AAAAAAAALLNOECAA|1914-01-22|168|734|57|1914|4|1|22|1|1914|57|734|Thursday|1914Q1|N|N|N|2420134|2420133|2419790|2420063|N|N|N|N|N| +2420156|AAAAAAAAMLNOECAA|1914-01-23|168|734|57|1914|5|1|23|1|1914|57|734|Friday|1914Q1|N|Y|N|2420134|2420133|2419791|2420064|N|N|N|N|N| +2420157|AAAAAAAANLNOECAA|1914-01-24|168|734|57|1914|6|1|24|1|1914|57|734|Saturday|1914Q1|N|Y|N|2420134|2420133|2419792|2420065|N|N|N|N|N| +2420158|AAAAAAAAOLNOECAA|1914-01-25|168|734|57|1914|0|1|25|1|1914|57|734|Sunday|1914Q1|N|N|N|2420134|2420133|2419793|2420066|N|N|N|N|N| +2420159|AAAAAAAAPLNOECAA|1914-01-26|168|734|57|1914|1|1|26|1|1914|57|734|Monday|1914Q1|N|N|N|2420134|2420133|2419794|2420067|N|N|N|N|N| +2420160|AAAAAAAAAMNOECAA|1914-01-27|168|735|57|1914|2|1|27|1|1914|57|735|Tuesday|1914Q1|N|N|N|2420134|2420133|2419795|2420068|N|N|N|N|N| +2420161|AAAAAAAABMNOECAA|1914-01-28|168|735|57|1914|3|1|28|1|1914|57|735|Wednesday|1914Q1|N|N|N|2420134|2420133|2419796|2420069|N|N|N|N|N| +2420162|AAAAAAAACMNOECAA|1914-01-29|168|735|57|1914|4|1|29|1|1914|57|735|Thursday|1914Q1|N|N|N|2420134|2420133|2419797|2420070|N|N|N|N|N| +2420163|AAAAAAAADMNOECAA|1914-01-30|168|735|57|1914|5|1|30|1|1914|57|735|Friday|1914Q1|N|Y|N|2420134|2420133|2419798|2420071|N|N|N|N|N| +2420164|AAAAAAAAEMNOECAA|1914-01-31|168|735|57|1914|6|1|31|1|1914|57|735|Saturday|1914Q1|N|Y|N|2420134|2420133|2419799|2420072|N|N|N|N|N| +2420165|AAAAAAAAFMNOECAA|1914-02-01|169|735|57|1914|0|2|1|1|1914|57|735|Sunday|1914Q1|N|N|N|2420165|2420195|2419800|2420073|N|N|N|N|N| +2420166|AAAAAAAAGMNOECAA|1914-02-02|169|735|57|1914|1|2|2|1|1914|57|735|Monday|1914Q1|N|N|N|2420165|2420195|2419801|2420074|N|N|N|N|N| +2420167|AAAAAAAAHMNOECAA|1914-02-03|169|736|57|1914|2|2|3|1|1914|57|736|Tuesday|1914Q1|N|N|N|2420165|2420195|2419802|2420075|N|N|N|N|N| +2420168|AAAAAAAAIMNOECAA|1914-02-04|169|736|57|1914|3|2|4|1|1914|57|736|Wednesday|1914Q1|N|N|N|2420165|2420195|2419803|2420076|N|N|N|N|N| +2420169|AAAAAAAAJMNOECAA|1914-02-05|169|736|57|1914|4|2|5|1|1914|57|736|Thursday|1914Q1|N|N|N|2420165|2420195|2419804|2420077|N|N|N|N|N| +2420170|AAAAAAAAKMNOECAA|1914-02-06|169|736|57|1914|5|2|6|1|1914|57|736|Friday|1914Q1|N|Y|N|2420165|2420195|2419805|2420078|N|N|N|N|N| +2420171|AAAAAAAALMNOECAA|1914-02-07|169|736|57|1914|6|2|7|1|1914|57|736|Saturday|1914Q1|N|Y|N|2420165|2420195|2419806|2420079|N|N|N|N|N| +2420172|AAAAAAAAMMNOECAA|1914-02-08|169|736|57|1914|0|2|8|1|1914|57|736|Sunday|1914Q1|N|N|N|2420165|2420195|2419807|2420080|N|N|N|N|N| +2420173|AAAAAAAANMNOECAA|1914-02-09|169|736|57|1914|1|2|9|1|1914|57|736|Monday|1914Q1|N|N|N|2420165|2420195|2419808|2420081|N|N|N|N|N| +2420174|AAAAAAAAOMNOECAA|1914-02-10|169|737|57|1914|2|2|10|1|1914|57|737|Tuesday|1914Q1|N|N|N|2420165|2420195|2419809|2420082|N|N|N|N|N| +2420175|AAAAAAAAPMNOECAA|1914-02-11|169|737|57|1914|3|2|11|1|1914|57|737|Wednesday|1914Q1|N|N|N|2420165|2420195|2419810|2420083|N|N|N|N|N| +2420176|AAAAAAAAANNOECAA|1914-02-12|169|737|57|1914|4|2|12|1|1914|57|737|Thursday|1914Q1|N|N|N|2420165|2420195|2419811|2420084|N|N|N|N|N| +2420177|AAAAAAAABNNOECAA|1914-02-13|169|737|57|1914|5|2|13|1|1914|57|737|Friday|1914Q1|N|Y|N|2420165|2420195|2419812|2420085|N|N|N|N|N| +2420178|AAAAAAAACNNOECAA|1914-02-14|169|737|57|1914|6|2|14|1|1914|57|737|Saturday|1914Q1|N|Y|N|2420165|2420195|2419813|2420086|N|N|N|N|N| +2420179|AAAAAAAADNNOECAA|1914-02-15|169|737|57|1914|0|2|15|1|1914|57|737|Sunday|1914Q1|N|N|N|2420165|2420195|2419814|2420087|N|N|N|N|N| +2420180|AAAAAAAAENNOECAA|1914-02-16|169|737|57|1914|1|2|16|1|1914|57|737|Monday|1914Q1|N|N|N|2420165|2420195|2419815|2420088|N|N|N|N|N| +2420181|AAAAAAAAFNNOECAA|1914-02-17|169|738|57|1914|2|2|17|1|1914|57|738|Tuesday|1914Q1|N|N|N|2420165|2420195|2419816|2420089|N|N|N|N|N| +2420182|AAAAAAAAGNNOECAA|1914-02-18|169|738|57|1914|3|2|18|1|1914|57|738|Wednesday|1914Q1|N|N|N|2420165|2420195|2419817|2420090|N|N|N|N|N| +2420183|AAAAAAAAHNNOECAA|1914-02-19|169|738|57|1914|4|2|19|1|1914|57|738|Thursday|1914Q1|N|N|N|2420165|2420195|2419818|2420091|N|N|N|N|N| +2420184|AAAAAAAAINNOECAA|1914-02-20|169|738|57|1914|5|2|20|1|1914|57|738|Friday|1914Q1|N|Y|N|2420165|2420195|2419819|2420092|N|N|N|N|N| +2420185|AAAAAAAAJNNOECAA|1914-02-21|169|738|57|1914|6|2|21|1|1914|57|738|Saturday|1914Q1|N|Y|N|2420165|2420195|2419820|2420093|N|N|N|N|N| +2420186|AAAAAAAAKNNOECAA|1914-02-22|169|738|57|1914|0|2|22|1|1914|57|738|Sunday|1914Q1|N|N|N|2420165|2420195|2419821|2420094|N|N|N|N|N| +2420187|AAAAAAAALNNOECAA|1914-02-23|169|738|57|1914|1|2|23|1|1914|57|738|Monday|1914Q1|N|N|N|2420165|2420195|2419822|2420095|N|N|N|N|N| +2420188|AAAAAAAAMNNOECAA|1914-02-24|169|739|57|1914|2|2|24|1|1914|57|739|Tuesday|1914Q1|N|N|N|2420165|2420195|2419823|2420096|N|N|N|N|N| +2420189|AAAAAAAANNNOECAA|1914-02-25|169|739|57|1914|3|2|25|1|1914|57|739|Wednesday|1914Q1|N|N|N|2420165|2420195|2419824|2420097|N|N|N|N|N| +2420190|AAAAAAAAONNOECAA|1914-02-26|169|739|57|1914|4|2|26|1|1914|57|739|Thursday|1914Q1|N|N|N|2420165|2420195|2419825|2420098|N|N|N|N|N| +2420191|AAAAAAAAPNNOECAA|1914-02-27|169|739|57|1914|5|2|27|1|1914|57|739|Friday|1914Q1|N|Y|N|2420165|2420195|2419826|2420099|N|N|N|N|N| +2420192|AAAAAAAAAONOECAA|1914-02-28|169|739|57|1914|6|2|28|1|1914|57|739|Saturday|1914Q1|N|Y|N|2420165|2420195|2419827|2420100|N|N|N|N|N| +2420193|AAAAAAAABONOECAA|1914-03-01|170|739|58|1914|0|3|1|1|1914|58|739|Sunday|1914Q1|N|N|N|2420193|2420251|2419828|2420101|N|N|N|N|N| +2420194|AAAAAAAACONOECAA|1914-03-02|170|739|58|1914|1|3|2|1|1914|58|739|Monday|1914Q1|N|N|N|2420193|2420251|2419829|2420102|N|N|N|N|N| +2420195|AAAAAAAADONOECAA|1914-03-03|170|740|58|1914|2|3|3|1|1914|58|740|Tuesday|1914Q1|N|N|N|2420193|2420251|2419830|2420103|N|N|N|N|N| +2420196|AAAAAAAAEONOECAA|1914-03-04|170|740|58|1914|3|3|4|1|1914|58|740|Wednesday|1914Q1|N|N|N|2420193|2420251|2419831|2420104|N|N|N|N|N| +2420197|AAAAAAAAFONOECAA|1914-03-05|170|740|58|1914|4|3|5|1|1914|58|740|Thursday|1914Q1|N|N|N|2420193|2420251|2419832|2420105|N|N|N|N|N| +2420198|AAAAAAAAGONOECAA|1914-03-06|170|740|58|1914|5|3|6|1|1914|58|740|Friday|1914Q1|N|Y|N|2420193|2420251|2419833|2420106|N|N|N|N|N| +2420199|AAAAAAAAHONOECAA|1914-03-07|170|740|58|1914|6|3|7|1|1914|58|740|Saturday|1914Q1|N|Y|N|2420193|2420251|2419834|2420107|N|N|N|N|N| +2420200|AAAAAAAAIONOECAA|1914-03-08|170|740|58|1914|0|3|8|1|1914|58|740|Sunday|1914Q1|N|N|N|2420193|2420251|2419835|2420108|N|N|N|N|N| +2420201|AAAAAAAAJONOECAA|1914-03-09|170|740|58|1914|1|3|9|1|1914|58|740|Monday|1914Q1|N|N|N|2420193|2420251|2419836|2420109|N|N|N|N|N| +2420202|AAAAAAAAKONOECAA|1914-03-10|170|741|58|1914|2|3|10|1|1914|58|741|Tuesday|1914Q1|N|N|N|2420193|2420251|2419837|2420110|N|N|N|N|N| +2420203|AAAAAAAALONOECAA|1914-03-11|170|741|58|1914|3|3|11|1|1914|58|741|Wednesday|1914Q1|N|N|N|2420193|2420251|2419838|2420111|N|N|N|N|N| +2420204|AAAAAAAAMONOECAA|1914-03-12|170|741|58|1914|4|3|12|1|1914|58|741|Thursday|1914Q1|N|N|N|2420193|2420251|2419839|2420112|N|N|N|N|N| +2420205|AAAAAAAANONOECAA|1914-03-13|170|741|58|1914|5|3|13|1|1914|58|741|Friday|1914Q1|N|Y|N|2420193|2420251|2419840|2420113|N|N|N|N|N| +2420206|AAAAAAAAOONOECAA|1914-03-14|170|741|58|1914|6|3|14|1|1914|58|741|Saturday|1914Q1|N|Y|N|2420193|2420251|2419841|2420114|N|N|N|N|N| +2420207|AAAAAAAAPONOECAA|1914-03-15|170|741|58|1914|0|3|15|1|1914|58|741|Sunday|1914Q1|N|N|N|2420193|2420251|2419842|2420115|N|N|N|N|N| +2420208|AAAAAAAAAPNOECAA|1914-03-16|170|741|58|1914|1|3|16|1|1914|58|741|Monday|1914Q1|N|N|N|2420193|2420251|2419843|2420116|N|N|N|N|N| +2420209|AAAAAAAABPNOECAA|1914-03-17|170|742|58|1914|2|3|17|1|1914|58|742|Tuesday|1914Q1|N|N|N|2420193|2420251|2419844|2420117|N|N|N|N|N| +2420210|AAAAAAAACPNOECAA|1914-03-18|170|742|58|1914|3|3|18|1|1914|58|742|Wednesday|1914Q1|N|N|N|2420193|2420251|2419845|2420118|N|N|N|N|N| +2420211|AAAAAAAADPNOECAA|1914-03-19|170|742|58|1914|4|3|19|1|1914|58|742|Thursday|1914Q1|N|N|N|2420193|2420251|2419846|2420119|N|N|N|N|N| +2420212|AAAAAAAAEPNOECAA|1914-03-20|170|742|58|1914|5|3|20|1|1914|58|742|Friday|1914Q1|N|Y|N|2420193|2420251|2419847|2420120|N|N|N|N|N| +2420213|AAAAAAAAFPNOECAA|1914-03-21|170|742|58|1914|6|3|21|1|1914|58|742|Saturday|1914Q1|N|Y|N|2420193|2420251|2419848|2420121|N|N|N|N|N| +2420214|AAAAAAAAGPNOECAA|1914-03-22|170|742|58|1914|0|3|22|1|1914|58|742|Sunday|1914Q1|N|N|N|2420193|2420251|2419849|2420122|N|N|N|N|N| +2420215|AAAAAAAAHPNOECAA|1914-03-23|170|742|58|1914|1|3|23|1|1914|58|742|Monday|1914Q1|N|N|N|2420193|2420251|2419850|2420123|N|N|N|N|N| +2420216|AAAAAAAAIPNOECAA|1914-03-24|170|743|58|1914|2|3|24|1|1914|58|743|Tuesday|1914Q1|N|N|N|2420193|2420251|2419851|2420124|N|N|N|N|N| +2420217|AAAAAAAAJPNOECAA|1914-03-25|170|743|58|1914|3|3|25|1|1914|58|743|Wednesday|1914Q1|N|N|N|2420193|2420251|2419852|2420125|N|N|N|N|N| +2420218|AAAAAAAAKPNOECAA|1914-03-26|170|743|58|1914|4|3|26|1|1914|58|743|Thursday|1914Q1|N|N|N|2420193|2420251|2419853|2420126|N|N|N|N|N| +2420219|AAAAAAAALPNOECAA|1914-03-27|170|743|58|1914|5|3|27|1|1914|58|743|Friday|1914Q1|N|Y|N|2420193|2420251|2419854|2420127|N|N|N|N|N| +2420220|AAAAAAAAMPNOECAA|1914-03-28|170|743|58|1914|6|3|28|1|1914|58|743|Saturday|1914Q1|N|Y|N|2420193|2420251|2419855|2420128|N|N|N|N|N| +2420221|AAAAAAAANPNOECAA|1914-03-29|170|743|58|1914|0|3|29|1|1914|58|743|Sunday|1914Q1|N|N|N|2420193|2420251|2419856|2420129|N|N|N|N|N| +2420222|AAAAAAAAOPNOECAA|1914-03-30|170|743|58|1914|1|3|30|1|1914|58|743|Monday|1914Q1|N|N|N|2420193|2420251|2419857|2420130|N|N|N|N|N| +2420223|AAAAAAAAPPNOECAA|1914-03-31|170|744|58|1914|2|3|31|1|1914|58|744|Tuesday|1914Q1|N|N|N|2420193|2420251|2419858|2420131|N|N|N|N|N| +2420224|AAAAAAAAAAOOECAA|1914-04-01|171|744|58|1914|3|4|1|1|1914|58|744|Wednesday|1914Q1|N|N|N|2420224|2420313|2419859|2420134|N|N|N|N|N| +2420225|AAAAAAAABAOOECAA|1914-04-02|171|744|58|1914|4|4|2|2|1914|58|744|Thursday|1914Q2|N|N|N|2420224|2420313|2419860|2420135|N|N|N|N|N| +2420226|AAAAAAAACAOOECAA|1914-04-03|171|744|58|1914|5|4|3|2|1914|58|744|Friday|1914Q2|N|Y|N|2420224|2420313|2419861|2420136|N|N|N|N|N| +2420227|AAAAAAAADAOOECAA|1914-04-04|171|744|58|1914|6|4|4|2|1914|58|744|Saturday|1914Q2|N|Y|N|2420224|2420313|2419862|2420137|N|N|N|N|N| +2420228|AAAAAAAAEAOOECAA|1914-04-05|171|744|58|1914|0|4|5|2|1914|58|744|Sunday|1914Q2|N|N|N|2420224|2420313|2419863|2420138|N|N|N|N|N| +2420229|AAAAAAAAFAOOECAA|1914-04-06|171|744|58|1914|1|4|6|2|1914|58|744|Monday|1914Q2|N|N|N|2420224|2420313|2419864|2420139|N|N|N|N|N| +2420230|AAAAAAAAGAOOECAA|1914-04-07|171|745|58|1914|2|4|7|2|1914|58|745|Tuesday|1914Q2|N|N|N|2420224|2420313|2419865|2420140|N|N|N|N|N| +2420231|AAAAAAAAHAOOECAA|1914-04-08|171|745|58|1914|3|4|8|2|1914|58|745|Wednesday|1914Q2|N|N|N|2420224|2420313|2419866|2420141|N|N|N|N|N| +2420232|AAAAAAAAIAOOECAA|1914-04-09|171|745|58|1914|4|4|9|2|1914|58|745|Thursday|1914Q2|N|N|N|2420224|2420313|2419867|2420142|N|N|N|N|N| +2420233|AAAAAAAAJAOOECAA|1914-04-10|171|745|58|1914|5|4|10|2|1914|58|745|Friday|1914Q2|N|Y|N|2420224|2420313|2419868|2420143|N|N|N|N|N| +2420234|AAAAAAAAKAOOECAA|1914-04-11|171|745|58|1914|6|4|11|2|1914|58|745|Saturday|1914Q2|N|Y|N|2420224|2420313|2419869|2420144|N|N|N|N|N| +2420235|AAAAAAAALAOOECAA|1914-04-12|171|745|58|1914|0|4|12|2|1914|58|745|Sunday|1914Q2|N|N|N|2420224|2420313|2419870|2420145|N|N|N|N|N| +2420236|AAAAAAAAMAOOECAA|1914-04-13|171|745|58|1914|1|4|13|2|1914|58|745|Monday|1914Q2|N|N|N|2420224|2420313|2419871|2420146|N|N|N|N|N| +2420237|AAAAAAAANAOOECAA|1914-04-14|171|746|58|1914|2|4|14|2|1914|58|746|Tuesday|1914Q2|N|N|N|2420224|2420313|2419872|2420147|N|N|N|N|N| +2420238|AAAAAAAAOAOOECAA|1914-04-15|171|746|58|1914|3|4|15|2|1914|58|746|Wednesday|1914Q2|N|N|N|2420224|2420313|2419873|2420148|N|N|N|N|N| +2420239|AAAAAAAAPAOOECAA|1914-04-16|171|746|58|1914|4|4|16|2|1914|58|746|Thursday|1914Q2|N|N|N|2420224|2420313|2419874|2420149|N|N|N|N|N| +2420240|AAAAAAAAABOOECAA|1914-04-17|171|746|58|1914|5|4|17|2|1914|58|746|Friday|1914Q2|N|Y|N|2420224|2420313|2419875|2420150|N|N|N|N|N| +2420241|AAAAAAAABBOOECAA|1914-04-18|171|746|58|1914|6|4|18|2|1914|58|746|Saturday|1914Q2|N|Y|N|2420224|2420313|2419876|2420151|N|N|N|N|N| +2420242|AAAAAAAACBOOECAA|1914-04-19|171|746|58|1914|0|4|19|2|1914|58|746|Sunday|1914Q2|N|N|N|2420224|2420313|2419877|2420152|N|N|N|N|N| +2420243|AAAAAAAADBOOECAA|1914-04-20|171|746|58|1914|1|4|20|2|1914|58|746|Monday|1914Q2|N|N|N|2420224|2420313|2419878|2420153|N|N|N|N|N| +2420244|AAAAAAAAEBOOECAA|1914-04-21|171|747|58|1914|2|4|21|2|1914|58|747|Tuesday|1914Q2|N|N|N|2420224|2420313|2419879|2420154|N|N|N|N|N| +2420245|AAAAAAAAFBOOECAA|1914-04-22|171|747|58|1914|3|4|22|2|1914|58|747|Wednesday|1914Q2|N|N|N|2420224|2420313|2419880|2420155|N|N|N|N|N| +2420246|AAAAAAAAGBOOECAA|1914-04-23|171|747|58|1914|4|4|23|2|1914|58|747|Thursday|1914Q2|N|N|N|2420224|2420313|2419881|2420156|N|N|N|N|N| +2420247|AAAAAAAAHBOOECAA|1914-04-24|171|747|58|1914|5|4|24|2|1914|58|747|Friday|1914Q2|N|Y|N|2420224|2420313|2419882|2420157|N|N|N|N|N| +2420248|AAAAAAAAIBOOECAA|1914-04-25|171|747|58|1914|6|4|25|2|1914|58|747|Saturday|1914Q2|N|Y|N|2420224|2420313|2419883|2420158|N|N|N|N|N| +2420249|AAAAAAAAJBOOECAA|1914-04-26|171|747|58|1914|0|4|26|2|1914|58|747|Sunday|1914Q2|N|N|N|2420224|2420313|2419884|2420159|N|N|N|N|N| +2420250|AAAAAAAAKBOOECAA|1914-04-27|171|747|58|1914|1|4|27|2|1914|58|747|Monday|1914Q2|N|N|N|2420224|2420313|2419885|2420160|N|N|N|N|N| +2420251|AAAAAAAALBOOECAA|1914-04-28|171|748|58|1914|2|4|28|2|1914|58|748|Tuesday|1914Q2|N|N|N|2420224|2420313|2419886|2420161|N|N|N|N|N| +2420252|AAAAAAAAMBOOECAA|1914-04-29|171|748|58|1914|3|4|29|2|1914|58|748|Wednesday|1914Q2|N|N|N|2420224|2420313|2419887|2420162|N|N|N|N|N| +2420253|AAAAAAAANBOOECAA|1914-04-30|171|748|58|1914|4|4|30|2|1914|58|748|Thursday|1914Q2|N|N|N|2420224|2420313|2419888|2420163|N|N|N|N|N| +2420254|AAAAAAAAOBOOECAA|1914-05-01|172|748|58|1914|5|5|1|2|1914|58|748|Friday|1914Q2|N|Y|N|2420254|2420373|2419889|2420164|N|N|N|N|N| +2420255|AAAAAAAAPBOOECAA|1914-05-02|172|748|58|1914|6|5|2|2|1914|58|748|Saturday|1914Q2|N|Y|N|2420254|2420373|2419890|2420165|N|N|N|N|N| +2420256|AAAAAAAAACOOECAA|1914-05-03|172|748|58|1914|0|5|3|2|1914|58|748|Sunday|1914Q2|N|N|N|2420254|2420373|2419891|2420166|N|N|N|N|N| +2420257|AAAAAAAABCOOECAA|1914-05-04|172|748|58|1914|1|5|4|2|1914|58|748|Monday|1914Q2|N|N|N|2420254|2420373|2419892|2420167|N|N|N|N|N| +2420258|AAAAAAAACCOOECAA|1914-05-05|172|749|58|1914|2|5|5|2|1914|58|749|Tuesday|1914Q2|N|N|N|2420254|2420373|2419893|2420168|N|N|N|N|N| +2420259|AAAAAAAADCOOECAA|1914-05-06|172|749|58|1914|3|5|6|2|1914|58|749|Wednesday|1914Q2|N|N|N|2420254|2420373|2419894|2420169|N|N|N|N|N| +2420260|AAAAAAAAECOOECAA|1914-05-07|172|749|58|1914|4|5|7|2|1914|58|749|Thursday|1914Q2|N|N|N|2420254|2420373|2419895|2420170|N|N|N|N|N| +2420261|AAAAAAAAFCOOECAA|1914-05-08|172|749|58|1914|5|5|8|2|1914|58|749|Friday|1914Q2|N|Y|N|2420254|2420373|2419896|2420171|N|N|N|N|N| +2420262|AAAAAAAAGCOOECAA|1914-05-09|172|749|58|1914|6|5|9|2|1914|58|749|Saturday|1914Q2|N|Y|N|2420254|2420373|2419897|2420172|N|N|N|N|N| +2420263|AAAAAAAAHCOOECAA|1914-05-10|172|749|58|1914|0|5|10|2|1914|58|749|Sunday|1914Q2|N|N|N|2420254|2420373|2419898|2420173|N|N|N|N|N| +2420264|AAAAAAAAICOOECAA|1914-05-11|172|749|58|1914|1|5|11|2|1914|58|749|Monday|1914Q2|N|N|N|2420254|2420373|2419899|2420174|N|N|N|N|N| +2420265|AAAAAAAAJCOOECAA|1914-05-12|172|750|58|1914|2|5|12|2|1914|58|750|Tuesday|1914Q2|N|N|N|2420254|2420373|2419900|2420175|N|N|N|N|N| +2420266|AAAAAAAAKCOOECAA|1914-05-13|172|750|58|1914|3|5|13|2|1914|58|750|Wednesday|1914Q2|N|N|N|2420254|2420373|2419901|2420176|N|N|N|N|N| +2420267|AAAAAAAALCOOECAA|1914-05-14|172|750|58|1914|4|5|14|2|1914|58|750|Thursday|1914Q2|N|N|N|2420254|2420373|2419902|2420177|N|N|N|N|N| +2420268|AAAAAAAAMCOOECAA|1914-05-15|172|750|58|1914|5|5|15|2|1914|58|750|Friday|1914Q2|N|Y|N|2420254|2420373|2419903|2420178|N|N|N|N|N| +2420269|AAAAAAAANCOOECAA|1914-05-16|172|750|58|1914|6|5|16|2|1914|58|750|Saturday|1914Q2|N|Y|N|2420254|2420373|2419904|2420179|N|N|N|N|N| +2420270|AAAAAAAAOCOOECAA|1914-05-17|172|750|58|1914|0|5|17|2|1914|58|750|Sunday|1914Q2|N|N|N|2420254|2420373|2419905|2420180|N|N|N|N|N| +2420271|AAAAAAAAPCOOECAA|1914-05-18|172|750|58|1914|1|5|18|2|1914|58|750|Monday|1914Q2|N|N|N|2420254|2420373|2419906|2420181|N|N|N|N|N| +2420272|AAAAAAAAADOOECAA|1914-05-19|172|751|58|1914|2|5|19|2|1914|58|751|Tuesday|1914Q2|N|N|N|2420254|2420373|2419907|2420182|N|N|N|N|N| +2420273|AAAAAAAABDOOECAA|1914-05-20|172|751|58|1914|3|5|20|2|1914|58|751|Wednesday|1914Q2|N|N|N|2420254|2420373|2419908|2420183|N|N|N|N|N| +2420274|AAAAAAAACDOOECAA|1914-05-21|172|751|58|1914|4|5|21|2|1914|58|751|Thursday|1914Q2|N|N|N|2420254|2420373|2419909|2420184|N|N|N|N|N| +2420275|AAAAAAAADDOOECAA|1914-05-22|172|751|58|1914|5|5|22|2|1914|58|751|Friday|1914Q2|N|Y|N|2420254|2420373|2419910|2420185|N|N|N|N|N| +2420276|AAAAAAAAEDOOECAA|1914-05-23|172|751|58|1914|6|5|23|2|1914|58|751|Saturday|1914Q2|N|Y|N|2420254|2420373|2419911|2420186|N|N|N|N|N| +2420277|AAAAAAAAFDOOECAA|1914-05-24|172|751|58|1914|0|5|24|2|1914|58|751|Sunday|1914Q2|N|N|N|2420254|2420373|2419912|2420187|N|N|N|N|N| +2420278|AAAAAAAAGDOOECAA|1914-05-25|172|751|58|1914|1|5|25|2|1914|58|751|Monday|1914Q2|N|N|N|2420254|2420373|2419913|2420188|N|N|N|N|N| +2420279|AAAAAAAAHDOOECAA|1914-05-26|172|752|58|1914|2|5|26|2|1914|58|752|Tuesday|1914Q2|N|N|N|2420254|2420373|2419914|2420189|N|N|N|N|N| +2420280|AAAAAAAAIDOOECAA|1914-05-27|172|752|58|1914|3|5|27|2|1914|58|752|Wednesday|1914Q2|N|N|N|2420254|2420373|2419915|2420190|N|N|N|N|N| +2420281|AAAAAAAAJDOOECAA|1914-05-28|172|752|58|1914|4|5|28|2|1914|58|752|Thursday|1914Q2|N|N|N|2420254|2420373|2419916|2420191|N|N|N|N|N| +2420282|AAAAAAAAKDOOECAA|1914-05-29|172|752|58|1914|5|5|29|2|1914|58|752|Friday|1914Q2|N|Y|N|2420254|2420373|2419917|2420192|N|N|N|N|N| +2420283|AAAAAAAALDOOECAA|1914-05-30|172|752|58|1914|6|5|30|2|1914|58|752|Saturday|1914Q2|N|Y|N|2420254|2420373|2419918|2420193|N|N|N|N|N| +2420284|AAAAAAAAMDOOECAA|1914-05-31|172|752|58|1914|0|5|31|2|1914|58|752|Sunday|1914Q2|N|N|N|2420254|2420373|2419919|2420194|N|N|N|N|N| +2420285|AAAAAAAANDOOECAA|1914-06-01|173|752|59|1914|1|6|1|2|1914|59|752|Monday|1914Q2|N|N|N|2420285|2420435|2419920|2420195|N|N|N|N|N| +2420286|AAAAAAAAODOOECAA|1914-06-02|173|753|59|1914|2|6|2|2|1914|59|753|Tuesday|1914Q2|N|N|N|2420285|2420435|2419921|2420196|N|N|N|N|N| +2420287|AAAAAAAAPDOOECAA|1914-06-03|173|753|59|1914|3|6|3|2|1914|59|753|Wednesday|1914Q2|N|N|N|2420285|2420435|2419922|2420197|N|N|N|N|N| +2420288|AAAAAAAAAEOOECAA|1914-06-04|173|753|59|1914|4|6|4|2|1914|59|753|Thursday|1914Q2|N|N|N|2420285|2420435|2419923|2420198|N|N|N|N|N| +2420289|AAAAAAAABEOOECAA|1914-06-05|173|753|59|1914|5|6|5|2|1914|59|753|Friday|1914Q2|N|Y|N|2420285|2420435|2419924|2420199|N|N|N|N|N| +2420290|AAAAAAAACEOOECAA|1914-06-06|173|753|59|1914|6|6|6|2|1914|59|753|Saturday|1914Q2|N|Y|N|2420285|2420435|2419925|2420200|N|N|N|N|N| +2420291|AAAAAAAADEOOECAA|1914-06-07|173|753|59|1914|0|6|7|2|1914|59|753|Sunday|1914Q2|N|N|N|2420285|2420435|2419926|2420201|N|N|N|N|N| +2420292|AAAAAAAAEEOOECAA|1914-06-08|173|753|59|1914|1|6|8|2|1914|59|753|Monday|1914Q2|N|N|N|2420285|2420435|2419927|2420202|N|N|N|N|N| +2420293|AAAAAAAAFEOOECAA|1914-06-09|173|754|59|1914|2|6|9|2|1914|59|754|Tuesday|1914Q2|N|N|N|2420285|2420435|2419928|2420203|N|N|N|N|N| +2420294|AAAAAAAAGEOOECAA|1914-06-10|173|754|59|1914|3|6|10|2|1914|59|754|Wednesday|1914Q2|N|N|N|2420285|2420435|2419929|2420204|N|N|N|N|N| +2420295|AAAAAAAAHEOOECAA|1914-06-11|173|754|59|1914|4|6|11|2|1914|59|754|Thursday|1914Q2|N|N|N|2420285|2420435|2419930|2420205|N|N|N|N|N| +2420296|AAAAAAAAIEOOECAA|1914-06-12|173|754|59|1914|5|6|12|2|1914|59|754|Friday|1914Q2|N|Y|N|2420285|2420435|2419931|2420206|N|N|N|N|N| +2420297|AAAAAAAAJEOOECAA|1914-06-13|173|754|59|1914|6|6|13|2|1914|59|754|Saturday|1914Q2|N|Y|N|2420285|2420435|2419932|2420207|N|N|N|N|N| +2420298|AAAAAAAAKEOOECAA|1914-06-14|173|754|59|1914|0|6|14|2|1914|59|754|Sunday|1914Q2|N|N|N|2420285|2420435|2419933|2420208|N|N|N|N|N| +2420299|AAAAAAAALEOOECAA|1914-06-15|173|754|59|1914|1|6|15|2|1914|59|754|Monday|1914Q2|N|N|N|2420285|2420435|2419934|2420209|N|N|N|N|N| +2420300|AAAAAAAAMEOOECAA|1914-06-16|173|755|59|1914|2|6|16|2|1914|59|755|Tuesday|1914Q2|N|N|N|2420285|2420435|2419935|2420210|N|N|N|N|N| +2420301|AAAAAAAANEOOECAA|1914-06-17|173|755|59|1914|3|6|17|2|1914|59|755|Wednesday|1914Q2|N|N|N|2420285|2420435|2419936|2420211|N|N|N|N|N| +2420302|AAAAAAAAOEOOECAA|1914-06-18|173|755|59|1914|4|6|18|2|1914|59|755|Thursday|1914Q2|N|N|N|2420285|2420435|2419937|2420212|N|N|N|N|N| +2420303|AAAAAAAAPEOOECAA|1914-06-19|173|755|59|1914|5|6|19|2|1914|59|755|Friday|1914Q2|N|Y|N|2420285|2420435|2419938|2420213|N|N|N|N|N| +2420304|AAAAAAAAAFOOECAA|1914-06-20|173|755|59|1914|6|6|20|2|1914|59|755|Saturday|1914Q2|N|Y|N|2420285|2420435|2419939|2420214|N|N|N|N|N| +2420305|AAAAAAAABFOOECAA|1914-06-21|173|755|59|1914|0|6|21|2|1914|59|755|Sunday|1914Q2|N|N|N|2420285|2420435|2419940|2420215|N|N|N|N|N| +2420306|AAAAAAAACFOOECAA|1914-06-22|173|755|59|1914|1|6|22|2|1914|59|755|Monday|1914Q2|N|N|N|2420285|2420435|2419941|2420216|N|N|N|N|N| +2420307|AAAAAAAADFOOECAA|1914-06-23|173|756|59|1914|2|6|23|2|1914|59|756|Tuesday|1914Q2|N|N|N|2420285|2420435|2419942|2420217|N|N|N|N|N| +2420308|AAAAAAAAEFOOECAA|1914-06-24|173|756|59|1914|3|6|24|2|1914|59|756|Wednesday|1914Q2|N|N|N|2420285|2420435|2419943|2420218|N|N|N|N|N| +2420309|AAAAAAAAFFOOECAA|1914-06-25|173|756|59|1914|4|6|25|2|1914|59|756|Thursday|1914Q2|N|N|N|2420285|2420435|2419944|2420219|N|N|N|N|N| +2420310|AAAAAAAAGFOOECAA|1914-06-26|173|756|59|1914|5|6|26|2|1914|59|756|Friday|1914Q2|N|Y|N|2420285|2420435|2419945|2420220|N|N|N|N|N| +2420311|AAAAAAAAHFOOECAA|1914-06-27|173|756|59|1914|6|6|27|2|1914|59|756|Saturday|1914Q2|N|Y|N|2420285|2420435|2419946|2420221|N|N|N|N|N| +2420312|AAAAAAAAIFOOECAA|1914-06-28|173|756|59|1914|0|6|28|2|1914|59|756|Sunday|1914Q2|N|N|N|2420285|2420435|2419947|2420222|N|N|N|N|N| +2420313|AAAAAAAAJFOOECAA|1914-06-29|173|756|59|1914|1|6|29|2|1914|59|756|Monday|1914Q2|N|N|N|2420285|2420435|2419948|2420223|N|N|N|N|N| +2420314|AAAAAAAAKFOOECAA|1914-06-30|173|757|59|1914|2|6|30|2|1914|59|757|Tuesday|1914Q2|N|N|N|2420285|2420435|2419949|2420224|N|N|N|N|N| +2420315|AAAAAAAALFOOECAA|1914-07-01|174|757|59|1914|3|7|1|2|1914|59|757|Wednesday|1914Q2|N|N|N|2420315|2420495|2419950|2420224|N|N|N|N|N| +2420316|AAAAAAAAMFOOECAA|1914-07-02|174|757|59|1914|4|7|2|3|1914|59|757|Thursday|1914Q3|N|N|N|2420315|2420495|2419951|2420225|N|N|N|N|N| +2420317|AAAAAAAANFOOECAA|1914-07-03|174|757|59|1914|5|7|3|3|1914|59|757|Friday|1914Q3|N|Y|N|2420315|2420495|2419952|2420226|N|N|N|N|N| +2420318|AAAAAAAAOFOOECAA|1914-07-04|174|757|59|1914|6|7|4|3|1914|59|757|Saturday|1914Q3|N|Y|N|2420315|2420495|2419953|2420227|N|N|N|N|N| +2420319|AAAAAAAAPFOOECAA|1914-07-05|174|757|59|1914|0|7|5|3|1914|59|757|Sunday|1914Q3|Y|N|N|2420315|2420495|2419954|2420228|N|N|N|N|N| +2420320|AAAAAAAAAGOOECAA|1914-07-06|174|757|59|1914|1|7|6|3|1914|59|757|Monday|1914Q3|N|N|Y|2420315|2420495|2419955|2420229|N|N|N|N|N| +2420321|AAAAAAAABGOOECAA|1914-07-07|174|758|59|1914|2|7|7|3|1914|59|758|Tuesday|1914Q3|N|N|N|2420315|2420495|2419956|2420230|N|N|N|N|N| +2420322|AAAAAAAACGOOECAA|1914-07-08|174|758|59|1914|3|7|8|3|1914|59|758|Wednesday|1914Q3|N|N|N|2420315|2420495|2419957|2420231|N|N|N|N|N| +2420323|AAAAAAAADGOOECAA|1914-07-09|174|758|59|1914|4|7|9|3|1914|59|758|Thursday|1914Q3|N|N|N|2420315|2420495|2419958|2420232|N|N|N|N|N| +2420324|AAAAAAAAEGOOECAA|1914-07-10|174|758|59|1914|5|7|10|3|1914|59|758|Friday|1914Q3|N|Y|N|2420315|2420495|2419959|2420233|N|N|N|N|N| +2420325|AAAAAAAAFGOOECAA|1914-07-11|174|758|59|1914|6|7|11|3|1914|59|758|Saturday|1914Q3|N|Y|N|2420315|2420495|2419960|2420234|N|N|N|N|N| +2420326|AAAAAAAAGGOOECAA|1914-07-12|174|758|59|1914|0|7|12|3|1914|59|758|Sunday|1914Q3|N|N|N|2420315|2420495|2419961|2420235|N|N|N|N|N| +2420327|AAAAAAAAHGOOECAA|1914-07-13|174|758|59|1914|1|7|13|3|1914|59|758|Monday|1914Q3|N|N|N|2420315|2420495|2419962|2420236|N|N|N|N|N| +2420328|AAAAAAAAIGOOECAA|1914-07-14|174|759|59|1914|2|7|14|3|1914|59|759|Tuesday|1914Q3|N|N|N|2420315|2420495|2419963|2420237|N|N|N|N|N| +2420329|AAAAAAAAJGOOECAA|1914-07-15|174|759|59|1914|3|7|15|3|1914|59|759|Wednesday|1914Q3|N|N|N|2420315|2420495|2419964|2420238|N|N|N|N|N| +2420330|AAAAAAAAKGOOECAA|1914-07-16|174|759|59|1914|4|7|16|3|1914|59|759|Thursday|1914Q3|N|N|N|2420315|2420495|2419965|2420239|N|N|N|N|N| +2420331|AAAAAAAALGOOECAA|1914-07-17|174|759|59|1914|5|7|17|3|1914|59|759|Friday|1914Q3|N|Y|N|2420315|2420495|2419966|2420240|N|N|N|N|N| +2420332|AAAAAAAAMGOOECAA|1914-07-18|174|759|59|1914|6|7|18|3|1914|59|759|Saturday|1914Q3|N|Y|N|2420315|2420495|2419967|2420241|N|N|N|N|N| +2420333|AAAAAAAANGOOECAA|1914-07-19|174|759|59|1914|0|7|19|3|1914|59|759|Sunday|1914Q3|N|N|N|2420315|2420495|2419968|2420242|N|N|N|N|N| +2420334|AAAAAAAAOGOOECAA|1914-07-20|174|759|59|1914|1|7|20|3|1914|59|759|Monday|1914Q3|N|N|N|2420315|2420495|2419969|2420243|N|N|N|N|N| +2420335|AAAAAAAAPGOOECAA|1914-07-21|174|760|59|1914|2|7|21|3|1914|59|760|Tuesday|1914Q3|N|N|N|2420315|2420495|2419970|2420244|N|N|N|N|N| +2420336|AAAAAAAAAHOOECAA|1914-07-22|174|760|59|1914|3|7|22|3|1914|59|760|Wednesday|1914Q3|N|N|N|2420315|2420495|2419971|2420245|N|N|N|N|N| +2420337|AAAAAAAABHOOECAA|1914-07-23|174|760|59|1914|4|7|23|3|1914|59|760|Thursday|1914Q3|N|N|N|2420315|2420495|2419972|2420246|N|N|N|N|N| +2420338|AAAAAAAACHOOECAA|1914-07-24|174|760|59|1914|5|7|24|3|1914|59|760|Friday|1914Q3|N|Y|N|2420315|2420495|2419973|2420247|N|N|N|N|N| +2420339|AAAAAAAADHOOECAA|1914-07-25|174|760|59|1914|6|7|25|3|1914|59|760|Saturday|1914Q3|N|Y|N|2420315|2420495|2419974|2420248|N|N|N|N|N| +2420340|AAAAAAAAEHOOECAA|1914-07-26|174|760|59|1914|0|7|26|3|1914|59|760|Sunday|1914Q3|N|N|N|2420315|2420495|2419975|2420249|N|N|N|N|N| +2420341|AAAAAAAAFHOOECAA|1914-07-27|174|760|59|1914|1|7|27|3|1914|59|760|Monday|1914Q3|N|N|N|2420315|2420495|2419976|2420250|N|N|N|N|N| +2420342|AAAAAAAAGHOOECAA|1914-07-28|174|761|59|1914|2|7|28|3|1914|59|761|Tuesday|1914Q3|N|N|N|2420315|2420495|2419977|2420251|N|N|N|N|N| +2420343|AAAAAAAAHHOOECAA|1914-07-29|174|761|59|1914|3|7|29|3|1914|59|761|Wednesday|1914Q3|N|N|N|2420315|2420495|2419978|2420252|N|N|N|N|N| +2420344|AAAAAAAAIHOOECAA|1914-07-30|174|761|59|1914|4|7|30|3|1914|59|761|Thursday|1914Q3|N|N|N|2420315|2420495|2419979|2420253|N|N|N|N|N| +2420345|AAAAAAAAJHOOECAA|1914-07-31|174|761|59|1914|5|7|31|3|1914|59|761|Friday|1914Q3|N|Y|N|2420315|2420495|2419980|2420254|N|N|N|N|N| +2420346|AAAAAAAAKHOOECAA|1914-08-01|175|761|59|1914|6|8|1|3|1914|59|761|Saturday|1914Q3|N|Y|N|2420346|2420557|2419981|2420255|N|N|N|N|N| +2420347|AAAAAAAALHOOECAA|1914-08-02|175|761|59|1914|0|8|2|3|1914|59|761|Sunday|1914Q3|N|N|N|2420346|2420557|2419982|2420256|N|N|N|N|N| +2420348|AAAAAAAAMHOOECAA|1914-08-03|175|761|59|1914|1|8|3|3|1914|59|761|Monday|1914Q3|N|N|N|2420346|2420557|2419983|2420257|N|N|N|N|N| +2420349|AAAAAAAANHOOECAA|1914-08-04|175|762|59|1914|2|8|4|3|1914|59|762|Tuesday|1914Q3|N|N|N|2420346|2420557|2419984|2420258|N|N|N|N|N| +2420350|AAAAAAAAOHOOECAA|1914-08-05|175|762|59|1914|3|8|5|3|1914|59|762|Wednesday|1914Q3|N|N|N|2420346|2420557|2419985|2420259|N|N|N|N|N| +2420351|AAAAAAAAPHOOECAA|1914-08-06|175|762|59|1914|4|8|6|3|1914|59|762|Thursday|1914Q3|N|N|N|2420346|2420557|2419986|2420260|N|N|N|N|N| +2420352|AAAAAAAAAIOOECAA|1914-08-07|175|762|59|1914|5|8|7|3|1914|59|762|Friday|1914Q3|N|Y|N|2420346|2420557|2419987|2420261|N|N|N|N|N| +2420353|AAAAAAAABIOOECAA|1914-08-08|175|762|59|1914|6|8|8|3|1914|59|762|Saturday|1914Q3|N|Y|N|2420346|2420557|2419988|2420262|N|N|N|N|N| +2420354|AAAAAAAACIOOECAA|1914-08-09|175|762|59|1914|0|8|9|3|1914|59|762|Sunday|1914Q3|N|N|N|2420346|2420557|2419989|2420263|N|N|N|N|N| +2420355|AAAAAAAADIOOECAA|1914-08-10|175|762|59|1914|1|8|10|3|1914|59|762|Monday|1914Q3|N|N|N|2420346|2420557|2419990|2420264|N|N|N|N|N| +2420356|AAAAAAAAEIOOECAA|1914-08-11|175|763|59|1914|2|8|11|3|1914|59|763|Tuesday|1914Q3|N|N|N|2420346|2420557|2419991|2420265|N|N|N|N|N| +2420357|AAAAAAAAFIOOECAA|1914-08-12|175|763|59|1914|3|8|12|3|1914|59|763|Wednesday|1914Q3|N|N|N|2420346|2420557|2419992|2420266|N|N|N|N|N| +2420358|AAAAAAAAGIOOECAA|1914-08-13|175|763|59|1914|4|8|13|3|1914|59|763|Thursday|1914Q3|N|N|N|2420346|2420557|2419993|2420267|N|N|N|N|N| +2420359|AAAAAAAAHIOOECAA|1914-08-14|175|763|59|1914|5|8|14|3|1914|59|763|Friday|1914Q3|N|Y|N|2420346|2420557|2419994|2420268|N|N|N|N|N| +2420360|AAAAAAAAIIOOECAA|1914-08-15|175|763|59|1914|6|8|15|3|1914|59|763|Saturday|1914Q3|N|Y|N|2420346|2420557|2419995|2420269|N|N|N|N|N| +2420361|AAAAAAAAJIOOECAA|1914-08-16|175|763|59|1914|0|8|16|3|1914|59|763|Sunday|1914Q3|N|N|N|2420346|2420557|2419996|2420270|N|N|N|N|N| +2420362|AAAAAAAAKIOOECAA|1914-08-17|175|763|59|1914|1|8|17|3|1914|59|763|Monday|1914Q3|N|N|N|2420346|2420557|2419997|2420271|N|N|N|N|N| +2420363|AAAAAAAALIOOECAA|1914-08-18|175|764|59|1914|2|8|18|3|1914|59|764|Tuesday|1914Q3|N|N|N|2420346|2420557|2419998|2420272|N|N|N|N|N| +2420364|AAAAAAAAMIOOECAA|1914-08-19|175|764|59|1914|3|8|19|3|1914|59|764|Wednesday|1914Q3|N|N|N|2420346|2420557|2419999|2420273|N|N|N|N|N| +2420365|AAAAAAAANIOOECAA|1914-08-20|175|764|59|1914|4|8|20|3|1914|59|764|Thursday|1914Q3|N|N|N|2420346|2420557|2420000|2420274|N|N|N|N|N| +2420366|AAAAAAAAOIOOECAA|1914-08-21|175|764|59|1914|5|8|21|3|1914|59|764|Friday|1914Q3|N|Y|N|2420346|2420557|2420001|2420275|N|N|N|N|N| +2420367|AAAAAAAAPIOOECAA|1914-08-22|175|764|59|1914|6|8|22|3|1914|59|764|Saturday|1914Q3|N|Y|N|2420346|2420557|2420002|2420276|N|N|N|N|N| +2420368|AAAAAAAAAJOOECAA|1914-08-23|175|764|59|1914|0|8|23|3|1914|59|764|Sunday|1914Q3|N|N|N|2420346|2420557|2420003|2420277|N|N|N|N|N| +2420369|AAAAAAAABJOOECAA|1914-08-24|175|764|59|1914|1|8|24|3|1914|59|764|Monday|1914Q3|N|N|N|2420346|2420557|2420004|2420278|N|N|N|N|N| +2420370|AAAAAAAACJOOECAA|1914-08-25|175|765|59|1914|2|8|25|3|1914|59|765|Tuesday|1914Q3|N|N|N|2420346|2420557|2420005|2420279|N|N|N|N|N| +2420371|AAAAAAAADJOOECAA|1914-08-26|175|765|59|1914|3|8|26|3|1914|59|765|Wednesday|1914Q3|N|N|N|2420346|2420557|2420006|2420280|N|N|N|N|N| +2420372|AAAAAAAAEJOOECAA|1914-08-27|175|765|59|1914|4|8|27|3|1914|59|765|Thursday|1914Q3|N|N|N|2420346|2420557|2420007|2420281|N|N|N|N|N| +2420373|AAAAAAAAFJOOECAA|1914-08-28|175|765|59|1914|5|8|28|3|1914|59|765|Friday|1914Q3|N|Y|N|2420346|2420557|2420008|2420282|N|N|N|N|N| +2420374|AAAAAAAAGJOOECAA|1914-08-29|175|765|59|1914|6|8|29|3|1914|59|765|Saturday|1914Q3|N|Y|N|2420346|2420557|2420009|2420283|N|N|N|N|N| +2420375|AAAAAAAAHJOOECAA|1914-08-30|175|765|59|1914|0|8|30|3|1914|59|765|Sunday|1914Q3|N|N|N|2420346|2420557|2420010|2420284|N|N|N|N|N| +2420376|AAAAAAAAIJOOECAA|1914-08-31|175|765|59|1914|1|8|31|3|1914|59|765|Monday|1914Q3|N|N|N|2420346|2420557|2420011|2420285|N|N|N|N|N| +2420377|AAAAAAAAJJOOECAA|1914-09-01|176|766|60|1914|2|9|1|3|1914|60|766|Tuesday|1914Q3|N|N|N|2420377|2420619|2420012|2420286|N|N|N|N|N| +2420378|AAAAAAAAKJOOECAA|1914-09-02|176|766|60|1914|3|9|2|3|1914|60|766|Wednesday|1914Q3|N|N|N|2420377|2420619|2420013|2420287|N|N|N|N|N| +2420379|AAAAAAAALJOOECAA|1914-09-03|176|766|60|1914|4|9|3|3|1914|60|766|Thursday|1914Q3|N|N|N|2420377|2420619|2420014|2420288|N|N|N|N|N| +2420380|AAAAAAAAMJOOECAA|1914-09-04|176|766|60|1914|5|9|4|3|1914|60|766|Friday|1914Q3|N|Y|N|2420377|2420619|2420015|2420289|N|N|N|N|N| +2420381|AAAAAAAANJOOECAA|1914-09-05|176|766|60|1914|6|9|5|3|1914|60|766|Saturday|1914Q3|N|Y|N|2420377|2420619|2420016|2420290|N|N|N|N|N| +2420382|AAAAAAAAOJOOECAA|1914-09-06|176|766|60|1914|0|9|6|3|1914|60|766|Sunday|1914Q3|N|N|N|2420377|2420619|2420017|2420291|N|N|N|N|N| +2420383|AAAAAAAAPJOOECAA|1914-09-07|176|766|60|1914|1|9|7|3|1914|60|766|Monday|1914Q3|N|N|N|2420377|2420619|2420018|2420292|N|N|N|N|N| +2420384|AAAAAAAAAKOOECAA|1914-09-08|176|767|60|1914|2|9|8|3|1914|60|767|Tuesday|1914Q3|N|N|N|2420377|2420619|2420019|2420293|N|N|N|N|N| +2420385|AAAAAAAABKOOECAA|1914-09-09|176|767|60|1914|3|9|9|3|1914|60|767|Wednesday|1914Q3|N|N|N|2420377|2420619|2420020|2420294|N|N|N|N|N| +2420386|AAAAAAAACKOOECAA|1914-09-10|176|767|60|1914|4|9|10|3|1914|60|767|Thursday|1914Q3|N|N|N|2420377|2420619|2420021|2420295|N|N|N|N|N| +2420387|AAAAAAAADKOOECAA|1914-09-11|176|767|60|1914|5|9|11|3|1914|60|767|Friday|1914Q3|N|Y|N|2420377|2420619|2420022|2420296|N|N|N|N|N| +2420388|AAAAAAAAEKOOECAA|1914-09-12|176|767|60|1914|6|9|12|3|1914|60|767|Saturday|1914Q3|N|Y|N|2420377|2420619|2420023|2420297|N|N|N|N|N| +2420389|AAAAAAAAFKOOECAA|1914-09-13|176|767|60|1914|0|9|13|3|1914|60|767|Sunday|1914Q3|N|N|N|2420377|2420619|2420024|2420298|N|N|N|N|N| +2420390|AAAAAAAAGKOOECAA|1914-09-14|176|767|60|1914|1|9|14|3|1914|60|767|Monday|1914Q3|N|N|N|2420377|2420619|2420025|2420299|N|N|N|N|N| +2420391|AAAAAAAAHKOOECAA|1914-09-15|176|768|60|1914|2|9|15|3|1914|60|768|Tuesday|1914Q3|N|N|N|2420377|2420619|2420026|2420300|N|N|N|N|N| +2420392|AAAAAAAAIKOOECAA|1914-09-16|176|768|60|1914|3|9|16|3|1914|60|768|Wednesday|1914Q3|N|N|N|2420377|2420619|2420027|2420301|N|N|N|N|N| +2420393|AAAAAAAAJKOOECAA|1914-09-17|176|768|60|1914|4|9|17|3|1914|60|768|Thursday|1914Q3|N|N|N|2420377|2420619|2420028|2420302|N|N|N|N|N| +2420394|AAAAAAAAKKOOECAA|1914-09-18|176|768|60|1914|5|9|18|3|1914|60|768|Friday|1914Q3|N|Y|N|2420377|2420619|2420029|2420303|N|N|N|N|N| +2420395|AAAAAAAALKOOECAA|1914-09-19|176|768|60|1914|6|9|19|3|1914|60|768|Saturday|1914Q3|N|Y|N|2420377|2420619|2420030|2420304|N|N|N|N|N| +2420396|AAAAAAAAMKOOECAA|1914-09-20|176|768|60|1914|0|9|20|3|1914|60|768|Sunday|1914Q3|N|N|N|2420377|2420619|2420031|2420305|N|N|N|N|N| +2420397|AAAAAAAANKOOECAA|1914-09-21|176|768|60|1914|1|9|21|3|1914|60|768|Monday|1914Q3|N|N|N|2420377|2420619|2420032|2420306|N|N|N|N|N| +2420398|AAAAAAAAOKOOECAA|1914-09-22|176|769|60|1914|2|9|22|3|1914|60|769|Tuesday|1914Q3|N|N|N|2420377|2420619|2420033|2420307|N|N|N|N|N| +2420399|AAAAAAAAPKOOECAA|1914-09-23|176|769|60|1914|3|9|23|3|1914|60|769|Wednesday|1914Q3|N|N|N|2420377|2420619|2420034|2420308|N|N|N|N|N| +2420400|AAAAAAAAALOOECAA|1914-09-24|176|769|60|1914|4|9|24|3|1914|60|769|Thursday|1914Q3|N|N|N|2420377|2420619|2420035|2420309|N|N|N|N|N| +2420401|AAAAAAAABLOOECAA|1914-09-25|176|769|60|1914|5|9|25|3|1914|60|769|Friday|1914Q3|N|Y|N|2420377|2420619|2420036|2420310|N|N|N|N|N| +2420402|AAAAAAAACLOOECAA|1914-09-26|176|769|60|1914|6|9|26|3|1914|60|769|Saturday|1914Q3|N|Y|N|2420377|2420619|2420037|2420311|N|N|N|N|N| +2420403|AAAAAAAADLOOECAA|1914-09-27|176|769|60|1914|0|9|27|3|1914|60|769|Sunday|1914Q3|N|N|N|2420377|2420619|2420038|2420312|N|N|N|N|N| +2420404|AAAAAAAAELOOECAA|1914-09-28|176|769|60|1914|1|9|28|3|1914|60|769|Monday|1914Q3|N|N|N|2420377|2420619|2420039|2420313|N|N|N|N|N| +2420405|AAAAAAAAFLOOECAA|1914-09-29|176|770|60|1914|2|9|29|3|1914|60|770|Tuesday|1914Q3|N|N|N|2420377|2420619|2420040|2420314|N|N|N|N|N| +2420406|AAAAAAAAGLOOECAA|1914-09-30|176|770|60|1914|3|9|30|3|1914|60|770|Wednesday|1914Q3|N|N|N|2420377|2420619|2420041|2420315|N|N|N|N|N| +2420407|AAAAAAAAHLOOECAA|1914-10-01|177|770|60|1914|4|10|1|3|1914|60|770|Thursday|1914Q3|N|N|N|2420407|2420679|2420042|2420315|N|N|N|N|N| +2420408|AAAAAAAAILOOECAA|1914-10-02|177|770|60|1914|5|10|2|4|1914|60|770|Friday|1914Q4|N|Y|N|2420407|2420679|2420043|2420316|N|N|N|N|N| +2420409|AAAAAAAAJLOOECAA|1914-10-03|177|770|60|1914|6|10|3|4|1914|60|770|Saturday|1914Q4|N|Y|N|2420407|2420679|2420044|2420317|N|N|N|N|N| +2420410|AAAAAAAAKLOOECAA|1914-10-04|177|770|60|1914|0|10|4|4|1914|60|770|Sunday|1914Q4|N|N|N|2420407|2420679|2420045|2420318|N|N|N|N|N| +2420411|AAAAAAAALLOOECAA|1914-10-05|177|770|60|1914|1|10|5|4|1914|60|770|Monday|1914Q4|N|N|N|2420407|2420679|2420046|2420319|N|N|N|N|N| +2420412|AAAAAAAAMLOOECAA|1914-10-06|177|771|60|1914|2|10|6|4|1914|60|771|Tuesday|1914Q4|N|N|N|2420407|2420679|2420047|2420320|N|N|N|N|N| +2420413|AAAAAAAANLOOECAA|1914-10-07|177|771|60|1914|3|10|7|4|1914|60|771|Wednesday|1914Q4|N|N|N|2420407|2420679|2420048|2420321|N|N|N|N|N| +2420414|AAAAAAAAOLOOECAA|1914-10-08|177|771|60|1914|4|10|8|4|1914|60|771|Thursday|1914Q4|N|N|N|2420407|2420679|2420049|2420322|N|N|N|N|N| +2420415|AAAAAAAAPLOOECAA|1914-10-09|177|771|60|1914|5|10|9|4|1914|60|771|Friday|1914Q4|N|Y|N|2420407|2420679|2420050|2420323|N|N|N|N|N| +2420416|AAAAAAAAAMOOECAA|1914-10-10|177|771|60|1914|6|10|10|4|1914|60|771|Saturday|1914Q4|N|Y|N|2420407|2420679|2420051|2420324|N|N|N|N|N| +2420417|AAAAAAAABMOOECAA|1914-10-11|177|771|60|1914|0|10|11|4|1914|60|771|Sunday|1914Q4|N|N|N|2420407|2420679|2420052|2420325|N|N|N|N|N| +2420418|AAAAAAAACMOOECAA|1914-10-12|177|771|60|1914|1|10|12|4|1914|60|771|Monday|1914Q4|N|N|N|2420407|2420679|2420053|2420326|N|N|N|N|N| +2420419|AAAAAAAADMOOECAA|1914-10-13|177|772|60|1914|2|10|13|4|1914|60|772|Tuesday|1914Q4|N|N|N|2420407|2420679|2420054|2420327|N|N|N|N|N| +2420420|AAAAAAAAEMOOECAA|1914-10-14|177|772|60|1914|3|10|14|4|1914|60|772|Wednesday|1914Q4|N|N|N|2420407|2420679|2420055|2420328|N|N|N|N|N| +2420421|AAAAAAAAFMOOECAA|1914-10-15|177|772|60|1914|4|10|15|4|1914|60|772|Thursday|1914Q4|N|N|N|2420407|2420679|2420056|2420329|N|N|N|N|N| +2420422|AAAAAAAAGMOOECAA|1914-10-16|177|772|60|1914|5|10|16|4|1914|60|772|Friday|1914Q4|N|Y|N|2420407|2420679|2420057|2420330|N|N|N|N|N| +2420423|AAAAAAAAHMOOECAA|1914-10-17|177|772|60|1914|6|10|17|4|1914|60|772|Saturday|1914Q4|N|Y|N|2420407|2420679|2420058|2420331|N|N|N|N|N| +2420424|AAAAAAAAIMOOECAA|1914-10-18|177|772|60|1914|0|10|18|4|1914|60|772|Sunday|1914Q4|N|N|N|2420407|2420679|2420059|2420332|N|N|N|N|N| +2420425|AAAAAAAAJMOOECAA|1914-10-19|177|772|60|1914|1|10|19|4|1914|60|772|Monday|1914Q4|N|N|N|2420407|2420679|2420060|2420333|N|N|N|N|N| +2420426|AAAAAAAAKMOOECAA|1914-10-20|177|773|60|1914|2|10|20|4|1914|60|773|Tuesday|1914Q4|N|N|N|2420407|2420679|2420061|2420334|N|N|N|N|N| +2420427|AAAAAAAALMOOECAA|1914-10-21|177|773|60|1914|3|10|21|4|1914|60|773|Wednesday|1914Q4|N|N|N|2420407|2420679|2420062|2420335|N|N|N|N|N| +2420428|AAAAAAAAMMOOECAA|1914-10-22|177|773|60|1914|4|10|22|4|1914|60|773|Thursday|1914Q4|N|N|N|2420407|2420679|2420063|2420336|N|N|N|N|N| +2420429|AAAAAAAANMOOECAA|1914-10-23|177|773|60|1914|5|10|23|4|1914|60|773|Friday|1914Q4|N|Y|N|2420407|2420679|2420064|2420337|N|N|N|N|N| +2420430|AAAAAAAAOMOOECAA|1914-10-24|177|773|60|1914|6|10|24|4|1914|60|773|Saturday|1914Q4|N|Y|N|2420407|2420679|2420065|2420338|N|N|N|N|N| +2420431|AAAAAAAAPMOOECAA|1914-10-25|177|773|60|1914|0|10|25|4|1914|60|773|Sunday|1914Q4|N|N|N|2420407|2420679|2420066|2420339|N|N|N|N|N| +2420432|AAAAAAAAANOOECAA|1914-10-26|177|773|60|1914|1|10|26|4|1914|60|773|Monday|1914Q4|N|N|N|2420407|2420679|2420067|2420340|N|N|N|N|N| +2420433|AAAAAAAABNOOECAA|1914-10-27|177|774|60|1914|2|10|27|4|1914|60|774|Tuesday|1914Q4|N|N|N|2420407|2420679|2420068|2420341|N|N|N|N|N| +2420434|AAAAAAAACNOOECAA|1914-10-28|177|774|60|1914|3|10|28|4|1914|60|774|Wednesday|1914Q4|N|N|N|2420407|2420679|2420069|2420342|N|N|N|N|N| +2420435|AAAAAAAADNOOECAA|1914-10-29|177|774|60|1914|4|10|29|4|1914|60|774|Thursday|1914Q4|N|N|N|2420407|2420679|2420070|2420343|N|N|N|N|N| +2420436|AAAAAAAAENOOECAA|1914-10-30|177|774|60|1914|5|10|30|4|1914|60|774|Friday|1914Q4|N|Y|N|2420407|2420679|2420071|2420344|N|N|N|N|N| +2420437|AAAAAAAAFNOOECAA|1914-10-31|177|774|60|1914|6|10|31|4|1914|60|774|Saturday|1914Q4|N|Y|N|2420407|2420679|2420072|2420345|N|N|N|N|N| +2420438|AAAAAAAAGNOOECAA|1914-11-01|178|774|60|1914|0|11|1|4|1914|60|774|Sunday|1914Q4|N|N|N|2420438|2420741|2420073|2420346|N|N|N|N|N| +2420439|AAAAAAAAHNOOECAA|1914-11-02|178|774|60|1914|1|11|2|4|1914|60|774|Monday|1914Q4|N|N|N|2420438|2420741|2420074|2420347|N|N|N|N|N| +2420440|AAAAAAAAINOOECAA|1914-11-03|178|775|60|1914|2|11|3|4|1914|60|775|Tuesday|1914Q4|N|N|N|2420438|2420741|2420075|2420348|N|N|N|N|N| +2420441|AAAAAAAAJNOOECAA|1914-11-04|178|775|60|1914|3|11|4|4|1914|60|775|Wednesday|1914Q4|N|N|N|2420438|2420741|2420076|2420349|N|N|N|N|N| +2420442|AAAAAAAAKNOOECAA|1914-11-05|178|775|60|1914|4|11|5|4|1914|60|775|Thursday|1914Q4|N|N|N|2420438|2420741|2420077|2420350|N|N|N|N|N| +2420443|AAAAAAAALNOOECAA|1914-11-06|178|775|60|1914|5|11|6|4|1914|60|775|Friday|1914Q4|N|Y|N|2420438|2420741|2420078|2420351|N|N|N|N|N| +2420444|AAAAAAAAMNOOECAA|1914-11-07|178|775|60|1914|6|11|7|4|1914|60|775|Saturday|1914Q4|N|Y|N|2420438|2420741|2420079|2420352|N|N|N|N|N| +2420445|AAAAAAAANNOOECAA|1914-11-08|178|775|60|1914|0|11|8|4|1914|60|775|Sunday|1914Q4|N|N|N|2420438|2420741|2420080|2420353|N|N|N|N|N| +2420446|AAAAAAAAONOOECAA|1914-11-09|178|775|60|1914|1|11|9|4|1914|60|775|Monday|1914Q4|N|N|N|2420438|2420741|2420081|2420354|N|N|N|N|N| +2420447|AAAAAAAAPNOOECAA|1914-11-10|178|776|60|1914|2|11|10|4|1914|60|776|Tuesday|1914Q4|N|N|N|2420438|2420741|2420082|2420355|N|N|N|N|N| +2420448|AAAAAAAAAOOOECAA|1914-11-11|178|776|60|1914|3|11|11|4|1914|60|776|Wednesday|1914Q4|N|N|N|2420438|2420741|2420083|2420356|N|N|N|N|N| +2420449|AAAAAAAABOOOECAA|1914-11-12|178|776|60|1914|4|11|12|4|1914|60|776|Thursday|1914Q4|N|N|N|2420438|2420741|2420084|2420357|N|N|N|N|N| +2420450|AAAAAAAACOOOECAA|1914-11-13|178|776|60|1914|5|11|13|4|1914|60|776|Friday|1914Q4|N|Y|N|2420438|2420741|2420085|2420358|N|N|N|N|N| +2420451|AAAAAAAADOOOECAA|1914-11-14|178|776|60|1914|6|11|14|4|1914|60|776|Saturday|1914Q4|N|Y|N|2420438|2420741|2420086|2420359|N|N|N|N|N| +2420452|AAAAAAAAEOOOECAA|1914-11-15|178|776|60|1914|0|11|15|4|1914|60|776|Sunday|1914Q4|N|N|N|2420438|2420741|2420087|2420360|N|N|N|N|N| +2420453|AAAAAAAAFOOOECAA|1914-11-16|178|776|60|1914|1|11|16|4|1914|60|776|Monday|1914Q4|N|N|N|2420438|2420741|2420088|2420361|N|N|N|N|N| +2420454|AAAAAAAAGOOOECAA|1914-11-17|178|777|60|1914|2|11|17|4|1914|60|777|Tuesday|1914Q4|N|N|N|2420438|2420741|2420089|2420362|N|N|N|N|N| +2420455|AAAAAAAAHOOOECAA|1914-11-18|178|777|60|1914|3|11|18|4|1914|60|777|Wednesday|1914Q4|N|N|N|2420438|2420741|2420090|2420363|N|N|N|N|N| +2420456|AAAAAAAAIOOOECAA|1914-11-19|178|777|60|1914|4|11|19|4|1914|60|777|Thursday|1914Q4|N|N|N|2420438|2420741|2420091|2420364|N|N|N|N|N| +2420457|AAAAAAAAJOOOECAA|1914-11-20|178|777|60|1914|5|11|20|4|1914|60|777|Friday|1914Q4|N|Y|N|2420438|2420741|2420092|2420365|N|N|N|N|N| +2420458|AAAAAAAAKOOOECAA|1914-11-21|178|777|60|1914|6|11|21|4|1914|60|777|Saturday|1914Q4|N|Y|N|2420438|2420741|2420093|2420366|N|N|N|N|N| +2420459|AAAAAAAALOOOECAA|1914-11-22|178|777|60|1914|0|11|22|4|1914|60|777|Sunday|1914Q4|N|N|N|2420438|2420741|2420094|2420367|N|N|N|N|N| +2420460|AAAAAAAAMOOOECAA|1914-11-23|178|777|60|1914|1|11|23|4|1914|60|777|Monday|1914Q4|N|N|N|2420438|2420741|2420095|2420368|N|N|N|N|N| +2420461|AAAAAAAANOOOECAA|1914-11-24|178|778|60|1914|2|11|24|4|1914|60|778|Tuesday|1914Q4|N|N|N|2420438|2420741|2420096|2420369|N|N|N|N|N| +2420462|AAAAAAAAOOOOECAA|1914-11-25|178|778|60|1914|3|11|25|4|1914|60|778|Wednesday|1914Q4|N|N|N|2420438|2420741|2420097|2420370|N|N|N|N|N| +2420463|AAAAAAAAPOOOECAA|1914-11-26|178|778|60|1914|4|11|26|4|1914|60|778|Thursday|1914Q4|N|N|N|2420438|2420741|2420098|2420371|N|N|N|N|N| +2420464|AAAAAAAAAPOOECAA|1914-11-27|178|778|60|1914|5|11|27|4|1914|60|778|Friday|1914Q4|N|Y|N|2420438|2420741|2420099|2420372|N|N|N|N|N| +2420465|AAAAAAAABPOOECAA|1914-11-28|178|778|60|1914|6|11|28|4|1914|60|778|Saturday|1914Q4|N|Y|N|2420438|2420741|2420100|2420373|N|N|N|N|N| +2420466|AAAAAAAACPOOECAA|1914-11-29|178|778|60|1914|0|11|29|4|1914|60|778|Sunday|1914Q4|N|N|N|2420438|2420741|2420101|2420374|N|N|N|N|N| +2420467|AAAAAAAADPOOECAA|1914-11-30|178|778|60|1914|1|11|30|4|1914|60|778|Monday|1914Q4|N|N|N|2420438|2420741|2420102|2420375|N|N|N|N|N| +2420468|AAAAAAAAEPOOECAA|1914-12-01|179|779|61|1914|2|12|1|4|1914|61|779|Tuesday|1914Q4|N|N|N|2420468|2420801|2420103|2420376|N|N|N|N|N| +2420469|AAAAAAAAFPOOECAA|1914-12-02|179|779|61|1914|3|12|2|4|1914|61|779|Wednesday|1914Q4|N|N|N|2420468|2420801|2420104|2420377|N|N|N|N|N| +2420470|AAAAAAAAGPOOECAA|1914-12-03|179|779|61|1914|4|12|3|4|1914|61|779|Thursday|1914Q4|N|N|N|2420468|2420801|2420105|2420378|N|N|N|N|N| +2420471|AAAAAAAAHPOOECAA|1914-12-04|179|779|61|1914|5|12|4|4|1914|61|779|Friday|1914Q4|N|Y|N|2420468|2420801|2420106|2420379|N|N|N|N|N| +2420472|AAAAAAAAIPOOECAA|1914-12-05|179|779|61|1914|6|12|5|4|1914|61|779|Saturday|1914Q4|N|Y|N|2420468|2420801|2420107|2420380|N|N|N|N|N| +2420473|AAAAAAAAJPOOECAA|1914-12-06|179|779|61|1914|0|12|6|4|1914|61|779|Sunday|1914Q4|N|N|N|2420468|2420801|2420108|2420381|N|N|N|N|N| +2420474|AAAAAAAAKPOOECAA|1914-12-07|179|779|61|1914|1|12|7|4|1914|61|779|Monday|1914Q4|N|N|N|2420468|2420801|2420109|2420382|N|N|N|N|N| +2420475|AAAAAAAALPOOECAA|1914-12-08|179|780|61|1914|2|12|8|4|1914|61|780|Tuesday|1914Q4|N|N|N|2420468|2420801|2420110|2420383|N|N|N|N|N| +2420476|AAAAAAAAMPOOECAA|1914-12-09|179|780|61|1914|3|12|9|4|1914|61|780|Wednesday|1914Q4|N|N|N|2420468|2420801|2420111|2420384|N|N|N|N|N| +2420477|AAAAAAAANPOOECAA|1914-12-10|179|780|61|1914|4|12|10|4|1914|61|780|Thursday|1914Q4|N|N|N|2420468|2420801|2420112|2420385|N|N|N|N|N| +2420478|AAAAAAAAOPOOECAA|1914-12-11|179|780|61|1914|5|12|11|4|1914|61|780|Friday|1914Q4|N|Y|N|2420468|2420801|2420113|2420386|N|N|N|N|N| +2420479|AAAAAAAAPPOOECAA|1914-12-12|179|780|61|1914|6|12|12|4|1914|61|780|Saturday|1914Q4|N|Y|N|2420468|2420801|2420114|2420387|N|N|N|N|N| +2420480|AAAAAAAAAAPOECAA|1914-12-13|179|780|61|1914|0|12|13|4|1914|61|780|Sunday|1914Q4|N|N|N|2420468|2420801|2420115|2420388|N|N|N|N|N| +2420481|AAAAAAAABAPOECAA|1914-12-14|179|780|61|1914|1|12|14|4|1914|61|780|Monday|1914Q4|N|N|N|2420468|2420801|2420116|2420389|N|N|N|N|N| +2420482|AAAAAAAACAPOECAA|1914-12-15|179|781|61|1914|2|12|15|4|1914|61|781|Tuesday|1914Q4|N|N|N|2420468|2420801|2420117|2420390|N|N|N|N|N| +2420483|AAAAAAAADAPOECAA|1914-12-16|179|781|61|1914|3|12|16|4|1914|61|781|Wednesday|1914Q4|N|N|N|2420468|2420801|2420118|2420391|N|N|N|N|N| +2420484|AAAAAAAAEAPOECAA|1914-12-17|179|781|61|1914|4|12|17|4|1914|61|781|Thursday|1914Q4|N|N|N|2420468|2420801|2420119|2420392|N|N|N|N|N| +2420485|AAAAAAAAFAPOECAA|1914-12-18|179|781|61|1914|5|12|18|4|1914|61|781|Friday|1914Q4|N|Y|N|2420468|2420801|2420120|2420393|N|N|N|N|N| +2420486|AAAAAAAAGAPOECAA|1914-12-19|179|781|61|1914|6|12|19|4|1914|61|781|Saturday|1914Q4|N|Y|N|2420468|2420801|2420121|2420394|N|N|N|N|N| +2420487|AAAAAAAAHAPOECAA|1914-12-20|179|781|61|1914|0|12|20|4|1914|61|781|Sunday|1914Q4|N|N|N|2420468|2420801|2420122|2420395|N|N|N|N|N| +2420488|AAAAAAAAIAPOECAA|1914-12-21|179|781|61|1914|1|12|21|4|1914|61|781|Monday|1914Q4|N|N|N|2420468|2420801|2420123|2420396|N|N|N|N|N| +2420489|AAAAAAAAJAPOECAA|1914-12-22|179|782|61|1914|2|12|22|4|1914|61|782|Tuesday|1914Q4|N|N|N|2420468|2420801|2420124|2420397|N|N|N|N|N| +2420490|AAAAAAAAKAPOECAA|1914-12-23|179|782|61|1914|3|12|23|4|1914|61|782|Wednesday|1914Q4|N|N|N|2420468|2420801|2420125|2420398|N|N|N|N|N| +2420491|AAAAAAAALAPOECAA|1914-12-24|179|782|61|1914|4|12|24|4|1914|61|782|Thursday|1914Q4|N|N|N|2420468|2420801|2420126|2420399|N|N|N|N|N| +2420492|AAAAAAAAMAPOECAA|1914-12-25|179|782|61|1914|5|12|25|4|1914|61|782|Friday|1914Q4|N|Y|N|2420468|2420801|2420127|2420400|N|N|N|N|N| +2420493|AAAAAAAANAPOECAA|1914-12-26|179|782|61|1914|6|12|26|4|1914|61|782|Saturday|1914Q4|Y|Y|N|2420468|2420801|2420128|2420401|N|N|N|N|N| +2420494|AAAAAAAAOAPOECAA|1914-12-27|179|782|61|1914|0|12|27|4|1914|61|782|Sunday|1914Q4|N|N|Y|2420468|2420801|2420129|2420402|N|N|N|N|N| +2420495|AAAAAAAAPAPOECAA|1914-12-28|179|782|61|1914|1|12|28|4|1914|61|782|Monday|1914Q4|N|N|N|2420468|2420801|2420130|2420403|N|N|N|N|N| +2420496|AAAAAAAAABPOECAA|1914-12-29|179|783|61|1914|2|12|29|4|1914|61|783|Tuesday|1914Q4|N|N|N|2420468|2420801|2420131|2420404|N|N|N|N|N| +2420497|AAAAAAAABBPOECAA|1914-12-30|179|783|61|1914|3|12|30|4|1914|61|783|Wednesday|1914Q4|N|N|N|2420468|2420801|2420132|2420405|N|N|N|N|N| +2420498|AAAAAAAACBPOECAA|1914-12-31|179|783|61|1914|4|12|31|4|1914|61|783|Thursday|1914Q4|N|N|N|2420468|2420801|2420133|2420406|N|N|N|N|N| +2420499|AAAAAAAADBPOECAA|1915-01-01|180|783|61|1915|5|1|1|1|1915|61|783|Friday|1915Q1|Y|Y|N|2420499|2420498|2420134|2420407|N|N|N|N|N| +2420500|AAAAAAAAEBPOECAA|1915-01-02|180|783|61|1915|6|1|2|1|1915|61|783|Saturday|1915Q1|N|Y|Y|2420499|2420498|2420135|2420408|N|N|N|N|N| +2420501|AAAAAAAAFBPOECAA|1915-01-03|180|783|61|1915|0|1|3|1|1915|61|783|Sunday|1915Q1|N|N|N|2420499|2420498|2420136|2420409|N|N|N|N|N| +2420502|AAAAAAAAGBPOECAA|1915-01-04|180|783|61|1915|1|1|4|1|1915|61|783|Monday|1915Q1|N|N|N|2420499|2420498|2420137|2420410|N|N|N|N|N| +2420503|AAAAAAAAHBPOECAA|1915-01-05|180|784|61|1915|2|1|5|1|1915|61|784|Tuesday|1915Q1|N|N|N|2420499|2420498|2420138|2420411|N|N|N|N|N| +2420504|AAAAAAAAIBPOECAA|1915-01-06|180|784|61|1915|3|1|6|1|1915|61|784|Wednesday|1915Q1|N|N|N|2420499|2420498|2420139|2420412|N|N|N|N|N| +2420505|AAAAAAAAJBPOECAA|1915-01-07|180|784|61|1915|4|1|7|1|1915|61|784|Thursday|1915Q1|N|N|N|2420499|2420498|2420140|2420413|N|N|N|N|N| +2420506|AAAAAAAAKBPOECAA|1915-01-08|180|784|61|1915|5|1|8|1|1915|61|784|Friday|1915Q1|N|Y|N|2420499|2420498|2420141|2420414|N|N|N|N|N| +2420507|AAAAAAAALBPOECAA|1915-01-09|180|784|61|1915|6|1|9|1|1915|61|784|Saturday|1915Q1|N|Y|N|2420499|2420498|2420142|2420415|N|N|N|N|N| +2420508|AAAAAAAAMBPOECAA|1915-01-10|180|784|61|1915|0|1|10|1|1915|61|784|Sunday|1915Q1|N|N|N|2420499|2420498|2420143|2420416|N|N|N|N|N| +2420509|AAAAAAAANBPOECAA|1915-01-11|180|784|61|1915|1|1|11|1|1915|61|784|Monday|1915Q1|N|N|N|2420499|2420498|2420144|2420417|N|N|N|N|N| +2420510|AAAAAAAAOBPOECAA|1915-01-12|180|785|61|1915|2|1|12|1|1915|61|785|Tuesday|1915Q1|N|N|N|2420499|2420498|2420145|2420418|N|N|N|N|N| +2420511|AAAAAAAAPBPOECAA|1915-01-13|180|785|61|1915|3|1|13|1|1915|61|785|Wednesday|1915Q1|N|N|N|2420499|2420498|2420146|2420419|N|N|N|N|N| +2420512|AAAAAAAAACPOECAA|1915-01-14|180|785|61|1915|4|1|14|1|1915|61|785|Thursday|1915Q1|N|N|N|2420499|2420498|2420147|2420420|N|N|N|N|N| +2420513|AAAAAAAABCPOECAA|1915-01-15|180|785|61|1915|5|1|15|1|1915|61|785|Friday|1915Q1|N|Y|N|2420499|2420498|2420148|2420421|N|N|N|N|N| +2420514|AAAAAAAACCPOECAA|1915-01-16|180|785|61|1915|6|1|16|1|1915|61|785|Saturday|1915Q1|N|Y|N|2420499|2420498|2420149|2420422|N|N|N|N|N| +2420515|AAAAAAAADCPOECAA|1915-01-17|180|785|61|1915|0|1|17|1|1915|61|785|Sunday|1915Q1|N|N|N|2420499|2420498|2420150|2420423|N|N|N|N|N| +2420516|AAAAAAAAECPOECAA|1915-01-18|180|785|61|1915|1|1|18|1|1915|61|785|Monday|1915Q1|N|N|N|2420499|2420498|2420151|2420424|N|N|N|N|N| +2420517|AAAAAAAAFCPOECAA|1915-01-19|180|786|61|1915|2|1|19|1|1915|61|786|Tuesday|1915Q1|N|N|N|2420499|2420498|2420152|2420425|N|N|N|N|N| +2420518|AAAAAAAAGCPOECAA|1915-01-20|180|786|61|1915|3|1|20|1|1915|61|786|Wednesday|1915Q1|N|N|N|2420499|2420498|2420153|2420426|N|N|N|N|N| +2420519|AAAAAAAAHCPOECAA|1915-01-21|180|786|61|1915|4|1|21|1|1915|61|786|Thursday|1915Q1|N|N|N|2420499|2420498|2420154|2420427|N|N|N|N|N| +2420520|AAAAAAAAICPOECAA|1915-01-22|180|786|61|1915|5|1|22|1|1915|61|786|Friday|1915Q1|N|Y|N|2420499|2420498|2420155|2420428|N|N|N|N|N| +2420521|AAAAAAAAJCPOECAA|1915-01-23|180|786|61|1915|6|1|23|1|1915|61|786|Saturday|1915Q1|N|Y|N|2420499|2420498|2420156|2420429|N|N|N|N|N| +2420522|AAAAAAAAKCPOECAA|1915-01-24|180|786|61|1915|0|1|24|1|1915|61|786|Sunday|1915Q1|N|N|N|2420499|2420498|2420157|2420430|N|N|N|N|N| +2420523|AAAAAAAALCPOECAA|1915-01-25|180|786|61|1915|1|1|25|1|1915|61|786|Monday|1915Q1|N|N|N|2420499|2420498|2420158|2420431|N|N|N|N|N| +2420524|AAAAAAAAMCPOECAA|1915-01-26|180|787|61|1915|2|1|26|1|1915|61|787|Tuesday|1915Q1|N|N|N|2420499|2420498|2420159|2420432|N|N|N|N|N| +2420525|AAAAAAAANCPOECAA|1915-01-27|180|787|61|1915|3|1|27|1|1915|61|787|Wednesday|1915Q1|N|N|N|2420499|2420498|2420160|2420433|N|N|N|N|N| +2420526|AAAAAAAAOCPOECAA|1915-01-28|180|787|61|1915|4|1|28|1|1915|61|787|Thursday|1915Q1|N|N|N|2420499|2420498|2420161|2420434|N|N|N|N|N| +2420527|AAAAAAAAPCPOECAA|1915-01-29|180|787|61|1915|5|1|29|1|1915|61|787|Friday|1915Q1|N|Y|N|2420499|2420498|2420162|2420435|N|N|N|N|N| +2420528|AAAAAAAAADPOECAA|1915-01-30|180|787|61|1915|6|1|30|1|1915|61|787|Saturday|1915Q1|N|Y|N|2420499|2420498|2420163|2420436|N|N|N|N|N| +2420529|AAAAAAAABDPOECAA|1915-01-31|180|787|61|1915|0|1|31|1|1915|61|787|Sunday|1915Q1|N|N|N|2420499|2420498|2420164|2420437|N|N|N|N|N| +2420530|AAAAAAAACDPOECAA|1915-02-01|181|787|61|1915|1|2|1|1|1915|61|787|Monday|1915Q1|N|N|N|2420530|2420560|2420165|2420438|N|N|N|N|N| +2420531|AAAAAAAADDPOECAA|1915-02-02|181|788|61|1915|2|2|2|1|1915|61|788|Tuesday|1915Q1|N|N|N|2420530|2420560|2420166|2420439|N|N|N|N|N| +2420532|AAAAAAAAEDPOECAA|1915-02-03|181|788|61|1915|3|2|3|1|1915|61|788|Wednesday|1915Q1|N|N|N|2420530|2420560|2420167|2420440|N|N|N|N|N| +2420533|AAAAAAAAFDPOECAA|1915-02-04|181|788|61|1915|4|2|4|1|1915|61|788|Thursday|1915Q1|N|N|N|2420530|2420560|2420168|2420441|N|N|N|N|N| +2420534|AAAAAAAAGDPOECAA|1915-02-05|181|788|61|1915|5|2|5|1|1915|61|788|Friday|1915Q1|N|Y|N|2420530|2420560|2420169|2420442|N|N|N|N|N| +2420535|AAAAAAAAHDPOECAA|1915-02-06|181|788|61|1915|6|2|6|1|1915|61|788|Saturday|1915Q1|N|Y|N|2420530|2420560|2420170|2420443|N|N|N|N|N| +2420536|AAAAAAAAIDPOECAA|1915-02-07|181|788|61|1915|0|2|7|1|1915|61|788|Sunday|1915Q1|N|N|N|2420530|2420560|2420171|2420444|N|N|N|N|N| +2420537|AAAAAAAAJDPOECAA|1915-02-08|181|788|61|1915|1|2|8|1|1915|61|788|Monday|1915Q1|N|N|N|2420530|2420560|2420172|2420445|N|N|N|N|N| +2420538|AAAAAAAAKDPOECAA|1915-02-09|181|789|61|1915|2|2|9|1|1915|61|789|Tuesday|1915Q1|N|N|N|2420530|2420560|2420173|2420446|N|N|N|N|N| +2420539|AAAAAAAALDPOECAA|1915-02-10|181|789|61|1915|3|2|10|1|1915|61|789|Wednesday|1915Q1|N|N|N|2420530|2420560|2420174|2420447|N|N|N|N|N| +2420540|AAAAAAAAMDPOECAA|1915-02-11|181|789|61|1915|4|2|11|1|1915|61|789|Thursday|1915Q1|N|N|N|2420530|2420560|2420175|2420448|N|N|N|N|N| +2420541|AAAAAAAANDPOECAA|1915-02-12|181|789|61|1915|5|2|12|1|1915|61|789|Friday|1915Q1|N|Y|N|2420530|2420560|2420176|2420449|N|N|N|N|N| +2420542|AAAAAAAAODPOECAA|1915-02-13|181|789|61|1915|6|2|13|1|1915|61|789|Saturday|1915Q1|N|Y|N|2420530|2420560|2420177|2420450|N|N|N|N|N| +2420543|AAAAAAAAPDPOECAA|1915-02-14|181|789|61|1915|0|2|14|1|1915|61|789|Sunday|1915Q1|N|N|N|2420530|2420560|2420178|2420451|N|N|N|N|N| +2420544|AAAAAAAAAEPOECAA|1915-02-15|181|789|61|1915|1|2|15|1|1915|61|789|Monday|1915Q1|N|N|N|2420530|2420560|2420179|2420452|N|N|N|N|N| +2420545|AAAAAAAABEPOECAA|1915-02-16|181|790|61|1915|2|2|16|1|1915|61|790|Tuesday|1915Q1|N|N|N|2420530|2420560|2420180|2420453|N|N|N|N|N| +2420546|AAAAAAAACEPOECAA|1915-02-17|181|790|61|1915|3|2|17|1|1915|61|790|Wednesday|1915Q1|N|N|N|2420530|2420560|2420181|2420454|N|N|N|N|N| +2420547|AAAAAAAADEPOECAA|1915-02-18|181|790|61|1915|4|2|18|1|1915|61|790|Thursday|1915Q1|N|N|N|2420530|2420560|2420182|2420455|N|N|N|N|N| +2420548|AAAAAAAAEEPOECAA|1915-02-19|181|790|61|1915|5|2|19|1|1915|61|790|Friday|1915Q1|N|Y|N|2420530|2420560|2420183|2420456|N|N|N|N|N| +2420549|AAAAAAAAFEPOECAA|1915-02-20|181|790|61|1915|6|2|20|1|1915|61|790|Saturday|1915Q1|N|Y|N|2420530|2420560|2420184|2420457|N|N|N|N|N| +2420550|AAAAAAAAGEPOECAA|1915-02-21|181|790|61|1915|0|2|21|1|1915|61|790|Sunday|1915Q1|N|N|N|2420530|2420560|2420185|2420458|N|N|N|N|N| +2420551|AAAAAAAAHEPOECAA|1915-02-22|181|790|61|1915|1|2|22|1|1915|61|790|Monday|1915Q1|N|N|N|2420530|2420560|2420186|2420459|N|N|N|N|N| +2420552|AAAAAAAAIEPOECAA|1915-02-23|181|791|61|1915|2|2|23|1|1915|61|791|Tuesday|1915Q1|N|N|N|2420530|2420560|2420187|2420460|N|N|N|N|N| +2420553|AAAAAAAAJEPOECAA|1915-02-24|181|791|61|1915|3|2|24|1|1915|61|791|Wednesday|1915Q1|N|N|N|2420530|2420560|2420188|2420461|N|N|N|N|N| +2420554|AAAAAAAAKEPOECAA|1915-02-25|181|791|61|1915|4|2|25|1|1915|61|791|Thursday|1915Q1|N|N|N|2420530|2420560|2420189|2420462|N|N|N|N|N| +2420555|AAAAAAAALEPOECAA|1915-02-26|181|791|61|1915|5|2|26|1|1915|61|791|Friday|1915Q1|N|Y|N|2420530|2420560|2420190|2420463|N|N|N|N|N| +2420556|AAAAAAAAMEPOECAA|1915-02-27|181|791|61|1915|6|2|27|1|1915|61|791|Saturday|1915Q1|N|Y|N|2420530|2420560|2420191|2420464|N|N|N|N|N| +2420557|AAAAAAAANEPOECAA|1915-02-28|181|791|61|1915|0|2|28|1|1915|61|791|Sunday|1915Q1|N|N|N|2420530|2420560|2420192|2420465|N|N|N|N|N| +2420558|AAAAAAAAOEPOECAA|1915-03-01|182|791|62|1915|1|3|1|1|1915|62|791|Monday|1915Q1|N|N|N|2420558|2420616|2420193|2420466|N|N|N|N|N| +2420559|AAAAAAAAPEPOECAA|1915-03-02|182|792|62|1915|2|3|2|1|1915|62|792|Tuesday|1915Q1|N|N|N|2420558|2420616|2420194|2420467|N|N|N|N|N| +2420560|AAAAAAAAAFPOECAA|1915-03-03|182|792|62|1915|3|3|3|1|1915|62|792|Wednesday|1915Q1|N|N|N|2420558|2420616|2420195|2420468|N|N|N|N|N| +2420561|AAAAAAAABFPOECAA|1915-03-04|182|792|62|1915|4|3|4|1|1915|62|792|Thursday|1915Q1|N|N|N|2420558|2420616|2420196|2420469|N|N|N|N|N| +2420562|AAAAAAAACFPOECAA|1915-03-05|182|792|62|1915|5|3|5|1|1915|62|792|Friday|1915Q1|N|Y|N|2420558|2420616|2420197|2420470|N|N|N|N|N| +2420563|AAAAAAAADFPOECAA|1915-03-06|182|792|62|1915|6|3|6|1|1915|62|792|Saturday|1915Q1|N|Y|N|2420558|2420616|2420198|2420471|N|N|N|N|N| +2420564|AAAAAAAAEFPOECAA|1915-03-07|182|792|62|1915|0|3|7|1|1915|62|792|Sunday|1915Q1|N|N|N|2420558|2420616|2420199|2420472|N|N|N|N|N| +2420565|AAAAAAAAFFPOECAA|1915-03-08|182|792|62|1915|1|3|8|1|1915|62|792|Monday|1915Q1|N|N|N|2420558|2420616|2420200|2420473|N|N|N|N|N| +2420566|AAAAAAAAGFPOECAA|1915-03-09|182|793|62|1915|2|3|9|1|1915|62|793|Tuesday|1915Q1|N|N|N|2420558|2420616|2420201|2420474|N|N|N|N|N| +2420567|AAAAAAAAHFPOECAA|1915-03-10|182|793|62|1915|3|3|10|1|1915|62|793|Wednesday|1915Q1|N|N|N|2420558|2420616|2420202|2420475|N|N|N|N|N| +2420568|AAAAAAAAIFPOECAA|1915-03-11|182|793|62|1915|4|3|11|1|1915|62|793|Thursday|1915Q1|N|N|N|2420558|2420616|2420203|2420476|N|N|N|N|N| +2420569|AAAAAAAAJFPOECAA|1915-03-12|182|793|62|1915|5|3|12|1|1915|62|793|Friday|1915Q1|N|Y|N|2420558|2420616|2420204|2420477|N|N|N|N|N| +2420570|AAAAAAAAKFPOECAA|1915-03-13|182|793|62|1915|6|3|13|1|1915|62|793|Saturday|1915Q1|N|Y|N|2420558|2420616|2420205|2420478|N|N|N|N|N| +2420571|AAAAAAAALFPOECAA|1915-03-14|182|793|62|1915|0|3|14|1|1915|62|793|Sunday|1915Q1|N|N|N|2420558|2420616|2420206|2420479|N|N|N|N|N| +2420572|AAAAAAAAMFPOECAA|1915-03-15|182|793|62|1915|1|3|15|1|1915|62|793|Monday|1915Q1|N|N|N|2420558|2420616|2420207|2420480|N|N|N|N|N| +2420573|AAAAAAAANFPOECAA|1915-03-16|182|794|62|1915|2|3|16|1|1915|62|794|Tuesday|1915Q1|N|N|N|2420558|2420616|2420208|2420481|N|N|N|N|N| +2420574|AAAAAAAAOFPOECAA|1915-03-17|182|794|62|1915|3|3|17|1|1915|62|794|Wednesday|1915Q1|N|N|N|2420558|2420616|2420209|2420482|N|N|N|N|N| +2420575|AAAAAAAAPFPOECAA|1915-03-18|182|794|62|1915|4|3|18|1|1915|62|794|Thursday|1915Q1|N|N|N|2420558|2420616|2420210|2420483|N|N|N|N|N| +2420576|AAAAAAAAAGPOECAA|1915-03-19|182|794|62|1915|5|3|19|1|1915|62|794|Friday|1915Q1|N|Y|N|2420558|2420616|2420211|2420484|N|N|N|N|N| +2420577|AAAAAAAABGPOECAA|1915-03-20|182|794|62|1915|6|3|20|1|1915|62|794|Saturday|1915Q1|N|Y|N|2420558|2420616|2420212|2420485|N|N|N|N|N| +2420578|AAAAAAAACGPOECAA|1915-03-21|182|794|62|1915|0|3|21|1|1915|62|794|Sunday|1915Q1|N|N|N|2420558|2420616|2420213|2420486|N|N|N|N|N| +2420579|AAAAAAAADGPOECAA|1915-03-22|182|794|62|1915|1|3|22|1|1915|62|794|Monday|1915Q1|N|N|N|2420558|2420616|2420214|2420487|N|N|N|N|N| +2420580|AAAAAAAAEGPOECAA|1915-03-23|182|795|62|1915|2|3|23|1|1915|62|795|Tuesday|1915Q1|N|N|N|2420558|2420616|2420215|2420488|N|N|N|N|N| +2420581|AAAAAAAAFGPOECAA|1915-03-24|182|795|62|1915|3|3|24|1|1915|62|795|Wednesday|1915Q1|N|N|N|2420558|2420616|2420216|2420489|N|N|N|N|N| +2420582|AAAAAAAAGGPOECAA|1915-03-25|182|795|62|1915|4|3|25|1|1915|62|795|Thursday|1915Q1|N|N|N|2420558|2420616|2420217|2420490|N|N|N|N|N| +2420583|AAAAAAAAHGPOECAA|1915-03-26|182|795|62|1915|5|3|26|1|1915|62|795|Friday|1915Q1|N|Y|N|2420558|2420616|2420218|2420491|N|N|N|N|N| +2420584|AAAAAAAAIGPOECAA|1915-03-27|182|795|62|1915|6|3|27|1|1915|62|795|Saturday|1915Q1|N|Y|N|2420558|2420616|2420219|2420492|N|N|N|N|N| +2420585|AAAAAAAAJGPOECAA|1915-03-28|182|795|62|1915|0|3|28|1|1915|62|795|Sunday|1915Q1|N|N|N|2420558|2420616|2420220|2420493|N|N|N|N|N| +2420586|AAAAAAAAKGPOECAA|1915-03-29|182|795|62|1915|1|3|29|1|1915|62|795|Monday|1915Q1|N|N|N|2420558|2420616|2420221|2420494|N|N|N|N|N| +2420587|AAAAAAAALGPOECAA|1915-03-30|182|796|62|1915|2|3|30|1|1915|62|796|Tuesday|1915Q1|N|N|N|2420558|2420616|2420222|2420495|N|N|N|N|N| +2420588|AAAAAAAAMGPOECAA|1915-03-31|182|796|62|1915|3|3|31|1|1915|62|796|Wednesday|1915Q1|N|N|N|2420558|2420616|2420223|2420496|N|N|N|N|N| +2420589|AAAAAAAANGPOECAA|1915-04-01|183|796|62|1915|4|4|1|1|1915|62|796|Thursday|1915Q1|N|N|N|2420589|2420678|2420224|2420499|N|N|N|N|N| +2420590|AAAAAAAAOGPOECAA|1915-04-02|183|796|62|1915|5|4|2|2|1915|62|796|Friday|1915Q2|N|Y|N|2420589|2420678|2420225|2420500|N|N|N|N|N| +2420591|AAAAAAAAPGPOECAA|1915-04-03|183|796|62|1915|6|4|3|2|1915|62|796|Saturday|1915Q2|N|Y|N|2420589|2420678|2420226|2420501|N|N|N|N|N| +2420592|AAAAAAAAAHPOECAA|1915-04-04|183|796|62|1915|0|4|4|2|1915|62|796|Sunday|1915Q2|N|N|N|2420589|2420678|2420227|2420502|N|N|N|N|N| +2420593|AAAAAAAABHPOECAA|1915-04-05|183|796|62|1915|1|4|5|2|1915|62|796|Monday|1915Q2|N|N|N|2420589|2420678|2420228|2420503|N|N|N|N|N| +2420594|AAAAAAAACHPOECAA|1915-04-06|183|797|62|1915|2|4|6|2|1915|62|797|Tuesday|1915Q2|N|N|N|2420589|2420678|2420229|2420504|N|N|N|N|N| +2420595|AAAAAAAADHPOECAA|1915-04-07|183|797|62|1915|3|4|7|2|1915|62|797|Wednesday|1915Q2|N|N|N|2420589|2420678|2420230|2420505|N|N|N|N|N| +2420596|AAAAAAAAEHPOECAA|1915-04-08|183|797|62|1915|4|4|8|2|1915|62|797|Thursday|1915Q2|N|N|N|2420589|2420678|2420231|2420506|N|N|N|N|N| +2420597|AAAAAAAAFHPOECAA|1915-04-09|183|797|62|1915|5|4|9|2|1915|62|797|Friday|1915Q2|N|Y|N|2420589|2420678|2420232|2420507|N|N|N|N|N| +2420598|AAAAAAAAGHPOECAA|1915-04-10|183|797|62|1915|6|4|10|2|1915|62|797|Saturday|1915Q2|N|Y|N|2420589|2420678|2420233|2420508|N|N|N|N|N| +2420599|AAAAAAAAHHPOECAA|1915-04-11|183|797|62|1915|0|4|11|2|1915|62|797|Sunday|1915Q2|N|N|N|2420589|2420678|2420234|2420509|N|N|N|N|N| +2420600|AAAAAAAAIHPOECAA|1915-04-12|183|797|62|1915|1|4|12|2|1915|62|797|Monday|1915Q2|N|N|N|2420589|2420678|2420235|2420510|N|N|N|N|N| +2420601|AAAAAAAAJHPOECAA|1915-04-13|183|798|62|1915|2|4|13|2|1915|62|798|Tuesday|1915Q2|N|N|N|2420589|2420678|2420236|2420511|N|N|N|N|N| +2420602|AAAAAAAAKHPOECAA|1915-04-14|183|798|62|1915|3|4|14|2|1915|62|798|Wednesday|1915Q2|N|N|N|2420589|2420678|2420237|2420512|N|N|N|N|N| +2420603|AAAAAAAALHPOECAA|1915-04-15|183|798|62|1915|4|4|15|2|1915|62|798|Thursday|1915Q2|N|N|N|2420589|2420678|2420238|2420513|N|N|N|N|N| +2420604|AAAAAAAAMHPOECAA|1915-04-16|183|798|62|1915|5|4|16|2|1915|62|798|Friday|1915Q2|N|Y|N|2420589|2420678|2420239|2420514|N|N|N|N|N| +2420605|AAAAAAAANHPOECAA|1915-04-17|183|798|62|1915|6|4|17|2|1915|62|798|Saturday|1915Q2|N|Y|N|2420589|2420678|2420240|2420515|N|N|N|N|N| +2420606|AAAAAAAAOHPOECAA|1915-04-18|183|798|62|1915|0|4|18|2|1915|62|798|Sunday|1915Q2|N|N|N|2420589|2420678|2420241|2420516|N|N|N|N|N| +2420607|AAAAAAAAPHPOECAA|1915-04-19|183|798|62|1915|1|4|19|2|1915|62|798|Monday|1915Q2|N|N|N|2420589|2420678|2420242|2420517|N|N|N|N|N| +2420608|AAAAAAAAAIPOECAA|1915-04-20|183|799|62|1915|2|4|20|2|1915|62|799|Tuesday|1915Q2|N|N|N|2420589|2420678|2420243|2420518|N|N|N|N|N| +2420609|AAAAAAAABIPOECAA|1915-04-21|183|799|62|1915|3|4|21|2|1915|62|799|Wednesday|1915Q2|N|N|N|2420589|2420678|2420244|2420519|N|N|N|N|N| +2420610|AAAAAAAACIPOECAA|1915-04-22|183|799|62|1915|4|4|22|2|1915|62|799|Thursday|1915Q2|N|N|N|2420589|2420678|2420245|2420520|N|N|N|N|N| +2420611|AAAAAAAADIPOECAA|1915-04-23|183|799|62|1915|5|4|23|2|1915|62|799|Friday|1915Q2|N|Y|N|2420589|2420678|2420246|2420521|N|N|N|N|N| +2420612|AAAAAAAAEIPOECAA|1915-04-24|183|799|62|1915|6|4|24|2|1915|62|799|Saturday|1915Q2|N|Y|N|2420589|2420678|2420247|2420522|N|N|N|N|N| +2420613|AAAAAAAAFIPOECAA|1915-04-25|183|799|62|1915|0|4|25|2|1915|62|799|Sunday|1915Q2|N|N|N|2420589|2420678|2420248|2420523|N|N|N|N|N| +2420614|AAAAAAAAGIPOECAA|1915-04-26|183|799|62|1915|1|4|26|2|1915|62|799|Monday|1915Q2|N|N|N|2420589|2420678|2420249|2420524|N|N|N|N|N| +2420615|AAAAAAAAHIPOECAA|1915-04-27|183|800|62|1915|2|4|27|2|1915|62|800|Tuesday|1915Q2|N|N|N|2420589|2420678|2420250|2420525|N|N|N|N|N| +2420616|AAAAAAAAIIPOECAA|1915-04-28|183|800|62|1915|3|4|28|2|1915|62|800|Wednesday|1915Q2|N|N|N|2420589|2420678|2420251|2420526|N|N|N|N|N| +2420617|AAAAAAAAJIPOECAA|1915-04-29|183|800|62|1915|4|4|29|2|1915|62|800|Thursday|1915Q2|N|N|N|2420589|2420678|2420252|2420527|N|N|N|N|N| +2420618|AAAAAAAAKIPOECAA|1915-04-30|183|800|62|1915|5|4|30|2|1915|62|800|Friday|1915Q2|N|Y|N|2420589|2420678|2420253|2420528|N|N|N|N|N| +2420619|AAAAAAAALIPOECAA|1915-05-01|184|800|62|1915|6|5|1|2|1915|62|800|Saturday|1915Q2|N|Y|N|2420619|2420738|2420254|2420529|N|N|N|N|N| +2420620|AAAAAAAAMIPOECAA|1915-05-02|184|800|62|1915|0|5|2|2|1915|62|800|Sunday|1915Q2|N|N|N|2420619|2420738|2420255|2420530|N|N|N|N|N| +2420621|AAAAAAAANIPOECAA|1915-05-03|184|800|62|1915|1|5|3|2|1915|62|800|Monday|1915Q2|N|N|N|2420619|2420738|2420256|2420531|N|N|N|N|N| +2420622|AAAAAAAAOIPOECAA|1915-05-04|184|801|62|1915|2|5|4|2|1915|62|801|Tuesday|1915Q2|N|N|N|2420619|2420738|2420257|2420532|N|N|N|N|N| +2420623|AAAAAAAAPIPOECAA|1915-05-05|184|801|62|1915|3|5|5|2|1915|62|801|Wednesday|1915Q2|N|N|N|2420619|2420738|2420258|2420533|N|N|N|N|N| +2420624|AAAAAAAAAJPOECAA|1915-05-06|184|801|62|1915|4|5|6|2|1915|62|801|Thursday|1915Q2|N|N|N|2420619|2420738|2420259|2420534|N|N|N|N|N| +2420625|AAAAAAAABJPOECAA|1915-05-07|184|801|62|1915|5|5|7|2|1915|62|801|Friday|1915Q2|N|Y|N|2420619|2420738|2420260|2420535|N|N|N|N|N| +2420626|AAAAAAAACJPOECAA|1915-05-08|184|801|62|1915|6|5|8|2|1915|62|801|Saturday|1915Q2|N|Y|N|2420619|2420738|2420261|2420536|N|N|N|N|N| +2420627|AAAAAAAADJPOECAA|1915-05-09|184|801|62|1915|0|5|9|2|1915|62|801|Sunday|1915Q2|N|N|N|2420619|2420738|2420262|2420537|N|N|N|N|N| +2420628|AAAAAAAAEJPOECAA|1915-05-10|184|801|62|1915|1|5|10|2|1915|62|801|Monday|1915Q2|N|N|N|2420619|2420738|2420263|2420538|N|N|N|N|N| +2420629|AAAAAAAAFJPOECAA|1915-05-11|184|802|62|1915|2|5|11|2|1915|62|802|Tuesday|1915Q2|N|N|N|2420619|2420738|2420264|2420539|N|N|N|N|N| +2420630|AAAAAAAAGJPOECAA|1915-05-12|184|802|62|1915|3|5|12|2|1915|62|802|Wednesday|1915Q2|N|N|N|2420619|2420738|2420265|2420540|N|N|N|N|N| +2420631|AAAAAAAAHJPOECAA|1915-05-13|184|802|62|1915|4|5|13|2|1915|62|802|Thursday|1915Q2|N|N|N|2420619|2420738|2420266|2420541|N|N|N|N|N| +2420632|AAAAAAAAIJPOECAA|1915-05-14|184|802|62|1915|5|5|14|2|1915|62|802|Friday|1915Q2|N|Y|N|2420619|2420738|2420267|2420542|N|N|N|N|N| +2420633|AAAAAAAAJJPOECAA|1915-05-15|184|802|62|1915|6|5|15|2|1915|62|802|Saturday|1915Q2|N|Y|N|2420619|2420738|2420268|2420543|N|N|N|N|N| +2420634|AAAAAAAAKJPOECAA|1915-05-16|184|802|62|1915|0|5|16|2|1915|62|802|Sunday|1915Q2|N|N|N|2420619|2420738|2420269|2420544|N|N|N|N|N| +2420635|AAAAAAAALJPOECAA|1915-05-17|184|802|62|1915|1|5|17|2|1915|62|802|Monday|1915Q2|N|N|N|2420619|2420738|2420270|2420545|N|N|N|N|N| +2420636|AAAAAAAAMJPOECAA|1915-05-18|184|803|62|1915|2|5|18|2|1915|62|803|Tuesday|1915Q2|N|N|N|2420619|2420738|2420271|2420546|N|N|N|N|N| +2420637|AAAAAAAANJPOECAA|1915-05-19|184|803|62|1915|3|5|19|2|1915|62|803|Wednesday|1915Q2|N|N|N|2420619|2420738|2420272|2420547|N|N|N|N|N| +2420638|AAAAAAAAOJPOECAA|1915-05-20|184|803|62|1915|4|5|20|2|1915|62|803|Thursday|1915Q2|N|N|N|2420619|2420738|2420273|2420548|N|N|N|N|N| +2420639|AAAAAAAAPJPOECAA|1915-05-21|184|803|62|1915|5|5|21|2|1915|62|803|Friday|1915Q2|N|Y|N|2420619|2420738|2420274|2420549|N|N|N|N|N| +2420640|AAAAAAAAAKPOECAA|1915-05-22|184|803|62|1915|6|5|22|2|1915|62|803|Saturday|1915Q2|N|Y|N|2420619|2420738|2420275|2420550|N|N|N|N|N| +2420641|AAAAAAAABKPOECAA|1915-05-23|184|803|62|1915|0|5|23|2|1915|62|803|Sunday|1915Q2|N|N|N|2420619|2420738|2420276|2420551|N|N|N|N|N| +2420642|AAAAAAAACKPOECAA|1915-05-24|184|803|62|1915|1|5|24|2|1915|62|803|Monday|1915Q2|N|N|N|2420619|2420738|2420277|2420552|N|N|N|N|N| +2420643|AAAAAAAADKPOECAA|1915-05-25|184|804|62|1915|2|5|25|2|1915|62|804|Tuesday|1915Q2|N|N|N|2420619|2420738|2420278|2420553|N|N|N|N|N| +2420644|AAAAAAAAEKPOECAA|1915-05-26|184|804|62|1915|3|5|26|2|1915|62|804|Wednesday|1915Q2|N|N|N|2420619|2420738|2420279|2420554|N|N|N|N|N| +2420645|AAAAAAAAFKPOECAA|1915-05-27|184|804|62|1915|4|5|27|2|1915|62|804|Thursday|1915Q2|N|N|N|2420619|2420738|2420280|2420555|N|N|N|N|N| +2420646|AAAAAAAAGKPOECAA|1915-05-28|184|804|62|1915|5|5|28|2|1915|62|804|Friday|1915Q2|N|Y|N|2420619|2420738|2420281|2420556|N|N|N|N|N| +2420647|AAAAAAAAHKPOECAA|1915-05-29|184|804|62|1915|6|5|29|2|1915|62|804|Saturday|1915Q2|N|Y|N|2420619|2420738|2420282|2420557|N|N|N|N|N| +2420648|AAAAAAAAIKPOECAA|1915-05-30|184|804|62|1915|0|5|30|2|1915|62|804|Sunday|1915Q2|N|N|N|2420619|2420738|2420283|2420558|N|N|N|N|N| +2420649|AAAAAAAAJKPOECAA|1915-05-31|184|804|62|1915|1|5|31|2|1915|62|804|Monday|1915Q2|N|N|N|2420619|2420738|2420284|2420559|N|N|N|N|N| +2420650|AAAAAAAAKKPOECAA|1915-06-01|185|805|63|1915|2|6|1|2|1915|63|805|Tuesday|1915Q2|N|N|N|2420650|2420800|2420285|2420560|N|N|N|N|N| +2420651|AAAAAAAALKPOECAA|1915-06-02|185|805|63|1915|3|6|2|2|1915|63|805|Wednesday|1915Q2|N|N|N|2420650|2420800|2420286|2420561|N|N|N|N|N| +2420652|AAAAAAAAMKPOECAA|1915-06-03|185|805|63|1915|4|6|3|2|1915|63|805|Thursday|1915Q2|N|N|N|2420650|2420800|2420287|2420562|N|N|N|N|N| +2420653|AAAAAAAANKPOECAA|1915-06-04|185|805|63|1915|5|6|4|2|1915|63|805|Friday|1915Q2|N|Y|N|2420650|2420800|2420288|2420563|N|N|N|N|N| +2420654|AAAAAAAAOKPOECAA|1915-06-05|185|805|63|1915|6|6|5|2|1915|63|805|Saturday|1915Q2|N|Y|N|2420650|2420800|2420289|2420564|N|N|N|N|N| +2420655|AAAAAAAAPKPOECAA|1915-06-06|185|805|63|1915|0|6|6|2|1915|63|805|Sunday|1915Q2|N|N|N|2420650|2420800|2420290|2420565|N|N|N|N|N| +2420656|AAAAAAAAALPOECAA|1915-06-07|185|805|63|1915|1|6|7|2|1915|63|805|Monday|1915Q2|N|N|N|2420650|2420800|2420291|2420566|N|N|N|N|N| +2420657|AAAAAAAABLPOECAA|1915-06-08|185|806|63|1915|2|6|8|2|1915|63|806|Tuesday|1915Q2|N|N|N|2420650|2420800|2420292|2420567|N|N|N|N|N| +2420658|AAAAAAAACLPOECAA|1915-06-09|185|806|63|1915|3|6|9|2|1915|63|806|Wednesday|1915Q2|N|N|N|2420650|2420800|2420293|2420568|N|N|N|N|N| +2420659|AAAAAAAADLPOECAA|1915-06-10|185|806|63|1915|4|6|10|2|1915|63|806|Thursday|1915Q2|N|N|N|2420650|2420800|2420294|2420569|N|N|N|N|N| +2420660|AAAAAAAAELPOECAA|1915-06-11|185|806|63|1915|5|6|11|2|1915|63|806|Friday|1915Q2|N|Y|N|2420650|2420800|2420295|2420570|N|N|N|N|N| +2420661|AAAAAAAAFLPOECAA|1915-06-12|185|806|63|1915|6|6|12|2|1915|63|806|Saturday|1915Q2|N|Y|N|2420650|2420800|2420296|2420571|N|N|N|N|N| +2420662|AAAAAAAAGLPOECAA|1915-06-13|185|806|63|1915|0|6|13|2|1915|63|806|Sunday|1915Q2|N|N|N|2420650|2420800|2420297|2420572|N|N|N|N|N| +2420663|AAAAAAAAHLPOECAA|1915-06-14|185|806|63|1915|1|6|14|2|1915|63|806|Monday|1915Q2|N|N|N|2420650|2420800|2420298|2420573|N|N|N|N|N| +2420664|AAAAAAAAILPOECAA|1915-06-15|185|807|63|1915|2|6|15|2|1915|63|807|Tuesday|1915Q2|N|N|N|2420650|2420800|2420299|2420574|N|N|N|N|N| +2420665|AAAAAAAAJLPOECAA|1915-06-16|185|807|63|1915|3|6|16|2|1915|63|807|Wednesday|1915Q2|N|N|N|2420650|2420800|2420300|2420575|N|N|N|N|N| +2420666|AAAAAAAAKLPOECAA|1915-06-17|185|807|63|1915|4|6|17|2|1915|63|807|Thursday|1915Q2|N|N|N|2420650|2420800|2420301|2420576|N|N|N|N|N| +2420667|AAAAAAAALLPOECAA|1915-06-18|185|807|63|1915|5|6|18|2|1915|63|807|Friday|1915Q2|N|Y|N|2420650|2420800|2420302|2420577|N|N|N|N|N| +2420668|AAAAAAAAMLPOECAA|1915-06-19|185|807|63|1915|6|6|19|2|1915|63|807|Saturday|1915Q2|N|Y|N|2420650|2420800|2420303|2420578|N|N|N|N|N| +2420669|AAAAAAAANLPOECAA|1915-06-20|185|807|63|1915|0|6|20|2|1915|63|807|Sunday|1915Q2|N|N|N|2420650|2420800|2420304|2420579|N|N|N|N|N| +2420670|AAAAAAAAOLPOECAA|1915-06-21|185|807|63|1915|1|6|21|2|1915|63|807|Monday|1915Q2|N|N|N|2420650|2420800|2420305|2420580|N|N|N|N|N| +2420671|AAAAAAAAPLPOECAA|1915-06-22|185|808|63|1915|2|6|22|2|1915|63|808|Tuesday|1915Q2|N|N|N|2420650|2420800|2420306|2420581|N|N|N|N|N| +2420672|AAAAAAAAAMPOECAA|1915-06-23|185|808|63|1915|3|6|23|2|1915|63|808|Wednesday|1915Q2|N|N|N|2420650|2420800|2420307|2420582|N|N|N|N|N| +2420673|AAAAAAAABMPOECAA|1915-06-24|185|808|63|1915|4|6|24|2|1915|63|808|Thursday|1915Q2|N|N|N|2420650|2420800|2420308|2420583|N|N|N|N|N| +2420674|AAAAAAAACMPOECAA|1915-06-25|185|808|63|1915|5|6|25|2|1915|63|808|Friday|1915Q2|N|Y|N|2420650|2420800|2420309|2420584|N|N|N|N|N| +2420675|AAAAAAAADMPOECAA|1915-06-26|185|808|63|1915|6|6|26|2|1915|63|808|Saturday|1915Q2|N|Y|N|2420650|2420800|2420310|2420585|N|N|N|N|N| +2420676|AAAAAAAAEMPOECAA|1915-06-27|185|808|63|1915|0|6|27|2|1915|63|808|Sunday|1915Q2|N|N|N|2420650|2420800|2420311|2420586|N|N|N|N|N| +2420677|AAAAAAAAFMPOECAA|1915-06-28|185|808|63|1915|1|6|28|2|1915|63|808|Monday|1915Q2|N|N|N|2420650|2420800|2420312|2420587|N|N|N|N|N| +2420678|AAAAAAAAGMPOECAA|1915-06-29|185|809|63|1915|2|6|29|2|1915|63|809|Tuesday|1915Q2|N|N|N|2420650|2420800|2420313|2420588|N|N|N|N|N| +2420679|AAAAAAAAHMPOECAA|1915-06-30|185|809|63|1915|3|6|30|2|1915|63|809|Wednesday|1915Q2|N|N|N|2420650|2420800|2420314|2420589|N|N|N|N|N| +2420680|AAAAAAAAIMPOECAA|1915-07-01|186|809|63|1915|4|7|1|2|1915|63|809|Thursday|1915Q2|N|N|N|2420680|2420860|2420315|2420589|N|N|N|N|N| +2420681|AAAAAAAAJMPOECAA|1915-07-02|186|809|63|1915|5|7|2|3|1915|63|809|Friday|1915Q3|N|Y|N|2420680|2420860|2420316|2420590|N|N|N|N|N| +2420682|AAAAAAAAKMPOECAA|1915-07-03|186|809|63|1915|6|7|3|3|1915|63|809|Saturday|1915Q3|N|Y|N|2420680|2420860|2420317|2420591|N|N|N|N|N| +2420683|AAAAAAAALMPOECAA|1915-07-04|186|809|63|1915|0|7|4|3|1915|63|809|Sunday|1915Q3|N|N|N|2420680|2420860|2420318|2420592|N|N|N|N|N| +2420684|AAAAAAAAMMPOECAA|1915-07-05|186|809|63|1915|1|7|5|3|1915|63|809|Monday|1915Q3|Y|N|N|2420680|2420860|2420319|2420593|N|N|N|N|N| +2420685|AAAAAAAANMPOECAA|1915-07-06|186|810|63|1915|2|7|6|3|1915|63|810|Tuesday|1915Q3|N|N|Y|2420680|2420860|2420320|2420594|N|N|N|N|N| +2420686|AAAAAAAAOMPOECAA|1915-07-07|186|810|63|1915|3|7|7|3|1915|63|810|Wednesday|1915Q3|N|N|N|2420680|2420860|2420321|2420595|N|N|N|N|N| +2420687|AAAAAAAAPMPOECAA|1915-07-08|186|810|63|1915|4|7|8|3|1915|63|810|Thursday|1915Q3|N|N|N|2420680|2420860|2420322|2420596|N|N|N|N|N| +2420688|AAAAAAAAANPOECAA|1915-07-09|186|810|63|1915|5|7|9|3|1915|63|810|Friday|1915Q3|N|Y|N|2420680|2420860|2420323|2420597|N|N|N|N|N| +2420689|AAAAAAAABNPOECAA|1915-07-10|186|810|63|1915|6|7|10|3|1915|63|810|Saturday|1915Q3|N|Y|N|2420680|2420860|2420324|2420598|N|N|N|N|N| +2420690|AAAAAAAACNPOECAA|1915-07-11|186|810|63|1915|0|7|11|3|1915|63|810|Sunday|1915Q3|N|N|N|2420680|2420860|2420325|2420599|N|N|N|N|N| +2420691|AAAAAAAADNPOECAA|1915-07-12|186|810|63|1915|1|7|12|3|1915|63|810|Monday|1915Q3|N|N|N|2420680|2420860|2420326|2420600|N|N|N|N|N| +2420692|AAAAAAAAENPOECAA|1915-07-13|186|811|63|1915|2|7|13|3|1915|63|811|Tuesday|1915Q3|N|N|N|2420680|2420860|2420327|2420601|N|N|N|N|N| +2420693|AAAAAAAAFNPOECAA|1915-07-14|186|811|63|1915|3|7|14|3|1915|63|811|Wednesday|1915Q3|N|N|N|2420680|2420860|2420328|2420602|N|N|N|N|N| +2420694|AAAAAAAAGNPOECAA|1915-07-15|186|811|63|1915|4|7|15|3|1915|63|811|Thursday|1915Q3|N|N|N|2420680|2420860|2420329|2420603|N|N|N|N|N| +2420695|AAAAAAAAHNPOECAA|1915-07-16|186|811|63|1915|5|7|16|3|1915|63|811|Friday|1915Q3|N|Y|N|2420680|2420860|2420330|2420604|N|N|N|N|N| +2420696|AAAAAAAAINPOECAA|1915-07-17|186|811|63|1915|6|7|17|3|1915|63|811|Saturday|1915Q3|N|Y|N|2420680|2420860|2420331|2420605|N|N|N|N|N| +2420697|AAAAAAAAJNPOECAA|1915-07-18|186|811|63|1915|0|7|18|3|1915|63|811|Sunday|1915Q3|N|N|N|2420680|2420860|2420332|2420606|N|N|N|N|N| +2420698|AAAAAAAAKNPOECAA|1915-07-19|186|811|63|1915|1|7|19|3|1915|63|811|Monday|1915Q3|N|N|N|2420680|2420860|2420333|2420607|N|N|N|N|N| +2420699|AAAAAAAALNPOECAA|1915-07-20|186|812|63|1915|2|7|20|3|1915|63|812|Tuesday|1915Q3|N|N|N|2420680|2420860|2420334|2420608|N|N|N|N|N| +2420700|AAAAAAAAMNPOECAA|1915-07-21|186|812|63|1915|3|7|21|3|1915|63|812|Wednesday|1915Q3|N|N|N|2420680|2420860|2420335|2420609|N|N|N|N|N| +2420701|AAAAAAAANNPOECAA|1915-07-22|186|812|63|1915|4|7|22|3|1915|63|812|Thursday|1915Q3|N|N|N|2420680|2420860|2420336|2420610|N|N|N|N|N| +2420702|AAAAAAAAONPOECAA|1915-07-23|186|812|63|1915|5|7|23|3|1915|63|812|Friday|1915Q3|N|Y|N|2420680|2420860|2420337|2420611|N|N|N|N|N| +2420703|AAAAAAAAPNPOECAA|1915-07-24|186|812|63|1915|6|7|24|3|1915|63|812|Saturday|1915Q3|N|Y|N|2420680|2420860|2420338|2420612|N|N|N|N|N| +2420704|AAAAAAAAAOPOECAA|1915-07-25|186|812|63|1915|0|7|25|3|1915|63|812|Sunday|1915Q3|N|N|N|2420680|2420860|2420339|2420613|N|N|N|N|N| +2420705|AAAAAAAABOPOECAA|1915-07-26|186|812|63|1915|1|7|26|3|1915|63|812|Monday|1915Q3|N|N|N|2420680|2420860|2420340|2420614|N|N|N|N|N| +2420706|AAAAAAAACOPOECAA|1915-07-27|186|813|63|1915|2|7|27|3|1915|63|813|Tuesday|1915Q3|N|N|N|2420680|2420860|2420341|2420615|N|N|N|N|N| +2420707|AAAAAAAADOPOECAA|1915-07-28|186|813|63|1915|3|7|28|3|1915|63|813|Wednesday|1915Q3|N|N|N|2420680|2420860|2420342|2420616|N|N|N|N|N| +2420708|AAAAAAAAEOPOECAA|1915-07-29|186|813|63|1915|4|7|29|3|1915|63|813|Thursday|1915Q3|N|N|N|2420680|2420860|2420343|2420617|N|N|N|N|N| +2420709|AAAAAAAAFOPOECAA|1915-07-30|186|813|63|1915|5|7|30|3|1915|63|813|Friday|1915Q3|N|Y|N|2420680|2420860|2420344|2420618|N|N|N|N|N| +2420710|AAAAAAAAGOPOECAA|1915-07-31|186|813|63|1915|6|7|31|3|1915|63|813|Saturday|1915Q3|N|Y|N|2420680|2420860|2420345|2420619|N|N|N|N|N| +2420711|AAAAAAAAHOPOECAA|1915-08-01|187|813|63|1915|0|8|1|3|1915|63|813|Sunday|1915Q3|N|N|N|2420711|2420922|2420346|2420620|N|N|N|N|N| +2420712|AAAAAAAAIOPOECAA|1915-08-02|187|813|63|1915|1|8|2|3|1915|63|813|Monday|1915Q3|N|N|N|2420711|2420922|2420347|2420621|N|N|N|N|N| +2420713|AAAAAAAAJOPOECAA|1915-08-03|187|814|63|1915|2|8|3|3|1915|63|814|Tuesday|1915Q3|N|N|N|2420711|2420922|2420348|2420622|N|N|N|N|N| +2420714|AAAAAAAAKOPOECAA|1915-08-04|187|814|63|1915|3|8|4|3|1915|63|814|Wednesday|1915Q3|N|N|N|2420711|2420922|2420349|2420623|N|N|N|N|N| +2420715|AAAAAAAALOPOECAA|1915-08-05|187|814|63|1915|4|8|5|3|1915|63|814|Thursday|1915Q3|N|N|N|2420711|2420922|2420350|2420624|N|N|N|N|N| +2420716|AAAAAAAAMOPOECAA|1915-08-06|187|814|63|1915|5|8|6|3|1915|63|814|Friday|1915Q3|N|Y|N|2420711|2420922|2420351|2420625|N|N|N|N|N| +2420717|AAAAAAAANOPOECAA|1915-08-07|187|814|63|1915|6|8|7|3|1915|63|814|Saturday|1915Q3|N|Y|N|2420711|2420922|2420352|2420626|N|N|N|N|N| +2420718|AAAAAAAAOOPOECAA|1915-08-08|187|814|63|1915|0|8|8|3|1915|63|814|Sunday|1915Q3|N|N|N|2420711|2420922|2420353|2420627|N|N|N|N|N| +2420719|AAAAAAAAPOPOECAA|1915-08-09|187|814|63|1915|1|8|9|3|1915|63|814|Monday|1915Q3|N|N|N|2420711|2420922|2420354|2420628|N|N|N|N|N| +2420720|AAAAAAAAAPPOECAA|1915-08-10|187|815|63|1915|2|8|10|3|1915|63|815|Tuesday|1915Q3|N|N|N|2420711|2420922|2420355|2420629|N|N|N|N|N| +2420721|AAAAAAAABPPOECAA|1915-08-11|187|815|63|1915|3|8|11|3|1915|63|815|Wednesday|1915Q3|N|N|N|2420711|2420922|2420356|2420630|N|N|N|N|N| +2420722|AAAAAAAACPPOECAA|1915-08-12|187|815|63|1915|4|8|12|3|1915|63|815|Thursday|1915Q3|N|N|N|2420711|2420922|2420357|2420631|N|N|N|N|N| +2420723|AAAAAAAADPPOECAA|1915-08-13|187|815|63|1915|5|8|13|3|1915|63|815|Friday|1915Q3|N|Y|N|2420711|2420922|2420358|2420632|N|N|N|N|N| +2420724|AAAAAAAAEPPOECAA|1915-08-14|187|815|63|1915|6|8|14|3|1915|63|815|Saturday|1915Q3|N|Y|N|2420711|2420922|2420359|2420633|N|N|N|N|N| +2420725|AAAAAAAAFPPOECAA|1915-08-15|187|815|63|1915|0|8|15|3|1915|63|815|Sunday|1915Q3|N|N|N|2420711|2420922|2420360|2420634|N|N|N|N|N| +2420726|AAAAAAAAGPPOECAA|1915-08-16|187|815|63|1915|1|8|16|3|1915|63|815|Monday|1915Q3|N|N|N|2420711|2420922|2420361|2420635|N|N|N|N|N| +2420727|AAAAAAAAHPPOECAA|1915-08-17|187|816|63|1915|2|8|17|3|1915|63|816|Tuesday|1915Q3|N|N|N|2420711|2420922|2420362|2420636|N|N|N|N|N| +2420728|AAAAAAAAIPPOECAA|1915-08-18|187|816|63|1915|3|8|18|3|1915|63|816|Wednesday|1915Q3|N|N|N|2420711|2420922|2420363|2420637|N|N|N|N|N| +2420729|AAAAAAAAJPPOECAA|1915-08-19|187|816|63|1915|4|8|19|3|1915|63|816|Thursday|1915Q3|N|N|N|2420711|2420922|2420364|2420638|N|N|N|N|N| +2420730|AAAAAAAAKPPOECAA|1915-08-20|187|816|63|1915|5|8|20|3|1915|63|816|Friday|1915Q3|N|Y|N|2420711|2420922|2420365|2420639|N|N|N|N|N| +2420731|AAAAAAAALPPOECAA|1915-08-21|187|816|63|1915|6|8|21|3|1915|63|816|Saturday|1915Q3|N|Y|N|2420711|2420922|2420366|2420640|N|N|N|N|N| +2420732|AAAAAAAAMPPOECAA|1915-08-22|187|816|63|1915|0|8|22|3|1915|63|816|Sunday|1915Q3|N|N|N|2420711|2420922|2420367|2420641|N|N|N|N|N| +2420733|AAAAAAAANPPOECAA|1915-08-23|187|816|63|1915|1|8|23|3|1915|63|816|Monday|1915Q3|N|N|N|2420711|2420922|2420368|2420642|N|N|N|N|N| +2420734|AAAAAAAAOPPOECAA|1915-08-24|187|817|63|1915|2|8|24|3|1915|63|817|Tuesday|1915Q3|N|N|N|2420711|2420922|2420369|2420643|N|N|N|N|N| +2420735|AAAAAAAAPPPOECAA|1915-08-25|187|817|63|1915|3|8|25|3|1915|63|817|Wednesday|1915Q3|N|N|N|2420711|2420922|2420370|2420644|N|N|N|N|N| +2420736|AAAAAAAAAAAPECAA|1915-08-26|187|817|63|1915|4|8|26|3|1915|63|817|Thursday|1915Q3|N|N|N|2420711|2420922|2420371|2420645|N|N|N|N|N| +2420737|AAAAAAAABAAPECAA|1915-08-27|187|817|63|1915|5|8|27|3|1915|63|817|Friday|1915Q3|N|Y|N|2420711|2420922|2420372|2420646|N|N|N|N|N| +2420738|AAAAAAAACAAPECAA|1915-08-28|187|817|63|1915|6|8|28|3|1915|63|817|Saturday|1915Q3|N|Y|N|2420711|2420922|2420373|2420647|N|N|N|N|N| +2420739|AAAAAAAADAAPECAA|1915-08-29|187|817|63|1915|0|8|29|3|1915|63|817|Sunday|1915Q3|N|N|N|2420711|2420922|2420374|2420648|N|N|N|N|N| +2420740|AAAAAAAAEAAPECAA|1915-08-30|187|817|63|1915|1|8|30|3|1915|63|817|Monday|1915Q3|N|N|N|2420711|2420922|2420375|2420649|N|N|N|N|N| +2420741|AAAAAAAAFAAPECAA|1915-08-31|187|818|63|1915|2|8|31|3|1915|63|818|Tuesday|1915Q3|N|N|N|2420711|2420922|2420376|2420650|N|N|N|N|N| +2420742|AAAAAAAAGAAPECAA|1915-09-01|188|818|64|1915|3|9|1|3|1915|64|818|Wednesday|1915Q3|N|N|N|2420742|2420984|2420377|2420651|N|N|N|N|N| +2420743|AAAAAAAAHAAPECAA|1915-09-02|188|818|64|1915|4|9|2|3|1915|64|818|Thursday|1915Q3|N|N|N|2420742|2420984|2420378|2420652|N|N|N|N|N| +2420744|AAAAAAAAIAAPECAA|1915-09-03|188|818|64|1915|5|9|3|3|1915|64|818|Friday|1915Q3|N|Y|N|2420742|2420984|2420379|2420653|N|N|N|N|N| +2420745|AAAAAAAAJAAPECAA|1915-09-04|188|818|64|1915|6|9|4|3|1915|64|818|Saturday|1915Q3|N|Y|N|2420742|2420984|2420380|2420654|N|N|N|N|N| +2420746|AAAAAAAAKAAPECAA|1915-09-05|188|818|64|1915|0|9|5|3|1915|64|818|Sunday|1915Q3|N|N|N|2420742|2420984|2420381|2420655|N|N|N|N|N| +2420747|AAAAAAAALAAPECAA|1915-09-06|188|818|64|1915|1|9|6|3|1915|64|818|Monday|1915Q3|N|N|N|2420742|2420984|2420382|2420656|N|N|N|N|N| +2420748|AAAAAAAAMAAPECAA|1915-09-07|188|819|64|1915|2|9|7|3|1915|64|819|Tuesday|1915Q3|N|N|N|2420742|2420984|2420383|2420657|N|N|N|N|N| +2420749|AAAAAAAANAAPECAA|1915-09-08|188|819|64|1915|3|9|8|3|1915|64|819|Wednesday|1915Q3|N|N|N|2420742|2420984|2420384|2420658|N|N|N|N|N| +2420750|AAAAAAAAOAAPECAA|1915-09-09|188|819|64|1915|4|9|9|3|1915|64|819|Thursday|1915Q3|N|N|N|2420742|2420984|2420385|2420659|N|N|N|N|N| +2420751|AAAAAAAAPAAPECAA|1915-09-10|188|819|64|1915|5|9|10|3|1915|64|819|Friday|1915Q3|N|Y|N|2420742|2420984|2420386|2420660|N|N|N|N|N| +2420752|AAAAAAAAABAPECAA|1915-09-11|188|819|64|1915|6|9|11|3|1915|64|819|Saturday|1915Q3|N|Y|N|2420742|2420984|2420387|2420661|N|N|N|N|N| +2420753|AAAAAAAABBAPECAA|1915-09-12|188|819|64|1915|0|9|12|3|1915|64|819|Sunday|1915Q3|N|N|N|2420742|2420984|2420388|2420662|N|N|N|N|N| +2420754|AAAAAAAACBAPECAA|1915-09-13|188|819|64|1915|1|9|13|3|1915|64|819|Monday|1915Q3|N|N|N|2420742|2420984|2420389|2420663|N|N|N|N|N| +2420755|AAAAAAAADBAPECAA|1915-09-14|188|820|64|1915|2|9|14|3|1915|64|820|Tuesday|1915Q3|N|N|N|2420742|2420984|2420390|2420664|N|N|N|N|N| +2420756|AAAAAAAAEBAPECAA|1915-09-15|188|820|64|1915|3|9|15|3|1915|64|820|Wednesday|1915Q3|N|N|N|2420742|2420984|2420391|2420665|N|N|N|N|N| +2420757|AAAAAAAAFBAPECAA|1915-09-16|188|820|64|1915|4|9|16|3|1915|64|820|Thursday|1915Q3|N|N|N|2420742|2420984|2420392|2420666|N|N|N|N|N| +2420758|AAAAAAAAGBAPECAA|1915-09-17|188|820|64|1915|5|9|17|3|1915|64|820|Friday|1915Q3|N|Y|N|2420742|2420984|2420393|2420667|N|N|N|N|N| +2420759|AAAAAAAAHBAPECAA|1915-09-18|188|820|64|1915|6|9|18|3|1915|64|820|Saturday|1915Q3|N|Y|N|2420742|2420984|2420394|2420668|N|N|N|N|N| +2420760|AAAAAAAAIBAPECAA|1915-09-19|188|820|64|1915|0|9|19|3|1915|64|820|Sunday|1915Q3|N|N|N|2420742|2420984|2420395|2420669|N|N|N|N|N| +2420761|AAAAAAAAJBAPECAA|1915-09-20|188|820|64|1915|1|9|20|3|1915|64|820|Monday|1915Q3|N|N|N|2420742|2420984|2420396|2420670|N|N|N|N|N| +2420762|AAAAAAAAKBAPECAA|1915-09-21|188|821|64|1915|2|9|21|3|1915|64|821|Tuesday|1915Q3|N|N|N|2420742|2420984|2420397|2420671|N|N|N|N|N| +2420763|AAAAAAAALBAPECAA|1915-09-22|188|821|64|1915|3|9|22|3|1915|64|821|Wednesday|1915Q3|N|N|N|2420742|2420984|2420398|2420672|N|N|N|N|N| +2420764|AAAAAAAAMBAPECAA|1915-09-23|188|821|64|1915|4|9|23|3|1915|64|821|Thursday|1915Q3|N|N|N|2420742|2420984|2420399|2420673|N|N|N|N|N| +2420765|AAAAAAAANBAPECAA|1915-09-24|188|821|64|1915|5|9|24|3|1915|64|821|Friday|1915Q3|N|Y|N|2420742|2420984|2420400|2420674|N|N|N|N|N| +2420766|AAAAAAAAOBAPECAA|1915-09-25|188|821|64|1915|6|9|25|3|1915|64|821|Saturday|1915Q3|N|Y|N|2420742|2420984|2420401|2420675|N|N|N|N|N| +2420767|AAAAAAAAPBAPECAA|1915-09-26|188|821|64|1915|0|9|26|3|1915|64|821|Sunday|1915Q3|N|N|N|2420742|2420984|2420402|2420676|N|N|N|N|N| +2420768|AAAAAAAAACAPECAA|1915-09-27|188|821|64|1915|1|9|27|3|1915|64|821|Monday|1915Q3|N|N|N|2420742|2420984|2420403|2420677|N|N|N|N|N| +2420769|AAAAAAAABCAPECAA|1915-09-28|188|822|64|1915|2|9|28|3|1915|64|822|Tuesday|1915Q3|N|N|N|2420742|2420984|2420404|2420678|N|N|N|N|N| +2420770|AAAAAAAACCAPECAA|1915-09-29|188|822|64|1915|3|9|29|3|1915|64|822|Wednesday|1915Q3|N|N|N|2420742|2420984|2420405|2420679|N|N|N|N|N| +2420771|AAAAAAAADCAPECAA|1915-09-30|188|822|64|1915|4|9|30|3|1915|64|822|Thursday|1915Q3|N|N|N|2420742|2420984|2420406|2420680|N|N|N|N|N| +2420772|AAAAAAAAECAPECAA|1915-10-01|189|822|64|1915|5|10|1|3|1915|64|822|Friday|1915Q3|N|Y|N|2420772|2421044|2420407|2420680|N|N|N|N|N| +2420773|AAAAAAAAFCAPECAA|1915-10-02|189|822|64|1915|6|10|2|4|1915|64|822|Saturday|1915Q4|N|Y|N|2420772|2421044|2420408|2420681|N|N|N|N|N| +2420774|AAAAAAAAGCAPECAA|1915-10-03|189|822|64|1915|0|10|3|4|1915|64|822|Sunday|1915Q4|N|N|N|2420772|2421044|2420409|2420682|N|N|N|N|N| +2420775|AAAAAAAAHCAPECAA|1915-10-04|189|822|64|1915|1|10|4|4|1915|64|822|Monday|1915Q4|N|N|N|2420772|2421044|2420410|2420683|N|N|N|N|N| +2420776|AAAAAAAAICAPECAA|1915-10-05|189|823|64|1915|2|10|5|4|1915|64|823|Tuesday|1915Q4|N|N|N|2420772|2421044|2420411|2420684|N|N|N|N|N| +2420777|AAAAAAAAJCAPECAA|1915-10-06|189|823|64|1915|3|10|6|4|1915|64|823|Wednesday|1915Q4|N|N|N|2420772|2421044|2420412|2420685|N|N|N|N|N| +2420778|AAAAAAAAKCAPECAA|1915-10-07|189|823|64|1915|4|10|7|4|1915|64|823|Thursday|1915Q4|N|N|N|2420772|2421044|2420413|2420686|N|N|N|N|N| +2420779|AAAAAAAALCAPECAA|1915-10-08|189|823|64|1915|5|10|8|4|1915|64|823|Friday|1915Q4|N|Y|N|2420772|2421044|2420414|2420687|N|N|N|N|N| +2420780|AAAAAAAAMCAPECAA|1915-10-09|189|823|64|1915|6|10|9|4|1915|64|823|Saturday|1915Q4|N|Y|N|2420772|2421044|2420415|2420688|N|N|N|N|N| +2420781|AAAAAAAANCAPECAA|1915-10-10|189|823|64|1915|0|10|10|4|1915|64|823|Sunday|1915Q4|N|N|N|2420772|2421044|2420416|2420689|N|N|N|N|N| +2420782|AAAAAAAAOCAPECAA|1915-10-11|189|823|64|1915|1|10|11|4|1915|64|823|Monday|1915Q4|N|N|N|2420772|2421044|2420417|2420690|N|N|N|N|N| +2420783|AAAAAAAAPCAPECAA|1915-10-12|189|824|64|1915|2|10|12|4|1915|64|824|Tuesday|1915Q4|N|N|N|2420772|2421044|2420418|2420691|N|N|N|N|N| +2420784|AAAAAAAAADAPECAA|1915-10-13|189|824|64|1915|3|10|13|4|1915|64|824|Wednesday|1915Q4|N|N|N|2420772|2421044|2420419|2420692|N|N|N|N|N| +2420785|AAAAAAAABDAPECAA|1915-10-14|189|824|64|1915|4|10|14|4|1915|64|824|Thursday|1915Q4|N|N|N|2420772|2421044|2420420|2420693|N|N|N|N|N| +2420786|AAAAAAAACDAPECAA|1915-10-15|189|824|64|1915|5|10|15|4|1915|64|824|Friday|1915Q4|N|Y|N|2420772|2421044|2420421|2420694|N|N|N|N|N| +2420787|AAAAAAAADDAPECAA|1915-10-16|189|824|64|1915|6|10|16|4|1915|64|824|Saturday|1915Q4|N|Y|N|2420772|2421044|2420422|2420695|N|N|N|N|N| +2420788|AAAAAAAAEDAPECAA|1915-10-17|189|824|64|1915|0|10|17|4|1915|64|824|Sunday|1915Q4|N|N|N|2420772|2421044|2420423|2420696|N|N|N|N|N| +2420789|AAAAAAAAFDAPECAA|1915-10-18|189|824|64|1915|1|10|18|4|1915|64|824|Monday|1915Q4|N|N|N|2420772|2421044|2420424|2420697|N|N|N|N|N| +2420790|AAAAAAAAGDAPECAA|1915-10-19|189|825|64|1915|2|10|19|4|1915|64|825|Tuesday|1915Q4|N|N|N|2420772|2421044|2420425|2420698|N|N|N|N|N| +2420791|AAAAAAAAHDAPECAA|1915-10-20|189|825|64|1915|3|10|20|4|1915|64|825|Wednesday|1915Q4|N|N|N|2420772|2421044|2420426|2420699|N|N|N|N|N| +2420792|AAAAAAAAIDAPECAA|1915-10-21|189|825|64|1915|4|10|21|4|1915|64|825|Thursday|1915Q4|N|N|N|2420772|2421044|2420427|2420700|N|N|N|N|N| +2420793|AAAAAAAAJDAPECAA|1915-10-22|189|825|64|1915|5|10|22|4|1915|64|825|Friday|1915Q4|N|Y|N|2420772|2421044|2420428|2420701|N|N|N|N|N| +2420794|AAAAAAAAKDAPECAA|1915-10-23|189|825|64|1915|6|10|23|4|1915|64|825|Saturday|1915Q4|N|Y|N|2420772|2421044|2420429|2420702|N|N|N|N|N| +2420795|AAAAAAAALDAPECAA|1915-10-24|189|825|64|1915|0|10|24|4|1915|64|825|Sunday|1915Q4|N|N|N|2420772|2421044|2420430|2420703|N|N|N|N|N| +2420796|AAAAAAAAMDAPECAA|1915-10-25|189|825|64|1915|1|10|25|4|1915|64|825|Monday|1915Q4|N|N|N|2420772|2421044|2420431|2420704|N|N|N|N|N| +2420797|AAAAAAAANDAPECAA|1915-10-26|189|826|64|1915|2|10|26|4|1915|64|826|Tuesday|1915Q4|N|N|N|2420772|2421044|2420432|2420705|N|N|N|N|N| +2420798|AAAAAAAAODAPECAA|1915-10-27|189|826|64|1915|3|10|27|4|1915|64|826|Wednesday|1915Q4|N|N|N|2420772|2421044|2420433|2420706|N|N|N|N|N| +2420799|AAAAAAAAPDAPECAA|1915-10-28|189|826|64|1915|4|10|28|4|1915|64|826|Thursday|1915Q4|N|N|N|2420772|2421044|2420434|2420707|N|N|N|N|N| +2420800|AAAAAAAAAEAPECAA|1915-10-29|189|826|64|1915|5|10|29|4|1915|64|826|Friday|1915Q4|N|Y|N|2420772|2421044|2420435|2420708|N|N|N|N|N| +2420801|AAAAAAAABEAPECAA|1915-10-30|189|826|64|1915|6|10|30|4|1915|64|826|Saturday|1915Q4|N|Y|N|2420772|2421044|2420436|2420709|N|N|N|N|N| +2420802|AAAAAAAACEAPECAA|1915-10-31|189|826|64|1915|0|10|31|4|1915|64|826|Sunday|1915Q4|N|N|N|2420772|2421044|2420437|2420710|N|N|N|N|N| +2420803|AAAAAAAADEAPECAA|1915-11-01|190|826|64|1915|1|11|1|4|1915|64|826|Monday|1915Q4|N|N|N|2420803|2421106|2420438|2420711|N|N|N|N|N| +2420804|AAAAAAAAEEAPECAA|1915-11-02|190|827|64|1915|2|11|2|4|1915|64|827|Tuesday|1915Q4|N|N|N|2420803|2421106|2420439|2420712|N|N|N|N|N| +2420805|AAAAAAAAFEAPECAA|1915-11-03|190|827|64|1915|3|11|3|4|1915|64|827|Wednesday|1915Q4|N|N|N|2420803|2421106|2420440|2420713|N|N|N|N|N| +2420806|AAAAAAAAGEAPECAA|1915-11-04|190|827|64|1915|4|11|4|4|1915|64|827|Thursday|1915Q4|N|N|N|2420803|2421106|2420441|2420714|N|N|N|N|N| +2420807|AAAAAAAAHEAPECAA|1915-11-05|190|827|64|1915|5|11|5|4|1915|64|827|Friday|1915Q4|N|Y|N|2420803|2421106|2420442|2420715|N|N|N|N|N| +2420808|AAAAAAAAIEAPECAA|1915-11-06|190|827|64|1915|6|11|6|4|1915|64|827|Saturday|1915Q4|N|Y|N|2420803|2421106|2420443|2420716|N|N|N|N|N| +2420809|AAAAAAAAJEAPECAA|1915-11-07|190|827|64|1915|0|11|7|4|1915|64|827|Sunday|1915Q4|N|N|N|2420803|2421106|2420444|2420717|N|N|N|N|N| +2420810|AAAAAAAAKEAPECAA|1915-11-08|190|827|64|1915|1|11|8|4|1915|64|827|Monday|1915Q4|N|N|N|2420803|2421106|2420445|2420718|N|N|N|N|N| +2420811|AAAAAAAALEAPECAA|1915-11-09|190|828|64|1915|2|11|9|4|1915|64|828|Tuesday|1915Q4|N|N|N|2420803|2421106|2420446|2420719|N|N|N|N|N| +2420812|AAAAAAAAMEAPECAA|1915-11-10|190|828|64|1915|3|11|10|4|1915|64|828|Wednesday|1915Q4|N|N|N|2420803|2421106|2420447|2420720|N|N|N|N|N| +2420813|AAAAAAAANEAPECAA|1915-11-11|190|828|64|1915|4|11|11|4|1915|64|828|Thursday|1915Q4|N|N|N|2420803|2421106|2420448|2420721|N|N|N|N|N| +2420814|AAAAAAAAOEAPECAA|1915-11-12|190|828|64|1915|5|11|12|4|1915|64|828|Friday|1915Q4|N|Y|N|2420803|2421106|2420449|2420722|N|N|N|N|N| +2420815|AAAAAAAAPEAPECAA|1915-11-13|190|828|64|1915|6|11|13|4|1915|64|828|Saturday|1915Q4|N|Y|N|2420803|2421106|2420450|2420723|N|N|N|N|N| +2420816|AAAAAAAAAFAPECAA|1915-11-14|190|828|64|1915|0|11|14|4|1915|64|828|Sunday|1915Q4|N|N|N|2420803|2421106|2420451|2420724|N|N|N|N|N| +2420817|AAAAAAAABFAPECAA|1915-11-15|190|828|64|1915|1|11|15|4|1915|64|828|Monday|1915Q4|N|N|N|2420803|2421106|2420452|2420725|N|N|N|N|N| +2420818|AAAAAAAACFAPECAA|1915-11-16|190|829|64|1915|2|11|16|4|1915|64|829|Tuesday|1915Q4|N|N|N|2420803|2421106|2420453|2420726|N|N|N|N|N| +2420819|AAAAAAAADFAPECAA|1915-11-17|190|829|64|1915|3|11|17|4|1915|64|829|Wednesday|1915Q4|N|N|N|2420803|2421106|2420454|2420727|N|N|N|N|N| +2420820|AAAAAAAAEFAPECAA|1915-11-18|190|829|64|1915|4|11|18|4|1915|64|829|Thursday|1915Q4|N|N|N|2420803|2421106|2420455|2420728|N|N|N|N|N| +2420821|AAAAAAAAFFAPECAA|1915-11-19|190|829|64|1915|5|11|19|4|1915|64|829|Friday|1915Q4|N|Y|N|2420803|2421106|2420456|2420729|N|N|N|N|N| +2420822|AAAAAAAAGFAPECAA|1915-11-20|190|829|64|1915|6|11|20|4|1915|64|829|Saturday|1915Q4|N|Y|N|2420803|2421106|2420457|2420730|N|N|N|N|N| +2420823|AAAAAAAAHFAPECAA|1915-11-21|190|829|64|1915|0|11|21|4|1915|64|829|Sunday|1915Q4|N|N|N|2420803|2421106|2420458|2420731|N|N|N|N|N| +2420824|AAAAAAAAIFAPECAA|1915-11-22|190|829|64|1915|1|11|22|4|1915|64|829|Monday|1915Q4|N|N|N|2420803|2421106|2420459|2420732|N|N|N|N|N| +2420825|AAAAAAAAJFAPECAA|1915-11-23|190|830|64|1915|2|11|23|4|1915|64|830|Tuesday|1915Q4|N|N|N|2420803|2421106|2420460|2420733|N|N|N|N|N| +2420826|AAAAAAAAKFAPECAA|1915-11-24|190|830|64|1915|3|11|24|4|1915|64|830|Wednesday|1915Q4|N|N|N|2420803|2421106|2420461|2420734|N|N|N|N|N| +2420827|AAAAAAAALFAPECAA|1915-11-25|190|830|64|1915|4|11|25|4|1915|64|830|Thursday|1915Q4|N|N|N|2420803|2421106|2420462|2420735|N|N|N|N|N| +2420828|AAAAAAAAMFAPECAA|1915-11-26|190|830|64|1915|5|11|26|4|1915|64|830|Friday|1915Q4|N|Y|N|2420803|2421106|2420463|2420736|N|N|N|N|N| +2420829|AAAAAAAANFAPECAA|1915-11-27|190|830|64|1915|6|11|27|4|1915|64|830|Saturday|1915Q4|N|Y|N|2420803|2421106|2420464|2420737|N|N|N|N|N| +2420830|AAAAAAAAOFAPECAA|1915-11-28|190|830|64|1915|0|11|28|4|1915|64|830|Sunday|1915Q4|N|N|N|2420803|2421106|2420465|2420738|N|N|N|N|N| +2420831|AAAAAAAAPFAPECAA|1915-11-29|190|830|64|1915|1|11|29|4|1915|64|830|Monday|1915Q4|N|N|N|2420803|2421106|2420466|2420739|N|N|N|N|N| +2420832|AAAAAAAAAGAPECAA|1915-11-30|190|831|64|1915|2|11|30|4|1915|64|831|Tuesday|1915Q4|N|N|N|2420803|2421106|2420467|2420740|N|N|N|N|N| +2420833|AAAAAAAABGAPECAA|1915-12-01|191|831|65|1915|3|12|1|4|1915|65|831|Wednesday|1915Q4|N|N|N|2420833|2421166|2420468|2420741|N|N|N|N|N| +2420834|AAAAAAAACGAPECAA|1915-12-02|191|831|65|1915|4|12|2|4|1915|65|831|Thursday|1915Q4|N|N|N|2420833|2421166|2420469|2420742|N|N|N|N|N| +2420835|AAAAAAAADGAPECAA|1915-12-03|191|831|65|1915|5|12|3|4|1915|65|831|Friday|1915Q4|N|Y|N|2420833|2421166|2420470|2420743|N|N|N|N|N| +2420836|AAAAAAAAEGAPECAA|1915-12-04|191|831|65|1915|6|12|4|4|1915|65|831|Saturday|1915Q4|N|Y|N|2420833|2421166|2420471|2420744|N|N|N|N|N| +2420837|AAAAAAAAFGAPECAA|1915-12-05|191|831|65|1915|0|12|5|4|1915|65|831|Sunday|1915Q4|N|N|N|2420833|2421166|2420472|2420745|N|N|N|N|N| +2420838|AAAAAAAAGGAPECAA|1915-12-06|191|831|65|1915|1|12|6|4|1915|65|831|Monday|1915Q4|N|N|N|2420833|2421166|2420473|2420746|N|N|N|N|N| +2420839|AAAAAAAAHGAPECAA|1915-12-07|191|832|65|1915|2|12|7|4|1915|65|832|Tuesday|1915Q4|N|N|N|2420833|2421166|2420474|2420747|N|N|N|N|N| +2420840|AAAAAAAAIGAPECAA|1915-12-08|191|832|65|1915|3|12|8|4|1915|65|832|Wednesday|1915Q4|N|N|N|2420833|2421166|2420475|2420748|N|N|N|N|N| +2420841|AAAAAAAAJGAPECAA|1915-12-09|191|832|65|1915|4|12|9|4|1915|65|832|Thursday|1915Q4|N|N|N|2420833|2421166|2420476|2420749|N|N|N|N|N| +2420842|AAAAAAAAKGAPECAA|1915-12-10|191|832|65|1915|5|12|10|4|1915|65|832|Friday|1915Q4|N|Y|N|2420833|2421166|2420477|2420750|N|N|N|N|N| +2420843|AAAAAAAALGAPECAA|1915-12-11|191|832|65|1915|6|12|11|4|1915|65|832|Saturday|1915Q4|N|Y|N|2420833|2421166|2420478|2420751|N|N|N|N|N| +2420844|AAAAAAAAMGAPECAA|1915-12-12|191|832|65|1915|0|12|12|4|1915|65|832|Sunday|1915Q4|N|N|N|2420833|2421166|2420479|2420752|N|N|N|N|N| +2420845|AAAAAAAANGAPECAA|1915-12-13|191|832|65|1915|1|12|13|4|1915|65|832|Monday|1915Q4|N|N|N|2420833|2421166|2420480|2420753|N|N|N|N|N| +2420846|AAAAAAAAOGAPECAA|1915-12-14|191|833|65|1915|2|12|14|4|1915|65|833|Tuesday|1915Q4|N|N|N|2420833|2421166|2420481|2420754|N|N|N|N|N| +2420847|AAAAAAAAPGAPECAA|1915-12-15|191|833|65|1915|3|12|15|4|1915|65|833|Wednesday|1915Q4|N|N|N|2420833|2421166|2420482|2420755|N|N|N|N|N| +2420848|AAAAAAAAAHAPECAA|1915-12-16|191|833|65|1915|4|12|16|4|1915|65|833|Thursday|1915Q4|N|N|N|2420833|2421166|2420483|2420756|N|N|N|N|N| +2420849|AAAAAAAABHAPECAA|1915-12-17|191|833|65|1915|5|12|17|4|1915|65|833|Friday|1915Q4|N|Y|N|2420833|2421166|2420484|2420757|N|N|N|N|N| +2420850|AAAAAAAACHAPECAA|1915-12-18|191|833|65|1915|6|12|18|4|1915|65|833|Saturday|1915Q4|N|Y|N|2420833|2421166|2420485|2420758|N|N|N|N|N| +2420851|AAAAAAAADHAPECAA|1915-12-19|191|833|65|1915|0|12|19|4|1915|65|833|Sunday|1915Q4|N|N|N|2420833|2421166|2420486|2420759|N|N|N|N|N| +2420852|AAAAAAAAEHAPECAA|1915-12-20|191|833|65|1915|1|12|20|4|1915|65|833|Monday|1915Q4|N|N|N|2420833|2421166|2420487|2420760|N|N|N|N|N| +2420853|AAAAAAAAFHAPECAA|1915-12-21|191|834|65|1915|2|12|21|4|1915|65|834|Tuesday|1915Q4|N|N|N|2420833|2421166|2420488|2420761|N|N|N|N|N| +2420854|AAAAAAAAGHAPECAA|1915-12-22|191|834|65|1915|3|12|22|4|1915|65|834|Wednesday|1915Q4|N|N|N|2420833|2421166|2420489|2420762|N|N|N|N|N| +2420855|AAAAAAAAHHAPECAA|1915-12-23|191|834|65|1915|4|12|23|4|1915|65|834|Thursday|1915Q4|N|N|N|2420833|2421166|2420490|2420763|N|N|N|N|N| +2420856|AAAAAAAAIHAPECAA|1915-12-24|191|834|65|1915|5|12|24|4|1915|65|834|Friday|1915Q4|N|Y|N|2420833|2421166|2420491|2420764|N|N|N|N|N| +2420857|AAAAAAAAJHAPECAA|1915-12-25|191|834|65|1915|6|12|25|4|1915|65|834|Saturday|1915Q4|N|Y|N|2420833|2421166|2420492|2420765|N|N|N|N|N| +2420858|AAAAAAAAKHAPECAA|1915-12-26|191|834|65|1915|0|12|26|4|1915|65|834|Sunday|1915Q4|Y|N|N|2420833|2421166|2420493|2420766|N|N|N|N|N| +2420859|AAAAAAAALHAPECAA|1915-12-27|191|834|65|1915|1|12|27|4|1915|65|834|Monday|1915Q4|N|N|Y|2420833|2421166|2420494|2420767|N|N|N|N|N| +2420860|AAAAAAAAMHAPECAA|1915-12-28|191|835|65|1915|2|12|28|4|1915|65|835|Tuesday|1915Q4|N|N|N|2420833|2421166|2420495|2420768|N|N|N|N|N| +2420861|AAAAAAAANHAPECAA|1915-12-29|191|835|65|1915|3|12|29|4|1915|65|835|Wednesday|1915Q4|N|N|N|2420833|2421166|2420496|2420769|N|N|N|N|N| +2420862|AAAAAAAAOHAPECAA|1915-12-30|191|835|65|1915|4|12|30|4|1915|65|835|Thursday|1915Q4|N|N|N|2420833|2421166|2420497|2420770|N|N|N|N|N| +2420863|AAAAAAAAPHAPECAA|1915-12-31|191|835|65|1915|5|12|31|4|1915|65|835|Friday|1915Q4|N|Y|N|2420833|2421166|2420498|2420771|N|N|N|N|N| +2420864|AAAAAAAAAIAPECAA|1916-01-01|192|835|65|1916|6|1|1|1|1916|65|835|Saturday|1916Q1|Y|Y|N|2420864|2420863|2420499|2420772|N|N|N|N|N| +2420865|AAAAAAAABIAPECAA|1916-01-02|192|835|65|1916|0|1|2|1|1916|65|835|Sunday|1916Q1|N|N|Y|2420864|2420863|2420500|2420773|N|N|N|N|N| +2420866|AAAAAAAACIAPECAA|1916-01-03|192|835|65|1916|1|1|3|1|1916|65|835|Monday|1916Q1|N|N|N|2420864|2420863|2420501|2420774|N|N|N|N|N| +2420867|AAAAAAAADIAPECAA|1916-01-04|192|836|65|1916|2|1|4|1|1916|65|836|Tuesday|1916Q1|N|N|N|2420864|2420863|2420502|2420775|N|N|N|N|N| +2420868|AAAAAAAAEIAPECAA|1916-01-05|192|836|65|1916|3|1|5|1|1916|65|836|Wednesday|1916Q1|N|N|N|2420864|2420863|2420503|2420776|N|N|N|N|N| +2420869|AAAAAAAAFIAPECAA|1916-01-06|192|836|65|1916|4|1|6|1|1916|65|836|Thursday|1916Q1|N|N|N|2420864|2420863|2420504|2420777|N|N|N|N|N| +2420870|AAAAAAAAGIAPECAA|1916-01-07|192|836|65|1916|5|1|7|1|1916|65|836|Friday|1916Q1|N|Y|N|2420864|2420863|2420505|2420778|N|N|N|N|N| +2420871|AAAAAAAAHIAPECAA|1916-01-08|192|836|65|1916|6|1|8|1|1916|65|836|Saturday|1916Q1|N|Y|N|2420864|2420863|2420506|2420779|N|N|N|N|N| +2420872|AAAAAAAAIIAPECAA|1916-01-09|192|836|65|1916|0|1|9|1|1916|65|836|Sunday|1916Q1|N|N|N|2420864|2420863|2420507|2420780|N|N|N|N|N| +2420873|AAAAAAAAJIAPECAA|1916-01-10|192|836|65|1916|1|1|10|1|1916|65|836|Monday|1916Q1|N|N|N|2420864|2420863|2420508|2420781|N|N|N|N|N| +2420874|AAAAAAAAKIAPECAA|1916-01-11|192|837|65|1916|2|1|11|1|1916|65|837|Tuesday|1916Q1|N|N|N|2420864|2420863|2420509|2420782|N|N|N|N|N| +2420875|AAAAAAAALIAPECAA|1916-01-12|192|837|65|1916|3|1|12|1|1916|65|837|Wednesday|1916Q1|N|N|N|2420864|2420863|2420510|2420783|N|N|N|N|N| +2420876|AAAAAAAAMIAPECAA|1916-01-13|192|837|65|1916|4|1|13|1|1916|65|837|Thursday|1916Q1|N|N|N|2420864|2420863|2420511|2420784|N|N|N|N|N| +2420877|AAAAAAAANIAPECAA|1916-01-14|192|837|65|1916|5|1|14|1|1916|65|837|Friday|1916Q1|N|Y|N|2420864|2420863|2420512|2420785|N|N|N|N|N| +2420878|AAAAAAAAOIAPECAA|1916-01-15|192|837|65|1916|6|1|15|1|1916|65|837|Saturday|1916Q1|N|Y|N|2420864|2420863|2420513|2420786|N|N|N|N|N| +2420879|AAAAAAAAPIAPECAA|1916-01-16|192|837|65|1916|0|1|16|1|1916|65|837|Sunday|1916Q1|N|N|N|2420864|2420863|2420514|2420787|N|N|N|N|N| +2420880|AAAAAAAAAJAPECAA|1916-01-17|192|837|65|1916|1|1|17|1|1916|65|837|Monday|1916Q1|N|N|N|2420864|2420863|2420515|2420788|N|N|N|N|N| +2420881|AAAAAAAABJAPECAA|1916-01-18|192|838|65|1916|2|1|18|1|1916|65|838|Tuesday|1916Q1|N|N|N|2420864|2420863|2420516|2420789|N|N|N|N|N| +2420882|AAAAAAAACJAPECAA|1916-01-19|192|838|65|1916|3|1|19|1|1916|65|838|Wednesday|1916Q1|N|N|N|2420864|2420863|2420517|2420790|N|N|N|N|N| +2420883|AAAAAAAADJAPECAA|1916-01-20|192|838|65|1916|4|1|20|1|1916|65|838|Thursday|1916Q1|N|N|N|2420864|2420863|2420518|2420791|N|N|N|N|N| +2420884|AAAAAAAAEJAPECAA|1916-01-21|192|838|65|1916|5|1|21|1|1916|65|838|Friday|1916Q1|N|Y|N|2420864|2420863|2420519|2420792|N|N|N|N|N| +2420885|AAAAAAAAFJAPECAA|1916-01-22|192|838|65|1916|6|1|22|1|1916|65|838|Saturday|1916Q1|N|Y|N|2420864|2420863|2420520|2420793|N|N|N|N|N| +2420886|AAAAAAAAGJAPECAA|1916-01-23|192|838|65|1916|0|1|23|1|1916|65|838|Sunday|1916Q1|N|N|N|2420864|2420863|2420521|2420794|N|N|N|N|N| +2420887|AAAAAAAAHJAPECAA|1916-01-24|192|838|65|1916|1|1|24|1|1916|65|838|Monday|1916Q1|N|N|N|2420864|2420863|2420522|2420795|N|N|N|N|N| +2420888|AAAAAAAAIJAPECAA|1916-01-25|192|839|65|1916|2|1|25|1|1916|65|839|Tuesday|1916Q1|N|N|N|2420864|2420863|2420523|2420796|N|N|N|N|N| +2420889|AAAAAAAAJJAPECAA|1916-01-26|192|839|65|1916|3|1|26|1|1916|65|839|Wednesday|1916Q1|N|N|N|2420864|2420863|2420524|2420797|N|N|N|N|N| +2420890|AAAAAAAAKJAPECAA|1916-01-27|192|839|65|1916|4|1|27|1|1916|65|839|Thursday|1916Q1|N|N|N|2420864|2420863|2420525|2420798|N|N|N|N|N| +2420891|AAAAAAAALJAPECAA|1916-01-28|192|839|65|1916|5|1|28|1|1916|65|839|Friday|1916Q1|N|Y|N|2420864|2420863|2420526|2420799|N|N|N|N|N| +2420892|AAAAAAAAMJAPECAA|1916-01-29|192|839|65|1916|6|1|29|1|1916|65|839|Saturday|1916Q1|N|Y|N|2420864|2420863|2420527|2420800|N|N|N|N|N| +2420893|AAAAAAAANJAPECAA|1916-01-30|192|839|65|1916|0|1|30|1|1916|65|839|Sunday|1916Q1|N|N|N|2420864|2420863|2420528|2420801|N|N|N|N|N| +2420894|AAAAAAAAOJAPECAA|1916-01-31|192|839|65|1916|1|1|31|1|1916|65|839|Monday|1916Q1|N|N|N|2420864|2420863|2420529|2420802|N|N|N|N|N| +2420895|AAAAAAAAPJAPECAA|1916-02-01|193|840|65|1916|2|2|1|1|1916|65|840|Tuesday|1916Q1|N|N|N|2420895|2420925|2420530|2420803|N|N|N|N|N| +2420896|AAAAAAAAAKAPECAA|1916-02-02|193|840|65|1916|3|2|2|1|1916|65|840|Wednesday|1916Q1|N|N|N|2420895|2420925|2420531|2420804|N|N|N|N|N| +2420897|AAAAAAAABKAPECAA|1916-02-03|193|840|65|1916|4|2|3|1|1916|65|840|Thursday|1916Q1|N|N|N|2420895|2420925|2420532|2420805|N|N|N|N|N| +2420898|AAAAAAAACKAPECAA|1916-02-04|193|840|65|1916|5|2|4|1|1916|65|840|Friday|1916Q1|N|Y|N|2420895|2420925|2420533|2420806|N|N|N|N|N| +2420899|AAAAAAAADKAPECAA|1916-02-05|193|840|65|1916|6|2|5|1|1916|65|840|Saturday|1916Q1|N|Y|N|2420895|2420925|2420534|2420807|N|N|N|N|N| +2420900|AAAAAAAAEKAPECAA|1916-02-06|193|840|65|1916|0|2|6|1|1916|65|840|Sunday|1916Q1|N|N|N|2420895|2420925|2420535|2420808|N|N|N|N|N| +2420901|AAAAAAAAFKAPECAA|1916-02-07|193|840|65|1916|1|2|7|1|1916|65|840|Monday|1916Q1|N|N|N|2420895|2420925|2420536|2420809|N|N|N|N|N| +2420902|AAAAAAAAGKAPECAA|1916-02-08|193|841|65|1916|2|2|8|1|1916|65|841|Tuesday|1916Q1|N|N|N|2420895|2420925|2420537|2420810|N|N|N|N|N| +2420903|AAAAAAAAHKAPECAA|1916-02-09|193|841|65|1916|3|2|9|1|1916|65|841|Wednesday|1916Q1|N|N|N|2420895|2420925|2420538|2420811|N|N|N|N|N| +2420904|AAAAAAAAIKAPECAA|1916-02-10|193|841|65|1916|4|2|10|1|1916|65|841|Thursday|1916Q1|N|N|N|2420895|2420925|2420539|2420812|N|N|N|N|N| +2420905|AAAAAAAAJKAPECAA|1916-02-11|193|841|65|1916|5|2|11|1|1916|65|841|Friday|1916Q1|N|Y|N|2420895|2420925|2420540|2420813|N|N|N|N|N| +2420906|AAAAAAAAKKAPECAA|1916-02-12|193|841|65|1916|6|2|12|1|1916|65|841|Saturday|1916Q1|N|Y|N|2420895|2420925|2420541|2420814|N|N|N|N|N| +2420907|AAAAAAAALKAPECAA|1916-02-13|193|841|65|1916|0|2|13|1|1916|65|841|Sunday|1916Q1|N|N|N|2420895|2420925|2420542|2420815|N|N|N|N|N| +2420908|AAAAAAAAMKAPECAA|1916-02-14|193|841|65|1916|1|2|14|1|1916|65|841|Monday|1916Q1|N|N|N|2420895|2420925|2420543|2420816|N|N|N|N|N| +2420909|AAAAAAAANKAPECAA|1916-02-15|193|842|65|1916|2|2|15|1|1916|65|842|Tuesday|1916Q1|N|N|N|2420895|2420925|2420544|2420817|N|N|N|N|N| +2420910|AAAAAAAAOKAPECAA|1916-02-16|193|842|65|1916|3|2|16|1|1916|65|842|Wednesday|1916Q1|N|N|N|2420895|2420925|2420545|2420818|N|N|N|N|N| +2420911|AAAAAAAAPKAPECAA|1916-02-17|193|842|65|1916|4|2|17|1|1916|65|842|Thursday|1916Q1|N|N|N|2420895|2420925|2420546|2420819|N|N|N|N|N| +2420912|AAAAAAAAALAPECAA|1916-02-18|193|842|65|1916|5|2|18|1|1916|65|842|Friday|1916Q1|N|Y|N|2420895|2420925|2420547|2420820|N|N|N|N|N| +2420913|AAAAAAAABLAPECAA|1916-02-19|193|842|65|1916|6|2|19|1|1916|65|842|Saturday|1916Q1|N|Y|N|2420895|2420925|2420548|2420821|N|N|N|N|N| +2420914|AAAAAAAACLAPECAA|1916-02-20|193|842|65|1916|0|2|20|1|1916|65|842|Sunday|1916Q1|N|N|N|2420895|2420925|2420549|2420822|N|N|N|N|N| +2420915|AAAAAAAADLAPECAA|1916-02-21|193|842|65|1916|1|2|21|1|1916|65|842|Monday|1916Q1|N|N|N|2420895|2420925|2420550|2420823|N|N|N|N|N| +2420916|AAAAAAAAELAPECAA|1916-02-22|193|843|65|1916|2|2|22|1|1916|65|843|Tuesday|1916Q1|N|N|N|2420895|2420925|2420551|2420824|N|N|N|N|N| +2420917|AAAAAAAAFLAPECAA|1916-02-23|193|843|65|1916|3|2|23|1|1916|65|843|Wednesday|1916Q1|N|N|N|2420895|2420925|2420552|2420825|N|N|N|N|N| +2420918|AAAAAAAAGLAPECAA|1916-02-24|193|843|65|1916|4|2|24|1|1916|65|843|Thursday|1916Q1|N|N|N|2420895|2420925|2420553|2420826|N|N|N|N|N| +2420919|AAAAAAAAHLAPECAA|1916-02-25|193|843|65|1916|5|2|25|1|1916|65|843|Friday|1916Q1|N|Y|N|2420895|2420925|2420554|2420827|N|N|N|N|N| +2420920|AAAAAAAAILAPECAA|1916-02-26|193|843|65|1916|6|2|26|1|1916|65|843|Saturday|1916Q1|N|Y|N|2420895|2420925|2420555|2420828|N|N|N|N|N| +2420921|AAAAAAAAJLAPECAA|1916-02-27|193|843|65|1916|0|2|27|1|1916|65|843|Sunday|1916Q1|N|N|N|2420895|2420925|2420556|2420829|N|N|N|N|N| +2420922|AAAAAAAAKLAPECAA|1916-02-28|193|843|65|1916|1|2|28|1|1916|65|843|Monday|1916Q1|N|N|N|2420895|2420925|2420557|2420830|N|N|N|N|N| +2420923|AAAAAAAALLAPECAA|1916-02-29|193|844|65|1916|2|2|29|1|1916|65|844|Tuesday|1916Q1|N|N|N|2420895|2420925|2420557|2420831|N|N|N|N|N| +2420924|AAAAAAAAMLAPECAA|1916-03-01|194|844|66|1916|3|3|1|1|1916|66|844|Wednesday|1916Q1|N|N|N|2420924|2420983|2420558|2420832|N|N|N|N|N| +2420925|AAAAAAAANLAPECAA|1916-03-02|194|844|66|1916|4|3|2|1|1916|66|844|Thursday|1916Q1|N|N|N|2420924|2420983|2420559|2420833|N|N|N|N|N| +2420926|AAAAAAAAOLAPECAA|1916-03-03|194|844|66|1916|5|3|3|1|1916|66|844|Friday|1916Q1|N|Y|N|2420924|2420983|2420560|2420834|N|N|N|N|N| +2420927|AAAAAAAAPLAPECAA|1916-03-04|194|844|66|1916|6|3|4|1|1916|66|844|Saturday|1916Q1|N|Y|N|2420924|2420983|2420561|2420835|N|N|N|N|N| +2420928|AAAAAAAAAMAPECAA|1916-03-05|194|844|66|1916|0|3|5|1|1916|66|844|Sunday|1916Q1|N|N|N|2420924|2420983|2420562|2420836|N|N|N|N|N| +2420929|AAAAAAAABMAPECAA|1916-03-06|194|844|66|1916|1|3|6|1|1916|66|844|Monday|1916Q1|N|N|N|2420924|2420983|2420563|2420837|N|N|N|N|N| +2420930|AAAAAAAACMAPECAA|1916-03-07|194|845|66|1916|2|3|7|1|1916|66|845|Tuesday|1916Q1|N|N|N|2420924|2420983|2420564|2420838|N|N|N|N|N| +2420931|AAAAAAAADMAPECAA|1916-03-08|194|845|66|1916|3|3|8|1|1916|66|845|Wednesday|1916Q1|N|N|N|2420924|2420983|2420565|2420839|N|N|N|N|N| +2420932|AAAAAAAAEMAPECAA|1916-03-09|194|845|66|1916|4|3|9|1|1916|66|845|Thursday|1916Q1|N|N|N|2420924|2420983|2420566|2420840|N|N|N|N|N| +2420933|AAAAAAAAFMAPECAA|1916-03-10|194|845|66|1916|5|3|10|1|1916|66|845|Friday|1916Q1|N|Y|N|2420924|2420983|2420567|2420841|N|N|N|N|N| +2420934|AAAAAAAAGMAPECAA|1916-03-11|194|845|66|1916|6|3|11|1|1916|66|845|Saturday|1916Q1|N|Y|N|2420924|2420983|2420568|2420842|N|N|N|N|N| +2420935|AAAAAAAAHMAPECAA|1916-03-12|194|845|66|1916|0|3|12|1|1916|66|845|Sunday|1916Q1|N|N|N|2420924|2420983|2420569|2420843|N|N|N|N|N| +2420936|AAAAAAAAIMAPECAA|1916-03-13|194|845|66|1916|1|3|13|1|1916|66|845|Monday|1916Q1|N|N|N|2420924|2420983|2420570|2420844|N|N|N|N|N| +2420937|AAAAAAAAJMAPECAA|1916-03-14|194|846|66|1916|2|3|14|1|1916|66|846|Tuesday|1916Q1|N|N|N|2420924|2420983|2420571|2420845|N|N|N|N|N| +2420938|AAAAAAAAKMAPECAA|1916-03-15|194|846|66|1916|3|3|15|1|1916|66|846|Wednesday|1916Q1|N|N|N|2420924|2420983|2420572|2420846|N|N|N|N|N| +2420939|AAAAAAAALMAPECAA|1916-03-16|194|846|66|1916|4|3|16|1|1916|66|846|Thursday|1916Q1|N|N|N|2420924|2420983|2420573|2420847|N|N|N|N|N| +2420940|AAAAAAAAMMAPECAA|1916-03-17|194|846|66|1916|5|3|17|1|1916|66|846|Friday|1916Q1|N|Y|N|2420924|2420983|2420574|2420848|N|N|N|N|N| +2420941|AAAAAAAANMAPECAA|1916-03-18|194|846|66|1916|6|3|18|1|1916|66|846|Saturday|1916Q1|N|Y|N|2420924|2420983|2420575|2420849|N|N|N|N|N| +2420942|AAAAAAAAOMAPECAA|1916-03-19|194|846|66|1916|0|3|19|1|1916|66|846|Sunday|1916Q1|N|N|N|2420924|2420983|2420576|2420850|N|N|N|N|N| +2420943|AAAAAAAAPMAPECAA|1916-03-20|194|846|66|1916|1|3|20|1|1916|66|846|Monday|1916Q1|N|N|N|2420924|2420983|2420577|2420851|N|N|N|N|N| +2420944|AAAAAAAAANAPECAA|1916-03-21|194|847|66|1916|2|3|21|1|1916|66|847|Tuesday|1916Q1|N|N|N|2420924|2420983|2420578|2420852|N|N|N|N|N| +2420945|AAAAAAAABNAPECAA|1916-03-22|194|847|66|1916|3|3|22|1|1916|66|847|Wednesday|1916Q1|N|N|N|2420924|2420983|2420579|2420853|N|N|N|N|N| +2420946|AAAAAAAACNAPECAA|1916-03-23|194|847|66|1916|4|3|23|1|1916|66|847|Thursday|1916Q1|N|N|N|2420924|2420983|2420580|2420854|N|N|N|N|N| +2420947|AAAAAAAADNAPECAA|1916-03-24|194|847|66|1916|5|3|24|1|1916|66|847|Friday|1916Q1|N|Y|N|2420924|2420983|2420581|2420855|N|N|N|N|N| +2420948|AAAAAAAAENAPECAA|1916-03-25|194|847|66|1916|6|3|25|1|1916|66|847|Saturday|1916Q1|N|Y|N|2420924|2420983|2420582|2420856|N|N|N|N|N| +2420949|AAAAAAAAFNAPECAA|1916-03-26|194|847|66|1916|0|3|26|1|1916|66|847|Sunday|1916Q1|N|N|N|2420924|2420983|2420583|2420857|N|N|N|N|N| +2420950|AAAAAAAAGNAPECAA|1916-03-27|194|847|66|1916|1|3|27|1|1916|66|847|Monday|1916Q1|N|N|N|2420924|2420983|2420584|2420858|N|N|N|N|N| +2420951|AAAAAAAAHNAPECAA|1916-03-28|194|848|66|1916|2|3|28|1|1916|66|848|Tuesday|1916Q1|N|N|N|2420924|2420983|2420585|2420859|N|N|N|N|N| +2420952|AAAAAAAAINAPECAA|1916-03-29|194|848|66|1916|3|3|29|1|1916|66|848|Wednesday|1916Q1|N|N|N|2420924|2420983|2420586|2420860|N|N|N|N|N| +2420953|AAAAAAAAJNAPECAA|1916-03-30|194|848|66|1916|4|3|30|1|1916|66|848|Thursday|1916Q1|N|N|N|2420924|2420983|2420587|2420861|N|N|N|N|N| +2420954|AAAAAAAAKNAPECAA|1916-03-31|194|848|66|1916|5|3|31|1|1916|66|848|Friday|1916Q1|N|Y|N|2420924|2420983|2420588|2420862|N|N|N|N|N| +2420955|AAAAAAAALNAPECAA|1916-04-01|195|848|66|1916|6|4|1|2|1916|66|848|Saturday|1916Q2|N|Y|N|2420955|2421045|2420589|2420864|N|N|N|N|N| +2420956|AAAAAAAAMNAPECAA|1916-04-02|195|848|66|1916|0|4|2|2|1916|66|848|Sunday|1916Q2|N|N|N|2420955|2421045|2420590|2420865|N|N|N|N|N| +2420957|AAAAAAAANNAPECAA|1916-04-03|195|848|66|1916|1|4|3|2|1916|66|848|Monday|1916Q2|N|N|N|2420955|2421045|2420591|2420866|N|N|N|N|N| +2420958|AAAAAAAAONAPECAA|1916-04-04|195|849|66|1916|2|4|4|2|1916|66|849|Tuesday|1916Q2|N|N|N|2420955|2421045|2420592|2420867|N|N|N|N|N| +2420959|AAAAAAAAPNAPECAA|1916-04-05|195|849|66|1916|3|4|5|2|1916|66|849|Wednesday|1916Q2|N|N|N|2420955|2421045|2420593|2420868|N|N|N|N|N| +2420960|AAAAAAAAAOAPECAA|1916-04-06|195|849|66|1916|4|4|6|2|1916|66|849|Thursday|1916Q2|N|N|N|2420955|2421045|2420594|2420869|N|N|N|N|N| +2420961|AAAAAAAABOAPECAA|1916-04-07|195|849|66|1916|5|4|7|2|1916|66|849|Friday|1916Q2|N|Y|N|2420955|2421045|2420595|2420870|N|N|N|N|N| +2420962|AAAAAAAACOAPECAA|1916-04-08|195|849|66|1916|6|4|8|2|1916|66|849|Saturday|1916Q2|N|Y|N|2420955|2421045|2420596|2420871|N|N|N|N|N| +2420963|AAAAAAAADOAPECAA|1916-04-09|195|849|66|1916|0|4|9|2|1916|66|849|Sunday|1916Q2|N|N|N|2420955|2421045|2420597|2420872|N|N|N|N|N| +2420964|AAAAAAAAEOAPECAA|1916-04-10|195|849|66|1916|1|4|10|2|1916|66|849|Monday|1916Q2|N|N|N|2420955|2421045|2420598|2420873|N|N|N|N|N| +2420965|AAAAAAAAFOAPECAA|1916-04-11|195|850|66|1916|2|4|11|2|1916|66|850|Tuesday|1916Q2|N|N|N|2420955|2421045|2420599|2420874|N|N|N|N|N| +2420966|AAAAAAAAGOAPECAA|1916-04-12|195|850|66|1916|3|4|12|2|1916|66|850|Wednesday|1916Q2|N|N|N|2420955|2421045|2420600|2420875|N|N|N|N|N| +2420967|AAAAAAAAHOAPECAA|1916-04-13|195|850|66|1916|4|4|13|2|1916|66|850|Thursday|1916Q2|N|N|N|2420955|2421045|2420601|2420876|N|N|N|N|N| +2420968|AAAAAAAAIOAPECAA|1916-04-14|195|850|66|1916|5|4|14|2|1916|66|850|Friday|1916Q2|N|Y|N|2420955|2421045|2420602|2420877|N|N|N|N|N| +2420969|AAAAAAAAJOAPECAA|1916-04-15|195|850|66|1916|6|4|15|2|1916|66|850|Saturday|1916Q2|N|Y|N|2420955|2421045|2420603|2420878|N|N|N|N|N| +2420970|AAAAAAAAKOAPECAA|1916-04-16|195|850|66|1916|0|4|16|2|1916|66|850|Sunday|1916Q2|N|N|N|2420955|2421045|2420604|2420879|N|N|N|N|N| +2420971|AAAAAAAALOAPECAA|1916-04-17|195|850|66|1916|1|4|17|2|1916|66|850|Monday|1916Q2|N|N|N|2420955|2421045|2420605|2420880|N|N|N|N|N| +2420972|AAAAAAAAMOAPECAA|1916-04-18|195|851|66|1916|2|4|18|2|1916|66|851|Tuesday|1916Q2|N|N|N|2420955|2421045|2420606|2420881|N|N|N|N|N| +2420973|AAAAAAAANOAPECAA|1916-04-19|195|851|66|1916|3|4|19|2|1916|66|851|Wednesday|1916Q2|N|N|N|2420955|2421045|2420607|2420882|N|N|N|N|N| +2420974|AAAAAAAAOOAPECAA|1916-04-20|195|851|66|1916|4|4|20|2|1916|66|851|Thursday|1916Q2|N|N|N|2420955|2421045|2420608|2420883|N|N|N|N|N| +2420975|AAAAAAAAPOAPECAA|1916-04-21|195|851|66|1916|5|4|21|2|1916|66|851|Friday|1916Q2|N|Y|N|2420955|2421045|2420609|2420884|N|N|N|N|N| +2420976|AAAAAAAAAPAPECAA|1916-04-22|195|851|66|1916|6|4|22|2|1916|66|851|Saturday|1916Q2|N|Y|N|2420955|2421045|2420610|2420885|N|N|N|N|N| +2420977|AAAAAAAABPAPECAA|1916-04-23|195|851|66|1916|0|4|23|2|1916|66|851|Sunday|1916Q2|N|N|N|2420955|2421045|2420611|2420886|N|N|N|N|N| +2420978|AAAAAAAACPAPECAA|1916-04-24|195|851|66|1916|1|4|24|2|1916|66|851|Monday|1916Q2|N|N|N|2420955|2421045|2420612|2420887|N|N|N|N|N| +2420979|AAAAAAAADPAPECAA|1916-04-25|195|852|66|1916|2|4|25|2|1916|66|852|Tuesday|1916Q2|N|N|N|2420955|2421045|2420613|2420888|N|N|N|N|N| +2420980|AAAAAAAAEPAPECAA|1916-04-26|195|852|66|1916|3|4|26|2|1916|66|852|Wednesday|1916Q2|N|N|N|2420955|2421045|2420614|2420889|N|N|N|N|N| +2420981|AAAAAAAAFPAPECAA|1916-04-27|195|852|66|1916|4|4|27|2|1916|66|852|Thursday|1916Q2|N|N|N|2420955|2421045|2420615|2420890|N|N|N|N|N| +2420982|AAAAAAAAGPAPECAA|1916-04-28|195|852|66|1916|5|4|28|2|1916|66|852|Friday|1916Q2|N|Y|N|2420955|2421045|2420616|2420891|N|N|N|N|N| +2420983|AAAAAAAAHPAPECAA|1916-04-29|195|852|66|1916|6|4|29|2|1916|66|852|Saturday|1916Q2|N|Y|N|2420955|2421045|2420617|2420892|N|N|N|N|N| +2420984|AAAAAAAAIPAPECAA|1916-04-30|195|852|66|1916|0|4|30|2|1916|66|852|Sunday|1916Q2|N|N|N|2420955|2421045|2420618|2420893|N|N|N|N|N| +2420985|AAAAAAAAJPAPECAA|1916-05-01|196|852|66|1916|1|5|1|2|1916|66|852|Monday|1916Q2|N|N|N|2420985|2421105|2420619|2420894|N|N|N|N|N| +2420986|AAAAAAAAKPAPECAA|1916-05-02|196|853|66|1916|2|5|2|2|1916|66|853|Tuesday|1916Q2|N|N|N|2420985|2421105|2420620|2420895|N|N|N|N|N| +2420987|AAAAAAAALPAPECAA|1916-05-03|196|853|66|1916|3|5|3|2|1916|66|853|Wednesday|1916Q2|N|N|N|2420985|2421105|2420621|2420896|N|N|N|N|N| +2420988|AAAAAAAAMPAPECAA|1916-05-04|196|853|66|1916|4|5|4|2|1916|66|853|Thursday|1916Q2|N|N|N|2420985|2421105|2420622|2420897|N|N|N|N|N| +2420989|AAAAAAAANPAPECAA|1916-05-05|196|853|66|1916|5|5|5|2|1916|66|853|Friday|1916Q2|N|Y|N|2420985|2421105|2420623|2420898|N|N|N|N|N| +2420990|AAAAAAAAOPAPECAA|1916-05-06|196|853|66|1916|6|5|6|2|1916|66|853|Saturday|1916Q2|N|Y|N|2420985|2421105|2420624|2420899|N|N|N|N|N| +2420991|AAAAAAAAPPAPECAA|1916-05-07|196|853|66|1916|0|5|7|2|1916|66|853|Sunday|1916Q2|N|N|N|2420985|2421105|2420625|2420900|N|N|N|N|N| +2420992|AAAAAAAAAABPECAA|1916-05-08|196|853|66|1916|1|5|8|2|1916|66|853|Monday|1916Q2|N|N|N|2420985|2421105|2420626|2420901|N|N|N|N|N| +2420993|AAAAAAAABABPECAA|1916-05-09|196|854|66|1916|2|5|9|2|1916|66|854|Tuesday|1916Q2|N|N|N|2420985|2421105|2420627|2420902|N|N|N|N|N| +2420994|AAAAAAAACABPECAA|1916-05-10|196|854|66|1916|3|5|10|2|1916|66|854|Wednesday|1916Q2|N|N|N|2420985|2421105|2420628|2420903|N|N|N|N|N| +2420995|AAAAAAAADABPECAA|1916-05-11|196|854|66|1916|4|5|11|2|1916|66|854|Thursday|1916Q2|N|N|N|2420985|2421105|2420629|2420904|N|N|N|N|N| +2420996|AAAAAAAAEABPECAA|1916-05-12|196|854|66|1916|5|5|12|2|1916|66|854|Friday|1916Q2|N|Y|N|2420985|2421105|2420630|2420905|N|N|N|N|N| +2420997|AAAAAAAAFABPECAA|1916-05-13|196|854|66|1916|6|5|13|2|1916|66|854|Saturday|1916Q2|N|Y|N|2420985|2421105|2420631|2420906|N|N|N|N|N| +2420998|AAAAAAAAGABPECAA|1916-05-14|196|854|66|1916|0|5|14|2|1916|66|854|Sunday|1916Q2|N|N|N|2420985|2421105|2420632|2420907|N|N|N|N|N| +2420999|AAAAAAAAHABPECAA|1916-05-15|196|854|66|1916|1|5|15|2|1916|66|854|Monday|1916Q2|N|N|N|2420985|2421105|2420633|2420908|N|N|N|N|N| +2421000|AAAAAAAAIABPECAA|1916-05-16|196|855|66|1916|2|5|16|2|1916|66|855|Tuesday|1916Q2|N|N|N|2420985|2421105|2420634|2420909|N|N|N|N|N| +2421001|AAAAAAAAJABPECAA|1916-05-17|196|855|66|1916|3|5|17|2|1916|66|855|Wednesday|1916Q2|N|N|N|2420985|2421105|2420635|2420910|N|N|N|N|N| +2421002|AAAAAAAAKABPECAA|1916-05-18|196|855|66|1916|4|5|18|2|1916|66|855|Thursday|1916Q2|N|N|N|2420985|2421105|2420636|2420911|N|N|N|N|N| +2421003|AAAAAAAALABPECAA|1916-05-19|196|855|66|1916|5|5|19|2|1916|66|855|Friday|1916Q2|N|Y|N|2420985|2421105|2420637|2420912|N|N|N|N|N| +2421004|AAAAAAAAMABPECAA|1916-05-20|196|855|66|1916|6|5|20|2|1916|66|855|Saturday|1916Q2|N|Y|N|2420985|2421105|2420638|2420913|N|N|N|N|N| +2421005|AAAAAAAANABPECAA|1916-05-21|196|855|66|1916|0|5|21|2|1916|66|855|Sunday|1916Q2|N|N|N|2420985|2421105|2420639|2420914|N|N|N|N|N| +2421006|AAAAAAAAOABPECAA|1916-05-22|196|855|66|1916|1|5|22|2|1916|66|855|Monday|1916Q2|N|N|N|2420985|2421105|2420640|2420915|N|N|N|N|N| +2421007|AAAAAAAAPABPECAA|1916-05-23|196|856|66|1916|2|5|23|2|1916|66|856|Tuesday|1916Q2|N|N|N|2420985|2421105|2420641|2420916|N|N|N|N|N| +2421008|AAAAAAAAABBPECAA|1916-05-24|196|856|66|1916|3|5|24|2|1916|66|856|Wednesday|1916Q2|N|N|N|2420985|2421105|2420642|2420917|N|N|N|N|N| +2421009|AAAAAAAABBBPECAA|1916-05-25|196|856|66|1916|4|5|25|2|1916|66|856|Thursday|1916Q2|N|N|N|2420985|2421105|2420643|2420918|N|N|N|N|N| +2421010|AAAAAAAACBBPECAA|1916-05-26|196|856|66|1916|5|5|26|2|1916|66|856|Friday|1916Q2|N|Y|N|2420985|2421105|2420644|2420919|N|N|N|N|N| +2421011|AAAAAAAADBBPECAA|1916-05-27|196|856|66|1916|6|5|27|2|1916|66|856|Saturday|1916Q2|N|Y|N|2420985|2421105|2420645|2420920|N|N|N|N|N| +2421012|AAAAAAAAEBBPECAA|1916-05-28|196|856|66|1916|0|5|28|2|1916|66|856|Sunday|1916Q2|N|N|N|2420985|2421105|2420646|2420921|N|N|N|N|N| +2421013|AAAAAAAAFBBPECAA|1916-05-29|196|856|66|1916|1|5|29|2|1916|66|856|Monday|1916Q2|N|N|N|2420985|2421105|2420647|2420922|N|N|N|N|N| +2421014|AAAAAAAAGBBPECAA|1916-05-30|196|857|66|1916|2|5|30|2|1916|66|857|Tuesday|1916Q2|N|N|N|2420985|2421105|2420648|2420923|N|N|N|N|N| +2421015|AAAAAAAAHBBPECAA|1916-05-31|196|857|66|1916|3|5|31|2|1916|66|857|Wednesday|1916Q2|N|N|N|2420985|2421105|2420649|2420924|N|N|N|N|N| +2421016|AAAAAAAAIBBPECAA|1916-06-01|197|857|67|1916|4|6|1|2|1916|67|857|Thursday|1916Q2|N|N|N|2421016|2421167|2420650|2420925|N|N|N|N|N| +2421017|AAAAAAAAJBBPECAA|1916-06-02|197|857|67|1916|5|6|2|2|1916|67|857|Friday|1916Q2|N|Y|N|2421016|2421167|2420651|2420926|N|N|N|N|N| +2421018|AAAAAAAAKBBPECAA|1916-06-03|197|857|67|1916|6|6|3|2|1916|67|857|Saturday|1916Q2|N|Y|N|2421016|2421167|2420652|2420927|N|N|N|N|N| +2421019|AAAAAAAALBBPECAA|1916-06-04|197|857|67|1916|0|6|4|2|1916|67|857|Sunday|1916Q2|N|N|N|2421016|2421167|2420653|2420928|N|N|N|N|N| +2421020|AAAAAAAAMBBPECAA|1916-06-05|197|857|67|1916|1|6|5|2|1916|67|857|Monday|1916Q2|N|N|N|2421016|2421167|2420654|2420929|N|N|N|N|N| +2421021|AAAAAAAANBBPECAA|1916-06-06|197|858|67|1916|2|6|6|2|1916|67|858|Tuesday|1916Q2|N|N|N|2421016|2421167|2420655|2420930|N|N|N|N|N| +2421022|AAAAAAAAOBBPECAA|1916-06-07|197|858|67|1916|3|6|7|2|1916|67|858|Wednesday|1916Q2|N|N|N|2421016|2421167|2420656|2420931|N|N|N|N|N| +2421023|AAAAAAAAPBBPECAA|1916-06-08|197|858|67|1916|4|6|8|2|1916|67|858|Thursday|1916Q2|N|N|N|2421016|2421167|2420657|2420932|N|N|N|N|N| +2421024|AAAAAAAAACBPECAA|1916-06-09|197|858|67|1916|5|6|9|2|1916|67|858|Friday|1916Q2|N|Y|N|2421016|2421167|2420658|2420933|N|N|N|N|N| +2421025|AAAAAAAABCBPECAA|1916-06-10|197|858|67|1916|6|6|10|2|1916|67|858|Saturday|1916Q2|N|Y|N|2421016|2421167|2420659|2420934|N|N|N|N|N| +2421026|AAAAAAAACCBPECAA|1916-06-11|197|858|67|1916|0|6|11|2|1916|67|858|Sunday|1916Q2|N|N|N|2421016|2421167|2420660|2420935|N|N|N|N|N| +2421027|AAAAAAAADCBPECAA|1916-06-12|197|858|67|1916|1|6|12|2|1916|67|858|Monday|1916Q2|N|N|N|2421016|2421167|2420661|2420936|N|N|N|N|N| +2421028|AAAAAAAAECBPECAA|1916-06-13|197|859|67|1916|2|6|13|2|1916|67|859|Tuesday|1916Q2|N|N|N|2421016|2421167|2420662|2420937|N|N|N|N|N| +2421029|AAAAAAAAFCBPECAA|1916-06-14|197|859|67|1916|3|6|14|2|1916|67|859|Wednesday|1916Q2|N|N|N|2421016|2421167|2420663|2420938|N|N|N|N|N| +2421030|AAAAAAAAGCBPECAA|1916-06-15|197|859|67|1916|4|6|15|2|1916|67|859|Thursday|1916Q2|N|N|N|2421016|2421167|2420664|2420939|N|N|N|N|N| +2421031|AAAAAAAAHCBPECAA|1916-06-16|197|859|67|1916|5|6|16|2|1916|67|859|Friday|1916Q2|N|Y|N|2421016|2421167|2420665|2420940|N|N|N|N|N| +2421032|AAAAAAAAICBPECAA|1916-06-17|197|859|67|1916|6|6|17|2|1916|67|859|Saturday|1916Q2|N|Y|N|2421016|2421167|2420666|2420941|N|N|N|N|N| +2421033|AAAAAAAAJCBPECAA|1916-06-18|197|859|67|1916|0|6|18|2|1916|67|859|Sunday|1916Q2|N|N|N|2421016|2421167|2420667|2420942|N|N|N|N|N| +2421034|AAAAAAAAKCBPECAA|1916-06-19|197|859|67|1916|1|6|19|2|1916|67|859|Monday|1916Q2|N|N|N|2421016|2421167|2420668|2420943|N|N|N|N|N| +2421035|AAAAAAAALCBPECAA|1916-06-20|197|860|67|1916|2|6|20|2|1916|67|860|Tuesday|1916Q2|N|N|N|2421016|2421167|2420669|2420944|N|N|N|N|N| +2421036|AAAAAAAAMCBPECAA|1916-06-21|197|860|67|1916|3|6|21|2|1916|67|860|Wednesday|1916Q2|N|N|N|2421016|2421167|2420670|2420945|N|N|N|N|N| +2421037|AAAAAAAANCBPECAA|1916-06-22|197|860|67|1916|4|6|22|2|1916|67|860|Thursday|1916Q2|N|N|N|2421016|2421167|2420671|2420946|N|N|N|N|N| +2421038|AAAAAAAAOCBPECAA|1916-06-23|197|860|67|1916|5|6|23|2|1916|67|860|Friday|1916Q2|N|Y|N|2421016|2421167|2420672|2420947|N|N|N|N|N| +2421039|AAAAAAAAPCBPECAA|1916-06-24|197|860|67|1916|6|6|24|2|1916|67|860|Saturday|1916Q2|N|Y|N|2421016|2421167|2420673|2420948|N|N|N|N|N| +2421040|AAAAAAAAADBPECAA|1916-06-25|197|860|67|1916|0|6|25|2|1916|67|860|Sunday|1916Q2|N|N|N|2421016|2421167|2420674|2420949|N|N|N|N|N| +2421041|AAAAAAAABDBPECAA|1916-06-26|197|860|67|1916|1|6|26|2|1916|67|860|Monday|1916Q2|N|N|N|2421016|2421167|2420675|2420950|N|N|N|N|N| +2421042|AAAAAAAACDBPECAA|1916-06-27|197|861|67|1916|2|6|27|2|1916|67|861|Tuesday|1916Q2|N|N|N|2421016|2421167|2420676|2420951|N|N|N|N|N| +2421043|AAAAAAAADDBPECAA|1916-06-28|197|861|67|1916|3|6|28|2|1916|67|861|Wednesday|1916Q2|N|N|N|2421016|2421167|2420677|2420952|N|N|N|N|N| +2421044|AAAAAAAAEDBPECAA|1916-06-29|197|861|67|1916|4|6|29|2|1916|67|861|Thursday|1916Q2|N|N|N|2421016|2421167|2420678|2420953|N|N|N|N|N| +2421045|AAAAAAAAFDBPECAA|1916-06-30|197|861|67|1916|5|6|30|2|1916|67|861|Friday|1916Q2|N|Y|N|2421016|2421167|2420679|2420954|N|N|N|N|N| +2421046|AAAAAAAAGDBPECAA|1916-07-01|198|861|67|1916|6|7|1|3|1916|67|861|Saturday|1916Q3|N|Y|N|2421046|2421227|2420680|2420955|N|N|N|N|N| +2421047|AAAAAAAAHDBPECAA|1916-07-02|198|861|67|1916|0|7|2|3|1916|67|861|Sunday|1916Q3|N|N|N|2421046|2421227|2420681|2420956|N|N|N|N|N| +2421048|AAAAAAAAIDBPECAA|1916-07-03|198|861|67|1916|1|7|3|3|1916|67|861|Monday|1916Q3|N|N|N|2421046|2421227|2420682|2420957|N|N|N|N|N| +2421049|AAAAAAAAJDBPECAA|1916-07-04|198|862|67|1916|2|7|4|3|1916|67|862|Tuesday|1916Q3|Y|N|N|2421046|2421227|2420683|2420958|N|N|N|N|N| +2421050|AAAAAAAAKDBPECAA|1916-07-05|198|862|67|1916|3|7|5|3|1916|67|862|Wednesday|1916Q3|N|N|Y|2421046|2421227|2420684|2420959|N|N|N|N|N| +2421051|AAAAAAAALDBPECAA|1916-07-06|198|862|67|1916|4|7|6|3|1916|67|862|Thursday|1916Q3|N|N|N|2421046|2421227|2420685|2420960|N|N|N|N|N| +2421052|AAAAAAAAMDBPECAA|1916-07-07|198|862|67|1916|5|7|7|3|1916|67|862|Friday|1916Q3|N|Y|N|2421046|2421227|2420686|2420961|N|N|N|N|N| +2421053|AAAAAAAANDBPECAA|1916-07-08|198|862|67|1916|6|7|8|3|1916|67|862|Saturday|1916Q3|N|Y|N|2421046|2421227|2420687|2420962|N|N|N|N|N| +2421054|AAAAAAAAODBPECAA|1916-07-09|198|862|67|1916|0|7|9|3|1916|67|862|Sunday|1916Q3|N|N|N|2421046|2421227|2420688|2420963|N|N|N|N|N| +2421055|AAAAAAAAPDBPECAA|1916-07-10|198|862|67|1916|1|7|10|3|1916|67|862|Monday|1916Q3|N|N|N|2421046|2421227|2420689|2420964|N|N|N|N|N| +2421056|AAAAAAAAAEBPECAA|1916-07-11|198|863|67|1916|2|7|11|3|1916|67|863|Tuesday|1916Q3|N|N|N|2421046|2421227|2420690|2420965|N|N|N|N|N| +2421057|AAAAAAAABEBPECAA|1916-07-12|198|863|67|1916|3|7|12|3|1916|67|863|Wednesday|1916Q3|N|N|N|2421046|2421227|2420691|2420966|N|N|N|N|N| +2421058|AAAAAAAACEBPECAA|1916-07-13|198|863|67|1916|4|7|13|3|1916|67|863|Thursday|1916Q3|N|N|N|2421046|2421227|2420692|2420967|N|N|N|N|N| +2421059|AAAAAAAADEBPECAA|1916-07-14|198|863|67|1916|5|7|14|3|1916|67|863|Friday|1916Q3|N|Y|N|2421046|2421227|2420693|2420968|N|N|N|N|N| +2421060|AAAAAAAAEEBPECAA|1916-07-15|198|863|67|1916|6|7|15|3|1916|67|863|Saturday|1916Q3|N|Y|N|2421046|2421227|2420694|2420969|N|N|N|N|N| +2421061|AAAAAAAAFEBPECAA|1916-07-16|198|863|67|1916|0|7|16|3|1916|67|863|Sunday|1916Q3|N|N|N|2421046|2421227|2420695|2420970|N|N|N|N|N| +2421062|AAAAAAAAGEBPECAA|1916-07-17|198|863|67|1916|1|7|17|3|1916|67|863|Monday|1916Q3|N|N|N|2421046|2421227|2420696|2420971|N|N|N|N|N| +2421063|AAAAAAAAHEBPECAA|1916-07-18|198|864|67|1916|2|7|18|3|1916|67|864|Tuesday|1916Q3|N|N|N|2421046|2421227|2420697|2420972|N|N|N|N|N| +2421064|AAAAAAAAIEBPECAA|1916-07-19|198|864|67|1916|3|7|19|3|1916|67|864|Wednesday|1916Q3|N|N|N|2421046|2421227|2420698|2420973|N|N|N|N|N| +2421065|AAAAAAAAJEBPECAA|1916-07-20|198|864|67|1916|4|7|20|3|1916|67|864|Thursday|1916Q3|N|N|N|2421046|2421227|2420699|2420974|N|N|N|N|N| +2421066|AAAAAAAAKEBPECAA|1916-07-21|198|864|67|1916|5|7|21|3|1916|67|864|Friday|1916Q3|N|Y|N|2421046|2421227|2420700|2420975|N|N|N|N|N| +2421067|AAAAAAAALEBPECAA|1916-07-22|198|864|67|1916|6|7|22|3|1916|67|864|Saturday|1916Q3|N|Y|N|2421046|2421227|2420701|2420976|N|N|N|N|N| +2421068|AAAAAAAAMEBPECAA|1916-07-23|198|864|67|1916|0|7|23|3|1916|67|864|Sunday|1916Q3|N|N|N|2421046|2421227|2420702|2420977|N|N|N|N|N| +2421069|AAAAAAAANEBPECAA|1916-07-24|198|864|67|1916|1|7|24|3|1916|67|864|Monday|1916Q3|N|N|N|2421046|2421227|2420703|2420978|N|N|N|N|N| +2421070|AAAAAAAAOEBPECAA|1916-07-25|198|865|67|1916|2|7|25|3|1916|67|865|Tuesday|1916Q3|N|N|N|2421046|2421227|2420704|2420979|N|N|N|N|N| +2421071|AAAAAAAAPEBPECAA|1916-07-26|198|865|67|1916|3|7|26|3|1916|67|865|Wednesday|1916Q3|N|N|N|2421046|2421227|2420705|2420980|N|N|N|N|N| +2421072|AAAAAAAAAFBPECAA|1916-07-27|198|865|67|1916|4|7|27|3|1916|67|865|Thursday|1916Q3|N|N|N|2421046|2421227|2420706|2420981|N|N|N|N|N| +2421073|AAAAAAAABFBPECAA|1916-07-28|198|865|67|1916|5|7|28|3|1916|67|865|Friday|1916Q3|N|Y|N|2421046|2421227|2420707|2420982|N|N|N|N|N| +2421074|AAAAAAAACFBPECAA|1916-07-29|198|865|67|1916|6|7|29|3|1916|67|865|Saturday|1916Q3|N|Y|N|2421046|2421227|2420708|2420983|N|N|N|N|N| +2421075|AAAAAAAADFBPECAA|1916-07-30|198|865|67|1916|0|7|30|3|1916|67|865|Sunday|1916Q3|N|N|N|2421046|2421227|2420709|2420984|N|N|N|N|N| +2421076|AAAAAAAAEFBPECAA|1916-07-31|198|865|67|1916|1|7|31|3|1916|67|865|Monday|1916Q3|N|N|N|2421046|2421227|2420710|2420985|N|N|N|N|N| +2421077|AAAAAAAAFFBPECAA|1916-08-01|199|866|67|1916|2|8|1|3|1916|67|866|Tuesday|1916Q3|N|N|N|2421077|2421289|2420711|2420986|N|N|N|N|N| +2421078|AAAAAAAAGFBPECAA|1916-08-02|199|866|67|1916|3|8|2|3|1916|67|866|Wednesday|1916Q3|N|N|N|2421077|2421289|2420712|2420987|N|N|N|N|N| +2421079|AAAAAAAAHFBPECAA|1916-08-03|199|866|67|1916|4|8|3|3|1916|67|866|Thursday|1916Q3|N|N|N|2421077|2421289|2420713|2420988|N|N|N|N|N| +2421080|AAAAAAAAIFBPECAA|1916-08-04|199|866|67|1916|5|8|4|3|1916|67|866|Friday|1916Q3|N|Y|N|2421077|2421289|2420714|2420989|N|N|N|N|N| +2421081|AAAAAAAAJFBPECAA|1916-08-05|199|866|67|1916|6|8|5|3|1916|67|866|Saturday|1916Q3|N|Y|N|2421077|2421289|2420715|2420990|N|N|N|N|N| +2421082|AAAAAAAAKFBPECAA|1916-08-06|199|866|67|1916|0|8|6|3|1916|67|866|Sunday|1916Q3|N|N|N|2421077|2421289|2420716|2420991|N|N|N|N|N| +2421083|AAAAAAAALFBPECAA|1916-08-07|199|866|67|1916|1|8|7|3|1916|67|866|Monday|1916Q3|N|N|N|2421077|2421289|2420717|2420992|N|N|N|N|N| +2421084|AAAAAAAAMFBPECAA|1916-08-08|199|867|67|1916|2|8|8|3|1916|67|867|Tuesday|1916Q3|N|N|N|2421077|2421289|2420718|2420993|N|N|N|N|N| +2421085|AAAAAAAANFBPECAA|1916-08-09|199|867|67|1916|3|8|9|3|1916|67|867|Wednesday|1916Q3|N|N|N|2421077|2421289|2420719|2420994|N|N|N|N|N| +2421086|AAAAAAAAOFBPECAA|1916-08-10|199|867|67|1916|4|8|10|3|1916|67|867|Thursday|1916Q3|N|N|N|2421077|2421289|2420720|2420995|N|N|N|N|N| +2421087|AAAAAAAAPFBPECAA|1916-08-11|199|867|67|1916|5|8|11|3|1916|67|867|Friday|1916Q3|N|Y|N|2421077|2421289|2420721|2420996|N|N|N|N|N| +2421088|AAAAAAAAAGBPECAA|1916-08-12|199|867|67|1916|6|8|12|3|1916|67|867|Saturday|1916Q3|N|Y|N|2421077|2421289|2420722|2420997|N|N|N|N|N| +2421089|AAAAAAAABGBPECAA|1916-08-13|199|867|67|1916|0|8|13|3|1916|67|867|Sunday|1916Q3|N|N|N|2421077|2421289|2420723|2420998|N|N|N|N|N| +2421090|AAAAAAAACGBPECAA|1916-08-14|199|867|67|1916|1|8|14|3|1916|67|867|Monday|1916Q3|N|N|N|2421077|2421289|2420724|2420999|N|N|N|N|N| +2421091|AAAAAAAADGBPECAA|1916-08-15|199|868|67|1916|2|8|15|3|1916|67|868|Tuesday|1916Q3|N|N|N|2421077|2421289|2420725|2421000|N|N|N|N|N| +2421092|AAAAAAAAEGBPECAA|1916-08-16|199|868|67|1916|3|8|16|3|1916|67|868|Wednesday|1916Q3|N|N|N|2421077|2421289|2420726|2421001|N|N|N|N|N| +2421093|AAAAAAAAFGBPECAA|1916-08-17|199|868|67|1916|4|8|17|3|1916|67|868|Thursday|1916Q3|N|N|N|2421077|2421289|2420727|2421002|N|N|N|N|N| +2421094|AAAAAAAAGGBPECAA|1916-08-18|199|868|67|1916|5|8|18|3|1916|67|868|Friday|1916Q3|N|Y|N|2421077|2421289|2420728|2421003|N|N|N|N|N| +2421095|AAAAAAAAHGBPECAA|1916-08-19|199|868|67|1916|6|8|19|3|1916|67|868|Saturday|1916Q3|N|Y|N|2421077|2421289|2420729|2421004|N|N|N|N|N| +2421096|AAAAAAAAIGBPECAA|1916-08-20|199|868|67|1916|0|8|20|3|1916|67|868|Sunday|1916Q3|N|N|N|2421077|2421289|2420730|2421005|N|N|N|N|N| +2421097|AAAAAAAAJGBPECAA|1916-08-21|199|868|67|1916|1|8|21|3|1916|67|868|Monday|1916Q3|N|N|N|2421077|2421289|2420731|2421006|N|N|N|N|N| +2421098|AAAAAAAAKGBPECAA|1916-08-22|199|869|67|1916|2|8|22|3|1916|67|869|Tuesday|1916Q3|N|N|N|2421077|2421289|2420732|2421007|N|N|N|N|N| +2421099|AAAAAAAALGBPECAA|1916-08-23|199|869|67|1916|3|8|23|3|1916|67|869|Wednesday|1916Q3|N|N|N|2421077|2421289|2420733|2421008|N|N|N|N|N| +2421100|AAAAAAAAMGBPECAA|1916-08-24|199|869|67|1916|4|8|24|3|1916|67|869|Thursday|1916Q3|N|N|N|2421077|2421289|2420734|2421009|N|N|N|N|N| +2421101|AAAAAAAANGBPECAA|1916-08-25|199|869|67|1916|5|8|25|3|1916|67|869|Friday|1916Q3|N|Y|N|2421077|2421289|2420735|2421010|N|N|N|N|N| +2421102|AAAAAAAAOGBPECAA|1916-08-26|199|869|67|1916|6|8|26|3|1916|67|869|Saturday|1916Q3|N|Y|N|2421077|2421289|2420736|2421011|N|N|N|N|N| +2421103|AAAAAAAAPGBPECAA|1916-08-27|199|869|67|1916|0|8|27|3|1916|67|869|Sunday|1916Q3|N|N|N|2421077|2421289|2420737|2421012|N|N|N|N|N| +2421104|AAAAAAAAAHBPECAA|1916-08-28|199|869|67|1916|1|8|28|3|1916|67|869|Monday|1916Q3|N|N|N|2421077|2421289|2420738|2421013|N|N|N|N|N| +2421105|AAAAAAAABHBPECAA|1916-08-29|199|870|67|1916|2|8|29|3|1916|67|870|Tuesday|1916Q3|N|N|N|2421077|2421289|2420739|2421014|N|N|N|N|N| +2421106|AAAAAAAACHBPECAA|1916-08-30|199|870|67|1916|3|8|30|3|1916|67|870|Wednesday|1916Q3|N|N|N|2421077|2421289|2420740|2421015|N|N|N|N|N| +2421107|AAAAAAAADHBPECAA|1916-08-31|199|870|67|1916|4|8|31|3|1916|67|870|Thursday|1916Q3|N|N|N|2421077|2421289|2420741|2421016|N|N|N|N|N| +2421108|AAAAAAAAEHBPECAA|1916-09-01|200|870|68|1916|5|9|1|3|1916|68|870|Friday|1916Q3|N|Y|N|2421108|2421351|2420742|2421017|N|N|N|N|N| +2421109|AAAAAAAAFHBPECAA|1916-09-02|200|870|68|1916|6|9|2|3|1916|68|870|Saturday|1916Q3|N|Y|N|2421108|2421351|2420743|2421018|N|N|N|N|N| +2421110|AAAAAAAAGHBPECAA|1916-09-03|200|870|68|1916|0|9|3|3|1916|68|870|Sunday|1916Q3|N|N|N|2421108|2421351|2420744|2421019|N|N|N|N|N| +2421111|AAAAAAAAHHBPECAA|1916-09-04|200|870|68|1916|1|9|4|3|1916|68|870|Monday|1916Q3|N|N|N|2421108|2421351|2420745|2421020|N|N|N|N|N| +2421112|AAAAAAAAIHBPECAA|1916-09-05|200|871|68|1916|2|9|5|3|1916|68|871|Tuesday|1916Q3|N|N|N|2421108|2421351|2420746|2421021|N|N|N|N|N| +2421113|AAAAAAAAJHBPECAA|1916-09-06|200|871|68|1916|3|9|6|3|1916|68|871|Wednesday|1916Q3|N|N|N|2421108|2421351|2420747|2421022|N|N|N|N|N| +2421114|AAAAAAAAKHBPECAA|1916-09-07|200|871|68|1916|4|9|7|3|1916|68|871|Thursday|1916Q3|N|N|N|2421108|2421351|2420748|2421023|N|N|N|N|N| +2421115|AAAAAAAALHBPECAA|1916-09-08|200|871|68|1916|5|9|8|3|1916|68|871|Friday|1916Q3|N|Y|N|2421108|2421351|2420749|2421024|N|N|N|N|N| +2421116|AAAAAAAAMHBPECAA|1916-09-09|200|871|68|1916|6|9|9|3|1916|68|871|Saturday|1916Q3|N|Y|N|2421108|2421351|2420750|2421025|N|N|N|N|N| +2421117|AAAAAAAANHBPECAA|1916-09-10|200|871|68|1916|0|9|10|3|1916|68|871|Sunday|1916Q3|N|N|N|2421108|2421351|2420751|2421026|N|N|N|N|N| +2421118|AAAAAAAAOHBPECAA|1916-09-11|200|871|68|1916|1|9|11|3|1916|68|871|Monday|1916Q3|N|N|N|2421108|2421351|2420752|2421027|N|N|N|N|N| +2421119|AAAAAAAAPHBPECAA|1916-09-12|200|872|68|1916|2|9|12|3|1916|68|872|Tuesday|1916Q3|N|N|N|2421108|2421351|2420753|2421028|N|N|N|N|N| +2421120|AAAAAAAAAIBPECAA|1916-09-13|200|872|68|1916|3|9|13|3|1916|68|872|Wednesday|1916Q3|N|N|N|2421108|2421351|2420754|2421029|N|N|N|N|N| +2421121|AAAAAAAABIBPECAA|1916-09-14|200|872|68|1916|4|9|14|3|1916|68|872|Thursday|1916Q3|N|N|N|2421108|2421351|2420755|2421030|N|N|N|N|N| +2421122|AAAAAAAACIBPECAA|1916-09-15|200|872|68|1916|5|9|15|3|1916|68|872|Friday|1916Q3|N|Y|N|2421108|2421351|2420756|2421031|N|N|N|N|N| +2421123|AAAAAAAADIBPECAA|1916-09-16|200|872|68|1916|6|9|16|3|1916|68|872|Saturday|1916Q3|N|Y|N|2421108|2421351|2420757|2421032|N|N|N|N|N| +2421124|AAAAAAAAEIBPECAA|1916-09-17|200|872|68|1916|0|9|17|3|1916|68|872|Sunday|1916Q3|N|N|N|2421108|2421351|2420758|2421033|N|N|N|N|N| +2421125|AAAAAAAAFIBPECAA|1916-09-18|200|872|68|1916|1|9|18|3|1916|68|872|Monday|1916Q3|N|N|N|2421108|2421351|2420759|2421034|N|N|N|N|N| +2421126|AAAAAAAAGIBPECAA|1916-09-19|200|873|68|1916|2|9|19|3|1916|68|873|Tuesday|1916Q3|N|N|N|2421108|2421351|2420760|2421035|N|N|N|N|N| +2421127|AAAAAAAAHIBPECAA|1916-09-20|200|873|68|1916|3|9|20|3|1916|68|873|Wednesday|1916Q3|N|N|N|2421108|2421351|2420761|2421036|N|N|N|N|N| +2421128|AAAAAAAAIIBPECAA|1916-09-21|200|873|68|1916|4|9|21|3|1916|68|873|Thursday|1916Q3|N|N|N|2421108|2421351|2420762|2421037|N|N|N|N|N| +2421129|AAAAAAAAJIBPECAA|1916-09-22|200|873|68|1916|5|9|22|3|1916|68|873|Friday|1916Q3|N|Y|N|2421108|2421351|2420763|2421038|N|N|N|N|N| +2421130|AAAAAAAAKIBPECAA|1916-09-23|200|873|68|1916|6|9|23|3|1916|68|873|Saturday|1916Q3|N|Y|N|2421108|2421351|2420764|2421039|N|N|N|N|N| +2421131|AAAAAAAALIBPECAA|1916-09-24|200|873|68|1916|0|9|24|3|1916|68|873|Sunday|1916Q3|N|N|N|2421108|2421351|2420765|2421040|N|N|N|N|N| +2421132|AAAAAAAAMIBPECAA|1916-09-25|200|873|68|1916|1|9|25|3|1916|68|873|Monday|1916Q3|N|N|N|2421108|2421351|2420766|2421041|N|N|N|N|N| +2421133|AAAAAAAANIBPECAA|1916-09-26|200|874|68|1916|2|9|26|3|1916|68|874|Tuesday|1916Q3|N|N|N|2421108|2421351|2420767|2421042|N|N|N|N|N| +2421134|AAAAAAAAOIBPECAA|1916-09-27|200|874|68|1916|3|9|27|3|1916|68|874|Wednesday|1916Q3|N|N|N|2421108|2421351|2420768|2421043|N|N|N|N|N| +2421135|AAAAAAAAPIBPECAA|1916-09-28|200|874|68|1916|4|9|28|3|1916|68|874|Thursday|1916Q3|N|N|N|2421108|2421351|2420769|2421044|N|N|N|N|N| +2421136|AAAAAAAAAJBPECAA|1916-09-29|200|874|68|1916|5|9|29|3|1916|68|874|Friday|1916Q3|N|Y|N|2421108|2421351|2420770|2421045|N|N|N|N|N| +2421137|AAAAAAAABJBPECAA|1916-09-30|200|874|68|1916|6|9|30|3|1916|68|874|Saturday|1916Q3|N|Y|N|2421108|2421351|2420771|2421046|N|N|N|N|N| +2421138|AAAAAAAACJBPECAA|1916-10-01|201|874|68|1916|0|10|1|4|1916|68|874|Sunday|1916Q4|N|N|N|2421138|2421411|2420772|2421046|N|N|N|N|N| +2421139|AAAAAAAADJBPECAA|1916-10-02|201|874|68|1916|1|10|2|4|1916|68|874|Monday|1916Q4|N|N|N|2421138|2421411|2420773|2421047|N|N|N|N|N| +2421140|AAAAAAAAEJBPECAA|1916-10-03|201|875|68|1916|2|10|3|4|1916|68|875|Tuesday|1916Q4|N|N|N|2421138|2421411|2420774|2421048|N|N|N|N|N| +2421141|AAAAAAAAFJBPECAA|1916-10-04|201|875|68|1916|3|10|4|4|1916|68|875|Wednesday|1916Q4|N|N|N|2421138|2421411|2420775|2421049|N|N|N|N|N| +2421142|AAAAAAAAGJBPECAA|1916-10-05|201|875|68|1916|4|10|5|4|1916|68|875|Thursday|1916Q4|N|N|N|2421138|2421411|2420776|2421050|N|N|N|N|N| +2421143|AAAAAAAAHJBPECAA|1916-10-06|201|875|68|1916|5|10|6|4|1916|68|875|Friday|1916Q4|N|Y|N|2421138|2421411|2420777|2421051|N|N|N|N|N| +2421144|AAAAAAAAIJBPECAA|1916-10-07|201|875|68|1916|6|10|7|4|1916|68|875|Saturday|1916Q4|N|Y|N|2421138|2421411|2420778|2421052|N|N|N|N|N| +2421145|AAAAAAAAJJBPECAA|1916-10-08|201|875|68|1916|0|10|8|4|1916|68|875|Sunday|1916Q4|N|N|N|2421138|2421411|2420779|2421053|N|N|N|N|N| +2421146|AAAAAAAAKJBPECAA|1916-10-09|201|875|68|1916|1|10|9|4|1916|68|875|Monday|1916Q4|N|N|N|2421138|2421411|2420780|2421054|N|N|N|N|N| +2421147|AAAAAAAALJBPECAA|1916-10-10|201|876|68|1916|2|10|10|4|1916|68|876|Tuesday|1916Q4|N|N|N|2421138|2421411|2420781|2421055|N|N|N|N|N| +2421148|AAAAAAAAMJBPECAA|1916-10-11|201|876|68|1916|3|10|11|4|1916|68|876|Wednesday|1916Q4|N|N|N|2421138|2421411|2420782|2421056|N|N|N|N|N| +2421149|AAAAAAAANJBPECAA|1916-10-12|201|876|68|1916|4|10|12|4|1916|68|876|Thursday|1916Q4|N|N|N|2421138|2421411|2420783|2421057|N|N|N|N|N| +2421150|AAAAAAAAOJBPECAA|1916-10-13|201|876|68|1916|5|10|13|4|1916|68|876|Friday|1916Q4|N|Y|N|2421138|2421411|2420784|2421058|N|N|N|N|N| +2421151|AAAAAAAAPJBPECAA|1916-10-14|201|876|68|1916|6|10|14|4|1916|68|876|Saturday|1916Q4|N|Y|N|2421138|2421411|2420785|2421059|N|N|N|N|N| +2421152|AAAAAAAAAKBPECAA|1916-10-15|201|876|68|1916|0|10|15|4|1916|68|876|Sunday|1916Q4|N|N|N|2421138|2421411|2420786|2421060|N|N|N|N|N| +2421153|AAAAAAAABKBPECAA|1916-10-16|201|876|68|1916|1|10|16|4|1916|68|876|Monday|1916Q4|N|N|N|2421138|2421411|2420787|2421061|N|N|N|N|N| +2421154|AAAAAAAACKBPECAA|1916-10-17|201|877|68|1916|2|10|17|4|1916|68|877|Tuesday|1916Q4|N|N|N|2421138|2421411|2420788|2421062|N|N|N|N|N| +2421155|AAAAAAAADKBPECAA|1916-10-18|201|877|68|1916|3|10|18|4|1916|68|877|Wednesday|1916Q4|N|N|N|2421138|2421411|2420789|2421063|N|N|N|N|N| +2421156|AAAAAAAAEKBPECAA|1916-10-19|201|877|68|1916|4|10|19|4|1916|68|877|Thursday|1916Q4|N|N|N|2421138|2421411|2420790|2421064|N|N|N|N|N| +2421157|AAAAAAAAFKBPECAA|1916-10-20|201|877|68|1916|5|10|20|4|1916|68|877|Friday|1916Q4|N|Y|N|2421138|2421411|2420791|2421065|N|N|N|N|N| +2421158|AAAAAAAAGKBPECAA|1916-10-21|201|877|68|1916|6|10|21|4|1916|68|877|Saturday|1916Q4|N|Y|N|2421138|2421411|2420792|2421066|N|N|N|N|N| +2421159|AAAAAAAAHKBPECAA|1916-10-22|201|877|68|1916|0|10|22|4|1916|68|877|Sunday|1916Q4|N|N|N|2421138|2421411|2420793|2421067|N|N|N|N|N| +2421160|AAAAAAAAIKBPECAA|1916-10-23|201|877|68|1916|1|10|23|4|1916|68|877|Monday|1916Q4|N|N|N|2421138|2421411|2420794|2421068|N|N|N|N|N| +2421161|AAAAAAAAJKBPECAA|1916-10-24|201|878|68|1916|2|10|24|4|1916|68|878|Tuesday|1916Q4|N|N|N|2421138|2421411|2420795|2421069|N|N|N|N|N| +2421162|AAAAAAAAKKBPECAA|1916-10-25|201|878|68|1916|3|10|25|4|1916|68|878|Wednesday|1916Q4|N|N|N|2421138|2421411|2420796|2421070|N|N|N|N|N| +2421163|AAAAAAAALKBPECAA|1916-10-26|201|878|68|1916|4|10|26|4|1916|68|878|Thursday|1916Q4|N|N|N|2421138|2421411|2420797|2421071|N|N|N|N|N| +2421164|AAAAAAAAMKBPECAA|1916-10-27|201|878|68|1916|5|10|27|4|1916|68|878|Friday|1916Q4|N|Y|N|2421138|2421411|2420798|2421072|N|N|N|N|N| +2421165|AAAAAAAANKBPECAA|1916-10-28|201|878|68|1916|6|10|28|4|1916|68|878|Saturday|1916Q4|N|Y|N|2421138|2421411|2420799|2421073|N|N|N|N|N| +2421166|AAAAAAAAOKBPECAA|1916-10-29|201|878|68|1916|0|10|29|4|1916|68|878|Sunday|1916Q4|N|N|N|2421138|2421411|2420800|2421074|N|N|N|N|N| +2421167|AAAAAAAAPKBPECAA|1916-10-30|201|878|68|1916|1|10|30|4|1916|68|878|Monday|1916Q4|N|N|N|2421138|2421411|2420801|2421075|N|N|N|N|N| +2421168|AAAAAAAAALBPECAA|1916-10-31|201|879|68|1916|2|10|31|4|1916|68|879|Tuesday|1916Q4|N|N|N|2421138|2421411|2420802|2421076|N|N|N|N|N| +2421169|AAAAAAAABLBPECAA|1916-11-01|202|879|68|1916|3|11|1|4|1916|68|879|Wednesday|1916Q4|N|N|N|2421169|2421473|2420803|2421077|N|N|N|N|N| +2421170|AAAAAAAACLBPECAA|1916-11-02|202|879|68|1916|4|11|2|4|1916|68|879|Thursday|1916Q4|N|N|N|2421169|2421473|2420804|2421078|N|N|N|N|N| +2421171|AAAAAAAADLBPECAA|1916-11-03|202|879|68|1916|5|11|3|4|1916|68|879|Friday|1916Q4|N|Y|N|2421169|2421473|2420805|2421079|N|N|N|N|N| +2421172|AAAAAAAAELBPECAA|1916-11-04|202|879|68|1916|6|11|4|4|1916|68|879|Saturday|1916Q4|N|Y|N|2421169|2421473|2420806|2421080|N|N|N|N|N| +2421173|AAAAAAAAFLBPECAA|1916-11-05|202|879|68|1916|0|11|5|4|1916|68|879|Sunday|1916Q4|N|N|N|2421169|2421473|2420807|2421081|N|N|N|N|N| +2421174|AAAAAAAAGLBPECAA|1916-11-06|202|879|68|1916|1|11|6|4|1916|68|879|Monday|1916Q4|N|N|N|2421169|2421473|2420808|2421082|N|N|N|N|N| +2421175|AAAAAAAAHLBPECAA|1916-11-07|202|880|68|1916|2|11|7|4|1916|68|880|Tuesday|1916Q4|N|N|N|2421169|2421473|2420809|2421083|N|N|N|N|N| +2421176|AAAAAAAAILBPECAA|1916-11-08|202|880|68|1916|3|11|8|4|1916|68|880|Wednesday|1916Q4|N|N|N|2421169|2421473|2420810|2421084|N|N|N|N|N| +2421177|AAAAAAAAJLBPECAA|1916-11-09|202|880|68|1916|4|11|9|4|1916|68|880|Thursday|1916Q4|N|N|N|2421169|2421473|2420811|2421085|N|N|N|N|N| +2421178|AAAAAAAAKLBPECAA|1916-11-10|202|880|68|1916|5|11|10|4|1916|68|880|Friday|1916Q4|N|Y|N|2421169|2421473|2420812|2421086|N|N|N|N|N| +2421179|AAAAAAAALLBPECAA|1916-11-11|202|880|68|1916|6|11|11|4|1916|68|880|Saturday|1916Q4|N|Y|N|2421169|2421473|2420813|2421087|N|N|N|N|N| +2421180|AAAAAAAAMLBPECAA|1916-11-12|202|880|68|1916|0|11|12|4|1916|68|880|Sunday|1916Q4|N|N|N|2421169|2421473|2420814|2421088|N|N|N|N|N| +2421181|AAAAAAAANLBPECAA|1916-11-13|202|880|68|1916|1|11|13|4|1916|68|880|Monday|1916Q4|N|N|N|2421169|2421473|2420815|2421089|N|N|N|N|N| +2421182|AAAAAAAAOLBPECAA|1916-11-14|202|881|68|1916|2|11|14|4|1916|68|881|Tuesday|1916Q4|N|N|N|2421169|2421473|2420816|2421090|N|N|N|N|N| +2421183|AAAAAAAAPLBPECAA|1916-11-15|202|881|68|1916|3|11|15|4|1916|68|881|Wednesday|1916Q4|N|N|N|2421169|2421473|2420817|2421091|N|N|N|N|N| +2421184|AAAAAAAAAMBPECAA|1916-11-16|202|881|68|1916|4|11|16|4|1916|68|881|Thursday|1916Q4|N|N|N|2421169|2421473|2420818|2421092|N|N|N|N|N| +2421185|AAAAAAAABMBPECAA|1916-11-17|202|881|68|1916|5|11|17|4|1916|68|881|Friday|1916Q4|N|Y|N|2421169|2421473|2420819|2421093|N|N|N|N|N| +2421186|AAAAAAAACMBPECAA|1916-11-18|202|881|68|1916|6|11|18|4|1916|68|881|Saturday|1916Q4|N|Y|N|2421169|2421473|2420820|2421094|N|N|N|N|N| +2421187|AAAAAAAADMBPECAA|1916-11-19|202|881|68|1916|0|11|19|4|1916|68|881|Sunday|1916Q4|N|N|N|2421169|2421473|2420821|2421095|N|N|N|N|N| +2421188|AAAAAAAAEMBPECAA|1916-11-20|202|881|68|1916|1|11|20|4|1916|68|881|Monday|1916Q4|N|N|N|2421169|2421473|2420822|2421096|N|N|N|N|N| +2421189|AAAAAAAAFMBPECAA|1916-11-21|202|882|68|1916|2|11|21|4|1916|68|882|Tuesday|1916Q4|N|N|N|2421169|2421473|2420823|2421097|N|N|N|N|N| +2421190|AAAAAAAAGMBPECAA|1916-11-22|202|882|68|1916|3|11|22|4|1916|68|882|Wednesday|1916Q4|N|N|N|2421169|2421473|2420824|2421098|N|N|N|N|N| +2421191|AAAAAAAAHMBPECAA|1916-11-23|202|882|68|1916|4|11|23|4|1916|68|882|Thursday|1916Q4|N|N|N|2421169|2421473|2420825|2421099|N|N|N|N|N| +2421192|AAAAAAAAIMBPECAA|1916-11-24|202|882|68|1916|5|11|24|4|1916|68|882|Friday|1916Q4|N|Y|N|2421169|2421473|2420826|2421100|N|N|N|N|N| +2421193|AAAAAAAAJMBPECAA|1916-11-25|202|882|68|1916|6|11|25|4|1916|68|882|Saturday|1916Q4|N|Y|N|2421169|2421473|2420827|2421101|N|N|N|N|N| +2421194|AAAAAAAAKMBPECAA|1916-11-26|202|882|68|1916|0|11|26|4|1916|68|882|Sunday|1916Q4|N|N|N|2421169|2421473|2420828|2421102|N|N|N|N|N| +2421195|AAAAAAAALMBPECAA|1916-11-27|202|882|68|1916|1|11|27|4|1916|68|882|Monday|1916Q4|N|N|N|2421169|2421473|2420829|2421103|N|N|N|N|N| +2421196|AAAAAAAAMMBPECAA|1916-11-28|202|883|68|1916|2|11|28|4|1916|68|883|Tuesday|1916Q4|N|N|N|2421169|2421473|2420830|2421104|N|N|N|N|N| +2421197|AAAAAAAANMBPECAA|1916-11-29|202|883|68|1916|3|11|29|4|1916|68|883|Wednesday|1916Q4|N|N|N|2421169|2421473|2420831|2421105|N|N|N|N|N| +2421198|AAAAAAAAOMBPECAA|1916-11-30|202|883|68|1916|4|11|30|4|1916|68|883|Thursday|1916Q4|N|N|N|2421169|2421473|2420832|2421106|N|N|N|N|N| +2421199|AAAAAAAAPMBPECAA|1916-12-01|203|883|69|1916|5|12|1|4|1916|69|883|Friday|1916Q4|N|Y|N|2421199|2421533|2420833|2421107|N|N|N|N|N| +2421200|AAAAAAAAANBPECAA|1916-12-02|203|883|69|1916|6|12|2|4|1916|69|883|Saturday|1916Q4|N|Y|N|2421199|2421533|2420834|2421108|N|N|N|N|N| +2421201|AAAAAAAABNBPECAA|1916-12-03|203|883|69|1916|0|12|3|4|1916|69|883|Sunday|1916Q4|N|N|N|2421199|2421533|2420835|2421109|N|N|N|N|N| +2421202|AAAAAAAACNBPECAA|1916-12-04|203|883|69|1916|1|12|4|4|1916|69|883|Monday|1916Q4|N|N|N|2421199|2421533|2420836|2421110|N|N|N|N|N| +2421203|AAAAAAAADNBPECAA|1916-12-05|203|884|69|1916|2|12|5|4|1916|69|884|Tuesday|1916Q4|N|N|N|2421199|2421533|2420837|2421111|N|N|N|N|N| +2421204|AAAAAAAAENBPECAA|1916-12-06|203|884|69|1916|3|12|6|4|1916|69|884|Wednesday|1916Q4|N|N|N|2421199|2421533|2420838|2421112|N|N|N|N|N| +2421205|AAAAAAAAFNBPECAA|1916-12-07|203|884|69|1916|4|12|7|4|1916|69|884|Thursday|1916Q4|N|N|N|2421199|2421533|2420839|2421113|N|N|N|N|N| +2421206|AAAAAAAAGNBPECAA|1916-12-08|203|884|69|1916|5|12|8|4|1916|69|884|Friday|1916Q4|N|Y|N|2421199|2421533|2420840|2421114|N|N|N|N|N| +2421207|AAAAAAAAHNBPECAA|1916-12-09|203|884|69|1916|6|12|9|4|1916|69|884|Saturday|1916Q4|N|Y|N|2421199|2421533|2420841|2421115|N|N|N|N|N| +2421208|AAAAAAAAINBPECAA|1916-12-10|203|884|69|1916|0|12|10|4|1916|69|884|Sunday|1916Q4|N|N|N|2421199|2421533|2420842|2421116|N|N|N|N|N| +2421209|AAAAAAAAJNBPECAA|1916-12-11|203|884|69|1916|1|12|11|4|1916|69|884|Monday|1916Q4|N|N|N|2421199|2421533|2420843|2421117|N|N|N|N|N| +2421210|AAAAAAAAKNBPECAA|1916-12-12|203|885|69|1916|2|12|12|4|1916|69|885|Tuesday|1916Q4|N|N|N|2421199|2421533|2420844|2421118|N|N|N|N|N| +2421211|AAAAAAAALNBPECAA|1916-12-13|203|885|69|1916|3|12|13|4|1916|69|885|Wednesday|1916Q4|N|N|N|2421199|2421533|2420845|2421119|N|N|N|N|N| +2421212|AAAAAAAAMNBPECAA|1916-12-14|203|885|69|1916|4|12|14|4|1916|69|885|Thursday|1916Q4|N|N|N|2421199|2421533|2420846|2421120|N|N|N|N|N| +2421213|AAAAAAAANNBPECAA|1916-12-15|203|885|69|1916|5|12|15|4|1916|69|885|Friday|1916Q4|N|Y|N|2421199|2421533|2420847|2421121|N|N|N|N|N| +2421214|AAAAAAAAONBPECAA|1916-12-16|203|885|69|1916|6|12|16|4|1916|69|885|Saturday|1916Q4|N|Y|N|2421199|2421533|2420848|2421122|N|N|N|N|N| +2421215|AAAAAAAAPNBPECAA|1916-12-17|203|885|69|1916|0|12|17|4|1916|69|885|Sunday|1916Q4|N|N|N|2421199|2421533|2420849|2421123|N|N|N|N|N| +2421216|AAAAAAAAAOBPECAA|1916-12-18|203|885|69|1916|1|12|18|4|1916|69|885|Monday|1916Q4|N|N|N|2421199|2421533|2420850|2421124|N|N|N|N|N| +2421217|AAAAAAAABOBPECAA|1916-12-19|203|886|69|1916|2|12|19|4|1916|69|886|Tuesday|1916Q4|N|N|N|2421199|2421533|2420851|2421125|N|N|N|N|N| +2421218|AAAAAAAACOBPECAA|1916-12-20|203|886|69|1916|3|12|20|4|1916|69|886|Wednesday|1916Q4|N|N|N|2421199|2421533|2420852|2421126|N|N|N|N|N| +2421219|AAAAAAAADOBPECAA|1916-12-21|203|886|69|1916|4|12|21|4|1916|69|886|Thursday|1916Q4|N|N|N|2421199|2421533|2420853|2421127|N|N|N|N|N| +2421220|AAAAAAAAEOBPECAA|1916-12-22|203|886|69|1916|5|12|22|4|1916|69|886|Friday|1916Q4|N|Y|N|2421199|2421533|2420854|2421128|N|N|N|N|N| +2421221|AAAAAAAAFOBPECAA|1916-12-23|203|886|69|1916|6|12|23|4|1916|69|886|Saturday|1916Q4|N|Y|N|2421199|2421533|2420855|2421129|N|N|N|N|N| +2421222|AAAAAAAAGOBPECAA|1916-12-24|203|886|69|1916|0|12|24|4|1916|69|886|Sunday|1916Q4|N|N|N|2421199|2421533|2420856|2421130|N|N|N|N|N| +2421223|AAAAAAAAHOBPECAA|1916-12-25|203|886|69|1916|1|12|25|4|1916|69|886|Monday|1916Q4|Y|N|N|2421199|2421533|2420857|2421131|N|N|N|N|N| +2421224|AAAAAAAAIOBPECAA|1916-12-26|203|887|69|1916|2|12|26|4|1916|69|887|Tuesday|1916Q4|N|N|Y|2421199|2421533|2420858|2421132|N|N|N|N|N| +2421225|AAAAAAAAJOBPECAA|1916-12-27|203|887|69|1916|3|12|27|4|1916|69|887|Wednesday|1916Q4|N|N|N|2421199|2421533|2420859|2421133|N|N|N|N|N| +2421226|AAAAAAAAKOBPECAA|1916-12-28|203|887|69|1916|4|12|28|4|1916|69|887|Thursday|1916Q4|N|N|N|2421199|2421533|2420860|2421134|N|N|N|N|N| +2421227|AAAAAAAALOBPECAA|1916-12-29|203|887|69|1916|5|12|29|4|1916|69|887|Friday|1916Q4|N|Y|N|2421199|2421533|2420861|2421135|N|N|N|N|N| +2421228|AAAAAAAAMOBPECAA|1916-12-30|203|887|69|1916|6|12|30|4|1916|69|887|Saturday|1916Q4|N|Y|N|2421199|2421533|2420862|2421136|N|N|N|N|N| +2421229|AAAAAAAANOBPECAA|1916-12-31|203|887|69|1916|0|12|31|4|1916|69|887|Sunday|1916Q4|Y|N|N|2421199|2421533|2420863|2421137|N|N|N|N|N| +2421230|AAAAAAAAOOBPECAA|1917-01-01|204|887|69|1917|1|1|1|1|1917|69|887|Monday|1917Q1|Y|N|Y|2421230|2421229|2420864|2421138|N|N|N|N|N| +2421231|AAAAAAAAPOBPECAA|1917-01-02|204|888|69|1917|2|1|2|1|1917|69|888|Tuesday|1917Q1|N|N|Y|2421230|2421229|2420865|2421139|N|N|N|N|N| +2421232|AAAAAAAAAPBPECAA|1917-01-03|204|888|69|1917|3|1|3|1|1917|69|888|Wednesday|1917Q1|N|N|N|2421230|2421229|2420866|2421140|N|N|N|N|N| +2421233|AAAAAAAABPBPECAA|1917-01-04|204|888|69|1917|4|1|4|1|1917|69|888|Thursday|1917Q1|N|N|N|2421230|2421229|2420867|2421141|N|N|N|N|N| +2421234|AAAAAAAACPBPECAA|1917-01-05|204|888|69|1917|5|1|5|1|1917|69|888|Friday|1917Q1|N|Y|N|2421230|2421229|2420868|2421142|N|N|N|N|N| +2421235|AAAAAAAADPBPECAA|1917-01-06|204|888|69|1917|6|1|6|1|1917|69|888|Saturday|1917Q1|N|Y|N|2421230|2421229|2420869|2421143|N|N|N|N|N| +2421236|AAAAAAAAEPBPECAA|1917-01-07|204|888|69|1917|0|1|7|1|1917|69|888|Sunday|1917Q1|N|N|N|2421230|2421229|2420870|2421144|N|N|N|N|N| +2421237|AAAAAAAAFPBPECAA|1917-01-08|204|888|69|1917|1|1|8|1|1917|69|888|Monday|1917Q1|N|N|N|2421230|2421229|2420871|2421145|N|N|N|N|N| +2421238|AAAAAAAAGPBPECAA|1917-01-09|204|889|69|1917|2|1|9|1|1917|69|889|Tuesday|1917Q1|N|N|N|2421230|2421229|2420872|2421146|N|N|N|N|N| +2421239|AAAAAAAAHPBPECAA|1917-01-10|204|889|69|1917|3|1|10|1|1917|69|889|Wednesday|1917Q1|N|N|N|2421230|2421229|2420873|2421147|N|N|N|N|N| +2421240|AAAAAAAAIPBPECAA|1917-01-11|204|889|69|1917|4|1|11|1|1917|69|889|Thursday|1917Q1|N|N|N|2421230|2421229|2420874|2421148|N|N|N|N|N| +2421241|AAAAAAAAJPBPECAA|1917-01-12|204|889|69|1917|5|1|12|1|1917|69|889|Friday|1917Q1|N|Y|N|2421230|2421229|2420875|2421149|N|N|N|N|N| +2421242|AAAAAAAAKPBPECAA|1917-01-13|204|889|69|1917|6|1|13|1|1917|69|889|Saturday|1917Q1|N|Y|N|2421230|2421229|2420876|2421150|N|N|N|N|N| +2421243|AAAAAAAALPBPECAA|1917-01-14|204|889|69|1917|0|1|14|1|1917|69|889|Sunday|1917Q1|N|N|N|2421230|2421229|2420877|2421151|N|N|N|N|N| +2421244|AAAAAAAAMPBPECAA|1917-01-15|204|889|69|1917|1|1|15|1|1917|69|889|Monday|1917Q1|N|N|N|2421230|2421229|2420878|2421152|N|N|N|N|N| +2421245|AAAAAAAANPBPECAA|1917-01-16|204|890|69|1917|2|1|16|1|1917|69|890|Tuesday|1917Q1|N|N|N|2421230|2421229|2420879|2421153|N|N|N|N|N| +2421246|AAAAAAAAOPBPECAA|1917-01-17|204|890|69|1917|3|1|17|1|1917|69|890|Wednesday|1917Q1|N|N|N|2421230|2421229|2420880|2421154|N|N|N|N|N| +2421247|AAAAAAAAPPBPECAA|1917-01-18|204|890|69|1917|4|1|18|1|1917|69|890|Thursday|1917Q1|N|N|N|2421230|2421229|2420881|2421155|N|N|N|N|N| +2421248|AAAAAAAAAACPECAA|1917-01-19|204|890|69|1917|5|1|19|1|1917|69|890|Friday|1917Q1|N|Y|N|2421230|2421229|2420882|2421156|N|N|N|N|N| +2421249|AAAAAAAABACPECAA|1917-01-20|204|890|69|1917|6|1|20|1|1917|69|890|Saturday|1917Q1|N|Y|N|2421230|2421229|2420883|2421157|N|N|N|N|N| +2421250|AAAAAAAACACPECAA|1917-01-21|204|890|69|1917|0|1|21|1|1917|69|890|Sunday|1917Q1|N|N|N|2421230|2421229|2420884|2421158|N|N|N|N|N| +2421251|AAAAAAAADACPECAA|1917-01-22|204|890|69|1917|1|1|22|1|1917|69|890|Monday|1917Q1|N|N|N|2421230|2421229|2420885|2421159|N|N|N|N|N| +2421252|AAAAAAAAEACPECAA|1917-01-23|204|891|69|1917|2|1|23|1|1917|69|891|Tuesday|1917Q1|N|N|N|2421230|2421229|2420886|2421160|N|N|N|N|N| +2421253|AAAAAAAAFACPECAA|1917-01-24|204|891|69|1917|3|1|24|1|1917|69|891|Wednesday|1917Q1|N|N|N|2421230|2421229|2420887|2421161|N|N|N|N|N| +2421254|AAAAAAAAGACPECAA|1917-01-25|204|891|69|1917|4|1|25|1|1917|69|891|Thursday|1917Q1|N|N|N|2421230|2421229|2420888|2421162|N|N|N|N|N| +2421255|AAAAAAAAHACPECAA|1917-01-26|204|891|69|1917|5|1|26|1|1917|69|891|Friday|1917Q1|N|Y|N|2421230|2421229|2420889|2421163|N|N|N|N|N| +2421256|AAAAAAAAIACPECAA|1917-01-27|204|891|69|1917|6|1|27|1|1917|69|891|Saturday|1917Q1|N|Y|N|2421230|2421229|2420890|2421164|N|N|N|N|N| +2421257|AAAAAAAAJACPECAA|1917-01-28|204|891|69|1917|0|1|28|1|1917|69|891|Sunday|1917Q1|N|N|N|2421230|2421229|2420891|2421165|N|N|N|N|N| +2421258|AAAAAAAAKACPECAA|1917-01-29|204|891|69|1917|1|1|29|1|1917|69|891|Monday|1917Q1|N|N|N|2421230|2421229|2420892|2421166|N|N|N|N|N| +2421259|AAAAAAAALACPECAA|1917-01-30|204|892|69|1917|2|1|30|1|1917|69|892|Tuesday|1917Q1|N|N|N|2421230|2421229|2420893|2421167|N|N|N|N|N| +2421260|AAAAAAAAMACPECAA|1917-01-31|204|892|69|1917|3|1|31|1|1917|69|892|Wednesday|1917Q1|N|N|N|2421230|2421229|2420894|2421168|N|N|N|N|N| +2421261|AAAAAAAANACPECAA|1917-02-01|205|892|69|1917|4|2|1|1|1917|69|892|Thursday|1917Q1|N|N|N|2421261|2421291|2420895|2421169|N|N|N|N|N| +2421262|AAAAAAAAOACPECAA|1917-02-02|205|892|69|1917|5|2|2|1|1917|69|892|Friday|1917Q1|N|Y|N|2421261|2421291|2420896|2421170|N|N|N|N|N| +2421263|AAAAAAAAPACPECAA|1917-02-03|205|892|69|1917|6|2|3|1|1917|69|892|Saturday|1917Q1|N|Y|N|2421261|2421291|2420897|2421171|N|N|N|N|N| +2421264|AAAAAAAAABCPECAA|1917-02-04|205|892|69|1917|0|2|4|1|1917|69|892|Sunday|1917Q1|N|N|N|2421261|2421291|2420898|2421172|N|N|N|N|N| +2421265|AAAAAAAABBCPECAA|1917-02-05|205|892|69|1917|1|2|5|1|1917|69|892|Monday|1917Q1|N|N|N|2421261|2421291|2420899|2421173|N|N|N|N|N| +2421266|AAAAAAAACBCPECAA|1917-02-06|205|893|69|1917|2|2|6|1|1917|69|893|Tuesday|1917Q1|N|N|N|2421261|2421291|2420900|2421174|N|N|N|N|N| +2421267|AAAAAAAADBCPECAA|1917-02-07|205|893|69|1917|3|2|7|1|1917|69|893|Wednesday|1917Q1|N|N|N|2421261|2421291|2420901|2421175|N|N|N|N|N| +2421268|AAAAAAAAEBCPECAA|1917-02-08|205|893|69|1917|4|2|8|1|1917|69|893|Thursday|1917Q1|N|N|N|2421261|2421291|2420902|2421176|N|N|N|N|N| +2421269|AAAAAAAAFBCPECAA|1917-02-09|205|893|69|1917|5|2|9|1|1917|69|893|Friday|1917Q1|N|Y|N|2421261|2421291|2420903|2421177|N|N|N|N|N| +2421270|AAAAAAAAGBCPECAA|1917-02-10|205|893|69|1917|6|2|10|1|1917|69|893|Saturday|1917Q1|N|Y|N|2421261|2421291|2420904|2421178|N|N|N|N|N| +2421271|AAAAAAAAHBCPECAA|1917-02-11|205|893|69|1917|0|2|11|1|1917|69|893|Sunday|1917Q1|N|N|N|2421261|2421291|2420905|2421179|N|N|N|N|N| +2421272|AAAAAAAAIBCPECAA|1917-02-12|205|893|69|1917|1|2|12|1|1917|69|893|Monday|1917Q1|N|N|N|2421261|2421291|2420906|2421180|N|N|N|N|N| +2421273|AAAAAAAAJBCPECAA|1917-02-13|205|894|69|1917|2|2|13|1|1917|69|894|Tuesday|1917Q1|N|N|N|2421261|2421291|2420907|2421181|N|N|N|N|N| +2421274|AAAAAAAAKBCPECAA|1917-02-14|205|894|69|1917|3|2|14|1|1917|69|894|Wednesday|1917Q1|N|N|N|2421261|2421291|2420908|2421182|N|N|N|N|N| +2421275|AAAAAAAALBCPECAA|1917-02-15|205|894|69|1917|4|2|15|1|1917|69|894|Thursday|1917Q1|N|N|N|2421261|2421291|2420909|2421183|N|N|N|N|N| +2421276|AAAAAAAAMBCPECAA|1917-02-16|205|894|69|1917|5|2|16|1|1917|69|894|Friday|1917Q1|N|Y|N|2421261|2421291|2420910|2421184|N|N|N|N|N| +2421277|AAAAAAAANBCPECAA|1917-02-17|205|894|69|1917|6|2|17|1|1917|69|894|Saturday|1917Q1|N|Y|N|2421261|2421291|2420911|2421185|N|N|N|N|N| +2421278|AAAAAAAAOBCPECAA|1917-02-18|205|894|69|1917|0|2|18|1|1917|69|894|Sunday|1917Q1|N|N|N|2421261|2421291|2420912|2421186|N|N|N|N|N| +2421279|AAAAAAAAPBCPECAA|1917-02-19|205|894|69|1917|1|2|19|1|1917|69|894|Monday|1917Q1|N|N|N|2421261|2421291|2420913|2421187|N|N|N|N|N| +2421280|AAAAAAAAACCPECAA|1917-02-20|205|895|69|1917|2|2|20|1|1917|69|895|Tuesday|1917Q1|N|N|N|2421261|2421291|2420914|2421188|N|N|N|N|N| +2421281|AAAAAAAABCCPECAA|1917-02-21|205|895|69|1917|3|2|21|1|1917|69|895|Wednesday|1917Q1|N|N|N|2421261|2421291|2420915|2421189|N|N|N|N|N| +2421282|AAAAAAAACCCPECAA|1917-02-22|205|895|69|1917|4|2|22|1|1917|69|895|Thursday|1917Q1|N|N|N|2421261|2421291|2420916|2421190|N|N|N|N|N| +2421283|AAAAAAAADCCPECAA|1917-02-23|205|895|69|1917|5|2|23|1|1917|69|895|Friday|1917Q1|N|Y|N|2421261|2421291|2420917|2421191|N|N|N|N|N| +2421284|AAAAAAAAECCPECAA|1917-02-24|205|895|69|1917|6|2|24|1|1917|69|895|Saturday|1917Q1|N|Y|N|2421261|2421291|2420918|2421192|N|N|N|N|N| +2421285|AAAAAAAAFCCPECAA|1917-02-25|205|895|69|1917|0|2|25|1|1917|69|895|Sunday|1917Q1|N|N|N|2421261|2421291|2420919|2421193|N|N|N|N|N| +2421286|AAAAAAAAGCCPECAA|1917-02-26|205|895|69|1917|1|2|26|1|1917|69|895|Monday|1917Q1|N|N|N|2421261|2421291|2420920|2421194|N|N|N|N|N| +2421287|AAAAAAAAHCCPECAA|1917-02-27|205|896|69|1917|2|2|27|1|1917|69|896|Tuesday|1917Q1|N|N|N|2421261|2421291|2420921|2421195|N|N|N|N|N| +2421288|AAAAAAAAICCPECAA|1917-02-28|205|896|69|1917|3|2|28|1|1917|69|896|Wednesday|1917Q1|N|N|N|2421261|2421291|2420922|2421196|N|N|N|N|N| +2421289|AAAAAAAAJCCPECAA|1917-03-01|206|896|70|1917|4|3|1|1|1917|70|896|Thursday|1917Q1|N|N|N|2421289|2421347|2420924|2421197|N|N|N|N|N| +2421290|AAAAAAAAKCCPECAA|1917-03-02|206|896|70|1917|5|3|2|1|1917|70|896|Friday|1917Q1|N|Y|N|2421289|2421347|2420925|2421198|N|N|N|N|N| +2421291|AAAAAAAALCCPECAA|1917-03-03|206|896|70|1917|6|3|3|1|1917|70|896|Saturday|1917Q1|N|Y|N|2421289|2421347|2420926|2421199|N|N|N|N|N| +2421292|AAAAAAAAMCCPECAA|1917-03-04|206|896|70|1917|0|3|4|1|1917|70|896|Sunday|1917Q1|N|N|N|2421289|2421347|2420927|2421200|N|N|N|N|N| +2421293|AAAAAAAANCCPECAA|1917-03-05|206|896|70|1917|1|3|5|1|1917|70|896|Monday|1917Q1|N|N|N|2421289|2421347|2420928|2421201|N|N|N|N|N| +2421294|AAAAAAAAOCCPECAA|1917-03-06|206|897|70|1917|2|3|6|1|1917|70|897|Tuesday|1917Q1|N|N|N|2421289|2421347|2420929|2421202|N|N|N|N|N| +2421295|AAAAAAAAPCCPECAA|1917-03-07|206|897|70|1917|3|3|7|1|1917|70|897|Wednesday|1917Q1|N|N|N|2421289|2421347|2420930|2421203|N|N|N|N|N| +2421296|AAAAAAAAADCPECAA|1917-03-08|206|897|70|1917|4|3|8|1|1917|70|897|Thursday|1917Q1|N|N|N|2421289|2421347|2420931|2421204|N|N|N|N|N| +2421297|AAAAAAAABDCPECAA|1917-03-09|206|897|70|1917|5|3|9|1|1917|70|897|Friday|1917Q1|N|Y|N|2421289|2421347|2420932|2421205|N|N|N|N|N| +2421298|AAAAAAAACDCPECAA|1917-03-10|206|897|70|1917|6|3|10|1|1917|70|897|Saturday|1917Q1|N|Y|N|2421289|2421347|2420933|2421206|N|N|N|N|N| +2421299|AAAAAAAADDCPECAA|1917-03-11|206|897|70|1917|0|3|11|1|1917|70|897|Sunday|1917Q1|N|N|N|2421289|2421347|2420934|2421207|N|N|N|N|N| +2421300|AAAAAAAAEDCPECAA|1917-03-12|206|897|70|1917|1|3|12|1|1917|70|897|Monday|1917Q1|N|N|N|2421289|2421347|2420935|2421208|N|N|N|N|N| +2421301|AAAAAAAAFDCPECAA|1917-03-13|206|898|70|1917|2|3|13|1|1917|70|898|Tuesday|1917Q1|N|N|N|2421289|2421347|2420936|2421209|N|N|N|N|N| +2421302|AAAAAAAAGDCPECAA|1917-03-14|206|898|70|1917|3|3|14|1|1917|70|898|Wednesday|1917Q1|N|N|N|2421289|2421347|2420937|2421210|N|N|N|N|N| +2421303|AAAAAAAAHDCPECAA|1917-03-15|206|898|70|1917|4|3|15|1|1917|70|898|Thursday|1917Q1|N|N|N|2421289|2421347|2420938|2421211|N|N|N|N|N| +2421304|AAAAAAAAIDCPECAA|1917-03-16|206|898|70|1917|5|3|16|1|1917|70|898|Friday|1917Q1|N|Y|N|2421289|2421347|2420939|2421212|N|N|N|N|N| +2421305|AAAAAAAAJDCPECAA|1917-03-17|206|898|70|1917|6|3|17|1|1917|70|898|Saturday|1917Q1|N|Y|N|2421289|2421347|2420940|2421213|N|N|N|N|N| +2421306|AAAAAAAAKDCPECAA|1917-03-18|206|898|70|1917|0|3|18|1|1917|70|898|Sunday|1917Q1|N|N|N|2421289|2421347|2420941|2421214|N|N|N|N|N| +2421307|AAAAAAAALDCPECAA|1917-03-19|206|898|70|1917|1|3|19|1|1917|70|898|Monday|1917Q1|N|N|N|2421289|2421347|2420942|2421215|N|N|N|N|N| +2421308|AAAAAAAAMDCPECAA|1917-03-20|206|899|70|1917|2|3|20|1|1917|70|899|Tuesday|1917Q1|N|N|N|2421289|2421347|2420943|2421216|N|N|N|N|N| +2421309|AAAAAAAANDCPECAA|1917-03-21|206|899|70|1917|3|3|21|1|1917|70|899|Wednesday|1917Q1|N|N|N|2421289|2421347|2420944|2421217|N|N|N|N|N| +2421310|AAAAAAAAODCPECAA|1917-03-22|206|899|70|1917|4|3|22|1|1917|70|899|Thursday|1917Q1|N|N|N|2421289|2421347|2420945|2421218|N|N|N|N|N| +2421311|AAAAAAAAPDCPECAA|1917-03-23|206|899|70|1917|5|3|23|1|1917|70|899|Friday|1917Q1|N|Y|N|2421289|2421347|2420946|2421219|N|N|N|N|N| +2421312|AAAAAAAAAECPECAA|1917-03-24|206|899|70|1917|6|3|24|1|1917|70|899|Saturday|1917Q1|N|Y|N|2421289|2421347|2420947|2421220|N|N|N|N|N| +2421313|AAAAAAAABECPECAA|1917-03-25|206|899|70|1917|0|3|25|1|1917|70|899|Sunday|1917Q1|N|N|N|2421289|2421347|2420948|2421221|N|N|N|N|N| +2421314|AAAAAAAACECPECAA|1917-03-26|206|899|70|1917|1|3|26|1|1917|70|899|Monday|1917Q1|N|N|N|2421289|2421347|2420949|2421222|N|N|N|N|N| +2421315|AAAAAAAADECPECAA|1917-03-27|206|900|70|1917|2|3|27|1|1917|70|900|Tuesday|1917Q1|N|N|N|2421289|2421347|2420950|2421223|N|N|N|N|N| +2421316|AAAAAAAAEECPECAA|1917-03-28|206|900|70|1917|3|3|28|1|1917|70|900|Wednesday|1917Q1|N|N|N|2421289|2421347|2420951|2421224|N|N|N|N|N| +2421317|AAAAAAAAFECPECAA|1917-03-29|206|900|70|1917|4|3|29|1|1917|70|900|Thursday|1917Q1|N|N|N|2421289|2421347|2420952|2421225|N|N|N|N|N| +2421318|AAAAAAAAGECPECAA|1917-03-30|206|900|70|1917|5|3|30|1|1917|70|900|Friday|1917Q1|N|Y|N|2421289|2421347|2420953|2421226|N|N|N|N|N| +2421319|AAAAAAAAHECPECAA|1917-03-31|206|900|70|1917|6|3|31|1|1917|70|900|Saturday|1917Q1|N|Y|N|2421289|2421347|2420954|2421227|N|N|N|N|N| +2421320|AAAAAAAAIECPECAA|1917-04-01|207|900|70|1917|0|4|1|1|1917|70|900|Sunday|1917Q1|N|N|N|2421320|2421409|2420955|2421230|N|N|N|N|N| +2421321|AAAAAAAAJECPECAA|1917-04-02|207|900|70|1917|1|4|2|2|1917|70|900|Monday|1917Q2|N|N|N|2421320|2421409|2420956|2421231|N|N|N|N|N| +2421322|AAAAAAAAKECPECAA|1917-04-03|207|901|70|1917|2|4|3|2|1917|70|901|Tuesday|1917Q2|N|N|N|2421320|2421409|2420957|2421232|N|N|N|N|N| +2421323|AAAAAAAALECPECAA|1917-04-04|207|901|70|1917|3|4|4|2|1917|70|901|Wednesday|1917Q2|N|N|N|2421320|2421409|2420958|2421233|N|N|N|N|N| +2421324|AAAAAAAAMECPECAA|1917-04-05|207|901|70|1917|4|4|5|2|1917|70|901|Thursday|1917Q2|N|N|N|2421320|2421409|2420959|2421234|N|N|N|N|N| +2421325|AAAAAAAANECPECAA|1917-04-06|207|901|70|1917|5|4|6|2|1917|70|901|Friday|1917Q2|N|Y|N|2421320|2421409|2420960|2421235|N|N|N|N|N| +2421326|AAAAAAAAOECPECAA|1917-04-07|207|901|70|1917|6|4|7|2|1917|70|901|Saturday|1917Q2|N|Y|N|2421320|2421409|2420961|2421236|N|N|N|N|N| +2421327|AAAAAAAAPECPECAA|1917-04-08|207|901|70|1917|0|4|8|2|1917|70|901|Sunday|1917Q2|N|N|N|2421320|2421409|2420962|2421237|N|N|N|N|N| +2421328|AAAAAAAAAFCPECAA|1917-04-09|207|901|70|1917|1|4|9|2|1917|70|901|Monday|1917Q2|N|N|N|2421320|2421409|2420963|2421238|N|N|N|N|N| +2421329|AAAAAAAABFCPECAA|1917-04-10|207|902|70|1917|2|4|10|2|1917|70|902|Tuesday|1917Q2|N|N|N|2421320|2421409|2420964|2421239|N|N|N|N|N| +2421330|AAAAAAAACFCPECAA|1917-04-11|207|902|70|1917|3|4|11|2|1917|70|902|Wednesday|1917Q2|N|N|N|2421320|2421409|2420965|2421240|N|N|N|N|N| +2421331|AAAAAAAADFCPECAA|1917-04-12|207|902|70|1917|4|4|12|2|1917|70|902|Thursday|1917Q2|N|N|N|2421320|2421409|2420966|2421241|N|N|N|N|N| +2421332|AAAAAAAAEFCPECAA|1917-04-13|207|902|70|1917|5|4|13|2|1917|70|902|Friday|1917Q2|N|Y|N|2421320|2421409|2420967|2421242|N|N|N|N|N| +2421333|AAAAAAAAFFCPECAA|1917-04-14|207|902|70|1917|6|4|14|2|1917|70|902|Saturday|1917Q2|N|Y|N|2421320|2421409|2420968|2421243|N|N|N|N|N| +2421334|AAAAAAAAGFCPECAA|1917-04-15|207|902|70|1917|0|4|15|2|1917|70|902|Sunday|1917Q2|N|N|N|2421320|2421409|2420969|2421244|N|N|N|N|N| +2421335|AAAAAAAAHFCPECAA|1917-04-16|207|902|70|1917|1|4|16|2|1917|70|902|Monday|1917Q2|N|N|N|2421320|2421409|2420970|2421245|N|N|N|N|N| +2421336|AAAAAAAAIFCPECAA|1917-04-17|207|903|70|1917|2|4|17|2|1917|70|903|Tuesday|1917Q2|N|N|N|2421320|2421409|2420971|2421246|N|N|N|N|N| +2421337|AAAAAAAAJFCPECAA|1917-04-18|207|903|70|1917|3|4|18|2|1917|70|903|Wednesday|1917Q2|N|N|N|2421320|2421409|2420972|2421247|N|N|N|N|N| +2421338|AAAAAAAAKFCPECAA|1917-04-19|207|903|70|1917|4|4|19|2|1917|70|903|Thursday|1917Q2|N|N|N|2421320|2421409|2420973|2421248|N|N|N|N|N| +2421339|AAAAAAAALFCPECAA|1917-04-20|207|903|70|1917|5|4|20|2|1917|70|903|Friday|1917Q2|N|Y|N|2421320|2421409|2420974|2421249|N|N|N|N|N| +2421340|AAAAAAAAMFCPECAA|1917-04-21|207|903|70|1917|6|4|21|2|1917|70|903|Saturday|1917Q2|N|Y|N|2421320|2421409|2420975|2421250|N|N|N|N|N| +2421341|AAAAAAAANFCPECAA|1917-04-22|207|903|70|1917|0|4|22|2|1917|70|903|Sunday|1917Q2|N|N|N|2421320|2421409|2420976|2421251|N|N|N|N|N| +2421342|AAAAAAAAOFCPECAA|1917-04-23|207|903|70|1917|1|4|23|2|1917|70|903|Monday|1917Q2|N|N|N|2421320|2421409|2420977|2421252|N|N|N|N|N| +2421343|AAAAAAAAPFCPECAA|1917-04-24|207|904|70|1917|2|4|24|2|1917|70|904|Tuesday|1917Q2|N|N|N|2421320|2421409|2420978|2421253|N|N|N|N|N| +2421344|AAAAAAAAAGCPECAA|1917-04-25|207|904|70|1917|3|4|25|2|1917|70|904|Wednesday|1917Q2|N|N|N|2421320|2421409|2420979|2421254|N|N|N|N|N| +2421345|AAAAAAAABGCPECAA|1917-04-26|207|904|70|1917|4|4|26|2|1917|70|904|Thursday|1917Q2|N|N|N|2421320|2421409|2420980|2421255|N|N|N|N|N| +2421346|AAAAAAAACGCPECAA|1917-04-27|207|904|70|1917|5|4|27|2|1917|70|904|Friday|1917Q2|N|Y|N|2421320|2421409|2420981|2421256|N|N|N|N|N| +2421347|AAAAAAAADGCPECAA|1917-04-28|207|904|70|1917|6|4|28|2|1917|70|904|Saturday|1917Q2|N|Y|N|2421320|2421409|2420982|2421257|N|N|N|N|N| +2421348|AAAAAAAAEGCPECAA|1917-04-29|207|904|70|1917|0|4|29|2|1917|70|904|Sunday|1917Q2|N|N|N|2421320|2421409|2420983|2421258|N|N|N|N|N| +2421349|AAAAAAAAFGCPECAA|1917-04-30|207|904|70|1917|1|4|30|2|1917|70|904|Monday|1917Q2|N|N|N|2421320|2421409|2420984|2421259|N|N|N|N|N| +2421350|AAAAAAAAGGCPECAA|1917-05-01|208|905|70|1917|2|5|1|2|1917|70|905|Tuesday|1917Q2|N|N|N|2421350|2421469|2420985|2421260|N|N|N|N|N| +2421351|AAAAAAAAHGCPECAA|1917-05-02|208|905|70|1917|3|5|2|2|1917|70|905|Wednesday|1917Q2|N|N|N|2421350|2421469|2420986|2421261|N|N|N|N|N| +2421352|AAAAAAAAIGCPECAA|1917-05-03|208|905|70|1917|4|5|3|2|1917|70|905|Thursday|1917Q2|N|N|N|2421350|2421469|2420987|2421262|N|N|N|N|N| +2421353|AAAAAAAAJGCPECAA|1917-05-04|208|905|70|1917|5|5|4|2|1917|70|905|Friday|1917Q2|N|Y|N|2421350|2421469|2420988|2421263|N|N|N|N|N| +2421354|AAAAAAAAKGCPECAA|1917-05-05|208|905|70|1917|6|5|5|2|1917|70|905|Saturday|1917Q2|N|Y|N|2421350|2421469|2420989|2421264|N|N|N|N|N| +2421355|AAAAAAAALGCPECAA|1917-05-06|208|905|70|1917|0|5|6|2|1917|70|905|Sunday|1917Q2|N|N|N|2421350|2421469|2420990|2421265|N|N|N|N|N| +2421356|AAAAAAAAMGCPECAA|1917-05-07|208|905|70|1917|1|5|7|2|1917|70|905|Monday|1917Q2|N|N|N|2421350|2421469|2420991|2421266|N|N|N|N|N| +2421357|AAAAAAAANGCPECAA|1917-05-08|208|906|70|1917|2|5|8|2|1917|70|906|Tuesday|1917Q2|N|N|N|2421350|2421469|2420992|2421267|N|N|N|N|N| +2421358|AAAAAAAAOGCPECAA|1917-05-09|208|906|70|1917|3|5|9|2|1917|70|906|Wednesday|1917Q2|N|N|N|2421350|2421469|2420993|2421268|N|N|N|N|N| +2421359|AAAAAAAAPGCPECAA|1917-05-10|208|906|70|1917|4|5|10|2|1917|70|906|Thursday|1917Q2|N|N|N|2421350|2421469|2420994|2421269|N|N|N|N|N| +2421360|AAAAAAAAAHCPECAA|1917-05-11|208|906|70|1917|5|5|11|2|1917|70|906|Friday|1917Q2|N|Y|N|2421350|2421469|2420995|2421270|N|N|N|N|N| +2421361|AAAAAAAABHCPECAA|1917-05-12|208|906|70|1917|6|5|12|2|1917|70|906|Saturday|1917Q2|N|Y|N|2421350|2421469|2420996|2421271|N|N|N|N|N| +2421362|AAAAAAAACHCPECAA|1917-05-13|208|906|70|1917|0|5|13|2|1917|70|906|Sunday|1917Q2|N|N|N|2421350|2421469|2420997|2421272|N|N|N|N|N| +2421363|AAAAAAAADHCPECAA|1917-05-14|208|906|70|1917|1|5|14|2|1917|70|906|Monday|1917Q2|N|N|N|2421350|2421469|2420998|2421273|N|N|N|N|N| +2421364|AAAAAAAAEHCPECAA|1917-05-15|208|907|70|1917|2|5|15|2|1917|70|907|Tuesday|1917Q2|N|N|N|2421350|2421469|2420999|2421274|N|N|N|N|N| +2421365|AAAAAAAAFHCPECAA|1917-05-16|208|907|70|1917|3|5|16|2|1917|70|907|Wednesday|1917Q2|N|N|N|2421350|2421469|2421000|2421275|N|N|N|N|N| +2421366|AAAAAAAAGHCPECAA|1917-05-17|208|907|70|1917|4|5|17|2|1917|70|907|Thursday|1917Q2|N|N|N|2421350|2421469|2421001|2421276|N|N|N|N|N| +2421367|AAAAAAAAHHCPECAA|1917-05-18|208|907|70|1917|5|5|18|2|1917|70|907|Friday|1917Q2|N|Y|N|2421350|2421469|2421002|2421277|N|N|N|N|N| +2421368|AAAAAAAAIHCPECAA|1917-05-19|208|907|70|1917|6|5|19|2|1917|70|907|Saturday|1917Q2|N|Y|N|2421350|2421469|2421003|2421278|N|N|N|N|N| +2421369|AAAAAAAAJHCPECAA|1917-05-20|208|907|70|1917|0|5|20|2|1917|70|907|Sunday|1917Q2|N|N|N|2421350|2421469|2421004|2421279|N|N|N|N|N| +2421370|AAAAAAAAKHCPECAA|1917-05-21|208|907|70|1917|1|5|21|2|1917|70|907|Monday|1917Q2|N|N|N|2421350|2421469|2421005|2421280|N|N|N|N|N| +2421371|AAAAAAAALHCPECAA|1917-05-22|208|908|70|1917|2|5|22|2|1917|70|908|Tuesday|1917Q2|N|N|N|2421350|2421469|2421006|2421281|N|N|N|N|N| +2421372|AAAAAAAAMHCPECAA|1917-05-23|208|908|70|1917|3|5|23|2|1917|70|908|Wednesday|1917Q2|N|N|N|2421350|2421469|2421007|2421282|N|N|N|N|N| +2421373|AAAAAAAANHCPECAA|1917-05-24|208|908|70|1917|4|5|24|2|1917|70|908|Thursday|1917Q2|N|N|N|2421350|2421469|2421008|2421283|N|N|N|N|N| +2421374|AAAAAAAAOHCPECAA|1917-05-25|208|908|70|1917|5|5|25|2|1917|70|908|Friday|1917Q2|N|Y|N|2421350|2421469|2421009|2421284|N|N|N|N|N| +2421375|AAAAAAAAPHCPECAA|1917-05-26|208|908|70|1917|6|5|26|2|1917|70|908|Saturday|1917Q2|N|Y|N|2421350|2421469|2421010|2421285|N|N|N|N|N| +2421376|AAAAAAAAAICPECAA|1917-05-27|208|908|70|1917|0|5|27|2|1917|70|908|Sunday|1917Q2|N|N|N|2421350|2421469|2421011|2421286|N|N|N|N|N| +2421377|AAAAAAAABICPECAA|1917-05-28|208|908|70|1917|1|5|28|2|1917|70|908|Monday|1917Q2|N|N|N|2421350|2421469|2421012|2421287|N|N|N|N|N| +2421378|AAAAAAAACICPECAA|1917-05-29|208|909|70|1917|2|5|29|2|1917|70|909|Tuesday|1917Q2|N|N|N|2421350|2421469|2421013|2421288|N|N|N|N|N| +2421379|AAAAAAAADICPECAA|1917-05-30|208|909|70|1917|3|5|30|2|1917|70|909|Wednesday|1917Q2|N|N|N|2421350|2421469|2421014|2421289|N|N|N|N|N| +2421380|AAAAAAAAEICPECAA|1917-05-31|208|909|70|1917|4|5|31|2|1917|70|909|Thursday|1917Q2|N|N|N|2421350|2421469|2421015|2421290|N|N|N|N|N| +2421381|AAAAAAAAFICPECAA|1917-06-01|209|909|71|1917|5|6|1|2|1917|71|909|Friday|1917Q2|N|Y|N|2421381|2421531|2421016|2421291|N|N|N|N|N| +2421382|AAAAAAAAGICPECAA|1917-06-02|209|909|71|1917|6|6|2|2|1917|71|909|Saturday|1917Q2|N|Y|N|2421381|2421531|2421017|2421292|N|N|N|N|N| +2421383|AAAAAAAAHICPECAA|1917-06-03|209|909|71|1917|0|6|3|2|1917|71|909|Sunday|1917Q2|N|N|N|2421381|2421531|2421018|2421293|N|N|N|N|N| +2421384|AAAAAAAAIICPECAA|1917-06-04|209|909|71|1917|1|6|4|2|1917|71|909|Monday|1917Q2|N|N|N|2421381|2421531|2421019|2421294|N|N|N|N|N| +2421385|AAAAAAAAJICPECAA|1917-06-05|209|910|71|1917|2|6|5|2|1917|71|910|Tuesday|1917Q2|N|N|N|2421381|2421531|2421020|2421295|N|N|N|N|N| +2421386|AAAAAAAAKICPECAA|1917-06-06|209|910|71|1917|3|6|6|2|1917|71|910|Wednesday|1917Q2|N|N|N|2421381|2421531|2421021|2421296|N|N|N|N|N| +2421387|AAAAAAAALICPECAA|1917-06-07|209|910|71|1917|4|6|7|2|1917|71|910|Thursday|1917Q2|N|N|N|2421381|2421531|2421022|2421297|N|N|N|N|N| +2421388|AAAAAAAAMICPECAA|1917-06-08|209|910|71|1917|5|6|8|2|1917|71|910|Friday|1917Q2|N|Y|N|2421381|2421531|2421023|2421298|N|N|N|N|N| +2421389|AAAAAAAANICPECAA|1917-06-09|209|910|71|1917|6|6|9|2|1917|71|910|Saturday|1917Q2|N|Y|N|2421381|2421531|2421024|2421299|N|N|N|N|N| +2421390|AAAAAAAAOICPECAA|1917-06-10|209|910|71|1917|0|6|10|2|1917|71|910|Sunday|1917Q2|N|N|N|2421381|2421531|2421025|2421300|N|N|N|N|N| +2421391|AAAAAAAAPICPECAA|1917-06-11|209|910|71|1917|1|6|11|2|1917|71|910|Monday|1917Q2|N|N|N|2421381|2421531|2421026|2421301|N|N|N|N|N| +2421392|AAAAAAAAAJCPECAA|1917-06-12|209|911|71|1917|2|6|12|2|1917|71|911|Tuesday|1917Q2|N|N|N|2421381|2421531|2421027|2421302|N|N|N|N|N| +2421393|AAAAAAAABJCPECAA|1917-06-13|209|911|71|1917|3|6|13|2|1917|71|911|Wednesday|1917Q2|N|N|N|2421381|2421531|2421028|2421303|N|N|N|N|N| +2421394|AAAAAAAACJCPECAA|1917-06-14|209|911|71|1917|4|6|14|2|1917|71|911|Thursday|1917Q2|N|N|N|2421381|2421531|2421029|2421304|N|N|N|N|N| +2421395|AAAAAAAADJCPECAA|1917-06-15|209|911|71|1917|5|6|15|2|1917|71|911|Friday|1917Q2|N|Y|N|2421381|2421531|2421030|2421305|N|N|N|N|N| +2421396|AAAAAAAAEJCPECAA|1917-06-16|209|911|71|1917|6|6|16|2|1917|71|911|Saturday|1917Q2|N|Y|N|2421381|2421531|2421031|2421306|N|N|N|N|N| +2421397|AAAAAAAAFJCPECAA|1917-06-17|209|911|71|1917|0|6|17|2|1917|71|911|Sunday|1917Q2|N|N|N|2421381|2421531|2421032|2421307|N|N|N|N|N| +2421398|AAAAAAAAGJCPECAA|1917-06-18|209|911|71|1917|1|6|18|2|1917|71|911|Monday|1917Q2|N|N|N|2421381|2421531|2421033|2421308|N|N|N|N|N| +2421399|AAAAAAAAHJCPECAA|1917-06-19|209|912|71|1917|2|6|19|2|1917|71|912|Tuesday|1917Q2|N|N|N|2421381|2421531|2421034|2421309|N|N|N|N|N| +2421400|AAAAAAAAIJCPECAA|1917-06-20|209|912|71|1917|3|6|20|2|1917|71|912|Wednesday|1917Q2|N|N|N|2421381|2421531|2421035|2421310|N|N|N|N|N| +2421401|AAAAAAAAJJCPECAA|1917-06-21|209|912|71|1917|4|6|21|2|1917|71|912|Thursday|1917Q2|N|N|N|2421381|2421531|2421036|2421311|N|N|N|N|N| +2421402|AAAAAAAAKJCPECAA|1917-06-22|209|912|71|1917|5|6|22|2|1917|71|912|Friday|1917Q2|N|Y|N|2421381|2421531|2421037|2421312|N|N|N|N|N| +2421403|AAAAAAAALJCPECAA|1917-06-23|209|912|71|1917|6|6|23|2|1917|71|912|Saturday|1917Q2|N|Y|N|2421381|2421531|2421038|2421313|N|N|N|N|N| +2421404|AAAAAAAAMJCPECAA|1917-06-24|209|912|71|1917|0|6|24|2|1917|71|912|Sunday|1917Q2|N|N|N|2421381|2421531|2421039|2421314|N|N|N|N|N| +2421405|AAAAAAAANJCPECAA|1917-06-25|209|912|71|1917|1|6|25|2|1917|71|912|Monday|1917Q2|N|N|N|2421381|2421531|2421040|2421315|N|N|N|N|N| +2421406|AAAAAAAAOJCPECAA|1917-06-26|209|913|71|1917|2|6|26|2|1917|71|913|Tuesday|1917Q2|N|N|N|2421381|2421531|2421041|2421316|N|N|N|N|N| +2421407|AAAAAAAAPJCPECAA|1917-06-27|209|913|71|1917|3|6|27|2|1917|71|913|Wednesday|1917Q2|N|N|N|2421381|2421531|2421042|2421317|N|N|N|N|N| +2421408|AAAAAAAAAKCPECAA|1917-06-28|209|913|71|1917|4|6|28|2|1917|71|913|Thursday|1917Q2|N|N|N|2421381|2421531|2421043|2421318|N|N|N|N|N| +2421409|AAAAAAAABKCPECAA|1917-06-29|209|913|71|1917|5|6|29|2|1917|71|913|Friday|1917Q2|N|Y|N|2421381|2421531|2421044|2421319|N|N|N|N|N| +2421410|AAAAAAAACKCPECAA|1917-06-30|209|913|71|1917|6|6|30|2|1917|71|913|Saturday|1917Q2|N|Y|N|2421381|2421531|2421045|2421320|N|N|N|N|N| +2421411|AAAAAAAADKCPECAA|1917-07-01|210|913|71|1917|0|7|1|2|1917|71|913|Sunday|1917Q2|N|N|N|2421411|2421591|2421046|2421320|N|N|N|N|N| +2421412|AAAAAAAAEKCPECAA|1917-07-02|210|913|71|1917|1|7|2|3|1917|71|913|Monday|1917Q3|N|N|N|2421411|2421591|2421047|2421321|N|N|N|N|N| +2421413|AAAAAAAAFKCPECAA|1917-07-03|210|914|71|1917|2|7|3|3|1917|71|914|Tuesday|1917Q3|N|N|N|2421411|2421591|2421048|2421322|N|N|N|N|N| +2421414|AAAAAAAAGKCPECAA|1917-07-04|210|914|71|1917|3|7|4|3|1917|71|914|Wednesday|1917Q3|N|N|N|2421411|2421591|2421049|2421323|N|N|N|N|N| +2421415|AAAAAAAAHKCPECAA|1917-07-05|210|914|71|1917|4|7|5|3|1917|71|914|Thursday|1917Q3|Y|N|N|2421411|2421591|2421050|2421324|N|N|N|N|N| +2421416|AAAAAAAAIKCPECAA|1917-07-06|210|914|71|1917|5|7|6|3|1917|71|914|Friday|1917Q3|N|Y|Y|2421411|2421591|2421051|2421325|N|N|N|N|N| +2421417|AAAAAAAAJKCPECAA|1917-07-07|210|914|71|1917|6|7|7|3|1917|71|914|Saturday|1917Q3|N|Y|N|2421411|2421591|2421052|2421326|N|N|N|N|N| +2421418|AAAAAAAAKKCPECAA|1917-07-08|210|914|71|1917|0|7|8|3|1917|71|914|Sunday|1917Q3|N|N|N|2421411|2421591|2421053|2421327|N|N|N|N|N| +2421419|AAAAAAAALKCPECAA|1917-07-09|210|914|71|1917|1|7|9|3|1917|71|914|Monday|1917Q3|N|N|N|2421411|2421591|2421054|2421328|N|N|N|N|N| +2421420|AAAAAAAAMKCPECAA|1917-07-10|210|915|71|1917|2|7|10|3|1917|71|915|Tuesday|1917Q3|N|N|N|2421411|2421591|2421055|2421329|N|N|N|N|N| +2421421|AAAAAAAANKCPECAA|1917-07-11|210|915|71|1917|3|7|11|3|1917|71|915|Wednesday|1917Q3|N|N|N|2421411|2421591|2421056|2421330|N|N|N|N|N| +2421422|AAAAAAAAOKCPECAA|1917-07-12|210|915|71|1917|4|7|12|3|1917|71|915|Thursday|1917Q3|N|N|N|2421411|2421591|2421057|2421331|N|N|N|N|N| +2421423|AAAAAAAAPKCPECAA|1917-07-13|210|915|71|1917|5|7|13|3|1917|71|915|Friday|1917Q3|N|Y|N|2421411|2421591|2421058|2421332|N|N|N|N|N| +2421424|AAAAAAAAALCPECAA|1917-07-14|210|915|71|1917|6|7|14|3|1917|71|915|Saturday|1917Q3|N|Y|N|2421411|2421591|2421059|2421333|N|N|N|N|N| +2421425|AAAAAAAABLCPECAA|1917-07-15|210|915|71|1917|0|7|15|3|1917|71|915|Sunday|1917Q3|N|N|N|2421411|2421591|2421060|2421334|N|N|N|N|N| +2421426|AAAAAAAACLCPECAA|1917-07-16|210|915|71|1917|1|7|16|3|1917|71|915|Monday|1917Q3|N|N|N|2421411|2421591|2421061|2421335|N|N|N|N|N| +2421427|AAAAAAAADLCPECAA|1917-07-17|210|916|71|1917|2|7|17|3|1917|71|916|Tuesday|1917Q3|N|N|N|2421411|2421591|2421062|2421336|N|N|N|N|N| +2421428|AAAAAAAAELCPECAA|1917-07-18|210|916|71|1917|3|7|18|3|1917|71|916|Wednesday|1917Q3|N|N|N|2421411|2421591|2421063|2421337|N|N|N|N|N| +2421429|AAAAAAAAFLCPECAA|1917-07-19|210|916|71|1917|4|7|19|3|1917|71|916|Thursday|1917Q3|N|N|N|2421411|2421591|2421064|2421338|N|N|N|N|N| +2421430|AAAAAAAAGLCPECAA|1917-07-20|210|916|71|1917|5|7|20|3|1917|71|916|Friday|1917Q3|N|Y|N|2421411|2421591|2421065|2421339|N|N|N|N|N| +2421431|AAAAAAAAHLCPECAA|1917-07-21|210|916|71|1917|6|7|21|3|1917|71|916|Saturday|1917Q3|N|Y|N|2421411|2421591|2421066|2421340|N|N|N|N|N| +2421432|AAAAAAAAILCPECAA|1917-07-22|210|916|71|1917|0|7|22|3|1917|71|916|Sunday|1917Q3|N|N|N|2421411|2421591|2421067|2421341|N|N|N|N|N| +2421433|AAAAAAAAJLCPECAA|1917-07-23|210|916|71|1917|1|7|23|3|1917|71|916|Monday|1917Q3|N|N|N|2421411|2421591|2421068|2421342|N|N|N|N|N| +2421434|AAAAAAAAKLCPECAA|1917-07-24|210|917|71|1917|2|7|24|3|1917|71|917|Tuesday|1917Q3|N|N|N|2421411|2421591|2421069|2421343|N|N|N|N|N| +2421435|AAAAAAAALLCPECAA|1917-07-25|210|917|71|1917|3|7|25|3|1917|71|917|Wednesday|1917Q3|N|N|N|2421411|2421591|2421070|2421344|N|N|N|N|N| +2421436|AAAAAAAAMLCPECAA|1917-07-26|210|917|71|1917|4|7|26|3|1917|71|917|Thursday|1917Q3|N|N|N|2421411|2421591|2421071|2421345|N|N|N|N|N| +2421437|AAAAAAAANLCPECAA|1917-07-27|210|917|71|1917|5|7|27|3|1917|71|917|Friday|1917Q3|N|Y|N|2421411|2421591|2421072|2421346|N|N|N|N|N| +2421438|AAAAAAAAOLCPECAA|1917-07-28|210|917|71|1917|6|7|28|3|1917|71|917|Saturday|1917Q3|N|Y|N|2421411|2421591|2421073|2421347|N|N|N|N|N| +2421439|AAAAAAAAPLCPECAA|1917-07-29|210|917|71|1917|0|7|29|3|1917|71|917|Sunday|1917Q3|N|N|N|2421411|2421591|2421074|2421348|N|N|N|N|N| +2421440|AAAAAAAAAMCPECAA|1917-07-30|210|917|71|1917|1|7|30|3|1917|71|917|Monday|1917Q3|N|N|N|2421411|2421591|2421075|2421349|N|N|N|N|N| +2421441|AAAAAAAABMCPECAA|1917-07-31|210|918|71|1917|2|7|31|3|1917|71|918|Tuesday|1917Q3|N|N|N|2421411|2421591|2421076|2421350|N|N|N|N|N| +2421442|AAAAAAAACMCPECAA|1917-08-01|211|918|71|1917|3|8|1|3|1917|71|918|Wednesday|1917Q3|N|N|N|2421442|2421653|2421077|2421351|N|N|N|N|N| +2421443|AAAAAAAADMCPECAA|1917-08-02|211|918|71|1917|4|8|2|3|1917|71|918|Thursday|1917Q3|N|N|N|2421442|2421653|2421078|2421352|N|N|N|N|N| +2421444|AAAAAAAAEMCPECAA|1917-08-03|211|918|71|1917|5|8|3|3|1917|71|918|Friday|1917Q3|N|Y|N|2421442|2421653|2421079|2421353|N|N|N|N|N| +2421445|AAAAAAAAFMCPECAA|1917-08-04|211|918|71|1917|6|8|4|3|1917|71|918|Saturday|1917Q3|N|Y|N|2421442|2421653|2421080|2421354|N|N|N|N|N| +2421446|AAAAAAAAGMCPECAA|1917-08-05|211|918|71|1917|0|8|5|3|1917|71|918|Sunday|1917Q3|N|N|N|2421442|2421653|2421081|2421355|N|N|N|N|N| +2421447|AAAAAAAAHMCPECAA|1917-08-06|211|918|71|1917|1|8|6|3|1917|71|918|Monday|1917Q3|N|N|N|2421442|2421653|2421082|2421356|N|N|N|N|N| +2421448|AAAAAAAAIMCPECAA|1917-08-07|211|919|71|1917|2|8|7|3|1917|71|919|Tuesday|1917Q3|N|N|N|2421442|2421653|2421083|2421357|N|N|N|N|N| +2421449|AAAAAAAAJMCPECAA|1917-08-08|211|919|71|1917|3|8|8|3|1917|71|919|Wednesday|1917Q3|N|N|N|2421442|2421653|2421084|2421358|N|N|N|N|N| +2421450|AAAAAAAAKMCPECAA|1917-08-09|211|919|71|1917|4|8|9|3|1917|71|919|Thursday|1917Q3|N|N|N|2421442|2421653|2421085|2421359|N|N|N|N|N| +2421451|AAAAAAAALMCPECAA|1917-08-10|211|919|71|1917|5|8|10|3|1917|71|919|Friday|1917Q3|N|Y|N|2421442|2421653|2421086|2421360|N|N|N|N|N| +2421452|AAAAAAAAMMCPECAA|1917-08-11|211|919|71|1917|6|8|11|3|1917|71|919|Saturday|1917Q3|N|Y|N|2421442|2421653|2421087|2421361|N|N|N|N|N| +2421453|AAAAAAAANMCPECAA|1917-08-12|211|919|71|1917|0|8|12|3|1917|71|919|Sunday|1917Q3|N|N|N|2421442|2421653|2421088|2421362|N|N|N|N|N| +2421454|AAAAAAAAOMCPECAA|1917-08-13|211|919|71|1917|1|8|13|3|1917|71|919|Monday|1917Q3|N|N|N|2421442|2421653|2421089|2421363|N|N|N|N|N| +2421455|AAAAAAAAPMCPECAA|1917-08-14|211|920|71|1917|2|8|14|3|1917|71|920|Tuesday|1917Q3|N|N|N|2421442|2421653|2421090|2421364|N|N|N|N|N| +2421456|AAAAAAAAANCPECAA|1917-08-15|211|920|71|1917|3|8|15|3|1917|71|920|Wednesday|1917Q3|N|N|N|2421442|2421653|2421091|2421365|N|N|N|N|N| +2421457|AAAAAAAABNCPECAA|1917-08-16|211|920|71|1917|4|8|16|3|1917|71|920|Thursday|1917Q3|N|N|N|2421442|2421653|2421092|2421366|N|N|N|N|N| +2421458|AAAAAAAACNCPECAA|1917-08-17|211|920|71|1917|5|8|17|3|1917|71|920|Friday|1917Q3|N|Y|N|2421442|2421653|2421093|2421367|N|N|N|N|N| +2421459|AAAAAAAADNCPECAA|1917-08-18|211|920|71|1917|6|8|18|3|1917|71|920|Saturday|1917Q3|N|Y|N|2421442|2421653|2421094|2421368|N|N|N|N|N| +2421460|AAAAAAAAENCPECAA|1917-08-19|211|920|71|1917|0|8|19|3|1917|71|920|Sunday|1917Q3|N|N|N|2421442|2421653|2421095|2421369|N|N|N|N|N| +2421461|AAAAAAAAFNCPECAA|1917-08-20|211|920|71|1917|1|8|20|3|1917|71|920|Monday|1917Q3|N|N|N|2421442|2421653|2421096|2421370|N|N|N|N|N| +2421462|AAAAAAAAGNCPECAA|1917-08-21|211|921|71|1917|2|8|21|3|1917|71|921|Tuesday|1917Q3|N|N|N|2421442|2421653|2421097|2421371|N|N|N|N|N| +2421463|AAAAAAAAHNCPECAA|1917-08-22|211|921|71|1917|3|8|22|3|1917|71|921|Wednesday|1917Q3|N|N|N|2421442|2421653|2421098|2421372|N|N|N|N|N| +2421464|AAAAAAAAINCPECAA|1917-08-23|211|921|71|1917|4|8|23|3|1917|71|921|Thursday|1917Q3|N|N|N|2421442|2421653|2421099|2421373|N|N|N|N|N| +2421465|AAAAAAAAJNCPECAA|1917-08-24|211|921|71|1917|5|8|24|3|1917|71|921|Friday|1917Q3|N|Y|N|2421442|2421653|2421100|2421374|N|N|N|N|N| +2421466|AAAAAAAAKNCPECAA|1917-08-25|211|921|71|1917|6|8|25|3|1917|71|921|Saturday|1917Q3|N|Y|N|2421442|2421653|2421101|2421375|N|N|N|N|N| +2421467|AAAAAAAALNCPECAA|1917-08-26|211|921|71|1917|0|8|26|3|1917|71|921|Sunday|1917Q3|N|N|N|2421442|2421653|2421102|2421376|N|N|N|N|N| +2421468|AAAAAAAAMNCPECAA|1917-08-27|211|921|71|1917|1|8|27|3|1917|71|921|Monday|1917Q3|N|N|N|2421442|2421653|2421103|2421377|N|N|N|N|N| +2421469|AAAAAAAANNCPECAA|1917-08-28|211|922|71|1917|2|8|28|3|1917|71|922|Tuesday|1917Q3|N|N|N|2421442|2421653|2421104|2421378|N|N|N|N|N| +2421470|AAAAAAAAONCPECAA|1917-08-29|211|922|71|1917|3|8|29|3|1917|71|922|Wednesday|1917Q3|N|N|N|2421442|2421653|2421105|2421379|N|N|N|N|N| +2421471|AAAAAAAAPNCPECAA|1917-08-30|211|922|71|1917|4|8|30|3|1917|71|922|Thursday|1917Q3|N|N|N|2421442|2421653|2421106|2421380|N|N|N|N|N| +2421472|AAAAAAAAAOCPECAA|1917-08-31|211|922|71|1917|5|8|31|3|1917|71|922|Friday|1917Q3|N|Y|N|2421442|2421653|2421107|2421381|N|N|N|N|N| +2421473|AAAAAAAABOCPECAA|1917-09-01|212|922|72|1917|6|9|1|3|1917|72|922|Saturday|1917Q3|N|Y|N|2421473|2421715|2421108|2421382|N|N|N|N|N| +2421474|AAAAAAAACOCPECAA|1917-09-02|212|922|72|1917|0|9|2|3|1917|72|922|Sunday|1917Q3|N|N|N|2421473|2421715|2421109|2421383|N|N|N|N|N| +2421475|AAAAAAAADOCPECAA|1917-09-03|212|922|72|1917|1|9|3|3|1917|72|922|Monday|1917Q3|N|N|N|2421473|2421715|2421110|2421384|N|N|N|N|N| +2421476|AAAAAAAAEOCPECAA|1917-09-04|212|923|72|1917|2|9|4|3|1917|72|923|Tuesday|1917Q3|N|N|N|2421473|2421715|2421111|2421385|N|N|N|N|N| +2421477|AAAAAAAAFOCPECAA|1917-09-05|212|923|72|1917|3|9|5|3|1917|72|923|Wednesday|1917Q3|N|N|N|2421473|2421715|2421112|2421386|N|N|N|N|N| +2421478|AAAAAAAAGOCPECAA|1917-09-06|212|923|72|1917|4|9|6|3|1917|72|923|Thursday|1917Q3|N|N|N|2421473|2421715|2421113|2421387|N|N|N|N|N| +2421479|AAAAAAAAHOCPECAA|1917-09-07|212|923|72|1917|5|9|7|3|1917|72|923|Friday|1917Q3|N|Y|N|2421473|2421715|2421114|2421388|N|N|N|N|N| +2421480|AAAAAAAAIOCPECAA|1917-09-08|212|923|72|1917|6|9|8|3|1917|72|923|Saturday|1917Q3|N|Y|N|2421473|2421715|2421115|2421389|N|N|N|N|N| +2421481|AAAAAAAAJOCPECAA|1917-09-09|212|923|72|1917|0|9|9|3|1917|72|923|Sunday|1917Q3|N|N|N|2421473|2421715|2421116|2421390|N|N|N|N|N| +2421482|AAAAAAAAKOCPECAA|1917-09-10|212|923|72|1917|1|9|10|3|1917|72|923|Monday|1917Q3|N|N|N|2421473|2421715|2421117|2421391|N|N|N|N|N| +2421483|AAAAAAAALOCPECAA|1917-09-11|212|924|72|1917|2|9|11|3|1917|72|924|Tuesday|1917Q3|N|N|N|2421473|2421715|2421118|2421392|N|N|N|N|N| +2421484|AAAAAAAAMOCPECAA|1917-09-12|212|924|72|1917|3|9|12|3|1917|72|924|Wednesday|1917Q3|N|N|N|2421473|2421715|2421119|2421393|N|N|N|N|N| +2421485|AAAAAAAANOCPECAA|1917-09-13|212|924|72|1917|4|9|13|3|1917|72|924|Thursday|1917Q3|N|N|N|2421473|2421715|2421120|2421394|N|N|N|N|N| +2421486|AAAAAAAAOOCPECAA|1917-09-14|212|924|72|1917|5|9|14|3|1917|72|924|Friday|1917Q3|N|Y|N|2421473|2421715|2421121|2421395|N|N|N|N|N| +2421487|AAAAAAAAPOCPECAA|1917-09-15|212|924|72|1917|6|9|15|3|1917|72|924|Saturday|1917Q3|N|Y|N|2421473|2421715|2421122|2421396|N|N|N|N|N| +2421488|AAAAAAAAAPCPECAA|1917-09-16|212|924|72|1917|0|9|16|3|1917|72|924|Sunday|1917Q3|N|N|N|2421473|2421715|2421123|2421397|N|N|N|N|N| +2421489|AAAAAAAABPCPECAA|1917-09-17|212|924|72|1917|1|9|17|3|1917|72|924|Monday|1917Q3|N|N|N|2421473|2421715|2421124|2421398|N|N|N|N|N| +2421490|AAAAAAAACPCPECAA|1917-09-18|212|925|72|1917|2|9|18|3|1917|72|925|Tuesday|1917Q3|N|N|N|2421473|2421715|2421125|2421399|N|N|N|N|N| +2421491|AAAAAAAADPCPECAA|1917-09-19|212|925|72|1917|3|9|19|3|1917|72|925|Wednesday|1917Q3|N|N|N|2421473|2421715|2421126|2421400|N|N|N|N|N| +2421492|AAAAAAAAEPCPECAA|1917-09-20|212|925|72|1917|4|9|20|3|1917|72|925|Thursday|1917Q3|N|N|N|2421473|2421715|2421127|2421401|N|N|N|N|N| +2421493|AAAAAAAAFPCPECAA|1917-09-21|212|925|72|1917|5|9|21|3|1917|72|925|Friday|1917Q3|N|Y|N|2421473|2421715|2421128|2421402|N|N|N|N|N| +2421494|AAAAAAAAGPCPECAA|1917-09-22|212|925|72|1917|6|9|22|3|1917|72|925|Saturday|1917Q3|N|Y|N|2421473|2421715|2421129|2421403|N|N|N|N|N| +2421495|AAAAAAAAHPCPECAA|1917-09-23|212|925|72|1917|0|9|23|3|1917|72|925|Sunday|1917Q3|N|N|N|2421473|2421715|2421130|2421404|N|N|N|N|N| +2421496|AAAAAAAAIPCPECAA|1917-09-24|212|925|72|1917|1|9|24|3|1917|72|925|Monday|1917Q3|N|N|N|2421473|2421715|2421131|2421405|N|N|N|N|N| +2421497|AAAAAAAAJPCPECAA|1917-09-25|212|926|72|1917|2|9|25|3|1917|72|926|Tuesday|1917Q3|N|N|N|2421473|2421715|2421132|2421406|N|N|N|N|N| +2421498|AAAAAAAAKPCPECAA|1917-09-26|212|926|72|1917|3|9|26|3|1917|72|926|Wednesday|1917Q3|N|N|N|2421473|2421715|2421133|2421407|N|N|N|N|N| +2421499|AAAAAAAALPCPECAA|1917-09-27|212|926|72|1917|4|9|27|3|1917|72|926|Thursday|1917Q3|N|N|N|2421473|2421715|2421134|2421408|N|N|N|N|N| +2421500|AAAAAAAAMPCPECAA|1917-09-28|212|926|72|1917|5|9|28|3|1917|72|926|Friday|1917Q3|N|Y|N|2421473|2421715|2421135|2421409|N|N|N|N|N| +2421501|AAAAAAAANPCPECAA|1917-09-29|212|926|72|1917|6|9|29|3|1917|72|926|Saturday|1917Q3|N|Y|N|2421473|2421715|2421136|2421410|N|N|N|N|N| +2421502|AAAAAAAAOPCPECAA|1917-09-30|212|926|72|1917|0|9|30|3|1917|72|926|Sunday|1917Q3|N|N|N|2421473|2421715|2421137|2421411|N|N|N|N|N| +2421503|AAAAAAAAPPCPECAA|1917-10-01|213|926|72|1917|1|10|1|3|1917|72|926|Monday|1917Q3|N|N|N|2421503|2421775|2421138|2421411|N|N|N|N|N| +2421504|AAAAAAAAAADPECAA|1917-10-02|213|927|72|1917|2|10|2|4|1917|72|927|Tuesday|1917Q4|N|N|N|2421503|2421775|2421139|2421412|N|N|N|N|N| +2421505|AAAAAAAABADPECAA|1917-10-03|213|927|72|1917|3|10|3|4|1917|72|927|Wednesday|1917Q4|N|N|N|2421503|2421775|2421140|2421413|N|N|N|N|N| +2421506|AAAAAAAACADPECAA|1917-10-04|213|927|72|1917|4|10|4|4|1917|72|927|Thursday|1917Q4|N|N|N|2421503|2421775|2421141|2421414|N|N|N|N|N| +2421507|AAAAAAAADADPECAA|1917-10-05|213|927|72|1917|5|10|5|4|1917|72|927|Friday|1917Q4|N|Y|N|2421503|2421775|2421142|2421415|N|N|N|N|N| +2421508|AAAAAAAAEADPECAA|1917-10-06|213|927|72|1917|6|10|6|4|1917|72|927|Saturday|1917Q4|N|Y|N|2421503|2421775|2421143|2421416|N|N|N|N|N| +2421509|AAAAAAAAFADPECAA|1917-10-07|213|927|72|1917|0|10|7|4|1917|72|927|Sunday|1917Q4|N|N|N|2421503|2421775|2421144|2421417|N|N|N|N|N| +2421510|AAAAAAAAGADPECAA|1917-10-08|213|927|72|1917|1|10|8|4|1917|72|927|Monday|1917Q4|N|N|N|2421503|2421775|2421145|2421418|N|N|N|N|N| +2421511|AAAAAAAAHADPECAA|1917-10-09|213|928|72|1917|2|10|9|4|1917|72|928|Tuesday|1917Q4|N|N|N|2421503|2421775|2421146|2421419|N|N|N|N|N| +2421512|AAAAAAAAIADPECAA|1917-10-10|213|928|72|1917|3|10|10|4|1917|72|928|Wednesday|1917Q4|N|N|N|2421503|2421775|2421147|2421420|N|N|N|N|N| +2421513|AAAAAAAAJADPECAA|1917-10-11|213|928|72|1917|4|10|11|4|1917|72|928|Thursday|1917Q4|N|N|N|2421503|2421775|2421148|2421421|N|N|N|N|N| +2421514|AAAAAAAAKADPECAA|1917-10-12|213|928|72|1917|5|10|12|4|1917|72|928|Friday|1917Q4|N|Y|N|2421503|2421775|2421149|2421422|N|N|N|N|N| +2421515|AAAAAAAALADPECAA|1917-10-13|213|928|72|1917|6|10|13|4|1917|72|928|Saturday|1917Q4|N|Y|N|2421503|2421775|2421150|2421423|N|N|N|N|N| +2421516|AAAAAAAAMADPECAA|1917-10-14|213|928|72|1917|0|10|14|4|1917|72|928|Sunday|1917Q4|N|N|N|2421503|2421775|2421151|2421424|N|N|N|N|N| +2421517|AAAAAAAANADPECAA|1917-10-15|213|928|72|1917|1|10|15|4|1917|72|928|Monday|1917Q4|N|N|N|2421503|2421775|2421152|2421425|N|N|N|N|N| +2421518|AAAAAAAAOADPECAA|1917-10-16|213|929|72|1917|2|10|16|4|1917|72|929|Tuesday|1917Q4|N|N|N|2421503|2421775|2421153|2421426|N|N|N|N|N| +2421519|AAAAAAAAPADPECAA|1917-10-17|213|929|72|1917|3|10|17|4|1917|72|929|Wednesday|1917Q4|N|N|N|2421503|2421775|2421154|2421427|N|N|N|N|N| +2421520|AAAAAAAAABDPECAA|1917-10-18|213|929|72|1917|4|10|18|4|1917|72|929|Thursday|1917Q4|N|N|N|2421503|2421775|2421155|2421428|N|N|N|N|N| +2421521|AAAAAAAABBDPECAA|1917-10-19|213|929|72|1917|5|10|19|4|1917|72|929|Friday|1917Q4|N|Y|N|2421503|2421775|2421156|2421429|N|N|N|N|N| +2421522|AAAAAAAACBDPECAA|1917-10-20|213|929|72|1917|6|10|20|4|1917|72|929|Saturday|1917Q4|N|Y|N|2421503|2421775|2421157|2421430|N|N|N|N|N| +2421523|AAAAAAAADBDPECAA|1917-10-21|213|929|72|1917|0|10|21|4|1917|72|929|Sunday|1917Q4|N|N|N|2421503|2421775|2421158|2421431|N|N|N|N|N| +2421524|AAAAAAAAEBDPECAA|1917-10-22|213|929|72|1917|1|10|22|4|1917|72|929|Monday|1917Q4|N|N|N|2421503|2421775|2421159|2421432|N|N|N|N|N| +2421525|AAAAAAAAFBDPECAA|1917-10-23|213|930|72|1917|2|10|23|4|1917|72|930|Tuesday|1917Q4|N|N|N|2421503|2421775|2421160|2421433|N|N|N|N|N| +2421526|AAAAAAAAGBDPECAA|1917-10-24|213|930|72|1917|3|10|24|4|1917|72|930|Wednesday|1917Q4|N|N|N|2421503|2421775|2421161|2421434|N|N|N|N|N| +2421527|AAAAAAAAHBDPECAA|1917-10-25|213|930|72|1917|4|10|25|4|1917|72|930|Thursday|1917Q4|N|N|N|2421503|2421775|2421162|2421435|N|N|N|N|N| +2421528|AAAAAAAAIBDPECAA|1917-10-26|213|930|72|1917|5|10|26|4|1917|72|930|Friday|1917Q4|N|Y|N|2421503|2421775|2421163|2421436|N|N|N|N|N| +2421529|AAAAAAAAJBDPECAA|1917-10-27|213|930|72|1917|6|10|27|4|1917|72|930|Saturday|1917Q4|N|Y|N|2421503|2421775|2421164|2421437|N|N|N|N|N| +2421530|AAAAAAAAKBDPECAA|1917-10-28|213|930|72|1917|0|10|28|4|1917|72|930|Sunday|1917Q4|N|N|N|2421503|2421775|2421165|2421438|N|N|N|N|N| +2421531|AAAAAAAALBDPECAA|1917-10-29|213|930|72|1917|1|10|29|4|1917|72|930|Monday|1917Q4|N|N|N|2421503|2421775|2421166|2421439|N|N|N|N|N| +2421532|AAAAAAAAMBDPECAA|1917-10-30|213|931|72|1917|2|10|30|4|1917|72|931|Tuesday|1917Q4|N|N|N|2421503|2421775|2421167|2421440|N|N|N|N|N| +2421533|AAAAAAAANBDPECAA|1917-10-31|213|931|72|1917|3|10|31|4|1917|72|931|Wednesday|1917Q4|N|N|N|2421503|2421775|2421168|2421441|N|N|N|N|N| +2421534|AAAAAAAAOBDPECAA|1917-11-01|214|931|72|1917|4|11|1|4|1917|72|931|Thursday|1917Q4|N|N|N|2421534|2421837|2421169|2421442|N|N|N|N|N| +2421535|AAAAAAAAPBDPECAA|1917-11-02|214|931|72|1917|5|11|2|4|1917|72|931|Friday|1917Q4|N|Y|N|2421534|2421837|2421170|2421443|N|N|N|N|N| +2421536|AAAAAAAAACDPECAA|1917-11-03|214|931|72|1917|6|11|3|4|1917|72|931|Saturday|1917Q4|N|Y|N|2421534|2421837|2421171|2421444|N|N|N|N|N| +2421537|AAAAAAAABCDPECAA|1917-11-04|214|931|72|1917|0|11|4|4|1917|72|931|Sunday|1917Q4|N|N|N|2421534|2421837|2421172|2421445|N|N|N|N|N| +2421538|AAAAAAAACCDPECAA|1917-11-05|214|931|72|1917|1|11|5|4|1917|72|931|Monday|1917Q4|N|N|N|2421534|2421837|2421173|2421446|N|N|N|N|N| +2421539|AAAAAAAADCDPECAA|1917-11-06|214|932|72|1917|2|11|6|4|1917|72|932|Tuesday|1917Q4|N|N|N|2421534|2421837|2421174|2421447|N|N|N|N|N| +2421540|AAAAAAAAECDPECAA|1917-11-07|214|932|72|1917|3|11|7|4|1917|72|932|Wednesday|1917Q4|N|N|N|2421534|2421837|2421175|2421448|N|N|N|N|N| +2421541|AAAAAAAAFCDPECAA|1917-11-08|214|932|72|1917|4|11|8|4|1917|72|932|Thursday|1917Q4|N|N|N|2421534|2421837|2421176|2421449|N|N|N|N|N| +2421542|AAAAAAAAGCDPECAA|1917-11-09|214|932|72|1917|5|11|9|4|1917|72|932|Friday|1917Q4|N|Y|N|2421534|2421837|2421177|2421450|N|N|N|N|N| +2421543|AAAAAAAAHCDPECAA|1917-11-10|214|932|72|1917|6|11|10|4|1917|72|932|Saturday|1917Q4|N|Y|N|2421534|2421837|2421178|2421451|N|N|N|N|N| +2421544|AAAAAAAAICDPECAA|1917-11-11|214|932|72|1917|0|11|11|4|1917|72|932|Sunday|1917Q4|N|N|N|2421534|2421837|2421179|2421452|N|N|N|N|N| +2421545|AAAAAAAAJCDPECAA|1917-11-12|214|932|72|1917|1|11|12|4|1917|72|932|Monday|1917Q4|N|N|N|2421534|2421837|2421180|2421453|N|N|N|N|N| +2421546|AAAAAAAAKCDPECAA|1917-11-13|214|933|72|1917|2|11|13|4|1917|72|933|Tuesday|1917Q4|N|N|N|2421534|2421837|2421181|2421454|N|N|N|N|N| +2421547|AAAAAAAALCDPECAA|1917-11-14|214|933|72|1917|3|11|14|4|1917|72|933|Wednesday|1917Q4|N|N|N|2421534|2421837|2421182|2421455|N|N|N|N|N| +2421548|AAAAAAAAMCDPECAA|1917-11-15|214|933|72|1917|4|11|15|4|1917|72|933|Thursday|1917Q4|N|N|N|2421534|2421837|2421183|2421456|N|N|N|N|N| +2421549|AAAAAAAANCDPECAA|1917-11-16|214|933|72|1917|5|11|16|4|1917|72|933|Friday|1917Q4|N|Y|N|2421534|2421837|2421184|2421457|N|N|N|N|N| +2421550|AAAAAAAAOCDPECAA|1917-11-17|214|933|72|1917|6|11|17|4|1917|72|933|Saturday|1917Q4|N|Y|N|2421534|2421837|2421185|2421458|N|N|N|N|N| +2421551|AAAAAAAAPCDPECAA|1917-11-18|214|933|72|1917|0|11|18|4|1917|72|933|Sunday|1917Q4|N|N|N|2421534|2421837|2421186|2421459|N|N|N|N|N| +2421552|AAAAAAAAADDPECAA|1917-11-19|214|933|72|1917|1|11|19|4|1917|72|933|Monday|1917Q4|N|N|N|2421534|2421837|2421187|2421460|N|N|N|N|N| +2421553|AAAAAAAABDDPECAA|1917-11-20|214|934|72|1917|2|11|20|4|1917|72|934|Tuesday|1917Q4|N|N|N|2421534|2421837|2421188|2421461|N|N|N|N|N| +2421554|AAAAAAAACDDPECAA|1917-11-21|214|934|72|1917|3|11|21|4|1917|72|934|Wednesday|1917Q4|N|N|N|2421534|2421837|2421189|2421462|N|N|N|N|N| +2421555|AAAAAAAADDDPECAA|1917-11-22|214|934|72|1917|4|11|22|4|1917|72|934|Thursday|1917Q4|N|N|N|2421534|2421837|2421190|2421463|N|N|N|N|N| +2421556|AAAAAAAAEDDPECAA|1917-11-23|214|934|72|1917|5|11|23|4|1917|72|934|Friday|1917Q4|N|Y|N|2421534|2421837|2421191|2421464|N|N|N|N|N| +2421557|AAAAAAAAFDDPECAA|1917-11-24|214|934|72|1917|6|11|24|4|1917|72|934|Saturday|1917Q4|N|Y|N|2421534|2421837|2421192|2421465|N|N|N|N|N| +2421558|AAAAAAAAGDDPECAA|1917-11-25|214|934|72|1917|0|11|25|4|1917|72|934|Sunday|1917Q4|N|N|N|2421534|2421837|2421193|2421466|N|N|N|N|N| +2421559|AAAAAAAAHDDPECAA|1917-11-26|214|934|72|1917|1|11|26|4|1917|72|934|Monday|1917Q4|N|N|N|2421534|2421837|2421194|2421467|N|N|N|N|N| +2421560|AAAAAAAAIDDPECAA|1917-11-27|214|935|72|1917|2|11|27|4|1917|72|935|Tuesday|1917Q4|N|N|N|2421534|2421837|2421195|2421468|N|N|N|N|N| +2421561|AAAAAAAAJDDPECAA|1917-11-28|214|935|72|1917|3|11|28|4|1917|72|935|Wednesday|1917Q4|N|N|N|2421534|2421837|2421196|2421469|N|N|N|N|N| +2421562|AAAAAAAAKDDPECAA|1917-11-29|214|935|72|1917|4|11|29|4|1917|72|935|Thursday|1917Q4|N|N|N|2421534|2421837|2421197|2421470|N|N|N|N|N| +2421563|AAAAAAAALDDPECAA|1917-11-30|214|935|72|1917|5|11|30|4|1917|72|935|Friday|1917Q4|N|Y|N|2421534|2421837|2421198|2421471|N|N|N|N|N| +2421564|AAAAAAAAMDDPECAA|1917-12-01|215|935|73|1917|6|12|1|4|1917|73|935|Saturday|1917Q4|N|Y|N|2421564|2421897|2421199|2421472|N|N|N|N|N| +2421565|AAAAAAAANDDPECAA|1917-12-02|215|935|73|1917|0|12|2|4|1917|73|935|Sunday|1917Q4|N|N|N|2421564|2421897|2421200|2421473|N|N|N|N|N| +2421566|AAAAAAAAODDPECAA|1917-12-03|215|935|73|1917|1|12|3|4|1917|73|935|Monday|1917Q4|N|N|N|2421564|2421897|2421201|2421474|N|N|N|N|N| +2421567|AAAAAAAAPDDPECAA|1917-12-04|215|936|73|1917|2|12|4|4|1917|73|936|Tuesday|1917Q4|N|N|N|2421564|2421897|2421202|2421475|N|N|N|N|N| +2421568|AAAAAAAAAEDPECAA|1917-12-05|215|936|73|1917|3|12|5|4|1917|73|936|Wednesday|1917Q4|N|N|N|2421564|2421897|2421203|2421476|N|N|N|N|N| +2421569|AAAAAAAABEDPECAA|1917-12-06|215|936|73|1917|4|12|6|4|1917|73|936|Thursday|1917Q4|N|N|N|2421564|2421897|2421204|2421477|N|N|N|N|N| +2421570|AAAAAAAACEDPECAA|1917-12-07|215|936|73|1917|5|12|7|4|1917|73|936|Friday|1917Q4|N|Y|N|2421564|2421897|2421205|2421478|N|N|N|N|N| +2421571|AAAAAAAADEDPECAA|1917-12-08|215|936|73|1917|6|12|8|4|1917|73|936|Saturday|1917Q4|N|Y|N|2421564|2421897|2421206|2421479|N|N|N|N|N| +2421572|AAAAAAAAEEDPECAA|1917-12-09|215|936|73|1917|0|12|9|4|1917|73|936|Sunday|1917Q4|N|N|N|2421564|2421897|2421207|2421480|N|N|N|N|N| +2421573|AAAAAAAAFEDPECAA|1917-12-10|215|936|73|1917|1|12|10|4|1917|73|936|Monday|1917Q4|N|N|N|2421564|2421897|2421208|2421481|N|N|N|N|N| +2421574|AAAAAAAAGEDPECAA|1917-12-11|215|937|73|1917|2|12|11|4|1917|73|937|Tuesday|1917Q4|N|N|N|2421564|2421897|2421209|2421482|N|N|N|N|N| +2421575|AAAAAAAAHEDPECAA|1917-12-12|215|937|73|1917|3|12|12|4|1917|73|937|Wednesday|1917Q4|N|N|N|2421564|2421897|2421210|2421483|N|N|N|N|N| +2421576|AAAAAAAAIEDPECAA|1917-12-13|215|937|73|1917|4|12|13|4|1917|73|937|Thursday|1917Q4|N|N|N|2421564|2421897|2421211|2421484|N|N|N|N|N| +2421577|AAAAAAAAJEDPECAA|1917-12-14|215|937|73|1917|5|12|14|4|1917|73|937|Friday|1917Q4|N|Y|N|2421564|2421897|2421212|2421485|N|N|N|N|N| +2421578|AAAAAAAAKEDPECAA|1917-12-15|215|937|73|1917|6|12|15|4|1917|73|937|Saturday|1917Q4|N|Y|N|2421564|2421897|2421213|2421486|N|N|N|N|N| +2421579|AAAAAAAALEDPECAA|1917-12-16|215|937|73|1917|0|12|16|4|1917|73|937|Sunday|1917Q4|N|N|N|2421564|2421897|2421214|2421487|N|N|N|N|N| +2421580|AAAAAAAAMEDPECAA|1917-12-17|215|937|73|1917|1|12|17|4|1917|73|937|Monday|1917Q4|N|N|N|2421564|2421897|2421215|2421488|N|N|N|N|N| +2421581|AAAAAAAANEDPECAA|1917-12-18|215|938|73|1917|2|12|18|4|1917|73|938|Tuesday|1917Q4|N|N|N|2421564|2421897|2421216|2421489|N|N|N|N|N| +2421582|AAAAAAAAOEDPECAA|1917-12-19|215|938|73|1917|3|12|19|4|1917|73|938|Wednesday|1917Q4|N|N|N|2421564|2421897|2421217|2421490|N|N|N|N|N| +2421583|AAAAAAAAPEDPECAA|1917-12-20|215|938|73|1917|4|12|20|4|1917|73|938|Thursday|1917Q4|N|N|N|2421564|2421897|2421218|2421491|N|N|N|N|N| +2421584|AAAAAAAAAFDPECAA|1917-12-21|215|938|73|1917|5|12|21|4|1917|73|938|Friday|1917Q4|N|Y|N|2421564|2421897|2421219|2421492|N|N|N|N|N| +2421585|AAAAAAAABFDPECAA|1917-12-22|215|938|73|1917|6|12|22|4|1917|73|938|Saturday|1917Q4|N|Y|N|2421564|2421897|2421220|2421493|N|N|N|N|N| +2421586|AAAAAAAACFDPECAA|1917-12-23|215|938|73|1917|0|12|23|4|1917|73|938|Sunday|1917Q4|N|N|N|2421564|2421897|2421221|2421494|N|N|N|N|N| +2421587|AAAAAAAADFDPECAA|1917-12-24|215|938|73|1917|1|12|24|4|1917|73|938|Monday|1917Q4|N|N|N|2421564|2421897|2421222|2421495|N|N|N|N|N| +2421588|AAAAAAAAEFDPECAA|1917-12-25|215|939|73|1917|2|12|25|4|1917|73|939|Tuesday|1917Q4|N|N|N|2421564|2421897|2421223|2421496|N|N|N|N|N| +2421589|AAAAAAAAFFDPECAA|1917-12-26|215|939|73|1917|3|12|26|4|1917|73|939|Wednesday|1917Q4|Y|N|N|2421564|2421897|2421224|2421497|N|N|N|N|N| +2421590|AAAAAAAAGFDPECAA|1917-12-27|215|939|73|1917|4|12|27|4|1917|73|939|Thursday|1917Q4|N|N|Y|2421564|2421897|2421225|2421498|N|N|N|N|N| +2421591|AAAAAAAAHFDPECAA|1917-12-28|215|939|73|1917|5|12|28|4|1917|73|939|Friday|1917Q4|N|Y|N|2421564|2421897|2421226|2421499|N|N|N|N|N| +2421592|AAAAAAAAIFDPECAA|1917-12-29|215|939|73|1917|6|12|29|4|1917|73|939|Saturday|1917Q4|N|Y|N|2421564|2421897|2421227|2421500|N|N|N|N|N| +2421593|AAAAAAAAJFDPECAA|1917-12-30|215|939|73|1917|0|12|30|4|1917|73|939|Sunday|1917Q4|N|N|N|2421564|2421897|2421228|2421501|N|N|N|N|N| +2421594|AAAAAAAAKFDPECAA|1917-12-31|215|939|73|1917|1|12|31|4|1917|73|939|Monday|1917Q4|N|N|N|2421564|2421897|2421229|2421502|N|N|N|N|N| +2421595|AAAAAAAALFDPECAA|1918-01-01|216|940|73|1918|2|1|1|1|1918|73|940|Tuesday|1918Q1|Y|N|N|2421595|2421594|2421230|2421503|N|N|N|N|N| +2421596|AAAAAAAAMFDPECAA|1918-01-02|216|940|73|1918|3|1|2|1|1918|73|940|Wednesday|1918Q1|N|N|Y|2421595|2421594|2421231|2421504|N|N|N|N|N| +2421597|AAAAAAAANFDPECAA|1918-01-03|216|940|73|1918|4|1|3|1|1918|73|940|Thursday|1918Q1|N|N|N|2421595|2421594|2421232|2421505|N|N|N|N|N| +2421598|AAAAAAAAOFDPECAA|1918-01-04|216|940|73|1918|5|1|4|1|1918|73|940|Friday|1918Q1|N|Y|N|2421595|2421594|2421233|2421506|N|N|N|N|N| +2421599|AAAAAAAAPFDPECAA|1918-01-05|216|940|73|1918|6|1|5|1|1918|73|940|Saturday|1918Q1|N|Y|N|2421595|2421594|2421234|2421507|N|N|N|N|N| +2421600|AAAAAAAAAGDPECAA|1918-01-06|216|940|73|1918|0|1|6|1|1918|73|940|Sunday|1918Q1|N|N|N|2421595|2421594|2421235|2421508|N|N|N|N|N| +2421601|AAAAAAAABGDPECAA|1918-01-07|216|940|73|1918|1|1|7|1|1918|73|940|Monday|1918Q1|N|N|N|2421595|2421594|2421236|2421509|N|N|N|N|N| +2421602|AAAAAAAACGDPECAA|1918-01-08|216|941|73|1918|2|1|8|1|1918|73|941|Tuesday|1918Q1|N|N|N|2421595|2421594|2421237|2421510|N|N|N|N|N| +2421603|AAAAAAAADGDPECAA|1918-01-09|216|941|73|1918|3|1|9|1|1918|73|941|Wednesday|1918Q1|N|N|N|2421595|2421594|2421238|2421511|N|N|N|N|N| +2421604|AAAAAAAAEGDPECAA|1918-01-10|216|941|73|1918|4|1|10|1|1918|73|941|Thursday|1918Q1|N|N|N|2421595|2421594|2421239|2421512|N|N|N|N|N| +2421605|AAAAAAAAFGDPECAA|1918-01-11|216|941|73|1918|5|1|11|1|1918|73|941|Friday|1918Q1|N|Y|N|2421595|2421594|2421240|2421513|N|N|N|N|N| +2421606|AAAAAAAAGGDPECAA|1918-01-12|216|941|73|1918|6|1|12|1|1918|73|941|Saturday|1918Q1|N|Y|N|2421595|2421594|2421241|2421514|N|N|N|N|N| +2421607|AAAAAAAAHGDPECAA|1918-01-13|216|941|73|1918|0|1|13|1|1918|73|941|Sunday|1918Q1|N|N|N|2421595|2421594|2421242|2421515|N|N|N|N|N| +2421608|AAAAAAAAIGDPECAA|1918-01-14|216|941|73|1918|1|1|14|1|1918|73|941|Monday|1918Q1|N|N|N|2421595|2421594|2421243|2421516|N|N|N|N|N| +2421609|AAAAAAAAJGDPECAA|1918-01-15|216|942|73|1918|2|1|15|1|1918|73|942|Tuesday|1918Q1|N|N|N|2421595|2421594|2421244|2421517|N|N|N|N|N| +2421610|AAAAAAAAKGDPECAA|1918-01-16|216|942|73|1918|3|1|16|1|1918|73|942|Wednesday|1918Q1|N|N|N|2421595|2421594|2421245|2421518|N|N|N|N|N| +2421611|AAAAAAAALGDPECAA|1918-01-17|216|942|73|1918|4|1|17|1|1918|73|942|Thursday|1918Q1|N|N|N|2421595|2421594|2421246|2421519|N|N|N|N|N| +2421612|AAAAAAAAMGDPECAA|1918-01-18|216|942|73|1918|5|1|18|1|1918|73|942|Friday|1918Q1|N|Y|N|2421595|2421594|2421247|2421520|N|N|N|N|N| +2421613|AAAAAAAANGDPECAA|1918-01-19|216|942|73|1918|6|1|19|1|1918|73|942|Saturday|1918Q1|N|Y|N|2421595|2421594|2421248|2421521|N|N|N|N|N| +2421614|AAAAAAAAOGDPECAA|1918-01-20|216|942|73|1918|0|1|20|1|1918|73|942|Sunday|1918Q1|N|N|N|2421595|2421594|2421249|2421522|N|N|N|N|N| +2421615|AAAAAAAAPGDPECAA|1918-01-21|216|942|73|1918|1|1|21|1|1918|73|942|Monday|1918Q1|N|N|N|2421595|2421594|2421250|2421523|N|N|N|N|N| +2421616|AAAAAAAAAHDPECAA|1918-01-22|216|943|73|1918|2|1|22|1|1918|73|943|Tuesday|1918Q1|N|N|N|2421595|2421594|2421251|2421524|N|N|N|N|N| +2421617|AAAAAAAABHDPECAA|1918-01-23|216|943|73|1918|3|1|23|1|1918|73|943|Wednesday|1918Q1|N|N|N|2421595|2421594|2421252|2421525|N|N|N|N|N| +2421618|AAAAAAAACHDPECAA|1918-01-24|216|943|73|1918|4|1|24|1|1918|73|943|Thursday|1918Q1|N|N|N|2421595|2421594|2421253|2421526|N|N|N|N|N| +2421619|AAAAAAAADHDPECAA|1918-01-25|216|943|73|1918|5|1|25|1|1918|73|943|Friday|1918Q1|N|Y|N|2421595|2421594|2421254|2421527|N|N|N|N|N| +2421620|AAAAAAAAEHDPECAA|1918-01-26|216|943|73|1918|6|1|26|1|1918|73|943|Saturday|1918Q1|N|Y|N|2421595|2421594|2421255|2421528|N|N|N|N|N| +2421621|AAAAAAAAFHDPECAA|1918-01-27|216|943|73|1918|0|1|27|1|1918|73|943|Sunday|1918Q1|N|N|N|2421595|2421594|2421256|2421529|N|N|N|N|N| +2421622|AAAAAAAAGHDPECAA|1918-01-28|216|943|73|1918|1|1|28|1|1918|73|943|Monday|1918Q1|N|N|N|2421595|2421594|2421257|2421530|N|N|N|N|N| +2421623|AAAAAAAAHHDPECAA|1918-01-29|216|944|73|1918|2|1|29|1|1918|73|944|Tuesday|1918Q1|N|N|N|2421595|2421594|2421258|2421531|N|N|N|N|N| +2421624|AAAAAAAAIHDPECAA|1918-01-30|216|944|73|1918|3|1|30|1|1918|73|944|Wednesday|1918Q1|N|N|N|2421595|2421594|2421259|2421532|N|N|N|N|N| +2421625|AAAAAAAAJHDPECAA|1918-01-31|216|944|73|1918|4|1|31|1|1918|73|944|Thursday|1918Q1|N|N|N|2421595|2421594|2421260|2421533|N|N|N|N|N| +2421626|AAAAAAAAKHDPECAA|1918-02-01|217|944|73|1918|5|2|1|1|1918|73|944|Friday|1918Q1|N|Y|N|2421626|2421656|2421261|2421534|N|N|N|N|N| +2421627|AAAAAAAALHDPECAA|1918-02-02|217|944|73|1918|6|2|2|1|1918|73|944|Saturday|1918Q1|N|Y|N|2421626|2421656|2421262|2421535|N|N|N|N|N| +2421628|AAAAAAAAMHDPECAA|1918-02-03|217|944|73|1918|0|2|3|1|1918|73|944|Sunday|1918Q1|N|N|N|2421626|2421656|2421263|2421536|N|N|N|N|N| +2421629|AAAAAAAANHDPECAA|1918-02-04|217|944|73|1918|1|2|4|1|1918|73|944|Monday|1918Q1|N|N|N|2421626|2421656|2421264|2421537|N|N|N|N|N| +2421630|AAAAAAAAOHDPECAA|1918-02-05|217|945|73|1918|2|2|5|1|1918|73|945|Tuesday|1918Q1|N|N|N|2421626|2421656|2421265|2421538|N|N|N|N|N| +2421631|AAAAAAAAPHDPECAA|1918-02-06|217|945|73|1918|3|2|6|1|1918|73|945|Wednesday|1918Q1|N|N|N|2421626|2421656|2421266|2421539|N|N|N|N|N| +2421632|AAAAAAAAAIDPECAA|1918-02-07|217|945|73|1918|4|2|7|1|1918|73|945|Thursday|1918Q1|N|N|N|2421626|2421656|2421267|2421540|N|N|N|N|N| +2421633|AAAAAAAABIDPECAA|1918-02-08|217|945|73|1918|5|2|8|1|1918|73|945|Friday|1918Q1|N|Y|N|2421626|2421656|2421268|2421541|N|N|N|N|N| +2421634|AAAAAAAACIDPECAA|1918-02-09|217|945|73|1918|6|2|9|1|1918|73|945|Saturday|1918Q1|N|Y|N|2421626|2421656|2421269|2421542|N|N|N|N|N| +2421635|AAAAAAAADIDPECAA|1918-02-10|217|945|73|1918|0|2|10|1|1918|73|945|Sunday|1918Q1|N|N|N|2421626|2421656|2421270|2421543|N|N|N|N|N| +2421636|AAAAAAAAEIDPECAA|1918-02-11|217|945|73|1918|1|2|11|1|1918|73|945|Monday|1918Q1|N|N|N|2421626|2421656|2421271|2421544|N|N|N|N|N| +2421637|AAAAAAAAFIDPECAA|1918-02-12|217|946|73|1918|2|2|12|1|1918|73|946|Tuesday|1918Q1|N|N|N|2421626|2421656|2421272|2421545|N|N|N|N|N| +2421638|AAAAAAAAGIDPECAA|1918-02-13|217|946|73|1918|3|2|13|1|1918|73|946|Wednesday|1918Q1|N|N|N|2421626|2421656|2421273|2421546|N|N|N|N|N| +2421639|AAAAAAAAHIDPECAA|1918-02-14|217|946|73|1918|4|2|14|1|1918|73|946|Thursday|1918Q1|N|N|N|2421626|2421656|2421274|2421547|N|N|N|N|N| +2421640|AAAAAAAAIIDPECAA|1918-02-15|217|946|73|1918|5|2|15|1|1918|73|946|Friday|1918Q1|N|Y|N|2421626|2421656|2421275|2421548|N|N|N|N|N| +2421641|AAAAAAAAJIDPECAA|1918-02-16|217|946|73|1918|6|2|16|1|1918|73|946|Saturday|1918Q1|N|Y|N|2421626|2421656|2421276|2421549|N|N|N|N|N| +2421642|AAAAAAAAKIDPECAA|1918-02-17|217|946|73|1918|0|2|17|1|1918|73|946|Sunday|1918Q1|N|N|N|2421626|2421656|2421277|2421550|N|N|N|N|N| +2421643|AAAAAAAALIDPECAA|1918-02-18|217|946|73|1918|1|2|18|1|1918|73|946|Monday|1918Q1|N|N|N|2421626|2421656|2421278|2421551|N|N|N|N|N| +2421644|AAAAAAAAMIDPECAA|1918-02-19|217|947|73|1918|2|2|19|1|1918|73|947|Tuesday|1918Q1|N|N|N|2421626|2421656|2421279|2421552|N|N|N|N|N| +2421645|AAAAAAAANIDPECAA|1918-02-20|217|947|73|1918|3|2|20|1|1918|73|947|Wednesday|1918Q1|N|N|N|2421626|2421656|2421280|2421553|N|N|N|N|N| +2421646|AAAAAAAAOIDPECAA|1918-02-21|217|947|73|1918|4|2|21|1|1918|73|947|Thursday|1918Q1|N|N|N|2421626|2421656|2421281|2421554|N|N|N|N|N| +2421647|AAAAAAAAPIDPECAA|1918-02-22|217|947|73|1918|5|2|22|1|1918|73|947|Friday|1918Q1|N|Y|N|2421626|2421656|2421282|2421555|N|N|N|N|N| +2421648|AAAAAAAAAJDPECAA|1918-02-23|217|947|73|1918|6|2|23|1|1918|73|947|Saturday|1918Q1|N|Y|N|2421626|2421656|2421283|2421556|N|N|N|N|N| +2421649|AAAAAAAABJDPECAA|1918-02-24|217|947|73|1918|0|2|24|1|1918|73|947|Sunday|1918Q1|N|N|N|2421626|2421656|2421284|2421557|N|N|N|N|N| +2421650|AAAAAAAACJDPECAA|1918-02-25|217|947|73|1918|1|2|25|1|1918|73|947|Monday|1918Q1|N|N|N|2421626|2421656|2421285|2421558|N|N|N|N|N| +2421651|AAAAAAAADJDPECAA|1918-02-26|217|948|73|1918|2|2|26|1|1918|73|948|Tuesday|1918Q1|N|N|N|2421626|2421656|2421286|2421559|N|N|N|N|N| +2421652|AAAAAAAAEJDPECAA|1918-02-27|217|948|73|1918|3|2|27|1|1918|73|948|Wednesday|1918Q1|N|N|N|2421626|2421656|2421287|2421560|N|N|N|N|N| +2421653|AAAAAAAAFJDPECAA|1918-02-28|217|948|73|1918|4|2|28|1|1918|73|948|Thursday|1918Q1|N|N|N|2421626|2421656|2421288|2421561|N|N|N|N|N| +2421654|AAAAAAAAGJDPECAA|1918-03-01|218|948|74|1918|5|3|1|1|1918|74|948|Friday|1918Q1|N|Y|N|2421654|2421712|2421289|2421562|N|N|N|N|N| +2421655|AAAAAAAAHJDPECAA|1918-03-02|218|948|74|1918|6|3|2|1|1918|74|948|Saturday|1918Q1|N|Y|N|2421654|2421712|2421290|2421563|N|N|N|N|N| +2421656|AAAAAAAAIJDPECAA|1918-03-03|218|948|74|1918|0|3|3|1|1918|74|948|Sunday|1918Q1|N|N|N|2421654|2421712|2421291|2421564|N|N|N|N|N| +2421657|AAAAAAAAJJDPECAA|1918-03-04|218|948|74|1918|1|3|4|1|1918|74|948|Monday|1918Q1|N|N|N|2421654|2421712|2421292|2421565|N|N|N|N|N| +2421658|AAAAAAAAKJDPECAA|1918-03-05|218|949|74|1918|2|3|5|1|1918|74|949|Tuesday|1918Q1|N|N|N|2421654|2421712|2421293|2421566|N|N|N|N|N| +2421659|AAAAAAAALJDPECAA|1918-03-06|218|949|74|1918|3|3|6|1|1918|74|949|Wednesday|1918Q1|N|N|N|2421654|2421712|2421294|2421567|N|N|N|N|N| +2421660|AAAAAAAAMJDPECAA|1918-03-07|218|949|74|1918|4|3|7|1|1918|74|949|Thursday|1918Q1|N|N|N|2421654|2421712|2421295|2421568|N|N|N|N|N| +2421661|AAAAAAAANJDPECAA|1918-03-08|218|949|74|1918|5|3|8|1|1918|74|949|Friday|1918Q1|N|Y|N|2421654|2421712|2421296|2421569|N|N|N|N|N| +2421662|AAAAAAAAOJDPECAA|1918-03-09|218|949|74|1918|6|3|9|1|1918|74|949|Saturday|1918Q1|N|Y|N|2421654|2421712|2421297|2421570|N|N|N|N|N| +2421663|AAAAAAAAPJDPECAA|1918-03-10|218|949|74|1918|0|3|10|1|1918|74|949|Sunday|1918Q1|N|N|N|2421654|2421712|2421298|2421571|N|N|N|N|N| +2421664|AAAAAAAAAKDPECAA|1918-03-11|218|949|74|1918|1|3|11|1|1918|74|949|Monday|1918Q1|N|N|N|2421654|2421712|2421299|2421572|N|N|N|N|N| +2421665|AAAAAAAABKDPECAA|1918-03-12|218|950|74|1918|2|3|12|1|1918|74|950|Tuesday|1918Q1|N|N|N|2421654|2421712|2421300|2421573|N|N|N|N|N| +2421666|AAAAAAAACKDPECAA|1918-03-13|218|950|74|1918|3|3|13|1|1918|74|950|Wednesday|1918Q1|N|N|N|2421654|2421712|2421301|2421574|N|N|N|N|N| +2421667|AAAAAAAADKDPECAA|1918-03-14|218|950|74|1918|4|3|14|1|1918|74|950|Thursday|1918Q1|N|N|N|2421654|2421712|2421302|2421575|N|N|N|N|N| +2421668|AAAAAAAAEKDPECAA|1918-03-15|218|950|74|1918|5|3|15|1|1918|74|950|Friday|1918Q1|N|Y|N|2421654|2421712|2421303|2421576|N|N|N|N|N| +2421669|AAAAAAAAFKDPECAA|1918-03-16|218|950|74|1918|6|3|16|1|1918|74|950|Saturday|1918Q1|N|Y|N|2421654|2421712|2421304|2421577|N|N|N|N|N| +2421670|AAAAAAAAGKDPECAA|1918-03-17|218|950|74|1918|0|3|17|1|1918|74|950|Sunday|1918Q1|N|N|N|2421654|2421712|2421305|2421578|N|N|N|N|N| +2421671|AAAAAAAAHKDPECAA|1918-03-18|218|950|74|1918|1|3|18|1|1918|74|950|Monday|1918Q1|N|N|N|2421654|2421712|2421306|2421579|N|N|N|N|N| +2421672|AAAAAAAAIKDPECAA|1918-03-19|218|951|74|1918|2|3|19|1|1918|74|951|Tuesday|1918Q1|N|N|N|2421654|2421712|2421307|2421580|N|N|N|N|N| +2421673|AAAAAAAAJKDPECAA|1918-03-20|218|951|74|1918|3|3|20|1|1918|74|951|Wednesday|1918Q1|N|N|N|2421654|2421712|2421308|2421581|N|N|N|N|N| +2421674|AAAAAAAAKKDPECAA|1918-03-21|218|951|74|1918|4|3|21|1|1918|74|951|Thursday|1918Q1|N|N|N|2421654|2421712|2421309|2421582|N|N|N|N|N| +2421675|AAAAAAAALKDPECAA|1918-03-22|218|951|74|1918|5|3|22|1|1918|74|951|Friday|1918Q1|N|Y|N|2421654|2421712|2421310|2421583|N|N|N|N|N| +2421676|AAAAAAAAMKDPECAA|1918-03-23|218|951|74|1918|6|3|23|1|1918|74|951|Saturday|1918Q1|N|Y|N|2421654|2421712|2421311|2421584|N|N|N|N|N| +2421677|AAAAAAAANKDPECAA|1918-03-24|218|951|74|1918|0|3|24|1|1918|74|951|Sunday|1918Q1|N|N|N|2421654|2421712|2421312|2421585|N|N|N|N|N| +2421678|AAAAAAAAOKDPECAA|1918-03-25|218|951|74|1918|1|3|25|1|1918|74|951|Monday|1918Q1|N|N|N|2421654|2421712|2421313|2421586|N|N|N|N|N| +2421679|AAAAAAAAPKDPECAA|1918-03-26|218|952|74|1918|2|3|26|1|1918|74|952|Tuesday|1918Q1|N|N|N|2421654|2421712|2421314|2421587|N|N|N|N|N| +2421680|AAAAAAAAALDPECAA|1918-03-27|218|952|74|1918|3|3|27|1|1918|74|952|Wednesday|1918Q1|N|N|N|2421654|2421712|2421315|2421588|N|N|N|N|N| +2421681|AAAAAAAABLDPECAA|1918-03-28|218|952|74|1918|4|3|28|1|1918|74|952|Thursday|1918Q1|N|N|N|2421654|2421712|2421316|2421589|N|N|N|N|N| +2421682|AAAAAAAACLDPECAA|1918-03-29|218|952|74|1918|5|3|29|1|1918|74|952|Friday|1918Q1|N|Y|N|2421654|2421712|2421317|2421590|N|N|N|N|N| +2421683|AAAAAAAADLDPECAA|1918-03-30|218|952|74|1918|6|3|30|1|1918|74|952|Saturday|1918Q1|N|Y|N|2421654|2421712|2421318|2421591|N|N|N|N|N| +2421684|AAAAAAAAELDPECAA|1918-03-31|218|952|74|1918|0|3|31|1|1918|74|952|Sunday|1918Q1|N|N|N|2421654|2421712|2421319|2421592|N|N|N|N|N| +2421685|AAAAAAAAFLDPECAA|1918-04-01|219|952|74|1918|1|4|1|1|1918|74|952|Monday|1918Q1|N|N|N|2421685|2421774|2421320|2421595|N|N|N|N|N| +2421686|AAAAAAAAGLDPECAA|1918-04-02|219|953|74|1918|2|4|2|2|1918|74|953|Tuesday|1918Q2|N|N|N|2421685|2421774|2421321|2421596|N|N|N|N|N| +2421687|AAAAAAAAHLDPECAA|1918-04-03|219|953|74|1918|3|4|3|2|1918|74|953|Wednesday|1918Q2|N|N|N|2421685|2421774|2421322|2421597|N|N|N|N|N| +2421688|AAAAAAAAILDPECAA|1918-04-04|219|953|74|1918|4|4|4|2|1918|74|953|Thursday|1918Q2|N|N|N|2421685|2421774|2421323|2421598|N|N|N|N|N| +2421689|AAAAAAAAJLDPECAA|1918-04-05|219|953|74|1918|5|4|5|2|1918|74|953|Friday|1918Q2|N|Y|N|2421685|2421774|2421324|2421599|N|N|N|N|N| +2421690|AAAAAAAAKLDPECAA|1918-04-06|219|953|74|1918|6|4|6|2|1918|74|953|Saturday|1918Q2|N|Y|N|2421685|2421774|2421325|2421600|N|N|N|N|N| +2421691|AAAAAAAALLDPECAA|1918-04-07|219|953|74|1918|0|4|7|2|1918|74|953|Sunday|1918Q2|N|N|N|2421685|2421774|2421326|2421601|N|N|N|N|N| +2421692|AAAAAAAAMLDPECAA|1918-04-08|219|953|74|1918|1|4|8|2|1918|74|953|Monday|1918Q2|N|N|N|2421685|2421774|2421327|2421602|N|N|N|N|N| +2421693|AAAAAAAANLDPECAA|1918-04-09|219|954|74|1918|2|4|9|2|1918|74|954|Tuesday|1918Q2|N|N|N|2421685|2421774|2421328|2421603|N|N|N|N|N| +2421694|AAAAAAAAOLDPECAA|1918-04-10|219|954|74|1918|3|4|10|2|1918|74|954|Wednesday|1918Q2|N|N|N|2421685|2421774|2421329|2421604|N|N|N|N|N| +2421695|AAAAAAAAPLDPECAA|1918-04-11|219|954|74|1918|4|4|11|2|1918|74|954|Thursday|1918Q2|N|N|N|2421685|2421774|2421330|2421605|N|N|N|N|N| +2421696|AAAAAAAAAMDPECAA|1918-04-12|219|954|74|1918|5|4|12|2|1918|74|954|Friday|1918Q2|N|Y|N|2421685|2421774|2421331|2421606|N|N|N|N|N| +2421697|AAAAAAAABMDPECAA|1918-04-13|219|954|74|1918|6|4|13|2|1918|74|954|Saturday|1918Q2|N|Y|N|2421685|2421774|2421332|2421607|N|N|N|N|N| +2421698|AAAAAAAACMDPECAA|1918-04-14|219|954|74|1918|0|4|14|2|1918|74|954|Sunday|1918Q2|N|N|N|2421685|2421774|2421333|2421608|N|N|N|N|N| +2421699|AAAAAAAADMDPECAA|1918-04-15|219|954|74|1918|1|4|15|2|1918|74|954|Monday|1918Q2|N|N|N|2421685|2421774|2421334|2421609|N|N|N|N|N| +2421700|AAAAAAAAEMDPECAA|1918-04-16|219|955|74|1918|2|4|16|2|1918|74|955|Tuesday|1918Q2|N|N|N|2421685|2421774|2421335|2421610|N|N|N|N|N| +2421701|AAAAAAAAFMDPECAA|1918-04-17|219|955|74|1918|3|4|17|2|1918|74|955|Wednesday|1918Q2|N|N|N|2421685|2421774|2421336|2421611|N|N|N|N|N| +2421702|AAAAAAAAGMDPECAA|1918-04-18|219|955|74|1918|4|4|18|2|1918|74|955|Thursday|1918Q2|N|N|N|2421685|2421774|2421337|2421612|N|N|N|N|N| +2421703|AAAAAAAAHMDPECAA|1918-04-19|219|955|74|1918|5|4|19|2|1918|74|955|Friday|1918Q2|N|Y|N|2421685|2421774|2421338|2421613|N|N|N|N|N| +2421704|AAAAAAAAIMDPECAA|1918-04-20|219|955|74|1918|6|4|20|2|1918|74|955|Saturday|1918Q2|N|Y|N|2421685|2421774|2421339|2421614|N|N|N|N|N| +2421705|AAAAAAAAJMDPECAA|1918-04-21|219|955|74|1918|0|4|21|2|1918|74|955|Sunday|1918Q2|N|N|N|2421685|2421774|2421340|2421615|N|N|N|N|N| +2421706|AAAAAAAAKMDPECAA|1918-04-22|219|955|74|1918|1|4|22|2|1918|74|955|Monday|1918Q2|N|N|N|2421685|2421774|2421341|2421616|N|N|N|N|N| +2421707|AAAAAAAALMDPECAA|1918-04-23|219|956|74|1918|2|4|23|2|1918|74|956|Tuesday|1918Q2|N|N|N|2421685|2421774|2421342|2421617|N|N|N|N|N| +2421708|AAAAAAAAMMDPECAA|1918-04-24|219|956|74|1918|3|4|24|2|1918|74|956|Wednesday|1918Q2|N|N|N|2421685|2421774|2421343|2421618|N|N|N|N|N| +2421709|AAAAAAAANMDPECAA|1918-04-25|219|956|74|1918|4|4|25|2|1918|74|956|Thursday|1918Q2|N|N|N|2421685|2421774|2421344|2421619|N|N|N|N|N| +2421710|AAAAAAAAOMDPECAA|1918-04-26|219|956|74|1918|5|4|26|2|1918|74|956|Friday|1918Q2|N|Y|N|2421685|2421774|2421345|2421620|N|N|N|N|N| +2421711|AAAAAAAAPMDPECAA|1918-04-27|219|956|74|1918|6|4|27|2|1918|74|956|Saturday|1918Q2|N|Y|N|2421685|2421774|2421346|2421621|N|N|N|N|N| +2421712|AAAAAAAAANDPECAA|1918-04-28|219|956|74|1918|0|4|28|2|1918|74|956|Sunday|1918Q2|N|N|N|2421685|2421774|2421347|2421622|N|N|N|N|N| +2421713|AAAAAAAABNDPECAA|1918-04-29|219|956|74|1918|1|4|29|2|1918|74|956|Monday|1918Q2|N|N|N|2421685|2421774|2421348|2421623|N|N|N|N|N| +2421714|AAAAAAAACNDPECAA|1918-04-30|219|957|74|1918|2|4|30|2|1918|74|957|Tuesday|1918Q2|N|N|N|2421685|2421774|2421349|2421624|N|N|N|N|N| +2421715|AAAAAAAADNDPECAA|1918-05-01|220|957|74|1918|3|5|1|2|1918|74|957|Wednesday|1918Q2|N|N|N|2421715|2421834|2421350|2421625|N|N|N|N|N| +2421716|AAAAAAAAENDPECAA|1918-05-02|220|957|74|1918|4|5|2|2|1918|74|957|Thursday|1918Q2|N|N|N|2421715|2421834|2421351|2421626|N|N|N|N|N| +2421717|AAAAAAAAFNDPECAA|1918-05-03|220|957|74|1918|5|5|3|2|1918|74|957|Friday|1918Q2|N|Y|N|2421715|2421834|2421352|2421627|N|N|N|N|N| +2421718|AAAAAAAAGNDPECAA|1918-05-04|220|957|74|1918|6|5|4|2|1918|74|957|Saturday|1918Q2|N|Y|N|2421715|2421834|2421353|2421628|N|N|N|N|N| +2421719|AAAAAAAAHNDPECAA|1918-05-05|220|957|74|1918|0|5|5|2|1918|74|957|Sunday|1918Q2|N|N|N|2421715|2421834|2421354|2421629|N|N|N|N|N| +2421720|AAAAAAAAINDPECAA|1918-05-06|220|957|74|1918|1|5|6|2|1918|74|957|Monday|1918Q2|N|N|N|2421715|2421834|2421355|2421630|N|N|N|N|N| +2421721|AAAAAAAAJNDPECAA|1918-05-07|220|958|74|1918|2|5|7|2|1918|74|958|Tuesday|1918Q2|N|N|N|2421715|2421834|2421356|2421631|N|N|N|N|N| +2421722|AAAAAAAAKNDPECAA|1918-05-08|220|958|74|1918|3|5|8|2|1918|74|958|Wednesday|1918Q2|N|N|N|2421715|2421834|2421357|2421632|N|N|N|N|N| +2421723|AAAAAAAALNDPECAA|1918-05-09|220|958|74|1918|4|5|9|2|1918|74|958|Thursday|1918Q2|N|N|N|2421715|2421834|2421358|2421633|N|N|N|N|N| +2421724|AAAAAAAAMNDPECAA|1918-05-10|220|958|74|1918|5|5|10|2|1918|74|958|Friday|1918Q2|N|Y|N|2421715|2421834|2421359|2421634|N|N|N|N|N| +2421725|AAAAAAAANNDPECAA|1918-05-11|220|958|74|1918|6|5|11|2|1918|74|958|Saturday|1918Q2|N|Y|N|2421715|2421834|2421360|2421635|N|N|N|N|N| +2421726|AAAAAAAAONDPECAA|1918-05-12|220|958|74|1918|0|5|12|2|1918|74|958|Sunday|1918Q2|N|N|N|2421715|2421834|2421361|2421636|N|N|N|N|N| +2421727|AAAAAAAAPNDPECAA|1918-05-13|220|958|74|1918|1|5|13|2|1918|74|958|Monday|1918Q2|N|N|N|2421715|2421834|2421362|2421637|N|N|N|N|N| +2421728|AAAAAAAAAODPECAA|1918-05-14|220|959|74|1918|2|5|14|2|1918|74|959|Tuesday|1918Q2|N|N|N|2421715|2421834|2421363|2421638|N|N|N|N|N| +2421729|AAAAAAAABODPECAA|1918-05-15|220|959|74|1918|3|5|15|2|1918|74|959|Wednesday|1918Q2|N|N|N|2421715|2421834|2421364|2421639|N|N|N|N|N| +2421730|AAAAAAAACODPECAA|1918-05-16|220|959|74|1918|4|5|16|2|1918|74|959|Thursday|1918Q2|N|N|N|2421715|2421834|2421365|2421640|N|N|N|N|N| +2421731|AAAAAAAADODPECAA|1918-05-17|220|959|74|1918|5|5|17|2|1918|74|959|Friday|1918Q2|N|Y|N|2421715|2421834|2421366|2421641|N|N|N|N|N| +2421732|AAAAAAAAEODPECAA|1918-05-18|220|959|74|1918|6|5|18|2|1918|74|959|Saturday|1918Q2|N|Y|N|2421715|2421834|2421367|2421642|N|N|N|N|N| +2421733|AAAAAAAAFODPECAA|1918-05-19|220|959|74|1918|0|5|19|2|1918|74|959|Sunday|1918Q2|N|N|N|2421715|2421834|2421368|2421643|N|N|N|N|N| +2421734|AAAAAAAAGODPECAA|1918-05-20|220|959|74|1918|1|5|20|2|1918|74|959|Monday|1918Q2|N|N|N|2421715|2421834|2421369|2421644|N|N|N|N|N| +2421735|AAAAAAAAHODPECAA|1918-05-21|220|960|74|1918|2|5|21|2|1918|74|960|Tuesday|1918Q2|N|N|N|2421715|2421834|2421370|2421645|N|N|N|N|N| +2421736|AAAAAAAAIODPECAA|1918-05-22|220|960|74|1918|3|5|22|2|1918|74|960|Wednesday|1918Q2|N|N|N|2421715|2421834|2421371|2421646|N|N|N|N|N| +2421737|AAAAAAAAJODPECAA|1918-05-23|220|960|74|1918|4|5|23|2|1918|74|960|Thursday|1918Q2|N|N|N|2421715|2421834|2421372|2421647|N|N|N|N|N| +2421738|AAAAAAAAKODPECAA|1918-05-24|220|960|74|1918|5|5|24|2|1918|74|960|Friday|1918Q2|N|Y|N|2421715|2421834|2421373|2421648|N|N|N|N|N| +2421739|AAAAAAAALODPECAA|1918-05-25|220|960|74|1918|6|5|25|2|1918|74|960|Saturday|1918Q2|N|Y|N|2421715|2421834|2421374|2421649|N|N|N|N|N| +2421740|AAAAAAAAMODPECAA|1918-05-26|220|960|74|1918|0|5|26|2|1918|74|960|Sunday|1918Q2|N|N|N|2421715|2421834|2421375|2421650|N|N|N|N|N| +2421741|AAAAAAAANODPECAA|1918-05-27|220|960|74|1918|1|5|27|2|1918|74|960|Monday|1918Q2|N|N|N|2421715|2421834|2421376|2421651|N|N|N|N|N| +2421742|AAAAAAAAOODPECAA|1918-05-28|220|961|74|1918|2|5|28|2|1918|74|961|Tuesday|1918Q2|N|N|N|2421715|2421834|2421377|2421652|N|N|N|N|N| +2421743|AAAAAAAAPODPECAA|1918-05-29|220|961|74|1918|3|5|29|2|1918|74|961|Wednesday|1918Q2|N|N|N|2421715|2421834|2421378|2421653|N|N|N|N|N| +2421744|AAAAAAAAAPDPECAA|1918-05-30|220|961|74|1918|4|5|30|2|1918|74|961|Thursday|1918Q2|N|N|N|2421715|2421834|2421379|2421654|N|N|N|N|N| +2421745|AAAAAAAABPDPECAA|1918-05-31|220|961|74|1918|5|5|31|2|1918|74|961|Friday|1918Q2|N|Y|N|2421715|2421834|2421380|2421655|N|N|N|N|N| +2421746|AAAAAAAACPDPECAA|1918-06-01|221|961|75|1918|6|6|1|2|1918|75|961|Saturday|1918Q2|N|Y|N|2421746|2421896|2421381|2421656|N|N|N|N|N| +2421747|AAAAAAAADPDPECAA|1918-06-02|221|961|75|1918|0|6|2|2|1918|75|961|Sunday|1918Q2|N|N|N|2421746|2421896|2421382|2421657|N|N|N|N|N| +2421748|AAAAAAAAEPDPECAA|1918-06-03|221|961|75|1918|1|6|3|2|1918|75|961|Monday|1918Q2|N|N|N|2421746|2421896|2421383|2421658|N|N|N|N|N| +2421749|AAAAAAAAFPDPECAA|1918-06-04|221|962|75|1918|2|6|4|2|1918|75|962|Tuesday|1918Q2|N|N|N|2421746|2421896|2421384|2421659|N|N|N|N|N| +2421750|AAAAAAAAGPDPECAA|1918-06-05|221|962|75|1918|3|6|5|2|1918|75|962|Wednesday|1918Q2|N|N|N|2421746|2421896|2421385|2421660|N|N|N|N|N| +2421751|AAAAAAAAHPDPECAA|1918-06-06|221|962|75|1918|4|6|6|2|1918|75|962|Thursday|1918Q2|N|N|N|2421746|2421896|2421386|2421661|N|N|N|N|N| +2421752|AAAAAAAAIPDPECAA|1918-06-07|221|962|75|1918|5|6|7|2|1918|75|962|Friday|1918Q2|N|Y|N|2421746|2421896|2421387|2421662|N|N|N|N|N| +2421753|AAAAAAAAJPDPECAA|1918-06-08|221|962|75|1918|6|6|8|2|1918|75|962|Saturday|1918Q2|N|Y|N|2421746|2421896|2421388|2421663|N|N|N|N|N| +2421754|AAAAAAAAKPDPECAA|1918-06-09|221|962|75|1918|0|6|9|2|1918|75|962|Sunday|1918Q2|N|N|N|2421746|2421896|2421389|2421664|N|N|N|N|N| +2421755|AAAAAAAALPDPECAA|1918-06-10|221|962|75|1918|1|6|10|2|1918|75|962|Monday|1918Q2|N|N|N|2421746|2421896|2421390|2421665|N|N|N|N|N| +2421756|AAAAAAAAMPDPECAA|1918-06-11|221|963|75|1918|2|6|11|2|1918|75|963|Tuesday|1918Q2|N|N|N|2421746|2421896|2421391|2421666|N|N|N|N|N| +2421757|AAAAAAAANPDPECAA|1918-06-12|221|963|75|1918|3|6|12|2|1918|75|963|Wednesday|1918Q2|N|N|N|2421746|2421896|2421392|2421667|N|N|N|N|N| +2421758|AAAAAAAAOPDPECAA|1918-06-13|221|963|75|1918|4|6|13|2|1918|75|963|Thursday|1918Q2|N|N|N|2421746|2421896|2421393|2421668|N|N|N|N|N| +2421759|AAAAAAAAPPDPECAA|1918-06-14|221|963|75|1918|5|6|14|2|1918|75|963|Friday|1918Q2|N|Y|N|2421746|2421896|2421394|2421669|N|N|N|N|N| +2421760|AAAAAAAAAAEPECAA|1918-06-15|221|963|75|1918|6|6|15|2|1918|75|963|Saturday|1918Q2|N|Y|N|2421746|2421896|2421395|2421670|N|N|N|N|N| +2421761|AAAAAAAABAEPECAA|1918-06-16|221|963|75|1918|0|6|16|2|1918|75|963|Sunday|1918Q2|N|N|N|2421746|2421896|2421396|2421671|N|N|N|N|N| +2421762|AAAAAAAACAEPECAA|1918-06-17|221|963|75|1918|1|6|17|2|1918|75|963|Monday|1918Q2|N|N|N|2421746|2421896|2421397|2421672|N|N|N|N|N| +2421763|AAAAAAAADAEPECAA|1918-06-18|221|964|75|1918|2|6|18|2|1918|75|964|Tuesday|1918Q2|N|N|N|2421746|2421896|2421398|2421673|N|N|N|N|N| +2421764|AAAAAAAAEAEPECAA|1918-06-19|221|964|75|1918|3|6|19|2|1918|75|964|Wednesday|1918Q2|N|N|N|2421746|2421896|2421399|2421674|N|N|N|N|N| +2421765|AAAAAAAAFAEPECAA|1918-06-20|221|964|75|1918|4|6|20|2|1918|75|964|Thursday|1918Q2|N|N|N|2421746|2421896|2421400|2421675|N|N|N|N|N| +2421766|AAAAAAAAGAEPECAA|1918-06-21|221|964|75|1918|5|6|21|2|1918|75|964|Friday|1918Q2|N|Y|N|2421746|2421896|2421401|2421676|N|N|N|N|N| +2421767|AAAAAAAAHAEPECAA|1918-06-22|221|964|75|1918|6|6|22|2|1918|75|964|Saturday|1918Q2|N|Y|N|2421746|2421896|2421402|2421677|N|N|N|N|N| +2421768|AAAAAAAAIAEPECAA|1918-06-23|221|964|75|1918|0|6|23|2|1918|75|964|Sunday|1918Q2|N|N|N|2421746|2421896|2421403|2421678|N|N|N|N|N| +2421769|AAAAAAAAJAEPECAA|1918-06-24|221|964|75|1918|1|6|24|2|1918|75|964|Monday|1918Q2|N|N|N|2421746|2421896|2421404|2421679|N|N|N|N|N| +2421770|AAAAAAAAKAEPECAA|1918-06-25|221|965|75|1918|2|6|25|2|1918|75|965|Tuesday|1918Q2|N|N|N|2421746|2421896|2421405|2421680|N|N|N|N|N| +2421771|AAAAAAAALAEPECAA|1918-06-26|221|965|75|1918|3|6|26|2|1918|75|965|Wednesday|1918Q2|N|N|N|2421746|2421896|2421406|2421681|N|N|N|N|N| +2421772|AAAAAAAAMAEPECAA|1918-06-27|221|965|75|1918|4|6|27|2|1918|75|965|Thursday|1918Q2|N|N|N|2421746|2421896|2421407|2421682|N|N|N|N|N| +2421773|AAAAAAAANAEPECAA|1918-06-28|221|965|75|1918|5|6|28|2|1918|75|965|Friday|1918Q2|N|Y|N|2421746|2421896|2421408|2421683|N|N|N|N|N| +2421774|AAAAAAAAOAEPECAA|1918-06-29|221|965|75|1918|6|6|29|2|1918|75|965|Saturday|1918Q2|N|Y|N|2421746|2421896|2421409|2421684|N|N|N|N|N| +2421775|AAAAAAAAPAEPECAA|1918-06-30|221|965|75|1918|0|6|30|2|1918|75|965|Sunday|1918Q2|N|N|N|2421746|2421896|2421410|2421685|N|N|N|N|N| +2421776|AAAAAAAAABEPECAA|1918-07-01|222|965|75|1918|1|7|1|2|1918|75|965|Monday|1918Q2|N|N|N|2421776|2421956|2421411|2421685|N|N|N|N|N| +2421777|AAAAAAAABBEPECAA|1918-07-02|222|966|75|1918|2|7|2|3|1918|75|966|Tuesday|1918Q3|N|N|N|2421776|2421956|2421412|2421686|N|N|N|N|N| +2421778|AAAAAAAACBEPECAA|1918-07-03|222|966|75|1918|3|7|3|3|1918|75|966|Wednesday|1918Q3|N|N|N|2421776|2421956|2421413|2421687|N|N|N|N|N| +2421779|AAAAAAAADBEPECAA|1918-07-04|222|966|75|1918|4|7|4|3|1918|75|966|Thursday|1918Q3|N|N|N|2421776|2421956|2421414|2421688|N|N|N|N|N| +2421780|AAAAAAAAEBEPECAA|1918-07-05|222|966|75|1918|5|7|5|3|1918|75|966|Friday|1918Q3|Y|Y|N|2421776|2421956|2421415|2421689|N|N|N|N|N| +2421781|AAAAAAAAFBEPECAA|1918-07-06|222|966|75|1918|6|7|6|3|1918|75|966|Saturday|1918Q3|N|Y|Y|2421776|2421956|2421416|2421690|N|N|N|N|N| +2421782|AAAAAAAAGBEPECAA|1918-07-07|222|966|75|1918|0|7|7|3|1918|75|966|Sunday|1918Q3|N|N|N|2421776|2421956|2421417|2421691|N|N|N|N|N| +2421783|AAAAAAAAHBEPECAA|1918-07-08|222|966|75|1918|1|7|8|3|1918|75|966|Monday|1918Q3|N|N|N|2421776|2421956|2421418|2421692|N|N|N|N|N| +2421784|AAAAAAAAIBEPECAA|1918-07-09|222|967|75|1918|2|7|9|3|1918|75|967|Tuesday|1918Q3|N|N|N|2421776|2421956|2421419|2421693|N|N|N|N|N| +2421785|AAAAAAAAJBEPECAA|1918-07-10|222|967|75|1918|3|7|10|3|1918|75|967|Wednesday|1918Q3|N|N|N|2421776|2421956|2421420|2421694|N|N|N|N|N| +2421786|AAAAAAAAKBEPECAA|1918-07-11|222|967|75|1918|4|7|11|3|1918|75|967|Thursday|1918Q3|N|N|N|2421776|2421956|2421421|2421695|N|N|N|N|N| +2421787|AAAAAAAALBEPECAA|1918-07-12|222|967|75|1918|5|7|12|3|1918|75|967|Friday|1918Q3|N|Y|N|2421776|2421956|2421422|2421696|N|N|N|N|N| +2421788|AAAAAAAAMBEPECAA|1918-07-13|222|967|75|1918|6|7|13|3|1918|75|967|Saturday|1918Q3|N|Y|N|2421776|2421956|2421423|2421697|N|N|N|N|N| +2421789|AAAAAAAANBEPECAA|1918-07-14|222|967|75|1918|0|7|14|3|1918|75|967|Sunday|1918Q3|N|N|N|2421776|2421956|2421424|2421698|N|N|N|N|N| +2421790|AAAAAAAAOBEPECAA|1918-07-15|222|967|75|1918|1|7|15|3|1918|75|967|Monday|1918Q3|N|N|N|2421776|2421956|2421425|2421699|N|N|N|N|N| +2421791|AAAAAAAAPBEPECAA|1918-07-16|222|968|75|1918|2|7|16|3|1918|75|968|Tuesday|1918Q3|N|N|N|2421776|2421956|2421426|2421700|N|N|N|N|N| +2421792|AAAAAAAAACEPECAA|1918-07-17|222|968|75|1918|3|7|17|3|1918|75|968|Wednesday|1918Q3|N|N|N|2421776|2421956|2421427|2421701|N|N|N|N|N| +2421793|AAAAAAAABCEPECAA|1918-07-18|222|968|75|1918|4|7|18|3|1918|75|968|Thursday|1918Q3|N|N|N|2421776|2421956|2421428|2421702|N|N|N|N|N| +2421794|AAAAAAAACCEPECAA|1918-07-19|222|968|75|1918|5|7|19|3|1918|75|968|Friday|1918Q3|N|Y|N|2421776|2421956|2421429|2421703|N|N|N|N|N| +2421795|AAAAAAAADCEPECAA|1918-07-20|222|968|75|1918|6|7|20|3|1918|75|968|Saturday|1918Q3|N|Y|N|2421776|2421956|2421430|2421704|N|N|N|N|N| +2421796|AAAAAAAAECEPECAA|1918-07-21|222|968|75|1918|0|7|21|3|1918|75|968|Sunday|1918Q3|N|N|N|2421776|2421956|2421431|2421705|N|N|N|N|N| +2421797|AAAAAAAAFCEPECAA|1918-07-22|222|968|75|1918|1|7|22|3|1918|75|968|Monday|1918Q3|N|N|N|2421776|2421956|2421432|2421706|N|N|N|N|N| +2421798|AAAAAAAAGCEPECAA|1918-07-23|222|969|75|1918|2|7|23|3|1918|75|969|Tuesday|1918Q3|N|N|N|2421776|2421956|2421433|2421707|N|N|N|N|N| +2421799|AAAAAAAAHCEPECAA|1918-07-24|222|969|75|1918|3|7|24|3|1918|75|969|Wednesday|1918Q3|N|N|N|2421776|2421956|2421434|2421708|N|N|N|N|N| +2421800|AAAAAAAAICEPECAA|1918-07-25|222|969|75|1918|4|7|25|3|1918|75|969|Thursday|1918Q3|N|N|N|2421776|2421956|2421435|2421709|N|N|N|N|N| +2421801|AAAAAAAAJCEPECAA|1918-07-26|222|969|75|1918|5|7|26|3|1918|75|969|Friday|1918Q3|N|Y|N|2421776|2421956|2421436|2421710|N|N|N|N|N| +2421802|AAAAAAAAKCEPECAA|1918-07-27|222|969|75|1918|6|7|27|3|1918|75|969|Saturday|1918Q3|N|Y|N|2421776|2421956|2421437|2421711|N|N|N|N|N| +2421803|AAAAAAAALCEPECAA|1918-07-28|222|969|75|1918|0|7|28|3|1918|75|969|Sunday|1918Q3|N|N|N|2421776|2421956|2421438|2421712|N|N|N|N|N| +2421804|AAAAAAAAMCEPECAA|1918-07-29|222|969|75|1918|1|7|29|3|1918|75|969|Monday|1918Q3|N|N|N|2421776|2421956|2421439|2421713|N|N|N|N|N| +2421805|AAAAAAAANCEPECAA|1918-07-30|222|970|75|1918|2|7|30|3|1918|75|970|Tuesday|1918Q3|N|N|N|2421776|2421956|2421440|2421714|N|N|N|N|N| +2421806|AAAAAAAAOCEPECAA|1918-07-31|222|970|75|1918|3|7|31|3|1918|75|970|Wednesday|1918Q3|N|N|N|2421776|2421956|2421441|2421715|N|N|N|N|N| +2421807|AAAAAAAAPCEPECAA|1918-08-01|223|970|75|1918|4|8|1|3|1918|75|970|Thursday|1918Q3|N|N|N|2421807|2422018|2421442|2421716|N|N|N|N|N| +2421808|AAAAAAAAADEPECAA|1918-08-02|223|970|75|1918|5|8|2|3|1918|75|970|Friday|1918Q3|N|Y|N|2421807|2422018|2421443|2421717|N|N|N|N|N| +2421809|AAAAAAAABDEPECAA|1918-08-03|223|970|75|1918|6|8|3|3|1918|75|970|Saturday|1918Q3|N|Y|N|2421807|2422018|2421444|2421718|N|N|N|N|N| +2421810|AAAAAAAACDEPECAA|1918-08-04|223|970|75|1918|0|8|4|3|1918|75|970|Sunday|1918Q3|N|N|N|2421807|2422018|2421445|2421719|N|N|N|N|N| +2421811|AAAAAAAADDEPECAA|1918-08-05|223|970|75|1918|1|8|5|3|1918|75|970|Monday|1918Q3|N|N|N|2421807|2422018|2421446|2421720|N|N|N|N|N| +2421812|AAAAAAAAEDEPECAA|1918-08-06|223|971|75|1918|2|8|6|3|1918|75|971|Tuesday|1918Q3|N|N|N|2421807|2422018|2421447|2421721|N|N|N|N|N| +2421813|AAAAAAAAFDEPECAA|1918-08-07|223|971|75|1918|3|8|7|3|1918|75|971|Wednesday|1918Q3|N|N|N|2421807|2422018|2421448|2421722|N|N|N|N|N| +2421814|AAAAAAAAGDEPECAA|1918-08-08|223|971|75|1918|4|8|8|3|1918|75|971|Thursday|1918Q3|N|N|N|2421807|2422018|2421449|2421723|N|N|N|N|N| +2421815|AAAAAAAAHDEPECAA|1918-08-09|223|971|75|1918|5|8|9|3|1918|75|971|Friday|1918Q3|N|Y|N|2421807|2422018|2421450|2421724|N|N|N|N|N| +2421816|AAAAAAAAIDEPECAA|1918-08-10|223|971|75|1918|6|8|10|3|1918|75|971|Saturday|1918Q3|N|Y|N|2421807|2422018|2421451|2421725|N|N|N|N|N| +2421817|AAAAAAAAJDEPECAA|1918-08-11|223|971|75|1918|0|8|11|3|1918|75|971|Sunday|1918Q3|N|N|N|2421807|2422018|2421452|2421726|N|N|N|N|N| +2421818|AAAAAAAAKDEPECAA|1918-08-12|223|971|75|1918|1|8|12|3|1918|75|971|Monday|1918Q3|N|N|N|2421807|2422018|2421453|2421727|N|N|N|N|N| +2421819|AAAAAAAALDEPECAA|1918-08-13|223|972|75|1918|2|8|13|3|1918|75|972|Tuesday|1918Q3|N|N|N|2421807|2422018|2421454|2421728|N|N|N|N|N| +2421820|AAAAAAAAMDEPECAA|1918-08-14|223|972|75|1918|3|8|14|3|1918|75|972|Wednesday|1918Q3|N|N|N|2421807|2422018|2421455|2421729|N|N|N|N|N| +2421821|AAAAAAAANDEPECAA|1918-08-15|223|972|75|1918|4|8|15|3|1918|75|972|Thursday|1918Q3|N|N|N|2421807|2422018|2421456|2421730|N|N|N|N|N| +2421822|AAAAAAAAODEPECAA|1918-08-16|223|972|75|1918|5|8|16|3|1918|75|972|Friday|1918Q3|N|Y|N|2421807|2422018|2421457|2421731|N|N|N|N|N| +2421823|AAAAAAAAPDEPECAA|1918-08-17|223|972|75|1918|6|8|17|3|1918|75|972|Saturday|1918Q3|N|Y|N|2421807|2422018|2421458|2421732|N|N|N|N|N| +2421824|AAAAAAAAAEEPECAA|1918-08-18|223|972|75|1918|0|8|18|3|1918|75|972|Sunday|1918Q3|N|N|N|2421807|2422018|2421459|2421733|N|N|N|N|N| +2421825|AAAAAAAABEEPECAA|1918-08-19|223|972|75|1918|1|8|19|3|1918|75|972|Monday|1918Q3|N|N|N|2421807|2422018|2421460|2421734|N|N|N|N|N| +2421826|AAAAAAAACEEPECAA|1918-08-20|223|973|75|1918|2|8|20|3|1918|75|973|Tuesday|1918Q3|N|N|N|2421807|2422018|2421461|2421735|N|N|N|N|N| +2421827|AAAAAAAADEEPECAA|1918-08-21|223|973|75|1918|3|8|21|3|1918|75|973|Wednesday|1918Q3|N|N|N|2421807|2422018|2421462|2421736|N|N|N|N|N| +2421828|AAAAAAAAEEEPECAA|1918-08-22|223|973|75|1918|4|8|22|3|1918|75|973|Thursday|1918Q3|N|N|N|2421807|2422018|2421463|2421737|N|N|N|N|N| +2421829|AAAAAAAAFEEPECAA|1918-08-23|223|973|75|1918|5|8|23|3|1918|75|973|Friday|1918Q3|N|Y|N|2421807|2422018|2421464|2421738|N|N|N|N|N| +2421830|AAAAAAAAGEEPECAA|1918-08-24|223|973|75|1918|6|8|24|3|1918|75|973|Saturday|1918Q3|N|Y|N|2421807|2422018|2421465|2421739|N|N|N|N|N| +2421831|AAAAAAAAHEEPECAA|1918-08-25|223|973|75|1918|0|8|25|3|1918|75|973|Sunday|1918Q3|N|N|N|2421807|2422018|2421466|2421740|N|N|N|N|N| +2421832|AAAAAAAAIEEPECAA|1918-08-26|223|973|75|1918|1|8|26|3|1918|75|973|Monday|1918Q3|N|N|N|2421807|2422018|2421467|2421741|N|N|N|N|N| +2421833|AAAAAAAAJEEPECAA|1918-08-27|223|974|75|1918|2|8|27|3|1918|75|974|Tuesday|1918Q3|N|N|N|2421807|2422018|2421468|2421742|N|N|N|N|N| +2421834|AAAAAAAAKEEPECAA|1918-08-28|223|974|75|1918|3|8|28|3|1918|75|974|Wednesday|1918Q3|N|N|N|2421807|2422018|2421469|2421743|N|N|N|N|N| +2421835|AAAAAAAALEEPECAA|1918-08-29|223|974|75|1918|4|8|29|3|1918|75|974|Thursday|1918Q3|N|N|N|2421807|2422018|2421470|2421744|N|N|N|N|N| +2421836|AAAAAAAAMEEPECAA|1918-08-30|223|974|75|1918|5|8|30|3|1918|75|974|Friday|1918Q3|N|Y|N|2421807|2422018|2421471|2421745|N|N|N|N|N| +2421837|AAAAAAAANEEPECAA|1918-08-31|223|974|75|1918|6|8|31|3|1918|75|974|Saturday|1918Q3|N|Y|N|2421807|2422018|2421472|2421746|N|N|N|N|N| +2421838|AAAAAAAAOEEPECAA|1918-09-01|224|974|76|1918|0|9|1|3|1918|76|974|Sunday|1918Q3|N|N|N|2421838|2422080|2421473|2421747|N|N|N|N|N| +2421839|AAAAAAAAPEEPECAA|1918-09-02|224|974|76|1918|1|9|2|3|1918|76|974|Monday|1918Q3|N|N|N|2421838|2422080|2421474|2421748|N|N|N|N|N| +2421840|AAAAAAAAAFEPECAA|1918-09-03|224|975|76|1918|2|9|3|3|1918|76|975|Tuesday|1918Q3|N|N|N|2421838|2422080|2421475|2421749|N|N|N|N|N| +2421841|AAAAAAAABFEPECAA|1918-09-04|224|975|76|1918|3|9|4|3|1918|76|975|Wednesday|1918Q3|N|N|N|2421838|2422080|2421476|2421750|N|N|N|N|N| +2421842|AAAAAAAACFEPECAA|1918-09-05|224|975|76|1918|4|9|5|3|1918|76|975|Thursday|1918Q3|N|N|N|2421838|2422080|2421477|2421751|N|N|N|N|N| +2421843|AAAAAAAADFEPECAA|1918-09-06|224|975|76|1918|5|9|6|3|1918|76|975|Friday|1918Q3|N|Y|N|2421838|2422080|2421478|2421752|N|N|N|N|N| +2421844|AAAAAAAAEFEPECAA|1918-09-07|224|975|76|1918|6|9|7|3|1918|76|975|Saturday|1918Q3|N|Y|N|2421838|2422080|2421479|2421753|N|N|N|N|N| +2421845|AAAAAAAAFFEPECAA|1918-09-08|224|975|76|1918|0|9|8|3|1918|76|975|Sunday|1918Q3|N|N|N|2421838|2422080|2421480|2421754|N|N|N|N|N| +2421846|AAAAAAAAGFEPECAA|1918-09-09|224|975|76|1918|1|9|9|3|1918|76|975|Monday|1918Q3|N|N|N|2421838|2422080|2421481|2421755|N|N|N|N|N| +2421847|AAAAAAAAHFEPECAA|1918-09-10|224|976|76|1918|2|9|10|3|1918|76|976|Tuesday|1918Q3|N|N|N|2421838|2422080|2421482|2421756|N|N|N|N|N| +2421848|AAAAAAAAIFEPECAA|1918-09-11|224|976|76|1918|3|9|11|3|1918|76|976|Wednesday|1918Q3|N|N|N|2421838|2422080|2421483|2421757|N|N|N|N|N| +2421849|AAAAAAAAJFEPECAA|1918-09-12|224|976|76|1918|4|9|12|3|1918|76|976|Thursday|1918Q3|N|N|N|2421838|2422080|2421484|2421758|N|N|N|N|N| +2421850|AAAAAAAAKFEPECAA|1918-09-13|224|976|76|1918|5|9|13|3|1918|76|976|Friday|1918Q3|N|Y|N|2421838|2422080|2421485|2421759|N|N|N|N|N| +2421851|AAAAAAAALFEPECAA|1918-09-14|224|976|76|1918|6|9|14|3|1918|76|976|Saturday|1918Q3|N|Y|N|2421838|2422080|2421486|2421760|N|N|N|N|N| +2421852|AAAAAAAAMFEPECAA|1918-09-15|224|976|76|1918|0|9|15|3|1918|76|976|Sunday|1918Q3|N|N|N|2421838|2422080|2421487|2421761|N|N|N|N|N| +2421853|AAAAAAAANFEPECAA|1918-09-16|224|976|76|1918|1|9|16|3|1918|76|976|Monday|1918Q3|N|N|N|2421838|2422080|2421488|2421762|N|N|N|N|N| +2421854|AAAAAAAAOFEPECAA|1918-09-17|224|977|76|1918|2|9|17|3|1918|76|977|Tuesday|1918Q3|N|N|N|2421838|2422080|2421489|2421763|N|N|N|N|N| +2421855|AAAAAAAAPFEPECAA|1918-09-18|224|977|76|1918|3|9|18|3|1918|76|977|Wednesday|1918Q3|N|N|N|2421838|2422080|2421490|2421764|N|N|N|N|N| +2421856|AAAAAAAAAGEPECAA|1918-09-19|224|977|76|1918|4|9|19|3|1918|76|977|Thursday|1918Q3|N|N|N|2421838|2422080|2421491|2421765|N|N|N|N|N| +2421857|AAAAAAAABGEPECAA|1918-09-20|224|977|76|1918|5|9|20|3|1918|76|977|Friday|1918Q3|N|Y|N|2421838|2422080|2421492|2421766|N|N|N|N|N| +2421858|AAAAAAAACGEPECAA|1918-09-21|224|977|76|1918|6|9|21|3|1918|76|977|Saturday|1918Q3|N|Y|N|2421838|2422080|2421493|2421767|N|N|N|N|N| +2421859|AAAAAAAADGEPECAA|1918-09-22|224|977|76|1918|0|9|22|3|1918|76|977|Sunday|1918Q3|N|N|N|2421838|2422080|2421494|2421768|N|N|N|N|N| +2421860|AAAAAAAAEGEPECAA|1918-09-23|224|977|76|1918|1|9|23|3|1918|76|977|Monday|1918Q3|N|N|N|2421838|2422080|2421495|2421769|N|N|N|N|N| +2421861|AAAAAAAAFGEPECAA|1918-09-24|224|978|76|1918|2|9|24|3|1918|76|978|Tuesday|1918Q3|N|N|N|2421838|2422080|2421496|2421770|N|N|N|N|N| +2421862|AAAAAAAAGGEPECAA|1918-09-25|224|978|76|1918|3|9|25|3|1918|76|978|Wednesday|1918Q3|N|N|N|2421838|2422080|2421497|2421771|N|N|N|N|N| +2421863|AAAAAAAAHGEPECAA|1918-09-26|224|978|76|1918|4|9|26|3|1918|76|978|Thursday|1918Q3|N|N|N|2421838|2422080|2421498|2421772|N|N|N|N|N| +2421864|AAAAAAAAIGEPECAA|1918-09-27|224|978|76|1918|5|9|27|3|1918|76|978|Friday|1918Q3|N|Y|N|2421838|2422080|2421499|2421773|N|N|N|N|N| +2421865|AAAAAAAAJGEPECAA|1918-09-28|224|978|76|1918|6|9|28|3|1918|76|978|Saturday|1918Q3|N|Y|N|2421838|2422080|2421500|2421774|N|N|N|N|N| +2421866|AAAAAAAAKGEPECAA|1918-09-29|224|978|76|1918|0|9|29|3|1918|76|978|Sunday|1918Q3|N|N|N|2421838|2422080|2421501|2421775|N|N|N|N|N| +2421867|AAAAAAAALGEPECAA|1918-09-30|224|978|76|1918|1|9|30|3|1918|76|978|Monday|1918Q3|N|N|N|2421838|2422080|2421502|2421776|N|N|N|N|N| +2421868|AAAAAAAAMGEPECAA|1918-10-01|225|979|76|1918|2|10|1|3|1918|76|979|Tuesday|1918Q3|N|N|N|2421868|2422140|2421503|2421776|N|N|N|N|N| +2421869|AAAAAAAANGEPECAA|1918-10-02|225|979|76|1918|3|10|2|4|1918|76|979|Wednesday|1918Q4|N|N|N|2421868|2422140|2421504|2421777|N|N|N|N|N| +2421870|AAAAAAAAOGEPECAA|1918-10-03|225|979|76|1918|4|10|3|4|1918|76|979|Thursday|1918Q4|N|N|N|2421868|2422140|2421505|2421778|N|N|N|N|N| +2421871|AAAAAAAAPGEPECAA|1918-10-04|225|979|76|1918|5|10|4|4|1918|76|979|Friday|1918Q4|N|Y|N|2421868|2422140|2421506|2421779|N|N|N|N|N| +2421872|AAAAAAAAAHEPECAA|1918-10-05|225|979|76|1918|6|10|5|4|1918|76|979|Saturday|1918Q4|N|Y|N|2421868|2422140|2421507|2421780|N|N|N|N|N| +2421873|AAAAAAAABHEPECAA|1918-10-06|225|979|76|1918|0|10|6|4|1918|76|979|Sunday|1918Q4|N|N|N|2421868|2422140|2421508|2421781|N|N|N|N|N| +2421874|AAAAAAAACHEPECAA|1918-10-07|225|979|76|1918|1|10|7|4|1918|76|979|Monday|1918Q4|N|N|N|2421868|2422140|2421509|2421782|N|N|N|N|N| +2421875|AAAAAAAADHEPECAA|1918-10-08|225|980|76|1918|2|10|8|4|1918|76|980|Tuesday|1918Q4|N|N|N|2421868|2422140|2421510|2421783|N|N|N|N|N| +2421876|AAAAAAAAEHEPECAA|1918-10-09|225|980|76|1918|3|10|9|4|1918|76|980|Wednesday|1918Q4|N|N|N|2421868|2422140|2421511|2421784|N|N|N|N|N| +2421877|AAAAAAAAFHEPECAA|1918-10-10|225|980|76|1918|4|10|10|4|1918|76|980|Thursday|1918Q4|N|N|N|2421868|2422140|2421512|2421785|N|N|N|N|N| +2421878|AAAAAAAAGHEPECAA|1918-10-11|225|980|76|1918|5|10|11|4|1918|76|980|Friday|1918Q4|N|Y|N|2421868|2422140|2421513|2421786|N|N|N|N|N| +2421879|AAAAAAAAHHEPECAA|1918-10-12|225|980|76|1918|6|10|12|4|1918|76|980|Saturday|1918Q4|N|Y|N|2421868|2422140|2421514|2421787|N|N|N|N|N| +2421880|AAAAAAAAIHEPECAA|1918-10-13|225|980|76|1918|0|10|13|4|1918|76|980|Sunday|1918Q4|N|N|N|2421868|2422140|2421515|2421788|N|N|N|N|N| +2421881|AAAAAAAAJHEPECAA|1918-10-14|225|980|76|1918|1|10|14|4|1918|76|980|Monday|1918Q4|N|N|N|2421868|2422140|2421516|2421789|N|N|N|N|N| +2421882|AAAAAAAAKHEPECAA|1918-10-15|225|981|76|1918|2|10|15|4|1918|76|981|Tuesday|1918Q4|N|N|N|2421868|2422140|2421517|2421790|N|N|N|N|N| +2421883|AAAAAAAALHEPECAA|1918-10-16|225|981|76|1918|3|10|16|4|1918|76|981|Wednesday|1918Q4|N|N|N|2421868|2422140|2421518|2421791|N|N|N|N|N| +2421884|AAAAAAAAMHEPECAA|1918-10-17|225|981|76|1918|4|10|17|4|1918|76|981|Thursday|1918Q4|N|N|N|2421868|2422140|2421519|2421792|N|N|N|N|N| +2421885|AAAAAAAANHEPECAA|1918-10-18|225|981|76|1918|5|10|18|4|1918|76|981|Friday|1918Q4|N|Y|N|2421868|2422140|2421520|2421793|N|N|N|N|N| +2421886|AAAAAAAAOHEPECAA|1918-10-19|225|981|76|1918|6|10|19|4|1918|76|981|Saturday|1918Q4|N|Y|N|2421868|2422140|2421521|2421794|N|N|N|N|N| +2421887|AAAAAAAAPHEPECAA|1918-10-20|225|981|76|1918|0|10|20|4|1918|76|981|Sunday|1918Q4|N|N|N|2421868|2422140|2421522|2421795|N|N|N|N|N| +2421888|AAAAAAAAAIEPECAA|1918-10-21|225|981|76|1918|1|10|21|4|1918|76|981|Monday|1918Q4|N|N|N|2421868|2422140|2421523|2421796|N|N|N|N|N| +2421889|AAAAAAAABIEPECAA|1918-10-22|225|982|76|1918|2|10|22|4|1918|76|982|Tuesday|1918Q4|N|N|N|2421868|2422140|2421524|2421797|N|N|N|N|N| +2421890|AAAAAAAACIEPECAA|1918-10-23|225|982|76|1918|3|10|23|4|1918|76|982|Wednesday|1918Q4|N|N|N|2421868|2422140|2421525|2421798|N|N|N|N|N| +2421891|AAAAAAAADIEPECAA|1918-10-24|225|982|76|1918|4|10|24|4|1918|76|982|Thursday|1918Q4|N|N|N|2421868|2422140|2421526|2421799|N|N|N|N|N| +2421892|AAAAAAAAEIEPECAA|1918-10-25|225|982|76|1918|5|10|25|4|1918|76|982|Friday|1918Q4|N|Y|N|2421868|2422140|2421527|2421800|N|N|N|N|N| +2421893|AAAAAAAAFIEPECAA|1918-10-26|225|982|76|1918|6|10|26|4|1918|76|982|Saturday|1918Q4|N|Y|N|2421868|2422140|2421528|2421801|N|N|N|N|N| +2421894|AAAAAAAAGIEPECAA|1918-10-27|225|982|76|1918|0|10|27|4|1918|76|982|Sunday|1918Q4|N|N|N|2421868|2422140|2421529|2421802|N|N|N|N|N| +2421895|AAAAAAAAHIEPECAA|1918-10-28|225|982|76|1918|1|10|28|4|1918|76|982|Monday|1918Q4|N|N|N|2421868|2422140|2421530|2421803|N|N|N|N|N| +2421896|AAAAAAAAIIEPECAA|1918-10-29|225|983|76|1918|2|10|29|4|1918|76|983|Tuesday|1918Q4|N|N|N|2421868|2422140|2421531|2421804|N|N|N|N|N| +2421897|AAAAAAAAJIEPECAA|1918-10-30|225|983|76|1918|3|10|30|4|1918|76|983|Wednesday|1918Q4|N|N|N|2421868|2422140|2421532|2421805|N|N|N|N|N| +2421898|AAAAAAAAKIEPECAA|1918-10-31|225|983|76|1918|4|10|31|4|1918|76|983|Thursday|1918Q4|N|N|N|2421868|2422140|2421533|2421806|N|N|N|N|N| +2421899|AAAAAAAALIEPECAA|1918-11-01|226|983|76|1918|5|11|1|4|1918|76|983|Friday|1918Q4|N|Y|N|2421899|2422202|2421534|2421807|N|N|N|N|N| +2421900|AAAAAAAAMIEPECAA|1918-11-02|226|983|76|1918|6|11|2|4|1918|76|983|Saturday|1918Q4|N|Y|N|2421899|2422202|2421535|2421808|N|N|N|N|N| +2421901|AAAAAAAANIEPECAA|1918-11-03|226|983|76|1918|0|11|3|4|1918|76|983|Sunday|1918Q4|N|N|N|2421899|2422202|2421536|2421809|N|N|N|N|N| +2421902|AAAAAAAAOIEPECAA|1918-11-04|226|983|76|1918|1|11|4|4|1918|76|983|Monday|1918Q4|N|N|N|2421899|2422202|2421537|2421810|N|N|N|N|N| +2421903|AAAAAAAAPIEPECAA|1918-11-05|226|984|76|1918|2|11|5|4|1918|76|984|Tuesday|1918Q4|N|N|N|2421899|2422202|2421538|2421811|N|N|N|N|N| +2421904|AAAAAAAAAJEPECAA|1918-11-06|226|984|76|1918|3|11|6|4|1918|76|984|Wednesday|1918Q4|N|N|N|2421899|2422202|2421539|2421812|N|N|N|N|N| +2421905|AAAAAAAABJEPECAA|1918-11-07|226|984|76|1918|4|11|7|4|1918|76|984|Thursday|1918Q4|N|N|N|2421899|2422202|2421540|2421813|N|N|N|N|N| +2421906|AAAAAAAACJEPECAA|1918-11-08|226|984|76|1918|5|11|8|4|1918|76|984|Friday|1918Q4|N|Y|N|2421899|2422202|2421541|2421814|N|N|N|N|N| +2421907|AAAAAAAADJEPECAA|1918-11-09|226|984|76|1918|6|11|9|4|1918|76|984|Saturday|1918Q4|N|Y|N|2421899|2422202|2421542|2421815|N|N|N|N|N| +2421908|AAAAAAAAEJEPECAA|1918-11-10|226|984|76|1918|0|11|10|4|1918|76|984|Sunday|1918Q4|N|N|N|2421899|2422202|2421543|2421816|N|N|N|N|N| +2421909|AAAAAAAAFJEPECAA|1918-11-11|226|984|76|1918|1|11|11|4|1918|76|984|Monday|1918Q4|N|N|N|2421899|2422202|2421544|2421817|N|N|N|N|N| +2421910|AAAAAAAAGJEPECAA|1918-11-12|226|985|76|1918|2|11|12|4|1918|76|985|Tuesday|1918Q4|N|N|N|2421899|2422202|2421545|2421818|N|N|N|N|N| +2421911|AAAAAAAAHJEPECAA|1918-11-13|226|985|76|1918|3|11|13|4|1918|76|985|Wednesday|1918Q4|N|N|N|2421899|2422202|2421546|2421819|N|N|N|N|N| +2421912|AAAAAAAAIJEPECAA|1918-11-14|226|985|76|1918|4|11|14|4|1918|76|985|Thursday|1918Q4|N|N|N|2421899|2422202|2421547|2421820|N|N|N|N|N| +2421913|AAAAAAAAJJEPECAA|1918-11-15|226|985|76|1918|5|11|15|4|1918|76|985|Friday|1918Q4|N|Y|N|2421899|2422202|2421548|2421821|N|N|N|N|N| +2421914|AAAAAAAAKJEPECAA|1918-11-16|226|985|76|1918|6|11|16|4|1918|76|985|Saturday|1918Q4|N|Y|N|2421899|2422202|2421549|2421822|N|N|N|N|N| +2421915|AAAAAAAALJEPECAA|1918-11-17|226|985|76|1918|0|11|17|4|1918|76|985|Sunday|1918Q4|N|N|N|2421899|2422202|2421550|2421823|N|N|N|N|N| +2421916|AAAAAAAAMJEPECAA|1918-11-18|226|985|76|1918|1|11|18|4|1918|76|985|Monday|1918Q4|N|N|N|2421899|2422202|2421551|2421824|N|N|N|N|N| +2421917|AAAAAAAANJEPECAA|1918-11-19|226|986|76|1918|2|11|19|4|1918|76|986|Tuesday|1918Q4|N|N|N|2421899|2422202|2421552|2421825|N|N|N|N|N| +2421918|AAAAAAAAOJEPECAA|1918-11-20|226|986|76|1918|3|11|20|4|1918|76|986|Wednesday|1918Q4|N|N|N|2421899|2422202|2421553|2421826|N|N|N|N|N| +2421919|AAAAAAAAPJEPECAA|1918-11-21|226|986|76|1918|4|11|21|4|1918|76|986|Thursday|1918Q4|N|N|N|2421899|2422202|2421554|2421827|N|N|N|N|N| +2421920|AAAAAAAAAKEPECAA|1918-11-22|226|986|76|1918|5|11|22|4|1918|76|986|Friday|1918Q4|N|Y|N|2421899|2422202|2421555|2421828|N|N|N|N|N| +2421921|AAAAAAAABKEPECAA|1918-11-23|226|986|76|1918|6|11|23|4|1918|76|986|Saturday|1918Q4|N|Y|N|2421899|2422202|2421556|2421829|N|N|N|N|N| +2421922|AAAAAAAACKEPECAA|1918-11-24|226|986|76|1918|0|11|24|4|1918|76|986|Sunday|1918Q4|N|N|N|2421899|2422202|2421557|2421830|N|N|N|N|N| +2421923|AAAAAAAADKEPECAA|1918-11-25|226|986|76|1918|1|11|25|4|1918|76|986|Monday|1918Q4|N|N|N|2421899|2422202|2421558|2421831|N|N|N|N|N| +2421924|AAAAAAAAEKEPECAA|1918-11-26|226|987|76|1918|2|11|26|4|1918|76|987|Tuesday|1918Q4|N|N|N|2421899|2422202|2421559|2421832|N|N|N|N|N| +2421925|AAAAAAAAFKEPECAA|1918-11-27|226|987|76|1918|3|11|27|4|1918|76|987|Wednesday|1918Q4|N|N|N|2421899|2422202|2421560|2421833|N|N|N|N|N| +2421926|AAAAAAAAGKEPECAA|1918-11-28|226|987|76|1918|4|11|28|4|1918|76|987|Thursday|1918Q4|N|N|N|2421899|2422202|2421561|2421834|N|N|N|N|N| +2421927|AAAAAAAAHKEPECAA|1918-11-29|226|987|76|1918|5|11|29|4|1918|76|987|Friday|1918Q4|N|Y|N|2421899|2422202|2421562|2421835|N|N|N|N|N| +2421928|AAAAAAAAIKEPECAA|1918-11-30|226|987|76|1918|6|11|30|4|1918|76|987|Saturday|1918Q4|N|Y|N|2421899|2422202|2421563|2421836|N|N|N|N|N| +2421929|AAAAAAAAJKEPECAA|1918-12-01|227|987|77|1918|0|12|1|4|1918|77|987|Sunday|1918Q4|N|N|N|2421929|2422262|2421564|2421837|N|N|N|N|N| +2421930|AAAAAAAAKKEPECAA|1918-12-02|227|987|77|1918|1|12|2|4|1918|77|987|Monday|1918Q4|N|N|N|2421929|2422262|2421565|2421838|N|N|N|N|N| +2421931|AAAAAAAALKEPECAA|1918-12-03|227|988|77|1918|2|12|3|4|1918|77|988|Tuesday|1918Q4|N|N|N|2421929|2422262|2421566|2421839|N|N|N|N|N| +2421932|AAAAAAAAMKEPECAA|1918-12-04|227|988|77|1918|3|12|4|4|1918|77|988|Wednesday|1918Q4|N|N|N|2421929|2422262|2421567|2421840|N|N|N|N|N| +2421933|AAAAAAAANKEPECAA|1918-12-05|227|988|77|1918|4|12|5|4|1918|77|988|Thursday|1918Q4|N|N|N|2421929|2422262|2421568|2421841|N|N|N|N|N| +2421934|AAAAAAAAOKEPECAA|1918-12-06|227|988|77|1918|5|12|6|4|1918|77|988|Friday|1918Q4|N|Y|N|2421929|2422262|2421569|2421842|N|N|N|N|N| +2421935|AAAAAAAAPKEPECAA|1918-12-07|227|988|77|1918|6|12|7|4|1918|77|988|Saturday|1918Q4|N|Y|N|2421929|2422262|2421570|2421843|N|N|N|N|N| +2421936|AAAAAAAAALEPECAA|1918-12-08|227|988|77|1918|0|12|8|4|1918|77|988|Sunday|1918Q4|N|N|N|2421929|2422262|2421571|2421844|N|N|N|N|N| +2421937|AAAAAAAABLEPECAA|1918-12-09|227|988|77|1918|1|12|9|4|1918|77|988|Monday|1918Q4|N|N|N|2421929|2422262|2421572|2421845|N|N|N|N|N| +2421938|AAAAAAAACLEPECAA|1918-12-10|227|989|77|1918|2|12|10|4|1918|77|989|Tuesday|1918Q4|N|N|N|2421929|2422262|2421573|2421846|N|N|N|N|N| +2421939|AAAAAAAADLEPECAA|1918-12-11|227|989|77|1918|3|12|11|4|1918|77|989|Wednesday|1918Q4|N|N|N|2421929|2422262|2421574|2421847|N|N|N|N|N| +2421940|AAAAAAAAELEPECAA|1918-12-12|227|989|77|1918|4|12|12|4|1918|77|989|Thursday|1918Q4|N|N|N|2421929|2422262|2421575|2421848|N|N|N|N|N| +2421941|AAAAAAAAFLEPECAA|1918-12-13|227|989|77|1918|5|12|13|4|1918|77|989|Friday|1918Q4|N|Y|N|2421929|2422262|2421576|2421849|N|N|N|N|N| +2421942|AAAAAAAAGLEPECAA|1918-12-14|227|989|77|1918|6|12|14|4|1918|77|989|Saturday|1918Q4|N|Y|N|2421929|2422262|2421577|2421850|N|N|N|N|N| +2421943|AAAAAAAAHLEPECAA|1918-12-15|227|989|77|1918|0|12|15|4|1918|77|989|Sunday|1918Q4|N|N|N|2421929|2422262|2421578|2421851|N|N|N|N|N| +2421944|AAAAAAAAILEPECAA|1918-12-16|227|989|77|1918|1|12|16|4|1918|77|989|Monday|1918Q4|N|N|N|2421929|2422262|2421579|2421852|N|N|N|N|N| +2421945|AAAAAAAAJLEPECAA|1918-12-17|227|990|77|1918|2|12|17|4|1918|77|990|Tuesday|1918Q4|N|N|N|2421929|2422262|2421580|2421853|N|N|N|N|N| +2421946|AAAAAAAAKLEPECAA|1918-12-18|227|990|77|1918|3|12|18|4|1918|77|990|Wednesday|1918Q4|N|N|N|2421929|2422262|2421581|2421854|N|N|N|N|N| +2421947|AAAAAAAALLEPECAA|1918-12-19|227|990|77|1918|4|12|19|4|1918|77|990|Thursday|1918Q4|N|N|N|2421929|2422262|2421582|2421855|N|N|N|N|N| +2421948|AAAAAAAAMLEPECAA|1918-12-20|227|990|77|1918|5|12|20|4|1918|77|990|Friday|1918Q4|N|Y|N|2421929|2422262|2421583|2421856|N|N|N|N|N| +2421949|AAAAAAAANLEPECAA|1918-12-21|227|990|77|1918|6|12|21|4|1918|77|990|Saturday|1918Q4|N|Y|N|2421929|2422262|2421584|2421857|N|N|N|N|N| +2421950|AAAAAAAAOLEPECAA|1918-12-22|227|990|77|1918|0|12|22|4|1918|77|990|Sunday|1918Q4|N|N|N|2421929|2422262|2421585|2421858|N|N|N|N|N| +2421951|AAAAAAAAPLEPECAA|1918-12-23|227|990|77|1918|1|12|23|4|1918|77|990|Monday|1918Q4|N|N|N|2421929|2422262|2421586|2421859|N|N|N|N|N| +2421952|AAAAAAAAAMEPECAA|1918-12-24|227|991|77|1918|2|12|24|4|1918|77|991|Tuesday|1918Q4|N|N|N|2421929|2422262|2421587|2421860|N|N|N|N|N| +2421953|AAAAAAAABMEPECAA|1918-12-25|227|991|77|1918|3|12|25|4|1918|77|991|Wednesday|1918Q4|N|N|N|2421929|2422262|2421588|2421861|N|N|N|N|N| +2421954|AAAAAAAACMEPECAA|1918-12-26|227|991|77|1918|4|12|26|4|1918|77|991|Thursday|1918Q4|Y|N|N|2421929|2422262|2421589|2421862|N|N|N|N|N| +2421955|AAAAAAAADMEPECAA|1918-12-27|227|991|77|1918|5|12|27|4|1918|77|991|Friday|1918Q4|N|Y|Y|2421929|2422262|2421590|2421863|N|N|N|N|N| +2421956|AAAAAAAAEMEPECAA|1918-12-28|227|991|77|1918|6|12|28|4|1918|77|991|Saturday|1918Q4|N|Y|N|2421929|2422262|2421591|2421864|N|N|N|N|N| +2421957|AAAAAAAAFMEPECAA|1918-12-29|227|991|77|1918|0|12|29|4|1918|77|991|Sunday|1918Q4|N|N|N|2421929|2422262|2421592|2421865|N|N|N|N|N| +2421958|AAAAAAAAGMEPECAA|1918-12-30|227|991|77|1918|1|12|30|4|1918|77|991|Monday|1918Q4|N|N|N|2421929|2422262|2421593|2421866|N|N|N|N|N| +2421959|AAAAAAAAHMEPECAA|1918-12-31|227|992|77|1918|2|12|31|4|1918|77|992|Tuesday|1918Q4|N|N|N|2421929|2422262|2421594|2421867|N|N|N|N|N| +2421960|AAAAAAAAIMEPECAA|1919-01-01|228|992|77|1919|3|1|1|1|1919|77|992|Wednesday|1919Q1|Y|N|N|2421960|2421959|2421595|2421868|N|N|N|N|N| +2421961|AAAAAAAAJMEPECAA|1919-01-02|228|992|77|1919|4|1|2|1|1919|77|992|Thursday|1919Q1|N|N|Y|2421960|2421959|2421596|2421869|N|N|N|N|N| +2421962|AAAAAAAAKMEPECAA|1919-01-03|228|992|77|1919|5|1|3|1|1919|77|992|Friday|1919Q1|N|Y|N|2421960|2421959|2421597|2421870|N|N|N|N|N| +2421963|AAAAAAAALMEPECAA|1919-01-04|228|992|77|1919|6|1|4|1|1919|77|992|Saturday|1919Q1|N|Y|N|2421960|2421959|2421598|2421871|N|N|N|N|N| +2421964|AAAAAAAAMMEPECAA|1919-01-05|228|992|77|1919|0|1|5|1|1919|77|992|Sunday|1919Q1|N|N|N|2421960|2421959|2421599|2421872|N|N|N|N|N| +2421965|AAAAAAAANMEPECAA|1919-01-06|228|992|77|1919|1|1|6|1|1919|77|992|Monday|1919Q1|N|N|N|2421960|2421959|2421600|2421873|N|N|N|N|N| +2421966|AAAAAAAAOMEPECAA|1919-01-07|228|993|77|1919|2|1|7|1|1919|77|993|Tuesday|1919Q1|N|N|N|2421960|2421959|2421601|2421874|N|N|N|N|N| +2421967|AAAAAAAAPMEPECAA|1919-01-08|228|993|77|1919|3|1|8|1|1919|77|993|Wednesday|1919Q1|N|N|N|2421960|2421959|2421602|2421875|N|N|N|N|N| +2421968|AAAAAAAAANEPECAA|1919-01-09|228|993|77|1919|4|1|9|1|1919|77|993|Thursday|1919Q1|N|N|N|2421960|2421959|2421603|2421876|N|N|N|N|N| +2421969|AAAAAAAABNEPECAA|1919-01-10|228|993|77|1919|5|1|10|1|1919|77|993|Friday|1919Q1|N|Y|N|2421960|2421959|2421604|2421877|N|N|N|N|N| +2421970|AAAAAAAACNEPECAA|1919-01-11|228|993|77|1919|6|1|11|1|1919|77|993|Saturday|1919Q1|N|Y|N|2421960|2421959|2421605|2421878|N|N|N|N|N| +2421971|AAAAAAAADNEPECAA|1919-01-12|228|993|77|1919|0|1|12|1|1919|77|993|Sunday|1919Q1|N|N|N|2421960|2421959|2421606|2421879|N|N|N|N|N| +2421972|AAAAAAAAENEPECAA|1919-01-13|228|993|77|1919|1|1|13|1|1919|77|993|Monday|1919Q1|N|N|N|2421960|2421959|2421607|2421880|N|N|N|N|N| +2421973|AAAAAAAAFNEPECAA|1919-01-14|228|994|77|1919|2|1|14|1|1919|77|994|Tuesday|1919Q1|N|N|N|2421960|2421959|2421608|2421881|N|N|N|N|N| +2421974|AAAAAAAAGNEPECAA|1919-01-15|228|994|77|1919|3|1|15|1|1919|77|994|Wednesday|1919Q1|N|N|N|2421960|2421959|2421609|2421882|N|N|N|N|N| +2421975|AAAAAAAAHNEPECAA|1919-01-16|228|994|77|1919|4|1|16|1|1919|77|994|Thursday|1919Q1|N|N|N|2421960|2421959|2421610|2421883|N|N|N|N|N| +2421976|AAAAAAAAINEPECAA|1919-01-17|228|994|77|1919|5|1|17|1|1919|77|994|Friday|1919Q1|N|Y|N|2421960|2421959|2421611|2421884|N|N|N|N|N| +2421977|AAAAAAAAJNEPECAA|1919-01-18|228|994|77|1919|6|1|18|1|1919|77|994|Saturday|1919Q1|N|Y|N|2421960|2421959|2421612|2421885|N|N|N|N|N| +2421978|AAAAAAAAKNEPECAA|1919-01-19|228|994|77|1919|0|1|19|1|1919|77|994|Sunday|1919Q1|N|N|N|2421960|2421959|2421613|2421886|N|N|N|N|N| +2421979|AAAAAAAALNEPECAA|1919-01-20|228|994|77|1919|1|1|20|1|1919|77|994|Monday|1919Q1|N|N|N|2421960|2421959|2421614|2421887|N|N|N|N|N| +2421980|AAAAAAAAMNEPECAA|1919-01-21|228|995|77|1919|2|1|21|1|1919|77|995|Tuesday|1919Q1|N|N|N|2421960|2421959|2421615|2421888|N|N|N|N|N| +2421981|AAAAAAAANNEPECAA|1919-01-22|228|995|77|1919|3|1|22|1|1919|77|995|Wednesday|1919Q1|N|N|N|2421960|2421959|2421616|2421889|N|N|N|N|N| +2421982|AAAAAAAAONEPECAA|1919-01-23|228|995|77|1919|4|1|23|1|1919|77|995|Thursday|1919Q1|N|N|N|2421960|2421959|2421617|2421890|N|N|N|N|N| +2421983|AAAAAAAAPNEPECAA|1919-01-24|228|995|77|1919|5|1|24|1|1919|77|995|Friday|1919Q1|N|Y|N|2421960|2421959|2421618|2421891|N|N|N|N|N| +2421984|AAAAAAAAAOEPECAA|1919-01-25|228|995|77|1919|6|1|25|1|1919|77|995|Saturday|1919Q1|N|Y|N|2421960|2421959|2421619|2421892|N|N|N|N|N| +2421985|AAAAAAAABOEPECAA|1919-01-26|228|995|77|1919|0|1|26|1|1919|77|995|Sunday|1919Q1|N|N|N|2421960|2421959|2421620|2421893|N|N|N|N|N| +2421986|AAAAAAAACOEPECAA|1919-01-27|228|995|77|1919|1|1|27|1|1919|77|995|Monday|1919Q1|N|N|N|2421960|2421959|2421621|2421894|N|N|N|N|N| +2421987|AAAAAAAADOEPECAA|1919-01-28|228|996|77|1919|2|1|28|1|1919|77|996|Tuesday|1919Q1|N|N|N|2421960|2421959|2421622|2421895|N|N|N|N|N| +2421988|AAAAAAAAEOEPECAA|1919-01-29|228|996|77|1919|3|1|29|1|1919|77|996|Wednesday|1919Q1|N|N|N|2421960|2421959|2421623|2421896|N|N|N|N|N| +2421989|AAAAAAAAFOEPECAA|1919-01-30|228|996|77|1919|4|1|30|1|1919|77|996|Thursday|1919Q1|N|N|N|2421960|2421959|2421624|2421897|N|N|N|N|N| +2421990|AAAAAAAAGOEPECAA|1919-01-31|228|996|77|1919|5|1|31|1|1919|77|996|Friday|1919Q1|N|Y|N|2421960|2421959|2421625|2421898|N|N|N|N|N| +2421991|AAAAAAAAHOEPECAA|1919-02-01|229|996|77|1919|6|2|1|1|1919|77|996|Saturday|1919Q1|N|Y|N|2421991|2422021|2421626|2421899|N|N|N|N|N| +2421992|AAAAAAAAIOEPECAA|1919-02-02|229|996|77|1919|0|2|2|1|1919|77|996|Sunday|1919Q1|N|N|N|2421991|2422021|2421627|2421900|N|N|N|N|N| +2421993|AAAAAAAAJOEPECAA|1919-02-03|229|996|77|1919|1|2|3|1|1919|77|996|Monday|1919Q1|N|N|N|2421991|2422021|2421628|2421901|N|N|N|N|N| +2421994|AAAAAAAAKOEPECAA|1919-02-04|229|997|77|1919|2|2|4|1|1919|77|997|Tuesday|1919Q1|N|N|N|2421991|2422021|2421629|2421902|N|N|N|N|N| +2421995|AAAAAAAALOEPECAA|1919-02-05|229|997|77|1919|3|2|5|1|1919|77|997|Wednesday|1919Q1|N|N|N|2421991|2422021|2421630|2421903|N|N|N|N|N| +2421996|AAAAAAAAMOEPECAA|1919-02-06|229|997|77|1919|4|2|6|1|1919|77|997|Thursday|1919Q1|N|N|N|2421991|2422021|2421631|2421904|N|N|N|N|N| +2421997|AAAAAAAANOEPECAA|1919-02-07|229|997|77|1919|5|2|7|1|1919|77|997|Friday|1919Q1|N|Y|N|2421991|2422021|2421632|2421905|N|N|N|N|N| +2421998|AAAAAAAAOOEPECAA|1919-02-08|229|997|77|1919|6|2|8|1|1919|77|997|Saturday|1919Q1|N|Y|N|2421991|2422021|2421633|2421906|N|N|N|N|N| +2421999|AAAAAAAAPOEPECAA|1919-02-09|229|997|77|1919|0|2|9|1|1919|77|997|Sunday|1919Q1|N|N|N|2421991|2422021|2421634|2421907|N|N|N|N|N| +2422000|AAAAAAAAAPEPECAA|1919-02-10|229|997|77|1919|1|2|10|1|1919|77|997|Monday|1919Q1|N|N|N|2421991|2422021|2421635|2421908|N|N|N|N|N| +2422001|AAAAAAAABPEPECAA|1919-02-11|229|998|77|1919|2|2|11|1|1919|77|998|Tuesday|1919Q1|N|N|N|2421991|2422021|2421636|2421909|N|N|N|N|N| +2422002|AAAAAAAACPEPECAA|1919-02-12|229|998|77|1919|3|2|12|1|1919|77|998|Wednesday|1919Q1|N|N|N|2421991|2422021|2421637|2421910|N|N|N|N|N| +2422003|AAAAAAAADPEPECAA|1919-02-13|229|998|77|1919|4|2|13|1|1919|77|998|Thursday|1919Q1|N|N|N|2421991|2422021|2421638|2421911|N|N|N|N|N| +2422004|AAAAAAAAEPEPECAA|1919-02-14|229|998|77|1919|5|2|14|1|1919|77|998|Friday|1919Q1|N|Y|N|2421991|2422021|2421639|2421912|N|N|N|N|N| +2422005|AAAAAAAAFPEPECAA|1919-02-15|229|998|77|1919|6|2|15|1|1919|77|998|Saturday|1919Q1|N|Y|N|2421991|2422021|2421640|2421913|N|N|N|N|N| +2422006|AAAAAAAAGPEPECAA|1919-02-16|229|998|77|1919|0|2|16|1|1919|77|998|Sunday|1919Q1|N|N|N|2421991|2422021|2421641|2421914|N|N|N|N|N| +2422007|AAAAAAAAHPEPECAA|1919-02-17|229|998|77|1919|1|2|17|1|1919|77|998|Monday|1919Q1|N|N|N|2421991|2422021|2421642|2421915|N|N|N|N|N| +2422008|AAAAAAAAIPEPECAA|1919-02-18|229|999|77|1919|2|2|18|1|1919|77|999|Tuesday|1919Q1|N|N|N|2421991|2422021|2421643|2421916|N|N|N|N|N| +2422009|AAAAAAAAJPEPECAA|1919-02-19|229|999|77|1919|3|2|19|1|1919|77|999|Wednesday|1919Q1|N|N|N|2421991|2422021|2421644|2421917|N|N|N|N|N| +2422010|AAAAAAAAKPEPECAA|1919-02-20|229|999|77|1919|4|2|20|1|1919|77|999|Thursday|1919Q1|N|N|N|2421991|2422021|2421645|2421918|N|N|N|N|N| +2422011|AAAAAAAALPEPECAA|1919-02-21|229|999|77|1919|5|2|21|1|1919|77|999|Friday|1919Q1|N|Y|N|2421991|2422021|2421646|2421919|N|N|N|N|N| +2422012|AAAAAAAAMPEPECAA|1919-02-22|229|999|77|1919|6|2|22|1|1919|77|999|Saturday|1919Q1|N|Y|N|2421991|2422021|2421647|2421920|N|N|N|N|N| +2422013|AAAAAAAANPEPECAA|1919-02-23|229|999|77|1919|0|2|23|1|1919|77|999|Sunday|1919Q1|N|N|N|2421991|2422021|2421648|2421921|N|N|N|N|N| +2422014|AAAAAAAAOPEPECAA|1919-02-24|229|999|77|1919|1|2|24|1|1919|77|999|Monday|1919Q1|N|N|N|2421991|2422021|2421649|2421922|N|N|N|N|N| +2422015|AAAAAAAAPPEPECAA|1919-02-25|229|1000|77|1919|2|2|25|1|1919|77|1000|Tuesday|1919Q1|N|N|N|2421991|2422021|2421650|2421923|N|N|N|N|N| +2422016|AAAAAAAAAAFPECAA|1919-02-26|229|1000|77|1919|3|2|26|1|1919|77|1000|Wednesday|1919Q1|N|N|N|2421991|2422021|2421651|2421924|N|N|N|N|N| +2422017|AAAAAAAABAFPECAA|1919-02-27|229|1000|77|1919|4|2|27|1|1919|77|1000|Thursday|1919Q1|N|N|N|2421991|2422021|2421652|2421925|N|N|N|N|N| +2422018|AAAAAAAACAFPECAA|1919-02-28|229|1000|77|1919|5|2|28|1|1919|77|1000|Friday|1919Q1|N|Y|N|2421991|2422021|2421653|2421926|N|N|N|N|N| +2422019|AAAAAAAADAFPECAA|1919-03-01|230|1000|78|1919|6|3|1|1|1919|78|1000|Saturday|1919Q1|N|Y|N|2422019|2422077|2421654|2421927|N|N|N|N|N| +2422020|AAAAAAAAEAFPECAA|1919-03-02|230|1000|78|1919|0|3|2|1|1919|78|1000|Sunday|1919Q1|N|N|N|2422019|2422077|2421655|2421928|N|N|N|N|N| +2422021|AAAAAAAAFAFPECAA|1919-03-03|230|1000|78|1919|1|3|3|1|1919|78|1000|Monday|1919Q1|N|N|N|2422019|2422077|2421656|2421929|N|N|N|N|N| +2422022|AAAAAAAAGAFPECAA|1919-03-04|230|1001|78|1919|2|3|4|1|1919|78|1001|Tuesday|1919Q1|N|N|N|2422019|2422077|2421657|2421930|N|N|N|N|N| +2422023|AAAAAAAAHAFPECAA|1919-03-05|230|1001|78|1919|3|3|5|1|1919|78|1001|Wednesday|1919Q1|N|N|N|2422019|2422077|2421658|2421931|N|N|N|N|N| +2422024|AAAAAAAAIAFPECAA|1919-03-06|230|1001|78|1919|4|3|6|1|1919|78|1001|Thursday|1919Q1|N|N|N|2422019|2422077|2421659|2421932|N|N|N|N|N| +2422025|AAAAAAAAJAFPECAA|1919-03-07|230|1001|78|1919|5|3|7|1|1919|78|1001|Friday|1919Q1|N|Y|N|2422019|2422077|2421660|2421933|N|N|N|N|N| +2422026|AAAAAAAAKAFPECAA|1919-03-08|230|1001|78|1919|6|3|8|1|1919|78|1001|Saturday|1919Q1|N|Y|N|2422019|2422077|2421661|2421934|N|N|N|N|N| +2422027|AAAAAAAALAFPECAA|1919-03-09|230|1001|78|1919|0|3|9|1|1919|78|1001|Sunday|1919Q1|N|N|N|2422019|2422077|2421662|2421935|N|N|N|N|N| +2422028|AAAAAAAAMAFPECAA|1919-03-10|230|1001|78|1919|1|3|10|1|1919|78|1001|Monday|1919Q1|N|N|N|2422019|2422077|2421663|2421936|N|N|N|N|N| +2422029|AAAAAAAANAFPECAA|1919-03-11|230|1002|78|1919|2|3|11|1|1919|78|1002|Tuesday|1919Q1|N|N|N|2422019|2422077|2421664|2421937|N|N|N|N|N| +2422030|AAAAAAAAOAFPECAA|1919-03-12|230|1002|78|1919|3|3|12|1|1919|78|1002|Wednesday|1919Q1|N|N|N|2422019|2422077|2421665|2421938|N|N|N|N|N| +2422031|AAAAAAAAPAFPECAA|1919-03-13|230|1002|78|1919|4|3|13|1|1919|78|1002|Thursday|1919Q1|N|N|N|2422019|2422077|2421666|2421939|N|N|N|N|N| +2422032|AAAAAAAAABFPECAA|1919-03-14|230|1002|78|1919|5|3|14|1|1919|78|1002|Friday|1919Q1|N|Y|N|2422019|2422077|2421667|2421940|N|N|N|N|N| +2422033|AAAAAAAABBFPECAA|1919-03-15|230|1002|78|1919|6|3|15|1|1919|78|1002|Saturday|1919Q1|N|Y|N|2422019|2422077|2421668|2421941|N|N|N|N|N| +2422034|AAAAAAAACBFPECAA|1919-03-16|230|1002|78|1919|0|3|16|1|1919|78|1002|Sunday|1919Q1|N|N|N|2422019|2422077|2421669|2421942|N|N|N|N|N| +2422035|AAAAAAAADBFPECAA|1919-03-17|230|1002|78|1919|1|3|17|1|1919|78|1002|Monday|1919Q1|N|N|N|2422019|2422077|2421670|2421943|N|N|N|N|N| +2422036|AAAAAAAAEBFPECAA|1919-03-18|230|1003|78|1919|2|3|18|1|1919|78|1003|Tuesday|1919Q1|N|N|N|2422019|2422077|2421671|2421944|N|N|N|N|N| +2422037|AAAAAAAAFBFPECAA|1919-03-19|230|1003|78|1919|3|3|19|1|1919|78|1003|Wednesday|1919Q1|N|N|N|2422019|2422077|2421672|2421945|N|N|N|N|N| +2422038|AAAAAAAAGBFPECAA|1919-03-20|230|1003|78|1919|4|3|20|1|1919|78|1003|Thursday|1919Q1|N|N|N|2422019|2422077|2421673|2421946|N|N|N|N|N| +2422039|AAAAAAAAHBFPECAA|1919-03-21|230|1003|78|1919|5|3|21|1|1919|78|1003|Friday|1919Q1|N|Y|N|2422019|2422077|2421674|2421947|N|N|N|N|N| +2422040|AAAAAAAAIBFPECAA|1919-03-22|230|1003|78|1919|6|3|22|1|1919|78|1003|Saturday|1919Q1|N|Y|N|2422019|2422077|2421675|2421948|N|N|N|N|N| +2422041|AAAAAAAAJBFPECAA|1919-03-23|230|1003|78|1919|0|3|23|1|1919|78|1003|Sunday|1919Q1|N|N|N|2422019|2422077|2421676|2421949|N|N|N|N|N| +2422042|AAAAAAAAKBFPECAA|1919-03-24|230|1003|78|1919|1|3|24|1|1919|78|1003|Monday|1919Q1|N|N|N|2422019|2422077|2421677|2421950|N|N|N|N|N| +2422043|AAAAAAAALBFPECAA|1919-03-25|230|1004|78|1919|2|3|25|1|1919|78|1004|Tuesday|1919Q1|N|N|N|2422019|2422077|2421678|2421951|N|N|N|N|N| +2422044|AAAAAAAAMBFPECAA|1919-03-26|230|1004|78|1919|3|3|26|1|1919|78|1004|Wednesday|1919Q1|N|N|N|2422019|2422077|2421679|2421952|N|N|N|N|N| +2422045|AAAAAAAANBFPECAA|1919-03-27|230|1004|78|1919|4|3|27|1|1919|78|1004|Thursday|1919Q1|N|N|N|2422019|2422077|2421680|2421953|N|N|N|N|N| +2422046|AAAAAAAAOBFPECAA|1919-03-28|230|1004|78|1919|5|3|28|1|1919|78|1004|Friday|1919Q1|N|Y|N|2422019|2422077|2421681|2421954|N|N|N|N|N| +2422047|AAAAAAAAPBFPECAA|1919-03-29|230|1004|78|1919|6|3|29|1|1919|78|1004|Saturday|1919Q1|N|Y|N|2422019|2422077|2421682|2421955|N|N|N|N|N| +2422048|AAAAAAAAACFPECAA|1919-03-30|230|1004|78|1919|0|3|30|1|1919|78|1004|Sunday|1919Q1|N|N|N|2422019|2422077|2421683|2421956|N|N|N|N|N| +2422049|AAAAAAAABCFPECAA|1919-03-31|230|1004|78|1919|1|3|31|1|1919|78|1004|Monday|1919Q1|N|N|N|2422019|2422077|2421684|2421957|N|N|N|N|N| +2422050|AAAAAAAACCFPECAA|1919-04-01|231|1005|78|1919|2|4|1|1|1919|78|1005|Tuesday|1919Q1|N|N|N|2422050|2422139|2421685|2421960|N|N|N|N|N| +2422051|AAAAAAAADCFPECAA|1919-04-02|231|1005|78|1919|3|4|2|2|1919|78|1005|Wednesday|1919Q2|N|N|N|2422050|2422139|2421686|2421961|N|N|N|N|N| +2422052|AAAAAAAAECFPECAA|1919-04-03|231|1005|78|1919|4|4|3|2|1919|78|1005|Thursday|1919Q2|N|N|N|2422050|2422139|2421687|2421962|N|N|N|N|N| +2422053|AAAAAAAAFCFPECAA|1919-04-04|231|1005|78|1919|5|4|4|2|1919|78|1005|Friday|1919Q2|N|Y|N|2422050|2422139|2421688|2421963|N|N|N|N|N| +2422054|AAAAAAAAGCFPECAA|1919-04-05|231|1005|78|1919|6|4|5|2|1919|78|1005|Saturday|1919Q2|N|Y|N|2422050|2422139|2421689|2421964|N|N|N|N|N| +2422055|AAAAAAAAHCFPECAA|1919-04-06|231|1005|78|1919|0|4|6|2|1919|78|1005|Sunday|1919Q2|N|N|N|2422050|2422139|2421690|2421965|N|N|N|N|N| +2422056|AAAAAAAAICFPECAA|1919-04-07|231|1005|78|1919|1|4|7|2|1919|78|1005|Monday|1919Q2|N|N|N|2422050|2422139|2421691|2421966|N|N|N|N|N| +2422057|AAAAAAAAJCFPECAA|1919-04-08|231|1006|78|1919|2|4|8|2|1919|78|1006|Tuesday|1919Q2|N|N|N|2422050|2422139|2421692|2421967|N|N|N|N|N| +2422058|AAAAAAAAKCFPECAA|1919-04-09|231|1006|78|1919|3|4|9|2|1919|78|1006|Wednesday|1919Q2|N|N|N|2422050|2422139|2421693|2421968|N|N|N|N|N| +2422059|AAAAAAAALCFPECAA|1919-04-10|231|1006|78|1919|4|4|10|2|1919|78|1006|Thursday|1919Q2|N|N|N|2422050|2422139|2421694|2421969|N|N|N|N|N| +2422060|AAAAAAAAMCFPECAA|1919-04-11|231|1006|78|1919|5|4|11|2|1919|78|1006|Friday|1919Q2|N|Y|N|2422050|2422139|2421695|2421970|N|N|N|N|N| +2422061|AAAAAAAANCFPECAA|1919-04-12|231|1006|78|1919|6|4|12|2|1919|78|1006|Saturday|1919Q2|N|Y|N|2422050|2422139|2421696|2421971|N|N|N|N|N| +2422062|AAAAAAAAOCFPECAA|1919-04-13|231|1006|78|1919|0|4|13|2|1919|78|1006|Sunday|1919Q2|N|N|N|2422050|2422139|2421697|2421972|N|N|N|N|N| +2422063|AAAAAAAAPCFPECAA|1919-04-14|231|1006|78|1919|1|4|14|2|1919|78|1006|Monday|1919Q2|N|N|N|2422050|2422139|2421698|2421973|N|N|N|N|N| +2422064|AAAAAAAAADFPECAA|1919-04-15|231|1007|78|1919|2|4|15|2|1919|78|1007|Tuesday|1919Q2|N|N|N|2422050|2422139|2421699|2421974|N|N|N|N|N| +2422065|AAAAAAAABDFPECAA|1919-04-16|231|1007|78|1919|3|4|16|2|1919|78|1007|Wednesday|1919Q2|N|N|N|2422050|2422139|2421700|2421975|N|N|N|N|N| +2422066|AAAAAAAACDFPECAA|1919-04-17|231|1007|78|1919|4|4|17|2|1919|78|1007|Thursday|1919Q2|N|N|N|2422050|2422139|2421701|2421976|N|N|N|N|N| +2422067|AAAAAAAADDFPECAA|1919-04-18|231|1007|78|1919|5|4|18|2|1919|78|1007|Friday|1919Q2|N|Y|N|2422050|2422139|2421702|2421977|N|N|N|N|N| +2422068|AAAAAAAAEDFPECAA|1919-04-19|231|1007|78|1919|6|4|19|2|1919|78|1007|Saturday|1919Q2|N|Y|N|2422050|2422139|2421703|2421978|N|N|N|N|N| +2422069|AAAAAAAAFDFPECAA|1919-04-20|231|1007|78|1919|0|4|20|2|1919|78|1007|Sunday|1919Q2|N|N|N|2422050|2422139|2421704|2421979|N|N|N|N|N| +2422070|AAAAAAAAGDFPECAA|1919-04-21|231|1007|78|1919|1|4|21|2|1919|78|1007|Monday|1919Q2|N|N|N|2422050|2422139|2421705|2421980|N|N|N|N|N| +2422071|AAAAAAAAHDFPECAA|1919-04-22|231|1008|78|1919|2|4|22|2|1919|78|1008|Tuesday|1919Q2|N|N|N|2422050|2422139|2421706|2421981|N|N|N|N|N| +2422072|AAAAAAAAIDFPECAA|1919-04-23|231|1008|78|1919|3|4|23|2|1919|78|1008|Wednesday|1919Q2|N|N|N|2422050|2422139|2421707|2421982|N|N|N|N|N| +2422073|AAAAAAAAJDFPECAA|1919-04-24|231|1008|78|1919|4|4|24|2|1919|78|1008|Thursday|1919Q2|N|N|N|2422050|2422139|2421708|2421983|N|N|N|N|N| +2422074|AAAAAAAAKDFPECAA|1919-04-25|231|1008|78|1919|5|4|25|2|1919|78|1008|Friday|1919Q2|N|Y|N|2422050|2422139|2421709|2421984|N|N|N|N|N| +2422075|AAAAAAAALDFPECAA|1919-04-26|231|1008|78|1919|6|4|26|2|1919|78|1008|Saturday|1919Q2|N|Y|N|2422050|2422139|2421710|2421985|N|N|N|N|N| +2422076|AAAAAAAAMDFPECAA|1919-04-27|231|1008|78|1919|0|4|27|2|1919|78|1008|Sunday|1919Q2|N|N|N|2422050|2422139|2421711|2421986|N|N|N|N|N| +2422077|AAAAAAAANDFPECAA|1919-04-28|231|1008|78|1919|1|4|28|2|1919|78|1008|Monday|1919Q2|N|N|N|2422050|2422139|2421712|2421987|N|N|N|N|N| +2422078|AAAAAAAAODFPECAA|1919-04-29|231|1009|78|1919|2|4|29|2|1919|78|1009|Tuesday|1919Q2|N|N|N|2422050|2422139|2421713|2421988|N|N|N|N|N| +2422079|AAAAAAAAPDFPECAA|1919-04-30|231|1009|78|1919|3|4|30|2|1919|78|1009|Wednesday|1919Q2|N|N|N|2422050|2422139|2421714|2421989|N|N|N|N|N| +2422080|AAAAAAAAAEFPECAA|1919-05-01|232|1009|78|1919|4|5|1|2|1919|78|1009|Thursday|1919Q2|N|N|N|2422080|2422199|2421715|2421990|N|N|N|N|N| +2422081|AAAAAAAABEFPECAA|1919-05-02|232|1009|78|1919|5|5|2|2|1919|78|1009|Friday|1919Q2|N|Y|N|2422080|2422199|2421716|2421991|N|N|N|N|N| +2422082|AAAAAAAACEFPECAA|1919-05-03|232|1009|78|1919|6|5|3|2|1919|78|1009|Saturday|1919Q2|N|Y|N|2422080|2422199|2421717|2421992|N|N|N|N|N| +2422083|AAAAAAAADEFPECAA|1919-05-04|232|1009|78|1919|0|5|4|2|1919|78|1009|Sunday|1919Q2|N|N|N|2422080|2422199|2421718|2421993|N|N|N|N|N| +2422084|AAAAAAAAEEFPECAA|1919-05-05|232|1009|78|1919|1|5|5|2|1919|78|1009|Monday|1919Q2|N|N|N|2422080|2422199|2421719|2421994|N|N|N|N|N| +2422085|AAAAAAAAFEFPECAA|1919-05-06|232|1010|78|1919|2|5|6|2|1919|78|1010|Tuesday|1919Q2|N|N|N|2422080|2422199|2421720|2421995|N|N|N|N|N| +2422086|AAAAAAAAGEFPECAA|1919-05-07|232|1010|78|1919|3|5|7|2|1919|78|1010|Wednesday|1919Q2|N|N|N|2422080|2422199|2421721|2421996|N|N|N|N|N| +2422087|AAAAAAAAHEFPECAA|1919-05-08|232|1010|78|1919|4|5|8|2|1919|78|1010|Thursday|1919Q2|N|N|N|2422080|2422199|2421722|2421997|N|N|N|N|N| +2422088|AAAAAAAAIEFPECAA|1919-05-09|232|1010|78|1919|5|5|9|2|1919|78|1010|Friday|1919Q2|N|Y|N|2422080|2422199|2421723|2421998|N|N|N|N|N| +2422089|AAAAAAAAJEFPECAA|1919-05-10|232|1010|78|1919|6|5|10|2|1919|78|1010|Saturday|1919Q2|N|Y|N|2422080|2422199|2421724|2421999|N|N|N|N|N| +2422090|AAAAAAAAKEFPECAA|1919-05-11|232|1010|78|1919|0|5|11|2|1919|78|1010|Sunday|1919Q2|N|N|N|2422080|2422199|2421725|2422000|N|N|N|N|N| +2422091|AAAAAAAALEFPECAA|1919-05-12|232|1010|78|1919|1|5|12|2|1919|78|1010|Monday|1919Q2|N|N|N|2422080|2422199|2421726|2422001|N|N|N|N|N| +2422092|AAAAAAAAMEFPECAA|1919-05-13|232|1011|78|1919|2|5|13|2|1919|78|1011|Tuesday|1919Q2|N|N|N|2422080|2422199|2421727|2422002|N|N|N|N|N| +2422093|AAAAAAAANEFPECAA|1919-05-14|232|1011|78|1919|3|5|14|2|1919|78|1011|Wednesday|1919Q2|N|N|N|2422080|2422199|2421728|2422003|N|N|N|N|N| +2422094|AAAAAAAAOEFPECAA|1919-05-15|232|1011|78|1919|4|5|15|2|1919|78|1011|Thursday|1919Q2|N|N|N|2422080|2422199|2421729|2422004|N|N|N|N|N| +2422095|AAAAAAAAPEFPECAA|1919-05-16|232|1011|78|1919|5|5|16|2|1919|78|1011|Friday|1919Q2|N|Y|N|2422080|2422199|2421730|2422005|N|N|N|N|N| +2422096|AAAAAAAAAFFPECAA|1919-05-17|232|1011|78|1919|6|5|17|2|1919|78|1011|Saturday|1919Q2|N|Y|N|2422080|2422199|2421731|2422006|N|N|N|N|N| +2422097|AAAAAAAABFFPECAA|1919-05-18|232|1011|78|1919|0|5|18|2|1919|78|1011|Sunday|1919Q2|N|N|N|2422080|2422199|2421732|2422007|N|N|N|N|N| +2422098|AAAAAAAACFFPECAA|1919-05-19|232|1011|78|1919|1|5|19|2|1919|78|1011|Monday|1919Q2|N|N|N|2422080|2422199|2421733|2422008|N|N|N|N|N| +2422099|AAAAAAAADFFPECAA|1919-05-20|232|1012|78|1919|2|5|20|2|1919|78|1012|Tuesday|1919Q2|N|N|N|2422080|2422199|2421734|2422009|N|N|N|N|N| +2422100|AAAAAAAAEFFPECAA|1919-05-21|232|1012|78|1919|3|5|21|2|1919|78|1012|Wednesday|1919Q2|N|N|N|2422080|2422199|2421735|2422010|N|N|N|N|N| +2422101|AAAAAAAAFFFPECAA|1919-05-22|232|1012|78|1919|4|5|22|2|1919|78|1012|Thursday|1919Q2|N|N|N|2422080|2422199|2421736|2422011|N|N|N|N|N| +2422102|AAAAAAAAGFFPECAA|1919-05-23|232|1012|78|1919|5|5|23|2|1919|78|1012|Friday|1919Q2|N|Y|N|2422080|2422199|2421737|2422012|N|N|N|N|N| +2422103|AAAAAAAAHFFPECAA|1919-05-24|232|1012|78|1919|6|5|24|2|1919|78|1012|Saturday|1919Q2|N|Y|N|2422080|2422199|2421738|2422013|N|N|N|N|N| +2422104|AAAAAAAAIFFPECAA|1919-05-25|232|1012|78|1919|0|5|25|2|1919|78|1012|Sunday|1919Q2|N|N|N|2422080|2422199|2421739|2422014|N|N|N|N|N| +2422105|AAAAAAAAJFFPECAA|1919-05-26|232|1012|78|1919|1|5|26|2|1919|78|1012|Monday|1919Q2|N|N|N|2422080|2422199|2421740|2422015|N|N|N|N|N| +2422106|AAAAAAAAKFFPECAA|1919-05-27|232|1013|78|1919|2|5|27|2|1919|78|1013|Tuesday|1919Q2|N|N|N|2422080|2422199|2421741|2422016|N|N|N|N|N| +2422107|AAAAAAAALFFPECAA|1919-05-28|232|1013|78|1919|3|5|28|2|1919|78|1013|Wednesday|1919Q2|N|N|N|2422080|2422199|2421742|2422017|N|N|N|N|N| +2422108|AAAAAAAAMFFPECAA|1919-05-29|232|1013|78|1919|4|5|29|2|1919|78|1013|Thursday|1919Q2|N|N|N|2422080|2422199|2421743|2422018|N|N|N|N|N| +2422109|AAAAAAAANFFPECAA|1919-05-30|232|1013|78|1919|5|5|30|2|1919|78|1013|Friday|1919Q2|N|Y|N|2422080|2422199|2421744|2422019|N|N|N|N|N| +2422110|AAAAAAAAOFFPECAA|1919-05-31|232|1013|78|1919|6|5|31|2|1919|78|1013|Saturday|1919Q2|N|Y|N|2422080|2422199|2421745|2422020|N|N|N|N|N| +2422111|AAAAAAAAPFFPECAA|1919-06-01|233|1013|79|1919|0|6|1|2|1919|79|1013|Sunday|1919Q2|N|N|N|2422111|2422261|2421746|2422021|N|N|N|N|N| +2422112|AAAAAAAAAGFPECAA|1919-06-02|233|1013|79|1919|1|6|2|2|1919|79|1013|Monday|1919Q2|N|N|N|2422111|2422261|2421747|2422022|N|N|N|N|N| +2422113|AAAAAAAABGFPECAA|1919-06-03|233|1014|79|1919|2|6|3|2|1919|79|1014|Tuesday|1919Q2|N|N|N|2422111|2422261|2421748|2422023|N|N|N|N|N| +2422114|AAAAAAAACGFPECAA|1919-06-04|233|1014|79|1919|3|6|4|2|1919|79|1014|Wednesday|1919Q2|N|N|N|2422111|2422261|2421749|2422024|N|N|N|N|N| +2422115|AAAAAAAADGFPECAA|1919-06-05|233|1014|79|1919|4|6|5|2|1919|79|1014|Thursday|1919Q2|N|N|N|2422111|2422261|2421750|2422025|N|N|N|N|N| +2422116|AAAAAAAAEGFPECAA|1919-06-06|233|1014|79|1919|5|6|6|2|1919|79|1014|Friday|1919Q2|N|Y|N|2422111|2422261|2421751|2422026|N|N|N|N|N| +2422117|AAAAAAAAFGFPECAA|1919-06-07|233|1014|79|1919|6|6|7|2|1919|79|1014|Saturday|1919Q2|N|Y|N|2422111|2422261|2421752|2422027|N|N|N|N|N| +2422118|AAAAAAAAGGFPECAA|1919-06-08|233|1014|79|1919|0|6|8|2|1919|79|1014|Sunday|1919Q2|N|N|N|2422111|2422261|2421753|2422028|N|N|N|N|N| +2422119|AAAAAAAAHGFPECAA|1919-06-09|233|1014|79|1919|1|6|9|2|1919|79|1014|Monday|1919Q2|N|N|N|2422111|2422261|2421754|2422029|N|N|N|N|N| +2422120|AAAAAAAAIGFPECAA|1919-06-10|233|1015|79|1919|2|6|10|2|1919|79|1015|Tuesday|1919Q2|N|N|N|2422111|2422261|2421755|2422030|N|N|N|N|N| +2422121|AAAAAAAAJGFPECAA|1919-06-11|233|1015|79|1919|3|6|11|2|1919|79|1015|Wednesday|1919Q2|N|N|N|2422111|2422261|2421756|2422031|N|N|N|N|N| +2422122|AAAAAAAAKGFPECAA|1919-06-12|233|1015|79|1919|4|6|12|2|1919|79|1015|Thursday|1919Q2|N|N|N|2422111|2422261|2421757|2422032|N|N|N|N|N| +2422123|AAAAAAAALGFPECAA|1919-06-13|233|1015|79|1919|5|6|13|2|1919|79|1015|Friday|1919Q2|N|Y|N|2422111|2422261|2421758|2422033|N|N|N|N|N| +2422124|AAAAAAAAMGFPECAA|1919-06-14|233|1015|79|1919|6|6|14|2|1919|79|1015|Saturday|1919Q2|N|Y|N|2422111|2422261|2421759|2422034|N|N|N|N|N| +2422125|AAAAAAAANGFPECAA|1919-06-15|233|1015|79|1919|0|6|15|2|1919|79|1015|Sunday|1919Q2|N|N|N|2422111|2422261|2421760|2422035|N|N|N|N|N| +2422126|AAAAAAAAOGFPECAA|1919-06-16|233|1015|79|1919|1|6|16|2|1919|79|1015|Monday|1919Q2|N|N|N|2422111|2422261|2421761|2422036|N|N|N|N|N| +2422127|AAAAAAAAPGFPECAA|1919-06-17|233|1016|79|1919|2|6|17|2|1919|79|1016|Tuesday|1919Q2|N|N|N|2422111|2422261|2421762|2422037|N|N|N|N|N| +2422128|AAAAAAAAAHFPECAA|1919-06-18|233|1016|79|1919|3|6|18|2|1919|79|1016|Wednesday|1919Q2|N|N|N|2422111|2422261|2421763|2422038|N|N|N|N|N| +2422129|AAAAAAAABHFPECAA|1919-06-19|233|1016|79|1919|4|6|19|2|1919|79|1016|Thursday|1919Q2|N|N|N|2422111|2422261|2421764|2422039|N|N|N|N|N| +2422130|AAAAAAAACHFPECAA|1919-06-20|233|1016|79|1919|5|6|20|2|1919|79|1016|Friday|1919Q2|N|Y|N|2422111|2422261|2421765|2422040|N|N|N|N|N| +2422131|AAAAAAAADHFPECAA|1919-06-21|233|1016|79|1919|6|6|21|2|1919|79|1016|Saturday|1919Q2|N|Y|N|2422111|2422261|2421766|2422041|N|N|N|N|N| +2422132|AAAAAAAAEHFPECAA|1919-06-22|233|1016|79|1919|0|6|22|2|1919|79|1016|Sunday|1919Q2|N|N|N|2422111|2422261|2421767|2422042|N|N|N|N|N| +2422133|AAAAAAAAFHFPECAA|1919-06-23|233|1016|79|1919|1|6|23|2|1919|79|1016|Monday|1919Q2|N|N|N|2422111|2422261|2421768|2422043|N|N|N|N|N| +2422134|AAAAAAAAGHFPECAA|1919-06-24|233|1017|79|1919|2|6|24|2|1919|79|1017|Tuesday|1919Q2|N|N|N|2422111|2422261|2421769|2422044|N|N|N|N|N| +2422135|AAAAAAAAHHFPECAA|1919-06-25|233|1017|79|1919|3|6|25|2|1919|79|1017|Wednesday|1919Q2|N|N|N|2422111|2422261|2421770|2422045|N|N|N|N|N| +2422136|AAAAAAAAIHFPECAA|1919-06-26|233|1017|79|1919|4|6|26|2|1919|79|1017|Thursday|1919Q2|N|N|N|2422111|2422261|2421771|2422046|N|N|N|N|N| +2422137|AAAAAAAAJHFPECAA|1919-06-27|233|1017|79|1919|5|6|27|2|1919|79|1017|Friday|1919Q2|N|Y|N|2422111|2422261|2421772|2422047|N|N|N|N|N| +2422138|AAAAAAAAKHFPECAA|1919-06-28|233|1017|79|1919|6|6|28|2|1919|79|1017|Saturday|1919Q2|N|Y|N|2422111|2422261|2421773|2422048|N|N|N|N|N| +2422139|AAAAAAAALHFPECAA|1919-06-29|233|1017|79|1919|0|6|29|2|1919|79|1017|Sunday|1919Q2|N|N|N|2422111|2422261|2421774|2422049|N|N|N|N|N| +2422140|AAAAAAAAMHFPECAA|1919-06-30|233|1017|79|1919|1|6|30|2|1919|79|1017|Monday|1919Q2|N|N|N|2422111|2422261|2421775|2422050|N|N|N|N|N| +2422141|AAAAAAAANHFPECAA|1919-07-01|234|1018|79|1919|2|7|1|2|1919|79|1018|Tuesday|1919Q2|N|N|N|2422141|2422321|2421776|2422050|N|N|N|N|N| +2422142|AAAAAAAAOHFPECAA|1919-07-02|234|1018|79|1919|3|7|2|3|1919|79|1018|Wednesday|1919Q3|N|N|N|2422141|2422321|2421777|2422051|N|N|N|N|N| +2422143|AAAAAAAAPHFPECAA|1919-07-03|234|1018|79|1919|4|7|3|3|1919|79|1018|Thursday|1919Q3|N|N|N|2422141|2422321|2421778|2422052|N|N|N|N|N| +2422144|AAAAAAAAAIFPECAA|1919-07-04|234|1018|79|1919|5|7|4|3|1919|79|1018|Friday|1919Q3|N|Y|N|2422141|2422321|2421779|2422053|N|N|N|N|N| +2422145|AAAAAAAABIFPECAA|1919-07-05|234|1018|79|1919|6|7|5|3|1919|79|1018|Saturday|1919Q3|Y|Y|N|2422141|2422321|2421780|2422054|N|N|N|N|N| +2422146|AAAAAAAACIFPECAA|1919-07-06|234|1018|79|1919|0|7|6|3|1919|79|1018|Sunday|1919Q3|N|N|Y|2422141|2422321|2421781|2422055|N|N|N|N|N| +2422147|AAAAAAAADIFPECAA|1919-07-07|234|1018|79|1919|1|7|7|3|1919|79|1018|Monday|1919Q3|N|N|N|2422141|2422321|2421782|2422056|N|N|N|N|N| +2422148|AAAAAAAAEIFPECAA|1919-07-08|234|1019|79|1919|2|7|8|3|1919|79|1019|Tuesday|1919Q3|N|N|N|2422141|2422321|2421783|2422057|N|N|N|N|N| +2422149|AAAAAAAAFIFPECAA|1919-07-09|234|1019|79|1919|3|7|9|3|1919|79|1019|Wednesday|1919Q3|N|N|N|2422141|2422321|2421784|2422058|N|N|N|N|N| +2422150|AAAAAAAAGIFPECAA|1919-07-10|234|1019|79|1919|4|7|10|3|1919|79|1019|Thursday|1919Q3|N|N|N|2422141|2422321|2421785|2422059|N|N|N|N|N| +2422151|AAAAAAAAHIFPECAA|1919-07-11|234|1019|79|1919|5|7|11|3|1919|79|1019|Friday|1919Q3|N|Y|N|2422141|2422321|2421786|2422060|N|N|N|N|N| +2422152|AAAAAAAAIIFPECAA|1919-07-12|234|1019|79|1919|6|7|12|3|1919|79|1019|Saturday|1919Q3|N|Y|N|2422141|2422321|2421787|2422061|N|N|N|N|N| +2422153|AAAAAAAAJIFPECAA|1919-07-13|234|1019|79|1919|0|7|13|3|1919|79|1019|Sunday|1919Q3|N|N|N|2422141|2422321|2421788|2422062|N|N|N|N|N| +2422154|AAAAAAAAKIFPECAA|1919-07-14|234|1019|79|1919|1|7|14|3|1919|79|1019|Monday|1919Q3|N|N|N|2422141|2422321|2421789|2422063|N|N|N|N|N| +2422155|AAAAAAAALIFPECAA|1919-07-15|234|1020|79|1919|2|7|15|3|1919|79|1020|Tuesday|1919Q3|N|N|N|2422141|2422321|2421790|2422064|N|N|N|N|N| +2422156|AAAAAAAAMIFPECAA|1919-07-16|234|1020|79|1919|3|7|16|3|1919|79|1020|Wednesday|1919Q3|N|N|N|2422141|2422321|2421791|2422065|N|N|N|N|N| +2422157|AAAAAAAANIFPECAA|1919-07-17|234|1020|79|1919|4|7|17|3|1919|79|1020|Thursday|1919Q3|N|N|N|2422141|2422321|2421792|2422066|N|N|N|N|N| +2422158|AAAAAAAAOIFPECAA|1919-07-18|234|1020|79|1919|5|7|18|3|1919|79|1020|Friday|1919Q3|N|Y|N|2422141|2422321|2421793|2422067|N|N|N|N|N| +2422159|AAAAAAAAPIFPECAA|1919-07-19|234|1020|79|1919|6|7|19|3|1919|79|1020|Saturday|1919Q3|N|Y|N|2422141|2422321|2421794|2422068|N|N|N|N|N| +2422160|AAAAAAAAAJFPECAA|1919-07-20|234|1020|79|1919|0|7|20|3|1919|79|1020|Sunday|1919Q3|N|N|N|2422141|2422321|2421795|2422069|N|N|N|N|N| +2422161|AAAAAAAABJFPECAA|1919-07-21|234|1020|79|1919|1|7|21|3|1919|79|1020|Monday|1919Q3|N|N|N|2422141|2422321|2421796|2422070|N|N|N|N|N| +2422162|AAAAAAAACJFPECAA|1919-07-22|234|1021|79|1919|2|7|22|3|1919|79|1021|Tuesday|1919Q3|N|N|N|2422141|2422321|2421797|2422071|N|N|N|N|N| +2422163|AAAAAAAADJFPECAA|1919-07-23|234|1021|79|1919|3|7|23|3|1919|79|1021|Wednesday|1919Q3|N|N|N|2422141|2422321|2421798|2422072|N|N|N|N|N| +2422164|AAAAAAAAEJFPECAA|1919-07-24|234|1021|79|1919|4|7|24|3|1919|79|1021|Thursday|1919Q3|N|N|N|2422141|2422321|2421799|2422073|N|N|N|N|N| +2422165|AAAAAAAAFJFPECAA|1919-07-25|234|1021|79|1919|5|7|25|3|1919|79|1021|Friday|1919Q3|N|Y|N|2422141|2422321|2421800|2422074|N|N|N|N|N| +2422166|AAAAAAAAGJFPECAA|1919-07-26|234|1021|79|1919|6|7|26|3|1919|79|1021|Saturday|1919Q3|N|Y|N|2422141|2422321|2421801|2422075|N|N|N|N|N| +2422167|AAAAAAAAHJFPECAA|1919-07-27|234|1021|79|1919|0|7|27|3|1919|79|1021|Sunday|1919Q3|N|N|N|2422141|2422321|2421802|2422076|N|N|N|N|N| +2422168|AAAAAAAAIJFPECAA|1919-07-28|234|1021|79|1919|1|7|28|3|1919|79|1021|Monday|1919Q3|N|N|N|2422141|2422321|2421803|2422077|N|N|N|N|N| +2422169|AAAAAAAAJJFPECAA|1919-07-29|234|1022|79|1919|2|7|29|3|1919|79|1022|Tuesday|1919Q3|N|N|N|2422141|2422321|2421804|2422078|N|N|N|N|N| +2422170|AAAAAAAAKJFPECAA|1919-07-30|234|1022|79|1919|3|7|30|3|1919|79|1022|Wednesday|1919Q3|N|N|N|2422141|2422321|2421805|2422079|N|N|N|N|N| +2422171|AAAAAAAALJFPECAA|1919-07-31|234|1022|79|1919|4|7|31|3|1919|79|1022|Thursday|1919Q3|N|N|N|2422141|2422321|2421806|2422080|N|N|N|N|N| +2422172|AAAAAAAAMJFPECAA|1919-08-01|235|1022|79|1919|5|8|1|3|1919|79|1022|Friday|1919Q3|N|Y|N|2422172|2422383|2421807|2422081|N|N|N|N|N| +2422173|AAAAAAAANJFPECAA|1919-08-02|235|1022|79|1919|6|8|2|3|1919|79|1022|Saturday|1919Q3|N|Y|N|2422172|2422383|2421808|2422082|N|N|N|N|N| +2422174|AAAAAAAAOJFPECAA|1919-08-03|235|1022|79|1919|0|8|3|3|1919|79|1022|Sunday|1919Q3|N|N|N|2422172|2422383|2421809|2422083|N|N|N|N|N| +2422175|AAAAAAAAPJFPECAA|1919-08-04|235|1022|79|1919|1|8|4|3|1919|79|1022|Monday|1919Q3|N|N|N|2422172|2422383|2421810|2422084|N|N|N|N|N| +2422176|AAAAAAAAAKFPECAA|1919-08-05|235|1023|79|1919|2|8|5|3|1919|79|1023|Tuesday|1919Q3|N|N|N|2422172|2422383|2421811|2422085|N|N|N|N|N| +2422177|AAAAAAAABKFPECAA|1919-08-06|235|1023|79|1919|3|8|6|3|1919|79|1023|Wednesday|1919Q3|N|N|N|2422172|2422383|2421812|2422086|N|N|N|N|N| +2422178|AAAAAAAACKFPECAA|1919-08-07|235|1023|79|1919|4|8|7|3|1919|79|1023|Thursday|1919Q3|N|N|N|2422172|2422383|2421813|2422087|N|N|N|N|N| +2422179|AAAAAAAADKFPECAA|1919-08-08|235|1023|79|1919|5|8|8|3|1919|79|1023|Friday|1919Q3|N|Y|N|2422172|2422383|2421814|2422088|N|N|N|N|N| +2422180|AAAAAAAAEKFPECAA|1919-08-09|235|1023|79|1919|6|8|9|3|1919|79|1023|Saturday|1919Q3|N|Y|N|2422172|2422383|2421815|2422089|N|N|N|N|N| +2422181|AAAAAAAAFKFPECAA|1919-08-10|235|1023|79|1919|0|8|10|3|1919|79|1023|Sunday|1919Q3|N|N|N|2422172|2422383|2421816|2422090|N|N|N|N|N| +2422182|AAAAAAAAGKFPECAA|1919-08-11|235|1023|79|1919|1|8|11|3|1919|79|1023|Monday|1919Q3|N|N|N|2422172|2422383|2421817|2422091|N|N|N|N|N| +2422183|AAAAAAAAHKFPECAA|1919-08-12|235|1024|79|1919|2|8|12|3|1919|79|1024|Tuesday|1919Q3|N|N|N|2422172|2422383|2421818|2422092|N|N|N|N|N| +2422184|AAAAAAAAIKFPECAA|1919-08-13|235|1024|79|1919|3|8|13|3|1919|79|1024|Wednesday|1919Q3|N|N|N|2422172|2422383|2421819|2422093|N|N|N|N|N| +2422185|AAAAAAAAJKFPECAA|1919-08-14|235|1024|79|1919|4|8|14|3|1919|79|1024|Thursday|1919Q3|N|N|N|2422172|2422383|2421820|2422094|N|N|N|N|N| +2422186|AAAAAAAAKKFPECAA|1919-08-15|235|1024|79|1919|5|8|15|3|1919|79|1024|Friday|1919Q3|N|Y|N|2422172|2422383|2421821|2422095|N|N|N|N|N| +2422187|AAAAAAAALKFPECAA|1919-08-16|235|1024|79|1919|6|8|16|3|1919|79|1024|Saturday|1919Q3|N|Y|N|2422172|2422383|2421822|2422096|N|N|N|N|N| +2422188|AAAAAAAAMKFPECAA|1919-08-17|235|1024|79|1919|0|8|17|3|1919|79|1024|Sunday|1919Q3|N|N|N|2422172|2422383|2421823|2422097|N|N|N|N|N| +2422189|AAAAAAAANKFPECAA|1919-08-18|235|1024|79|1919|1|8|18|3|1919|79|1024|Monday|1919Q3|N|N|N|2422172|2422383|2421824|2422098|N|N|N|N|N| +2422190|AAAAAAAAOKFPECAA|1919-08-19|235|1025|79|1919|2|8|19|3|1919|79|1025|Tuesday|1919Q3|N|N|N|2422172|2422383|2421825|2422099|N|N|N|N|N| +2422191|AAAAAAAAPKFPECAA|1919-08-20|235|1025|79|1919|3|8|20|3|1919|79|1025|Wednesday|1919Q3|N|N|N|2422172|2422383|2421826|2422100|N|N|N|N|N| +2422192|AAAAAAAAALFPECAA|1919-08-21|235|1025|79|1919|4|8|21|3|1919|79|1025|Thursday|1919Q3|N|N|N|2422172|2422383|2421827|2422101|N|N|N|N|N| +2422193|AAAAAAAABLFPECAA|1919-08-22|235|1025|79|1919|5|8|22|3|1919|79|1025|Friday|1919Q3|N|Y|N|2422172|2422383|2421828|2422102|N|N|N|N|N| +2422194|AAAAAAAACLFPECAA|1919-08-23|235|1025|79|1919|6|8|23|3|1919|79|1025|Saturday|1919Q3|N|Y|N|2422172|2422383|2421829|2422103|N|N|N|N|N| +2422195|AAAAAAAADLFPECAA|1919-08-24|235|1025|79|1919|0|8|24|3|1919|79|1025|Sunday|1919Q3|N|N|N|2422172|2422383|2421830|2422104|N|N|N|N|N| +2422196|AAAAAAAAELFPECAA|1919-08-25|235|1025|79|1919|1|8|25|3|1919|79|1025|Monday|1919Q3|N|N|N|2422172|2422383|2421831|2422105|N|N|N|N|N| +2422197|AAAAAAAAFLFPECAA|1919-08-26|235|1026|79|1919|2|8|26|3|1919|79|1026|Tuesday|1919Q3|N|N|N|2422172|2422383|2421832|2422106|N|N|N|N|N| +2422198|AAAAAAAAGLFPECAA|1919-08-27|235|1026|79|1919|3|8|27|3|1919|79|1026|Wednesday|1919Q3|N|N|N|2422172|2422383|2421833|2422107|N|N|N|N|N| +2422199|AAAAAAAAHLFPECAA|1919-08-28|235|1026|79|1919|4|8|28|3|1919|79|1026|Thursday|1919Q3|N|N|N|2422172|2422383|2421834|2422108|N|N|N|N|N| +2422200|AAAAAAAAILFPECAA|1919-08-29|235|1026|79|1919|5|8|29|3|1919|79|1026|Friday|1919Q3|N|Y|N|2422172|2422383|2421835|2422109|N|N|N|N|N| +2422201|AAAAAAAAJLFPECAA|1919-08-30|235|1026|79|1919|6|8|30|3|1919|79|1026|Saturday|1919Q3|N|Y|N|2422172|2422383|2421836|2422110|N|N|N|N|N| +2422202|AAAAAAAAKLFPECAA|1919-08-31|235|1026|79|1919|0|8|31|3|1919|79|1026|Sunday|1919Q3|N|N|N|2422172|2422383|2421837|2422111|N|N|N|N|N| +2422203|AAAAAAAALLFPECAA|1919-09-01|236|1026|80|1919|1|9|1|3|1919|80|1026|Monday|1919Q3|N|N|N|2422203|2422445|2421838|2422112|N|N|N|N|N| +2422204|AAAAAAAAMLFPECAA|1919-09-02|236|1027|80|1919|2|9|2|3|1919|80|1027|Tuesday|1919Q3|N|N|N|2422203|2422445|2421839|2422113|N|N|N|N|N| +2422205|AAAAAAAANLFPECAA|1919-09-03|236|1027|80|1919|3|9|3|3|1919|80|1027|Wednesday|1919Q3|N|N|N|2422203|2422445|2421840|2422114|N|N|N|N|N| +2422206|AAAAAAAAOLFPECAA|1919-09-04|236|1027|80|1919|4|9|4|3|1919|80|1027|Thursday|1919Q3|N|N|N|2422203|2422445|2421841|2422115|N|N|N|N|N| +2422207|AAAAAAAAPLFPECAA|1919-09-05|236|1027|80|1919|5|9|5|3|1919|80|1027|Friday|1919Q3|N|Y|N|2422203|2422445|2421842|2422116|N|N|N|N|N| +2422208|AAAAAAAAAMFPECAA|1919-09-06|236|1027|80|1919|6|9|6|3|1919|80|1027|Saturday|1919Q3|N|Y|N|2422203|2422445|2421843|2422117|N|N|N|N|N| +2422209|AAAAAAAABMFPECAA|1919-09-07|236|1027|80|1919|0|9|7|3|1919|80|1027|Sunday|1919Q3|N|N|N|2422203|2422445|2421844|2422118|N|N|N|N|N| +2422210|AAAAAAAACMFPECAA|1919-09-08|236|1027|80|1919|1|9|8|3|1919|80|1027|Monday|1919Q3|N|N|N|2422203|2422445|2421845|2422119|N|N|N|N|N| +2422211|AAAAAAAADMFPECAA|1919-09-09|236|1028|80|1919|2|9|9|3|1919|80|1028|Tuesday|1919Q3|N|N|N|2422203|2422445|2421846|2422120|N|N|N|N|N| +2422212|AAAAAAAAEMFPECAA|1919-09-10|236|1028|80|1919|3|9|10|3|1919|80|1028|Wednesday|1919Q3|N|N|N|2422203|2422445|2421847|2422121|N|N|N|N|N| +2422213|AAAAAAAAFMFPECAA|1919-09-11|236|1028|80|1919|4|9|11|3|1919|80|1028|Thursday|1919Q3|N|N|N|2422203|2422445|2421848|2422122|N|N|N|N|N| +2422214|AAAAAAAAGMFPECAA|1919-09-12|236|1028|80|1919|5|9|12|3|1919|80|1028|Friday|1919Q3|N|Y|N|2422203|2422445|2421849|2422123|N|N|N|N|N| +2422215|AAAAAAAAHMFPECAA|1919-09-13|236|1028|80|1919|6|9|13|3|1919|80|1028|Saturday|1919Q3|N|Y|N|2422203|2422445|2421850|2422124|N|N|N|N|N| +2422216|AAAAAAAAIMFPECAA|1919-09-14|236|1028|80|1919|0|9|14|3|1919|80|1028|Sunday|1919Q3|N|N|N|2422203|2422445|2421851|2422125|N|N|N|N|N| +2422217|AAAAAAAAJMFPECAA|1919-09-15|236|1028|80|1919|1|9|15|3|1919|80|1028|Monday|1919Q3|N|N|N|2422203|2422445|2421852|2422126|N|N|N|N|N| +2422218|AAAAAAAAKMFPECAA|1919-09-16|236|1029|80|1919|2|9|16|3|1919|80|1029|Tuesday|1919Q3|N|N|N|2422203|2422445|2421853|2422127|N|N|N|N|N| +2422219|AAAAAAAALMFPECAA|1919-09-17|236|1029|80|1919|3|9|17|3|1919|80|1029|Wednesday|1919Q3|N|N|N|2422203|2422445|2421854|2422128|N|N|N|N|N| +2422220|AAAAAAAAMMFPECAA|1919-09-18|236|1029|80|1919|4|9|18|3|1919|80|1029|Thursday|1919Q3|N|N|N|2422203|2422445|2421855|2422129|N|N|N|N|N| +2422221|AAAAAAAANMFPECAA|1919-09-19|236|1029|80|1919|5|9|19|3|1919|80|1029|Friday|1919Q3|N|Y|N|2422203|2422445|2421856|2422130|N|N|N|N|N| +2422222|AAAAAAAAOMFPECAA|1919-09-20|236|1029|80|1919|6|9|20|3|1919|80|1029|Saturday|1919Q3|N|Y|N|2422203|2422445|2421857|2422131|N|N|N|N|N| +2422223|AAAAAAAAPMFPECAA|1919-09-21|236|1029|80|1919|0|9|21|3|1919|80|1029|Sunday|1919Q3|N|N|N|2422203|2422445|2421858|2422132|N|N|N|N|N| +2422224|AAAAAAAAANFPECAA|1919-09-22|236|1029|80|1919|1|9|22|3|1919|80|1029|Monday|1919Q3|N|N|N|2422203|2422445|2421859|2422133|N|N|N|N|N| +2422225|AAAAAAAABNFPECAA|1919-09-23|236|1030|80|1919|2|9|23|3|1919|80|1030|Tuesday|1919Q3|N|N|N|2422203|2422445|2421860|2422134|N|N|N|N|N| +2422226|AAAAAAAACNFPECAA|1919-09-24|236|1030|80|1919|3|9|24|3|1919|80|1030|Wednesday|1919Q3|N|N|N|2422203|2422445|2421861|2422135|N|N|N|N|N| +2422227|AAAAAAAADNFPECAA|1919-09-25|236|1030|80|1919|4|9|25|3|1919|80|1030|Thursday|1919Q3|N|N|N|2422203|2422445|2421862|2422136|N|N|N|N|N| +2422228|AAAAAAAAENFPECAA|1919-09-26|236|1030|80|1919|5|9|26|3|1919|80|1030|Friday|1919Q3|N|Y|N|2422203|2422445|2421863|2422137|N|N|N|N|N| +2422229|AAAAAAAAFNFPECAA|1919-09-27|236|1030|80|1919|6|9|27|3|1919|80|1030|Saturday|1919Q3|N|Y|N|2422203|2422445|2421864|2422138|N|N|N|N|N| +2422230|AAAAAAAAGNFPECAA|1919-09-28|236|1030|80|1919|0|9|28|3|1919|80|1030|Sunday|1919Q3|N|N|N|2422203|2422445|2421865|2422139|N|N|N|N|N| +2422231|AAAAAAAAHNFPECAA|1919-09-29|236|1030|80|1919|1|9|29|3|1919|80|1030|Monday|1919Q3|N|N|N|2422203|2422445|2421866|2422140|N|N|N|N|N| +2422232|AAAAAAAAINFPECAA|1919-09-30|236|1031|80|1919|2|9|30|3|1919|80|1031|Tuesday|1919Q3|N|N|N|2422203|2422445|2421867|2422141|N|N|N|N|N| +2422233|AAAAAAAAJNFPECAA|1919-10-01|237|1031|80|1919|3|10|1|3|1919|80|1031|Wednesday|1919Q3|N|N|N|2422233|2422505|2421868|2422141|N|N|N|N|N| +2422234|AAAAAAAAKNFPECAA|1919-10-02|237|1031|80|1919|4|10|2|4|1919|80|1031|Thursday|1919Q4|N|N|N|2422233|2422505|2421869|2422142|N|N|N|N|N| +2422235|AAAAAAAALNFPECAA|1919-10-03|237|1031|80|1919|5|10|3|4|1919|80|1031|Friday|1919Q4|N|Y|N|2422233|2422505|2421870|2422143|N|N|N|N|N| +2422236|AAAAAAAAMNFPECAA|1919-10-04|237|1031|80|1919|6|10|4|4|1919|80|1031|Saturday|1919Q4|N|Y|N|2422233|2422505|2421871|2422144|N|N|N|N|N| +2422237|AAAAAAAANNFPECAA|1919-10-05|237|1031|80|1919|0|10|5|4|1919|80|1031|Sunday|1919Q4|N|N|N|2422233|2422505|2421872|2422145|N|N|N|N|N| +2422238|AAAAAAAAONFPECAA|1919-10-06|237|1031|80|1919|1|10|6|4|1919|80|1031|Monday|1919Q4|N|N|N|2422233|2422505|2421873|2422146|N|N|N|N|N| +2422239|AAAAAAAAPNFPECAA|1919-10-07|237|1032|80|1919|2|10|7|4|1919|80|1032|Tuesday|1919Q4|N|N|N|2422233|2422505|2421874|2422147|N|N|N|N|N| +2422240|AAAAAAAAAOFPECAA|1919-10-08|237|1032|80|1919|3|10|8|4|1919|80|1032|Wednesday|1919Q4|N|N|N|2422233|2422505|2421875|2422148|N|N|N|N|N| +2422241|AAAAAAAABOFPECAA|1919-10-09|237|1032|80|1919|4|10|9|4|1919|80|1032|Thursday|1919Q4|N|N|N|2422233|2422505|2421876|2422149|N|N|N|N|N| +2422242|AAAAAAAACOFPECAA|1919-10-10|237|1032|80|1919|5|10|10|4|1919|80|1032|Friday|1919Q4|N|Y|N|2422233|2422505|2421877|2422150|N|N|N|N|N| +2422243|AAAAAAAADOFPECAA|1919-10-11|237|1032|80|1919|6|10|11|4|1919|80|1032|Saturday|1919Q4|N|Y|N|2422233|2422505|2421878|2422151|N|N|N|N|N| +2422244|AAAAAAAAEOFPECAA|1919-10-12|237|1032|80|1919|0|10|12|4|1919|80|1032|Sunday|1919Q4|N|N|N|2422233|2422505|2421879|2422152|N|N|N|N|N| +2422245|AAAAAAAAFOFPECAA|1919-10-13|237|1032|80|1919|1|10|13|4|1919|80|1032|Monday|1919Q4|N|N|N|2422233|2422505|2421880|2422153|N|N|N|N|N| +2422246|AAAAAAAAGOFPECAA|1919-10-14|237|1033|80|1919|2|10|14|4|1919|80|1033|Tuesday|1919Q4|N|N|N|2422233|2422505|2421881|2422154|N|N|N|N|N| +2422247|AAAAAAAAHOFPECAA|1919-10-15|237|1033|80|1919|3|10|15|4|1919|80|1033|Wednesday|1919Q4|N|N|N|2422233|2422505|2421882|2422155|N|N|N|N|N| +2422248|AAAAAAAAIOFPECAA|1919-10-16|237|1033|80|1919|4|10|16|4|1919|80|1033|Thursday|1919Q4|N|N|N|2422233|2422505|2421883|2422156|N|N|N|N|N| +2422249|AAAAAAAAJOFPECAA|1919-10-17|237|1033|80|1919|5|10|17|4|1919|80|1033|Friday|1919Q4|N|Y|N|2422233|2422505|2421884|2422157|N|N|N|N|N| +2422250|AAAAAAAAKOFPECAA|1919-10-18|237|1033|80|1919|6|10|18|4|1919|80|1033|Saturday|1919Q4|N|Y|N|2422233|2422505|2421885|2422158|N|N|N|N|N| +2422251|AAAAAAAALOFPECAA|1919-10-19|237|1033|80|1919|0|10|19|4|1919|80|1033|Sunday|1919Q4|N|N|N|2422233|2422505|2421886|2422159|N|N|N|N|N| +2422252|AAAAAAAAMOFPECAA|1919-10-20|237|1033|80|1919|1|10|20|4|1919|80|1033|Monday|1919Q4|N|N|N|2422233|2422505|2421887|2422160|N|N|N|N|N| +2422253|AAAAAAAANOFPECAA|1919-10-21|237|1034|80|1919|2|10|21|4|1919|80|1034|Tuesday|1919Q4|N|N|N|2422233|2422505|2421888|2422161|N|N|N|N|N| +2422254|AAAAAAAAOOFPECAA|1919-10-22|237|1034|80|1919|3|10|22|4|1919|80|1034|Wednesday|1919Q4|N|N|N|2422233|2422505|2421889|2422162|N|N|N|N|N| +2422255|AAAAAAAAPOFPECAA|1919-10-23|237|1034|80|1919|4|10|23|4|1919|80|1034|Thursday|1919Q4|N|N|N|2422233|2422505|2421890|2422163|N|N|N|N|N| +2422256|AAAAAAAAAPFPECAA|1919-10-24|237|1034|80|1919|5|10|24|4|1919|80|1034|Friday|1919Q4|N|Y|N|2422233|2422505|2421891|2422164|N|N|N|N|N| +2422257|AAAAAAAABPFPECAA|1919-10-25|237|1034|80|1919|6|10|25|4|1919|80|1034|Saturday|1919Q4|N|Y|N|2422233|2422505|2421892|2422165|N|N|N|N|N| +2422258|AAAAAAAACPFPECAA|1919-10-26|237|1034|80|1919|0|10|26|4|1919|80|1034|Sunday|1919Q4|N|N|N|2422233|2422505|2421893|2422166|N|N|N|N|N| +2422259|AAAAAAAADPFPECAA|1919-10-27|237|1034|80|1919|1|10|27|4|1919|80|1034|Monday|1919Q4|N|N|N|2422233|2422505|2421894|2422167|N|N|N|N|N| +2422260|AAAAAAAAEPFPECAA|1919-10-28|237|1035|80|1919|2|10|28|4|1919|80|1035|Tuesday|1919Q4|N|N|N|2422233|2422505|2421895|2422168|N|N|N|N|N| +2422261|AAAAAAAAFPFPECAA|1919-10-29|237|1035|80|1919|3|10|29|4|1919|80|1035|Wednesday|1919Q4|N|N|N|2422233|2422505|2421896|2422169|N|N|N|N|N| +2422262|AAAAAAAAGPFPECAA|1919-10-30|237|1035|80|1919|4|10|30|4|1919|80|1035|Thursday|1919Q4|N|N|N|2422233|2422505|2421897|2422170|N|N|N|N|N| +2422263|AAAAAAAAHPFPECAA|1919-10-31|237|1035|80|1919|5|10|31|4|1919|80|1035|Friday|1919Q4|N|Y|N|2422233|2422505|2421898|2422171|N|N|N|N|N| +2422264|AAAAAAAAIPFPECAA|1919-11-01|238|1035|80|1919|6|11|1|4|1919|80|1035|Saturday|1919Q4|N|Y|N|2422264|2422567|2421899|2422172|N|N|N|N|N| +2422265|AAAAAAAAJPFPECAA|1919-11-02|238|1035|80|1919|0|11|2|4|1919|80|1035|Sunday|1919Q4|N|N|N|2422264|2422567|2421900|2422173|N|N|N|N|N| +2422266|AAAAAAAAKPFPECAA|1919-11-03|238|1035|80|1919|1|11|3|4|1919|80|1035|Monday|1919Q4|N|N|N|2422264|2422567|2421901|2422174|N|N|N|N|N| +2422267|AAAAAAAALPFPECAA|1919-11-04|238|1036|80|1919|2|11|4|4|1919|80|1036|Tuesday|1919Q4|N|N|N|2422264|2422567|2421902|2422175|N|N|N|N|N| +2422268|AAAAAAAAMPFPECAA|1919-11-05|238|1036|80|1919|3|11|5|4|1919|80|1036|Wednesday|1919Q4|N|N|N|2422264|2422567|2421903|2422176|N|N|N|N|N| +2422269|AAAAAAAANPFPECAA|1919-11-06|238|1036|80|1919|4|11|6|4|1919|80|1036|Thursday|1919Q4|N|N|N|2422264|2422567|2421904|2422177|N|N|N|N|N| +2422270|AAAAAAAAOPFPECAA|1919-11-07|238|1036|80|1919|5|11|7|4|1919|80|1036|Friday|1919Q4|N|Y|N|2422264|2422567|2421905|2422178|N|N|N|N|N| +2422271|AAAAAAAAPPFPECAA|1919-11-08|238|1036|80|1919|6|11|8|4|1919|80|1036|Saturday|1919Q4|N|Y|N|2422264|2422567|2421906|2422179|N|N|N|N|N| +2422272|AAAAAAAAAAGPECAA|1919-11-09|238|1036|80|1919|0|11|9|4|1919|80|1036|Sunday|1919Q4|N|N|N|2422264|2422567|2421907|2422180|N|N|N|N|N| +2422273|AAAAAAAABAGPECAA|1919-11-10|238|1036|80|1919|1|11|10|4|1919|80|1036|Monday|1919Q4|N|N|N|2422264|2422567|2421908|2422181|N|N|N|N|N| +2422274|AAAAAAAACAGPECAA|1919-11-11|238|1037|80|1919|2|11|11|4|1919|80|1037|Tuesday|1919Q4|N|N|N|2422264|2422567|2421909|2422182|N|N|N|N|N| +2422275|AAAAAAAADAGPECAA|1919-11-12|238|1037|80|1919|3|11|12|4|1919|80|1037|Wednesday|1919Q4|N|N|N|2422264|2422567|2421910|2422183|N|N|N|N|N| +2422276|AAAAAAAAEAGPECAA|1919-11-13|238|1037|80|1919|4|11|13|4|1919|80|1037|Thursday|1919Q4|N|N|N|2422264|2422567|2421911|2422184|N|N|N|N|N| +2422277|AAAAAAAAFAGPECAA|1919-11-14|238|1037|80|1919|5|11|14|4|1919|80|1037|Friday|1919Q4|N|Y|N|2422264|2422567|2421912|2422185|N|N|N|N|N| +2422278|AAAAAAAAGAGPECAA|1919-11-15|238|1037|80|1919|6|11|15|4|1919|80|1037|Saturday|1919Q4|N|Y|N|2422264|2422567|2421913|2422186|N|N|N|N|N| +2422279|AAAAAAAAHAGPECAA|1919-11-16|238|1037|80|1919|0|11|16|4|1919|80|1037|Sunday|1919Q4|N|N|N|2422264|2422567|2421914|2422187|N|N|N|N|N| +2422280|AAAAAAAAIAGPECAA|1919-11-17|238|1037|80|1919|1|11|17|4|1919|80|1037|Monday|1919Q4|N|N|N|2422264|2422567|2421915|2422188|N|N|N|N|N| +2422281|AAAAAAAAJAGPECAA|1919-11-18|238|1038|80|1919|2|11|18|4|1919|80|1038|Tuesday|1919Q4|N|N|N|2422264|2422567|2421916|2422189|N|N|N|N|N| +2422282|AAAAAAAAKAGPECAA|1919-11-19|238|1038|80|1919|3|11|19|4|1919|80|1038|Wednesday|1919Q4|N|N|N|2422264|2422567|2421917|2422190|N|N|N|N|N| +2422283|AAAAAAAALAGPECAA|1919-11-20|238|1038|80|1919|4|11|20|4|1919|80|1038|Thursday|1919Q4|N|N|N|2422264|2422567|2421918|2422191|N|N|N|N|N| +2422284|AAAAAAAAMAGPECAA|1919-11-21|238|1038|80|1919|5|11|21|4|1919|80|1038|Friday|1919Q4|N|Y|N|2422264|2422567|2421919|2422192|N|N|N|N|N| +2422285|AAAAAAAANAGPECAA|1919-11-22|238|1038|80|1919|6|11|22|4|1919|80|1038|Saturday|1919Q4|N|Y|N|2422264|2422567|2421920|2422193|N|N|N|N|N| +2422286|AAAAAAAAOAGPECAA|1919-11-23|238|1038|80|1919|0|11|23|4|1919|80|1038|Sunday|1919Q4|N|N|N|2422264|2422567|2421921|2422194|N|N|N|N|N| +2422287|AAAAAAAAPAGPECAA|1919-11-24|238|1038|80|1919|1|11|24|4|1919|80|1038|Monday|1919Q4|N|N|N|2422264|2422567|2421922|2422195|N|N|N|N|N| +2422288|AAAAAAAAABGPECAA|1919-11-25|238|1039|80|1919|2|11|25|4|1919|80|1039|Tuesday|1919Q4|N|N|N|2422264|2422567|2421923|2422196|N|N|N|N|N| +2422289|AAAAAAAABBGPECAA|1919-11-26|238|1039|80|1919|3|11|26|4|1919|80|1039|Wednesday|1919Q4|N|N|N|2422264|2422567|2421924|2422197|N|N|N|N|N| +2422290|AAAAAAAACBGPECAA|1919-11-27|238|1039|80|1919|4|11|27|4|1919|80|1039|Thursday|1919Q4|N|N|N|2422264|2422567|2421925|2422198|N|N|N|N|N| +2422291|AAAAAAAADBGPECAA|1919-11-28|238|1039|80|1919|5|11|28|4|1919|80|1039|Friday|1919Q4|N|Y|N|2422264|2422567|2421926|2422199|N|N|N|N|N| +2422292|AAAAAAAAEBGPECAA|1919-11-29|238|1039|80|1919|6|11|29|4|1919|80|1039|Saturday|1919Q4|N|Y|N|2422264|2422567|2421927|2422200|N|N|N|N|N| +2422293|AAAAAAAAFBGPECAA|1919-11-30|238|1039|80|1919|0|11|30|4|1919|80|1039|Sunday|1919Q4|N|N|N|2422264|2422567|2421928|2422201|N|N|N|N|N| +2422294|AAAAAAAAGBGPECAA|1919-12-01|239|1039|81|1919|1|12|1|4|1919|81|1039|Monday|1919Q4|N|N|N|2422294|2422627|2421929|2422202|N|N|N|N|N| +2422295|AAAAAAAAHBGPECAA|1919-12-02|239|1040|81|1919|2|12|2|4|1919|81|1040|Tuesday|1919Q4|N|N|N|2422294|2422627|2421930|2422203|N|N|N|N|N| +2422296|AAAAAAAAIBGPECAA|1919-12-03|239|1040|81|1919|3|12|3|4|1919|81|1040|Wednesday|1919Q4|N|N|N|2422294|2422627|2421931|2422204|N|N|N|N|N| +2422297|AAAAAAAAJBGPECAA|1919-12-04|239|1040|81|1919|4|12|4|4|1919|81|1040|Thursday|1919Q4|N|N|N|2422294|2422627|2421932|2422205|N|N|N|N|N| +2422298|AAAAAAAAKBGPECAA|1919-12-05|239|1040|81|1919|5|12|5|4|1919|81|1040|Friday|1919Q4|N|Y|N|2422294|2422627|2421933|2422206|N|N|N|N|N| +2422299|AAAAAAAALBGPECAA|1919-12-06|239|1040|81|1919|6|12|6|4|1919|81|1040|Saturday|1919Q4|N|Y|N|2422294|2422627|2421934|2422207|N|N|N|N|N| +2422300|AAAAAAAAMBGPECAA|1919-12-07|239|1040|81|1919|0|12|7|4|1919|81|1040|Sunday|1919Q4|N|N|N|2422294|2422627|2421935|2422208|N|N|N|N|N| +2422301|AAAAAAAANBGPECAA|1919-12-08|239|1040|81|1919|1|12|8|4|1919|81|1040|Monday|1919Q4|N|N|N|2422294|2422627|2421936|2422209|N|N|N|N|N| +2422302|AAAAAAAAOBGPECAA|1919-12-09|239|1041|81|1919|2|12|9|4|1919|81|1041|Tuesday|1919Q4|N|N|N|2422294|2422627|2421937|2422210|N|N|N|N|N| +2422303|AAAAAAAAPBGPECAA|1919-12-10|239|1041|81|1919|3|12|10|4|1919|81|1041|Wednesday|1919Q4|N|N|N|2422294|2422627|2421938|2422211|N|N|N|N|N| +2422304|AAAAAAAAACGPECAA|1919-12-11|239|1041|81|1919|4|12|11|4|1919|81|1041|Thursday|1919Q4|N|N|N|2422294|2422627|2421939|2422212|N|N|N|N|N| +2422305|AAAAAAAABCGPECAA|1919-12-12|239|1041|81|1919|5|12|12|4|1919|81|1041|Friday|1919Q4|N|Y|N|2422294|2422627|2421940|2422213|N|N|N|N|N| +2422306|AAAAAAAACCGPECAA|1919-12-13|239|1041|81|1919|6|12|13|4|1919|81|1041|Saturday|1919Q4|N|Y|N|2422294|2422627|2421941|2422214|N|N|N|N|N| +2422307|AAAAAAAADCGPECAA|1919-12-14|239|1041|81|1919|0|12|14|4|1919|81|1041|Sunday|1919Q4|N|N|N|2422294|2422627|2421942|2422215|N|N|N|N|N| +2422308|AAAAAAAAECGPECAA|1919-12-15|239|1041|81|1919|1|12|15|4|1919|81|1041|Monday|1919Q4|N|N|N|2422294|2422627|2421943|2422216|N|N|N|N|N| +2422309|AAAAAAAAFCGPECAA|1919-12-16|239|1042|81|1919|2|12|16|4|1919|81|1042|Tuesday|1919Q4|N|N|N|2422294|2422627|2421944|2422217|N|N|N|N|N| +2422310|AAAAAAAAGCGPECAA|1919-12-17|239|1042|81|1919|3|12|17|4|1919|81|1042|Wednesday|1919Q4|N|N|N|2422294|2422627|2421945|2422218|N|N|N|N|N| +2422311|AAAAAAAAHCGPECAA|1919-12-18|239|1042|81|1919|4|12|18|4|1919|81|1042|Thursday|1919Q4|N|N|N|2422294|2422627|2421946|2422219|N|N|N|N|N| +2422312|AAAAAAAAICGPECAA|1919-12-19|239|1042|81|1919|5|12|19|4|1919|81|1042|Friday|1919Q4|N|Y|N|2422294|2422627|2421947|2422220|N|N|N|N|N| +2422313|AAAAAAAAJCGPECAA|1919-12-20|239|1042|81|1919|6|12|20|4|1919|81|1042|Saturday|1919Q4|N|Y|N|2422294|2422627|2421948|2422221|N|N|N|N|N| +2422314|AAAAAAAAKCGPECAA|1919-12-21|239|1042|81|1919|0|12|21|4|1919|81|1042|Sunday|1919Q4|N|N|N|2422294|2422627|2421949|2422222|N|N|N|N|N| +2422315|AAAAAAAALCGPECAA|1919-12-22|239|1042|81|1919|1|12|22|4|1919|81|1042|Monday|1919Q4|N|N|N|2422294|2422627|2421950|2422223|N|N|N|N|N| +2422316|AAAAAAAAMCGPECAA|1919-12-23|239|1043|81|1919|2|12|23|4|1919|81|1043|Tuesday|1919Q4|N|N|N|2422294|2422627|2421951|2422224|N|N|N|N|N| +2422317|AAAAAAAANCGPECAA|1919-12-24|239|1043|81|1919|3|12|24|4|1919|81|1043|Wednesday|1919Q4|N|N|N|2422294|2422627|2421952|2422225|N|N|N|N|N| +2422318|AAAAAAAAOCGPECAA|1919-12-25|239|1043|81|1919|4|12|25|4|1919|81|1043|Thursday|1919Q4|N|N|N|2422294|2422627|2421953|2422226|N|N|N|N|N| +2422319|AAAAAAAAPCGPECAA|1919-12-26|239|1043|81|1919|5|12|26|4|1919|81|1043|Friday|1919Q4|Y|Y|N|2422294|2422627|2421954|2422227|N|N|N|N|N| +2422320|AAAAAAAAADGPECAA|1919-12-27|239|1043|81|1919|6|12|27|4|1919|81|1043|Saturday|1919Q4|N|Y|Y|2422294|2422627|2421955|2422228|N|N|N|N|N| +2422321|AAAAAAAABDGPECAA|1919-12-28|239|1043|81|1919|0|12|28|4|1919|81|1043|Sunday|1919Q4|N|N|N|2422294|2422627|2421956|2422229|N|N|N|N|N| +2422322|AAAAAAAACDGPECAA|1919-12-29|239|1043|81|1919|1|12|29|4|1919|81|1043|Monday|1919Q4|N|N|N|2422294|2422627|2421957|2422230|N|N|N|N|N| +2422323|AAAAAAAADDGPECAA|1919-12-30|239|1044|81|1919|2|12|30|4|1919|81|1044|Tuesday|1919Q4|N|N|N|2422294|2422627|2421958|2422231|N|N|N|N|N| +2422324|AAAAAAAAEDGPECAA|1919-12-31|239|1044|81|1919|3|12|31|4|1919|81|1044|Wednesday|1919Q4|N|N|N|2422294|2422627|2421959|2422232|N|N|N|N|N| +2422325|AAAAAAAAFDGPECAA|1920-01-01|240|1044|81|1920|4|1|1|1|1920|81|1044|Thursday|1920Q1|Y|N|N|2422325|2422324|2421960|2422233|N|N|N|N|N| +2422326|AAAAAAAAGDGPECAA|1920-01-02|240|1044|81|1920|5|1|2|1|1920|81|1044|Friday|1920Q1|N|Y|Y|2422325|2422324|2421961|2422234|N|N|N|N|N| +2422327|AAAAAAAAHDGPECAA|1920-01-03|240|1044|81|1920|6|1|3|1|1920|81|1044|Saturday|1920Q1|N|Y|N|2422325|2422324|2421962|2422235|N|N|N|N|N| +2422328|AAAAAAAAIDGPECAA|1920-01-04|240|1044|81|1920|0|1|4|1|1920|81|1044|Sunday|1920Q1|N|N|N|2422325|2422324|2421963|2422236|N|N|N|N|N| +2422329|AAAAAAAAJDGPECAA|1920-01-05|240|1044|81|1920|1|1|5|1|1920|81|1044|Monday|1920Q1|N|N|N|2422325|2422324|2421964|2422237|N|N|N|N|N| +2422330|AAAAAAAAKDGPECAA|1920-01-06|240|1045|81|1920|2|1|6|1|1920|81|1045|Tuesday|1920Q1|N|N|N|2422325|2422324|2421965|2422238|N|N|N|N|N| +2422331|AAAAAAAALDGPECAA|1920-01-07|240|1045|81|1920|3|1|7|1|1920|81|1045|Wednesday|1920Q1|N|N|N|2422325|2422324|2421966|2422239|N|N|N|N|N| +2422332|AAAAAAAAMDGPECAA|1920-01-08|240|1045|81|1920|4|1|8|1|1920|81|1045|Thursday|1920Q1|N|N|N|2422325|2422324|2421967|2422240|N|N|N|N|N| +2422333|AAAAAAAANDGPECAA|1920-01-09|240|1045|81|1920|5|1|9|1|1920|81|1045|Friday|1920Q1|N|Y|N|2422325|2422324|2421968|2422241|N|N|N|N|N| +2422334|AAAAAAAAODGPECAA|1920-01-10|240|1045|81|1920|6|1|10|1|1920|81|1045|Saturday|1920Q1|N|Y|N|2422325|2422324|2421969|2422242|N|N|N|N|N| +2422335|AAAAAAAAPDGPECAA|1920-01-11|240|1045|81|1920|0|1|11|1|1920|81|1045|Sunday|1920Q1|N|N|N|2422325|2422324|2421970|2422243|N|N|N|N|N| +2422336|AAAAAAAAAEGPECAA|1920-01-12|240|1045|81|1920|1|1|12|1|1920|81|1045|Monday|1920Q1|N|N|N|2422325|2422324|2421971|2422244|N|N|N|N|N| +2422337|AAAAAAAABEGPECAA|1920-01-13|240|1046|81|1920|2|1|13|1|1920|81|1046|Tuesday|1920Q1|N|N|N|2422325|2422324|2421972|2422245|N|N|N|N|N| +2422338|AAAAAAAACEGPECAA|1920-01-14|240|1046|81|1920|3|1|14|1|1920|81|1046|Wednesday|1920Q1|N|N|N|2422325|2422324|2421973|2422246|N|N|N|N|N| +2422339|AAAAAAAADEGPECAA|1920-01-15|240|1046|81|1920|4|1|15|1|1920|81|1046|Thursday|1920Q1|N|N|N|2422325|2422324|2421974|2422247|N|N|N|N|N| +2422340|AAAAAAAAEEGPECAA|1920-01-16|240|1046|81|1920|5|1|16|1|1920|81|1046|Friday|1920Q1|N|Y|N|2422325|2422324|2421975|2422248|N|N|N|N|N| +2422341|AAAAAAAAFEGPECAA|1920-01-17|240|1046|81|1920|6|1|17|1|1920|81|1046|Saturday|1920Q1|N|Y|N|2422325|2422324|2421976|2422249|N|N|N|N|N| +2422342|AAAAAAAAGEGPECAA|1920-01-18|240|1046|81|1920|0|1|18|1|1920|81|1046|Sunday|1920Q1|N|N|N|2422325|2422324|2421977|2422250|N|N|N|N|N| +2422343|AAAAAAAAHEGPECAA|1920-01-19|240|1046|81|1920|1|1|19|1|1920|81|1046|Monday|1920Q1|N|N|N|2422325|2422324|2421978|2422251|N|N|N|N|N| +2422344|AAAAAAAAIEGPECAA|1920-01-20|240|1047|81|1920|2|1|20|1|1920|81|1047|Tuesday|1920Q1|N|N|N|2422325|2422324|2421979|2422252|N|N|N|N|N| +2422345|AAAAAAAAJEGPECAA|1920-01-21|240|1047|81|1920|3|1|21|1|1920|81|1047|Wednesday|1920Q1|N|N|N|2422325|2422324|2421980|2422253|N|N|N|N|N| +2422346|AAAAAAAAKEGPECAA|1920-01-22|240|1047|81|1920|4|1|22|1|1920|81|1047|Thursday|1920Q1|N|N|N|2422325|2422324|2421981|2422254|N|N|N|N|N| +2422347|AAAAAAAALEGPECAA|1920-01-23|240|1047|81|1920|5|1|23|1|1920|81|1047|Friday|1920Q1|N|Y|N|2422325|2422324|2421982|2422255|N|N|N|N|N| +2422348|AAAAAAAAMEGPECAA|1920-01-24|240|1047|81|1920|6|1|24|1|1920|81|1047|Saturday|1920Q1|N|Y|N|2422325|2422324|2421983|2422256|N|N|N|N|N| +2422349|AAAAAAAANEGPECAA|1920-01-25|240|1047|81|1920|0|1|25|1|1920|81|1047|Sunday|1920Q1|N|N|N|2422325|2422324|2421984|2422257|N|N|N|N|N| +2422350|AAAAAAAAOEGPECAA|1920-01-26|240|1047|81|1920|1|1|26|1|1920|81|1047|Monday|1920Q1|N|N|N|2422325|2422324|2421985|2422258|N|N|N|N|N| +2422351|AAAAAAAAPEGPECAA|1920-01-27|240|1048|81|1920|2|1|27|1|1920|81|1048|Tuesday|1920Q1|N|N|N|2422325|2422324|2421986|2422259|N|N|N|N|N| +2422352|AAAAAAAAAFGPECAA|1920-01-28|240|1048|81|1920|3|1|28|1|1920|81|1048|Wednesday|1920Q1|N|N|N|2422325|2422324|2421987|2422260|N|N|N|N|N| +2422353|AAAAAAAABFGPECAA|1920-01-29|240|1048|81|1920|4|1|29|1|1920|81|1048|Thursday|1920Q1|N|N|N|2422325|2422324|2421988|2422261|N|N|N|N|N| +2422354|AAAAAAAACFGPECAA|1920-01-30|240|1048|81|1920|5|1|30|1|1920|81|1048|Friday|1920Q1|N|Y|N|2422325|2422324|2421989|2422262|N|N|N|N|N| +2422355|AAAAAAAADFGPECAA|1920-01-31|240|1048|81|1920|6|1|31|1|1920|81|1048|Saturday|1920Q1|N|Y|N|2422325|2422324|2421990|2422263|N|N|N|N|N| +2422356|AAAAAAAAEFGPECAA|1920-02-01|241|1048|81|1920|0|2|1|1|1920|81|1048|Sunday|1920Q1|N|N|N|2422356|2422386|2421991|2422264|N|N|N|N|N| +2422357|AAAAAAAAFFGPECAA|1920-02-02|241|1048|81|1920|1|2|2|1|1920|81|1048|Monday|1920Q1|N|N|N|2422356|2422386|2421992|2422265|N|N|N|N|N| +2422358|AAAAAAAAGFGPECAA|1920-02-03|241|1049|81|1920|2|2|3|1|1920|81|1049|Tuesday|1920Q1|N|N|N|2422356|2422386|2421993|2422266|N|N|N|N|N| +2422359|AAAAAAAAHFGPECAA|1920-02-04|241|1049|81|1920|3|2|4|1|1920|81|1049|Wednesday|1920Q1|N|N|N|2422356|2422386|2421994|2422267|N|N|N|N|N| +2422360|AAAAAAAAIFGPECAA|1920-02-05|241|1049|81|1920|4|2|5|1|1920|81|1049|Thursday|1920Q1|N|N|N|2422356|2422386|2421995|2422268|N|N|N|N|N| +2422361|AAAAAAAAJFGPECAA|1920-02-06|241|1049|81|1920|5|2|6|1|1920|81|1049|Friday|1920Q1|N|Y|N|2422356|2422386|2421996|2422269|N|N|N|N|N| +2422362|AAAAAAAAKFGPECAA|1920-02-07|241|1049|81|1920|6|2|7|1|1920|81|1049|Saturday|1920Q1|N|Y|N|2422356|2422386|2421997|2422270|N|N|N|N|N| +2422363|AAAAAAAALFGPECAA|1920-02-08|241|1049|81|1920|0|2|8|1|1920|81|1049|Sunday|1920Q1|N|N|N|2422356|2422386|2421998|2422271|N|N|N|N|N| +2422364|AAAAAAAAMFGPECAA|1920-02-09|241|1049|81|1920|1|2|9|1|1920|81|1049|Monday|1920Q1|N|N|N|2422356|2422386|2421999|2422272|N|N|N|N|N| +2422365|AAAAAAAANFGPECAA|1920-02-10|241|1050|81|1920|2|2|10|1|1920|81|1050|Tuesday|1920Q1|N|N|N|2422356|2422386|2422000|2422273|N|N|N|N|N| +2422366|AAAAAAAAOFGPECAA|1920-02-11|241|1050|81|1920|3|2|11|1|1920|81|1050|Wednesday|1920Q1|N|N|N|2422356|2422386|2422001|2422274|N|N|N|N|N| +2422367|AAAAAAAAPFGPECAA|1920-02-12|241|1050|81|1920|4|2|12|1|1920|81|1050|Thursday|1920Q1|N|N|N|2422356|2422386|2422002|2422275|N|N|N|N|N| +2422368|AAAAAAAAAGGPECAA|1920-02-13|241|1050|81|1920|5|2|13|1|1920|81|1050|Friday|1920Q1|N|Y|N|2422356|2422386|2422003|2422276|N|N|N|N|N| +2422369|AAAAAAAABGGPECAA|1920-02-14|241|1050|81|1920|6|2|14|1|1920|81|1050|Saturday|1920Q1|N|Y|N|2422356|2422386|2422004|2422277|N|N|N|N|N| +2422370|AAAAAAAACGGPECAA|1920-02-15|241|1050|81|1920|0|2|15|1|1920|81|1050|Sunday|1920Q1|N|N|N|2422356|2422386|2422005|2422278|N|N|N|N|N| +2422371|AAAAAAAADGGPECAA|1920-02-16|241|1050|81|1920|1|2|16|1|1920|81|1050|Monday|1920Q1|N|N|N|2422356|2422386|2422006|2422279|N|N|N|N|N| +2422372|AAAAAAAAEGGPECAA|1920-02-17|241|1051|81|1920|2|2|17|1|1920|81|1051|Tuesday|1920Q1|N|N|N|2422356|2422386|2422007|2422280|N|N|N|N|N| +2422373|AAAAAAAAFGGPECAA|1920-02-18|241|1051|81|1920|3|2|18|1|1920|81|1051|Wednesday|1920Q1|N|N|N|2422356|2422386|2422008|2422281|N|N|N|N|N| +2422374|AAAAAAAAGGGPECAA|1920-02-19|241|1051|81|1920|4|2|19|1|1920|81|1051|Thursday|1920Q1|N|N|N|2422356|2422386|2422009|2422282|N|N|N|N|N| +2422375|AAAAAAAAHGGPECAA|1920-02-20|241|1051|81|1920|5|2|20|1|1920|81|1051|Friday|1920Q1|N|Y|N|2422356|2422386|2422010|2422283|N|N|N|N|N| +2422376|AAAAAAAAIGGPECAA|1920-02-21|241|1051|81|1920|6|2|21|1|1920|81|1051|Saturday|1920Q1|N|Y|N|2422356|2422386|2422011|2422284|N|N|N|N|N| +2422377|AAAAAAAAJGGPECAA|1920-02-22|241|1051|81|1920|0|2|22|1|1920|81|1051|Sunday|1920Q1|N|N|N|2422356|2422386|2422012|2422285|N|N|N|N|N| +2422378|AAAAAAAAKGGPECAA|1920-02-23|241|1051|81|1920|1|2|23|1|1920|81|1051|Monday|1920Q1|N|N|N|2422356|2422386|2422013|2422286|N|N|N|N|N| +2422379|AAAAAAAALGGPECAA|1920-02-24|241|1052|81|1920|2|2|24|1|1920|81|1052|Tuesday|1920Q1|N|N|N|2422356|2422386|2422014|2422287|N|N|N|N|N| +2422380|AAAAAAAAMGGPECAA|1920-02-25|241|1052|81|1920|3|2|25|1|1920|81|1052|Wednesday|1920Q1|N|N|N|2422356|2422386|2422015|2422288|N|N|N|N|N| +2422381|AAAAAAAANGGPECAA|1920-02-26|241|1052|81|1920|4|2|26|1|1920|81|1052|Thursday|1920Q1|N|N|N|2422356|2422386|2422016|2422289|N|N|N|N|N| +2422382|AAAAAAAAOGGPECAA|1920-02-27|241|1052|81|1920|5|2|27|1|1920|81|1052|Friday|1920Q1|N|Y|N|2422356|2422386|2422017|2422290|N|N|N|N|N| +2422383|AAAAAAAAPGGPECAA|1920-02-28|241|1052|81|1920|6|2|28|1|1920|81|1052|Saturday|1920Q1|N|Y|N|2422356|2422386|2422018|2422291|N|N|N|N|N| +2422384|AAAAAAAAAHGPECAA|1920-02-29|241|1052|81|1920|0|2|29|1|1920|81|1052|Sunday|1920Q1|N|N|N|2422356|2422386|2422018|2422292|N|N|N|N|N| +2422385|AAAAAAAABHGPECAA|1920-03-01|242|1052|82|1920|1|3|1|1|1920|82|1052|Monday|1920Q1|N|N|N|2422385|2422444|2422019|2422293|N|N|N|N|N| +2422386|AAAAAAAACHGPECAA|1920-03-02|242|1053|82|1920|2|3|2|1|1920|82|1053|Tuesday|1920Q1|N|N|N|2422385|2422444|2422020|2422294|N|N|N|N|N| +2422387|AAAAAAAADHGPECAA|1920-03-03|242|1053|82|1920|3|3|3|1|1920|82|1053|Wednesday|1920Q1|N|N|N|2422385|2422444|2422021|2422295|N|N|N|N|N| +2422388|AAAAAAAAEHGPECAA|1920-03-04|242|1053|82|1920|4|3|4|1|1920|82|1053|Thursday|1920Q1|N|N|N|2422385|2422444|2422022|2422296|N|N|N|N|N| +2422389|AAAAAAAAFHGPECAA|1920-03-05|242|1053|82|1920|5|3|5|1|1920|82|1053|Friday|1920Q1|N|Y|N|2422385|2422444|2422023|2422297|N|N|N|N|N| +2422390|AAAAAAAAGHGPECAA|1920-03-06|242|1053|82|1920|6|3|6|1|1920|82|1053|Saturday|1920Q1|N|Y|N|2422385|2422444|2422024|2422298|N|N|N|N|N| +2422391|AAAAAAAAHHGPECAA|1920-03-07|242|1053|82|1920|0|3|7|1|1920|82|1053|Sunday|1920Q1|N|N|N|2422385|2422444|2422025|2422299|N|N|N|N|N| +2422392|AAAAAAAAIHGPECAA|1920-03-08|242|1053|82|1920|1|3|8|1|1920|82|1053|Monday|1920Q1|N|N|N|2422385|2422444|2422026|2422300|N|N|N|N|N| +2422393|AAAAAAAAJHGPECAA|1920-03-09|242|1054|82|1920|2|3|9|1|1920|82|1054|Tuesday|1920Q1|N|N|N|2422385|2422444|2422027|2422301|N|N|N|N|N| +2422394|AAAAAAAAKHGPECAA|1920-03-10|242|1054|82|1920|3|3|10|1|1920|82|1054|Wednesday|1920Q1|N|N|N|2422385|2422444|2422028|2422302|N|N|N|N|N| +2422395|AAAAAAAALHGPECAA|1920-03-11|242|1054|82|1920|4|3|11|1|1920|82|1054|Thursday|1920Q1|N|N|N|2422385|2422444|2422029|2422303|N|N|N|N|N| +2422396|AAAAAAAAMHGPECAA|1920-03-12|242|1054|82|1920|5|3|12|1|1920|82|1054|Friday|1920Q1|N|Y|N|2422385|2422444|2422030|2422304|N|N|N|N|N| +2422397|AAAAAAAANHGPECAA|1920-03-13|242|1054|82|1920|6|3|13|1|1920|82|1054|Saturday|1920Q1|N|Y|N|2422385|2422444|2422031|2422305|N|N|N|N|N| +2422398|AAAAAAAAOHGPECAA|1920-03-14|242|1054|82|1920|0|3|14|1|1920|82|1054|Sunday|1920Q1|N|N|N|2422385|2422444|2422032|2422306|N|N|N|N|N| +2422399|AAAAAAAAPHGPECAA|1920-03-15|242|1054|82|1920|1|3|15|1|1920|82|1054|Monday|1920Q1|N|N|N|2422385|2422444|2422033|2422307|N|N|N|N|N| +2422400|AAAAAAAAAIGPECAA|1920-03-16|242|1055|82|1920|2|3|16|1|1920|82|1055|Tuesday|1920Q1|N|N|N|2422385|2422444|2422034|2422308|N|N|N|N|N| +2422401|AAAAAAAABIGPECAA|1920-03-17|242|1055|82|1920|3|3|17|1|1920|82|1055|Wednesday|1920Q1|N|N|N|2422385|2422444|2422035|2422309|N|N|N|N|N| +2422402|AAAAAAAACIGPECAA|1920-03-18|242|1055|82|1920|4|3|18|1|1920|82|1055|Thursday|1920Q1|N|N|N|2422385|2422444|2422036|2422310|N|N|N|N|N| +2422403|AAAAAAAADIGPECAA|1920-03-19|242|1055|82|1920|5|3|19|1|1920|82|1055|Friday|1920Q1|N|Y|N|2422385|2422444|2422037|2422311|N|N|N|N|N| +2422404|AAAAAAAAEIGPECAA|1920-03-20|242|1055|82|1920|6|3|20|1|1920|82|1055|Saturday|1920Q1|N|Y|N|2422385|2422444|2422038|2422312|N|N|N|N|N| +2422405|AAAAAAAAFIGPECAA|1920-03-21|242|1055|82|1920|0|3|21|1|1920|82|1055|Sunday|1920Q1|N|N|N|2422385|2422444|2422039|2422313|N|N|N|N|N| +2422406|AAAAAAAAGIGPECAA|1920-03-22|242|1055|82|1920|1|3|22|1|1920|82|1055|Monday|1920Q1|N|N|N|2422385|2422444|2422040|2422314|N|N|N|N|N| +2422407|AAAAAAAAHIGPECAA|1920-03-23|242|1056|82|1920|2|3|23|1|1920|82|1056|Tuesday|1920Q1|N|N|N|2422385|2422444|2422041|2422315|N|N|N|N|N| +2422408|AAAAAAAAIIGPECAA|1920-03-24|242|1056|82|1920|3|3|24|1|1920|82|1056|Wednesday|1920Q1|N|N|N|2422385|2422444|2422042|2422316|N|N|N|N|N| +2422409|AAAAAAAAJIGPECAA|1920-03-25|242|1056|82|1920|4|3|25|1|1920|82|1056|Thursday|1920Q1|N|N|N|2422385|2422444|2422043|2422317|N|N|N|N|N| +2422410|AAAAAAAAKIGPECAA|1920-03-26|242|1056|82|1920|5|3|26|1|1920|82|1056|Friday|1920Q1|N|Y|N|2422385|2422444|2422044|2422318|N|N|N|N|N| +2422411|AAAAAAAALIGPECAA|1920-03-27|242|1056|82|1920|6|3|27|1|1920|82|1056|Saturday|1920Q1|N|Y|N|2422385|2422444|2422045|2422319|N|N|N|N|N| +2422412|AAAAAAAAMIGPECAA|1920-03-28|242|1056|82|1920|0|3|28|1|1920|82|1056|Sunday|1920Q1|N|N|N|2422385|2422444|2422046|2422320|N|N|N|N|N| +2422413|AAAAAAAANIGPECAA|1920-03-29|242|1056|82|1920|1|3|29|1|1920|82|1056|Monday|1920Q1|N|N|N|2422385|2422444|2422047|2422321|N|N|N|N|N| +2422414|AAAAAAAAOIGPECAA|1920-03-30|242|1057|82|1920|2|3|30|1|1920|82|1057|Tuesday|1920Q1|N|N|N|2422385|2422444|2422048|2422322|N|N|N|N|N| +2422415|AAAAAAAAPIGPECAA|1920-03-31|242|1057|82|1920|3|3|31|1|1920|82|1057|Wednesday|1920Q1|N|N|N|2422385|2422444|2422049|2422323|N|N|N|N|N| +2422416|AAAAAAAAAJGPECAA|1920-04-01|243|1057|82|1920|4|4|1|2|1920|82|1057|Thursday|1920Q2|N|N|N|2422416|2422506|2422050|2422325|N|N|N|N|N| +2422417|AAAAAAAABJGPECAA|1920-04-02|243|1057|82|1920|5|4|2|2|1920|82|1057|Friday|1920Q2|N|Y|N|2422416|2422506|2422051|2422326|N|N|N|N|N| +2422418|AAAAAAAACJGPECAA|1920-04-03|243|1057|82|1920|6|4|3|2|1920|82|1057|Saturday|1920Q2|N|Y|N|2422416|2422506|2422052|2422327|N|N|N|N|N| +2422419|AAAAAAAADJGPECAA|1920-04-04|243|1057|82|1920|0|4|4|2|1920|82|1057|Sunday|1920Q2|N|N|N|2422416|2422506|2422053|2422328|N|N|N|N|N| +2422420|AAAAAAAAEJGPECAA|1920-04-05|243|1057|82|1920|1|4|5|2|1920|82|1057|Monday|1920Q2|N|N|N|2422416|2422506|2422054|2422329|N|N|N|N|N| +2422421|AAAAAAAAFJGPECAA|1920-04-06|243|1058|82|1920|2|4|6|2|1920|82|1058|Tuesday|1920Q2|N|N|N|2422416|2422506|2422055|2422330|N|N|N|N|N| +2422422|AAAAAAAAGJGPECAA|1920-04-07|243|1058|82|1920|3|4|7|2|1920|82|1058|Wednesday|1920Q2|N|N|N|2422416|2422506|2422056|2422331|N|N|N|N|N| +2422423|AAAAAAAAHJGPECAA|1920-04-08|243|1058|82|1920|4|4|8|2|1920|82|1058|Thursday|1920Q2|N|N|N|2422416|2422506|2422057|2422332|N|N|N|N|N| +2422424|AAAAAAAAIJGPECAA|1920-04-09|243|1058|82|1920|5|4|9|2|1920|82|1058|Friday|1920Q2|N|Y|N|2422416|2422506|2422058|2422333|N|N|N|N|N| +2422425|AAAAAAAAJJGPECAA|1920-04-10|243|1058|82|1920|6|4|10|2|1920|82|1058|Saturday|1920Q2|N|Y|N|2422416|2422506|2422059|2422334|N|N|N|N|N| +2422426|AAAAAAAAKJGPECAA|1920-04-11|243|1058|82|1920|0|4|11|2|1920|82|1058|Sunday|1920Q2|N|N|N|2422416|2422506|2422060|2422335|N|N|N|N|N| +2422427|AAAAAAAALJGPECAA|1920-04-12|243|1058|82|1920|1|4|12|2|1920|82|1058|Monday|1920Q2|N|N|N|2422416|2422506|2422061|2422336|N|N|N|N|N| +2422428|AAAAAAAAMJGPECAA|1920-04-13|243|1059|82|1920|2|4|13|2|1920|82|1059|Tuesday|1920Q2|N|N|N|2422416|2422506|2422062|2422337|N|N|N|N|N| +2422429|AAAAAAAANJGPECAA|1920-04-14|243|1059|82|1920|3|4|14|2|1920|82|1059|Wednesday|1920Q2|N|N|N|2422416|2422506|2422063|2422338|N|N|N|N|N| +2422430|AAAAAAAAOJGPECAA|1920-04-15|243|1059|82|1920|4|4|15|2|1920|82|1059|Thursday|1920Q2|N|N|N|2422416|2422506|2422064|2422339|N|N|N|N|N| +2422431|AAAAAAAAPJGPECAA|1920-04-16|243|1059|82|1920|5|4|16|2|1920|82|1059|Friday|1920Q2|N|Y|N|2422416|2422506|2422065|2422340|N|N|N|N|N| +2422432|AAAAAAAAAKGPECAA|1920-04-17|243|1059|82|1920|6|4|17|2|1920|82|1059|Saturday|1920Q2|N|Y|N|2422416|2422506|2422066|2422341|N|N|N|N|N| +2422433|AAAAAAAABKGPECAA|1920-04-18|243|1059|82|1920|0|4|18|2|1920|82|1059|Sunday|1920Q2|N|N|N|2422416|2422506|2422067|2422342|N|N|N|N|N| +2422434|AAAAAAAACKGPECAA|1920-04-19|243|1059|82|1920|1|4|19|2|1920|82|1059|Monday|1920Q2|N|N|N|2422416|2422506|2422068|2422343|N|N|N|N|N| +2422435|AAAAAAAADKGPECAA|1920-04-20|243|1060|82|1920|2|4|20|2|1920|82|1060|Tuesday|1920Q2|N|N|N|2422416|2422506|2422069|2422344|N|N|N|N|N| +2422436|AAAAAAAAEKGPECAA|1920-04-21|243|1060|82|1920|3|4|21|2|1920|82|1060|Wednesday|1920Q2|N|N|N|2422416|2422506|2422070|2422345|N|N|N|N|N| +2422437|AAAAAAAAFKGPECAA|1920-04-22|243|1060|82|1920|4|4|22|2|1920|82|1060|Thursday|1920Q2|N|N|N|2422416|2422506|2422071|2422346|N|N|N|N|N| +2422438|AAAAAAAAGKGPECAA|1920-04-23|243|1060|82|1920|5|4|23|2|1920|82|1060|Friday|1920Q2|N|Y|N|2422416|2422506|2422072|2422347|N|N|N|N|N| +2422439|AAAAAAAAHKGPECAA|1920-04-24|243|1060|82|1920|6|4|24|2|1920|82|1060|Saturday|1920Q2|N|Y|N|2422416|2422506|2422073|2422348|N|N|N|N|N| +2422440|AAAAAAAAIKGPECAA|1920-04-25|243|1060|82|1920|0|4|25|2|1920|82|1060|Sunday|1920Q2|N|N|N|2422416|2422506|2422074|2422349|N|N|N|N|N| +2422441|AAAAAAAAJKGPECAA|1920-04-26|243|1060|82|1920|1|4|26|2|1920|82|1060|Monday|1920Q2|N|N|N|2422416|2422506|2422075|2422350|N|N|N|N|N| +2422442|AAAAAAAAKKGPECAA|1920-04-27|243|1061|82|1920|2|4|27|2|1920|82|1061|Tuesday|1920Q2|N|N|N|2422416|2422506|2422076|2422351|N|N|N|N|N| +2422443|AAAAAAAALKGPECAA|1920-04-28|243|1061|82|1920|3|4|28|2|1920|82|1061|Wednesday|1920Q2|N|N|N|2422416|2422506|2422077|2422352|N|N|N|N|N| +2422444|AAAAAAAAMKGPECAA|1920-04-29|243|1061|82|1920|4|4|29|2|1920|82|1061|Thursday|1920Q2|N|N|N|2422416|2422506|2422078|2422353|N|N|N|N|N| +2422445|AAAAAAAANKGPECAA|1920-04-30|243|1061|82|1920|5|4|30|2|1920|82|1061|Friday|1920Q2|N|Y|N|2422416|2422506|2422079|2422354|N|N|N|N|N| +2422446|AAAAAAAAOKGPECAA|1920-05-01|244|1061|82|1920|6|5|1|2|1920|82|1061|Saturday|1920Q2|N|Y|N|2422446|2422566|2422080|2422355|N|N|N|N|N| +2422447|AAAAAAAAPKGPECAA|1920-05-02|244|1061|82|1920|0|5|2|2|1920|82|1061|Sunday|1920Q2|N|N|N|2422446|2422566|2422081|2422356|N|N|N|N|N| +2422448|AAAAAAAAALGPECAA|1920-05-03|244|1061|82|1920|1|5|3|2|1920|82|1061|Monday|1920Q2|N|N|N|2422446|2422566|2422082|2422357|N|N|N|N|N| +2422449|AAAAAAAABLGPECAA|1920-05-04|244|1062|82|1920|2|5|4|2|1920|82|1062|Tuesday|1920Q2|N|N|N|2422446|2422566|2422083|2422358|N|N|N|N|N| +2422450|AAAAAAAACLGPECAA|1920-05-05|244|1062|82|1920|3|5|5|2|1920|82|1062|Wednesday|1920Q2|N|N|N|2422446|2422566|2422084|2422359|N|N|N|N|N| +2422451|AAAAAAAADLGPECAA|1920-05-06|244|1062|82|1920|4|5|6|2|1920|82|1062|Thursday|1920Q2|N|N|N|2422446|2422566|2422085|2422360|N|N|N|N|N| +2422452|AAAAAAAAELGPECAA|1920-05-07|244|1062|82|1920|5|5|7|2|1920|82|1062|Friday|1920Q2|N|Y|N|2422446|2422566|2422086|2422361|N|N|N|N|N| +2422453|AAAAAAAAFLGPECAA|1920-05-08|244|1062|82|1920|6|5|8|2|1920|82|1062|Saturday|1920Q2|N|Y|N|2422446|2422566|2422087|2422362|N|N|N|N|N| +2422454|AAAAAAAAGLGPECAA|1920-05-09|244|1062|82|1920|0|5|9|2|1920|82|1062|Sunday|1920Q2|N|N|N|2422446|2422566|2422088|2422363|N|N|N|N|N| +2422455|AAAAAAAAHLGPECAA|1920-05-10|244|1062|82|1920|1|5|10|2|1920|82|1062|Monday|1920Q2|N|N|N|2422446|2422566|2422089|2422364|N|N|N|N|N| +2422456|AAAAAAAAILGPECAA|1920-05-11|244|1063|82|1920|2|5|11|2|1920|82|1063|Tuesday|1920Q2|N|N|N|2422446|2422566|2422090|2422365|N|N|N|N|N| +2422457|AAAAAAAAJLGPECAA|1920-05-12|244|1063|82|1920|3|5|12|2|1920|82|1063|Wednesday|1920Q2|N|N|N|2422446|2422566|2422091|2422366|N|N|N|N|N| +2422458|AAAAAAAAKLGPECAA|1920-05-13|244|1063|82|1920|4|5|13|2|1920|82|1063|Thursday|1920Q2|N|N|N|2422446|2422566|2422092|2422367|N|N|N|N|N| +2422459|AAAAAAAALLGPECAA|1920-05-14|244|1063|82|1920|5|5|14|2|1920|82|1063|Friday|1920Q2|N|Y|N|2422446|2422566|2422093|2422368|N|N|N|N|N| +2422460|AAAAAAAAMLGPECAA|1920-05-15|244|1063|82|1920|6|5|15|2|1920|82|1063|Saturday|1920Q2|N|Y|N|2422446|2422566|2422094|2422369|N|N|N|N|N| +2422461|AAAAAAAANLGPECAA|1920-05-16|244|1063|82|1920|0|5|16|2|1920|82|1063|Sunday|1920Q2|N|N|N|2422446|2422566|2422095|2422370|N|N|N|N|N| +2422462|AAAAAAAAOLGPECAA|1920-05-17|244|1063|82|1920|1|5|17|2|1920|82|1063|Monday|1920Q2|N|N|N|2422446|2422566|2422096|2422371|N|N|N|N|N| +2422463|AAAAAAAAPLGPECAA|1920-05-18|244|1064|82|1920|2|5|18|2|1920|82|1064|Tuesday|1920Q2|N|N|N|2422446|2422566|2422097|2422372|N|N|N|N|N| +2422464|AAAAAAAAAMGPECAA|1920-05-19|244|1064|82|1920|3|5|19|2|1920|82|1064|Wednesday|1920Q2|N|N|N|2422446|2422566|2422098|2422373|N|N|N|N|N| +2422465|AAAAAAAABMGPECAA|1920-05-20|244|1064|82|1920|4|5|20|2|1920|82|1064|Thursday|1920Q2|N|N|N|2422446|2422566|2422099|2422374|N|N|N|N|N| +2422466|AAAAAAAACMGPECAA|1920-05-21|244|1064|82|1920|5|5|21|2|1920|82|1064|Friday|1920Q2|N|Y|N|2422446|2422566|2422100|2422375|N|N|N|N|N| +2422467|AAAAAAAADMGPECAA|1920-05-22|244|1064|82|1920|6|5|22|2|1920|82|1064|Saturday|1920Q2|N|Y|N|2422446|2422566|2422101|2422376|N|N|N|N|N| +2422468|AAAAAAAAEMGPECAA|1920-05-23|244|1064|82|1920|0|5|23|2|1920|82|1064|Sunday|1920Q2|N|N|N|2422446|2422566|2422102|2422377|N|N|N|N|N| +2422469|AAAAAAAAFMGPECAA|1920-05-24|244|1064|82|1920|1|5|24|2|1920|82|1064|Monday|1920Q2|N|N|N|2422446|2422566|2422103|2422378|N|N|N|N|N| +2422470|AAAAAAAAGMGPECAA|1920-05-25|244|1065|82|1920|2|5|25|2|1920|82|1065|Tuesday|1920Q2|N|N|N|2422446|2422566|2422104|2422379|N|N|N|N|N| +2422471|AAAAAAAAHMGPECAA|1920-05-26|244|1065|82|1920|3|5|26|2|1920|82|1065|Wednesday|1920Q2|N|N|N|2422446|2422566|2422105|2422380|N|N|N|N|N| +2422472|AAAAAAAAIMGPECAA|1920-05-27|244|1065|82|1920|4|5|27|2|1920|82|1065|Thursday|1920Q2|N|N|N|2422446|2422566|2422106|2422381|N|N|N|N|N| +2422473|AAAAAAAAJMGPECAA|1920-05-28|244|1065|82|1920|5|5|28|2|1920|82|1065|Friday|1920Q2|N|Y|N|2422446|2422566|2422107|2422382|N|N|N|N|N| +2422474|AAAAAAAAKMGPECAA|1920-05-29|244|1065|82|1920|6|5|29|2|1920|82|1065|Saturday|1920Q2|N|Y|N|2422446|2422566|2422108|2422383|N|N|N|N|N| +2422475|AAAAAAAALMGPECAA|1920-05-30|244|1065|82|1920|0|5|30|2|1920|82|1065|Sunday|1920Q2|N|N|N|2422446|2422566|2422109|2422384|N|N|N|N|N| +2422476|AAAAAAAAMMGPECAA|1920-05-31|244|1065|82|1920|1|5|31|2|1920|82|1065|Monday|1920Q2|N|N|N|2422446|2422566|2422110|2422385|N|N|N|N|N| +2422477|AAAAAAAANMGPECAA|1920-06-01|245|1066|83|1920|2|6|1|2|1920|83|1066|Tuesday|1920Q2|N|N|N|2422477|2422628|2422111|2422386|N|N|N|N|N| +2422478|AAAAAAAAOMGPECAA|1920-06-02|245|1066|83|1920|3|6|2|2|1920|83|1066|Wednesday|1920Q2|N|N|N|2422477|2422628|2422112|2422387|N|N|N|N|N| +2422479|AAAAAAAAPMGPECAA|1920-06-03|245|1066|83|1920|4|6|3|2|1920|83|1066|Thursday|1920Q2|N|N|N|2422477|2422628|2422113|2422388|N|N|N|N|N| +2422480|AAAAAAAAANGPECAA|1920-06-04|245|1066|83|1920|5|6|4|2|1920|83|1066|Friday|1920Q2|N|Y|N|2422477|2422628|2422114|2422389|N|N|N|N|N| +2422481|AAAAAAAABNGPECAA|1920-06-05|245|1066|83|1920|6|6|5|2|1920|83|1066|Saturday|1920Q2|N|Y|N|2422477|2422628|2422115|2422390|N|N|N|N|N| +2422482|AAAAAAAACNGPECAA|1920-06-06|245|1066|83|1920|0|6|6|2|1920|83|1066|Sunday|1920Q2|N|N|N|2422477|2422628|2422116|2422391|N|N|N|N|N| +2422483|AAAAAAAADNGPECAA|1920-06-07|245|1066|83|1920|1|6|7|2|1920|83|1066|Monday|1920Q2|N|N|N|2422477|2422628|2422117|2422392|N|N|N|N|N| +2422484|AAAAAAAAENGPECAA|1920-06-08|245|1067|83|1920|2|6|8|2|1920|83|1067|Tuesday|1920Q2|N|N|N|2422477|2422628|2422118|2422393|N|N|N|N|N| +2422485|AAAAAAAAFNGPECAA|1920-06-09|245|1067|83|1920|3|6|9|2|1920|83|1067|Wednesday|1920Q2|N|N|N|2422477|2422628|2422119|2422394|N|N|N|N|N| +2422486|AAAAAAAAGNGPECAA|1920-06-10|245|1067|83|1920|4|6|10|2|1920|83|1067|Thursday|1920Q2|N|N|N|2422477|2422628|2422120|2422395|N|N|N|N|N| +2422487|AAAAAAAAHNGPECAA|1920-06-11|245|1067|83|1920|5|6|11|2|1920|83|1067|Friday|1920Q2|N|Y|N|2422477|2422628|2422121|2422396|N|N|N|N|N| +2422488|AAAAAAAAINGPECAA|1920-06-12|245|1067|83|1920|6|6|12|2|1920|83|1067|Saturday|1920Q2|N|Y|N|2422477|2422628|2422122|2422397|N|N|N|N|N| +2422489|AAAAAAAAJNGPECAA|1920-06-13|245|1067|83|1920|0|6|13|2|1920|83|1067|Sunday|1920Q2|N|N|N|2422477|2422628|2422123|2422398|N|N|N|N|N| +2422490|AAAAAAAAKNGPECAA|1920-06-14|245|1067|83|1920|1|6|14|2|1920|83|1067|Monday|1920Q2|N|N|N|2422477|2422628|2422124|2422399|N|N|N|N|N| +2422491|AAAAAAAALNGPECAA|1920-06-15|245|1068|83|1920|2|6|15|2|1920|83|1068|Tuesday|1920Q2|N|N|N|2422477|2422628|2422125|2422400|N|N|N|N|N| +2422492|AAAAAAAAMNGPECAA|1920-06-16|245|1068|83|1920|3|6|16|2|1920|83|1068|Wednesday|1920Q2|N|N|N|2422477|2422628|2422126|2422401|N|N|N|N|N| +2422493|AAAAAAAANNGPECAA|1920-06-17|245|1068|83|1920|4|6|17|2|1920|83|1068|Thursday|1920Q2|N|N|N|2422477|2422628|2422127|2422402|N|N|N|N|N| +2422494|AAAAAAAAONGPECAA|1920-06-18|245|1068|83|1920|5|6|18|2|1920|83|1068|Friday|1920Q2|N|Y|N|2422477|2422628|2422128|2422403|N|N|N|N|N| +2422495|AAAAAAAAPNGPECAA|1920-06-19|245|1068|83|1920|6|6|19|2|1920|83|1068|Saturday|1920Q2|N|Y|N|2422477|2422628|2422129|2422404|N|N|N|N|N| +2422496|AAAAAAAAAOGPECAA|1920-06-20|245|1068|83|1920|0|6|20|2|1920|83|1068|Sunday|1920Q2|N|N|N|2422477|2422628|2422130|2422405|N|N|N|N|N| +2422497|AAAAAAAABOGPECAA|1920-06-21|245|1068|83|1920|1|6|21|2|1920|83|1068|Monday|1920Q2|N|N|N|2422477|2422628|2422131|2422406|N|N|N|N|N| +2422498|AAAAAAAACOGPECAA|1920-06-22|245|1069|83|1920|2|6|22|2|1920|83|1069|Tuesday|1920Q2|N|N|N|2422477|2422628|2422132|2422407|N|N|N|N|N| +2422499|AAAAAAAADOGPECAA|1920-06-23|245|1069|83|1920|3|6|23|2|1920|83|1069|Wednesday|1920Q2|N|N|N|2422477|2422628|2422133|2422408|N|N|N|N|N| +2422500|AAAAAAAAEOGPECAA|1920-06-24|245|1069|83|1920|4|6|24|2|1920|83|1069|Thursday|1920Q2|N|N|N|2422477|2422628|2422134|2422409|N|N|N|N|N| +2422501|AAAAAAAAFOGPECAA|1920-06-25|245|1069|83|1920|5|6|25|2|1920|83|1069|Friday|1920Q2|N|Y|N|2422477|2422628|2422135|2422410|N|N|N|N|N| +2422502|AAAAAAAAGOGPECAA|1920-06-26|245|1069|83|1920|6|6|26|2|1920|83|1069|Saturday|1920Q2|N|Y|N|2422477|2422628|2422136|2422411|N|N|N|N|N| +2422503|AAAAAAAAHOGPECAA|1920-06-27|245|1069|83|1920|0|6|27|2|1920|83|1069|Sunday|1920Q2|N|N|N|2422477|2422628|2422137|2422412|N|N|N|N|N| +2422504|AAAAAAAAIOGPECAA|1920-06-28|245|1069|83|1920|1|6|28|2|1920|83|1069|Monday|1920Q2|N|N|N|2422477|2422628|2422138|2422413|N|N|N|N|N| +2422505|AAAAAAAAJOGPECAA|1920-06-29|245|1070|83|1920|2|6|29|2|1920|83|1070|Tuesday|1920Q2|N|N|N|2422477|2422628|2422139|2422414|N|N|N|N|N| +2422506|AAAAAAAAKOGPECAA|1920-06-30|245|1070|83|1920|3|6|30|2|1920|83|1070|Wednesday|1920Q2|N|N|N|2422477|2422628|2422140|2422415|N|N|N|N|N| +2422507|AAAAAAAALOGPECAA|1920-07-01|246|1070|83|1920|4|7|1|3|1920|83|1070|Thursday|1920Q3|N|N|N|2422507|2422688|2422141|2422416|N|N|N|N|N| +2422508|AAAAAAAAMOGPECAA|1920-07-02|246|1070|83|1920|5|7|2|3|1920|83|1070|Friday|1920Q3|N|Y|N|2422507|2422688|2422142|2422417|N|N|N|N|N| +2422509|AAAAAAAANOGPECAA|1920-07-03|246|1070|83|1920|6|7|3|3|1920|83|1070|Saturday|1920Q3|N|Y|N|2422507|2422688|2422143|2422418|N|N|N|N|N| +2422510|AAAAAAAAOOGPECAA|1920-07-04|246|1070|83|1920|0|7|4|3|1920|83|1070|Sunday|1920Q3|Y|N|N|2422507|2422688|2422144|2422419|N|N|N|N|N| +2422511|AAAAAAAAPOGPECAA|1920-07-05|246|1070|83|1920|1|7|5|3|1920|83|1070|Monday|1920Q3|N|N|Y|2422507|2422688|2422145|2422420|N|N|N|N|N| +2422512|AAAAAAAAAPGPECAA|1920-07-06|246|1071|83|1920|2|7|6|3|1920|83|1071|Tuesday|1920Q3|N|N|N|2422507|2422688|2422146|2422421|N|N|N|N|N| +2422513|AAAAAAAABPGPECAA|1920-07-07|246|1071|83|1920|3|7|7|3|1920|83|1071|Wednesday|1920Q3|N|N|N|2422507|2422688|2422147|2422422|N|N|N|N|N| +2422514|AAAAAAAACPGPECAA|1920-07-08|246|1071|83|1920|4|7|8|3|1920|83|1071|Thursday|1920Q3|N|N|N|2422507|2422688|2422148|2422423|N|N|N|N|N| +2422515|AAAAAAAADPGPECAA|1920-07-09|246|1071|83|1920|5|7|9|3|1920|83|1071|Friday|1920Q3|N|Y|N|2422507|2422688|2422149|2422424|N|N|N|N|N| +2422516|AAAAAAAAEPGPECAA|1920-07-10|246|1071|83|1920|6|7|10|3|1920|83|1071|Saturday|1920Q3|N|Y|N|2422507|2422688|2422150|2422425|N|N|N|N|N| +2422517|AAAAAAAAFPGPECAA|1920-07-11|246|1071|83|1920|0|7|11|3|1920|83|1071|Sunday|1920Q3|N|N|N|2422507|2422688|2422151|2422426|N|N|N|N|N| +2422518|AAAAAAAAGPGPECAA|1920-07-12|246|1071|83|1920|1|7|12|3|1920|83|1071|Monday|1920Q3|N|N|N|2422507|2422688|2422152|2422427|N|N|N|N|N| +2422519|AAAAAAAAHPGPECAA|1920-07-13|246|1072|83|1920|2|7|13|3|1920|83|1072|Tuesday|1920Q3|N|N|N|2422507|2422688|2422153|2422428|N|N|N|N|N| +2422520|AAAAAAAAIPGPECAA|1920-07-14|246|1072|83|1920|3|7|14|3|1920|83|1072|Wednesday|1920Q3|N|N|N|2422507|2422688|2422154|2422429|N|N|N|N|N| +2422521|AAAAAAAAJPGPECAA|1920-07-15|246|1072|83|1920|4|7|15|3|1920|83|1072|Thursday|1920Q3|N|N|N|2422507|2422688|2422155|2422430|N|N|N|N|N| +2422522|AAAAAAAAKPGPECAA|1920-07-16|246|1072|83|1920|5|7|16|3|1920|83|1072|Friday|1920Q3|N|Y|N|2422507|2422688|2422156|2422431|N|N|N|N|N| +2422523|AAAAAAAALPGPECAA|1920-07-17|246|1072|83|1920|6|7|17|3|1920|83|1072|Saturday|1920Q3|N|Y|N|2422507|2422688|2422157|2422432|N|N|N|N|N| +2422524|AAAAAAAAMPGPECAA|1920-07-18|246|1072|83|1920|0|7|18|3|1920|83|1072|Sunday|1920Q3|N|N|N|2422507|2422688|2422158|2422433|N|N|N|N|N| +2422525|AAAAAAAANPGPECAA|1920-07-19|246|1072|83|1920|1|7|19|3|1920|83|1072|Monday|1920Q3|N|N|N|2422507|2422688|2422159|2422434|N|N|N|N|N| +2422526|AAAAAAAAOPGPECAA|1920-07-20|246|1073|83|1920|2|7|20|3|1920|83|1073|Tuesday|1920Q3|N|N|N|2422507|2422688|2422160|2422435|N|N|N|N|N| +2422527|AAAAAAAAPPGPECAA|1920-07-21|246|1073|83|1920|3|7|21|3|1920|83|1073|Wednesday|1920Q3|N|N|N|2422507|2422688|2422161|2422436|N|N|N|N|N| +2422528|AAAAAAAAAAHPECAA|1920-07-22|246|1073|83|1920|4|7|22|3|1920|83|1073|Thursday|1920Q3|N|N|N|2422507|2422688|2422162|2422437|N|N|N|N|N| +2422529|AAAAAAAABAHPECAA|1920-07-23|246|1073|83|1920|5|7|23|3|1920|83|1073|Friday|1920Q3|N|Y|N|2422507|2422688|2422163|2422438|N|N|N|N|N| +2422530|AAAAAAAACAHPECAA|1920-07-24|246|1073|83|1920|6|7|24|3|1920|83|1073|Saturday|1920Q3|N|Y|N|2422507|2422688|2422164|2422439|N|N|N|N|N| +2422531|AAAAAAAADAHPECAA|1920-07-25|246|1073|83|1920|0|7|25|3|1920|83|1073|Sunday|1920Q3|N|N|N|2422507|2422688|2422165|2422440|N|N|N|N|N| +2422532|AAAAAAAAEAHPECAA|1920-07-26|246|1073|83|1920|1|7|26|3|1920|83|1073|Monday|1920Q3|N|N|N|2422507|2422688|2422166|2422441|N|N|N|N|N| +2422533|AAAAAAAAFAHPECAA|1920-07-27|246|1074|83|1920|2|7|27|3|1920|83|1074|Tuesday|1920Q3|N|N|N|2422507|2422688|2422167|2422442|N|N|N|N|N| +2422534|AAAAAAAAGAHPECAA|1920-07-28|246|1074|83|1920|3|7|28|3|1920|83|1074|Wednesday|1920Q3|N|N|N|2422507|2422688|2422168|2422443|N|N|N|N|N| +2422535|AAAAAAAAHAHPECAA|1920-07-29|246|1074|83|1920|4|7|29|3|1920|83|1074|Thursday|1920Q3|N|N|N|2422507|2422688|2422169|2422444|N|N|N|N|N| +2422536|AAAAAAAAIAHPECAA|1920-07-30|246|1074|83|1920|5|7|30|3|1920|83|1074|Friday|1920Q3|N|Y|N|2422507|2422688|2422170|2422445|N|N|N|N|N| +2422537|AAAAAAAAJAHPECAA|1920-07-31|246|1074|83|1920|6|7|31|3|1920|83|1074|Saturday|1920Q3|N|Y|N|2422507|2422688|2422171|2422446|N|N|N|N|N| +2422538|AAAAAAAAKAHPECAA|1920-08-01|247|1074|83|1920|0|8|1|3|1920|83|1074|Sunday|1920Q3|N|N|N|2422538|2422750|2422172|2422447|N|N|N|N|N| +2422539|AAAAAAAALAHPECAA|1920-08-02|247|1074|83|1920|1|8|2|3|1920|83|1074|Monday|1920Q3|N|N|N|2422538|2422750|2422173|2422448|N|N|N|N|N| +2422540|AAAAAAAAMAHPECAA|1920-08-03|247|1075|83|1920|2|8|3|3|1920|83|1075|Tuesday|1920Q3|N|N|N|2422538|2422750|2422174|2422449|N|N|N|N|N| +2422541|AAAAAAAANAHPECAA|1920-08-04|247|1075|83|1920|3|8|4|3|1920|83|1075|Wednesday|1920Q3|N|N|N|2422538|2422750|2422175|2422450|N|N|N|N|N| +2422542|AAAAAAAAOAHPECAA|1920-08-05|247|1075|83|1920|4|8|5|3|1920|83|1075|Thursday|1920Q3|N|N|N|2422538|2422750|2422176|2422451|N|N|N|N|N| +2422543|AAAAAAAAPAHPECAA|1920-08-06|247|1075|83|1920|5|8|6|3|1920|83|1075|Friday|1920Q3|N|Y|N|2422538|2422750|2422177|2422452|N|N|N|N|N| +2422544|AAAAAAAAABHPECAA|1920-08-07|247|1075|83|1920|6|8|7|3|1920|83|1075|Saturday|1920Q3|N|Y|N|2422538|2422750|2422178|2422453|N|N|N|N|N| +2422545|AAAAAAAABBHPECAA|1920-08-08|247|1075|83|1920|0|8|8|3|1920|83|1075|Sunday|1920Q3|N|N|N|2422538|2422750|2422179|2422454|N|N|N|N|N| +2422546|AAAAAAAACBHPECAA|1920-08-09|247|1075|83|1920|1|8|9|3|1920|83|1075|Monday|1920Q3|N|N|N|2422538|2422750|2422180|2422455|N|N|N|N|N| +2422547|AAAAAAAADBHPECAA|1920-08-10|247|1076|83|1920|2|8|10|3|1920|83|1076|Tuesday|1920Q3|N|N|N|2422538|2422750|2422181|2422456|N|N|N|N|N| +2422548|AAAAAAAAEBHPECAA|1920-08-11|247|1076|83|1920|3|8|11|3|1920|83|1076|Wednesday|1920Q3|N|N|N|2422538|2422750|2422182|2422457|N|N|N|N|N| +2422549|AAAAAAAAFBHPECAA|1920-08-12|247|1076|83|1920|4|8|12|3|1920|83|1076|Thursday|1920Q3|N|N|N|2422538|2422750|2422183|2422458|N|N|N|N|N| +2422550|AAAAAAAAGBHPECAA|1920-08-13|247|1076|83|1920|5|8|13|3|1920|83|1076|Friday|1920Q3|N|Y|N|2422538|2422750|2422184|2422459|N|N|N|N|N| +2422551|AAAAAAAAHBHPECAA|1920-08-14|247|1076|83|1920|6|8|14|3|1920|83|1076|Saturday|1920Q3|N|Y|N|2422538|2422750|2422185|2422460|N|N|N|N|N| +2422552|AAAAAAAAIBHPECAA|1920-08-15|247|1076|83|1920|0|8|15|3|1920|83|1076|Sunday|1920Q3|N|N|N|2422538|2422750|2422186|2422461|N|N|N|N|N| +2422553|AAAAAAAAJBHPECAA|1920-08-16|247|1076|83|1920|1|8|16|3|1920|83|1076|Monday|1920Q3|N|N|N|2422538|2422750|2422187|2422462|N|N|N|N|N| +2422554|AAAAAAAAKBHPECAA|1920-08-17|247|1077|83|1920|2|8|17|3|1920|83|1077|Tuesday|1920Q3|N|N|N|2422538|2422750|2422188|2422463|N|N|N|N|N| +2422555|AAAAAAAALBHPECAA|1920-08-18|247|1077|83|1920|3|8|18|3|1920|83|1077|Wednesday|1920Q3|N|N|N|2422538|2422750|2422189|2422464|N|N|N|N|N| +2422556|AAAAAAAAMBHPECAA|1920-08-19|247|1077|83|1920|4|8|19|3|1920|83|1077|Thursday|1920Q3|N|N|N|2422538|2422750|2422190|2422465|N|N|N|N|N| +2422557|AAAAAAAANBHPECAA|1920-08-20|247|1077|83|1920|5|8|20|3|1920|83|1077|Friday|1920Q3|N|Y|N|2422538|2422750|2422191|2422466|N|N|N|N|N| +2422558|AAAAAAAAOBHPECAA|1920-08-21|247|1077|83|1920|6|8|21|3|1920|83|1077|Saturday|1920Q3|N|Y|N|2422538|2422750|2422192|2422467|N|N|N|N|N| +2422559|AAAAAAAAPBHPECAA|1920-08-22|247|1077|83|1920|0|8|22|3|1920|83|1077|Sunday|1920Q3|N|N|N|2422538|2422750|2422193|2422468|N|N|N|N|N| +2422560|AAAAAAAAACHPECAA|1920-08-23|247|1077|83|1920|1|8|23|3|1920|83|1077|Monday|1920Q3|N|N|N|2422538|2422750|2422194|2422469|N|N|N|N|N| +2422561|AAAAAAAABCHPECAA|1920-08-24|247|1078|83|1920|2|8|24|3|1920|83|1078|Tuesday|1920Q3|N|N|N|2422538|2422750|2422195|2422470|N|N|N|N|N| +2422562|AAAAAAAACCHPECAA|1920-08-25|247|1078|83|1920|3|8|25|3|1920|83|1078|Wednesday|1920Q3|N|N|N|2422538|2422750|2422196|2422471|N|N|N|N|N| +2422563|AAAAAAAADCHPECAA|1920-08-26|247|1078|83|1920|4|8|26|3|1920|83|1078|Thursday|1920Q3|N|N|N|2422538|2422750|2422197|2422472|N|N|N|N|N| +2422564|AAAAAAAAECHPECAA|1920-08-27|247|1078|83|1920|5|8|27|3|1920|83|1078|Friday|1920Q3|N|Y|N|2422538|2422750|2422198|2422473|N|N|N|N|N| +2422565|AAAAAAAAFCHPECAA|1920-08-28|247|1078|83|1920|6|8|28|3|1920|83|1078|Saturday|1920Q3|N|Y|N|2422538|2422750|2422199|2422474|N|N|N|N|N| +2422566|AAAAAAAAGCHPECAA|1920-08-29|247|1078|83|1920|0|8|29|3|1920|83|1078|Sunday|1920Q3|N|N|N|2422538|2422750|2422200|2422475|N|N|N|N|N| +2422567|AAAAAAAAHCHPECAA|1920-08-30|247|1078|83|1920|1|8|30|3|1920|83|1078|Monday|1920Q3|N|N|N|2422538|2422750|2422201|2422476|N|N|N|N|N| +2422568|AAAAAAAAICHPECAA|1920-08-31|247|1079|83|1920|2|8|31|3|1920|83|1079|Tuesday|1920Q3|N|N|N|2422538|2422750|2422202|2422477|N|N|N|N|N| +2422569|AAAAAAAAJCHPECAA|1920-09-01|248|1079|84|1920|3|9|1|3|1920|84|1079|Wednesday|1920Q3|N|N|N|2422569|2422812|2422203|2422478|N|N|N|N|N| +2422570|AAAAAAAAKCHPECAA|1920-09-02|248|1079|84|1920|4|9|2|3|1920|84|1079|Thursday|1920Q3|N|N|N|2422569|2422812|2422204|2422479|N|N|N|N|N| +2422571|AAAAAAAALCHPECAA|1920-09-03|248|1079|84|1920|5|9|3|3|1920|84|1079|Friday|1920Q3|N|Y|N|2422569|2422812|2422205|2422480|N|N|N|N|N| +2422572|AAAAAAAAMCHPECAA|1920-09-04|248|1079|84|1920|6|9|4|3|1920|84|1079|Saturday|1920Q3|N|Y|N|2422569|2422812|2422206|2422481|N|N|N|N|N| +2422573|AAAAAAAANCHPECAA|1920-09-05|248|1079|84|1920|0|9|5|3|1920|84|1079|Sunday|1920Q3|N|N|N|2422569|2422812|2422207|2422482|N|N|N|N|N| +2422574|AAAAAAAAOCHPECAA|1920-09-06|248|1079|84|1920|1|9|6|3|1920|84|1079|Monday|1920Q3|N|N|N|2422569|2422812|2422208|2422483|N|N|N|N|N| +2422575|AAAAAAAAPCHPECAA|1920-09-07|248|1080|84|1920|2|9|7|3|1920|84|1080|Tuesday|1920Q3|N|N|N|2422569|2422812|2422209|2422484|N|N|N|N|N| +2422576|AAAAAAAAADHPECAA|1920-09-08|248|1080|84|1920|3|9|8|3|1920|84|1080|Wednesday|1920Q3|N|N|N|2422569|2422812|2422210|2422485|N|N|N|N|N| +2422577|AAAAAAAABDHPECAA|1920-09-09|248|1080|84|1920|4|9|9|3|1920|84|1080|Thursday|1920Q3|N|N|N|2422569|2422812|2422211|2422486|N|N|N|N|N| +2422578|AAAAAAAACDHPECAA|1920-09-10|248|1080|84|1920|5|9|10|3|1920|84|1080|Friday|1920Q3|N|Y|N|2422569|2422812|2422212|2422487|N|N|N|N|N| +2422579|AAAAAAAADDHPECAA|1920-09-11|248|1080|84|1920|6|9|11|3|1920|84|1080|Saturday|1920Q3|N|Y|N|2422569|2422812|2422213|2422488|N|N|N|N|N| +2422580|AAAAAAAAEDHPECAA|1920-09-12|248|1080|84|1920|0|9|12|3|1920|84|1080|Sunday|1920Q3|N|N|N|2422569|2422812|2422214|2422489|N|N|N|N|N| +2422581|AAAAAAAAFDHPECAA|1920-09-13|248|1080|84|1920|1|9|13|3|1920|84|1080|Monday|1920Q3|N|N|N|2422569|2422812|2422215|2422490|N|N|N|N|N| +2422582|AAAAAAAAGDHPECAA|1920-09-14|248|1081|84|1920|2|9|14|3|1920|84|1081|Tuesday|1920Q3|N|N|N|2422569|2422812|2422216|2422491|N|N|N|N|N| +2422583|AAAAAAAAHDHPECAA|1920-09-15|248|1081|84|1920|3|9|15|3|1920|84|1081|Wednesday|1920Q3|N|N|N|2422569|2422812|2422217|2422492|N|N|N|N|N| +2422584|AAAAAAAAIDHPECAA|1920-09-16|248|1081|84|1920|4|9|16|3|1920|84|1081|Thursday|1920Q3|N|N|N|2422569|2422812|2422218|2422493|N|N|N|N|N| +2422585|AAAAAAAAJDHPECAA|1920-09-17|248|1081|84|1920|5|9|17|3|1920|84|1081|Friday|1920Q3|N|Y|N|2422569|2422812|2422219|2422494|N|N|N|N|N| +2422586|AAAAAAAAKDHPECAA|1920-09-18|248|1081|84|1920|6|9|18|3|1920|84|1081|Saturday|1920Q3|N|Y|N|2422569|2422812|2422220|2422495|N|N|N|N|N| +2422587|AAAAAAAALDHPECAA|1920-09-19|248|1081|84|1920|0|9|19|3|1920|84|1081|Sunday|1920Q3|N|N|N|2422569|2422812|2422221|2422496|N|N|N|N|N| +2422588|AAAAAAAAMDHPECAA|1920-09-20|248|1081|84|1920|1|9|20|3|1920|84|1081|Monday|1920Q3|N|N|N|2422569|2422812|2422222|2422497|N|N|N|N|N| +2422589|AAAAAAAANDHPECAA|1920-09-21|248|1082|84|1920|2|9|21|3|1920|84|1082|Tuesday|1920Q3|N|N|N|2422569|2422812|2422223|2422498|N|N|N|N|N| +2422590|AAAAAAAAODHPECAA|1920-09-22|248|1082|84|1920|3|9|22|3|1920|84|1082|Wednesday|1920Q3|N|N|N|2422569|2422812|2422224|2422499|N|N|N|N|N| +2422591|AAAAAAAAPDHPECAA|1920-09-23|248|1082|84|1920|4|9|23|3|1920|84|1082|Thursday|1920Q3|N|N|N|2422569|2422812|2422225|2422500|N|N|N|N|N| +2422592|AAAAAAAAAEHPECAA|1920-09-24|248|1082|84|1920|5|9|24|3|1920|84|1082|Friday|1920Q3|N|Y|N|2422569|2422812|2422226|2422501|N|N|N|N|N| +2422593|AAAAAAAABEHPECAA|1920-09-25|248|1082|84|1920|6|9|25|3|1920|84|1082|Saturday|1920Q3|N|Y|N|2422569|2422812|2422227|2422502|N|N|N|N|N| +2422594|AAAAAAAACEHPECAA|1920-09-26|248|1082|84|1920|0|9|26|3|1920|84|1082|Sunday|1920Q3|N|N|N|2422569|2422812|2422228|2422503|N|N|N|N|N| +2422595|AAAAAAAADEHPECAA|1920-09-27|248|1082|84|1920|1|9|27|3|1920|84|1082|Monday|1920Q3|N|N|N|2422569|2422812|2422229|2422504|N|N|N|N|N| +2422596|AAAAAAAAEEHPECAA|1920-09-28|248|1083|84|1920|2|9|28|3|1920|84|1083|Tuesday|1920Q3|N|N|N|2422569|2422812|2422230|2422505|N|N|N|N|N| +2422597|AAAAAAAAFEHPECAA|1920-09-29|248|1083|84|1920|3|9|29|3|1920|84|1083|Wednesday|1920Q3|N|N|N|2422569|2422812|2422231|2422506|N|N|N|N|N| +2422598|AAAAAAAAGEHPECAA|1920-09-30|248|1083|84|1920|4|9|30|3|1920|84|1083|Thursday|1920Q3|N|N|N|2422569|2422812|2422232|2422507|N|N|N|N|N| +2422599|AAAAAAAAHEHPECAA|1920-10-01|249|1083|84|1920|5|10|1|4|1920|84|1083|Friday|1920Q4|N|Y|N|2422599|2422872|2422233|2422507|N|N|N|N|N| +2422600|AAAAAAAAIEHPECAA|1920-10-02|249|1083|84|1920|6|10|2|4|1920|84|1083|Saturday|1920Q4|N|Y|N|2422599|2422872|2422234|2422508|N|N|N|N|N| +2422601|AAAAAAAAJEHPECAA|1920-10-03|249|1083|84|1920|0|10|3|4|1920|84|1083|Sunday|1920Q4|N|N|N|2422599|2422872|2422235|2422509|N|N|N|N|N| +2422602|AAAAAAAAKEHPECAA|1920-10-04|249|1083|84|1920|1|10|4|4|1920|84|1083|Monday|1920Q4|N|N|N|2422599|2422872|2422236|2422510|N|N|N|N|N| +2422603|AAAAAAAALEHPECAA|1920-10-05|249|1084|84|1920|2|10|5|4|1920|84|1084|Tuesday|1920Q4|N|N|N|2422599|2422872|2422237|2422511|N|N|N|N|N| +2422604|AAAAAAAAMEHPECAA|1920-10-06|249|1084|84|1920|3|10|6|4|1920|84|1084|Wednesday|1920Q4|N|N|N|2422599|2422872|2422238|2422512|N|N|N|N|N| +2422605|AAAAAAAANEHPECAA|1920-10-07|249|1084|84|1920|4|10|7|4|1920|84|1084|Thursday|1920Q4|N|N|N|2422599|2422872|2422239|2422513|N|N|N|N|N| +2422606|AAAAAAAAOEHPECAA|1920-10-08|249|1084|84|1920|5|10|8|4|1920|84|1084|Friday|1920Q4|N|Y|N|2422599|2422872|2422240|2422514|N|N|N|N|N| +2422607|AAAAAAAAPEHPECAA|1920-10-09|249|1084|84|1920|6|10|9|4|1920|84|1084|Saturday|1920Q4|N|Y|N|2422599|2422872|2422241|2422515|N|N|N|N|N| +2422608|AAAAAAAAAFHPECAA|1920-10-10|249|1084|84|1920|0|10|10|4|1920|84|1084|Sunday|1920Q4|N|N|N|2422599|2422872|2422242|2422516|N|N|N|N|N| +2422609|AAAAAAAABFHPECAA|1920-10-11|249|1084|84|1920|1|10|11|4|1920|84|1084|Monday|1920Q4|N|N|N|2422599|2422872|2422243|2422517|N|N|N|N|N| +2422610|AAAAAAAACFHPECAA|1920-10-12|249|1085|84|1920|2|10|12|4|1920|84|1085|Tuesday|1920Q4|N|N|N|2422599|2422872|2422244|2422518|N|N|N|N|N| +2422611|AAAAAAAADFHPECAA|1920-10-13|249|1085|84|1920|3|10|13|4|1920|84|1085|Wednesday|1920Q4|N|N|N|2422599|2422872|2422245|2422519|N|N|N|N|N| +2422612|AAAAAAAAEFHPECAA|1920-10-14|249|1085|84|1920|4|10|14|4|1920|84|1085|Thursday|1920Q4|N|N|N|2422599|2422872|2422246|2422520|N|N|N|N|N| +2422613|AAAAAAAAFFHPECAA|1920-10-15|249|1085|84|1920|5|10|15|4|1920|84|1085|Friday|1920Q4|N|Y|N|2422599|2422872|2422247|2422521|N|N|N|N|N| +2422614|AAAAAAAAGFHPECAA|1920-10-16|249|1085|84|1920|6|10|16|4|1920|84|1085|Saturday|1920Q4|N|Y|N|2422599|2422872|2422248|2422522|N|N|N|N|N| +2422615|AAAAAAAAHFHPECAA|1920-10-17|249|1085|84|1920|0|10|17|4|1920|84|1085|Sunday|1920Q4|N|N|N|2422599|2422872|2422249|2422523|N|N|N|N|N| +2422616|AAAAAAAAIFHPECAA|1920-10-18|249|1085|84|1920|1|10|18|4|1920|84|1085|Monday|1920Q4|N|N|N|2422599|2422872|2422250|2422524|N|N|N|N|N| +2422617|AAAAAAAAJFHPECAA|1920-10-19|249|1086|84|1920|2|10|19|4|1920|84|1086|Tuesday|1920Q4|N|N|N|2422599|2422872|2422251|2422525|N|N|N|N|N| +2422618|AAAAAAAAKFHPECAA|1920-10-20|249|1086|84|1920|3|10|20|4|1920|84|1086|Wednesday|1920Q4|N|N|N|2422599|2422872|2422252|2422526|N|N|N|N|N| +2422619|AAAAAAAALFHPECAA|1920-10-21|249|1086|84|1920|4|10|21|4|1920|84|1086|Thursday|1920Q4|N|N|N|2422599|2422872|2422253|2422527|N|N|N|N|N| +2422620|AAAAAAAAMFHPECAA|1920-10-22|249|1086|84|1920|5|10|22|4|1920|84|1086|Friday|1920Q4|N|Y|N|2422599|2422872|2422254|2422528|N|N|N|N|N| +2422621|AAAAAAAANFHPECAA|1920-10-23|249|1086|84|1920|6|10|23|4|1920|84|1086|Saturday|1920Q4|N|Y|N|2422599|2422872|2422255|2422529|N|N|N|N|N| +2422622|AAAAAAAAOFHPECAA|1920-10-24|249|1086|84|1920|0|10|24|4|1920|84|1086|Sunday|1920Q4|N|N|N|2422599|2422872|2422256|2422530|N|N|N|N|N| +2422623|AAAAAAAAPFHPECAA|1920-10-25|249|1086|84|1920|1|10|25|4|1920|84|1086|Monday|1920Q4|N|N|N|2422599|2422872|2422257|2422531|N|N|N|N|N| +2422624|AAAAAAAAAGHPECAA|1920-10-26|249|1087|84|1920|2|10|26|4|1920|84|1087|Tuesday|1920Q4|N|N|N|2422599|2422872|2422258|2422532|N|N|N|N|N| +2422625|AAAAAAAABGHPECAA|1920-10-27|249|1087|84|1920|3|10|27|4|1920|84|1087|Wednesday|1920Q4|N|N|N|2422599|2422872|2422259|2422533|N|N|N|N|N| +2422626|AAAAAAAACGHPECAA|1920-10-28|249|1087|84|1920|4|10|28|4|1920|84|1087|Thursday|1920Q4|N|N|N|2422599|2422872|2422260|2422534|N|N|N|N|N| +2422627|AAAAAAAADGHPECAA|1920-10-29|249|1087|84|1920|5|10|29|4|1920|84|1087|Friday|1920Q4|N|Y|N|2422599|2422872|2422261|2422535|N|N|N|N|N| +2422628|AAAAAAAAEGHPECAA|1920-10-30|249|1087|84|1920|6|10|30|4|1920|84|1087|Saturday|1920Q4|N|Y|N|2422599|2422872|2422262|2422536|N|N|N|N|N| +2422629|AAAAAAAAFGHPECAA|1920-10-31|249|1087|84|1920|0|10|31|4|1920|84|1087|Sunday|1920Q4|N|N|N|2422599|2422872|2422263|2422537|N|N|N|N|N| +2422630|AAAAAAAAGGHPECAA|1920-11-01|250|1087|84|1920|1|11|1|4|1920|84|1087|Monday|1920Q4|N|N|N|2422630|2422934|2422264|2422538|N|N|N|N|N| +2422631|AAAAAAAAHGHPECAA|1920-11-02|250|1088|84|1920|2|11|2|4|1920|84|1088|Tuesday|1920Q4|N|N|N|2422630|2422934|2422265|2422539|N|N|N|N|N| +2422632|AAAAAAAAIGHPECAA|1920-11-03|250|1088|84|1920|3|11|3|4|1920|84|1088|Wednesday|1920Q4|N|N|N|2422630|2422934|2422266|2422540|N|N|N|N|N| +2422633|AAAAAAAAJGHPECAA|1920-11-04|250|1088|84|1920|4|11|4|4|1920|84|1088|Thursday|1920Q4|N|N|N|2422630|2422934|2422267|2422541|N|N|N|N|N| +2422634|AAAAAAAAKGHPECAA|1920-11-05|250|1088|84|1920|5|11|5|4|1920|84|1088|Friday|1920Q4|N|Y|N|2422630|2422934|2422268|2422542|N|N|N|N|N| +2422635|AAAAAAAALGHPECAA|1920-11-06|250|1088|84|1920|6|11|6|4|1920|84|1088|Saturday|1920Q4|N|Y|N|2422630|2422934|2422269|2422543|N|N|N|N|N| +2422636|AAAAAAAAMGHPECAA|1920-11-07|250|1088|84|1920|0|11|7|4|1920|84|1088|Sunday|1920Q4|N|N|N|2422630|2422934|2422270|2422544|N|N|N|N|N| +2422637|AAAAAAAANGHPECAA|1920-11-08|250|1088|84|1920|1|11|8|4|1920|84|1088|Monday|1920Q4|N|N|N|2422630|2422934|2422271|2422545|N|N|N|N|N| +2422638|AAAAAAAAOGHPECAA|1920-11-09|250|1089|84|1920|2|11|9|4|1920|84|1089|Tuesday|1920Q4|N|N|N|2422630|2422934|2422272|2422546|N|N|N|N|N| +2422639|AAAAAAAAPGHPECAA|1920-11-10|250|1089|84|1920|3|11|10|4|1920|84|1089|Wednesday|1920Q4|N|N|N|2422630|2422934|2422273|2422547|N|N|N|N|N| +2422640|AAAAAAAAAHHPECAA|1920-11-11|250|1089|84|1920|4|11|11|4|1920|84|1089|Thursday|1920Q4|N|N|N|2422630|2422934|2422274|2422548|N|N|N|N|N| +2422641|AAAAAAAABHHPECAA|1920-11-12|250|1089|84|1920|5|11|12|4|1920|84|1089|Friday|1920Q4|N|Y|N|2422630|2422934|2422275|2422549|N|N|N|N|N| +2422642|AAAAAAAACHHPECAA|1920-11-13|250|1089|84|1920|6|11|13|4|1920|84|1089|Saturday|1920Q4|N|Y|N|2422630|2422934|2422276|2422550|N|N|N|N|N| +2422643|AAAAAAAADHHPECAA|1920-11-14|250|1089|84|1920|0|11|14|4|1920|84|1089|Sunday|1920Q4|N|N|N|2422630|2422934|2422277|2422551|N|N|N|N|N| +2422644|AAAAAAAAEHHPECAA|1920-11-15|250|1089|84|1920|1|11|15|4|1920|84|1089|Monday|1920Q4|N|N|N|2422630|2422934|2422278|2422552|N|N|N|N|N| +2422645|AAAAAAAAFHHPECAA|1920-11-16|250|1090|84|1920|2|11|16|4|1920|84|1090|Tuesday|1920Q4|N|N|N|2422630|2422934|2422279|2422553|N|N|N|N|N| +2422646|AAAAAAAAGHHPECAA|1920-11-17|250|1090|84|1920|3|11|17|4|1920|84|1090|Wednesday|1920Q4|N|N|N|2422630|2422934|2422280|2422554|N|N|N|N|N| +2422647|AAAAAAAAHHHPECAA|1920-11-18|250|1090|84|1920|4|11|18|4|1920|84|1090|Thursday|1920Q4|N|N|N|2422630|2422934|2422281|2422555|N|N|N|N|N| +2422648|AAAAAAAAIHHPECAA|1920-11-19|250|1090|84|1920|5|11|19|4|1920|84|1090|Friday|1920Q4|N|Y|N|2422630|2422934|2422282|2422556|N|N|N|N|N| +2422649|AAAAAAAAJHHPECAA|1920-11-20|250|1090|84|1920|6|11|20|4|1920|84|1090|Saturday|1920Q4|N|Y|N|2422630|2422934|2422283|2422557|N|N|N|N|N| +2422650|AAAAAAAAKHHPECAA|1920-11-21|250|1090|84|1920|0|11|21|4|1920|84|1090|Sunday|1920Q4|N|N|N|2422630|2422934|2422284|2422558|N|N|N|N|N| +2422651|AAAAAAAALHHPECAA|1920-11-22|250|1090|84|1920|1|11|22|4|1920|84|1090|Monday|1920Q4|N|N|N|2422630|2422934|2422285|2422559|N|N|N|N|N| +2422652|AAAAAAAAMHHPECAA|1920-11-23|250|1091|84|1920|2|11|23|4|1920|84|1091|Tuesday|1920Q4|N|N|N|2422630|2422934|2422286|2422560|N|N|N|N|N| +2422653|AAAAAAAANHHPECAA|1920-11-24|250|1091|84|1920|3|11|24|4|1920|84|1091|Wednesday|1920Q4|N|N|N|2422630|2422934|2422287|2422561|N|N|N|N|N| +2422654|AAAAAAAAOHHPECAA|1920-11-25|250|1091|84|1920|4|11|25|4|1920|84|1091|Thursday|1920Q4|N|N|N|2422630|2422934|2422288|2422562|N|N|N|N|N| +2422655|AAAAAAAAPHHPECAA|1920-11-26|250|1091|84|1920|5|11|26|4|1920|84|1091|Friday|1920Q4|N|Y|N|2422630|2422934|2422289|2422563|N|N|N|N|N| +2422656|AAAAAAAAAIHPECAA|1920-11-27|250|1091|84|1920|6|11|27|4|1920|84|1091|Saturday|1920Q4|N|Y|N|2422630|2422934|2422290|2422564|N|N|N|N|N| +2422657|AAAAAAAABIHPECAA|1920-11-28|250|1091|84|1920|0|11|28|4|1920|84|1091|Sunday|1920Q4|N|N|N|2422630|2422934|2422291|2422565|N|N|N|N|N| +2422658|AAAAAAAACIHPECAA|1920-11-29|250|1091|84|1920|1|11|29|4|1920|84|1091|Monday|1920Q4|N|N|N|2422630|2422934|2422292|2422566|N|N|N|N|N| +2422659|AAAAAAAADIHPECAA|1920-11-30|250|1092|84|1920|2|11|30|4|1920|84|1092|Tuesday|1920Q4|N|N|N|2422630|2422934|2422293|2422567|N|N|N|N|N| +2422660|AAAAAAAAEIHPECAA|1920-12-01|251|1092|85|1920|3|12|1|4|1920|85|1092|Wednesday|1920Q4|N|N|N|2422660|2422994|2422294|2422568|N|N|N|N|N| +2422661|AAAAAAAAFIHPECAA|1920-12-02|251|1092|85|1920|4|12|2|4|1920|85|1092|Thursday|1920Q4|N|N|N|2422660|2422994|2422295|2422569|N|N|N|N|N| +2422662|AAAAAAAAGIHPECAA|1920-12-03|251|1092|85|1920|5|12|3|4|1920|85|1092|Friday|1920Q4|N|Y|N|2422660|2422994|2422296|2422570|N|N|N|N|N| +2422663|AAAAAAAAHIHPECAA|1920-12-04|251|1092|85|1920|6|12|4|4|1920|85|1092|Saturday|1920Q4|N|Y|N|2422660|2422994|2422297|2422571|N|N|N|N|N| +2422664|AAAAAAAAIIHPECAA|1920-12-05|251|1092|85|1920|0|12|5|4|1920|85|1092|Sunday|1920Q4|N|N|N|2422660|2422994|2422298|2422572|N|N|N|N|N| +2422665|AAAAAAAAJIHPECAA|1920-12-06|251|1092|85|1920|1|12|6|4|1920|85|1092|Monday|1920Q4|N|N|N|2422660|2422994|2422299|2422573|N|N|N|N|N| +2422666|AAAAAAAAKIHPECAA|1920-12-07|251|1093|85|1920|2|12|7|4|1920|85|1093|Tuesday|1920Q4|N|N|N|2422660|2422994|2422300|2422574|N|N|N|N|N| +2422667|AAAAAAAALIHPECAA|1920-12-08|251|1093|85|1920|3|12|8|4|1920|85|1093|Wednesday|1920Q4|N|N|N|2422660|2422994|2422301|2422575|N|N|N|N|N| +2422668|AAAAAAAAMIHPECAA|1920-12-09|251|1093|85|1920|4|12|9|4|1920|85|1093|Thursday|1920Q4|N|N|N|2422660|2422994|2422302|2422576|N|N|N|N|N| +2422669|AAAAAAAANIHPECAA|1920-12-10|251|1093|85|1920|5|12|10|4|1920|85|1093|Friday|1920Q4|N|Y|N|2422660|2422994|2422303|2422577|N|N|N|N|N| +2422670|AAAAAAAAOIHPECAA|1920-12-11|251|1093|85|1920|6|12|11|4|1920|85|1093|Saturday|1920Q4|N|Y|N|2422660|2422994|2422304|2422578|N|N|N|N|N| +2422671|AAAAAAAAPIHPECAA|1920-12-12|251|1093|85|1920|0|12|12|4|1920|85|1093|Sunday|1920Q4|N|N|N|2422660|2422994|2422305|2422579|N|N|N|N|N| +2422672|AAAAAAAAAJHPECAA|1920-12-13|251|1093|85|1920|1|12|13|4|1920|85|1093|Monday|1920Q4|N|N|N|2422660|2422994|2422306|2422580|N|N|N|N|N| +2422673|AAAAAAAABJHPECAA|1920-12-14|251|1094|85|1920|2|12|14|4|1920|85|1094|Tuesday|1920Q4|N|N|N|2422660|2422994|2422307|2422581|N|N|N|N|N| +2422674|AAAAAAAACJHPECAA|1920-12-15|251|1094|85|1920|3|12|15|4|1920|85|1094|Wednesday|1920Q4|N|N|N|2422660|2422994|2422308|2422582|N|N|N|N|N| +2422675|AAAAAAAADJHPECAA|1920-12-16|251|1094|85|1920|4|12|16|4|1920|85|1094|Thursday|1920Q4|N|N|N|2422660|2422994|2422309|2422583|N|N|N|N|N| +2422676|AAAAAAAAEJHPECAA|1920-12-17|251|1094|85|1920|5|12|17|4|1920|85|1094|Friday|1920Q4|N|Y|N|2422660|2422994|2422310|2422584|N|N|N|N|N| +2422677|AAAAAAAAFJHPECAA|1920-12-18|251|1094|85|1920|6|12|18|4|1920|85|1094|Saturday|1920Q4|N|Y|N|2422660|2422994|2422311|2422585|N|N|N|N|N| +2422678|AAAAAAAAGJHPECAA|1920-12-19|251|1094|85|1920|0|12|19|4|1920|85|1094|Sunday|1920Q4|N|N|N|2422660|2422994|2422312|2422586|N|N|N|N|N| +2422679|AAAAAAAAHJHPECAA|1920-12-20|251|1094|85|1920|1|12|20|4|1920|85|1094|Monday|1920Q4|N|N|N|2422660|2422994|2422313|2422587|N|N|N|N|N| +2422680|AAAAAAAAIJHPECAA|1920-12-21|251|1095|85|1920|2|12|21|4|1920|85|1095|Tuesday|1920Q4|N|N|N|2422660|2422994|2422314|2422588|N|N|N|N|N| +2422681|AAAAAAAAJJHPECAA|1920-12-22|251|1095|85|1920|3|12|22|4|1920|85|1095|Wednesday|1920Q4|N|N|N|2422660|2422994|2422315|2422589|N|N|N|N|N| +2422682|AAAAAAAAKJHPECAA|1920-12-23|251|1095|85|1920|4|12|23|4|1920|85|1095|Thursday|1920Q4|N|N|N|2422660|2422994|2422316|2422590|N|N|N|N|N| +2422683|AAAAAAAALJHPECAA|1920-12-24|251|1095|85|1920|5|12|24|4|1920|85|1095|Friday|1920Q4|N|Y|N|2422660|2422994|2422317|2422591|N|N|N|N|N| +2422684|AAAAAAAAMJHPECAA|1920-12-25|251|1095|85|1920|6|12|25|4|1920|85|1095|Saturday|1920Q4|Y|Y|N|2422660|2422994|2422318|2422592|N|N|N|N|N| +2422685|AAAAAAAANJHPECAA|1920-12-26|251|1095|85|1920|0|12|26|4|1920|85|1095|Sunday|1920Q4|N|N|Y|2422660|2422994|2422319|2422593|N|N|N|N|N| +2422686|AAAAAAAAOJHPECAA|1920-12-27|251|1095|85|1920|1|12|27|4|1920|85|1095|Monday|1920Q4|N|N|N|2422660|2422994|2422320|2422594|N|N|N|N|N| +2422687|AAAAAAAAPJHPECAA|1920-12-28|251|1096|85|1920|2|12|28|4|1920|85|1096|Tuesday|1920Q4|N|N|N|2422660|2422994|2422321|2422595|N|N|N|N|N| +2422688|AAAAAAAAAKHPECAA|1920-12-29|251|1096|85|1920|3|12|29|4|1920|85|1096|Wednesday|1920Q4|N|N|N|2422660|2422994|2422322|2422596|N|N|N|N|N| +2422689|AAAAAAAABKHPECAA|1920-12-30|251|1096|85|1920|4|12|30|4|1920|85|1096|Thursday|1920Q4|N|N|N|2422660|2422994|2422323|2422597|N|N|N|N|N| +2422690|AAAAAAAACKHPECAA|1920-12-31|251|1096|85|1920|5|12|31|4|1920|85|1096|Friday|1920Q4|Y|Y|N|2422660|2422994|2422324|2422598|N|N|N|N|N| +2422691|AAAAAAAADKHPECAA|1921-01-01|252|1096|85|1921|6|1|1|1|1921|85|1096|Saturday|1921Q1|Y|Y|Y|2422691|2422690|2422325|2422599|N|N|N|N|N| +2422692|AAAAAAAAEKHPECAA|1921-01-02|252|1096|85|1921|0|1|2|1|1921|85|1096|Sunday|1921Q1|N|N|Y|2422691|2422690|2422326|2422600|N|N|N|N|N| +2422693|AAAAAAAAFKHPECAA|1921-01-03|252|1096|85|1921|1|1|3|1|1921|85|1096|Monday|1921Q1|N|N|N|2422691|2422690|2422327|2422601|N|N|N|N|N| +2422694|AAAAAAAAGKHPECAA|1921-01-04|252|1097|85|1921|2|1|4|1|1921|85|1097|Tuesday|1921Q1|N|N|N|2422691|2422690|2422328|2422602|N|N|N|N|N| +2422695|AAAAAAAAHKHPECAA|1921-01-05|252|1097|85|1921|3|1|5|1|1921|85|1097|Wednesday|1921Q1|N|N|N|2422691|2422690|2422329|2422603|N|N|N|N|N| +2422696|AAAAAAAAIKHPECAA|1921-01-06|252|1097|85|1921|4|1|6|1|1921|85|1097|Thursday|1921Q1|N|N|N|2422691|2422690|2422330|2422604|N|N|N|N|N| +2422697|AAAAAAAAJKHPECAA|1921-01-07|252|1097|85|1921|5|1|7|1|1921|85|1097|Friday|1921Q1|N|Y|N|2422691|2422690|2422331|2422605|N|N|N|N|N| +2422698|AAAAAAAAKKHPECAA|1921-01-08|252|1097|85|1921|6|1|8|1|1921|85|1097|Saturday|1921Q1|N|Y|N|2422691|2422690|2422332|2422606|N|N|N|N|N| +2422699|AAAAAAAALKHPECAA|1921-01-09|252|1097|85|1921|0|1|9|1|1921|85|1097|Sunday|1921Q1|N|N|N|2422691|2422690|2422333|2422607|N|N|N|N|N| +2422700|AAAAAAAAMKHPECAA|1921-01-10|252|1097|85|1921|1|1|10|1|1921|85|1097|Monday|1921Q1|N|N|N|2422691|2422690|2422334|2422608|N|N|N|N|N| +2422701|AAAAAAAANKHPECAA|1921-01-11|252|1098|85|1921|2|1|11|1|1921|85|1098|Tuesday|1921Q1|N|N|N|2422691|2422690|2422335|2422609|N|N|N|N|N| +2422702|AAAAAAAAOKHPECAA|1921-01-12|252|1098|85|1921|3|1|12|1|1921|85|1098|Wednesday|1921Q1|N|N|N|2422691|2422690|2422336|2422610|N|N|N|N|N| +2422703|AAAAAAAAPKHPECAA|1921-01-13|252|1098|85|1921|4|1|13|1|1921|85|1098|Thursday|1921Q1|N|N|N|2422691|2422690|2422337|2422611|N|N|N|N|N| +2422704|AAAAAAAAALHPECAA|1921-01-14|252|1098|85|1921|5|1|14|1|1921|85|1098|Friday|1921Q1|N|Y|N|2422691|2422690|2422338|2422612|N|N|N|N|N| +2422705|AAAAAAAABLHPECAA|1921-01-15|252|1098|85|1921|6|1|15|1|1921|85|1098|Saturday|1921Q1|N|Y|N|2422691|2422690|2422339|2422613|N|N|N|N|N| +2422706|AAAAAAAACLHPECAA|1921-01-16|252|1098|85|1921|0|1|16|1|1921|85|1098|Sunday|1921Q1|N|N|N|2422691|2422690|2422340|2422614|N|N|N|N|N| +2422707|AAAAAAAADLHPECAA|1921-01-17|252|1098|85|1921|1|1|17|1|1921|85|1098|Monday|1921Q1|N|N|N|2422691|2422690|2422341|2422615|N|N|N|N|N| +2422708|AAAAAAAAELHPECAA|1921-01-18|252|1099|85|1921|2|1|18|1|1921|85|1099|Tuesday|1921Q1|N|N|N|2422691|2422690|2422342|2422616|N|N|N|N|N| +2422709|AAAAAAAAFLHPECAA|1921-01-19|252|1099|85|1921|3|1|19|1|1921|85|1099|Wednesday|1921Q1|N|N|N|2422691|2422690|2422343|2422617|N|N|N|N|N| +2422710|AAAAAAAAGLHPECAA|1921-01-20|252|1099|85|1921|4|1|20|1|1921|85|1099|Thursday|1921Q1|N|N|N|2422691|2422690|2422344|2422618|N|N|N|N|N| +2422711|AAAAAAAAHLHPECAA|1921-01-21|252|1099|85|1921|5|1|21|1|1921|85|1099|Friday|1921Q1|N|Y|N|2422691|2422690|2422345|2422619|N|N|N|N|N| +2422712|AAAAAAAAILHPECAA|1921-01-22|252|1099|85|1921|6|1|22|1|1921|85|1099|Saturday|1921Q1|N|Y|N|2422691|2422690|2422346|2422620|N|N|N|N|N| +2422713|AAAAAAAAJLHPECAA|1921-01-23|252|1099|85|1921|0|1|23|1|1921|85|1099|Sunday|1921Q1|N|N|N|2422691|2422690|2422347|2422621|N|N|N|N|N| +2422714|AAAAAAAAKLHPECAA|1921-01-24|252|1099|85|1921|1|1|24|1|1921|85|1099|Monday|1921Q1|N|N|N|2422691|2422690|2422348|2422622|N|N|N|N|N| +2422715|AAAAAAAALLHPECAA|1921-01-25|252|1100|85|1921|2|1|25|1|1921|85|1100|Tuesday|1921Q1|N|N|N|2422691|2422690|2422349|2422623|N|N|N|N|N| +2422716|AAAAAAAAMLHPECAA|1921-01-26|252|1100|85|1921|3|1|26|1|1921|85|1100|Wednesday|1921Q1|N|N|N|2422691|2422690|2422350|2422624|N|N|N|N|N| +2422717|AAAAAAAANLHPECAA|1921-01-27|252|1100|85|1921|4|1|27|1|1921|85|1100|Thursday|1921Q1|N|N|N|2422691|2422690|2422351|2422625|N|N|N|N|N| +2422718|AAAAAAAAOLHPECAA|1921-01-28|252|1100|85|1921|5|1|28|1|1921|85|1100|Friday|1921Q1|N|Y|N|2422691|2422690|2422352|2422626|N|N|N|N|N| +2422719|AAAAAAAAPLHPECAA|1921-01-29|252|1100|85|1921|6|1|29|1|1921|85|1100|Saturday|1921Q1|N|Y|N|2422691|2422690|2422353|2422627|N|N|N|N|N| +2422720|AAAAAAAAAMHPECAA|1921-01-30|252|1100|85|1921|0|1|30|1|1921|85|1100|Sunday|1921Q1|N|N|N|2422691|2422690|2422354|2422628|N|N|N|N|N| +2422721|AAAAAAAABMHPECAA|1921-01-31|252|1100|85|1921|1|1|31|1|1921|85|1100|Monday|1921Q1|N|N|N|2422691|2422690|2422355|2422629|N|N|N|N|N| +2422722|AAAAAAAACMHPECAA|1921-02-01|253|1101|85|1921|2|2|1|1|1921|85|1101|Tuesday|1921Q1|N|N|N|2422722|2422752|2422356|2422630|N|N|N|N|N| +2422723|AAAAAAAADMHPECAA|1921-02-02|253|1101|85|1921|3|2|2|1|1921|85|1101|Wednesday|1921Q1|N|N|N|2422722|2422752|2422357|2422631|N|N|N|N|N| +2422724|AAAAAAAAEMHPECAA|1921-02-03|253|1101|85|1921|4|2|3|1|1921|85|1101|Thursday|1921Q1|N|N|N|2422722|2422752|2422358|2422632|N|N|N|N|N| +2422725|AAAAAAAAFMHPECAA|1921-02-04|253|1101|85|1921|5|2|4|1|1921|85|1101|Friday|1921Q1|N|Y|N|2422722|2422752|2422359|2422633|N|N|N|N|N| +2422726|AAAAAAAAGMHPECAA|1921-02-05|253|1101|85|1921|6|2|5|1|1921|85|1101|Saturday|1921Q1|N|Y|N|2422722|2422752|2422360|2422634|N|N|N|N|N| +2422727|AAAAAAAAHMHPECAA|1921-02-06|253|1101|85|1921|0|2|6|1|1921|85|1101|Sunday|1921Q1|N|N|N|2422722|2422752|2422361|2422635|N|N|N|N|N| +2422728|AAAAAAAAIMHPECAA|1921-02-07|253|1101|85|1921|1|2|7|1|1921|85|1101|Monday|1921Q1|N|N|N|2422722|2422752|2422362|2422636|N|N|N|N|N| +2422729|AAAAAAAAJMHPECAA|1921-02-08|253|1102|85|1921|2|2|8|1|1921|85|1102|Tuesday|1921Q1|N|N|N|2422722|2422752|2422363|2422637|N|N|N|N|N| +2422730|AAAAAAAAKMHPECAA|1921-02-09|253|1102|85|1921|3|2|9|1|1921|85|1102|Wednesday|1921Q1|N|N|N|2422722|2422752|2422364|2422638|N|N|N|N|N| +2422731|AAAAAAAALMHPECAA|1921-02-10|253|1102|85|1921|4|2|10|1|1921|85|1102|Thursday|1921Q1|N|N|N|2422722|2422752|2422365|2422639|N|N|N|N|N| +2422732|AAAAAAAAMMHPECAA|1921-02-11|253|1102|85|1921|5|2|11|1|1921|85|1102|Friday|1921Q1|N|Y|N|2422722|2422752|2422366|2422640|N|N|N|N|N| +2422733|AAAAAAAANMHPECAA|1921-02-12|253|1102|85|1921|6|2|12|1|1921|85|1102|Saturday|1921Q1|N|Y|N|2422722|2422752|2422367|2422641|N|N|N|N|N| +2422734|AAAAAAAAOMHPECAA|1921-02-13|253|1102|85|1921|0|2|13|1|1921|85|1102|Sunday|1921Q1|N|N|N|2422722|2422752|2422368|2422642|N|N|N|N|N| +2422735|AAAAAAAAPMHPECAA|1921-02-14|253|1102|85|1921|1|2|14|1|1921|85|1102|Monday|1921Q1|N|N|N|2422722|2422752|2422369|2422643|N|N|N|N|N| +2422736|AAAAAAAAANHPECAA|1921-02-15|253|1103|85|1921|2|2|15|1|1921|85|1103|Tuesday|1921Q1|N|N|N|2422722|2422752|2422370|2422644|N|N|N|N|N| +2422737|AAAAAAAABNHPECAA|1921-02-16|253|1103|85|1921|3|2|16|1|1921|85|1103|Wednesday|1921Q1|N|N|N|2422722|2422752|2422371|2422645|N|N|N|N|N| +2422738|AAAAAAAACNHPECAA|1921-02-17|253|1103|85|1921|4|2|17|1|1921|85|1103|Thursday|1921Q1|N|N|N|2422722|2422752|2422372|2422646|N|N|N|N|N| +2422739|AAAAAAAADNHPECAA|1921-02-18|253|1103|85|1921|5|2|18|1|1921|85|1103|Friday|1921Q1|N|Y|N|2422722|2422752|2422373|2422647|N|N|N|N|N| +2422740|AAAAAAAAENHPECAA|1921-02-19|253|1103|85|1921|6|2|19|1|1921|85|1103|Saturday|1921Q1|N|Y|N|2422722|2422752|2422374|2422648|N|N|N|N|N| +2422741|AAAAAAAAFNHPECAA|1921-02-20|253|1103|85|1921|0|2|20|1|1921|85|1103|Sunday|1921Q1|N|N|N|2422722|2422752|2422375|2422649|N|N|N|N|N| +2422742|AAAAAAAAGNHPECAA|1921-02-21|253|1103|85|1921|1|2|21|1|1921|85|1103|Monday|1921Q1|N|N|N|2422722|2422752|2422376|2422650|N|N|N|N|N| +2422743|AAAAAAAAHNHPECAA|1921-02-22|253|1104|85|1921|2|2|22|1|1921|85|1104|Tuesday|1921Q1|N|N|N|2422722|2422752|2422377|2422651|N|N|N|N|N| +2422744|AAAAAAAAINHPECAA|1921-02-23|253|1104|85|1921|3|2|23|1|1921|85|1104|Wednesday|1921Q1|N|N|N|2422722|2422752|2422378|2422652|N|N|N|N|N| +2422745|AAAAAAAAJNHPECAA|1921-02-24|253|1104|85|1921|4|2|24|1|1921|85|1104|Thursday|1921Q1|N|N|N|2422722|2422752|2422379|2422653|N|N|N|N|N| +2422746|AAAAAAAAKNHPECAA|1921-02-25|253|1104|85|1921|5|2|25|1|1921|85|1104|Friday|1921Q1|N|Y|N|2422722|2422752|2422380|2422654|N|N|N|N|N| +2422747|AAAAAAAALNHPECAA|1921-02-26|253|1104|85|1921|6|2|26|1|1921|85|1104|Saturday|1921Q1|N|Y|N|2422722|2422752|2422381|2422655|N|N|N|N|N| +2422748|AAAAAAAAMNHPECAA|1921-02-27|253|1104|85|1921|0|2|27|1|1921|85|1104|Sunday|1921Q1|N|N|N|2422722|2422752|2422382|2422656|N|N|N|N|N| +2422749|AAAAAAAANNHPECAA|1921-02-28|253|1104|85|1921|1|2|28|1|1921|85|1104|Monday|1921Q1|N|N|N|2422722|2422752|2422383|2422657|N|N|N|N|N| +2422750|AAAAAAAAONHPECAA|1921-03-01|254|1105|86|1921|2|3|1|1|1921|86|1105|Tuesday|1921Q1|N|N|N|2422750|2422808|2422385|2422658|N|N|N|N|N| +2422751|AAAAAAAAPNHPECAA|1921-03-02|254|1105|86|1921|3|3|2|1|1921|86|1105|Wednesday|1921Q1|N|N|N|2422750|2422808|2422386|2422659|N|N|N|N|N| +2422752|AAAAAAAAAOHPECAA|1921-03-03|254|1105|86|1921|4|3|3|1|1921|86|1105|Thursday|1921Q1|N|N|N|2422750|2422808|2422387|2422660|N|N|N|N|N| +2422753|AAAAAAAABOHPECAA|1921-03-04|254|1105|86|1921|5|3|4|1|1921|86|1105|Friday|1921Q1|N|Y|N|2422750|2422808|2422388|2422661|N|N|N|N|N| +2422754|AAAAAAAACOHPECAA|1921-03-05|254|1105|86|1921|6|3|5|1|1921|86|1105|Saturday|1921Q1|N|Y|N|2422750|2422808|2422389|2422662|N|N|N|N|N| +2422755|AAAAAAAADOHPECAA|1921-03-06|254|1105|86|1921|0|3|6|1|1921|86|1105|Sunday|1921Q1|N|N|N|2422750|2422808|2422390|2422663|N|N|N|N|N| +2422756|AAAAAAAAEOHPECAA|1921-03-07|254|1105|86|1921|1|3|7|1|1921|86|1105|Monday|1921Q1|N|N|N|2422750|2422808|2422391|2422664|N|N|N|N|N| +2422757|AAAAAAAAFOHPECAA|1921-03-08|254|1106|86|1921|2|3|8|1|1921|86|1106|Tuesday|1921Q1|N|N|N|2422750|2422808|2422392|2422665|N|N|N|N|N| +2422758|AAAAAAAAGOHPECAA|1921-03-09|254|1106|86|1921|3|3|9|1|1921|86|1106|Wednesday|1921Q1|N|N|N|2422750|2422808|2422393|2422666|N|N|N|N|N| +2422759|AAAAAAAAHOHPECAA|1921-03-10|254|1106|86|1921|4|3|10|1|1921|86|1106|Thursday|1921Q1|N|N|N|2422750|2422808|2422394|2422667|N|N|N|N|N| +2422760|AAAAAAAAIOHPECAA|1921-03-11|254|1106|86|1921|5|3|11|1|1921|86|1106|Friday|1921Q1|N|Y|N|2422750|2422808|2422395|2422668|N|N|N|N|N| +2422761|AAAAAAAAJOHPECAA|1921-03-12|254|1106|86|1921|6|3|12|1|1921|86|1106|Saturday|1921Q1|N|Y|N|2422750|2422808|2422396|2422669|N|N|N|N|N| +2422762|AAAAAAAAKOHPECAA|1921-03-13|254|1106|86|1921|0|3|13|1|1921|86|1106|Sunday|1921Q1|N|N|N|2422750|2422808|2422397|2422670|N|N|N|N|N| +2422763|AAAAAAAALOHPECAA|1921-03-14|254|1106|86|1921|1|3|14|1|1921|86|1106|Monday|1921Q1|N|N|N|2422750|2422808|2422398|2422671|N|N|N|N|N| +2422764|AAAAAAAAMOHPECAA|1921-03-15|254|1107|86|1921|2|3|15|1|1921|86|1107|Tuesday|1921Q1|N|N|N|2422750|2422808|2422399|2422672|N|N|N|N|N| +2422765|AAAAAAAANOHPECAA|1921-03-16|254|1107|86|1921|3|3|16|1|1921|86|1107|Wednesday|1921Q1|N|N|N|2422750|2422808|2422400|2422673|N|N|N|N|N| +2422766|AAAAAAAAOOHPECAA|1921-03-17|254|1107|86|1921|4|3|17|1|1921|86|1107|Thursday|1921Q1|N|N|N|2422750|2422808|2422401|2422674|N|N|N|N|N| +2422767|AAAAAAAAPOHPECAA|1921-03-18|254|1107|86|1921|5|3|18|1|1921|86|1107|Friday|1921Q1|N|Y|N|2422750|2422808|2422402|2422675|N|N|N|N|N| +2422768|AAAAAAAAAPHPECAA|1921-03-19|254|1107|86|1921|6|3|19|1|1921|86|1107|Saturday|1921Q1|N|Y|N|2422750|2422808|2422403|2422676|N|N|N|N|N| +2422769|AAAAAAAABPHPECAA|1921-03-20|254|1107|86|1921|0|3|20|1|1921|86|1107|Sunday|1921Q1|N|N|N|2422750|2422808|2422404|2422677|N|N|N|N|N| +2422770|AAAAAAAACPHPECAA|1921-03-21|254|1107|86|1921|1|3|21|1|1921|86|1107|Monday|1921Q1|N|N|N|2422750|2422808|2422405|2422678|N|N|N|N|N| +2422771|AAAAAAAADPHPECAA|1921-03-22|254|1108|86|1921|2|3|22|1|1921|86|1108|Tuesday|1921Q1|N|N|N|2422750|2422808|2422406|2422679|N|N|N|N|N| +2422772|AAAAAAAAEPHPECAA|1921-03-23|254|1108|86|1921|3|3|23|1|1921|86|1108|Wednesday|1921Q1|N|N|N|2422750|2422808|2422407|2422680|N|N|N|N|N| +2422773|AAAAAAAAFPHPECAA|1921-03-24|254|1108|86|1921|4|3|24|1|1921|86|1108|Thursday|1921Q1|N|N|N|2422750|2422808|2422408|2422681|N|N|N|N|N| +2422774|AAAAAAAAGPHPECAA|1921-03-25|254|1108|86|1921|5|3|25|1|1921|86|1108|Friday|1921Q1|N|Y|N|2422750|2422808|2422409|2422682|N|N|N|N|N| +2422775|AAAAAAAAHPHPECAA|1921-03-26|254|1108|86|1921|6|3|26|1|1921|86|1108|Saturday|1921Q1|N|Y|N|2422750|2422808|2422410|2422683|N|N|N|N|N| +2422776|AAAAAAAAIPHPECAA|1921-03-27|254|1108|86|1921|0|3|27|1|1921|86|1108|Sunday|1921Q1|N|N|N|2422750|2422808|2422411|2422684|N|N|N|N|N| +2422777|AAAAAAAAJPHPECAA|1921-03-28|254|1108|86|1921|1|3|28|1|1921|86|1108|Monday|1921Q1|N|N|N|2422750|2422808|2422412|2422685|N|N|N|N|N| +2422778|AAAAAAAAKPHPECAA|1921-03-29|254|1109|86|1921|2|3|29|1|1921|86|1109|Tuesday|1921Q1|N|N|N|2422750|2422808|2422413|2422686|N|N|N|N|N| +2422779|AAAAAAAALPHPECAA|1921-03-30|254|1109|86|1921|3|3|30|1|1921|86|1109|Wednesday|1921Q1|N|N|N|2422750|2422808|2422414|2422687|N|N|N|N|N| +2422780|AAAAAAAAMPHPECAA|1921-03-31|254|1109|86|1921|4|3|31|1|1921|86|1109|Thursday|1921Q1|N|N|N|2422750|2422808|2422415|2422688|N|N|N|N|N| +2422781|AAAAAAAANPHPECAA|1921-04-01|255|1109|86|1921|5|4|1|1|1921|86|1109|Friday|1921Q1|N|Y|N|2422781|2422870|2422416|2422691|N|N|N|N|N| +2422782|AAAAAAAAOPHPECAA|1921-04-02|255|1109|86|1921|6|4|2|2|1921|86|1109|Saturday|1921Q2|N|Y|N|2422781|2422870|2422417|2422692|N|N|N|N|N| +2422783|AAAAAAAAPPHPECAA|1921-04-03|255|1109|86|1921|0|4|3|2|1921|86|1109|Sunday|1921Q2|N|N|N|2422781|2422870|2422418|2422693|N|N|N|N|N| +2422784|AAAAAAAAAAIPECAA|1921-04-04|255|1109|86|1921|1|4|4|2|1921|86|1109|Monday|1921Q2|N|N|N|2422781|2422870|2422419|2422694|N|N|N|N|N| +2422785|AAAAAAAABAIPECAA|1921-04-05|255|1110|86|1921|2|4|5|2|1921|86|1110|Tuesday|1921Q2|N|N|N|2422781|2422870|2422420|2422695|N|N|N|N|N| +2422786|AAAAAAAACAIPECAA|1921-04-06|255|1110|86|1921|3|4|6|2|1921|86|1110|Wednesday|1921Q2|N|N|N|2422781|2422870|2422421|2422696|N|N|N|N|N| +2422787|AAAAAAAADAIPECAA|1921-04-07|255|1110|86|1921|4|4|7|2|1921|86|1110|Thursday|1921Q2|N|N|N|2422781|2422870|2422422|2422697|N|N|N|N|N| +2422788|AAAAAAAAEAIPECAA|1921-04-08|255|1110|86|1921|5|4|8|2|1921|86|1110|Friday|1921Q2|N|Y|N|2422781|2422870|2422423|2422698|N|N|N|N|N| +2422789|AAAAAAAAFAIPECAA|1921-04-09|255|1110|86|1921|6|4|9|2|1921|86|1110|Saturday|1921Q2|N|Y|N|2422781|2422870|2422424|2422699|N|N|N|N|N| +2422790|AAAAAAAAGAIPECAA|1921-04-10|255|1110|86|1921|0|4|10|2|1921|86|1110|Sunday|1921Q2|N|N|N|2422781|2422870|2422425|2422700|N|N|N|N|N| +2422791|AAAAAAAAHAIPECAA|1921-04-11|255|1110|86|1921|1|4|11|2|1921|86|1110|Monday|1921Q2|N|N|N|2422781|2422870|2422426|2422701|N|N|N|N|N| +2422792|AAAAAAAAIAIPECAA|1921-04-12|255|1111|86|1921|2|4|12|2|1921|86|1111|Tuesday|1921Q2|N|N|N|2422781|2422870|2422427|2422702|N|N|N|N|N| +2422793|AAAAAAAAJAIPECAA|1921-04-13|255|1111|86|1921|3|4|13|2|1921|86|1111|Wednesday|1921Q2|N|N|N|2422781|2422870|2422428|2422703|N|N|N|N|N| +2422794|AAAAAAAAKAIPECAA|1921-04-14|255|1111|86|1921|4|4|14|2|1921|86|1111|Thursday|1921Q2|N|N|N|2422781|2422870|2422429|2422704|N|N|N|N|N| +2422795|AAAAAAAALAIPECAA|1921-04-15|255|1111|86|1921|5|4|15|2|1921|86|1111|Friday|1921Q2|N|Y|N|2422781|2422870|2422430|2422705|N|N|N|N|N| +2422796|AAAAAAAAMAIPECAA|1921-04-16|255|1111|86|1921|6|4|16|2|1921|86|1111|Saturday|1921Q2|N|Y|N|2422781|2422870|2422431|2422706|N|N|N|N|N| +2422797|AAAAAAAANAIPECAA|1921-04-17|255|1111|86|1921|0|4|17|2|1921|86|1111|Sunday|1921Q2|N|N|N|2422781|2422870|2422432|2422707|N|N|N|N|N| +2422798|AAAAAAAAOAIPECAA|1921-04-18|255|1111|86|1921|1|4|18|2|1921|86|1111|Monday|1921Q2|N|N|N|2422781|2422870|2422433|2422708|N|N|N|N|N| +2422799|AAAAAAAAPAIPECAA|1921-04-19|255|1112|86|1921|2|4|19|2|1921|86|1112|Tuesday|1921Q2|N|N|N|2422781|2422870|2422434|2422709|N|N|N|N|N| +2422800|AAAAAAAAABIPECAA|1921-04-20|255|1112|86|1921|3|4|20|2|1921|86|1112|Wednesday|1921Q2|N|N|N|2422781|2422870|2422435|2422710|N|N|N|N|N| +2422801|AAAAAAAABBIPECAA|1921-04-21|255|1112|86|1921|4|4|21|2|1921|86|1112|Thursday|1921Q2|N|N|N|2422781|2422870|2422436|2422711|N|N|N|N|N| +2422802|AAAAAAAACBIPECAA|1921-04-22|255|1112|86|1921|5|4|22|2|1921|86|1112|Friday|1921Q2|N|Y|N|2422781|2422870|2422437|2422712|N|N|N|N|N| +2422803|AAAAAAAADBIPECAA|1921-04-23|255|1112|86|1921|6|4|23|2|1921|86|1112|Saturday|1921Q2|N|Y|N|2422781|2422870|2422438|2422713|N|N|N|N|N| +2422804|AAAAAAAAEBIPECAA|1921-04-24|255|1112|86|1921|0|4|24|2|1921|86|1112|Sunday|1921Q2|N|N|N|2422781|2422870|2422439|2422714|N|N|N|N|N| +2422805|AAAAAAAAFBIPECAA|1921-04-25|255|1112|86|1921|1|4|25|2|1921|86|1112|Monday|1921Q2|N|N|N|2422781|2422870|2422440|2422715|N|N|N|N|N| +2422806|AAAAAAAAGBIPECAA|1921-04-26|255|1113|86|1921|2|4|26|2|1921|86|1113|Tuesday|1921Q2|N|N|N|2422781|2422870|2422441|2422716|N|N|N|N|N| +2422807|AAAAAAAAHBIPECAA|1921-04-27|255|1113|86|1921|3|4|27|2|1921|86|1113|Wednesday|1921Q2|N|N|N|2422781|2422870|2422442|2422717|N|N|N|N|N| +2422808|AAAAAAAAIBIPECAA|1921-04-28|255|1113|86|1921|4|4|28|2|1921|86|1113|Thursday|1921Q2|N|N|N|2422781|2422870|2422443|2422718|N|N|N|N|N| +2422809|AAAAAAAAJBIPECAA|1921-04-29|255|1113|86|1921|5|4|29|2|1921|86|1113|Friday|1921Q2|N|Y|N|2422781|2422870|2422444|2422719|N|N|N|N|N| +2422810|AAAAAAAAKBIPECAA|1921-04-30|255|1113|86|1921|6|4|30|2|1921|86|1113|Saturday|1921Q2|N|Y|N|2422781|2422870|2422445|2422720|N|N|N|N|N| +2422811|AAAAAAAALBIPECAA|1921-05-01|256|1113|86|1921|0|5|1|2|1921|86|1113|Sunday|1921Q2|N|N|N|2422811|2422930|2422446|2422721|N|N|N|N|N| +2422812|AAAAAAAAMBIPECAA|1921-05-02|256|1113|86|1921|1|5|2|2|1921|86|1113|Monday|1921Q2|N|N|N|2422811|2422930|2422447|2422722|N|N|N|N|N| +2422813|AAAAAAAANBIPECAA|1921-05-03|256|1114|86|1921|2|5|3|2|1921|86|1114|Tuesday|1921Q2|N|N|N|2422811|2422930|2422448|2422723|N|N|N|N|N| +2422814|AAAAAAAAOBIPECAA|1921-05-04|256|1114|86|1921|3|5|4|2|1921|86|1114|Wednesday|1921Q2|N|N|N|2422811|2422930|2422449|2422724|N|N|N|N|N| +2422815|AAAAAAAAPBIPECAA|1921-05-05|256|1114|86|1921|4|5|5|2|1921|86|1114|Thursday|1921Q2|N|N|N|2422811|2422930|2422450|2422725|N|N|N|N|N| +2422816|AAAAAAAAACIPECAA|1921-05-06|256|1114|86|1921|5|5|6|2|1921|86|1114|Friday|1921Q2|N|Y|N|2422811|2422930|2422451|2422726|N|N|N|N|N| +2422817|AAAAAAAABCIPECAA|1921-05-07|256|1114|86|1921|6|5|7|2|1921|86|1114|Saturday|1921Q2|N|Y|N|2422811|2422930|2422452|2422727|N|N|N|N|N| +2422818|AAAAAAAACCIPECAA|1921-05-08|256|1114|86|1921|0|5|8|2|1921|86|1114|Sunday|1921Q2|N|N|N|2422811|2422930|2422453|2422728|N|N|N|N|N| +2422819|AAAAAAAADCIPECAA|1921-05-09|256|1114|86|1921|1|5|9|2|1921|86|1114|Monday|1921Q2|N|N|N|2422811|2422930|2422454|2422729|N|N|N|N|N| +2422820|AAAAAAAAECIPECAA|1921-05-10|256|1115|86|1921|2|5|10|2|1921|86|1115|Tuesday|1921Q2|N|N|N|2422811|2422930|2422455|2422730|N|N|N|N|N| +2422821|AAAAAAAAFCIPECAA|1921-05-11|256|1115|86|1921|3|5|11|2|1921|86|1115|Wednesday|1921Q2|N|N|N|2422811|2422930|2422456|2422731|N|N|N|N|N| +2422822|AAAAAAAAGCIPECAA|1921-05-12|256|1115|86|1921|4|5|12|2|1921|86|1115|Thursday|1921Q2|N|N|N|2422811|2422930|2422457|2422732|N|N|N|N|N| +2422823|AAAAAAAAHCIPECAA|1921-05-13|256|1115|86|1921|5|5|13|2|1921|86|1115|Friday|1921Q2|N|Y|N|2422811|2422930|2422458|2422733|N|N|N|N|N| +2422824|AAAAAAAAICIPECAA|1921-05-14|256|1115|86|1921|6|5|14|2|1921|86|1115|Saturday|1921Q2|N|Y|N|2422811|2422930|2422459|2422734|N|N|N|N|N| +2422825|AAAAAAAAJCIPECAA|1921-05-15|256|1115|86|1921|0|5|15|2|1921|86|1115|Sunday|1921Q2|N|N|N|2422811|2422930|2422460|2422735|N|N|N|N|N| +2422826|AAAAAAAAKCIPECAA|1921-05-16|256|1115|86|1921|1|5|16|2|1921|86|1115|Monday|1921Q2|N|N|N|2422811|2422930|2422461|2422736|N|N|N|N|N| +2422827|AAAAAAAALCIPECAA|1921-05-17|256|1116|86|1921|2|5|17|2|1921|86|1116|Tuesday|1921Q2|N|N|N|2422811|2422930|2422462|2422737|N|N|N|N|N| +2422828|AAAAAAAAMCIPECAA|1921-05-18|256|1116|86|1921|3|5|18|2|1921|86|1116|Wednesday|1921Q2|N|N|N|2422811|2422930|2422463|2422738|N|N|N|N|N| +2422829|AAAAAAAANCIPECAA|1921-05-19|256|1116|86|1921|4|5|19|2|1921|86|1116|Thursday|1921Q2|N|N|N|2422811|2422930|2422464|2422739|N|N|N|N|N| +2422830|AAAAAAAAOCIPECAA|1921-05-20|256|1116|86|1921|5|5|20|2|1921|86|1116|Friday|1921Q2|N|Y|N|2422811|2422930|2422465|2422740|N|N|N|N|N| +2422831|AAAAAAAAPCIPECAA|1921-05-21|256|1116|86|1921|6|5|21|2|1921|86|1116|Saturday|1921Q2|N|Y|N|2422811|2422930|2422466|2422741|N|N|N|N|N| +2422832|AAAAAAAAADIPECAA|1921-05-22|256|1116|86|1921|0|5|22|2|1921|86|1116|Sunday|1921Q2|N|N|N|2422811|2422930|2422467|2422742|N|N|N|N|N| +2422833|AAAAAAAABDIPECAA|1921-05-23|256|1116|86|1921|1|5|23|2|1921|86|1116|Monday|1921Q2|N|N|N|2422811|2422930|2422468|2422743|N|N|N|N|N| +2422834|AAAAAAAACDIPECAA|1921-05-24|256|1117|86|1921|2|5|24|2|1921|86|1117|Tuesday|1921Q2|N|N|N|2422811|2422930|2422469|2422744|N|N|N|N|N| +2422835|AAAAAAAADDIPECAA|1921-05-25|256|1117|86|1921|3|5|25|2|1921|86|1117|Wednesday|1921Q2|N|N|N|2422811|2422930|2422470|2422745|N|N|N|N|N| +2422836|AAAAAAAAEDIPECAA|1921-05-26|256|1117|86|1921|4|5|26|2|1921|86|1117|Thursday|1921Q2|N|N|N|2422811|2422930|2422471|2422746|N|N|N|N|N| +2422837|AAAAAAAAFDIPECAA|1921-05-27|256|1117|86|1921|5|5|27|2|1921|86|1117|Friday|1921Q2|N|Y|N|2422811|2422930|2422472|2422747|N|N|N|N|N| +2422838|AAAAAAAAGDIPECAA|1921-05-28|256|1117|86|1921|6|5|28|2|1921|86|1117|Saturday|1921Q2|N|Y|N|2422811|2422930|2422473|2422748|N|N|N|N|N| +2422839|AAAAAAAAHDIPECAA|1921-05-29|256|1117|86|1921|0|5|29|2|1921|86|1117|Sunday|1921Q2|N|N|N|2422811|2422930|2422474|2422749|N|N|N|N|N| +2422840|AAAAAAAAIDIPECAA|1921-05-30|256|1117|86|1921|1|5|30|2|1921|86|1117|Monday|1921Q2|N|N|N|2422811|2422930|2422475|2422750|N|N|N|N|N| +2422841|AAAAAAAAJDIPECAA|1921-05-31|256|1118|86|1921|2|5|31|2|1921|86|1118|Tuesday|1921Q2|N|N|N|2422811|2422930|2422476|2422751|N|N|N|N|N| +2422842|AAAAAAAAKDIPECAA|1921-06-01|257|1118|87|1921|3|6|1|2|1921|87|1118|Wednesday|1921Q2|N|N|N|2422842|2422992|2422477|2422752|N|N|N|N|N| +2422843|AAAAAAAALDIPECAA|1921-06-02|257|1118|87|1921|4|6|2|2|1921|87|1118|Thursday|1921Q2|N|N|N|2422842|2422992|2422478|2422753|N|N|N|N|N| +2422844|AAAAAAAAMDIPECAA|1921-06-03|257|1118|87|1921|5|6|3|2|1921|87|1118|Friday|1921Q2|N|Y|N|2422842|2422992|2422479|2422754|N|N|N|N|N| +2422845|AAAAAAAANDIPECAA|1921-06-04|257|1118|87|1921|6|6|4|2|1921|87|1118|Saturday|1921Q2|N|Y|N|2422842|2422992|2422480|2422755|N|N|N|N|N| +2422846|AAAAAAAAODIPECAA|1921-06-05|257|1118|87|1921|0|6|5|2|1921|87|1118|Sunday|1921Q2|N|N|N|2422842|2422992|2422481|2422756|N|N|N|N|N| +2422847|AAAAAAAAPDIPECAA|1921-06-06|257|1118|87|1921|1|6|6|2|1921|87|1118|Monday|1921Q2|N|N|N|2422842|2422992|2422482|2422757|N|N|N|N|N| +2422848|AAAAAAAAAEIPECAA|1921-06-07|257|1119|87|1921|2|6|7|2|1921|87|1119|Tuesday|1921Q2|N|N|N|2422842|2422992|2422483|2422758|N|N|N|N|N| +2422849|AAAAAAAABEIPECAA|1921-06-08|257|1119|87|1921|3|6|8|2|1921|87|1119|Wednesday|1921Q2|N|N|N|2422842|2422992|2422484|2422759|N|N|N|N|N| +2422850|AAAAAAAACEIPECAA|1921-06-09|257|1119|87|1921|4|6|9|2|1921|87|1119|Thursday|1921Q2|N|N|N|2422842|2422992|2422485|2422760|N|N|N|N|N| +2422851|AAAAAAAADEIPECAA|1921-06-10|257|1119|87|1921|5|6|10|2|1921|87|1119|Friday|1921Q2|N|Y|N|2422842|2422992|2422486|2422761|N|N|N|N|N| +2422852|AAAAAAAAEEIPECAA|1921-06-11|257|1119|87|1921|6|6|11|2|1921|87|1119|Saturday|1921Q2|N|Y|N|2422842|2422992|2422487|2422762|N|N|N|N|N| +2422853|AAAAAAAAFEIPECAA|1921-06-12|257|1119|87|1921|0|6|12|2|1921|87|1119|Sunday|1921Q2|N|N|N|2422842|2422992|2422488|2422763|N|N|N|N|N| +2422854|AAAAAAAAGEIPECAA|1921-06-13|257|1119|87|1921|1|6|13|2|1921|87|1119|Monday|1921Q2|N|N|N|2422842|2422992|2422489|2422764|N|N|N|N|N| +2422855|AAAAAAAAHEIPECAA|1921-06-14|257|1120|87|1921|2|6|14|2|1921|87|1120|Tuesday|1921Q2|N|N|N|2422842|2422992|2422490|2422765|N|N|N|N|N| +2422856|AAAAAAAAIEIPECAA|1921-06-15|257|1120|87|1921|3|6|15|2|1921|87|1120|Wednesday|1921Q2|N|N|N|2422842|2422992|2422491|2422766|N|N|N|N|N| +2422857|AAAAAAAAJEIPECAA|1921-06-16|257|1120|87|1921|4|6|16|2|1921|87|1120|Thursday|1921Q2|N|N|N|2422842|2422992|2422492|2422767|N|N|N|N|N| +2422858|AAAAAAAAKEIPECAA|1921-06-17|257|1120|87|1921|5|6|17|2|1921|87|1120|Friday|1921Q2|N|Y|N|2422842|2422992|2422493|2422768|N|N|N|N|N| +2422859|AAAAAAAALEIPECAA|1921-06-18|257|1120|87|1921|6|6|18|2|1921|87|1120|Saturday|1921Q2|N|Y|N|2422842|2422992|2422494|2422769|N|N|N|N|N| +2422860|AAAAAAAAMEIPECAA|1921-06-19|257|1120|87|1921|0|6|19|2|1921|87|1120|Sunday|1921Q2|N|N|N|2422842|2422992|2422495|2422770|N|N|N|N|N| +2422861|AAAAAAAANEIPECAA|1921-06-20|257|1120|87|1921|1|6|20|2|1921|87|1120|Monday|1921Q2|N|N|N|2422842|2422992|2422496|2422771|N|N|N|N|N| +2422862|AAAAAAAAOEIPECAA|1921-06-21|257|1121|87|1921|2|6|21|2|1921|87|1121|Tuesday|1921Q2|N|N|N|2422842|2422992|2422497|2422772|N|N|N|N|N| +2422863|AAAAAAAAPEIPECAA|1921-06-22|257|1121|87|1921|3|6|22|2|1921|87|1121|Wednesday|1921Q2|N|N|N|2422842|2422992|2422498|2422773|N|N|N|N|N| +2422864|AAAAAAAAAFIPECAA|1921-06-23|257|1121|87|1921|4|6|23|2|1921|87|1121|Thursday|1921Q2|N|N|N|2422842|2422992|2422499|2422774|N|N|N|N|N| +2422865|AAAAAAAABFIPECAA|1921-06-24|257|1121|87|1921|5|6|24|2|1921|87|1121|Friday|1921Q2|N|Y|N|2422842|2422992|2422500|2422775|N|N|N|N|N| +2422866|AAAAAAAACFIPECAA|1921-06-25|257|1121|87|1921|6|6|25|2|1921|87|1121|Saturday|1921Q2|N|Y|N|2422842|2422992|2422501|2422776|N|N|N|N|N| +2422867|AAAAAAAADFIPECAA|1921-06-26|257|1121|87|1921|0|6|26|2|1921|87|1121|Sunday|1921Q2|N|N|N|2422842|2422992|2422502|2422777|N|N|N|N|N| +2422868|AAAAAAAAEFIPECAA|1921-06-27|257|1121|87|1921|1|6|27|2|1921|87|1121|Monday|1921Q2|N|N|N|2422842|2422992|2422503|2422778|N|N|N|N|N| +2422869|AAAAAAAAFFIPECAA|1921-06-28|257|1122|87|1921|2|6|28|2|1921|87|1122|Tuesday|1921Q2|N|N|N|2422842|2422992|2422504|2422779|N|N|N|N|N| +2422870|AAAAAAAAGFIPECAA|1921-06-29|257|1122|87|1921|3|6|29|2|1921|87|1122|Wednesday|1921Q2|N|N|N|2422842|2422992|2422505|2422780|N|N|N|N|N| +2422871|AAAAAAAAHFIPECAA|1921-06-30|257|1122|87|1921|4|6|30|2|1921|87|1122|Thursday|1921Q2|N|N|N|2422842|2422992|2422506|2422781|N|N|N|N|N| +2422872|AAAAAAAAIFIPECAA|1921-07-01|258|1122|87|1921|5|7|1|2|1921|87|1122|Friday|1921Q2|N|Y|N|2422872|2423052|2422507|2422781|N|N|N|N|N| +2422873|AAAAAAAAJFIPECAA|1921-07-02|258|1122|87|1921|6|7|2|3|1921|87|1122|Saturday|1921Q3|N|Y|N|2422872|2423052|2422508|2422782|N|N|N|N|N| +2422874|AAAAAAAAKFIPECAA|1921-07-03|258|1122|87|1921|0|7|3|3|1921|87|1122|Sunday|1921Q3|N|N|N|2422872|2423052|2422509|2422783|N|N|N|N|N| +2422875|AAAAAAAALFIPECAA|1921-07-04|258|1122|87|1921|1|7|4|3|1921|87|1122|Monday|1921Q3|N|N|N|2422872|2423052|2422510|2422784|N|N|N|N|N| +2422876|AAAAAAAAMFIPECAA|1921-07-05|258|1123|87|1921|2|7|5|3|1921|87|1123|Tuesday|1921Q3|Y|N|N|2422872|2423052|2422511|2422785|N|N|N|N|N| +2422877|AAAAAAAANFIPECAA|1921-07-06|258|1123|87|1921|3|7|6|3|1921|87|1123|Wednesday|1921Q3|N|N|Y|2422872|2423052|2422512|2422786|N|N|N|N|N| +2422878|AAAAAAAAOFIPECAA|1921-07-07|258|1123|87|1921|4|7|7|3|1921|87|1123|Thursday|1921Q3|N|N|N|2422872|2423052|2422513|2422787|N|N|N|N|N| +2422879|AAAAAAAAPFIPECAA|1921-07-08|258|1123|87|1921|5|7|8|3|1921|87|1123|Friday|1921Q3|N|Y|N|2422872|2423052|2422514|2422788|N|N|N|N|N| +2422880|AAAAAAAAAGIPECAA|1921-07-09|258|1123|87|1921|6|7|9|3|1921|87|1123|Saturday|1921Q3|N|Y|N|2422872|2423052|2422515|2422789|N|N|N|N|N| +2422881|AAAAAAAABGIPECAA|1921-07-10|258|1123|87|1921|0|7|10|3|1921|87|1123|Sunday|1921Q3|N|N|N|2422872|2423052|2422516|2422790|N|N|N|N|N| +2422882|AAAAAAAACGIPECAA|1921-07-11|258|1123|87|1921|1|7|11|3|1921|87|1123|Monday|1921Q3|N|N|N|2422872|2423052|2422517|2422791|N|N|N|N|N| +2422883|AAAAAAAADGIPECAA|1921-07-12|258|1124|87|1921|2|7|12|3|1921|87|1124|Tuesday|1921Q3|N|N|N|2422872|2423052|2422518|2422792|N|N|N|N|N| +2422884|AAAAAAAAEGIPECAA|1921-07-13|258|1124|87|1921|3|7|13|3|1921|87|1124|Wednesday|1921Q3|N|N|N|2422872|2423052|2422519|2422793|N|N|N|N|N| +2422885|AAAAAAAAFGIPECAA|1921-07-14|258|1124|87|1921|4|7|14|3|1921|87|1124|Thursday|1921Q3|N|N|N|2422872|2423052|2422520|2422794|N|N|N|N|N| +2422886|AAAAAAAAGGIPECAA|1921-07-15|258|1124|87|1921|5|7|15|3|1921|87|1124|Friday|1921Q3|N|Y|N|2422872|2423052|2422521|2422795|N|N|N|N|N| +2422887|AAAAAAAAHGIPECAA|1921-07-16|258|1124|87|1921|6|7|16|3|1921|87|1124|Saturday|1921Q3|N|Y|N|2422872|2423052|2422522|2422796|N|N|N|N|N| +2422888|AAAAAAAAIGIPECAA|1921-07-17|258|1124|87|1921|0|7|17|3|1921|87|1124|Sunday|1921Q3|N|N|N|2422872|2423052|2422523|2422797|N|N|N|N|N| +2422889|AAAAAAAAJGIPECAA|1921-07-18|258|1124|87|1921|1|7|18|3|1921|87|1124|Monday|1921Q3|N|N|N|2422872|2423052|2422524|2422798|N|N|N|N|N| +2422890|AAAAAAAAKGIPECAA|1921-07-19|258|1125|87|1921|2|7|19|3|1921|87|1125|Tuesday|1921Q3|N|N|N|2422872|2423052|2422525|2422799|N|N|N|N|N| +2422891|AAAAAAAALGIPECAA|1921-07-20|258|1125|87|1921|3|7|20|3|1921|87|1125|Wednesday|1921Q3|N|N|N|2422872|2423052|2422526|2422800|N|N|N|N|N| +2422892|AAAAAAAAMGIPECAA|1921-07-21|258|1125|87|1921|4|7|21|3|1921|87|1125|Thursday|1921Q3|N|N|N|2422872|2423052|2422527|2422801|N|N|N|N|N| +2422893|AAAAAAAANGIPECAA|1921-07-22|258|1125|87|1921|5|7|22|3|1921|87|1125|Friday|1921Q3|N|Y|N|2422872|2423052|2422528|2422802|N|N|N|N|N| +2422894|AAAAAAAAOGIPECAA|1921-07-23|258|1125|87|1921|6|7|23|3|1921|87|1125|Saturday|1921Q3|N|Y|N|2422872|2423052|2422529|2422803|N|N|N|N|N| +2422895|AAAAAAAAPGIPECAA|1921-07-24|258|1125|87|1921|0|7|24|3|1921|87|1125|Sunday|1921Q3|N|N|N|2422872|2423052|2422530|2422804|N|N|N|N|N| +2422896|AAAAAAAAAHIPECAA|1921-07-25|258|1125|87|1921|1|7|25|3|1921|87|1125|Monday|1921Q3|N|N|N|2422872|2423052|2422531|2422805|N|N|N|N|N| +2422897|AAAAAAAABHIPECAA|1921-07-26|258|1126|87|1921|2|7|26|3|1921|87|1126|Tuesday|1921Q3|N|N|N|2422872|2423052|2422532|2422806|N|N|N|N|N| +2422898|AAAAAAAACHIPECAA|1921-07-27|258|1126|87|1921|3|7|27|3|1921|87|1126|Wednesday|1921Q3|N|N|N|2422872|2423052|2422533|2422807|N|N|N|N|N| +2422899|AAAAAAAADHIPECAA|1921-07-28|258|1126|87|1921|4|7|28|3|1921|87|1126|Thursday|1921Q3|N|N|N|2422872|2423052|2422534|2422808|N|N|N|N|N| +2422900|AAAAAAAAEHIPECAA|1921-07-29|258|1126|87|1921|5|7|29|3|1921|87|1126|Friday|1921Q3|N|Y|N|2422872|2423052|2422535|2422809|N|N|N|N|N| +2422901|AAAAAAAAFHIPECAA|1921-07-30|258|1126|87|1921|6|7|30|3|1921|87|1126|Saturday|1921Q3|N|Y|N|2422872|2423052|2422536|2422810|N|N|N|N|N| +2422902|AAAAAAAAGHIPECAA|1921-07-31|258|1126|87|1921|0|7|31|3|1921|87|1126|Sunday|1921Q3|N|N|N|2422872|2423052|2422537|2422811|N|N|N|N|N| +2422903|AAAAAAAAHHIPECAA|1921-08-01|259|1126|87|1921|1|8|1|3|1921|87|1126|Monday|1921Q3|N|N|N|2422903|2423114|2422538|2422812|N|N|N|N|N| +2422904|AAAAAAAAIHIPECAA|1921-08-02|259|1127|87|1921|2|8|2|3|1921|87|1127|Tuesday|1921Q3|N|N|N|2422903|2423114|2422539|2422813|N|N|N|N|N| +2422905|AAAAAAAAJHIPECAA|1921-08-03|259|1127|87|1921|3|8|3|3|1921|87|1127|Wednesday|1921Q3|N|N|N|2422903|2423114|2422540|2422814|N|N|N|N|N| +2422906|AAAAAAAAKHIPECAA|1921-08-04|259|1127|87|1921|4|8|4|3|1921|87|1127|Thursday|1921Q3|N|N|N|2422903|2423114|2422541|2422815|N|N|N|N|N| +2422907|AAAAAAAALHIPECAA|1921-08-05|259|1127|87|1921|5|8|5|3|1921|87|1127|Friday|1921Q3|N|Y|N|2422903|2423114|2422542|2422816|N|N|N|N|N| +2422908|AAAAAAAAMHIPECAA|1921-08-06|259|1127|87|1921|6|8|6|3|1921|87|1127|Saturday|1921Q3|N|Y|N|2422903|2423114|2422543|2422817|N|N|N|N|N| +2422909|AAAAAAAANHIPECAA|1921-08-07|259|1127|87|1921|0|8|7|3|1921|87|1127|Sunday|1921Q3|N|N|N|2422903|2423114|2422544|2422818|N|N|N|N|N| +2422910|AAAAAAAAOHIPECAA|1921-08-08|259|1127|87|1921|1|8|8|3|1921|87|1127|Monday|1921Q3|N|N|N|2422903|2423114|2422545|2422819|N|N|N|N|N| +2422911|AAAAAAAAPHIPECAA|1921-08-09|259|1128|87|1921|2|8|9|3|1921|87|1128|Tuesday|1921Q3|N|N|N|2422903|2423114|2422546|2422820|N|N|N|N|N| +2422912|AAAAAAAAAIIPECAA|1921-08-10|259|1128|87|1921|3|8|10|3|1921|87|1128|Wednesday|1921Q3|N|N|N|2422903|2423114|2422547|2422821|N|N|N|N|N| +2422913|AAAAAAAABIIPECAA|1921-08-11|259|1128|87|1921|4|8|11|3|1921|87|1128|Thursday|1921Q3|N|N|N|2422903|2423114|2422548|2422822|N|N|N|N|N| +2422914|AAAAAAAACIIPECAA|1921-08-12|259|1128|87|1921|5|8|12|3|1921|87|1128|Friday|1921Q3|N|Y|N|2422903|2423114|2422549|2422823|N|N|N|N|N| +2422915|AAAAAAAADIIPECAA|1921-08-13|259|1128|87|1921|6|8|13|3|1921|87|1128|Saturday|1921Q3|N|Y|N|2422903|2423114|2422550|2422824|N|N|N|N|N| +2422916|AAAAAAAAEIIPECAA|1921-08-14|259|1128|87|1921|0|8|14|3|1921|87|1128|Sunday|1921Q3|N|N|N|2422903|2423114|2422551|2422825|N|N|N|N|N| +2422917|AAAAAAAAFIIPECAA|1921-08-15|259|1128|87|1921|1|8|15|3|1921|87|1128|Monday|1921Q3|N|N|N|2422903|2423114|2422552|2422826|N|N|N|N|N| +2422918|AAAAAAAAGIIPECAA|1921-08-16|259|1129|87|1921|2|8|16|3|1921|87|1129|Tuesday|1921Q3|N|N|N|2422903|2423114|2422553|2422827|N|N|N|N|N| +2422919|AAAAAAAAHIIPECAA|1921-08-17|259|1129|87|1921|3|8|17|3|1921|87|1129|Wednesday|1921Q3|N|N|N|2422903|2423114|2422554|2422828|N|N|N|N|N| +2422920|AAAAAAAAIIIPECAA|1921-08-18|259|1129|87|1921|4|8|18|3|1921|87|1129|Thursday|1921Q3|N|N|N|2422903|2423114|2422555|2422829|N|N|N|N|N| +2422921|AAAAAAAAJIIPECAA|1921-08-19|259|1129|87|1921|5|8|19|3|1921|87|1129|Friday|1921Q3|N|Y|N|2422903|2423114|2422556|2422830|N|N|N|N|N| +2422922|AAAAAAAAKIIPECAA|1921-08-20|259|1129|87|1921|6|8|20|3|1921|87|1129|Saturday|1921Q3|N|Y|N|2422903|2423114|2422557|2422831|N|N|N|N|N| +2422923|AAAAAAAALIIPECAA|1921-08-21|259|1129|87|1921|0|8|21|3|1921|87|1129|Sunday|1921Q3|N|N|N|2422903|2423114|2422558|2422832|N|N|N|N|N| +2422924|AAAAAAAAMIIPECAA|1921-08-22|259|1129|87|1921|1|8|22|3|1921|87|1129|Monday|1921Q3|N|N|N|2422903|2423114|2422559|2422833|N|N|N|N|N| +2422925|AAAAAAAANIIPECAA|1921-08-23|259|1130|87|1921|2|8|23|3|1921|87|1130|Tuesday|1921Q3|N|N|N|2422903|2423114|2422560|2422834|N|N|N|N|N| +2422926|AAAAAAAAOIIPECAA|1921-08-24|259|1130|87|1921|3|8|24|3|1921|87|1130|Wednesday|1921Q3|N|N|N|2422903|2423114|2422561|2422835|N|N|N|N|N| +2422927|AAAAAAAAPIIPECAA|1921-08-25|259|1130|87|1921|4|8|25|3|1921|87|1130|Thursday|1921Q3|N|N|N|2422903|2423114|2422562|2422836|N|N|N|N|N| +2422928|AAAAAAAAAJIPECAA|1921-08-26|259|1130|87|1921|5|8|26|3|1921|87|1130|Friday|1921Q3|N|Y|N|2422903|2423114|2422563|2422837|N|N|N|N|N| +2422929|AAAAAAAABJIPECAA|1921-08-27|259|1130|87|1921|6|8|27|3|1921|87|1130|Saturday|1921Q3|N|Y|N|2422903|2423114|2422564|2422838|N|N|N|N|N| +2422930|AAAAAAAACJIPECAA|1921-08-28|259|1130|87|1921|0|8|28|3|1921|87|1130|Sunday|1921Q3|N|N|N|2422903|2423114|2422565|2422839|N|N|N|N|N| +2422931|AAAAAAAADJIPECAA|1921-08-29|259|1130|87|1921|1|8|29|3|1921|87|1130|Monday|1921Q3|N|N|N|2422903|2423114|2422566|2422840|N|N|N|N|N| +2422932|AAAAAAAAEJIPECAA|1921-08-30|259|1131|87|1921|2|8|30|3|1921|87|1131|Tuesday|1921Q3|N|N|N|2422903|2423114|2422567|2422841|N|N|N|N|N| +2422933|AAAAAAAAFJIPECAA|1921-08-31|259|1131|87|1921|3|8|31|3|1921|87|1131|Wednesday|1921Q3|N|N|N|2422903|2423114|2422568|2422842|N|N|N|N|N| +2422934|AAAAAAAAGJIPECAA|1921-09-01|260|1131|88|1921|4|9|1|3|1921|88|1131|Thursday|1921Q3|N|N|N|2422934|2423176|2422569|2422843|N|N|N|N|N| +2422935|AAAAAAAAHJIPECAA|1921-09-02|260|1131|88|1921|5|9|2|3|1921|88|1131|Friday|1921Q3|N|Y|N|2422934|2423176|2422570|2422844|N|N|N|N|N| +2422936|AAAAAAAAIJIPECAA|1921-09-03|260|1131|88|1921|6|9|3|3|1921|88|1131|Saturday|1921Q3|N|Y|N|2422934|2423176|2422571|2422845|N|N|N|N|N| +2422937|AAAAAAAAJJIPECAA|1921-09-04|260|1131|88|1921|0|9|4|3|1921|88|1131|Sunday|1921Q3|N|N|N|2422934|2423176|2422572|2422846|N|N|N|N|N| +2422938|AAAAAAAAKJIPECAA|1921-09-05|260|1131|88|1921|1|9|5|3|1921|88|1131|Monday|1921Q3|N|N|N|2422934|2423176|2422573|2422847|N|N|N|N|N| +2422939|AAAAAAAALJIPECAA|1921-09-06|260|1132|88|1921|2|9|6|3|1921|88|1132|Tuesday|1921Q3|N|N|N|2422934|2423176|2422574|2422848|N|N|N|N|N| +2422940|AAAAAAAAMJIPECAA|1921-09-07|260|1132|88|1921|3|9|7|3|1921|88|1132|Wednesday|1921Q3|N|N|N|2422934|2423176|2422575|2422849|N|N|N|N|N| +2422941|AAAAAAAANJIPECAA|1921-09-08|260|1132|88|1921|4|9|8|3|1921|88|1132|Thursday|1921Q3|N|N|N|2422934|2423176|2422576|2422850|N|N|N|N|N| +2422942|AAAAAAAAOJIPECAA|1921-09-09|260|1132|88|1921|5|9|9|3|1921|88|1132|Friday|1921Q3|N|Y|N|2422934|2423176|2422577|2422851|N|N|N|N|N| +2422943|AAAAAAAAPJIPECAA|1921-09-10|260|1132|88|1921|6|9|10|3|1921|88|1132|Saturday|1921Q3|N|Y|N|2422934|2423176|2422578|2422852|N|N|N|N|N| +2422944|AAAAAAAAAKIPECAA|1921-09-11|260|1132|88|1921|0|9|11|3|1921|88|1132|Sunday|1921Q3|N|N|N|2422934|2423176|2422579|2422853|N|N|N|N|N| +2422945|AAAAAAAABKIPECAA|1921-09-12|260|1132|88|1921|1|9|12|3|1921|88|1132|Monday|1921Q3|N|N|N|2422934|2423176|2422580|2422854|N|N|N|N|N| +2422946|AAAAAAAACKIPECAA|1921-09-13|260|1133|88|1921|2|9|13|3|1921|88|1133|Tuesday|1921Q3|N|N|N|2422934|2423176|2422581|2422855|N|N|N|N|N| +2422947|AAAAAAAADKIPECAA|1921-09-14|260|1133|88|1921|3|9|14|3|1921|88|1133|Wednesday|1921Q3|N|N|N|2422934|2423176|2422582|2422856|N|N|N|N|N| +2422948|AAAAAAAAEKIPECAA|1921-09-15|260|1133|88|1921|4|9|15|3|1921|88|1133|Thursday|1921Q3|N|N|N|2422934|2423176|2422583|2422857|N|N|N|N|N| +2422949|AAAAAAAAFKIPECAA|1921-09-16|260|1133|88|1921|5|9|16|3|1921|88|1133|Friday|1921Q3|N|Y|N|2422934|2423176|2422584|2422858|N|N|N|N|N| +2422950|AAAAAAAAGKIPECAA|1921-09-17|260|1133|88|1921|6|9|17|3|1921|88|1133|Saturday|1921Q3|N|Y|N|2422934|2423176|2422585|2422859|N|N|N|N|N| +2422951|AAAAAAAAHKIPECAA|1921-09-18|260|1133|88|1921|0|9|18|3|1921|88|1133|Sunday|1921Q3|N|N|N|2422934|2423176|2422586|2422860|N|N|N|N|N| +2422952|AAAAAAAAIKIPECAA|1921-09-19|260|1133|88|1921|1|9|19|3|1921|88|1133|Monday|1921Q3|N|N|N|2422934|2423176|2422587|2422861|N|N|N|N|N| +2422953|AAAAAAAAJKIPECAA|1921-09-20|260|1134|88|1921|2|9|20|3|1921|88|1134|Tuesday|1921Q3|N|N|N|2422934|2423176|2422588|2422862|N|N|N|N|N| +2422954|AAAAAAAAKKIPECAA|1921-09-21|260|1134|88|1921|3|9|21|3|1921|88|1134|Wednesday|1921Q3|N|N|N|2422934|2423176|2422589|2422863|N|N|N|N|N| +2422955|AAAAAAAALKIPECAA|1921-09-22|260|1134|88|1921|4|9|22|3|1921|88|1134|Thursday|1921Q3|N|N|N|2422934|2423176|2422590|2422864|N|N|N|N|N| +2422956|AAAAAAAAMKIPECAA|1921-09-23|260|1134|88|1921|5|9|23|3|1921|88|1134|Friday|1921Q3|N|Y|N|2422934|2423176|2422591|2422865|N|N|N|N|N| +2422957|AAAAAAAANKIPECAA|1921-09-24|260|1134|88|1921|6|9|24|3|1921|88|1134|Saturday|1921Q3|N|Y|N|2422934|2423176|2422592|2422866|N|N|N|N|N| +2422958|AAAAAAAAOKIPECAA|1921-09-25|260|1134|88|1921|0|9|25|3|1921|88|1134|Sunday|1921Q3|N|N|N|2422934|2423176|2422593|2422867|N|N|N|N|N| +2422959|AAAAAAAAPKIPECAA|1921-09-26|260|1134|88|1921|1|9|26|3|1921|88|1134|Monday|1921Q3|N|N|N|2422934|2423176|2422594|2422868|N|N|N|N|N| +2422960|AAAAAAAAALIPECAA|1921-09-27|260|1135|88|1921|2|9|27|3|1921|88|1135|Tuesday|1921Q3|N|N|N|2422934|2423176|2422595|2422869|N|N|N|N|N| +2422961|AAAAAAAABLIPECAA|1921-09-28|260|1135|88|1921|3|9|28|3|1921|88|1135|Wednesday|1921Q3|N|N|N|2422934|2423176|2422596|2422870|N|N|N|N|N| +2422962|AAAAAAAACLIPECAA|1921-09-29|260|1135|88|1921|4|9|29|3|1921|88|1135|Thursday|1921Q3|N|N|N|2422934|2423176|2422597|2422871|N|N|N|N|N| +2422963|AAAAAAAADLIPECAA|1921-09-30|260|1135|88|1921|5|9|30|3|1921|88|1135|Friday|1921Q3|N|Y|N|2422934|2423176|2422598|2422872|N|N|N|N|N| +2422964|AAAAAAAAELIPECAA|1921-10-01|261|1135|88|1921|6|10|1|3|1921|88|1135|Saturday|1921Q3|N|Y|N|2422964|2423236|2422599|2422872|N|N|N|N|N| +2422965|AAAAAAAAFLIPECAA|1921-10-02|261|1135|88|1921|0|10|2|4|1921|88|1135|Sunday|1921Q4|N|N|N|2422964|2423236|2422600|2422873|N|N|N|N|N| +2422966|AAAAAAAAGLIPECAA|1921-10-03|261|1135|88|1921|1|10|3|4|1921|88|1135|Monday|1921Q4|N|N|N|2422964|2423236|2422601|2422874|N|N|N|N|N| +2422967|AAAAAAAAHLIPECAA|1921-10-04|261|1136|88|1921|2|10|4|4|1921|88|1136|Tuesday|1921Q4|N|N|N|2422964|2423236|2422602|2422875|N|N|N|N|N| +2422968|AAAAAAAAILIPECAA|1921-10-05|261|1136|88|1921|3|10|5|4|1921|88|1136|Wednesday|1921Q4|N|N|N|2422964|2423236|2422603|2422876|N|N|N|N|N| +2422969|AAAAAAAAJLIPECAA|1921-10-06|261|1136|88|1921|4|10|6|4|1921|88|1136|Thursday|1921Q4|N|N|N|2422964|2423236|2422604|2422877|N|N|N|N|N| +2422970|AAAAAAAAKLIPECAA|1921-10-07|261|1136|88|1921|5|10|7|4|1921|88|1136|Friday|1921Q4|N|Y|N|2422964|2423236|2422605|2422878|N|N|N|N|N| +2422971|AAAAAAAALLIPECAA|1921-10-08|261|1136|88|1921|6|10|8|4|1921|88|1136|Saturday|1921Q4|N|Y|N|2422964|2423236|2422606|2422879|N|N|N|N|N| +2422972|AAAAAAAAMLIPECAA|1921-10-09|261|1136|88|1921|0|10|9|4|1921|88|1136|Sunday|1921Q4|N|N|N|2422964|2423236|2422607|2422880|N|N|N|N|N| +2422973|AAAAAAAANLIPECAA|1921-10-10|261|1136|88|1921|1|10|10|4|1921|88|1136|Monday|1921Q4|N|N|N|2422964|2423236|2422608|2422881|N|N|N|N|N| +2422974|AAAAAAAAOLIPECAA|1921-10-11|261|1137|88|1921|2|10|11|4|1921|88|1137|Tuesday|1921Q4|N|N|N|2422964|2423236|2422609|2422882|N|N|N|N|N| +2422975|AAAAAAAAPLIPECAA|1921-10-12|261|1137|88|1921|3|10|12|4|1921|88|1137|Wednesday|1921Q4|N|N|N|2422964|2423236|2422610|2422883|N|N|N|N|N| +2422976|AAAAAAAAAMIPECAA|1921-10-13|261|1137|88|1921|4|10|13|4|1921|88|1137|Thursday|1921Q4|N|N|N|2422964|2423236|2422611|2422884|N|N|N|N|N| +2422977|AAAAAAAABMIPECAA|1921-10-14|261|1137|88|1921|5|10|14|4|1921|88|1137|Friday|1921Q4|N|Y|N|2422964|2423236|2422612|2422885|N|N|N|N|N| +2422978|AAAAAAAACMIPECAA|1921-10-15|261|1137|88|1921|6|10|15|4|1921|88|1137|Saturday|1921Q4|N|Y|N|2422964|2423236|2422613|2422886|N|N|N|N|N| +2422979|AAAAAAAADMIPECAA|1921-10-16|261|1137|88|1921|0|10|16|4|1921|88|1137|Sunday|1921Q4|N|N|N|2422964|2423236|2422614|2422887|N|N|N|N|N| +2422980|AAAAAAAAEMIPECAA|1921-10-17|261|1137|88|1921|1|10|17|4|1921|88|1137|Monday|1921Q4|N|N|N|2422964|2423236|2422615|2422888|N|N|N|N|N| +2422981|AAAAAAAAFMIPECAA|1921-10-18|261|1138|88|1921|2|10|18|4|1921|88|1138|Tuesday|1921Q4|N|N|N|2422964|2423236|2422616|2422889|N|N|N|N|N| +2422982|AAAAAAAAGMIPECAA|1921-10-19|261|1138|88|1921|3|10|19|4|1921|88|1138|Wednesday|1921Q4|N|N|N|2422964|2423236|2422617|2422890|N|N|N|N|N| +2422983|AAAAAAAAHMIPECAA|1921-10-20|261|1138|88|1921|4|10|20|4|1921|88|1138|Thursday|1921Q4|N|N|N|2422964|2423236|2422618|2422891|N|N|N|N|N| +2422984|AAAAAAAAIMIPECAA|1921-10-21|261|1138|88|1921|5|10|21|4|1921|88|1138|Friday|1921Q4|N|Y|N|2422964|2423236|2422619|2422892|N|N|N|N|N| +2422985|AAAAAAAAJMIPECAA|1921-10-22|261|1138|88|1921|6|10|22|4|1921|88|1138|Saturday|1921Q4|N|Y|N|2422964|2423236|2422620|2422893|N|N|N|N|N| +2422986|AAAAAAAAKMIPECAA|1921-10-23|261|1138|88|1921|0|10|23|4|1921|88|1138|Sunday|1921Q4|N|N|N|2422964|2423236|2422621|2422894|N|N|N|N|N| +2422987|AAAAAAAALMIPECAA|1921-10-24|261|1138|88|1921|1|10|24|4|1921|88|1138|Monday|1921Q4|N|N|N|2422964|2423236|2422622|2422895|N|N|N|N|N| +2422988|AAAAAAAAMMIPECAA|1921-10-25|261|1139|88|1921|2|10|25|4|1921|88|1139|Tuesday|1921Q4|N|N|N|2422964|2423236|2422623|2422896|N|N|N|N|N| +2422989|AAAAAAAANMIPECAA|1921-10-26|261|1139|88|1921|3|10|26|4|1921|88|1139|Wednesday|1921Q4|N|N|N|2422964|2423236|2422624|2422897|N|N|N|N|N| +2422990|AAAAAAAAOMIPECAA|1921-10-27|261|1139|88|1921|4|10|27|4|1921|88|1139|Thursday|1921Q4|N|N|N|2422964|2423236|2422625|2422898|N|N|N|N|N| +2422991|AAAAAAAAPMIPECAA|1921-10-28|261|1139|88|1921|5|10|28|4|1921|88|1139|Friday|1921Q4|N|Y|N|2422964|2423236|2422626|2422899|N|N|N|N|N| +2422992|AAAAAAAAANIPECAA|1921-10-29|261|1139|88|1921|6|10|29|4|1921|88|1139|Saturday|1921Q4|N|Y|N|2422964|2423236|2422627|2422900|N|N|N|N|N| +2422993|AAAAAAAABNIPECAA|1921-10-30|261|1139|88|1921|0|10|30|4|1921|88|1139|Sunday|1921Q4|N|N|N|2422964|2423236|2422628|2422901|N|N|N|N|N| +2422994|AAAAAAAACNIPECAA|1921-10-31|261|1139|88|1921|1|10|31|4|1921|88|1139|Monday|1921Q4|N|N|N|2422964|2423236|2422629|2422902|N|N|N|N|N| +2422995|AAAAAAAADNIPECAA|1921-11-01|262|1140|88|1921|2|11|1|4|1921|88|1140|Tuesday|1921Q4|N|N|N|2422995|2423298|2422630|2422903|N|N|N|N|N| +2422996|AAAAAAAAENIPECAA|1921-11-02|262|1140|88|1921|3|11|2|4|1921|88|1140|Wednesday|1921Q4|N|N|N|2422995|2423298|2422631|2422904|N|N|N|N|N| +2422997|AAAAAAAAFNIPECAA|1921-11-03|262|1140|88|1921|4|11|3|4|1921|88|1140|Thursday|1921Q4|N|N|N|2422995|2423298|2422632|2422905|N|N|N|N|N| +2422998|AAAAAAAAGNIPECAA|1921-11-04|262|1140|88|1921|5|11|4|4|1921|88|1140|Friday|1921Q4|N|Y|N|2422995|2423298|2422633|2422906|N|N|N|N|N| +2422999|AAAAAAAAHNIPECAA|1921-11-05|262|1140|88|1921|6|11|5|4|1921|88|1140|Saturday|1921Q4|N|Y|N|2422995|2423298|2422634|2422907|N|N|N|N|N| +2423000|AAAAAAAAINIPECAA|1921-11-06|262|1140|88|1921|0|11|6|4|1921|88|1140|Sunday|1921Q4|N|N|N|2422995|2423298|2422635|2422908|N|N|N|N|N| +2423001|AAAAAAAAJNIPECAA|1921-11-07|262|1140|88|1921|1|11|7|4|1921|88|1140|Monday|1921Q4|N|N|N|2422995|2423298|2422636|2422909|N|N|N|N|N| +2423002|AAAAAAAAKNIPECAA|1921-11-08|262|1141|88|1921|2|11|8|4|1921|88|1141|Tuesday|1921Q4|N|N|N|2422995|2423298|2422637|2422910|N|N|N|N|N| +2423003|AAAAAAAALNIPECAA|1921-11-09|262|1141|88|1921|3|11|9|4|1921|88|1141|Wednesday|1921Q4|N|N|N|2422995|2423298|2422638|2422911|N|N|N|N|N| +2423004|AAAAAAAAMNIPECAA|1921-11-10|262|1141|88|1921|4|11|10|4|1921|88|1141|Thursday|1921Q4|N|N|N|2422995|2423298|2422639|2422912|N|N|N|N|N| +2423005|AAAAAAAANNIPECAA|1921-11-11|262|1141|88|1921|5|11|11|4|1921|88|1141|Friday|1921Q4|N|Y|N|2422995|2423298|2422640|2422913|N|N|N|N|N| +2423006|AAAAAAAAONIPECAA|1921-11-12|262|1141|88|1921|6|11|12|4|1921|88|1141|Saturday|1921Q4|N|Y|N|2422995|2423298|2422641|2422914|N|N|N|N|N| +2423007|AAAAAAAAPNIPECAA|1921-11-13|262|1141|88|1921|0|11|13|4|1921|88|1141|Sunday|1921Q4|N|N|N|2422995|2423298|2422642|2422915|N|N|N|N|N| +2423008|AAAAAAAAAOIPECAA|1921-11-14|262|1141|88|1921|1|11|14|4|1921|88|1141|Monday|1921Q4|N|N|N|2422995|2423298|2422643|2422916|N|N|N|N|N| +2423009|AAAAAAAABOIPECAA|1921-11-15|262|1142|88|1921|2|11|15|4|1921|88|1142|Tuesday|1921Q4|N|N|N|2422995|2423298|2422644|2422917|N|N|N|N|N| +2423010|AAAAAAAACOIPECAA|1921-11-16|262|1142|88|1921|3|11|16|4|1921|88|1142|Wednesday|1921Q4|N|N|N|2422995|2423298|2422645|2422918|N|N|N|N|N| +2423011|AAAAAAAADOIPECAA|1921-11-17|262|1142|88|1921|4|11|17|4|1921|88|1142|Thursday|1921Q4|N|N|N|2422995|2423298|2422646|2422919|N|N|N|N|N| +2423012|AAAAAAAAEOIPECAA|1921-11-18|262|1142|88|1921|5|11|18|4|1921|88|1142|Friday|1921Q4|N|Y|N|2422995|2423298|2422647|2422920|N|N|N|N|N| +2423013|AAAAAAAAFOIPECAA|1921-11-19|262|1142|88|1921|6|11|19|4|1921|88|1142|Saturday|1921Q4|N|Y|N|2422995|2423298|2422648|2422921|N|N|N|N|N| +2423014|AAAAAAAAGOIPECAA|1921-11-20|262|1142|88|1921|0|11|20|4|1921|88|1142|Sunday|1921Q4|N|N|N|2422995|2423298|2422649|2422922|N|N|N|N|N| +2423015|AAAAAAAAHOIPECAA|1921-11-21|262|1142|88|1921|1|11|21|4|1921|88|1142|Monday|1921Q4|N|N|N|2422995|2423298|2422650|2422923|N|N|N|N|N| +2423016|AAAAAAAAIOIPECAA|1921-11-22|262|1143|88|1921|2|11|22|4|1921|88|1143|Tuesday|1921Q4|N|N|N|2422995|2423298|2422651|2422924|N|N|N|N|N| +2423017|AAAAAAAAJOIPECAA|1921-11-23|262|1143|88|1921|3|11|23|4|1921|88|1143|Wednesday|1921Q4|N|N|N|2422995|2423298|2422652|2422925|N|N|N|N|N| +2423018|AAAAAAAAKOIPECAA|1921-11-24|262|1143|88|1921|4|11|24|4|1921|88|1143|Thursday|1921Q4|N|N|N|2422995|2423298|2422653|2422926|N|N|N|N|N| +2423019|AAAAAAAALOIPECAA|1921-11-25|262|1143|88|1921|5|11|25|4|1921|88|1143|Friday|1921Q4|N|Y|N|2422995|2423298|2422654|2422927|N|N|N|N|N| +2423020|AAAAAAAAMOIPECAA|1921-11-26|262|1143|88|1921|6|11|26|4|1921|88|1143|Saturday|1921Q4|N|Y|N|2422995|2423298|2422655|2422928|N|N|N|N|N| +2423021|AAAAAAAANOIPECAA|1921-11-27|262|1143|88|1921|0|11|27|4|1921|88|1143|Sunday|1921Q4|N|N|N|2422995|2423298|2422656|2422929|N|N|N|N|N| +2423022|AAAAAAAAOOIPECAA|1921-11-28|262|1143|88|1921|1|11|28|4|1921|88|1143|Monday|1921Q4|N|N|N|2422995|2423298|2422657|2422930|N|N|N|N|N| +2423023|AAAAAAAAPOIPECAA|1921-11-29|262|1144|88|1921|2|11|29|4|1921|88|1144|Tuesday|1921Q4|N|N|N|2422995|2423298|2422658|2422931|N|N|N|N|N| +2423024|AAAAAAAAAPIPECAA|1921-11-30|262|1144|88|1921|3|11|30|4|1921|88|1144|Wednesday|1921Q4|N|N|N|2422995|2423298|2422659|2422932|N|N|N|N|N| +2423025|AAAAAAAABPIPECAA|1921-12-01|263|1144|89|1921|4|12|1|4|1921|89|1144|Thursday|1921Q4|N|N|N|2423025|2423358|2422660|2422933|N|N|N|N|N| +2423026|AAAAAAAACPIPECAA|1921-12-02|263|1144|89|1921|5|12|2|4|1921|89|1144|Friday|1921Q4|N|Y|N|2423025|2423358|2422661|2422934|N|N|N|N|N| +2423027|AAAAAAAADPIPECAA|1921-12-03|263|1144|89|1921|6|12|3|4|1921|89|1144|Saturday|1921Q4|N|Y|N|2423025|2423358|2422662|2422935|N|N|N|N|N| +2423028|AAAAAAAAEPIPECAA|1921-12-04|263|1144|89|1921|0|12|4|4|1921|89|1144|Sunday|1921Q4|N|N|N|2423025|2423358|2422663|2422936|N|N|N|N|N| +2423029|AAAAAAAAFPIPECAA|1921-12-05|263|1144|89|1921|1|12|5|4|1921|89|1144|Monday|1921Q4|N|N|N|2423025|2423358|2422664|2422937|N|N|N|N|N| +2423030|AAAAAAAAGPIPECAA|1921-12-06|263|1145|89|1921|2|12|6|4|1921|89|1145|Tuesday|1921Q4|N|N|N|2423025|2423358|2422665|2422938|N|N|N|N|N| +2423031|AAAAAAAAHPIPECAA|1921-12-07|263|1145|89|1921|3|12|7|4|1921|89|1145|Wednesday|1921Q4|N|N|N|2423025|2423358|2422666|2422939|N|N|N|N|N| +2423032|AAAAAAAAIPIPECAA|1921-12-08|263|1145|89|1921|4|12|8|4|1921|89|1145|Thursday|1921Q4|N|N|N|2423025|2423358|2422667|2422940|N|N|N|N|N| +2423033|AAAAAAAAJPIPECAA|1921-12-09|263|1145|89|1921|5|12|9|4|1921|89|1145|Friday|1921Q4|N|Y|N|2423025|2423358|2422668|2422941|N|N|N|N|N| +2423034|AAAAAAAAKPIPECAA|1921-12-10|263|1145|89|1921|6|12|10|4|1921|89|1145|Saturday|1921Q4|N|Y|N|2423025|2423358|2422669|2422942|N|N|N|N|N| +2423035|AAAAAAAALPIPECAA|1921-12-11|263|1145|89|1921|0|12|11|4|1921|89|1145|Sunday|1921Q4|N|N|N|2423025|2423358|2422670|2422943|N|N|N|N|N| +2423036|AAAAAAAAMPIPECAA|1921-12-12|263|1145|89|1921|1|12|12|4|1921|89|1145|Monday|1921Q4|N|N|N|2423025|2423358|2422671|2422944|N|N|N|N|N| +2423037|AAAAAAAANPIPECAA|1921-12-13|263|1146|89|1921|2|12|13|4|1921|89|1146|Tuesday|1921Q4|N|N|N|2423025|2423358|2422672|2422945|N|N|N|N|N| +2423038|AAAAAAAAOPIPECAA|1921-12-14|263|1146|89|1921|3|12|14|4|1921|89|1146|Wednesday|1921Q4|N|N|N|2423025|2423358|2422673|2422946|N|N|N|N|N| +2423039|AAAAAAAAPPIPECAA|1921-12-15|263|1146|89|1921|4|12|15|4|1921|89|1146|Thursday|1921Q4|N|N|N|2423025|2423358|2422674|2422947|N|N|N|N|N| +2423040|AAAAAAAAAAJPECAA|1921-12-16|263|1146|89|1921|5|12|16|4|1921|89|1146|Friday|1921Q4|N|Y|N|2423025|2423358|2422675|2422948|N|N|N|N|N| +2423041|AAAAAAAABAJPECAA|1921-12-17|263|1146|89|1921|6|12|17|4|1921|89|1146|Saturday|1921Q4|N|Y|N|2423025|2423358|2422676|2422949|N|N|N|N|N| +2423042|AAAAAAAACAJPECAA|1921-12-18|263|1146|89|1921|0|12|18|4|1921|89|1146|Sunday|1921Q4|N|N|N|2423025|2423358|2422677|2422950|N|N|N|N|N| +2423043|AAAAAAAADAJPECAA|1921-12-19|263|1146|89|1921|1|12|19|4|1921|89|1146|Monday|1921Q4|N|N|N|2423025|2423358|2422678|2422951|N|N|N|N|N| +2423044|AAAAAAAAEAJPECAA|1921-12-20|263|1147|89|1921|2|12|20|4|1921|89|1147|Tuesday|1921Q4|N|N|N|2423025|2423358|2422679|2422952|N|N|N|N|N| +2423045|AAAAAAAAFAJPECAA|1921-12-21|263|1147|89|1921|3|12|21|4|1921|89|1147|Wednesday|1921Q4|N|N|N|2423025|2423358|2422680|2422953|N|N|N|N|N| +2423046|AAAAAAAAGAJPECAA|1921-12-22|263|1147|89|1921|4|12|22|4|1921|89|1147|Thursday|1921Q4|N|N|N|2423025|2423358|2422681|2422954|N|N|N|N|N| +2423047|AAAAAAAAHAJPECAA|1921-12-23|263|1147|89|1921|5|12|23|4|1921|89|1147|Friday|1921Q4|N|Y|N|2423025|2423358|2422682|2422955|N|N|N|N|N| +2423048|AAAAAAAAIAJPECAA|1921-12-24|263|1147|89|1921|6|12|24|4|1921|89|1147|Saturday|1921Q4|N|Y|N|2423025|2423358|2422683|2422956|N|N|N|N|N| +2423049|AAAAAAAAJAJPECAA|1921-12-25|263|1147|89|1921|0|12|25|4|1921|89|1147|Sunday|1921Q4|N|N|N|2423025|2423358|2422684|2422957|N|N|N|N|N| +2423050|AAAAAAAAKAJPECAA|1921-12-26|263|1147|89|1921|1|12|26|4|1921|89|1147|Monday|1921Q4|Y|N|N|2423025|2423358|2422685|2422958|N|N|N|N|N| +2423051|AAAAAAAALAJPECAA|1921-12-27|263|1148|89|1921|2|12|27|4|1921|89|1148|Tuesday|1921Q4|N|N|Y|2423025|2423358|2422686|2422959|N|N|N|N|N| +2423052|AAAAAAAAMAJPECAA|1921-12-28|263|1148|89|1921|3|12|28|4|1921|89|1148|Wednesday|1921Q4|N|N|N|2423025|2423358|2422687|2422960|N|N|N|N|N| +2423053|AAAAAAAANAJPECAA|1921-12-29|263|1148|89|1921|4|12|29|4|1921|89|1148|Thursday|1921Q4|N|N|N|2423025|2423358|2422688|2422961|N|N|N|N|N| +2423054|AAAAAAAAOAJPECAA|1921-12-30|263|1148|89|1921|5|12|30|4|1921|89|1148|Friday|1921Q4|N|Y|N|2423025|2423358|2422689|2422962|N|N|N|N|N| +2423055|AAAAAAAAPAJPECAA|1921-12-31|263|1148|89|1921|6|12|31|4|1921|89|1148|Saturday|1921Q4|N|Y|N|2423025|2423358|2422690|2422963|N|N|N|N|N| +2423056|AAAAAAAAABJPECAA|1922-01-01|264|1148|89|1922|0|1|1|1|1922|89|1148|Sunday|1922Q1|Y|N|N|2423056|2423055|2422691|2422964|N|N|N|N|N| +2423057|AAAAAAAABBJPECAA|1922-01-02|264|1148|89|1922|1|1|2|1|1922|89|1148|Monday|1922Q1|N|N|Y|2423056|2423055|2422692|2422965|N|N|N|N|N| +2423058|AAAAAAAACBJPECAA|1922-01-03|264|1149|89|1922|2|1|3|1|1922|89|1149|Tuesday|1922Q1|N|N|N|2423056|2423055|2422693|2422966|N|N|N|N|N| +2423059|AAAAAAAADBJPECAA|1922-01-04|264|1149|89|1922|3|1|4|1|1922|89|1149|Wednesday|1922Q1|N|N|N|2423056|2423055|2422694|2422967|N|N|N|N|N| +2423060|AAAAAAAAEBJPECAA|1922-01-05|264|1149|89|1922|4|1|5|1|1922|89|1149|Thursday|1922Q1|N|N|N|2423056|2423055|2422695|2422968|N|N|N|N|N| +2423061|AAAAAAAAFBJPECAA|1922-01-06|264|1149|89|1922|5|1|6|1|1922|89|1149|Friday|1922Q1|N|Y|N|2423056|2423055|2422696|2422969|N|N|N|N|N| +2423062|AAAAAAAAGBJPECAA|1922-01-07|264|1149|89|1922|6|1|7|1|1922|89|1149|Saturday|1922Q1|N|Y|N|2423056|2423055|2422697|2422970|N|N|N|N|N| +2423063|AAAAAAAAHBJPECAA|1922-01-08|264|1149|89|1922|0|1|8|1|1922|89|1149|Sunday|1922Q1|N|N|N|2423056|2423055|2422698|2422971|N|N|N|N|N| +2423064|AAAAAAAAIBJPECAA|1922-01-09|264|1149|89|1922|1|1|9|1|1922|89|1149|Monday|1922Q1|N|N|N|2423056|2423055|2422699|2422972|N|N|N|N|N| +2423065|AAAAAAAAJBJPECAA|1922-01-10|264|1150|89|1922|2|1|10|1|1922|89|1150|Tuesday|1922Q1|N|N|N|2423056|2423055|2422700|2422973|N|N|N|N|N| +2423066|AAAAAAAAKBJPECAA|1922-01-11|264|1150|89|1922|3|1|11|1|1922|89|1150|Wednesday|1922Q1|N|N|N|2423056|2423055|2422701|2422974|N|N|N|N|N| +2423067|AAAAAAAALBJPECAA|1922-01-12|264|1150|89|1922|4|1|12|1|1922|89|1150|Thursday|1922Q1|N|N|N|2423056|2423055|2422702|2422975|N|N|N|N|N| +2423068|AAAAAAAAMBJPECAA|1922-01-13|264|1150|89|1922|5|1|13|1|1922|89|1150|Friday|1922Q1|N|Y|N|2423056|2423055|2422703|2422976|N|N|N|N|N| +2423069|AAAAAAAANBJPECAA|1922-01-14|264|1150|89|1922|6|1|14|1|1922|89|1150|Saturday|1922Q1|N|Y|N|2423056|2423055|2422704|2422977|N|N|N|N|N| +2423070|AAAAAAAAOBJPECAA|1922-01-15|264|1150|89|1922|0|1|15|1|1922|89|1150|Sunday|1922Q1|N|N|N|2423056|2423055|2422705|2422978|N|N|N|N|N| +2423071|AAAAAAAAPBJPECAA|1922-01-16|264|1150|89|1922|1|1|16|1|1922|89|1150|Monday|1922Q1|N|N|N|2423056|2423055|2422706|2422979|N|N|N|N|N| +2423072|AAAAAAAAACJPECAA|1922-01-17|264|1151|89|1922|2|1|17|1|1922|89|1151|Tuesday|1922Q1|N|N|N|2423056|2423055|2422707|2422980|N|N|N|N|N| +2423073|AAAAAAAABCJPECAA|1922-01-18|264|1151|89|1922|3|1|18|1|1922|89|1151|Wednesday|1922Q1|N|N|N|2423056|2423055|2422708|2422981|N|N|N|N|N| +2423074|AAAAAAAACCJPECAA|1922-01-19|264|1151|89|1922|4|1|19|1|1922|89|1151|Thursday|1922Q1|N|N|N|2423056|2423055|2422709|2422982|N|N|N|N|N| +2423075|AAAAAAAADCJPECAA|1922-01-20|264|1151|89|1922|5|1|20|1|1922|89|1151|Friday|1922Q1|N|Y|N|2423056|2423055|2422710|2422983|N|N|N|N|N| +2423076|AAAAAAAAECJPECAA|1922-01-21|264|1151|89|1922|6|1|21|1|1922|89|1151|Saturday|1922Q1|N|Y|N|2423056|2423055|2422711|2422984|N|N|N|N|N| +2423077|AAAAAAAAFCJPECAA|1922-01-22|264|1151|89|1922|0|1|22|1|1922|89|1151|Sunday|1922Q1|N|N|N|2423056|2423055|2422712|2422985|N|N|N|N|N| +2423078|AAAAAAAAGCJPECAA|1922-01-23|264|1151|89|1922|1|1|23|1|1922|89|1151|Monday|1922Q1|N|N|N|2423056|2423055|2422713|2422986|N|N|N|N|N| +2423079|AAAAAAAAHCJPECAA|1922-01-24|264|1152|89|1922|2|1|24|1|1922|89|1152|Tuesday|1922Q1|N|N|N|2423056|2423055|2422714|2422987|N|N|N|N|N| +2423080|AAAAAAAAICJPECAA|1922-01-25|264|1152|89|1922|3|1|25|1|1922|89|1152|Wednesday|1922Q1|N|N|N|2423056|2423055|2422715|2422988|N|N|N|N|N| +2423081|AAAAAAAAJCJPECAA|1922-01-26|264|1152|89|1922|4|1|26|1|1922|89|1152|Thursday|1922Q1|N|N|N|2423056|2423055|2422716|2422989|N|N|N|N|N| +2423082|AAAAAAAAKCJPECAA|1922-01-27|264|1152|89|1922|5|1|27|1|1922|89|1152|Friday|1922Q1|N|Y|N|2423056|2423055|2422717|2422990|N|N|N|N|N| +2423083|AAAAAAAALCJPECAA|1922-01-28|264|1152|89|1922|6|1|28|1|1922|89|1152|Saturday|1922Q1|N|Y|N|2423056|2423055|2422718|2422991|N|N|N|N|N| +2423084|AAAAAAAAMCJPECAA|1922-01-29|264|1152|89|1922|0|1|29|1|1922|89|1152|Sunday|1922Q1|N|N|N|2423056|2423055|2422719|2422992|N|N|N|N|N| +2423085|AAAAAAAANCJPECAA|1922-01-30|264|1152|89|1922|1|1|30|1|1922|89|1152|Monday|1922Q1|N|N|N|2423056|2423055|2422720|2422993|N|N|N|N|N| +2423086|AAAAAAAAOCJPECAA|1922-01-31|264|1153|89|1922|2|1|31|1|1922|89|1153|Tuesday|1922Q1|N|N|N|2423056|2423055|2422721|2422994|N|N|N|N|N| +2423087|AAAAAAAAPCJPECAA|1922-02-01|265|1153|89|1922|3|2|1|1|1922|89|1153|Wednesday|1922Q1|N|N|N|2423087|2423117|2422722|2422995|N|N|N|N|N| +2423088|AAAAAAAAADJPECAA|1922-02-02|265|1153|89|1922|4|2|2|1|1922|89|1153|Thursday|1922Q1|N|N|N|2423087|2423117|2422723|2422996|N|N|N|N|N| +2423089|AAAAAAAABDJPECAA|1922-02-03|265|1153|89|1922|5|2|3|1|1922|89|1153|Friday|1922Q1|N|Y|N|2423087|2423117|2422724|2422997|N|N|N|N|N| +2423090|AAAAAAAACDJPECAA|1922-02-04|265|1153|89|1922|6|2|4|1|1922|89|1153|Saturday|1922Q1|N|Y|N|2423087|2423117|2422725|2422998|N|N|N|N|N| +2423091|AAAAAAAADDJPECAA|1922-02-05|265|1153|89|1922|0|2|5|1|1922|89|1153|Sunday|1922Q1|N|N|N|2423087|2423117|2422726|2422999|N|N|N|N|N| +2423092|AAAAAAAAEDJPECAA|1922-02-06|265|1153|89|1922|1|2|6|1|1922|89|1153|Monday|1922Q1|N|N|N|2423087|2423117|2422727|2423000|N|N|N|N|N| +2423093|AAAAAAAAFDJPECAA|1922-02-07|265|1154|89|1922|2|2|7|1|1922|89|1154|Tuesday|1922Q1|N|N|N|2423087|2423117|2422728|2423001|N|N|N|N|N| +2423094|AAAAAAAAGDJPECAA|1922-02-08|265|1154|89|1922|3|2|8|1|1922|89|1154|Wednesday|1922Q1|N|N|N|2423087|2423117|2422729|2423002|N|N|N|N|N| +2423095|AAAAAAAAHDJPECAA|1922-02-09|265|1154|89|1922|4|2|9|1|1922|89|1154|Thursday|1922Q1|N|N|N|2423087|2423117|2422730|2423003|N|N|N|N|N| +2423096|AAAAAAAAIDJPECAA|1922-02-10|265|1154|89|1922|5|2|10|1|1922|89|1154|Friday|1922Q1|N|Y|N|2423087|2423117|2422731|2423004|N|N|N|N|N| +2423097|AAAAAAAAJDJPECAA|1922-02-11|265|1154|89|1922|6|2|11|1|1922|89|1154|Saturday|1922Q1|N|Y|N|2423087|2423117|2422732|2423005|N|N|N|N|N| +2423098|AAAAAAAAKDJPECAA|1922-02-12|265|1154|89|1922|0|2|12|1|1922|89|1154|Sunday|1922Q1|N|N|N|2423087|2423117|2422733|2423006|N|N|N|N|N| +2423099|AAAAAAAALDJPECAA|1922-02-13|265|1154|89|1922|1|2|13|1|1922|89|1154|Monday|1922Q1|N|N|N|2423087|2423117|2422734|2423007|N|N|N|N|N| +2423100|AAAAAAAAMDJPECAA|1922-02-14|265|1155|89|1922|2|2|14|1|1922|89|1155|Tuesday|1922Q1|N|N|N|2423087|2423117|2422735|2423008|N|N|N|N|N| +2423101|AAAAAAAANDJPECAA|1922-02-15|265|1155|89|1922|3|2|15|1|1922|89|1155|Wednesday|1922Q1|N|N|N|2423087|2423117|2422736|2423009|N|N|N|N|N| +2423102|AAAAAAAAODJPECAA|1922-02-16|265|1155|89|1922|4|2|16|1|1922|89|1155|Thursday|1922Q1|N|N|N|2423087|2423117|2422737|2423010|N|N|N|N|N| +2423103|AAAAAAAAPDJPECAA|1922-02-17|265|1155|89|1922|5|2|17|1|1922|89|1155|Friday|1922Q1|N|Y|N|2423087|2423117|2422738|2423011|N|N|N|N|N| +2423104|AAAAAAAAAEJPECAA|1922-02-18|265|1155|89|1922|6|2|18|1|1922|89|1155|Saturday|1922Q1|N|Y|N|2423087|2423117|2422739|2423012|N|N|N|N|N| +2423105|AAAAAAAABEJPECAA|1922-02-19|265|1155|89|1922|0|2|19|1|1922|89|1155|Sunday|1922Q1|N|N|N|2423087|2423117|2422740|2423013|N|N|N|N|N| +2423106|AAAAAAAACEJPECAA|1922-02-20|265|1155|89|1922|1|2|20|1|1922|89|1155|Monday|1922Q1|N|N|N|2423087|2423117|2422741|2423014|N|N|N|N|N| +2423107|AAAAAAAADEJPECAA|1922-02-21|265|1156|89|1922|2|2|21|1|1922|89|1156|Tuesday|1922Q1|N|N|N|2423087|2423117|2422742|2423015|N|N|N|N|N| +2423108|AAAAAAAAEEJPECAA|1922-02-22|265|1156|89|1922|3|2|22|1|1922|89|1156|Wednesday|1922Q1|N|N|N|2423087|2423117|2422743|2423016|N|N|N|N|N| +2423109|AAAAAAAAFEJPECAA|1922-02-23|265|1156|89|1922|4|2|23|1|1922|89|1156|Thursday|1922Q1|N|N|N|2423087|2423117|2422744|2423017|N|N|N|N|N| +2423110|AAAAAAAAGEJPECAA|1922-02-24|265|1156|89|1922|5|2|24|1|1922|89|1156|Friday|1922Q1|N|Y|N|2423087|2423117|2422745|2423018|N|N|N|N|N| +2423111|AAAAAAAAHEJPECAA|1922-02-25|265|1156|89|1922|6|2|25|1|1922|89|1156|Saturday|1922Q1|N|Y|N|2423087|2423117|2422746|2423019|N|N|N|N|N| +2423112|AAAAAAAAIEJPECAA|1922-02-26|265|1156|89|1922|0|2|26|1|1922|89|1156|Sunday|1922Q1|N|N|N|2423087|2423117|2422747|2423020|N|N|N|N|N| +2423113|AAAAAAAAJEJPECAA|1922-02-27|265|1156|89|1922|1|2|27|1|1922|89|1156|Monday|1922Q1|N|N|N|2423087|2423117|2422748|2423021|N|N|N|N|N| +2423114|AAAAAAAAKEJPECAA|1922-02-28|265|1157|89|1922|2|2|28|1|1922|89|1157|Tuesday|1922Q1|N|N|N|2423087|2423117|2422749|2423022|N|N|N|N|N| +2423115|AAAAAAAALEJPECAA|1922-03-01|266|1157|90|1922|3|3|1|1|1922|90|1157|Wednesday|1922Q1|N|N|N|2423115|2423173|2422750|2423023|N|N|N|N|N| +2423116|AAAAAAAAMEJPECAA|1922-03-02|266|1157|90|1922|4|3|2|1|1922|90|1157|Thursday|1922Q1|N|N|N|2423115|2423173|2422751|2423024|N|N|N|N|N| +2423117|AAAAAAAANEJPECAA|1922-03-03|266|1157|90|1922|5|3|3|1|1922|90|1157|Friday|1922Q1|N|Y|N|2423115|2423173|2422752|2423025|N|N|N|N|N| +2423118|AAAAAAAAOEJPECAA|1922-03-04|266|1157|90|1922|6|3|4|1|1922|90|1157|Saturday|1922Q1|N|Y|N|2423115|2423173|2422753|2423026|N|N|N|N|N| +2423119|AAAAAAAAPEJPECAA|1922-03-05|266|1157|90|1922|0|3|5|1|1922|90|1157|Sunday|1922Q1|N|N|N|2423115|2423173|2422754|2423027|N|N|N|N|N| +2423120|AAAAAAAAAFJPECAA|1922-03-06|266|1157|90|1922|1|3|6|1|1922|90|1157|Monday|1922Q1|N|N|N|2423115|2423173|2422755|2423028|N|N|N|N|N| +2423121|AAAAAAAABFJPECAA|1922-03-07|266|1158|90|1922|2|3|7|1|1922|90|1158|Tuesday|1922Q1|N|N|N|2423115|2423173|2422756|2423029|N|N|N|N|N| +2423122|AAAAAAAACFJPECAA|1922-03-08|266|1158|90|1922|3|3|8|1|1922|90|1158|Wednesday|1922Q1|N|N|N|2423115|2423173|2422757|2423030|N|N|N|N|N| +2423123|AAAAAAAADFJPECAA|1922-03-09|266|1158|90|1922|4|3|9|1|1922|90|1158|Thursday|1922Q1|N|N|N|2423115|2423173|2422758|2423031|N|N|N|N|N| +2423124|AAAAAAAAEFJPECAA|1922-03-10|266|1158|90|1922|5|3|10|1|1922|90|1158|Friday|1922Q1|N|Y|N|2423115|2423173|2422759|2423032|N|N|N|N|N| +2423125|AAAAAAAAFFJPECAA|1922-03-11|266|1158|90|1922|6|3|11|1|1922|90|1158|Saturday|1922Q1|N|Y|N|2423115|2423173|2422760|2423033|N|N|N|N|N| +2423126|AAAAAAAAGFJPECAA|1922-03-12|266|1158|90|1922|0|3|12|1|1922|90|1158|Sunday|1922Q1|N|N|N|2423115|2423173|2422761|2423034|N|N|N|N|N| +2423127|AAAAAAAAHFJPECAA|1922-03-13|266|1158|90|1922|1|3|13|1|1922|90|1158|Monday|1922Q1|N|N|N|2423115|2423173|2422762|2423035|N|N|N|N|N| +2423128|AAAAAAAAIFJPECAA|1922-03-14|266|1159|90|1922|2|3|14|1|1922|90|1159|Tuesday|1922Q1|N|N|N|2423115|2423173|2422763|2423036|N|N|N|N|N| +2423129|AAAAAAAAJFJPECAA|1922-03-15|266|1159|90|1922|3|3|15|1|1922|90|1159|Wednesday|1922Q1|N|N|N|2423115|2423173|2422764|2423037|N|N|N|N|N| +2423130|AAAAAAAAKFJPECAA|1922-03-16|266|1159|90|1922|4|3|16|1|1922|90|1159|Thursday|1922Q1|N|N|N|2423115|2423173|2422765|2423038|N|N|N|N|N| +2423131|AAAAAAAALFJPECAA|1922-03-17|266|1159|90|1922|5|3|17|1|1922|90|1159|Friday|1922Q1|N|Y|N|2423115|2423173|2422766|2423039|N|N|N|N|N| +2423132|AAAAAAAAMFJPECAA|1922-03-18|266|1159|90|1922|6|3|18|1|1922|90|1159|Saturday|1922Q1|N|Y|N|2423115|2423173|2422767|2423040|N|N|N|N|N| +2423133|AAAAAAAANFJPECAA|1922-03-19|266|1159|90|1922|0|3|19|1|1922|90|1159|Sunday|1922Q1|N|N|N|2423115|2423173|2422768|2423041|N|N|N|N|N| +2423134|AAAAAAAAOFJPECAA|1922-03-20|266|1159|90|1922|1|3|20|1|1922|90|1159|Monday|1922Q1|N|N|N|2423115|2423173|2422769|2423042|N|N|N|N|N| +2423135|AAAAAAAAPFJPECAA|1922-03-21|266|1160|90|1922|2|3|21|1|1922|90|1160|Tuesday|1922Q1|N|N|N|2423115|2423173|2422770|2423043|N|N|N|N|N| +2423136|AAAAAAAAAGJPECAA|1922-03-22|266|1160|90|1922|3|3|22|1|1922|90|1160|Wednesday|1922Q1|N|N|N|2423115|2423173|2422771|2423044|N|N|N|N|N| +2423137|AAAAAAAABGJPECAA|1922-03-23|266|1160|90|1922|4|3|23|1|1922|90|1160|Thursday|1922Q1|N|N|N|2423115|2423173|2422772|2423045|N|N|N|N|N| +2423138|AAAAAAAACGJPECAA|1922-03-24|266|1160|90|1922|5|3|24|1|1922|90|1160|Friday|1922Q1|N|Y|N|2423115|2423173|2422773|2423046|N|N|N|N|N| +2423139|AAAAAAAADGJPECAA|1922-03-25|266|1160|90|1922|6|3|25|1|1922|90|1160|Saturday|1922Q1|N|Y|N|2423115|2423173|2422774|2423047|N|N|N|N|N| +2423140|AAAAAAAAEGJPECAA|1922-03-26|266|1160|90|1922|0|3|26|1|1922|90|1160|Sunday|1922Q1|N|N|N|2423115|2423173|2422775|2423048|N|N|N|N|N| +2423141|AAAAAAAAFGJPECAA|1922-03-27|266|1160|90|1922|1|3|27|1|1922|90|1160|Monday|1922Q1|N|N|N|2423115|2423173|2422776|2423049|N|N|N|N|N| +2423142|AAAAAAAAGGJPECAA|1922-03-28|266|1161|90|1922|2|3|28|1|1922|90|1161|Tuesday|1922Q1|N|N|N|2423115|2423173|2422777|2423050|N|N|N|N|N| +2423143|AAAAAAAAHGJPECAA|1922-03-29|266|1161|90|1922|3|3|29|1|1922|90|1161|Wednesday|1922Q1|N|N|N|2423115|2423173|2422778|2423051|N|N|N|N|N| +2423144|AAAAAAAAIGJPECAA|1922-03-30|266|1161|90|1922|4|3|30|1|1922|90|1161|Thursday|1922Q1|N|N|N|2423115|2423173|2422779|2423052|N|N|N|N|N| +2423145|AAAAAAAAJGJPECAA|1922-03-31|266|1161|90|1922|5|3|31|1|1922|90|1161|Friday|1922Q1|N|Y|N|2423115|2423173|2422780|2423053|N|N|N|N|N| +2423146|AAAAAAAAKGJPECAA|1922-04-01|267|1161|90|1922|6|4|1|1|1922|90|1161|Saturday|1922Q1|N|Y|N|2423146|2423235|2422781|2423056|N|N|N|N|N| +2423147|AAAAAAAALGJPECAA|1922-04-02|267|1161|90|1922|0|4|2|2|1922|90|1161|Sunday|1922Q2|N|N|N|2423146|2423235|2422782|2423057|N|N|N|N|N| +2423148|AAAAAAAAMGJPECAA|1922-04-03|267|1161|90|1922|1|4|3|2|1922|90|1161|Monday|1922Q2|N|N|N|2423146|2423235|2422783|2423058|N|N|N|N|N| +2423149|AAAAAAAANGJPECAA|1922-04-04|267|1162|90|1922|2|4|4|2|1922|90|1162|Tuesday|1922Q2|N|N|N|2423146|2423235|2422784|2423059|N|N|N|N|N| +2423150|AAAAAAAAOGJPECAA|1922-04-05|267|1162|90|1922|3|4|5|2|1922|90|1162|Wednesday|1922Q2|N|N|N|2423146|2423235|2422785|2423060|N|N|N|N|N| +2423151|AAAAAAAAPGJPECAA|1922-04-06|267|1162|90|1922|4|4|6|2|1922|90|1162|Thursday|1922Q2|N|N|N|2423146|2423235|2422786|2423061|N|N|N|N|N| +2423152|AAAAAAAAAHJPECAA|1922-04-07|267|1162|90|1922|5|4|7|2|1922|90|1162|Friday|1922Q2|N|Y|N|2423146|2423235|2422787|2423062|N|N|N|N|N| +2423153|AAAAAAAABHJPECAA|1922-04-08|267|1162|90|1922|6|4|8|2|1922|90|1162|Saturday|1922Q2|N|Y|N|2423146|2423235|2422788|2423063|N|N|N|N|N| +2423154|AAAAAAAACHJPECAA|1922-04-09|267|1162|90|1922|0|4|9|2|1922|90|1162|Sunday|1922Q2|N|N|N|2423146|2423235|2422789|2423064|N|N|N|N|N| +2423155|AAAAAAAADHJPECAA|1922-04-10|267|1162|90|1922|1|4|10|2|1922|90|1162|Monday|1922Q2|N|N|N|2423146|2423235|2422790|2423065|N|N|N|N|N| +2423156|AAAAAAAAEHJPECAA|1922-04-11|267|1163|90|1922|2|4|11|2|1922|90|1163|Tuesday|1922Q2|N|N|N|2423146|2423235|2422791|2423066|N|N|N|N|N| +2423157|AAAAAAAAFHJPECAA|1922-04-12|267|1163|90|1922|3|4|12|2|1922|90|1163|Wednesday|1922Q2|N|N|N|2423146|2423235|2422792|2423067|N|N|N|N|N| +2423158|AAAAAAAAGHJPECAA|1922-04-13|267|1163|90|1922|4|4|13|2|1922|90|1163|Thursday|1922Q2|N|N|N|2423146|2423235|2422793|2423068|N|N|N|N|N| +2423159|AAAAAAAAHHJPECAA|1922-04-14|267|1163|90|1922|5|4|14|2|1922|90|1163|Friday|1922Q2|N|Y|N|2423146|2423235|2422794|2423069|N|N|N|N|N| +2423160|AAAAAAAAIHJPECAA|1922-04-15|267|1163|90|1922|6|4|15|2|1922|90|1163|Saturday|1922Q2|N|Y|N|2423146|2423235|2422795|2423070|N|N|N|N|N| +2423161|AAAAAAAAJHJPECAA|1922-04-16|267|1163|90|1922|0|4|16|2|1922|90|1163|Sunday|1922Q2|N|N|N|2423146|2423235|2422796|2423071|N|N|N|N|N| +2423162|AAAAAAAAKHJPECAA|1922-04-17|267|1163|90|1922|1|4|17|2|1922|90|1163|Monday|1922Q2|N|N|N|2423146|2423235|2422797|2423072|N|N|N|N|N| +2423163|AAAAAAAALHJPECAA|1922-04-18|267|1164|90|1922|2|4|18|2|1922|90|1164|Tuesday|1922Q2|N|N|N|2423146|2423235|2422798|2423073|N|N|N|N|N| +2423164|AAAAAAAAMHJPECAA|1922-04-19|267|1164|90|1922|3|4|19|2|1922|90|1164|Wednesday|1922Q2|N|N|N|2423146|2423235|2422799|2423074|N|N|N|N|N| +2423165|AAAAAAAANHJPECAA|1922-04-20|267|1164|90|1922|4|4|20|2|1922|90|1164|Thursday|1922Q2|N|N|N|2423146|2423235|2422800|2423075|N|N|N|N|N| +2423166|AAAAAAAAOHJPECAA|1922-04-21|267|1164|90|1922|5|4|21|2|1922|90|1164|Friday|1922Q2|N|Y|N|2423146|2423235|2422801|2423076|N|N|N|N|N| +2423167|AAAAAAAAPHJPECAA|1922-04-22|267|1164|90|1922|6|4|22|2|1922|90|1164|Saturday|1922Q2|N|Y|N|2423146|2423235|2422802|2423077|N|N|N|N|N| +2423168|AAAAAAAAAIJPECAA|1922-04-23|267|1164|90|1922|0|4|23|2|1922|90|1164|Sunday|1922Q2|N|N|N|2423146|2423235|2422803|2423078|N|N|N|N|N| +2423169|AAAAAAAABIJPECAA|1922-04-24|267|1164|90|1922|1|4|24|2|1922|90|1164|Monday|1922Q2|N|N|N|2423146|2423235|2422804|2423079|N|N|N|N|N| +2423170|AAAAAAAACIJPECAA|1922-04-25|267|1165|90|1922|2|4|25|2|1922|90|1165|Tuesday|1922Q2|N|N|N|2423146|2423235|2422805|2423080|N|N|N|N|N| +2423171|AAAAAAAADIJPECAA|1922-04-26|267|1165|90|1922|3|4|26|2|1922|90|1165|Wednesday|1922Q2|N|N|N|2423146|2423235|2422806|2423081|N|N|N|N|N| +2423172|AAAAAAAAEIJPECAA|1922-04-27|267|1165|90|1922|4|4|27|2|1922|90|1165|Thursday|1922Q2|N|N|N|2423146|2423235|2422807|2423082|N|N|N|N|N| +2423173|AAAAAAAAFIJPECAA|1922-04-28|267|1165|90|1922|5|4|28|2|1922|90|1165|Friday|1922Q2|N|Y|N|2423146|2423235|2422808|2423083|N|N|N|N|N| +2423174|AAAAAAAAGIJPECAA|1922-04-29|267|1165|90|1922|6|4|29|2|1922|90|1165|Saturday|1922Q2|N|Y|N|2423146|2423235|2422809|2423084|N|N|N|N|N| +2423175|AAAAAAAAHIJPECAA|1922-04-30|267|1165|90|1922|0|4|30|2|1922|90|1165|Sunday|1922Q2|N|N|N|2423146|2423235|2422810|2423085|N|N|N|N|N| +2423176|AAAAAAAAIIJPECAA|1922-05-01|268|1165|90|1922|1|5|1|2|1922|90|1165|Monday|1922Q2|N|N|N|2423176|2423295|2422811|2423086|N|N|N|N|N| +2423177|AAAAAAAAJIJPECAA|1922-05-02|268|1166|90|1922|2|5|2|2|1922|90|1166|Tuesday|1922Q2|N|N|N|2423176|2423295|2422812|2423087|N|N|N|N|N| +2423178|AAAAAAAAKIJPECAA|1922-05-03|268|1166|90|1922|3|5|3|2|1922|90|1166|Wednesday|1922Q2|N|N|N|2423176|2423295|2422813|2423088|N|N|N|N|N| +2423179|AAAAAAAALIJPECAA|1922-05-04|268|1166|90|1922|4|5|4|2|1922|90|1166|Thursday|1922Q2|N|N|N|2423176|2423295|2422814|2423089|N|N|N|N|N| +2423180|AAAAAAAAMIJPECAA|1922-05-05|268|1166|90|1922|5|5|5|2|1922|90|1166|Friday|1922Q2|N|Y|N|2423176|2423295|2422815|2423090|N|N|N|N|N| +2423181|AAAAAAAANIJPECAA|1922-05-06|268|1166|90|1922|6|5|6|2|1922|90|1166|Saturday|1922Q2|N|Y|N|2423176|2423295|2422816|2423091|N|N|N|N|N| +2423182|AAAAAAAAOIJPECAA|1922-05-07|268|1166|90|1922|0|5|7|2|1922|90|1166|Sunday|1922Q2|N|N|N|2423176|2423295|2422817|2423092|N|N|N|N|N| +2423183|AAAAAAAAPIJPECAA|1922-05-08|268|1166|90|1922|1|5|8|2|1922|90|1166|Monday|1922Q2|N|N|N|2423176|2423295|2422818|2423093|N|N|N|N|N| +2423184|AAAAAAAAAJJPECAA|1922-05-09|268|1167|90|1922|2|5|9|2|1922|90|1167|Tuesday|1922Q2|N|N|N|2423176|2423295|2422819|2423094|N|N|N|N|N| +2423185|AAAAAAAABJJPECAA|1922-05-10|268|1167|90|1922|3|5|10|2|1922|90|1167|Wednesday|1922Q2|N|N|N|2423176|2423295|2422820|2423095|N|N|N|N|N| +2423186|AAAAAAAACJJPECAA|1922-05-11|268|1167|90|1922|4|5|11|2|1922|90|1167|Thursday|1922Q2|N|N|N|2423176|2423295|2422821|2423096|N|N|N|N|N| +2423187|AAAAAAAADJJPECAA|1922-05-12|268|1167|90|1922|5|5|12|2|1922|90|1167|Friday|1922Q2|N|Y|N|2423176|2423295|2422822|2423097|N|N|N|N|N| +2423188|AAAAAAAAEJJPECAA|1922-05-13|268|1167|90|1922|6|5|13|2|1922|90|1167|Saturday|1922Q2|N|Y|N|2423176|2423295|2422823|2423098|N|N|N|N|N| +2423189|AAAAAAAAFJJPECAA|1922-05-14|268|1167|90|1922|0|5|14|2|1922|90|1167|Sunday|1922Q2|N|N|N|2423176|2423295|2422824|2423099|N|N|N|N|N| +2423190|AAAAAAAAGJJPECAA|1922-05-15|268|1167|90|1922|1|5|15|2|1922|90|1167|Monday|1922Q2|N|N|N|2423176|2423295|2422825|2423100|N|N|N|N|N| +2423191|AAAAAAAAHJJPECAA|1922-05-16|268|1168|90|1922|2|5|16|2|1922|90|1168|Tuesday|1922Q2|N|N|N|2423176|2423295|2422826|2423101|N|N|N|N|N| +2423192|AAAAAAAAIJJPECAA|1922-05-17|268|1168|90|1922|3|5|17|2|1922|90|1168|Wednesday|1922Q2|N|N|N|2423176|2423295|2422827|2423102|N|N|N|N|N| +2423193|AAAAAAAAJJJPECAA|1922-05-18|268|1168|90|1922|4|5|18|2|1922|90|1168|Thursday|1922Q2|N|N|N|2423176|2423295|2422828|2423103|N|N|N|N|N| +2423194|AAAAAAAAKJJPECAA|1922-05-19|268|1168|90|1922|5|5|19|2|1922|90|1168|Friday|1922Q2|N|Y|N|2423176|2423295|2422829|2423104|N|N|N|N|N| +2423195|AAAAAAAALJJPECAA|1922-05-20|268|1168|90|1922|6|5|20|2|1922|90|1168|Saturday|1922Q2|N|Y|N|2423176|2423295|2422830|2423105|N|N|N|N|N| +2423196|AAAAAAAAMJJPECAA|1922-05-21|268|1168|90|1922|0|5|21|2|1922|90|1168|Sunday|1922Q2|N|N|N|2423176|2423295|2422831|2423106|N|N|N|N|N| +2423197|AAAAAAAANJJPECAA|1922-05-22|268|1168|90|1922|1|5|22|2|1922|90|1168|Monday|1922Q2|N|N|N|2423176|2423295|2422832|2423107|N|N|N|N|N| +2423198|AAAAAAAAOJJPECAA|1922-05-23|268|1169|90|1922|2|5|23|2|1922|90|1169|Tuesday|1922Q2|N|N|N|2423176|2423295|2422833|2423108|N|N|N|N|N| +2423199|AAAAAAAAPJJPECAA|1922-05-24|268|1169|90|1922|3|5|24|2|1922|90|1169|Wednesday|1922Q2|N|N|N|2423176|2423295|2422834|2423109|N|N|N|N|N| +2423200|AAAAAAAAAKJPECAA|1922-05-25|268|1169|90|1922|4|5|25|2|1922|90|1169|Thursday|1922Q2|N|N|N|2423176|2423295|2422835|2423110|N|N|N|N|N| +2423201|AAAAAAAABKJPECAA|1922-05-26|268|1169|90|1922|5|5|26|2|1922|90|1169|Friday|1922Q2|N|Y|N|2423176|2423295|2422836|2423111|N|N|N|N|N| +2423202|AAAAAAAACKJPECAA|1922-05-27|268|1169|90|1922|6|5|27|2|1922|90|1169|Saturday|1922Q2|N|Y|N|2423176|2423295|2422837|2423112|N|N|N|N|N| +2423203|AAAAAAAADKJPECAA|1922-05-28|268|1169|90|1922|0|5|28|2|1922|90|1169|Sunday|1922Q2|N|N|N|2423176|2423295|2422838|2423113|N|N|N|N|N| +2423204|AAAAAAAAEKJPECAA|1922-05-29|268|1169|90|1922|1|5|29|2|1922|90|1169|Monday|1922Q2|N|N|N|2423176|2423295|2422839|2423114|N|N|N|N|N| +2423205|AAAAAAAAFKJPECAA|1922-05-30|268|1170|90|1922|2|5|30|2|1922|90|1170|Tuesday|1922Q2|N|N|N|2423176|2423295|2422840|2423115|N|N|N|N|N| +2423206|AAAAAAAAGKJPECAA|1922-05-31|268|1170|90|1922|3|5|31|2|1922|90|1170|Wednesday|1922Q2|N|N|N|2423176|2423295|2422841|2423116|N|N|N|N|N| +2423207|AAAAAAAAHKJPECAA|1922-06-01|269|1170|91|1922|4|6|1|2|1922|91|1170|Thursday|1922Q2|N|N|N|2423207|2423357|2422842|2423117|N|N|N|N|N| +2423208|AAAAAAAAIKJPECAA|1922-06-02|269|1170|91|1922|5|6|2|2|1922|91|1170|Friday|1922Q2|N|Y|N|2423207|2423357|2422843|2423118|N|N|N|N|N| +2423209|AAAAAAAAJKJPECAA|1922-06-03|269|1170|91|1922|6|6|3|2|1922|91|1170|Saturday|1922Q2|N|Y|N|2423207|2423357|2422844|2423119|N|N|N|N|N| +2423210|AAAAAAAAKKJPECAA|1922-06-04|269|1170|91|1922|0|6|4|2|1922|91|1170|Sunday|1922Q2|N|N|N|2423207|2423357|2422845|2423120|N|N|N|N|N| +2423211|AAAAAAAALKJPECAA|1922-06-05|269|1170|91|1922|1|6|5|2|1922|91|1170|Monday|1922Q2|N|N|N|2423207|2423357|2422846|2423121|N|N|N|N|N| +2423212|AAAAAAAAMKJPECAA|1922-06-06|269|1171|91|1922|2|6|6|2|1922|91|1171|Tuesday|1922Q2|N|N|N|2423207|2423357|2422847|2423122|N|N|N|N|N| +2423213|AAAAAAAANKJPECAA|1922-06-07|269|1171|91|1922|3|6|7|2|1922|91|1171|Wednesday|1922Q2|N|N|N|2423207|2423357|2422848|2423123|N|N|N|N|N| +2423214|AAAAAAAAOKJPECAA|1922-06-08|269|1171|91|1922|4|6|8|2|1922|91|1171|Thursday|1922Q2|N|N|N|2423207|2423357|2422849|2423124|N|N|N|N|N| +2423215|AAAAAAAAPKJPECAA|1922-06-09|269|1171|91|1922|5|6|9|2|1922|91|1171|Friday|1922Q2|N|Y|N|2423207|2423357|2422850|2423125|N|N|N|N|N| +2423216|AAAAAAAAALJPECAA|1922-06-10|269|1171|91|1922|6|6|10|2|1922|91|1171|Saturday|1922Q2|N|Y|N|2423207|2423357|2422851|2423126|N|N|N|N|N| +2423217|AAAAAAAABLJPECAA|1922-06-11|269|1171|91|1922|0|6|11|2|1922|91|1171|Sunday|1922Q2|N|N|N|2423207|2423357|2422852|2423127|N|N|N|N|N| +2423218|AAAAAAAACLJPECAA|1922-06-12|269|1171|91|1922|1|6|12|2|1922|91|1171|Monday|1922Q2|N|N|N|2423207|2423357|2422853|2423128|N|N|N|N|N| +2423219|AAAAAAAADLJPECAA|1922-06-13|269|1172|91|1922|2|6|13|2|1922|91|1172|Tuesday|1922Q2|N|N|N|2423207|2423357|2422854|2423129|N|N|N|N|N| +2423220|AAAAAAAAELJPECAA|1922-06-14|269|1172|91|1922|3|6|14|2|1922|91|1172|Wednesday|1922Q2|N|N|N|2423207|2423357|2422855|2423130|N|N|N|N|N| +2423221|AAAAAAAAFLJPECAA|1922-06-15|269|1172|91|1922|4|6|15|2|1922|91|1172|Thursday|1922Q2|N|N|N|2423207|2423357|2422856|2423131|N|N|N|N|N| +2423222|AAAAAAAAGLJPECAA|1922-06-16|269|1172|91|1922|5|6|16|2|1922|91|1172|Friday|1922Q2|N|Y|N|2423207|2423357|2422857|2423132|N|N|N|N|N| +2423223|AAAAAAAAHLJPECAA|1922-06-17|269|1172|91|1922|6|6|17|2|1922|91|1172|Saturday|1922Q2|N|Y|N|2423207|2423357|2422858|2423133|N|N|N|N|N| +2423224|AAAAAAAAILJPECAA|1922-06-18|269|1172|91|1922|0|6|18|2|1922|91|1172|Sunday|1922Q2|N|N|N|2423207|2423357|2422859|2423134|N|N|N|N|N| +2423225|AAAAAAAAJLJPECAA|1922-06-19|269|1172|91|1922|1|6|19|2|1922|91|1172|Monday|1922Q2|N|N|N|2423207|2423357|2422860|2423135|N|N|N|N|N| +2423226|AAAAAAAAKLJPECAA|1922-06-20|269|1173|91|1922|2|6|20|2|1922|91|1173|Tuesday|1922Q2|N|N|N|2423207|2423357|2422861|2423136|N|N|N|N|N| +2423227|AAAAAAAALLJPECAA|1922-06-21|269|1173|91|1922|3|6|21|2|1922|91|1173|Wednesday|1922Q2|N|N|N|2423207|2423357|2422862|2423137|N|N|N|N|N| +2423228|AAAAAAAAMLJPECAA|1922-06-22|269|1173|91|1922|4|6|22|2|1922|91|1173|Thursday|1922Q2|N|N|N|2423207|2423357|2422863|2423138|N|N|N|N|N| +2423229|AAAAAAAANLJPECAA|1922-06-23|269|1173|91|1922|5|6|23|2|1922|91|1173|Friday|1922Q2|N|Y|N|2423207|2423357|2422864|2423139|N|N|N|N|N| +2423230|AAAAAAAAOLJPECAA|1922-06-24|269|1173|91|1922|6|6|24|2|1922|91|1173|Saturday|1922Q2|N|Y|N|2423207|2423357|2422865|2423140|N|N|N|N|N| +2423231|AAAAAAAAPLJPECAA|1922-06-25|269|1173|91|1922|0|6|25|2|1922|91|1173|Sunday|1922Q2|N|N|N|2423207|2423357|2422866|2423141|N|N|N|N|N| +2423232|AAAAAAAAAMJPECAA|1922-06-26|269|1173|91|1922|1|6|26|2|1922|91|1173|Monday|1922Q2|N|N|N|2423207|2423357|2422867|2423142|N|N|N|N|N| +2423233|AAAAAAAABMJPECAA|1922-06-27|269|1174|91|1922|2|6|27|2|1922|91|1174|Tuesday|1922Q2|N|N|N|2423207|2423357|2422868|2423143|N|N|N|N|N| +2423234|AAAAAAAACMJPECAA|1922-06-28|269|1174|91|1922|3|6|28|2|1922|91|1174|Wednesday|1922Q2|N|N|N|2423207|2423357|2422869|2423144|N|N|N|N|N| +2423235|AAAAAAAADMJPECAA|1922-06-29|269|1174|91|1922|4|6|29|2|1922|91|1174|Thursday|1922Q2|N|N|N|2423207|2423357|2422870|2423145|N|N|N|N|N| +2423236|AAAAAAAAEMJPECAA|1922-06-30|269|1174|91|1922|5|6|30|2|1922|91|1174|Friday|1922Q2|N|Y|N|2423207|2423357|2422871|2423146|N|N|N|N|N| +2423237|AAAAAAAAFMJPECAA|1922-07-01|270|1174|91|1922|6|7|1|2|1922|91|1174|Saturday|1922Q2|N|Y|N|2423237|2423417|2422872|2423146|N|N|N|N|N| +2423238|AAAAAAAAGMJPECAA|1922-07-02|270|1174|91|1922|0|7|2|3|1922|91|1174|Sunday|1922Q3|N|N|N|2423237|2423417|2422873|2423147|N|N|N|N|N| +2423239|AAAAAAAAHMJPECAA|1922-07-03|270|1174|91|1922|1|7|3|3|1922|91|1174|Monday|1922Q3|N|N|N|2423237|2423417|2422874|2423148|N|N|N|N|N| +2423240|AAAAAAAAIMJPECAA|1922-07-04|270|1175|91|1922|2|7|4|3|1922|91|1175|Tuesday|1922Q3|N|N|N|2423237|2423417|2422875|2423149|N|N|N|N|N| +2423241|AAAAAAAAJMJPECAA|1922-07-05|270|1175|91|1922|3|7|5|3|1922|91|1175|Wednesday|1922Q3|Y|N|N|2423237|2423417|2422876|2423150|N|N|N|N|N| +2423242|AAAAAAAAKMJPECAA|1922-07-06|270|1175|91|1922|4|7|6|3|1922|91|1175|Thursday|1922Q3|N|N|Y|2423237|2423417|2422877|2423151|N|N|N|N|N| +2423243|AAAAAAAALMJPECAA|1922-07-07|270|1175|91|1922|5|7|7|3|1922|91|1175|Friday|1922Q3|N|Y|N|2423237|2423417|2422878|2423152|N|N|N|N|N| +2423244|AAAAAAAAMMJPECAA|1922-07-08|270|1175|91|1922|6|7|8|3|1922|91|1175|Saturday|1922Q3|N|Y|N|2423237|2423417|2422879|2423153|N|N|N|N|N| +2423245|AAAAAAAANMJPECAA|1922-07-09|270|1175|91|1922|0|7|9|3|1922|91|1175|Sunday|1922Q3|N|N|N|2423237|2423417|2422880|2423154|N|N|N|N|N| +2423246|AAAAAAAAOMJPECAA|1922-07-10|270|1175|91|1922|1|7|10|3|1922|91|1175|Monday|1922Q3|N|N|N|2423237|2423417|2422881|2423155|N|N|N|N|N| +2423247|AAAAAAAAPMJPECAA|1922-07-11|270|1176|91|1922|2|7|11|3|1922|91|1176|Tuesday|1922Q3|N|N|N|2423237|2423417|2422882|2423156|N|N|N|N|N| +2423248|AAAAAAAAANJPECAA|1922-07-12|270|1176|91|1922|3|7|12|3|1922|91|1176|Wednesday|1922Q3|N|N|N|2423237|2423417|2422883|2423157|N|N|N|N|N| +2423249|AAAAAAAABNJPECAA|1922-07-13|270|1176|91|1922|4|7|13|3|1922|91|1176|Thursday|1922Q3|N|N|N|2423237|2423417|2422884|2423158|N|N|N|N|N| +2423250|AAAAAAAACNJPECAA|1922-07-14|270|1176|91|1922|5|7|14|3|1922|91|1176|Friday|1922Q3|N|Y|N|2423237|2423417|2422885|2423159|N|N|N|N|N| +2423251|AAAAAAAADNJPECAA|1922-07-15|270|1176|91|1922|6|7|15|3|1922|91|1176|Saturday|1922Q3|N|Y|N|2423237|2423417|2422886|2423160|N|N|N|N|N| +2423252|AAAAAAAAENJPECAA|1922-07-16|270|1176|91|1922|0|7|16|3|1922|91|1176|Sunday|1922Q3|N|N|N|2423237|2423417|2422887|2423161|N|N|N|N|N| +2423253|AAAAAAAAFNJPECAA|1922-07-17|270|1176|91|1922|1|7|17|3|1922|91|1176|Monday|1922Q3|N|N|N|2423237|2423417|2422888|2423162|N|N|N|N|N| +2423254|AAAAAAAAGNJPECAA|1922-07-18|270|1177|91|1922|2|7|18|3|1922|91|1177|Tuesday|1922Q3|N|N|N|2423237|2423417|2422889|2423163|N|N|N|N|N| +2423255|AAAAAAAAHNJPECAA|1922-07-19|270|1177|91|1922|3|7|19|3|1922|91|1177|Wednesday|1922Q3|N|N|N|2423237|2423417|2422890|2423164|N|N|N|N|N| +2423256|AAAAAAAAINJPECAA|1922-07-20|270|1177|91|1922|4|7|20|3|1922|91|1177|Thursday|1922Q3|N|N|N|2423237|2423417|2422891|2423165|N|N|N|N|N| +2423257|AAAAAAAAJNJPECAA|1922-07-21|270|1177|91|1922|5|7|21|3|1922|91|1177|Friday|1922Q3|N|Y|N|2423237|2423417|2422892|2423166|N|N|N|N|N| +2423258|AAAAAAAAKNJPECAA|1922-07-22|270|1177|91|1922|6|7|22|3|1922|91|1177|Saturday|1922Q3|N|Y|N|2423237|2423417|2422893|2423167|N|N|N|N|N| +2423259|AAAAAAAALNJPECAA|1922-07-23|270|1177|91|1922|0|7|23|3|1922|91|1177|Sunday|1922Q3|N|N|N|2423237|2423417|2422894|2423168|N|N|N|N|N| +2423260|AAAAAAAAMNJPECAA|1922-07-24|270|1177|91|1922|1|7|24|3|1922|91|1177|Monday|1922Q3|N|N|N|2423237|2423417|2422895|2423169|N|N|N|N|N| +2423261|AAAAAAAANNJPECAA|1922-07-25|270|1178|91|1922|2|7|25|3|1922|91|1178|Tuesday|1922Q3|N|N|N|2423237|2423417|2422896|2423170|N|N|N|N|N| +2423262|AAAAAAAAONJPECAA|1922-07-26|270|1178|91|1922|3|7|26|3|1922|91|1178|Wednesday|1922Q3|N|N|N|2423237|2423417|2422897|2423171|N|N|N|N|N| +2423263|AAAAAAAAPNJPECAA|1922-07-27|270|1178|91|1922|4|7|27|3|1922|91|1178|Thursday|1922Q3|N|N|N|2423237|2423417|2422898|2423172|N|N|N|N|N| +2423264|AAAAAAAAAOJPECAA|1922-07-28|270|1178|91|1922|5|7|28|3|1922|91|1178|Friday|1922Q3|N|Y|N|2423237|2423417|2422899|2423173|N|N|N|N|N| +2423265|AAAAAAAABOJPECAA|1922-07-29|270|1178|91|1922|6|7|29|3|1922|91|1178|Saturday|1922Q3|N|Y|N|2423237|2423417|2422900|2423174|N|N|N|N|N| +2423266|AAAAAAAACOJPECAA|1922-07-30|270|1178|91|1922|0|7|30|3|1922|91|1178|Sunday|1922Q3|N|N|N|2423237|2423417|2422901|2423175|N|N|N|N|N| +2423267|AAAAAAAADOJPECAA|1922-07-31|270|1178|91|1922|1|7|31|3|1922|91|1178|Monday|1922Q3|N|N|N|2423237|2423417|2422902|2423176|N|N|N|N|N| +2423268|AAAAAAAAEOJPECAA|1922-08-01|271|1179|91|1922|2|8|1|3|1922|91|1179|Tuesday|1922Q3|N|N|N|2423268|2423479|2422903|2423177|N|N|N|N|N| +2423269|AAAAAAAAFOJPECAA|1922-08-02|271|1179|91|1922|3|8|2|3|1922|91|1179|Wednesday|1922Q3|N|N|N|2423268|2423479|2422904|2423178|N|N|N|N|N| +2423270|AAAAAAAAGOJPECAA|1922-08-03|271|1179|91|1922|4|8|3|3|1922|91|1179|Thursday|1922Q3|N|N|N|2423268|2423479|2422905|2423179|N|N|N|N|N| +2423271|AAAAAAAAHOJPECAA|1922-08-04|271|1179|91|1922|5|8|4|3|1922|91|1179|Friday|1922Q3|N|Y|N|2423268|2423479|2422906|2423180|N|N|N|N|N| +2423272|AAAAAAAAIOJPECAA|1922-08-05|271|1179|91|1922|6|8|5|3|1922|91|1179|Saturday|1922Q3|N|Y|N|2423268|2423479|2422907|2423181|N|N|N|N|N| +2423273|AAAAAAAAJOJPECAA|1922-08-06|271|1179|91|1922|0|8|6|3|1922|91|1179|Sunday|1922Q3|N|N|N|2423268|2423479|2422908|2423182|N|N|N|N|N| +2423274|AAAAAAAAKOJPECAA|1922-08-07|271|1179|91|1922|1|8|7|3|1922|91|1179|Monday|1922Q3|N|N|N|2423268|2423479|2422909|2423183|N|N|N|N|N| +2423275|AAAAAAAALOJPECAA|1922-08-08|271|1180|91|1922|2|8|8|3|1922|91|1180|Tuesday|1922Q3|N|N|N|2423268|2423479|2422910|2423184|N|N|N|N|N| +2423276|AAAAAAAAMOJPECAA|1922-08-09|271|1180|91|1922|3|8|9|3|1922|91|1180|Wednesday|1922Q3|N|N|N|2423268|2423479|2422911|2423185|N|N|N|N|N| +2423277|AAAAAAAANOJPECAA|1922-08-10|271|1180|91|1922|4|8|10|3|1922|91|1180|Thursday|1922Q3|N|N|N|2423268|2423479|2422912|2423186|N|N|N|N|N| +2423278|AAAAAAAAOOJPECAA|1922-08-11|271|1180|91|1922|5|8|11|3|1922|91|1180|Friday|1922Q3|N|Y|N|2423268|2423479|2422913|2423187|N|N|N|N|N| +2423279|AAAAAAAAPOJPECAA|1922-08-12|271|1180|91|1922|6|8|12|3|1922|91|1180|Saturday|1922Q3|N|Y|N|2423268|2423479|2422914|2423188|N|N|N|N|N| +2423280|AAAAAAAAAPJPECAA|1922-08-13|271|1180|91|1922|0|8|13|3|1922|91|1180|Sunday|1922Q3|N|N|N|2423268|2423479|2422915|2423189|N|N|N|N|N| +2423281|AAAAAAAABPJPECAA|1922-08-14|271|1180|91|1922|1|8|14|3|1922|91|1180|Monday|1922Q3|N|N|N|2423268|2423479|2422916|2423190|N|N|N|N|N| +2423282|AAAAAAAACPJPECAA|1922-08-15|271|1181|91|1922|2|8|15|3|1922|91|1181|Tuesday|1922Q3|N|N|N|2423268|2423479|2422917|2423191|N|N|N|N|N| +2423283|AAAAAAAADPJPECAA|1922-08-16|271|1181|91|1922|3|8|16|3|1922|91|1181|Wednesday|1922Q3|N|N|N|2423268|2423479|2422918|2423192|N|N|N|N|N| +2423284|AAAAAAAAEPJPECAA|1922-08-17|271|1181|91|1922|4|8|17|3|1922|91|1181|Thursday|1922Q3|N|N|N|2423268|2423479|2422919|2423193|N|N|N|N|N| +2423285|AAAAAAAAFPJPECAA|1922-08-18|271|1181|91|1922|5|8|18|3|1922|91|1181|Friday|1922Q3|N|Y|N|2423268|2423479|2422920|2423194|N|N|N|N|N| +2423286|AAAAAAAAGPJPECAA|1922-08-19|271|1181|91|1922|6|8|19|3|1922|91|1181|Saturday|1922Q3|N|Y|N|2423268|2423479|2422921|2423195|N|N|N|N|N| +2423287|AAAAAAAAHPJPECAA|1922-08-20|271|1181|91|1922|0|8|20|3|1922|91|1181|Sunday|1922Q3|N|N|N|2423268|2423479|2422922|2423196|N|N|N|N|N| +2423288|AAAAAAAAIPJPECAA|1922-08-21|271|1181|91|1922|1|8|21|3|1922|91|1181|Monday|1922Q3|N|N|N|2423268|2423479|2422923|2423197|N|N|N|N|N| +2423289|AAAAAAAAJPJPECAA|1922-08-22|271|1182|91|1922|2|8|22|3|1922|91|1182|Tuesday|1922Q3|N|N|N|2423268|2423479|2422924|2423198|N|N|N|N|N| +2423290|AAAAAAAAKPJPECAA|1922-08-23|271|1182|91|1922|3|8|23|3|1922|91|1182|Wednesday|1922Q3|N|N|N|2423268|2423479|2422925|2423199|N|N|N|N|N| +2423291|AAAAAAAALPJPECAA|1922-08-24|271|1182|91|1922|4|8|24|3|1922|91|1182|Thursday|1922Q3|N|N|N|2423268|2423479|2422926|2423200|N|N|N|N|N| +2423292|AAAAAAAAMPJPECAA|1922-08-25|271|1182|91|1922|5|8|25|3|1922|91|1182|Friday|1922Q3|N|Y|N|2423268|2423479|2422927|2423201|N|N|N|N|N| +2423293|AAAAAAAANPJPECAA|1922-08-26|271|1182|91|1922|6|8|26|3|1922|91|1182|Saturday|1922Q3|N|Y|N|2423268|2423479|2422928|2423202|N|N|N|N|N| +2423294|AAAAAAAAOPJPECAA|1922-08-27|271|1182|91|1922|0|8|27|3|1922|91|1182|Sunday|1922Q3|N|N|N|2423268|2423479|2422929|2423203|N|N|N|N|N| +2423295|AAAAAAAAPPJPECAA|1922-08-28|271|1182|91|1922|1|8|28|3|1922|91|1182|Monday|1922Q3|N|N|N|2423268|2423479|2422930|2423204|N|N|N|N|N| +2423296|AAAAAAAAAAKPECAA|1922-08-29|271|1183|91|1922|2|8|29|3|1922|91|1183|Tuesday|1922Q3|N|N|N|2423268|2423479|2422931|2423205|N|N|N|N|N| +2423297|AAAAAAAABAKPECAA|1922-08-30|271|1183|91|1922|3|8|30|3|1922|91|1183|Wednesday|1922Q3|N|N|N|2423268|2423479|2422932|2423206|N|N|N|N|N| +2423298|AAAAAAAACAKPECAA|1922-08-31|271|1183|91|1922|4|8|31|3|1922|91|1183|Thursday|1922Q3|N|N|N|2423268|2423479|2422933|2423207|N|N|N|N|N| +2423299|AAAAAAAADAKPECAA|1922-09-01|272|1183|92|1922|5|9|1|3|1922|92|1183|Friday|1922Q3|N|Y|N|2423299|2423541|2422934|2423208|N|N|N|N|N| +2423300|AAAAAAAAEAKPECAA|1922-09-02|272|1183|92|1922|6|9|2|3|1922|92|1183|Saturday|1922Q3|N|Y|N|2423299|2423541|2422935|2423209|N|N|N|N|N| +2423301|AAAAAAAAFAKPECAA|1922-09-03|272|1183|92|1922|0|9|3|3|1922|92|1183|Sunday|1922Q3|N|N|N|2423299|2423541|2422936|2423210|N|N|N|N|N| +2423302|AAAAAAAAGAKPECAA|1922-09-04|272|1183|92|1922|1|9|4|3|1922|92|1183|Monday|1922Q3|N|N|N|2423299|2423541|2422937|2423211|N|N|N|N|N| +2423303|AAAAAAAAHAKPECAA|1922-09-05|272|1184|92|1922|2|9|5|3|1922|92|1184|Tuesday|1922Q3|N|N|N|2423299|2423541|2422938|2423212|N|N|N|N|N| +2423304|AAAAAAAAIAKPECAA|1922-09-06|272|1184|92|1922|3|9|6|3|1922|92|1184|Wednesday|1922Q3|N|N|N|2423299|2423541|2422939|2423213|N|N|N|N|N| +2423305|AAAAAAAAJAKPECAA|1922-09-07|272|1184|92|1922|4|9|7|3|1922|92|1184|Thursday|1922Q3|N|N|N|2423299|2423541|2422940|2423214|N|N|N|N|N| +2423306|AAAAAAAAKAKPECAA|1922-09-08|272|1184|92|1922|5|9|8|3|1922|92|1184|Friday|1922Q3|N|Y|N|2423299|2423541|2422941|2423215|N|N|N|N|N| +2423307|AAAAAAAALAKPECAA|1922-09-09|272|1184|92|1922|6|9|9|3|1922|92|1184|Saturday|1922Q3|N|Y|N|2423299|2423541|2422942|2423216|N|N|N|N|N| +2423308|AAAAAAAAMAKPECAA|1922-09-10|272|1184|92|1922|0|9|10|3|1922|92|1184|Sunday|1922Q3|N|N|N|2423299|2423541|2422943|2423217|N|N|N|N|N| +2423309|AAAAAAAANAKPECAA|1922-09-11|272|1184|92|1922|1|9|11|3|1922|92|1184|Monday|1922Q3|N|N|N|2423299|2423541|2422944|2423218|N|N|N|N|N| +2423310|AAAAAAAAOAKPECAA|1922-09-12|272|1185|92|1922|2|9|12|3|1922|92|1185|Tuesday|1922Q3|N|N|N|2423299|2423541|2422945|2423219|N|N|N|N|N| +2423311|AAAAAAAAPAKPECAA|1922-09-13|272|1185|92|1922|3|9|13|3|1922|92|1185|Wednesday|1922Q3|N|N|N|2423299|2423541|2422946|2423220|N|N|N|N|N| +2423312|AAAAAAAAABKPECAA|1922-09-14|272|1185|92|1922|4|9|14|3|1922|92|1185|Thursday|1922Q3|N|N|N|2423299|2423541|2422947|2423221|N|N|N|N|N| +2423313|AAAAAAAABBKPECAA|1922-09-15|272|1185|92|1922|5|9|15|3|1922|92|1185|Friday|1922Q3|N|Y|N|2423299|2423541|2422948|2423222|N|N|N|N|N| +2423314|AAAAAAAACBKPECAA|1922-09-16|272|1185|92|1922|6|9|16|3|1922|92|1185|Saturday|1922Q3|N|Y|N|2423299|2423541|2422949|2423223|N|N|N|N|N| +2423315|AAAAAAAADBKPECAA|1922-09-17|272|1185|92|1922|0|9|17|3|1922|92|1185|Sunday|1922Q3|N|N|N|2423299|2423541|2422950|2423224|N|N|N|N|N| +2423316|AAAAAAAAEBKPECAA|1922-09-18|272|1185|92|1922|1|9|18|3|1922|92|1185|Monday|1922Q3|N|N|N|2423299|2423541|2422951|2423225|N|N|N|N|N| +2423317|AAAAAAAAFBKPECAA|1922-09-19|272|1186|92|1922|2|9|19|3|1922|92|1186|Tuesday|1922Q3|N|N|N|2423299|2423541|2422952|2423226|N|N|N|N|N| +2423318|AAAAAAAAGBKPECAA|1922-09-20|272|1186|92|1922|3|9|20|3|1922|92|1186|Wednesday|1922Q3|N|N|N|2423299|2423541|2422953|2423227|N|N|N|N|N| +2423319|AAAAAAAAHBKPECAA|1922-09-21|272|1186|92|1922|4|9|21|3|1922|92|1186|Thursday|1922Q3|N|N|N|2423299|2423541|2422954|2423228|N|N|N|N|N| +2423320|AAAAAAAAIBKPECAA|1922-09-22|272|1186|92|1922|5|9|22|3|1922|92|1186|Friday|1922Q3|N|Y|N|2423299|2423541|2422955|2423229|N|N|N|N|N| +2423321|AAAAAAAAJBKPECAA|1922-09-23|272|1186|92|1922|6|9|23|3|1922|92|1186|Saturday|1922Q3|N|Y|N|2423299|2423541|2422956|2423230|N|N|N|N|N| +2423322|AAAAAAAAKBKPECAA|1922-09-24|272|1186|92|1922|0|9|24|3|1922|92|1186|Sunday|1922Q3|N|N|N|2423299|2423541|2422957|2423231|N|N|N|N|N| +2423323|AAAAAAAALBKPECAA|1922-09-25|272|1186|92|1922|1|9|25|3|1922|92|1186|Monday|1922Q3|N|N|N|2423299|2423541|2422958|2423232|N|N|N|N|N| +2423324|AAAAAAAAMBKPECAA|1922-09-26|272|1187|92|1922|2|9|26|3|1922|92|1187|Tuesday|1922Q3|N|N|N|2423299|2423541|2422959|2423233|N|N|N|N|N| +2423325|AAAAAAAANBKPECAA|1922-09-27|272|1187|92|1922|3|9|27|3|1922|92|1187|Wednesday|1922Q3|N|N|N|2423299|2423541|2422960|2423234|N|N|N|N|N| +2423326|AAAAAAAAOBKPECAA|1922-09-28|272|1187|92|1922|4|9|28|3|1922|92|1187|Thursday|1922Q3|N|N|N|2423299|2423541|2422961|2423235|N|N|N|N|N| +2423327|AAAAAAAAPBKPECAA|1922-09-29|272|1187|92|1922|5|9|29|3|1922|92|1187|Friday|1922Q3|N|Y|N|2423299|2423541|2422962|2423236|N|N|N|N|N| +2423328|AAAAAAAAACKPECAA|1922-09-30|272|1187|92|1922|6|9|30|3|1922|92|1187|Saturday|1922Q3|N|Y|N|2423299|2423541|2422963|2423237|N|N|N|N|N| +2423329|AAAAAAAABCKPECAA|1922-10-01|273|1187|92|1922|0|10|1|3|1922|92|1187|Sunday|1922Q3|N|N|N|2423329|2423601|2422964|2423237|N|N|N|N|N| +2423330|AAAAAAAACCKPECAA|1922-10-02|273|1187|92|1922|1|10|2|4|1922|92|1187|Monday|1922Q4|N|N|N|2423329|2423601|2422965|2423238|N|N|N|N|N| +2423331|AAAAAAAADCKPECAA|1922-10-03|273|1188|92|1922|2|10|3|4|1922|92|1188|Tuesday|1922Q4|N|N|N|2423329|2423601|2422966|2423239|N|N|N|N|N| +2423332|AAAAAAAAECKPECAA|1922-10-04|273|1188|92|1922|3|10|4|4|1922|92|1188|Wednesday|1922Q4|N|N|N|2423329|2423601|2422967|2423240|N|N|N|N|N| +2423333|AAAAAAAAFCKPECAA|1922-10-05|273|1188|92|1922|4|10|5|4|1922|92|1188|Thursday|1922Q4|N|N|N|2423329|2423601|2422968|2423241|N|N|N|N|N| +2423334|AAAAAAAAGCKPECAA|1922-10-06|273|1188|92|1922|5|10|6|4|1922|92|1188|Friday|1922Q4|N|Y|N|2423329|2423601|2422969|2423242|N|N|N|N|N| +2423335|AAAAAAAAHCKPECAA|1922-10-07|273|1188|92|1922|6|10|7|4|1922|92|1188|Saturday|1922Q4|N|Y|N|2423329|2423601|2422970|2423243|N|N|N|N|N| +2423336|AAAAAAAAICKPECAA|1922-10-08|273|1188|92|1922|0|10|8|4|1922|92|1188|Sunday|1922Q4|N|N|N|2423329|2423601|2422971|2423244|N|N|N|N|N| +2423337|AAAAAAAAJCKPECAA|1922-10-09|273|1188|92|1922|1|10|9|4|1922|92|1188|Monday|1922Q4|N|N|N|2423329|2423601|2422972|2423245|N|N|N|N|N| +2423338|AAAAAAAAKCKPECAA|1922-10-10|273|1189|92|1922|2|10|10|4|1922|92|1189|Tuesday|1922Q4|N|N|N|2423329|2423601|2422973|2423246|N|N|N|N|N| +2423339|AAAAAAAALCKPECAA|1922-10-11|273|1189|92|1922|3|10|11|4|1922|92|1189|Wednesday|1922Q4|N|N|N|2423329|2423601|2422974|2423247|N|N|N|N|N| +2423340|AAAAAAAAMCKPECAA|1922-10-12|273|1189|92|1922|4|10|12|4|1922|92|1189|Thursday|1922Q4|N|N|N|2423329|2423601|2422975|2423248|N|N|N|N|N| +2423341|AAAAAAAANCKPECAA|1922-10-13|273|1189|92|1922|5|10|13|4|1922|92|1189|Friday|1922Q4|N|Y|N|2423329|2423601|2422976|2423249|N|N|N|N|N| +2423342|AAAAAAAAOCKPECAA|1922-10-14|273|1189|92|1922|6|10|14|4|1922|92|1189|Saturday|1922Q4|N|Y|N|2423329|2423601|2422977|2423250|N|N|N|N|N| +2423343|AAAAAAAAPCKPECAA|1922-10-15|273|1189|92|1922|0|10|15|4|1922|92|1189|Sunday|1922Q4|N|N|N|2423329|2423601|2422978|2423251|N|N|N|N|N| +2423344|AAAAAAAAADKPECAA|1922-10-16|273|1189|92|1922|1|10|16|4|1922|92|1189|Monday|1922Q4|N|N|N|2423329|2423601|2422979|2423252|N|N|N|N|N| +2423345|AAAAAAAABDKPECAA|1922-10-17|273|1190|92|1922|2|10|17|4|1922|92|1190|Tuesday|1922Q4|N|N|N|2423329|2423601|2422980|2423253|N|N|N|N|N| +2423346|AAAAAAAACDKPECAA|1922-10-18|273|1190|92|1922|3|10|18|4|1922|92|1190|Wednesday|1922Q4|N|N|N|2423329|2423601|2422981|2423254|N|N|N|N|N| +2423347|AAAAAAAADDKPECAA|1922-10-19|273|1190|92|1922|4|10|19|4|1922|92|1190|Thursday|1922Q4|N|N|N|2423329|2423601|2422982|2423255|N|N|N|N|N| +2423348|AAAAAAAAEDKPECAA|1922-10-20|273|1190|92|1922|5|10|20|4|1922|92|1190|Friday|1922Q4|N|Y|N|2423329|2423601|2422983|2423256|N|N|N|N|N| +2423349|AAAAAAAAFDKPECAA|1922-10-21|273|1190|92|1922|6|10|21|4|1922|92|1190|Saturday|1922Q4|N|Y|N|2423329|2423601|2422984|2423257|N|N|N|N|N| +2423350|AAAAAAAAGDKPECAA|1922-10-22|273|1190|92|1922|0|10|22|4|1922|92|1190|Sunday|1922Q4|N|N|N|2423329|2423601|2422985|2423258|N|N|N|N|N| +2423351|AAAAAAAAHDKPECAA|1922-10-23|273|1190|92|1922|1|10|23|4|1922|92|1190|Monday|1922Q4|N|N|N|2423329|2423601|2422986|2423259|N|N|N|N|N| +2423352|AAAAAAAAIDKPECAA|1922-10-24|273|1191|92|1922|2|10|24|4|1922|92|1191|Tuesday|1922Q4|N|N|N|2423329|2423601|2422987|2423260|N|N|N|N|N| +2423353|AAAAAAAAJDKPECAA|1922-10-25|273|1191|92|1922|3|10|25|4|1922|92|1191|Wednesday|1922Q4|N|N|N|2423329|2423601|2422988|2423261|N|N|N|N|N| +2423354|AAAAAAAAKDKPECAA|1922-10-26|273|1191|92|1922|4|10|26|4|1922|92|1191|Thursday|1922Q4|N|N|N|2423329|2423601|2422989|2423262|N|N|N|N|N| +2423355|AAAAAAAALDKPECAA|1922-10-27|273|1191|92|1922|5|10|27|4|1922|92|1191|Friday|1922Q4|N|Y|N|2423329|2423601|2422990|2423263|N|N|N|N|N| +2423356|AAAAAAAAMDKPECAA|1922-10-28|273|1191|92|1922|6|10|28|4|1922|92|1191|Saturday|1922Q4|N|Y|N|2423329|2423601|2422991|2423264|N|N|N|N|N| +2423357|AAAAAAAANDKPECAA|1922-10-29|273|1191|92|1922|0|10|29|4|1922|92|1191|Sunday|1922Q4|N|N|N|2423329|2423601|2422992|2423265|N|N|N|N|N| +2423358|AAAAAAAAODKPECAA|1922-10-30|273|1191|92|1922|1|10|30|4|1922|92|1191|Monday|1922Q4|N|N|N|2423329|2423601|2422993|2423266|N|N|N|N|N| +2423359|AAAAAAAAPDKPECAA|1922-10-31|273|1192|92|1922|2|10|31|4|1922|92|1192|Tuesday|1922Q4|N|N|N|2423329|2423601|2422994|2423267|N|N|N|N|N| +2423360|AAAAAAAAAEKPECAA|1922-11-01|274|1192|92|1922|3|11|1|4|1922|92|1192|Wednesday|1922Q4|N|N|N|2423360|2423663|2422995|2423268|N|N|N|N|N| +2423361|AAAAAAAABEKPECAA|1922-11-02|274|1192|92|1922|4|11|2|4|1922|92|1192|Thursday|1922Q4|N|N|N|2423360|2423663|2422996|2423269|N|N|N|N|N| +2423362|AAAAAAAACEKPECAA|1922-11-03|274|1192|92|1922|5|11|3|4|1922|92|1192|Friday|1922Q4|N|Y|N|2423360|2423663|2422997|2423270|N|N|N|N|N| +2423363|AAAAAAAADEKPECAA|1922-11-04|274|1192|92|1922|6|11|4|4|1922|92|1192|Saturday|1922Q4|N|Y|N|2423360|2423663|2422998|2423271|N|N|N|N|N| +2423364|AAAAAAAAEEKPECAA|1922-11-05|274|1192|92|1922|0|11|5|4|1922|92|1192|Sunday|1922Q4|N|N|N|2423360|2423663|2422999|2423272|N|N|N|N|N| +2423365|AAAAAAAAFEKPECAA|1922-11-06|274|1192|92|1922|1|11|6|4|1922|92|1192|Monday|1922Q4|N|N|N|2423360|2423663|2423000|2423273|N|N|N|N|N| +2423366|AAAAAAAAGEKPECAA|1922-11-07|274|1193|92|1922|2|11|7|4|1922|92|1193|Tuesday|1922Q4|N|N|N|2423360|2423663|2423001|2423274|N|N|N|N|N| +2423367|AAAAAAAAHEKPECAA|1922-11-08|274|1193|92|1922|3|11|8|4|1922|92|1193|Wednesday|1922Q4|N|N|N|2423360|2423663|2423002|2423275|N|N|N|N|N| +2423368|AAAAAAAAIEKPECAA|1922-11-09|274|1193|92|1922|4|11|9|4|1922|92|1193|Thursday|1922Q4|N|N|N|2423360|2423663|2423003|2423276|N|N|N|N|N| +2423369|AAAAAAAAJEKPECAA|1922-11-10|274|1193|92|1922|5|11|10|4|1922|92|1193|Friday|1922Q4|N|Y|N|2423360|2423663|2423004|2423277|N|N|N|N|N| +2423370|AAAAAAAAKEKPECAA|1922-11-11|274|1193|92|1922|6|11|11|4|1922|92|1193|Saturday|1922Q4|N|Y|N|2423360|2423663|2423005|2423278|N|N|N|N|N| +2423371|AAAAAAAALEKPECAA|1922-11-12|274|1193|92|1922|0|11|12|4|1922|92|1193|Sunday|1922Q4|N|N|N|2423360|2423663|2423006|2423279|N|N|N|N|N| +2423372|AAAAAAAAMEKPECAA|1922-11-13|274|1193|92|1922|1|11|13|4|1922|92|1193|Monday|1922Q4|N|N|N|2423360|2423663|2423007|2423280|N|N|N|N|N| +2423373|AAAAAAAANEKPECAA|1922-11-14|274|1194|92|1922|2|11|14|4|1922|92|1194|Tuesday|1922Q4|N|N|N|2423360|2423663|2423008|2423281|N|N|N|N|N| +2423374|AAAAAAAAOEKPECAA|1922-11-15|274|1194|92|1922|3|11|15|4|1922|92|1194|Wednesday|1922Q4|N|N|N|2423360|2423663|2423009|2423282|N|N|N|N|N| +2423375|AAAAAAAAPEKPECAA|1922-11-16|274|1194|92|1922|4|11|16|4|1922|92|1194|Thursday|1922Q4|N|N|N|2423360|2423663|2423010|2423283|N|N|N|N|N| +2423376|AAAAAAAAAFKPECAA|1922-11-17|274|1194|92|1922|5|11|17|4|1922|92|1194|Friday|1922Q4|N|Y|N|2423360|2423663|2423011|2423284|N|N|N|N|N| +2423377|AAAAAAAABFKPECAA|1922-11-18|274|1194|92|1922|6|11|18|4|1922|92|1194|Saturday|1922Q4|N|Y|N|2423360|2423663|2423012|2423285|N|N|N|N|N| +2423378|AAAAAAAACFKPECAA|1922-11-19|274|1194|92|1922|0|11|19|4|1922|92|1194|Sunday|1922Q4|N|N|N|2423360|2423663|2423013|2423286|N|N|N|N|N| +2423379|AAAAAAAADFKPECAA|1922-11-20|274|1194|92|1922|1|11|20|4|1922|92|1194|Monday|1922Q4|N|N|N|2423360|2423663|2423014|2423287|N|N|N|N|N| +2423380|AAAAAAAAEFKPECAA|1922-11-21|274|1195|92|1922|2|11|21|4|1922|92|1195|Tuesday|1922Q4|N|N|N|2423360|2423663|2423015|2423288|N|N|N|N|N| +2423381|AAAAAAAAFFKPECAA|1922-11-22|274|1195|92|1922|3|11|22|4|1922|92|1195|Wednesday|1922Q4|N|N|N|2423360|2423663|2423016|2423289|N|N|N|N|N| +2423382|AAAAAAAAGFKPECAA|1922-11-23|274|1195|92|1922|4|11|23|4|1922|92|1195|Thursday|1922Q4|N|N|N|2423360|2423663|2423017|2423290|N|N|N|N|N| +2423383|AAAAAAAAHFKPECAA|1922-11-24|274|1195|92|1922|5|11|24|4|1922|92|1195|Friday|1922Q4|N|Y|N|2423360|2423663|2423018|2423291|N|N|N|N|N| +2423384|AAAAAAAAIFKPECAA|1922-11-25|274|1195|92|1922|6|11|25|4|1922|92|1195|Saturday|1922Q4|N|Y|N|2423360|2423663|2423019|2423292|N|N|N|N|N| +2423385|AAAAAAAAJFKPECAA|1922-11-26|274|1195|92|1922|0|11|26|4|1922|92|1195|Sunday|1922Q4|N|N|N|2423360|2423663|2423020|2423293|N|N|N|N|N| +2423386|AAAAAAAAKFKPECAA|1922-11-27|274|1195|92|1922|1|11|27|4|1922|92|1195|Monday|1922Q4|N|N|N|2423360|2423663|2423021|2423294|N|N|N|N|N| +2423387|AAAAAAAALFKPECAA|1922-11-28|274|1196|92|1922|2|11|28|4|1922|92|1196|Tuesday|1922Q4|N|N|N|2423360|2423663|2423022|2423295|N|N|N|N|N| +2423388|AAAAAAAAMFKPECAA|1922-11-29|274|1196|92|1922|3|11|29|4|1922|92|1196|Wednesday|1922Q4|N|N|N|2423360|2423663|2423023|2423296|N|N|N|N|N| +2423389|AAAAAAAANFKPECAA|1922-11-30|274|1196|92|1922|4|11|30|4|1922|92|1196|Thursday|1922Q4|N|N|N|2423360|2423663|2423024|2423297|N|N|N|N|N| +2423390|AAAAAAAAOFKPECAA|1922-12-01|275|1196|93|1922|5|12|1|4|1922|93|1196|Friday|1922Q4|N|Y|N|2423390|2423723|2423025|2423298|N|N|N|N|N| +2423391|AAAAAAAAPFKPECAA|1922-12-02|275|1196|93|1922|6|12|2|4|1922|93|1196|Saturday|1922Q4|N|Y|N|2423390|2423723|2423026|2423299|N|N|N|N|N| +2423392|AAAAAAAAAGKPECAA|1922-12-03|275|1196|93|1922|0|12|3|4|1922|93|1196|Sunday|1922Q4|N|N|N|2423390|2423723|2423027|2423300|N|N|N|N|N| +2423393|AAAAAAAABGKPECAA|1922-12-04|275|1196|93|1922|1|12|4|4|1922|93|1196|Monday|1922Q4|N|N|N|2423390|2423723|2423028|2423301|N|N|N|N|N| +2423394|AAAAAAAACGKPECAA|1922-12-05|275|1197|93|1922|2|12|5|4|1922|93|1197|Tuesday|1922Q4|N|N|N|2423390|2423723|2423029|2423302|N|N|N|N|N| +2423395|AAAAAAAADGKPECAA|1922-12-06|275|1197|93|1922|3|12|6|4|1922|93|1197|Wednesday|1922Q4|N|N|N|2423390|2423723|2423030|2423303|N|N|N|N|N| +2423396|AAAAAAAAEGKPECAA|1922-12-07|275|1197|93|1922|4|12|7|4|1922|93|1197|Thursday|1922Q4|N|N|N|2423390|2423723|2423031|2423304|N|N|N|N|N| +2423397|AAAAAAAAFGKPECAA|1922-12-08|275|1197|93|1922|5|12|8|4|1922|93|1197|Friday|1922Q4|N|Y|N|2423390|2423723|2423032|2423305|N|N|N|N|N| +2423398|AAAAAAAAGGKPECAA|1922-12-09|275|1197|93|1922|6|12|9|4|1922|93|1197|Saturday|1922Q4|N|Y|N|2423390|2423723|2423033|2423306|N|N|N|N|N| +2423399|AAAAAAAAHGKPECAA|1922-12-10|275|1197|93|1922|0|12|10|4|1922|93|1197|Sunday|1922Q4|N|N|N|2423390|2423723|2423034|2423307|N|N|N|N|N| +2423400|AAAAAAAAIGKPECAA|1922-12-11|275|1197|93|1922|1|12|11|4|1922|93|1197|Monday|1922Q4|N|N|N|2423390|2423723|2423035|2423308|N|N|N|N|N| +2423401|AAAAAAAAJGKPECAA|1922-12-12|275|1198|93|1922|2|12|12|4|1922|93|1198|Tuesday|1922Q4|N|N|N|2423390|2423723|2423036|2423309|N|N|N|N|N| +2423402|AAAAAAAAKGKPECAA|1922-12-13|275|1198|93|1922|3|12|13|4|1922|93|1198|Wednesday|1922Q4|N|N|N|2423390|2423723|2423037|2423310|N|N|N|N|N| +2423403|AAAAAAAALGKPECAA|1922-12-14|275|1198|93|1922|4|12|14|4|1922|93|1198|Thursday|1922Q4|N|N|N|2423390|2423723|2423038|2423311|N|N|N|N|N| +2423404|AAAAAAAAMGKPECAA|1922-12-15|275|1198|93|1922|5|12|15|4|1922|93|1198|Friday|1922Q4|N|Y|N|2423390|2423723|2423039|2423312|N|N|N|N|N| +2423405|AAAAAAAANGKPECAA|1922-12-16|275|1198|93|1922|6|12|16|4|1922|93|1198|Saturday|1922Q4|N|Y|N|2423390|2423723|2423040|2423313|N|N|N|N|N| +2423406|AAAAAAAAOGKPECAA|1922-12-17|275|1198|93|1922|0|12|17|4|1922|93|1198|Sunday|1922Q4|N|N|N|2423390|2423723|2423041|2423314|N|N|N|N|N| +2423407|AAAAAAAAPGKPECAA|1922-12-18|275|1198|93|1922|1|12|18|4|1922|93|1198|Monday|1922Q4|N|N|N|2423390|2423723|2423042|2423315|N|N|N|N|N| +2423408|AAAAAAAAAHKPECAA|1922-12-19|275|1199|93|1922|2|12|19|4|1922|93|1199|Tuesday|1922Q4|N|N|N|2423390|2423723|2423043|2423316|N|N|N|N|N| +2423409|AAAAAAAABHKPECAA|1922-12-20|275|1199|93|1922|3|12|20|4|1922|93|1199|Wednesday|1922Q4|N|N|N|2423390|2423723|2423044|2423317|N|N|N|N|N| +2423410|AAAAAAAACHKPECAA|1922-12-21|275|1199|93|1922|4|12|21|4|1922|93|1199|Thursday|1922Q4|N|N|N|2423390|2423723|2423045|2423318|N|N|N|N|N| +2423411|AAAAAAAADHKPECAA|1922-12-22|275|1199|93|1922|5|12|22|4|1922|93|1199|Friday|1922Q4|N|Y|N|2423390|2423723|2423046|2423319|N|N|N|N|N| +2423412|AAAAAAAAEHKPECAA|1922-12-23|275|1199|93|1922|6|12|23|4|1922|93|1199|Saturday|1922Q4|N|Y|N|2423390|2423723|2423047|2423320|N|N|N|N|N| +2423413|AAAAAAAAFHKPECAA|1922-12-24|275|1199|93|1922|0|12|24|4|1922|93|1199|Sunday|1922Q4|N|N|N|2423390|2423723|2423048|2423321|N|N|N|N|N| +2423414|AAAAAAAAGHKPECAA|1922-12-25|275|1199|93|1922|1|12|25|4|1922|93|1199|Monday|1922Q4|N|N|N|2423390|2423723|2423049|2423322|N|N|N|N|N| +2423415|AAAAAAAAHHKPECAA|1922-12-26|275|1200|93|1922|2|12|26|4|1922|93|1200|Tuesday|1922Q4|Y|N|N|2423390|2423723|2423050|2423323|N|N|N|N|N| +2423416|AAAAAAAAIHKPECAA|1922-12-27|275|1200|93|1922|3|12|27|4|1922|93|1200|Wednesday|1922Q4|N|N|Y|2423390|2423723|2423051|2423324|N|N|N|N|N| +2423417|AAAAAAAAJHKPECAA|1922-12-28|275|1200|93|1922|4|12|28|4|1922|93|1200|Thursday|1922Q4|N|N|N|2423390|2423723|2423052|2423325|N|N|N|N|N| +2423418|AAAAAAAAKHKPECAA|1922-12-29|275|1200|93|1922|5|12|29|4|1922|93|1200|Friday|1922Q4|N|Y|N|2423390|2423723|2423053|2423326|N|N|N|N|N| +2423419|AAAAAAAALHKPECAA|1922-12-30|275|1200|93|1922|6|12|30|4|1922|93|1200|Saturday|1922Q4|N|Y|N|2423390|2423723|2423054|2423327|N|N|N|N|N| +2423420|AAAAAAAAMHKPECAA|1922-12-31|275|1200|93|1922|0|12|31|4|1922|93|1200|Sunday|1922Q4|N|N|N|2423390|2423723|2423055|2423328|N|N|N|N|N| +2423421|AAAAAAAANHKPECAA|1923-01-01|276|1200|93|1923|1|1|1|1|1923|93|1200|Monday|1923Q1|Y|N|N|2423421|2423420|2423056|2423329|N|N|N|N|N| +2423422|AAAAAAAAOHKPECAA|1923-01-02|276|1201|93|1923|2|1|2|1|1923|93|1201|Tuesday|1923Q1|N|N|Y|2423421|2423420|2423057|2423330|N|N|N|N|N| +2423423|AAAAAAAAPHKPECAA|1923-01-03|276|1201|93|1923|3|1|3|1|1923|93|1201|Wednesday|1923Q1|N|N|N|2423421|2423420|2423058|2423331|N|N|N|N|N| +2423424|AAAAAAAAAIKPECAA|1923-01-04|276|1201|93|1923|4|1|4|1|1923|93|1201|Thursday|1923Q1|N|N|N|2423421|2423420|2423059|2423332|N|N|N|N|N| +2423425|AAAAAAAABIKPECAA|1923-01-05|276|1201|93|1923|5|1|5|1|1923|93|1201|Friday|1923Q1|N|Y|N|2423421|2423420|2423060|2423333|N|N|N|N|N| +2423426|AAAAAAAACIKPECAA|1923-01-06|276|1201|93|1923|6|1|6|1|1923|93|1201|Saturday|1923Q1|N|Y|N|2423421|2423420|2423061|2423334|N|N|N|N|N| +2423427|AAAAAAAADIKPECAA|1923-01-07|276|1201|93|1923|0|1|7|1|1923|93|1201|Sunday|1923Q1|N|N|N|2423421|2423420|2423062|2423335|N|N|N|N|N| +2423428|AAAAAAAAEIKPECAA|1923-01-08|276|1201|93|1923|1|1|8|1|1923|93|1201|Monday|1923Q1|N|N|N|2423421|2423420|2423063|2423336|N|N|N|N|N| +2423429|AAAAAAAAFIKPECAA|1923-01-09|276|1202|93|1923|2|1|9|1|1923|93|1202|Tuesday|1923Q1|N|N|N|2423421|2423420|2423064|2423337|N|N|N|N|N| +2423430|AAAAAAAAGIKPECAA|1923-01-10|276|1202|93|1923|3|1|10|1|1923|93|1202|Wednesday|1923Q1|N|N|N|2423421|2423420|2423065|2423338|N|N|N|N|N| +2423431|AAAAAAAAHIKPECAA|1923-01-11|276|1202|93|1923|4|1|11|1|1923|93|1202|Thursday|1923Q1|N|N|N|2423421|2423420|2423066|2423339|N|N|N|N|N| +2423432|AAAAAAAAIIKPECAA|1923-01-12|276|1202|93|1923|5|1|12|1|1923|93|1202|Friday|1923Q1|N|Y|N|2423421|2423420|2423067|2423340|N|N|N|N|N| +2423433|AAAAAAAAJIKPECAA|1923-01-13|276|1202|93|1923|6|1|13|1|1923|93|1202|Saturday|1923Q1|N|Y|N|2423421|2423420|2423068|2423341|N|N|N|N|N| +2423434|AAAAAAAAKIKPECAA|1923-01-14|276|1202|93|1923|0|1|14|1|1923|93|1202|Sunday|1923Q1|N|N|N|2423421|2423420|2423069|2423342|N|N|N|N|N| +2423435|AAAAAAAALIKPECAA|1923-01-15|276|1202|93|1923|1|1|15|1|1923|93|1202|Monday|1923Q1|N|N|N|2423421|2423420|2423070|2423343|N|N|N|N|N| +2423436|AAAAAAAAMIKPECAA|1923-01-16|276|1203|93|1923|2|1|16|1|1923|93|1203|Tuesday|1923Q1|N|N|N|2423421|2423420|2423071|2423344|N|N|N|N|N| +2423437|AAAAAAAANIKPECAA|1923-01-17|276|1203|93|1923|3|1|17|1|1923|93|1203|Wednesday|1923Q1|N|N|N|2423421|2423420|2423072|2423345|N|N|N|N|N| +2423438|AAAAAAAAOIKPECAA|1923-01-18|276|1203|93|1923|4|1|18|1|1923|93|1203|Thursday|1923Q1|N|N|N|2423421|2423420|2423073|2423346|N|N|N|N|N| +2423439|AAAAAAAAPIKPECAA|1923-01-19|276|1203|93|1923|5|1|19|1|1923|93|1203|Friday|1923Q1|N|Y|N|2423421|2423420|2423074|2423347|N|N|N|N|N| +2423440|AAAAAAAAAJKPECAA|1923-01-20|276|1203|93|1923|6|1|20|1|1923|93|1203|Saturday|1923Q1|N|Y|N|2423421|2423420|2423075|2423348|N|N|N|N|N| +2423441|AAAAAAAABJKPECAA|1923-01-21|276|1203|93|1923|0|1|21|1|1923|93|1203|Sunday|1923Q1|N|N|N|2423421|2423420|2423076|2423349|N|N|N|N|N| +2423442|AAAAAAAACJKPECAA|1923-01-22|276|1203|93|1923|1|1|22|1|1923|93|1203|Monday|1923Q1|N|N|N|2423421|2423420|2423077|2423350|N|N|N|N|N| +2423443|AAAAAAAADJKPECAA|1923-01-23|276|1204|93|1923|2|1|23|1|1923|93|1204|Tuesday|1923Q1|N|N|N|2423421|2423420|2423078|2423351|N|N|N|N|N| +2423444|AAAAAAAAEJKPECAA|1923-01-24|276|1204|93|1923|3|1|24|1|1923|93|1204|Wednesday|1923Q1|N|N|N|2423421|2423420|2423079|2423352|N|N|N|N|N| +2423445|AAAAAAAAFJKPECAA|1923-01-25|276|1204|93|1923|4|1|25|1|1923|93|1204|Thursday|1923Q1|N|N|N|2423421|2423420|2423080|2423353|N|N|N|N|N| +2423446|AAAAAAAAGJKPECAA|1923-01-26|276|1204|93|1923|5|1|26|1|1923|93|1204|Friday|1923Q1|N|Y|N|2423421|2423420|2423081|2423354|N|N|N|N|N| +2423447|AAAAAAAAHJKPECAA|1923-01-27|276|1204|93|1923|6|1|27|1|1923|93|1204|Saturday|1923Q1|N|Y|N|2423421|2423420|2423082|2423355|N|N|N|N|N| +2423448|AAAAAAAAIJKPECAA|1923-01-28|276|1204|93|1923|0|1|28|1|1923|93|1204|Sunday|1923Q1|N|N|N|2423421|2423420|2423083|2423356|N|N|N|N|N| +2423449|AAAAAAAAJJKPECAA|1923-01-29|276|1204|93|1923|1|1|29|1|1923|93|1204|Monday|1923Q1|N|N|N|2423421|2423420|2423084|2423357|N|N|N|N|N| +2423450|AAAAAAAAKJKPECAA|1923-01-30|276|1205|93|1923|2|1|30|1|1923|93|1205|Tuesday|1923Q1|N|N|N|2423421|2423420|2423085|2423358|N|N|N|N|N| +2423451|AAAAAAAALJKPECAA|1923-01-31|276|1205|93|1923|3|1|31|1|1923|93|1205|Wednesday|1923Q1|N|N|N|2423421|2423420|2423086|2423359|N|N|N|N|N| +2423452|AAAAAAAAMJKPECAA|1923-02-01|277|1205|93|1923|4|2|1|1|1923|93|1205|Thursday|1923Q1|N|N|N|2423452|2423482|2423087|2423360|N|N|N|N|N| +2423453|AAAAAAAANJKPECAA|1923-02-02|277|1205|93|1923|5|2|2|1|1923|93|1205|Friday|1923Q1|N|Y|N|2423452|2423482|2423088|2423361|N|N|N|N|N| +2423454|AAAAAAAAOJKPECAA|1923-02-03|277|1205|93|1923|6|2|3|1|1923|93|1205|Saturday|1923Q1|N|Y|N|2423452|2423482|2423089|2423362|N|N|N|N|N| +2423455|AAAAAAAAPJKPECAA|1923-02-04|277|1205|93|1923|0|2|4|1|1923|93|1205|Sunday|1923Q1|N|N|N|2423452|2423482|2423090|2423363|N|N|N|N|N| +2423456|AAAAAAAAAKKPECAA|1923-02-05|277|1205|93|1923|1|2|5|1|1923|93|1205|Monday|1923Q1|N|N|N|2423452|2423482|2423091|2423364|N|N|N|N|N| +2423457|AAAAAAAABKKPECAA|1923-02-06|277|1206|93|1923|2|2|6|1|1923|93|1206|Tuesday|1923Q1|N|N|N|2423452|2423482|2423092|2423365|N|N|N|N|N| +2423458|AAAAAAAACKKPECAA|1923-02-07|277|1206|93|1923|3|2|7|1|1923|93|1206|Wednesday|1923Q1|N|N|N|2423452|2423482|2423093|2423366|N|N|N|N|N| +2423459|AAAAAAAADKKPECAA|1923-02-08|277|1206|93|1923|4|2|8|1|1923|93|1206|Thursday|1923Q1|N|N|N|2423452|2423482|2423094|2423367|N|N|N|N|N| +2423460|AAAAAAAAEKKPECAA|1923-02-09|277|1206|93|1923|5|2|9|1|1923|93|1206|Friday|1923Q1|N|Y|N|2423452|2423482|2423095|2423368|N|N|N|N|N| +2423461|AAAAAAAAFKKPECAA|1923-02-10|277|1206|93|1923|6|2|10|1|1923|93|1206|Saturday|1923Q1|N|Y|N|2423452|2423482|2423096|2423369|N|N|N|N|N| +2423462|AAAAAAAAGKKPECAA|1923-02-11|277|1206|93|1923|0|2|11|1|1923|93|1206|Sunday|1923Q1|N|N|N|2423452|2423482|2423097|2423370|N|N|N|N|N| +2423463|AAAAAAAAHKKPECAA|1923-02-12|277|1206|93|1923|1|2|12|1|1923|93|1206|Monday|1923Q1|N|N|N|2423452|2423482|2423098|2423371|N|N|N|N|N| +2423464|AAAAAAAAIKKPECAA|1923-02-13|277|1207|93|1923|2|2|13|1|1923|93|1207|Tuesday|1923Q1|N|N|N|2423452|2423482|2423099|2423372|N|N|N|N|N| +2423465|AAAAAAAAJKKPECAA|1923-02-14|277|1207|93|1923|3|2|14|1|1923|93|1207|Wednesday|1923Q1|N|N|N|2423452|2423482|2423100|2423373|N|N|N|N|N| +2423466|AAAAAAAAKKKPECAA|1923-02-15|277|1207|93|1923|4|2|15|1|1923|93|1207|Thursday|1923Q1|N|N|N|2423452|2423482|2423101|2423374|N|N|N|N|N| +2423467|AAAAAAAALKKPECAA|1923-02-16|277|1207|93|1923|5|2|16|1|1923|93|1207|Friday|1923Q1|N|Y|N|2423452|2423482|2423102|2423375|N|N|N|N|N| +2423468|AAAAAAAAMKKPECAA|1923-02-17|277|1207|93|1923|6|2|17|1|1923|93|1207|Saturday|1923Q1|N|Y|N|2423452|2423482|2423103|2423376|N|N|N|N|N| +2423469|AAAAAAAANKKPECAA|1923-02-18|277|1207|93|1923|0|2|18|1|1923|93|1207|Sunday|1923Q1|N|N|N|2423452|2423482|2423104|2423377|N|N|N|N|N| +2423470|AAAAAAAAOKKPECAA|1923-02-19|277|1207|93|1923|1|2|19|1|1923|93|1207|Monday|1923Q1|N|N|N|2423452|2423482|2423105|2423378|N|N|N|N|N| +2423471|AAAAAAAAPKKPECAA|1923-02-20|277|1208|93|1923|2|2|20|1|1923|93|1208|Tuesday|1923Q1|N|N|N|2423452|2423482|2423106|2423379|N|N|N|N|N| +2423472|AAAAAAAAALKPECAA|1923-02-21|277|1208|93|1923|3|2|21|1|1923|93|1208|Wednesday|1923Q1|N|N|N|2423452|2423482|2423107|2423380|N|N|N|N|N| +2423473|AAAAAAAABLKPECAA|1923-02-22|277|1208|93|1923|4|2|22|1|1923|93|1208|Thursday|1923Q1|N|N|N|2423452|2423482|2423108|2423381|N|N|N|N|N| +2423474|AAAAAAAACLKPECAA|1923-02-23|277|1208|93|1923|5|2|23|1|1923|93|1208|Friday|1923Q1|N|Y|N|2423452|2423482|2423109|2423382|N|N|N|N|N| +2423475|AAAAAAAADLKPECAA|1923-02-24|277|1208|93|1923|6|2|24|1|1923|93|1208|Saturday|1923Q1|N|Y|N|2423452|2423482|2423110|2423383|N|N|N|N|N| +2423476|AAAAAAAAELKPECAA|1923-02-25|277|1208|93|1923|0|2|25|1|1923|93|1208|Sunday|1923Q1|N|N|N|2423452|2423482|2423111|2423384|N|N|N|N|N| +2423477|AAAAAAAAFLKPECAA|1923-02-26|277|1208|93|1923|1|2|26|1|1923|93|1208|Monday|1923Q1|N|N|N|2423452|2423482|2423112|2423385|N|N|N|N|N| +2423478|AAAAAAAAGLKPECAA|1923-02-27|277|1209|93|1923|2|2|27|1|1923|93|1209|Tuesday|1923Q1|N|N|N|2423452|2423482|2423113|2423386|N|N|N|N|N| +2423479|AAAAAAAAHLKPECAA|1923-02-28|277|1209|93|1923|3|2|28|1|1923|93|1209|Wednesday|1923Q1|N|N|N|2423452|2423482|2423114|2423387|N|N|N|N|N| +2423480|AAAAAAAAILKPECAA|1923-03-01|278|1209|94|1923|4|3|1|1|1923|94|1209|Thursday|1923Q1|N|N|N|2423480|2423538|2423115|2423388|N|N|N|N|N| +2423481|AAAAAAAAJLKPECAA|1923-03-02|278|1209|94|1923|5|3|2|1|1923|94|1209|Friday|1923Q1|N|Y|N|2423480|2423538|2423116|2423389|N|N|N|N|N| +2423482|AAAAAAAAKLKPECAA|1923-03-03|278|1209|94|1923|6|3|3|1|1923|94|1209|Saturday|1923Q1|N|Y|N|2423480|2423538|2423117|2423390|N|N|N|N|N| +2423483|AAAAAAAALLKPECAA|1923-03-04|278|1209|94|1923|0|3|4|1|1923|94|1209|Sunday|1923Q1|N|N|N|2423480|2423538|2423118|2423391|N|N|N|N|N| +2423484|AAAAAAAAMLKPECAA|1923-03-05|278|1209|94|1923|1|3|5|1|1923|94|1209|Monday|1923Q1|N|N|N|2423480|2423538|2423119|2423392|N|N|N|N|N| +2423485|AAAAAAAANLKPECAA|1923-03-06|278|1210|94|1923|2|3|6|1|1923|94|1210|Tuesday|1923Q1|N|N|N|2423480|2423538|2423120|2423393|N|N|N|N|N| +2423486|AAAAAAAAOLKPECAA|1923-03-07|278|1210|94|1923|3|3|7|1|1923|94|1210|Wednesday|1923Q1|N|N|N|2423480|2423538|2423121|2423394|N|N|N|N|N| +2423487|AAAAAAAAPLKPECAA|1923-03-08|278|1210|94|1923|4|3|8|1|1923|94|1210|Thursday|1923Q1|N|N|N|2423480|2423538|2423122|2423395|N|N|N|N|N| +2423488|AAAAAAAAAMKPECAA|1923-03-09|278|1210|94|1923|5|3|9|1|1923|94|1210|Friday|1923Q1|N|Y|N|2423480|2423538|2423123|2423396|N|N|N|N|N| +2423489|AAAAAAAABMKPECAA|1923-03-10|278|1210|94|1923|6|3|10|1|1923|94|1210|Saturday|1923Q1|N|Y|N|2423480|2423538|2423124|2423397|N|N|N|N|N| +2423490|AAAAAAAACMKPECAA|1923-03-11|278|1210|94|1923|0|3|11|1|1923|94|1210|Sunday|1923Q1|N|N|N|2423480|2423538|2423125|2423398|N|N|N|N|N| +2423491|AAAAAAAADMKPECAA|1923-03-12|278|1210|94|1923|1|3|12|1|1923|94|1210|Monday|1923Q1|N|N|N|2423480|2423538|2423126|2423399|N|N|N|N|N| +2423492|AAAAAAAAEMKPECAA|1923-03-13|278|1211|94|1923|2|3|13|1|1923|94|1211|Tuesday|1923Q1|N|N|N|2423480|2423538|2423127|2423400|N|N|N|N|N| +2423493|AAAAAAAAFMKPECAA|1923-03-14|278|1211|94|1923|3|3|14|1|1923|94|1211|Wednesday|1923Q1|N|N|N|2423480|2423538|2423128|2423401|N|N|N|N|N| +2423494|AAAAAAAAGMKPECAA|1923-03-15|278|1211|94|1923|4|3|15|1|1923|94|1211|Thursday|1923Q1|N|N|N|2423480|2423538|2423129|2423402|N|N|N|N|N| +2423495|AAAAAAAAHMKPECAA|1923-03-16|278|1211|94|1923|5|3|16|1|1923|94|1211|Friday|1923Q1|N|Y|N|2423480|2423538|2423130|2423403|N|N|N|N|N| +2423496|AAAAAAAAIMKPECAA|1923-03-17|278|1211|94|1923|6|3|17|1|1923|94|1211|Saturday|1923Q1|N|Y|N|2423480|2423538|2423131|2423404|N|N|N|N|N| +2423497|AAAAAAAAJMKPECAA|1923-03-18|278|1211|94|1923|0|3|18|1|1923|94|1211|Sunday|1923Q1|N|N|N|2423480|2423538|2423132|2423405|N|N|N|N|N| +2423498|AAAAAAAAKMKPECAA|1923-03-19|278|1211|94|1923|1|3|19|1|1923|94|1211|Monday|1923Q1|N|N|N|2423480|2423538|2423133|2423406|N|N|N|N|N| +2423499|AAAAAAAALMKPECAA|1923-03-20|278|1212|94|1923|2|3|20|1|1923|94|1212|Tuesday|1923Q1|N|N|N|2423480|2423538|2423134|2423407|N|N|N|N|N| +2423500|AAAAAAAAMMKPECAA|1923-03-21|278|1212|94|1923|3|3|21|1|1923|94|1212|Wednesday|1923Q1|N|N|N|2423480|2423538|2423135|2423408|N|N|N|N|N| +2423501|AAAAAAAANMKPECAA|1923-03-22|278|1212|94|1923|4|3|22|1|1923|94|1212|Thursday|1923Q1|N|N|N|2423480|2423538|2423136|2423409|N|N|N|N|N| +2423502|AAAAAAAAOMKPECAA|1923-03-23|278|1212|94|1923|5|3|23|1|1923|94|1212|Friday|1923Q1|N|Y|N|2423480|2423538|2423137|2423410|N|N|N|N|N| +2423503|AAAAAAAAPMKPECAA|1923-03-24|278|1212|94|1923|6|3|24|1|1923|94|1212|Saturday|1923Q1|N|Y|N|2423480|2423538|2423138|2423411|N|N|N|N|N| +2423504|AAAAAAAAANKPECAA|1923-03-25|278|1212|94|1923|0|3|25|1|1923|94|1212|Sunday|1923Q1|N|N|N|2423480|2423538|2423139|2423412|N|N|N|N|N| +2423505|AAAAAAAABNKPECAA|1923-03-26|278|1212|94|1923|1|3|26|1|1923|94|1212|Monday|1923Q1|N|N|N|2423480|2423538|2423140|2423413|N|N|N|N|N| +2423506|AAAAAAAACNKPECAA|1923-03-27|278|1213|94|1923|2|3|27|1|1923|94|1213|Tuesday|1923Q1|N|N|N|2423480|2423538|2423141|2423414|N|N|N|N|N| +2423507|AAAAAAAADNKPECAA|1923-03-28|278|1213|94|1923|3|3|28|1|1923|94|1213|Wednesday|1923Q1|N|N|N|2423480|2423538|2423142|2423415|N|N|N|N|N| +2423508|AAAAAAAAENKPECAA|1923-03-29|278|1213|94|1923|4|3|29|1|1923|94|1213|Thursday|1923Q1|N|N|N|2423480|2423538|2423143|2423416|N|N|N|N|N| +2423509|AAAAAAAAFNKPECAA|1923-03-30|278|1213|94|1923|5|3|30|1|1923|94|1213|Friday|1923Q1|N|Y|N|2423480|2423538|2423144|2423417|N|N|N|N|N| +2423510|AAAAAAAAGNKPECAA|1923-03-31|278|1213|94|1923|6|3|31|1|1923|94|1213|Saturday|1923Q1|N|Y|N|2423480|2423538|2423145|2423418|N|N|N|N|N| +2423511|AAAAAAAAHNKPECAA|1923-04-01|279|1213|94|1923|0|4|1|1|1923|94|1213|Sunday|1923Q1|N|N|N|2423511|2423600|2423146|2423421|N|N|N|N|N| +2423512|AAAAAAAAINKPECAA|1923-04-02|279|1213|94|1923|1|4|2|2|1923|94|1213|Monday|1923Q2|N|N|N|2423511|2423600|2423147|2423422|N|N|N|N|N| +2423513|AAAAAAAAJNKPECAA|1923-04-03|279|1214|94|1923|2|4|3|2|1923|94|1214|Tuesday|1923Q2|N|N|N|2423511|2423600|2423148|2423423|N|N|N|N|N| +2423514|AAAAAAAAKNKPECAA|1923-04-04|279|1214|94|1923|3|4|4|2|1923|94|1214|Wednesday|1923Q2|N|N|N|2423511|2423600|2423149|2423424|N|N|N|N|N| +2423515|AAAAAAAALNKPECAA|1923-04-05|279|1214|94|1923|4|4|5|2|1923|94|1214|Thursday|1923Q2|N|N|N|2423511|2423600|2423150|2423425|N|N|N|N|N| +2423516|AAAAAAAAMNKPECAA|1923-04-06|279|1214|94|1923|5|4|6|2|1923|94|1214|Friday|1923Q2|N|Y|N|2423511|2423600|2423151|2423426|N|N|N|N|N| +2423517|AAAAAAAANNKPECAA|1923-04-07|279|1214|94|1923|6|4|7|2|1923|94|1214|Saturday|1923Q2|N|Y|N|2423511|2423600|2423152|2423427|N|N|N|N|N| +2423518|AAAAAAAAONKPECAA|1923-04-08|279|1214|94|1923|0|4|8|2|1923|94|1214|Sunday|1923Q2|N|N|N|2423511|2423600|2423153|2423428|N|N|N|N|N| +2423519|AAAAAAAAPNKPECAA|1923-04-09|279|1214|94|1923|1|4|9|2|1923|94|1214|Monday|1923Q2|N|N|N|2423511|2423600|2423154|2423429|N|N|N|N|N| +2423520|AAAAAAAAAOKPECAA|1923-04-10|279|1215|94|1923|2|4|10|2|1923|94|1215|Tuesday|1923Q2|N|N|N|2423511|2423600|2423155|2423430|N|N|N|N|N| +2423521|AAAAAAAABOKPECAA|1923-04-11|279|1215|94|1923|3|4|11|2|1923|94|1215|Wednesday|1923Q2|N|N|N|2423511|2423600|2423156|2423431|N|N|N|N|N| +2423522|AAAAAAAACOKPECAA|1923-04-12|279|1215|94|1923|4|4|12|2|1923|94|1215|Thursday|1923Q2|N|N|N|2423511|2423600|2423157|2423432|N|N|N|N|N| +2423523|AAAAAAAADOKPECAA|1923-04-13|279|1215|94|1923|5|4|13|2|1923|94|1215|Friday|1923Q2|N|Y|N|2423511|2423600|2423158|2423433|N|N|N|N|N| +2423524|AAAAAAAAEOKPECAA|1923-04-14|279|1215|94|1923|6|4|14|2|1923|94|1215|Saturday|1923Q2|N|Y|N|2423511|2423600|2423159|2423434|N|N|N|N|N| +2423525|AAAAAAAAFOKPECAA|1923-04-15|279|1215|94|1923|0|4|15|2|1923|94|1215|Sunday|1923Q2|N|N|N|2423511|2423600|2423160|2423435|N|N|N|N|N| +2423526|AAAAAAAAGOKPECAA|1923-04-16|279|1215|94|1923|1|4|16|2|1923|94|1215|Monday|1923Q2|N|N|N|2423511|2423600|2423161|2423436|N|N|N|N|N| +2423527|AAAAAAAAHOKPECAA|1923-04-17|279|1216|94|1923|2|4|17|2|1923|94|1216|Tuesday|1923Q2|N|N|N|2423511|2423600|2423162|2423437|N|N|N|N|N| +2423528|AAAAAAAAIOKPECAA|1923-04-18|279|1216|94|1923|3|4|18|2|1923|94|1216|Wednesday|1923Q2|N|N|N|2423511|2423600|2423163|2423438|N|N|N|N|N| +2423529|AAAAAAAAJOKPECAA|1923-04-19|279|1216|94|1923|4|4|19|2|1923|94|1216|Thursday|1923Q2|N|N|N|2423511|2423600|2423164|2423439|N|N|N|N|N| +2423530|AAAAAAAAKOKPECAA|1923-04-20|279|1216|94|1923|5|4|20|2|1923|94|1216|Friday|1923Q2|N|Y|N|2423511|2423600|2423165|2423440|N|N|N|N|N| +2423531|AAAAAAAALOKPECAA|1923-04-21|279|1216|94|1923|6|4|21|2|1923|94|1216|Saturday|1923Q2|N|Y|N|2423511|2423600|2423166|2423441|N|N|N|N|N| +2423532|AAAAAAAAMOKPECAA|1923-04-22|279|1216|94|1923|0|4|22|2|1923|94|1216|Sunday|1923Q2|N|N|N|2423511|2423600|2423167|2423442|N|N|N|N|N| +2423533|AAAAAAAANOKPECAA|1923-04-23|279|1216|94|1923|1|4|23|2|1923|94|1216|Monday|1923Q2|N|N|N|2423511|2423600|2423168|2423443|N|N|N|N|N| +2423534|AAAAAAAAOOKPECAA|1923-04-24|279|1217|94|1923|2|4|24|2|1923|94|1217|Tuesday|1923Q2|N|N|N|2423511|2423600|2423169|2423444|N|N|N|N|N| +2423535|AAAAAAAAPOKPECAA|1923-04-25|279|1217|94|1923|3|4|25|2|1923|94|1217|Wednesday|1923Q2|N|N|N|2423511|2423600|2423170|2423445|N|N|N|N|N| +2423536|AAAAAAAAAPKPECAA|1923-04-26|279|1217|94|1923|4|4|26|2|1923|94|1217|Thursday|1923Q2|N|N|N|2423511|2423600|2423171|2423446|N|N|N|N|N| +2423537|AAAAAAAABPKPECAA|1923-04-27|279|1217|94|1923|5|4|27|2|1923|94|1217|Friday|1923Q2|N|Y|N|2423511|2423600|2423172|2423447|N|N|N|N|N| +2423538|AAAAAAAACPKPECAA|1923-04-28|279|1217|94|1923|6|4|28|2|1923|94|1217|Saturday|1923Q2|N|Y|N|2423511|2423600|2423173|2423448|N|N|N|N|N| +2423539|AAAAAAAADPKPECAA|1923-04-29|279|1217|94|1923|0|4|29|2|1923|94|1217|Sunday|1923Q2|N|N|N|2423511|2423600|2423174|2423449|N|N|N|N|N| +2423540|AAAAAAAAEPKPECAA|1923-04-30|279|1217|94|1923|1|4|30|2|1923|94|1217|Monday|1923Q2|N|N|N|2423511|2423600|2423175|2423450|N|N|N|N|N| +2423541|AAAAAAAAFPKPECAA|1923-05-01|280|1218|94|1923|2|5|1|2|1923|94|1218|Tuesday|1923Q2|N|N|N|2423541|2423660|2423176|2423451|N|N|N|N|N| +2423542|AAAAAAAAGPKPECAA|1923-05-02|280|1218|94|1923|3|5|2|2|1923|94|1218|Wednesday|1923Q2|N|N|N|2423541|2423660|2423177|2423452|N|N|N|N|N| +2423543|AAAAAAAAHPKPECAA|1923-05-03|280|1218|94|1923|4|5|3|2|1923|94|1218|Thursday|1923Q2|N|N|N|2423541|2423660|2423178|2423453|N|N|N|N|N| +2423544|AAAAAAAAIPKPECAA|1923-05-04|280|1218|94|1923|5|5|4|2|1923|94|1218|Friday|1923Q2|N|Y|N|2423541|2423660|2423179|2423454|N|N|N|N|N| +2423545|AAAAAAAAJPKPECAA|1923-05-05|280|1218|94|1923|6|5|5|2|1923|94|1218|Saturday|1923Q2|N|Y|N|2423541|2423660|2423180|2423455|N|N|N|N|N| +2423546|AAAAAAAAKPKPECAA|1923-05-06|280|1218|94|1923|0|5|6|2|1923|94|1218|Sunday|1923Q2|N|N|N|2423541|2423660|2423181|2423456|N|N|N|N|N| +2423547|AAAAAAAALPKPECAA|1923-05-07|280|1218|94|1923|1|5|7|2|1923|94|1218|Monday|1923Q2|N|N|N|2423541|2423660|2423182|2423457|N|N|N|N|N| +2423548|AAAAAAAAMPKPECAA|1923-05-08|280|1219|94|1923|2|5|8|2|1923|94|1219|Tuesday|1923Q2|N|N|N|2423541|2423660|2423183|2423458|N|N|N|N|N| +2423549|AAAAAAAANPKPECAA|1923-05-09|280|1219|94|1923|3|5|9|2|1923|94|1219|Wednesday|1923Q2|N|N|N|2423541|2423660|2423184|2423459|N|N|N|N|N| +2423550|AAAAAAAAOPKPECAA|1923-05-10|280|1219|94|1923|4|5|10|2|1923|94|1219|Thursday|1923Q2|N|N|N|2423541|2423660|2423185|2423460|N|N|N|N|N| +2423551|AAAAAAAAPPKPECAA|1923-05-11|280|1219|94|1923|5|5|11|2|1923|94|1219|Friday|1923Q2|N|Y|N|2423541|2423660|2423186|2423461|N|N|N|N|N| +2423552|AAAAAAAAAALPECAA|1923-05-12|280|1219|94|1923|6|5|12|2|1923|94|1219|Saturday|1923Q2|N|Y|N|2423541|2423660|2423187|2423462|N|N|N|N|N| +2423553|AAAAAAAABALPECAA|1923-05-13|280|1219|94|1923|0|5|13|2|1923|94|1219|Sunday|1923Q2|N|N|N|2423541|2423660|2423188|2423463|N|N|N|N|N| +2423554|AAAAAAAACALPECAA|1923-05-14|280|1219|94|1923|1|5|14|2|1923|94|1219|Monday|1923Q2|N|N|N|2423541|2423660|2423189|2423464|N|N|N|N|N| +2423555|AAAAAAAADALPECAA|1923-05-15|280|1220|94|1923|2|5|15|2|1923|94|1220|Tuesday|1923Q2|N|N|N|2423541|2423660|2423190|2423465|N|N|N|N|N| +2423556|AAAAAAAAEALPECAA|1923-05-16|280|1220|94|1923|3|5|16|2|1923|94|1220|Wednesday|1923Q2|N|N|N|2423541|2423660|2423191|2423466|N|N|N|N|N| +2423557|AAAAAAAAFALPECAA|1923-05-17|280|1220|94|1923|4|5|17|2|1923|94|1220|Thursday|1923Q2|N|N|N|2423541|2423660|2423192|2423467|N|N|N|N|N| +2423558|AAAAAAAAGALPECAA|1923-05-18|280|1220|94|1923|5|5|18|2|1923|94|1220|Friday|1923Q2|N|Y|N|2423541|2423660|2423193|2423468|N|N|N|N|N| +2423559|AAAAAAAAHALPECAA|1923-05-19|280|1220|94|1923|6|5|19|2|1923|94|1220|Saturday|1923Q2|N|Y|N|2423541|2423660|2423194|2423469|N|N|N|N|N| +2423560|AAAAAAAAIALPECAA|1923-05-20|280|1220|94|1923|0|5|20|2|1923|94|1220|Sunday|1923Q2|N|N|N|2423541|2423660|2423195|2423470|N|N|N|N|N| +2423561|AAAAAAAAJALPECAA|1923-05-21|280|1220|94|1923|1|5|21|2|1923|94|1220|Monday|1923Q2|N|N|N|2423541|2423660|2423196|2423471|N|N|N|N|N| +2423562|AAAAAAAAKALPECAA|1923-05-22|280|1221|94|1923|2|5|22|2|1923|94|1221|Tuesday|1923Q2|N|N|N|2423541|2423660|2423197|2423472|N|N|N|N|N| +2423563|AAAAAAAALALPECAA|1923-05-23|280|1221|94|1923|3|5|23|2|1923|94|1221|Wednesday|1923Q2|N|N|N|2423541|2423660|2423198|2423473|N|N|N|N|N| +2423564|AAAAAAAAMALPECAA|1923-05-24|280|1221|94|1923|4|5|24|2|1923|94|1221|Thursday|1923Q2|N|N|N|2423541|2423660|2423199|2423474|N|N|N|N|N| +2423565|AAAAAAAANALPECAA|1923-05-25|280|1221|94|1923|5|5|25|2|1923|94|1221|Friday|1923Q2|N|Y|N|2423541|2423660|2423200|2423475|N|N|N|N|N| +2423566|AAAAAAAAOALPECAA|1923-05-26|280|1221|94|1923|6|5|26|2|1923|94|1221|Saturday|1923Q2|N|Y|N|2423541|2423660|2423201|2423476|N|N|N|N|N| +2423567|AAAAAAAAPALPECAA|1923-05-27|280|1221|94|1923|0|5|27|2|1923|94|1221|Sunday|1923Q2|N|N|N|2423541|2423660|2423202|2423477|N|N|N|N|N| +2423568|AAAAAAAAABLPECAA|1923-05-28|280|1221|94|1923|1|5|28|2|1923|94|1221|Monday|1923Q2|N|N|N|2423541|2423660|2423203|2423478|N|N|N|N|N| +2423569|AAAAAAAABBLPECAA|1923-05-29|280|1222|94|1923|2|5|29|2|1923|94|1222|Tuesday|1923Q2|N|N|N|2423541|2423660|2423204|2423479|N|N|N|N|N| +2423570|AAAAAAAACBLPECAA|1923-05-30|280|1222|94|1923|3|5|30|2|1923|94|1222|Wednesday|1923Q2|N|N|N|2423541|2423660|2423205|2423480|N|N|N|N|N| +2423571|AAAAAAAADBLPECAA|1923-05-31|280|1222|94|1923|4|5|31|2|1923|94|1222|Thursday|1923Q2|N|N|N|2423541|2423660|2423206|2423481|N|N|N|N|N| +2423572|AAAAAAAAEBLPECAA|1923-06-01|281|1222|95|1923|5|6|1|2|1923|95|1222|Friday|1923Q2|N|Y|N|2423572|2423722|2423207|2423482|N|N|N|N|N| +2423573|AAAAAAAAFBLPECAA|1923-06-02|281|1222|95|1923|6|6|2|2|1923|95|1222|Saturday|1923Q2|N|Y|N|2423572|2423722|2423208|2423483|N|N|N|N|N| +2423574|AAAAAAAAGBLPECAA|1923-06-03|281|1222|95|1923|0|6|3|2|1923|95|1222|Sunday|1923Q2|N|N|N|2423572|2423722|2423209|2423484|N|N|N|N|N| +2423575|AAAAAAAAHBLPECAA|1923-06-04|281|1222|95|1923|1|6|4|2|1923|95|1222|Monday|1923Q2|N|N|N|2423572|2423722|2423210|2423485|N|N|N|N|N| +2423576|AAAAAAAAIBLPECAA|1923-06-05|281|1223|95|1923|2|6|5|2|1923|95|1223|Tuesday|1923Q2|N|N|N|2423572|2423722|2423211|2423486|N|N|N|N|N| +2423577|AAAAAAAAJBLPECAA|1923-06-06|281|1223|95|1923|3|6|6|2|1923|95|1223|Wednesday|1923Q2|N|N|N|2423572|2423722|2423212|2423487|N|N|N|N|N| +2423578|AAAAAAAAKBLPECAA|1923-06-07|281|1223|95|1923|4|6|7|2|1923|95|1223|Thursday|1923Q2|N|N|N|2423572|2423722|2423213|2423488|N|N|N|N|N| +2423579|AAAAAAAALBLPECAA|1923-06-08|281|1223|95|1923|5|6|8|2|1923|95|1223|Friday|1923Q2|N|Y|N|2423572|2423722|2423214|2423489|N|N|N|N|N| +2423580|AAAAAAAAMBLPECAA|1923-06-09|281|1223|95|1923|6|6|9|2|1923|95|1223|Saturday|1923Q2|N|Y|N|2423572|2423722|2423215|2423490|N|N|N|N|N| +2423581|AAAAAAAANBLPECAA|1923-06-10|281|1223|95|1923|0|6|10|2|1923|95|1223|Sunday|1923Q2|N|N|N|2423572|2423722|2423216|2423491|N|N|N|N|N| +2423582|AAAAAAAAOBLPECAA|1923-06-11|281|1223|95|1923|1|6|11|2|1923|95|1223|Monday|1923Q2|N|N|N|2423572|2423722|2423217|2423492|N|N|N|N|N| +2423583|AAAAAAAAPBLPECAA|1923-06-12|281|1224|95|1923|2|6|12|2|1923|95|1224|Tuesday|1923Q2|N|N|N|2423572|2423722|2423218|2423493|N|N|N|N|N| +2423584|AAAAAAAAACLPECAA|1923-06-13|281|1224|95|1923|3|6|13|2|1923|95|1224|Wednesday|1923Q2|N|N|N|2423572|2423722|2423219|2423494|N|N|N|N|N| +2423585|AAAAAAAABCLPECAA|1923-06-14|281|1224|95|1923|4|6|14|2|1923|95|1224|Thursday|1923Q2|N|N|N|2423572|2423722|2423220|2423495|N|N|N|N|N| +2423586|AAAAAAAACCLPECAA|1923-06-15|281|1224|95|1923|5|6|15|2|1923|95|1224|Friday|1923Q2|N|Y|N|2423572|2423722|2423221|2423496|N|N|N|N|N| +2423587|AAAAAAAADCLPECAA|1923-06-16|281|1224|95|1923|6|6|16|2|1923|95|1224|Saturday|1923Q2|N|Y|N|2423572|2423722|2423222|2423497|N|N|N|N|N| +2423588|AAAAAAAAECLPECAA|1923-06-17|281|1224|95|1923|0|6|17|2|1923|95|1224|Sunday|1923Q2|N|N|N|2423572|2423722|2423223|2423498|N|N|N|N|N| +2423589|AAAAAAAAFCLPECAA|1923-06-18|281|1224|95|1923|1|6|18|2|1923|95|1224|Monday|1923Q2|N|N|N|2423572|2423722|2423224|2423499|N|N|N|N|N| +2423590|AAAAAAAAGCLPECAA|1923-06-19|281|1225|95|1923|2|6|19|2|1923|95|1225|Tuesday|1923Q2|N|N|N|2423572|2423722|2423225|2423500|N|N|N|N|N| +2423591|AAAAAAAAHCLPECAA|1923-06-20|281|1225|95|1923|3|6|20|2|1923|95|1225|Wednesday|1923Q2|N|N|N|2423572|2423722|2423226|2423501|N|N|N|N|N| +2423592|AAAAAAAAICLPECAA|1923-06-21|281|1225|95|1923|4|6|21|2|1923|95|1225|Thursday|1923Q2|N|N|N|2423572|2423722|2423227|2423502|N|N|N|N|N| +2423593|AAAAAAAAJCLPECAA|1923-06-22|281|1225|95|1923|5|6|22|2|1923|95|1225|Friday|1923Q2|N|Y|N|2423572|2423722|2423228|2423503|N|N|N|N|N| +2423594|AAAAAAAAKCLPECAA|1923-06-23|281|1225|95|1923|6|6|23|2|1923|95|1225|Saturday|1923Q2|N|Y|N|2423572|2423722|2423229|2423504|N|N|N|N|N| +2423595|AAAAAAAALCLPECAA|1923-06-24|281|1225|95|1923|0|6|24|2|1923|95|1225|Sunday|1923Q2|N|N|N|2423572|2423722|2423230|2423505|N|N|N|N|N| +2423596|AAAAAAAAMCLPECAA|1923-06-25|281|1225|95|1923|1|6|25|2|1923|95|1225|Monday|1923Q2|N|N|N|2423572|2423722|2423231|2423506|N|N|N|N|N| +2423597|AAAAAAAANCLPECAA|1923-06-26|281|1226|95|1923|2|6|26|2|1923|95|1226|Tuesday|1923Q2|N|N|N|2423572|2423722|2423232|2423507|N|N|N|N|N| +2423598|AAAAAAAAOCLPECAA|1923-06-27|281|1226|95|1923|3|6|27|2|1923|95|1226|Wednesday|1923Q2|N|N|N|2423572|2423722|2423233|2423508|N|N|N|N|N| +2423599|AAAAAAAAPCLPECAA|1923-06-28|281|1226|95|1923|4|6|28|2|1923|95|1226|Thursday|1923Q2|N|N|N|2423572|2423722|2423234|2423509|N|N|N|N|N| +2423600|AAAAAAAAADLPECAA|1923-06-29|281|1226|95|1923|5|6|29|2|1923|95|1226|Friday|1923Q2|N|Y|N|2423572|2423722|2423235|2423510|N|N|N|N|N| +2423601|AAAAAAAABDLPECAA|1923-06-30|281|1226|95|1923|6|6|30|2|1923|95|1226|Saturday|1923Q2|N|Y|N|2423572|2423722|2423236|2423511|N|N|N|N|N| +2423602|AAAAAAAACDLPECAA|1923-07-01|282|1226|95|1923|0|7|1|2|1923|95|1226|Sunday|1923Q2|N|N|N|2423602|2423782|2423237|2423511|N|N|N|N|N| +2423603|AAAAAAAADDLPECAA|1923-07-02|282|1226|95|1923|1|7|2|3|1923|95|1226|Monday|1923Q3|N|N|N|2423602|2423782|2423238|2423512|N|N|N|N|N| +2423604|AAAAAAAAEDLPECAA|1923-07-03|282|1227|95|1923|2|7|3|3|1923|95|1227|Tuesday|1923Q3|N|N|N|2423602|2423782|2423239|2423513|N|N|N|N|N| +2423605|AAAAAAAAFDLPECAA|1923-07-04|282|1227|95|1923|3|7|4|3|1923|95|1227|Wednesday|1923Q3|N|N|N|2423602|2423782|2423240|2423514|N|N|N|N|N| +2423606|AAAAAAAAGDLPECAA|1923-07-05|282|1227|95|1923|4|7|5|3|1923|95|1227|Thursday|1923Q3|Y|N|N|2423602|2423782|2423241|2423515|N|N|N|N|N| +2423607|AAAAAAAAHDLPECAA|1923-07-06|282|1227|95|1923|5|7|6|3|1923|95|1227|Friday|1923Q3|N|Y|Y|2423602|2423782|2423242|2423516|N|N|N|N|N| +2423608|AAAAAAAAIDLPECAA|1923-07-07|282|1227|95|1923|6|7|7|3|1923|95|1227|Saturday|1923Q3|N|Y|N|2423602|2423782|2423243|2423517|N|N|N|N|N| +2423609|AAAAAAAAJDLPECAA|1923-07-08|282|1227|95|1923|0|7|8|3|1923|95|1227|Sunday|1923Q3|N|N|N|2423602|2423782|2423244|2423518|N|N|N|N|N| +2423610|AAAAAAAAKDLPECAA|1923-07-09|282|1227|95|1923|1|7|9|3|1923|95|1227|Monday|1923Q3|N|N|N|2423602|2423782|2423245|2423519|N|N|N|N|N| +2423611|AAAAAAAALDLPECAA|1923-07-10|282|1228|95|1923|2|7|10|3|1923|95|1228|Tuesday|1923Q3|N|N|N|2423602|2423782|2423246|2423520|N|N|N|N|N| +2423612|AAAAAAAAMDLPECAA|1923-07-11|282|1228|95|1923|3|7|11|3|1923|95|1228|Wednesday|1923Q3|N|N|N|2423602|2423782|2423247|2423521|N|N|N|N|N| +2423613|AAAAAAAANDLPECAA|1923-07-12|282|1228|95|1923|4|7|12|3|1923|95|1228|Thursday|1923Q3|N|N|N|2423602|2423782|2423248|2423522|N|N|N|N|N| +2423614|AAAAAAAAODLPECAA|1923-07-13|282|1228|95|1923|5|7|13|3|1923|95|1228|Friday|1923Q3|N|Y|N|2423602|2423782|2423249|2423523|N|N|N|N|N| +2423615|AAAAAAAAPDLPECAA|1923-07-14|282|1228|95|1923|6|7|14|3|1923|95|1228|Saturday|1923Q3|N|Y|N|2423602|2423782|2423250|2423524|N|N|N|N|N| +2423616|AAAAAAAAAELPECAA|1923-07-15|282|1228|95|1923|0|7|15|3|1923|95|1228|Sunday|1923Q3|N|N|N|2423602|2423782|2423251|2423525|N|N|N|N|N| +2423617|AAAAAAAABELPECAA|1923-07-16|282|1228|95|1923|1|7|16|3|1923|95|1228|Monday|1923Q3|N|N|N|2423602|2423782|2423252|2423526|N|N|N|N|N| +2423618|AAAAAAAACELPECAA|1923-07-17|282|1229|95|1923|2|7|17|3|1923|95|1229|Tuesday|1923Q3|N|N|N|2423602|2423782|2423253|2423527|N|N|N|N|N| +2423619|AAAAAAAADELPECAA|1923-07-18|282|1229|95|1923|3|7|18|3|1923|95|1229|Wednesday|1923Q3|N|N|N|2423602|2423782|2423254|2423528|N|N|N|N|N| +2423620|AAAAAAAAEELPECAA|1923-07-19|282|1229|95|1923|4|7|19|3|1923|95|1229|Thursday|1923Q3|N|N|N|2423602|2423782|2423255|2423529|N|N|N|N|N| +2423621|AAAAAAAAFELPECAA|1923-07-20|282|1229|95|1923|5|7|20|3|1923|95|1229|Friday|1923Q3|N|Y|N|2423602|2423782|2423256|2423530|N|N|N|N|N| +2423622|AAAAAAAAGELPECAA|1923-07-21|282|1229|95|1923|6|7|21|3|1923|95|1229|Saturday|1923Q3|N|Y|N|2423602|2423782|2423257|2423531|N|N|N|N|N| +2423623|AAAAAAAAHELPECAA|1923-07-22|282|1229|95|1923|0|7|22|3|1923|95|1229|Sunday|1923Q3|N|N|N|2423602|2423782|2423258|2423532|N|N|N|N|N| +2423624|AAAAAAAAIELPECAA|1923-07-23|282|1229|95|1923|1|7|23|3|1923|95|1229|Monday|1923Q3|N|N|N|2423602|2423782|2423259|2423533|N|N|N|N|N| +2423625|AAAAAAAAJELPECAA|1923-07-24|282|1230|95|1923|2|7|24|3|1923|95|1230|Tuesday|1923Q3|N|N|N|2423602|2423782|2423260|2423534|N|N|N|N|N| +2423626|AAAAAAAAKELPECAA|1923-07-25|282|1230|95|1923|3|7|25|3|1923|95|1230|Wednesday|1923Q3|N|N|N|2423602|2423782|2423261|2423535|N|N|N|N|N| +2423627|AAAAAAAALELPECAA|1923-07-26|282|1230|95|1923|4|7|26|3|1923|95|1230|Thursday|1923Q3|N|N|N|2423602|2423782|2423262|2423536|N|N|N|N|N| +2423628|AAAAAAAAMELPECAA|1923-07-27|282|1230|95|1923|5|7|27|3|1923|95|1230|Friday|1923Q3|N|Y|N|2423602|2423782|2423263|2423537|N|N|N|N|N| +2423629|AAAAAAAANELPECAA|1923-07-28|282|1230|95|1923|6|7|28|3|1923|95|1230|Saturday|1923Q3|N|Y|N|2423602|2423782|2423264|2423538|N|N|N|N|N| +2423630|AAAAAAAAOELPECAA|1923-07-29|282|1230|95|1923|0|7|29|3|1923|95|1230|Sunday|1923Q3|N|N|N|2423602|2423782|2423265|2423539|N|N|N|N|N| +2423631|AAAAAAAAPELPECAA|1923-07-30|282|1230|95|1923|1|7|30|3|1923|95|1230|Monday|1923Q3|N|N|N|2423602|2423782|2423266|2423540|N|N|N|N|N| +2423632|AAAAAAAAAFLPECAA|1923-07-31|282|1231|95|1923|2|7|31|3|1923|95|1231|Tuesday|1923Q3|N|N|N|2423602|2423782|2423267|2423541|N|N|N|N|N| +2423633|AAAAAAAABFLPECAA|1923-08-01|283|1231|95|1923|3|8|1|3|1923|95|1231|Wednesday|1923Q3|N|N|N|2423633|2423844|2423268|2423542|N|N|N|N|N| +2423634|AAAAAAAACFLPECAA|1923-08-02|283|1231|95|1923|4|8|2|3|1923|95|1231|Thursday|1923Q3|N|N|N|2423633|2423844|2423269|2423543|N|N|N|N|N| +2423635|AAAAAAAADFLPECAA|1923-08-03|283|1231|95|1923|5|8|3|3|1923|95|1231|Friday|1923Q3|N|Y|N|2423633|2423844|2423270|2423544|N|N|N|N|N| +2423636|AAAAAAAAEFLPECAA|1923-08-04|283|1231|95|1923|6|8|4|3|1923|95|1231|Saturday|1923Q3|N|Y|N|2423633|2423844|2423271|2423545|N|N|N|N|N| +2423637|AAAAAAAAFFLPECAA|1923-08-05|283|1231|95|1923|0|8|5|3|1923|95|1231|Sunday|1923Q3|N|N|N|2423633|2423844|2423272|2423546|N|N|N|N|N| +2423638|AAAAAAAAGFLPECAA|1923-08-06|283|1231|95|1923|1|8|6|3|1923|95|1231|Monday|1923Q3|N|N|N|2423633|2423844|2423273|2423547|N|N|N|N|N| +2423639|AAAAAAAAHFLPECAA|1923-08-07|283|1232|95|1923|2|8|7|3|1923|95|1232|Tuesday|1923Q3|N|N|N|2423633|2423844|2423274|2423548|N|N|N|N|N| +2423640|AAAAAAAAIFLPECAA|1923-08-08|283|1232|95|1923|3|8|8|3|1923|95|1232|Wednesday|1923Q3|N|N|N|2423633|2423844|2423275|2423549|N|N|N|N|N| +2423641|AAAAAAAAJFLPECAA|1923-08-09|283|1232|95|1923|4|8|9|3|1923|95|1232|Thursday|1923Q3|N|N|N|2423633|2423844|2423276|2423550|N|N|N|N|N| +2423642|AAAAAAAAKFLPECAA|1923-08-10|283|1232|95|1923|5|8|10|3|1923|95|1232|Friday|1923Q3|N|Y|N|2423633|2423844|2423277|2423551|N|N|N|N|N| +2423643|AAAAAAAALFLPECAA|1923-08-11|283|1232|95|1923|6|8|11|3|1923|95|1232|Saturday|1923Q3|N|Y|N|2423633|2423844|2423278|2423552|N|N|N|N|N| +2423644|AAAAAAAAMFLPECAA|1923-08-12|283|1232|95|1923|0|8|12|3|1923|95|1232|Sunday|1923Q3|N|N|N|2423633|2423844|2423279|2423553|N|N|N|N|N| +2423645|AAAAAAAANFLPECAA|1923-08-13|283|1232|95|1923|1|8|13|3|1923|95|1232|Monday|1923Q3|N|N|N|2423633|2423844|2423280|2423554|N|N|N|N|N| +2423646|AAAAAAAAOFLPECAA|1923-08-14|283|1233|95|1923|2|8|14|3|1923|95|1233|Tuesday|1923Q3|N|N|N|2423633|2423844|2423281|2423555|N|N|N|N|N| +2423647|AAAAAAAAPFLPECAA|1923-08-15|283|1233|95|1923|3|8|15|3|1923|95|1233|Wednesday|1923Q3|N|N|N|2423633|2423844|2423282|2423556|N|N|N|N|N| +2423648|AAAAAAAAAGLPECAA|1923-08-16|283|1233|95|1923|4|8|16|3|1923|95|1233|Thursday|1923Q3|N|N|N|2423633|2423844|2423283|2423557|N|N|N|N|N| +2423649|AAAAAAAABGLPECAA|1923-08-17|283|1233|95|1923|5|8|17|3|1923|95|1233|Friday|1923Q3|N|Y|N|2423633|2423844|2423284|2423558|N|N|N|N|N| +2423650|AAAAAAAACGLPECAA|1923-08-18|283|1233|95|1923|6|8|18|3|1923|95|1233|Saturday|1923Q3|N|Y|N|2423633|2423844|2423285|2423559|N|N|N|N|N| +2423651|AAAAAAAADGLPECAA|1923-08-19|283|1233|95|1923|0|8|19|3|1923|95|1233|Sunday|1923Q3|N|N|N|2423633|2423844|2423286|2423560|N|N|N|N|N| +2423652|AAAAAAAAEGLPECAA|1923-08-20|283|1233|95|1923|1|8|20|3|1923|95|1233|Monday|1923Q3|N|N|N|2423633|2423844|2423287|2423561|N|N|N|N|N| +2423653|AAAAAAAAFGLPECAA|1923-08-21|283|1234|95|1923|2|8|21|3|1923|95|1234|Tuesday|1923Q3|N|N|N|2423633|2423844|2423288|2423562|N|N|N|N|N| +2423654|AAAAAAAAGGLPECAA|1923-08-22|283|1234|95|1923|3|8|22|3|1923|95|1234|Wednesday|1923Q3|N|N|N|2423633|2423844|2423289|2423563|N|N|N|N|N| +2423655|AAAAAAAAHGLPECAA|1923-08-23|283|1234|95|1923|4|8|23|3|1923|95|1234|Thursday|1923Q3|N|N|N|2423633|2423844|2423290|2423564|N|N|N|N|N| +2423656|AAAAAAAAIGLPECAA|1923-08-24|283|1234|95|1923|5|8|24|3|1923|95|1234|Friday|1923Q3|N|Y|N|2423633|2423844|2423291|2423565|N|N|N|N|N| +2423657|AAAAAAAAJGLPECAA|1923-08-25|283|1234|95|1923|6|8|25|3|1923|95|1234|Saturday|1923Q3|N|Y|N|2423633|2423844|2423292|2423566|N|N|N|N|N| +2423658|AAAAAAAAKGLPECAA|1923-08-26|283|1234|95|1923|0|8|26|3|1923|95|1234|Sunday|1923Q3|N|N|N|2423633|2423844|2423293|2423567|N|N|N|N|N| +2423659|AAAAAAAALGLPECAA|1923-08-27|283|1234|95|1923|1|8|27|3|1923|95|1234|Monday|1923Q3|N|N|N|2423633|2423844|2423294|2423568|N|N|N|N|N| +2423660|AAAAAAAAMGLPECAA|1923-08-28|283|1235|95|1923|2|8|28|3|1923|95|1235|Tuesday|1923Q3|N|N|N|2423633|2423844|2423295|2423569|N|N|N|N|N| +2423661|AAAAAAAANGLPECAA|1923-08-29|283|1235|95|1923|3|8|29|3|1923|95|1235|Wednesday|1923Q3|N|N|N|2423633|2423844|2423296|2423570|N|N|N|N|N| +2423662|AAAAAAAAOGLPECAA|1923-08-30|283|1235|95|1923|4|8|30|3|1923|95|1235|Thursday|1923Q3|N|N|N|2423633|2423844|2423297|2423571|N|N|N|N|N| +2423663|AAAAAAAAPGLPECAA|1923-08-31|283|1235|95|1923|5|8|31|3|1923|95|1235|Friday|1923Q3|N|Y|N|2423633|2423844|2423298|2423572|N|N|N|N|N| +2423664|AAAAAAAAAHLPECAA|1923-09-01|284|1235|96|1923|6|9|1|3|1923|96|1235|Saturday|1923Q3|N|Y|N|2423664|2423906|2423299|2423573|N|N|N|N|N| +2423665|AAAAAAAABHLPECAA|1923-09-02|284|1235|96|1923|0|9|2|3|1923|96|1235|Sunday|1923Q3|N|N|N|2423664|2423906|2423300|2423574|N|N|N|N|N| +2423666|AAAAAAAACHLPECAA|1923-09-03|284|1235|96|1923|1|9|3|3|1923|96|1235|Monday|1923Q3|N|N|N|2423664|2423906|2423301|2423575|N|N|N|N|N| +2423667|AAAAAAAADHLPECAA|1923-09-04|284|1236|96|1923|2|9|4|3|1923|96|1236|Tuesday|1923Q3|N|N|N|2423664|2423906|2423302|2423576|N|N|N|N|N| +2423668|AAAAAAAAEHLPECAA|1923-09-05|284|1236|96|1923|3|9|5|3|1923|96|1236|Wednesday|1923Q3|N|N|N|2423664|2423906|2423303|2423577|N|N|N|N|N| +2423669|AAAAAAAAFHLPECAA|1923-09-06|284|1236|96|1923|4|9|6|3|1923|96|1236|Thursday|1923Q3|N|N|N|2423664|2423906|2423304|2423578|N|N|N|N|N| +2423670|AAAAAAAAGHLPECAA|1923-09-07|284|1236|96|1923|5|9|7|3|1923|96|1236|Friday|1923Q3|N|Y|N|2423664|2423906|2423305|2423579|N|N|N|N|N| +2423671|AAAAAAAAHHLPECAA|1923-09-08|284|1236|96|1923|6|9|8|3|1923|96|1236|Saturday|1923Q3|N|Y|N|2423664|2423906|2423306|2423580|N|N|N|N|N| +2423672|AAAAAAAAIHLPECAA|1923-09-09|284|1236|96|1923|0|9|9|3|1923|96|1236|Sunday|1923Q3|N|N|N|2423664|2423906|2423307|2423581|N|N|N|N|N| +2423673|AAAAAAAAJHLPECAA|1923-09-10|284|1236|96|1923|1|9|10|3|1923|96|1236|Monday|1923Q3|N|N|N|2423664|2423906|2423308|2423582|N|N|N|N|N| +2423674|AAAAAAAAKHLPECAA|1923-09-11|284|1237|96|1923|2|9|11|3|1923|96|1237|Tuesday|1923Q3|N|N|N|2423664|2423906|2423309|2423583|N|N|N|N|N| +2423675|AAAAAAAALHLPECAA|1923-09-12|284|1237|96|1923|3|9|12|3|1923|96|1237|Wednesday|1923Q3|N|N|N|2423664|2423906|2423310|2423584|N|N|N|N|N| +2423676|AAAAAAAAMHLPECAA|1923-09-13|284|1237|96|1923|4|9|13|3|1923|96|1237|Thursday|1923Q3|N|N|N|2423664|2423906|2423311|2423585|N|N|N|N|N| +2423677|AAAAAAAANHLPECAA|1923-09-14|284|1237|96|1923|5|9|14|3|1923|96|1237|Friday|1923Q3|N|Y|N|2423664|2423906|2423312|2423586|N|N|N|N|N| +2423678|AAAAAAAAOHLPECAA|1923-09-15|284|1237|96|1923|6|9|15|3|1923|96|1237|Saturday|1923Q3|N|Y|N|2423664|2423906|2423313|2423587|N|N|N|N|N| +2423679|AAAAAAAAPHLPECAA|1923-09-16|284|1237|96|1923|0|9|16|3|1923|96|1237|Sunday|1923Q3|N|N|N|2423664|2423906|2423314|2423588|N|N|N|N|N| +2423680|AAAAAAAAAILPECAA|1923-09-17|284|1237|96|1923|1|9|17|3|1923|96|1237|Monday|1923Q3|N|N|N|2423664|2423906|2423315|2423589|N|N|N|N|N| +2423681|AAAAAAAABILPECAA|1923-09-18|284|1238|96|1923|2|9|18|3|1923|96|1238|Tuesday|1923Q3|N|N|N|2423664|2423906|2423316|2423590|N|N|N|N|N| +2423682|AAAAAAAACILPECAA|1923-09-19|284|1238|96|1923|3|9|19|3|1923|96|1238|Wednesday|1923Q3|N|N|N|2423664|2423906|2423317|2423591|N|N|N|N|N| +2423683|AAAAAAAADILPECAA|1923-09-20|284|1238|96|1923|4|9|20|3|1923|96|1238|Thursday|1923Q3|N|N|N|2423664|2423906|2423318|2423592|N|N|N|N|N| +2423684|AAAAAAAAEILPECAA|1923-09-21|284|1238|96|1923|5|9|21|3|1923|96|1238|Friday|1923Q3|N|Y|N|2423664|2423906|2423319|2423593|N|N|N|N|N| +2423685|AAAAAAAAFILPECAA|1923-09-22|284|1238|96|1923|6|9|22|3|1923|96|1238|Saturday|1923Q3|N|Y|N|2423664|2423906|2423320|2423594|N|N|N|N|N| +2423686|AAAAAAAAGILPECAA|1923-09-23|284|1238|96|1923|0|9|23|3|1923|96|1238|Sunday|1923Q3|N|N|N|2423664|2423906|2423321|2423595|N|N|N|N|N| +2423687|AAAAAAAAHILPECAA|1923-09-24|284|1238|96|1923|1|9|24|3|1923|96|1238|Monday|1923Q3|N|N|N|2423664|2423906|2423322|2423596|N|N|N|N|N| +2423688|AAAAAAAAIILPECAA|1923-09-25|284|1239|96|1923|2|9|25|3|1923|96|1239|Tuesday|1923Q3|N|N|N|2423664|2423906|2423323|2423597|N|N|N|N|N| +2423689|AAAAAAAAJILPECAA|1923-09-26|284|1239|96|1923|3|9|26|3|1923|96|1239|Wednesday|1923Q3|N|N|N|2423664|2423906|2423324|2423598|N|N|N|N|N| +2423690|AAAAAAAAKILPECAA|1923-09-27|284|1239|96|1923|4|9|27|3|1923|96|1239|Thursday|1923Q3|N|N|N|2423664|2423906|2423325|2423599|N|N|N|N|N| +2423691|AAAAAAAALILPECAA|1923-09-28|284|1239|96|1923|5|9|28|3|1923|96|1239|Friday|1923Q3|N|Y|N|2423664|2423906|2423326|2423600|N|N|N|N|N| +2423692|AAAAAAAAMILPECAA|1923-09-29|284|1239|96|1923|6|9|29|3|1923|96|1239|Saturday|1923Q3|N|Y|N|2423664|2423906|2423327|2423601|N|N|N|N|N| +2423693|AAAAAAAANILPECAA|1923-09-30|284|1239|96|1923|0|9|30|3|1923|96|1239|Sunday|1923Q3|N|N|N|2423664|2423906|2423328|2423602|N|N|N|N|N| +2423694|AAAAAAAAOILPECAA|1923-10-01|285|1239|96|1923|1|10|1|3|1923|96|1239|Monday|1923Q3|N|N|N|2423694|2423966|2423329|2423602|N|N|N|N|N| +2423695|AAAAAAAAPILPECAA|1923-10-02|285|1240|96|1923|2|10|2|4|1923|96|1240|Tuesday|1923Q4|N|N|N|2423694|2423966|2423330|2423603|N|N|N|N|N| +2423696|AAAAAAAAAJLPECAA|1923-10-03|285|1240|96|1923|3|10|3|4|1923|96|1240|Wednesday|1923Q4|N|N|N|2423694|2423966|2423331|2423604|N|N|N|N|N| +2423697|AAAAAAAABJLPECAA|1923-10-04|285|1240|96|1923|4|10|4|4|1923|96|1240|Thursday|1923Q4|N|N|N|2423694|2423966|2423332|2423605|N|N|N|N|N| +2423698|AAAAAAAACJLPECAA|1923-10-05|285|1240|96|1923|5|10|5|4|1923|96|1240|Friday|1923Q4|N|Y|N|2423694|2423966|2423333|2423606|N|N|N|N|N| +2423699|AAAAAAAADJLPECAA|1923-10-06|285|1240|96|1923|6|10|6|4|1923|96|1240|Saturday|1923Q4|N|Y|N|2423694|2423966|2423334|2423607|N|N|N|N|N| +2423700|AAAAAAAAEJLPECAA|1923-10-07|285|1240|96|1923|0|10|7|4|1923|96|1240|Sunday|1923Q4|N|N|N|2423694|2423966|2423335|2423608|N|N|N|N|N| +2423701|AAAAAAAAFJLPECAA|1923-10-08|285|1240|96|1923|1|10|8|4|1923|96|1240|Monday|1923Q4|N|N|N|2423694|2423966|2423336|2423609|N|N|N|N|N| +2423702|AAAAAAAAGJLPECAA|1923-10-09|285|1241|96|1923|2|10|9|4|1923|96|1241|Tuesday|1923Q4|N|N|N|2423694|2423966|2423337|2423610|N|N|N|N|N| +2423703|AAAAAAAAHJLPECAA|1923-10-10|285|1241|96|1923|3|10|10|4|1923|96|1241|Wednesday|1923Q4|N|N|N|2423694|2423966|2423338|2423611|N|N|N|N|N| +2423704|AAAAAAAAIJLPECAA|1923-10-11|285|1241|96|1923|4|10|11|4|1923|96|1241|Thursday|1923Q4|N|N|N|2423694|2423966|2423339|2423612|N|N|N|N|N| +2423705|AAAAAAAAJJLPECAA|1923-10-12|285|1241|96|1923|5|10|12|4|1923|96|1241|Friday|1923Q4|N|Y|N|2423694|2423966|2423340|2423613|N|N|N|N|N| +2423706|AAAAAAAAKJLPECAA|1923-10-13|285|1241|96|1923|6|10|13|4|1923|96|1241|Saturday|1923Q4|N|Y|N|2423694|2423966|2423341|2423614|N|N|N|N|N| +2423707|AAAAAAAALJLPECAA|1923-10-14|285|1241|96|1923|0|10|14|4|1923|96|1241|Sunday|1923Q4|N|N|N|2423694|2423966|2423342|2423615|N|N|N|N|N| +2423708|AAAAAAAAMJLPECAA|1923-10-15|285|1241|96|1923|1|10|15|4|1923|96|1241|Monday|1923Q4|N|N|N|2423694|2423966|2423343|2423616|N|N|N|N|N| +2423709|AAAAAAAANJLPECAA|1923-10-16|285|1242|96|1923|2|10|16|4|1923|96|1242|Tuesday|1923Q4|N|N|N|2423694|2423966|2423344|2423617|N|N|N|N|N| +2423710|AAAAAAAAOJLPECAA|1923-10-17|285|1242|96|1923|3|10|17|4|1923|96|1242|Wednesday|1923Q4|N|N|N|2423694|2423966|2423345|2423618|N|N|N|N|N| +2423711|AAAAAAAAPJLPECAA|1923-10-18|285|1242|96|1923|4|10|18|4|1923|96|1242|Thursday|1923Q4|N|N|N|2423694|2423966|2423346|2423619|N|N|N|N|N| +2423712|AAAAAAAAAKLPECAA|1923-10-19|285|1242|96|1923|5|10|19|4|1923|96|1242|Friday|1923Q4|N|Y|N|2423694|2423966|2423347|2423620|N|N|N|N|N| +2423713|AAAAAAAABKLPECAA|1923-10-20|285|1242|96|1923|6|10|20|4|1923|96|1242|Saturday|1923Q4|N|Y|N|2423694|2423966|2423348|2423621|N|N|N|N|N| +2423714|AAAAAAAACKLPECAA|1923-10-21|285|1242|96|1923|0|10|21|4|1923|96|1242|Sunday|1923Q4|N|N|N|2423694|2423966|2423349|2423622|N|N|N|N|N| +2423715|AAAAAAAADKLPECAA|1923-10-22|285|1242|96|1923|1|10|22|4|1923|96|1242|Monday|1923Q4|N|N|N|2423694|2423966|2423350|2423623|N|N|N|N|N| +2423716|AAAAAAAAEKLPECAA|1923-10-23|285|1243|96|1923|2|10|23|4|1923|96|1243|Tuesday|1923Q4|N|N|N|2423694|2423966|2423351|2423624|N|N|N|N|N| +2423717|AAAAAAAAFKLPECAA|1923-10-24|285|1243|96|1923|3|10|24|4|1923|96|1243|Wednesday|1923Q4|N|N|N|2423694|2423966|2423352|2423625|N|N|N|N|N| +2423718|AAAAAAAAGKLPECAA|1923-10-25|285|1243|96|1923|4|10|25|4|1923|96|1243|Thursday|1923Q4|N|N|N|2423694|2423966|2423353|2423626|N|N|N|N|N| +2423719|AAAAAAAAHKLPECAA|1923-10-26|285|1243|96|1923|5|10|26|4|1923|96|1243|Friday|1923Q4|N|Y|N|2423694|2423966|2423354|2423627|N|N|N|N|N| +2423720|AAAAAAAAIKLPECAA|1923-10-27|285|1243|96|1923|6|10|27|4|1923|96|1243|Saturday|1923Q4|N|Y|N|2423694|2423966|2423355|2423628|N|N|N|N|N| +2423721|AAAAAAAAJKLPECAA|1923-10-28|285|1243|96|1923|0|10|28|4|1923|96|1243|Sunday|1923Q4|N|N|N|2423694|2423966|2423356|2423629|N|N|N|N|N| +2423722|AAAAAAAAKKLPECAA|1923-10-29|285|1243|96|1923|1|10|29|4|1923|96|1243|Monday|1923Q4|N|N|N|2423694|2423966|2423357|2423630|N|N|N|N|N| +2423723|AAAAAAAALKLPECAA|1923-10-30|285|1244|96|1923|2|10|30|4|1923|96|1244|Tuesday|1923Q4|N|N|N|2423694|2423966|2423358|2423631|N|N|N|N|N| +2423724|AAAAAAAAMKLPECAA|1923-10-31|285|1244|96|1923|3|10|31|4|1923|96|1244|Wednesday|1923Q4|N|N|N|2423694|2423966|2423359|2423632|N|N|N|N|N| +2423725|AAAAAAAANKLPECAA|1923-11-01|286|1244|96|1923|4|11|1|4|1923|96|1244|Thursday|1923Q4|N|N|N|2423725|2424028|2423360|2423633|N|N|N|N|N| +2423726|AAAAAAAAOKLPECAA|1923-11-02|286|1244|96|1923|5|11|2|4|1923|96|1244|Friday|1923Q4|N|Y|N|2423725|2424028|2423361|2423634|N|N|N|N|N| +2423727|AAAAAAAAPKLPECAA|1923-11-03|286|1244|96|1923|6|11|3|4|1923|96|1244|Saturday|1923Q4|N|Y|N|2423725|2424028|2423362|2423635|N|N|N|N|N| +2423728|AAAAAAAAALLPECAA|1923-11-04|286|1244|96|1923|0|11|4|4|1923|96|1244|Sunday|1923Q4|N|N|N|2423725|2424028|2423363|2423636|N|N|N|N|N| +2423729|AAAAAAAABLLPECAA|1923-11-05|286|1244|96|1923|1|11|5|4|1923|96|1244|Monday|1923Q4|N|N|N|2423725|2424028|2423364|2423637|N|N|N|N|N| +2423730|AAAAAAAACLLPECAA|1923-11-06|286|1245|96|1923|2|11|6|4|1923|96|1245|Tuesday|1923Q4|N|N|N|2423725|2424028|2423365|2423638|N|N|N|N|N| +2423731|AAAAAAAADLLPECAA|1923-11-07|286|1245|96|1923|3|11|7|4|1923|96|1245|Wednesday|1923Q4|N|N|N|2423725|2424028|2423366|2423639|N|N|N|N|N| +2423732|AAAAAAAAELLPECAA|1923-11-08|286|1245|96|1923|4|11|8|4|1923|96|1245|Thursday|1923Q4|N|N|N|2423725|2424028|2423367|2423640|N|N|N|N|N| +2423733|AAAAAAAAFLLPECAA|1923-11-09|286|1245|96|1923|5|11|9|4|1923|96|1245|Friday|1923Q4|N|Y|N|2423725|2424028|2423368|2423641|N|N|N|N|N| +2423734|AAAAAAAAGLLPECAA|1923-11-10|286|1245|96|1923|6|11|10|4|1923|96|1245|Saturday|1923Q4|N|Y|N|2423725|2424028|2423369|2423642|N|N|N|N|N| +2423735|AAAAAAAAHLLPECAA|1923-11-11|286|1245|96|1923|0|11|11|4|1923|96|1245|Sunday|1923Q4|N|N|N|2423725|2424028|2423370|2423643|N|N|N|N|N| +2423736|AAAAAAAAILLPECAA|1923-11-12|286|1245|96|1923|1|11|12|4|1923|96|1245|Monday|1923Q4|N|N|N|2423725|2424028|2423371|2423644|N|N|N|N|N| +2423737|AAAAAAAAJLLPECAA|1923-11-13|286|1246|96|1923|2|11|13|4|1923|96|1246|Tuesday|1923Q4|N|N|N|2423725|2424028|2423372|2423645|N|N|N|N|N| +2423738|AAAAAAAAKLLPECAA|1923-11-14|286|1246|96|1923|3|11|14|4|1923|96|1246|Wednesday|1923Q4|N|N|N|2423725|2424028|2423373|2423646|N|N|N|N|N| +2423739|AAAAAAAALLLPECAA|1923-11-15|286|1246|96|1923|4|11|15|4|1923|96|1246|Thursday|1923Q4|N|N|N|2423725|2424028|2423374|2423647|N|N|N|N|N| +2423740|AAAAAAAAMLLPECAA|1923-11-16|286|1246|96|1923|5|11|16|4|1923|96|1246|Friday|1923Q4|N|Y|N|2423725|2424028|2423375|2423648|N|N|N|N|N| +2423741|AAAAAAAANLLPECAA|1923-11-17|286|1246|96|1923|6|11|17|4|1923|96|1246|Saturday|1923Q4|N|Y|N|2423725|2424028|2423376|2423649|N|N|N|N|N| +2423742|AAAAAAAAOLLPECAA|1923-11-18|286|1246|96|1923|0|11|18|4|1923|96|1246|Sunday|1923Q4|N|N|N|2423725|2424028|2423377|2423650|N|N|N|N|N| +2423743|AAAAAAAAPLLPECAA|1923-11-19|286|1246|96|1923|1|11|19|4|1923|96|1246|Monday|1923Q4|N|N|N|2423725|2424028|2423378|2423651|N|N|N|N|N| +2423744|AAAAAAAAAMLPECAA|1923-11-20|286|1247|96|1923|2|11|20|4|1923|96|1247|Tuesday|1923Q4|N|N|N|2423725|2424028|2423379|2423652|N|N|N|N|N| +2423745|AAAAAAAABMLPECAA|1923-11-21|286|1247|96|1923|3|11|21|4|1923|96|1247|Wednesday|1923Q4|N|N|N|2423725|2424028|2423380|2423653|N|N|N|N|N| +2423746|AAAAAAAACMLPECAA|1923-11-22|286|1247|96|1923|4|11|22|4|1923|96|1247|Thursday|1923Q4|N|N|N|2423725|2424028|2423381|2423654|N|N|N|N|N| +2423747|AAAAAAAADMLPECAA|1923-11-23|286|1247|96|1923|5|11|23|4|1923|96|1247|Friday|1923Q4|N|Y|N|2423725|2424028|2423382|2423655|N|N|N|N|N| +2423748|AAAAAAAAEMLPECAA|1923-11-24|286|1247|96|1923|6|11|24|4|1923|96|1247|Saturday|1923Q4|N|Y|N|2423725|2424028|2423383|2423656|N|N|N|N|N| +2423749|AAAAAAAAFMLPECAA|1923-11-25|286|1247|96|1923|0|11|25|4|1923|96|1247|Sunday|1923Q4|N|N|N|2423725|2424028|2423384|2423657|N|N|N|N|N| +2423750|AAAAAAAAGMLPECAA|1923-11-26|286|1247|96|1923|1|11|26|4|1923|96|1247|Monday|1923Q4|N|N|N|2423725|2424028|2423385|2423658|N|N|N|N|N| +2423751|AAAAAAAAHMLPECAA|1923-11-27|286|1248|96|1923|2|11|27|4|1923|96|1248|Tuesday|1923Q4|N|N|N|2423725|2424028|2423386|2423659|N|N|N|N|N| +2423752|AAAAAAAAIMLPECAA|1923-11-28|286|1248|96|1923|3|11|28|4|1923|96|1248|Wednesday|1923Q4|N|N|N|2423725|2424028|2423387|2423660|N|N|N|N|N| +2423753|AAAAAAAAJMLPECAA|1923-11-29|286|1248|96|1923|4|11|29|4|1923|96|1248|Thursday|1923Q4|N|N|N|2423725|2424028|2423388|2423661|N|N|N|N|N| +2423754|AAAAAAAAKMLPECAA|1923-11-30|286|1248|96|1923|5|11|30|4|1923|96|1248|Friday|1923Q4|N|Y|N|2423725|2424028|2423389|2423662|N|N|N|N|N| +2423755|AAAAAAAALMLPECAA|1923-12-01|287|1248|97|1923|6|12|1|4|1923|97|1248|Saturday|1923Q4|N|Y|N|2423755|2424088|2423390|2423663|N|N|N|N|N| +2423756|AAAAAAAAMMLPECAA|1923-12-02|287|1248|97|1923|0|12|2|4|1923|97|1248|Sunday|1923Q4|N|N|N|2423755|2424088|2423391|2423664|N|N|N|N|N| +2423757|AAAAAAAANMLPECAA|1923-12-03|287|1248|97|1923|1|12|3|4|1923|97|1248|Monday|1923Q4|N|N|N|2423755|2424088|2423392|2423665|N|N|N|N|N| +2423758|AAAAAAAAOMLPECAA|1923-12-04|287|1249|97|1923|2|12|4|4|1923|97|1249|Tuesday|1923Q4|N|N|N|2423755|2424088|2423393|2423666|N|N|N|N|N| +2423759|AAAAAAAAPMLPECAA|1923-12-05|287|1249|97|1923|3|12|5|4|1923|97|1249|Wednesday|1923Q4|N|N|N|2423755|2424088|2423394|2423667|N|N|N|N|N| +2423760|AAAAAAAAANLPECAA|1923-12-06|287|1249|97|1923|4|12|6|4|1923|97|1249|Thursday|1923Q4|N|N|N|2423755|2424088|2423395|2423668|N|N|N|N|N| +2423761|AAAAAAAABNLPECAA|1923-12-07|287|1249|97|1923|5|12|7|4|1923|97|1249|Friday|1923Q4|N|Y|N|2423755|2424088|2423396|2423669|N|N|N|N|N| +2423762|AAAAAAAACNLPECAA|1923-12-08|287|1249|97|1923|6|12|8|4|1923|97|1249|Saturday|1923Q4|N|Y|N|2423755|2424088|2423397|2423670|N|N|N|N|N| +2423763|AAAAAAAADNLPECAA|1923-12-09|287|1249|97|1923|0|12|9|4|1923|97|1249|Sunday|1923Q4|N|N|N|2423755|2424088|2423398|2423671|N|N|N|N|N| +2423764|AAAAAAAAENLPECAA|1923-12-10|287|1249|97|1923|1|12|10|4|1923|97|1249|Monday|1923Q4|N|N|N|2423755|2424088|2423399|2423672|N|N|N|N|N| +2423765|AAAAAAAAFNLPECAA|1923-12-11|287|1250|97|1923|2|12|11|4|1923|97|1250|Tuesday|1923Q4|N|N|N|2423755|2424088|2423400|2423673|N|N|N|N|N| +2423766|AAAAAAAAGNLPECAA|1923-12-12|287|1250|97|1923|3|12|12|4|1923|97|1250|Wednesday|1923Q4|N|N|N|2423755|2424088|2423401|2423674|N|N|N|N|N| +2423767|AAAAAAAAHNLPECAA|1923-12-13|287|1250|97|1923|4|12|13|4|1923|97|1250|Thursday|1923Q4|N|N|N|2423755|2424088|2423402|2423675|N|N|N|N|N| +2423768|AAAAAAAAINLPECAA|1923-12-14|287|1250|97|1923|5|12|14|4|1923|97|1250|Friday|1923Q4|N|Y|N|2423755|2424088|2423403|2423676|N|N|N|N|N| +2423769|AAAAAAAAJNLPECAA|1923-12-15|287|1250|97|1923|6|12|15|4|1923|97|1250|Saturday|1923Q4|N|Y|N|2423755|2424088|2423404|2423677|N|N|N|N|N| +2423770|AAAAAAAAKNLPECAA|1923-12-16|287|1250|97|1923|0|12|16|4|1923|97|1250|Sunday|1923Q4|N|N|N|2423755|2424088|2423405|2423678|N|N|N|N|N| +2423771|AAAAAAAALNLPECAA|1923-12-17|287|1250|97|1923|1|12|17|4|1923|97|1250|Monday|1923Q4|N|N|N|2423755|2424088|2423406|2423679|N|N|N|N|N| +2423772|AAAAAAAAMNLPECAA|1923-12-18|287|1251|97|1923|2|12|18|4|1923|97|1251|Tuesday|1923Q4|N|N|N|2423755|2424088|2423407|2423680|N|N|N|N|N| +2423773|AAAAAAAANNLPECAA|1923-12-19|287|1251|97|1923|3|12|19|4|1923|97|1251|Wednesday|1923Q4|N|N|N|2423755|2424088|2423408|2423681|N|N|N|N|N| +2423774|AAAAAAAAONLPECAA|1923-12-20|287|1251|97|1923|4|12|20|4|1923|97|1251|Thursday|1923Q4|N|N|N|2423755|2424088|2423409|2423682|N|N|N|N|N| +2423775|AAAAAAAAPNLPECAA|1923-12-21|287|1251|97|1923|5|12|21|4|1923|97|1251|Friday|1923Q4|N|Y|N|2423755|2424088|2423410|2423683|N|N|N|N|N| +2423776|AAAAAAAAAOLPECAA|1923-12-22|287|1251|97|1923|6|12|22|4|1923|97|1251|Saturday|1923Q4|N|Y|N|2423755|2424088|2423411|2423684|N|N|N|N|N| +2423777|AAAAAAAABOLPECAA|1923-12-23|287|1251|97|1923|0|12|23|4|1923|97|1251|Sunday|1923Q4|N|N|N|2423755|2424088|2423412|2423685|N|N|N|N|N| +2423778|AAAAAAAACOLPECAA|1923-12-24|287|1251|97|1923|1|12|24|4|1923|97|1251|Monday|1923Q4|N|N|N|2423755|2424088|2423413|2423686|N|N|N|N|N| +2423779|AAAAAAAADOLPECAA|1923-12-25|287|1252|97|1923|2|12|25|4|1923|97|1252|Tuesday|1923Q4|N|N|N|2423755|2424088|2423414|2423687|N|N|N|N|N| +2423780|AAAAAAAAEOLPECAA|1923-12-26|287|1252|97|1923|3|12|26|4|1923|97|1252|Wednesday|1923Q4|Y|N|N|2423755|2424088|2423415|2423688|N|N|N|N|N| +2423781|AAAAAAAAFOLPECAA|1923-12-27|287|1252|97|1923|4|12|27|4|1923|97|1252|Thursday|1923Q4|N|N|Y|2423755|2424088|2423416|2423689|N|N|N|N|N| +2423782|AAAAAAAAGOLPECAA|1923-12-28|287|1252|97|1923|5|12|28|4|1923|97|1252|Friday|1923Q4|N|Y|N|2423755|2424088|2423417|2423690|N|N|N|N|N| +2423783|AAAAAAAAHOLPECAA|1923-12-29|287|1252|97|1923|6|12|29|4|1923|97|1252|Saturday|1923Q4|N|Y|N|2423755|2424088|2423418|2423691|N|N|N|N|N| +2423784|AAAAAAAAIOLPECAA|1923-12-30|287|1252|97|1923|0|12|30|4|1923|97|1252|Sunday|1923Q4|N|N|N|2423755|2424088|2423419|2423692|N|N|N|N|N| +2423785|AAAAAAAAJOLPECAA|1923-12-31|287|1252|97|1923|1|12|31|4|1923|97|1252|Monday|1923Q4|N|N|N|2423755|2424088|2423420|2423693|N|N|N|N|N| +2423786|AAAAAAAAKOLPECAA|1924-01-01|288|1253|97|1924|2|1|1|1|1924|97|1253|Tuesday|1924Q1|Y|N|N|2423786|2423785|2423421|2423694|N|N|N|N|N| +2423787|AAAAAAAALOLPECAA|1924-01-02|288|1253|97|1924|3|1|2|1|1924|97|1253|Wednesday|1924Q1|N|N|Y|2423786|2423785|2423422|2423695|N|N|N|N|N| +2423788|AAAAAAAAMOLPECAA|1924-01-03|288|1253|97|1924|4|1|3|1|1924|97|1253|Thursday|1924Q1|N|N|N|2423786|2423785|2423423|2423696|N|N|N|N|N| +2423789|AAAAAAAANOLPECAA|1924-01-04|288|1253|97|1924|5|1|4|1|1924|97|1253|Friday|1924Q1|N|Y|N|2423786|2423785|2423424|2423697|N|N|N|N|N| +2423790|AAAAAAAAOOLPECAA|1924-01-05|288|1253|97|1924|6|1|5|1|1924|97|1253|Saturday|1924Q1|N|Y|N|2423786|2423785|2423425|2423698|N|N|N|N|N| +2423791|AAAAAAAAPOLPECAA|1924-01-06|288|1253|97|1924|0|1|6|1|1924|97|1253|Sunday|1924Q1|N|N|N|2423786|2423785|2423426|2423699|N|N|N|N|N| +2423792|AAAAAAAAAPLPECAA|1924-01-07|288|1253|97|1924|1|1|7|1|1924|97|1253|Monday|1924Q1|N|N|N|2423786|2423785|2423427|2423700|N|N|N|N|N| +2423793|AAAAAAAABPLPECAA|1924-01-08|288|1254|97|1924|2|1|8|1|1924|97|1254|Tuesday|1924Q1|N|N|N|2423786|2423785|2423428|2423701|N|N|N|N|N| +2423794|AAAAAAAACPLPECAA|1924-01-09|288|1254|97|1924|3|1|9|1|1924|97|1254|Wednesday|1924Q1|N|N|N|2423786|2423785|2423429|2423702|N|N|N|N|N| +2423795|AAAAAAAADPLPECAA|1924-01-10|288|1254|97|1924|4|1|10|1|1924|97|1254|Thursday|1924Q1|N|N|N|2423786|2423785|2423430|2423703|N|N|N|N|N| +2423796|AAAAAAAAEPLPECAA|1924-01-11|288|1254|97|1924|5|1|11|1|1924|97|1254|Friday|1924Q1|N|Y|N|2423786|2423785|2423431|2423704|N|N|N|N|N| +2423797|AAAAAAAAFPLPECAA|1924-01-12|288|1254|97|1924|6|1|12|1|1924|97|1254|Saturday|1924Q1|N|Y|N|2423786|2423785|2423432|2423705|N|N|N|N|N| +2423798|AAAAAAAAGPLPECAA|1924-01-13|288|1254|97|1924|0|1|13|1|1924|97|1254|Sunday|1924Q1|N|N|N|2423786|2423785|2423433|2423706|N|N|N|N|N| +2423799|AAAAAAAAHPLPECAA|1924-01-14|288|1254|97|1924|1|1|14|1|1924|97|1254|Monday|1924Q1|N|N|N|2423786|2423785|2423434|2423707|N|N|N|N|N| +2423800|AAAAAAAAIPLPECAA|1924-01-15|288|1255|97|1924|2|1|15|1|1924|97|1255|Tuesday|1924Q1|N|N|N|2423786|2423785|2423435|2423708|N|N|N|N|N| +2423801|AAAAAAAAJPLPECAA|1924-01-16|288|1255|97|1924|3|1|16|1|1924|97|1255|Wednesday|1924Q1|N|N|N|2423786|2423785|2423436|2423709|N|N|N|N|N| +2423802|AAAAAAAAKPLPECAA|1924-01-17|288|1255|97|1924|4|1|17|1|1924|97|1255|Thursday|1924Q1|N|N|N|2423786|2423785|2423437|2423710|N|N|N|N|N| +2423803|AAAAAAAALPLPECAA|1924-01-18|288|1255|97|1924|5|1|18|1|1924|97|1255|Friday|1924Q1|N|Y|N|2423786|2423785|2423438|2423711|N|N|N|N|N| +2423804|AAAAAAAAMPLPECAA|1924-01-19|288|1255|97|1924|6|1|19|1|1924|97|1255|Saturday|1924Q1|N|Y|N|2423786|2423785|2423439|2423712|N|N|N|N|N| +2423805|AAAAAAAANPLPECAA|1924-01-20|288|1255|97|1924|0|1|20|1|1924|97|1255|Sunday|1924Q1|N|N|N|2423786|2423785|2423440|2423713|N|N|N|N|N| +2423806|AAAAAAAAOPLPECAA|1924-01-21|288|1255|97|1924|1|1|21|1|1924|97|1255|Monday|1924Q1|N|N|N|2423786|2423785|2423441|2423714|N|N|N|N|N| +2423807|AAAAAAAAPPLPECAA|1924-01-22|288|1256|97|1924|2|1|22|1|1924|97|1256|Tuesday|1924Q1|N|N|N|2423786|2423785|2423442|2423715|N|N|N|N|N| +2423808|AAAAAAAAAAMPECAA|1924-01-23|288|1256|97|1924|3|1|23|1|1924|97|1256|Wednesday|1924Q1|N|N|N|2423786|2423785|2423443|2423716|N|N|N|N|N| +2423809|AAAAAAAABAMPECAA|1924-01-24|288|1256|97|1924|4|1|24|1|1924|97|1256|Thursday|1924Q1|N|N|N|2423786|2423785|2423444|2423717|N|N|N|N|N| +2423810|AAAAAAAACAMPECAA|1924-01-25|288|1256|97|1924|5|1|25|1|1924|97|1256|Friday|1924Q1|N|Y|N|2423786|2423785|2423445|2423718|N|N|N|N|N| +2423811|AAAAAAAADAMPECAA|1924-01-26|288|1256|97|1924|6|1|26|1|1924|97|1256|Saturday|1924Q1|N|Y|N|2423786|2423785|2423446|2423719|N|N|N|N|N| +2423812|AAAAAAAAEAMPECAA|1924-01-27|288|1256|97|1924|0|1|27|1|1924|97|1256|Sunday|1924Q1|N|N|N|2423786|2423785|2423447|2423720|N|N|N|N|N| +2423813|AAAAAAAAFAMPECAA|1924-01-28|288|1256|97|1924|1|1|28|1|1924|97|1256|Monday|1924Q1|N|N|N|2423786|2423785|2423448|2423721|N|N|N|N|N| +2423814|AAAAAAAAGAMPECAA|1924-01-29|288|1257|97|1924|2|1|29|1|1924|97|1257|Tuesday|1924Q1|N|N|N|2423786|2423785|2423449|2423722|N|N|N|N|N| +2423815|AAAAAAAAHAMPECAA|1924-01-30|288|1257|97|1924|3|1|30|1|1924|97|1257|Wednesday|1924Q1|N|N|N|2423786|2423785|2423450|2423723|N|N|N|N|N| +2423816|AAAAAAAAIAMPECAA|1924-01-31|288|1257|97|1924|4|1|31|1|1924|97|1257|Thursday|1924Q1|N|N|N|2423786|2423785|2423451|2423724|N|N|N|N|N| +2423817|AAAAAAAAJAMPECAA|1924-02-01|289|1257|97|1924|5|2|1|1|1924|97|1257|Friday|1924Q1|N|Y|N|2423817|2423847|2423452|2423725|N|N|N|N|N| +2423818|AAAAAAAAKAMPECAA|1924-02-02|289|1257|97|1924|6|2|2|1|1924|97|1257|Saturday|1924Q1|N|Y|N|2423817|2423847|2423453|2423726|N|N|N|N|N| +2423819|AAAAAAAALAMPECAA|1924-02-03|289|1257|97|1924|0|2|3|1|1924|97|1257|Sunday|1924Q1|N|N|N|2423817|2423847|2423454|2423727|N|N|N|N|N| +2423820|AAAAAAAAMAMPECAA|1924-02-04|289|1257|97|1924|1|2|4|1|1924|97|1257|Monday|1924Q1|N|N|N|2423817|2423847|2423455|2423728|N|N|N|N|N| +2423821|AAAAAAAANAMPECAA|1924-02-05|289|1258|97|1924|2|2|5|1|1924|97|1258|Tuesday|1924Q1|N|N|N|2423817|2423847|2423456|2423729|N|N|N|N|N| +2423822|AAAAAAAAOAMPECAA|1924-02-06|289|1258|97|1924|3|2|6|1|1924|97|1258|Wednesday|1924Q1|N|N|N|2423817|2423847|2423457|2423730|N|N|N|N|N| +2423823|AAAAAAAAPAMPECAA|1924-02-07|289|1258|97|1924|4|2|7|1|1924|97|1258|Thursday|1924Q1|N|N|N|2423817|2423847|2423458|2423731|N|N|N|N|N| +2423824|AAAAAAAAABMPECAA|1924-02-08|289|1258|97|1924|5|2|8|1|1924|97|1258|Friday|1924Q1|N|Y|N|2423817|2423847|2423459|2423732|N|N|N|N|N| +2423825|AAAAAAAABBMPECAA|1924-02-09|289|1258|97|1924|6|2|9|1|1924|97|1258|Saturday|1924Q1|N|Y|N|2423817|2423847|2423460|2423733|N|N|N|N|N| +2423826|AAAAAAAACBMPECAA|1924-02-10|289|1258|97|1924|0|2|10|1|1924|97|1258|Sunday|1924Q1|N|N|N|2423817|2423847|2423461|2423734|N|N|N|N|N| +2423827|AAAAAAAADBMPECAA|1924-02-11|289|1258|97|1924|1|2|11|1|1924|97|1258|Monday|1924Q1|N|N|N|2423817|2423847|2423462|2423735|N|N|N|N|N| +2423828|AAAAAAAAEBMPECAA|1924-02-12|289|1259|97|1924|2|2|12|1|1924|97|1259|Tuesday|1924Q1|N|N|N|2423817|2423847|2423463|2423736|N|N|N|N|N| +2423829|AAAAAAAAFBMPECAA|1924-02-13|289|1259|97|1924|3|2|13|1|1924|97|1259|Wednesday|1924Q1|N|N|N|2423817|2423847|2423464|2423737|N|N|N|N|N| +2423830|AAAAAAAAGBMPECAA|1924-02-14|289|1259|97|1924|4|2|14|1|1924|97|1259|Thursday|1924Q1|N|N|N|2423817|2423847|2423465|2423738|N|N|N|N|N| +2423831|AAAAAAAAHBMPECAA|1924-02-15|289|1259|97|1924|5|2|15|1|1924|97|1259|Friday|1924Q1|N|Y|N|2423817|2423847|2423466|2423739|N|N|N|N|N| +2423832|AAAAAAAAIBMPECAA|1924-02-16|289|1259|97|1924|6|2|16|1|1924|97|1259|Saturday|1924Q1|N|Y|N|2423817|2423847|2423467|2423740|N|N|N|N|N| +2423833|AAAAAAAAJBMPECAA|1924-02-17|289|1259|97|1924|0|2|17|1|1924|97|1259|Sunday|1924Q1|N|N|N|2423817|2423847|2423468|2423741|N|N|N|N|N| +2423834|AAAAAAAAKBMPECAA|1924-02-18|289|1259|97|1924|1|2|18|1|1924|97|1259|Monday|1924Q1|N|N|N|2423817|2423847|2423469|2423742|N|N|N|N|N| +2423835|AAAAAAAALBMPECAA|1924-02-19|289|1260|97|1924|2|2|19|1|1924|97|1260|Tuesday|1924Q1|N|N|N|2423817|2423847|2423470|2423743|N|N|N|N|N| +2423836|AAAAAAAAMBMPECAA|1924-02-20|289|1260|97|1924|3|2|20|1|1924|97|1260|Wednesday|1924Q1|N|N|N|2423817|2423847|2423471|2423744|N|N|N|N|N| +2423837|AAAAAAAANBMPECAA|1924-02-21|289|1260|97|1924|4|2|21|1|1924|97|1260|Thursday|1924Q1|N|N|N|2423817|2423847|2423472|2423745|N|N|N|N|N| +2423838|AAAAAAAAOBMPECAA|1924-02-22|289|1260|97|1924|5|2|22|1|1924|97|1260|Friday|1924Q1|N|Y|N|2423817|2423847|2423473|2423746|N|N|N|N|N| +2423839|AAAAAAAAPBMPECAA|1924-02-23|289|1260|97|1924|6|2|23|1|1924|97|1260|Saturday|1924Q1|N|Y|N|2423817|2423847|2423474|2423747|N|N|N|N|N| +2423840|AAAAAAAAACMPECAA|1924-02-24|289|1260|97|1924|0|2|24|1|1924|97|1260|Sunday|1924Q1|N|N|N|2423817|2423847|2423475|2423748|N|N|N|N|N| +2423841|AAAAAAAABCMPECAA|1924-02-25|289|1260|97|1924|1|2|25|1|1924|97|1260|Monday|1924Q1|N|N|N|2423817|2423847|2423476|2423749|N|N|N|N|N| +2423842|AAAAAAAACCMPECAA|1924-02-26|289|1261|97|1924|2|2|26|1|1924|97|1261|Tuesday|1924Q1|N|N|N|2423817|2423847|2423477|2423750|N|N|N|N|N| +2423843|AAAAAAAADCMPECAA|1924-02-27|289|1261|97|1924|3|2|27|1|1924|97|1261|Wednesday|1924Q1|N|N|N|2423817|2423847|2423478|2423751|N|N|N|N|N| +2423844|AAAAAAAAECMPECAA|1924-02-28|289|1261|97|1924|4|2|28|1|1924|97|1261|Thursday|1924Q1|N|N|N|2423817|2423847|2423479|2423752|N|N|N|N|N| +2423845|AAAAAAAAFCMPECAA|1924-02-29|289|1261|97|1924|5|2|29|1|1924|97|1261|Friday|1924Q1|N|Y|N|2423817|2423847|2423479|2423753|N|N|N|N|N| +2423846|AAAAAAAAGCMPECAA|1924-03-01|290|1261|98|1924|6|3|1|1|1924|98|1261|Saturday|1924Q1|N|Y|N|2423846|2423905|2423480|2423754|N|N|N|N|N| +2423847|AAAAAAAAHCMPECAA|1924-03-02|290|1261|98|1924|0|3|2|1|1924|98|1261|Sunday|1924Q1|N|N|N|2423846|2423905|2423481|2423755|N|N|N|N|N| +2423848|AAAAAAAAICMPECAA|1924-03-03|290|1261|98|1924|1|3|3|1|1924|98|1261|Monday|1924Q1|N|N|N|2423846|2423905|2423482|2423756|N|N|N|N|N| +2423849|AAAAAAAAJCMPECAA|1924-03-04|290|1262|98|1924|2|3|4|1|1924|98|1262|Tuesday|1924Q1|N|N|N|2423846|2423905|2423483|2423757|N|N|N|N|N| +2423850|AAAAAAAAKCMPECAA|1924-03-05|290|1262|98|1924|3|3|5|1|1924|98|1262|Wednesday|1924Q1|N|N|N|2423846|2423905|2423484|2423758|N|N|N|N|N| +2423851|AAAAAAAALCMPECAA|1924-03-06|290|1262|98|1924|4|3|6|1|1924|98|1262|Thursday|1924Q1|N|N|N|2423846|2423905|2423485|2423759|N|N|N|N|N| +2423852|AAAAAAAAMCMPECAA|1924-03-07|290|1262|98|1924|5|3|7|1|1924|98|1262|Friday|1924Q1|N|Y|N|2423846|2423905|2423486|2423760|N|N|N|N|N| +2423853|AAAAAAAANCMPECAA|1924-03-08|290|1262|98|1924|6|3|8|1|1924|98|1262|Saturday|1924Q1|N|Y|N|2423846|2423905|2423487|2423761|N|N|N|N|N| +2423854|AAAAAAAAOCMPECAA|1924-03-09|290|1262|98|1924|0|3|9|1|1924|98|1262|Sunday|1924Q1|N|N|N|2423846|2423905|2423488|2423762|N|N|N|N|N| +2423855|AAAAAAAAPCMPECAA|1924-03-10|290|1262|98|1924|1|3|10|1|1924|98|1262|Monday|1924Q1|N|N|N|2423846|2423905|2423489|2423763|N|N|N|N|N| +2423856|AAAAAAAAADMPECAA|1924-03-11|290|1263|98|1924|2|3|11|1|1924|98|1263|Tuesday|1924Q1|N|N|N|2423846|2423905|2423490|2423764|N|N|N|N|N| +2423857|AAAAAAAABDMPECAA|1924-03-12|290|1263|98|1924|3|3|12|1|1924|98|1263|Wednesday|1924Q1|N|N|N|2423846|2423905|2423491|2423765|N|N|N|N|N| +2423858|AAAAAAAACDMPECAA|1924-03-13|290|1263|98|1924|4|3|13|1|1924|98|1263|Thursday|1924Q1|N|N|N|2423846|2423905|2423492|2423766|N|N|N|N|N| +2423859|AAAAAAAADDMPECAA|1924-03-14|290|1263|98|1924|5|3|14|1|1924|98|1263|Friday|1924Q1|N|Y|N|2423846|2423905|2423493|2423767|N|N|N|N|N| +2423860|AAAAAAAAEDMPECAA|1924-03-15|290|1263|98|1924|6|3|15|1|1924|98|1263|Saturday|1924Q1|N|Y|N|2423846|2423905|2423494|2423768|N|N|N|N|N| +2423861|AAAAAAAAFDMPECAA|1924-03-16|290|1263|98|1924|0|3|16|1|1924|98|1263|Sunday|1924Q1|N|N|N|2423846|2423905|2423495|2423769|N|N|N|N|N| +2423862|AAAAAAAAGDMPECAA|1924-03-17|290|1263|98|1924|1|3|17|1|1924|98|1263|Monday|1924Q1|N|N|N|2423846|2423905|2423496|2423770|N|N|N|N|N| +2423863|AAAAAAAAHDMPECAA|1924-03-18|290|1264|98|1924|2|3|18|1|1924|98|1264|Tuesday|1924Q1|N|N|N|2423846|2423905|2423497|2423771|N|N|N|N|N| +2423864|AAAAAAAAIDMPECAA|1924-03-19|290|1264|98|1924|3|3|19|1|1924|98|1264|Wednesday|1924Q1|N|N|N|2423846|2423905|2423498|2423772|N|N|N|N|N| +2423865|AAAAAAAAJDMPECAA|1924-03-20|290|1264|98|1924|4|3|20|1|1924|98|1264|Thursday|1924Q1|N|N|N|2423846|2423905|2423499|2423773|N|N|N|N|N| +2423866|AAAAAAAAKDMPECAA|1924-03-21|290|1264|98|1924|5|3|21|1|1924|98|1264|Friday|1924Q1|N|Y|N|2423846|2423905|2423500|2423774|N|N|N|N|N| +2423867|AAAAAAAALDMPECAA|1924-03-22|290|1264|98|1924|6|3|22|1|1924|98|1264|Saturday|1924Q1|N|Y|N|2423846|2423905|2423501|2423775|N|N|N|N|N| +2423868|AAAAAAAAMDMPECAA|1924-03-23|290|1264|98|1924|0|3|23|1|1924|98|1264|Sunday|1924Q1|N|N|N|2423846|2423905|2423502|2423776|N|N|N|N|N| +2423869|AAAAAAAANDMPECAA|1924-03-24|290|1264|98|1924|1|3|24|1|1924|98|1264|Monday|1924Q1|N|N|N|2423846|2423905|2423503|2423777|N|N|N|N|N| +2423870|AAAAAAAAODMPECAA|1924-03-25|290|1265|98|1924|2|3|25|1|1924|98|1265|Tuesday|1924Q1|N|N|N|2423846|2423905|2423504|2423778|N|N|N|N|N| +2423871|AAAAAAAAPDMPECAA|1924-03-26|290|1265|98|1924|3|3|26|1|1924|98|1265|Wednesday|1924Q1|N|N|N|2423846|2423905|2423505|2423779|N|N|N|N|N| +2423872|AAAAAAAAAEMPECAA|1924-03-27|290|1265|98|1924|4|3|27|1|1924|98|1265|Thursday|1924Q1|N|N|N|2423846|2423905|2423506|2423780|N|N|N|N|N| +2423873|AAAAAAAABEMPECAA|1924-03-28|290|1265|98|1924|5|3|28|1|1924|98|1265|Friday|1924Q1|N|Y|N|2423846|2423905|2423507|2423781|N|N|N|N|N| +2423874|AAAAAAAACEMPECAA|1924-03-29|290|1265|98|1924|6|3|29|1|1924|98|1265|Saturday|1924Q1|N|Y|N|2423846|2423905|2423508|2423782|N|N|N|N|N| +2423875|AAAAAAAADEMPECAA|1924-03-30|290|1265|98|1924|0|3|30|1|1924|98|1265|Sunday|1924Q1|N|N|N|2423846|2423905|2423509|2423783|N|N|N|N|N| +2423876|AAAAAAAAEEMPECAA|1924-03-31|290|1265|98|1924|1|3|31|1|1924|98|1265|Monday|1924Q1|N|N|N|2423846|2423905|2423510|2423784|N|N|N|N|N| +2423877|AAAAAAAAFEMPECAA|1924-04-01|291|1266|98|1924|2|4|1|2|1924|98|1266|Tuesday|1924Q2|N|N|N|2423877|2423967|2423511|2423786|N|N|N|N|N| +2423878|AAAAAAAAGEMPECAA|1924-04-02|291|1266|98|1924|3|4|2|2|1924|98|1266|Wednesday|1924Q2|N|N|N|2423877|2423967|2423512|2423787|N|N|N|N|N| +2423879|AAAAAAAAHEMPECAA|1924-04-03|291|1266|98|1924|4|4|3|2|1924|98|1266|Thursday|1924Q2|N|N|N|2423877|2423967|2423513|2423788|N|N|N|N|N| +2423880|AAAAAAAAIEMPECAA|1924-04-04|291|1266|98|1924|5|4|4|2|1924|98|1266|Friday|1924Q2|N|Y|N|2423877|2423967|2423514|2423789|N|N|N|N|N| +2423881|AAAAAAAAJEMPECAA|1924-04-05|291|1266|98|1924|6|4|5|2|1924|98|1266|Saturday|1924Q2|N|Y|N|2423877|2423967|2423515|2423790|N|N|N|N|N| +2423882|AAAAAAAAKEMPECAA|1924-04-06|291|1266|98|1924|0|4|6|2|1924|98|1266|Sunday|1924Q2|N|N|N|2423877|2423967|2423516|2423791|N|N|N|N|N| +2423883|AAAAAAAALEMPECAA|1924-04-07|291|1266|98|1924|1|4|7|2|1924|98|1266|Monday|1924Q2|N|N|N|2423877|2423967|2423517|2423792|N|N|N|N|N| +2423884|AAAAAAAAMEMPECAA|1924-04-08|291|1267|98|1924|2|4|8|2|1924|98|1267|Tuesday|1924Q2|N|N|N|2423877|2423967|2423518|2423793|N|N|N|N|N| +2423885|AAAAAAAANEMPECAA|1924-04-09|291|1267|98|1924|3|4|9|2|1924|98|1267|Wednesday|1924Q2|N|N|N|2423877|2423967|2423519|2423794|N|N|N|N|N| +2423886|AAAAAAAAOEMPECAA|1924-04-10|291|1267|98|1924|4|4|10|2|1924|98|1267|Thursday|1924Q2|N|N|N|2423877|2423967|2423520|2423795|N|N|N|N|N| +2423887|AAAAAAAAPEMPECAA|1924-04-11|291|1267|98|1924|5|4|11|2|1924|98|1267|Friday|1924Q2|N|Y|N|2423877|2423967|2423521|2423796|N|N|N|N|N| +2423888|AAAAAAAAAFMPECAA|1924-04-12|291|1267|98|1924|6|4|12|2|1924|98|1267|Saturday|1924Q2|N|Y|N|2423877|2423967|2423522|2423797|N|N|N|N|N| +2423889|AAAAAAAABFMPECAA|1924-04-13|291|1267|98|1924|0|4|13|2|1924|98|1267|Sunday|1924Q2|N|N|N|2423877|2423967|2423523|2423798|N|N|N|N|N| +2423890|AAAAAAAACFMPECAA|1924-04-14|291|1267|98|1924|1|4|14|2|1924|98|1267|Monday|1924Q2|N|N|N|2423877|2423967|2423524|2423799|N|N|N|N|N| +2423891|AAAAAAAADFMPECAA|1924-04-15|291|1268|98|1924|2|4|15|2|1924|98|1268|Tuesday|1924Q2|N|N|N|2423877|2423967|2423525|2423800|N|N|N|N|N| +2423892|AAAAAAAAEFMPECAA|1924-04-16|291|1268|98|1924|3|4|16|2|1924|98|1268|Wednesday|1924Q2|N|N|N|2423877|2423967|2423526|2423801|N|N|N|N|N| +2423893|AAAAAAAAFFMPECAA|1924-04-17|291|1268|98|1924|4|4|17|2|1924|98|1268|Thursday|1924Q2|N|N|N|2423877|2423967|2423527|2423802|N|N|N|N|N| +2423894|AAAAAAAAGFMPECAA|1924-04-18|291|1268|98|1924|5|4|18|2|1924|98|1268|Friday|1924Q2|N|Y|N|2423877|2423967|2423528|2423803|N|N|N|N|N| +2423895|AAAAAAAAHFMPECAA|1924-04-19|291|1268|98|1924|6|4|19|2|1924|98|1268|Saturday|1924Q2|N|Y|N|2423877|2423967|2423529|2423804|N|N|N|N|N| +2423896|AAAAAAAAIFMPECAA|1924-04-20|291|1268|98|1924|0|4|20|2|1924|98|1268|Sunday|1924Q2|N|N|N|2423877|2423967|2423530|2423805|N|N|N|N|N| +2423897|AAAAAAAAJFMPECAA|1924-04-21|291|1268|98|1924|1|4|21|2|1924|98|1268|Monday|1924Q2|N|N|N|2423877|2423967|2423531|2423806|N|N|N|N|N| +2423898|AAAAAAAAKFMPECAA|1924-04-22|291|1269|98|1924|2|4|22|2|1924|98|1269|Tuesday|1924Q2|N|N|N|2423877|2423967|2423532|2423807|N|N|N|N|N| +2423899|AAAAAAAALFMPECAA|1924-04-23|291|1269|98|1924|3|4|23|2|1924|98|1269|Wednesday|1924Q2|N|N|N|2423877|2423967|2423533|2423808|N|N|N|N|N| +2423900|AAAAAAAAMFMPECAA|1924-04-24|291|1269|98|1924|4|4|24|2|1924|98|1269|Thursday|1924Q2|N|N|N|2423877|2423967|2423534|2423809|N|N|N|N|N| +2423901|AAAAAAAANFMPECAA|1924-04-25|291|1269|98|1924|5|4|25|2|1924|98|1269|Friday|1924Q2|N|Y|N|2423877|2423967|2423535|2423810|N|N|N|N|N| +2423902|AAAAAAAAOFMPECAA|1924-04-26|291|1269|98|1924|6|4|26|2|1924|98|1269|Saturday|1924Q2|N|Y|N|2423877|2423967|2423536|2423811|N|N|N|N|N| +2423903|AAAAAAAAPFMPECAA|1924-04-27|291|1269|98|1924|0|4|27|2|1924|98|1269|Sunday|1924Q2|N|N|N|2423877|2423967|2423537|2423812|N|N|N|N|N| +2423904|AAAAAAAAAGMPECAA|1924-04-28|291|1269|98|1924|1|4|28|2|1924|98|1269|Monday|1924Q2|N|N|N|2423877|2423967|2423538|2423813|N|N|N|N|N| +2423905|AAAAAAAABGMPECAA|1924-04-29|291|1270|98|1924|2|4|29|2|1924|98|1270|Tuesday|1924Q2|N|N|N|2423877|2423967|2423539|2423814|N|N|N|N|N| +2423906|AAAAAAAACGMPECAA|1924-04-30|291|1270|98|1924|3|4|30|2|1924|98|1270|Wednesday|1924Q2|N|N|N|2423877|2423967|2423540|2423815|N|N|N|N|N| +2423907|AAAAAAAADGMPECAA|1924-05-01|292|1270|98|1924|4|5|1|2|1924|98|1270|Thursday|1924Q2|N|N|N|2423907|2424027|2423541|2423816|N|N|N|N|N| +2423908|AAAAAAAAEGMPECAA|1924-05-02|292|1270|98|1924|5|5|2|2|1924|98|1270|Friday|1924Q2|N|Y|N|2423907|2424027|2423542|2423817|N|N|N|N|N| +2423909|AAAAAAAAFGMPECAA|1924-05-03|292|1270|98|1924|6|5|3|2|1924|98|1270|Saturday|1924Q2|N|Y|N|2423907|2424027|2423543|2423818|N|N|N|N|N| +2423910|AAAAAAAAGGMPECAA|1924-05-04|292|1270|98|1924|0|5|4|2|1924|98|1270|Sunday|1924Q2|N|N|N|2423907|2424027|2423544|2423819|N|N|N|N|N| +2423911|AAAAAAAAHGMPECAA|1924-05-05|292|1270|98|1924|1|5|5|2|1924|98|1270|Monday|1924Q2|N|N|N|2423907|2424027|2423545|2423820|N|N|N|N|N| +2423912|AAAAAAAAIGMPECAA|1924-05-06|292|1271|98|1924|2|5|6|2|1924|98|1271|Tuesday|1924Q2|N|N|N|2423907|2424027|2423546|2423821|N|N|N|N|N| +2423913|AAAAAAAAJGMPECAA|1924-05-07|292|1271|98|1924|3|5|7|2|1924|98|1271|Wednesday|1924Q2|N|N|N|2423907|2424027|2423547|2423822|N|N|N|N|N| +2423914|AAAAAAAAKGMPECAA|1924-05-08|292|1271|98|1924|4|5|8|2|1924|98|1271|Thursday|1924Q2|N|N|N|2423907|2424027|2423548|2423823|N|N|N|N|N| +2423915|AAAAAAAALGMPECAA|1924-05-09|292|1271|98|1924|5|5|9|2|1924|98|1271|Friday|1924Q2|N|Y|N|2423907|2424027|2423549|2423824|N|N|N|N|N| +2423916|AAAAAAAAMGMPECAA|1924-05-10|292|1271|98|1924|6|5|10|2|1924|98|1271|Saturday|1924Q2|N|Y|N|2423907|2424027|2423550|2423825|N|N|N|N|N| +2423917|AAAAAAAANGMPECAA|1924-05-11|292|1271|98|1924|0|5|11|2|1924|98|1271|Sunday|1924Q2|N|N|N|2423907|2424027|2423551|2423826|N|N|N|N|N| +2423918|AAAAAAAAOGMPECAA|1924-05-12|292|1271|98|1924|1|5|12|2|1924|98|1271|Monday|1924Q2|N|N|N|2423907|2424027|2423552|2423827|N|N|N|N|N| +2423919|AAAAAAAAPGMPECAA|1924-05-13|292|1272|98|1924|2|5|13|2|1924|98|1272|Tuesday|1924Q2|N|N|N|2423907|2424027|2423553|2423828|N|N|N|N|N| +2423920|AAAAAAAAAHMPECAA|1924-05-14|292|1272|98|1924|3|5|14|2|1924|98|1272|Wednesday|1924Q2|N|N|N|2423907|2424027|2423554|2423829|N|N|N|N|N| +2423921|AAAAAAAABHMPECAA|1924-05-15|292|1272|98|1924|4|5|15|2|1924|98|1272|Thursday|1924Q2|N|N|N|2423907|2424027|2423555|2423830|N|N|N|N|N| +2423922|AAAAAAAACHMPECAA|1924-05-16|292|1272|98|1924|5|5|16|2|1924|98|1272|Friday|1924Q2|N|Y|N|2423907|2424027|2423556|2423831|N|N|N|N|N| +2423923|AAAAAAAADHMPECAA|1924-05-17|292|1272|98|1924|6|5|17|2|1924|98|1272|Saturday|1924Q2|N|Y|N|2423907|2424027|2423557|2423832|N|N|N|N|N| +2423924|AAAAAAAAEHMPECAA|1924-05-18|292|1272|98|1924|0|5|18|2|1924|98|1272|Sunday|1924Q2|N|N|N|2423907|2424027|2423558|2423833|N|N|N|N|N| +2423925|AAAAAAAAFHMPECAA|1924-05-19|292|1272|98|1924|1|5|19|2|1924|98|1272|Monday|1924Q2|N|N|N|2423907|2424027|2423559|2423834|N|N|N|N|N| +2423926|AAAAAAAAGHMPECAA|1924-05-20|292|1273|98|1924|2|5|20|2|1924|98|1273|Tuesday|1924Q2|N|N|N|2423907|2424027|2423560|2423835|N|N|N|N|N| +2423927|AAAAAAAAHHMPECAA|1924-05-21|292|1273|98|1924|3|5|21|2|1924|98|1273|Wednesday|1924Q2|N|N|N|2423907|2424027|2423561|2423836|N|N|N|N|N| +2423928|AAAAAAAAIHMPECAA|1924-05-22|292|1273|98|1924|4|5|22|2|1924|98|1273|Thursday|1924Q2|N|N|N|2423907|2424027|2423562|2423837|N|N|N|N|N| +2423929|AAAAAAAAJHMPECAA|1924-05-23|292|1273|98|1924|5|5|23|2|1924|98|1273|Friday|1924Q2|N|Y|N|2423907|2424027|2423563|2423838|N|N|N|N|N| +2423930|AAAAAAAAKHMPECAA|1924-05-24|292|1273|98|1924|6|5|24|2|1924|98|1273|Saturday|1924Q2|N|Y|N|2423907|2424027|2423564|2423839|N|N|N|N|N| +2423931|AAAAAAAALHMPECAA|1924-05-25|292|1273|98|1924|0|5|25|2|1924|98|1273|Sunday|1924Q2|N|N|N|2423907|2424027|2423565|2423840|N|N|N|N|N| +2423932|AAAAAAAAMHMPECAA|1924-05-26|292|1273|98|1924|1|5|26|2|1924|98|1273|Monday|1924Q2|N|N|N|2423907|2424027|2423566|2423841|N|N|N|N|N| +2423933|AAAAAAAANHMPECAA|1924-05-27|292|1274|98|1924|2|5|27|2|1924|98|1274|Tuesday|1924Q2|N|N|N|2423907|2424027|2423567|2423842|N|N|N|N|N| +2423934|AAAAAAAAOHMPECAA|1924-05-28|292|1274|98|1924|3|5|28|2|1924|98|1274|Wednesday|1924Q2|N|N|N|2423907|2424027|2423568|2423843|N|N|N|N|N| +2423935|AAAAAAAAPHMPECAA|1924-05-29|292|1274|98|1924|4|5|29|2|1924|98|1274|Thursday|1924Q2|N|N|N|2423907|2424027|2423569|2423844|N|N|N|N|N| +2423936|AAAAAAAAAIMPECAA|1924-05-30|292|1274|98|1924|5|5|30|2|1924|98|1274|Friday|1924Q2|N|Y|N|2423907|2424027|2423570|2423845|N|N|N|N|N| +2423937|AAAAAAAABIMPECAA|1924-05-31|292|1274|98|1924|6|5|31|2|1924|98|1274|Saturday|1924Q2|N|Y|N|2423907|2424027|2423571|2423846|N|N|N|N|N| +2423938|AAAAAAAACIMPECAA|1924-06-01|293|1274|99|1924|0|6|1|2|1924|99|1274|Sunday|1924Q2|N|N|N|2423938|2424089|2423572|2423847|N|N|N|N|N| +2423939|AAAAAAAADIMPECAA|1924-06-02|293|1274|99|1924|1|6|2|2|1924|99|1274|Monday|1924Q2|N|N|N|2423938|2424089|2423573|2423848|N|N|N|N|N| +2423940|AAAAAAAAEIMPECAA|1924-06-03|293|1275|99|1924|2|6|3|2|1924|99|1275|Tuesday|1924Q2|N|N|N|2423938|2424089|2423574|2423849|N|N|N|N|N| +2423941|AAAAAAAAFIMPECAA|1924-06-04|293|1275|99|1924|3|6|4|2|1924|99|1275|Wednesday|1924Q2|N|N|N|2423938|2424089|2423575|2423850|N|N|N|N|N| +2423942|AAAAAAAAGIMPECAA|1924-06-05|293|1275|99|1924|4|6|5|2|1924|99|1275|Thursday|1924Q2|N|N|N|2423938|2424089|2423576|2423851|N|N|N|N|N| +2423943|AAAAAAAAHIMPECAA|1924-06-06|293|1275|99|1924|5|6|6|2|1924|99|1275|Friday|1924Q2|N|Y|N|2423938|2424089|2423577|2423852|N|N|N|N|N| +2423944|AAAAAAAAIIMPECAA|1924-06-07|293|1275|99|1924|6|6|7|2|1924|99|1275|Saturday|1924Q2|N|Y|N|2423938|2424089|2423578|2423853|N|N|N|N|N| +2423945|AAAAAAAAJIMPECAA|1924-06-08|293|1275|99|1924|0|6|8|2|1924|99|1275|Sunday|1924Q2|N|N|N|2423938|2424089|2423579|2423854|N|N|N|N|N| +2423946|AAAAAAAAKIMPECAA|1924-06-09|293|1275|99|1924|1|6|9|2|1924|99|1275|Monday|1924Q2|N|N|N|2423938|2424089|2423580|2423855|N|N|N|N|N| +2423947|AAAAAAAALIMPECAA|1924-06-10|293|1276|99|1924|2|6|10|2|1924|99|1276|Tuesday|1924Q2|N|N|N|2423938|2424089|2423581|2423856|N|N|N|N|N| +2423948|AAAAAAAAMIMPECAA|1924-06-11|293|1276|99|1924|3|6|11|2|1924|99|1276|Wednesday|1924Q2|N|N|N|2423938|2424089|2423582|2423857|N|N|N|N|N| +2423949|AAAAAAAANIMPECAA|1924-06-12|293|1276|99|1924|4|6|12|2|1924|99|1276|Thursday|1924Q2|N|N|N|2423938|2424089|2423583|2423858|N|N|N|N|N| +2423950|AAAAAAAAOIMPECAA|1924-06-13|293|1276|99|1924|5|6|13|2|1924|99|1276|Friday|1924Q2|N|Y|N|2423938|2424089|2423584|2423859|N|N|N|N|N| +2423951|AAAAAAAAPIMPECAA|1924-06-14|293|1276|99|1924|6|6|14|2|1924|99|1276|Saturday|1924Q2|N|Y|N|2423938|2424089|2423585|2423860|N|N|N|N|N| +2423952|AAAAAAAAAJMPECAA|1924-06-15|293|1276|99|1924|0|6|15|2|1924|99|1276|Sunday|1924Q2|N|N|N|2423938|2424089|2423586|2423861|N|N|N|N|N| +2423953|AAAAAAAABJMPECAA|1924-06-16|293|1276|99|1924|1|6|16|2|1924|99|1276|Monday|1924Q2|N|N|N|2423938|2424089|2423587|2423862|N|N|N|N|N| +2423954|AAAAAAAACJMPECAA|1924-06-17|293|1277|99|1924|2|6|17|2|1924|99|1277|Tuesday|1924Q2|N|N|N|2423938|2424089|2423588|2423863|N|N|N|N|N| +2423955|AAAAAAAADJMPECAA|1924-06-18|293|1277|99|1924|3|6|18|2|1924|99|1277|Wednesday|1924Q2|N|N|N|2423938|2424089|2423589|2423864|N|N|N|N|N| +2423956|AAAAAAAAEJMPECAA|1924-06-19|293|1277|99|1924|4|6|19|2|1924|99|1277|Thursday|1924Q2|N|N|N|2423938|2424089|2423590|2423865|N|N|N|N|N| +2423957|AAAAAAAAFJMPECAA|1924-06-20|293|1277|99|1924|5|6|20|2|1924|99|1277|Friday|1924Q2|N|Y|N|2423938|2424089|2423591|2423866|N|N|N|N|N| +2423958|AAAAAAAAGJMPECAA|1924-06-21|293|1277|99|1924|6|6|21|2|1924|99|1277|Saturday|1924Q2|N|Y|N|2423938|2424089|2423592|2423867|N|N|N|N|N| +2423959|AAAAAAAAHJMPECAA|1924-06-22|293|1277|99|1924|0|6|22|2|1924|99|1277|Sunday|1924Q2|N|N|N|2423938|2424089|2423593|2423868|N|N|N|N|N| +2423960|AAAAAAAAIJMPECAA|1924-06-23|293|1277|99|1924|1|6|23|2|1924|99|1277|Monday|1924Q2|N|N|N|2423938|2424089|2423594|2423869|N|N|N|N|N| +2423961|AAAAAAAAJJMPECAA|1924-06-24|293|1278|99|1924|2|6|24|2|1924|99|1278|Tuesday|1924Q2|N|N|N|2423938|2424089|2423595|2423870|N|N|N|N|N| +2423962|AAAAAAAAKJMPECAA|1924-06-25|293|1278|99|1924|3|6|25|2|1924|99|1278|Wednesday|1924Q2|N|N|N|2423938|2424089|2423596|2423871|N|N|N|N|N| +2423963|AAAAAAAALJMPECAA|1924-06-26|293|1278|99|1924|4|6|26|2|1924|99|1278|Thursday|1924Q2|N|N|N|2423938|2424089|2423597|2423872|N|N|N|N|N| +2423964|AAAAAAAAMJMPECAA|1924-06-27|293|1278|99|1924|5|6|27|2|1924|99|1278|Friday|1924Q2|N|Y|N|2423938|2424089|2423598|2423873|N|N|N|N|N| +2423965|AAAAAAAANJMPECAA|1924-06-28|293|1278|99|1924|6|6|28|2|1924|99|1278|Saturday|1924Q2|N|Y|N|2423938|2424089|2423599|2423874|N|N|N|N|N| +2423966|AAAAAAAAOJMPECAA|1924-06-29|293|1278|99|1924|0|6|29|2|1924|99|1278|Sunday|1924Q2|N|N|N|2423938|2424089|2423600|2423875|N|N|N|N|N| +2423967|AAAAAAAAPJMPECAA|1924-06-30|293|1278|99|1924|1|6|30|2|1924|99|1278|Monday|1924Q2|N|N|N|2423938|2424089|2423601|2423876|N|N|N|N|N| +2423968|AAAAAAAAAKMPECAA|1924-07-01|294|1279|99|1924|2|7|1|3|1924|99|1279|Tuesday|1924Q3|N|N|N|2423968|2424149|2423602|2423877|N|N|N|N|N| +2423969|AAAAAAAABKMPECAA|1924-07-02|294|1279|99|1924|3|7|2|3|1924|99|1279|Wednesday|1924Q3|N|N|N|2423968|2424149|2423603|2423878|N|N|N|N|N| +2423970|AAAAAAAACKMPECAA|1924-07-03|294|1279|99|1924|4|7|3|3|1924|99|1279|Thursday|1924Q3|N|N|N|2423968|2424149|2423604|2423879|N|N|N|N|N| +2423971|AAAAAAAADKMPECAA|1924-07-04|294|1279|99|1924|5|7|4|3|1924|99|1279|Friday|1924Q3|Y|Y|N|2423968|2424149|2423605|2423880|N|N|N|N|N| +2423972|AAAAAAAAEKMPECAA|1924-07-05|294|1279|99|1924|6|7|5|3|1924|99|1279|Saturday|1924Q3|N|Y|Y|2423968|2424149|2423606|2423881|N|N|N|N|N| +2423973|AAAAAAAAFKMPECAA|1924-07-06|294|1279|99|1924|0|7|6|3|1924|99|1279|Sunday|1924Q3|N|N|N|2423968|2424149|2423607|2423882|N|N|N|N|N| +2423974|AAAAAAAAGKMPECAA|1924-07-07|294|1279|99|1924|1|7|7|3|1924|99|1279|Monday|1924Q3|N|N|N|2423968|2424149|2423608|2423883|N|N|N|N|N| +2423975|AAAAAAAAHKMPECAA|1924-07-08|294|1280|99|1924|2|7|8|3|1924|99|1280|Tuesday|1924Q3|N|N|N|2423968|2424149|2423609|2423884|N|N|N|N|N| +2423976|AAAAAAAAIKMPECAA|1924-07-09|294|1280|99|1924|3|7|9|3|1924|99|1280|Wednesday|1924Q3|N|N|N|2423968|2424149|2423610|2423885|N|N|N|N|N| +2423977|AAAAAAAAJKMPECAA|1924-07-10|294|1280|99|1924|4|7|10|3|1924|99|1280|Thursday|1924Q3|N|N|N|2423968|2424149|2423611|2423886|N|N|N|N|N| +2423978|AAAAAAAAKKMPECAA|1924-07-11|294|1280|99|1924|5|7|11|3|1924|99|1280|Friday|1924Q3|N|Y|N|2423968|2424149|2423612|2423887|N|N|N|N|N| +2423979|AAAAAAAALKMPECAA|1924-07-12|294|1280|99|1924|6|7|12|3|1924|99|1280|Saturday|1924Q3|N|Y|N|2423968|2424149|2423613|2423888|N|N|N|N|N| +2423980|AAAAAAAAMKMPECAA|1924-07-13|294|1280|99|1924|0|7|13|3|1924|99|1280|Sunday|1924Q3|N|N|N|2423968|2424149|2423614|2423889|N|N|N|N|N| +2423981|AAAAAAAANKMPECAA|1924-07-14|294|1280|99|1924|1|7|14|3|1924|99|1280|Monday|1924Q3|N|N|N|2423968|2424149|2423615|2423890|N|N|N|N|N| +2423982|AAAAAAAAOKMPECAA|1924-07-15|294|1281|99|1924|2|7|15|3|1924|99|1281|Tuesday|1924Q3|N|N|N|2423968|2424149|2423616|2423891|N|N|N|N|N| +2423983|AAAAAAAAPKMPECAA|1924-07-16|294|1281|99|1924|3|7|16|3|1924|99|1281|Wednesday|1924Q3|N|N|N|2423968|2424149|2423617|2423892|N|N|N|N|N| +2423984|AAAAAAAAALMPECAA|1924-07-17|294|1281|99|1924|4|7|17|3|1924|99|1281|Thursday|1924Q3|N|N|N|2423968|2424149|2423618|2423893|N|N|N|N|N| +2423985|AAAAAAAABLMPECAA|1924-07-18|294|1281|99|1924|5|7|18|3|1924|99|1281|Friday|1924Q3|N|Y|N|2423968|2424149|2423619|2423894|N|N|N|N|N| +2423986|AAAAAAAACLMPECAA|1924-07-19|294|1281|99|1924|6|7|19|3|1924|99|1281|Saturday|1924Q3|N|Y|N|2423968|2424149|2423620|2423895|N|N|N|N|N| +2423987|AAAAAAAADLMPECAA|1924-07-20|294|1281|99|1924|0|7|20|3|1924|99|1281|Sunday|1924Q3|N|N|N|2423968|2424149|2423621|2423896|N|N|N|N|N| +2423988|AAAAAAAAELMPECAA|1924-07-21|294|1281|99|1924|1|7|21|3|1924|99|1281|Monday|1924Q3|N|N|N|2423968|2424149|2423622|2423897|N|N|N|N|N| +2423989|AAAAAAAAFLMPECAA|1924-07-22|294|1282|99|1924|2|7|22|3|1924|99|1282|Tuesday|1924Q3|N|N|N|2423968|2424149|2423623|2423898|N|N|N|N|N| +2423990|AAAAAAAAGLMPECAA|1924-07-23|294|1282|99|1924|3|7|23|3|1924|99|1282|Wednesday|1924Q3|N|N|N|2423968|2424149|2423624|2423899|N|N|N|N|N| +2423991|AAAAAAAAHLMPECAA|1924-07-24|294|1282|99|1924|4|7|24|3|1924|99|1282|Thursday|1924Q3|N|N|N|2423968|2424149|2423625|2423900|N|N|N|N|N| +2423992|AAAAAAAAILMPECAA|1924-07-25|294|1282|99|1924|5|7|25|3|1924|99|1282|Friday|1924Q3|N|Y|N|2423968|2424149|2423626|2423901|N|N|N|N|N| +2423993|AAAAAAAAJLMPECAA|1924-07-26|294|1282|99|1924|6|7|26|3|1924|99|1282|Saturday|1924Q3|N|Y|N|2423968|2424149|2423627|2423902|N|N|N|N|N| +2423994|AAAAAAAAKLMPECAA|1924-07-27|294|1282|99|1924|0|7|27|3|1924|99|1282|Sunday|1924Q3|N|N|N|2423968|2424149|2423628|2423903|N|N|N|N|N| +2423995|AAAAAAAALLMPECAA|1924-07-28|294|1282|99|1924|1|7|28|3|1924|99|1282|Monday|1924Q3|N|N|N|2423968|2424149|2423629|2423904|N|N|N|N|N| +2423996|AAAAAAAAMLMPECAA|1924-07-29|294|1283|99|1924|2|7|29|3|1924|99|1283|Tuesday|1924Q3|N|N|N|2423968|2424149|2423630|2423905|N|N|N|N|N| +2423997|AAAAAAAANLMPECAA|1924-07-30|294|1283|99|1924|3|7|30|3|1924|99|1283|Wednesday|1924Q3|N|N|N|2423968|2424149|2423631|2423906|N|N|N|N|N| +2423998|AAAAAAAAOLMPECAA|1924-07-31|294|1283|99|1924|4|7|31|3|1924|99|1283|Thursday|1924Q3|N|N|N|2423968|2424149|2423632|2423907|N|N|N|N|N| +2423999|AAAAAAAAPLMPECAA|1924-08-01|295|1283|99|1924|5|8|1|3|1924|99|1283|Friday|1924Q3|N|Y|N|2423999|2424211|2423633|2423908|N|N|N|N|N| +2424000|AAAAAAAAAMMPECAA|1924-08-02|295|1283|99|1924|6|8|2|3|1924|99|1283|Saturday|1924Q3|N|Y|N|2423999|2424211|2423634|2423909|N|N|N|N|N| +2424001|AAAAAAAABMMPECAA|1924-08-03|295|1283|99|1924|0|8|3|3|1924|99|1283|Sunday|1924Q3|N|N|N|2423999|2424211|2423635|2423910|N|N|N|N|N| +2424002|AAAAAAAACMMPECAA|1924-08-04|295|1283|99|1924|1|8|4|3|1924|99|1283|Monday|1924Q3|N|N|N|2423999|2424211|2423636|2423911|N|N|N|N|N| +2424003|AAAAAAAADMMPECAA|1924-08-05|295|1284|99|1924|2|8|5|3|1924|99|1284|Tuesday|1924Q3|N|N|N|2423999|2424211|2423637|2423912|N|N|N|N|N| +2424004|AAAAAAAAEMMPECAA|1924-08-06|295|1284|99|1924|3|8|6|3|1924|99|1284|Wednesday|1924Q3|N|N|N|2423999|2424211|2423638|2423913|N|N|N|N|N| +2424005|AAAAAAAAFMMPECAA|1924-08-07|295|1284|99|1924|4|8|7|3|1924|99|1284|Thursday|1924Q3|N|N|N|2423999|2424211|2423639|2423914|N|N|N|N|N| +2424006|AAAAAAAAGMMPECAA|1924-08-08|295|1284|99|1924|5|8|8|3|1924|99|1284|Friday|1924Q3|N|Y|N|2423999|2424211|2423640|2423915|N|N|N|N|N| +2424007|AAAAAAAAHMMPECAA|1924-08-09|295|1284|99|1924|6|8|9|3|1924|99|1284|Saturday|1924Q3|N|Y|N|2423999|2424211|2423641|2423916|N|N|N|N|N| +2424008|AAAAAAAAIMMPECAA|1924-08-10|295|1284|99|1924|0|8|10|3|1924|99|1284|Sunday|1924Q3|N|N|N|2423999|2424211|2423642|2423917|N|N|N|N|N| +2424009|AAAAAAAAJMMPECAA|1924-08-11|295|1284|99|1924|1|8|11|3|1924|99|1284|Monday|1924Q3|N|N|N|2423999|2424211|2423643|2423918|N|N|N|N|N| +2424010|AAAAAAAAKMMPECAA|1924-08-12|295|1285|99|1924|2|8|12|3|1924|99|1285|Tuesday|1924Q3|N|N|N|2423999|2424211|2423644|2423919|N|N|N|N|N| +2424011|AAAAAAAALMMPECAA|1924-08-13|295|1285|99|1924|3|8|13|3|1924|99|1285|Wednesday|1924Q3|N|N|N|2423999|2424211|2423645|2423920|N|N|N|N|N| +2424012|AAAAAAAAMMMPECAA|1924-08-14|295|1285|99|1924|4|8|14|3|1924|99|1285|Thursday|1924Q3|N|N|N|2423999|2424211|2423646|2423921|N|N|N|N|N| +2424013|AAAAAAAANMMPECAA|1924-08-15|295|1285|99|1924|5|8|15|3|1924|99|1285|Friday|1924Q3|N|Y|N|2423999|2424211|2423647|2423922|N|N|N|N|N| +2424014|AAAAAAAAOMMPECAA|1924-08-16|295|1285|99|1924|6|8|16|3|1924|99|1285|Saturday|1924Q3|N|Y|N|2423999|2424211|2423648|2423923|N|N|N|N|N| +2424015|AAAAAAAAPMMPECAA|1924-08-17|295|1285|99|1924|0|8|17|3|1924|99|1285|Sunday|1924Q3|N|N|N|2423999|2424211|2423649|2423924|N|N|N|N|N| +2424016|AAAAAAAAANMPECAA|1924-08-18|295|1285|99|1924|1|8|18|3|1924|99|1285|Monday|1924Q3|N|N|N|2423999|2424211|2423650|2423925|N|N|N|N|N| +2424017|AAAAAAAABNMPECAA|1924-08-19|295|1286|99|1924|2|8|19|3|1924|99|1286|Tuesday|1924Q3|N|N|N|2423999|2424211|2423651|2423926|N|N|N|N|N| +2424018|AAAAAAAACNMPECAA|1924-08-20|295|1286|99|1924|3|8|20|3|1924|99|1286|Wednesday|1924Q3|N|N|N|2423999|2424211|2423652|2423927|N|N|N|N|N| +2424019|AAAAAAAADNMPECAA|1924-08-21|295|1286|99|1924|4|8|21|3|1924|99|1286|Thursday|1924Q3|N|N|N|2423999|2424211|2423653|2423928|N|N|N|N|N| +2424020|AAAAAAAAENMPECAA|1924-08-22|295|1286|99|1924|5|8|22|3|1924|99|1286|Friday|1924Q3|N|Y|N|2423999|2424211|2423654|2423929|N|N|N|N|N| +2424021|AAAAAAAAFNMPECAA|1924-08-23|295|1286|99|1924|6|8|23|3|1924|99|1286|Saturday|1924Q3|N|Y|N|2423999|2424211|2423655|2423930|N|N|N|N|N| +2424022|AAAAAAAAGNMPECAA|1924-08-24|295|1286|99|1924|0|8|24|3|1924|99|1286|Sunday|1924Q3|N|N|N|2423999|2424211|2423656|2423931|N|N|N|N|N| +2424023|AAAAAAAAHNMPECAA|1924-08-25|295|1286|99|1924|1|8|25|3|1924|99|1286|Monday|1924Q3|N|N|N|2423999|2424211|2423657|2423932|N|N|N|N|N| +2424024|AAAAAAAAINMPECAA|1924-08-26|295|1287|99|1924|2|8|26|3|1924|99|1287|Tuesday|1924Q3|N|N|N|2423999|2424211|2423658|2423933|N|N|N|N|N| +2424025|AAAAAAAAJNMPECAA|1924-08-27|295|1287|99|1924|3|8|27|3|1924|99|1287|Wednesday|1924Q3|N|N|N|2423999|2424211|2423659|2423934|N|N|N|N|N| +2424026|AAAAAAAAKNMPECAA|1924-08-28|295|1287|99|1924|4|8|28|3|1924|99|1287|Thursday|1924Q3|N|N|N|2423999|2424211|2423660|2423935|N|N|N|N|N| +2424027|AAAAAAAALNMPECAA|1924-08-29|295|1287|99|1924|5|8|29|3|1924|99|1287|Friday|1924Q3|N|Y|N|2423999|2424211|2423661|2423936|N|N|N|N|N| +2424028|AAAAAAAAMNMPECAA|1924-08-30|295|1287|99|1924|6|8|30|3|1924|99|1287|Saturday|1924Q3|N|Y|N|2423999|2424211|2423662|2423937|N|N|N|N|N| +2424029|AAAAAAAANNMPECAA|1924-08-31|295|1287|99|1924|0|8|31|3|1924|99|1287|Sunday|1924Q3|N|N|N|2423999|2424211|2423663|2423938|N|N|N|N|N| +2424030|AAAAAAAAONMPECAA|1924-09-01|296|1287|100|1924|1|9|1|3|1924|100|1287|Monday|1924Q3|N|N|N|2424030|2424273|2423664|2423939|N|N|N|N|N| +2424031|AAAAAAAAPNMPECAA|1924-09-02|296|1288|100|1924|2|9|2|3|1924|100|1288|Tuesday|1924Q3|N|N|N|2424030|2424273|2423665|2423940|N|N|N|N|N| +2424032|AAAAAAAAAOMPECAA|1924-09-03|296|1288|100|1924|3|9|3|3|1924|100|1288|Wednesday|1924Q3|N|N|N|2424030|2424273|2423666|2423941|N|N|N|N|N| +2424033|AAAAAAAABOMPECAA|1924-09-04|296|1288|100|1924|4|9|4|3|1924|100|1288|Thursday|1924Q3|N|N|N|2424030|2424273|2423667|2423942|N|N|N|N|N| +2424034|AAAAAAAACOMPECAA|1924-09-05|296|1288|100|1924|5|9|5|3|1924|100|1288|Friday|1924Q3|N|Y|N|2424030|2424273|2423668|2423943|N|N|N|N|N| +2424035|AAAAAAAADOMPECAA|1924-09-06|296|1288|100|1924|6|9|6|3|1924|100|1288|Saturday|1924Q3|N|Y|N|2424030|2424273|2423669|2423944|N|N|N|N|N| +2424036|AAAAAAAAEOMPECAA|1924-09-07|296|1288|100|1924|0|9|7|3|1924|100|1288|Sunday|1924Q3|N|N|N|2424030|2424273|2423670|2423945|N|N|N|N|N| +2424037|AAAAAAAAFOMPECAA|1924-09-08|296|1288|100|1924|1|9|8|3|1924|100|1288|Monday|1924Q3|N|N|N|2424030|2424273|2423671|2423946|N|N|N|N|N| +2424038|AAAAAAAAGOMPECAA|1924-09-09|296|1289|100|1924|2|9|9|3|1924|100|1289|Tuesday|1924Q3|N|N|N|2424030|2424273|2423672|2423947|N|N|N|N|N| +2424039|AAAAAAAAHOMPECAA|1924-09-10|296|1289|100|1924|3|9|10|3|1924|100|1289|Wednesday|1924Q3|N|N|N|2424030|2424273|2423673|2423948|N|N|N|N|N| +2424040|AAAAAAAAIOMPECAA|1924-09-11|296|1289|100|1924|4|9|11|3|1924|100|1289|Thursday|1924Q3|N|N|N|2424030|2424273|2423674|2423949|N|N|N|N|N| +2424041|AAAAAAAAJOMPECAA|1924-09-12|296|1289|100|1924|5|9|12|3|1924|100|1289|Friday|1924Q3|N|Y|N|2424030|2424273|2423675|2423950|N|N|N|N|N| +2424042|AAAAAAAAKOMPECAA|1924-09-13|296|1289|100|1924|6|9|13|3|1924|100|1289|Saturday|1924Q3|N|Y|N|2424030|2424273|2423676|2423951|N|N|N|N|N| +2424043|AAAAAAAALOMPECAA|1924-09-14|296|1289|100|1924|0|9|14|3|1924|100|1289|Sunday|1924Q3|N|N|N|2424030|2424273|2423677|2423952|N|N|N|N|N| +2424044|AAAAAAAAMOMPECAA|1924-09-15|296|1289|100|1924|1|9|15|3|1924|100|1289|Monday|1924Q3|N|N|N|2424030|2424273|2423678|2423953|N|N|N|N|N| +2424045|AAAAAAAANOMPECAA|1924-09-16|296|1290|100|1924|2|9|16|3|1924|100|1290|Tuesday|1924Q3|N|N|N|2424030|2424273|2423679|2423954|N|N|N|N|N| +2424046|AAAAAAAAOOMPECAA|1924-09-17|296|1290|100|1924|3|9|17|3|1924|100|1290|Wednesday|1924Q3|N|N|N|2424030|2424273|2423680|2423955|N|N|N|N|N| +2424047|AAAAAAAAPOMPECAA|1924-09-18|296|1290|100|1924|4|9|18|3|1924|100|1290|Thursday|1924Q3|N|N|N|2424030|2424273|2423681|2423956|N|N|N|N|N| +2424048|AAAAAAAAAPMPECAA|1924-09-19|296|1290|100|1924|5|9|19|3|1924|100|1290|Friday|1924Q3|N|Y|N|2424030|2424273|2423682|2423957|N|N|N|N|N| +2424049|AAAAAAAABPMPECAA|1924-09-20|296|1290|100|1924|6|9|20|3|1924|100|1290|Saturday|1924Q3|N|Y|N|2424030|2424273|2423683|2423958|N|N|N|N|N| +2424050|AAAAAAAACPMPECAA|1924-09-21|296|1290|100|1924|0|9|21|3|1924|100|1290|Sunday|1924Q3|N|N|N|2424030|2424273|2423684|2423959|N|N|N|N|N| +2424051|AAAAAAAADPMPECAA|1924-09-22|296|1290|100|1924|1|9|22|3|1924|100|1290|Monday|1924Q3|N|N|N|2424030|2424273|2423685|2423960|N|N|N|N|N| +2424052|AAAAAAAAEPMPECAA|1924-09-23|296|1291|100|1924|2|9|23|3|1924|100|1291|Tuesday|1924Q3|N|N|N|2424030|2424273|2423686|2423961|N|N|N|N|N| +2424053|AAAAAAAAFPMPECAA|1924-09-24|296|1291|100|1924|3|9|24|3|1924|100|1291|Wednesday|1924Q3|N|N|N|2424030|2424273|2423687|2423962|N|N|N|N|N| +2424054|AAAAAAAAGPMPECAA|1924-09-25|296|1291|100|1924|4|9|25|3|1924|100|1291|Thursday|1924Q3|N|N|N|2424030|2424273|2423688|2423963|N|N|N|N|N| +2424055|AAAAAAAAHPMPECAA|1924-09-26|296|1291|100|1924|5|9|26|3|1924|100|1291|Friday|1924Q3|N|Y|N|2424030|2424273|2423689|2423964|N|N|N|N|N| +2424056|AAAAAAAAIPMPECAA|1924-09-27|296|1291|100|1924|6|9|27|3|1924|100|1291|Saturday|1924Q3|N|Y|N|2424030|2424273|2423690|2423965|N|N|N|N|N| +2424057|AAAAAAAAJPMPECAA|1924-09-28|296|1291|100|1924|0|9|28|3|1924|100|1291|Sunday|1924Q3|N|N|N|2424030|2424273|2423691|2423966|N|N|N|N|N| +2424058|AAAAAAAAKPMPECAA|1924-09-29|296|1291|100|1924|1|9|29|3|1924|100|1291|Monday|1924Q3|N|N|N|2424030|2424273|2423692|2423967|N|N|N|N|N| +2424059|AAAAAAAALPMPECAA|1924-09-30|296|1292|100|1924|2|9|30|3|1924|100|1292|Tuesday|1924Q3|N|N|N|2424030|2424273|2423693|2423968|N|N|N|N|N| +2424060|AAAAAAAAMPMPECAA|1924-10-01|297|1292|100|1924|3|10|1|4|1924|100|1292|Wednesday|1924Q4|N|N|N|2424060|2424333|2423694|2423968|N|N|N|N|N| +2424061|AAAAAAAANPMPECAA|1924-10-02|297|1292|100|1924|4|10|2|4|1924|100|1292|Thursday|1924Q4|N|N|N|2424060|2424333|2423695|2423969|N|N|N|N|N| +2424062|AAAAAAAAOPMPECAA|1924-10-03|297|1292|100|1924|5|10|3|4|1924|100|1292|Friday|1924Q4|N|Y|N|2424060|2424333|2423696|2423970|N|N|N|N|N| +2424063|AAAAAAAAPPMPECAA|1924-10-04|297|1292|100|1924|6|10|4|4|1924|100|1292|Saturday|1924Q4|N|Y|N|2424060|2424333|2423697|2423971|N|N|N|N|N| +2424064|AAAAAAAAAANPECAA|1924-10-05|297|1292|100|1924|0|10|5|4|1924|100|1292|Sunday|1924Q4|N|N|N|2424060|2424333|2423698|2423972|N|N|N|N|N| +2424065|AAAAAAAABANPECAA|1924-10-06|297|1292|100|1924|1|10|6|4|1924|100|1292|Monday|1924Q4|N|N|N|2424060|2424333|2423699|2423973|N|N|N|N|N| +2424066|AAAAAAAACANPECAA|1924-10-07|297|1293|100|1924|2|10|7|4|1924|100|1293|Tuesday|1924Q4|N|N|N|2424060|2424333|2423700|2423974|N|N|N|N|N| +2424067|AAAAAAAADANPECAA|1924-10-08|297|1293|100|1924|3|10|8|4|1924|100|1293|Wednesday|1924Q4|N|N|N|2424060|2424333|2423701|2423975|N|N|N|N|N| +2424068|AAAAAAAAEANPECAA|1924-10-09|297|1293|100|1924|4|10|9|4|1924|100|1293|Thursday|1924Q4|N|N|N|2424060|2424333|2423702|2423976|N|N|N|N|N| +2424069|AAAAAAAAFANPECAA|1924-10-10|297|1293|100|1924|5|10|10|4|1924|100|1293|Friday|1924Q4|N|Y|N|2424060|2424333|2423703|2423977|N|N|N|N|N| +2424070|AAAAAAAAGANPECAA|1924-10-11|297|1293|100|1924|6|10|11|4|1924|100|1293|Saturday|1924Q4|N|Y|N|2424060|2424333|2423704|2423978|N|N|N|N|N| +2424071|AAAAAAAAHANPECAA|1924-10-12|297|1293|100|1924|0|10|12|4|1924|100|1293|Sunday|1924Q4|N|N|N|2424060|2424333|2423705|2423979|N|N|N|N|N| +2424072|AAAAAAAAIANPECAA|1924-10-13|297|1293|100|1924|1|10|13|4|1924|100|1293|Monday|1924Q4|N|N|N|2424060|2424333|2423706|2423980|N|N|N|N|N| +2424073|AAAAAAAAJANPECAA|1924-10-14|297|1294|100|1924|2|10|14|4|1924|100|1294|Tuesday|1924Q4|N|N|N|2424060|2424333|2423707|2423981|N|N|N|N|N| +2424074|AAAAAAAAKANPECAA|1924-10-15|297|1294|100|1924|3|10|15|4|1924|100|1294|Wednesday|1924Q4|N|N|N|2424060|2424333|2423708|2423982|N|N|N|N|N| +2424075|AAAAAAAALANPECAA|1924-10-16|297|1294|100|1924|4|10|16|4|1924|100|1294|Thursday|1924Q4|N|N|N|2424060|2424333|2423709|2423983|N|N|N|N|N| +2424076|AAAAAAAAMANPECAA|1924-10-17|297|1294|100|1924|5|10|17|4|1924|100|1294|Friday|1924Q4|N|Y|N|2424060|2424333|2423710|2423984|N|N|N|N|N| +2424077|AAAAAAAANANPECAA|1924-10-18|297|1294|100|1924|6|10|18|4|1924|100|1294|Saturday|1924Q4|N|Y|N|2424060|2424333|2423711|2423985|N|N|N|N|N| +2424078|AAAAAAAAOANPECAA|1924-10-19|297|1294|100|1924|0|10|19|4|1924|100|1294|Sunday|1924Q4|N|N|N|2424060|2424333|2423712|2423986|N|N|N|N|N| +2424079|AAAAAAAAPANPECAA|1924-10-20|297|1294|100|1924|1|10|20|4|1924|100|1294|Monday|1924Q4|N|N|N|2424060|2424333|2423713|2423987|N|N|N|N|N| +2424080|AAAAAAAAABNPECAA|1924-10-21|297|1295|100|1924|2|10|21|4|1924|100|1295|Tuesday|1924Q4|N|N|N|2424060|2424333|2423714|2423988|N|N|N|N|N| +2424081|AAAAAAAABBNPECAA|1924-10-22|297|1295|100|1924|3|10|22|4|1924|100|1295|Wednesday|1924Q4|N|N|N|2424060|2424333|2423715|2423989|N|N|N|N|N| +2424082|AAAAAAAACBNPECAA|1924-10-23|297|1295|100|1924|4|10|23|4|1924|100|1295|Thursday|1924Q4|N|N|N|2424060|2424333|2423716|2423990|N|N|N|N|N| +2424083|AAAAAAAADBNPECAA|1924-10-24|297|1295|100|1924|5|10|24|4|1924|100|1295|Friday|1924Q4|N|Y|N|2424060|2424333|2423717|2423991|N|N|N|N|N| +2424084|AAAAAAAAEBNPECAA|1924-10-25|297|1295|100|1924|6|10|25|4|1924|100|1295|Saturday|1924Q4|N|Y|N|2424060|2424333|2423718|2423992|N|N|N|N|N| +2424085|AAAAAAAAFBNPECAA|1924-10-26|297|1295|100|1924|0|10|26|4|1924|100|1295|Sunday|1924Q4|N|N|N|2424060|2424333|2423719|2423993|N|N|N|N|N| +2424086|AAAAAAAAGBNPECAA|1924-10-27|297|1295|100|1924|1|10|27|4|1924|100|1295|Monday|1924Q4|N|N|N|2424060|2424333|2423720|2423994|N|N|N|N|N| +2424087|AAAAAAAAHBNPECAA|1924-10-28|297|1296|100|1924|2|10|28|4|1924|100|1296|Tuesday|1924Q4|N|N|N|2424060|2424333|2423721|2423995|N|N|N|N|N| +2424088|AAAAAAAAIBNPECAA|1924-10-29|297|1296|100|1924|3|10|29|4|1924|100|1296|Wednesday|1924Q4|N|N|N|2424060|2424333|2423722|2423996|N|N|N|N|N| +2424089|AAAAAAAAJBNPECAA|1924-10-30|297|1296|100|1924|4|10|30|4|1924|100|1296|Thursday|1924Q4|N|N|N|2424060|2424333|2423723|2423997|N|N|N|N|N| +2424090|AAAAAAAAKBNPECAA|1924-10-31|297|1296|100|1924|5|10|31|4|1924|100|1296|Friday|1924Q4|N|Y|N|2424060|2424333|2423724|2423998|N|N|N|N|N| +2424091|AAAAAAAALBNPECAA|1924-11-01|298|1296|100|1924|6|11|1|4|1924|100|1296|Saturday|1924Q4|N|Y|N|2424091|2424395|2423725|2423999|N|N|N|N|N| +2424092|AAAAAAAAMBNPECAA|1924-11-02|298|1296|100|1924|0|11|2|4|1924|100|1296|Sunday|1924Q4|N|N|N|2424091|2424395|2423726|2424000|N|N|N|N|N| +2424093|AAAAAAAANBNPECAA|1924-11-03|298|1296|100|1924|1|11|3|4|1924|100|1296|Monday|1924Q4|N|N|N|2424091|2424395|2423727|2424001|N|N|N|N|N| +2424094|AAAAAAAAOBNPECAA|1924-11-04|298|1297|100|1924|2|11|4|4|1924|100|1297|Tuesday|1924Q4|N|N|N|2424091|2424395|2423728|2424002|N|N|N|N|N| +2424095|AAAAAAAAPBNPECAA|1924-11-05|298|1297|100|1924|3|11|5|4|1924|100|1297|Wednesday|1924Q4|N|N|N|2424091|2424395|2423729|2424003|N|N|N|N|N| +2424096|AAAAAAAAACNPECAA|1924-11-06|298|1297|100|1924|4|11|6|4|1924|100|1297|Thursday|1924Q4|N|N|N|2424091|2424395|2423730|2424004|N|N|N|N|N| +2424097|AAAAAAAABCNPECAA|1924-11-07|298|1297|100|1924|5|11|7|4|1924|100|1297|Friday|1924Q4|N|Y|N|2424091|2424395|2423731|2424005|N|N|N|N|N| +2424098|AAAAAAAACCNPECAA|1924-11-08|298|1297|100|1924|6|11|8|4|1924|100|1297|Saturday|1924Q4|N|Y|N|2424091|2424395|2423732|2424006|N|N|N|N|N| +2424099|AAAAAAAADCNPECAA|1924-11-09|298|1297|100|1924|0|11|9|4|1924|100|1297|Sunday|1924Q4|N|N|N|2424091|2424395|2423733|2424007|N|N|N|N|N| +2424100|AAAAAAAAECNPECAA|1924-11-10|298|1297|100|1924|1|11|10|4|1924|100|1297|Monday|1924Q4|N|N|N|2424091|2424395|2423734|2424008|N|N|N|N|N| +2424101|AAAAAAAAFCNPECAA|1924-11-11|298|1298|100|1924|2|11|11|4|1924|100|1298|Tuesday|1924Q4|N|N|N|2424091|2424395|2423735|2424009|N|N|N|N|N| +2424102|AAAAAAAAGCNPECAA|1924-11-12|298|1298|100|1924|3|11|12|4|1924|100|1298|Wednesday|1924Q4|N|N|N|2424091|2424395|2423736|2424010|N|N|N|N|N| +2424103|AAAAAAAAHCNPECAA|1924-11-13|298|1298|100|1924|4|11|13|4|1924|100|1298|Thursday|1924Q4|N|N|N|2424091|2424395|2423737|2424011|N|N|N|N|N| +2424104|AAAAAAAAICNPECAA|1924-11-14|298|1298|100|1924|5|11|14|4|1924|100|1298|Friday|1924Q4|N|Y|N|2424091|2424395|2423738|2424012|N|N|N|N|N| +2424105|AAAAAAAAJCNPECAA|1924-11-15|298|1298|100|1924|6|11|15|4|1924|100|1298|Saturday|1924Q4|N|Y|N|2424091|2424395|2423739|2424013|N|N|N|N|N| +2424106|AAAAAAAAKCNPECAA|1924-11-16|298|1298|100|1924|0|11|16|4|1924|100|1298|Sunday|1924Q4|N|N|N|2424091|2424395|2423740|2424014|N|N|N|N|N| +2424107|AAAAAAAALCNPECAA|1924-11-17|298|1298|100|1924|1|11|17|4|1924|100|1298|Monday|1924Q4|N|N|N|2424091|2424395|2423741|2424015|N|N|N|N|N| +2424108|AAAAAAAAMCNPECAA|1924-11-18|298|1299|100|1924|2|11|18|4|1924|100|1299|Tuesday|1924Q4|N|N|N|2424091|2424395|2423742|2424016|N|N|N|N|N| +2424109|AAAAAAAANCNPECAA|1924-11-19|298|1299|100|1924|3|11|19|4|1924|100|1299|Wednesday|1924Q4|N|N|N|2424091|2424395|2423743|2424017|N|N|N|N|N| +2424110|AAAAAAAAOCNPECAA|1924-11-20|298|1299|100|1924|4|11|20|4|1924|100|1299|Thursday|1924Q4|N|N|N|2424091|2424395|2423744|2424018|N|N|N|N|N| +2424111|AAAAAAAAPCNPECAA|1924-11-21|298|1299|100|1924|5|11|21|4|1924|100|1299|Friday|1924Q4|N|Y|N|2424091|2424395|2423745|2424019|N|N|N|N|N| +2424112|AAAAAAAAADNPECAA|1924-11-22|298|1299|100|1924|6|11|22|4|1924|100|1299|Saturday|1924Q4|N|Y|N|2424091|2424395|2423746|2424020|N|N|N|N|N| +2424113|AAAAAAAABDNPECAA|1924-11-23|298|1299|100|1924|0|11|23|4|1924|100|1299|Sunday|1924Q4|N|N|N|2424091|2424395|2423747|2424021|N|N|N|N|N| +2424114|AAAAAAAACDNPECAA|1924-11-24|298|1299|100|1924|1|11|24|4|1924|100|1299|Monday|1924Q4|N|N|N|2424091|2424395|2423748|2424022|N|N|N|N|N| +2424115|AAAAAAAADDNPECAA|1924-11-25|298|1300|100|1924|2|11|25|4|1924|100|1300|Tuesday|1924Q4|N|N|N|2424091|2424395|2423749|2424023|N|N|N|N|N| +2424116|AAAAAAAAEDNPECAA|1924-11-26|298|1300|100|1924|3|11|26|4|1924|100|1300|Wednesday|1924Q4|N|N|N|2424091|2424395|2423750|2424024|N|N|N|N|N| +2424117|AAAAAAAAFDNPECAA|1924-11-27|298|1300|100|1924|4|11|27|4|1924|100|1300|Thursday|1924Q4|N|N|N|2424091|2424395|2423751|2424025|N|N|N|N|N| +2424118|AAAAAAAAGDNPECAA|1924-11-28|298|1300|100|1924|5|11|28|4|1924|100|1300|Friday|1924Q4|N|Y|N|2424091|2424395|2423752|2424026|N|N|N|N|N| +2424119|AAAAAAAAHDNPECAA|1924-11-29|298|1300|100|1924|6|11|29|4|1924|100|1300|Saturday|1924Q4|N|Y|N|2424091|2424395|2423753|2424027|N|N|N|N|N| +2424120|AAAAAAAAIDNPECAA|1924-11-30|298|1300|100|1924|0|11|30|4|1924|100|1300|Sunday|1924Q4|N|N|N|2424091|2424395|2423754|2424028|N|N|N|N|N| +2424121|AAAAAAAAJDNPECAA|1924-12-01|299|1300|101|1924|1|12|1|4|1924|101|1300|Monday|1924Q4|N|N|N|2424121|2424455|2423755|2424029|N|N|N|N|N| +2424122|AAAAAAAAKDNPECAA|1924-12-02|299|1301|101|1924|2|12|2|4|1924|101|1301|Tuesday|1924Q4|N|N|N|2424121|2424455|2423756|2424030|N|N|N|N|N| +2424123|AAAAAAAALDNPECAA|1924-12-03|299|1301|101|1924|3|12|3|4|1924|101|1301|Wednesday|1924Q4|N|N|N|2424121|2424455|2423757|2424031|N|N|N|N|N| +2424124|AAAAAAAAMDNPECAA|1924-12-04|299|1301|101|1924|4|12|4|4|1924|101|1301|Thursday|1924Q4|N|N|N|2424121|2424455|2423758|2424032|N|N|N|N|N| +2424125|AAAAAAAANDNPECAA|1924-12-05|299|1301|101|1924|5|12|5|4|1924|101|1301|Friday|1924Q4|N|Y|N|2424121|2424455|2423759|2424033|N|N|N|N|N| +2424126|AAAAAAAAODNPECAA|1924-12-06|299|1301|101|1924|6|12|6|4|1924|101|1301|Saturday|1924Q4|N|Y|N|2424121|2424455|2423760|2424034|N|N|N|N|N| +2424127|AAAAAAAAPDNPECAA|1924-12-07|299|1301|101|1924|0|12|7|4|1924|101|1301|Sunday|1924Q4|N|N|N|2424121|2424455|2423761|2424035|N|N|N|N|N| +2424128|AAAAAAAAAENPECAA|1924-12-08|299|1301|101|1924|1|12|8|4|1924|101|1301|Monday|1924Q4|N|N|N|2424121|2424455|2423762|2424036|N|N|N|N|N| +2424129|AAAAAAAABENPECAA|1924-12-09|299|1302|101|1924|2|12|9|4|1924|101|1302|Tuesday|1924Q4|N|N|N|2424121|2424455|2423763|2424037|N|N|N|N|N| +2424130|AAAAAAAACENPECAA|1924-12-10|299|1302|101|1924|3|12|10|4|1924|101|1302|Wednesday|1924Q4|N|N|N|2424121|2424455|2423764|2424038|N|N|N|N|N| +2424131|AAAAAAAADENPECAA|1924-12-11|299|1302|101|1924|4|12|11|4|1924|101|1302|Thursday|1924Q4|N|N|N|2424121|2424455|2423765|2424039|N|N|N|N|N| +2424132|AAAAAAAAEENPECAA|1924-12-12|299|1302|101|1924|5|12|12|4|1924|101|1302|Friday|1924Q4|N|Y|N|2424121|2424455|2423766|2424040|N|N|N|N|N| +2424133|AAAAAAAAFENPECAA|1924-12-13|299|1302|101|1924|6|12|13|4|1924|101|1302|Saturday|1924Q4|N|Y|N|2424121|2424455|2423767|2424041|N|N|N|N|N| +2424134|AAAAAAAAGENPECAA|1924-12-14|299|1302|101|1924|0|12|14|4|1924|101|1302|Sunday|1924Q4|N|N|N|2424121|2424455|2423768|2424042|N|N|N|N|N| +2424135|AAAAAAAAHENPECAA|1924-12-15|299|1302|101|1924|1|12|15|4|1924|101|1302|Monday|1924Q4|N|N|N|2424121|2424455|2423769|2424043|N|N|N|N|N| +2424136|AAAAAAAAIENPECAA|1924-12-16|299|1303|101|1924|2|12|16|4|1924|101|1303|Tuesday|1924Q4|N|N|N|2424121|2424455|2423770|2424044|N|N|N|N|N| +2424137|AAAAAAAAJENPECAA|1924-12-17|299|1303|101|1924|3|12|17|4|1924|101|1303|Wednesday|1924Q4|N|N|N|2424121|2424455|2423771|2424045|N|N|N|N|N| +2424138|AAAAAAAAKENPECAA|1924-12-18|299|1303|101|1924|4|12|18|4|1924|101|1303|Thursday|1924Q4|N|N|N|2424121|2424455|2423772|2424046|N|N|N|N|N| +2424139|AAAAAAAALENPECAA|1924-12-19|299|1303|101|1924|5|12|19|4|1924|101|1303|Friday|1924Q4|N|Y|N|2424121|2424455|2423773|2424047|N|N|N|N|N| +2424140|AAAAAAAAMENPECAA|1924-12-20|299|1303|101|1924|6|12|20|4|1924|101|1303|Saturday|1924Q4|N|Y|N|2424121|2424455|2423774|2424048|N|N|N|N|N| +2424141|AAAAAAAANENPECAA|1924-12-21|299|1303|101|1924|0|12|21|4|1924|101|1303|Sunday|1924Q4|N|N|N|2424121|2424455|2423775|2424049|N|N|N|N|N| +2424142|AAAAAAAAOENPECAA|1924-12-22|299|1303|101|1924|1|12|22|4|1924|101|1303|Monday|1924Q4|N|N|N|2424121|2424455|2423776|2424050|N|N|N|N|N| +2424143|AAAAAAAAPENPECAA|1924-12-23|299|1304|101|1924|2|12|23|4|1924|101|1304|Tuesday|1924Q4|N|N|N|2424121|2424455|2423777|2424051|N|N|N|N|N| +2424144|AAAAAAAAAFNPECAA|1924-12-24|299|1304|101|1924|3|12|24|4|1924|101|1304|Wednesday|1924Q4|N|N|N|2424121|2424455|2423778|2424052|N|N|N|N|N| +2424145|AAAAAAAABFNPECAA|1924-12-25|299|1304|101|1924|4|12|25|4|1924|101|1304|Thursday|1924Q4|Y|N|N|2424121|2424455|2423779|2424053|N|N|N|N|N| +2424146|AAAAAAAACFNPECAA|1924-12-26|299|1304|101|1924|5|12|26|4|1924|101|1304|Friday|1924Q4|N|Y|Y|2424121|2424455|2423780|2424054|N|N|N|N|N| +2424147|AAAAAAAADFNPECAA|1924-12-27|299|1304|101|1924|6|12|27|4|1924|101|1304|Saturday|1924Q4|N|Y|N|2424121|2424455|2423781|2424055|N|N|N|N|N| +2424148|AAAAAAAAEFNPECAA|1924-12-28|299|1304|101|1924|0|12|28|4|1924|101|1304|Sunday|1924Q4|N|N|N|2424121|2424455|2423782|2424056|N|N|N|N|N| +2424149|AAAAAAAAFFNPECAA|1924-12-29|299|1304|101|1924|1|12|29|4|1924|101|1304|Monday|1924Q4|N|N|N|2424121|2424455|2423783|2424057|N|N|N|N|N| +2424150|AAAAAAAAGFNPECAA|1924-12-30|299|1305|101|1924|2|12|30|4|1924|101|1305|Tuesday|1924Q4|N|N|N|2424121|2424455|2423784|2424058|N|N|N|N|N| +2424151|AAAAAAAAHFNPECAA|1924-12-31|299|1305|101|1924|3|12|31|4|1924|101|1305|Wednesday|1924Q4|Y|N|N|2424121|2424455|2423785|2424059|N|N|N|N|N| +2424152|AAAAAAAAIFNPECAA|1925-01-01|300|1305|101|1925|4|1|1|1|1925|101|1305|Thursday|1925Q1|Y|N|Y|2424152|2424151|2423786|2424060|N|N|N|N|N| +2424153|AAAAAAAAJFNPECAA|1925-01-02|300|1305|101|1925|5|1|2|1|1925|101|1305|Friday|1925Q1|N|Y|Y|2424152|2424151|2423787|2424061|N|N|N|N|N| +2424154|AAAAAAAAKFNPECAA|1925-01-03|300|1305|101|1925|6|1|3|1|1925|101|1305|Saturday|1925Q1|N|Y|N|2424152|2424151|2423788|2424062|N|N|N|N|N| +2424155|AAAAAAAALFNPECAA|1925-01-04|300|1305|101|1925|0|1|4|1|1925|101|1305|Sunday|1925Q1|N|N|N|2424152|2424151|2423789|2424063|N|N|N|N|N| +2424156|AAAAAAAAMFNPECAA|1925-01-05|300|1305|101|1925|1|1|5|1|1925|101|1305|Monday|1925Q1|N|N|N|2424152|2424151|2423790|2424064|N|N|N|N|N| +2424157|AAAAAAAANFNPECAA|1925-01-06|300|1306|101|1925|2|1|6|1|1925|101|1306|Tuesday|1925Q1|N|N|N|2424152|2424151|2423791|2424065|N|N|N|N|N| +2424158|AAAAAAAAOFNPECAA|1925-01-07|300|1306|101|1925|3|1|7|1|1925|101|1306|Wednesday|1925Q1|N|N|N|2424152|2424151|2423792|2424066|N|N|N|N|N| +2424159|AAAAAAAAPFNPECAA|1925-01-08|300|1306|101|1925|4|1|8|1|1925|101|1306|Thursday|1925Q1|N|N|N|2424152|2424151|2423793|2424067|N|N|N|N|N| +2424160|AAAAAAAAAGNPECAA|1925-01-09|300|1306|101|1925|5|1|9|1|1925|101|1306|Friday|1925Q1|N|Y|N|2424152|2424151|2423794|2424068|N|N|N|N|N| +2424161|AAAAAAAABGNPECAA|1925-01-10|300|1306|101|1925|6|1|10|1|1925|101|1306|Saturday|1925Q1|N|Y|N|2424152|2424151|2423795|2424069|N|N|N|N|N| +2424162|AAAAAAAACGNPECAA|1925-01-11|300|1306|101|1925|0|1|11|1|1925|101|1306|Sunday|1925Q1|N|N|N|2424152|2424151|2423796|2424070|N|N|N|N|N| +2424163|AAAAAAAADGNPECAA|1925-01-12|300|1306|101|1925|1|1|12|1|1925|101|1306|Monday|1925Q1|N|N|N|2424152|2424151|2423797|2424071|N|N|N|N|N| +2424164|AAAAAAAAEGNPECAA|1925-01-13|300|1307|101|1925|2|1|13|1|1925|101|1307|Tuesday|1925Q1|N|N|N|2424152|2424151|2423798|2424072|N|N|N|N|N| +2424165|AAAAAAAAFGNPECAA|1925-01-14|300|1307|101|1925|3|1|14|1|1925|101|1307|Wednesday|1925Q1|N|N|N|2424152|2424151|2423799|2424073|N|N|N|N|N| +2424166|AAAAAAAAGGNPECAA|1925-01-15|300|1307|101|1925|4|1|15|1|1925|101|1307|Thursday|1925Q1|N|N|N|2424152|2424151|2423800|2424074|N|N|N|N|N| +2424167|AAAAAAAAHGNPECAA|1925-01-16|300|1307|101|1925|5|1|16|1|1925|101|1307|Friday|1925Q1|N|Y|N|2424152|2424151|2423801|2424075|N|N|N|N|N| +2424168|AAAAAAAAIGNPECAA|1925-01-17|300|1307|101|1925|6|1|17|1|1925|101|1307|Saturday|1925Q1|N|Y|N|2424152|2424151|2423802|2424076|N|N|N|N|N| +2424169|AAAAAAAAJGNPECAA|1925-01-18|300|1307|101|1925|0|1|18|1|1925|101|1307|Sunday|1925Q1|N|N|N|2424152|2424151|2423803|2424077|N|N|N|N|N| +2424170|AAAAAAAAKGNPECAA|1925-01-19|300|1307|101|1925|1|1|19|1|1925|101|1307|Monday|1925Q1|N|N|N|2424152|2424151|2423804|2424078|N|N|N|N|N| +2424171|AAAAAAAALGNPECAA|1925-01-20|300|1308|101|1925|2|1|20|1|1925|101|1308|Tuesday|1925Q1|N|N|N|2424152|2424151|2423805|2424079|N|N|N|N|N| +2424172|AAAAAAAAMGNPECAA|1925-01-21|300|1308|101|1925|3|1|21|1|1925|101|1308|Wednesday|1925Q1|N|N|N|2424152|2424151|2423806|2424080|N|N|N|N|N| +2424173|AAAAAAAANGNPECAA|1925-01-22|300|1308|101|1925|4|1|22|1|1925|101|1308|Thursday|1925Q1|N|N|N|2424152|2424151|2423807|2424081|N|N|N|N|N| +2424174|AAAAAAAAOGNPECAA|1925-01-23|300|1308|101|1925|5|1|23|1|1925|101|1308|Friday|1925Q1|N|Y|N|2424152|2424151|2423808|2424082|N|N|N|N|N| +2424175|AAAAAAAAPGNPECAA|1925-01-24|300|1308|101|1925|6|1|24|1|1925|101|1308|Saturday|1925Q1|N|Y|N|2424152|2424151|2423809|2424083|N|N|N|N|N| +2424176|AAAAAAAAAHNPECAA|1925-01-25|300|1308|101|1925|0|1|25|1|1925|101|1308|Sunday|1925Q1|N|N|N|2424152|2424151|2423810|2424084|N|N|N|N|N| +2424177|AAAAAAAABHNPECAA|1925-01-26|300|1308|101|1925|1|1|26|1|1925|101|1308|Monday|1925Q1|N|N|N|2424152|2424151|2423811|2424085|N|N|N|N|N| +2424178|AAAAAAAACHNPECAA|1925-01-27|300|1309|101|1925|2|1|27|1|1925|101|1309|Tuesday|1925Q1|N|N|N|2424152|2424151|2423812|2424086|N|N|N|N|N| +2424179|AAAAAAAADHNPECAA|1925-01-28|300|1309|101|1925|3|1|28|1|1925|101|1309|Wednesday|1925Q1|N|N|N|2424152|2424151|2423813|2424087|N|N|N|N|N| +2424180|AAAAAAAAEHNPECAA|1925-01-29|300|1309|101|1925|4|1|29|1|1925|101|1309|Thursday|1925Q1|N|N|N|2424152|2424151|2423814|2424088|N|N|N|N|N| +2424181|AAAAAAAAFHNPECAA|1925-01-30|300|1309|101|1925|5|1|30|1|1925|101|1309|Friday|1925Q1|N|Y|N|2424152|2424151|2423815|2424089|N|N|N|N|N| +2424182|AAAAAAAAGHNPECAA|1925-01-31|300|1309|101|1925|6|1|31|1|1925|101|1309|Saturday|1925Q1|N|Y|N|2424152|2424151|2423816|2424090|N|N|N|N|N| +2424183|AAAAAAAAHHNPECAA|1925-02-01|301|1309|101|1925|0|2|1|1|1925|101|1309|Sunday|1925Q1|N|N|N|2424183|2424213|2423817|2424091|N|N|N|N|N| +2424184|AAAAAAAAIHNPECAA|1925-02-02|301|1309|101|1925|1|2|2|1|1925|101|1309|Monday|1925Q1|N|N|N|2424183|2424213|2423818|2424092|N|N|N|N|N| +2424185|AAAAAAAAJHNPECAA|1925-02-03|301|1310|101|1925|2|2|3|1|1925|101|1310|Tuesday|1925Q1|N|N|N|2424183|2424213|2423819|2424093|N|N|N|N|N| +2424186|AAAAAAAAKHNPECAA|1925-02-04|301|1310|101|1925|3|2|4|1|1925|101|1310|Wednesday|1925Q1|N|N|N|2424183|2424213|2423820|2424094|N|N|N|N|N| +2424187|AAAAAAAALHNPECAA|1925-02-05|301|1310|101|1925|4|2|5|1|1925|101|1310|Thursday|1925Q1|N|N|N|2424183|2424213|2423821|2424095|N|N|N|N|N| +2424188|AAAAAAAAMHNPECAA|1925-02-06|301|1310|101|1925|5|2|6|1|1925|101|1310|Friday|1925Q1|N|Y|N|2424183|2424213|2423822|2424096|N|N|N|N|N| +2424189|AAAAAAAANHNPECAA|1925-02-07|301|1310|101|1925|6|2|7|1|1925|101|1310|Saturday|1925Q1|N|Y|N|2424183|2424213|2423823|2424097|N|N|N|N|N| +2424190|AAAAAAAAOHNPECAA|1925-02-08|301|1310|101|1925|0|2|8|1|1925|101|1310|Sunday|1925Q1|N|N|N|2424183|2424213|2423824|2424098|N|N|N|N|N| +2424191|AAAAAAAAPHNPECAA|1925-02-09|301|1310|101|1925|1|2|9|1|1925|101|1310|Monday|1925Q1|N|N|N|2424183|2424213|2423825|2424099|N|N|N|N|N| +2424192|AAAAAAAAAINPECAA|1925-02-10|301|1311|101|1925|2|2|10|1|1925|101|1311|Tuesday|1925Q1|N|N|N|2424183|2424213|2423826|2424100|N|N|N|N|N| +2424193|AAAAAAAABINPECAA|1925-02-11|301|1311|101|1925|3|2|11|1|1925|101|1311|Wednesday|1925Q1|N|N|N|2424183|2424213|2423827|2424101|N|N|N|N|N| +2424194|AAAAAAAACINPECAA|1925-02-12|301|1311|101|1925|4|2|12|1|1925|101|1311|Thursday|1925Q1|N|N|N|2424183|2424213|2423828|2424102|N|N|N|N|N| +2424195|AAAAAAAADINPECAA|1925-02-13|301|1311|101|1925|5|2|13|1|1925|101|1311|Friday|1925Q1|N|Y|N|2424183|2424213|2423829|2424103|N|N|N|N|N| +2424196|AAAAAAAAEINPECAA|1925-02-14|301|1311|101|1925|6|2|14|1|1925|101|1311|Saturday|1925Q1|N|Y|N|2424183|2424213|2423830|2424104|N|N|N|N|N| +2424197|AAAAAAAAFINPECAA|1925-02-15|301|1311|101|1925|0|2|15|1|1925|101|1311|Sunday|1925Q1|N|N|N|2424183|2424213|2423831|2424105|N|N|N|N|N| +2424198|AAAAAAAAGINPECAA|1925-02-16|301|1311|101|1925|1|2|16|1|1925|101|1311|Monday|1925Q1|N|N|N|2424183|2424213|2423832|2424106|N|N|N|N|N| +2424199|AAAAAAAAHINPECAA|1925-02-17|301|1312|101|1925|2|2|17|1|1925|101|1312|Tuesday|1925Q1|N|N|N|2424183|2424213|2423833|2424107|N|N|N|N|N| +2424200|AAAAAAAAIINPECAA|1925-02-18|301|1312|101|1925|3|2|18|1|1925|101|1312|Wednesday|1925Q1|N|N|N|2424183|2424213|2423834|2424108|N|N|N|N|N| +2424201|AAAAAAAAJINPECAA|1925-02-19|301|1312|101|1925|4|2|19|1|1925|101|1312|Thursday|1925Q1|N|N|N|2424183|2424213|2423835|2424109|N|N|N|N|N| +2424202|AAAAAAAAKINPECAA|1925-02-20|301|1312|101|1925|5|2|20|1|1925|101|1312|Friday|1925Q1|N|Y|N|2424183|2424213|2423836|2424110|N|N|N|N|N| +2424203|AAAAAAAALINPECAA|1925-02-21|301|1312|101|1925|6|2|21|1|1925|101|1312|Saturday|1925Q1|N|Y|N|2424183|2424213|2423837|2424111|N|N|N|N|N| +2424204|AAAAAAAAMINPECAA|1925-02-22|301|1312|101|1925|0|2|22|1|1925|101|1312|Sunday|1925Q1|N|N|N|2424183|2424213|2423838|2424112|N|N|N|N|N| +2424205|AAAAAAAANINPECAA|1925-02-23|301|1312|101|1925|1|2|23|1|1925|101|1312|Monday|1925Q1|N|N|N|2424183|2424213|2423839|2424113|N|N|N|N|N| +2424206|AAAAAAAAOINPECAA|1925-02-24|301|1313|101|1925|2|2|24|1|1925|101|1313|Tuesday|1925Q1|N|N|N|2424183|2424213|2423840|2424114|N|N|N|N|N| +2424207|AAAAAAAAPINPECAA|1925-02-25|301|1313|101|1925|3|2|25|1|1925|101|1313|Wednesday|1925Q1|N|N|N|2424183|2424213|2423841|2424115|N|N|N|N|N| +2424208|AAAAAAAAAJNPECAA|1925-02-26|301|1313|101|1925|4|2|26|1|1925|101|1313|Thursday|1925Q1|N|N|N|2424183|2424213|2423842|2424116|N|N|N|N|N| +2424209|AAAAAAAABJNPECAA|1925-02-27|301|1313|101|1925|5|2|27|1|1925|101|1313|Friday|1925Q1|N|Y|N|2424183|2424213|2423843|2424117|N|N|N|N|N| +2424210|AAAAAAAACJNPECAA|1925-02-28|301|1313|101|1925|6|2|28|1|1925|101|1313|Saturday|1925Q1|N|Y|N|2424183|2424213|2423844|2424118|N|N|N|N|N| +2424211|AAAAAAAADJNPECAA|1925-03-01|302|1313|102|1925|0|3|1|1|1925|102|1313|Sunday|1925Q1|N|N|N|2424211|2424269|2423846|2424119|N|N|N|N|N| +2424212|AAAAAAAAEJNPECAA|1925-03-02|302|1313|102|1925|1|3|2|1|1925|102|1313|Monday|1925Q1|N|N|N|2424211|2424269|2423847|2424120|N|N|N|N|N| +2424213|AAAAAAAAFJNPECAA|1925-03-03|302|1314|102|1925|2|3|3|1|1925|102|1314|Tuesday|1925Q1|N|N|N|2424211|2424269|2423848|2424121|N|N|N|N|N| +2424214|AAAAAAAAGJNPECAA|1925-03-04|302|1314|102|1925|3|3|4|1|1925|102|1314|Wednesday|1925Q1|N|N|N|2424211|2424269|2423849|2424122|N|N|N|N|N| +2424215|AAAAAAAAHJNPECAA|1925-03-05|302|1314|102|1925|4|3|5|1|1925|102|1314|Thursday|1925Q1|N|N|N|2424211|2424269|2423850|2424123|N|N|N|N|N| +2424216|AAAAAAAAIJNPECAA|1925-03-06|302|1314|102|1925|5|3|6|1|1925|102|1314|Friday|1925Q1|N|Y|N|2424211|2424269|2423851|2424124|N|N|N|N|N| +2424217|AAAAAAAAJJNPECAA|1925-03-07|302|1314|102|1925|6|3|7|1|1925|102|1314|Saturday|1925Q1|N|Y|N|2424211|2424269|2423852|2424125|N|N|N|N|N| +2424218|AAAAAAAAKJNPECAA|1925-03-08|302|1314|102|1925|0|3|8|1|1925|102|1314|Sunday|1925Q1|N|N|N|2424211|2424269|2423853|2424126|N|N|N|N|N| +2424219|AAAAAAAALJNPECAA|1925-03-09|302|1314|102|1925|1|3|9|1|1925|102|1314|Monday|1925Q1|N|N|N|2424211|2424269|2423854|2424127|N|N|N|N|N| +2424220|AAAAAAAAMJNPECAA|1925-03-10|302|1315|102|1925|2|3|10|1|1925|102|1315|Tuesday|1925Q1|N|N|N|2424211|2424269|2423855|2424128|N|N|N|N|N| +2424221|AAAAAAAANJNPECAA|1925-03-11|302|1315|102|1925|3|3|11|1|1925|102|1315|Wednesday|1925Q1|N|N|N|2424211|2424269|2423856|2424129|N|N|N|N|N| +2424222|AAAAAAAAOJNPECAA|1925-03-12|302|1315|102|1925|4|3|12|1|1925|102|1315|Thursday|1925Q1|N|N|N|2424211|2424269|2423857|2424130|N|N|N|N|N| +2424223|AAAAAAAAPJNPECAA|1925-03-13|302|1315|102|1925|5|3|13|1|1925|102|1315|Friday|1925Q1|N|Y|N|2424211|2424269|2423858|2424131|N|N|N|N|N| +2424224|AAAAAAAAAKNPECAA|1925-03-14|302|1315|102|1925|6|3|14|1|1925|102|1315|Saturday|1925Q1|N|Y|N|2424211|2424269|2423859|2424132|N|N|N|N|N| +2424225|AAAAAAAABKNPECAA|1925-03-15|302|1315|102|1925|0|3|15|1|1925|102|1315|Sunday|1925Q1|N|N|N|2424211|2424269|2423860|2424133|N|N|N|N|N| +2424226|AAAAAAAACKNPECAA|1925-03-16|302|1315|102|1925|1|3|16|1|1925|102|1315|Monday|1925Q1|N|N|N|2424211|2424269|2423861|2424134|N|N|N|N|N| +2424227|AAAAAAAADKNPECAA|1925-03-17|302|1316|102|1925|2|3|17|1|1925|102|1316|Tuesday|1925Q1|N|N|N|2424211|2424269|2423862|2424135|N|N|N|N|N| +2424228|AAAAAAAAEKNPECAA|1925-03-18|302|1316|102|1925|3|3|18|1|1925|102|1316|Wednesday|1925Q1|N|N|N|2424211|2424269|2423863|2424136|N|N|N|N|N| +2424229|AAAAAAAAFKNPECAA|1925-03-19|302|1316|102|1925|4|3|19|1|1925|102|1316|Thursday|1925Q1|N|N|N|2424211|2424269|2423864|2424137|N|N|N|N|N| +2424230|AAAAAAAAGKNPECAA|1925-03-20|302|1316|102|1925|5|3|20|1|1925|102|1316|Friday|1925Q1|N|Y|N|2424211|2424269|2423865|2424138|N|N|N|N|N| +2424231|AAAAAAAAHKNPECAA|1925-03-21|302|1316|102|1925|6|3|21|1|1925|102|1316|Saturday|1925Q1|N|Y|N|2424211|2424269|2423866|2424139|N|N|N|N|N| +2424232|AAAAAAAAIKNPECAA|1925-03-22|302|1316|102|1925|0|3|22|1|1925|102|1316|Sunday|1925Q1|N|N|N|2424211|2424269|2423867|2424140|N|N|N|N|N| +2424233|AAAAAAAAJKNPECAA|1925-03-23|302|1316|102|1925|1|3|23|1|1925|102|1316|Monday|1925Q1|N|N|N|2424211|2424269|2423868|2424141|N|N|N|N|N| +2424234|AAAAAAAAKKNPECAA|1925-03-24|302|1317|102|1925|2|3|24|1|1925|102|1317|Tuesday|1925Q1|N|N|N|2424211|2424269|2423869|2424142|N|N|N|N|N| +2424235|AAAAAAAALKNPECAA|1925-03-25|302|1317|102|1925|3|3|25|1|1925|102|1317|Wednesday|1925Q1|N|N|N|2424211|2424269|2423870|2424143|N|N|N|N|N| +2424236|AAAAAAAAMKNPECAA|1925-03-26|302|1317|102|1925|4|3|26|1|1925|102|1317|Thursday|1925Q1|N|N|N|2424211|2424269|2423871|2424144|N|N|N|N|N| +2424237|AAAAAAAANKNPECAA|1925-03-27|302|1317|102|1925|5|3|27|1|1925|102|1317|Friday|1925Q1|N|Y|N|2424211|2424269|2423872|2424145|N|N|N|N|N| +2424238|AAAAAAAAOKNPECAA|1925-03-28|302|1317|102|1925|6|3|28|1|1925|102|1317|Saturday|1925Q1|N|Y|N|2424211|2424269|2423873|2424146|N|N|N|N|N| +2424239|AAAAAAAAPKNPECAA|1925-03-29|302|1317|102|1925|0|3|29|1|1925|102|1317|Sunday|1925Q1|N|N|N|2424211|2424269|2423874|2424147|N|N|N|N|N| +2424240|AAAAAAAAALNPECAA|1925-03-30|302|1317|102|1925|1|3|30|1|1925|102|1317|Monday|1925Q1|N|N|N|2424211|2424269|2423875|2424148|N|N|N|N|N| +2424241|AAAAAAAABLNPECAA|1925-03-31|302|1318|102|1925|2|3|31|1|1925|102|1318|Tuesday|1925Q1|N|N|N|2424211|2424269|2423876|2424149|N|N|N|N|N| +2424242|AAAAAAAACLNPECAA|1925-04-01|303|1318|102|1925|3|4|1|1|1925|102|1318|Wednesday|1925Q1|N|N|N|2424242|2424331|2423877|2424152|N|N|N|N|N| +2424243|AAAAAAAADLNPECAA|1925-04-02|303|1318|102|1925|4|4|2|2|1925|102|1318|Thursday|1925Q2|N|N|N|2424242|2424331|2423878|2424153|N|N|N|N|N| +2424244|AAAAAAAAELNPECAA|1925-04-03|303|1318|102|1925|5|4|3|2|1925|102|1318|Friday|1925Q2|N|Y|N|2424242|2424331|2423879|2424154|N|N|N|N|N| +2424245|AAAAAAAAFLNPECAA|1925-04-04|303|1318|102|1925|6|4|4|2|1925|102|1318|Saturday|1925Q2|N|Y|N|2424242|2424331|2423880|2424155|N|N|N|N|N| +2424246|AAAAAAAAGLNPECAA|1925-04-05|303|1318|102|1925|0|4|5|2|1925|102|1318|Sunday|1925Q2|N|N|N|2424242|2424331|2423881|2424156|N|N|N|N|N| +2424247|AAAAAAAAHLNPECAA|1925-04-06|303|1318|102|1925|1|4|6|2|1925|102|1318|Monday|1925Q2|N|N|N|2424242|2424331|2423882|2424157|N|N|N|N|N| +2424248|AAAAAAAAILNPECAA|1925-04-07|303|1319|102|1925|2|4|7|2|1925|102|1319|Tuesday|1925Q2|N|N|N|2424242|2424331|2423883|2424158|N|N|N|N|N| +2424249|AAAAAAAAJLNPECAA|1925-04-08|303|1319|102|1925|3|4|8|2|1925|102|1319|Wednesday|1925Q2|N|N|N|2424242|2424331|2423884|2424159|N|N|N|N|N| +2424250|AAAAAAAAKLNPECAA|1925-04-09|303|1319|102|1925|4|4|9|2|1925|102|1319|Thursday|1925Q2|N|N|N|2424242|2424331|2423885|2424160|N|N|N|N|N| +2424251|AAAAAAAALLNPECAA|1925-04-10|303|1319|102|1925|5|4|10|2|1925|102|1319|Friday|1925Q2|N|Y|N|2424242|2424331|2423886|2424161|N|N|N|N|N| +2424252|AAAAAAAAMLNPECAA|1925-04-11|303|1319|102|1925|6|4|11|2|1925|102|1319|Saturday|1925Q2|N|Y|N|2424242|2424331|2423887|2424162|N|N|N|N|N| +2424253|AAAAAAAANLNPECAA|1925-04-12|303|1319|102|1925|0|4|12|2|1925|102|1319|Sunday|1925Q2|N|N|N|2424242|2424331|2423888|2424163|N|N|N|N|N| +2424254|AAAAAAAAOLNPECAA|1925-04-13|303|1319|102|1925|1|4|13|2|1925|102|1319|Monday|1925Q2|N|N|N|2424242|2424331|2423889|2424164|N|N|N|N|N| +2424255|AAAAAAAAPLNPECAA|1925-04-14|303|1320|102|1925|2|4|14|2|1925|102|1320|Tuesday|1925Q2|N|N|N|2424242|2424331|2423890|2424165|N|N|N|N|N| +2424256|AAAAAAAAAMNPECAA|1925-04-15|303|1320|102|1925|3|4|15|2|1925|102|1320|Wednesday|1925Q2|N|N|N|2424242|2424331|2423891|2424166|N|N|N|N|N| +2424257|AAAAAAAABMNPECAA|1925-04-16|303|1320|102|1925|4|4|16|2|1925|102|1320|Thursday|1925Q2|N|N|N|2424242|2424331|2423892|2424167|N|N|N|N|N| +2424258|AAAAAAAACMNPECAA|1925-04-17|303|1320|102|1925|5|4|17|2|1925|102|1320|Friday|1925Q2|N|Y|N|2424242|2424331|2423893|2424168|N|N|N|N|N| +2424259|AAAAAAAADMNPECAA|1925-04-18|303|1320|102|1925|6|4|18|2|1925|102|1320|Saturday|1925Q2|N|Y|N|2424242|2424331|2423894|2424169|N|N|N|N|N| +2424260|AAAAAAAAEMNPECAA|1925-04-19|303|1320|102|1925|0|4|19|2|1925|102|1320|Sunday|1925Q2|N|N|N|2424242|2424331|2423895|2424170|N|N|N|N|N| +2424261|AAAAAAAAFMNPECAA|1925-04-20|303|1320|102|1925|1|4|20|2|1925|102|1320|Monday|1925Q2|N|N|N|2424242|2424331|2423896|2424171|N|N|N|N|N| +2424262|AAAAAAAAGMNPECAA|1925-04-21|303|1321|102|1925|2|4|21|2|1925|102|1321|Tuesday|1925Q2|N|N|N|2424242|2424331|2423897|2424172|N|N|N|N|N| +2424263|AAAAAAAAHMNPECAA|1925-04-22|303|1321|102|1925|3|4|22|2|1925|102|1321|Wednesday|1925Q2|N|N|N|2424242|2424331|2423898|2424173|N|N|N|N|N| +2424264|AAAAAAAAIMNPECAA|1925-04-23|303|1321|102|1925|4|4|23|2|1925|102|1321|Thursday|1925Q2|N|N|N|2424242|2424331|2423899|2424174|N|N|N|N|N| +2424265|AAAAAAAAJMNPECAA|1925-04-24|303|1321|102|1925|5|4|24|2|1925|102|1321|Friday|1925Q2|N|Y|N|2424242|2424331|2423900|2424175|N|N|N|N|N| +2424266|AAAAAAAAKMNPECAA|1925-04-25|303|1321|102|1925|6|4|25|2|1925|102|1321|Saturday|1925Q2|N|Y|N|2424242|2424331|2423901|2424176|N|N|N|N|N| +2424267|AAAAAAAALMNPECAA|1925-04-26|303|1321|102|1925|0|4|26|2|1925|102|1321|Sunday|1925Q2|N|N|N|2424242|2424331|2423902|2424177|N|N|N|N|N| +2424268|AAAAAAAAMMNPECAA|1925-04-27|303|1321|102|1925|1|4|27|2|1925|102|1321|Monday|1925Q2|N|N|N|2424242|2424331|2423903|2424178|N|N|N|N|N| +2424269|AAAAAAAANMNPECAA|1925-04-28|303|1322|102|1925|2|4|28|2|1925|102|1322|Tuesday|1925Q2|N|N|N|2424242|2424331|2423904|2424179|N|N|N|N|N| +2424270|AAAAAAAAOMNPECAA|1925-04-29|303|1322|102|1925|3|4|29|2|1925|102|1322|Wednesday|1925Q2|N|N|N|2424242|2424331|2423905|2424180|N|N|N|N|N| +2424271|AAAAAAAAPMNPECAA|1925-04-30|303|1322|102|1925|4|4|30|2|1925|102|1322|Thursday|1925Q2|N|N|N|2424242|2424331|2423906|2424181|N|N|N|N|N| +2424272|AAAAAAAAANNPECAA|1925-05-01|304|1322|102|1925|5|5|1|2|1925|102|1322|Friday|1925Q2|N|Y|N|2424272|2424391|2423907|2424182|N|N|N|N|N| +2424273|AAAAAAAABNNPECAA|1925-05-02|304|1322|102|1925|6|5|2|2|1925|102|1322|Saturday|1925Q2|N|Y|N|2424272|2424391|2423908|2424183|N|N|N|N|N| +2424274|AAAAAAAACNNPECAA|1925-05-03|304|1322|102|1925|0|5|3|2|1925|102|1322|Sunday|1925Q2|N|N|N|2424272|2424391|2423909|2424184|N|N|N|N|N| +2424275|AAAAAAAADNNPECAA|1925-05-04|304|1322|102|1925|1|5|4|2|1925|102|1322|Monday|1925Q2|N|N|N|2424272|2424391|2423910|2424185|N|N|N|N|N| +2424276|AAAAAAAAENNPECAA|1925-05-05|304|1323|102|1925|2|5|5|2|1925|102|1323|Tuesday|1925Q2|N|N|N|2424272|2424391|2423911|2424186|N|N|N|N|N| +2424277|AAAAAAAAFNNPECAA|1925-05-06|304|1323|102|1925|3|5|6|2|1925|102|1323|Wednesday|1925Q2|N|N|N|2424272|2424391|2423912|2424187|N|N|N|N|N| +2424278|AAAAAAAAGNNPECAA|1925-05-07|304|1323|102|1925|4|5|7|2|1925|102|1323|Thursday|1925Q2|N|N|N|2424272|2424391|2423913|2424188|N|N|N|N|N| +2424279|AAAAAAAAHNNPECAA|1925-05-08|304|1323|102|1925|5|5|8|2|1925|102|1323|Friday|1925Q2|N|Y|N|2424272|2424391|2423914|2424189|N|N|N|N|N| +2424280|AAAAAAAAINNPECAA|1925-05-09|304|1323|102|1925|6|5|9|2|1925|102|1323|Saturday|1925Q2|N|Y|N|2424272|2424391|2423915|2424190|N|N|N|N|N| +2424281|AAAAAAAAJNNPECAA|1925-05-10|304|1323|102|1925|0|5|10|2|1925|102|1323|Sunday|1925Q2|N|N|N|2424272|2424391|2423916|2424191|N|N|N|N|N| +2424282|AAAAAAAAKNNPECAA|1925-05-11|304|1323|102|1925|1|5|11|2|1925|102|1323|Monday|1925Q2|N|N|N|2424272|2424391|2423917|2424192|N|N|N|N|N| +2424283|AAAAAAAALNNPECAA|1925-05-12|304|1324|102|1925|2|5|12|2|1925|102|1324|Tuesday|1925Q2|N|N|N|2424272|2424391|2423918|2424193|N|N|N|N|N| +2424284|AAAAAAAAMNNPECAA|1925-05-13|304|1324|102|1925|3|5|13|2|1925|102|1324|Wednesday|1925Q2|N|N|N|2424272|2424391|2423919|2424194|N|N|N|N|N| +2424285|AAAAAAAANNNPECAA|1925-05-14|304|1324|102|1925|4|5|14|2|1925|102|1324|Thursday|1925Q2|N|N|N|2424272|2424391|2423920|2424195|N|N|N|N|N| +2424286|AAAAAAAAONNPECAA|1925-05-15|304|1324|102|1925|5|5|15|2|1925|102|1324|Friday|1925Q2|N|Y|N|2424272|2424391|2423921|2424196|N|N|N|N|N| +2424287|AAAAAAAAPNNPECAA|1925-05-16|304|1324|102|1925|6|5|16|2|1925|102|1324|Saturday|1925Q2|N|Y|N|2424272|2424391|2423922|2424197|N|N|N|N|N| +2424288|AAAAAAAAAONPECAA|1925-05-17|304|1324|102|1925|0|5|17|2|1925|102|1324|Sunday|1925Q2|N|N|N|2424272|2424391|2423923|2424198|N|N|N|N|N| +2424289|AAAAAAAABONPECAA|1925-05-18|304|1324|102|1925|1|5|18|2|1925|102|1324|Monday|1925Q2|N|N|N|2424272|2424391|2423924|2424199|N|N|N|N|N| +2424290|AAAAAAAACONPECAA|1925-05-19|304|1325|102|1925|2|5|19|2|1925|102|1325|Tuesday|1925Q2|N|N|N|2424272|2424391|2423925|2424200|N|N|N|N|N| +2424291|AAAAAAAADONPECAA|1925-05-20|304|1325|102|1925|3|5|20|2|1925|102|1325|Wednesday|1925Q2|N|N|N|2424272|2424391|2423926|2424201|N|N|N|N|N| +2424292|AAAAAAAAEONPECAA|1925-05-21|304|1325|102|1925|4|5|21|2|1925|102|1325|Thursday|1925Q2|N|N|N|2424272|2424391|2423927|2424202|N|N|N|N|N| +2424293|AAAAAAAAFONPECAA|1925-05-22|304|1325|102|1925|5|5|22|2|1925|102|1325|Friday|1925Q2|N|Y|N|2424272|2424391|2423928|2424203|N|N|N|N|N| +2424294|AAAAAAAAGONPECAA|1925-05-23|304|1325|102|1925|6|5|23|2|1925|102|1325|Saturday|1925Q2|N|Y|N|2424272|2424391|2423929|2424204|N|N|N|N|N| +2424295|AAAAAAAAHONPECAA|1925-05-24|304|1325|102|1925|0|5|24|2|1925|102|1325|Sunday|1925Q2|N|N|N|2424272|2424391|2423930|2424205|N|N|N|N|N| +2424296|AAAAAAAAIONPECAA|1925-05-25|304|1325|102|1925|1|5|25|2|1925|102|1325|Monday|1925Q2|N|N|N|2424272|2424391|2423931|2424206|N|N|N|N|N| +2424297|AAAAAAAAJONPECAA|1925-05-26|304|1326|102|1925|2|5|26|2|1925|102|1326|Tuesday|1925Q2|N|N|N|2424272|2424391|2423932|2424207|N|N|N|N|N| +2424298|AAAAAAAAKONPECAA|1925-05-27|304|1326|102|1925|3|5|27|2|1925|102|1326|Wednesday|1925Q2|N|N|N|2424272|2424391|2423933|2424208|N|N|N|N|N| +2424299|AAAAAAAALONPECAA|1925-05-28|304|1326|102|1925|4|5|28|2|1925|102|1326|Thursday|1925Q2|N|N|N|2424272|2424391|2423934|2424209|N|N|N|N|N| +2424300|AAAAAAAAMONPECAA|1925-05-29|304|1326|102|1925|5|5|29|2|1925|102|1326|Friday|1925Q2|N|Y|N|2424272|2424391|2423935|2424210|N|N|N|N|N| +2424301|AAAAAAAANONPECAA|1925-05-30|304|1326|102|1925|6|5|30|2|1925|102|1326|Saturday|1925Q2|N|Y|N|2424272|2424391|2423936|2424211|N|N|N|N|N| +2424302|AAAAAAAAOONPECAA|1925-05-31|304|1326|102|1925|0|5|31|2|1925|102|1326|Sunday|1925Q2|N|N|N|2424272|2424391|2423937|2424212|N|N|N|N|N| +2424303|AAAAAAAAPONPECAA|1925-06-01|305|1326|103|1925|1|6|1|2|1925|103|1326|Monday|1925Q2|N|N|N|2424303|2424453|2423938|2424213|N|N|N|N|N| +2424304|AAAAAAAAAPNPECAA|1925-06-02|305|1327|103|1925|2|6|2|2|1925|103|1327|Tuesday|1925Q2|N|N|N|2424303|2424453|2423939|2424214|N|N|N|N|N| +2424305|AAAAAAAABPNPECAA|1925-06-03|305|1327|103|1925|3|6|3|2|1925|103|1327|Wednesday|1925Q2|N|N|N|2424303|2424453|2423940|2424215|N|N|N|N|N| +2424306|AAAAAAAACPNPECAA|1925-06-04|305|1327|103|1925|4|6|4|2|1925|103|1327|Thursday|1925Q2|N|N|N|2424303|2424453|2423941|2424216|N|N|N|N|N| +2424307|AAAAAAAADPNPECAA|1925-06-05|305|1327|103|1925|5|6|5|2|1925|103|1327|Friday|1925Q2|N|Y|N|2424303|2424453|2423942|2424217|N|N|N|N|N| +2424308|AAAAAAAAEPNPECAA|1925-06-06|305|1327|103|1925|6|6|6|2|1925|103|1327|Saturday|1925Q2|N|Y|N|2424303|2424453|2423943|2424218|N|N|N|N|N| +2424309|AAAAAAAAFPNPECAA|1925-06-07|305|1327|103|1925|0|6|7|2|1925|103|1327|Sunday|1925Q2|N|N|N|2424303|2424453|2423944|2424219|N|N|N|N|N| +2424310|AAAAAAAAGPNPECAA|1925-06-08|305|1327|103|1925|1|6|8|2|1925|103|1327|Monday|1925Q2|N|N|N|2424303|2424453|2423945|2424220|N|N|N|N|N| +2424311|AAAAAAAAHPNPECAA|1925-06-09|305|1328|103|1925|2|6|9|2|1925|103|1328|Tuesday|1925Q2|N|N|N|2424303|2424453|2423946|2424221|N|N|N|N|N| +2424312|AAAAAAAAIPNPECAA|1925-06-10|305|1328|103|1925|3|6|10|2|1925|103|1328|Wednesday|1925Q2|N|N|N|2424303|2424453|2423947|2424222|N|N|N|N|N| +2424313|AAAAAAAAJPNPECAA|1925-06-11|305|1328|103|1925|4|6|11|2|1925|103|1328|Thursday|1925Q2|N|N|N|2424303|2424453|2423948|2424223|N|N|N|N|N| +2424314|AAAAAAAAKPNPECAA|1925-06-12|305|1328|103|1925|5|6|12|2|1925|103|1328|Friday|1925Q2|N|Y|N|2424303|2424453|2423949|2424224|N|N|N|N|N| +2424315|AAAAAAAALPNPECAA|1925-06-13|305|1328|103|1925|6|6|13|2|1925|103|1328|Saturday|1925Q2|N|Y|N|2424303|2424453|2423950|2424225|N|N|N|N|N| +2424316|AAAAAAAAMPNPECAA|1925-06-14|305|1328|103|1925|0|6|14|2|1925|103|1328|Sunday|1925Q2|N|N|N|2424303|2424453|2423951|2424226|N|N|N|N|N| +2424317|AAAAAAAANPNPECAA|1925-06-15|305|1328|103|1925|1|6|15|2|1925|103|1328|Monday|1925Q2|N|N|N|2424303|2424453|2423952|2424227|N|N|N|N|N| +2424318|AAAAAAAAOPNPECAA|1925-06-16|305|1329|103|1925|2|6|16|2|1925|103|1329|Tuesday|1925Q2|N|N|N|2424303|2424453|2423953|2424228|N|N|N|N|N| +2424319|AAAAAAAAPPNPECAA|1925-06-17|305|1329|103|1925|3|6|17|2|1925|103|1329|Wednesday|1925Q2|N|N|N|2424303|2424453|2423954|2424229|N|N|N|N|N| +2424320|AAAAAAAAAAOPECAA|1925-06-18|305|1329|103|1925|4|6|18|2|1925|103|1329|Thursday|1925Q2|N|N|N|2424303|2424453|2423955|2424230|N|N|N|N|N| +2424321|AAAAAAAABAOPECAA|1925-06-19|305|1329|103|1925|5|6|19|2|1925|103|1329|Friday|1925Q2|N|Y|N|2424303|2424453|2423956|2424231|N|N|N|N|N| +2424322|AAAAAAAACAOPECAA|1925-06-20|305|1329|103|1925|6|6|20|2|1925|103|1329|Saturday|1925Q2|N|Y|N|2424303|2424453|2423957|2424232|N|N|N|N|N| +2424323|AAAAAAAADAOPECAA|1925-06-21|305|1329|103|1925|0|6|21|2|1925|103|1329|Sunday|1925Q2|N|N|N|2424303|2424453|2423958|2424233|N|N|N|N|N| +2424324|AAAAAAAAEAOPECAA|1925-06-22|305|1329|103|1925|1|6|22|2|1925|103|1329|Monday|1925Q2|N|N|N|2424303|2424453|2423959|2424234|N|N|N|N|N| +2424325|AAAAAAAAFAOPECAA|1925-06-23|305|1330|103|1925|2|6|23|2|1925|103|1330|Tuesday|1925Q2|N|N|N|2424303|2424453|2423960|2424235|N|N|N|N|N| +2424326|AAAAAAAAGAOPECAA|1925-06-24|305|1330|103|1925|3|6|24|2|1925|103|1330|Wednesday|1925Q2|N|N|N|2424303|2424453|2423961|2424236|N|N|N|N|N| +2424327|AAAAAAAAHAOPECAA|1925-06-25|305|1330|103|1925|4|6|25|2|1925|103|1330|Thursday|1925Q2|N|N|N|2424303|2424453|2423962|2424237|N|N|N|N|N| +2424328|AAAAAAAAIAOPECAA|1925-06-26|305|1330|103|1925|5|6|26|2|1925|103|1330|Friday|1925Q2|N|Y|N|2424303|2424453|2423963|2424238|N|N|N|N|N| +2424329|AAAAAAAAJAOPECAA|1925-06-27|305|1330|103|1925|6|6|27|2|1925|103|1330|Saturday|1925Q2|N|Y|N|2424303|2424453|2423964|2424239|N|N|N|N|N| +2424330|AAAAAAAAKAOPECAA|1925-06-28|305|1330|103|1925|0|6|28|2|1925|103|1330|Sunday|1925Q2|N|N|N|2424303|2424453|2423965|2424240|N|N|N|N|N| +2424331|AAAAAAAALAOPECAA|1925-06-29|305|1330|103|1925|1|6|29|2|1925|103|1330|Monday|1925Q2|N|N|N|2424303|2424453|2423966|2424241|N|N|N|N|N| +2424332|AAAAAAAAMAOPECAA|1925-06-30|305|1331|103|1925|2|6|30|2|1925|103|1331|Tuesday|1925Q2|N|N|N|2424303|2424453|2423967|2424242|N|N|N|N|N| +2424333|AAAAAAAANAOPECAA|1925-07-01|306|1331|103|1925|3|7|1|2|1925|103|1331|Wednesday|1925Q2|N|N|N|2424333|2424513|2423968|2424242|N|N|N|N|N| +2424334|AAAAAAAAOAOPECAA|1925-07-02|306|1331|103|1925|4|7|2|3|1925|103|1331|Thursday|1925Q3|N|N|N|2424333|2424513|2423969|2424243|N|N|N|N|N| +2424335|AAAAAAAAPAOPECAA|1925-07-03|306|1331|103|1925|5|7|3|3|1925|103|1331|Friday|1925Q3|N|Y|N|2424333|2424513|2423970|2424244|N|N|N|N|N| +2424336|AAAAAAAAABOPECAA|1925-07-04|306|1331|103|1925|6|7|4|3|1925|103|1331|Saturday|1925Q3|N|Y|N|2424333|2424513|2423971|2424245|N|N|N|N|N| +2424337|AAAAAAAABBOPECAA|1925-07-05|306|1331|103|1925|0|7|5|3|1925|103|1331|Sunday|1925Q3|Y|N|N|2424333|2424513|2423972|2424246|N|N|N|N|N| +2424338|AAAAAAAACBOPECAA|1925-07-06|306|1331|103|1925|1|7|6|3|1925|103|1331|Monday|1925Q3|N|N|Y|2424333|2424513|2423973|2424247|N|N|N|N|N| +2424339|AAAAAAAADBOPECAA|1925-07-07|306|1332|103|1925|2|7|7|3|1925|103|1332|Tuesday|1925Q3|N|N|N|2424333|2424513|2423974|2424248|N|N|N|N|N| +2424340|AAAAAAAAEBOPECAA|1925-07-08|306|1332|103|1925|3|7|8|3|1925|103|1332|Wednesday|1925Q3|N|N|N|2424333|2424513|2423975|2424249|N|N|N|N|N| +2424341|AAAAAAAAFBOPECAA|1925-07-09|306|1332|103|1925|4|7|9|3|1925|103|1332|Thursday|1925Q3|N|N|N|2424333|2424513|2423976|2424250|N|N|N|N|N| +2424342|AAAAAAAAGBOPECAA|1925-07-10|306|1332|103|1925|5|7|10|3|1925|103|1332|Friday|1925Q3|N|Y|N|2424333|2424513|2423977|2424251|N|N|N|N|N| +2424343|AAAAAAAAHBOPECAA|1925-07-11|306|1332|103|1925|6|7|11|3|1925|103|1332|Saturday|1925Q3|N|Y|N|2424333|2424513|2423978|2424252|N|N|N|N|N| +2424344|AAAAAAAAIBOPECAA|1925-07-12|306|1332|103|1925|0|7|12|3|1925|103|1332|Sunday|1925Q3|N|N|N|2424333|2424513|2423979|2424253|N|N|N|N|N| +2424345|AAAAAAAAJBOPECAA|1925-07-13|306|1332|103|1925|1|7|13|3|1925|103|1332|Monday|1925Q3|N|N|N|2424333|2424513|2423980|2424254|N|N|N|N|N| +2424346|AAAAAAAAKBOPECAA|1925-07-14|306|1333|103|1925|2|7|14|3|1925|103|1333|Tuesday|1925Q3|N|N|N|2424333|2424513|2423981|2424255|N|N|N|N|N| +2424347|AAAAAAAALBOPECAA|1925-07-15|306|1333|103|1925|3|7|15|3|1925|103|1333|Wednesday|1925Q3|N|N|N|2424333|2424513|2423982|2424256|N|N|N|N|N| +2424348|AAAAAAAAMBOPECAA|1925-07-16|306|1333|103|1925|4|7|16|3|1925|103|1333|Thursday|1925Q3|N|N|N|2424333|2424513|2423983|2424257|N|N|N|N|N| +2424349|AAAAAAAANBOPECAA|1925-07-17|306|1333|103|1925|5|7|17|3|1925|103|1333|Friday|1925Q3|N|Y|N|2424333|2424513|2423984|2424258|N|N|N|N|N| +2424350|AAAAAAAAOBOPECAA|1925-07-18|306|1333|103|1925|6|7|18|3|1925|103|1333|Saturday|1925Q3|N|Y|N|2424333|2424513|2423985|2424259|N|N|N|N|N| +2424351|AAAAAAAAPBOPECAA|1925-07-19|306|1333|103|1925|0|7|19|3|1925|103|1333|Sunday|1925Q3|N|N|N|2424333|2424513|2423986|2424260|N|N|N|N|N| +2424352|AAAAAAAAACOPECAA|1925-07-20|306|1333|103|1925|1|7|20|3|1925|103|1333|Monday|1925Q3|N|N|N|2424333|2424513|2423987|2424261|N|N|N|N|N| +2424353|AAAAAAAABCOPECAA|1925-07-21|306|1334|103|1925|2|7|21|3|1925|103|1334|Tuesday|1925Q3|N|N|N|2424333|2424513|2423988|2424262|N|N|N|N|N| +2424354|AAAAAAAACCOPECAA|1925-07-22|306|1334|103|1925|3|7|22|3|1925|103|1334|Wednesday|1925Q3|N|N|N|2424333|2424513|2423989|2424263|N|N|N|N|N| +2424355|AAAAAAAADCOPECAA|1925-07-23|306|1334|103|1925|4|7|23|3|1925|103|1334|Thursday|1925Q3|N|N|N|2424333|2424513|2423990|2424264|N|N|N|N|N| +2424356|AAAAAAAAECOPECAA|1925-07-24|306|1334|103|1925|5|7|24|3|1925|103|1334|Friday|1925Q3|N|Y|N|2424333|2424513|2423991|2424265|N|N|N|N|N| +2424357|AAAAAAAAFCOPECAA|1925-07-25|306|1334|103|1925|6|7|25|3|1925|103|1334|Saturday|1925Q3|N|Y|N|2424333|2424513|2423992|2424266|N|N|N|N|N| +2424358|AAAAAAAAGCOPECAA|1925-07-26|306|1334|103|1925|0|7|26|3|1925|103|1334|Sunday|1925Q3|N|N|N|2424333|2424513|2423993|2424267|N|N|N|N|N| +2424359|AAAAAAAAHCOPECAA|1925-07-27|306|1334|103|1925|1|7|27|3|1925|103|1334|Monday|1925Q3|N|N|N|2424333|2424513|2423994|2424268|N|N|N|N|N| +2424360|AAAAAAAAICOPECAA|1925-07-28|306|1335|103|1925|2|7|28|3|1925|103|1335|Tuesday|1925Q3|N|N|N|2424333|2424513|2423995|2424269|N|N|N|N|N| +2424361|AAAAAAAAJCOPECAA|1925-07-29|306|1335|103|1925|3|7|29|3|1925|103|1335|Wednesday|1925Q3|N|N|N|2424333|2424513|2423996|2424270|N|N|N|N|N| +2424362|AAAAAAAAKCOPECAA|1925-07-30|306|1335|103|1925|4|7|30|3|1925|103|1335|Thursday|1925Q3|N|N|N|2424333|2424513|2423997|2424271|N|N|N|N|N| +2424363|AAAAAAAALCOPECAA|1925-07-31|306|1335|103|1925|5|7|31|3|1925|103|1335|Friday|1925Q3|N|Y|N|2424333|2424513|2423998|2424272|N|N|N|N|N| +2424364|AAAAAAAAMCOPECAA|1925-08-01|307|1335|103|1925|6|8|1|3|1925|103|1335|Saturday|1925Q3|N|Y|N|2424364|2424575|2423999|2424273|N|N|N|N|N| +2424365|AAAAAAAANCOPECAA|1925-08-02|307|1335|103|1925|0|8|2|3|1925|103|1335|Sunday|1925Q3|N|N|N|2424364|2424575|2424000|2424274|N|N|N|N|N| +2424366|AAAAAAAAOCOPECAA|1925-08-03|307|1335|103|1925|1|8|3|3|1925|103|1335|Monday|1925Q3|N|N|N|2424364|2424575|2424001|2424275|N|N|N|N|N| +2424367|AAAAAAAAPCOPECAA|1925-08-04|307|1336|103|1925|2|8|4|3|1925|103|1336|Tuesday|1925Q3|N|N|N|2424364|2424575|2424002|2424276|N|N|N|N|N| +2424368|AAAAAAAAADOPECAA|1925-08-05|307|1336|103|1925|3|8|5|3|1925|103|1336|Wednesday|1925Q3|N|N|N|2424364|2424575|2424003|2424277|N|N|N|N|N| +2424369|AAAAAAAABDOPECAA|1925-08-06|307|1336|103|1925|4|8|6|3|1925|103|1336|Thursday|1925Q3|N|N|N|2424364|2424575|2424004|2424278|N|N|N|N|N| +2424370|AAAAAAAACDOPECAA|1925-08-07|307|1336|103|1925|5|8|7|3|1925|103|1336|Friday|1925Q3|N|Y|N|2424364|2424575|2424005|2424279|N|N|N|N|N| +2424371|AAAAAAAADDOPECAA|1925-08-08|307|1336|103|1925|6|8|8|3|1925|103|1336|Saturday|1925Q3|N|Y|N|2424364|2424575|2424006|2424280|N|N|N|N|N| +2424372|AAAAAAAAEDOPECAA|1925-08-09|307|1336|103|1925|0|8|9|3|1925|103|1336|Sunday|1925Q3|N|N|N|2424364|2424575|2424007|2424281|N|N|N|N|N| +2424373|AAAAAAAAFDOPECAA|1925-08-10|307|1336|103|1925|1|8|10|3|1925|103|1336|Monday|1925Q3|N|N|N|2424364|2424575|2424008|2424282|N|N|N|N|N| +2424374|AAAAAAAAGDOPECAA|1925-08-11|307|1337|103|1925|2|8|11|3|1925|103|1337|Tuesday|1925Q3|N|N|N|2424364|2424575|2424009|2424283|N|N|N|N|N| +2424375|AAAAAAAAHDOPECAA|1925-08-12|307|1337|103|1925|3|8|12|3|1925|103|1337|Wednesday|1925Q3|N|N|N|2424364|2424575|2424010|2424284|N|N|N|N|N| +2424376|AAAAAAAAIDOPECAA|1925-08-13|307|1337|103|1925|4|8|13|3|1925|103|1337|Thursday|1925Q3|N|N|N|2424364|2424575|2424011|2424285|N|N|N|N|N| +2424377|AAAAAAAAJDOPECAA|1925-08-14|307|1337|103|1925|5|8|14|3|1925|103|1337|Friday|1925Q3|N|Y|N|2424364|2424575|2424012|2424286|N|N|N|N|N| +2424378|AAAAAAAAKDOPECAA|1925-08-15|307|1337|103|1925|6|8|15|3|1925|103|1337|Saturday|1925Q3|N|Y|N|2424364|2424575|2424013|2424287|N|N|N|N|N| +2424379|AAAAAAAALDOPECAA|1925-08-16|307|1337|103|1925|0|8|16|3|1925|103|1337|Sunday|1925Q3|N|N|N|2424364|2424575|2424014|2424288|N|N|N|N|N| +2424380|AAAAAAAAMDOPECAA|1925-08-17|307|1337|103|1925|1|8|17|3|1925|103|1337|Monday|1925Q3|N|N|N|2424364|2424575|2424015|2424289|N|N|N|N|N| +2424381|AAAAAAAANDOPECAA|1925-08-18|307|1338|103|1925|2|8|18|3|1925|103|1338|Tuesday|1925Q3|N|N|N|2424364|2424575|2424016|2424290|N|N|N|N|N| +2424382|AAAAAAAAODOPECAA|1925-08-19|307|1338|103|1925|3|8|19|3|1925|103|1338|Wednesday|1925Q3|N|N|N|2424364|2424575|2424017|2424291|N|N|N|N|N| +2424383|AAAAAAAAPDOPECAA|1925-08-20|307|1338|103|1925|4|8|20|3|1925|103|1338|Thursday|1925Q3|N|N|N|2424364|2424575|2424018|2424292|N|N|N|N|N| +2424384|AAAAAAAAAEOPECAA|1925-08-21|307|1338|103|1925|5|8|21|3|1925|103|1338|Friday|1925Q3|N|Y|N|2424364|2424575|2424019|2424293|N|N|N|N|N| +2424385|AAAAAAAABEOPECAA|1925-08-22|307|1338|103|1925|6|8|22|3|1925|103|1338|Saturday|1925Q3|N|Y|N|2424364|2424575|2424020|2424294|N|N|N|N|N| +2424386|AAAAAAAACEOPECAA|1925-08-23|307|1338|103|1925|0|8|23|3|1925|103|1338|Sunday|1925Q3|N|N|N|2424364|2424575|2424021|2424295|N|N|N|N|N| +2424387|AAAAAAAADEOPECAA|1925-08-24|307|1338|103|1925|1|8|24|3|1925|103|1338|Monday|1925Q3|N|N|N|2424364|2424575|2424022|2424296|N|N|N|N|N| +2424388|AAAAAAAAEEOPECAA|1925-08-25|307|1339|103|1925|2|8|25|3|1925|103|1339|Tuesday|1925Q3|N|N|N|2424364|2424575|2424023|2424297|N|N|N|N|N| +2424389|AAAAAAAAFEOPECAA|1925-08-26|307|1339|103|1925|3|8|26|3|1925|103|1339|Wednesday|1925Q3|N|N|N|2424364|2424575|2424024|2424298|N|N|N|N|N| +2424390|AAAAAAAAGEOPECAA|1925-08-27|307|1339|103|1925|4|8|27|3|1925|103|1339|Thursday|1925Q3|N|N|N|2424364|2424575|2424025|2424299|N|N|N|N|N| +2424391|AAAAAAAAHEOPECAA|1925-08-28|307|1339|103|1925|5|8|28|3|1925|103|1339|Friday|1925Q3|N|Y|N|2424364|2424575|2424026|2424300|N|N|N|N|N| +2424392|AAAAAAAAIEOPECAA|1925-08-29|307|1339|103|1925|6|8|29|3|1925|103|1339|Saturday|1925Q3|N|Y|N|2424364|2424575|2424027|2424301|N|N|N|N|N| +2424393|AAAAAAAAJEOPECAA|1925-08-30|307|1339|103|1925|0|8|30|3|1925|103|1339|Sunday|1925Q3|N|N|N|2424364|2424575|2424028|2424302|N|N|N|N|N| +2424394|AAAAAAAAKEOPECAA|1925-08-31|307|1339|103|1925|1|8|31|3|1925|103|1339|Monday|1925Q3|N|N|N|2424364|2424575|2424029|2424303|N|N|N|N|N| +2424395|AAAAAAAALEOPECAA|1925-09-01|308|1340|104|1925|2|9|1|3|1925|104|1340|Tuesday|1925Q3|N|N|N|2424395|2424637|2424030|2424304|N|N|N|N|N| +2424396|AAAAAAAAMEOPECAA|1925-09-02|308|1340|104|1925|3|9|2|3|1925|104|1340|Wednesday|1925Q3|N|N|N|2424395|2424637|2424031|2424305|N|N|N|N|N| +2424397|AAAAAAAANEOPECAA|1925-09-03|308|1340|104|1925|4|9|3|3|1925|104|1340|Thursday|1925Q3|N|N|N|2424395|2424637|2424032|2424306|N|N|N|N|N| +2424398|AAAAAAAAOEOPECAA|1925-09-04|308|1340|104|1925|5|9|4|3|1925|104|1340|Friday|1925Q3|N|Y|N|2424395|2424637|2424033|2424307|N|N|N|N|N| +2424399|AAAAAAAAPEOPECAA|1925-09-05|308|1340|104|1925|6|9|5|3|1925|104|1340|Saturday|1925Q3|N|Y|N|2424395|2424637|2424034|2424308|N|N|N|N|N| +2424400|AAAAAAAAAFOPECAA|1925-09-06|308|1340|104|1925|0|9|6|3|1925|104|1340|Sunday|1925Q3|N|N|N|2424395|2424637|2424035|2424309|N|N|N|N|N| +2424401|AAAAAAAABFOPECAA|1925-09-07|308|1340|104|1925|1|9|7|3|1925|104|1340|Monday|1925Q3|N|N|N|2424395|2424637|2424036|2424310|N|N|N|N|N| +2424402|AAAAAAAACFOPECAA|1925-09-08|308|1341|104|1925|2|9|8|3|1925|104|1341|Tuesday|1925Q3|N|N|N|2424395|2424637|2424037|2424311|N|N|N|N|N| +2424403|AAAAAAAADFOPECAA|1925-09-09|308|1341|104|1925|3|9|9|3|1925|104|1341|Wednesday|1925Q3|N|N|N|2424395|2424637|2424038|2424312|N|N|N|N|N| +2424404|AAAAAAAAEFOPECAA|1925-09-10|308|1341|104|1925|4|9|10|3|1925|104|1341|Thursday|1925Q3|N|N|N|2424395|2424637|2424039|2424313|N|N|N|N|N| +2424405|AAAAAAAAFFOPECAA|1925-09-11|308|1341|104|1925|5|9|11|3|1925|104|1341|Friday|1925Q3|N|Y|N|2424395|2424637|2424040|2424314|N|N|N|N|N| +2424406|AAAAAAAAGFOPECAA|1925-09-12|308|1341|104|1925|6|9|12|3|1925|104|1341|Saturday|1925Q3|N|Y|N|2424395|2424637|2424041|2424315|N|N|N|N|N| +2424407|AAAAAAAAHFOPECAA|1925-09-13|308|1341|104|1925|0|9|13|3|1925|104|1341|Sunday|1925Q3|N|N|N|2424395|2424637|2424042|2424316|N|N|N|N|N| +2424408|AAAAAAAAIFOPECAA|1925-09-14|308|1341|104|1925|1|9|14|3|1925|104|1341|Monday|1925Q3|N|N|N|2424395|2424637|2424043|2424317|N|N|N|N|N| +2424409|AAAAAAAAJFOPECAA|1925-09-15|308|1342|104|1925|2|9|15|3|1925|104|1342|Tuesday|1925Q3|N|N|N|2424395|2424637|2424044|2424318|N|N|N|N|N| +2424410|AAAAAAAAKFOPECAA|1925-09-16|308|1342|104|1925|3|9|16|3|1925|104|1342|Wednesday|1925Q3|N|N|N|2424395|2424637|2424045|2424319|N|N|N|N|N| +2424411|AAAAAAAALFOPECAA|1925-09-17|308|1342|104|1925|4|9|17|3|1925|104|1342|Thursday|1925Q3|N|N|N|2424395|2424637|2424046|2424320|N|N|N|N|N| +2424412|AAAAAAAAMFOPECAA|1925-09-18|308|1342|104|1925|5|9|18|3|1925|104|1342|Friday|1925Q3|N|Y|N|2424395|2424637|2424047|2424321|N|N|N|N|N| +2424413|AAAAAAAANFOPECAA|1925-09-19|308|1342|104|1925|6|9|19|3|1925|104|1342|Saturday|1925Q3|N|Y|N|2424395|2424637|2424048|2424322|N|N|N|N|N| +2424414|AAAAAAAAOFOPECAA|1925-09-20|308|1342|104|1925|0|9|20|3|1925|104|1342|Sunday|1925Q3|N|N|N|2424395|2424637|2424049|2424323|N|N|N|N|N| +2424415|AAAAAAAAPFOPECAA|1925-09-21|308|1342|104|1925|1|9|21|3|1925|104|1342|Monday|1925Q3|N|N|N|2424395|2424637|2424050|2424324|N|N|N|N|N| +2424416|AAAAAAAAAGOPECAA|1925-09-22|308|1343|104|1925|2|9|22|3|1925|104|1343|Tuesday|1925Q3|N|N|N|2424395|2424637|2424051|2424325|N|N|N|N|N| +2424417|AAAAAAAABGOPECAA|1925-09-23|308|1343|104|1925|3|9|23|3|1925|104|1343|Wednesday|1925Q3|N|N|N|2424395|2424637|2424052|2424326|N|N|N|N|N| +2424418|AAAAAAAACGOPECAA|1925-09-24|308|1343|104|1925|4|9|24|3|1925|104|1343|Thursday|1925Q3|N|N|N|2424395|2424637|2424053|2424327|N|N|N|N|N| +2424419|AAAAAAAADGOPECAA|1925-09-25|308|1343|104|1925|5|9|25|3|1925|104|1343|Friday|1925Q3|N|Y|N|2424395|2424637|2424054|2424328|N|N|N|N|N| +2424420|AAAAAAAAEGOPECAA|1925-09-26|308|1343|104|1925|6|9|26|3|1925|104|1343|Saturday|1925Q3|N|Y|N|2424395|2424637|2424055|2424329|N|N|N|N|N| +2424421|AAAAAAAAFGOPECAA|1925-09-27|308|1343|104|1925|0|9|27|3|1925|104|1343|Sunday|1925Q3|N|N|N|2424395|2424637|2424056|2424330|N|N|N|N|N| +2424422|AAAAAAAAGGOPECAA|1925-09-28|308|1343|104|1925|1|9|28|3|1925|104|1343|Monday|1925Q3|N|N|N|2424395|2424637|2424057|2424331|N|N|N|N|N| +2424423|AAAAAAAAHGOPECAA|1925-09-29|308|1344|104|1925|2|9|29|3|1925|104|1344|Tuesday|1925Q3|N|N|N|2424395|2424637|2424058|2424332|N|N|N|N|N| +2424424|AAAAAAAAIGOPECAA|1925-09-30|308|1344|104|1925|3|9|30|3|1925|104|1344|Wednesday|1925Q3|N|N|N|2424395|2424637|2424059|2424333|N|N|N|N|N| +2424425|AAAAAAAAJGOPECAA|1925-10-01|309|1344|104|1925|4|10|1|3|1925|104|1344|Thursday|1925Q3|N|N|N|2424425|2424697|2424060|2424333|N|N|N|N|N| +2424426|AAAAAAAAKGOPECAA|1925-10-02|309|1344|104|1925|5|10|2|4|1925|104|1344|Friday|1925Q4|N|Y|N|2424425|2424697|2424061|2424334|N|N|N|N|N| +2424427|AAAAAAAALGOPECAA|1925-10-03|309|1344|104|1925|6|10|3|4|1925|104|1344|Saturday|1925Q4|N|Y|N|2424425|2424697|2424062|2424335|N|N|N|N|N| +2424428|AAAAAAAAMGOPECAA|1925-10-04|309|1344|104|1925|0|10|4|4|1925|104|1344|Sunday|1925Q4|N|N|N|2424425|2424697|2424063|2424336|N|N|N|N|N| +2424429|AAAAAAAANGOPECAA|1925-10-05|309|1344|104|1925|1|10|5|4|1925|104|1344|Monday|1925Q4|N|N|N|2424425|2424697|2424064|2424337|N|N|N|N|N| +2424430|AAAAAAAAOGOPECAA|1925-10-06|309|1345|104|1925|2|10|6|4|1925|104|1345|Tuesday|1925Q4|N|N|N|2424425|2424697|2424065|2424338|N|N|N|N|N| +2424431|AAAAAAAAPGOPECAA|1925-10-07|309|1345|104|1925|3|10|7|4|1925|104|1345|Wednesday|1925Q4|N|N|N|2424425|2424697|2424066|2424339|N|N|N|N|N| +2424432|AAAAAAAAAHOPECAA|1925-10-08|309|1345|104|1925|4|10|8|4|1925|104|1345|Thursday|1925Q4|N|N|N|2424425|2424697|2424067|2424340|N|N|N|N|N| +2424433|AAAAAAAABHOPECAA|1925-10-09|309|1345|104|1925|5|10|9|4|1925|104|1345|Friday|1925Q4|N|Y|N|2424425|2424697|2424068|2424341|N|N|N|N|N| +2424434|AAAAAAAACHOPECAA|1925-10-10|309|1345|104|1925|6|10|10|4|1925|104|1345|Saturday|1925Q4|N|Y|N|2424425|2424697|2424069|2424342|N|N|N|N|N| +2424435|AAAAAAAADHOPECAA|1925-10-11|309|1345|104|1925|0|10|11|4|1925|104|1345|Sunday|1925Q4|N|N|N|2424425|2424697|2424070|2424343|N|N|N|N|N| +2424436|AAAAAAAAEHOPECAA|1925-10-12|309|1345|104|1925|1|10|12|4|1925|104|1345|Monday|1925Q4|N|N|N|2424425|2424697|2424071|2424344|N|N|N|N|N| +2424437|AAAAAAAAFHOPECAA|1925-10-13|309|1346|104|1925|2|10|13|4|1925|104|1346|Tuesday|1925Q4|N|N|N|2424425|2424697|2424072|2424345|N|N|N|N|N| +2424438|AAAAAAAAGHOPECAA|1925-10-14|309|1346|104|1925|3|10|14|4|1925|104|1346|Wednesday|1925Q4|N|N|N|2424425|2424697|2424073|2424346|N|N|N|N|N| +2424439|AAAAAAAAHHOPECAA|1925-10-15|309|1346|104|1925|4|10|15|4|1925|104|1346|Thursday|1925Q4|N|N|N|2424425|2424697|2424074|2424347|N|N|N|N|N| +2424440|AAAAAAAAIHOPECAA|1925-10-16|309|1346|104|1925|5|10|16|4|1925|104|1346|Friday|1925Q4|N|Y|N|2424425|2424697|2424075|2424348|N|N|N|N|N| +2424441|AAAAAAAAJHOPECAA|1925-10-17|309|1346|104|1925|6|10|17|4|1925|104|1346|Saturday|1925Q4|N|Y|N|2424425|2424697|2424076|2424349|N|N|N|N|N| +2424442|AAAAAAAAKHOPECAA|1925-10-18|309|1346|104|1925|0|10|18|4|1925|104|1346|Sunday|1925Q4|N|N|N|2424425|2424697|2424077|2424350|N|N|N|N|N| +2424443|AAAAAAAALHOPECAA|1925-10-19|309|1346|104|1925|1|10|19|4|1925|104|1346|Monday|1925Q4|N|N|N|2424425|2424697|2424078|2424351|N|N|N|N|N| +2424444|AAAAAAAAMHOPECAA|1925-10-20|309|1347|104|1925|2|10|20|4|1925|104|1347|Tuesday|1925Q4|N|N|N|2424425|2424697|2424079|2424352|N|N|N|N|N| +2424445|AAAAAAAANHOPECAA|1925-10-21|309|1347|104|1925|3|10|21|4|1925|104|1347|Wednesday|1925Q4|N|N|N|2424425|2424697|2424080|2424353|N|N|N|N|N| +2424446|AAAAAAAAOHOPECAA|1925-10-22|309|1347|104|1925|4|10|22|4|1925|104|1347|Thursday|1925Q4|N|N|N|2424425|2424697|2424081|2424354|N|N|N|N|N| +2424447|AAAAAAAAPHOPECAA|1925-10-23|309|1347|104|1925|5|10|23|4|1925|104|1347|Friday|1925Q4|N|Y|N|2424425|2424697|2424082|2424355|N|N|N|N|N| +2424448|AAAAAAAAAIOPECAA|1925-10-24|309|1347|104|1925|6|10|24|4|1925|104|1347|Saturday|1925Q4|N|Y|N|2424425|2424697|2424083|2424356|N|N|N|N|N| +2424449|AAAAAAAABIOPECAA|1925-10-25|309|1347|104|1925|0|10|25|4|1925|104|1347|Sunday|1925Q4|N|N|N|2424425|2424697|2424084|2424357|N|N|N|N|N| +2424450|AAAAAAAACIOPECAA|1925-10-26|309|1347|104|1925|1|10|26|4|1925|104|1347|Monday|1925Q4|N|N|N|2424425|2424697|2424085|2424358|N|N|N|N|N| +2424451|AAAAAAAADIOPECAA|1925-10-27|309|1348|104|1925|2|10|27|4|1925|104|1348|Tuesday|1925Q4|N|N|N|2424425|2424697|2424086|2424359|N|N|N|N|N| +2424452|AAAAAAAAEIOPECAA|1925-10-28|309|1348|104|1925|3|10|28|4|1925|104|1348|Wednesday|1925Q4|N|N|N|2424425|2424697|2424087|2424360|N|N|N|N|N| +2424453|AAAAAAAAFIOPECAA|1925-10-29|309|1348|104|1925|4|10|29|4|1925|104|1348|Thursday|1925Q4|N|N|N|2424425|2424697|2424088|2424361|N|N|N|N|N| +2424454|AAAAAAAAGIOPECAA|1925-10-30|309|1348|104|1925|5|10|30|4|1925|104|1348|Friday|1925Q4|N|Y|N|2424425|2424697|2424089|2424362|N|N|N|N|N| +2424455|AAAAAAAAHIOPECAA|1925-10-31|309|1348|104|1925|6|10|31|4|1925|104|1348|Saturday|1925Q4|N|Y|N|2424425|2424697|2424090|2424363|N|N|N|N|N| +2424456|AAAAAAAAIIOPECAA|1925-11-01|310|1348|104|1925|0|11|1|4|1925|104|1348|Sunday|1925Q4|N|N|N|2424456|2424759|2424091|2424364|N|N|N|N|N| +2424457|AAAAAAAAJIOPECAA|1925-11-02|310|1348|104|1925|1|11|2|4|1925|104|1348|Monday|1925Q4|N|N|N|2424456|2424759|2424092|2424365|N|N|N|N|N| +2424458|AAAAAAAAKIOPECAA|1925-11-03|310|1349|104|1925|2|11|3|4|1925|104|1349|Tuesday|1925Q4|N|N|N|2424456|2424759|2424093|2424366|N|N|N|N|N| +2424459|AAAAAAAALIOPECAA|1925-11-04|310|1349|104|1925|3|11|4|4|1925|104|1349|Wednesday|1925Q4|N|N|N|2424456|2424759|2424094|2424367|N|N|N|N|N| +2424460|AAAAAAAAMIOPECAA|1925-11-05|310|1349|104|1925|4|11|5|4|1925|104|1349|Thursday|1925Q4|N|N|N|2424456|2424759|2424095|2424368|N|N|N|N|N| +2424461|AAAAAAAANIOPECAA|1925-11-06|310|1349|104|1925|5|11|6|4|1925|104|1349|Friday|1925Q4|N|Y|N|2424456|2424759|2424096|2424369|N|N|N|N|N| +2424462|AAAAAAAAOIOPECAA|1925-11-07|310|1349|104|1925|6|11|7|4|1925|104|1349|Saturday|1925Q4|N|Y|N|2424456|2424759|2424097|2424370|N|N|N|N|N| +2424463|AAAAAAAAPIOPECAA|1925-11-08|310|1349|104|1925|0|11|8|4|1925|104|1349|Sunday|1925Q4|N|N|N|2424456|2424759|2424098|2424371|N|N|N|N|N| +2424464|AAAAAAAAAJOPECAA|1925-11-09|310|1349|104|1925|1|11|9|4|1925|104|1349|Monday|1925Q4|N|N|N|2424456|2424759|2424099|2424372|N|N|N|N|N| +2424465|AAAAAAAABJOPECAA|1925-11-10|310|1350|104|1925|2|11|10|4|1925|104|1350|Tuesday|1925Q4|N|N|N|2424456|2424759|2424100|2424373|N|N|N|N|N| +2424466|AAAAAAAACJOPECAA|1925-11-11|310|1350|104|1925|3|11|11|4|1925|104|1350|Wednesday|1925Q4|N|N|N|2424456|2424759|2424101|2424374|N|N|N|N|N| +2424467|AAAAAAAADJOPECAA|1925-11-12|310|1350|104|1925|4|11|12|4|1925|104|1350|Thursday|1925Q4|N|N|N|2424456|2424759|2424102|2424375|N|N|N|N|N| +2424468|AAAAAAAAEJOPECAA|1925-11-13|310|1350|104|1925|5|11|13|4|1925|104|1350|Friday|1925Q4|N|Y|N|2424456|2424759|2424103|2424376|N|N|N|N|N| +2424469|AAAAAAAAFJOPECAA|1925-11-14|310|1350|104|1925|6|11|14|4|1925|104|1350|Saturday|1925Q4|N|Y|N|2424456|2424759|2424104|2424377|N|N|N|N|N| +2424470|AAAAAAAAGJOPECAA|1925-11-15|310|1350|104|1925|0|11|15|4|1925|104|1350|Sunday|1925Q4|N|N|N|2424456|2424759|2424105|2424378|N|N|N|N|N| +2424471|AAAAAAAAHJOPECAA|1925-11-16|310|1350|104|1925|1|11|16|4|1925|104|1350|Monday|1925Q4|N|N|N|2424456|2424759|2424106|2424379|N|N|N|N|N| +2424472|AAAAAAAAIJOPECAA|1925-11-17|310|1351|104|1925|2|11|17|4|1925|104|1351|Tuesday|1925Q4|N|N|N|2424456|2424759|2424107|2424380|N|N|N|N|N| +2424473|AAAAAAAAJJOPECAA|1925-11-18|310|1351|104|1925|3|11|18|4|1925|104|1351|Wednesday|1925Q4|N|N|N|2424456|2424759|2424108|2424381|N|N|N|N|N| +2424474|AAAAAAAAKJOPECAA|1925-11-19|310|1351|104|1925|4|11|19|4|1925|104|1351|Thursday|1925Q4|N|N|N|2424456|2424759|2424109|2424382|N|N|N|N|N| +2424475|AAAAAAAALJOPECAA|1925-11-20|310|1351|104|1925|5|11|20|4|1925|104|1351|Friday|1925Q4|N|Y|N|2424456|2424759|2424110|2424383|N|N|N|N|N| +2424476|AAAAAAAAMJOPECAA|1925-11-21|310|1351|104|1925|6|11|21|4|1925|104|1351|Saturday|1925Q4|N|Y|N|2424456|2424759|2424111|2424384|N|N|N|N|N| +2424477|AAAAAAAANJOPECAA|1925-11-22|310|1351|104|1925|0|11|22|4|1925|104|1351|Sunday|1925Q4|N|N|N|2424456|2424759|2424112|2424385|N|N|N|N|N| +2424478|AAAAAAAAOJOPECAA|1925-11-23|310|1351|104|1925|1|11|23|4|1925|104|1351|Monday|1925Q4|N|N|N|2424456|2424759|2424113|2424386|N|N|N|N|N| +2424479|AAAAAAAAPJOPECAA|1925-11-24|310|1352|104|1925|2|11|24|4|1925|104|1352|Tuesday|1925Q4|N|N|N|2424456|2424759|2424114|2424387|N|N|N|N|N| +2424480|AAAAAAAAAKOPECAA|1925-11-25|310|1352|104|1925|3|11|25|4|1925|104|1352|Wednesday|1925Q4|N|N|N|2424456|2424759|2424115|2424388|N|N|N|N|N| +2424481|AAAAAAAABKOPECAA|1925-11-26|310|1352|104|1925|4|11|26|4|1925|104|1352|Thursday|1925Q4|N|N|N|2424456|2424759|2424116|2424389|N|N|N|N|N| +2424482|AAAAAAAACKOPECAA|1925-11-27|310|1352|104|1925|5|11|27|4|1925|104|1352|Friday|1925Q4|N|Y|N|2424456|2424759|2424117|2424390|N|N|N|N|N| +2424483|AAAAAAAADKOPECAA|1925-11-28|310|1352|104|1925|6|11|28|4|1925|104|1352|Saturday|1925Q4|N|Y|N|2424456|2424759|2424118|2424391|N|N|N|N|N| +2424484|AAAAAAAAEKOPECAA|1925-11-29|310|1352|104|1925|0|11|29|4|1925|104|1352|Sunday|1925Q4|N|N|N|2424456|2424759|2424119|2424392|N|N|N|N|N| +2424485|AAAAAAAAFKOPECAA|1925-11-30|310|1352|104|1925|1|11|30|4|1925|104|1352|Monday|1925Q4|N|N|N|2424456|2424759|2424120|2424393|N|N|N|N|N| +2424486|AAAAAAAAGKOPECAA|1925-12-01|311|1353|105|1925|2|12|1|4|1925|105|1353|Tuesday|1925Q4|N|N|N|2424486|2424819|2424121|2424394|N|N|N|N|N| +2424487|AAAAAAAAHKOPECAA|1925-12-02|311|1353|105|1925|3|12|2|4|1925|105|1353|Wednesday|1925Q4|N|N|N|2424486|2424819|2424122|2424395|N|N|N|N|N| +2424488|AAAAAAAAIKOPECAA|1925-12-03|311|1353|105|1925|4|12|3|4|1925|105|1353|Thursday|1925Q4|N|N|N|2424486|2424819|2424123|2424396|N|N|N|N|N| +2424489|AAAAAAAAJKOPECAA|1925-12-04|311|1353|105|1925|5|12|4|4|1925|105|1353|Friday|1925Q4|N|Y|N|2424486|2424819|2424124|2424397|N|N|N|N|N| +2424490|AAAAAAAAKKOPECAA|1925-12-05|311|1353|105|1925|6|12|5|4|1925|105|1353|Saturday|1925Q4|N|Y|N|2424486|2424819|2424125|2424398|N|N|N|N|N| +2424491|AAAAAAAALKOPECAA|1925-12-06|311|1353|105|1925|0|12|6|4|1925|105|1353|Sunday|1925Q4|N|N|N|2424486|2424819|2424126|2424399|N|N|N|N|N| +2424492|AAAAAAAAMKOPECAA|1925-12-07|311|1353|105|1925|1|12|7|4|1925|105|1353|Monday|1925Q4|N|N|N|2424486|2424819|2424127|2424400|N|N|N|N|N| +2424493|AAAAAAAANKOPECAA|1925-12-08|311|1354|105|1925|2|12|8|4|1925|105|1354|Tuesday|1925Q4|N|N|N|2424486|2424819|2424128|2424401|N|N|N|N|N| +2424494|AAAAAAAAOKOPECAA|1925-12-09|311|1354|105|1925|3|12|9|4|1925|105|1354|Wednesday|1925Q4|N|N|N|2424486|2424819|2424129|2424402|N|N|N|N|N| +2424495|AAAAAAAAPKOPECAA|1925-12-10|311|1354|105|1925|4|12|10|4|1925|105|1354|Thursday|1925Q4|N|N|N|2424486|2424819|2424130|2424403|N|N|N|N|N| +2424496|AAAAAAAAALOPECAA|1925-12-11|311|1354|105|1925|5|12|11|4|1925|105|1354|Friday|1925Q4|N|Y|N|2424486|2424819|2424131|2424404|N|N|N|N|N| +2424497|AAAAAAAABLOPECAA|1925-12-12|311|1354|105|1925|6|12|12|4|1925|105|1354|Saturday|1925Q4|N|Y|N|2424486|2424819|2424132|2424405|N|N|N|N|N| +2424498|AAAAAAAACLOPECAA|1925-12-13|311|1354|105|1925|0|12|13|4|1925|105|1354|Sunday|1925Q4|N|N|N|2424486|2424819|2424133|2424406|N|N|N|N|N| +2424499|AAAAAAAADLOPECAA|1925-12-14|311|1354|105|1925|1|12|14|4|1925|105|1354|Monday|1925Q4|N|N|N|2424486|2424819|2424134|2424407|N|N|N|N|N| +2424500|AAAAAAAAELOPECAA|1925-12-15|311|1355|105|1925|2|12|15|4|1925|105|1355|Tuesday|1925Q4|N|N|N|2424486|2424819|2424135|2424408|N|N|N|N|N| +2424501|AAAAAAAAFLOPECAA|1925-12-16|311|1355|105|1925|3|12|16|4|1925|105|1355|Wednesday|1925Q4|N|N|N|2424486|2424819|2424136|2424409|N|N|N|N|N| +2424502|AAAAAAAAGLOPECAA|1925-12-17|311|1355|105|1925|4|12|17|4|1925|105|1355|Thursday|1925Q4|N|N|N|2424486|2424819|2424137|2424410|N|N|N|N|N| +2424503|AAAAAAAAHLOPECAA|1925-12-18|311|1355|105|1925|5|12|18|4|1925|105|1355|Friday|1925Q4|N|Y|N|2424486|2424819|2424138|2424411|N|N|N|N|N| +2424504|AAAAAAAAILOPECAA|1925-12-19|311|1355|105|1925|6|12|19|4|1925|105|1355|Saturday|1925Q4|N|Y|N|2424486|2424819|2424139|2424412|N|N|N|N|N| +2424505|AAAAAAAAJLOPECAA|1925-12-20|311|1355|105|1925|0|12|20|4|1925|105|1355|Sunday|1925Q4|N|N|N|2424486|2424819|2424140|2424413|N|N|N|N|N| +2424506|AAAAAAAAKLOPECAA|1925-12-21|311|1355|105|1925|1|12|21|4|1925|105|1355|Monday|1925Q4|N|N|N|2424486|2424819|2424141|2424414|N|N|N|N|N| +2424507|AAAAAAAALLOPECAA|1925-12-22|311|1356|105|1925|2|12|22|4|1925|105|1356|Tuesday|1925Q4|N|N|N|2424486|2424819|2424142|2424415|N|N|N|N|N| +2424508|AAAAAAAAMLOPECAA|1925-12-23|311|1356|105|1925|3|12|23|4|1925|105|1356|Wednesday|1925Q4|N|N|N|2424486|2424819|2424143|2424416|N|N|N|N|N| +2424509|AAAAAAAANLOPECAA|1925-12-24|311|1356|105|1925|4|12|24|4|1925|105|1356|Thursday|1925Q4|N|N|N|2424486|2424819|2424144|2424417|N|N|N|N|N| +2424510|AAAAAAAAOLOPECAA|1925-12-25|311|1356|105|1925|5|12|25|4|1925|105|1356|Friday|1925Q4|N|Y|N|2424486|2424819|2424145|2424418|N|N|N|N|N| +2424511|AAAAAAAAPLOPECAA|1925-12-26|311|1356|105|1925|6|12|26|4|1925|105|1356|Saturday|1925Q4|Y|Y|N|2424486|2424819|2424146|2424419|N|N|N|N|N| +2424512|AAAAAAAAAMOPECAA|1925-12-27|311|1356|105|1925|0|12|27|4|1925|105|1356|Sunday|1925Q4|N|N|Y|2424486|2424819|2424147|2424420|N|N|N|N|N| +2424513|AAAAAAAABMOPECAA|1925-12-28|311|1356|105|1925|1|12|28|4|1925|105|1356|Monday|1925Q4|N|N|N|2424486|2424819|2424148|2424421|N|N|N|N|N| +2424514|AAAAAAAACMOPECAA|1925-12-29|311|1357|105|1925|2|12|29|4|1925|105|1357|Tuesday|1925Q4|N|N|N|2424486|2424819|2424149|2424422|N|N|N|N|N| +2424515|AAAAAAAADMOPECAA|1925-12-30|311|1357|105|1925|3|12|30|4|1925|105|1357|Wednesday|1925Q4|N|N|N|2424486|2424819|2424150|2424423|N|N|N|N|N| +2424516|AAAAAAAAEMOPECAA|1925-12-31|311|1357|105|1925|4|12|31|4|1925|105|1357|Thursday|1925Q4|N|N|N|2424486|2424819|2424151|2424424|N|N|N|N|N| +2424517|AAAAAAAAFMOPECAA|1926-01-01|312|1357|105|1926|5|1|1|1|1926|105|1357|Friday|1926Q1|Y|Y|N|2424517|2424516|2424152|2424425|N|N|N|N|N| +2424518|AAAAAAAAGMOPECAA|1926-01-02|312|1357|105|1926|6|1|2|1|1926|105|1357|Saturday|1926Q1|N|Y|Y|2424517|2424516|2424153|2424426|N|N|N|N|N| +2424519|AAAAAAAAHMOPECAA|1926-01-03|312|1357|105|1926|0|1|3|1|1926|105|1357|Sunday|1926Q1|N|N|N|2424517|2424516|2424154|2424427|N|N|N|N|N| +2424520|AAAAAAAAIMOPECAA|1926-01-04|312|1357|105|1926|1|1|4|1|1926|105|1357|Monday|1926Q1|N|N|N|2424517|2424516|2424155|2424428|N|N|N|N|N| +2424521|AAAAAAAAJMOPECAA|1926-01-05|312|1358|105|1926|2|1|5|1|1926|105|1358|Tuesday|1926Q1|N|N|N|2424517|2424516|2424156|2424429|N|N|N|N|N| +2424522|AAAAAAAAKMOPECAA|1926-01-06|312|1358|105|1926|3|1|6|1|1926|105|1358|Wednesday|1926Q1|N|N|N|2424517|2424516|2424157|2424430|N|N|N|N|N| +2424523|AAAAAAAALMOPECAA|1926-01-07|312|1358|105|1926|4|1|7|1|1926|105|1358|Thursday|1926Q1|N|N|N|2424517|2424516|2424158|2424431|N|N|N|N|N| +2424524|AAAAAAAAMMOPECAA|1926-01-08|312|1358|105|1926|5|1|8|1|1926|105|1358|Friday|1926Q1|N|Y|N|2424517|2424516|2424159|2424432|N|N|N|N|N| +2424525|AAAAAAAANMOPECAA|1926-01-09|312|1358|105|1926|6|1|9|1|1926|105|1358|Saturday|1926Q1|N|Y|N|2424517|2424516|2424160|2424433|N|N|N|N|N| +2424526|AAAAAAAAOMOPECAA|1926-01-10|312|1358|105|1926|0|1|10|1|1926|105|1358|Sunday|1926Q1|N|N|N|2424517|2424516|2424161|2424434|N|N|N|N|N| +2424527|AAAAAAAAPMOPECAA|1926-01-11|312|1358|105|1926|1|1|11|1|1926|105|1358|Monday|1926Q1|N|N|N|2424517|2424516|2424162|2424435|N|N|N|N|N| +2424528|AAAAAAAAANOPECAA|1926-01-12|312|1359|105|1926|2|1|12|1|1926|105|1359|Tuesday|1926Q1|N|N|N|2424517|2424516|2424163|2424436|N|N|N|N|N| +2424529|AAAAAAAABNOPECAA|1926-01-13|312|1359|105|1926|3|1|13|1|1926|105|1359|Wednesday|1926Q1|N|N|N|2424517|2424516|2424164|2424437|N|N|N|N|N| +2424530|AAAAAAAACNOPECAA|1926-01-14|312|1359|105|1926|4|1|14|1|1926|105|1359|Thursday|1926Q1|N|N|N|2424517|2424516|2424165|2424438|N|N|N|N|N| +2424531|AAAAAAAADNOPECAA|1926-01-15|312|1359|105|1926|5|1|15|1|1926|105|1359|Friday|1926Q1|N|Y|N|2424517|2424516|2424166|2424439|N|N|N|N|N| +2424532|AAAAAAAAENOPECAA|1926-01-16|312|1359|105|1926|6|1|16|1|1926|105|1359|Saturday|1926Q1|N|Y|N|2424517|2424516|2424167|2424440|N|N|N|N|N| +2424533|AAAAAAAAFNOPECAA|1926-01-17|312|1359|105|1926|0|1|17|1|1926|105|1359|Sunday|1926Q1|N|N|N|2424517|2424516|2424168|2424441|N|N|N|N|N| +2424534|AAAAAAAAGNOPECAA|1926-01-18|312|1359|105|1926|1|1|18|1|1926|105|1359|Monday|1926Q1|N|N|N|2424517|2424516|2424169|2424442|N|N|N|N|N| +2424535|AAAAAAAAHNOPECAA|1926-01-19|312|1360|105|1926|2|1|19|1|1926|105|1360|Tuesday|1926Q1|N|N|N|2424517|2424516|2424170|2424443|N|N|N|N|N| +2424536|AAAAAAAAINOPECAA|1926-01-20|312|1360|105|1926|3|1|20|1|1926|105|1360|Wednesday|1926Q1|N|N|N|2424517|2424516|2424171|2424444|N|N|N|N|N| +2424537|AAAAAAAAJNOPECAA|1926-01-21|312|1360|105|1926|4|1|21|1|1926|105|1360|Thursday|1926Q1|N|N|N|2424517|2424516|2424172|2424445|N|N|N|N|N| +2424538|AAAAAAAAKNOPECAA|1926-01-22|312|1360|105|1926|5|1|22|1|1926|105|1360|Friday|1926Q1|N|Y|N|2424517|2424516|2424173|2424446|N|N|N|N|N| +2424539|AAAAAAAALNOPECAA|1926-01-23|312|1360|105|1926|6|1|23|1|1926|105|1360|Saturday|1926Q1|N|Y|N|2424517|2424516|2424174|2424447|N|N|N|N|N| +2424540|AAAAAAAAMNOPECAA|1926-01-24|312|1360|105|1926|0|1|24|1|1926|105|1360|Sunday|1926Q1|N|N|N|2424517|2424516|2424175|2424448|N|N|N|N|N| +2424541|AAAAAAAANNOPECAA|1926-01-25|312|1360|105|1926|1|1|25|1|1926|105|1360|Monday|1926Q1|N|N|N|2424517|2424516|2424176|2424449|N|N|N|N|N| +2424542|AAAAAAAAONOPECAA|1926-01-26|312|1361|105|1926|2|1|26|1|1926|105|1361|Tuesday|1926Q1|N|N|N|2424517|2424516|2424177|2424450|N|N|N|N|N| +2424543|AAAAAAAAPNOPECAA|1926-01-27|312|1361|105|1926|3|1|27|1|1926|105|1361|Wednesday|1926Q1|N|N|N|2424517|2424516|2424178|2424451|N|N|N|N|N| +2424544|AAAAAAAAAOOPECAA|1926-01-28|312|1361|105|1926|4|1|28|1|1926|105|1361|Thursday|1926Q1|N|N|N|2424517|2424516|2424179|2424452|N|N|N|N|N| +2424545|AAAAAAAABOOPECAA|1926-01-29|312|1361|105|1926|5|1|29|1|1926|105|1361|Friday|1926Q1|N|Y|N|2424517|2424516|2424180|2424453|N|N|N|N|N| +2424546|AAAAAAAACOOPECAA|1926-01-30|312|1361|105|1926|6|1|30|1|1926|105|1361|Saturday|1926Q1|N|Y|N|2424517|2424516|2424181|2424454|N|N|N|N|N| +2424547|AAAAAAAADOOPECAA|1926-01-31|312|1361|105|1926|0|1|31|1|1926|105|1361|Sunday|1926Q1|N|N|N|2424517|2424516|2424182|2424455|N|N|N|N|N| +2424548|AAAAAAAAEOOPECAA|1926-02-01|313|1361|105|1926|1|2|1|1|1926|105|1361|Monday|1926Q1|N|N|N|2424548|2424578|2424183|2424456|N|N|N|N|N| +2424549|AAAAAAAAFOOPECAA|1926-02-02|313|1362|105|1926|2|2|2|1|1926|105|1362|Tuesday|1926Q1|N|N|N|2424548|2424578|2424184|2424457|N|N|N|N|N| +2424550|AAAAAAAAGOOPECAA|1926-02-03|313|1362|105|1926|3|2|3|1|1926|105|1362|Wednesday|1926Q1|N|N|N|2424548|2424578|2424185|2424458|N|N|N|N|N| +2424551|AAAAAAAAHOOPECAA|1926-02-04|313|1362|105|1926|4|2|4|1|1926|105|1362|Thursday|1926Q1|N|N|N|2424548|2424578|2424186|2424459|N|N|N|N|N| +2424552|AAAAAAAAIOOPECAA|1926-02-05|313|1362|105|1926|5|2|5|1|1926|105|1362|Friday|1926Q1|N|Y|N|2424548|2424578|2424187|2424460|N|N|N|N|N| +2424553|AAAAAAAAJOOPECAA|1926-02-06|313|1362|105|1926|6|2|6|1|1926|105|1362|Saturday|1926Q1|N|Y|N|2424548|2424578|2424188|2424461|N|N|N|N|N| +2424554|AAAAAAAAKOOPECAA|1926-02-07|313|1362|105|1926|0|2|7|1|1926|105|1362|Sunday|1926Q1|N|N|N|2424548|2424578|2424189|2424462|N|N|N|N|N| +2424555|AAAAAAAALOOPECAA|1926-02-08|313|1362|105|1926|1|2|8|1|1926|105|1362|Monday|1926Q1|N|N|N|2424548|2424578|2424190|2424463|N|N|N|N|N| +2424556|AAAAAAAAMOOPECAA|1926-02-09|313|1363|105|1926|2|2|9|1|1926|105|1363|Tuesday|1926Q1|N|N|N|2424548|2424578|2424191|2424464|N|N|N|N|N| +2424557|AAAAAAAANOOPECAA|1926-02-10|313|1363|105|1926|3|2|10|1|1926|105|1363|Wednesday|1926Q1|N|N|N|2424548|2424578|2424192|2424465|N|N|N|N|N| +2424558|AAAAAAAAOOOPECAA|1926-02-11|313|1363|105|1926|4|2|11|1|1926|105|1363|Thursday|1926Q1|N|N|N|2424548|2424578|2424193|2424466|N|N|N|N|N| +2424559|AAAAAAAAPOOPECAA|1926-02-12|313|1363|105|1926|5|2|12|1|1926|105|1363|Friday|1926Q1|N|Y|N|2424548|2424578|2424194|2424467|N|N|N|N|N| +2424560|AAAAAAAAAPOPECAA|1926-02-13|313|1363|105|1926|6|2|13|1|1926|105|1363|Saturday|1926Q1|N|Y|N|2424548|2424578|2424195|2424468|N|N|N|N|N| +2424561|AAAAAAAABPOPECAA|1926-02-14|313|1363|105|1926|0|2|14|1|1926|105|1363|Sunday|1926Q1|N|N|N|2424548|2424578|2424196|2424469|N|N|N|N|N| +2424562|AAAAAAAACPOPECAA|1926-02-15|313|1363|105|1926|1|2|15|1|1926|105|1363|Monday|1926Q1|N|N|N|2424548|2424578|2424197|2424470|N|N|N|N|N| +2424563|AAAAAAAADPOPECAA|1926-02-16|313|1364|105|1926|2|2|16|1|1926|105|1364|Tuesday|1926Q1|N|N|N|2424548|2424578|2424198|2424471|N|N|N|N|N| +2424564|AAAAAAAAEPOPECAA|1926-02-17|313|1364|105|1926|3|2|17|1|1926|105|1364|Wednesday|1926Q1|N|N|N|2424548|2424578|2424199|2424472|N|N|N|N|N| +2424565|AAAAAAAAFPOPECAA|1926-02-18|313|1364|105|1926|4|2|18|1|1926|105|1364|Thursday|1926Q1|N|N|N|2424548|2424578|2424200|2424473|N|N|N|N|N| +2424566|AAAAAAAAGPOPECAA|1926-02-19|313|1364|105|1926|5|2|19|1|1926|105|1364|Friday|1926Q1|N|Y|N|2424548|2424578|2424201|2424474|N|N|N|N|N| +2424567|AAAAAAAAHPOPECAA|1926-02-20|313|1364|105|1926|6|2|20|1|1926|105|1364|Saturday|1926Q1|N|Y|N|2424548|2424578|2424202|2424475|N|N|N|N|N| +2424568|AAAAAAAAIPOPECAA|1926-02-21|313|1364|105|1926|0|2|21|1|1926|105|1364|Sunday|1926Q1|N|N|N|2424548|2424578|2424203|2424476|N|N|N|N|N| +2424569|AAAAAAAAJPOPECAA|1926-02-22|313|1364|105|1926|1|2|22|1|1926|105|1364|Monday|1926Q1|N|N|N|2424548|2424578|2424204|2424477|N|N|N|N|N| +2424570|AAAAAAAAKPOPECAA|1926-02-23|313|1365|105|1926|2|2|23|1|1926|105|1365|Tuesday|1926Q1|N|N|N|2424548|2424578|2424205|2424478|N|N|N|N|N| +2424571|AAAAAAAALPOPECAA|1926-02-24|313|1365|105|1926|3|2|24|1|1926|105|1365|Wednesday|1926Q1|N|N|N|2424548|2424578|2424206|2424479|N|N|N|N|N| +2424572|AAAAAAAAMPOPECAA|1926-02-25|313|1365|105|1926|4|2|25|1|1926|105|1365|Thursday|1926Q1|N|N|N|2424548|2424578|2424207|2424480|N|N|N|N|N| +2424573|AAAAAAAANPOPECAA|1926-02-26|313|1365|105|1926|5|2|26|1|1926|105|1365|Friday|1926Q1|N|Y|N|2424548|2424578|2424208|2424481|N|N|N|N|N| +2424574|AAAAAAAAOPOPECAA|1926-02-27|313|1365|105|1926|6|2|27|1|1926|105|1365|Saturday|1926Q1|N|Y|N|2424548|2424578|2424209|2424482|N|N|N|N|N| +2424575|AAAAAAAAPPOPECAA|1926-02-28|313|1365|105|1926|0|2|28|1|1926|105|1365|Sunday|1926Q1|N|N|N|2424548|2424578|2424210|2424483|N|N|N|N|N| +2424576|AAAAAAAAAAPPECAA|1926-03-01|314|1365|106|1926|1|3|1|1|1926|106|1365|Monday|1926Q1|N|N|N|2424576|2424634|2424211|2424484|N|N|N|N|N| +2424577|AAAAAAAABAPPECAA|1926-03-02|314|1366|106|1926|2|3|2|1|1926|106|1366|Tuesday|1926Q1|N|N|N|2424576|2424634|2424212|2424485|N|N|N|N|N| +2424578|AAAAAAAACAPPECAA|1926-03-03|314|1366|106|1926|3|3|3|1|1926|106|1366|Wednesday|1926Q1|N|N|N|2424576|2424634|2424213|2424486|N|N|N|N|N| +2424579|AAAAAAAADAPPECAA|1926-03-04|314|1366|106|1926|4|3|4|1|1926|106|1366|Thursday|1926Q1|N|N|N|2424576|2424634|2424214|2424487|N|N|N|N|N| +2424580|AAAAAAAAEAPPECAA|1926-03-05|314|1366|106|1926|5|3|5|1|1926|106|1366|Friday|1926Q1|N|Y|N|2424576|2424634|2424215|2424488|N|N|N|N|N| +2424581|AAAAAAAAFAPPECAA|1926-03-06|314|1366|106|1926|6|3|6|1|1926|106|1366|Saturday|1926Q1|N|Y|N|2424576|2424634|2424216|2424489|N|N|N|N|N| +2424582|AAAAAAAAGAPPECAA|1926-03-07|314|1366|106|1926|0|3|7|1|1926|106|1366|Sunday|1926Q1|N|N|N|2424576|2424634|2424217|2424490|N|N|N|N|N| +2424583|AAAAAAAAHAPPECAA|1926-03-08|314|1366|106|1926|1|3|8|1|1926|106|1366|Monday|1926Q1|N|N|N|2424576|2424634|2424218|2424491|N|N|N|N|N| +2424584|AAAAAAAAIAPPECAA|1926-03-09|314|1367|106|1926|2|3|9|1|1926|106|1367|Tuesday|1926Q1|N|N|N|2424576|2424634|2424219|2424492|N|N|N|N|N| +2424585|AAAAAAAAJAPPECAA|1926-03-10|314|1367|106|1926|3|3|10|1|1926|106|1367|Wednesday|1926Q1|N|N|N|2424576|2424634|2424220|2424493|N|N|N|N|N| +2424586|AAAAAAAAKAPPECAA|1926-03-11|314|1367|106|1926|4|3|11|1|1926|106|1367|Thursday|1926Q1|N|N|N|2424576|2424634|2424221|2424494|N|N|N|N|N| +2424587|AAAAAAAALAPPECAA|1926-03-12|314|1367|106|1926|5|3|12|1|1926|106|1367|Friday|1926Q1|N|Y|N|2424576|2424634|2424222|2424495|N|N|N|N|N| +2424588|AAAAAAAAMAPPECAA|1926-03-13|314|1367|106|1926|6|3|13|1|1926|106|1367|Saturday|1926Q1|N|Y|N|2424576|2424634|2424223|2424496|N|N|N|N|N| +2424589|AAAAAAAANAPPECAA|1926-03-14|314|1367|106|1926|0|3|14|1|1926|106|1367|Sunday|1926Q1|N|N|N|2424576|2424634|2424224|2424497|N|N|N|N|N| +2424590|AAAAAAAAOAPPECAA|1926-03-15|314|1367|106|1926|1|3|15|1|1926|106|1367|Monday|1926Q1|N|N|N|2424576|2424634|2424225|2424498|N|N|N|N|N| +2424591|AAAAAAAAPAPPECAA|1926-03-16|314|1368|106|1926|2|3|16|1|1926|106|1368|Tuesday|1926Q1|N|N|N|2424576|2424634|2424226|2424499|N|N|N|N|N| +2424592|AAAAAAAAABPPECAA|1926-03-17|314|1368|106|1926|3|3|17|1|1926|106|1368|Wednesday|1926Q1|N|N|N|2424576|2424634|2424227|2424500|N|N|N|N|N| +2424593|AAAAAAAABBPPECAA|1926-03-18|314|1368|106|1926|4|3|18|1|1926|106|1368|Thursday|1926Q1|N|N|N|2424576|2424634|2424228|2424501|N|N|N|N|N| +2424594|AAAAAAAACBPPECAA|1926-03-19|314|1368|106|1926|5|3|19|1|1926|106|1368|Friday|1926Q1|N|Y|N|2424576|2424634|2424229|2424502|N|N|N|N|N| +2424595|AAAAAAAADBPPECAA|1926-03-20|314|1368|106|1926|6|3|20|1|1926|106|1368|Saturday|1926Q1|N|Y|N|2424576|2424634|2424230|2424503|N|N|N|N|N| +2424596|AAAAAAAAEBPPECAA|1926-03-21|314|1368|106|1926|0|3|21|1|1926|106|1368|Sunday|1926Q1|N|N|N|2424576|2424634|2424231|2424504|N|N|N|N|N| +2424597|AAAAAAAAFBPPECAA|1926-03-22|314|1368|106|1926|1|3|22|1|1926|106|1368|Monday|1926Q1|N|N|N|2424576|2424634|2424232|2424505|N|N|N|N|N| +2424598|AAAAAAAAGBPPECAA|1926-03-23|314|1369|106|1926|2|3|23|1|1926|106|1369|Tuesday|1926Q1|N|N|N|2424576|2424634|2424233|2424506|N|N|N|N|N| +2424599|AAAAAAAAHBPPECAA|1926-03-24|314|1369|106|1926|3|3|24|1|1926|106|1369|Wednesday|1926Q1|N|N|N|2424576|2424634|2424234|2424507|N|N|N|N|N| +2424600|AAAAAAAAIBPPECAA|1926-03-25|314|1369|106|1926|4|3|25|1|1926|106|1369|Thursday|1926Q1|N|N|N|2424576|2424634|2424235|2424508|N|N|N|N|N| +2424601|AAAAAAAAJBPPECAA|1926-03-26|314|1369|106|1926|5|3|26|1|1926|106|1369|Friday|1926Q1|N|Y|N|2424576|2424634|2424236|2424509|N|N|N|N|N| +2424602|AAAAAAAAKBPPECAA|1926-03-27|314|1369|106|1926|6|3|27|1|1926|106|1369|Saturday|1926Q1|N|Y|N|2424576|2424634|2424237|2424510|N|N|N|N|N| +2424603|AAAAAAAALBPPECAA|1926-03-28|314|1369|106|1926|0|3|28|1|1926|106|1369|Sunday|1926Q1|N|N|N|2424576|2424634|2424238|2424511|N|N|N|N|N| +2424604|AAAAAAAAMBPPECAA|1926-03-29|314|1369|106|1926|1|3|29|1|1926|106|1369|Monday|1926Q1|N|N|N|2424576|2424634|2424239|2424512|N|N|N|N|N| +2424605|AAAAAAAANBPPECAA|1926-03-30|314|1370|106|1926|2|3|30|1|1926|106|1370|Tuesday|1926Q1|N|N|N|2424576|2424634|2424240|2424513|N|N|N|N|N| +2424606|AAAAAAAAOBPPECAA|1926-03-31|314|1370|106|1926|3|3|31|1|1926|106|1370|Wednesday|1926Q1|N|N|N|2424576|2424634|2424241|2424514|N|N|N|N|N| +2424607|AAAAAAAAPBPPECAA|1926-04-01|315|1370|106|1926|4|4|1|1|1926|106|1370|Thursday|1926Q1|N|N|N|2424607|2424696|2424242|2424517|N|N|N|N|N| +2424608|AAAAAAAAACPPECAA|1926-04-02|315|1370|106|1926|5|4|2|2|1926|106|1370|Friday|1926Q2|N|Y|N|2424607|2424696|2424243|2424518|N|N|N|N|N| +2424609|AAAAAAAABCPPECAA|1926-04-03|315|1370|106|1926|6|4|3|2|1926|106|1370|Saturday|1926Q2|N|Y|N|2424607|2424696|2424244|2424519|N|N|N|N|N| +2424610|AAAAAAAACCPPECAA|1926-04-04|315|1370|106|1926|0|4|4|2|1926|106|1370|Sunday|1926Q2|N|N|N|2424607|2424696|2424245|2424520|N|N|N|N|N| +2424611|AAAAAAAADCPPECAA|1926-04-05|315|1370|106|1926|1|4|5|2|1926|106|1370|Monday|1926Q2|N|N|N|2424607|2424696|2424246|2424521|N|N|N|N|N| +2424612|AAAAAAAAECPPECAA|1926-04-06|315|1371|106|1926|2|4|6|2|1926|106|1371|Tuesday|1926Q2|N|N|N|2424607|2424696|2424247|2424522|N|N|N|N|N| +2424613|AAAAAAAAFCPPECAA|1926-04-07|315|1371|106|1926|3|4|7|2|1926|106|1371|Wednesday|1926Q2|N|N|N|2424607|2424696|2424248|2424523|N|N|N|N|N| +2424614|AAAAAAAAGCPPECAA|1926-04-08|315|1371|106|1926|4|4|8|2|1926|106|1371|Thursday|1926Q2|N|N|N|2424607|2424696|2424249|2424524|N|N|N|N|N| +2424615|AAAAAAAAHCPPECAA|1926-04-09|315|1371|106|1926|5|4|9|2|1926|106|1371|Friday|1926Q2|N|Y|N|2424607|2424696|2424250|2424525|N|N|N|N|N| +2424616|AAAAAAAAICPPECAA|1926-04-10|315|1371|106|1926|6|4|10|2|1926|106|1371|Saturday|1926Q2|N|Y|N|2424607|2424696|2424251|2424526|N|N|N|N|N| +2424617|AAAAAAAAJCPPECAA|1926-04-11|315|1371|106|1926|0|4|11|2|1926|106|1371|Sunday|1926Q2|N|N|N|2424607|2424696|2424252|2424527|N|N|N|N|N| +2424618|AAAAAAAAKCPPECAA|1926-04-12|315|1371|106|1926|1|4|12|2|1926|106|1371|Monday|1926Q2|N|N|N|2424607|2424696|2424253|2424528|N|N|N|N|N| +2424619|AAAAAAAALCPPECAA|1926-04-13|315|1372|106|1926|2|4|13|2|1926|106|1372|Tuesday|1926Q2|N|N|N|2424607|2424696|2424254|2424529|N|N|N|N|N| +2424620|AAAAAAAAMCPPECAA|1926-04-14|315|1372|106|1926|3|4|14|2|1926|106|1372|Wednesday|1926Q2|N|N|N|2424607|2424696|2424255|2424530|N|N|N|N|N| +2424621|AAAAAAAANCPPECAA|1926-04-15|315|1372|106|1926|4|4|15|2|1926|106|1372|Thursday|1926Q2|N|N|N|2424607|2424696|2424256|2424531|N|N|N|N|N| +2424622|AAAAAAAAOCPPECAA|1926-04-16|315|1372|106|1926|5|4|16|2|1926|106|1372|Friday|1926Q2|N|Y|N|2424607|2424696|2424257|2424532|N|N|N|N|N| +2424623|AAAAAAAAPCPPECAA|1926-04-17|315|1372|106|1926|6|4|17|2|1926|106|1372|Saturday|1926Q2|N|Y|N|2424607|2424696|2424258|2424533|N|N|N|N|N| +2424624|AAAAAAAAADPPECAA|1926-04-18|315|1372|106|1926|0|4|18|2|1926|106|1372|Sunday|1926Q2|N|N|N|2424607|2424696|2424259|2424534|N|N|N|N|N| +2424625|AAAAAAAABDPPECAA|1926-04-19|315|1372|106|1926|1|4|19|2|1926|106|1372|Monday|1926Q2|N|N|N|2424607|2424696|2424260|2424535|N|N|N|N|N| +2424626|AAAAAAAACDPPECAA|1926-04-20|315|1373|106|1926|2|4|20|2|1926|106|1373|Tuesday|1926Q2|N|N|N|2424607|2424696|2424261|2424536|N|N|N|N|N| +2424627|AAAAAAAADDPPECAA|1926-04-21|315|1373|106|1926|3|4|21|2|1926|106|1373|Wednesday|1926Q2|N|N|N|2424607|2424696|2424262|2424537|N|N|N|N|N| +2424628|AAAAAAAAEDPPECAA|1926-04-22|315|1373|106|1926|4|4|22|2|1926|106|1373|Thursday|1926Q2|N|N|N|2424607|2424696|2424263|2424538|N|N|N|N|N| +2424629|AAAAAAAAFDPPECAA|1926-04-23|315|1373|106|1926|5|4|23|2|1926|106|1373|Friday|1926Q2|N|Y|N|2424607|2424696|2424264|2424539|N|N|N|N|N| +2424630|AAAAAAAAGDPPECAA|1926-04-24|315|1373|106|1926|6|4|24|2|1926|106|1373|Saturday|1926Q2|N|Y|N|2424607|2424696|2424265|2424540|N|N|N|N|N| +2424631|AAAAAAAAHDPPECAA|1926-04-25|315|1373|106|1926|0|4|25|2|1926|106|1373|Sunday|1926Q2|N|N|N|2424607|2424696|2424266|2424541|N|N|N|N|N| +2424632|AAAAAAAAIDPPECAA|1926-04-26|315|1373|106|1926|1|4|26|2|1926|106|1373|Monday|1926Q2|N|N|N|2424607|2424696|2424267|2424542|N|N|N|N|N| +2424633|AAAAAAAAJDPPECAA|1926-04-27|315|1374|106|1926|2|4|27|2|1926|106|1374|Tuesday|1926Q2|N|N|N|2424607|2424696|2424268|2424543|N|N|N|N|N| +2424634|AAAAAAAAKDPPECAA|1926-04-28|315|1374|106|1926|3|4|28|2|1926|106|1374|Wednesday|1926Q2|N|N|N|2424607|2424696|2424269|2424544|N|N|N|N|N| +2424635|AAAAAAAALDPPECAA|1926-04-29|315|1374|106|1926|4|4|29|2|1926|106|1374|Thursday|1926Q2|N|N|N|2424607|2424696|2424270|2424545|N|N|N|N|N| +2424636|AAAAAAAAMDPPECAA|1926-04-30|315|1374|106|1926|5|4|30|2|1926|106|1374|Friday|1926Q2|N|Y|N|2424607|2424696|2424271|2424546|N|N|N|N|N| +2424637|AAAAAAAANDPPECAA|1926-05-01|316|1374|106|1926|6|5|1|2|1926|106|1374|Saturday|1926Q2|N|Y|N|2424637|2424756|2424272|2424547|N|N|N|N|N| +2424638|AAAAAAAAODPPECAA|1926-05-02|316|1374|106|1926|0|5|2|2|1926|106|1374|Sunday|1926Q2|N|N|N|2424637|2424756|2424273|2424548|N|N|N|N|N| +2424639|AAAAAAAAPDPPECAA|1926-05-03|316|1374|106|1926|1|5|3|2|1926|106|1374|Monday|1926Q2|N|N|N|2424637|2424756|2424274|2424549|N|N|N|N|N| +2424640|AAAAAAAAAEPPECAA|1926-05-04|316|1375|106|1926|2|5|4|2|1926|106|1375|Tuesday|1926Q2|N|N|N|2424637|2424756|2424275|2424550|N|N|N|N|N| +2424641|AAAAAAAABEPPECAA|1926-05-05|316|1375|106|1926|3|5|5|2|1926|106|1375|Wednesday|1926Q2|N|N|N|2424637|2424756|2424276|2424551|N|N|N|N|N| +2424642|AAAAAAAACEPPECAA|1926-05-06|316|1375|106|1926|4|5|6|2|1926|106|1375|Thursday|1926Q2|N|N|N|2424637|2424756|2424277|2424552|N|N|N|N|N| +2424643|AAAAAAAADEPPECAA|1926-05-07|316|1375|106|1926|5|5|7|2|1926|106|1375|Friday|1926Q2|N|Y|N|2424637|2424756|2424278|2424553|N|N|N|N|N| +2424644|AAAAAAAAEEPPECAA|1926-05-08|316|1375|106|1926|6|5|8|2|1926|106|1375|Saturday|1926Q2|N|Y|N|2424637|2424756|2424279|2424554|N|N|N|N|N| +2424645|AAAAAAAAFEPPECAA|1926-05-09|316|1375|106|1926|0|5|9|2|1926|106|1375|Sunday|1926Q2|N|N|N|2424637|2424756|2424280|2424555|N|N|N|N|N| +2424646|AAAAAAAAGEPPECAA|1926-05-10|316|1375|106|1926|1|5|10|2|1926|106|1375|Monday|1926Q2|N|N|N|2424637|2424756|2424281|2424556|N|N|N|N|N| +2424647|AAAAAAAAHEPPECAA|1926-05-11|316|1376|106|1926|2|5|11|2|1926|106|1376|Tuesday|1926Q2|N|N|N|2424637|2424756|2424282|2424557|N|N|N|N|N| +2424648|AAAAAAAAIEPPECAA|1926-05-12|316|1376|106|1926|3|5|12|2|1926|106|1376|Wednesday|1926Q2|N|N|N|2424637|2424756|2424283|2424558|N|N|N|N|N| +2424649|AAAAAAAAJEPPECAA|1926-05-13|316|1376|106|1926|4|5|13|2|1926|106|1376|Thursday|1926Q2|N|N|N|2424637|2424756|2424284|2424559|N|N|N|N|N| +2424650|AAAAAAAAKEPPECAA|1926-05-14|316|1376|106|1926|5|5|14|2|1926|106|1376|Friday|1926Q2|N|Y|N|2424637|2424756|2424285|2424560|N|N|N|N|N| +2424651|AAAAAAAALEPPECAA|1926-05-15|316|1376|106|1926|6|5|15|2|1926|106|1376|Saturday|1926Q2|N|Y|N|2424637|2424756|2424286|2424561|N|N|N|N|N| +2424652|AAAAAAAAMEPPECAA|1926-05-16|316|1376|106|1926|0|5|16|2|1926|106|1376|Sunday|1926Q2|N|N|N|2424637|2424756|2424287|2424562|N|N|N|N|N| +2424653|AAAAAAAANEPPECAA|1926-05-17|316|1376|106|1926|1|5|17|2|1926|106|1376|Monday|1926Q2|N|N|N|2424637|2424756|2424288|2424563|N|N|N|N|N| +2424654|AAAAAAAAOEPPECAA|1926-05-18|316|1377|106|1926|2|5|18|2|1926|106|1377|Tuesday|1926Q2|N|N|N|2424637|2424756|2424289|2424564|N|N|N|N|N| +2424655|AAAAAAAAPEPPECAA|1926-05-19|316|1377|106|1926|3|5|19|2|1926|106|1377|Wednesday|1926Q2|N|N|N|2424637|2424756|2424290|2424565|N|N|N|N|N| +2424656|AAAAAAAAAFPPECAA|1926-05-20|316|1377|106|1926|4|5|20|2|1926|106|1377|Thursday|1926Q2|N|N|N|2424637|2424756|2424291|2424566|N|N|N|N|N| +2424657|AAAAAAAABFPPECAA|1926-05-21|316|1377|106|1926|5|5|21|2|1926|106|1377|Friday|1926Q2|N|Y|N|2424637|2424756|2424292|2424567|N|N|N|N|N| +2424658|AAAAAAAACFPPECAA|1926-05-22|316|1377|106|1926|6|5|22|2|1926|106|1377|Saturday|1926Q2|N|Y|N|2424637|2424756|2424293|2424568|N|N|N|N|N| +2424659|AAAAAAAADFPPECAA|1926-05-23|316|1377|106|1926|0|5|23|2|1926|106|1377|Sunday|1926Q2|N|N|N|2424637|2424756|2424294|2424569|N|N|N|N|N| +2424660|AAAAAAAAEFPPECAA|1926-05-24|316|1377|106|1926|1|5|24|2|1926|106|1377|Monday|1926Q2|N|N|N|2424637|2424756|2424295|2424570|N|N|N|N|N| +2424661|AAAAAAAAFFPPECAA|1926-05-25|316|1378|106|1926|2|5|25|2|1926|106|1378|Tuesday|1926Q2|N|N|N|2424637|2424756|2424296|2424571|N|N|N|N|N| +2424662|AAAAAAAAGFPPECAA|1926-05-26|316|1378|106|1926|3|5|26|2|1926|106|1378|Wednesday|1926Q2|N|N|N|2424637|2424756|2424297|2424572|N|N|N|N|N| +2424663|AAAAAAAAHFPPECAA|1926-05-27|316|1378|106|1926|4|5|27|2|1926|106|1378|Thursday|1926Q2|N|N|N|2424637|2424756|2424298|2424573|N|N|N|N|N| +2424664|AAAAAAAAIFPPECAA|1926-05-28|316|1378|106|1926|5|5|28|2|1926|106|1378|Friday|1926Q2|N|Y|N|2424637|2424756|2424299|2424574|N|N|N|N|N| +2424665|AAAAAAAAJFPPECAA|1926-05-29|316|1378|106|1926|6|5|29|2|1926|106|1378|Saturday|1926Q2|N|Y|N|2424637|2424756|2424300|2424575|N|N|N|N|N| +2424666|AAAAAAAAKFPPECAA|1926-05-30|316|1378|106|1926|0|5|30|2|1926|106|1378|Sunday|1926Q2|N|N|N|2424637|2424756|2424301|2424576|N|N|N|N|N| +2424667|AAAAAAAALFPPECAA|1926-05-31|316|1378|106|1926|1|5|31|2|1926|106|1378|Monday|1926Q2|N|N|N|2424637|2424756|2424302|2424577|N|N|N|N|N| +2424668|AAAAAAAAMFPPECAA|1926-06-01|317|1379|107|1926|2|6|1|2|1926|107|1379|Tuesday|1926Q2|N|N|N|2424668|2424818|2424303|2424578|N|N|N|N|N| +2424669|AAAAAAAANFPPECAA|1926-06-02|317|1379|107|1926|3|6|2|2|1926|107|1379|Wednesday|1926Q2|N|N|N|2424668|2424818|2424304|2424579|N|N|N|N|N| +2424670|AAAAAAAAOFPPECAA|1926-06-03|317|1379|107|1926|4|6|3|2|1926|107|1379|Thursday|1926Q2|N|N|N|2424668|2424818|2424305|2424580|N|N|N|N|N| +2424671|AAAAAAAAPFPPECAA|1926-06-04|317|1379|107|1926|5|6|4|2|1926|107|1379|Friday|1926Q2|N|Y|N|2424668|2424818|2424306|2424581|N|N|N|N|N| +2424672|AAAAAAAAAGPPECAA|1926-06-05|317|1379|107|1926|6|6|5|2|1926|107|1379|Saturday|1926Q2|N|Y|N|2424668|2424818|2424307|2424582|N|N|N|N|N| +2424673|AAAAAAAABGPPECAA|1926-06-06|317|1379|107|1926|0|6|6|2|1926|107|1379|Sunday|1926Q2|N|N|N|2424668|2424818|2424308|2424583|N|N|N|N|N| +2424674|AAAAAAAACGPPECAA|1926-06-07|317|1379|107|1926|1|6|7|2|1926|107|1379|Monday|1926Q2|N|N|N|2424668|2424818|2424309|2424584|N|N|N|N|N| +2424675|AAAAAAAADGPPECAA|1926-06-08|317|1380|107|1926|2|6|8|2|1926|107|1380|Tuesday|1926Q2|N|N|N|2424668|2424818|2424310|2424585|N|N|N|N|N| +2424676|AAAAAAAAEGPPECAA|1926-06-09|317|1380|107|1926|3|6|9|2|1926|107|1380|Wednesday|1926Q2|N|N|N|2424668|2424818|2424311|2424586|N|N|N|N|N| +2424677|AAAAAAAAFGPPECAA|1926-06-10|317|1380|107|1926|4|6|10|2|1926|107|1380|Thursday|1926Q2|N|N|N|2424668|2424818|2424312|2424587|N|N|N|N|N| +2424678|AAAAAAAAGGPPECAA|1926-06-11|317|1380|107|1926|5|6|11|2|1926|107|1380|Friday|1926Q2|N|Y|N|2424668|2424818|2424313|2424588|N|N|N|N|N| +2424679|AAAAAAAAHGPPECAA|1926-06-12|317|1380|107|1926|6|6|12|2|1926|107|1380|Saturday|1926Q2|N|Y|N|2424668|2424818|2424314|2424589|N|N|N|N|N| +2424680|AAAAAAAAIGPPECAA|1926-06-13|317|1380|107|1926|0|6|13|2|1926|107|1380|Sunday|1926Q2|N|N|N|2424668|2424818|2424315|2424590|N|N|N|N|N| +2424681|AAAAAAAAJGPPECAA|1926-06-14|317|1380|107|1926|1|6|14|2|1926|107|1380|Monday|1926Q2|N|N|N|2424668|2424818|2424316|2424591|N|N|N|N|N| +2424682|AAAAAAAAKGPPECAA|1926-06-15|317|1381|107|1926|2|6|15|2|1926|107|1381|Tuesday|1926Q2|N|N|N|2424668|2424818|2424317|2424592|N|N|N|N|N| +2424683|AAAAAAAALGPPECAA|1926-06-16|317|1381|107|1926|3|6|16|2|1926|107|1381|Wednesday|1926Q2|N|N|N|2424668|2424818|2424318|2424593|N|N|N|N|N| +2424684|AAAAAAAAMGPPECAA|1926-06-17|317|1381|107|1926|4|6|17|2|1926|107|1381|Thursday|1926Q2|N|N|N|2424668|2424818|2424319|2424594|N|N|N|N|N| +2424685|AAAAAAAANGPPECAA|1926-06-18|317|1381|107|1926|5|6|18|2|1926|107|1381|Friday|1926Q2|N|Y|N|2424668|2424818|2424320|2424595|N|N|N|N|N| +2424686|AAAAAAAAOGPPECAA|1926-06-19|317|1381|107|1926|6|6|19|2|1926|107|1381|Saturday|1926Q2|N|Y|N|2424668|2424818|2424321|2424596|N|N|N|N|N| +2424687|AAAAAAAAPGPPECAA|1926-06-20|317|1381|107|1926|0|6|20|2|1926|107|1381|Sunday|1926Q2|N|N|N|2424668|2424818|2424322|2424597|N|N|N|N|N| +2424688|AAAAAAAAAHPPECAA|1926-06-21|317|1381|107|1926|1|6|21|2|1926|107|1381|Monday|1926Q2|N|N|N|2424668|2424818|2424323|2424598|N|N|N|N|N| +2424689|AAAAAAAABHPPECAA|1926-06-22|317|1382|107|1926|2|6|22|2|1926|107|1382|Tuesday|1926Q2|N|N|N|2424668|2424818|2424324|2424599|N|N|N|N|N| +2424690|AAAAAAAACHPPECAA|1926-06-23|317|1382|107|1926|3|6|23|2|1926|107|1382|Wednesday|1926Q2|N|N|N|2424668|2424818|2424325|2424600|N|N|N|N|N| +2424691|AAAAAAAADHPPECAA|1926-06-24|317|1382|107|1926|4|6|24|2|1926|107|1382|Thursday|1926Q2|N|N|N|2424668|2424818|2424326|2424601|N|N|N|N|N| +2424692|AAAAAAAAEHPPECAA|1926-06-25|317|1382|107|1926|5|6|25|2|1926|107|1382|Friday|1926Q2|N|Y|N|2424668|2424818|2424327|2424602|N|N|N|N|N| +2424693|AAAAAAAAFHPPECAA|1926-06-26|317|1382|107|1926|6|6|26|2|1926|107|1382|Saturday|1926Q2|N|Y|N|2424668|2424818|2424328|2424603|N|N|N|N|N| +2424694|AAAAAAAAGHPPECAA|1926-06-27|317|1382|107|1926|0|6|27|2|1926|107|1382|Sunday|1926Q2|N|N|N|2424668|2424818|2424329|2424604|N|N|N|N|N| +2424695|AAAAAAAAHHPPECAA|1926-06-28|317|1382|107|1926|1|6|28|2|1926|107|1382|Monday|1926Q2|N|N|N|2424668|2424818|2424330|2424605|N|N|N|N|N| +2424696|AAAAAAAAIHPPECAA|1926-06-29|317|1383|107|1926|2|6|29|2|1926|107|1383|Tuesday|1926Q2|N|N|N|2424668|2424818|2424331|2424606|N|N|N|N|N| +2424697|AAAAAAAAJHPPECAA|1926-06-30|317|1383|107|1926|3|6|30|2|1926|107|1383|Wednesday|1926Q2|N|N|N|2424668|2424818|2424332|2424607|N|N|N|N|N| +2424698|AAAAAAAAKHPPECAA|1926-07-01|318|1383|107|1926|4|7|1|2|1926|107|1383|Thursday|1926Q2|N|N|N|2424698|2424878|2424333|2424607|N|N|N|N|N| +2424699|AAAAAAAALHPPECAA|1926-07-02|318|1383|107|1926|5|7|2|3|1926|107|1383|Friday|1926Q3|N|Y|N|2424698|2424878|2424334|2424608|N|N|N|N|N| +2424700|AAAAAAAAMHPPECAA|1926-07-03|318|1383|107|1926|6|7|3|3|1926|107|1383|Saturday|1926Q3|N|Y|N|2424698|2424878|2424335|2424609|N|N|N|N|N| +2424701|AAAAAAAANHPPECAA|1926-07-04|318|1383|107|1926|0|7|4|3|1926|107|1383|Sunday|1926Q3|N|N|N|2424698|2424878|2424336|2424610|N|N|N|N|N| +2424702|AAAAAAAAOHPPECAA|1926-07-05|318|1383|107|1926|1|7|5|3|1926|107|1383|Monday|1926Q3|Y|N|N|2424698|2424878|2424337|2424611|N|N|N|N|N| +2424703|AAAAAAAAPHPPECAA|1926-07-06|318|1384|107|1926|2|7|6|3|1926|107|1384|Tuesday|1926Q3|N|N|Y|2424698|2424878|2424338|2424612|N|N|N|N|N| +2424704|AAAAAAAAAIPPECAA|1926-07-07|318|1384|107|1926|3|7|7|3|1926|107|1384|Wednesday|1926Q3|N|N|N|2424698|2424878|2424339|2424613|N|N|N|N|N| +2424705|AAAAAAAABIPPECAA|1926-07-08|318|1384|107|1926|4|7|8|3|1926|107|1384|Thursday|1926Q3|N|N|N|2424698|2424878|2424340|2424614|N|N|N|N|N| +2424706|AAAAAAAACIPPECAA|1926-07-09|318|1384|107|1926|5|7|9|3|1926|107|1384|Friday|1926Q3|N|Y|N|2424698|2424878|2424341|2424615|N|N|N|N|N| +2424707|AAAAAAAADIPPECAA|1926-07-10|318|1384|107|1926|6|7|10|3|1926|107|1384|Saturday|1926Q3|N|Y|N|2424698|2424878|2424342|2424616|N|N|N|N|N| +2424708|AAAAAAAAEIPPECAA|1926-07-11|318|1384|107|1926|0|7|11|3|1926|107|1384|Sunday|1926Q3|N|N|N|2424698|2424878|2424343|2424617|N|N|N|N|N| +2424709|AAAAAAAAFIPPECAA|1926-07-12|318|1384|107|1926|1|7|12|3|1926|107|1384|Monday|1926Q3|N|N|N|2424698|2424878|2424344|2424618|N|N|N|N|N| +2424710|AAAAAAAAGIPPECAA|1926-07-13|318|1385|107|1926|2|7|13|3|1926|107|1385|Tuesday|1926Q3|N|N|N|2424698|2424878|2424345|2424619|N|N|N|N|N| +2424711|AAAAAAAAHIPPECAA|1926-07-14|318|1385|107|1926|3|7|14|3|1926|107|1385|Wednesday|1926Q3|N|N|N|2424698|2424878|2424346|2424620|N|N|N|N|N| +2424712|AAAAAAAAIIPPECAA|1926-07-15|318|1385|107|1926|4|7|15|3|1926|107|1385|Thursday|1926Q3|N|N|N|2424698|2424878|2424347|2424621|N|N|N|N|N| +2424713|AAAAAAAAJIPPECAA|1926-07-16|318|1385|107|1926|5|7|16|3|1926|107|1385|Friday|1926Q3|N|Y|N|2424698|2424878|2424348|2424622|N|N|N|N|N| +2424714|AAAAAAAAKIPPECAA|1926-07-17|318|1385|107|1926|6|7|17|3|1926|107|1385|Saturday|1926Q3|N|Y|N|2424698|2424878|2424349|2424623|N|N|N|N|N| +2424715|AAAAAAAALIPPECAA|1926-07-18|318|1385|107|1926|0|7|18|3|1926|107|1385|Sunday|1926Q3|N|N|N|2424698|2424878|2424350|2424624|N|N|N|N|N| +2424716|AAAAAAAAMIPPECAA|1926-07-19|318|1385|107|1926|1|7|19|3|1926|107|1385|Monday|1926Q3|N|N|N|2424698|2424878|2424351|2424625|N|N|N|N|N| +2424717|AAAAAAAANIPPECAA|1926-07-20|318|1386|107|1926|2|7|20|3|1926|107|1386|Tuesday|1926Q3|N|N|N|2424698|2424878|2424352|2424626|N|N|N|N|N| +2424718|AAAAAAAAOIPPECAA|1926-07-21|318|1386|107|1926|3|7|21|3|1926|107|1386|Wednesday|1926Q3|N|N|N|2424698|2424878|2424353|2424627|N|N|N|N|N| +2424719|AAAAAAAAPIPPECAA|1926-07-22|318|1386|107|1926|4|7|22|3|1926|107|1386|Thursday|1926Q3|N|N|N|2424698|2424878|2424354|2424628|N|N|N|N|N| +2424720|AAAAAAAAAJPPECAA|1926-07-23|318|1386|107|1926|5|7|23|3|1926|107|1386|Friday|1926Q3|N|Y|N|2424698|2424878|2424355|2424629|N|N|N|N|N| +2424721|AAAAAAAABJPPECAA|1926-07-24|318|1386|107|1926|6|7|24|3|1926|107|1386|Saturday|1926Q3|N|Y|N|2424698|2424878|2424356|2424630|N|N|N|N|N| +2424722|AAAAAAAACJPPECAA|1926-07-25|318|1386|107|1926|0|7|25|3|1926|107|1386|Sunday|1926Q3|N|N|N|2424698|2424878|2424357|2424631|N|N|N|N|N| +2424723|AAAAAAAADJPPECAA|1926-07-26|318|1386|107|1926|1|7|26|3|1926|107|1386|Monday|1926Q3|N|N|N|2424698|2424878|2424358|2424632|N|N|N|N|N| +2424724|AAAAAAAAEJPPECAA|1926-07-27|318|1387|107|1926|2|7|27|3|1926|107|1387|Tuesday|1926Q3|N|N|N|2424698|2424878|2424359|2424633|N|N|N|N|N| +2424725|AAAAAAAAFJPPECAA|1926-07-28|318|1387|107|1926|3|7|28|3|1926|107|1387|Wednesday|1926Q3|N|N|N|2424698|2424878|2424360|2424634|N|N|N|N|N| +2424726|AAAAAAAAGJPPECAA|1926-07-29|318|1387|107|1926|4|7|29|3|1926|107|1387|Thursday|1926Q3|N|N|N|2424698|2424878|2424361|2424635|N|N|N|N|N| +2424727|AAAAAAAAHJPPECAA|1926-07-30|318|1387|107|1926|5|7|30|3|1926|107|1387|Friday|1926Q3|N|Y|N|2424698|2424878|2424362|2424636|N|N|N|N|N| +2424728|AAAAAAAAIJPPECAA|1926-07-31|318|1387|107|1926|6|7|31|3|1926|107|1387|Saturday|1926Q3|N|Y|N|2424698|2424878|2424363|2424637|N|N|N|N|N| +2424729|AAAAAAAAJJPPECAA|1926-08-01|319|1387|107|1926|0|8|1|3|1926|107|1387|Sunday|1926Q3|N|N|N|2424729|2424940|2424364|2424638|N|N|N|N|N| +2424730|AAAAAAAAKJPPECAA|1926-08-02|319|1387|107|1926|1|8|2|3|1926|107|1387|Monday|1926Q3|N|N|N|2424729|2424940|2424365|2424639|N|N|N|N|N| +2424731|AAAAAAAALJPPECAA|1926-08-03|319|1388|107|1926|2|8|3|3|1926|107|1388|Tuesday|1926Q3|N|N|N|2424729|2424940|2424366|2424640|N|N|N|N|N| +2424732|AAAAAAAAMJPPECAA|1926-08-04|319|1388|107|1926|3|8|4|3|1926|107|1388|Wednesday|1926Q3|N|N|N|2424729|2424940|2424367|2424641|N|N|N|N|N| +2424733|AAAAAAAANJPPECAA|1926-08-05|319|1388|107|1926|4|8|5|3|1926|107|1388|Thursday|1926Q3|N|N|N|2424729|2424940|2424368|2424642|N|N|N|N|N| +2424734|AAAAAAAAOJPPECAA|1926-08-06|319|1388|107|1926|5|8|6|3|1926|107|1388|Friday|1926Q3|N|Y|N|2424729|2424940|2424369|2424643|N|N|N|N|N| +2424735|AAAAAAAAPJPPECAA|1926-08-07|319|1388|107|1926|6|8|7|3|1926|107|1388|Saturday|1926Q3|N|Y|N|2424729|2424940|2424370|2424644|N|N|N|N|N| +2424736|AAAAAAAAAKPPECAA|1926-08-08|319|1388|107|1926|0|8|8|3|1926|107|1388|Sunday|1926Q3|N|N|N|2424729|2424940|2424371|2424645|N|N|N|N|N| +2424737|AAAAAAAABKPPECAA|1926-08-09|319|1388|107|1926|1|8|9|3|1926|107|1388|Monday|1926Q3|N|N|N|2424729|2424940|2424372|2424646|N|N|N|N|N| +2424738|AAAAAAAACKPPECAA|1926-08-10|319|1389|107|1926|2|8|10|3|1926|107|1389|Tuesday|1926Q3|N|N|N|2424729|2424940|2424373|2424647|N|N|N|N|N| +2424739|AAAAAAAADKPPECAA|1926-08-11|319|1389|107|1926|3|8|11|3|1926|107|1389|Wednesday|1926Q3|N|N|N|2424729|2424940|2424374|2424648|N|N|N|N|N| +2424740|AAAAAAAAEKPPECAA|1926-08-12|319|1389|107|1926|4|8|12|3|1926|107|1389|Thursday|1926Q3|N|N|N|2424729|2424940|2424375|2424649|N|N|N|N|N| +2424741|AAAAAAAAFKPPECAA|1926-08-13|319|1389|107|1926|5|8|13|3|1926|107|1389|Friday|1926Q3|N|Y|N|2424729|2424940|2424376|2424650|N|N|N|N|N| +2424742|AAAAAAAAGKPPECAA|1926-08-14|319|1389|107|1926|6|8|14|3|1926|107|1389|Saturday|1926Q3|N|Y|N|2424729|2424940|2424377|2424651|N|N|N|N|N| +2424743|AAAAAAAAHKPPECAA|1926-08-15|319|1389|107|1926|0|8|15|3|1926|107|1389|Sunday|1926Q3|N|N|N|2424729|2424940|2424378|2424652|N|N|N|N|N| +2424744|AAAAAAAAIKPPECAA|1926-08-16|319|1389|107|1926|1|8|16|3|1926|107|1389|Monday|1926Q3|N|N|N|2424729|2424940|2424379|2424653|N|N|N|N|N| +2424745|AAAAAAAAJKPPECAA|1926-08-17|319|1390|107|1926|2|8|17|3|1926|107|1390|Tuesday|1926Q3|N|N|N|2424729|2424940|2424380|2424654|N|N|N|N|N| +2424746|AAAAAAAAKKPPECAA|1926-08-18|319|1390|107|1926|3|8|18|3|1926|107|1390|Wednesday|1926Q3|N|N|N|2424729|2424940|2424381|2424655|N|N|N|N|N| +2424747|AAAAAAAALKPPECAA|1926-08-19|319|1390|107|1926|4|8|19|3|1926|107|1390|Thursday|1926Q3|N|N|N|2424729|2424940|2424382|2424656|N|N|N|N|N| +2424748|AAAAAAAAMKPPECAA|1926-08-20|319|1390|107|1926|5|8|20|3|1926|107|1390|Friday|1926Q3|N|Y|N|2424729|2424940|2424383|2424657|N|N|N|N|N| +2424749|AAAAAAAANKPPECAA|1926-08-21|319|1390|107|1926|6|8|21|3|1926|107|1390|Saturday|1926Q3|N|Y|N|2424729|2424940|2424384|2424658|N|N|N|N|N| +2424750|AAAAAAAAOKPPECAA|1926-08-22|319|1390|107|1926|0|8|22|3|1926|107|1390|Sunday|1926Q3|N|N|N|2424729|2424940|2424385|2424659|N|N|N|N|N| +2424751|AAAAAAAAPKPPECAA|1926-08-23|319|1390|107|1926|1|8|23|3|1926|107|1390|Monday|1926Q3|N|N|N|2424729|2424940|2424386|2424660|N|N|N|N|N| +2424752|AAAAAAAAALPPECAA|1926-08-24|319|1391|107|1926|2|8|24|3|1926|107|1391|Tuesday|1926Q3|N|N|N|2424729|2424940|2424387|2424661|N|N|N|N|N| +2424753|AAAAAAAABLPPECAA|1926-08-25|319|1391|107|1926|3|8|25|3|1926|107|1391|Wednesday|1926Q3|N|N|N|2424729|2424940|2424388|2424662|N|N|N|N|N| +2424754|AAAAAAAACLPPECAA|1926-08-26|319|1391|107|1926|4|8|26|3|1926|107|1391|Thursday|1926Q3|N|N|N|2424729|2424940|2424389|2424663|N|N|N|N|N| +2424755|AAAAAAAADLPPECAA|1926-08-27|319|1391|107|1926|5|8|27|3|1926|107|1391|Friday|1926Q3|N|Y|N|2424729|2424940|2424390|2424664|N|N|N|N|N| +2424756|AAAAAAAAELPPECAA|1926-08-28|319|1391|107|1926|6|8|28|3|1926|107|1391|Saturday|1926Q3|N|Y|N|2424729|2424940|2424391|2424665|N|N|N|N|N| +2424757|AAAAAAAAFLPPECAA|1926-08-29|319|1391|107|1926|0|8|29|3|1926|107|1391|Sunday|1926Q3|N|N|N|2424729|2424940|2424392|2424666|N|N|N|N|N| +2424758|AAAAAAAAGLPPECAA|1926-08-30|319|1391|107|1926|1|8|30|3|1926|107|1391|Monday|1926Q3|N|N|N|2424729|2424940|2424393|2424667|N|N|N|N|N| +2424759|AAAAAAAAHLPPECAA|1926-08-31|319|1392|107|1926|2|8|31|3|1926|107|1392|Tuesday|1926Q3|N|N|N|2424729|2424940|2424394|2424668|N|N|N|N|N| +2424760|AAAAAAAAILPPECAA|1926-09-01|320|1392|108|1926|3|9|1|3|1926|108|1392|Wednesday|1926Q3|N|N|N|2424760|2425002|2424395|2424669|N|N|N|N|N| +2424761|AAAAAAAAJLPPECAA|1926-09-02|320|1392|108|1926|4|9|2|3|1926|108|1392|Thursday|1926Q3|N|N|N|2424760|2425002|2424396|2424670|N|N|N|N|N| +2424762|AAAAAAAAKLPPECAA|1926-09-03|320|1392|108|1926|5|9|3|3|1926|108|1392|Friday|1926Q3|N|Y|N|2424760|2425002|2424397|2424671|N|N|N|N|N| +2424763|AAAAAAAALLPPECAA|1926-09-04|320|1392|108|1926|6|9|4|3|1926|108|1392|Saturday|1926Q3|N|Y|N|2424760|2425002|2424398|2424672|N|N|N|N|N| +2424764|AAAAAAAAMLPPECAA|1926-09-05|320|1392|108|1926|0|9|5|3|1926|108|1392|Sunday|1926Q3|N|N|N|2424760|2425002|2424399|2424673|N|N|N|N|N| +2424765|AAAAAAAANLPPECAA|1926-09-06|320|1392|108|1926|1|9|6|3|1926|108|1392|Monday|1926Q3|N|N|N|2424760|2425002|2424400|2424674|N|N|N|N|N| +2424766|AAAAAAAAOLPPECAA|1926-09-07|320|1393|108|1926|2|9|7|3|1926|108|1393|Tuesday|1926Q3|N|N|N|2424760|2425002|2424401|2424675|N|N|N|N|N| +2424767|AAAAAAAAPLPPECAA|1926-09-08|320|1393|108|1926|3|9|8|3|1926|108|1393|Wednesday|1926Q3|N|N|N|2424760|2425002|2424402|2424676|N|N|N|N|N| +2424768|AAAAAAAAAMPPECAA|1926-09-09|320|1393|108|1926|4|9|9|3|1926|108|1393|Thursday|1926Q3|N|N|N|2424760|2425002|2424403|2424677|N|N|N|N|N| +2424769|AAAAAAAABMPPECAA|1926-09-10|320|1393|108|1926|5|9|10|3|1926|108|1393|Friday|1926Q3|N|Y|N|2424760|2425002|2424404|2424678|N|N|N|N|N| +2424770|AAAAAAAACMPPECAA|1926-09-11|320|1393|108|1926|6|9|11|3|1926|108|1393|Saturday|1926Q3|N|Y|N|2424760|2425002|2424405|2424679|N|N|N|N|N| +2424771|AAAAAAAADMPPECAA|1926-09-12|320|1393|108|1926|0|9|12|3|1926|108|1393|Sunday|1926Q3|N|N|N|2424760|2425002|2424406|2424680|N|N|N|N|N| +2424772|AAAAAAAAEMPPECAA|1926-09-13|320|1393|108|1926|1|9|13|3|1926|108|1393|Monday|1926Q3|N|N|N|2424760|2425002|2424407|2424681|N|N|N|N|N| +2424773|AAAAAAAAFMPPECAA|1926-09-14|320|1394|108|1926|2|9|14|3|1926|108|1394|Tuesday|1926Q3|N|N|N|2424760|2425002|2424408|2424682|N|N|N|N|N| +2424774|AAAAAAAAGMPPECAA|1926-09-15|320|1394|108|1926|3|9|15|3|1926|108|1394|Wednesday|1926Q3|N|N|N|2424760|2425002|2424409|2424683|N|N|N|N|N| +2424775|AAAAAAAAHMPPECAA|1926-09-16|320|1394|108|1926|4|9|16|3|1926|108|1394|Thursday|1926Q3|N|N|N|2424760|2425002|2424410|2424684|N|N|N|N|N| +2424776|AAAAAAAAIMPPECAA|1926-09-17|320|1394|108|1926|5|9|17|3|1926|108|1394|Friday|1926Q3|N|Y|N|2424760|2425002|2424411|2424685|N|N|N|N|N| +2424777|AAAAAAAAJMPPECAA|1926-09-18|320|1394|108|1926|6|9|18|3|1926|108|1394|Saturday|1926Q3|N|Y|N|2424760|2425002|2424412|2424686|N|N|N|N|N| +2424778|AAAAAAAAKMPPECAA|1926-09-19|320|1394|108|1926|0|9|19|3|1926|108|1394|Sunday|1926Q3|N|N|N|2424760|2425002|2424413|2424687|N|N|N|N|N| +2424779|AAAAAAAALMPPECAA|1926-09-20|320|1394|108|1926|1|9|20|3|1926|108|1394|Monday|1926Q3|N|N|N|2424760|2425002|2424414|2424688|N|N|N|N|N| +2424780|AAAAAAAAMMPPECAA|1926-09-21|320|1395|108|1926|2|9|21|3|1926|108|1395|Tuesday|1926Q3|N|N|N|2424760|2425002|2424415|2424689|N|N|N|N|N| +2424781|AAAAAAAANMPPECAA|1926-09-22|320|1395|108|1926|3|9|22|3|1926|108|1395|Wednesday|1926Q3|N|N|N|2424760|2425002|2424416|2424690|N|N|N|N|N| +2424782|AAAAAAAAOMPPECAA|1926-09-23|320|1395|108|1926|4|9|23|3|1926|108|1395|Thursday|1926Q3|N|N|N|2424760|2425002|2424417|2424691|N|N|N|N|N| +2424783|AAAAAAAAPMPPECAA|1926-09-24|320|1395|108|1926|5|9|24|3|1926|108|1395|Friday|1926Q3|N|Y|N|2424760|2425002|2424418|2424692|N|N|N|N|N| +2424784|AAAAAAAAANPPECAA|1926-09-25|320|1395|108|1926|6|9|25|3|1926|108|1395|Saturday|1926Q3|N|Y|N|2424760|2425002|2424419|2424693|N|N|N|N|N| +2424785|AAAAAAAABNPPECAA|1926-09-26|320|1395|108|1926|0|9|26|3|1926|108|1395|Sunday|1926Q3|N|N|N|2424760|2425002|2424420|2424694|N|N|N|N|N| +2424786|AAAAAAAACNPPECAA|1926-09-27|320|1395|108|1926|1|9|27|3|1926|108|1395|Monday|1926Q3|N|N|N|2424760|2425002|2424421|2424695|N|N|N|N|N| +2424787|AAAAAAAADNPPECAA|1926-09-28|320|1396|108|1926|2|9|28|3|1926|108|1396|Tuesday|1926Q3|N|N|N|2424760|2425002|2424422|2424696|N|N|N|N|N| +2424788|AAAAAAAAENPPECAA|1926-09-29|320|1396|108|1926|3|9|29|3|1926|108|1396|Wednesday|1926Q3|N|N|N|2424760|2425002|2424423|2424697|N|N|N|N|N| +2424789|AAAAAAAAFNPPECAA|1926-09-30|320|1396|108|1926|4|9|30|3|1926|108|1396|Thursday|1926Q3|N|N|N|2424760|2425002|2424424|2424698|N|N|N|N|N| +2424790|AAAAAAAAGNPPECAA|1926-10-01|321|1396|108|1926|5|10|1|3|1926|108|1396|Friday|1926Q3|N|Y|N|2424790|2425062|2424425|2424698|N|N|N|N|N| +2424791|AAAAAAAAHNPPECAA|1926-10-02|321|1396|108|1926|6|10|2|4|1926|108|1396|Saturday|1926Q4|N|Y|N|2424790|2425062|2424426|2424699|N|N|N|N|N| +2424792|AAAAAAAAINPPECAA|1926-10-03|321|1396|108|1926|0|10|3|4|1926|108|1396|Sunday|1926Q4|N|N|N|2424790|2425062|2424427|2424700|N|N|N|N|N| +2424793|AAAAAAAAJNPPECAA|1926-10-04|321|1396|108|1926|1|10|4|4|1926|108|1396|Monday|1926Q4|N|N|N|2424790|2425062|2424428|2424701|N|N|N|N|N| +2424794|AAAAAAAAKNPPECAA|1926-10-05|321|1397|108|1926|2|10|5|4|1926|108|1397|Tuesday|1926Q4|N|N|N|2424790|2425062|2424429|2424702|N|N|N|N|N| +2424795|AAAAAAAALNPPECAA|1926-10-06|321|1397|108|1926|3|10|6|4|1926|108|1397|Wednesday|1926Q4|N|N|N|2424790|2425062|2424430|2424703|N|N|N|N|N| +2424796|AAAAAAAAMNPPECAA|1926-10-07|321|1397|108|1926|4|10|7|4|1926|108|1397|Thursday|1926Q4|N|N|N|2424790|2425062|2424431|2424704|N|N|N|N|N| +2424797|AAAAAAAANNPPECAA|1926-10-08|321|1397|108|1926|5|10|8|4|1926|108|1397|Friday|1926Q4|N|Y|N|2424790|2425062|2424432|2424705|N|N|N|N|N| +2424798|AAAAAAAAONPPECAA|1926-10-09|321|1397|108|1926|6|10|9|4|1926|108|1397|Saturday|1926Q4|N|Y|N|2424790|2425062|2424433|2424706|N|N|N|N|N| +2424799|AAAAAAAAPNPPECAA|1926-10-10|321|1397|108|1926|0|10|10|4|1926|108|1397|Sunday|1926Q4|N|N|N|2424790|2425062|2424434|2424707|N|N|N|N|N| +2424800|AAAAAAAAAOPPECAA|1926-10-11|321|1397|108|1926|1|10|11|4|1926|108|1397|Monday|1926Q4|N|N|N|2424790|2425062|2424435|2424708|N|N|N|N|N| +2424801|AAAAAAAABOPPECAA|1926-10-12|321|1398|108|1926|2|10|12|4|1926|108|1398|Tuesday|1926Q4|N|N|N|2424790|2425062|2424436|2424709|N|N|N|N|N| +2424802|AAAAAAAACOPPECAA|1926-10-13|321|1398|108|1926|3|10|13|4|1926|108|1398|Wednesday|1926Q4|N|N|N|2424790|2425062|2424437|2424710|N|N|N|N|N| +2424803|AAAAAAAADOPPECAA|1926-10-14|321|1398|108|1926|4|10|14|4|1926|108|1398|Thursday|1926Q4|N|N|N|2424790|2425062|2424438|2424711|N|N|N|N|N| +2424804|AAAAAAAAEOPPECAA|1926-10-15|321|1398|108|1926|5|10|15|4|1926|108|1398|Friday|1926Q4|N|Y|N|2424790|2425062|2424439|2424712|N|N|N|N|N| +2424805|AAAAAAAAFOPPECAA|1926-10-16|321|1398|108|1926|6|10|16|4|1926|108|1398|Saturday|1926Q4|N|Y|N|2424790|2425062|2424440|2424713|N|N|N|N|N| +2424806|AAAAAAAAGOPPECAA|1926-10-17|321|1398|108|1926|0|10|17|4|1926|108|1398|Sunday|1926Q4|N|N|N|2424790|2425062|2424441|2424714|N|N|N|N|N| +2424807|AAAAAAAAHOPPECAA|1926-10-18|321|1398|108|1926|1|10|18|4|1926|108|1398|Monday|1926Q4|N|N|N|2424790|2425062|2424442|2424715|N|N|N|N|N| +2424808|AAAAAAAAIOPPECAA|1926-10-19|321|1399|108|1926|2|10|19|4|1926|108|1399|Tuesday|1926Q4|N|N|N|2424790|2425062|2424443|2424716|N|N|N|N|N| +2424809|AAAAAAAAJOPPECAA|1926-10-20|321|1399|108|1926|3|10|20|4|1926|108|1399|Wednesday|1926Q4|N|N|N|2424790|2425062|2424444|2424717|N|N|N|N|N| +2424810|AAAAAAAAKOPPECAA|1926-10-21|321|1399|108|1926|4|10|21|4|1926|108|1399|Thursday|1926Q4|N|N|N|2424790|2425062|2424445|2424718|N|N|N|N|N| +2424811|AAAAAAAALOPPECAA|1926-10-22|321|1399|108|1926|5|10|22|4|1926|108|1399|Friday|1926Q4|N|Y|N|2424790|2425062|2424446|2424719|N|N|N|N|N| +2424812|AAAAAAAAMOPPECAA|1926-10-23|321|1399|108|1926|6|10|23|4|1926|108|1399|Saturday|1926Q4|N|Y|N|2424790|2425062|2424447|2424720|N|N|N|N|N| +2424813|AAAAAAAANOPPECAA|1926-10-24|321|1399|108|1926|0|10|24|4|1926|108|1399|Sunday|1926Q4|N|N|N|2424790|2425062|2424448|2424721|N|N|N|N|N| +2424814|AAAAAAAAOOPPECAA|1926-10-25|321|1399|108|1926|1|10|25|4|1926|108|1399|Monday|1926Q4|N|N|N|2424790|2425062|2424449|2424722|N|N|N|N|N| +2424815|AAAAAAAAPOPPECAA|1926-10-26|321|1400|108|1926|2|10|26|4|1926|108|1400|Tuesday|1926Q4|N|N|N|2424790|2425062|2424450|2424723|N|N|N|N|N| +2424816|AAAAAAAAAPPPECAA|1926-10-27|321|1400|108|1926|3|10|27|4|1926|108|1400|Wednesday|1926Q4|N|N|N|2424790|2425062|2424451|2424724|N|N|N|N|N| +2424817|AAAAAAAABPPPECAA|1926-10-28|321|1400|108|1926|4|10|28|4|1926|108|1400|Thursday|1926Q4|N|N|N|2424790|2425062|2424452|2424725|N|N|N|N|N| +2424818|AAAAAAAACPPPECAA|1926-10-29|321|1400|108|1926|5|10|29|4|1926|108|1400|Friday|1926Q4|N|Y|N|2424790|2425062|2424453|2424726|N|N|N|N|N| +2424819|AAAAAAAADPPPECAA|1926-10-30|321|1400|108|1926|6|10|30|4|1926|108|1400|Saturday|1926Q4|N|Y|N|2424790|2425062|2424454|2424727|N|N|N|N|N| +2424820|AAAAAAAAEPPPECAA|1926-10-31|321|1400|108|1926|0|10|31|4|1926|108|1400|Sunday|1926Q4|N|N|N|2424790|2425062|2424455|2424728|N|N|N|N|N| +2424821|AAAAAAAAFPPPECAA|1926-11-01|322|1400|108|1926|1|11|1|4|1926|108|1400|Monday|1926Q4|N|N|N|2424821|2425124|2424456|2424729|N|N|N|N|N| +2424822|AAAAAAAAGPPPECAA|1926-11-02|322|1401|108|1926|2|11|2|4|1926|108|1401|Tuesday|1926Q4|N|N|N|2424821|2425124|2424457|2424730|N|N|N|N|N| +2424823|AAAAAAAAHPPPECAA|1926-11-03|322|1401|108|1926|3|11|3|4|1926|108|1401|Wednesday|1926Q4|N|N|N|2424821|2425124|2424458|2424731|N|N|N|N|N| +2424824|AAAAAAAAIPPPECAA|1926-11-04|322|1401|108|1926|4|11|4|4|1926|108|1401|Thursday|1926Q4|N|N|N|2424821|2425124|2424459|2424732|N|N|N|N|N| +2424825|AAAAAAAAJPPPECAA|1926-11-05|322|1401|108|1926|5|11|5|4|1926|108|1401|Friday|1926Q4|N|Y|N|2424821|2425124|2424460|2424733|N|N|N|N|N| +2424826|AAAAAAAAKPPPECAA|1926-11-06|322|1401|108|1926|6|11|6|4|1926|108|1401|Saturday|1926Q4|N|Y|N|2424821|2425124|2424461|2424734|N|N|N|N|N| +2424827|AAAAAAAALPPPECAA|1926-11-07|322|1401|108|1926|0|11|7|4|1926|108|1401|Sunday|1926Q4|N|N|N|2424821|2425124|2424462|2424735|N|N|N|N|N| +2424828|AAAAAAAAMPPPECAA|1926-11-08|322|1401|108|1926|1|11|8|4|1926|108|1401|Monday|1926Q4|N|N|N|2424821|2425124|2424463|2424736|N|N|N|N|N| +2424829|AAAAAAAANPPPECAA|1926-11-09|322|1402|108|1926|2|11|9|4|1926|108|1402|Tuesday|1926Q4|N|N|N|2424821|2425124|2424464|2424737|N|N|N|N|N| +2424830|AAAAAAAAOPPPECAA|1926-11-10|322|1402|108|1926|3|11|10|4|1926|108|1402|Wednesday|1926Q4|N|N|N|2424821|2425124|2424465|2424738|N|N|N|N|N| +2424831|AAAAAAAAPPPPECAA|1926-11-11|322|1402|108|1926|4|11|11|4|1926|108|1402|Thursday|1926Q4|N|N|N|2424821|2425124|2424466|2424739|N|N|N|N|N| +2424832|AAAAAAAAAAAAFCAA|1926-11-12|322|1402|108|1926|5|11|12|4|1926|108|1402|Friday|1926Q4|N|Y|N|2424821|2425124|2424467|2424740|N|N|N|N|N| +2424833|AAAAAAAABAAAFCAA|1926-11-13|322|1402|108|1926|6|11|13|4|1926|108|1402|Saturday|1926Q4|N|Y|N|2424821|2425124|2424468|2424741|N|N|N|N|N| +2424834|AAAAAAAACAAAFCAA|1926-11-14|322|1402|108|1926|0|11|14|4|1926|108|1402|Sunday|1926Q4|N|N|N|2424821|2425124|2424469|2424742|N|N|N|N|N| +2424835|AAAAAAAADAAAFCAA|1926-11-15|322|1402|108|1926|1|11|15|4|1926|108|1402|Monday|1926Q4|N|N|N|2424821|2425124|2424470|2424743|N|N|N|N|N| +2424836|AAAAAAAAEAAAFCAA|1926-11-16|322|1403|108|1926|2|11|16|4|1926|108|1403|Tuesday|1926Q4|N|N|N|2424821|2425124|2424471|2424744|N|N|N|N|N| +2424837|AAAAAAAAFAAAFCAA|1926-11-17|322|1403|108|1926|3|11|17|4|1926|108|1403|Wednesday|1926Q4|N|N|N|2424821|2425124|2424472|2424745|N|N|N|N|N| +2424838|AAAAAAAAGAAAFCAA|1926-11-18|322|1403|108|1926|4|11|18|4|1926|108|1403|Thursday|1926Q4|N|N|N|2424821|2425124|2424473|2424746|N|N|N|N|N| +2424839|AAAAAAAAHAAAFCAA|1926-11-19|322|1403|108|1926|5|11|19|4|1926|108|1403|Friday|1926Q4|N|Y|N|2424821|2425124|2424474|2424747|N|N|N|N|N| +2424840|AAAAAAAAIAAAFCAA|1926-11-20|322|1403|108|1926|6|11|20|4|1926|108|1403|Saturday|1926Q4|N|Y|N|2424821|2425124|2424475|2424748|N|N|N|N|N| +2424841|AAAAAAAAJAAAFCAA|1926-11-21|322|1403|108|1926|0|11|21|4|1926|108|1403|Sunday|1926Q4|N|N|N|2424821|2425124|2424476|2424749|N|N|N|N|N| +2424842|AAAAAAAAKAAAFCAA|1926-11-22|322|1403|108|1926|1|11|22|4|1926|108|1403|Monday|1926Q4|N|N|N|2424821|2425124|2424477|2424750|N|N|N|N|N| +2424843|AAAAAAAALAAAFCAA|1926-11-23|322|1404|108|1926|2|11|23|4|1926|108|1404|Tuesday|1926Q4|N|N|N|2424821|2425124|2424478|2424751|N|N|N|N|N| +2424844|AAAAAAAAMAAAFCAA|1926-11-24|322|1404|108|1926|3|11|24|4|1926|108|1404|Wednesday|1926Q4|N|N|N|2424821|2425124|2424479|2424752|N|N|N|N|N| +2424845|AAAAAAAANAAAFCAA|1926-11-25|322|1404|108|1926|4|11|25|4|1926|108|1404|Thursday|1926Q4|N|N|N|2424821|2425124|2424480|2424753|N|N|N|N|N| +2424846|AAAAAAAAOAAAFCAA|1926-11-26|322|1404|108|1926|5|11|26|4|1926|108|1404|Friday|1926Q4|N|Y|N|2424821|2425124|2424481|2424754|N|N|N|N|N| +2424847|AAAAAAAAPAAAFCAA|1926-11-27|322|1404|108|1926|6|11|27|4|1926|108|1404|Saturday|1926Q4|N|Y|N|2424821|2425124|2424482|2424755|N|N|N|N|N| +2424848|AAAAAAAAABAAFCAA|1926-11-28|322|1404|108|1926|0|11|28|4|1926|108|1404|Sunday|1926Q4|N|N|N|2424821|2425124|2424483|2424756|N|N|N|N|N| +2424849|AAAAAAAABBAAFCAA|1926-11-29|322|1404|108|1926|1|11|29|4|1926|108|1404|Monday|1926Q4|N|N|N|2424821|2425124|2424484|2424757|N|N|N|N|N| +2424850|AAAAAAAACBAAFCAA|1926-11-30|322|1405|108|1926|2|11|30|4|1926|108|1405|Tuesday|1926Q4|N|N|N|2424821|2425124|2424485|2424758|N|N|N|N|N| +2424851|AAAAAAAADBAAFCAA|1926-12-01|323|1405|109|1926|3|12|1|4|1926|109|1405|Wednesday|1926Q4|N|N|N|2424851|2425184|2424486|2424759|N|N|N|N|N| +2424852|AAAAAAAAEBAAFCAA|1926-12-02|323|1405|109|1926|4|12|2|4|1926|109|1405|Thursday|1926Q4|N|N|N|2424851|2425184|2424487|2424760|N|N|N|N|N| +2424853|AAAAAAAAFBAAFCAA|1926-12-03|323|1405|109|1926|5|12|3|4|1926|109|1405|Friday|1926Q4|N|Y|N|2424851|2425184|2424488|2424761|N|N|N|N|N| +2424854|AAAAAAAAGBAAFCAA|1926-12-04|323|1405|109|1926|6|12|4|4|1926|109|1405|Saturday|1926Q4|N|Y|N|2424851|2425184|2424489|2424762|N|N|N|N|N| +2424855|AAAAAAAAHBAAFCAA|1926-12-05|323|1405|109|1926|0|12|5|4|1926|109|1405|Sunday|1926Q4|N|N|N|2424851|2425184|2424490|2424763|N|N|N|N|N| +2424856|AAAAAAAAIBAAFCAA|1926-12-06|323|1405|109|1926|1|12|6|4|1926|109|1405|Monday|1926Q4|N|N|N|2424851|2425184|2424491|2424764|N|N|N|N|N| +2424857|AAAAAAAAJBAAFCAA|1926-12-07|323|1406|109|1926|2|12|7|4|1926|109|1406|Tuesday|1926Q4|N|N|N|2424851|2425184|2424492|2424765|N|N|N|N|N| +2424858|AAAAAAAAKBAAFCAA|1926-12-08|323|1406|109|1926|3|12|8|4|1926|109|1406|Wednesday|1926Q4|N|N|N|2424851|2425184|2424493|2424766|N|N|N|N|N| +2424859|AAAAAAAALBAAFCAA|1926-12-09|323|1406|109|1926|4|12|9|4|1926|109|1406|Thursday|1926Q4|N|N|N|2424851|2425184|2424494|2424767|N|N|N|N|N| +2424860|AAAAAAAAMBAAFCAA|1926-12-10|323|1406|109|1926|5|12|10|4|1926|109|1406|Friday|1926Q4|N|Y|N|2424851|2425184|2424495|2424768|N|N|N|N|N| +2424861|AAAAAAAANBAAFCAA|1926-12-11|323|1406|109|1926|6|12|11|4|1926|109|1406|Saturday|1926Q4|N|Y|N|2424851|2425184|2424496|2424769|N|N|N|N|N| +2424862|AAAAAAAAOBAAFCAA|1926-12-12|323|1406|109|1926|0|12|12|4|1926|109|1406|Sunday|1926Q4|N|N|N|2424851|2425184|2424497|2424770|N|N|N|N|N| +2424863|AAAAAAAAPBAAFCAA|1926-12-13|323|1406|109|1926|1|12|13|4|1926|109|1406|Monday|1926Q4|N|N|N|2424851|2425184|2424498|2424771|N|N|N|N|N| +2424864|AAAAAAAAACAAFCAA|1926-12-14|323|1407|109|1926|2|12|14|4|1926|109|1407|Tuesday|1926Q4|N|N|N|2424851|2425184|2424499|2424772|N|N|N|N|N| +2424865|AAAAAAAABCAAFCAA|1926-12-15|323|1407|109|1926|3|12|15|4|1926|109|1407|Wednesday|1926Q4|N|N|N|2424851|2425184|2424500|2424773|N|N|N|N|N| +2424866|AAAAAAAACCAAFCAA|1926-12-16|323|1407|109|1926|4|12|16|4|1926|109|1407|Thursday|1926Q4|N|N|N|2424851|2425184|2424501|2424774|N|N|N|N|N| +2424867|AAAAAAAADCAAFCAA|1926-12-17|323|1407|109|1926|5|12|17|4|1926|109|1407|Friday|1926Q4|N|Y|N|2424851|2425184|2424502|2424775|N|N|N|N|N| +2424868|AAAAAAAAECAAFCAA|1926-12-18|323|1407|109|1926|6|12|18|4|1926|109|1407|Saturday|1926Q4|N|Y|N|2424851|2425184|2424503|2424776|N|N|N|N|N| +2424869|AAAAAAAAFCAAFCAA|1926-12-19|323|1407|109|1926|0|12|19|4|1926|109|1407|Sunday|1926Q4|N|N|N|2424851|2425184|2424504|2424777|N|N|N|N|N| +2424870|AAAAAAAAGCAAFCAA|1926-12-20|323|1407|109|1926|1|12|20|4|1926|109|1407|Monday|1926Q4|N|N|N|2424851|2425184|2424505|2424778|N|N|N|N|N| +2424871|AAAAAAAAHCAAFCAA|1926-12-21|323|1408|109|1926|2|12|21|4|1926|109|1408|Tuesday|1926Q4|N|N|N|2424851|2425184|2424506|2424779|N|N|N|N|N| +2424872|AAAAAAAAICAAFCAA|1926-12-22|323|1408|109|1926|3|12|22|4|1926|109|1408|Wednesday|1926Q4|N|N|N|2424851|2425184|2424507|2424780|N|N|N|N|N| +2424873|AAAAAAAAJCAAFCAA|1926-12-23|323|1408|109|1926|4|12|23|4|1926|109|1408|Thursday|1926Q4|N|N|N|2424851|2425184|2424508|2424781|N|N|N|N|N| +2424874|AAAAAAAAKCAAFCAA|1926-12-24|323|1408|109|1926|5|12|24|4|1926|109|1408|Friday|1926Q4|N|Y|N|2424851|2425184|2424509|2424782|N|N|N|N|N| +2424875|AAAAAAAALCAAFCAA|1926-12-25|323|1408|109|1926|6|12|25|4|1926|109|1408|Saturday|1926Q4|N|Y|N|2424851|2425184|2424510|2424783|N|N|N|N|N| +2424876|AAAAAAAAMCAAFCAA|1926-12-26|323|1408|109|1926|0|12|26|4|1926|109|1408|Sunday|1926Q4|Y|N|N|2424851|2425184|2424511|2424784|N|N|N|N|N| +2424877|AAAAAAAANCAAFCAA|1926-12-27|323|1408|109|1926|1|12|27|4|1926|109|1408|Monday|1926Q4|N|N|Y|2424851|2425184|2424512|2424785|N|N|N|N|N| +2424878|AAAAAAAAOCAAFCAA|1926-12-28|323|1409|109|1926|2|12|28|4|1926|109|1409|Tuesday|1926Q4|N|N|N|2424851|2425184|2424513|2424786|N|N|N|N|N| +2424879|AAAAAAAAPCAAFCAA|1926-12-29|323|1409|109|1926|3|12|29|4|1926|109|1409|Wednesday|1926Q4|N|N|N|2424851|2425184|2424514|2424787|N|N|N|N|N| +2424880|AAAAAAAAADAAFCAA|1926-12-30|323|1409|109|1926|4|12|30|4|1926|109|1409|Thursday|1926Q4|N|N|N|2424851|2425184|2424515|2424788|N|N|N|N|N| +2424881|AAAAAAAABDAAFCAA|1926-12-31|323|1409|109|1926|5|12|31|4|1926|109|1409|Friday|1926Q4|N|Y|N|2424851|2425184|2424516|2424789|N|N|N|N|N| +2424882|AAAAAAAACDAAFCAA|1927-01-01|324|1409|109|1927|6|1|1|1|1927|109|1409|Saturday|1927Q1|Y|Y|N|2424882|2424881|2424517|2424790|N|N|N|N|N| +2424883|AAAAAAAADDAAFCAA|1927-01-02|324|1409|109|1927|0|1|2|1|1927|109|1409|Sunday|1927Q1|N|N|Y|2424882|2424881|2424518|2424791|N|N|N|N|N| +2424884|AAAAAAAAEDAAFCAA|1927-01-03|324|1409|109|1927|1|1|3|1|1927|109|1409|Monday|1927Q1|N|N|N|2424882|2424881|2424519|2424792|N|N|N|N|N| +2424885|AAAAAAAAFDAAFCAA|1927-01-04|324|1410|109|1927|2|1|4|1|1927|109|1410|Tuesday|1927Q1|N|N|N|2424882|2424881|2424520|2424793|N|N|N|N|N| +2424886|AAAAAAAAGDAAFCAA|1927-01-05|324|1410|109|1927|3|1|5|1|1927|109|1410|Wednesday|1927Q1|N|N|N|2424882|2424881|2424521|2424794|N|N|N|N|N| +2424887|AAAAAAAAHDAAFCAA|1927-01-06|324|1410|109|1927|4|1|6|1|1927|109|1410|Thursday|1927Q1|N|N|N|2424882|2424881|2424522|2424795|N|N|N|N|N| +2424888|AAAAAAAAIDAAFCAA|1927-01-07|324|1410|109|1927|5|1|7|1|1927|109|1410|Friday|1927Q1|N|Y|N|2424882|2424881|2424523|2424796|N|N|N|N|N| +2424889|AAAAAAAAJDAAFCAA|1927-01-08|324|1410|109|1927|6|1|8|1|1927|109|1410|Saturday|1927Q1|N|Y|N|2424882|2424881|2424524|2424797|N|N|N|N|N| +2424890|AAAAAAAAKDAAFCAA|1927-01-09|324|1410|109|1927|0|1|9|1|1927|109|1410|Sunday|1927Q1|N|N|N|2424882|2424881|2424525|2424798|N|N|N|N|N| +2424891|AAAAAAAALDAAFCAA|1927-01-10|324|1410|109|1927|1|1|10|1|1927|109|1410|Monday|1927Q1|N|N|N|2424882|2424881|2424526|2424799|N|N|N|N|N| +2424892|AAAAAAAAMDAAFCAA|1927-01-11|324|1411|109|1927|2|1|11|1|1927|109|1411|Tuesday|1927Q1|N|N|N|2424882|2424881|2424527|2424800|N|N|N|N|N| +2424893|AAAAAAAANDAAFCAA|1927-01-12|324|1411|109|1927|3|1|12|1|1927|109|1411|Wednesday|1927Q1|N|N|N|2424882|2424881|2424528|2424801|N|N|N|N|N| +2424894|AAAAAAAAODAAFCAA|1927-01-13|324|1411|109|1927|4|1|13|1|1927|109|1411|Thursday|1927Q1|N|N|N|2424882|2424881|2424529|2424802|N|N|N|N|N| +2424895|AAAAAAAAPDAAFCAA|1927-01-14|324|1411|109|1927|5|1|14|1|1927|109|1411|Friday|1927Q1|N|Y|N|2424882|2424881|2424530|2424803|N|N|N|N|N| +2424896|AAAAAAAAAEAAFCAA|1927-01-15|324|1411|109|1927|6|1|15|1|1927|109|1411|Saturday|1927Q1|N|Y|N|2424882|2424881|2424531|2424804|N|N|N|N|N| +2424897|AAAAAAAABEAAFCAA|1927-01-16|324|1411|109|1927|0|1|16|1|1927|109|1411|Sunday|1927Q1|N|N|N|2424882|2424881|2424532|2424805|N|N|N|N|N| +2424898|AAAAAAAACEAAFCAA|1927-01-17|324|1411|109|1927|1|1|17|1|1927|109|1411|Monday|1927Q1|N|N|N|2424882|2424881|2424533|2424806|N|N|N|N|N| +2424899|AAAAAAAADEAAFCAA|1927-01-18|324|1412|109|1927|2|1|18|1|1927|109|1412|Tuesday|1927Q1|N|N|N|2424882|2424881|2424534|2424807|N|N|N|N|N| +2424900|AAAAAAAAEEAAFCAA|1927-01-19|324|1412|109|1927|3|1|19|1|1927|109|1412|Wednesday|1927Q1|N|N|N|2424882|2424881|2424535|2424808|N|N|N|N|N| +2424901|AAAAAAAAFEAAFCAA|1927-01-20|324|1412|109|1927|4|1|20|1|1927|109|1412|Thursday|1927Q1|N|N|N|2424882|2424881|2424536|2424809|N|N|N|N|N| +2424902|AAAAAAAAGEAAFCAA|1927-01-21|324|1412|109|1927|5|1|21|1|1927|109|1412|Friday|1927Q1|N|Y|N|2424882|2424881|2424537|2424810|N|N|N|N|N| +2424903|AAAAAAAAHEAAFCAA|1927-01-22|324|1412|109|1927|6|1|22|1|1927|109|1412|Saturday|1927Q1|N|Y|N|2424882|2424881|2424538|2424811|N|N|N|N|N| +2424904|AAAAAAAAIEAAFCAA|1927-01-23|324|1412|109|1927|0|1|23|1|1927|109|1412|Sunday|1927Q1|N|N|N|2424882|2424881|2424539|2424812|N|N|N|N|N| +2424905|AAAAAAAAJEAAFCAA|1927-01-24|324|1412|109|1927|1|1|24|1|1927|109|1412|Monday|1927Q1|N|N|N|2424882|2424881|2424540|2424813|N|N|N|N|N| +2424906|AAAAAAAAKEAAFCAA|1927-01-25|324|1413|109|1927|2|1|25|1|1927|109|1413|Tuesday|1927Q1|N|N|N|2424882|2424881|2424541|2424814|N|N|N|N|N| +2424907|AAAAAAAALEAAFCAA|1927-01-26|324|1413|109|1927|3|1|26|1|1927|109|1413|Wednesday|1927Q1|N|N|N|2424882|2424881|2424542|2424815|N|N|N|N|N| +2424908|AAAAAAAAMEAAFCAA|1927-01-27|324|1413|109|1927|4|1|27|1|1927|109|1413|Thursday|1927Q1|N|N|N|2424882|2424881|2424543|2424816|N|N|N|N|N| +2424909|AAAAAAAANEAAFCAA|1927-01-28|324|1413|109|1927|5|1|28|1|1927|109|1413|Friday|1927Q1|N|Y|N|2424882|2424881|2424544|2424817|N|N|N|N|N| +2424910|AAAAAAAAOEAAFCAA|1927-01-29|324|1413|109|1927|6|1|29|1|1927|109|1413|Saturday|1927Q1|N|Y|N|2424882|2424881|2424545|2424818|N|N|N|N|N| +2424911|AAAAAAAAPEAAFCAA|1927-01-30|324|1413|109|1927|0|1|30|1|1927|109|1413|Sunday|1927Q1|N|N|N|2424882|2424881|2424546|2424819|N|N|N|N|N| +2424912|AAAAAAAAAFAAFCAA|1927-01-31|324|1413|109|1927|1|1|31|1|1927|109|1413|Monday|1927Q1|N|N|N|2424882|2424881|2424547|2424820|N|N|N|N|N| +2424913|AAAAAAAABFAAFCAA|1927-02-01|325|1414|109|1927|2|2|1|1|1927|109|1414|Tuesday|1927Q1|N|N|N|2424913|2424943|2424548|2424821|N|N|N|N|N| +2424914|AAAAAAAACFAAFCAA|1927-02-02|325|1414|109|1927|3|2|2|1|1927|109|1414|Wednesday|1927Q1|N|N|N|2424913|2424943|2424549|2424822|N|N|N|N|N| +2424915|AAAAAAAADFAAFCAA|1927-02-03|325|1414|109|1927|4|2|3|1|1927|109|1414|Thursday|1927Q1|N|N|N|2424913|2424943|2424550|2424823|N|N|N|N|N| +2424916|AAAAAAAAEFAAFCAA|1927-02-04|325|1414|109|1927|5|2|4|1|1927|109|1414|Friday|1927Q1|N|Y|N|2424913|2424943|2424551|2424824|N|N|N|N|N| +2424917|AAAAAAAAFFAAFCAA|1927-02-05|325|1414|109|1927|6|2|5|1|1927|109|1414|Saturday|1927Q1|N|Y|N|2424913|2424943|2424552|2424825|N|N|N|N|N| +2424918|AAAAAAAAGFAAFCAA|1927-02-06|325|1414|109|1927|0|2|6|1|1927|109|1414|Sunday|1927Q1|N|N|N|2424913|2424943|2424553|2424826|N|N|N|N|N| +2424919|AAAAAAAAHFAAFCAA|1927-02-07|325|1414|109|1927|1|2|7|1|1927|109|1414|Monday|1927Q1|N|N|N|2424913|2424943|2424554|2424827|N|N|N|N|N| +2424920|AAAAAAAAIFAAFCAA|1927-02-08|325|1415|109|1927|2|2|8|1|1927|109|1415|Tuesday|1927Q1|N|N|N|2424913|2424943|2424555|2424828|N|N|N|N|N| +2424921|AAAAAAAAJFAAFCAA|1927-02-09|325|1415|109|1927|3|2|9|1|1927|109|1415|Wednesday|1927Q1|N|N|N|2424913|2424943|2424556|2424829|N|N|N|N|N| +2424922|AAAAAAAAKFAAFCAA|1927-02-10|325|1415|109|1927|4|2|10|1|1927|109|1415|Thursday|1927Q1|N|N|N|2424913|2424943|2424557|2424830|N|N|N|N|N| +2424923|AAAAAAAALFAAFCAA|1927-02-11|325|1415|109|1927|5|2|11|1|1927|109|1415|Friday|1927Q1|N|Y|N|2424913|2424943|2424558|2424831|N|N|N|N|N| +2424924|AAAAAAAAMFAAFCAA|1927-02-12|325|1415|109|1927|6|2|12|1|1927|109|1415|Saturday|1927Q1|N|Y|N|2424913|2424943|2424559|2424832|N|N|N|N|N| +2424925|AAAAAAAANFAAFCAA|1927-02-13|325|1415|109|1927|0|2|13|1|1927|109|1415|Sunday|1927Q1|N|N|N|2424913|2424943|2424560|2424833|N|N|N|N|N| +2424926|AAAAAAAAOFAAFCAA|1927-02-14|325|1415|109|1927|1|2|14|1|1927|109|1415|Monday|1927Q1|N|N|N|2424913|2424943|2424561|2424834|N|N|N|N|N| +2424927|AAAAAAAAPFAAFCAA|1927-02-15|325|1416|109|1927|2|2|15|1|1927|109|1416|Tuesday|1927Q1|N|N|N|2424913|2424943|2424562|2424835|N|N|N|N|N| +2424928|AAAAAAAAAGAAFCAA|1927-02-16|325|1416|109|1927|3|2|16|1|1927|109|1416|Wednesday|1927Q1|N|N|N|2424913|2424943|2424563|2424836|N|N|N|N|N| +2424929|AAAAAAAABGAAFCAA|1927-02-17|325|1416|109|1927|4|2|17|1|1927|109|1416|Thursday|1927Q1|N|N|N|2424913|2424943|2424564|2424837|N|N|N|N|N| +2424930|AAAAAAAACGAAFCAA|1927-02-18|325|1416|109|1927|5|2|18|1|1927|109|1416|Friday|1927Q1|N|Y|N|2424913|2424943|2424565|2424838|N|N|N|N|N| +2424931|AAAAAAAADGAAFCAA|1927-02-19|325|1416|109|1927|6|2|19|1|1927|109|1416|Saturday|1927Q1|N|Y|N|2424913|2424943|2424566|2424839|N|N|N|N|N| +2424932|AAAAAAAAEGAAFCAA|1927-02-20|325|1416|109|1927|0|2|20|1|1927|109|1416|Sunday|1927Q1|N|N|N|2424913|2424943|2424567|2424840|N|N|N|N|N| +2424933|AAAAAAAAFGAAFCAA|1927-02-21|325|1416|109|1927|1|2|21|1|1927|109|1416|Monday|1927Q1|N|N|N|2424913|2424943|2424568|2424841|N|N|N|N|N| +2424934|AAAAAAAAGGAAFCAA|1927-02-22|325|1417|109|1927|2|2|22|1|1927|109|1417|Tuesday|1927Q1|N|N|N|2424913|2424943|2424569|2424842|N|N|N|N|N| +2424935|AAAAAAAAHGAAFCAA|1927-02-23|325|1417|109|1927|3|2|23|1|1927|109|1417|Wednesday|1927Q1|N|N|N|2424913|2424943|2424570|2424843|N|N|N|N|N| +2424936|AAAAAAAAIGAAFCAA|1927-02-24|325|1417|109|1927|4|2|24|1|1927|109|1417|Thursday|1927Q1|N|N|N|2424913|2424943|2424571|2424844|N|N|N|N|N| +2424937|AAAAAAAAJGAAFCAA|1927-02-25|325|1417|109|1927|5|2|25|1|1927|109|1417|Friday|1927Q1|N|Y|N|2424913|2424943|2424572|2424845|N|N|N|N|N| +2424938|AAAAAAAAKGAAFCAA|1927-02-26|325|1417|109|1927|6|2|26|1|1927|109|1417|Saturday|1927Q1|N|Y|N|2424913|2424943|2424573|2424846|N|N|N|N|N| +2424939|AAAAAAAALGAAFCAA|1927-02-27|325|1417|109|1927|0|2|27|1|1927|109|1417|Sunday|1927Q1|N|N|N|2424913|2424943|2424574|2424847|N|N|N|N|N| +2424940|AAAAAAAAMGAAFCAA|1927-02-28|325|1417|109|1927|1|2|28|1|1927|109|1417|Monday|1927Q1|N|N|N|2424913|2424943|2424575|2424848|N|N|N|N|N| +2424941|AAAAAAAANGAAFCAA|1927-03-01|326|1418|110|1927|2|3|1|1|1927|110|1418|Tuesday|1927Q1|N|N|N|2424941|2424999|2424576|2424849|N|N|N|N|N| +2424942|AAAAAAAAOGAAFCAA|1927-03-02|326|1418|110|1927|3|3|2|1|1927|110|1418|Wednesday|1927Q1|N|N|N|2424941|2424999|2424577|2424850|N|N|N|N|N| +2424943|AAAAAAAAPGAAFCAA|1927-03-03|326|1418|110|1927|4|3|3|1|1927|110|1418|Thursday|1927Q1|N|N|N|2424941|2424999|2424578|2424851|N|N|N|N|N| +2424944|AAAAAAAAAHAAFCAA|1927-03-04|326|1418|110|1927|5|3|4|1|1927|110|1418|Friday|1927Q1|N|Y|N|2424941|2424999|2424579|2424852|N|N|N|N|N| +2424945|AAAAAAAABHAAFCAA|1927-03-05|326|1418|110|1927|6|3|5|1|1927|110|1418|Saturday|1927Q1|N|Y|N|2424941|2424999|2424580|2424853|N|N|N|N|N| +2424946|AAAAAAAACHAAFCAA|1927-03-06|326|1418|110|1927|0|3|6|1|1927|110|1418|Sunday|1927Q1|N|N|N|2424941|2424999|2424581|2424854|N|N|N|N|N| +2424947|AAAAAAAADHAAFCAA|1927-03-07|326|1418|110|1927|1|3|7|1|1927|110|1418|Monday|1927Q1|N|N|N|2424941|2424999|2424582|2424855|N|N|N|N|N| +2424948|AAAAAAAAEHAAFCAA|1927-03-08|326|1419|110|1927|2|3|8|1|1927|110|1419|Tuesday|1927Q1|N|N|N|2424941|2424999|2424583|2424856|N|N|N|N|N| +2424949|AAAAAAAAFHAAFCAA|1927-03-09|326|1419|110|1927|3|3|9|1|1927|110|1419|Wednesday|1927Q1|N|N|N|2424941|2424999|2424584|2424857|N|N|N|N|N| +2424950|AAAAAAAAGHAAFCAA|1927-03-10|326|1419|110|1927|4|3|10|1|1927|110|1419|Thursday|1927Q1|N|N|N|2424941|2424999|2424585|2424858|N|N|N|N|N| +2424951|AAAAAAAAHHAAFCAA|1927-03-11|326|1419|110|1927|5|3|11|1|1927|110|1419|Friday|1927Q1|N|Y|N|2424941|2424999|2424586|2424859|N|N|N|N|N| +2424952|AAAAAAAAIHAAFCAA|1927-03-12|326|1419|110|1927|6|3|12|1|1927|110|1419|Saturday|1927Q1|N|Y|N|2424941|2424999|2424587|2424860|N|N|N|N|N| +2424953|AAAAAAAAJHAAFCAA|1927-03-13|326|1419|110|1927|0|3|13|1|1927|110|1419|Sunday|1927Q1|N|N|N|2424941|2424999|2424588|2424861|N|N|N|N|N| +2424954|AAAAAAAAKHAAFCAA|1927-03-14|326|1419|110|1927|1|3|14|1|1927|110|1419|Monday|1927Q1|N|N|N|2424941|2424999|2424589|2424862|N|N|N|N|N| +2424955|AAAAAAAALHAAFCAA|1927-03-15|326|1420|110|1927|2|3|15|1|1927|110|1420|Tuesday|1927Q1|N|N|N|2424941|2424999|2424590|2424863|N|N|N|N|N| +2424956|AAAAAAAAMHAAFCAA|1927-03-16|326|1420|110|1927|3|3|16|1|1927|110|1420|Wednesday|1927Q1|N|N|N|2424941|2424999|2424591|2424864|N|N|N|N|N| +2424957|AAAAAAAANHAAFCAA|1927-03-17|326|1420|110|1927|4|3|17|1|1927|110|1420|Thursday|1927Q1|N|N|N|2424941|2424999|2424592|2424865|N|N|N|N|N| +2424958|AAAAAAAAOHAAFCAA|1927-03-18|326|1420|110|1927|5|3|18|1|1927|110|1420|Friday|1927Q1|N|Y|N|2424941|2424999|2424593|2424866|N|N|N|N|N| +2424959|AAAAAAAAPHAAFCAA|1927-03-19|326|1420|110|1927|6|3|19|1|1927|110|1420|Saturday|1927Q1|N|Y|N|2424941|2424999|2424594|2424867|N|N|N|N|N| +2424960|AAAAAAAAAIAAFCAA|1927-03-20|326|1420|110|1927|0|3|20|1|1927|110|1420|Sunday|1927Q1|N|N|N|2424941|2424999|2424595|2424868|N|N|N|N|N| +2424961|AAAAAAAABIAAFCAA|1927-03-21|326|1420|110|1927|1|3|21|1|1927|110|1420|Monday|1927Q1|N|N|N|2424941|2424999|2424596|2424869|N|N|N|N|N| +2424962|AAAAAAAACIAAFCAA|1927-03-22|326|1421|110|1927|2|3|22|1|1927|110|1421|Tuesday|1927Q1|N|N|N|2424941|2424999|2424597|2424870|N|N|N|N|N| +2424963|AAAAAAAADIAAFCAA|1927-03-23|326|1421|110|1927|3|3|23|1|1927|110|1421|Wednesday|1927Q1|N|N|N|2424941|2424999|2424598|2424871|N|N|N|N|N| +2424964|AAAAAAAAEIAAFCAA|1927-03-24|326|1421|110|1927|4|3|24|1|1927|110|1421|Thursday|1927Q1|N|N|N|2424941|2424999|2424599|2424872|N|N|N|N|N| +2424965|AAAAAAAAFIAAFCAA|1927-03-25|326|1421|110|1927|5|3|25|1|1927|110|1421|Friday|1927Q1|N|Y|N|2424941|2424999|2424600|2424873|N|N|N|N|N| +2424966|AAAAAAAAGIAAFCAA|1927-03-26|326|1421|110|1927|6|3|26|1|1927|110|1421|Saturday|1927Q1|N|Y|N|2424941|2424999|2424601|2424874|N|N|N|N|N| +2424967|AAAAAAAAHIAAFCAA|1927-03-27|326|1421|110|1927|0|3|27|1|1927|110|1421|Sunday|1927Q1|N|N|N|2424941|2424999|2424602|2424875|N|N|N|N|N| +2424968|AAAAAAAAIIAAFCAA|1927-03-28|326|1421|110|1927|1|3|28|1|1927|110|1421|Monday|1927Q1|N|N|N|2424941|2424999|2424603|2424876|N|N|N|N|N| +2424969|AAAAAAAAJIAAFCAA|1927-03-29|326|1422|110|1927|2|3|29|1|1927|110|1422|Tuesday|1927Q1|N|N|N|2424941|2424999|2424604|2424877|N|N|N|N|N| +2424970|AAAAAAAAKIAAFCAA|1927-03-30|326|1422|110|1927|3|3|30|1|1927|110|1422|Wednesday|1927Q1|N|N|N|2424941|2424999|2424605|2424878|N|N|N|N|N| +2424971|AAAAAAAALIAAFCAA|1927-03-31|326|1422|110|1927|4|3|31|1|1927|110|1422|Thursday|1927Q1|N|N|N|2424941|2424999|2424606|2424879|N|N|N|N|N| +2424972|AAAAAAAAMIAAFCAA|1927-04-01|327|1422|110|1927|5|4|1|1|1927|110|1422|Friday|1927Q1|N|Y|N|2424972|2425061|2424607|2424882|N|N|N|N|N| +2424973|AAAAAAAANIAAFCAA|1927-04-02|327|1422|110|1927|6|4|2|2|1927|110|1422|Saturday|1927Q2|N|Y|N|2424972|2425061|2424608|2424883|N|N|N|N|N| +2424974|AAAAAAAAOIAAFCAA|1927-04-03|327|1422|110|1927|0|4|3|2|1927|110|1422|Sunday|1927Q2|N|N|N|2424972|2425061|2424609|2424884|N|N|N|N|N| +2424975|AAAAAAAAPIAAFCAA|1927-04-04|327|1422|110|1927|1|4|4|2|1927|110|1422|Monday|1927Q2|N|N|N|2424972|2425061|2424610|2424885|N|N|N|N|N| +2424976|AAAAAAAAAJAAFCAA|1927-04-05|327|1423|110|1927|2|4|5|2|1927|110|1423|Tuesday|1927Q2|N|N|N|2424972|2425061|2424611|2424886|N|N|N|N|N| +2424977|AAAAAAAABJAAFCAA|1927-04-06|327|1423|110|1927|3|4|6|2|1927|110|1423|Wednesday|1927Q2|N|N|N|2424972|2425061|2424612|2424887|N|N|N|N|N| +2424978|AAAAAAAACJAAFCAA|1927-04-07|327|1423|110|1927|4|4|7|2|1927|110|1423|Thursday|1927Q2|N|N|N|2424972|2425061|2424613|2424888|N|N|N|N|N| +2424979|AAAAAAAADJAAFCAA|1927-04-08|327|1423|110|1927|5|4|8|2|1927|110|1423|Friday|1927Q2|N|Y|N|2424972|2425061|2424614|2424889|N|N|N|N|N| +2424980|AAAAAAAAEJAAFCAA|1927-04-09|327|1423|110|1927|6|4|9|2|1927|110|1423|Saturday|1927Q2|N|Y|N|2424972|2425061|2424615|2424890|N|N|N|N|N| +2424981|AAAAAAAAFJAAFCAA|1927-04-10|327|1423|110|1927|0|4|10|2|1927|110|1423|Sunday|1927Q2|N|N|N|2424972|2425061|2424616|2424891|N|N|N|N|N| +2424982|AAAAAAAAGJAAFCAA|1927-04-11|327|1423|110|1927|1|4|11|2|1927|110|1423|Monday|1927Q2|N|N|N|2424972|2425061|2424617|2424892|N|N|N|N|N| +2424983|AAAAAAAAHJAAFCAA|1927-04-12|327|1424|110|1927|2|4|12|2|1927|110|1424|Tuesday|1927Q2|N|N|N|2424972|2425061|2424618|2424893|N|N|N|N|N| +2424984|AAAAAAAAIJAAFCAA|1927-04-13|327|1424|110|1927|3|4|13|2|1927|110|1424|Wednesday|1927Q2|N|N|N|2424972|2425061|2424619|2424894|N|N|N|N|N| +2424985|AAAAAAAAJJAAFCAA|1927-04-14|327|1424|110|1927|4|4|14|2|1927|110|1424|Thursday|1927Q2|N|N|N|2424972|2425061|2424620|2424895|N|N|N|N|N| +2424986|AAAAAAAAKJAAFCAA|1927-04-15|327|1424|110|1927|5|4|15|2|1927|110|1424|Friday|1927Q2|N|Y|N|2424972|2425061|2424621|2424896|N|N|N|N|N| +2424987|AAAAAAAALJAAFCAA|1927-04-16|327|1424|110|1927|6|4|16|2|1927|110|1424|Saturday|1927Q2|N|Y|N|2424972|2425061|2424622|2424897|N|N|N|N|N| +2424988|AAAAAAAAMJAAFCAA|1927-04-17|327|1424|110|1927|0|4|17|2|1927|110|1424|Sunday|1927Q2|N|N|N|2424972|2425061|2424623|2424898|N|N|N|N|N| +2424989|AAAAAAAANJAAFCAA|1927-04-18|327|1424|110|1927|1|4|18|2|1927|110|1424|Monday|1927Q2|N|N|N|2424972|2425061|2424624|2424899|N|N|N|N|N| +2424990|AAAAAAAAOJAAFCAA|1927-04-19|327|1425|110|1927|2|4|19|2|1927|110|1425|Tuesday|1927Q2|N|N|N|2424972|2425061|2424625|2424900|N|N|N|N|N| +2424991|AAAAAAAAPJAAFCAA|1927-04-20|327|1425|110|1927|3|4|20|2|1927|110|1425|Wednesday|1927Q2|N|N|N|2424972|2425061|2424626|2424901|N|N|N|N|N| +2424992|AAAAAAAAAKAAFCAA|1927-04-21|327|1425|110|1927|4|4|21|2|1927|110|1425|Thursday|1927Q2|N|N|N|2424972|2425061|2424627|2424902|N|N|N|N|N| +2424993|AAAAAAAABKAAFCAA|1927-04-22|327|1425|110|1927|5|4|22|2|1927|110|1425|Friday|1927Q2|N|Y|N|2424972|2425061|2424628|2424903|N|N|N|N|N| +2424994|AAAAAAAACKAAFCAA|1927-04-23|327|1425|110|1927|6|4|23|2|1927|110|1425|Saturday|1927Q2|N|Y|N|2424972|2425061|2424629|2424904|N|N|N|N|N| +2424995|AAAAAAAADKAAFCAA|1927-04-24|327|1425|110|1927|0|4|24|2|1927|110|1425|Sunday|1927Q2|N|N|N|2424972|2425061|2424630|2424905|N|N|N|N|N| +2424996|AAAAAAAAEKAAFCAA|1927-04-25|327|1425|110|1927|1|4|25|2|1927|110|1425|Monday|1927Q2|N|N|N|2424972|2425061|2424631|2424906|N|N|N|N|N| +2424997|AAAAAAAAFKAAFCAA|1927-04-26|327|1426|110|1927|2|4|26|2|1927|110|1426|Tuesday|1927Q2|N|N|N|2424972|2425061|2424632|2424907|N|N|N|N|N| +2424998|AAAAAAAAGKAAFCAA|1927-04-27|327|1426|110|1927|3|4|27|2|1927|110|1426|Wednesday|1927Q2|N|N|N|2424972|2425061|2424633|2424908|N|N|N|N|N| +2424999|AAAAAAAAHKAAFCAA|1927-04-28|327|1426|110|1927|4|4|28|2|1927|110|1426|Thursday|1927Q2|N|N|N|2424972|2425061|2424634|2424909|N|N|N|N|N| +2425000|AAAAAAAAIKAAFCAA|1927-04-29|327|1426|110|1927|5|4|29|2|1927|110|1426|Friday|1927Q2|N|Y|N|2424972|2425061|2424635|2424910|N|N|N|N|N| +2425001|AAAAAAAAJKAAFCAA|1927-04-30|327|1426|110|1927|6|4|30|2|1927|110|1426|Saturday|1927Q2|N|Y|N|2424972|2425061|2424636|2424911|N|N|N|N|N| +2425002|AAAAAAAAKKAAFCAA|1927-05-01|328|1426|110|1927|0|5|1|2|1927|110|1426|Sunday|1927Q2|N|N|N|2425002|2425121|2424637|2424912|N|N|N|N|N| +2425003|AAAAAAAALKAAFCAA|1927-05-02|328|1426|110|1927|1|5|2|2|1927|110|1426|Monday|1927Q2|N|N|N|2425002|2425121|2424638|2424913|N|N|N|N|N| +2425004|AAAAAAAAMKAAFCAA|1927-05-03|328|1427|110|1927|2|5|3|2|1927|110|1427|Tuesday|1927Q2|N|N|N|2425002|2425121|2424639|2424914|N|N|N|N|N| +2425005|AAAAAAAANKAAFCAA|1927-05-04|328|1427|110|1927|3|5|4|2|1927|110|1427|Wednesday|1927Q2|N|N|N|2425002|2425121|2424640|2424915|N|N|N|N|N| +2425006|AAAAAAAAOKAAFCAA|1927-05-05|328|1427|110|1927|4|5|5|2|1927|110|1427|Thursday|1927Q2|N|N|N|2425002|2425121|2424641|2424916|N|N|N|N|N| +2425007|AAAAAAAAPKAAFCAA|1927-05-06|328|1427|110|1927|5|5|6|2|1927|110|1427|Friday|1927Q2|N|Y|N|2425002|2425121|2424642|2424917|N|N|N|N|N| +2425008|AAAAAAAAALAAFCAA|1927-05-07|328|1427|110|1927|6|5|7|2|1927|110|1427|Saturday|1927Q2|N|Y|N|2425002|2425121|2424643|2424918|N|N|N|N|N| +2425009|AAAAAAAABLAAFCAA|1927-05-08|328|1427|110|1927|0|5|8|2|1927|110|1427|Sunday|1927Q2|N|N|N|2425002|2425121|2424644|2424919|N|N|N|N|N| +2425010|AAAAAAAACLAAFCAA|1927-05-09|328|1427|110|1927|1|5|9|2|1927|110|1427|Monday|1927Q2|N|N|N|2425002|2425121|2424645|2424920|N|N|N|N|N| +2425011|AAAAAAAADLAAFCAA|1927-05-10|328|1428|110|1927|2|5|10|2|1927|110|1428|Tuesday|1927Q2|N|N|N|2425002|2425121|2424646|2424921|N|N|N|N|N| +2425012|AAAAAAAAELAAFCAA|1927-05-11|328|1428|110|1927|3|5|11|2|1927|110|1428|Wednesday|1927Q2|N|N|N|2425002|2425121|2424647|2424922|N|N|N|N|N| +2425013|AAAAAAAAFLAAFCAA|1927-05-12|328|1428|110|1927|4|5|12|2|1927|110|1428|Thursday|1927Q2|N|N|N|2425002|2425121|2424648|2424923|N|N|N|N|N| +2425014|AAAAAAAAGLAAFCAA|1927-05-13|328|1428|110|1927|5|5|13|2|1927|110|1428|Friday|1927Q2|N|Y|N|2425002|2425121|2424649|2424924|N|N|N|N|N| +2425015|AAAAAAAAHLAAFCAA|1927-05-14|328|1428|110|1927|6|5|14|2|1927|110|1428|Saturday|1927Q2|N|Y|N|2425002|2425121|2424650|2424925|N|N|N|N|N| +2425016|AAAAAAAAILAAFCAA|1927-05-15|328|1428|110|1927|0|5|15|2|1927|110|1428|Sunday|1927Q2|N|N|N|2425002|2425121|2424651|2424926|N|N|N|N|N| +2425017|AAAAAAAAJLAAFCAA|1927-05-16|328|1428|110|1927|1|5|16|2|1927|110|1428|Monday|1927Q2|N|N|N|2425002|2425121|2424652|2424927|N|N|N|N|N| +2425018|AAAAAAAAKLAAFCAA|1927-05-17|328|1429|110|1927|2|5|17|2|1927|110|1429|Tuesday|1927Q2|N|N|N|2425002|2425121|2424653|2424928|N|N|N|N|N| +2425019|AAAAAAAALLAAFCAA|1927-05-18|328|1429|110|1927|3|5|18|2|1927|110|1429|Wednesday|1927Q2|N|N|N|2425002|2425121|2424654|2424929|N|N|N|N|N| +2425020|AAAAAAAAMLAAFCAA|1927-05-19|328|1429|110|1927|4|5|19|2|1927|110|1429|Thursday|1927Q2|N|N|N|2425002|2425121|2424655|2424930|N|N|N|N|N| +2425021|AAAAAAAANLAAFCAA|1927-05-20|328|1429|110|1927|5|5|20|2|1927|110|1429|Friday|1927Q2|N|Y|N|2425002|2425121|2424656|2424931|N|N|N|N|N| +2425022|AAAAAAAAOLAAFCAA|1927-05-21|328|1429|110|1927|6|5|21|2|1927|110|1429|Saturday|1927Q2|N|Y|N|2425002|2425121|2424657|2424932|N|N|N|N|N| +2425023|AAAAAAAAPLAAFCAA|1927-05-22|328|1429|110|1927|0|5|22|2|1927|110|1429|Sunday|1927Q2|N|N|N|2425002|2425121|2424658|2424933|N|N|N|N|N| +2425024|AAAAAAAAAMAAFCAA|1927-05-23|328|1429|110|1927|1|5|23|2|1927|110|1429|Monday|1927Q2|N|N|N|2425002|2425121|2424659|2424934|N|N|N|N|N| +2425025|AAAAAAAABMAAFCAA|1927-05-24|328|1430|110|1927|2|5|24|2|1927|110|1430|Tuesday|1927Q2|N|N|N|2425002|2425121|2424660|2424935|N|N|N|N|N| +2425026|AAAAAAAACMAAFCAA|1927-05-25|328|1430|110|1927|3|5|25|2|1927|110|1430|Wednesday|1927Q2|N|N|N|2425002|2425121|2424661|2424936|N|N|N|N|N| +2425027|AAAAAAAADMAAFCAA|1927-05-26|328|1430|110|1927|4|5|26|2|1927|110|1430|Thursday|1927Q2|N|N|N|2425002|2425121|2424662|2424937|N|N|N|N|N| +2425028|AAAAAAAAEMAAFCAA|1927-05-27|328|1430|110|1927|5|5|27|2|1927|110|1430|Friday|1927Q2|N|Y|N|2425002|2425121|2424663|2424938|N|N|N|N|N| +2425029|AAAAAAAAFMAAFCAA|1927-05-28|328|1430|110|1927|6|5|28|2|1927|110|1430|Saturday|1927Q2|N|Y|N|2425002|2425121|2424664|2424939|N|N|N|N|N| +2425030|AAAAAAAAGMAAFCAA|1927-05-29|328|1430|110|1927|0|5|29|2|1927|110|1430|Sunday|1927Q2|N|N|N|2425002|2425121|2424665|2424940|N|N|N|N|N| +2425031|AAAAAAAAHMAAFCAA|1927-05-30|328|1430|110|1927|1|5|30|2|1927|110|1430|Monday|1927Q2|N|N|N|2425002|2425121|2424666|2424941|N|N|N|N|N| +2425032|AAAAAAAAIMAAFCAA|1927-05-31|328|1431|110|1927|2|5|31|2|1927|110|1431|Tuesday|1927Q2|N|N|N|2425002|2425121|2424667|2424942|N|N|N|N|N| +2425033|AAAAAAAAJMAAFCAA|1927-06-01|329|1431|111|1927|3|6|1|2|1927|111|1431|Wednesday|1927Q2|N|N|N|2425033|2425183|2424668|2424943|N|N|N|N|N| +2425034|AAAAAAAAKMAAFCAA|1927-06-02|329|1431|111|1927|4|6|2|2|1927|111|1431|Thursday|1927Q2|N|N|N|2425033|2425183|2424669|2424944|N|N|N|N|N| +2425035|AAAAAAAALMAAFCAA|1927-06-03|329|1431|111|1927|5|6|3|2|1927|111|1431|Friday|1927Q2|N|Y|N|2425033|2425183|2424670|2424945|N|N|N|N|N| +2425036|AAAAAAAAMMAAFCAA|1927-06-04|329|1431|111|1927|6|6|4|2|1927|111|1431|Saturday|1927Q2|N|Y|N|2425033|2425183|2424671|2424946|N|N|N|N|N| +2425037|AAAAAAAANMAAFCAA|1927-06-05|329|1431|111|1927|0|6|5|2|1927|111|1431|Sunday|1927Q2|N|N|N|2425033|2425183|2424672|2424947|N|N|N|N|N| +2425038|AAAAAAAAOMAAFCAA|1927-06-06|329|1431|111|1927|1|6|6|2|1927|111|1431|Monday|1927Q2|N|N|N|2425033|2425183|2424673|2424948|N|N|N|N|N| +2425039|AAAAAAAAPMAAFCAA|1927-06-07|329|1432|111|1927|2|6|7|2|1927|111|1432|Tuesday|1927Q2|N|N|N|2425033|2425183|2424674|2424949|N|N|N|N|N| +2425040|AAAAAAAAANAAFCAA|1927-06-08|329|1432|111|1927|3|6|8|2|1927|111|1432|Wednesday|1927Q2|N|N|N|2425033|2425183|2424675|2424950|N|N|N|N|N| +2425041|AAAAAAAABNAAFCAA|1927-06-09|329|1432|111|1927|4|6|9|2|1927|111|1432|Thursday|1927Q2|N|N|N|2425033|2425183|2424676|2424951|N|N|N|N|N| +2425042|AAAAAAAACNAAFCAA|1927-06-10|329|1432|111|1927|5|6|10|2|1927|111|1432|Friday|1927Q2|N|Y|N|2425033|2425183|2424677|2424952|N|N|N|N|N| +2425043|AAAAAAAADNAAFCAA|1927-06-11|329|1432|111|1927|6|6|11|2|1927|111|1432|Saturday|1927Q2|N|Y|N|2425033|2425183|2424678|2424953|N|N|N|N|N| +2425044|AAAAAAAAENAAFCAA|1927-06-12|329|1432|111|1927|0|6|12|2|1927|111|1432|Sunday|1927Q2|N|N|N|2425033|2425183|2424679|2424954|N|N|N|N|N| +2425045|AAAAAAAAFNAAFCAA|1927-06-13|329|1432|111|1927|1|6|13|2|1927|111|1432|Monday|1927Q2|N|N|N|2425033|2425183|2424680|2424955|N|N|N|N|N| +2425046|AAAAAAAAGNAAFCAA|1927-06-14|329|1433|111|1927|2|6|14|2|1927|111|1433|Tuesday|1927Q2|N|N|N|2425033|2425183|2424681|2424956|N|N|N|N|N| +2425047|AAAAAAAAHNAAFCAA|1927-06-15|329|1433|111|1927|3|6|15|2|1927|111|1433|Wednesday|1927Q2|N|N|N|2425033|2425183|2424682|2424957|N|N|N|N|N| +2425048|AAAAAAAAINAAFCAA|1927-06-16|329|1433|111|1927|4|6|16|2|1927|111|1433|Thursday|1927Q2|N|N|N|2425033|2425183|2424683|2424958|N|N|N|N|N| +2425049|AAAAAAAAJNAAFCAA|1927-06-17|329|1433|111|1927|5|6|17|2|1927|111|1433|Friday|1927Q2|N|Y|N|2425033|2425183|2424684|2424959|N|N|N|N|N| +2425050|AAAAAAAAKNAAFCAA|1927-06-18|329|1433|111|1927|6|6|18|2|1927|111|1433|Saturday|1927Q2|N|Y|N|2425033|2425183|2424685|2424960|N|N|N|N|N| +2425051|AAAAAAAALNAAFCAA|1927-06-19|329|1433|111|1927|0|6|19|2|1927|111|1433|Sunday|1927Q2|N|N|N|2425033|2425183|2424686|2424961|N|N|N|N|N| +2425052|AAAAAAAAMNAAFCAA|1927-06-20|329|1433|111|1927|1|6|20|2|1927|111|1433|Monday|1927Q2|N|N|N|2425033|2425183|2424687|2424962|N|N|N|N|N| +2425053|AAAAAAAANNAAFCAA|1927-06-21|329|1434|111|1927|2|6|21|2|1927|111|1434|Tuesday|1927Q2|N|N|N|2425033|2425183|2424688|2424963|N|N|N|N|N| +2425054|AAAAAAAAONAAFCAA|1927-06-22|329|1434|111|1927|3|6|22|2|1927|111|1434|Wednesday|1927Q2|N|N|N|2425033|2425183|2424689|2424964|N|N|N|N|N| +2425055|AAAAAAAAPNAAFCAA|1927-06-23|329|1434|111|1927|4|6|23|2|1927|111|1434|Thursday|1927Q2|N|N|N|2425033|2425183|2424690|2424965|N|N|N|N|N| +2425056|AAAAAAAAAOAAFCAA|1927-06-24|329|1434|111|1927|5|6|24|2|1927|111|1434|Friday|1927Q2|N|Y|N|2425033|2425183|2424691|2424966|N|N|N|N|N| +2425057|AAAAAAAABOAAFCAA|1927-06-25|329|1434|111|1927|6|6|25|2|1927|111|1434|Saturday|1927Q2|N|Y|N|2425033|2425183|2424692|2424967|N|N|N|N|N| +2425058|AAAAAAAACOAAFCAA|1927-06-26|329|1434|111|1927|0|6|26|2|1927|111|1434|Sunday|1927Q2|N|N|N|2425033|2425183|2424693|2424968|N|N|N|N|N| +2425059|AAAAAAAADOAAFCAA|1927-06-27|329|1434|111|1927|1|6|27|2|1927|111|1434|Monday|1927Q2|N|N|N|2425033|2425183|2424694|2424969|N|N|N|N|N| +2425060|AAAAAAAAEOAAFCAA|1927-06-28|329|1435|111|1927|2|6|28|2|1927|111|1435|Tuesday|1927Q2|N|N|N|2425033|2425183|2424695|2424970|N|N|N|N|N| +2425061|AAAAAAAAFOAAFCAA|1927-06-29|329|1435|111|1927|3|6|29|2|1927|111|1435|Wednesday|1927Q2|N|N|N|2425033|2425183|2424696|2424971|N|N|N|N|N| +2425062|AAAAAAAAGOAAFCAA|1927-06-30|329|1435|111|1927|4|6|30|2|1927|111|1435|Thursday|1927Q2|N|N|N|2425033|2425183|2424697|2424972|N|N|N|N|N| +2425063|AAAAAAAAHOAAFCAA|1927-07-01|330|1435|111|1927|5|7|1|2|1927|111|1435|Friday|1927Q2|N|Y|N|2425063|2425243|2424698|2424972|N|N|N|N|N| +2425064|AAAAAAAAIOAAFCAA|1927-07-02|330|1435|111|1927|6|7|2|3|1927|111|1435|Saturday|1927Q3|N|Y|N|2425063|2425243|2424699|2424973|N|N|N|N|N| +2425065|AAAAAAAAJOAAFCAA|1927-07-03|330|1435|111|1927|0|7|3|3|1927|111|1435|Sunday|1927Q3|N|N|N|2425063|2425243|2424700|2424974|N|N|N|N|N| +2425066|AAAAAAAAKOAAFCAA|1927-07-04|330|1435|111|1927|1|7|4|3|1927|111|1435|Monday|1927Q3|N|N|N|2425063|2425243|2424701|2424975|N|N|N|N|N| +2425067|AAAAAAAALOAAFCAA|1927-07-05|330|1436|111|1927|2|7|5|3|1927|111|1436|Tuesday|1927Q3|Y|N|N|2425063|2425243|2424702|2424976|N|N|N|N|N| +2425068|AAAAAAAAMOAAFCAA|1927-07-06|330|1436|111|1927|3|7|6|3|1927|111|1436|Wednesday|1927Q3|N|N|Y|2425063|2425243|2424703|2424977|N|N|N|N|N| +2425069|AAAAAAAANOAAFCAA|1927-07-07|330|1436|111|1927|4|7|7|3|1927|111|1436|Thursday|1927Q3|N|N|N|2425063|2425243|2424704|2424978|N|N|N|N|N| +2425070|AAAAAAAAOOAAFCAA|1927-07-08|330|1436|111|1927|5|7|8|3|1927|111|1436|Friday|1927Q3|N|Y|N|2425063|2425243|2424705|2424979|N|N|N|N|N| +2425071|AAAAAAAAPOAAFCAA|1927-07-09|330|1436|111|1927|6|7|9|3|1927|111|1436|Saturday|1927Q3|N|Y|N|2425063|2425243|2424706|2424980|N|N|N|N|N| +2425072|AAAAAAAAAPAAFCAA|1927-07-10|330|1436|111|1927|0|7|10|3|1927|111|1436|Sunday|1927Q3|N|N|N|2425063|2425243|2424707|2424981|N|N|N|N|N| +2425073|AAAAAAAABPAAFCAA|1927-07-11|330|1436|111|1927|1|7|11|3|1927|111|1436|Monday|1927Q3|N|N|N|2425063|2425243|2424708|2424982|N|N|N|N|N| +2425074|AAAAAAAACPAAFCAA|1927-07-12|330|1437|111|1927|2|7|12|3|1927|111|1437|Tuesday|1927Q3|N|N|N|2425063|2425243|2424709|2424983|N|N|N|N|N| +2425075|AAAAAAAADPAAFCAA|1927-07-13|330|1437|111|1927|3|7|13|3|1927|111|1437|Wednesday|1927Q3|N|N|N|2425063|2425243|2424710|2424984|N|N|N|N|N| +2425076|AAAAAAAAEPAAFCAA|1927-07-14|330|1437|111|1927|4|7|14|3|1927|111|1437|Thursday|1927Q3|N|N|N|2425063|2425243|2424711|2424985|N|N|N|N|N| +2425077|AAAAAAAAFPAAFCAA|1927-07-15|330|1437|111|1927|5|7|15|3|1927|111|1437|Friday|1927Q3|N|Y|N|2425063|2425243|2424712|2424986|N|N|N|N|N| +2425078|AAAAAAAAGPAAFCAA|1927-07-16|330|1437|111|1927|6|7|16|3|1927|111|1437|Saturday|1927Q3|N|Y|N|2425063|2425243|2424713|2424987|N|N|N|N|N| +2425079|AAAAAAAAHPAAFCAA|1927-07-17|330|1437|111|1927|0|7|17|3|1927|111|1437|Sunday|1927Q3|N|N|N|2425063|2425243|2424714|2424988|N|N|N|N|N| +2425080|AAAAAAAAIPAAFCAA|1927-07-18|330|1437|111|1927|1|7|18|3|1927|111|1437|Monday|1927Q3|N|N|N|2425063|2425243|2424715|2424989|N|N|N|N|N| +2425081|AAAAAAAAJPAAFCAA|1927-07-19|330|1438|111|1927|2|7|19|3|1927|111|1438|Tuesday|1927Q3|N|N|N|2425063|2425243|2424716|2424990|N|N|N|N|N| +2425082|AAAAAAAAKPAAFCAA|1927-07-20|330|1438|111|1927|3|7|20|3|1927|111|1438|Wednesday|1927Q3|N|N|N|2425063|2425243|2424717|2424991|N|N|N|N|N| +2425083|AAAAAAAALPAAFCAA|1927-07-21|330|1438|111|1927|4|7|21|3|1927|111|1438|Thursday|1927Q3|N|N|N|2425063|2425243|2424718|2424992|N|N|N|N|N| +2425084|AAAAAAAAMPAAFCAA|1927-07-22|330|1438|111|1927|5|7|22|3|1927|111|1438|Friday|1927Q3|N|Y|N|2425063|2425243|2424719|2424993|N|N|N|N|N| +2425085|AAAAAAAANPAAFCAA|1927-07-23|330|1438|111|1927|6|7|23|3|1927|111|1438|Saturday|1927Q3|N|Y|N|2425063|2425243|2424720|2424994|N|N|N|N|N| +2425086|AAAAAAAAOPAAFCAA|1927-07-24|330|1438|111|1927|0|7|24|3|1927|111|1438|Sunday|1927Q3|N|N|N|2425063|2425243|2424721|2424995|N|N|N|N|N| +2425087|AAAAAAAAPPAAFCAA|1927-07-25|330|1438|111|1927|1|7|25|3|1927|111|1438|Monday|1927Q3|N|N|N|2425063|2425243|2424722|2424996|N|N|N|N|N| +2425088|AAAAAAAAAABAFCAA|1927-07-26|330|1439|111|1927|2|7|26|3|1927|111|1439|Tuesday|1927Q3|N|N|N|2425063|2425243|2424723|2424997|N|N|N|N|N| +2425089|AAAAAAAABABAFCAA|1927-07-27|330|1439|111|1927|3|7|27|3|1927|111|1439|Wednesday|1927Q3|N|N|N|2425063|2425243|2424724|2424998|N|N|N|N|N| +2425090|AAAAAAAACABAFCAA|1927-07-28|330|1439|111|1927|4|7|28|3|1927|111|1439|Thursday|1927Q3|N|N|N|2425063|2425243|2424725|2424999|N|N|N|N|N| +2425091|AAAAAAAADABAFCAA|1927-07-29|330|1439|111|1927|5|7|29|3|1927|111|1439|Friday|1927Q3|N|Y|N|2425063|2425243|2424726|2425000|N|N|N|N|N| +2425092|AAAAAAAAEABAFCAA|1927-07-30|330|1439|111|1927|6|7|30|3|1927|111|1439|Saturday|1927Q3|N|Y|N|2425063|2425243|2424727|2425001|N|N|N|N|N| +2425093|AAAAAAAAFABAFCAA|1927-07-31|330|1439|111|1927|0|7|31|3|1927|111|1439|Sunday|1927Q3|N|N|N|2425063|2425243|2424728|2425002|N|N|N|N|N| +2425094|AAAAAAAAGABAFCAA|1927-08-01|331|1439|111|1927|1|8|1|3|1927|111|1439|Monday|1927Q3|N|N|N|2425094|2425305|2424729|2425003|N|N|N|N|N| +2425095|AAAAAAAAHABAFCAA|1927-08-02|331|1440|111|1927|2|8|2|3|1927|111|1440|Tuesday|1927Q3|N|N|N|2425094|2425305|2424730|2425004|N|N|N|N|N| +2425096|AAAAAAAAIABAFCAA|1927-08-03|331|1440|111|1927|3|8|3|3|1927|111|1440|Wednesday|1927Q3|N|N|N|2425094|2425305|2424731|2425005|N|N|N|N|N| +2425097|AAAAAAAAJABAFCAA|1927-08-04|331|1440|111|1927|4|8|4|3|1927|111|1440|Thursday|1927Q3|N|N|N|2425094|2425305|2424732|2425006|N|N|N|N|N| +2425098|AAAAAAAAKABAFCAA|1927-08-05|331|1440|111|1927|5|8|5|3|1927|111|1440|Friday|1927Q3|N|Y|N|2425094|2425305|2424733|2425007|N|N|N|N|N| +2425099|AAAAAAAALABAFCAA|1927-08-06|331|1440|111|1927|6|8|6|3|1927|111|1440|Saturday|1927Q3|N|Y|N|2425094|2425305|2424734|2425008|N|N|N|N|N| +2425100|AAAAAAAAMABAFCAA|1927-08-07|331|1440|111|1927|0|8|7|3|1927|111|1440|Sunday|1927Q3|N|N|N|2425094|2425305|2424735|2425009|N|N|N|N|N| +2425101|AAAAAAAANABAFCAA|1927-08-08|331|1440|111|1927|1|8|8|3|1927|111|1440|Monday|1927Q3|N|N|N|2425094|2425305|2424736|2425010|N|N|N|N|N| +2425102|AAAAAAAAOABAFCAA|1927-08-09|331|1441|111|1927|2|8|9|3|1927|111|1441|Tuesday|1927Q3|N|N|N|2425094|2425305|2424737|2425011|N|N|N|N|N| +2425103|AAAAAAAAPABAFCAA|1927-08-10|331|1441|111|1927|3|8|10|3|1927|111|1441|Wednesday|1927Q3|N|N|N|2425094|2425305|2424738|2425012|N|N|N|N|N| +2425104|AAAAAAAAABBAFCAA|1927-08-11|331|1441|111|1927|4|8|11|3|1927|111|1441|Thursday|1927Q3|N|N|N|2425094|2425305|2424739|2425013|N|N|N|N|N| +2425105|AAAAAAAABBBAFCAA|1927-08-12|331|1441|111|1927|5|8|12|3|1927|111|1441|Friday|1927Q3|N|Y|N|2425094|2425305|2424740|2425014|N|N|N|N|N| +2425106|AAAAAAAACBBAFCAA|1927-08-13|331|1441|111|1927|6|8|13|3|1927|111|1441|Saturday|1927Q3|N|Y|N|2425094|2425305|2424741|2425015|N|N|N|N|N| +2425107|AAAAAAAADBBAFCAA|1927-08-14|331|1441|111|1927|0|8|14|3|1927|111|1441|Sunday|1927Q3|N|N|N|2425094|2425305|2424742|2425016|N|N|N|N|N| +2425108|AAAAAAAAEBBAFCAA|1927-08-15|331|1441|111|1927|1|8|15|3|1927|111|1441|Monday|1927Q3|N|N|N|2425094|2425305|2424743|2425017|N|N|N|N|N| +2425109|AAAAAAAAFBBAFCAA|1927-08-16|331|1442|111|1927|2|8|16|3|1927|111|1442|Tuesday|1927Q3|N|N|N|2425094|2425305|2424744|2425018|N|N|N|N|N| +2425110|AAAAAAAAGBBAFCAA|1927-08-17|331|1442|111|1927|3|8|17|3|1927|111|1442|Wednesday|1927Q3|N|N|N|2425094|2425305|2424745|2425019|N|N|N|N|N| +2425111|AAAAAAAAHBBAFCAA|1927-08-18|331|1442|111|1927|4|8|18|3|1927|111|1442|Thursday|1927Q3|N|N|N|2425094|2425305|2424746|2425020|N|N|N|N|N| +2425112|AAAAAAAAIBBAFCAA|1927-08-19|331|1442|111|1927|5|8|19|3|1927|111|1442|Friday|1927Q3|N|Y|N|2425094|2425305|2424747|2425021|N|N|N|N|N| +2425113|AAAAAAAAJBBAFCAA|1927-08-20|331|1442|111|1927|6|8|20|3|1927|111|1442|Saturday|1927Q3|N|Y|N|2425094|2425305|2424748|2425022|N|N|N|N|N| +2425114|AAAAAAAAKBBAFCAA|1927-08-21|331|1442|111|1927|0|8|21|3|1927|111|1442|Sunday|1927Q3|N|N|N|2425094|2425305|2424749|2425023|N|N|N|N|N| +2425115|AAAAAAAALBBAFCAA|1927-08-22|331|1442|111|1927|1|8|22|3|1927|111|1442|Monday|1927Q3|N|N|N|2425094|2425305|2424750|2425024|N|N|N|N|N| +2425116|AAAAAAAAMBBAFCAA|1927-08-23|331|1443|111|1927|2|8|23|3|1927|111|1443|Tuesday|1927Q3|N|N|N|2425094|2425305|2424751|2425025|N|N|N|N|N| +2425117|AAAAAAAANBBAFCAA|1927-08-24|331|1443|111|1927|3|8|24|3|1927|111|1443|Wednesday|1927Q3|N|N|N|2425094|2425305|2424752|2425026|N|N|N|N|N| +2425118|AAAAAAAAOBBAFCAA|1927-08-25|331|1443|111|1927|4|8|25|3|1927|111|1443|Thursday|1927Q3|N|N|N|2425094|2425305|2424753|2425027|N|N|N|N|N| +2425119|AAAAAAAAPBBAFCAA|1927-08-26|331|1443|111|1927|5|8|26|3|1927|111|1443|Friday|1927Q3|N|Y|N|2425094|2425305|2424754|2425028|N|N|N|N|N| +2425120|AAAAAAAAACBAFCAA|1927-08-27|331|1443|111|1927|6|8|27|3|1927|111|1443|Saturday|1927Q3|N|Y|N|2425094|2425305|2424755|2425029|N|N|N|N|N| +2425121|AAAAAAAABCBAFCAA|1927-08-28|331|1443|111|1927|0|8|28|3|1927|111|1443|Sunday|1927Q3|N|N|N|2425094|2425305|2424756|2425030|N|N|N|N|N| +2425122|AAAAAAAACCBAFCAA|1927-08-29|331|1443|111|1927|1|8|29|3|1927|111|1443|Monday|1927Q3|N|N|N|2425094|2425305|2424757|2425031|N|N|N|N|N| +2425123|AAAAAAAADCBAFCAA|1927-08-30|331|1444|111|1927|2|8|30|3|1927|111|1444|Tuesday|1927Q3|N|N|N|2425094|2425305|2424758|2425032|N|N|N|N|N| +2425124|AAAAAAAAECBAFCAA|1927-08-31|331|1444|111|1927|3|8|31|3|1927|111|1444|Wednesday|1927Q3|N|N|N|2425094|2425305|2424759|2425033|N|N|N|N|N| +2425125|AAAAAAAAFCBAFCAA|1927-09-01|332|1444|112|1927|4|9|1|3|1927|112|1444|Thursday|1927Q3|N|N|N|2425125|2425367|2424760|2425034|N|N|N|N|N| +2425126|AAAAAAAAGCBAFCAA|1927-09-02|332|1444|112|1927|5|9|2|3|1927|112|1444|Friday|1927Q3|N|Y|N|2425125|2425367|2424761|2425035|N|N|N|N|N| +2425127|AAAAAAAAHCBAFCAA|1927-09-03|332|1444|112|1927|6|9|3|3|1927|112|1444|Saturday|1927Q3|N|Y|N|2425125|2425367|2424762|2425036|N|N|N|N|N| +2425128|AAAAAAAAICBAFCAA|1927-09-04|332|1444|112|1927|0|9|4|3|1927|112|1444|Sunday|1927Q3|N|N|N|2425125|2425367|2424763|2425037|N|N|N|N|N| +2425129|AAAAAAAAJCBAFCAA|1927-09-05|332|1444|112|1927|1|9|5|3|1927|112|1444|Monday|1927Q3|N|N|N|2425125|2425367|2424764|2425038|N|N|N|N|N| +2425130|AAAAAAAAKCBAFCAA|1927-09-06|332|1445|112|1927|2|9|6|3|1927|112|1445|Tuesday|1927Q3|N|N|N|2425125|2425367|2424765|2425039|N|N|N|N|N| +2425131|AAAAAAAALCBAFCAA|1927-09-07|332|1445|112|1927|3|9|7|3|1927|112|1445|Wednesday|1927Q3|N|N|N|2425125|2425367|2424766|2425040|N|N|N|N|N| +2425132|AAAAAAAAMCBAFCAA|1927-09-08|332|1445|112|1927|4|9|8|3|1927|112|1445|Thursday|1927Q3|N|N|N|2425125|2425367|2424767|2425041|N|N|N|N|N| +2425133|AAAAAAAANCBAFCAA|1927-09-09|332|1445|112|1927|5|9|9|3|1927|112|1445|Friday|1927Q3|N|Y|N|2425125|2425367|2424768|2425042|N|N|N|N|N| +2425134|AAAAAAAAOCBAFCAA|1927-09-10|332|1445|112|1927|6|9|10|3|1927|112|1445|Saturday|1927Q3|N|Y|N|2425125|2425367|2424769|2425043|N|N|N|N|N| +2425135|AAAAAAAAPCBAFCAA|1927-09-11|332|1445|112|1927|0|9|11|3|1927|112|1445|Sunday|1927Q3|N|N|N|2425125|2425367|2424770|2425044|N|N|N|N|N| +2425136|AAAAAAAAADBAFCAA|1927-09-12|332|1445|112|1927|1|9|12|3|1927|112|1445|Monday|1927Q3|N|N|N|2425125|2425367|2424771|2425045|N|N|N|N|N| +2425137|AAAAAAAABDBAFCAA|1927-09-13|332|1446|112|1927|2|9|13|3|1927|112|1446|Tuesday|1927Q3|N|N|N|2425125|2425367|2424772|2425046|N|N|N|N|N| +2425138|AAAAAAAACDBAFCAA|1927-09-14|332|1446|112|1927|3|9|14|3|1927|112|1446|Wednesday|1927Q3|N|N|N|2425125|2425367|2424773|2425047|N|N|N|N|N| +2425139|AAAAAAAADDBAFCAA|1927-09-15|332|1446|112|1927|4|9|15|3|1927|112|1446|Thursday|1927Q3|N|N|N|2425125|2425367|2424774|2425048|N|N|N|N|N| +2425140|AAAAAAAAEDBAFCAA|1927-09-16|332|1446|112|1927|5|9|16|3|1927|112|1446|Friday|1927Q3|N|Y|N|2425125|2425367|2424775|2425049|N|N|N|N|N| +2425141|AAAAAAAAFDBAFCAA|1927-09-17|332|1446|112|1927|6|9|17|3|1927|112|1446|Saturday|1927Q3|N|Y|N|2425125|2425367|2424776|2425050|N|N|N|N|N| +2425142|AAAAAAAAGDBAFCAA|1927-09-18|332|1446|112|1927|0|9|18|3|1927|112|1446|Sunday|1927Q3|N|N|N|2425125|2425367|2424777|2425051|N|N|N|N|N| +2425143|AAAAAAAAHDBAFCAA|1927-09-19|332|1446|112|1927|1|9|19|3|1927|112|1446|Monday|1927Q3|N|N|N|2425125|2425367|2424778|2425052|N|N|N|N|N| +2425144|AAAAAAAAIDBAFCAA|1927-09-20|332|1447|112|1927|2|9|20|3|1927|112|1447|Tuesday|1927Q3|N|N|N|2425125|2425367|2424779|2425053|N|N|N|N|N| +2425145|AAAAAAAAJDBAFCAA|1927-09-21|332|1447|112|1927|3|9|21|3|1927|112|1447|Wednesday|1927Q3|N|N|N|2425125|2425367|2424780|2425054|N|N|N|N|N| +2425146|AAAAAAAAKDBAFCAA|1927-09-22|332|1447|112|1927|4|9|22|3|1927|112|1447|Thursday|1927Q3|N|N|N|2425125|2425367|2424781|2425055|N|N|N|N|N| +2425147|AAAAAAAALDBAFCAA|1927-09-23|332|1447|112|1927|5|9|23|3|1927|112|1447|Friday|1927Q3|N|Y|N|2425125|2425367|2424782|2425056|N|N|N|N|N| +2425148|AAAAAAAAMDBAFCAA|1927-09-24|332|1447|112|1927|6|9|24|3|1927|112|1447|Saturday|1927Q3|N|Y|N|2425125|2425367|2424783|2425057|N|N|N|N|N| +2425149|AAAAAAAANDBAFCAA|1927-09-25|332|1447|112|1927|0|9|25|3|1927|112|1447|Sunday|1927Q3|N|N|N|2425125|2425367|2424784|2425058|N|N|N|N|N| +2425150|AAAAAAAAODBAFCAA|1927-09-26|332|1447|112|1927|1|9|26|3|1927|112|1447|Monday|1927Q3|N|N|N|2425125|2425367|2424785|2425059|N|N|N|N|N| +2425151|AAAAAAAAPDBAFCAA|1927-09-27|332|1448|112|1927|2|9|27|3|1927|112|1448|Tuesday|1927Q3|N|N|N|2425125|2425367|2424786|2425060|N|N|N|N|N| +2425152|AAAAAAAAAEBAFCAA|1927-09-28|332|1448|112|1927|3|9|28|3|1927|112|1448|Wednesday|1927Q3|N|N|N|2425125|2425367|2424787|2425061|N|N|N|N|N| +2425153|AAAAAAAABEBAFCAA|1927-09-29|332|1448|112|1927|4|9|29|3|1927|112|1448|Thursday|1927Q3|N|N|N|2425125|2425367|2424788|2425062|N|N|N|N|N| +2425154|AAAAAAAACEBAFCAA|1927-09-30|332|1448|112|1927|5|9|30|3|1927|112|1448|Friday|1927Q3|N|Y|N|2425125|2425367|2424789|2425063|N|N|N|N|N| +2425155|AAAAAAAADEBAFCAA|1927-10-01|333|1448|112|1927|6|10|1|3|1927|112|1448|Saturday|1927Q3|N|Y|N|2425155|2425427|2424790|2425063|N|N|N|N|N| +2425156|AAAAAAAAEEBAFCAA|1927-10-02|333|1448|112|1927|0|10|2|4|1927|112|1448|Sunday|1927Q4|N|N|N|2425155|2425427|2424791|2425064|N|N|N|N|N| +2425157|AAAAAAAAFEBAFCAA|1927-10-03|333|1448|112|1927|1|10|3|4|1927|112|1448|Monday|1927Q4|N|N|N|2425155|2425427|2424792|2425065|N|N|N|N|N| +2425158|AAAAAAAAGEBAFCAA|1927-10-04|333|1449|112|1927|2|10|4|4|1927|112|1449|Tuesday|1927Q4|N|N|N|2425155|2425427|2424793|2425066|N|N|N|N|N| +2425159|AAAAAAAAHEBAFCAA|1927-10-05|333|1449|112|1927|3|10|5|4|1927|112|1449|Wednesday|1927Q4|N|N|N|2425155|2425427|2424794|2425067|N|N|N|N|N| +2425160|AAAAAAAAIEBAFCAA|1927-10-06|333|1449|112|1927|4|10|6|4|1927|112|1449|Thursday|1927Q4|N|N|N|2425155|2425427|2424795|2425068|N|N|N|N|N| +2425161|AAAAAAAAJEBAFCAA|1927-10-07|333|1449|112|1927|5|10|7|4|1927|112|1449|Friday|1927Q4|N|Y|N|2425155|2425427|2424796|2425069|N|N|N|N|N| +2425162|AAAAAAAAKEBAFCAA|1927-10-08|333|1449|112|1927|6|10|8|4|1927|112|1449|Saturday|1927Q4|N|Y|N|2425155|2425427|2424797|2425070|N|N|N|N|N| +2425163|AAAAAAAALEBAFCAA|1927-10-09|333|1449|112|1927|0|10|9|4|1927|112|1449|Sunday|1927Q4|N|N|N|2425155|2425427|2424798|2425071|N|N|N|N|N| +2425164|AAAAAAAAMEBAFCAA|1927-10-10|333|1449|112|1927|1|10|10|4|1927|112|1449|Monday|1927Q4|N|N|N|2425155|2425427|2424799|2425072|N|N|N|N|N| +2425165|AAAAAAAANEBAFCAA|1927-10-11|333|1450|112|1927|2|10|11|4|1927|112|1450|Tuesday|1927Q4|N|N|N|2425155|2425427|2424800|2425073|N|N|N|N|N| +2425166|AAAAAAAAOEBAFCAA|1927-10-12|333|1450|112|1927|3|10|12|4|1927|112|1450|Wednesday|1927Q4|N|N|N|2425155|2425427|2424801|2425074|N|N|N|N|N| +2425167|AAAAAAAAPEBAFCAA|1927-10-13|333|1450|112|1927|4|10|13|4|1927|112|1450|Thursday|1927Q4|N|N|N|2425155|2425427|2424802|2425075|N|N|N|N|N| +2425168|AAAAAAAAAFBAFCAA|1927-10-14|333|1450|112|1927|5|10|14|4|1927|112|1450|Friday|1927Q4|N|Y|N|2425155|2425427|2424803|2425076|N|N|N|N|N| +2425169|AAAAAAAABFBAFCAA|1927-10-15|333|1450|112|1927|6|10|15|4|1927|112|1450|Saturday|1927Q4|N|Y|N|2425155|2425427|2424804|2425077|N|N|N|N|N| +2425170|AAAAAAAACFBAFCAA|1927-10-16|333|1450|112|1927|0|10|16|4|1927|112|1450|Sunday|1927Q4|N|N|N|2425155|2425427|2424805|2425078|N|N|N|N|N| +2425171|AAAAAAAADFBAFCAA|1927-10-17|333|1450|112|1927|1|10|17|4|1927|112|1450|Monday|1927Q4|N|N|N|2425155|2425427|2424806|2425079|N|N|N|N|N| +2425172|AAAAAAAAEFBAFCAA|1927-10-18|333|1451|112|1927|2|10|18|4|1927|112|1451|Tuesday|1927Q4|N|N|N|2425155|2425427|2424807|2425080|N|N|N|N|N| +2425173|AAAAAAAAFFBAFCAA|1927-10-19|333|1451|112|1927|3|10|19|4|1927|112|1451|Wednesday|1927Q4|N|N|N|2425155|2425427|2424808|2425081|N|N|N|N|N| +2425174|AAAAAAAAGFBAFCAA|1927-10-20|333|1451|112|1927|4|10|20|4|1927|112|1451|Thursday|1927Q4|N|N|N|2425155|2425427|2424809|2425082|N|N|N|N|N| +2425175|AAAAAAAAHFBAFCAA|1927-10-21|333|1451|112|1927|5|10|21|4|1927|112|1451|Friday|1927Q4|N|Y|N|2425155|2425427|2424810|2425083|N|N|N|N|N| +2425176|AAAAAAAAIFBAFCAA|1927-10-22|333|1451|112|1927|6|10|22|4|1927|112|1451|Saturday|1927Q4|N|Y|N|2425155|2425427|2424811|2425084|N|N|N|N|N| +2425177|AAAAAAAAJFBAFCAA|1927-10-23|333|1451|112|1927|0|10|23|4|1927|112|1451|Sunday|1927Q4|N|N|N|2425155|2425427|2424812|2425085|N|N|N|N|N| +2425178|AAAAAAAAKFBAFCAA|1927-10-24|333|1451|112|1927|1|10|24|4|1927|112|1451|Monday|1927Q4|N|N|N|2425155|2425427|2424813|2425086|N|N|N|N|N| +2425179|AAAAAAAALFBAFCAA|1927-10-25|333|1452|112|1927|2|10|25|4|1927|112|1452|Tuesday|1927Q4|N|N|N|2425155|2425427|2424814|2425087|N|N|N|N|N| +2425180|AAAAAAAAMFBAFCAA|1927-10-26|333|1452|112|1927|3|10|26|4|1927|112|1452|Wednesday|1927Q4|N|N|N|2425155|2425427|2424815|2425088|N|N|N|N|N| +2425181|AAAAAAAANFBAFCAA|1927-10-27|333|1452|112|1927|4|10|27|4|1927|112|1452|Thursday|1927Q4|N|N|N|2425155|2425427|2424816|2425089|N|N|N|N|N| +2425182|AAAAAAAAOFBAFCAA|1927-10-28|333|1452|112|1927|5|10|28|4|1927|112|1452|Friday|1927Q4|N|Y|N|2425155|2425427|2424817|2425090|N|N|N|N|N| +2425183|AAAAAAAAPFBAFCAA|1927-10-29|333|1452|112|1927|6|10|29|4|1927|112|1452|Saturday|1927Q4|N|Y|N|2425155|2425427|2424818|2425091|N|N|N|N|N| +2425184|AAAAAAAAAGBAFCAA|1927-10-30|333|1452|112|1927|0|10|30|4|1927|112|1452|Sunday|1927Q4|N|N|N|2425155|2425427|2424819|2425092|N|N|N|N|N| +2425185|AAAAAAAABGBAFCAA|1927-10-31|333|1452|112|1927|1|10|31|4|1927|112|1452|Monday|1927Q4|N|N|N|2425155|2425427|2424820|2425093|N|N|N|N|N| +2425186|AAAAAAAACGBAFCAA|1927-11-01|334|1453|112|1927|2|11|1|4|1927|112|1453|Tuesday|1927Q4|N|N|N|2425186|2425489|2424821|2425094|N|N|N|N|N| +2425187|AAAAAAAADGBAFCAA|1927-11-02|334|1453|112|1927|3|11|2|4|1927|112|1453|Wednesday|1927Q4|N|N|N|2425186|2425489|2424822|2425095|N|N|N|N|N| +2425188|AAAAAAAAEGBAFCAA|1927-11-03|334|1453|112|1927|4|11|3|4|1927|112|1453|Thursday|1927Q4|N|N|N|2425186|2425489|2424823|2425096|N|N|N|N|N| +2425189|AAAAAAAAFGBAFCAA|1927-11-04|334|1453|112|1927|5|11|4|4|1927|112|1453|Friday|1927Q4|N|Y|N|2425186|2425489|2424824|2425097|N|N|N|N|N| +2425190|AAAAAAAAGGBAFCAA|1927-11-05|334|1453|112|1927|6|11|5|4|1927|112|1453|Saturday|1927Q4|N|Y|N|2425186|2425489|2424825|2425098|N|N|N|N|N| +2425191|AAAAAAAAHGBAFCAA|1927-11-06|334|1453|112|1927|0|11|6|4|1927|112|1453|Sunday|1927Q4|N|N|N|2425186|2425489|2424826|2425099|N|N|N|N|N| +2425192|AAAAAAAAIGBAFCAA|1927-11-07|334|1453|112|1927|1|11|7|4|1927|112|1453|Monday|1927Q4|N|N|N|2425186|2425489|2424827|2425100|N|N|N|N|N| +2425193|AAAAAAAAJGBAFCAA|1927-11-08|334|1454|112|1927|2|11|8|4|1927|112|1454|Tuesday|1927Q4|N|N|N|2425186|2425489|2424828|2425101|N|N|N|N|N| +2425194|AAAAAAAAKGBAFCAA|1927-11-09|334|1454|112|1927|3|11|9|4|1927|112|1454|Wednesday|1927Q4|N|N|N|2425186|2425489|2424829|2425102|N|N|N|N|N| +2425195|AAAAAAAALGBAFCAA|1927-11-10|334|1454|112|1927|4|11|10|4|1927|112|1454|Thursday|1927Q4|N|N|N|2425186|2425489|2424830|2425103|N|N|N|N|N| +2425196|AAAAAAAAMGBAFCAA|1927-11-11|334|1454|112|1927|5|11|11|4|1927|112|1454|Friday|1927Q4|N|Y|N|2425186|2425489|2424831|2425104|N|N|N|N|N| +2425197|AAAAAAAANGBAFCAA|1927-11-12|334|1454|112|1927|6|11|12|4|1927|112|1454|Saturday|1927Q4|N|Y|N|2425186|2425489|2424832|2425105|N|N|N|N|N| +2425198|AAAAAAAAOGBAFCAA|1927-11-13|334|1454|112|1927|0|11|13|4|1927|112|1454|Sunday|1927Q4|N|N|N|2425186|2425489|2424833|2425106|N|N|N|N|N| +2425199|AAAAAAAAPGBAFCAA|1927-11-14|334|1454|112|1927|1|11|14|4|1927|112|1454|Monday|1927Q4|N|N|N|2425186|2425489|2424834|2425107|N|N|N|N|N| +2425200|AAAAAAAAAHBAFCAA|1927-11-15|334|1455|112|1927|2|11|15|4|1927|112|1455|Tuesday|1927Q4|N|N|N|2425186|2425489|2424835|2425108|N|N|N|N|N| +2425201|AAAAAAAABHBAFCAA|1927-11-16|334|1455|112|1927|3|11|16|4|1927|112|1455|Wednesday|1927Q4|N|N|N|2425186|2425489|2424836|2425109|N|N|N|N|N| +2425202|AAAAAAAACHBAFCAA|1927-11-17|334|1455|112|1927|4|11|17|4|1927|112|1455|Thursday|1927Q4|N|N|N|2425186|2425489|2424837|2425110|N|N|N|N|N| +2425203|AAAAAAAADHBAFCAA|1927-11-18|334|1455|112|1927|5|11|18|4|1927|112|1455|Friday|1927Q4|N|Y|N|2425186|2425489|2424838|2425111|N|N|N|N|N| +2425204|AAAAAAAAEHBAFCAA|1927-11-19|334|1455|112|1927|6|11|19|4|1927|112|1455|Saturday|1927Q4|N|Y|N|2425186|2425489|2424839|2425112|N|N|N|N|N| +2425205|AAAAAAAAFHBAFCAA|1927-11-20|334|1455|112|1927|0|11|20|4|1927|112|1455|Sunday|1927Q4|N|N|N|2425186|2425489|2424840|2425113|N|N|N|N|N| +2425206|AAAAAAAAGHBAFCAA|1927-11-21|334|1455|112|1927|1|11|21|4|1927|112|1455|Monday|1927Q4|N|N|N|2425186|2425489|2424841|2425114|N|N|N|N|N| +2425207|AAAAAAAAHHBAFCAA|1927-11-22|334|1456|112|1927|2|11|22|4|1927|112|1456|Tuesday|1927Q4|N|N|N|2425186|2425489|2424842|2425115|N|N|N|N|N| +2425208|AAAAAAAAIHBAFCAA|1927-11-23|334|1456|112|1927|3|11|23|4|1927|112|1456|Wednesday|1927Q4|N|N|N|2425186|2425489|2424843|2425116|N|N|N|N|N| +2425209|AAAAAAAAJHBAFCAA|1927-11-24|334|1456|112|1927|4|11|24|4|1927|112|1456|Thursday|1927Q4|N|N|N|2425186|2425489|2424844|2425117|N|N|N|N|N| +2425210|AAAAAAAAKHBAFCAA|1927-11-25|334|1456|112|1927|5|11|25|4|1927|112|1456|Friday|1927Q4|N|Y|N|2425186|2425489|2424845|2425118|N|N|N|N|N| +2425211|AAAAAAAALHBAFCAA|1927-11-26|334|1456|112|1927|6|11|26|4|1927|112|1456|Saturday|1927Q4|N|Y|N|2425186|2425489|2424846|2425119|N|N|N|N|N| +2425212|AAAAAAAAMHBAFCAA|1927-11-27|334|1456|112|1927|0|11|27|4|1927|112|1456|Sunday|1927Q4|N|N|N|2425186|2425489|2424847|2425120|N|N|N|N|N| +2425213|AAAAAAAANHBAFCAA|1927-11-28|334|1456|112|1927|1|11|28|4|1927|112|1456|Monday|1927Q4|N|N|N|2425186|2425489|2424848|2425121|N|N|N|N|N| +2425214|AAAAAAAAOHBAFCAA|1927-11-29|334|1457|112|1927|2|11|29|4|1927|112|1457|Tuesday|1927Q4|N|N|N|2425186|2425489|2424849|2425122|N|N|N|N|N| +2425215|AAAAAAAAPHBAFCAA|1927-11-30|334|1457|112|1927|3|11|30|4|1927|112|1457|Wednesday|1927Q4|N|N|N|2425186|2425489|2424850|2425123|N|N|N|N|N| +2425216|AAAAAAAAAIBAFCAA|1927-12-01|335|1457|113|1927|4|12|1|4|1927|113|1457|Thursday|1927Q4|N|N|N|2425216|2425549|2424851|2425124|N|N|N|N|N| +2425217|AAAAAAAABIBAFCAA|1927-12-02|335|1457|113|1927|5|12|2|4|1927|113|1457|Friday|1927Q4|N|Y|N|2425216|2425549|2424852|2425125|N|N|N|N|N| +2425218|AAAAAAAACIBAFCAA|1927-12-03|335|1457|113|1927|6|12|3|4|1927|113|1457|Saturday|1927Q4|N|Y|N|2425216|2425549|2424853|2425126|N|N|N|N|N| +2425219|AAAAAAAADIBAFCAA|1927-12-04|335|1457|113|1927|0|12|4|4|1927|113|1457|Sunday|1927Q4|N|N|N|2425216|2425549|2424854|2425127|N|N|N|N|N| +2425220|AAAAAAAAEIBAFCAA|1927-12-05|335|1457|113|1927|1|12|5|4|1927|113|1457|Monday|1927Q4|N|N|N|2425216|2425549|2424855|2425128|N|N|N|N|N| +2425221|AAAAAAAAFIBAFCAA|1927-12-06|335|1458|113|1927|2|12|6|4|1927|113|1458|Tuesday|1927Q4|N|N|N|2425216|2425549|2424856|2425129|N|N|N|N|N| +2425222|AAAAAAAAGIBAFCAA|1927-12-07|335|1458|113|1927|3|12|7|4|1927|113|1458|Wednesday|1927Q4|N|N|N|2425216|2425549|2424857|2425130|N|N|N|N|N| +2425223|AAAAAAAAHIBAFCAA|1927-12-08|335|1458|113|1927|4|12|8|4|1927|113|1458|Thursday|1927Q4|N|N|N|2425216|2425549|2424858|2425131|N|N|N|N|N| +2425224|AAAAAAAAIIBAFCAA|1927-12-09|335|1458|113|1927|5|12|9|4|1927|113|1458|Friday|1927Q4|N|Y|N|2425216|2425549|2424859|2425132|N|N|N|N|N| +2425225|AAAAAAAAJIBAFCAA|1927-12-10|335|1458|113|1927|6|12|10|4|1927|113|1458|Saturday|1927Q4|N|Y|N|2425216|2425549|2424860|2425133|N|N|N|N|N| +2425226|AAAAAAAAKIBAFCAA|1927-12-11|335|1458|113|1927|0|12|11|4|1927|113|1458|Sunday|1927Q4|N|N|N|2425216|2425549|2424861|2425134|N|N|N|N|N| +2425227|AAAAAAAALIBAFCAA|1927-12-12|335|1458|113|1927|1|12|12|4|1927|113|1458|Monday|1927Q4|N|N|N|2425216|2425549|2424862|2425135|N|N|N|N|N| +2425228|AAAAAAAAMIBAFCAA|1927-12-13|335|1459|113|1927|2|12|13|4|1927|113|1459|Tuesday|1927Q4|N|N|N|2425216|2425549|2424863|2425136|N|N|N|N|N| +2425229|AAAAAAAANIBAFCAA|1927-12-14|335|1459|113|1927|3|12|14|4|1927|113|1459|Wednesday|1927Q4|N|N|N|2425216|2425549|2424864|2425137|N|N|N|N|N| +2425230|AAAAAAAAOIBAFCAA|1927-12-15|335|1459|113|1927|4|12|15|4|1927|113|1459|Thursday|1927Q4|N|N|N|2425216|2425549|2424865|2425138|N|N|N|N|N| +2425231|AAAAAAAAPIBAFCAA|1927-12-16|335|1459|113|1927|5|12|16|4|1927|113|1459|Friday|1927Q4|N|Y|N|2425216|2425549|2424866|2425139|N|N|N|N|N| +2425232|AAAAAAAAAJBAFCAA|1927-12-17|335|1459|113|1927|6|12|17|4|1927|113|1459|Saturday|1927Q4|N|Y|N|2425216|2425549|2424867|2425140|N|N|N|N|N| +2425233|AAAAAAAABJBAFCAA|1927-12-18|335|1459|113|1927|0|12|18|4|1927|113|1459|Sunday|1927Q4|N|N|N|2425216|2425549|2424868|2425141|N|N|N|N|N| +2425234|AAAAAAAACJBAFCAA|1927-12-19|335|1459|113|1927|1|12|19|4|1927|113|1459|Monday|1927Q4|N|N|N|2425216|2425549|2424869|2425142|N|N|N|N|N| +2425235|AAAAAAAADJBAFCAA|1927-12-20|335|1460|113|1927|2|12|20|4|1927|113|1460|Tuesday|1927Q4|N|N|N|2425216|2425549|2424870|2425143|N|N|N|N|N| +2425236|AAAAAAAAEJBAFCAA|1927-12-21|335|1460|113|1927|3|12|21|4|1927|113|1460|Wednesday|1927Q4|N|N|N|2425216|2425549|2424871|2425144|N|N|N|N|N| +2425237|AAAAAAAAFJBAFCAA|1927-12-22|335|1460|113|1927|4|12|22|4|1927|113|1460|Thursday|1927Q4|N|N|N|2425216|2425549|2424872|2425145|N|N|N|N|N| +2425238|AAAAAAAAGJBAFCAA|1927-12-23|335|1460|113|1927|5|12|23|4|1927|113|1460|Friday|1927Q4|N|Y|N|2425216|2425549|2424873|2425146|N|N|N|N|N| +2425239|AAAAAAAAHJBAFCAA|1927-12-24|335|1460|113|1927|6|12|24|4|1927|113|1460|Saturday|1927Q4|N|Y|N|2425216|2425549|2424874|2425147|N|N|N|N|N| +2425240|AAAAAAAAIJBAFCAA|1927-12-25|335|1460|113|1927|0|12|25|4|1927|113|1460|Sunday|1927Q4|N|N|N|2425216|2425549|2424875|2425148|N|N|N|N|N| +2425241|AAAAAAAAJJBAFCAA|1927-12-26|335|1460|113|1927|1|12|26|4|1927|113|1460|Monday|1927Q4|Y|N|N|2425216|2425549|2424876|2425149|N|N|N|N|N| +2425242|AAAAAAAAKJBAFCAA|1927-12-27|335|1461|113|1927|2|12|27|4|1927|113|1461|Tuesday|1927Q4|N|N|Y|2425216|2425549|2424877|2425150|N|N|N|N|N| +2425243|AAAAAAAALJBAFCAA|1927-12-28|335|1461|113|1927|3|12|28|4|1927|113|1461|Wednesday|1927Q4|N|N|N|2425216|2425549|2424878|2425151|N|N|N|N|N| +2425244|AAAAAAAAMJBAFCAA|1927-12-29|335|1461|113|1927|4|12|29|4|1927|113|1461|Thursday|1927Q4|N|N|N|2425216|2425549|2424879|2425152|N|N|N|N|N| +2425245|AAAAAAAANJBAFCAA|1927-12-30|335|1461|113|1927|5|12|30|4|1927|113|1461|Friday|1927Q4|N|Y|N|2425216|2425549|2424880|2425153|N|N|N|N|N| +2425246|AAAAAAAAOJBAFCAA|1927-12-31|335|1461|113|1927|6|12|31|4|1927|113|1461|Saturday|1927Q4|N|Y|N|2425216|2425549|2424881|2425154|N|N|N|N|N| +2425247|AAAAAAAAPJBAFCAA|1928-01-01|336|1461|113|1928|0|1|1|1|1928|113|1461|Sunday|1928Q1|Y|N|N|2425247|2425246|2424882|2425155|N|N|N|N|N| +2425248|AAAAAAAAAKBAFCAA|1928-01-02|336|1461|113|1928|1|1|2|1|1928|113|1461|Monday|1928Q1|N|N|Y|2425247|2425246|2424883|2425156|N|N|N|N|N| +2425249|AAAAAAAABKBAFCAA|1928-01-03|336|1462|113|1928|2|1|3|1|1928|113|1462|Tuesday|1928Q1|N|N|N|2425247|2425246|2424884|2425157|N|N|N|N|N| +2425250|AAAAAAAACKBAFCAA|1928-01-04|336|1462|113|1928|3|1|4|1|1928|113|1462|Wednesday|1928Q1|N|N|N|2425247|2425246|2424885|2425158|N|N|N|N|N| +2425251|AAAAAAAADKBAFCAA|1928-01-05|336|1462|113|1928|4|1|5|1|1928|113|1462|Thursday|1928Q1|N|N|N|2425247|2425246|2424886|2425159|N|N|N|N|N| +2425252|AAAAAAAAEKBAFCAA|1928-01-06|336|1462|113|1928|5|1|6|1|1928|113|1462|Friday|1928Q1|N|Y|N|2425247|2425246|2424887|2425160|N|N|N|N|N| +2425253|AAAAAAAAFKBAFCAA|1928-01-07|336|1462|113|1928|6|1|7|1|1928|113|1462|Saturday|1928Q1|N|Y|N|2425247|2425246|2424888|2425161|N|N|N|N|N| +2425254|AAAAAAAAGKBAFCAA|1928-01-08|336|1462|113|1928|0|1|8|1|1928|113|1462|Sunday|1928Q1|N|N|N|2425247|2425246|2424889|2425162|N|N|N|N|N| +2425255|AAAAAAAAHKBAFCAA|1928-01-09|336|1462|113|1928|1|1|9|1|1928|113|1462|Monday|1928Q1|N|N|N|2425247|2425246|2424890|2425163|N|N|N|N|N| +2425256|AAAAAAAAIKBAFCAA|1928-01-10|336|1463|113|1928|2|1|10|1|1928|113|1463|Tuesday|1928Q1|N|N|N|2425247|2425246|2424891|2425164|N|N|N|N|N| +2425257|AAAAAAAAJKBAFCAA|1928-01-11|336|1463|113|1928|3|1|11|1|1928|113|1463|Wednesday|1928Q1|N|N|N|2425247|2425246|2424892|2425165|N|N|N|N|N| +2425258|AAAAAAAAKKBAFCAA|1928-01-12|336|1463|113|1928|4|1|12|1|1928|113|1463|Thursday|1928Q1|N|N|N|2425247|2425246|2424893|2425166|N|N|N|N|N| +2425259|AAAAAAAALKBAFCAA|1928-01-13|336|1463|113|1928|5|1|13|1|1928|113|1463|Friday|1928Q1|N|Y|N|2425247|2425246|2424894|2425167|N|N|N|N|N| +2425260|AAAAAAAAMKBAFCAA|1928-01-14|336|1463|113|1928|6|1|14|1|1928|113|1463|Saturday|1928Q1|N|Y|N|2425247|2425246|2424895|2425168|N|N|N|N|N| +2425261|AAAAAAAANKBAFCAA|1928-01-15|336|1463|113|1928|0|1|15|1|1928|113|1463|Sunday|1928Q1|N|N|N|2425247|2425246|2424896|2425169|N|N|N|N|N| +2425262|AAAAAAAAOKBAFCAA|1928-01-16|336|1463|113|1928|1|1|16|1|1928|113|1463|Monday|1928Q1|N|N|N|2425247|2425246|2424897|2425170|N|N|N|N|N| +2425263|AAAAAAAAPKBAFCAA|1928-01-17|336|1464|113|1928|2|1|17|1|1928|113|1464|Tuesday|1928Q1|N|N|N|2425247|2425246|2424898|2425171|N|N|N|N|N| +2425264|AAAAAAAAALBAFCAA|1928-01-18|336|1464|113|1928|3|1|18|1|1928|113|1464|Wednesday|1928Q1|N|N|N|2425247|2425246|2424899|2425172|N|N|N|N|N| +2425265|AAAAAAAABLBAFCAA|1928-01-19|336|1464|113|1928|4|1|19|1|1928|113|1464|Thursday|1928Q1|N|N|N|2425247|2425246|2424900|2425173|N|N|N|N|N| +2425266|AAAAAAAACLBAFCAA|1928-01-20|336|1464|113|1928|5|1|20|1|1928|113|1464|Friday|1928Q1|N|Y|N|2425247|2425246|2424901|2425174|N|N|N|N|N| +2425267|AAAAAAAADLBAFCAA|1928-01-21|336|1464|113|1928|6|1|21|1|1928|113|1464|Saturday|1928Q1|N|Y|N|2425247|2425246|2424902|2425175|N|N|N|N|N| +2425268|AAAAAAAAELBAFCAA|1928-01-22|336|1464|113|1928|0|1|22|1|1928|113|1464|Sunday|1928Q1|N|N|N|2425247|2425246|2424903|2425176|N|N|N|N|N| +2425269|AAAAAAAAFLBAFCAA|1928-01-23|336|1464|113|1928|1|1|23|1|1928|113|1464|Monday|1928Q1|N|N|N|2425247|2425246|2424904|2425177|N|N|N|N|N| +2425270|AAAAAAAAGLBAFCAA|1928-01-24|336|1465|113|1928|2|1|24|1|1928|113|1465|Tuesday|1928Q1|N|N|N|2425247|2425246|2424905|2425178|N|N|N|N|N| +2425271|AAAAAAAAHLBAFCAA|1928-01-25|336|1465|113|1928|3|1|25|1|1928|113|1465|Wednesday|1928Q1|N|N|N|2425247|2425246|2424906|2425179|N|N|N|N|N| +2425272|AAAAAAAAILBAFCAA|1928-01-26|336|1465|113|1928|4|1|26|1|1928|113|1465|Thursday|1928Q1|N|N|N|2425247|2425246|2424907|2425180|N|N|N|N|N| +2425273|AAAAAAAAJLBAFCAA|1928-01-27|336|1465|113|1928|5|1|27|1|1928|113|1465|Friday|1928Q1|N|Y|N|2425247|2425246|2424908|2425181|N|N|N|N|N| +2425274|AAAAAAAAKLBAFCAA|1928-01-28|336|1465|113|1928|6|1|28|1|1928|113|1465|Saturday|1928Q1|N|Y|N|2425247|2425246|2424909|2425182|N|N|N|N|N| +2425275|AAAAAAAALLBAFCAA|1928-01-29|336|1465|113|1928|0|1|29|1|1928|113|1465|Sunday|1928Q1|N|N|N|2425247|2425246|2424910|2425183|N|N|N|N|N| +2425276|AAAAAAAAMLBAFCAA|1928-01-30|336|1465|113|1928|1|1|30|1|1928|113|1465|Monday|1928Q1|N|N|N|2425247|2425246|2424911|2425184|N|N|N|N|N| +2425277|AAAAAAAANLBAFCAA|1928-01-31|336|1466|113|1928|2|1|31|1|1928|113|1466|Tuesday|1928Q1|N|N|N|2425247|2425246|2424912|2425185|N|N|N|N|N| +2425278|AAAAAAAAOLBAFCAA|1928-02-01|337|1466|113|1928|3|2|1|1|1928|113|1466|Wednesday|1928Q1|N|N|N|2425278|2425308|2424913|2425186|N|N|N|N|N| +2425279|AAAAAAAAPLBAFCAA|1928-02-02|337|1466|113|1928|4|2|2|1|1928|113|1466|Thursday|1928Q1|N|N|N|2425278|2425308|2424914|2425187|N|N|N|N|N| +2425280|AAAAAAAAAMBAFCAA|1928-02-03|337|1466|113|1928|5|2|3|1|1928|113|1466|Friday|1928Q1|N|Y|N|2425278|2425308|2424915|2425188|N|N|N|N|N| +2425281|AAAAAAAABMBAFCAA|1928-02-04|337|1466|113|1928|6|2|4|1|1928|113|1466|Saturday|1928Q1|N|Y|N|2425278|2425308|2424916|2425189|N|N|N|N|N| +2425282|AAAAAAAACMBAFCAA|1928-02-05|337|1466|113|1928|0|2|5|1|1928|113|1466|Sunday|1928Q1|N|N|N|2425278|2425308|2424917|2425190|N|N|N|N|N| +2425283|AAAAAAAADMBAFCAA|1928-02-06|337|1466|113|1928|1|2|6|1|1928|113|1466|Monday|1928Q1|N|N|N|2425278|2425308|2424918|2425191|N|N|N|N|N| +2425284|AAAAAAAAEMBAFCAA|1928-02-07|337|1467|113|1928|2|2|7|1|1928|113|1467|Tuesday|1928Q1|N|N|N|2425278|2425308|2424919|2425192|N|N|N|N|N| +2425285|AAAAAAAAFMBAFCAA|1928-02-08|337|1467|113|1928|3|2|8|1|1928|113|1467|Wednesday|1928Q1|N|N|N|2425278|2425308|2424920|2425193|N|N|N|N|N| +2425286|AAAAAAAAGMBAFCAA|1928-02-09|337|1467|113|1928|4|2|9|1|1928|113|1467|Thursday|1928Q1|N|N|N|2425278|2425308|2424921|2425194|N|N|N|N|N| +2425287|AAAAAAAAHMBAFCAA|1928-02-10|337|1467|113|1928|5|2|10|1|1928|113|1467|Friday|1928Q1|N|Y|N|2425278|2425308|2424922|2425195|N|N|N|N|N| +2425288|AAAAAAAAIMBAFCAA|1928-02-11|337|1467|113|1928|6|2|11|1|1928|113|1467|Saturday|1928Q1|N|Y|N|2425278|2425308|2424923|2425196|N|N|N|N|N| +2425289|AAAAAAAAJMBAFCAA|1928-02-12|337|1467|113|1928|0|2|12|1|1928|113|1467|Sunday|1928Q1|N|N|N|2425278|2425308|2424924|2425197|N|N|N|N|N| +2425290|AAAAAAAAKMBAFCAA|1928-02-13|337|1467|113|1928|1|2|13|1|1928|113|1467|Monday|1928Q1|N|N|N|2425278|2425308|2424925|2425198|N|N|N|N|N| +2425291|AAAAAAAALMBAFCAA|1928-02-14|337|1468|113|1928|2|2|14|1|1928|113|1468|Tuesday|1928Q1|N|N|N|2425278|2425308|2424926|2425199|N|N|N|N|N| +2425292|AAAAAAAAMMBAFCAA|1928-02-15|337|1468|113|1928|3|2|15|1|1928|113|1468|Wednesday|1928Q1|N|N|N|2425278|2425308|2424927|2425200|N|N|N|N|N| +2425293|AAAAAAAANMBAFCAA|1928-02-16|337|1468|113|1928|4|2|16|1|1928|113|1468|Thursday|1928Q1|N|N|N|2425278|2425308|2424928|2425201|N|N|N|N|N| +2425294|AAAAAAAAOMBAFCAA|1928-02-17|337|1468|113|1928|5|2|17|1|1928|113|1468|Friday|1928Q1|N|Y|N|2425278|2425308|2424929|2425202|N|N|N|N|N| +2425295|AAAAAAAAPMBAFCAA|1928-02-18|337|1468|113|1928|6|2|18|1|1928|113|1468|Saturday|1928Q1|N|Y|N|2425278|2425308|2424930|2425203|N|N|N|N|N| +2425296|AAAAAAAAANBAFCAA|1928-02-19|337|1468|113|1928|0|2|19|1|1928|113|1468|Sunday|1928Q1|N|N|N|2425278|2425308|2424931|2425204|N|N|N|N|N| +2425297|AAAAAAAABNBAFCAA|1928-02-20|337|1468|113|1928|1|2|20|1|1928|113|1468|Monday|1928Q1|N|N|N|2425278|2425308|2424932|2425205|N|N|N|N|N| +2425298|AAAAAAAACNBAFCAA|1928-02-21|337|1469|113|1928|2|2|21|1|1928|113|1469|Tuesday|1928Q1|N|N|N|2425278|2425308|2424933|2425206|N|N|N|N|N| +2425299|AAAAAAAADNBAFCAA|1928-02-22|337|1469|113|1928|3|2|22|1|1928|113|1469|Wednesday|1928Q1|N|N|N|2425278|2425308|2424934|2425207|N|N|N|N|N| +2425300|AAAAAAAAENBAFCAA|1928-02-23|337|1469|113|1928|4|2|23|1|1928|113|1469|Thursday|1928Q1|N|N|N|2425278|2425308|2424935|2425208|N|N|N|N|N| +2425301|AAAAAAAAFNBAFCAA|1928-02-24|337|1469|113|1928|5|2|24|1|1928|113|1469|Friday|1928Q1|N|Y|N|2425278|2425308|2424936|2425209|N|N|N|N|N| +2425302|AAAAAAAAGNBAFCAA|1928-02-25|337|1469|113|1928|6|2|25|1|1928|113|1469|Saturday|1928Q1|N|Y|N|2425278|2425308|2424937|2425210|N|N|N|N|N| +2425303|AAAAAAAAHNBAFCAA|1928-02-26|337|1469|113|1928|0|2|26|1|1928|113|1469|Sunday|1928Q1|N|N|N|2425278|2425308|2424938|2425211|N|N|N|N|N| +2425304|AAAAAAAAINBAFCAA|1928-02-27|337|1469|113|1928|1|2|27|1|1928|113|1469|Monday|1928Q1|N|N|N|2425278|2425308|2424939|2425212|N|N|N|N|N| +2425305|AAAAAAAAJNBAFCAA|1928-02-28|337|1470|113|1928|2|2|28|1|1928|113|1470|Tuesday|1928Q1|N|N|N|2425278|2425308|2424940|2425213|N|N|N|N|N| +2425306|AAAAAAAAKNBAFCAA|1928-02-29|337|1470|113|1928|3|2|29|1|1928|113|1470|Wednesday|1928Q1|N|N|N|2425278|2425308|2424940|2425214|N|N|N|N|N| +2425307|AAAAAAAALNBAFCAA|1928-03-01|338|1470|114|1928|4|3|1|1|1928|114|1470|Thursday|1928Q1|N|N|N|2425307|2425366|2424941|2425215|N|N|N|N|N| +2425308|AAAAAAAAMNBAFCAA|1928-03-02|338|1470|114|1928|5|3|2|1|1928|114|1470|Friday|1928Q1|N|Y|N|2425307|2425366|2424942|2425216|N|N|N|N|N| +2425309|AAAAAAAANNBAFCAA|1928-03-03|338|1470|114|1928|6|3|3|1|1928|114|1470|Saturday|1928Q1|N|Y|N|2425307|2425366|2424943|2425217|N|N|N|N|N| +2425310|AAAAAAAAONBAFCAA|1928-03-04|338|1470|114|1928|0|3|4|1|1928|114|1470|Sunday|1928Q1|N|N|N|2425307|2425366|2424944|2425218|N|N|N|N|N| +2425311|AAAAAAAAPNBAFCAA|1928-03-05|338|1470|114|1928|1|3|5|1|1928|114|1470|Monday|1928Q1|N|N|N|2425307|2425366|2424945|2425219|N|N|N|N|N| +2425312|AAAAAAAAAOBAFCAA|1928-03-06|338|1471|114|1928|2|3|6|1|1928|114|1471|Tuesday|1928Q1|N|N|N|2425307|2425366|2424946|2425220|N|N|N|N|N| +2425313|AAAAAAAABOBAFCAA|1928-03-07|338|1471|114|1928|3|3|7|1|1928|114|1471|Wednesday|1928Q1|N|N|N|2425307|2425366|2424947|2425221|N|N|N|N|N| +2425314|AAAAAAAACOBAFCAA|1928-03-08|338|1471|114|1928|4|3|8|1|1928|114|1471|Thursday|1928Q1|N|N|N|2425307|2425366|2424948|2425222|N|N|N|N|N| +2425315|AAAAAAAADOBAFCAA|1928-03-09|338|1471|114|1928|5|3|9|1|1928|114|1471|Friday|1928Q1|N|Y|N|2425307|2425366|2424949|2425223|N|N|N|N|N| +2425316|AAAAAAAAEOBAFCAA|1928-03-10|338|1471|114|1928|6|3|10|1|1928|114|1471|Saturday|1928Q1|N|Y|N|2425307|2425366|2424950|2425224|N|N|N|N|N| +2425317|AAAAAAAAFOBAFCAA|1928-03-11|338|1471|114|1928|0|3|11|1|1928|114|1471|Sunday|1928Q1|N|N|N|2425307|2425366|2424951|2425225|N|N|N|N|N| +2425318|AAAAAAAAGOBAFCAA|1928-03-12|338|1471|114|1928|1|3|12|1|1928|114|1471|Monday|1928Q1|N|N|N|2425307|2425366|2424952|2425226|N|N|N|N|N| +2425319|AAAAAAAAHOBAFCAA|1928-03-13|338|1472|114|1928|2|3|13|1|1928|114|1472|Tuesday|1928Q1|N|N|N|2425307|2425366|2424953|2425227|N|N|N|N|N| +2425320|AAAAAAAAIOBAFCAA|1928-03-14|338|1472|114|1928|3|3|14|1|1928|114|1472|Wednesday|1928Q1|N|N|N|2425307|2425366|2424954|2425228|N|N|N|N|N| +2425321|AAAAAAAAJOBAFCAA|1928-03-15|338|1472|114|1928|4|3|15|1|1928|114|1472|Thursday|1928Q1|N|N|N|2425307|2425366|2424955|2425229|N|N|N|N|N| +2425322|AAAAAAAAKOBAFCAA|1928-03-16|338|1472|114|1928|5|3|16|1|1928|114|1472|Friday|1928Q1|N|Y|N|2425307|2425366|2424956|2425230|N|N|N|N|N| +2425323|AAAAAAAALOBAFCAA|1928-03-17|338|1472|114|1928|6|3|17|1|1928|114|1472|Saturday|1928Q1|N|Y|N|2425307|2425366|2424957|2425231|N|N|N|N|N| +2425324|AAAAAAAAMOBAFCAA|1928-03-18|338|1472|114|1928|0|3|18|1|1928|114|1472|Sunday|1928Q1|N|N|N|2425307|2425366|2424958|2425232|N|N|N|N|N| +2425325|AAAAAAAANOBAFCAA|1928-03-19|338|1472|114|1928|1|3|19|1|1928|114|1472|Monday|1928Q1|N|N|N|2425307|2425366|2424959|2425233|N|N|N|N|N| +2425326|AAAAAAAAOOBAFCAA|1928-03-20|338|1473|114|1928|2|3|20|1|1928|114|1473|Tuesday|1928Q1|N|N|N|2425307|2425366|2424960|2425234|N|N|N|N|N| +2425327|AAAAAAAAPOBAFCAA|1928-03-21|338|1473|114|1928|3|3|21|1|1928|114|1473|Wednesday|1928Q1|N|N|N|2425307|2425366|2424961|2425235|N|N|N|N|N| +2425328|AAAAAAAAAPBAFCAA|1928-03-22|338|1473|114|1928|4|3|22|1|1928|114|1473|Thursday|1928Q1|N|N|N|2425307|2425366|2424962|2425236|N|N|N|N|N| +2425329|AAAAAAAABPBAFCAA|1928-03-23|338|1473|114|1928|5|3|23|1|1928|114|1473|Friday|1928Q1|N|Y|N|2425307|2425366|2424963|2425237|N|N|N|N|N| +2425330|AAAAAAAACPBAFCAA|1928-03-24|338|1473|114|1928|6|3|24|1|1928|114|1473|Saturday|1928Q1|N|Y|N|2425307|2425366|2424964|2425238|N|N|N|N|N| +2425331|AAAAAAAADPBAFCAA|1928-03-25|338|1473|114|1928|0|3|25|1|1928|114|1473|Sunday|1928Q1|N|N|N|2425307|2425366|2424965|2425239|N|N|N|N|N| +2425332|AAAAAAAAEPBAFCAA|1928-03-26|338|1473|114|1928|1|3|26|1|1928|114|1473|Monday|1928Q1|N|N|N|2425307|2425366|2424966|2425240|N|N|N|N|N| +2425333|AAAAAAAAFPBAFCAA|1928-03-27|338|1474|114|1928|2|3|27|1|1928|114|1474|Tuesday|1928Q1|N|N|N|2425307|2425366|2424967|2425241|N|N|N|N|N| +2425334|AAAAAAAAGPBAFCAA|1928-03-28|338|1474|114|1928|3|3|28|1|1928|114|1474|Wednesday|1928Q1|N|N|N|2425307|2425366|2424968|2425242|N|N|N|N|N| +2425335|AAAAAAAAHPBAFCAA|1928-03-29|338|1474|114|1928|4|3|29|1|1928|114|1474|Thursday|1928Q1|N|N|N|2425307|2425366|2424969|2425243|N|N|N|N|N| +2425336|AAAAAAAAIPBAFCAA|1928-03-30|338|1474|114|1928|5|3|30|1|1928|114|1474|Friday|1928Q1|N|Y|N|2425307|2425366|2424970|2425244|N|N|N|N|N| +2425337|AAAAAAAAJPBAFCAA|1928-03-31|338|1474|114|1928|6|3|31|1|1928|114|1474|Saturday|1928Q1|N|Y|N|2425307|2425366|2424971|2425245|N|N|N|N|N| +2425338|AAAAAAAAKPBAFCAA|1928-04-01|339|1474|114|1928|0|4|1|2|1928|114|1474|Sunday|1928Q2|N|N|N|2425338|2425428|2424972|2425247|N|N|N|N|N| +2425339|AAAAAAAALPBAFCAA|1928-04-02|339|1474|114|1928|1|4|2|2|1928|114|1474|Monday|1928Q2|N|N|N|2425338|2425428|2424973|2425248|N|N|N|N|N| +2425340|AAAAAAAAMPBAFCAA|1928-04-03|339|1475|114|1928|2|4|3|2|1928|114|1475|Tuesday|1928Q2|N|N|N|2425338|2425428|2424974|2425249|N|N|N|N|N| +2425341|AAAAAAAANPBAFCAA|1928-04-04|339|1475|114|1928|3|4|4|2|1928|114|1475|Wednesday|1928Q2|N|N|N|2425338|2425428|2424975|2425250|N|N|N|N|N| +2425342|AAAAAAAAOPBAFCAA|1928-04-05|339|1475|114|1928|4|4|5|2|1928|114|1475|Thursday|1928Q2|N|N|N|2425338|2425428|2424976|2425251|N|N|N|N|N| +2425343|AAAAAAAAPPBAFCAA|1928-04-06|339|1475|114|1928|5|4|6|2|1928|114|1475|Friday|1928Q2|N|Y|N|2425338|2425428|2424977|2425252|N|N|N|N|N| +2425344|AAAAAAAAAACAFCAA|1928-04-07|339|1475|114|1928|6|4|7|2|1928|114|1475|Saturday|1928Q2|N|Y|N|2425338|2425428|2424978|2425253|N|N|N|N|N| +2425345|AAAAAAAABACAFCAA|1928-04-08|339|1475|114|1928|0|4|8|2|1928|114|1475|Sunday|1928Q2|N|N|N|2425338|2425428|2424979|2425254|N|N|N|N|N| +2425346|AAAAAAAACACAFCAA|1928-04-09|339|1475|114|1928|1|4|9|2|1928|114|1475|Monday|1928Q2|N|N|N|2425338|2425428|2424980|2425255|N|N|N|N|N| +2425347|AAAAAAAADACAFCAA|1928-04-10|339|1476|114|1928|2|4|10|2|1928|114|1476|Tuesday|1928Q2|N|N|N|2425338|2425428|2424981|2425256|N|N|N|N|N| +2425348|AAAAAAAAEACAFCAA|1928-04-11|339|1476|114|1928|3|4|11|2|1928|114|1476|Wednesday|1928Q2|N|N|N|2425338|2425428|2424982|2425257|N|N|N|N|N| +2425349|AAAAAAAAFACAFCAA|1928-04-12|339|1476|114|1928|4|4|12|2|1928|114|1476|Thursday|1928Q2|N|N|N|2425338|2425428|2424983|2425258|N|N|N|N|N| +2425350|AAAAAAAAGACAFCAA|1928-04-13|339|1476|114|1928|5|4|13|2|1928|114|1476|Friday|1928Q2|N|Y|N|2425338|2425428|2424984|2425259|N|N|N|N|N| +2425351|AAAAAAAAHACAFCAA|1928-04-14|339|1476|114|1928|6|4|14|2|1928|114|1476|Saturday|1928Q2|N|Y|N|2425338|2425428|2424985|2425260|N|N|N|N|N| +2425352|AAAAAAAAIACAFCAA|1928-04-15|339|1476|114|1928|0|4|15|2|1928|114|1476|Sunday|1928Q2|N|N|N|2425338|2425428|2424986|2425261|N|N|N|N|N| +2425353|AAAAAAAAJACAFCAA|1928-04-16|339|1476|114|1928|1|4|16|2|1928|114|1476|Monday|1928Q2|N|N|N|2425338|2425428|2424987|2425262|N|N|N|N|N| +2425354|AAAAAAAAKACAFCAA|1928-04-17|339|1477|114|1928|2|4|17|2|1928|114|1477|Tuesday|1928Q2|N|N|N|2425338|2425428|2424988|2425263|N|N|N|N|N| +2425355|AAAAAAAALACAFCAA|1928-04-18|339|1477|114|1928|3|4|18|2|1928|114|1477|Wednesday|1928Q2|N|N|N|2425338|2425428|2424989|2425264|N|N|N|N|N| +2425356|AAAAAAAAMACAFCAA|1928-04-19|339|1477|114|1928|4|4|19|2|1928|114|1477|Thursday|1928Q2|N|N|N|2425338|2425428|2424990|2425265|N|N|N|N|N| +2425357|AAAAAAAANACAFCAA|1928-04-20|339|1477|114|1928|5|4|20|2|1928|114|1477|Friday|1928Q2|N|Y|N|2425338|2425428|2424991|2425266|N|N|N|N|N| +2425358|AAAAAAAAOACAFCAA|1928-04-21|339|1477|114|1928|6|4|21|2|1928|114|1477|Saturday|1928Q2|N|Y|N|2425338|2425428|2424992|2425267|N|N|N|N|N| +2425359|AAAAAAAAPACAFCAA|1928-04-22|339|1477|114|1928|0|4|22|2|1928|114|1477|Sunday|1928Q2|N|N|N|2425338|2425428|2424993|2425268|N|N|N|N|N| +2425360|AAAAAAAAABCAFCAA|1928-04-23|339|1477|114|1928|1|4|23|2|1928|114|1477|Monday|1928Q2|N|N|N|2425338|2425428|2424994|2425269|N|N|N|N|N| +2425361|AAAAAAAABBCAFCAA|1928-04-24|339|1478|114|1928|2|4|24|2|1928|114|1478|Tuesday|1928Q2|N|N|N|2425338|2425428|2424995|2425270|N|N|N|N|N| +2425362|AAAAAAAACBCAFCAA|1928-04-25|339|1478|114|1928|3|4|25|2|1928|114|1478|Wednesday|1928Q2|N|N|N|2425338|2425428|2424996|2425271|N|N|N|N|N| +2425363|AAAAAAAADBCAFCAA|1928-04-26|339|1478|114|1928|4|4|26|2|1928|114|1478|Thursday|1928Q2|N|N|N|2425338|2425428|2424997|2425272|N|N|N|N|N| +2425364|AAAAAAAAEBCAFCAA|1928-04-27|339|1478|114|1928|5|4|27|2|1928|114|1478|Friday|1928Q2|N|Y|N|2425338|2425428|2424998|2425273|N|N|N|N|N| +2425365|AAAAAAAAFBCAFCAA|1928-04-28|339|1478|114|1928|6|4|28|2|1928|114|1478|Saturday|1928Q2|N|Y|N|2425338|2425428|2424999|2425274|N|N|N|N|N| +2425366|AAAAAAAAGBCAFCAA|1928-04-29|339|1478|114|1928|0|4|29|2|1928|114|1478|Sunday|1928Q2|N|N|N|2425338|2425428|2425000|2425275|N|N|N|N|N| +2425367|AAAAAAAAHBCAFCAA|1928-04-30|339|1478|114|1928|1|4|30|2|1928|114|1478|Monday|1928Q2|N|N|N|2425338|2425428|2425001|2425276|N|N|N|N|N| +2425368|AAAAAAAAIBCAFCAA|1928-05-01|340|1479|114|1928|2|5|1|2|1928|114|1479|Tuesday|1928Q2|N|N|N|2425368|2425488|2425002|2425277|N|N|N|N|N| +2425369|AAAAAAAAJBCAFCAA|1928-05-02|340|1479|114|1928|3|5|2|2|1928|114|1479|Wednesday|1928Q2|N|N|N|2425368|2425488|2425003|2425278|N|N|N|N|N| +2425370|AAAAAAAAKBCAFCAA|1928-05-03|340|1479|114|1928|4|5|3|2|1928|114|1479|Thursday|1928Q2|N|N|N|2425368|2425488|2425004|2425279|N|N|N|N|N| +2425371|AAAAAAAALBCAFCAA|1928-05-04|340|1479|114|1928|5|5|4|2|1928|114|1479|Friday|1928Q2|N|Y|N|2425368|2425488|2425005|2425280|N|N|N|N|N| +2425372|AAAAAAAAMBCAFCAA|1928-05-05|340|1479|114|1928|6|5|5|2|1928|114|1479|Saturday|1928Q2|N|Y|N|2425368|2425488|2425006|2425281|N|N|N|N|N| +2425373|AAAAAAAANBCAFCAA|1928-05-06|340|1479|114|1928|0|5|6|2|1928|114|1479|Sunday|1928Q2|N|N|N|2425368|2425488|2425007|2425282|N|N|N|N|N| +2425374|AAAAAAAAOBCAFCAA|1928-05-07|340|1479|114|1928|1|5|7|2|1928|114|1479|Monday|1928Q2|N|N|N|2425368|2425488|2425008|2425283|N|N|N|N|N| +2425375|AAAAAAAAPBCAFCAA|1928-05-08|340|1480|114|1928|2|5|8|2|1928|114|1480|Tuesday|1928Q2|N|N|N|2425368|2425488|2425009|2425284|N|N|N|N|N| +2425376|AAAAAAAAACCAFCAA|1928-05-09|340|1480|114|1928|3|5|9|2|1928|114|1480|Wednesday|1928Q2|N|N|N|2425368|2425488|2425010|2425285|N|N|N|N|N| +2425377|AAAAAAAABCCAFCAA|1928-05-10|340|1480|114|1928|4|5|10|2|1928|114|1480|Thursday|1928Q2|N|N|N|2425368|2425488|2425011|2425286|N|N|N|N|N| +2425378|AAAAAAAACCCAFCAA|1928-05-11|340|1480|114|1928|5|5|11|2|1928|114|1480|Friday|1928Q2|N|Y|N|2425368|2425488|2425012|2425287|N|N|N|N|N| +2425379|AAAAAAAADCCAFCAA|1928-05-12|340|1480|114|1928|6|5|12|2|1928|114|1480|Saturday|1928Q2|N|Y|N|2425368|2425488|2425013|2425288|N|N|N|N|N| +2425380|AAAAAAAAECCAFCAA|1928-05-13|340|1480|114|1928|0|5|13|2|1928|114|1480|Sunday|1928Q2|N|N|N|2425368|2425488|2425014|2425289|N|N|N|N|N| +2425381|AAAAAAAAFCCAFCAA|1928-05-14|340|1480|114|1928|1|5|14|2|1928|114|1480|Monday|1928Q2|N|N|N|2425368|2425488|2425015|2425290|N|N|N|N|N| +2425382|AAAAAAAAGCCAFCAA|1928-05-15|340|1481|114|1928|2|5|15|2|1928|114|1481|Tuesday|1928Q2|N|N|N|2425368|2425488|2425016|2425291|N|N|N|N|N| +2425383|AAAAAAAAHCCAFCAA|1928-05-16|340|1481|114|1928|3|5|16|2|1928|114|1481|Wednesday|1928Q2|N|N|N|2425368|2425488|2425017|2425292|N|N|N|N|N| +2425384|AAAAAAAAICCAFCAA|1928-05-17|340|1481|114|1928|4|5|17|2|1928|114|1481|Thursday|1928Q2|N|N|N|2425368|2425488|2425018|2425293|N|N|N|N|N| +2425385|AAAAAAAAJCCAFCAA|1928-05-18|340|1481|114|1928|5|5|18|2|1928|114|1481|Friday|1928Q2|N|Y|N|2425368|2425488|2425019|2425294|N|N|N|N|N| +2425386|AAAAAAAAKCCAFCAA|1928-05-19|340|1481|114|1928|6|5|19|2|1928|114|1481|Saturday|1928Q2|N|Y|N|2425368|2425488|2425020|2425295|N|N|N|N|N| +2425387|AAAAAAAALCCAFCAA|1928-05-20|340|1481|114|1928|0|5|20|2|1928|114|1481|Sunday|1928Q2|N|N|N|2425368|2425488|2425021|2425296|N|N|N|N|N| +2425388|AAAAAAAAMCCAFCAA|1928-05-21|340|1481|114|1928|1|5|21|2|1928|114|1481|Monday|1928Q2|N|N|N|2425368|2425488|2425022|2425297|N|N|N|N|N| +2425389|AAAAAAAANCCAFCAA|1928-05-22|340|1482|114|1928|2|5|22|2|1928|114|1482|Tuesday|1928Q2|N|N|N|2425368|2425488|2425023|2425298|N|N|N|N|N| +2425390|AAAAAAAAOCCAFCAA|1928-05-23|340|1482|114|1928|3|5|23|2|1928|114|1482|Wednesday|1928Q2|N|N|N|2425368|2425488|2425024|2425299|N|N|N|N|N| +2425391|AAAAAAAAPCCAFCAA|1928-05-24|340|1482|114|1928|4|5|24|2|1928|114|1482|Thursday|1928Q2|N|N|N|2425368|2425488|2425025|2425300|N|N|N|N|N| +2425392|AAAAAAAAADCAFCAA|1928-05-25|340|1482|114|1928|5|5|25|2|1928|114|1482|Friday|1928Q2|N|Y|N|2425368|2425488|2425026|2425301|N|N|N|N|N| +2425393|AAAAAAAABDCAFCAA|1928-05-26|340|1482|114|1928|6|5|26|2|1928|114|1482|Saturday|1928Q2|N|Y|N|2425368|2425488|2425027|2425302|N|N|N|N|N| +2425394|AAAAAAAACDCAFCAA|1928-05-27|340|1482|114|1928|0|5|27|2|1928|114|1482|Sunday|1928Q2|N|N|N|2425368|2425488|2425028|2425303|N|N|N|N|N| +2425395|AAAAAAAADDCAFCAA|1928-05-28|340|1482|114|1928|1|5|28|2|1928|114|1482|Monday|1928Q2|N|N|N|2425368|2425488|2425029|2425304|N|N|N|N|N| +2425396|AAAAAAAAEDCAFCAA|1928-05-29|340|1483|114|1928|2|5|29|2|1928|114|1483|Tuesday|1928Q2|N|N|N|2425368|2425488|2425030|2425305|N|N|N|N|N| +2425397|AAAAAAAAFDCAFCAA|1928-05-30|340|1483|114|1928|3|5|30|2|1928|114|1483|Wednesday|1928Q2|N|N|N|2425368|2425488|2425031|2425306|N|N|N|N|N| +2425398|AAAAAAAAGDCAFCAA|1928-05-31|340|1483|114|1928|4|5|31|2|1928|114|1483|Thursday|1928Q2|N|N|N|2425368|2425488|2425032|2425307|N|N|N|N|N| +2425399|AAAAAAAAHDCAFCAA|1928-06-01|341|1483|115|1928|5|6|1|2|1928|115|1483|Friday|1928Q2|N|Y|N|2425399|2425550|2425033|2425308|N|N|N|N|N| +2425400|AAAAAAAAIDCAFCAA|1928-06-02|341|1483|115|1928|6|6|2|2|1928|115|1483|Saturday|1928Q2|N|Y|N|2425399|2425550|2425034|2425309|N|N|N|N|N| +2425401|AAAAAAAAJDCAFCAA|1928-06-03|341|1483|115|1928|0|6|3|2|1928|115|1483|Sunday|1928Q2|N|N|N|2425399|2425550|2425035|2425310|N|N|N|N|N| +2425402|AAAAAAAAKDCAFCAA|1928-06-04|341|1483|115|1928|1|6|4|2|1928|115|1483|Monday|1928Q2|N|N|N|2425399|2425550|2425036|2425311|N|N|N|N|N| +2425403|AAAAAAAALDCAFCAA|1928-06-05|341|1484|115|1928|2|6|5|2|1928|115|1484|Tuesday|1928Q2|N|N|N|2425399|2425550|2425037|2425312|N|N|N|N|N| +2425404|AAAAAAAAMDCAFCAA|1928-06-06|341|1484|115|1928|3|6|6|2|1928|115|1484|Wednesday|1928Q2|N|N|N|2425399|2425550|2425038|2425313|N|N|N|N|N| +2425405|AAAAAAAANDCAFCAA|1928-06-07|341|1484|115|1928|4|6|7|2|1928|115|1484|Thursday|1928Q2|N|N|N|2425399|2425550|2425039|2425314|N|N|N|N|N| +2425406|AAAAAAAAODCAFCAA|1928-06-08|341|1484|115|1928|5|6|8|2|1928|115|1484|Friday|1928Q2|N|Y|N|2425399|2425550|2425040|2425315|N|N|N|N|N| +2425407|AAAAAAAAPDCAFCAA|1928-06-09|341|1484|115|1928|6|6|9|2|1928|115|1484|Saturday|1928Q2|N|Y|N|2425399|2425550|2425041|2425316|N|N|N|N|N| +2425408|AAAAAAAAAECAFCAA|1928-06-10|341|1484|115|1928|0|6|10|2|1928|115|1484|Sunday|1928Q2|N|N|N|2425399|2425550|2425042|2425317|N|N|N|N|N| +2425409|AAAAAAAABECAFCAA|1928-06-11|341|1484|115|1928|1|6|11|2|1928|115|1484|Monday|1928Q2|N|N|N|2425399|2425550|2425043|2425318|N|N|N|N|N| +2425410|AAAAAAAACECAFCAA|1928-06-12|341|1485|115|1928|2|6|12|2|1928|115|1485|Tuesday|1928Q2|N|N|N|2425399|2425550|2425044|2425319|N|N|N|N|N| +2425411|AAAAAAAADECAFCAA|1928-06-13|341|1485|115|1928|3|6|13|2|1928|115|1485|Wednesday|1928Q2|N|N|N|2425399|2425550|2425045|2425320|N|N|N|N|N| +2425412|AAAAAAAAEECAFCAA|1928-06-14|341|1485|115|1928|4|6|14|2|1928|115|1485|Thursday|1928Q2|N|N|N|2425399|2425550|2425046|2425321|N|N|N|N|N| +2425413|AAAAAAAAFECAFCAA|1928-06-15|341|1485|115|1928|5|6|15|2|1928|115|1485|Friday|1928Q2|N|Y|N|2425399|2425550|2425047|2425322|N|N|N|N|N| +2425414|AAAAAAAAGECAFCAA|1928-06-16|341|1485|115|1928|6|6|16|2|1928|115|1485|Saturday|1928Q2|N|Y|N|2425399|2425550|2425048|2425323|N|N|N|N|N| +2425415|AAAAAAAAHECAFCAA|1928-06-17|341|1485|115|1928|0|6|17|2|1928|115|1485|Sunday|1928Q2|N|N|N|2425399|2425550|2425049|2425324|N|N|N|N|N| +2425416|AAAAAAAAIECAFCAA|1928-06-18|341|1485|115|1928|1|6|18|2|1928|115|1485|Monday|1928Q2|N|N|N|2425399|2425550|2425050|2425325|N|N|N|N|N| +2425417|AAAAAAAAJECAFCAA|1928-06-19|341|1486|115|1928|2|6|19|2|1928|115|1486|Tuesday|1928Q2|N|N|N|2425399|2425550|2425051|2425326|N|N|N|N|N| +2425418|AAAAAAAAKECAFCAA|1928-06-20|341|1486|115|1928|3|6|20|2|1928|115|1486|Wednesday|1928Q2|N|N|N|2425399|2425550|2425052|2425327|N|N|N|N|N| +2425419|AAAAAAAALECAFCAA|1928-06-21|341|1486|115|1928|4|6|21|2|1928|115|1486|Thursday|1928Q2|N|N|N|2425399|2425550|2425053|2425328|N|N|N|N|N| +2425420|AAAAAAAAMECAFCAA|1928-06-22|341|1486|115|1928|5|6|22|2|1928|115|1486|Friday|1928Q2|N|Y|N|2425399|2425550|2425054|2425329|N|N|N|N|N| +2425421|AAAAAAAANECAFCAA|1928-06-23|341|1486|115|1928|6|6|23|2|1928|115|1486|Saturday|1928Q2|N|Y|N|2425399|2425550|2425055|2425330|N|N|N|N|N| +2425422|AAAAAAAAOECAFCAA|1928-06-24|341|1486|115|1928|0|6|24|2|1928|115|1486|Sunday|1928Q2|N|N|N|2425399|2425550|2425056|2425331|N|N|N|N|N| +2425423|AAAAAAAAPECAFCAA|1928-06-25|341|1486|115|1928|1|6|25|2|1928|115|1486|Monday|1928Q2|N|N|N|2425399|2425550|2425057|2425332|N|N|N|N|N| +2425424|AAAAAAAAAFCAFCAA|1928-06-26|341|1487|115|1928|2|6|26|2|1928|115|1487|Tuesday|1928Q2|N|N|N|2425399|2425550|2425058|2425333|N|N|N|N|N| +2425425|AAAAAAAABFCAFCAA|1928-06-27|341|1487|115|1928|3|6|27|2|1928|115|1487|Wednesday|1928Q2|N|N|N|2425399|2425550|2425059|2425334|N|N|N|N|N| +2425426|AAAAAAAACFCAFCAA|1928-06-28|341|1487|115|1928|4|6|28|2|1928|115|1487|Thursday|1928Q2|N|N|N|2425399|2425550|2425060|2425335|N|N|N|N|N| +2425427|AAAAAAAADFCAFCAA|1928-06-29|341|1487|115|1928|5|6|29|2|1928|115|1487|Friday|1928Q2|N|Y|N|2425399|2425550|2425061|2425336|N|N|N|N|N| +2425428|AAAAAAAAEFCAFCAA|1928-06-30|341|1487|115|1928|6|6|30|2|1928|115|1487|Saturday|1928Q2|N|Y|N|2425399|2425550|2425062|2425337|N|N|N|N|N| +2425429|AAAAAAAAFFCAFCAA|1928-07-01|342|1487|115|1928|0|7|1|3|1928|115|1487|Sunday|1928Q3|N|N|N|2425429|2425610|2425063|2425338|N|N|N|N|N| +2425430|AAAAAAAAGFCAFCAA|1928-07-02|342|1487|115|1928|1|7|2|3|1928|115|1487|Monday|1928Q3|N|N|N|2425429|2425610|2425064|2425339|N|N|N|N|N| +2425431|AAAAAAAAHFCAFCAA|1928-07-03|342|1488|115|1928|2|7|3|3|1928|115|1488|Tuesday|1928Q3|N|N|N|2425429|2425610|2425065|2425340|N|N|N|N|N| +2425432|AAAAAAAAIFCAFCAA|1928-07-04|342|1488|115|1928|3|7|4|3|1928|115|1488|Wednesday|1928Q3|Y|N|N|2425429|2425610|2425066|2425341|N|N|N|N|N| +2425433|AAAAAAAAJFCAFCAA|1928-07-05|342|1488|115|1928|4|7|5|3|1928|115|1488|Thursday|1928Q3|N|N|Y|2425429|2425610|2425067|2425342|N|N|N|N|N| +2425434|AAAAAAAAKFCAFCAA|1928-07-06|342|1488|115|1928|5|7|6|3|1928|115|1488|Friday|1928Q3|N|Y|N|2425429|2425610|2425068|2425343|N|N|N|N|N| +2425435|AAAAAAAALFCAFCAA|1928-07-07|342|1488|115|1928|6|7|7|3|1928|115|1488|Saturday|1928Q3|N|Y|N|2425429|2425610|2425069|2425344|N|N|N|N|N| +2425436|AAAAAAAAMFCAFCAA|1928-07-08|342|1488|115|1928|0|7|8|3|1928|115|1488|Sunday|1928Q3|N|N|N|2425429|2425610|2425070|2425345|N|N|N|N|N| +2425437|AAAAAAAANFCAFCAA|1928-07-09|342|1488|115|1928|1|7|9|3|1928|115|1488|Monday|1928Q3|N|N|N|2425429|2425610|2425071|2425346|N|N|N|N|N| +2425438|AAAAAAAAOFCAFCAA|1928-07-10|342|1489|115|1928|2|7|10|3|1928|115|1489|Tuesday|1928Q3|N|N|N|2425429|2425610|2425072|2425347|N|N|N|N|N| +2425439|AAAAAAAAPFCAFCAA|1928-07-11|342|1489|115|1928|3|7|11|3|1928|115|1489|Wednesday|1928Q3|N|N|N|2425429|2425610|2425073|2425348|N|N|N|N|N| +2425440|AAAAAAAAAGCAFCAA|1928-07-12|342|1489|115|1928|4|7|12|3|1928|115|1489|Thursday|1928Q3|N|N|N|2425429|2425610|2425074|2425349|N|N|N|N|N| +2425441|AAAAAAAABGCAFCAA|1928-07-13|342|1489|115|1928|5|7|13|3|1928|115|1489|Friday|1928Q3|N|Y|N|2425429|2425610|2425075|2425350|N|N|N|N|N| +2425442|AAAAAAAACGCAFCAA|1928-07-14|342|1489|115|1928|6|7|14|3|1928|115|1489|Saturday|1928Q3|N|Y|N|2425429|2425610|2425076|2425351|N|N|N|N|N| +2425443|AAAAAAAADGCAFCAA|1928-07-15|342|1489|115|1928|0|7|15|3|1928|115|1489|Sunday|1928Q3|N|N|N|2425429|2425610|2425077|2425352|N|N|N|N|N| +2425444|AAAAAAAAEGCAFCAA|1928-07-16|342|1489|115|1928|1|7|16|3|1928|115|1489|Monday|1928Q3|N|N|N|2425429|2425610|2425078|2425353|N|N|N|N|N| +2425445|AAAAAAAAFGCAFCAA|1928-07-17|342|1490|115|1928|2|7|17|3|1928|115|1490|Tuesday|1928Q3|N|N|N|2425429|2425610|2425079|2425354|N|N|N|N|N| +2425446|AAAAAAAAGGCAFCAA|1928-07-18|342|1490|115|1928|3|7|18|3|1928|115|1490|Wednesday|1928Q3|N|N|N|2425429|2425610|2425080|2425355|N|N|N|N|N| +2425447|AAAAAAAAHGCAFCAA|1928-07-19|342|1490|115|1928|4|7|19|3|1928|115|1490|Thursday|1928Q3|N|N|N|2425429|2425610|2425081|2425356|N|N|N|N|N| +2425448|AAAAAAAAIGCAFCAA|1928-07-20|342|1490|115|1928|5|7|20|3|1928|115|1490|Friday|1928Q3|N|Y|N|2425429|2425610|2425082|2425357|N|N|N|N|N| +2425449|AAAAAAAAJGCAFCAA|1928-07-21|342|1490|115|1928|6|7|21|3|1928|115|1490|Saturday|1928Q3|N|Y|N|2425429|2425610|2425083|2425358|N|N|N|N|N| +2425450|AAAAAAAAKGCAFCAA|1928-07-22|342|1490|115|1928|0|7|22|3|1928|115|1490|Sunday|1928Q3|N|N|N|2425429|2425610|2425084|2425359|N|N|N|N|N| +2425451|AAAAAAAALGCAFCAA|1928-07-23|342|1490|115|1928|1|7|23|3|1928|115|1490|Monday|1928Q3|N|N|N|2425429|2425610|2425085|2425360|N|N|N|N|N| +2425452|AAAAAAAAMGCAFCAA|1928-07-24|342|1491|115|1928|2|7|24|3|1928|115|1491|Tuesday|1928Q3|N|N|N|2425429|2425610|2425086|2425361|N|N|N|N|N| +2425453|AAAAAAAANGCAFCAA|1928-07-25|342|1491|115|1928|3|7|25|3|1928|115|1491|Wednesday|1928Q3|N|N|N|2425429|2425610|2425087|2425362|N|N|N|N|N| +2425454|AAAAAAAAOGCAFCAA|1928-07-26|342|1491|115|1928|4|7|26|3|1928|115|1491|Thursday|1928Q3|N|N|N|2425429|2425610|2425088|2425363|N|N|N|N|N| +2425455|AAAAAAAAPGCAFCAA|1928-07-27|342|1491|115|1928|5|7|27|3|1928|115|1491|Friday|1928Q3|N|Y|N|2425429|2425610|2425089|2425364|N|N|N|N|N| +2425456|AAAAAAAAAHCAFCAA|1928-07-28|342|1491|115|1928|6|7|28|3|1928|115|1491|Saturday|1928Q3|N|Y|N|2425429|2425610|2425090|2425365|N|N|N|N|N| +2425457|AAAAAAAABHCAFCAA|1928-07-29|342|1491|115|1928|0|7|29|3|1928|115|1491|Sunday|1928Q3|N|N|N|2425429|2425610|2425091|2425366|N|N|N|N|N| +2425458|AAAAAAAACHCAFCAA|1928-07-30|342|1491|115|1928|1|7|30|3|1928|115|1491|Monday|1928Q3|N|N|N|2425429|2425610|2425092|2425367|N|N|N|N|N| +2425459|AAAAAAAADHCAFCAA|1928-07-31|342|1492|115|1928|2|7|31|3|1928|115|1492|Tuesday|1928Q3|N|N|N|2425429|2425610|2425093|2425368|N|N|N|N|N| +2425460|AAAAAAAAEHCAFCAA|1928-08-01|343|1492|115|1928|3|8|1|3|1928|115|1492|Wednesday|1928Q3|N|N|N|2425460|2425672|2425094|2425369|N|N|N|N|N| +2425461|AAAAAAAAFHCAFCAA|1928-08-02|343|1492|115|1928|4|8|2|3|1928|115|1492|Thursday|1928Q3|N|N|N|2425460|2425672|2425095|2425370|N|N|N|N|N| +2425462|AAAAAAAAGHCAFCAA|1928-08-03|343|1492|115|1928|5|8|3|3|1928|115|1492|Friday|1928Q3|N|Y|N|2425460|2425672|2425096|2425371|N|N|N|N|N| +2425463|AAAAAAAAHHCAFCAA|1928-08-04|343|1492|115|1928|6|8|4|3|1928|115|1492|Saturday|1928Q3|N|Y|N|2425460|2425672|2425097|2425372|N|N|N|N|N| +2425464|AAAAAAAAIHCAFCAA|1928-08-05|343|1492|115|1928|0|8|5|3|1928|115|1492|Sunday|1928Q3|N|N|N|2425460|2425672|2425098|2425373|N|N|N|N|N| +2425465|AAAAAAAAJHCAFCAA|1928-08-06|343|1492|115|1928|1|8|6|3|1928|115|1492|Monday|1928Q3|N|N|N|2425460|2425672|2425099|2425374|N|N|N|N|N| +2425466|AAAAAAAAKHCAFCAA|1928-08-07|343|1493|115|1928|2|8|7|3|1928|115|1493|Tuesday|1928Q3|N|N|N|2425460|2425672|2425100|2425375|N|N|N|N|N| +2425467|AAAAAAAALHCAFCAA|1928-08-08|343|1493|115|1928|3|8|8|3|1928|115|1493|Wednesday|1928Q3|N|N|N|2425460|2425672|2425101|2425376|N|N|N|N|N| +2425468|AAAAAAAAMHCAFCAA|1928-08-09|343|1493|115|1928|4|8|9|3|1928|115|1493|Thursday|1928Q3|N|N|N|2425460|2425672|2425102|2425377|N|N|N|N|N| +2425469|AAAAAAAANHCAFCAA|1928-08-10|343|1493|115|1928|5|8|10|3|1928|115|1493|Friday|1928Q3|N|Y|N|2425460|2425672|2425103|2425378|N|N|N|N|N| +2425470|AAAAAAAAOHCAFCAA|1928-08-11|343|1493|115|1928|6|8|11|3|1928|115|1493|Saturday|1928Q3|N|Y|N|2425460|2425672|2425104|2425379|N|N|N|N|N| +2425471|AAAAAAAAPHCAFCAA|1928-08-12|343|1493|115|1928|0|8|12|3|1928|115|1493|Sunday|1928Q3|N|N|N|2425460|2425672|2425105|2425380|N|N|N|N|N| +2425472|AAAAAAAAAICAFCAA|1928-08-13|343|1493|115|1928|1|8|13|3|1928|115|1493|Monday|1928Q3|N|N|N|2425460|2425672|2425106|2425381|N|N|N|N|N| +2425473|AAAAAAAABICAFCAA|1928-08-14|343|1494|115|1928|2|8|14|3|1928|115|1494|Tuesday|1928Q3|N|N|N|2425460|2425672|2425107|2425382|N|N|N|N|N| +2425474|AAAAAAAACICAFCAA|1928-08-15|343|1494|115|1928|3|8|15|3|1928|115|1494|Wednesday|1928Q3|N|N|N|2425460|2425672|2425108|2425383|N|N|N|N|N| +2425475|AAAAAAAADICAFCAA|1928-08-16|343|1494|115|1928|4|8|16|3|1928|115|1494|Thursday|1928Q3|N|N|N|2425460|2425672|2425109|2425384|N|N|N|N|N| +2425476|AAAAAAAAEICAFCAA|1928-08-17|343|1494|115|1928|5|8|17|3|1928|115|1494|Friday|1928Q3|N|Y|N|2425460|2425672|2425110|2425385|N|N|N|N|N| +2425477|AAAAAAAAFICAFCAA|1928-08-18|343|1494|115|1928|6|8|18|3|1928|115|1494|Saturday|1928Q3|N|Y|N|2425460|2425672|2425111|2425386|N|N|N|N|N| +2425478|AAAAAAAAGICAFCAA|1928-08-19|343|1494|115|1928|0|8|19|3|1928|115|1494|Sunday|1928Q3|N|N|N|2425460|2425672|2425112|2425387|N|N|N|N|N| +2425479|AAAAAAAAHICAFCAA|1928-08-20|343|1494|115|1928|1|8|20|3|1928|115|1494|Monday|1928Q3|N|N|N|2425460|2425672|2425113|2425388|N|N|N|N|N| +2425480|AAAAAAAAIICAFCAA|1928-08-21|343|1495|115|1928|2|8|21|3|1928|115|1495|Tuesday|1928Q3|N|N|N|2425460|2425672|2425114|2425389|N|N|N|N|N| +2425481|AAAAAAAAJICAFCAA|1928-08-22|343|1495|115|1928|3|8|22|3|1928|115|1495|Wednesday|1928Q3|N|N|N|2425460|2425672|2425115|2425390|N|N|N|N|N| +2425482|AAAAAAAAKICAFCAA|1928-08-23|343|1495|115|1928|4|8|23|3|1928|115|1495|Thursday|1928Q3|N|N|N|2425460|2425672|2425116|2425391|N|N|N|N|N| +2425483|AAAAAAAALICAFCAA|1928-08-24|343|1495|115|1928|5|8|24|3|1928|115|1495|Friday|1928Q3|N|Y|N|2425460|2425672|2425117|2425392|N|N|N|N|N| +2425484|AAAAAAAAMICAFCAA|1928-08-25|343|1495|115|1928|6|8|25|3|1928|115|1495|Saturday|1928Q3|N|Y|N|2425460|2425672|2425118|2425393|N|N|N|N|N| +2425485|AAAAAAAANICAFCAA|1928-08-26|343|1495|115|1928|0|8|26|3|1928|115|1495|Sunday|1928Q3|N|N|N|2425460|2425672|2425119|2425394|N|N|N|N|N| +2425486|AAAAAAAAOICAFCAA|1928-08-27|343|1495|115|1928|1|8|27|3|1928|115|1495|Monday|1928Q3|N|N|N|2425460|2425672|2425120|2425395|N|N|N|N|N| +2425487|AAAAAAAAPICAFCAA|1928-08-28|343|1496|115|1928|2|8|28|3|1928|115|1496|Tuesday|1928Q3|N|N|N|2425460|2425672|2425121|2425396|N|N|N|N|N| +2425488|AAAAAAAAAJCAFCAA|1928-08-29|343|1496|115|1928|3|8|29|3|1928|115|1496|Wednesday|1928Q3|N|N|N|2425460|2425672|2425122|2425397|N|N|N|N|N| +2425489|AAAAAAAABJCAFCAA|1928-08-30|343|1496|115|1928|4|8|30|3|1928|115|1496|Thursday|1928Q3|N|N|N|2425460|2425672|2425123|2425398|N|N|N|N|N| +2425490|AAAAAAAACJCAFCAA|1928-08-31|343|1496|115|1928|5|8|31|3|1928|115|1496|Friday|1928Q3|N|Y|N|2425460|2425672|2425124|2425399|N|N|N|N|N| +2425491|AAAAAAAADJCAFCAA|1928-09-01|344|1496|116|1928|6|9|1|3|1928|116|1496|Saturday|1928Q3|N|Y|N|2425491|2425734|2425125|2425400|N|N|N|N|N| +2425492|AAAAAAAAEJCAFCAA|1928-09-02|344|1496|116|1928|0|9|2|3|1928|116|1496|Sunday|1928Q3|N|N|N|2425491|2425734|2425126|2425401|N|N|N|N|N| +2425493|AAAAAAAAFJCAFCAA|1928-09-03|344|1496|116|1928|1|9|3|3|1928|116|1496|Monday|1928Q3|N|N|N|2425491|2425734|2425127|2425402|N|N|N|N|N| +2425494|AAAAAAAAGJCAFCAA|1928-09-04|344|1497|116|1928|2|9|4|3|1928|116|1497|Tuesday|1928Q3|N|N|N|2425491|2425734|2425128|2425403|N|N|N|N|N| +2425495|AAAAAAAAHJCAFCAA|1928-09-05|344|1497|116|1928|3|9|5|3|1928|116|1497|Wednesday|1928Q3|N|N|N|2425491|2425734|2425129|2425404|N|N|N|N|N| +2425496|AAAAAAAAIJCAFCAA|1928-09-06|344|1497|116|1928|4|9|6|3|1928|116|1497|Thursday|1928Q3|N|N|N|2425491|2425734|2425130|2425405|N|N|N|N|N| +2425497|AAAAAAAAJJCAFCAA|1928-09-07|344|1497|116|1928|5|9|7|3|1928|116|1497|Friday|1928Q3|N|Y|N|2425491|2425734|2425131|2425406|N|N|N|N|N| +2425498|AAAAAAAAKJCAFCAA|1928-09-08|344|1497|116|1928|6|9|8|3|1928|116|1497|Saturday|1928Q3|N|Y|N|2425491|2425734|2425132|2425407|N|N|N|N|N| +2425499|AAAAAAAALJCAFCAA|1928-09-09|344|1497|116|1928|0|9|9|3|1928|116|1497|Sunday|1928Q3|N|N|N|2425491|2425734|2425133|2425408|N|N|N|N|N| +2425500|AAAAAAAAMJCAFCAA|1928-09-10|344|1497|116|1928|1|9|10|3|1928|116|1497|Monday|1928Q3|N|N|N|2425491|2425734|2425134|2425409|N|N|N|N|N| +2425501|AAAAAAAANJCAFCAA|1928-09-11|344|1498|116|1928|2|9|11|3|1928|116|1498|Tuesday|1928Q3|N|N|N|2425491|2425734|2425135|2425410|N|N|N|N|N| +2425502|AAAAAAAAOJCAFCAA|1928-09-12|344|1498|116|1928|3|9|12|3|1928|116|1498|Wednesday|1928Q3|N|N|N|2425491|2425734|2425136|2425411|N|N|N|N|N| +2425503|AAAAAAAAPJCAFCAA|1928-09-13|344|1498|116|1928|4|9|13|3|1928|116|1498|Thursday|1928Q3|N|N|N|2425491|2425734|2425137|2425412|N|N|N|N|N| +2425504|AAAAAAAAAKCAFCAA|1928-09-14|344|1498|116|1928|5|9|14|3|1928|116|1498|Friday|1928Q3|N|Y|N|2425491|2425734|2425138|2425413|N|N|N|N|N| +2425505|AAAAAAAABKCAFCAA|1928-09-15|344|1498|116|1928|6|9|15|3|1928|116|1498|Saturday|1928Q3|N|Y|N|2425491|2425734|2425139|2425414|N|N|N|N|N| +2425506|AAAAAAAACKCAFCAA|1928-09-16|344|1498|116|1928|0|9|16|3|1928|116|1498|Sunday|1928Q3|N|N|N|2425491|2425734|2425140|2425415|N|N|N|N|N| +2425507|AAAAAAAADKCAFCAA|1928-09-17|344|1498|116|1928|1|9|17|3|1928|116|1498|Monday|1928Q3|N|N|N|2425491|2425734|2425141|2425416|N|N|N|N|N| +2425508|AAAAAAAAEKCAFCAA|1928-09-18|344|1499|116|1928|2|9|18|3|1928|116|1499|Tuesday|1928Q3|N|N|N|2425491|2425734|2425142|2425417|N|N|N|N|N| +2425509|AAAAAAAAFKCAFCAA|1928-09-19|344|1499|116|1928|3|9|19|3|1928|116|1499|Wednesday|1928Q3|N|N|N|2425491|2425734|2425143|2425418|N|N|N|N|N| +2425510|AAAAAAAAGKCAFCAA|1928-09-20|344|1499|116|1928|4|9|20|3|1928|116|1499|Thursday|1928Q3|N|N|N|2425491|2425734|2425144|2425419|N|N|N|N|N| +2425511|AAAAAAAAHKCAFCAA|1928-09-21|344|1499|116|1928|5|9|21|3|1928|116|1499|Friday|1928Q3|N|Y|N|2425491|2425734|2425145|2425420|N|N|N|N|N| +2425512|AAAAAAAAIKCAFCAA|1928-09-22|344|1499|116|1928|6|9|22|3|1928|116|1499|Saturday|1928Q3|N|Y|N|2425491|2425734|2425146|2425421|N|N|N|N|N| +2425513|AAAAAAAAJKCAFCAA|1928-09-23|344|1499|116|1928|0|9|23|3|1928|116|1499|Sunday|1928Q3|N|N|N|2425491|2425734|2425147|2425422|N|N|N|N|N| +2425514|AAAAAAAAKKCAFCAA|1928-09-24|344|1499|116|1928|1|9|24|3|1928|116|1499|Monday|1928Q3|N|N|N|2425491|2425734|2425148|2425423|N|N|N|N|N| +2425515|AAAAAAAALKCAFCAA|1928-09-25|344|1500|116|1928|2|9|25|3|1928|116|1500|Tuesday|1928Q3|N|N|N|2425491|2425734|2425149|2425424|N|N|N|N|N| +2425516|AAAAAAAAMKCAFCAA|1928-09-26|344|1500|116|1928|3|9|26|3|1928|116|1500|Wednesday|1928Q3|N|N|N|2425491|2425734|2425150|2425425|N|N|N|N|N| +2425517|AAAAAAAANKCAFCAA|1928-09-27|344|1500|116|1928|4|9|27|3|1928|116|1500|Thursday|1928Q3|N|N|N|2425491|2425734|2425151|2425426|N|N|N|N|N| +2425518|AAAAAAAAOKCAFCAA|1928-09-28|344|1500|116|1928|5|9|28|3|1928|116|1500|Friday|1928Q3|N|Y|N|2425491|2425734|2425152|2425427|N|N|N|N|N| +2425519|AAAAAAAAPKCAFCAA|1928-09-29|344|1500|116|1928|6|9|29|3|1928|116|1500|Saturday|1928Q3|N|Y|N|2425491|2425734|2425153|2425428|N|N|N|N|N| +2425520|AAAAAAAAALCAFCAA|1928-09-30|344|1500|116|1928|0|9|30|3|1928|116|1500|Sunday|1928Q3|N|N|N|2425491|2425734|2425154|2425429|N|N|N|N|N| +2425521|AAAAAAAABLCAFCAA|1928-10-01|345|1500|116|1928|1|10|1|4|1928|116|1500|Monday|1928Q4|N|N|N|2425521|2425794|2425155|2425429|N|N|N|N|N| +2425522|AAAAAAAACLCAFCAA|1928-10-02|345|1501|116|1928|2|10|2|4|1928|116|1501|Tuesday|1928Q4|N|N|N|2425521|2425794|2425156|2425430|N|N|N|N|N| +2425523|AAAAAAAADLCAFCAA|1928-10-03|345|1501|116|1928|3|10|3|4|1928|116|1501|Wednesday|1928Q4|N|N|N|2425521|2425794|2425157|2425431|N|N|N|N|N| +2425524|AAAAAAAAELCAFCAA|1928-10-04|345|1501|116|1928|4|10|4|4|1928|116|1501|Thursday|1928Q4|N|N|N|2425521|2425794|2425158|2425432|N|N|N|N|N| +2425525|AAAAAAAAFLCAFCAA|1928-10-05|345|1501|116|1928|5|10|5|4|1928|116|1501|Friday|1928Q4|N|Y|N|2425521|2425794|2425159|2425433|N|N|N|N|N| +2425526|AAAAAAAAGLCAFCAA|1928-10-06|345|1501|116|1928|6|10|6|4|1928|116|1501|Saturday|1928Q4|N|Y|N|2425521|2425794|2425160|2425434|N|N|N|N|N| +2425527|AAAAAAAAHLCAFCAA|1928-10-07|345|1501|116|1928|0|10|7|4|1928|116|1501|Sunday|1928Q4|N|N|N|2425521|2425794|2425161|2425435|N|N|N|N|N| +2425528|AAAAAAAAILCAFCAA|1928-10-08|345|1501|116|1928|1|10|8|4|1928|116|1501|Monday|1928Q4|N|N|N|2425521|2425794|2425162|2425436|N|N|N|N|N| +2425529|AAAAAAAAJLCAFCAA|1928-10-09|345|1502|116|1928|2|10|9|4|1928|116|1502|Tuesday|1928Q4|N|N|N|2425521|2425794|2425163|2425437|N|N|N|N|N| +2425530|AAAAAAAAKLCAFCAA|1928-10-10|345|1502|116|1928|3|10|10|4|1928|116|1502|Wednesday|1928Q4|N|N|N|2425521|2425794|2425164|2425438|N|N|N|N|N| +2425531|AAAAAAAALLCAFCAA|1928-10-11|345|1502|116|1928|4|10|11|4|1928|116|1502|Thursday|1928Q4|N|N|N|2425521|2425794|2425165|2425439|N|N|N|N|N| +2425532|AAAAAAAAMLCAFCAA|1928-10-12|345|1502|116|1928|5|10|12|4|1928|116|1502|Friday|1928Q4|N|Y|N|2425521|2425794|2425166|2425440|N|N|N|N|N| +2425533|AAAAAAAANLCAFCAA|1928-10-13|345|1502|116|1928|6|10|13|4|1928|116|1502|Saturday|1928Q4|N|Y|N|2425521|2425794|2425167|2425441|N|N|N|N|N| +2425534|AAAAAAAAOLCAFCAA|1928-10-14|345|1502|116|1928|0|10|14|4|1928|116|1502|Sunday|1928Q4|N|N|N|2425521|2425794|2425168|2425442|N|N|N|N|N| +2425535|AAAAAAAAPLCAFCAA|1928-10-15|345|1502|116|1928|1|10|15|4|1928|116|1502|Monday|1928Q4|N|N|N|2425521|2425794|2425169|2425443|N|N|N|N|N| +2425536|AAAAAAAAAMCAFCAA|1928-10-16|345|1503|116|1928|2|10|16|4|1928|116|1503|Tuesday|1928Q4|N|N|N|2425521|2425794|2425170|2425444|N|N|N|N|N| +2425537|AAAAAAAABMCAFCAA|1928-10-17|345|1503|116|1928|3|10|17|4|1928|116|1503|Wednesday|1928Q4|N|N|N|2425521|2425794|2425171|2425445|N|N|N|N|N| +2425538|AAAAAAAACMCAFCAA|1928-10-18|345|1503|116|1928|4|10|18|4|1928|116|1503|Thursday|1928Q4|N|N|N|2425521|2425794|2425172|2425446|N|N|N|N|N| +2425539|AAAAAAAADMCAFCAA|1928-10-19|345|1503|116|1928|5|10|19|4|1928|116|1503|Friday|1928Q4|N|Y|N|2425521|2425794|2425173|2425447|N|N|N|N|N| +2425540|AAAAAAAAEMCAFCAA|1928-10-20|345|1503|116|1928|6|10|20|4|1928|116|1503|Saturday|1928Q4|N|Y|N|2425521|2425794|2425174|2425448|N|N|N|N|N| +2425541|AAAAAAAAFMCAFCAA|1928-10-21|345|1503|116|1928|0|10|21|4|1928|116|1503|Sunday|1928Q4|N|N|N|2425521|2425794|2425175|2425449|N|N|N|N|N| +2425542|AAAAAAAAGMCAFCAA|1928-10-22|345|1503|116|1928|1|10|22|4|1928|116|1503|Monday|1928Q4|N|N|N|2425521|2425794|2425176|2425450|N|N|N|N|N| +2425543|AAAAAAAAHMCAFCAA|1928-10-23|345|1504|116|1928|2|10|23|4|1928|116|1504|Tuesday|1928Q4|N|N|N|2425521|2425794|2425177|2425451|N|N|N|N|N| +2425544|AAAAAAAAIMCAFCAA|1928-10-24|345|1504|116|1928|3|10|24|4|1928|116|1504|Wednesday|1928Q4|N|N|N|2425521|2425794|2425178|2425452|N|N|N|N|N| +2425545|AAAAAAAAJMCAFCAA|1928-10-25|345|1504|116|1928|4|10|25|4|1928|116|1504|Thursday|1928Q4|N|N|N|2425521|2425794|2425179|2425453|N|N|N|N|N| +2425546|AAAAAAAAKMCAFCAA|1928-10-26|345|1504|116|1928|5|10|26|4|1928|116|1504|Friday|1928Q4|N|Y|N|2425521|2425794|2425180|2425454|N|N|N|N|N| +2425547|AAAAAAAALMCAFCAA|1928-10-27|345|1504|116|1928|6|10|27|4|1928|116|1504|Saturday|1928Q4|N|Y|N|2425521|2425794|2425181|2425455|N|N|N|N|N| +2425548|AAAAAAAAMMCAFCAA|1928-10-28|345|1504|116|1928|0|10|28|4|1928|116|1504|Sunday|1928Q4|N|N|N|2425521|2425794|2425182|2425456|N|N|N|N|N| +2425549|AAAAAAAANMCAFCAA|1928-10-29|345|1504|116|1928|1|10|29|4|1928|116|1504|Monday|1928Q4|N|N|N|2425521|2425794|2425183|2425457|N|N|N|N|N| +2425550|AAAAAAAAOMCAFCAA|1928-10-30|345|1505|116|1928|2|10|30|4|1928|116|1505|Tuesday|1928Q4|N|N|N|2425521|2425794|2425184|2425458|N|N|N|N|N| +2425551|AAAAAAAAPMCAFCAA|1928-10-31|345|1505|116|1928|3|10|31|4|1928|116|1505|Wednesday|1928Q4|N|N|N|2425521|2425794|2425185|2425459|N|N|N|N|N| +2425552|AAAAAAAAANCAFCAA|1928-11-01|346|1505|116|1928|4|11|1|4|1928|116|1505|Thursday|1928Q4|N|N|N|2425552|2425856|2425186|2425460|N|N|N|N|N| +2425553|AAAAAAAABNCAFCAA|1928-11-02|346|1505|116|1928|5|11|2|4|1928|116|1505|Friday|1928Q4|N|Y|N|2425552|2425856|2425187|2425461|N|N|N|N|N| +2425554|AAAAAAAACNCAFCAA|1928-11-03|346|1505|116|1928|6|11|3|4|1928|116|1505|Saturday|1928Q4|N|Y|N|2425552|2425856|2425188|2425462|N|N|N|N|N| +2425555|AAAAAAAADNCAFCAA|1928-11-04|346|1505|116|1928|0|11|4|4|1928|116|1505|Sunday|1928Q4|N|N|N|2425552|2425856|2425189|2425463|N|N|N|N|N| +2425556|AAAAAAAAENCAFCAA|1928-11-05|346|1505|116|1928|1|11|5|4|1928|116|1505|Monday|1928Q4|N|N|N|2425552|2425856|2425190|2425464|N|N|N|N|N| +2425557|AAAAAAAAFNCAFCAA|1928-11-06|346|1506|116|1928|2|11|6|4|1928|116|1506|Tuesday|1928Q4|N|N|N|2425552|2425856|2425191|2425465|N|N|N|N|N| +2425558|AAAAAAAAGNCAFCAA|1928-11-07|346|1506|116|1928|3|11|7|4|1928|116|1506|Wednesday|1928Q4|N|N|N|2425552|2425856|2425192|2425466|N|N|N|N|N| +2425559|AAAAAAAAHNCAFCAA|1928-11-08|346|1506|116|1928|4|11|8|4|1928|116|1506|Thursday|1928Q4|N|N|N|2425552|2425856|2425193|2425467|N|N|N|N|N| +2425560|AAAAAAAAINCAFCAA|1928-11-09|346|1506|116|1928|5|11|9|4|1928|116|1506|Friday|1928Q4|N|Y|N|2425552|2425856|2425194|2425468|N|N|N|N|N| +2425561|AAAAAAAAJNCAFCAA|1928-11-10|346|1506|116|1928|6|11|10|4|1928|116|1506|Saturday|1928Q4|N|Y|N|2425552|2425856|2425195|2425469|N|N|N|N|N| +2425562|AAAAAAAAKNCAFCAA|1928-11-11|346|1506|116|1928|0|11|11|4|1928|116|1506|Sunday|1928Q4|N|N|N|2425552|2425856|2425196|2425470|N|N|N|N|N| +2425563|AAAAAAAALNCAFCAA|1928-11-12|346|1506|116|1928|1|11|12|4|1928|116|1506|Monday|1928Q4|N|N|N|2425552|2425856|2425197|2425471|N|N|N|N|N| +2425564|AAAAAAAAMNCAFCAA|1928-11-13|346|1507|116|1928|2|11|13|4|1928|116|1507|Tuesday|1928Q4|N|N|N|2425552|2425856|2425198|2425472|N|N|N|N|N| +2425565|AAAAAAAANNCAFCAA|1928-11-14|346|1507|116|1928|3|11|14|4|1928|116|1507|Wednesday|1928Q4|N|N|N|2425552|2425856|2425199|2425473|N|N|N|N|N| +2425566|AAAAAAAAONCAFCAA|1928-11-15|346|1507|116|1928|4|11|15|4|1928|116|1507|Thursday|1928Q4|N|N|N|2425552|2425856|2425200|2425474|N|N|N|N|N| +2425567|AAAAAAAAPNCAFCAA|1928-11-16|346|1507|116|1928|5|11|16|4|1928|116|1507|Friday|1928Q4|N|Y|N|2425552|2425856|2425201|2425475|N|N|N|N|N| +2425568|AAAAAAAAAOCAFCAA|1928-11-17|346|1507|116|1928|6|11|17|4|1928|116|1507|Saturday|1928Q4|N|Y|N|2425552|2425856|2425202|2425476|N|N|N|N|N| +2425569|AAAAAAAABOCAFCAA|1928-11-18|346|1507|116|1928|0|11|18|4|1928|116|1507|Sunday|1928Q4|N|N|N|2425552|2425856|2425203|2425477|N|N|N|N|N| +2425570|AAAAAAAACOCAFCAA|1928-11-19|346|1507|116|1928|1|11|19|4|1928|116|1507|Monday|1928Q4|N|N|N|2425552|2425856|2425204|2425478|N|N|N|N|N| +2425571|AAAAAAAADOCAFCAA|1928-11-20|346|1508|116|1928|2|11|20|4|1928|116|1508|Tuesday|1928Q4|N|N|N|2425552|2425856|2425205|2425479|N|N|N|N|N| +2425572|AAAAAAAAEOCAFCAA|1928-11-21|346|1508|116|1928|3|11|21|4|1928|116|1508|Wednesday|1928Q4|N|N|N|2425552|2425856|2425206|2425480|N|N|N|N|N| +2425573|AAAAAAAAFOCAFCAA|1928-11-22|346|1508|116|1928|4|11|22|4|1928|116|1508|Thursday|1928Q4|N|N|N|2425552|2425856|2425207|2425481|N|N|N|N|N| +2425574|AAAAAAAAGOCAFCAA|1928-11-23|346|1508|116|1928|5|11|23|4|1928|116|1508|Friday|1928Q4|N|Y|N|2425552|2425856|2425208|2425482|N|N|N|N|N| +2425575|AAAAAAAAHOCAFCAA|1928-11-24|346|1508|116|1928|6|11|24|4|1928|116|1508|Saturday|1928Q4|N|Y|N|2425552|2425856|2425209|2425483|N|N|N|N|N| +2425576|AAAAAAAAIOCAFCAA|1928-11-25|346|1508|116|1928|0|11|25|4|1928|116|1508|Sunday|1928Q4|N|N|N|2425552|2425856|2425210|2425484|N|N|N|N|N| +2425577|AAAAAAAAJOCAFCAA|1928-11-26|346|1508|116|1928|1|11|26|4|1928|116|1508|Monday|1928Q4|N|N|N|2425552|2425856|2425211|2425485|N|N|N|N|N| +2425578|AAAAAAAAKOCAFCAA|1928-11-27|346|1509|116|1928|2|11|27|4|1928|116|1509|Tuesday|1928Q4|N|N|N|2425552|2425856|2425212|2425486|N|N|N|N|N| +2425579|AAAAAAAALOCAFCAA|1928-11-28|346|1509|116|1928|3|11|28|4|1928|116|1509|Wednesday|1928Q4|N|N|N|2425552|2425856|2425213|2425487|N|N|N|N|N| +2425580|AAAAAAAAMOCAFCAA|1928-11-29|346|1509|116|1928|4|11|29|4|1928|116|1509|Thursday|1928Q4|N|N|N|2425552|2425856|2425214|2425488|N|N|N|N|N| +2425581|AAAAAAAANOCAFCAA|1928-11-30|346|1509|116|1928|5|11|30|4|1928|116|1509|Friday|1928Q4|N|Y|N|2425552|2425856|2425215|2425489|N|N|N|N|N| +2425582|AAAAAAAAOOCAFCAA|1928-12-01|347|1509|117|1928|6|12|1|4|1928|117|1509|Saturday|1928Q4|N|Y|N|2425582|2425916|2425216|2425490|N|N|N|N|N| +2425583|AAAAAAAAPOCAFCAA|1928-12-02|347|1509|117|1928|0|12|2|4|1928|117|1509|Sunday|1928Q4|N|N|N|2425582|2425916|2425217|2425491|N|N|N|N|N| +2425584|AAAAAAAAAPCAFCAA|1928-12-03|347|1509|117|1928|1|12|3|4|1928|117|1509|Monday|1928Q4|N|N|N|2425582|2425916|2425218|2425492|N|N|N|N|N| +2425585|AAAAAAAABPCAFCAA|1928-12-04|347|1510|117|1928|2|12|4|4|1928|117|1510|Tuesday|1928Q4|N|N|N|2425582|2425916|2425219|2425493|N|N|N|N|N| +2425586|AAAAAAAACPCAFCAA|1928-12-05|347|1510|117|1928|3|12|5|4|1928|117|1510|Wednesday|1928Q4|N|N|N|2425582|2425916|2425220|2425494|N|N|N|N|N| +2425587|AAAAAAAADPCAFCAA|1928-12-06|347|1510|117|1928|4|12|6|4|1928|117|1510|Thursday|1928Q4|N|N|N|2425582|2425916|2425221|2425495|N|N|N|N|N| +2425588|AAAAAAAAEPCAFCAA|1928-12-07|347|1510|117|1928|5|12|7|4|1928|117|1510|Friday|1928Q4|N|Y|N|2425582|2425916|2425222|2425496|N|N|N|N|N| +2425589|AAAAAAAAFPCAFCAA|1928-12-08|347|1510|117|1928|6|12|8|4|1928|117|1510|Saturday|1928Q4|N|Y|N|2425582|2425916|2425223|2425497|N|N|N|N|N| +2425590|AAAAAAAAGPCAFCAA|1928-12-09|347|1510|117|1928|0|12|9|4|1928|117|1510|Sunday|1928Q4|N|N|N|2425582|2425916|2425224|2425498|N|N|N|N|N| +2425591|AAAAAAAAHPCAFCAA|1928-12-10|347|1510|117|1928|1|12|10|4|1928|117|1510|Monday|1928Q4|N|N|N|2425582|2425916|2425225|2425499|N|N|N|N|N| +2425592|AAAAAAAAIPCAFCAA|1928-12-11|347|1511|117|1928|2|12|11|4|1928|117|1511|Tuesday|1928Q4|N|N|N|2425582|2425916|2425226|2425500|N|N|N|N|N| +2425593|AAAAAAAAJPCAFCAA|1928-12-12|347|1511|117|1928|3|12|12|4|1928|117|1511|Wednesday|1928Q4|N|N|N|2425582|2425916|2425227|2425501|N|N|N|N|N| +2425594|AAAAAAAAKPCAFCAA|1928-12-13|347|1511|117|1928|4|12|13|4|1928|117|1511|Thursday|1928Q4|N|N|N|2425582|2425916|2425228|2425502|N|N|N|N|N| +2425595|AAAAAAAALPCAFCAA|1928-12-14|347|1511|117|1928|5|12|14|4|1928|117|1511|Friday|1928Q4|N|Y|N|2425582|2425916|2425229|2425503|N|N|N|N|N| +2425596|AAAAAAAAMPCAFCAA|1928-12-15|347|1511|117|1928|6|12|15|4|1928|117|1511|Saturday|1928Q4|N|Y|N|2425582|2425916|2425230|2425504|N|N|N|N|N| +2425597|AAAAAAAANPCAFCAA|1928-12-16|347|1511|117|1928|0|12|16|4|1928|117|1511|Sunday|1928Q4|N|N|N|2425582|2425916|2425231|2425505|N|N|N|N|N| +2425598|AAAAAAAAOPCAFCAA|1928-12-17|347|1511|117|1928|1|12|17|4|1928|117|1511|Monday|1928Q4|N|N|N|2425582|2425916|2425232|2425506|N|N|N|N|N| +2425599|AAAAAAAAPPCAFCAA|1928-12-18|347|1512|117|1928|2|12|18|4|1928|117|1512|Tuesday|1928Q4|N|N|N|2425582|2425916|2425233|2425507|N|N|N|N|N| +2425600|AAAAAAAAAADAFCAA|1928-12-19|347|1512|117|1928|3|12|19|4|1928|117|1512|Wednesday|1928Q4|N|N|N|2425582|2425916|2425234|2425508|N|N|N|N|N| +2425601|AAAAAAAABADAFCAA|1928-12-20|347|1512|117|1928|4|12|20|4|1928|117|1512|Thursday|1928Q4|N|N|N|2425582|2425916|2425235|2425509|N|N|N|N|N| +2425602|AAAAAAAACADAFCAA|1928-12-21|347|1512|117|1928|5|12|21|4|1928|117|1512|Friday|1928Q4|N|Y|N|2425582|2425916|2425236|2425510|N|N|N|N|N| +2425603|AAAAAAAADADAFCAA|1928-12-22|347|1512|117|1928|6|12|22|4|1928|117|1512|Saturday|1928Q4|N|Y|N|2425582|2425916|2425237|2425511|N|N|N|N|N| +2425604|AAAAAAAAEADAFCAA|1928-12-23|347|1512|117|1928|0|12|23|4|1928|117|1512|Sunday|1928Q4|N|N|N|2425582|2425916|2425238|2425512|N|N|N|N|N| +2425605|AAAAAAAAFADAFCAA|1928-12-24|347|1512|117|1928|1|12|24|4|1928|117|1512|Monday|1928Q4|N|N|N|2425582|2425916|2425239|2425513|N|N|N|N|N| +2425606|AAAAAAAAGADAFCAA|1928-12-25|347|1513|117|1928|2|12|25|4|1928|117|1513|Tuesday|1928Q4|Y|N|N|2425582|2425916|2425240|2425514|N|N|N|N|N| +2425607|AAAAAAAAHADAFCAA|1928-12-26|347|1513|117|1928|3|12|26|4|1928|117|1513|Wednesday|1928Q4|N|N|Y|2425582|2425916|2425241|2425515|N|N|N|N|N| +2425608|AAAAAAAAIADAFCAA|1928-12-27|347|1513|117|1928|4|12|27|4|1928|117|1513|Thursday|1928Q4|N|N|N|2425582|2425916|2425242|2425516|N|N|N|N|N| +2425609|AAAAAAAAJADAFCAA|1928-12-28|347|1513|117|1928|5|12|28|4|1928|117|1513|Friday|1928Q4|N|Y|N|2425582|2425916|2425243|2425517|N|N|N|N|N| +2425610|AAAAAAAAKADAFCAA|1928-12-29|347|1513|117|1928|6|12|29|4|1928|117|1513|Saturday|1928Q4|N|Y|N|2425582|2425916|2425244|2425518|N|N|N|N|N| +2425611|AAAAAAAALADAFCAA|1928-12-30|347|1513|117|1928|0|12|30|4|1928|117|1513|Sunday|1928Q4|N|N|N|2425582|2425916|2425245|2425519|N|N|N|N|N| +2425612|AAAAAAAAMADAFCAA|1928-12-31|347|1513|117|1928|1|12|31|4|1928|117|1513|Monday|1928Q4|Y|N|N|2425582|2425916|2425246|2425520|N|N|N|N|N| +2425613|AAAAAAAANADAFCAA|1929-01-01|348|1514|117|1929|2|1|1|1|1929|117|1514|Tuesday|1929Q1|Y|N|Y|2425613|2425612|2425247|2425521|N|N|N|N|N| +2425614|AAAAAAAAOADAFCAA|1929-01-02|348|1514|117|1929|3|1|2|1|1929|117|1514|Wednesday|1929Q1|N|N|Y|2425613|2425612|2425248|2425522|N|N|N|N|N| +2425615|AAAAAAAAPADAFCAA|1929-01-03|348|1514|117|1929|4|1|3|1|1929|117|1514|Thursday|1929Q1|N|N|N|2425613|2425612|2425249|2425523|N|N|N|N|N| +2425616|AAAAAAAAABDAFCAA|1929-01-04|348|1514|117|1929|5|1|4|1|1929|117|1514|Friday|1929Q1|N|Y|N|2425613|2425612|2425250|2425524|N|N|N|N|N| +2425617|AAAAAAAABBDAFCAA|1929-01-05|348|1514|117|1929|6|1|5|1|1929|117|1514|Saturday|1929Q1|N|Y|N|2425613|2425612|2425251|2425525|N|N|N|N|N| +2425618|AAAAAAAACBDAFCAA|1929-01-06|348|1514|117|1929|0|1|6|1|1929|117|1514|Sunday|1929Q1|N|N|N|2425613|2425612|2425252|2425526|N|N|N|N|N| +2425619|AAAAAAAADBDAFCAA|1929-01-07|348|1514|117|1929|1|1|7|1|1929|117|1514|Monday|1929Q1|N|N|N|2425613|2425612|2425253|2425527|N|N|N|N|N| +2425620|AAAAAAAAEBDAFCAA|1929-01-08|348|1515|117|1929|2|1|8|1|1929|117|1515|Tuesday|1929Q1|N|N|N|2425613|2425612|2425254|2425528|N|N|N|N|N| +2425621|AAAAAAAAFBDAFCAA|1929-01-09|348|1515|117|1929|3|1|9|1|1929|117|1515|Wednesday|1929Q1|N|N|N|2425613|2425612|2425255|2425529|N|N|N|N|N| +2425622|AAAAAAAAGBDAFCAA|1929-01-10|348|1515|117|1929|4|1|10|1|1929|117|1515|Thursday|1929Q1|N|N|N|2425613|2425612|2425256|2425530|N|N|N|N|N| +2425623|AAAAAAAAHBDAFCAA|1929-01-11|348|1515|117|1929|5|1|11|1|1929|117|1515|Friday|1929Q1|N|Y|N|2425613|2425612|2425257|2425531|N|N|N|N|N| +2425624|AAAAAAAAIBDAFCAA|1929-01-12|348|1515|117|1929|6|1|12|1|1929|117|1515|Saturday|1929Q1|N|Y|N|2425613|2425612|2425258|2425532|N|N|N|N|N| +2425625|AAAAAAAAJBDAFCAA|1929-01-13|348|1515|117|1929|0|1|13|1|1929|117|1515|Sunday|1929Q1|N|N|N|2425613|2425612|2425259|2425533|N|N|N|N|N| +2425626|AAAAAAAAKBDAFCAA|1929-01-14|348|1515|117|1929|1|1|14|1|1929|117|1515|Monday|1929Q1|N|N|N|2425613|2425612|2425260|2425534|N|N|N|N|N| +2425627|AAAAAAAALBDAFCAA|1929-01-15|348|1516|117|1929|2|1|15|1|1929|117|1516|Tuesday|1929Q1|N|N|N|2425613|2425612|2425261|2425535|N|N|N|N|N| +2425628|AAAAAAAAMBDAFCAA|1929-01-16|348|1516|117|1929|3|1|16|1|1929|117|1516|Wednesday|1929Q1|N|N|N|2425613|2425612|2425262|2425536|N|N|N|N|N| +2425629|AAAAAAAANBDAFCAA|1929-01-17|348|1516|117|1929|4|1|17|1|1929|117|1516|Thursday|1929Q1|N|N|N|2425613|2425612|2425263|2425537|N|N|N|N|N| +2425630|AAAAAAAAOBDAFCAA|1929-01-18|348|1516|117|1929|5|1|18|1|1929|117|1516|Friday|1929Q1|N|Y|N|2425613|2425612|2425264|2425538|N|N|N|N|N| +2425631|AAAAAAAAPBDAFCAA|1929-01-19|348|1516|117|1929|6|1|19|1|1929|117|1516|Saturday|1929Q1|N|Y|N|2425613|2425612|2425265|2425539|N|N|N|N|N| +2425632|AAAAAAAAACDAFCAA|1929-01-20|348|1516|117|1929|0|1|20|1|1929|117|1516|Sunday|1929Q1|N|N|N|2425613|2425612|2425266|2425540|N|N|N|N|N| +2425633|AAAAAAAABCDAFCAA|1929-01-21|348|1516|117|1929|1|1|21|1|1929|117|1516|Monday|1929Q1|N|N|N|2425613|2425612|2425267|2425541|N|N|N|N|N| +2425634|AAAAAAAACCDAFCAA|1929-01-22|348|1517|117|1929|2|1|22|1|1929|117|1517|Tuesday|1929Q1|N|N|N|2425613|2425612|2425268|2425542|N|N|N|N|N| +2425635|AAAAAAAADCDAFCAA|1929-01-23|348|1517|117|1929|3|1|23|1|1929|117|1517|Wednesday|1929Q1|N|N|N|2425613|2425612|2425269|2425543|N|N|N|N|N| +2425636|AAAAAAAAECDAFCAA|1929-01-24|348|1517|117|1929|4|1|24|1|1929|117|1517|Thursday|1929Q1|N|N|N|2425613|2425612|2425270|2425544|N|N|N|N|N| +2425637|AAAAAAAAFCDAFCAA|1929-01-25|348|1517|117|1929|5|1|25|1|1929|117|1517|Friday|1929Q1|N|Y|N|2425613|2425612|2425271|2425545|N|N|N|N|N| +2425638|AAAAAAAAGCDAFCAA|1929-01-26|348|1517|117|1929|6|1|26|1|1929|117|1517|Saturday|1929Q1|N|Y|N|2425613|2425612|2425272|2425546|N|N|N|N|N| +2425639|AAAAAAAAHCDAFCAA|1929-01-27|348|1517|117|1929|0|1|27|1|1929|117|1517|Sunday|1929Q1|N|N|N|2425613|2425612|2425273|2425547|N|N|N|N|N| +2425640|AAAAAAAAICDAFCAA|1929-01-28|348|1517|117|1929|1|1|28|1|1929|117|1517|Monday|1929Q1|N|N|N|2425613|2425612|2425274|2425548|N|N|N|N|N| +2425641|AAAAAAAAJCDAFCAA|1929-01-29|348|1518|117|1929|2|1|29|1|1929|117|1518|Tuesday|1929Q1|N|N|N|2425613|2425612|2425275|2425549|N|N|N|N|N| +2425642|AAAAAAAAKCDAFCAA|1929-01-30|348|1518|117|1929|3|1|30|1|1929|117|1518|Wednesday|1929Q1|N|N|N|2425613|2425612|2425276|2425550|N|N|N|N|N| +2425643|AAAAAAAALCDAFCAA|1929-01-31|348|1518|117|1929|4|1|31|1|1929|117|1518|Thursday|1929Q1|N|N|N|2425613|2425612|2425277|2425551|N|N|N|N|N| +2425644|AAAAAAAAMCDAFCAA|1929-02-01|349|1518|117|1929|5|2|1|1|1929|117|1518|Friday|1929Q1|N|Y|N|2425644|2425674|2425278|2425552|N|N|N|N|N| +2425645|AAAAAAAANCDAFCAA|1929-02-02|349|1518|117|1929|6|2|2|1|1929|117|1518|Saturday|1929Q1|N|Y|N|2425644|2425674|2425279|2425553|N|N|N|N|N| +2425646|AAAAAAAAOCDAFCAA|1929-02-03|349|1518|117|1929|0|2|3|1|1929|117|1518|Sunday|1929Q1|N|N|N|2425644|2425674|2425280|2425554|N|N|N|N|N| +2425647|AAAAAAAAPCDAFCAA|1929-02-04|349|1518|117|1929|1|2|4|1|1929|117|1518|Monday|1929Q1|N|N|N|2425644|2425674|2425281|2425555|N|N|N|N|N| +2425648|AAAAAAAAADDAFCAA|1929-02-05|349|1519|117|1929|2|2|5|1|1929|117|1519|Tuesday|1929Q1|N|N|N|2425644|2425674|2425282|2425556|N|N|N|N|N| +2425649|AAAAAAAABDDAFCAA|1929-02-06|349|1519|117|1929|3|2|6|1|1929|117|1519|Wednesday|1929Q1|N|N|N|2425644|2425674|2425283|2425557|N|N|N|N|N| +2425650|AAAAAAAACDDAFCAA|1929-02-07|349|1519|117|1929|4|2|7|1|1929|117|1519|Thursday|1929Q1|N|N|N|2425644|2425674|2425284|2425558|N|N|N|N|N| +2425651|AAAAAAAADDDAFCAA|1929-02-08|349|1519|117|1929|5|2|8|1|1929|117|1519|Friday|1929Q1|N|Y|N|2425644|2425674|2425285|2425559|N|N|N|N|N| +2425652|AAAAAAAAEDDAFCAA|1929-02-09|349|1519|117|1929|6|2|9|1|1929|117|1519|Saturday|1929Q1|N|Y|N|2425644|2425674|2425286|2425560|N|N|N|N|N| +2425653|AAAAAAAAFDDAFCAA|1929-02-10|349|1519|117|1929|0|2|10|1|1929|117|1519|Sunday|1929Q1|N|N|N|2425644|2425674|2425287|2425561|N|N|N|N|N| +2425654|AAAAAAAAGDDAFCAA|1929-02-11|349|1519|117|1929|1|2|11|1|1929|117|1519|Monday|1929Q1|N|N|N|2425644|2425674|2425288|2425562|N|N|N|N|N| +2425655|AAAAAAAAHDDAFCAA|1929-02-12|349|1520|117|1929|2|2|12|1|1929|117|1520|Tuesday|1929Q1|N|N|N|2425644|2425674|2425289|2425563|N|N|N|N|N| +2425656|AAAAAAAAIDDAFCAA|1929-02-13|349|1520|117|1929|3|2|13|1|1929|117|1520|Wednesday|1929Q1|N|N|N|2425644|2425674|2425290|2425564|N|N|N|N|N| +2425657|AAAAAAAAJDDAFCAA|1929-02-14|349|1520|117|1929|4|2|14|1|1929|117|1520|Thursday|1929Q1|N|N|N|2425644|2425674|2425291|2425565|N|N|N|N|N| +2425658|AAAAAAAAKDDAFCAA|1929-02-15|349|1520|117|1929|5|2|15|1|1929|117|1520|Friday|1929Q1|N|Y|N|2425644|2425674|2425292|2425566|N|N|N|N|N| +2425659|AAAAAAAALDDAFCAA|1929-02-16|349|1520|117|1929|6|2|16|1|1929|117|1520|Saturday|1929Q1|N|Y|N|2425644|2425674|2425293|2425567|N|N|N|N|N| +2425660|AAAAAAAAMDDAFCAA|1929-02-17|349|1520|117|1929|0|2|17|1|1929|117|1520|Sunday|1929Q1|N|N|N|2425644|2425674|2425294|2425568|N|N|N|N|N| +2425661|AAAAAAAANDDAFCAA|1929-02-18|349|1520|117|1929|1|2|18|1|1929|117|1520|Monday|1929Q1|N|N|N|2425644|2425674|2425295|2425569|N|N|N|N|N| +2425662|AAAAAAAAODDAFCAA|1929-02-19|349|1521|117|1929|2|2|19|1|1929|117|1521|Tuesday|1929Q1|N|N|N|2425644|2425674|2425296|2425570|N|N|N|N|N| +2425663|AAAAAAAAPDDAFCAA|1929-02-20|349|1521|117|1929|3|2|20|1|1929|117|1521|Wednesday|1929Q1|N|N|N|2425644|2425674|2425297|2425571|N|N|N|N|N| +2425664|AAAAAAAAAEDAFCAA|1929-02-21|349|1521|117|1929|4|2|21|1|1929|117|1521|Thursday|1929Q1|N|N|N|2425644|2425674|2425298|2425572|N|N|N|N|N| +2425665|AAAAAAAABEDAFCAA|1929-02-22|349|1521|117|1929|5|2|22|1|1929|117|1521|Friday|1929Q1|N|Y|N|2425644|2425674|2425299|2425573|N|N|N|N|N| +2425666|AAAAAAAACEDAFCAA|1929-02-23|349|1521|117|1929|6|2|23|1|1929|117|1521|Saturday|1929Q1|N|Y|N|2425644|2425674|2425300|2425574|N|N|N|N|N| +2425667|AAAAAAAADEDAFCAA|1929-02-24|349|1521|117|1929|0|2|24|1|1929|117|1521|Sunday|1929Q1|N|N|N|2425644|2425674|2425301|2425575|N|N|N|N|N| +2425668|AAAAAAAAEEDAFCAA|1929-02-25|349|1521|117|1929|1|2|25|1|1929|117|1521|Monday|1929Q1|N|N|N|2425644|2425674|2425302|2425576|N|N|N|N|N| +2425669|AAAAAAAAFEDAFCAA|1929-02-26|349|1522|117|1929|2|2|26|1|1929|117|1522|Tuesday|1929Q1|N|N|N|2425644|2425674|2425303|2425577|N|N|N|N|N| +2425670|AAAAAAAAGEDAFCAA|1929-02-27|349|1522|117|1929|3|2|27|1|1929|117|1522|Wednesday|1929Q1|N|N|N|2425644|2425674|2425304|2425578|N|N|N|N|N| +2425671|AAAAAAAAHEDAFCAA|1929-02-28|349|1522|117|1929|4|2|28|1|1929|117|1522|Thursday|1929Q1|N|N|N|2425644|2425674|2425305|2425579|N|N|N|N|N| +2425672|AAAAAAAAIEDAFCAA|1929-03-01|350|1522|118|1929|5|3|1|1|1929|118|1522|Friday|1929Q1|N|Y|N|2425672|2425730|2425307|2425580|N|N|N|N|N| +2425673|AAAAAAAAJEDAFCAA|1929-03-02|350|1522|118|1929|6|3|2|1|1929|118|1522|Saturday|1929Q1|N|Y|N|2425672|2425730|2425308|2425581|N|N|N|N|N| +2425674|AAAAAAAAKEDAFCAA|1929-03-03|350|1522|118|1929|0|3|3|1|1929|118|1522|Sunday|1929Q1|N|N|N|2425672|2425730|2425309|2425582|N|N|N|N|N| +2425675|AAAAAAAALEDAFCAA|1929-03-04|350|1522|118|1929|1|3|4|1|1929|118|1522|Monday|1929Q1|N|N|N|2425672|2425730|2425310|2425583|N|N|N|N|N| +2425676|AAAAAAAAMEDAFCAA|1929-03-05|350|1523|118|1929|2|3|5|1|1929|118|1523|Tuesday|1929Q1|N|N|N|2425672|2425730|2425311|2425584|N|N|N|N|N| +2425677|AAAAAAAANEDAFCAA|1929-03-06|350|1523|118|1929|3|3|6|1|1929|118|1523|Wednesday|1929Q1|N|N|N|2425672|2425730|2425312|2425585|N|N|N|N|N| +2425678|AAAAAAAAOEDAFCAA|1929-03-07|350|1523|118|1929|4|3|7|1|1929|118|1523|Thursday|1929Q1|N|N|N|2425672|2425730|2425313|2425586|N|N|N|N|N| +2425679|AAAAAAAAPEDAFCAA|1929-03-08|350|1523|118|1929|5|3|8|1|1929|118|1523|Friday|1929Q1|N|Y|N|2425672|2425730|2425314|2425587|N|N|N|N|N| +2425680|AAAAAAAAAFDAFCAA|1929-03-09|350|1523|118|1929|6|3|9|1|1929|118|1523|Saturday|1929Q1|N|Y|N|2425672|2425730|2425315|2425588|N|N|N|N|N| +2425681|AAAAAAAABFDAFCAA|1929-03-10|350|1523|118|1929|0|3|10|1|1929|118|1523|Sunday|1929Q1|N|N|N|2425672|2425730|2425316|2425589|N|N|N|N|N| +2425682|AAAAAAAACFDAFCAA|1929-03-11|350|1523|118|1929|1|3|11|1|1929|118|1523|Monday|1929Q1|N|N|N|2425672|2425730|2425317|2425590|N|N|N|N|N| +2425683|AAAAAAAADFDAFCAA|1929-03-12|350|1524|118|1929|2|3|12|1|1929|118|1524|Tuesday|1929Q1|N|N|N|2425672|2425730|2425318|2425591|N|N|N|N|N| +2425684|AAAAAAAAEFDAFCAA|1929-03-13|350|1524|118|1929|3|3|13|1|1929|118|1524|Wednesday|1929Q1|N|N|N|2425672|2425730|2425319|2425592|N|N|N|N|N| +2425685|AAAAAAAAFFDAFCAA|1929-03-14|350|1524|118|1929|4|3|14|1|1929|118|1524|Thursday|1929Q1|N|N|N|2425672|2425730|2425320|2425593|N|N|N|N|N| +2425686|AAAAAAAAGFDAFCAA|1929-03-15|350|1524|118|1929|5|3|15|1|1929|118|1524|Friday|1929Q1|N|Y|N|2425672|2425730|2425321|2425594|N|N|N|N|N| +2425687|AAAAAAAAHFDAFCAA|1929-03-16|350|1524|118|1929|6|3|16|1|1929|118|1524|Saturday|1929Q1|N|Y|N|2425672|2425730|2425322|2425595|N|N|N|N|N| +2425688|AAAAAAAAIFDAFCAA|1929-03-17|350|1524|118|1929|0|3|17|1|1929|118|1524|Sunday|1929Q1|N|N|N|2425672|2425730|2425323|2425596|N|N|N|N|N| +2425689|AAAAAAAAJFDAFCAA|1929-03-18|350|1524|118|1929|1|3|18|1|1929|118|1524|Monday|1929Q1|N|N|N|2425672|2425730|2425324|2425597|N|N|N|N|N| +2425690|AAAAAAAAKFDAFCAA|1929-03-19|350|1525|118|1929|2|3|19|1|1929|118|1525|Tuesday|1929Q1|N|N|N|2425672|2425730|2425325|2425598|N|N|N|N|N| +2425691|AAAAAAAALFDAFCAA|1929-03-20|350|1525|118|1929|3|3|20|1|1929|118|1525|Wednesday|1929Q1|N|N|N|2425672|2425730|2425326|2425599|N|N|N|N|N| +2425692|AAAAAAAAMFDAFCAA|1929-03-21|350|1525|118|1929|4|3|21|1|1929|118|1525|Thursday|1929Q1|N|N|N|2425672|2425730|2425327|2425600|N|N|N|N|N| +2425693|AAAAAAAANFDAFCAA|1929-03-22|350|1525|118|1929|5|3|22|1|1929|118|1525|Friday|1929Q1|N|Y|N|2425672|2425730|2425328|2425601|N|N|N|N|N| +2425694|AAAAAAAAOFDAFCAA|1929-03-23|350|1525|118|1929|6|3|23|1|1929|118|1525|Saturday|1929Q1|N|Y|N|2425672|2425730|2425329|2425602|N|N|N|N|N| +2425695|AAAAAAAAPFDAFCAA|1929-03-24|350|1525|118|1929|0|3|24|1|1929|118|1525|Sunday|1929Q1|N|N|N|2425672|2425730|2425330|2425603|N|N|N|N|N| +2425696|AAAAAAAAAGDAFCAA|1929-03-25|350|1525|118|1929|1|3|25|1|1929|118|1525|Monday|1929Q1|N|N|N|2425672|2425730|2425331|2425604|N|N|N|N|N| +2425697|AAAAAAAABGDAFCAA|1929-03-26|350|1526|118|1929|2|3|26|1|1929|118|1526|Tuesday|1929Q1|N|N|N|2425672|2425730|2425332|2425605|N|N|N|N|N| +2425698|AAAAAAAACGDAFCAA|1929-03-27|350|1526|118|1929|3|3|27|1|1929|118|1526|Wednesday|1929Q1|N|N|N|2425672|2425730|2425333|2425606|N|N|N|N|N| +2425699|AAAAAAAADGDAFCAA|1929-03-28|350|1526|118|1929|4|3|28|1|1929|118|1526|Thursday|1929Q1|N|N|N|2425672|2425730|2425334|2425607|N|N|N|N|N| +2425700|AAAAAAAAEGDAFCAA|1929-03-29|350|1526|118|1929|5|3|29|1|1929|118|1526|Friday|1929Q1|N|Y|N|2425672|2425730|2425335|2425608|N|N|N|N|N| +2425701|AAAAAAAAFGDAFCAA|1929-03-30|350|1526|118|1929|6|3|30|1|1929|118|1526|Saturday|1929Q1|N|Y|N|2425672|2425730|2425336|2425609|N|N|N|N|N| +2425702|AAAAAAAAGGDAFCAA|1929-03-31|350|1526|118|1929|0|3|31|1|1929|118|1526|Sunday|1929Q1|N|N|N|2425672|2425730|2425337|2425610|N|N|N|N|N| +2425703|AAAAAAAAHGDAFCAA|1929-04-01|351|1526|118|1929|1|4|1|1|1929|118|1526|Monday|1929Q1|N|N|N|2425703|2425792|2425338|2425613|N|N|N|N|N| +2425704|AAAAAAAAIGDAFCAA|1929-04-02|351|1527|118|1929|2|4|2|2|1929|118|1527|Tuesday|1929Q2|N|N|N|2425703|2425792|2425339|2425614|N|N|N|N|N| +2425705|AAAAAAAAJGDAFCAA|1929-04-03|351|1527|118|1929|3|4|3|2|1929|118|1527|Wednesday|1929Q2|N|N|N|2425703|2425792|2425340|2425615|N|N|N|N|N| +2425706|AAAAAAAAKGDAFCAA|1929-04-04|351|1527|118|1929|4|4|4|2|1929|118|1527|Thursday|1929Q2|N|N|N|2425703|2425792|2425341|2425616|N|N|N|N|N| +2425707|AAAAAAAALGDAFCAA|1929-04-05|351|1527|118|1929|5|4|5|2|1929|118|1527|Friday|1929Q2|N|Y|N|2425703|2425792|2425342|2425617|N|N|N|N|N| +2425708|AAAAAAAAMGDAFCAA|1929-04-06|351|1527|118|1929|6|4|6|2|1929|118|1527|Saturday|1929Q2|N|Y|N|2425703|2425792|2425343|2425618|N|N|N|N|N| +2425709|AAAAAAAANGDAFCAA|1929-04-07|351|1527|118|1929|0|4|7|2|1929|118|1527|Sunday|1929Q2|N|N|N|2425703|2425792|2425344|2425619|N|N|N|N|N| +2425710|AAAAAAAAOGDAFCAA|1929-04-08|351|1527|118|1929|1|4|8|2|1929|118|1527|Monday|1929Q2|N|N|N|2425703|2425792|2425345|2425620|N|N|N|N|N| +2425711|AAAAAAAAPGDAFCAA|1929-04-09|351|1528|118|1929|2|4|9|2|1929|118|1528|Tuesday|1929Q2|N|N|N|2425703|2425792|2425346|2425621|N|N|N|N|N| +2425712|AAAAAAAAAHDAFCAA|1929-04-10|351|1528|118|1929|3|4|10|2|1929|118|1528|Wednesday|1929Q2|N|N|N|2425703|2425792|2425347|2425622|N|N|N|N|N| +2425713|AAAAAAAABHDAFCAA|1929-04-11|351|1528|118|1929|4|4|11|2|1929|118|1528|Thursday|1929Q2|N|N|N|2425703|2425792|2425348|2425623|N|N|N|N|N| +2425714|AAAAAAAACHDAFCAA|1929-04-12|351|1528|118|1929|5|4|12|2|1929|118|1528|Friday|1929Q2|N|Y|N|2425703|2425792|2425349|2425624|N|N|N|N|N| +2425715|AAAAAAAADHDAFCAA|1929-04-13|351|1528|118|1929|6|4|13|2|1929|118|1528|Saturday|1929Q2|N|Y|N|2425703|2425792|2425350|2425625|N|N|N|N|N| +2425716|AAAAAAAAEHDAFCAA|1929-04-14|351|1528|118|1929|0|4|14|2|1929|118|1528|Sunday|1929Q2|N|N|N|2425703|2425792|2425351|2425626|N|N|N|N|N| +2425717|AAAAAAAAFHDAFCAA|1929-04-15|351|1528|118|1929|1|4|15|2|1929|118|1528|Monday|1929Q2|N|N|N|2425703|2425792|2425352|2425627|N|N|N|N|N| +2425718|AAAAAAAAGHDAFCAA|1929-04-16|351|1529|118|1929|2|4|16|2|1929|118|1529|Tuesday|1929Q2|N|N|N|2425703|2425792|2425353|2425628|N|N|N|N|N| +2425719|AAAAAAAAHHDAFCAA|1929-04-17|351|1529|118|1929|3|4|17|2|1929|118|1529|Wednesday|1929Q2|N|N|N|2425703|2425792|2425354|2425629|N|N|N|N|N| +2425720|AAAAAAAAIHDAFCAA|1929-04-18|351|1529|118|1929|4|4|18|2|1929|118|1529|Thursday|1929Q2|N|N|N|2425703|2425792|2425355|2425630|N|N|N|N|N| +2425721|AAAAAAAAJHDAFCAA|1929-04-19|351|1529|118|1929|5|4|19|2|1929|118|1529|Friday|1929Q2|N|Y|N|2425703|2425792|2425356|2425631|N|N|N|N|N| +2425722|AAAAAAAAKHDAFCAA|1929-04-20|351|1529|118|1929|6|4|20|2|1929|118|1529|Saturday|1929Q2|N|Y|N|2425703|2425792|2425357|2425632|N|N|N|N|N| +2425723|AAAAAAAALHDAFCAA|1929-04-21|351|1529|118|1929|0|4|21|2|1929|118|1529|Sunday|1929Q2|N|N|N|2425703|2425792|2425358|2425633|N|N|N|N|N| +2425724|AAAAAAAAMHDAFCAA|1929-04-22|351|1529|118|1929|1|4|22|2|1929|118|1529|Monday|1929Q2|N|N|N|2425703|2425792|2425359|2425634|N|N|N|N|N| +2425725|AAAAAAAANHDAFCAA|1929-04-23|351|1530|118|1929|2|4|23|2|1929|118|1530|Tuesday|1929Q2|N|N|N|2425703|2425792|2425360|2425635|N|N|N|N|N| +2425726|AAAAAAAAOHDAFCAA|1929-04-24|351|1530|118|1929|3|4|24|2|1929|118|1530|Wednesday|1929Q2|N|N|N|2425703|2425792|2425361|2425636|N|N|N|N|N| +2425727|AAAAAAAAPHDAFCAA|1929-04-25|351|1530|118|1929|4|4|25|2|1929|118|1530|Thursday|1929Q2|N|N|N|2425703|2425792|2425362|2425637|N|N|N|N|N| +2425728|AAAAAAAAAIDAFCAA|1929-04-26|351|1530|118|1929|5|4|26|2|1929|118|1530|Friday|1929Q2|N|Y|N|2425703|2425792|2425363|2425638|N|N|N|N|N| +2425729|AAAAAAAABIDAFCAA|1929-04-27|351|1530|118|1929|6|4|27|2|1929|118|1530|Saturday|1929Q2|N|Y|N|2425703|2425792|2425364|2425639|N|N|N|N|N| +2425730|AAAAAAAACIDAFCAA|1929-04-28|351|1530|118|1929|0|4|28|2|1929|118|1530|Sunday|1929Q2|N|N|N|2425703|2425792|2425365|2425640|N|N|N|N|N| +2425731|AAAAAAAADIDAFCAA|1929-04-29|351|1530|118|1929|1|4|29|2|1929|118|1530|Monday|1929Q2|N|N|N|2425703|2425792|2425366|2425641|N|N|N|N|N| +2425732|AAAAAAAAEIDAFCAA|1929-04-30|351|1531|118|1929|2|4|30|2|1929|118|1531|Tuesday|1929Q2|N|N|N|2425703|2425792|2425367|2425642|N|N|N|N|N| +2425733|AAAAAAAAFIDAFCAA|1929-05-01|352|1531|118|1929|3|5|1|2|1929|118|1531|Wednesday|1929Q2|N|N|N|2425733|2425852|2425368|2425643|N|N|N|N|N| +2425734|AAAAAAAAGIDAFCAA|1929-05-02|352|1531|118|1929|4|5|2|2|1929|118|1531|Thursday|1929Q2|N|N|N|2425733|2425852|2425369|2425644|N|N|N|N|N| +2425735|AAAAAAAAHIDAFCAA|1929-05-03|352|1531|118|1929|5|5|3|2|1929|118|1531|Friday|1929Q2|N|Y|N|2425733|2425852|2425370|2425645|N|N|N|N|N| +2425736|AAAAAAAAIIDAFCAA|1929-05-04|352|1531|118|1929|6|5|4|2|1929|118|1531|Saturday|1929Q2|N|Y|N|2425733|2425852|2425371|2425646|N|N|N|N|N| +2425737|AAAAAAAAJIDAFCAA|1929-05-05|352|1531|118|1929|0|5|5|2|1929|118|1531|Sunday|1929Q2|N|N|N|2425733|2425852|2425372|2425647|N|N|N|N|N| +2425738|AAAAAAAAKIDAFCAA|1929-05-06|352|1531|118|1929|1|5|6|2|1929|118|1531|Monday|1929Q2|N|N|N|2425733|2425852|2425373|2425648|N|N|N|N|N| +2425739|AAAAAAAALIDAFCAA|1929-05-07|352|1532|118|1929|2|5|7|2|1929|118|1532|Tuesday|1929Q2|N|N|N|2425733|2425852|2425374|2425649|N|N|N|N|N| +2425740|AAAAAAAAMIDAFCAA|1929-05-08|352|1532|118|1929|3|5|8|2|1929|118|1532|Wednesday|1929Q2|N|N|N|2425733|2425852|2425375|2425650|N|N|N|N|N| +2425741|AAAAAAAANIDAFCAA|1929-05-09|352|1532|118|1929|4|5|9|2|1929|118|1532|Thursday|1929Q2|N|N|N|2425733|2425852|2425376|2425651|N|N|N|N|N| +2425742|AAAAAAAAOIDAFCAA|1929-05-10|352|1532|118|1929|5|5|10|2|1929|118|1532|Friday|1929Q2|N|Y|N|2425733|2425852|2425377|2425652|N|N|N|N|N| +2425743|AAAAAAAAPIDAFCAA|1929-05-11|352|1532|118|1929|6|5|11|2|1929|118|1532|Saturday|1929Q2|N|Y|N|2425733|2425852|2425378|2425653|N|N|N|N|N| +2425744|AAAAAAAAAJDAFCAA|1929-05-12|352|1532|118|1929|0|5|12|2|1929|118|1532|Sunday|1929Q2|N|N|N|2425733|2425852|2425379|2425654|N|N|N|N|N| +2425745|AAAAAAAABJDAFCAA|1929-05-13|352|1532|118|1929|1|5|13|2|1929|118|1532|Monday|1929Q2|N|N|N|2425733|2425852|2425380|2425655|N|N|N|N|N| +2425746|AAAAAAAACJDAFCAA|1929-05-14|352|1533|118|1929|2|5|14|2|1929|118|1533|Tuesday|1929Q2|N|N|N|2425733|2425852|2425381|2425656|N|N|N|N|N| +2425747|AAAAAAAADJDAFCAA|1929-05-15|352|1533|118|1929|3|5|15|2|1929|118|1533|Wednesday|1929Q2|N|N|N|2425733|2425852|2425382|2425657|N|N|N|N|N| +2425748|AAAAAAAAEJDAFCAA|1929-05-16|352|1533|118|1929|4|5|16|2|1929|118|1533|Thursday|1929Q2|N|N|N|2425733|2425852|2425383|2425658|N|N|N|N|N| +2425749|AAAAAAAAFJDAFCAA|1929-05-17|352|1533|118|1929|5|5|17|2|1929|118|1533|Friday|1929Q2|N|Y|N|2425733|2425852|2425384|2425659|N|N|N|N|N| +2425750|AAAAAAAAGJDAFCAA|1929-05-18|352|1533|118|1929|6|5|18|2|1929|118|1533|Saturday|1929Q2|N|Y|N|2425733|2425852|2425385|2425660|N|N|N|N|N| +2425751|AAAAAAAAHJDAFCAA|1929-05-19|352|1533|118|1929|0|5|19|2|1929|118|1533|Sunday|1929Q2|N|N|N|2425733|2425852|2425386|2425661|N|N|N|N|N| +2425752|AAAAAAAAIJDAFCAA|1929-05-20|352|1533|118|1929|1|5|20|2|1929|118|1533|Monday|1929Q2|N|N|N|2425733|2425852|2425387|2425662|N|N|N|N|N| +2425753|AAAAAAAAJJDAFCAA|1929-05-21|352|1534|118|1929|2|5|21|2|1929|118|1534|Tuesday|1929Q2|N|N|N|2425733|2425852|2425388|2425663|N|N|N|N|N| +2425754|AAAAAAAAKJDAFCAA|1929-05-22|352|1534|118|1929|3|5|22|2|1929|118|1534|Wednesday|1929Q2|N|N|N|2425733|2425852|2425389|2425664|N|N|N|N|N| +2425755|AAAAAAAALJDAFCAA|1929-05-23|352|1534|118|1929|4|5|23|2|1929|118|1534|Thursday|1929Q2|N|N|N|2425733|2425852|2425390|2425665|N|N|N|N|N| +2425756|AAAAAAAAMJDAFCAA|1929-05-24|352|1534|118|1929|5|5|24|2|1929|118|1534|Friday|1929Q2|N|Y|N|2425733|2425852|2425391|2425666|N|N|N|N|N| +2425757|AAAAAAAANJDAFCAA|1929-05-25|352|1534|118|1929|6|5|25|2|1929|118|1534|Saturday|1929Q2|N|Y|N|2425733|2425852|2425392|2425667|N|N|N|N|N| +2425758|AAAAAAAAOJDAFCAA|1929-05-26|352|1534|118|1929|0|5|26|2|1929|118|1534|Sunday|1929Q2|N|N|N|2425733|2425852|2425393|2425668|N|N|N|N|N| +2425759|AAAAAAAAPJDAFCAA|1929-05-27|352|1534|118|1929|1|5|27|2|1929|118|1534|Monday|1929Q2|N|N|N|2425733|2425852|2425394|2425669|N|N|N|N|N| +2425760|AAAAAAAAAKDAFCAA|1929-05-28|352|1535|118|1929|2|5|28|2|1929|118|1535|Tuesday|1929Q2|N|N|N|2425733|2425852|2425395|2425670|N|N|N|N|N| +2425761|AAAAAAAABKDAFCAA|1929-05-29|352|1535|118|1929|3|5|29|2|1929|118|1535|Wednesday|1929Q2|N|N|N|2425733|2425852|2425396|2425671|N|N|N|N|N| +2425762|AAAAAAAACKDAFCAA|1929-05-30|352|1535|118|1929|4|5|30|2|1929|118|1535|Thursday|1929Q2|N|N|N|2425733|2425852|2425397|2425672|N|N|N|N|N| +2425763|AAAAAAAADKDAFCAA|1929-05-31|352|1535|118|1929|5|5|31|2|1929|118|1535|Friday|1929Q2|N|Y|N|2425733|2425852|2425398|2425673|N|N|N|N|N| +2425764|AAAAAAAAEKDAFCAA|1929-06-01|353|1535|119|1929|6|6|1|2|1929|119|1535|Saturday|1929Q2|N|Y|N|2425764|2425914|2425399|2425674|N|N|N|N|N| +2425765|AAAAAAAAFKDAFCAA|1929-06-02|353|1535|119|1929|0|6|2|2|1929|119|1535|Sunday|1929Q2|N|N|N|2425764|2425914|2425400|2425675|N|N|N|N|N| +2425766|AAAAAAAAGKDAFCAA|1929-06-03|353|1535|119|1929|1|6|3|2|1929|119|1535|Monday|1929Q2|N|N|N|2425764|2425914|2425401|2425676|N|N|N|N|N| +2425767|AAAAAAAAHKDAFCAA|1929-06-04|353|1536|119|1929|2|6|4|2|1929|119|1536|Tuesday|1929Q2|N|N|N|2425764|2425914|2425402|2425677|N|N|N|N|N| +2425768|AAAAAAAAIKDAFCAA|1929-06-05|353|1536|119|1929|3|6|5|2|1929|119|1536|Wednesday|1929Q2|N|N|N|2425764|2425914|2425403|2425678|N|N|N|N|N| +2425769|AAAAAAAAJKDAFCAA|1929-06-06|353|1536|119|1929|4|6|6|2|1929|119|1536|Thursday|1929Q2|N|N|N|2425764|2425914|2425404|2425679|N|N|N|N|N| +2425770|AAAAAAAAKKDAFCAA|1929-06-07|353|1536|119|1929|5|6|7|2|1929|119|1536|Friday|1929Q2|N|Y|N|2425764|2425914|2425405|2425680|N|N|N|N|N| +2425771|AAAAAAAALKDAFCAA|1929-06-08|353|1536|119|1929|6|6|8|2|1929|119|1536|Saturday|1929Q2|N|Y|N|2425764|2425914|2425406|2425681|N|N|N|N|N| +2425772|AAAAAAAAMKDAFCAA|1929-06-09|353|1536|119|1929|0|6|9|2|1929|119|1536|Sunday|1929Q2|N|N|N|2425764|2425914|2425407|2425682|N|N|N|N|N| +2425773|AAAAAAAANKDAFCAA|1929-06-10|353|1536|119|1929|1|6|10|2|1929|119|1536|Monday|1929Q2|N|N|N|2425764|2425914|2425408|2425683|N|N|N|N|N| +2425774|AAAAAAAAOKDAFCAA|1929-06-11|353|1537|119|1929|2|6|11|2|1929|119|1537|Tuesday|1929Q2|N|N|N|2425764|2425914|2425409|2425684|N|N|N|N|N| +2425775|AAAAAAAAPKDAFCAA|1929-06-12|353|1537|119|1929|3|6|12|2|1929|119|1537|Wednesday|1929Q2|N|N|N|2425764|2425914|2425410|2425685|N|N|N|N|N| +2425776|AAAAAAAAALDAFCAA|1929-06-13|353|1537|119|1929|4|6|13|2|1929|119|1537|Thursday|1929Q2|N|N|N|2425764|2425914|2425411|2425686|N|N|N|N|N| +2425777|AAAAAAAABLDAFCAA|1929-06-14|353|1537|119|1929|5|6|14|2|1929|119|1537|Friday|1929Q2|N|Y|N|2425764|2425914|2425412|2425687|N|N|N|N|N| +2425778|AAAAAAAACLDAFCAA|1929-06-15|353|1537|119|1929|6|6|15|2|1929|119|1537|Saturday|1929Q2|N|Y|N|2425764|2425914|2425413|2425688|N|N|N|N|N| +2425779|AAAAAAAADLDAFCAA|1929-06-16|353|1537|119|1929|0|6|16|2|1929|119|1537|Sunday|1929Q2|N|N|N|2425764|2425914|2425414|2425689|N|N|N|N|N| +2425780|AAAAAAAAELDAFCAA|1929-06-17|353|1537|119|1929|1|6|17|2|1929|119|1537|Monday|1929Q2|N|N|N|2425764|2425914|2425415|2425690|N|N|N|N|N| +2425781|AAAAAAAAFLDAFCAA|1929-06-18|353|1538|119|1929|2|6|18|2|1929|119|1538|Tuesday|1929Q2|N|N|N|2425764|2425914|2425416|2425691|N|N|N|N|N| +2425782|AAAAAAAAGLDAFCAA|1929-06-19|353|1538|119|1929|3|6|19|2|1929|119|1538|Wednesday|1929Q2|N|N|N|2425764|2425914|2425417|2425692|N|N|N|N|N| +2425783|AAAAAAAAHLDAFCAA|1929-06-20|353|1538|119|1929|4|6|20|2|1929|119|1538|Thursday|1929Q2|N|N|N|2425764|2425914|2425418|2425693|N|N|N|N|N| +2425784|AAAAAAAAILDAFCAA|1929-06-21|353|1538|119|1929|5|6|21|2|1929|119|1538|Friday|1929Q2|N|Y|N|2425764|2425914|2425419|2425694|N|N|N|N|N| +2425785|AAAAAAAAJLDAFCAA|1929-06-22|353|1538|119|1929|6|6|22|2|1929|119|1538|Saturday|1929Q2|N|Y|N|2425764|2425914|2425420|2425695|N|N|N|N|N| +2425786|AAAAAAAAKLDAFCAA|1929-06-23|353|1538|119|1929|0|6|23|2|1929|119|1538|Sunday|1929Q2|N|N|N|2425764|2425914|2425421|2425696|N|N|N|N|N| +2425787|AAAAAAAALLDAFCAA|1929-06-24|353|1538|119|1929|1|6|24|2|1929|119|1538|Monday|1929Q2|N|N|N|2425764|2425914|2425422|2425697|N|N|N|N|N| +2425788|AAAAAAAAMLDAFCAA|1929-06-25|353|1539|119|1929|2|6|25|2|1929|119|1539|Tuesday|1929Q2|N|N|N|2425764|2425914|2425423|2425698|N|N|N|N|N| +2425789|AAAAAAAANLDAFCAA|1929-06-26|353|1539|119|1929|3|6|26|2|1929|119|1539|Wednesday|1929Q2|N|N|N|2425764|2425914|2425424|2425699|N|N|N|N|N| +2425790|AAAAAAAAOLDAFCAA|1929-06-27|353|1539|119|1929|4|6|27|2|1929|119|1539|Thursday|1929Q2|N|N|N|2425764|2425914|2425425|2425700|N|N|N|N|N| +2425791|AAAAAAAAPLDAFCAA|1929-06-28|353|1539|119|1929|5|6|28|2|1929|119|1539|Friday|1929Q2|N|Y|N|2425764|2425914|2425426|2425701|N|N|N|N|N| +2425792|AAAAAAAAAMDAFCAA|1929-06-29|353|1539|119|1929|6|6|29|2|1929|119|1539|Saturday|1929Q2|N|Y|N|2425764|2425914|2425427|2425702|N|N|N|N|N| +2425793|AAAAAAAABMDAFCAA|1929-06-30|353|1539|119|1929|0|6|30|2|1929|119|1539|Sunday|1929Q2|N|N|N|2425764|2425914|2425428|2425703|N|N|N|N|N| +2425794|AAAAAAAACMDAFCAA|1929-07-01|354|1539|119|1929|1|7|1|2|1929|119|1539|Monday|1929Q2|N|N|N|2425794|2425974|2425429|2425703|N|N|N|N|N| +2425795|AAAAAAAADMDAFCAA|1929-07-02|354|1540|119|1929|2|7|2|3|1929|119|1540|Tuesday|1929Q3|N|N|N|2425794|2425974|2425430|2425704|N|N|N|N|N| +2425796|AAAAAAAAEMDAFCAA|1929-07-03|354|1540|119|1929|3|7|3|3|1929|119|1540|Wednesday|1929Q3|N|N|N|2425794|2425974|2425431|2425705|N|N|N|N|N| +2425797|AAAAAAAAFMDAFCAA|1929-07-04|354|1540|119|1929|4|7|4|3|1929|119|1540|Thursday|1929Q3|N|N|N|2425794|2425974|2425432|2425706|N|N|N|N|N| +2425798|AAAAAAAAGMDAFCAA|1929-07-05|354|1540|119|1929|5|7|5|3|1929|119|1540|Friday|1929Q3|Y|Y|N|2425794|2425974|2425433|2425707|N|N|N|N|N| +2425799|AAAAAAAAHMDAFCAA|1929-07-06|354|1540|119|1929|6|7|6|3|1929|119|1540|Saturday|1929Q3|N|Y|Y|2425794|2425974|2425434|2425708|N|N|N|N|N| +2425800|AAAAAAAAIMDAFCAA|1929-07-07|354|1540|119|1929|0|7|7|3|1929|119|1540|Sunday|1929Q3|N|N|N|2425794|2425974|2425435|2425709|N|N|N|N|N| +2425801|AAAAAAAAJMDAFCAA|1929-07-08|354|1540|119|1929|1|7|8|3|1929|119|1540|Monday|1929Q3|N|N|N|2425794|2425974|2425436|2425710|N|N|N|N|N| +2425802|AAAAAAAAKMDAFCAA|1929-07-09|354|1541|119|1929|2|7|9|3|1929|119|1541|Tuesday|1929Q3|N|N|N|2425794|2425974|2425437|2425711|N|N|N|N|N| +2425803|AAAAAAAALMDAFCAA|1929-07-10|354|1541|119|1929|3|7|10|3|1929|119|1541|Wednesday|1929Q3|N|N|N|2425794|2425974|2425438|2425712|N|N|N|N|N| +2425804|AAAAAAAAMMDAFCAA|1929-07-11|354|1541|119|1929|4|7|11|3|1929|119|1541|Thursday|1929Q3|N|N|N|2425794|2425974|2425439|2425713|N|N|N|N|N| +2425805|AAAAAAAANMDAFCAA|1929-07-12|354|1541|119|1929|5|7|12|3|1929|119|1541|Friday|1929Q3|N|Y|N|2425794|2425974|2425440|2425714|N|N|N|N|N| +2425806|AAAAAAAAOMDAFCAA|1929-07-13|354|1541|119|1929|6|7|13|3|1929|119|1541|Saturday|1929Q3|N|Y|N|2425794|2425974|2425441|2425715|N|N|N|N|N| +2425807|AAAAAAAAPMDAFCAA|1929-07-14|354|1541|119|1929|0|7|14|3|1929|119|1541|Sunday|1929Q3|N|N|N|2425794|2425974|2425442|2425716|N|N|N|N|N| +2425808|AAAAAAAAANDAFCAA|1929-07-15|354|1541|119|1929|1|7|15|3|1929|119|1541|Monday|1929Q3|N|N|N|2425794|2425974|2425443|2425717|N|N|N|N|N| +2425809|AAAAAAAABNDAFCAA|1929-07-16|354|1542|119|1929|2|7|16|3|1929|119|1542|Tuesday|1929Q3|N|N|N|2425794|2425974|2425444|2425718|N|N|N|N|N| +2425810|AAAAAAAACNDAFCAA|1929-07-17|354|1542|119|1929|3|7|17|3|1929|119|1542|Wednesday|1929Q3|N|N|N|2425794|2425974|2425445|2425719|N|N|N|N|N| +2425811|AAAAAAAADNDAFCAA|1929-07-18|354|1542|119|1929|4|7|18|3|1929|119|1542|Thursday|1929Q3|N|N|N|2425794|2425974|2425446|2425720|N|N|N|N|N| +2425812|AAAAAAAAENDAFCAA|1929-07-19|354|1542|119|1929|5|7|19|3|1929|119|1542|Friday|1929Q3|N|Y|N|2425794|2425974|2425447|2425721|N|N|N|N|N| +2425813|AAAAAAAAFNDAFCAA|1929-07-20|354|1542|119|1929|6|7|20|3|1929|119|1542|Saturday|1929Q3|N|Y|N|2425794|2425974|2425448|2425722|N|N|N|N|N| +2425814|AAAAAAAAGNDAFCAA|1929-07-21|354|1542|119|1929|0|7|21|3|1929|119|1542|Sunday|1929Q3|N|N|N|2425794|2425974|2425449|2425723|N|N|N|N|N| +2425815|AAAAAAAAHNDAFCAA|1929-07-22|354|1542|119|1929|1|7|22|3|1929|119|1542|Monday|1929Q3|N|N|N|2425794|2425974|2425450|2425724|N|N|N|N|N| +2425816|AAAAAAAAINDAFCAA|1929-07-23|354|1543|119|1929|2|7|23|3|1929|119|1543|Tuesday|1929Q3|N|N|N|2425794|2425974|2425451|2425725|N|N|N|N|N| +2425817|AAAAAAAAJNDAFCAA|1929-07-24|354|1543|119|1929|3|7|24|3|1929|119|1543|Wednesday|1929Q3|N|N|N|2425794|2425974|2425452|2425726|N|N|N|N|N| +2425818|AAAAAAAAKNDAFCAA|1929-07-25|354|1543|119|1929|4|7|25|3|1929|119|1543|Thursday|1929Q3|N|N|N|2425794|2425974|2425453|2425727|N|N|N|N|N| +2425819|AAAAAAAALNDAFCAA|1929-07-26|354|1543|119|1929|5|7|26|3|1929|119|1543|Friday|1929Q3|N|Y|N|2425794|2425974|2425454|2425728|N|N|N|N|N| +2425820|AAAAAAAAMNDAFCAA|1929-07-27|354|1543|119|1929|6|7|27|3|1929|119|1543|Saturday|1929Q3|N|Y|N|2425794|2425974|2425455|2425729|N|N|N|N|N| +2425821|AAAAAAAANNDAFCAA|1929-07-28|354|1543|119|1929|0|7|28|3|1929|119|1543|Sunday|1929Q3|N|N|N|2425794|2425974|2425456|2425730|N|N|N|N|N| +2425822|AAAAAAAAONDAFCAA|1929-07-29|354|1543|119|1929|1|7|29|3|1929|119|1543|Monday|1929Q3|N|N|N|2425794|2425974|2425457|2425731|N|N|N|N|N| +2425823|AAAAAAAAPNDAFCAA|1929-07-30|354|1544|119|1929|2|7|30|3|1929|119|1544|Tuesday|1929Q3|N|N|N|2425794|2425974|2425458|2425732|N|N|N|N|N| +2425824|AAAAAAAAAODAFCAA|1929-07-31|354|1544|119|1929|3|7|31|3|1929|119|1544|Wednesday|1929Q3|N|N|N|2425794|2425974|2425459|2425733|N|N|N|N|N| +2425825|AAAAAAAABODAFCAA|1929-08-01|355|1544|119|1929|4|8|1|3|1929|119|1544|Thursday|1929Q3|N|N|N|2425825|2426036|2425460|2425734|N|N|N|N|N| +2425826|AAAAAAAACODAFCAA|1929-08-02|355|1544|119|1929|5|8|2|3|1929|119|1544|Friday|1929Q3|N|Y|N|2425825|2426036|2425461|2425735|N|N|N|N|N| +2425827|AAAAAAAADODAFCAA|1929-08-03|355|1544|119|1929|6|8|3|3|1929|119|1544|Saturday|1929Q3|N|Y|N|2425825|2426036|2425462|2425736|N|N|N|N|N| +2425828|AAAAAAAAEODAFCAA|1929-08-04|355|1544|119|1929|0|8|4|3|1929|119|1544|Sunday|1929Q3|N|N|N|2425825|2426036|2425463|2425737|N|N|N|N|N| +2425829|AAAAAAAAFODAFCAA|1929-08-05|355|1544|119|1929|1|8|5|3|1929|119|1544|Monday|1929Q3|N|N|N|2425825|2426036|2425464|2425738|N|N|N|N|N| +2425830|AAAAAAAAGODAFCAA|1929-08-06|355|1545|119|1929|2|8|6|3|1929|119|1545|Tuesday|1929Q3|N|N|N|2425825|2426036|2425465|2425739|N|N|N|N|N| +2425831|AAAAAAAAHODAFCAA|1929-08-07|355|1545|119|1929|3|8|7|3|1929|119|1545|Wednesday|1929Q3|N|N|N|2425825|2426036|2425466|2425740|N|N|N|N|N| +2425832|AAAAAAAAIODAFCAA|1929-08-08|355|1545|119|1929|4|8|8|3|1929|119|1545|Thursday|1929Q3|N|N|N|2425825|2426036|2425467|2425741|N|N|N|N|N| +2425833|AAAAAAAAJODAFCAA|1929-08-09|355|1545|119|1929|5|8|9|3|1929|119|1545|Friday|1929Q3|N|Y|N|2425825|2426036|2425468|2425742|N|N|N|N|N| +2425834|AAAAAAAAKODAFCAA|1929-08-10|355|1545|119|1929|6|8|10|3|1929|119|1545|Saturday|1929Q3|N|Y|N|2425825|2426036|2425469|2425743|N|N|N|N|N| +2425835|AAAAAAAALODAFCAA|1929-08-11|355|1545|119|1929|0|8|11|3|1929|119|1545|Sunday|1929Q3|N|N|N|2425825|2426036|2425470|2425744|N|N|N|N|N| +2425836|AAAAAAAAMODAFCAA|1929-08-12|355|1545|119|1929|1|8|12|3|1929|119|1545|Monday|1929Q3|N|N|N|2425825|2426036|2425471|2425745|N|N|N|N|N| +2425837|AAAAAAAANODAFCAA|1929-08-13|355|1546|119|1929|2|8|13|3|1929|119|1546|Tuesday|1929Q3|N|N|N|2425825|2426036|2425472|2425746|N|N|N|N|N| +2425838|AAAAAAAAOODAFCAA|1929-08-14|355|1546|119|1929|3|8|14|3|1929|119|1546|Wednesday|1929Q3|N|N|N|2425825|2426036|2425473|2425747|N|N|N|N|N| +2425839|AAAAAAAAPODAFCAA|1929-08-15|355|1546|119|1929|4|8|15|3|1929|119|1546|Thursday|1929Q3|N|N|N|2425825|2426036|2425474|2425748|N|N|N|N|N| +2425840|AAAAAAAAAPDAFCAA|1929-08-16|355|1546|119|1929|5|8|16|3|1929|119|1546|Friday|1929Q3|N|Y|N|2425825|2426036|2425475|2425749|N|N|N|N|N| +2425841|AAAAAAAABPDAFCAA|1929-08-17|355|1546|119|1929|6|8|17|3|1929|119|1546|Saturday|1929Q3|N|Y|N|2425825|2426036|2425476|2425750|N|N|N|N|N| +2425842|AAAAAAAACPDAFCAA|1929-08-18|355|1546|119|1929|0|8|18|3|1929|119|1546|Sunday|1929Q3|N|N|N|2425825|2426036|2425477|2425751|N|N|N|N|N| +2425843|AAAAAAAADPDAFCAA|1929-08-19|355|1546|119|1929|1|8|19|3|1929|119|1546|Monday|1929Q3|N|N|N|2425825|2426036|2425478|2425752|N|N|N|N|N| +2425844|AAAAAAAAEPDAFCAA|1929-08-20|355|1547|119|1929|2|8|20|3|1929|119|1547|Tuesday|1929Q3|N|N|N|2425825|2426036|2425479|2425753|N|N|N|N|N| +2425845|AAAAAAAAFPDAFCAA|1929-08-21|355|1547|119|1929|3|8|21|3|1929|119|1547|Wednesday|1929Q3|N|N|N|2425825|2426036|2425480|2425754|N|N|N|N|N| +2425846|AAAAAAAAGPDAFCAA|1929-08-22|355|1547|119|1929|4|8|22|3|1929|119|1547|Thursday|1929Q3|N|N|N|2425825|2426036|2425481|2425755|N|N|N|N|N| +2425847|AAAAAAAAHPDAFCAA|1929-08-23|355|1547|119|1929|5|8|23|3|1929|119|1547|Friday|1929Q3|N|Y|N|2425825|2426036|2425482|2425756|N|N|N|N|N| +2425848|AAAAAAAAIPDAFCAA|1929-08-24|355|1547|119|1929|6|8|24|3|1929|119|1547|Saturday|1929Q3|N|Y|N|2425825|2426036|2425483|2425757|N|N|N|N|N| +2425849|AAAAAAAAJPDAFCAA|1929-08-25|355|1547|119|1929|0|8|25|3|1929|119|1547|Sunday|1929Q3|N|N|N|2425825|2426036|2425484|2425758|N|N|N|N|N| +2425850|AAAAAAAAKPDAFCAA|1929-08-26|355|1547|119|1929|1|8|26|3|1929|119|1547|Monday|1929Q3|N|N|N|2425825|2426036|2425485|2425759|N|N|N|N|N| +2425851|AAAAAAAALPDAFCAA|1929-08-27|355|1548|119|1929|2|8|27|3|1929|119|1548|Tuesday|1929Q3|N|N|N|2425825|2426036|2425486|2425760|N|N|N|N|N| +2425852|AAAAAAAAMPDAFCAA|1929-08-28|355|1548|119|1929|3|8|28|3|1929|119|1548|Wednesday|1929Q3|N|N|N|2425825|2426036|2425487|2425761|N|N|N|N|N| +2425853|AAAAAAAANPDAFCAA|1929-08-29|355|1548|119|1929|4|8|29|3|1929|119|1548|Thursday|1929Q3|N|N|N|2425825|2426036|2425488|2425762|N|N|N|N|N| +2425854|AAAAAAAAOPDAFCAA|1929-08-30|355|1548|119|1929|5|8|30|3|1929|119|1548|Friday|1929Q3|N|Y|N|2425825|2426036|2425489|2425763|N|N|N|N|N| +2425855|AAAAAAAAPPDAFCAA|1929-08-31|355|1548|119|1929|6|8|31|3|1929|119|1548|Saturday|1929Q3|N|Y|N|2425825|2426036|2425490|2425764|N|N|N|N|N| +2425856|AAAAAAAAAAEAFCAA|1929-09-01|356|1548|120|1929|0|9|1|3|1929|120|1548|Sunday|1929Q3|N|N|N|2425856|2426098|2425491|2425765|N|N|N|N|N| +2425857|AAAAAAAABAEAFCAA|1929-09-02|356|1548|120|1929|1|9|2|3|1929|120|1548|Monday|1929Q3|N|N|N|2425856|2426098|2425492|2425766|N|N|N|N|N| +2425858|AAAAAAAACAEAFCAA|1929-09-03|356|1549|120|1929|2|9|3|3|1929|120|1549|Tuesday|1929Q3|N|N|N|2425856|2426098|2425493|2425767|N|N|N|N|N| +2425859|AAAAAAAADAEAFCAA|1929-09-04|356|1549|120|1929|3|9|4|3|1929|120|1549|Wednesday|1929Q3|N|N|N|2425856|2426098|2425494|2425768|N|N|N|N|N| +2425860|AAAAAAAAEAEAFCAA|1929-09-05|356|1549|120|1929|4|9|5|3|1929|120|1549|Thursday|1929Q3|N|N|N|2425856|2426098|2425495|2425769|N|N|N|N|N| +2425861|AAAAAAAAFAEAFCAA|1929-09-06|356|1549|120|1929|5|9|6|3|1929|120|1549|Friday|1929Q3|N|Y|N|2425856|2426098|2425496|2425770|N|N|N|N|N| +2425862|AAAAAAAAGAEAFCAA|1929-09-07|356|1549|120|1929|6|9|7|3|1929|120|1549|Saturday|1929Q3|N|Y|N|2425856|2426098|2425497|2425771|N|N|N|N|N| +2425863|AAAAAAAAHAEAFCAA|1929-09-08|356|1549|120|1929|0|9|8|3|1929|120|1549|Sunday|1929Q3|N|N|N|2425856|2426098|2425498|2425772|N|N|N|N|N| +2425864|AAAAAAAAIAEAFCAA|1929-09-09|356|1549|120|1929|1|9|9|3|1929|120|1549|Monday|1929Q3|N|N|N|2425856|2426098|2425499|2425773|N|N|N|N|N| +2425865|AAAAAAAAJAEAFCAA|1929-09-10|356|1550|120|1929|2|9|10|3|1929|120|1550|Tuesday|1929Q3|N|N|N|2425856|2426098|2425500|2425774|N|N|N|N|N| +2425866|AAAAAAAAKAEAFCAA|1929-09-11|356|1550|120|1929|3|9|11|3|1929|120|1550|Wednesday|1929Q3|N|N|N|2425856|2426098|2425501|2425775|N|N|N|N|N| +2425867|AAAAAAAALAEAFCAA|1929-09-12|356|1550|120|1929|4|9|12|3|1929|120|1550|Thursday|1929Q3|N|N|N|2425856|2426098|2425502|2425776|N|N|N|N|N| +2425868|AAAAAAAAMAEAFCAA|1929-09-13|356|1550|120|1929|5|9|13|3|1929|120|1550|Friday|1929Q3|N|Y|N|2425856|2426098|2425503|2425777|N|N|N|N|N| +2425869|AAAAAAAANAEAFCAA|1929-09-14|356|1550|120|1929|6|9|14|3|1929|120|1550|Saturday|1929Q3|N|Y|N|2425856|2426098|2425504|2425778|N|N|N|N|N| +2425870|AAAAAAAAOAEAFCAA|1929-09-15|356|1550|120|1929|0|9|15|3|1929|120|1550|Sunday|1929Q3|N|N|N|2425856|2426098|2425505|2425779|N|N|N|N|N| +2425871|AAAAAAAAPAEAFCAA|1929-09-16|356|1550|120|1929|1|9|16|3|1929|120|1550|Monday|1929Q3|N|N|N|2425856|2426098|2425506|2425780|N|N|N|N|N| +2425872|AAAAAAAAABEAFCAA|1929-09-17|356|1551|120|1929|2|9|17|3|1929|120|1551|Tuesday|1929Q3|N|N|N|2425856|2426098|2425507|2425781|N|N|N|N|N| +2425873|AAAAAAAABBEAFCAA|1929-09-18|356|1551|120|1929|3|9|18|3|1929|120|1551|Wednesday|1929Q3|N|N|N|2425856|2426098|2425508|2425782|N|N|N|N|N| +2425874|AAAAAAAACBEAFCAA|1929-09-19|356|1551|120|1929|4|9|19|3|1929|120|1551|Thursday|1929Q3|N|N|N|2425856|2426098|2425509|2425783|N|N|N|N|N| +2425875|AAAAAAAADBEAFCAA|1929-09-20|356|1551|120|1929|5|9|20|3|1929|120|1551|Friday|1929Q3|N|Y|N|2425856|2426098|2425510|2425784|N|N|N|N|N| +2425876|AAAAAAAAEBEAFCAA|1929-09-21|356|1551|120|1929|6|9|21|3|1929|120|1551|Saturday|1929Q3|N|Y|N|2425856|2426098|2425511|2425785|N|N|N|N|N| +2425877|AAAAAAAAFBEAFCAA|1929-09-22|356|1551|120|1929|0|9|22|3|1929|120|1551|Sunday|1929Q3|N|N|N|2425856|2426098|2425512|2425786|N|N|N|N|N| +2425878|AAAAAAAAGBEAFCAA|1929-09-23|356|1551|120|1929|1|9|23|3|1929|120|1551|Monday|1929Q3|N|N|N|2425856|2426098|2425513|2425787|N|N|N|N|N| +2425879|AAAAAAAAHBEAFCAA|1929-09-24|356|1552|120|1929|2|9|24|3|1929|120|1552|Tuesday|1929Q3|N|N|N|2425856|2426098|2425514|2425788|N|N|N|N|N| +2425880|AAAAAAAAIBEAFCAA|1929-09-25|356|1552|120|1929|3|9|25|3|1929|120|1552|Wednesday|1929Q3|N|N|N|2425856|2426098|2425515|2425789|N|N|N|N|N| +2425881|AAAAAAAAJBEAFCAA|1929-09-26|356|1552|120|1929|4|9|26|3|1929|120|1552|Thursday|1929Q3|N|N|N|2425856|2426098|2425516|2425790|N|N|N|N|N| +2425882|AAAAAAAAKBEAFCAA|1929-09-27|356|1552|120|1929|5|9|27|3|1929|120|1552|Friday|1929Q3|N|Y|N|2425856|2426098|2425517|2425791|N|N|N|N|N| +2425883|AAAAAAAALBEAFCAA|1929-09-28|356|1552|120|1929|6|9|28|3|1929|120|1552|Saturday|1929Q3|N|Y|N|2425856|2426098|2425518|2425792|N|N|N|N|N| +2425884|AAAAAAAAMBEAFCAA|1929-09-29|356|1552|120|1929|0|9|29|3|1929|120|1552|Sunday|1929Q3|N|N|N|2425856|2426098|2425519|2425793|N|N|N|N|N| +2425885|AAAAAAAANBEAFCAA|1929-09-30|356|1552|120|1929|1|9|30|3|1929|120|1552|Monday|1929Q3|N|N|N|2425856|2426098|2425520|2425794|N|N|N|N|N| +2425886|AAAAAAAAOBEAFCAA|1929-10-01|357|1553|120|1929|2|10|1|3|1929|120|1553|Tuesday|1929Q3|N|N|N|2425886|2426158|2425521|2425794|N|N|N|N|N| +2425887|AAAAAAAAPBEAFCAA|1929-10-02|357|1553|120|1929|3|10|2|4|1929|120|1553|Wednesday|1929Q4|N|N|N|2425886|2426158|2425522|2425795|N|N|N|N|N| +2425888|AAAAAAAAACEAFCAA|1929-10-03|357|1553|120|1929|4|10|3|4|1929|120|1553|Thursday|1929Q4|N|N|N|2425886|2426158|2425523|2425796|N|N|N|N|N| +2425889|AAAAAAAABCEAFCAA|1929-10-04|357|1553|120|1929|5|10|4|4|1929|120|1553|Friday|1929Q4|N|Y|N|2425886|2426158|2425524|2425797|N|N|N|N|N| +2425890|AAAAAAAACCEAFCAA|1929-10-05|357|1553|120|1929|6|10|5|4|1929|120|1553|Saturday|1929Q4|N|Y|N|2425886|2426158|2425525|2425798|N|N|N|N|N| +2425891|AAAAAAAADCEAFCAA|1929-10-06|357|1553|120|1929|0|10|6|4|1929|120|1553|Sunday|1929Q4|N|N|N|2425886|2426158|2425526|2425799|N|N|N|N|N| +2425892|AAAAAAAAECEAFCAA|1929-10-07|357|1553|120|1929|1|10|7|4|1929|120|1553|Monday|1929Q4|N|N|N|2425886|2426158|2425527|2425800|N|N|N|N|N| +2425893|AAAAAAAAFCEAFCAA|1929-10-08|357|1554|120|1929|2|10|8|4|1929|120|1554|Tuesday|1929Q4|N|N|N|2425886|2426158|2425528|2425801|N|N|N|N|N| +2425894|AAAAAAAAGCEAFCAA|1929-10-09|357|1554|120|1929|3|10|9|4|1929|120|1554|Wednesday|1929Q4|N|N|N|2425886|2426158|2425529|2425802|N|N|N|N|N| +2425895|AAAAAAAAHCEAFCAA|1929-10-10|357|1554|120|1929|4|10|10|4|1929|120|1554|Thursday|1929Q4|N|N|N|2425886|2426158|2425530|2425803|N|N|N|N|N| +2425896|AAAAAAAAICEAFCAA|1929-10-11|357|1554|120|1929|5|10|11|4|1929|120|1554|Friday|1929Q4|N|Y|N|2425886|2426158|2425531|2425804|N|N|N|N|N| +2425897|AAAAAAAAJCEAFCAA|1929-10-12|357|1554|120|1929|6|10|12|4|1929|120|1554|Saturday|1929Q4|N|Y|N|2425886|2426158|2425532|2425805|N|N|N|N|N| +2425898|AAAAAAAAKCEAFCAA|1929-10-13|357|1554|120|1929|0|10|13|4|1929|120|1554|Sunday|1929Q4|N|N|N|2425886|2426158|2425533|2425806|N|N|N|N|N| +2425899|AAAAAAAALCEAFCAA|1929-10-14|357|1554|120|1929|1|10|14|4|1929|120|1554|Monday|1929Q4|N|N|N|2425886|2426158|2425534|2425807|N|N|N|N|N| +2425900|AAAAAAAAMCEAFCAA|1929-10-15|357|1555|120|1929|2|10|15|4|1929|120|1555|Tuesday|1929Q4|N|N|N|2425886|2426158|2425535|2425808|N|N|N|N|N| +2425901|AAAAAAAANCEAFCAA|1929-10-16|357|1555|120|1929|3|10|16|4|1929|120|1555|Wednesday|1929Q4|N|N|N|2425886|2426158|2425536|2425809|N|N|N|N|N| +2425902|AAAAAAAAOCEAFCAA|1929-10-17|357|1555|120|1929|4|10|17|4|1929|120|1555|Thursday|1929Q4|N|N|N|2425886|2426158|2425537|2425810|N|N|N|N|N| +2425903|AAAAAAAAPCEAFCAA|1929-10-18|357|1555|120|1929|5|10|18|4|1929|120|1555|Friday|1929Q4|N|Y|N|2425886|2426158|2425538|2425811|N|N|N|N|N| +2425904|AAAAAAAAADEAFCAA|1929-10-19|357|1555|120|1929|6|10|19|4|1929|120|1555|Saturday|1929Q4|N|Y|N|2425886|2426158|2425539|2425812|N|N|N|N|N| +2425905|AAAAAAAABDEAFCAA|1929-10-20|357|1555|120|1929|0|10|20|4|1929|120|1555|Sunday|1929Q4|N|N|N|2425886|2426158|2425540|2425813|N|N|N|N|N| +2425906|AAAAAAAACDEAFCAA|1929-10-21|357|1555|120|1929|1|10|21|4|1929|120|1555|Monday|1929Q4|N|N|N|2425886|2426158|2425541|2425814|N|N|N|N|N| +2425907|AAAAAAAADDEAFCAA|1929-10-22|357|1556|120|1929|2|10|22|4|1929|120|1556|Tuesday|1929Q4|N|N|N|2425886|2426158|2425542|2425815|N|N|N|N|N| +2425908|AAAAAAAAEDEAFCAA|1929-10-23|357|1556|120|1929|3|10|23|4|1929|120|1556|Wednesday|1929Q4|N|N|N|2425886|2426158|2425543|2425816|N|N|N|N|N| +2425909|AAAAAAAAFDEAFCAA|1929-10-24|357|1556|120|1929|4|10|24|4|1929|120|1556|Thursday|1929Q4|N|N|N|2425886|2426158|2425544|2425817|N|N|N|N|N| +2425910|AAAAAAAAGDEAFCAA|1929-10-25|357|1556|120|1929|5|10|25|4|1929|120|1556|Friday|1929Q4|N|Y|N|2425886|2426158|2425545|2425818|N|N|N|N|N| +2425911|AAAAAAAAHDEAFCAA|1929-10-26|357|1556|120|1929|6|10|26|4|1929|120|1556|Saturday|1929Q4|N|Y|N|2425886|2426158|2425546|2425819|N|N|N|N|N| +2425912|AAAAAAAAIDEAFCAA|1929-10-27|357|1556|120|1929|0|10|27|4|1929|120|1556|Sunday|1929Q4|N|N|N|2425886|2426158|2425547|2425820|N|N|N|N|N| +2425913|AAAAAAAAJDEAFCAA|1929-10-28|357|1556|120|1929|1|10|28|4|1929|120|1556|Monday|1929Q4|N|N|N|2425886|2426158|2425548|2425821|N|N|N|N|N| +2425914|AAAAAAAAKDEAFCAA|1929-10-29|357|1557|120|1929|2|10|29|4|1929|120|1557|Tuesday|1929Q4|N|N|N|2425886|2426158|2425549|2425822|N|N|N|N|N| +2425915|AAAAAAAALDEAFCAA|1929-10-30|357|1557|120|1929|3|10|30|4|1929|120|1557|Wednesday|1929Q4|N|N|N|2425886|2426158|2425550|2425823|N|N|N|N|N| +2425916|AAAAAAAAMDEAFCAA|1929-10-31|357|1557|120|1929|4|10|31|4|1929|120|1557|Thursday|1929Q4|N|N|N|2425886|2426158|2425551|2425824|N|N|N|N|N| +2425917|AAAAAAAANDEAFCAA|1929-11-01|358|1557|120|1929|5|11|1|4|1929|120|1557|Friday|1929Q4|N|Y|N|2425917|2426220|2425552|2425825|N|N|N|N|N| +2425918|AAAAAAAAODEAFCAA|1929-11-02|358|1557|120|1929|6|11|2|4|1929|120|1557|Saturday|1929Q4|N|Y|N|2425917|2426220|2425553|2425826|N|N|N|N|N| +2425919|AAAAAAAAPDEAFCAA|1929-11-03|358|1557|120|1929|0|11|3|4|1929|120|1557|Sunday|1929Q4|N|N|N|2425917|2426220|2425554|2425827|N|N|N|N|N| +2425920|AAAAAAAAAEEAFCAA|1929-11-04|358|1557|120|1929|1|11|4|4|1929|120|1557|Monday|1929Q4|N|N|N|2425917|2426220|2425555|2425828|N|N|N|N|N| +2425921|AAAAAAAABEEAFCAA|1929-11-05|358|1558|120|1929|2|11|5|4|1929|120|1558|Tuesday|1929Q4|N|N|N|2425917|2426220|2425556|2425829|N|N|N|N|N| +2425922|AAAAAAAACEEAFCAA|1929-11-06|358|1558|120|1929|3|11|6|4|1929|120|1558|Wednesday|1929Q4|N|N|N|2425917|2426220|2425557|2425830|N|N|N|N|N| +2425923|AAAAAAAADEEAFCAA|1929-11-07|358|1558|120|1929|4|11|7|4|1929|120|1558|Thursday|1929Q4|N|N|N|2425917|2426220|2425558|2425831|N|N|N|N|N| +2425924|AAAAAAAAEEEAFCAA|1929-11-08|358|1558|120|1929|5|11|8|4|1929|120|1558|Friday|1929Q4|N|Y|N|2425917|2426220|2425559|2425832|N|N|N|N|N| +2425925|AAAAAAAAFEEAFCAA|1929-11-09|358|1558|120|1929|6|11|9|4|1929|120|1558|Saturday|1929Q4|N|Y|N|2425917|2426220|2425560|2425833|N|N|N|N|N| +2425926|AAAAAAAAGEEAFCAA|1929-11-10|358|1558|120|1929|0|11|10|4|1929|120|1558|Sunday|1929Q4|N|N|N|2425917|2426220|2425561|2425834|N|N|N|N|N| +2425927|AAAAAAAAHEEAFCAA|1929-11-11|358|1558|120|1929|1|11|11|4|1929|120|1558|Monday|1929Q4|N|N|N|2425917|2426220|2425562|2425835|N|N|N|N|N| +2425928|AAAAAAAAIEEAFCAA|1929-11-12|358|1559|120|1929|2|11|12|4|1929|120|1559|Tuesday|1929Q4|N|N|N|2425917|2426220|2425563|2425836|N|N|N|N|N| +2425929|AAAAAAAAJEEAFCAA|1929-11-13|358|1559|120|1929|3|11|13|4|1929|120|1559|Wednesday|1929Q4|N|N|N|2425917|2426220|2425564|2425837|N|N|N|N|N| +2425930|AAAAAAAAKEEAFCAA|1929-11-14|358|1559|120|1929|4|11|14|4|1929|120|1559|Thursday|1929Q4|N|N|N|2425917|2426220|2425565|2425838|N|N|N|N|N| +2425931|AAAAAAAALEEAFCAA|1929-11-15|358|1559|120|1929|5|11|15|4|1929|120|1559|Friday|1929Q4|N|Y|N|2425917|2426220|2425566|2425839|N|N|N|N|N| +2425932|AAAAAAAAMEEAFCAA|1929-11-16|358|1559|120|1929|6|11|16|4|1929|120|1559|Saturday|1929Q4|N|Y|N|2425917|2426220|2425567|2425840|N|N|N|N|N| +2425933|AAAAAAAANEEAFCAA|1929-11-17|358|1559|120|1929|0|11|17|4|1929|120|1559|Sunday|1929Q4|N|N|N|2425917|2426220|2425568|2425841|N|N|N|N|N| +2425934|AAAAAAAAOEEAFCAA|1929-11-18|358|1559|120|1929|1|11|18|4|1929|120|1559|Monday|1929Q4|N|N|N|2425917|2426220|2425569|2425842|N|N|N|N|N| +2425935|AAAAAAAAPEEAFCAA|1929-11-19|358|1560|120|1929|2|11|19|4|1929|120|1560|Tuesday|1929Q4|N|N|N|2425917|2426220|2425570|2425843|N|N|N|N|N| +2425936|AAAAAAAAAFEAFCAA|1929-11-20|358|1560|120|1929|3|11|20|4|1929|120|1560|Wednesday|1929Q4|N|N|N|2425917|2426220|2425571|2425844|N|N|N|N|N| +2425937|AAAAAAAABFEAFCAA|1929-11-21|358|1560|120|1929|4|11|21|4|1929|120|1560|Thursday|1929Q4|N|N|N|2425917|2426220|2425572|2425845|N|N|N|N|N| +2425938|AAAAAAAACFEAFCAA|1929-11-22|358|1560|120|1929|5|11|22|4|1929|120|1560|Friday|1929Q4|N|Y|N|2425917|2426220|2425573|2425846|N|N|N|N|N| +2425939|AAAAAAAADFEAFCAA|1929-11-23|358|1560|120|1929|6|11|23|4|1929|120|1560|Saturday|1929Q4|N|Y|N|2425917|2426220|2425574|2425847|N|N|N|N|N| +2425940|AAAAAAAAEFEAFCAA|1929-11-24|358|1560|120|1929|0|11|24|4|1929|120|1560|Sunday|1929Q4|N|N|N|2425917|2426220|2425575|2425848|N|N|N|N|N| +2425941|AAAAAAAAFFEAFCAA|1929-11-25|358|1560|120|1929|1|11|25|4|1929|120|1560|Monday|1929Q4|N|N|N|2425917|2426220|2425576|2425849|N|N|N|N|N| +2425942|AAAAAAAAGFEAFCAA|1929-11-26|358|1561|120|1929|2|11|26|4|1929|120|1561|Tuesday|1929Q4|N|N|N|2425917|2426220|2425577|2425850|N|N|N|N|N| +2425943|AAAAAAAAHFEAFCAA|1929-11-27|358|1561|120|1929|3|11|27|4|1929|120|1561|Wednesday|1929Q4|N|N|N|2425917|2426220|2425578|2425851|N|N|N|N|N| +2425944|AAAAAAAAIFEAFCAA|1929-11-28|358|1561|120|1929|4|11|28|4|1929|120|1561|Thursday|1929Q4|N|N|N|2425917|2426220|2425579|2425852|N|N|N|N|N| +2425945|AAAAAAAAJFEAFCAA|1929-11-29|358|1561|120|1929|5|11|29|4|1929|120|1561|Friday|1929Q4|N|Y|N|2425917|2426220|2425580|2425853|N|N|N|N|N| +2425946|AAAAAAAAKFEAFCAA|1929-11-30|358|1561|120|1929|6|11|30|4|1929|120|1561|Saturday|1929Q4|N|Y|N|2425917|2426220|2425581|2425854|N|N|N|N|N| +2425947|AAAAAAAALFEAFCAA|1929-12-01|359|1561|121|1929|0|12|1|4|1929|121|1561|Sunday|1929Q4|N|N|N|2425947|2426280|2425582|2425855|N|N|N|N|N| +2425948|AAAAAAAAMFEAFCAA|1929-12-02|359|1561|121|1929|1|12|2|4|1929|121|1561|Monday|1929Q4|N|N|N|2425947|2426280|2425583|2425856|N|N|N|N|N| +2425949|AAAAAAAANFEAFCAA|1929-12-03|359|1562|121|1929|2|12|3|4|1929|121|1562|Tuesday|1929Q4|N|N|N|2425947|2426280|2425584|2425857|N|N|N|N|N| +2425950|AAAAAAAAOFEAFCAA|1929-12-04|359|1562|121|1929|3|12|4|4|1929|121|1562|Wednesday|1929Q4|N|N|N|2425947|2426280|2425585|2425858|N|N|N|N|N| +2425951|AAAAAAAAPFEAFCAA|1929-12-05|359|1562|121|1929|4|12|5|4|1929|121|1562|Thursday|1929Q4|N|N|N|2425947|2426280|2425586|2425859|N|N|N|N|N| +2425952|AAAAAAAAAGEAFCAA|1929-12-06|359|1562|121|1929|5|12|6|4|1929|121|1562|Friday|1929Q4|N|Y|N|2425947|2426280|2425587|2425860|N|N|N|N|N| +2425953|AAAAAAAABGEAFCAA|1929-12-07|359|1562|121|1929|6|12|7|4|1929|121|1562|Saturday|1929Q4|N|Y|N|2425947|2426280|2425588|2425861|N|N|N|N|N| +2425954|AAAAAAAACGEAFCAA|1929-12-08|359|1562|121|1929|0|12|8|4|1929|121|1562|Sunday|1929Q4|N|N|N|2425947|2426280|2425589|2425862|N|N|N|N|N| +2425955|AAAAAAAADGEAFCAA|1929-12-09|359|1562|121|1929|1|12|9|4|1929|121|1562|Monday|1929Q4|N|N|N|2425947|2426280|2425590|2425863|N|N|N|N|N| +2425956|AAAAAAAAEGEAFCAA|1929-12-10|359|1563|121|1929|2|12|10|4|1929|121|1563|Tuesday|1929Q4|N|N|N|2425947|2426280|2425591|2425864|N|N|N|N|N| +2425957|AAAAAAAAFGEAFCAA|1929-12-11|359|1563|121|1929|3|12|11|4|1929|121|1563|Wednesday|1929Q4|N|N|N|2425947|2426280|2425592|2425865|N|N|N|N|N| +2425958|AAAAAAAAGGEAFCAA|1929-12-12|359|1563|121|1929|4|12|12|4|1929|121|1563|Thursday|1929Q4|N|N|N|2425947|2426280|2425593|2425866|N|N|N|N|N| +2425959|AAAAAAAAHGEAFCAA|1929-12-13|359|1563|121|1929|5|12|13|4|1929|121|1563|Friday|1929Q4|N|Y|N|2425947|2426280|2425594|2425867|N|N|N|N|N| +2425960|AAAAAAAAIGEAFCAA|1929-12-14|359|1563|121|1929|6|12|14|4|1929|121|1563|Saturday|1929Q4|N|Y|N|2425947|2426280|2425595|2425868|N|N|N|N|N| +2425961|AAAAAAAAJGEAFCAA|1929-12-15|359|1563|121|1929|0|12|15|4|1929|121|1563|Sunday|1929Q4|N|N|N|2425947|2426280|2425596|2425869|N|N|N|N|N| +2425962|AAAAAAAAKGEAFCAA|1929-12-16|359|1563|121|1929|1|12|16|4|1929|121|1563|Monday|1929Q4|N|N|N|2425947|2426280|2425597|2425870|N|N|N|N|N| +2425963|AAAAAAAALGEAFCAA|1929-12-17|359|1564|121|1929|2|12|17|4|1929|121|1564|Tuesday|1929Q4|N|N|N|2425947|2426280|2425598|2425871|N|N|N|N|N| +2425964|AAAAAAAAMGEAFCAA|1929-12-18|359|1564|121|1929|3|12|18|4|1929|121|1564|Wednesday|1929Q4|N|N|N|2425947|2426280|2425599|2425872|N|N|N|N|N| +2425965|AAAAAAAANGEAFCAA|1929-12-19|359|1564|121|1929|4|12|19|4|1929|121|1564|Thursday|1929Q4|N|N|N|2425947|2426280|2425600|2425873|N|N|N|N|N| +2425966|AAAAAAAAOGEAFCAA|1929-12-20|359|1564|121|1929|5|12|20|4|1929|121|1564|Friday|1929Q4|N|Y|N|2425947|2426280|2425601|2425874|N|N|N|N|N| +2425967|AAAAAAAAPGEAFCAA|1929-12-21|359|1564|121|1929|6|12|21|4|1929|121|1564|Saturday|1929Q4|N|Y|N|2425947|2426280|2425602|2425875|N|N|N|N|N| +2425968|AAAAAAAAAHEAFCAA|1929-12-22|359|1564|121|1929|0|12|22|4|1929|121|1564|Sunday|1929Q4|N|N|N|2425947|2426280|2425603|2425876|N|N|N|N|N| +2425969|AAAAAAAABHEAFCAA|1929-12-23|359|1564|121|1929|1|12|23|4|1929|121|1564|Monday|1929Q4|N|N|N|2425947|2426280|2425604|2425877|N|N|N|N|N| +2425970|AAAAAAAACHEAFCAA|1929-12-24|359|1565|121|1929|2|12|24|4|1929|121|1565|Tuesday|1929Q4|N|N|N|2425947|2426280|2425605|2425878|N|N|N|N|N| +2425971|AAAAAAAADHEAFCAA|1929-12-25|359|1565|121|1929|3|12|25|4|1929|121|1565|Wednesday|1929Q4|N|N|N|2425947|2426280|2425606|2425879|N|N|N|N|N| +2425972|AAAAAAAAEHEAFCAA|1929-12-26|359|1565|121|1929|4|12|26|4|1929|121|1565|Thursday|1929Q4|Y|N|N|2425947|2426280|2425607|2425880|N|N|N|N|N| +2425973|AAAAAAAAFHEAFCAA|1929-12-27|359|1565|121|1929|5|12|27|4|1929|121|1565|Friday|1929Q4|N|Y|Y|2425947|2426280|2425608|2425881|N|N|N|N|N| +2425974|AAAAAAAAGHEAFCAA|1929-12-28|359|1565|121|1929|6|12|28|4|1929|121|1565|Saturday|1929Q4|N|Y|N|2425947|2426280|2425609|2425882|N|N|N|N|N| +2425975|AAAAAAAAHHEAFCAA|1929-12-29|359|1565|121|1929|0|12|29|4|1929|121|1565|Sunday|1929Q4|N|N|N|2425947|2426280|2425610|2425883|N|N|N|N|N| +2425976|AAAAAAAAIHEAFCAA|1929-12-30|359|1565|121|1929|1|12|30|4|1929|121|1565|Monday|1929Q4|N|N|N|2425947|2426280|2425611|2425884|N|N|N|N|N| +2425977|AAAAAAAAJHEAFCAA|1929-12-31|359|1566|121|1929|2|12|31|4|1929|121|1566|Tuesday|1929Q4|N|N|N|2425947|2426280|2425612|2425885|N|N|N|N|N| +2425978|AAAAAAAAKHEAFCAA|1930-01-01|360|1566|121|1930|3|1|1|1|1930|121|1566|Wednesday|1930Q1|Y|N|N|2425978|2425977|2425613|2425886|N|N|N|N|N| +2425979|AAAAAAAALHEAFCAA|1930-01-02|360|1566|121|1930|4|1|2|1|1930|121|1566|Thursday|1930Q1|N|N|Y|2425978|2425977|2425614|2425887|N|N|N|N|N| +2425980|AAAAAAAAMHEAFCAA|1930-01-03|360|1566|121|1930|5|1|3|1|1930|121|1566|Friday|1930Q1|N|Y|N|2425978|2425977|2425615|2425888|N|N|N|N|N| +2425981|AAAAAAAANHEAFCAA|1930-01-04|360|1566|121|1930|6|1|4|1|1930|121|1566|Saturday|1930Q1|N|Y|N|2425978|2425977|2425616|2425889|N|N|N|N|N| +2425982|AAAAAAAAOHEAFCAA|1930-01-05|360|1566|121|1930|0|1|5|1|1930|121|1566|Sunday|1930Q1|N|N|N|2425978|2425977|2425617|2425890|N|N|N|N|N| +2425983|AAAAAAAAPHEAFCAA|1930-01-06|360|1566|121|1930|1|1|6|1|1930|121|1566|Monday|1930Q1|N|N|N|2425978|2425977|2425618|2425891|N|N|N|N|N| +2425984|AAAAAAAAAIEAFCAA|1930-01-07|360|1567|121|1930|2|1|7|1|1930|121|1567|Tuesday|1930Q1|N|N|N|2425978|2425977|2425619|2425892|N|N|N|N|N| +2425985|AAAAAAAABIEAFCAA|1930-01-08|360|1567|121|1930|3|1|8|1|1930|121|1567|Wednesday|1930Q1|N|N|N|2425978|2425977|2425620|2425893|N|N|N|N|N| +2425986|AAAAAAAACIEAFCAA|1930-01-09|360|1567|121|1930|4|1|9|1|1930|121|1567|Thursday|1930Q1|N|N|N|2425978|2425977|2425621|2425894|N|N|N|N|N| +2425987|AAAAAAAADIEAFCAA|1930-01-10|360|1567|121|1930|5|1|10|1|1930|121|1567|Friday|1930Q1|N|Y|N|2425978|2425977|2425622|2425895|N|N|N|N|N| +2425988|AAAAAAAAEIEAFCAA|1930-01-11|360|1567|121|1930|6|1|11|1|1930|121|1567|Saturday|1930Q1|N|Y|N|2425978|2425977|2425623|2425896|N|N|N|N|N| +2425989|AAAAAAAAFIEAFCAA|1930-01-12|360|1567|121|1930|0|1|12|1|1930|121|1567|Sunday|1930Q1|N|N|N|2425978|2425977|2425624|2425897|N|N|N|N|N| +2425990|AAAAAAAAGIEAFCAA|1930-01-13|360|1567|121|1930|1|1|13|1|1930|121|1567|Monday|1930Q1|N|N|N|2425978|2425977|2425625|2425898|N|N|N|N|N| +2425991|AAAAAAAAHIEAFCAA|1930-01-14|360|1568|121|1930|2|1|14|1|1930|121|1568|Tuesday|1930Q1|N|N|N|2425978|2425977|2425626|2425899|N|N|N|N|N| +2425992|AAAAAAAAIIEAFCAA|1930-01-15|360|1568|121|1930|3|1|15|1|1930|121|1568|Wednesday|1930Q1|N|N|N|2425978|2425977|2425627|2425900|N|N|N|N|N| +2425993|AAAAAAAAJIEAFCAA|1930-01-16|360|1568|121|1930|4|1|16|1|1930|121|1568|Thursday|1930Q1|N|N|N|2425978|2425977|2425628|2425901|N|N|N|N|N| +2425994|AAAAAAAAKIEAFCAA|1930-01-17|360|1568|121|1930|5|1|17|1|1930|121|1568|Friday|1930Q1|N|Y|N|2425978|2425977|2425629|2425902|N|N|N|N|N| +2425995|AAAAAAAALIEAFCAA|1930-01-18|360|1568|121|1930|6|1|18|1|1930|121|1568|Saturday|1930Q1|N|Y|N|2425978|2425977|2425630|2425903|N|N|N|N|N| +2425996|AAAAAAAAMIEAFCAA|1930-01-19|360|1568|121|1930|0|1|19|1|1930|121|1568|Sunday|1930Q1|N|N|N|2425978|2425977|2425631|2425904|N|N|N|N|N| +2425997|AAAAAAAANIEAFCAA|1930-01-20|360|1568|121|1930|1|1|20|1|1930|121|1568|Monday|1930Q1|N|N|N|2425978|2425977|2425632|2425905|N|N|N|N|N| +2425998|AAAAAAAAOIEAFCAA|1930-01-21|360|1569|121|1930|2|1|21|1|1930|121|1569|Tuesday|1930Q1|N|N|N|2425978|2425977|2425633|2425906|N|N|N|N|N| +2425999|AAAAAAAAPIEAFCAA|1930-01-22|360|1569|121|1930|3|1|22|1|1930|121|1569|Wednesday|1930Q1|N|N|N|2425978|2425977|2425634|2425907|N|N|N|N|N| +2426000|AAAAAAAAAJEAFCAA|1930-01-23|360|1569|121|1930|4|1|23|1|1930|121|1569|Thursday|1930Q1|N|N|N|2425978|2425977|2425635|2425908|N|N|N|N|N| +2426001|AAAAAAAABJEAFCAA|1930-01-24|360|1569|121|1930|5|1|24|1|1930|121|1569|Friday|1930Q1|N|Y|N|2425978|2425977|2425636|2425909|N|N|N|N|N| +2426002|AAAAAAAACJEAFCAA|1930-01-25|360|1569|121|1930|6|1|25|1|1930|121|1569|Saturday|1930Q1|N|Y|N|2425978|2425977|2425637|2425910|N|N|N|N|N| +2426003|AAAAAAAADJEAFCAA|1930-01-26|360|1569|121|1930|0|1|26|1|1930|121|1569|Sunday|1930Q1|N|N|N|2425978|2425977|2425638|2425911|N|N|N|N|N| +2426004|AAAAAAAAEJEAFCAA|1930-01-27|360|1569|121|1930|1|1|27|1|1930|121|1569|Monday|1930Q1|N|N|N|2425978|2425977|2425639|2425912|N|N|N|N|N| +2426005|AAAAAAAAFJEAFCAA|1930-01-28|360|1570|121|1930|2|1|28|1|1930|121|1570|Tuesday|1930Q1|N|N|N|2425978|2425977|2425640|2425913|N|N|N|N|N| +2426006|AAAAAAAAGJEAFCAA|1930-01-29|360|1570|121|1930|3|1|29|1|1930|121|1570|Wednesday|1930Q1|N|N|N|2425978|2425977|2425641|2425914|N|N|N|N|N| +2426007|AAAAAAAAHJEAFCAA|1930-01-30|360|1570|121|1930|4|1|30|1|1930|121|1570|Thursday|1930Q1|N|N|N|2425978|2425977|2425642|2425915|N|N|N|N|N| +2426008|AAAAAAAAIJEAFCAA|1930-01-31|360|1570|121|1930|5|1|31|1|1930|121|1570|Friday|1930Q1|N|Y|N|2425978|2425977|2425643|2425916|N|N|N|N|N| +2426009|AAAAAAAAJJEAFCAA|1930-02-01|361|1570|121|1930|6|2|1|1|1930|121|1570|Saturday|1930Q1|N|Y|N|2426009|2426039|2425644|2425917|N|N|N|N|N| +2426010|AAAAAAAAKJEAFCAA|1930-02-02|361|1570|121|1930|0|2|2|1|1930|121|1570|Sunday|1930Q1|N|N|N|2426009|2426039|2425645|2425918|N|N|N|N|N| +2426011|AAAAAAAALJEAFCAA|1930-02-03|361|1570|121|1930|1|2|3|1|1930|121|1570|Monday|1930Q1|N|N|N|2426009|2426039|2425646|2425919|N|N|N|N|N| +2426012|AAAAAAAAMJEAFCAA|1930-02-04|361|1571|121|1930|2|2|4|1|1930|121|1571|Tuesday|1930Q1|N|N|N|2426009|2426039|2425647|2425920|N|N|N|N|N| +2426013|AAAAAAAANJEAFCAA|1930-02-05|361|1571|121|1930|3|2|5|1|1930|121|1571|Wednesday|1930Q1|N|N|N|2426009|2426039|2425648|2425921|N|N|N|N|N| +2426014|AAAAAAAAOJEAFCAA|1930-02-06|361|1571|121|1930|4|2|6|1|1930|121|1571|Thursday|1930Q1|N|N|N|2426009|2426039|2425649|2425922|N|N|N|N|N| +2426015|AAAAAAAAPJEAFCAA|1930-02-07|361|1571|121|1930|5|2|7|1|1930|121|1571|Friday|1930Q1|N|Y|N|2426009|2426039|2425650|2425923|N|N|N|N|N| +2426016|AAAAAAAAAKEAFCAA|1930-02-08|361|1571|121|1930|6|2|8|1|1930|121|1571|Saturday|1930Q1|N|Y|N|2426009|2426039|2425651|2425924|N|N|N|N|N| +2426017|AAAAAAAABKEAFCAA|1930-02-09|361|1571|121|1930|0|2|9|1|1930|121|1571|Sunday|1930Q1|N|N|N|2426009|2426039|2425652|2425925|N|N|N|N|N| +2426018|AAAAAAAACKEAFCAA|1930-02-10|361|1571|121|1930|1|2|10|1|1930|121|1571|Monday|1930Q1|N|N|N|2426009|2426039|2425653|2425926|N|N|N|N|N| +2426019|AAAAAAAADKEAFCAA|1930-02-11|361|1572|121|1930|2|2|11|1|1930|121|1572|Tuesday|1930Q1|N|N|N|2426009|2426039|2425654|2425927|N|N|N|N|N| +2426020|AAAAAAAAEKEAFCAA|1930-02-12|361|1572|121|1930|3|2|12|1|1930|121|1572|Wednesday|1930Q1|N|N|N|2426009|2426039|2425655|2425928|N|N|N|N|N| +2426021|AAAAAAAAFKEAFCAA|1930-02-13|361|1572|121|1930|4|2|13|1|1930|121|1572|Thursday|1930Q1|N|N|N|2426009|2426039|2425656|2425929|N|N|N|N|N| +2426022|AAAAAAAAGKEAFCAA|1930-02-14|361|1572|121|1930|5|2|14|1|1930|121|1572|Friday|1930Q1|N|Y|N|2426009|2426039|2425657|2425930|N|N|N|N|N| +2426023|AAAAAAAAHKEAFCAA|1930-02-15|361|1572|121|1930|6|2|15|1|1930|121|1572|Saturday|1930Q1|N|Y|N|2426009|2426039|2425658|2425931|N|N|N|N|N| +2426024|AAAAAAAAIKEAFCAA|1930-02-16|361|1572|121|1930|0|2|16|1|1930|121|1572|Sunday|1930Q1|N|N|N|2426009|2426039|2425659|2425932|N|N|N|N|N| +2426025|AAAAAAAAJKEAFCAA|1930-02-17|361|1572|121|1930|1|2|17|1|1930|121|1572|Monday|1930Q1|N|N|N|2426009|2426039|2425660|2425933|N|N|N|N|N| +2426026|AAAAAAAAKKEAFCAA|1930-02-18|361|1573|121|1930|2|2|18|1|1930|121|1573|Tuesday|1930Q1|N|N|N|2426009|2426039|2425661|2425934|N|N|N|N|N| +2426027|AAAAAAAALKEAFCAA|1930-02-19|361|1573|121|1930|3|2|19|1|1930|121|1573|Wednesday|1930Q1|N|N|N|2426009|2426039|2425662|2425935|N|N|N|N|N| +2426028|AAAAAAAAMKEAFCAA|1930-02-20|361|1573|121|1930|4|2|20|1|1930|121|1573|Thursday|1930Q1|N|N|N|2426009|2426039|2425663|2425936|N|N|N|N|N| +2426029|AAAAAAAANKEAFCAA|1930-02-21|361|1573|121|1930|5|2|21|1|1930|121|1573|Friday|1930Q1|N|Y|N|2426009|2426039|2425664|2425937|N|N|N|N|N| +2426030|AAAAAAAAOKEAFCAA|1930-02-22|361|1573|121|1930|6|2|22|1|1930|121|1573|Saturday|1930Q1|N|Y|N|2426009|2426039|2425665|2425938|N|N|N|N|N| +2426031|AAAAAAAAPKEAFCAA|1930-02-23|361|1573|121|1930|0|2|23|1|1930|121|1573|Sunday|1930Q1|N|N|N|2426009|2426039|2425666|2425939|N|N|N|N|N| +2426032|AAAAAAAAALEAFCAA|1930-02-24|361|1573|121|1930|1|2|24|1|1930|121|1573|Monday|1930Q1|N|N|N|2426009|2426039|2425667|2425940|N|N|N|N|N| +2426033|AAAAAAAABLEAFCAA|1930-02-25|361|1574|121|1930|2|2|25|1|1930|121|1574|Tuesday|1930Q1|N|N|N|2426009|2426039|2425668|2425941|N|N|N|N|N| +2426034|AAAAAAAACLEAFCAA|1930-02-26|361|1574|121|1930|3|2|26|1|1930|121|1574|Wednesday|1930Q1|N|N|N|2426009|2426039|2425669|2425942|N|N|N|N|N| +2426035|AAAAAAAADLEAFCAA|1930-02-27|361|1574|121|1930|4|2|27|1|1930|121|1574|Thursday|1930Q1|N|N|N|2426009|2426039|2425670|2425943|N|N|N|N|N| +2426036|AAAAAAAAELEAFCAA|1930-02-28|361|1574|121|1930|5|2|28|1|1930|121|1574|Friday|1930Q1|N|Y|N|2426009|2426039|2425671|2425944|N|N|N|N|N| +2426037|AAAAAAAAFLEAFCAA|1930-03-01|362|1574|122|1930|6|3|1|1|1930|122|1574|Saturday|1930Q1|N|Y|N|2426037|2426095|2425672|2425945|N|N|N|N|N| +2426038|AAAAAAAAGLEAFCAA|1930-03-02|362|1574|122|1930|0|3|2|1|1930|122|1574|Sunday|1930Q1|N|N|N|2426037|2426095|2425673|2425946|N|N|N|N|N| +2426039|AAAAAAAAHLEAFCAA|1930-03-03|362|1574|122|1930|1|3|3|1|1930|122|1574|Monday|1930Q1|N|N|N|2426037|2426095|2425674|2425947|N|N|N|N|N| +2426040|AAAAAAAAILEAFCAA|1930-03-04|362|1575|122|1930|2|3|4|1|1930|122|1575|Tuesday|1930Q1|N|N|N|2426037|2426095|2425675|2425948|N|N|N|N|N| +2426041|AAAAAAAAJLEAFCAA|1930-03-05|362|1575|122|1930|3|3|5|1|1930|122|1575|Wednesday|1930Q1|N|N|N|2426037|2426095|2425676|2425949|N|N|N|N|N| +2426042|AAAAAAAAKLEAFCAA|1930-03-06|362|1575|122|1930|4|3|6|1|1930|122|1575|Thursday|1930Q1|N|N|N|2426037|2426095|2425677|2425950|N|N|N|N|N| +2426043|AAAAAAAALLEAFCAA|1930-03-07|362|1575|122|1930|5|3|7|1|1930|122|1575|Friday|1930Q1|N|Y|N|2426037|2426095|2425678|2425951|N|N|N|N|N| +2426044|AAAAAAAAMLEAFCAA|1930-03-08|362|1575|122|1930|6|3|8|1|1930|122|1575|Saturday|1930Q1|N|Y|N|2426037|2426095|2425679|2425952|N|N|N|N|N| +2426045|AAAAAAAANLEAFCAA|1930-03-09|362|1575|122|1930|0|3|9|1|1930|122|1575|Sunday|1930Q1|N|N|N|2426037|2426095|2425680|2425953|N|N|N|N|N| +2426046|AAAAAAAAOLEAFCAA|1930-03-10|362|1575|122|1930|1|3|10|1|1930|122|1575|Monday|1930Q1|N|N|N|2426037|2426095|2425681|2425954|N|N|N|N|N| +2426047|AAAAAAAAPLEAFCAA|1930-03-11|362|1576|122|1930|2|3|11|1|1930|122|1576|Tuesday|1930Q1|N|N|N|2426037|2426095|2425682|2425955|N|N|N|N|N| +2426048|AAAAAAAAAMEAFCAA|1930-03-12|362|1576|122|1930|3|3|12|1|1930|122|1576|Wednesday|1930Q1|N|N|N|2426037|2426095|2425683|2425956|N|N|N|N|N| +2426049|AAAAAAAABMEAFCAA|1930-03-13|362|1576|122|1930|4|3|13|1|1930|122|1576|Thursday|1930Q1|N|N|N|2426037|2426095|2425684|2425957|N|N|N|N|N| +2426050|AAAAAAAACMEAFCAA|1930-03-14|362|1576|122|1930|5|3|14|1|1930|122|1576|Friday|1930Q1|N|Y|N|2426037|2426095|2425685|2425958|N|N|N|N|N| +2426051|AAAAAAAADMEAFCAA|1930-03-15|362|1576|122|1930|6|3|15|1|1930|122|1576|Saturday|1930Q1|N|Y|N|2426037|2426095|2425686|2425959|N|N|N|N|N| +2426052|AAAAAAAAEMEAFCAA|1930-03-16|362|1576|122|1930|0|3|16|1|1930|122|1576|Sunday|1930Q1|N|N|N|2426037|2426095|2425687|2425960|N|N|N|N|N| +2426053|AAAAAAAAFMEAFCAA|1930-03-17|362|1576|122|1930|1|3|17|1|1930|122|1576|Monday|1930Q1|N|N|N|2426037|2426095|2425688|2425961|N|N|N|N|N| +2426054|AAAAAAAAGMEAFCAA|1930-03-18|362|1577|122|1930|2|3|18|1|1930|122|1577|Tuesday|1930Q1|N|N|N|2426037|2426095|2425689|2425962|N|N|N|N|N| +2426055|AAAAAAAAHMEAFCAA|1930-03-19|362|1577|122|1930|3|3|19|1|1930|122|1577|Wednesday|1930Q1|N|N|N|2426037|2426095|2425690|2425963|N|N|N|N|N| +2426056|AAAAAAAAIMEAFCAA|1930-03-20|362|1577|122|1930|4|3|20|1|1930|122|1577|Thursday|1930Q1|N|N|N|2426037|2426095|2425691|2425964|N|N|N|N|N| +2426057|AAAAAAAAJMEAFCAA|1930-03-21|362|1577|122|1930|5|3|21|1|1930|122|1577|Friday|1930Q1|N|Y|N|2426037|2426095|2425692|2425965|N|N|N|N|N| +2426058|AAAAAAAAKMEAFCAA|1930-03-22|362|1577|122|1930|6|3|22|1|1930|122|1577|Saturday|1930Q1|N|Y|N|2426037|2426095|2425693|2425966|N|N|N|N|N| +2426059|AAAAAAAALMEAFCAA|1930-03-23|362|1577|122|1930|0|3|23|1|1930|122|1577|Sunday|1930Q1|N|N|N|2426037|2426095|2425694|2425967|N|N|N|N|N| +2426060|AAAAAAAAMMEAFCAA|1930-03-24|362|1577|122|1930|1|3|24|1|1930|122|1577|Monday|1930Q1|N|N|N|2426037|2426095|2425695|2425968|N|N|N|N|N| +2426061|AAAAAAAANMEAFCAA|1930-03-25|362|1578|122|1930|2|3|25|1|1930|122|1578|Tuesday|1930Q1|N|N|N|2426037|2426095|2425696|2425969|N|N|N|N|N| +2426062|AAAAAAAAOMEAFCAA|1930-03-26|362|1578|122|1930|3|3|26|1|1930|122|1578|Wednesday|1930Q1|N|N|N|2426037|2426095|2425697|2425970|N|N|N|N|N| +2426063|AAAAAAAAPMEAFCAA|1930-03-27|362|1578|122|1930|4|3|27|1|1930|122|1578|Thursday|1930Q1|N|N|N|2426037|2426095|2425698|2425971|N|N|N|N|N| +2426064|AAAAAAAAANEAFCAA|1930-03-28|362|1578|122|1930|5|3|28|1|1930|122|1578|Friday|1930Q1|N|Y|N|2426037|2426095|2425699|2425972|N|N|N|N|N| +2426065|AAAAAAAABNEAFCAA|1930-03-29|362|1578|122|1930|6|3|29|1|1930|122|1578|Saturday|1930Q1|N|Y|N|2426037|2426095|2425700|2425973|N|N|N|N|N| +2426066|AAAAAAAACNEAFCAA|1930-03-30|362|1578|122|1930|0|3|30|1|1930|122|1578|Sunday|1930Q1|N|N|N|2426037|2426095|2425701|2425974|N|N|N|N|N| +2426067|AAAAAAAADNEAFCAA|1930-03-31|362|1578|122|1930|1|3|31|1|1930|122|1578|Monday|1930Q1|N|N|N|2426037|2426095|2425702|2425975|N|N|N|N|N| +2426068|AAAAAAAAENEAFCAA|1930-04-01|363|1579|122|1930|2|4|1|1|1930|122|1579|Tuesday|1930Q1|N|N|N|2426068|2426157|2425703|2425978|N|N|N|N|N| +2426069|AAAAAAAAFNEAFCAA|1930-04-02|363|1579|122|1930|3|4|2|2|1930|122|1579|Wednesday|1930Q2|N|N|N|2426068|2426157|2425704|2425979|N|N|N|N|N| +2426070|AAAAAAAAGNEAFCAA|1930-04-03|363|1579|122|1930|4|4|3|2|1930|122|1579|Thursday|1930Q2|N|N|N|2426068|2426157|2425705|2425980|N|N|N|N|N| +2426071|AAAAAAAAHNEAFCAA|1930-04-04|363|1579|122|1930|5|4|4|2|1930|122|1579|Friday|1930Q2|N|Y|N|2426068|2426157|2425706|2425981|N|N|N|N|N| +2426072|AAAAAAAAINEAFCAA|1930-04-05|363|1579|122|1930|6|4|5|2|1930|122|1579|Saturday|1930Q2|N|Y|N|2426068|2426157|2425707|2425982|N|N|N|N|N| +2426073|AAAAAAAAJNEAFCAA|1930-04-06|363|1579|122|1930|0|4|6|2|1930|122|1579|Sunday|1930Q2|N|N|N|2426068|2426157|2425708|2425983|N|N|N|N|N| +2426074|AAAAAAAAKNEAFCAA|1930-04-07|363|1579|122|1930|1|4|7|2|1930|122|1579|Monday|1930Q2|N|N|N|2426068|2426157|2425709|2425984|N|N|N|N|N| +2426075|AAAAAAAALNEAFCAA|1930-04-08|363|1580|122|1930|2|4|8|2|1930|122|1580|Tuesday|1930Q2|N|N|N|2426068|2426157|2425710|2425985|N|N|N|N|N| +2426076|AAAAAAAAMNEAFCAA|1930-04-09|363|1580|122|1930|3|4|9|2|1930|122|1580|Wednesday|1930Q2|N|N|N|2426068|2426157|2425711|2425986|N|N|N|N|N| +2426077|AAAAAAAANNEAFCAA|1930-04-10|363|1580|122|1930|4|4|10|2|1930|122|1580|Thursday|1930Q2|N|N|N|2426068|2426157|2425712|2425987|N|N|N|N|N| +2426078|AAAAAAAAONEAFCAA|1930-04-11|363|1580|122|1930|5|4|11|2|1930|122|1580|Friday|1930Q2|N|Y|N|2426068|2426157|2425713|2425988|N|N|N|N|N| +2426079|AAAAAAAAPNEAFCAA|1930-04-12|363|1580|122|1930|6|4|12|2|1930|122|1580|Saturday|1930Q2|N|Y|N|2426068|2426157|2425714|2425989|N|N|N|N|N| +2426080|AAAAAAAAAOEAFCAA|1930-04-13|363|1580|122|1930|0|4|13|2|1930|122|1580|Sunday|1930Q2|N|N|N|2426068|2426157|2425715|2425990|N|N|N|N|N| +2426081|AAAAAAAABOEAFCAA|1930-04-14|363|1580|122|1930|1|4|14|2|1930|122|1580|Monday|1930Q2|N|N|N|2426068|2426157|2425716|2425991|N|N|N|N|N| +2426082|AAAAAAAACOEAFCAA|1930-04-15|363|1581|122|1930|2|4|15|2|1930|122|1581|Tuesday|1930Q2|N|N|N|2426068|2426157|2425717|2425992|N|N|N|N|N| +2426083|AAAAAAAADOEAFCAA|1930-04-16|363|1581|122|1930|3|4|16|2|1930|122|1581|Wednesday|1930Q2|N|N|N|2426068|2426157|2425718|2425993|N|N|N|N|N| +2426084|AAAAAAAAEOEAFCAA|1930-04-17|363|1581|122|1930|4|4|17|2|1930|122|1581|Thursday|1930Q2|N|N|N|2426068|2426157|2425719|2425994|N|N|N|N|N| +2426085|AAAAAAAAFOEAFCAA|1930-04-18|363|1581|122|1930|5|4|18|2|1930|122|1581|Friday|1930Q2|N|Y|N|2426068|2426157|2425720|2425995|N|N|N|N|N| +2426086|AAAAAAAAGOEAFCAA|1930-04-19|363|1581|122|1930|6|4|19|2|1930|122|1581|Saturday|1930Q2|N|Y|N|2426068|2426157|2425721|2425996|N|N|N|N|N| +2426087|AAAAAAAAHOEAFCAA|1930-04-20|363|1581|122|1930|0|4|20|2|1930|122|1581|Sunday|1930Q2|N|N|N|2426068|2426157|2425722|2425997|N|N|N|N|N| +2426088|AAAAAAAAIOEAFCAA|1930-04-21|363|1581|122|1930|1|4|21|2|1930|122|1581|Monday|1930Q2|N|N|N|2426068|2426157|2425723|2425998|N|N|N|N|N| +2426089|AAAAAAAAJOEAFCAA|1930-04-22|363|1582|122|1930|2|4|22|2|1930|122|1582|Tuesday|1930Q2|N|N|N|2426068|2426157|2425724|2425999|N|N|N|N|N| +2426090|AAAAAAAAKOEAFCAA|1930-04-23|363|1582|122|1930|3|4|23|2|1930|122|1582|Wednesday|1930Q2|N|N|N|2426068|2426157|2425725|2426000|N|N|N|N|N| +2426091|AAAAAAAALOEAFCAA|1930-04-24|363|1582|122|1930|4|4|24|2|1930|122|1582|Thursday|1930Q2|N|N|N|2426068|2426157|2425726|2426001|N|N|N|N|N| +2426092|AAAAAAAAMOEAFCAA|1930-04-25|363|1582|122|1930|5|4|25|2|1930|122|1582|Friday|1930Q2|N|Y|N|2426068|2426157|2425727|2426002|N|N|N|N|N| +2426093|AAAAAAAANOEAFCAA|1930-04-26|363|1582|122|1930|6|4|26|2|1930|122|1582|Saturday|1930Q2|N|Y|N|2426068|2426157|2425728|2426003|N|N|N|N|N| +2426094|AAAAAAAAOOEAFCAA|1930-04-27|363|1582|122|1930|0|4|27|2|1930|122|1582|Sunday|1930Q2|N|N|N|2426068|2426157|2425729|2426004|N|N|N|N|N| +2426095|AAAAAAAAPOEAFCAA|1930-04-28|363|1582|122|1930|1|4|28|2|1930|122|1582|Monday|1930Q2|N|N|N|2426068|2426157|2425730|2426005|N|N|N|N|N| +2426096|AAAAAAAAAPEAFCAA|1930-04-29|363|1583|122|1930|2|4|29|2|1930|122|1583|Tuesday|1930Q2|N|N|N|2426068|2426157|2425731|2426006|N|N|N|N|N| +2426097|AAAAAAAABPEAFCAA|1930-04-30|363|1583|122|1930|3|4|30|2|1930|122|1583|Wednesday|1930Q2|N|N|N|2426068|2426157|2425732|2426007|N|N|N|N|N| +2426098|AAAAAAAACPEAFCAA|1930-05-01|364|1583|122|1930|4|5|1|2|1930|122|1583|Thursday|1930Q2|N|N|N|2426098|2426217|2425733|2426008|N|N|N|N|N| +2426099|AAAAAAAADPEAFCAA|1930-05-02|364|1583|122|1930|5|5|2|2|1930|122|1583|Friday|1930Q2|N|Y|N|2426098|2426217|2425734|2426009|N|N|N|N|N| +2426100|AAAAAAAAEPEAFCAA|1930-05-03|364|1583|122|1930|6|5|3|2|1930|122|1583|Saturday|1930Q2|N|Y|N|2426098|2426217|2425735|2426010|N|N|N|N|N| +2426101|AAAAAAAAFPEAFCAA|1930-05-04|364|1583|122|1930|0|5|4|2|1930|122|1583|Sunday|1930Q2|N|N|N|2426098|2426217|2425736|2426011|N|N|N|N|N| +2426102|AAAAAAAAGPEAFCAA|1930-05-05|364|1583|122|1930|1|5|5|2|1930|122|1583|Monday|1930Q2|N|N|N|2426098|2426217|2425737|2426012|N|N|N|N|N| +2426103|AAAAAAAAHPEAFCAA|1930-05-06|364|1584|122|1930|2|5|6|2|1930|122|1584|Tuesday|1930Q2|N|N|N|2426098|2426217|2425738|2426013|N|N|N|N|N| +2426104|AAAAAAAAIPEAFCAA|1930-05-07|364|1584|122|1930|3|5|7|2|1930|122|1584|Wednesday|1930Q2|N|N|N|2426098|2426217|2425739|2426014|N|N|N|N|N| +2426105|AAAAAAAAJPEAFCAA|1930-05-08|364|1584|122|1930|4|5|8|2|1930|122|1584|Thursday|1930Q2|N|N|N|2426098|2426217|2425740|2426015|N|N|N|N|N| +2426106|AAAAAAAAKPEAFCAA|1930-05-09|364|1584|122|1930|5|5|9|2|1930|122|1584|Friday|1930Q2|N|Y|N|2426098|2426217|2425741|2426016|N|N|N|N|N| +2426107|AAAAAAAALPEAFCAA|1930-05-10|364|1584|122|1930|6|5|10|2|1930|122|1584|Saturday|1930Q2|N|Y|N|2426098|2426217|2425742|2426017|N|N|N|N|N| +2426108|AAAAAAAAMPEAFCAA|1930-05-11|364|1584|122|1930|0|5|11|2|1930|122|1584|Sunday|1930Q2|N|N|N|2426098|2426217|2425743|2426018|N|N|N|N|N| +2426109|AAAAAAAANPEAFCAA|1930-05-12|364|1584|122|1930|1|5|12|2|1930|122|1584|Monday|1930Q2|N|N|N|2426098|2426217|2425744|2426019|N|N|N|N|N| +2426110|AAAAAAAAOPEAFCAA|1930-05-13|364|1585|122|1930|2|5|13|2|1930|122|1585|Tuesday|1930Q2|N|N|N|2426098|2426217|2425745|2426020|N|N|N|N|N| +2426111|AAAAAAAAPPEAFCAA|1930-05-14|364|1585|122|1930|3|5|14|2|1930|122|1585|Wednesday|1930Q2|N|N|N|2426098|2426217|2425746|2426021|N|N|N|N|N| +2426112|AAAAAAAAAAFAFCAA|1930-05-15|364|1585|122|1930|4|5|15|2|1930|122|1585|Thursday|1930Q2|N|N|N|2426098|2426217|2425747|2426022|N|N|N|N|N| +2426113|AAAAAAAABAFAFCAA|1930-05-16|364|1585|122|1930|5|5|16|2|1930|122|1585|Friday|1930Q2|N|Y|N|2426098|2426217|2425748|2426023|N|N|N|N|N| +2426114|AAAAAAAACAFAFCAA|1930-05-17|364|1585|122|1930|6|5|17|2|1930|122|1585|Saturday|1930Q2|N|Y|N|2426098|2426217|2425749|2426024|N|N|N|N|N| +2426115|AAAAAAAADAFAFCAA|1930-05-18|364|1585|122|1930|0|5|18|2|1930|122|1585|Sunday|1930Q2|N|N|N|2426098|2426217|2425750|2426025|N|N|N|N|N| +2426116|AAAAAAAAEAFAFCAA|1930-05-19|364|1585|122|1930|1|5|19|2|1930|122|1585|Monday|1930Q2|N|N|N|2426098|2426217|2425751|2426026|N|N|N|N|N| +2426117|AAAAAAAAFAFAFCAA|1930-05-20|364|1586|122|1930|2|5|20|2|1930|122|1586|Tuesday|1930Q2|N|N|N|2426098|2426217|2425752|2426027|N|N|N|N|N| +2426118|AAAAAAAAGAFAFCAA|1930-05-21|364|1586|122|1930|3|5|21|2|1930|122|1586|Wednesday|1930Q2|N|N|N|2426098|2426217|2425753|2426028|N|N|N|N|N| +2426119|AAAAAAAAHAFAFCAA|1930-05-22|364|1586|122|1930|4|5|22|2|1930|122|1586|Thursday|1930Q2|N|N|N|2426098|2426217|2425754|2426029|N|N|N|N|N| +2426120|AAAAAAAAIAFAFCAA|1930-05-23|364|1586|122|1930|5|5|23|2|1930|122|1586|Friday|1930Q2|N|Y|N|2426098|2426217|2425755|2426030|N|N|N|N|N| +2426121|AAAAAAAAJAFAFCAA|1930-05-24|364|1586|122|1930|6|5|24|2|1930|122|1586|Saturday|1930Q2|N|Y|N|2426098|2426217|2425756|2426031|N|N|N|N|N| +2426122|AAAAAAAAKAFAFCAA|1930-05-25|364|1586|122|1930|0|5|25|2|1930|122|1586|Sunday|1930Q2|N|N|N|2426098|2426217|2425757|2426032|N|N|N|N|N| +2426123|AAAAAAAALAFAFCAA|1930-05-26|364|1586|122|1930|1|5|26|2|1930|122|1586|Monday|1930Q2|N|N|N|2426098|2426217|2425758|2426033|N|N|N|N|N| +2426124|AAAAAAAAMAFAFCAA|1930-05-27|364|1587|122|1930|2|5|27|2|1930|122|1587|Tuesday|1930Q2|N|N|N|2426098|2426217|2425759|2426034|N|N|N|N|N| +2426125|AAAAAAAANAFAFCAA|1930-05-28|364|1587|122|1930|3|5|28|2|1930|122|1587|Wednesday|1930Q2|N|N|N|2426098|2426217|2425760|2426035|N|N|N|N|N| +2426126|AAAAAAAAOAFAFCAA|1930-05-29|364|1587|122|1930|4|5|29|2|1930|122|1587|Thursday|1930Q2|N|N|N|2426098|2426217|2425761|2426036|N|N|N|N|N| +2426127|AAAAAAAAPAFAFCAA|1930-05-30|364|1587|122|1930|5|5|30|2|1930|122|1587|Friday|1930Q2|N|Y|N|2426098|2426217|2425762|2426037|N|N|N|N|N| +2426128|AAAAAAAAABFAFCAA|1930-05-31|364|1587|122|1930|6|5|31|2|1930|122|1587|Saturday|1930Q2|N|Y|N|2426098|2426217|2425763|2426038|N|N|N|N|N| +2426129|AAAAAAAABBFAFCAA|1930-06-01|365|1587|123|1930|0|6|1|2|1930|123|1587|Sunday|1930Q2|N|N|N|2426129|2426279|2425764|2426039|N|N|N|N|N| +2426130|AAAAAAAACBFAFCAA|1930-06-02|365|1587|123|1930|1|6|2|2|1930|123|1587|Monday|1930Q2|N|N|N|2426129|2426279|2425765|2426040|N|N|N|N|N| +2426131|AAAAAAAADBFAFCAA|1930-06-03|365|1588|123|1930|2|6|3|2|1930|123|1588|Tuesday|1930Q2|N|N|N|2426129|2426279|2425766|2426041|N|N|N|N|N| +2426132|AAAAAAAAEBFAFCAA|1930-06-04|365|1588|123|1930|3|6|4|2|1930|123|1588|Wednesday|1930Q2|N|N|N|2426129|2426279|2425767|2426042|N|N|N|N|N| +2426133|AAAAAAAAFBFAFCAA|1930-06-05|365|1588|123|1930|4|6|5|2|1930|123|1588|Thursday|1930Q2|N|N|N|2426129|2426279|2425768|2426043|N|N|N|N|N| +2426134|AAAAAAAAGBFAFCAA|1930-06-06|365|1588|123|1930|5|6|6|2|1930|123|1588|Friday|1930Q2|N|Y|N|2426129|2426279|2425769|2426044|N|N|N|N|N| +2426135|AAAAAAAAHBFAFCAA|1930-06-07|365|1588|123|1930|6|6|7|2|1930|123|1588|Saturday|1930Q2|N|Y|N|2426129|2426279|2425770|2426045|N|N|N|N|N| +2426136|AAAAAAAAIBFAFCAA|1930-06-08|365|1588|123|1930|0|6|8|2|1930|123|1588|Sunday|1930Q2|N|N|N|2426129|2426279|2425771|2426046|N|N|N|N|N| +2426137|AAAAAAAAJBFAFCAA|1930-06-09|365|1588|123|1930|1|6|9|2|1930|123|1588|Monday|1930Q2|N|N|N|2426129|2426279|2425772|2426047|N|N|N|N|N| +2426138|AAAAAAAAKBFAFCAA|1930-06-10|365|1589|123|1930|2|6|10|2|1930|123|1589|Tuesday|1930Q2|N|N|N|2426129|2426279|2425773|2426048|N|N|N|N|N| +2426139|AAAAAAAALBFAFCAA|1930-06-11|365|1589|123|1930|3|6|11|2|1930|123|1589|Wednesday|1930Q2|N|N|N|2426129|2426279|2425774|2426049|N|N|N|N|N| +2426140|AAAAAAAAMBFAFCAA|1930-06-12|365|1589|123|1930|4|6|12|2|1930|123|1589|Thursday|1930Q2|N|N|N|2426129|2426279|2425775|2426050|N|N|N|N|N| +2426141|AAAAAAAANBFAFCAA|1930-06-13|365|1589|123|1930|5|6|13|2|1930|123|1589|Friday|1930Q2|N|Y|N|2426129|2426279|2425776|2426051|N|N|N|N|N| +2426142|AAAAAAAAOBFAFCAA|1930-06-14|365|1589|123|1930|6|6|14|2|1930|123|1589|Saturday|1930Q2|N|Y|N|2426129|2426279|2425777|2426052|N|N|N|N|N| +2426143|AAAAAAAAPBFAFCAA|1930-06-15|365|1589|123|1930|0|6|15|2|1930|123|1589|Sunday|1930Q2|N|N|N|2426129|2426279|2425778|2426053|N|N|N|N|N| +2426144|AAAAAAAAACFAFCAA|1930-06-16|365|1589|123|1930|1|6|16|2|1930|123|1589|Monday|1930Q2|N|N|N|2426129|2426279|2425779|2426054|N|N|N|N|N| +2426145|AAAAAAAABCFAFCAA|1930-06-17|365|1590|123|1930|2|6|17|2|1930|123|1590|Tuesday|1930Q2|N|N|N|2426129|2426279|2425780|2426055|N|N|N|N|N| +2426146|AAAAAAAACCFAFCAA|1930-06-18|365|1590|123|1930|3|6|18|2|1930|123|1590|Wednesday|1930Q2|N|N|N|2426129|2426279|2425781|2426056|N|N|N|N|N| +2426147|AAAAAAAADCFAFCAA|1930-06-19|365|1590|123|1930|4|6|19|2|1930|123|1590|Thursday|1930Q2|N|N|N|2426129|2426279|2425782|2426057|N|N|N|N|N| +2426148|AAAAAAAAECFAFCAA|1930-06-20|365|1590|123|1930|5|6|20|2|1930|123|1590|Friday|1930Q2|N|Y|N|2426129|2426279|2425783|2426058|N|N|N|N|N| +2426149|AAAAAAAAFCFAFCAA|1930-06-21|365|1590|123|1930|6|6|21|2|1930|123|1590|Saturday|1930Q2|N|Y|N|2426129|2426279|2425784|2426059|N|N|N|N|N| +2426150|AAAAAAAAGCFAFCAA|1930-06-22|365|1590|123|1930|0|6|22|2|1930|123|1590|Sunday|1930Q2|N|N|N|2426129|2426279|2425785|2426060|N|N|N|N|N| +2426151|AAAAAAAAHCFAFCAA|1930-06-23|365|1590|123|1930|1|6|23|2|1930|123|1590|Monday|1930Q2|N|N|N|2426129|2426279|2425786|2426061|N|N|N|N|N| +2426152|AAAAAAAAICFAFCAA|1930-06-24|365|1591|123|1930|2|6|24|2|1930|123|1591|Tuesday|1930Q2|N|N|N|2426129|2426279|2425787|2426062|N|N|N|N|N| +2426153|AAAAAAAAJCFAFCAA|1930-06-25|365|1591|123|1930|3|6|25|2|1930|123|1591|Wednesday|1930Q2|N|N|N|2426129|2426279|2425788|2426063|N|N|N|N|N| +2426154|AAAAAAAAKCFAFCAA|1930-06-26|365|1591|123|1930|4|6|26|2|1930|123|1591|Thursday|1930Q2|N|N|N|2426129|2426279|2425789|2426064|N|N|N|N|N| +2426155|AAAAAAAALCFAFCAA|1930-06-27|365|1591|123|1930|5|6|27|2|1930|123|1591|Friday|1930Q2|N|Y|N|2426129|2426279|2425790|2426065|N|N|N|N|N| +2426156|AAAAAAAAMCFAFCAA|1930-06-28|365|1591|123|1930|6|6|28|2|1930|123|1591|Saturday|1930Q2|N|Y|N|2426129|2426279|2425791|2426066|N|N|N|N|N| +2426157|AAAAAAAANCFAFCAA|1930-06-29|365|1591|123|1930|0|6|29|2|1930|123|1591|Sunday|1930Q2|N|N|N|2426129|2426279|2425792|2426067|N|N|N|N|N| +2426158|AAAAAAAAOCFAFCAA|1930-06-30|365|1591|123|1930|1|6|30|2|1930|123|1591|Monday|1930Q2|N|N|N|2426129|2426279|2425793|2426068|N|N|N|N|N| +2426159|AAAAAAAAPCFAFCAA|1930-07-01|366|1592|123|1930|2|7|1|2|1930|123|1592|Tuesday|1930Q2|N|N|N|2426159|2426339|2425794|2426068|N|N|N|N|N| +2426160|AAAAAAAAADFAFCAA|1930-07-02|366|1592|123|1930|3|7|2|3|1930|123|1592|Wednesday|1930Q3|N|N|N|2426159|2426339|2425795|2426069|N|N|N|N|N| +2426161|AAAAAAAABDFAFCAA|1930-07-03|366|1592|123|1930|4|7|3|3|1930|123|1592|Thursday|1930Q3|N|N|N|2426159|2426339|2425796|2426070|N|N|N|N|N| +2426162|AAAAAAAACDFAFCAA|1930-07-04|366|1592|123|1930|5|7|4|3|1930|123|1592|Friday|1930Q3|N|Y|N|2426159|2426339|2425797|2426071|N|N|N|N|N| +2426163|AAAAAAAADDFAFCAA|1930-07-05|366|1592|123|1930|6|7|5|3|1930|123|1592|Saturday|1930Q3|Y|Y|N|2426159|2426339|2425798|2426072|N|N|N|N|N| +2426164|AAAAAAAAEDFAFCAA|1930-07-06|366|1592|123|1930|0|7|6|3|1930|123|1592|Sunday|1930Q3|N|N|Y|2426159|2426339|2425799|2426073|N|N|N|N|N| +2426165|AAAAAAAAFDFAFCAA|1930-07-07|366|1592|123|1930|1|7|7|3|1930|123|1592|Monday|1930Q3|N|N|N|2426159|2426339|2425800|2426074|N|N|N|N|N| +2426166|AAAAAAAAGDFAFCAA|1930-07-08|366|1593|123|1930|2|7|8|3|1930|123|1593|Tuesday|1930Q3|N|N|N|2426159|2426339|2425801|2426075|N|N|N|N|N| +2426167|AAAAAAAAHDFAFCAA|1930-07-09|366|1593|123|1930|3|7|9|3|1930|123|1593|Wednesday|1930Q3|N|N|N|2426159|2426339|2425802|2426076|N|N|N|N|N| +2426168|AAAAAAAAIDFAFCAA|1930-07-10|366|1593|123|1930|4|7|10|3|1930|123|1593|Thursday|1930Q3|N|N|N|2426159|2426339|2425803|2426077|N|N|N|N|N| +2426169|AAAAAAAAJDFAFCAA|1930-07-11|366|1593|123|1930|5|7|11|3|1930|123|1593|Friday|1930Q3|N|Y|N|2426159|2426339|2425804|2426078|N|N|N|N|N| +2426170|AAAAAAAAKDFAFCAA|1930-07-12|366|1593|123|1930|6|7|12|3|1930|123|1593|Saturday|1930Q3|N|Y|N|2426159|2426339|2425805|2426079|N|N|N|N|N| +2426171|AAAAAAAALDFAFCAA|1930-07-13|366|1593|123|1930|0|7|13|3|1930|123|1593|Sunday|1930Q3|N|N|N|2426159|2426339|2425806|2426080|N|N|N|N|N| +2426172|AAAAAAAAMDFAFCAA|1930-07-14|366|1593|123|1930|1|7|14|3|1930|123|1593|Monday|1930Q3|N|N|N|2426159|2426339|2425807|2426081|N|N|N|N|N| +2426173|AAAAAAAANDFAFCAA|1930-07-15|366|1594|123|1930|2|7|15|3|1930|123|1594|Tuesday|1930Q3|N|N|N|2426159|2426339|2425808|2426082|N|N|N|N|N| +2426174|AAAAAAAAODFAFCAA|1930-07-16|366|1594|123|1930|3|7|16|3|1930|123|1594|Wednesday|1930Q3|N|N|N|2426159|2426339|2425809|2426083|N|N|N|N|N| +2426175|AAAAAAAAPDFAFCAA|1930-07-17|366|1594|123|1930|4|7|17|3|1930|123|1594|Thursday|1930Q3|N|N|N|2426159|2426339|2425810|2426084|N|N|N|N|N| +2426176|AAAAAAAAAEFAFCAA|1930-07-18|366|1594|123|1930|5|7|18|3|1930|123|1594|Friday|1930Q3|N|Y|N|2426159|2426339|2425811|2426085|N|N|N|N|N| +2426177|AAAAAAAABEFAFCAA|1930-07-19|366|1594|123|1930|6|7|19|3|1930|123|1594|Saturday|1930Q3|N|Y|N|2426159|2426339|2425812|2426086|N|N|N|N|N| +2426178|AAAAAAAACEFAFCAA|1930-07-20|366|1594|123|1930|0|7|20|3|1930|123|1594|Sunday|1930Q3|N|N|N|2426159|2426339|2425813|2426087|N|N|N|N|N| +2426179|AAAAAAAADEFAFCAA|1930-07-21|366|1594|123|1930|1|7|21|3|1930|123|1594|Monday|1930Q3|N|N|N|2426159|2426339|2425814|2426088|N|N|N|N|N| +2426180|AAAAAAAAEEFAFCAA|1930-07-22|366|1595|123|1930|2|7|22|3|1930|123|1595|Tuesday|1930Q3|N|N|N|2426159|2426339|2425815|2426089|N|N|N|N|N| +2426181|AAAAAAAAFEFAFCAA|1930-07-23|366|1595|123|1930|3|7|23|3|1930|123|1595|Wednesday|1930Q3|N|N|N|2426159|2426339|2425816|2426090|N|N|N|N|N| +2426182|AAAAAAAAGEFAFCAA|1930-07-24|366|1595|123|1930|4|7|24|3|1930|123|1595|Thursday|1930Q3|N|N|N|2426159|2426339|2425817|2426091|N|N|N|N|N| +2426183|AAAAAAAAHEFAFCAA|1930-07-25|366|1595|123|1930|5|7|25|3|1930|123|1595|Friday|1930Q3|N|Y|N|2426159|2426339|2425818|2426092|N|N|N|N|N| +2426184|AAAAAAAAIEFAFCAA|1930-07-26|366|1595|123|1930|6|7|26|3|1930|123|1595|Saturday|1930Q3|N|Y|N|2426159|2426339|2425819|2426093|N|N|N|N|N| +2426185|AAAAAAAAJEFAFCAA|1930-07-27|366|1595|123|1930|0|7|27|3|1930|123|1595|Sunday|1930Q3|N|N|N|2426159|2426339|2425820|2426094|N|N|N|N|N| +2426186|AAAAAAAAKEFAFCAA|1930-07-28|366|1595|123|1930|1|7|28|3|1930|123|1595|Monday|1930Q3|N|N|N|2426159|2426339|2425821|2426095|N|N|N|N|N| +2426187|AAAAAAAALEFAFCAA|1930-07-29|366|1596|123|1930|2|7|29|3|1930|123|1596|Tuesday|1930Q3|N|N|N|2426159|2426339|2425822|2426096|N|N|N|N|N| +2426188|AAAAAAAAMEFAFCAA|1930-07-30|366|1596|123|1930|3|7|30|3|1930|123|1596|Wednesday|1930Q3|N|N|N|2426159|2426339|2425823|2426097|N|N|N|N|N| +2426189|AAAAAAAANEFAFCAA|1930-07-31|366|1596|123|1930|4|7|31|3|1930|123|1596|Thursday|1930Q3|N|N|N|2426159|2426339|2425824|2426098|N|N|N|N|N| +2426190|AAAAAAAAOEFAFCAA|1930-08-01|367|1596|123|1930|5|8|1|3|1930|123|1596|Friday|1930Q3|N|Y|N|2426190|2426401|2425825|2426099|N|N|N|N|N| +2426191|AAAAAAAAPEFAFCAA|1930-08-02|367|1596|123|1930|6|8|2|3|1930|123|1596|Saturday|1930Q3|N|Y|N|2426190|2426401|2425826|2426100|N|N|N|N|N| +2426192|AAAAAAAAAFFAFCAA|1930-08-03|367|1596|123|1930|0|8|3|3|1930|123|1596|Sunday|1930Q3|N|N|N|2426190|2426401|2425827|2426101|N|N|N|N|N| +2426193|AAAAAAAABFFAFCAA|1930-08-04|367|1596|123|1930|1|8|4|3|1930|123|1596|Monday|1930Q3|N|N|N|2426190|2426401|2425828|2426102|N|N|N|N|N| +2426194|AAAAAAAACFFAFCAA|1930-08-05|367|1597|123|1930|2|8|5|3|1930|123|1597|Tuesday|1930Q3|N|N|N|2426190|2426401|2425829|2426103|N|N|N|N|N| +2426195|AAAAAAAADFFAFCAA|1930-08-06|367|1597|123|1930|3|8|6|3|1930|123|1597|Wednesday|1930Q3|N|N|N|2426190|2426401|2425830|2426104|N|N|N|N|N| +2426196|AAAAAAAAEFFAFCAA|1930-08-07|367|1597|123|1930|4|8|7|3|1930|123|1597|Thursday|1930Q3|N|N|N|2426190|2426401|2425831|2426105|N|N|N|N|N| +2426197|AAAAAAAAFFFAFCAA|1930-08-08|367|1597|123|1930|5|8|8|3|1930|123|1597|Friday|1930Q3|N|Y|N|2426190|2426401|2425832|2426106|N|N|N|N|N| +2426198|AAAAAAAAGFFAFCAA|1930-08-09|367|1597|123|1930|6|8|9|3|1930|123|1597|Saturday|1930Q3|N|Y|N|2426190|2426401|2425833|2426107|N|N|N|N|N| +2426199|AAAAAAAAHFFAFCAA|1930-08-10|367|1597|123|1930|0|8|10|3|1930|123|1597|Sunday|1930Q3|N|N|N|2426190|2426401|2425834|2426108|N|N|N|N|N| +2426200|AAAAAAAAIFFAFCAA|1930-08-11|367|1597|123|1930|1|8|11|3|1930|123|1597|Monday|1930Q3|N|N|N|2426190|2426401|2425835|2426109|N|N|N|N|N| +2426201|AAAAAAAAJFFAFCAA|1930-08-12|367|1598|123|1930|2|8|12|3|1930|123|1598|Tuesday|1930Q3|N|N|N|2426190|2426401|2425836|2426110|N|N|N|N|N| +2426202|AAAAAAAAKFFAFCAA|1930-08-13|367|1598|123|1930|3|8|13|3|1930|123|1598|Wednesday|1930Q3|N|N|N|2426190|2426401|2425837|2426111|N|N|N|N|N| +2426203|AAAAAAAALFFAFCAA|1930-08-14|367|1598|123|1930|4|8|14|3|1930|123|1598|Thursday|1930Q3|N|N|N|2426190|2426401|2425838|2426112|N|N|N|N|N| +2426204|AAAAAAAAMFFAFCAA|1930-08-15|367|1598|123|1930|5|8|15|3|1930|123|1598|Friday|1930Q3|N|Y|N|2426190|2426401|2425839|2426113|N|N|N|N|N| +2426205|AAAAAAAANFFAFCAA|1930-08-16|367|1598|123|1930|6|8|16|3|1930|123|1598|Saturday|1930Q3|N|Y|N|2426190|2426401|2425840|2426114|N|N|N|N|N| +2426206|AAAAAAAAOFFAFCAA|1930-08-17|367|1598|123|1930|0|8|17|3|1930|123|1598|Sunday|1930Q3|N|N|N|2426190|2426401|2425841|2426115|N|N|N|N|N| +2426207|AAAAAAAAPFFAFCAA|1930-08-18|367|1598|123|1930|1|8|18|3|1930|123|1598|Monday|1930Q3|N|N|N|2426190|2426401|2425842|2426116|N|N|N|N|N| +2426208|AAAAAAAAAGFAFCAA|1930-08-19|367|1599|123|1930|2|8|19|3|1930|123|1599|Tuesday|1930Q3|N|N|N|2426190|2426401|2425843|2426117|N|N|N|N|N| +2426209|AAAAAAAABGFAFCAA|1930-08-20|367|1599|123|1930|3|8|20|3|1930|123|1599|Wednesday|1930Q3|N|N|N|2426190|2426401|2425844|2426118|N|N|N|N|N| +2426210|AAAAAAAACGFAFCAA|1930-08-21|367|1599|123|1930|4|8|21|3|1930|123|1599|Thursday|1930Q3|N|N|N|2426190|2426401|2425845|2426119|N|N|N|N|N| +2426211|AAAAAAAADGFAFCAA|1930-08-22|367|1599|123|1930|5|8|22|3|1930|123|1599|Friday|1930Q3|N|Y|N|2426190|2426401|2425846|2426120|N|N|N|N|N| +2426212|AAAAAAAAEGFAFCAA|1930-08-23|367|1599|123|1930|6|8|23|3|1930|123|1599|Saturday|1930Q3|N|Y|N|2426190|2426401|2425847|2426121|N|N|N|N|N| +2426213|AAAAAAAAFGFAFCAA|1930-08-24|367|1599|123|1930|0|8|24|3|1930|123|1599|Sunday|1930Q3|N|N|N|2426190|2426401|2425848|2426122|N|N|N|N|N| +2426214|AAAAAAAAGGFAFCAA|1930-08-25|367|1599|123|1930|1|8|25|3|1930|123|1599|Monday|1930Q3|N|N|N|2426190|2426401|2425849|2426123|N|N|N|N|N| +2426215|AAAAAAAAHGFAFCAA|1930-08-26|367|1600|123|1930|2|8|26|3|1930|123|1600|Tuesday|1930Q3|N|N|N|2426190|2426401|2425850|2426124|N|N|N|N|N| +2426216|AAAAAAAAIGFAFCAA|1930-08-27|367|1600|123|1930|3|8|27|3|1930|123|1600|Wednesday|1930Q3|N|N|N|2426190|2426401|2425851|2426125|N|N|N|N|N| +2426217|AAAAAAAAJGFAFCAA|1930-08-28|367|1600|123|1930|4|8|28|3|1930|123|1600|Thursday|1930Q3|N|N|N|2426190|2426401|2425852|2426126|N|N|N|N|N| +2426218|AAAAAAAAKGFAFCAA|1930-08-29|367|1600|123|1930|5|8|29|3|1930|123|1600|Friday|1930Q3|N|Y|N|2426190|2426401|2425853|2426127|N|N|N|N|N| +2426219|AAAAAAAALGFAFCAA|1930-08-30|367|1600|123|1930|6|8|30|3|1930|123|1600|Saturday|1930Q3|N|Y|N|2426190|2426401|2425854|2426128|N|N|N|N|N| +2426220|AAAAAAAAMGFAFCAA|1930-08-31|367|1600|123|1930|0|8|31|3|1930|123|1600|Sunday|1930Q3|N|N|N|2426190|2426401|2425855|2426129|N|N|N|N|N| +2426221|AAAAAAAANGFAFCAA|1930-09-01|368|1600|124|1930|1|9|1|3|1930|124|1600|Monday|1930Q3|N|N|N|2426221|2426463|2425856|2426130|N|N|N|N|N| +2426222|AAAAAAAAOGFAFCAA|1930-09-02|368|1601|124|1930|2|9|2|3|1930|124|1601|Tuesday|1930Q3|N|N|N|2426221|2426463|2425857|2426131|N|N|N|N|N| +2426223|AAAAAAAAPGFAFCAA|1930-09-03|368|1601|124|1930|3|9|3|3|1930|124|1601|Wednesday|1930Q3|N|N|N|2426221|2426463|2425858|2426132|N|N|N|N|N| +2426224|AAAAAAAAAHFAFCAA|1930-09-04|368|1601|124|1930|4|9|4|3|1930|124|1601|Thursday|1930Q3|N|N|N|2426221|2426463|2425859|2426133|N|N|N|N|N| +2426225|AAAAAAAABHFAFCAA|1930-09-05|368|1601|124|1930|5|9|5|3|1930|124|1601|Friday|1930Q3|N|Y|N|2426221|2426463|2425860|2426134|N|N|N|N|N| +2426226|AAAAAAAACHFAFCAA|1930-09-06|368|1601|124|1930|6|9|6|3|1930|124|1601|Saturday|1930Q3|N|Y|N|2426221|2426463|2425861|2426135|N|N|N|N|N| +2426227|AAAAAAAADHFAFCAA|1930-09-07|368|1601|124|1930|0|9|7|3|1930|124|1601|Sunday|1930Q3|N|N|N|2426221|2426463|2425862|2426136|N|N|N|N|N| +2426228|AAAAAAAAEHFAFCAA|1930-09-08|368|1601|124|1930|1|9|8|3|1930|124|1601|Monday|1930Q3|N|N|N|2426221|2426463|2425863|2426137|N|N|N|N|N| +2426229|AAAAAAAAFHFAFCAA|1930-09-09|368|1602|124|1930|2|9|9|3|1930|124|1602|Tuesday|1930Q3|N|N|N|2426221|2426463|2425864|2426138|N|N|N|N|N| +2426230|AAAAAAAAGHFAFCAA|1930-09-10|368|1602|124|1930|3|9|10|3|1930|124|1602|Wednesday|1930Q3|N|N|N|2426221|2426463|2425865|2426139|N|N|N|N|N| +2426231|AAAAAAAAHHFAFCAA|1930-09-11|368|1602|124|1930|4|9|11|3|1930|124|1602|Thursday|1930Q3|N|N|N|2426221|2426463|2425866|2426140|N|N|N|N|N| +2426232|AAAAAAAAIHFAFCAA|1930-09-12|368|1602|124|1930|5|9|12|3|1930|124|1602|Friday|1930Q3|N|Y|N|2426221|2426463|2425867|2426141|N|N|N|N|N| +2426233|AAAAAAAAJHFAFCAA|1930-09-13|368|1602|124|1930|6|9|13|3|1930|124|1602|Saturday|1930Q3|N|Y|N|2426221|2426463|2425868|2426142|N|N|N|N|N| +2426234|AAAAAAAAKHFAFCAA|1930-09-14|368|1602|124|1930|0|9|14|3|1930|124|1602|Sunday|1930Q3|N|N|N|2426221|2426463|2425869|2426143|N|N|N|N|N| +2426235|AAAAAAAALHFAFCAA|1930-09-15|368|1602|124|1930|1|9|15|3|1930|124|1602|Monday|1930Q3|N|N|N|2426221|2426463|2425870|2426144|N|N|N|N|N| +2426236|AAAAAAAAMHFAFCAA|1930-09-16|368|1603|124|1930|2|9|16|3|1930|124|1603|Tuesday|1930Q3|N|N|N|2426221|2426463|2425871|2426145|N|N|N|N|N| +2426237|AAAAAAAANHFAFCAA|1930-09-17|368|1603|124|1930|3|9|17|3|1930|124|1603|Wednesday|1930Q3|N|N|N|2426221|2426463|2425872|2426146|N|N|N|N|N| +2426238|AAAAAAAAOHFAFCAA|1930-09-18|368|1603|124|1930|4|9|18|3|1930|124|1603|Thursday|1930Q3|N|N|N|2426221|2426463|2425873|2426147|N|N|N|N|N| +2426239|AAAAAAAAPHFAFCAA|1930-09-19|368|1603|124|1930|5|9|19|3|1930|124|1603|Friday|1930Q3|N|Y|N|2426221|2426463|2425874|2426148|N|N|N|N|N| +2426240|AAAAAAAAAIFAFCAA|1930-09-20|368|1603|124|1930|6|9|20|3|1930|124|1603|Saturday|1930Q3|N|Y|N|2426221|2426463|2425875|2426149|N|N|N|N|N| +2426241|AAAAAAAABIFAFCAA|1930-09-21|368|1603|124|1930|0|9|21|3|1930|124|1603|Sunday|1930Q3|N|N|N|2426221|2426463|2425876|2426150|N|N|N|N|N| +2426242|AAAAAAAACIFAFCAA|1930-09-22|368|1603|124|1930|1|9|22|3|1930|124|1603|Monday|1930Q3|N|N|N|2426221|2426463|2425877|2426151|N|N|N|N|N| +2426243|AAAAAAAADIFAFCAA|1930-09-23|368|1604|124|1930|2|9|23|3|1930|124|1604|Tuesday|1930Q3|N|N|N|2426221|2426463|2425878|2426152|N|N|N|N|N| +2426244|AAAAAAAAEIFAFCAA|1930-09-24|368|1604|124|1930|3|9|24|3|1930|124|1604|Wednesday|1930Q3|N|N|N|2426221|2426463|2425879|2426153|N|N|N|N|N| +2426245|AAAAAAAAFIFAFCAA|1930-09-25|368|1604|124|1930|4|9|25|3|1930|124|1604|Thursday|1930Q3|N|N|N|2426221|2426463|2425880|2426154|N|N|N|N|N| +2426246|AAAAAAAAGIFAFCAA|1930-09-26|368|1604|124|1930|5|9|26|3|1930|124|1604|Friday|1930Q3|N|Y|N|2426221|2426463|2425881|2426155|N|N|N|N|N| +2426247|AAAAAAAAHIFAFCAA|1930-09-27|368|1604|124|1930|6|9|27|3|1930|124|1604|Saturday|1930Q3|N|Y|N|2426221|2426463|2425882|2426156|N|N|N|N|N| +2426248|AAAAAAAAIIFAFCAA|1930-09-28|368|1604|124|1930|0|9|28|3|1930|124|1604|Sunday|1930Q3|N|N|N|2426221|2426463|2425883|2426157|N|N|N|N|N| +2426249|AAAAAAAAJIFAFCAA|1930-09-29|368|1604|124|1930|1|9|29|3|1930|124|1604|Monday|1930Q3|N|N|N|2426221|2426463|2425884|2426158|N|N|N|N|N| +2426250|AAAAAAAAKIFAFCAA|1930-09-30|368|1605|124|1930|2|9|30|3|1930|124|1605|Tuesday|1930Q3|N|N|N|2426221|2426463|2425885|2426159|N|N|N|N|N| +2426251|AAAAAAAALIFAFCAA|1930-10-01|369|1605|124|1930|3|10|1|3|1930|124|1605|Wednesday|1930Q3|N|N|N|2426251|2426523|2425886|2426159|N|N|N|N|N| +2426252|AAAAAAAAMIFAFCAA|1930-10-02|369|1605|124|1930|4|10|2|4|1930|124|1605|Thursday|1930Q4|N|N|N|2426251|2426523|2425887|2426160|N|N|N|N|N| +2426253|AAAAAAAANIFAFCAA|1930-10-03|369|1605|124|1930|5|10|3|4|1930|124|1605|Friday|1930Q4|N|Y|N|2426251|2426523|2425888|2426161|N|N|N|N|N| +2426254|AAAAAAAAOIFAFCAA|1930-10-04|369|1605|124|1930|6|10|4|4|1930|124|1605|Saturday|1930Q4|N|Y|N|2426251|2426523|2425889|2426162|N|N|N|N|N| +2426255|AAAAAAAAPIFAFCAA|1930-10-05|369|1605|124|1930|0|10|5|4|1930|124|1605|Sunday|1930Q4|N|N|N|2426251|2426523|2425890|2426163|N|N|N|N|N| +2426256|AAAAAAAAAJFAFCAA|1930-10-06|369|1605|124|1930|1|10|6|4|1930|124|1605|Monday|1930Q4|N|N|N|2426251|2426523|2425891|2426164|N|N|N|N|N| +2426257|AAAAAAAABJFAFCAA|1930-10-07|369|1606|124|1930|2|10|7|4|1930|124|1606|Tuesday|1930Q4|N|N|N|2426251|2426523|2425892|2426165|N|N|N|N|N| +2426258|AAAAAAAACJFAFCAA|1930-10-08|369|1606|124|1930|3|10|8|4|1930|124|1606|Wednesday|1930Q4|N|N|N|2426251|2426523|2425893|2426166|N|N|N|N|N| +2426259|AAAAAAAADJFAFCAA|1930-10-09|369|1606|124|1930|4|10|9|4|1930|124|1606|Thursday|1930Q4|N|N|N|2426251|2426523|2425894|2426167|N|N|N|N|N| +2426260|AAAAAAAAEJFAFCAA|1930-10-10|369|1606|124|1930|5|10|10|4|1930|124|1606|Friday|1930Q4|N|Y|N|2426251|2426523|2425895|2426168|N|N|N|N|N| +2426261|AAAAAAAAFJFAFCAA|1930-10-11|369|1606|124|1930|6|10|11|4|1930|124|1606|Saturday|1930Q4|N|Y|N|2426251|2426523|2425896|2426169|N|N|N|N|N| +2426262|AAAAAAAAGJFAFCAA|1930-10-12|369|1606|124|1930|0|10|12|4|1930|124|1606|Sunday|1930Q4|N|N|N|2426251|2426523|2425897|2426170|N|N|N|N|N| +2426263|AAAAAAAAHJFAFCAA|1930-10-13|369|1606|124|1930|1|10|13|4|1930|124|1606|Monday|1930Q4|N|N|N|2426251|2426523|2425898|2426171|N|N|N|N|N| +2426264|AAAAAAAAIJFAFCAA|1930-10-14|369|1607|124|1930|2|10|14|4|1930|124|1607|Tuesday|1930Q4|N|N|N|2426251|2426523|2425899|2426172|N|N|N|N|N| +2426265|AAAAAAAAJJFAFCAA|1930-10-15|369|1607|124|1930|3|10|15|4|1930|124|1607|Wednesday|1930Q4|N|N|N|2426251|2426523|2425900|2426173|N|N|N|N|N| +2426266|AAAAAAAAKJFAFCAA|1930-10-16|369|1607|124|1930|4|10|16|4|1930|124|1607|Thursday|1930Q4|N|N|N|2426251|2426523|2425901|2426174|N|N|N|N|N| +2426267|AAAAAAAALJFAFCAA|1930-10-17|369|1607|124|1930|5|10|17|4|1930|124|1607|Friday|1930Q4|N|Y|N|2426251|2426523|2425902|2426175|N|N|N|N|N| +2426268|AAAAAAAAMJFAFCAA|1930-10-18|369|1607|124|1930|6|10|18|4|1930|124|1607|Saturday|1930Q4|N|Y|N|2426251|2426523|2425903|2426176|N|N|N|N|N| +2426269|AAAAAAAANJFAFCAA|1930-10-19|369|1607|124|1930|0|10|19|4|1930|124|1607|Sunday|1930Q4|N|N|N|2426251|2426523|2425904|2426177|N|N|N|N|N| +2426270|AAAAAAAAOJFAFCAA|1930-10-20|369|1607|124|1930|1|10|20|4|1930|124|1607|Monday|1930Q4|N|N|N|2426251|2426523|2425905|2426178|N|N|N|N|N| +2426271|AAAAAAAAPJFAFCAA|1930-10-21|369|1608|124|1930|2|10|21|4|1930|124|1608|Tuesday|1930Q4|N|N|N|2426251|2426523|2425906|2426179|N|N|N|N|N| +2426272|AAAAAAAAAKFAFCAA|1930-10-22|369|1608|124|1930|3|10|22|4|1930|124|1608|Wednesday|1930Q4|N|N|N|2426251|2426523|2425907|2426180|N|N|N|N|N| +2426273|AAAAAAAABKFAFCAA|1930-10-23|369|1608|124|1930|4|10|23|4|1930|124|1608|Thursday|1930Q4|N|N|N|2426251|2426523|2425908|2426181|N|N|N|N|N| +2426274|AAAAAAAACKFAFCAA|1930-10-24|369|1608|124|1930|5|10|24|4|1930|124|1608|Friday|1930Q4|N|Y|N|2426251|2426523|2425909|2426182|N|N|N|N|N| +2426275|AAAAAAAADKFAFCAA|1930-10-25|369|1608|124|1930|6|10|25|4|1930|124|1608|Saturday|1930Q4|N|Y|N|2426251|2426523|2425910|2426183|N|N|N|N|N| +2426276|AAAAAAAAEKFAFCAA|1930-10-26|369|1608|124|1930|0|10|26|4|1930|124|1608|Sunday|1930Q4|N|N|N|2426251|2426523|2425911|2426184|N|N|N|N|N| +2426277|AAAAAAAAFKFAFCAA|1930-10-27|369|1608|124|1930|1|10|27|4|1930|124|1608|Monday|1930Q4|N|N|N|2426251|2426523|2425912|2426185|N|N|N|N|N| +2426278|AAAAAAAAGKFAFCAA|1930-10-28|369|1609|124|1930|2|10|28|4|1930|124|1609|Tuesday|1930Q4|N|N|N|2426251|2426523|2425913|2426186|N|N|N|N|N| +2426279|AAAAAAAAHKFAFCAA|1930-10-29|369|1609|124|1930|3|10|29|4|1930|124|1609|Wednesday|1930Q4|N|N|N|2426251|2426523|2425914|2426187|N|N|N|N|N| +2426280|AAAAAAAAIKFAFCAA|1930-10-30|369|1609|124|1930|4|10|30|4|1930|124|1609|Thursday|1930Q4|N|N|N|2426251|2426523|2425915|2426188|N|N|N|N|N| +2426281|AAAAAAAAJKFAFCAA|1930-10-31|369|1609|124|1930|5|10|31|4|1930|124|1609|Friday|1930Q4|N|Y|N|2426251|2426523|2425916|2426189|N|N|N|N|N| +2426282|AAAAAAAAKKFAFCAA|1930-11-01|370|1609|124|1930|6|11|1|4|1930|124|1609|Saturday|1930Q4|N|Y|N|2426282|2426585|2425917|2426190|N|N|N|N|N| +2426283|AAAAAAAALKFAFCAA|1930-11-02|370|1609|124|1930|0|11|2|4|1930|124|1609|Sunday|1930Q4|N|N|N|2426282|2426585|2425918|2426191|N|N|N|N|N| +2426284|AAAAAAAAMKFAFCAA|1930-11-03|370|1609|124|1930|1|11|3|4|1930|124|1609|Monday|1930Q4|N|N|N|2426282|2426585|2425919|2426192|N|N|N|N|N| +2426285|AAAAAAAANKFAFCAA|1930-11-04|370|1610|124|1930|2|11|4|4|1930|124|1610|Tuesday|1930Q4|N|N|N|2426282|2426585|2425920|2426193|N|N|N|N|N| +2426286|AAAAAAAAOKFAFCAA|1930-11-05|370|1610|124|1930|3|11|5|4|1930|124|1610|Wednesday|1930Q4|N|N|N|2426282|2426585|2425921|2426194|N|N|N|N|N| +2426287|AAAAAAAAPKFAFCAA|1930-11-06|370|1610|124|1930|4|11|6|4|1930|124|1610|Thursday|1930Q4|N|N|N|2426282|2426585|2425922|2426195|N|N|N|N|N| +2426288|AAAAAAAAALFAFCAA|1930-11-07|370|1610|124|1930|5|11|7|4|1930|124|1610|Friday|1930Q4|N|Y|N|2426282|2426585|2425923|2426196|N|N|N|N|N| +2426289|AAAAAAAABLFAFCAA|1930-11-08|370|1610|124|1930|6|11|8|4|1930|124|1610|Saturday|1930Q4|N|Y|N|2426282|2426585|2425924|2426197|N|N|N|N|N| +2426290|AAAAAAAACLFAFCAA|1930-11-09|370|1610|124|1930|0|11|9|4|1930|124|1610|Sunday|1930Q4|N|N|N|2426282|2426585|2425925|2426198|N|N|N|N|N| +2426291|AAAAAAAADLFAFCAA|1930-11-10|370|1610|124|1930|1|11|10|4|1930|124|1610|Monday|1930Q4|N|N|N|2426282|2426585|2425926|2426199|N|N|N|N|N| +2426292|AAAAAAAAELFAFCAA|1930-11-11|370|1611|124|1930|2|11|11|4|1930|124|1611|Tuesday|1930Q4|N|N|N|2426282|2426585|2425927|2426200|N|N|N|N|N| +2426293|AAAAAAAAFLFAFCAA|1930-11-12|370|1611|124|1930|3|11|12|4|1930|124|1611|Wednesday|1930Q4|N|N|N|2426282|2426585|2425928|2426201|N|N|N|N|N| +2426294|AAAAAAAAGLFAFCAA|1930-11-13|370|1611|124|1930|4|11|13|4|1930|124|1611|Thursday|1930Q4|N|N|N|2426282|2426585|2425929|2426202|N|N|N|N|N| +2426295|AAAAAAAAHLFAFCAA|1930-11-14|370|1611|124|1930|5|11|14|4|1930|124|1611|Friday|1930Q4|N|Y|N|2426282|2426585|2425930|2426203|N|N|N|N|N| +2426296|AAAAAAAAILFAFCAA|1930-11-15|370|1611|124|1930|6|11|15|4|1930|124|1611|Saturday|1930Q4|N|Y|N|2426282|2426585|2425931|2426204|N|N|N|N|N| +2426297|AAAAAAAAJLFAFCAA|1930-11-16|370|1611|124|1930|0|11|16|4|1930|124|1611|Sunday|1930Q4|N|N|N|2426282|2426585|2425932|2426205|N|N|N|N|N| +2426298|AAAAAAAAKLFAFCAA|1930-11-17|370|1611|124|1930|1|11|17|4|1930|124|1611|Monday|1930Q4|N|N|N|2426282|2426585|2425933|2426206|N|N|N|N|N| +2426299|AAAAAAAALLFAFCAA|1930-11-18|370|1612|124|1930|2|11|18|4|1930|124|1612|Tuesday|1930Q4|N|N|N|2426282|2426585|2425934|2426207|N|N|N|N|N| +2426300|AAAAAAAAMLFAFCAA|1930-11-19|370|1612|124|1930|3|11|19|4|1930|124|1612|Wednesday|1930Q4|N|N|N|2426282|2426585|2425935|2426208|N|N|N|N|N| +2426301|AAAAAAAANLFAFCAA|1930-11-20|370|1612|124|1930|4|11|20|4|1930|124|1612|Thursday|1930Q4|N|N|N|2426282|2426585|2425936|2426209|N|N|N|N|N| +2426302|AAAAAAAAOLFAFCAA|1930-11-21|370|1612|124|1930|5|11|21|4|1930|124|1612|Friday|1930Q4|N|Y|N|2426282|2426585|2425937|2426210|N|N|N|N|N| +2426303|AAAAAAAAPLFAFCAA|1930-11-22|370|1612|124|1930|6|11|22|4|1930|124|1612|Saturday|1930Q4|N|Y|N|2426282|2426585|2425938|2426211|N|N|N|N|N| +2426304|AAAAAAAAAMFAFCAA|1930-11-23|370|1612|124|1930|0|11|23|4|1930|124|1612|Sunday|1930Q4|N|N|N|2426282|2426585|2425939|2426212|N|N|N|N|N| +2426305|AAAAAAAABMFAFCAA|1930-11-24|370|1612|124|1930|1|11|24|4|1930|124|1612|Monday|1930Q4|N|N|N|2426282|2426585|2425940|2426213|N|N|N|N|N| +2426306|AAAAAAAACMFAFCAA|1930-11-25|370|1613|124|1930|2|11|25|4|1930|124|1613|Tuesday|1930Q4|N|N|N|2426282|2426585|2425941|2426214|N|N|N|N|N| +2426307|AAAAAAAADMFAFCAA|1930-11-26|370|1613|124|1930|3|11|26|4|1930|124|1613|Wednesday|1930Q4|N|N|N|2426282|2426585|2425942|2426215|N|N|N|N|N| +2426308|AAAAAAAAEMFAFCAA|1930-11-27|370|1613|124|1930|4|11|27|4|1930|124|1613|Thursday|1930Q4|N|N|N|2426282|2426585|2425943|2426216|N|N|N|N|N| +2426309|AAAAAAAAFMFAFCAA|1930-11-28|370|1613|124|1930|5|11|28|4|1930|124|1613|Friday|1930Q4|N|Y|N|2426282|2426585|2425944|2426217|N|N|N|N|N| +2426310|AAAAAAAAGMFAFCAA|1930-11-29|370|1613|124|1930|6|11|29|4|1930|124|1613|Saturday|1930Q4|N|Y|N|2426282|2426585|2425945|2426218|N|N|N|N|N| +2426311|AAAAAAAAHMFAFCAA|1930-11-30|370|1613|124|1930|0|11|30|4|1930|124|1613|Sunday|1930Q4|N|N|N|2426282|2426585|2425946|2426219|N|N|N|N|N| +2426312|AAAAAAAAIMFAFCAA|1930-12-01|371|1613|125|1930|1|12|1|4|1930|125|1613|Monday|1930Q4|N|N|N|2426312|2426645|2425947|2426220|N|N|N|N|N| +2426313|AAAAAAAAJMFAFCAA|1930-12-02|371|1614|125|1930|2|12|2|4|1930|125|1614|Tuesday|1930Q4|N|N|N|2426312|2426645|2425948|2426221|N|N|N|N|N| +2426314|AAAAAAAAKMFAFCAA|1930-12-03|371|1614|125|1930|3|12|3|4|1930|125|1614|Wednesday|1930Q4|N|N|N|2426312|2426645|2425949|2426222|N|N|N|N|N| +2426315|AAAAAAAALMFAFCAA|1930-12-04|371|1614|125|1930|4|12|4|4|1930|125|1614|Thursday|1930Q4|N|N|N|2426312|2426645|2425950|2426223|N|N|N|N|N| +2426316|AAAAAAAAMMFAFCAA|1930-12-05|371|1614|125|1930|5|12|5|4|1930|125|1614|Friday|1930Q4|N|Y|N|2426312|2426645|2425951|2426224|N|N|N|N|N| +2426317|AAAAAAAANMFAFCAA|1930-12-06|371|1614|125|1930|6|12|6|4|1930|125|1614|Saturday|1930Q4|N|Y|N|2426312|2426645|2425952|2426225|N|N|N|N|N| +2426318|AAAAAAAAOMFAFCAA|1930-12-07|371|1614|125|1930|0|12|7|4|1930|125|1614|Sunday|1930Q4|N|N|N|2426312|2426645|2425953|2426226|N|N|N|N|N| +2426319|AAAAAAAAPMFAFCAA|1930-12-08|371|1614|125|1930|1|12|8|4|1930|125|1614|Monday|1930Q4|N|N|N|2426312|2426645|2425954|2426227|N|N|N|N|N| +2426320|AAAAAAAAANFAFCAA|1930-12-09|371|1615|125|1930|2|12|9|4|1930|125|1615|Tuesday|1930Q4|N|N|N|2426312|2426645|2425955|2426228|N|N|N|N|N| +2426321|AAAAAAAABNFAFCAA|1930-12-10|371|1615|125|1930|3|12|10|4|1930|125|1615|Wednesday|1930Q4|N|N|N|2426312|2426645|2425956|2426229|N|N|N|N|N| +2426322|AAAAAAAACNFAFCAA|1930-12-11|371|1615|125|1930|4|12|11|4|1930|125|1615|Thursday|1930Q4|N|N|N|2426312|2426645|2425957|2426230|N|N|N|N|N| +2426323|AAAAAAAADNFAFCAA|1930-12-12|371|1615|125|1930|5|12|12|4|1930|125|1615|Friday|1930Q4|N|Y|N|2426312|2426645|2425958|2426231|N|N|N|N|N| +2426324|AAAAAAAAENFAFCAA|1930-12-13|371|1615|125|1930|6|12|13|4|1930|125|1615|Saturday|1930Q4|N|Y|N|2426312|2426645|2425959|2426232|N|N|N|N|N| +2426325|AAAAAAAAFNFAFCAA|1930-12-14|371|1615|125|1930|0|12|14|4|1930|125|1615|Sunday|1930Q4|N|N|N|2426312|2426645|2425960|2426233|N|N|N|N|N| +2426326|AAAAAAAAGNFAFCAA|1930-12-15|371|1615|125|1930|1|12|15|4|1930|125|1615|Monday|1930Q4|N|N|N|2426312|2426645|2425961|2426234|N|N|N|N|N| +2426327|AAAAAAAAHNFAFCAA|1930-12-16|371|1616|125|1930|2|12|16|4|1930|125|1616|Tuesday|1930Q4|N|N|N|2426312|2426645|2425962|2426235|N|N|N|N|N| +2426328|AAAAAAAAINFAFCAA|1930-12-17|371|1616|125|1930|3|12|17|4|1930|125|1616|Wednesday|1930Q4|N|N|N|2426312|2426645|2425963|2426236|N|N|N|N|N| +2426329|AAAAAAAAJNFAFCAA|1930-12-18|371|1616|125|1930|4|12|18|4|1930|125|1616|Thursday|1930Q4|N|N|N|2426312|2426645|2425964|2426237|N|N|N|N|N| +2426330|AAAAAAAAKNFAFCAA|1930-12-19|371|1616|125|1930|5|12|19|4|1930|125|1616|Friday|1930Q4|N|Y|N|2426312|2426645|2425965|2426238|N|N|N|N|N| +2426331|AAAAAAAALNFAFCAA|1930-12-20|371|1616|125|1930|6|12|20|4|1930|125|1616|Saturday|1930Q4|N|Y|N|2426312|2426645|2425966|2426239|N|N|N|N|N| +2426332|AAAAAAAAMNFAFCAA|1930-12-21|371|1616|125|1930|0|12|21|4|1930|125|1616|Sunday|1930Q4|N|N|N|2426312|2426645|2425967|2426240|N|N|N|N|N| +2426333|AAAAAAAANNFAFCAA|1930-12-22|371|1616|125|1930|1|12|22|4|1930|125|1616|Monday|1930Q4|N|N|N|2426312|2426645|2425968|2426241|N|N|N|N|N| +2426334|AAAAAAAAONFAFCAA|1930-12-23|371|1617|125|1930|2|12|23|4|1930|125|1617|Tuesday|1930Q4|N|N|N|2426312|2426645|2425969|2426242|N|N|N|N|N| +2426335|AAAAAAAAPNFAFCAA|1930-12-24|371|1617|125|1930|3|12|24|4|1930|125|1617|Wednesday|1930Q4|N|N|N|2426312|2426645|2425970|2426243|N|N|N|N|N| +2426336|AAAAAAAAAOFAFCAA|1930-12-25|371|1617|125|1930|4|12|25|4|1930|125|1617|Thursday|1930Q4|N|N|N|2426312|2426645|2425971|2426244|N|N|N|N|N| +2426337|AAAAAAAABOFAFCAA|1930-12-26|371|1617|125|1930|5|12|26|4|1930|125|1617|Friday|1930Q4|Y|Y|N|2426312|2426645|2425972|2426245|N|N|N|N|N| +2426338|AAAAAAAACOFAFCAA|1930-12-27|371|1617|125|1930|6|12|27|4|1930|125|1617|Saturday|1930Q4|N|Y|Y|2426312|2426645|2425973|2426246|N|N|N|N|N| +2426339|AAAAAAAADOFAFCAA|1930-12-28|371|1617|125|1930|0|12|28|4|1930|125|1617|Sunday|1930Q4|N|N|N|2426312|2426645|2425974|2426247|N|N|N|N|N| +2426340|AAAAAAAAEOFAFCAA|1930-12-29|371|1617|125|1930|1|12|29|4|1930|125|1617|Monday|1930Q4|N|N|N|2426312|2426645|2425975|2426248|N|N|N|N|N| +2426341|AAAAAAAAFOFAFCAA|1930-12-30|371|1618|125|1930|2|12|30|4|1930|125|1618|Tuesday|1930Q4|N|N|N|2426312|2426645|2425976|2426249|N|N|N|N|N| +2426342|AAAAAAAAGOFAFCAA|1930-12-31|371|1618|125|1930|3|12|31|4|1930|125|1618|Wednesday|1930Q4|N|N|N|2426312|2426645|2425977|2426250|N|N|N|N|N| +2426343|AAAAAAAAHOFAFCAA|1931-01-01|372|1618|125|1931|4|1|1|1|1931|125|1618|Thursday|1931Q1|Y|N|N|2426343|2426342|2425978|2426251|N|N|N|N|N| +2426344|AAAAAAAAIOFAFCAA|1931-01-02|372|1618|125|1931|5|1|2|1|1931|125|1618|Friday|1931Q1|N|Y|Y|2426343|2426342|2425979|2426252|N|N|N|N|N| +2426345|AAAAAAAAJOFAFCAA|1931-01-03|372|1618|125|1931|6|1|3|1|1931|125|1618|Saturday|1931Q1|N|Y|N|2426343|2426342|2425980|2426253|N|N|N|N|N| +2426346|AAAAAAAAKOFAFCAA|1931-01-04|372|1618|125|1931|0|1|4|1|1931|125|1618|Sunday|1931Q1|N|N|N|2426343|2426342|2425981|2426254|N|N|N|N|N| +2426347|AAAAAAAALOFAFCAA|1931-01-05|372|1618|125|1931|1|1|5|1|1931|125|1618|Monday|1931Q1|N|N|N|2426343|2426342|2425982|2426255|N|N|N|N|N| +2426348|AAAAAAAAMOFAFCAA|1931-01-06|372|1619|125|1931|2|1|6|1|1931|125|1619|Tuesday|1931Q1|N|N|N|2426343|2426342|2425983|2426256|N|N|N|N|N| +2426349|AAAAAAAANOFAFCAA|1931-01-07|372|1619|125|1931|3|1|7|1|1931|125|1619|Wednesday|1931Q1|N|N|N|2426343|2426342|2425984|2426257|N|N|N|N|N| +2426350|AAAAAAAAOOFAFCAA|1931-01-08|372|1619|125|1931|4|1|8|1|1931|125|1619|Thursday|1931Q1|N|N|N|2426343|2426342|2425985|2426258|N|N|N|N|N| +2426351|AAAAAAAAPOFAFCAA|1931-01-09|372|1619|125|1931|5|1|9|1|1931|125|1619|Friday|1931Q1|N|Y|N|2426343|2426342|2425986|2426259|N|N|N|N|N| +2426352|AAAAAAAAAPFAFCAA|1931-01-10|372|1619|125|1931|6|1|10|1|1931|125|1619|Saturday|1931Q1|N|Y|N|2426343|2426342|2425987|2426260|N|N|N|N|N| +2426353|AAAAAAAABPFAFCAA|1931-01-11|372|1619|125|1931|0|1|11|1|1931|125|1619|Sunday|1931Q1|N|N|N|2426343|2426342|2425988|2426261|N|N|N|N|N| +2426354|AAAAAAAACPFAFCAA|1931-01-12|372|1619|125|1931|1|1|12|1|1931|125|1619|Monday|1931Q1|N|N|N|2426343|2426342|2425989|2426262|N|N|N|N|N| +2426355|AAAAAAAADPFAFCAA|1931-01-13|372|1620|125|1931|2|1|13|1|1931|125|1620|Tuesday|1931Q1|N|N|N|2426343|2426342|2425990|2426263|N|N|N|N|N| +2426356|AAAAAAAAEPFAFCAA|1931-01-14|372|1620|125|1931|3|1|14|1|1931|125|1620|Wednesday|1931Q1|N|N|N|2426343|2426342|2425991|2426264|N|N|N|N|N| +2426357|AAAAAAAAFPFAFCAA|1931-01-15|372|1620|125|1931|4|1|15|1|1931|125|1620|Thursday|1931Q1|N|N|N|2426343|2426342|2425992|2426265|N|N|N|N|N| +2426358|AAAAAAAAGPFAFCAA|1931-01-16|372|1620|125|1931|5|1|16|1|1931|125|1620|Friday|1931Q1|N|Y|N|2426343|2426342|2425993|2426266|N|N|N|N|N| +2426359|AAAAAAAAHPFAFCAA|1931-01-17|372|1620|125|1931|6|1|17|1|1931|125|1620|Saturday|1931Q1|N|Y|N|2426343|2426342|2425994|2426267|N|N|N|N|N| +2426360|AAAAAAAAIPFAFCAA|1931-01-18|372|1620|125|1931|0|1|18|1|1931|125|1620|Sunday|1931Q1|N|N|N|2426343|2426342|2425995|2426268|N|N|N|N|N| +2426361|AAAAAAAAJPFAFCAA|1931-01-19|372|1620|125|1931|1|1|19|1|1931|125|1620|Monday|1931Q1|N|N|N|2426343|2426342|2425996|2426269|N|N|N|N|N| +2426362|AAAAAAAAKPFAFCAA|1931-01-20|372|1621|125|1931|2|1|20|1|1931|125|1621|Tuesday|1931Q1|N|N|N|2426343|2426342|2425997|2426270|N|N|N|N|N| +2426363|AAAAAAAALPFAFCAA|1931-01-21|372|1621|125|1931|3|1|21|1|1931|125|1621|Wednesday|1931Q1|N|N|N|2426343|2426342|2425998|2426271|N|N|N|N|N| +2426364|AAAAAAAAMPFAFCAA|1931-01-22|372|1621|125|1931|4|1|22|1|1931|125|1621|Thursday|1931Q1|N|N|N|2426343|2426342|2425999|2426272|N|N|N|N|N| +2426365|AAAAAAAANPFAFCAA|1931-01-23|372|1621|125|1931|5|1|23|1|1931|125|1621|Friday|1931Q1|N|Y|N|2426343|2426342|2426000|2426273|N|N|N|N|N| +2426366|AAAAAAAAOPFAFCAA|1931-01-24|372|1621|125|1931|6|1|24|1|1931|125|1621|Saturday|1931Q1|N|Y|N|2426343|2426342|2426001|2426274|N|N|N|N|N| +2426367|AAAAAAAAPPFAFCAA|1931-01-25|372|1621|125|1931|0|1|25|1|1931|125|1621|Sunday|1931Q1|N|N|N|2426343|2426342|2426002|2426275|N|N|N|N|N| +2426368|AAAAAAAAAAGAFCAA|1931-01-26|372|1621|125|1931|1|1|26|1|1931|125|1621|Monday|1931Q1|N|N|N|2426343|2426342|2426003|2426276|N|N|N|N|N| +2426369|AAAAAAAABAGAFCAA|1931-01-27|372|1622|125|1931|2|1|27|1|1931|125|1622|Tuesday|1931Q1|N|N|N|2426343|2426342|2426004|2426277|N|N|N|N|N| +2426370|AAAAAAAACAGAFCAA|1931-01-28|372|1622|125|1931|3|1|28|1|1931|125|1622|Wednesday|1931Q1|N|N|N|2426343|2426342|2426005|2426278|N|N|N|N|N| +2426371|AAAAAAAADAGAFCAA|1931-01-29|372|1622|125|1931|4|1|29|1|1931|125|1622|Thursday|1931Q1|N|N|N|2426343|2426342|2426006|2426279|N|N|N|N|N| +2426372|AAAAAAAAEAGAFCAA|1931-01-30|372|1622|125|1931|5|1|30|1|1931|125|1622|Friday|1931Q1|N|Y|N|2426343|2426342|2426007|2426280|N|N|N|N|N| +2426373|AAAAAAAAFAGAFCAA|1931-01-31|372|1622|125|1931|6|1|31|1|1931|125|1622|Saturday|1931Q1|N|Y|N|2426343|2426342|2426008|2426281|N|N|N|N|N| +2426374|AAAAAAAAGAGAFCAA|1931-02-01|373|1622|125|1931|0|2|1|1|1931|125|1622|Sunday|1931Q1|N|N|N|2426374|2426404|2426009|2426282|N|N|N|N|N| +2426375|AAAAAAAAHAGAFCAA|1931-02-02|373|1622|125|1931|1|2|2|1|1931|125|1622|Monday|1931Q1|N|N|N|2426374|2426404|2426010|2426283|N|N|N|N|N| +2426376|AAAAAAAAIAGAFCAA|1931-02-03|373|1623|125|1931|2|2|3|1|1931|125|1623|Tuesday|1931Q1|N|N|N|2426374|2426404|2426011|2426284|N|N|N|N|N| +2426377|AAAAAAAAJAGAFCAA|1931-02-04|373|1623|125|1931|3|2|4|1|1931|125|1623|Wednesday|1931Q1|N|N|N|2426374|2426404|2426012|2426285|N|N|N|N|N| +2426378|AAAAAAAAKAGAFCAA|1931-02-05|373|1623|125|1931|4|2|5|1|1931|125|1623|Thursday|1931Q1|N|N|N|2426374|2426404|2426013|2426286|N|N|N|N|N| +2426379|AAAAAAAALAGAFCAA|1931-02-06|373|1623|125|1931|5|2|6|1|1931|125|1623|Friday|1931Q1|N|Y|N|2426374|2426404|2426014|2426287|N|N|N|N|N| +2426380|AAAAAAAAMAGAFCAA|1931-02-07|373|1623|125|1931|6|2|7|1|1931|125|1623|Saturday|1931Q1|N|Y|N|2426374|2426404|2426015|2426288|N|N|N|N|N| +2426381|AAAAAAAANAGAFCAA|1931-02-08|373|1623|125|1931|0|2|8|1|1931|125|1623|Sunday|1931Q1|N|N|N|2426374|2426404|2426016|2426289|N|N|N|N|N| +2426382|AAAAAAAAOAGAFCAA|1931-02-09|373|1623|125|1931|1|2|9|1|1931|125|1623|Monday|1931Q1|N|N|N|2426374|2426404|2426017|2426290|N|N|N|N|N| +2426383|AAAAAAAAPAGAFCAA|1931-02-10|373|1624|125|1931|2|2|10|1|1931|125|1624|Tuesday|1931Q1|N|N|N|2426374|2426404|2426018|2426291|N|N|N|N|N| +2426384|AAAAAAAAABGAFCAA|1931-02-11|373|1624|125|1931|3|2|11|1|1931|125|1624|Wednesday|1931Q1|N|N|N|2426374|2426404|2426019|2426292|N|N|N|N|N| +2426385|AAAAAAAABBGAFCAA|1931-02-12|373|1624|125|1931|4|2|12|1|1931|125|1624|Thursday|1931Q1|N|N|N|2426374|2426404|2426020|2426293|N|N|N|N|N| +2426386|AAAAAAAACBGAFCAA|1931-02-13|373|1624|125|1931|5|2|13|1|1931|125|1624|Friday|1931Q1|N|Y|N|2426374|2426404|2426021|2426294|N|N|N|N|N| +2426387|AAAAAAAADBGAFCAA|1931-02-14|373|1624|125|1931|6|2|14|1|1931|125|1624|Saturday|1931Q1|N|Y|N|2426374|2426404|2426022|2426295|N|N|N|N|N| +2426388|AAAAAAAAEBGAFCAA|1931-02-15|373|1624|125|1931|0|2|15|1|1931|125|1624|Sunday|1931Q1|N|N|N|2426374|2426404|2426023|2426296|N|N|N|N|N| +2426389|AAAAAAAAFBGAFCAA|1931-02-16|373|1624|125|1931|1|2|16|1|1931|125|1624|Monday|1931Q1|N|N|N|2426374|2426404|2426024|2426297|N|N|N|N|N| +2426390|AAAAAAAAGBGAFCAA|1931-02-17|373|1625|125|1931|2|2|17|1|1931|125|1625|Tuesday|1931Q1|N|N|N|2426374|2426404|2426025|2426298|N|N|N|N|N| +2426391|AAAAAAAAHBGAFCAA|1931-02-18|373|1625|125|1931|3|2|18|1|1931|125|1625|Wednesday|1931Q1|N|N|N|2426374|2426404|2426026|2426299|N|N|N|N|N| +2426392|AAAAAAAAIBGAFCAA|1931-02-19|373|1625|125|1931|4|2|19|1|1931|125|1625|Thursday|1931Q1|N|N|N|2426374|2426404|2426027|2426300|N|N|N|N|N| +2426393|AAAAAAAAJBGAFCAA|1931-02-20|373|1625|125|1931|5|2|20|1|1931|125|1625|Friday|1931Q1|N|Y|N|2426374|2426404|2426028|2426301|N|N|N|N|N| +2426394|AAAAAAAAKBGAFCAA|1931-02-21|373|1625|125|1931|6|2|21|1|1931|125|1625|Saturday|1931Q1|N|Y|N|2426374|2426404|2426029|2426302|N|N|N|N|N| +2426395|AAAAAAAALBGAFCAA|1931-02-22|373|1625|125|1931|0|2|22|1|1931|125|1625|Sunday|1931Q1|N|N|N|2426374|2426404|2426030|2426303|N|N|N|N|N| +2426396|AAAAAAAAMBGAFCAA|1931-02-23|373|1625|125|1931|1|2|23|1|1931|125|1625|Monday|1931Q1|N|N|N|2426374|2426404|2426031|2426304|N|N|N|N|N| +2426397|AAAAAAAANBGAFCAA|1931-02-24|373|1626|125|1931|2|2|24|1|1931|125|1626|Tuesday|1931Q1|N|N|N|2426374|2426404|2426032|2426305|N|N|N|N|N| +2426398|AAAAAAAAOBGAFCAA|1931-02-25|373|1626|125|1931|3|2|25|1|1931|125|1626|Wednesday|1931Q1|N|N|N|2426374|2426404|2426033|2426306|N|N|N|N|N| +2426399|AAAAAAAAPBGAFCAA|1931-02-26|373|1626|125|1931|4|2|26|1|1931|125|1626|Thursday|1931Q1|N|N|N|2426374|2426404|2426034|2426307|N|N|N|N|N| +2426400|AAAAAAAAACGAFCAA|1931-02-27|373|1626|125|1931|5|2|27|1|1931|125|1626|Friday|1931Q1|N|Y|N|2426374|2426404|2426035|2426308|N|N|N|N|N| +2426401|AAAAAAAABCGAFCAA|1931-02-28|373|1626|125|1931|6|2|28|1|1931|125|1626|Saturday|1931Q1|N|Y|N|2426374|2426404|2426036|2426309|N|N|N|N|N| +2426402|AAAAAAAACCGAFCAA|1931-03-01|374|1626|126|1931|0|3|1|1|1931|126|1626|Sunday|1931Q1|N|N|N|2426402|2426460|2426037|2426310|N|N|N|N|N| +2426403|AAAAAAAADCGAFCAA|1931-03-02|374|1626|126|1931|1|3|2|1|1931|126|1626|Monday|1931Q1|N|N|N|2426402|2426460|2426038|2426311|N|N|N|N|N| +2426404|AAAAAAAAECGAFCAA|1931-03-03|374|1627|126|1931|2|3|3|1|1931|126|1627|Tuesday|1931Q1|N|N|N|2426402|2426460|2426039|2426312|N|N|N|N|N| +2426405|AAAAAAAAFCGAFCAA|1931-03-04|374|1627|126|1931|3|3|4|1|1931|126|1627|Wednesday|1931Q1|N|N|N|2426402|2426460|2426040|2426313|N|N|N|N|N| +2426406|AAAAAAAAGCGAFCAA|1931-03-05|374|1627|126|1931|4|3|5|1|1931|126|1627|Thursday|1931Q1|N|N|N|2426402|2426460|2426041|2426314|N|N|N|N|N| +2426407|AAAAAAAAHCGAFCAA|1931-03-06|374|1627|126|1931|5|3|6|1|1931|126|1627|Friday|1931Q1|N|Y|N|2426402|2426460|2426042|2426315|N|N|N|N|N| +2426408|AAAAAAAAICGAFCAA|1931-03-07|374|1627|126|1931|6|3|7|1|1931|126|1627|Saturday|1931Q1|N|Y|N|2426402|2426460|2426043|2426316|N|N|N|N|N| +2426409|AAAAAAAAJCGAFCAA|1931-03-08|374|1627|126|1931|0|3|8|1|1931|126|1627|Sunday|1931Q1|N|N|N|2426402|2426460|2426044|2426317|N|N|N|N|N| +2426410|AAAAAAAAKCGAFCAA|1931-03-09|374|1627|126|1931|1|3|9|1|1931|126|1627|Monday|1931Q1|N|N|N|2426402|2426460|2426045|2426318|N|N|N|N|N| +2426411|AAAAAAAALCGAFCAA|1931-03-10|374|1628|126|1931|2|3|10|1|1931|126|1628|Tuesday|1931Q1|N|N|N|2426402|2426460|2426046|2426319|N|N|N|N|N| +2426412|AAAAAAAAMCGAFCAA|1931-03-11|374|1628|126|1931|3|3|11|1|1931|126|1628|Wednesday|1931Q1|N|N|N|2426402|2426460|2426047|2426320|N|N|N|N|N| +2426413|AAAAAAAANCGAFCAA|1931-03-12|374|1628|126|1931|4|3|12|1|1931|126|1628|Thursday|1931Q1|N|N|N|2426402|2426460|2426048|2426321|N|N|N|N|N| +2426414|AAAAAAAAOCGAFCAA|1931-03-13|374|1628|126|1931|5|3|13|1|1931|126|1628|Friday|1931Q1|N|Y|N|2426402|2426460|2426049|2426322|N|N|N|N|N| +2426415|AAAAAAAAPCGAFCAA|1931-03-14|374|1628|126|1931|6|3|14|1|1931|126|1628|Saturday|1931Q1|N|Y|N|2426402|2426460|2426050|2426323|N|N|N|N|N| +2426416|AAAAAAAAADGAFCAA|1931-03-15|374|1628|126|1931|0|3|15|1|1931|126|1628|Sunday|1931Q1|N|N|N|2426402|2426460|2426051|2426324|N|N|N|N|N| +2426417|AAAAAAAABDGAFCAA|1931-03-16|374|1628|126|1931|1|3|16|1|1931|126|1628|Monday|1931Q1|N|N|N|2426402|2426460|2426052|2426325|N|N|N|N|N| +2426418|AAAAAAAACDGAFCAA|1931-03-17|374|1629|126|1931|2|3|17|1|1931|126|1629|Tuesday|1931Q1|N|N|N|2426402|2426460|2426053|2426326|N|N|N|N|N| +2426419|AAAAAAAADDGAFCAA|1931-03-18|374|1629|126|1931|3|3|18|1|1931|126|1629|Wednesday|1931Q1|N|N|N|2426402|2426460|2426054|2426327|N|N|N|N|N| +2426420|AAAAAAAAEDGAFCAA|1931-03-19|374|1629|126|1931|4|3|19|1|1931|126|1629|Thursday|1931Q1|N|N|N|2426402|2426460|2426055|2426328|N|N|N|N|N| +2426421|AAAAAAAAFDGAFCAA|1931-03-20|374|1629|126|1931|5|3|20|1|1931|126|1629|Friday|1931Q1|N|Y|N|2426402|2426460|2426056|2426329|N|N|N|N|N| +2426422|AAAAAAAAGDGAFCAA|1931-03-21|374|1629|126|1931|6|3|21|1|1931|126|1629|Saturday|1931Q1|N|Y|N|2426402|2426460|2426057|2426330|N|N|N|N|N| +2426423|AAAAAAAAHDGAFCAA|1931-03-22|374|1629|126|1931|0|3|22|1|1931|126|1629|Sunday|1931Q1|N|N|N|2426402|2426460|2426058|2426331|N|N|N|N|N| +2426424|AAAAAAAAIDGAFCAA|1931-03-23|374|1629|126|1931|1|3|23|1|1931|126|1629|Monday|1931Q1|N|N|N|2426402|2426460|2426059|2426332|N|N|N|N|N| +2426425|AAAAAAAAJDGAFCAA|1931-03-24|374|1630|126|1931|2|3|24|1|1931|126|1630|Tuesday|1931Q1|N|N|N|2426402|2426460|2426060|2426333|N|N|N|N|N| +2426426|AAAAAAAAKDGAFCAA|1931-03-25|374|1630|126|1931|3|3|25|1|1931|126|1630|Wednesday|1931Q1|N|N|N|2426402|2426460|2426061|2426334|N|N|N|N|N| +2426427|AAAAAAAALDGAFCAA|1931-03-26|374|1630|126|1931|4|3|26|1|1931|126|1630|Thursday|1931Q1|N|N|N|2426402|2426460|2426062|2426335|N|N|N|N|N| +2426428|AAAAAAAAMDGAFCAA|1931-03-27|374|1630|126|1931|5|3|27|1|1931|126|1630|Friday|1931Q1|N|Y|N|2426402|2426460|2426063|2426336|N|N|N|N|N| +2426429|AAAAAAAANDGAFCAA|1931-03-28|374|1630|126|1931|6|3|28|1|1931|126|1630|Saturday|1931Q1|N|Y|N|2426402|2426460|2426064|2426337|N|N|N|N|N| +2426430|AAAAAAAAODGAFCAA|1931-03-29|374|1630|126|1931|0|3|29|1|1931|126|1630|Sunday|1931Q1|N|N|N|2426402|2426460|2426065|2426338|N|N|N|N|N| +2426431|AAAAAAAAPDGAFCAA|1931-03-30|374|1630|126|1931|1|3|30|1|1931|126|1630|Monday|1931Q1|N|N|N|2426402|2426460|2426066|2426339|N|N|N|N|N| +2426432|AAAAAAAAAEGAFCAA|1931-03-31|374|1631|126|1931|2|3|31|1|1931|126|1631|Tuesday|1931Q1|N|N|N|2426402|2426460|2426067|2426340|N|N|N|N|N| +2426433|AAAAAAAABEGAFCAA|1931-04-01|375|1631|126|1931|3|4|1|1|1931|126|1631|Wednesday|1931Q1|N|N|N|2426433|2426522|2426068|2426343|N|N|N|N|N| +2426434|AAAAAAAACEGAFCAA|1931-04-02|375|1631|126|1931|4|4|2|2|1931|126|1631|Thursday|1931Q2|N|N|N|2426433|2426522|2426069|2426344|N|N|N|N|N| +2426435|AAAAAAAADEGAFCAA|1931-04-03|375|1631|126|1931|5|4|3|2|1931|126|1631|Friday|1931Q2|N|Y|N|2426433|2426522|2426070|2426345|N|N|N|N|N| +2426436|AAAAAAAAEEGAFCAA|1931-04-04|375|1631|126|1931|6|4|4|2|1931|126|1631|Saturday|1931Q2|N|Y|N|2426433|2426522|2426071|2426346|N|N|N|N|N| +2426437|AAAAAAAAFEGAFCAA|1931-04-05|375|1631|126|1931|0|4|5|2|1931|126|1631|Sunday|1931Q2|N|N|N|2426433|2426522|2426072|2426347|N|N|N|N|N| +2426438|AAAAAAAAGEGAFCAA|1931-04-06|375|1631|126|1931|1|4|6|2|1931|126|1631|Monday|1931Q2|N|N|N|2426433|2426522|2426073|2426348|N|N|N|N|N| +2426439|AAAAAAAAHEGAFCAA|1931-04-07|375|1632|126|1931|2|4|7|2|1931|126|1632|Tuesday|1931Q2|N|N|N|2426433|2426522|2426074|2426349|N|N|N|N|N| +2426440|AAAAAAAAIEGAFCAA|1931-04-08|375|1632|126|1931|3|4|8|2|1931|126|1632|Wednesday|1931Q2|N|N|N|2426433|2426522|2426075|2426350|N|N|N|N|N| +2426441|AAAAAAAAJEGAFCAA|1931-04-09|375|1632|126|1931|4|4|9|2|1931|126|1632|Thursday|1931Q2|N|N|N|2426433|2426522|2426076|2426351|N|N|N|N|N| +2426442|AAAAAAAAKEGAFCAA|1931-04-10|375|1632|126|1931|5|4|10|2|1931|126|1632|Friday|1931Q2|N|Y|N|2426433|2426522|2426077|2426352|N|N|N|N|N| +2426443|AAAAAAAALEGAFCAA|1931-04-11|375|1632|126|1931|6|4|11|2|1931|126|1632|Saturday|1931Q2|N|Y|N|2426433|2426522|2426078|2426353|N|N|N|N|N| +2426444|AAAAAAAAMEGAFCAA|1931-04-12|375|1632|126|1931|0|4|12|2|1931|126|1632|Sunday|1931Q2|N|N|N|2426433|2426522|2426079|2426354|N|N|N|N|N| +2426445|AAAAAAAANEGAFCAA|1931-04-13|375|1632|126|1931|1|4|13|2|1931|126|1632|Monday|1931Q2|N|N|N|2426433|2426522|2426080|2426355|N|N|N|N|N| +2426446|AAAAAAAAOEGAFCAA|1931-04-14|375|1633|126|1931|2|4|14|2|1931|126|1633|Tuesday|1931Q2|N|N|N|2426433|2426522|2426081|2426356|N|N|N|N|N| +2426447|AAAAAAAAPEGAFCAA|1931-04-15|375|1633|126|1931|3|4|15|2|1931|126|1633|Wednesday|1931Q2|N|N|N|2426433|2426522|2426082|2426357|N|N|N|N|N| +2426448|AAAAAAAAAFGAFCAA|1931-04-16|375|1633|126|1931|4|4|16|2|1931|126|1633|Thursday|1931Q2|N|N|N|2426433|2426522|2426083|2426358|N|N|N|N|N| +2426449|AAAAAAAABFGAFCAA|1931-04-17|375|1633|126|1931|5|4|17|2|1931|126|1633|Friday|1931Q2|N|Y|N|2426433|2426522|2426084|2426359|N|N|N|N|N| +2426450|AAAAAAAACFGAFCAA|1931-04-18|375|1633|126|1931|6|4|18|2|1931|126|1633|Saturday|1931Q2|N|Y|N|2426433|2426522|2426085|2426360|N|N|N|N|N| +2426451|AAAAAAAADFGAFCAA|1931-04-19|375|1633|126|1931|0|4|19|2|1931|126|1633|Sunday|1931Q2|N|N|N|2426433|2426522|2426086|2426361|N|N|N|N|N| +2426452|AAAAAAAAEFGAFCAA|1931-04-20|375|1633|126|1931|1|4|20|2|1931|126|1633|Monday|1931Q2|N|N|N|2426433|2426522|2426087|2426362|N|N|N|N|N| +2426453|AAAAAAAAFFGAFCAA|1931-04-21|375|1634|126|1931|2|4|21|2|1931|126|1634|Tuesday|1931Q2|N|N|N|2426433|2426522|2426088|2426363|N|N|N|N|N| +2426454|AAAAAAAAGFGAFCAA|1931-04-22|375|1634|126|1931|3|4|22|2|1931|126|1634|Wednesday|1931Q2|N|N|N|2426433|2426522|2426089|2426364|N|N|N|N|N| +2426455|AAAAAAAAHFGAFCAA|1931-04-23|375|1634|126|1931|4|4|23|2|1931|126|1634|Thursday|1931Q2|N|N|N|2426433|2426522|2426090|2426365|N|N|N|N|N| +2426456|AAAAAAAAIFGAFCAA|1931-04-24|375|1634|126|1931|5|4|24|2|1931|126|1634|Friday|1931Q2|N|Y|N|2426433|2426522|2426091|2426366|N|N|N|N|N| +2426457|AAAAAAAAJFGAFCAA|1931-04-25|375|1634|126|1931|6|4|25|2|1931|126|1634|Saturday|1931Q2|N|Y|N|2426433|2426522|2426092|2426367|N|N|N|N|N| +2426458|AAAAAAAAKFGAFCAA|1931-04-26|375|1634|126|1931|0|4|26|2|1931|126|1634|Sunday|1931Q2|N|N|N|2426433|2426522|2426093|2426368|N|N|N|N|N| +2426459|AAAAAAAALFGAFCAA|1931-04-27|375|1634|126|1931|1|4|27|2|1931|126|1634|Monday|1931Q2|N|N|N|2426433|2426522|2426094|2426369|N|N|N|N|N| +2426460|AAAAAAAAMFGAFCAA|1931-04-28|375|1635|126|1931|2|4|28|2|1931|126|1635|Tuesday|1931Q2|N|N|N|2426433|2426522|2426095|2426370|N|N|N|N|N| +2426461|AAAAAAAANFGAFCAA|1931-04-29|375|1635|126|1931|3|4|29|2|1931|126|1635|Wednesday|1931Q2|N|N|N|2426433|2426522|2426096|2426371|N|N|N|N|N| +2426462|AAAAAAAAOFGAFCAA|1931-04-30|375|1635|126|1931|4|4|30|2|1931|126|1635|Thursday|1931Q2|N|N|N|2426433|2426522|2426097|2426372|N|N|N|N|N| +2426463|AAAAAAAAPFGAFCAA|1931-05-01|376|1635|126|1931|5|5|1|2|1931|126|1635|Friday|1931Q2|N|Y|N|2426463|2426582|2426098|2426373|N|N|N|N|N| +2426464|AAAAAAAAAGGAFCAA|1931-05-02|376|1635|126|1931|6|5|2|2|1931|126|1635|Saturday|1931Q2|N|Y|N|2426463|2426582|2426099|2426374|N|N|N|N|N| +2426465|AAAAAAAABGGAFCAA|1931-05-03|376|1635|126|1931|0|5|3|2|1931|126|1635|Sunday|1931Q2|N|N|N|2426463|2426582|2426100|2426375|N|N|N|N|N| +2426466|AAAAAAAACGGAFCAA|1931-05-04|376|1635|126|1931|1|5|4|2|1931|126|1635|Monday|1931Q2|N|N|N|2426463|2426582|2426101|2426376|N|N|N|N|N| +2426467|AAAAAAAADGGAFCAA|1931-05-05|376|1636|126|1931|2|5|5|2|1931|126|1636|Tuesday|1931Q2|N|N|N|2426463|2426582|2426102|2426377|N|N|N|N|N| +2426468|AAAAAAAAEGGAFCAA|1931-05-06|376|1636|126|1931|3|5|6|2|1931|126|1636|Wednesday|1931Q2|N|N|N|2426463|2426582|2426103|2426378|N|N|N|N|N| +2426469|AAAAAAAAFGGAFCAA|1931-05-07|376|1636|126|1931|4|5|7|2|1931|126|1636|Thursday|1931Q2|N|N|N|2426463|2426582|2426104|2426379|N|N|N|N|N| +2426470|AAAAAAAAGGGAFCAA|1931-05-08|376|1636|126|1931|5|5|8|2|1931|126|1636|Friday|1931Q2|N|Y|N|2426463|2426582|2426105|2426380|N|N|N|N|N| +2426471|AAAAAAAAHGGAFCAA|1931-05-09|376|1636|126|1931|6|5|9|2|1931|126|1636|Saturday|1931Q2|N|Y|N|2426463|2426582|2426106|2426381|N|N|N|N|N| +2426472|AAAAAAAAIGGAFCAA|1931-05-10|376|1636|126|1931|0|5|10|2|1931|126|1636|Sunday|1931Q2|N|N|N|2426463|2426582|2426107|2426382|N|N|N|N|N| +2426473|AAAAAAAAJGGAFCAA|1931-05-11|376|1636|126|1931|1|5|11|2|1931|126|1636|Monday|1931Q2|N|N|N|2426463|2426582|2426108|2426383|N|N|N|N|N| +2426474|AAAAAAAAKGGAFCAA|1931-05-12|376|1637|126|1931|2|5|12|2|1931|126|1637|Tuesday|1931Q2|N|N|N|2426463|2426582|2426109|2426384|N|N|N|N|N| +2426475|AAAAAAAALGGAFCAA|1931-05-13|376|1637|126|1931|3|5|13|2|1931|126|1637|Wednesday|1931Q2|N|N|N|2426463|2426582|2426110|2426385|N|N|N|N|N| +2426476|AAAAAAAAMGGAFCAA|1931-05-14|376|1637|126|1931|4|5|14|2|1931|126|1637|Thursday|1931Q2|N|N|N|2426463|2426582|2426111|2426386|N|N|N|N|N| +2426477|AAAAAAAANGGAFCAA|1931-05-15|376|1637|126|1931|5|5|15|2|1931|126|1637|Friday|1931Q2|N|Y|N|2426463|2426582|2426112|2426387|N|N|N|N|N| +2426478|AAAAAAAAOGGAFCAA|1931-05-16|376|1637|126|1931|6|5|16|2|1931|126|1637|Saturday|1931Q2|N|Y|N|2426463|2426582|2426113|2426388|N|N|N|N|N| +2426479|AAAAAAAAPGGAFCAA|1931-05-17|376|1637|126|1931|0|5|17|2|1931|126|1637|Sunday|1931Q2|N|N|N|2426463|2426582|2426114|2426389|N|N|N|N|N| +2426480|AAAAAAAAAHGAFCAA|1931-05-18|376|1637|126|1931|1|5|18|2|1931|126|1637|Monday|1931Q2|N|N|N|2426463|2426582|2426115|2426390|N|N|N|N|N| +2426481|AAAAAAAABHGAFCAA|1931-05-19|376|1638|126|1931|2|5|19|2|1931|126|1638|Tuesday|1931Q2|N|N|N|2426463|2426582|2426116|2426391|N|N|N|N|N| +2426482|AAAAAAAACHGAFCAA|1931-05-20|376|1638|126|1931|3|5|20|2|1931|126|1638|Wednesday|1931Q2|N|N|N|2426463|2426582|2426117|2426392|N|N|N|N|N| +2426483|AAAAAAAADHGAFCAA|1931-05-21|376|1638|126|1931|4|5|21|2|1931|126|1638|Thursday|1931Q2|N|N|N|2426463|2426582|2426118|2426393|N|N|N|N|N| +2426484|AAAAAAAAEHGAFCAA|1931-05-22|376|1638|126|1931|5|5|22|2|1931|126|1638|Friday|1931Q2|N|Y|N|2426463|2426582|2426119|2426394|N|N|N|N|N| +2426485|AAAAAAAAFHGAFCAA|1931-05-23|376|1638|126|1931|6|5|23|2|1931|126|1638|Saturday|1931Q2|N|Y|N|2426463|2426582|2426120|2426395|N|N|N|N|N| +2426486|AAAAAAAAGHGAFCAA|1931-05-24|376|1638|126|1931|0|5|24|2|1931|126|1638|Sunday|1931Q2|N|N|N|2426463|2426582|2426121|2426396|N|N|N|N|N| +2426487|AAAAAAAAHHGAFCAA|1931-05-25|376|1638|126|1931|1|5|25|2|1931|126|1638|Monday|1931Q2|N|N|N|2426463|2426582|2426122|2426397|N|N|N|N|N| +2426488|AAAAAAAAIHGAFCAA|1931-05-26|376|1639|126|1931|2|5|26|2|1931|126|1639|Tuesday|1931Q2|N|N|N|2426463|2426582|2426123|2426398|N|N|N|N|N| +2426489|AAAAAAAAJHGAFCAA|1931-05-27|376|1639|126|1931|3|5|27|2|1931|126|1639|Wednesday|1931Q2|N|N|N|2426463|2426582|2426124|2426399|N|N|N|N|N| +2426490|AAAAAAAAKHGAFCAA|1931-05-28|376|1639|126|1931|4|5|28|2|1931|126|1639|Thursday|1931Q2|N|N|N|2426463|2426582|2426125|2426400|N|N|N|N|N| +2426491|AAAAAAAALHGAFCAA|1931-05-29|376|1639|126|1931|5|5|29|2|1931|126|1639|Friday|1931Q2|N|Y|N|2426463|2426582|2426126|2426401|N|N|N|N|N| +2426492|AAAAAAAAMHGAFCAA|1931-05-30|376|1639|126|1931|6|5|30|2|1931|126|1639|Saturday|1931Q2|N|Y|N|2426463|2426582|2426127|2426402|N|N|N|N|N| +2426493|AAAAAAAANHGAFCAA|1931-05-31|376|1639|126|1931|0|5|31|2|1931|126|1639|Sunday|1931Q2|N|N|N|2426463|2426582|2426128|2426403|N|N|N|N|N| +2426494|AAAAAAAAOHGAFCAA|1931-06-01|377|1639|127|1931|1|6|1|2|1931|127|1639|Monday|1931Q2|N|N|N|2426494|2426644|2426129|2426404|N|N|N|N|N| +2426495|AAAAAAAAPHGAFCAA|1931-06-02|377|1640|127|1931|2|6|2|2|1931|127|1640|Tuesday|1931Q2|N|N|N|2426494|2426644|2426130|2426405|N|N|N|N|N| +2426496|AAAAAAAAAIGAFCAA|1931-06-03|377|1640|127|1931|3|6|3|2|1931|127|1640|Wednesday|1931Q2|N|N|N|2426494|2426644|2426131|2426406|N|N|N|N|N| +2426497|AAAAAAAABIGAFCAA|1931-06-04|377|1640|127|1931|4|6|4|2|1931|127|1640|Thursday|1931Q2|N|N|N|2426494|2426644|2426132|2426407|N|N|N|N|N| +2426498|AAAAAAAACIGAFCAA|1931-06-05|377|1640|127|1931|5|6|5|2|1931|127|1640|Friday|1931Q2|N|Y|N|2426494|2426644|2426133|2426408|N|N|N|N|N| +2426499|AAAAAAAADIGAFCAA|1931-06-06|377|1640|127|1931|6|6|6|2|1931|127|1640|Saturday|1931Q2|N|Y|N|2426494|2426644|2426134|2426409|N|N|N|N|N| +2426500|AAAAAAAAEIGAFCAA|1931-06-07|377|1640|127|1931|0|6|7|2|1931|127|1640|Sunday|1931Q2|N|N|N|2426494|2426644|2426135|2426410|N|N|N|N|N| +2426501|AAAAAAAAFIGAFCAA|1931-06-08|377|1640|127|1931|1|6|8|2|1931|127|1640|Monday|1931Q2|N|N|N|2426494|2426644|2426136|2426411|N|N|N|N|N| +2426502|AAAAAAAAGIGAFCAA|1931-06-09|377|1641|127|1931|2|6|9|2|1931|127|1641|Tuesday|1931Q2|N|N|N|2426494|2426644|2426137|2426412|N|N|N|N|N| +2426503|AAAAAAAAHIGAFCAA|1931-06-10|377|1641|127|1931|3|6|10|2|1931|127|1641|Wednesday|1931Q2|N|N|N|2426494|2426644|2426138|2426413|N|N|N|N|N| +2426504|AAAAAAAAIIGAFCAA|1931-06-11|377|1641|127|1931|4|6|11|2|1931|127|1641|Thursday|1931Q2|N|N|N|2426494|2426644|2426139|2426414|N|N|N|N|N| +2426505|AAAAAAAAJIGAFCAA|1931-06-12|377|1641|127|1931|5|6|12|2|1931|127|1641|Friday|1931Q2|N|Y|N|2426494|2426644|2426140|2426415|N|N|N|N|N| +2426506|AAAAAAAAKIGAFCAA|1931-06-13|377|1641|127|1931|6|6|13|2|1931|127|1641|Saturday|1931Q2|N|Y|N|2426494|2426644|2426141|2426416|N|N|N|N|N| +2426507|AAAAAAAALIGAFCAA|1931-06-14|377|1641|127|1931|0|6|14|2|1931|127|1641|Sunday|1931Q2|N|N|N|2426494|2426644|2426142|2426417|N|N|N|N|N| +2426508|AAAAAAAAMIGAFCAA|1931-06-15|377|1641|127|1931|1|6|15|2|1931|127|1641|Monday|1931Q2|N|N|N|2426494|2426644|2426143|2426418|N|N|N|N|N| +2426509|AAAAAAAANIGAFCAA|1931-06-16|377|1642|127|1931|2|6|16|2|1931|127|1642|Tuesday|1931Q2|N|N|N|2426494|2426644|2426144|2426419|N|N|N|N|N| +2426510|AAAAAAAAOIGAFCAA|1931-06-17|377|1642|127|1931|3|6|17|2|1931|127|1642|Wednesday|1931Q2|N|N|N|2426494|2426644|2426145|2426420|N|N|N|N|N| +2426511|AAAAAAAAPIGAFCAA|1931-06-18|377|1642|127|1931|4|6|18|2|1931|127|1642|Thursday|1931Q2|N|N|N|2426494|2426644|2426146|2426421|N|N|N|N|N| +2426512|AAAAAAAAAJGAFCAA|1931-06-19|377|1642|127|1931|5|6|19|2|1931|127|1642|Friday|1931Q2|N|Y|N|2426494|2426644|2426147|2426422|N|N|N|N|N| +2426513|AAAAAAAABJGAFCAA|1931-06-20|377|1642|127|1931|6|6|20|2|1931|127|1642|Saturday|1931Q2|N|Y|N|2426494|2426644|2426148|2426423|N|N|N|N|N| +2426514|AAAAAAAACJGAFCAA|1931-06-21|377|1642|127|1931|0|6|21|2|1931|127|1642|Sunday|1931Q2|N|N|N|2426494|2426644|2426149|2426424|N|N|N|N|N| +2426515|AAAAAAAADJGAFCAA|1931-06-22|377|1642|127|1931|1|6|22|2|1931|127|1642|Monday|1931Q2|N|N|N|2426494|2426644|2426150|2426425|N|N|N|N|N| +2426516|AAAAAAAAEJGAFCAA|1931-06-23|377|1643|127|1931|2|6|23|2|1931|127|1643|Tuesday|1931Q2|N|N|N|2426494|2426644|2426151|2426426|N|N|N|N|N| +2426517|AAAAAAAAFJGAFCAA|1931-06-24|377|1643|127|1931|3|6|24|2|1931|127|1643|Wednesday|1931Q2|N|N|N|2426494|2426644|2426152|2426427|N|N|N|N|N| +2426518|AAAAAAAAGJGAFCAA|1931-06-25|377|1643|127|1931|4|6|25|2|1931|127|1643|Thursday|1931Q2|N|N|N|2426494|2426644|2426153|2426428|N|N|N|N|N| +2426519|AAAAAAAAHJGAFCAA|1931-06-26|377|1643|127|1931|5|6|26|2|1931|127|1643|Friday|1931Q2|N|Y|N|2426494|2426644|2426154|2426429|N|N|N|N|N| +2426520|AAAAAAAAIJGAFCAA|1931-06-27|377|1643|127|1931|6|6|27|2|1931|127|1643|Saturday|1931Q2|N|Y|N|2426494|2426644|2426155|2426430|N|N|N|N|N| +2426521|AAAAAAAAJJGAFCAA|1931-06-28|377|1643|127|1931|0|6|28|2|1931|127|1643|Sunday|1931Q2|N|N|N|2426494|2426644|2426156|2426431|N|N|N|N|N| +2426522|AAAAAAAAKJGAFCAA|1931-06-29|377|1643|127|1931|1|6|29|2|1931|127|1643|Monday|1931Q2|N|N|N|2426494|2426644|2426157|2426432|N|N|N|N|N| +2426523|AAAAAAAALJGAFCAA|1931-06-30|377|1644|127|1931|2|6|30|2|1931|127|1644|Tuesday|1931Q2|N|N|N|2426494|2426644|2426158|2426433|N|N|N|N|N| +2426524|AAAAAAAAMJGAFCAA|1931-07-01|378|1644|127|1931|3|7|1|2|1931|127|1644|Wednesday|1931Q2|N|N|N|2426524|2426704|2426159|2426433|N|N|N|N|N| +2426525|AAAAAAAANJGAFCAA|1931-07-02|378|1644|127|1931|4|7|2|3|1931|127|1644|Thursday|1931Q3|N|N|N|2426524|2426704|2426160|2426434|N|N|N|N|N| +2426526|AAAAAAAAOJGAFCAA|1931-07-03|378|1644|127|1931|5|7|3|3|1931|127|1644|Friday|1931Q3|N|Y|N|2426524|2426704|2426161|2426435|N|N|N|N|N| +2426527|AAAAAAAAPJGAFCAA|1931-07-04|378|1644|127|1931|6|7|4|3|1931|127|1644|Saturday|1931Q3|N|Y|N|2426524|2426704|2426162|2426436|N|N|N|N|N| +2426528|AAAAAAAAAKGAFCAA|1931-07-05|378|1644|127|1931|0|7|5|3|1931|127|1644|Sunday|1931Q3|Y|N|N|2426524|2426704|2426163|2426437|N|N|N|N|N| +2426529|AAAAAAAABKGAFCAA|1931-07-06|378|1644|127|1931|1|7|6|3|1931|127|1644|Monday|1931Q3|N|N|Y|2426524|2426704|2426164|2426438|N|N|N|N|N| +2426530|AAAAAAAACKGAFCAA|1931-07-07|378|1645|127|1931|2|7|7|3|1931|127|1645|Tuesday|1931Q3|N|N|N|2426524|2426704|2426165|2426439|N|N|N|N|N| +2426531|AAAAAAAADKGAFCAA|1931-07-08|378|1645|127|1931|3|7|8|3|1931|127|1645|Wednesday|1931Q3|N|N|N|2426524|2426704|2426166|2426440|N|N|N|N|N| +2426532|AAAAAAAAEKGAFCAA|1931-07-09|378|1645|127|1931|4|7|9|3|1931|127|1645|Thursday|1931Q3|N|N|N|2426524|2426704|2426167|2426441|N|N|N|N|N| +2426533|AAAAAAAAFKGAFCAA|1931-07-10|378|1645|127|1931|5|7|10|3|1931|127|1645|Friday|1931Q3|N|Y|N|2426524|2426704|2426168|2426442|N|N|N|N|N| +2426534|AAAAAAAAGKGAFCAA|1931-07-11|378|1645|127|1931|6|7|11|3|1931|127|1645|Saturday|1931Q3|N|Y|N|2426524|2426704|2426169|2426443|N|N|N|N|N| +2426535|AAAAAAAAHKGAFCAA|1931-07-12|378|1645|127|1931|0|7|12|3|1931|127|1645|Sunday|1931Q3|N|N|N|2426524|2426704|2426170|2426444|N|N|N|N|N| +2426536|AAAAAAAAIKGAFCAA|1931-07-13|378|1645|127|1931|1|7|13|3|1931|127|1645|Monday|1931Q3|N|N|N|2426524|2426704|2426171|2426445|N|N|N|N|N| +2426537|AAAAAAAAJKGAFCAA|1931-07-14|378|1646|127|1931|2|7|14|3|1931|127|1646|Tuesday|1931Q3|N|N|N|2426524|2426704|2426172|2426446|N|N|N|N|N| +2426538|AAAAAAAAKKGAFCAA|1931-07-15|378|1646|127|1931|3|7|15|3|1931|127|1646|Wednesday|1931Q3|N|N|N|2426524|2426704|2426173|2426447|N|N|N|N|N| +2426539|AAAAAAAALKGAFCAA|1931-07-16|378|1646|127|1931|4|7|16|3|1931|127|1646|Thursday|1931Q3|N|N|N|2426524|2426704|2426174|2426448|N|N|N|N|N| +2426540|AAAAAAAAMKGAFCAA|1931-07-17|378|1646|127|1931|5|7|17|3|1931|127|1646|Friday|1931Q3|N|Y|N|2426524|2426704|2426175|2426449|N|N|N|N|N| +2426541|AAAAAAAANKGAFCAA|1931-07-18|378|1646|127|1931|6|7|18|3|1931|127|1646|Saturday|1931Q3|N|Y|N|2426524|2426704|2426176|2426450|N|N|N|N|N| +2426542|AAAAAAAAOKGAFCAA|1931-07-19|378|1646|127|1931|0|7|19|3|1931|127|1646|Sunday|1931Q3|N|N|N|2426524|2426704|2426177|2426451|N|N|N|N|N| +2426543|AAAAAAAAPKGAFCAA|1931-07-20|378|1646|127|1931|1|7|20|3|1931|127|1646|Monday|1931Q3|N|N|N|2426524|2426704|2426178|2426452|N|N|N|N|N| +2426544|AAAAAAAAALGAFCAA|1931-07-21|378|1647|127|1931|2|7|21|3|1931|127|1647|Tuesday|1931Q3|N|N|N|2426524|2426704|2426179|2426453|N|N|N|N|N| +2426545|AAAAAAAABLGAFCAA|1931-07-22|378|1647|127|1931|3|7|22|3|1931|127|1647|Wednesday|1931Q3|N|N|N|2426524|2426704|2426180|2426454|N|N|N|N|N| +2426546|AAAAAAAACLGAFCAA|1931-07-23|378|1647|127|1931|4|7|23|3|1931|127|1647|Thursday|1931Q3|N|N|N|2426524|2426704|2426181|2426455|N|N|N|N|N| +2426547|AAAAAAAADLGAFCAA|1931-07-24|378|1647|127|1931|5|7|24|3|1931|127|1647|Friday|1931Q3|N|Y|N|2426524|2426704|2426182|2426456|N|N|N|N|N| +2426548|AAAAAAAAELGAFCAA|1931-07-25|378|1647|127|1931|6|7|25|3|1931|127|1647|Saturday|1931Q3|N|Y|N|2426524|2426704|2426183|2426457|N|N|N|N|N| +2426549|AAAAAAAAFLGAFCAA|1931-07-26|378|1647|127|1931|0|7|26|3|1931|127|1647|Sunday|1931Q3|N|N|N|2426524|2426704|2426184|2426458|N|N|N|N|N| +2426550|AAAAAAAAGLGAFCAA|1931-07-27|378|1647|127|1931|1|7|27|3|1931|127|1647|Monday|1931Q3|N|N|N|2426524|2426704|2426185|2426459|N|N|N|N|N| +2426551|AAAAAAAAHLGAFCAA|1931-07-28|378|1648|127|1931|2|7|28|3|1931|127|1648|Tuesday|1931Q3|N|N|N|2426524|2426704|2426186|2426460|N|N|N|N|N| +2426552|AAAAAAAAILGAFCAA|1931-07-29|378|1648|127|1931|3|7|29|3|1931|127|1648|Wednesday|1931Q3|N|N|N|2426524|2426704|2426187|2426461|N|N|N|N|N| +2426553|AAAAAAAAJLGAFCAA|1931-07-30|378|1648|127|1931|4|7|30|3|1931|127|1648|Thursday|1931Q3|N|N|N|2426524|2426704|2426188|2426462|N|N|N|N|N| +2426554|AAAAAAAAKLGAFCAA|1931-07-31|378|1648|127|1931|5|7|31|3|1931|127|1648|Friday|1931Q3|N|Y|N|2426524|2426704|2426189|2426463|N|N|N|N|N| +2426555|AAAAAAAALLGAFCAA|1931-08-01|379|1648|127|1931|6|8|1|3|1931|127|1648|Saturday|1931Q3|N|Y|N|2426555|2426766|2426190|2426464|N|N|N|N|N| +2426556|AAAAAAAAMLGAFCAA|1931-08-02|379|1648|127|1931|0|8|2|3|1931|127|1648|Sunday|1931Q3|N|N|N|2426555|2426766|2426191|2426465|N|N|N|N|N| +2426557|AAAAAAAANLGAFCAA|1931-08-03|379|1648|127|1931|1|8|3|3|1931|127|1648|Monday|1931Q3|N|N|N|2426555|2426766|2426192|2426466|N|N|N|N|N| +2426558|AAAAAAAAOLGAFCAA|1931-08-04|379|1649|127|1931|2|8|4|3|1931|127|1649|Tuesday|1931Q3|N|N|N|2426555|2426766|2426193|2426467|N|N|N|N|N| +2426559|AAAAAAAAPLGAFCAA|1931-08-05|379|1649|127|1931|3|8|5|3|1931|127|1649|Wednesday|1931Q3|N|N|N|2426555|2426766|2426194|2426468|N|N|N|N|N| +2426560|AAAAAAAAAMGAFCAA|1931-08-06|379|1649|127|1931|4|8|6|3|1931|127|1649|Thursday|1931Q3|N|N|N|2426555|2426766|2426195|2426469|N|N|N|N|N| +2426561|AAAAAAAABMGAFCAA|1931-08-07|379|1649|127|1931|5|8|7|3|1931|127|1649|Friday|1931Q3|N|Y|N|2426555|2426766|2426196|2426470|N|N|N|N|N| +2426562|AAAAAAAACMGAFCAA|1931-08-08|379|1649|127|1931|6|8|8|3|1931|127|1649|Saturday|1931Q3|N|Y|N|2426555|2426766|2426197|2426471|N|N|N|N|N| +2426563|AAAAAAAADMGAFCAA|1931-08-09|379|1649|127|1931|0|8|9|3|1931|127|1649|Sunday|1931Q3|N|N|N|2426555|2426766|2426198|2426472|N|N|N|N|N| +2426564|AAAAAAAAEMGAFCAA|1931-08-10|379|1649|127|1931|1|8|10|3|1931|127|1649|Monday|1931Q3|N|N|N|2426555|2426766|2426199|2426473|N|N|N|N|N| +2426565|AAAAAAAAFMGAFCAA|1931-08-11|379|1650|127|1931|2|8|11|3|1931|127|1650|Tuesday|1931Q3|N|N|N|2426555|2426766|2426200|2426474|N|N|N|N|N| +2426566|AAAAAAAAGMGAFCAA|1931-08-12|379|1650|127|1931|3|8|12|3|1931|127|1650|Wednesday|1931Q3|N|N|N|2426555|2426766|2426201|2426475|N|N|N|N|N| +2426567|AAAAAAAAHMGAFCAA|1931-08-13|379|1650|127|1931|4|8|13|3|1931|127|1650|Thursday|1931Q3|N|N|N|2426555|2426766|2426202|2426476|N|N|N|N|N| +2426568|AAAAAAAAIMGAFCAA|1931-08-14|379|1650|127|1931|5|8|14|3|1931|127|1650|Friday|1931Q3|N|Y|N|2426555|2426766|2426203|2426477|N|N|N|N|N| +2426569|AAAAAAAAJMGAFCAA|1931-08-15|379|1650|127|1931|6|8|15|3|1931|127|1650|Saturday|1931Q3|N|Y|N|2426555|2426766|2426204|2426478|N|N|N|N|N| +2426570|AAAAAAAAKMGAFCAA|1931-08-16|379|1650|127|1931|0|8|16|3|1931|127|1650|Sunday|1931Q3|N|N|N|2426555|2426766|2426205|2426479|N|N|N|N|N| +2426571|AAAAAAAALMGAFCAA|1931-08-17|379|1650|127|1931|1|8|17|3|1931|127|1650|Monday|1931Q3|N|N|N|2426555|2426766|2426206|2426480|N|N|N|N|N| +2426572|AAAAAAAAMMGAFCAA|1931-08-18|379|1651|127|1931|2|8|18|3|1931|127|1651|Tuesday|1931Q3|N|N|N|2426555|2426766|2426207|2426481|N|N|N|N|N| +2426573|AAAAAAAANMGAFCAA|1931-08-19|379|1651|127|1931|3|8|19|3|1931|127|1651|Wednesday|1931Q3|N|N|N|2426555|2426766|2426208|2426482|N|N|N|N|N| +2426574|AAAAAAAAOMGAFCAA|1931-08-20|379|1651|127|1931|4|8|20|3|1931|127|1651|Thursday|1931Q3|N|N|N|2426555|2426766|2426209|2426483|N|N|N|N|N| +2426575|AAAAAAAAPMGAFCAA|1931-08-21|379|1651|127|1931|5|8|21|3|1931|127|1651|Friday|1931Q3|N|Y|N|2426555|2426766|2426210|2426484|N|N|N|N|N| +2426576|AAAAAAAAANGAFCAA|1931-08-22|379|1651|127|1931|6|8|22|3|1931|127|1651|Saturday|1931Q3|N|Y|N|2426555|2426766|2426211|2426485|N|N|N|N|N| +2426577|AAAAAAAABNGAFCAA|1931-08-23|379|1651|127|1931|0|8|23|3|1931|127|1651|Sunday|1931Q3|N|N|N|2426555|2426766|2426212|2426486|N|N|N|N|N| +2426578|AAAAAAAACNGAFCAA|1931-08-24|379|1651|127|1931|1|8|24|3|1931|127|1651|Monday|1931Q3|N|N|N|2426555|2426766|2426213|2426487|N|N|N|N|N| +2426579|AAAAAAAADNGAFCAA|1931-08-25|379|1652|127|1931|2|8|25|3|1931|127|1652|Tuesday|1931Q3|N|N|N|2426555|2426766|2426214|2426488|N|N|N|N|N| +2426580|AAAAAAAAENGAFCAA|1931-08-26|379|1652|127|1931|3|8|26|3|1931|127|1652|Wednesday|1931Q3|N|N|N|2426555|2426766|2426215|2426489|N|N|N|N|N| +2426581|AAAAAAAAFNGAFCAA|1931-08-27|379|1652|127|1931|4|8|27|3|1931|127|1652|Thursday|1931Q3|N|N|N|2426555|2426766|2426216|2426490|N|N|N|N|N| +2426582|AAAAAAAAGNGAFCAA|1931-08-28|379|1652|127|1931|5|8|28|3|1931|127|1652|Friday|1931Q3|N|Y|N|2426555|2426766|2426217|2426491|N|N|N|N|N| +2426583|AAAAAAAAHNGAFCAA|1931-08-29|379|1652|127|1931|6|8|29|3|1931|127|1652|Saturday|1931Q3|N|Y|N|2426555|2426766|2426218|2426492|N|N|N|N|N| +2426584|AAAAAAAAINGAFCAA|1931-08-30|379|1652|127|1931|0|8|30|3|1931|127|1652|Sunday|1931Q3|N|N|N|2426555|2426766|2426219|2426493|N|N|N|N|N| +2426585|AAAAAAAAJNGAFCAA|1931-08-31|379|1652|127|1931|1|8|31|3|1931|127|1652|Monday|1931Q3|N|N|N|2426555|2426766|2426220|2426494|N|N|N|N|N| +2426586|AAAAAAAAKNGAFCAA|1931-09-01|380|1653|128|1931|2|9|1|3|1931|128|1653|Tuesday|1931Q3|N|N|N|2426586|2426828|2426221|2426495|N|N|N|N|N| +2426587|AAAAAAAALNGAFCAA|1931-09-02|380|1653|128|1931|3|9|2|3|1931|128|1653|Wednesday|1931Q3|N|N|N|2426586|2426828|2426222|2426496|N|N|N|N|N| +2426588|AAAAAAAAMNGAFCAA|1931-09-03|380|1653|128|1931|4|9|3|3|1931|128|1653|Thursday|1931Q3|N|N|N|2426586|2426828|2426223|2426497|N|N|N|N|N| +2426589|AAAAAAAANNGAFCAA|1931-09-04|380|1653|128|1931|5|9|4|3|1931|128|1653|Friday|1931Q3|N|Y|N|2426586|2426828|2426224|2426498|N|N|N|N|N| +2426590|AAAAAAAAONGAFCAA|1931-09-05|380|1653|128|1931|6|9|5|3|1931|128|1653|Saturday|1931Q3|N|Y|N|2426586|2426828|2426225|2426499|N|N|N|N|N| +2426591|AAAAAAAAPNGAFCAA|1931-09-06|380|1653|128|1931|0|9|6|3|1931|128|1653|Sunday|1931Q3|N|N|N|2426586|2426828|2426226|2426500|N|N|N|N|N| +2426592|AAAAAAAAAOGAFCAA|1931-09-07|380|1653|128|1931|1|9|7|3|1931|128|1653|Monday|1931Q3|N|N|N|2426586|2426828|2426227|2426501|N|N|N|N|N| +2426593|AAAAAAAABOGAFCAA|1931-09-08|380|1654|128|1931|2|9|8|3|1931|128|1654|Tuesday|1931Q3|N|N|N|2426586|2426828|2426228|2426502|N|N|N|N|N| +2426594|AAAAAAAACOGAFCAA|1931-09-09|380|1654|128|1931|3|9|9|3|1931|128|1654|Wednesday|1931Q3|N|N|N|2426586|2426828|2426229|2426503|N|N|N|N|N| +2426595|AAAAAAAADOGAFCAA|1931-09-10|380|1654|128|1931|4|9|10|3|1931|128|1654|Thursday|1931Q3|N|N|N|2426586|2426828|2426230|2426504|N|N|N|N|N| +2426596|AAAAAAAAEOGAFCAA|1931-09-11|380|1654|128|1931|5|9|11|3|1931|128|1654|Friday|1931Q3|N|Y|N|2426586|2426828|2426231|2426505|N|N|N|N|N| +2426597|AAAAAAAAFOGAFCAA|1931-09-12|380|1654|128|1931|6|9|12|3|1931|128|1654|Saturday|1931Q3|N|Y|N|2426586|2426828|2426232|2426506|N|N|N|N|N| +2426598|AAAAAAAAGOGAFCAA|1931-09-13|380|1654|128|1931|0|9|13|3|1931|128|1654|Sunday|1931Q3|N|N|N|2426586|2426828|2426233|2426507|N|N|N|N|N| +2426599|AAAAAAAAHOGAFCAA|1931-09-14|380|1654|128|1931|1|9|14|3|1931|128|1654|Monday|1931Q3|N|N|N|2426586|2426828|2426234|2426508|N|N|N|N|N| +2426600|AAAAAAAAIOGAFCAA|1931-09-15|380|1655|128|1931|2|9|15|3|1931|128|1655|Tuesday|1931Q3|N|N|N|2426586|2426828|2426235|2426509|N|N|N|N|N| +2426601|AAAAAAAAJOGAFCAA|1931-09-16|380|1655|128|1931|3|9|16|3|1931|128|1655|Wednesday|1931Q3|N|N|N|2426586|2426828|2426236|2426510|N|N|N|N|N| +2426602|AAAAAAAAKOGAFCAA|1931-09-17|380|1655|128|1931|4|9|17|3|1931|128|1655|Thursday|1931Q3|N|N|N|2426586|2426828|2426237|2426511|N|N|N|N|N| +2426603|AAAAAAAALOGAFCAA|1931-09-18|380|1655|128|1931|5|9|18|3|1931|128|1655|Friday|1931Q3|N|Y|N|2426586|2426828|2426238|2426512|N|N|N|N|N| +2426604|AAAAAAAAMOGAFCAA|1931-09-19|380|1655|128|1931|6|9|19|3|1931|128|1655|Saturday|1931Q3|N|Y|N|2426586|2426828|2426239|2426513|N|N|N|N|N| +2426605|AAAAAAAANOGAFCAA|1931-09-20|380|1655|128|1931|0|9|20|3|1931|128|1655|Sunday|1931Q3|N|N|N|2426586|2426828|2426240|2426514|N|N|N|N|N| +2426606|AAAAAAAAOOGAFCAA|1931-09-21|380|1655|128|1931|1|9|21|3|1931|128|1655|Monday|1931Q3|N|N|N|2426586|2426828|2426241|2426515|N|N|N|N|N| +2426607|AAAAAAAAPOGAFCAA|1931-09-22|380|1656|128|1931|2|9|22|3|1931|128|1656|Tuesday|1931Q3|N|N|N|2426586|2426828|2426242|2426516|N|N|N|N|N| +2426608|AAAAAAAAAPGAFCAA|1931-09-23|380|1656|128|1931|3|9|23|3|1931|128|1656|Wednesday|1931Q3|N|N|N|2426586|2426828|2426243|2426517|N|N|N|N|N| +2426609|AAAAAAAABPGAFCAA|1931-09-24|380|1656|128|1931|4|9|24|3|1931|128|1656|Thursday|1931Q3|N|N|N|2426586|2426828|2426244|2426518|N|N|N|N|N| +2426610|AAAAAAAACPGAFCAA|1931-09-25|380|1656|128|1931|5|9|25|3|1931|128|1656|Friday|1931Q3|N|Y|N|2426586|2426828|2426245|2426519|N|N|N|N|N| +2426611|AAAAAAAADPGAFCAA|1931-09-26|380|1656|128|1931|6|9|26|3|1931|128|1656|Saturday|1931Q3|N|Y|N|2426586|2426828|2426246|2426520|N|N|N|N|N| +2426612|AAAAAAAAEPGAFCAA|1931-09-27|380|1656|128|1931|0|9|27|3|1931|128|1656|Sunday|1931Q3|N|N|N|2426586|2426828|2426247|2426521|N|N|N|N|N| +2426613|AAAAAAAAFPGAFCAA|1931-09-28|380|1656|128|1931|1|9|28|3|1931|128|1656|Monday|1931Q3|N|N|N|2426586|2426828|2426248|2426522|N|N|N|N|N| +2426614|AAAAAAAAGPGAFCAA|1931-09-29|380|1657|128|1931|2|9|29|3|1931|128|1657|Tuesday|1931Q3|N|N|N|2426586|2426828|2426249|2426523|N|N|N|N|N| +2426615|AAAAAAAAHPGAFCAA|1931-09-30|380|1657|128|1931|3|9|30|3|1931|128|1657|Wednesday|1931Q3|N|N|N|2426586|2426828|2426250|2426524|N|N|N|N|N| +2426616|AAAAAAAAIPGAFCAA|1931-10-01|381|1657|128|1931|4|10|1|3|1931|128|1657|Thursday|1931Q3|N|N|N|2426616|2426888|2426251|2426524|N|N|N|N|N| +2426617|AAAAAAAAJPGAFCAA|1931-10-02|381|1657|128|1931|5|10|2|4|1931|128|1657|Friday|1931Q4|N|Y|N|2426616|2426888|2426252|2426525|N|N|N|N|N| +2426618|AAAAAAAAKPGAFCAA|1931-10-03|381|1657|128|1931|6|10|3|4|1931|128|1657|Saturday|1931Q4|N|Y|N|2426616|2426888|2426253|2426526|N|N|N|N|N| +2426619|AAAAAAAALPGAFCAA|1931-10-04|381|1657|128|1931|0|10|4|4|1931|128|1657|Sunday|1931Q4|N|N|N|2426616|2426888|2426254|2426527|N|N|N|N|N| +2426620|AAAAAAAAMPGAFCAA|1931-10-05|381|1657|128|1931|1|10|5|4|1931|128|1657|Monday|1931Q4|N|N|N|2426616|2426888|2426255|2426528|N|N|N|N|N| +2426621|AAAAAAAANPGAFCAA|1931-10-06|381|1658|128|1931|2|10|6|4|1931|128|1658|Tuesday|1931Q4|N|N|N|2426616|2426888|2426256|2426529|N|N|N|N|N| +2426622|AAAAAAAAOPGAFCAA|1931-10-07|381|1658|128|1931|3|10|7|4|1931|128|1658|Wednesday|1931Q4|N|N|N|2426616|2426888|2426257|2426530|N|N|N|N|N| +2426623|AAAAAAAAPPGAFCAA|1931-10-08|381|1658|128|1931|4|10|8|4|1931|128|1658|Thursday|1931Q4|N|N|N|2426616|2426888|2426258|2426531|N|N|N|N|N| +2426624|AAAAAAAAAAHAFCAA|1931-10-09|381|1658|128|1931|5|10|9|4|1931|128|1658|Friday|1931Q4|N|Y|N|2426616|2426888|2426259|2426532|N|N|N|N|N| +2426625|AAAAAAAABAHAFCAA|1931-10-10|381|1658|128|1931|6|10|10|4|1931|128|1658|Saturday|1931Q4|N|Y|N|2426616|2426888|2426260|2426533|N|N|N|N|N| +2426626|AAAAAAAACAHAFCAA|1931-10-11|381|1658|128|1931|0|10|11|4|1931|128|1658|Sunday|1931Q4|N|N|N|2426616|2426888|2426261|2426534|N|N|N|N|N| +2426627|AAAAAAAADAHAFCAA|1931-10-12|381|1658|128|1931|1|10|12|4|1931|128|1658|Monday|1931Q4|N|N|N|2426616|2426888|2426262|2426535|N|N|N|N|N| +2426628|AAAAAAAAEAHAFCAA|1931-10-13|381|1659|128|1931|2|10|13|4|1931|128|1659|Tuesday|1931Q4|N|N|N|2426616|2426888|2426263|2426536|N|N|N|N|N| +2426629|AAAAAAAAFAHAFCAA|1931-10-14|381|1659|128|1931|3|10|14|4|1931|128|1659|Wednesday|1931Q4|N|N|N|2426616|2426888|2426264|2426537|N|N|N|N|N| +2426630|AAAAAAAAGAHAFCAA|1931-10-15|381|1659|128|1931|4|10|15|4|1931|128|1659|Thursday|1931Q4|N|N|N|2426616|2426888|2426265|2426538|N|N|N|N|N| +2426631|AAAAAAAAHAHAFCAA|1931-10-16|381|1659|128|1931|5|10|16|4|1931|128|1659|Friday|1931Q4|N|Y|N|2426616|2426888|2426266|2426539|N|N|N|N|N| +2426632|AAAAAAAAIAHAFCAA|1931-10-17|381|1659|128|1931|6|10|17|4|1931|128|1659|Saturday|1931Q4|N|Y|N|2426616|2426888|2426267|2426540|N|N|N|N|N| +2426633|AAAAAAAAJAHAFCAA|1931-10-18|381|1659|128|1931|0|10|18|4|1931|128|1659|Sunday|1931Q4|N|N|N|2426616|2426888|2426268|2426541|N|N|N|N|N| +2426634|AAAAAAAAKAHAFCAA|1931-10-19|381|1659|128|1931|1|10|19|4|1931|128|1659|Monday|1931Q4|N|N|N|2426616|2426888|2426269|2426542|N|N|N|N|N| +2426635|AAAAAAAALAHAFCAA|1931-10-20|381|1660|128|1931|2|10|20|4|1931|128|1660|Tuesday|1931Q4|N|N|N|2426616|2426888|2426270|2426543|N|N|N|N|N| +2426636|AAAAAAAAMAHAFCAA|1931-10-21|381|1660|128|1931|3|10|21|4|1931|128|1660|Wednesday|1931Q4|N|N|N|2426616|2426888|2426271|2426544|N|N|N|N|N| +2426637|AAAAAAAANAHAFCAA|1931-10-22|381|1660|128|1931|4|10|22|4|1931|128|1660|Thursday|1931Q4|N|N|N|2426616|2426888|2426272|2426545|N|N|N|N|N| +2426638|AAAAAAAAOAHAFCAA|1931-10-23|381|1660|128|1931|5|10|23|4|1931|128|1660|Friday|1931Q4|N|Y|N|2426616|2426888|2426273|2426546|N|N|N|N|N| +2426639|AAAAAAAAPAHAFCAA|1931-10-24|381|1660|128|1931|6|10|24|4|1931|128|1660|Saturday|1931Q4|N|Y|N|2426616|2426888|2426274|2426547|N|N|N|N|N| +2426640|AAAAAAAAABHAFCAA|1931-10-25|381|1660|128|1931|0|10|25|4|1931|128|1660|Sunday|1931Q4|N|N|N|2426616|2426888|2426275|2426548|N|N|N|N|N| +2426641|AAAAAAAABBHAFCAA|1931-10-26|381|1660|128|1931|1|10|26|4|1931|128|1660|Monday|1931Q4|N|N|N|2426616|2426888|2426276|2426549|N|N|N|N|N| +2426642|AAAAAAAACBHAFCAA|1931-10-27|381|1661|128|1931|2|10|27|4|1931|128|1661|Tuesday|1931Q4|N|N|N|2426616|2426888|2426277|2426550|N|N|N|N|N| +2426643|AAAAAAAADBHAFCAA|1931-10-28|381|1661|128|1931|3|10|28|4|1931|128|1661|Wednesday|1931Q4|N|N|N|2426616|2426888|2426278|2426551|N|N|N|N|N| +2426644|AAAAAAAAEBHAFCAA|1931-10-29|381|1661|128|1931|4|10|29|4|1931|128|1661|Thursday|1931Q4|N|N|N|2426616|2426888|2426279|2426552|N|N|N|N|N| +2426645|AAAAAAAAFBHAFCAA|1931-10-30|381|1661|128|1931|5|10|30|4|1931|128|1661|Friday|1931Q4|N|Y|N|2426616|2426888|2426280|2426553|N|N|N|N|N| +2426646|AAAAAAAAGBHAFCAA|1931-10-31|381|1661|128|1931|6|10|31|4|1931|128|1661|Saturday|1931Q4|N|Y|N|2426616|2426888|2426281|2426554|N|N|N|N|N| +2426647|AAAAAAAAHBHAFCAA|1931-11-01|382|1661|128|1931|0|11|1|4|1931|128|1661|Sunday|1931Q4|N|N|N|2426647|2426950|2426282|2426555|N|N|N|N|N| +2426648|AAAAAAAAIBHAFCAA|1931-11-02|382|1661|128|1931|1|11|2|4|1931|128|1661|Monday|1931Q4|N|N|N|2426647|2426950|2426283|2426556|N|N|N|N|N| +2426649|AAAAAAAAJBHAFCAA|1931-11-03|382|1662|128|1931|2|11|3|4|1931|128|1662|Tuesday|1931Q4|N|N|N|2426647|2426950|2426284|2426557|N|N|N|N|N| +2426650|AAAAAAAAKBHAFCAA|1931-11-04|382|1662|128|1931|3|11|4|4|1931|128|1662|Wednesday|1931Q4|N|N|N|2426647|2426950|2426285|2426558|N|N|N|N|N| +2426651|AAAAAAAALBHAFCAA|1931-11-05|382|1662|128|1931|4|11|5|4|1931|128|1662|Thursday|1931Q4|N|N|N|2426647|2426950|2426286|2426559|N|N|N|N|N| +2426652|AAAAAAAAMBHAFCAA|1931-11-06|382|1662|128|1931|5|11|6|4|1931|128|1662|Friday|1931Q4|N|Y|N|2426647|2426950|2426287|2426560|N|N|N|N|N| +2426653|AAAAAAAANBHAFCAA|1931-11-07|382|1662|128|1931|6|11|7|4|1931|128|1662|Saturday|1931Q4|N|Y|N|2426647|2426950|2426288|2426561|N|N|N|N|N| +2426654|AAAAAAAAOBHAFCAA|1931-11-08|382|1662|128|1931|0|11|8|4|1931|128|1662|Sunday|1931Q4|N|N|N|2426647|2426950|2426289|2426562|N|N|N|N|N| +2426655|AAAAAAAAPBHAFCAA|1931-11-09|382|1662|128|1931|1|11|9|4|1931|128|1662|Monday|1931Q4|N|N|N|2426647|2426950|2426290|2426563|N|N|N|N|N| +2426656|AAAAAAAAACHAFCAA|1931-11-10|382|1663|128|1931|2|11|10|4|1931|128|1663|Tuesday|1931Q4|N|N|N|2426647|2426950|2426291|2426564|N|N|N|N|N| +2426657|AAAAAAAABCHAFCAA|1931-11-11|382|1663|128|1931|3|11|11|4|1931|128|1663|Wednesday|1931Q4|N|N|N|2426647|2426950|2426292|2426565|N|N|N|N|N| +2426658|AAAAAAAACCHAFCAA|1931-11-12|382|1663|128|1931|4|11|12|4|1931|128|1663|Thursday|1931Q4|N|N|N|2426647|2426950|2426293|2426566|N|N|N|N|N| +2426659|AAAAAAAADCHAFCAA|1931-11-13|382|1663|128|1931|5|11|13|4|1931|128|1663|Friday|1931Q4|N|Y|N|2426647|2426950|2426294|2426567|N|N|N|N|N| +2426660|AAAAAAAAECHAFCAA|1931-11-14|382|1663|128|1931|6|11|14|4|1931|128|1663|Saturday|1931Q4|N|Y|N|2426647|2426950|2426295|2426568|N|N|N|N|N| +2426661|AAAAAAAAFCHAFCAA|1931-11-15|382|1663|128|1931|0|11|15|4|1931|128|1663|Sunday|1931Q4|N|N|N|2426647|2426950|2426296|2426569|N|N|N|N|N| +2426662|AAAAAAAAGCHAFCAA|1931-11-16|382|1663|128|1931|1|11|16|4|1931|128|1663|Monday|1931Q4|N|N|N|2426647|2426950|2426297|2426570|N|N|N|N|N| +2426663|AAAAAAAAHCHAFCAA|1931-11-17|382|1664|128|1931|2|11|17|4|1931|128|1664|Tuesday|1931Q4|N|N|N|2426647|2426950|2426298|2426571|N|N|N|N|N| +2426664|AAAAAAAAICHAFCAA|1931-11-18|382|1664|128|1931|3|11|18|4|1931|128|1664|Wednesday|1931Q4|N|N|N|2426647|2426950|2426299|2426572|N|N|N|N|N| +2426665|AAAAAAAAJCHAFCAA|1931-11-19|382|1664|128|1931|4|11|19|4|1931|128|1664|Thursday|1931Q4|N|N|N|2426647|2426950|2426300|2426573|N|N|N|N|N| +2426666|AAAAAAAAKCHAFCAA|1931-11-20|382|1664|128|1931|5|11|20|4|1931|128|1664|Friday|1931Q4|N|Y|N|2426647|2426950|2426301|2426574|N|N|N|N|N| +2426667|AAAAAAAALCHAFCAA|1931-11-21|382|1664|128|1931|6|11|21|4|1931|128|1664|Saturday|1931Q4|N|Y|N|2426647|2426950|2426302|2426575|N|N|N|N|N| +2426668|AAAAAAAAMCHAFCAA|1931-11-22|382|1664|128|1931|0|11|22|4|1931|128|1664|Sunday|1931Q4|N|N|N|2426647|2426950|2426303|2426576|N|N|N|N|N| +2426669|AAAAAAAANCHAFCAA|1931-11-23|382|1664|128|1931|1|11|23|4|1931|128|1664|Monday|1931Q4|N|N|N|2426647|2426950|2426304|2426577|N|N|N|N|N| +2426670|AAAAAAAAOCHAFCAA|1931-11-24|382|1665|128|1931|2|11|24|4|1931|128|1665|Tuesday|1931Q4|N|N|N|2426647|2426950|2426305|2426578|N|N|N|N|N| +2426671|AAAAAAAAPCHAFCAA|1931-11-25|382|1665|128|1931|3|11|25|4|1931|128|1665|Wednesday|1931Q4|N|N|N|2426647|2426950|2426306|2426579|N|N|N|N|N| +2426672|AAAAAAAAADHAFCAA|1931-11-26|382|1665|128|1931|4|11|26|4|1931|128|1665|Thursday|1931Q4|N|N|N|2426647|2426950|2426307|2426580|N|N|N|N|N| +2426673|AAAAAAAABDHAFCAA|1931-11-27|382|1665|128|1931|5|11|27|4|1931|128|1665|Friday|1931Q4|N|Y|N|2426647|2426950|2426308|2426581|N|N|N|N|N| +2426674|AAAAAAAACDHAFCAA|1931-11-28|382|1665|128|1931|6|11|28|4|1931|128|1665|Saturday|1931Q4|N|Y|N|2426647|2426950|2426309|2426582|N|N|N|N|N| +2426675|AAAAAAAADDHAFCAA|1931-11-29|382|1665|128|1931|0|11|29|4|1931|128|1665|Sunday|1931Q4|N|N|N|2426647|2426950|2426310|2426583|N|N|N|N|N| +2426676|AAAAAAAAEDHAFCAA|1931-11-30|382|1665|128|1931|1|11|30|4|1931|128|1665|Monday|1931Q4|N|N|N|2426647|2426950|2426311|2426584|N|N|N|N|N| +2426677|AAAAAAAAFDHAFCAA|1931-12-01|383|1666|129|1931|2|12|1|4|1931|129|1666|Tuesday|1931Q4|N|N|N|2426677|2427010|2426312|2426585|N|N|N|N|N| +2426678|AAAAAAAAGDHAFCAA|1931-12-02|383|1666|129|1931|3|12|2|4|1931|129|1666|Wednesday|1931Q4|N|N|N|2426677|2427010|2426313|2426586|N|N|N|N|N| +2426679|AAAAAAAAHDHAFCAA|1931-12-03|383|1666|129|1931|4|12|3|4|1931|129|1666|Thursday|1931Q4|N|N|N|2426677|2427010|2426314|2426587|N|N|N|N|N| +2426680|AAAAAAAAIDHAFCAA|1931-12-04|383|1666|129|1931|5|12|4|4|1931|129|1666|Friday|1931Q4|N|Y|N|2426677|2427010|2426315|2426588|N|N|N|N|N| +2426681|AAAAAAAAJDHAFCAA|1931-12-05|383|1666|129|1931|6|12|5|4|1931|129|1666|Saturday|1931Q4|N|Y|N|2426677|2427010|2426316|2426589|N|N|N|N|N| +2426682|AAAAAAAAKDHAFCAA|1931-12-06|383|1666|129|1931|0|12|6|4|1931|129|1666|Sunday|1931Q4|N|N|N|2426677|2427010|2426317|2426590|N|N|N|N|N| +2426683|AAAAAAAALDHAFCAA|1931-12-07|383|1666|129|1931|1|12|7|4|1931|129|1666|Monday|1931Q4|N|N|N|2426677|2427010|2426318|2426591|N|N|N|N|N| +2426684|AAAAAAAAMDHAFCAA|1931-12-08|383|1667|129|1931|2|12|8|4|1931|129|1667|Tuesday|1931Q4|N|N|N|2426677|2427010|2426319|2426592|N|N|N|N|N| +2426685|AAAAAAAANDHAFCAA|1931-12-09|383|1667|129|1931|3|12|9|4|1931|129|1667|Wednesday|1931Q4|N|N|N|2426677|2427010|2426320|2426593|N|N|N|N|N| +2426686|AAAAAAAAODHAFCAA|1931-12-10|383|1667|129|1931|4|12|10|4|1931|129|1667|Thursday|1931Q4|N|N|N|2426677|2427010|2426321|2426594|N|N|N|N|N| +2426687|AAAAAAAAPDHAFCAA|1931-12-11|383|1667|129|1931|5|12|11|4|1931|129|1667|Friday|1931Q4|N|Y|N|2426677|2427010|2426322|2426595|N|N|N|N|N| +2426688|AAAAAAAAAEHAFCAA|1931-12-12|383|1667|129|1931|6|12|12|4|1931|129|1667|Saturday|1931Q4|N|Y|N|2426677|2427010|2426323|2426596|N|N|N|N|N| +2426689|AAAAAAAABEHAFCAA|1931-12-13|383|1667|129|1931|0|12|13|4|1931|129|1667|Sunday|1931Q4|N|N|N|2426677|2427010|2426324|2426597|N|N|N|N|N| +2426690|AAAAAAAACEHAFCAA|1931-12-14|383|1667|129|1931|1|12|14|4|1931|129|1667|Monday|1931Q4|N|N|N|2426677|2427010|2426325|2426598|N|N|N|N|N| +2426691|AAAAAAAADEHAFCAA|1931-12-15|383|1668|129|1931|2|12|15|4|1931|129|1668|Tuesday|1931Q4|N|N|N|2426677|2427010|2426326|2426599|N|N|N|N|N| +2426692|AAAAAAAAEEHAFCAA|1931-12-16|383|1668|129|1931|3|12|16|4|1931|129|1668|Wednesday|1931Q4|N|N|N|2426677|2427010|2426327|2426600|N|N|N|N|N| +2426693|AAAAAAAAFEHAFCAA|1931-12-17|383|1668|129|1931|4|12|17|4|1931|129|1668|Thursday|1931Q4|N|N|N|2426677|2427010|2426328|2426601|N|N|N|N|N| +2426694|AAAAAAAAGEHAFCAA|1931-12-18|383|1668|129|1931|5|12|18|4|1931|129|1668|Friday|1931Q4|N|Y|N|2426677|2427010|2426329|2426602|N|N|N|N|N| +2426695|AAAAAAAAHEHAFCAA|1931-12-19|383|1668|129|1931|6|12|19|4|1931|129|1668|Saturday|1931Q4|N|Y|N|2426677|2427010|2426330|2426603|N|N|N|N|N| +2426696|AAAAAAAAIEHAFCAA|1931-12-20|383|1668|129|1931|0|12|20|4|1931|129|1668|Sunday|1931Q4|N|N|N|2426677|2427010|2426331|2426604|N|N|N|N|N| +2426697|AAAAAAAAJEHAFCAA|1931-12-21|383|1668|129|1931|1|12|21|4|1931|129|1668|Monday|1931Q4|N|N|N|2426677|2427010|2426332|2426605|N|N|N|N|N| +2426698|AAAAAAAAKEHAFCAA|1931-12-22|383|1669|129|1931|2|12|22|4|1931|129|1669|Tuesday|1931Q4|N|N|N|2426677|2427010|2426333|2426606|N|N|N|N|N| +2426699|AAAAAAAALEHAFCAA|1931-12-23|383|1669|129|1931|3|12|23|4|1931|129|1669|Wednesday|1931Q4|N|N|N|2426677|2427010|2426334|2426607|N|N|N|N|N| +2426700|AAAAAAAAMEHAFCAA|1931-12-24|383|1669|129|1931|4|12|24|4|1931|129|1669|Thursday|1931Q4|N|N|N|2426677|2427010|2426335|2426608|N|N|N|N|N| +2426701|AAAAAAAANEHAFCAA|1931-12-25|383|1669|129|1931|5|12|25|4|1931|129|1669|Friday|1931Q4|N|Y|N|2426677|2427010|2426336|2426609|N|N|N|N|N| +2426702|AAAAAAAAOEHAFCAA|1931-12-26|383|1669|129|1931|6|12|26|4|1931|129|1669|Saturday|1931Q4|Y|Y|N|2426677|2427010|2426337|2426610|N|N|N|N|N| +2426703|AAAAAAAAPEHAFCAA|1931-12-27|383|1669|129|1931|0|12|27|4|1931|129|1669|Sunday|1931Q4|N|N|Y|2426677|2427010|2426338|2426611|N|N|N|N|N| +2426704|AAAAAAAAAFHAFCAA|1931-12-28|383|1669|129|1931|1|12|28|4|1931|129|1669|Monday|1931Q4|N|N|N|2426677|2427010|2426339|2426612|N|N|N|N|N| +2426705|AAAAAAAABFHAFCAA|1931-12-29|383|1670|129|1931|2|12|29|4|1931|129|1670|Tuesday|1931Q4|N|N|N|2426677|2427010|2426340|2426613|N|N|N|N|N| +2426706|AAAAAAAACFHAFCAA|1931-12-30|383|1670|129|1931|3|12|30|4|1931|129|1670|Wednesday|1931Q4|N|N|N|2426677|2427010|2426341|2426614|N|N|N|N|N| +2426707|AAAAAAAADFHAFCAA|1931-12-31|383|1670|129|1931|4|12|31|4|1931|129|1670|Thursday|1931Q4|N|N|N|2426677|2427010|2426342|2426615|N|N|N|N|N| +2426708|AAAAAAAAEFHAFCAA|1932-01-01|384|1670|129|1932|5|1|1|1|1932|129|1670|Friday|1932Q1|Y|Y|N|2426708|2426707|2426343|2426616|N|N|N|N|N| +2426709|AAAAAAAAFFHAFCAA|1932-01-02|384|1670|129|1932|6|1|2|1|1932|129|1670|Saturday|1932Q1|N|Y|Y|2426708|2426707|2426344|2426617|N|N|N|N|N| +2426710|AAAAAAAAGFHAFCAA|1932-01-03|384|1670|129|1932|0|1|3|1|1932|129|1670|Sunday|1932Q1|N|N|N|2426708|2426707|2426345|2426618|N|N|N|N|N| +2426711|AAAAAAAAHFHAFCAA|1932-01-04|384|1670|129|1932|1|1|4|1|1932|129|1670|Monday|1932Q1|N|N|N|2426708|2426707|2426346|2426619|N|N|N|N|N| +2426712|AAAAAAAAIFHAFCAA|1932-01-05|384|1671|129|1932|2|1|5|1|1932|129|1671|Tuesday|1932Q1|N|N|N|2426708|2426707|2426347|2426620|N|N|N|N|N| +2426713|AAAAAAAAJFHAFCAA|1932-01-06|384|1671|129|1932|3|1|6|1|1932|129|1671|Wednesday|1932Q1|N|N|N|2426708|2426707|2426348|2426621|N|N|N|N|N| +2426714|AAAAAAAAKFHAFCAA|1932-01-07|384|1671|129|1932|4|1|7|1|1932|129|1671|Thursday|1932Q1|N|N|N|2426708|2426707|2426349|2426622|N|N|N|N|N| +2426715|AAAAAAAALFHAFCAA|1932-01-08|384|1671|129|1932|5|1|8|1|1932|129|1671|Friday|1932Q1|N|Y|N|2426708|2426707|2426350|2426623|N|N|N|N|N| +2426716|AAAAAAAAMFHAFCAA|1932-01-09|384|1671|129|1932|6|1|9|1|1932|129|1671|Saturday|1932Q1|N|Y|N|2426708|2426707|2426351|2426624|N|N|N|N|N| +2426717|AAAAAAAANFHAFCAA|1932-01-10|384|1671|129|1932|0|1|10|1|1932|129|1671|Sunday|1932Q1|N|N|N|2426708|2426707|2426352|2426625|N|N|N|N|N| +2426718|AAAAAAAAOFHAFCAA|1932-01-11|384|1671|129|1932|1|1|11|1|1932|129|1671|Monday|1932Q1|N|N|N|2426708|2426707|2426353|2426626|N|N|N|N|N| +2426719|AAAAAAAAPFHAFCAA|1932-01-12|384|1672|129|1932|2|1|12|1|1932|129|1672|Tuesday|1932Q1|N|N|N|2426708|2426707|2426354|2426627|N|N|N|N|N| +2426720|AAAAAAAAAGHAFCAA|1932-01-13|384|1672|129|1932|3|1|13|1|1932|129|1672|Wednesday|1932Q1|N|N|N|2426708|2426707|2426355|2426628|N|N|N|N|N| +2426721|AAAAAAAABGHAFCAA|1932-01-14|384|1672|129|1932|4|1|14|1|1932|129|1672|Thursday|1932Q1|N|N|N|2426708|2426707|2426356|2426629|N|N|N|N|N| +2426722|AAAAAAAACGHAFCAA|1932-01-15|384|1672|129|1932|5|1|15|1|1932|129|1672|Friday|1932Q1|N|Y|N|2426708|2426707|2426357|2426630|N|N|N|N|N| +2426723|AAAAAAAADGHAFCAA|1932-01-16|384|1672|129|1932|6|1|16|1|1932|129|1672|Saturday|1932Q1|N|Y|N|2426708|2426707|2426358|2426631|N|N|N|N|N| +2426724|AAAAAAAAEGHAFCAA|1932-01-17|384|1672|129|1932|0|1|17|1|1932|129|1672|Sunday|1932Q1|N|N|N|2426708|2426707|2426359|2426632|N|N|N|N|N| +2426725|AAAAAAAAFGHAFCAA|1932-01-18|384|1672|129|1932|1|1|18|1|1932|129|1672|Monday|1932Q1|N|N|N|2426708|2426707|2426360|2426633|N|N|N|N|N| +2426726|AAAAAAAAGGHAFCAA|1932-01-19|384|1673|129|1932|2|1|19|1|1932|129|1673|Tuesday|1932Q1|N|N|N|2426708|2426707|2426361|2426634|N|N|N|N|N| +2426727|AAAAAAAAHGHAFCAA|1932-01-20|384|1673|129|1932|3|1|20|1|1932|129|1673|Wednesday|1932Q1|N|N|N|2426708|2426707|2426362|2426635|N|N|N|N|N| +2426728|AAAAAAAAIGHAFCAA|1932-01-21|384|1673|129|1932|4|1|21|1|1932|129|1673|Thursday|1932Q1|N|N|N|2426708|2426707|2426363|2426636|N|N|N|N|N| +2426729|AAAAAAAAJGHAFCAA|1932-01-22|384|1673|129|1932|5|1|22|1|1932|129|1673|Friday|1932Q1|N|Y|N|2426708|2426707|2426364|2426637|N|N|N|N|N| +2426730|AAAAAAAAKGHAFCAA|1932-01-23|384|1673|129|1932|6|1|23|1|1932|129|1673|Saturday|1932Q1|N|Y|N|2426708|2426707|2426365|2426638|N|N|N|N|N| +2426731|AAAAAAAALGHAFCAA|1932-01-24|384|1673|129|1932|0|1|24|1|1932|129|1673|Sunday|1932Q1|N|N|N|2426708|2426707|2426366|2426639|N|N|N|N|N| +2426732|AAAAAAAAMGHAFCAA|1932-01-25|384|1673|129|1932|1|1|25|1|1932|129|1673|Monday|1932Q1|N|N|N|2426708|2426707|2426367|2426640|N|N|N|N|N| +2426733|AAAAAAAANGHAFCAA|1932-01-26|384|1674|129|1932|2|1|26|1|1932|129|1674|Tuesday|1932Q1|N|N|N|2426708|2426707|2426368|2426641|N|N|N|N|N| +2426734|AAAAAAAAOGHAFCAA|1932-01-27|384|1674|129|1932|3|1|27|1|1932|129|1674|Wednesday|1932Q1|N|N|N|2426708|2426707|2426369|2426642|N|N|N|N|N| +2426735|AAAAAAAAPGHAFCAA|1932-01-28|384|1674|129|1932|4|1|28|1|1932|129|1674|Thursday|1932Q1|N|N|N|2426708|2426707|2426370|2426643|N|N|N|N|N| +2426736|AAAAAAAAAHHAFCAA|1932-01-29|384|1674|129|1932|5|1|29|1|1932|129|1674|Friday|1932Q1|N|Y|N|2426708|2426707|2426371|2426644|N|N|N|N|N| +2426737|AAAAAAAABHHAFCAA|1932-01-30|384|1674|129|1932|6|1|30|1|1932|129|1674|Saturday|1932Q1|N|Y|N|2426708|2426707|2426372|2426645|N|N|N|N|N| +2426738|AAAAAAAACHHAFCAA|1932-01-31|384|1674|129|1932|0|1|31|1|1932|129|1674|Sunday|1932Q1|N|N|N|2426708|2426707|2426373|2426646|N|N|N|N|N| +2426739|AAAAAAAADHHAFCAA|1932-02-01|385|1674|129|1932|1|2|1|1|1932|129|1674|Monday|1932Q1|N|N|N|2426739|2426769|2426374|2426647|N|N|N|N|N| +2426740|AAAAAAAAEHHAFCAA|1932-02-02|385|1675|129|1932|2|2|2|1|1932|129|1675|Tuesday|1932Q1|N|N|N|2426739|2426769|2426375|2426648|N|N|N|N|N| +2426741|AAAAAAAAFHHAFCAA|1932-02-03|385|1675|129|1932|3|2|3|1|1932|129|1675|Wednesday|1932Q1|N|N|N|2426739|2426769|2426376|2426649|N|N|N|N|N| +2426742|AAAAAAAAGHHAFCAA|1932-02-04|385|1675|129|1932|4|2|4|1|1932|129|1675|Thursday|1932Q1|N|N|N|2426739|2426769|2426377|2426650|N|N|N|N|N| +2426743|AAAAAAAAHHHAFCAA|1932-02-05|385|1675|129|1932|5|2|5|1|1932|129|1675|Friday|1932Q1|N|Y|N|2426739|2426769|2426378|2426651|N|N|N|N|N| +2426744|AAAAAAAAIHHAFCAA|1932-02-06|385|1675|129|1932|6|2|6|1|1932|129|1675|Saturday|1932Q1|N|Y|N|2426739|2426769|2426379|2426652|N|N|N|N|N| +2426745|AAAAAAAAJHHAFCAA|1932-02-07|385|1675|129|1932|0|2|7|1|1932|129|1675|Sunday|1932Q1|N|N|N|2426739|2426769|2426380|2426653|N|N|N|N|N| +2426746|AAAAAAAAKHHAFCAA|1932-02-08|385|1675|129|1932|1|2|8|1|1932|129|1675|Monday|1932Q1|N|N|N|2426739|2426769|2426381|2426654|N|N|N|N|N| +2426747|AAAAAAAALHHAFCAA|1932-02-09|385|1676|129|1932|2|2|9|1|1932|129|1676|Tuesday|1932Q1|N|N|N|2426739|2426769|2426382|2426655|N|N|N|N|N| +2426748|AAAAAAAAMHHAFCAA|1932-02-10|385|1676|129|1932|3|2|10|1|1932|129|1676|Wednesday|1932Q1|N|N|N|2426739|2426769|2426383|2426656|N|N|N|N|N| +2426749|AAAAAAAANHHAFCAA|1932-02-11|385|1676|129|1932|4|2|11|1|1932|129|1676|Thursday|1932Q1|N|N|N|2426739|2426769|2426384|2426657|N|N|N|N|N| +2426750|AAAAAAAAOHHAFCAA|1932-02-12|385|1676|129|1932|5|2|12|1|1932|129|1676|Friday|1932Q1|N|Y|N|2426739|2426769|2426385|2426658|N|N|N|N|N| +2426751|AAAAAAAAPHHAFCAA|1932-02-13|385|1676|129|1932|6|2|13|1|1932|129|1676|Saturday|1932Q1|N|Y|N|2426739|2426769|2426386|2426659|N|N|N|N|N| +2426752|AAAAAAAAAIHAFCAA|1932-02-14|385|1676|129|1932|0|2|14|1|1932|129|1676|Sunday|1932Q1|N|N|N|2426739|2426769|2426387|2426660|N|N|N|N|N| +2426753|AAAAAAAABIHAFCAA|1932-02-15|385|1676|129|1932|1|2|15|1|1932|129|1676|Monday|1932Q1|N|N|N|2426739|2426769|2426388|2426661|N|N|N|N|N| +2426754|AAAAAAAACIHAFCAA|1932-02-16|385|1677|129|1932|2|2|16|1|1932|129|1677|Tuesday|1932Q1|N|N|N|2426739|2426769|2426389|2426662|N|N|N|N|N| +2426755|AAAAAAAADIHAFCAA|1932-02-17|385|1677|129|1932|3|2|17|1|1932|129|1677|Wednesday|1932Q1|N|N|N|2426739|2426769|2426390|2426663|N|N|N|N|N| +2426756|AAAAAAAAEIHAFCAA|1932-02-18|385|1677|129|1932|4|2|18|1|1932|129|1677|Thursday|1932Q1|N|N|N|2426739|2426769|2426391|2426664|N|N|N|N|N| +2426757|AAAAAAAAFIHAFCAA|1932-02-19|385|1677|129|1932|5|2|19|1|1932|129|1677|Friday|1932Q1|N|Y|N|2426739|2426769|2426392|2426665|N|N|N|N|N| +2426758|AAAAAAAAGIHAFCAA|1932-02-20|385|1677|129|1932|6|2|20|1|1932|129|1677|Saturday|1932Q1|N|Y|N|2426739|2426769|2426393|2426666|N|N|N|N|N| +2426759|AAAAAAAAHIHAFCAA|1932-02-21|385|1677|129|1932|0|2|21|1|1932|129|1677|Sunday|1932Q1|N|N|N|2426739|2426769|2426394|2426667|N|N|N|N|N| +2426760|AAAAAAAAIIHAFCAA|1932-02-22|385|1677|129|1932|1|2|22|1|1932|129|1677|Monday|1932Q1|N|N|N|2426739|2426769|2426395|2426668|N|N|N|N|N| +2426761|AAAAAAAAJIHAFCAA|1932-02-23|385|1678|129|1932|2|2|23|1|1932|129|1678|Tuesday|1932Q1|N|N|N|2426739|2426769|2426396|2426669|N|N|N|N|N| +2426762|AAAAAAAAKIHAFCAA|1932-02-24|385|1678|129|1932|3|2|24|1|1932|129|1678|Wednesday|1932Q1|N|N|N|2426739|2426769|2426397|2426670|N|N|N|N|N| +2426763|AAAAAAAALIHAFCAA|1932-02-25|385|1678|129|1932|4|2|25|1|1932|129|1678|Thursday|1932Q1|N|N|N|2426739|2426769|2426398|2426671|N|N|N|N|N| +2426764|AAAAAAAAMIHAFCAA|1932-02-26|385|1678|129|1932|5|2|26|1|1932|129|1678|Friday|1932Q1|N|Y|N|2426739|2426769|2426399|2426672|N|N|N|N|N| +2426765|AAAAAAAANIHAFCAA|1932-02-27|385|1678|129|1932|6|2|27|1|1932|129|1678|Saturday|1932Q1|N|Y|N|2426739|2426769|2426400|2426673|N|N|N|N|N| +2426766|AAAAAAAAOIHAFCAA|1932-02-28|385|1678|129|1932|0|2|28|1|1932|129|1678|Sunday|1932Q1|N|N|N|2426739|2426769|2426401|2426674|N|N|N|N|N| +2426767|AAAAAAAAPIHAFCAA|1932-02-29|385|1678|129|1932|1|2|29|1|1932|129|1678|Monday|1932Q1|N|N|N|2426739|2426769|2426401|2426675|N|N|N|N|N| +2426768|AAAAAAAAAJHAFCAA|1932-03-01|386|1679|130|1932|2|3|1|1|1932|130|1679|Tuesday|1932Q1|N|N|N|2426768|2426827|2426402|2426676|N|N|N|N|N| +2426769|AAAAAAAABJHAFCAA|1932-03-02|386|1679|130|1932|3|3|2|1|1932|130|1679|Wednesday|1932Q1|N|N|N|2426768|2426827|2426403|2426677|N|N|N|N|N| +2426770|AAAAAAAACJHAFCAA|1932-03-03|386|1679|130|1932|4|3|3|1|1932|130|1679|Thursday|1932Q1|N|N|N|2426768|2426827|2426404|2426678|N|N|N|N|N| +2426771|AAAAAAAADJHAFCAA|1932-03-04|386|1679|130|1932|5|3|4|1|1932|130|1679|Friday|1932Q1|N|Y|N|2426768|2426827|2426405|2426679|N|N|N|N|N| +2426772|AAAAAAAAEJHAFCAA|1932-03-05|386|1679|130|1932|6|3|5|1|1932|130|1679|Saturday|1932Q1|N|Y|N|2426768|2426827|2426406|2426680|N|N|N|N|N| +2426773|AAAAAAAAFJHAFCAA|1932-03-06|386|1679|130|1932|0|3|6|1|1932|130|1679|Sunday|1932Q1|N|N|N|2426768|2426827|2426407|2426681|N|N|N|N|N| +2426774|AAAAAAAAGJHAFCAA|1932-03-07|386|1679|130|1932|1|3|7|1|1932|130|1679|Monday|1932Q1|N|N|N|2426768|2426827|2426408|2426682|N|N|N|N|N| +2426775|AAAAAAAAHJHAFCAA|1932-03-08|386|1680|130|1932|2|3|8|1|1932|130|1680|Tuesday|1932Q1|N|N|N|2426768|2426827|2426409|2426683|N|N|N|N|N| +2426776|AAAAAAAAIJHAFCAA|1932-03-09|386|1680|130|1932|3|3|9|1|1932|130|1680|Wednesday|1932Q1|N|N|N|2426768|2426827|2426410|2426684|N|N|N|N|N| +2426777|AAAAAAAAJJHAFCAA|1932-03-10|386|1680|130|1932|4|3|10|1|1932|130|1680|Thursday|1932Q1|N|N|N|2426768|2426827|2426411|2426685|N|N|N|N|N| +2426778|AAAAAAAAKJHAFCAA|1932-03-11|386|1680|130|1932|5|3|11|1|1932|130|1680|Friday|1932Q1|N|Y|N|2426768|2426827|2426412|2426686|N|N|N|N|N| +2426779|AAAAAAAALJHAFCAA|1932-03-12|386|1680|130|1932|6|3|12|1|1932|130|1680|Saturday|1932Q1|N|Y|N|2426768|2426827|2426413|2426687|N|N|N|N|N| +2426780|AAAAAAAAMJHAFCAA|1932-03-13|386|1680|130|1932|0|3|13|1|1932|130|1680|Sunday|1932Q1|N|N|N|2426768|2426827|2426414|2426688|N|N|N|N|N| +2426781|AAAAAAAANJHAFCAA|1932-03-14|386|1680|130|1932|1|3|14|1|1932|130|1680|Monday|1932Q1|N|N|N|2426768|2426827|2426415|2426689|N|N|N|N|N| +2426782|AAAAAAAAOJHAFCAA|1932-03-15|386|1681|130|1932|2|3|15|1|1932|130|1681|Tuesday|1932Q1|N|N|N|2426768|2426827|2426416|2426690|N|N|N|N|N| +2426783|AAAAAAAAPJHAFCAA|1932-03-16|386|1681|130|1932|3|3|16|1|1932|130|1681|Wednesday|1932Q1|N|N|N|2426768|2426827|2426417|2426691|N|N|N|N|N| +2426784|AAAAAAAAAKHAFCAA|1932-03-17|386|1681|130|1932|4|3|17|1|1932|130|1681|Thursday|1932Q1|N|N|N|2426768|2426827|2426418|2426692|N|N|N|N|N| +2426785|AAAAAAAABKHAFCAA|1932-03-18|386|1681|130|1932|5|3|18|1|1932|130|1681|Friday|1932Q1|N|Y|N|2426768|2426827|2426419|2426693|N|N|N|N|N| +2426786|AAAAAAAACKHAFCAA|1932-03-19|386|1681|130|1932|6|3|19|1|1932|130|1681|Saturday|1932Q1|N|Y|N|2426768|2426827|2426420|2426694|N|N|N|N|N| +2426787|AAAAAAAADKHAFCAA|1932-03-20|386|1681|130|1932|0|3|20|1|1932|130|1681|Sunday|1932Q1|N|N|N|2426768|2426827|2426421|2426695|N|N|N|N|N| +2426788|AAAAAAAAEKHAFCAA|1932-03-21|386|1681|130|1932|1|3|21|1|1932|130|1681|Monday|1932Q1|N|N|N|2426768|2426827|2426422|2426696|N|N|N|N|N| +2426789|AAAAAAAAFKHAFCAA|1932-03-22|386|1682|130|1932|2|3|22|1|1932|130|1682|Tuesday|1932Q1|N|N|N|2426768|2426827|2426423|2426697|N|N|N|N|N| +2426790|AAAAAAAAGKHAFCAA|1932-03-23|386|1682|130|1932|3|3|23|1|1932|130|1682|Wednesday|1932Q1|N|N|N|2426768|2426827|2426424|2426698|N|N|N|N|N| +2426791|AAAAAAAAHKHAFCAA|1932-03-24|386|1682|130|1932|4|3|24|1|1932|130|1682|Thursday|1932Q1|N|N|N|2426768|2426827|2426425|2426699|N|N|N|N|N| +2426792|AAAAAAAAIKHAFCAA|1932-03-25|386|1682|130|1932|5|3|25|1|1932|130|1682|Friday|1932Q1|N|Y|N|2426768|2426827|2426426|2426700|N|N|N|N|N| +2426793|AAAAAAAAJKHAFCAA|1932-03-26|386|1682|130|1932|6|3|26|1|1932|130|1682|Saturday|1932Q1|N|Y|N|2426768|2426827|2426427|2426701|N|N|N|N|N| +2426794|AAAAAAAAKKHAFCAA|1932-03-27|386|1682|130|1932|0|3|27|1|1932|130|1682|Sunday|1932Q1|N|N|N|2426768|2426827|2426428|2426702|N|N|N|N|N| +2426795|AAAAAAAALKHAFCAA|1932-03-28|386|1682|130|1932|1|3|28|1|1932|130|1682|Monday|1932Q1|N|N|N|2426768|2426827|2426429|2426703|N|N|N|N|N| +2426796|AAAAAAAAMKHAFCAA|1932-03-29|386|1683|130|1932|2|3|29|1|1932|130|1683|Tuesday|1932Q1|N|N|N|2426768|2426827|2426430|2426704|N|N|N|N|N| +2426797|AAAAAAAANKHAFCAA|1932-03-30|386|1683|130|1932|3|3|30|1|1932|130|1683|Wednesday|1932Q1|N|N|N|2426768|2426827|2426431|2426705|N|N|N|N|N| +2426798|AAAAAAAAOKHAFCAA|1932-03-31|386|1683|130|1932|4|3|31|1|1932|130|1683|Thursday|1932Q1|N|N|N|2426768|2426827|2426432|2426706|N|N|N|N|N| +2426799|AAAAAAAAPKHAFCAA|1932-04-01|387|1683|130|1932|5|4|1|2|1932|130|1683|Friday|1932Q2|N|Y|N|2426799|2426889|2426433|2426708|N|N|N|N|N| +2426800|AAAAAAAAALHAFCAA|1932-04-02|387|1683|130|1932|6|4|2|2|1932|130|1683|Saturday|1932Q2|N|Y|N|2426799|2426889|2426434|2426709|N|N|N|N|N| +2426801|AAAAAAAABLHAFCAA|1932-04-03|387|1683|130|1932|0|4|3|2|1932|130|1683|Sunday|1932Q2|N|N|N|2426799|2426889|2426435|2426710|N|N|N|N|N| +2426802|AAAAAAAACLHAFCAA|1932-04-04|387|1683|130|1932|1|4|4|2|1932|130|1683|Monday|1932Q2|N|N|N|2426799|2426889|2426436|2426711|N|N|N|N|N| +2426803|AAAAAAAADLHAFCAA|1932-04-05|387|1684|130|1932|2|4|5|2|1932|130|1684|Tuesday|1932Q2|N|N|N|2426799|2426889|2426437|2426712|N|N|N|N|N| +2426804|AAAAAAAAELHAFCAA|1932-04-06|387|1684|130|1932|3|4|6|2|1932|130|1684|Wednesday|1932Q2|N|N|N|2426799|2426889|2426438|2426713|N|N|N|N|N| +2426805|AAAAAAAAFLHAFCAA|1932-04-07|387|1684|130|1932|4|4|7|2|1932|130|1684|Thursday|1932Q2|N|N|N|2426799|2426889|2426439|2426714|N|N|N|N|N| +2426806|AAAAAAAAGLHAFCAA|1932-04-08|387|1684|130|1932|5|4|8|2|1932|130|1684|Friday|1932Q2|N|Y|N|2426799|2426889|2426440|2426715|N|N|N|N|N| +2426807|AAAAAAAAHLHAFCAA|1932-04-09|387|1684|130|1932|6|4|9|2|1932|130|1684|Saturday|1932Q2|N|Y|N|2426799|2426889|2426441|2426716|N|N|N|N|N| +2426808|AAAAAAAAILHAFCAA|1932-04-10|387|1684|130|1932|0|4|10|2|1932|130|1684|Sunday|1932Q2|N|N|N|2426799|2426889|2426442|2426717|N|N|N|N|N| +2426809|AAAAAAAAJLHAFCAA|1932-04-11|387|1684|130|1932|1|4|11|2|1932|130|1684|Monday|1932Q2|N|N|N|2426799|2426889|2426443|2426718|N|N|N|N|N| +2426810|AAAAAAAAKLHAFCAA|1932-04-12|387|1685|130|1932|2|4|12|2|1932|130|1685|Tuesday|1932Q2|N|N|N|2426799|2426889|2426444|2426719|N|N|N|N|N| +2426811|AAAAAAAALLHAFCAA|1932-04-13|387|1685|130|1932|3|4|13|2|1932|130|1685|Wednesday|1932Q2|N|N|N|2426799|2426889|2426445|2426720|N|N|N|N|N| +2426812|AAAAAAAAMLHAFCAA|1932-04-14|387|1685|130|1932|4|4|14|2|1932|130|1685|Thursday|1932Q2|N|N|N|2426799|2426889|2426446|2426721|N|N|N|N|N| +2426813|AAAAAAAANLHAFCAA|1932-04-15|387|1685|130|1932|5|4|15|2|1932|130|1685|Friday|1932Q2|N|Y|N|2426799|2426889|2426447|2426722|N|N|N|N|N| +2426814|AAAAAAAAOLHAFCAA|1932-04-16|387|1685|130|1932|6|4|16|2|1932|130|1685|Saturday|1932Q2|N|Y|N|2426799|2426889|2426448|2426723|N|N|N|N|N| +2426815|AAAAAAAAPLHAFCAA|1932-04-17|387|1685|130|1932|0|4|17|2|1932|130|1685|Sunday|1932Q2|N|N|N|2426799|2426889|2426449|2426724|N|N|N|N|N| +2426816|AAAAAAAAAMHAFCAA|1932-04-18|387|1685|130|1932|1|4|18|2|1932|130|1685|Monday|1932Q2|N|N|N|2426799|2426889|2426450|2426725|N|N|N|N|N| +2426817|AAAAAAAABMHAFCAA|1932-04-19|387|1686|130|1932|2|4|19|2|1932|130|1686|Tuesday|1932Q2|N|N|N|2426799|2426889|2426451|2426726|N|N|N|N|N| +2426818|AAAAAAAACMHAFCAA|1932-04-20|387|1686|130|1932|3|4|20|2|1932|130|1686|Wednesday|1932Q2|N|N|N|2426799|2426889|2426452|2426727|N|N|N|N|N| +2426819|AAAAAAAADMHAFCAA|1932-04-21|387|1686|130|1932|4|4|21|2|1932|130|1686|Thursday|1932Q2|N|N|N|2426799|2426889|2426453|2426728|N|N|N|N|N| +2426820|AAAAAAAAEMHAFCAA|1932-04-22|387|1686|130|1932|5|4|22|2|1932|130|1686|Friday|1932Q2|N|Y|N|2426799|2426889|2426454|2426729|N|N|N|N|N| +2426821|AAAAAAAAFMHAFCAA|1932-04-23|387|1686|130|1932|6|4|23|2|1932|130|1686|Saturday|1932Q2|N|Y|N|2426799|2426889|2426455|2426730|N|N|N|N|N| +2426822|AAAAAAAAGMHAFCAA|1932-04-24|387|1686|130|1932|0|4|24|2|1932|130|1686|Sunday|1932Q2|N|N|N|2426799|2426889|2426456|2426731|N|N|N|N|N| +2426823|AAAAAAAAHMHAFCAA|1932-04-25|387|1686|130|1932|1|4|25|2|1932|130|1686|Monday|1932Q2|N|N|N|2426799|2426889|2426457|2426732|N|N|N|N|N| +2426824|AAAAAAAAIMHAFCAA|1932-04-26|387|1687|130|1932|2|4|26|2|1932|130|1687|Tuesday|1932Q2|N|N|N|2426799|2426889|2426458|2426733|N|N|N|N|N| +2426825|AAAAAAAAJMHAFCAA|1932-04-27|387|1687|130|1932|3|4|27|2|1932|130|1687|Wednesday|1932Q2|N|N|N|2426799|2426889|2426459|2426734|N|N|N|N|N| +2426826|AAAAAAAAKMHAFCAA|1932-04-28|387|1687|130|1932|4|4|28|2|1932|130|1687|Thursday|1932Q2|N|N|N|2426799|2426889|2426460|2426735|N|N|N|N|N| +2426827|AAAAAAAALMHAFCAA|1932-04-29|387|1687|130|1932|5|4|29|2|1932|130|1687|Friday|1932Q2|N|Y|N|2426799|2426889|2426461|2426736|N|N|N|N|N| +2426828|AAAAAAAAMMHAFCAA|1932-04-30|387|1687|130|1932|6|4|30|2|1932|130|1687|Saturday|1932Q2|N|Y|N|2426799|2426889|2426462|2426737|N|N|N|N|N| +2426829|AAAAAAAANMHAFCAA|1932-05-01|388|1687|130|1932|0|5|1|2|1932|130|1687|Sunday|1932Q2|N|N|N|2426829|2426949|2426463|2426738|N|N|N|N|N| +2426830|AAAAAAAAOMHAFCAA|1932-05-02|388|1687|130|1932|1|5|2|2|1932|130|1687|Monday|1932Q2|N|N|N|2426829|2426949|2426464|2426739|N|N|N|N|N| +2426831|AAAAAAAAPMHAFCAA|1932-05-03|388|1688|130|1932|2|5|3|2|1932|130|1688|Tuesday|1932Q2|N|N|N|2426829|2426949|2426465|2426740|N|N|N|N|N| +2426832|AAAAAAAAANHAFCAA|1932-05-04|388|1688|130|1932|3|5|4|2|1932|130|1688|Wednesday|1932Q2|N|N|N|2426829|2426949|2426466|2426741|N|N|N|N|N| +2426833|AAAAAAAABNHAFCAA|1932-05-05|388|1688|130|1932|4|5|5|2|1932|130|1688|Thursday|1932Q2|N|N|N|2426829|2426949|2426467|2426742|N|N|N|N|N| +2426834|AAAAAAAACNHAFCAA|1932-05-06|388|1688|130|1932|5|5|6|2|1932|130|1688|Friday|1932Q2|N|Y|N|2426829|2426949|2426468|2426743|N|N|N|N|N| +2426835|AAAAAAAADNHAFCAA|1932-05-07|388|1688|130|1932|6|5|7|2|1932|130|1688|Saturday|1932Q2|N|Y|N|2426829|2426949|2426469|2426744|N|N|N|N|N| +2426836|AAAAAAAAENHAFCAA|1932-05-08|388|1688|130|1932|0|5|8|2|1932|130|1688|Sunday|1932Q2|N|N|N|2426829|2426949|2426470|2426745|N|N|N|N|N| +2426837|AAAAAAAAFNHAFCAA|1932-05-09|388|1688|130|1932|1|5|9|2|1932|130|1688|Monday|1932Q2|N|N|N|2426829|2426949|2426471|2426746|N|N|N|N|N| +2426838|AAAAAAAAGNHAFCAA|1932-05-10|388|1689|130|1932|2|5|10|2|1932|130|1689|Tuesday|1932Q2|N|N|N|2426829|2426949|2426472|2426747|N|N|N|N|N| +2426839|AAAAAAAAHNHAFCAA|1932-05-11|388|1689|130|1932|3|5|11|2|1932|130|1689|Wednesday|1932Q2|N|N|N|2426829|2426949|2426473|2426748|N|N|N|N|N| +2426840|AAAAAAAAINHAFCAA|1932-05-12|388|1689|130|1932|4|5|12|2|1932|130|1689|Thursday|1932Q2|N|N|N|2426829|2426949|2426474|2426749|N|N|N|N|N| +2426841|AAAAAAAAJNHAFCAA|1932-05-13|388|1689|130|1932|5|5|13|2|1932|130|1689|Friday|1932Q2|N|Y|N|2426829|2426949|2426475|2426750|N|N|N|N|N| +2426842|AAAAAAAAKNHAFCAA|1932-05-14|388|1689|130|1932|6|5|14|2|1932|130|1689|Saturday|1932Q2|N|Y|N|2426829|2426949|2426476|2426751|N|N|N|N|N| +2426843|AAAAAAAALNHAFCAA|1932-05-15|388|1689|130|1932|0|5|15|2|1932|130|1689|Sunday|1932Q2|N|N|N|2426829|2426949|2426477|2426752|N|N|N|N|N| +2426844|AAAAAAAAMNHAFCAA|1932-05-16|388|1689|130|1932|1|5|16|2|1932|130|1689|Monday|1932Q2|N|N|N|2426829|2426949|2426478|2426753|N|N|N|N|N| +2426845|AAAAAAAANNHAFCAA|1932-05-17|388|1690|130|1932|2|5|17|2|1932|130|1690|Tuesday|1932Q2|N|N|N|2426829|2426949|2426479|2426754|N|N|N|N|N| +2426846|AAAAAAAAONHAFCAA|1932-05-18|388|1690|130|1932|3|5|18|2|1932|130|1690|Wednesday|1932Q2|N|N|N|2426829|2426949|2426480|2426755|N|N|N|N|N| +2426847|AAAAAAAAPNHAFCAA|1932-05-19|388|1690|130|1932|4|5|19|2|1932|130|1690|Thursday|1932Q2|N|N|N|2426829|2426949|2426481|2426756|N|N|N|N|N| +2426848|AAAAAAAAAOHAFCAA|1932-05-20|388|1690|130|1932|5|5|20|2|1932|130|1690|Friday|1932Q2|N|Y|N|2426829|2426949|2426482|2426757|N|N|N|N|N| +2426849|AAAAAAAABOHAFCAA|1932-05-21|388|1690|130|1932|6|5|21|2|1932|130|1690|Saturday|1932Q2|N|Y|N|2426829|2426949|2426483|2426758|N|N|N|N|N| +2426850|AAAAAAAACOHAFCAA|1932-05-22|388|1690|130|1932|0|5|22|2|1932|130|1690|Sunday|1932Q2|N|N|N|2426829|2426949|2426484|2426759|N|N|N|N|N| +2426851|AAAAAAAADOHAFCAA|1932-05-23|388|1690|130|1932|1|5|23|2|1932|130|1690|Monday|1932Q2|N|N|N|2426829|2426949|2426485|2426760|N|N|N|N|N| +2426852|AAAAAAAAEOHAFCAA|1932-05-24|388|1691|130|1932|2|5|24|2|1932|130|1691|Tuesday|1932Q2|N|N|N|2426829|2426949|2426486|2426761|N|N|N|N|N| +2426853|AAAAAAAAFOHAFCAA|1932-05-25|388|1691|130|1932|3|5|25|2|1932|130|1691|Wednesday|1932Q2|N|N|N|2426829|2426949|2426487|2426762|N|N|N|N|N| +2426854|AAAAAAAAGOHAFCAA|1932-05-26|388|1691|130|1932|4|5|26|2|1932|130|1691|Thursday|1932Q2|N|N|N|2426829|2426949|2426488|2426763|N|N|N|N|N| +2426855|AAAAAAAAHOHAFCAA|1932-05-27|388|1691|130|1932|5|5|27|2|1932|130|1691|Friday|1932Q2|N|Y|N|2426829|2426949|2426489|2426764|N|N|N|N|N| +2426856|AAAAAAAAIOHAFCAA|1932-05-28|388|1691|130|1932|6|5|28|2|1932|130|1691|Saturday|1932Q2|N|Y|N|2426829|2426949|2426490|2426765|N|N|N|N|N| +2426857|AAAAAAAAJOHAFCAA|1932-05-29|388|1691|130|1932|0|5|29|2|1932|130|1691|Sunday|1932Q2|N|N|N|2426829|2426949|2426491|2426766|N|N|N|N|N| +2426858|AAAAAAAAKOHAFCAA|1932-05-30|388|1691|130|1932|1|5|30|2|1932|130|1691|Monday|1932Q2|N|N|N|2426829|2426949|2426492|2426767|N|N|N|N|N| +2426859|AAAAAAAALOHAFCAA|1932-05-31|388|1692|130|1932|2|5|31|2|1932|130|1692|Tuesday|1932Q2|N|N|N|2426829|2426949|2426493|2426768|N|N|N|N|N| +2426860|AAAAAAAAMOHAFCAA|1932-06-01|389|1692|131|1932|3|6|1|2|1932|131|1692|Wednesday|1932Q2|N|N|N|2426860|2427011|2426494|2426769|N|N|N|N|N| +2426861|AAAAAAAANOHAFCAA|1932-06-02|389|1692|131|1932|4|6|2|2|1932|131|1692|Thursday|1932Q2|N|N|N|2426860|2427011|2426495|2426770|N|N|N|N|N| +2426862|AAAAAAAAOOHAFCAA|1932-06-03|389|1692|131|1932|5|6|3|2|1932|131|1692|Friday|1932Q2|N|Y|N|2426860|2427011|2426496|2426771|N|N|N|N|N| +2426863|AAAAAAAAPOHAFCAA|1932-06-04|389|1692|131|1932|6|6|4|2|1932|131|1692|Saturday|1932Q2|N|Y|N|2426860|2427011|2426497|2426772|N|N|N|N|N| +2426864|AAAAAAAAAPHAFCAA|1932-06-05|389|1692|131|1932|0|6|5|2|1932|131|1692|Sunday|1932Q2|N|N|N|2426860|2427011|2426498|2426773|N|N|N|N|N| +2426865|AAAAAAAABPHAFCAA|1932-06-06|389|1692|131|1932|1|6|6|2|1932|131|1692|Monday|1932Q2|N|N|N|2426860|2427011|2426499|2426774|N|N|N|N|N| +2426866|AAAAAAAACPHAFCAA|1932-06-07|389|1693|131|1932|2|6|7|2|1932|131|1693|Tuesday|1932Q2|N|N|N|2426860|2427011|2426500|2426775|N|N|N|N|N| +2426867|AAAAAAAADPHAFCAA|1932-06-08|389|1693|131|1932|3|6|8|2|1932|131|1693|Wednesday|1932Q2|N|N|N|2426860|2427011|2426501|2426776|N|N|N|N|N| +2426868|AAAAAAAAEPHAFCAA|1932-06-09|389|1693|131|1932|4|6|9|2|1932|131|1693|Thursday|1932Q2|N|N|N|2426860|2427011|2426502|2426777|N|N|N|N|N| +2426869|AAAAAAAAFPHAFCAA|1932-06-10|389|1693|131|1932|5|6|10|2|1932|131|1693|Friday|1932Q2|N|Y|N|2426860|2427011|2426503|2426778|N|N|N|N|N| +2426870|AAAAAAAAGPHAFCAA|1932-06-11|389|1693|131|1932|6|6|11|2|1932|131|1693|Saturday|1932Q2|N|Y|N|2426860|2427011|2426504|2426779|N|N|N|N|N| +2426871|AAAAAAAAHPHAFCAA|1932-06-12|389|1693|131|1932|0|6|12|2|1932|131|1693|Sunday|1932Q2|N|N|N|2426860|2427011|2426505|2426780|N|N|N|N|N| +2426872|AAAAAAAAIPHAFCAA|1932-06-13|389|1693|131|1932|1|6|13|2|1932|131|1693|Monday|1932Q2|N|N|N|2426860|2427011|2426506|2426781|N|N|N|N|N| +2426873|AAAAAAAAJPHAFCAA|1932-06-14|389|1694|131|1932|2|6|14|2|1932|131|1694|Tuesday|1932Q2|N|N|N|2426860|2427011|2426507|2426782|N|N|N|N|N| +2426874|AAAAAAAAKPHAFCAA|1932-06-15|389|1694|131|1932|3|6|15|2|1932|131|1694|Wednesday|1932Q2|N|N|N|2426860|2427011|2426508|2426783|N|N|N|N|N| +2426875|AAAAAAAALPHAFCAA|1932-06-16|389|1694|131|1932|4|6|16|2|1932|131|1694|Thursday|1932Q2|N|N|N|2426860|2427011|2426509|2426784|N|N|N|N|N| +2426876|AAAAAAAAMPHAFCAA|1932-06-17|389|1694|131|1932|5|6|17|2|1932|131|1694|Friday|1932Q2|N|Y|N|2426860|2427011|2426510|2426785|N|N|N|N|N| +2426877|AAAAAAAANPHAFCAA|1932-06-18|389|1694|131|1932|6|6|18|2|1932|131|1694|Saturday|1932Q2|N|Y|N|2426860|2427011|2426511|2426786|N|N|N|N|N| +2426878|AAAAAAAAOPHAFCAA|1932-06-19|389|1694|131|1932|0|6|19|2|1932|131|1694|Sunday|1932Q2|N|N|N|2426860|2427011|2426512|2426787|N|N|N|N|N| +2426879|AAAAAAAAPPHAFCAA|1932-06-20|389|1694|131|1932|1|6|20|2|1932|131|1694|Monday|1932Q2|N|N|N|2426860|2427011|2426513|2426788|N|N|N|N|N| +2426880|AAAAAAAAAAIAFCAA|1932-06-21|389|1695|131|1932|2|6|21|2|1932|131|1695|Tuesday|1932Q2|N|N|N|2426860|2427011|2426514|2426789|N|N|N|N|N| +2426881|AAAAAAAABAIAFCAA|1932-06-22|389|1695|131|1932|3|6|22|2|1932|131|1695|Wednesday|1932Q2|N|N|N|2426860|2427011|2426515|2426790|N|N|N|N|N| +2426882|AAAAAAAACAIAFCAA|1932-06-23|389|1695|131|1932|4|6|23|2|1932|131|1695|Thursday|1932Q2|N|N|N|2426860|2427011|2426516|2426791|N|N|N|N|N| +2426883|AAAAAAAADAIAFCAA|1932-06-24|389|1695|131|1932|5|6|24|2|1932|131|1695|Friday|1932Q2|N|Y|N|2426860|2427011|2426517|2426792|N|N|N|N|N| +2426884|AAAAAAAAEAIAFCAA|1932-06-25|389|1695|131|1932|6|6|25|2|1932|131|1695|Saturday|1932Q2|N|Y|N|2426860|2427011|2426518|2426793|N|N|N|N|N| +2426885|AAAAAAAAFAIAFCAA|1932-06-26|389|1695|131|1932|0|6|26|2|1932|131|1695|Sunday|1932Q2|N|N|N|2426860|2427011|2426519|2426794|N|N|N|N|N| +2426886|AAAAAAAAGAIAFCAA|1932-06-27|389|1695|131|1932|1|6|27|2|1932|131|1695|Monday|1932Q2|N|N|N|2426860|2427011|2426520|2426795|N|N|N|N|N| +2426887|AAAAAAAAHAIAFCAA|1932-06-28|389|1696|131|1932|2|6|28|2|1932|131|1696|Tuesday|1932Q2|N|N|N|2426860|2427011|2426521|2426796|N|N|N|N|N| +2426888|AAAAAAAAIAIAFCAA|1932-06-29|389|1696|131|1932|3|6|29|2|1932|131|1696|Wednesday|1932Q2|N|N|N|2426860|2427011|2426522|2426797|N|N|N|N|N| +2426889|AAAAAAAAJAIAFCAA|1932-06-30|389|1696|131|1932|4|6|30|2|1932|131|1696|Thursday|1932Q2|N|N|N|2426860|2427011|2426523|2426798|N|N|N|N|N| +2426890|AAAAAAAAKAIAFCAA|1932-07-01|390|1696|131|1932|5|7|1|3|1932|131|1696|Friday|1932Q3|N|Y|N|2426890|2427071|2426524|2426799|N|N|N|N|N| +2426891|AAAAAAAALAIAFCAA|1932-07-02|390|1696|131|1932|6|7|2|3|1932|131|1696|Saturday|1932Q3|N|Y|N|2426890|2427071|2426525|2426800|N|N|N|N|N| +2426892|AAAAAAAAMAIAFCAA|1932-07-03|390|1696|131|1932|0|7|3|3|1932|131|1696|Sunday|1932Q3|N|N|N|2426890|2427071|2426526|2426801|N|N|N|N|N| +2426893|AAAAAAAANAIAFCAA|1932-07-04|390|1696|131|1932|1|7|4|3|1932|131|1696|Monday|1932Q3|Y|N|N|2426890|2427071|2426527|2426802|N|N|N|N|N| +2426894|AAAAAAAAOAIAFCAA|1932-07-05|390|1697|131|1932|2|7|5|3|1932|131|1697|Tuesday|1932Q3|N|N|Y|2426890|2427071|2426528|2426803|N|N|N|N|N| +2426895|AAAAAAAAPAIAFCAA|1932-07-06|390|1697|131|1932|3|7|6|3|1932|131|1697|Wednesday|1932Q3|N|N|N|2426890|2427071|2426529|2426804|N|N|N|N|N| +2426896|AAAAAAAAABIAFCAA|1932-07-07|390|1697|131|1932|4|7|7|3|1932|131|1697|Thursday|1932Q3|N|N|N|2426890|2427071|2426530|2426805|N|N|N|N|N| +2426897|AAAAAAAABBIAFCAA|1932-07-08|390|1697|131|1932|5|7|8|3|1932|131|1697|Friday|1932Q3|N|Y|N|2426890|2427071|2426531|2426806|N|N|N|N|N| +2426898|AAAAAAAACBIAFCAA|1932-07-09|390|1697|131|1932|6|7|9|3|1932|131|1697|Saturday|1932Q3|N|Y|N|2426890|2427071|2426532|2426807|N|N|N|N|N| +2426899|AAAAAAAADBIAFCAA|1932-07-10|390|1697|131|1932|0|7|10|3|1932|131|1697|Sunday|1932Q3|N|N|N|2426890|2427071|2426533|2426808|N|N|N|N|N| +2426900|AAAAAAAAEBIAFCAA|1932-07-11|390|1697|131|1932|1|7|11|3|1932|131|1697|Monday|1932Q3|N|N|N|2426890|2427071|2426534|2426809|N|N|N|N|N| +2426901|AAAAAAAAFBIAFCAA|1932-07-12|390|1698|131|1932|2|7|12|3|1932|131|1698|Tuesday|1932Q3|N|N|N|2426890|2427071|2426535|2426810|N|N|N|N|N| +2426902|AAAAAAAAGBIAFCAA|1932-07-13|390|1698|131|1932|3|7|13|3|1932|131|1698|Wednesday|1932Q3|N|N|N|2426890|2427071|2426536|2426811|N|N|N|N|N| +2426903|AAAAAAAAHBIAFCAA|1932-07-14|390|1698|131|1932|4|7|14|3|1932|131|1698|Thursday|1932Q3|N|N|N|2426890|2427071|2426537|2426812|N|N|N|N|N| +2426904|AAAAAAAAIBIAFCAA|1932-07-15|390|1698|131|1932|5|7|15|3|1932|131|1698|Friday|1932Q3|N|Y|N|2426890|2427071|2426538|2426813|N|N|N|N|N| +2426905|AAAAAAAAJBIAFCAA|1932-07-16|390|1698|131|1932|6|7|16|3|1932|131|1698|Saturday|1932Q3|N|Y|N|2426890|2427071|2426539|2426814|N|N|N|N|N| +2426906|AAAAAAAAKBIAFCAA|1932-07-17|390|1698|131|1932|0|7|17|3|1932|131|1698|Sunday|1932Q3|N|N|N|2426890|2427071|2426540|2426815|N|N|N|N|N| +2426907|AAAAAAAALBIAFCAA|1932-07-18|390|1698|131|1932|1|7|18|3|1932|131|1698|Monday|1932Q3|N|N|N|2426890|2427071|2426541|2426816|N|N|N|N|N| +2426908|AAAAAAAAMBIAFCAA|1932-07-19|390|1699|131|1932|2|7|19|3|1932|131|1699|Tuesday|1932Q3|N|N|N|2426890|2427071|2426542|2426817|N|N|N|N|N| +2426909|AAAAAAAANBIAFCAA|1932-07-20|390|1699|131|1932|3|7|20|3|1932|131|1699|Wednesday|1932Q3|N|N|N|2426890|2427071|2426543|2426818|N|N|N|N|N| +2426910|AAAAAAAAOBIAFCAA|1932-07-21|390|1699|131|1932|4|7|21|3|1932|131|1699|Thursday|1932Q3|N|N|N|2426890|2427071|2426544|2426819|N|N|N|N|N| +2426911|AAAAAAAAPBIAFCAA|1932-07-22|390|1699|131|1932|5|7|22|3|1932|131|1699|Friday|1932Q3|N|Y|N|2426890|2427071|2426545|2426820|N|N|N|N|N| +2426912|AAAAAAAAACIAFCAA|1932-07-23|390|1699|131|1932|6|7|23|3|1932|131|1699|Saturday|1932Q3|N|Y|N|2426890|2427071|2426546|2426821|N|N|N|N|N| +2426913|AAAAAAAABCIAFCAA|1932-07-24|390|1699|131|1932|0|7|24|3|1932|131|1699|Sunday|1932Q3|N|N|N|2426890|2427071|2426547|2426822|N|N|N|N|N| +2426914|AAAAAAAACCIAFCAA|1932-07-25|390|1699|131|1932|1|7|25|3|1932|131|1699|Monday|1932Q3|N|N|N|2426890|2427071|2426548|2426823|N|N|N|N|N| +2426915|AAAAAAAADCIAFCAA|1932-07-26|390|1700|131|1932|2|7|26|3|1932|131|1700|Tuesday|1932Q3|N|N|N|2426890|2427071|2426549|2426824|N|N|N|N|N| +2426916|AAAAAAAAECIAFCAA|1932-07-27|390|1700|131|1932|3|7|27|3|1932|131|1700|Wednesday|1932Q3|N|N|N|2426890|2427071|2426550|2426825|N|N|N|N|N| +2426917|AAAAAAAAFCIAFCAA|1932-07-28|390|1700|131|1932|4|7|28|3|1932|131|1700|Thursday|1932Q3|N|N|N|2426890|2427071|2426551|2426826|N|N|N|N|N| +2426918|AAAAAAAAGCIAFCAA|1932-07-29|390|1700|131|1932|5|7|29|3|1932|131|1700|Friday|1932Q3|N|Y|N|2426890|2427071|2426552|2426827|N|N|N|N|N| +2426919|AAAAAAAAHCIAFCAA|1932-07-30|390|1700|131|1932|6|7|30|3|1932|131|1700|Saturday|1932Q3|N|Y|N|2426890|2427071|2426553|2426828|N|N|N|N|N| +2426920|AAAAAAAAICIAFCAA|1932-07-31|390|1700|131|1932|0|7|31|3|1932|131|1700|Sunday|1932Q3|N|N|N|2426890|2427071|2426554|2426829|N|N|N|N|N| +2426921|AAAAAAAAJCIAFCAA|1932-08-01|391|1700|131|1932|1|8|1|3|1932|131|1700|Monday|1932Q3|N|N|N|2426921|2427133|2426555|2426830|N|N|N|N|N| +2426922|AAAAAAAAKCIAFCAA|1932-08-02|391|1701|131|1932|2|8|2|3|1932|131|1701|Tuesday|1932Q3|N|N|N|2426921|2427133|2426556|2426831|N|N|N|N|N| +2426923|AAAAAAAALCIAFCAA|1932-08-03|391|1701|131|1932|3|8|3|3|1932|131|1701|Wednesday|1932Q3|N|N|N|2426921|2427133|2426557|2426832|N|N|N|N|N| +2426924|AAAAAAAAMCIAFCAA|1932-08-04|391|1701|131|1932|4|8|4|3|1932|131|1701|Thursday|1932Q3|N|N|N|2426921|2427133|2426558|2426833|N|N|N|N|N| +2426925|AAAAAAAANCIAFCAA|1932-08-05|391|1701|131|1932|5|8|5|3|1932|131|1701|Friday|1932Q3|N|Y|N|2426921|2427133|2426559|2426834|N|N|N|N|N| +2426926|AAAAAAAAOCIAFCAA|1932-08-06|391|1701|131|1932|6|8|6|3|1932|131|1701|Saturday|1932Q3|N|Y|N|2426921|2427133|2426560|2426835|N|N|N|N|N| +2426927|AAAAAAAAPCIAFCAA|1932-08-07|391|1701|131|1932|0|8|7|3|1932|131|1701|Sunday|1932Q3|N|N|N|2426921|2427133|2426561|2426836|N|N|N|N|N| +2426928|AAAAAAAAADIAFCAA|1932-08-08|391|1701|131|1932|1|8|8|3|1932|131|1701|Monday|1932Q3|N|N|N|2426921|2427133|2426562|2426837|N|N|N|N|N| +2426929|AAAAAAAABDIAFCAA|1932-08-09|391|1702|131|1932|2|8|9|3|1932|131|1702|Tuesday|1932Q3|N|N|N|2426921|2427133|2426563|2426838|N|N|N|N|N| +2426930|AAAAAAAACDIAFCAA|1932-08-10|391|1702|131|1932|3|8|10|3|1932|131|1702|Wednesday|1932Q3|N|N|N|2426921|2427133|2426564|2426839|N|N|N|N|N| +2426931|AAAAAAAADDIAFCAA|1932-08-11|391|1702|131|1932|4|8|11|3|1932|131|1702|Thursday|1932Q3|N|N|N|2426921|2427133|2426565|2426840|N|N|N|N|N| +2426932|AAAAAAAAEDIAFCAA|1932-08-12|391|1702|131|1932|5|8|12|3|1932|131|1702|Friday|1932Q3|N|Y|N|2426921|2427133|2426566|2426841|N|N|N|N|N| +2426933|AAAAAAAAFDIAFCAA|1932-08-13|391|1702|131|1932|6|8|13|3|1932|131|1702|Saturday|1932Q3|N|Y|N|2426921|2427133|2426567|2426842|N|N|N|N|N| +2426934|AAAAAAAAGDIAFCAA|1932-08-14|391|1702|131|1932|0|8|14|3|1932|131|1702|Sunday|1932Q3|N|N|N|2426921|2427133|2426568|2426843|N|N|N|N|N| +2426935|AAAAAAAAHDIAFCAA|1932-08-15|391|1702|131|1932|1|8|15|3|1932|131|1702|Monday|1932Q3|N|N|N|2426921|2427133|2426569|2426844|N|N|N|N|N| +2426936|AAAAAAAAIDIAFCAA|1932-08-16|391|1703|131|1932|2|8|16|3|1932|131|1703|Tuesday|1932Q3|N|N|N|2426921|2427133|2426570|2426845|N|N|N|N|N| +2426937|AAAAAAAAJDIAFCAA|1932-08-17|391|1703|131|1932|3|8|17|3|1932|131|1703|Wednesday|1932Q3|N|N|N|2426921|2427133|2426571|2426846|N|N|N|N|N| +2426938|AAAAAAAAKDIAFCAA|1932-08-18|391|1703|131|1932|4|8|18|3|1932|131|1703|Thursday|1932Q3|N|N|N|2426921|2427133|2426572|2426847|N|N|N|N|N| +2426939|AAAAAAAALDIAFCAA|1932-08-19|391|1703|131|1932|5|8|19|3|1932|131|1703|Friday|1932Q3|N|Y|N|2426921|2427133|2426573|2426848|N|N|N|N|N| +2426940|AAAAAAAAMDIAFCAA|1932-08-20|391|1703|131|1932|6|8|20|3|1932|131|1703|Saturday|1932Q3|N|Y|N|2426921|2427133|2426574|2426849|N|N|N|N|N| +2426941|AAAAAAAANDIAFCAA|1932-08-21|391|1703|131|1932|0|8|21|3|1932|131|1703|Sunday|1932Q3|N|N|N|2426921|2427133|2426575|2426850|N|N|N|N|N| +2426942|AAAAAAAAODIAFCAA|1932-08-22|391|1703|131|1932|1|8|22|3|1932|131|1703|Monday|1932Q3|N|N|N|2426921|2427133|2426576|2426851|N|N|N|N|N| +2426943|AAAAAAAAPDIAFCAA|1932-08-23|391|1704|131|1932|2|8|23|3|1932|131|1704|Tuesday|1932Q3|N|N|N|2426921|2427133|2426577|2426852|N|N|N|N|N| +2426944|AAAAAAAAAEIAFCAA|1932-08-24|391|1704|131|1932|3|8|24|3|1932|131|1704|Wednesday|1932Q3|N|N|N|2426921|2427133|2426578|2426853|N|N|N|N|N| +2426945|AAAAAAAABEIAFCAA|1932-08-25|391|1704|131|1932|4|8|25|3|1932|131|1704|Thursday|1932Q3|N|N|N|2426921|2427133|2426579|2426854|N|N|N|N|N| +2426946|AAAAAAAACEIAFCAA|1932-08-26|391|1704|131|1932|5|8|26|3|1932|131|1704|Friday|1932Q3|N|Y|N|2426921|2427133|2426580|2426855|N|N|N|N|N| +2426947|AAAAAAAADEIAFCAA|1932-08-27|391|1704|131|1932|6|8|27|3|1932|131|1704|Saturday|1932Q3|N|Y|N|2426921|2427133|2426581|2426856|N|N|N|N|N| +2426948|AAAAAAAAEEIAFCAA|1932-08-28|391|1704|131|1932|0|8|28|3|1932|131|1704|Sunday|1932Q3|N|N|N|2426921|2427133|2426582|2426857|N|N|N|N|N| +2426949|AAAAAAAAFEIAFCAA|1932-08-29|391|1704|131|1932|1|8|29|3|1932|131|1704|Monday|1932Q3|N|N|N|2426921|2427133|2426583|2426858|N|N|N|N|N| +2426950|AAAAAAAAGEIAFCAA|1932-08-30|391|1705|131|1932|2|8|30|3|1932|131|1705|Tuesday|1932Q3|N|N|N|2426921|2427133|2426584|2426859|N|N|N|N|N| +2426951|AAAAAAAAHEIAFCAA|1932-08-31|391|1705|131|1932|3|8|31|3|1932|131|1705|Wednesday|1932Q3|N|N|N|2426921|2427133|2426585|2426860|N|N|N|N|N| +2426952|AAAAAAAAIEIAFCAA|1932-09-01|392|1705|132|1932|4|9|1|3|1932|132|1705|Thursday|1932Q3|N|N|N|2426952|2427195|2426586|2426861|N|N|N|N|N| +2426953|AAAAAAAAJEIAFCAA|1932-09-02|392|1705|132|1932|5|9|2|3|1932|132|1705|Friday|1932Q3|N|Y|N|2426952|2427195|2426587|2426862|N|N|N|N|N| +2426954|AAAAAAAAKEIAFCAA|1932-09-03|392|1705|132|1932|6|9|3|3|1932|132|1705|Saturday|1932Q3|N|Y|N|2426952|2427195|2426588|2426863|N|N|N|N|N| +2426955|AAAAAAAALEIAFCAA|1932-09-04|392|1705|132|1932|0|9|4|3|1932|132|1705|Sunday|1932Q3|N|N|N|2426952|2427195|2426589|2426864|N|N|N|N|N| +2426956|AAAAAAAAMEIAFCAA|1932-09-05|392|1705|132|1932|1|9|5|3|1932|132|1705|Monday|1932Q3|N|N|N|2426952|2427195|2426590|2426865|N|N|N|N|N| +2426957|AAAAAAAANEIAFCAA|1932-09-06|392|1706|132|1932|2|9|6|3|1932|132|1706|Tuesday|1932Q3|N|N|N|2426952|2427195|2426591|2426866|N|N|N|N|N| +2426958|AAAAAAAAOEIAFCAA|1932-09-07|392|1706|132|1932|3|9|7|3|1932|132|1706|Wednesday|1932Q3|N|N|N|2426952|2427195|2426592|2426867|N|N|N|N|N| +2426959|AAAAAAAAPEIAFCAA|1932-09-08|392|1706|132|1932|4|9|8|3|1932|132|1706|Thursday|1932Q3|N|N|N|2426952|2427195|2426593|2426868|N|N|N|N|N| +2426960|AAAAAAAAAFIAFCAA|1932-09-09|392|1706|132|1932|5|9|9|3|1932|132|1706|Friday|1932Q3|N|Y|N|2426952|2427195|2426594|2426869|N|N|N|N|N| +2426961|AAAAAAAABFIAFCAA|1932-09-10|392|1706|132|1932|6|9|10|3|1932|132|1706|Saturday|1932Q3|N|Y|N|2426952|2427195|2426595|2426870|N|N|N|N|N| +2426962|AAAAAAAACFIAFCAA|1932-09-11|392|1706|132|1932|0|9|11|3|1932|132|1706|Sunday|1932Q3|N|N|N|2426952|2427195|2426596|2426871|N|N|N|N|N| +2426963|AAAAAAAADFIAFCAA|1932-09-12|392|1706|132|1932|1|9|12|3|1932|132|1706|Monday|1932Q3|N|N|N|2426952|2427195|2426597|2426872|N|N|N|N|N| +2426964|AAAAAAAAEFIAFCAA|1932-09-13|392|1707|132|1932|2|9|13|3|1932|132|1707|Tuesday|1932Q3|N|N|N|2426952|2427195|2426598|2426873|N|N|N|N|N| +2426965|AAAAAAAAFFIAFCAA|1932-09-14|392|1707|132|1932|3|9|14|3|1932|132|1707|Wednesday|1932Q3|N|N|N|2426952|2427195|2426599|2426874|N|N|N|N|N| +2426966|AAAAAAAAGFIAFCAA|1932-09-15|392|1707|132|1932|4|9|15|3|1932|132|1707|Thursday|1932Q3|N|N|N|2426952|2427195|2426600|2426875|N|N|N|N|N| +2426967|AAAAAAAAHFIAFCAA|1932-09-16|392|1707|132|1932|5|9|16|3|1932|132|1707|Friday|1932Q3|N|Y|N|2426952|2427195|2426601|2426876|N|N|N|N|N| +2426968|AAAAAAAAIFIAFCAA|1932-09-17|392|1707|132|1932|6|9|17|3|1932|132|1707|Saturday|1932Q3|N|Y|N|2426952|2427195|2426602|2426877|N|N|N|N|N| +2426969|AAAAAAAAJFIAFCAA|1932-09-18|392|1707|132|1932|0|9|18|3|1932|132|1707|Sunday|1932Q3|N|N|N|2426952|2427195|2426603|2426878|N|N|N|N|N| +2426970|AAAAAAAAKFIAFCAA|1932-09-19|392|1707|132|1932|1|9|19|3|1932|132|1707|Monday|1932Q3|N|N|N|2426952|2427195|2426604|2426879|N|N|N|N|N| +2426971|AAAAAAAALFIAFCAA|1932-09-20|392|1708|132|1932|2|9|20|3|1932|132|1708|Tuesday|1932Q3|N|N|N|2426952|2427195|2426605|2426880|N|N|N|N|N| +2426972|AAAAAAAAMFIAFCAA|1932-09-21|392|1708|132|1932|3|9|21|3|1932|132|1708|Wednesday|1932Q3|N|N|N|2426952|2427195|2426606|2426881|N|N|N|N|N| +2426973|AAAAAAAANFIAFCAA|1932-09-22|392|1708|132|1932|4|9|22|3|1932|132|1708|Thursday|1932Q3|N|N|N|2426952|2427195|2426607|2426882|N|N|N|N|N| +2426974|AAAAAAAAOFIAFCAA|1932-09-23|392|1708|132|1932|5|9|23|3|1932|132|1708|Friday|1932Q3|N|Y|N|2426952|2427195|2426608|2426883|N|N|N|N|N| +2426975|AAAAAAAAPFIAFCAA|1932-09-24|392|1708|132|1932|6|9|24|3|1932|132|1708|Saturday|1932Q3|N|Y|N|2426952|2427195|2426609|2426884|N|N|N|N|N| +2426976|AAAAAAAAAGIAFCAA|1932-09-25|392|1708|132|1932|0|9|25|3|1932|132|1708|Sunday|1932Q3|N|N|N|2426952|2427195|2426610|2426885|N|N|N|N|N| +2426977|AAAAAAAABGIAFCAA|1932-09-26|392|1708|132|1932|1|9|26|3|1932|132|1708|Monday|1932Q3|N|N|N|2426952|2427195|2426611|2426886|N|N|N|N|N| +2426978|AAAAAAAACGIAFCAA|1932-09-27|392|1709|132|1932|2|9|27|3|1932|132|1709|Tuesday|1932Q3|N|N|N|2426952|2427195|2426612|2426887|N|N|N|N|N| +2426979|AAAAAAAADGIAFCAA|1932-09-28|392|1709|132|1932|3|9|28|3|1932|132|1709|Wednesday|1932Q3|N|N|N|2426952|2427195|2426613|2426888|N|N|N|N|N| +2426980|AAAAAAAAEGIAFCAA|1932-09-29|392|1709|132|1932|4|9|29|3|1932|132|1709|Thursday|1932Q3|N|N|N|2426952|2427195|2426614|2426889|N|N|N|N|N| +2426981|AAAAAAAAFGIAFCAA|1932-09-30|392|1709|132|1932|5|9|30|3|1932|132|1709|Friday|1932Q3|N|Y|N|2426952|2427195|2426615|2426890|N|N|N|N|N| +2426982|AAAAAAAAGGIAFCAA|1932-10-01|393|1709|132|1932|6|10|1|4|1932|132|1709|Saturday|1932Q4|N|Y|N|2426982|2427255|2426616|2426890|N|N|N|N|N| +2426983|AAAAAAAAHGIAFCAA|1932-10-02|393|1709|132|1932|0|10|2|4|1932|132|1709|Sunday|1932Q4|N|N|N|2426982|2427255|2426617|2426891|N|N|N|N|N| +2426984|AAAAAAAAIGIAFCAA|1932-10-03|393|1709|132|1932|1|10|3|4|1932|132|1709|Monday|1932Q4|N|N|N|2426982|2427255|2426618|2426892|N|N|N|N|N| +2426985|AAAAAAAAJGIAFCAA|1932-10-04|393|1710|132|1932|2|10|4|4|1932|132|1710|Tuesday|1932Q4|N|N|N|2426982|2427255|2426619|2426893|N|N|N|N|N| +2426986|AAAAAAAAKGIAFCAA|1932-10-05|393|1710|132|1932|3|10|5|4|1932|132|1710|Wednesday|1932Q4|N|N|N|2426982|2427255|2426620|2426894|N|N|N|N|N| +2426987|AAAAAAAALGIAFCAA|1932-10-06|393|1710|132|1932|4|10|6|4|1932|132|1710|Thursday|1932Q4|N|N|N|2426982|2427255|2426621|2426895|N|N|N|N|N| +2426988|AAAAAAAAMGIAFCAA|1932-10-07|393|1710|132|1932|5|10|7|4|1932|132|1710|Friday|1932Q4|N|Y|N|2426982|2427255|2426622|2426896|N|N|N|N|N| +2426989|AAAAAAAANGIAFCAA|1932-10-08|393|1710|132|1932|6|10|8|4|1932|132|1710|Saturday|1932Q4|N|Y|N|2426982|2427255|2426623|2426897|N|N|N|N|N| +2426990|AAAAAAAAOGIAFCAA|1932-10-09|393|1710|132|1932|0|10|9|4|1932|132|1710|Sunday|1932Q4|N|N|N|2426982|2427255|2426624|2426898|N|N|N|N|N| +2426991|AAAAAAAAPGIAFCAA|1932-10-10|393|1710|132|1932|1|10|10|4|1932|132|1710|Monday|1932Q4|N|N|N|2426982|2427255|2426625|2426899|N|N|N|N|N| +2426992|AAAAAAAAAHIAFCAA|1932-10-11|393|1711|132|1932|2|10|11|4|1932|132|1711|Tuesday|1932Q4|N|N|N|2426982|2427255|2426626|2426900|N|N|N|N|N| +2426993|AAAAAAAABHIAFCAA|1932-10-12|393|1711|132|1932|3|10|12|4|1932|132|1711|Wednesday|1932Q4|N|N|N|2426982|2427255|2426627|2426901|N|N|N|N|N| +2426994|AAAAAAAACHIAFCAA|1932-10-13|393|1711|132|1932|4|10|13|4|1932|132|1711|Thursday|1932Q4|N|N|N|2426982|2427255|2426628|2426902|N|N|N|N|N| +2426995|AAAAAAAADHIAFCAA|1932-10-14|393|1711|132|1932|5|10|14|4|1932|132|1711|Friday|1932Q4|N|Y|N|2426982|2427255|2426629|2426903|N|N|N|N|N| +2426996|AAAAAAAAEHIAFCAA|1932-10-15|393|1711|132|1932|6|10|15|4|1932|132|1711|Saturday|1932Q4|N|Y|N|2426982|2427255|2426630|2426904|N|N|N|N|N| +2426997|AAAAAAAAFHIAFCAA|1932-10-16|393|1711|132|1932|0|10|16|4|1932|132|1711|Sunday|1932Q4|N|N|N|2426982|2427255|2426631|2426905|N|N|N|N|N| +2426998|AAAAAAAAGHIAFCAA|1932-10-17|393|1711|132|1932|1|10|17|4|1932|132|1711|Monday|1932Q4|N|N|N|2426982|2427255|2426632|2426906|N|N|N|N|N| +2426999|AAAAAAAAHHIAFCAA|1932-10-18|393|1712|132|1932|2|10|18|4|1932|132|1712|Tuesday|1932Q4|N|N|N|2426982|2427255|2426633|2426907|N|N|N|N|N| +2427000|AAAAAAAAIHIAFCAA|1932-10-19|393|1712|132|1932|3|10|19|4|1932|132|1712|Wednesday|1932Q4|N|N|N|2426982|2427255|2426634|2426908|N|N|N|N|N| +2427001|AAAAAAAAJHIAFCAA|1932-10-20|393|1712|132|1932|4|10|20|4|1932|132|1712|Thursday|1932Q4|N|N|N|2426982|2427255|2426635|2426909|N|N|N|N|N| +2427002|AAAAAAAAKHIAFCAA|1932-10-21|393|1712|132|1932|5|10|21|4|1932|132|1712|Friday|1932Q4|N|Y|N|2426982|2427255|2426636|2426910|N|N|N|N|N| +2427003|AAAAAAAALHIAFCAA|1932-10-22|393|1712|132|1932|6|10|22|4|1932|132|1712|Saturday|1932Q4|N|Y|N|2426982|2427255|2426637|2426911|N|N|N|N|N| +2427004|AAAAAAAAMHIAFCAA|1932-10-23|393|1712|132|1932|0|10|23|4|1932|132|1712|Sunday|1932Q4|N|N|N|2426982|2427255|2426638|2426912|N|N|N|N|N| +2427005|AAAAAAAANHIAFCAA|1932-10-24|393|1712|132|1932|1|10|24|4|1932|132|1712|Monday|1932Q4|N|N|N|2426982|2427255|2426639|2426913|N|N|N|N|N| +2427006|AAAAAAAAOHIAFCAA|1932-10-25|393|1713|132|1932|2|10|25|4|1932|132|1713|Tuesday|1932Q4|N|N|N|2426982|2427255|2426640|2426914|N|N|N|N|N| +2427007|AAAAAAAAPHIAFCAA|1932-10-26|393|1713|132|1932|3|10|26|4|1932|132|1713|Wednesday|1932Q4|N|N|N|2426982|2427255|2426641|2426915|N|N|N|N|N| +2427008|AAAAAAAAAIIAFCAA|1932-10-27|393|1713|132|1932|4|10|27|4|1932|132|1713|Thursday|1932Q4|N|N|N|2426982|2427255|2426642|2426916|N|N|N|N|N| +2427009|AAAAAAAABIIAFCAA|1932-10-28|393|1713|132|1932|5|10|28|4|1932|132|1713|Friday|1932Q4|N|Y|N|2426982|2427255|2426643|2426917|N|N|N|N|N| +2427010|AAAAAAAACIIAFCAA|1932-10-29|393|1713|132|1932|6|10|29|4|1932|132|1713|Saturday|1932Q4|N|Y|N|2426982|2427255|2426644|2426918|N|N|N|N|N| +2427011|AAAAAAAADIIAFCAA|1932-10-30|393|1713|132|1932|0|10|30|4|1932|132|1713|Sunday|1932Q4|N|N|N|2426982|2427255|2426645|2426919|N|N|N|N|N| +2427012|AAAAAAAAEIIAFCAA|1932-10-31|393|1713|132|1932|1|10|31|4|1932|132|1713|Monday|1932Q4|N|N|N|2426982|2427255|2426646|2426920|N|N|N|N|N| +2427013|AAAAAAAAFIIAFCAA|1932-11-01|394|1714|132|1932|2|11|1|4|1932|132|1714|Tuesday|1932Q4|N|N|N|2427013|2427317|2426647|2426921|N|N|N|N|N| +2427014|AAAAAAAAGIIAFCAA|1932-11-02|394|1714|132|1932|3|11|2|4|1932|132|1714|Wednesday|1932Q4|N|N|N|2427013|2427317|2426648|2426922|N|N|N|N|N| +2427015|AAAAAAAAHIIAFCAA|1932-11-03|394|1714|132|1932|4|11|3|4|1932|132|1714|Thursday|1932Q4|N|N|N|2427013|2427317|2426649|2426923|N|N|N|N|N| +2427016|AAAAAAAAIIIAFCAA|1932-11-04|394|1714|132|1932|5|11|4|4|1932|132|1714|Friday|1932Q4|N|Y|N|2427013|2427317|2426650|2426924|N|N|N|N|N| +2427017|AAAAAAAAJIIAFCAA|1932-11-05|394|1714|132|1932|6|11|5|4|1932|132|1714|Saturday|1932Q4|N|Y|N|2427013|2427317|2426651|2426925|N|N|N|N|N| +2427018|AAAAAAAAKIIAFCAA|1932-11-06|394|1714|132|1932|0|11|6|4|1932|132|1714|Sunday|1932Q4|N|N|N|2427013|2427317|2426652|2426926|N|N|N|N|N| +2427019|AAAAAAAALIIAFCAA|1932-11-07|394|1714|132|1932|1|11|7|4|1932|132|1714|Monday|1932Q4|N|N|N|2427013|2427317|2426653|2426927|N|N|N|N|N| +2427020|AAAAAAAAMIIAFCAA|1932-11-08|394|1715|132|1932|2|11|8|4|1932|132|1715|Tuesday|1932Q4|N|N|N|2427013|2427317|2426654|2426928|N|N|N|N|N| +2427021|AAAAAAAANIIAFCAA|1932-11-09|394|1715|132|1932|3|11|9|4|1932|132|1715|Wednesday|1932Q4|N|N|N|2427013|2427317|2426655|2426929|N|N|N|N|N| +2427022|AAAAAAAAOIIAFCAA|1932-11-10|394|1715|132|1932|4|11|10|4|1932|132|1715|Thursday|1932Q4|N|N|N|2427013|2427317|2426656|2426930|N|N|N|N|N| +2427023|AAAAAAAAPIIAFCAA|1932-11-11|394|1715|132|1932|5|11|11|4|1932|132|1715|Friday|1932Q4|N|Y|N|2427013|2427317|2426657|2426931|N|N|N|N|N| +2427024|AAAAAAAAAJIAFCAA|1932-11-12|394|1715|132|1932|6|11|12|4|1932|132|1715|Saturday|1932Q4|N|Y|N|2427013|2427317|2426658|2426932|N|N|N|N|N| +2427025|AAAAAAAABJIAFCAA|1932-11-13|394|1715|132|1932|0|11|13|4|1932|132|1715|Sunday|1932Q4|N|N|N|2427013|2427317|2426659|2426933|N|N|N|N|N| +2427026|AAAAAAAACJIAFCAA|1932-11-14|394|1715|132|1932|1|11|14|4|1932|132|1715|Monday|1932Q4|N|N|N|2427013|2427317|2426660|2426934|N|N|N|N|N| +2427027|AAAAAAAADJIAFCAA|1932-11-15|394|1716|132|1932|2|11|15|4|1932|132|1716|Tuesday|1932Q4|N|N|N|2427013|2427317|2426661|2426935|N|N|N|N|N| +2427028|AAAAAAAAEJIAFCAA|1932-11-16|394|1716|132|1932|3|11|16|4|1932|132|1716|Wednesday|1932Q4|N|N|N|2427013|2427317|2426662|2426936|N|N|N|N|N| +2427029|AAAAAAAAFJIAFCAA|1932-11-17|394|1716|132|1932|4|11|17|4|1932|132|1716|Thursday|1932Q4|N|N|N|2427013|2427317|2426663|2426937|N|N|N|N|N| +2427030|AAAAAAAAGJIAFCAA|1932-11-18|394|1716|132|1932|5|11|18|4|1932|132|1716|Friday|1932Q4|N|Y|N|2427013|2427317|2426664|2426938|N|N|N|N|N| +2427031|AAAAAAAAHJIAFCAA|1932-11-19|394|1716|132|1932|6|11|19|4|1932|132|1716|Saturday|1932Q4|N|Y|N|2427013|2427317|2426665|2426939|N|N|N|N|N| +2427032|AAAAAAAAIJIAFCAA|1932-11-20|394|1716|132|1932|0|11|20|4|1932|132|1716|Sunday|1932Q4|N|N|N|2427013|2427317|2426666|2426940|N|N|N|N|N| +2427033|AAAAAAAAJJIAFCAA|1932-11-21|394|1716|132|1932|1|11|21|4|1932|132|1716|Monday|1932Q4|N|N|N|2427013|2427317|2426667|2426941|N|N|N|N|N| +2427034|AAAAAAAAKJIAFCAA|1932-11-22|394|1717|132|1932|2|11|22|4|1932|132|1717|Tuesday|1932Q4|N|N|N|2427013|2427317|2426668|2426942|N|N|N|N|N| +2427035|AAAAAAAALJIAFCAA|1932-11-23|394|1717|132|1932|3|11|23|4|1932|132|1717|Wednesday|1932Q4|N|N|N|2427013|2427317|2426669|2426943|N|N|N|N|N| +2427036|AAAAAAAAMJIAFCAA|1932-11-24|394|1717|132|1932|4|11|24|4|1932|132|1717|Thursday|1932Q4|N|N|N|2427013|2427317|2426670|2426944|N|N|N|N|N| +2427037|AAAAAAAANJIAFCAA|1932-11-25|394|1717|132|1932|5|11|25|4|1932|132|1717|Friday|1932Q4|N|Y|N|2427013|2427317|2426671|2426945|N|N|N|N|N| +2427038|AAAAAAAAOJIAFCAA|1932-11-26|394|1717|132|1932|6|11|26|4|1932|132|1717|Saturday|1932Q4|N|Y|N|2427013|2427317|2426672|2426946|N|N|N|N|N| +2427039|AAAAAAAAPJIAFCAA|1932-11-27|394|1717|132|1932|0|11|27|4|1932|132|1717|Sunday|1932Q4|N|N|N|2427013|2427317|2426673|2426947|N|N|N|N|N| +2427040|AAAAAAAAAKIAFCAA|1932-11-28|394|1717|132|1932|1|11|28|4|1932|132|1717|Monday|1932Q4|N|N|N|2427013|2427317|2426674|2426948|N|N|N|N|N| +2427041|AAAAAAAABKIAFCAA|1932-11-29|394|1718|132|1932|2|11|29|4|1932|132|1718|Tuesday|1932Q4|N|N|N|2427013|2427317|2426675|2426949|N|N|N|N|N| +2427042|AAAAAAAACKIAFCAA|1932-11-30|394|1718|132|1932|3|11|30|4|1932|132|1718|Wednesday|1932Q4|N|N|N|2427013|2427317|2426676|2426950|N|N|N|N|N| +2427043|AAAAAAAADKIAFCAA|1932-12-01|395|1718|133|1932|4|12|1|4|1932|133|1718|Thursday|1932Q4|N|N|N|2427043|2427377|2426677|2426951|N|N|N|N|N| +2427044|AAAAAAAAEKIAFCAA|1932-12-02|395|1718|133|1932|5|12|2|4|1932|133|1718|Friday|1932Q4|N|Y|N|2427043|2427377|2426678|2426952|N|N|N|N|N| +2427045|AAAAAAAAFKIAFCAA|1932-12-03|395|1718|133|1932|6|12|3|4|1932|133|1718|Saturday|1932Q4|N|Y|N|2427043|2427377|2426679|2426953|N|N|N|N|N| +2427046|AAAAAAAAGKIAFCAA|1932-12-04|395|1718|133|1932|0|12|4|4|1932|133|1718|Sunday|1932Q4|N|N|N|2427043|2427377|2426680|2426954|N|N|N|N|N| +2427047|AAAAAAAAHKIAFCAA|1932-12-05|395|1718|133|1932|1|12|5|4|1932|133|1718|Monday|1932Q4|N|N|N|2427043|2427377|2426681|2426955|N|N|N|N|N| +2427048|AAAAAAAAIKIAFCAA|1932-12-06|395|1719|133|1932|2|12|6|4|1932|133|1719|Tuesday|1932Q4|N|N|N|2427043|2427377|2426682|2426956|N|N|N|N|N| +2427049|AAAAAAAAJKIAFCAA|1932-12-07|395|1719|133|1932|3|12|7|4|1932|133|1719|Wednesday|1932Q4|N|N|N|2427043|2427377|2426683|2426957|N|N|N|N|N| +2427050|AAAAAAAAKKIAFCAA|1932-12-08|395|1719|133|1932|4|12|8|4|1932|133|1719|Thursday|1932Q4|N|N|N|2427043|2427377|2426684|2426958|N|N|N|N|N| +2427051|AAAAAAAALKIAFCAA|1932-12-09|395|1719|133|1932|5|12|9|4|1932|133|1719|Friday|1932Q4|N|Y|N|2427043|2427377|2426685|2426959|N|N|N|N|N| +2427052|AAAAAAAAMKIAFCAA|1932-12-10|395|1719|133|1932|6|12|10|4|1932|133|1719|Saturday|1932Q4|N|Y|N|2427043|2427377|2426686|2426960|N|N|N|N|N| +2427053|AAAAAAAANKIAFCAA|1932-12-11|395|1719|133|1932|0|12|11|4|1932|133|1719|Sunday|1932Q4|N|N|N|2427043|2427377|2426687|2426961|N|N|N|N|N| +2427054|AAAAAAAAOKIAFCAA|1932-12-12|395|1719|133|1932|1|12|12|4|1932|133|1719|Monday|1932Q4|N|N|N|2427043|2427377|2426688|2426962|N|N|N|N|N| +2427055|AAAAAAAAPKIAFCAA|1932-12-13|395|1720|133|1932|2|12|13|4|1932|133|1720|Tuesday|1932Q4|N|N|N|2427043|2427377|2426689|2426963|N|N|N|N|N| +2427056|AAAAAAAAALIAFCAA|1932-12-14|395|1720|133|1932|3|12|14|4|1932|133|1720|Wednesday|1932Q4|N|N|N|2427043|2427377|2426690|2426964|N|N|N|N|N| +2427057|AAAAAAAABLIAFCAA|1932-12-15|395|1720|133|1932|4|12|15|4|1932|133|1720|Thursday|1932Q4|N|N|N|2427043|2427377|2426691|2426965|N|N|N|N|N| +2427058|AAAAAAAACLIAFCAA|1932-12-16|395|1720|133|1932|5|12|16|4|1932|133|1720|Friday|1932Q4|N|Y|N|2427043|2427377|2426692|2426966|N|N|N|N|N| +2427059|AAAAAAAADLIAFCAA|1932-12-17|395|1720|133|1932|6|12|17|4|1932|133|1720|Saturday|1932Q4|N|Y|N|2427043|2427377|2426693|2426967|N|N|N|N|N| +2427060|AAAAAAAAELIAFCAA|1932-12-18|395|1720|133|1932|0|12|18|4|1932|133|1720|Sunday|1932Q4|N|N|N|2427043|2427377|2426694|2426968|N|N|N|N|N| +2427061|AAAAAAAAFLIAFCAA|1932-12-19|395|1720|133|1932|1|12|19|4|1932|133|1720|Monday|1932Q4|N|N|N|2427043|2427377|2426695|2426969|N|N|N|N|N| +2427062|AAAAAAAAGLIAFCAA|1932-12-20|395|1721|133|1932|2|12|20|4|1932|133|1721|Tuesday|1932Q4|N|N|N|2427043|2427377|2426696|2426970|N|N|N|N|N| +2427063|AAAAAAAAHLIAFCAA|1932-12-21|395|1721|133|1932|3|12|21|4|1932|133|1721|Wednesday|1932Q4|N|N|N|2427043|2427377|2426697|2426971|N|N|N|N|N| +2427064|AAAAAAAAILIAFCAA|1932-12-22|395|1721|133|1932|4|12|22|4|1932|133|1721|Thursday|1932Q4|N|N|N|2427043|2427377|2426698|2426972|N|N|N|N|N| +2427065|AAAAAAAAJLIAFCAA|1932-12-23|395|1721|133|1932|5|12|23|4|1932|133|1721|Friday|1932Q4|N|Y|N|2427043|2427377|2426699|2426973|N|N|N|N|N| +2427066|AAAAAAAAKLIAFCAA|1932-12-24|395|1721|133|1932|6|12|24|4|1932|133|1721|Saturday|1932Q4|N|Y|N|2427043|2427377|2426700|2426974|N|N|N|N|N| +2427067|AAAAAAAALLIAFCAA|1932-12-25|395|1721|133|1932|0|12|25|4|1932|133|1721|Sunday|1932Q4|Y|N|N|2427043|2427377|2426701|2426975|N|N|N|N|N| +2427068|AAAAAAAAMLIAFCAA|1932-12-26|395|1721|133|1932|1|12|26|4|1932|133|1721|Monday|1932Q4|N|N|Y|2427043|2427377|2426702|2426976|N|N|N|N|N| +2427069|AAAAAAAANLIAFCAA|1932-12-27|395|1722|133|1932|2|12|27|4|1932|133|1722|Tuesday|1932Q4|N|N|N|2427043|2427377|2426703|2426977|N|N|N|N|N| +2427070|AAAAAAAAOLIAFCAA|1932-12-28|395|1722|133|1932|3|12|28|4|1932|133|1722|Wednesday|1932Q4|N|N|N|2427043|2427377|2426704|2426978|N|N|N|N|N| +2427071|AAAAAAAAPLIAFCAA|1932-12-29|395|1722|133|1932|4|12|29|4|1932|133|1722|Thursday|1932Q4|N|N|N|2427043|2427377|2426705|2426979|N|N|N|N|N| +2427072|AAAAAAAAAMIAFCAA|1932-12-30|395|1722|133|1932|5|12|30|4|1932|133|1722|Friday|1932Q4|N|Y|N|2427043|2427377|2426706|2426980|N|N|N|N|N| +2427073|AAAAAAAABMIAFCAA|1932-12-31|395|1722|133|1932|6|12|31|4|1932|133|1722|Saturday|1932Q4|Y|Y|N|2427043|2427377|2426707|2426981|N|N|N|N|N| +2427074|AAAAAAAACMIAFCAA|1933-01-01|396|1722|133|1933|0|1|1|1|1933|133|1722|Sunday|1933Q1|Y|N|Y|2427074|2427073|2426708|2426982|N|N|N|N|N| +2427075|AAAAAAAADMIAFCAA|1933-01-02|396|1722|133|1933|1|1|2|1|1933|133|1722|Monday|1933Q1|N|N|Y|2427074|2427073|2426709|2426983|N|N|N|N|N| +2427076|AAAAAAAAEMIAFCAA|1933-01-03|396|1723|133|1933|2|1|3|1|1933|133|1723|Tuesday|1933Q1|N|N|N|2427074|2427073|2426710|2426984|N|N|N|N|N| +2427077|AAAAAAAAFMIAFCAA|1933-01-04|396|1723|133|1933|3|1|4|1|1933|133|1723|Wednesday|1933Q1|N|N|N|2427074|2427073|2426711|2426985|N|N|N|N|N| +2427078|AAAAAAAAGMIAFCAA|1933-01-05|396|1723|133|1933|4|1|5|1|1933|133|1723|Thursday|1933Q1|N|N|N|2427074|2427073|2426712|2426986|N|N|N|N|N| +2427079|AAAAAAAAHMIAFCAA|1933-01-06|396|1723|133|1933|5|1|6|1|1933|133|1723|Friday|1933Q1|N|Y|N|2427074|2427073|2426713|2426987|N|N|N|N|N| +2427080|AAAAAAAAIMIAFCAA|1933-01-07|396|1723|133|1933|6|1|7|1|1933|133|1723|Saturday|1933Q1|N|Y|N|2427074|2427073|2426714|2426988|N|N|N|N|N| +2427081|AAAAAAAAJMIAFCAA|1933-01-08|396|1723|133|1933|0|1|8|1|1933|133|1723|Sunday|1933Q1|N|N|N|2427074|2427073|2426715|2426989|N|N|N|N|N| +2427082|AAAAAAAAKMIAFCAA|1933-01-09|396|1723|133|1933|1|1|9|1|1933|133|1723|Monday|1933Q1|N|N|N|2427074|2427073|2426716|2426990|N|N|N|N|N| +2427083|AAAAAAAALMIAFCAA|1933-01-10|396|1724|133|1933|2|1|10|1|1933|133|1724|Tuesday|1933Q1|N|N|N|2427074|2427073|2426717|2426991|N|N|N|N|N| +2427084|AAAAAAAAMMIAFCAA|1933-01-11|396|1724|133|1933|3|1|11|1|1933|133|1724|Wednesday|1933Q1|N|N|N|2427074|2427073|2426718|2426992|N|N|N|N|N| +2427085|AAAAAAAANMIAFCAA|1933-01-12|396|1724|133|1933|4|1|12|1|1933|133|1724|Thursday|1933Q1|N|N|N|2427074|2427073|2426719|2426993|N|N|N|N|N| +2427086|AAAAAAAAOMIAFCAA|1933-01-13|396|1724|133|1933|5|1|13|1|1933|133|1724|Friday|1933Q1|N|Y|N|2427074|2427073|2426720|2426994|N|N|N|N|N| +2427087|AAAAAAAAPMIAFCAA|1933-01-14|396|1724|133|1933|6|1|14|1|1933|133|1724|Saturday|1933Q1|N|Y|N|2427074|2427073|2426721|2426995|N|N|N|N|N| +2427088|AAAAAAAAANIAFCAA|1933-01-15|396|1724|133|1933|0|1|15|1|1933|133|1724|Sunday|1933Q1|N|N|N|2427074|2427073|2426722|2426996|N|N|N|N|N| +2427089|AAAAAAAABNIAFCAA|1933-01-16|396|1724|133|1933|1|1|16|1|1933|133|1724|Monday|1933Q1|N|N|N|2427074|2427073|2426723|2426997|N|N|N|N|N| +2427090|AAAAAAAACNIAFCAA|1933-01-17|396|1725|133|1933|2|1|17|1|1933|133|1725|Tuesday|1933Q1|N|N|N|2427074|2427073|2426724|2426998|N|N|N|N|N| +2427091|AAAAAAAADNIAFCAA|1933-01-18|396|1725|133|1933|3|1|18|1|1933|133|1725|Wednesday|1933Q1|N|N|N|2427074|2427073|2426725|2426999|N|N|N|N|N| +2427092|AAAAAAAAENIAFCAA|1933-01-19|396|1725|133|1933|4|1|19|1|1933|133|1725|Thursday|1933Q1|N|N|N|2427074|2427073|2426726|2427000|N|N|N|N|N| +2427093|AAAAAAAAFNIAFCAA|1933-01-20|396|1725|133|1933|5|1|20|1|1933|133|1725|Friday|1933Q1|N|Y|N|2427074|2427073|2426727|2427001|N|N|N|N|N| +2427094|AAAAAAAAGNIAFCAA|1933-01-21|396|1725|133|1933|6|1|21|1|1933|133|1725|Saturday|1933Q1|N|Y|N|2427074|2427073|2426728|2427002|N|N|N|N|N| +2427095|AAAAAAAAHNIAFCAA|1933-01-22|396|1725|133|1933|0|1|22|1|1933|133|1725|Sunday|1933Q1|N|N|N|2427074|2427073|2426729|2427003|N|N|N|N|N| +2427096|AAAAAAAAINIAFCAA|1933-01-23|396|1725|133|1933|1|1|23|1|1933|133|1725|Monday|1933Q1|N|N|N|2427074|2427073|2426730|2427004|N|N|N|N|N| +2427097|AAAAAAAAJNIAFCAA|1933-01-24|396|1726|133|1933|2|1|24|1|1933|133|1726|Tuesday|1933Q1|N|N|N|2427074|2427073|2426731|2427005|N|N|N|N|N| +2427098|AAAAAAAAKNIAFCAA|1933-01-25|396|1726|133|1933|3|1|25|1|1933|133|1726|Wednesday|1933Q1|N|N|N|2427074|2427073|2426732|2427006|N|N|N|N|N| +2427099|AAAAAAAALNIAFCAA|1933-01-26|396|1726|133|1933|4|1|26|1|1933|133|1726|Thursday|1933Q1|N|N|N|2427074|2427073|2426733|2427007|N|N|N|N|N| +2427100|AAAAAAAAMNIAFCAA|1933-01-27|396|1726|133|1933|5|1|27|1|1933|133|1726|Friday|1933Q1|N|Y|N|2427074|2427073|2426734|2427008|N|N|N|N|N| +2427101|AAAAAAAANNIAFCAA|1933-01-28|396|1726|133|1933|6|1|28|1|1933|133|1726|Saturday|1933Q1|N|Y|N|2427074|2427073|2426735|2427009|N|N|N|N|N| +2427102|AAAAAAAAONIAFCAA|1933-01-29|396|1726|133|1933|0|1|29|1|1933|133|1726|Sunday|1933Q1|N|N|N|2427074|2427073|2426736|2427010|N|N|N|N|N| +2427103|AAAAAAAAPNIAFCAA|1933-01-30|396|1726|133|1933|1|1|30|1|1933|133|1726|Monday|1933Q1|N|N|N|2427074|2427073|2426737|2427011|N|N|N|N|N| +2427104|AAAAAAAAAOIAFCAA|1933-01-31|396|1727|133|1933|2|1|31|1|1933|133|1727|Tuesday|1933Q1|N|N|N|2427074|2427073|2426738|2427012|N|N|N|N|N| +2427105|AAAAAAAABOIAFCAA|1933-02-01|397|1727|133|1933|3|2|1|1|1933|133|1727|Wednesday|1933Q1|N|N|N|2427105|2427135|2426739|2427013|N|N|N|N|N| +2427106|AAAAAAAACOIAFCAA|1933-02-02|397|1727|133|1933|4|2|2|1|1933|133|1727|Thursday|1933Q1|N|N|N|2427105|2427135|2426740|2427014|N|N|N|N|N| +2427107|AAAAAAAADOIAFCAA|1933-02-03|397|1727|133|1933|5|2|3|1|1933|133|1727|Friday|1933Q1|N|Y|N|2427105|2427135|2426741|2427015|N|N|N|N|N| +2427108|AAAAAAAAEOIAFCAA|1933-02-04|397|1727|133|1933|6|2|4|1|1933|133|1727|Saturday|1933Q1|N|Y|N|2427105|2427135|2426742|2427016|N|N|N|N|N| +2427109|AAAAAAAAFOIAFCAA|1933-02-05|397|1727|133|1933|0|2|5|1|1933|133|1727|Sunday|1933Q1|N|N|N|2427105|2427135|2426743|2427017|N|N|N|N|N| +2427110|AAAAAAAAGOIAFCAA|1933-02-06|397|1727|133|1933|1|2|6|1|1933|133|1727|Monday|1933Q1|N|N|N|2427105|2427135|2426744|2427018|N|N|N|N|N| +2427111|AAAAAAAAHOIAFCAA|1933-02-07|397|1728|133|1933|2|2|7|1|1933|133|1728|Tuesday|1933Q1|N|N|N|2427105|2427135|2426745|2427019|N|N|N|N|N| +2427112|AAAAAAAAIOIAFCAA|1933-02-08|397|1728|133|1933|3|2|8|1|1933|133|1728|Wednesday|1933Q1|N|N|N|2427105|2427135|2426746|2427020|N|N|N|N|N| +2427113|AAAAAAAAJOIAFCAA|1933-02-09|397|1728|133|1933|4|2|9|1|1933|133|1728|Thursday|1933Q1|N|N|N|2427105|2427135|2426747|2427021|N|N|N|N|N| +2427114|AAAAAAAAKOIAFCAA|1933-02-10|397|1728|133|1933|5|2|10|1|1933|133|1728|Friday|1933Q1|N|Y|N|2427105|2427135|2426748|2427022|N|N|N|N|N| +2427115|AAAAAAAALOIAFCAA|1933-02-11|397|1728|133|1933|6|2|11|1|1933|133|1728|Saturday|1933Q1|N|Y|N|2427105|2427135|2426749|2427023|N|N|N|N|N| +2427116|AAAAAAAAMOIAFCAA|1933-02-12|397|1728|133|1933|0|2|12|1|1933|133|1728|Sunday|1933Q1|N|N|N|2427105|2427135|2426750|2427024|N|N|N|N|N| +2427117|AAAAAAAANOIAFCAA|1933-02-13|397|1728|133|1933|1|2|13|1|1933|133|1728|Monday|1933Q1|N|N|N|2427105|2427135|2426751|2427025|N|N|N|N|N| +2427118|AAAAAAAAOOIAFCAA|1933-02-14|397|1729|133|1933|2|2|14|1|1933|133|1729|Tuesday|1933Q1|N|N|N|2427105|2427135|2426752|2427026|N|N|N|N|N| +2427119|AAAAAAAAPOIAFCAA|1933-02-15|397|1729|133|1933|3|2|15|1|1933|133|1729|Wednesday|1933Q1|N|N|N|2427105|2427135|2426753|2427027|N|N|N|N|N| +2427120|AAAAAAAAAPIAFCAA|1933-02-16|397|1729|133|1933|4|2|16|1|1933|133|1729|Thursday|1933Q1|N|N|N|2427105|2427135|2426754|2427028|N|N|N|N|N| +2427121|AAAAAAAABPIAFCAA|1933-02-17|397|1729|133|1933|5|2|17|1|1933|133|1729|Friday|1933Q1|N|Y|N|2427105|2427135|2426755|2427029|N|N|N|N|N| +2427122|AAAAAAAACPIAFCAA|1933-02-18|397|1729|133|1933|6|2|18|1|1933|133|1729|Saturday|1933Q1|N|Y|N|2427105|2427135|2426756|2427030|N|N|N|N|N| +2427123|AAAAAAAADPIAFCAA|1933-02-19|397|1729|133|1933|0|2|19|1|1933|133|1729|Sunday|1933Q1|N|N|N|2427105|2427135|2426757|2427031|N|N|N|N|N| +2427124|AAAAAAAAEPIAFCAA|1933-02-20|397|1729|133|1933|1|2|20|1|1933|133|1729|Monday|1933Q1|N|N|N|2427105|2427135|2426758|2427032|N|N|N|N|N| +2427125|AAAAAAAAFPIAFCAA|1933-02-21|397|1730|133|1933|2|2|21|1|1933|133|1730|Tuesday|1933Q1|N|N|N|2427105|2427135|2426759|2427033|N|N|N|N|N| +2427126|AAAAAAAAGPIAFCAA|1933-02-22|397|1730|133|1933|3|2|22|1|1933|133|1730|Wednesday|1933Q1|N|N|N|2427105|2427135|2426760|2427034|N|N|N|N|N| +2427127|AAAAAAAAHPIAFCAA|1933-02-23|397|1730|133|1933|4|2|23|1|1933|133|1730|Thursday|1933Q1|N|N|N|2427105|2427135|2426761|2427035|N|N|N|N|N| +2427128|AAAAAAAAIPIAFCAA|1933-02-24|397|1730|133|1933|5|2|24|1|1933|133|1730|Friday|1933Q1|N|Y|N|2427105|2427135|2426762|2427036|N|N|N|N|N| +2427129|AAAAAAAAJPIAFCAA|1933-02-25|397|1730|133|1933|6|2|25|1|1933|133|1730|Saturday|1933Q1|N|Y|N|2427105|2427135|2426763|2427037|N|N|N|N|N| +2427130|AAAAAAAAKPIAFCAA|1933-02-26|397|1730|133|1933|0|2|26|1|1933|133|1730|Sunday|1933Q1|N|N|N|2427105|2427135|2426764|2427038|N|N|N|N|N| +2427131|AAAAAAAALPIAFCAA|1933-02-27|397|1730|133|1933|1|2|27|1|1933|133|1730|Monday|1933Q1|N|N|N|2427105|2427135|2426765|2427039|N|N|N|N|N| +2427132|AAAAAAAAMPIAFCAA|1933-02-28|397|1731|133|1933|2|2|28|1|1933|133|1731|Tuesday|1933Q1|N|N|N|2427105|2427135|2426766|2427040|N|N|N|N|N| +2427133|AAAAAAAANPIAFCAA|1933-03-01|398|1731|134|1933|3|3|1|1|1933|134|1731|Wednesday|1933Q1|N|N|N|2427133|2427191|2426768|2427041|N|N|N|N|N| +2427134|AAAAAAAAOPIAFCAA|1933-03-02|398|1731|134|1933|4|3|2|1|1933|134|1731|Thursday|1933Q1|N|N|N|2427133|2427191|2426769|2427042|N|N|N|N|N| +2427135|AAAAAAAAPPIAFCAA|1933-03-03|398|1731|134|1933|5|3|3|1|1933|134|1731|Friday|1933Q1|N|Y|N|2427133|2427191|2426770|2427043|N|N|N|N|N| +2427136|AAAAAAAAAAJAFCAA|1933-03-04|398|1731|134|1933|6|3|4|1|1933|134|1731|Saturday|1933Q1|N|Y|N|2427133|2427191|2426771|2427044|N|N|N|N|N| +2427137|AAAAAAAABAJAFCAA|1933-03-05|398|1731|134|1933|0|3|5|1|1933|134|1731|Sunday|1933Q1|N|N|N|2427133|2427191|2426772|2427045|N|N|N|N|N| +2427138|AAAAAAAACAJAFCAA|1933-03-06|398|1731|134|1933|1|3|6|1|1933|134|1731|Monday|1933Q1|N|N|N|2427133|2427191|2426773|2427046|N|N|N|N|N| +2427139|AAAAAAAADAJAFCAA|1933-03-07|398|1732|134|1933|2|3|7|1|1933|134|1732|Tuesday|1933Q1|N|N|N|2427133|2427191|2426774|2427047|N|N|N|N|N| +2427140|AAAAAAAAEAJAFCAA|1933-03-08|398|1732|134|1933|3|3|8|1|1933|134|1732|Wednesday|1933Q1|N|N|N|2427133|2427191|2426775|2427048|N|N|N|N|N| +2427141|AAAAAAAAFAJAFCAA|1933-03-09|398|1732|134|1933|4|3|9|1|1933|134|1732|Thursday|1933Q1|N|N|N|2427133|2427191|2426776|2427049|N|N|N|N|N| +2427142|AAAAAAAAGAJAFCAA|1933-03-10|398|1732|134|1933|5|3|10|1|1933|134|1732|Friday|1933Q1|N|Y|N|2427133|2427191|2426777|2427050|N|N|N|N|N| +2427143|AAAAAAAAHAJAFCAA|1933-03-11|398|1732|134|1933|6|3|11|1|1933|134|1732|Saturday|1933Q1|N|Y|N|2427133|2427191|2426778|2427051|N|N|N|N|N| +2427144|AAAAAAAAIAJAFCAA|1933-03-12|398|1732|134|1933|0|3|12|1|1933|134|1732|Sunday|1933Q1|N|N|N|2427133|2427191|2426779|2427052|N|N|N|N|N| +2427145|AAAAAAAAJAJAFCAA|1933-03-13|398|1732|134|1933|1|3|13|1|1933|134|1732|Monday|1933Q1|N|N|N|2427133|2427191|2426780|2427053|N|N|N|N|N| +2427146|AAAAAAAAKAJAFCAA|1933-03-14|398|1733|134|1933|2|3|14|1|1933|134|1733|Tuesday|1933Q1|N|N|N|2427133|2427191|2426781|2427054|N|N|N|N|N| +2427147|AAAAAAAALAJAFCAA|1933-03-15|398|1733|134|1933|3|3|15|1|1933|134|1733|Wednesday|1933Q1|N|N|N|2427133|2427191|2426782|2427055|N|N|N|N|N| +2427148|AAAAAAAAMAJAFCAA|1933-03-16|398|1733|134|1933|4|3|16|1|1933|134|1733|Thursday|1933Q1|N|N|N|2427133|2427191|2426783|2427056|N|N|N|N|N| +2427149|AAAAAAAANAJAFCAA|1933-03-17|398|1733|134|1933|5|3|17|1|1933|134|1733|Friday|1933Q1|N|Y|N|2427133|2427191|2426784|2427057|N|N|N|N|N| +2427150|AAAAAAAAOAJAFCAA|1933-03-18|398|1733|134|1933|6|3|18|1|1933|134|1733|Saturday|1933Q1|N|Y|N|2427133|2427191|2426785|2427058|N|N|N|N|N| +2427151|AAAAAAAAPAJAFCAA|1933-03-19|398|1733|134|1933|0|3|19|1|1933|134|1733|Sunday|1933Q1|N|N|N|2427133|2427191|2426786|2427059|N|N|N|N|N| +2427152|AAAAAAAAABJAFCAA|1933-03-20|398|1733|134|1933|1|3|20|1|1933|134|1733|Monday|1933Q1|N|N|N|2427133|2427191|2426787|2427060|N|N|N|N|N| +2427153|AAAAAAAABBJAFCAA|1933-03-21|398|1734|134|1933|2|3|21|1|1933|134|1734|Tuesday|1933Q1|N|N|N|2427133|2427191|2426788|2427061|N|N|N|N|N| +2427154|AAAAAAAACBJAFCAA|1933-03-22|398|1734|134|1933|3|3|22|1|1933|134|1734|Wednesday|1933Q1|N|N|N|2427133|2427191|2426789|2427062|N|N|N|N|N| +2427155|AAAAAAAADBJAFCAA|1933-03-23|398|1734|134|1933|4|3|23|1|1933|134|1734|Thursday|1933Q1|N|N|N|2427133|2427191|2426790|2427063|N|N|N|N|N| +2427156|AAAAAAAAEBJAFCAA|1933-03-24|398|1734|134|1933|5|3|24|1|1933|134|1734|Friday|1933Q1|N|Y|N|2427133|2427191|2426791|2427064|N|N|N|N|N| +2427157|AAAAAAAAFBJAFCAA|1933-03-25|398|1734|134|1933|6|3|25|1|1933|134|1734|Saturday|1933Q1|N|Y|N|2427133|2427191|2426792|2427065|N|N|N|N|N| +2427158|AAAAAAAAGBJAFCAA|1933-03-26|398|1734|134|1933|0|3|26|1|1933|134|1734|Sunday|1933Q1|N|N|N|2427133|2427191|2426793|2427066|N|N|N|N|N| +2427159|AAAAAAAAHBJAFCAA|1933-03-27|398|1734|134|1933|1|3|27|1|1933|134|1734|Monday|1933Q1|N|N|N|2427133|2427191|2426794|2427067|N|N|N|N|N| +2427160|AAAAAAAAIBJAFCAA|1933-03-28|398|1735|134|1933|2|3|28|1|1933|134|1735|Tuesday|1933Q1|N|N|N|2427133|2427191|2426795|2427068|N|N|N|N|N| +2427161|AAAAAAAAJBJAFCAA|1933-03-29|398|1735|134|1933|3|3|29|1|1933|134|1735|Wednesday|1933Q1|N|N|N|2427133|2427191|2426796|2427069|N|N|N|N|N| +2427162|AAAAAAAAKBJAFCAA|1933-03-30|398|1735|134|1933|4|3|30|1|1933|134|1735|Thursday|1933Q1|N|N|N|2427133|2427191|2426797|2427070|N|N|N|N|N| +2427163|AAAAAAAALBJAFCAA|1933-03-31|398|1735|134|1933|5|3|31|1|1933|134|1735|Friday|1933Q1|N|Y|N|2427133|2427191|2426798|2427071|N|N|N|N|N| +2427164|AAAAAAAAMBJAFCAA|1933-04-01|399|1735|134|1933|6|4|1|1|1933|134|1735|Saturday|1933Q1|N|Y|N|2427164|2427253|2426799|2427074|N|N|N|N|N| +2427165|AAAAAAAANBJAFCAA|1933-04-02|399|1735|134|1933|0|4|2|2|1933|134|1735|Sunday|1933Q2|N|N|N|2427164|2427253|2426800|2427075|N|N|N|N|N| +2427166|AAAAAAAAOBJAFCAA|1933-04-03|399|1735|134|1933|1|4|3|2|1933|134|1735|Monday|1933Q2|N|N|N|2427164|2427253|2426801|2427076|N|N|N|N|N| +2427167|AAAAAAAAPBJAFCAA|1933-04-04|399|1736|134|1933|2|4|4|2|1933|134|1736|Tuesday|1933Q2|N|N|N|2427164|2427253|2426802|2427077|N|N|N|N|N| +2427168|AAAAAAAAACJAFCAA|1933-04-05|399|1736|134|1933|3|4|5|2|1933|134|1736|Wednesday|1933Q2|N|N|N|2427164|2427253|2426803|2427078|N|N|N|N|N| +2427169|AAAAAAAABCJAFCAA|1933-04-06|399|1736|134|1933|4|4|6|2|1933|134|1736|Thursday|1933Q2|N|N|N|2427164|2427253|2426804|2427079|N|N|N|N|N| +2427170|AAAAAAAACCJAFCAA|1933-04-07|399|1736|134|1933|5|4|7|2|1933|134|1736|Friday|1933Q2|N|Y|N|2427164|2427253|2426805|2427080|N|N|N|N|N| +2427171|AAAAAAAADCJAFCAA|1933-04-08|399|1736|134|1933|6|4|8|2|1933|134|1736|Saturday|1933Q2|N|Y|N|2427164|2427253|2426806|2427081|N|N|N|N|N| +2427172|AAAAAAAAECJAFCAA|1933-04-09|399|1736|134|1933|0|4|9|2|1933|134|1736|Sunday|1933Q2|N|N|N|2427164|2427253|2426807|2427082|N|N|N|N|N| +2427173|AAAAAAAAFCJAFCAA|1933-04-10|399|1736|134|1933|1|4|10|2|1933|134|1736|Monday|1933Q2|N|N|N|2427164|2427253|2426808|2427083|N|N|N|N|N| +2427174|AAAAAAAAGCJAFCAA|1933-04-11|399|1737|134|1933|2|4|11|2|1933|134|1737|Tuesday|1933Q2|N|N|N|2427164|2427253|2426809|2427084|N|N|N|N|N| +2427175|AAAAAAAAHCJAFCAA|1933-04-12|399|1737|134|1933|3|4|12|2|1933|134|1737|Wednesday|1933Q2|N|N|N|2427164|2427253|2426810|2427085|N|N|N|N|N| +2427176|AAAAAAAAICJAFCAA|1933-04-13|399|1737|134|1933|4|4|13|2|1933|134|1737|Thursday|1933Q2|N|N|N|2427164|2427253|2426811|2427086|N|N|N|N|N| +2427177|AAAAAAAAJCJAFCAA|1933-04-14|399|1737|134|1933|5|4|14|2|1933|134|1737|Friday|1933Q2|N|Y|N|2427164|2427253|2426812|2427087|N|N|N|N|N| +2427178|AAAAAAAAKCJAFCAA|1933-04-15|399|1737|134|1933|6|4|15|2|1933|134|1737|Saturday|1933Q2|N|Y|N|2427164|2427253|2426813|2427088|N|N|N|N|N| +2427179|AAAAAAAALCJAFCAA|1933-04-16|399|1737|134|1933|0|4|16|2|1933|134|1737|Sunday|1933Q2|N|N|N|2427164|2427253|2426814|2427089|N|N|N|N|N| +2427180|AAAAAAAAMCJAFCAA|1933-04-17|399|1737|134|1933|1|4|17|2|1933|134|1737|Monday|1933Q2|N|N|N|2427164|2427253|2426815|2427090|N|N|N|N|N| +2427181|AAAAAAAANCJAFCAA|1933-04-18|399|1738|134|1933|2|4|18|2|1933|134|1738|Tuesday|1933Q2|N|N|N|2427164|2427253|2426816|2427091|N|N|N|N|N| +2427182|AAAAAAAAOCJAFCAA|1933-04-19|399|1738|134|1933|3|4|19|2|1933|134|1738|Wednesday|1933Q2|N|N|N|2427164|2427253|2426817|2427092|N|N|N|N|N| +2427183|AAAAAAAAPCJAFCAA|1933-04-20|399|1738|134|1933|4|4|20|2|1933|134|1738|Thursday|1933Q2|N|N|N|2427164|2427253|2426818|2427093|N|N|N|N|N| +2427184|AAAAAAAAADJAFCAA|1933-04-21|399|1738|134|1933|5|4|21|2|1933|134|1738|Friday|1933Q2|N|Y|N|2427164|2427253|2426819|2427094|N|N|N|N|N| +2427185|AAAAAAAABDJAFCAA|1933-04-22|399|1738|134|1933|6|4|22|2|1933|134|1738|Saturday|1933Q2|N|Y|N|2427164|2427253|2426820|2427095|N|N|N|N|N| +2427186|AAAAAAAACDJAFCAA|1933-04-23|399|1738|134|1933|0|4|23|2|1933|134|1738|Sunday|1933Q2|N|N|N|2427164|2427253|2426821|2427096|N|N|N|N|N| +2427187|AAAAAAAADDJAFCAA|1933-04-24|399|1738|134|1933|1|4|24|2|1933|134|1738|Monday|1933Q2|N|N|N|2427164|2427253|2426822|2427097|N|N|N|N|N| +2427188|AAAAAAAAEDJAFCAA|1933-04-25|399|1739|134|1933|2|4|25|2|1933|134|1739|Tuesday|1933Q2|N|N|N|2427164|2427253|2426823|2427098|N|N|N|N|N| +2427189|AAAAAAAAFDJAFCAA|1933-04-26|399|1739|134|1933|3|4|26|2|1933|134|1739|Wednesday|1933Q2|N|N|N|2427164|2427253|2426824|2427099|N|N|N|N|N| +2427190|AAAAAAAAGDJAFCAA|1933-04-27|399|1739|134|1933|4|4|27|2|1933|134|1739|Thursday|1933Q2|N|N|N|2427164|2427253|2426825|2427100|N|N|N|N|N| +2427191|AAAAAAAAHDJAFCAA|1933-04-28|399|1739|134|1933|5|4|28|2|1933|134|1739|Friday|1933Q2|N|Y|N|2427164|2427253|2426826|2427101|N|N|N|N|N| +2427192|AAAAAAAAIDJAFCAA|1933-04-29|399|1739|134|1933|6|4|29|2|1933|134|1739|Saturday|1933Q2|N|Y|N|2427164|2427253|2426827|2427102|N|N|N|N|N| +2427193|AAAAAAAAJDJAFCAA|1933-04-30|399|1739|134|1933|0|4|30|2|1933|134|1739|Sunday|1933Q2|N|N|N|2427164|2427253|2426828|2427103|N|N|N|N|N| +2427194|AAAAAAAAKDJAFCAA|1933-05-01|400|1739|134|1933|1|5|1|2|1933|134|1739|Monday|1933Q2|N|N|N|2427194|2427313|2426829|2427104|N|N|N|N|N| +2427195|AAAAAAAALDJAFCAA|1933-05-02|400|1740|134|1933|2|5|2|2|1933|134|1740|Tuesday|1933Q2|N|N|N|2427194|2427313|2426830|2427105|N|N|N|N|N| +2427196|AAAAAAAAMDJAFCAA|1933-05-03|400|1740|134|1933|3|5|3|2|1933|134|1740|Wednesday|1933Q2|N|N|N|2427194|2427313|2426831|2427106|N|N|N|N|N| +2427197|AAAAAAAANDJAFCAA|1933-05-04|400|1740|134|1933|4|5|4|2|1933|134|1740|Thursday|1933Q2|N|N|N|2427194|2427313|2426832|2427107|N|N|N|N|N| +2427198|AAAAAAAAODJAFCAA|1933-05-05|400|1740|134|1933|5|5|5|2|1933|134|1740|Friday|1933Q2|N|Y|N|2427194|2427313|2426833|2427108|N|N|N|N|N| +2427199|AAAAAAAAPDJAFCAA|1933-05-06|400|1740|134|1933|6|5|6|2|1933|134|1740|Saturday|1933Q2|N|Y|N|2427194|2427313|2426834|2427109|N|N|N|N|N| +2427200|AAAAAAAAAEJAFCAA|1933-05-07|400|1740|134|1933|0|5|7|2|1933|134|1740|Sunday|1933Q2|N|N|N|2427194|2427313|2426835|2427110|N|N|N|N|N| +2427201|AAAAAAAABEJAFCAA|1933-05-08|400|1740|134|1933|1|5|8|2|1933|134|1740|Monday|1933Q2|N|N|N|2427194|2427313|2426836|2427111|N|N|N|N|N| +2427202|AAAAAAAACEJAFCAA|1933-05-09|400|1741|134|1933|2|5|9|2|1933|134|1741|Tuesday|1933Q2|N|N|N|2427194|2427313|2426837|2427112|N|N|N|N|N| +2427203|AAAAAAAADEJAFCAA|1933-05-10|400|1741|134|1933|3|5|10|2|1933|134|1741|Wednesday|1933Q2|N|N|N|2427194|2427313|2426838|2427113|N|N|N|N|N| +2427204|AAAAAAAAEEJAFCAA|1933-05-11|400|1741|134|1933|4|5|11|2|1933|134|1741|Thursday|1933Q2|N|N|N|2427194|2427313|2426839|2427114|N|N|N|N|N| +2427205|AAAAAAAAFEJAFCAA|1933-05-12|400|1741|134|1933|5|5|12|2|1933|134|1741|Friday|1933Q2|N|Y|N|2427194|2427313|2426840|2427115|N|N|N|N|N| +2427206|AAAAAAAAGEJAFCAA|1933-05-13|400|1741|134|1933|6|5|13|2|1933|134|1741|Saturday|1933Q2|N|Y|N|2427194|2427313|2426841|2427116|N|N|N|N|N| +2427207|AAAAAAAAHEJAFCAA|1933-05-14|400|1741|134|1933|0|5|14|2|1933|134|1741|Sunday|1933Q2|N|N|N|2427194|2427313|2426842|2427117|N|N|N|N|N| +2427208|AAAAAAAAIEJAFCAA|1933-05-15|400|1741|134|1933|1|5|15|2|1933|134|1741|Monday|1933Q2|N|N|N|2427194|2427313|2426843|2427118|N|N|N|N|N| +2427209|AAAAAAAAJEJAFCAA|1933-05-16|400|1742|134|1933|2|5|16|2|1933|134|1742|Tuesday|1933Q2|N|N|N|2427194|2427313|2426844|2427119|N|N|N|N|N| +2427210|AAAAAAAAKEJAFCAA|1933-05-17|400|1742|134|1933|3|5|17|2|1933|134|1742|Wednesday|1933Q2|N|N|N|2427194|2427313|2426845|2427120|N|N|N|N|N| +2427211|AAAAAAAALEJAFCAA|1933-05-18|400|1742|134|1933|4|5|18|2|1933|134|1742|Thursday|1933Q2|N|N|N|2427194|2427313|2426846|2427121|N|N|N|N|N| +2427212|AAAAAAAAMEJAFCAA|1933-05-19|400|1742|134|1933|5|5|19|2|1933|134|1742|Friday|1933Q2|N|Y|N|2427194|2427313|2426847|2427122|N|N|N|N|N| +2427213|AAAAAAAANEJAFCAA|1933-05-20|400|1742|134|1933|6|5|20|2|1933|134|1742|Saturday|1933Q2|N|Y|N|2427194|2427313|2426848|2427123|N|N|N|N|N| +2427214|AAAAAAAAOEJAFCAA|1933-05-21|400|1742|134|1933|0|5|21|2|1933|134|1742|Sunday|1933Q2|N|N|N|2427194|2427313|2426849|2427124|N|N|N|N|N| +2427215|AAAAAAAAPEJAFCAA|1933-05-22|400|1742|134|1933|1|5|22|2|1933|134|1742|Monday|1933Q2|N|N|N|2427194|2427313|2426850|2427125|N|N|N|N|N| +2427216|AAAAAAAAAFJAFCAA|1933-05-23|400|1743|134|1933|2|5|23|2|1933|134|1743|Tuesday|1933Q2|N|N|N|2427194|2427313|2426851|2427126|N|N|N|N|N| +2427217|AAAAAAAABFJAFCAA|1933-05-24|400|1743|134|1933|3|5|24|2|1933|134|1743|Wednesday|1933Q2|N|N|N|2427194|2427313|2426852|2427127|N|N|N|N|N| +2427218|AAAAAAAACFJAFCAA|1933-05-25|400|1743|134|1933|4|5|25|2|1933|134|1743|Thursday|1933Q2|N|N|N|2427194|2427313|2426853|2427128|N|N|N|N|N| +2427219|AAAAAAAADFJAFCAA|1933-05-26|400|1743|134|1933|5|5|26|2|1933|134|1743|Friday|1933Q2|N|Y|N|2427194|2427313|2426854|2427129|N|N|N|N|N| +2427220|AAAAAAAAEFJAFCAA|1933-05-27|400|1743|134|1933|6|5|27|2|1933|134|1743|Saturday|1933Q2|N|Y|N|2427194|2427313|2426855|2427130|N|N|N|N|N| +2427221|AAAAAAAAFFJAFCAA|1933-05-28|400|1743|134|1933|0|5|28|2|1933|134|1743|Sunday|1933Q2|N|N|N|2427194|2427313|2426856|2427131|N|N|N|N|N| +2427222|AAAAAAAAGFJAFCAA|1933-05-29|400|1743|134|1933|1|5|29|2|1933|134|1743|Monday|1933Q2|N|N|N|2427194|2427313|2426857|2427132|N|N|N|N|N| +2427223|AAAAAAAAHFJAFCAA|1933-05-30|400|1744|134|1933|2|5|30|2|1933|134|1744|Tuesday|1933Q2|N|N|N|2427194|2427313|2426858|2427133|N|N|N|N|N| +2427224|AAAAAAAAIFJAFCAA|1933-05-31|400|1744|134|1933|3|5|31|2|1933|134|1744|Wednesday|1933Q2|N|N|N|2427194|2427313|2426859|2427134|N|N|N|N|N| +2427225|AAAAAAAAJFJAFCAA|1933-06-01|401|1744|135|1933|4|6|1|2|1933|135|1744|Thursday|1933Q2|N|N|N|2427225|2427375|2426860|2427135|N|N|N|N|N| +2427226|AAAAAAAAKFJAFCAA|1933-06-02|401|1744|135|1933|5|6|2|2|1933|135|1744|Friday|1933Q2|N|Y|N|2427225|2427375|2426861|2427136|N|N|N|N|N| +2427227|AAAAAAAALFJAFCAA|1933-06-03|401|1744|135|1933|6|6|3|2|1933|135|1744|Saturday|1933Q2|N|Y|N|2427225|2427375|2426862|2427137|N|N|N|N|N| +2427228|AAAAAAAAMFJAFCAA|1933-06-04|401|1744|135|1933|0|6|4|2|1933|135|1744|Sunday|1933Q2|N|N|N|2427225|2427375|2426863|2427138|N|N|N|N|N| +2427229|AAAAAAAANFJAFCAA|1933-06-05|401|1744|135|1933|1|6|5|2|1933|135|1744|Monday|1933Q2|N|N|N|2427225|2427375|2426864|2427139|N|N|N|N|N| +2427230|AAAAAAAAOFJAFCAA|1933-06-06|401|1745|135|1933|2|6|6|2|1933|135|1745|Tuesday|1933Q2|N|N|N|2427225|2427375|2426865|2427140|N|N|N|N|N| +2427231|AAAAAAAAPFJAFCAA|1933-06-07|401|1745|135|1933|3|6|7|2|1933|135|1745|Wednesday|1933Q2|N|N|N|2427225|2427375|2426866|2427141|N|N|N|N|N| +2427232|AAAAAAAAAGJAFCAA|1933-06-08|401|1745|135|1933|4|6|8|2|1933|135|1745|Thursday|1933Q2|N|N|N|2427225|2427375|2426867|2427142|N|N|N|N|N| +2427233|AAAAAAAABGJAFCAA|1933-06-09|401|1745|135|1933|5|6|9|2|1933|135|1745|Friday|1933Q2|N|Y|N|2427225|2427375|2426868|2427143|N|N|N|N|N| +2427234|AAAAAAAACGJAFCAA|1933-06-10|401|1745|135|1933|6|6|10|2|1933|135|1745|Saturday|1933Q2|N|Y|N|2427225|2427375|2426869|2427144|N|N|N|N|N| +2427235|AAAAAAAADGJAFCAA|1933-06-11|401|1745|135|1933|0|6|11|2|1933|135|1745|Sunday|1933Q2|N|N|N|2427225|2427375|2426870|2427145|N|N|N|N|N| +2427236|AAAAAAAAEGJAFCAA|1933-06-12|401|1745|135|1933|1|6|12|2|1933|135|1745|Monday|1933Q2|N|N|N|2427225|2427375|2426871|2427146|N|N|N|N|N| +2427237|AAAAAAAAFGJAFCAA|1933-06-13|401|1746|135|1933|2|6|13|2|1933|135|1746|Tuesday|1933Q2|N|N|N|2427225|2427375|2426872|2427147|N|N|N|N|N| +2427238|AAAAAAAAGGJAFCAA|1933-06-14|401|1746|135|1933|3|6|14|2|1933|135|1746|Wednesday|1933Q2|N|N|N|2427225|2427375|2426873|2427148|N|N|N|N|N| +2427239|AAAAAAAAHGJAFCAA|1933-06-15|401|1746|135|1933|4|6|15|2|1933|135|1746|Thursday|1933Q2|N|N|N|2427225|2427375|2426874|2427149|N|N|N|N|N| +2427240|AAAAAAAAIGJAFCAA|1933-06-16|401|1746|135|1933|5|6|16|2|1933|135|1746|Friday|1933Q2|N|Y|N|2427225|2427375|2426875|2427150|N|N|N|N|N| +2427241|AAAAAAAAJGJAFCAA|1933-06-17|401|1746|135|1933|6|6|17|2|1933|135|1746|Saturday|1933Q2|N|Y|N|2427225|2427375|2426876|2427151|N|N|N|N|N| +2427242|AAAAAAAAKGJAFCAA|1933-06-18|401|1746|135|1933|0|6|18|2|1933|135|1746|Sunday|1933Q2|N|N|N|2427225|2427375|2426877|2427152|N|N|N|N|N| +2427243|AAAAAAAALGJAFCAA|1933-06-19|401|1746|135|1933|1|6|19|2|1933|135|1746|Monday|1933Q2|N|N|N|2427225|2427375|2426878|2427153|N|N|N|N|N| +2427244|AAAAAAAAMGJAFCAA|1933-06-20|401|1747|135|1933|2|6|20|2|1933|135|1747|Tuesday|1933Q2|N|N|N|2427225|2427375|2426879|2427154|N|N|N|N|N| +2427245|AAAAAAAANGJAFCAA|1933-06-21|401|1747|135|1933|3|6|21|2|1933|135|1747|Wednesday|1933Q2|N|N|N|2427225|2427375|2426880|2427155|N|N|N|N|N| +2427246|AAAAAAAAOGJAFCAA|1933-06-22|401|1747|135|1933|4|6|22|2|1933|135|1747|Thursday|1933Q2|N|N|N|2427225|2427375|2426881|2427156|N|N|N|N|N| +2427247|AAAAAAAAPGJAFCAA|1933-06-23|401|1747|135|1933|5|6|23|2|1933|135|1747|Friday|1933Q2|N|Y|N|2427225|2427375|2426882|2427157|N|N|N|N|N| +2427248|AAAAAAAAAHJAFCAA|1933-06-24|401|1747|135|1933|6|6|24|2|1933|135|1747|Saturday|1933Q2|N|Y|N|2427225|2427375|2426883|2427158|N|N|N|N|N| +2427249|AAAAAAAABHJAFCAA|1933-06-25|401|1747|135|1933|0|6|25|2|1933|135|1747|Sunday|1933Q2|N|N|N|2427225|2427375|2426884|2427159|N|N|N|N|N| +2427250|AAAAAAAACHJAFCAA|1933-06-26|401|1747|135|1933|1|6|26|2|1933|135|1747|Monday|1933Q2|N|N|N|2427225|2427375|2426885|2427160|N|N|N|N|N| +2427251|AAAAAAAADHJAFCAA|1933-06-27|401|1748|135|1933|2|6|27|2|1933|135|1748|Tuesday|1933Q2|N|N|N|2427225|2427375|2426886|2427161|N|N|N|N|N| +2427252|AAAAAAAAEHJAFCAA|1933-06-28|401|1748|135|1933|3|6|28|2|1933|135|1748|Wednesday|1933Q2|N|N|N|2427225|2427375|2426887|2427162|N|N|N|N|N| +2427253|AAAAAAAAFHJAFCAA|1933-06-29|401|1748|135|1933|4|6|29|2|1933|135|1748|Thursday|1933Q2|N|N|N|2427225|2427375|2426888|2427163|N|N|N|N|N| +2427254|AAAAAAAAGHJAFCAA|1933-06-30|401|1748|135|1933|5|6|30|2|1933|135|1748|Friday|1933Q2|N|Y|N|2427225|2427375|2426889|2427164|N|N|N|N|N| +2427255|AAAAAAAAHHJAFCAA|1933-07-01|402|1748|135|1933|6|7|1|2|1933|135|1748|Saturday|1933Q2|N|Y|N|2427255|2427435|2426890|2427164|N|N|N|N|N| +2427256|AAAAAAAAIHJAFCAA|1933-07-02|402|1748|135|1933|0|7|2|3|1933|135|1748|Sunday|1933Q3|N|N|N|2427255|2427435|2426891|2427165|N|N|N|N|N| +2427257|AAAAAAAAJHJAFCAA|1933-07-03|402|1748|135|1933|1|7|3|3|1933|135|1748|Monday|1933Q3|N|N|N|2427255|2427435|2426892|2427166|N|N|N|N|N| +2427258|AAAAAAAAKHJAFCAA|1933-07-04|402|1749|135|1933|2|7|4|3|1933|135|1749|Tuesday|1933Q3|N|N|N|2427255|2427435|2426893|2427167|N|N|N|N|N| +2427259|AAAAAAAALHJAFCAA|1933-07-05|402|1749|135|1933|3|7|5|3|1933|135|1749|Wednesday|1933Q3|Y|N|N|2427255|2427435|2426894|2427168|N|N|N|N|N| +2427260|AAAAAAAAMHJAFCAA|1933-07-06|402|1749|135|1933|4|7|6|3|1933|135|1749|Thursday|1933Q3|N|N|Y|2427255|2427435|2426895|2427169|N|N|N|N|N| +2427261|AAAAAAAANHJAFCAA|1933-07-07|402|1749|135|1933|5|7|7|3|1933|135|1749|Friday|1933Q3|N|Y|N|2427255|2427435|2426896|2427170|N|N|N|N|N| +2427262|AAAAAAAAOHJAFCAA|1933-07-08|402|1749|135|1933|6|7|8|3|1933|135|1749|Saturday|1933Q3|N|Y|N|2427255|2427435|2426897|2427171|N|N|N|N|N| +2427263|AAAAAAAAPHJAFCAA|1933-07-09|402|1749|135|1933|0|7|9|3|1933|135|1749|Sunday|1933Q3|N|N|N|2427255|2427435|2426898|2427172|N|N|N|N|N| +2427264|AAAAAAAAAIJAFCAA|1933-07-10|402|1749|135|1933|1|7|10|3|1933|135|1749|Monday|1933Q3|N|N|N|2427255|2427435|2426899|2427173|N|N|N|N|N| +2427265|AAAAAAAABIJAFCAA|1933-07-11|402|1750|135|1933|2|7|11|3|1933|135|1750|Tuesday|1933Q3|N|N|N|2427255|2427435|2426900|2427174|N|N|N|N|N| +2427266|AAAAAAAACIJAFCAA|1933-07-12|402|1750|135|1933|3|7|12|3|1933|135|1750|Wednesday|1933Q3|N|N|N|2427255|2427435|2426901|2427175|N|N|N|N|N| +2427267|AAAAAAAADIJAFCAA|1933-07-13|402|1750|135|1933|4|7|13|3|1933|135|1750|Thursday|1933Q3|N|N|N|2427255|2427435|2426902|2427176|N|N|N|N|N| +2427268|AAAAAAAAEIJAFCAA|1933-07-14|402|1750|135|1933|5|7|14|3|1933|135|1750|Friday|1933Q3|N|Y|N|2427255|2427435|2426903|2427177|N|N|N|N|N| +2427269|AAAAAAAAFIJAFCAA|1933-07-15|402|1750|135|1933|6|7|15|3|1933|135|1750|Saturday|1933Q3|N|Y|N|2427255|2427435|2426904|2427178|N|N|N|N|N| +2427270|AAAAAAAAGIJAFCAA|1933-07-16|402|1750|135|1933|0|7|16|3|1933|135|1750|Sunday|1933Q3|N|N|N|2427255|2427435|2426905|2427179|N|N|N|N|N| +2427271|AAAAAAAAHIJAFCAA|1933-07-17|402|1750|135|1933|1|7|17|3|1933|135|1750|Monday|1933Q3|N|N|N|2427255|2427435|2426906|2427180|N|N|N|N|N| +2427272|AAAAAAAAIIJAFCAA|1933-07-18|402|1751|135|1933|2|7|18|3|1933|135|1751|Tuesday|1933Q3|N|N|N|2427255|2427435|2426907|2427181|N|N|N|N|N| +2427273|AAAAAAAAJIJAFCAA|1933-07-19|402|1751|135|1933|3|7|19|3|1933|135|1751|Wednesday|1933Q3|N|N|N|2427255|2427435|2426908|2427182|N|N|N|N|N| +2427274|AAAAAAAAKIJAFCAA|1933-07-20|402|1751|135|1933|4|7|20|3|1933|135|1751|Thursday|1933Q3|N|N|N|2427255|2427435|2426909|2427183|N|N|N|N|N| +2427275|AAAAAAAALIJAFCAA|1933-07-21|402|1751|135|1933|5|7|21|3|1933|135|1751|Friday|1933Q3|N|Y|N|2427255|2427435|2426910|2427184|N|N|N|N|N| +2427276|AAAAAAAAMIJAFCAA|1933-07-22|402|1751|135|1933|6|7|22|3|1933|135|1751|Saturday|1933Q3|N|Y|N|2427255|2427435|2426911|2427185|N|N|N|N|N| +2427277|AAAAAAAANIJAFCAA|1933-07-23|402|1751|135|1933|0|7|23|3|1933|135|1751|Sunday|1933Q3|N|N|N|2427255|2427435|2426912|2427186|N|N|N|N|N| +2427278|AAAAAAAAOIJAFCAA|1933-07-24|402|1751|135|1933|1|7|24|3|1933|135|1751|Monday|1933Q3|N|N|N|2427255|2427435|2426913|2427187|N|N|N|N|N| +2427279|AAAAAAAAPIJAFCAA|1933-07-25|402|1752|135|1933|2|7|25|3|1933|135|1752|Tuesday|1933Q3|N|N|N|2427255|2427435|2426914|2427188|N|N|N|N|N| +2427280|AAAAAAAAAJJAFCAA|1933-07-26|402|1752|135|1933|3|7|26|3|1933|135|1752|Wednesday|1933Q3|N|N|N|2427255|2427435|2426915|2427189|N|N|N|N|N| +2427281|AAAAAAAABJJAFCAA|1933-07-27|402|1752|135|1933|4|7|27|3|1933|135|1752|Thursday|1933Q3|N|N|N|2427255|2427435|2426916|2427190|N|N|N|N|N| +2427282|AAAAAAAACJJAFCAA|1933-07-28|402|1752|135|1933|5|7|28|3|1933|135|1752|Friday|1933Q3|N|Y|N|2427255|2427435|2426917|2427191|N|N|N|N|N| +2427283|AAAAAAAADJJAFCAA|1933-07-29|402|1752|135|1933|6|7|29|3|1933|135|1752|Saturday|1933Q3|N|Y|N|2427255|2427435|2426918|2427192|N|N|N|N|N| +2427284|AAAAAAAAEJJAFCAA|1933-07-30|402|1752|135|1933|0|7|30|3|1933|135|1752|Sunday|1933Q3|N|N|N|2427255|2427435|2426919|2427193|N|N|N|N|N| +2427285|AAAAAAAAFJJAFCAA|1933-07-31|402|1752|135|1933|1|7|31|3|1933|135|1752|Monday|1933Q3|N|N|N|2427255|2427435|2426920|2427194|N|N|N|N|N| +2427286|AAAAAAAAGJJAFCAA|1933-08-01|403|1753|135|1933|2|8|1|3|1933|135|1753|Tuesday|1933Q3|N|N|N|2427286|2427497|2426921|2427195|N|N|N|N|N| +2427287|AAAAAAAAHJJAFCAA|1933-08-02|403|1753|135|1933|3|8|2|3|1933|135|1753|Wednesday|1933Q3|N|N|N|2427286|2427497|2426922|2427196|N|N|N|N|N| +2427288|AAAAAAAAIJJAFCAA|1933-08-03|403|1753|135|1933|4|8|3|3|1933|135|1753|Thursday|1933Q3|N|N|N|2427286|2427497|2426923|2427197|N|N|N|N|N| +2427289|AAAAAAAAJJJAFCAA|1933-08-04|403|1753|135|1933|5|8|4|3|1933|135|1753|Friday|1933Q3|N|Y|N|2427286|2427497|2426924|2427198|N|N|N|N|N| +2427290|AAAAAAAAKJJAFCAA|1933-08-05|403|1753|135|1933|6|8|5|3|1933|135|1753|Saturday|1933Q3|N|Y|N|2427286|2427497|2426925|2427199|N|N|N|N|N| +2427291|AAAAAAAALJJAFCAA|1933-08-06|403|1753|135|1933|0|8|6|3|1933|135|1753|Sunday|1933Q3|N|N|N|2427286|2427497|2426926|2427200|N|N|N|N|N| +2427292|AAAAAAAAMJJAFCAA|1933-08-07|403|1753|135|1933|1|8|7|3|1933|135|1753|Monday|1933Q3|N|N|N|2427286|2427497|2426927|2427201|N|N|N|N|N| +2427293|AAAAAAAANJJAFCAA|1933-08-08|403|1754|135|1933|2|8|8|3|1933|135|1754|Tuesday|1933Q3|N|N|N|2427286|2427497|2426928|2427202|N|N|N|N|N| +2427294|AAAAAAAAOJJAFCAA|1933-08-09|403|1754|135|1933|3|8|9|3|1933|135|1754|Wednesday|1933Q3|N|N|N|2427286|2427497|2426929|2427203|N|N|N|N|N| +2427295|AAAAAAAAPJJAFCAA|1933-08-10|403|1754|135|1933|4|8|10|3|1933|135|1754|Thursday|1933Q3|N|N|N|2427286|2427497|2426930|2427204|N|N|N|N|N| +2427296|AAAAAAAAAKJAFCAA|1933-08-11|403|1754|135|1933|5|8|11|3|1933|135|1754|Friday|1933Q3|N|Y|N|2427286|2427497|2426931|2427205|N|N|N|N|N| +2427297|AAAAAAAABKJAFCAA|1933-08-12|403|1754|135|1933|6|8|12|3|1933|135|1754|Saturday|1933Q3|N|Y|N|2427286|2427497|2426932|2427206|N|N|N|N|N| +2427298|AAAAAAAACKJAFCAA|1933-08-13|403|1754|135|1933|0|8|13|3|1933|135|1754|Sunday|1933Q3|N|N|N|2427286|2427497|2426933|2427207|N|N|N|N|N| +2427299|AAAAAAAADKJAFCAA|1933-08-14|403|1754|135|1933|1|8|14|3|1933|135|1754|Monday|1933Q3|N|N|N|2427286|2427497|2426934|2427208|N|N|N|N|N| +2427300|AAAAAAAAEKJAFCAA|1933-08-15|403|1755|135|1933|2|8|15|3|1933|135|1755|Tuesday|1933Q3|N|N|N|2427286|2427497|2426935|2427209|N|N|N|N|N| +2427301|AAAAAAAAFKJAFCAA|1933-08-16|403|1755|135|1933|3|8|16|3|1933|135|1755|Wednesday|1933Q3|N|N|N|2427286|2427497|2426936|2427210|N|N|N|N|N| +2427302|AAAAAAAAGKJAFCAA|1933-08-17|403|1755|135|1933|4|8|17|3|1933|135|1755|Thursday|1933Q3|N|N|N|2427286|2427497|2426937|2427211|N|N|N|N|N| +2427303|AAAAAAAAHKJAFCAA|1933-08-18|403|1755|135|1933|5|8|18|3|1933|135|1755|Friday|1933Q3|N|Y|N|2427286|2427497|2426938|2427212|N|N|N|N|N| +2427304|AAAAAAAAIKJAFCAA|1933-08-19|403|1755|135|1933|6|8|19|3|1933|135|1755|Saturday|1933Q3|N|Y|N|2427286|2427497|2426939|2427213|N|N|N|N|N| +2427305|AAAAAAAAJKJAFCAA|1933-08-20|403|1755|135|1933|0|8|20|3|1933|135|1755|Sunday|1933Q3|N|N|N|2427286|2427497|2426940|2427214|N|N|N|N|N| +2427306|AAAAAAAAKKJAFCAA|1933-08-21|403|1755|135|1933|1|8|21|3|1933|135|1755|Monday|1933Q3|N|N|N|2427286|2427497|2426941|2427215|N|N|N|N|N| +2427307|AAAAAAAALKJAFCAA|1933-08-22|403|1756|135|1933|2|8|22|3|1933|135|1756|Tuesday|1933Q3|N|N|N|2427286|2427497|2426942|2427216|N|N|N|N|N| +2427308|AAAAAAAAMKJAFCAA|1933-08-23|403|1756|135|1933|3|8|23|3|1933|135|1756|Wednesday|1933Q3|N|N|N|2427286|2427497|2426943|2427217|N|N|N|N|N| +2427309|AAAAAAAANKJAFCAA|1933-08-24|403|1756|135|1933|4|8|24|3|1933|135|1756|Thursday|1933Q3|N|N|N|2427286|2427497|2426944|2427218|N|N|N|N|N| +2427310|AAAAAAAAOKJAFCAA|1933-08-25|403|1756|135|1933|5|8|25|3|1933|135|1756|Friday|1933Q3|N|Y|N|2427286|2427497|2426945|2427219|N|N|N|N|N| +2427311|AAAAAAAAPKJAFCAA|1933-08-26|403|1756|135|1933|6|8|26|3|1933|135|1756|Saturday|1933Q3|N|Y|N|2427286|2427497|2426946|2427220|N|N|N|N|N| +2427312|AAAAAAAAALJAFCAA|1933-08-27|403|1756|135|1933|0|8|27|3|1933|135|1756|Sunday|1933Q3|N|N|N|2427286|2427497|2426947|2427221|N|N|N|N|N| +2427313|AAAAAAAABLJAFCAA|1933-08-28|403|1756|135|1933|1|8|28|3|1933|135|1756|Monday|1933Q3|N|N|N|2427286|2427497|2426948|2427222|N|N|N|N|N| +2427314|AAAAAAAACLJAFCAA|1933-08-29|403|1757|135|1933|2|8|29|3|1933|135|1757|Tuesday|1933Q3|N|N|N|2427286|2427497|2426949|2427223|N|N|N|N|N| +2427315|AAAAAAAADLJAFCAA|1933-08-30|403|1757|135|1933|3|8|30|3|1933|135|1757|Wednesday|1933Q3|N|N|N|2427286|2427497|2426950|2427224|N|N|N|N|N| +2427316|AAAAAAAAELJAFCAA|1933-08-31|403|1757|135|1933|4|8|31|3|1933|135|1757|Thursday|1933Q3|N|N|N|2427286|2427497|2426951|2427225|N|N|N|N|N| +2427317|AAAAAAAAFLJAFCAA|1933-09-01|404|1757|136|1933|5|9|1|3|1933|136|1757|Friday|1933Q3|N|Y|N|2427317|2427559|2426952|2427226|N|N|N|N|N| +2427318|AAAAAAAAGLJAFCAA|1933-09-02|404|1757|136|1933|6|9|2|3|1933|136|1757|Saturday|1933Q3|N|Y|N|2427317|2427559|2426953|2427227|N|N|N|N|N| +2427319|AAAAAAAAHLJAFCAA|1933-09-03|404|1757|136|1933|0|9|3|3|1933|136|1757|Sunday|1933Q3|N|N|N|2427317|2427559|2426954|2427228|N|N|N|N|N| +2427320|AAAAAAAAILJAFCAA|1933-09-04|404|1757|136|1933|1|9|4|3|1933|136|1757|Monday|1933Q3|N|N|N|2427317|2427559|2426955|2427229|N|N|N|N|N| +2427321|AAAAAAAAJLJAFCAA|1933-09-05|404|1758|136|1933|2|9|5|3|1933|136|1758|Tuesday|1933Q3|N|N|N|2427317|2427559|2426956|2427230|N|N|N|N|N| +2427322|AAAAAAAAKLJAFCAA|1933-09-06|404|1758|136|1933|3|9|6|3|1933|136|1758|Wednesday|1933Q3|N|N|N|2427317|2427559|2426957|2427231|N|N|N|N|N| +2427323|AAAAAAAALLJAFCAA|1933-09-07|404|1758|136|1933|4|9|7|3|1933|136|1758|Thursday|1933Q3|N|N|N|2427317|2427559|2426958|2427232|N|N|N|N|N| +2427324|AAAAAAAAMLJAFCAA|1933-09-08|404|1758|136|1933|5|9|8|3|1933|136|1758|Friday|1933Q3|N|Y|N|2427317|2427559|2426959|2427233|N|N|N|N|N| +2427325|AAAAAAAANLJAFCAA|1933-09-09|404|1758|136|1933|6|9|9|3|1933|136|1758|Saturday|1933Q3|N|Y|N|2427317|2427559|2426960|2427234|N|N|N|N|N| +2427326|AAAAAAAAOLJAFCAA|1933-09-10|404|1758|136|1933|0|9|10|3|1933|136|1758|Sunday|1933Q3|N|N|N|2427317|2427559|2426961|2427235|N|N|N|N|N| +2427327|AAAAAAAAPLJAFCAA|1933-09-11|404|1758|136|1933|1|9|11|3|1933|136|1758|Monday|1933Q3|N|N|N|2427317|2427559|2426962|2427236|N|N|N|N|N| +2427328|AAAAAAAAAMJAFCAA|1933-09-12|404|1759|136|1933|2|9|12|3|1933|136|1759|Tuesday|1933Q3|N|N|N|2427317|2427559|2426963|2427237|N|N|N|N|N| +2427329|AAAAAAAABMJAFCAA|1933-09-13|404|1759|136|1933|3|9|13|3|1933|136|1759|Wednesday|1933Q3|N|N|N|2427317|2427559|2426964|2427238|N|N|N|N|N| +2427330|AAAAAAAACMJAFCAA|1933-09-14|404|1759|136|1933|4|9|14|3|1933|136|1759|Thursday|1933Q3|N|N|N|2427317|2427559|2426965|2427239|N|N|N|N|N| +2427331|AAAAAAAADMJAFCAA|1933-09-15|404|1759|136|1933|5|9|15|3|1933|136|1759|Friday|1933Q3|N|Y|N|2427317|2427559|2426966|2427240|N|N|N|N|N| +2427332|AAAAAAAAEMJAFCAA|1933-09-16|404|1759|136|1933|6|9|16|3|1933|136|1759|Saturday|1933Q3|N|Y|N|2427317|2427559|2426967|2427241|N|N|N|N|N| +2427333|AAAAAAAAFMJAFCAA|1933-09-17|404|1759|136|1933|0|9|17|3|1933|136|1759|Sunday|1933Q3|N|N|N|2427317|2427559|2426968|2427242|N|N|N|N|N| +2427334|AAAAAAAAGMJAFCAA|1933-09-18|404|1759|136|1933|1|9|18|3|1933|136|1759|Monday|1933Q3|N|N|N|2427317|2427559|2426969|2427243|N|N|N|N|N| +2427335|AAAAAAAAHMJAFCAA|1933-09-19|404|1760|136|1933|2|9|19|3|1933|136|1760|Tuesday|1933Q3|N|N|N|2427317|2427559|2426970|2427244|N|N|N|N|N| +2427336|AAAAAAAAIMJAFCAA|1933-09-20|404|1760|136|1933|3|9|20|3|1933|136|1760|Wednesday|1933Q3|N|N|N|2427317|2427559|2426971|2427245|N|N|N|N|N| +2427337|AAAAAAAAJMJAFCAA|1933-09-21|404|1760|136|1933|4|9|21|3|1933|136|1760|Thursday|1933Q3|N|N|N|2427317|2427559|2426972|2427246|N|N|N|N|N| +2427338|AAAAAAAAKMJAFCAA|1933-09-22|404|1760|136|1933|5|9|22|3|1933|136|1760|Friday|1933Q3|N|Y|N|2427317|2427559|2426973|2427247|N|N|N|N|N| +2427339|AAAAAAAALMJAFCAA|1933-09-23|404|1760|136|1933|6|9|23|3|1933|136|1760|Saturday|1933Q3|N|Y|N|2427317|2427559|2426974|2427248|N|N|N|N|N| +2427340|AAAAAAAAMMJAFCAA|1933-09-24|404|1760|136|1933|0|9|24|3|1933|136|1760|Sunday|1933Q3|N|N|N|2427317|2427559|2426975|2427249|N|N|N|N|N| +2427341|AAAAAAAANMJAFCAA|1933-09-25|404|1760|136|1933|1|9|25|3|1933|136|1760|Monday|1933Q3|N|N|N|2427317|2427559|2426976|2427250|N|N|N|N|N| +2427342|AAAAAAAAOMJAFCAA|1933-09-26|404|1761|136|1933|2|9|26|3|1933|136|1761|Tuesday|1933Q3|N|N|N|2427317|2427559|2426977|2427251|N|N|N|N|N| +2427343|AAAAAAAAPMJAFCAA|1933-09-27|404|1761|136|1933|3|9|27|3|1933|136|1761|Wednesday|1933Q3|N|N|N|2427317|2427559|2426978|2427252|N|N|N|N|N| +2427344|AAAAAAAAANJAFCAA|1933-09-28|404|1761|136|1933|4|9|28|3|1933|136|1761|Thursday|1933Q3|N|N|N|2427317|2427559|2426979|2427253|N|N|N|N|N| +2427345|AAAAAAAABNJAFCAA|1933-09-29|404|1761|136|1933|5|9|29|3|1933|136|1761|Friday|1933Q3|N|Y|N|2427317|2427559|2426980|2427254|N|N|N|N|N| +2427346|AAAAAAAACNJAFCAA|1933-09-30|404|1761|136|1933|6|9|30|3|1933|136|1761|Saturday|1933Q3|N|Y|N|2427317|2427559|2426981|2427255|N|N|N|N|N| +2427347|AAAAAAAADNJAFCAA|1933-10-01|405|1761|136|1933|0|10|1|3|1933|136|1761|Sunday|1933Q3|N|N|N|2427347|2427619|2426982|2427255|N|N|N|N|N| +2427348|AAAAAAAAENJAFCAA|1933-10-02|405|1761|136|1933|1|10|2|4|1933|136|1761|Monday|1933Q4|N|N|N|2427347|2427619|2426983|2427256|N|N|N|N|N| +2427349|AAAAAAAAFNJAFCAA|1933-10-03|405|1762|136|1933|2|10|3|4|1933|136|1762|Tuesday|1933Q4|N|N|N|2427347|2427619|2426984|2427257|N|N|N|N|N| +2427350|AAAAAAAAGNJAFCAA|1933-10-04|405|1762|136|1933|3|10|4|4|1933|136|1762|Wednesday|1933Q4|N|N|N|2427347|2427619|2426985|2427258|N|N|N|N|N| +2427351|AAAAAAAAHNJAFCAA|1933-10-05|405|1762|136|1933|4|10|5|4|1933|136|1762|Thursday|1933Q4|N|N|N|2427347|2427619|2426986|2427259|N|N|N|N|N| +2427352|AAAAAAAAINJAFCAA|1933-10-06|405|1762|136|1933|5|10|6|4|1933|136|1762|Friday|1933Q4|N|Y|N|2427347|2427619|2426987|2427260|N|N|N|N|N| +2427353|AAAAAAAAJNJAFCAA|1933-10-07|405|1762|136|1933|6|10|7|4|1933|136|1762|Saturday|1933Q4|N|Y|N|2427347|2427619|2426988|2427261|N|N|N|N|N| +2427354|AAAAAAAAKNJAFCAA|1933-10-08|405|1762|136|1933|0|10|8|4|1933|136|1762|Sunday|1933Q4|N|N|N|2427347|2427619|2426989|2427262|N|N|N|N|N| +2427355|AAAAAAAALNJAFCAA|1933-10-09|405|1762|136|1933|1|10|9|4|1933|136|1762|Monday|1933Q4|N|N|N|2427347|2427619|2426990|2427263|N|N|N|N|N| +2427356|AAAAAAAAMNJAFCAA|1933-10-10|405|1763|136|1933|2|10|10|4|1933|136|1763|Tuesday|1933Q4|N|N|N|2427347|2427619|2426991|2427264|N|N|N|N|N| +2427357|AAAAAAAANNJAFCAA|1933-10-11|405|1763|136|1933|3|10|11|4|1933|136|1763|Wednesday|1933Q4|N|N|N|2427347|2427619|2426992|2427265|N|N|N|N|N| +2427358|AAAAAAAAONJAFCAA|1933-10-12|405|1763|136|1933|4|10|12|4|1933|136|1763|Thursday|1933Q4|N|N|N|2427347|2427619|2426993|2427266|N|N|N|N|N| +2427359|AAAAAAAAPNJAFCAA|1933-10-13|405|1763|136|1933|5|10|13|4|1933|136|1763|Friday|1933Q4|N|Y|N|2427347|2427619|2426994|2427267|N|N|N|N|N| +2427360|AAAAAAAAAOJAFCAA|1933-10-14|405|1763|136|1933|6|10|14|4|1933|136|1763|Saturday|1933Q4|N|Y|N|2427347|2427619|2426995|2427268|N|N|N|N|N| +2427361|AAAAAAAABOJAFCAA|1933-10-15|405|1763|136|1933|0|10|15|4|1933|136|1763|Sunday|1933Q4|N|N|N|2427347|2427619|2426996|2427269|N|N|N|N|N| +2427362|AAAAAAAACOJAFCAA|1933-10-16|405|1763|136|1933|1|10|16|4|1933|136|1763|Monday|1933Q4|N|N|N|2427347|2427619|2426997|2427270|N|N|N|N|N| +2427363|AAAAAAAADOJAFCAA|1933-10-17|405|1764|136|1933|2|10|17|4|1933|136|1764|Tuesday|1933Q4|N|N|N|2427347|2427619|2426998|2427271|N|N|N|N|N| +2427364|AAAAAAAAEOJAFCAA|1933-10-18|405|1764|136|1933|3|10|18|4|1933|136|1764|Wednesday|1933Q4|N|N|N|2427347|2427619|2426999|2427272|N|N|N|N|N| +2427365|AAAAAAAAFOJAFCAA|1933-10-19|405|1764|136|1933|4|10|19|4|1933|136|1764|Thursday|1933Q4|N|N|N|2427347|2427619|2427000|2427273|N|N|N|N|N| +2427366|AAAAAAAAGOJAFCAA|1933-10-20|405|1764|136|1933|5|10|20|4|1933|136|1764|Friday|1933Q4|N|Y|N|2427347|2427619|2427001|2427274|N|N|N|N|N| +2427367|AAAAAAAAHOJAFCAA|1933-10-21|405|1764|136|1933|6|10|21|4|1933|136|1764|Saturday|1933Q4|N|Y|N|2427347|2427619|2427002|2427275|N|N|N|N|N| +2427368|AAAAAAAAIOJAFCAA|1933-10-22|405|1764|136|1933|0|10|22|4|1933|136|1764|Sunday|1933Q4|N|N|N|2427347|2427619|2427003|2427276|N|N|N|N|N| +2427369|AAAAAAAAJOJAFCAA|1933-10-23|405|1764|136|1933|1|10|23|4|1933|136|1764|Monday|1933Q4|N|N|N|2427347|2427619|2427004|2427277|N|N|N|N|N| +2427370|AAAAAAAAKOJAFCAA|1933-10-24|405|1765|136|1933|2|10|24|4|1933|136|1765|Tuesday|1933Q4|N|N|N|2427347|2427619|2427005|2427278|N|N|N|N|N| +2427371|AAAAAAAALOJAFCAA|1933-10-25|405|1765|136|1933|3|10|25|4|1933|136|1765|Wednesday|1933Q4|N|N|N|2427347|2427619|2427006|2427279|N|N|N|N|N| +2427372|AAAAAAAAMOJAFCAA|1933-10-26|405|1765|136|1933|4|10|26|4|1933|136|1765|Thursday|1933Q4|N|N|N|2427347|2427619|2427007|2427280|N|N|N|N|N| +2427373|AAAAAAAANOJAFCAA|1933-10-27|405|1765|136|1933|5|10|27|4|1933|136|1765|Friday|1933Q4|N|Y|N|2427347|2427619|2427008|2427281|N|N|N|N|N| +2427374|AAAAAAAAOOJAFCAA|1933-10-28|405|1765|136|1933|6|10|28|4|1933|136|1765|Saturday|1933Q4|N|Y|N|2427347|2427619|2427009|2427282|N|N|N|N|N| +2427375|AAAAAAAAPOJAFCAA|1933-10-29|405|1765|136|1933|0|10|29|4|1933|136|1765|Sunday|1933Q4|N|N|N|2427347|2427619|2427010|2427283|N|N|N|N|N| +2427376|AAAAAAAAAPJAFCAA|1933-10-30|405|1765|136|1933|1|10|30|4|1933|136|1765|Monday|1933Q4|N|N|N|2427347|2427619|2427011|2427284|N|N|N|N|N| +2427377|AAAAAAAABPJAFCAA|1933-10-31|405|1766|136|1933|2|10|31|4|1933|136|1766|Tuesday|1933Q4|N|N|N|2427347|2427619|2427012|2427285|N|N|N|N|N| +2427378|AAAAAAAACPJAFCAA|1933-11-01|406|1766|136|1933|3|11|1|4|1933|136|1766|Wednesday|1933Q4|N|N|N|2427378|2427681|2427013|2427286|N|N|N|N|N| +2427379|AAAAAAAADPJAFCAA|1933-11-02|406|1766|136|1933|4|11|2|4|1933|136|1766|Thursday|1933Q4|N|N|N|2427378|2427681|2427014|2427287|N|N|N|N|N| +2427380|AAAAAAAAEPJAFCAA|1933-11-03|406|1766|136|1933|5|11|3|4|1933|136|1766|Friday|1933Q4|N|Y|N|2427378|2427681|2427015|2427288|N|N|N|N|N| +2427381|AAAAAAAAFPJAFCAA|1933-11-04|406|1766|136|1933|6|11|4|4|1933|136|1766|Saturday|1933Q4|N|Y|N|2427378|2427681|2427016|2427289|N|N|N|N|N| +2427382|AAAAAAAAGPJAFCAA|1933-11-05|406|1766|136|1933|0|11|5|4|1933|136|1766|Sunday|1933Q4|N|N|N|2427378|2427681|2427017|2427290|N|N|N|N|N| +2427383|AAAAAAAAHPJAFCAA|1933-11-06|406|1766|136|1933|1|11|6|4|1933|136|1766|Monday|1933Q4|N|N|N|2427378|2427681|2427018|2427291|N|N|N|N|N| +2427384|AAAAAAAAIPJAFCAA|1933-11-07|406|1767|136|1933|2|11|7|4|1933|136|1767|Tuesday|1933Q4|N|N|N|2427378|2427681|2427019|2427292|N|N|N|N|N| +2427385|AAAAAAAAJPJAFCAA|1933-11-08|406|1767|136|1933|3|11|8|4|1933|136|1767|Wednesday|1933Q4|N|N|N|2427378|2427681|2427020|2427293|N|N|N|N|N| +2427386|AAAAAAAAKPJAFCAA|1933-11-09|406|1767|136|1933|4|11|9|4|1933|136|1767|Thursday|1933Q4|N|N|N|2427378|2427681|2427021|2427294|N|N|N|N|N| +2427387|AAAAAAAALPJAFCAA|1933-11-10|406|1767|136|1933|5|11|10|4|1933|136|1767|Friday|1933Q4|N|Y|N|2427378|2427681|2427022|2427295|N|N|N|N|N| +2427388|AAAAAAAAMPJAFCAA|1933-11-11|406|1767|136|1933|6|11|11|4|1933|136|1767|Saturday|1933Q4|N|Y|N|2427378|2427681|2427023|2427296|N|N|N|N|N| +2427389|AAAAAAAANPJAFCAA|1933-11-12|406|1767|136|1933|0|11|12|4|1933|136|1767|Sunday|1933Q4|N|N|N|2427378|2427681|2427024|2427297|N|N|N|N|N| +2427390|AAAAAAAAOPJAFCAA|1933-11-13|406|1767|136|1933|1|11|13|4|1933|136|1767|Monday|1933Q4|N|N|N|2427378|2427681|2427025|2427298|N|N|N|N|N| +2427391|AAAAAAAAPPJAFCAA|1933-11-14|406|1768|136|1933|2|11|14|4|1933|136|1768|Tuesday|1933Q4|N|N|N|2427378|2427681|2427026|2427299|N|N|N|N|N| +2427392|AAAAAAAAAAKAFCAA|1933-11-15|406|1768|136|1933|3|11|15|4|1933|136|1768|Wednesday|1933Q4|N|N|N|2427378|2427681|2427027|2427300|N|N|N|N|N| +2427393|AAAAAAAABAKAFCAA|1933-11-16|406|1768|136|1933|4|11|16|4|1933|136|1768|Thursday|1933Q4|N|N|N|2427378|2427681|2427028|2427301|N|N|N|N|N| +2427394|AAAAAAAACAKAFCAA|1933-11-17|406|1768|136|1933|5|11|17|4|1933|136|1768|Friday|1933Q4|N|Y|N|2427378|2427681|2427029|2427302|N|N|N|N|N| +2427395|AAAAAAAADAKAFCAA|1933-11-18|406|1768|136|1933|6|11|18|4|1933|136|1768|Saturday|1933Q4|N|Y|N|2427378|2427681|2427030|2427303|N|N|N|N|N| +2427396|AAAAAAAAEAKAFCAA|1933-11-19|406|1768|136|1933|0|11|19|4|1933|136|1768|Sunday|1933Q4|N|N|N|2427378|2427681|2427031|2427304|N|N|N|N|N| +2427397|AAAAAAAAFAKAFCAA|1933-11-20|406|1768|136|1933|1|11|20|4|1933|136|1768|Monday|1933Q4|N|N|N|2427378|2427681|2427032|2427305|N|N|N|N|N| +2427398|AAAAAAAAGAKAFCAA|1933-11-21|406|1769|136|1933|2|11|21|4|1933|136|1769|Tuesday|1933Q4|N|N|N|2427378|2427681|2427033|2427306|N|N|N|N|N| +2427399|AAAAAAAAHAKAFCAA|1933-11-22|406|1769|136|1933|3|11|22|4|1933|136|1769|Wednesday|1933Q4|N|N|N|2427378|2427681|2427034|2427307|N|N|N|N|N| +2427400|AAAAAAAAIAKAFCAA|1933-11-23|406|1769|136|1933|4|11|23|4|1933|136|1769|Thursday|1933Q4|N|N|N|2427378|2427681|2427035|2427308|N|N|N|N|N| +2427401|AAAAAAAAJAKAFCAA|1933-11-24|406|1769|136|1933|5|11|24|4|1933|136|1769|Friday|1933Q4|N|Y|N|2427378|2427681|2427036|2427309|N|N|N|N|N| +2427402|AAAAAAAAKAKAFCAA|1933-11-25|406|1769|136|1933|6|11|25|4|1933|136|1769|Saturday|1933Q4|N|Y|N|2427378|2427681|2427037|2427310|N|N|N|N|N| +2427403|AAAAAAAALAKAFCAA|1933-11-26|406|1769|136|1933|0|11|26|4|1933|136|1769|Sunday|1933Q4|N|N|N|2427378|2427681|2427038|2427311|N|N|N|N|N| +2427404|AAAAAAAAMAKAFCAA|1933-11-27|406|1769|136|1933|1|11|27|4|1933|136|1769|Monday|1933Q4|N|N|N|2427378|2427681|2427039|2427312|N|N|N|N|N| +2427405|AAAAAAAANAKAFCAA|1933-11-28|406|1770|136|1933|2|11|28|4|1933|136|1770|Tuesday|1933Q4|N|N|N|2427378|2427681|2427040|2427313|N|N|N|N|N| +2427406|AAAAAAAAOAKAFCAA|1933-11-29|406|1770|136|1933|3|11|29|4|1933|136|1770|Wednesday|1933Q4|N|N|N|2427378|2427681|2427041|2427314|N|N|N|N|N| +2427407|AAAAAAAAPAKAFCAA|1933-11-30|406|1770|136|1933|4|11|30|4|1933|136|1770|Thursday|1933Q4|N|N|N|2427378|2427681|2427042|2427315|N|N|N|N|N| +2427408|AAAAAAAAABKAFCAA|1933-12-01|407|1770|137|1933|5|12|1|4|1933|137|1770|Friday|1933Q4|N|Y|N|2427408|2427741|2427043|2427316|N|N|N|N|N| +2427409|AAAAAAAABBKAFCAA|1933-12-02|407|1770|137|1933|6|12|2|4|1933|137|1770|Saturday|1933Q4|N|Y|N|2427408|2427741|2427044|2427317|N|N|N|N|N| +2427410|AAAAAAAACBKAFCAA|1933-12-03|407|1770|137|1933|0|12|3|4|1933|137|1770|Sunday|1933Q4|N|N|N|2427408|2427741|2427045|2427318|N|N|N|N|N| +2427411|AAAAAAAADBKAFCAA|1933-12-04|407|1770|137|1933|1|12|4|4|1933|137|1770|Monday|1933Q4|N|N|N|2427408|2427741|2427046|2427319|N|N|N|N|N| +2427412|AAAAAAAAEBKAFCAA|1933-12-05|407|1771|137|1933|2|12|5|4|1933|137|1771|Tuesday|1933Q4|N|N|N|2427408|2427741|2427047|2427320|N|N|N|N|N| +2427413|AAAAAAAAFBKAFCAA|1933-12-06|407|1771|137|1933|3|12|6|4|1933|137|1771|Wednesday|1933Q4|N|N|N|2427408|2427741|2427048|2427321|N|N|N|N|N| +2427414|AAAAAAAAGBKAFCAA|1933-12-07|407|1771|137|1933|4|12|7|4|1933|137|1771|Thursday|1933Q4|N|N|N|2427408|2427741|2427049|2427322|N|N|N|N|N| +2427415|AAAAAAAAHBKAFCAA|1933-12-08|407|1771|137|1933|5|12|8|4|1933|137|1771|Friday|1933Q4|N|Y|N|2427408|2427741|2427050|2427323|N|N|N|N|N| +2427416|AAAAAAAAIBKAFCAA|1933-12-09|407|1771|137|1933|6|12|9|4|1933|137|1771|Saturday|1933Q4|N|Y|N|2427408|2427741|2427051|2427324|N|N|N|N|N| +2427417|AAAAAAAAJBKAFCAA|1933-12-10|407|1771|137|1933|0|12|10|4|1933|137|1771|Sunday|1933Q4|N|N|N|2427408|2427741|2427052|2427325|N|N|N|N|N| +2427418|AAAAAAAAKBKAFCAA|1933-12-11|407|1771|137|1933|1|12|11|4|1933|137|1771|Monday|1933Q4|N|N|N|2427408|2427741|2427053|2427326|N|N|N|N|N| +2427419|AAAAAAAALBKAFCAA|1933-12-12|407|1772|137|1933|2|12|12|4|1933|137|1772|Tuesday|1933Q4|N|N|N|2427408|2427741|2427054|2427327|N|N|N|N|N| +2427420|AAAAAAAAMBKAFCAA|1933-12-13|407|1772|137|1933|3|12|13|4|1933|137|1772|Wednesday|1933Q4|N|N|N|2427408|2427741|2427055|2427328|N|N|N|N|N| +2427421|AAAAAAAANBKAFCAA|1933-12-14|407|1772|137|1933|4|12|14|4|1933|137|1772|Thursday|1933Q4|N|N|N|2427408|2427741|2427056|2427329|N|N|N|N|N| +2427422|AAAAAAAAOBKAFCAA|1933-12-15|407|1772|137|1933|5|12|15|4|1933|137|1772|Friday|1933Q4|N|Y|N|2427408|2427741|2427057|2427330|N|N|N|N|N| +2427423|AAAAAAAAPBKAFCAA|1933-12-16|407|1772|137|1933|6|12|16|4|1933|137|1772|Saturday|1933Q4|N|Y|N|2427408|2427741|2427058|2427331|N|N|N|N|N| +2427424|AAAAAAAAACKAFCAA|1933-12-17|407|1772|137|1933|0|12|17|4|1933|137|1772|Sunday|1933Q4|N|N|N|2427408|2427741|2427059|2427332|N|N|N|N|N| +2427425|AAAAAAAABCKAFCAA|1933-12-18|407|1772|137|1933|1|12|18|4|1933|137|1772|Monday|1933Q4|N|N|N|2427408|2427741|2427060|2427333|N|N|N|N|N| +2427426|AAAAAAAACCKAFCAA|1933-12-19|407|1773|137|1933|2|12|19|4|1933|137|1773|Tuesday|1933Q4|N|N|N|2427408|2427741|2427061|2427334|N|N|N|N|N| +2427427|AAAAAAAADCKAFCAA|1933-12-20|407|1773|137|1933|3|12|20|4|1933|137|1773|Wednesday|1933Q4|N|N|N|2427408|2427741|2427062|2427335|N|N|N|N|N| +2427428|AAAAAAAAECKAFCAA|1933-12-21|407|1773|137|1933|4|12|21|4|1933|137|1773|Thursday|1933Q4|N|N|N|2427408|2427741|2427063|2427336|N|N|N|N|N| +2427429|AAAAAAAAFCKAFCAA|1933-12-22|407|1773|137|1933|5|12|22|4|1933|137|1773|Friday|1933Q4|N|Y|N|2427408|2427741|2427064|2427337|N|N|N|N|N| +2427430|AAAAAAAAGCKAFCAA|1933-12-23|407|1773|137|1933|6|12|23|4|1933|137|1773|Saturday|1933Q4|N|Y|N|2427408|2427741|2427065|2427338|N|N|N|N|N| +2427431|AAAAAAAAHCKAFCAA|1933-12-24|407|1773|137|1933|0|12|24|4|1933|137|1773|Sunday|1933Q4|N|N|N|2427408|2427741|2427066|2427339|N|N|N|N|N| +2427432|AAAAAAAAICKAFCAA|1933-12-25|407|1773|137|1933|1|12|25|4|1933|137|1773|Monday|1933Q4|N|N|N|2427408|2427741|2427067|2427340|N|N|N|N|N| +2427433|AAAAAAAAJCKAFCAA|1933-12-26|407|1774|137|1933|2|12|26|4|1933|137|1774|Tuesday|1933Q4|Y|N|N|2427408|2427741|2427068|2427341|N|N|N|N|N| +2427434|AAAAAAAAKCKAFCAA|1933-12-27|407|1774|137|1933|3|12|27|4|1933|137|1774|Wednesday|1933Q4|N|N|Y|2427408|2427741|2427069|2427342|N|N|N|N|N| +2427435|AAAAAAAALCKAFCAA|1933-12-28|407|1774|137|1933|4|12|28|4|1933|137|1774|Thursday|1933Q4|N|N|N|2427408|2427741|2427070|2427343|N|N|N|N|N| +2427436|AAAAAAAAMCKAFCAA|1933-12-29|407|1774|137|1933|5|12|29|4|1933|137|1774|Friday|1933Q4|N|Y|N|2427408|2427741|2427071|2427344|N|N|N|N|N| +2427437|AAAAAAAANCKAFCAA|1933-12-30|407|1774|137|1933|6|12|30|4|1933|137|1774|Saturday|1933Q4|N|Y|N|2427408|2427741|2427072|2427345|N|N|N|N|N| +2427438|AAAAAAAAOCKAFCAA|1933-12-31|407|1774|137|1933|0|12|31|4|1933|137|1774|Sunday|1933Q4|N|N|N|2427408|2427741|2427073|2427346|N|N|N|N|N| +2427439|AAAAAAAAPCKAFCAA|1934-01-01|408|1774|137|1934|1|1|1|1|1934|137|1774|Monday|1934Q1|Y|N|N|2427439|2427438|2427074|2427347|N|N|N|N|N| +2427440|AAAAAAAAADKAFCAA|1934-01-02|408|1775|137|1934|2|1|2|1|1934|137|1775|Tuesday|1934Q1|N|N|Y|2427439|2427438|2427075|2427348|N|N|N|N|N| +2427441|AAAAAAAABDKAFCAA|1934-01-03|408|1775|137|1934|3|1|3|1|1934|137|1775|Wednesday|1934Q1|N|N|N|2427439|2427438|2427076|2427349|N|N|N|N|N| +2427442|AAAAAAAACDKAFCAA|1934-01-04|408|1775|137|1934|4|1|4|1|1934|137|1775|Thursday|1934Q1|N|N|N|2427439|2427438|2427077|2427350|N|N|N|N|N| +2427443|AAAAAAAADDKAFCAA|1934-01-05|408|1775|137|1934|5|1|5|1|1934|137|1775|Friday|1934Q1|N|Y|N|2427439|2427438|2427078|2427351|N|N|N|N|N| +2427444|AAAAAAAAEDKAFCAA|1934-01-06|408|1775|137|1934|6|1|6|1|1934|137|1775|Saturday|1934Q1|N|Y|N|2427439|2427438|2427079|2427352|N|N|N|N|N| +2427445|AAAAAAAAFDKAFCAA|1934-01-07|408|1775|137|1934|0|1|7|1|1934|137|1775|Sunday|1934Q1|N|N|N|2427439|2427438|2427080|2427353|N|N|N|N|N| +2427446|AAAAAAAAGDKAFCAA|1934-01-08|408|1775|137|1934|1|1|8|1|1934|137|1775|Monday|1934Q1|N|N|N|2427439|2427438|2427081|2427354|N|N|N|N|N| +2427447|AAAAAAAAHDKAFCAA|1934-01-09|408|1776|137|1934|2|1|9|1|1934|137|1776|Tuesday|1934Q1|N|N|N|2427439|2427438|2427082|2427355|N|N|N|N|N| +2427448|AAAAAAAAIDKAFCAA|1934-01-10|408|1776|137|1934|3|1|10|1|1934|137|1776|Wednesday|1934Q1|N|N|N|2427439|2427438|2427083|2427356|N|N|N|N|N| +2427449|AAAAAAAAJDKAFCAA|1934-01-11|408|1776|137|1934|4|1|11|1|1934|137|1776|Thursday|1934Q1|N|N|N|2427439|2427438|2427084|2427357|N|N|N|N|N| +2427450|AAAAAAAAKDKAFCAA|1934-01-12|408|1776|137|1934|5|1|12|1|1934|137|1776|Friday|1934Q1|N|Y|N|2427439|2427438|2427085|2427358|N|N|N|N|N| +2427451|AAAAAAAALDKAFCAA|1934-01-13|408|1776|137|1934|6|1|13|1|1934|137|1776|Saturday|1934Q1|N|Y|N|2427439|2427438|2427086|2427359|N|N|N|N|N| +2427452|AAAAAAAAMDKAFCAA|1934-01-14|408|1776|137|1934|0|1|14|1|1934|137|1776|Sunday|1934Q1|N|N|N|2427439|2427438|2427087|2427360|N|N|N|N|N| +2427453|AAAAAAAANDKAFCAA|1934-01-15|408|1776|137|1934|1|1|15|1|1934|137|1776|Monday|1934Q1|N|N|N|2427439|2427438|2427088|2427361|N|N|N|N|N| +2427454|AAAAAAAAODKAFCAA|1934-01-16|408|1777|137|1934|2|1|16|1|1934|137|1777|Tuesday|1934Q1|N|N|N|2427439|2427438|2427089|2427362|N|N|N|N|N| +2427455|AAAAAAAAPDKAFCAA|1934-01-17|408|1777|137|1934|3|1|17|1|1934|137|1777|Wednesday|1934Q1|N|N|N|2427439|2427438|2427090|2427363|N|N|N|N|N| +2427456|AAAAAAAAAEKAFCAA|1934-01-18|408|1777|137|1934|4|1|18|1|1934|137|1777|Thursday|1934Q1|N|N|N|2427439|2427438|2427091|2427364|N|N|N|N|N| +2427457|AAAAAAAABEKAFCAA|1934-01-19|408|1777|137|1934|5|1|19|1|1934|137|1777|Friday|1934Q1|N|Y|N|2427439|2427438|2427092|2427365|N|N|N|N|N| +2427458|AAAAAAAACEKAFCAA|1934-01-20|408|1777|137|1934|6|1|20|1|1934|137|1777|Saturday|1934Q1|N|Y|N|2427439|2427438|2427093|2427366|N|N|N|N|N| +2427459|AAAAAAAADEKAFCAA|1934-01-21|408|1777|137|1934|0|1|21|1|1934|137|1777|Sunday|1934Q1|N|N|N|2427439|2427438|2427094|2427367|N|N|N|N|N| +2427460|AAAAAAAAEEKAFCAA|1934-01-22|408|1777|137|1934|1|1|22|1|1934|137|1777|Monday|1934Q1|N|N|N|2427439|2427438|2427095|2427368|N|N|N|N|N| +2427461|AAAAAAAAFEKAFCAA|1934-01-23|408|1778|137|1934|2|1|23|1|1934|137|1778|Tuesday|1934Q1|N|N|N|2427439|2427438|2427096|2427369|N|N|N|N|N| +2427462|AAAAAAAAGEKAFCAA|1934-01-24|408|1778|137|1934|3|1|24|1|1934|137|1778|Wednesday|1934Q1|N|N|N|2427439|2427438|2427097|2427370|N|N|N|N|N| +2427463|AAAAAAAAHEKAFCAA|1934-01-25|408|1778|137|1934|4|1|25|1|1934|137|1778|Thursday|1934Q1|N|N|N|2427439|2427438|2427098|2427371|N|N|N|N|N| +2427464|AAAAAAAAIEKAFCAA|1934-01-26|408|1778|137|1934|5|1|26|1|1934|137|1778|Friday|1934Q1|N|Y|N|2427439|2427438|2427099|2427372|N|N|N|N|N| +2427465|AAAAAAAAJEKAFCAA|1934-01-27|408|1778|137|1934|6|1|27|1|1934|137|1778|Saturday|1934Q1|N|Y|N|2427439|2427438|2427100|2427373|N|N|N|N|N| +2427466|AAAAAAAAKEKAFCAA|1934-01-28|408|1778|137|1934|0|1|28|1|1934|137|1778|Sunday|1934Q1|N|N|N|2427439|2427438|2427101|2427374|N|N|N|N|N| +2427467|AAAAAAAALEKAFCAA|1934-01-29|408|1778|137|1934|1|1|29|1|1934|137|1778|Monday|1934Q1|N|N|N|2427439|2427438|2427102|2427375|N|N|N|N|N| +2427468|AAAAAAAAMEKAFCAA|1934-01-30|408|1779|137|1934|2|1|30|1|1934|137|1779|Tuesday|1934Q1|N|N|N|2427439|2427438|2427103|2427376|N|N|N|N|N| +2427469|AAAAAAAANEKAFCAA|1934-01-31|408|1779|137|1934|3|1|31|1|1934|137|1779|Wednesday|1934Q1|N|N|N|2427439|2427438|2427104|2427377|N|N|N|N|N| +2427470|AAAAAAAAOEKAFCAA|1934-02-01|409|1779|137|1934|4|2|1|1|1934|137|1779|Thursday|1934Q1|N|N|N|2427470|2427500|2427105|2427378|N|N|N|N|N| +2427471|AAAAAAAAPEKAFCAA|1934-02-02|409|1779|137|1934|5|2|2|1|1934|137|1779|Friday|1934Q1|N|Y|N|2427470|2427500|2427106|2427379|N|N|N|N|N| +2427472|AAAAAAAAAFKAFCAA|1934-02-03|409|1779|137|1934|6|2|3|1|1934|137|1779|Saturday|1934Q1|N|Y|N|2427470|2427500|2427107|2427380|N|N|N|N|N| +2427473|AAAAAAAABFKAFCAA|1934-02-04|409|1779|137|1934|0|2|4|1|1934|137|1779|Sunday|1934Q1|N|N|N|2427470|2427500|2427108|2427381|N|N|N|N|N| +2427474|AAAAAAAACFKAFCAA|1934-02-05|409|1779|137|1934|1|2|5|1|1934|137|1779|Monday|1934Q1|N|N|N|2427470|2427500|2427109|2427382|N|N|N|N|N| +2427475|AAAAAAAADFKAFCAA|1934-02-06|409|1780|137|1934|2|2|6|1|1934|137|1780|Tuesday|1934Q1|N|N|N|2427470|2427500|2427110|2427383|N|N|N|N|N| +2427476|AAAAAAAAEFKAFCAA|1934-02-07|409|1780|137|1934|3|2|7|1|1934|137|1780|Wednesday|1934Q1|N|N|N|2427470|2427500|2427111|2427384|N|N|N|N|N| +2427477|AAAAAAAAFFKAFCAA|1934-02-08|409|1780|137|1934|4|2|8|1|1934|137|1780|Thursday|1934Q1|N|N|N|2427470|2427500|2427112|2427385|N|N|N|N|N| +2427478|AAAAAAAAGFKAFCAA|1934-02-09|409|1780|137|1934|5|2|9|1|1934|137|1780|Friday|1934Q1|N|Y|N|2427470|2427500|2427113|2427386|N|N|N|N|N| +2427479|AAAAAAAAHFKAFCAA|1934-02-10|409|1780|137|1934|6|2|10|1|1934|137|1780|Saturday|1934Q1|N|Y|N|2427470|2427500|2427114|2427387|N|N|N|N|N| +2427480|AAAAAAAAIFKAFCAA|1934-02-11|409|1780|137|1934|0|2|11|1|1934|137|1780|Sunday|1934Q1|N|N|N|2427470|2427500|2427115|2427388|N|N|N|N|N| +2427481|AAAAAAAAJFKAFCAA|1934-02-12|409|1780|137|1934|1|2|12|1|1934|137|1780|Monday|1934Q1|N|N|N|2427470|2427500|2427116|2427389|N|N|N|N|N| +2427482|AAAAAAAAKFKAFCAA|1934-02-13|409|1781|137|1934|2|2|13|1|1934|137|1781|Tuesday|1934Q1|N|N|N|2427470|2427500|2427117|2427390|N|N|N|N|N| +2427483|AAAAAAAALFKAFCAA|1934-02-14|409|1781|137|1934|3|2|14|1|1934|137|1781|Wednesday|1934Q1|N|N|N|2427470|2427500|2427118|2427391|N|N|N|N|N| +2427484|AAAAAAAAMFKAFCAA|1934-02-15|409|1781|137|1934|4|2|15|1|1934|137|1781|Thursday|1934Q1|N|N|N|2427470|2427500|2427119|2427392|N|N|N|N|N| +2427485|AAAAAAAANFKAFCAA|1934-02-16|409|1781|137|1934|5|2|16|1|1934|137|1781|Friday|1934Q1|N|Y|N|2427470|2427500|2427120|2427393|N|N|N|N|N| +2427486|AAAAAAAAOFKAFCAA|1934-02-17|409|1781|137|1934|6|2|17|1|1934|137|1781|Saturday|1934Q1|N|Y|N|2427470|2427500|2427121|2427394|N|N|N|N|N| +2427487|AAAAAAAAPFKAFCAA|1934-02-18|409|1781|137|1934|0|2|18|1|1934|137|1781|Sunday|1934Q1|N|N|N|2427470|2427500|2427122|2427395|N|N|N|N|N| +2427488|AAAAAAAAAGKAFCAA|1934-02-19|409|1781|137|1934|1|2|19|1|1934|137|1781|Monday|1934Q1|N|N|N|2427470|2427500|2427123|2427396|N|N|N|N|N| +2427489|AAAAAAAABGKAFCAA|1934-02-20|409|1782|137|1934|2|2|20|1|1934|137|1782|Tuesday|1934Q1|N|N|N|2427470|2427500|2427124|2427397|N|N|N|N|N| +2427490|AAAAAAAACGKAFCAA|1934-02-21|409|1782|137|1934|3|2|21|1|1934|137|1782|Wednesday|1934Q1|N|N|N|2427470|2427500|2427125|2427398|N|N|N|N|N| +2427491|AAAAAAAADGKAFCAA|1934-02-22|409|1782|137|1934|4|2|22|1|1934|137|1782|Thursday|1934Q1|N|N|N|2427470|2427500|2427126|2427399|N|N|N|N|N| +2427492|AAAAAAAAEGKAFCAA|1934-02-23|409|1782|137|1934|5|2|23|1|1934|137|1782|Friday|1934Q1|N|Y|N|2427470|2427500|2427127|2427400|N|N|N|N|N| +2427493|AAAAAAAAFGKAFCAA|1934-02-24|409|1782|137|1934|6|2|24|1|1934|137|1782|Saturday|1934Q1|N|Y|N|2427470|2427500|2427128|2427401|N|N|N|N|N| +2427494|AAAAAAAAGGKAFCAA|1934-02-25|409|1782|137|1934|0|2|25|1|1934|137|1782|Sunday|1934Q1|N|N|N|2427470|2427500|2427129|2427402|N|N|N|N|N| +2427495|AAAAAAAAHGKAFCAA|1934-02-26|409|1782|137|1934|1|2|26|1|1934|137|1782|Monday|1934Q1|N|N|N|2427470|2427500|2427130|2427403|N|N|N|N|N| +2427496|AAAAAAAAIGKAFCAA|1934-02-27|409|1783|137|1934|2|2|27|1|1934|137|1783|Tuesday|1934Q1|N|N|N|2427470|2427500|2427131|2427404|N|N|N|N|N| +2427497|AAAAAAAAJGKAFCAA|1934-02-28|409|1783|137|1934|3|2|28|1|1934|137|1783|Wednesday|1934Q1|N|N|N|2427470|2427500|2427132|2427405|N|N|N|N|N| +2427498|AAAAAAAAKGKAFCAA|1934-03-01|410|1783|138|1934|4|3|1|1|1934|138|1783|Thursday|1934Q1|N|N|N|2427498|2427556|2427133|2427406|N|N|N|N|N| +2427499|AAAAAAAALGKAFCAA|1934-03-02|410|1783|138|1934|5|3|2|1|1934|138|1783|Friday|1934Q1|N|Y|N|2427498|2427556|2427134|2427407|N|N|N|N|N| +2427500|AAAAAAAAMGKAFCAA|1934-03-03|410|1783|138|1934|6|3|3|1|1934|138|1783|Saturday|1934Q1|N|Y|N|2427498|2427556|2427135|2427408|N|N|N|N|N| +2427501|AAAAAAAANGKAFCAA|1934-03-04|410|1783|138|1934|0|3|4|1|1934|138|1783|Sunday|1934Q1|N|N|N|2427498|2427556|2427136|2427409|N|N|N|N|N| +2427502|AAAAAAAAOGKAFCAA|1934-03-05|410|1783|138|1934|1|3|5|1|1934|138|1783|Monday|1934Q1|N|N|N|2427498|2427556|2427137|2427410|N|N|N|N|N| +2427503|AAAAAAAAPGKAFCAA|1934-03-06|410|1784|138|1934|2|3|6|1|1934|138|1784|Tuesday|1934Q1|N|N|N|2427498|2427556|2427138|2427411|N|N|N|N|N| +2427504|AAAAAAAAAHKAFCAA|1934-03-07|410|1784|138|1934|3|3|7|1|1934|138|1784|Wednesday|1934Q1|N|N|N|2427498|2427556|2427139|2427412|N|N|N|N|N| +2427505|AAAAAAAABHKAFCAA|1934-03-08|410|1784|138|1934|4|3|8|1|1934|138|1784|Thursday|1934Q1|N|N|N|2427498|2427556|2427140|2427413|N|N|N|N|N| +2427506|AAAAAAAACHKAFCAA|1934-03-09|410|1784|138|1934|5|3|9|1|1934|138|1784|Friday|1934Q1|N|Y|N|2427498|2427556|2427141|2427414|N|N|N|N|N| +2427507|AAAAAAAADHKAFCAA|1934-03-10|410|1784|138|1934|6|3|10|1|1934|138|1784|Saturday|1934Q1|N|Y|N|2427498|2427556|2427142|2427415|N|N|N|N|N| +2427508|AAAAAAAAEHKAFCAA|1934-03-11|410|1784|138|1934|0|3|11|1|1934|138|1784|Sunday|1934Q1|N|N|N|2427498|2427556|2427143|2427416|N|N|N|N|N| +2427509|AAAAAAAAFHKAFCAA|1934-03-12|410|1784|138|1934|1|3|12|1|1934|138|1784|Monday|1934Q1|N|N|N|2427498|2427556|2427144|2427417|N|N|N|N|N| +2427510|AAAAAAAAGHKAFCAA|1934-03-13|410|1785|138|1934|2|3|13|1|1934|138|1785|Tuesday|1934Q1|N|N|N|2427498|2427556|2427145|2427418|N|N|N|N|N| +2427511|AAAAAAAAHHKAFCAA|1934-03-14|410|1785|138|1934|3|3|14|1|1934|138|1785|Wednesday|1934Q1|N|N|N|2427498|2427556|2427146|2427419|N|N|N|N|N| +2427512|AAAAAAAAIHKAFCAA|1934-03-15|410|1785|138|1934|4|3|15|1|1934|138|1785|Thursday|1934Q1|N|N|N|2427498|2427556|2427147|2427420|N|N|N|N|N| +2427513|AAAAAAAAJHKAFCAA|1934-03-16|410|1785|138|1934|5|3|16|1|1934|138|1785|Friday|1934Q1|N|Y|N|2427498|2427556|2427148|2427421|N|N|N|N|N| +2427514|AAAAAAAAKHKAFCAA|1934-03-17|410|1785|138|1934|6|3|17|1|1934|138|1785|Saturday|1934Q1|N|Y|N|2427498|2427556|2427149|2427422|N|N|N|N|N| +2427515|AAAAAAAALHKAFCAA|1934-03-18|410|1785|138|1934|0|3|18|1|1934|138|1785|Sunday|1934Q1|N|N|N|2427498|2427556|2427150|2427423|N|N|N|N|N| +2427516|AAAAAAAAMHKAFCAA|1934-03-19|410|1785|138|1934|1|3|19|1|1934|138|1785|Monday|1934Q1|N|N|N|2427498|2427556|2427151|2427424|N|N|N|N|N| +2427517|AAAAAAAANHKAFCAA|1934-03-20|410|1786|138|1934|2|3|20|1|1934|138|1786|Tuesday|1934Q1|N|N|N|2427498|2427556|2427152|2427425|N|N|N|N|N| +2427518|AAAAAAAAOHKAFCAA|1934-03-21|410|1786|138|1934|3|3|21|1|1934|138|1786|Wednesday|1934Q1|N|N|N|2427498|2427556|2427153|2427426|N|N|N|N|N| +2427519|AAAAAAAAPHKAFCAA|1934-03-22|410|1786|138|1934|4|3|22|1|1934|138|1786|Thursday|1934Q1|N|N|N|2427498|2427556|2427154|2427427|N|N|N|N|N| +2427520|AAAAAAAAAIKAFCAA|1934-03-23|410|1786|138|1934|5|3|23|1|1934|138|1786|Friday|1934Q1|N|Y|N|2427498|2427556|2427155|2427428|N|N|N|N|N| +2427521|AAAAAAAABIKAFCAA|1934-03-24|410|1786|138|1934|6|3|24|1|1934|138|1786|Saturday|1934Q1|N|Y|N|2427498|2427556|2427156|2427429|N|N|N|N|N| +2427522|AAAAAAAACIKAFCAA|1934-03-25|410|1786|138|1934|0|3|25|1|1934|138|1786|Sunday|1934Q1|N|N|N|2427498|2427556|2427157|2427430|N|N|N|N|N| +2427523|AAAAAAAADIKAFCAA|1934-03-26|410|1786|138|1934|1|3|26|1|1934|138|1786|Monday|1934Q1|N|N|N|2427498|2427556|2427158|2427431|N|N|N|N|N| +2427524|AAAAAAAAEIKAFCAA|1934-03-27|410|1787|138|1934|2|3|27|1|1934|138|1787|Tuesday|1934Q1|N|N|N|2427498|2427556|2427159|2427432|N|N|N|N|N| +2427525|AAAAAAAAFIKAFCAA|1934-03-28|410|1787|138|1934|3|3|28|1|1934|138|1787|Wednesday|1934Q1|N|N|N|2427498|2427556|2427160|2427433|N|N|N|N|N| +2427526|AAAAAAAAGIKAFCAA|1934-03-29|410|1787|138|1934|4|3|29|1|1934|138|1787|Thursday|1934Q1|N|N|N|2427498|2427556|2427161|2427434|N|N|N|N|N| +2427527|AAAAAAAAHIKAFCAA|1934-03-30|410|1787|138|1934|5|3|30|1|1934|138|1787|Friday|1934Q1|N|Y|N|2427498|2427556|2427162|2427435|N|N|N|N|N| +2427528|AAAAAAAAIIKAFCAA|1934-03-31|410|1787|138|1934|6|3|31|1|1934|138|1787|Saturday|1934Q1|N|Y|N|2427498|2427556|2427163|2427436|N|N|N|N|N| +2427529|AAAAAAAAJIKAFCAA|1934-04-01|411|1787|138|1934|0|4|1|1|1934|138|1787|Sunday|1934Q1|N|N|N|2427529|2427618|2427164|2427439|N|N|N|N|N| +2427530|AAAAAAAAKIKAFCAA|1934-04-02|411|1787|138|1934|1|4|2|2|1934|138|1787|Monday|1934Q2|N|N|N|2427529|2427618|2427165|2427440|N|N|N|N|N| +2427531|AAAAAAAALIKAFCAA|1934-04-03|411|1788|138|1934|2|4|3|2|1934|138|1788|Tuesday|1934Q2|N|N|N|2427529|2427618|2427166|2427441|N|N|N|N|N| +2427532|AAAAAAAAMIKAFCAA|1934-04-04|411|1788|138|1934|3|4|4|2|1934|138|1788|Wednesday|1934Q2|N|N|N|2427529|2427618|2427167|2427442|N|N|N|N|N| +2427533|AAAAAAAANIKAFCAA|1934-04-05|411|1788|138|1934|4|4|5|2|1934|138|1788|Thursday|1934Q2|N|N|N|2427529|2427618|2427168|2427443|N|N|N|N|N| +2427534|AAAAAAAAOIKAFCAA|1934-04-06|411|1788|138|1934|5|4|6|2|1934|138|1788|Friday|1934Q2|N|Y|N|2427529|2427618|2427169|2427444|N|N|N|N|N| +2427535|AAAAAAAAPIKAFCAA|1934-04-07|411|1788|138|1934|6|4|7|2|1934|138|1788|Saturday|1934Q2|N|Y|N|2427529|2427618|2427170|2427445|N|N|N|N|N| +2427536|AAAAAAAAAJKAFCAA|1934-04-08|411|1788|138|1934|0|4|8|2|1934|138|1788|Sunday|1934Q2|N|N|N|2427529|2427618|2427171|2427446|N|N|N|N|N| +2427537|AAAAAAAABJKAFCAA|1934-04-09|411|1788|138|1934|1|4|9|2|1934|138|1788|Monday|1934Q2|N|N|N|2427529|2427618|2427172|2427447|N|N|N|N|N| +2427538|AAAAAAAACJKAFCAA|1934-04-10|411|1789|138|1934|2|4|10|2|1934|138|1789|Tuesday|1934Q2|N|N|N|2427529|2427618|2427173|2427448|N|N|N|N|N| +2427539|AAAAAAAADJKAFCAA|1934-04-11|411|1789|138|1934|3|4|11|2|1934|138|1789|Wednesday|1934Q2|N|N|N|2427529|2427618|2427174|2427449|N|N|N|N|N| +2427540|AAAAAAAAEJKAFCAA|1934-04-12|411|1789|138|1934|4|4|12|2|1934|138|1789|Thursday|1934Q2|N|N|N|2427529|2427618|2427175|2427450|N|N|N|N|N| +2427541|AAAAAAAAFJKAFCAA|1934-04-13|411|1789|138|1934|5|4|13|2|1934|138|1789|Friday|1934Q2|N|Y|N|2427529|2427618|2427176|2427451|N|N|N|N|N| +2427542|AAAAAAAAGJKAFCAA|1934-04-14|411|1789|138|1934|6|4|14|2|1934|138|1789|Saturday|1934Q2|N|Y|N|2427529|2427618|2427177|2427452|N|N|N|N|N| +2427543|AAAAAAAAHJKAFCAA|1934-04-15|411|1789|138|1934|0|4|15|2|1934|138|1789|Sunday|1934Q2|N|N|N|2427529|2427618|2427178|2427453|N|N|N|N|N| +2427544|AAAAAAAAIJKAFCAA|1934-04-16|411|1789|138|1934|1|4|16|2|1934|138|1789|Monday|1934Q2|N|N|N|2427529|2427618|2427179|2427454|N|N|N|N|N| +2427545|AAAAAAAAJJKAFCAA|1934-04-17|411|1790|138|1934|2|4|17|2|1934|138|1790|Tuesday|1934Q2|N|N|N|2427529|2427618|2427180|2427455|N|N|N|N|N| +2427546|AAAAAAAAKJKAFCAA|1934-04-18|411|1790|138|1934|3|4|18|2|1934|138|1790|Wednesday|1934Q2|N|N|N|2427529|2427618|2427181|2427456|N|N|N|N|N| +2427547|AAAAAAAALJKAFCAA|1934-04-19|411|1790|138|1934|4|4|19|2|1934|138|1790|Thursday|1934Q2|N|N|N|2427529|2427618|2427182|2427457|N|N|N|N|N| +2427548|AAAAAAAAMJKAFCAA|1934-04-20|411|1790|138|1934|5|4|20|2|1934|138|1790|Friday|1934Q2|N|Y|N|2427529|2427618|2427183|2427458|N|N|N|N|N| +2427549|AAAAAAAANJKAFCAA|1934-04-21|411|1790|138|1934|6|4|21|2|1934|138|1790|Saturday|1934Q2|N|Y|N|2427529|2427618|2427184|2427459|N|N|N|N|N| +2427550|AAAAAAAAOJKAFCAA|1934-04-22|411|1790|138|1934|0|4|22|2|1934|138|1790|Sunday|1934Q2|N|N|N|2427529|2427618|2427185|2427460|N|N|N|N|N| +2427551|AAAAAAAAPJKAFCAA|1934-04-23|411|1790|138|1934|1|4|23|2|1934|138|1790|Monday|1934Q2|N|N|N|2427529|2427618|2427186|2427461|N|N|N|N|N| +2427552|AAAAAAAAAKKAFCAA|1934-04-24|411|1791|138|1934|2|4|24|2|1934|138|1791|Tuesday|1934Q2|N|N|N|2427529|2427618|2427187|2427462|N|N|N|N|N| +2427553|AAAAAAAABKKAFCAA|1934-04-25|411|1791|138|1934|3|4|25|2|1934|138|1791|Wednesday|1934Q2|N|N|N|2427529|2427618|2427188|2427463|N|N|N|N|N| +2427554|AAAAAAAACKKAFCAA|1934-04-26|411|1791|138|1934|4|4|26|2|1934|138|1791|Thursday|1934Q2|N|N|N|2427529|2427618|2427189|2427464|N|N|N|N|N| +2427555|AAAAAAAADKKAFCAA|1934-04-27|411|1791|138|1934|5|4|27|2|1934|138|1791|Friday|1934Q2|N|Y|N|2427529|2427618|2427190|2427465|N|N|N|N|N| +2427556|AAAAAAAAEKKAFCAA|1934-04-28|411|1791|138|1934|6|4|28|2|1934|138|1791|Saturday|1934Q2|N|Y|N|2427529|2427618|2427191|2427466|N|N|N|N|N| +2427557|AAAAAAAAFKKAFCAA|1934-04-29|411|1791|138|1934|0|4|29|2|1934|138|1791|Sunday|1934Q2|N|N|N|2427529|2427618|2427192|2427467|N|N|N|N|N| +2427558|AAAAAAAAGKKAFCAA|1934-04-30|411|1791|138|1934|1|4|30|2|1934|138|1791|Monday|1934Q2|N|N|N|2427529|2427618|2427193|2427468|N|N|N|N|N| +2427559|AAAAAAAAHKKAFCAA|1934-05-01|412|1792|138|1934|2|5|1|2|1934|138|1792|Tuesday|1934Q2|N|N|N|2427559|2427678|2427194|2427469|N|N|N|N|N| +2427560|AAAAAAAAIKKAFCAA|1934-05-02|412|1792|138|1934|3|5|2|2|1934|138|1792|Wednesday|1934Q2|N|N|N|2427559|2427678|2427195|2427470|N|N|N|N|N| +2427561|AAAAAAAAJKKAFCAA|1934-05-03|412|1792|138|1934|4|5|3|2|1934|138|1792|Thursday|1934Q2|N|N|N|2427559|2427678|2427196|2427471|N|N|N|N|N| +2427562|AAAAAAAAKKKAFCAA|1934-05-04|412|1792|138|1934|5|5|4|2|1934|138|1792|Friday|1934Q2|N|Y|N|2427559|2427678|2427197|2427472|N|N|N|N|N| +2427563|AAAAAAAALKKAFCAA|1934-05-05|412|1792|138|1934|6|5|5|2|1934|138|1792|Saturday|1934Q2|N|Y|N|2427559|2427678|2427198|2427473|N|N|N|N|N| +2427564|AAAAAAAAMKKAFCAA|1934-05-06|412|1792|138|1934|0|5|6|2|1934|138|1792|Sunday|1934Q2|N|N|N|2427559|2427678|2427199|2427474|N|N|N|N|N| +2427565|AAAAAAAANKKAFCAA|1934-05-07|412|1792|138|1934|1|5|7|2|1934|138|1792|Monday|1934Q2|N|N|N|2427559|2427678|2427200|2427475|N|N|N|N|N| +2427566|AAAAAAAAOKKAFCAA|1934-05-08|412|1793|138|1934|2|5|8|2|1934|138|1793|Tuesday|1934Q2|N|N|N|2427559|2427678|2427201|2427476|N|N|N|N|N| +2427567|AAAAAAAAPKKAFCAA|1934-05-09|412|1793|138|1934|3|5|9|2|1934|138|1793|Wednesday|1934Q2|N|N|N|2427559|2427678|2427202|2427477|N|N|N|N|N| +2427568|AAAAAAAAALKAFCAA|1934-05-10|412|1793|138|1934|4|5|10|2|1934|138|1793|Thursday|1934Q2|N|N|N|2427559|2427678|2427203|2427478|N|N|N|N|N| +2427569|AAAAAAAABLKAFCAA|1934-05-11|412|1793|138|1934|5|5|11|2|1934|138|1793|Friday|1934Q2|N|Y|N|2427559|2427678|2427204|2427479|N|N|N|N|N| +2427570|AAAAAAAACLKAFCAA|1934-05-12|412|1793|138|1934|6|5|12|2|1934|138|1793|Saturday|1934Q2|N|Y|N|2427559|2427678|2427205|2427480|N|N|N|N|N| +2427571|AAAAAAAADLKAFCAA|1934-05-13|412|1793|138|1934|0|5|13|2|1934|138|1793|Sunday|1934Q2|N|N|N|2427559|2427678|2427206|2427481|N|N|N|N|N| +2427572|AAAAAAAAELKAFCAA|1934-05-14|412|1793|138|1934|1|5|14|2|1934|138|1793|Monday|1934Q2|N|N|N|2427559|2427678|2427207|2427482|N|N|N|N|N| +2427573|AAAAAAAAFLKAFCAA|1934-05-15|412|1794|138|1934|2|5|15|2|1934|138|1794|Tuesday|1934Q2|N|N|N|2427559|2427678|2427208|2427483|N|N|N|N|N| +2427574|AAAAAAAAGLKAFCAA|1934-05-16|412|1794|138|1934|3|5|16|2|1934|138|1794|Wednesday|1934Q2|N|N|N|2427559|2427678|2427209|2427484|N|N|N|N|N| +2427575|AAAAAAAAHLKAFCAA|1934-05-17|412|1794|138|1934|4|5|17|2|1934|138|1794|Thursday|1934Q2|N|N|N|2427559|2427678|2427210|2427485|N|N|N|N|N| +2427576|AAAAAAAAILKAFCAA|1934-05-18|412|1794|138|1934|5|5|18|2|1934|138|1794|Friday|1934Q2|N|Y|N|2427559|2427678|2427211|2427486|N|N|N|N|N| +2427577|AAAAAAAAJLKAFCAA|1934-05-19|412|1794|138|1934|6|5|19|2|1934|138|1794|Saturday|1934Q2|N|Y|N|2427559|2427678|2427212|2427487|N|N|N|N|N| +2427578|AAAAAAAAKLKAFCAA|1934-05-20|412|1794|138|1934|0|5|20|2|1934|138|1794|Sunday|1934Q2|N|N|N|2427559|2427678|2427213|2427488|N|N|N|N|N| +2427579|AAAAAAAALLKAFCAA|1934-05-21|412|1794|138|1934|1|5|21|2|1934|138|1794|Monday|1934Q2|N|N|N|2427559|2427678|2427214|2427489|N|N|N|N|N| +2427580|AAAAAAAAMLKAFCAA|1934-05-22|412|1795|138|1934|2|5|22|2|1934|138|1795|Tuesday|1934Q2|N|N|N|2427559|2427678|2427215|2427490|N|N|N|N|N| +2427581|AAAAAAAANLKAFCAA|1934-05-23|412|1795|138|1934|3|5|23|2|1934|138|1795|Wednesday|1934Q2|N|N|N|2427559|2427678|2427216|2427491|N|N|N|N|N| +2427582|AAAAAAAAOLKAFCAA|1934-05-24|412|1795|138|1934|4|5|24|2|1934|138|1795|Thursday|1934Q2|N|N|N|2427559|2427678|2427217|2427492|N|N|N|N|N| +2427583|AAAAAAAAPLKAFCAA|1934-05-25|412|1795|138|1934|5|5|25|2|1934|138|1795|Friday|1934Q2|N|Y|N|2427559|2427678|2427218|2427493|N|N|N|N|N| +2427584|AAAAAAAAAMKAFCAA|1934-05-26|412|1795|138|1934|6|5|26|2|1934|138|1795|Saturday|1934Q2|N|Y|N|2427559|2427678|2427219|2427494|N|N|N|N|N| +2427585|AAAAAAAABMKAFCAA|1934-05-27|412|1795|138|1934|0|5|27|2|1934|138|1795|Sunday|1934Q2|N|N|N|2427559|2427678|2427220|2427495|N|N|N|N|N| +2427586|AAAAAAAACMKAFCAA|1934-05-28|412|1795|138|1934|1|5|28|2|1934|138|1795|Monday|1934Q2|N|N|N|2427559|2427678|2427221|2427496|N|N|N|N|N| +2427587|AAAAAAAADMKAFCAA|1934-05-29|412|1796|138|1934|2|5|29|2|1934|138|1796|Tuesday|1934Q2|N|N|N|2427559|2427678|2427222|2427497|N|N|N|N|N| +2427588|AAAAAAAAEMKAFCAA|1934-05-30|412|1796|138|1934|3|5|30|2|1934|138|1796|Wednesday|1934Q2|N|N|N|2427559|2427678|2427223|2427498|N|N|N|N|N| +2427589|AAAAAAAAFMKAFCAA|1934-05-31|412|1796|138|1934|4|5|31|2|1934|138|1796|Thursday|1934Q2|N|N|N|2427559|2427678|2427224|2427499|N|N|N|N|N| +2427590|AAAAAAAAGMKAFCAA|1934-06-01|413|1796|139|1934|5|6|1|2|1934|139|1796|Friday|1934Q2|N|Y|N|2427590|2427740|2427225|2427500|N|N|N|N|N| +2427591|AAAAAAAAHMKAFCAA|1934-06-02|413|1796|139|1934|6|6|2|2|1934|139|1796|Saturday|1934Q2|N|Y|N|2427590|2427740|2427226|2427501|N|N|N|N|N| +2427592|AAAAAAAAIMKAFCAA|1934-06-03|413|1796|139|1934|0|6|3|2|1934|139|1796|Sunday|1934Q2|N|N|N|2427590|2427740|2427227|2427502|N|N|N|N|N| +2427593|AAAAAAAAJMKAFCAA|1934-06-04|413|1796|139|1934|1|6|4|2|1934|139|1796|Monday|1934Q2|N|N|N|2427590|2427740|2427228|2427503|N|N|N|N|N| +2427594|AAAAAAAAKMKAFCAA|1934-06-05|413|1797|139|1934|2|6|5|2|1934|139|1797|Tuesday|1934Q2|N|N|N|2427590|2427740|2427229|2427504|N|N|N|N|N| +2427595|AAAAAAAALMKAFCAA|1934-06-06|413|1797|139|1934|3|6|6|2|1934|139|1797|Wednesday|1934Q2|N|N|N|2427590|2427740|2427230|2427505|N|N|N|N|N| +2427596|AAAAAAAAMMKAFCAA|1934-06-07|413|1797|139|1934|4|6|7|2|1934|139|1797|Thursday|1934Q2|N|N|N|2427590|2427740|2427231|2427506|N|N|N|N|N| +2427597|AAAAAAAANMKAFCAA|1934-06-08|413|1797|139|1934|5|6|8|2|1934|139|1797|Friday|1934Q2|N|Y|N|2427590|2427740|2427232|2427507|N|N|N|N|N| +2427598|AAAAAAAAOMKAFCAA|1934-06-09|413|1797|139|1934|6|6|9|2|1934|139|1797|Saturday|1934Q2|N|Y|N|2427590|2427740|2427233|2427508|N|N|N|N|N| +2427599|AAAAAAAAPMKAFCAA|1934-06-10|413|1797|139|1934|0|6|10|2|1934|139|1797|Sunday|1934Q2|N|N|N|2427590|2427740|2427234|2427509|N|N|N|N|N| +2427600|AAAAAAAAANKAFCAA|1934-06-11|413|1797|139|1934|1|6|11|2|1934|139|1797|Monday|1934Q2|N|N|N|2427590|2427740|2427235|2427510|N|N|N|N|N| +2427601|AAAAAAAABNKAFCAA|1934-06-12|413|1798|139|1934|2|6|12|2|1934|139|1798|Tuesday|1934Q2|N|N|N|2427590|2427740|2427236|2427511|N|N|N|N|N| +2427602|AAAAAAAACNKAFCAA|1934-06-13|413|1798|139|1934|3|6|13|2|1934|139|1798|Wednesday|1934Q2|N|N|N|2427590|2427740|2427237|2427512|N|N|N|N|N| +2427603|AAAAAAAADNKAFCAA|1934-06-14|413|1798|139|1934|4|6|14|2|1934|139|1798|Thursday|1934Q2|N|N|N|2427590|2427740|2427238|2427513|N|N|N|N|N| +2427604|AAAAAAAAENKAFCAA|1934-06-15|413|1798|139|1934|5|6|15|2|1934|139|1798|Friday|1934Q2|N|Y|N|2427590|2427740|2427239|2427514|N|N|N|N|N| +2427605|AAAAAAAAFNKAFCAA|1934-06-16|413|1798|139|1934|6|6|16|2|1934|139|1798|Saturday|1934Q2|N|Y|N|2427590|2427740|2427240|2427515|N|N|N|N|N| +2427606|AAAAAAAAGNKAFCAA|1934-06-17|413|1798|139|1934|0|6|17|2|1934|139|1798|Sunday|1934Q2|N|N|N|2427590|2427740|2427241|2427516|N|N|N|N|N| +2427607|AAAAAAAAHNKAFCAA|1934-06-18|413|1798|139|1934|1|6|18|2|1934|139|1798|Monday|1934Q2|N|N|N|2427590|2427740|2427242|2427517|N|N|N|N|N| +2427608|AAAAAAAAINKAFCAA|1934-06-19|413|1799|139|1934|2|6|19|2|1934|139|1799|Tuesday|1934Q2|N|N|N|2427590|2427740|2427243|2427518|N|N|N|N|N| +2427609|AAAAAAAAJNKAFCAA|1934-06-20|413|1799|139|1934|3|6|20|2|1934|139|1799|Wednesday|1934Q2|N|N|N|2427590|2427740|2427244|2427519|N|N|N|N|N| +2427610|AAAAAAAAKNKAFCAA|1934-06-21|413|1799|139|1934|4|6|21|2|1934|139|1799|Thursday|1934Q2|N|N|N|2427590|2427740|2427245|2427520|N|N|N|N|N| +2427611|AAAAAAAALNKAFCAA|1934-06-22|413|1799|139|1934|5|6|22|2|1934|139|1799|Friday|1934Q2|N|Y|N|2427590|2427740|2427246|2427521|N|N|N|N|N| +2427612|AAAAAAAAMNKAFCAA|1934-06-23|413|1799|139|1934|6|6|23|2|1934|139|1799|Saturday|1934Q2|N|Y|N|2427590|2427740|2427247|2427522|N|N|N|N|N| +2427613|AAAAAAAANNKAFCAA|1934-06-24|413|1799|139|1934|0|6|24|2|1934|139|1799|Sunday|1934Q2|N|N|N|2427590|2427740|2427248|2427523|N|N|N|N|N| +2427614|AAAAAAAAONKAFCAA|1934-06-25|413|1799|139|1934|1|6|25|2|1934|139|1799|Monday|1934Q2|N|N|N|2427590|2427740|2427249|2427524|N|N|N|N|N| +2427615|AAAAAAAAPNKAFCAA|1934-06-26|413|1800|139|1934|2|6|26|2|1934|139|1800|Tuesday|1934Q2|N|N|N|2427590|2427740|2427250|2427525|N|N|N|N|N| +2427616|AAAAAAAAAOKAFCAA|1934-06-27|413|1800|139|1934|3|6|27|2|1934|139|1800|Wednesday|1934Q2|N|N|N|2427590|2427740|2427251|2427526|N|N|N|N|N| +2427617|AAAAAAAABOKAFCAA|1934-06-28|413|1800|139|1934|4|6|28|2|1934|139|1800|Thursday|1934Q2|N|N|N|2427590|2427740|2427252|2427527|N|N|N|N|N| +2427618|AAAAAAAACOKAFCAA|1934-06-29|413|1800|139|1934|5|6|29|2|1934|139|1800|Friday|1934Q2|N|Y|N|2427590|2427740|2427253|2427528|N|N|N|N|N| +2427619|AAAAAAAADOKAFCAA|1934-06-30|413|1800|139|1934|6|6|30|2|1934|139|1800|Saturday|1934Q2|N|Y|N|2427590|2427740|2427254|2427529|N|N|N|N|N| +2427620|AAAAAAAAEOKAFCAA|1934-07-01|414|1800|139|1934|0|7|1|2|1934|139|1800|Sunday|1934Q2|N|N|N|2427620|2427800|2427255|2427529|N|N|N|N|N| +2427621|AAAAAAAAFOKAFCAA|1934-07-02|414|1800|139|1934|1|7|2|3|1934|139|1800|Monday|1934Q3|N|N|N|2427620|2427800|2427256|2427530|N|N|N|N|N| +2427622|AAAAAAAAGOKAFCAA|1934-07-03|414|1801|139|1934|2|7|3|3|1934|139|1801|Tuesday|1934Q3|N|N|N|2427620|2427800|2427257|2427531|N|N|N|N|N| +2427623|AAAAAAAAHOKAFCAA|1934-07-04|414|1801|139|1934|3|7|4|3|1934|139|1801|Wednesday|1934Q3|N|N|N|2427620|2427800|2427258|2427532|N|N|N|N|N| +2427624|AAAAAAAAIOKAFCAA|1934-07-05|414|1801|139|1934|4|7|5|3|1934|139|1801|Thursday|1934Q3|Y|N|N|2427620|2427800|2427259|2427533|N|N|N|N|N| +2427625|AAAAAAAAJOKAFCAA|1934-07-06|414|1801|139|1934|5|7|6|3|1934|139|1801|Friday|1934Q3|N|Y|Y|2427620|2427800|2427260|2427534|N|N|N|N|N| +2427626|AAAAAAAAKOKAFCAA|1934-07-07|414|1801|139|1934|6|7|7|3|1934|139|1801|Saturday|1934Q3|N|Y|N|2427620|2427800|2427261|2427535|N|N|N|N|N| +2427627|AAAAAAAALOKAFCAA|1934-07-08|414|1801|139|1934|0|7|8|3|1934|139|1801|Sunday|1934Q3|N|N|N|2427620|2427800|2427262|2427536|N|N|N|N|N| +2427628|AAAAAAAAMOKAFCAA|1934-07-09|414|1801|139|1934|1|7|9|3|1934|139|1801|Monday|1934Q3|N|N|N|2427620|2427800|2427263|2427537|N|N|N|N|N| +2427629|AAAAAAAANOKAFCAA|1934-07-10|414|1802|139|1934|2|7|10|3|1934|139|1802|Tuesday|1934Q3|N|N|N|2427620|2427800|2427264|2427538|N|N|N|N|N| +2427630|AAAAAAAAOOKAFCAA|1934-07-11|414|1802|139|1934|3|7|11|3|1934|139|1802|Wednesday|1934Q3|N|N|N|2427620|2427800|2427265|2427539|N|N|N|N|N| +2427631|AAAAAAAAPOKAFCAA|1934-07-12|414|1802|139|1934|4|7|12|3|1934|139|1802|Thursday|1934Q3|N|N|N|2427620|2427800|2427266|2427540|N|N|N|N|N| +2427632|AAAAAAAAAPKAFCAA|1934-07-13|414|1802|139|1934|5|7|13|3|1934|139|1802|Friday|1934Q3|N|Y|N|2427620|2427800|2427267|2427541|N|N|N|N|N| +2427633|AAAAAAAABPKAFCAA|1934-07-14|414|1802|139|1934|6|7|14|3|1934|139|1802|Saturday|1934Q3|N|Y|N|2427620|2427800|2427268|2427542|N|N|N|N|N| +2427634|AAAAAAAACPKAFCAA|1934-07-15|414|1802|139|1934|0|7|15|3|1934|139|1802|Sunday|1934Q3|N|N|N|2427620|2427800|2427269|2427543|N|N|N|N|N| +2427635|AAAAAAAADPKAFCAA|1934-07-16|414|1802|139|1934|1|7|16|3|1934|139|1802|Monday|1934Q3|N|N|N|2427620|2427800|2427270|2427544|N|N|N|N|N| +2427636|AAAAAAAAEPKAFCAA|1934-07-17|414|1803|139|1934|2|7|17|3|1934|139|1803|Tuesday|1934Q3|N|N|N|2427620|2427800|2427271|2427545|N|N|N|N|N| +2427637|AAAAAAAAFPKAFCAA|1934-07-18|414|1803|139|1934|3|7|18|3|1934|139|1803|Wednesday|1934Q3|N|N|N|2427620|2427800|2427272|2427546|N|N|N|N|N| +2427638|AAAAAAAAGPKAFCAA|1934-07-19|414|1803|139|1934|4|7|19|3|1934|139|1803|Thursday|1934Q3|N|N|N|2427620|2427800|2427273|2427547|N|N|N|N|N| +2427639|AAAAAAAAHPKAFCAA|1934-07-20|414|1803|139|1934|5|7|20|3|1934|139|1803|Friday|1934Q3|N|Y|N|2427620|2427800|2427274|2427548|N|N|N|N|N| +2427640|AAAAAAAAIPKAFCAA|1934-07-21|414|1803|139|1934|6|7|21|3|1934|139|1803|Saturday|1934Q3|N|Y|N|2427620|2427800|2427275|2427549|N|N|N|N|N| +2427641|AAAAAAAAJPKAFCAA|1934-07-22|414|1803|139|1934|0|7|22|3|1934|139|1803|Sunday|1934Q3|N|N|N|2427620|2427800|2427276|2427550|N|N|N|N|N| +2427642|AAAAAAAAKPKAFCAA|1934-07-23|414|1803|139|1934|1|7|23|3|1934|139|1803|Monday|1934Q3|N|N|N|2427620|2427800|2427277|2427551|N|N|N|N|N| +2427643|AAAAAAAALPKAFCAA|1934-07-24|414|1804|139|1934|2|7|24|3|1934|139|1804|Tuesday|1934Q3|N|N|N|2427620|2427800|2427278|2427552|N|N|N|N|N| +2427644|AAAAAAAAMPKAFCAA|1934-07-25|414|1804|139|1934|3|7|25|3|1934|139|1804|Wednesday|1934Q3|N|N|N|2427620|2427800|2427279|2427553|N|N|N|N|N| +2427645|AAAAAAAANPKAFCAA|1934-07-26|414|1804|139|1934|4|7|26|3|1934|139|1804|Thursday|1934Q3|N|N|N|2427620|2427800|2427280|2427554|N|N|N|N|N| +2427646|AAAAAAAAOPKAFCAA|1934-07-27|414|1804|139|1934|5|7|27|3|1934|139|1804|Friday|1934Q3|N|Y|N|2427620|2427800|2427281|2427555|N|N|N|N|N| +2427647|AAAAAAAAPPKAFCAA|1934-07-28|414|1804|139|1934|6|7|28|3|1934|139|1804|Saturday|1934Q3|N|Y|N|2427620|2427800|2427282|2427556|N|N|N|N|N| +2427648|AAAAAAAAAALAFCAA|1934-07-29|414|1804|139|1934|0|7|29|3|1934|139|1804|Sunday|1934Q3|N|N|N|2427620|2427800|2427283|2427557|N|N|N|N|N| +2427649|AAAAAAAABALAFCAA|1934-07-30|414|1804|139|1934|1|7|30|3|1934|139|1804|Monday|1934Q3|N|N|N|2427620|2427800|2427284|2427558|N|N|N|N|N| +2427650|AAAAAAAACALAFCAA|1934-07-31|414|1805|139|1934|2|7|31|3|1934|139|1805|Tuesday|1934Q3|N|N|N|2427620|2427800|2427285|2427559|N|N|N|N|N| +2427651|AAAAAAAADALAFCAA|1934-08-01|415|1805|139|1934|3|8|1|3|1934|139|1805|Wednesday|1934Q3|N|N|N|2427651|2427862|2427286|2427560|N|N|N|N|N| +2427652|AAAAAAAAEALAFCAA|1934-08-02|415|1805|139|1934|4|8|2|3|1934|139|1805|Thursday|1934Q3|N|N|N|2427651|2427862|2427287|2427561|N|N|N|N|N| +2427653|AAAAAAAAFALAFCAA|1934-08-03|415|1805|139|1934|5|8|3|3|1934|139|1805|Friday|1934Q3|N|Y|N|2427651|2427862|2427288|2427562|N|N|N|N|N| +2427654|AAAAAAAAGALAFCAA|1934-08-04|415|1805|139|1934|6|8|4|3|1934|139|1805|Saturday|1934Q3|N|Y|N|2427651|2427862|2427289|2427563|N|N|N|N|N| +2427655|AAAAAAAAHALAFCAA|1934-08-05|415|1805|139|1934|0|8|5|3|1934|139|1805|Sunday|1934Q3|N|N|N|2427651|2427862|2427290|2427564|N|N|N|N|N| +2427656|AAAAAAAAIALAFCAA|1934-08-06|415|1805|139|1934|1|8|6|3|1934|139|1805|Monday|1934Q3|N|N|N|2427651|2427862|2427291|2427565|N|N|N|N|N| +2427657|AAAAAAAAJALAFCAA|1934-08-07|415|1806|139|1934|2|8|7|3|1934|139|1806|Tuesday|1934Q3|N|N|N|2427651|2427862|2427292|2427566|N|N|N|N|N| +2427658|AAAAAAAAKALAFCAA|1934-08-08|415|1806|139|1934|3|8|8|3|1934|139|1806|Wednesday|1934Q3|N|N|N|2427651|2427862|2427293|2427567|N|N|N|N|N| +2427659|AAAAAAAALALAFCAA|1934-08-09|415|1806|139|1934|4|8|9|3|1934|139|1806|Thursday|1934Q3|N|N|N|2427651|2427862|2427294|2427568|N|N|N|N|N| +2427660|AAAAAAAAMALAFCAA|1934-08-10|415|1806|139|1934|5|8|10|3|1934|139|1806|Friday|1934Q3|N|Y|N|2427651|2427862|2427295|2427569|N|N|N|N|N| +2427661|AAAAAAAANALAFCAA|1934-08-11|415|1806|139|1934|6|8|11|3|1934|139|1806|Saturday|1934Q3|N|Y|N|2427651|2427862|2427296|2427570|N|N|N|N|N| +2427662|AAAAAAAAOALAFCAA|1934-08-12|415|1806|139|1934|0|8|12|3|1934|139|1806|Sunday|1934Q3|N|N|N|2427651|2427862|2427297|2427571|N|N|N|N|N| +2427663|AAAAAAAAPALAFCAA|1934-08-13|415|1806|139|1934|1|8|13|3|1934|139|1806|Monday|1934Q3|N|N|N|2427651|2427862|2427298|2427572|N|N|N|N|N| +2427664|AAAAAAAAABLAFCAA|1934-08-14|415|1807|139|1934|2|8|14|3|1934|139|1807|Tuesday|1934Q3|N|N|N|2427651|2427862|2427299|2427573|N|N|N|N|N| +2427665|AAAAAAAABBLAFCAA|1934-08-15|415|1807|139|1934|3|8|15|3|1934|139|1807|Wednesday|1934Q3|N|N|N|2427651|2427862|2427300|2427574|N|N|N|N|N| +2427666|AAAAAAAACBLAFCAA|1934-08-16|415|1807|139|1934|4|8|16|3|1934|139|1807|Thursday|1934Q3|N|N|N|2427651|2427862|2427301|2427575|N|N|N|N|N| +2427667|AAAAAAAADBLAFCAA|1934-08-17|415|1807|139|1934|5|8|17|3|1934|139|1807|Friday|1934Q3|N|Y|N|2427651|2427862|2427302|2427576|N|N|N|N|N| +2427668|AAAAAAAAEBLAFCAA|1934-08-18|415|1807|139|1934|6|8|18|3|1934|139|1807|Saturday|1934Q3|N|Y|N|2427651|2427862|2427303|2427577|N|N|N|N|N| +2427669|AAAAAAAAFBLAFCAA|1934-08-19|415|1807|139|1934|0|8|19|3|1934|139|1807|Sunday|1934Q3|N|N|N|2427651|2427862|2427304|2427578|N|N|N|N|N| +2427670|AAAAAAAAGBLAFCAA|1934-08-20|415|1807|139|1934|1|8|20|3|1934|139|1807|Monday|1934Q3|N|N|N|2427651|2427862|2427305|2427579|N|N|N|N|N| +2427671|AAAAAAAAHBLAFCAA|1934-08-21|415|1808|139|1934|2|8|21|3|1934|139|1808|Tuesday|1934Q3|N|N|N|2427651|2427862|2427306|2427580|N|N|N|N|N| +2427672|AAAAAAAAIBLAFCAA|1934-08-22|415|1808|139|1934|3|8|22|3|1934|139|1808|Wednesday|1934Q3|N|N|N|2427651|2427862|2427307|2427581|N|N|N|N|N| +2427673|AAAAAAAAJBLAFCAA|1934-08-23|415|1808|139|1934|4|8|23|3|1934|139|1808|Thursday|1934Q3|N|N|N|2427651|2427862|2427308|2427582|N|N|N|N|N| +2427674|AAAAAAAAKBLAFCAA|1934-08-24|415|1808|139|1934|5|8|24|3|1934|139|1808|Friday|1934Q3|N|Y|N|2427651|2427862|2427309|2427583|N|N|N|N|N| +2427675|AAAAAAAALBLAFCAA|1934-08-25|415|1808|139|1934|6|8|25|3|1934|139|1808|Saturday|1934Q3|N|Y|N|2427651|2427862|2427310|2427584|N|N|N|N|N| +2427676|AAAAAAAAMBLAFCAA|1934-08-26|415|1808|139|1934|0|8|26|3|1934|139|1808|Sunday|1934Q3|N|N|N|2427651|2427862|2427311|2427585|N|N|N|N|N| +2427677|AAAAAAAANBLAFCAA|1934-08-27|415|1808|139|1934|1|8|27|3|1934|139|1808|Monday|1934Q3|N|N|N|2427651|2427862|2427312|2427586|N|N|N|N|N| +2427678|AAAAAAAAOBLAFCAA|1934-08-28|415|1809|139|1934|2|8|28|3|1934|139|1809|Tuesday|1934Q3|N|N|N|2427651|2427862|2427313|2427587|N|N|N|N|N| +2427679|AAAAAAAAPBLAFCAA|1934-08-29|415|1809|139|1934|3|8|29|3|1934|139|1809|Wednesday|1934Q3|N|N|N|2427651|2427862|2427314|2427588|N|N|N|N|N| +2427680|AAAAAAAAACLAFCAA|1934-08-30|415|1809|139|1934|4|8|30|3|1934|139|1809|Thursday|1934Q3|N|N|N|2427651|2427862|2427315|2427589|N|N|N|N|N| +2427681|AAAAAAAABCLAFCAA|1934-08-31|415|1809|139|1934|5|8|31|3|1934|139|1809|Friday|1934Q3|N|Y|N|2427651|2427862|2427316|2427590|N|N|N|N|N| +2427682|AAAAAAAACCLAFCAA|1934-09-01|416|1809|140|1934|6|9|1|3|1934|140|1809|Saturday|1934Q3|N|Y|N|2427682|2427924|2427317|2427591|N|N|N|N|N| +2427683|AAAAAAAADCLAFCAA|1934-09-02|416|1809|140|1934|0|9|2|3|1934|140|1809|Sunday|1934Q3|N|N|N|2427682|2427924|2427318|2427592|N|N|N|N|N| +2427684|AAAAAAAAECLAFCAA|1934-09-03|416|1809|140|1934|1|9|3|3|1934|140|1809|Monday|1934Q3|N|N|N|2427682|2427924|2427319|2427593|N|N|N|N|N| +2427685|AAAAAAAAFCLAFCAA|1934-09-04|416|1810|140|1934|2|9|4|3|1934|140|1810|Tuesday|1934Q3|N|N|N|2427682|2427924|2427320|2427594|N|N|N|N|N| +2427686|AAAAAAAAGCLAFCAA|1934-09-05|416|1810|140|1934|3|9|5|3|1934|140|1810|Wednesday|1934Q3|N|N|N|2427682|2427924|2427321|2427595|N|N|N|N|N| +2427687|AAAAAAAAHCLAFCAA|1934-09-06|416|1810|140|1934|4|9|6|3|1934|140|1810|Thursday|1934Q3|N|N|N|2427682|2427924|2427322|2427596|N|N|N|N|N| +2427688|AAAAAAAAICLAFCAA|1934-09-07|416|1810|140|1934|5|9|7|3|1934|140|1810|Friday|1934Q3|N|Y|N|2427682|2427924|2427323|2427597|N|N|N|N|N| +2427689|AAAAAAAAJCLAFCAA|1934-09-08|416|1810|140|1934|6|9|8|3|1934|140|1810|Saturday|1934Q3|N|Y|N|2427682|2427924|2427324|2427598|N|N|N|N|N| +2427690|AAAAAAAAKCLAFCAA|1934-09-09|416|1810|140|1934|0|9|9|3|1934|140|1810|Sunday|1934Q3|N|N|N|2427682|2427924|2427325|2427599|N|N|N|N|N| +2427691|AAAAAAAALCLAFCAA|1934-09-10|416|1810|140|1934|1|9|10|3|1934|140|1810|Monday|1934Q3|N|N|N|2427682|2427924|2427326|2427600|N|N|N|N|N| +2427692|AAAAAAAAMCLAFCAA|1934-09-11|416|1811|140|1934|2|9|11|3|1934|140|1811|Tuesday|1934Q3|N|N|N|2427682|2427924|2427327|2427601|N|N|N|N|N| +2427693|AAAAAAAANCLAFCAA|1934-09-12|416|1811|140|1934|3|9|12|3|1934|140|1811|Wednesday|1934Q3|N|N|N|2427682|2427924|2427328|2427602|N|N|N|N|N| +2427694|AAAAAAAAOCLAFCAA|1934-09-13|416|1811|140|1934|4|9|13|3|1934|140|1811|Thursday|1934Q3|N|N|N|2427682|2427924|2427329|2427603|N|N|N|N|N| +2427695|AAAAAAAAPCLAFCAA|1934-09-14|416|1811|140|1934|5|9|14|3|1934|140|1811|Friday|1934Q3|N|Y|N|2427682|2427924|2427330|2427604|N|N|N|N|N| +2427696|AAAAAAAAADLAFCAA|1934-09-15|416|1811|140|1934|6|9|15|3|1934|140|1811|Saturday|1934Q3|N|Y|N|2427682|2427924|2427331|2427605|N|N|N|N|N| +2427697|AAAAAAAABDLAFCAA|1934-09-16|416|1811|140|1934|0|9|16|3|1934|140|1811|Sunday|1934Q3|N|N|N|2427682|2427924|2427332|2427606|N|N|N|N|N| +2427698|AAAAAAAACDLAFCAA|1934-09-17|416|1811|140|1934|1|9|17|3|1934|140|1811|Monday|1934Q3|N|N|N|2427682|2427924|2427333|2427607|N|N|N|N|N| +2427699|AAAAAAAADDLAFCAA|1934-09-18|416|1812|140|1934|2|9|18|3|1934|140|1812|Tuesday|1934Q3|N|N|N|2427682|2427924|2427334|2427608|N|N|N|N|N| +2427700|AAAAAAAAEDLAFCAA|1934-09-19|416|1812|140|1934|3|9|19|3|1934|140|1812|Wednesday|1934Q3|N|N|N|2427682|2427924|2427335|2427609|N|N|N|N|N| +2427701|AAAAAAAAFDLAFCAA|1934-09-20|416|1812|140|1934|4|9|20|3|1934|140|1812|Thursday|1934Q3|N|N|N|2427682|2427924|2427336|2427610|N|N|N|N|N| +2427702|AAAAAAAAGDLAFCAA|1934-09-21|416|1812|140|1934|5|9|21|3|1934|140|1812|Friday|1934Q3|N|Y|N|2427682|2427924|2427337|2427611|N|N|N|N|N| +2427703|AAAAAAAAHDLAFCAA|1934-09-22|416|1812|140|1934|6|9|22|3|1934|140|1812|Saturday|1934Q3|N|Y|N|2427682|2427924|2427338|2427612|N|N|N|N|N| +2427704|AAAAAAAAIDLAFCAA|1934-09-23|416|1812|140|1934|0|9|23|3|1934|140|1812|Sunday|1934Q3|N|N|N|2427682|2427924|2427339|2427613|N|N|N|N|N| +2427705|AAAAAAAAJDLAFCAA|1934-09-24|416|1812|140|1934|1|9|24|3|1934|140|1812|Monday|1934Q3|N|N|N|2427682|2427924|2427340|2427614|N|N|N|N|N| +2427706|AAAAAAAAKDLAFCAA|1934-09-25|416|1813|140|1934|2|9|25|3|1934|140|1813|Tuesday|1934Q3|N|N|N|2427682|2427924|2427341|2427615|N|N|N|N|N| +2427707|AAAAAAAALDLAFCAA|1934-09-26|416|1813|140|1934|3|9|26|3|1934|140|1813|Wednesday|1934Q3|N|N|N|2427682|2427924|2427342|2427616|N|N|N|N|N| +2427708|AAAAAAAAMDLAFCAA|1934-09-27|416|1813|140|1934|4|9|27|3|1934|140|1813|Thursday|1934Q3|N|N|N|2427682|2427924|2427343|2427617|N|N|N|N|N| +2427709|AAAAAAAANDLAFCAA|1934-09-28|416|1813|140|1934|5|9|28|3|1934|140|1813|Friday|1934Q3|N|Y|N|2427682|2427924|2427344|2427618|N|N|N|N|N| +2427710|AAAAAAAAODLAFCAA|1934-09-29|416|1813|140|1934|6|9|29|3|1934|140|1813|Saturday|1934Q3|N|Y|N|2427682|2427924|2427345|2427619|N|N|N|N|N| +2427711|AAAAAAAAPDLAFCAA|1934-09-30|416|1813|140|1934|0|9|30|3|1934|140|1813|Sunday|1934Q3|N|N|N|2427682|2427924|2427346|2427620|N|N|N|N|N| +2427712|AAAAAAAAAELAFCAA|1934-10-01|417|1813|140|1934|1|10|1|3|1934|140|1813|Monday|1934Q3|N|N|N|2427712|2427984|2427347|2427620|N|N|N|N|N| +2427713|AAAAAAAABELAFCAA|1934-10-02|417|1814|140|1934|2|10|2|4|1934|140|1814|Tuesday|1934Q4|N|N|N|2427712|2427984|2427348|2427621|N|N|N|N|N| +2427714|AAAAAAAACELAFCAA|1934-10-03|417|1814|140|1934|3|10|3|4|1934|140|1814|Wednesday|1934Q4|N|N|N|2427712|2427984|2427349|2427622|N|N|N|N|N| +2427715|AAAAAAAADELAFCAA|1934-10-04|417|1814|140|1934|4|10|4|4|1934|140|1814|Thursday|1934Q4|N|N|N|2427712|2427984|2427350|2427623|N|N|N|N|N| +2427716|AAAAAAAAEELAFCAA|1934-10-05|417|1814|140|1934|5|10|5|4|1934|140|1814|Friday|1934Q4|N|Y|N|2427712|2427984|2427351|2427624|N|N|N|N|N| +2427717|AAAAAAAAFELAFCAA|1934-10-06|417|1814|140|1934|6|10|6|4|1934|140|1814|Saturday|1934Q4|N|Y|N|2427712|2427984|2427352|2427625|N|N|N|N|N| +2427718|AAAAAAAAGELAFCAA|1934-10-07|417|1814|140|1934|0|10|7|4|1934|140|1814|Sunday|1934Q4|N|N|N|2427712|2427984|2427353|2427626|N|N|N|N|N| +2427719|AAAAAAAAHELAFCAA|1934-10-08|417|1814|140|1934|1|10|8|4|1934|140|1814|Monday|1934Q4|N|N|N|2427712|2427984|2427354|2427627|N|N|N|N|N| +2427720|AAAAAAAAIELAFCAA|1934-10-09|417|1815|140|1934|2|10|9|4|1934|140|1815|Tuesday|1934Q4|N|N|N|2427712|2427984|2427355|2427628|N|N|N|N|N| +2427721|AAAAAAAAJELAFCAA|1934-10-10|417|1815|140|1934|3|10|10|4|1934|140|1815|Wednesday|1934Q4|N|N|N|2427712|2427984|2427356|2427629|N|N|N|N|N| +2427722|AAAAAAAAKELAFCAA|1934-10-11|417|1815|140|1934|4|10|11|4|1934|140|1815|Thursday|1934Q4|N|N|N|2427712|2427984|2427357|2427630|N|N|N|N|N| +2427723|AAAAAAAALELAFCAA|1934-10-12|417|1815|140|1934|5|10|12|4|1934|140|1815|Friday|1934Q4|N|Y|N|2427712|2427984|2427358|2427631|N|N|N|N|N| +2427724|AAAAAAAAMELAFCAA|1934-10-13|417|1815|140|1934|6|10|13|4|1934|140|1815|Saturday|1934Q4|N|Y|N|2427712|2427984|2427359|2427632|N|N|N|N|N| +2427725|AAAAAAAANELAFCAA|1934-10-14|417|1815|140|1934|0|10|14|4|1934|140|1815|Sunday|1934Q4|N|N|N|2427712|2427984|2427360|2427633|N|N|N|N|N| +2427726|AAAAAAAAOELAFCAA|1934-10-15|417|1815|140|1934|1|10|15|4|1934|140|1815|Monday|1934Q4|N|N|N|2427712|2427984|2427361|2427634|N|N|N|N|N| +2427727|AAAAAAAAPELAFCAA|1934-10-16|417|1816|140|1934|2|10|16|4|1934|140|1816|Tuesday|1934Q4|N|N|N|2427712|2427984|2427362|2427635|N|N|N|N|N| +2427728|AAAAAAAAAFLAFCAA|1934-10-17|417|1816|140|1934|3|10|17|4|1934|140|1816|Wednesday|1934Q4|N|N|N|2427712|2427984|2427363|2427636|N|N|N|N|N| +2427729|AAAAAAAABFLAFCAA|1934-10-18|417|1816|140|1934|4|10|18|4|1934|140|1816|Thursday|1934Q4|N|N|N|2427712|2427984|2427364|2427637|N|N|N|N|N| +2427730|AAAAAAAACFLAFCAA|1934-10-19|417|1816|140|1934|5|10|19|4|1934|140|1816|Friday|1934Q4|N|Y|N|2427712|2427984|2427365|2427638|N|N|N|N|N| +2427731|AAAAAAAADFLAFCAA|1934-10-20|417|1816|140|1934|6|10|20|4|1934|140|1816|Saturday|1934Q4|N|Y|N|2427712|2427984|2427366|2427639|N|N|N|N|N| +2427732|AAAAAAAAEFLAFCAA|1934-10-21|417|1816|140|1934|0|10|21|4|1934|140|1816|Sunday|1934Q4|N|N|N|2427712|2427984|2427367|2427640|N|N|N|N|N| +2427733|AAAAAAAAFFLAFCAA|1934-10-22|417|1816|140|1934|1|10|22|4|1934|140|1816|Monday|1934Q4|N|N|N|2427712|2427984|2427368|2427641|N|N|N|N|N| +2427734|AAAAAAAAGFLAFCAA|1934-10-23|417|1817|140|1934|2|10|23|4|1934|140|1817|Tuesday|1934Q4|N|N|N|2427712|2427984|2427369|2427642|N|N|N|N|N| +2427735|AAAAAAAAHFLAFCAA|1934-10-24|417|1817|140|1934|3|10|24|4|1934|140|1817|Wednesday|1934Q4|N|N|N|2427712|2427984|2427370|2427643|N|N|N|N|N| +2427736|AAAAAAAAIFLAFCAA|1934-10-25|417|1817|140|1934|4|10|25|4|1934|140|1817|Thursday|1934Q4|N|N|N|2427712|2427984|2427371|2427644|N|N|N|N|N| +2427737|AAAAAAAAJFLAFCAA|1934-10-26|417|1817|140|1934|5|10|26|4|1934|140|1817|Friday|1934Q4|N|Y|N|2427712|2427984|2427372|2427645|N|N|N|N|N| +2427738|AAAAAAAAKFLAFCAA|1934-10-27|417|1817|140|1934|6|10|27|4|1934|140|1817|Saturday|1934Q4|N|Y|N|2427712|2427984|2427373|2427646|N|N|N|N|N| +2427739|AAAAAAAALFLAFCAA|1934-10-28|417|1817|140|1934|0|10|28|4|1934|140|1817|Sunday|1934Q4|N|N|N|2427712|2427984|2427374|2427647|N|N|N|N|N| +2427740|AAAAAAAAMFLAFCAA|1934-10-29|417|1817|140|1934|1|10|29|4|1934|140|1817|Monday|1934Q4|N|N|N|2427712|2427984|2427375|2427648|N|N|N|N|N| +2427741|AAAAAAAANFLAFCAA|1934-10-30|417|1818|140|1934|2|10|30|4|1934|140|1818|Tuesday|1934Q4|N|N|N|2427712|2427984|2427376|2427649|N|N|N|N|N| +2427742|AAAAAAAAOFLAFCAA|1934-10-31|417|1818|140|1934|3|10|31|4|1934|140|1818|Wednesday|1934Q4|N|N|N|2427712|2427984|2427377|2427650|N|N|N|N|N| +2427743|AAAAAAAAPFLAFCAA|1934-11-01|418|1818|140|1934|4|11|1|4|1934|140|1818|Thursday|1934Q4|N|N|N|2427743|2428046|2427378|2427651|N|N|N|N|N| +2427744|AAAAAAAAAGLAFCAA|1934-11-02|418|1818|140|1934|5|11|2|4|1934|140|1818|Friday|1934Q4|N|Y|N|2427743|2428046|2427379|2427652|N|N|N|N|N| +2427745|AAAAAAAABGLAFCAA|1934-11-03|418|1818|140|1934|6|11|3|4|1934|140|1818|Saturday|1934Q4|N|Y|N|2427743|2428046|2427380|2427653|N|N|N|N|N| +2427746|AAAAAAAACGLAFCAA|1934-11-04|418|1818|140|1934|0|11|4|4|1934|140|1818|Sunday|1934Q4|N|N|N|2427743|2428046|2427381|2427654|N|N|N|N|N| +2427747|AAAAAAAADGLAFCAA|1934-11-05|418|1818|140|1934|1|11|5|4|1934|140|1818|Monday|1934Q4|N|N|N|2427743|2428046|2427382|2427655|N|N|N|N|N| +2427748|AAAAAAAAEGLAFCAA|1934-11-06|418|1819|140|1934|2|11|6|4|1934|140|1819|Tuesday|1934Q4|N|N|N|2427743|2428046|2427383|2427656|N|N|N|N|N| +2427749|AAAAAAAAFGLAFCAA|1934-11-07|418|1819|140|1934|3|11|7|4|1934|140|1819|Wednesday|1934Q4|N|N|N|2427743|2428046|2427384|2427657|N|N|N|N|N| +2427750|AAAAAAAAGGLAFCAA|1934-11-08|418|1819|140|1934|4|11|8|4|1934|140|1819|Thursday|1934Q4|N|N|N|2427743|2428046|2427385|2427658|N|N|N|N|N| +2427751|AAAAAAAAHGLAFCAA|1934-11-09|418|1819|140|1934|5|11|9|4|1934|140|1819|Friday|1934Q4|N|Y|N|2427743|2428046|2427386|2427659|N|N|N|N|N| +2427752|AAAAAAAAIGLAFCAA|1934-11-10|418|1819|140|1934|6|11|10|4|1934|140|1819|Saturday|1934Q4|N|Y|N|2427743|2428046|2427387|2427660|N|N|N|N|N| +2427753|AAAAAAAAJGLAFCAA|1934-11-11|418|1819|140|1934|0|11|11|4|1934|140|1819|Sunday|1934Q4|N|N|N|2427743|2428046|2427388|2427661|N|N|N|N|N| +2427754|AAAAAAAAKGLAFCAA|1934-11-12|418|1819|140|1934|1|11|12|4|1934|140|1819|Monday|1934Q4|N|N|N|2427743|2428046|2427389|2427662|N|N|N|N|N| +2427755|AAAAAAAALGLAFCAA|1934-11-13|418|1820|140|1934|2|11|13|4|1934|140|1820|Tuesday|1934Q4|N|N|N|2427743|2428046|2427390|2427663|N|N|N|N|N| +2427756|AAAAAAAAMGLAFCAA|1934-11-14|418|1820|140|1934|3|11|14|4|1934|140|1820|Wednesday|1934Q4|N|N|N|2427743|2428046|2427391|2427664|N|N|N|N|N| +2427757|AAAAAAAANGLAFCAA|1934-11-15|418|1820|140|1934|4|11|15|4|1934|140|1820|Thursday|1934Q4|N|N|N|2427743|2428046|2427392|2427665|N|N|N|N|N| +2427758|AAAAAAAAOGLAFCAA|1934-11-16|418|1820|140|1934|5|11|16|4|1934|140|1820|Friday|1934Q4|N|Y|N|2427743|2428046|2427393|2427666|N|N|N|N|N| +2427759|AAAAAAAAPGLAFCAA|1934-11-17|418|1820|140|1934|6|11|17|4|1934|140|1820|Saturday|1934Q4|N|Y|N|2427743|2428046|2427394|2427667|N|N|N|N|N| +2427760|AAAAAAAAAHLAFCAA|1934-11-18|418|1820|140|1934|0|11|18|4|1934|140|1820|Sunday|1934Q4|N|N|N|2427743|2428046|2427395|2427668|N|N|N|N|N| +2427761|AAAAAAAABHLAFCAA|1934-11-19|418|1820|140|1934|1|11|19|4|1934|140|1820|Monday|1934Q4|N|N|N|2427743|2428046|2427396|2427669|N|N|N|N|N| +2427762|AAAAAAAACHLAFCAA|1934-11-20|418|1821|140|1934|2|11|20|4|1934|140|1821|Tuesday|1934Q4|N|N|N|2427743|2428046|2427397|2427670|N|N|N|N|N| +2427763|AAAAAAAADHLAFCAA|1934-11-21|418|1821|140|1934|3|11|21|4|1934|140|1821|Wednesday|1934Q4|N|N|N|2427743|2428046|2427398|2427671|N|N|N|N|N| +2427764|AAAAAAAAEHLAFCAA|1934-11-22|418|1821|140|1934|4|11|22|4|1934|140|1821|Thursday|1934Q4|N|N|N|2427743|2428046|2427399|2427672|N|N|N|N|N| +2427765|AAAAAAAAFHLAFCAA|1934-11-23|418|1821|140|1934|5|11|23|4|1934|140|1821|Friday|1934Q4|N|Y|N|2427743|2428046|2427400|2427673|N|N|N|N|N| +2427766|AAAAAAAAGHLAFCAA|1934-11-24|418|1821|140|1934|6|11|24|4|1934|140|1821|Saturday|1934Q4|N|Y|N|2427743|2428046|2427401|2427674|N|N|N|N|N| +2427767|AAAAAAAAHHLAFCAA|1934-11-25|418|1821|140|1934|0|11|25|4|1934|140|1821|Sunday|1934Q4|N|N|N|2427743|2428046|2427402|2427675|N|N|N|N|N| +2427768|AAAAAAAAIHLAFCAA|1934-11-26|418|1821|140|1934|1|11|26|4|1934|140|1821|Monday|1934Q4|N|N|N|2427743|2428046|2427403|2427676|N|N|N|N|N| +2427769|AAAAAAAAJHLAFCAA|1934-11-27|418|1822|140|1934|2|11|27|4|1934|140|1822|Tuesday|1934Q4|N|N|N|2427743|2428046|2427404|2427677|N|N|N|N|N| +2427770|AAAAAAAAKHLAFCAA|1934-11-28|418|1822|140|1934|3|11|28|4|1934|140|1822|Wednesday|1934Q4|N|N|N|2427743|2428046|2427405|2427678|N|N|N|N|N| +2427771|AAAAAAAALHLAFCAA|1934-11-29|418|1822|140|1934|4|11|29|4|1934|140|1822|Thursday|1934Q4|N|N|N|2427743|2428046|2427406|2427679|N|N|N|N|N| +2427772|AAAAAAAAMHLAFCAA|1934-11-30|418|1822|140|1934|5|11|30|4|1934|140|1822|Friday|1934Q4|N|Y|N|2427743|2428046|2427407|2427680|N|N|N|N|N| +2427773|AAAAAAAANHLAFCAA|1934-12-01|419|1822|141|1934|6|12|1|4|1934|141|1822|Saturday|1934Q4|N|Y|N|2427773|2428106|2427408|2427681|N|N|N|N|N| +2427774|AAAAAAAAOHLAFCAA|1934-12-02|419|1822|141|1934|0|12|2|4|1934|141|1822|Sunday|1934Q4|N|N|N|2427773|2428106|2427409|2427682|N|N|N|N|N| +2427775|AAAAAAAAPHLAFCAA|1934-12-03|419|1822|141|1934|1|12|3|4|1934|141|1822|Monday|1934Q4|N|N|N|2427773|2428106|2427410|2427683|N|N|N|N|N| +2427776|AAAAAAAAAILAFCAA|1934-12-04|419|1823|141|1934|2|12|4|4|1934|141|1823|Tuesday|1934Q4|N|N|N|2427773|2428106|2427411|2427684|N|N|N|N|N| +2427777|AAAAAAAABILAFCAA|1934-12-05|419|1823|141|1934|3|12|5|4|1934|141|1823|Wednesday|1934Q4|N|N|N|2427773|2428106|2427412|2427685|N|N|N|N|N| +2427778|AAAAAAAACILAFCAA|1934-12-06|419|1823|141|1934|4|12|6|4|1934|141|1823|Thursday|1934Q4|N|N|N|2427773|2428106|2427413|2427686|N|N|N|N|N| +2427779|AAAAAAAADILAFCAA|1934-12-07|419|1823|141|1934|5|12|7|4|1934|141|1823|Friday|1934Q4|N|Y|N|2427773|2428106|2427414|2427687|N|N|N|N|N| +2427780|AAAAAAAAEILAFCAA|1934-12-08|419|1823|141|1934|6|12|8|4|1934|141|1823|Saturday|1934Q4|N|Y|N|2427773|2428106|2427415|2427688|N|N|N|N|N| +2427781|AAAAAAAAFILAFCAA|1934-12-09|419|1823|141|1934|0|12|9|4|1934|141|1823|Sunday|1934Q4|N|N|N|2427773|2428106|2427416|2427689|N|N|N|N|N| +2427782|AAAAAAAAGILAFCAA|1934-12-10|419|1823|141|1934|1|12|10|4|1934|141|1823|Monday|1934Q4|N|N|N|2427773|2428106|2427417|2427690|N|N|N|N|N| +2427783|AAAAAAAAHILAFCAA|1934-12-11|419|1824|141|1934|2|12|11|4|1934|141|1824|Tuesday|1934Q4|N|N|N|2427773|2428106|2427418|2427691|N|N|N|N|N| +2427784|AAAAAAAAIILAFCAA|1934-12-12|419|1824|141|1934|3|12|12|4|1934|141|1824|Wednesday|1934Q4|N|N|N|2427773|2428106|2427419|2427692|N|N|N|N|N| +2427785|AAAAAAAAJILAFCAA|1934-12-13|419|1824|141|1934|4|12|13|4|1934|141|1824|Thursday|1934Q4|N|N|N|2427773|2428106|2427420|2427693|N|N|N|N|N| +2427786|AAAAAAAAKILAFCAA|1934-12-14|419|1824|141|1934|5|12|14|4|1934|141|1824|Friday|1934Q4|N|Y|N|2427773|2428106|2427421|2427694|N|N|N|N|N| +2427787|AAAAAAAALILAFCAA|1934-12-15|419|1824|141|1934|6|12|15|4|1934|141|1824|Saturday|1934Q4|N|Y|N|2427773|2428106|2427422|2427695|N|N|N|N|N| +2427788|AAAAAAAAMILAFCAA|1934-12-16|419|1824|141|1934|0|12|16|4|1934|141|1824|Sunday|1934Q4|N|N|N|2427773|2428106|2427423|2427696|N|N|N|N|N| +2427789|AAAAAAAANILAFCAA|1934-12-17|419|1824|141|1934|1|12|17|4|1934|141|1824|Monday|1934Q4|N|N|N|2427773|2428106|2427424|2427697|N|N|N|N|N| +2427790|AAAAAAAAOILAFCAA|1934-12-18|419|1825|141|1934|2|12|18|4|1934|141|1825|Tuesday|1934Q4|N|N|N|2427773|2428106|2427425|2427698|N|N|N|N|N| +2427791|AAAAAAAAPILAFCAA|1934-12-19|419|1825|141|1934|3|12|19|4|1934|141|1825|Wednesday|1934Q4|N|N|N|2427773|2428106|2427426|2427699|N|N|N|N|N| +2427792|AAAAAAAAAJLAFCAA|1934-12-20|419|1825|141|1934|4|12|20|4|1934|141|1825|Thursday|1934Q4|N|N|N|2427773|2428106|2427427|2427700|N|N|N|N|N| +2427793|AAAAAAAABJLAFCAA|1934-12-21|419|1825|141|1934|5|12|21|4|1934|141|1825|Friday|1934Q4|N|Y|N|2427773|2428106|2427428|2427701|N|N|N|N|N| +2427794|AAAAAAAACJLAFCAA|1934-12-22|419|1825|141|1934|6|12|22|4|1934|141|1825|Saturday|1934Q4|N|Y|N|2427773|2428106|2427429|2427702|N|N|N|N|N| +2427795|AAAAAAAADJLAFCAA|1934-12-23|419|1825|141|1934|0|12|23|4|1934|141|1825|Sunday|1934Q4|N|N|N|2427773|2428106|2427430|2427703|N|N|N|N|N| +2427796|AAAAAAAAEJLAFCAA|1934-12-24|419|1825|141|1934|1|12|24|4|1934|141|1825|Monday|1934Q4|N|N|N|2427773|2428106|2427431|2427704|N|N|N|N|N| +2427797|AAAAAAAAFJLAFCAA|1934-12-25|419|1826|141|1934|2|12|25|4|1934|141|1826|Tuesday|1934Q4|N|N|N|2427773|2428106|2427432|2427705|N|N|N|N|N| +2427798|AAAAAAAAGJLAFCAA|1934-12-26|419|1826|141|1934|3|12|26|4|1934|141|1826|Wednesday|1934Q4|Y|N|N|2427773|2428106|2427433|2427706|N|N|N|N|N| +2427799|AAAAAAAAHJLAFCAA|1934-12-27|419|1826|141|1934|4|12|27|4|1934|141|1826|Thursday|1934Q4|N|N|Y|2427773|2428106|2427434|2427707|N|N|N|N|N| +2427800|AAAAAAAAIJLAFCAA|1934-12-28|419|1826|141|1934|5|12|28|4|1934|141|1826|Friday|1934Q4|N|Y|N|2427773|2428106|2427435|2427708|N|N|N|N|N| +2427801|AAAAAAAAJJLAFCAA|1934-12-29|419|1826|141|1934|6|12|29|4|1934|141|1826|Saturday|1934Q4|N|Y|N|2427773|2428106|2427436|2427709|N|N|N|N|N| +2427802|AAAAAAAAKJLAFCAA|1934-12-30|419|1826|141|1934|0|12|30|4|1934|141|1826|Sunday|1934Q4|N|N|N|2427773|2428106|2427437|2427710|N|N|N|N|N| +2427803|AAAAAAAALJLAFCAA|1934-12-31|419|1826|141|1934|1|12|31|4|1934|141|1826|Monday|1934Q4|N|N|N|2427773|2428106|2427438|2427711|N|N|N|N|N| +2427804|AAAAAAAAMJLAFCAA|1935-01-01|420|1827|141|1935|2|1|1|1|1935|141|1827|Tuesday|1935Q1|Y|N|N|2427804|2427803|2427439|2427712|N|N|N|N|N| +2427805|AAAAAAAANJLAFCAA|1935-01-02|420|1827|141|1935|3|1|2|1|1935|141|1827|Wednesday|1935Q1|N|N|Y|2427804|2427803|2427440|2427713|N|N|N|N|N| +2427806|AAAAAAAAOJLAFCAA|1935-01-03|420|1827|141|1935|4|1|3|1|1935|141|1827|Thursday|1935Q1|N|N|N|2427804|2427803|2427441|2427714|N|N|N|N|N| +2427807|AAAAAAAAPJLAFCAA|1935-01-04|420|1827|141|1935|5|1|4|1|1935|141|1827|Friday|1935Q1|N|Y|N|2427804|2427803|2427442|2427715|N|N|N|N|N| +2427808|AAAAAAAAAKLAFCAA|1935-01-05|420|1827|141|1935|6|1|5|1|1935|141|1827|Saturday|1935Q1|N|Y|N|2427804|2427803|2427443|2427716|N|N|N|N|N| +2427809|AAAAAAAABKLAFCAA|1935-01-06|420|1827|141|1935|0|1|6|1|1935|141|1827|Sunday|1935Q1|N|N|N|2427804|2427803|2427444|2427717|N|N|N|N|N| +2427810|AAAAAAAACKLAFCAA|1935-01-07|420|1827|141|1935|1|1|7|1|1935|141|1827|Monday|1935Q1|N|N|N|2427804|2427803|2427445|2427718|N|N|N|N|N| +2427811|AAAAAAAADKLAFCAA|1935-01-08|420|1828|141|1935|2|1|8|1|1935|141|1828|Tuesday|1935Q1|N|N|N|2427804|2427803|2427446|2427719|N|N|N|N|N| +2427812|AAAAAAAAEKLAFCAA|1935-01-09|420|1828|141|1935|3|1|9|1|1935|141|1828|Wednesday|1935Q1|N|N|N|2427804|2427803|2427447|2427720|N|N|N|N|N| +2427813|AAAAAAAAFKLAFCAA|1935-01-10|420|1828|141|1935|4|1|10|1|1935|141|1828|Thursday|1935Q1|N|N|N|2427804|2427803|2427448|2427721|N|N|N|N|N| +2427814|AAAAAAAAGKLAFCAA|1935-01-11|420|1828|141|1935|5|1|11|1|1935|141|1828|Friday|1935Q1|N|Y|N|2427804|2427803|2427449|2427722|N|N|N|N|N| +2427815|AAAAAAAAHKLAFCAA|1935-01-12|420|1828|141|1935|6|1|12|1|1935|141|1828|Saturday|1935Q1|N|Y|N|2427804|2427803|2427450|2427723|N|N|N|N|N| +2427816|AAAAAAAAIKLAFCAA|1935-01-13|420|1828|141|1935|0|1|13|1|1935|141|1828|Sunday|1935Q1|N|N|N|2427804|2427803|2427451|2427724|N|N|N|N|N| +2427817|AAAAAAAAJKLAFCAA|1935-01-14|420|1828|141|1935|1|1|14|1|1935|141|1828|Monday|1935Q1|N|N|N|2427804|2427803|2427452|2427725|N|N|N|N|N| +2427818|AAAAAAAAKKLAFCAA|1935-01-15|420|1829|141|1935|2|1|15|1|1935|141|1829|Tuesday|1935Q1|N|N|N|2427804|2427803|2427453|2427726|N|N|N|N|N| +2427819|AAAAAAAALKLAFCAA|1935-01-16|420|1829|141|1935|3|1|16|1|1935|141|1829|Wednesday|1935Q1|N|N|N|2427804|2427803|2427454|2427727|N|N|N|N|N| +2427820|AAAAAAAAMKLAFCAA|1935-01-17|420|1829|141|1935|4|1|17|1|1935|141|1829|Thursday|1935Q1|N|N|N|2427804|2427803|2427455|2427728|N|N|N|N|N| +2427821|AAAAAAAANKLAFCAA|1935-01-18|420|1829|141|1935|5|1|18|1|1935|141|1829|Friday|1935Q1|N|Y|N|2427804|2427803|2427456|2427729|N|N|N|N|N| +2427822|AAAAAAAAOKLAFCAA|1935-01-19|420|1829|141|1935|6|1|19|1|1935|141|1829|Saturday|1935Q1|N|Y|N|2427804|2427803|2427457|2427730|N|N|N|N|N| +2427823|AAAAAAAAPKLAFCAA|1935-01-20|420|1829|141|1935|0|1|20|1|1935|141|1829|Sunday|1935Q1|N|N|N|2427804|2427803|2427458|2427731|N|N|N|N|N| +2427824|AAAAAAAAALLAFCAA|1935-01-21|420|1829|141|1935|1|1|21|1|1935|141|1829|Monday|1935Q1|N|N|N|2427804|2427803|2427459|2427732|N|N|N|N|N| +2427825|AAAAAAAABLLAFCAA|1935-01-22|420|1830|141|1935|2|1|22|1|1935|141|1830|Tuesday|1935Q1|N|N|N|2427804|2427803|2427460|2427733|N|N|N|N|N| +2427826|AAAAAAAACLLAFCAA|1935-01-23|420|1830|141|1935|3|1|23|1|1935|141|1830|Wednesday|1935Q1|N|N|N|2427804|2427803|2427461|2427734|N|N|N|N|N| +2427827|AAAAAAAADLLAFCAA|1935-01-24|420|1830|141|1935|4|1|24|1|1935|141|1830|Thursday|1935Q1|N|N|N|2427804|2427803|2427462|2427735|N|N|N|N|N| +2427828|AAAAAAAAELLAFCAA|1935-01-25|420|1830|141|1935|5|1|25|1|1935|141|1830|Friday|1935Q1|N|Y|N|2427804|2427803|2427463|2427736|N|N|N|N|N| +2427829|AAAAAAAAFLLAFCAA|1935-01-26|420|1830|141|1935|6|1|26|1|1935|141|1830|Saturday|1935Q1|N|Y|N|2427804|2427803|2427464|2427737|N|N|N|N|N| +2427830|AAAAAAAAGLLAFCAA|1935-01-27|420|1830|141|1935|0|1|27|1|1935|141|1830|Sunday|1935Q1|N|N|N|2427804|2427803|2427465|2427738|N|N|N|N|N| +2427831|AAAAAAAAHLLAFCAA|1935-01-28|420|1830|141|1935|1|1|28|1|1935|141|1830|Monday|1935Q1|N|N|N|2427804|2427803|2427466|2427739|N|N|N|N|N| +2427832|AAAAAAAAILLAFCAA|1935-01-29|420|1831|141|1935|2|1|29|1|1935|141|1831|Tuesday|1935Q1|N|N|N|2427804|2427803|2427467|2427740|N|N|N|N|N| +2427833|AAAAAAAAJLLAFCAA|1935-01-30|420|1831|141|1935|3|1|30|1|1935|141|1831|Wednesday|1935Q1|N|N|N|2427804|2427803|2427468|2427741|N|N|N|N|N| +2427834|AAAAAAAAKLLAFCAA|1935-01-31|420|1831|141|1935|4|1|31|1|1935|141|1831|Thursday|1935Q1|N|N|N|2427804|2427803|2427469|2427742|N|N|N|N|N| +2427835|AAAAAAAALLLAFCAA|1935-02-01|421|1831|141|1935|5|2|1|1|1935|141|1831|Friday|1935Q1|N|Y|N|2427835|2427865|2427470|2427743|N|N|N|N|N| +2427836|AAAAAAAAMLLAFCAA|1935-02-02|421|1831|141|1935|6|2|2|1|1935|141|1831|Saturday|1935Q1|N|Y|N|2427835|2427865|2427471|2427744|N|N|N|N|N| +2427837|AAAAAAAANLLAFCAA|1935-02-03|421|1831|141|1935|0|2|3|1|1935|141|1831|Sunday|1935Q1|N|N|N|2427835|2427865|2427472|2427745|N|N|N|N|N| +2427838|AAAAAAAAOLLAFCAA|1935-02-04|421|1831|141|1935|1|2|4|1|1935|141|1831|Monday|1935Q1|N|N|N|2427835|2427865|2427473|2427746|N|N|N|N|N| +2427839|AAAAAAAAPLLAFCAA|1935-02-05|421|1832|141|1935|2|2|5|1|1935|141|1832|Tuesday|1935Q1|N|N|N|2427835|2427865|2427474|2427747|N|N|N|N|N| +2427840|AAAAAAAAAMLAFCAA|1935-02-06|421|1832|141|1935|3|2|6|1|1935|141|1832|Wednesday|1935Q1|N|N|N|2427835|2427865|2427475|2427748|N|N|N|N|N| +2427841|AAAAAAAABMLAFCAA|1935-02-07|421|1832|141|1935|4|2|7|1|1935|141|1832|Thursday|1935Q1|N|N|N|2427835|2427865|2427476|2427749|N|N|N|N|N| +2427842|AAAAAAAACMLAFCAA|1935-02-08|421|1832|141|1935|5|2|8|1|1935|141|1832|Friday|1935Q1|N|Y|N|2427835|2427865|2427477|2427750|N|N|N|N|N| +2427843|AAAAAAAADMLAFCAA|1935-02-09|421|1832|141|1935|6|2|9|1|1935|141|1832|Saturday|1935Q1|N|Y|N|2427835|2427865|2427478|2427751|N|N|N|N|N| +2427844|AAAAAAAAEMLAFCAA|1935-02-10|421|1832|141|1935|0|2|10|1|1935|141|1832|Sunday|1935Q1|N|N|N|2427835|2427865|2427479|2427752|N|N|N|N|N| +2427845|AAAAAAAAFMLAFCAA|1935-02-11|421|1832|141|1935|1|2|11|1|1935|141|1832|Monday|1935Q1|N|N|N|2427835|2427865|2427480|2427753|N|N|N|N|N| +2427846|AAAAAAAAGMLAFCAA|1935-02-12|421|1833|141|1935|2|2|12|1|1935|141|1833|Tuesday|1935Q1|N|N|N|2427835|2427865|2427481|2427754|N|N|N|N|N| +2427847|AAAAAAAAHMLAFCAA|1935-02-13|421|1833|141|1935|3|2|13|1|1935|141|1833|Wednesday|1935Q1|N|N|N|2427835|2427865|2427482|2427755|N|N|N|N|N| +2427848|AAAAAAAAIMLAFCAA|1935-02-14|421|1833|141|1935|4|2|14|1|1935|141|1833|Thursday|1935Q1|N|N|N|2427835|2427865|2427483|2427756|N|N|N|N|N| +2427849|AAAAAAAAJMLAFCAA|1935-02-15|421|1833|141|1935|5|2|15|1|1935|141|1833|Friday|1935Q1|N|Y|N|2427835|2427865|2427484|2427757|N|N|N|N|N| +2427850|AAAAAAAAKMLAFCAA|1935-02-16|421|1833|141|1935|6|2|16|1|1935|141|1833|Saturday|1935Q1|N|Y|N|2427835|2427865|2427485|2427758|N|N|N|N|N| +2427851|AAAAAAAALMLAFCAA|1935-02-17|421|1833|141|1935|0|2|17|1|1935|141|1833|Sunday|1935Q1|N|N|N|2427835|2427865|2427486|2427759|N|N|N|N|N| +2427852|AAAAAAAAMMLAFCAA|1935-02-18|421|1833|141|1935|1|2|18|1|1935|141|1833|Monday|1935Q1|N|N|N|2427835|2427865|2427487|2427760|N|N|N|N|N| +2427853|AAAAAAAANMLAFCAA|1935-02-19|421|1834|141|1935|2|2|19|1|1935|141|1834|Tuesday|1935Q1|N|N|N|2427835|2427865|2427488|2427761|N|N|N|N|N| +2427854|AAAAAAAAOMLAFCAA|1935-02-20|421|1834|141|1935|3|2|20|1|1935|141|1834|Wednesday|1935Q1|N|N|N|2427835|2427865|2427489|2427762|N|N|N|N|N| +2427855|AAAAAAAAPMLAFCAA|1935-02-21|421|1834|141|1935|4|2|21|1|1935|141|1834|Thursday|1935Q1|N|N|N|2427835|2427865|2427490|2427763|N|N|N|N|N| +2427856|AAAAAAAAANLAFCAA|1935-02-22|421|1834|141|1935|5|2|22|1|1935|141|1834|Friday|1935Q1|N|Y|N|2427835|2427865|2427491|2427764|N|N|N|N|N| +2427857|AAAAAAAABNLAFCAA|1935-02-23|421|1834|141|1935|6|2|23|1|1935|141|1834|Saturday|1935Q1|N|Y|N|2427835|2427865|2427492|2427765|N|N|N|N|N| +2427858|AAAAAAAACNLAFCAA|1935-02-24|421|1834|141|1935|0|2|24|1|1935|141|1834|Sunday|1935Q1|N|N|N|2427835|2427865|2427493|2427766|N|N|N|N|N| +2427859|AAAAAAAADNLAFCAA|1935-02-25|421|1834|141|1935|1|2|25|1|1935|141|1834|Monday|1935Q1|N|N|N|2427835|2427865|2427494|2427767|N|N|N|N|N| +2427860|AAAAAAAAENLAFCAA|1935-02-26|421|1835|141|1935|2|2|26|1|1935|141|1835|Tuesday|1935Q1|N|N|N|2427835|2427865|2427495|2427768|N|N|N|N|N| +2427861|AAAAAAAAFNLAFCAA|1935-02-27|421|1835|141|1935|3|2|27|1|1935|141|1835|Wednesday|1935Q1|N|N|N|2427835|2427865|2427496|2427769|N|N|N|N|N| +2427862|AAAAAAAAGNLAFCAA|1935-02-28|421|1835|141|1935|4|2|28|1|1935|141|1835|Thursday|1935Q1|N|N|N|2427835|2427865|2427497|2427770|N|N|N|N|N| +2427863|AAAAAAAAHNLAFCAA|1935-03-01|422|1835|142|1935|5|3|1|1|1935|142|1835|Friday|1935Q1|N|Y|N|2427863|2427921|2427498|2427771|N|N|N|N|N| +2427864|AAAAAAAAINLAFCAA|1935-03-02|422|1835|142|1935|6|3|2|1|1935|142|1835|Saturday|1935Q1|N|Y|N|2427863|2427921|2427499|2427772|N|N|N|N|N| +2427865|AAAAAAAAJNLAFCAA|1935-03-03|422|1835|142|1935|0|3|3|1|1935|142|1835|Sunday|1935Q1|N|N|N|2427863|2427921|2427500|2427773|N|N|N|N|N| +2427866|AAAAAAAAKNLAFCAA|1935-03-04|422|1835|142|1935|1|3|4|1|1935|142|1835|Monday|1935Q1|N|N|N|2427863|2427921|2427501|2427774|N|N|N|N|N| +2427867|AAAAAAAALNLAFCAA|1935-03-05|422|1836|142|1935|2|3|5|1|1935|142|1836|Tuesday|1935Q1|N|N|N|2427863|2427921|2427502|2427775|N|N|N|N|N| +2427868|AAAAAAAAMNLAFCAA|1935-03-06|422|1836|142|1935|3|3|6|1|1935|142|1836|Wednesday|1935Q1|N|N|N|2427863|2427921|2427503|2427776|N|N|N|N|N| +2427869|AAAAAAAANNLAFCAA|1935-03-07|422|1836|142|1935|4|3|7|1|1935|142|1836|Thursday|1935Q1|N|N|N|2427863|2427921|2427504|2427777|N|N|N|N|N| +2427870|AAAAAAAAONLAFCAA|1935-03-08|422|1836|142|1935|5|3|8|1|1935|142|1836|Friday|1935Q1|N|Y|N|2427863|2427921|2427505|2427778|N|N|N|N|N| +2427871|AAAAAAAAPNLAFCAA|1935-03-09|422|1836|142|1935|6|3|9|1|1935|142|1836|Saturday|1935Q1|N|Y|N|2427863|2427921|2427506|2427779|N|N|N|N|N| +2427872|AAAAAAAAAOLAFCAA|1935-03-10|422|1836|142|1935|0|3|10|1|1935|142|1836|Sunday|1935Q1|N|N|N|2427863|2427921|2427507|2427780|N|N|N|N|N| +2427873|AAAAAAAABOLAFCAA|1935-03-11|422|1836|142|1935|1|3|11|1|1935|142|1836|Monday|1935Q1|N|N|N|2427863|2427921|2427508|2427781|N|N|N|N|N| +2427874|AAAAAAAACOLAFCAA|1935-03-12|422|1837|142|1935|2|3|12|1|1935|142|1837|Tuesday|1935Q1|N|N|N|2427863|2427921|2427509|2427782|N|N|N|N|N| +2427875|AAAAAAAADOLAFCAA|1935-03-13|422|1837|142|1935|3|3|13|1|1935|142|1837|Wednesday|1935Q1|N|N|N|2427863|2427921|2427510|2427783|N|N|N|N|N| +2427876|AAAAAAAAEOLAFCAA|1935-03-14|422|1837|142|1935|4|3|14|1|1935|142|1837|Thursday|1935Q1|N|N|N|2427863|2427921|2427511|2427784|N|N|N|N|N| +2427877|AAAAAAAAFOLAFCAA|1935-03-15|422|1837|142|1935|5|3|15|1|1935|142|1837|Friday|1935Q1|N|Y|N|2427863|2427921|2427512|2427785|N|N|N|N|N| +2427878|AAAAAAAAGOLAFCAA|1935-03-16|422|1837|142|1935|6|3|16|1|1935|142|1837|Saturday|1935Q1|N|Y|N|2427863|2427921|2427513|2427786|N|N|N|N|N| +2427879|AAAAAAAAHOLAFCAA|1935-03-17|422|1837|142|1935|0|3|17|1|1935|142|1837|Sunday|1935Q1|N|N|N|2427863|2427921|2427514|2427787|N|N|N|N|N| +2427880|AAAAAAAAIOLAFCAA|1935-03-18|422|1837|142|1935|1|3|18|1|1935|142|1837|Monday|1935Q1|N|N|N|2427863|2427921|2427515|2427788|N|N|N|N|N| +2427881|AAAAAAAAJOLAFCAA|1935-03-19|422|1838|142|1935|2|3|19|1|1935|142|1838|Tuesday|1935Q1|N|N|N|2427863|2427921|2427516|2427789|N|N|N|N|N| +2427882|AAAAAAAAKOLAFCAA|1935-03-20|422|1838|142|1935|3|3|20|1|1935|142|1838|Wednesday|1935Q1|N|N|N|2427863|2427921|2427517|2427790|N|N|N|N|N| +2427883|AAAAAAAALOLAFCAA|1935-03-21|422|1838|142|1935|4|3|21|1|1935|142|1838|Thursday|1935Q1|N|N|N|2427863|2427921|2427518|2427791|N|N|N|N|N| +2427884|AAAAAAAAMOLAFCAA|1935-03-22|422|1838|142|1935|5|3|22|1|1935|142|1838|Friday|1935Q1|N|Y|N|2427863|2427921|2427519|2427792|N|N|N|N|N| +2427885|AAAAAAAANOLAFCAA|1935-03-23|422|1838|142|1935|6|3|23|1|1935|142|1838|Saturday|1935Q1|N|Y|N|2427863|2427921|2427520|2427793|N|N|N|N|N| +2427886|AAAAAAAAOOLAFCAA|1935-03-24|422|1838|142|1935|0|3|24|1|1935|142|1838|Sunday|1935Q1|N|N|N|2427863|2427921|2427521|2427794|N|N|N|N|N| +2427887|AAAAAAAAPOLAFCAA|1935-03-25|422|1838|142|1935|1|3|25|1|1935|142|1838|Monday|1935Q1|N|N|N|2427863|2427921|2427522|2427795|N|N|N|N|N| +2427888|AAAAAAAAAPLAFCAA|1935-03-26|422|1839|142|1935|2|3|26|1|1935|142|1839|Tuesday|1935Q1|N|N|N|2427863|2427921|2427523|2427796|N|N|N|N|N| +2427889|AAAAAAAABPLAFCAA|1935-03-27|422|1839|142|1935|3|3|27|1|1935|142|1839|Wednesday|1935Q1|N|N|N|2427863|2427921|2427524|2427797|N|N|N|N|N| +2427890|AAAAAAAACPLAFCAA|1935-03-28|422|1839|142|1935|4|3|28|1|1935|142|1839|Thursday|1935Q1|N|N|N|2427863|2427921|2427525|2427798|N|N|N|N|N| +2427891|AAAAAAAADPLAFCAA|1935-03-29|422|1839|142|1935|5|3|29|1|1935|142|1839|Friday|1935Q1|N|Y|N|2427863|2427921|2427526|2427799|N|N|N|N|N| +2427892|AAAAAAAAEPLAFCAA|1935-03-30|422|1839|142|1935|6|3|30|1|1935|142|1839|Saturday|1935Q1|N|Y|N|2427863|2427921|2427527|2427800|N|N|N|N|N| +2427893|AAAAAAAAFPLAFCAA|1935-03-31|422|1839|142|1935|0|3|31|1|1935|142|1839|Sunday|1935Q1|N|N|N|2427863|2427921|2427528|2427801|N|N|N|N|N| +2427894|AAAAAAAAGPLAFCAA|1935-04-01|423|1839|142|1935|1|4|1|1|1935|142|1839|Monday|1935Q1|N|N|N|2427894|2427983|2427529|2427804|N|N|N|N|N| +2427895|AAAAAAAAHPLAFCAA|1935-04-02|423|1840|142|1935|2|4|2|2|1935|142|1840|Tuesday|1935Q2|N|N|N|2427894|2427983|2427530|2427805|N|N|N|N|N| +2427896|AAAAAAAAIPLAFCAA|1935-04-03|423|1840|142|1935|3|4|3|2|1935|142|1840|Wednesday|1935Q2|N|N|N|2427894|2427983|2427531|2427806|N|N|N|N|N| +2427897|AAAAAAAAJPLAFCAA|1935-04-04|423|1840|142|1935|4|4|4|2|1935|142|1840|Thursday|1935Q2|N|N|N|2427894|2427983|2427532|2427807|N|N|N|N|N| +2427898|AAAAAAAAKPLAFCAA|1935-04-05|423|1840|142|1935|5|4|5|2|1935|142|1840|Friday|1935Q2|N|Y|N|2427894|2427983|2427533|2427808|N|N|N|N|N| +2427899|AAAAAAAALPLAFCAA|1935-04-06|423|1840|142|1935|6|4|6|2|1935|142|1840|Saturday|1935Q2|N|Y|N|2427894|2427983|2427534|2427809|N|N|N|N|N| +2427900|AAAAAAAAMPLAFCAA|1935-04-07|423|1840|142|1935|0|4|7|2|1935|142|1840|Sunday|1935Q2|N|N|N|2427894|2427983|2427535|2427810|N|N|N|N|N| +2427901|AAAAAAAANPLAFCAA|1935-04-08|423|1840|142|1935|1|4|8|2|1935|142|1840|Monday|1935Q2|N|N|N|2427894|2427983|2427536|2427811|N|N|N|N|N| +2427902|AAAAAAAAOPLAFCAA|1935-04-09|423|1841|142|1935|2|4|9|2|1935|142|1841|Tuesday|1935Q2|N|N|N|2427894|2427983|2427537|2427812|N|N|N|N|N| +2427903|AAAAAAAAPPLAFCAA|1935-04-10|423|1841|142|1935|3|4|10|2|1935|142|1841|Wednesday|1935Q2|N|N|N|2427894|2427983|2427538|2427813|N|N|N|N|N| +2427904|AAAAAAAAAAMAFCAA|1935-04-11|423|1841|142|1935|4|4|11|2|1935|142|1841|Thursday|1935Q2|N|N|N|2427894|2427983|2427539|2427814|N|N|N|N|N| +2427905|AAAAAAAABAMAFCAA|1935-04-12|423|1841|142|1935|5|4|12|2|1935|142|1841|Friday|1935Q2|N|Y|N|2427894|2427983|2427540|2427815|N|N|N|N|N| +2427906|AAAAAAAACAMAFCAA|1935-04-13|423|1841|142|1935|6|4|13|2|1935|142|1841|Saturday|1935Q2|N|Y|N|2427894|2427983|2427541|2427816|N|N|N|N|N| +2427907|AAAAAAAADAMAFCAA|1935-04-14|423|1841|142|1935|0|4|14|2|1935|142|1841|Sunday|1935Q2|N|N|N|2427894|2427983|2427542|2427817|N|N|N|N|N| +2427908|AAAAAAAAEAMAFCAA|1935-04-15|423|1841|142|1935|1|4|15|2|1935|142|1841|Monday|1935Q2|N|N|N|2427894|2427983|2427543|2427818|N|N|N|N|N| +2427909|AAAAAAAAFAMAFCAA|1935-04-16|423|1842|142|1935|2|4|16|2|1935|142|1842|Tuesday|1935Q2|N|N|N|2427894|2427983|2427544|2427819|N|N|N|N|N| +2427910|AAAAAAAAGAMAFCAA|1935-04-17|423|1842|142|1935|3|4|17|2|1935|142|1842|Wednesday|1935Q2|N|N|N|2427894|2427983|2427545|2427820|N|N|N|N|N| +2427911|AAAAAAAAHAMAFCAA|1935-04-18|423|1842|142|1935|4|4|18|2|1935|142|1842|Thursday|1935Q2|N|N|N|2427894|2427983|2427546|2427821|N|N|N|N|N| +2427912|AAAAAAAAIAMAFCAA|1935-04-19|423|1842|142|1935|5|4|19|2|1935|142|1842|Friday|1935Q2|N|Y|N|2427894|2427983|2427547|2427822|N|N|N|N|N| +2427913|AAAAAAAAJAMAFCAA|1935-04-20|423|1842|142|1935|6|4|20|2|1935|142|1842|Saturday|1935Q2|N|Y|N|2427894|2427983|2427548|2427823|N|N|N|N|N| +2427914|AAAAAAAAKAMAFCAA|1935-04-21|423|1842|142|1935|0|4|21|2|1935|142|1842|Sunday|1935Q2|N|N|N|2427894|2427983|2427549|2427824|N|N|N|N|N| +2427915|AAAAAAAALAMAFCAA|1935-04-22|423|1842|142|1935|1|4|22|2|1935|142|1842|Monday|1935Q2|N|N|N|2427894|2427983|2427550|2427825|N|N|N|N|N| +2427916|AAAAAAAAMAMAFCAA|1935-04-23|423|1843|142|1935|2|4|23|2|1935|142|1843|Tuesday|1935Q2|N|N|N|2427894|2427983|2427551|2427826|N|N|N|N|N| +2427917|AAAAAAAANAMAFCAA|1935-04-24|423|1843|142|1935|3|4|24|2|1935|142|1843|Wednesday|1935Q2|N|N|N|2427894|2427983|2427552|2427827|N|N|N|N|N| +2427918|AAAAAAAAOAMAFCAA|1935-04-25|423|1843|142|1935|4|4|25|2|1935|142|1843|Thursday|1935Q2|N|N|N|2427894|2427983|2427553|2427828|N|N|N|N|N| +2427919|AAAAAAAAPAMAFCAA|1935-04-26|423|1843|142|1935|5|4|26|2|1935|142|1843|Friday|1935Q2|N|Y|N|2427894|2427983|2427554|2427829|N|N|N|N|N| +2427920|AAAAAAAAABMAFCAA|1935-04-27|423|1843|142|1935|6|4|27|2|1935|142|1843|Saturday|1935Q2|N|Y|N|2427894|2427983|2427555|2427830|N|N|N|N|N| +2427921|AAAAAAAABBMAFCAA|1935-04-28|423|1843|142|1935|0|4|28|2|1935|142|1843|Sunday|1935Q2|N|N|N|2427894|2427983|2427556|2427831|N|N|N|N|N| +2427922|AAAAAAAACBMAFCAA|1935-04-29|423|1843|142|1935|1|4|29|2|1935|142|1843|Monday|1935Q2|N|N|N|2427894|2427983|2427557|2427832|N|N|N|N|N| +2427923|AAAAAAAADBMAFCAA|1935-04-30|423|1844|142|1935|2|4|30|2|1935|142|1844|Tuesday|1935Q2|N|N|N|2427894|2427983|2427558|2427833|N|N|N|N|N| +2427924|AAAAAAAAEBMAFCAA|1935-05-01|424|1844|142|1935|3|5|1|2|1935|142|1844|Wednesday|1935Q2|N|N|N|2427924|2428043|2427559|2427834|N|N|N|N|N| +2427925|AAAAAAAAFBMAFCAA|1935-05-02|424|1844|142|1935|4|5|2|2|1935|142|1844|Thursday|1935Q2|N|N|N|2427924|2428043|2427560|2427835|N|N|N|N|N| +2427926|AAAAAAAAGBMAFCAA|1935-05-03|424|1844|142|1935|5|5|3|2|1935|142|1844|Friday|1935Q2|N|Y|N|2427924|2428043|2427561|2427836|N|N|N|N|N| +2427927|AAAAAAAAHBMAFCAA|1935-05-04|424|1844|142|1935|6|5|4|2|1935|142|1844|Saturday|1935Q2|N|Y|N|2427924|2428043|2427562|2427837|N|N|N|N|N| +2427928|AAAAAAAAIBMAFCAA|1935-05-05|424|1844|142|1935|0|5|5|2|1935|142|1844|Sunday|1935Q2|N|N|N|2427924|2428043|2427563|2427838|N|N|N|N|N| +2427929|AAAAAAAAJBMAFCAA|1935-05-06|424|1844|142|1935|1|5|6|2|1935|142|1844|Monday|1935Q2|N|N|N|2427924|2428043|2427564|2427839|N|N|N|N|N| +2427930|AAAAAAAAKBMAFCAA|1935-05-07|424|1845|142|1935|2|5|7|2|1935|142|1845|Tuesday|1935Q2|N|N|N|2427924|2428043|2427565|2427840|N|N|N|N|N| +2427931|AAAAAAAALBMAFCAA|1935-05-08|424|1845|142|1935|3|5|8|2|1935|142|1845|Wednesday|1935Q2|N|N|N|2427924|2428043|2427566|2427841|N|N|N|N|N| +2427932|AAAAAAAAMBMAFCAA|1935-05-09|424|1845|142|1935|4|5|9|2|1935|142|1845|Thursday|1935Q2|N|N|N|2427924|2428043|2427567|2427842|N|N|N|N|N| +2427933|AAAAAAAANBMAFCAA|1935-05-10|424|1845|142|1935|5|5|10|2|1935|142|1845|Friday|1935Q2|N|Y|N|2427924|2428043|2427568|2427843|N|N|N|N|N| +2427934|AAAAAAAAOBMAFCAA|1935-05-11|424|1845|142|1935|6|5|11|2|1935|142|1845|Saturday|1935Q2|N|Y|N|2427924|2428043|2427569|2427844|N|N|N|N|N| +2427935|AAAAAAAAPBMAFCAA|1935-05-12|424|1845|142|1935|0|5|12|2|1935|142|1845|Sunday|1935Q2|N|N|N|2427924|2428043|2427570|2427845|N|N|N|N|N| +2427936|AAAAAAAAACMAFCAA|1935-05-13|424|1845|142|1935|1|5|13|2|1935|142|1845|Monday|1935Q2|N|N|N|2427924|2428043|2427571|2427846|N|N|N|N|N| +2427937|AAAAAAAABCMAFCAA|1935-05-14|424|1846|142|1935|2|5|14|2|1935|142|1846|Tuesday|1935Q2|N|N|N|2427924|2428043|2427572|2427847|N|N|N|N|N| +2427938|AAAAAAAACCMAFCAA|1935-05-15|424|1846|142|1935|3|5|15|2|1935|142|1846|Wednesday|1935Q2|N|N|N|2427924|2428043|2427573|2427848|N|N|N|N|N| +2427939|AAAAAAAADCMAFCAA|1935-05-16|424|1846|142|1935|4|5|16|2|1935|142|1846|Thursday|1935Q2|N|N|N|2427924|2428043|2427574|2427849|N|N|N|N|N| +2427940|AAAAAAAAECMAFCAA|1935-05-17|424|1846|142|1935|5|5|17|2|1935|142|1846|Friday|1935Q2|N|Y|N|2427924|2428043|2427575|2427850|N|N|N|N|N| +2427941|AAAAAAAAFCMAFCAA|1935-05-18|424|1846|142|1935|6|5|18|2|1935|142|1846|Saturday|1935Q2|N|Y|N|2427924|2428043|2427576|2427851|N|N|N|N|N| +2427942|AAAAAAAAGCMAFCAA|1935-05-19|424|1846|142|1935|0|5|19|2|1935|142|1846|Sunday|1935Q2|N|N|N|2427924|2428043|2427577|2427852|N|N|N|N|N| +2427943|AAAAAAAAHCMAFCAA|1935-05-20|424|1846|142|1935|1|5|20|2|1935|142|1846|Monday|1935Q2|N|N|N|2427924|2428043|2427578|2427853|N|N|N|N|N| +2427944|AAAAAAAAICMAFCAA|1935-05-21|424|1847|142|1935|2|5|21|2|1935|142|1847|Tuesday|1935Q2|N|N|N|2427924|2428043|2427579|2427854|N|N|N|N|N| +2427945|AAAAAAAAJCMAFCAA|1935-05-22|424|1847|142|1935|3|5|22|2|1935|142|1847|Wednesday|1935Q2|N|N|N|2427924|2428043|2427580|2427855|N|N|N|N|N| +2427946|AAAAAAAAKCMAFCAA|1935-05-23|424|1847|142|1935|4|5|23|2|1935|142|1847|Thursday|1935Q2|N|N|N|2427924|2428043|2427581|2427856|N|N|N|N|N| +2427947|AAAAAAAALCMAFCAA|1935-05-24|424|1847|142|1935|5|5|24|2|1935|142|1847|Friday|1935Q2|N|Y|N|2427924|2428043|2427582|2427857|N|N|N|N|N| +2427948|AAAAAAAAMCMAFCAA|1935-05-25|424|1847|142|1935|6|5|25|2|1935|142|1847|Saturday|1935Q2|N|Y|N|2427924|2428043|2427583|2427858|N|N|N|N|N| +2427949|AAAAAAAANCMAFCAA|1935-05-26|424|1847|142|1935|0|5|26|2|1935|142|1847|Sunday|1935Q2|N|N|N|2427924|2428043|2427584|2427859|N|N|N|N|N| +2427950|AAAAAAAAOCMAFCAA|1935-05-27|424|1847|142|1935|1|5|27|2|1935|142|1847|Monday|1935Q2|N|N|N|2427924|2428043|2427585|2427860|N|N|N|N|N| +2427951|AAAAAAAAPCMAFCAA|1935-05-28|424|1848|142|1935|2|5|28|2|1935|142|1848|Tuesday|1935Q2|N|N|N|2427924|2428043|2427586|2427861|N|N|N|N|N| +2427952|AAAAAAAAADMAFCAA|1935-05-29|424|1848|142|1935|3|5|29|2|1935|142|1848|Wednesday|1935Q2|N|N|N|2427924|2428043|2427587|2427862|N|N|N|N|N| +2427953|AAAAAAAABDMAFCAA|1935-05-30|424|1848|142|1935|4|5|30|2|1935|142|1848|Thursday|1935Q2|N|N|N|2427924|2428043|2427588|2427863|N|N|N|N|N| +2427954|AAAAAAAACDMAFCAA|1935-05-31|424|1848|142|1935|5|5|31|2|1935|142|1848|Friday|1935Q2|N|Y|N|2427924|2428043|2427589|2427864|N|N|N|N|N| +2427955|AAAAAAAADDMAFCAA|1935-06-01|425|1848|143|1935|6|6|1|2|1935|143|1848|Saturday|1935Q2|N|Y|N|2427955|2428105|2427590|2427865|N|N|N|N|N| +2427956|AAAAAAAAEDMAFCAA|1935-06-02|425|1848|143|1935|0|6|2|2|1935|143|1848|Sunday|1935Q2|N|N|N|2427955|2428105|2427591|2427866|N|N|N|N|N| +2427957|AAAAAAAAFDMAFCAA|1935-06-03|425|1848|143|1935|1|6|3|2|1935|143|1848|Monday|1935Q2|N|N|N|2427955|2428105|2427592|2427867|N|N|N|N|N| +2427958|AAAAAAAAGDMAFCAA|1935-06-04|425|1849|143|1935|2|6|4|2|1935|143|1849|Tuesday|1935Q2|N|N|N|2427955|2428105|2427593|2427868|N|N|N|N|N| +2427959|AAAAAAAAHDMAFCAA|1935-06-05|425|1849|143|1935|3|6|5|2|1935|143|1849|Wednesday|1935Q2|N|N|N|2427955|2428105|2427594|2427869|N|N|N|N|N| +2427960|AAAAAAAAIDMAFCAA|1935-06-06|425|1849|143|1935|4|6|6|2|1935|143|1849|Thursday|1935Q2|N|N|N|2427955|2428105|2427595|2427870|N|N|N|N|N| +2427961|AAAAAAAAJDMAFCAA|1935-06-07|425|1849|143|1935|5|6|7|2|1935|143|1849|Friday|1935Q2|N|Y|N|2427955|2428105|2427596|2427871|N|N|N|N|N| +2427962|AAAAAAAAKDMAFCAA|1935-06-08|425|1849|143|1935|6|6|8|2|1935|143|1849|Saturday|1935Q2|N|Y|N|2427955|2428105|2427597|2427872|N|N|N|N|N| +2427963|AAAAAAAALDMAFCAA|1935-06-09|425|1849|143|1935|0|6|9|2|1935|143|1849|Sunday|1935Q2|N|N|N|2427955|2428105|2427598|2427873|N|N|N|N|N| +2427964|AAAAAAAAMDMAFCAA|1935-06-10|425|1849|143|1935|1|6|10|2|1935|143|1849|Monday|1935Q2|N|N|N|2427955|2428105|2427599|2427874|N|N|N|N|N| +2427965|AAAAAAAANDMAFCAA|1935-06-11|425|1850|143|1935|2|6|11|2|1935|143|1850|Tuesday|1935Q2|N|N|N|2427955|2428105|2427600|2427875|N|N|N|N|N| +2427966|AAAAAAAAODMAFCAA|1935-06-12|425|1850|143|1935|3|6|12|2|1935|143|1850|Wednesday|1935Q2|N|N|N|2427955|2428105|2427601|2427876|N|N|N|N|N| +2427967|AAAAAAAAPDMAFCAA|1935-06-13|425|1850|143|1935|4|6|13|2|1935|143|1850|Thursday|1935Q2|N|N|N|2427955|2428105|2427602|2427877|N|N|N|N|N| +2427968|AAAAAAAAAEMAFCAA|1935-06-14|425|1850|143|1935|5|6|14|2|1935|143|1850|Friday|1935Q2|N|Y|N|2427955|2428105|2427603|2427878|N|N|N|N|N| +2427969|AAAAAAAABEMAFCAA|1935-06-15|425|1850|143|1935|6|6|15|2|1935|143|1850|Saturday|1935Q2|N|Y|N|2427955|2428105|2427604|2427879|N|N|N|N|N| +2427970|AAAAAAAACEMAFCAA|1935-06-16|425|1850|143|1935|0|6|16|2|1935|143|1850|Sunday|1935Q2|N|N|N|2427955|2428105|2427605|2427880|N|N|N|N|N| +2427971|AAAAAAAADEMAFCAA|1935-06-17|425|1850|143|1935|1|6|17|2|1935|143|1850|Monday|1935Q2|N|N|N|2427955|2428105|2427606|2427881|N|N|N|N|N| +2427972|AAAAAAAAEEMAFCAA|1935-06-18|425|1851|143|1935|2|6|18|2|1935|143|1851|Tuesday|1935Q2|N|N|N|2427955|2428105|2427607|2427882|N|N|N|N|N| +2427973|AAAAAAAAFEMAFCAA|1935-06-19|425|1851|143|1935|3|6|19|2|1935|143|1851|Wednesday|1935Q2|N|N|N|2427955|2428105|2427608|2427883|N|N|N|N|N| +2427974|AAAAAAAAGEMAFCAA|1935-06-20|425|1851|143|1935|4|6|20|2|1935|143|1851|Thursday|1935Q2|N|N|N|2427955|2428105|2427609|2427884|N|N|N|N|N| +2427975|AAAAAAAAHEMAFCAA|1935-06-21|425|1851|143|1935|5|6|21|2|1935|143|1851|Friday|1935Q2|N|Y|N|2427955|2428105|2427610|2427885|N|N|N|N|N| +2427976|AAAAAAAAIEMAFCAA|1935-06-22|425|1851|143|1935|6|6|22|2|1935|143|1851|Saturday|1935Q2|N|Y|N|2427955|2428105|2427611|2427886|N|N|N|N|N| +2427977|AAAAAAAAJEMAFCAA|1935-06-23|425|1851|143|1935|0|6|23|2|1935|143|1851|Sunday|1935Q2|N|N|N|2427955|2428105|2427612|2427887|N|N|N|N|N| +2427978|AAAAAAAAKEMAFCAA|1935-06-24|425|1851|143|1935|1|6|24|2|1935|143|1851|Monday|1935Q2|N|N|N|2427955|2428105|2427613|2427888|N|N|N|N|N| +2427979|AAAAAAAALEMAFCAA|1935-06-25|425|1852|143|1935|2|6|25|2|1935|143|1852|Tuesday|1935Q2|N|N|N|2427955|2428105|2427614|2427889|N|N|N|N|N| +2427980|AAAAAAAAMEMAFCAA|1935-06-26|425|1852|143|1935|3|6|26|2|1935|143|1852|Wednesday|1935Q2|N|N|N|2427955|2428105|2427615|2427890|N|N|N|N|N| +2427981|AAAAAAAANEMAFCAA|1935-06-27|425|1852|143|1935|4|6|27|2|1935|143|1852|Thursday|1935Q2|N|N|N|2427955|2428105|2427616|2427891|N|N|N|N|N| +2427982|AAAAAAAAOEMAFCAA|1935-06-28|425|1852|143|1935|5|6|28|2|1935|143|1852|Friday|1935Q2|N|Y|N|2427955|2428105|2427617|2427892|N|N|N|N|N| +2427983|AAAAAAAAPEMAFCAA|1935-06-29|425|1852|143|1935|6|6|29|2|1935|143|1852|Saturday|1935Q2|N|Y|N|2427955|2428105|2427618|2427893|N|N|N|N|N| +2427984|AAAAAAAAAFMAFCAA|1935-06-30|425|1852|143|1935|0|6|30|2|1935|143|1852|Sunday|1935Q2|N|N|N|2427955|2428105|2427619|2427894|N|N|N|N|N| +2427985|AAAAAAAABFMAFCAA|1935-07-01|426|1852|143|1935|1|7|1|2|1935|143|1852|Monday|1935Q2|N|N|N|2427985|2428165|2427620|2427894|N|N|N|N|N| +2427986|AAAAAAAACFMAFCAA|1935-07-02|426|1853|143|1935|2|7|2|3|1935|143|1853|Tuesday|1935Q3|N|N|N|2427985|2428165|2427621|2427895|N|N|N|N|N| +2427987|AAAAAAAADFMAFCAA|1935-07-03|426|1853|143|1935|3|7|3|3|1935|143|1853|Wednesday|1935Q3|N|N|N|2427985|2428165|2427622|2427896|N|N|N|N|N| +2427988|AAAAAAAAEFMAFCAA|1935-07-04|426|1853|143|1935|4|7|4|3|1935|143|1853|Thursday|1935Q3|N|N|N|2427985|2428165|2427623|2427897|N|N|N|N|N| +2427989|AAAAAAAAFFMAFCAA|1935-07-05|426|1853|143|1935|5|7|5|3|1935|143|1853|Friday|1935Q3|Y|Y|N|2427985|2428165|2427624|2427898|N|N|N|N|N| +2427990|AAAAAAAAGFMAFCAA|1935-07-06|426|1853|143|1935|6|7|6|3|1935|143|1853|Saturday|1935Q3|N|Y|Y|2427985|2428165|2427625|2427899|N|N|N|N|N| +2427991|AAAAAAAAHFMAFCAA|1935-07-07|426|1853|143|1935|0|7|7|3|1935|143|1853|Sunday|1935Q3|N|N|N|2427985|2428165|2427626|2427900|N|N|N|N|N| +2427992|AAAAAAAAIFMAFCAA|1935-07-08|426|1853|143|1935|1|7|8|3|1935|143|1853|Monday|1935Q3|N|N|N|2427985|2428165|2427627|2427901|N|N|N|N|N| +2427993|AAAAAAAAJFMAFCAA|1935-07-09|426|1854|143|1935|2|7|9|3|1935|143|1854|Tuesday|1935Q3|N|N|N|2427985|2428165|2427628|2427902|N|N|N|N|N| +2427994|AAAAAAAAKFMAFCAA|1935-07-10|426|1854|143|1935|3|7|10|3|1935|143|1854|Wednesday|1935Q3|N|N|N|2427985|2428165|2427629|2427903|N|N|N|N|N| +2427995|AAAAAAAALFMAFCAA|1935-07-11|426|1854|143|1935|4|7|11|3|1935|143|1854|Thursday|1935Q3|N|N|N|2427985|2428165|2427630|2427904|N|N|N|N|N| +2427996|AAAAAAAAMFMAFCAA|1935-07-12|426|1854|143|1935|5|7|12|3|1935|143|1854|Friday|1935Q3|N|Y|N|2427985|2428165|2427631|2427905|N|N|N|N|N| +2427997|AAAAAAAANFMAFCAA|1935-07-13|426|1854|143|1935|6|7|13|3|1935|143|1854|Saturday|1935Q3|N|Y|N|2427985|2428165|2427632|2427906|N|N|N|N|N| +2427998|AAAAAAAAOFMAFCAA|1935-07-14|426|1854|143|1935|0|7|14|3|1935|143|1854|Sunday|1935Q3|N|N|N|2427985|2428165|2427633|2427907|N|N|N|N|N| +2427999|AAAAAAAAPFMAFCAA|1935-07-15|426|1854|143|1935|1|7|15|3|1935|143|1854|Monday|1935Q3|N|N|N|2427985|2428165|2427634|2427908|N|N|N|N|N| +2428000|AAAAAAAAAGMAFCAA|1935-07-16|426|1855|143|1935|2|7|16|3|1935|143|1855|Tuesday|1935Q3|N|N|N|2427985|2428165|2427635|2427909|N|N|N|N|N| +2428001|AAAAAAAABGMAFCAA|1935-07-17|426|1855|143|1935|3|7|17|3|1935|143|1855|Wednesday|1935Q3|N|N|N|2427985|2428165|2427636|2427910|N|N|N|N|N| +2428002|AAAAAAAACGMAFCAA|1935-07-18|426|1855|143|1935|4|7|18|3|1935|143|1855|Thursday|1935Q3|N|N|N|2427985|2428165|2427637|2427911|N|N|N|N|N| +2428003|AAAAAAAADGMAFCAA|1935-07-19|426|1855|143|1935|5|7|19|3|1935|143|1855|Friday|1935Q3|N|Y|N|2427985|2428165|2427638|2427912|N|N|N|N|N| +2428004|AAAAAAAAEGMAFCAA|1935-07-20|426|1855|143|1935|6|7|20|3|1935|143|1855|Saturday|1935Q3|N|Y|N|2427985|2428165|2427639|2427913|N|N|N|N|N| +2428005|AAAAAAAAFGMAFCAA|1935-07-21|426|1855|143|1935|0|7|21|3|1935|143|1855|Sunday|1935Q3|N|N|N|2427985|2428165|2427640|2427914|N|N|N|N|N| +2428006|AAAAAAAAGGMAFCAA|1935-07-22|426|1855|143|1935|1|7|22|3|1935|143|1855|Monday|1935Q3|N|N|N|2427985|2428165|2427641|2427915|N|N|N|N|N| +2428007|AAAAAAAAHGMAFCAA|1935-07-23|426|1856|143|1935|2|7|23|3|1935|143|1856|Tuesday|1935Q3|N|N|N|2427985|2428165|2427642|2427916|N|N|N|N|N| +2428008|AAAAAAAAIGMAFCAA|1935-07-24|426|1856|143|1935|3|7|24|3|1935|143|1856|Wednesday|1935Q3|N|N|N|2427985|2428165|2427643|2427917|N|N|N|N|N| +2428009|AAAAAAAAJGMAFCAA|1935-07-25|426|1856|143|1935|4|7|25|3|1935|143|1856|Thursday|1935Q3|N|N|N|2427985|2428165|2427644|2427918|N|N|N|N|N| +2428010|AAAAAAAAKGMAFCAA|1935-07-26|426|1856|143|1935|5|7|26|3|1935|143|1856|Friday|1935Q3|N|Y|N|2427985|2428165|2427645|2427919|N|N|N|N|N| +2428011|AAAAAAAALGMAFCAA|1935-07-27|426|1856|143|1935|6|7|27|3|1935|143|1856|Saturday|1935Q3|N|Y|N|2427985|2428165|2427646|2427920|N|N|N|N|N| +2428012|AAAAAAAAMGMAFCAA|1935-07-28|426|1856|143|1935|0|7|28|3|1935|143|1856|Sunday|1935Q3|N|N|N|2427985|2428165|2427647|2427921|N|N|N|N|N| +2428013|AAAAAAAANGMAFCAA|1935-07-29|426|1856|143|1935|1|7|29|3|1935|143|1856|Monday|1935Q3|N|N|N|2427985|2428165|2427648|2427922|N|N|N|N|N| +2428014|AAAAAAAAOGMAFCAA|1935-07-30|426|1857|143|1935|2|7|30|3|1935|143|1857|Tuesday|1935Q3|N|N|N|2427985|2428165|2427649|2427923|N|N|N|N|N| +2428015|AAAAAAAAPGMAFCAA|1935-07-31|426|1857|143|1935|3|7|31|3|1935|143|1857|Wednesday|1935Q3|N|N|N|2427985|2428165|2427650|2427924|N|N|N|N|N| +2428016|AAAAAAAAAHMAFCAA|1935-08-01|427|1857|143|1935|4|8|1|3|1935|143|1857|Thursday|1935Q3|N|N|N|2428016|2428227|2427651|2427925|N|N|N|N|N| +2428017|AAAAAAAABHMAFCAA|1935-08-02|427|1857|143|1935|5|8|2|3|1935|143|1857|Friday|1935Q3|N|Y|N|2428016|2428227|2427652|2427926|N|N|N|N|N| +2428018|AAAAAAAACHMAFCAA|1935-08-03|427|1857|143|1935|6|8|3|3|1935|143|1857|Saturday|1935Q3|N|Y|N|2428016|2428227|2427653|2427927|N|N|N|N|N| +2428019|AAAAAAAADHMAFCAA|1935-08-04|427|1857|143|1935|0|8|4|3|1935|143|1857|Sunday|1935Q3|N|N|N|2428016|2428227|2427654|2427928|N|N|N|N|N| +2428020|AAAAAAAAEHMAFCAA|1935-08-05|427|1857|143|1935|1|8|5|3|1935|143|1857|Monday|1935Q3|N|N|N|2428016|2428227|2427655|2427929|N|N|N|N|N| +2428021|AAAAAAAAFHMAFCAA|1935-08-06|427|1858|143|1935|2|8|6|3|1935|143|1858|Tuesday|1935Q3|N|N|N|2428016|2428227|2427656|2427930|N|N|N|N|N| +2428022|AAAAAAAAGHMAFCAA|1935-08-07|427|1858|143|1935|3|8|7|3|1935|143|1858|Wednesday|1935Q3|N|N|N|2428016|2428227|2427657|2427931|N|N|N|N|N| +2428023|AAAAAAAAHHMAFCAA|1935-08-08|427|1858|143|1935|4|8|8|3|1935|143|1858|Thursday|1935Q3|N|N|N|2428016|2428227|2427658|2427932|N|N|N|N|N| +2428024|AAAAAAAAIHMAFCAA|1935-08-09|427|1858|143|1935|5|8|9|3|1935|143|1858|Friday|1935Q3|N|Y|N|2428016|2428227|2427659|2427933|N|N|N|N|N| +2428025|AAAAAAAAJHMAFCAA|1935-08-10|427|1858|143|1935|6|8|10|3|1935|143|1858|Saturday|1935Q3|N|Y|N|2428016|2428227|2427660|2427934|N|N|N|N|N| +2428026|AAAAAAAAKHMAFCAA|1935-08-11|427|1858|143|1935|0|8|11|3|1935|143|1858|Sunday|1935Q3|N|N|N|2428016|2428227|2427661|2427935|N|N|N|N|N| +2428027|AAAAAAAALHMAFCAA|1935-08-12|427|1858|143|1935|1|8|12|3|1935|143|1858|Monday|1935Q3|N|N|N|2428016|2428227|2427662|2427936|N|N|N|N|N| +2428028|AAAAAAAAMHMAFCAA|1935-08-13|427|1859|143|1935|2|8|13|3|1935|143|1859|Tuesday|1935Q3|N|N|N|2428016|2428227|2427663|2427937|N|N|N|N|N| +2428029|AAAAAAAANHMAFCAA|1935-08-14|427|1859|143|1935|3|8|14|3|1935|143|1859|Wednesday|1935Q3|N|N|N|2428016|2428227|2427664|2427938|N|N|N|N|N| +2428030|AAAAAAAAOHMAFCAA|1935-08-15|427|1859|143|1935|4|8|15|3|1935|143|1859|Thursday|1935Q3|N|N|N|2428016|2428227|2427665|2427939|N|N|N|N|N| +2428031|AAAAAAAAPHMAFCAA|1935-08-16|427|1859|143|1935|5|8|16|3|1935|143|1859|Friday|1935Q3|N|Y|N|2428016|2428227|2427666|2427940|N|N|N|N|N| +2428032|AAAAAAAAAIMAFCAA|1935-08-17|427|1859|143|1935|6|8|17|3|1935|143|1859|Saturday|1935Q3|N|Y|N|2428016|2428227|2427667|2427941|N|N|N|N|N| +2428033|AAAAAAAABIMAFCAA|1935-08-18|427|1859|143|1935|0|8|18|3|1935|143|1859|Sunday|1935Q3|N|N|N|2428016|2428227|2427668|2427942|N|N|N|N|N| +2428034|AAAAAAAACIMAFCAA|1935-08-19|427|1859|143|1935|1|8|19|3|1935|143|1859|Monday|1935Q3|N|N|N|2428016|2428227|2427669|2427943|N|N|N|N|N| +2428035|AAAAAAAADIMAFCAA|1935-08-20|427|1860|143|1935|2|8|20|3|1935|143|1860|Tuesday|1935Q3|N|N|N|2428016|2428227|2427670|2427944|N|N|N|N|N| +2428036|AAAAAAAAEIMAFCAA|1935-08-21|427|1860|143|1935|3|8|21|3|1935|143|1860|Wednesday|1935Q3|N|N|N|2428016|2428227|2427671|2427945|N|N|N|N|N| +2428037|AAAAAAAAFIMAFCAA|1935-08-22|427|1860|143|1935|4|8|22|3|1935|143|1860|Thursday|1935Q3|N|N|N|2428016|2428227|2427672|2427946|N|N|N|N|N| +2428038|AAAAAAAAGIMAFCAA|1935-08-23|427|1860|143|1935|5|8|23|3|1935|143|1860|Friday|1935Q3|N|Y|N|2428016|2428227|2427673|2427947|N|N|N|N|N| +2428039|AAAAAAAAHIMAFCAA|1935-08-24|427|1860|143|1935|6|8|24|3|1935|143|1860|Saturday|1935Q3|N|Y|N|2428016|2428227|2427674|2427948|N|N|N|N|N| +2428040|AAAAAAAAIIMAFCAA|1935-08-25|427|1860|143|1935|0|8|25|3|1935|143|1860|Sunday|1935Q3|N|N|N|2428016|2428227|2427675|2427949|N|N|N|N|N| +2428041|AAAAAAAAJIMAFCAA|1935-08-26|427|1860|143|1935|1|8|26|3|1935|143|1860|Monday|1935Q3|N|N|N|2428016|2428227|2427676|2427950|N|N|N|N|N| +2428042|AAAAAAAAKIMAFCAA|1935-08-27|427|1861|143|1935|2|8|27|3|1935|143|1861|Tuesday|1935Q3|N|N|N|2428016|2428227|2427677|2427951|N|N|N|N|N| +2428043|AAAAAAAALIMAFCAA|1935-08-28|427|1861|143|1935|3|8|28|3|1935|143|1861|Wednesday|1935Q3|N|N|N|2428016|2428227|2427678|2427952|N|N|N|N|N| +2428044|AAAAAAAAMIMAFCAA|1935-08-29|427|1861|143|1935|4|8|29|3|1935|143|1861|Thursday|1935Q3|N|N|N|2428016|2428227|2427679|2427953|N|N|N|N|N| +2428045|AAAAAAAANIMAFCAA|1935-08-30|427|1861|143|1935|5|8|30|3|1935|143|1861|Friday|1935Q3|N|Y|N|2428016|2428227|2427680|2427954|N|N|N|N|N| +2428046|AAAAAAAAOIMAFCAA|1935-08-31|427|1861|143|1935|6|8|31|3|1935|143|1861|Saturday|1935Q3|N|Y|N|2428016|2428227|2427681|2427955|N|N|N|N|N| +2428047|AAAAAAAAPIMAFCAA|1935-09-01|428|1861|144|1935|0|9|1|3|1935|144|1861|Sunday|1935Q3|N|N|N|2428047|2428289|2427682|2427956|N|N|N|N|N| +2428048|AAAAAAAAAJMAFCAA|1935-09-02|428|1861|144|1935|1|9|2|3|1935|144|1861|Monday|1935Q3|N|N|N|2428047|2428289|2427683|2427957|N|N|N|N|N| +2428049|AAAAAAAABJMAFCAA|1935-09-03|428|1862|144|1935|2|9|3|3|1935|144|1862|Tuesday|1935Q3|N|N|N|2428047|2428289|2427684|2427958|N|N|N|N|N| +2428050|AAAAAAAACJMAFCAA|1935-09-04|428|1862|144|1935|3|9|4|3|1935|144|1862|Wednesday|1935Q3|N|N|N|2428047|2428289|2427685|2427959|N|N|N|N|N| +2428051|AAAAAAAADJMAFCAA|1935-09-05|428|1862|144|1935|4|9|5|3|1935|144|1862|Thursday|1935Q3|N|N|N|2428047|2428289|2427686|2427960|N|N|N|N|N| +2428052|AAAAAAAAEJMAFCAA|1935-09-06|428|1862|144|1935|5|9|6|3|1935|144|1862|Friday|1935Q3|N|Y|N|2428047|2428289|2427687|2427961|N|N|N|N|N| +2428053|AAAAAAAAFJMAFCAA|1935-09-07|428|1862|144|1935|6|9|7|3|1935|144|1862|Saturday|1935Q3|N|Y|N|2428047|2428289|2427688|2427962|N|N|N|N|N| +2428054|AAAAAAAAGJMAFCAA|1935-09-08|428|1862|144|1935|0|9|8|3|1935|144|1862|Sunday|1935Q3|N|N|N|2428047|2428289|2427689|2427963|N|N|N|N|N| +2428055|AAAAAAAAHJMAFCAA|1935-09-09|428|1862|144|1935|1|9|9|3|1935|144|1862|Monday|1935Q3|N|N|N|2428047|2428289|2427690|2427964|N|N|N|N|N| +2428056|AAAAAAAAIJMAFCAA|1935-09-10|428|1863|144|1935|2|9|10|3|1935|144|1863|Tuesday|1935Q3|N|N|N|2428047|2428289|2427691|2427965|N|N|N|N|N| +2428057|AAAAAAAAJJMAFCAA|1935-09-11|428|1863|144|1935|3|9|11|3|1935|144|1863|Wednesday|1935Q3|N|N|N|2428047|2428289|2427692|2427966|N|N|N|N|N| +2428058|AAAAAAAAKJMAFCAA|1935-09-12|428|1863|144|1935|4|9|12|3|1935|144|1863|Thursday|1935Q3|N|N|N|2428047|2428289|2427693|2427967|N|N|N|N|N| +2428059|AAAAAAAALJMAFCAA|1935-09-13|428|1863|144|1935|5|9|13|3|1935|144|1863|Friday|1935Q3|N|Y|N|2428047|2428289|2427694|2427968|N|N|N|N|N| +2428060|AAAAAAAAMJMAFCAA|1935-09-14|428|1863|144|1935|6|9|14|3|1935|144|1863|Saturday|1935Q3|N|Y|N|2428047|2428289|2427695|2427969|N|N|N|N|N| +2428061|AAAAAAAANJMAFCAA|1935-09-15|428|1863|144|1935|0|9|15|3|1935|144|1863|Sunday|1935Q3|N|N|N|2428047|2428289|2427696|2427970|N|N|N|N|N| +2428062|AAAAAAAAOJMAFCAA|1935-09-16|428|1863|144|1935|1|9|16|3|1935|144|1863|Monday|1935Q3|N|N|N|2428047|2428289|2427697|2427971|N|N|N|N|N| +2428063|AAAAAAAAPJMAFCAA|1935-09-17|428|1864|144|1935|2|9|17|3|1935|144|1864|Tuesday|1935Q3|N|N|N|2428047|2428289|2427698|2427972|N|N|N|N|N| +2428064|AAAAAAAAAKMAFCAA|1935-09-18|428|1864|144|1935|3|9|18|3|1935|144|1864|Wednesday|1935Q3|N|N|N|2428047|2428289|2427699|2427973|N|N|N|N|N| +2428065|AAAAAAAABKMAFCAA|1935-09-19|428|1864|144|1935|4|9|19|3|1935|144|1864|Thursday|1935Q3|N|N|N|2428047|2428289|2427700|2427974|N|N|N|N|N| +2428066|AAAAAAAACKMAFCAA|1935-09-20|428|1864|144|1935|5|9|20|3|1935|144|1864|Friday|1935Q3|N|Y|N|2428047|2428289|2427701|2427975|N|N|N|N|N| +2428067|AAAAAAAADKMAFCAA|1935-09-21|428|1864|144|1935|6|9|21|3|1935|144|1864|Saturday|1935Q3|N|Y|N|2428047|2428289|2427702|2427976|N|N|N|N|N| +2428068|AAAAAAAAEKMAFCAA|1935-09-22|428|1864|144|1935|0|9|22|3|1935|144|1864|Sunday|1935Q3|N|N|N|2428047|2428289|2427703|2427977|N|N|N|N|N| +2428069|AAAAAAAAFKMAFCAA|1935-09-23|428|1864|144|1935|1|9|23|3|1935|144|1864|Monday|1935Q3|N|N|N|2428047|2428289|2427704|2427978|N|N|N|N|N| +2428070|AAAAAAAAGKMAFCAA|1935-09-24|428|1865|144|1935|2|9|24|3|1935|144|1865|Tuesday|1935Q3|N|N|N|2428047|2428289|2427705|2427979|N|N|N|N|N| +2428071|AAAAAAAAHKMAFCAA|1935-09-25|428|1865|144|1935|3|9|25|3|1935|144|1865|Wednesday|1935Q3|N|N|N|2428047|2428289|2427706|2427980|N|N|N|N|N| +2428072|AAAAAAAAIKMAFCAA|1935-09-26|428|1865|144|1935|4|9|26|3|1935|144|1865|Thursday|1935Q3|N|N|N|2428047|2428289|2427707|2427981|N|N|N|N|N| +2428073|AAAAAAAAJKMAFCAA|1935-09-27|428|1865|144|1935|5|9|27|3|1935|144|1865|Friday|1935Q3|N|Y|N|2428047|2428289|2427708|2427982|N|N|N|N|N| +2428074|AAAAAAAAKKMAFCAA|1935-09-28|428|1865|144|1935|6|9|28|3|1935|144|1865|Saturday|1935Q3|N|Y|N|2428047|2428289|2427709|2427983|N|N|N|N|N| +2428075|AAAAAAAALKMAFCAA|1935-09-29|428|1865|144|1935|0|9|29|3|1935|144|1865|Sunday|1935Q3|N|N|N|2428047|2428289|2427710|2427984|N|N|N|N|N| +2428076|AAAAAAAAMKMAFCAA|1935-09-30|428|1865|144|1935|1|9|30|3|1935|144|1865|Monday|1935Q3|N|N|N|2428047|2428289|2427711|2427985|N|N|N|N|N| +2428077|AAAAAAAANKMAFCAA|1935-10-01|429|1866|144|1935|2|10|1|3|1935|144|1866|Tuesday|1935Q3|N|N|N|2428077|2428349|2427712|2427985|N|N|N|N|N| +2428078|AAAAAAAAOKMAFCAA|1935-10-02|429|1866|144|1935|3|10|2|4|1935|144|1866|Wednesday|1935Q4|N|N|N|2428077|2428349|2427713|2427986|N|N|N|N|N| +2428079|AAAAAAAAPKMAFCAA|1935-10-03|429|1866|144|1935|4|10|3|4|1935|144|1866|Thursday|1935Q4|N|N|N|2428077|2428349|2427714|2427987|N|N|N|N|N| +2428080|AAAAAAAAALMAFCAA|1935-10-04|429|1866|144|1935|5|10|4|4|1935|144|1866|Friday|1935Q4|N|Y|N|2428077|2428349|2427715|2427988|N|N|N|N|N| +2428081|AAAAAAAABLMAFCAA|1935-10-05|429|1866|144|1935|6|10|5|4|1935|144|1866|Saturday|1935Q4|N|Y|N|2428077|2428349|2427716|2427989|N|N|N|N|N| +2428082|AAAAAAAACLMAFCAA|1935-10-06|429|1866|144|1935|0|10|6|4|1935|144|1866|Sunday|1935Q4|N|N|N|2428077|2428349|2427717|2427990|N|N|N|N|N| +2428083|AAAAAAAADLMAFCAA|1935-10-07|429|1866|144|1935|1|10|7|4|1935|144|1866|Monday|1935Q4|N|N|N|2428077|2428349|2427718|2427991|N|N|N|N|N| +2428084|AAAAAAAAELMAFCAA|1935-10-08|429|1867|144|1935|2|10|8|4|1935|144|1867|Tuesday|1935Q4|N|N|N|2428077|2428349|2427719|2427992|N|N|N|N|N| +2428085|AAAAAAAAFLMAFCAA|1935-10-09|429|1867|144|1935|3|10|9|4|1935|144|1867|Wednesday|1935Q4|N|N|N|2428077|2428349|2427720|2427993|N|N|N|N|N| +2428086|AAAAAAAAGLMAFCAA|1935-10-10|429|1867|144|1935|4|10|10|4|1935|144|1867|Thursday|1935Q4|N|N|N|2428077|2428349|2427721|2427994|N|N|N|N|N| +2428087|AAAAAAAAHLMAFCAA|1935-10-11|429|1867|144|1935|5|10|11|4|1935|144|1867|Friday|1935Q4|N|Y|N|2428077|2428349|2427722|2427995|N|N|N|N|N| +2428088|AAAAAAAAILMAFCAA|1935-10-12|429|1867|144|1935|6|10|12|4|1935|144|1867|Saturday|1935Q4|N|Y|N|2428077|2428349|2427723|2427996|N|N|N|N|N| +2428089|AAAAAAAAJLMAFCAA|1935-10-13|429|1867|144|1935|0|10|13|4|1935|144|1867|Sunday|1935Q4|N|N|N|2428077|2428349|2427724|2427997|N|N|N|N|N| +2428090|AAAAAAAAKLMAFCAA|1935-10-14|429|1867|144|1935|1|10|14|4|1935|144|1867|Monday|1935Q4|N|N|N|2428077|2428349|2427725|2427998|N|N|N|N|N| +2428091|AAAAAAAALLMAFCAA|1935-10-15|429|1868|144|1935|2|10|15|4|1935|144|1868|Tuesday|1935Q4|N|N|N|2428077|2428349|2427726|2427999|N|N|N|N|N| +2428092|AAAAAAAAMLMAFCAA|1935-10-16|429|1868|144|1935|3|10|16|4|1935|144|1868|Wednesday|1935Q4|N|N|N|2428077|2428349|2427727|2428000|N|N|N|N|N| +2428093|AAAAAAAANLMAFCAA|1935-10-17|429|1868|144|1935|4|10|17|4|1935|144|1868|Thursday|1935Q4|N|N|N|2428077|2428349|2427728|2428001|N|N|N|N|N| +2428094|AAAAAAAAOLMAFCAA|1935-10-18|429|1868|144|1935|5|10|18|4|1935|144|1868|Friday|1935Q4|N|Y|N|2428077|2428349|2427729|2428002|N|N|N|N|N| +2428095|AAAAAAAAPLMAFCAA|1935-10-19|429|1868|144|1935|6|10|19|4|1935|144|1868|Saturday|1935Q4|N|Y|N|2428077|2428349|2427730|2428003|N|N|N|N|N| +2428096|AAAAAAAAAMMAFCAA|1935-10-20|429|1868|144|1935|0|10|20|4|1935|144|1868|Sunday|1935Q4|N|N|N|2428077|2428349|2427731|2428004|N|N|N|N|N| +2428097|AAAAAAAABMMAFCAA|1935-10-21|429|1868|144|1935|1|10|21|4|1935|144|1868|Monday|1935Q4|N|N|N|2428077|2428349|2427732|2428005|N|N|N|N|N| +2428098|AAAAAAAACMMAFCAA|1935-10-22|429|1869|144|1935|2|10|22|4|1935|144|1869|Tuesday|1935Q4|N|N|N|2428077|2428349|2427733|2428006|N|N|N|N|N| +2428099|AAAAAAAADMMAFCAA|1935-10-23|429|1869|144|1935|3|10|23|4|1935|144|1869|Wednesday|1935Q4|N|N|N|2428077|2428349|2427734|2428007|N|N|N|N|N| +2428100|AAAAAAAAEMMAFCAA|1935-10-24|429|1869|144|1935|4|10|24|4|1935|144|1869|Thursday|1935Q4|N|N|N|2428077|2428349|2427735|2428008|N|N|N|N|N| +2428101|AAAAAAAAFMMAFCAA|1935-10-25|429|1869|144|1935|5|10|25|4|1935|144|1869|Friday|1935Q4|N|Y|N|2428077|2428349|2427736|2428009|N|N|N|N|N| +2428102|AAAAAAAAGMMAFCAA|1935-10-26|429|1869|144|1935|6|10|26|4|1935|144|1869|Saturday|1935Q4|N|Y|N|2428077|2428349|2427737|2428010|N|N|N|N|N| +2428103|AAAAAAAAHMMAFCAA|1935-10-27|429|1869|144|1935|0|10|27|4|1935|144|1869|Sunday|1935Q4|N|N|N|2428077|2428349|2427738|2428011|N|N|N|N|N| +2428104|AAAAAAAAIMMAFCAA|1935-10-28|429|1869|144|1935|1|10|28|4|1935|144|1869|Monday|1935Q4|N|N|N|2428077|2428349|2427739|2428012|N|N|N|N|N| +2428105|AAAAAAAAJMMAFCAA|1935-10-29|429|1870|144|1935|2|10|29|4|1935|144|1870|Tuesday|1935Q4|N|N|N|2428077|2428349|2427740|2428013|N|N|N|N|N| +2428106|AAAAAAAAKMMAFCAA|1935-10-30|429|1870|144|1935|3|10|30|4|1935|144|1870|Wednesday|1935Q4|N|N|N|2428077|2428349|2427741|2428014|N|N|N|N|N| +2428107|AAAAAAAALMMAFCAA|1935-10-31|429|1870|144|1935|4|10|31|4|1935|144|1870|Thursday|1935Q4|N|N|N|2428077|2428349|2427742|2428015|N|N|N|N|N| +2428108|AAAAAAAAMMMAFCAA|1935-11-01|430|1870|144|1935|5|11|1|4|1935|144|1870|Friday|1935Q4|N|Y|N|2428108|2428411|2427743|2428016|N|N|N|N|N| +2428109|AAAAAAAANMMAFCAA|1935-11-02|430|1870|144|1935|6|11|2|4|1935|144|1870|Saturday|1935Q4|N|Y|N|2428108|2428411|2427744|2428017|N|N|N|N|N| +2428110|AAAAAAAAOMMAFCAA|1935-11-03|430|1870|144|1935|0|11|3|4|1935|144|1870|Sunday|1935Q4|N|N|N|2428108|2428411|2427745|2428018|N|N|N|N|N| +2428111|AAAAAAAAPMMAFCAA|1935-11-04|430|1870|144|1935|1|11|4|4|1935|144|1870|Monday|1935Q4|N|N|N|2428108|2428411|2427746|2428019|N|N|N|N|N| +2428112|AAAAAAAAANMAFCAA|1935-11-05|430|1871|144|1935|2|11|5|4|1935|144|1871|Tuesday|1935Q4|N|N|N|2428108|2428411|2427747|2428020|N|N|N|N|N| +2428113|AAAAAAAABNMAFCAA|1935-11-06|430|1871|144|1935|3|11|6|4|1935|144|1871|Wednesday|1935Q4|N|N|N|2428108|2428411|2427748|2428021|N|N|N|N|N| +2428114|AAAAAAAACNMAFCAA|1935-11-07|430|1871|144|1935|4|11|7|4|1935|144|1871|Thursday|1935Q4|N|N|N|2428108|2428411|2427749|2428022|N|N|N|N|N| +2428115|AAAAAAAADNMAFCAA|1935-11-08|430|1871|144|1935|5|11|8|4|1935|144|1871|Friday|1935Q4|N|Y|N|2428108|2428411|2427750|2428023|N|N|N|N|N| +2428116|AAAAAAAAENMAFCAA|1935-11-09|430|1871|144|1935|6|11|9|4|1935|144|1871|Saturday|1935Q4|N|Y|N|2428108|2428411|2427751|2428024|N|N|N|N|N| +2428117|AAAAAAAAFNMAFCAA|1935-11-10|430|1871|144|1935|0|11|10|4|1935|144|1871|Sunday|1935Q4|N|N|N|2428108|2428411|2427752|2428025|N|N|N|N|N| +2428118|AAAAAAAAGNMAFCAA|1935-11-11|430|1871|144|1935|1|11|11|4|1935|144|1871|Monday|1935Q4|N|N|N|2428108|2428411|2427753|2428026|N|N|N|N|N| +2428119|AAAAAAAAHNMAFCAA|1935-11-12|430|1872|144|1935|2|11|12|4|1935|144|1872|Tuesday|1935Q4|N|N|N|2428108|2428411|2427754|2428027|N|N|N|N|N| +2428120|AAAAAAAAINMAFCAA|1935-11-13|430|1872|144|1935|3|11|13|4|1935|144|1872|Wednesday|1935Q4|N|N|N|2428108|2428411|2427755|2428028|N|N|N|N|N| +2428121|AAAAAAAAJNMAFCAA|1935-11-14|430|1872|144|1935|4|11|14|4|1935|144|1872|Thursday|1935Q4|N|N|N|2428108|2428411|2427756|2428029|N|N|N|N|N| +2428122|AAAAAAAAKNMAFCAA|1935-11-15|430|1872|144|1935|5|11|15|4|1935|144|1872|Friday|1935Q4|N|Y|N|2428108|2428411|2427757|2428030|N|N|N|N|N| +2428123|AAAAAAAALNMAFCAA|1935-11-16|430|1872|144|1935|6|11|16|4|1935|144|1872|Saturday|1935Q4|N|Y|N|2428108|2428411|2427758|2428031|N|N|N|N|N| +2428124|AAAAAAAAMNMAFCAA|1935-11-17|430|1872|144|1935|0|11|17|4|1935|144|1872|Sunday|1935Q4|N|N|N|2428108|2428411|2427759|2428032|N|N|N|N|N| +2428125|AAAAAAAANNMAFCAA|1935-11-18|430|1872|144|1935|1|11|18|4|1935|144|1872|Monday|1935Q4|N|N|N|2428108|2428411|2427760|2428033|N|N|N|N|N| +2428126|AAAAAAAAONMAFCAA|1935-11-19|430|1873|144|1935|2|11|19|4|1935|144|1873|Tuesday|1935Q4|N|N|N|2428108|2428411|2427761|2428034|N|N|N|N|N| +2428127|AAAAAAAAPNMAFCAA|1935-11-20|430|1873|144|1935|3|11|20|4|1935|144|1873|Wednesday|1935Q4|N|N|N|2428108|2428411|2427762|2428035|N|N|N|N|N| +2428128|AAAAAAAAAOMAFCAA|1935-11-21|430|1873|144|1935|4|11|21|4|1935|144|1873|Thursday|1935Q4|N|N|N|2428108|2428411|2427763|2428036|N|N|N|N|N| +2428129|AAAAAAAABOMAFCAA|1935-11-22|430|1873|144|1935|5|11|22|4|1935|144|1873|Friday|1935Q4|N|Y|N|2428108|2428411|2427764|2428037|N|N|N|N|N| +2428130|AAAAAAAACOMAFCAA|1935-11-23|430|1873|144|1935|6|11|23|4|1935|144|1873|Saturday|1935Q4|N|Y|N|2428108|2428411|2427765|2428038|N|N|N|N|N| +2428131|AAAAAAAADOMAFCAA|1935-11-24|430|1873|144|1935|0|11|24|4|1935|144|1873|Sunday|1935Q4|N|N|N|2428108|2428411|2427766|2428039|N|N|N|N|N| +2428132|AAAAAAAAEOMAFCAA|1935-11-25|430|1873|144|1935|1|11|25|4|1935|144|1873|Monday|1935Q4|N|N|N|2428108|2428411|2427767|2428040|N|N|N|N|N| +2428133|AAAAAAAAFOMAFCAA|1935-11-26|430|1874|144|1935|2|11|26|4|1935|144|1874|Tuesday|1935Q4|N|N|N|2428108|2428411|2427768|2428041|N|N|N|N|N| +2428134|AAAAAAAAGOMAFCAA|1935-11-27|430|1874|144|1935|3|11|27|4|1935|144|1874|Wednesday|1935Q4|N|N|N|2428108|2428411|2427769|2428042|N|N|N|N|N| +2428135|AAAAAAAAHOMAFCAA|1935-11-28|430|1874|144|1935|4|11|28|4|1935|144|1874|Thursday|1935Q4|N|N|N|2428108|2428411|2427770|2428043|N|N|N|N|N| +2428136|AAAAAAAAIOMAFCAA|1935-11-29|430|1874|144|1935|5|11|29|4|1935|144|1874|Friday|1935Q4|N|Y|N|2428108|2428411|2427771|2428044|N|N|N|N|N| +2428137|AAAAAAAAJOMAFCAA|1935-11-30|430|1874|144|1935|6|11|30|4|1935|144|1874|Saturday|1935Q4|N|Y|N|2428108|2428411|2427772|2428045|N|N|N|N|N| +2428138|AAAAAAAAKOMAFCAA|1935-12-01|431|1874|145|1935|0|12|1|4|1935|145|1874|Sunday|1935Q4|N|N|N|2428138|2428471|2427773|2428046|N|N|N|N|N| +2428139|AAAAAAAALOMAFCAA|1935-12-02|431|1874|145|1935|1|12|2|4|1935|145|1874|Monday|1935Q4|N|N|N|2428138|2428471|2427774|2428047|N|N|N|N|N| +2428140|AAAAAAAAMOMAFCAA|1935-12-03|431|1875|145|1935|2|12|3|4|1935|145|1875|Tuesday|1935Q4|N|N|N|2428138|2428471|2427775|2428048|N|N|N|N|N| +2428141|AAAAAAAANOMAFCAA|1935-12-04|431|1875|145|1935|3|12|4|4|1935|145|1875|Wednesday|1935Q4|N|N|N|2428138|2428471|2427776|2428049|N|N|N|N|N| +2428142|AAAAAAAAOOMAFCAA|1935-12-05|431|1875|145|1935|4|12|5|4|1935|145|1875|Thursday|1935Q4|N|N|N|2428138|2428471|2427777|2428050|N|N|N|N|N| +2428143|AAAAAAAAPOMAFCAA|1935-12-06|431|1875|145|1935|5|12|6|4|1935|145|1875|Friday|1935Q4|N|Y|N|2428138|2428471|2427778|2428051|N|N|N|N|N| +2428144|AAAAAAAAAPMAFCAA|1935-12-07|431|1875|145|1935|6|12|7|4|1935|145|1875|Saturday|1935Q4|N|Y|N|2428138|2428471|2427779|2428052|N|N|N|N|N| +2428145|AAAAAAAABPMAFCAA|1935-12-08|431|1875|145|1935|0|12|8|4|1935|145|1875|Sunday|1935Q4|N|N|N|2428138|2428471|2427780|2428053|N|N|N|N|N| +2428146|AAAAAAAACPMAFCAA|1935-12-09|431|1875|145|1935|1|12|9|4|1935|145|1875|Monday|1935Q4|N|N|N|2428138|2428471|2427781|2428054|N|N|N|N|N| +2428147|AAAAAAAADPMAFCAA|1935-12-10|431|1876|145|1935|2|12|10|4|1935|145|1876|Tuesday|1935Q4|N|N|N|2428138|2428471|2427782|2428055|N|N|N|N|N| +2428148|AAAAAAAAEPMAFCAA|1935-12-11|431|1876|145|1935|3|12|11|4|1935|145|1876|Wednesday|1935Q4|N|N|N|2428138|2428471|2427783|2428056|N|N|N|N|N| +2428149|AAAAAAAAFPMAFCAA|1935-12-12|431|1876|145|1935|4|12|12|4|1935|145|1876|Thursday|1935Q4|N|N|N|2428138|2428471|2427784|2428057|N|N|N|N|N| +2428150|AAAAAAAAGPMAFCAA|1935-12-13|431|1876|145|1935|5|12|13|4|1935|145|1876|Friday|1935Q4|N|Y|N|2428138|2428471|2427785|2428058|N|N|N|N|N| +2428151|AAAAAAAAHPMAFCAA|1935-12-14|431|1876|145|1935|6|12|14|4|1935|145|1876|Saturday|1935Q4|N|Y|N|2428138|2428471|2427786|2428059|N|N|N|N|N| +2428152|AAAAAAAAIPMAFCAA|1935-12-15|431|1876|145|1935|0|12|15|4|1935|145|1876|Sunday|1935Q4|N|N|N|2428138|2428471|2427787|2428060|N|N|N|N|N| +2428153|AAAAAAAAJPMAFCAA|1935-12-16|431|1876|145|1935|1|12|16|4|1935|145|1876|Monday|1935Q4|N|N|N|2428138|2428471|2427788|2428061|N|N|N|N|N| +2428154|AAAAAAAAKPMAFCAA|1935-12-17|431|1877|145|1935|2|12|17|4|1935|145|1877|Tuesday|1935Q4|N|N|N|2428138|2428471|2427789|2428062|N|N|N|N|N| +2428155|AAAAAAAALPMAFCAA|1935-12-18|431|1877|145|1935|3|12|18|4|1935|145|1877|Wednesday|1935Q4|N|N|N|2428138|2428471|2427790|2428063|N|N|N|N|N| +2428156|AAAAAAAAMPMAFCAA|1935-12-19|431|1877|145|1935|4|12|19|4|1935|145|1877|Thursday|1935Q4|N|N|N|2428138|2428471|2427791|2428064|N|N|N|N|N| +2428157|AAAAAAAANPMAFCAA|1935-12-20|431|1877|145|1935|5|12|20|4|1935|145|1877|Friday|1935Q4|N|Y|N|2428138|2428471|2427792|2428065|N|N|N|N|N| +2428158|AAAAAAAAOPMAFCAA|1935-12-21|431|1877|145|1935|6|12|21|4|1935|145|1877|Saturday|1935Q4|N|Y|N|2428138|2428471|2427793|2428066|N|N|N|N|N| +2428159|AAAAAAAAPPMAFCAA|1935-12-22|431|1877|145|1935|0|12|22|4|1935|145|1877|Sunday|1935Q4|N|N|N|2428138|2428471|2427794|2428067|N|N|N|N|N| +2428160|AAAAAAAAAANAFCAA|1935-12-23|431|1877|145|1935|1|12|23|4|1935|145|1877|Monday|1935Q4|N|N|N|2428138|2428471|2427795|2428068|N|N|N|N|N| +2428161|AAAAAAAABANAFCAA|1935-12-24|431|1878|145|1935|2|12|24|4|1935|145|1878|Tuesday|1935Q4|N|N|N|2428138|2428471|2427796|2428069|N|N|N|N|N| +2428162|AAAAAAAACANAFCAA|1935-12-25|431|1878|145|1935|3|12|25|4|1935|145|1878|Wednesday|1935Q4|N|N|N|2428138|2428471|2427797|2428070|N|N|N|N|N| +2428163|AAAAAAAADANAFCAA|1935-12-26|431|1878|145|1935|4|12|26|4|1935|145|1878|Thursday|1935Q4|Y|N|N|2428138|2428471|2427798|2428071|N|N|N|N|N| +2428164|AAAAAAAAEANAFCAA|1935-12-27|431|1878|145|1935|5|12|27|4|1935|145|1878|Friday|1935Q4|N|Y|Y|2428138|2428471|2427799|2428072|N|N|N|N|N| +2428165|AAAAAAAAFANAFCAA|1935-12-28|431|1878|145|1935|6|12|28|4|1935|145|1878|Saturday|1935Q4|N|Y|N|2428138|2428471|2427800|2428073|N|N|N|N|N| +2428166|AAAAAAAAGANAFCAA|1935-12-29|431|1878|145|1935|0|12|29|4|1935|145|1878|Sunday|1935Q4|N|N|N|2428138|2428471|2427801|2428074|N|N|N|N|N| +2428167|AAAAAAAAHANAFCAA|1935-12-30|431|1878|145|1935|1|12|30|4|1935|145|1878|Monday|1935Q4|N|N|N|2428138|2428471|2427802|2428075|N|N|N|N|N| +2428168|AAAAAAAAIANAFCAA|1935-12-31|431|1879|145|1935|2|12|31|4|1935|145|1879|Tuesday|1935Q4|N|N|N|2428138|2428471|2427803|2428076|N|N|N|N|N| +2428169|AAAAAAAAJANAFCAA|1936-01-01|432|1879|145|1936|3|1|1|1|1936|145|1879|Wednesday|1936Q1|Y|N|N|2428169|2428168|2427804|2428077|N|N|N|N|N| +2428170|AAAAAAAAKANAFCAA|1936-01-02|432|1879|145|1936|4|1|2|1|1936|145|1879|Thursday|1936Q1|N|N|Y|2428169|2428168|2427805|2428078|N|N|N|N|N| +2428171|AAAAAAAALANAFCAA|1936-01-03|432|1879|145|1936|5|1|3|1|1936|145|1879|Friday|1936Q1|N|Y|N|2428169|2428168|2427806|2428079|N|N|N|N|N| +2428172|AAAAAAAAMANAFCAA|1936-01-04|432|1879|145|1936|6|1|4|1|1936|145|1879|Saturday|1936Q1|N|Y|N|2428169|2428168|2427807|2428080|N|N|N|N|N| +2428173|AAAAAAAANANAFCAA|1936-01-05|432|1879|145|1936|0|1|5|1|1936|145|1879|Sunday|1936Q1|N|N|N|2428169|2428168|2427808|2428081|N|N|N|N|N| +2428174|AAAAAAAAOANAFCAA|1936-01-06|432|1879|145|1936|1|1|6|1|1936|145|1879|Monday|1936Q1|N|N|N|2428169|2428168|2427809|2428082|N|N|N|N|N| +2428175|AAAAAAAAPANAFCAA|1936-01-07|432|1880|145|1936|2|1|7|1|1936|145|1880|Tuesday|1936Q1|N|N|N|2428169|2428168|2427810|2428083|N|N|N|N|N| +2428176|AAAAAAAAABNAFCAA|1936-01-08|432|1880|145|1936|3|1|8|1|1936|145|1880|Wednesday|1936Q1|N|N|N|2428169|2428168|2427811|2428084|N|N|N|N|N| +2428177|AAAAAAAABBNAFCAA|1936-01-09|432|1880|145|1936|4|1|9|1|1936|145|1880|Thursday|1936Q1|N|N|N|2428169|2428168|2427812|2428085|N|N|N|N|N| +2428178|AAAAAAAACBNAFCAA|1936-01-10|432|1880|145|1936|5|1|10|1|1936|145|1880|Friday|1936Q1|N|Y|N|2428169|2428168|2427813|2428086|N|N|N|N|N| +2428179|AAAAAAAADBNAFCAA|1936-01-11|432|1880|145|1936|6|1|11|1|1936|145|1880|Saturday|1936Q1|N|Y|N|2428169|2428168|2427814|2428087|N|N|N|N|N| +2428180|AAAAAAAAEBNAFCAA|1936-01-12|432|1880|145|1936|0|1|12|1|1936|145|1880|Sunday|1936Q1|N|N|N|2428169|2428168|2427815|2428088|N|N|N|N|N| +2428181|AAAAAAAAFBNAFCAA|1936-01-13|432|1880|145|1936|1|1|13|1|1936|145|1880|Monday|1936Q1|N|N|N|2428169|2428168|2427816|2428089|N|N|N|N|N| +2428182|AAAAAAAAGBNAFCAA|1936-01-14|432|1881|145|1936|2|1|14|1|1936|145|1881|Tuesday|1936Q1|N|N|N|2428169|2428168|2427817|2428090|N|N|N|N|N| +2428183|AAAAAAAAHBNAFCAA|1936-01-15|432|1881|145|1936|3|1|15|1|1936|145|1881|Wednesday|1936Q1|N|N|N|2428169|2428168|2427818|2428091|N|N|N|N|N| +2428184|AAAAAAAAIBNAFCAA|1936-01-16|432|1881|145|1936|4|1|16|1|1936|145|1881|Thursday|1936Q1|N|N|N|2428169|2428168|2427819|2428092|N|N|N|N|N| +2428185|AAAAAAAAJBNAFCAA|1936-01-17|432|1881|145|1936|5|1|17|1|1936|145|1881|Friday|1936Q1|N|Y|N|2428169|2428168|2427820|2428093|N|N|N|N|N| +2428186|AAAAAAAAKBNAFCAA|1936-01-18|432|1881|145|1936|6|1|18|1|1936|145|1881|Saturday|1936Q1|N|Y|N|2428169|2428168|2427821|2428094|N|N|N|N|N| +2428187|AAAAAAAALBNAFCAA|1936-01-19|432|1881|145|1936|0|1|19|1|1936|145|1881|Sunday|1936Q1|N|N|N|2428169|2428168|2427822|2428095|N|N|N|N|N| +2428188|AAAAAAAAMBNAFCAA|1936-01-20|432|1881|145|1936|1|1|20|1|1936|145|1881|Monday|1936Q1|N|N|N|2428169|2428168|2427823|2428096|N|N|N|N|N| +2428189|AAAAAAAANBNAFCAA|1936-01-21|432|1882|145|1936|2|1|21|1|1936|145|1882|Tuesday|1936Q1|N|N|N|2428169|2428168|2427824|2428097|N|N|N|N|N| +2428190|AAAAAAAAOBNAFCAA|1936-01-22|432|1882|145|1936|3|1|22|1|1936|145|1882|Wednesday|1936Q1|N|N|N|2428169|2428168|2427825|2428098|N|N|N|N|N| +2428191|AAAAAAAAPBNAFCAA|1936-01-23|432|1882|145|1936|4|1|23|1|1936|145|1882|Thursday|1936Q1|N|N|N|2428169|2428168|2427826|2428099|N|N|N|N|N| +2428192|AAAAAAAAACNAFCAA|1936-01-24|432|1882|145|1936|5|1|24|1|1936|145|1882|Friday|1936Q1|N|Y|N|2428169|2428168|2427827|2428100|N|N|N|N|N| +2428193|AAAAAAAABCNAFCAA|1936-01-25|432|1882|145|1936|6|1|25|1|1936|145|1882|Saturday|1936Q1|N|Y|N|2428169|2428168|2427828|2428101|N|N|N|N|N| +2428194|AAAAAAAACCNAFCAA|1936-01-26|432|1882|145|1936|0|1|26|1|1936|145|1882|Sunday|1936Q1|N|N|N|2428169|2428168|2427829|2428102|N|N|N|N|N| +2428195|AAAAAAAADCNAFCAA|1936-01-27|432|1882|145|1936|1|1|27|1|1936|145|1882|Monday|1936Q1|N|N|N|2428169|2428168|2427830|2428103|N|N|N|N|N| +2428196|AAAAAAAAECNAFCAA|1936-01-28|432|1883|145|1936|2|1|28|1|1936|145|1883|Tuesday|1936Q1|N|N|N|2428169|2428168|2427831|2428104|N|N|N|N|N| +2428197|AAAAAAAAFCNAFCAA|1936-01-29|432|1883|145|1936|3|1|29|1|1936|145|1883|Wednesday|1936Q1|N|N|N|2428169|2428168|2427832|2428105|N|N|N|N|N| +2428198|AAAAAAAAGCNAFCAA|1936-01-30|432|1883|145|1936|4|1|30|1|1936|145|1883|Thursday|1936Q1|N|N|N|2428169|2428168|2427833|2428106|N|N|N|N|N| +2428199|AAAAAAAAHCNAFCAA|1936-01-31|432|1883|145|1936|5|1|31|1|1936|145|1883|Friday|1936Q1|N|Y|N|2428169|2428168|2427834|2428107|N|N|N|N|N| +2428200|AAAAAAAAICNAFCAA|1936-02-01|433|1883|145|1936|6|2|1|1|1936|145|1883|Saturday|1936Q1|N|Y|N|2428200|2428230|2427835|2428108|N|N|N|N|N| +2428201|AAAAAAAAJCNAFCAA|1936-02-02|433|1883|145|1936|0|2|2|1|1936|145|1883|Sunday|1936Q1|N|N|N|2428200|2428230|2427836|2428109|N|N|N|N|N| +2428202|AAAAAAAAKCNAFCAA|1936-02-03|433|1883|145|1936|1|2|3|1|1936|145|1883|Monday|1936Q1|N|N|N|2428200|2428230|2427837|2428110|N|N|N|N|N| +2428203|AAAAAAAALCNAFCAA|1936-02-04|433|1884|145|1936|2|2|4|1|1936|145|1884|Tuesday|1936Q1|N|N|N|2428200|2428230|2427838|2428111|N|N|N|N|N| +2428204|AAAAAAAAMCNAFCAA|1936-02-05|433|1884|145|1936|3|2|5|1|1936|145|1884|Wednesday|1936Q1|N|N|N|2428200|2428230|2427839|2428112|N|N|N|N|N| +2428205|AAAAAAAANCNAFCAA|1936-02-06|433|1884|145|1936|4|2|6|1|1936|145|1884|Thursday|1936Q1|N|N|N|2428200|2428230|2427840|2428113|N|N|N|N|N| +2428206|AAAAAAAAOCNAFCAA|1936-02-07|433|1884|145|1936|5|2|7|1|1936|145|1884|Friday|1936Q1|N|Y|N|2428200|2428230|2427841|2428114|N|N|N|N|N| +2428207|AAAAAAAAPCNAFCAA|1936-02-08|433|1884|145|1936|6|2|8|1|1936|145|1884|Saturday|1936Q1|N|Y|N|2428200|2428230|2427842|2428115|N|N|N|N|N| +2428208|AAAAAAAAADNAFCAA|1936-02-09|433|1884|145|1936|0|2|9|1|1936|145|1884|Sunday|1936Q1|N|N|N|2428200|2428230|2427843|2428116|N|N|N|N|N| +2428209|AAAAAAAABDNAFCAA|1936-02-10|433|1884|145|1936|1|2|10|1|1936|145|1884|Monday|1936Q1|N|N|N|2428200|2428230|2427844|2428117|N|N|N|N|N| +2428210|AAAAAAAACDNAFCAA|1936-02-11|433|1885|145|1936|2|2|11|1|1936|145|1885|Tuesday|1936Q1|N|N|N|2428200|2428230|2427845|2428118|N|N|N|N|N| +2428211|AAAAAAAADDNAFCAA|1936-02-12|433|1885|145|1936|3|2|12|1|1936|145|1885|Wednesday|1936Q1|N|N|N|2428200|2428230|2427846|2428119|N|N|N|N|N| +2428212|AAAAAAAAEDNAFCAA|1936-02-13|433|1885|145|1936|4|2|13|1|1936|145|1885|Thursday|1936Q1|N|N|N|2428200|2428230|2427847|2428120|N|N|N|N|N| +2428213|AAAAAAAAFDNAFCAA|1936-02-14|433|1885|145|1936|5|2|14|1|1936|145|1885|Friday|1936Q1|N|Y|N|2428200|2428230|2427848|2428121|N|N|N|N|N| +2428214|AAAAAAAAGDNAFCAA|1936-02-15|433|1885|145|1936|6|2|15|1|1936|145|1885|Saturday|1936Q1|N|Y|N|2428200|2428230|2427849|2428122|N|N|N|N|N| +2428215|AAAAAAAAHDNAFCAA|1936-02-16|433|1885|145|1936|0|2|16|1|1936|145|1885|Sunday|1936Q1|N|N|N|2428200|2428230|2427850|2428123|N|N|N|N|N| +2428216|AAAAAAAAIDNAFCAA|1936-02-17|433|1885|145|1936|1|2|17|1|1936|145|1885|Monday|1936Q1|N|N|N|2428200|2428230|2427851|2428124|N|N|N|N|N| +2428217|AAAAAAAAJDNAFCAA|1936-02-18|433|1886|145|1936|2|2|18|1|1936|145|1886|Tuesday|1936Q1|N|N|N|2428200|2428230|2427852|2428125|N|N|N|N|N| +2428218|AAAAAAAAKDNAFCAA|1936-02-19|433|1886|145|1936|3|2|19|1|1936|145|1886|Wednesday|1936Q1|N|N|N|2428200|2428230|2427853|2428126|N|N|N|N|N| +2428219|AAAAAAAALDNAFCAA|1936-02-20|433|1886|145|1936|4|2|20|1|1936|145|1886|Thursday|1936Q1|N|N|N|2428200|2428230|2427854|2428127|N|N|N|N|N| +2428220|AAAAAAAAMDNAFCAA|1936-02-21|433|1886|145|1936|5|2|21|1|1936|145|1886|Friday|1936Q1|N|Y|N|2428200|2428230|2427855|2428128|N|N|N|N|N| +2428221|AAAAAAAANDNAFCAA|1936-02-22|433|1886|145|1936|6|2|22|1|1936|145|1886|Saturday|1936Q1|N|Y|N|2428200|2428230|2427856|2428129|N|N|N|N|N| +2428222|AAAAAAAAODNAFCAA|1936-02-23|433|1886|145|1936|0|2|23|1|1936|145|1886|Sunday|1936Q1|N|N|N|2428200|2428230|2427857|2428130|N|N|N|N|N| +2428223|AAAAAAAAPDNAFCAA|1936-02-24|433|1886|145|1936|1|2|24|1|1936|145|1886|Monday|1936Q1|N|N|N|2428200|2428230|2427858|2428131|N|N|N|N|N| +2428224|AAAAAAAAAENAFCAA|1936-02-25|433|1887|145|1936|2|2|25|1|1936|145|1887|Tuesday|1936Q1|N|N|N|2428200|2428230|2427859|2428132|N|N|N|N|N| +2428225|AAAAAAAABENAFCAA|1936-02-26|433|1887|145|1936|3|2|26|1|1936|145|1887|Wednesday|1936Q1|N|N|N|2428200|2428230|2427860|2428133|N|N|N|N|N| +2428226|AAAAAAAACENAFCAA|1936-02-27|433|1887|145|1936|4|2|27|1|1936|145|1887|Thursday|1936Q1|N|N|N|2428200|2428230|2427861|2428134|N|N|N|N|N| +2428227|AAAAAAAADENAFCAA|1936-02-28|433|1887|145|1936|5|2|28|1|1936|145|1887|Friday|1936Q1|N|Y|N|2428200|2428230|2427862|2428135|N|N|N|N|N| +2428228|AAAAAAAAEENAFCAA|1936-02-29|433|1887|145|1936|6|2|29|1|1936|145|1887|Saturday|1936Q1|N|Y|N|2428200|2428230|2427862|2428136|N|N|N|N|N| +2428229|AAAAAAAAFENAFCAA|1936-03-01|434|1887|146|1936|0|3|1|1|1936|146|1887|Sunday|1936Q1|N|N|N|2428229|2428288|2427863|2428137|N|N|N|N|N| +2428230|AAAAAAAAGENAFCAA|1936-03-02|434|1887|146|1936|1|3|2|1|1936|146|1887|Monday|1936Q1|N|N|N|2428229|2428288|2427864|2428138|N|N|N|N|N| +2428231|AAAAAAAAHENAFCAA|1936-03-03|434|1888|146|1936|2|3|3|1|1936|146|1888|Tuesday|1936Q1|N|N|N|2428229|2428288|2427865|2428139|N|N|N|N|N| +2428232|AAAAAAAAIENAFCAA|1936-03-04|434|1888|146|1936|3|3|4|1|1936|146|1888|Wednesday|1936Q1|N|N|N|2428229|2428288|2427866|2428140|N|N|N|N|N| +2428233|AAAAAAAAJENAFCAA|1936-03-05|434|1888|146|1936|4|3|5|1|1936|146|1888|Thursday|1936Q1|N|N|N|2428229|2428288|2427867|2428141|N|N|N|N|N| +2428234|AAAAAAAAKENAFCAA|1936-03-06|434|1888|146|1936|5|3|6|1|1936|146|1888|Friday|1936Q1|N|Y|N|2428229|2428288|2427868|2428142|N|N|N|N|N| +2428235|AAAAAAAALENAFCAA|1936-03-07|434|1888|146|1936|6|3|7|1|1936|146|1888|Saturday|1936Q1|N|Y|N|2428229|2428288|2427869|2428143|N|N|N|N|N| +2428236|AAAAAAAAMENAFCAA|1936-03-08|434|1888|146|1936|0|3|8|1|1936|146|1888|Sunday|1936Q1|N|N|N|2428229|2428288|2427870|2428144|N|N|N|N|N| +2428237|AAAAAAAANENAFCAA|1936-03-09|434|1888|146|1936|1|3|9|1|1936|146|1888|Monday|1936Q1|N|N|N|2428229|2428288|2427871|2428145|N|N|N|N|N| +2428238|AAAAAAAAOENAFCAA|1936-03-10|434|1889|146|1936|2|3|10|1|1936|146|1889|Tuesday|1936Q1|N|N|N|2428229|2428288|2427872|2428146|N|N|N|N|N| +2428239|AAAAAAAAPENAFCAA|1936-03-11|434|1889|146|1936|3|3|11|1|1936|146|1889|Wednesday|1936Q1|N|N|N|2428229|2428288|2427873|2428147|N|N|N|N|N| +2428240|AAAAAAAAAFNAFCAA|1936-03-12|434|1889|146|1936|4|3|12|1|1936|146|1889|Thursday|1936Q1|N|N|N|2428229|2428288|2427874|2428148|N|N|N|N|N| +2428241|AAAAAAAABFNAFCAA|1936-03-13|434|1889|146|1936|5|3|13|1|1936|146|1889|Friday|1936Q1|N|Y|N|2428229|2428288|2427875|2428149|N|N|N|N|N| +2428242|AAAAAAAACFNAFCAA|1936-03-14|434|1889|146|1936|6|3|14|1|1936|146|1889|Saturday|1936Q1|N|Y|N|2428229|2428288|2427876|2428150|N|N|N|N|N| +2428243|AAAAAAAADFNAFCAA|1936-03-15|434|1889|146|1936|0|3|15|1|1936|146|1889|Sunday|1936Q1|N|N|N|2428229|2428288|2427877|2428151|N|N|N|N|N| +2428244|AAAAAAAAEFNAFCAA|1936-03-16|434|1889|146|1936|1|3|16|1|1936|146|1889|Monday|1936Q1|N|N|N|2428229|2428288|2427878|2428152|N|N|N|N|N| +2428245|AAAAAAAAFFNAFCAA|1936-03-17|434|1890|146|1936|2|3|17|1|1936|146|1890|Tuesday|1936Q1|N|N|N|2428229|2428288|2427879|2428153|N|N|N|N|N| +2428246|AAAAAAAAGFNAFCAA|1936-03-18|434|1890|146|1936|3|3|18|1|1936|146|1890|Wednesday|1936Q1|N|N|N|2428229|2428288|2427880|2428154|N|N|N|N|N| +2428247|AAAAAAAAHFNAFCAA|1936-03-19|434|1890|146|1936|4|3|19|1|1936|146|1890|Thursday|1936Q1|N|N|N|2428229|2428288|2427881|2428155|N|N|N|N|N| +2428248|AAAAAAAAIFNAFCAA|1936-03-20|434|1890|146|1936|5|3|20|1|1936|146|1890|Friday|1936Q1|N|Y|N|2428229|2428288|2427882|2428156|N|N|N|N|N| +2428249|AAAAAAAAJFNAFCAA|1936-03-21|434|1890|146|1936|6|3|21|1|1936|146|1890|Saturday|1936Q1|N|Y|N|2428229|2428288|2427883|2428157|N|N|N|N|N| +2428250|AAAAAAAAKFNAFCAA|1936-03-22|434|1890|146|1936|0|3|22|1|1936|146|1890|Sunday|1936Q1|N|N|N|2428229|2428288|2427884|2428158|N|N|N|N|N| +2428251|AAAAAAAALFNAFCAA|1936-03-23|434|1890|146|1936|1|3|23|1|1936|146|1890|Monday|1936Q1|N|N|N|2428229|2428288|2427885|2428159|N|N|N|N|N| +2428252|AAAAAAAAMFNAFCAA|1936-03-24|434|1891|146|1936|2|3|24|1|1936|146|1891|Tuesday|1936Q1|N|N|N|2428229|2428288|2427886|2428160|N|N|N|N|N| +2428253|AAAAAAAANFNAFCAA|1936-03-25|434|1891|146|1936|3|3|25|1|1936|146|1891|Wednesday|1936Q1|N|N|N|2428229|2428288|2427887|2428161|N|N|N|N|N| +2428254|AAAAAAAAOFNAFCAA|1936-03-26|434|1891|146|1936|4|3|26|1|1936|146|1891|Thursday|1936Q1|N|N|N|2428229|2428288|2427888|2428162|N|N|N|N|N| +2428255|AAAAAAAAPFNAFCAA|1936-03-27|434|1891|146|1936|5|3|27|1|1936|146|1891|Friday|1936Q1|N|Y|N|2428229|2428288|2427889|2428163|N|N|N|N|N| +2428256|AAAAAAAAAGNAFCAA|1936-03-28|434|1891|146|1936|6|3|28|1|1936|146|1891|Saturday|1936Q1|N|Y|N|2428229|2428288|2427890|2428164|N|N|N|N|N| +2428257|AAAAAAAABGNAFCAA|1936-03-29|434|1891|146|1936|0|3|29|1|1936|146|1891|Sunday|1936Q1|N|N|N|2428229|2428288|2427891|2428165|N|N|N|N|N| +2428258|AAAAAAAACGNAFCAA|1936-03-30|434|1891|146|1936|1|3|30|1|1936|146|1891|Monday|1936Q1|N|N|N|2428229|2428288|2427892|2428166|N|N|N|N|N| +2428259|AAAAAAAADGNAFCAA|1936-03-31|434|1892|146|1936|2|3|31|1|1936|146|1892|Tuesday|1936Q1|N|N|N|2428229|2428288|2427893|2428167|N|N|N|N|N| +2428260|AAAAAAAAEGNAFCAA|1936-04-01|435|1892|146|1936|3|4|1|2|1936|146|1892|Wednesday|1936Q2|N|N|N|2428260|2428350|2427894|2428169|N|N|N|N|N| +2428261|AAAAAAAAFGNAFCAA|1936-04-02|435|1892|146|1936|4|4|2|2|1936|146|1892|Thursday|1936Q2|N|N|N|2428260|2428350|2427895|2428170|N|N|N|N|N| +2428262|AAAAAAAAGGNAFCAA|1936-04-03|435|1892|146|1936|5|4|3|2|1936|146|1892|Friday|1936Q2|N|Y|N|2428260|2428350|2427896|2428171|N|N|N|N|N| +2428263|AAAAAAAAHGNAFCAA|1936-04-04|435|1892|146|1936|6|4|4|2|1936|146|1892|Saturday|1936Q2|N|Y|N|2428260|2428350|2427897|2428172|N|N|N|N|N| +2428264|AAAAAAAAIGNAFCAA|1936-04-05|435|1892|146|1936|0|4|5|2|1936|146|1892|Sunday|1936Q2|N|N|N|2428260|2428350|2427898|2428173|N|N|N|N|N| +2428265|AAAAAAAAJGNAFCAA|1936-04-06|435|1892|146|1936|1|4|6|2|1936|146|1892|Monday|1936Q2|N|N|N|2428260|2428350|2427899|2428174|N|N|N|N|N| +2428266|AAAAAAAAKGNAFCAA|1936-04-07|435|1893|146|1936|2|4|7|2|1936|146|1893|Tuesday|1936Q2|N|N|N|2428260|2428350|2427900|2428175|N|N|N|N|N| +2428267|AAAAAAAALGNAFCAA|1936-04-08|435|1893|146|1936|3|4|8|2|1936|146|1893|Wednesday|1936Q2|N|N|N|2428260|2428350|2427901|2428176|N|N|N|N|N| +2428268|AAAAAAAAMGNAFCAA|1936-04-09|435|1893|146|1936|4|4|9|2|1936|146|1893|Thursday|1936Q2|N|N|N|2428260|2428350|2427902|2428177|N|N|N|N|N| +2428269|AAAAAAAANGNAFCAA|1936-04-10|435|1893|146|1936|5|4|10|2|1936|146|1893|Friday|1936Q2|N|Y|N|2428260|2428350|2427903|2428178|N|N|N|N|N| +2428270|AAAAAAAAOGNAFCAA|1936-04-11|435|1893|146|1936|6|4|11|2|1936|146|1893|Saturday|1936Q2|N|Y|N|2428260|2428350|2427904|2428179|N|N|N|N|N| +2428271|AAAAAAAAPGNAFCAA|1936-04-12|435|1893|146|1936|0|4|12|2|1936|146|1893|Sunday|1936Q2|N|N|N|2428260|2428350|2427905|2428180|N|N|N|N|N| +2428272|AAAAAAAAAHNAFCAA|1936-04-13|435|1893|146|1936|1|4|13|2|1936|146|1893|Monday|1936Q2|N|N|N|2428260|2428350|2427906|2428181|N|N|N|N|N| +2428273|AAAAAAAABHNAFCAA|1936-04-14|435|1894|146|1936|2|4|14|2|1936|146|1894|Tuesday|1936Q2|N|N|N|2428260|2428350|2427907|2428182|N|N|N|N|N| +2428274|AAAAAAAACHNAFCAA|1936-04-15|435|1894|146|1936|3|4|15|2|1936|146|1894|Wednesday|1936Q2|N|N|N|2428260|2428350|2427908|2428183|N|N|N|N|N| +2428275|AAAAAAAADHNAFCAA|1936-04-16|435|1894|146|1936|4|4|16|2|1936|146|1894|Thursday|1936Q2|N|N|N|2428260|2428350|2427909|2428184|N|N|N|N|N| +2428276|AAAAAAAAEHNAFCAA|1936-04-17|435|1894|146|1936|5|4|17|2|1936|146|1894|Friday|1936Q2|N|Y|N|2428260|2428350|2427910|2428185|N|N|N|N|N| +2428277|AAAAAAAAFHNAFCAA|1936-04-18|435|1894|146|1936|6|4|18|2|1936|146|1894|Saturday|1936Q2|N|Y|N|2428260|2428350|2427911|2428186|N|N|N|N|N| +2428278|AAAAAAAAGHNAFCAA|1936-04-19|435|1894|146|1936|0|4|19|2|1936|146|1894|Sunday|1936Q2|N|N|N|2428260|2428350|2427912|2428187|N|N|N|N|N| +2428279|AAAAAAAAHHNAFCAA|1936-04-20|435|1894|146|1936|1|4|20|2|1936|146|1894|Monday|1936Q2|N|N|N|2428260|2428350|2427913|2428188|N|N|N|N|N| +2428280|AAAAAAAAIHNAFCAA|1936-04-21|435|1895|146|1936|2|4|21|2|1936|146|1895|Tuesday|1936Q2|N|N|N|2428260|2428350|2427914|2428189|N|N|N|N|N| +2428281|AAAAAAAAJHNAFCAA|1936-04-22|435|1895|146|1936|3|4|22|2|1936|146|1895|Wednesday|1936Q2|N|N|N|2428260|2428350|2427915|2428190|N|N|N|N|N| +2428282|AAAAAAAAKHNAFCAA|1936-04-23|435|1895|146|1936|4|4|23|2|1936|146|1895|Thursday|1936Q2|N|N|N|2428260|2428350|2427916|2428191|N|N|N|N|N| +2428283|AAAAAAAALHNAFCAA|1936-04-24|435|1895|146|1936|5|4|24|2|1936|146|1895|Friday|1936Q2|N|Y|N|2428260|2428350|2427917|2428192|N|N|N|N|N| +2428284|AAAAAAAAMHNAFCAA|1936-04-25|435|1895|146|1936|6|4|25|2|1936|146|1895|Saturday|1936Q2|N|Y|N|2428260|2428350|2427918|2428193|N|N|N|N|N| +2428285|AAAAAAAANHNAFCAA|1936-04-26|435|1895|146|1936|0|4|26|2|1936|146|1895|Sunday|1936Q2|N|N|N|2428260|2428350|2427919|2428194|N|N|N|N|N| +2428286|AAAAAAAAOHNAFCAA|1936-04-27|435|1895|146|1936|1|4|27|2|1936|146|1895|Monday|1936Q2|N|N|N|2428260|2428350|2427920|2428195|N|N|N|N|N| +2428287|AAAAAAAAPHNAFCAA|1936-04-28|435|1896|146|1936|2|4|28|2|1936|146|1896|Tuesday|1936Q2|N|N|N|2428260|2428350|2427921|2428196|N|N|N|N|N| +2428288|AAAAAAAAAINAFCAA|1936-04-29|435|1896|146|1936|3|4|29|2|1936|146|1896|Wednesday|1936Q2|N|N|N|2428260|2428350|2427922|2428197|N|N|N|N|N| +2428289|AAAAAAAABINAFCAA|1936-04-30|435|1896|146|1936|4|4|30|2|1936|146|1896|Thursday|1936Q2|N|N|N|2428260|2428350|2427923|2428198|N|N|N|N|N| +2428290|AAAAAAAACINAFCAA|1936-05-01|436|1896|146|1936|5|5|1|2|1936|146|1896|Friday|1936Q2|N|Y|N|2428290|2428410|2427924|2428199|N|N|N|N|N| +2428291|AAAAAAAADINAFCAA|1936-05-02|436|1896|146|1936|6|5|2|2|1936|146|1896|Saturday|1936Q2|N|Y|N|2428290|2428410|2427925|2428200|N|N|N|N|N| +2428292|AAAAAAAAEINAFCAA|1936-05-03|436|1896|146|1936|0|5|3|2|1936|146|1896|Sunday|1936Q2|N|N|N|2428290|2428410|2427926|2428201|N|N|N|N|N| +2428293|AAAAAAAAFINAFCAA|1936-05-04|436|1896|146|1936|1|5|4|2|1936|146|1896|Monday|1936Q2|N|N|N|2428290|2428410|2427927|2428202|N|N|N|N|N| +2428294|AAAAAAAAGINAFCAA|1936-05-05|436|1897|146|1936|2|5|5|2|1936|146|1897|Tuesday|1936Q2|N|N|N|2428290|2428410|2427928|2428203|N|N|N|N|N| +2428295|AAAAAAAAHINAFCAA|1936-05-06|436|1897|146|1936|3|5|6|2|1936|146|1897|Wednesday|1936Q2|N|N|N|2428290|2428410|2427929|2428204|N|N|N|N|N| +2428296|AAAAAAAAIINAFCAA|1936-05-07|436|1897|146|1936|4|5|7|2|1936|146|1897|Thursday|1936Q2|N|N|N|2428290|2428410|2427930|2428205|N|N|N|N|N| +2428297|AAAAAAAAJINAFCAA|1936-05-08|436|1897|146|1936|5|5|8|2|1936|146|1897|Friday|1936Q2|N|Y|N|2428290|2428410|2427931|2428206|N|N|N|N|N| +2428298|AAAAAAAAKINAFCAA|1936-05-09|436|1897|146|1936|6|5|9|2|1936|146|1897|Saturday|1936Q2|N|Y|N|2428290|2428410|2427932|2428207|N|N|N|N|N| +2428299|AAAAAAAALINAFCAA|1936-05-10|436|1897|146|1936|0|5|10|2|1936|146|1897|Sunday|1936Q2|N|N|N|2428290|2428410|2427933|2428208|N|N|N|N|N| +2428300|AAAAAAAAMINAFCAA|1936-05-11|436|1897|146|1936|1|5|11|2|1936|146|1897|Monday|1936Q2|N|N|N|2428290|2428410|2427934|2428209|N|N|N|N|N| +2428301|AAAAAAAANINAFCAA|1936-05-12|436|1898|146|1936|2|5|12|2|1936|146|1898|Tuesday|1936Q2|N|N|N|2428290|2428410|2427935|2428210|N|N|N|N|N| +2428302|AAAAAAAAOINAFCAA|1936-05-13|436|1898|146|1936|3|5|13|2|1936|146|1898|Wednesday|1936Q2|N|N|N|2428290|2428410|2427936|2428211|N|N|N|N|N| +2428303|AAAAAAAAPINAFCAA|1936-05-14|436|1898|146|1936|4|5|14|2|1936|146|1898|Thursday|1936Q2|N|N|N|2428290|2428410|2427937|2428212|N|N|N|N|N| +2428304|AAAAAAAAAJNAFCAA|1936-05-15|436|1898|146|1936|5|5|15|2|1936|146|1898|Friday|1936Q2|N|Y|N|2428290|2428410|2427938|2428213|N|N|N|N|N| +2428305|AAAAAAAABJNAFCAA|1936-05-16|436|1898|146|1936|6|5|16|2|1936|146|1898|Saturday|1936Q2|N|Y|N|2428290|2428410|2427939|2428214|N|N|N|N|N| +2428306|AAAAAAAACJNAFCAA|1936-05-17|436|1898|146|1936|0|5|17|2|1936|146|1898|Sunday|1936Q2|N|N|N|2428290|2428410|2427940|2428215|N|N|N|N|N| +2428307|AAAAAAAADJNAFCAA|1936-05-18|436|1898|146|1936|1|5|18|2|1936|146|1898|Monday|1936Q2|N|N|N|2428290|2428410|2427941|2428216|N|N|N|N|N| +2428308|AAAAAAAAEJNAFCAA|1936-05-19|436|1899|146|1936|2|5|19|2|1936|146|1899|Tuesday|1936Q2|N|N|N|2428290|2428410|2427942|2428217|N|N|N|N|N| +2428309|AAAAAAAAFJNAFCAA|1936-05-20|436|1899|146|1936|3|5|20|2|1936|146|1899|Wednesday|1936Q2|N|N|N|2428290|2428410|2427943|2428218|N|N|N|N|N| +2428310|AAAAAAAAGJNAFCAA|1936-05-21|436|1899|146|1936|4|5|21|2|1936|146|1899|Thursday|1936Q2|N|N|N|2428290|2428410|2427944|2428219|N|N|N|N|N| +2428311|AAAAAAAAHJNAFCAA|1936-05-22|436|1899|146|1936|5|5|22|2|1936|146|1899|Friday|1936Q2|N|Y|N|2428290|2428410|2427945|2428220|N|N|N|N|N| +2428312|AAAAAAAAIJNAFCAA|1936-05-23|436|1899|146|1936|6|5|23|2|1936|146|1899|Saturday|1936Q2|N|Y|N|2428290|2428410|2427946|2428221|N|N|N|N|N| +2428313|AAAAAAAAJJNAFCAA|1936-05-24|436|1899|146|1936|0|5|24|2|1936|146|1899|Sunday|1936Q2|N|N|N|2428290|2428410|2427947|2428222|N|N|N|N|N| +2428314|AAAAAAAAKJNAFCAA|1936-05-25|436|1899|146|1936|1|5|25|2|1936|146|1899|Monday|1936Q2|N|N|N|2428290|2428410|2427948|2428223|N|N|N|N|N| +2428315|AAAAAAAALJNAFCAA|1936-05-26|436|1900|146|1936|2|5|26|2|1936|146|1900|Tuesday|1936Q2|N|N|N|2428290|2428410|2427949|2428224|N|N|N|N|N| +2428316|AAAAAAAAMJNAFCAA|1936-05-27|436|1900|146|1936|3|5|27|2|1936|146|1900|Wednesday|1936Q2|N|N|N|2428290|2428410|2427950|2428225|N|N|N|N|N| +2428317|AAAAAAAANJNAFCAA|1936-05-28|436|1900|146|1936|4|5|28|2|1936|146|1900|Thursday|1936Q2|N|N|N|2428290|2428410|2427951|2428226|N|N|N|N|N| +2428318|AAAAAAAAOJNAFCAA|1936-05-29|436|1900|146|1936|5|5|29|2|1936|146|1900|Friday|1936Q2|N|Y|N|2428290|2428410|2427952|2428227|N|N|N|N|N| +2428319|AAAAAAAAPJNAFCAA|1936-05-30|436|1900|146|1936|6|5|30|2|1936|146|1900|Saturday|1936Q2|N|Y|N|2428290|2428410|2427953|2428228|N|N|N|N|N| +2428320|AAAAAAAAAKNAFCAA|1936-05-31|436|1900|146|1936|0|5|31|2|1936|146|1900|Sunday|1936Q2|N|N|N|2428290|2428410|2427954|2428229|N|N|N|N|N| +2428321|AAAAAAAABKNAFCAA|1936-06-01|437|1900|147|1936|1|6|1|2|1936|147|1900|Monday|1936Q2|N|N|N|2428321|2428472|2427955|2428230|N|N|N|N|N| +2428322|AAAAAAAACKNAFCAA|1936-06-02|437|1901|147|1936|2|6|2|2|1936|147|1901|Tuesday|1936Q2|N|N|N|2428321|2428472|2427956|2428231|N|N|N|N|N| +2428323|AAAAAAAADKNAFCAA|1936-06-03|437|1901|147|1936|3|6|3|2|1936|147|1901|Wednesday|1936Q2|N|N|N|2428321|2428472|2427957|2428232|N|N|N|N|N| +2428324|AAAAAAAAEKNAFCAA|1936-06-04|437|1901|147|1936|4|6|4|2|1936|147|1901|Thursday|1936Q2|N|N|N|2428321|2428472|2427958|2428233|N|N|N|N|N| +2428325|AAAAAAAAFKNAFCAA|1936-06-05|437|1901|147|1936|5|6|5|2|1936|147|1901|Friday|1936Q2|N|Y|N|2428321|2428472|2427959|2428234|N|N|N|N|N| +2428326|AAAAAAAAGKNAFCAA|1936-06-06|437|1901|147|1936|6|6|6|2|1936|147|1901|Saturday|1936Q2|N|Y|N|2428321|2428472|2427960|2428235|N|N|N|N|N| +2428327|AAAAAAAAHKNAFCAA|1936-06-07|437|1901|147|1936|0|6|7|2|1936|147|1901|Sunday|1936Q2|N|N|N|2428321|2428472|2427961|2428236|N|N|N|N|N| +2428328|AAAAAAAAIKNAFCAA|1936-06-08|437|1901|147|1936|1|6|8|2|1936|147|1901|Monday|1936Q2|N|N|N|2428321|2428472|2427962|2428237|N|N|N|N|N| +2428329|AAAAAAAAJKNAFCAA|1936-06-09|437|1902|147|1936|2|6|9|2|1936|147|1902|Tuesday|1936Q2|N|N|N|2428321|2428472|2427963|2428238|N|N|N|N|N| +2428330|AAAAAAAAKKNAFCAA|1936-06-10|437|1902|147|1936|3|6|10|2|1936|147|1902|Wednesday|1936Q2|N|N|N|2428321|2428472|2427964|2428239|N|N|N|N|N| +2428331|AAAAAAAALKNAFCAA|1936-06-11|437|1902|147|1936|4|6|11|2|1936|147|1902|Thursday|1936Q2|N|N|N|2428321|2428472|2427965|2428240|N|N|N|N|N| +2428332|AAAAAAAAMKNAFCAA|1936-06-12|437|1902|147|1936|5|6|12|2|1936|147|1902|Friday|1936Q2|N|Y|N|2428321|2428472|2427966|2428241|N|N|N|N|N| +2428333|AAAAAAAANKNAFCAA|1936-06-13|437|1902|147|1936|6|6|13|2|1936|147|1902|Saturday|1936Q2|N|Y|N|2428321|2428472|2427967|2428242|N|N|N|N|N| +2428334|AAAAAAAAOKNAFCAA|1936-06-14|437|1902|147|1936|0|6|14|2|1936|147|1902|Sunday|1936Q2|N|N|N|2428321|2428472|2427968|2428243|N|N|N|N|N| +2428335|AAAAAAAAPKNAFCAA|1936-06-15|437|1902|147|1936|1|6|15|2|1936|147|1902|Monday|1936Q2|N|N|N|2428321|2428472|2427969|2428244|N|N|N|N|N| +2428336|AAAAAAAAALNAFCAA|1936-06-16|437|1903|147|1936|2|6|16|2|1936|147|1903|Tuesday|1936Q2|N|N|N|2428321|2428472|2427970|2428245|N|N|N|N|N| +2428337|AAAAAAAABLNAFCAA|1936-06-17|437|1903|147|1936|3|6|17|2|1936|147|1903|Wednesday|1936Q2|N|N|N|2428321|2428472|2427971|2428246|N|N|N|N|N| +2428338|AAAAAAAACLNAFCAA|1936-06-18|437|1903|147|1936|4|6|18|2|1936|147|1903|Thursday|1936Q2|N|N|N|2428321|2428472|2427972|2428247|N|N|N|N|N| +2428339|AAAAAAAADLNAFCAA|1936-06-19|437|1903|147|1936|5|6|19|2|1936|147|1903|Friday|1936Q2|N|Y|N|2428321|2428472|2427973|2428248|N|N|N|N|N| +2428340|AAAAAAAAELNAFCAA|1936-06-20|437|1903|147|1936|6|6|20|2|1936|147|1903|Saturday|1936Q2|N|Y|N|2428321|2428472|2427974|2428249|N|N|N|N|N| +2428341|AAAAAAAAFLNAFCAA|1936-06-21|437|1903|147|1936|0|6|21|2|1936|147|1903|Sunday|1936Q2|N|N|N|2428321|2428472|2427975|2428250|N|N|N|N|N| +2428342|AAAAAAAAGLNAFCAA|1936-06-22|437|1903|147|1936|1|6|22|2|1936|147|1903|Monday|1936Q2|N|N|N|2428321|2428472|2427976|2428251|N|N|N|N|N| +2428343|AAAAAAAAHLNAFCAA|1936-06-23|437|1904|147|1936|2|6|23|2|1936|147|1904|Tuesday|1936Q2|N|N|N|2428321|2428472|2427977|2428252|N|N|N|N|N| +2428344|AAAAAAAAILNAFCAA|1936-06-24|437|1904|147|1936|3|6|24|2|1936|147|1904|Wednesday|1936Q2|N|N|N|2428321|2428472|2427978|2428253|N|N|N|N|N| +2428345|AAAAAAAAJLNAFCAA|1936-06-25|437|1904|147|1936|4|6|25|2|1936|147|1904|Thursday|1936Q2|N|N|N|2428321|2428472|2427979|2428254|N|N|N|N|N| +2428346|AAAAAAAAKLNAFCAA|1936-06-26|437|1904|147|1936|5|6|26|2|1936|147|1904|Friday|1936Q2|N|Y|N|2428321|2428472|2427980|2428255|N|N|N|N|N| +2428347|AAAAAAAALLNAFCAA|1936-06-27|437|1904|147|1936|6|6|27|2|1936|147|1904|Saturday|1936Q2|N|Y|N|2428321|2428472|2427981|2428256|N|N|N|N|N| +2428348|AAAAAAAAMLNAFCAA|1936-06-28|437|1904|147|1936|0|6|28|2|1936|147|1904|Sunday|1936Q2|N|N|N|2428321|2428472|2427982|2428257|N|N|N|N|N| +2428349|AAAAAAAANLNAFCAA|1936-06-29|437|1904|147|1936|1|6|29|2|1936|147|1904|Monday|1936Q2|N|N|N|2428321|2428472|2427983|2428258|N|N|N|N|N| +2428350|AAAAAAAAOLNAFCAA|1936-06-30|437|1905|147|1936|2|6|30|2|1936|147|1905|Tuesday|1936Q2|N|N|N|2428321|2428472|2427984|2428259|N|N|N|N|N| +2428351|AAAAAAAAPLNAFCAA|1936-07-01|438|1905|147|1936|3|7|1|3|1936|147|1905|Wednesday|1936Q3|N|N|N|2428351|2428532|2427985|2428260|N|N|N|N|N| +2428352|AAAAAAAAAMNAFCAA|1936-07-02|438|1905|147|1936|4|7|2|3|1936|147|1905|Thursday|1936Q3|N|N|N|2428351|2428532|2427986|2428261|N|N|N|N|N| +2428353|AAAAAAAABMNAFCAA|1936-07-03|438|1905|147|1936|5|7|3|3|1936|147|1905|Friday|1936Q3|N|Y|N|2428351|2428532|2427987|2428262|N|N|N|N|N| +2428354|AAAAAAAACMNAFCAA|1936-07-04|438|1905|147|1936|6|7|4|3|1936|147|1905|Saturday|1936Q3|Y|Y|N|2428351|2428532|2427988|2428263|N|N|N|N|N| +2428355|AAAAAAAADMNAFCAA|1936-07-05|438|1905|147|1936|0|7|5|3|1936|147|1905|Sunday|1936Q3|N|N|Y|2428351|2428532|2427989|2428264|N|N|N|N|N| +2428356|AAAAAAAAEMNAFCAA|1936-07-06|438|1905|147|1936|1|7|6|3|1936|147|1905|Monday|1936Q3|N|N|N|2428351|2428532|2427990|2428265|N|N|N|N|N| +2428357|AAAAAAAAFMNAFCAA|1936-07-07|438|1906|147|1936|2|7|7|3|1936|147|1906|Tuesday|1936Q3|N|N|N|2428351|2428532|2427991|2428266|N|N|N|N|N| +2428358|AAAAAAAAGMNAFCAA|1936-07-08|438|1906|147|1936|3|7|8|3|1936|147|1906|Wednesday|1936Q3|N|N|N|2428351|2428532|2427992|2428267|N|N|N|N|N| +2428359|AAAAAAAAHMNAFCAA|1936-07-09|438|1906|147|1936|4|7|9|3|1936|147|1906|Thursday|1936Q3|N|N|N|2428351|2428532|2427993|2428268|N|N|N|N|N| +2428360|AAAAAAAAIMNAFCAA|1936-07-10|438|1906|147|1936|5|7|10|3|1936|147|1906|Friday|1936Q3|N|Y|N|2428351|2428532|2427994|2428269|N|N|N|N|N| +2428361|AAAAAAAAJMNAFCAA|1936-07-11|438|1906|147|1936|6|7|11|3|1936|147|1906|Saturday|1936Q3|N|Y|N|2428351|2428532|2427995|2428270|N|N|N|N|N| +2428362|AAAAAAAAKMNAFCAA|1936-07-12|438|1906|147|1936|0|7|12|3|1936|147|1906|Sunday|1936Q3|N|N|N|2428351|2428532|2427996|2428271|N|N|N|N|N| +2428363|AAAAAAAALMNAFCAA|1936-07-13|438|1906|147|1936|1|7|13|3|1936|147|1906|Monday|1936Q3|N|N|N|2428351|2428532|2427997|2428272|N|N|N|N|N| +2428364|AAAAAAAAMMNAFCAA|1936-07-14|438|1907|147|1936|2|7|14|3|1936|147|1907|Tuesday|1936Q3|N|N|N|2428351|2428532|2427998|2428273|N|N|N|N|N| +2428365|AAAAAAAANMNAFCAA|1936-07-15|438|1907|147|1936|3|7|15|3|1936|147|1907|Wednesday|1936Q3|N|N|N|2428351|2428532|2427999|2428274|N|N|N|N|N| +2428366|AAAAAAAAOMNAFCAA|1936-07-16|438|1907|147|1936|4|7|16|3|1936|147|1907|Thursday|1936Q3|N|N|N|2428351|2428532|2428000|2428275|N|N|N|N|N| +2428367|AAAAAAAAPMNAFCAA|1936-07-17|438|1907|147|1936|5|7|17|3|1936|147|1907|Friday|1936Q3|N|Y|N|2428351|2428532|2428001|2428276|N|N|N|N|N| +2428368|AAAAAAAAANNAFCAA|1936-07-18|438|1907|147|1936|6|7|18|3|1936|147|1907|Saturday|1936Q3|N|Y|N|2428351|2428532|2428002|2428277|N|N|N|N|N| +2428369|AAAAAAAABNNAFCAA|1936-07-19|438|1907|147|1936|0|7|19|3|1936|147|1907|Sunday|1936Q3|N|N|N|2428351|2428532|2428003|2428278|N|N|N|N|N| +2428370|AAAAAAAACNNAFCAA|1936-07-20|438|1907|147|1936|1|7|20|3|1936|147|1907|Monday|1936Q3|N|N|N|2428351|2428532|2428004|2428279|N|N|N|N|N| +2428371|AAAAAAAADNNAFCAA|1936-07-21|438|1908|147|1936|2|7|21|3|1936|147|1908|Tuesday|1936Q3|N|N|N|2428351|2428532|2428005|2428280|N|N|N|N|N| +2428372|AAAAAAAAENNAFCAA|1936-07-22|438|1908|147|1936|3|7|22|3|1936|147|1908|Wednesday|1936Q3|N|N|N|2428351|2428532|2428006|2428281|N|N|N|N|N| +2428373|AAAAAAAAFNNAFCAA|1936-07-23|438|1908|147|1936|4|7|23|3|1936|147|1908|Thursday|1936Q3|N|N|N|2428351|2428532|2428007|2428282|N|N|N|N|N| +2428374|AAAAAAAAGNNAFCAA|1936-07-24|438|1908|147|1936|5|7|24|3|1936|147|1908|Friday|1936Q3|N|Y|N|2428351|2428532|2428008|2428283|N|N|N|N|N| +2428375|AAAAAAAAHNNAFCAA|1936-07-25|438|1908|147|1936|6|7|25|3|1936|147|1908|Saturday|1936Q3|N|Y|N|2428351|2428532|2428009|2428284|N|N|N|N|N| +2428376|AAAAAAAAINNAFCAA|1936-07-26|438|1908|147|1936|0|7|26|3|1936|147|1908|Sunday|1936Q3|N|N|N|2428351|2428532|2428010|2428285|N|N|N|N|N| +2428377|AAAAAAAAJNNAFCAA|1936-07-27|438|1908|147|1936|1|7|27|3|1936|147|1908|Monday|1936Q3|N|N|N|2428351|2428532|2428011|2428286|N|N|N|N|N| +2428378|AAAAAAAAKNNAFCAA|1936-07-28|438|1909|147|1936|2|7|28|3|1936|147|1909|Tuesday|1936Q3|N|N|N|2428351|2428532|2428012|2428287|N|N|N|N|N| +2428379|AAAAAAAALNNAFCAA|1936-07-29|438|1909|147|1936|3|7|29|3|1936|147|1909|Wednesday|1936Q3|N|N|N|2428351|2428532|2428013|2428288|N|N|N|N|N| +2428380|AAAAAAAAMNNAFCAA|1936-07-30|438|1909|147|1936|4|7|30|3|1936|147|1909|Thursday|1936Q3|N|N|N|2428351|2428532|2428014|2428289|N|N|N|N|N| +2428381|AAAAAAAANNNAFCAA|1936-07-31|438|1909|147|1936|5|7|31|3|1936|147|1909|Friday|1936Q3|N|Y|N|2428351|2428532|2428015|2428290|N|N|N|N|N| +2428382|AAAAAAAAONNAFCAA|1936-08-01|439|1909|147|1936|6|8|1|3|1936|147|1909|Saturday|1936Q3|N|Y|N|2428382|2428594|2428016|2428291|N|N|N|N|N| +2428383|AAAAAAAAPNNAFCAA|1936-08-02|439|1909|147|1936|0|8|2|3|1936|147|1909|Sunday|1936Q3|N|N|N|2428382|2428594|2428017|2428292|N|N|N|N|N| +2428384|AAAAAAAAAONAFCAA|1936-08-03|439|1909|147|1936|1|8|3|3|1936|147|1909|Monday|1936Q3|N|N|N|2428382|2428594|2428018|2428293|N|N|N|N|N| +2428385|AAAAAAAABONAFCAA|1936-08-04|439|1910|147|1936|2|8|4|3|1936|147|1910|Tuesday|1936Q3|N|N|N|2428382|2428594|2428019|2428294|N|N|N|N|N| +2428386|AAAAAAAACONAFCAA|1936-08-05|439|1910|147|1936|3|8|5|3|1936|147|1910|Wednesday|1936Q3|N|N|N|2428382|2428594|2428020|2428295|N|N|N|N|N| +2428387|AAAAAAAADONAFCAA|1936-08-06|439|1910|147|1936|4|8|6|3|1936|147|1910|Thursday|1936Q3|N|N|N|2428382|2428594|2428021|2428296|N|N|N|N|N| +2428388|AAAAAAAAEONAFCAA|1936-08-07|439|1910|147|1936|5|8|7|3|1936|147|1910|Friday|1936Q3|N|Y|N|2428382|2428594|2428022|2428297|N|N|N|N|N| +2428389|AAAAAAAAFONAFCAA|1936-08-08|439|1910|147|1936|6|8|8|3|1936|147|1910|Saturday|1936Q3|N|Y|N|2428382|2428594|2428023|2428298|N|N|N|N|N| +2428390|AAAAAAAAGONAFCAA|1936-08-09|439|1910|147|1936|0|8|9|3|1936|147|1910|Sunday|1936Q3|N|N|N|2428382|2428594|2428024|2428299|N|N|N|N|N| +2428391|AAAAAAAAHONAFCAA|1936-08-10|439|1910|147|1936|1|8|10|3|1936|147|1910|Monday|1936Q3|N|N|N|2428382|2428594|2428025|2428300|N|N|N|N|N| +2428392|AAAAAAAAIONAFCAA|1936-08-11|439|1911|147|1936|2|8|11|3|1936|147|1911|Tuesday|1936Q3|N|N|N|2428382|2428594|2428026|2428301|N|N|N|N|N| +2428393|AAAAAAAAJONAFCAA|1936-08-12|439|1911|147|1936|3|8|12|3|1936|147|1911|Wednesday|1936Q3|N|N|N|2428382|2428594|2428027|2428302|N|N|N|N|N| +2428394|AAAAAAAAKONAFCAA|1936-08-13|439|1911|147|1936|4|8|13|3|1936|147|1911|Thursday|1936Q3|N|N|N|2428382|2428594|2428028|2428303|N|N|N|N|N| +2428395|AAAAAAAALONAFCAA|1936-08-14|439|1911|147|1936|5|8|14|3|1936|147|1911|Friday|1936Q3|N|Y|N|2428382|2428594|2428029|2428304|N|N|N|N|N| +2428396|AAAAAAAAMONAFCAA|1936-08-15|439|1911|147|1936|6|8|15|3|1936|147|1911|Saturday|1936Q3|N|Y|N|2428382|2428594|2428030|2428305|N|N|N|N|N| +2428397|AAAAAAAANONAFCAA|1936-08-16|439|1911|147|1936|0|8|16|3|1936|147|1911|Sunday|1936Q3|N|N|N|2428382|2428594|2428031|2428306|N|N|N|N|N| +2428398|AAAAAAAAOONAFCAA|1936-08-17|439|1911|147|1936|1|8|17|3|1936|147|1911|Monday|1936Q3|N|N|N|2428382|2428594|2428032|2428307|N|N|N|N|N| +2428399|AAAAAAAAPONAFCAA|1936-08-18|439|1912|147|1936|2|8|18|3|1936|147|1912|Tuesday|1936Q3|N|N|N|2428382|2428594|2428033|2428308|N|N|N|N|N| +2428400|AAAAAAAAAPNAFCAA|1936-08-19|439|1912|147|1936|3|8|19|3|1936|147|1912|Wednesday|1936Q3|N|N|N|2428382|2428594|2428034|2428309|N|N|N|N|N| +2428401|AAAAAAAABPNAFCAA|1936-08-20|439|1912|147|1936|4|8|20|3|1936|147|1912|Thursday|1936Q3|N|N|N|2428382|2428594|2428035|2428310|N|N|N|N|N| +2428402|AAAAAAAACPNAFCAA|1936-08-21|439|1912|147|1936|5|8|21|3|1936|147|1912|Friday|1936Q3|N|Y|N|2428382|2428594|2428036|2428311|N|N|N|N|N| +2428403|AAAAAAAADPNAFCAA|1936-08-22|439|1912|147|1936|6|8|22|3|1936|147|1912|Saturday|1936Q3|N|Y|N|2428382|2428594|2428037|2428312|N|N|N|N|N| +2428404|AAAAAAAAEPNAFCAA|1936-08-23|439|1912|147|1936|0|8|23|3|1936|147|1912|Sunday|1936Q3|N|N|N|2428382|2428594|2428038|2428313|N|N|N|N|N| +2428405|AAAAAAAAFPNAFCAA|1936-08-24|439|1912|147|1936|1|8|24|3|1936|147|1912|Monday|1936Q3|N|N|N|2428382|2428594|2428039|2428314|N|N|N|N|N| +2428406|AAAAAAAAGPNAFCAA|1936-08-25|439|1913|147|1936|2|8|25|3|1936|147|1913|Tuesday|1936Q3|N|N|N|2428382|2428594|2428040|2428315|N|N|N|N|N| +2428407|AAAAAAAAHPNAFCAA|1936-08-26|439|1913|147|1936|3|8|26|3|1936|147|1913|Wednesday|1936Q3|N|N|N|2428382|2428594|2428041|2428316|N|N|N|N|N| +2428408|AAAAAAAAIPNAFCAA|1936-08-27|439|1913|147|1936|4|8|27|3|1936|147|1913|Thursday|1936Q3|N|N|N|2428382|2428594|2428042|2428317|N|N|N|N|N| +2428409|AAAAAAAAJPNAFCAA|1936-08-28|439|1913|147|1936|5|8|28|3|1936|147|1913|Friday|1936Q3|N|Y|N|2428382|2428594|2428043|2428318|N|N|N|N|N| +2428410|AAAAAAAAKPNAFCAA|1936-08-29|439|1913|147|1936|6|8|29|3|1936|147|1913|Saturday|1936Q3|N|Y|N|2428382|2428594|2428044|2428319|N|N|N|N|N| +2428411|AAAAAAAALPNAFCAA|1936-08-30|439|1913|147|1936|0|8|30|3|1936|147|1913|Sunday|1936Q3|N|N|N|2428382|2428594|2428045|2428320|N|N|N|N|N| +2428412|AAAAAAAAMPNAFCAA|1936-08-31|439|1913|147|1936|1|8|31|3|1936|147|1913|Monday|1936Q3|N|N|N|2428382|2428594|2428046|2428321|N|N|N|N|N| +2428413|AAAAAAAANPNAFCAA|1936-09-01|440|1914|148|1936|2|9|1|3|1936|148|1914|Tuesday|1936Q3|N|N|N|2428413|2428656|2428047|2428322|N|N|N|N|N| +2428414|AAAAAAAAOPNAFCAA|1936-09-02|440|1914|148|1936|3|9|2|3|1936|148|1914|Wednesday|1936Q3|N|N|N|2428413|2428656|2428048|2428323|N|N|N|N|N| +2428415|AAAAAAAAPPNAFCAA|1936-09-03|440|1914|148|1936|4|9|3|3|1936|148|1914|Thursday|1936Q3|N|N|N|2428413|2428656|2428049|2428324|N|N|N|N|N| +2428416|AAAAAAAAAAOAFCAA|1936-09-04|440|1914|148|1936|5|9|4|3|1936|148|1914|Friday|1936Q3|N|Y|N|2428413|2428656|2428050|2428325|N|N|N|N|N| +2428417|AAAAAAAABAOAFCAA|1936-09-05|440|1914|148|1936|6|9|5|3|1936|148|1914|Saturday|1936Q3|N|Y|N|2428413|2428656|2428051|2428326|N|N|N|N|N| +2428418|AAAAAAAACAOAFCAA|1936-09-06|440|1914|148|1936|0|9|6|3|1936|148|1914|Sunday|1936Q3|N|N|N|2428413|2428656|2428052|2428327|N|N|N|N|N| +2428419|AAAAAAAADAOAFCAA|1936-09-07|440|1914|148|1936|1|9|7|3|1936|148|1914|Monday|1936Q3|N|N|N|2428413|2428656|2428053|2428328|N|N|N|N|N| +2428420|AAAAAAAAEAOAFCAA|1936-09-08|440|1915|148|1936|2|9|8|3|1936|148|1915|Tuesday|1936Q3|N|N|N|2428413|2428656|2428054|2428329|N|N|N|N|N| +2428421|AAAAAAAAFAOAFCAA|1936-09-09|440|1915|148|1936|3|9|9|3|1936|148|1915|Wednesday|1936Q3|N|N|N|2428413|2428656|2428055|2428330|N|N|N|N|N| +2428422|AAAAAAAAGAOAFCAA|1936-09-10|440|1915|148|1936|4|9|10|3|1936|148|1915|Thursday|1936Q3|N|N|N|2428413|2428656|2428056|2428331|N|N|N|N|N| +2428423|AAAAAAAAHAOAFCAA|1936-09-11|440|1915|148|1936|5|9|11|3|1936|148|1915|Friday|1936Q3|N|Y|N|2428413|2428656|2428057|2428332|N|N|N|N|N| +2428424|AAAAAAAAIAOAFCAA|1936-09-12|440|1915|148|1936|6|9|12|3|1936|148|1915|Saturday|1936Q3|N|Y|N|2428413|2428656|2428058|2428333|N|N|N|N|N| +2428425|AAAAAAAAJAOAFCAA|1936-09-13|440|1915|148|1936|0|9|13|3|1936|148|1915|Sunday|1936Q3|N|N|N|2428413|2428656|2428059|2428334|N|N|N|N|N| +2428426|AAAAAAAAKAOAFCAA|1936-09-14|440|1915|148|1936|1|9|14|3|1936|148|1915|Monday|1936Q3|N|N|N|2428413|2428656|2428060|2428335|N|N|N|N|N| +2428427|AAAAAAAALAOAFCAA|1936-09-15|440|1916|148|1936|2|9|15|3|1936|148|1916|Tuesday|1936Q3|N|N|N|2428413|2428656|2428061|2428336|N|N|N|N|N| +2428428|AAAAAAAAMAOAFCAA|1936-09-16|440|1916|148|1936|3|9|16|3|1936|148|1916|Wednesday|1936Q3|N|N|N|2428413|2428656|2428062|2428337|N|N|N|N|N| +2428429|AAAAAAAANAOAFCAA|1936-09-17|440|1916|148|1936|4|9|17|3|1936|148|1916|Thursday|1936Q3|N|N|N|2428413|2428656|2428063|2428338|N|N|N|N|N| +2428430|AAAAAAAAOAOAFCAA|1936-09-18|440|1916|148|1936|5|9|18|3|1936|148|1916|Friday|1936Q3|N|Y|N|2428413|2428656|2428064|2428339|N|N|N|N|N| +2428431|AAAAAAAAPAOAFCAA|1936-09-19|440|1916|148|1936|6|9|19|3|1936|148|1916|Saturday|1936Q3|N|Y|N|2428413|2428656|2428065|2428340|N|N|N|N|N| +2428432|AAAAAAAAABOAFCAA|1936-09-20|440|1916|148|1936|0|9|20|3|1936|148|1916|Sunday|1936Q3|N|N|N|2428413|2428656|2428066|2428341|N|N|N|N|N| +2428433|AAAAAAAABBOAFCAA|1936-09-21|440|1916|148|1936|1|9|21|3|1936|148|1916|Monday|1936Q3|N|N|N|2428413|2428656|2428067|2428342|N|N|N|N|N| +2428434|AAAAAAAACBOAFCAA|1936-09-22|440|1917|148|1936|2|9|22|3|1936|148|1917|Tuesday|1936Q3|N|N|N|2428413|2428656|2428068|2428343|N|N|N|N|N| +2428435|AAAAAAAADBOAFCAA|1936-09-23|440|1917|148|1936|3|9|23|3|1936|148|1917|Wednesday|1936Q3|N|N|N|2428413|2428656|2428069|2428344|N|N|N|N|N| +2428436|AAAAAAAAEBOAFCAA|1936-09-24|440|1917|148|1936|4|9|24|3|1936|148|1917|Thursday|1936Q3|N|N|N|2428413|2428656|2428070|2428345|N|N|N|N|N| +2428437|AAAAAAAAFBOAFCAA|1936-09-25|440|1917|148|1936|5|9|25|3|1936|148|1917|Friday|1936Q3|N|Y|N|2428413|2428656|2428071|2428346|N|N|N|N|N| +2428438|AAAAAAAAGBOAFCAA|1936-09-26|440|1917|148|1936|6|9|26|3|1936|148|1917|Saturday|1936Q3|N|Y|N|2428413|2428656|2428072|2428347|N|N|N|N|N| +2428439|AAAAAAAAHBOAFCAA|1936-09-27|440|1917|148|1936|0|9|27|3|1936|148|1917|Sunday|1936Q3|N|N|N|2428413|2428656|2428073|2428348|N|N|N|N|N| +2428440|AAAAAAAAIBOAFCAA|1936-09-28|440|1917|148|1936|1|9|28|3|1936|148|1917|Monday|1936Q3|N|N|N|2428413|2428656|2428074|2428349|N|N|N|N|N| +2428441|AAAAAAAAJBOAFCAA|1936-09-29|440|1918|148|1936|2|9|29|3|1936|148|1918|Tuesday|1936Q3|N|N|N|2428413|2428656|2428075|2428350|N|N|N|N|N| +2428442|AAAAAAAAKBOAFCAA|1936-09-30|440|1918|148|1936|3|9|30|3|1936|148|1918|Wednesday|1936Q3|N|N|N|2428413|2428656|2428076|2428351|N|N|N|N|N| +2428443|AAAAAAAALBOAFCAA|1936-10-01|441|1918|148|1936|4|10|1|4|1936|148|1918|Thursday|1936Q4|N|N|N|2428443|2428716|2428077|2428351|N|N|N|N|N| +2428444|AAAAAAAAMBOAFCAA|1936-10-02|441|1918|148|1936|5|10|2|4|1936|148|1918|Friday|1936Q4|N|Y|N|2428443|2428716|2428078|2428352|N|N|N|N|N| +2428445|AAAAAAAANBOAFCAA|1936-10-03|441|1918|148|1936|6|10|3|4|1936|148|1918|Saturday|1936Q4|N|Y|N|2428443|2428716|2428079|2428353|N|N|N|N|N| +2428446|AAAAAAAAOBOAFCAA|1936-10-04|441|1918|148|1936|0|10|4|4|1936|148|1918|Sunday|1936Q4|N|N|N|2428443|2428716|2428080|2428354|N|N|N|N|N| +2428447|AAAAAAAAPBOAFCAA|1936-10-05|441|1918|148|1936|1|10|5|4|1936|148|1918|Monday|1936Q4|N|N|N|2428443|2428716|2428081|2428355|N|N|N|N|N| +2428448|AAAAAAAAACOAFCAA|1936-10-06|441|1919|148|1936|2|10|6|4|1936|148|1919|Tuesday|1936Q4|N|N|N|2428443|2428716|2428082|2428356|N|N|N|N|N| +2428449|AAAAAAAABCOAFCAA|1936-10-07|441|1919|148|1936|3|10|7|4|1936|148|1919|Wednesday|1936Q4|N|N|N|2428443|2428716|2428083|2428357|N|N|N|N|N| +2428450|AAAAAAAACCOAFCAA|1936-10-08|441|1919|148|1936|4|10|8|4|1936|148|1919|Thursday|1936Q4|N|N|N|2428443|2428716|2428084|2428358|N|N|N|N|N| +2428451|AAAAAAAADCOAFCAA|1936-10-09|441|1919|148|1936|5|10|9|4|1936|148|1919|Friday|1936Q4|N|Y|N|2428443|2428716|2428085|2428359|N|N|N|N|N| +2428452|AAAAAAAAECOAFCAA|1936-10-10|441|1919|148|1936|6|10|10|4|1936|148|1919|Saturday|1936Q4|N|Y|N|2428443|2428716|2428086|2428360|N|N|N|N|N| +2428453|AAAAAAAAFCOAFCAA|1936-10-11|441|1919|148|1936|0|10|11|4|1936|148|1919|Sunday|1936Q4|N|N|N|2428443|2428716|2428087|2428361|N|N|N|N|N| +2428454|AAAAAAAAGCOAFCAA|1936-10-12|441|1919|148|1936|1|10|12|4|1936|148|1919|Monday|1936Q4|N|N|N|2428443|2428716|2428088|2428362|N|N|N|N|N| +2428455|AAAAAAAAHCOAFCAA|1936-10-13|441|1920|148|1936|2|10|13|4|1936|148|1920|Tuesday|1936Q4|N|N|N|2428443|2428716|2428089|2428363|N|N|N|N|N| +2428456|AAAAAAAAICOAFCAA|1936-10-14|441|1920|148|1936|3|10|14|4|1936|148|1920|Wednesday|1936Q4|N|N|N|2428443|2428716|2428090|2428364|N|N|N|N|N| +2428457|AAAAAAAAJCOAFCAA|1936-10-15|441|1920|148|1936|4|10|15|4|1936|148|1920|Thursday|1936Q4|N|N|N|2428443|2428716|2428091|2428365|N|N|N|N|N| +2428458|AAAAAAAAKCOAFCAA|1936-10-16|441|1920|148|1936|5|10|16|4|1936|148|1920|Friday|1936Q4|N|Y|N|2428443|2428716|2428092|2428366|N|N|N|N|N| +2428459|AAAAAAAALCOAFCAA|1936-10-17|441|1920|148|1936|6|10|17|4|1936|148|1920|Saturday|1936Q4|N|Y|N|2428443|2428716|2428093|2428367|N|N|N|N|N| +2428460|AAAAAAAAMCOAFCAA|1936-10-18|441|1920|148|1936|0|10|18|4|1936|148|1920|Sunday|1936Q4|N|N|N|2428443|2428716|2428094|2428368|N|N|N|N|N| +2428461|AAAAAAAANCOAFCAA|1936-10-19|441|1920|148|1936|1|10|19|4|1936|148|1920|Monday|1936Q4|N|N|N|2428443|2428716|2428095|2428369|N|N|N|N|N| +2428462|AAAAAAAAOCOAFCAA|1936-10-20|441|1921|148|1936|2|10|20|4|1936|148|1921|Tuesday|1936Q4|N|N|N|2428443|2428716|2428096|2428370|N|N|N|N|N| +2428463|AAAAAAAAPCOAFCAA|1936-10-21|441|1921|148|1936|3|10|21|4|1936|148|1921|Wednesday|1936Q4|N|N|N|2428443|2428716|2428097|2428371|N|N|N|N|N| +2428464|AAAAAAAAADOAFCAA|1936-10-22|441|1921|148|1936|4|10|22|4|1936|148|1921|Thursday|1936Q4|N|N|N|2428443|2428716|2428098|2428372|N|N|N|N|N| +2428465|AAAAAAAABDOAFCAA|1936-10-23|441|1921|148|1936|5|10|23|4|1936|148|1921|Friday|1936Q4|N|Y|N|2428443|2428716|2428099|2428373|N|N|N|N|N| +2428466|AAAAAAAACDOAFCAA|1936-10-24|441|1921|148|1936|6|10|24|4|1936|148|1921|Saturday|1936Q4|N|Y|N|2428443|2428716|2428100|2428374|N|N|N|N|N| +2428467|AAAAAAAADDOAFCAA|1936-10-25|441|1921|148|1936|0|10|25|4|1936|148|1921|Sunday|1936Q4|N|N|N|2428443|2428716|2428101|2428375|N|N|N|N|N| +2428468|AAAAAAAAEDOAFCAA|1936-10-26|441|1921|148|1936|1|10|26|4|1936|148|1921|Monday|1936Q4|N|N|N|2428443|2428716|2428102|2428376|N|N|N|N|N| +2428469|AAAAAAAAFDOAFCAA|1936-10-27|441|1922|148|1936|2|10|27|4|1936|148|1922|Tuesday|1936Q4|N|N|N|2428443|2428716|2428103|2428377|N|N|N|N|N| +2428470|AAAAAAAAGDOAFCAA|1936-10-28|441|1922|148|1936|3|10|28|4|1936|148|1922|Wednesday|1936Q4|N|N|N|2428443|2428716|2428104|2428378|N|N|N|N|N| +2428471|AAAAAAAAHDOAFCAA|1936-10-29|441|1922|148|1936|4|10|29|4|1936|148|1922|Thursday|1936Q4|N|N|N|2428443|2428716|2428105|2428379|N|N|N|N|N| +2428472|AAAAAAAAIDOAFCAA|1936-10-30|441|1922|148|1936|5|10|30|4|1936|148|1922|Friday|1936Q4|N|Y|N|2428443|2428716|2428106|2428380|N|N|N|N|N| +2428473|AAAAAAAAJDOAFCAA|1936-10-31|441|1922|148|1936|6|10|31|4|1936|148|1922|Saturday|1936Q4|N|Y|N|2428443|2428716|2428107|2428381|N|N|N|N|N| +2428474|AAAAAAAAKDOAFCAA|1936-11-01|442|1922|148|1936|0|11|1|4|1936|148|1922|Sunday|1936Q4|N|N|N|2428474|2428778|2428108|2428382|N|N|N|N|N| +2428475|AAAAAAAALDOAFCAA|1936-11-02|442|1922|148|1936|1|11|2|4|1936|148|1922|Monday|1936Q4|N|N|N|2428474|2428778|2428109|2428383|N|N|N|N|N| +2428476|AAAAAAAAMDOAFCAA|1936-11-03|442|1923|148|1936|2|11|3|4|1936|148|1923|Tuesday|1936Q4|N|N|N|2428474|2428778|2428110|2428384|N|N|N|N|N| +2428477|AAAAAAAANDOAFCAA|1936-11-04|442|1923|148|1936|3|11|4|4|1936|148|1923|Wednesday|1936Q4|N|N|N|2428474|2428778|2428111|2428385|N|N|N|N|N| +2428478|AAAAAAAAODOAFCAA|1936-11-05|442|1923|148|1936|4|11|5|4|1936|148|1923|Thursday|1936Q4|N|N|N|2428474|2428778|2428112|2428386|N|N|N|N|N| +2428479|AAAAAAAAPDOAFCAA|1936-11-06|442|1923|148|1936|5|11|6|4|1936|148|1923|Friday|1936Q4|N|Y|N|2428474|2428778|2428113|2428387|N|N|N|N|N| +2428480|AAAAAAAAAEOAFCAA|1936-11-07|442|1923|148|1936|6|11|7|4|1936|148|1923|Saturday|1936Q4|N|Y|N|2428474|2428778|2428114|2428388|N|N|N|N|N| +2428481|AAAAAAAABEOAFCAA|1936-11-08|442|1923|148|1936|0|11|8|4|1936|148|1923|Sunday|1936Q4|N|N|N|2428474|2428778|2428115|2428389|N|N|N|N|N| +2428482|AAAAAAAACEOAFCAA|1936-11-09|442|1923|148|1936|1|11|9|4|1936|148|1923|Monday|1936Q4|N|N|N|2428474|2428778|2428116|2428390|N|N|N|N|N| +2428483|AAAAAAAADEOAFCAA|1936-11-10|442|1924|148|1936|2|11|10|4|1936|148|1924|Tuesday|1936Q4|N|N|N|2428474|2428778|2428117|2428391|N|N|N|N|N| +2428484|AAAAAAAAEEOAFCAA|1936-11-11|442|1924|148|1936|3|11|11|4|1936|148|1924|Wednesday|1936Q4|N|N|N|2428474|2428778|2428118|2428392|N|N|N|N|N| +2428485|AAAAAAAAFEOAFCAA|1936-11-12|442|1924|148|1936|4|11|12|4|1936|148|1924|Thursday|1936Q4|N|N|N|2428474|2428778|2428119|2428393|N|N|N|N|N| +2428486|AAAAAAAAGEOAFCAA|1936-11-13|442|1924|148|1936|5|11|13|4|1936|148|1924|Friday|1936Q4|N|Y|N|2428474|2428778|2428120|2428394|N|N|N|N|N| +2428487|AAAAAAAAHEOAFCAA|1936-11-14|442|1924|148|1936|6|11|14|4|1936|148|1924|Saturday|1936Q4|N|Y|N|2428474|2428778|2428121|2428395|N|N|N|N|N| +2428488|AAAAAAAAIEOAFCAA|1936-11-15|442|1924|148|1936|0|11|15|4|1936|148|1924|Sunday|1936Q4|N|N|N|2428474|2428778|2428122|2428396|N|N|N|N|N| +2428489|AAAAAAAAJEOAFCAA|1936-11-16|442|1924|148|1936|1|11|16|4|1936|148|1924|Monday|1936Q4|N|N|N|2428474|2428778|2428123|2428397|N|N|N|N|N| +2428490|AAAAAAAAKEOAFCAA|1936-11-17|442|1925|148|1936|2|11|17|4|1936|148|1925|Tuesday|1936Q4|N|N|N|2428474|2428778|2428124|2428398|N|N|N|N|N| +2428491|AAAAAAAALEOAFCAA|1936-11-18|442|1925|148|1936|3|11|18|4|1936|148|1925|Wednesday|1936Q4|N|N|N|2428474|2428778|2428125|2428399|N|N|N|N|N| +2428492|AAAAAAAAMEOAFCAA|1936-11-19|442|1925|148|1936|4|11|19|4|1936|148|1925|Thursday|1936Q4|N|N|N|2428474|2428778|2428126|2428400|N|N|N|N|N| +2428493|AAAAAAAANEOAFCAA|1936-11-20|442|1925|148|1936|5|11|20|4|1936|148|1925|Friday|1936Q4|N|Y|N|2428474|2428778|2428127|2428401|N|N|N|N|N| +2428494|AAAAAAAAOEOAFCAA|1936-11-21|442|1925|148|1936|6|11|21|4|1936|148|1925|Saturday|1936Q4|N|Y|N|2428474|2428778|2428128|2428402|N|N|N|N|N| +2428495|AAAAAAAAPEOAFCAA|1936-11-22|442|1925|148|1936|0|11|22|4|1936|148|1925|Sunday|1936Q4|N|N|N|2428474|2428778|2428129|2428403|N|N|N|N|N| +2428496|AAAAAAAAAFOAFCAA|1936-11-23|442|1925|148|1936|1|11|23|4|1936|148|1925|Monday|1936Q4|N|N|N|2428474|2428778|2428130|2428404|N|N|N|N|N| +2428497|AAAAAAAABFOAFCAA|1936-11-24|442|1926|148|1936|2|11|24|4|1936|148|1926|Tuesday|1936Q4|N|N|N|2428474|2428778|2428131|2428405|N|N|N|N|N| +2428498|AAAAAAAACFOAFCAA|1936-11-25|442|1926|148|1936|3|11|25|4|1936|148|1926|Wednesday|1936Q4|N|N|N|2428474|2428778|2428132|2428406|N|N|N|N|N| +2428499|AAAAAAAADFOAFCAA|1936-11-26|442|1926|148|1936|4|11|26|4|1936|148|1926|Thursday|1936Q4|N|N|N|2428474|2428778|2428133|2428407|N|N|N|N|N| +2428500|AAAAAAAAEFOAFCAA|1936-11-27|442|1926|148|1936|5|11|27|4|1936|148|1926|Friday|1936Q4|N|Y|N|2428474|2428778|2428134|2428408|N|N|N|N|N| +2428501|AAAAAAAAFFOAFCAA|1936-11-28|442|1926|148|1936|6|11|28|4|1936|148|1926|Saturday|1936Q4|N|Y|N|2428474|2428778|2428135|2428409|N|N|N|N|N| +2428502|AAAAAAAAGFOAFCAA|1936-11-29|442|1926|148|1936|0|11|29|4|1936|148|1926|Sunday|1936Q4|N|N|N|2428474|2428778|2428136|2428410|N|N|N|N|N| +2428503|AAAAAAAAHFOAFCAA|1936-11-30|442|1926|148|1936|1|11|30|4|1936|148|1926|Monday|1936Q4|N|N|N|2428474|2428778|2428137|2428411|N|N|N|N|N| +2428504|AAAAAAAAIFOAFCAA|1936-12-01|443|1927|149|1936|2|12|1|4|1936|149|1927|Tuesday|1936Q4|N|N|N|2428504|2428838|2428138|2428412|N|N|N|N|N| +2428505|AAAAAAAAJFOAFCAA|1936-12-02|443|1927|149|1936|3|12|2|4|1936|149|1927|Wednesday|1936Q4|N|N|N|2428504|2428838|2428139|2428413|N|N|N|N|N| +2428506|AAAAAAAAKFOAFCAA|1936-12-03|443|1927|149|1936|4|12|3|4|1936|149|1927|Thursday|1936Q4|N|N|N|2428504|2428838|2428140|2428414|N|N|N|N|N| +2428507|AAAAAAAALFOAFCAA|1936-12-04|443|1927|149|1936|5|12|4|4|1936|149|1927|Friday|1936Q4|N|Y|N|2428504|2428838|2428141|2428415|N|N|N|N|N| +2428508|AAAAAAAAMFOAFCAA|1936-12-05|443|1927|149|1936|6|12|5|4|1936|149|1927|Saturday|1936Q4|N|Y|N|2428504|2428838|2428142|2428416|N|N|N|N|N| +2428509|AAAAAAAANFOAFCAA|1936-12-06|443|1927|149|1936|0|12|6|4|1936|149|1927|Sunday|1936Q4|N|N|N|2428504|2428838|2428143|2428417|N|N|N|N|N| +2428510|AAAAAAAAOFOAFCAA|1936-12-07|443|1927|149|1936|1|12|7|4|1936|149|1927|Monday|1936Q4|N|N|N|2428504|2428838|2428144|2428418|N|N|N|N|N| +2428511|AAAAAAAAPFOAFCAA|1936-12-08|443|1928|149|1936|2|12|8|4|1936|149|1928|Tuesday|1936Q4|N|N|N|2428504|2428838|2428145|2428419|N|N|N|N|N| +2428512|AAAAAAAAAGOAFCAA|1936-12-09|443|1928|149|1936|3|12|9|4|1936|149|1928|Wednesday|1936Q4|N|N|N|2428504|2428838|2428146|2428420|N|N|N|N|N| +2428513|AAAAAAAABGOAFCAA|1936-12-10|443|1928|149|1936|4|12|10|4|1936|149|1928|Thursday|1936Q4|N|N|N|2428504|2428838|2428147|2428421|N|N|N|N|N| +2428514|AAAAAAAACGOAFCAA|1936-12-11|443|1928|149|1936|5|12|11|4|1936|149|1928|Friday|1936Q4|N|Y|N|2428504|2428838|2428148|2428422|N|N|N|N|N| +2428515|AAAAAAAADGOAFCAA|1936-12-12|443|1928|149|1936|6|12|12|4|1936|149|1928|Saturday|1936Q4|N|Y|N|2428504|2428838|2428149|2428423|N|N|N|N|N| +2428516|AAAAAAAAEGOAFCAA|1936-12-13|443|1928|149|1936|0|12|13|4|1936|149|1928|Sunday|1936Q4|N|N|N|2428504|2428838|2428150|2428424|N|N|N|N|N| +2428517|AAAAAAAAFGOAFCAA|1936-12-14|443|1928|149|1936|1|12|14|4|1936|149|1928|Monday|1936Q4|N|N|N|2428504|2428838|2428151|2428425|N|N|N|N|N| +2428518|AAAAAAAAGGOAFCAA|1936-12-15|443|1929|149|1936|2|12|15|4|1936|149|1929|Tuesday|1936Q4|N|N|N|2428504|2428838|2428152|2428426|N|N|N|N|N| +2428519|AAAAAAAAHGOAFCAA|1936-12-16|443|1929|149|1936|3|12|16|4|1936|149|1929|Wednesday|1936Q4|N|N|N|2428504|2428838|2428153|2428427|N|N|N|N|N| +2428520|AAAAAAAAIGOAFCAA|1936-12-17|443|1929|149|1936|4|12|17|4|1936|149|1929|Thursday|1936Q4|N|N|N|2428504|2428838|2428154|2428428|N|N|N|N|N| +2428521|AAAAAAAAJGOAFCAA|1936-12-18|443|1929|149|1936|5|12|18|4|1936|149|1929|Friday|1936Q4|N|Y|N|2428504|2428838|2428155|2428429|N|N|N|N|N| +2428522|AAAAAAAAKGOAFCAA|1936-12-19|443|1929|149|1936|6|12|19|4|1936|149|1929|Saturday|1936Q4|N|Y|N|2428504|2428838|2428156|2428430|N|N|N|N|N| +2428523|AAAAAAAALGOAFCAA|1936-12-20|443|1929|149|1936|0|12|20|4|1936|149|1929|Sunday|1936Q4|N|N|N|2428504|2428838|2428157|2428431|N|N|N|N|N| +2428524|AAAAAAAAMGOAFCAA|1936-12-21|443|1929|149|1936|1|12|21|4|1936|149|1929|Monday|1936Q4|N|N|N|2428504|2428838|2428158|2428432|N|N|N|N|N| +2428525|AAAAAAAANGOAFCAA|1936-12-22|443|1930|149|1936|2|12|22|4|1936|149|1930|Tuesday|1936Q4|N|N|N|2428504|2428838|2428159|2428433|N|N|N|N|N| +2428526|AAAAAAAAOGOAFCAA|1936-12-23|443|1930|149|1936|3|12|23|4|1936|149|1930|Wednesday|1936Q4|N|N|N|2428504|2428838|2428160|2428434|N|N|N|N|N| +2428527|AAAAAAAAPGOAFCAA|1936-12-24|443|1930|149|1936|4|12|24|4|1936|149|1930|Thursday|1936Q4|N|N|N|2428504|2428838|2428161|2428435|N|N|N|N|N| +2428528|AAAAAAAAAHOAFCAA|1936-12-25|443|1930|149|1936|5|12|25|4|1936|149|1930|Friday|1936Q4|Y|Y|N|2428504|2428838|2428162|2428436|N|N|N|N|N| +2428529|AAAAAAAABHOAFCAA|1936-12-26|443|1930|149|1936|6|12|26|4|1936|149|1930|Saturday|1936Q4|N|Y|Y|2428504|2428838|2428163|2428437|N|N|N|N|N| +2428530|AAAAAAAACHOAFCAA|1936-12-27|443|1930|149|1936|0|12|27|4|1936|149|1930|Sunday|1936Q4|N|N|N|2428504|2428838|2428164|2428438|N|N|N|N|N| +2428531|AAAAAAAADHOAFCAA|1936-12-28|443|1930|149|1936|1|12|28|4|1936|149|1930|Monday|1936Q4|N|N|N|2428504|2428838|2428165|2428439|N|N|N|N|N| +2428532|AAAAAAAAEHOAFCAA|1936-12-29|443|1931|149|1936|2|12|29|4|1936|149|1931|Tuesday|1936Q4|N|N|N|2428504|2428838|2428166|2428440|N|N|N|N|N| +2428533|AAAAAAAAFHOAFCAA|1936-12-30|443|1931|149|1936|3|12|30|4|1936|149|1931|Wednesday|1936Q4|N|N|N|2428504|2428838|2428167|2428441|N|N|N|N|N| +2428534|AAAAAAAAGHOAFCAA|1936-12-31|443|1931|149|1936|4|12|31|4|1936|149|1931|Thursday|1936Q4|Y|N|N|2428504|2428838|2428168|2428442|N|N|N|N|N| +2428535|AAAAAAAAHHOAFCAA|1937-01-01|444|1931|149|1937|5|1|1|1|1937|149|1931|Friday|1937Q1|Y|Y|Y|2428535|2428534|2428169|2428443|N|N|N|N|N| +2428536|AAAAAAAAIHOAFCAA|1937-01-02|444|1931|149|1937|6|1|2|1|1937|149|1931|Saturday|1937Q1|N|Y|Y|2428535|2428534|2428170|2428444|N|N|N|N|N| +2428537|AAAAAAAAJHOAFCAA|1937-01-03|444|1931|149|1937|0|1|3|1|1937|149|1931|Sunday|1937Q1|N|N|N|2428535|2428534|2428171|2428445|N|N|N|N|N| +2428538|AAAAAAAAKHOAFCAA|1937-01-04|444|1931|149|1937|1|1|4|1|1937|149|1931|Monday|1937Q1|N|N|N|2428535|2428534|2428172|2428446|N|N|N|N|N| +2428539|AAAAAAAALHOAFCAA|1937-01-05|444|1932|149|1937|2|1|5|1|1937|149|1932|Tuesday|1937Q1|N|N|N|2428535|2428534|2428173|2428447|N|N|N|N|N| +2428540|AAAAAAAAMHOAFCAA|1937-01-06|444|1932|149|1937|3|1|6|1|1937|149|1932|Wednesday|1937Q1|N|N|N|2428535|2428534|2428174|2428448|N|N|N|N|N| +2428541|AAAAAAAANHOAFCAA|1937-01-07|444|1932|149|1937|4|1|7|1|1937|149|1932|Thursday|1937Q1|N|N|N|2428535|2428534|2428175|2428449|N|N|N|N|N| +2428542|AAAAAAAAOHOAFCAA|1937-01-08|444|1932|149|1937|5|1|8|1|1937|149|1932|Friday|1937Q1|N|Y|N|2428535|2428534|2428176|2428450|N|N|N|N|N| +2428543|AAAAAAAAPHOAFCAA|1937-01-09|444|1932|149|1937|6|1|9|1|1937|149|1932|Saturday|1937Q1|N|Y|N|2428535|2428534|2428177|2428451|N|N|N|N|N| +2428544|AAAAAAAAAIOAFCAA|1937-01-10|444|1932|149|1937|0|1|10|1|1937|149|1932|Sunday|1937Q1|N|N|N|2428535|2428534|2428178|2428452|N|N|N|N|N| +2428545|AAAAAAAABIOAFCAA|1937-01-11|444|1932|149|1937|1|1|11|1|1937|149|1932|Monday|1937Q1|N|N|N|2428535|2428534|2428179|2428453|N|N|N|N|N| +2428546|AAAAAAAACIOAFCAA|1937-01-12|444|1933|149|1937|2|1|12|1|1937|149|1933|Tuesday|1937Q1|N|N|N|2428535|2428534|2428180|2428454|N|N|N|N|N| +2428547|AAAAAAAADIOAFCAA|1937-01-13|444|1933|149|1937|3|1|13|1|1937|149|1933|Wednesday|1937Q1|N|N|N|2428535|2428534|2428181|2428455|N|N|N|N|N| +2428548|AAAAAAAAEIOAFCAA|1937-01-14|444|1933|149|1937|4|1|14|1|1937|149|1933|Thursday|1937Q1|N|N|N|2428535|2428534|2428182|2428456|N|N|N|N|N| +2428549|AAAAAAAAFIOAFCAA|1937-01-15|444|1933|149|1937|5|1|15|1|1937|149|1933|Friday|1937Q1|N|Y|N|2428535|2428534|2428183|2428457|N|N|N|N|N| +2428550|AAAAAAAAGIOAFCAA|1937-01-16|444|1933|149|1937|6|1|16|1|1937|149|1933|Saturday|1937Q1|N|Y|N|2428535|2428534|2428184|2428458|N|N|N|N|N| +2428551|AAAAAAAAHIOAFCAA|1937-01-17|444|1933|149|1937|0|1|17|1|1937|149|1933|Sunday|1937Q1|N|N|N|2428535|2428534|2428185|2428459|N|N|N|N|N| +2428552|AAAAAAAAIIOAFCAA|1937-01-18|444|1933|149|1937|1|1|18|1|1937|149|1933|Monday|1937Q1|N|N|N|2428535|2428534|2428186|2428460|N|N|N|N|N| +2428553|AAAAAAAAJIOAFCAA|1937-01-19|444|1934|149|1937|2|1|19|1|1937|149|1934|Tuesday|1937Q1|N|N|N|2428535|2428534|2428187|2428461|N|N|N|N|N| +2428554|AAAAAAAAKIOAFCAA|1937-01-20|444|1934|149|1937|3|1|20|1|1937|149|1934|Wednesday|1937Q1|N|N|N|2428535|2428534|2428188|2428462|N|N|N|N|N| +2428555|AAAAAAAALIOAFCAA|1937-01-21|444|1934|149|1937|4|1|21|1|1937|149|1934|Thursday|1937Q1|N|N|N|2428535|2428534|2428189|2428463|N|N|N|N|N| +2428556|AAAAAAAAMIOAFCAA|1937-01-22|444|1934|149|1937|5|1|22|1|1937|149|1934|Friday|1937Q1|N|Y|N|2428535|2428534|2428190|2428464|N|N|N|N|N| +2428557|AAAAAAAANIOAFCAA|1937-01-23|444|1934|149|1937|6|1|23|1|1937|149|1934|Saturday|1937Q1|N|Y|N|2428535|2428534|2428191|2428465|N|N|N|N|N| +2428558|AAAAAAAAOIOAFCAA|1937-01-24|444|1934|149|1937|0|1|24|1|1937|149|1934|Sunday|1937Q1|N|N|N|2428535|2428534|2428192|2428466|N|N|N|N|N| +2428559|AAAAAAAAPIOAFCAA|1937-01-25|444|1934|149|1937|1|1|25|1|1937|149|1934|Monday|1937Q1|N|N|N|2428535|2428534|2428193|2428467|N|N|N|N|N| +2428560|AAAAAAAAAJOAFCAA|1937-01-26|444|1935|149|1937|2|1|26|1|1937|149|1935|Tuesday|1937Q1|N|N|N|2428535|2428534|2428194|2428468|N|N|N|N|N| +2428561|AAAAAAAABJOAFCAA|1937-01-27|444|1935|149|1937|3|1|27|1|1937|149|1935|Wednesday|1937Q1|N|N|N|2428535|2428534|2428195|2428469|N|N|N|N|N| +2428562|AAAAAAAACJOAFCAA|1937-01-28|444|1935|149|1937|4|1|28|1|1937|149|1935|Thursday|1937Q1|N|N|N|2428535|2428534|2428196|2428470|N|N|N|N|N| +2428563|AAAAAAAADJOAFCAA|1937-01-29|444|1935|149|1937|5|1|29|1|1937|149|1935|Friday|1937Q1|N|Y|N|2428535|2428534|2428197|2428471|N|N|N|N|N| +2428564|AAAAAAAAEJOAFCAA|1937-01-30|444|1935|149|1937|6|1|30|1|1937|149|1935|Saturday|1937Q1|N|Y|N|2428535|2428534|2428198|2428472|N|N|N|N|N| +2428565|AAAAAAAAFJOAFCAA|1937-01-31|444|1935|149|1937|0|1|31|1|1937|149|1935|Sunday|1937Q1|N|N|N|2428535|2428534|2428199|2428473|N|N|N|N|N| +2428566|AAAAAAAAGJOAFCAA|1937-02-01|445|1935|149|1937|1|2|1|1|1937|149|1935|Monday|1937Q1|N|N|N|2428566|2428596|2428200|2428474|N|N|N|N|N| +2428567|AAAAAAAAHJOAFCAA|1937-02-02|445|1936|149|1937|2|2|2|1|1937|149|1936|Tuesday|1937Q1|N|N|N|2428566|2428596|2428201|2428475|N|N|N|N|N| +2428568|AAAAAAAAIJOAFCAA|1937-02-03|445|1936|149|1937|3|2|3|1|1937|149|1936|Wednesday|1937Q1|N|N|N|2428566|2428596|2428202|2428476|N|N|N|N|N| +2428569|AAAAAAAAJJOAFCAA|1937-02-04|445|1936|149|1937|4|2|4|1|1937|149|1936|Thursday|1937Q1|N|N|N|2428566|2428596|2428203|2428477|N|N|N|N|N| +2428570|AAAAAAAAKJOAFCAA|1937-02-05|445|1936|149|1937|5|2|5|1|1937|149|1936|Friday|1937Q1|N|Y|N|2428566|2428596|2428204|2428478|N|N|N|N|N| +2428571|AAAAAAAALJOAFCAA|1937-02-06|445|1936|149|1937|6|2|6|1|1937|149|1936|Saturday|1937Q1|N|Y|N|2428566|2428596|2428205|2428479|N|N|N|N|N| +2428572|AAAAAAAAMJOAFCAA|1937-02-07|445|1936|149|1937|0|2|7|1|1937|149|1936|Sunday|1937Q1|N|N|N|2428566|2428596|2428206|2428480|N|N|N|N|N| +2428573|AAAAAAAANJOAFCAA|1937-02-08|445|1936|149|1937|1|2|8|1|1937|149|1936|Monday|1937Q1|N|N|N|2428566|2428596|2428207|2428481|N|N|N|N|N| +2428574|AAAAAAAAOJOAFCAA|1937-02-09|445|1937|149|1937|2|2|9|1|1937|149|1937|Tuesday|1937Q1|N|N|N|2428566|2428596|2428208|2428482|N|N|N|N|N| +2428575|AAAAAAAAPJOAFCAA|1937-02-10|445|1937|149|1937|3|2|10|1|1937|149|1937|Wednesday|1937Q1|N|N|N|2428566|2428596|2428209|2428483|N|N|N|N|N| +2428576|AAAAAAAAAKOAFCAA|1937-02-11|445|1937|149|1937|4|2|11|1|1937|149|1937|Thursday|1937Q1|N|N|N|2428566|2428596|2428210|2428484|N|N|N|N|N| +2428577|AAAAAAAABKOAFCAA|1937-02-12|445|1937|149|1937|5|2|12|1|1937|149|1937|Friday|1937Q1|N|Y|N|2428566|2428596|2428211|2428485|N|N|N|N|N| +2428578|AAAAAAAACKOAFCAA|1937-02-13|445|1937|149|1937|6|2|13|1|1937|149|1937|Saturday|1937Q1|N|Y|N|2428566|2428596|2428212|2428486|N|N|N|N|N| +2428579|AAAAAAAADKOAFCAA|1937-02-14|445|1937|149|1937|0|2|14|1|1937|149|1937|Sunday|1937Q1|N|N|N|2428566|2428596|2428213|2428487|N|N|N|N|N| +2428580|AAAAAAAAEKOAFCAA|1937-02-15|445|1937|149|1937|1|2|15|1|1937|149|1937|Monday|1937Q1|N|N|N|2428566|2428596|2428214|2428488|N|N|N|N|N| +2428581|AAAAAAAAFKOAFCAA|1937-02-16|445|1938|149|1937|2|2|16|1|1937|149|1938|Tuesday|1937Q1|N|N|N|2428566|2428596|2428215|2428489|N|N|N|N|N| +2428582|AAAAAAAAGKOAFCAA|1937-02-17|445|1938|149|1937|3|2|17|1|1937|149|1938|Wednesday|1937Q1|N|N|N|2428566|2428596|2428216|2428490|N|N|N|N|N| +2428583|AAAAAAAAHKOAFCAA|1937-02-18|445|1938|149|1937|4|2|18|1|1937|149|1938|Thursday|1937Q1|N|N|N|2428566|2428596|2428217|2428491|N|N|N|N|N| +2428584|AAAAAAAAIKOAFCAA|1937-02-19|445|1938|149|1937|5|2|19|1|1937|149|1938|Friday|1937Q1|N|Y|N|2428566|2428596|2428218|2428492|N|N|N|N|N| +2428585|AAAAAAAAJKOAFCAA|1937-02-20|445|1938|149|1937|6|2|20|1|1937|149|1938|Saturday|1937Q1|N|Y|N|2428566|2428596|2428219|2428493|N|N|N|N|N| +2428586|AAAAAAAAKKOAFCAA|1937-02-21|445|1938|149|1937|0|2|21|1|1937|149|1938|Sunday|1937Q1|N|N|N|2428566|2428596|2428220|2428494|N|N|N|N|N| +2428587|AAAAAAAALKOAFCAA|1937-02-22|445|1938|149|1937|1|2|22|1|1937|149|1938|Monday|1937Q1|N|N|N|2428566|2428596|2428221|2428495|N|N|N|N|N| +2428588|AAAAAAAAMKOAFCAA|1937-02-23|445|1939|149|1937|2|2|23|1|1937|149|1939|Tuesday|1937Q1|N|N|N|2428566|2428596|2428222|2428496|N|N|N|N|N| +2428589|AAAAAAAANKOAFCAA|1937-02-24|445|1939|149|1937|3|2|24|1|1937|149|1939|Wednesday|1937Q1|N|N|N|2428566|2428596|2428223|2428497|N|N|N|N|N| +2428590|AAAAAAAAOKOAFCAA|1937-02-25|445|1939|149|1937|4|2|25|1|1937|149|1939|Thursday|1937Q1|N|N|N|2428566|2428596|2428224|2428498|N|N|N|N|N| +2428591|AAAAAAAAPKOAFCAA|1937-02-26|445|1939|149|1937|5|2|26|1|1937|149|1939|Friday|1937Q1|N|Y|N|2428566|2428596|2428225|2428499|N|N|N|N|N| +2428592|AAAAAAAAALOAFCAA|1937-02-27|445|1939|149|1937|6|2|27|1|1937|149|1939|Saturday|1937Q1|N|Y|N|2428566|2428596|2428226|2428500|N|N|N|N|N| +2428593|AAAAAAAABLOAFCAA|1937-02-28|445|1939|149|1937|0|2|28|1|1937|149|1939|Sunday|1937Q1|N|N|N|2428566|2428596|2428227|2428501|N|N|N|N|N| +2428594|AAAAAAAACLOAFCAA|1937-03-01|446|1939|150|1937|1|3|1|1|1937|150|1939|Monday|1937Q1|N|N|N|2428594|2428652|2428229|2428502|N|N|N|N|N| +2428595|AAAAAAAADLOAFCAA|1937-03-02|446|1940|150|1937|2|3|2|1|1937|150|1940|Tuesday|1937Q1|N|N|N|2428594|2428652|2428230|2428503|N|N|N|N|N| +2428596|AAAAAAAAELOAFCAA|1937-03-03|446|1940|150|1937|3|3|3|1|1937|150|1940|Wednesday|1937Q1|N|N|N|2428594|2428652|2428231|2428504|N|N|N|N|N| +2428597|AAAAAAAAFLOAFCAA|1937-03-04|446|1940|150|1937|4|3|4|1|1937|150|1940|Thursday|1937Q1|N|N|N|2428594|2428652|2428232|2428505|N|N|N|N|N| +2428598|AAAAAAAAGLOAFCAA|1937-03-05|446|1940|150|1937|5|3|5|1|1937|150|1940|Friday|1937Q1|N|Y|N|2428594|2428652|2428233|2428506|N|N|N|N|N| +2428599|AAAAAAAAHLOAFCAA|1937-03-06|446|1940|150|1937|6|3|6|1|1937|150|1940|Saturday|1937Q1|N|Y|N|2428594|2428652|2428234|2428507|N|N|N|N|N| +2428600|AAAAAAAAILOAFCAA|1937-03-07|446|1940|150|1937|0|3|7|1|1937|150|1940|Sunday|1937Q1|N|N|N|2428594|2428652|2428235|2428508|N|N|N|N|N| +2428601|AAAAAAAAJLOAFCAA|1937-03-08|446|1940|150|1937|1|3|8|1|1937|150|1940|Monday|1937Q1|N|N|N|2428594|2428652|2428236|2428509|N|N|N|N|N| +2428602|AAAAAAAAKLOAFCAA|1937-03-09|446|1941|150|1937|2|3|9|1|1937|150|1941|Tuesday|1937Q1|N|N|N|2428594|2428652|2428237|2428510|N|N|N|N|N| +2428603|AAAAAAAALLOAFCAA|1937-03-10|446|1941|150|1937|3|3|10|1|1937|150|1941|Wednesday|1937Q1|N|N|N|2428594|2428652|2428238|2428511|N|N|N|N|N| +2428604|AAAAAAAAMLOAFCAA|1937-03-11|446|1941|150|1937|4|3|11|1|1937|150|1941|Thursday|1937Q1|N|N|N|2428594|2428652|2428239|2428512|N|N|N|N|N| +2428605|AAAAAAAANLOAFCAA|1937-03-12|446|1941|150|1937|5|3|12|1|1937|150|1941|Friday|1937Q1|N|Y|N|2428594|2428652|2428240|2428513|N|N|N|N|N| +2428606|AAAAAAAAOLOAFCAA|1937-03-13|446|1941|150|1937|6|3|13|1|1937|150|1941|Saturday|1937Q1|N|Y|N|2428594|2428652|2428241|2428514|N|N|N|N|N| +2428607|AAAAAAAAPLOAFCAA|1937-03-14|446|1941|150|1937|0|3|14|1|1937|150|1941|Sunday|1937Q1|N|N|N|2428594|2428652|2428242|2428515|N|N|N|N|N| +2428608|AAAAAAAAAMOAFCAA|1937-03-15|446|1941|150|1937|1|3|15|1|1937|150|1941|Monday|1937Q1|N|N|N|2428594|2428652|2428243|2428516|N|N|N|N|N| +2428609|AAAAAAAABMOAFCAA|1937-03-16|446|1942|150|1937|2|3|16|1|1937|150|1942|Tuesday|1937Q1|N|N|N|2428594|2428652|2428244|2428517|N|N|N|N|N| +2428610|AAAAAAAACMOAFCAA|1937-03-17|446|1942|150|1937|3|3|17|1|1937|150|1942|Wednesday|1937Q1|N|N|N|2428594|2428652|2428245|2428518|N|N|N|N|N| +2428611|AAAAAAAADMOAFCAA|1937-03-18|446|1942|150|1937|4|3|18|1|1937|150|1942|Thursday|1937Q1|N|N|N|2428594|2428652|2428246|2428519|N|N|N|N|N| +2428612|AAAAAAAAEMOAFCAA|1937-03-19|446|1942|150|1937|5|3|19|1|1937|150|1942|Friday|1937Q1|N|Y|N|2428594|2428652|2428247|2428520|N|N|N|N|N| +2428613|AAAAAAAAFMOAFCAA|1937-03-20|446|1942|150|1937|6|3|20|1|1937|150|1942|Saturday|1937Q1|N|Y|N|2428594|2428652|2428248|2428521|N|N|N|N|N| +2428614|AAAAAAAAGMOAFCAA|1937-03-21|446|1942|150|1937|0|3|21|1|1937|150|1942|Sunday|1937Q1|N|N|N|2428594|2428652|2428249|2428522|N|N|N|N|N| +2428615|AAAAAAAAHMOAFCAA|1937-03-22|446|1942|150|1937|1|3|22|1|1937|150|1942|Monday|1937Q1|N|N|N|2428594|2428652|2428250|2428523|N|N|N|N|N| +2428616|AAAAAAAAIMOAFCAA|1937-03-23|446|1943|150|1937|2|3|23|1|1937|150|1943|Tuesday|1937Q1|N|N|N|2428594|2428652|2428251|2428524|N|N|N|N|N| +2428617|AAAAAAAAJMOAFCAA|1937-03-24|446|1943|150|1937|3|3|24|1|1937|150|1943|Wednesday|1937Q1|N|N|N|2428594|2428652|2428252|2428525|N|N|N|N|N| +2428618|AAAAAAAAKMOAFCAA|1937-03-25|446|1943|150|1937|4|3|25|1|1937|150|1943|Thursday|1937Q1|N|N|N|2428594|2428652|2428253|2428526|N|N|N|N|N| +2428619|AAAAAAAALMOAFCAA|1937-03-26|446|1943|150|1937|5|3|26|1|1937|150|1943|Friday|1937Q1|N|Y|N|2428594|2428652|2428254|2428527|N|N|N|N|N| +2428620|AAAAAAAAMMOAFCAA|1937-03-27|446|1943|150|1937|6|3|27|1|1937|150|1943|Saturday|1937Q1|N|Y|N|2428594|2428652|2428255|2428528|N|N|N|N|N| +2428621|AAAAAAAANMOAFCAA|1937-03-28|446|1943|150|1937|0|3|28|1|1937|150|1943|Sunday|1937Q1|N|N|N|2428594|2428652|2428256|2428529|N|N|N|N|N| +2428622|AAAAAAAAOMOAFCAA|1937-03-29|446|1943|150|1937|1|3|29|1|1937|150|1943|Monday|1937Q1|N|N|N|2428594|2428652|2428257|2428530|N|N|N|N|N| +2428623|AAAAAAAAPMOAFCAA|1937-03-30|446|1944|150|1937|2|3|30|1|1937|150|1944|Tuesday|1937Q1|N|N|N|2428594|2428652|2428258|2428531|N|N|N|N|N| +2428624|AAAAAAAAANOAFCAA|1937-03-31|446|1944|150|1937|3|3|31|1|1937|150|1944|Wednesday|1937Q1|N|N|N|2428594|2428652|2428259|2428532|N|N|N|N|N| +2428625|AAAAAAAABNOAFCAA|1937-04-01|447|1944|150|1937|4|4|1|1|1937|150|1944|Thursday|1937Q1|N|N|N|2428625|2428714|2428260|2428535|N|N|N|N|N| +2428626|AAAAAAAACNOAFCAA|1937-04-02|447|1944|150|1937|5|4|2|2|1937|150|1944|Friday|1937Q2|N|Y|N|2428625|2428714|2428261|2428536|N|N|N|N|N| +2428627|AAAAAAAADNOAFCAA|1937-04-03|447|1944|150|1937|6|4|3|2|1937|150|1944|Saturday|1937Q2|N|Y|N|2428625|2428714|2428262|2428537|N|N|N|N|N| +2428628|AAAAAAAAENOAFCAA|1937-04-04|447|1944|150|1937|0|4|4|2|1937|150|1944|Sunday|1937Q2|N|N|N|2428625|2428714|2428263|2428538|N|N|N|N|N| +2428629|AAAAAAAAFNOAFCAA|1937-04-05|447|1944|150|1937|1|4|5|2|1937|150|1944|Monday|1937Q2|N|N|N|2428625|2428714|2428264|2428539|N|N|N|N|N| +2428630|AAAAAAAAGNOAFCAA|1937-04-06|447|1945|150|1937|2|4|6|2|1937|150|1945|Tuesday|1937Q2|N|N|N|2428625|2428714|2428265|2428540|N|N|N|N|N| +2428631|AAAAAAAAHNOAFCAA|1937-04-07|447|1945|150|1937|3|4|7|2|1937|150|1945|Wednesday|1937Q2|N|N|N|2428625|2428714|2428266|2428541|N|N|N|N|N| +2428632|AAAAAAAAINOAFCAA|1937-04-08|447|1945|150|1937|4|4|8|2|1937|150|1945|Thursday|1937Q2|N|N|N|2428625|2428714|2428267|2428542|N|N|N|N|N| +2428633|AAAAAAAAJNOAFCAA|1937-04-09|447|1945|150|1937|5|4|9|2|1937|150|1945|Friday|1937Q2|N|Y|N|2428625|2428714|2428268|2428543|N|N|N|N|N| +2428634|AAAAAAAAKNOAFCAA|1937-04-10|447|1945|150|1937|6|4|10|2|1937|150|1945|Saturday|1937Q2|N|Y|N|2428625|2428714|2428269|2428544|N|N|N|N|N| +2428635|AAAAAAAALNOAFCAA|1937-04-11|447|1945|150|1937|0|4|11|2|1937|150|1945|Sunday|1937Q2|N|N|N|2428625|2428714|2428270|2428545|N|N|N|N|N| +2428636|AAAAAAAAMNOAFCAA|1937-04-12|447|1945|150|1937|1|4|12|2|1937|150|1945|Monday|1937Q2|N|N|N|2428625|2428714|2428271|2428546|N|N|N|N|N| +2428637|AAAAAAAANNOAFCAA|1937-04-13|447|1946|150|1937|2|4|13|2|1937|150|1946|Tuesday|1937Q2|N|N|N|2428625|2428714|2428272|2428547|N|N|N|N|N| +2428638|AAAAAAAAONOAFCAA|1937-04-14|447|1946|150|1937|3|4|14|2|1937|150|1946|Wednesday|1937Q2|N|N|N|2428625|2428714|2428273|2428548|N|N|N|N|N| +2428639|AAAAAAAAPNOAFCAA|1937-04-15|447|1946|150|1937|4|4|15|2|1937|150|1946|Thursday|1937Q2|N|N|N|2428625|2428714|2428274|2428549|N|N|N|N|N| +2428640|AAAAAAAAAOOAFCAA|1937-04-16|447|1946|150|1937|5|4|16|2|1937|150|1946|Friday|1937Q2|N|Y|N|2428625|2428714|2428275|2428550|N|N|N|N|N| +2428641|AAAAAAAABOOAFCAA|1937-04-17|447|1946|150|1937|6|4|17|2|1937|150|1946|Saturday|1937Q2|N|Y|N|2428625|2428714|2428276|2428551|N|N|N|N|N| +2428642|AAAAAAAACOOAFCAA|1937-04-18|447|1946|150|1937|0|4|18|2|1937|150|1946|Sunday|1937Q2|N|N|N|2428625|2428714|2428277|2428552|N|N|N|N|N| +2428643|AAAAAAAADOOAFCAA|1937-04-19|447|1946|150|1937|1|4|19|2|1937|150|1946|Monday|1937Q2|N|N|N|2428625|2428714|2428278|2428553|N|N|N|N|N| +2428644|AAAAAAAAEOOAFCAA|1937-04-20|447|1947|150|1937|2|4|20|2|1937|150|1947|Tuesday|1937Q2|N|N|N|2428625|2428714|2428279|2428554|N|N|N|N|N| +2428645|AAAAAAAAFOOAFCAA|1937-04-21|447|1947|150|1937|3|4|21|2|1937|150|1947|Wednesday|1937Q2|N|N|N|2428625|2428714|2428280|2428555|N|N|N|N|N| +2428646|AAAAAAAAGOOAFCAA|1937-04-22|447|1947|150|1937|4|4|22|2|1937|150|1947|Thursday|1937Q2|N|N|N|2428625|2428714|2428281|2428556|N|N|N|N|N| +2428647|AAAAAAAAHOOAFCAA|1937-04-23|447|1947|150|1937|5|4|23|2|1937|150|1947|Friday|1937Q2|N|Y|N|2428625|2428714|2428282|2428557|N|N|N|N|N| +2428648|AAAAAAAAIOOAFCAA|1937-04-24|447|1947|150|1937|6|4|24|2|1937|150|1947|Saturday|1937Q2|N|Y|N|2428625|2428714|2428283|2428558|N|N|N|N|N| +2428649|AAAAAAAAJOOAFCAA|1937-04-25|447|1947|150|1937|0|4|25|2|1937|150|1947|Sunday|1937Q2|N|N|N|2428625|2428714|2428284|2428559|N|N|N|N|N| +2428650|AAAAAAAAKOOAFCAA|1937-04-26|447|1947|150|1937|1|4|26|2|1937|150|1947|Monday|1937Q2|N|N|N|2428625|2428714|2428285|2428560|N|N|N|N|N| +2428651|AAAAAAAALOOAFCAA|1937-04-27|447|1948|150|1937|2|4|27|2|1937|150|1948|Tuesday|1937Q2|N|N|N|2428625|2428714|2428286|2428561|N|N|N|N|N| +2428652|AAAAAAAAMOOAFCAA|1937-04-28|447|1948|150|1937|3|4|28|2|1937|150|1948|Wednesday|1937Q2|N|N|N|2428625|2428714|2428287|2428562|N|N|N|N|N| +2428653|AAAAAAAANOOAFCAA|1937-04-29|447|1948|150|1937|4|4|29|2|1937|150|1948|Thursday|1937Q2|N|N|N|2428625|2428714|2428288|2428563|N|N|N|N|N| +2428654|AAAAAAAAOOOAFCAA|1937-04-30|447|1948|150|1937|5|4|30|2|1937|150|1948|Friday|1937Q2|N|Y|N|2428625|2428714|2428289|2428564|N|N|N|N|N| +2428655|AAAAAAAAPOOAFCAA|1937-05-01|448|1948|150|1937|6|5|1|2|1937|150|1948|Saturday|1937Q2|N|Y|N|2428655|2428774|2428290|2428565|N|N|N|N|N| +2428656|AAAAAAAAAPOAFCAA|1937-05-02|448|1948|150|1937|0|5|2|2|1937|150|1948|Sunday|1937Q2|N|N|N|2428655|2428774|2428291|2428566|N|N|N|N|N| +2428657|AAAAAAAABPOAFCAA|1937-05-03|448|1948|150|1937|1|5|3|2|1937|150|1948|Monday|1937Q2|N|N|N|2428655|2428774|2428292|2428567|N|N|N|N|N| +2428658|AAAAAAAACPOAFCAA|1937-05-04|448|1949|150|1937|2|5|4|2|1937|150|1949|Tuesday|1937Q2|N|N|N|2428655|2428774|2428293|2428568|N|N|N|N|N| +2428659|AAAAAAAADPOAFCAA|1937-05-05|448|1949|150|1937|3|5|5|2|1937|150|1949|Wednesday|1937Q2|N|N|N|2428655|2428774|2428294|2428569|N|N|N|N|N| +2428660|AAAAAAAAEPOAFCAA|1937-05-06|448|1949|150|1937|4|5|6|2|1937|150|1949|Thursday|1937Q2|N|N|N|2428655|2428774|2428295|2428570|N|N|N|N|N| +2428661|AAAAAAAAFPOAFCAA|1937-05-07|448|1949|150|1937|5|5|7|2|1937|150|1949|Friday|1937Q2|N|Y|N|2428655|2428774|2428296|2428571|N|N|N|N|N| +2428662|AAAAAAAAGPOAFCAA|1937-05-08|448|1949|150|1937|6|5|8|2|1937|150|1949|Saturday|1937Q2|N|Y|N|2428655|2428774|2428297|2428572|N|N|N|N|N| +2428663|AAAAAAAAHPOAFCAA|1937-05-09|448|1949|150|1937|0|5|9|2|1937|150|1949|Sunday|1937Q2|N|N|N|2428655|2428774|2428298|2428573|N|N|N|N|N| +2428664|AAAAAAAAIPOAFCAA|1937-05-10|448|1949|150|1937|1|5|10|2|1937|150|1949|Monday|1937Q2|N|N|N|2428655|2428774|2428299|2428574|N|N|N|N|N| +2428665|AAAAAAAAJPOAFCAA|1937-05-11|448|1950|150|1937|2|5|11|2|1937|150|1950|Tuesday|1937Q2|N|N|N|2428655|2428774|2428300|2428575|N|N|N|N|N| +2428666|AAAAAAAAKPOAFCAA|1937-05-12|448|1950|150|1937|3|5|12|2|1937|150|1950|Wednesday|1937Q2|N|N|N|2428655|2428774|2428301|2428576|N|N|N|N|N| +2428667|AAAAAAAALPOAFCAA|1937-05-13|448|1950|150|1937|4|5|13|2|1937|150|1950|Thursday|1937Q2|N|N|N|2428655|2428774|2428302|2428577|N|N|N|N|N| +2428668|AAAAAAAAMPOAFCAA|1937-05-14|448|1950|150|1937|5|5|14|2|1937|150|1950|Friday|1937Q2|N|Y|N|2428655|2428774|2428303|2428578|N|N|N|N|N| +2428669|AAAAAAAANPOAFCAA|1937-05-15|448|1950|150|1937|6|5|15|2|1937|150|1950|Saturday|1937Q2|N|Y|N|2428655|2428774|2428304|2428579|N|N|N|N|N| +2428670|AAAAAAAAOPOAFCAA|1937-05-16|448|1950|150|1937|0|5|16|2|1937|150|1950|Sunday|1937Q2|N|N|N|2428655|2428774|2428305|2428580|N|N|N|N|N| +2428671|AAAAAAAAPPOAFCAA|1937-05-17|448|1950|150|1937|1|5|17|2|1937|150|1950|Monday|1937Q2|N|N|N|2428655|2428774|2428306|2428581|N|N|N|N|N| +2428672|AAAAAAAAAAPAFCAA|1937-05-18|448|1951|150|1937|2|5|18|2|1937|150|1951|Tuesday|1937Q2|N|N|N|2428655|2428774|2428307|2428582|N|N|N|N|N| +2428673|AAAAAAAABAPAFCAA|1937-05-19|448|1951|150|1937|3|5|19|2|1937|150|1951|Wednesday|1937Q2|N|N|N|2428655|2428774|2428308|2428583|N|N|N|N|N| +2428674|AAAAAAAACAPAFCAA|1937-05-20|448|1951|150|1937|4|5|20|2|1937|150|1951|Thursday|1937Q2|N|N|N|2428655|2428774|2428309|2428584|N|N|N|N|N| +2428675|AAAAAAAADAPAFCAA|1937-05-21|448|1951|150|1937|5|5|21|2|1937|150|1951|Friday|1937Q2|N|Y|N|2428655|2428774|2428310|2428585|N|N|N|N|N| +2428676|AAAAAAAAEAPAFCAA|1937-05-22|448|1951|150|1937|6|5|22|2|1937|150|1951|Saturday|1937Q2|N|Y|N|2428655|2428774|2428311|2428586|N|N|N|N|N| +2428677|AAAAAAAAFAPAFCAA|1937-05-23|448|1951|150|1937|0|5|23|2|1937|150|1951|Sunday|1937Q2|N|N|N|2428655|2428774|2428312|2428587|N|N|N|N|N| +2428678|AAAAAAAAGAPAFCAA|1937-05-24|448|1951|150|1937|1|5|24|2|1937|150|1951|Monday|1937Q2|N|N|N|2428655|2428774|2428313|2428588|N|N|N|N|N| +2428679|AAAAAAAAHAPAFCAA|1937-05-25|448|1952|150|1937|2|5|25|2|1937|150|1952|Tuesday|1937Q2|N|N|N|2428655|2428774|2428314|2428589|N|N|N|N|N| +2428680|AAAAAAAAIAPAFCAA|1937-05-26|448|1952|150|1937|3|5|26|2|1937|150|1952|Wednesday|1937Q2|N|N|N|2428655|2428774|2428315|2428590|N|N|N|N|N| +2428681|AAAAAAAAJAPAFCAA|1937-05-27|448|1952|150|1937|4|5|27|2|1937|150|1952|Thursday|1937Q2|N|N|N|2428655|2428774|2428316|2428591|N|N|N|N|N| +2428682|AAAAAAAAKAPAFCAA|1937-05-28|448|1952|150|1937|5|5|28|2|1937|150|1952|Friday|1937Q2|N|Y|N|2428655|2428774|2428317|2428592|N|N|N|N|N| +2428683|AAAAAAAALAPAFCAA|1937-05-29|448|1952|150|1937|6|5|29|2|1937|150|1952|Saturday|1937Q2|N|Y|N|2428655|2428774|2428318|2428593|N|N|N|N|N| +2428684|AAAAAAAAMAPAFCAA|1937-05-30|448|1952|150|1937|0|5|30|2|1937|150|1952|Sunday|1937Q2|N|N|N|2428655|2428774|2428319|2428594|N|N|N|N|N| +2428685|AAAAAAAANAPAFCAA|1937-05-31|448|1952|150|1937|1|5|31|2|1937|150|1952|Monday|1937Q2|N|N|N|2428655|2428774|2428320|2428595|N|N|N|N|N| +2428686|AAAAAAAAOAPAFCAA|1937-06-01|449|1953|151|1937|2|6|1|2|1937|151|1953|Tuesday|1937Q2|N|N|N|2428686|2428836|2428321|2428596|N|N|N|N|N| +2428687|AAAAAAAAPAPAFCAA|1937-06-02|449|1953|151|1937|3|6|2|2|1937|151|1953|Wednesday|1937Q2|N|N|N|2428686|2428836|2428322|2428597|N|N|N|N|N| +2428688|AAAAAAAAABPAFCAA|1937-06-03|449|1953|151|1937|4|6|3|2|1937|151|1953|Thursday|1937Q2|N|N|N|2428686|2428836|2428323|2428598|N|N|N|N|N| +2428689|AAAAAAAABBPAFCAA|1937-06-04|449|1953|151|1937|5|6|4|2|1937|151|1953|Friday|1937Q2|N|Y|N|2428686|2428836|2428324|2428599|N|N|N|N|N| +2428690|AAAAAAAACBPAFCAA|1937-06-05|449|1953|151|1937|6|6|5|2|1937|151|1953|Saturday|1937Q2|N|Y|N|2428686|2428836|2428325|2428600|N|N|N|N|N| +2428691|AAAAAAAADBPAFCAA|1937-06-06|449|1953|151|1937|0|6|6|2|1937|151|1953|Sunday|1937Q2|N|N|N|2428686|2428836|2428326|2428601|N|N|N|N|N| +2428692|AAAAAAAAEBPAFCAA|1937-06-07|449|1953|151|1937|1|6|7|2|1937|151|1953|Monday|1937Q2|N|N|N|2428686|2428836|2428327|2428602|N|N|N|N|N| +2428693|AAAAAAAAFBPAFCAA|1937-06-08|449|1954|151|1937|2|6|8|2|1937|151|1954|Tuesday|1937Q2|N|N|N|2428686|2428836|2428328|2428603|N|N|N|N|N| +2428694|AAAAAAAAGBPAFCAA|1937-06-09|449|1954|151|1937|3|6|9|2|1937|151|1954|Wednesday|1937Q2|N|N|N|2428686|2428836|2428329|2428604|N|N|N|N|N| +2428695|AAAAAAAAHBPAFCAA|1937-06-10|449|1954|151|1937|4|6|10|2|1937|151|1954|Thursday|1937Q2|N|N|N|2428686|2428836|2428330|2428605|N|N|N|N|N| +2428696|AAAAAAAAIBPAFCAA|1937-06-11|449|1954|151|1937|5|6|11|2|1937|151|1954|Friday|1937Q2|N|Y|N|2428686|2428836|2428331|2428606|N|N|N|N|N| +2428697|AAAAAAAAJBPAFCAA|1937-06-12|449|1954|151|1937|6|6|12|2|1937|151|1954|Saturday|1937Q2|N|Y|N|2428686|2428836|2428332|2428607|N|N|N|N|N| +2428698|AAAAAAAAKBPAFCAA|1937-06-13|449|1954|151|1937|0|6|13|2|1937|151|1954|Sunday|1937Q2|N|N|N|2428686|2428836|2428333|2428608|N|N|N|N|N| +2428699|AAAAAAAALBPAFCAA|1937-06-14|449|1954|151|1937|1|6|14|2|1937|151|1954|Monday|1937Q2|N|N|N|2428686|2428836|2428334|2428609|N|N|N|N|N| +2428700|AAAAAAAAMBPAFCAA|1937-06-15|449|1955|151|1937|2|6|15|2|1937|151|1955|Tuesday|1937Q2|N|N|N|2428686|2428836|2428335|2428610|N|N|N|N|N| +2428701|AAAAAAAANBPAFCAA|1937-06-16|449|1955|151|1937|3|6|16|2|1937|151|1955|Wednesday|1937Q2|N|N|N|2428686|2428836|2428336|2428611|N|N|N|N|N| +2428702|AAAAAAAAOBPAFCAA|1937-06-17|449|1955|151|1937|4|6|17|2|1937|151|1955|Thursday|1937Q2|N|N|N|2428686|2428836|2428337|2428612|N|N|N|N|N| +2428703|AAAAAAAAPBPAFCAA|1937-06-18|449|1955|151|1937|5|6|18|2|1937|151|1955|Friday|1937Q2|N|Y|N|2428686|2428836|2428338|2428613|N|N|N|N|N| +2428704|AAAAAAAAACPAFCAA|1937-06-19|449|1955|151|1937|6|6|19|2|1937|151|1955|Saturday|1937Q2|N|Y|N|2428686|2428836|2428339|2428614|N|N|N|N|N| +2428705|AAAAAAAABCPAFCAA|1937-06-20|449|1955|151|1937|0|6|20|2|1937|151|1955|Sunday|1937Q2|N|N|N|2428686|2428836|2428340|2428615|N|N|N|N|N| +2428706|AAAAAAAACCPAFCAA|1937-06-21|449|1955|151|1937|1|6|21|2|1937|151|1955|Monday|1937Q2|N|N|N|2428686|2428836|2428341|2428616|N|N|N|N|N| +2428707|AAAAAAAADCPAFCAA|1937-06-22|449|1956|151|1937|2|6|22|2|1937|151|1956|Tuesday|1937Q2|N|N|N|2428686|2428836|2428342|2428617|N|N|N|N|N| +2428708|AAAAAAAAECPAFCAA|1937-06-23|449|1956|151|1937|3|6|23|2|1937|151|1956|Wednesday|1937Q2|N|N|N|2428686|2428836|2428343|2428618|N|N|N|N|N| +2428709|AAAAAAAAFCPAFCAA|1937-06-24|449|1956|151|1937|4|6|24|2|1937|151|1956|Thursday|1937Q2|N|N|N|2428686|2428836|2428344|2428619|N|N|N|N|N| +2428710|AAAAAAAAGCPAFCAA|1937-06-25|449|1956|151|1937|5|6|25|2|1937|151|1956|Friday|1937Q2|N|Y|N|2428686|2428836|2428345|2428620|N|N|N|N|N| +2428711|AAAAAAAAHCPAFCAA|1937-06-26|449|1956|151|1937|6|6|26|2|1937|151|1956|Saturday|1937Q2|N|Y|N|2428686|2428836|2428346|2428621|N|N|N|N|N| +2428712|AAAAAAAAICPAFCAA|1937-06-27|449|1956|151|1937|0|6|27|2|1937|151|1956|Sunday|1937Q2|N|N|N|2428686|2428836|2428347|2428622|N|N|N|N|N| +2428713|AAAAAAAAJCPAFCAA|1937-06-28|449|1956|151|1937|1|6|28|2|1937|151|1956|Monday|1937Q2|N|N|N|2428686|2428836|2428348|2428623|N|N|N|N|N| +2428714|AAAAAAAAKCPAFCAA|1937-06-29|449|1957|151|1937|2|6|29|2|1937|151|1957|Tuesday|1937Q2|N|N|N|2428686|2428836|2428349|2428624|N|N|N|N|N| +2428715|AAAAAAAALCPAFCAA|1937-06-30|449|1957|151|1937|3|6|30|2|1937|151|1957|Wednesday|1937Q2|N|N|N|2428686|2428836|2428350|2428625|N|N|N|N|N| +2428716|AAAAAAAAMCPAFCAA|1937-07-01|450|1957|151|1937|4|7|1|2|1937|151|1957|Thursday|1937Q2|N|N|N|2428716|2428896|2428351|2428625|N|N|N|N|N| +2428717|AAAAAAAANCPAFCAA|1937-07-02|450|1957|151|1937|5|7|2|3|1937|151|1957|Friday|1937Q3|N|Y|N|2428716|2428896|2428352|2428626|N|N|N|N|N| +2428718|AAAAAAAAOCPAFCAA|1937-07-03|450|1957|151|1937|6|7|3|3|1937|151|1957|Saturday|1937Q3|N|Y|N|2428716|2428896|2428353|2428627|N|N|N|N|N| +2428719|AAAAAAAAPCPAFCAA|1937-07-04|450|1957|151|1937|0|7|4|3|1937|151|1957|Sunday|1937Q3|N|N|N|2428716|2428896|2428354|2428628|N|N|N|N|N| +2428720|AAAAAAAAADPAFCAA|1937-07-05|450|1957|151|1937|1|7|5|3|1937|151|1957|Monday|1937Q3|Y|N|N|2428716|2428896|2428355|2428629|N|N|N|N|N| +2428721|AAAAAAAABDPAFCAA|1937-07-06|450|1958|151|1937|2|7|6|3|1937|151|1958|Tuesday|1937Q3|N|N|Y|2428716|2428896|2428356|2428630|N|N|N|N|N| +2428722|AAAAAAAACDPAFCAA|1937-07-07|450|1958|151|1937|3|7|7|3|1937|151|1958|Wednesday|1937Q3|N|N|N|2428716|2428896|2428357|2428631|N|N|N|N|N| +2428723|AAAAAAAADDPAFCAA|1937-07-08|450|1958|151|1937|4|7|8|3|1937|151|1958|Thursday|1937Q3|N|N|N|2428716|2428896|2428358|2428632|N|N|N|N|N| +2428724|AAAAAAAAEDPAFCAA|1937-07-09|450|1958|151|1937|5|7|9|3|1937|151|1958|Friday|1937Q3|N|Y|N|2428716|2428896|2428359|2428633|N|N|N|N|N| +2428725|AAAAAAAAFDPAFCAA|1937-07-10|450|1958|151|1937|6|7|10|3|1937|151|1958|Saturday|1937Q3|N|Y|N|2428716|2428896|2428360|2428634|N|N|N|N|N| +2428726|AAAAAAAAGDPAFCAA|1937-07-11|450|1958|151|1937|0|7|11|3|1937|151|1958|Sunday|1937Q3|N|N|N|2428716|2428896|2428361|2428635|N|N|N|N|N| +2428727|AAAAAAAAHDPAFCAA|1937-07-12|450|1958|151|1937|1|7|12|3|1937|151|1958|Monday|1937Q3|N|N|N|2428716|2428896|2428362|2428636|N|N|N|N|N| +2428728|AAAAAAAAIDPAFCAA|1937-07-13|450|1959|151|1937|2|7|13|3|1937|151|1959|Tuesday|1937Q3|N|N|N|2428716|2428896|2428363|2428637|N|N|N|N|N| +2428729|AAAAAAAAJDPAFCAA|1937-07-14|450|1959|151|1937|3|7|14|3|1937|151|1959|Wednesday|1937Q3|N|N|N|2428716|2428896|2428364|2428638|N|N|N|N|N| +2428730|AAAAAAAAKDPAFCAA|1937-07-15|450|1959|151|1937|4|7|15|3|1937|151|1959|Thursday|1937Q3|N|N|N|2428716|2428896|2428365|2428639|N|N|N|N|N| +2428731|AAAAAAAALDPAFCAA|1937-07-16|450|1959|151|1937|5|7|16|3|1937|151|1959|Friday|1937Q3|N|Y|N|2428716|2428896|2428366|2428640|N|N|N|N|N| +2428732|AAAAAAAAMDPAFCAA|1937-07-17|450|1959|151|1937|6|7|17|3|1937|151|1959|Saturday|1937Q3|N|Y|N|2428716|2428896|2428367|2428641|N|N|N|N|N| +2428733|AAAAAAAANDPAFCAA|1937-07-18|450|1959|151|1937|0|7|18|3|1937|151|1959|Sunday|1937Q3|N|N|N|2428716|2428896|2428368|2428642|N|N|N|N|N| +2428734|AAAAAAAAODPAFCAA|1937-07-19|450|1959|151|1937|1|7|19|3|1937|151|1959|Monday|1937Q3|N|N|N|2428716|2428896|2428369|2428643|N|N|N|N|N| +2428735|AAAAAAAAPDPAFCAA|1937-07-20|450|1960|151|1937|2|7|20|3|1937|151|1960|Tuesday|1937Q3|N|N|N|2428716|2428896|2428370|2428644|N|N|N|N|N| +2428736|AAAAAAAAAEPAFCAA|1937-07-21|450|1960|151|1937|3|7|21|3|1937|151|1960|Wednesday|1937Q3|N|N|N|2428716|2428896|2428371|2428645|N|N|N|N|N| +2428737|AAAAAAAABEPAFCAA|1937-07-22|450|1960|151|1937|4|7|22|3|1937|151|1960|Thursday|1937Q3|N|N|N|2428716|2428896|2428372|2428646|N|N|N|N|N| +2428738|AAAAAAAACEPAFCAA|1937-07-23|450|1960|151|1937|5|7|23|3|1937|151|1960|Friday|1937Q3|N|Y|N|2428716|2428896|2428373|2428647|N|N|N|N|N| +2428739|AAAAAAAADEPAFCAA|1937-07-24|450|1960|151|1937|6|7|24|3|1937|151|1960|Saturday|1937Q3|N|Y|N|2428716|2428896|2428374|2428648|N|N|N|N|N| +2428740|AAAAAAAAEEPAFCAA|1937-07-25|450|1960|151|1937|0|7|25|3|1937|151|1960|Sunday|1937Q3|N|N|N|2428716|2428896|2428375|2428649|N|N|N|N|N| +2428741|AAAAAAAAFEPAFCAA|1937-07-26|450|1960|151|1937|1|7|26|3|1937|151|1960|Monday|1937Q3|N|N|N|2428716|2428896|2428376|2428650|N|N|N|N|N| +2428742|AAAAAAAAGEPAFCAA|1937-07-27|450|1961|151|1937|2|7|27|3|1937|151|1961|Tuesday|1937Q3|N|N|N|2428716|2428896|2428377|2428651|N|N|N|N|N| +2428743|AAAAAAAAHEPAFCAA|1937-07-28|450|1961|151|1937|3|7|28|3|1937|151|1961|Wednesday|1937Q3|N|N|N|2428716|2428896|2428378|2428652|N|N|N|N|N| +2428744|AAAAAAAAIEPAFCAA|1937-07-29|450|1961|151|1937|4|7|29|3|1937|151|1961|Thursday|1937Q3|N|N|N|2428716|2428896|2428379|2428653|N|N|N|N|N| +2428745|AAAAAAAAJEPAFCAA|1937-07-30|450|1961|151|1937|5|7|30|3|1937|151|1961|Friday|1937Q3|N|Y|N|2428716|2428896|2428380|2428654|N|N|N|N|N| +2428746|AAAAAAAAKEPAFCAA|1937-07-31|450|1961|151|1937|6|7|31|3|1937|151|1961|Saturday|1937Q3|N|Y|N|2428716|2428896|2428381|2428655|N|N|N|N|N| +2428747|AAAAAAAALEPAFCAA|1937-08-01|451|1961|151|1937|0|8|1|3|1937|151|1961|Sunday|1937Q3|N|N|N|2428747|2428958|2428382|2428656|N|N|N|N|N| +2428748|AAAAAAAAMEPAFCAA|1937-08-02|451|1961|151|1937|1|8|2|3|1937|151|1961|Monday|1937Q3|N|N|N|2428747|2428958|2428383|2428657|N|N|N|N|N| +2428749|AAAAAAAANEPAFCAA|1937-08-03|451|1962|151|1937|2|8|3|3|1937|151|1962|Tuesday|1937Q3|N|N|N|2428747|2428958|2428384|2428658|N|N|N|N|N| +2428750|AAAAAAAAOEPAFCAA|1937-08-04|451|1962|151|1937|3|8|4|3|1937|151|1962|Wednesday|1937Q3|N|N|N|2428747|2428958|2428385|2428659|N|N|N|N|N| +2428751|AAAAAAAAPEPAFCAA|1937-08-05|451|1962|151|1937|4|8|5|3|1937|151|1962|Thursday|1937Q3|N|N|N|2428747|2428958|2428386|2428660|N|N|N|N|N| +2428752|AAAAAAAAAFPAFCAA|1937-08-06|451|1962|151|1937|5|8|6|3|1937|151|1962|Friday|1937Q3|N|Y|N|2428747|2428958|2428387|2428661|N|N|N|N|N| +2428753|AAAAAAAABFPAFCAA|1937-08-07|451|1962|151|1937|6|8|7|3|1937|151|1962|Saturday|1937Q3|N|Y|N|2428747|2428958|2428388|2428662|N|N|N|N|N| +2428754|AAAAAAAACFPAFCAA|1937-08-08|451|1962|151|1937|0|8|8|3|1937|151|1962|Sunday|1937Q3|N|N|N|2428747|2428958|2428389|2428663|N|N|N|N|N| +2428755|AAAAAAAADFPAFCAA|1937-08-09|451|1962|151|1937|1|8|9|3|1937|151|1962|Monday|1937Q3|N|N|N|2428747|2428958|2428390|2428664|N|N|N|N|N| +2428756|AAAAAAAAEFPAFCAA|1937-08-10|451|1963|151|1937|2|8|10|3|1937|151|1963|Tuesday|1937Q3|N|N|N|2428747|2428958|2428391|2428665|N|N|N|N|N| +2428757|AAAAAAAAFFPAFCAA|1937-08-11|451|1963|151|1937|3|8|11|3|1937|151|1963|Wednesday|1937Q3|N|N|N|2428747|2428958|2428392|2428666|N|N|N|N|N| +2428758|AAAAAAAAGFPAFCAA|1937-08-12|451|1963|151|1937|4|8|12|3|1937|151|1963|Thursday|1937Q3|N|N|N|2428747|2428958|2428393|2428667|N|N|N|N|N| +2428759|AAAAAAAAHFPAFCAA|1937-08-13|451|1963|151|1937|5|8|13|3|1937|151|1963|Friday|1937Q3|N|Y|N|2428747|2428958|2428394|2428668|N|N|N|N|N| +2428760|AAAAAAAAIFPAFCAA|1937-08-14|451|1963|151|1937|6|8|14|3|1937|151|1963|Saturday|1937Q3|N|Y|N|2428747|2428958|2428395|2428669|N|N|N|N|N| +2428761|AAAAAAAAJFPAFCAA|1937-08-15|451|1963|151|1937|0|8|15|3|1937|151|1963|Sunday|1937Q3|N|N|N|2428747|2428958|2428396|2428670|N|N|N|N|N| +2428762|AAAAAAAAKFPAFCAA|1937-08-16|451|1963|151|1937|1|8|16|3|1937|151|1963|Monday|1937Q3|N|N|N|2428747|2428958|2428397|2428671|N|N|N|N|N| +2428763|AAAAAAAALFPAFCAA|1937-08-17|451|1964|151|1937|2|8|17|3|1937|151|1964|Tuesday|1937Q3|N|N|N|2428747|2428958|2428398|2428672|N|N|N|N|N| +2428764|AAAAAAAAMFPAFCAA|1937-08-18|451|1964|151|1937|3|8|18|3|1937|151|1964|Wednesday|1937Q3|N|N|N|2428747|2428958|2428399|2428673|N|N|N|N|N| +2428765|AAAAAAAANFPAFCAA|1937-08-19|451|1964|151|1937|4|8|19|3|1937|151|1964|Thursday|1937Q3|N|N|N|2428747|2428958|2428400|2428674|N|N|N|N|N| +2428766|AAAAAAAAOFPAFCAA|1937-08-20|451|1964|151|1937|5|8|20|3|1937|151|1964|Friday|1937Q3|N|Y|N|2428747|2428958|2428401|2428675|N|N|N|N|N| +2428767|AAAAAAAAPFPAFCAA|1937-08-21|451|1964|151|1937|6|8|21|3|1937|151|1964|Saturday|1937Q3|N|Y|N|2428747|2428958|2428402|2428676|N|N|N|N|N| +2428768|AAAAAAAAAGPAFCAA|1937-08-22|451|1964|151|1937|0|8|22|3|1937|151|1964|Sunday|1937Q3|N|N|N|2428747|2428958|2428403|2428677|N|N|N|N|N| +2428769|AAAAAAAABGPAFCAA|1937-08-23|451|1964|151|1937|1|8|23|3|1937|151|1964|Monday|1937Q3|N|N|N|2428747|2428958|2428404|2428678|N|N|N|N|N| +2428770|AAAAAAAACGPAFCAA|1937-08-24|451|1965|151|1937|2|8|24|3|1937|151|1965|Tuesday|1937Q3|N|N|N|2428747|2428958|2428405|2428679|N|N|N|N|N| +2428771|AAAAAAAADGPAFCAA|1937-08-25|451|1965|151|1937|3|8|25|3|1937|151|1965|Wednesday|1937Q3|N|N|N|2428747|2428958|2428406|2428680|N|N|N|N|N| +2428772|AAAAAAAAEGPAFCAA|1937-08-26|451|1965|151|1937|4|8|26|3|1937|151|1965|Thursday|1937Q3|N|N|N|2428747|2428958|2428407|2428681|N|N|N|N|N| +2428773|AAAAAAAAFGPAFCAA|1937-08-27|451|1965|151|1937|5|8|27|3|1937|151|1965|Friday|1937Q3|N|Y|N|2428747|2428958|2428408|2428682|N|N|N|N|N| +2428774|AAAAAAAAGGPAFCAA|1937-08-28|451|1965|151|1937|6|8|28|3|1937|151|1965|Saturday|1937Q3|N|Y|N|2428747|2428958|2428409|2428683|N|N|N|N|N| +2428775|AAAAAAAAHGPAFCAA|1937-08-29|451|1965|151|1937|0|8|29|3|1937|151|1965|Sunday|1937Q3|N|N|N|2428747|2428958|2428410|2428684|N|N|N|N|N| +2428776|AAAAAAAAIGPAFCAA|1937-08-30|451|1965|151|1937|1|8|30|3|1937|151|1965|Monday|1937Q3|N|N|N|2428747|2428958|2428411|2428685|N|N|N|N|N| +2428777|AAAAAAAAJGPAFCAA|1937-08-31|451|1966|151|1937|2|8|31|3|1937|151|1966|Tuesday|1937Q3|N|N|N|2428747|2428958|2428412|2428686|N|N|N|N|N| +2428778|AAAAAAAAKGPAFCAA|1937-09-01|452|1966|152|1937|3|9|1|3|1937|152|1966|Wednesday|1937Q3|N|N|N|2428778|2429020|2428413|2428687|N|N|N|N|N| +2428779|AAAAAAAALGPAFCAA|1937-09-02|452|1966|152|1937|4|9|2|3|1937|152|1966|Thursday|1937Q3|N|N|N|2428778|2429020|2428414|2428688|N|N|N|N|N| +2428780|AAAAAAAAMGPAFCAA|1937-09-03|452|1966|152|1937|5|9|3|3|1937|152|1966|Friday|1937Q3|N|Y|N|2428778|2429020|2428415|2428689|N|N|N|N|N| +2428781|AAAAAAAANGPAFCAA|1937-09-04|452|1966|152|1937|6|9|4|3|1937|152|1966|Saturday|1937Q3|N|Y|N|2428778|2429020|2428416|2428690|N|N|N|N|N| +2428782|AAAAAAAAOGPAFCAA|1937-09-05|452|1966|152|1937|0|9|5|3|1937|152|1966|Sunday|1937Q3|N|N|N|2428778|2429020|2428417|2428691|N|N|N|N|N| +2428783|AAAAAAAAPGPAFCAA|1937-09-06|452|1966|152|1937|1|9|6|3|1937|152|1966|Monday|1937Q3|N|N|N|2428778|2429020|2428418|2428692|N|N|N|N|N| +2428784|AAAAAAAAAHPAFCAA|1937-09-07|452|1967|152|1937|2|9|7|3|1937|152|1967|Tuesday|1937Q3|N|N|N|2428778|2429020|2428419|2428693|N|N|N|N|N| +2428785|AAAAAAAABHPAFCAA|1937-09-08|452|1967|152|1937|3|9|8|3|1937|152|1967|Wednesday|1937Q3|N|N|N|2428778|2429020|2428420|2428694|N|N|N|N|N| +2428786|AAAAAAAACHPAFCAA|1937-09-09|452|1967|152|1937|4|9|9|3|1937|152|1967|Thursday|1937Q3|N|N|N|2428778|2429020|2428421|2428695|N|N|N|N|N| +2428787|AAAAAAAADHPAFCAA|1937-09-10|452|1967|152|1937|5|9|10|3|1937|152|1967|Friday|1937Q3|N|Y|N|2428778|2429020|2428422|2428696|N|N|N|N|N| +2428788|AAAAAAAAEHPAFCAA|1937-09-11|452|1967|152|1937|6|9|11|3|1937|152|1967|Saturday|1937Q3|N|Y|N|2428778|2429020|2428423|2428697|N|N|N|N|N| +2428789|AAAAAAAAFHPAFCAA|1937-09-12|452|1967|152|1937|0|9|12|3|1937|152|1967|Sunday|1937Q3|N|N|N|2428778|2429020|2428424|2428698|N|N|N|N|N| +2428790|AAAAAAAAGHPAFCAA|1937-09-13|452|1967|152|1937|1|9|13|3|1937|152|1967|Monday|1937Q3|N|N|N|2428778|2429020|2428425|2428699|N|N|N|N|N| +2428791|AAAAAAAAHHPAFCAA|1937-09-14|452|1968|152|1937|2|9|14|3|1937|152|1968|Tuesday|1937Q3|N|N|N|2428778|2429020|2428426|2428700|N|N|N|N|N| +2428792|AAAAAAAAIHPAFCAA|1937-09-15|452|1968|152|1937|3|9|15|3|1937|152|1968|Wednesday|1937Q3|N|N|N|2428778|2429020|2428427|2428701|N|N|N|N|N| +2428793|AAAAAAAAJHPAFCAA|1937-09-16|452|1968|152|1937|4|9|16|3|1937|152|1968|Thursday|1937Q3|N|N|N|2428778|2429020|2428428|2428702|N|N|N|N|N| +2428794|AAAAAAAAKHPAFCAA|1937-09-17|452|1968|152|1937|5|9|17|3|1937|152|1968|Friday|1937Q3|N|Y|N|2428778|2429020|2428429|2428703|N|N|N|N|N| +2428795|AAAAAAAALHPAFCAA|1937-09-18|452|1968|152|1937|6|9|18|3|1937|152|1968|Saturday|1937Q3|N|Y|N|2428778|2429020|2428430|2428704|N|N|N|N|N| +2428796|AAAAAAAAMHPAFCAA|1937-09-19|452|1968|152|1937|0|9|19|3|1937|152|1968|Sunday|1937Q3|N|N|N|2428778|2429020|2428431|2428705|N|N|N|N|N| +2428797|AAAAAAAANHPAFCAA|1937-09-20|452|1968|152|1937|1|9|20|3|1937|152|1968|Monday|1937Q3|N|N|N|2428778|2429020|2428432|2428706|N|N|N|N|N| +2428798|AAAAAAAAOHPAFCAA|1937-09-21|452|1969|152|1937|2|9|21|3|1937|152|1969|Tuesday|1937Q3|N|N|N|2428778|2429020|2428433|2428707|N|N|N|N|N| +2428799|AAAAAAAAPHPAFCAA|1937-09-22|452|1969|152|1937|3|9|22|3|1937|152|1969|Wednesday|1937Q3|N|N|N|2428778|2429020|2428434|2428708|N|N|N|N|N| +2428800|AAAAAAAAAIPAFCAA|1937-09-23|452|1969|152|1937|4|9|23|3|1937|152|1969|Thursday|1937Q3|N|N|N|2428778|2429020|2428435|2428709|N|N|N|N|N| +2428801|AAAAAAAABIPAFCAA|1937-09-24|452|1969|152|1937|5|9|24|3|1937|152|1969|Friday|1937Q3|N|Y|N|2428778|2429020|2428436|2428710|N|N|N|N|N| +2428802|AAAAAAAACIPAFCAA|1937-09-25|452|1969|152|1937|6|9|25|3|1937|152|1969|Saturday|1937Q3|N|Y|N|2428778|2429020|2428437|2428711|N|N|N|N|N| +2428803|AAAAAAAADIPAFCAA|1937-09-26|452|1969|152|1937|0|9|26|3|1937|152|1969|Sunday|1937Q3|N|N|N|2428778|2429020|2428438|2428712|N|N|N|N|N| +2428804|AAAAAAAAEIPAFCAA|1937-09-27|452|1969|152|1937|1|9|27|3|1937|152|1969|Monday|1937Q3|N|N|N|2428778|2429020|2428439|2428713|N|N|N|N|N| +2428805|AAAAAAAAFIPAFCAA|1937-09-28|452|1970|152|1937|2|9|28|3|1937|152|1970|Tuesday|1937Q3|N|N|N|2428778|2429020|2428440|2428714|N|N|N|N|N| +2428806|AAAAAAAAGIPAFCAA|1937-09-29|452|1970|152|1937|3|9|29|3|1937|152|1970|Wednesday|1937Q3|N|N|N|2428778|2429020|2428441|2428715|N|N|N|N|N| +2428807|AAAAAAAAHIPAFCAA|1937-09-30|452|1970|152|1937|4|9|30|3|1937|152|1970|Thursday|1937Q3|N|N|N|2428778|2429020|2428442|2428716|N|N|N|N|N| +2428808|AAAAAAAAIIPAFCAA|1937-10-01|453|1970|152|1937|5|10|1|3|1937|152|1970|Friday|1937Q3|N|Y|N|2428808|2429080|2428443|2428716|N|N|N|N|N| +2428809|AAAAAAAAJIPAFCAA|1937-10-02|453|1970|152|1937|6|10|2|4|1937|152|1970|Saturday|1937Q4|N|Y|N|2428808|2429080|2428444|2428717|N|N|N|N|N| +2428810|AAAAAAAAKIPAFCAA|1937-10-03|453|1970|152|1937|0|10|3|4|1937|152|1970|Sunday|1937Q4|N|N|N|2428808|2429080|2428445|2428718|N|N|N|N|N| +2428811|AAAAAAAALIPAFCAA|1937-10-04|453|1970|152|1937|1|10|4|4|1937|152|1970|Monday|1937Q4|N|N|N|2428808|2429080|2428446|2428719|N|N|N|N|N| +2428812|AAAAAAAAMIPAFCAA|1937-10-05|453|1971|152|1937|2|10|5|4|1937|152|1971|Tuesday|1937Q4|N|N|N|2428808|2429080|2428447|2428720|N|N|N|N|N| +2428813|AAAAAAAANIPAFCAA|1937-10-06|453|1971|152|1937|3|10|6|4|1937|152|1971|Wednesday|1937Q4|N|N|N|2428808|2429080|2428448|2428721|N|N|N|N|N| +2428814|AAAAAAAAOIPAFCAA|1937-10-07|453|1971|152|1937|4|10|7|4|1937|152|1971|Thursday|1937Q4|N|N|N|2428808|2429080|2428449|2428722|N|N|N|N|N| +2428815|AAAAAAAAPIPAFCAA|1937-10-08|453|1971|152|1937|5|10|8|4|1937|152|1971|Friday|1937Q4|N|Y|N|2428808|2429080|2428450|2428723|N|N|N|N|N| +2428816|AAAAAAAAAJPAFCAA|1937-10-09|453|1971|152|1937|6|10|9|4|1937|152|1971|Saturday|1937Q4|N|Y|N|2428808|2429080|2428451|2428724|N|N|N|N|N| +2428817|AAAAAAAABJPAFCAA|1937-10-10|453|1971|152|1937|0|10|10|4|1937|152|1971|Sunday|1937Q4|N|N|N|2428808|2429080|2428452|2428725|N|N|N|N|N| +2428818|AAAAAAAACJPAFCAA|1937-10-11|453|1971|152|1937|1|10|11|4|1937|152|1971|Monday|1937Q4|N|N|N|2428808|2429080|2428453|2428726|N|N|N|N|N| +2428819|AAAAAAAADJPAFCAA|1937-10-12|453|1972|152|1937|2|10|12|4|1937|152|1972|Tuesday|1937Q4|N|N|N|2428808|2429080|2428454|2428727|N|N|N|N|N| +2428820|AAAAAAAAEJPAFCAA|1937-10-13|453|1972|152|1937|3|10|13|4|1937|152|1972|Wednesday|1937Q4|N|N|N|2428808|2429080|2428455|2428728|N|N|N|N|N| +2428821|AAAAAAAAFJPAFCAA|1937-10-14|453|1972|152|1937|4|10|14|4|1937|152|1972|Thursday|1937Q4|N|N|N|2428808|2429080|2428456|2428729|N|N|N|N|N| +2428822|AAAAAAAAGJPAFCAA|1937-10-15|453|1972|152|1937|5|10|15|4|1937|152|1972|Friday|1937Q4|N|Y|N|2428808|2429080|2428457|2428730|N|N|N|N|N| +2428823|AAAAAAAAHJPAFCAA|1937-10-16|453|1972|152|1937|6|10|16|4|1937|152|1972|Saturday|1937Q4|N|Y|N|2428808|2429080|2428458|2428731|N|N|N|N|N| +2428824|AAAAAAAAIJPAFCAA|1937-10-17|453|1972|152|1937|0|10|17|4|1937|152|1972|Sunday|1937Q4|N|N|N|2428808|2429080|2428459|2428732|N|N|N|N|N| +2428825|AAAAAAAAJJPAFCAA|1937-10-18|453|1972|152|1937|1|10|18|4|1937|152|1972|Monday|1937Q4|N|N|N|2428808|2429080|2428460|2428733|N|N|N|N|N| +2428826|AAAAAAAAKJPAFCAA|1937-10-19|453|1973|152|1937|2|10|19|4|1937|152|1973|Tuesday|1937Q4|N|N|N|2428808|2429080|2428461|2428734|N|N|N|N|N| +2428827|AAAAAAAALJPAFCAA|1937-10-20|453|1973|152|1937|3|10|20|4|1937|152|1973|Wednesday|1937Q4|N|N|N|2428808|2429080|2428462|2428735|N|N|N|N|N| +2428828|AAAAAAAAMJPAFCAA|1937-10-21|453|1973|152|1937|4|10|21|4|1937|152|1973|Thursday|1937Q4|N|N|N|2428808|2429080|2428463|2428736|N|N|N|N|N| +2428829|AAAAAAAANJPAFCAA|1937-10-22|453|1973|152|1937|5|10|22|4|1937|152|1973|Friday|1937Q4|N|Y|N|2428808|2429080|2428464|2428737|N|N|N|N|N| +2428830|AAAAAAAAOJPAFCAA|1937-10-23|453|1973|152|1937|6|10|23|4|1937|152|1973|Saturday|1937Q4|N|Y|N|2428808|2429080|2428465|2428738|N|N|N|N|N| +2428831|AAAAAAAAPJPAFCAA|1937-10-24|453|1973|152|1937|0|10|24|4|1937|152|1973|Sunday|1937Q4|N|N|N|2428808|2429080|2428466|2428739|N|N|N|N|N| +2428832|AAAAAAAAAKPAFCAA|1937-10-25|453|1973|152|1937|1|10|25|4|1937|152|1973|Monday|1937Q4|N|N|N|2428808|2429080|2428467|2428740|N|N|N|N|N| +2428833|AAAAAAAABKPAFCAA|1937-10-26|453|1974|152|1937|2|10|26|4|1937|152|1974|Tuesday|1937Q4|N|N|N|2428808|2429080|2428468|2428741|N|N|N|N|N| +2428834|AAAAAAAACKPAFCAA|1937-10-27|453|1974|152|1937|3|10|27|4|1937|152|1974|Wednesday|1937Q4|N|N|N|2428808|2429080|2428469|2428742|N|N|N|N|N| +2428835|AAAAAAAADKPAFCAA|1937-10-28|453|1974|152|1937|4|10|28|4|1937|152|1974|Thursday|1937Q4|N|N|N|2428808|2429080|2428470|2428743|N|N|N|N|N| +2428836|AAAAAAAAEKPAFCAA|1937-10-29|453|1974|152|1937|5|10|29|4|1937|152|1974|Friday|1937Q4|N|Y|N|2428808|2429080|2428471|2428744|N|N|N|N|N| +2428837|AAAAAAAAFKPAFCAA|1937-10-30|453|1974|152|1937|6|10|30|4|1937|152|1974|Saturday|1937Q4|N|Y|N|2428808|2429080|2428472|2428745|N|N|N|N|N| +2428838|AAAAAAAAGKPAFCAA|1937-10-31|453|1974|152|1937|0|10|31|4|1937|152|1974|Sunday|1937Q4|N|N|N|2428808|2429080|2428473|2428746|N|N|N|N|N| +2428839|AAAAAAAAHKPAFCAA|1937-11-01|454|1974|152|1937|1|11|1|4|1937|152|1974|Monday|1937Q4|N|N|N|2428839|2429142|2428474|2428747|N|N|N|N|N| +2428840|AAAAAAAAIKPAFCAA|1937-11-02|454|1975|152|1937|2|11|2|4|1937|152|1975|Tuesday|1937Q4|N|N|N|2428839|2429142|2428475|2428748|N|N|N|N|N| +2428841|AAAAAAAAJKPAFCAA|1937-11-03|454|1975|152|1937|3|11|3|4|1937|152|1975|Wednesday|1937Q4|N|N|N|2428839|2429142|2428476|2428749|N|N|N|N|N| +2428842|AAAAAAAAKKPAFCAA|1937-11-04|454|1975|152|1937|4|11|4|4|1937|152|1975|Thursday|1937Q4|N|N|N|2428839|2429142|2428477|2428750|N|N|N|N|N| +2428843|AAAAAAAALKPAFCAA|1937-11-05|454|1975|152|1937|5|11|5|4|1937|152|1975|Friday|1937Q4|N|Y|N|2428839|2429142|2428478|2428751|N|N|N|N|N| +2428844|AAAAAAAAMKPAFCAA|1937-11-06|454|1975|152|1937|6|11|6|4|1937|152|1975|Saturday|1937Q4|N|Y|N|2428839|2429142|2428479|2428752|N|N|N|N|N| +2428845|AAAAAAAANKPAFCAA|1937-11-07|454|1975|152|1937|0|11|7|4|1937|152|1975|Sunday|1937Q4|N|N|N|2428839|2429142|2428480|2428753|N|N|N|N|N| +2428846|AAAAAAAAOKPAFCAA|1937-11-08|454|1975|152|1937|1|11|8|4|1937|152|1975|Monday|1937Q4|N|N|N|2428839|2429142|2428481|2428754|N|N|N|N|N| +2428847|AAAAAAAAPKPAFCAA|1937-11-09|454|1976|152|1937|2|11|9|4|1937|152|1976|Tuesday|1937Q4|N|N|N|2428839|2429142|2428482|2428755|N|N|N|N|N| +2428848|AAAAAAAAALPAFCAA|1937-11-10|454|1976|152|1937|3|11|10|4|1937|152|1976|Wednesday|1937Q4|N|N|N|2428839|2429142|2428483|2428756|N|N|N|N|N| +2428849|AAAAAAAABLPAFCAA|1937-11-11|454|1976|152|1937|4|11|11|4|1937|152|1976|Thursday|1937Q4|N|N|N|2428839|2429142|2428484|2428757|N|N|N|N|N| +2428850|AAAAAAAACLPAFCAA|1937-11-12|454|1976|152|1937|5|11|12|4|1937|152|1976|Friday|1937Q4|N|Y|N|2428839|2429142|2428485|2428758|N|N|N|N|N| +2428851|AAAAAAAADLPAFCAA|1937-11-13|454|1976|152|1937|6|11|13|4|1937|152|1976|Saturday|1937Q4|N|Y|N|2428839|2429142|2428486|2428759|N|N|N|N|N| +2428852|AAAAAAAAELPAFCAA|1937-11-14|454|1976|152|1937|0|11|14|4|1937|152|1976|Sunday|1937Q4|N|N|N|2428839|2429142|2428487|2428760|N|N|N|N|N| +2428853|AAAAAAAAFLPAFCAA|1937-11-15|454|1976|152|1937|1|11|15|4|1937|152|1976|Monday|1937Q4|N|N|N|2428839|2429142|2428488|2428761|N|N|N|N|N| +2428854|AAAAAAAAGLPAFCAA|1937-11-16|454|1977|152|1937|2|11|16|4|1937|152|1977|Tuesday|1937Q4|N|N|N|2428839|2429142|2428489|2428762|N|N|N|N|N| +2428855|AAAAAAAAHLPAFCAA|1937-11-17|454|1977|152|1937|3|11|17|4|1937|152|1977|Wednesday|1937Q4|N|N|N|2428839|2429142|2428490|2428763|N|N|N|N|N| +2428856|AAAAAAAAILPAFCAA|1937-11-18|454|1977|152|1937|4|11|18|4|1937|152|1977|Thursday|1937Q4|N|N|N|2428839|2429142|2428491|2428764|N|N|N|N|N| +2428857|AAAAAAAAJLPAFCAA|1937-11-19|454|1977|152|1937|5|11|19|4|1937|152|1977|Friday|1937Q4|N|Y|N|2428839|2429142|2428492|2428765|N|N|N|N|N| +2428858|AAAAAAAAKLPAFCAA|1937-11-20|454|1977|152|1937|6|11|20|4|1937|152|1977|Saturday|1937Q4|N|Y|N|2428839|2429142|2428493|2428766|N|N|N|N|N| +2428859|AAAAAAAALLPAFCAA|1937-11-21|454|1977|152|1937|0|11|21|4|1937|152|1977|Sunday|1937Q4|N|N|N|2428839|2429142|2428494|2428767|N|N|N|N|N| +2428860|AAAAAAAAMLPAFCAA|1937-11-22|454|1977|152|1937|1|11|22|4|1937|152|1977|Monday|1937Q4|N|N|N|2428839|2429142|2428495|2428768|N|N|N|N|N| +2428861|AAAAAAAANLPAFCAA|1937-11-23|454|1978|152|1937|2|11|23|4|1937|152|1978|Tuesday|1937Q4|N|N|N|2428839|2429142|2428496|2428769|N|N|N|N|N| +2428862|AAAAAAAAOLPAFCAA|1937-11-24|454|1978|152|1937|3|11|24|4|1937|152|1978|Wednesday|1937Q4|N|N|N|2428839|2429142|2428497|2428770|N|N|N|N|N| +2428863|AAAAAAAAPLPAFCAA|1937-11-25|454|1978|152|1937|4|11|25|4|1937|152|1978|Thursday|1937Q4|N|N|N|2428839|2429142|2428498|2428771|N|N|N|N|N| +2428864|AAAAAAAAAMPAFCAA|1937-11-26|454|1978|152|1937|5|11|26|4|1937|152|1978|Friday|1937Q4|N|Y|N|2428839|2429142|2428499|2428772|N|N|N|N|N| +2428865|AAAAAAAABMPAFCAA|1937-11-27|454|1978|152|1937|6|11|27|4|1937|152|1978|Saturday|1937Q4|N|Y|N|2428839|2429142|2428500|2428773|N|N|N|N|N| +2428866|AAAAAAAACMPAFCAA|1937-11-28|454|1978|152|1937|0|11|28|4|1937|152|1978|Sunday|1937Q4|N|N|N|2428839|2429142|2428501|2428774|N|N|N|N|N| +2428867|AAAAAAAADMPAFCAA|1937-11-29|454|1978|152|1937|1|11|29|4|1937|152|1978|Monday|1937Q4|N|N|N|2428839|2429142|2428502|2428775|N|N|N|N|N| +2428868|AAAAAAAAEMPAFCAA|1937-11-30|454|1979|152|1937|2|11|30|4|1937|152|1979|Tuesday|1937Q4|N|N|N|2428839|2429142|2428503|2428776|N|N|N|N|N| +2428869|AAAAAAAAFMPAFCAA|1937-12-01|455|1979|153|1937|3|12|1|4|1937|153|1979|Wednesday|1937Q4|N|N|N|2428869|2429202|2428504|2428777|N|N|N|N|N| +2428870|AAAAAAAAGMPAFCAA|1937-12-02|455|1979|153|1937|4|12|2|4|1937|153|1979|Thursday|1937Q4|N|N|N|2428869|2429202|2428505|2428778|N|N|N|N|N| +2428871|AAAAAAAAHMPAFCAA|1937-12-03|455|1979|153|1937|5|12|3|4|1937|153|1979|Friday|1937Q4|N|Y|N|2428869|2429202|2428506|2428779|N|N|N|N|N| +2428872|AAAAAAAAIMPAFCAA|1937-12-04|455|1979|153|1937|6|12|4|4|1937|153|1979|Saturday|1937Q4|N|Y|N|2428869|2429202|2428507|2428780|N|N|N|N|N| +2428873|AAAAAAAAJMPAFCAA|1937-12-05|455|1979|153|1937|0|12|5|4|1937|153|1979|Sunday|1937Q4|N|N|N|2428869|2429202|2428508|2428781|N|N|N|N|N| +2428874|AAAAAAAAKMPAFCAA|1937-12-06|455|1979|153|1937|1|12|6|4|1937|153|1979|Monday|1937Q4|N|N|N|2428869|2429202|2428509|2428782|N|N|N|N|N| +2428875|AAAAAAAALMPAFCAA|1937-12-07|455|1980|153|1937|2|12|7|4|1937|153|1980|Tuesday|1937Q4|N|N|N|2428869|2429202|2428510|2428783|N|N|N|N|N| +2428876|AAAAAAAAMMPAFCAA|1937-12-08|455|1980|153|1937|3|12|8|4|1937|153|1980|Wednesday|1937Q4|N|N|N|2428869|2429202|2428511|2428784|N|N|N|N|N| +2428877|AAAAAAAANMPAFCAA|1937-12-09|455|1980|153|1937|4|12|9|4|1937|153|1980|Thursday|1937Q4|N|N|N|2428869|2429202|2428512|2428785|N|N|N|N|N| +2428878|AAAAAAAAOMPAFCAA|1937-12-10|455|1980|153|1937|5|12|10|4|1937|153|1980|Friday|1937Q4|N|Y|N|2428869|2429202|2428513|2428786|N|N|N|N|N| +2428879|AAAAAAAAPMPAFCAA|1937-12-11|455|1980|153|1937|6|12|11|4|1937|153|1980|Saturday|1937Q4|N|Y|N|2428869|2429202|2428514|2428787|N|N|N|N|N| +2428880|AAAAAAAAANPAFCAA|1937-12-12|455|1980|153|1937|0|12|12|4|1937|153|1980|Sunday|1937Q4|N|N|N|2428869|2429202|2428515|2428788|N|N|N|N|N| +2428881|AAAAAAAABNPAFCAA|1937-12-13|455|1980|153|1937|1|12|13|4|1937|153|1980|Monday|1937Q4|N|N|N|2428869|2429202|2428516|2428789|N|N|N|N|N| +2428882|AAAAAAAACNPAFCAA|1937-12-14|455|1981|153|1937|2|12|14|4|1937|153|1981|Tuesday|1937Q4|N|N|N|2428869|2429202|2428517|2428790|N|N|N|N|N| +2428883|AAAAAAAADNPAFCAA|1937-12-15|455|1981|153|1937|3|12|15|4|1937|153|1981|Wednesday|1937Q4|N|N|N|2428869|2429202|2428518|2428791|N|N|N|N|N| +2428884|AAAAAAAAENPAFCAA|1937-12-16|455|1981|153|1937|4|12|16|4|1937|153|1981|Thursday|1937Q4|N|N|N|2428869|2429202|2428519|2428792|N|N|N|N|N| +2428885|AAAAAAAAFNPAFCAA|1937-12-17|455|1981|153|1937|5|12|17|4|1937|153|1981|Friday|1937Q4|N|Y|N|2428869|2429202|2428520|2428793|N|N|N|N|N| +2428886|AAAAAAAAGNPAFCAA|1937-12-18|455|1981|153|1937|6|12|18|4|1937|153|1981|Saturday|1937Q4|N|Y|N|2428869|2429202|2428521|2428794|N|N|N|N|N| +2428887|AAAAAAAAHNPAFCAA|1937-12-19|455|1981|153|1937|0|12|19|4|1937|153|1981|Sunday|1937Q4|N|N|N|2428869|2429202|2428522|2428795|N|N|N|N|N| +2428888|AAAAAAAAINPAFCAA|1937-12-20|455|1981|153|1937|1|12|20|4|1937|153|1981|Monday|1937Q4|N|N|N|2428869|2429202|2428523|2428796|N|N|N|N|N| +2428889|AAAAAAAAJNPAFCAA|1937-12-21|455|1982|153|1937|2|12|21|4|1937|153|1982|Tuesday|1937Q4|N|N|N|2428869|2429202|2428524|2428797|N|N|N|N|N| +2428890|AAAAAAAAKNPAFCAA|1937-12-22|455|1982|153|1937|3|12|22|4|1937|153|1982|Wednesday|1937Q4|N|N|N|2428869|2429202|2428525|2428798|N|N|N|N|N| +2428891|AAAAAAAALNPAFCAA|1937-12-23|455|1982|153|1937|4|12|23|4|1937|153|1982|Thursday|1937Q4|N|N|N|2428869|2429202|2428526|2428799|N|N|N|N|N| +2428892|AAAAAAAAMNPAFCAA|1937-12-24|455|1982|153|1937|5|12|24|4|1937|153|1982|Friday|1937Q4|N|Y|N|2428869|2429202|2428527|2428800|N|N|N|N|N| +2428893|AAAAAAAANNPAFCAA|1937-12-25|455|1982|153|1937|6|12|25|4|1937|153|1982|Saturday|1937Q4|N|Y|N|2428869|2429202|2428528|2428801|N|N|N|N|N| +2428894|AAAAAAAAONPAFCAA|1937-12-26|455|1982|153|1937|0|12|26|4|1937|153|1982|Sunday|1937Q4|Y|N|N|2428869|2429202|2428529|2428802|N|N|N|N|N| +2428895|AAAAAAAAPNPAFCAA|1937-12-27|455|1982|153|1937|1|12|27|4|1937|153|1982|Monday|1937Q4|N|N|Y|2428869|2429202|2428530|2428803|N|N|N|N|N| +2428896|AAAAAAAAAOPAFCAA|1937-12-28|455|1983|153|1937|2|12|28|4|1937|153|1983|Tuesday|1937Q4|N|N|N|2428869|2429202|2428531|2428804|N|N|N|N|N| +2428897|AAAAAAAABOPAFCAA|1937-12-29|455|1983|153|1937|3|12|29|4|1937|153|1983|Wednesday|1937Q4|N|N|N|2428869|2429202|2428532|2428805|N|N|N|N|N| +2428898|AAAAAAAACOPAFCAA|1937-12-30|455|1983|153|1937|4|12|30|4|1937|153|1983|Thursday|1937Q4|N|N|N|2428869|2429202|2428533|2428806|N|N|N|N|N| +2428899|AAAAAAAADOPAFCAA|1937-12-31|455|1983|153|1937|5|12|31|4|1937|153|1983|Friday|1937Q4|N|Y|N|2428869|2429202|2428534|2428807|N|N|N|N|N| +2428900|AAAAAAAAEOPAFCAA|1938-01-01|456|1983|153|1938|6|1|1|1|1938|153|1983|Saturday|1938Q1|Y|Y|N|2428900|2428899|2428535|2428808|N|N|N|N|N| +2428901|AAAAAAAAFOPAFCAA|1938-01-02|456|1983|153|1938|0|1|2|1|1938|153|1983|Sunday|1938Q1|N|N|Y|2428900|2428899|2428536|2428809|N|N|N|N|N| +2428902|AAAAAAAAGOPAFCAA|1938-01-03|456|1983|153|1938|1|1|3|1|1938|153|1983|Monday|1938Q1|N|N|N|2428900|2428899|2428537|2428810|N|N|N|N|N| +2428903|AAAAAAAAHOPAFCAA|1938-01-04|456|1984|153|1938|2|1|4|1|1938|153|1984|Tuesday|1938Q1|N|N|N|2428900|2428899|2428538|2428811|N|N|N|N|N| +2428904|AAAAAAAAIOPAFCAA|1938-01-05|456|1984|153|1938|3|1|5|1|1938|153|1984|Wednesday|1938Q1|N|N|N|2428900|2428899|2428539|2428812|N|N|N|N|N| +2428905|AAAAAAAAJOPAFCAA|1938-01-06|456|1984|153|1938|4|1|6|1|1938|153|1984|Thursday|1938Q1|N|N|N|2428900|2428899|2428540|2428813|N|N|N|N|N| +2428906|AAAAAAAAKOPAFCAA|1938-01-07|456|1984|153|1938|5|1|7|1|1938|153|1984|Friday|1938Q1|N|Y|N|2428900|2428899|2428541|2428814|N|N|N|N|N| +2428907|AAAAAAAALOPAFCAA|1938-01-08|456|1984|153|1938|6|1|8|1|1938|153|1984|Saturday|1938Q1|N|Y|N|2428900|2428899|2428542|2428815|N|N|N|N|N| +2428908|AAAAAAAAMOPAFCAA|1938-01-09|456|1984|153|1938|0|1|9|1|1938|153|1984|Sunday|1938Q1|N|N|N|2428900|2428899|2428543|2428816|N|N|N|N|N| +2428909|AAAAAAAANOPAFCAA|1938-01-10|456|1984|153|1938|1|1|10|1|1938|153|1984|Monday|1938Q1|N|N|N|2428900|2428899|2428544|2428817|N|N|N|N|N| +2428910|AAAAAAAAOOPAFCAA|1938-01-11|456|1985|153|1938|2|1|11|1|1938|153|1985|Tuesday|1938Q1|N|N|N|2428900|2428899|2428545|2428818|N|N|N|N|N| +2428911|AAAAAAAAPOPAFCAA|1938-01-12|456|1985|153|1938|3|1|12|1|1938|153|1985|Wednesday|1938Q1|N|N|N|2428900|2428899|2428546|2428819|N|N|N|N|N| +2428912|AAAAAAAAAPPAFCAA|1938-01-13|456|1985|153|1938|4|1|13|1|1938|153|1985|Thursday|1938Q1|N|N|N|2428900|2428899|2428547|2428820|N|N|N|N|N| +2428913|AAAAAAAABPPAFCAA|1938-01-14|456|1985|153|1938|5|1|14|1|1938|153|1985|Friday|1938Q1|N|Y|N|2428900|2428899|2428548|2428821|N|N|N|N|N| +2428914|AAAAAAAACPPAFCAA|1938-01-15|456|1985|153|1938|6|1|15|1|1938|153|1985|Saturday|1938Q1|N|Y|N|2428900|2428899|2428549|2428822|N|N|N|N|N| +2428915|AAAAAAAADPPAFCAA|1938-01-16|456|1985|153|1938|0|1|16|1|1938|153|1985|Sunday|1938Q1|N|N|N|2428900|2428899|2428550|2428823|N|N|N|N|N| +2428916|AAAAAAAAEPPAFCAA|1938-01-17|456|1985|153|1938|1|1|17|1|1938|153|1985|Monday|1938Q1|N|N|N|2428900|2428899|2428551|2428824|N|N|N|N|N| +2428917|AAAAAAAAFPPAFCAA|1938-01-18|456|1986|153|1938|2|1|18|1|1938|153|1986|Tuesday|1938Q1|N|N|N|2428900|2428899|2428552|2428825|N|N|N|N|N| +2428918|AAAAAAAAGPPAFCAA|1938-01-19|456|1986|153|1938|3|1|19|1|1938|153|1986|Wednesday|1938Q1|N|N|N|2428900|2428899|2428553|2428826|N|N|N|N|N| +2428919|AAAAAAAAHPPAFCAA|1938-01-20|456|1986|153|1938|4|1|20|1|1938|153|1986|Thursday|1938Q1|N|N|N|2428900|2428899|2428554|2428827|N|N|N|N|N| +2428920|AAAAAAAAIPPAFCAA|1938-01-21|456|1986|153|1938|5|1|21|1|1938|153|1986|Friday|1938Q1|N|Y|N|2428900|2428899|2428555|2428828|N|N|N|N|N| +2428921|AAAAAAAAJPPAFCAA|1938-01-22|456|1986|153|1938|6|1|22|1|1938|153|1986|Saturday|1938Q1|N|Y|N|2428900|2428899|2428556|2428829|N|N|N|N|N| +2428922|AAAAAAAAKPPAFCAA|1938-01-23|456|1986|153|1938|0|1|23|1|1938|153|1986|Sunday|1938Q1|N|N|N|2428900|2428899|2428557|2428830|N|N|N|N|N| +2428923|AAAAAAAALPPAFCAA|1938-01-24|456|1986|153|1938|1|1|24|1|1938|153|1986|Monday|1938Q1|N|N|N|2428900|2428899|2428558|2428831|N|N|N|N|N| +2428924|AAAAAAAAMPPAFCAA|1938-01-25|456|1987|153|1938|2|1|25|1|1938|153|1987|Tuesday|1938Q1|N|N|N|2428900|2428899|2428559|2428832|N|N|N|N|N| +2428925|AAAAAAAANPPAFCAA|1938-01-26|456|1987|153|1938|3|1|26|1|1938|153|1987|Wednesday|1938Q1|N|N|N|2428900|2428899|2428560|2428833|N|N|N|N|N| +2428926|AAAAAAAAOPPAFCAA|1938-01-27|456|1987|153|1938|4|1|27|1|1938|153|1987|Thursday|1938Q1|N|N|N|2428900|2428899|2428561|2428834|N|N|N|N|N| +2428927|AAAAAAAAPPPAFCAA|1938-01-28|456|1987|153|1938|5|1|28|1|1938|153|1987|Friday|1938Q1|N|Y|N|2428900|2428899|2428562|2428835|N|N|N|N|N| +2428928|AAAAAAAAAAABFCAA|1938-01-29|456|1987|153|1938|6|1|29|1|1938|153|1987|Saturday|1938Q1|N|Y|N|2428900|2428899|2428563|2428836|N|N|N|N|N| +2428929|AAAAAAAABAABFCAA|1938-01-30|456|1987|153|1938|0|1|30|1|1938|153|1987|Sunday|1938Q1|N|N|N|2428900|2428899|2428564|2428837|N|N|N|N|N| +2428930|AAAAAAAACAABFCAA|1938-01-31|456|1987|153|1938|1|1|31|1|1938|153|1987|Monday|1938Q1|N|N|N|2428900|2428899|2428565|2428838|N|N|N|N|N| +2428931|AAAAAAAADAABFCAA|1938-02-01|457|1988|153|1938|2|2|1|1|1938|153|1988|Tuesday|1938Q1|N|N|N|2428931|2428961|2428566|2428839|N|N|N|N|N| +2428932|AAAAAAAAEAABFCAA|1938-02-02|457|1988|153|1938|3|2|2|1|1938|153|1988|Wednesday|1938Q1|N|N|N|2428931|2428961|2428567|2428840|N|N|N|N|N| +2428933|AAAAAAAAFAABFCAA|1938-02-03|457|1988|153|1938|4|2|3|1|1938|153|1988|Thursday|1938Q1|N|N|N|2428931|2428961|2428568|2428841|N|N|N|N|N| +2428934|AAAAAAAAGAABFCAA|1938-02-04|457|1988|153|1938|5|2|4|1|1938|153|1988|Friday|1938Q1|N|Y|N|2428931|2428961|2428569|2428842|N|N|N|N|N| +2428935|AAAAAAAAHAABFCAA|1938-02-05|457|1988|153|1938|6|2|5|1|1938|153|1988|Saturday|1938Q1|N|Y|N|2428931|2428961|2428570|2428843|N|N|N|N|N| +2428936|AAAAAAAAIAABFCAA|1938-02-06|457|1988|153|1938|0|2|6|1|1938|153|1988|Sunday|1938Q1|N|N|N|2428931|2428961|2428571|2428844|N|N|N|N|N| +2428937|AAAAAAAAJAABFCAA|1938-02-07|457|1988|153|1938|1|2|7|1|1938|153|1988|Monday|1938Q1|N|N|N|2428931|2428961|2428572|2428845|N|N|N|N|N| +2428938|AAAAAAAAKAABFCAA|1938-02-08|457|1989|153|1938|2|2|8|1|1938|153|1989|Tuesday|1938Q1|N|N|N|2428931|2428961|2428573|2428846|N|N|N|N|N| +2428939|AAAAAAAALAABFCAA|1938-02-09|457|1989|153|1938|3|2|9|1|1938|153|1989|Wednesday|1938Q1|N|N|N|2428931|2428961|2428574|2428847|N|N|N|N|N| +2428940|AAAAAAAAMAABFCAA|1938-02-10|457|1989|153|1938|4|2|10|1|1938|153|1989|Thursday|1938Q1|N|N|N|2428931|2428961|2428575|2428848|N|N|N|N|N| +2428941|AAAAAAAANAABFCAA|1938-02-11|457|1989|153|1938|5|2|11|1|1938|153|1989|Friday|1938Q1|N|Y|N|2428931|2428961|2428576|2428849|N|N|N|N|N| +2428942|AAAAAAAAOAABFCAA|1938-02-12|457|1989|153|1938|6|2|12|1|1938|153|1989|Saturday|1938Q1|N|Y|N|2428931|2428961|2428577|2428850|N|N|N|N|N| +2428943|AAAAAAAAPAABFCAA|1938-02-13|457|1989|153|1938|0|2|13|1|1938|153|1989|Sunday|1938Q1|N|N|N|2428931|2428961|2428578|2428851|N|N|N|N|N| +2428944|AAAAAAAAABABFCAA|1938-02-14|457|1989|153|1938|1|2|14|1|1938|153|1989|Monday|1938Q1|N|N|N|2428931|2428961|2428579|2428852|N|N|N|N|N| +2428945|AAAAAAAABBABFCAA|1938-02-15|457|1990|153|1938|2|2|15|1|1938|153|1990|Tuesday|1938Q1|N|N|N|2428931|2428961|2428580|2428853|N|N|N|N|N| +2428946|AAAAAAAACBABFCAA|1938-02-16|457|1990|153|1938|3|2|16|1|1938|153|1990|Wednesday|1938Q1|N|N|N|2428931|2428961|2428581|2428854|N|N|N|N|N| +2428947|AAAAAAAADBABFCAA|1938-02-17|457|1990|153|1938|4|2|17|1|1938|153|1990|Thursday|1938Q1|N|N|N|2428931|2428961|2428582|2428855|N|N|N|N|N| +2428948|AAAAAAAAEBABFCAA|1938-02-18|457|1990|153|1938|5|2|18|1|1938|153|1990|Friday|1938Q1|N|Y|N|2428931|2428961|2428583|2428856|N|N|N|N|N| +2428949|AAAAAAAAFBABFCAA|1938-02-19|457|1990|153|1938|6|2|19|1|1938|153|1990|Saturday|1938Q1|N|Y|N|2428931|2428961|2428584|2428857|N|N|N|N|N| +2428950|AAAAAAAAGBABFCAA|1938-02-20|457|1990|153|1938|0|2|20|1|1938|153|1990|Sunday|1938Q1|N|N|N|2428931|2428961|2428585|2428858|N|N|N|N|N| +2428951|AAAAAAAAHBABFCAA|1938-02-21|457|1990|153|1938|1|2|21|1|1938|153|1990|Monday|1938Q1|N|N|N|2428931|2428961|2428586|2428859|N|N|N|N|N| +2428952|AAAAAAAAIBABFCAA|1938-02-22|457|1991|153|1938|2|2|22|1|1938|153|1991|Tuesday|1938Q1|N|N|N|2428931|2428961|2428587|2428860|N|N|N|N|N| +2428953|AAAAAAAAJBABFCAA|1938-02-23|457|1991|153|1938|3|2|23|1|1938|153|1991|Wednesday|1938Q1|N|N|N|2428931|2428961|2428588|2428861|N|N|N|N|N| +2428954|AAAAAAAAKBABFCAA|1938-02-24|457|1991|153|1938|4|2|24|1|1938|153|1991|Thursday|1938Q1|N|N|N|2428931|2428961|2428589|2428862|N|N|N|N|N| +2428955|AAAAAAAALBABFCAA|1938-02-25|457|1991|153|1938|5|2|25|1|1938|153|1991|Friday|1938Q1|N|Y|N|2428931|2428961|2428590|2428863|N|N|N|N|N| +2428956|AAAAAAAAMBABFCAA|1938-02-26|457|1991|153|1938|6|2|26|1|1938|153|1991|Saturday|1938Q1|N|Y|N|2428931|2428961|2428591|2428864|N|N|N|N|N| +2428957|AAAAAAAANBABFCAA|1938-02-27|457|1991|153|1938|0|2|27|1|1938|153|1991|Sunday|1938Q1|N|N|N|2428931|2428961|2428592|2428865|N|N|N|N|N| +2428958|AAAAAAAAOBABFCAA|1938-02-28|457|1991|153|1938|1|2|28|1|1938|153|1991|Monday|1938Q1|N|N|N|2428931|2428961|2428593|2428866|N|N|N|N|N| +2428959|AAAAAAAAPBABFCAA|1938-03-01|458|1992|154|1938|2|3|1|1|1938|154|1992|Tuesday|1938Q1|N|N|N|2428959|2429017|2428594|2428867|N|N|N|N|N| +2428960|AAAAAAAAACABFCAA|1938-03-02|458|1992|154|1938|3|3|2|1|1938|154|1992|Wednesday|1938Q1|N|N|N|2428959|2429017|2428595|2428868|N|N|N|N|N| +2428961|AAAAAAAABCABFCAA|1938-03-03|458|1992|154|1938|4|3|3|1|1938|154|1992|Thursday|1938Q1|N|N|N|2428959|2429017|2428596|2428869|N|N|N|N|N| +2428962|AAAAAAAACCABFCAA|1938-03-04|458|1992|154|1938|5|3|4|1|1938|154|1992|Friday|1938Q1|N|Y|N|2428959|2429017|2428597|2428870|N|N|N|N|N| +2428963|AAAAAAAADCABFCAA|1938-03-05|458|1992|154|1938|6|3|5|1|1938|154|1992|Saturday|1938Q1|N|Y|N|2428959|2429017|2428598|2428871|N|N|N|N|N| +2428964|AAAAAAAAECABFCAA|1938-03-06|458|1992|154|1938|0|3|6|1|1938|154|1992|Sunday|1938Q1|N|N|N|2428959|2429017|2428599|2428872|N|N|N|N|N| +2428965|AAAAAAAAFCABFCAA|1938-03-07|458|1992|154|1938|1|3|7|1|1938|154|1992|Monday|1938Q1|N|N|N|2428959|2429017|2428600|2428873|N|N|N|N|N| +2428966|AAAAAAAAGCABFCAA|1938-03-08|458|1993|154|1938|2|3|8|1|1938|154|1993|Tuesday|1938Q1|N|N|N|2428959|2429017|2428601|2428874|N|N|N|N|N| +2428967|AAAAAAAAHCABFCAA|1938-03-09|458|1993|154|1938|3|3|9|1|1938|154|1993|Wednesday|1938Q1|N|N|N|2428959|2429017|2428602|2428875|N|N|N|N|N| +2428968|AAAAAAAAICABFCAA|1938-03-10|458|1993|154|1938|4|3|10|1|1938|154|1993|Thursday|1938Q1|N|N|N|2428959|2429017|2428603|2428876|N|N|N|N|N| +2428969|AAAAAAAAJCABFCAA|1938-03-11|458|1993|154|1938|5|3|11|1|1938|154|1993|Friday|1938Q1|N|Y|N|2428959|2429017|2428604|2428877|N|N|N|N|N| +2428970|AAAAAAAAKCABFCAA|1938-03-12|458|1993|154|1938|6|3|12|1|1938|154|1993|Saturday|1938Q1|N|Y|N|2428959|2429017|2428605|2428878|N|N|N|N|N| +2428971|AAAAAAAALCABFCAA|1938-03-13|458|1993|154|1938|0|3|13|1|1938|154|1993|Sunday|1938Q1|N|N|N|2428959|2429017|2428606|2428879|N|N|N|N|N| +2428972|AAAAAAAAMCABFCAA|1938-03-14|458|1993|154|1938|1|3|14|1|1938|154|1993|Monday|1938Q1|N|N|N|2428959|2429017|2428607|2428880|N|N|N|N|N| +2428973|AAAAAAAANCABFCAA|1938-03-15|458|1994|154|1938|2|3|15|1|1938|154|1994|Tuesday|1938Q1|N|N|N|2428959|2429017|2428608|2428881|N|N|N|N|N| +2428974|AAAAAAAAOCABFCAA|1938-03-16|458|1994|154|1938|3|3|16|1|1938|154|1994|Wednesday|1938Q1|N|N|N|2428959|2429017|2428609|2428882|N|N|N|N|N| +2428975|AAAAAAAAPCABFCAA|1938-03-17|458|1994|154|1938|4|3|17|1|1938|154|1994|Thursday|1938Q1|N|N|N|2428959|2429017|2428610|2428883|N|N|N|N|N| +2428976|AAAAAAAAADABFCAA|1938-03-18|458|1994|154|1938|5|3|18|1|1938|154|1994|Friday|1938Q1|N|Y|N|2428959|2429017|2428611|2428884|N|N|N|N|N| +2428977|AAAAAAAABDABFCAA|1938-03-19|458|1994|154|1938|6|3|19|1|1938|154|1994|Saturday|1938Q1|N|Y|N|2428959|2429017|2428612|2428885|N|N|N|N|N| +2428978|AAAAAAAACDABFCAA|1938-03-20|458|1994|154|1938|0|3|20|1|1938|154|1994|Sunday|1938Q1|N|N|N|2428959|2429017|2428613|2428886|N|N|N|N|N| +2428979|AAAAAAAADDABFCAA|1938-03-21|458|1994|154|1938|1|3|21|1|1938|154|1994|Monday|1938Q1|N|N|N|2428959|2429017|2428614|2428887|N|N|N|N|N| +2428980|AAAAAAAAEDABFCAA|1938-03-22|458|1995|154|1938|2|3|22|1|1938|154|1995|Tuesday|1938Q1|N|N|N|2428959|2429017|2428615|2428888|N|N|N|N|N| +2428981|AAAAAAAAFDABFCAA|1938-03-23|458|1995|154|1938|3|3|23|1|1938|154|1995|Wednesday|1938Q1|N|N|N|2428959|2429017|2428616|2428889|N|N|N|N|N| +2428982|AAAAAAAAGDABFCAA|1938-03-24|458|1995|154|1938|4|3|24|1|1938|154|1995|Thursday|1938Q1|N|N|N|2428959|2429017|2428617|2428890|N|N|N|N|N| +2428983|AAAAAAAAHDABFCAA|1938-03-25|458|1995|154|1938|5|3|25|1|1938|154|1995|Friday|1938Q1|N|Y|N|2428959|2429017|2428618|2428891|N|N|N|N|N| +2428984|AAAAAAAAIDABFCAA|1938-03-26|458|1995|154|1938|6|3|26|1|1938|154|1995|Saturday|1938Q1|N|Y|N|2428959|2429017|2428619|2428892|N|N|N|N|N| +2428985|AAAAAAAAJDABFCAA|1938-03-27|458|1995|154|1938|0|3|27|1|1938|154|1995|Sunday|1938Q1|N|N|N|2428959|2429017|2428620|2428893|N|N|N|N|N| +2428986|AAAAAAAAKDABFCAA|1938-03-28|458|1995|154|1938|1|3|28|1|1938|154|1995|Monday|1938Q1|N|N|N|2428959|2429017|2428621|2428894|N|N|N|N|N| +2428987|AAAAAAAALDABFCAA|1938-03-29|458|1996|154|1938|2|3|29|1|1938|154|1996|Tuesday|1938Q1|N|N|N|2428959|2429017|2428622|2428895|N|N|N|N|N| +2428988|AAAAAAAAMDABFCAA|1938-03-30|458|1996|154|1938|3|3|30|1|1938|154|1996|Wednesday|1938Q1|N|N|N|2428959|2429017|2428623|2428896|N|N|N|N|N| +2428989|AAAAAAAANDABFCAA|1938-03-31|458|1996|154|1938|4|3|31|1|1938|154|1996|Thursday|1938Q1|N|N|N|2428959|2429017|2428624|2428897|N|N|N|N|N| +2428990|AAAAAAAAODABFCAA|1938-04-01|459|1996|154|1938|5|4|1|1|1938|154|1996|Friday|1938Q1|N|Y|N|2428990|2429079|2428625|2428900|N|N|N|N|N| +2428991|AAAAAAAAPDABFCAA|1938-04-02|459|1996|154|1938|6|4|2|2|1938|154|1996|Saturday|1938Q2|N|Y|N|2428990|2429079|2428626|2428901|N|N|N|N|N| +2428992|AAAAAAAAAEABFCAA|1938-04-03|459|1996|154|1938|0|4|3|2|1938|154|1996|Sunday|1938Q2|N|N|N|2428990|2429079|2428627|2428902|N|N|N|N|N| +2428993|AAAAAAAABEABFCAA|1938-04-04|459|1996|154|1938|1|4|4|2|1938|154|1996|Monday|1938Q2|N|N|N|2428990|2429079|2428628|2428903|N|N|N|N|N| +2428994|AAAAAAAACEABFCAA|1938-04-05|459|1997|154|1938|2|4|5|2|1938|154|1997|Tuesday|1938Q2|N|N|N|2428990|2429079|2428629|2428904|N|N|N|N|N| +2428995|AAAAAAAADEABFCAA|1938-04-06|459|1997|154|1938|3|4|6|2|1938|154|1997|Wednesday|1938Q2|N|N|N|2428990|2429079|2428630|2428905|N|N|N|N|N| +2428996|AAAAAAAAEEABFCAA|1938-04-07|459|1997|154|1938|4|4|7|2|1938|154|1997|Thursday|1938Q2|N|N|N|2428990|2429079|2428631|2428906|N|N|N|N|N| +2428997|AAAAAAAAFEABFCAA|1938-04-08|459|1997|154|1938|5|4|8|2|1938|154|1997|Friday|1938Q2|N|Y|N|2428990|2429079|2428632|2428907|N|N|N|N|N| +2428998|AAAAAAAAGEABFCAA|1938-04-09|459|1997|154|1938|6|4|9|2|1938|154|1997|Saturday|1938Q2|N|Y|N|2428990|2429079|2428633|2428908|N|N|N|N|N| +2428999|AAAAAAAAHEABFCAA|1938-04-10|459|1997|154|1938|0|4|10|2|1938|154|1997|Sunday|1938Q2|N|N|N|2428990|2429079|2428634|2428909|N|N|N|N|N| +2429000|AAAAAAAAIEABFCAA|1938-04-11|459|1997|154|1938|1|4|11|2|1938|154|1997|Monday|1938Q2|N|N|N|2428990|2429079|2428635|2428910|N|N|N|N|N| +2429001|AAAAAAAAJEABFCAA|1938-04-12|459|1998|154|1938|2|4|12|2|1938|154|1998|Tuesday|1938Q2|N|N|N|2428990|2429079|2428636|2428911|N|N|N|N|N| +2429002|AAAAAAAAKEABFCAA|1938-04-13|459|1998|154|1938|3|4|13|2|1938|154|1998|Wednesday|1938Q2|N|N|N|2428990|2429079|2428637|2428912|N|N|N|N|N| +2429003|AAAAAAAALEABFCAA|1938-04-14|459|1998|154|1938|4|4|14|2|1938|154|1998|Thursday|1938Q2|N|N|N|2428990|2429079|2428638|2428913|N|N|N|N|N| +2429004|AAAAAAAAMEABFCAA|1938-04-15|459|1998|154|1938|5|4|15|2|1938|154|1998|Friday|1938Q2|N|Y|N|2428990|2429079|2428639|2428914|N|N|N|N|N| +2429005|AAAAAAAANEABFCAA|1938-04-16|459|1998|154|1938|6|4|16|2|1938|154|1998|Saturday|1938Q2|N|Y|N|2428990|2429079|2428640|2428915|N|N|N|N|N| +2429006|AAAAAAAAOEABFCAA|1938-04-17|459|1998|154|1938|0|4|17|2|1938|154|1998|Sunday|1938Q2|N|N|N|2428990|2429079|2428641|2428916|N|N|N|N|N| +2429007|AAAAAAAAPEABFCAA|1938-04-18|459|1998|154|1938|1|4|18|2|1938|154|1998|Monday|1938Q2|N|N|N|2428990|2429079|2428642|2428917|N|N|N|N|N| +2429008|AAAAAAAAAFABFCAA|1938-04-19|459|1999|154|1938|2|4|19|2|1938|154|1999|Tuesday|1938Q2|N|N|N|2428990|2429079|2428643|2428918|N|N|N|N|N| +2429009|AAAAAAAABFABFCAA|1938-04-20|459|1999|154|1938|3|4|20|2|1938|154|1999|Wednesday|1938Q2|N|N|N|2428990|2429079|2428644|2428919|N|N|N|N|N| +2429010|AAAAAAAACFABFCAA|1938-04-21|459|1999|154|1938|4|4|21|2|1938|154|1999|Thursday|1938Q2|N|N|N|2428990|2429079|2428645|2428920|N|N|N|N|N| +2429011|AAAAAAAADFABFCAA|1938-04-22|459|1999|154|1938|5|4|22|2|1938|154|1999|Friday|1938Q2|N|Y|N|2428990|2429079|2428646|2428921|N|N|N|N|N| +2429012|AAAAAAAAEFABFCAA|1938-04-23|459|1999|154|1938|6|4|23|2|1938|154|1999|Saturday|1938Q2|N|Y|N|2428990|2429079|2428647|2428922|N|N|N|N|N| +2429013|AAAAAAAAFFABFCAA|1938-04-24|459|1999|154|1938|0|4|24|2|1938|154|1999|Sunday|1938Q2|N|N|N|2428990|2429079|2428648|2428923|N|N|N|N|N| +2429014|AAAAAAAAGFABFCAA|1938-04-25|459|1999|154|1938|1|4|25|2|1938|154|1999|Monday|1938Q2|N|N|N|2428990|2429079|2428649|2428924|N|N|N|N|N| +2429015|AAAAAAAAHFABFCAA|1938-04-26|459|2000|154|1938|2|4|26|2|1938|154|2000|Tuesday|1938Q2|N|N|N|2428990|2429079|2428650|2428925|N|N|N|N|N| +2429016|AAAAAAAAIFABFCAA|1938-04-27|459|2000|154|1938|3|4|27|2|1938|154|2000|Wednesday|1938Q2|N|N|N|2428990|2429079|2428651|2428926|N|N|N|N|N| +2429017|AAAAAAAAJFABFCAA|1938-04-28|459|2000|154|1938|4|4|28|2|1938|154|2000|Thursday|1938Q2|N|N|N|2428990|2429079|2428652|2428927|N|N|N|N|N| +2429018|AAAAAAAAKFABFCAA|1938-04-29|459|2000|154|1938|5|4|29|2|1938|154|2000|Friday|1938Q2|N|Y|N|2428990|2429079|2428653|2428928|N|N|N|N|N| +2429019|AAAAAAAALFABFCAA|1938-04-30|459|2000|154|1938|6|4|30|2|1938|154|2000|Saturday|1938Q2|N|Y|N|2428990|2429079|2428654|2428929|N|N|N|N|N| +2429020|AAAAAAAAMFABFCAA|1938-05-01|460|2000|154|1938|0|5|1|2|1938|154|2000|Sunday|1938Q2|N|N|N|2429020|2429139|2428655|2428930|N|N|N|N|N| +2429021|AAAAAAAANFABFCAA|1938-05-02|460|2000|154|1938|1|5|2|2|1938|154|2000|Monday|1938Q2|N|N|N|2429020|2429139|2428656|2428931|N|N|N|N|N| +2429022|AAAAAAAAOFABFCAA|1938-05-03|460|2001|154|1938|2|5|3|2|1938|154|2001|Tuesday|1938Q2|N|N|N|2429020|2429139|2428657|2428932|N|N|N|N|N| +2429023|AAAAAAAAPFABFCAA|1938-05-04|460|2001|154|1938|3|5|4|2|1938|154|2001|Wednesday|1938Q2|N|N|N|2429020|2429139|2428658|2428933|N|N|N|N|N| +2429024|AAAAAAAAAGABFCAA|1938-05-05|460|2001|154|1938|4|5|5|2|1938|154|2001|Thursday|1938Q2|N|N|N|2429020|2429139|2428659|2428934|N|N|N|N|N| +2429025|AAAAAAAABGABFCAA|1938-05-06|460|2001|154|1938|5|5|6|2|1938|154|2001|Friday|1938Q2|N|Y|N|2429020|2429139|2428660|2428935|N|N|N|N|N| +2429026|AAAAAAAACGABFCAA|1938-05-07|460|2001|154|1938|6|5|7|2|1938|154|2001|Saturday|1938Q2|N|Y|N|2429020|2429139|2428661|2428936|N|N|N|N|N| +2429027|AAAAAAAADGABFCAA|1938-05-08|460|2001|154|1938|0|5|8|2|1938|154|2001|Sunday|1938Q2|N|N|N|2429020|2429139|2428662|2428937|N|N|N|N|N| +2429028|AAAAAAAAEGABFCAA|1938-05-09|460|2001|154|1938|1|5|9|2|1938|154|2001|Monday|1938Q2|N|N|N|2429020|2429139|2428663|2428938|N|N|N|N|N| +2429029|AAAAAAAAFGABFCAA|1938-05-10|460|2002|154|1938|2|5|10|2|1938|154|2002|Tuesday|1938Q2|N|N|N|2429020|2429139|2428664|2428939|N|N|N|N|N| +2429030|AAAAAAAAGGABFCAA|1938-05-11|460|2002|154|1938|3|5|11|2|1938|154|2002|Wednesday|1938Q2|N|N|N|2429020|2429139|2428665|2428940|N|N|N|N|N| +2429031|AAAAAAAAHGABFCAA|1938-05-12|460|2002|154|1938|4|5|12|2|1938|154|2002|Thursday|1938Q2|N|N|N|2429020|2429139|2428666|2428941|N|N|N|N|N| +2429032|AAAAAAAAIGABFCAA|1938-05-13|460|2002|154|1938|5|5|13|2|1938|154|2002|Friday|1938Q2|N|Y|N|2429020|2429139|2428667|2428942|N|N|N|N|N| +2429033|AAAAAAAAJGABFCAA|1938-05-14|460|2002|154|1938|6|5|14|2|1938|154|2002|Saturday|1938Q2|N|Y|N|2429020|2429139|2428668|2428943|N|N|N|N|N| +2429034|AAAAAAAAKGABFCAA|1938-05-15|460|2002|154|1938|0|5|15|2|1938|154|2002|Sunday|1938Q2|N|N|N|2429020|2429139|2428669|2428944|N|N|N|N|N| +2429035|AAAAAAAALGABFCAA|1938-05-16|460|2002|154|1938|1|5|16|2|1938|154|2002|Monday|1938Q2|N|N|N|2429020|2429139|2428670|2428945|N|N|N|N|N| +2429036|AAAAAAAAMGABFCAA|1938-05-17|460|2003|154|1938|2|5|17|2|1938|154|2003|Tuesday|1938Q2|N|N|N|2429020|2429139|2428671|2428946|N|N|N|N|N| +2429037|AAAAAAAANGABFCAA|1938-05-18|460|2003|154|1938|3|5|18|2|1938|154|2003|Wednesday|1938Q2|N|N|N|2429020|2429139|2428672|2428947|N|N|N|N|N| +2429038|AAAAAAAAOGABFCAA|1938-05-19|460|2003|154|1938|4|5|19|2|1938|154|2003|Thursday|1938Q2|N|N|N|2429020|2429139|2428673|2428948|N|N|N|N|N| +2429039|AAAAAAAAPGABFCAA|1938-05-20|460|2003|154|1938|5|5|20|2|1938|154|2003|Friday|1938Q2|N|Y|N|2429020|2429139|2428674|2428949|N|N|N|N|N| +2429040|AAAAAAAAAHABFCAA|1938-05-21|460|2003|154|1938|6|5|21|2|1938|154|2003|Saturday|1938Q2|N|Y|N|2429020|2429139|2428675|2428950|N|N|N|N|N| +2429041|AAAAAAAABHABFCAA|1938-05-22|460|2003|154|1938|0|5|22|2|1938|154|2003|Sunday|1938Q2|N|N|N|2429020|2429139|2428676|2428951|N|N|N|N|N| +2429042|AAAAAAAACHABFCAA|1938-05-23|460|2003|154|1938|1|5|23|2|1938|154|2003|Monday|1938Q2|N|N|N|2429020|2429139|2428677|2428952|N|N|N|N|N| +2429043|AAAAAAAADHABFCAA|1938-05-24|460|2004|154|1938|2|5|24|2|1938|154|2004|Tuesday|1938Q2|N|N|N|2429020|2429139|2428678|2428953|N|N|N|N|N| +2429044|AAAAAAAAEHABFCAA|1938-05-25|460|2004|154|1938|3|5|25|2|1938|154|2004|Wednesday|1938Q2|N|N|N|2429020|2429139|2428679|2428954|N|N|N|N|N| +2429045|AAAAAAAAFHABFCAA|1938-05-26|460|2004|154|1938|4|5|26|2|1938|154|2004|Thursday|1938Q2|N|N|N|2429020|2429139|2428680|2428955|N|N|N|N|N| +2429046|AAAAAAAAGHABFCAA|1938-05-27|460|2004|154|1938|5|5|27|2|1938|154|2004|Friday|1938Q2|N|Y|N|2429020|2429139|2428681|2428956|N|N|N|N|N| +2429047|AAAAAAAAHHABFCAA|1938-05-28|460|2004|154|1938|6|5|28|2|1938|154|2004|Saturday|1938Q2|N|Y|N|2429020|2429139|2428682|2428957|N|N|N|N|N| +2429048|AAAAAAAAIHABFCAA|1938-05-29|460|2004|154|1938|0|5|29|2|1938|154|2004|Sunday|1938Q2|N|N|N|2429020|2429139|2428683|2428958|N|N|N|N|N| +2429049|AAAAAAAAJHABFCAA|1938-05-30|460|2004|154|1938|1|5|30|2|1938|154|2004|Monday|1938Q2|N|N|N|2429020|2429139|2428684|2428959|N|N|N|N|N| +2429050|AAAAAAAAKHABFCAA|1938-05-31|460|2005|154|1938|2|5|31|2|1938|154|2005|Tuesday|1938Q2|N|N|N|2429020|2429139|2428685|2428960|N|N|N|N|N| +2429051|AAAAAAAALHABFCAA|1938-06-01|461|2005|155|1938|3|6|1|2|1938|155|2005|Wednesday|1938Q2|N|N|N|2429051|2429201|2428686|2428961|N|N|N|N|N| +2429052|AAAAAAAAMHABFCAA|1938-06-02|461|2005|155|1938|4|6|2|2|1938|155|2005|Thursday|1938Q2|N|N|N|2429051|2429201|2428687|2428962|N|N|N|N|N| +2429053|AAAAAAAANHABFCAA|1938-06-03|461|2005|155|1938|5|6|3|2|1938|155|2005|Friday|1938Q2|N|Y|N|2429051|2429201|2428688|2428963|N|N|N|N|N| +2429054|AAAAAAAAOHABFCAA|1938-06-04|461|2005|155|1938|6|6|4|2|1938|155|2005|Saturday|1938Q2|N|Y|N|2429051|2429201|2428689|2428964|N|N|N|N|N| +2429055|AAAAAAAAPHABFCAA|1938-06-05|461|2005|155|1938|0|6|5|2|1938|155|2005|Sunday|1938Q2|N|N|N|2429051|2429201|2428690|2428965|N|N|N|N|N| +2429056|AAAAAAAAAIABFCAA|1938-06-06|461|2005|155|1938|1|6|6|2|1938|155|2005|Monday|1938Q2|N|N|N|2429051|2429201|2428691|2428966|N|N|N|N|N| +2429057|AAAAAAAABIABFCAA|1938-06-07|461|2006|155|1938|2|6|7|2|1938|155|2006|Tuesday|1938Q2|N|N|N|2429051|2429201|2428692|2428967|N|N|N|N|N| +2429058|AAAAAAAACIABFCAA|1938-06-08|461|2006|155|1938|3|6|8|2|1938|155|2006|Wednesday|1938Q2|N|N|N|2429051|2429201|2428693|2428968|N|N|N|N|N| +2429059|AAAAAAAADIABFCAA|1938-06-09|461|2006|155|1938|4|6|9|2|1938|155|2006|Thursday|1938Q2|N|N|N|2429051|2429201|2428694|2428969|N|N|N|N|N| +2429060|AAAAAAAAEIABFCAA|1938-06-10|461|2006|155|1938|5|6|10|2|1938|155|2006|Friday|1938Q2|N|Y|N|2429051|2429201|2428695|2428970|N|N|N|N|N| +2429061|AAAAAAAAFIABFCAA|1938-06-11|461|2006|155|1938|6|6|11|2|1938|155|2006|Saturday|1938Q2|N|Y|N|2429051|2429201|2428696|2428971|N|N|N|N|N| +2429062|AAAAAAAAGIABFCAA|1938-06-12|461|2006|155|1938|0|6|12|2|1938|155|2006|Sunday|1938Q2|N|N|N|2429051|2429201|2428697|2428972|N|N|N|N|N| +2429063|AAAAAAAAHIABFCAA|1938-06-13|461|2006|155|1938|1|6|13|2|1938|155|2006|Monday|1938Q2|N|N|N|2429051|2429201|2428698|2428973|N|N|N|N|N| +2429064|AAAAAAAAIIABFCAA|1938-06-14|461|2007|155|1938|2|6|14|2|1938|155|2007|Tuesday|1938Q2|N|N|N|2429051|2429201|2428699|2428974|N|N|N|N|N| +2429065|AAAAAAAAJIABFCAA|1938-06-15|461|2007|155|1938|3|6|15|2|1938|155|2007|Wednesday|1938Q2|N|N|N|2429051|2429201|2428700|2428975|N|N|N|N|N| +2429066|AAAAAAAAKIABFCAA|1938-06-16|461|2007|155|1938|4|6|16|2|1938|155|2007|Thursday|1938Q2|N|N|N|2429051|2429201|2428701|2428976|N|N|N|N|N| +2429067|AAAAAAAALIABFCAA|1938-06-17|461|2007|155|1938|5|6|17|2|1938|155|2007|Friday|1938Q2|N|Y|N|2429051|2429201|2428702|2428977|N|N|N|N|N| +2429068|AAAAAAAAMIABFCAA|1938-06-18|461|2007|155|1938|6|6|18|2|1938|155|2007|Saturday|1938Q2|N|Y|N|2429051|2429201|2428703|2428978|N|N|N|N|N| +2429069|AAAAAAAANIABFCAA|1938-06-19|461|2007|155|1938|0|6|19|2|1938|155|2007|Sunday|1938Q2|N|N|N|2429051|2429201|2428704|2428979|N|N|N|N|N| +2429070|AAAAAAAAOIABFCAA|1938-06-20|461|2007|155|1938|1|6|20|2|1938|155|2007|Monday|1938Q2|N|N|N|2429051|2429201|2428705|2428980|N|N|N|N|N| +2429071|AAAAAAAAPIABFCAA|1938-06-21|461|2008|155|1938|2|6|21|2|1938|155|2008|Tuesday|1938Q2|N|N|N|2429051|2429201|2428706|2428981|N|N|N|N|N| +2429072|AAAAAAAAAJABFCAA|1938-06-22|461|2008|155|1938|3|6|22|2|1938|155|2008|Wednesday|1938Q2|N|N|N|2429051|2429201|2428707|2428982|N|N|N|N|N| +2429073|AAAAAAAABJABFCAA|1938-06-23|461|2008|155|1938|4|6|23|2|1938|155|2008|Thursday|1938Q2|N|N|N|2429051|2429201|2428708|2428983|N|N|N|N|N| +2429074|AAAAAAAACJABFCAA|1938-06-24|461|2008|155|1938|5|6|24|2|1938|155|2008|Friday|1938Q2|N|Y|N|2429051|2429201|2428709|2428984|N|N|N|N|N| +2429075|AAAAAAAADJABFCAA|1938-06-25|461|2008|155|1938|6|6|25|2|1938|155|2008|Saturday|1938Q2|N|Y|N|2429051|2429201|2428710|2428985|N|N|N|N|N| +2429076|AAAAAAAAEJABFCAA|1938-06-26|461|2008|155|1938|0|6|26|2|1938|155|2008|Sunday|1938Q2|N|N|N|2429051|2429201|2428711|2428986|N|N|N|N|N| +2429077|AAAAAAAAFJABFCAA|1938-06-27|461|2008|155|1938|1|6|27|2|1938|155|2008|Monday|1938Q2|N|N|N|2429051|2429201|2428712|2428987|N|N|N|N|N| +2429078|AAAAAAAAGJABFCAA|1938-06-28|461|2009|155|1938|2|6|28|2|1938|155|2009|Tuesday|1938Q2|N|N|N|2429051|2429201|2428713|2428988|N|N|N|N|N| +2429079|AAAAAAAAHJABFCAA|1938-06-29|461|2009|155|1938|3|6|29|2|1938|155|2009|Wednesday|1938Q2|N|N|N|2429051|2429201|2428714|2428989|N|N|N|N|N| +2429080|AAAAAAAAIJABFCAA|1938-06-30|461|2009|155|1938|4|6|30|2|1938|155|2009|Thursday|1938Q2|N|N|N|2429051|2429201|2428715|2428990|N|N|N|N|N| +2429081|AAAAAAAAJJABFCAA|1938-07-01|462|2009|155|1938|5|7|1|2|1938|155|2009|Friday|1938Q2|N|Y|N|2429081|2429261|2428716|2428990|N|N|N|N|N| +2429082|AAAAAAAAKJABFCAA|1938-07-02|462|2009|155|1938|6|7|2|3|1938|155|2009|Saturday|1938Q3|N|Y|N|2429081|2429261|2428717|2428991|N|N|N|N|N| +2429083|AAAAAAAALJABFCAA|1938-07-03|462|2009|155|1938|0|7|3|3|1938|155|2009|Sunday|1938Q3|N|N|N|2429081|2429261|2428718|2428992|N|N|N|N|N| +2429084|AAAAAAAAMJABFCAA|1938-07-04|462|2009|155|1938|1|7|4|3|1938|155|2009|Monday|1938Q3|N|N|N|2429081|2429261|2428719|2428993|N|N|N|N|N| +2429085|AAAAAAAANJABFCAA|1938-07-05|462|2010|155|1938|2|7|5|3|1938|155|2010|Tuesday|1938Q3|Y|N|N|2429081|2429261|2428720|2428994|N|N|N|N|N| +2429086|AAAAAAAAOJABFCAA|1938-07-06|462|2010|155|1938|3|7|6|3|1938|155|2010|Wednesday|1938Q3|N|N|Y|2429081|2429261|2428721|2428995|N|N|N|N|N| +2429087|AAAAAAAAPJABFCAA|1938-07-07|462|2010|155|1938|4|7|7|3|1938|155|2010|Thursday|1938Q3|N|N|N|2429081|2429261|2428722|2428996|N|N|N|N|N| +2429088|AAAAAAAAAKABFCAA|1938-07-08|462|2010|155|1938|5|7|8|3|1938|155|2010|Friday|1938Q3|N|Y|N|2429081|2429261|2428723|2428997|N|N|N|N|N| +2429089|AAAAAAAABKABFCAA|1938-07-09|462|2010|155|1938|6|7|9|3|1938|155|2010|Saturday|1938Q3|N|Y|N|2429081|2429261|2428724|2428998|N|N|N|N|N| +2429090|AAAAAAAACKABFCAA|1938-07-10|462|2010|155|1938|0|7|10|3|1938|155|2010|Sunday|1938Q3|N|N|N|2429081|2429261|2428725|2428999|N|N|N|N|N| +2429091|AAAAAAAADKABFCAA|1938-07-11|462|2010|155|1938|1|7|11|3|1938|155|2010|Monday|1938Q3|N|N|N|2429081|2429261|2428726|2429000|N|N|N|N|N| +2429092|AAAAAAAAEKABFCAA|1938-07-12|462|2011|155|1938|2|7|12|3|1938|155|2011|Tuesday|1938Q3|N|N|N|2429081|2429261|2428727|2429001|N|N|N|N|N| +2429093|AAAAAAAAFKABFCAA|1938-07-13|462|2011|155|1938|3|7|13|3|1938|155|2011|Wednesday|1938Q3|N|N|N|2429081|2429261|2428728|2429002|N|N|N|N|N| +2429094|AAAAAAAAGKABFCAA|1938-07-14|462|2011|155|1938|4|7|14|3|1938|155|2011|Thursday|1938Q3|N|N|N|2429081|2429261|2428729|2429003|N|N|N|N|N| +2429095|AAAAAAAAHKABFCAA|1938-07-15|462|2011|155|1938|5|7|15|3|1938|155|2011|Friday|1938Q3|N|Y|N|2429081|2429261|2428730|2429004|N|N|N|N|N| +2429096|AAAAAAAAIKABFCAA|1938-07-16|462|2011|155|1938|6|7|16|3|1938|155|2011|Saturday|1938Q3|N|Y|N|2429081|2429261|2428731|2429005|N|N|N|N|N| +2429097|AAAAAAAAJKABFCAA|1938-07-17|462|2011|155|1938|0|7|17|3|1938|155|2011|Sunday|1938Q3|N|N|N|2429081|2429261|2428732|2429006|N|N|N|N|N| +2429098|AAAAAAAAKKABFCAA|1938-07-18|462|2011|155|1938|1|7|18|3|1938|155|2011|Monday|1938Q3|N|N|N|2429081|2429261|2428733|2429007|N|N|N|N|N| +2429099|AAAAAAAALKABFCAA|1938-07-19|462|2012|155|1938|2|7|19|3|1938|155|2012|Tuesday|1938Q3|N|N|N|2429081|2429261|2428734|2429008|N|N|N|N|N| +2429100|AAAAAAAAMKABFCAA|1938-07-20|462|2012|155|1938|3|7|20|3|1938|155|2012|Wednesday|1938Q3|N|N|N|2429081|2429261|2428735|2429009|N|N|N|N|N| +2429101|AAAAAAAANKABFCAA|1938-07-21|462|2012|155|1938|4|7|21|3|1938|155|2012|Thursday|1938Q3|N|N|N|2429081|2429261|2428736|2429010|N|N|N|N|N| +2429102|AAAAAAAAOKABFCAA|1938-07-22|462|2012|155|1938|5|7|22|3|1938|155|2012|Friday|1938Q3|N|Y|N|2429081|2429261|2428737|2429011|N|N|N|N|N| +2429103|AAAAAAAAPKABFCAA|1938-07-23|462|2012|155|1938|6|7|23|3|1938|155|2012|Saturday|1938Q3|N|Y|N|2429081|2429261|2428738|2429012|N|N|N|N|N| +2429104|AAAAAAAAALABFCAA|1938-07-24|462|2012|155|1938|0|7|24|3|1938|155|2012|Sunday|1938Q3|N|N|N|2429081|2429261|2428739|2429013|N|N|N|N|N| +2429105|AAAAAAAABLABFCAA|1938-07-25|462|2012|155|1938|1|7|25|3|1938|155|2012|Monday|1938Q3|N|N|N|2429081|2429261|2428740|2429014|N|N|N|N|N| +2429106|AAAAAAAACLABFCAA|1938-07-26|462|2013|155|1938|2|7|26|3|1938|155|2013|Tuesday|1938Q3|N|N|N|2429081|2429261|2428741|2429015|N|N|N|N|N| +2429107|AAAAAAAADLABFCAA|1938-07-27|462|2013|155|1938|3|7|27|3|1938|155|2013|Wednesday|1938Q3|N|N|N|2429081|2429261|2428742|2429016|N|N|N|N|N| +2429108|AAAAAAAAELABFCAA|1938-07-28|462|2013|155|1938|4|7|28|3|1938|155|2013|Thursday|1938Q3|N|N|N|2429081|2429261|2428743|2429017|N|N|N|N|N| +2429109|AAAAAAAAFLABFCAA|1938-07-29|462|2013|155|1938|5|7|29|3|1938|155|2013|Friday|1938Q3|N|Y|N|2429081|2429261|2428744|2429018|N|N|N|N|N| +2429110|AAAAAAAAGLABFCAA|1938-07-30|462|2013|155|1938|6|7|30|3|1938|155|2013|Saturday|1938Q3|N|Y|N|2429081|2429261|2428745|2429019|N|N|N|N|N| +2429111|AAAAAAAAHLABFCAA|1938-07-31|462|2013|155|1938|0|7|31|3|1938|155|2013|Sunday|1938Q3|N|N|N|2429081|2429261|2428746|2429020|N|N|N|N|N| +2429112|AAAAAAAAILABFCAA|1938-08-01|463|2013|155|1938|1|8|1|3|1938|155|2013|Monday|1938Q3|N|N|N|2429112|2429323|2428747|2429021|N|N|N|N|N| +2429113|AAAAAAAAJLABFCAA|1938-08-02|463|2014|155|1938|2|8|2|3|1938|155|2014|Tuesday|1938Q3|N|N|N|2429112|2429323|2428748|2429022|N|N|N|N|N| +2429114|AAAAAAAAKLABFCAA|1938-08-03|463|2014|155|1938|3|8|3|3|1938|155|2014|Wednesday|1938Q3|N|N|N|2429112|2429323|2428749|2429023|N|N|N|N|N| +2429115|AAAAAAAALLABFCAA|1938-08-04|463|2014|155|1938|4|8|4|3|1938|155|2014|Thursday|1938Q3|N|N|N|2429112|2429323|2428750|2429024|N|N|N|N|N| +2429116|AAAAAAAAMLABFCAA|1938-08-05|463|2014|155|1938|5|8|5|3|1938|155|2014|Friday|1938Q3|N|Y|N|2429112|2429323|2428751|2429025|N|N|N|N|N| +2429117|AAAAAAAANLABFCAA|1938-08-06|463|2014|155|1938|6|8|6|3|1938|155|2014|Saturday|1938Q3|N|Y|N|2429112|2429323|2428752|2429026|N|N|N|N|N| +2429118|AAAAAAAAOLABFCAA|1938-08-07|463|2014|155|1938|0|8|7|3|1938|155|2014|Sunday|1938Q3|N|N|N|2429112|2429323|2428753|2429027|N|N|N|N|N| +2429119|AAAAAAAAPLABFCAA|1938-08-08|463|2014|155|1938|1|8|8|3|1938|155|2014|Monday|1938Q3|N|N|N|2429112|2429323|2428754|2429028|N|N|N|N|N| +2429120|AAAAAAAAAMABFCAA|1938-08-09|463|2015|155|1938|2|8|9|3|1938|155|2015|Tuesday|1938Q3|N|N|N|2429112|2429323|2428755|2429029|N|N|N|N|N| +2429121|AAAAAAAABMABFCAA|1938-08-10|463|2015|155|1938|3|8|10|3|1938|155|2015|Wednesday|1938Q3|N|N|N|2429112|2429323|2428756|2429030|N|N|N|N|N| +2429122|AAAAAAAACMABFCAA|1938-08-11|463|2015|155|1938|4|8|11|3|1938|155|2015|Thursday|1938Q3|N|N|N|2429112|2429323|2428757|2429031|N|N|N|N|N| +2429123|AAAAAAAADMABFCAA|1938-08-12|463|2015|155|1938|5|8|12|3|1938|155|2015|Friday|1938Q3|N|Y|N|2429112|2429323|2428758|2429032|N|N|N|N|N| +2429124|AAAAAAAAEMABFCAA|1938-08-13|463|2015|155|1938|6|8|13|3|1938|155|2015|Saturday|1938Q3|N|Y|N|2429112|2429323|2428759|2429033|N|N|N|N|N| +2429125|AAAAAAAAFMABFCAA|1938-08-14|463|2015|155|1938|0|8|14|3|1938|155|2015|Sunday|1938Q3|N|N|N|2429112|2429323|2428760|2429034|N|N|N|N|N| +2429126|AAAAAAAAGMABFCAA|1938-08-15|463|2015|155|1938|1|8|15|3|1938|155|2015|Monday|1938Q3|N|N|N|2429112|2429323|2428761|2429035|N|N|N|N|N| +2429127|AAAAAAAAHMABFCAA|1938-08-16|463|2016|155|1938|2|8|16|3|1938|155|2016|Tuesday|1938Q3|N|N|N|2429112|2429323|2428762|2429036|N|N|N|N|N| +2429128|AAAAAAAAIMABFCAA|1938-08-17|463|2016|155|1938|3|8|17|3|1938|155|2016|Wednesday|1938Q3|N|N|N|2429112|2429323|2428763|2429037|N|N|N|N|N| +2429129|AAAAAAAAJMABFCAA|1938-08-18|463|2016|155|1938|4|8|18|3|1938|155|2016|Thursday|1938Q3|N|N|N|2429112|2429323|2428764|2429038|N|N|N|N|N| +2429130|AAAAAAAAKMABFCAA|1938-08-19|463|2016|155|1938|5|8|19|3|1938|155|2016|Friday|1938Q3|N|Y|N|2429112|2429323|2428765|2429039|N|N|N|N|N| +2429131|AAAAAAAALMABFCAA|1938-08-20|463|2016|155|1938|6|8|20|3|1938|155|2016|Saturday|1938Q3|N|Y|N|2429112|2429323|2428766|2429040|N|N|N|N|N| +2429132|AAAAAAAAMMABFCAA|1938-08-21|463|2016|155|1938|0|8|21|3|1938|155|2016|Sunday|1938Q3|N|N|N|2429112|2429323|2428767|2429041|N|N|N|N|N| +2429133|AAAAAAAANMABFCAA|1938-08-22|463|2016|155|1938|1|8|22|3|1938|155|2016|Monday|1938Q3|N|N|N|2429112|2429323|2428768|2429042|N|N|N|N|N| +2429134|AAAAAAAAOMABFCAA|1938-08-23|463|2017|155|1938|2|8|23|3|1938|155|2017|Tuesday|1938Q3|N|N|N|2429112|2429323|2428769|2429043|N|N|N|N|N| +2429135|AAAAAAAAPMABFCAA|1938-08-24|463|2017|155|1938|3|8|24|3|1938|155|2017|Wednesday|1938Q3|N|N|N|2429112|2429323|2428770|2429044|N|N|N|N|N| +2429136|AAAAAAAAANABFCAA|1938-08-25|463|2017|155|1938|4|8|25|3|1938|155|2017|Thursday|1938Q3|N|N|N|2429112|2429323|2428771|2429045|N|N|N|N|N| +2429137|AAAAAAAABNABFCAA|1938-08-26|463|2017|155|1938|5|8|26|3|1938|155|2017|Friday|1938Q3|N|Y|N|2429112|2429323|2428772|2429046|N|N|N|N|N| +2429138|AAAAAAAACNABFCAA|1938-08-27|463|2017|155|1938|6|8|27|3|1938|155|2017|Saturday|1938Q3|N|Y|N|2429112|2429323|2428773|2429047|N|N|N|N|N| +2429139|AAAAAAAADNABFCAA|1938-08-28|463|2017|155|1938|0|8|28|3|1938|155|2017|Sunday|1938Q3|N|N|N|2429112|2429323|2428774|2429048|N|N|N|N|N| +2429140|AAAAAAAAENABFCAA|1938-08-29|463|2017|155|1938|1|8|29|3|1938|155|2017|Monday|1938Q3|N|N|N|2429112|2429323|2428775|2429049|N|N|N|N|N| +2429141|AAAAAAAAFNABFCAA|1938-08-30|463|2018|155|1938|2|8|30|3|1938|155|2018|Tuesday|1938Q3|N|N|N|2429112|2429323|2428776|2429050|N|N|N|N|N| +2429142|AAAAAAAAGNABFCAA|1938-08-31|463|2018|155|1938|3|8|31|3|1938|155|2018|Wednesday|1938Q3|N|N|N|2429112|2429323|2428777|2429051|N|N|N|N|N| +2429143|AAAAAAAAHNABFCAA|1938-09-01|464|2018|156|1938|4|9|1|3|1938|156|2018|Thursday|1938Q3|N|N|N|2429143|2429385|2428778|2429052|N|N|N|N|N| +2429144|AAAAAAAAINABFCAA|1938-09-02|464|2018|156|1938|5|9|2|3|1938|156|2018|Friday|1938Q3|N|Y|N|2429143|2429385|2428779|2429053|N|N|N|N|N| +2429145|AAAAAAAAJNABFCAA|1938-09-03|464|2018|156|1938|6|9|3|3|1938|156|2018|Saturday|1938Q3|N|Y|N|2429143|2429385|2428780|2429054|N|N|N|N|N| +2429146|AAAAAAAAKNABFCAA|1938-09-04|464|2018|156|1938|0|9|4|3|1938|156|2018|Sunday|1938Q3|N|N|N|2429143|2429385|2428781|2429055|N|N|N|N|N| +2429147|AAAAAAAALNABFCAA|1938-09-05|464|2018|156|1938|1|9|5|3|1938|156|2018|Monday|1938Q3|N|N|N|2429143|2429385|2428782|2429056|N|N|N|N|N| +2429148|AAAAAAAAMNABFCAA|1938-09-06|464|2019|156|1938|2|9|6|3|1938|156|2019|Tuesday|1938Q3|N|N|N|2429143|2429385|2428783|2429057|N|N|N|N|N| +2429149|AAAAAAAANNABFCAA|1938-09-07|464|2019|156|1938|3|9|7|3|1938|156|2019|Wednesday|1938Q3|N|N|N|2429143|2429385|2428784|2429058|N|N|N|N|N| +2429150|AAAAAAAAONABFCAA|1938-09-08|464|2019|156|1938|4|9|8|3|1938|156|2019|Thursday|1938Q3|N|N|N|2429143|2429385|2428785|2429059|N|N|N|N|N| +2429151|AAAAAAAAPNABFCAA|1938-09-09|464|2019|156|1938|5|9|9|3|1938|156|2019|Friday|1938Q3|N|Y|N|2429143|2429385|2428786|2429060|N|N|N|N|N| +2429152|AAAAAAAAAOABFCAA|1938-09-10|464|2019|156|1938|6|9|10|3|1938|156|2019|Saturday|1938Q3|N|Y|N|2429143|2429385|2428787|2429061|N|N|N|N|N| +2429153|AAAAAAAABOABFCAA|1938-09-11|464|2019|156|1938|0|9|11|3|1938|156|2019|Sunday|1938Q3|N|N|N|2429143|2429385|2428788|2429062|N|N|N|N|N| +2429154|AAAAAAAACOABFCAA|1938-09-12|464|2019|156|1938|1|9|12|3|1938|156|2019|Monday|1938Q3|N|N|N|2429143|2429385|2428789|2429063|N|N|N|N|N| +2429155|AAAAAAAADOABFCAA|1938-09-13|464|2020|156|1938|2|9|13|3|1938|156|2020|Tuesday|1938Q3|N|N|N|2429143|2429385|2428790|2429064|N|N|N|N|N| +2429156|AAAAAAAAEOABFCAA|1938-09-14|464|2020|156|1938|3|9|14|3|1938|156|2020|Wednesday|1938Q3|N|N|N|2429143|2429385|2428791|2429065|N|N|N|N|N| +2429157|AAAAAAAAFOABFCAA|1938-09-15|464|2020|156|1938|4|9|15|3|1938|156|2020|Thursday|1938Q3|N|N|N|2429143|2429385|2428792|2429066|N|N|N|N|N| +2429158|AAAAAAAAGOABFCAA|1938-09-16|464|2020|156|1938|5|9|16|3|1938|156|2020|Friday|1938Q3|N|Y|N|2429143|2429385|2428793|2429067|N|N|N|N|N| +2429159|AAAAAAAAHOABFCAA|1938-09-17|464|2020|156|1938|6|9|17|3|1938|156|2020|Saturday|1938Q3|N|Y|N|2429143|2429385|2428794|2429068|N|N|N|N|N| +2429160|AAAAAAAAIOABFCAA|1938-09-18|464|2020|156|1938|0|9|18|3|1938|156|2020|Sunday|1938Q3|N|N|N|2429143|2429385|2428795|2429069|N|N|N|N|N| +2429161|AAAAAAAAJOABFCAA|1938-09-19|464|2020|156|1938|1|9|19|3|1938|156|2020|Monday|1938Q3|N|N|N|2429143|2429385|2428796|2429070|N|N|N|N|N| +2429162|AAAAAAAAKOABFCAA|1938-09-20|464|2021|156|1938|2|9|20|3|1938|156|2021|Tuesday|1938Q3|N|N|N|2429143|2429385|2428797|2429071|N|N|N|N|N| +2429163|AAAAAAAALOABFCAA|1938-09-21|464|2021|156|1938|3|9|21|3|1938|156|2021|Wednesday|1938Q3|N|N|N|2429143|2429385|2428798|2429072|N|N|N|N|N| +2429164|AAAAAAAAMOABFCAA|1938-09-22|464|2021|156|1938|4|9|22|3|1938|156|2021|Thursday|1938Q3|N|N|N|2429143|2429385|2428799|2429073|N|N|N|N|N| +2429165|AAAAAAAANOABFCAA|1938-09-23|464|2021|156|1938|5|9|23|3|1938|156|2021|Friday|1938Q3|N|Y|N|2429143|2429385|2428800|2429074|N|N|N|N|N| +2429166|AAAAAAAAOOABFCAA|1938-09-24|464|2021|156|1938|6|9|24|3|1938|156|2021|Saturday|1938Q3|N|Y|N|2429143|2429385|2428801|2429075|N|N|N|N|N| +2429167|AAAAAAAAPOABFCAA|1938-09-25|464|2021|156|1938|0|9|25|3|1938|156|2021|Sunday|1938Q3|N|N|N|2429143|2429385|2428802|2429076|N|N|N|N|N| +2429168|AAAAAAAAAPABFCAA|1938-09-26|464|2021|156|1938|1|9|26|3|1938|156|2021|Monday|1938Q3|N|N|N|2429143|2429385|2428803|2429077|N|N|N|N|N| +2429169|AAAAAAAABPABFCAA|1938-09-27|464|2022|156|1938|2|9|27|3|1938|156|2022|Tuesday|1938Q3|N|N|N|2429143|2429385|2428804|2429078|N|N|N|N|N| +2429170|AAAAAAAACPABFCAA|1938-09-28|464|2022|156|1938|3|9|28|3|1938|156|2022|Wednesday|1938Q3|N|N|N|2429143|2429385|2428805|2429079|N|N|N|N|N| +2429171|AAAAAAAADPABFCAA|1938-09-29|464|2022|156|1938|4|9|29|3|1938|156|2022|Thursday|1938Q3|N|N|N|2429143|2429385|2428806|2429080|N|N|N|N|N| +2429172|AAAAAAAAEPABFCAA|1938-09-30|464|2022|156|1938|5|9|30|3|1938|156|2022|Friday|1938Q3|N|Y|N|2429143|2429385|2428807|2429081|N|N|N|N|N| +2429173|AAAAAAAAFPABFCAA|1938-10-01|465|2022|156|1938|6|10|1|3|1938|156|2022|Saturday|1938Q3|N|Y|N|2429173|2429445|2428808|2429081|N|N|N|N|N| +2429174|AAAAAAAAGPABFCAA|1938-10-02|465|2022|156|1938|0|10|2|4|1938|156|2022|Sunday|1938Q4|N|N|N|2429173|2429445|2428809|2429082|N|N|N|N|N| +2429175|AAAAAAAAHPABFCAA|1938-10-03|465|2022|156|1938|1|10|3|4|1938|156|2022|Monday|1938Q4|N|N|N|2429173|2429445|2428810|2429083|N|N|N|N|N| +2429176|AAAAAAAAIPABFCAA|1938-10-04|465|2023|156|1938|2|10|4|4|1938|156|2023|Tuesday|1938Q4|N|N|N|2429173|2429445|2428811|2429084|N|N|N|N|N| +2429177|AAAAAAAAJPABFCAA|1938-10-05|465|2023|156|1938|3|10|5|4|1938|156|2023|Wednesday|1938Q4|N|N|N|2429173|2429445|2428812|2429085|N|N|N|N|N| +2429178|AAAAAAAAKPABFCAA|1938-10-06|465|2023|156|1938|4|10|6|4|1938|156|2023|Thursday|1938Q4|N|N|N|2429173|2429445|2428813|2429086|N|N|N|N|N| +2429179|AAAAAAAALPABFCAA|1938-10-07|465|2023|156|1938|5|10|7|4|1938|156|2023|Friday|1938Q4|N|Y|N|2429173|2429445|2428814|2429087|N|N|N|N|N| +2429180|AAAAAAAAMPABFCAA|1938-10-08|465|2023|156|1938|6|10|8|4|1938|156|2023|Saturday|1938Q4|N|Y|N|2429173|2429445|2428815|2429088|N|N|N|N|N| +2429181|AAAAAAAANPABFCAA|1938-10-09|465|2023|156|1938|0|10|9|4|1938|156|2023|Sunday|1938Q4|N|N|N|2429173|2429445|2428816|2429089|N|N|N|N|N| +2429182|AAAAAAAAOPABFCAA|1938-10-10|465|2023|156|1938|1|10|10|4|1938|156|2023|Monday|1938Q4|N|N|N|2429173|2429445|2428817|2429090|N|N|N|N|N| +2429183|AAAAAAAAPPABFCAA|1938-10-11|465|2024|156|1938|2|10|11|4|1938|156|2024|Tuesday|1938Q4|N|N|N|2429173|2429445|2428818|2429091|N|N|N|N|N| +2429184|AAAAAAAAAABBFCAA|1938-10-12|465|2024|156|1938|3|10|12|4|1938|156|2024|Wednesday|1938Q4|N|N|N|2429173|2429445|2428819|2429092|N|N|N|N|N| +2429185|AAAAAAAABABBFCAA|1938-10-13|465|2024|156|1938|4|10|13|4|1938|156|2024|Thursday|1938Q4|N|N|N|2429173|2429445|2428820|2429093|N|N|N|N|N| +2429186|AAAAAAAACABBFCAA|1938-10-14|465|2024|156|1938|5|10|14|4|1938|156|2024|Friday|1938Q4|N|Y|N|2429173|2429445|2428821|2429094|N|N|N|N|N| +2429187|AAAAAAAADABBFCAA|1938-10-15|465|2024|156|1938|6|10|15|4|1938|156|2024|Saturday|1938Q4|N|Y|N|2429173|2429445|2428822|2429095|N|N|N|N|N| +2429188|AAAAAAAAEABBFCAA|1938-10-16|465|2024|156|1938|0|10|16|4|1938|156|2024|Sunday|1938Q4|N|N|N|2429173|2429445|2428823|2429096|N|N|N|N|N| +2429189|AAAAAAAAFABBFCAA|1938-10-17|465|2024|156|1938|1|10|17|4|1938|156|2024|Monday|1938Q4|N|N|N|2429173|2429445|2428824|2429097|N|N|N|N|N| +2429190|AAAAAAAAGABBFCAA|1938-10-18|465|2025|156|1938|2|10|18|4|1938|156|2025|Tuesday|1938Q4|N|N|N|2429173|2429445|2428825|2429098|N|N|N|N|N| +2429191|AAAAAAAAHABBFCAA|1938-10-19|465|2025|156|1938|3|10|19|4|1938|156|2025|Wednesday|1938Q4|N|N|N|2429173|2429445|2428826|2429099|N|N|N|N|N| +2429192|AAAAAAAAIABBFCAA|1938-10-20|465|2025|156|1938|4|10|20|4|1938|156|2025|Thursday|1938Q4|N|N|N|2429173|2429445|2428827|2429100|N|N|N|N|N| +2429193|AAAAAAAAJABBFCAA|1938-10-21|465|2025|156|1938|5|10|21|4|1938|156|2025|Friday|1938Q4|N|Y|N|2429173|2429445|2428828|2429101|N|N|N|N|N| +2429194|AAAAAAAAKABBFCAA|1938-10-22|465|2025|156|1938|6|10|22|4|1938|156|2025|Saturday|1938Q4|N|Y|N|2429173|2429445|2428829|2429102|N|N|N|N|N| +2429195|AAAAAAAALABBFCAA|1938-10-23|465|2025|156|1938|0|10|23|4|1938|156|2025|Sunday|1938Q4|N|N|N|2429173|2429445|2428830|2429103|N|N|N|N|N| +2429196|AAAAAAAAMABBFCAA|1938-10-24|465|2025|156|1938|1|10|24|4|1938|156|2025|Monday|1938Q4|N|N|N|2429173|2429445|2428831|2429104|N|N|N|N|N| +2429197|AAAAAAAANABBFCAA|1938-10-25|465|2026|156|1938|2|10|25|4|1938|156|2026|Tuesday|1938Q4|N|N|N|2429173|2429445|2428832|2429105|N|N|N|N|N| +2429198|AAAAAAAAOABBFCAA|1938-10-26|465|2026|156|1938|3|10|26|4|1938|156|2026|Wednesday|1938Q4|N|N|N|2429173|2429445|2428833|2429106|N|N|N|N|N| +2429199|AAAAAAAAPABBFCAA|1938-10-27|465|2026|156|1938|4|10|27|4|1938|156|2026|Thursday|1938Q4|N|N|N|2429173|2429445|2428834|2429107|N|N|N|N|N| +2429200|AAAAAAAAABBBFCAA|1938-10-28|465|2026|156|1938|5|10|28|4|1938|156|2026|Friday|1938Q4|N|Y|N|2429173|2429445|2428835|2429108|N|N|N|N|N| +2429201|AAAAAAAABBBBFCAA|1938-10-29|465|2026|156|1938|6|10|29|4|1938|156|2026|Saturday|1938Q4|N|Y|N|2429173|2429445|2428836|2429109|N|N|N|N|N| +2429202|AAAAAAAACBBBFCAA|1938-10-30|465|2026|156|1938|0|10|30|4|1938|156|2026|Sunday|1938Q4|N|N|N|2429173|2429445|2428837|2429110|N|N|N|N|N| +2429203|AAAAAAAADBBBFCAA|1938-10-31|465|2026|156|1938|1|10|31|4|1938|156|2026|Monday|1938Q4|N|N|N|2429173|2429445|2428838|2429111|N|N|N|N|N| +2429204|AAAAAAAAEBBBFCAA|1938-11-01|466|2027|156|1938|2|11|1|4|1938|156|2027|Tuesday|1938Q4|N|N|N|2429204|2429507|2428839|2429112|N|N|N|N|N| +2429205|AAAAAAAAFBBBFCAA|1938-11-02|466|2027|156|1938|3|11|2|4|1938|156|2027|Wednesday|1938Q4|N|N|N|2429204|2429507|2428840|2429113|N|N|N|N|N| +2429206|AAAAAAAAGBBBFCAA|1938-11-03|466|2027|156|1938|4|11|3|4|1938|156|2027|Thursday|1938Q4|N|N|N|2429204|2429507|2428841|2429114|N|N|N|N|N| +2429207|AAAAAAAAHBBBFCAA|1938-11-04|466|2027|156|1938|5|11|4|4|1938|156|2027|Friday|1938Q4|N|Y|N|2429204|2429507|2428842|2429115|N|N|N|N|N| +2429208|AAAAAAAAIBBBFCAA|1938-11-05|466|2027|156|1938|6|11|5|4|1938|156|2027|Saturday|1938Q4|N|Y|N|2429204|2429507|2428843|2429116|N|N|N|N|N| +2429209|AAAAAAAAJBBBFCAA|1938-11-06|466|2027|156|1938|0|11|6|4|1938|156|2027|Sunday|1938Q4|N|N|N|2429204|2429507|2428844|2429117|N|N|N|N|N| +2429210|AAAAAAAAKBBBFCAA|1938-11-07|466|2027|156|1938|1|11|7|4|1938|156|2027|Monday|1938Q4|N|N|N|2429204|2429507|2428845|2429118|N|N|N|N|N| +2429211|AAAAAAAALBBBFCAA|1938-11-08|466|2028|156|1938|2|11|8|4|1938|156|2028|Tuesday|1938Q4|N|N|N|2429204|2429507|2428846|2429119|N|N|N|N|N| +2429212|AAAAAAAAMBBBFCAA|1938-11-09|466|2028|156|1938|3|11|9|4|1938|156|2028|Wednesday|1938Q4|N|N|N|2429204|2429507|2428847|2429120|N|N|N|N|N| +2429213|AAAAAAAANBBBFCAA|1938-11-10|466|2028|156|1938|4|11|10|4|1938|156|2028|Thursday|1938Q4|N|N|N|2429204|2429507|2428848|2429121|N|N|N|N|N| +2429214|AAAAAAAAOBBBFCAA|1938-11-11|466|2028|156|1938|5|11|11|4|1938|156|2028|Friday|1938Q4|N|Y|N|2429204|2429507|2428849|2429122|N|N|N|N|N| +2429215|AAAAAAAAPBBBFCAA|1938-11-12|466|2028|156|1938|6|11|12|4|1938|156|2028|Saturday|1938Q4|N|Y|N|2429204|2429507|2428850|2429123|N|N|N|N|N| +2429216|AAAAAAAAACBBFCAA|1938-11-13|466|2028|156|1938|0|11|13|4|1938|156|2028|Sunday|1938Q4|N|N|N|2429204|2429507|2428851|2429124|N|N|N|N|N| +2429217|AAAAAAAABCBBFCAA|1938-11-14|466|2028|156|1938|1|11|14|4|1938|156|2028|Monday|1938Q4|N|N|N|2429204|2429507|2428852|2429125|N|N|N|N|N| +2429218|AAAAAAAACCBBFCAA|1938-11-15|466|2029|156|1938|2|11|15|4|1938|156|2029|Tuesday|1938Q4|N|N|N|2429204|2429507|2428853|2429126|N|N|N|N|N| +2429219|AAAAAAAADCBBFCAA|1938-11-16|466|2029|156|1938|3|11|16|4|1938|156|2029|Wednesday|1938Q4|N|N|N|2429204|2429507|2428854|2429127|N|N|N|N|N| +2429220|AAAAAAAAECBBFCAA|1938-11-17|466|2029|156|1938|4|11|17|4|1938|156|2029|Thursday|1938Q4|N|N|N|2429204|2429507|2428855|2429128|N|N|N|N|N| +2429221|AAAAAAAAFCBBFCAA|1938-11-18|466|2029|156|1938|5|11|18|4|1938|156|2029|Friday|1938Q4|N|Y|N|2429204|2429507|2428856|2429129|N|N|N|N|N| +2429222|AAAAAAAAGCBBFCAA|1938-11-19|466|2029|156|1938|6|11|19|4|1938|156|2029|Saturday|1938Q4|N|Y|N|2429204|2429507|2428857|2429130|N|N|N|N|N| +2429223|AAAAAAAAHCBBFCAA|1938-11-20|466|2029|156|1938|0|11|20|4|1938|156|2029|Sunday|1938Q4|N|N|N|2429204|2429507|2428858|2429131|N|N|N|N|N| +2429224|AAAAAAAAICBBFCAA|1938-11-21|466|2029|156|1938|1|11|21|4|1938|156|2029|Monday|1938Q4|N|N|N|2429204|2429507|2428859|2429132|N|N|N|N|N| +2429225|AAAAAAAAJCBBFCAA|1938-11-22|466|2030|156|1938|2|11|22|4|1938|156|2030|Tuesday|1938Q4|N|N|N|2429204|2429507|2428860|2429133|N|N|N|N|N| +2429226|AAAAAAAAKCBBFCAA|1938-11-23|466|2030|156|1938|3|11|23|4|1938|156|2030|Wednesday|1938Q4|N|N|N|2429204|2429507|2428861|2429134|N|N|N|N|N| +2429227|AAAAAAAALCBBFCAA|1938-11-24|466|2030|156|1938|4|11|24|4|1938|156|2030|Thursday|1938Q4|N|N|N|2429204|2429507|2428862|2429135|N|N|N|N|N| +2429228|AAAAAAAAMCBBFCAA|1938-11-25|466|2030|156|1938|5|11|25|4|1938|156|2030|Friday|1938Q4|N|Y|N|2429204|2429507|2428863|2429136|N|N|N|N|N| +2429229|AAAAAAAANCBBFCAA|1938-11-26|466|2030|156|1938|6|11|26|4|1938|156|2030|Saturday|1938Q4|N|Y|N|2429204|2429507|2428864|2429137|N|N|N|N|N| +2429230|AAAAAAAAOCBBFCAA|1938-11-27|466|2030|156|1938|0|11|27|4|1938|156|2030|Sunday|1938Q4|N|N|N|2429204|2429507|2428865|2429138|N|N|N|N|N| +2429231|AAAAAAAAPCBBFCAA|1938-11-28|466|2030|156|1938|1|11|28|4|1938|156|2030|Monday|1938Q4|N|N|N|2429204|2429507|2428866|2429139|N|N|N|N|N| +2429232|AAAAAAAAADBBFCAA|1938-11-29|466|2031|156|1938|2|11|29|4|1938|156|2031|Tuesday|1938Q4|N|N|N|2429204|2429507|2428867|2429140|N|N|N|N|N| +2429233|AAAAAAAABDBBFCAA|1938-11-30|466|2031|156|1938|3|11|30|4|1938|156|2031|Wednesday|1938Q4|N|N|N|2429204|2429507|2428868|2429141|N|N|N|N|N| +2429234|AAAAAAAACDBBFCAA|1938-12-01|467|2031|157|1938|4|12|1|4|1938|157|2031|Thursday|1938Q4|N|N|N|2429234|2429567|2428869|2429142|N|N|N|N|N| +2429235|AAAAAAAADDBBFCAA|1938-12-02|467|2031|157|1938|5|12|2|4|1938|157|2031|Friday|1938Q4|N|Y|N|2429234|2429567|2428870|2429143|N|N|N|N|N| +2429236|AAAAAAAAEDBBFCAA|1938-12-03|467|2031|157|1938|6|12|3|4|1938|157|2031|Saturday|1938Q4|N|Y|N|2429234|2429567|2428871|2429144|N|N|N|N|N| +2429237|AAAAAAAAFDBBFCAA|1938-12-04|467|2031|157|1938|0|12|4|4|1938|157|2031|Sunday|1938Q4|N|N|N|2429234|2429567|2428872|2429145|N|N|N|N|N| +2429238|AAAAAAAAGDBBFCAA|1938-12-05|467|2031|157|1938|1|12|5|4|1938|157|2031|Monday|1938Q4|N|N|N|2429234|2429567|2428873|2429146|N|N|N|N|N| +2429239|AAAAAAAAHDBBFCAA|1938-12-06|467|2032|157|1938|2|12|6|4|1938|157|2032|Tuesday|1938Q4|N|N|N|2429234|2429567|2428874|2429147|N|N|N|N|N| +2429240|AAAAAAAAIDBBFCAA|1938-12-07|467|2032|157|1938|3|12|7|4|1938|157|2032|Wednesday|1938Q4|N|N|N|2429234|2429567|2428875|2429148|N|N|N|N|N| +2429241|AAAAAAAAJDBBFCAA|1938-12-08|467|2032|157|1938|4|12|8|4|1938|157|2032|Thursday|1938Q4|N|N|N|2429234|2429567|2428876|2429149|N|N|N|N|N| +2429242|AAAAAAAAKDBBFCAA|1938-12-09|467|2032|157|1938|5|12|9|4|1938|157|2032|Friday|1938Q4|N|Y|N|2429234|2429567|2428877|2429150|N|N|N|N|N| +2429243|AAAAAAAALDBBFCAA|1938-12-10|467|2032|157|1938|6|12|10|4|1938|157|2032|Saturday|1938Q4|N|Y|N|2429234|2429567|2428878|2429151|N|N|N|N|N| +2429244|AAAAAAAAMDBBFCAA|1938-12-11|467|2032|157|1938|0|12|11|4|1938|157|2032|Sunday|1938Q4|N|N|N|2429234|2429567|2428879|2429152|N|N|N|N|N| +2429245|AAAAAAAANDBBFCAA|1938-12-12|467|2032|157|1938|1|12|12|4|1938|157|2032|Monday|1938Q4|N|N|N|2429234|2429567|2428880|2429153|N|N|N|N|N| +2429246|AAAAAAAAODBBFCAA|1938-12-13|467|2033|157|1938|2|12|13|4|1938|157|2033|Tuesday|1938Q4|N|N|N|2429234|2429567|2428881|2429154|N|N|N|N|N| +2429247|AAAAAAAAPDBBFCAA|1938-12-14|467|2033|157|1938|3|12|14|4|1938|157|2033|Wednesday|1938Q4|N|N|N|2429234|2429567|2428882|2429155|N|N|N|N|N| +2429248|AAAAAAAAAEBBFCAA|1938-12-15|467|2033|157|1938|4|12|15|4|1938|157|2033|Thursday|1938Q4|N|N|N|2429234|2429567|2428883|2429156|N|N|N|N|N| +2429249|AAAAAAAABEBBFCAA|1938-12-16|467|2033|157|1938|5|12|16|4|1938|157|2033|Friday|1938Q4|N|Y|N|2429234|2429567|2428884|2429157|N|N|N|N|N| +2429250|AAAAAAAACEBBFCAA|1938-12-17|467|2033|157|1938|6|12|17|4|1938|157|2033|Saturday|1938Q4|N|Y|N|2429234|2429567|2428885|2429158|N|N|N|N|N| +2429251|AAAAAAAADEBBFCAA|1938-12-18|467|2033|157|1938|0|12|18|4|1938|157|2033|Sunday|1938Q4|N|N|N|2429234|2429567|2428886|2429159|N|N|N|N|N| +2429252|AAAAAAAAEEBBFCAA|1938-12-19|467|2033|157|1938|1|12|19|4|1938|157|2033|Monday|1938Q4|N|N|N|2429234|2429567|2428887|2429160|N|N|N|N|N| +2429253|AAAAAAAAFEBBFCAA|1938-12-20|467|2034|157|1938|2|12|20|4|1938|157|2034|Tuesday|1938Q4|N|N|N|2429234|2429567|2428888|2429161|N|N|N|N|N| +2429254|AAAAAAAAGEBBFCAA|1938-12-21|467|2034|157|1938|3|12|21|4|1938|157|2034|Wednesday|1938Q4|N|N|N|2429234|2429567|2428889|2429162|N|N|N|N|N| +2429255|AAAAAAAAHEBBFCAA|1938-12-22|467|2034|157|1938|4|12|22|4|1938|157|2034|Thursday|1938Q4|N|N|N|2429234|2429567|2428890|2429163|N|N|N|N|N| +2429256|AAAAAAAAIEBBFCAA|1938-12-23|467|2034|157|1938|5|12|23|4|1938|157|2034|Friday|1938Q4|N|Y|N|2429234|2429567|2428891|2429164|N|N|N|N|N| +2429257|AAAAAAAAJEBBFCAA|1938-12-24|467|2034|157|1938|6|12|24|4|1938|157|2034|Saturday|1938Q4|N|Y|N|2429234|2429567|2428892|2429165|N|N|N|N|N| +2429258|AAAAAAAAKEBBFCAA|1938-12-25|467|2034|157|1938|0|12|25|4|1938|157|2034|Sunday|1938Q4|N|N|N|2429234|2429567|2428893|2429166|N|N|N|N|N| +2429259|AAAAAAAALEBBFCAA|1938-12-26|467|2034|157|1938|1|12|26|4|1938|157|2034|Monday|1938Q4|Y|N|N|2429234|2429567|2428894|2429167|N|N|N|N|N| +2429260|AAAAAAAAMEBBFCAA|1938-12-27|467|2035|157|1938|2|12|27|4|1938|157|2035|Tuesday|1938Q4|N|N|Y|2429234|2429567|2428895|2429168|N|N|N|N|N| +2429261|AAAAAAAANEBBFCAA|1938-12-28|467|2035|157|1938|3|12|28|4|1938|157|2035|Wednesday|1938Q4|N|N|N|2429234|2429567|2428896|2429169|N|N|N|N|N| +2429262|AAAAAAAAOEBBFCAA|1938-12-29|467|2035|157|1938|4|12|29|4|1938|157|2035|Thursday|1938Q4|N|N|N|2429234|2429567|2428897|2429170|N|N|N|N|N| +2429263|AAAAAAAAPEBBFCAA|1938-12-30|467|2035|157|1938|5|12|30|4|1938|157|2035|Friday|1938Q4|N|Y|N|2429234|2429567|2428898|2429171|N|N|N|N|N| +2429264|AAAAAAAAAFBBFCAA|1938-12-31|467|2035|157|1938|6|12|31|4|1938|157|2035|Saturday|1938Q4|N|Y|N|2429234|2429567|2428899|2429172|N|N|N|N|N| +2429265|AAAAAAAABFBBFCAA|1939-01-01|468|2035|157|1939|0|1|1|1|1939|157|2035|Sunday|1939Q1|Y|N|N|2429265|2429264|2428900|2429173|N|N|N|N|N| +2429266|AAAAAAAACFBBFCAA|1939-01-02|468|2035|157|1939|1|1|2|1|1939|157|2035|Monday|1939Q1|N|N|Y|2429265|2429264|2428901|2429174|N|N|N|N|N| +2429267|AAAAAAAADFBBFCAA|1939-01-03|468|2036|157|1939|2|1|3|1|1939|157|2036|Tuesday|1939Q1|N|N|N|2429265|2429264|2428902|2429175|N|N|N|N|N| +2429268|AAAAAAAAEFBBFCAA|1939-01-04|468|2036|157|1939|3|1|4|1|1939|157|2036|Wednesday|1939Q1|N|N|N|2429265|2429264|2428903|2429176|N|N|N|N|N| +2429269|AAAAAAAAFFBBFCAA|1939-01-05|468|2036|157|1939|4|1|5|1|1939|157|2036|Thursday|1939Q1|N|N|N|2429265|2429264|2428904|2429177|N|N|N|N|N| +2429270|AAAAAAAAGFBBFCAA|1939-01-06|468|2036|157|1939|5|1|6|1|1939|157|2036|Friday|1939Q1|N|Y|N|2429265|2429264|2428905|2429178|N|N|N|N|N| +2429271|AAAAAAAAHFBBFCAA|1939-01-07|468|2036|157|1939|6|1|7|1|1939|157|2036|Saturday|1939Q1|N|Y|N|2429265|2429264|2428906|2429179|N|N|N|N|N| +2429272|AAAAAAAAIFBBFCAA|1939-01-08|468|2036|157|1939|0|1|8|1|1939|157|2036|Sunday|1939Q1|N|N|N|2429265|2429264|2428907|2429180|N|N|N|N|N| +2429273|AAAAAAAAJFBBFCAA|1939-01-09|468|2036|157|1939|1|1|9|1|1939|157|2036|Monday|1939Q1|N|N|N|2429265|2429264|2428908|2429181|N|N|N|N|N| +2429274|AAAAAAAAKFBBFCAA|1939-01-10|468|2037|157|1939|2|1|10|1|1939|157|2037|Tuesday|1939Q1|N|N|N|2429265|2429264|2428909|2429182|N|N|N|N|N| +2429275|AAAAAAAALFBBFCAA|1939-01-11|468|2037|157|1939|3|1|11|1|1939|157|2037|Wednesday|1939Q1|N|N|N|2429265|2429264|2428910|2429183|N|N|N|N|N| +2429276|AAAAAAAAMFBBFCAA|1939-01-12|468|2037|157|1939|4|1|12|1|1939|157|2037|Thursday|1939Q1|N|N|N|2429265|2429264|2428911|2429184|N|N|N|N|N| +2429277|AAAAAAAANFBBFCAA|1939-01-13|468|2037|157|1939|5|1|13|1|1939|157|2037|Friday|1939Q1|N|Y|N|2429265|2429264|2428912|2429185|N|N|N|N|N| +2429278|AAAAAAAAOFBBFCAA|1939-01-14|468|2037|157|1939|6|1|14|1|1939|157|2037|Saturday|1939Q1|N|Y|N|2429265|2429264|2428913|2429186|N|N|N|N|N| +2429279|AAAAAAAAPFBBFCAA|1939-01-15|468|2037|157|1939|0|1|15|1|1939|157|2037|Sunday|1939Q1|N|N|N|2429265|2429264|2428914|2429187|N|N|N|N|N| +2429280|AAAAAAAAAGBBFCAA|1939-01-16|468|2037|157|1939|1|1|16|1|1939|157|2037|Monday|1939Q1|N|N|N|2429265|2429264|2428915|2429188|N|N|N|N|N| +2429281|AAAAAAAABGBBFCAA|1939-01-17|468|2038|157|1939|2|1|17|1|1939|157|2038|Tuesday|1939Q1|N|N|N|2429265|2429264|2428916|2429189|N|N|N|N|N| +2429282|AAAAAAAACGBBFCAA|1939-01-18|468|2038|157|1939|3|1|18|1|1939|157|2038|Wednesday|1939Q1|N|N|N|2429265|2429264|2428917|2429190|N|N|N|N|N| +2429283|AAAAAAAADGBBFCAA|1939-01-19|468|2038|157|1939|4|1|19|1|1939|157|2038|Thursday|1939Q1|N|N|N|2429265|2429264|2428918|2429191|N|N|N|N|N| +2429284|AAAAAAAAEGBBFCAA|1939-01-20|468|2038|157|1939|5|1|20|1|1939|157|2038|Friday|1939Q1|N|Y|N|2429265|2429264|2428919|2429192|N|N|N|N|N| +2429285|AAAAAAAAFGBBFCAA|1939-01-21|468|2038|157|1939|6|1|21|1|1939|157|2038|Saturday|1939Q1|N|Y|N|2429265|2429264|2428920|2429193|N|N|N|N|N| +2429286|AAAAAAAAGGBBFCAA|1939-01-22|468|2038|157|1939|0|1|22|1|1939|157|2038|Sunday|1939Q1|N|N|N|2429265|2429264|2428921|2429194|N|N|N|N|N| +2429287|AAAAAAAAHGBBFCAA|1939-01-23|468|2038|157|1939|1|1|23|1|1939|157|2038|Monday|1939Q1|N|N|N|2429265|2429264|2428922|2429195|N|N|N|N|N| +2429288|AAAAAAAAIGBBFCAA|1939-01-24|468|2039|157|1939|2|1|24|1|1939|157|2039|Tuesday|1939Q1|N|N|N|2429265|2429264|2428923|2429196|N|N|N|N|N| +2429289|AAAAAAAAJGBBFCAA|1939-01-25|468|2039|157|1939|3|1|25|1|1939|157|2039|Wednesday|1939Q1|N|N|N|2429265|2429264|2428924|2429197|N|N|N|N|N| +2429290|AAAAAAAAKGBBFCAA|1939-01-26|468|2039|157|1939|4|1|26|1|1939|157|2039|Thursday|1939Q1|N|N|N|2429265|2429264|2428925|2429198|N|N|N|N|N| +2429291|AAAAAAAALGBBFCAA|1939-01-27|468|2039|157|1939|5|1|27|1|1939|157|2039|Friday|1939Q1|N|Y|N|2429265|2429264|2428926|2429199|N|N|N|N|N| +2429292|AAAAAAAAMGBBFCAA|1939-01-28|468|2039|157|1939|6|1|28|1|1939|157|2039|Saturday|1939Q1|N|Y|N|2429265|2429264|2428927|2429200|N|N|N|N|N| +2429293|AAAAAAAANGBBFCAA|1939-01-29|468|2039|157|1939|0|1|29|1|1939|157|2039|Sunday|1939Q1|N|N|N|2429265|2429264|2428928|2429201|N|N|N|N|N| +2429294|AAAAAAAAOGBBFCAA|1939-01-30|468|2039|157|1939|1|1|30|1|1939|157|2039|Monday|1939Q1|N|N|N|2429265|2429264|2428929|2429202|N|N|N|N|N| +2429295|AAAAAAAAPGBBFCAA|1939-01-31|468|2040|157|1939|2|1|31|1|1939|157|2040|Tuesday|1939Q1|N|N|N|2429265|2429264|2428930|2429203|N|N|N|N|N| +2429296|AAAAAAAAAHBBFCAA|1939-02-01|469|2040|157|1939|3|2|1|1|1939|157|2040|Wednesday|1939Q1|N|N|N|2429296|2429326|2428931|2429204|N|N|N|N|N| +2429297|AAAAAAAABHBBFCAA|1939-02-02|469|2040|157|1939|4|2|2|1|1939|157|2040|Thursday|1939Q1|N|N|N|2429296|2429326|2428932|2429205|N|N|N|N|N| +2429298|AAAAAAAACHBBFCAA|1939-02-03|469|2040|157|1939|5|2|3|1|1939|157|2040|Friday|1939Q1|N|Y|N|2429296|2429326|2428933|2429206|N|N|N|N|N| +2429299|AAAAAAAADHBBFCAA|1939-02-04|469|2040|157|1939|6|2|4|1|1939|157|2040|Saturday|1939Q1|N|Y|N|2429296|2429326|2428934|2429207|N|N|N|N|N| +2429300|AAAAAAAAEHBBFCAA|1939-02-05|469|2040|157|1939|0|2|5|1|1939|157|2040|Sunday|1939Q1|N|N|N|2429296|2429326|2428935|2429208|N|N|N|N|N| +2429301|AAAAAAAAFHBBFCAA|1939-02-06|469|2040|157|1939|1|2|6|1|1939|157|2040|Monday|1939Q1|N|N|N|2429296|2429326|2428936|2429209|N|N|N|N|N| +2429302|AAAAAAAAGHBBFCAA|1939-02-07|469|2041|157|1939|2|2|7|1|1939|157|2041|Tuesday|1939Q1|N|N|N|2429296|2429326|2428937|2429210|N|N|N|N|N| +2429303|AAAAAAAAHHBBFCAA|1939-02-08|469|2041|157|1939|3|2|8|1|1939|157|2041|Wednesday|1939Q1|N|N|N|2429296|2429326|2428938|2429211|N|N|N|N|N| +2429304|AAAAAAAAIHBBFCAA|1939-02-09|469|2041|157|1939|4|2|9|1|1939|157|2041|Thursday|1939Q1|N|N|N|2429296|2429326|2428939|2429212|N|N|N|N|N| +2429305|AAAAAAAAJHBBFCAA|1939-02-10|469|2041|157|1939|5|2|10|1|1939|157|2041|Friday|1939Q1|N|Y|N|2429296|2429326|2428940|2429213|N|N|N|N|N| +2429306|AAAAAAAAKHBBFCAA|1939-02-11|469|2041|157|1939|6|2|11|1|1939|157|2041|Saturday|1939Q1|N|Y|N|2429296|2429326|2428941|2429214|N|N|N|N|N| +2429307|AAAAAAAALHBBFCAA|1939-02-12|469|2041|157|1939|0|2|12|1|1939|157|2041|Sunday|1939Q1|N|N|N|2429296|2429326|2428942|2429215|N|N|N|N|N| +2429308|AAAAAAAAMHBBFCAA|1939-02-13|469|2041|157|1939|1|2|13|1|1939|157|2041|Monday|1939Q1|N|N|N|2429296|2429326|2428943|2429216|N|N|N|N|N| +2429309|AAAAAAAANHBBFCAA|1939-02-14|469|2042|157|1939|2|2|14|1|1939|157|2042|Tuesday|1939Q1|N|N|N|2429296|2429326|2428944|2429217|N|N|N|N|N| +2429310|AAAAAAAAOHBBFCAA|1939-02-15|469|2042|157|1939|3|2|15|1|1939|157|2042|Wednesday|1939Q1|N|N|N|2429296|2429326|2428945|2429218|N|N|N|N|N| +2429311|AAAAAAAAPHBBFCAA|1939-02-16|469|2042|157|1939|4|2|16|1|1939|157|2042|Thursday|1939Q1|N|N|N|2429296|2429326|2428946|2429219|N|N|N|N|N| +2429312|AAAAAAAAAIBBFCAA|1939-02-17|469|2042|157|1939|5|2|17|1|1939|157|2042|Friday|1939Q1|N|Y|N|2429296|2429326|2428947|2429220|N|N|N|N|N| +2429313|AAAAAAAABIBBFCAA|1939-02-18|469|2042|157|1939|6|2|18|1|1939|157|2042|Saturday|1939Q1|N|Y|N|2429296|2429326|2428948|2429221|N|N|N|N|N| +2429314|AAAAAAAACIBBFCAA|1939-02-19|469|2042|157|1939|0|2|19|1|1939|157|2042|Sunday|1939Q1|N|N|N|2429296|2429326|2428949|2429222|N|N|N|N|N| +2429315|AAAAAAAADIBBFCAA|1939-02-20|469|2042|157|1939|1|2|20|1|1939|157|2042|Monday|1939Q1|N|N|N|2429296|2429326|2428950|2429223|N|N|N|N|N| +2429316|AAAAAAAAEIBBFCAA|1939-02-21|469|2043|157|1939|2|2|21|1|1939|157|2043|Tuesday|1939Q1|N|N|N|2429296|2429326|2428951|2429224|N|N|N|N|N| +2429317|AAAAAAAAFIBBFCAA|1939-02-22|469|2043|157|1939|3|2|22|1|1939|157|2043|Wednesday|1939Q1|N|N|N|2429296|2429326|2428952|2429225|N|N|N|N|N| +2429318|AAAAAAAAGIBBFCAA|1939-02-23|469|2043|157|1939|4|2|23|1|1939|157|2043|Thursday|1939Q1|N|N|N|2429296|2429326|2428953|2429226|N|N|N|N|N| +2429319|AAAAAAAAHIBBFCAA|1939-02-24|469|2043|157|1939|5|2|24|1|1939|157|2043|Friday|1939Q1|N|Y|N|2429296|2429326|2428954|2429227|N|N|N|N|N| +2429320|AAAAAAAAIIBBFCAA|1939-02-25|469|2043|157|1939|6|2|25|1|1939|157|2043|Saturday|1939Q1|N|Y|N|2429296|2429326|2428955|2429228|N|N|N|N|N| +2429321|AAAAAAAAJIBBFCAA|1939-02-26|469|2043|157|1939|0|2|26|1|1939|157|2043|Sunday|1939Q1|N|N|N|2429296|2429326|2428956|2429229|N|N|N|N|N| +2429322|AAAAAAAAKIBBFCAA|1939-02-27|469|2043|157|1939|1|2|27|1|1939|157|2043|Monday|1939Q1|N|N|N|2429296|2429326|2428957|2429230|N|N|N|N|N| +2429323|AAAAAAAALIBBFCAA|1939-02-28|469|2044|157|1939|2|2|28|1|1939|157|2044|Tuesday|1939Q1|N|N|N|2429296|2429326|2428958|2429231|N|N|N|N|N| +2429324|AAAAAAAAMIBBFCAA|1939-03-01|470|2044|158|1939|3|3|1|1|1939|158|2044|Wednesday|1939Q1|N|N|N|2429324|2429382|2428959|2429232|N|N|N|N|N| +2429325|AAAAAAAANIBBFCAA|1939-03-02|470|2044|158|1939|4|3|2|1|1939|158|2044|Thursday|1939Q1|N|N|N|2429324|2429382|2428960|2429233|N|N|N|N|N| +2429326|AAAAAAAAOIBBFCAA|1939-03-03|470|2044|158|1939|5|3|3|1|1939|158|2044|Friday|1939Q1|N|Y|N|2429324|2429382|2428961|2429234|N|N|N|N|N| +2429327|AAAAAAAAPIBBFCAA|1939-03-04|470|2044|158|1939|6|3|4|1|1939|158|2044|Saturday|1939Q1|N|Y|N|2429324|2429382|2428962|2429235|N|N|N|N|N| +2429328|AAAAAAAAAJBBFCAA|1939-03-05|470|2044|158|1939|0|3|5|1|1939|158|2044|Sunday|1939Q1|N|N|N|2429324|2429382|2428963|2429236|N|N|N|N|N| +2429329|AAAAAAAABJBBFCAA|1939-03-06|470|2044|158|1939|1|3|6|1|1939|158|2044|Monday|1939Q1|N|N|N|2429324|2429382|2428964|2429237|N|N|N|N|N| +2429330|AAAAAAAACJBBFCAA|1939-03-07|470|2045|158|1939|2|3|7|1|1939|158|2045|Tuesday|1939Q1|N|N|N|2429324|2429382|2428965|2429238|N|N|N|N|N| +2429331|AAAAAAAADJBBFCAA|1939-03-08|470|2045|158|1939|3|3|8|1|1939|158|2045|Wednesday|1939Q1|N|N|N|2429324|2429382|2428966|2429239|N|N|N|N|N| +2429332|AAAAAAAAEJBBFCAA|1939-03-09|470|2045|158|1939|4|3|9|1|1939|158|2045|Thursday|1939Q1|N|N|N|2429324|2429382|2428967|2429240|N|N|N|N|N| +2429333|AAAAAAAAFJBBFCAA|1939-03-10|470|2045|158|1939|5|3|10|1|1939|158|2045|Friday|1939Q1|N|Y|N|2429324|2429382|2428968|2429241|N|N|N|N|N| +2429334|AAAAAAAAGJBBFCAA|1939-03-11|470|2045|158|1939|6|3|11|1|1939|158|2045|Saturday|1939Q1|N|Y|N|2429324|2429382|2428969|2429242|N|N|N|N|N| +2429335|AAAAAAAAHJBBFCAA|1939-03-12|470|2045|158|1939|0|3|12|1|1939|158|2045|Sunday|1939Q1|N|N|N|2429324|2429382|2428970|2429243|N|N|N|N|N| +2429336|AAAAAAAAIJBBFCAA|1939-03-13|470|2045|158|1939|1|3|13|1|1939|158|2045|Monday|1939Q1|N|N|N|2429324|2429382|2428971|2429244|N|N|N|N|N| +2429337|AAAAAAAAJJBBFCAA|1939-03-14|470|2046|158|1939|2|3|14|1|1939|158|2046|Tuesday|1939Q1|N|N|N|2429324|2429382|2428972|2429245|N|N|N|N|N| +2429338|AAAAAAAAKJBBFCAA|1939-03-15|470|2046|158|1939|3|3|15|1|1939|158|2046|Wednesday|1939Q1|N|N|N|2429324|2429382|2428973|2429246|N|N|N|N|N| +2429339|AAAAAAAALJBBFCAA|1939-03-16|470|2046|158|1939|4|3|16|1|1939|158|2046|Thursday|1939Q1|N|N|N|2429324|2429382|2428974|2429247|N|N|N|N|N| +2429340|AAAAAAAAMJBBFCAA|1939-03-17|470|2046|158|1939|5|3|17|1|1939|158|2046|Friday|1939Q1|N|Y|N|2429324|2429382|2428975|2429248|N|N|N|N|N| +2429341|AAAAAAAANJBBFCAA|1939-03-18|470|2046|158|1939|6|3|18|1|1939|158|2046|Saturday|1939Q1|N|Y|N|2429324|2429382|2428976|2429249|N|N|N|N|N| +2429342|AAAAAAAAOJBBFCAA|1939-03-19|470|2046|158|1939|0|3|19|1|1939|158|2046|Sunday|1939Q1|N|N|N|2429324|2429382|2428977|2429250|N|N|N|N|N| +2429343|AAAAAAAAPJBBFCAA|1939-03-20|470|2046|158|1939|1|3|20|1|1939|158|2046|Monday|1939Q1|N|N|N|2429324|2429382|2428978|2429251|N|N|N|N|N| +2429344|AAAAAAAAAKBBFCAA|1939-03-21|470|2047|158|1939|2|3|21|1|1939|158|2047|Tuesday|1939Q1|N|N|N|2429324|2429382|2428979|2429252|N|N|N|N|N| +2429345|AAAAAAAABKBBFCAA|1939-03-22|470|2047|158|1939|3|3|22|1|1939|158|2047|Wednesday|1939Q1|N|N|N|2429324|2429382|2428980|2429253|N|N|N|N|N| +2429346|AAAAAAAACKBBFCAA|1939-03-23|470|2047|158|1939|4|3|23|1|1939|158|2047|Thursday|1939Q1|N|N|N|2429324|2429382|2428981|2429254|N|N|N|N|N| +2429347|AAAAAAAADKBBFCAA|1939-03-24|470|2047|158|1939|5|3|24|1|1939|158|2047|Friday|1939Q1|N|Y|N|2429324|2429382|2428982|2429255|N|N|N|N|N| +2429348|AAAAAAAAEKBBFCAA|1939-03-25|470|2047|158|1939|6|3|25|1|1939|158|2047|Saturday|1939Q1|N|Y|N|2429324|2429382|2428983|2429256|N|N|N|N|N| +2429349|AAAAAAAAFKBBFCAA|1939-03-26|470|2047|158|1939|0|3|26|1|1939|158|2047|Sunday|1939Q1|N|N|N|2429324|2429382|2428984|2429257|N|N|N|N|N| +2429350|AAAAAAAAGKBBFCAA|1939-03-27|470|2047|158|1939|1|3|27|1|1939|158|2047|Monday|1939Q1|N|N|N|2429324|2429382|2428985|2429258|N|N|N|N|N| +2429351|AAAAAAAAHKBBFCAA|1939-03-28|470|2048|158|1939|2|3|28|1|1939|158|2048|Tuesday|1939Q1|N|N|N|2429324|2429382|2428986|2429259|N|N|N|N|N| +2429352|AAAAAAAAIKBBFCAA|1939-03-29|470|2048|158|1939|3|3|29|1|1939|158|2048|Wednesday|1939Q1|N|N|N|2429324|2429382|2428987|2429260|N|N|N|N|N| +2429353|AAAAAAAAJKBBFCAA|1939-03-30|470|2048|158|1939|4|3|30|1|1939|158|2048|Thursday|1939Q1|N|N|N|2429324|2429382|2428988|2429261|N|N|N|N|N| +2429354|AAAAAAAAKKBBFCAA|1939-03-31|470|2048|158|1939|5|3|31|1|1939|158|2048|Friday|1939Q1|N|Y|N|2429324|2429382|2428989|2429262|N|N|N|N|N| +2429355|AAAAAAAALKBBFCAA|1939-04-01|471|2048|158|1939|6|4|1|1|1939|158|2048|Saturday|1939Q1|N|Y|N|2429355|2429444|2428990|2429265|N|N|N|N|N| +2429356|AAAAAAAAMKBBFCAA|1939-04-02|471|2048|158|1939|0|4|2|2|1939|158|2048|Sunday|1939Q2|N|N|N|2429355|2429444|2428991|2429266|N|N|N|N|N| +2429357|AAAAAAAANKBBFCAA|1939-04-03|471|2048|158|1939|1|4|3|2|1939|158|2048|Monday|1939Q2|N|N|N|2429355|2429444|2428992|2429267|N|N|N|N|N| +2429358|AAAAAAAAOKBBFCAA|1939-04-04|471|2049|158|1939|2|4|4|2|1939|158|2049|Tuesday|1939Q2|N|N|N|2429355|2429444|2428993|2429268|N|N|N|N|N| +2429359|AAAAAAAAPKBBFCAA|1939-04-05|471|2049|158|1939|3|4|5|2|1939|158|2049|Wednesday|1939Q2|N|N|N|2429355|2429444|2428994|2429269|N|N|N|N|N| +2429360|AAAAAAAAALBBFCAA|1939-04-06|471|2049|158|1939|4|4|6|2|1939|158|2049|Thursday|1939Q2|N|N|N|2429355|2429444|2428995|2429270|N|N|N|N|N| +2429361|AAAAAAAABLBBFCAA|1939-04-07|471|2049|158|1939|5|4|7|2|1939|158|2049|Friday|1939Q2|N|Y|N|2429355|2429444|2428996|2429271|N|N|N|N|N| +2429362|AAAAAAAACLBBFCAA|1939-04-08|471|2049|158|1939|6|4|8|2|1939|158|2049|Saturday|1939Q2|N|Y|N|2429355|2429444|2428997|2429272|N|N|N|N|N| +2429363|AAAAAAAADLBBFCAA|1939-04-09|471|2049|158|1939|0|4|9|2|1939|158|2049|Sunday|1939Q2|N|N|N|2429355|2429444|2428998|2429273|N|N|N|N|N| +2429364|AAAAAAAAELBBFCAA|1939-04-10|471|2049|158|1939|1|4|10|2|1939|158|2049|Monday|1939Q2|N|N|N|2429355|2429444|2428999|2429274|N|N|N|N|N| +2429365|AAAAAAAAFLBBFCAA|1939-04-11|471|2050|158|1939|2|4|11|2|1939|158|2050|Tuesday|1939Q2|N|N|N|2429355|2429444|2429000|2429275|N|N|N|N|N| +2429366|AAAAAAAAGLBBFCAA|1939-04-12|471|2050|158|1939|3|4|12|2|1939|158|2050|Wednesday|1939Q2|N|N|N|2429355|2429444|2429001|2429276|N|N|N|N|N| +2429367|AAAAAAAAHLBBFCAA|1939-04-13|471|2050|158|1939|4|4|13|2|1939|158|2050|Thursday|1939Q2|N|N|N|2429355|2429444|2429002|2429277|N|N|N|N|N| +2429368|AAAAAAAAILBBFCAA|1939-04-14|471|2050|158|1939|5|4|14|2|1939|158|2050|Friday|1939Q2|N|Y|N|2429355|2429444|2429003|2429278|N|N|N|N|N| +2429369|AAAAAAAAJLBBFCAA|1939-04-15|471|2050|158|1939|6|4|15|2|1939|158|2050|Saturday|1939Q2|N|Y|N|2429355|2429444|2429004|2429279|N|N|N|N|N| +2429370|AAAAAAAAKLBBFCAA|1939-04-16|471|2050|158|1939|0|4|16|2|1939|158|2050|Sunday|1939Q2|N|N|N|2429355|2429444|2429005|2429280|N|N|N|N|N| +2429371|AAAAAAAALLBBFCAA|1939-04-17|471|2050|158|1939|1|4|17|2|1939|158|2050|Monday|1939Q2|N|N|N|2429355|2429444|2429006|2429281|N|N|N|N|N| +2429372|AAAAAAAAMLBBFCAA|1939-04-18|471|2051|158|1939|2|4|18|2|1939|158|2051|Tuesday|1939Q2|N|N|N|2429355|2429444|2429007|2429282|N|N|N|N|N| +2429373|AAAAAAAANLBBFCAA|1939-04-19|471|2051|158|1939|3|4|19|2|1939|158|2051|Wednesday|1939Q2|N|N|N|2429355|2429444|2429008|2429283|N|N|N|N|N| +2429374|AAAAAAAAOLBBFCAA|1939-04-20|471|2051|158|1939|4|4|20|2|1939|158|2051|Thursday|1939Q2|N|N|N|2429355|2429444|2429009|2429284|N|N|N|N|N| +2429375|AAAAAAAAPLBBFCAA|1939-04-21|471|2051|158|1939|5|4|21|2|1939|158|2051|Friday|1939Q2|N|Y|N|2429355|2429444|2429010|2429285|N|N|N|N|N| +2429376|AAAAAAAAAMBBFCAA|1939-04-22|471|2051|158|1939|6|4|22|2|1939|158|2051|Saturday|1939Q2|N|Y|N|2429355|2429444|2429011|2429286|N|N|N|N|N| +2429377|AAAAAAAABMBBFCAA|1939-04-23|471|2051|158|1939|0|4|23|2|1939|158|2051|Sunday|1939Q2|N|N|N|2429355|2429444|2429012|2429287|N|N|N|N|N| +2429378|AAAAAAAACMBBFCAA|1939-04-24|471|2051|158|1939|1|4|24|2|1939|158|2051|Monday|1939Q2|N|N|N|2429355|2429444|2429013|2429288|N|N|N|N|N| +2429379|AAAAAAAADMBBFCAA|1939-04-25|471|2052|158|1939|2|4|25|2|1939|158|2052|Tuesday|1939Q2|N|N|N|2429355|2429444|2429014|2429289|N|N|N|N|N| +2429380|AAAAAAAAEMBBFCAA|1939-04-26|471|2052|158|1939|3|4|26|2|1939|158|2052|Wednesday|1939Q2|N|N|N|2429355|2429444|2429015|2429290|N|N|N|N|N| +2429381|AAAAAAAAFMBBFCAA|1939-04-27|471|2052|158|1939|4|4|27|2|1939|158|2052|Thursday|1939Q2|N|N|N|2429355|2429444|2429016|2429291|N|N|N|N|N| +2429382|AAAAAAAAGMBBFCAA|1939-04-28|471|2052|158|1939|5|4|28|2|1939|158|2052|Friday|1939Q2|N|Y|N|2429355|2429444|2429017|2429292|N|N|N|N|N| +2429383|AAAAAAAAHMBBFCAA|1939-04-29|471|2052|158|1939|6|4|29|2|1939|158|2052|Saturday|1939Q2|N|Y|N|2429355|2429444|2429018|2429293|N|N|N|N|N| +2429384|AAAAAAAAIMBBFCAA|1939-04-30|471|2052|158|1939|0|4|30|2|1939|158|2052|Sunday|1939Q2|N|N|N|2429355|2429444|2429019|2429294|N|N|N|N|N| +2429385|AAAAAAAAJMBBFCAA|1939-05-01|472|2052|158|1939|1|5|1|2|1939|158|2052|Monday|1939Q2|N|N|N|2429385|2429504|2429020|2429295|N|N|N|N|N| +2429386|AAAAAAAAKMBBFCAA|1939-05-02|472|2053|158|1939|2|5|2|2|1939|158|2053|Tuesday|1939Q2|N|N|N|2429385|2429504|2429021|2429296|N|N|N|N|N| +2429387|AAAAAAAALMBBFCAA|1939-05-03|472|2053|158|1939|3|5|3|2|1939|158|2053|Wednesday|1939Q2|N|N|N|2429385|2429504|2429022|2429297|N|N|N|N|N| +2429388|AAAAAAAAMMBBFCAA|1939-05-04|472|2053|158|1939|4|5|4|2|1939|158|2053|Thursday|1939Q2|N|N|N|2429385|2429504|2429023|2429298|N|N|N|N|N| +2429389|AAAAAAAANMBBFCAA|1939-05-05|472|2053|158|1939|5|5|5|2|1939|158|2053|Friday|1939Q2|N|Y|N|2429385|2429504|2429024|2429299|N|N|N|N|N| +2429390|AAAAAAAAOMBBFCAA|1939-05-06|472|2053|158|1939|6|5|6|2|1939|158|2053|Saturday|1939Q2|N|Y|N|2429385|2429504|2429025|2429300|N|N|N|N|N| +2429391|AAAAAAAAPMBBFCAA|1939-05-07|472|2053|158|1939|0|5|7|2|1939|158|2053|Sunday|1939Q2|N|N|N|2429385|2429504|2429026|2429301|N|N|N|N|N| +2429392|AAAAAAAAANBBFCAA|1939-05-08|472|2053|158|1939|1|5|8|2|1939|158|2053|Monday|1939Q2|N|N|N|2429385|2429504|2429027|2429302|N|N|N|N|N| +2429393|AAAAAAAABNBBFCAA|1939-05-09|472|2054|158|1939|2|5|9|2|1939|158|2054|Tuesday|1939Q2|N|N|N|2429385|2429504|2429028|2429303|N|N|N|N|N| +2429394|AAAAAAAACNBBFCAA|1939-05-10|472|2054|158|1939|3|5|10|2|1939|158|2054|Wednesday|1939Q2|N|N|N|2429385|2429504|2429029|2429304|N|N|N|N|N| +2429395|AAAAAAAADNBBFCAA|1939-05-11|472|2054|158|1939|4|5|11|2|1939|158|2054|Thursday|1939Q2|N|N|N|2429385|2429504|2429030|2429305|N|N|N|N|N| +2429396|AAAAAAAAENBBFCAA|1939-05-12|472|2054|158|1939|5|5|12|2|1939|158|2054|Friday|1939Q2|N|Y|N|2429385|2429504|2429031|2429306|N|N|N|N|N| +2429397|AAAAAAAAFNBBFCAA|1939-05-13|472|2054|158|1939|6|5|13|2|1939|158|2054|Saturday|1939Q2|N|Y|N|2429385|2429504|2429032|2429307|N|N|N|N|N| +2429398|AAAAAAAAGNBBFCAA|1939-05-14|472|2054|158|1939|0|5|14|2|1939|158|2054|Sunday|1939Q2|N|N|N|2429385|2429504|2429033|2429308|N|N|N|N|N| +2429399|AAAAAAAAHNBBFCAA|1939-05-15|472|2054|158|1939|1|5|15|2|1939|158|2054|Monday|1939Q2|N|N|N|2429385|2429504|2429034|2429309|N|N|N|N|N| +2429400|AAAAAAAAINBBFCAA|1939-05-16|472|2055|158|1939|2|5|16|2|1939|158|2055|Tuesday|1939Q2|N|N|N|2429385|2429504|2429035|2429310|N|N|N|N|N| +2429401|AAAAAAAAJNBBFCAA|1939-05-17|472|2055|158|1939|3|5|17|2|1939|158|2055|Wednesday|1939Q2|N|N|N|2429385|2429504|2429036|2429311|N|N|N|N|N| +2429402|AAAAAAAAKNBBFCAA|1939-05-18|472|2055|158|1939|4|5|18|2|1939|158|2055|Thursday|1939Q2|N|N|N|2429385|2429504|2429037|2429312|N|N|N|N|N| +2429403|AAAAAAAALNBBFCAA|1939-05-19|472|2055|158|1939|5|5|19|2|1939|158|2055|Friday|1939Q2|N|Y|N|2429385|2429504|2429038|2429313|N|N|N|N|N| +2429404|AAAAAAAAMNBBFCAA|1939-05-20|472|2055|158|1939|6|5|20|2|1939|158|2055|Saturday|1939Q2|N|Y|N|2429385|2429504|2429039|2429314|N|N|N|N|N| +2429405|AAAAAAAANNBBFCAA|1939-05-21|472|2055|158|1939|0|5|21|2|1939|158|2055|Sunday|1939Q2|N|N|N|2429385|2429504|2429040|2429315|N|N|N|N|N| +2429406|AAAAAAAAONBBFCAA|1939-05-22|472|2055|158|1939|1|5|22|2|1939|158|2055|Monday|1939Q2|N|N|N|2429385|2429504|2429041|2429316|N|N|N|N|N| +2429407|AAAAAAAAPNBBFCAA|1939-05-23|472|2056|158|1939|2|5|23|2|1939|158|2056|Tuesday|1939Q2|N|N|N|2429385|2429504|2429042|2429317|N|N|N|N|N| +2429408|AAAAAAAAAOBBFCAA|1939-05-24|472|2056|158|1939|3|5|24|2|1939|158|2056|Wednesday|1939Q2|N|N|N|2429385|2429504|2429043|2429318|N|N|N|N|N| +2429409|AAAAAAAABOBBFCAA|1939-05-25|472|2056|158|1939|4|5|25|2|1939|158|2056|Thursday|1939Q2|N|N|N|2429385|2429504|2429044|2429319|N|N|N|N|N| +2429410|AAAAAAAACOBBFCAA|1939-05-26|472|2056|158|1939|5|5|26|2|1939|158|2056|Friday|1939Q2|N|Y|N|2429385|2429504|2429045|2429320|N|N|N|N|N| +2429411|AAAAAAAADOBBFCAA|1939-05-27|472|2056|158|1939|6|5|27|2|1939|158|2056|Saturday|1939Q2|N|Y|N|2429385|2429504|2429046|2429321|N|N|N|N|N| +2429412|AAAAAAAAEOBBFCAA|1939-05-28|472|2056|158|1939|0|5|28|2|1939|158|2056|Sunday|1939Q2|N|N|N|2429385|2429504|2429047|2429322|N|N|N|N|N| +2429413|AAAAAAAAFOBBFCAA|1939-05-29|472|2056|158|1939|1|5|29|2|1939|158|2056|Monday|1939Q2|N|N|N|2429385|2429504|2429048|2429323|N|N|N|N|N| +2429414|AAAAAAAAGOBBFCAA|1939-05-30|472|2057|158|1939|2|5|30|2|1939|158|2057|Tuesday|1939Q2|N|N|N|2429385|2429504|2429049|2429324|N|N|N|N|N| +2429415|AAAAAAAAHOBBFCAA|1939-05-31|472|2057|158|1939|3|5|31|2|1939|158|2057|Wednesday|1939Q2|N|N|N|2429385|2429504|2429050|2429325|N|N|N|N|N| +2429416|AAAAAAAAIOBBFCAA|1939-06-01|473|2057|159|1939|4|6|1|2|1939|159|2057|Thursday|1939Q2|N|N|N|2429416|2429566|2429051|2429326|N|N|N|N|N| +2429417|AAAAAAAAJOBBFCAA|1939-06-02|473|2057|159|1939|5|6|2|2|1939|159|2057|Friday|1939Q2|N|Y|N|2429416|2429566|2429052|2429327|N|N|N|N|N| +2429418|AAAAAAAAKOBBFCAA|1939-06-03|473|2057|159|1939|6|6|3|2|1939|159|2057|Saturday|1939Q2|N|Y|N|2429416|2429566|2429053|2429328|N|N|N|N|N| +2429419|AAAAAAAALOBBFCAA|1939-06-04|473|2057|159|1939|0|6|4|2|1939|159|2057|Sunday|1939Q2|N|N|N|2429416|2429566|2429054|2429329|N|N|N|N|N| +2429420|AAAAAAAAMOBBFCAA|1939-06-05|473|2057|159|1939|1|6|5|2|1939|159|2057|Monday|1939Q2|N|N|N|2429416|2429566|2429055|2429330|N|N|N|N|N| +2429421|AAAAAAAANOBBFCAA|1939-06-06|473|2058|159|1939|2|6|6|2|1939|159|2058|Tuesday|1939Q2|N|N|N|2429416|2429566|2429056|2429331|N|N|N|N|N| +2429422|AAAAAAAAOOBBFCAA|1939-06-07|473|2058|159|1939|3|6|7|2|1939|159|2058|Wednesday|1939Q2|N|N|N|2429416|2429566|2429057|2429332|N|N|N|N|N| +2429423|AAAAAAAAPOBBFCAA|1939-06-08|473|2058|159|1939|4|6|8|2|1939|159|2058|Thursday|1939Q2|N|N|N|2429416|2429566|2429058|2429333|N|N|N|N|N| +2429424|AAAAAAAAAPBBFCAA|1939-06-09|473|2058|159|1939|5|6|9|2|1939|159|2058|Friday|1939Q2|N|Y|N|2429416|2429566|2429059|2429334|N|N|N|N|N| +2429425|AAAAAAAABPBBFCAA|1939-06-10|473|2058|159|1939|6|6|10|2|1939|159|2058|Saturday|1939Q2|N|Y|N|2429416|2429566|2429060|2429335|N|N|N|N|N| +2429426|AAAAAAAACPBBFCAA|1939-06-11|473|2058|159|1939|0|6|11|2|1939|159|2058|Sunday|1939Q2|N|N|N|2429416|2429566|2429061|2429336|N|N|N|N|N| +2429427|AAAAAAAADPBBFCAA|1939-06-12|473|2058|159|1939|1|6|12|2|1939|159|2058|Monday|1939Q2|N|N|N|2429416|2429566|2429062|2429337|N|N|N|N|N| +2429428|AAAAAAAAEPBBFCAA|1939-06-13|473|2059|159|1939|2|6|13|2|1939|159|2059|Tuesday|1939Q2|N|N|N|2429416|2429566|2429063|2429338|N|N|N|N|N| +2429429|AAAAAAAAFPBBFCAA|1939-06-14|473|2059|159|1939|3|6|14|2|1939|159|2059|Wednesday|1939Q2|N|N|N|2429416|2429566|2429064|2429339|N|N|N|N|N| +2429430|AAAAAAAAGPBBFCAA|1939-06-15|473|2059|159|1939|4|6|15|2|1939|159|2059|Thursday|1939Q2|N|N|N|2429416|2429566|2429065|2429340|N|N|N|N|N| +2429431|AAAAAAAAHPBBFCAA|1939-06-16|473|2059|159|1939|5|6|16|2|1939|159|2059|Friday|1939Q2|N|Y|N|2429416|2429566|2429066|2429341|N|N|N|N|N| +2429432|AAAAAAAAIPBBFCAA|1939-06-17|473|2059|159|1939|6|6|17|2|1939|159|2059|Saturday|1939Q2|N|Y|N|2429416|2429566|2429067|2429342|N|N|N|N|N| +2429433|AAAAAAAAJPBBFCAA|1939-06-18|473|2059|159|1939|0|6|18|2|1939|159|2059|Sunday|1939Q2|N|N|N|2429416|2429566|2429068|2429343|N|N|N|N|N| +2429434|AAAAAAAAKPBBFCAA|1939-06-19|473|2059|159|1939|1|6|19|2|1939|159|2059|Monday|1939Q2|N|N|N|2429416|2429566|2429069|2429344|N|N|N|N|N| +2429435|AAAAAAAALPBBFCAA|1939-06-20|473|2060|159|1939|2|6|20|2|1939|159|2060|Tuesday|1939Q2|N|N|N|2429416|2429566|2429070|2429345|N|N|N|N|N| +2429436|AAAAAAAAMPBBFCAA|1939-06-21|473|2060|159|1939|3|6|21|2|1939|159|2060|Wednesday|1939Q2|N|N|N|2429416|2429566|2429071|2429346|N|N|N|N|N| +2429437|AAAAAAAANPBBFCAA|1939-06-22|473|2060|159|1939|4|6|22|2|1939|159|2060|Thursday|1939Q2|N|N|N|2429416|2429566|2429072|2429347|N|N|N|N|N| +2429438|AAAAAAAAOPBBFCAA|1939-06-23|473|2060|159|1939|5|6|23|2|1939|159|2060|Friday|1939Q2|N|Y|N|2429416|2429566|2429073|2429348|N|N|N|N|N| +2429439|AAAAAAAAPPBBFCAA|1939-06-24|473|2060|159|1939|6|6|24|2|1939|159|2060|Saturday|1939Q2|N|Y|N|2429416|2429566|2429074|2429349|N|N|N|N|N| +2429440|AAAAAAAAAACBFCAA|1939-06-25|473|2060|159|1939|0|6|25|2|1939|159|2060|Sunday|1939Q2|N|N|N|2429416|2429566|2429075|2429350|N|N|N|N|N| +2429441|AAAAAAAABACBFCAA|1939-06-26|473|2060|159|1939|1|6|26|2|1939|159|2060|Monday|1939Q2|N|N|N|2429416|2429566|2429076|2429351|N|N|N|N|N| +2429442|AAAAAAAACACBFCAA|1939-06-27|473|2061|159|1939|2|6|27|2|1939|159|2061|Tuesday|1939Q2|N|N|N|2429416|2429566|2429077|2429352|N|N|N|N|N| +2429443|AAAAAAAADACBFCAA|1939-06-28|473|2061|159|1939|3|6|28|2|1939|159|2061|Wednesday|1939Q2|N|N|N|2429416|2429566|2429078|2429353|N|N|N|N|N| +2429444|AAAAAAAAEACBFCAA|1939-06-29|473|2061|159|1939|4|6|29|2|1939|159|2061|Thursday|1939Q2|N|N|N|2429416|2429566|2429079|2429354|N|N|N|N|N| +2429445|AAAAAAAAFACBFCAA|1939-06-30|473|2061|159|1939|5|6|30|2|1939|159|2061|Friday|1939Q2|N|Y|N|2429416|2429566|2429080|2429355|N|N|N|N|N| +2429446|AAAAAAAAGACBFCAA|1939-07-01|474|2061|159|1939|6|7|1|2|1939|159|2061|Saturday|1939Q2|N|Y|N|2429446|2429626|2429081|2429355|N|N|N|N|N| +2429447|AAAAAAAAHACBFCAA|1939-07-02|474|2061|159|1939|0|7|2|3|1939|159|2061|Sunday|1939Q3|N|N|N|2429446|2429626|2429082|2429356|N|N|N|N|N| +2429448|AAAAAAAAIACBFCAA|1939-07-03|474|2061|159|1939|1|7|3|3|1939|159|2061|Monday|1939Q3|N|N|N|2429446|2429626|2429083|2429357|N|N|N|N|N| +2429449|AAAAAAAAJACBFCAA|1939-07-04|474|2062|159|1939|2|7|4|3|1939|159|2062|Tuesday|1939Q3|N|N|N|2429446|2429626|2429084|2429358|N|N|N|N|N| +2429450|AAAAAAAAKACBFCAA|1939-07-05|474|2062|159|1939|3|7|5|3|1939|159|2062|Wednesday|1939Q3|Y|N|N|2429446|2429626|2429085|2429359|N|N|N|N|N| +2429451|AAAAAAAALACBFCAA|1939-07-06|474|2062|159|1939|4|7|6|3|1939|159|2062|Thursday|1939Q3|N|N|Y|2429446|2429626|2429086|2429360|N|N|N|N|N| +2429452|AAAAAAAAMACBFCAA|1939-07-07|474|2062|159|1939|5|7|7|3|1939|159|2062|Friday|1939Q3|N|Y|N|2429446|2429626|2429087|2429361|N|N|N|N|N| +2429453|AAAAAAAANACBFCAA|1939-07-08|474|2062|159|1939|6|7|8|3|1939|159|2062|Saturday|1939Q3|N|Y|N|2429446|2429626|2429088|2429362|N|N|N|N|N| +2429454|AAAAAAAAOACBFCAA|1939-07-09|474|2062|159|1939|0|7|9|3|1939|159|2062|Sunday|1939Q3|N|N|N|2429446|2429626|2429089|2429363|N|N|N|N|N| +2429455|AAAAAAAAPACBFCAA|1939-07-10|474|2062|159|1939|1|7|10|3|1939|159|2062|Monday|1939Q3|N|N|N|2429446|2429626|2429090|2429364|N|N|N|N|N| +2429456|AAAAAAAAABCBFCAA|1939-07-11|474|2063|159|1939|2|7|11|3|1939|159|2063|Tuesday|1939Q3|N|N|N|2429446|2429626|2429091|2429365|N|N|N|N|N| +2429457|AAAAAAAABBCBFCAA|1939-07-12|474|2063|159|1939|3|7|12|3|1939|159|2063|Wednesday|1939Q3|N|N|N|2429446|2429626|2429092|2429366|N|N|N|N|N| +2429458|AAAAAAAACBCBFCAA|1939-07-13|474|2063|159|1939|4|7|13|3|1939|159|2063|Thursday|1939Q3|N|N|N|2429446|2429626|2429093|2429367|N|N|N|N|N| +2429459|AAAAAAAADBCBFCAA|1939-07-14|474|2063|159|1939|5|7|14|3|1939|159|2063|Friday|1939Q3|N|Y|N|2429446|2429626|2429094|2429368|N|N|N|N|N| +2429460|AAAAAAAAEBCBFCAA|1939-07-15|474|2063|159|1939|6|7|15|3|1939|159|2063|Saturday|1939Q3|N|Y|N|2429446|2429626|2429095|2429369|N|N|N|N|N| +2429461|AAAAAAAAFBCBFCAA|1939-07-16|474|2063|159|1939|0|7|16|3|1939|159|2063|Sunday|1939Q3|N|N|N|2429446|2429626|2429096|2429370|N|N|N|N|N| +2429462|AAAAAAAAGBCBFCAA|1939-07-17|474|2063|159|1939|1|7|17|3|1939|159|2063|Monday|1939Q3|N|N|N|2429446|2429626|2429097|2429371|N|N|N|N|N| +2429463|AAAAAAAAHBCBFCAA|1939-07-18|474|2064|159|1939|2|7|18|3|1939|159|2064|Tuesday|1939Q3|N|N|N|2429446|2429626|2429098|2429372|N|N|N|N|N| +2429464|AAAAAAAAIBCBFCAA|1939-07-19|474|2064|159|1939|3|7|19|3|1939|159|2064|Wednesday|1939Q3|N|N|N|2429446|2429626|2429099|2429373|N|N|N|N|N| +2429465|AAAAAAAAJBCBFCAA|1939-07-20|474|2064|159|1939|4|7|20|3|1939|159|2064|Thursday|1939Q3|N|N|N|2429446|2429626|2429100|2429374|N|N|N|N|N| +2429466|AAAAAAAAKBCBFCAA|1939-07-21|474|2064|159|1939|5|7|21|3|1939|159|2064|Friday|1939Q3|N|Y|N|2429446|2429626|2429101|2429375|N|N|N|N|N| +2429467|AAAAAAAALBCBFCAA|1939-07-22|474|2064|159|1939|6|7|22|3|1939|159|2064|Saturday|1939Q3|N|Y|N|2429446|2429626|2429102|2429376|N|N|N|N|N| +2429468|AAAAAAAAMBCBFCAA|1939-07-23|474|2064|159|1939|0|7|23|3|1939|159|2064|Sunday|1939Q3|N|N|N|2429446|2429626|2429103|2429377|N|N|N|N|N| +2429469|AAAAAAAANBCBFCAA|1939-07-24|474|2064|159|1939|1|7|24|3|1939|159|2064|Monday|1939Q3|N|N|N|2429446|2429626|2429104|2429378|N|N|N|N|N| +2429470|AAAAAAAAOBCBFCAA|1939-07-25|474|2065|159|1939|2|7|25|3|1939|159|2065|Tuesday|1939Q3|N|N|N|2429446|2429626|2429105|2429379|N|N|N|N|N| +2429471|AAAAAAAAPBCBFCAA|1939-07-26|474|2065|159|1939|3|7|26|3|1939|159|2065|Wednesday|1939Q3|N|N|N|2429446|2429626|2429106|2429380|N|N|N|N|N| +2429472|AAAAAAAAACCBFCAA|1939-07-27|474|2065|159|1939|4|7|27|3|1939|159|2065|Thursday|1939Q3|N|N|N|2429446|2429626|2429107|2429381|N|N|N|N|N| +2429473|AAAAAAAABCCBFCAA|1939-07-28|474|2065|159|1939|5|7|28|3|1939|159|2065|Friday|1939Q3|N|Y|N|2429446|2429626|2429108|2429382|N|N|N|N|N| +2429474|AAAAAAAACCCBFCAA|1939-07-29|474|2065|159|1939|6|7|29|3|1939|159|2065|Saturday|1939Q3|N|Y|N|2429446|2429626|2429109|2429383|N|N|N|N|N| +2429475|AAAAAAAADCCBFCAA|1939-07-30|474|2065|159|1939|0|7|30|3|1939|159|2065|Sunday|1939Q3|N|N|N|2429446|2429626|2429110|2429384|N|N|N|N|N| +2429476|AAAAAAAAECCBFCAA|1939-07-31|474|2065|159|1939|1|7|31|3|1939|159|2065|Monday|1939Q3|N|N|N|2429446|2429626|2429111|2429385|N|N|N|N|N| +2429477|AAAAAAAAFCCBFCAA|1939-08-01|475|2066|159|1939|2|8|1|3|1939|159|2066|Tuesday|1939Q3|N|N|N|2429477|2429688|2429112|2429386|N|N|N|N|N| +2429478|AAAAAAAAGCCBFCAA|1939-08-02|475|2066|159|1939|3|8|2|3|1939|159|2066|Wednesday|1939Q3|N|N|N|2429477|2429688|2429113|2429387|N|N|N|N|N| +2429479|AAAAAAAAHCCBFCAA|1939-08-03|475|2066|159|1939|4|8|3|3|1939|159|2066|Thursday|1939Q3|N|N|N|2429477|2429688|2429114|2429388|N|N|N|N|N| +2429480|AAAAAAAAICCBFCAA|1939-08-04|475|2066|159|1939|5|8|4|3|1939|159|2066|Friday|1939Q3|N|Y|N|2429477|2429688|2429115|2429389|N|N|N|N|N| +2429481|AAAAAAAAJCCBFCAA|1939-08-05|475|2066|159|1939|6|8|5|3|1939|159|2066|Saturday|1939Q3|N|Y|N|2429477|2429688|2429116|2429390|N|N|N|N|N| +2429482|AAAAAAAAKCCBFCAA|1939-08-06|475|2066|159|1939|0|8|6|3|1939|159|2066|Sunday|1939Q3|N|N|N|2429477|2429688|2429117|2429391|N|N|N|N|N| +2429483|AAAAAAAALCCBFCAA|1939-08-07|475|2066|159|1939|1|8|7|3|1939|159|2066|Monday|1939Q3|N|N|N|2429477|2429688|2429118|2429392|N|N|N|N|N| +2429484|AAAAAAAAMCCBFCAA|1939-08-08|475|2067|159|1939|2|8|8|3|1939|159|2067|Tuesday|1939Q3|N|N|N|2429477|2429688|2429119|2429393|N|N|N|N|N| +2429485|AAAAAAAANCCBFCAA|1939-08-09|475|2067|159|1939|3|8|9|3|1939|159|2067|Wednesday|1939Q3|N|N|N|2429477|2429688|2429120|2429394|N|N|N|N|N| +2429486|AAAAAAAAOCCBFCAA|1939-08-10|475|2067|159|1939|4|8|10|3|1939|159|2067|Thursday|1939Q3|N|N|N|2429477|2429688|2429121|2429395|N|N|N|N|N| +2429487|AAAAAAAAPCCBFCAA|1939-08-11|475|2067|159|1939|5|8|11|3|1939|159|2067|Friday|1939Q3|N|Y|N|2429477|2429688|2429122|2429396|N|N|N|N|N| +2429488|AAAAAAAAADCBFCAA|1939-08-12|475|2067|159|1939|6|8|12|3|1939|159|2067|Saturday|1939Q3|N|Y|N|2429477|2429688|2429123|2429397|N|N|N|N|N| +2429489|AAAAAAAABDCBFCAA|1939-08-13|475|2067|159|1939|0|8|13|3|1939|159|2067|Sunday|1939Q3|N|N|N|2429477|2429688|2429124|2429398|N|N|N|N|N| +2429490|AAAAAAAACDCBFCAA|1939-08-14|475|2067|159|1939|1|8|14|3|1939|159|2067|Monday|1939Q3|N|N|N|2429477|2429688|2429125|2429399|N|N|N|N|N| +2429491|AAAAAAAADDCBFCAA|1939-08-15|475|2068|159|1939|2|8|15|3|1939|159|2068|Tuesday|1939Q3|N|N|N|2429477|2429688|2429126|2429400|N|N|N|N|N| +2429492|AAAAAAAAEDCBFCAA|1939-08-16|475|2068|159|1939|3|8|16|3|1939|159|2068|Wednesday|1939Q3|N|N|N|2429477|2429688|2429127|2429401|N|N|N|N|N| +2429493|AAAAAAAAFDCBFCAA|1939-08-17|475|2068|159|1939|4|8|17|3|1939|159|2068|Thursday|1939Q3|N|N|N|2429477|2429688|2429128|2429402|N|N|N|N|N| +2429494|AAAAAAAAGDCBFCAA|1939-08-18|475|2068|159|1939|5|8|18|3|1939|159|2068|Friday|1939Q3|N|Y|N|2429477|2429688|2429129|2429403|N|N|N|N|N| +2429495|AAAAAAAAHDCBFCAA|1939-08-19|475|2068|159|1939|6|8|19|3|1939|159|2068|Saturday|1939Q3|N|Y|N|2429477|2429688|2429130|2429404|N|N|N|N|N| +2429496|AAAAAAAAIDCBFCAA|1939-08-20|475|2068|159|1939|0|8|20|3|1939|159|2068|Sunday|1939Q3|N|N|N|2429477|2429688|2429131|2429405|N|N|N|N|N| +2429497|AAAAAAAAJDCBFCAA|1939-08-21|475|2068|159|1939|1|8|21|3|1939|159|2068|Monday|1939Q3|N|N|N|2429477|2429688|2429132|2429406|N|N|N|N|N| +2429498|AAAAAAAAKDCBFCAA|1939-08-22|475|2069|159|1939|2|8|22|3|1939|159|2069|Tuesday|1939Q3|N|N|N|2429477|2429688|2429133|2429407|N|N|N|N|N| +2429499|AAAAAAAALDCBFCAA|1939-08-23|475|2069|159|1939|3|8|23|3|1939|159|2069|Wednesday|1939Q3|N|N|N|2429477|2429688|2429134|2429408|N|N|N|N|N| +2429500|AAAAAAAAMDCBFCAA|1939-08-24|475|2069|159|1939|4|8|24|3|1939|159|2069|Thursday|1939Q3|N|N|N|2429477|2429688|2429135|2429409|N|N|N|N|N| +2429501|AAAAAAAANDCBFCAA|1939-08-25|475|2069|159|1939|5|8|25|3|1939|159|2069|Friday|1939Q3|N|Y|N|2429477|2429688|2429136|2429410|N|N|N|N|N| +2429502|AAAAAAAAODCBFCAA|1939-08-26|475|2069|159|1939|6|8|26|3|1939|159|2069|Saturday|1939Q3|N|Y|N|2429477|2429688|2429137|2429411|N|N|N|N|N| +2429503|AAAAAAAAPDCBFCAA|1939-08-27|475|2069|159|1939|0|8|27|3|1939|159|2069|Sunday|1939Q3|N|N|N|2429477|2429688|2429138|2429412|N|N|N|N|N| +2429504|AAAAAAAAAECBFCAA|1939-08-28|475|2069|159|1939|1|8|28|3|1939|159|2069|Monday|1939Q3|N|N|N|2429477|2429688|2429139|2429413|N|N|N|N|N| +2429505|AAAAAAAABECBFCAA|1939-08-29|475|2070|159|1939|2|8|29|3|1939|159|2070|Tuesday|1939Q3|N|N|N|2429477|2429688|2429140|2429414|N|N|N|N|N| +2429506|AAAAAAAACECBFCAA|1939-08-30|475|2070|159|1939|3|8|30|3|1939|159|2070|Wednesday|1939Q3|N|N|N|2429477|2429688|2429141|2429415|N|N|N|N|N| +2429507|AAAAAAAADECBFCAA|1939-08-31|475|2070|159|1939|4|8|31|3|1939|159|2070|Thursday|1939Q3|N|N|N|2429477|2429688|2429142|2429416|N|N|N|N|N| +2429508|AAAAAAAAEECBFCAA|1939-09-01|476|2070|160|1939|5|9|1|3|1939|160|2070|Friday|1939Q3|N|Y|N|2429508|2429750|2429143|2429417|N|N|N|N|N| +2429509|AAAAAAAAFECBFCAA|1939-09-02|476|2070|160|1939|6|9|2|3|1939|160|2070|Saturday|1939Q3|N|Y|N|2429508|2429750|2429144|2429418|N|N|N|N|N| +2429510|AAAAAAAAGECBFCAA|1939-09-03|476|2070|160|1939|0|9|3|3|1939|160|2070|Sunday|1939Q3|N|N|N|2429508|2429750|2429145|2429419|N|N|N|N|N| +2429511|AAAAAAAAHECBFCAA|1939-09-04|476|2070|160|1939|1|9|4|3|1939|160|2070|Monday|1939Q3|N|N|N|2429508|2429750|2429146|2429420|N|N|N|N|N| +2429512|AAAAAAAAIECBFCAA|1939-09-05|476|2071|160|1939|2|9|5|3|1939|160|2071|Tuesday|1939Q3|N|N|N|2429508|2429750|2429147|2429421|N|N|N|N|N| +2429513|AAAAAAAAJECBFCAA|1939-09-06|476|2071|160|1939|3|9|6|3|1939|160|2071|Wednesday|1939Q3|N|N|N|2429508|2429750|2429148|2429422|N|N|N|N|N| +2429514|AAAAAAAAKECBFCAA|1939-09-07|476|2071|160|1939|4|9|7|3|1939|160|2071|Thursday|1939Q3|N|N|N|2429508|2429750|2429149|2429423|N|N|N|N|N| +2429515|AAAAAAAALECBFCAA|1939-09-08|476|2071|160|1939|5|9|8|3|1939|160|2071|Friday|1939Q3|N|Y|N|2429508|2429750|2429150|2429424|N|N|N|N|N| +2429516|AAAAAAAAMECBFCAA|1939-09-09|476|2071|160|1939|6|9|9|3|1939|160|2071|Saturday|1939Q3|N|Y|N|2429508|2429750|2429151|2429425|N|N|N|N|N| +2429517|AAAAAAAANECBFCAA|1939-09-10|476|2071|160|1939|0|9|10|3|1939|160|2071|Sunday|1939Q3|N|N|N|2429508|2429750|2429152|2429426|N|N|N|N|N| +2429518|AAAAAAAAOECBFCAA|1939-09-11|476|2071|160|1939|1|9|11|3|1939|160|2071|Monday|1939Q3|N|N|N|2429508|2429750|2429153|2429427|N|N|N|N|N| +2429519|AAAAAAAAPECBFCAA|1939-09-12|476|2072|160|1939|2|9|12|3|1939|160|2072|Tuesday|1939Q3|N|N|N|2429508|2429750|2429154|2429428|N|N|N|N|N| +2429520|AAAAAAAAAFCBFCAA|1939-09-13|476|2072|160|1939|3|9|13|3|1939|160|2072|Wednesday|1939Q3|N|N|N|2429508|2429750|2429155|2429429|N|N|N|N|N| +2429521|AAAAAAAABFCBFCAA|1939-09-14|476|2072|160|1939|4|9|14|3|1939|160|2072|Thursday|1939Q3|N|N|N|2429508|2429750|2429156|2429430|N|N|N|N|N| +2429522|AAAAAAAACFCBFCAA|1939-09-15|476|2072|160|1939|5|9|15|3|1939|160|2072|Friday|1939Q3|N|Y|N|2429508|2429750|2429157|2429431|N|N|N|N|N| +2429523|AAAAAAAADFCBFCAA|1939-09-16|476|2072|160|1939|6|9|16|3|1939|160|2072|Saturday|1939Q3|N|Y|N|2429508|2429750|2429158|2429432|N|N|N|N|N| +2429524|AAAAAAAAEFCBFCAA|1939-09-17|476|2072|160|1939|0|9|17|3|1939|160|2072|Sunday|1939Q3|N|N|N|2429508|2429750|2429159|2429433|N|N|N|N|N| +2429525|AAAAAAAAFFCBFCAA|1939-09-18|476|2072|160|1939|1|9|18|3|1939|160|2072|Monday|1939Q3|N|N|N|2429508|2429750|2429160|2429434|N|N|N|N|N| +2429526|AAAAAAAAGFCBFCAA|1939-09-19|476|2073|160|1939|2|9|19|3|1939|160|2073|Tuesday|1939Q3|N|N|N|2429508|2429750|2429161|2429435|N|N|N|N|N| +2429527|AAAAAAAAHFCBFCAA|1939-09-20|476|2073|160|1939|3|9|20|3|1939|160|2073|Wednesday|1939Q3|N|N|N|2429508|2429750|2429162|2429436|N|N|N|N|N| +2429528|AAAAAAAAIFCBFCAA|1939-09-21|476|2073|160|1939|4|9|21|3|1939|160|2073|Thursday|1939Q3|N|N|N|2429508|2429750|2429163|2429437|N|N|N|N|N| +2429529|AAAAAAAAJFCBFCAA|1939-09-22|476|2073|160|1939|5|9|22|3|1939|160|2073|Friday|1939Q3|N|Y|N|2429508|2429750|2429164|2429438|N|N|N|N|N| +2429530|AAAAAAAAKFCBFCAA|1939-09-23|476|2073|160|1939|6|9|23|3|1939|160|2073|Saturday|1939Q3|N|Y|N|2429508|2429750|2429165|2429439|N|N|N|N|N| +2429531|AAAAAAAALFCBFCAA|1939-09-24|476|2073|160|1939|0|9|24|3|1939|160|2073|Sunday|1939Q3|N|N|N|2429508|2429750|2429166|2429440|N|N|N|N|N| +2429532|AAAAAAAAMFCBFCAA|1939-09-25|476|2073|160|1939|1|9|25|3|1939|160|2073|Monday|1939Q3|N|N|N|2429508|2429750|2429167|2429441|N|N|N|N|N| +2429533|AAAAAAAANFCBFCAA|1939-09-26|476|2074|160|1939|2|9|26|3|1939|160|2074|Tuesday|1939Q3|N|N|N|2429508|2429750|2429168|2429442|N|N|N|N|N| +2429534|AAAAAAAAOFCBFCAA|1939-09-27|476|2074|160|1939|3|9|27|3|1939|160|2074|Wednesday|1939Q3|N|N|N|2429508|2429750|2429169|2429443|N|N|N|N|N| +2429535|AAAAAAAAPFCBFCAA|1939-09-28|476|2074|160|1939|4|9|28|3|1939|160|2074|Thursday|1939Q3|N|N|N|2429508|2429750|2429170|2429444|N|N|N|N|N| +2429536|AAAAAAAAAGCBFCAA|1939-09-29|476|2074|160|1939|5|9|29|3|1939|160|2074|Friday|1939Q3|N|Y|N|2429508|2429750|2429171|2429445|N|N|N|N|N| +2429537|AAAAAAAABGCBFCAA|1939-09-30|476|2074|160|1939|6|9|30|3|1939|160|2074|Saturday|1939Q3|N|Y|N|2429508|2429750|2429172|2429446|N|N|N|N|N| +2429538|AAAAAAAACGCBFCAA|1939-10-01|477|2074|160|1939|0|10|1|3|1939|160|2074|Sunday|1939Q3|N|N|N|2429538|2429810|2429173|2429446|N|N|N|N|N| +2429539|AAAAAAAADGCBFCAA|1939-10-02|477|2074|160|1939|1|10|2|4|1939|160|2074|Monday|1939Q4|N|N|N|2429538|2429810|2429174|2429447|N|N|N|N|N| +2429540|AAAAAAAAEGCBFCAA|1939-10-03|477|2075|160|1939|2|10|3|4|1939|160|2075|Tuesday|1939Q4|N|N|N|2429538|2429810|2429175|2429448|N|N|N|N|N| +2429541|AAAAAAAAFGCBFCAA|1939-10-04|477|2075|160|1939|3|10|4|4|1939|160|2075|Wednesday|1939Q4|N|N|N|2429538|2429810|2429176|2429449|N|N|N|N|N| +2429542|AAAAAAAAGGCBFCAA|1939-10-05|477|2075|160|1939|4|10|5|4|1939|160|2075|Thursday|1939Q4|N|N|N|2429538|2429810|2429177|2429450|N|N|N|N|N| +2429543|AAAAAAAAHGCBFCAA|1939-10-06|477|2075|160|1939|5|10|6|4|1939|160|2075|Friday|1939Q4|N|Y|N|2429538|2429810|2429178|2429451|N|N|N|N|N| +2429544|AAAAAAAAIGCBFCAA|1939-10-07|477|2075|160|1939|6|10|7|4|1939|160|2075|Saturday|1939Q4|N|Y|N|2429538|2429810|2429179|2429452|N|N|N|N|N| +2429545|AAAAAAAAJGCBFCAA|1939-10-08|477|2075|160|1939|0|10|8|4|1939|160|2075|Sunday|1939Q4|N|N|N|2429538|2429810|2429180|2429453|N|N|N|N|N| +2429546|AAAAAAAAKGCBFCAA|1939-10-09|477|2075|160|1939|1|10|9|4|1939|160|2075|Monday|1939Q4|N|N|N|2429538|2429810|2429181|2429454|N|N|N|N|N| +2429547|AAAAAAAALGCBFCAA|1939-10-10|477|2076|160|1939|2|10|10|4|1939|160|2076|Tuesday|1939Q4|N|N|N|2429538|2429810|2429182|2429455|N|N|N|N|N| +2429548|AAAAAAAAMGCBFCAA|1939-10-11|477|2076|160|1939|3|10|11|4|1939|160|2076|Wednesday|1939Q4|N|N|N|2429538|2429810|2429183|2429456|N|N|N|N|N| +2429549|AAAAAAAANGCBFCAA|1939-10-12|477|2076|160|1939|4|10|12|4|1939|160|2076|Thursday|1939Q4|N|N|N|2429538|2429810|2429184|2429457|N|N|N|N|N| +2429550|AAAAAAAAOGCBFCAA|1939-10-13|477|2076|160|1939|5|10|13|4|1939|160|2076|Friday|1939Q4|N|Y|N|2429538|2429810|2429185|2429458|N|N|N|N|N| +2429551|AAAAAAAAPGCBFCAA|1939-10-14|477|2076|160|1939|6|10|14|4|1939|160|2076|Saturday|1939Q4|N|Y|N|2429538|2429810|2429186|2429459|N|N|N|N|N| +2429552|AAAAAAAAAHCBFCAA|1939-10-15|477|2076|160|1939|0|10|15|4|1939|160|2076|Sunday|1939Q4|N|N|N|2429538|2429810|2429187|2429460|N|N|N|N|N| +2429553|AAAAAAAABHCBFCAA|1939-10-16|477|2076|160|1939|1|10|16|4|1939|160|2076|Monday|1939Q4|N|N|N|2429538|2429810|2429188|2429461|N|N|N|N|N| +2429554|AAAAAAAACHCBFCAA|1939-10-17|477|2077|160|1939|2|10|17|4|1939|160|2077|Tuesday|1939Q4|N|N|N|2429538|2429810|2429189|2429462|N|N|N|N|N| +2429555|AAAAAAAADHCBFCAA|1939-10-18|477|2077|160|1939|3|10|18|4|1939|160|2077|Wednesday|1939Q4|N|N|N|2429538|2429810|2429190|2429463|N|N|N|N|N| +2429556|AAAAAAAAEHCBFCAA|1939-10-19|477|2077|160|1939|4|10|19|4|1939|160|2077|Thursday|1939Q4|N|N|N|2429538|2429810|2429191|2429464|N|N|N|N|N| +2429557|AAAAAAAAFHCBFCAA|1939-10-20|477|2077|160|1939|5|10|20|4|1939|160|2077|Friday|1939Q4|N|Y|N|2429538|2429810|2429192|2429465|N|N|N|N|N| +2429558|AAAAAAAAGHCBFCAA|1939-10-21|477|2077|160|1939|6|10|21|4|1939|160|2077|Saturday|1939Q4|N|Y|N|2429538|2429810|2429193|2429466|N|N|N|N|N| +2429559|AAAAAAAAHHCBFCAA|1939-10-22|477|2077|160|1939|0|10|22|4|1939|160|2077|Sunday|1939Q4|N|N|N|2429538|2429810|2429194|2429467|N|N|N|N|N| +2429560|AAAAAAAAIHCBFCAA|1939-10-23|477|2077|160|1939|1|10|23|4|1939|160|2077|Monday|1939Q4|N|N|N|2429538|2429810|2429195|2429468|N|N|N|N|N| +2429561|AAAAAAAAJHCBFCAA|1939-10-24|477|2078|160|1939|2|10|24|4|1939|160|2078|Tuesday|1939Q4|N|N|N|2429538|2429810|2429196|2429469|N|N|N|N|N| +2429562|AAAAAAAAKHCBFCAA|1939-10-25|477|2078|160|1939|3|10|25|4|1939|160|2078|Wednesday|1939Q4|N|N|N|2429538|2429810|2429197|2429470|N|N|N|N|N| +2429563|AAAAAAAALHCBFCAA|1939-10-26|477|2078|160|1939|4|10|26|4|1939|160|2078|Thursday|1939Q4|N|N|N|2429538|2429810|2429198|2429471|N|N|N|N|N| +2429564|AAAAAAAAMHCBFCAA|1939-10-27|477|2078|160|1939|5|10|27|4|1939|160|2078|Friday|1939Q4|N|Y|N|2429538|2429810|2429199|2429472|N|N|N|N|N| +2429565|AAAAAAAANHCBFCAA|1939-10-28|477|2078|160|1939|6|10|28|4|1939|160|2078|Saturday|1939Q4|N|Y|N|2429538|2429810|2429200|2429473|N|N|N|N|N| +2429566|AAAAAAAAOHCBFCAA|1939-10-29|477|2078|160|1939|0|10|29|4|1939|160|2078|Sunday|1939Q4|N|N|N|2429538|2429810|2429201|2429474|N|N|N|N|N| +2429567|AAAAAAAAPHCBFCAA|1939-10-30|477|2078|160|1939|1|10|30|4|1939|160|2078|Monday|1939Q4|N|N|N|2429538|2429810|2429202|2429475|N|N|N|N|N| +2429568|AAAAAAAAAICBFCAA|1939-10-31|477|2079|160|1939|2|10|31|4|1939|160|2079|Tuesday|1939Q4|N|N|N|2429538|2429810|2429203|2429476|N|N|N|N|N| +2429569|AAAAAAAABICBFCAA|1939-11-01|478|2079|160|1939|3|11|1|4|1939|160|2079|Wednesday|1939Q4|N|N|N|2429569|2429872|2429204|2429477|N|N|N|N|N| +2429570|AAAAAAAACICBFCAA|1939-11-02|478|2079|160|1939|4|11|2|4|1939|160|2079|Thursday|1939Q4|N|N|N|2429569|2429872|2429205|2429478|N|N|N|N|N| +2429571|AAAAAAAADICBFCAA|1939-11-03|478|2079|160|1939|5|11|3|4|1939|160|2079|Friday|1939Q4|N|Y|N|2429569|2429872|2429206|2429479|N|N|N|N|N| +2429572|AAAAAAAAEICBFCAA|1939-11-04|478|2079|160|1939|6|11|4|4|1939|160|2079|Saturday|1939Q4|N|Y|N|2429569|2429872|2429207|2429480|N|N|N|N|N| +2429573|AAAAAAAAFICBFCAA|1939-11-05|478|2079|160|1939|0|11|5|4|1939|160|2079|Sunday|1939Q4|N|N|N|2429569|2429872|2429208|2429481|N|N|N|N|N| +2429574|AAAAAAAAGICBFCAA|1939-11-06|478|2079|160|1939|1|11|6|4|1939|160|2079|Monday|1939Q4|N|N|N|2429569|2429872|2429209|2429482|N|N|N|N|N| +2429575|AAAAAAAAHICBFCAA|1939-11-07|478|2080|160|1939|2|11|7|4|1939|160|2080|Tuesday|1939Q4|N|N|N|2429569|2429872|2429210|2429483|N|N|N|N|N| +2429576|AAAAAAAAIICBFCAA|1939-11-08|478|2080|160|1939|3|11|8|4|1939|160|2080|Wednesday|1939Q4|N|N|N|2429569|2429872|2429211|2429484|N|N|N|N|N| +2429577|AAAAAAAAJICBFCAA|1939-11-09|478|2080|160|1939|4|11|9|4|1939|160|2080|Thursday|1939Q4|N|N|N|2429569|2429872|2429212|2429485|N|N|N|N|N| +2429578|AAAAAAAAKICBFCAA|1939-11-10|478|2080|160|1939|5|11|10|4|1939|160|2080|Friday|1939Q4|N|Y|N|2429569|2429872|2429213|2429486|N|N|N|N|N| +2429579|AAAAAAAALICBFCAA|1939-11-11|478|2080|160|1939|6|11|11|4|1939|160|2080|Saturday|1939Q4|N|Y|N|2429569|2429872|2429214|2429487|N|N|N|N|N| +2429580|AAAAAAAAMICBFCAA|1939-11-12|478|2080|160|1939|0|11|12|4|1939|160|2080|Sunday|1939Q4|N|N|N|2429569|2429872|2429215|2429488|N|N|N|N|N| +2429581|AAAAAAAANICBFCAA|1939-11-13|478|2080|160|1939|1|11|13|4|1939|160|2080|Monday|1939Q4|N|N|N|2429569|2429872|2429216|2429489|N|N|N|N|N| +2429582|AAAAAAAAOICBFCAA|1939-11-14|478|2081|160|1939|2|11|14|4|1939|160|2081|Tuesday|1939Q4|N|N|N|2429569|2429872|2429217|2429490|N|N|N|N|N| +2429583|AAAAAAAAPICBFCAA|1939-11-15|478|2081|160|1939|3|11|15|4|1939|160|2081|Wednesday|1939Q4|N|N|N|2429569|2429872|2429218|2429491|N|N|N|N|N| +2429584|AAAAAAAAAJCBFCAA|1939-11-16|478|2081|160|1939|4|11|16|4|1939|160|2081|Thursday|1939Q4|N|N|N|2429569|2429872|2429219|2429492|N|N|N|N|N| +2429585|AAAAAAAABJCBFCAA|1939-11-17|478|2081|160|1939|5|11|17|4|1939|160|2081|Friday|1939Q4|N|Y|N|2429569|2429872|2429220|2429493|N|N|N|N|N| +2429586|AAAAAAAACJCBFCAA|1939-11-18|478|2081|160|1939|6|11|18|4|1939|160|2081|Saturday|1939Q4|N|Y|N|2429569|2429872|2429221|2429494|N|N|N|N|N| +2429587|AAAAAAAADJCBFCAA|1939-11-19|478|2081|160|1939|0|11|19|4|1939|160|2081|Sunday|1939Q4|N|N|N|2429569|2429872|2429222|2429495|N|N|N|N|N| +2429588|AAAAAAAAEJCBFCAA|1939-11-20|478|2081|160|1939|1|11|20|4|1939|160|2081|Monday|1939Q4|N|N|N|2429569|2429872|2429223|2429496|N|N|N|N|N| +2429589|AAAAAAAAFJCBFCAA|1939-11-21|478|2082|160|1939|2|11|21|4|1939|160|2082|Tuesday|1939Q4|N|N|N|2429569|2429872|2429224|2429497|N|N|N|N|N| +2429590|AAAAAAAAGJCBFCAA|1939-11-22|478|2082|160|1939|3|11|22|4|1939|160|2082|Wednesday|1939Q4|N|N|N|2429569|2429872|2429225|2429498|N|N|N|N|N| +2429591|AAAAAAAAHJCBFCAA|1939-11-23|478|2082|160|1939|4|11|23|4|1939|160|2082|Thursday|1939Q4|N|N|N|2429569|2429872|2429226|2429499|N|N|N|N|N| +2429592|AAAAAAAAIJCBFCAA|1939-11-24|478|2082|160|1939|5|11|24|4|1939|160|2082|Friday|1939Q4|N|Y|N|2429569|2429872|2429227|2429500|N|N|N|N|N| +2429593|AAAAAAAAJJCBFCAA|1939-11-25|478|2082|160|1939|6|11|25|4|1939|160|2082|Saturday|1939Q4|N|Y|N|2429569|2429872|2429228|2429501|N|N|N|N|N| +2429594|AAAAAAAAKJCBFCAA|1939-11-26|478|2082|160|1939|0|11|26|4|1939|160|2082|Sunday|1939Q4|N|N|N|2429569|2429872|2429229|2429502|N|N|N|N|N| +2429595|AAAAAAAALJCBFCAA|1939-11-27|478|2082|160|1939|1|11|27|4|1939|160|2082|Monday|1939Q4|N|N|N|2429569|2429872|2429230|2429503|N|N|N|N|N| +2429596|AAAAAAAAMJCBFCAA|1939-11-28|478|2083|160|1939|2|11|28|4|1939|160|2083|Tuesday|1939Q4|N|N|N|2429569|2429872|2429231|2429504|N|N|N|N|N| +2429597|AAAAAAAANJCBFCAA|1939-11-29|478|2083|160|1939|3|11|29|4|1939|160|2083|Wednesday|1939Q4|N|N|N|2429569|2429872|2429232|2429505|N|N|N|N|N| +2429598|AAAAAAAAOJCBFCAA|1939-11-30|478|2083|160|1939|4|11|30|4|1939|160|2083|Thursday|1939Q4|N|N|N|2429569|2429872|2429233|2429506|N|N|N|N|N| +2429599|AAAAAAAAPJCBFCAA|1939-12-01|479|2083|161|1939|5|12|1|4|1939|161|2083|Friday|1939Q4|N|Y|N|2429599|2429932|2429234|2429507|N|N|N|N|N| +2429600|AAAAAAAAAKCBFCAA|1939-12-02|479|2083|161|1939|6|12|2|4|1939|161|2083|Saturday|1939Q4|N|Y|N|2429599|2429932|2429235|2429508|N|N|N|N|N| +2429601|AAAAAAAABKCBFCAA|1939-12-03|479|2083|161|1939|0|12|3|4|1939|161|2083|Sunday|1939Q4|N|N|N|2429599|2429932|2429236|2429509|N|N|N|N|N| +2429602|AAAAAAAACKCBFCAA|1939-12-04|479|2083|161|1939|1|12|4|4|1939|161|2083|Monday|1939Q4|N|N|N|2429599|2429932|2429237|2429510|N|N|N|N|N| +2429603|AAAAAAAADKCBFCAA|1939-12-05|479|2084|161|1939|2|12|5|4|1939|161|2084|Tuesday|1939Q4|N|N|N|2429599|2429932|2429238|2429511|N|N|N|N|N| +2429604|AAAAAAAAEKCBFCAA|1939-12-06|479|2084|161|1939|3|12|6|4|1939|161|2084|Wednesday|1939Q4|N|N|N|2429599|2429932|2429239|2429512|N|N|N|N|N| +2429605|AAAAAAAAFKCBFCAA|1939-12-07|479|2084|161|1939|4|12|7|4|1939|161|2084|Thursday|1939Q4|N|N|N|2429599|2429932|2429240|2429513|N|N|N|N|N| +2429606|AAAAAAAAGKCBFCAA|1939-12-08|479|2084|161|1939|5|12|8|4|1939|161|2084|Friday|1939Q4|N|Y|N|2429599|2429932|2429241|2429514|N|N|N|N|N| +2429607|AAAAAAAAHKCBFCAA|1939-12-09|479|2084|161|1939|6|12|9|4|1939|161|2084|Saturday|1939Q4|N|Y|N|2429599|2429932|2429242|2429515|N|N|N|N|N| +2429608|AAAAAAAAIKCBFCAA|1939-12-10|479|2084|161|1939|0|12|10|4|1939|161|2084|Sunday|1939Q4|N|N|N|2429599|2429932|2429243|2429516|N|N|N|N|N| +2429609|AAAAAAAAJKCBFCAA|1939-12-11|479|2084|161|1939|1|12|11|4|1939|161|2084|Monday|1939Q4|N|N|N|2429599|2429932|2429244|2429517|N|N|N|N|N| +2429610|AAAAAAAAKKCBFCAA|1939-12-12|479|2085|161|1939|2|12|12|4|1939|161|2085|Tuesday|1939Q4|N|N|N|2429599|2429932|2429245|2429518|N|N|N|N|N| +2429611|AAAAAAAALKCBFCAA|1939-12-13|479|2085|161|1939|3|12|13|4|1939|161|2085|Wednesday|1939Q4|N|N|N|2429599|2429932|2429246|2429519|N|N|N|N|N| +2429612|AAAAAAAAMKCBFCAA|1939-12-14|479|2085|161|1939|4|12|14|4|1939|161|2085|Thursday|1939Q4|N|N|N|2429599|2429932|2429247|2429520|N|N|N|N|N| +2429613|AAAAAAAANKCBFCAA|1939-12-15|479|2085|161|1939|5|12|15|4|1939|161|2085|Friday|1939Q4|N|Y|N|2429599|2429932|2429248|2429521|N|N|N|N|N| +2429614|AAAAAAAAOKCBFCAA|1939-12-16|479|2085|161|1939|6|12|16|4|1939|161|2085|Saturday|1939Q4|N|Y|N|2429599|2429932|2429249|2429522|N|N|N|N|N| +2429615|AAAAAAAAPKCBFCAA|1939-12-17|479|2085|161|1939|0|12|17|4|1939|161|2085|Sunday|1939Q4|N|N|N|2429599|2429932|2429250|2429523|N|N|N|N|N| +2429616|AAAAAAAAALCBFCAA|1939-12-18|479|2085|161|1939|1|12|18|4|1939|161|2085|Monday|1939Q4|N|N|N|2429599|2429932|2429251|2429524|N|N|N|N|N| +2429617|AAAAAAAABLCBFCAA|1939-12-19|479|2086|161|1939|2|12|19|4|1939|161|2086|Tuesday|1939Q4|N|N|N|2429599|2429932|2429252|2429525|N|N|N|N|N| +2429618|AAAAAAAACLCBFCAA|1939-12-20|479|2086|161|1939|3|12|20|4|1939|161|2086|Wednesday|1939Q4|N|N|N|2429599|2429932|2429253|2429526|N|N|N|N|N| +2429619|AAAAAAAADLCBFCAA|1939-12-21|479|2086|161|1939|4|12|21|4|1939|161|2086|Thursday|1939Q4|N|N|N|2429599|2429932|2429254|2429527|N|N|N|N|N| +2429620|AAAAAAAAELCBFCAA|1939-12-22|479|2086|161|1939|5|12|22|4|1939|161|2086|Friday|1939Q4|N|Y|N|2429599|2429932|2429255|2429528|N|N|N|N|N| +2429621|AAAAAAAAFLCBFCAA|1939-12-23|479|2086|161|1939|6|12|23|4|1939|161|2086|Saturday|1939Q4|N|Y|N|2429599|2429932|2429256|2429529|N|N|N|N|N| +2429622|AAAAAAAAGLCBFCAA|1939-12-24|479|2086|161|1939|0|12|24|4|1939|161|2086|Sunday|1939Q4|N|N|N|2429599|2429932|2429257|2429530|N|N|N|N|N| +2429623|AAAAAAAAHLCBFCAA|1939-12-25|479|2086|161|1939|1|12|25|4|1939|161|2086|Monday|1939Q4|N|N|N|2429599|2429932|2429258|2429531|N|N|N|N|N| +2429624|AAAAAAAAILCBFCAA|1939-12-26|479|2087|161|1939|2|12|26|4|1939|161|2087|Tuesday|1939Q4|Y|N|N|2429599|2429932|2429259|2429532|N|N|N|N|N| +2429625|AAAAAAAAJLCBFCAA|1939-12-27|479|2087|161|1939|3|12|27|4|1939|161|2087|Wednesday|1939Q4|N|N|Y|2429599|2429932|2429260|2429533|N|N|N|N|N| +2429626|AAAAAAAAKLCBFCAA|1939-12-28|479|2087|161|1939|4|12|28|4|1939|161|2087|Thursday|1939Q4|N|N|N|2429599|2429932|2429261|2429534|N|N|N|N|N| +2429627|AAAAAAAALLCBFCAA|1939-12-29|479|2087|161|1939|5|12|29|4|1939|161|2087|Friday|1939Q4|N|Y|N|2429599|2429932|2429262|2429535|N|N|N|N|N| +2429628|AAAAAAAAMLCBFCAA|1939-12-30|479|2087|161|1939|6|12|30|4|1939|161|2087|Saturday|1939Q4|N|Y|N|2429599|2429932|2429263|2429536|N|N|N|N|N| +2429629|AAAAAAAANLCBFCAA|1939-12-31|479|2087|161|1939|0|12|31|4|1939|161|2087|Sunday|1939Q4|N|N|N|2429599|2429932|2429264|2429537|N|N|N|N|N| +2429630|AAAAAAAAOLCBFCAA|1940-01-01|480|2087|161|1940|1|1|1|1|1940|161|2087|Monday|1940Q1|Y|N|N|2429630|2429629|2429265|2429538|N|N|N|N|N| +2429631|AAAAAAAAPLCBFCAA|1940-01-02|480|2088|161|1940|2|1|2|1|1940|161|2088|Tuesday|1940Q1|N|N|Y|2429630|2429629|2429266|2429539|N|N|N|N|N| +2429632|AAAAAAAAAMCBFCAA|1940-01-03|480|2088|161|1940|3|1|3|1|1940|161|2088|Wednesday|1940Q1|N|N|N|2429630|2429629|2429267|2429540|N|N|N|N|N| +2429633|AAAAAAAABMCBFCAA|1940-01-04|480|2088|161|1940|4|1|4|1|1940|161|2088|Thursday|1940Q1|N|N|N|2429630|2429629|2429268|2429541|N|N|N|N|N| +2429634|AAAAAAAACMCBFCAA|1940-01-05|480|2088|161|1940|5|1|5|1|1940|161|2088|Friday|1940Q1|N|Y|N|2429630|2429629|2429269|2429542|N|N|N|N|N| +2429635|AAAAAAAADMCBFCAA|1940-01-06|480|2088|161|1940|6|1|6|1|1940|161|2088|Saturday|1940Q1|N|Y|N|2429630|2429629|2429270|2429543|N|N|N|N|N| +2429636|AAAAAAAAEMCBFCAA|1940-01-07|480|2088|161|1940|0|1|7|1|1940|161|2088|Sunday|1940Q1|N|N|N|2429630|2429629|2429271|2429544|N|N|N|N|N| +2429637|AAAAAAAAFMCBFCAA|1940-01-08|480|2088|161|1940|1|1|8|1|1940|161|2088|Monday|1940Q1|N|N|N|2429630|2429629|2429272|2429545|N|N|N|N|N| +2429638|AAAAAAAAGMCBFCAA|1940-01-09|480|2089|161|1940|2|1|9|1|1940|161|2089|Tuesday|1940Q1|N|N|N|2429630|2429629|2429273|2429546|N|N|N|N|N| +2429639|AAAAAAAAHMCBFCAA|1940-01-10|480|2089|161|1940|3|1|10|1|1940|161|2089|Wednesday|1940Q1|N|N|N|2429630|2429629|2429274|2429547|N|N|N|N|N| +2429640|AAAAAAAAIMCBFCAA|1940-01-11|480|2089|161|1940|4|1|11|1|1940|161|2089|Thursday|1940Q1|N|N|N|2429630|2429629|2429275|2429548|N|N|N|N|N| +2429641|AAAAAAAAJMCBFCAA|1940-01-12|480|2089|161|1940|5|1|12|1|1940|161|2089|Friday|1940Q1|N|Y|N|2429630|2429629|2429276|2429549|N|N|N|N|N| +2429642|AAAAAAAAKMCBFCAA|1940-01-13|480|2089|161|1940|6|1|13|1|1940|161|2089|Saturday|1940Q1|N|Y|N|2429630|2429629|2429277|2429550|N|N|N|N|N| +2429643|AAAAAAAALMCBFCAA|1940-01-14|480|2089|161|1940|0|1|14|1|1940|161|2089|Sunday|1940Q1|N|N|N|2429630|2429629|2429278|2429551|N|N|N|N|N| +2429644|AAAAAAAAMMCBFCAA|1940-01-15|480|2089|161|1940|1|1|15|1|1940|161|2089|Monday|1940Q1|N|N|N|2429630|2429629|2429279|2429552|N|N|N|N|N| +2429645|AAAAAAAANMCBFCAA|1940-01-16|480|2090|161|1940|2|1|16|1|1940|161|2090|Tuesday|1940Q1|N|N|N|2429630|2429629|2429280|2429553|N|N|N|N|N| +2429646|AAAAAAAAOMCBFCAA|1940-01-17|480|2090|161|1940|3|1|17|1|1940|161|2090|Wednesday|1940Q1|N|N|N|2429630|2429629|2429281|2429554|N|N|N|N|N| +2429647|AAAAAAAAPMCBFCAA|1940-01-18|480|2090|161|1940|4|1|18|1|1940|161|2090|Thursday|1940Q1|N|N|N|2429630|2429629|2429282|2429555|N|N|N|N|N| +2429648|AAAAAAAAANCBFCAA|1940-01-19|480|2090|161|1940|5|1|19|1|1940|161|2090|Friday|1940Q1|N|Y|N|2429630|2429629|2429283|2429556|N|N|N|N|N| +2429649|AAAAAAAABNCBFCAA|1940-01-20|480|2090|161|1940|6|1|20|1|1940|161|2090|Saturday|1940Q1|N|Y|N|2429630|2429629|2429284|2429557|N|N|N|N|N| +2429650|AAAAAAAACNCBFCAA|1940-01-21|480|2090|161|1940|0|1|21|1|1940|161|2090|Sunday|1940Q1|N|N|N|2429630|2429629|2429285|2429558|N|N|N|N|N| +2429651|AAAAAAAADNCBFCAA|1940-01-22|480|2090|161|1940|1|1|22|1|1940|161|2090|Monday|1940Q1|N|N|N|2429630|2429629|2429286|2429559|N|N|N|N|N| +2429652|AAAAAAAAENCBFCAA|1940-01-23|480|2091|161|1940|2|1|23|1|1940|161|2091|Tuesday|1940Q1|N|N|N|2429630|2429629|2429287|2429560|N|N|N|N|N| +2429653|AAAAAAAAFNCBFCAA|1940-01-24|480|2091|161|1940|3|1|24|1|1940|161|2091|Wednesday|1940Q1|N|N|N|2429630|2429629|2429288|2429561|N|N|N|N|N| +2429654|AAAAAAAAGNCBFCAA|1940-01-25|480|2091|161|1940|4|1|25|1|1940|161|2091|Thursday|1940Q1|N|N|N|2429630|2429629|2429289|2429562|N|N|N|N|N| +2429655|AAAAAAAAHNCBFCAA|1940-01-26|480|2091|161|1940|5|1|26|1|1940|161|2091|Friday|1940Q1|N|Y|N|2429630|2429629|2429290|2429563|N|N|N|N|N| +2429656|AAAAAAAAINCBFCAA|1940-01-27|480|2091|161|1940|6|1|27|1|1940|161|2091|Saturday|1940Q1|N|Y|N|2429630|2429629|2429291|2429564|N|N|N|N|N| +2429657|AAAAAAAAJNCBFCAA|1940-01-28|480|2091|161|1940|0|1|28|1|1940|161|2091|Sunday|1940Q1|N|N|N|2429630|2429629|2429292|2429565|N|N|N|N|N| +2429658|AAAAAAAAKNCBFCAA|1940-01-29|480|2091|161|1940|1|1|29|1|1940|161|2091|Monday|1940Q1|N|N|N|2429630|2429629|2429293|2429566|N|N|N|N|N| +2429659|AAAAAAAALNCBFCAA|1940-01-30|480|2092|161|1940|2|1|30|1|1940|161|2092|Tuesday|1940Q1|N|N|N|2429630|2429629|2429294|2429567|N|N|N|N|N| +2429660|AAAAAAAAMNCBFCAA|1940-01-31|480|2092|161|1940|3|1|31|1|1940|161|2092|Wednesday|1940Q1|N|N|N|2429630|2429629|2429295|2429568|N|N|N|N|N| +2429661|AAAAAAAANNCBFCAA|1940-02-01|481|2092|161|1940|4|2|1|1|1940|161|2092|Thursday|1940Q1|N|N|N|2429661|2429691|2429296|2429569|N|N|N|N|N| +2429662|AAAAAAAAONCBFCAA|1940-02-02|481|2092|161|1940|5|2|2|1|1940|161|2092|Friday|1940Q1|N|Y|N|2429661|2429691|2429297|2429570|N|N|N|N|N| +2429663|AAAAAAAAPNCBFCAA|1940-02-03|481|2092|161|1940|6|2|3|1|1940|161|2092|Saturday|1940Q1|N|Y|N|2429661|2429691|2429298|2429571|N|N|N|N|N| +2429664|AAAAAAAAAOCBFCAA|1940-02-04|481|2092|161|1940|0|2|4|1|1940|161|2092|Sunday|1940Q1|N|N|N|2429661|2429691|2429299|2429572|N|N|N|N|N| +2429665|AAAAAAAABOCBFCAA|1940-02-05|481|2092|161|1940|1|2|5|1|1940|161|2092|Monday|1940Q1|N|N|N|2429661|2429691|2429300|2429573|N|N|N|N|N| +2429666|AAAAAAAACOCBFCAA|1940-02-06|481|2093|161|1940|2|2|6|1|1940|161|2093|Tuesday|1940Q1|N|N|N|2429661|2429691|2429301|2429574|N|N|N|N|N| +2429667|AAAAAAAADOCBFCAA|1940-02-07|481|2093|161|1940|3|2|7|1|1940|161|2093|Wednesday|1940Q1|N|N|N|2429661|2429691|2429302|2429575|N|N|N|N|N| +2429668|AAAAAAAAEOCBFCAA|1940-02-08|481|2093|161|1940|4|2|8|1|1940|161|2093|Thursday|1940Q1|N|N|N|2429661|2429691|2429303|2429576|N|N|N|N|N| +2429669|AAAAAAAAFOCBFCAA|1940-02-09|481|2093|161|1940|5|2|9|1|1940|161|2093|Friday|1940Q1|N|Y|N|2429661|2429691|2429304|2429577|N|N|N|N|N| +2429670|AAAAAAAAGOCBFCAA|1940-02-10|481|2093|161|1940|6|2|10|1|1940|161|2093|Saturday|1940Q1|N|Y|N|2429661|2429691|2429305|2429578|N|N|N|N|N| +2429671|AAAAAAAAHOCBFCAA|1940-02-11|481|2093|161|1940|0|2|11|1|1940|161|2093|Sunday|1940Q1|N|N|N|2429661|2429691|2429306|2429579|N|N|N|N|N| +2429672|AAAAAAAAIOCBFCAA|1940-02-12|481|2093|161|1940|1|2|12|1|1940|161|2093|Monday|1940Q1|N|N|N|2429661|2429691|2429307|2429580|N|N|N|N|N| +2429673|AAAAAAAAJOCBFCAA|1940-02-13|481|2094|161|1940|2|2|13|1|1940|161|2094|Tuesday|1940Q1|N|N|N|2429661|2429691|2429308|2429581|N|N|N|N|N| +2429674|AAAAAAAAKOCBFCAA|1940-02-14|481|2094|161|1940|3|2|14|1|1940|161|2094|Wednesday|1940Q1|N|N|N|2429661|2429691|2429309|2429582|N|N|N|N|N| +2429675|AAAAAAAALOCBFCAA|1940-02-15|481|2094|161|1940|4|2|15|1|1940|161|2094|Thursday|1940Q1|N|N|N|2429661|2429691|2429310|2429583|N|N|N|N|N| +2429676|AAAAAAAAMOCBFCAA|1940-02-16|481|2094|161|1940|5|2|16|1|1940|161|2094|Friday|1940Q1|N|Y|N|2429661|2429691|2429311|2429584|N|N|N|N|N| +2429677|AAAAAAAANOCBFCAA|1940-02-17|481|2094|161|1940|6|2|17|1|1940|161|2094|Saturday|1940Q1|N|Y|N|2429661|2429691|2429312|2429585|N|N|N|N|N| +2429678|AAAAAAAAOOCBFCAA|1940-02-18|481|2094|161|1940|0|2|18|1|1940|161|2094|Sunday|1940Q1|N|N|N|2429661|2429691|2429313|2429586|N|N|N|N|N| +2429679|AAAAAAAAPOCBFCAA|1940-02-19|481|2094|161|1940|1|2|19|1|1940|161|2094|Monday|1940Q1|N|N|N|2429661|2429691|2429314|2429587|N|N|N|N|N| +2429680|AAAAAAAAAPCBFCAA|1940-02-20|481|2095|161|1940|2|2|20|1|1940|161|2095|Tuesday|1940Q1|N|N|N|2429661|2429691|2429315|2429588|N|N|N|N|N| +2429681|AAAAAAAABPCBFCAA|1940-02-21|481|2095|161|1940|3|2|21|1|1940|161|2095|Wednesday|1940Q1|N|N|N|2429661|2429691|2429316|2429589|N|N|N|N|N| +2429682|AAAAAAAACPCBFCAA|1940-02-22|481|2095|161|1940|4|2|22|1|1940|161|2095|Thursday|1940Q1|N|N|N|2429661|2429691|2429317|2429590|N|N|N|N|N| +2429683|AAAAAAAADPCBFCAA|1940-02-23|481|2095|161|1940|5|2|23|1|1940|161|2095|Friday|1940Q1|N|Y|N|2429661|2429691|2429318|2429591|N|N|N|N|N| +2429684|AAAAAAAAEPCBFCAA|1940-02-24|481|2095|161|1940|6|2|24|1|1940|161|2095|Saturday|1940Q1|N|Y|N|2429661|2429691|2429319|2429592|N|N|N|N|N| +2429685|AAAAAAAAFPCBFCAA|1940-02-25|481|2095|161|1940|0|2|25|1|1940|161|2095|Sunday|1940Q1|N|N|N|2429661|2429691|2429320|2429593|N|N|N|N|N| +2429686|AAAAAAAAGPCBFCAA|1940-02-26|481|2095|161|1940|1|2|26|1|1940|161|2095|Monday|1940Q1|N|N|N|2429661|2429691|2429321|2429594|N|N|N|N|N| +2429687|AAAAAAAAHPCBFCAA|1940-02-27|481|2096|161|1940|2|2|27|1|1940|161|2096|Tuesday|1940Q1|N|N|N|2429661|2429691|2429322|2429595|N|N|N|N|N| +2429688|AAAAAAAAIPCBFCAA|1940-02-28|481|2096|161|1940|3|2|28|1|1940|161|2096|Wednesday|1940Q1|N|N|N|2429661|2429691|2429323|2429596|N|N|N|N|N| +2429689|AAAAAAAAJPCBFCAA|1940-02-29|481|2096|161|1940|4|2|29|1|1940|161|2096|Thursday|1940Q1|N|N|N|2429661|2429691|2429323|2429597|N|N|N|N|N| +2429690|AAAAAAAAKPCBFCAA|1940-03-01|482|2096|162|1940|5|3|1|1|1940|162|2096|Friday|1940Q1|N|Y|N|2429690|2429749|2429324|2429598|N|N|N|N|N| +2429691|AAAAAAAALPCBFCAA|1940-03-02|482|2096|162|1940|6|3|2|1|1940|162|2096|Saturday|1940Q1|N|Y|N|2429690|2429749|2429325|2429599|N|N|N|N|N| +2429692|AAAAAAAAMPCBFCAA|1940-03-03|482|2096|162|1940|0|3|3|1|1940|162|2096|Sunday|1940Q1|N|N|N|2429690|2429749|2429326|2429600|N|N|N|N|N| +2429693|AAAAAAAANPCBFCAA|1940-03-04|482|2096|162|1940|1|3|4|1|1940|162|2096|Monday|1940Q1|N|N|N|2429690|2429749|2429327|2429601|N|N|N|N|N| +2429694|AAAAAAAAOPCBFCAA|1940-03-05|482|2097|162|1940|2|3|5|1|1940|162|2097|Tuesday|1940Q1|N|N|N|2429690|2429749|2429328|2429602|N|N|N|N|N| +2429695|AAAAAAAAPPCBFCAA|1940-03-06|482|2097|162|1940|3|3|6|1|1940|162|2097|Wednesday|1940Q1|N|N|N|2429690|2429749|2429329|2429603|N|N|N|N|N| +2429696|AAAAAAAAAADBFCAA|1940-03-07|482|2097|162|1940|4|3|7|1|1940|162|2097|Thursday|1940Q1|N|N|N|2429690|2429749|2429330|2429604|N|N|N|N|N| +2429697|AAAAAAAABADBFCAA|1940-03-08|482|2097|162|1940|5|3|8|1|1940|162|2097|Friday|1940Q1|N|Y|N|2429690|2429749|2429331|2429605|N|N|N|N|N| +2429698|AAAAAAAACADBFCAA|1940-03-09|482|2097|162|1940|6|3|9|1|1940|162|2097|Saturday|1940Q1|N|Y|N|2429690|2429749|2429332|2429606|N|N|N|N|N| +2429699|AAAAAAAADADBFCAA|1940-03-10|482|2097|162|1940|0|3|10|1|1940|162|2097|Sunday|1940Q1|N|N|N|2429690|2429749|2429333|2429607|N|N|N|N|N| +2429700|AAAAAAAAEADBFCAA|1940-03-11|482|2097|162|1940|1|3|11|1|1940|162|2097|Monday|1940Q1|N|N|N|2429690|2429749|2429334|2429608|N|N|N|N|N| +2429701|AAAAAAAAFADBFCAA|1940-03-12|482|2098|162|1940|2|3|12|1|1940|162|2098|Tuesday|1940Q1|N|N|N|2429690|2429749|2429335|2429609|N|N|N|N|N| +2429702|AAAAAAAAGADBFCAA|1940-03-13|482|2098|162|1940|3|3|13|1|1940|162|2098|Wednesday|1940Q1|N|N|N|2429690|2429749|2429336|2429610|N|N|N|N|N| +2429703|AAAAAAAAHADBFCAA|1940-03-14|482|2098|162|1940|4|3|14|1|1940|162|2098|Thursday|1940Q1|N|N|N|2429690|2429749|2429337|2429611|N|N|N|N|N| +2429704|AAAAAAAAIADBFCAA|1940-03-15|482|2098|162|1940|5|3|15|1|1940|162|2098|Friday|1940Q1|N|Y|N|2429690|2429749|2429338|2429612|N|N|N|N|N| +2429705|AAAAAAAAJADBFCAA|1940-03-16|482|2098|162|1940|6|3|16|1|1940|162|2098|Saturday|1940Q1|N|Y|N|2429690|2429749|2429339|2429613|N|N|N|N|N| +2429706|AAAAAAAAKADBFCAA|1940-03-17|482|2098|162|1940|0|3|17|1|1940|162|2098|Sunday|1940Q1|N|N|N|2429690|2429749|2429340|2429614|N|N|N|N|N| +2429707|AAAAAAAALADBFCAA|1940-03-18|482|2098|162|1940|1|3|18|1|1940|162|2098|Monday|1940Q1|N|N|N|2429690|2429749|2429341|2429615|N|N|N|N|N| +2429708|AAAAAAAAMADBFCAA|1940-03-19|482|2099|162|1940|2|3|19|1|1940|162|2099|Tuesday|1940Q1|N|N|N|2429690|2429749|2429342|2429616|N|N|N|N|N| +2429709|AAAAAAAANADBFCAA|1940-03-20|482|2099|162|1940|3|3|20|1|1940|162|2099|Wednesday|1940Q1|N|N|N|2429690|2429749|2429343|2429617|N|N|N|N|N| +2429710|AAAAAAAAOADBFCAA|1940-03-21|482|2099|162|1940|4|3|21|1|1940|162|2099|Thursday|1940Q1|N|N|N|2429690|2429749|2429344|2429618|N|N|N|N|N| +2429711|AAAAAAAAPADBFCAA|1940-03-22|482|2099|162|1940|5|3|22|1|1940|162|2099|Friday|1940Q1|N|Y|N|2429690|2429749|2429345|2429619|N|N|N|N|N| +2429712|AAAAAAAAABDBFCAA|1940-03-23|482|2099|162|1940|6|3|23|1|1940|162|2099|Saturday|1940Q1|N|Y|N|2429690|2429749|2429346|2429620|N|N|N|N|N| +2429713|AAAAAAAABBDBFCAA|1940-03-24|482|2099|162|1940|0|3|24|1|1940|162|2099|Sunday|1940Q1|N|N|N|2429690|2429749|2429347|2429621|N|N|N|N|N| +2429714|AAAAAAAACBDBFCAA|1940-03-25|482|2099|162|1940|1|3|25|1|1940|162|2099|Monday|1940Q1|N|N|N|2429690|2429749|2429348|2429622|N|N|N|N|N| +2429715|AAAAAAAADBDBFCAA|1940-03-26|482|2100|162|1940|2|3|26|1|1940|162|2100|Tuesday|1940Q1|N|N|N|2429690|2429749|2429349|2429623|N|N|N|N|N| +2429716|AAAAAAAAEBDBFCAA|1940-03-27|482|2100|162|1940|3|3|27|1|1940|162|2100|Wednesday|1940Q1|N|N|N|2429690|2429749|2429350|2429624|N|N|N|N|N| +2429717|AAAAAAAAFBDBFCAA|1940-03-28|482|2100|162|1940|4|3|28|1|1940|162|2100|Thursday|1940Q1|N|N|N|2429690|2429749|2429351|2429625|N|N|N|N|N| +2429718|AAAAAAAAGBDBFCAA|1940-03-29|482|2100|162|1940|5|3|29|1|1940|162|2100|Friday|1940Q1|N|Y|N|2429690|2429749|2429352|2429626|N|N|N|N|N| +2429719|AAAAAAAAHBDBFCAA|1940-03-30|482|2100|162|1940|6|3|30|1|1940|162|2100|Saturday|1940Q1|N|Y|N|2429690|2429749|2429353|2429627|N|N|N|N|N| +2429720|AAAAAAAAIBDBFCAA|1940-03-31|482|2100|162|1940|0|3|31|1|1940|162|2100|Sunday|1940Q1|N|N|N|2429690|2429749|2429354|2429628|N|N|N|N|N| +2429721|AAAAAAAAJBDBFCAA|1940-04-01|483|2100|162|1940|1|4|1|2|1940|162|2100|Monday|1940Q2|N|N|N|2429721|2429811|2429355|2429630|N|N|N|N|N| +2429722|AAAAAAAAKBDBFCAA|1940-04-02|483|2101|162|1940|2|4|2|2|1940|162|2101|Tuesday|1940Q2|N|N|N|2429721|2429811|2429356|2429631|N|N|N|N|N| +2429723|AAAAAAAALBDBFCAA|1940-04-03|483|2101|162|1940|3|4|3|2|1940|162|2101|Wednesday|1940Q2|N|N|N|2429721|2429811|2429357|2429632|N|N|N|N|N| +2429724|AAAAAAAAMBDBFCAA|1940-04-04|483|2101|162|1940|4|4|4|2|1940|162|2101|Thursday|1940Q2|N|N|N|2429721|2429811|2429358|2429633|N|N|N|N|N| +2429725|AAAAAAAANBDBFCAA|1940-04-05|483|2101|162|1940|5|4|5|2|1940|162|2101|Friday|1940Q2|N|Y|N|2429721|2429811|2429359|2429634|N|N|N|N|N| +2429726|AAAAAAAAOBDBFCAA|1940-04-06|483|2101|162|1940|6|4|6|2|1940|162|2101|Saturday|1940Q2|N|Y|N|2429721|2429811|2429360|2429635|N|N|N|N|N| +2429727|AAAAAAAAPBDBFCAA|1940-04-07|483|2101|162|1940|0|4|7|2|1940|162|2101|Sunday|1940Q2|N|N|N|2429721|2429811|2429361|2429636|N|N|N|N|N| +2429728|AAAAAAAAACDBFCAA|1940-04-08|483|2101|162|1940|1|4|8|2|1940|162|2101|Monday|1940Q2|N|N|N|2429721|2429811|2429362|2429637|N|N|N|N|N| +2429729|AAAAAAAABCDBFCAA|1940-04-09|483|2102|162|1940|2|4|9|2|1940|162|2102|Tuesday|1940Q2|N|N|N|2429721|2429811|2429363|2429638|N|N|N|N|N| +2429730|AAAAAAAACCDBFCAA|1940-04-10|483|2102|162|1940|3|4|10|2|1940|162|2102|Wednesday|1940Q2|N|N|N|2429721|2429811|2429364|2429639|N|N|N|N|N| +2429731|AAAAAAAADCDBFCAA|1940-04-11|483|2102|162|1940|4|4|11|2|1940|162|2102|Thursday|1940Q2|N|N|N|2429721|2429811|2429365|2429640|N|N|N|N|N| +2429732|AAAAAAAAECDBFCAA|1940-04-12|483|2102|162|1940|5|4|12|2|1940|162|2102|Friday|1940Q2|N|Y|N|2429721|2429811|2429366|2429641|N|N|N|N|N| +2429733|AAAAAAAAFCDBFCAA|1940-04-13|483|2102|162|1940|6|4|13|2|1940|162|2102|Saturday|1940Q2|N|Y|N|2429721|2429811|2429367|2429642|N|N|N|N|N| +2429734|AAAAAAAAGCDBFCAA|1940-04-14|483|2102|162|1940|0|4|14|2|1940|162|2102|Sunday|1940Q2|N|N|N|2429721|2429811|2429368|2429643|N|N|N|N|N| +2429735|AAAAAAAAHCDBFCAA|1940-04-15|483|2102|162|1940|1|4|15|2|1940|162|2102|Monday|1940Q2|N|N|N|2429721|2429811|2429369|2429644|N|N|N|N|N| +2429736|AAAAAAAAICDBFCAA|1940-04-16|483|2103|162|1940|2|4|16|2|1940|162|2103|Tuesday|1940Q2|N|N|N|2429721|2429811|2429370|2429645|N|N|N|N|N| +2429737|AAAAAAAAJCDBFCAA|1940-04-17|483|2103|162|1940|3|4|17|2|1940|162|2103|Wednesday|1940Q2|N|N|N|2429721|2429811|2429371|2429646|N|N|N|N|N| +2429738|AAAAAAAAKCDBFCAA|1940-04-18|483|2103|162|1940|4|4|18|2|1940|162|2103|Thursday|1940Q2|N|N|N|2429721|2429811|2429372|2429647|N|N|N|N|N| +2429739|AAAAAAAALCDBFCAA|1940-04-19|483|2103|162|1940|5|4|19|2|1940|162|2103|Friday|1940Q2|N|Y|N|2429721|2429811|2429373|2429648|N|N|N|N|N| +2429740|AAAAAAAAMCDBFCAA|1940-04-20|483|2103|162|1940|6|4|20|2|1940|162|2103|Saturday|1940Q2|N|Y|N|2429721|2429811|2429374|2429649|N|N|N|N|N| +2429741|AAAAAAAANCDBFCAA|1940-04-21|483|2103|162|1940|0|4|21|2|1940|162|2103|Sunday|1940Q2|N|N|N|2429721|2429811|2429375|2429650|N|N|N|N|N| +2429742|AAAAAAAAOCDBFCAA|1940-04-22|483|2103|162|1940|1|4|22|2|1940|162|2103|Monday|1940Q2|N|N|N|2429721|2429811|2429376|2429651|N|N|N|N|N| +2429743|AAAAAAAAPCDBFCAA|1940-04-23|483|2104|162|1940|2|4|23|2|1940|162|2104|Tuesday|1940Q2|N|N|N|2429721|2429811|2429377|2429652|N|N|N|N|N| +2429744|AAAAAAAAADDBFCAA|1940-04-24|483|2104|162|1940|3|4|24|2|1940|162|2104|Wednesday|1940Q2|N|N|N|2429721|2429811|2429378|2429653|N|N|N|N|N| +2429745|AAAAAAAABDDBFCAA|1940-04-25|483|2104|162|1940|4|4|25|2|1940|162|2104|Thursday|1940Q2|N|N|N|2429721|2429811|2429379|2429654|N|N|N|N|N| +2429746|AAAAAAAACDDBFCAA|1940-04-26|483|2104|162|1940|5|4|26|2|1940|162|2104|Friday|1940Q2|N|Y|N|2429721|2429811|2429380|2429655|N|N|N|N|N| +2429747|AAAAAAAADDDBFCAA|1940-04-27|483|2104|162|1940|6|4|27|2|1940|162|2104|Saturday|1940Q2|N|Y|N|2429721|2429811|2429381|2429656|N|N|N|N|N| +2429748|AAAAAAAAEDDBFCAA|1940-04-28|483|2104|162|1940|0|4|28|2|1940|162|2104|Sunday|1940Q2|N|N|N|2429721|2429811|2429382|2429657|N|N|N|N|N| +2429749|AAAAAAAAFDDBFCAA|1940-04-29|483|2104|162|1940|1|4|29|2|1940|162|2104|Monday|1940Q2|N|N|N|2429721|2429811|2429383|2429658|N|N|N|N|N| +2429750|AAAAAAAAGDDBFCAA|1940-04-30|483|2105|162|1940|2|4|30|2|1940|162|2105|Tuesday|1940Q2|N|N|N|2429721|2429811|2429384|2429659|N|N|N|N|N| +2429751|AAAAAAAAHDDBFCAA|1940-05-01|484|2105|162|1940|3|5|1|2|1940|162|2105|Wednesday|1940Q2|N|N|N|2429751|2429871|2429385|2429660|N|N|N|N|N| +2429752|AAAAAAAAIDDBFCAA|1940-05-02|484|2105|162|1940|4|5|2|2|1940|162|2105|Thursday|1940Q2|N|N|N|2429751|2429871|2429386|2429661|N|N|N|N|N| +2429753|AAAAAAAAJDDBFCAA|1940-05-03|484|2105|162|1940|5|5|3|2|1940|162|2105|Friday|1940Q2|N|Y|N|2429751|2429871|2429387|2429662|N|N|N|N|N| +2429754|AAAAAAAAKDDBFCAA|1940-05-04|484|2105|162|1940|6|5|4|2|1940|162|2105|Saturday|1940Q2|N|Y|N|2429751|2429871|2429388|2429663|N|N|N|N|N| +2429755|AAAAAAAALDDBFCAA|1940-05-05|484|2105|162|1940|0|5|5|2|1940|162|2105|Sunday|1940Q2|N|N|N|2429751|2429871|2429389|2429664|N|N|N|N|N| +2429756|AAAAAAAAMDDBFCAA|1940-05-06|484|2105|162|1940|1|5|6|2|1940|162|2105|Monday|1940Q2|N|N|N|2429751|2429871|2429390|2429665|N|N|N|N|N| +2429757|AAAAAAAANDDBFCAA|1940-05-07|484|2106|162|1940|2|5|7|2|1940|162|2106|Tuesday|1940Q2|N|N|N|2429751|2429871|2429391|2429666|N|N|N|N|N| +2429758|AAAAAAAAODDBFCAA|1940-05-08|484|2106|162|1940|3|5|8|2|1940|162|2106|Wednesday|1940Q2|N|N|N|2429751|2429871|2429392|2429667|N|N|N|N|N| +2429759|AAAAAAAAPDDBFCAA|1940-05-09|484|2106|162|1940|4|5|9|2|1940|162|2106|Thursday|1940Q2|N|N|N|2429751|2429871|2429393|2429668|N|N|N|N|N| +2429760|AAAAAAAAAEDBFCAA|1940-05-10|484|2106|162|1940|5|5|10|2|1940|162|2106|Friday|1940Q2|N|Y|N|2429751|2429871|2429394|2429669|N|N|N|N|N| +2429761|AAAAAAAABEDBFCAA|1940-05-11|484|2106|162|1940|6|5|11|2|1940|162|2106|Saturday|1940Q2|N|Y|N|2429751|2429871|2429395|2429670|N|N|N|N|N| +2429762|AAAAAAAACEDBFCAA|1940-05-12|484|2106|162|1940|0|5|12|2|1940|162|2106|Sunday|1940Q2|N|N|N|2429751|2429871|2429396|2429671|N|N|N|N|N| +2429763|AAAAAAAADEDBFCAA|1940-05-13|484|2106|162|1940|1|5|13|2|1940|162|2106|Monday|1940Q2|N|N|N|2429751|2429871|2429397|2429672|N|N|N|N|N| +2429764|AAAAAAAAEEDBFCAA|1940-05-14|484|2107|162|1940|2|5|14|2|1940|162|2107|Tuesday|1940Q2|N|N|N|2429751|2429871|2429398|2429673|N|N|N|N|N| +2429765|AAAAAAAAFEDBFCAA|1940-05-15|484|2107|162|1940|3|5|15|2|1940|162|2107|Wednesday|1940Q2|N|N|N|2429751|2429871|2429399|2429674|N|N|N|N|N| +2429766|AAAAAAAAGEDBFCAA|1940-05-16|484|2107|162|1940|4|5|16|2|1940|162|2107|Thursday|1940Q2|N|N|N|2429751|2429871|2429400|2429675|N|N|N|N|N| +2429767|AAAAAAAAHEDBFCAA|1940-05-17|484|2107|162|1940|5|5|17|2|1940|162|2107|Friday|1940Q2|N|Y|N|2429751|2429871|2429401|2429676|N|N|N|N|N| +2429768|AAAAAAAAIEDBFCAA|1940-05-18|484|2107|162|1940|6|5|18|2|1940|162|2107|Saturday|1940Q2|N|Y|N|2429751|2429871|2429402|2429677|N|N|N|N|N| +2429769|AAAAAAAAJEDBFCAA|1940-05-19|484|2107|162|1940|0|5|19|2|1940|162|2107|Sunday|1940Q2|N|N|N|2429751|2429871|2429403|2429678|N|N|N|N|N| +2429770|AAAAAAAAKEDBFCAA|1940-05-20|484|2107|162|1940|1|5|20|2|1940|162|2107|Monday|1940Q2|N|N|N|2429751|2429871|2429404|2429679|N|N|N|N|N| +2429771|AAAAAAAALEDBFCAA|1940-05-21|484|2108|162|1940|2|5|21|2|1940|162|2108|Tuesday|1940Q2|N|N|N|2429751|2429871|2429405|2429680|N|N|N|N|N| +2429772|AAAAAAAAMEDBFCAA|1940-05-22|484|2108|162|1940|3|5|22|2|1940|162|2108|Wednesday|1940Q2|N|N|N|2429751|2429871|2429406|2429681|N|N|N|N|N| +2429773|AAAAAAAANEDBFCAA|1940-05-23|484|2108|162|1940|4|5|23|2|1940|162|2108|Thursday|1940Q2|N|N|N|2429751|2429871|2429407|2429682|N|N|N|N|N| +2429774|AAAAAAAAOEDBFCAA|1940-05-24|484|2108|162|1940|5|5|24|2|1940|162|2108|Friday|1940Q2|N|Y|N|2429751|2429871|2429408|2429683|N|N|N|N|N| +2429775|AAAAAAAAPEDBFCAA|1940-05-25|484|2108|162|1940|6|5|25|2|1940|162|2108|Saturday|1940Q2|N|Y|N|2429751|2429871|2429409|2429684|N|N|N|N|N| +2429776|AAAAAAAAAFDBFCAA|1940-05-26|484|2108|162|1940|0|5|26|2|1940|162|2108|Sunday|1940Q2|N|N|N|2429751|2429871|2429410|2429685|N|N|N|N|N| +2429777|AAAAAAAABFDBFCAA|1940-05-27|484|2108|162|1940|1|5|27|2|1940|162|2108|Monday|1940Q2|N|N|N|2429751|2429871|2429411|2429686|N|N|N|N|N| +2429778|AAAAAAAACFDBFCAA|1940-05-28|484|2109|162|1940|2|5|28|2|1940|162|2109|Tuesday|1940Q2|N|N|N|2429751|2429871|2429412|2429687|N|N|N|N|N| +2429779|AAAAAAAADFDBFCAA|1940-05-29|484|2109|162|1940|3|5|29|2|1940|162|2109|Wednesday|1940Q2|N|N|N|2429751|2429871|2429413|2429688|N|N|N|N|N| +2429780|AAAAAAAAEFDBFCAA|1940-05-30|484|2109|162|1940|4|5|30|2|1940|162|2109|Thursday|1940Q2|N|N|N|2429751|2429871|2429414|2429689|N|N|N|N|N| +2429781|AAAAAAAAFFDBFCAA|1940-05-31|484|2109|162|1940|5|5|31|2|1940|162|2109|Friday|1940Q2|N|Y|N|2429751|2429871|2429415|2429690|N|N|N|N|N| +2429782|AAAAAAAAGFDBFCAA|1940-06-01|485|2109|163|1940|6|6|1|2|1940|163|2109|Saturday|1940Q2|N|Y|N|2429782|2429933|2429416|2429691|N|N|N|N|N| +2429783|AAAAAAAAHFDBFCAA|1940-06-02|485|2109|163|1940|0|6|2|2|1940|163|2109|Sunday|1940Q2|N|N|N|2429782|2429933|2429417|2429692|N|N|N|N|N| +2429784|AAAAAAAAIFDBFCAA|1940-06-03|485|2109|163|1940|1|6|3|2|1940|163|2109|Monday|1940Q2|N|N|N|2429782|2429933|2429418|2429693|N|N|N|N|N| +2429785|AAAAAAAAJFDBFCAA|1940-06-04|485|2110|163|1940|2|6|4|2|1940|163|2110|Tuesday|1940Q2|N|N|N|2429782|2429933|2429419|2429694|N|N|N|N|N| +2429786|AAAAAAAAKFDBFCAA|1940-06-05|485|2110|163|1940|3|6|5|2|1940|163|2110|Wednesday|1940Q2|N|N|N|2429782|2429933|2429420|2429695|N|N|N|N|N| +2429787|AAAAAAAALFDBFCAA|1940-06-06|485|2110|163|1940|4|6|6|2|1940|163|2110|Thursday|1940Q2|N|N|N|2429782|2429933|2429421|2429696|N|N|N|N|N| +2429788|AAAAAAAAMFDBFCAA|1940-06-07|485|2110|163|1940|5|6|7|2|1940|163|2110|Friday|1940Q2|N|Y|N|2429782|2429933|2429422|2429697|N|N|N|N|N| +2429789|AAAAAAAANFDBFCAA|1940-06-08|485|2110|163|1940|6|6|8|2|1940|163|2110|Saturday|1940Q2|N|Y|N|2429782|2429933|2429423|2429698|N|N|N|N|N| +2429790|AAAAAAAAOFDBFCAA|1940-06-09|485|2110|163|1940|0|6|9|2|1940|163|2110|Sunday|1940Q2|N|N|N|2429782|2429933|2429424|2429699|N|N|N|N|N| +2429791|AAAAAAAAPFDBFCAA|1940-06-10|485|2110|163|1940|1|6|10|2|1940|163|2110|Monday|1940Q2|N|N|N|2429782|2429933|2429425|2429700|N|N|N|N|N| +2429792|AAAAAAAAAGDBFCAA|1940-06-11|485|2111|163|1940|2|6|11|2|1940|163|2111|Tuesday|1940Q2|N|N|N|2429782|2429933|2429426|2429701|N|N|N|N|N| +2429793|AAAAAAAABGDBFCAA|1940-06-12|485|2111|163|1940|3|6|12|2|1940|163|2111|Wednesday|1940Q2|N|N|N|2429782|2429933|2429427|2429702|N|N|N|N|N| +2429794|AAAAAAAACGDBFCAA|1940-06-13|485|2111|163|1940|4|6|13|2|1940|163|2111|Thursday|1940Q2|N|N|N|2429782|2429933|2429428|2429703|N|N|N|N|N| +2429795|AAAAAAAADGDBFCAA|1940-06-14|485|2111|163|1940|5|6|14|2|1940|163|2111|Friday|1940Q2|N|Y|N|2429782|2429933|2429429|2429704|N|N|N|N|N| +2429796|AAAAAAAAEGDBFCAA|1940-06-15|485|2111|163|1940|6|6|15|2|1940|163|2111|Saturday|1940Q2|N|Y|N|2429782|2429933|2429430|2429705|N|N|N|N|N| +2429797|AAAAAAAAFGDBFCAA|1940-06-16|485|2111|163|1940|0|6|16|2|1940|163|2111|Sunday|1940Q2|N|N|N|2429782|2429933|2429431|2429706|N|N|N|N|N| +2429798|AAAAAAAAGGDBFCAA|1940-06-17|485|2111|163|1940|1|6|17|2|1940|163|2111|Monday|1940Q2|N|N|N|2429782|2429933|2429432|2429707|N|N|N|N|N| +2429799|AAAAAAAAHGDBFCAA|1940-06-18|485|2112|163|1940|2|6|18|2|1940|163|2112|Tuesday|1940Q2|N|N|N|2429782|2429933|2429433|2429708|N|N|N|N|N| +2429800|AAAAAAAAIGDBFCAA|1940-06-19|485|2112|163|1940|3|6|19|2|1940|163|2112|Wednesday|1940Q2|N|N|N|2429782|2429933|2429434|2429709|N|N|N|N|N| +2429801|AAAAAAAAJGDBFCAA|1940-06-20|485|2112|163|1940|4|6|20|2|1940|163|2112|Thursday|1940Q2|N|N|N|2429782|2429933|2429435|2429710|N|N|N|N|N| +2429802|AAAAAAAAKGDBFCAA|1940-06-21|485|2112|163|1940|5|6|21|2|1940|163|2112|Friday|1940Q2|N|Y|N|2429782|2429933|2429436|2429711|N|N|N|N|N| +2429803|AAAAAAAALGDBFCAA|1940-06-22|485|2112|163|1940|6|6|22|2|1940|163|2112|Saturday|1940Q2|N|Y|N|2429782|2429933|2429437|2429712|N|N|N|N|N| +2429804|AAAAAAAAMGDBFCAA|1940-06-23|485|2112|163|1940|0|6|23|2|1940|163|2112|Sunday|1940Q2|N|N|N|2429782|2429933|2429438|2429713|N|N|N|N|N| +2429805|AAAAAAAANGDBFCAA|1940-06-24|485|2112|163|1940|1|6|24|2|1940|163|2112|Monday|1940Q2|N|N|N|2429782|2429933|2429439|2429714|N|N|N|N|N| +2429806|AAAAAAAAOGDBFCAA|1940-06-25|485|2113|163|1940|2|6|25|2|1940|163|2113|Tuesday|1940Q2|N|N|N|2429782|2429933|2429440|2429715|N|N|N|N|N| +2429807|AAAAAAAAPGDBFCAA|1940-06-26|485|2113|163|1940|3|6|26|2|1940|163|2113|Wednesday|1940Q2|N|N|N|2429782|2429933|2429441|2429716|N|N|N|N|N| +2429808|AAAAAAAAAHDBFCAA|1940-06-27|485|2113|163|1940|4|6|27|2|1940|163|2113|Thursday|1940Q2|N|N|N|2429782|2429933|2429442|2429717|N|N|N|N|N| +2429809|AAAAAAAABHDBFCAA|1940-06-28|485|2113|163|1940|5|6|28|2|1940|163|2113|Friday|1940Q2|N|Y|N|2429782|2429933|2429443|2429718|N|N|N|N|N| +2429810|AAAAAAAACHDBFCAA|1940-06-29|485|2113|163|1940|6|6|29|2|1940|163|2113|Saturday|1940Q2|N|Y|N|2429782|2429933|2429444|2429719|N|N|N|N|N| +2429811|AAAAAAAADHDBFCAA|1940-06-30|485|2113|163|1940|0|6|30|2|1940|163|2113|Sunday|1940Q2|N|N|N|2429782|2429933|2429445|2429720|N|N|N|N|N| +2429812|AAAAAAAAEHDBFCAA|1940-07-01|486|2113|163|1940|1|7|1|3|1940|163|2113|Monday|1940Q3|N|N|N|2429812|2429993|2429446|2429721|N|N|N|N|N| +2429813|AAAAAAAAFHDBFCAA|1940-07-02|486|2114|163|1940|2|7|2|3|1940|163|2114|Tuesday|1940Q3|N|N|N|2429812|2429993|2429447|2429722|N|N|N|N|N| +2429814|AAAAAAAAGHDBFCAA|1940-07-03|486|2114|163|1940|3|7|3|3|1940|163|2114|Wednesday|1940Q3|N|N|N|2429812|2429993|2429448|2429723|N|N|N|N|N| +2429815|AAAAAAAAHHDBFCAA|1940-07-04|486|2114|163|1940|4|7|4|3|1940|163|2114|Thursday|1940Q3|Y|N|N|2429812|2429993|2429449|2429724|N|N|N|N|N| +2429816|AAAAAAAAIHDBFCAA|1940-07-05|486|2114|163|1940|5|7|5|3|1940|163|2114|Friday|1940Q3|N|Y|Y|2429812|2429993|2429450|2429725|N|N|N|N|N| +2429817|AAAAAAAAJHDBFCAA|1940-07-06|486|2114|163|1940|6|7|6|3|1940|163|2114|Saturday|1940Q3|N|Y|N|2429812|2429993|2429451|2429726|N|N|N|N|N| +2429818|AAAAAAAAKHDBFCAA|1940-07-07|486|2114|163|1940|0|7|7|3|1940|163|2114|Sunday|1940Q3|N|N|N|2429812|2429993|2429452|2429727|N|N|N|N|N| +2429819|AAAAAAAALHDBFCAA|1940-07-08|486|2114|163|1940|1|7|8|3|1940|163|2114|Monday|1940Q3|N|N|N|2429812|2429993|2429453|2429728|N|N|N|N|N| +2429820|AAAAAAAAMHDBFCAA|1940-07-09|486|2115|163|1940|2|7|9|3|1940|163|2115|Tuesday|1940Q3|N|N|N|2429812|2429993|2429454|2429729|N|N|N|N|N| +2429821|AAAAAAAANHDBFCAA|1940-07-10|486|2115|163|1940|3|7|10|3|1940|163|2115|Wednesday|1940Q3|N|N|N|2429812|2429993|2429455|2429730|N|N|N|N|N| +2429822|AAAAAAAAOHDBFCAA|1940-07-11|486|2115|163|1940|4|7|11|3|1940|163|2115|Thursday|1940Q3|N|N|N|2429812|2429993|2429456|2429731|N|N|N|N|N| +2429823|AAAAAAAAPHDBFCAA|1940-07-12|486|2115|163|1940|5|7|12|3|1940|163|2115|Friday|1940Q3|N|Y|N|2429812|2429993|2429457|2429732|N|N|N|N|N| +2429824|AAAAAAAAAIDBFCAA|1940-07-13|486|2115|163|1940|6|7|13|3|1940|163|2115|Saturday|1940Q3|N|Y|N|2429812|2429993|2429458|2429733|N|N|N|N|N| +2429825|AAAAAAAABIDBFCAA|1940-07-14|486|2115|163|1940|0|7|14|3|1940|163|2115|Sunday|1940Q3|N|N|N|2429812|2429993|2429459|2429734|N|N|N|N|N| +2429826|AAAAAAAACIDBFCAA|1940-07-15|486|2115|163|1940|1|7|15|3|1940|163|2115|Monday|1940Q3|N|N|N|2429812|2429993|2429460|2429735|N|N|N|N|N| +2429827|AAAAAAAADIDBFCAA|1940-07-16|486|2116|163|1940|2|7|16|3|1940|163|2116|Tuesday|1940Q3|N|N|N|2429812|2429993|2429461|2429736|N|N|N|N|N| +2429828|AAAAAAAAEIDBFCAA|1940-07-17|486|2116|163|1940|3|7|17|3|1940|163|2116|Wednesday|1940Q3|N|N|N|2429812|2429993|2429462|2429737|N|N|N|N|N| +2429829|AAAAAAAAFIDBFCAA|1940-07-18|486|2116|163|1940|4|7|18|3|1940|163|2116|Thursday|1940Q3|N|N|N|2429812|2429993|2429463|2429738|N|N|N|N|N| +2429830|AAAAAAAAGIDBFCAA|1940-07-19|486|2116|163|1940|5|7|19|3|1940|163|2116|Friday|1940Q3|N|Y|N|2429812|2429993|2429464|2429739|N|N|N|N|N| +2429831|AAAAAAAAHIDBFCAA|1940-07-20|486|2116|163|1940|6|7|20|3|1940|163|2116|Saturday|1940Q3|N|Y|N|2429812|2429993|2429465|2429740|N|N|N|N|N| +2429832|AAAAAAAAIIDBFCAA|1940-07-21|486|2116|163|1940|0|7|21|3|1940|163|2116|Sunday|1940Q3|N|N|N|2429812|2429993|2429466|2429741|N|N|N|N|N| +2429833|AAAAAAAAJIDBFCAA|1940-07-22|486|2116|163|1940|1|7|22|3|1940|163|2116|Monday|1940Q3|N|N|N|2429812|2429993|2429467|2429742|N|N|N|N|N| +2429834|AAAAAAAAKIDBFCAA|1940-07-23|486|2117|163|1940|2|7|23|3|1940|163|2117|Tuesday|1940Q3|N|N|N|2429812|2429993|2429468|2429743|N|N|N|N|N| +2429835|AAAAAAAALIDBFCAA|1940-07-24|486|2117|163|1940|3|7|24|3|1940|163|2117|Wednesday|1940Q3|N|N|N|2429812|2429993|2429469|2429744|N|N|N|N|N| +2429836|AAAAAAAAMIDBFCAA|1940-07-25|486|2117|163|1940|4|7|25|3|1940|163|2117|Thursday|1940Q3|N|N|N|2429812|2429993|2429470|2429745|N|N|N|N|N| +2429837|AAAAAAAANIDBFCAA|1940-07-26|486|2117|163|1940|5|7|26|3|1940|163|2117|Friday|1940Q3|N|Y|N|2429812|2429993|2429471|2429746|N|N|N|N|N| +2429838|AAAAAAAAOIDBFCAA|1940-07-27|486|2117|163|1940|6|7|27|3|1940|163|2117|Saturday|1940Q3|N|Y|N|2429812|2429993|2429472|2429747|N|N|N|N|N| +2429839|AAAAAAAAPIDBFCAA|1940-07-28|486|2117|163|1940|0|7|28|3|1940|163|2117|Sunday|1940Q3|N|N|N|2429812|2429993|2429473|2429748|N|N|N|N|N| +2429840|AAAAAAAAAJDBFCAA|1940-07-29|486|2117|163|1940|1|7|29|3|1940|163|2117|Monday|1940Q3|N|N|N|2429812|2429993|2429474|2429749|N|N|N|N|N| +2429841|AAAAAAAABJDBFCAA|1940-07-30|486|2118|163|1940|2|7|30|3|1940|163|2118|Tuesday|1940Q3|N|N|N|2429812|2429993|2429475|2429750|N|N|N|N|N| +2429842|AAAAAAAACJDBFCAA|1940-07-31|486|2118|163|1940|3|7|31|3|1940|163|2118|Wednesday|1940Q3|N|N|N|2429812|2429993|2429476|2429751|N|N|N|N|N| +2429843|AAAAAAAADJDBFCAA|1940-08-01|487|2118|163|1940|4|8|1|3|1940|163|2118|Thursday|1940Q3|N|N|N|2429843|2430055|2429477|2429752|N|N|N|N|N| +2429844|AAAAAAAAEJDBFCAA|1940-08-02|487|2118|163|1940|5|8|2|3|1940|163|2118|Friday|1940Q3|N|Y|N|2429843|2430055|2429478|2429753|N|N|N|N|N| +2429845|AAAAAAAAFJDBFCAA|1940-08-03|487|2118|163|1940|6|8|3|3|1940|163|2118|Saturday|1940Q3|N|Y|N|2429843|2430055|2429479|2429754|N|N|N|N|N| +2429846|AAAAAAAAGJDBFCAA|1940-08-04|487|2118|163|1940|0|8|4|3|1940|163|2118|Sunday|1940Q3|N|N|N|2429843|2430055|2429480|2429755|N|N|N|N|N| +2429847|AAAAAAAAHJDBFCAA|1940-08-05|487|2118|163|1940|1|8|5|3|1940|163|2118|Monday|1940Q3|N|N|N|2429843|2430055|2429481|2429756|N|N|N|N|N| +2429848|AAAAAAAAIJDBFCAA|1940-08-06|487|2119|163|1940|2|8|6|3|1940|163|2119|Tuesday|1940Q3|N|N|N|2429843|2430055|2429482|2429757|N|N|N|N|N| +2429849|AAAAAAAAJJDBFCAA|1940-08-07|487|2119|163|1940|3|8|7|3|1940|163|2119|Wednesday|1940Q3|N|N|N|2429843|2430055|2429483|2429758|N|N|N|N|N| +2429850|AAAAAAAAKJDBFCAA|1940-08-08|487|2119|163|1940|4|8|8|3|1940|163|2119|Thursday|1940Q3|N|N|N|2429843|2430055|2429484|2429759|N|N|N|N|N| +2429851|AAAAAAAALJDBFCAA|1940-08-09|487|2119|163|1940|5|8|9|3|1940|163|2119|Friday|1940Q3|N|Y|N|2429843|2430055|2429485|2429760|N|N|N|N|N| +2429852|AAAAAAAAMJDBFCAA|1940-08-10|487|2119|163|1940|6|8|10|3|1940|163|2119|Saturday|1940Q3|N|Y|N|2429843|2430055|2429486|2429761|N|N|N|N|N| +2429853|AAAAAAAANJDBFCAA|1940-08-11|487|2119|163|1940|0|8|11|3|1940|163|2119|Sunday|1940Q3|N|N|N|2429843|2430055|2429487|2429762|N|N|N|N|N| +2429854|AAAAAAAAOJDBFCAA|1940-08-12|487|2119|163|1940|1|8|12|3|1940|163|2119|Monday|1940Q3|N|N|N|2429843|2430055|2429488|2429763|N|N|N|N|N| +2429855|AAAAAAAAPJDBFCAA|1940-08-13|487|2120|163|1940|2|8|13|3|1940|163|2120|Tuesday|1940Q3|N|N|N|2429843|2430055|2429489|2429764|N|N|N|N|N| +2429856|AAAAAAAAAKDBFCAA|1940-08-14|487|2120|163|1940|3|8|14|3|1940|163|2120|Wednesday|1940Q3|N|N|N|2429843|2430055|2429490|2429765|N|N|N|N|N| +2429857|AAAAAAAABKDBFCAA|1940-08-15|487|2120|163|1940|4|8|15|3|1940|163|2120|Thursday|1940Q3|N|N|N|2429843|2430055|2429491|2429766|N|N|N|N|N| +2429858|AAAAAAAACKDBFCAA|1940-08-16|487|2120|163|1940|5|8|16|3|1940|163|2120|Friday|1940Q3|N|Y|N|2429843|2430055|2429492|2429767|N|N|N|N|N| +2429859|AAAAAAAADKDBFCAA|1940-08-17|487|2120|163|1940|6|8|17|3|1940|163|2120|Saturday|1940Q3|N|Y|N|2429843|2430055|2429493|2429768|N|N|N|N|N| +2429860|AAAAAAAAEKDBFCAA|1940-08-18|487|2120|163|1940|0|8|18|3|1940|163|2120|Sunday|1940Q3|N|N|N|2429843|2430055|2429494|2429769|N|N|N|N|N| +2429861|AAAAAAAAFKDBFCAA|1940-08-19|487|2120|163|1940|1|8|19|3|1940|163|2120|Monday|1940Q3|N|N|N|2429843|2430055|2429495|2429770|N|N|N|N|N| +2429862|AAAAAAAAGKDBFCAA|1940-08-20|487|2121|163|1940|2|8|20|3|1940|163|2121|Tuesday|1940Q3|N|N|N|2429843|2430055|2429496|2429771|N|N|N|N|N| +2429863|AAAAAAAAHKDBFCAA|1940-08-21|487|2121|163|1940|3|8|21|3|1940|163|2121|Wednesday|1940Q3|N|N|N|2429843|2430055|2429497|2429772|N|N|N|N|N| +2429864|AAAAAAAAIKDBFCAA|1940-08-22|487|2121|163|1940|4|8|22|3|1940|163|2121|Thursday|1940Q3|N|N|N|2429843|2430055|2429498|2429773|N|N|N|N|N| +2429865|AAAAAAAAJKDBFCAA|1940-08-23|487|2121|163|1940|5|8|23|3|1940|163|2121|Friday|1940Q3|N|Y|N|2429843|2430055|2429499|2429774|N|N|N|N|N| +2429866|AAAAAAAAKKDBFCAA|1940-08-24|487|2121|163|1940|6|8|24|3|1940|163|2121|Saturday|1940Q3|N|Y|N|2429843|2430055|2429500|2429775|N|N|N|N|N| +2429867|AAAAAAAALKDBFCAA|1940-08-25|487|2121|163|1940|0|8|25|3|1940|163|2121|Sunday|1940Q3|N|N|N|2429843|2430055|2429501|2429776|N|N|N|N|N| +2429868|AAAAAAAAMKDBFCAA|1940-08-26|487|2121|163|1940|1|8|26|3|1940|163|2121|Monday|1940Q3|N|N|N|2429843|2430055|2429502|2429777|N|N|N|N|N| +2429869|AAAAAAAANKDBFCAA|1940-08-27|487|2122|163|1940|2|8|27|3|1940|163|2122|Tuesday|1940Q3|N|N|N|2429843|2430055|2429503|2429778|N|N|N|N|N| +2429870|AAAAAAAAOKDBFCAA|1940-08-28|487|2122|163|1940|3|8|28|3|1940|163|2122|Wednesday|1940Q3|N|N|N|2429843|2430055|2429504|2429779|N|N|N|N|N| +2429871|AAAAAAAAPKDBFCAA|1940-08-29|487|2122|163|1940|4|8|29|3|1940|163|2122|Thursday|1940Q3|N|N|N|2429843|2430055|2429505|2429780|N|N|N|N|N| +2429872|AAAAAAAAALDBFCAA|1940-08-30|487|2122|163|1940|5|8|30|3|1940|163|2122|Friday|1940Q3|N|Y|N|2429843|2430055|2429506|2429781|N|N|N|N|N| +2429873|AAAAAAAABLDBFCAA|1940-08-31|487|2122|163|1940|6|8|31|3|1940|163|2122|Saturday|1940Q3|N|Y|N|2429843|2430055|2429507|2429782|N|N|N|N|N| +2429874|AAAAAAAACLDBFCAA|1940-09-01|488|2122|164|1940|0|9|1|3|1940|164|2122|Sunday|1940Q3|N|N|N|2429874|2430117|2429508|2429783|N|N|N|N|N| +2429875|AAAAAAAADLDBFCAA|1940-09-02|488|2122|164|1940|1|9|2|3|1940|164|2122|Monday|1940Q3|N|N|N|2429874|2430117|2429509|2429784|N|N|N|N|N| +2429876|AAAAAAAAELDBFCAA|1940-09-03|488|2123|164|1940|2|9|3|3|1940|164|2123|Tuesday|1940Q3|N|N|N|2429874|2430117|2429510|2429785|N|N|N|N|N| +2429877|AAAAAAAAFLDBFCAA|1940-09-04|488|2123|164|1940|3|9|4|3|1940|164|2123|Wednesday|1940Q3|N|N|N|2429874|2430117|2429511|2429786|N|N|N|N|N| +2429878|AAAAAAAAGLDBFCAA|1940-09-05|488|2123|164|1940|4|9|5|3|1940|164|2123|Thursday|1940Q3|N|N|N|2429874|2430117|2429512|2429787|N|N|N|N|N| +2429879|AAAAAAAAHLDBFCAA|1940-09-06|488|2123|164|1940|5|9|6|3|1940|164|2123|Friday|1940Q3|N|Y|N|2429874|2430117|2429513|2429788|N|N|N|N|N| +2429880|AAAAAAAAILDBFCAA|1940-09-07|488|2123|164|1940|6|9|7|3|1940|164|2123|Saturday|1940Q3|N|Y|N|2429874|2430117|2429514|2429789|N|N|N|N|N| +2429881|AAAAAAAAJLDBFCAA|1940-09-08|488|2123|164|1940|0|9|8|3|1940|164|2123|Sunday|1940Q3|N|N|N|2429874|2430117|2429515|2429790|N|N|N|N|N| +2429882|AAAAAAAAKLDBFCAA|1940-09-09|488|2123|164|1940|1|9|9|3|1940|164|2123|Monday|1940Q3|N|N|N|2429874|2430117|2429516|2429791|N|N|N|N|N| +2429883|AAAAAAAALLDBFCAA|1940-09-10|488|2124|164|1940|2|9|10|3|1940|164|2124|Tuesday|1940Q3|N|N|N|2429874|2430117|2429517|2429792|N|N|N|N|N| +2429884|AAAAAAAAMLDBFCAA|1940-09-11|488|2124|164|1940|3|9|11|3|1940|164|2124|Wednesday|1940Q3|N|N|N|2429874|2430117|2429518|2429793|N|N|N|N|N| +2429885|AAAAAAAANLDBFCAA|1940-09-12|488|2124|164|1940|4|9|12|3|1940|164|2124|Thursday|1940Q3|N|N|N|2429874|2430117|2429519|2429794|N|N|N|N|N| +2429886|AAAAAAAAOLDBFCAA|1940-09-13|488|2124|164|1940|5|9|13|3|1940|164|2124|Friday|1940Q3|N|Y|N|2429874|2430117|2429520|2429795|N|N|N|N|N| +2429887|AAAAAAAAPLDBFCAA|1940-09-14|488|2124|164|1940|6|9|14|3|1940|164|2124|Saturday|1940Q3|N|Y|N|2429874|2430117|2429521|2429796|N|N|N|N|N| +2429888|AAAAAAAAAMDBFCAA|1940-09-15|488|2124|164|1940|0|9|15|3|1940|164|2124|Sunday|1940Q3|N|N|N|2429874|2430117|2429522|2429797|N|N|N|N|N| +2429889|AAAAAAAABMDBFCAA|1940-09-16|488|2124|164|1940|1|9|16|3|1940|164|2124|Monday|1940Q3|N|N|N|2429874|2430117|2429523|2429798|N|N|N|N|N| +2429890|AAAAAAAACMDBFCAA|1940-09-17|488|2125|164|1940|2|9|17|3|1940|164|2125|Tuesday|1940Q3|N|N|N|2429874|2430117|2429524|2429799|N|N|N|N|N| +2429891|AAAAAAAADMDBFCAA|1940-09-18|488|2125|164|1940|3|9|18|3|1940|164|2125|Wednesday|1940Q3|N|N|N|2429874|2430117|2429525|2429800|N|N|N|N|N| +2429892|AAAAAAAAEMDBFCAA|1940-09-19|488|2125|164|1940|4|9|19|3|1940|164|2125|Thursday|1940Q3|N|N|N|2429874|2430117|2429526|2429801|N|N|N|N|N| +2429893|AAAAAAAAFMDBFCAA|1940-09-20|488|2125|164|1940|5|9|20|3|1940|164|2125|Friday|1940Q3|N|Y|N|2429874|2430117|2429527|2429802|N|N|N|N|N| +2429894|AAAAAAAAGMDBFCAA|1940-09-21|488|2125|164|1940|6|9|21|3|1940|164|2125|Saturday|1940Q3|N|Y|N|2429874|2430117|2429528|2429803|N|N|N|N|N| +2429895|AAAAAAAAHMDBFCAA|1940-09-22|488|2125|164|1940|0|9|22|3|1940|164|2125|Sunday|1940Q3|N|N|N|2429874|2430117|2429529|2429804|N|N|N|N|N| +2429896|AAAAAAAAIMDBFCAA|1940-09-23|488|2125|164|1940|1|9|23|3|1940|164|2125|Monday|1940Q3|N|N|N|2429874|2430117|2429530|2429805|N|N|N|N|N| +2429897|AAAAAAAAJMDBFCAA|1940-09-24|488|2126|164|1940|2|9|24|3|1940|164|2126|Tuesday|1940Q3|N|N|N|2429874|2430117|2429531|2429806|N|N|N|N|N| +2429898|AAAAAAAAKMDBFCAA|1940-09-25|488|2126|164|1940|3|9|25|3|1940|164|2126|Wednesday|1940Q3|N|N|N|2429874|2430117|2429532|2429807|N|N|N|N|N| +2429899|AAAAAAAALMDBFCAA|1940-09-26|488|2126|164|1940|4|9|26|3|1940|164|2126|Thursday|1940Q3|N|N|N|2429874|2430117|2429533|2429808|N|N|N|N|N| +2429900|AAAAAAAAMMDBFCAA|1940-09-27|488|2126|164|1940|5|9|27|3|1940|164|2126|Friday|1940Q3|N|Y|N|2429874|2430117|2429534|2429809|N|N|N|N|N| +2429901|AAAAAAAANMDBFCAA|1940-09-28|488|2126|164|1940|6|9|28|3|1940|164|2126|Saturday|1940Q3|N|Y|N|2429874|2430117|2429535|2429810|N|N|N|N|N| +2429902|AAAAAAAAOMDBFCAA|1940-09-29|488|2126|164|1940|0|9|29|3|1940|164|2126|Sunday|1940Q3|N|N|N|2429874|2430117|2429536|2429811|N|N|N|N|N| +2429903|AAAAAAAAPMDBFCAA|1940-09-30|488|2126|164|1940|1|9|30|3|1940|164|2126|Monday|1940Q3|N|N|N|2429874|2430117|2429537|2429812|N|N|N|N|N| +2429904|AAAAAAAAANDBFCAA|1940-10-01|489|2127|164|1940|2|10|1|4|1940|164|2127|Tuesday|1940Q4|N|N|N|2429904|2430177|2429538|2429812|N|N|N|N|N| +2429905|AAAAAAAABNDBFCAA|1940-10-02|489|2127|164|1940|3|10|2|4|1940|164|2127|Wednesday|1940Q4|N|N|N|2429904|2430177|2429539|2429813|N|N|N|N|N| +2429906|AAAAAAAACNDBFCAA|1940-10-03|489|2127|164|1940|4|10|3|4|1940|164|2127|Thursday|1940Q4|N|N|N|2429904|2430177|2429540|2429814|N|N|N|N|N| +2429907|AAAAAAAADNDBFCAA|1940-10-04|489|2127|164|1940|5|10|4|4|1940|164|2127|Friday|1940Q4|N|Y|N|2429904|2430177|2429541|2429815|N|N|N|N|N| +2429908|AAAAAAAAENDBFCAA|1940-10-05|489|2127|164|1940|6|10|5|4|1940|164|2127|Saturday|1940Q4|N|Y|N|2429904|2430177|2429542|2429816|N|N|N|N|N| +2429909|AAAAAAAAFNDBFCAA|1940-10-06|489|2127|164|1940|0|10|6|4|1940|164|2127|Sunday|1940Q4|N|N|N|2429904|2430177|2429543|2429817|N|N|N|N|N| +2429910|AAAAAAAAGNDBFCAA|1940-10-07|489|2127|164|1940|1|10|7|4|1940|164|2127|Monday|1940Q4|N|N|N|2429904|2430177|2429544|2429818|N|N|N|N|N| +2429911|AAAAAAAAHNDBFCAA|1940-10-08|489|2128|164|1940|2|10|8|4|1940|164|2128|Tuesday|1940Q4|N|N|N|2429904|2430177|2429545|2429819|N|N|N|N|N| +2429912|AAAAAAAAINDBFCAA|1940-10-09|489|2128|164|1940|3|10|9|4|1940|164|2128|Wednesday|1940Q4|N|N|N|2429904|2430177|2429546|2429820|N|N|N|N|N| +2429913|AAAAAAAAJNDBFCAA|1940-10-10|489|2128|164|1940|4|10|10|4|1940|164|2128|Thursday|1940Q4|N|N|N|2429904|2430177|2429547|2429821|N|N|N|N|N| +2429914|AAAAAAAAKNDBFCAA|1940-10-11|489|2128|164|1940|5|10|11|4|1940|164|2128|Friday|1940Q4|N|Y|N|2429904|2430177|2429548|2429822|N|N|N|N|N| +2429915|AAAAAAAALNDBFCAA|1940-10-12|489|2128|164|1940|6|10|12|4|1940|164|2128|Saturday|1940Q4|N|Y|N|2429904|2430177|2429549|2429823|N|N|N|N|N| +2429916|AAAAAAAAMNDBFCAA|1940-10-13|489|2128|164|1940|0|10|13|4|1940|164|2128|Sunday|1940Q4|N|N|N|2429904|2430177|2429550|2429824|N|N|N|N|N| +2429917|AAAAAAAANNDBFCAA|1940-10-14|489|2128|164|1940|1|10|14|4|1940|164|2128|Monday|1940Q4|N|N|N|2429904|2430177|2429551|2429825|N|N|N|N|N| +2429918|AAAAAAAAONDBFCAA|1940-10-15|489|2129|164|1940|2|10|15|4|1940|164|2129|Tuesday|1940Q4|N|N|N|2429904|2430177|2429552|2429826|N|N|N|N|N| +2429919|AAAAAAAAPNDBFCAA|1940-10-16|489|2129|164|1940|3|10|16|4|1940|164|2129|Wednesday|1940Q4|N|N|N|2429904|2430177|2429553|2429827|N|N|N|N|N| +2429920|AAAAAAAAAODBFCAA|1940-10-17|489|2129|164|1940|4|10|17|4|1940|164|2129|Thursday|1940Q4|N|N|N|2429904|2430177|2429554|2429828|N|N|N|N|N| +2429921|AAAAAAAABODBFCAA|1940-10-18|489|2129|164|1940|5|10|18|4|1940|164|2129|Friday|1940Q4|N|Y|N|2429904|2430177|2429555|2429829|N|N|N|N|N| +2429922|AAAAAAAACODBFCAA|1940-10-19|489|2129|164|1940|6|10|19|4|1940|164|2129|Saturday|1940Q4|N|Y|N|2429904|2430177|2429556|2429830|N|N|N|N|N| +2429923|AAAAAAAADODBFCAA|1940-10-20|489|2129|164|1940|0|10|20|4|1940|164|2129|Sunday|1940Q4|N|N|N|2429904|2430177|2429557|2429831|N|N|N|N|N| +2429924|AAAAAAAAEODBFCAA|1940-10-21|489|2129|164|1940|1|10|21|4|1940|164|2129|Monday|1940Q4|N|N|N|2429904|2430177|2429558|2429832|N|N|N|N|N| +2429925|AAAAAAAAFODBFCAA|1940-10-22|489|2130|164|1940|2|10|22|4|1940|164|2130|Tuesday|1940Q4|N|N|N|2429904|2430177|2429559|2429833|N|N|N|N|N| +2429926|AAAAAAAAGODBFCAA|1940-10-23|489|2130|164|1940|3|10|23|4|1940|164|2130|Wednesday|1940Q4|N|N|N|2429904|2430177|2429560|2429834|N|N|N|N|N| +2429927|AAAAAAAAHODBFCAA|1940-10-24|489|2130|164|1940|4|10|24|4|1940|164|2130|Thursday|1940Q4|N|N|N|2429904|2430177|2429561|2429835|N|N|N|N|N| +2429928|AAAAAAAAIODBFCAA|1940-10-25|489|2130|164|1940|5|10|25|4|1940|164|2130|Friday|1940Q4|N|Y|N|2429904|2430177|2429562|2429836|N|N|N|N|N| +2429929|AAAAAAAAJODBFCAA|1940-10-26|489|2130|164|1940|6|10|26|4|1940|164|2130|Saturday|1940Q4|N|Y|N|2429904|2430177|2429563|2429837|N|N|N|N|N| +2429930|AAAAAAAAKODBFCAA|1940-10-27|489|2130|164|1940|0|10|27|4|1940|164|2130|Sunday|1940Q4|N|N|N|2429904|2430177|2429564|2429838|N|N|N|N|N| +2429931|AAAAAAAALODBFCAA|1940-10-28|489|2130|164|1940|1|10|28|4|1940|164|2130|Monday|1940Q4|N|N|N|2429904|2430177|2429565|2429839|N|N|N|N|N| +2429932|AAAAAAAAMODBFCAA|1940-10-29|489|2131|164|1940|2|10|29|4|1940|164|2131|Tuesday|1940Q4|N|N|N|2429904|2430177|2429566|2429840|N|N|N|N|N| +2429933|AAAAAAAANODBFCAA|1940-10-30|489|2131|164|1940|3|10|30|4|1940|164|2131|Wednesday|1940Q4|N|N|N|2429904|2430177|2429567|2429841|N|N|N|N|N| +2429934|AAAAAAAAOODBFCAA|1940-10-31|489|2131|164|1940|4|10|31|4|1940|164|2131|Thursday|1940Q4|N|N|N|2429904|2430177|2429568|2429842|N|N|N|N|N| +2429935|AAAAAAAAPODBFCAA|1940-11-01|490|2131|164|1940|5|11|1|4|1940|164|2131|Friday|1940Q4|N|Y|N|2429935|2430239|2429569|2429843|N|N|N|N|N| +2429936|AAAAAAAAAPDBFCAA|1940-11-02|490|2131|164|1940|6|11|2|4|1940|164|2131|Saturday|1940Q4|N|Y|N|2429935|2430239|2429570|2429844|N|N|N|N|N| +2429937|AAAAAAAABPDBFCAA|1940-11-03|490|2131|164|1940|0|11|3|4|1940|164|2131|Sunday|1940Q4|N|N|N|2429935|2430239|2429571|2429845|N|N|N|N|N| +2429938|AAAAAAAACPDBFCAA|1940-11-04|490|2131|164|1940|1|11|4|4|1940|164|2131|Monday|1940Q4|N|N|N|2429935|2430239|2429572|2429846|N|N|N|N|N| +2429939|AAAAAAAADPDBFCAA|1940-11-05|490|2132|164|1940|2|11|5|4|1940|164|2132|Tuesday|1940Q4|N|N|N|2429935|2430239|2429573|2429847|N|N|N|N|N| +2429940|AAAAAAAAEPDBFCAA|1940-11-06|490|2132|164|1940|3|11|6|4|1940|164|2132|Wednesday|1940Q4|N|N|N|2429935|2430239|2429574|2429848|N|N|N|N|N| +2429941|AAAAAAAAFPDBFCAA|1940-11-07|490|2132|164|1940|4|11|7|4|1940|164|2132|Thursday|1940Q4|N|N|N|2429935|2430239|2429575|2429849|N|N|N|N|N| +2429942|AAAAAAAAGPDBFCAA|1940-11-08|490|2132|164|1940|5|11|8|4|1940|164|2132|Friday|1940Q4|N|Y|N|2429935|2430239|2429576|2429850|N|N|N|N|N| +2429943|AAAAAAAAHPDBFCAA|1940-11-09|490|2132|164|1940|6|11|9|4|1940|164|2132|Saturday|1940Q4|N|Y|N|2429935|2430239|2429577|2429851|N|N|N|N|N| +2429944|AAAAAAAAIPDBFCAA|1940-11-10|490|2132|164|1940|0|11|10|4|1940|164|2132|Sunday|1940Q4|N|N|N|2429935|2430239|2429578|2429852|N|N|N|N|N| +2429945|AAAAAAAAJPDBFCAA|1940-11-11|490|2132|164|1940|1|11|11|4|1940|164|2132|Monday|1940Q4|N|N|N|2429935|2430239|2429579|2429853|N|N|N|N|N| +2429946|AAAAAAAAKPDBFCAA|1940-11-12|490|2133|164|1940|2|11|12|4|1940|164|2133|Tuesday|1940Q4|N|N|N|2429935|2430239|2429580|2429854|N|N|N|N|N| +2429947|AAAAAAAALPDBFCAA|1940-11-13|490|2133|164|1940|3|11|13|4|1940|164|2133|Wednesday|1940Q4|N|N|N|2429935|2430239|2429581|2429855|N|N|N|N|N| +2429948|AAAAAAAAMPDBFCAA|1940-11-14|490|2133|164|1940|4|11|14|4|1940|164|2133|Thursday|1940Q4|N|N|N|2429935|2430239|2429582|2429856|N|N|N|N|N| +2429949|AAAAAAAANPDBFCAA|1940-11-15|490|2133|164|1940|5|11|15|4|1940|164|2133|Friday|1940Q4|N|Y|N|2429935|2430239|2429583|2429857|N|N|N|N|N| +2429950|AAAAAAAAOPDBFCAA|1940-11-16|490|2133|164|1940|6|11|16|4|1940|164|2133|Saturday|1940Q4|N|Y|N|2429935|2430239|2429584|2429858|N|N|N|N|N| +2429951|AAAAAAAAPPDBFCAA|1940-11-17|490|2133|164|1940|0|11|17|4|1940|164|2133|Sunday|1940Q4|N|N|N|2429935|2430239|2429585|2429859|N|N|N|N|N| +2429952|AAAAAAAAAAEBFCAA|1940-11-18|490|2133|164|1940|1|11|18|4|1940|164|2133|Monday|1940Q4|N|N|N|2429935|2430239|2429586|2429860|N|N|N|N|N| +2429953|AAAAAAAABAEBFCAA|1940-11-19|490|2134|164|1940|2|11|19|4|1940|164|2134|Tuesday|1940Q4|N|N|N|2429935|2430239|2429587|2429861|N|N|N|N|N| +2429954|AAAAAAAACAEBFCAA|1940-11-20|490|2134|164|1940|3|11|20|4|1940|164|2134|Wednesday|1940Q4|N|N|N|2429935|2430239|2429588|2429862|N|N|N|N|N| +2429955|AAAAAAAADAEBFCAA|1940-11-21|490|2134|164|1940|4|11|21|4|1940|164|2134|Thursday|1940Q4|N|N|N|2429935|2430239|2429589|2429863|N|N|N|N|N| +2429956|AAAAAAAAEAEBFCAA|1940-11-22|490|2134|164|1940|5|11|22|4|1940|164|2134|Friday|1940Q4|N|Y|N|2429935|2430239|2429590|2429864|N|N|N|N|N| +2429957|AAAAAAAAFAEBFCAA|1940-11-23|490|2134|164|1940|6|11|23|4|1940|164|2134|Saturday|1940Q4|N|Y|N|2429935|2430239|2429591|2429865|N|N|N|N|N| +2429958|AAAAAAAAGAEBFCAA|1940-11-24|490|2134|164|1940|0|11|24|4|1940|164|2134|Sunday|1940Q4|N|N|N|2429935|2430239|2429592|2429866|N|N|N|N|N| +2429959|AAAAAAAAHAEBFCAA|1940-11-25|490|2134|164|1940|1|11|25|4|1940|164|2134|Monday|1940Q4|N|N|N|2429935|2430239|2429593|2429867|N|N|N|N|N| +2429960|AAAAAAAAIAEBFCAA|1940-11-26|490|2135|164|1940|2|11|26|4|1940|164|2135|Tuesday|1940Q4|N|N|N|2429935|2430239|2429594|2429868|N|N|N|N|N| +2429961|AAAAAAAAJAEBFCAA|1940-11-27|490|2135|164|1940|3|11|27|4|1940|164|2135|Wednesday|1940Q4|N|N|N|2429935|2430239|2429595|2429869|N|N|N|N|N| +2429962|AAAAAAAAKAEBFCAA|1940-11-28|490|2135|164|1940|4|11|28|4|1940|164|2135|Thursday|1940Q4|N|N|N|2429935|2430239|2429596|2429870|N|N|N|N|N| +2429963|AAAAAAAALAEBFCAA|1940-11-29|490|2135|164|1940|5|11|29|4|1940|164|2135|Friday|1940Q4|N|Y|N|2429935|2430239|2429597|2429871|N|N|N|N|N| +2429964|AAAAAAAAMAEBFCAA|1940-11-30|490|2135|164|1940|6|11|30|4|1940|164|2135|Saturday|1940Q4|N|Y|N|2429935|2430239|2429598|2429872|N|N|N|N|N| +2429965|AAAAAAAANAEBFCAA|1940-12-01|491|2135|165|1940|0|12|1|4|1940|165|2135|Sunday|1940Q4|N|N|N|2429965|2430299|2429599|2429873|N|N|N|N|N| +2429966|AAAAAAAAOAEBFCAA|1940-12-02|491|2135|165|1940|1|12|2|4|1940|165|2135|Monday|1940Q4|N|N|N|2429965|2430299|2429600|2429874|N|N|N|N|N| +2429967|AAAAAAAAPAEBFCAA|1940-12-03|491|2136|165|1940|2|12|3|4|1940|165|2136|Tuesday|1940Q4|N|N|N|2429965|2430299|2429601|2429875|N|N|N|N|N| +2429968|AAAAAAAAABEBFCAA|1940-12-04|491|2136|165|1940|3|12|4|4|1940|165|2136|Wednesday|1940Q4|N|N|N|2429965|2430299|2429602|2429876|N|N|N|N|N| +2429969|AAAAAAAABBEBFCAA|1940-12-05|491|2136|165|1940|4|12|5|4|1940|165|2136|Thursday|1940Q4|N|N|N|2429965|2430299|2429603|2429877|N|N|N|N|N| +2429970|AAAAAAAACBEBFCAA|1940-12-06|491|2136|165|1940|5|12|6|4|1940|165|2136|Friday|1940Q4|N|Y|N|2429965|2430299|2429604|2429878|N|N|N|N|N| +2429971|AAAAAAAADBEBFCAA|1940-12-07|491|2136|165|1940|6|12|7|4|1940|165|2136|Saturday|1940Q4|N|Y|N|2429965|2430299|2429605|2429879|N|N|N|N|N| +2429972|AAAAAAAAEBEBFCAA|1940-12-08|491|2136|165|1940|0|12|8|4|1940|165|2136|Sunday|1940Q4|N|N|N|2429965|2430299|2429606|2429880|N|N|N|N|N| +2429973|AAAAAAAAFBEBFCAA|1940-12-09|491|2136|165|1940|1|12|9|4|1940|165|2136|Monday|1940Q4|N|N|N|2429965|2430299|2429607|2429881|N|N|N|N|N| +2429974|AAAAAAAAGBEBFCAA|1940-12-10|491|2137|165|1940|2|12|10|4|1940|165|2137|Tuesday|1940Q4|N|N|N|2429965|2430299|2429608|2429882|N|N|N|N|N| +2429975|AAAAAAAAHBEBFCAA|1940-12-11|491|2137|165|1940|3|12|11|4|1940|165|2137|Wednesday|1940Q4|N|N|N|2429965|2430299|2429609|2429883|N|N|N|N|N| +2429976|AAAAAAAAIBEBFCAA|1940-12-12|491|2137|165|1940|4|12|12|4|1940|165|2137|Thursday|1940Q4|N|N|N|2429965|2430299|2429610|2429884|N|N|N|N|N| +2429977|AAAAAAAAJBEBFCAA|1940-12-13|491|2137|165|1940|5|12|13|4|1940|165|2137|Friday|1940Q4|N|Y|N|2429965|2430299|2429611|2429885|N|N|N|N|N| +2429978|AAAAAAAAKBEBFCAA|1940-12-14|491|2137|165|1940|6|12|14|4|1940|165|2137|Saturday|1940Q4|N|Y|N|2429965|2430299|2429612|2429886|N|N|N|N|N| +2429979|AAAAAAAALBEBFCAA|1940-12-15|491|2137|165|1940|0|12|15|4|1940|165|2137|Sunday|1940Q4|N|N|N|2429965|2430299|2429613|2429887|N|N|N|N|N| +2429980|AAAAAAAAMBEBFCAA|1940-12-16|491|2137|165|1940|1|12|16|4|1940|165|2137|Monday|1940Q4|N|N|N|2429965|2430299|2429614|2429888|N|N|N|N|N| +2429981|AAAAAAAANBEBFCAA|1940-12-17|491|2138|165|1940|2|12|17|4|1940|165|2138|Tuesday|1940Q4|N|N|N|2429965|2430299|2429615|2429889|N|N|N|N|N| +2429982|AAAAAAAAOBEBFCAA|1940-12-18|491|2138|165|1940|3|12|18|4|1940|165|2138|Wednesday|1940Q4|N|N|N|2429965|2430299|2429616|2429890|N|N|N|N|N| +2429983|AAAAAAAAPBEBFCAA|1940-12-19|491|2138|165|1940|4|12|19|4|1940|165|2138|Thursday|1940Q4|N|N|N|2429965|2430299|2429617|2429891|N|N|N|N|N| +2429984|AAAAAAAAACEBFCAA|1940-12-20|491|2138|165|1940|5|12|20|4|1940|165|2138|Friday|1940Q4|N|Y|N|2429965|2430299|2429618|2429892|N|N|N|N|N| +2429985|AAAAAAAABCEBFCAA|1940-12-21|491|2138|165|1940|6|12|21|4|1940|165|2138|Saturday|1940Q4|N|Y|N|2429965|2430299|2429619|2429893|N|N|N|N|N| +2429986|AAAAAAAACCEBFCAA|1940-12-22|491|2138|165|1940|0|12|22|4|1940|165|2138|Sunday|1940Q4|N|N|N|2429965|2430299|2429620|2429894|N|N|N|N|N| +2429987|AAAAAAAADCEBFCAA|1940-12-23|491|2138|165|1940|1|12|23|4|1940|165|2138|Monday|1940Q4|N|N|N|2429965|2430299|2429621|2429895|N|N|N|N|N| +2429988|AAAAAAAAECEBFCAA|1940-12-24|491|2139|165|1940|2|12|24|4|1940|165|2139|Tuesday|1940Q4|N|N|N|2429965|2430299|2429622|2429896|N|N|N|N|N| +2429989|AAAAAAAAFCEBFCAA|1940-12-25|491|2139|165|1940|3|12|25|4|1940|165|2139|Wednesday|1940Q4|Y|N|N|2429965|2430299|2429623|2429897|N|N|N|N|N| +2429990|AAAAAAAAGCEBFCAA|1940-12-26|491|2139|165|1940|4|12|26|4|1940|165|2139|Thursday|1940Q4|N|N|Y|2429965|2430299|2429624|2429898|N|N|N|N|N| +2429991|AAAAAAAAHCEBFCAA|1940-12-27|491|2139|165|1940|5|12|27|4|1940|165|2139|Friday|1940Q4|N|Y|N|2429965|2430299|2429625|2429899|N|N|N|N|N| +2429992|AAAAAAAAICEBFCAA|1940-12-28|491|2139|165|1940|6|12|28|4|1940|165|2139|Saturday|1940Q4|N|Y|N|2429965|2430299|2429626|2429900|N|N|N|N|N| +2429993|AAAAAAAAJCEBFCAA|1940-12-29|491|2139|165|1940|0|12|29|4|1940|165|2139|Sunday|1940Q4|N|N|N|2429965|2430299|2429627|2429901|N|N|N|N|N| +2429994|AAAAAAAAKCEBFCAA|1940-12-30|491|2139|165|1940|1|12|30|4|1940|165|2139|Monday|1940Q4|N|N|N|2429965|2430299|2429628|2429902|N|N|N|N|N| +2429995|AAAAAAAALCEBFCAA|1940-12-31|491|2140|165|1940|2|12|31|4|1940|165|2140|Tuesday|1940Q4|Y|N|N|2429965|2430299|2429629|2429903|N|N|N|N|N| +2429996|AAAAAAAAMCEBFCAA|1941-01-01|492|2140|165|1941|3|1|1|1|1941|165|2140|Wednesday|1941Q1|Y|N|Y|2429996|2429995|2429630|2429904|N|N|N|N|N| +2429997|AAAAAAAANCEBFCAA|1941-01-02|492|2140|165|1941|4|1|2|1|1941|165|2140|Thursday|1941Q1|N|N|Y|2429996|2429995|2429631|2429905|N|N|N|N|N| +2429998|AAAAAAAAOCEBFCAA|1941-01-03|492|2140|165|1941|5|1|3|1|1941|165|2140|Friday|1941Q1|N|Y|N|2429996|2429995|2429632|2429906|N|N|N|N|N| +2429999|AAAAAAAAPCEBFCAA|1941-01-04|492|2140|165|1941|6|1|4|1|1941|165|2140|Saturday|1941Q1|N|Y|N|2429996|2429995|2429633|2429907|N|N|N|N|N| +2430000|AAAAAAAAADEBFCAA|1941-01-05|492|2140|165|1941|0|1|5|1|1941|165|2140|Sunday|1941Q1|N|N|N|2429996|2429995|2429634|2429908|N|N|N|N|N| +2430001|AAAAAAAABDEBFCAA|1941-01-06|492|2140|165|1941|1|1|6|1|1941|165|2140|Monday|1941Q1|N|N|N|2429996|2429995|2429635|2429909|N|N|N|N|N| +2430002|AAAAAAAACDEBFCAA|1941-01-07|492|2141|165|1941|2|1|7|1|1941|165|2141|Tuesday|1941Q1|N|N|N|2429996|2429995|2429636|2429910|N|N|N|N|N| +2430003|AAAAAAAADDEBFCAA|1941-01-08|492|2141|165|1941|3|1|8|1|1941|165|2141|Wednesday|1941Q1|N|N|N|2429996|2429995|2429637|2429911|N|N|N|N|N| +2430004|AAAAAAAAEDEBFCAA|1941-01-09|492|2141|165|1941|4|1|9|1|1941|165|2141|Thursday|1941Q1|N|N|N|2429996|2429995|2429638|2429912|N|N|N|N|N| +2430005|AAAAAAAAFDEBFCAA|1941-01-10|492|2141|165|1941|5|1|10|1|1941|165|2141|Friday|1941Q1|N|Y|N|2429996|2429995|2429639|2429913|N|N|N|N|N| +2430006|AAAAAAAAGDEBFCAA|1941-01-11|492|2141|165|1941|6|1|11|1|1941|165|2141|Saturday|1941Q1|N|Y|N|2429996|2429995|2429640|2429914|N|N|N|N|N| +2430007|AAAAAAAAHDEBFCAA|1941-01-12|492|2141|165|1941|0|1|12|1|1941|165|2141|Sunday|1941Q1|N|N|N|2429996|2429995|2429641|2429915|N|N|N|N|N| +2430008|AAAAAAAAIDEBFCAA|1941-01-13|492|2141|165|1941|1|1|13|1|1941|165|2141|Monday|1941Q1|N|N|N|2429996|2429995|2429642|2429916|N|N|N|N|N| +2430009|AAAAAAAAJDEBFCAA|1941-01-14|492|2142|165|1941|2|1|14|1|1941|165|2142|Tuesday|1941Q1|N|N|N|2429996|2429995|2429643|2429917|N|N|N|N|N| +2430010|AAAAAAAAKDEBFCAA|1941-01-15|492|2142|165|1941|3|1|15|1|1941|165|2142|Wednesday|1941Q1|N|N|N|2429996|2429995|2429644|2429918|N|N|N|N|N| +2430011|AAAAAAAALDEBFCAA|1941-01-16|492|2142|165|1941|4|1|16|1|1941|165|2142|Thursday|1941Q1|N|N|N|2429996|2429995|2429645|2429919|N|N|N|N|N| +2430012|AAAAAAAAMDEBFCAA|1941-01-17|492|2142|165|1941|5|1|17|1|1941|165|2142|Friday|1941Q1|N|Y|N|2429996|2429995|2429646|2429920|N|N|N|N|N| +2430013|AAAAAAAANDEBFCAA|1941-01-18|492|2142|165|1941|6|1|18|1|1941|165|2142|Saturday|1941Q1|N|Y|N|2429996|2429995|2429647|2429921|N|N|N|N|N| +2430014|AAAAAAAAODEBFCAA|1941-01-19|492|2142|165|1941|0|1|19|1|1941|165|2142|Sunday|1941Q1|N|N|N|2429996|2429995|2429648|2429922|N|N|N|N|N| +2430015|AAAAAAAAPDEBFCAA|1941-01-20|492|2142|165|1941|1|1|20|1|1941|165|2142|Monday|1941Q1|N|N|N|2429996|2429995|2429649|2429923|N|N|N|N|N| +2430016|AAAAAAAAAEEBFCAA|1941-01-21|492|2143|165|1941|2|1|21|1|1941|165|2143|Tuesday|1941Q1|N|N|N|2429996|2429995|2429650|2429924|N|N|N|N|N| +2430017|AAAAAAAABEEBFCAA|1941-01-22|492|2143|165|1941|3|1|22|1|1941|165|2143|Wednesday|1941Q1|N|N|N|2429996|2429995|2429651|2429925|N|N|N|N|N| +2430018|AAAAAAAACEEBFCAA|1941-01-23|492|2143|165|1941|4|1|23|1|1941|165|2143|Thursday|1941Q1|N|N|N|2429996|2429995|2429652|2429926|N|N|N|N|N| +2430019|AAAAAAAADEEBFCAA|1941-01-24|492|2143|165|1941|5|1|24|1|1941|165|2143|Friday|1941Q1|N|Y|N|2429996|2429995|2429653|2429927|N|N|N|N|N| +2430020|AAAAAAAAEEEBFCAA|1941-01-25|492|2143|165|1941|6|1|25|1|1941|165|2143|Saturday|1941Q1|N|Y|N|2429996|2429995|2429654|2429928|N|N|N|N|N| +2430021|AAAAAAAAFEEBFCAA|1941-01-26|492|2143|165|1941|0|1|26|1|1941|165|2143|Sunday|1941Q1|N|N|N|2429996|2429995|2429655|2429929|N|N|N|N|N| +2430022|AAAAAAAAGEEBFCAA|1941-01-27|492|2143|165|1941|1|1|27|1|1941|165|2143|Monday|1941Q1|N|N|N|2429996|2429995|2429656|2429930|N|N|N|N|N| +2430023|AAAAAAAAHEEBFCAA|1941-01-28|492|2144|165|1941|2|1|28|1|1941|165|2144|Tuesday|1941Q1|N|N|N|2429996|2429995|2429657|2429931|N|N|N|N|N| +2430024|AAAAAAAAIEEBFCAA|1941-01-29|492|2144|165|1941|3|1|29|1|1941|165|2144|Wednesday|1941Q1|N|N|N|2429996|2429995|2429658|2429932|N|N|N|N|N| +2430025|AAAAAAAAJEEBFCAA|1941-01-30|492|2144|165|1941|4|1|30|1|1941|165|2144|Thursday|1941Q1|N|N|N|2429996|2429995|2429659|2429933|N|N|N|N|N| +2430026|AAAAAAAAKEEBFCAA|1941-01-31|492|2144|165|1941|5|1|31|1|1941|165|2144|Friday|1941Q1|N|Y|N|2429996|2429995|2429660|2429934|N|N|N|N|N| +2430027|AAAAAAAALEEBFCAA|1941-02-01|493|2144|165|1941|6|2|1|1|1941|165|2144|Saturday|1941Q1|N|Y|N|2430027|2430057|2429661|2429935|N|N|N|N|N| +2430028|AAAAAAAAMEEBFCAA|1941-02-02|493|2144|165|1941|0|2|2|1|1941|165|2144|Sunday|1941Q1|N|N|N|2430027|2430057|2429662|2429936|N|N|N|N|N| +2430029|AAAAAAAANEEBFCAA|1941-02-03|493|2144|165|1941|1|2|3|1|1941|165|2144|Monday|1941Q1|N|N|N|2430027|2430057|2429663|2429937|N|N|N|N|N| +2430030|AAAAAAAAOEEBFCAA|1941-02-04|493|2145|165|1941|2|2|4|1|1941|165|2145|Tuesday|1941Q1|N|N|N|2430027|2430057|2429664|2429938|N|N|N|N|N| +2430031|AAAAAAAAPEEBFCAA|1941-02-05|493|2145|165|1941|3|2|5|1|1941|165|2145|Wednesday|1941Q1|N|N|N|2430027|2430057|2429665|2429939|N|N|N|N|N| +2430032|AAAAAAAAAFEBFCAA|1941-02-06|493|2145|165|1941|4|2|6|1|1941|165|2145|Thursday|1941Q1|N|N|N|2430027|2430057|2429666|2429940|N|N|N|N|N| +2430033|AAAAAAAABFEBFCAA|1941-02-07|493|2145|165|1941|5|2|7|1|1941|165|2145|Friday|1941Q1|N|Y|N|2430027|2430057|2429667|2429941|N|N|N|N|N| +2430034|AAAAAAAACFEBFCAA|1941-02-08|493|2145|165|1941|6|2|8|1|1941|165|2145|Saturday|1941Q1|N|Y|N|2430027|2430057|2429668|2429942|N|N|N|N|N| +2430035|AAAAAAAADFEBFCAA|1941-02-09|493|2145|165|1941|0|2|9|1|1941|165|2145|Sunday|1941Q1|N|N|N|2430027|2430057|2429669|2429943|N|N|N|N|N| +2430036|AAAAAAAAEFEBFCAA|1941-02-10|493|2145|165|1941|1|2|10|1|1941|165|2145|Monday|1941Q1|N|N|N|2430027|2430057|2429670|2429944|N|N|N|N|N| +2430037|AAAAAAAAFFEBFCAA|1941-02-11|493|2146|165|1941|2|2|11|1|1941|165|2146|Tuesday|1941Q1|N|N|N|2430027|2430057|2429671|2429945|N|N|N|N|N| +2430038|AAAAAAAAGFEBFCAA|1941-02-12|493|2146|165|1941|3|2|12|1|1941|165|2146|Wednesday|1941Q1|N|N|N|2430027|2430057|2429672|2429946|N|N|N|N|N| +2430039|AAAAAAAAHFEBFCAA|1941-02-13|493|2146|165|1941|4|2|13|1|1941|165|2146|Thursday|1941Q1|N|N|N|2430027|2430057|2429673|2429947|N|N|N|N|N| +2430040|AAAAAAAAIFEBFCAA|1941-02-14|493|2146|165|1941|5|2|14|1|1941|165|2146|Friday|1941Q1|N|Y|N|2430027|2430057|2429674|2429948|N|N|N|N|N| +2430041|AAAAAAAAJFEBFCAA|1941-02-15|493|2146|165|1941|6|2|15|1|1941|165|2146|Saturday|1941Q1|N|Y|N|2430027|2430057|2429675|2429949|N|N|N|N|N| +2430042|AAAAAAAAKFEBFCAA|1941-02-16|493|2146|165|1941|0|2|16|1|1941|165|2146|Sunday|1941Q1|N|N|N|2430027|2430057|2429676|2429950|N|N|N|N|N| +2430043|AAAAAAAALFEBFCAA|1941-02-17|493|2146|165|1941|1|2|17|1|1941|165|2146|Monday|1941Q1|N|N|N|2430027|2430057|2429677|2429951|N|N|N|N|N| +2430044|AAAAAAAAMFEBFCAA|1941-02-18|493|2147|165|1941|2|2|18|1|1941|165|2147|Tuesday|1941Q1|N|N|N|2430027|2430057|2429678|2429952|N|N|N|N|N| +2430045|AAAAAAAANFEBFCAA|1941-02-19|493|2147|165|1941|3|2|19|1|1941|165|2147|Wednesday|1941Q1|N|N|N|2430027|2430057|2429679|2429953|N|N|N|N|N| +2430046|AAAAAAAAOFEBFCAA|1941-02-20|493|2147|165|1941|4|2|20|1|1941|165|2147|Thursday|1941Q1|N|N|N|2430027|2430057|2429680|2429954|N|N|N|N|N| +2430047|AAAAAAAAPFEBFCAA|1941-02-21|493|2147|165|1941|5|2|21|1|1941|165|2147|Friday|1941Q1|N|Y|N|2430027|2430057|2429681|2429955|N|N|N|N|N| +2430048|AAAAAAAAAGEBFCAA|1941-02-22|493|2147|165|1941|6|2|22|1|1941|165|2147|Saturday|1941Q1|N|Y|N|2430027|2430057|2429682|2429956|N|N|N|N|N| +2430049|AAAAAAAABGEBFCAA|1941-02-23|493|2147|165|1941|0|2|23|1|1941|165|2147|Sunday|1941Q1|N|N|N|2430027|2430057|2429683|2429957|N|N|N|N|N| +2430050|AAAAAAAACGEBFCAA|1941-02-24|493|2147|165|1941|1|2|24|1|1941|165|2147|Monday|1941Q1|N|N|N|2430027|2430057|2429684|2429958|N|N|N|N|N| +2430051|AAAAAAAADGEBFCAA|1941-02-25|493|2148|165|1941|2|2|25|1|1941|165|2148|Tuesday|1941Q1|N|N|N|2430027|2430057|2429685|2429959|N|N|N|N|N| +2430052|AAAAAAAAEGEBFCAA|1941-02-26|493|2148|165|1941|3|2|26|1|1941|165|2148|Wednesday|1941Q1|N|N|N|2430027|2430057|2429686|2429960|N|N|N|N|N| +2430053|AAAAAAAAFGEBFCAA|1941-02-27|493|2148|165|1941|4|2|27|1|1941|165|2148|Thursday|1941Q1|N|N|N|2430027|2430057|2429687|2429961|N|N|N|N|N| +2430054|AAAAAAAAGGEBFCAA|1941-02-28|493|2148|165|1941|5|2|28|1|1941|165|2148|Friday|1941Q1|N|Y|N|2430027|2430057|2429688|2429962|N|N|N|N|N| +2430055|AAAAAAAAHGEBFCAA|1941-03-01|494|2148|166|1941|6|3|1|1|1941|166|2148|Saturday|1941Q1|N|Y|N|2430055|2430113|2429690|2429963|N|N|N|N|N| +2430056|AAAAAAAAIGEBFCAA|1941-03-02|494|2148|166|1941|0|3|2|1|1941|166|2148|Sunday|1941Q1|N|N|N|2430055|2430113|2429691|2429964|N|N|N|N|N| +2430057|AAAAAAAAJGEBFCAA|1941-03-03|494|2148|166|1941|1|3|3|1|1941|166|2148|Monday|1941Q1|N|N|N|2430055|2430113|2429692|2429965|N|N|N|N|N| +2430058|AAAAAAAAKGEBFCAA|1941-03-04|494|2149|166|1941|2|3|4|1|1941|166|2149|Tuesday|1941Q1|N|N|N|2430055|2430113|2429693|2429966|N|N|N|N|N| +2430059|AAAAAAAALGEBFCAA|1941-03-05|494|2149|166|1941|3|3|5|1|1941|166|2149|Wednesday|1941Q1|N|N|N|2430055|2430113|2429694|2429967|N|N|N|N|N| +2430060|AAAAAAAAMGEBFCAA|1941-03-06|494|2149|166|1941|4|3|6|1|1941|166|2149|Thursday|1941Q1|N|N|N|2430055|2430113|2429695|2429968|N|N|N|N|N| +2430061|AAAAAAAANGEBFCAA|1941-03-07|494|2149|166|1941|5|3|7|1|1941|166|2149|Friday|1941Q1|N|Y|N|2430055|2430113|2429696|2429969|N|N|N|N|N| +2430062|AAAAAAAAOGEBFCAA|1941-03-08|494|2149|166|1941|6|3|8|1|1941|166|2149|Saturday|1941Q1|N|Y|N|2430055|2430113|2429697|2429970|N|N|N|N|N| +2430063|AAAAAAAAPGEBFCAA|1941-03-09|494|2149|166|1941|0|3|9|1|1941|166|2149|Sunday|1941Q1|N|N|N|2430055|2430113|2429698|2429971|N|N|N|N|N| +2430064|AAAAAAAAAHEBFCAA|1941-03-10|494|2149|166|1941|1|3|10|1|1941|166|2149|Monday|1941Q1|N|N|N|2430055|2430113|2429699|2429972|N|N|N|N|N| +2430065|AAAAAAAABHEBFCAA|1941-03-11|494|2150|166|1941|2|3|11|1|1941|166|2150|Tuesday|1941Q1|N|N|N|2430055|2430113|2429700|2429973|N|N|N|N|N| +2430066|AAAAAAAACHEBFCAA|1941-03-12|494|2150|166|1941|3|3|12|1|1941|166|2150|Wednesday|1941Q1|N|N|N|2430055|2430113|2429701|2429974|N|N|N|N|N| +2430067|AAAAAAAADHEBFCAA|1941-03-13|494|2150|166|1941|4|3|13|1|1941|166|2150|Thursday|1941Q1|N|N|N|2430055|2430113|2429702|2429975|N|N|N|N|N| +2430068|AAAAAAAAEHEBFCAA|1941-03-14|494|2150|166|1941|5|3|14|1|1941|166|2150|Friday|1941Q1|N|Y|N|2430055|2430113|2429703|2429976|N|N|N|N|N| +2430069|AAAAAAAAFHEBFCAA|1941-03-15|494|2150|166|1941|6|3|15|1|1941|166|2150|Saturday|1941Q1|N|Y|N|2430055|2430113|2429704|2429977|N|N|N|N|N| +2430070|AAAAAAAAGHEBFCAA|1941-03-16|494|2150|166|1941|0|3|16|1|1941|166|2150|Sunday|1941Q1|N|N|N|2430055|2430113|2429705|2429978|N|N|N|N|N| +2430071|AAAAAAAAHHEBFCAA|1941-03-17|494|2150|166|1941|1|3|17|1|1941|166|2150|Monday|1941Q1|N|N|N|2430055|2430113|2429706|2429979|N|N|N|N|N| +2430072|AAAAAAAAIHEBFCAA|1941-03-18|494|2151|166|1941|2|3|18|1|1941|166|2151|Tuesday|1941Q1|N|N|N|2430055|2430113|2429707|2429980|N|N|N|N|N| +2430073|AAAAAAAAJHEBFCAA|1941-03-19|494|2151|166|1941|3|3|19|1|1941|166|2151|Wednesday|1941Q1|N|N|N|2430055|2430113|2429708|2429981|N|N|N|N|N| +2430074|AAAAAAAAKHEBFCAA|1941-03-20|494|2151|166|1941|4|3|20|1|1941|166|2151|Thursday|1941Q1|N|N|N|2430055|2430113|2429709|2429982|N|N|N|N|N| +2430075|AAAAAAAALHEBFCAA|1941-03-21|494|2151|166|1941|5|3|21|1|1941|166|2151|Friday|1941Q1|N|Y|N|2430055|2430113|2429710|2429983|N|N|N|N|N| +2430076|AAAAAAAAMHEBFCAA|1941-03-22|494|2151|166|1941|6|3|22|1|1941|166|2151|Saturday|1941Q1|N|Y|N|2430055|2430113|2429711|2429984|N|N|N|N|N| +2430077|AAAAAAAANHEBFCAA|1941-03-23|494|2151|166|1941|0|3|23|1|1941|166|2151|Sunday|1941Q1|N|N|N|2430055|2430113|2429712|2429985|N|N|N|N|N| +2430078|AAAAAAAAOHEBFCAA|1941-03-24|494|2151|166|1941|1|3|24|1|1941|166|2151|Monday|1941Q1|N|N|N|2430055|2430113|2429713|2429986|N|N|N|N|N| +2430079|AAAAAAAAPHEBFCAA|1941-03-25|494|2152|166|1941|2|3|25|1|1941|166|2152|Tuesday|1941Q1|N|N|N|2430055|2430113|2429714|2429987|N|N|N|N|N| +2430080|AAAAAAAAAIEBFCAA|1941-03-26|494|2152|166|1941|3|3|26|1|1941|166|2152|Wednesday|1941Q1|N|N|N|2430055|2430113|2429715|2429988|N|N|N|N|N| +2430081|AAAAAAAABIEBFCAA|1941-03-27|494|2152|166|1941|4|3|27|1|1941|166|2152|Thursday|1941Q1|N|N|N|2430055|2430113|2429716|2429989|N|N|N|N|N| +2430082|AAAAAAAACIEBFCAA|1941-03-28|494|2152|166|1941|5|3|28|1|1941|166|2152|Friday|1941Q1|N|Y|N|2430055|2430113|2429717|2429990|N|N|N|N|N| +2430083|AAAAAAAADIEBFCAA|1941-03-29|494|2152|166|1941|6|3|29|1|1941|166|2152|Saturday|1941Q1|N|Y|N|2430055|2430113|2429718|2429991|N|N|N|N|N| +2430084|AAAAAAAAEIEBFCAA|1941-03-30|494|2152|166|1941|0|3|30|1|1941|166|2152|Sunday|1941Q1|N|N|N|2430055|2430113|2429719|2429992|N|N|N|N|N| +2430085|AAAAAAAAFIEBFCAA|1941-03-31|494|2152|166|1941|1|3|31|1|1941|166|2152|Monday|1941Q1|N|N|N|2430055|2430113|2429720|2429993|N|N|N|N|N| +2430086|AAAAAAAAGIEBFCAA|1941-04-01|495|2153|166|1941|2|4|1|1|1941|166|2153|Tuesday|1941Q1|N|N|N|2430086|2430175|2429721|2429996|N|N|N|N|N| +2430087|AAAAAAAAHIEBFCAA|1941-04-02|495|2153|166|1941|3|4|2|2|1941|166|2153|Wednesday|1941Q2|N|N|N|2430086|2430175|2429722|2429997|N|N|N|N|N| +2430088|AAAAAAAAIIEBFCAA|1941-04-03|495|2153|166|1941|4|4|3|2|1941|166|2153|Thursday|1941Q2|N|N|N|2430086|2430175|2429723|2429998|N|N|N|N|N| +2430089|AAAAAAAAJIEBFCAA|1941-04-04|495|2153|166|1941|5|4|4|2|1941|166|2153|Friday|1941Q2|N|Y|N|2430086|2430175|2429724|2429999|N|N|N|N|N| +2430090|AAAAAAAAKIEBFCAA|1941-04-05|495|2153|166|1941|6|4|5|2|1941|166|2153|Saturday|1941Q2|N|Y|N|2430086|2430175|2429725|2430000|N|N|N|N|N| +2430091|AAAAAAAALIEBFCAA|1941-04-06|495|2153|166|1941|0|4|6|2|1941|166|2153|Sunday|1941Q2|N|N|N|2430086|2430175|2429726|2430001|N|N|N|N|N| +2430092|AAAAAAAAMIEBFCAA|1941-04-07|495|2153|166|1941|1|4|7|2|1941|166|2153|Monday|1941Q2|N|N|N|2430086|2430175|2429727|2430002|N|N|N|N|N| +2430093|AAAAAAAANIEBFCAA|1941-04-08|495|2154|166|1941|2|4|8|2|1941|166|2154|Tuesday|1941Q2|N|N|N|2430086|2430175|2429728|2430003|N|N|N|N|N| +2430094|AAAAAAAAOIEBFCAA|1941-04-09|495|2154|166|1941|3|4|9|2|1941|166|2154|Wednesday|1941Q2|N|N|N|2430086|2430175|2429729|2430004|N|N|N|N|N| +2430095|AAAAAAAAPIEBFCAA|1941-04-10|495|2154|166|1941|4|4|10|2|1941|166|2154|Thursday|1941Q2|N|N|N|2430086|2430175|2429730|2430005|N|N|N|N|N| +2430096|AAAAAAAAAJEBFCAA|1941-04-11|495|2154|166|1941|5|4|11|2|1941|166|2154|Friday|1941Q2|N|Y|N|2430086|2430175|2429731|2430006|N|N|N|N|N| +2430097|AAAAAAAABJEBFCAA|1941-04-12|495|2154|166|1941|6|4|12|2|1941|166|2154|Saturday|1941Q2|N|Y|N|2430086|2430175|2429732|2430007|N|N|N|N|N| +2430098|AAAAAAAACJEBFCAA|1941-04-13|495|2154|166|1941|0|4|13|2|1941|166|2154|Sunday|1941Q2|N|N|N|2430086|2430175|2429733|2430008|N|N|N|N|N| +2430099|AAAAAAAADJEBFCAA|1941-04-14|495|2154|166|1941|1|4|14|2|1941|166|2154|Monday|1941Q2|N|N|N|2430086|2430175|2429734|2430009|N|N|N|N|N| +2430100|AAAAAAAAEJEBFCAA|1941-04-15|495|2155|166|1941|2|4|15|2|1941|166|2155|Tuesday|1941Q2|N|N|N|2430086|2430175|2429735|2430010|N|N|N|N|N| +2430101|AAAAAAAAFJEBFCAA|1941-04-16|495|2155|166|1941|3|4|16|2|1941|166|2155|Wednesday|1941Q2|N|N|N|2430086|2430175|2429736|2430011|N|N|N|N|N| +2430102|AAAAAAAAGJEBFCAA|1941-04-17|495|2155|166|1941|4|4|17|2|1941|166|2155|Thursday|1941Q2|N|N|N|2430086|2430175|2429737|2430012|N|N|N|N|N| +2430103|AAAAAAAAHJEBFCAA|1941-04-18|495|2155|166|1941|5|4|18|2|1941|166|2155|Friday|1941Q2|N|Y|N|2430086|2430175|2429738|2430013|N|N|N|N|N| +2430104|AAAAAAAAIJEBFCAA|1941-04-19|495|2155|166|1941|6|4|19|2|1941|166|2155|Saturday|1941Q2|N|Y|N|2430086|2430175|2429739|2430014|N|N|N|N|N| +2430105|AAAAAAAAJJEBFCAA|1941-04-20|495|2155|166|1941|0|4|20|2|1941|166|2155|Sunday|1941Q2|N|N|N|2430086|2430175|2429740|2430015|N|N|N|N|N| +2430106|AAAAAAAAKJEBFCAA|1941-04-21|495|2155|166|1941|1|4|21|2|1941|166|2155|Monday|1941Q2|N|N|N|2430086|2430175|2429741|2430016|N|N|N|N|N| +2430107|AAAAAAAALJEBFCAA|1941-04-22|495|2156|166|1941|2|4|22|2|1941|166|2156|Tuesday|1941Q2|N|N|N|2430086|2430175|2429742|2430017|N|N|N|N|N| +2430108|AAAAAAAAMJEBFCAA|1941-04-23|495|2156|166|1941|3|4|23|2|1941|166|2156|Wednesday|1941Q2|N|N|N|2430086|2430175|2429743|2430018|N|N|N|N|N| +2430109|AAAAAAAANJEBFCAA|1941-04-24|495|2156|166|1941|4|4|24|2|1941|166|2156|Thursday|1941Q2|N|N|N|2430086|2430175|2429744|2430019|N|N|N|N|N| +2430110|AAAAAAAAOJEBFCAA|1941-04-25|495|2156|166|1941|5|4|25|2|1941|166|2156|Friday|1941Q2|N|Y|N|2430086|2430175|2429745|2430020|N|N|N|N|N| +2430111|AAAAAAAAPJEBFCAA|1941-04-26|495|2156|166|1941|6|4|26|2|1941|166|2156|Saturday|1941Q2|N|Y|N|2430086|2430175|2429746|2430021|N|N|N|N|N| +2430112|AAAAAAAAAKEBFCAA|1941-04-27|495|2156|166|1941|0|4|27|2|1941|166|2156|Sunday|1941Q2|N|N|N|2430086|2430175|2429747|2430022|N|N|N|N|N| +2430113|AAAAAAAABKEBFCAA|1941-04-28|495|2156|166|1941|1|4|28|2|1941|166|2156|Monday|1941Q2|N|N|N|2430086|2430175|2429748|2430023|N|N|N|N|N| +2430114|AAAAAAAACKEBFCAA|1941-04-29|495|2157|166|1941|2|4|29|2|1941|166|2157|Tuesday|1941Q2|N|N|N|2430086|2430175|2429749|2430024|N|N|N|N|N| +2430115|AAAAAAAADKEBFCAA|1941-04-30|495|2157|166|1941|3|4|30|2|1941|166|2157|Wednesday|1941Q2|N|N|N|2430086|2430175|2429750|2430025|N|N|N|N|N| +2430116|AAAAAAAAEKEBFCAA|1941-05-01|496|2157|166|1941|4|5|1|2|1941|166|2157|Thursday|1941Q2|N|N|N|2430116|2430235|2429751|2430026|N|N|N|N|N| +2430117|AAAAAAAAFKEBFCAA|1941-05-02|496|2157|166|1941|5|5|2|2|1941|166|2157|Friday|1941Q2|N|Y|N|2430116|2430235|2429752|2430027|N|N|N|N|N| +2430118|AAAAAAAAGKEBFCAA|1941-05-03|496|2157|166|1941|6|5|3|2|1941|166|2157|Saturday|1941Q2|N|Y|N|2430116|2430235|2429753|2430028|N|N|N|N|N| +2430119|AAAAAAAAHKEBFCAA|1941-05-04|496|2157|166|1941|0|5|4|2|1941|166|2157|Sunday|1941Q2|N|N|N|2430116|2430235|2429754|2430029|N|N|N|N|N| +2430120|AAAAAAAAIKEBFCAA|1941-05-05|496|2157|166|1941|1|5|5|2|1941|166|2157|Monday|1941Q2|N|N|N|2430116|2430235|2429755|2430030|N|N|N|N|N| +2430121|AAAAAAAAJKEBFCAA|1941-05-06|496|2158|166|1941|2|5|6|2|1941|166|2158|Tuesday|1941Q2|N|N|N|2430116|2430235|2429756|2430031|N|N|N|N|N| +2430122|AAAAAAAAKKEBFCAA|1941-05-07|496|2158|166|1941|3|5|7|2|1941|166|2158|Wednesday|1941Q2|N|N|N|2430116|2430235|2429757|2430032|N|N|N|N|N| +2430123|AAAAAAAALKEBFCAA|1941-05-08|496|2158|166|1941|4|5|8|2|1941|166|2158|Thursday|1941Q2|N|N|N|2430116|2430235|2429758|2430033|N|N|N|N|N| +2430124|AAAAAAAAMKEBFCAA|1941-05-09|496|2158|166|1941|5|5|9|2|1941|166|2158|Friday|1941Q2|N|Y|N|2430116|2430235|2429759|2430034|N|N|N|N|N| +2430125|AAAAAAAANKEBFCAA|1941-05-10|496|2158|166|1941|6|5|10|2|1941|166|2158|Saturday|1941Q2|N|Y|N|2430116|2430235|2429760|2430035|N|N|N|N|N| +2430126|AAAAAAAAOKEBFCAA|1941-05-11|496|2158|166|1941|0|5|11|2|1941|166|2158|Sunday|1941Q2|N|N|N|2430116|2430235|2429761|2430036|N|N|N|N|N| +2430127|AAAAAAAAPKEBFCAA|1941-05-12|496|2158|166|1941|1|5|12|2|1941|166|2158|Monday|1941Q2|N|N|N|2430116|2430235|2429762|2430037|N|N|N|N|N| +2430128|AAAAAAAAALEBFCAA|1941-05-13|496|2159|166|1941|2|5|13|2|1941|166|2159|Tuesday|1941Q2|N|N|N|2430116|2430235|2429763|2430038|N|N|N|N|N| +2430129|AAAAAAAABLEBFCAA|1941-05-14|496|2159|166|1941|3|5|14|2|1941|166|2159|Wednesday|1941Q2|N|N|N|2430116|2430235|2429764|2430039|N|N|N|N|N| +2430130|AAAAAAAACLEBFCAA|1941-05-15|496|2159|166|1941|4|5|15|2|1941|166|2159|Thursday|1941Q2|N|N|N|2430116|2430235|2429765|2430040|N|N|N|N|N| +2430131|AAAAAAAADLEBFCAA|1941-05-16|496|2159|166|1941|5|5|16|2|1941|166|2159|Friday|1941Q2|N|Y|N|2430116|2430235|2429766|2430041|N|N|N|N|N| +2430132|AAAAAAAAELEBFCAA|1941-05-17|496|2159|166|1941|6|5|17|2|1941|166|2159|Saturday|1941Q2|N|Y|N|2430116|2430235|2429767|2430042|N|N|N|N|N| +2430133|AAAAAAAAFLEBFCAA|1941-05-18|496|2159|166|1941|0|5|18|2|1941|166|2159|Sunday|1941Q2|N|N|N|2430116|2430235|2429768|2430043|N|N|N|N|N| +2430134|AAAAAAAAGLEBFCAA|1941-05-19|496|2159|166|1941|1|5|19|2|1941|166|2159|Monday|1941Q2|N|N|N|2430116|2430235|2429769|2430044|N|N|N|N|N| +2430135|AAAAAAAAHLEBFCAA|1941-05-20|496|2160|166|1941|2|5|20|2|1941|166|2160|Tuesday|1941Q2|N|N|N|2430116|2430235|2429770|2430045|N|N|N|N|N| +2430136|AAAAAAAAILEBFCAA|1941-05-21|496|2160|166|1941|3|5|21|2|1941|166|2160|Wednesday|1941Q2|N|N|N|2430116|2430235|2429771|2430046|N|N|N|N|N| +2430137|AAAAAAAAJLEBFCAA|1941-05-22|496|2160|166|1941|4|5|22|2|1941|166|2160|Thursday|1941Q2|N|N|N|2430116|2430235|2429772|2430047|N|N|N|N|N| +2430138|AAAAAAAAKLEBFCAA|1941-05-23|496|2160|166|1941|5|5|23|2|1941|166|2160|Friday|1941Q2|N|Y|N|2430116|2430235|2429773|2430048|N|N|N|N|N| +2430139|AAAAAAAALLEBFCAA|1941-05-24|496|2160|166|1941|6|5|24|2|1941|166|2160|Saturday|1941Q2|N|Y|N|2430116|2430235|2429774|2430049|N|N|N|N|N| +2430140|AAAAAAAAMLEBFCAA|1941-05-25|496|2160|166|1941|0|5|25|2|1941|166|2160|Sunday|1941Q2|N|N|N|2430116|2430235|2429775|2430050|N|N|N|N|N| +2430141|AAAAAAAANLEBFCAA|1941-05-26|496|2160|166|1941|1|5|26|2|1941|166|2160|Monday|1941Q2|N|N|N|2430116|2430235|2429776|2430051|N|N|N|N|N| +2430142|AAAAAAAAOLEBFCAA|1941-05-27|496|2161|166|1941|2|5|27|2|1941|166|2161|Tuesday|1941Q2|N|N|N|2430116|2430235|2429777|2430052|N|N|N|N|N| +2430143|AAAAAAAAPLEBFCAA|1941-05-28|496|2161|166|1941|3|5|28|2|1941|166|2161|Wednesday|1941Q2|N|N|N|2430116|2430235|2429778|2430053|N|N|N|N|N| +2430144|AAAAAAAAAMEBFCAA|1941-05-29|496|2161|166|1941|4|5|29|2|1941|166|2161|Thursday|1941Q2|N|N|N|2430116|2430235|2429779|2430054|N|N|N|N|N| +2430145|AAAAAAAABMEBFCAA|1941-05-30|496|2161|166|1941|5|5|30|2|1941|166|2161|Friday|1941Q2|N|Y|N|2430116|2430235|2429780|2430055|N|N|N|N|N| +2430146|AAAAAAAACMEBFCAA|1941-05-31|496|2161|166|1941|6|5|31|2|1941|166|2161|Saturday|1941Q2|N|Y|N|2430116|2430235|2429781|2430056|N|N|N|N|N| +2430147|AAAAAAAADMEBFCAA|1941-06-01|497|2161|167|1941|0|6|1|2|1941|167|2161|Sunday|1941Q2|N|N|N|2430147|2430297|2429782|2430057|N|N|N|N|N| +2430148|AAAAAAAAEMEBFCAA|1941-06-02|497|2161|167|1941|1|6|2|2|1941|167|2161|Monday|1941Q2|N|N|N|2430147|2430297|2429783|2430058|N|N|N|N|N| +2430149|AAAAAAAAFMEBFCAA|1941-06-03|497|2162|167|1941|2|6|3|2|1941|167|2162|Tuesday|1941Q2|N|N|N|2430147|2430297|2429784|2430059|N|N|N|N|N| +2430150|AAAAAAAAGMEBFCAA|1941-06-04|497|2162|167|1941|3|6|4|2|1941|167|2162|Wednesday|1941Q2|N|N|N|2430147|2430297|2429785|2430060|N|N|N|N|N| +2430151|AAAAAAAAHMEBFCAA|1941-06-05|497|2162|167|1941|4|6|5|2|1941|167|2162|Thursday|1941Q2|N|N|N|2430147|2430297|2429786|2430061|N|N|N|N|N| +2430152|AAAAAAAAIMEBFCAA|1941-06-06|497|2162|167|1941|5|6|6|2|1941|167|2162|Friday|1941Q2|N|Y|N|2430147|2430297|2429787|2430062|N|N|N|N|N| +2430153|AAAAAAAAJMEBFCAA|1941-06-07|497|2162|167|1941|6|6|7|2|1941|167|2162|Saturday|1941Q2|N|Y|N|2430147|2430297|2429788|2430063|N|N|N|N|N| +2430154|AAAAAAAAKMEBFCAA|1941-06-08|497|2162|167|1941|0|6|8|2|1941|167|2162|Sunday|1941Q2|N|N|N|2430147|2430297|2429789|2430064|N|N|N|N|N| +2430155|AAAAAAAALMEBFCAA|1941-06-09|497|2162|167|1941|1|6|9|2|1941|167|2162|Monday|1941Q2|N|N|N|2430147|2430297|2429790|2430065|N|N|N|N|N| +2430156|AAAAAAAAMMEBFCAA|1941-06-10|497|2163|167|1941|2|6|10|2|1941|167|2163|Tuesday|1941Q2|N|N|N|2430147|2430297|2429791|2430066|N|N|N|N|N| +2430157|AAAAAAAANMEBFCAA|1941-06-11|497|2163|167|1941|3|6|11|2|1941|167|2163|Wednesday|1941Q2|N|N|N|2430147|2430297|2429792|2430067|N|N|N|N|N| +2430158|AAAAAAAAOMEBFCAA|1941-06-12|497|2163|167|1941|4|6|12|2|1941|167|2163|Thursday|1941Q2|N|N|N|2430147|2430297|2429793|2430068|N|N|N|N|N| +2430159|AAAAAAAAPMEBFCAA|1941-06-13|497|2163|167|1941|5|6|13|2|1941|167|2163|Friday|1941Q2|N|Y|N|2430147|2430297|2429794|2430069|N|N|N|N|N| +2430160|AAAAAAAAANEBFCAA|1941-06-14|497|2163|167|1941|6|6|14|2|1941|167|2163|Saturday|1941Q2|N|Y|N|2430147|2430297|2429795|2430070|N|N|N|N|N| +2430161|AAAAAAAABNEBFCAA|1941-06-15|497|2163|167|1941|0|6|15|2|1941|167|2163|Sunday|1941Q2|N|N|N|2430147|2430297|2429796|2430071|N|N|N|N|N| +2430162|AAAAAAAACNEBFCAA|1941-06-16|497|2163|167|1941|1|6|16|2|1941|167|2163|Monday|1941Q2|N|N|N|2430147|2430297|2429797|2430072|N|N|N|N|N| +2430163|AAAAAAAADNEBFCAA|1941-06-17|497|2164|167|1941|2|6|17|2|1941|167|2164|Tuesday|1941Q2|N|N|N|2430147|2430297|2429798|2430073|N|N|N|N|N| +2430164|AAAAAAAAENEBFCAA|1941-06-18|497|2164|167|1941|3|6|18|2|1941|167|2164|Wednesday|1941Q2|N|N|N|2430147|2430297|2429799|2430074|N|N|N|N|N| +2430165|AAAAAAAAFNEBFCAA|1941-06-19|497|2164|167|1941|4|6|19|2|1941|167|2164|Thursday|1941Q2|N|N|N|2430147|2430297|2429800|2430075|N|N|N|N|N| +2430166|AAAAAAAAGNEBFCAA|1941-06-20|497|2164|167|1941|5|6|20|2|1941|167|2164|Friday|1941Q2|N|Y|N|2430147|2430297|2429801|2430076|N|N|N|N|N| +2430167|AAAAAAAAHNEBFCAA|1941-06-21|497|2164|167|1941|6|6|21|2|1941|167|2164|Saturday|1941Q2|N|Y|N|2430147|2430297|2429802|2430077|N|N|N|N|N| +2430168|AAAAAAAAINEBFCAA|1941-06-22|497|2164|167|1941|0|6|22|2|1941|167|2164|Sunday|1941Q2|N|N|N|2430147|2430297|2429803|2430078|N|N|N|N|N| +2430169|AAAAAAAAJNEBFCAA|1941-06-23|497|2164|167|1941|1|6|23|2|1941|167|2164|Monday|1941Q2|N|N|N|2430147|2430297|2429804|2430079|N|N|N|N|N| +2430170|AAAAAAAAKNEBFCAA|1941-06-24|497|2165|167|1941|2|6|24|2|1941|167|2165|Tuesday|1941Q2|N|N|N|2430147|2430297|2429805|2430080|N|N|N|N|N| +2430171|AAAAAAAALNEBFCAA|1941-06-25|497|2165|167|1941|3|6|25|2|1941|167|2165|Wednesday|1941Q2|N|N|N|2430147|2430297|2429806|2430081|N|N|N|N|N| +2430172|AAAAAAAAMNEBFCAA|1941-06-26|497|2165|167|1941|4|6|26|2|1941|167|2165|Thursday|1941Q2|N|N|N|2430147|2430297|2429807|2430082|N|N|N|N|N| +2430173|AAAAAAAANNEBFCAA|1941-06-27|497|2165|167|1941|5|6|27|2|1941|167|2165|Friday|1941Q2|N|Y|N|2430147|2430297|2429808|2430083|N|N|N|N|N| +2430174|AAAAAAAAONEBFCAA|1941-06-28|497|2165|167|1941|6|6|28|2|1941|167|2165|Saturday|1941Q2|N|Y|N|2430147|2430297|2429809|2430084|N|N|N|N|N| +2430175|AAAAAAAAPNEBFCAA|1941-06-29|497|2165|167|1941|0|6|29|2|1941|167|2165|Sunday|1941Q2|N|N|N|2430147|2430297|2429810|2430085|N|N|N|N|N| +2430176|AAAAAAAAAOEBFCAA|1941-06-30|497|2165|167|1941|1|6|30|2|1941|167|2165|Monday|1941Q2|N|N|N|2430147|2430297|2429811|2430086|N|N|N|N|N| +2430177|AAAAAAAABOEBFCAA|1941-07-01|498|2166|167|1941|2|7|1|2|1941|167|2166|Tuesday|1941Q2|N|N|N|2430177|2430357|2429812|2430086|N|N|N|N|N| +2430178|AAAAAAAACOEBFCAA|1941-07-02|498|2166|167|1941|3|7|2|3|1941|167|2166|Wednesday|1941Q3|N|N|N|2430177|2430357|2429813|2430087|N|N|N|N|N| +2430179|AAAAAAAADOEBFCAA|1941-07-03|498|2166|167|1941|4|7|3|3|1941|167|2166|Thursday|1941Q3|N|N|N|2430177|2430357|2429814|2430088|N|N|N|N|N| +2430180|AAAAAAAAEOEBFCAA|1941-07-04|498|2166|167|1941|5|7|4|3|1941|167|2166|Friday|1941Q3|N|Y|N|2430177|2430357|2429815|2430089|N|N|N|N|N| +2430181|AAAAAAAAFOEBFCAA|1941-07-05|498|2166|167|1941|6|7|5|3|1941|167|2166|Saturday|1941Q3|Y|Y|N|2430177|2430357|2429816|2430090|N|N|N|N|N| +2430182|AAAAAAAAGOEBFCAA|1941-07-06|498|2166|167|1941|0|7|6|3|1941|167|2166|Sunday|1941Q3|N|N|Y|2430177|2430357|2429817|2430091|N|N|N|N|N| +2430183|AAAAAAAAHOEBFCAA|1941-07-07|498|2166|167|1941|1|7|7|3|1941|167|2166|Monday|1941Q3|N|N|N|2430177|2430357|2429818|2430092|N|N|N|N|N| +2430184|AAAAAAAAIOEBFCAA|1941-07-08|498|2167|167|1941|2|7|8|3|1941|167|2167|Tuesday|1941Q3|N|N|N|2430177|2430357|2429819|2430093|N|N|N|N|N| +2430185|AAAAAAAAJOEBFCAA|1941-07-09|498|2167|167|1941|3|7|9|3|1941|167|2167|Wednesday|1941Q3|N|N|N|2430177|2430357|2429820|2430094|N|N|N|N|N| +2430186|AAAAAAAAKOEBFCAA|1941-07-10|498|2167|167|1941|4|7|10|3|1941|167|2167|Thursday|1941Q3|N|N|N|2430177|2430357|2429821|2430095|N|N|N|N|N| +2430187|AAAAAAAALOEBFCAA|1941-07-11|498|2167|167|1941|5|7|11|3|1941|167|2167|Friday|1941Q3|N|Y|N|2430177|2430357|2429822|2430096|N|N|N|N|N| +2430188|AAAAAAAAMOEBFCAA|1941-07-12|498|2167|167|1941|6|7|12|3|1941|167|2167|Saturday|1941Q3|N|Y|N|2430177|2430357|2429823|2430097|N|N|N|N|N| +2430189|AAAAAAAANOEBFCAA|1941-07-13|498|2167|167|1941|0|7|13|3|1941|167|2167|Sunday|1941Q3|N|N|N|2430177|2430357|2429824|2430098|N|N|N|N|N| +2430190|AAAAAAAAOOEBFCAA|1941-07-14|498|2167|167|1941|1|7|14|3|1941|167|2167|Monday|1941Q3|N|N|N|2430177|2430357|2429825|2430099|N|N|N|N|N| +2430191|AAAAAAAAPOEBFCAA|1941-07-15|498|2168|167|1941|2|7|15|3|1941|167|2168|Tuesday|1941Q3|N|N|N|2430177|2430357|2429826|2430100|N|N|N|N|N| +2430192|AAAAAAAAAPEBFCAA|1941-07-16|498|2168|167|1941|3|7|16|3|1941|167|2168|Wednesday|1941Q3|N|N|N|2430177|2430357|2429827|2430101|N|N|N|N|N| +2430193|AAAAAAAABPEBFCAA|1941-07-17|498|2168|167|1941|4|7|17|3|1941|167|2168|Thursday|1941Q3|N|N|N|2430177|2430357|2429828|2430102|N|N|N|N|N| +2430194|AAAAAAAACPEBFCAA|1941-07-18|498|2168|167|1941|5|7|18|3|1941|167|2168|Friday|1941Q3|N|Y|N|2430177|2430357|2429829|2430103|N|N|N|N|N| +2430195|AAAAAAAADPEBFCAA|1941-07-19|498|2168|167|1941|6|7|19|3|1941|167|2168|Saturday|1941Q3|N|Y|N|2430177|2430357|2429830|2430104|N|N|N|N|N| +2430196|AAAAAAAAEPEBFCAA|1941-07-20|498|2168|167|1941|0|7|20|3|1941|167|2168|Sunday|1941Q3|N|N|N|2430177|2430357|2429831|2430105|N|N|N|N|N| +2430197|AAAAAAAAFPEBFCAA|1941-07-21|498|2168|167|1941|1|7|21|3|1941|167|2168|Monday|1941Q3|N|N|N|2430177|2430357|2429832|2430106|N|N|N|N|N| +2430198|AAAAAAAAGPEBFCAA|1941-07-22|498|2169|167|1941|2|7|22|3|1941|167|2169|Tuesday|1941Q3|N|N|N|2430177|2430357|2429833|2430107|N|N|N|N|N| +2430199|AAAAAAAAHPEBFCAA|1941-07-23|498|2169|167|1941|3|7|23|3|1941|167|2169|Wednesday|1941Q3|N|N|N|2430177|2430357|2429834|2430108|N|N|N|N|N| +2430200|AAAAAAAAIPEBFCAA|1941-07-24|498|2169|167|1941|4|7|24|3|1941|167|2169|Thursday|1941Q3|N|N|N|2430177|2430357|2429835|2430109|N|N|N|N|N| +2430201|AAAAAAAAJPEBFCAA|1941-07-25|498|2169|167|1941|5|7|25|3|1941|167|2169|Friday|1941Q3|N|Y|N|2430177|2430357|2429836|2430110|N|N|N|N|N| +2430202|AAAAAAAAKPEBFCAA|1941-07-26|498|2169|167|1941|6|7|26|3|1941|167|2169|Saturday|1941Q3|N|Y|N|2430177|2430357|2429837|2430111|N|N|N|N|N| +2430203|AAAAAAAALPEBFCAA|1941-07-27|498|2169|167|1941|0|7|27|3|1941|167|2169|Sunday|1941Q3|N|N|N|2430177|2430357|2429838|2430112|N|N|N|N|N| +2430204|AAAAAAAAMPEBFCAA|1941-07-28|498|2169|167|1941|1|7|28|3|1941|167|2169|Monday|1941Q3|N|N|N|2430177|2430357|2429839|2430113|N|N|N|N|N| +2430205|AAAAAAAANPEBFCAA|1941-07-29|498|2170|167|1941|2|7|29|3|1941|167|2170|Tuesday|1941Q3|N|N|N|2430177|2430357|2429840|2430114|N|N|N|N|N| +2430206|AAAAAAAAOPEBFCAA|1941-07-30|498|2170|167|1941|3|7|30|3|1941|167|2170|Wednesday|1941Q3|N|N|N|2430177|2430357|2429841|2430115|N|N|N|N|N| +2430207|AAAAAAAAPPEBFCAA|1941-07-31|498|2170|167|1941|4|7|31|3|1941|167|2170|Thursday|1941Q3|N|N|N|2430177|2430357|2429842|2430116|N|N|N|N|N| +2430208|AAAAAAAAAAFBFCAA|1941-08-01|499|2170|167|1941|5|8|1|3|1941|167|2170|Friday|1941Q3|N|Y|N|2430208|2430419|2429843|2430117|N|N|N|N|N| +2430209|AAAAAAAABAFBFCAA|1941-08-02|499|2170|167|1941|6|8|2|3|1941|167|2170|Saturday|1941Q3|N|Y|N|2430208|2430419|2429844|2430118|N|N|N|N|N| +2430210|AAAAAAAACAFBFCAA|1941-08-03|499|2170|167|1941|0|8|3|3|1941|167|2170|Sunday|1941Q3|N|N|N|2430208|2430419|2429845|2430119|N|N|N|N|N| +2430211|AAAAAAAADAFBFCAA|1941-08-04|499|2170|167|1941|1|8|4|3|1941|167|2170|Monday|1941Q3|N|N|N|2430208|2430419|2429846|2430120|N|N|N|N|N| +2430212|AAAAAAAAEAFBFCAA|1941-08-05|499|2171|167|1941|2|8|5|3|1941|167|2171|Tuesday|1941Q3|N|N|N|2430208|2430419|2429847|2430121|N|N|N|N|N| +2430213|AAAAAAAAFAFBFCAA|1941-08-06|499|2171|167|1941|3|8|6|3|1941|167|2171|Wednesday|1941Q3|N|N|N|2430208|2430419|2429848|2430122|N|N|N|N|N| +2430214|AAAAAAAAGAFBFCAA|1941-08-07|499|2171|167|1941|4|8|7|3|1941|167|2171|Thursday|1941Q3|N|N|N|2430208|2430419|2429849|2430123|N|N|N|N|N| +2430215|AAAAAAAAHAFBFCAA|1941-08-08|499|2171|167|1941|5|8|8|3|1941|167|2171|Friday|1941Q3|N|Y|N|2430208|2430419|2429850|2430124|N|N|N|N|N| +2430216|AAAAAAAAIAFBFCAA|1941-08-09|499|2171|167|1941|6|8|9|3|1941|167|2171|Saturday|1941Q3|N|Y|N|2430208|2430419|2429851|2430125|N|N|N|N|N| +2430217|AAAAAAAAJAFBFCAA|1941-08-10|499|2171|167|1941|0|8|10|3|1941|167|2171|Sunday|1941Q3|N|N|N|2430208|2430419|2429852|2430126|N|N|N|N|N| +2430218|AAAAAAAAKAFBFCAA|1941-08-11|499|2171|167|1941|1|8|11|3|1941|167|2171|Monday|1941Q3|N|N|N|2430208|2430419|2429853|2430127|N|N|N|N|N| +2430219|AAAAAAAALAFBFCAA|1941-08-12|499|2172|167|1941|2|8|12|3|1941|167|2172|Tuesday|1941Q3|N|N|N|2430208|2430419|2429854|2430128|N|N|N|N|N| +2430220|AAAAAAAAMAFBFCAA|1941-08-13|499|2172|167|1941|3|8|13|3|1941|167|2172|Wednesday|1941Q3|N|N|N|2430208|2430419|2429855|2430129|N|N|N|N|N| +2430221|AAAAAAAANAFBFCAA|1941-08-14|499|2172|167|1941|4|8|14|3|1941|167|2172|Thursday|1941Q3|N|N|N|2430208|2430419|2429856|2430130|N|N|N|N|N| +2430222|AAAAAAAAOAFBFCAA|1941-08-15|499|2172|167|1941|5|8|15|3|1941|167|2172|Friday|1941Q3|N|Y|N|2430208|2430419|2429857|2430131|N|N|N|N|N| +2430223|AAAAAAAAPAFBFCAA|1941-08-16|499|2172|167|1941|6|8|16|3|1941|167|2172|Saturday|1941Q3|N|Y|N|2430208|2430419|2429858|2430132|N|N|N|N|N| +2430224|AAAAAAAAABFBFCAA|1941-08-17|499|2172|167|1941|0|8|17|3|1941|167|2172|Sunday|1941Q3|N|N|N|2430208|2430419|2429859|2430133|N|N|N|N|N| +2430225|AAAAAAAABBFBFCAA|1941-08-18|499|2172|167|1941|1|8|18|3|1941|167|2172|Monday|1941Q3|N|N|N|2430208|2430419|2429860|2430134|N|N|N|N|N| +2430226|AAAAAAAACBFBFCAA|1941-08-19|499|2173|167|1941|2|8|19|3|1941|167|2173|Tuesday|1941Q3|N|N|N|2430208|2430419|2429861|2430135|N|N|N|N|N| +2430227|AAAAAAAADBFBFCAA|1941-08-20|499|2173|167|1941|3|8|20|3|1941|167|2173|Wednesday|1941Q3|N|N|N|2430208|2430419|2429862|2430136|N|N|N|N|N| +2430228|AAAAAAAAEBFBFCAA|1941-08-21|499|2173|167|1941|4|8|21|3|1941|167|2173|Thursday|1941Q3|N|N|N|2430208|2430419|2429863|2430137|N|N|N|N|N| +2430229|AAAAAAAAFBFBFCAA|1941-08-22|499|2173|167|1941|5|8|22|3|1941|167|2173|Friday|1941Q3|N|Y|N|2430208|2430419|2429864|2430138|N|N|N|N|N| +2430230|AAAAAAAAGBFBFCAA|1941-08-23|499|2173|167|1941|6|8|23|3|1941|167|2173|Saturday|1941Q3|N|Y|N|2430208|2430419|2429865|2430139|N|N|N|N|N| +2430231|AAAAAAAAHBFBFCAA|1941-08-24|499|2173|167|1941|0|8|24|3|1941|167|2173|Sunday|1941Q3|N|N|N|2430208|2430419|2429866|2430140|N|N|N|N|N| +2430232|AAAAAAAAIBFBFCAA|1941-08-25|499|2173|167|1941|1|8|25|3|1941|167|2173|Monday|1941Q3|N|N|N|2430208|2430419|2429867|2430141|N|N|N|N|N| +2430233|AAAAAAAAJBFBFCAA|1941-08-26|499|2174|167|1941|2|8|26|3|1941|167|2174|Tuesday|1941Q3|N|N|N|2430208|2430419|2429868|2430142|N|N|N|N|N| +2430234|AAAAAAAAKBFBFCAA|1941-08-27|499|2174|167|1941|3|8|27|3|1941|167|2174|Wednesday|1941Q3|N|N|N|2430208|2430419|2429869|2430143|N|N|N|N|N| +2430235|AAAAAAAALBFBFCAA|1941-08-28|499|2174|167|1941|4|8|28|3|1941|167|2174|Thursday|1941Q3|N|N|N|2430208|2430419|2429870|2430144|N|N|N|N|N| +2430236|AAAAAAAAMBFBFCAA|1941-08-29|499|2174|167|1941|5|8|29|3|1941|167|2174|Friday|1941Q3|N|Y|N|2430208|2430419|2429871|2430145|N|N|N|N|N| +2430237|AAAAAAAANBFBFCAA|1941-08-30|499|2174|167|1941|6|8|30|3|1941|167|2174|Saturday|1941Q3|N|Y|N|2430208|2430419|2429872|2430146|N|N|N|N|N| +2430238|AAAAAAAAOBFBFCAA|1941-08-31|499|2174|167|1941|0|8|31|3|1941|167|2174|Sunday|1941Q3|N|N|N|2430208|2430419|2429873|2430147|N|N|N|N|N| +2430239|AAAAAAAAPBFBFCAA|1941-09-01|500|2174|168|1941|1|9|1|3|1941|168|2174|Monday|1941Q3|N|N|N|2430239|2430481|2429874|2430148|N|N|N|N|N| +2430240|AAAAAAAAACFBFCAA|1941-09-02|500|2175|168|1941|2|9|2|3|1941|168|2175|Tuesday|1941Q3|N|N|N|2430239|2430481|2429875|2430149|N|N|N|N|N| +2430241|AAAAAAAABCFBFCAA|1941-09-03|500|2175|168|1941|3|9|3|3|1941|168|2175|Wednesday|1941Q3|N|N|N|2430239|2430481|2429876|2430150|N|N|N|N|N| +2430242|AAAAAAAACCFBFCAA|1941-09-04|500|2175|168|1941|4|9|4|3|1941|168|2175|Thursday|1941Q3|N|N|N|2430239|2430481|2429877|2430151|N|N|N|N|N| +2430243|AAAAAAAADCFBFCAA|1941-09-05|500|2175|168|1941|5|9|5|3|1941|168|2175|Friday|1941Q3|N|Y|N|2430239|2430481|2429878|2430152|N|N|N|N|N| +2430244|AAAAAAAAECFBFCAA|1941-09-06|500|2175|168|1941|6|9|6|3|1941|168|2175|Saturday|1941Q3|N|Y|N|2430239|2430481|2429879|2430153|N|N|N|N|N| +2430245|AAAAAAAAFCFBFCAA|1941-09-07|500|2175|168|1941|0|9|7|3|1941|168|2175|Sunday|1941Q3|N|N|N|2430239|2430481|2429880|2430154|N|N|N|N|N| +2430246|AAAAAAAAGCFBFCAA|1941-09-08|500|2175|168|1941|1|9|8|3|1941|168|2175|Monday|1941Q3|N|N|N|2430239|2430481|2429881|2430155|N|N|N|N|N| +2430247|AAAAAAAAHCFBFCAA|1941-09-09|500|2176|168|1941|2|9|9|3|1941|168|2176|Tuesday|1941Q3|N|N|N|2430239|2430481|2429882|2430156|N|N|N|N|N| +2430248|AAAAAAAAICFBFCAA|1941-09-10|500|2176|168|1941|3|9|10|3|1941|168|2176|Wednesday|1941Q3|N|N|N|2430239|2430481|2429883|2430157|N|N|N|N|N| +2430249|AAAAAAAAJCFBFCAA|1941-09-11|500|2176|168|1941|4|9|11|3|1941|168|2176|Thursday|1941Q3|N|N|N|2430239|2430481|2429884|2430158|N|N|N|N|N| +2430250|AAAAAAAAKCFBFCAA|1941-09-12|500|2176|168|1941|5|9|12|3|1941|168|2176|Friday|1941Q3|N|Y|N|2430239|2430481|2429885|2430159|N|N|N|N|N| +2430251|AAAAAAAALCFBFCAA|1941-09-13|500|2176|168|1941|6|9|13|3|1941|168|2176|Saturday|1941Q3|N|Y|N|2430239|2430481|2429886|2430160|N|N|N|N|N| +2430252|AAAAAAAAMCFBFCAA|1941-09-14|500|2176|168|1941|0|9|14|3|1941|168|2176|Sunday|1941Q3|N|N|N|2430239|2430481|2429887|2430161|N|N|N|N|N| +2430253|AAAAAAAANCFBFCAA|1941-09-15|500|2176|168|1941|1|9|15|3|1941|168|2176|Monday|1941Q3|N|N|N|2430239|2430481|2429888|2430162|N|N|N|N|N| +2430254|AAAAAAAAOCFBFCAA|1941-09-16|500|2177|168|1941|2|9|16|3|1941|168|2177|Tuesday|1941Q3|N|N|N|2430239|2430481|2429889|2430163|N|N|N|N|N| +2430255|AAAAAAAAPCFBFCAA|1941-09-17|500|2177|168|1941|3|9|17|3|1941|168|2177|Wednesday|1941Q3|N|N|N|2430239|2430481|2429890|2430164|N|N|N|N|N| +2430256|AAAAAAAAADFBFCAA|1941-09-18|500|2177|168|1941|4|9|18|3|1941|168|2177|Thursday|1941Q3|N|N|N|2430239|2430481|2429891|2430165|N|N|N|N|N| +2430257|AAAAAAAABDFBFCAA|1941-09-19|500|2177|168|1941|5|9|19|3|1941|168|2177|Friday|1941Q3|N|Y|N|2430239|2430481|2429892|2430166|N|N|N|N|N| +2430258|AAAAAAAACDFBFCAA|1941-09-20|500|2177|168|1941|6|9|20|3|1941|168|2177|Saturday|1941Q3|N|Y|N|2430239|2430481|2429893|2430167|N|N|N|N|N| +2430259|AAAAAAAADDFBFCAA|1941-09-21|500|2177|168|1941|0|9|21|3|1941|168|2177|Sunday|1941Q3|N|N|N|2430239|2430481|2429894|2430168|N|N|N|N|N| +2430260|AAAAAAAAEDFBFCAA|1941-09-22|500|2177|168|1941|1|9|22|3|1941|168|2177|Monday|1941Q3|N|N|N|2430239|2430481|2429895|2430169|N|N|N|N|N| +2430261|AAAAAAAAFDFBFCAA|1941-09-23|500|2178|168|1941|2|9|23|3|1941|168|2178|Tuesday|1941Q3|N|N|N|2430239|2430481|2429896|2430170|N|N|N|N|N| +2430262|AAAAAAAAGDFBFCAA|1941-09-24|500|2178|168|1941|3|9|24|3|1941|168|2178|Wednesday|1941Q3|N|N|N|2430239|2430481|2429897|2430171|N|N|N|N|N| +2430263|AAAAAAAAHDFBFCAA|1941-09-25|500|2178|168|1941|4|9|25|3|1941|168|2178|Thursday|1941Q3|N|N|N|2430239|2430481|2429898|2430172|N|N|N|N|N| +2430264|AAAAAAAAIDFBFCAA|1941-09-26|500|2178|168|1941|5|9|26|3|1941|168|2178|Friday|1941Q3|N|Y|N|2430239|2430481|2429899|2430173|N|N|N|N|N| +2430265|AAAAAAAAJDFBFCAA|1941-09-27|500|2178|168|1941|6|9|27|3|1941|168|2178|Saturday|1941Q3|N|Y|N|2430239|2430481|2429900|2430174|N|N|N|N|N| +2430266|AAAAAAAAKDFBFCAA|1941-09-28|500|2178|168|1941|0|9|28|3|1941|168|2178|Sunday|1941Q3|N|N|N|2430239|2430481|2429901|2430175|N|N|N|N|N| +2430267|AAAAAAAALDFBFCAA|1941-09-29|500|2178|168|1941|1|9|29|3|1941|168|2178|Monday|1941Q3|N|N|N|2430239|2430481|2429902|2430176|N|N|N|N|N| +2430268|AAAAAAAAMDFBFCAA|1941-09-30|500|2179|168|1941|2|9|30|3|1941|168|2179|Tuesday|1941Q3|N|N|N|2430239|2430481|2429903|2430177|N|N|N|N|N| +2430269|AAAAAAAANDFBFCAA|1941-10-01|501|2179|168|1941|3|10|1|3|1941|168|2179|Wednesday|1941Q3|N|N|N|2430269|2430541|2429904|2430177|N|N|N|N|N| +2430270|AAAAAAAAODFBFCAA|1941-10-02|501|2179|168|1941|4|10|2|4|1941|168|2179|Thursday|1941Q4|N|N|N|2430269|2430541|2429905|2430178|N|N|N|N|N| +2430271|AAAAAAAAPDFBFCAA|1941-10-03|501|2179|168|1941|5|10|3|4|1941|168|2179|Friday|1941Q4|N|Y|N|2430269|2430541|2429906|2430179|N|N|N|N|N| +2430272|AAAAAAAAAEFBFCAA|1941-10-04|501|2179|168|1941|6|10|4|4|1941|168|2179|Saturday|1941Q4|N|Y|N|2430269|2430541|2429907|2430180|N|N|N|N|N| +2430273|AAAAAAAABEFBFCAA|1941-10-05|501|2179|168|1941|0|10|5|4|1941|168|2179|Sunday|1941Q4|N|N|N|2430269|2430541|2429908|2430181|N|N|N|N|N| +2430274|AAAAAAAACEFBFCAA|1941-10-06|501|2179|168|1941|1|10|6|4|1941|168|2179|Monday|1941Q4|N|N|N|2430269|2430541|2429909|2430182|N|N|N|N|N| +2430275|AAAAAAAADEFBFCAA|1941-10-07|501|2180|168|1941|2|10|7|4|1941|168|2180|Tuesday|1941Q4|N|N|N|2430269|2430541|2429910|2430183|N|N|N|N|N| +2430276|AAAAAAAAEEFBFCAA|1941-10-08|501|2180|168|1941|3|10|8|4|1941|168|2180|Wednesday|1941Q4|N|N|N|2430269|2430541|2429911|2430184|N|N|N|N|N| +2430277|AAAAAAAAFEFBFCAA|1941-10-09|501|2180|168|1941|4|10|9|4|1941|168|2180|Thursday|1941Q4|N|N|N|2430269|2430541|2429912|2430185|N|N|N|N|N| +2430278|AAAAAAAAGEFBFCAA|1941-10-10|501|2180|168|1941|5|10|10|4|1941|168|2180|Friday|1941Q4|N|Y|N|2430269|2430541|2429913|2430186|N|N|N|N|N| +2430279|AAAAAAAAHEFBFCAA|1941-10-11|501|2180|168|1941|6|10|11|4|1941|168|2180|Saturday|1941Q4|N|Y|N|2430269|2430541|2429914|2430187|N|N|N|N|N| +2430280|AAAAAAAAIEFBFCAA|1941-10-12|501|2180|168|1941|0|10|12|4|1941|168|2180|Sunday|1941Q4|N|N|N|2430269|2430541|2429915|2430188|N|N|N|N|N| +2430281|AAAAAAAAJEFBFCAA|1941-10-13|501|2180|168|1941|1|10|13|4|1941|168|2180|Monday|1941Q4|N|N|N|2430269|2430541|2429916|2430189|N|N|N|N|N| +2430282|AAAAAAAAKEFBFCAA|1941-10-14|501|2181|168|1941|2|10|14|4|1941|168|2181|Tuesday|1941Q4|N|N|N|2430269|2430541|2429917|2430190|N|N|N|N|N| +2430283|AAAAAAAALEFBFCAA|1941-10-15|501|2181|168|1941|3|10|15|4|1941|168|2181|Wednesday|1941Q4|N|N|N|2430269|2430541|2429918|2430191|N|N|N|N|N| +2430284|AAAAAAAAMEFBFCAA|1941-10-16|501|2181|168|1941|4|10|16|4|1941|168|2181|Thursday|1941Q4|N|N|N|2430269|2430541|2429919|2430192|N|N|N|N|N| +2430285|AAAAAAAANEFBFCAA|1941-10-17|501|2181|168|1941|5|10|17|4|1941|168|2181|Friday|1941Q4|N|Y|N|2430269|2430541|2429920|2430193|N|N|N|N|N| +2430286|AAAAAAAAOEFBFCAA|1941-10-18|501|2181|168|1941|6|10|18|4|1941|168|2181|Saturday|1941Q4|N|Y|N|2430269|2430541|2429921|2430194|N|N|N|N|N| +2430287|AAAAAAAAPEFBFCAA|1941-10-19|501|2181|168|1941|0|10|19|4|1941|168|2181|Sunday|1941Q4|N|N|N|2430269|2430541|2429922|2430195|N|N|N|N|N| +2430288|AAAAAAAAAFFBFCAA|1941-10-20|501|2181|168|1941|1|10|20|4|1941|168|2181|Monday|1941Q4|N|N|N|2430269|2430541|2429923|2430196|N|N|N|N|N| +2430289|AAAAAAAABFFBFCAA|1941-10-21|501|2182|168|1941|2|10|21|4|1941|168|2182|Tuesday|1941Q4|N|N|N|2430269|2430541|2429924|2430197|N|N|N|N|N| +2430290|AAAAAAAACFFBFCAA|1941-10-22|501|2182|168|1941|3|10|22|4|1941|168|2182|Wednesday|1941Q4|N|N|N|2430269|2430541|2429925|2430198|N|N|N|N|N| +2430291|AAAAAAAADFFBFCAA|1941-10-23|501|2182|168|1941|4|10|23|4|1941|168|2182|Thursday|1941Q4|N|N|N|2430269|2430541|2429926|2430199|N|N|N|N|N| +2430292|AAAAAAAAEFFBFCAA|1941-10-24|501|2182|168|1941|5|10|24|4|1941|168|2182|Friday|1941Q4|N|Y|N|2430269|2430541|2429927|2430200|N|N|N|N|N| +2430293|AAAAAAAAFFFBFCAA|1941-10-25|501|2182|168|1941|6|10|25|4|1941|168|2182|Saturday|1941Q4|N|Y|N|2430269|2430541|2429928|2430201|N|N|N|N|N| +2430294|AAAAAAAAGFFBFCAA|1941-10-26|501|2182|168|1941|0|10|26|4|1941|168|2182|Sunday|1941Q4|N|N|N|2430269|2430541|2429929|2430202|N|N|N|N|N| +2430295|AAAAAAAAHFFBFCAA|1941-10-27|501|2182|168|1941|1|10|27|4|1941|168|2182|Monday|1941Q4|N|N|N|2430269|2430541|2429930|2430203|N|N|N|N|N| +2430296|AAAAAAAAIFFBFCAA|1941-10-28|501|2183|168|1941|2|10|28|4|1941|168|2183|Tuesday|1941Q4|N|N|N|2430269|2430541|2429931|2430204|N|N|N|N|N| +2430297|AAAAAAAAJFFBFCAA|1941-10-29|501|2183|168|1941|3|10|29|4|1941|168|2183|Wednesday|1941Q4|N|N|N|2430269|2430541|2429932|2430205|N|N|N|N|N| +2430298|AAAAAAAAKFFBFCAA|1941-10-30|501|2183|168|1941|4|10|30|4|1941|168|2183|Thursday|1941Q4|N|N|N|2430269|2430541|2429933|2430206|N|N|N|N|N| +2430299|AAAAAAAALFFBFCAA|1941-10-31|501|2183|168|1941|5|10|31|4|1941|168|2183|Friday|1941Q4|N|Y|N|2430269|2430541|2429934|2430207|N|N|N|N|N| +2430300|AAAAAAAAMFFBFCAA|1941-11-01|502|2183|168|1941|6|11|1|4|1941|168|2183|Saturday|1941Q4|N|Y|N|2430300|2430603|2429935|2430208|N|N|N|N|N| +2430301|AAAAAAAANFFBFCAA|1941-11-02|502|2183|168|1941|0|11|2|4|1941|168|2183|Sunday|1941Q4|N|N|N|2430300|2430603|2429936|2430209|N|N|N|N|N| +2430302|AAAAAAAAOFFBFCAA|1941-11-03|502|2183|168|1941|1|11|3|4|1941|168|2183|Monday|1941Q4|N|N|N|2430300|2430603|2429937|2430210|N|N|N|N|N| +2430303|AAAAAAAAPFFBFCAA|1941-11-04|502|2184|168|1941|2|11|4|4|1941|168|2184|Tuesday|1941Q4|N|N|N|2430300|2430603|2429938|2430211|N|N|N|N|N| +2430304|AAAAAAAAAGFBFCAA|1941-11-05|502|2184|168|1941|3|11|5|4|1941|168|2184|Wednesday|1941Q4|N|N|N|2430300|2430603|2429939|2430212|N|N|N|N|N| +2430305|AAAAAAAABGFBFCAA|1941-11-06|502|2184|168|1941|4|11|6|4|1941|168|2184|Thursday|1941Q4|N|N|N|2430300|2430603|2429940|2430213|N|N|N|N|N| +2430306|AAAAAAAACGFBFCAA|1941-11-07|502|2184|168|1941|5|11|7|4|1941|168|2184|Friday|1941Q4|N|Y|N|2430300|2430603|2429941|2430214|N|N|N|N|N| +2430307|AAAAAAAADGFBFCAA|1941-11-08|502|2184|168|1941|6|11|8|4|1941|168|2184|Saturday|1941Q4|N|Y|N|2430300|2430603|2429942|2430215|N|N|N|N|N| +2430308|AAAAAAAAEGFBFCAA|1941-11-09|502|2184|168|1941|0|11|9|4|1941|168|2184|Sunday|1941Q4|N|N|N|2430300|2430603|2429943|2430216|N|N|N|N|N| +2430309|AAAAAAAAFGFBFCAA|1941-11-10|502|2184|168|1941|1|11|10|4|1941|168|2184|Monday|1941Q4|N|N|N|2430300|2430603|2429944|2430217|N|N|N|N|N| +2430310|AAAAAAAAGGFBFCAA|1941-11-11|502|2185|168|1941|2|11|11|4|1941|168|2185|Tuesday|1941Q4|N|N|N|2430300|2430603|2429945|2430218|N|N|N|N|N| +2430311|AAAAAAAAHGFBFCAA|1941-11-12|502|2185|168|1941|3|11|12|4|1941|168|2185|Wednesday|1941Q4|N|N|N|2430300|2430603|2429946|2430219|N|N|N|N|N| +2430312|AAAAAAAAIGFBFCAA|1941-11-13|502|2185|168|1941|4|11|13|4|1941|168|2185|Thursday|1941Q4|N|N|N|2430300|2430603|2429947|2430220|N|N|N|N|N| +2430313|AAAAAAAAJGFBFCAA|1941-11-14|502|2185|168|1941|5|11|14|4|1941|168|2185|Friday|1941Q4|N|Y|N|2430300|2430603|2429948|2430221|N|N|N|N|N| +2430314|AAAAAAAAKGFBFCAA|1941-11-15|502|2185|168|1941|6|11|15|4|1941|168|2185|Saturday|1941Q4|N|Y|N|2430300|2430603|2429949|2430222|N|N|N|N|N| +2430315|AAAAAAAALGFBFCAA|1941-11-16|502|2185|168|1941|0|11|16|4|1941|168|2185|Sunday|1941Q4|N|N|N|2430300|2430603|2429950|2430223|N|N|N|N|N| +2430316|AAAAAAAAMGFBFCAA|1941-11-17|502|2185|168|1941|1|11|17|4|1941|168|2185|Monday|1941Q4|N|N|N|2430300|2430603|2429951|2430224|N|N|N|N|N| +2430317|AAAAAAAANGFBFCAA|1941-11-18|502|2186|168|1941|2|11|18|4|1941|168|2186|Tuesday|1941Q4|N|N|N|2430300|2430603|2429952|2430225|N|N|N|N|N| +2430318|AAAAAAAAOGFBFCAA|1941-11-19|502|2186|168|1941|3|11|19|4|1941|168|2186|Wednesday|1941Q4|N|N|N|2430300|2430603|2429953|2430226|N|N|N|N|N| +2430319|AAAAAAAAPGFBFCAA|1941-11-20|502|2186|168|1941|4|11|20|4|1941|168|2186|Thursday|1941Q4|N|N|N|2430300|2430603|2429954|2430227|N|N|N|N|N| +2430320|AAAAAAAAAHFBFCAA|1941-11-21|502|2186|168|1941|5|11|21|4|1941|168|2186|Friday|1941Q4|N|Y|N|2430300|2430603|2429955|2430228|N|N|N|N|N| +2430321|AAAAAAAABHFBFCAA|1941-11-22|502|2186|168|1941|6|11|22|4|1941|168|2186|Saturday|1941Q4|N|Y|N|2430300|2430603|2429956|2430229|N|N|N|N|N| +2430322|AAAAAAAACHFBFCAA|1941-11-23|502|2186|168|1941|0|11|23|4|1941|168|2186|Sunday|1941Q4|N|N|N|2430300|2430603|2429957|2430230|N|N|N|N|N| +2430323|AAAAAAAADHFBFCAA|1941-11-24|502|2186|168|1941|1|11|24|4|1941|168|2186|Monday|1941Q4|N|N|N|2430300|2430603|2429958|2430231|N|N|N|N|N| +2430324|AAAAAAAAEHFBFCAA|1941-11-25|502|2187|168|1941|2|11|25|4|1941|168|2187|Tuesday|1941Q4|N|N|N|2430300|2430603|2429959|2430232|N|N|N|N|N| +2430325|AAAAAAAAFHFBFCAA|1941-11-26|502|2187|168|1941|3|11|26|4|1941|168|2187|Wednesday|1941Q4|N|N|N|2430300|2430603|2429960|2430233|N|N|N|N|N| +2430326|AAAAAAAAGHFBFCAA|1941-11-27|502|2187|168|1941|4|11|27|4|1941|168|2187|Thursday|1941Q4|N|N|N|2430300|2430603|2429961|2430234|N|N|N|N|N| +2430327|AAAAAAAAHHFBFCAA|1941-11-28|502|2187|168|1941|5|11|28|4|1941|168|2187|Friday|1941Q4|N|Y|N|2430300|2430603|2429962|2430235|N|N|N|N|N| +2430328|AAAAAAAAIHFBFCAA|1941-11-29|502|2187|168|1941|6|11|29|4|1941|168|2187|Saturday|1941Q4|N|Y|N|2430300|2430603|2429963|2430236|N|N|N|N|N| +2430329|AAAAAAAAJHFBFCAA|1941-11-30|502|2187|168|1941|0|11|30|4|1941|168|2187|Sunday|1941Q4|N|N|N|2430300|2430603|2429964|2430237|N|N|N|N|N| +2430330|AAAAAAAAKHFBFCAA|1941-12-01|503|2187|169|1941|1|12|1|4|1941|169|2187|Monday|1941Q4|N|N|N|2430330|2430663|2429965|2430238|N|N|N|N|N| +2430331|AAAAAAAALHFBFCAA|1941-12-02|503|2188|169|1941|2|12|2|4|1941|169|2188|Tuesday|1941Q4|N|N|N|2430330|2430663|2429966|2430239|N|N|N|N|N| +2430332|AAAAAAAAMHFBFCAA|1941-12-03|503|2188|169|1941|3|12|3|4|1941|169|2188|Wednesday|1941Q4|N|N|N|2430330|2430663|2429967|2430240|N|N|N|N|N| +2430333|AAAAAAAANHFBFCAA|1941-12-04|503|2188|169|1941|4|12|4|4|1941|169|2188|Thursday|1941Q4|N|N|N|2430330|2430663|2429968|2430241|N|N|N|N|N| +2430334|AAAAAAAAOHFBFCAA|1941-12-05|503|2188|169|1941|5|12|5|4|1941|169|2188|Friday|1941Q4|N|Y|N|2430330|2430663|2429969|2430242|N|N|N|N|N| +2430335|AAAAAAAAPHFBFCAA|1941-12-06|503|2188|169|1941|6|12|6|4|1941|169|2188|Saturday|1941Q4|N|Y|N|2430330|2430663|2429970|2430243|N|N|N|N|N| +2430336|AAAAAAAAAIFBFCAA|1941-12-07|503|2188|169|1941|0|12|7|4|1941|169|2188|Sunday|1941Q4|N|N|N|2430330|2430663|2429971|2430244|N|N|N|N|N| +2430337|AAAAAAAABIFBFCAA|1941-12-08|503|2188|169|1941|1|12|8|4|1941|169|2188|Monday|1941Q4|N|N|N|2430330|2430663|2429972|2430245|N|N|N|N|N| +2430338|AAAAAAAACIFBFCAA|1941-12-09|503|2189|169|1941|2|12|9|4|1941|169|2189|Tuesday|1941Q4|N|N|N|2430330|2430663|2429973|2430246|N|N|N|N|N| +2430339|AAAAAAAADIFBFCAA|1941-12-10|503|2189|169|1941|3|12|10|4|1941|169|2189|Wednesday|1941Q4|N|N|N|2430330|2430663|2429974|2430247|N|N|N|N|N| +2430340|AAAAAAAAEIFBFCAA|1941-12-11|503|2189|169|1941|4|12|11|4|1941|169|2189|Thursday|1941Q4|N|N|N|2430330|2430663|2429975|2430248|N|N|N|N|N| +2430341|AAAAAAAAFIFBFCAA|1941-12-12|503|2189|169|1941|5|12|12|4|1941|169|2189|Friday|1941Q4|N|Y|N|2430330|2430663|2429976|2430249|N|N|N|N|N| +2430342|AAAAAAAAGIFBFCAA|1941-12-13|503|2189|169|1941|6|12|13|4|1941|169|2189|Saturday|1941Q4|N|Y|N|2430330|2430663|2429977|2430250|N|N|N|N|N| +2430343|AAAAAAAAHIFBFCAA|1941-12-14|503|2189|169|1941|0|12|14|4|1941|169|2189|Sunday|1941Q4|N|N|N|2430330|2430663|2429978|2430251|N|N|N|N|N| +2430344|AAAAAAAAIIFBFCAA|1941-12-15|503|2189|169|1941|1|12|15|4|1941|169|2189|Monday|1941Q4|N|N|N|2430330|2430663|2429979|2430252|N|N|N|N|N| +2430345|AAAAAAAAJIFBFCAA|1941-12-16|503|2190|169|1941|2|12|16|4|1941|169|2190|Tuesday|1941Q4|N|N|N|2430330|2430663|2429980|2430253|N|N|N|N|N| +2430346|AAAAAAAAKIFBFCAA|1941-12-17|503|2190|169|1941|3|12|17|4|1941|169|2190|Wednesday|1941Q4|N|N|N|2430330|2430663|2429981|2430254|N|N|N|N|N| +2430347|AAAAAAAALIFBFCAA|1941-12-18|503|2190|169|1941|4|12|18|4|1941|169|2190|Thursday|1941Q4|N|N|N|2430330|2430663|2429982|2430255|N|N|N|N|N| +2430348|AAAAAAAAMIFBFCAA|1941-12-19|503|2190|169|1941|5|12|19|4|1941|169|2190|Friday|1941Q4|N|Y|N|2430330|2430663|2429983|2430256|N|N|N|N|N| +2430349|AAAAAAAANIFBFCAA|1941-12-20|503|2190|169|1941|6|12|20|4|1941|169|2190|Saturday|1941Q4|N|Y|N|2430330|2430663|2429984|2430257|N|N|N|N|N| +2430350|AAAAAAAAOIFBFCAA|1941-12-21|503|2190|169|1941|0|12|21|4|1941|169|2190|Sunday|1941Q4|N|N|N|2430330|2430663|2429985|2430258|N|N|N|N|N| +2430351|AAAAAAAAPIFBFCAA|1941-12-22|503|2190|169|1941|1|12|22|4|1941|169|2190|Monday|1941Q4|N|N|N|2430330|2430663|2429986|2430259|N|N|N|N|N| +2430352|AAAAAAAAAJFBFCAA|1941-12-23|503|2191|169|1941|2|12|23|4|1941|169|2191|Tuesday|1941Q4|N|N|N|2430330|2430663|2429987|2430260|N|N|N|N|N| +2430353|AAAAAAAABJFBFCAA|1941-12-24|503|2191|169|1941|3|12|24|4|1941|169|2191|Wednesday|1941Q4|N|N|N|2430330|2430663|2429988|2430261|N|N|N|N|N| +2430354|AAAAAAAACJFBFCAA|1941-12-25|503|2191|169|1941|4|12|25|4|1941|169|2191|Thursday|1941Q4|N|N|N|2430330|2430663|2429989|2430262|N|N|N|N|N| +2430355|AAAAAAAADJFBFCAA|1941-12-26|503|2191|169|1941|5|12|26|4|1941|169|2191|Friday|1941Q4|Y|Y|N|2430330|2430663|2429990|2430263|N|N|N|N|N| +2430356|AAAAAAAAEJFBFCAA|1941-12-27|503|2191|169|1941|6|12|27|4|1941|169|2191|Saturday|1941Q4|N|Y|Y|2430330|2430663|2429991|2430264|N|N|N|N|N| +2430357|AAAAAAAAFJFBFCAA|1941-12-28|503|2191|169|1941|0|12|28|4|1941|169|2191|Sunday|1941Q4|N|N|N|2430330|2430663|2429992|2430265|N|N|N|N|N| +2430358|AAAAAAAAGJFBFCAA|1941-12-29|503|2191|169|1941|1|12|29|4|1941|169|2191|Monday|1941Q4|N|N|N|2430330|2430663|2429993|2430266|N|N|N|N|N| +2430359|AAAAAAAAHJFBFCAA|1941-12-30|503|2192|169|1941|2|12|30|4|1941|169|2192|Tuesday|1941Q4|N|N|N|2430330|2430663|2429994|2430267|N|N|N|N|N| +2430360|AAAAAAAAIJFBFCAA|1941-12-31|503|2192|169|1941|3|12|31|4|1941|169|2192|Wednesday|1941Q4|N|N|N|2430330|2430663|2429995|2430268|N|N|N|N|N| +2430361|AAAAAAAAJJFBFCAA|1942-01-01|504|2192|169|1942|4|1|1|1|1942|169|2192|Thursday|1942Q1|Y|N|N|2430361|2430360|2429996|2430269|N|N|N|N|N| +2430362|AAAAAAAAKJFBFCAA|1942-01-02|504|2192|169|1942|5|1|2|1|1942|169|2192|Friday|1942Q1|N|Y|Y|2430361|2430360|2429997|2430270|N|N|N|N|N| +2430363|AAAAAAAALJFBFCAA|1942-01-03|504|2192|169|1942|6|1|3|1|1942|169|2192|Saturday|1942Q1|N|Y|N|2430361|2430360|2429998|2430271|N|N|N|N|N| +2430364|AAAAAAAAMJFBFCAA|1942-01-04|504|2192|169|1942|0|1|4|1|1942|169|2192|Sunday|1942Q1|N|N|N|2430361|2430360|2429999|2430272|N|N|N|N|N| +2430365|AAAAAAAANJFBFCAA|1942-01-05|504|2192|169|1942|1|1|5|1|1942|169|2192|Monday|1942Q1|N|N|N|2430361|2430360|2430000|2430273|N|N|N|N|N| +2430366|AAAAAAAAOJFBFCAA|1942-01-06|504|2193|169|1942|2|1|6|1|1942|169|2193|Tuesday|1942Q1|N|N|N|2430361|2430360|2430001|2430274|N|N|N|N|N| +2430367|AAAAAAAAPJFBFCAA|1942-01-07|504|2193|169|1942|3|1|7|1|1942|169|2193|Wednesday|1942Q1|N|N|N|2430361|2430360|2430002|2430275|N|N|N|N|N| +2430368|AAAAAAAAAKFBFCAA|1942-01-08|504|2193|169|1942|4|1|8|1|1942|169|2193|Thursday|1942Q1|N|N|N|2430361|2430360|2430003|2430276|N|N|N|N|N| +2430369|AAAAAAAABKFBFCAA|1942-01-09|504|2193|169|1942|5|1|9|1|1942|169|2193|Friday|1942Q1|N|Y|N|2430361|2430360|2430004|2430277|N|N|N|N|N| +2430370|AAAAAAAACKFBFCAA|1942-01-10|504|2193|169|1942|6|1|10|1|1942|169|2193|Saturday|1942Q1|N|Y|N|2430361|2430360|2430005|2430278|N|N|N|N|N| +2430371|AAAAAAAADKFBFCAA|1942-01-11|504|2193|169|1942|0|1|11|1|1942|169|2193|Sunday|1942Q1|N|N|N|2430361|2430360|2430006|2430279|N|N|N|N|N| +2430372|AAAAAAAAEKFBFCAA|1942-01-12|504|2193|169|1942|1|1|12|1|1942|169|2193|Monday|1942Q1|N|N|N|2430361|2430360|2430007|2430280|N|N|N|N|N| +2430373|AAAAAAAAFKFBFCAA|1942-01-13|504|2194|169|1942|2|1|13|1|1942|169|2194|Tuesday|1942Q1|N|N|N|2430361|2430360|2430008|2430281|N|N|N|N|N| +2430374|AAAAAAAAGKFBFCAA|1942-01-14|504|2194|169|1942|3|1|14|1|1942|169|2194|Wednesday|1942Q1|N|N|N|2430361|2430360|2430009|2430282|N|N|N|N|N| +2430375|AAAAAAAAHKFBFCAA|1942-01-15|504|2194|169|1942|4|1|15|1|1942|169|2194|Thursday|1942Q1|N|N|N|2430361|2430360|2430010|2430283|N|N|N|N|N| +2430376|AAAAAAAAIKFBFCAA|1942-01-16|504|2194|169|1942|5|1|16|1|1942|169|2194|Friday|1942Q1|N|Y|N|2430361|2430360|2430011|2430284|N|N|N|N|N| +2430377|AAAAAAAAJKFBFCAA|1942-01-17|504|2194|169|1942|6|1|17|1|1942|169|2194|Saturday|1942Q1|N|Y|N|2430361|2430360|2430012|2430285|N|N|N|N|N| +2430378|AAAAAAAAKKFBFCAA|1942-01-18|504|2194|169|1942|0|1|18|1|1942|169|2194|Sunday|1942Q1|N|N|N|2430361|2430360|2430013|2430286|N|N|N|N|N| +2430379|AAAAAAAALKFBFCAA|1942-01-19|504|2194|169|1942|1|1|19|1|1942|169|2194|Monday|1942Q1|N|N|N|2430361|2430360|2430014|2430287|N|N|N|N|N| +2430380|AAAAAAAAMKFBFCAA|1942-01-20|504|2195|169|1942|2|1|20|1|1942|169|2195|Tuesday|1942Q1|N|N|N|2430361|2430360|2430015|2430288|N|N|N|N|N| +2430381|AAAAAAAANKFBFCAA|1942-01-21|504|2195|169|1942|3|1|21|1|1942|169|2195|Wednesday|1942Q1|N|N|N|2430361|2430360|2430016|2430289|N|N|N|N|N| +2430382|AAAAAAAAOKFBFCAA|1942-01-22|504|2195|169|1942|4|1|22|1|1942|169|2195|Thursday|1942Q1|N|N|N|2430361|2430360|2430017|2430290|N|N|N|N|N| +2430383|AAAAAAAAPKFBFCAA|1942-01-23|504|2195|169|1942|5|1|23|1|1942|169|2195|Friday|1942Q1|N|Y|N|2430361|2430360|2430018|2430291|N|N|N|N|N| +2430384|AAAAAAAAALFBFCAA|1942-01-24|504|2195|169|1942|6|1|24|1|1942|169|2195|Saturday|1942Q1|N|Y|N|2430361|2430360|2430019|2430292|N|N|N|N|N| +2430385|AAAAAAAABLFBFCAA|1942-01-25|504|2195|169|1942|0|1|25|1|1942|169|2195|Sunday|1942Q1|N|N|N|2430361|2430360|2430020|2430293|N|N|N|N|N| +2430386|AAAAAAAACLFBFCAA|1942-01-26|504|2195|169|1942|1|1|26|1|1942|169|2195|Monday|1942Q1|N|N|N|2430361|2430360|2430021|2430294|N|N|N|N|N| +2430387|AAAAAAAADLFBFCAA|1942-01-27|504|2196|169|1942|2|1|27|1|1942|169|2196|Tuesday|1942Q1|N|N|N|2430361|2430360|2430022|2430295|N|N|N|N|N| +2430388|AAAAAAAAELFBFCAA|1942-01-28|504|2196|169|1942|3|1|28|1|1942|169|2196|Wednesday|1942Q1|N|N|N|2430361|2430360|2430023|2430296|N|N|N|N|N| +2430389|AAAAAAAAFLFBFCAA|1942-01-29|504|2196|169|1942|4|1|29|1|1942|169|2196|Thursday|1942Q1|N|N|N|2430361|2430360|2430024|2430297|N|N|N|N|N| +2430390|AAAAAAAAGLFBFCAA|1942-01-30|504|2196|169|1942|5|1|30|1|1942|169|2196|Friday|1942Q1|N|Y|N|2430361|2430360|2430025|2430298|N|N|N|N|N| +2430391|AAAAAAAAHLFBFCAA|1942-01-31|504|2196|169|1942|6|1|31|1|1942|169|2196|Saturday|1942Q1|N|Y|N|2430361|2430360|2430026|2430299|N|N|N|N|N| +2430392|AAAAAAAAILFBFCAA|1942-02-01|505|2196|169|1942|0|2|1|1|1942|169|2196|Sunday|1942Q1|N|N|N|2430392|2430422|2430027|2430300|N|N|N|N|N| +2430393|AAAAAAAAJLFBFCAA|1942-02-02|505|2196|169|1942|1|2|2|1|1942|169|2196|Monday|1942Q1|N|N|N|2430392|2430422|2430028|2430301|N|N|N|N|N| +2430394|AAAAAAAAKLFBFCAA|1942-02-03|505|2197|169|1942|2|2|3|1|1942|169|2197|Tuesday|1942Q1|N|N|N|2430392|2430422|2430029|2430302|N|N|N|N|N| +2430395|AAAAAAAALLFBFCAA|1942-02-04|505|2197|169|1942|3|2|4|1|1942|169|2197|Wednesday|1942Q1|N|N|N|2430392|2430422|2430030|2430303|N|N|N|N|N| +2430396|AAAAAAAAMLFBFCAA|1942-02-05|505|2197|169|1942|4|2|5|1|1942|169|2197|Thursday|1942Q1|N|N|N|2430392|2430422|2430031|2430304|N|N|N|N|N| +2430397|AAAAAAAANLFBFCAA|1942-02-06|505|2197|169|1942|5|2|6|1|1942|169|2197|Friday|1942Q1|N|Y|N|2430392|2430422|2430032|2430305|N|N|N|N|N| +2430398|AAAAAAAAOLFBFCAA|1942-02-07|505|2197|169|1942|6|2|7|1|1942|169|2197|Saturday|1942Q1|N|Y|N|2430392|2430422|2430033|2430306|N|N|N|N|N| +2430399|AAAAAAAAPLFBFCAA|1942-02-08|505|2197|169|1942|0|2|8|1|1942|169|2197|Sunday|1942Q1|N|N|N|2430392|2430422|2430034|2430307|N|N|N|N|N| +2430400|AAAAAAAAAMFBFCAA|1942-02-09|505|2197|169|1942|1|2|9|1|1942|169|2197|Monday|1942Q1|N|N|N|2430392|2430422|2430035|2430308|N|N|N|N|N| +2430401|AAAAAAAABMFBFCAA|1942-02-10|505|2198|169|1942|2|2|10|1|1942|169|2198|Tuesday|1942Q1|N|N|N|2430392|2430422|2430036|2430309|N|N|N|N|N| +2430402|AAAAAAAACMFBFCAA|1942-02-11|505|2198|169|1942|3|2|11|1|1942|169|2198|Wednesday|1942Q1|N|N|N|2430392|2430422|2430037|2430310|N|N|N|N|N| +2430403|AAAAAAAADMFBFCAA|1942-02-12|505|2198|169|1942|4|2|12|1|1942|169|2198|Thursday|1942Q1|N|N|N|2430392|2430422|2430038|2430311|N|N|N|N|N| +2430404|AAAAAAAAEMFBFCAA|1942-02-13|505|2198|169|1942|5|2|13|1|1942|169|2198|Friday|1942Q1|N|Y|N|2430392|2430422|2430039|2430312|N|N|N|N|N| +2430405|AAAAAAAAFMFBFCAA|1942-02-14|505|2198|169|1942|6|2|14|1|1942|169|2198|Saturday|1942Q1|N|Y|N|2430392|2430422|2430040|2430313|N|N|N|N|N| +2430406|AAAAAAAAGMFBFCAA|1942-02-15|505|2198|169|1942|0|2|15|1|1942|169|2198|Sunday|1942Q1|N|N|N|2430392|2430422|2430041|2430314|N|N|N|N|N| +2430407|AAAAAAAAHMFBFCAA|1942-02-16|505|2198|169|1942|1|2|16|1|1942|169|2198|Monday|1942Q1|N|N|N|2430392|2430422|2430042|2430315|N|N|N|N|N| +2430408|AAAAAAAAIMFBFCAA|1942-02-17|505|2199|169|1942|2|2|17|1|1942|169|2199|Tuesday|1942Q1|N|N|N|2430392|2430422|2430043|2430316|N|N|N|N|N| +2430409|AAAAAAAAJMFBFCAA|1942-02-18|505|2199|169|1942|3|2|18|1|1942|169|2199|Wednesday|1942Q1|N|N|N|2430392|2430422|2430044|2430317|N|N|N|N|N| +2430410|AAAAAAAAKMFBFCAA|1942-02-19|505|2199|169|1942|4|2|19|1|1942|169|2199|Thursday|1942Q1|N|N|N|2430392|2430422|2430045|2430318|N|N|N|N|N| +2430411|AAAAAAAALMFBFCAA|1942-02-20|505|2199|169|1942|5|2|20|1|1942|169|2199|Friday|1942Q1|N|Y|N|2430392|2430422|2430046|2430319|N|N|N|N|N| +2430412|AAAAAAAAMMFBFCAA|1942-02-21|505|2199|169|1942|6|2|21|1|1942|169|2199|Saturday|1942Q1|N|Y|N|2430392|2430422|2430047|2430320|N|N|N|N|N| +2430413|AAAAAAAANMFBFCAA|1942-02-22|505|2199|169|1942|0|2|22|1|1942|169|2199|Sunday|1942Q1|N|N|N|2430392|2430422|2430048|2430321|N|N|N|N|N| +2430414|AAAAAAAAOMFBFCAA|1942-02-23|505|2199|169|1942|1|2|23|1|1942|169|2199|Monday|1942Q1|N|N|N|2430392|2430422|2430049|2430322|N|N|N|N|N| +2430415|AAAAAAAAPMFBFCAA|1942-02-24|505|2200|169|1942|2|2|24|1|1942|169|2200|Tuesday|1942Q1|N|N|N|2430392|2430422|2430050|2430323|N|N|N|N|N| +2430416|AAAAAAAAANFBFCAA|1942-02-25|505|2200|169|1942|3|2|25|1|1942|169|2200|Wednesday|1942Q1|N|N|N|2430392|2430422|2430051|2430324|N|N|N|N|N| +2430417|AAAAAAAABNFBFCAA|1942-02-26|505|2200|169|1942|4|2|26|1|1942|169|2200|Thursday|1942Q1|N|N|N|2430392|2430422|2430052|2430325|N|N|N|N|N| +2430418|AAAAAAAACNFBFCAA|1942-02-27|505|2200|169|1942|5|2|27|1|1942|169|2200|Friday|1942Q1|N|Y|N|2430392|2430422|2430053|2430326|N|N|N|N|N| +2430419|AAAAAAAADNFBFCAA|1942-02-28|505|2200|169|1942|6|2|28|1|1942|169|2200|Saturday|1942Q1|N|Y|N|2430392|2430422|2430054|2430327|N|N|N|N|N| +2430420|AAAAAAAAENFBFCAA|1942-03-01|506|2200|170|1942|0|3|1|1|1942|170|2200|Sunday|1942Q1|N|N|N|2430420|2430478|2430055|2430328|N|N|N|N|N| +2430421|AAAAAAAAFNFBFCAA|1942-03-02|506|2200|170|1942|1|3|2|1|1942|170|2200|Monday|1942Q1|N|N|N|2430420|2430478|2430056|2430329|N|N|N|N|N| +2430422|AAAAAAAAGNFBFCAA|1942-03-03|506|2201|170|1942|2|3|3|1|1942|170|2201|Tuesday|1942Q1|N|N|N|2430420|2430478|2430057|2430330|N|N|N|N|N| +2430423|AAAAAAAAHNFBFCAA|1942-03-04|506|2201|170|1942|3|3|4|1|1942|170|2201|Wednesday|1942Q1|N|N|N|2430420|2430478|2430058|2430331|N|N|N|N|N| +2430424|AAAAAAAAINFBFCAA|1942-03-05|506|2201|170|1942|4|3|5|1|1942|170|2201|Thursday|1942Q1|N|N|N|2430420|2430478|2430059|2430332|N|N|N|N|N| +2430425|AAAAAAAAJNFBFCAA|1942-03-06|506|2201|170|1942|5|3|6|1|1942|170|2201|Friday|1942Q1|N|Y|N|2430420|2430478|2430060|2430333|N|N|N|N|N| +2430426|AAAAAAAAKNFBFCAA|1942-03-07|506|2201|170|1942|6|3|7|1|1942|170|2201|Saturday|1942Q1|N|Y|N|2430420|2430478|2430061|2430334|N|N|N|N|N| +2430427|AAAAAAAALNFBFCAA|1942-03-08|506|2201|170|1942|0|3|8|1|1942|170|2201|Sunday|1942Q1|N|N|N|2430420|2430478|2430062|2430335|N|N|N|N|N| +2430428|AAAAAAAAMNFBFCAA|1942-03-09|506|2201|170|1942|1|3|9|1|1942|170|2201|Monday|1942Q1|N|N|N|2430420|2430478|2430063|2430336|N|N|N|N|N| +2430429|AAAAAAAANNFBFCAA|1942-03-10|506|2202|170|1942|2|3|10|1|1942|170|2202|Tuesday|1942Q1|N|N|N|2430420|2430478|2430064|2430337|N|N|N|N|N| +2430430|AAAAAAAAONFBFCAA|1942-03-11|506|2202|170|1942|3|3|11|1|1942|170|2202|Wednesday|1942Q1|N|N|N|2430420|2430478|2430065|2430338|N|N|N|N|N| +2430431|AAAAAAAAPNFBFCAA|1942-03-12|506|2202|170|1942|4|3|12|1|1942|170|2202|Thursday|1942Q1|N|N|N|2430420|2430478|2430066|2430339|N|N|N|N|N| +2430432|AAAAAAAAAOFBFCAA|1942-03-13|506|2202|170|1942|5|3|13|1|1942|170|2202|Friday|1942Q1|N|Y|N|2430420|2430478|2430067|2430340|N|N|N|N|N| +2430433|AAAAAAAABOFBFCAA|1942-03-14|506|2202|170|1942|6|3|14|1|1942|170|2202|Saturday|1942Q1|N|Y|N|2430420|2430478|2430068|2430341|N|N|N|N|N| +2430434|AAAAAAAACOFBFCAA|1942-03-15|506|2202|170|1942|0|3|15|1|1942|170|2202|Sunday|1942Q1|N|N|N|2430420|2430478|2430069|2430342|N|N|N|N|N| +2430435|AAAAAAAADOFBFCAA|1942-03-16|506|2202|170|1942|1|3|16|1|1942|170|2202|Monday|1942Q1|N|N|N|2430420|2430478|2430070|2430343|N|N|N|N|N| +2430436|AAAAAAAAEOFBFCAA|1942-03-17|506|2203|170|1942|2|3|17|1|1942|170|2203|Tuesday|1942Q1|N|N|N|2430420|2430478|2430071|2430344|N|N|N|N|N| +2430437|AAAAAAAAFOFBFCAA|1942-03-18|506|2203|170|1942|3|3|18|1|1942|170|2203|Wednesday|1942Q1|N|N|N|2430420|2430478|2430072|2430345|N|N|N|N|N| +2430438|AAAAAAAAGOFBFCAA|1942-03-19|506|2203|170|1942|4|3|19|1|1942|170|2203|Thursday|1942Q1|N|N|N|2430420|2430478|2430073|2430346|N|N|N|N|N| +2430439|AAAAAAAAHOFBFCAA|1942-03-20|506|2203|170|1942|5|3|20|1|1942|170|2203|Friday|1942Q1|N|Y|N|2430420|2430478|2430074|2430347|N|N|N|N|N| +2430440|AAAAAAAAIOFBFCAA|1942-03-21|506|2203|170|1942|6|3|21|1|1942|170|2203|Saturday|1942Q1|N|Y|N|2430420|2430478|2430075|2430348|N|N|N|N|N| +2430441|AAAAAAAAJOFBFCAA|1942-03-22|506|2203|170|1942|0|3|22|1|1942|170|2203|Sunday|1942Q1|N|N|N|2430420|2430478|2430076|2430349|N|N|N|N|N| +2430442|AAAAAAAAKOFBFCAA|1942-03-23|506|2203|170|1942|1|3|23|1|1942|170|2203|Monday|1942Q1|N|N|N|2430420|2430478|2430077|2430350|N|N|N|N|N| +2430443|AAAAAAAALOFBFCAA|1942-03-24|506|2204|170|1942|2|3|24|1|1942|170|2204|Tuesday|1942Q1|N|N|N|2430420|2430478|2430078|2430351|N|N|N|N|N| +2430444|AAAAAAAAMOFBFCAA|1942-03-25|506|2204|170|1942|3|3|25|1|1942|170|2204|Wednesday|1942Q1|N|N|N|2430420|2430478|2430079|2430352|N|N|N|N|N| +2430445|AAAAAAAANOFBFCAA|1942-03-26|506|2204|170|1942|4|3|26|1|1942|170|2204|Thursday|1942Q1|N|N|N|2430420|2430478|2430080|2430353|N|N|N|N|N| +2430446|AAAAAAAAOOFBFCAA|1942-03-27|506|2204|170|1942|5|3|27|1|1942|170|2204|Friday|1942Q1|N|Y|N|2430420|2430478|2430081|2430354|N|N|N|N|N| +2430447|AAAAAAAAPOFBFCAA|1942-03-28|506|2204|170|1942|6|3|28|1|1942|170|2204|Saturday|1942Q1|N|Y|N|2430420|2430478|2430082|2430355|N|N|N|N|N| +2430448|AAAAAAAAAPFBFCAA|1942-03-29|506|2204|170|1942|0|3|29|1|1942|170|2204|Sunday|1942Q1|N|N|N|2430420|2430478|2430083|2430356|N|N|N|N|N| +2430449|AAAAAAAABPFBFCAA|1942-03-30|506|2204|170|1942|1|3|30|1|1942|170|2204|Monday|1942Q1|N|N|N|2430420|2430478|2430084|2430357|N|N|N|N|N| +2430450|AAAAAAAACPFBFCAA|1942-03-31|506|2205|170|1942|2|3|31|1|1942|170|2205|Tuesday|1942Q1|N|N|N|2430420|2430478|2430085|2430358|N|N|N|N|N| +2430451|AAAAAAAADPFBFCAA|1942-04-01|507|2205|170|1942|3|4|1|1|1942|170|2205|Wednesday|1942Q1|N|N|N|2430451|2430540|2430086|2430361|N|N|N|N|N| +2430452|AAAAAAAAEPFBFCAA|1942-04-02|507|2205|170|1942|4|4|2|2|1942|170|2205|Thursday|1942Q2|N|N|N|2430451|2430540|2430087|2430362|N|N|N|N|N| +2430453|AAAAAAAAFPFBFCAA|1942-04-03|507|2205|170|1942|5|4|3|2|1942|170|2205|Friday|1942Q2|N|Y|N|2430451|2430540|2430088|2430363|N|N|N|N|N| +2430454|AAAAAAAAGPFBFCAA|1942-04-04|507|2205|170|1942|6|4|4|2|1942|170|2205|Saturday|1942Q2|N|Y|N|2430451|2430540|2430089|2430364|N|N|N|N|N| +2430455|AAAAAAAAHPFBFCAA|1942-04-05|507|2205|170|1942|0|4|5|2|1942|170|2205|Sunday|1942Q2|N|N|N|2430451|2430540|2430090|2430365|N|N|N|N|N| +2430456|AAAAAAAAIPFBFCAA|1942-04-06|507|2205|170|1942|1|4|6|2|1942|170|2205|Monday|1942Q2|N|N|N|2430451|2430540|2430091|2430366|N|N|N|N|N| +2430457|AAAAAAAAJPFBFCAA|1942-04-07|507|2206|170|1942|2|4|7|2|1942|170|2206|Tuesday|1942Q2|N|N|N|2430451|2430540|2430092|2430367|N|N|N|N|N| +2430458|AAAAAAAAKPFBFCAA|1942-04-08|507|2206|170|1942|3|4|8|2|1942|170|2206|Wednesday|1942Q2|N|N|N|2430451|2430540|2430093|2430368|N|N|N|N|N| +2430459|AAAAAAAALPFBFCAA|1942-04-09|507|2206|170|1942|4|4|9|2|1942|170|2206|Thursday|1942Q2|N|N|N|2430451|2430540|2430094|2430369|N|N|N|N|N| +2430460|AAAAAAAAMPFBFCAA|1942-04-10|507|2206|170|1942|5|4|10|2|1942|170|2206|Friday|1942Q2|N|Y|N|2430451|2430540|2430095|2430370|N|N|N|N|N| +2430461|AAAAAAAANPFBFCAA|1942-04-11|507|2206|170|1942|6|4|11|2|1942|170|2206|Saturday|1942Q2|N|Y|N|2430451|2430540|2430096|2430371|N|N|N|N|N| +2430462|AAAAAAAAOPFBFCAA|1942-04-12|507|2206|170|1942|0|4|12|2|1942|170|2206|Sunday|1942Q2|N|N|N|2430451|2430540|2430097|2430372|N|N|N|N|N| +2430463|AAAAAAAAPPFBFCAA|1942-04-13|507|2206|170|1942|1|4|13|2|1942|170|2206|Monday|1942Q2|N|N|N|2430451|2430540|2430098|2430373|N|N|N|N|N| +2430464|AAAAAAAAAAGBFCAA|1942-04-14|507|2207|170|1942|2|4|14|2|1942|170|2207|Tuesday|1942Q2|N|N|N|2430451|2430540|2430099|2430374|N|N|N|N|N| +2430465|AAAAAAAABAGBFCAA|1942-04-15|507|2207|170|1942|3|4|15|2|1942|170|2207|Wednesday|1942Q2|N|N|N|2430451|2430540|2430100|2430375|N|N|N|N|N| +2430466|AAAAAAAACAGBFCAA|1942-04-16|507|2207|170|1942|4|4|16|2|1942|170|2207|Thursday|1942Q2|N|N|N|2430451|2430540|2430101|2430376|N|N|N|N|N| +2430467|AAAAAAAADAGBFCAA|1942-04-17|507|2207|170|1942|5|4|17|2|1942|170|2207|Friday|1942Q2|N|Y|N|2430451|2430540|2430102|2430377|N|N|N|N|N| +2430468|AAAAAAAAEAGBFCAA|1942-04-18|507|2207|170|1942|6|4|18|2|1942|170|2207|Saturday|1942Q2|N|Y|N|2430451|2430540|2430103|2430378|N|N|N|N|N| +2430469|AAAAAAAAFAGBFCAA|1942-04-19|507|2207|170|1942|0|4|19|2|1942|170|2207|Sunday|1942Q2|N|N|N|2430451|2430540|2430104|2430379|N|N|N|N|N| +2430470|AAAAAAAAGAGBFCAA|1942-04-20|507|2207|170|1942|1|4|20|2|1942|170|2207|Monday|1942Q2|N|N|N|2430451|2430540|2430105|2430380|N|N|N|N|N| +2430471|AAAAAAAAHAGBFCAA|1942-04-21|507|2208|170|1942|2|4|21|2|1942|170|2208|Tuesday|1942Q2|N|N|N|2430451|2430540|2430106|2430381|N|N|N|N|N| +2430472|AAAAAAAAIAGBFCAA|1942-04-22|507|2208|170|1942|3|4|22|2|1942|170|2208|Wednesday|1942Q2|N|N|N|2430451|2430540|2430107|2430382|N|N|N|N|N| +2430473|AAAAAAAAJAGBFCAA|1942-04-23|507|2208|170|1942|4|4|23|2|1942|170|2208|Thursday|1942Q2|N|N|N|2430451|2430540|2430108|2430383|N|N|N|N|N| +2430474|AAAAAAAAKAGBFCAA|1942-04-24|507|2208|170|1942|5|4|24|2|1942|170|2208|Friday|1942Q2|N|Y|N|2430451|2430540|2430109|2430384|N|N|N|N|N| +2430475|AAAAAAAALAGBFCAA|1942-04-25|507|2208|170|1942|6|4|25|2|1942|170|2208|Saturday|1942Q2|N|Y|N|2430451|2430540|2430110|2430385|N|N|N|N|N| +2430476|AAAAAAAAMAGBFCAA|1942-04-26|507|2208|170|1942|0|4|26|2|1942|170|2208|Sunday|1942Q2|N|N|N|2430451|2430540|2430111|2430386|N|N|N|N|N| +2430477|AAAAAAAANAGBFCAA|1942-04-27|507|2208|170|1942|1|4|27|2|1942|170|2208|Monday|1942Q2|N|N|N|2430451|2430540|2430112|2430387|N|N|N|N|N| +2430478|AAAAAAAAOAGBFCAA|1942-04-28|507|2209|170|1942|2|4|28|2|1942|170|2209|Tuesday|1942Q2|N|N|N|2430451|2430540|2430113|2430388|N|N|N|N|N| +2430479|AAAAAAAAPAGBFCAA|1942-04-29|507|2209|170|1942|3|4|29|2|1942|170|2209|Wednesday|1942Q2|N|N|N|2430451|2430540|2430114|2430389|N|N|N|N|N| +2430480|AAAAAAAAABGBFCAA|1942-04-30|507|2209|170|1942|4|4|30|2|1942|170|2209|Thursday|1942Q2|N|N|N|2430451|2430540|2430115|2430390|N|N|N|N|N| +2430481|AAAAAAAABBGBFCAA|1942-05-01|508|2209|170|1942|5|5|1|2|1942|170|2209|Friday|1942Q2|N|Y|N|2430481|2430600|2430116|2430391|N|N|N|N|N| +2430482|AAAAAAAACBGBFCAA|1942-05-02|508|2209|170|1942|6|5|2|2|1942|170|2209|Saturday|1942Q2|N|Y|N|2430481|2430600|2430117|2430392|N|N|N|N|N| +2430483|AAAAAAAADBGBFCAA|1942-05-03|508|2209|170|1942|0|5|3|2|1942|170|2209|Sunday|1942Q2|N|N|N|2430481|2430600|2430118|2430393|N|N|N|N|N| +2430484|AAAAAAAAEBGBFCAA|1942-05-04|508|2209|170|1942|1|5|4|2|1942|170|2209|Monday|1942Q2|N|N|N|2430481|2430600|2430119|2430394|N|N|N|N|N| +2430485|AAAAAAAAFBGBFCAA|1942-05-05|508|2210|170|1942|2|5|5|2|1942|170|2210|Tuesday|1942Q2|N|N|N|2430481|2430600|2430120|2430395|N|N|N|N|N| +2430486|AAAAAAAAGBGBFCAA|1942-05-06|508|2210|170|1942|3|5|6|2|1942|170|2210|Wednesday|1942Q2|N|N|N|2430481|2430600|2430121|2430396|N|N|N|N|N| +2430487|AAAAAAAAHBGBFCAA|1942-05-07|508|2210|170|1942|4|5|7|2|1942|170|2210|Thursday|1942Q2|N|N|N|2430481|2430600|2430122|2430397|N|N|N|N|N| +2430488|AAAAAAAAIBGBFCAA|1942-05-08|508|2210|170|1942|5|5|8|2|1942|170|2210|Friday|1942Q2|N|Y|N|2430481|2430600|2430123|2430398|N|N|N|N|N| +2430489|AAAAAAAAJBGBFCAA|1942-05-09|508|2210|170|1942|6|5|9|2|1942|170|2210|Saturday|1942Q2|N|Y|N|2430481|2430600|2430124|2430399|N|N|N|N|N| +2430490|AAAAAAAAKBGBFCAA|1942-05-10|508|2210|170|1942|0|5|10|2|1942|170|2210|Sunday|1942Q2|N|N|N|2430481|2430600|2430125|2430400|N|N|N|N|N| +2430491|AAAAAAAALBGBFCAA|1942-05-11|508|2210|170|1942|1|5|11|2|1942|170|2210|Monday|1942Q2|N|N|N|2430481|2430600|2430126|2430401|N|N|N|N|N| +2430492|AAAAAAAAMBGBFCAA|1942-05-12|508|2211|170|1942|2|5|12|2|1942|170|2211|Tuesday|1942Q2|N|N|N|2430481|2430600|2430127|2430402|N|N|N|N|N| +2430493|AAAAAAAANBGBFCAA|1942-05-13|508|2211|170|1942|3|5|13|2|1942|170|2211|Wednesday|1942Q2|N|N|N|2430481|2430600|2430128|2430403|N|N|N|N|N| +2430494|AAAAAAAAOBGBFCAA|1942-05-14|508|2211|170|1942|4|5|14|2|1942|170|2211|Thursday|1942Q2|N|N|N|2430481|2430600|2430129|2430404|N|N|N|N|N| +2430495|AAAAAAAAPBGBFCAA|1942-05-15|508|2211|170|1942|5|5|15|2|1942|170|2211|Friday|1942Q2|N|Y|N|2430481|2430600|2430130|2430405|N|N|N|N|N| +2430496|AAAAAAAAACGBFCAA|1942-05-16|508|2211|170|1942|6|5|16|2|1942|170|2211|Saturday|1942Q2|N|Y|N|2430481|2430600|2430131|2430406|N|N|N|N|N| +2430497|AAAAAAAABCGBFCAA|1942-05-17|508|2211|170|1942|0|5|17|2|1942|170|2211|Sunday|1942Q2|N|N|N|2430481|2430600|2430132|2430407|N|N|N|N|N| +2430498|AAAAAAAACCGBFCAA|1942-05-18|508|2211|170|1942|1|5|18|2|1942|170|2211|Monday|1942Q2|N|N|N|2430481|2430600|2430133|2430408|N|N|N|N|N| +2430499|AAAAAAAADCGBFCAA|1942-05-19|508|2212|170|1942|2|5|19|2|1942|170|2212|Tuesday|1942Q2|N|N|N|2430481|2430600|2430134|2430409|N|N|N|N|N| +2430500|AAAAAAAAECGBFCAA|1942-05-20|508|2212|170|1942|3|5|20|2|1942|170|2212|Wednesday|1942Q2|N|N|N|2430481|2430600|2430135|2430410|N|N|N|N|N| +2430501|AAAAAAAAFCGBFCAA|1942-05-21|508|2212|170|1942|4|5|21|2|1942|170|2212|Thursday|1942Q2|N|N|N|2430481|2430600|2430136|2430411|N|N|N|N|N| +2430502|AAAAAAAAGCGBFCAA|1942-05-22|508|2212|170|1942|5|5|22|2|1942|170|2212|Friday|1942Q2|N|Y|N|2430481|2430600|2430137|2430412|N|N|N|N|N| +2430503|AAAAAAAAHCGBFCAA|1942-05-23|508|2212|170|1942|6|5|23|2|1942|170|2212|Saturday|1942Q2|N|Y|N|2430481|2430600|2430138|2430413|N|N|N|N|N| +2430504|AAAAAAAAICGBFCAA|1942-05-24|508|2212|170|1942|0|5|24|2|1942|170|2212|Sunday|1942Q2|N|N|N|2430481|2430600|2430139|2430414|N|N|N|N|N| +2430505|AAAAAAAAJCGBFCAA|1942-05-25|508|2212|170|1942|1|5|25|2|1942|170|2212|Monday|1942Q2|N|N|N|2430481|2430600|2430140|2430415|N|N|N|N|N| +2430506|AAAAAAAAKCGBFCAA|1942-05-26|508|2213|170|1942|2|5|26|2|1942|170|2213|Tuesday|1942Q2|N|N|N|2430481|2430600|2430141|2430416|N|N|N|N|N| +2430507|AAAAAAAALCGBFCAA|1942-05-27|508|2213|170|1942|3|5|27|2|1942|170|2213|Wednesday|1942Q2|N|N|N|2430481|2430600|2430142|2430417|N|N|N|N|N| +2430508|AAAAAAAAMCGBFCAA|1942-05-28|508|2213|170|1942|4|5|28|2|1942|170|2213|Thursday|1942Q2|N|N|N|2430481|2430600|2430143|2430418|N|N|N|N|N| +2430509|AAAAAAAANCGBFCAA|1942-05-29|508|2213|170|1942|5|5|29|2|1942|170|2213|Friday|1942Q2|N|Y|N|2430481|2430600|2430144|2430419|N|N|N|N|N| +2430510|AAAAAAAAOCGBFCAA|1942-05-30|508|2213|170|1942|6|5|30|2|1942|170|2213|Saturday|1942Q2|N|Y|N|2430481|2430600|2430145|2430420|N|N|N|N|N| +2430511|AAAAAAAAPCGBFCAA|1942-05-31|508|2213|170|1942|0|5|31|2|1942|170|2213|Sunday|1942Q2|N|N|N|2430481|2430600|2430146|2430421|N|N|N|N|N| +2430512|AAAAAAAAADGBFCAA|1942-06-01|509|2213|171|1942|1|6|1|2|1942|171|2213|Monday|1942Q2|N|N|N|2430512|2430662|2430147|2430422|N|N|N|N|N| +2430513|AAAAAAAABDGBFCAA|1942-06-02|509|2214|171|1942|2|6|2|2|1942|171|2214|Tuesday|1942Q2|N|N|N|2430512|2430662|2430148|2430423|N|N|N|N|N| +2430514|AAAAAAAACDGBFCAA|1942-06-03|509|2214|171|1942|3|6|3|2|1942|171|2214|Wednesday|1942Q2|N|N|N|2430512|2430662|2430149|2430424|N|N|N|N|N| +2430515|AAAAAAAADDGBFCAA|1942-06-04|509|2214|171|1942|4|6|4|2|1942|171|2214|Thursday|1942Q2|N|N|N|2430512|2430662|2430150|2430425|N|N|N|N|N| +2430516|AAAAAAAAEDGBFCAA|1942-06-05|509|2214|171|1942|5|6|5|2|1942|171|2214|Friday|1942Q2|N|Y|N|2430512|2430662|2430151|2430426|N|N|N|N|N| +2430517|AAAAAAAAFDGBFCAA|1942-06-06|509|2214|171|1942|6|6|6|2|1942|171|2214|Saturday|1942Q2|N|Y|N|2430512|2430662|2430152|2430427|N|N|N|N|N| +2430518|AAAAAAAAGDGBFCAA|1942-06-07|509|2214|171|1942|0|6|7|2|1942|171|2214|Sunday|1942Q2|N|N|N|2430512|2430662|2430153|2430428|N|N|N|N|N| +2430519|AAAAAAAAHDGBFCAA|1942-06-08|509|2214|171|1942|1|6|8|2|1942|171|2214|Monday|1942Q2|N|N|N|2430512|2430662|2430154|2430429|N|N|N|N|N| +2430520|AAAAAAAAIDGBFCAA|1942-06-09|509|2215|171|1942|2|6|9|2|1942|171|2215|Tuesday|1942Q2|N|N|N|2430512|2430662|2430155|2430430|N|N|N|N|N| +2430521|AAAAAAAAJDGBFCAA|1942-06-10|509|2215|171|1942|3|6|10|2|1942|171|2215|Wednesday|1942Q2|N|N|N|2430512|2430662|2430156|2430431|N|N|N|N|N| +2430522|AAAAAAAAKDGBFCAA|1942-06-11|509|2215|171|1942|4|6|11|2|1942|171|2215|Thursday|1942Q2|N|N|N|2430512|2430662|2430157|2430432|N|N|N|N|N| +2430523|AAAAAAAALDGBFCAA|1942-06-12|509|2215|171|1942|5|6|12|2|1942|171|2215|Friday|1942Q2|N|Y|N|2430512|2430662|2430158|2430433|N|N|N|N|N| +2430524|AAAAAAAAMDGBFCAA|1942-06-13|509|2215|171|1942|6|6|13|2|1942|171|2215|Saturday|1942Q2|N|Y|N|2430512|2430662|2430159|2430434|N|N|N|N|N| +2430525|AAAAAAAANDGBFCAA|1942-06-14|509|2215|171|1942|0|6|14|2|1942|171|2215|Sunday|1942Q2|N|N|N|2430512|2430662|2430160|2430435|N|N|N|N|N| +2430526|AAAAAAAAODGBFCAA|1942-06-15|509|2215|171|1942|1|6|15|2|1942|171|2215|Monday|1942Q2|N|N|N|2430512|2430662|2430161|2430436|N|N|N|N|N| +2430527|AAAAAAAAPDGBFCAA|1942-06-16|509|2216|171|1942|2|6|16|2|1942|171|2216|Tuesday|1942Q2|N|N|N|2430512|2430662|2430162|2430437|N|N|N|N|N| +2430528|AAAAAAAAAEGBFCAA|1942-06-17|509|2216|171|1942|3|6|17|2|1942|171|2216|Wednesday|1942Q2|N|N|N|2430512|2430662|2430163|2430438|N|N|N|N|N| +2430529|AAAAAAAABEGBFCAA|1942-06-18|509|2216|171|1942|4|6|18|2|1942|171|2216|Thursday|1942Q2|N|N|N|2430512|2430662|2430164|2430439|N|N|N|N|N| +2430530|AAAAAAAACEGBFCAA|1942-06-19|509|2216|171|1942|5|6|19|2|1942|171|2216|Friday|1942Q2|N|Y|N|2430512|2430662|2430165|2430440|N|N|N|N|N| +2430531|AAAAAAAADEGBFCAA|1942-06-20|509|2216|171|1942|6|6|20|2|1942|171|2216|Saturday|1942Q2|N|Y|N|2430512|2430662|2430166|2430441|N|N|N|N|N| +2430532|AAAAAAAAEEGBFCAA|1942-06-21|509|2216|171|1942|0|6|21|2|1942|171|2216|Sunday|1942Q2|N|N|N|2430512|2430662|2430167|2430442|N|N|N|N|N| +2430533|AAAAAAAAFEGBFCAA|1942-06-22|509|2216|171|1942|1|6|22|2|1942|171|2216|Monday|1942Q2|N|N|N|2430512|2430662|2430168|2430443|N|N|N|N|N| +2430534|AAAAAAAAGEGBFCAA|1942-06-23|509|2217|171|1942|2|6|23|2|1942|171|2217|Tuesday|1942Q2|N|N|N|2430512|2430662|2430169|2430444|N|N|N|N|N| +2430535|AAAAAAAAHEGBFCAA|1942-06-24|509|2217|171|1942|3|6|24|2|1942|171|2217|Wednesday|1942Q2|N|N|N|2430512|2430662|2430170|2430445|N|N|N|N|N| +2430536|AAAAAAAAIEGBFCAA|1942-06-25|509|2217|171|1942|4|6|25|2|1942|171|2217|Thursday|1942Q2|N|N|N|2430512|2430662|2430171|2430446|N|N|N|N|N| +2430537|AAAAAAAAJEGBFCAA|1942-06-26|509|2217|171|1942|5|6|26|2|1942|171|2217|Friday|1942Q2|N|Y|N|2430512|2430662|2430172|2430447|N|N|N|N|N| +2430538|AAAAAAAAKEGBFCAA|1942-06-27|509|2217|171|1942|6|6|27|2|1942|171|2217|Saturday|1942Q2|N|Y|N|2430512|2430662|2430173|2430448|N|N|N|N|N| +2430539|AAAAAAAALEGBFCAA|1942-06-28|509|2217|171|1942|0|6|28|2|1942|171|2217|Sunday|1942Q2|N|N|N|2430512|2430662|2430174|2430449|N|N|N|N|N| +2430540|AAAAAAAAMEGBFCAA|1942-06-29|509|2217|171|1942|1|6|29|2|1942|171|2217|Monday|1942Q2|N|N|N|2430512|2430662|2430175|2430450|N|N|N|N|N| +2430541|AAAAAAAANEGBFCAA|1942-06-30|509|2218|171|1942|2|6|30|2|1942|171|2218|Tuesday|1942Q2|N|N|N|2430512|2430662|2430176|2430451|N|N|N|N|N| +2430542|AAAAAAAAOEGBFCAA|1942-07-01|510|2218|171|1942|3|7|1|2|1942|171|2218|Wednesday|1942Q2|N|N|N|2430542|2430722|2430177|2430451|N|N|N|N|N| +2430543|AAAAAAAAPEGBFCAA|1942-07-02|510|2218|171|1942|4|7|2|3|1942|171|2218|Thursday|1942Q3|N|N|N|2430542|2430722|2430178|2430452|N|N|N|N|N| +2430544|AAAAAAAAAFGBFCAA|1942-07-03|510|2218|171|1942|5|7|3|3|1942|171|2218|Friday|1942Q3|N|Y|N|2430542|2430722|2430179|2430453|N|N|N|N|N| +2430545|AAAAAAAABFGBFCAA|1942-07-04|510|2218|171|1942|6|7|4|3|1942|171|2218|Saturday|1942Q3|N|Y|N|2430542|2430722|2430180|2430454|N|N|N|N|N| +2430546|AAAAAAAACFGBFCAA|1942-07-05|510|2218|171|1942|0|7|5|3|1942|171|2218|Sunday|1942Q3|Y|N|N|2430542|2430722|2430181|2430455|N|N|N|N|N| +2430547|AAAAAAAADFGBFCAA|1942-07-06|510|2218|171|1942|1|7|6|3|1942|171|2218|Monday|1942Q3|N|N|Y|2430542|2430722|2430182|2430456|N|N|N|N|N| +2430548|AAAAAAAAEFGBFCAA|1942-07-07|510|2219|171|1942|2|7|7|3|1942|171|2219|Tuesday|1942Q3|N|N|N|2430542|2430722|2430183|2430457|N|N|N|N|N| +2430549|AAAAAAAAFFGBFCAA|1942-07-08|510|2219|171|1942|3|7|8|3|1942|171|2219|Wednesday|1942Q3|N|N|N|2430542|2430722|2430184|2430458|N|N|N|N|N| +2430550|AAAAAAAAGFGBFCAA|1942-07-09|510|2219|171|1942|4|7|9|3|1942|171|2219|Thursday|1942Q3|N|N|N|2430542|2430722|2430185|2430459|N|N|N|N|N| +2430551|AAAAAAAAHFGBFCAA|1942-07-10|510|2219|171|1942|5|7|10|3|1942|171|2219|Friday|1942Q3|N|Y|N|2430542|2430722|2430186|2430460|N|N|N|N|N| +2430552|AAAAAAAAIFGBFCAA|1942-07-11|510|2219|171|1942|6|7|11|3|1942|171|2219|Saturday|1942Q3|N|Y|N|2430542|2430722|2430187|2430461|N|N|N|N|N| +2430553|AAAAAAAAJFGBFCAA|1942-07-12|510|2219|171|1942|0|7|12|3|1942|171|2219|Sunday|1942Q3|N|N|N|2430542|2430722|2430188|2430462|N|N|N|N|N| +2430554|AAAAAAAAKFGBFCAA|1942-07-13|510|2219|171|1942|1|7|13|3|1942|171|2219|Monday|1942Q3|N|N|N|2430542|2430722|2430189|2430463|N|N|N|N|N| +2430555|AAAAAAAALFGBFCAA|1942-07-14|510|2220|171|1942|2|7|14|3|1942|171|2220|Tuesday|1942Q3|N|N|N|2430542|2430722|2430190|2430464|N|N|N|N|N| +2430556|AAAAAAAAMFGBFCAA|1942-07-15|510|2220|171|1942|3|7|15|3|1942|171|2220|Wednesday|1942Q3|N|N|N|2430542|2430722|2430191|2430465|N|N|N|N|N| +2430557|AAAAAAAANFGBFCAA|1942-07-16|510|2220|171|1942|4|7|16|3|1942|171|2220|Thursday|1942Q3|N|N|N|2430542|2430722|2430192|2430466|N|N|N|N|N| +2430558|AAAAAAAAOFGBFCAA|1942-07-17|510|2220|171|1942|5|7|17|3|1942|171|2220|Friday|1942Q3|N|Y|N|2430542|2430722|2430193|2430467|N|N|N|N|N| +2430559|AAAAAAAAPFGBFCAA|1942-07-18|510|2220|171|1942|6|7|18|3|1942|171|2220|Saturday|1942Q3|N|Y|N|2430542|2430722|2430194|2430468|N|N|N|N|N| +2430560|AAAAAAAAAGGBFCAA|1942-07-19|510|2220|171|1942|0|7|19|3|1942|171|2220|Sunday|1942Q3|N|N|N|2430542|2430722|2430195|2430469|N|N|N|N|N| +2430561|AAAAAAAABGGBFCAA|1942-07-20|510|2220|171|1942|1|7|20|3|1942|171|2220|Monday|1942Q3|N|N|N|2430542|2430722|2430196|2430470|N|N|N|N|N| +2430562|AAAAAAAACGGBFCAA|1942-07-21|510|2221|171|1942|2|7|21|3|1942|171|2221|Tuesday|1942Q3|N|N|N|2430542|2430722|2430197|2430471|N|N|N|N|N| +2430563|AAAAAAAADGGBFCAA|1942-07-22|510|2221|171|1942|3|7|22|3|1942|171|2221|Wednesday|1942Q3|N|N|N|2430542|2430722|2430198|2430472|N|N|N|N|N| +2430564|AAAAAAAAEGGBFCAA|1942-07-23|510|2221|171|1942|4|7|23|3|1942|171|2221|Thursday|1942Q3|N|N|N|2430542|2430722|2430199|2430473|N|N|N|N|N| +2430565|AAAAAAAAFGGBFCAA|1942-07-24|510|2221|171|1942|5|7|24|3|1942|171|2221|Friday|1942Q3|N|Y|N|2430542|2430722|2430200|2430474|N|N|N|N|N| +2430566|AAAAAAAAGGGBFCAA|1942-07-25|510|2221|171|1942|6|7|25|3|1942|171|2221|Saturday|1942Q3|N|Y|N|2430542|2430722|2430201|2430475|N|N|N|N|N| +2430567|AAAAAAAAHGGBFCAA|1942-07-26|510|2221|171|1942|0|7|26|3|1942|171|2221|Sunday|1942Q3|N|N|N|2430542|2430722|2430202|2430476|N|N|N|N|N| +2430568|AAAAAAAAIGGBFCAA|1942-07-27|510|2221|171|1942|1|7|27|3|1942|171|2221|Monday|1942Q3|N|N|N|2430542|2430722|2430203|2430477|N|N|N|N|N| +2430569|AAAAAAAAJGGBFCAA|1942-07-28|510|2222|171|1942|2|7|28|3|1942|171|2222|Tuesday|1942Q3|N|N|N|2430542|2430722|2430204|2430478|N|N|N|N|N| +2430570|AAAAAAAAKGGBFCAA|1942-07-29|510|2222|171|1942|3|7|29|3|1942|171|2222|Wednesday|1942Q3|N|N|N|2430542|2430722|2430205|2430479|N|N|N|N|N| +2430571|AAAAAAAALGGBFCAA|1942-07-30|510|2222|171|1942|4|7|30|3|1942|171|2222|Thursday|1942Q3|N|N|N|2430542|2430722|2430206|2430480|N|N|N|N|N| +2430572|AAAAAAAAMGGBFCAA|1942-07-31|510|2222|171|1942|5|7|31|3|1942|171|2222|Friday|1942Q3|N|Y|N|2430542|2430722|2430207|2430481|N|N|N|N|N| +2430573|AAAAAAAANGGBFCAA|1942-08-01|511|2222|171|1942|6|8|1|3|1942|171|2222|Saturday|1942Q3|N|Y|N|2430573|2430784|2430208|2430482|N|N|N|N|N| +2430574|AAAAAAAAOGGBFCAA|1942-08-02|511|2222|171|1942|0|8|2|3|1942|171|2222|Sunday|1942Q3|N|N|N|2430573|2430784|2430209|2430483|N|N|N|N|N| +2430575|AAAAAAAAPGGBFCAA|1942-08-03|511|2222|171|1942|1|8|3|3|1942|171|2222|Monday|1942Q3|N|N|N|2430573|2430784|2430210|2430484|N|N|N|N|N| +2430576|AAAAAAAAAHGBFCAA|1942-08-04|511|2223|171|1942|2|8|4|3|1942|171|2223|Tuesday|1942Q3|N|N|N|2430573|2430784|2430211|2430485|N|N|N|N|N| +2430577|AAAAAAAABHGBFCAA|1942-08-05|511|2223|171|1942|3|8|5|3|1942|171|2223|Wednesday|1942Q3|N|N|N|2430573|2430784|2430212|2430486|N|N|N|N|N| +2430578|AAAAAAAACHGBFCAA|1942-08-06|511|2223|171|1942|4|8|6|3|1942|171|2223|Thursday|1942Q3|N|N|N|2430573|2430784|2430213|2430487|N|N|N|N|N| +2430579|AAAAAAAADHGBFCAA|1942-08-07|511|2223|171|1942|5|8|7|3|1942|171|2223|Friday|1942Q3|N|Y|N|2430573|2430784|2430214|2430488|N|N|N|N|N| +2430580|AAAAAAAAEHGBFCAA|1942-08-08|511|2223|171|1942|6|8|8|3|1942|171|2223|Saturday|1942Q3|N|Y|N|2430573|2430784|2430215|2430489|N|N|N|N|N| +2430581|AAAAAAAAFHGBFCAA|1942-08-09|511|2223|171|1942|0|8|9|3|1942|171|2223|Sunday|1942Q3|N|N|N|2430573|2430784|2430216|2430490|N|N|N|N|N| +2430582|AAAAAAAAGHGBFCAA|1942-08-10|511|2223|171|1942|1|8|10|3|1942|171|2223|Monday|1942Q3|N|N|N|2430573|2430784|2430217|2430491|N|N|N|N|N| +2430583|AAAAAAAAHHGBFCAA|1942-08-11|511|2224|171|1942|2|8|11|3|1942|171|2224|Tuesday|1942Q3|N|N|N|2430573|2430784|2430218|2430492|N|N|N|N|N| +2430584|AAAAAAAAIHGBFCAA|1942-08-12|511|2224|171|1942|3|8|12|3|1942|171|2224|Wednesday|1942Q3|N|N|N|2430573|2430784|2430219|2430493|N|N|N|N|N| +2430585|AAAAAAAAJHGBFCAA|1942-08-13|511|2224|171|1942|4|8|13|3|1942|171|2224|Thursday|1942Q3|N|N|N|2430573|2430784|2430220|2430494|N|N|N|N|N| +2430586|AAAAAAAAKHGBFCAA|1942-08-14|511|2224|171|1942|5|8|14|3|1942|171|2224|Friday|1942Q3|N|Y|N|2430573|2430784|2430221|2430495|N|N|N|N|N| +2430587|AAAAAAAALHGBFCAA|1942-08-15|511|2224|171|1942|6|8|15|3|1942|171|2224|Saturday|1942Q3|N|Y|N|2430573|2430784|2430222|2430496|N|N|N|N|N| +2430588|AAAAAAAAMHGBFCAA|1942-08-16|511|2224|171|1942|0|8|16|3|1942|171|2224|Sunday|1942Q3|N|N|N|2430573|2430784|2430223|2430497|N|N|N|N|N| +2430589|AAAAAAAANHGBFCAA|1942-08-17|511|2224|171|1942|1|8|17|3|1942|171|2224|Monday|1942Q3|N|N|N|2430573|2430784|2430224|2430498|N|N|N|N|N| +2430590|AAAAAAAAOHGBFCAA|1942-08-18|511|2225|171|1942|2|8|18|3|1942|171|2225|Tuesday|1942Q3|N|N|N|2430573|2430784|2430225|2430499|N|N|N|N|N| +2430591|AAAAAAAAPHGBFCAA|1942-08-19|511|2225|171|1942|3|8|19|3|1942|171|2225|Wednesday|1942Q3|N|N|N|2430573|2430784|2430226|2430500|N|N|N|N|N| +2430592|AAAAAAAAAIGBFCAA|1942-08-20|511|2225|171|1942|4|8|20|3|1942|171|2225|Thursday|1942Q3|N|N|N|2430573|2430784|2430227|2430501|N|N|N|N|N| +2430593|AAAAAAAABIGBFCAA|1942-08-21|511|2225|171|1942|5|8|21|3|1942|171|2225|Friday|1942Q3|N|Y|N|2430573|2430784|2430228|2430502|N|N|N|N|N| +2430594|AAAAAAAACIGBFCAA|1942-08-22|511|2225|171|1942|6|8|22|3|1942|171|2225|Saturday|1942Q3|N|Y|N|2430573|2430784|2430229|2430503|N|N|N|N|N| +2430595|AAAAAAAADIGBFCAA|1942-08-23|511|2225|171|1942|0|8|23|3|1942|171|2225|Sunday|1942Q3|N|N|N|2430573|2430784|2430230|2430504|N|N|N|N|N| +2430596|AAAAAAAAEIGBFCAA|1942-08-24|511|2225|171|1942|1|8|24|3|1942|171|2225|Monday|1942Q3|N|N|N|2430573|2430784|2430231|2430505|N|N|N|N|N| +2430597|AAAAAAAAFIGBFCAA|1942-08-25|511|2226|171|1942|2|8|25|3|1942|171|2226|Tuesday|1942Q3|N|N|N|2430573|2430784|2430232|2430506|N|N|N|N|N| +2430598|AAAAAAAAGIGBFCAA|1942-08-26|511|2226|171|1942|3|8|26|3|1942|171|2226|Wednesday|1942Q3|N|N|N|2430573|2430784|2430233|2430507|N|N|N|N|N| +2430599|AAAAAAAAHIGBFCAA|1942-08-27|511|2226|171|1942|4|8|27|3|1942|171|2226|Thursday|1942Q3|N|N|N|2430573|2430784|2430234|2430508|N|N|N|N|N| +2430600|AAAAAAAAIIGBFCAA|1942-08-28|511|2226|171|1942|5|8|28|3|1942|171|2226|Friday|1942Q3|N|Y|N|2430573|2430784|2430235|2430509|N|N|N|N|N| +2430601|AAAAAAAAJIGBFCAA|1942-08-29|511|2226|171|1942|6|8|29|3|1942|171|2226|Saturday|1942Q3|N|Y|N|2430573|2430784|2430236|2430510|N|N|N|N|N| +2430602|AAAAAAAAKIGBFCAA|1942-08-30|511|2226|171|1942|0|8|30|3|1942|171|2226|Sunday|1942Q3|N|N|N|2430573|2430784|2430237|2430511|N|N|N|N|N| +2430603|AAAAAAAALIGBFCAA|1942-08-31|511|2226|171|1942|1|8|31|3|1942|171|2226|Monday|1942Q3|N|N|N|2430573|2430784|2430238|2430512|N|N|N|N|N| +2430604|AAAAAAAAMIGBFCAA|1942-09-01|512|2227|172|1942|2|9|1|3|1942|172|2227|Tuesday|1942Q3|N|N|N|2430604|2430846|2430239|2430513|N|N|N|N|N| +2430605|AAAAAAAANIGBFCAA|1942-09-02|512|2227|172|1942|3|9|2|3|1942|172|2227|Wednesday|1942Q3|N|N|N|2430604|2430846|2430240|2430514|N|N|N|N|N| +2430606|AAAAAAAAOIGBFCAA|1942-09-03|512|2227|172|1942|4|9|3|3|1942|172|2227|Thursday|1942Q3|N|N|N|2430604|2430846|2430241|2430515|N|N|N|N|N| +2430607|AAAAAAAAPIGBFCAA|1942-09-04|512|2227|172|1942|5|9|4|3|1942|172|2227|Friday|1942Q3|N|Y|N|2430604|2430846|2430242|2430516|N|N|N|N|N| +2430608|AAAAAAAAAJGBFCAA|1942-09-05|512|2227|172|1942|6|9|5|3|1942|172|2227|Saturday|1942Q3|N|Y|N|2430604|2430846|2430243|2430517|N|N|N|N|N| +2430609|AAAAAAAABJGBFCAA|1942-09-06|512|2227|172|1942|0|9|6|3|1942|172|2227|Sunday|1942Q3|N|N|N|2430604|2430846|2430244|2430518|N|N|N|N|N| +2430610|AAAAAAAACJGBFCAA|1942-09-07|512|2227|172|1942|1|9|7|3|1942|172|2227|Monday|1942Q3|N|N|N|2430604|2430846|2430245|2430519|N|N|N|N|N| +2430611|AAAAAAAADJGBFCAA|1942-09-08|512|2228|172|1942|2|9|8|3|1942|172|2228|Tuesday|1942Q3|N|N|N|2430604|2430846|2430246|2430520|N|N|N|N|N| +2430612|AAAAAAAAEJGBFCAA|1942-09-09|512|2228|172|1942|3|9|9|3|1942|172|2228|Wednesday|1942Q3|N|N|N|2430604|2430846|2430247|2430521|N|N|N|N|N| +2430613|AAAAAAAAFJGBFCAA|1942-09-10|512|2228|172|1942|4|9|10|3|1942|172|2228|Thursday|1942Q3|N|N|N|2430604|2430846|2430248|2430522|N|N|N|N|N| +2430614|AAAAAAAAGJGBFCAA|1942-09-11|512|2228|172|1942|5|9|11|3|1942|172|2228|Friday|1942Q3|N|Y|N|2430604|2430846|2430249|2430523|N|N|N|N|N| +2430615|AAAAAAAAHJGBFCAA|1942-09-12|512|2228|172|1942|6|9|12|3|1942|172|2228|Saturday|1942Q3|N|Y|N|2430604|2430846|2430250|2430524|N|N|N|N|N| +2430616|AAAAAAAAIJGBFCAA|1942-09-13|512|2228|172|1942|0|9|13|3|1942|172|2228|Sunday|1942Q3|N|N|N|2430604|2430846|2430251|2430525|N|N|N|N|N| +2430617|AAAAAAAAJJGBFCAA|1942-09-14|512|2228|172|1942|1|9|14|3|1942|172|2228|Monday|1942Q3|N|N|N|2430604|2430846|2430252|2430526|N|N|N|N|N| +2430618|AAAAAAAAKJGBFCAA|1942-09-15|512|2229|172|1942|2|9|15|3|1942|172|2229|Tuesday|1942Q3|N|N|N|2430604|2430846|2430253|2430527|N|N|N|N|N| +2430619|AAAAAAAALJGBFCAA|1942-09-16|512|2229|172|1942|3|9|16|3|1942|172|2229|Wednesday|1942Q3|N|N|N|2430604|2430846|2430254|2430528|N|N|N|N|N| +2430620|AAAAAAAAMJGBFCAA|1942-09-17|512|2229|172|1942|4|9|17|3|1942|172|2229|Thursday|1942Q3|N|N|N|2430604|2430846|2430255|2430529|N|N|N|N|N| +2430621|AAAAAAAANJGBFCAA|1942-09-18|512|2229|172|1942|5|9|18|3|1942|172|2229|Friday|1942Q3|N|Y|N|2430604|2430846|2430256|2430530|N|N|N|N|N| +2430622|AAAAAAAAOJGBFCAA|1942-09-19|512|2229|172|1942|6|9|19|3|1942|172|2229|Saturday|1942Q3|N|Y|N|2430604|2430846|2430257|2430531|N|N|N|N|N| +2430623|AAAAAAAAPJGBFCAA|1942-09-20|512|2229|172|1942|0|9|20|3|1942|172|2229|Sunday|1942Q3|N|N|N|2430604|2430846|2430258|2430532|N|N|N|N|N| +2430624|AAAAAAAAAKGBFCAA|1942-09-21|512|2229|172|1942|1|9|21|3|1942|172|2229|Monday|1942Q3|N|N|N|2430604|2430846|2430259|2430533|N|N|N|N|N| +2430625|AAAAAAAABKGBFCAA|1942-09-22|512|2230|172|1942|2|9|22|3|1942|172|2230|Tuesday|1942Q3|N|N|N|2430604|2430846|2430260|2430534|N|N|N|N|N| +2430626|AAAAAAAACKGBFCAA|1942-09-23|512|2230|172|1942|3|9|23|3|1942|172|2230|Wednesday|1942Q3|N|N|N|2430604|2430846|2430261|2430535|N|N|N|N|N| +2430627|AAAAAAAADKGBFCAA|1942-09-24|512|2230|172|1942|4|9|24|3|1942|172|2230|Thursday|1942Q3|N|N|N|2430604|2430846|2430262|2430536|N|N|N|N|N| +2430628|AAAAAAAAEKGBFCAA|1942-09-25|512|2230|172|1942|5|9|25|3|1942|172|2230|Friday|1942Q3|N|Y|N|2430604|2430846|2430263|2430537|N|N|N|N|N| +2430629|AAAAAAAAFKGBFCAA|1942-09-26|512|2230|172|1942|6|9|26|3|1942|172|2230|Saturday|1942Q3|N|Y|N|2430604|2430846|2430264|2430538|N|N|N|N|N| +2430630|AAAAAAAAGKGBFCAA|1942-09-27|512|2230|172|1942|0|9|27|3|1942|172|2230|Sunday|1942Q3|N|N|N|2430604|2430846|2430265|2430539|N|N|N|N|N| +2430631|AAAAAAAAHKGBFCAA|1942-09-28|512|2230|172|1942|1|9|28|3|1942|172|2230|Monday|1942Q3|N|N|N|2430604|2430846|2430266|2430540|N|N|N|N|N| +2430632|AAAAAAAAIKGBFCAA|1942-09-29|512|2231|172|1942|2|9|29|3|1942|172|2231|Tuesday|1942Q3|N|N|N|2430604|2430846|2430267|2430541|N|N|N|N|N| +2430633|AAAAAAAAJKGBFCAA|1942-09-30|512|2231|172|1942|3|9|30|3|1942|172|2231|Wednesday|1942Q3|N|N|N|2430604|2430846|2430268|2430542|N|N|N|N|N| +2430634|AAAAAAAAKKGBFCAA|1942-10-01|513|2231|172|1942|4|10|1|3|1942|172|2231|Thursday|1942Q3|N|N|N|2430634|2430906|2430269|2430542|N|N|N|N|N| +2430635|AAAAAAAALKGBFCAA|1942-10-02|513|2231|172|1942|5|10|2|4|1942|172|2231|Friday|1942Q4|N|Y|N|2430634|2430906|2430270|2430543|N|N|N|N|N| +2430636|AAAAAAAAMKGBFCAA|1942-10-03|513|2231|172|1942|6|10|3|4|1942|172|2231|Saturday|1942Q4|N|Y|N|2430634|2430906|2430271|2430544|N|N|N|N|N| +2430637|AAAAAAAANKGBFCAA|1942-10-04|513|2231|172|1942|0|10|4|4|1942|172|2231|Sunday|1942Q4|N|N|N|2430634|2430906|2430272|2430545|N|N|N|N|N| +2430638|AAAAAAAAOKGBFCAA|1942-10-05|513|2231|172|1942|1|10|5|4|1942|172|2231|Monday|1942Q4|N|N|N|2430634|2430906|2430273|2430546|N|N|N|N|N| +2430639|AAAAAAAAPKGBFCAA|1942-10-06|513|2232|172|1942|2|10|6|4|1942|172|2232|Tuesday|1942Q4|N|N|N|2430634|2430906|2430274|2430547|N|N|N|N|N| +2430640|AAAAAAAAALGBFCAA|1942-10-07|513|2232|172|1942|3|10|7|4|1942|172|2232|Wednesday|1942Q4|N|N|N|2430634|2430906|2430275|2430548|N|N|N|N|N| +2430641|AAAAAAAABLGBFCAA|1942-10-08|513|2232|172|1942|4|10|8|4|1942|172|2232|Thursday|1942Q4|N|N|N|2430634|2430906|2430276|2430549|N|N|N|N|N| +2430642|AAAAAAAACLGBFCAA|1942-10-09|513|2232|172|1942|5|10|9|4|1942|172|2232|Friday|1942Q4|N|Y|N|2430634|2430906|2430277|2430550|N|N|N|N|N| +2430643|AAAAAAAADLGBFCAA|1942-10-10|513|2232|172|1942|6|10|10|4|1942|172|2232|Saturday|1942Q4|N|Y|N|2430634|2430906|2430278|2430551|N|N|N|N|N| +2430644|AAAAAAAAELGBFCAA|1942-10-11|513|2232|172|1942|0|10|11|4|1942|172|2232|Sunday|1942Q4|N|N|N|2430634|2430906|2430279|2430552|N|N|N|N|N| +2430645|AAAAAAAAFLGBFCAA|1942-10-12|513|2232|172|1942|1|10|12|4|1942|172|2232|Monday|1942Q4|N|N|N|2430634|2430906|2430280|2430553|N|N|N|N|N| +2430646|AAAAAAAAGLGBFCAA|1942-10-13|513|2233|172|1942|2|10|13|4|1942|172|2233|Tuesday|1942Q4|N|N|N|2430634|2430906|2430281|2430554|N|N|N|N|N| +2430647|AAAAAAAAHLGBFCAA|1942-10-14|513|2233|172|1942|3|10|14|4|1942|172|2233|Wednesday|1942Q4|N|N|N|2430634|2430906|2430282|2430555|N|N|N|N|N| +2430648|AAAAAAAAILGBFCAA|1942-10-15|513|2233|172|1942|4|10|15|4|1942|172|2233|Thursday|1942Q4|N|N|N|2430634|2430906|2430283|2430556|N|N|N|N|N| +2430649|AAAAAAAAJLGBFCAA|1942-10-16|513|2233|172|1942|5|10|16|4|1942|172|2233|Friday|1942Q4|N|Y|N|2430634|2430906|2430284|2430557|N|N|N|N|N| +2430650|AAAAAAAAKLGBFCAA|1942-10-17|513|2233|172|1942|6|10|17|4|1942|172|2233|Saturday|1942Q4|N|Y|N|2430634|2430906|2430285|2430558|N|N|N|N|N| +2430651|AAAAAAAALLGBFCAA|1942-10-18|513|2233|172|1942|0|10|18|4|1942|172|2233|Sunday|1942Q4|N|N|N|2430634|2430906|2430286|2430559|N|N|N|N|N| +2430652|AAAAAAAAMLGBFCAA|1942-10-19|513|2233|172|1942|1|10|19|4|1942|172|2233|Monday|1942Q4|N|N|N|2430634|2430906|2430287|2430560|N|N|N|N|N| +2430653|AAAAAAAANLGBFCAA|1942-10-20|513|2234|172|1942|2|10|20|4|1942|172|2234|Tuesday|1942Q4|N|N|N|2430634|2430906|2430288|2430561|N|N|N|N|N| +2430654|AAAAAAAAOLGBFCAA|1942-10-21|513|2234|172|1942|3|10|21|4|1942|172|2234|Wednesday|1942Q4|N|N|N|2430634|2430906|2430289|2430562|N|N|N|N|N| +2430655|AAAAAAAAPLGBFCAA|1942-10-22|513|2234|172|1942|4|10|22|4|1942|172|2234|Thursday|1942Q4|N|N|N|2430634|2430906|2430290|2430563|N|N|N|N|N| +2430656|AAAAAAAAAMGBFCAA|1942-10-23|513|2234|172|1942|5|10|23|4|1942|172|2234|Friday|1942Q4|N|Y|N|2430634|2430906|2430291|2430564|N|N|N|N|N| +2430657|AAAAAAAABMGBFCAA|1942-10-24|513|2234|172|1942|6|10|24|4|1942|172|2234|Saturday|1942Q4|N|Y|N|2430634|2430906|2430292|2430565|N|N|N|N|N| +2430658|AAAAAAAACMGBFCAA|1942-10-25|513|2234|172|1942|0|10|25|4|1942|172|2234|Sunday|1942Q4|N|N|N|2430634|2430906|2430293|2430566|N|N|N|N|N| +2430659|AAAAAAAADMGBFCAA|1942-10-26|513|2234|172|1942|1|10|26|4|1942|172|2234|Monday|1942Q4|N|N|N|2430634|2430906|2430294|2430567|N|N|N|N|N| +2430660|AAAAAAAAEMGBFCAA|1942-10-27|513|2235|172|1942|2|10|27|4|1942|172|2235|Tuesday|1942Q4|N|N|N|2430634|2430906|2430295|2430568|N|N|N|N|N| +2430661|AAAAAAAAFMGBFCAA|1942-10-28|513|2235|172|1942|3|10|28|4|1942|172|2235|Wednesday|1942Q4|N|N|N|2430634|2430906|2430296|2430569|N|N|N|N|N| +2430662|AAAAAAAAGMGBFCAA|1942-10-29|513|2235|172|1942|4|10|29|4|1942|172|2235|Thursday|1942Q4|N|N|N|2430634|2430906|2430297|2430570|N|N|N|N|N| +2430663|AAAAAAAAHMGBFCAA|1942-10-30|513|2235|172|1942|5|10|30|4|1942|172|2235|Friday|1942Q4|N|Y|N|2430634|2430906|2430298|2430571|N|N|N|N|N| +2430664|AAAAAAAAIMGBFCAA|1942-10-31|513|2235|172|1942|6|10|31|4|1942|172|2235|Saturday|1942Q4|N|Y|N|2430634|2430906|2430299|2430572|N|N|N|N|N| +2430665|AAAAAAAAJMGBFCAA|1942-11-01|514|2235|172|1942|0|11|1|4|1942|172|2235|Sunday|1942Q4|N|N|N|2430665|2430968|2430300|2430573|N|N|N|N|N| +2430666|AAAAAAAAKMGBFCAA|1942-11-02|514|2235|172|1942|1|11|2|4|1942|172|2235|Monday|1942Q4|N|N|N|2430665|2430968|2430301|2430574|N|N|N|N|N| +2430667|AAAAAAAALMGBFCAA|1942-11-03|514|2236|172|1942|2|11|3|4|1942|172|2236|Tuesday|1942Q4|N|N|N|2430665|2430968|2430302|2430575|N|N|N|N|N| +2430668|AAAAAAAAMMGBFCAA|1942-11-04|514|2236|172|1942|3|11|4|4|1942|172|2236|Wednesday|1942Q4|N|N|N|2430665|2430968|2430303|2430576|N|N|N|N|N| +2430669|AAAAAAAANMGBFCAA|1942-11-05|514|2236|172|1942|4|11|5|4|1942|172|2236|Thursday|1942Q4|N|N|N|2430665|2430968|2430304|2430577|N|N|N|N|N| +2430670|AAAAAAAAOMGBFCAA|1942-11-06|514|2236|172|1942|5|11|6|4|1942|172|2236|Friday|1942Q4|N|Y|N|2430665|2430968|2430305|2430578|N|N|N|N|N| +2430671|AAAAAAAAPMGBFCAA|1942-11-07|514|2236|172|1942|6|11|7|4|1942|172|2236|Saturday|1942Q4|N|Y|N|2430665|2430968|2430306|2430579|N|N|N|N|N| +2430672|AAAAAAAAANGBFCAA|1942-11-08|514|2236|172|1942|0|11|8|4|1942|172|2236|Sunday|1942Q4|N|N|N|2430665|2430968|2430307|2430580|N|N|N|N|N| +2430673|AAAAAAAABNGBFCAA|1942-11-09|514|2236|172|1942|1|11|9|4|1942|172|2236|Monday|1942Q4|N|N|N|2430665|2430968|2430308|2430581|N|N|N|N|N| +2430674|AAAAAAAACNGBFCAA|1942-11-10|514|2237|172|1942|2|11|10|4|1942|172|2237|Tuesday|1942Q4|N|N|N|2430665|2430968|2430309|2430582|N|N|N|N|N| +2430675|AAAAAAAADNGBFCAA|1942-11-11|514|2237|172|1942|3|11|11|4|1942|172|2237|Wednesday|1942Q4|N|N|N|2430665|2430968|2430310|2430583|N|N|N|N|N| +2430676|AAAAAAAAENGBFCAA|1942-11-12|514|2237|172|1942|4|11|12|4|1942|172|2237|Thursday|1942Q4|N|N|N|2430665|2430968|2430311|2430584|N|N|N|N|N| +2430677|AAAAAAAAFNGBFCAA|1942-11-13|514|2237|172|1942|5|11|13|4|1942|172|2237|Friday|1942Q4|N|Y|N|2430665|2430968|2430312|2430585|N|N|N|N|N| +2430678|AAAAAAAAGNGBFCAA|1942-11-14|514|2237|172|1942|6|11|14|4|1942|172|2237|Saturday|1942Q4|N|Y|N|2430665|2430968|2430313|2430586|N|N|N|N|N| +2430679|AAAAAAAAHNGBFCAA|1942-11-15|514|2237|172|1942|0|11|15|4|1942|172|2237|Sunday|1942Q4|N|N|N|2430665|2430968|2430314|2430587|N|N|N|N|N| +2430680|AAAAAAAAINGBFCAA|1942-11-16|514|2237|172|1942|1|11|16|4|1942|172|2237|Monday|1942Q4|N|N|N|2430665|2430968|2430315|2430588|N|N|N|N|N| +2430681|AAAAAAAAJNGBFCAA|1942-11-17|514|2238|172|1942|2|11|17|4|1942|172|2238|Tuesday|1942Q4|N|N|N|2430665|2430968|2430316|2430589|N|N|N|N|N| +2430682|AAAAAAAAKNGBFCAA|1942-11-18|514|2238|172|1942|3|11|18|4|1942|172|2238|Wednesday|1942Q4|N|N|N|2430665|2430968|2430317|2430590|N|N|N|N|N| +2430683|AAAAAAAALNGBFCAA|1942-11-19|514|2238|172|1942|4|11|19|4|1942|172|2238|Thursday|1942Q4|N|N|N|2430665|2430968|2430318|2430591|N|N|N|N|N| +2430684|AAAAAAAAMNGBFCAA|1942-11-20|514|2238|172|1942|5|11|20|4|1942|172|2238|Friday|1942Q4|N|Y|N|2430665|2430968|2430319|2430592|N|N|N|N|N| +2430685|AAAAAAAANNGBFCAA|1942-11-21|514|2238|172|1942|6|11|21|4|1942|172|2238|Saturday|1942Q4|N|Y|N|2430665|2430968|2430320|2430593|N|N|N|N|N| +2430686|AAAAAAAAONGBFCAA|1942-11-22|514|2238|172|1942|0|11|22|4|1942|172|2238|Sunday|1942Q4|N|N|N|2430665|2430968|2430321|2430594|N|N|N|N|N| +2430687|AAAAAAAAPNGBFCAA|1942-11-23|514|2238|172|1942|1|11|23|4|1942|172|2238|Monday|1942Q4|N|N|N|2430665|2430968|2430322|2430595|N|N|N|N|N| +2430688|AAAAAAAAAOGBFCAA|1942-11-24|514|2239|172|1942|2|11|24|4|1942|172|2239|Tuesday|1942Q4|N|N|N|2430665|2430968|2430323|2430596|N|N|N|N|N| +2430689|AAAAAAAABOGBFCAA|1942-11-25|514|2239|172|1942|3|11|25|4|1942|172|2239|Wednesday|1942Q4|N|N|N|2430665|2430968|2430324|2430597|N|N|N|N|N| +2430690|AAAAAAAACOGBFCAA|1942-11-26|514|2239|172|1942|4|11|26|4|1942|172|2239|Thursday|1942Q4|N|N|N|2430665|2430968|2430325|2430598|N|N|N|N|N| +2430691|AAAAAAAADOGBFCAA|1942-11-27|514|2239|172|1942|5|11|27|4|1942|172|2239|Friday|1942Q4|N|Y|N|2430665|2430968|2430326|2430599|N|N|N|N|N| +2430692|AAAAAAAAEOGBFCAA|1942-11-28|514|2239|172|1942|6|11|28|4|1942|172|2239|Saturday|1942Q4|N|Y|N|2430665|2430968|2430327|2430600|N|N|N|N|N| +2430693|AAAAAAAAFOGBFCAA|1942-11-29|514|2239|172|1942|0|11|29|4|1942|172|2239|Sunday|1942Q4|N|N|N|2430665|2430968|2430328|2430601|N|N|N|N|N| +2430694|AAAAAAAAGOGBFCAA|1942-11-30|514|2239|172|1942|1|11|30|4|1942|172|2239|Monday|1942Q4|N|N|N|2430665|2430968|2430329|2430602|N|N|N|N|N| +2430695|AAAAAAAAHOGBFCAA|1942-12-01|515|2240|173|1942|2|12|1|4|1942|173|2240|Tuesday|1942Q4|N|N|N|2430695|2431028|2430330|2430603|N|N|N|N|N| +2430696|AAAAAAAAIOGBFCAA|1942-12-02|515|2240|173|1942|3|12|2|4|1942|173|2240|Wednesday|1942Q4|N|N|N|2430695|2431028|2430331|2430604|N|N|N|N|N| +2430697|AAAAAAAAJOGBFCAA|1942-12-03|515|2240|173|1942|4|12|3|4|1942|173|2240|Thursday|1942Q4|N|N|N|2430695|2431028|2430332|2430605|N|N|N|N|N| +2430698|AAAAAAAAKOGBFCAA|1942-12-04|515|2240|173|1942|5|12|4|4|1942|173|2240|Friday|1942Q4|N|Y|N|2430695|2431028|2430333|2430606|N|N|N|N|N| +2430699|AAAAAAAALOGBFCAA|1942-12-05|515|2240|173|1942|6|12|5|4|1942|173|2240|Saturday|1942Q4|N|Y|N|2430695|2431028|2430334|2430607|N|N|N|N|N| +2430700|AAAAAAAAMOGBFCAA|1942-12-06|515|2240|173|1942|0|12|6|4|1942|173|2240|Sunday|1942Q4|N|N|N|2430695|2431028|2430335|2430608|N|N|N|N|N| +2430701|AAAAAAAANOGBFCAA|1942-12-07|515|2240|173|1942|1|12|7|4|1942|173|2240|Monday|1942Q4|N|N|N|2430695|2431028|2430336|2430609|N|N|N|N|N| +2430702|AAAAAAAAOOGBFCAA|1942-12-08|515|2241|173|1942|2|12|8|4|1942|173|2241|Tuesday|1942Q4|N|N|N|2430695|2431028|2430337|2430610|N|N|N|N|N| +2430703|AAAAAAAAPOGBFCAA|1942-12-09|515|2241|173|1942|3|12|9|4|1942|173|2241|Wednesday|1942Q4|N|N|N|2430695|2431028|2430338|2430611|N|N|N|N|N| +2430704|AAAAAAAAAPGBFCAA|1942-12-10|515|2241|173|1942|4|12|10|4|1942|173|2241|Thursday|1942Q4|N|N|N|2430695|2431028|2430339|2430612|N|N|N|N|N| +2430705|AAAAAAAABPGBFCAA|1942-12-11|515|2241|173|1942|5|12|11|4|1942|173|2241|Friday|1942Q4|N|Y|N|2430695|2431028|2430340|2430613|N|N|N|N|N| +2430706|AAAAAAAACPGBFCAA|1942-12-12|515|2241|173|1942|6|12|12|4|1942|173|2241|Saturday|1942Q4|N|Y|N|2430695|2431028|2430341|2430614|N|N|N|N|N| +2430707|AAAAAAAADPGBFCAA|1942-12-13|515|2241|173|1942|0|12|13|4|1942|173|2241|Sunday|1942Q4|N|N|N|2430695|2431028|2430342|2430615|N|N|N|N|N| +2430708|AAAAAAAAEPGBFCAA|1942-12-14|515|2241|173|1942|1|12|14|4|1942|173|2241|Monday|1942Q4|N|N|N|2430695|2431028|2430343|2430616|N|N|N|N|N| +2430709|AAAAAAAAFPGBFCAA|1942-12-15|515|2242|173|1942|2|12|15|4|1942|173|2242|Tuesday|1942Q4|N|N|N|2430695|2431028|2430344|2430617|N|N|N|N|N| +2430710|AAAAAAAAGPGBFCAA|1942-12-16|515|2242|173|1942|3|12|16|4|1942|173|2242|Wednesday|1942Q4|N|N|N|2430695|2431028|2430345|2430618|N|N|N|N|N| +2430711|AAAAAAAAHPGBFCAA|1942-12-17|515|2242|173|1942|4|12|17|4|1942|173|2242|Thursday|1942Q4|N|N|N|2430695|2431028|2430346|2430619|N|N|N|N|N| +2430712|AAAAAAAAIPGBFCAA|1942-12-18|515|2242|173|1942|5|12|18|4|1942|173|2242|Friday|1942Q4|N|Y|N|2430695|2431028|2430347|2430620|N|N|N|N|N| +2430713|AAAAAAAAJPGBFCAA|1942-12-19|515|2242|173|1942|6|12|19|4|1942|173|2242|Saturday|1942Q4|N|Y|N|2430695|2431028|2430348|2430621|N|N|N|N|N| +2430714|AAAAAAAAKPGBFCAA|1942-12-20|515|2242|173|1942|0|12|20|4|1942|173|2242|Sunday|1942Q4|N|N|N|2430695|2431028|2430349|2430622|N|N|N|N|N| +2430715|AAAAAAAALPGBFCAA|1942-12-21|515|2242|173|1942|1|12|21|4|1942|173|2242|Monday|1942Q4|N|N|N|2430695|2431028|2430350|2430623|N|N|N|N|N| +2430716|AAAAAAAAMPGBFCAA|1942-12-22|515|2243|173|1942|2|12|22|4|1942|173|2243|Tuesday|1942Q4|N|N|N|2430695|2431028|2430351|2430624|N|N|N|N|N| +2430717|AAAAAAAANPGBFCAA|1942-12-23|515|2243|173|1942|3|12|23|4|1942|173|2243|Wednesday|1942Q4|N|N|N|2430695|2431028|2430352|2430625|N|N|N|N|N| +2430718|AAAAAAAAOPGBFCAA|1942-12-24|515|2243|173|1942|4|12|24|4|1942|173|2243|Thursday|1942Q4|N|N|N|2430695|2431028|2430353|2430626|N|N|N|N|N| +2430719|AAAAAAAAPPGBFCAA|1942-12-25|515|2243|173|1942|5|12|25|4|1942|173|2243|Friday|1942Q4|N|Y|N|2430695|2431028|2430354|2430627|N|N|N|N|N| +2430720|AAAAAAAAAAHBFCAA|1942-12-26|515|2243|173|1942|6|12|26|4|1942|173|2243|Saturday|1942Q4|Y|Y|N|2430695|2431028|2430355|2430628|N|N|N|N|N| +2430721|AAAAAAAABAHBFCAA|1942-12-27|515|2243|173|1942|0|12|27|4|1942|173|2243|Sunday|1942Q4|N|N|Y|2430695|2431028|2430356|2430629|N|N|N|N|N| +2430722|AAAAAAAACAHBFCAA|1942-12-28|515|2243|173|1942|1|12|28|4|1942|173|2243|Monday|1942Q4|N|N|N|2430695|2431028|2430357|2430630|N|N|N|N|N| +2430723|AAAAAAAADAHBFCAA|1942-12-29|515|2244|173|1942|2|12|29|4|1942|173|2244|Tuesday|1942Q4|N|N|N|2430695|2431028|2430358|2430631|N|N|N|N|N| +2430724|AAAAAAAAEAHBFCAA|1942-12-30|515|2244|173|1942|3|12|30|4|1942|173|2244|Wednesday|1942Q4|N|N|N|2430695|2431028|2430359|2430632|N|N|N|N|N| +2430725|AAAAAAAAFAHBFCAA|1942-12-31|515|2244|173|1942|4|12|31|4|1942|173|2244|Thursday|1942Q4|N|N|N|2430695|2431028|2430360|2430633|N|N|N|N|N| +2430726|AAAAAAAAGAHBFCAA|1943-01-01|516|2244|173|1943|5|1|1|1|1943|173|2244|Friday|1943Q1|Y|Y|N|2430726|2430725|2430361|2430634|N|N|N|N|N| +2430727|AAAAAAAAHAHBFCAA|1943-01-02|516|2244|173|1943|6|1|2|1|1943|173|2244|Saturday|1943Q1|N|Y|Y|2430726|2430725|2430362|2430635|N|N|N|N|N| +2430728|AAAAAAAAIAHBFCAA|1943-01-03|516|2244|173|1943|0|1|3|1|1943|173|2244|Sunday|1943Q1|N|N|N|2430726|2430725|2430363|2430636|N|N|N|N|N| +2430729|AAAAAAAAJAHBFCAA|1943-01-04|516|2244|173|1943|1|1|4|1|1943|173|2244|Monday|1943Q1|N|N|N|2430726|2430725|2430364|2430637|N|N|N|N|N| +2430730|AAAAAAAAKAHBFCAA|1943-01-05|516|2245|173|1943|2|1|5|1|1943|173|2245|Tuesday|1943Q1|N|N|N|2430726|2430725|2430365|2430638|N|N|N|N|N| +2430731|AAAAAAAALAHBFCAA|1943-01-06|516|2245|173|1943|3|1|6|1|1943|173|2245|Wednesday|1943Q1|N|N|N|2430726|2430725|2430366|2430639|N|N|N|N|N| +2430732|AAAAAAAAMAHBFCAA|1943-01-07|516|2245|173|1943|4|1|7|1|1943|173|2245|Thursday|1943Q1|N|N|N|2430726|2430725|2430367|2430640|N|N|N|N|N| +2430733|AAAAAAAANAHBFCAA|1943-01-08|516|2245|173|1943|5|1|8|1|1943|173|2245|Friday|1943Q1|N|Y|N|2430726|2430725|2430368|2430641|N|N|N|N|N| +2430734|AAAAAAAAOAHBFCAA|1943-01-09|516|2245|173|1943|6|1|9|1|1943|173|2245|Saturday|1943Q1|N|Y|N|2430726|2430725|2430369|2430642|N|N|N|N|N| +2430735|AAAAAAAAPAHBFCAA|1943-01-10|516|2245|173|1943|0|1|10|1|1943|173|2245|Sunday|1943Q1|N|N|N|2430726|2430725|2430370|2430643|N|N|N|N|N| +2430736|AAAAAAAAABHBFCAA|1943-01-11|516|2245|173|1943|1|1|11|1|1943|173|2245|Monday|1943Q1|N|N|N|2430726|2430725|2430371|2430644|N|N|N|N|N| +2430737|AAAAAAAABBHBFCAA|1943-01-12|516|2246|173|1943|2|1|12|1|1943|173|2246|Tuesday|1943Q1|N|N|N|2430726|2430725|2430372|2430645|N|N|N|N|N| +2430738|AAAAAAAACBHBFCAA|1943-01-13|516|2246|173|1943|3|1|13|1|1943|173|2246|Wednesday|1943Q1|N|N|N|2430726|2430725|2430373|2430646|N|N|N|N|N| +2430739|AAAAAAAADBHBFCAA|1943-01-14|516|2246|173|1943|4|1|14|1|1943|173|2246|Thursday|1943Q1|N|N|N|2430726|2430725|2430374|2430647|N|N|N|N|N| +2430740|AAAAAAAAEBHBFCAA|1943-01-15|516|2246|173|1943|5|1|15|1|1943|173|2246|Friday|1943Q1|N|Y|N|2430726|2430725|2430375|2430648|N|N|N|N|N| +2430741|AAAAAAAAFBHBFCAA|1943-01-16|516|2246|173|1943|6|1|16|1|1943|173|2246|Saturday|1943Q1|N|Y|N|2430726|2430725|2430376|2430649|N|N|N|N|N| +2430742|AAAAAAAAGBHBFCAA|1943-01-17|516|2246|173|1943|0|1|17|1|1943|173|2246|Sunday|1943Q1|N|N|N|2430726|2430725|2430377|2430650|N|N|N|N|N| +2430743|AAAAAAAAHBHBFCAA|1943-01-18|516|2246|173|1943|1|1|18|1|1943|173|2246|Monday|1943Q1|N|N|N|2430726|2430725|2430378|2430651|N|N|N|N|N| +2430744|AAAAAAAAIBHBFCAA|1943-01-19|516|2247|173|1943|2|1|19|1|1943|173|2247|Tuesday|1943Q1|N|N|N|2430726|2430725|2430379|2430652|N|N|N|N|N| +2430745|AAAAAAAAJBHBFCAA|1943-01-20|516|2247|173|1943|3|1|20|1|1943|173|2247|Wednesday|1943Q1|N|N|N|2430726|2430725|2430380|2430653|N|N|N|N|N| +2430746|AAAAAAAAKBHBFCAA|1943-01-21|516|2247|173|1943|4|1|21|1|1943|173|2247|Thursday|1943Q1|N|N|N|2430726|2430725|2430381|2430654|N|N|N|N|N| +2430747|AAAAAAAALBHBFCAA|1943-01-22|516|2247|173|1943|5|1|22|1|1943|173|2247|Friday|1943Q1|N|Y|N|2430726|2430725|2430382|2430655|N|N|N|N|N| +2430748|AAAAAAAAMBHBFCAA|1943-01-23|516|2247|173|1943|6|1|23|1|1943|173|2247|Saturday|1943Q1|N|Y|N|2430726|2430725|2430383|2430656|N|N|N|N|N| +2430749|AAAAAAAANBHBFCAA|1943-01-24|516|2247|173|1943|0|1|24|1|1943|173|2247|Sunday|1943Q1|N|N|N|2430726|2430725|2430384|2430657|N|N|N|N|N| +2430750|AAAAAAAAOBHBFCAA|1943-01-25|516|2247|173|1943|1|1|25|1|1943|173|2247|Monday|1943Q1|N|N|N|2430726|2430725|2430385|2430658|N|N|N|N|N| +2430751|AAAAAAAAPBHBFCAA|1943-01-26|516|2248|173|1943|2|1|26|1|1943|173|2248|Tuesday|1943Q1|N|N|N|2430726|2430725|2430386|2430659|N|N|N|N|N| +2430752|AAAAAAAAACHBFCAA|1943-01-27|516|2248|173|1943|3|1|27|1|1943|173|2248|Wednesday|1943Q1|N|N|N|2430726|2430725|2430387|2430660|N|N|N|N|N| +2430753|AAAAAAAABCHBFCAA|1943-01-28|516|2248|173|1943|4|1|28|1|1943|173|2248|Thursday|1943Q1|N|N|N|2430726|2430725|2430388|2430661|N|N|N|N|N| +2430754|AAAAAAAACCHBFCAA|1943-01-29|516|2248|173|1943|5|1|29|1|1943|173|2248|Friday|1943Q1|N|Y|N|2430726|2430725|2430389|2430662|N|N|N|N|N| +2430755|AAAAAAAADCHBFCAA|1943-01-30|516|2248|173|1943|6|1|30|1|1943|173|2248|Saturday|1943Q1|N|Y|N|2430726|2430725|2430390|2430663|N|N|N|N|N| +2430756|AAAAAAAAECHBFCAA|1943-01-31|516|2248|173|1943|0|1|31|1|1943|173|2248|Sunday|1943Q1|N|N|N|2430726|2430725|2430391|2430664|N|N|N|N|N| +2430757|AAAAAAAAFCHBFCAA|1943-02-01|517|2248|173|1943|1|2|1|1|1943|173|2248|Monday|1943Q1|N|N|N|2430757|2430787|2430392|2430665|N|N|N|N|N| +2430758|AAAAAAAAGCHBFCAA|1943-02-02|517|2249|173|1943|2|2|2|1|1943|173|2249|Tuesday|1943Q1|N|N|N|2430757|2430787|2430393|2430666|N|N|N|N|N| +2430759|AAAAAAAAHCHBFCAA|1943-02-03|517|2249|173|1943|3|2|3|1|1943|173|2249|Wednesday|1943Q1|N|N|N|2430757|2430787|2430394|2430667|N|N|N|N|N| +2430760|AAAAAAAAICHBFCAA|1943-02-04|517|2249|173|1943|4|2|4|1|1943|173|2249|Thursday|1943Q1|N|N|N|2430757|2430787|2430395|2430668|N|N|N|N|N| +2430761|AAAAAAAAJCHBFCAA|1943-02-05|517|2249|173|1943|5|2|5|1|1943|173|2249|Friday|1943Q1|N|Y|N|2430757|2430787|2430396|2430669|N|N|N|N|N| +2430762|AAAAAAAAKCHBFCAA|1943-02-06|517|2249|173|1943|6|2|6|1|1943|173|2249|Saturday|1943Q1|N|Y|N|2430757|2430787|2430397|2430670|N|N|N|N|N| +2430763|AAAAAAAALCHBFCAA|1943-02-07|517|2249|173|1943|0|2|7|1|1943|173|2249|Sunday|1943Q1|N|N|N|2430757|2430787|2430398|2430671|N|N|N|N|N| +2430764|AAAAAAAAMCHBFCAA|1943-02-08|517|2249|173|1943|1|2|8|1|1943|173|2249|Monday|1943Q1|N|N|N|2430757|2430787|2430399|2430672|N|N|N|N|N| +2430765|AAAAAAAANCHBFCAA|1943-02-09|517|2250|173|1943|2|2|9|1|1943|173|2250|Tuesday|1943Q1|N|N|N|2430757|2430787|2430400|2430673|N|N|N|N|N| +2430766|AAAAAAAAOCHBFCAA|1943-02-10|517|2250|173|1943|3|2|10|1|1943|173|2250|Wednesday|1943Q1|N|N|N|2430757|2430787|2430401|2430674|N|N|N|N|N| +2430767|AAAAAAAAPCHBFCAA|1943-02-11|517|2250|173|1943|4|2|11|1|1943|173|2250|Thursday|1943Q1|N|N|N|2430757|2430787|2430402|2430675|N|N|N|N|N| +2430768|AAAAAAAAADHBFCAA|1943-02-12|517|2250|173|1943|5|2|12|1|1943|173|2250|Friday|1943Q1|N|Y|N|2430757|2430787|2430403|2430676|N|N|N|N|N| +2430769|AAAAAAAABDHBFCAA|1943-02-13|517|2250|173|1943|6|2|13|1|1943|173|2250|Saturday|1943Q1|N|Y|N|2430757|2430787|2430404|2430677|N|N|N|N|N| +2430770|AAAAAAAACDHBFCAA|1943-02-14|517|2250|173|1943|0|2|14|1|1943|173|2250|Sunday|1943Q1|N|N|N|2430757|2430787|2430405|2430678|N|N|N|N|N| +2430771|AAAAAAAADDHBFCAA|1943-02-15|517|2250|173|1943|1|2|15|1|1943|173|2250|Monday|1943Q1|N|N|N|2430757|2430787|2430406|2430679|N|N|N|N|N| +2430772|AAAAAAAAEDHBFCAA|1943-02-16|517|2251|173|1943|2|2|16|1|1943|173|2251|Tuesday|1943Q1|N|N|N|2430757|2430787|2430407|2430680|N|N|N|N|N| +2430773|AAAAAAAAFDHBFCAA|1943-02-17|517|2251|173|1943|3|2|17|1|1943|173|2251|Wednesday|1943Q1|N|N|N|2430757|2430787|2430408|2430681|N|N|N|N|N| +2430774|AAAAAAAAGDHBFCAA|1943-02-18|517|2251|173|1943|4|2|18|1|1943|173|2251|Thursday|1943Q1|N|N|N|2430757|2430787|2430409|2430682|N|N|N|N|N| +2430775|AAAAAAAAHDHBFCAA|1943-02-19|517|2251|173|1943|5|2|19|1|1943|173|2251|Friday|1943Q1|N|Y|N|2430757|2430787|2430410|2430683|N|N|N|N|N| +2430776|AAAAAAAAIDHBFCAA|1943-02-20|517|2251|173|1943|6|2|20|1|1943|173|2251|Saturday|1943Q1|N|Y|N|2430757|2430787|2430411|2430684|N|N|N|N|N| +2430777|AAAAAAAAJDHBFCAA|1943-02-21|517|2251|173|1943|0|2|21|1|1943|173|2251|Sunday|1943Q1|N|N|N|2430757|2430787|2430412|2430685|N|N|N|N|N| +2430778|AAAAAAAAKDHBFCAA|1943-02-22|517|2251|173|1943|1|2|22|1|1943|173|2251|Monday|1943Q1|N|N|N|2430757|2430787|2430413|2430686|N|N|N|N|N| +2430779|AAAAAAAALDHBFCAA|1943-02-23|517|2252|173|1943|2|2|23|1|1943|173|2252|Tuesday|1943Q1|N|N|N|2430757|2430787|2430414|2430687|N|N|N|N|N| +2430780|AAAAAAAAMDHBFCAA|1943-02-24|517|2252|173|1943|3|2|24|1|1943|173|2252|Wednesday|1943Q1|N|N|N|2430757|2430787|2430415|2430688|N|N|N|N|N| +2430781|AAAAAAAANDHBFCAA|1943-02-25|517|2252|173|1943|4|2|25|1|1943|173|2252|Thursday|1943Q1|N|N|N|2430757|2430787|2430416|2430689|N|N|N|N|N| +2430782|AAAAAAAAODHBFCAA|1943-02-26|517|2252|173|1943|5|2|26|1|1943|173|2252|Friday|1943Q1|N|Y|N|2430757|2430787|2430417|2430690|N|N|N|N|N| +2430783|AAAAAAAAPDHBFCAA|1943-02-27|517|2252|173|1943|6|2|27|1|1943|173|2252|Saturday|1943Q1|N|Y|N|2430757|2430787|2430418|2430691|N|N|N|N|N| +2430784|AAAAAAAAAEHBFCAA|1943-02-28|517|2252|173|1943|0|2|28|1|1943|173|2252|Sunday|1943Q1|N|N|N|2430757|2430787|2430419|2430692|N|N|N|N|N| +2430785|AAAAAAAABEHBFCAA|1943-03-01|518|2252|174|1943|1|3|1|1|1943|174|2252|Monday|1943Q1|N|N|N|2430785|2430843|2430420|2430693|N|N|N|N|N| +2430786|AAAAAAAACEHBFCAA|1943-03-02|518|2253|174|1943|2|3|2|1|1943|174|2253|Tuesday|1943Q1|N|N|N|2430785|2430843|2430421|2430694|N|N|N|N|N| +2430787|AAAAAAAADEHBFCAA|1943-03-03|518|2253|174|1943|3|3|3|1|1943|174|2253|Wednesday|1943Q1|N|N|N|2430785|2430843|2430422|2430695|N|N|N|N|N| +2430788|AAAAAAAAEEHBFCAA|1943-03-04|518|2253|174|1943|4|3|4|1|1943|174|2253|Thursday|1943Q1|N|N|N|2430785|2430843|2430423|2430696|N|N|N|N|N| +2430789|AAAAAAAAFEHBFCAA|1943-03-05|518|2253|174|1943|5|3|5|1|1943|174|2253|Friday|1943Q1|N|Y|N|2430785|2430843|2430424|2430697|N|N|N|N|N| +2430790|AAAAAAAAGEHBFCAA|1943-03-06|518|2253|174|1943|6|3|6|1|1943|174|2253|Saturday|1943Q1|N|Y|N|2430785|2430843|2430425|2430698|N|N|N|N|N| +2430791|AAAAAAAAHEHBFCAA|1943-03-07|518|2253|174|1943|0|3|7|1|1943|174|2253|Sunday|1943Q1|N|N|N|2430785|2430843|2430426|2430699|N|N|N|N|N| +2430792|AAAAAAAAIEHBFCAA|1943-03-08|518|2253|174|1943|1|3|8|1|1943|174|2253|Monday|1943Q1|N|N|N|2430785|2430843|2430427|2430700|N|N|N|N|N| +2430793|AAAAAAAAJEHBFCAA|1943-03-09|518|2254|174|1943|2|3|9|1|1943|174|2254|Tuesday|1943Q1|N|N|N|2430785|2430843|2430428|2430701|N|N|N|N|N| +2430794|AAAAAAAAKEHBFCAA|1943-03-10|518|2254|174|1943|3|3|10|1|1943|174|2254|Wednesday|1943Q1|N|N|N|2430785|2430843|2430429|2430702|N|N|N|N|N| +2430795|AAAAAAAALEHBFCAA|1943-03-11|518|2254|174|1943|4|3|11|1|1943|174|2254|Thursday|1943Q1|N|N|N|2430785|2430843|2430430|2430703|N|N|N|N|N| +2430796|AAAAAAAAMEHBFCAA|1943-03-12|518|2254|174|1943|5|3|12|1|1943|174|2254|Friday|1943Q1|N|Y|N|2430785|2430843|2430431|2430704|N|N|N|N|N| +2430797|AAAAAAAANEHBFCAA|1943-03-13|518|2254|174|1943|6|3|13|1|1943|174|2254|Saturday|1943Q1|N|Y|N|2430785|2430843|2430432|2430705|N|N|N|N|N| +2430798|AAAAAAAAOEHBFCAA|1943-03-14|518|2254|174|1943|0|3|14|1|1943|174|2254|Sunday|1943Q1|N|N|N|2430785|2430843|2430433|2430706|N|N|N|N|N| +2430799|AAAAAAAAPEHBFCAA|1943-03-15|518|2254|174|1943|1|3|15|1|1943|174|2254|Monday|1943Q1|N|N|N|2430785|2430843|2430434|2430707|N|N|N|N|N| +2430800|AAAAAAAAAFHBFCAA|1943-03-16|518|2255|174|1943|2|3|16|1|1943|174|2255|Tuesday|1943Q1|N|N|N|2430785|2430843|2430435|2430708|N|N|N|N|N| +2430801|AAAAAAAABFHBFCAA|1943-03-17|518|2255|174|1943|3|3|17|1|1943|174|2255|Wednesday|1943Q1|N|N|N|2430785|2430843|2430436|2430709|N|N|N|N|N| +2430802|AAAAAAAACFHBFCAA|1943-03-18|518|2255|174|1943|4|3|18|1|1943|174|2255|Thursday|1943Q1|N|N|N|2430785|2430843|2430437|2430710|N|N|N|N|N| +2430803|AAAAAAAADFHBFCAA|1943-03-19|518|2255|174|1943|5|3|19|1|1943|174|2255|Friday|1943Q1|N|Y|N|2430785|2430843|2430438|2430711|N|N|N|N|N| +2430804|AAAAAAAAEFHBFCAA|1943-03-20|518|2255|174|1943|6|3|20|1|1943|174|2255|Saturday|1943Q1|N|Y|N|2430785|2430843|2430439|2430712|N|N|N|N|N| +2430805|AAAAAAAAFFHBFCAA|1943-03-21|518|2255|174|1943|0|3|21|1|1943|174|2255|Sunday|1943Q1|N|N|N|2430785|2430843|2430440|2430713|N|N|N|N|N| +2430806|AAAAAAAAGFHBFCAA|1943-03-22|518|2255|174|1943|1|3|22|1|1943|174|2255|Monday|1943Q1|N|N|N|2430785|2430843|2430441|2430714|N|N|N|N|N| +2430807|AAAAAAAAHFHBFCAA|1943-03-23|518|2256|174|1943|2|3|23|1|1943|174|2256|Tuesday|1943Q1|N|N|N|2430785|2430843|2430442|2430715|N|N|N|N|N| +2430808|AAAAAAAAIFHBFCAA|1943-03-24|518|2256|174|1943|3|3|24|1|1943|174|2256|Wednesday|1943Q1|N|N|N|2430785|2430843|2430443|2430716|N|N|N|N|N| +2430809|AAAAAAAAJFHBFCAA|1943-03-25|518|2256|174|1943|4|3|25|1|1943|174|2256|Thursday|1943Q1|N|N|N|2430785|2430843|2430444|2430717|N|N|N|N|N| +2430810|AAAAAAAAKFHBFCAA|1943-03-26|518|2256|174|1943|5|3|26|1|1943|174|2256|Friday|1943Q1|N|Y|N|2430785|2430843|2430445|2430718|N|N|N|N|N| +2430811|AAAAAAAALFHBFCAA|1943-03-27|518|2256|174|1943|6|3|27|1|1943|174|2256|Saturday|1943Q1|N|Y|N|2430785|2430843|2430446|2430719|N|N|N|N|N| +2430812|AAAAAAAAMFHBFCAA|1943-03-28|518|2256|174|1943|0|3|28|1|1943|174|2256|Sunday|1943Q1|N|N|N|2430785|2430843|2430447|2430720|N|N|N|N|N| +2430813|AAAAAAAANFHBFCAA|1943-03-29|518|2256|174|1943|1|3|29|1|1943|174|2256|Monday|1943Q1|N|N|N|2430785|2430843|2430448|2430721|N|N|N|N|N| +2430814|AAAAAAAAOFHBFCAA|1943-03-30|518|2257|174|1943|2|3|30|1|1943|174|2257|Tuesday|1943Q1|N|N|N|2430785|2430843|2430449|2430722|N|N|N|N|N| +2430815|AAAAAAAAPFHBFCAA|1943-03-31|518|2257|174|1943|3|3|31|1|1943|174|2257|Wednesday|1943Q1|N|N|N|2430785|2430843|2430450|2430723|N|N|N|N|N| +2430816|AAAAAAAAAGHBFCAA|1943-04-01|519|2257|174|1943|4|4|1|1|1943|174|2257|Thursday|1943Q1|N|N|N|2430816|2430905|2430451|2430726|N|N|N|N|N| +2430817|AAAAAAAABGHBFCAA|1943-04-02|519|2257|174|1943|5|4|2|2|1943|174|2257|Friday|1943Q2|N|Y|N|2430816|2430905|2430452|2430727|N|N|N|N|N| +2430818|AAAAAAAACGHBFCAA|1943-04-03|519|2257|174|1943|6|4|3|2|1943|174|2257|Saturday|1943Q2|N|Y|N|2430816|2430905|2430453|2430728|N|N|N|N|N| +2430819|AAAAAAAADGHBFCAA|1943-04-04|519|2257|174|1943|0|4|4|2|1943|174|2257|Sunday|1943Q2|N|N|N|2430816|2430905|2430454|2430729|N|N|N|N|N| +2430820|AAAAAAAAEGHBFCAA|1943-04-05|519|2257|174|1943|1|4|5|2|1943|174|2257|Monday|1943Q2|N|N|N|2430816|2430905|2430455|2430730|N|N|N|N|N| +2430821|AAAAAAAAFGHBFCAA|1943-04-06|519|2258|174|1943|2|4|6|2|1943|174|2258|Tuesday|1943Q2|N|N|N|2430816|2430905|2430456|2430731|N|N|N|N|N| +2430822|AAAAAAAAGGHBFCAA|1943-04-07|519|2258|174|1943|3|4|7|2|1943|174|2258|Wednesday|1943Q2|N|N|N|2430816|2430905|2430457|2430732|N|N|N|N|N| +2430823|AAAAAAAAHGHBFCAA|1943-04-08|519|2258|174|1943|4|4|8|2|1943|174|2258|Thursday|1943Q2|N|N|N|2430816|2430905|2430458|2430733|N|N|N|N|N| +2430824|AAAAAAAAIGHBFCAA|1943-04-09|519|2258|174|1943|5|4|9|2|1943|174|2258|Friday|1943Q2|N|Y|N|2430816|2430905|2430459|2430734|N|N|N|N|N| +2430825|AAAAAAAAJGHBFCAA|1943-04-10|519|2258|174|1943|6|4|10|2|1943|174|2258|Saturday|1943Q2|N|Y|N|2430816|2430905|2430460|2430735|N|N|N|N|N| +2430826|AAAAAAAAKGHBFCAA|1943-04-11|519|2258|174|1943|0|4|11|2|1943|174|2258|Sunday|1943Q2|N|N|N|2430816|2430905|2430461|2430736|N|N|N|N|N| +2430827|AAAAAAAALGHBFCAA|1943-04-12|519|2258|174|1943|1|4|12|2|1943|174|2258|Monday|1943Q2|N|N|N|2430816|2430905|2430462|2430737|N|N|N|N|N| +2430828|AAAAAAAAMGHBFCAA|1943-04-13|519|2259|174|1943|2|4|13|2|1943|174|2259|Tuesday|1943Q2|N|N|N|2430816|2430905|2430463|2430738|N|N|N|N|N| +2430829|AAAAAAAANGHBFCAA|1943-04-14|519|2259|174|1943|3|4|14|2|1943|174|2259|Wednesday|1943Q2|N|N|N|2430816|2430905|2430464|2430739|N|N|N|N|N| +2430830|AAAAAAAAOGHBFCAA|1943-04-15|519|2259|174|1943|4|4|15|2|1943|174|2259|Thursday|1943Q2|N|N|N|2430816|2430905|2430465|2430740|N|N|N|N|N| +2430831|AAAAAAAAPGHBFCAA|1943-04-16|519|2259|174|1943|5|4|16|2|1943|174|2259|Friday|1943Q2|N|Y|N|2430816|2430905|2430466|2430741|N|N|N|N|N| +2430832|AAAAAAAAAHHBFCAA|1943-04-17|519|2259|174|1943|6|4|17|2|1943|174|2259|Saturday|1943Q2|N|Y|N|2430816|2430905|2430467|2430742|N|N|N|N|N| +2430833|AAAAAAAABHHBFCAA|1943-04-18|519|2259|174|1943|0|4|18|2|1943|174|2259|Sunday|1943Q2|N|N|N|2430816|2430905|2430468|2430743|N|N|N|N|N| +2430834|AAAAAAAACHHBFCAA|1943-04-19|519|2259|174|1943|1|4|19|2|1943|174|2259|Monday|1943Q2|N|N|N|2430816|2430905|2430469|2430744|N|N|N|N|N| +2430835|AAAAAAAADHHBFCAA|1943-04-20|519|2260|174|1943|2|4|20|2|1943|174|2260|Tuesday|1943Q2|N|N|N|2430816|2430905|2430470|2430745|N|N|N|N|N| +2430836|AAAAAAAAEHHBFCAA|1943-04-21|519|2260|174|1943|3|4|21|2|1943|174|2260|Wednesday|1943Q2|N|N|N|2430816|2430905|2430471|2430746|N|N|N|N|N| +2430837|AAAAAAAAFHHBFCAA|1943-04-22|519|2260|174|1943|4|4|22|2|1943|174|2260|Thursday|1943Q2|N|N|N|2430816|2430905|2430472|2430747|N|N|N|N|N| +2430838|AAAAAAAAGHHBFCAA|1943-04-23|519|2260|174|1943|5|4|23|2|1943|174|2260|Friday|1943Q2|N|Y|N|2430816|2430905|2430473|2430748|N|N|N|N|N| +2430839|AAAAAAAAHHHBFCAA|1943-04-24|519|2260|174|1943|6|4|24|2|1943|174|2260|Saturday|1943Q2|N|Y|N|2430816|2430905|2430474|2430749|N|N|N|N|N| +2430840|AAAAAAAAIHHBFCAA|1943-04-25|519|2260|174|1943|0|4|25|2|1943|174|2260|Sunday|1943Q2|N|N|N|2430816|2430905|2430475|2430750|N|N|N|N|N| +2430841|AAAAAAAAJHHBFCAA|1943-04-26|519|2260|174|1943|1|4|26|2|1943|174|2260|Monday|1943Q2|N|N|N|2430816|2430905|2430476|2430751|N|N|N|N|N| +2430842|AAAAAAAAKHHBFCAA|1943-04-27|519|2261|174|1943|2|4|27|2|1943|174|2261|Tuesday|1943Q2|N|N|N|2430816|2430905|2430477|2430752|N|N|N|N|N| +2430843|AAAAAAAALHHBFCAA|1943-04-28|519|2261|174|1943|3|4|28|2|1943|174|2261|Wednesday|1943Q2|N|N|N|2430816|2430905|2430478|2430753|N|N|N|N|N| +2430844|AAAAAAAAMHHBFCAA|1943-04-29|519|2261|174|1943|4|4|29|2|1943|174|2261|Thursday|1943Q2|N|N|N|2430816|2430905|2430479|2430754|N|N|N|N|N| +2430845|AAAAAAAANHHBFCAA|1943-04-30|519|2261|174|1943|5|4|30|2|1943|174|2261|Friday|1943Q2|N|Y|N|2430816|2430905|2430480|2430755|N|N|N|N|N| +2430846|AAAAAAAAOHHBFCAA|1943-05-01|520|2261|174|1943|6|5|1|2|1943|174|2261|Saturday|1943Q2|N|Y|N|2430846|2430965|2430481|2430756|N|N|N|N|N| +2430847|AAAAAAAAPHHBFCAA|1943-05-02|520|2261|174|1943|0|5|2|2|1943|174|2261|Sunday|1943Q2|N|N|N|2430846|2430965|2430482|2430757|N|N|N|N|N| +2430848|AAAAAAAAAIHBFCAA|1943-05-03|520|2261|174|1943|1|5|3|2|1943|174|2261|Monday|1943Q2|N|N|N|2430846|2430965|2430483|2430758|N|N|N|N|N| +2430849|AAAAAAAABIHBFCAA|1943-05-04|520|2262|174|1943|2|5|4|2|1943|174|2262|Tuesday|1943Q2|N|N|N|2430846|2430965|2430484|2430759|N|N|N|N|N| +2430850|AAAAAAAACIHBFCAA|1943-05-05|520|2262|174|1943|3|5|5|2|1943|174|2262|Wednesday|1943Q2|N|N|N|2430846|2430965|2430485|2430760|N|N|N|N|N| +2430851|AAAAAAAADIHBFCAA|1943-05-06|520|2262|174|1943|4|5|6|2|1943|174|2262|Thursday|1943Q2|N|N|N|2430846|2430965|2430486|2430761|N|N|N|N|N| +2430852|AAAAAAAAEIHBFCAA|1943-05-07|520|2262|174|1943|5|5|7|2|1943|174|2262|Friday|1943Q2|N|Y|N|2430846|2430965|2430487|2430762|N|N|N|N|N| +2430853|AAAAAAAAFIHBFCAA|1943-05-08|520|2262|174|1943|6|5|8|2|1943|174|2262|Saturday|1943Q2|N|Y|N|2430846|2430965|2430488|2430763|N|N|N|N|N| +2430854|AAAAAAAAGIHBFCAA|1943-05-09|520|2262|174|1943|0|5|9|2|1943|174|2262|Sunday|1943Q2|N|N|N|2430846|2430965|2430489|2430764|N|N|N|N|N| +2430855|AAAAAAAAHIHBFCAA|1943-05-10|520|2262|174|1943|1|5|10|2|1943|174|2262|Monday|1943Q2|N|N|N|2430846|2430965|2430490|2430765|N|N|N|N|N| +2430856|AAAAAAAAIIHBFCAA|1943-05-11|520|2263|174|1943|2|5|11|2|1943|174|2263|Tuesday|1943Q2|N|N|N|2430846|2430965|2430491|2430766|N|N|N|N|N| +2430857|AAAAAAAAJIHBFCAA|1943-05-12|520|2263|174|1943|3|5|12|2|1943|174|2263|Wednesday|1943Q2|N|N|N|2430846|2430965|2430492|2430767|N|N|N|N|N| +2430858|AAAAAAAAKIHBFCAA|1943-05-13|520|2263|174|1943|4|5|13|2|1943|174|2263|Thursday|1943Q2|N|N|N|2430846|2430965|2430493|2430768|N|N|N|N|N| +2430859|AAAAAAAALIHBFCAA|1943-05-14|520|2263|174|1943|5|5|14|2|1943|174|2263|Friday|1943Q2|N|Y|N|2430846|2430965|2430494|2430769|N|N|N|N|N| +2430860|AAAAAAAAMIHBFCAA|1943-05-15|520|2263|174|1943|6|5|15|2|1943|174|2263|Saturday|1943Q2|N|Y|N|2430846|2430965|2430495|2430770|N|N|N|N|N| +2430861|AAAAAAAANIHBFCAA|1943-05-16|520|2263|174|1943|0|5|16|2|1943|174|2263|Sunday|1943Q2|N|N|N|2430846|2430965|2430496|2430771|N|N|N|N|N| +2430862|AAAAAAAAOIHBFCAA|1943-05-17|520|2263|174|1943|1|5|17|2|1943|174|2263|Monday|1943Q2|N|N|N|2430846|2430965|2430497|2430772|N|N|N|N|N| +2430863|AAAAAAAAPIHBFCAA|1943-05-18|520|2264|174|1943|2|5|18|2|1943|174|2264|Tuesday|1943Q2|N|N|N|2430846|2430965|2430498|2430773|N|N|N|N|N| +2430864|AAAAAAAAAJHBFCAA|1943-05-19|520|2264|174|1943|3|5|19|2|1943|174|2264|Wednesday|1943Q2|N|N|N|2430846|2430965|2430499|2430774|N|N|N|N|N| +2430865|AAAAAAAABJHBFCAA|1943-05-20|520|2264|174|1943|4|5|20|2|1943|174|2264|Thursday|1943Q2|N|N|N|2430846|2430965|2430500|2430775|N|N|N|N|N| +2430866|AAAAAAAACJHBFCAA|1943-05-21|520|2264|174|1943|5|5|21|2|1943|174|2264|Friday|1943Q2|N|Y|N|2430846|2430965|2430501|2430776|N|N|N|N|N| +2430867|AAAAAAAADJHBFCAA|1943-05-22|520|2264|174|1943|6|5|22|2|1943|174|2264|Saturday|1943Q2|N|Y|N|2430846|2430965|2430502|2430777|N|N|N|N|N| +2430868|AAAAAAAAEJHBFCAA|1943-05-23|520|2264|174|1943|0|5|23|2|1943|174|2264|Sunday|1943Q2|N|N|N|2430846|2430965|2430503|2430778|N|N|N|N|N| +2430869|AAAAAAAAFJHBFCAA|1943-05-24|520|2264|174|1943|1|5|24|2|1943|174|2264|Monday|1943Q2|N|N|N|2430846|2430965|2430504|2430779|N|N|N|N|N| +2430870|AAAAAAAAGJHBFCAA|1943-05-25|520|2265|174|1943|2|5|25|2|1943|174|2265|Tuesday|1943Q2|N|N|N|2430846|2430965|2430505|2430780|N|N|N|N|N| +2430871|AAAAAAAAHJHBFCAA|1943-05-26|520|2265|174|1943|3|5|26|2|1943|174|2265|Wednesday|1943Q2|N|N|N|2430846|2430965|2430506|2430781|N|N|N|N|N| +2430872|AAAAAAAAIJHBFCAA|1943-05-27|520|2265|174|1943|4|5|27|2|1943|174|2265|Thursday|1943Q2|N|N|N|2430846|2430965|2430507|2430782|N|N|N|N|N| +2430873|AAAAAAAAJJHBFCAA|1943-05-28|520|2265|174|1943|5|5|28|2|1943|174|2265|Friday|1943Q2|N|Y|N|2430846|2430965|2430508|2430783|N|N|N|N|N| +2430874|AAAAAAAAKJHBFCAA|1943-05-29|520|2265|174|1943|6|5|29|2|1943|174|2265|Saturday|1943Q2|N|Y|N|2430846|2430965|2430509|2430784|N|N|N|N|N| +2430875|AAAAAAAALJHBFCAA|1943-05-30|520|2265|174|1943|0|5|30|2|1943|174|2265|Sunday|1943Q2|N|N|N|2430846|2430965|2430510|2430785|N|N|N|N|N| +2430876|AAAAAAAAMJHBFCAA|1943-05-31|520|2265|174|1943|1|5|31|2|1943|174|2265|Monday|1943Q2|N|N|N|2430846|2430965|2430511|2430786|N|N|N|N|N| +2430877|AAAAAAAANJHBFCAA|1943-06-01|521|2266|175|1943|2|6|1|2|1943|175|2266|Tuesday|1943Q2|N|N|N|2430877|2431027|2430512|2430787|N|N|N|N|N| +2430878|AAAAAAAAOJHBFCAA|1943-06-02|521|2266|175|1943|3|6|2|2|1943|175|2266|Wednesday|1943Q2|N|N|N|2430877|2431027|2430513|2430788|N|N|N|N|N| +2430879|AAAAAAAAPJHBFCAA|1943-06-03|521|2266|175|1943|4|6|3|2|1943|175|2266|Thursday|1943Q2|N|N|N|2430877|2431027|2430514|2430789|N|N|N|N|N| +2430880|AAAAAAAAAKHBFCAA|1943-06-04|521|2266|175|1943|5|6|4|2|1943|175|2266|Friday|1943Q2|N|Y|N|2430877|2431027|2430515|2430790|N|N|N|N|N| +2430881|AAAAAAAABKHBFCAA|1943-06-05|521|2266|175|1943|6|6|5|2|1943|175|2266|Saturday|1943Q2|N|Y|N|2430877|2431027|2430516|2430791|N|N|N|N|N| +2430882|AAAAAAAACKHBFCAA|1943-06-06|521|2266|175|1943|0|6|6|2|1943|175|2266|Sunday|1943Q2|N|N|N|2430877|2431027|2430517|2430792|N|N|N|N|N| +2430883|AAAAAAAADKHBFCAA|1943-06-07|521|2266|175|1943|1|6|7|2|1943|175|2266|Monday|1943Q2|N|N|N|2430877|2431027|2430518|2430793|N|N|N|N|N| +2430884|AAAAAAAAEKHBFCAA|1943-06-08|521|2267|175|1943|2|6|8|2|1943|175|2267|Tuesday|1943Q2|N|N|N|2430877|2431027|2430519|2430794|N|N|N|N|N| +2430885|AAAAAAAAFKHBFCAA|1943-06-09|521|2267|175|1943|3|6|9|2|1943|175|2267|Wednesday|1943Q2|N|N|N|2430877|2431027|2430520|2430795|N|N|N|N|N| +2430886|AAAAAAAAGKHBFCAA|1943-06-10|521|2267|175|1943|4|6|10|2|1943|175|2267|Thursday|1943Q2|N|N|N|2430877|2431027|2430521|2430796|N|N|N|N|N| +2430887|AAAAAAAAHKHBFCAA|1943-06-11|521|2267|175|1943|5|6|11|2|1943|175|2267|Friday|1943Q2|N|Y|N|2430877|2431027|2430522|2430797|N|N|N|N|N| +2430888|AAAAAAAAIKHBFCAA|1943-06-12|521|2267|175|1943|6|6|12|2|1943|175|2267|Saturday|1943Q2|N|Y|N|2430877|2431027|2430523|2430798|N|N|N|N|N| +2430889|AAAAAAAAJKHBFCAA|1943-06-13|521|2267|175|1943|0|6|13|2|1943|175|2267|Sunday|1943Q2|N|N|N|2430877|2431027|2430524|2430799|N|N|N|N|N| +2430890|AAAAAAAAKKHBFCAA|1943-06-14|521|2267|175|1943|1|6|14|2|1943|175|2267|Monday|1943Q2|N|N|N|2430877|2431027|2430525|2430800|N|N|N|N|N| +2430891|AAAAAAAALKHBFCAA|1943-06-15|521|2268|175|1943|2|6|15|2|1943|175|2268|Tuesday|1943Q2|N|N|N|2430877|2431027|2430526|2430801|N|N|N|N|N| +2430892|AAAAAAAAMKHBFCAA|1943-06-16|521|2268|175|1943|3|6|16|2|1943|175|2268|Wednesday|1943Q2|N|N|N|2430877|2431027|2430527|2430802|N|N|N|N|N| +2430893|AAAAAAAANKHBFCAA|1943-06-17|521|2268|175|1943|4|6|17|2|1943|175|2268|Thursday|1943Q2|N|N|N|2430877|2431027|2430528|2430803|N|N|N|N|N| +2430894|AAAAAAAAOKHBFCAA|1943-06-18|521|2268|175|1943|5|6|18|2|1943|175|2268|Friday|1943Q2|N|Y|N|2430877|2431027|2430529|2430804|N|N|N|N|N| +2430895|AAAAAAAAPKHBFCAA|1943-06-19|521|2268|175|1943|6|6|19|2|1943|175|2268|Saturday|1943Q2|N|Y|N|2430877|2431027|2430530|2430805|N|N|N|N|N| +2430896|AAAAAAAAALHBFCAA|1943-06-20|521|2268|175|1943|0|6|20|2|1943|175|2268|Sunday|1943Q2|N|N|N|2430877|2431027|2430531|2430806|N|N|N|N|N| +2430897|AAAAAAAABLHBFCAA|1943-06-21|521|2268|175|1943|1|6|21|2|1943|175|2268|Monday|1943Q2|N|N|N|2430877|2431027|2430532|2430807|N|N|N|N|N| +2430898|AAAAAAAACLHBFCAA|1943-06-22|521|2269|175|1943|2|6|22|2|1943|175|2269|Tuesday|1943Q2|N|N|N|2430877|2431027|2430533|2430808|N|N|N|N|N| +2430899|AAAAAAAADLHBFCAA|1943-06-23|521|2269|175|1943|3|6|23|2|1943|175|2269|Wednesday|1943Q2|N|N|N|2430877|2431027|2430534|2430809|N|N|N|N|N| +2430900|AAAAAAAAELHBFCAA|1943-06-24|521|2269|175|1943|4|6|24|2|1943|175|2269|Thursday|1943Q2|N|N|N|2430877|2431027|2430535|2430810|N|N|N|N|N| +2430901|AAAAAAAAFLHBFCAA|1943-06-25|521|2269|175|1943|5|6|25|2|1943|175|2269|Friday|1943Q2|N|Y|N|2430877|2431027|2430536|2430811|N|N|N|N|N| +2430902|AAAAAAAAGLHBFCAA|1943-06-26|521|2269|175|1943|6|6|26|2|1943|175|2269|Saturday|1943Q2|N|Y|N|2430877|2431027|2430537|2430812|N|N|N|N|N| +2430903|AAAAAAAAHLHBFCAA|1943-06-27|521|2269|175|1943|0|6|27|2|1943|175|2269|Sunday|1943Q2|N|N|N|2430877|2431027|2430538|2430813|N|N|N|N|N| +2430904|AAAAAAAAILHBFCAA|1943-06-28|521|2269|175|1943|1|6|28|2|1943|175|2269|Monday|1943Q2|N|N|N|2430877|2431027|2430539|2430814|N|N|N|N|N| +2430905|AAAAAAAAJLHBFCAA|1943-06-29|521|2270|175|1943|2|6|29|2|1943|175|2270|Tuesday|1943Q2|N|N|N|2430877|2431027|2430540|2430815|N|N|N|N|N| +2430906|AAAAAAAAKLHBFCAA|1943-06-30|521|2270|175|1943|3|6|30|2|1943|175|2270|Wednesday|1943Q2|N|N|N|2430877|2431027|2430541|2430816|N|N|N|N|N| +2430907|AAAAAAAALLHBFCAA|1943-07-01|522|2270|175|1943|4|7|1|2|1943|175|2270|Thursday|1943Q2|N|N|N|2430907|2431087|2430542|2430816|N|N|N|N|N| +2430908|AAAAAAAAMLHBFCAA|1943-07-02|522|2270|175|1943|5|7|2|3|1943|175|2270|Friday|1943Q3|N|Y|N|2430907|2431087|2430543|2430817|N|N|N|N|N| +2430909|AAAAAAAANLHBFCAA|1943-07-03|522|2270|175|1943|6|7|3|3|1943|175|2270|Saturday|1943Q3|N|Y|N|2430907|2431087|2430544|2430818|N|N|N|N|N| +2430910|AAAAAAAAOLHBFCAA|1943-07-04|522|2270|175|1943|0|7|4|3|1943|175|2270|Sunday|1943Q3|N|N|N|2430907|2431087|2430545|2430819|N|N|N|N|N| +2430911|AAAAAAAAPLHBFCAA|1943-07-05|522|2270|175|1943|1|7|5|3|1943|175|2270|Monday|1943Q3|Y|N|N|2430907|2431087|2430546|2430820|N|N|N|N|N| +2430912|AAAAAAAAAMHBFCAA|1943-07-06|522|2271|175|1943|2|7|6|3|1943|175|2271|Tuesday|1943Q3|N|N|Y|2430907|2431087|2430547|2430821|N|N|N|N|N| +2430913|AAAAAAAABMHBFCAA|1943-07-07|522|2271|175|1943|3|7|7|3|1943|175|2271|Wednesday|1943Q3|N|N|N|2430907|2431087|2430548|2430822|N|N|N|N|N| +2430914|AAAAAAAACMHBFCAA|1943-07-08|522|2271|175|1943|4|7|8|3|1943|175|2271|Thursday|1943Q3|N|N|N|2430907|2431087|2430549|2430823|N|N|N|N|N| +2430915|AAAAAAAADMHBFCAA|1943-07-09|522|2271|175|1943|5|7|9|3|1943|175|2271|Friday|1943Q3|N|Y|N|2430907|2431087|2430550|2430824|N|N|N|N|N| +2430916|AAAAAAAAEMHBFCAA|1943-07-10|522|2271|175|1943|6|7|10|3|1943|175|2271|Saturday|1943Q3|N|Y|N|2430907|2431087|2430551|2430825|N|N|N|N|N| +2430917|AAAAAAAAFMHBFCAA|1943-07-11|522|2271|175|1943|0|7|11|3|1943|175|2271|Sunday|1943Q3|N|N|N|2430907|2431087|2430552|2430826|N|N|N|N|N| +2430918|AAAAAAAAGMHBFCAA|1943-07-12|522|2271|175|1943|1|7|12|3|1943|175|2271|Monday|1943Q3|N|N|N|2430907|2431087|2430553|2430827|N|N|N|N|N| +2430919|AAAAAAAAHMHBFCAA|1943-07-13|522|2272|175|1943|2|7|13|3|1943|175|2272|Tuesday|1943Q3|N|N|N|2430907|2431087|2430554|2430828|N|N|N|N|N| +2430920|AAAAAAAAIMHBFCAA|1943-07-14|522|2272|175|1943|3|7|14|3|1943|175|2272|Wednesday|1943Q3|N|N|N|2430907|2431087|2430555|2430829|N|N|N|N|N| +2430921|AAAAAAAAJMHBFCAA|1943-07-15|522|2272|175|1943|4|7|15|3|1943|175|2272|Thursday|1943Q3|N|N|N|2430907|2431087|2430556|2430830|N|N|N|N|N| +2430922|AAAAAAAAKMHBFCAA|1943-07-16|522|2272|175|1943|5|7|16|3|1943|175|2272|Friday|1943Q3|N|Y|N|2430907|2431087|2430557|2430831|N|N|N|N|N| +2430923|AAAAAAAALMHBFCAA|1943-07-17|522|2272|175|1943|6|7|17|3|1943|175|2272|Saturday|1943Q3|N|Y|N|2430907|2431087|2430558|2430832|N|N|N|N|N| +2430924|AAAAAAAAMMHBFCAA|1943-07-18|522|2272|175|1943|0|7|18|3|1943|175|2272|Sunday|1943Q3|N|N|N|2430907|2431087|2430559|2430833|N|N|N|N|N| +2430925|AAAAAAAANMHBFCAA|1943-07-19|522|2272|175|1943|1|7|19|3|1943|175|2272|Monday|1943Q3|N|N|N|2430907|2431087|2430560|2430834|N|N|N|N|N| +2430926|AAAAAAAAOMHBFCAA|1943-07-20|522|2273|175|1943|2|7|20|3|1943|175|2273|Tuesday|1943Q3|N|N|N|2430907|2431087|2430561|2430835|N|N|N|N|N| +2430927|AAAAAAAAPMHBFCAA|1943-07-21|522|2273|175|1943|3|7|21|3|1943|175|2273|Wednesday|1943Q3|N|N|N|2430907|2431087|2430562|2430836|N|N|N|N|N| +2430928|AAAAAAAAANHBFCAA|1943-07-22|522|2273|175|1943|4|7|22|3|1943|175|2273|Thursday|1943Q3|N|N|N|2430907|2431087|2430563|2430837|N|N|N|N|N| +2430929|AAAAAAAABNHBFCAA|1943-07-23|522|2273|175|1943|5|7|23|3|1943|175|2273|Friday|1943Q3|N|Y|N|2430907|2431087|2430564|2430838|N|N|N|N|N| +2430930|AAAAAAAACNHBFCAA|1943-07-24|522|2273|175|1943|6|7|24|3|1943|175|2273|Saturday|1943Q3|N|Y|N|2430907|2431087|2430565|2430839|N|N|N|N|N| +2430931|AAAAAAAADNHBFCAA|1943-07-25|522|2273|175|1943|0|7|25|3|1943|175|2273|Sunday|1943Q3|N|N|N|2430907|2431087|2430566|2430840|N|N|N|N|N| +2430932|AAAAAAAAENHBFCAA|1943-07-26|522|2273|175|1943|1|7|26|3|1943|175|2273|Monday|1943Q3|N|N|N|2430907|2431087|2430567|2430841|N|N|N|N|N| +2430933|AAAAAAAAFNHBFCAA|1943-07-27|522|2274|175|1943|2|7|27|3|1943|175|2274|Tuesday|1943Q3|N|N|N|2430907|2431087|2430568|2430842|N|N|N|N|N| +2430934|AAAAAAAAGNHBFCAA|1943-07-28|522|2274|175|1943|3|7|28|3|1943|175|2274|Wednesday|1943Q3|N|N|N|2430907|2431087|2430569|2430843|N|N|N|N|N| +2430935|AAAAAAAAHNHBFCAA|1943-07-29|522|2274|175|1943|4|7|29|3|1943|175|2274|Thursday|1943Q3|N|N|N|2430907|2431087|2430570|2430844|N|N|N|N|N| +2430936|AAAAAAAAINHBFCAA|1943-07-30|522|2274|175|1943|5|7|30|3|1943|175|2274|Friday|1943Q3|N|Y|N|2430907|2431087|2430571|2430845|N|N|N|N|N| +2430937|AAAAAAAAJNHBFCAA|1943-07-31|522|2274|175|1943|6|7|31|3|1943|175|2274|Saturday|1943Q3|N|Y|N|2430907|2431087|2430572|2430846|N|N|N|N|N| +2430938|AAAAAAAAKNHBFCAA|1943-08-01|523|2274|175|1943|0|8|1|3|1943|175|2274|Sunday|1943Q3|N|N|N|2430938|2431149|2430573|2430847|N|N|N|N|N| +2430939|AAAAAAAALNHBFCAA|1943-08-02|523|2274|175|1943|1|8|2|3|1943|175|2274|Monday|1943Q3|N|N|N|2430938|2431149|2430574|2430848|N|N|N|N|N| +2430940|AAAAAAAAMNHBFCAA|1943-08-03|523|2275|175|1943|2|8|3|3|1943|175|2275|Tuesday|1943Q3|N|N|N|2430938|2431149|2430575|2430849|N|N|N|N|N| +2430941|AAAAAAAANNHBFCAA|1943-08-04|523|2275|175|1943|3|8|4|3|1943|175|2275|Wednesday|1943Q3|N|N|N|2430938|2431149|2430576|2430850|N|N|N|N|N| +2430942|AAAAAAAAONHBFCAA|1943-08-05|523|2275|175|1943|4|8|5|3|1943|175|2275|Thursday|1943Q3|N|N|N|2430938|2431149|2430577|2430851|N|N|N|N|N| +2430943|AAAAAAAAPNHBFCAA|1943-08-06|523|2275|175|1943|5|8|6|3|1943|175|2275|Friday|1943Q3|N|Y|N|2430938|2431149|2430578|2430852|N|N|N|N|N| +2430944|AAAAAAAAAOHBFCAA|1943-08-07|523|2275|175|1943|6|8|7|3|1943|175|2275|Saturday|1943Q3|N|Y|N|2430938|2431149|2430579|2430853|N|N|N|N|N| +2430945|AAAAAAAABOHBFCAA|1943-08-08|523|2275|175|1943|0|8|8|3|1943|175|2275|Sunday|1943Q3|N|N|N|2430938|2431149|2430580|2430854|N|N|N|N|N| +2430946|AAAAAAAACOHBFCAA|1943-08-09|523|2275|175|1943|1|8|9|3|1943|175|2275|Monday|1943Q3|N|N|N|2430938|2431149|2430581|2430855|N|N|N|N|N| +2430947|AAAAAAAADOHBFCAA|1943-08-10|523|2276|175|1943|2|8|10|3|1943|175|2276|Tuesday|1943Q3|N|N|N|2430938|2431149|2430582|2430856|N|N|N|N|N| +2430948|AAAAAAAAEOHBFCAA|1943-08-11|523|2276|175|1943|3|8|11|3|1943|175|2276|Wednesday|1943Q3|N|N|N|2430938|2431149|2430583|2430857|N|N|N|N|N| +2430949|AAAAAAAAFOHBFCAA|1943-08-12|523|2276|175|1943|4|8|12|3|1943|175|2276|Thursday|1943Q3|N|N|N|2430938|2431149|2430584|2430858|N|N|N|N|N| +2430950|AAAAAAAAGOHBFCAA|1943-08-13|523|2276|175|1943|5|8|13|3|1943|175|2276|Friday|1943Q3|N|Y|N|2430938|2431149|2430585|2430859|N|N|N|N|N| +2430951|AAAAAAAAHOHBFCAA|1943-08-14|523|2276|175|1943|6|8|14|3|1943|175|2276|Saturday|1943Q3|N|Y|N|2430938|2431149|2430586|2430860|N|N|N|N|N| +2430952|AAAAAAAAIOHBFCAA|1943-08-15|523|2276|175|1943|0|8|15|3|1943|175|2276|Sunday|1943Q3|N|N|N|2430938|2431149|2430587|2430861|N|N|N|N|N| +2430953|AAAAAAAAJOHBFCAA|1943-08-16|523|2276|175|1943|1|8|16|3|1943|175|2276|Monday|1943Q3|N|N|N|2430938|2431149|2430588|2430862|N|N|N|N|N| +2430954|AAAAAAAAKOHBFCAA|1943-08-17|523|2277|175|1943|2|8|17|3|1943|175|2277|Tuesday|1943Q3|N|N|N|2430938|2431149|2430589|2430863|N|N|N|N|N| +2430955|AAAAAAAALOHBFCAA|1943-08-18|523|2277|175|1943|3|8|18|3|1943|175|2277|Wednesday|1943Q3|N|N|N|2430938|2431149|2430590|2430864|N|N|N|N|N| +2430956|AAAAAAAAMOHBFCAA|1943-08-19|523|2277|175|1943|4|8|19|3|1943|175|2277|Thursday|1943Q3|N|N|N|2430938|2431149|2430591|2430865|N|N|N|N|N| +2430957|AAAAAAAANOHBFCAA|1943-08-20|523|2277|175|1943|5|8|20|3|1943|175|2277|Friday|1943Q3|N|Y|N|2430938|2431149|2430592|2430866|N|N|N|N|N| +2430958|AAAAAAAAOOHBFCAA|1943-08-21|523|2277|175|1943|6|8|21|3|1943|175|2277|Saturday|1943Q3|N|Y|N|2430938|2431149|2430593|2430867|N|N|N|N|N| +2430959|AAAAAAAAPOHBFCAA|1943-08-22|523|2277|175|1943|0|8|22|3|1943|175|2277|Sunday|1943Q3|N|N|N|2430938|2431149|2430594|2430868|N|N|N|N|N| +2430960|AAAAAAAAAPHBFCAA|1943-08-23|523|2277|175|1943|1|8|23|3|1943|175|2277|Monday|1943Q3|N|N|N|2430938|2431149|2430595|2430869|N|N|N|N|N| +2430961|AAAAAAAABPHBFCAA|1943-08-24|523|2278|175|1943|2|8|24|3|1943|175|2278|Tuesday|1943Q3|N|N|N|2430938|2431149|2430596|2430870|N|N|N|N|N| +2430962|AAAAAAAACPHBFCAA|1943-08-25|523|2278|175|1943|3|8|25|3|1943|175|2278|Wednesday|1943Q3|N|N|N|2430938|2431149|2430597|2430871|N|N|N|N|N| +2430963|AAAAAAAADPHBFCAA|1943-08-26|523|2278|175|1943|4|8|26|3|1943|175|2278|Thursday|1943Q3|N|N|N|2430938|2431149|2430598|2430872|N|N|N|N|N| +2430964|AAAAAAAAEPHBFCAA|1943-08-27|523|2278|175|1943|5|8|27|3|1943|175|2278|Friday|1943Q3|N|Y|N|2430938|2431149|2430599|2430873|N|N|N|N|N| +2430965|AAAAAAAAFPHBFCAA|1943-08-28|523|2278|175|1943|6|8|28|3|1943|175|2278|Saturday|1943Q3|N|Y|N|2430938|2431149|2430600|2430874|N|N|N|N|N| +2430966|AAAAAAAAGPHBFCAA|1943-08-29|523|2278|175|1943|0|8|29|3|1943|175|2278|Sunday|1943Q3|N|N|N|2430938|2431149|2430601|2430875|N|N|N|N|N| +2430967|AAAAAAAAHPHBFCAA|1943-08-30|523|2278|175|1943|1|8|30|3|1943|175|2278|Monday|1943Q3|N|N|N|2430938|2431149|2430602|2430876|N|N|N|N|N| +2430968|AAAAAAAAIPHBFCAA|1943-08-31|523|2279|175|1943|2|8|31|3|1943|175|2279|Tuesday|1943Q3|N|N|N|2430938|2431149|2430603|2430877|N|N|N|N|N| +2430969|AAAAAAAAJPHBFCAA|1943-09-01|524|2279|176|1943|3|9|1|3|1943|176|2279|Wednesday|1943Q3|N|N|N|2430969|2431211|2430604|2430878|N|N|N|N|N| +2430970|AAAAAAAAKPHBFCAA|1943-09-02|524|2279|176|1943|4|9|2|3|1943|176|2279|Thursday|1943Q3|N|N|N|2430969|2431211|2430605|2430879|N|N|N|N|N| +2430971|AAAAAAAALPHBFCAA|1943-09-03|524|2279|176|1943|5|9|3|3|1943|176|2279|Friday|1943Q3|N|Y|N|2430969|2431211|2430606|2430880|N|N|N|N|N| +2430972|AAAAAAAAMPHBFCAA|1943-09-04|524|2279|176|1943|6|9|4|3|1943|176|2279|Saturday|1943Q3|N|Y|N|2430969|2431211|2430607|2430881|N|N|N|N|N| +2430973|AAAAAAAANPHBFCAA|1943-09-05|524|2279|176|1943|0|9|5|3|1943|176|2279|Sunday|1943Q3|N|N|N|2430969|2431211|2430608|2430882|N|N|N|N|N| +2430974|AAAAAAAAOPHBFCAA|1943-09-06|524|2279|176|1943|1|9|6|3|1943|176|2279|Monday|1943Q3|N|N|N|2430969|2431211|2430609|2430883|N|N|N|N|N| +2430975|AAAAAAAAPPHBFCAA|1943-09-07|524|2280|176|1943|2|9|7|3|1943|176|2280|Tuesday|1943Q3|N|N|N|2430969|2431211|2430610|2430884|N|N|N|N|N| +2430976|AAAAAAAAAAIBFCAA|1943-09-08|524|2280|176|1943|3|9|8|3|1943|176|2280|Wednesday|1943Q3|N|N|N|2430969|2431211|2430611|2430885|N|N|N|N|N| +2430977|AAAAAAAABAIBFCAA|1943-09-09|524|2280|176|1943|4|9|9|3|1943|176|2280|Thursday|1943Q3|N|N|N|2430969|2431211|2430612|2430886|N|N|N|N|N| +2430978|AAAAAAAACAIBFCAA|1943-09-10|524|2280|176|1943|5|9|10|3|1943|176|2280|Friday|1943Q3|N|Y|N|2430969|2431211|2430613|2430887|N|N|N|N|N| +2430979|AAAAAAAADAIBFCAA|1943-09-11|524|2280|176|1943|6|9|11|3|1943|176|2280|Saturday|1943Q3|N|Y|N|2430969|2431211|2430614|2430888|N|N|N|N|N| +2430980|AAAAAAAAEAIBFCAA|1943-09-12|524|2280|176|1943|0|9|12|3|1943|176|2280|Sunday|1943Q3|N|N|N|2430969|2431211|2430615|2430889|N|N|N|N|N| +2430981|AAAAAAAAFAIBFCAA|1943-09-13|524|2280|176|1943|1|9|13|3|1943|176|2280|Monday|1943Q3|N|N|N|2430969|2431211|2430616|2430890|N|N|N|N|N| +2430982|AAAAAAAAGAIBFCAA|1943-09-14|524|2281|176|1943|2|9|14|3|1943|176|2281|Tuesday|1943Q3|N|N|N|2430969|2431211|2430617|2430891|N|N|N|N|N| +2430983|AAAAAAAAHAIBFCAA|1943-09-15|524|2281|176|1943|3|9|15|3|1943|176|2281|Wednesday|1943Q3|N|N|N|2430969|2431211|2430618|2430892|N|N|N|N|N| +2430984|AAAAAAAAIAIBFCAA|1943-09-16|524|2281|176|1943|4|9|16|3|1943|176|2281|Thursday|1943Q3|N|N|N|2430969|2431211|2430619|2430893|N|N|N|N|N| +2430985|AAAAAAAAJAIBFCAA|1943-09-17|524|2281|176|1943|5|9|17|3|1943|176|2281|Friday|1943Q3|N|Y|N|2430969|2431211|2430620|2430894|N|N|N|N|N| +2430986|AAAAAAAAKAIBFCAA|1943-09-18|524|2281|176|1943|6|9|18|3|1943|176|2281|Saturday|1943Q3|N|Y|N|2430969|2431211|2430621|2430895|N|N|N|N|N| +2430987|AAAAAAAALAIBFCAA|1943-09-19|524|2281|176|1943|0|9|19|3|1943|176|2281|Sunday|1943Q3|N|N|N|2430969|2431211|2430622|2430896|N|N|N|N|N| +2430988|AAAAAAAAMAIBFCAA|1943-09-20|524|2281|176|1943|1|9|20|3|1943|176|2281|Monday|1943Q3|N|N|N|2430969|2431211|2430623|2430897|N|N|N|N|N| +2430989|AAAAAAAANAIBFCAA|1943-09-21|524|2282|176|1943|2|9|21|3|1943|176|2282|Tuesday|1943Q3|N|N|N|2430969|2431211|2430624|2430898|N|N|N|N|N| +2430990|AAAAAAAAOAIBFCAA|1943-09-22|524|2282|176|1943|3|9|22|3|1943|176|2282|Wednesday|1943Q3|N|N|N|2430969|2431211|2430625|2430899|N|N|N|N|N| +2430991|AAAAAAAAPAIBFCAA|1943-09-23|524|2282|176|1943|4|9|23|3|1943|176|2282|Thursday|1943Q3|N|N|N|2430969|2431211|2430626|2430900|N|N|N|N|N| +2430992|AAAAAAAAABIBFCAA|1943-09-24|524|2282|176|1943|5|9|24|3|1943|176|2282|Friday|1943Q3|N|Y|N|2430969|2431211|2430627|2430901|N|N|N|N|N| +2430993|AAAAAAAABBIBFCAA|1943-09-25|524|2282|176|1943|6|9|25|3|1943|176|2282|Saturday|1943Q3|N|Y|N|2430969|2431211|2430628|2430902|N|N|N|N|N| +2430994|AAAAAAAACBIBFCAA|1943-09-26|524|2282|176|1943|0|9|26|3|1943|176|2282|Sunday|1943Q3|N|N|N|2430969|2431211|2430629|2430903|N|N|N|N|N| +2430995|AAAAAAAADBIBFCAA|1943-09-27|524|2282|176|1943|1|9|27|3|1943|176|2282|Monday|1943Q3|N|N|N|2430969|2431211|2430630|2430904|N|N|N|N|N| +2430996|AAAAAAAAEBIBFCAA|1943-09-28|524|2283|176|1943|2|9|28|3|1943|176|2283|Tuesday|1943Q3|N|N|N|2430969|2431211|2430631|2430905|N|N|N|N|N| +2430997|AAAAAAAAFBIBFCAA|1943-09-29|524|2283|176|1943|3|9|29|3|1943|176|2283|Wednesday|1943Q3|N|N|N|2430969|2431211|2430632|2430906|N|N|N|N|N| +2430998|AAAAAAAAGBIBFCAA|1943-09-30|524|2283|176|1943|4|9|30|3|1943|176|2283|Thursday|1943Q3|N|N|N|2430969|2431211|2430633|2430907|N|N|N|N|N| +2430999|AAAAAAAAHBIBFCAA|1943-10-01|525|2283|176|1943|5|10|1|3|1943|176|2283|Friday|1943Q3|N|Y|N|2430999|2431271|2430634|2430907|N|N|N|N|N| +2431000|AAAAAAAAIBIBFCAA|1943-10-02|525|2283|176|1943|6|10|2|4|1943|176|2283|Saturday|1943Q4|N|Y|N|2430999|2431271|2430635|2430908|N|N|N|N|N| +2431001|AAAAAAAAJBIBFCAA|1943-10-03|525|2283|176|1943|0|10|3|4|1943|176|2283|Sunday|1943Q4|N|N|N|2430999|2431271|2430636|2430909|N|N|N|N|N| +2431002|AAAAAAAAKBIBFCAA|1943-10-04|525|2283|176|1943|1|10|4|4|1943|176|2283|Monday|1943Q4|N|N|N|2430999|2431271|2430637|2430910|N|N|N|N|N| +2431003|AAAAAAAALBIBFCAA|1943-10-05|525|2284|176|1943|2|10|5|4|1943|176|2284|Tuesday|1943Q4|N|N|N|2430999|2431271|2430638|2430911|N|N|N|N|N| +2431004|AAAAAAAAMBIBFCAA|1943-10-06|525|2284|176|1943|3|10|6|4|1943|176|2284|Wednesday|1943Q4|N|N|N|2430999|2431271|2430639|2430912|N|N|N|N|N| +2431005|AAAAAAAANBIBFCAA|1943-10-07|525|2284|176|1943|4|10|7|4|1943|176|2284|Thursday|1943Q4|N|N|N|2430999|2431271|2430640|2430913|N|N|N|N|N| +2431006|AAAAAAAAOBIBFCAA|1943-10-08|525|2284|176|1943|5|10|8|4|1943|176|2284|Friday|1943Q4|N|Y|N|2430999|2431271|2430641|2430914|N|N|N|N|N| +2431007|AAAAAAAAPBIBFCAA|1943-10-09|525|2284|176|1943|6|10|9|4|1943|176|2284|Saturday|1943Q4|N|Y|N|2430999|2431271|2430642|2430915|N|N|N|N|N| +2431008|AAAAAAAAACIBFCAA|1943-10-10|525|2284|176|1943|0|10|10|4|1943|176|2284|Sunday|1943Q4|N|N|N|2430999|2431271|2430643|2430916|N|N|N|N|N| +2431009|AAAAAAAABCIBFCAA|1943-10-11|525|2284|176|1943|1|10|11|4|1943|176|2284|Monday|1943Q4|N|N|N|2430999|2431271|2430644|2430917|N|N|N|N|N| +2431010|AAAAAAAACCIBFCAA|1943-10-12|525|2285|176|1943|2|10|12|4|1943|176|2285|Tuesday|1943Q4|N|N|N|2430999|2431271|2430645|2430918|N|N|N|N|N| +2431011|AAAAAAAADCIBFCAA|1943-10-13|525|2285|176|1943|3|10|13|4|1943|176|2285|Wednesday|1943Q4|N|N|N|2430999|2431271|2430646|2430919|N|N|N|N|N| +2431012|AAAAAAAAECIBFCAA|1943-10-14|525|2285|176|1943|4|10|14|4|1943|176|2285|Thursday|1943Q4|N|N|N|2430999|2431271|2430647|2430920|N|N|N|N|N| +2431013|AAAAAAAAFCIBFCAA|1943-10-15|525|2285|176|1943|5|10|15|4|1943|176|2285|Friday|1943Q4|N|Y|N|2430999|2431271|2430648|2430921|N|N|N|N|N| +2431014|AAAAAAAAGCIBFCAA|1943-10-16|525|2285|176|1943|6|10|16|4|1943|176|2285|Saturday|1943Q4|N|Y|N|2430999|2431271|2430649|2430922|N|N|N|N|N| +2431015|AAAAAAAAHCIBFCAA|1943-10-17|525|2285|176|1943|0|10|17|4|1943|176|2285|Sunday|1943Q4|N|N|N|2430999|2431271|2430650|2430923|N|N|N|N|N| +2431016|AAAAAAAAICIBFCAA|1943-10-18|525|2285|176|1943|1|10|18|4|1943|176|2285|Monday|1943Q4|N|N|N|2430999|2431271|2430651|2430924|N|N|N|N|N| +2431017|AAAAAAAAJCIBFCAA|1943-10-19|525|2286|176|1943|2|10|19|4|1943|176|2286|Tuesday|1943Q4|N|N|N|2430999|2431271|2430652|2430925|N|N|N|N|N| +2431018|AAAAAAAAKCIBFCAA|1943-10-20|525|2286|176|1943|3|10|20|4|1943|176|2286|Wednesday|1943Q4|N|N|N|2430999|2431271|2430653|2430926|N|N|N|N|N| +2431019|AAAAAAAALCIBFCAA|1943-10-21|525|2286|176|1943|4|10|21|4|1943|176|2286|Thursday|1943Q4|N|N|N|2430999|2431271|2430654|2430927|N|N|N|N|N| +2431020|AAAAAAAAMCIBFCAA|1943-10-22|525|2286|176|1943|5|10|22|4|1943|176|2286|Friday|1943Q4|N|Y|N|2430999|2431271|2430655|2430928|N|N|N|N|N| +2431021|AAAAAAAANCIBFCAA|1943-10-23|525|2286|176|1943|6|10|23|4|1943|176|2286|Saturday|1943Q4|N|Y|N|2430999|2431271|2430656|2430929|N|N|N|N|N| +2431022|AAAAAAAAOCIBFCAA|1943-10-24|525|2286|176|1943|0|10|24|4|1943|176|2286|Sunday|1943Q4|N|N|N|2430999|2431271|2430657|2430930|N|N|N|N|N| +2431023|AAAAAAAAPCIBFCAA|1943-10-25|525|2286|176|1943|1|10|25|4|1943|176|2286|Monday|1943Q4|N|N|N|2430999|2431271|2430658|2430931|N|N|N|N|N| +2431024|AAAAAAAAADIBFCAA|1943-10-26|525|2287|176|1943|2|10|26|4|1943|176|2287|Tuesday|1943Q4|N|N|N|2430999|2431271|2430659|2430932|N|N|N|N|N| +2431025|AAAAAAAABDIBFCAA|1943-10-27|525|2287|176|1943|3|10|27|4|1943|176|2287|Wednesday|1943Q4|N|N|N|2430999|2431271|2430660|2430933|N|N|N|N|N| +2431026|AAAAAAAACDIBFCAA|1943-10-28|525|2287|176|1943|4|10|28|4|1943|176|2287|Thursday|1943Q4|N|N|N|2430999|2431271|2430661|2430934|N|N|N|N|N| +2431027|AAAAAAAADDIBFCAA|1943-10-29|525|2287|176|1943|5|10|29|4|1943|176|2287|Friday|1943Q4|N|Y|N|2430999|2431271|2430662|2430935|N|N|N|N|N| +2431028|AAAAAAAAEDIBFCAA|1943-10-30|525|2287|176|1943|6|10|30|4|1943|176|2287|Saturday|1943Q4|N|Y|N|2430999|2431271|2430663|2430936|N|N|N|N|N| +2431029|AAAAAAAAFDIBFCAA|1943-10-31|525|2287|176|1943|0|10|31|4|1943|176|2287|Sunday|1943Q4|N|N|N|2430999|2431271|2430664|2430937|N|N|N|N|N| +2431030|AAAAAAAAGDIBFCAA|1943-11-01|526|2287|176|1943|1|11|1|4|1943|176|2287|Monday|1943Q4|N|N|N|2431030|2431333|2430665|2430938|N|N|N|N|N| +2431031|AAAAAAAAHDIBFCAA|1943-11-02|526|2288|176|1943|2|11|2|4|1943|176|2288|Tuesday|1943Q4|N|N|N|2431030|2431333|2430666|2430939|N|N|N|N|N| +2431032|AAAAAAAAIDIBFCAA|1943-11-03|526|2288|176|1943|3|11|3|4|1943|176|2288|Wednesday|1943Q4|N|N|N|2431030|2431333|2430667|2430940|N|N|N|N|N| +2431033|AAAAAAAAJDIBFCAA|1943-11-04|526|2288|176|1943|4|11|4|4|1943|176|2288|Thursday|1943Q4|N|N|N|2431030|2431333|2430668|2430941|N|N|N|N|N| +2431034|AAAAAAAAKDIBFCAA|1943-11-05|526|2288|176|1943|5|11|5|4|1943|176|2288|Friday|1943Q4|N|Y|N|2431030|2431333|2430669|2430942|N|N|N|N|N| +2431035|AAAAAAAALDIBFCAA|1943-11-06|526|2288|176|1943|6|11|6|4|1943|176|2288|Saturday|1943Q4|N|Y|N|2431030|2431333|2430670|2430943|N|N|N|N|N| +2431036|AAAAAAAAMDIBFCAA|1943-11-07|526|2288|176|1943|0|11|7|4|1943|176|2288|Sunday|1943Q4|N|N|N|2431030|2431333|2430671|2430944|N|N|N|N|N| +2431037|AAAAAAAANDIBFCAA|1943-11-08|526|2288|176|1943|1|11|8|4|1943|176|2288|Monday|1943Q4|N|N|N|2431030|2431333|2430672|2430945|N|N|N|N|N| +2431038|AAAAAAAAODIBFCAA|1943-11-09|526|2289|176|1943|2|11|9|4|1943|176|2289|Tuesday|1943Q4|N|N|N|2431030|2431333|2430673|2430946|N|N|N|N|N| +2431039|AAAAAAAAPDIBFCAA|1943-11-10|526|2289|176|1943|3|11|10|4|1943|176|2289|Wednesday|1943Q4|N|N|N|2431030|2431333|2430674|2430947|N|N|N|N|N| +2431040|AAAAAAAAAEIBFCAA|1943-11-11|526|2289|176|1943|4|11|11|4|1943|176|2289|Thursday|1943Q4|N|N|N|2431030|2431333|2430675|2430948|N|N|N|N|N| +2431041|AAAAAAAABEIBFCAA|1943-11-12|526|2289|176|1943|5|11|12|4|1943|176|2289|Friday|1943Q4|N|Y|N|2431030|2431333|2430676|2430949|N|N|N|N|N| +2431042|AAAAAAAACEIBFCAA|1943-11-13|526|2289|176|1943|6|11|13|4|1943|176|2289|Saturday|1943Q4|N|Y|N|2431030|2431333|2430677|2430950|N|N|N|N|N| +2431043|AAAAAAAADEIBFCAA|1943-11-14|526|2289|176|1943|0|11|14|4|1943|176|2289|Sunday|1943Q4|N|N|N|2431030|2431333|2430678|2430951|N|N|N|N|N| +2431044|AAAAAAAAEEIBFCAA|1943-11-15|526|2289|176|1943|1|11|15|4|1943|176|2289|Monday|1943Q4|N|N|N|2431030|2431333|2430679|2430952|N|N|N|N|N| +2431045|AAAAAAAAFEIBFCAA|1943-11-16|526|2290|176|1943|2|11|16|4|1943|176|2290|Tuesday|1943Q4|N|N|N|2431030|2431333|2430680|2430953|N|N|N|N|N| +2431046|AAAAAAAAGEIBFCAA|1943-11-17|526|2290|176|1943|3|11|17|4|1943|176|2290|Wednesday|1943Q4|N|N|N|2431030|2431333|2430681|2430954|N|N|N|N|N| +2431047|AAAAAAAAHEIBFCAA|1943-11-18|526|2290|176|1943|4|11|18|4|1943|176|2290|Thursday|1943Q4|N|N|N|2431030|2431333|2430682|2430955|N|N|N|N|N| +2431048|AAAAAAAAIEIBFCAA|1943-11-19|526|2290|176|1943|5|11|19|4|1943|176|2290|Friday|1943Q4|N|Y|N|2431030|2431333|2430683|2430956|N|N|N|N|N| +2431049|AAAAAAAAJEIBFCAA|1943-11-20|526|2290|176|1943|6|11|20|4|1943|176|2290|Saturday|1943Q4|N|Y|N|2431030|2431333|2430684|2430957|N|N|N|N|N| +2431050|AAAAAAAAKEIBFCAA|1943-11-21|526|2290|176|1943|0|11|21|4|1943|176|2290|Sunday|1943Q4|N|N|N|2431030|2431333|2430685|2430958|N|N|N|N|N| +2431051|AAAAAAAALEIBFCAA|1943-11-22|526|2290|176|1943|1|11|22|4|1943|176|2290|Monday|1943Q4|N|N|N|2431030|2431333|2430686|2430959|N|N|N|N|N| +2431052|AAAAAAAAMEIBFCAA|1943-11-23|526|2291|176|1943|2|11|23|4|1943|176|2291|Tuesday|1943Q4|N|N|N|2431030|2431333|2430687|2430960|N|N|N|N|N| +2431053|AAAAAAAANEIBFCAA|1943-11-24|526|2291|176|1943|3|11|24|4|1943|176|2291|Wednesday|1943Q4|N|N|N|2431030|2431333|2430688|2430961|N|N|N|N|N| +2431054|AAAAAAAAOEIBFCAA|1943-11-25|526|2291|176|1943|4|11|25|4|1943|176|2291|Thursday|1943Q4|N|N|N|2431030|2431333|2430689|2430962|N|N|N|N|N| +2431055|AAAAAAAAPEIBFCAA|1943-11-26|526|2291|176|1943|5|11|26|4|1943|176|2291|Friday|1943Q4|N|Y|N|2431030|2431333|2430690|2430963|N|N|N|N|N| +2431056|AAAAAAAAAFIBFCAA|1943-11-27|526|2291|176|1943|6|11|27|4|1943|176|2291|Saturday|1943Q4|N|Y|N|2431030|2431333|2430691|2430964|N|N|N|N|N| +2431057|AAAAAAAABFIBFCAA|1943-11-28|526|2291|176|1943|0|11|28|4|1943|176|2291|Sunday|1943Q4|N|N|N|2431030|2431333|2430692|2430965|N|N|N|N|N| +2431058|AAAAAAAACFIBFCAA|1943-11-29|526|2291|176|1943|1|11|29|4|1943|176|2291|Monday|1943Q4|N|N|N|2431030|2431333|2430693|2430966|N|N|N|N|N| +2431059|AAAAAAAADFIBFCAA|1943-11-30|526|2292|176|1943|2|11|30|4|1943|176|2292|Tuesday|1943Q4|N|N|N|2431030|2431333|2430694|2430967|N|N|N|N|N| +2431060|AAAAAAAAEFIBFCAA|1943-12-01|527|2292|177|1943|3|12|1|4|1943|177|2292|Wednesday|1943Q4|N|N|N|2431060|2431393|2430695|2430968|N|N|N|N|N| +2431061|AAAAAAAAFFIBFCAA|1943-12-02|527|2292|177|1943|4|12|2|4|1943|177|2292|Thursday|1943Q4|N|N|N|2431060|2431393|2430696|2430969|N|N|N|N|N| +2431062|AAAAAAAAGFIBFCAA|1943-12-03|527|2292|177|1943|5|12|3|4|1943|177|2292|Friday|1943Q4|N|Y|N|2431060|2431393|2430697|2430970|N|N|N|N|N| +2431063|AAAAAAAAHFIBFCAA|1943-12-04|527|2292|177|1943|6|12|4|4|1943|177|2292|Saturday|1943Q4|N|Y|N|2431060|2431393|2430698|2430971|N|N|N|N|N| +2431064|AAAAAAAAIFIBFCAA|1943-12-05|527|2292|177|1943|0|12|5|4|1943|177|2292|Sunday|1943Q4|N|N|N|2431060|2431393|2430699|2430972|N|N|N|N|N| +2431065|AAAAAAAAJFIBFCAA|1943-12-06|527|2292|177|1943|1|12|6|4|1943|177|2292|Monday|1943Q4|N|N|N|2431060|2431393|2430700|2430973|N|N|N|N|N| +2431066|AAAAAAAAKFIBFCAA|1943-12-07|527|2293|177|1943|2|12|7|4|1943|177|2293|Tuesday|1943Q4|N|N|N|2431060|2431393|2430701|2430974|N|N|N|N|N| +2431067|AAAAAAAALFIBFCAA|1943-12-08|527|2293|177|1943|3|12|8|4|1943|177|2293|Wednesday|1943Q4|N|N|N|2431060|2431393|2430702|2430975|N|N|N|N|N| +2431068|AAAAAAAAMFIBFCAA|1943-12-09|527|2293|177|1943|4|12|9|4|1943|177|2293|Thursday|1943Q4|N|N|N|2431060|2431393|2430703|2430976|N|N|N|N|N| +2431069|AAAAAAAANFIBFCAA|1943-12-10|527|2293|177|1943|5|12|10|4|1943|177|2293|Friday|1943Q4|N|Y|N|2431060|2431393|2430704|2430977|N|N|N|N|N| +2431070|AAAAAAAAOFIBFCAA|1943-12-11|527|2293|177|1943|6|12|11|4|1943|177|2293|Saturday|1943Q4|N|Y|N|2431060|2431393|2430705|2430978|N|N|N|N|N| +2431071|AAAAAAAAPFIBFCAA|1943-12-12|527|2293|177|1943|0|12|12|4|1943|177|2293|Sunday|1943Q4|N|N|N|2431060|2431393|2430706|2430979|N|N|N|N|N| +2431072|AAAAAAAAAGIBFCAA|1943-12-13|527|2293|177|1943|1|12|13|4|1943|177|2293|Monday|1943Q4|N|N|N|2431060|2431393|2430707|2430980|N|N|N|N|N| +2431073|AAAAAAAABGIBFCAA|1943-12-14|527|2294|177|1943|2|12|14|4|1943|177|2294|Tuesday|1943Q4|N|N|N|2431060|2431393|2430708|2430981|N|N|N|N|N| +2431074|AAAAAAAACGIBFCAA|1943-12-15|527|2294|177|1943|3|12|15|4|1943|177|2294|Wednesday|1943Q4|N|N|N|2431060|2431393|2430709|2430982|N|N|N|N|N| +2431075|AAAAAAAADGIBFCAA|1943-12-16|527|2294|177|1943|4|12|16|4|1943|177|2294|Thursday|1943Q4|N|N|N|2431060|2431393|2430710|2430983|N|N|N|N|N| +2431076|AAAAAAAAEGIBFCAA|1943-12-17|527|2294|177|1943|5|12|17|4|1943|177|2294|Friday|1943Q4|N|Y|N|2431060|2431393|2430711|2430984|N|N|N|N|N| +2431077|AAAAAAAAFGIBFCAA|1943-12-18|527|2294|177|1943|6|12|18|4|1943|177|2294|Saturday|1943Q4|N|Y|N|2431060|2431393|2430712|2430985|N|N|N|N|N| +2431078|AAAAAAAAGGIBFCAA|1943-12-19|527|2294|177|1943|0|12|19|4|1943|177|2294|Sunday|1943Q4|N|N|N|2431060|2431393|2430713|2430986|N|N|N|N|N| +2431079|AAAAAAAAHGIBFCAA|1943-12-20|527|2294|177|1943|1|12|20|4|1943|177|2294|Monday|1943Q4|N|N|N|2431060|2431393|2430714|2430987|N|N|N|N|N| +2431080|AAAAAAAAIGIBFCAA|1943-12-21|527|2295|177|1943|2|12|21|4|1943|177|2295|Tuesday|1943Q4|N|N|N|2431060|2431393|2430715|2430988|N|N|N|N|N| +2431081|AAAAAAAAJGIBFCAA|1943-12-22|527|2295|177|1943|3|12|22|4|1943|177|2295|Wednesday|1943Q4|N|N|N|2431060|2431393|2430716|2430989|N|N|N|N|N| +2431082|AAAAAAAAKGIBFCAA|1943-12-23|527|2295|177|1943|4|12|23|4|1943|177|2295|Thursday|1943Q4|N|N|N|2431060|2431393|2430717|2430990|N|N|N|N|N| +2431083|AAAAAAAALGIBFCAA|1943-12-24|527|2295|177|1943|5|12|24|4|1943|177|2295|Friday|1943Q4|N|Y|N|2431060|2431393|2430718|2430991|N|N|N|N|N| +2431084|AAAAAAAAMGIBFCAA|1943-12-25|527|2295|177|1943|6|12|25|4|1943|177|2295|Saturday|1943Q4|N|Y|N|2431060|2431393|2430719|2430992|N|N|N|N|N| +2431085|AAAAAAAANGIBFCAA|1943-12-26|527|2295|177|1943|0|12|26|4|1943|177|2295|Sunday|1943Q4|Y|N|N|2431060|2431393|2430720|2430993|N|N|N|N|N| +2431086|AAAAAAAAOGIBFCAA|1943-12-27|527|2295|177|1943|1|12|27|4|1943|177|2295|Monday|1943Q4|N|N|Y|2431060|2431393|2430721|2430994|N|N|N|N|N| +2431087|AAAAAAAAPGIBFCAA|1943-12-28|527|2296|177|1943|2|12|28|4|1943|177|2296|Tuesday|1943Q4|N|N|N|2431060|2431393|2430722|2430995|N|N|N|N|N| +2431088|AAAAAAAAAHIBFCAA|1943-12-29|527|2296|177|1943|3|12|29|4|1943|177|2296|Wednesday|1943Q4|N|N|N|2431060|2431393|2430723|2430996|N|N|N|N|N| +2431089|AAAAAAAABHIBFCAA|1943-12-30|527|2296|177|1943|4|12|30|4|1943|177|2296|Thursday|1943Q4|N|N|N|2431060|2431393|2430724|2430997|N|N|N|N|N| +2431090|AAAAAAAACHIBFCAA|1943-12-31|527|2296|177|1943|5|12|31|4|1943|177|2296|Friday|1943Q4|N|Y|N|2431060|2431393|2430725|2430998|N|N|N|N|N| +2431091|AAAAAAAADHIBFCAA|1944-01-01|528|2296|177|1944|6|1|1|1|1944|177|2296|Saturday|1944Q1|Y|Y|N|2431091|2431090|2430726|2430999|N|N|N|N|N| +2431092|AAAAAAAAEHIBFCAA|1944-01-02|528|2296|177|1944|0|1|2|1|1944|177|2296|Sunday|1944Q1|N|N|Y|2431091|2431090|2430727|2431000|N|N|N|N|N| +2431093|AAAAAAAAFHIBFCAA|1944-01-03|528|2296|177|1944|1|1|3|1|1944|177|2296|Monday|1944Q1|N|N|N|2431091|2431090|2430728|2431001|N|N|N|N|N| +2431094|AAAAAAAAGHIBFCAA|1944-01-04|528|2297|177|1944|2|1|4|1|1944|177|2297|Tuesday|1944Q1|N|N|N|2431091|2431090|2430729|2431002|N|N|N|N|N| +2431095|AAAAAAAAHHIBFCAA|1944-01-05|528|2297|177|1944|3|1|5|1|1944|177|2297|Wednesday|1944Q1|N|N|N|2431091|2431090|2430730|2431003|N|N|N|N|N| +2431096|AAAAAAAAIHIBFCAA|1944-01-06|528|2297|177|1944|4|1|6|1|1944|177|2297|Thursday|1944Q1|N|N|N|2431091|2431090|2430731|2431004|N|N|N|N|N| +2431097|AAAAAAAAJHIBFCAA|1944-01-07|528|2297|177|1944|5|1|7|1|1944|177|2297|Friday|1944Q1|N|Y|N|2431091|2431090|2430732|2431005|N|N|N|N|N| +2431098|AAAAAAAAKHIBFCAA|1944-01-08|528|2297|177|1944|6|1|8|1|1944|177|2297|Saturday|1944Q1|N|Y|N|2431091|2431090|2430733|2431006|N|N|N|N|N| +2431099|AAAAAAAALHIBFCAA|1944-01-09|528|2297|177|1944|0|1|9|1|1944|177|2297|Sunday|1944Q1|N|N|N|2431091|2431090|2430734|2431007|N|N|N|N|N| +2431100|AAAAAAAAMHIBFCAA|1944-01-10|528|2297|177|1944|1|1|10|1|1944|177|2297|Monday|1944Q1|N|N|N|2431091|2431090|2430735|2431008|N|N|N|N|N| +2431101|AAAAAAAANHIBFCAA|1944-01-11|528|2298|177|1944|2|1|11|1|1944|177|2298|Tuesday|1944Q1|N|N|N|2431091|2431090|2430736|2431009|N|N|N|N|N| +2431102|AAAAAAAAOHIBFCAA|1944-01-12|528|2298|177|1944|3|1|12|1|1944|177|2298|Wednesday|1944Q1|N|N|N|2431091|2431090|2430737|2431010|N|N|N|N|N| +2431103|AAAAAAAAPHIBFCAA|1944-01-13|528|2298|177|1944|4|1|13|1|1944|177|2298|Thursday|1944Q1|N|N|N|2431091|2431090|2430738|2431011|N|N|N|N|N| +2431104|AAAAAAAAAIIBFCAA|1944-01-14|528|2298|177|1944|5|1|14|1|1944|177|2298|Friday|1944Q1|N|Y|N|2431091|2431090|2430739|2431012|N|N|N|N|N| +2431105|AAAAAAAABIIBFCAA|1944-01-15|528|2298|177|1944|6|1|15|1|1944|177|2298|Saturday|1944Q1|N|Y|N|2431091|2431090|2430740|2431013|N|N|N|N|N| +2431106|AAAAAAAACIIBFCAA|1944-01-16|528|2298|177|1944|0|1|16|1|1944|177|2298|Sunday|1944Q1|N|N|N|2431091|2431090|2430741|2431014|N|N|N|N|N| +2431107|AAAAAAAADIIBFCAA|1944-01-17|528|2298|177|1944|1|1|17|1|1944|177|2298|Monday|1944Q1|N|N|N|2431091|2431090|2430742|2431015|N|N|N|N|N| +2431108|AAAAAAAAEIIBFCAA|1944-01-18|528|2299|177|1944|2|1|18|1|1944|177|2299|Tuesday|1944Q1|N|N|N|2431091|2431090|2430743|2431016|N|N|N|N|N| +2431109|AAAAAAAAFIIBFCAA|1944-01-19|528|2299|177|1944|3|1|19|1|1944|177|2299|Wednesday|1944Q1|N|N|N|2431091|2431090|2430744|2431017|N|N|N|N|N| +2431110|AAAAAAAAGIIBFCAA|1944-01-20|528|2299|177|1944|4|1|20|1|1944|177|2299|Thursday|1944Q1|N|N|N|2431091|2431090|2430745|2431018|N|N|N|N|N| +2431111|AAAAAAAAHIIBFCAA|1944-01-21|528|2299|177|1944|5|1|21|1|1944|177|2299|Friday|1944Q1|N|Y|N|2431091|2431090|2430746|2431019|N|N|N|N|N| +2431112|AAAAAAAAIIIBFCAA|1944-01-22|528|2299|177|1944|6|1|22|1|1944|177|2299|Saturday|1944Q1|N|Y|N|2431091|2431090|2430747|2431020|N|N|N|N|N| +2431113|AAAAAAAAJIIBFCAA|1944-01-23|528|2299|177|1944|0|1|23|1|1944|177|2299|Sunday|1944Q1|N|N|N|2431091|2431090|2430748|2431021|N|N|N|N|N| +2431114|AAAAAAAAKIIBFCAA|1944-01-24|528|2299|177|1944|1|1|24|1|1944|177|2299|Monday|1944Q1|N|N|N|2431091|2431090|2430749|2431022|N|N|N|N|N| +2431115|AAAAAAAALIIBFCAA|1944-01-25|528|2300|177|1944|2|1|25|1|1944|177|2300|Tuesday|1944Q1|N|N|N|2431091|2431090|2430750|2431023|N|N|N|N|N| +2431116|AAAAAAAAMIIBFCAA|1944-01-26|528|2300|177|1944|3|1|26|1|1944|177|2300|Wednesday|1944Q1|N|N|N|2431091|2431090|2430751|2431024|N|N|N|N|N| +2431117|AAAAAAAANIIBFCAA|1944-01-27|528|2300|177|1944|4|1|27|1|1944|177|2300|Thursday|1944Q1|N|N|N|2431091|2431090|2430752|2431025|N|N|N|N|N| +2431118|AAAAAAAAOIIBFCAA|1944-01-28|528|2300|177|1944|5|1|28|1|1944|177|2300|Friday|1944Q1|N|Y|N|2431091|2431090|2430753|2431026|N|N|N|N|N| +2431119|AAAAAAAAPIIBFCAA|1944-01-29|528|2300|177|1944|6|1|29|1|1944|177|2300|Saturday|1944Q1|N|Y|N|2431091|2431090|2430754|2431027|N|N|N|N|N| +2431120|AAAAAAAAAJIBFCAA|1944-01-30|528|2300|177|1944|0|1|30|1|1944|177|2300|Sunday|1944Q1|N|N|N|2431091|2431090|2430755|2431028|N|N|N|N|N| +2431121|AAAAAAAABJIBFCAA|1944-01-31|528|2300|177|1944|1|1|31|1|1944|177|2300|Monday|1944Q1|N|N|N|2431091|2431090|2430756|2431029|N|N|N|N|N| +2431122|AAAAAAAACJIBFCAA|1944-02-01|529|2301|177|1944|2|2|1|1|1944|177|2301|Tuesday|1944Q1|N|N|N|2431122|2431152|2430757|2431030|N|N|N|N|N| +2431123|AAAAAAAADJIBFCAA|1944-02-02|529|2301|177|1944|3|2|2|1|1944|177|2301|Wednesday|1944Q1|N|N|N|2431122|2431152|2430758|2431031|N|N|N|N|N| +2431124|AAAAAAAAEJIBFCAA|1944-02-03|529|2301|177|1944|4|2|3|1|1944|177|2301|Thursday|1944Q1|N|N|N|2431122|2431152|2430759|2431032|N|N|N|N|N| +2431125|AAAAAAAAFJIBFCAA|1944-02-04|529|2301|177|1944|5|2|4|1|1944|177|2301|Friday|1944Q1|N|Y|N|2431122|2431152|2430760|2431033|N|N|N|N|N| +2431126|AAAAAAAAGJIBFCAA|1944-02-05|529|2301|177|1944|6|2|5|1|1944|177|2301|Saturday|1944Q1|N|Y|N|2431122|2431152|2430761|2431034|N|N|N|N|N| +2431127|AAAAAAAAHJIBFCAA|1944-02-06|529|2301|177|1944|0|2|6|1|1944|177|2301|Sunday|1944Q1|N|N|N|2431122|2431152|2430762|2431035|N|N|N|N|N| +2431128|AAAAAAAAIJIBFCAA|1944-02-07|529|2301|177|1944|1|2|7|1|1944|177|2301|Monday|1944Q1|N|N|N|2431122|2431152|2430763|2431036|N|N|N|N|N| +2431129|AAAAAAAAJJIBFCAA|1944-02-08|529|2302|177|1944|2|2|8|1|1944|177|2302|Tuesday|1944Q1|N|N|N|2431122|2431152|2430764|2431037|N|N|N|N|N| +2431130|AAAAAAAAKJIBFCAA|1944-02-09|529|2302|177|1944|3|2|9|1|1944|177|2302|Wednesday|1944Q1|N|N|N|2431122|2431152|2430765|2431038|N|N|N|N|N| +2431131|AAAAAAAALJIBFCAA|1944-02-10|529|2302|177|1944|4|2|10|1|1944|177|2302|Thursday|1944Q1|N|N|N|2431122|2431152|2430766|2431039|N|N|N|N|N| +2431132|AAAAAAAAMJIBFCAA|1944-02-11|529|2302|177|1944|5|2|11|1|1944|177|2302|Friday|1944Q1|N|Y|N|2431122|2431152|2430767|2431040|N|N|N|N|N| +2431133|AAAAAAAANJIBFCAA|1944-02-12|529|2302|177|1944|6|2|12|1|1944|177|2302|Saturday|1944Q1|N|Y|N|2431122|2431152|2430768|2431041|N|N|N|N|N| +2431134|AAAAAAAAOJIBFCAA|1944-02-13|529|2302|177|1944|0|2|13|1|1944|177|2302|Sunday|1944Q1|N|N|N|2431122|2431152|2430769|2431042|N|N|N|N|N| +2431135|AAAAAAAAPJIBFCAA|1944-02-14|529|2302|177|1944|1|2|14|1|1944|177|2302|Monday|1944Q1|N|N|N|2431122|2431152|2430770|2431043|N|N|N|N|N| +2431136|AAAAAAAAAKIBFCAA|1944-02-15|529|2303|177|1944|2|2|15|1|1944|177|2303|Tuesday|1944Q1|N|N|N|2431122|2431152|2430771|2431044|N|N|N|N|N| +2431137|AAAAAAAABKIBFCAA|1944-02-16|529|2303|177|1944|3|2|16|1|1944|177|2303|Wednesday|1944Q1|N|N|N|2431122|2431152|2430772|2431045|N|N|N|N|N| +2431138|AAAAAAAACKIBFCAA|1944-02-17|529|2303|177|1944|4|2|17|1|1944|177|2303|Thursday|1944Q1|N|N|N|2431122|2431152|2430773|2431046|N|N|N|N|N| +2431139|AAAAAAAADKIBFCAA|1944-02-18|529|2303|177|1944|5|2|18|1|1944|177|2303|Friday|1944Q1|N|Y|N|2431122|2431152|2430774|2431047|N|N|N|N|N| +2431140|AAAAAAAAEKIBFCAA|1944-02-19|529|2303|177|1944|6|2|19|1|1944|177|2303|Saturday|1944Q1|N|Y|N|2431122|2431152|2430775|2431048|N|N|N|N|N| +2431141|AAAAAAAAFKIBFCAA|1944-02-20|529|2303|177|1944|0|2|20|1|1944|177|2303|Sunday|1944Q1|N|N|N|2431122|2431152|2430776|2431049|N|N|N|N|N| +2431142|AAAAAAAAGKIBFCAA|1944-02-21|529|2303|177|1944|1|2|21|1|1944|177|2303|Monday|1944Q1|N|N|N|2431122|2431152|2430777|2431050|N|N|N|N|N| +2431143|AAAAAAAAHKIBFCAA|1944-02-22|529|2304|177|1944|2|2|22|1|1944|177|2304|Tuesday|1944Q1|N|N|N|2431122|2431152|2430778|2431051|N|N|N|N|N| +2431144|AAAAAAAAIKIBFCAA|1944-02-23|529|2304|177|1944|3|2|23|1|1944|177|2304|Wednesday|1944Q1|N|N|N|2431122|2431152|2430779|2431052|N|N|N|N|N| +2431145|AAAAAAAAJKIBFCAA|1944-02-24|529|2304|177|1944|4|2|24|1|1944|177|2304|Thursday|1944Q1|N|N|N|2431122|2431152|2430780|2431053|N|N|N|N|N| +2431146|AAAAAAAAKKIBFCAA|1944-02-25|529|2304|177|1944|5|2|25|1|1944|177|2304|Friday|1944Q1|N|Y|N|2431122|2431152|2430781|2431054|N|N|N|N|N| +2431147|AAAAAAAALKIBFCAA|1944-02-26|529|2304|177|1944|6|2|26|1|1944|177|2304|Saturday|1944Q1|N|Y|N|2431122|2431152|2430782|2431055|N|N|N|N|N| +2431148|AAAAAAAAMKIBFCAA|1944-02-27|529|2304|177|1944|0|2|27|1|1944|177|2304|Sunday|1944Q1|N|N|N|2431122|2431152|2430783|2431056|N|N|N|N|N| +2431149|AAAAAAAANKIBFCAA|1944-02-28|529|2304|177|1944|1|2|28|1|1944|177|2304|Monday|1944Q1|N|N|N|2431122|2431152|2430784|2431057|N|N|N|N|N| +2431150|AAAAAAAAOKIBFCAA|1944-02-29|529|2305|177|1944|2|2|29|1|1944|177|2305|Tuesday|1944Q1|N|N|N|2431122|2431152|2430784|2431058|N|N|N|N|N| +2431151|AAAAAAAAPKIBFCAA|1944-03-01|530|2305|178|1944|3|3|1|1|1944|178|2305|Wednesday|1944Q1|N|N|N|2431151|2431210|2430785|2431059|N|N|N|N|N| +2431152|AAAAAAAAALIBFCAA|1944-03-02|530|2305|178|1944|4|3|2|1|1944|178|2305|Thursday|1944Q1|N|N|N|2431151|2431210|2430786|2431060|N|N|N|N|N| +2431153|AAAAAAAABLIBFCAA|1944-03-03|530|2305|178|1944|5|3|3|1|1944|178|2305|Friday|1944Q1|N|Y|N|2431151|2431210|2430787|2431061|N|N|N|N|N| +2431154|AAAAAAAACLIBFCAA|1944-03-04|530|2305|178|1944|6|3|4|1|1944|178|2305|Saturday|1944Q1|N|Y|N|2431151|2431210|2430788|2431062|N|N|N|N|N| +2431155|AAAAAAAADLIBFCAA|1944-03-05|530|2305|178|1944|0|3|5|1|1944|178|2305|Sunday|1944Q1|N|N|N|2431151|2431210|2430789|2431063|N|N|N|N|N| +2431156|AAAAAAAAELIBFCAA|1944-03-06|530|2305|178|1944|1|3|6|1|1944|178|2305|Monday|1944Q1|N|N|N|2431151|2431210|2430790|2431064|N|N|N|N|N| +2431157|AAAAAAAAFLIBFCAA|1944-03-07|530|2306|178|1944|2|3|7|1|1944|178|2306|Tuesday|1944Q1|N|N|N|2431151|2431210|2430791|2431065|N|N|N|N|N| +2431158|AAAAAAAAGLIBFCAA|1944-03-08|530|2306|178|1944|3|3|8|1|1944|178|2306|Wednesday|1944Q1|N|N|N|2431151|2431210|2430792|2431066|N|N|N|N|N| +2431159|AAAAAAAAHLIBFCAA|1944-03-09|530|2306|178|1944|4|3|9|1|1944|178|2306|Thursday|1944Q1|N|N|N|2431151|2431210|2430793|2431067|N|N|N|N|N| +2431160|AAAAAAAAILIBFCAA|1944-03-10|530|2306|178|1944|5|3|10|1|1944|178|2306|Friday|1944Q1|N|Y|N|2431151|2431210|2430794|2431068|N|N|N|N|N| +2431161|AAAAAAAAJLIBFCAA|1944-03-11|530|2306|178|1944|6|3|11|1|1944|178|2306|Saturday|1944Q1|N|Y|N|2431151|2431210|2430795|2431069|N|N|N|N|N| +2431162|AAAAAAAAKLIBFCAA|1944-03-12|530|2306|178|1944|0|3|12|1|1944|178|2306|Sunday|1944Q1|N|N|N|2431151|2431210|2430796|2431070|N|N|N|N|N| +2431163|AAAAAAAALLIBFCAA|1944-03-13|530|2306|178|1944|1|3|13|1|1944|178|2306|Monday|1944Q1|N|N|N|2431151|2431210|2430797|2431071|N|N|N|N|N| +2431164|AAAAAAAAMLIBFCAA|1944-03-14|530|2307|178|1944|2|3|14|1|1944|178|2307|Tuesday|1944Q1|N|N|N|2431151|2431210|2430798|2431072|N|N|N|N|N| +2431165|AAAAAAAANLIBFCAA|1944-03-15|530|2307|178|1944|3|3|15|1|1944|178|2307|Wednesday|1944Q1|N|N|N|2431151|2431210|2430799|2431073|N|N|N|N|N| +2431166|AAAAAAAAOLIBFCAA|1944-03-16|530|2307|178|1944|4|3|16|1|1944|178|2307|Thursday|1944Q1|N|N|N|2431151|2431210|2430800|2431074|N|N|N|N|N| +2431167|AAAAAAAAPLIBFCAA|1944-03-17|530|2307|178|1944|5|3|17|1|1944|178|2307|Friday|1944Q1|N|Y|N|2431151|2431210|2430801|2431075|N|N|N|N|N| +2431168|AAAAAAAAAMIBFCAA|1944-03-18|530|2307|178|1944|6|3|18|1|1944|178|2307|Saturday|1944Q1|N|Y|N|2431151|2431210|2430802|2431076|N|N|N|N|N| +2431169|AAAAAAAABMIBFCAA|1944-03-19|530|2307|178|1944|0|3|19|1|1944|178|2307|Sunday|1944Q1|N|N|N|2431151|2431210|2430803|2431077|N|N|N|N|N| +2431170|AAAAAAAACMIBFCAA|1944-03-20|530|2307|178|1944|1|3|20|1|1944|178|2307|Monday|1944Q1|N|N|N|2431151|2431210|2430804|2431078|N|N|N|N|N| +2431171|AAAAAAAADMIBFCAA|1944-03-21|530|2308|178|1944|2|3|21|1|1944|178|2308|Tuesday|1944Q1|N|N|N|2431151|2431210|2430805|2431079|N|N|N|N|N| +2431172|AAAAAAAAEMIBFCAA|1944-03-22|530|2308|178|1944|3|3|22|1|1944|178|2308|Wednesday|1944Q1|N|N|N|2431151|2431210|2430806|2431080|N|N|N|N|N| +2431173|AAAAAAAAFMIBFCAA|1944-03-23|530|2308|178|1944|4|3|23|1|1944|178|2308|Thursday|1944Q1|N|N|N|2431151|2431210|2430807|2431081|N|N|N|N|N| +2431174|AAAAAAAAGMIBFCAA|1944-03-24|530|2308|178|1944|5|3|24|1|1944|178|2308|Friday|1944Q1|N|Y|N|2431151|2431210|2430808|2431082|N|N|N|N|N| +2431175|AAAAAAAAHMIBFCAA|1944-03-25|530|2308|178|1944|6|3|25|1|1944|178|2308|Saturday|1944Q1|N|Y|N|2431151|2431210|2430809|2431083|N|N|N|N|N| +2431176|AAAAAAAAIMIBFCAA|1944-03-26|530|2308|178|1944|0|3|26|1|1944|178|2308|Sunday|1944Q1|N|N|N|2431151|2431210|2430810|2431084|N|N|N|N|N| +2431177|AAAAAAAAJMIBFCAA|1944-03-27|530|2308|178|1944|1|3|27|1|1944|178|2308|Monday|1944Q1|N|N|N|2431151|2431210|2430811|2431085|N|N|N|N|N| +2431178|AAAAAAAAKMIBFCAA|1944-03-28|530|2309|178|1944|2|3|28|1|1944|178|2309|Tuesday|1944Q1|N|N|N|2431151|2431210|2430812|2431086|N|N|N|N|N| +2431179|AAAAAAAALMIBFCAA|1944-03-29|530|2309|178|1944|3|3|29|1|1944|178|2309|Wednesday|1944Q1|N|N|N|2431151|2431210|2430813|2431087|N|N|N|N|N| +2431180|AAAAAAAAMMIBFCAA|1944-03-30|530|2309|178|1944|4|3|30|1|1944|178|2309|Thursday|1944Q1|N|N|N|2431151|2431210|2430814|2431088|N|N|N|N|N| +2431181|AAAAAAAANMIBFCAA|1944-03-31|530|2309|178|1944|5|3|31|1|1944|178|2309|Friday|1944Q1|N|Y|N|2431151|2431210|2430815|2431089|N|N|N|N|N| +2431182|AAAAAAAAOMIBFCAA|1944-04-01|531|2309|178|1944|6|4|1|2|1944|178|2309|Saturday|1944Q2|N|Y|N|2431182|2431272|2430816|2431091|N|N|N|N|N| +2431183|AAAAAAAAPMIBFCAA|1944-04-02|531|2309|178|1944|0|4|2|2|1944|178|2309|Sunday|1944Q2|N|N|N|2431182|2431272|2430817|2431092|N|N|N|N|N| +2431184|AAAAAAAAANIBFCAA|1944-04-03|531|2309|178|1944|1|4|3|2|1944|178|2309|Monday|1944Q2|N|N|N|2431182|2431272|2430818|2431093|N|N|N|N|N| +2431185|AAAAAAAABNIBFCAA|1944-04-04|531|2310|178|1944|2|4|4|2|1944|178|2310|Tuesday|1944Q2|N|N|N|2431182|2431272|2430819|2431094|N|N|N|N|N| +2431186|AAAAAAAACNIBFCAA|1944-04-05|531|2310|178|1944|3|4|5|2|1944|178|2310|Wednesday|1944Q2|N|N|N|2431182|2431272|2430820|2431095|N|N|N|N|N| +2431187|AAAAAAAADNIBFCAA|1944-04-06|531|2310|178|1944|4|4|6|2|1944|178|2310|Thursday|1944Q2|N|N|N|2431182|2431272|2430821|2431096|N|N|N|N|N| +2431188|AAAAAAAAENIBFCAA|1944-04-07|531|2310|178|1944|5|4|7|2|1944|178|2310|Friday|1944Q2|N|Y|N|2431182|2431272|2430822|2431097|N|N|N|N|N| +2431189|AAAAAAAAFNIBFCAA|1944-04-08|531|2310|178|1944|6|4|8|2|1944|178|2310|Saturday|1944Q2|N|Y|N|2431182|2431272|2430823|2431098|N|N|N|N|N| +2431190|AAAAAAAAGNIBFCAA|1944-04-09|531|2310|178|1944|0|4|9|2|1944|178|2310|Sunday|1944Q2|N|N|N|2431182|2431272|2430824|2431099|N|N|N|N|N| +2431191|AAAAAAAAHNIBFCAA|1944-04-10|531|2310|178|1944|1|4|10|2|1944|178|2310|Monday|1944Q2|N|N|N|2431182|2431272|2430825|2431100|N|N|N|N|N| +2431192|AAAAAAAAINIBFCAA|1944-04-11|531|2311|178|1944|2|4|11|2|1944|178|2311|Tuesday|1944Q2|N|N|N|2431182|2431272|2430826|2431101|N|N|N|N|N| +2431193|AAAAAAAAJNIBFCAA|1944-04-12|531|2311|178|1944|3|4|12|2|1944|178|2311|Wednesday|1944Q2|N|N|N|2431182|2431272|2430827|2431102|N|N|N|N|N| +2431194|AAAAAAAAKNIBFCAA|1944-04-13|531|2311|178|1944|4|4|13|2|1944|178|2311|Thursday|1944Q2|N|N|N|2431182|2431272|2430828|2431103|N|N|N|N|N| +2431195|AAAAAAAALNIBFCAA|1944-04-14|531|2311|178|1944|5|4|14|2|1944|178|2311|Friday|1944Q2|N|Y|N|2431182|2431272|2430829|2431104|N|N|N|N|N| +2431196|AAAAAAAAMNIBFCAA|1944-04-15|531|2311|178|1944|6|4|15|2|1944|178|2311|Saturday|1944Q2|N|Y|N|2431182|2431272|2430830|2431105|N|N|N|N|N| +2431197|AAAAAAAANNIBFCAA|1944-04-16|531|2311|178|1944|0|4|16|2|1944|178|2311|Sunday|1944Q2|N|N|N|2431182|2431272|2430831|2431106|N|N|N|N|N| +2431198|AAAAAAAAONIBFCAA|1944-04-17|531|2311|178|1944|1|4|17|2|1944|178|2311|Monday|1944Q2|N|N|N|2431182|2431272|2430832|2431107|N|N|N|N|N| +2431199|AAAAAAAAPNIBFCAA|1944-04-18|531|2312|178|1944|2|4|18|2|1944|178|2312|Tuesday|1944Q2|N|N|N|2431182|2431272|2430833|2431108|N|N|N|N|N| +2431200|AAAAAAAAAOIBFCAA|1944-04-19|531|2312|178|1944|3|4|19|2|1944|178|2312|Wednesday|1944Q2|N|N|N|2431182|2431272|2430834|2431109|N|N|N|N|N| +2431201|AAAAAAAABOIBFCAA|1944-04-20|531|2312|178|1944|4|4|20|2|1944|178|2312|Thursday|1944Q2|N|N|N|2431182|2431272|2430835|2431110|N|N|N|N|N| +2431202|AAAAAAAACOIBFCAA|1944-04-21|531|2312|178|1944|5|4|21|2|1944|178|2312|Friday|1944Q2|N|Y|N|2431182|2431272|2430836|2431111|N|N|N|N|N| +2431203|AAAAAAAADOIBFCAA|1944-04-22|531|2312|178|1944|6|4|22|2|1944|178|2312|Saturday|1944Q2|N|Y|N|2431182|2431272|2430837|2431112|N|N|N|N|N| +2431204|AAAAAAAAEOIBFCAA|1944-04-23|531|2312|178|1944|0|4|23|2|1944|178|2312|Sunday|1944Q2|N|N|N|2431182|2431272|2430838|2431113|N|N|N|N|N| +2431205|AAAAAAAAFOIBFCAA|1944-04-24|531|2312|178|1944|1|4|24|2|1944|178|2312|Monday|1944Q2|N|N|N|2431182|2431272|2430839|2431114|N|N|N|N|N| +2431206|AAAAAAAAGOIBFCAA|1944-04-25|531|2313|178|1944|2|4|25|2|1944|178|2313|Tuesday|1944Q2|N|N|N|2431182|2431272|2430840|2431115|N|N|N|N|N| +2431207|AAAAAAAAHOIBFCAA|1944-04-26|531|2313|178|1944|3|4|26|2|1944|178|2313|Wednesday|1944Q2|N|N|N|2431182|2431272|2430841|2431116|N|N|N|N|N| +2431208|AAAAAAAAIOIBFCAA|1944-04-27|531|2313|178|1944|4|4|27|2|1944|178|2313|Thursday|1944Q2|N|N|N|2431182|2431272|2430842|2431117|N|N|N|N|N| +2431209|AAAAAAAAJOIBFCAA|1944-04-28|531|2313|178|1944|5|4|28|2|1944|178|2313|Friday|1944Q2|N|Y|N|2431182|2431272|2430843|2431118|N|N|N|N|N| +2431210|AAAAAAAAKOIBFCAA|1944-04-29|531|2313|178|1944|6|4|29|2|1944|178|2313|Saturday|1944Q2|N|Y|N|2431182|2431272|2430844|2431119|N|N|N|N|N| +2431211|AAAAAAAALOIBFCAA|1944-04-30|531|2313|178|1944|0|4|30|2|1944|178|2313|Sunday|1944Q2|N|N|N|2431182|2431272|2430845|2431120|N|N|N|N|N| +2431212|AAAAAAAAMOIBFCAA|1944-05-01|532|2313|178|1944|1|5|1|2|1944|178|2313|Monday|1944Q2|N|N|N|2431212|2431332|2430846|2431121|N|N|N|N|N| +2431213|AAAAAAAANOIBFCAA|1944-05-02|532|2314|178|1944|2|5|2|2|1944|178|2314|Tuesday|1944Q2|N|N|N|2431212|2431332|2430847|2431122|N|N|N|N|N| +2431214|AAAAAAAAOOIBFCAA|1944-05-03|532|2314|178|1944|3|5|3|2|1944|178|2314|Wednesday|1944Q2|N|N|N|2431212|2431332|2430848|2431123|N|N|N|N|N| +2431215|AAAAAAAAPOIBFCAA|1944-05-04|532|2314|178|1944|4|5|4|2|1944|178|2314|Thursday|1944Q2|N|N|N|2431212|2431332|2430849|2431124|N|N|N|N|N| +2431216|AAAAAAAAAPIBFCAA|1944-05-05|532|2314|178|1944|5|5|5|2|1944|178|2314|Friday|1944Q2|N|Y|N|2431212|2431332|2430850|2431125|N|N|N|N|N| +2431217|AAAAAAAABPIBFCAA|1944-05-06|532|2314|178|1944|6|5|6|2|1944|178|2314|Saturday|1944Q2|N|Y|N|2431212|2431332|2430851|2431126|N|N|N|N|N| +2431218|AAAAAAAACPIBFCAA|1944-05-07|532|2314|178|1944|0|5|7|2|1944|178|2314|Sunday|1944Q2|N|N|N|2431212|2431332|2430852|2431127|N|N|N|N|N| +2431219|AAAAAAAADPIBFCAA|1944-05-08|532|2314|178|1944|1|5|8|2|1944|178|2314|Monday|1944Q2|N|N|N|2431212|2431332|2430853|2431128|N|N|N|N|N| +2431220|AAAAAAAAEPIBFCAA|1944-05-09|532|2315|178|1944|2|5|9|2|1944|178|2315|Tuesday|1944Q2|N|N|N|2431212|2431332|2430854|2431129|N|N|N|N|N| +2431221|AAAAAAAAFPIBFCAA|1944-05-10|532|2315|178|1944|3|5|10|2|1944|178|2315|Wednesday|1944Q2|N|N|N|2431212|2431332|2430855|2431130|N|N|N|N|N| +2431222|AAAAAAAAGPIBFCAA|1944-05-11|532|2315|178|1944|4|5|11|2|1944|178|2315|Thursday|1944Q2|N|N|N|2431212|2431332|2430856|2431131|N|N|N|N|N| +2431223|AAAAAAAAHPIBFCAA|1944-05-12|532|2315|178|1944|5|5|12|2|1944|178|2315|Friday|1944Q2|N|Y|N|2431212|2431332|2430857|2431132|N|N|N|N|N| +2431224|AAAAAAAAIPIBFCAA|1944-05-13|532|2315|178|1944|6|5|13|2|1944|178|2315|Saturday|1944Q2|N|Y|N|2431212|2431332|2430858|2431133|N|N|N|N|N| +2431225|AAAAAAAAJPIBFCAA|1944-05-14|532|2315|178|1944|0|5|14|2|1944|178|2315|Sunday|1944Q2|N|N|N|2431212|2431332|2430859|2431134|N|N|N|N|N| +2431226|AAAAAAAAKPIBFCAA|1944-05-15|532|2315|178|1944|1|5|15|2|1944|178|2315|Monday|1944Q2|N|N|N|2431212|2431332|2430860|2431135|N|N|N|N|N| +2431227|AAAAAAAALPIBFCAA|1944-05-16|532|2316|178|1944|2|5|16|2|1944|178|2316|Tuesday|1944Q2|N|N|N|2431212|2431332|2430861|2431136|N|N|N|N|N| +2431228|AAAAAAAAMPIBFCAA|1944-05-17|532|2316|178|1944|3|5|17|2|1944|178|2316|Wednesday|1944Q2|N|N|N|2431212|2431332|2430862|2431137|N|N|N|N|N| +2431229|AAAAAAAANPIBFCAA|1944-05-18|532|2316|178|1944|4|5|18|2|1944|178|2316|Thursday|1944Q2|N|N|N|2431212|2431332|2430863|2431138|N|N|N|N|N| +2431230|AAAAAAAAOPIBFCAA|1944-05-19|532|2316|178|1944|5|5|19|2|1944|178|2316|Friday|1944Q2|N|Y|N|2431212|2431332|2430864|2431139|N|N|N|N|N| +2431231|AAAAAAAAPPIBFCAA|1944-05-20|532|2316|178|1944|6|5|20|2|1944|178|2316|Saturday|1944Q2|N|Y|N|2431212|2431332|2430865|2431140|N|N|N|N|N| +2431232|AAAAAAAAAAJBFCAA|1944-05-21|532|2316|178|1944|0|5|21|2|1944|178|2316|Sunday|1944Q2|N|N|N|2431212|2431332|2430866|2431141|N|N|N|N|N| +2431233|AAAAAAAABAJBFCAA|1944-05-22|532|2316|178|1944|1|5|22|2|1944|178|2316|Monday|1944Q2|N|N|N|2431212|2431332|2430867|2431142|N|N|N|N|N| +2431234|AAAAAAAACAJBFCAA|1944-05-23|532|2317|178|1944|2|5|23|2|1944|178|2317|Tuesday|1944Q2|N|N|N|2431212|2431332|2430868|2431143|N|N|N|N|N| +2431235|AAAAAAAADAJBFCAA|1944-05-24|532|2317|178|1944|3|5|24|2|1944|178|2317|Wednesday|1944Q2|N|N|N|2431212|2431332|2430869|2431144|N|N|N|N|N| +2431236|AAAAAAAAEAJBFCAA|1944-05-25|532|2317|178|1944|4|5|25|2|1944|178|2317|Thursday|1944Q2|N|N|N|2431212|2431332|2430870|2431145|N|N|N|N|N| +2431237|AAAAAAAAFAJBFCAA|1944-05-26|532|2317|178|1944|5|5|26|2|1944|178|2317|Friday|1944Q2|N|Y|N|2431212|2431332|2430871|2431146|N|N|N|N|N| +2431238|AAAAAAAAGAJBFCAA|1944-05-27|532|2317|178|1944|6|5|27|2|1944|178|2317|Saturday|1944Q2|N|Y|N|2431212|2431332|2430872|2431147|N|N|N|N|N| +2431239|AAAAAAAAHAJBFCAA|1944-05-28|532|2317|178|1944|0|5|28|2|1944|178|2317|Sunday|1944Q2|N|N|N|2431212|2431332|2430873|2431148|N|N|N|N|N| +2431240|AAAAAAAAIAJBFCAA|1944-05-29|532|2317|178|1944|1|5|29|2|1944|178|2317|Monday|1944Q2|N|N|N|2431212|2431332|2430874|2431149|N|N|N|N|N| +2431241|AAAAAAAAJAJBFCAA|1944-05-30|532|2318|178|1944|2|5|30|2|1944|178|2318|Tuesday|1944Q2|N|N|N|2431212|2431332|2430875|2431150|N|N|N|N|N| +2431242|AAAAAAAAKAJBFCAA|1944-05-31|532|2318|178|1944|3|5|31|2|1944|178|2318|Wednesday|1944Q2|N|N|N|2431212|2431332|2430876|2431151|N|N|N|N|N| +2431243|AAAAAAAALAJBFCAA|1944-06-01|533|2318|179|1944|4|6|1|2|1944|179|2318|Thursday|1944Q2|N|N|N|2431243|2431394|2430877|2431152|N|N|N|N|N| +2431244|AAAAAAAAMAJBFCAA|1944-06-02|533|2318|179|1944|5|6|2|2|1944|179|2318|Friday|1944Q2|N|Y|N|2431243|2431394|2430878|2431153|N|N|N|N|N| +2431245|AAAAAAAANAJBFCAA|1944-06-03|533|2318|179|1944|6|6|3|2|1944|179|2318|Saturday|1944Q2|N|Y|N|2431243|2431394|2430879|2431154|N|N|N|N|N| +2431246|AAAAAAAAOAJBFCAA|1944-06-04|533|2318|179|1944|0|6|4|2|1944|179|2318|Sunday|1944Q2|N|N|N|2431243|2431394|2430880|2431155|N|N|N|N|N| +2431247|AAAAAAAAPAJBFCAA|1944-06-05|533|2318|179|1944|1|6|5|2|1944|179|2318|Monday|1944Q2|N|N|N|2431243|2431394|2430881|2431156|N|N|N|N|N| +2431248|AAAAAAAAABJBFCAA|1944-06-06|533|2319|179|1944|2|6|6|2|1944|179|2319|Tuesday|1944Q2|N|N|N|2431243|2431394|2430882|2431157|N|N|N|N|N| +2431249|AAAAAAAABBJBFCAA|1944-06-07|533|2319|179|1944|3|6|7|2|1944|179|2319|Wednesday|1944Q2|N|N|N|2431243|2431394|2430883|2431158|N|N|N|N|N| +2431250|AAAAAAAACBJBFCAA|1944-06-08|533|2319|179|1944|4|6|8|2|1944|179|2319|Thursday|1944Q2|N|N|N|2431243|2431394|2430884|2431159|N|N|N|N|N| +2431251|AAAAAAAADBJBFCAA|1944-06-09|533|2319|179|1944|5|6|9|2|1944|179|2319|Friday|1944Q2|N|Y|N|2431243|2431394|2430885|2431160|N|N|N|N|N| +2431252|AAAAAAAAEBJBFCAA|1944-06-10|533|2319|179|1944|6|6|10|2|1944|179|2319|Saturday|1944Q2|N|Y|N|2431243|2431394|2430886|2431161|N|N|N|N|N| +2431253|AAAAAAAAFBJBFCAA|1944-06-11|533|2319|179|1944|0|6|11|2|1944|179|2319|Sunday|1944Q2|N|N|N|2431243|2431394|2430887|2431162|N|N|N|N|N| +2431254|AAAAAAAAGBJBFCAA|1944-06-12|533|2319|179|1944|1|6|12|2|1944|179|2319|Monday|1944Q2|N|N|N|2431243|2431394|2430888|2431163|N|N|N|N|N| +2431255|AAAAAAAAHBJBFCAA|1944-06-13|533|2320|179|1944|2|6|13|2|1944|179|2320|Tuesday|1944Q2|N|N|N|2431243|2431394|2430889|2431164|N|N|N|N|N| +2431256|AAAAAAAAIBJBFCAA|1944-06-14|533|2320|179|1944|3|6|14|2|1944|179|2320|Wednesday|1944Q2|N|N|N|2431243|2431394|2430890|2431165|N|N|N|N|N| +2431257|AAAAAAAAJBJBFCAA|1944-06-15|533|2320|179|1944|4|6|15|2|1944|179|2320|Thursday|1944Q2|N|N|N|2431243|2431394|2430891|2431166|N|N|N|N|N| +2431258|AAAAAAAAKBJBFCAA|1944-06-16|533|2320|179|1944|5|6|16|2|1944|179|2320|Friday|1944Q2|N|Y|N|2431243|2431394|2430892|2431167|N|N|N|N|N| +2431259|AAAAAAAALBJBFCAA|1944-06-17|533|2320|179|1944|6|6|17|2|1944|179|2320|Saturday|1944Q2|N|Y|N|2431243|2431394|2430893|2431168|N|N|N|N|N| +2431260|AAAAAAAAMBJBFCAA|1944-06-18|533|2320|179|1944|0|6|18|2|1944|179|2320|Sunday|1944Q2|N|N|N|2431243|2431394|2430894|2431169|N|N|N|N|N| +2431261|AAAAAAAANBJBFCAA|1944-06-19|533|2320|179|1944|1|6|19|2|1944|179|2320|Monday|1944Q2|N|N|N|2431243|2431394|2430895|2431170|N|N|N|N|N| +2431262|AAAAAAAAOBJBFCAA|1944-06-20|533|2321|179|1944|2|6|20|2|1944|179|2321|Tuesday|1944Q2|N|N|N|2431243|2431394|2430896|2431171|N|N|N|N|N| +2431263|AAAAAAAAPBJBFCAA|1944-06-21|533|2321|179|1944|3|6|21|2|1944|179|2321|Wednesday|1944Q2|N|N|N|2431243|2431394|2430897|2431172|N|N|N|N|N| +2431264|AAAAAAAAACJBFCAA|1944-06-22|533|2321|179|1944|4|6|22|2|1944|179|2321|Thursday|1944Q2|N|N|N|2431243|2431394|2430898|2431173|N|N|N|N|N| +2431265|AAAAAAAABCJBFCAA|1944-06-23|533|2321|179|1944|5|6|23|2|1944|179|2321|Friday|1944Q2|N|Y|N|2431243|2431394|2430899|2431174|N|N|N|N|N| +2431266|AAAAAAAACCJBFCAA|1944-06-24|533|2321|179|1944|6|6|24|2|1944|179|2321|Saturday|1944Q2|N|Y|N|2431243|2431394|2430900|2431175|N|N|N|N|N| +2431267|AAAAAAAADCJBFCAA|1944-06-25|533|2321|179|1944|0|6|25|2|1944|179|2321|Sunday|1944Q2|N|N|N|2431243|2431394|2430901|2431176|N|N|N|N|N| +2431268|AAAAAAAAECJBFCAA|1944-06-26|533|2321|179|1944|1|6|26|2|1944|179|2321|Monday|1944Q2|N|N|N|2431243|2431394|2430902|2431177|N|N|N|N|N| +2431269|AAAAAAAAFCJBFCAA|1944-06-27|533|2322|179|1944|2|6|27|2|1944|179|2322|Tuesday|1944Q2|N|N|N|2431243|2431394|2430903|2431178|N|N|N|N|N| +2431270|AAAAAAAAGCJBFCAA|1944-06-28|533|2322|179|1944|3|6|28|2|1944|179|2322|Wednesday|1944Q2|N|N|N|2431243|2431394|2430904|2431179|N|N|N|N|N| +2431271|AAAAAAAAHCJBFCAA|1944-06-29|533|2322|179|1944|4|6|29|2|1944|179|2322|Thursday|1944Q2|N|N|N|2431243|2431394|2430905|2431180|N|N|N|N|N| +2431272|AAAAAAAAICJBFCAA|1944-06-30|533|2322|179|1944|5|6|30|2|1944|179|2322|Friday|1944Q2|N|Y|N|2431243|2431394|2430906|2431181|N|N|N|N|N| +2431273|AAAAAAAAJCJBFCAA|1944-07-01|534|2322|179|1944|6|7|1|3|1944|179|2322|Saturday|1944Q3|N|Y|N|2431273|2431454|2430907|2431182|N|N|N|N|N| +2431274|AAAAAAAAKCJBFCAA|1944-07-02|534|2322|179|1944|0|7|2|3|1944|179|2322|Sunday|1944Q3|N|N|N|2431273|2431454|2430908|2431183|N|N|N|N|N| +2431275|AAAAAAAALCJBFCAA|1944-07-03|534|2322|179|1944|1|7|3|3|1944|179|2322|Monday|1944Q3|N|N|N|2431273|2431454|2430909|2431184|N|N|N|N|N| +2431276|AAAAAAAAMCJBFCAA|1944-07-04|534|2323|179|1944|2|7|4|3|1944|179|2323|Tuesday|1944Q3|Y|N|N|2431273|2431454|2430910|2431185|N|N|N|N|N| +2431277|AAAAAAAANCJBFCAA|1944-07-05|534|2323|179|1944|3|7|5|3|1944|179|2323|Wednesday|1944Q3|N|N|Y|2431273|2431454|2430911|2431186|N|N|N|N|N| +2431278|AAAAAAAAOCJBFCAA|1944-07-06|534|2323|179|1944|4|7|6|3|1944|179|2323|Thursday|1944Q3|N|N|N|2431273|2431454|2430912|2431187|N|N|N|N|N| +2431279|AAAAAAAAPCJBFCAA|1944-07-07|534|2323|179|1944|5|7|7|3|1944|179|2323|Friday|1944Q3|N|Y|N|2431273|2431454|2430913|2431188|N|N|N|N|N| +2431280|AAAAAAAAADJBFCAA|1944-07-08|534|2323|179|1944|6|7|8|3|1944|179|2323|Saturday|1944Q3|N|Y|N|2431273|2431454|2430914|2431189|N|N|N|N|N| +2431281|AAAAAAAABDJBFCAA|1944-07-09|534|2323|179|1944|0|7|9|3|1944|179|2323|Sunday|1944Q3|N|N|N|2431273|2431454|2430915|2431190|N|N|N|N|N| +2431282|AAAAAAAACDJBFCAA|1944-07-10|534|2323|179|1944|1|7|10|3|1944|179|2323|Monday|1944Q3|N|N|N|2431273|2431454|2430916|2431191|N|N|N|N|N| +2431283|AAAAAAAADDJBFCAA|1944-07-11|534|2324|179|1944|2|7|11|3|1944|179|2324|Tuesday|1944Q3|N|N|N|2431273|2431454|2430917|2431192|N|N|N|N|N| +2431284|AAAAAAAAEDJBFCAA|1944-07-12|534|2324|179|1944|3|7|12|3|1944|179|2324|Wednesday|1944Q3|N|N|N|2431273|2431454|2430918|2431193|N|N|N|N|N| +2431285|AAAAAAAAFDJBFCAA|1944-07-13|534|2324|179|1944|4|7|13|3|1944|179|2324|Thursday|1944Q3|N|N|N|2431273|2431454|2430919|2431194|N|N|N|N|N| +2431286|AAAAAAAAGDJBFCAA|1944-07-14|534|2324|179|1944|5|7|14|3|1944|179|2324|Friday|1944Q3|N|Y|N|2431273|2431454|2430920|2431195|N|N|N|N|N| +2431287|AAAAAAAAHDJBFCAA|1944-07-15|534|2324|179|1944|6|7|15|3|1944|179|2324|Saturday|1944Q3|N|Y|N|2431273|2431454|2430921|2431196|N|N|N|N|N| +2431288|AAAAAAAAIDJBFCAA|1944-07-16|534|2324|179|1944|0|7|16|3|1944|179|2324|Sunday|1944Q3|N|N|N|2431273|2431454|2430922|2431197|N|N|N|N|N| +2431289|AAAAAAAAJDJBFCAA|1944-07-17|534|2324|179|1944|1|7|17|3|1944|179|2324|Monday|1944Q3|N|N|N|2431273|2431454|2430923|2431198|N|N|N|N|N| +2431290|AAAAAAAAKDJBFCAA|1944-07-18|534|2325|179|1944|2|7|18|3|1944|179|2325|Tuesday|1944Q3|N|N|N|2431273|2431454|2430924|2431199|N|N|N|N|N| +2431291|AAAAAAAALDJBFCAA|1944-07-19|534|2325|179|1944|3|7|19|3|1944|179|2325|Wednesday|1944Q3|N|N|N|2431273|2431454|2430925|2431200|N|N|N|N|N| +2431292|AAAAAAAAMDJBFCAA|1944-07-20|534|2325|179|1944|4|7|20|3|1944|179|2325|Thursday|1944Q3|N|N|N|2431273|2431454|2430926|2431201|N|N|N|N|N| +2431293|AAAAAAAANDJBFCAA|1944-07-21|534|2325|179|1944|5|7|21|3|1944|179|2325|Friday|1944Q3|N|Y|N|2431273|2431454|2430927|2431202|N|N|N|N|N| +2431294|AAAAAAAAODJBFCAA|1944-07-22|534|2325|179|1944|6|7|22|3|1944|179|2325|Saturday|1944Q3|N|Y|N|2431273|2431454|2430928|2431203|N|N|N|N|N| +2431295|AAAAAAAAPDJBFCAA|1944-07-23|534|2325|179|1944|0|7|23|3|1944|179|2325|Sunday|1944Q3|N|N|N|2431273|2431454|2430929|2431204|N|N|N|N|N| +2431296|AAAAAAAAAEJBFCAA|1944-07-24|534|2325|179|1944|1|7|24|3|1944|179|2325|Monday|1944Q3|N|N|N|2431273|2431454|2430930|2431205|N|N|N|N|N| +2431297|AAAAAAAABEJBFCAA|1944-07-25|534|2326|179|1944|2|7|25|3|1944|179|2326|Tuesday|1944Q3|N|N|N|2431273|2431454|2430931|2431206|N|N|N|N|N| +2431298|AAAAAAAACEJBFCAA|1944-07-26|534|2326|179|1944|3|7|26|3|1944|179|2326|Wednesday|1944Q3|N|N|N|2431273|2431454|2430932|2431207|N|N|N|N|N| +2431299|AAAAAAAADEJBFCAA|1944-07-27|534|2326|179|1944|4|7|27|3|1944|179|2326|Thursday|1944Q3|N|N|N|2431273|2431454|2430933|2431208|N|N|N|N|N| +2431300|AAAAAAAAEEJBFCAA|1944-07-28|534|2326|179|1944|5|7|28|3|1944|179|2326|Friday|1944Q3|N|Y|N|2431273|2431454|2430934|2431209|N|N|N|N|N| +2431301|AAAAAAAAFEJBFCAA|1944-07-29|534|2326|179|1944|6|7|29|3|1944|179|2326|Saturday|1944Q3|N|Y|N|2431273|2431454|2430935|2431210|N|N|N|N|N| +2431302|AAAAAAAAGEJBFCAA|1944-07-30|534|2326|179|1944|0|7|30|3|1944|179|2326|Sunday|1944Q3|N|N|N|2431273|2431454|2430936|2431211|N|N|N|N|N| +2431303|AAAAAAAAHEJBFCAA|1944-07-31|534|2326|179|1944|1|7|31|3|1944|179|2326|Monday|1944Q3|N|N|N|2431273|2431454|2430937|2431212|N|N|N|N|N| +2431304|AAAAAAAAIEJBFCAA|1944-08-01|535|2327|179|1944|2|8|1|3|1944|179|2327|Tuesday|1944Q3|N|N|N|2431304|2431516|2430938|2431213|N|N|N|N|N| +2431305|AAAAAAAAJEJBFCAA|1944-08-02|535|2327|179|1944|3|8|2|3|1944|179|2327|Wednesday|1944Q3|N|N|N|2431304|2431516|2430939|2431214|N|N|N|N|N| +2431306|AAAAAAAAKEJBFCAA|1944-08-03|535|2327|179|1944|4|8|3|3|1944|179|2327|Thursday|1944Q3|N|N|N|2431304|2431516|2430940|2431215|N|N|N|N|N| +2431307|AAAAAAAALEJBFCAA|1944-08-04|535|2327|179|1944|5|8|4|3|1944|179|2327|Friday|1944Q3|N|Y|N|2431304|2431516|2430941|2431216|N|N|N|N|N| +2431308|AAAAAAAAMEJBFCAA|1944-08-05|535|2327|179|1944|6|8|5|3|1944|179|2327|Saturday|1944Q3|N|Y|N|2431304|2431516|2430942|2431217|N|N|N|N|N| +2431309|AAAAAAAANEJBFCAA|1944-08-06|535|2327|179|1944|0|8|6|3|1944|179|2327|Sunday|1944Q3|N|N|N|2431304|2431516|2430943|2431218|N|N|N|N|N| +2431310|AAAAAAAAOEJBFCAA|1944-08-07|535|2327|179|1944|1|8|7|3|1944|179|2327|Monday|1944Q3|N|N|N|2431304|2431516|2430944|2431219|N|N|N|N|N| +2431311|AAAAAAAAPEJBFCAA|1944-08-08|535|2328|179|1944|2|8|8|3|1944|179|2328|Tuesday|1944Q3|N|N|N|2431304|2431516|2430945|2431220|N|N|N|N|N| +2431312|AAAAAAAAAFJBFCAA|1944-08-09|535|2328|179|1944|3|8|9|3|1944|179|2328|Wednesday|1944Q3|N|N|N|2431304|2431516|2430946|2431221|N|N|N|N|N| +2431313|AAAAAAAABFJBFCAA|1944-08-10|535|2328|179|1944|4|8|10|3|1944|179|2328|Thursday|1944Q3|N|N|N|2431304|2431516|2430947|2431222|N|N|N|N|N| +2431314|AAAAAAAACFJBFCAA|1944-08-11|535|2328|179|1944|5|8|11|3|1944|179|2328|Friday|1944Q3|N|Y|N|2431304|2431516|2430948|2431223|N|N|N|N|N| +2431315|AAAAAAAADFJBFCAA|1944-08-12|535|2328|179|1944|6|8|12|3|1944|179|2328|Saturday|1944Q3|N|Y|N|2431304|2431516|2430949|2431224|N|N|N|N|N| +2431316|AAAAAAAAEFJBFCAA|1944-08-13|535|2328|179|1944|0|8|13|3|1944|179|2328|Sunday|1944Q3|N|N|N|2431304|2431516|2430950|2431225|N|N|N|N|N| +2431317|AAAAAAAAFFJBFCAA|1944-08-14|535|2328|179|1944|1|8|14|3|1944|179|2328|Monday|1944Q3|N|N|N|2431304|2431516|2430951|2431226|N|N|N|N|N| +2431318|AAAAAAAAGFJBFCAA|1944-08-15|535|2329|179|1944|2|8|15|3|1944|179|2329|Tuesday|1944Q3|N|N|N|2431304|2431516|2430952|2431227|N|N|N|N|N| +2431319|AAAAAAAAHFJBFCAA|1944-08-16|535|2329|179|1944|3|8|16|3|1944|179|2329|Wednesday|1944Q3|N|N|N|2431304|2431516|2430953|2431228|N|N|N|N|N| +2431320|AAAAAAAAIFJBFCAA|1944-08-17|535|2329|179|1944|4|8|17|3|1944|179|2329|Thursday|1944Q3|N|N|N|2431304|2431516|2430954|2431229|N|N|N|N|N| +2431321|AAAAAAAAJFJBFCAA|1944-08-18|535|2329|179|1944|5|8|18|3|1944|179|2329|Friday|1944Q3|N|Y|N|2431304|2431516|2430955|2431230|N|N|N|N|N| +2431322|AAAAAAAAKFJBFCAA|1944-08-19|535|2329|179|1944|6|8|19|3|1944|179|2329|Saturday|1944Q3|N|Y|N|2431304|2431516|2430956|2431231|N|N|N|N|N| +2431323|AAAAAAAALFJBFCAA|1944-08-20|535|2329|179|1944|0|8|20|3|1944|179|2329|Sunday|1944Q3|N|N|N|2431304|2431516|2430957|2431232|N|N|N|N|N| +2431324|AAAAAAAAMFJBFCAA|1944-08-21|535|2329|179|1944|1|8|21|3|1944|179|2329|Monday|1944Q3|N|N|N|2431304|2431516|2430958|2431233|N|N|N|N|N| +2431325|AAAAAAAANFJBFCAA|1944-08-22|535|2330|179|1944|2|8|22|3|1944|179|2330|Tuesday|1944Q3|N|N|N|2431304|2431516|2430959|2431234|N|N|N|N|N| +2431326|AAAAAAAAOFJBFCAA|1944-08-23|535|2330|179|1944|3|8|23|3|1944|179|2330|Wednesday|1944Q3|N|N|N|2431304|2431516|2430960|2431235|N|N|N|N|N| +2431327|AAAAAAAAPFJBFCAA|1944-08-24|535|2330|179|1944|4|8|24|3|1944|179|2330|Thursday|1944Q3|N|N|N|2431304|2431516|2430961|2431236|N|N|N|N|N| +2431328|AAAAAAAAAGJBFCAA|1944-08-25|535|2330|179|1944|5|8|25|3|1944|179|2330|Friday|1944Q3|N|Y|N|2431304|2431516|2430962|2431237|N|N|N|N|N| +2431329|AAAAAAAABGJBFCAA|1944-08-26|535|2330|179|1944|6|8|26|3|1944|179|2330|Saturday|1944Q3|N|Y|N|2431304|2431516|2430963|2431238|N|N|N|N|N| +2431330|AAAAAAAACGJBFCAA|1944-08-27|535|2330|179|1944|0|8|27|3|1944|179|2330|Sunday|1944Q3|N|N|N|2431304|2431516|2430964|2431239|N|N|N|N|N| +2431331|AAAAAAAADGJBFCAA|1944-08-28|535|2330|179|1944|1|8|28|3|1944|179|2330|Monday|1944Q3|N|N|N|2431304|2431516|2430965|2431240|N|N|N|N|N| +2431332|AAAAAAAAEGJBFCAA|1944-08-29|535|2331|179|1944|2|8|29|3|1944|179|2331|Tuesday|1944Q3|N|N|N|2431304|2431516|2430966|2431241|N|N|N|N|N| +2431333|AAAAAAAAFGJBFCAA|1944-08-30|535|2331|179|1944|3|8|30|3|1944|179|2331|Wednesday|1944Q3|N|N|N|2431304|2431516|2430967|2431242|N|N|N|N|N| +2431334|AAAAAAAAGGJBFCAA|1944-08-31|535|2331|179|1944|4|8|31|3|1944|179|2331|Thursday|1944Q3|N|N|N|2431304|2431516|2430968|2431243|N|N|N|N|N| +2431335|AAAAAAAAHGJBFCAA|1944-09-01|536|2331|180|1944|5|9|1|3|1944|180|2331|Friday|1944Q3|N|Y|N|2431335|2431578|2430969|2431244|N|N|N|N|N| +2431336|AAAAAAAAIGJBFCAA|1944-09-02|536|2331|180|1944|6|9|2|3|1944|180|2331|Saturday|1944Q3|N|Y|N|2431335|2431578|2430970|2431245|N|N|N|N|N| +2431337|AAAAAAAAJGJBFCAA|1944-09-03|536|2331|180|1944|0|9|3|3|1944|180|2331|Sunday|1944Q3|N|N|N|2431335|2431578|2430971|2431246|N|N|N|N|N| +2431338|AAAAAAAAKGJBFCAA|1944-09-04|536|2331|180|1944|1|9|4|3|1944|180|2331|Monday|1944Q3|N|N|N|2431335|2431578|2430972|2431247|N|N|N|N|N| +2431339|AAAAAAAALGJBFCAA|1944-09-05|536|2332|180|1944|2|9|5|3|1944|180|2332|Tuesday|1944Q3|N|N|N|2431335|2431578|2430973|2431248|N|N|N|N|N| +2431340|AAAAAAAAMGJBFCAA|1944-09-06|536|2332|180|1944|3|9|6|3|1944|180|2332|Wednesday|1944Q3|N|N|N|2431335|2431578|2430974|2431249|N|N|N|N|N| +2431341|AAAAAAAANGJBFCAA|1944-09-07|536|2332|180|1944|4|9|7|3|1944|180|2332|Thursday|1944Q3|N|N|N|2431335|2431578|2430975|2431250|N|N|N|N|N| +2431342|AAAAAAAAOGJBFCAA|1944-09-08|536|2332|180|1944|5|9|8|3|1944|180|2332|Friday|1944Q3|N|Y|N|2431335|2431578|2430976|2431251|N|N|N|N|N| +2431343|AAAAAAAAPGJBFCAA|1944-09-09|536|2332|180|1944|6|9|9|3|1944|180|2332|Saturday|1944Q3|N|Y|N|2431335|2431578|2430977|2431252|N|N|N|N|N| +2431344|AAAAAAAAAHJBFCAA|1944-09-10|536|2332|180|1944|0|9|10|3|1944|180|2332|Sunday|1944Q3|N|N|N|2431335|2431578|2430978|2431253|N|N|N|N|N| +2431345|AAAAAAAABHJBFCAA|1944-09-11|536|2332|180|1944|1|9|11|3|1944|180|2332|Monday|1944Q3|N|N|N|2431335|2431578|2430979|2431254|N|N|N|N|N| +2431346|AAAAAAAACHJBFCAA|1944-09-12|536|2333|180|1944|2|9|12|3|1944|180|2333|Tuesday|1944Q3|N|N|N|2431335|2431578|2430980|2431255|N|N|N|N|N| +2431347|AAAAAAAADHJBFCAA|1944-09-13|536|2333|180|1944|3|9|13|3|1944|180|2333|Wednesday|1944Q3|N|N|N|2431335|2431578|2430981|2431256|N|N|N|N|N| +2431348|AAAAAAAAEHJBFCAA|1944-09-14|536|2333|180|1944|4|9|14|3|1944|180|2333|Thursday|1944Q3|N|N|N|2431335|2431578|2430982|2431257|N|N|N|N|N| +2431349|AAAAAAAAFHJBFCAA|1944-09-15|536|2333|180|1944|5|9|15|3|1944|180|2333|Friday|1944Q3|N|Y|N|2431335|2431578|2430983|2431258|N|N|N|N|N| +2431350|AAAAAAAAGHJBFCAA|1944-09-16|536|2333|180|1944|6|9|16|3|1944|180|2333|Saturday|1944Q3|N|Y|N|2431335|2431578|2430984|2431259|N|N|N|N|N| +2431351|AAAAAAAAHHJBFCAA|1944-09-17|536|2333|180|1944|0|9|17|3|1944|180|2333|Sunday|1944Q3|N|N|N|2431335|2431578|2430985|2431260|N|N|N|N|N| +2431352|AAAAAAAAIHJBFCAA|1944-09-18|536|2333|180|1944|1|9|18|3|1944|180|2333|Monday|1944Q3|N|N|N|2431335|2431578|2430986|2431261|N|N|N|N|N| +2431353|AAAAAAAAJHJBFCAA|1944-09-19|536|2334|180|1944|2|9|19|3|1944|180|2334|Tuesday|1944Q3|N|N|N|2431335|2431578|2430987|2431262|N|N|N|N|N| +2431354|AAAAAAAAKHJBFCAA|1944-09-20|536|2334|180|1944|3|9|20|3|1944|180|2334|Wednesday|1944Q3|N|N|N|2431335|2431578|2430988|2431263|N|N|N|N|N| +2431355|AAAAAAAALHJBFCAA|1944-09-21|536|2334|180|1944|4|9|21|3|1944|180|2334|Thursday|1944Q3|N|N|N|2431335|2431578|2430989|2431264|N|N|N|N|N| +2431356|AAAAAAAAMHJBFCAA|1944-09-22|536|2334|180|1944|5|9|22|3|1944|180|2334|Friday|1944Q3|N|Y|N|2431335|2431578|2430990|2431265|N|N|N|N|N| +2431357|AAAAAAAANHJBFCAA|1944-09-23|536|2334|180|1944|6|9|23|3|1944|180|2334|Saturday|1944Q3|N|Y|N|2431335|2431578|2430991|2431266|N|N|N|N|N| +2431358|AAAAAAAAOHJBFCAA|1944-09-24|536|2334|180|1944|0|9|24|3|1944|180|2334|Sunday|1944Q3|N|N|N|2431335|2431578|2430992|2431267|N|N|N|N|N| +2431359|AAAAAAAAPHJBFCAA|1944-09-25|536|2334|180|1944|1|9|25|3|1944|180|2334|Monday|1944Q3|N|N|N|2431335|2431578|2430993|2431268|N|N|N|N|N| +2431360|AAAAAAAAAIJBFCAA|1944-09-26|536|2335|180|1944|2|9|26|3|1944|180|2335|Tuesday|1944Q3|N|N|N|2431335|2431578|2430994|2431269|N|N|N|N|N| +2431361|AAAAAAAABIJBFCAA|1944-09-27|536|2335|180|1944|3|9|27|3|1944|180|2335|Wednesday|1944Q3|N|N|N|2431335|2431578|2430995|2431270|N|N|N|N|N| +2431362|AAAAAAAACIJBFCAA|1944-09-28|536|2335|180|1944|4|9|28|3|1944|180|2335|Thursday|1944Q3|N|N|N|2431335|2431578|2430996|2431271|N|N|N|N|N| +2431363|AAAAAAAADIJBFCAA|1944-09-29|536|2335|180|1944|5|9|29|3|1944|180|2335|Friday|1944Q3|N|Y|N|2431335|2431578|2430997|2431272|N|N|N|N|N| +2431364|AAAAAAAAEIJBFCAA|1944-09-30|536|2335|180|1944|6|9|30|3|1944|180|2335|Saturday|1944Q3|N|Y|N|2431335|2431578|2430998|2431273|N|N|N|N|N| +2431365|AAAAAAAAFIJBFCAA|1944-10-01|537|2335|180|1944|0|10|1|4|1944|180|2335|Sunday|1944Q4|N|N|N|2431365|2431638|2430999|2431273|N|N|N|N|N| +2431366|AAAAAAAAGIJBFCAA|1944-10-02|537|2335|180|1944|1|10|2|4|1944|180|2335|Monday|1944Q4|N|N|N|2431365|2431638|2431000|2431274|N|N|N|N|N| +2431367|AAAAAAAAHIJBFCAA|1944-10-03|537|2336|180|1944|2|10|3|4|1944|180|2336|Tuesday|1944Q4|N|N|N|2431365|2431638|2431001|2431275|N|N|N|N|N| +2431368|AAAAAAAAIIJBFCAA|1944-10-04|537|2336|180|1944|3|10|4|4|1944|180|2336|Wednesday|1944Q4|N|N|N|2431365|2431638|2431002|2431276|N|N|N|N|N| +2431369|AAAAAAAAJIJBFCAA|1944-10-05|537|2336|180|1944|4|10|5|4|1944|180|2336|Thursday|1944Q4|N|N|N|2431365|2431638|2431003|2431277|N|N|N|N|N| +2431370|AAAAAAAAKIJBFCAA|1944-10-06|537|2336|180|1944|5|10|6|4|1944|180|2336|Friday|1944Q4|N|Y|N|2431365|2431638|2431004|2431278|N|N|N|N|N| +2431371|AAAAAAAALIJBFCAA|1944-10-07|537|2336|180|1944|6|10|7|4|1944|180|2336|Saturday|1944Q4|N|Y|N|2431365|2431638|2431005|2431279|N|N|N|N|N| +2431372|AAAAAAAAMIJBFCAA|1944-10-08|537|2336|180|1944|0|10|8|4|1944|180|2336|Sunday|1944Q4|N|N|N|2431365|2431638|2431006|2431280|N|N|N|N|N| +2431373|AAAAAAAANIJBFCAA|1944-10-09|537|2336|180|1944|1|10|9|4|1944|180|2336|Monday|1944Q4|N|N|N|2431365|2431638|2431007|2431281|N|N|N|N|N| +2431374|AAAAAAAAOIJBFCAA|1944-10-10|537|2337|180|1944|2|10|10|4|1944|180|2337|Tuesday|1944Q4|N|N|N|2431365|2431638|2431008|2431282|N|N|N|N|N| +2431375|AAAAAAAAPIJBFCAA|1944-10-11|537|2337|180|1944|3|10|11|4|1944|180|2337|Wednesday|1944Q4|N|N|N|2431365|2431638|2431009|2431283|N|N|N|N|N| +2431376|AAAAAAAAAJJBFCAA|1944-10-12|537|2337|180|1944|4|10|12|4|1944|180|2337|Thursday|1944Q4|N|N|N|2431365|2431638|2431010|2431284|N|N|N|N|N| +2431377|AAAAAAAABJJBFCAA|1944-10-13|537|2337|180|1944|5|10|13|4|1944|180|2337|Friday|1944Q4|N|Y|N|2431365|2431638|2431011|2431285|N|N|N|N|N| +2431378|AAAAAAAACJJBFCAA|1944-10-14|537|2337|180|1944|6|10|14|4|1944|180|2337|Saturday|1944Q4|N|Y|N|2431365|2431638|2431012|2431286|N|N|N|N|N| +2431379|AAAAAAAADJJBFCAA|1944-10-15|537|2337|180|1944|0|10|15|4|1944|180|2337|Sunday|1944Q4|N|N|N|2431365|2431638|2431013|2431287|N|N|N|N|N| +2431380|AAAAAAAAEJJBFCAA|1944-10-16|537|2337|180|1944|1|10|16|4|1944|180|2337|Monday|1944Q4|N|N|N|2431365|2431638|2431014|2431288|N|N|N|N|N| +2431381|AAAAAAAAFJJBFCAA|1944-10-17|537|2338|180|1944|2|10|17|4|1944|180|2338|Tuesday|1944Q4|N|N|N|2431365|2431638|2431015|2431289|N|N|N|N|N| +2431382|AAAAAAAAGJJBFCAA|1944-10-18|537|2338|180|1944|3|10|18|4|1944|180|2338|Wednesday|1944Q4|N|N|N|2431365|2431638|2431016|2431290|N|N|N|N|N| +2431383|AAAAAAAAHJJBFCAA|1944-10-19|537|2338|180|1944|4|10|19|4|1944|180|2338|Thursday|1944Q4|N|N|N|2431365|2431638|2431017|2431291|N|N|N|N|N| +2431384|AAAAAAAAIJJBFCAA|1944-10-20|537|2338|180|1944|5|10|20|4|1944|180|2338|Friday|1944Q4|N|Y|N|2431365|2431638|2431018|2431292|N|N|N|N|N| +2431385|AAAAAAAAJJJBFCAA|1944-10-21|537|2338|180|1944|6|10|21|4|1944|180|2338|Saturday|1944Q4|N|Y|N|2431365|2431638|2431019|2431293|N|N|N|N|N| +2431386|AAAAAAAAKJJBFCAA|1944-10-22|537|2338|180|1944|0|10|22|4|1944|180|2338|Sunday|1944Q4|N|N|N|2431365|2431638|2431020|2431294|N|N|N|N|N| +2431387|AAAAAAAALJJBFCAA|1944-10-23|537|2338|180|1944|1|10|23|4|1944|180|2338|Monday|1944Q4|N|N|N|2431365|2431638|2431021|2431295|N|N|N|N|N| +2431388|AAAAAAAAMJJBFCAA|1944-10-24|537|2339|180|1944|2|10|24|4|1944|180|2339|Tuesday|1944Q4|N|N|N|2431365|2431638|2431022|2431296|N|N|N|N|N| +2431389|AAAAAAAANJJBFCAA|1944-10-25|537|2339|180|1944|3|10|25|4|1944|180|2339|Wednesday|1944Q4|N|N|N|2431365|2431638|2431023|2431297|N|N|N|N|N| +2431390|AAAAAAAAOJJBFCAA|1944-10-26|537|2339|180|1944|4|10|26|4|1944|180|2339|Thursday|1944Q4|N|N|N|2431365|2431638|2431024|2431298|N|N|N|N|N| +2431391|AAAAAAAAPJJBFCAA|1944-10-27|537|2339|180|1944|5|10|27|4|1944|180|2339|Friday|1944Q4|N|Y|N|2431365|2431638|2431025|2431299|N|N|N|N|N| +2431392|AAAAAAAAAKJBFCAA|1944-10-28|537|2339|180|1944|6|10|28|4|1944|180|2339|Saturday|1944Q4|N|Y|N|2431365|2431638|2431026|2431300|N|N|N|N|N| +2431393|AAAAAAAABKJBFCAA|1944-10-29|537|2339|180|1944|0|10|29|4|1944|180|2339|Sunday|1944Q4|N|N|N|2431365|2431638|2431027|2431301|N|N|N|N|N| +2431394|AAAAAAAACKJBFCAA|1944-10-30|537|2339|180|1944|1|10|30|4|1944|180|2339|Monday|1944Q4|N|N|N|2431365|2431638|2431028|2431302|N|N|N|N|N| +2431395|AAAAAAAADKJBFCAA|1944-10-31|537|2340|180|1944|2|10|31|4|1944|180|2340|Tuesday|1944Q4|N|N|N|2431365|2431638|2431029|2431303|N|N|N|N|N| +2431396|AAAAAAAAEKJBFCAA|1944-11-01|538|2340|180|1944|3|11|1|4|1944|180|2340|Wednesday|1944Q4|N|N|N|2431396|2431700|2431030|2431304|N|N|N|N|N| +2431397|AAAAAAAAFKJBFCAA|1944-11-02|538|2340|180|1944|4|11|2|4|1944|180|2340|Thursday|1944Q4|N|N|N|2431396|2431700|2431031|2431305|N|N|N|N|N| +2431398|AAAAAAAAGKJBFCAA|1944-11-03|538|2340|180|1944|5|11|3|4|1944|180|2340|Friday|1944Q4|N|Y|N|2431396|2431700|2431032|2431306|N|N|N|N|N| +2431399|AAAAAAAAHKJBFCAA|1944-11-04|538|2340|180|1944|6|11|4|4|1944|180|2340|Saturday|1944Q4|N|Y|N|2431396|2431700|2431033|2431307|N|N|N|N|N| +2431400|AAAAAAAAIKJBFCAA|1944-11-05|538|2340|180|1944|0|11|5|4|1944|180|2340|Sunday|1944Q4|N|N|N|2431396|2431700|2431034|2431308|N|N|N|N|N| +2431401|AAAAAAAAJKJBFCAA|1944-11-06|538|2340|180|1944|1|11|6|4|1944|180|2340|Monday|1944Q4|N|N|N|2431396|2431700|2431035|2431309|N|N|N|N|N| +2431402|AAAAAAAAKKJBFCAA|1944-11-07|538|2341|180|1944|2|11|7|4|1944|180|2341|Tuesday|1944Q4|N|N|N|2431396|2431700|2431036|2431310|N|N|N|N|N| +2431403|AAAAAAAALKJBFCAA|1944-11-08|538|2341|180|1944|3|11|8|4|1944|180|2341|Wednesday|1944Q4|N|N|N|2431396|2431700|2431037|2431311|N|N|N|N|N| +2431404|AAAAAAAAMKJBFCAA|1944-11-09|538|2341|180|1944|4|11|9|4|1944|180|2341|Thursday|1944Q4|N|N|N|2431396|2431700|2431038|2431312|N|N|N|N|N| +2431405|AAAAAAAANKJBFCAA|1944-11-10|538|2341|180|1944|5|11|10|4|1944|180|2341|Friday|1944Q4|N|Y|N|2431396|2431700|2431039|2431313|N|N|N|N|N| +2431406|AAAAAAAAOKJBFCAA|1944-11-11|538|2341|180|1944|6|11|11|4|1944|180|2341|Saturday|1944Q4|N|Y|N|2431396|2431700|2431040|2431314|N|N|N|N|N| +2431407|AAAAAAAAPKJBFCAA|1944-11-12|538|2341|180|1944|0|11|12|4|1944|180|2341|Sunday|1944Q4|N|N|N|2431396|2431700|2431041|2431315|N|N|N|N|N| +2431408|AAAAAAAAALJBFCAA|1944-11-13|538|2341|180|1944|1|11|13|4|1944|180|2341|Monday|1944Q4|N|N|N|2431396|2431700|2431042|2431316|N|N|N|N|N| +2431409|AAAAAAAABLJBFCAA|1944-11-14|538|2342|180|1944|2|11|14|4|1944|180|2342|Tuesday|1944Q4|N|N|N|2431396|2431700|2431043|2431317|N|N|N|N|N| +2431410|AAAAAAAACLJBFCAA|1944-11-15|538|2342|180|1944|3|11|15|4|1944|180|2342|Wednesday|1944Q4|N|N|N|2431396|2431700|2431044|2431318|N|N|N|N|N| +2431411|AAAAAAAADLJBFCAA|1944-11-16|538|2342|180|1944|4|11|16|4|1944|180|2342|Thursday|1944Q4|N|N|N|2431396|2431700|2431045|2431319|N|N|N|N|N| +2431412|AAAAAAAAELJBFCAA|1944-11-17|538|2342|180|1944|5|11|17|4|1944|180|2342|Friday|1944Q4|N|Y|N|2431396|2431700|2431046|2431320|N|N|N|N|N| +2431413|AAAAAAAAFLJBFCAA|1944-11-18|538|2342|180|1944|6|11|18|4|1944|180|2342|Saturday|1944Q4|N|Y|N|2431396|2431700|2431047|2431321|N|N|N|N|N| +2431414|AAAAAAAAGLJBFCAA|1944-11-19|538|2342|180|1944|0|11|19|4|1944|180|2342|Sunday|1944Q4|N|N|N|2431396|2431700|2431048|2431322|N|N|N|N|N| +2431415|AAAAAAAAHLJBFCAA|1944-11-20|538|2342|180|1944|1|11|20|4|1944|180|2342|Monday|1944Q4|N|N|N|2431396|2431700|2431049|2431323|N|N|N|N|N| +2431416|AAAAAAAAILJBFCAA|1944-11-21|538|2343|180|1944|2|11|21|4|1944|180|2343|Tuesday|1944Q4|N|N|N|2431396|2431700|2431050|2431324|N|N|N|N|N| +2431417|AAAAAAAAJLJBFCAA|1944-11-22|538|2343|180|1944|3|11|22|4|1944|180|2343|Wednesday|1944Q4|N|N|N|2431396|2431700|2431051|2431325|N|N|N|N|N| +2431418|AAAAAAAAKLJBFCAA|1944-11-23|538|2343|180|1944|4|11|23|4|1944|180|2343|Thursday|1944Q4|N|N|N|2431396|2431700|2431052|2431326|N|N|N|N|N| +2431419|AAAAAAAALLJBFCAA|1944-11-24|538|2343|180|1944|5|11|24|4|1944|180|2343|Friday|1944Q4|N|Y|N|2431396|2431700|2431053|2431327|N|N|N|N|N| +2431420|AAAAAAAAMLJBFCAA|1944-11-25|538|2343|180|1944|6|11|25|4|1944|180|2343|Saturday|1944Q4|N|Y|N|2431396|2431700|2431054|2431328|N|N|N|N|N| +2431421|AAAAAAAANLJBFCAA|1944-11-26|538|2343|180|1944|0|11|26|4|1944|180|2343|Sunday|1944Q4|N|N|N|2431396|2431700|2431055|2431329|N|N|N|N|N| +2431422|AAAAAAAAOLJBFCAA|1944-11-27|538|2343|180|1944|1|11|27|4|1944|180|2343|Monday|1944Q4|N|N|N|2431396|2431700|2431056|2431330|N|N|N|N|N| +2431423|AAAAAAAAPLJBFCAA|1944-11-28|538|2344|180|1944|2|11|28|4|1944|180|2344|Tuesday|1944Q4|N|N|N|2431396|2431700|2431057|2431331|N|N|N|N|N| +2431424|AAAAAAAAAMJBFCAA|1944-11-29|538|2344|180|1944|3|11|29|4|1944|180|2344|Wednesday|1944Q4|N|N|N|2431396|2431700|2431058|2431332|N|N|N|N|N| +2431425|AAAAAAAABMJBFCAA|1944-11-30|538|2344|180|1944|4|11|30|4|1944|180|2344|Thursday|1944Q4|N|N|N|2431396|2431700|2431059|2431333|N|N|N|N|N| +2431426|AAAAAAAACMJBFCAA|1944-12-01|539|2344|181|1944|5|12|1|4|1944|181|2344|Friday|1944Q4|N|Y|N|2431426|2431760|2431060|2431334|N|N|N|N|N| +2431427|AAAAAAAADMJBFCAA|1944-12-02|539|2344|181|1944|6|12|2|4|1944|181|2344|Saturday|1944Q4|N|Y|N|2431426|2431760|2431061|2431335|N|N|N|N|N| +2431428|AAAAAAAAEMJBFCAA|1944-12-03|539|2344|181|1944|0|12|3|4|1944|181|2344|Sunday|1944Q4|N|N|N|2431426|2431760|2431062|2431336|N|N|N|N|N| +2431429|AAAAAAAAFMJBFCAA|1944-12-04|539|2344|181|1944|1|12|4|4|1944|181|2344|Monday|1944Q4|N|N|N|2431426|2431760|2431063|2431337|N|N|N|N|N| +2431430|AAAAAAAAGMJBFCAA|1944-12-05|539|2345|181|1944|2|12|5|4|1944|181|2345|Tuesday|1944Q4|N|N|N|2431426|2431760|2431064|2431338|N|N|N|N|N| +2431431|AAAAAAAAHMJBFCAA|1944-12-06|539|2345|181|1944|3|12|6|4|1944|181|2345|Wednesday|1944Q4|N|N|N|2431426|2431760|2431065|2431339|N|N|N|N|N| +2431432|AAAAAAAAIMJBFCAA|1944-12-07|539|2345|181|1944|4|12|7|4|1944|181|2345|Thursday|1944Q4|N|N|N|2431426|2431760|2431066|2431340|N|N|N|N|N| +2431433|AAAAAAAAJMJBFCAA|1944-12-08|539|2345|181|1944|5|12|8|4|1944|181|2345|Friday|1944Q4|N|Y|N|2431426|2431760|2431067|2431341|N|N|N|N|N| +2431434|AAAAAAAAKMJBFCAA|1944-12-09|539|2345|181|1944|6|12|9|4|1944|181|2345|Saturday|1944Q4|N|Y|N|2431426|2431760|2431068|2431342|N|N|N|N|N| +2431435|AAAAAAAALMJBFCAA|1944-12-10|539|2345|181|1944|0|12|10|4|1944|181|2345|Sunday|1944Q4|N|N|N|2431426|2431760|2431069|2431343|N|N|N|N|N| +2431436|AAAAAAAAMMJBFCAA|1944-12-11|539|2345|181|1944|1|12|11|4|1944|181|2345|Monday|1944Q4|N|N|N|2431426|2431760|2431070|2431344|N|N|N|N|N| +2431437|AAAAAAAANMJBFCAA|1944-12-12|539|2346|181|1944|2|12|12|4|1944|181|2346|Tuesday|1944Q4|N|N|N|2431426|2431760|2431071|2431345|N|N|N|N|N| +2431438|AAAAAAAAOMJBFCAA|1944-12-13|539|2346|181|1944|3|12|13|4|1944|181|2346|Wednesday|1944Q4|N|N|N|2431426|2431760|2431072|2431346|N|N|N|N|N| +2431439|AAAAAAAAPMJBFCAA|1944-12-14|539|2346|181|1944|4|12|14|4|1944|181|2346|Thursday|1944Q4|N|N|N|2431426|2431760|2431073|2431347|N|N|N|N|N| +2431440|AAAAAAAAANJBFCAA|1944-12-15|539|2346|181|1944|5|12|15|4|1944|181|2346|Friday|1944Q4|N|Y|N|2431426|2431760|2431074|2431348|N|N|N|N|N| +2431441|AAAAAAAABNJBFCAA|1944-12-16|539|2346|181|1944|6|12|16|4|1944|181|2346|Saturday|1944Q4|N|Y|N|2431426|2431760|2431075|2431349|N|N|N|N|N| +2431442|AAAAAAAACNJBFCAA|1944-12-17|539|2346|181|1944|0|12|17|4|1944|181|2346|Sunday|1944Q4|N|N|N|2431426|2431760|2431076|2431350|N|N|N|N|N| +2431443|AAAAAAAADNJBFCAA|1944-12-18|539|2346|181|1944|1|12|18|4|1944|181|2346|Monday|1944Q4|N|N|N|2431426|2431760|2431077|2431351|N|N|N|N|N| +2431444|AAAAAAAAENJBFCAA|1944-12-19|539|2347|181|1944|2|12|19|4|1944|181|2347|Tuesday|1944Q4|N|N|N|2431426|2431760|2431078|2431352|N|N|N|N|N| +2431445|AAAAAAAAFNJBFCAA|1944-12-20|539|2347|181|1944|3|12|20|4|1944|181|2347|Wednesday|1944Q4|N|N|N|2431426|2431760|2431079|2431353|N|N|N|N|N| +2431446|AAAAAAAAGNJBFCAA|1944-12-21|539|2347|181|1944|4|12|21|4|1944|181|2347|Thursday|1944Q4|N|N|N|2431426|2431760|2431080|2431354|N|N|N|N|N| +2431447|AAAAAAAAHNJBFCAA|1944-12-22|539|2347|181|1944|5|12|22|4|1944|181|2347|Friday|1944Q4|N|Y|N|2431426|2431760|2431081|2431355|N|N|N|N|N| +2431448|AAAAAAAAINJBFCAA|1944-12-23|539|2347|181|1944|6|12|23|4|1944|181|2347|Saturday|1944Q4|N|Y|N|2431426|2431760|2431082|2431356|N|N|N|N|N| +2431449|AAAAAAAAJNJBFCAA|1944-12-24|539|2347|181|1944|0|12|24|4|1944|181|2347|Sunday|1944Q4|N|N|N|2431426|2431760|2431083|2431357|N|N|N|N|N| +2431450|AAAAAAAAKNJBFCAA|1944-12-25|539|2347|181|1944|1|12|25|4|1944|181|2347|Monday|1944Q4|Y|N|N|2431426|2431760|2431084|2431358|N|N|N|N|N| +2431451|AAAAAAAALNJBFCAA|1944-12-26|539|2348|181|1944|2|12|26|4|1944|181|2348|Tuesday|1944Q4|N|N|Y|2431426|2431760|2431085|2431359|N|N|N|N|N| +2431452|AAAAAAAAMNJBFCAA|1944-12-27|539|2348|181|1944|3|12|27|4|1944|181|2348|Wednesday|1944Q4|N|N|N|2431426|2431760|2431086|2431360|N|N|N|N|N| +2431453|AAAAAAAANNJBFCAA|1944-12-28|539|2348|181|1944|4|12|28|4|1944|181|2348|Thursday|1944Q4|N|N|N|2431426|2431760|2431087|2431361|N|N|N|N|N| +2431454|AAAAAAAAONJBFCAA|1944-12-29|539|2348|181|1944|5|12|29|4|1944|181|2348|Friday|1944Q4|N|Y|N|2431426|2431760|2431088|2431362|N|N|N|N|N| +2431455|AAAAAAAAPNJBFCAA|1944-12-30|539|2348|181|1944|6|12|30|4|1944|181|2348|Saturday|1944Q4|N|Y|N|2431426|2431760|2431089|2431363|N|N|N|N|N| +2431456|AAAAAAAAAOJBFCAA|1944-12-31|539|2348|181|1944|0|12|31|4|1944|181|2348|Sunday|1944Q4|Y|N|N|2431426|2431760|2431090|2431364|N|N|N|N|N| +2431457|AAAAAAAABOJBFCAA|1945-01-01|540|2348|181|1945|1|1|1|1|1945|181|2348|Monday|1945Q1|Y|N|Y|2431457|2431456|2431091|2431365|N|N|N|N|N| +2431458|AAAAAAAACOJBFCAA|1945-01-02|540|2349|181|1945|2|1|2|1|1945|181|2349|Tuesday|1945Q1|N|N|Y|2431457|2431456|2431092|2431366|N|N|N|N|N| +2431459|AAAAAAAADOJBFCAA|1945-01-03|540|2349|181|1945|3|1|3|1|1945|181|2349|Wednesday|1945Q1|N|N|N|2431457|2431456|2431093|2431367|N|N|N|N|N| +2431460|AAAAAAAAEOJBFCAA|1945-01-04|540|2349|181|1945|4|1|4|1|1945|181|2349|Thursday|1945Q1|N|N|N|2431457|2431456|2431094|2431368|N|N|N|N|N| +2431461|AAAAAAAAFOJBFCAA|1945-01-05|540|2349|181|1945|5|1|5|1|1945|181|2349|Friday|1945Q1|N|Y|N|2431457|2431456|2431095|2431369|N|N|N|N|N| +2431462|AAAAAAAAGOJBFCAA|1945-01-06|540|2349|181|1945|6|1|6|1|1945|181|2349|Saturday|1945Q1|N|Y|N|2431457|2431456|2431096|2431370|N|N|N|N|N| +2431463|AAAAAAAAHOJBFCAA|1945-01-07|540|2349|181|1945|0|1|7|1|1945|181|2349|Sunday|1945Q1|N|N|N|2431457|2431456|2431097|2431371|N|N|N|N|N| +2431464|AAAAAAAAIOJBFCAA|1945-01-08|540|2349|181|1945|1|1|8|1|1945|181|2349|Monday|1945Q1|N|N|N|2431457|2431456|2431098|2431372|N|N|N|N|N| +2431465|AAAAAAAAJOJBFCAA|1945-01-09|540|2350|181|1945|2|1|9|1|1945|181|2350|Tuesday|1945Q1|N|N|N|2431457|2431456|2431099|2431373|N|N|N|N|N| +2431466|AAAAAAAAKOJBFCAA|1945-01-10|540|2350|181|1945|3|1|10|1|1945|181|2350|Wednesday|1945Q1|N|N|N|2431457|2431456|2431100|2431374|N|N|N|N|N| +2431467|AAAAAAAALOJBFCAA|1945-01-11|540|2350|181|1945|4|1|11|1|1945|181|2350|Thursday|1945Q1|N|N|N|2431457|2431456|2431101|2431375|N|N|N|N|N| +2431468|AAAAAAAAMOJBFCAA|1945-01-12|540|2350|181|1945|5|1|12|1|1945|181|2350|Friday|1945Q1|N|Y|N|2431457|2431456|2431102|2431376|N|N|N|N|N| +2431469|AAAAAAAANOJBFCAA|1945-01-13|540|2350|181|1945|6|1|13|1|1945|181|2350|Saturday|1945Q1|N|Y|N|2431457|2431456|2431103|2431377|N|N|N|N|N| +2431470|AAAAAAAAOOJBFCAA|1945-01-14|540|2350|181|1945|0|1|14|1|1945|181|2350|Sunday|1945Q1|N|N|N|2431457|2431456|2431104|2431378|N|N|N|N|N| +2431471|AAAAAAAAPOJBFCAA|1945-01-15|540|2350|181|1945|1|1|15|1|1945|181|2350|Monday|1945Q1|N|N|N|2431457|2431456|2431105|2431379|N|N|N|N|N| +2431472|AAAAAAAAAPJBFCAA|1945-01-16|540|2351|181|1945|2|1|16|1|1945|181|2351|Tuesday|1945Q1|N|N|N|2431457|2431456|2431106|2431380|N|N|N|N|N| +2431473|AAAAAAAABPJBFCAA|1945-01-17|540|2351|181|1945|3|1|17|1|1945|181|2351|Wednesday|1945Q1|N|N|N|2431457|2431456|2431107|2431381|N|N|N|N|N| +2431474|AAAAAAAACPJBFCAA|1945-01-18|540|2351|181|1945|4|1|18|1|1945|181|2351|Thursday|1945Q1|N|N|N|2431457|2431456|2431108|2431382|N|N|N|N|N| +2431475|AAAAAAAADPJBFCAA|1945-01-19|540|2351|181|1945|5|1|19|1|1945|181|2351|Friday|1945Q1|N|Y|N|2431457|2431456|2431109|2431383|N|N|N|N|N| +2431476|AAAAAAAAEPJBFCAA|1945-01-20|540|2351|181|1945|6|1|20|1|1945|181|2351|Saturday|1945Q1|N|Y|N|2431457|2431456|2431110|2431384|N|N|N|N|N| +2431477|AAAAAAAAFPJBFCAA|1945-01-21|540|2351|181|1945|0|1|21|1|1945|181|2351|Sunday|1945Q1|N|N|N|2431457|2431456|2431111|2431385|N|N|N|N|N| +2431478|AAAAAAAAGPJBFCAA|1945-01-22|540|2351|181|1945|1|1|22|1|1945|181|2351|Monday|1945Q1|N|N|N|2431457|2431456|2431112|2431386|N|N|N|N|N| +2431479|AAAAAAAAHPJBFCAA|1945-01-23|540|2352|181|1945|2|1|23|1|1945|181|2352|Tuesday|1945Q1|N|N|N|2431457|2431456|2431113|2431387|N|N|N|N|N| +2431480|AAAAAAAAIPJBFCAA|1945-01-24|540|2352|181|1945|3|1|24|1|1945|181|2352|Wednesday|1945Q1|N|N|N|2431457|2431456|2431114|2431388|N|N|N|N|N| +2431481|AAAAAAAAJPJBFCAA|1945-01-25|540|2352|181|1945|4|1|25|1|1945|181|2352|Thursday|1945Q1|N|N|N|2431457|2431456|2431115|2431389|N|N|N|N|N| +2431482|AAAAAAAAKPJBFCAA|1945-01-26|540|2352|181|1945|5|1|26|1|1945|181|2352|Friday|1945Q1|N|Y|N|2431457|2431456|2431116|2431390|N|N|N|N|N| +2431483|AAAAAAAALPJBFCAA|1945-01-27|540|2352|181|1945|6|1|27|1|1945|181|2352|Saturday|1945Q1|N|Y|N|2431457|2431456|2431117|2431391|N|N|N|N|N| +2431484|AAAAAAAAMPJBFCAA|1945-01-28|540|2352|181|1945|0|1|28|1|1945|181|2352|Sunday|1945Q1|N|N|N|2431457|2431456|2431118|2431392|N|N|N|N|N| +2431485|AAAAAAAANPJBFCAA|1945-01-29|540|2352|181|1945|1|1|29|1|1945|181|2352|Monday|1945Q1|N|N|N|2431457|2431456|2431119|2431393|N|N|N|N|N| +2431486|AAAAAAAAOPJBFCAA|1945-01-30|540|2353|181|1945|2|1|30|1|1945|181|2353|Tuesday|1945Q1|N|N|N|2431457|2431456|2431120|2431394|N|N|N|N|N| +2431487|AAAAAAAAPPJBFCAA|1945-01-31|540|2353|181|1945|3|1|31|1|1945|181|2353|Wednesday|1945Q1|N|N|N|2431457|2431456|2431121|2431395|N|N|N|N|N| +2431488|AAAAAAAAAAKBFCAA|1945-02-01|541|2353|181|1945|4|2|1|1|1945|181|2353|Thursday|1945Q1|N|N|N|2431488|2431518|2431122|2431396|N|N|N|N|N| +2431489|AAAAAAAABAKBFCAA|1945-02-02|541|2353|181|1945|5|2|2|1|1945|181|2353|Friday|1945Q1|N|Y|N|2431488|2431518|2431123|2431397|N|N|N|N|N| +2431490|AAAAAAAACAKBFCAA|1945-02-03|541|2353|181|1945|6|2|3|1|1945|181|2353|Saturday|1945Q1|N|Y|N|2431488|2431518|2431124|2431398|N|N|N|N|N| +2431491|AAAAAAAADAKBFCAA|1945-02-04|541|2353|181|1945|0|2|4|1|1945|181|2353|Sunday|1945Q1|N|N|N|2431488|2431518|2431125|2431399|N|N|N|N|N| +2431492|AAAAAAAAEAKBFCAA|1945-02-05|541|2353|181|1945|1|2|5|1|1945|181|2353|Monday|1945Q1|N|N|N|2431488|2431518|2431126|2431400|N|N|N|N|N| +2431493|AAAAAAAAFAKBFCAA|1945-02-06|541|2354|181|1945|2|2|6|1|1945|181|2354|Tuesday|1945Q1|N|N|N|2431488|2431518|2431127|2431401|N|N|N|N|N| +2431494|AAAAAAAAGAKBFCAA|1945-02-07|541|2354|181|1945|3|2|7|1|1945|181|2354|Wednesday|1945Q1|N|N|N|2431488|2431518|2431128|2431402|N|N|N|N|N| +2431495|AAAAAAAAHAKBFCAA|1945-02-08|541|2354|181|1945|4|2|8|1|1945|181|2354|Thursday|1945Q1|N|N|N|2431488|2431518|2431129|2431403|N|N|N|N|N| +2431496|AAAAAAAAIAKBFCAA|1945-02-09|541|2354|181|1945|5|2|9|1|1945|181|2354|Friday|1945Q1|N|Y|N|2431488|2431518|2431130|2431404|N|N|N|N|N| +2431497|AAAAAAAAJAKBFCAA|1945-02-10|541|2354|181|1945|6|2|10|1|1945|181|2354|Saturday|1945Q1|N|Y|N|2431488|2431518|2431131|2431405|N|N|N|N|N| +2431498|AAAAAAAAKAKBFCAA|1945-02-11|541|2354|181|1945|0|2|11|1|1945|181|2354|Sunday|1945Q1|N|N|N|2431488|2431518|2431132|2431406|N|N|N|N|N| +2431499|AAAAAAAALAKBFCAA|1945-02-12|541|2354|181|1945|1|2|12|1|1945|181|2354|Monday|1945Q1|N|N|N|2431488|2431518|2431133|2431407|N|N|N|N|N| +2431500|AAAAAAAAMAKBFCAA|1945-02-13|541|2355|181|1945|2|2|13|1|1945|181|2355|Tuesday|1945Q1|N|N|N|2431488|2431518|2431134|2431408|N|N|N|N|N| +2431501|AAAAAAAANAKBFCAA|1945-02-14|541|2355|181|1945|3|2|14|1|1945|181|2355|Wednesday|1945Q1|N|N|N|2431488|2431518|2431135|2431409|N|N|N|N|N| +2431502|AAAAAAAAOAKBFCAA|1945-02-15|541|2355|181|1945|4|2|15|1|1945|181|2355|Thursday|1945Q1|N|N|N|2431488|2431518|2431136|2431410|N|N|N|N|N| +2431503|AAAAAAAAPAKBFCAA|1945-02-16|541|2355|181|1945|5|2|16|1|1945|181|2355|Friday|1945Q1|N|Y|N|2431488|2431518|2431137|2431411|N|N|N|N|N| +2431504|AAAAAAAAABKBFCAA|1945-02-17|541|2355|181|1945|6|2|17|1|1945|181|2355|Saturday|1945Q1|N|Y|N|2431488|2431518|2431138|2431412|N|N|N|N|N| +2431505|AAAAAAAABBKBFCAA|1945-02-18|541|2355|181|1945|0|2|18|1|1945|181|2355|Sunday|1945Q1|N|N|N|2431488|2431518|2431139|2431413|N|N|N|N|N| +2431506|AAAAAAAACBKBFCAA|1945-02-19|541|2355|181|1945|1|2|19|1|1945|181|2355|Monday|1945Q1|N|N|N|2431488|2431518|2431140|2431414|N|N|N|N|N| +2431507|AAAAAAAADBKBFCAA|1945-02-20|541|2356|181|1945|2|2|20|1|1945|181|2356|Tuesday|1945Q1|N|N|N|2431488|2431518|2431141|2431415|N|N|N|N|N| +2431508|AAAAAAAAEBKBFCAA|1945-02-21|541|2356|181|1945|3|2|21|1|1945|181|2356|Wednesday|1945Q1|N|N|N|2431488|2431518|2431142|2431416|N|N|N|N|N| +2431509|AAAAAAAAFBKBFCAA|1945-02-22|541|2356|181|1945|4|2|22|1|1945|181|2356|Thursday|1945Q1|N|N|N|2431488|2431518|2431143|2431417|N|N|N|N|N| +2431510|AAAAAAAAGBKBFCAA|1945-02-23|541|2356|181|1945|5|2|23|1|1945|181|2356|Friday|1945Q1|N|Y|N|2431488|2431518|2431144|2431418|N|N|N|N|N| +2431511|AAAAAAAAHBKBFCAA|1945-02-24|541|2356|181|1945|6|2|24|1|1945|181|2356|Saturday|1945Q1|N|Y|N|2431488|2431518|2431145|2431419|N|N|N|N|N| +2431512|AAAAAAAAIBKBFCAA|1945-02-25|541|2356|181|1945|0|2|25|1|1945|181|2356|Sunday|1945Q1|N|N|N|2431488|2431518|2431146|2431420|N|N|N|N|N| +2431513|AAAAAAAAJBKBFCAA|1945-02-26|541|2356|181|1945|1|2|26|1|1945|181|2356|Monday|1945Q1|N|N|N|2431488|2431518|2431147|2431421|N|N|N|N|N| +2431514|AAAAAAAAKBKBFCAA|1945-02-27|541|2357|181|1945|2|2|27|1|1945|181|2357|Tuesday|1945Q1|N|N|N|2431488|2431518|2431148|2431422|N|N|N|N|N| +2431515|AAAAAAAALBKBFCAA|1945-02-28|541|2357|181|1945|3|2|28|1|1945|181|2357|Wednesday|1945Q1|N|N|N|2431488|2431518|2431149|2431423|N|N|N|N|N| +2431516|AAAAAAAAMBKBFCAA|1945-03-01|542|2357|182|1945|4|3|1|1|1945|182|2357|Thursday|1945Q1|N|N|N|2431516|2431574|2431151|2431424|N|N|N|N|N| +2431517|AAAAAAAANBKBFCAA|1945-03-02|542|2357|182|1945|5|3|2|1|1945|182|2357|Friday|1945Q1|N|Y|N|2431516|2431574|2431152|2431425|N|N|N|N|N| +2431518|AAAAAAAAOBKBFCAA|1945-03-03|542|2357|182|1945|6|3|3|1|1945|182|2357|Saturday|1945Q1|N|Y|N|2431516|2431574|2431153|2431426|N|N|N|N|N| +2431519|AAAAAAAAPBKBFCAA|1945-03-04|542|2357|182|1945|0|3|4|1|1945|182|2357|Sunday|1945Q1|N|N|N|2431516|2431574|2431154|2431427|N|N|N|N|N| +2431520|AAAAAAAAACKBFCAA|1945-03-05|542|2357|182|1945|1|3|5|1|1945|182|2357|Monday|1945Q1|N|N|N|2431516|2431574|2431155|2431428|N|N|N|N|N| +2431521|AAAAAAAABCKBFCAA|1945-03-06|542|2358|182|1945|2|3|6|1|1945|182|2358|Tuesday|1945Q1|N|N|N|2431516|2431574|2431156|2431429|N|N|N|N|N| +2431522|AAAAAAAACCKBFCAA|1945-03-07|542|2358|182|1945|3|3|7|1|1945|182|2358|Wednesday|1945Q1|N|N|N|2431516|2431574|2431157|2431430|N|N|N|N|N| +2431523|AAAAAAAADCKBFCAA|1945-03-08|542|2358|182|1945|4|3|8|1|1945|182|2358|Thursday|1945Q1|N|N|N|2431516|2431574|2431158|2431431|N|N|N|N|N| +2431524|AAAAAAAAECKBFCAA|1945-03-09|542|2358|182|1945|5|3|9|1|1945|182|2358|Friday|1945Q1|N|Y|N|2431516|2431574|2431159|2431432|N|N|N|N|N| +2431525|AAAAAAAAFCKBFCAA|1945-03-10|542|2358|182|1945|6|3|10|1|1945|182|2358|Saturday|1945Q1|N|Y|N|2431516|2431574|2431160|2431433|N|N|N|N|N| +2431526|AAAAAAAAGCKBFCAA|1945-03-11|542|2358|182|1945|0|3|11|1|1945|182|2358|Sunday|1945Q1|N|N|N|2431516|2431574|2431161|2431434|N|N|N|N|N| +2431527|AAAAAAAAHCKBFCAA|1945-03-12|542|2358|182|1945|1|3|12|1|1945|182|2358|Monday|1945Q1|N|N|N|2431516|2431574|2431162|2431435|N|N|N|N|N| +2431528|AAAAAAAAICKBFCAA|1945-03-13|542|2359|182|1945|2|3|13|1|1945|182|2359|Tuesday|1945Q1|N|N|N|2431516|2431574|2431163|2431436|N|N|N|N|N| +2431529|AAAAAAAAJCKBFCAA|1945-03-14|542|2359|182|1945|3|3|14|1|1945|182|2359|Wednesday|1945Q1|N|N|N|2431516|2431574|2431164|2431437|N|N|N|N|N| +2431530|AAAAAAAAKCKBFCAA|1945-03-15|542|2359|182|1945|4|3|15|1|1945|182|2359|Thursday|1945Q1|N|N|N|2431516|2431574|2431165|2431438|N|N|N|N|N| +2431531|AAAAAAAALCKBFCAA|1945-03-16|542|2359|182|1945|5|3|16|1|1945|182|2359|Friday|1945Q1|N|Y|N|2431516|2431574|2431166|2431439|N|N|N|N|N| +2431532|AAAAAAAAMCKBFCAA|1945-03-17|542|2359|182|1945|6|3|17|1|1945|182|2359|Saturday|1945Q1|N|Y|N|2431516|2431574|2431167|2431440|N|N|N|N|N| +2431533|AAAAAAAANCKBFCAA|1945-03-18|542|2359|182|1945|0|3|18|1|1945|182|2359|Sunday|1945Q1|N|N|N|2431516|2431574|2431168|2431441|N|N|N|N|N| +2431534|AAAAAAAAOCKBFCAA|1945-03-19|542|2359|182|1945|1|3|19|1|1945|182|2359|Monday|1945Q1|N|N|N|2431516|2431574|2431169|2431442|N|N|N|N|N| +2431535|AAAAAAAAPCKBFCAA|1945-03-20|542|2360|182|1945|2|3|20|1|1945|182|2360|Tuesday|1945Q1|N|N|N|2431516|2431574|2431170|2431443|N|N|N|N|N| +2431536|AAAAAAAAADKBFCAA|1945-03-21|542|2360|182|1945|3|3|21|1|1945|182|2360|Wednesday|1945Q1|N|N|N|2431516|2431574|2431171|2431444|N|N|N|N|N| +2431537|AAAAAAAABDKBFCAA|1945-03-22|542|2360|182|1945|4|3|22|1|1945|182|2360|Thursday|1945Q1|N|N|N|2431516|2431574|2431172|2431445|N|N|N|N|N| +2431538|AAAAAAAACDKBFCAA|1945-03-23|542|2360|182|1945|5|3|23|1|1945|182|2360|Friday|1945Q1|N|Y|N|2431516|2431574|2431173|2431446|N|N|N|N|N| +2431539|AAAAAAAADDKBFCAA|1945-03-24|542|2360|182|1945|6|3|24|1|1945|182|2360|Saturday|1945Q1|N|Y|N|2431516|2431574|2431174|2431447|N|N|N|N|N| +2431540|AAAAAAAAEDKBFCAA|1945-03-25|542|2360|182|1945|0|3|25|1|1945|182|2360|Sunday|1945Q1|N|N|N|2431516|2431574|2431175|2431448|N|N|N|N|N| +2431541|AAAAAAAAFDKBFCAA|1945-03-26|542|2360|182|1945|1|3|26|1|1945|182|2360|Monday|1945Q1|N|N|N|2431516|2431574|2431176|2431449|N|N|N|N|N| +2431542|AAAAAAAAGDKBFCAA|1945-03-27|542|2361|182|1945|2|3|27|1|1945|182|2361|Tuesday|1945Q1|N|N|N|2431516|2431574|2431177|2431450|N|N|N|N|N| +2431543|AAAAAAAAHDKBFCAA|1945-03-28|542|2361|182|1945|3|3|28|1|1945|182|2361|Wednesday|1945Q1|N|N|N|2431516|2431574|2431178|2431451|N|N|N|N|N| +2431544|AAAAAAAAIDKBFCAA|1945-03-29|542|2361|182|1945|4|3|29|1|1945|182|2361|Thursday|1945Q1|N|N|N|2431516|2431574|2431179|2431452|N|N|N|N|N| +2431545|AAAAAAAAJDKBFCAA|1945-03-30|542|2361|182|1945|5|3|30|1|1945|182|2361|Friday|1945Q1|N|Y|N|2431516|2431574|2431180|2431453|N|N|N|N|N| +2431546|AAAAAAAAKDKBFCAA|1945-03-31|542|2361|182|1945|6|3|31|1|1945|182|2361|Saturday|1945Q1|N|Y|N|2431516|2431574|2431181|2431454|N|N|N|N|N| +2431547|AAAAAAAALDKBFCAA|1945-04-01|543|2361|182|1945|0|4|1|1|1945|182|2361|Sunday|1945Q1|N|N|N|2431547|2431636|2431182|2431457|N|N|N|N|N| +2431548|AAAAAAAAMDKBFCAA|1945-04-02|543|2361|182|1945|1|4|2|2|1945|182|2361|Monday|1945Q2|N|N|N|2431547|2431636|2431183|2431458|N|N|N|N|N| +2431549|AAAAAAAANDKBFCAA|1945-04-03|543|2362|182|1945|2|4|3|2|1945|182|2362|Tuesday|1945Q2|N|N|N|2431547|2431636|2431184|2431459|N|N|N|N|N| +2431550|AAAAAAAAODKBFCAA|1945-04-04|543|2362|182|1945|3|4|4|2|1945|182|2362|Wednesday|1945Q2|N|N|N|2431547|2431636|2431185|2431460|N|N|N|N|N| +2431551|AAAAAAAAPDKBFCAA|1945-04-05|543|2362|182|1945|4|4|5|2|1945|182|2362|Thursday|1945Q2|N|N|N|2431547|2431636|2431186|2431461|N|N|N|N|N| +2431552|AAAAAAAAAEKBFCAA|1945-04-06|543|2362|182|1945|5|4|6|2|1945|182|2362|Friday|1945Q2|N|Y|N|2431547|2431636|2431187|2431462|N|N|N|N|N| +2431553|AAAAAAAABEKBFCAA|1945-04-07|543|2362|182|1945|6|4|7|2|1945|182|2362|Saturday|1945Q2|N|Y|N|2431547|2431636|2431188|2431463|N|N|N|N|N| +2431554|AAAAAAAACEKBFCAA|1945-04-08|543|2362|182|1945|0|4|8|2|1945|182|2362|Sunday|1945Q2|N|N|N|2431547|2431636|2431189|2431464|N|N|N|N|N| +2431555|AAAAAAAADEKBFCAA|1945-04-09|543|2362|182|1945|1|4|9|2|1945|182|2362|Monday|1945Q2|N|N|N|2431547|2431636|2431190|2431465|N|N|N|N|N| +2431556|AAAAAAAAEEKBFCAA|1945-04-10|543|2363|182|1945|2|4|10|2|1945|182|2363|Tuesday|1945Q2|N|N|N|2431547|2431636|2431191|2431466|N|N|N|N|N| +2431557|AAAAAAAAFEKBFCAA|1945-04-11|543|2363|182|1945|3|4|11|2|1945|182|2363|Wednesday|1945Q2|N|N|N|2431547|2431636|2431192|2431467|N|N|N|N|N| +2431558|AAAAAAAAGEKBFCAA|1945-04-12|543|2363|182|1945|4|4|12|2|1945|182|2363|Thursday|1945Q2|N|N|N|2431547|2431636|2431193|2431468|N|N|N|N|N| +2431559|AAAAAAAAHEKBFCAA|1945-04-13|543|2363|182|1945|5|4|13|2|1945|182|2363|Friday|1945Q2|N|Y|N|2431547|2431636|2431194|2431469|N|N|N|N|N| +2431560|AAAAAAAAIEKBFCAA|1945-04-14|543|2363|182|1945|6|4|14|2|1945|182|2363|Saturday|1945Q2|N|Y|N|2431547|2431636|2431195|2431470|N|N|N|N|N| +2431561|AAAAAAAAJEKBFCAA|1945-04-15|543|2363|182|1945|0|4|15|2|1945|182|2363|Sunday|1945Q2|N|N|N|2431547|2431636|2431196|2431471|N|N|N|N|N| +2431562|AAAAAAAAKEKBFCAA|1945-04-16|543|2363|182|1945|1|4|16|2|1945|182|2363|Monday|1945Q2|N|N|N|2431547|2431636|2431197|2431472|N|N|N|N|N| +2431563|AAAAAAAALEKBFCAA|1945-04-17|543|2364|182|1945|2|4|17|2|1945|182|2364|Tuesday|1945Q2|N|N|N|2431547|2431636|2431198|2431473|N|N|N|N|N| +2431564|AAAAAAAAMEKBFCAA|1945-04-18|543|2364|182|1945|3|4|18|2|1945|182|2364|Wednesday|1945Q2|N|N|N|2431547|2431636|2431199|2431474|N|N|N|N|N| +2431565|AAAAAAAANEKBFCAA|1945-04-19|543|2364|182|1945|4|4|19|2|1945|182|2364|Thursday|1945Q2|N|N|N|2431547|2431636|2431200|2431475|N|N|N|N|N| +2431566|AAAAAAAAOEKBFCAA|1945-04-20|543|2364|182|1945|5|4|20|2|1945|182|2364|Friday|1945Q2|N|Y|N|2431547|2431636|2431201|2431476|N|N|N|N|N| +2431567|AAAAAAAAPEKBFCAA|1945-04-21|543|2364|182|1945|6|4|21|2|1945|182|2364|Saturday|1945Q2|N|Y|N|2431547|2431636|2431202|2431477|N|N|N|N|N| +2431568|AAAAAAAAAFKBFCAA|1945-04-22|543|2364|182|1945|0|4|22|2|1945|182|2364|Sunday|1945Q2|N|N|N|2431547|2431636|2431203|2431478|N|N|N|N|N| +2431569|AAAAAAAABFKBFCAA|1945-04-23|543|2364|182|1945|1|4|23|2|1945|182|2364|Monday|1945Q2|N|N|N|2431547|2431636|2431204|2431479|N|N|N|N|N| +2431570|AAAAAAAACFKBFCAA|1945-04-24|543|2365|182|1945|2|4|24|2|1945|182|2365|Tuesday|1945Q2|N|N|N|2431547|2431636|2431205|2431480|N|N|N|N|N| +2431571|AAAAAAAADFKBFCAA|1945-04-25|543|2365|182|1945|3|4|25|2|1945|182|2365|Wednesday|1945Q2|N|N|N|2431547|2431636|2431206|2431481|N|N|N|N|N| +2431572|AAAAAAAAEFKBFCAA|1945-04-26|543|2365|182|1945|4|4|26|2|1945|182|2365|Thursday|1945Q2|N|N|N|2431547|2431636|2431207|2431482|N|N|N|N|N| +2431573|AAAAAAAAFFKBFCAA|1945-04-27|543|2365|182|1945|5|4|27|2|1945|182|2365|Friday|1945Q2|N|Y|N|2431547|2431636|2431208|2431483|N|N|N|N|N| +2431574|AAAAAAAAGFKBFCAA|1945-04-28|543|2365|182|1945|6|4|28|2|1945|182|2365|Saturday|1945Q2|N|Y|N|2431547|2431636|2431209|2431484|N|N|N|N|N| +2431575|AAAAAAAAHFKBFCAA|1945-04-29|543|2365|182|1945|0|4|29|2|1945|182|2365|Sunday|1945Q2|N|N|N|2431547|2431636|2431210|2431485|N|N|N|N|N| +2431576|AAAAAAAAIFKBFCAA|1945-04-30|543|2365|182|1945|1|4|30|2|1945|182|2365|Monday|1945Q2|N|N|N|2431547|2431636|2431211|2431486|N|N|N|N|N| +2431577|AAAAAAAAJFKBFCAA|1945-05-01|544|2366|182|1945|2|5|1|2|1945|182|2366|Tuesday|1945Q2|N|N|N|2431577|2431696|2431212|2431487|N|N|N|N|N| +2431578|AAAAAAAAKFKBFCAA|1945-05-02|544|2366|182|1945|3|5|2|2|1945|182|2366|Wednesday|1945Q2|N|N|N|2431577|2431696|2431213|2431488|N|N|N|N|N| +2431579|AAAAAAAALFKBFCAA|1945-05-03|544|2366|182|1945|4|5|3|2|1945|182|2366|Thursday|1945Q2|N|N|N|2431577|2431696|2431214|2431489|N|N|N|N|N| +2431580|AAAAAAAAMFKBFCAA|1945-05-04|544|2366|182|1945|5|5|4|2|1945|182|2366|Friday|1945Q2|N|Y|N|2431577|2431696|2431215|2431490|N|N|N|N|N| +2431581|AAAAAAAANFKBFCAA|1945-05-05|544|2366|182|1945|6|5|5|2|1945|182|2366|Saturday|1945Q2|N|Y|N|2431577|2431696|2431216|2431491|N|N|N|N|N| +2431582|AAAAAAAAOFKBFCAA|1945-05-06|544|2366|182|1945|0|5|6|2|1945|182|2366|Sunday|1945Q2|N|N|N|2431577|2431696|2431217|2431492|N|N|N|N|N| +2431583|AAAAAAAAPFKBFCAA|1945-05-07|544|2366|182|1945|1|5|7|2|1945|182|2366|Monday|1945Q2|N|N|N|2431577|2431696|2431218|2431493|N|N|N|N|N| +2431584|AAAAAAAAAGKBFCAA|1945-05-08|544|2367|182|1945|2|5|8|2|1945|182|2367|Tuesday|1945Q2|N|N|N|2431577|2431696|2431219|2431494|N|N|N|N|N| +2431585|AAAAAAAABGKBFCAA|1945-05-09|544|2367|182|1945|3|5|9|2|1945|182|2367|Wednesday|1945Q2|N|N|N|2431577|2431696|2431220|2431495|N|N|N|N|N| +2431586|AAAAAAAACGKBFCAA|1945-05-10|544|2367|182|1945|4|5|10|2|1945|182|2367|Thursday|1945Q2|N|N|N|2431577|2431696|2431221|2431496|N|N|N|N|N| +2431587|AAAAAAAADGKBFCAA|1945-05-11|544|2367|182|1945|5|5|11|2|1945|182|2367|Friday|1945Q2|N|Y|N|2431577|2431696|2431222|2431497|N|N|N|N|N| +2431588|AAAAAAAAEGKBFCAA|1945-05-12|544|2367|182|1945|6|5|12|2|1945|182|2367|Saturday|1945Q2|N|Y|N|2431577|2431696|2431223|2431498|N|N|N|N|N| +2431589|AAAAAAAAFGKBFCAA|1945-05-13|544|2367|182|1945|0|5|13|2|1945|182|2367|Sunday|1945Q2|N|N|N|2431577|2431696|2431224|2431499|N|N|N|N|N| +2431590|AAAAAAAAGGKBFCAA|1945-05-14|544|2367|182|1945|1|5|14|2|1945|182|2367|Monday|1945Q2|N|N|N|2431577|2431696|2431225|2431500|N|N|N|N|N| +2431591|AAAAAAAAHGKBFCAA|1945-05-15|544|2368|182|1945|2|5|15|2|1945|182|2368|Tuesday|1945Q2|N|N|N|2431577|2431696|2431226|2431501|N|N|N|N|N| +2431592|AAAAAAAAIGKBFCAA|1945-05-16|544|2368|182|1945|3|5|16|2|1945|182|2368|Wednesday|1945Q2|N|N|N|2431577|2431696|2431227|2431502|N|N|N|N|N| +2431593|AAAAAAAAJGKBFCAA|1945-05-17|544|2368|182|1945|4|5|17|2|1945|182|2368|Thursday|1945Q2|N|N|N|2431577|2431696|2431228|2431503|N|N|N|N|N| +2431594|AAAAAAAAKGKBFCAA|1945-05-18|544|2368|182|1945|5|5|18|2|1945|182|2368|Friday|1945Q2|N|Y|N|2431577|2431696|2431229|2431504|N|N|N|N|N| +2431595|AAAAAAAALGKBFCAA|1945-05-19|544|2368|182|1945|6|5|19|2|1945|182|2368|Saturday|1945Q2|N|Y|N|2431577|2431696|2431230|2431505|N|N|N|N|N| +2431596|AAAAAAAAMGKBFCAA|1945-05-20|544|2368|182|1945|0|5|20|2|1945|182|2368|Sunday|1945Q2|N|N|N|2431577|2431696|2431231|2431506|N|N|N|N|N| +2431597|AAAAAAAANGKBFCAA|1945-05-21|544|2368|182|1945|1|5|21|2|1945|182|2368|Monday|1945Q2|N|N|N|2431577|2431696|2431232|2431507|N|N|N|N|N| +2431598|AAAAAAAAOGKBFCAA|1945-05-22|544|2369|182|1945|2|5|22|2|1945|182|2369|Tuesday|1945Q2|N|N|N|2431577|2431696|2431233|2431508|N|N|N|N|N| +2431599|AAAAAAAAPGKBFCAA|1945-05-23|544|2369|182|1945|3|5|23|2|1945|182|2369|Wednesday|1945Q2|N|N|N|2431577|2431696|2431234|2431509|N|N|N|N|N| +2431600|AAAAAAAAAHKBFCAA|1945-05-24|544|2369|182|1945|4|5|24|2|1945|182|2369|Thursday|1945Q2|N|N|N|2431577|2431696|2431235|2431510|N|N|N|N|N| +2431601|AAAAAAAABHKBFCAA|1945-05-25|544|2369|182|1945|5|5|25|2|1945|182|2369|Friday|1945Q2|N|Y|N|2431577|2431696|2431236|2431511|N|N|N|N|N| +2431602|AAAAAAAACHKBFCAA|1945-05-26|544|2369|182|1945|6|5|26|2|1945|182|2369|Saturday|1945Q2|N|Y|N|2431577|2431696|2431237|2431512|N|N|N|N|N| +2431603|AAAAAAAADHKBFCAA|1945-05-27|544|2369|182|1945|0|5|27|2|1945|182|2369|Sunday|1945Q2|N|N|N|2431577|2431696|2431238|2431513|N|N|N|N|N| +2431604|AAAAAAAAEHKBFCAA|1945-05-28|544|2369|182|1945|1|5|28|2|1945|182|2369|Monday|1945Q2|N|N|N|2431577|2431696|2431239|2431514|N|N|N|N|N| +2431605|AAAAAAAAFHKBFCAA|1945-05-29|544|2370|182|1945|2|5|29|2|1945|182|2370|Tuesday|1945Q2|N|N|N|2431577|2431696|2431240|2431515|N|N|N|N|N| +2431606|AAAAAAAAGHKBFCAA|1945-05-30|544|2370|182|1945|3|5|30|2|1945|182|2370|Wednesday|1945Q2|N|N|N|2431577|2431696|2431241|2431516|N|N|N|N|N| +2431607|AAAAAAAAHHKBFCAA|1945-05-31|544|2370|182|1945|4|5|31|2|1945|182|2370|Thursday|1945Q2|N|N|N|2431577|2431696|2431242|2431517|N|N|N|N|N| +2431608|AAAAAAAAIHKBFCAA|1945-06-01|545|2370|183|1945|5|6|1|2|1945|183|2370|Friday|1945Q2|N|Y|N|2431608|2431758|2431243|2431518|N|N|N|N|N| +2431609|AAAAAAAAJHKBFCAA|1945-06-02|545|2370|183|1945|6|6|2|2|1945|183|2370|Saturday|1945Q2|N|Y|N|2431608|2431758|2431244|2431519|N|N|N|N|N| +2431610|AAAAAAAAKHKBFCAA|1945-06-03|545|2370|183|1945|0|6|3|2|1945|183|2370|Sunday|1945Q2|N|N|N|2431608|2431758|2431245|2431520|N|N|N|N|N| +2431611|AAAAAAAALHKBFCAA|1945-06-04|545|2370|183|1945|1|6|4|2|1945|183|2370|Monday|1945Q2|N|N|N|2431608|2431758|2431246|2431521|N|N|N|N|N| +2431612|AAAAAAAAMHKBFCAA|1945-06-05|545|2371|183|1945|2|6|5|2|1945|183|2371|Tuesday|1945Q2|N|N|N|2431608|2431758|2431247|2431522|N|N|N|N|N| +2431613|AAAAAAAANHKBFCAA|1945-06-06|545|2371|183|1945|3|6|6|2|1945|183|2371|Wednesday|1945Q2|N|N|N|2431608|2431758|2431248|2431523|N|N|N|N|N| +2431614|AAAAAAAAOHKBFCAA|1945-06-07|545|2371|183|1945|4|6|7|2|1945|183|2371|Thursday|1945Q2|N|N|N|2431608|2431758|2431249|2431524|N|N|N|N|N| +2431615|AAAAAAAAPHKBFCAA|1945-06-08|545|2371|183|1945|5|6|8|2|1945|183|2371|Friday|1945Q2|N|Y|N|2431608|2431758|2431250|2431525|N|N|N|N|N| +2431616|AAAAAAAAAIKBFCAA|1945-06-09|545|2371|183|1945|6|6|9|2|1945|183|2371|Saturday|1945Q2|N|Y|N|2431608|2431758|2431251|2431526|N|N|N|N|N| +2431617|AAAAAAAABIKBFCAA|1945-06-10|545|2371|183|1945|0|6|10|2|1945|183|2371|Sunday|1945Q2|N|N|N|2431608|2431758|2431252|2431527|N|N|N|N|N| +2431618|AAAAAAAACIKBFCAA|1945-06-11|545|2371|183|1945|1|6|11|2|1945|183|2371|Monday|1945Q2|N|N|N|2431608|2431758|2431253|2431528|N|N|N|N|N| +2431619|AAAAAAAADIKBFCAA|1945-06-12|545|2372|183|1945|2|6|12|2|1945|183|2372|Tuesday|1945Q2|N|N|N|2431608|2431758|2431254|2431529|N|N|N|N|N| +2431620|AAAAAAAAEIKBFCAA|1945-06-13|545|2372|183|1945|3|6|13|2|1945|183|2372|Wednesday|1945Q2|N|N|N|2431608|2431758|2431255|2431530|N|N|N|N|N| +2431621|AAAAAAAAFIKBFCAA|1945-06-14|545|2372|183|1945|4|6|14|2|1945|183|2372|Thursday|1945Q2|N|N|N|2431608|2431758|2431256|2431531|N|N|N|N|N| +2431622|AAAAAAAAGIKBFCAA|1945-06-15|545|2372|183|1945|5|6|15|2|1945|183|2372|Friday|1945Q2|N|Y|N|2431608|2431758|2431257|2431532|N|N|N|N|N| +2431623|AAAAAAAAHIKBFCAA|1945-06-16|545|2372|183|1945|6|6|16|2|1945|183|2372|Saturday|1945Q2|N|Y|N|2431608|2431758|2431258|2431533|N|N|N|N|N| +2431624|AAAAAAAAIIKBFCAA|1945-06-17|545|2372|183|1945|0|6|17|2|1945|183|2372|Sunday|1945Q2|N|N|N|2431608|2431758|2431259|2431534|N|N|N|N|N| +2431625|AAAAAAAAJIKBFCAA|1945-06-18|545|2372|183|1945|1|6|18|2|1945|183|2372|Monday|1945Q2|N|N|N|2431608|2431758|2431260|2431535|N|N|N|N|N| +2431626|AAAAAAAAKIKBFCAA|1945-06-19|545|2373|183|1945|2|6|19|2|1945|183|2373|Tuesday|1945Q2|N|N|N|2431608|2431758|2431261|2431536|N|N|N|N|N| +2431627|AAAAAAAALIKBFCAA|1945-06-20|545|2373|183|1945|3|6|20|2|1945|183|2373|Wednesday|1945Q2|N|N|N|2431608|2431758|2431262|2431537|N|N|N|N|N| +2431628|AAAAAAAAMIKBFCAA|1945-06-21|545|2373|183|1945|4|6|21|2|1945|183|2373|Thursday|1945Q2|N|N|N|2431608|2431758|2431263|2431538|N|N|N|N|N| +2431629|AAAAAAAANIKBFCAA|1945-06-22|545|2373|183|1945|5|6|22|2|1945|183|2373|Friday|1945Q2|N|Y|N|2431608|2431758|2431264|2431539|N|N|N|N|N| +2431630|AAAAAAAAOIKBFCAA|1945-06-23|545|2373|183|1945|6|6|23|2|1945|183|2373|Saturday|1945Q2|N|Y|N|2431608|2431758|2431265|2431540|N|N|N|N|N| +2431631|AAAAAAAAPIKBFCAA|1945-06-24|545|2373|183|1945|0|6|24|2|1945|183|2373|Sunday|1945Q2|N|N|N|2431608|2431758|2431266|2431541|N|N|N|N|N| +2431632|AAAAAAAAAJKBFCAA|1945-06-25|545|2373|183|1945|1|6|25|2|1945|183|2373|Monday|1945Q2|N|N|N|2431608|2431758|2431267|2431542|N|N|N|N|N| +2431633|AAAAAAAABJKBFCAA|1945-06-26|545|2374|183|1945|2|6|26|2|1945|183|2374|Tuesday|1945Q2|N|N|N|2431608|2431758|2431268|2431543|N|N|N|N|N| +2431634|AAAAAAAACJKBFCAA|1945-06-27|545|2374|183|1945|3|6|27|2|1945|183|2374|Wednesday|1945Q2|N|N|N|2431608|2431758|2431269|2431544|N|N|N|N|N| +2431635|AAAAAAAADJKBFCAA|1945-06-28|545|2374|183|1945|4|6|28|2|1945|183|2374|Thursday|1945Q2|N|N|N|2431608|2431758|2431270|2431545|N|N|N|N|N| +2431636|AAAAAAAAEJKBFCAA|1945-06-29|545|2374|183|1945|5|6|29|2|1945|183|2374|Friday|1945Q2|N|Y|N|2431608|2431758|2431271|2431546|N|N|N|N|N| +2431637|AAAAAAAAFJKBFCAA|1945-06-30|545|2374|183|1945|6|6|30|2|1945|183|2374|Saturday|1945Q2|N|Y|N|2431608|2431758|2431272|2431547|N|N|N|N|N| +2431638|AAAAAAAAGJKBFCAA|1945-07-01|546|2374|183|1945|0|7|1|2|1945|183|2374|Sunday|1945Q2|N|N|N|2431638|2431818|2431273|2431547|N|N|N|N|N| +2431639|AAAAAAAAHJKBFCAA|1945-07-02|546|2374|183|1945|1|7|2|3|1945|183|2374|Monday|1945Q3|N|N|N|2431638|2431818|2431274|2431548|N|N|N|N|N| +2431640|AAAAAAAAIJKBFCAA|1945-07-03|546|2375|183|1945|2|7|3|3|1945|183|2375|Tuesday|1945Q3|N|N|N|2431638|2431818|2431275|2431549|N|N|N|N|N| +2431641|AAAAAAAAJJKBFCAA|1945-07-04|546|2375|183|1945|3|7|4|3|1945|183|2375|Wednesday|1945Q3|N|N|N|2431638|2431818|2431276|2431550|N|N|N|N|N| +2431642|AAAAAAAAKJKBFCAA|1945-07-05|546|2375|183|1945|4|7|5|3|1945|183|2375|Thursday|1945Q3|Y|N|N|2431638|2431818|2431277|2431551|N|N|N|N|N| +2431643|AAAAAAAALJKBFCAA|1945-07-06|546|2375|183|1945|5|7|6|3|1945|183|2375|Friday|1945Q3|N|Y|Y|2431638|2431818|2431278|2431552|N|N|N|N|N| +2431644|AAAAAAAAMJKBFCAA|1945-07-07|546|2375|183|1945|6|7|7|3|1945|183|2375|Saturday|1945Q3|N|Y|N|2431638|2431818|2431279|2431553|N|N|N|N|N| +2431645|AAAAAAAANJKBFCAA|1945-07-08|546|2375|183|1945|0|7|8|3|1945|183|2375|Sunday|1945Q3|N|N|N|2431638|2431818|2431280|2431554|N|N|N|N|N| +2431646|AAAAAAAAOJKBFCAA|1945-07-09|546|2375|183|1945|1|7|9|3|1945|183|2375|Monday|1945Q3|N|N|N|2431638|2431818|2431281|2431555|N|N|N|N|N| +2431647|AAAAAAAAPJKBFCAA|1945-07-10|546|2376|183|1945|2|7|10|3|1945|183|2376|Tuesday|1945Q3|N|N|N|2431638|2431818|2431282|2431556|N|N|N|N|N| +2431648|AAAAAAAAAKKBFCAA|1945-07-11|546|2376|183|1945|3|7|11|3|1945|183|2376|Wednesday|1945Q3|N|N|N|2431638|2431818|2431283|2431557|N|N|N|N|N| +2431649|AAAAAAAABKKBFCAA|1945-07-12|546|2376|183|1945|4|7|12|3|1945|183|2376|Thursday|1945Q3|N|N|N|2431638|2431818|2431284|2431558|N|N|N|N|N| +2431650|AAAAAAAACKKBFCAA|1945-07-13|546|2376|183|1945|5|7|13|3|1945|183|2376|Friday|1945Q3|N|Y|N|2431638|2431818|2431285|2431559|N|N|N|N|N| +2431651|AAAAAAAADKKBFCAA|1945-07-14|546|2376|183|1945|6|7|14|3|1945|183|2376|Saturday|1945Q3|N|Y|N|2431638|2431818|2431286|2431560|N|N|N|N|N| +2431652|AAAAAAAAEKKBFCAA|1945-07-15|546|2376|183|1945|0|7|15|3|1945|183|2376|Sunday|1945Q3|N|N|N|2431638|2431818|2431287|2431561|N|N|N|N|N| +2431653|AAAAAAAAFKKBFCAA|1945-07-16|546|2376|183|1945|1|7|16|3|1945|183|2376|Monday|1945Q3|N|N|N|2431638|2431818|2431288|2431562|N|N|N|N|N| +2431654|AAAAAAAAGKKBFCAA|1945-07-17|546|2377|183|1945|2|7|17|3|1945|183|2377|Tuesday|1945Q3|N|N|N|2431638|2431818|2431289|2431563|N|N|N|N|N| +2431655|AAAAAAAAHKKBFCAA|1945-07-18|546|2377|183|1945|3|7|18|3|1945|183|2377|Wednesday|1945Q3|N|N|N|2431638|2431818|2431290|2431564|N|N|N|N|N| +2431656|AAAAAAAAIKKBFCAA|1945-07-19|546|2377|183|1945|4|7|19|3|1945|183|2377|Thursday|1945Q3|N|N|N|2431638|2431818|2431291|2431565|N|N|N|N|N| +2431657|AAAAAAAAJKKBFCAA|1945-07-20|546|2377|183|1945|5|7|20|3|1945|183|2377|Friday|1945Q3|N|Y|N|2431638|2431818|2431292|2431566|N|N|N|N|N| +2431658|AAAAAAAAKKKBFCAA|1945-07-21|546|2377|183|1945|6|7|21|3|1945|183|2377|Saturday|1945Q3|N|Y|N|2431638|2431818|2431293|2431567|N|N|N|N|N| +2431659|AAAAAAAALKKBFCAA|1945-07-22|546|2377|183|1945|0|7|22|3|1945|183|2377|Sunday|1945Q3|N|N|N|2431638|2431818|2431294|2431568|N|N|N|N|N| +2431660|AAAAAAAAMKKBFCAA|1945-07-23|546|2377|183|1945|1|7|23|3|1945|183|2377|Monday|1945Q3|N|N|N|2431638|2431818|2431295|2431569|N|N|N|N|N| +2431661|AAAAAAAANKKBFCAA|1945-07-24|546|2378|183|1945|2|7|24|3|1945|183|2378|Tuesday|1945Q3|N|N|N|2431638|2431818|2431296|2431570|N|N|N|N|N| +2431662|AAAAAAAAOKKBFCAA|1945-07-25|546|2378|183|1945|3|7|25|3|1945|183|2378|Wednesday|1945Q3|N|N|N|2431638|2431818|2431297|2431571|N|N|N|N|N| +2431663|AAAAAAAAPKKBFCAA|1945-07-26|546|2378|183|1945|4|7|26|3|1945|183|2378|Thursday|1945Q3|N|N|N|2431638|2431818|2431298|2431572|N|N|N|N|N| +2431664|AAAAAAAAALKBFCAA|1945-07-27|546|2378|183|1945|5|7|27|3|1945|183|2378|Friday|1945Q3|N|Y|N|2431638|2431818|2431299|2431573|N|N|N|N|N| +2431665|AAAAAAAABLKBFCAA|1945-07-28|546|2378|183|1945|6|7|28|3|1945|183|2378|Saturday|1945Q3|N|Y|N|2431638|2431818|2431300|2431574|N|N|N|N|N| +2431666|AAAAAAAACLKBFCAA|1945-07-29|546|2378|183|1945|0|7|29|3|1945|183|2378|Sunday|1945Q3|N|N|N|2431638|2431818|2431301|2431575|N|N|N|N|N| +2431667|AAAAAAAADLKBFCAA|1945-07-30|546|2378|183|1945|1|7|30|3|1945|183|2378|Monday|1945Q3|N|N|N|2431638|2431818|2431302|2431576|N|N|N|N|N| +2431668|AAAAAAAAELKBFCAA|1945-07-31|546|2379|183|1945|2|7|31|3|1945|183|2379|Tuesday|1945Q3|N|N|N|2431638|2431818|2431303|2431577|N|N|N|N|N| +2431669|AAAAAAAAFLKBFCAA|1945-08-01|547|2379|183|1945|3|8|1|3|1945|183|2379|Wednesday|1945Q3|N|N|N|2431669|2431880|2431304|2431578|N|N|N|N|N| +2431670|AAAAAAAAGLKBFCAA|1945-08-02|547|2379|183|1945|4|8|2|3|1945|183|2379|Thursday|1945Q3|N|N|N|2431669|2431880|2431305|2431579|N|N|N|N|N| +2431671|AAAAAAAAHLKBFCAA|1945-08-03|547|2379|183|1945|5|8|3|3|1945|183|2379|Friday|1945Q3|N|Y|N|2431669|2431880|2431306|2431580|N|N|N|N|N| +2431672|AAAAAAAAILKBFCAA|1945-08-04|547|2379|183|1945|6|8|4|3|1945|183|2379|Saturday|1945Q3|N|Y|N|2431669|2431880|2431307|2431581|N|N|N|N|N| +2431673|AAAAAAAAJLKBFCAA|1945-08-05|547|2379|183|1945|0|8|5|3|1945|183|2379|Sunday|1945Q3|N|N|N|2431669|2431880|2431308|2431582|N|N|N|N|N| +2431674|AAAAAAAAKLKBFCAA|1945-08-06|547|2379|183|1945|1|8|6|3|1945|183|2379|Monday|1945Q3|N|N|N|2431669|2431880|2431309|2431583|N|N|N|N|N| +2431675|AAAAAAAALLKBFCAA|1945-08-07|547|2380|183|1945|2|8|7|3|1945|183|2380|Tuesday|1945Q3|N|N|N|2431669|2431880|2431310|2431584|N|N|N|N|N| +2431676|AAAAAAAAMLKBFCAA|1945-08-08|547|2380|183|1945|3|8|8|3|1945|183|2380|Wednesday|1945Q3|N|N|N|2431669|2431880|2431311|2431585|N|N|N|N|N| +2431677|AAAAAAAANLKBFCAA|1945-08-09|547|2380|183|1945|4|8|9|3|1945|183|2380|Thursday|1945Q3|N|N|N|2431669|2431880|2431312|2431586|N|N|N|N|N| +2431678|AAAAAAAAOLKBFCAA|1945-08-10|547|2380|183|1945|5|8|10|3|1945|183|2380|Friday|1945Q3|N|Y|N|2431669|2431880|2431313|2431587|N|N|N|N|N| +2431679|AAAAAAAAPLKBFCAA|1945-08-11|547|2380|183|1945|6|8|11|3|1945|183|2380|Saturday|1945Q3|N|Y|N|2431669|2431880|2431314|2431588|N|N|N|N|N| +2431680|AAAAAAAAAMKBFCAA|1945-08-12|547|2380|183|1945|0|8|12|3|1945|183|2380|Sunday|1945Q3|N|N|N|2431669|2431880|2431315|2431589|N|N|N|N|N| +2431681|AAAAAAAABMKBFCAA|1945-08-13|547|2380|183|1945|1|8|13|3|1945|183|2380|Monday|1945Q3|N|N|N|2431669|2431880|2431316|2431590|N|N|N|N|N| +2431682|AAAAAAAACMKBFCAA|1945-08-14|547|2381|183|1945|2|8|14|3|1945|183|2381|Tuesday|1945Q3|N|N|N|2431669|2431880|2431317|2431591|N|N|N|N|N| +2431683|AAAAAAAADMKBFCAA|1945-08-15|547|2381|183|1945|3|8|15|3|1945|183|2381|Wednesday|1945Q3|N|N|N|2431669|2431880|2431318|2431592|N|N|N|N|N| +2431684|AAAAAAAAEMKBFCAA|1945-08-16|547|2381|183|1945|4|8|16|3|1945|183|2381|Thursday|1945Q3|N|N|N|2431669|2431880|2431319|2431593|N|N|N|N|N| +2431685|AAAAAAAAFMKBFCAA|1945-08-17|547|2381|183|1945|5|8|17|3|1945|183|2381|Friday|1945Q3|N|Y|N|2431669|2431880|2431320|2431594|N|N|N|N|N| +2431686|AAAAAAAAGMKBFCAA|1945-08-18|547|2381|183|1945|6|8|18|3|1945|183|2381|Saturday|1945Q3|N|Y|N|2431669|2431880|2431321|2431595|N|N|N|N|N| +2431687|AAAAAAAAHMKBFCAA|1945-08-19|547|2381|183|1945|0|8|19|3|1945|183|2381|Sunday|1945Q3|N|N|N|2431669|2431880|2431322|2431596|N|N|N|N|N| +2431688|AAAAAAAAIMKBFCAA|1945-08-20|547|2381|183|1945|1|8|20|3|1945|183|2381|Monday|1945Q3|N|N|N|2431669|2431880|2431323|2431597|N|N|N|N|N| +2431689|AAAAAAAAJMKBFCAA|1945-08-21|547|2382|183|1945|2|8|21|3|1945|183|2382|Tuesday|1945Q3|N|N|N|2431669|2431880|2431324|2431598|N|N|N|N|N| +2431690|AAAAAAAAKMKBFCAA|1945-08-22|547|2382|183|1945|3|8|22|3|1945|183|2382|Wednesday|1945Q3|N|N|N|2431669|2431880|2431325|2431599|N|N|N|N|N| +2431691|AAAAAAAALMKBFCAA|1945-08-23|547|2382|183|1945|4|8|23|3|1945|183|2382|Thursday|1945Q3|N|N|N|2431669|2431880|2431326|2431600|N|N|N|N|N| +2431692|AAAAAAAAMMKBFCAA|1945-08-24|547|2382|183|1945|5|8|24|3|1945|183|2382|Friday|1945Q3|N|Y|N|2431669|2431880|2431327|2431601|N|N|N|N|N| +2431693|AAAAAAAANMKBFCAA|1945-08-25|547|2382|183|1945|6|8|25|3|1945|183|2382|Saturday|1945Q3|N|Y|N|2431669|2431880|2431328|2431602|N|N|N|N|N| +2431694|AAAAAAAAOMKBFCAA|1945-08-26|547|2382|183|1945|0|8|26|3|1945|183|2382|Sunday|1945Q3|N|N|N|2431669|2431880|2431329|2431603|N|N|N|N|N| +2431695|AAAAAAAAPMKBFCAA|1945-08-27|547|2382|183|1945|1|8|27|3|1945|183|2382|Monday|1945Q3|N|N|N|2431669|2431880|2431330|2431604|N|N|N|N|N| +2431696|AAAAAAAAANKBFCAA|1945-08-28|547|2383|183|1945|2|8|28|3|1945|183|2383|Tuesday|1945Q3|N|N|N|2431669|2431880|2431331|2431605|N|N|N|N|N| +2431697|AAAAAAAABNKBFCAA|1945-08-29|547|2383|183|1945|3|8|29|3|1945|183|2383|Wednesday|1945Q3|N|N|N|2431669|2431880|2431332|2431606|N|N|N|N|N| +2431698|AAAAAAAACNKBFCAA|1945-08-30|547|2383|183|1945|4|8|30|3|1945|183|2383|Thursday|1945Q3|N|N|N|2431669|2431880|2431333|2431607|N|N|N|N|N| +2431699|AAAAAAAADNKBFCAA|1945-08-31|547|2383|183|1945|5|8|31|3|1945|183|2383|Friday|1945Q3|N|Y|N|2431669|2431880|2431334|2431608|N|N|N|N|N| +2431700|AAAAAAAAENKBFCAA|1945-09-01|548|2383|184|1945|6|9|1|3|1945|184|2383|Saturday|1945Q3|N|Y|N|2431700|2431942|2431335|2431609|N|N|N|N|N| +2431701|AAAAAAAAFNKBFCAA|1945-09-02|548|2383|184|1945|0|9|2|3|1945|184|2383|Sunday|1945Q3|N|N|N|2431700|2431942|2431336|2431610|N|N|N|N|N| +2431702|AAAAAAAAGNKBFCAA|1945-09-03|548|2383|184|1945|1|9|3|3|1945|184|2383|Monday|1945Q3|N|N|N|2431700|2431942|2431337|2431611|N|N|N|N|N| +2431703|AAAAAAAAHNKBFCAA|1945-09-04|548|2384|184|1945|2|9|4|3|1945|184|2384|Tuesday|1945Q3|N|N|N|2431700|2431942|2431338|2431612|N|N|N|N|N| +2431704|AAAAAAAAINKBFCAA|1945-09-05|548|2384|184|1945|3|9|5|3|1945|184|2384|Wednesday|1945Q3|N|N|N|2431700|2431942|2431339|2431613|N|N|N|N|N| +2431705|AAAAAAAAJNKBFCAA|1945-09-06|548|2384|184|1945|4|9|6|3|1945|184|2384|Thursday|1945Q3|N|N|N|2431700|2431942|2431340|2431614|N|N|N|N|N| +2431706|AAAAAAAAKNKBFCAA|1945-09-07|548|2384|184|1945|5|9|7|3|1945|184|2384|Friday|1945Q3|N|Y|N|2431700|2431942|2431341|2431615|N|N|N|N|N| +2431707|AAAAAAAALNKBFCAA|1945-09-08|548|2384|184|1945|6|9|8|3|1945|184|2384|Saturday|1945Q3|N|Y|N|2431700|2431942|2431342|2431616|N|N|N|N|N| +2431708|AAAAAAAAMNKBFCAA|1945-09-09|548|2384|184|1945|0|9|9|3|1945|184|2384|Sunday|1945Q3|N|N|N|2431700|2431942|2431343|2431617|N|N|N|N|N| +2431709|AAAAAAAANNKBFCAA|1945-09-10|548|2384|184|1945|1|9|10|3|1945|184|2384|Monday|1945Q3|N|N|N|2431700|2431942|2431344|2431618|N|N|N|N|N| +2431710|AAAAAAAAONKBFCAA|1945-09-11|548|2385|184|1945|2|9|11|3|1945|184|2385|Tuesday|1945Q3|N|N|N|2431700|2431942|2431345|2431619|N|N|N|N|N| +2431711|AAAAAAAAPNKBFCAA|1945-09-12|548|2385|184|1945|3|9|12|3|1945|184|2385|Wednesday|1945Q3|N|N|N|2431700|2431942|2431346|2431620|N|N|N|N|N| +2431712|AAAAAAAAAOKBFCAA|1945-09-13|548|2385|184|1945|4|9|13|3|1945|184|2385|Thursday|1945Q3|N|N|N|2431700|2431942|2431347|2431621|N|N|N|N|N| +2431713|AAAAAAAABOKBFCAA|1945-09-14|548|2385|184|1945|5|9|14|3|1945|184|2385|Friday|1945Q3|N|Y|N|2431700|2431942|2431348|2431622|N|N|N|N|N| +2431714|AAAAAAAACOKBFCAA|1945-09-15|548|2385|184|1945|6|9|15|3|1945|184|2385|Saturday|1945Q3|N|Y|N|2431700|2431942|2431349|2431623|N|N|N|N|N| +2431715|AAAAAAAADOKBFCAA|1945-09-16|548|2385|184|1945|0|9|16|3|1945|184|2385|Sunday|1945Q3|N|N|N|2431700|2431942|2431350|2431624|N|N|N|N|N| +2431716|AAAAAAAAEOKBFCAA|1945-09-17|548|2385|184|1945|1|9|17|3|1945|184|2385|Monday|1945Q3|N|N|N|2431700|2431942|2431351|2431625|N|N|N|N|N| +2431717|AAAAAAAAFOKBFCAA|1945-09-18|548|2386|184|1945|2|9|18|3|1945|184|2386|Tuesday|1945Q3|N|N|N|2431700|2431942|2431352|2431626|N|N|N|N|N| +2431718|AAAAAAAAGOKBFCAA|1945-09-19|548|2386|184|1945|3|9|19|3|1945|184|2386|Wednesday|1945Q3|N|N|N|2431700|2431942|2431353|2431627|N|N|N|N|N| +2431719|AAAAAAAAHOKBFCAA|1945-09-20|548|2386|184|1945|4|9|20|3|1945|184|2386|Thursday|1945Q3|N|N|N|2431700|2431942|2431354|2431628|N|N|N|N|N| +2431720|AAAAAAAAIOKBFCAA|1945-09-21|548|2386|184|1945|5|9|21|3|1945|184|2386|Friday|1945Q3|N|Y|N|2431700|2431942|2431355|2431629|N|N|N|N|N| +2431721|AAAAAAAAJOKBFCAA|1945-09-22|548|2386|184|1945|6|9|22|3|1945|184|2386|Saturday|1945Q3|N|Y|N|2431700|2431942|2431356|2431630|N|N|N|N|N| +2431722|AAAAAAAAKOKBFCAA|1945-09-23|548|2386|184|1945|0|9|23|3|1945|184|2386|Sunday|1945Q3|N|N|N|2431700|2431942|2431357|2431631|N|N|N|N|N| +2431723|AAAAAAAALOKBFCAA|1945-09-24|548|2386|184|1945|1|9|24|3|1945|184|2386|Monday|1945Q3|N|N|N|2431700|2431942|2431358|2431632|N|N|N|N|N| +2431724|AAAAAAAAMOKBFCAA|1945-09-25|548|2387|184|1945|2|9|25|3|1945|184|2387|Tuesday|1945Q3|N|N|N|2431700|2431942|2431359|2431633|N|N|N|N|N| +2431725|AAAAAAAANOKBFCAA|1945-09-26|548|2387|184|1945|3|9|26|3|1945|184|2387|Wednesday|1945Q3|N|N|N|2431700|2431942|2431360|2431634|N|N|N|N|N| +2431726|AAAAAAAAOOKBFCAA|1945-09-27|548|2387|184|1945|4|9|27|3|1945|184|2387|Thursday|1945Q3|N|N|N|2431700|2431942|2431361|2431635|N|N|N|N|N| +2431727|AAAAAAAAPOKBFCAA|1945-09-28|548|2387|184|1945|5|9|28|3|1945|184|2387|Friday|1945Q3|N|Y|N|2431700|2431942|2431362|2431636|N|N|N|N|N| +2431728|AAAAAAAAAPKBFCAA|1945-09-29|548|2387|184|1945|6|9|29|3|1945|184|2387|Saturday|1945Q3|N|Y|N|2431700|2431942|2431363|2431637|N|N|N|N|N| +2431729|AAAAAAAABPKBFCAA|1945-09-30|548|2387|184|1945|0|9|30|3|1945|184|2387|Sunday|1945Q3|N|N|N|2431700|2431942|2431364|2431638|N|N|N|N|N| +2431730|AAAAAAAACPKBFCAA|1945-10-01|549|2387|184|1945|1|10|1|3|1945|184|2387|Monday|1945Q3|N|N|N|2431730|2432002|2431365|2431638|N|N|N|N|N| +2431731|AAAAAAAADPKBFCAA|1945-10-02|549|2388|184|1945|2|10|2|4|1945|184|2388|Tuesday|1945Q4|N|N|N|2431730|2432002|2431366|2431639|N|N|N|N|N| +2431732|AAAAAAAAEPKBFCAA|1945-10-03|549|2388|184|1945|3|10|3|4|1945|184|2388|Wednesday|1945Q4|N|N|N|2431730|2432002|2431367|2431640|N|N|N|N|N| +2431733|AAAAAAAAFPKBFCAA|1945-10-04|549|2388|184|1945|4|10|4|4|1945|184|2388|Thursday|1945Q4|N|N|N|2431730|2432002|2431368|2431641|N|N|N|N|N| +2431734|AAAAAAAAGPKBFCAA|1945-10-05|549|2388|184|1945|5|10|5|4|1945|184|2388|Friday|1945Q4|N|Y|N|2431730|2432002|2431369|2431642|N|N|N|N|N| +2431735|AAAAAAAAHPKBFCAA|1945-10-06|549|2388|184|1945|6|10|6|4|1945|184|2388|Saturday|1945Q4|N|Y|N|2431730|2432002|2431370|2431643|N|N|N|N|N| +2431736|AAAAAAAAIPKBFCAA|1945-10-07|549|2388|184|1945|0|10|7|4|1945|184|2388|Sunday|1945Q4|N|N|N|2431730|2432002|2431371|2431644|N|N|N|N|N| +2431737|AAAAAAAAJPKBFCAA|1945-10-08|549|2388|184|1945|1|10|8|4|1945|184|2388|Monday|1945Q4|N|N|N|2431730|2432002|2431372|2431645|N|N|N|N|N| +2431738|AAAAAAAAKPKBFCAA|1945-10-09|549|2389|184|1945|2|10|9|4|1945|184|2389|Tuesday|1945Q4|N|N|N|2431730|2432002|2431373|2431646|N|N|N|N|N| +2431739|AAAAAAAALPKBFCAA|1945-10-10|549|2389|184|1945|3|10|10|4|1945|184|2389|Wednesday|1945Q4|N|N|N|2431730|2432002|2431374|2431647|N|N|N|N|N| +2431740|AAAAAAAAMPKBFCAA|1945-10-11|549|2389|184|1945|4|10|11|4|1945|184|2389|Thursday|1945Q4|N|N|N|2431730|2432002|2431375|2431648|N|N|N|N|N| +2431741|AAAAAAAANPKBFCAA|1945-10-12|549|2389|184|1945|5|10|12|4|1945|184|2389|Friday|1945Q4|N|Y|N|2431730|2432002|2431376|2431649|N|N|N|N|N| +2431742|AAAAAAAAOPKBFCAA|1945-10-13|549|2389|184|1945|6|10|13|4|1945|184|2389|Saturday|1945Q4|N|Y|N|2431730|2432002|2431377|2431650|N|N|N|N|N| +2431743|AAAAAAAAPPKBFCAA|1945-10-14|549|2389|184|1945|0|10|14|4|1945|184|2389|Sunday|1945Q4|N|N|N|2431730|2432002|2431378|2431651|N|N|N|N|N| +2431744|AAAAAAAAAALBFCAA|1945-10-15|549|2389|184|1945|1|10|15|4|1945|184|2389|Monday|1945Q4|N|N|N|2431730|2432002|2431379|2431652|N|N|N|N|N| +2431745|AAAAAAAABALBFCAA|1945-10-16|549|2390|184|1945|2|10|16|4|1945|184|2390|Tuesday|1945Q4|N|N|N|2431730|2432002|2431380|2431653|N|N|N|N|N| +2431746|AAAAAAAACALBFCAA|1945-10-17|549|2390|184|1945|3|10|17|4|1945|184|2390|Wednesday|1945Q4|N|N|N|2431730|2432002|2431381|2431654|N|N|N|N|N| +2431747|AAAAAAAADALBFCAA|1945-10-18|549|2390|184|1945|4|10|18|4|1945|184|2390|Thursday|1945Q4|N|N|N|2431730|2432002|2431382|2431655|N|N|N|N|N| +2431748|AAAAAAAAEALBFCAA|1945-10-19|549|2390|184|1945|5|10|19|4|1945|184|2390|Friday|1945Q4|N|Y|N|2431730|2432002|2431383|2431656|N|N|N|N|N| +2431749|AAAAAAAAFALBFCAA|1945-10-20|549|2390|184|1945|6|10|20|4|1945|184|2390|Saturday|1945Q4|N|Y|N|2431730|2432002|2431384|2431657|N|N|N|N|N| +2431750|AAAAAAAAGALBFCAA|1945-10-21|549|2390|184|1945|0|10|21|4|1945|184|2390|Sunday|1945Q4|N|N|N|2431730|2432002|2431385|2431658|N|N|N|N|N| +2431751|AAAAAAAAHALBFCAA|1945-10-22|549|2390|184|1945|1|10|22|4|1945|184|2390|Monday|1945Q4|N|N|N|2431730|2432002|2431386|2431659|N|N|N|N|N| +2431752|AAAAAAAAIALBFCAA|1945-10-23|549|2391|184|1945|2|10|23|4|1945|184|2391|Tuesday|1945Q4|N|N|N|2431730|2432002|2431387|2431660|N|N|N|N|N| +2431753|AAAAAAAAJALBFCAA|1945-10-24|549|2391|184|1945|3|10|24|4|1945|184|2391|Wednesday|1945Q4|N|N|N|2431730|2432002|2431388|2431661|N|N|N|N|N| +2431754|AAAAAAAAKALBFCAA|1945-10-25|549|2391|184|1945|4|10|25|4|1945|184|2391|Thursday|1945Q4|N|N|N|2431730|2432002|2431389|2431662|N|N|N|N|N| +2431755|AAAAAAAALALBFCAA|1945-10-26|549|2391|184|1945|5|10|26|4|1945|184|2391|Friday|1945Q4|N|Y|N|2431730|2432002|2431390|2431663|N|N|N|N|N| +2431756|AAAAAAAAMALBFCAA|1945-10-27|549|2391|184|1945|6|10|27|4|1945|184|2391|Saturday|1945Q4|N|Y|N|2431730|2432002|2431391|2431664|N|N|N|N|N| +2431757|AAAAAAAANALBFCAA|1945-10-28|549|2391|184|1945|0|10|28|4|1945|184|2391|Sunday|1945Q4|N|N|N|2431730|2432002|2431392|2431665|N|N|N|N|N| +2431758|AAAAAAAAOALBFCAA|1945-10-29|549|2391|184|1945|1|10|29|4|1945|184|2391|Monday|1945Q4|N|N|N|2431730|2432002|2431393|2431666|N|N|N|N|N| +2431759|AAAAAAAAPALBFCAA|1945-10-30|549|2392|184|1945|2|10|30|4|1945|184|2392|Tuesday|1945Q4|N|N|N|2431730|2432002|2431394|2431667|N|N|N|N|N| +2431760|AAAAAAAAABLBFCAA|1945-10-31|549|2392|184|1945|3|10|31|4|1945|184|2392|Wednesday|1945Q4|N|N|N|2431730|2432002|2431395|2431668|N|N|N|N|N| +2431761|AAAAAAAABBLBFCAA|1945-11-01|550|2392|184|1945|4|11|1|4|1945|184|2392|Thursday|1945Q4|N|N|N|2431761|2432064|2431396|2431669|N|N|N|N|N| +2431762|AAAAAAAACBLBFCAA|1945-11-02|550|2392|184|1945|5|11|2|4|1945|184|2392|Friday|1945Q4|N|Y|N|2431761|2432064|2431397|2431670|N|N|N|N|N| +2431763|AAAAAAAADBLBFCAA|1945-11-03|550|2392|184|1945|6|11|3|4|1945|184|2392|Saturday|1945Q4|N|Y|N|2431761|2432064|2431398|2431671|N|N|N|N|N| +2431764|AAAAAAAAEBLBFCAA|1945-11-04|550|2392|184|1945|0|11|4|4|1945|184|2392|Sunday|1945Q4|N|N|N|2431761|2432064|2431399|2431672|N|N|N|N|N| +2431765|AAAAAAAAFBLBFCAA|1945-11-05|550|2392|184|1945|1|11|5|4|1945|184|2392|Monday|1945Q4|N|N|N|2431761|2432064|2431400|2431673|N|N|N|N|N| +2431766|AAAAAAAAGBLBFCAA|1945-11-06|550|2393|184|1945|2|11|6|4|1945|184|2393|Tuesday|1945Q4|N|N|N|2431761|2432064|2431401|2431674|N|N|N|N|N| +2431767|AAAAAAAAHBLBFCAA|1945-11-07|550|2393|184|1945|3|11|7|4|1945|184|2393|Wednesday|1945Q4|N|N|N|2431761|2432064|2431402|2431675|N|N|N|N|N| +2431768|AAAAAAAAIBLBFCAA|1945-11-08|550|2393|184|1945|4|11|8|4|1945|184|2393|Thursday|1945Q4|N|N|N|2431761|2432064|2431403|2431676|N|N|N|N|N| +2431769|AAAAAAAAJBLBFCAA|1945-11-09|550|2393|184|1945|5|11|9|4|1945|184|2393|Friday|1945Q4|N|Y|N|2431761|2432064|2431404|2431677|N|N|N|N|N| +2431770|AAAAAAAAKBLBFCAA|1945-11-10|550|2393|184|1945|6|11|10|4|1945|184|2393|Saturday|1945Q4|N|Y|N|2431761|2432064|2431405|2431678|N|N|N|N|N| +2431771|AAAAAAAALBLBFCAA|1945-11-11|550|2393|184|1945|0|11|11|4|1945|184|2393|Sunday|1945Q4|N|N|N|2431761|2432064|2431406|2431679|N|N|N|N|N| +2431772|AAAAAAAAMBLBFCAA|1945-11-12|550|2393|184|1945|1|11|12|4|1945|184|2393|Monday|1945Q4|N|N|N|2431761|2432064|2431407|2431680|N|N|N|N|N| +2431773|AAAAAAAANBLBFCAA|1945-11-13|550|2394|184|1945|2|11|13|4|1945|184|2394|Tuesday|1945Q4|N|N|N|2431761|2432064|2431408|2431681|N|N|N|N|N| +2431774|AAAAAAAAOBLBFCAA|1945-11-14|550|2394|184|1945|3|11|14|4|1945|184|2394|Wednesday|1945Q4|N|N|N|2431761|2432064|2431409|2431682|N|N|N|N|N| +2431775|AAAAAAAAPBLBFCAA|1945-11-15|550|2394|184|1945|4|11|15|4|1945|184|2394|Thursday|1945Q4|N|N|N|2431761|2432064|2431410|2431683|N|N|N|N|N| +2431776|AAAAAAAAACLBFCAA|1945-11-16|550|2394|184|1945|5|11|16|4|1945|184|2394|Friday|1945Q4|N|Y|N|2431761|2432064|2431411|2431684|N|N|N|N|N| +2431777|AAAAAAAABCLBFCAA|1945-11-17|550|2394|184|1945|6|11|17|4|1945|184|2394|Saturday|1945Q4|N|Y|N|2431761|2432064|2431412|2431685|N|N|N|N|N| +2431778|AAAAAAAACCLBFCAA|1945-11-18|550|2394|184|1945|0|11|18|4|1945|184|2394|Sunday|1945Q4|N|N|N|2431761|2432064|2431413|2431686|N|N|N|N|N| +2431779|AAAAAAAADCLBFCAA|1945-11-19|550|2394|184|1945|1|11|19|4|1945|184|2394|Monday|1945Q4|N|N|N|2431761|2432064|2431414|2431687|N|N|N|N|N| +2431780|AAAAAAAAECLBFCAA|1945-11-20|550|2395|184|1945|2|11|20|4|1945|184|2395|Tuesday|1945Q4|N|N|N|2431761|2432064|2431415|2431688|N|N|N|N|N| +2431781|AAAAAAAAFCLBFCAA|1945-11-21|550|2395|184|1945|3|11|21|4|1945|184|2395|Wednesday|1945Q4|N|N|N|2431761|2432064|2431416|2431689|N|N|N|N|N| +2431782|AAAAAAAAGCLBFCAA|1945-11-22|550|2395|184|1945|4|11|22|4|1945|184|2395|Thursday|1945Q4|N|N|N|2431761|2432064|2431417|2431690|N|N|N|N|N| +2431783|AAAAAAAAHCLBFCAA|1945-11-23|550|2395|184|1945|5|11|23|4|1945|184|2395|Friday|1945Q4|N|Y|N|2431761|2432064|2431418|2431691|N|N|N|N|N| +2431784|AAAAAAAAICLBFCAA|1945-11-24|550|2395|184|1945|6|11|24|4|1945|184|2395|Saturday|1945Q4|N|Y|N|2431761|2432064|2431419|2431692|N|N|N|N|N| +2431785|AAAAAAAAJCLBFCAA|1945-11-25|550|2395|184|1945|0|11|25|4|1945|184|2395|Sunday|1945Q4|N|N|N|2431761|2432064|2431420|2431693|N|N|N|N|N| +2431786|AAAAAAAAKCLBFCAA|1945-11-26|550|2395|184|1945|1|11|26|4|1945|184|2395|Monday|1945Q4|N|N|N|2431761|2432064|2431421|2431694|N|N|N|N|N| +2431787|AAAAAAAALCLBFCAA|1945-11-27|550|2396|184|1945|2|11|27|4|1945|184|2396|Tuesday|1945Q4|N|N|N|2431761|2432064|2431422|2431695|N|N|N|N|N| +2431788|AAAAAAAAMCLBFCAA|1945-11-28|550|2396|184|1945|3|11|28|4|1945|184|2396|Wednesday|1945Q4|N|N|N|2431761|2432064|2431423|2431696|N|N|N|N|N| +2431789|AAAAAAAANCLBFCAA|1945-11-29|550|2396|184|1945|4|11|29|4|1945|184|2396|Thursday|1945Q4|N|N|N|2431761|2432064|2431424|2431697|N|N|N|N|N| +2431790|AAAAAAAAOCLBFCAA|1945-11-30|550|2396|184|1945|5|11|30|4|1945|184|2396|Friday|1945Q4|N|Y|N|2431761|2432064|2431425|2431698|N|N|N|N|N| +2431791|AAAAAAAAPCLBFCAA|1945-12-01|551|2396|185|1945|6|12|1|4|1945|185|2396|Saturday|1945Q4|N|Y|N|2431791|2432124|2431426|2431699|N|N|N|N|N| +2431792|AAAAAAAAADLBFCAA|1945-12-02|551|2396|185|1945|0|12|2|4|1945|185|2396|Sunday|1945Q4|N|N|N|2431791|2432124|2431427|2431700|N|N|N|N|N| +2431793|AAAAAAAABDLBFCAA|1945-12-03|551|2396|185|1945|1|12|3|4|1945|185|2396|Monday|1945Q4|N|N|N|2431791|2432124|2431428|2431701|N|N|N|N|N| +2431794|AAAAAAAACDLBFCAA|1945-12-04|551|2397|185|1945|2|12|4|4|1945|185|2397|Tuesday|1945Q4|N|N|N|2431791|2432124|2431429|2431702|N|N|N|N|N| +2431795|AAAAAAAADDLBFCAA|1945-12-05|551|2397|185|1945|3|12|5|4|1945|185|2397|Wednesday|1945Q4|N|N|N|2431791|2432124|2431430|2431703|N|N|N|N|N| +2431796|AAAAAAAAEDLBFCAA|1945-12-06|551|2397|185|1945|4|12|6|4|1945|185|2397|Thursday|1945Q4|N|N|N|2431791|2432124|2431431|2431704|N|N|N|N|N| +2431797|AAAAAAAAFDLBFCAA|1945-12-07|551|2397|185|1945|5|12|7|4|1945|185|2397|Friday|1945Q4|N|Y|N|2431791|2432124|2431432|2431705|N|N|N|N|N| +2431798|AAAAAAAAGDLBFCAA|1945-12-08|551|2397|185|1945|6|12|8|4|1945|185|2397|Saturday|1945Q4|N|Y|N|2431791|2432124|2431433|2431706|N|N|N|N|N| +2431799|AAAAAAAAHDLBFCAA|1945-12-09|551|2397|185|1945|0|12|9|4|1945|185|2397|Sunday|1945Q4|N|N|N|2431791|2432124|2431434|2431707|N|N|N|N|N| +2431800|AAAAAAAAIDLBFCAA|1945-12-10|551|2397|185|1945|1|12|10|4|1945|185|2397|Monday|1945Q4|N|N|N|2431791|2432124|2431435|2431708|N|N|N|N|N| +2431801|AAAAAAAAJDLBFCAA|1945-12-11|551|2398|185|1945|2|12|11|4|1945|185|2398|Tuesday|1945Q4|N|N|N|2431791|2432124|2431436|2431709|N|N|N|N|N| +2431802|AAAAAAAAKDLBFCAA|1945-12-12|551|2398|185|1945|3|12|12|4|1945|185|2398|Wednesday|1945Q4|N|N|N|2431791|2432124|2431437|2431710|N|N|N|N|N| +2431803|AAAAAAAALDLBFCAA|1945-12-13|551|2398|185|1945|4|12|13|4|1945|185|2398|Thursday|1945Q4|N|N|N|2431791|2432124|2431438|2431711|N|N|N|N|N| +2431804|AAAAAAAAMDLBFCAA|1945-12-14|551|2398|185|1945|5|12|14|4|1945|185|2398|Friday|1945Q4|N|Y|N|2431791|2432124|2431439|2431712|N|N|N|N|N| +2431805|AAAAAAAANDLBFCAA|1945-12-15|551|2398|185|1945|6|12|15|4|1945|185|2398|Saturday|1945Q4|N|Y|N|2431791|2432124|2431440|2431713|N|N|N|N|N| +2431806|AAAAAAAAODLBFCAA|1945-12-16|551|2398|185|1945|0|12|16|4|1945|185|2398|Sunday|1945Q4|N|N|N|2431791|2432124|2431441|2431714|N|N|N|N|N| +2431807|AAAAAAAAPDLBFCAA|1945-12-17|551|2398|185|1945|1|12|17|4|1945|185|2398|Monday|1945Q4|N|N|N|2431791|2432124|2431442|2431715|N|N|N|N|N| +2431808|AAAAAAAAAELBFCAA|1945-12-18|551|2399|185|1945|2|12|18|4|1945|185|2399|Tuesday|1945Q4|N|N|N|2431791|2432124|2431443|2431716|N|N|N|N|N| +2431809|AAAAAAAABELBFCAA|1945-12-19|551|2399|185|1945|3|12|19|4|1945|185|2399|Wednesday|1945Q4|N|N|N|2431791|2432124|2431444|2431717|N|N|N|N|N| +2431810|AAAAAAAACELBFCAA|1945-12-20|551|2399|185|1945|4|12|20|4|1945|185|2399|Thursday|1945Q4|N|N|N|2431791|2432124|2431445|2431718|N|N|N|N|N| +2431811|AAAAAAAADELBFCAA|1945-12-21|551|2399|185|1945|5|12|21|4|1945|185|2399|Friday|1945Q4|N|Y|N|2431791|2432124|2431446|2431719|N|N|N|N|N| +2431812|AAAAAAAAEELBFCAA|1945-12-22|551|2399|185|1945|6|12|22|4|1945|185|2399|Saturday|1945Q4|N|Y|N|2431791|2432124|2431447|2431720|N|N|N|N|N| +2431813|AAAAAAAAFELBFCAA|1945-12-23|551|2399|185|1945|0|12|23|4|1945|185|2399|Sunday|1945Q4|N|N|N|2431791|2432124|2431448|2431721|N|N|N|N|N| +2431814|AAAAAAAAGELBFCAA|1945-12-24|551|2399|185|1945|1|12|24|4|1945|185|2399|Monday|1945Q4|N|N|N|2431791|2432124|2431449|2431722|N|N|N|N|N| +2431815|AAAAAAAAHELBFCAA|1945-12-25|551|2400|185|1945|2|12|25|4|1945|185|2400|Tuesday|1945Q4|N|N|N|2431791|2432124|2431450|2431723|N|N|N|N|N| +2431816|AAAAAAAAIELBFCAA|1945-12-26|551|2400|185|1945|3|12|26|4|1945|185|2400|Wednesday|1945Q4|Y|N|N|2431791|2432124|2431451|2431724|N|N|N|N|N| +2431817|AAAAAAAAJELBFCAA|1945-12-27|551|2400|185|1945|4|12|27|4|1945|185|2400|Thursday|1945Q4|N|N|Y|2431791|2432124|2431452|2431725|N|N|N|N|N| +2431818|AAAAAAAAKELBFCAA|1945-12-28|551|2400|185|1945|5|12|28|4|1945|185|2400|Friday|1945Q4|N|Y|N|2431791|2432124|2431453|2431726|N|N|N|N|N| +2431819|AAAAAAAALELBFCAA|1945-12-29|551|2400|185|1945|6|12|29|4|1945|185|2400|Saturday|1945Q4|N|Y|N|2431791|2432124|2431454|2431727|N|N|N|N|N| +2431820|AAAAAAAAMELBFCAA|1945-12-30|551|2400|185|1945|0|12|30|4|1945|185|2400|Sunday|1945Q4|N|N|N|2431791|2432124|2431455|2431728|N|N|N|N|N| +2431821|AAAAAAAANELBFCAA|1945-12-31|551|2400|185|1945|1|12|31|4|1945|185|2400|Monday|1945Q4|N|N|N|2431791|2432124|2431456|2431729|N|N|N|N|N| +2431822|AAAAAAAAOELBFCAA|1946-01-01|552|2401|185|1946|2|1|1|1|1946|185|2401|Tuesday|1946Q1|Y|N|N|2431822|2431821|2431457|2431730|N|N|N|N|N| +2431823|AAAAAAAAPELBFCAA|1946-01-02|552|2401|185|1946|3|1|2|1|1946|185|2401|Wednesday|1946Q1|N|N|Y|2431822|2431821|2431458|2431731|N|N|N|N|N| +2431824|AAAAAAAAAFLBFCAA|1946-01-03|552|2401|185|1946|4|1|3|1|1946|185|2401|Thursday|1946Q1|N|N|N|2431822|2431821|2431459|2431732|N|N|N|N|N| +2431825|AAAAAAAABFLBFCAA|1946-01-04|552|2401|185|1946|5|1|4|1|1946|185|2401|Friday|1946Q1|N|Y|N|2431822|2431821|2431460|2431733|N|N|N|N|N| +2431826|AAAAAAAACFLBFCAA|1946-01-05|552|2401|185|1946|6|1|5|1|1946|185|2401|Saturday|1946Q1|N|Y|N|2431822|2431821|2431461|2431734|N|N|N|N|N| +2431827|AAAAAAAADFLBFCAA|1946-01-06|552|2401|185|1946|0|1|6|1|1946|185|2401|Sunday|1946Q1|N|N|N|2431822|2431821|2431462|2431735|N|N|N|N|N| +2431828|AAAAAAAAEFLBFCAA|1946-01-07|552|2401|185|1946|1|1|7|1|1946|185|2401|Monday|1946Q1|N|N|N|2431822|2431821|2431463|2431736|N|N|N|N|N| +2431829|AAAAAAAAFFLBFCAA|1946-01-08|552|2402|185|1946|2|1|8|1|1946|185|2402|Tuesday|1946Q1|N|N|N|2431822|2431821|2431464|2431737|N|N|N|N|N| +2431830|AAAAAAAAGFLBFCAA|1946-01-09|552|2402|185|1946|3|1|9|1|1946|185|2402|Wednesday|1946Q1|N|N|N|2431822|2431821|2431465|2431738|N|N|N|N|N| +2431831|AAAAAAAAHFLBFCAA|1946-01-10|552|2402|185|1946|4|1|10|1|1946|185|2402|Thursday|1946Q1|N|N|N|2431822|2431821|2431466|2431739|N|N|N|N|N| +2431832|AAAAAAAAIFLBFCAA|1946-01-11|552|2402|185|1946|5|1|11|1|1946|185|2402|Friday|1946Q1|N|Y|N|2431822|2431821|2431467|2431740|N|N|N|N|N| +2431833|AAAAAAAAJFLBFCAA|1946-01-12|552|2402|185|1946|6|1|12|1|1946|185|2402|Saturday|1946Q1|N|Y|N|2431822|2431821|2431468|2431741|N|N|N|N|N| +2431834|AAAAAAAAKFLBFCAA|1946-01-13|552|2402|185|1946|0|1|13|1|1946|185|2402|Sunday|1946Q1|N|N|N|2431822|2431821|2431469|2431742|N|N|N|N|N| +2431835|AAAAAAAALFLBFCAA|1946-01-14|552|2402|185|1946|1|1|14|1|1946|185|2402|Monday|1946Q1|N|N|N|2431822|2431821|2431470|2431743|N|N|N|N|N| +2431836|AAAAAAAAMFLBFCAA|1946-01-15|552|2403|185|1946|2|1|15|1|1946|185|2403|Tuesday|1946Q1|N|N|N|2431822|2431821|2431471|2431744|N|N|N|N|N| +2431837|AAAAAAAANFLBFCAA|1946-01-16|552|2403|185|1946|3|1|16|1|1946|185|2403|Wednesday|1946Q1|N|N|N|2431822|2431821|2431472|2431745|N|N|N|N|N| +2431838|AAAAAAAAOFLBFCAA|1946-01-17|552|2403|185|1946|4|1|17|1|1946|185|2403|Thursday|1946Q1|N|N|N|2431822|2431821|2431473|2431746|N|N|N|N|N| +2431839|AAAAAAAAPFLBFCAA|1946-01-18|552|2403|185|1946|5|1|18|1|1946|185|2403|Friday|1946Q1|N|Y|N|2431822|2431821|2431474|2431747|N|N|N|N|N| +2431840|AAAAAAAAAGLBFCAA|1946-01-19|552|2403|185|1946|6|1|19|1|1946|185|2403|Saturday|1946Q1|N|Y|N|2431822|2431821|2431475|2431748|N|N|N|N|N| +2431841|AAAAAAAABGLBFCAA|1946-01-20|552|2403|185|1946|0|1|20|1|1946|185|2403|Sunday|1946Q1|N|N|N|2431822|2431821|2431476|2431749|N|N|N|N|N| +2431842|AAAAAAAACGLBFCAA|1946-01-21|552|2403|185|1946|1|1|21|1|1946|185|2403|Monday|1946Q1|N|N|N|2431822|2431821|2431477|2431750|N|N|N|N|N| +2431843|AAAAAAAADGLBFCAA|1946-01-22|552|2404|185|1946|2|1|22|1|1946|185|2404|Tuesday|1946Q1|N|N|N|2431822|2431821|2431478|2431751|N|N|N|N|N| +2431844|AAAAAAAAEGLBFCAA|1946-01-23|552|2404|185|1946|3|1|23|1|1946|185|2404|Wednesday|1946Q1|N|N|N|2431822|2431821|2431479|2431752|N|N|N|N|N| +2431845|AAAAAAAAFGLBFCAA|1946-01-24|552|2404|185|1946|4|1|24|1|1946|185|2404|Thursday|1946Q1|N|N|N|2431822|2431821|2431480|2431753|N|N|N|N|N| +2431846|AAAAAAAAGGLBFCAA|1946-01-25|552|2404|185|1946|5|1|25|1|1946|185|2404|Friday|1946Q1|N|Y|N|2431822|2431821|2431481|2431754|N|N|N|N|N| +2431847|AAAAAAAAHGLBFCAA|1946-01-26|552|2404|185|1946|6|1|26|1|1946|185|2404|Saturday|1946Q1|N|Y|N|2431822|2431821|2431482|2431755|N|N|N|N|N| +2431848|AAAAAAAAIGLBFCAA|1946-01-27|552|2404|185|1946|0|1|27|1|1946|185|2404|Sunday|1946Q1|N|N|N|2431822|2431821|2431483|2431756|N|N|N|N|N| +2431849|AAAAAAAAJGLBFCAA|1946-01-28|552|2404|185|1946|1|1|28|1|1946|185|2404|Monday|1946Q1|N|N|N|2431822|2431821|2431484|2431757|N|N|N|N|N| +2431850|AAAAAAAAKGLBFCAA|1946-01-29|552|2405|185|1946|2|1|29|1|1946|185|2405|Tuesday|1946Q1|N|N|N|2431822|2431821|2431485|2431758|N|N|N|N|N| +2431851|AAAAAAAALGLBFCAA|1946-01-30|552|2405|185|1946|3|1|30|1|1946|185|2405|Wednesday|1946Q1|N|N|N|2431822|2431821|2431486|2431759|N|N|N|N|N| +2431852|AAAAAAAAMGLBFCAA|1946-01-31|552|2405|185|1946|4|1|31|1|1946|185|2405|Thursday|1946Q1|N|N|N|2431822|2431821|2431487|2431760|N|N|N|N|N| +2431853|AAAAAAAANGLBFCAA|1946-02-01|553|2405|185|1946|5|2|1|1|1946|185|2405|Friday|1946Q1|N|Y|N|2431853|2431883|2431488|2431761|N|N|N|N|N| +2431854|AAAAAAAAOGLBFCAA|1946-02-02|553|2405|185|1946|6|2|2|1|1946|185|2405|Saturday|1946Q1|N|Y|N|2431853|2431883|2431489|2431762|N|N|N|N|N| +2431855|AAAAAAAAPGLBFCAA|1946-02-03|553|2405|185|1946|0|2|3|1|1946|185|2405|Sunday|1946Q1|N|N|N|2431853|2431883|2431490|2431763|N|N|N|N|N| +2431856|AAAAAAAAAHLBFCAA|1946-02-04|553|2405|185|1946|1|2|4|1|1946|185|2405|Monday|1946Q1|N|N|N|2431853|2431883|2431491|2431764|N|N|N|N|N| +2431857|AAAAAAAABHLBFCAA|1946-02-05|553|2406|185|1946|2|2|5|1|1946|185|2406|Tuesday|1946Q1|N|N|N|2431853|2431883|2431492|2431765|N|N|N|N|N| +2431858|AAAAAAAACHLBFCAA|1946-02-06|553|2406|185|1946|3|2|6|1|1946|185|2406|Wednesday|1946Q1|N|N|N|2431853|2431883|2431493|2431766|N|N|N|N|N| +2431859|AAAAAAAADHLBFCAA|1946-02-07|553|2406|185|1946|4|2|7|1|1946|185|2406|Thursday|1946Q1|N|N|N|2431853|2431883|2431494|2431767|N|N|N|N|N| +2431860|AAAAAAAAEHLBFCAA|1946-02-08|553|2406|185|1946|5|2|8|1|1946|185|2406|Friday|1946Q1|N|Y|N|2431853|2431883|2431495|2431768|N|N|N|N|N| +2431861|AAAAAAAAFHLBFCAA|1946-02-09|553|2406|185|1946|6|2|9|1|1946|185|2406|Saturday|1946Q1|N|Y|N|2431853|2431883|2431496|2431769|N|N|N|N|N| +2431862|AAAAAAAAGHLBFCAA|1946-02-10|553|2406|185|1946|0|2|10|1|1946|185|2406|Sunday|1946Q1|N|N|N|2431853|2431883|2431497|2431770|N|N|N|N|N| +2431863|AAAAAAAAHHLBFCAA|1946-02-11|553|2406|185|1946|1|2|11|1|1946|185|2406|Monday|1946Q1|N|N|N|2431853|2431883|2431498|2431771|N|N|N|N|N| +2431864|AAAAAAAAIHLBFCAA|1946-02-12|553|2407|185|1946|2|2|12|1|1946|185|2407|Tuesday|1946Q1|N|N|N|2431853|2431883|2431499|2431772|N|N|N|N|N| +2431865|AAAAAAAAJHLBFCAA|1946-02-13|553|2407|185|1946|3|2|13|1|1946|185|2407|Wednesday|1946Q1|N|N|N|2431853|2431883|2431500|2431773|N|N|N|N|N| +2431866|AAAAAAAAKHLBFCAA|1946-02-14|553|2407|185|1946|4|2|14|1|1946|185|2407|Thursday|1946Q1|N|N|N|2431853|2431883|2431501|2431774|N|N|N|N|N| +2431867|AAAAAAAALHLBFCAA|1946-02-15|553|2407|185|1946|5|2|15|1|1946|185|2407|Friday|1946Q1|N|Y|N|2431853|2431883|2431502|2431775|N|N|N|N|N| +2431868|AAAAAAAAMHLBFCAA|1946-02-16|553|2407|185|1946|6|2|16|1|1946|185|2407|Saturday|1946Q1|N|Y|N|2431853|2431883|2431503|2431776|N|N|N|N|N| +2431869|AAAAAAAANHLBFCAA|1946-02-17|553|2407|185|1946|0|2|17|1|1946|185|2407|Sunday|1946Q1|N|N|N|2431853|2431883|2431504|2431777|N|N|N|N|N| +2431870|AAAAAAAAOHLBFCAA|1946-02-18|553|2407|185|1946|1|2|18|1|1946|185|2407|Monday|1946Q1|N|N|N|2431853|2431883|2431505|2431778|N|N|N|N|N| +2431871|AAAAAAAAPHLBFCAA|1946-02-19|553|2408|185|1946|2|2|19|1|1946|185|2408|Tuesday|1946Q1|N|N|N|2431853|2431883|2431506|2431779|N|N|N|N|N| +2431872|AAAAAAAAAILBFCAA|1946-02-20|553|2408|185|1946|3|2|20|1|1946|185|2408|Wednesday|1946Q1|N|N|N|2431853|2431883|2431507|2431780|N|N|N|N|N| +2431873|AAAAAAAABILBFCAA|1946-02-21|553|2408|185|1946|4|2|21|1|1946|185|2408|Thursday|1946Q1|N|N|N|2431853|2431883|2431508|2431781|N|N|N|N|N| +2431874|AAAAAAAACILBFCAA|1946-02-22|553|2408|185|1946|5|2|22|1|1946|185|2408|Friday|1946Q1|N|Y|N|2431853|2431883|2431509|2431782|N|N|N|N|N| +2431875|AAAAAAAADILBFCAA|1946-02-23|553|2408|185|1946|6|2|23|1|1946|185|2408|Saturday|1946Q1|N|Y|N|2431853|2431883|2431510|2431783|N|N|N|N|N| +2431876|AAAAAAAAEILBFCAA|1946-02-24|553|2408|185|1946|0|2|24|1|1946|185|2408|Sunday|1946Q1|N|N|N|2431853|2431883|2431511|2431784|N|N|N|N|N| +2431877|AAAAAAAAFILBFCAA|1946-02-25|553|2408|185|1946|1|2|25|1|1946|185|2408|Monday|1946Q1|N|N|N|2431853|2431883|2431512|2431785|N|N|N|N|N| +2431878|AAAAAAAAGILBFCAA|1946-02-26|553|2409|185|1946|2|2|26|1|1946|185|2409|Tuesday|1946Q1|N|N|N|2431853|2431883|2431513|2431786|N|N|N|N|N| +2431879|AAAAAAAAHILBFCAA|1946-02-27|553|2409|185|1946|3|2|27|1|1946|185|2409|Wednesday|1946Q1|N|N|N|2431853|2431883|2431514|2431787|N|N|N|N|N| +2431880|AAAAAAAAIILBFCAA|1946-02-28|553|2409|185|1946|4|2|28|1|1946|185|2409|Thursday|1946Q1|N|N|N|2431853|2431883|2431515|2431788|N|N|N|N|N| +2431881|AAAAAAAAJILBFCAA|1946-03-01|554|2409|186|1946|5|3|1|1|1946|186|2409|Friday|1946Q1|N|Y|N|2431881|2431939|2431516|2431789|N|N|N|N|N| +2431882|AAAAAAAAKILBFCAA|1946-03-02|554|2409|186|1946|6|3|2|1|1946|186|2409|Saturday|1946Q1|N|Y|N|2431881|2431939|2431517|2431790|N|N|N|N|N| +2431883|AAAAAAAALILBFCAA|1946-03-03|554|2409|186|1946|0|3|3|1|1946|186|2409|Sunday|1946Q1|N|N|N|2431881|2431939|2431518|2431791|N|N|N|N|N| +2431884|AAAAAAAAMILBFCAA|1946-03-04|554|2409|186|1946|1|3|4|1|1946|186|2409|Monday|1946Q1|N|N|N|2431881|2431939|2431519|2431792|N|N|N|N|N| +2431885|AAAAAAAANILBFCAA|1946-03-05|554|2410|186|1946|2|3|5|1|1946|186|2410|Tuesday|1946Q1|N|N|N|2431881|2431939|2431520|2431793|N|N|N|N|N| +2431886|AAAAAAAAOILBFCAA|1946-03-06|554|2410|186|1946|3|3|6|1|1946|186|2410|Wednesday|1946Q1|N|N|N|2431881|2431939|2431521|2431794|N|N|N|N|N| +2431887|AAAAAAAAPILBFCAA|1946-03-07|554|2410|186|1946|4|3|7|1|1946|186|2410|Thursday|1946Q1|N|N|N|2431881|2431939|2431522|2431795|N|N|N|N|N| +2431888|AAAAAAAAAJLBFCAA|1946-03-08|554|2410|186|1946|5|3|8|1|1946|186|2410|Friday|1946Q1|N|Y|N|2431881|2431939|2431523|2431796|N|N|N|N|N| +2431889|AAAAAAAABJLBFCAA|1946-03-09|554|2410|186|1946|6|3|9|1|1946|186|2410|Saturday|1946Q1|N|Y|N|2431881|2431939|2431524|2431797|N|N|N|N|N| +2431890|AAAAAAAACJLBFCAA|1946-03-10|554|2410|186|1946|0|3|10|1|1946|186|2410|Sunday|1946Q1|N|N|N|2431881|2431939|2431525|2431798|N|N|N|N|N| +2431891|AAAAAAAADJLBFCAA|1946-03-11|554|2410|186|1946|1|3|11|1|1946|186|2410|Monday|1946Q1|N|N|N|2431881|2431939|2431526|2431799|N|N|N|N|N| +2431892|AAAAAAAAEJLBFCAA|1946-03-12|554|2411|186|1946|2|3|12|1|1946|186|2411|Tuesday|1946Q1|N|N|N|2431881|2431939|2431527|2431800|N|N|N|N|N| +2431893|AAAAAAAAFJLBFCAA|1946-03-13|554|2411|186|1946|3|3|13|1|1946|186|2411|Wednesday|1946Q1|N|N|N|2431881|2431939|2431528|2431801|N|N|N|N|N| +2431894|AAAAAAAAGJLBFCAA|1946-03-14|554|2411|186|1946|4|3|14|1|1946|186|2411|Thursday|1946Q1|N|N|N|2431881|2431939|2431529|2431802|N|N|N|N|N| +2431895|AAAAAAAAHJLBFCAA|1946-03-15|554|2411|186|1946|5|3|15|1|1946|186|2411|Friday|1946Q1|N|Y|N|2431881|2431939|2431530|2431803|N|N|N|N|N| +2431896|AAAAAAAAIJLBFCAA|1946-03-16|554|2411|186|1946|6|3|16|1|1946|186|2411|Saturday|1946Q1|N|Y|N|2431881|2431939|2431531|2431804|N|N|N|N|N| +2431897|AAAAAAAAJJLBFCAA|1946-03-17|554|2411|186|1946|0|3|17|1|1946|186|2411|Sunday|1946Q1|N|N|N|2431881|2431939|2431532|2431805|N|N|N|N|N| +2431898|AAAAAAAAKJLBFCAA|1946-03-18|554|2411|186|1946|1|3|18|1|1946|186|2411|Monday|1946Q1|N|N|N|2431881|2431939|2431533|2431806|N|N|N|N|N| +2431899|AAAAAAAALJLBFCAA|1946-03-19|554|2412|186|1946|2|3|19|1|1946|186|2412|Tuesday|1946Q1|N|N|N|2431881|2431939|2431534|2431807|N|N|N|N|N| +2431900|AAAAAAAAMJLBFCAA|1946-03-20|554|2412|186|1946|3|3|20|1|1946|186|2412|Wednesday|1946Q1|N|N|N|2431881|2431939|2431535|2431808|N|N|N|N|N| +2431901|AAAAAAAANJLBFCAA|1946-03-21|554|2412|186|1946|4|3|21|1|1946|186|2412|Thursday|1946Q1|N|N|N|2431881|2431939|2431536|2431809|N|N|N|N|N| +2431902|AAAAAAAAOJLBFCAA|1946-03-22|554|2412|186|1946|5|3|22|1|1946|186|2412|Friday|1946Q1|N|Y|N|2431881|2431939|2431537|2431810|N|N|N|N|N| +2431903|AAAAAAAAPJLBFCAA|1946-03-23|554|2412|186|1946|6|3|23|1|1946|186|2412|Saturday|1946Q1|N|Y|N|2431881|2431939|2431538|2431811|N|N|N|N|N| +2431904|AAAAAAAAAKLBFCAA|1946-03-24|554|2412|186|1946|0|3|24|1|1946|186|2412|Sunday|1946Q1|N|N|N|2431881|2431939|2431539|2431812|N|N|N|N|N| +2431905|AAAAAAAABKLBFCAA|1946-03-25|554|2412|186|1946|1|3|25|1|1946|186|2412|Monday|1946Q1|N|N|N|2431881|2431939|2431540|2431813|N|N|N|N|N| +2431906|AAAAAAAACKLBFCAA|1946-03-26|554|2413|186|1946|2|3|26|1|1946|186|2413|Tuesday|1946Q1|N|N|N|2431881|2431939|2431541|2431814|N|N|N|N|N| +2431907|AAAAAAAADKLBFCAA|1946-03-27|554|2413|186|1946|3|3|27|1|1946|186|2413|Wednesday|1946Q1|N|N|N|2431881|2431939|2431542|2431815|N|N|N|N|N| +2431908|AAAAAAAAEKLBFCAA|1946-03-28|554|2413|186|1946|4|3|28|1|1946|186|2413|Thursday|1946Q1|N|N|N|2431881|2431939|2431543|2431816|N|N|N|N|N| +2431909|AAAAAAAAFKLBFCAA|1946-03-29|554|2413|186|1946|5|3|29|1|1946|186|2413|Friday|1946Q1|N|Y|N|2431881|2431939|2431544|2431817|N|N|N|N|N| +2431910|AAAAAAAAGKLBFCAA|1946-03-30|554|2413|186|1946|6|3|30|1|1946|186|2413|Saturday|1946Q1|N|Y|N|2431881|2431939|2431545|2431818|N|N|N|N|N| +2431911|AAAAAAAAHKLBFCAA|1946-03-31|554|2413|186|1946|0|3|31|1|1946|186|2413|Sunday|1946Q1|N|N|N|2431881|2431939|2431546|2431819|N|N|N|N|N| +2431912|AAAAAAAAIKLBFCAA|1946-04-01|555|2413|186|1946|1|4|1|1|1946|186|2413|Monday|1946Q1|N|N|N|2431912|2432001|2431547|2431822|N|N|N|N|N| +2431913|AAAAAAAAJKLBFCAA|1946-04-02|555|2414|186|1946|2|4|2|2|1946|186|2414|Tuesday|1946Q2|N|N|N|2431912|2432001|2431548|2431823|N|N|N|N|N| +2431914|AAAAAAAAKKLBFCAA|1946-04-03|555|2414|186|1946|3|4|3|2|1946|186|2414|Wednesday|1946Q2|N|N|N|2431912|2432001|2431549|2431824|N|N|N|N|N| +2431915|AAAAAAAALKLBFCAA|1946-04-04|555|2414|186|1946|4|4|4|2|1946|186|2414|Thursday|1946Q2|N|N|N|2431912|2432001|2431550|2431825|N|N|N|N|N| +2431916|AAAAAAAAMKLBFCAA|1946-04-05|555|2414|186|1946|5|4|5|2|1946|186|2414|Friday|1946Q2|N|Y|N|2431912|2432001|2431551|2431826|N|N|N|N|N| +2431917|AAAAAAAANKLBFCAA|1946-04-06|555|2414|186|1946|6|4|6|2|1946|186|2414|Saturday|1946Q2|N|Y|N|2431912|2432001|2431552|2431827|N|N|N|N|N| +2431918|AAAAAAAAOKLBFCAA|1946-04-07|555|2414|186|1946|0|4|7|2|1946|186|2414|Sunday|1946Q2|N|N|N|2431912|2432001|2431553|2431828|N|N|N|N|N| +2431919|AAAAAAAAPKLBFCAA|1946-04-08|555|2414|186|1946|1|4|8|2|1946|186|2414|Monday|1946Q2|N|N|N|2431912|2432001|2431554|2431829|N|N|N|N|N| +2431920|AAAAAAAAALLBFCAA|1946-04-09|555|2415|186|1946|2|4|9|2|1946|186|2415|Tuesday|1946Q2|N|N|N|2431912|2432001|2431555|2431830|N|N|N|N|N| +2431921|AAAAAAAABLLBFCAA|1946-04-10|555|2415|186|1946|3|4|10|2|1946|186|2415|Wednesday|1946Q2|N|N|N|2431912|2432001|2431556|2431831|N|N|N|N|N| +2431922|AAAAAAAACLLBFCAA|1946-04-11|555|2415|186|1946|4|4|11|2|1946|186|2415|Thursday|1946Q2|N|N|N|2431912|2432001|2431557|2431832|N|N|N|N|N| +2431923|AAAAAAAADLLBFCAA|1946-04-12|555|2415|186|1946|5|4|12|2|1946|186|2415|Friday|1946Q2|N|Y|N|2431912|2432001|2431558|2431833|N|N|N|N|N| +2431924|AAAAAAAAELLBFCAA|1946-04-13|555|2415|186|1946|6|4|13|2|1946|186|2415|Saturday|1946Q2|N|Y|N|2431912|2432001|2431559|2431834|N|N|N|N|N| +2431925|AAAAAAAAFLLBFCAA|1946-04-14|555|2415|186|1946|0|4|14|2|1946|186|2415|Sunday|1946Q2|N|N|N|2431912|2432001|2431560|2431835|N|N|N|N|N| +2431926|AAAAAAAAGLLBFCAA|1946-04-15|555|2415|186|1946|1|4|15|2|1946|186|2415|Monday|1946Q2|N|N|N|2431912|2432001|2431561|2431836|N|N|N|N|N| +2431927|AAAAAAAAHLLBFCAA|1946-04-16|555|2416|186|1946|2|4|16|2|1946|186|2416|Tuesday|1946Q2|N|N|N|2431912|2432001|2431562|2431837|N|N|N|N|N| +2431928|AAAAAAAAILLBFCAA|1946-04-17|555|2416|186|1946|3|4|17|2|1946|186|2416|Wednesday|1946Q2|N|N|N|2431912|2432001|2431563|2431838|N|N|N|N|N| +2431929|AAAAAAAAJLLBFCAA|1946-04-18|555|2416|186|1946|4|4|18|2|1946|186|2416|Thursday|1946Q2|N|N|N|2431912|2432001|2431564|2431839|N|N|N|N|N| +2431930|AAAAAAAAKLLBFCAA|1946-04-19|555|2416|186|1946|5|4|19|2|1946|186|2416|Friday|1946Q2|N|Y|N|2431912|2432001|2431565|2431840|N|N|N|N|N| +2431931|AAAAAAAALLLBFCAA|1946-04-20|555|2416|186|1946|6|4|20|2|1946|186|2416|Saturday|1946Q2|N|Y|N|2431912|2432001|2431566|2431841|N|N|N|N|N| +2431932|AAAAAAAAMLLBFCAA|1946-04-21|555|2416|186|1946|0|4|21|2|1946|186|2416|Sunday|1946Q2|N|N|N|2431912|2432001|2431567|2431842|N|N|N|N|N| +2431933|AAAAAAAANLLBFCAA|1946-04-22|555|2416|186|1946|1|4|22|2|1946|186|2416|Monday|1946Q2|N|N|N|2431912|2432001|2431568|2431843|N|N|N|N|N| +2431934|AAAAAAAAOLLBFCAA|1946-04-23|555|2417|186|1946|2|4|23|2|1946|186|2417|Tuesday|1946Q2|N|N|N|2431912|2432001|2431569|2431844|N|N|N|N|N| +2431935|AAAAAAAAPLLBFCAA|1946-04-24|555|2417|186|1946|3|4|24|2|1946|186|2417|Wednesday|1946Q2|N|N|N|2431912|2432001|2431570|2431845|N|N|N|N|N| +2431936|AAAAAAAAAMLBFCAA|1946-04-25|555|2417|186|1946|4|4|25|2|1946|186|2417|Thursday|1946Q2|N|N|N|2431912|2432001|2431571|2431846|N|N|N|N|N| +2431937|AAAAAAAABMLBFCAA|1946-04-26|555|2417|186|1946|5|4|26|2|1946|186|2417|Friday|1946Q2|N|Y|N|2431912|2432001|2431572|2431847|N|N|N|N|N| +2431938|AAAAAAAACMLBFCAA|1946-04-27|555|2417|186|1946|6|4|27|2|1946|186|2417|Saturday|1946Q2|N|Y|N|2431912|2432001|2431573|2431848|N|N|N|N|N| +2431939|AAAAAAAADMLBFCAA|1946-04-28|555|2417|186|1946|0|4|28|2|1946|186|2417|Sunday|1946Q2|N|N|N|2431912|2432001|2431574|2431849|N|N|N|N|N| +2431940|AAAAAAAAEMLBFCAA|1946-04-29|555|2417|186|1946|1|4|29|2|1946|186|2417|Monday|1946Q2|N|N|N|2431912|2432001|2431575|2431850|N|N|N|N|N| +2431941|AAAAAAAAFMLBFCAA|1946-04-30|555|2418|186|1946|2|4|30|2|1946|186|2418|Tuesday|1946Q2|N|N|N|2431912|2432001|2431576|2431851|N|N|N|N|N| +2431942|AAAAAAAAGMLBFCAA|1946-05-01|556|2418|186|1946|3|5|1|2|1946|186|2418|Wednesday|1946Q2|N|N|N|2431942|2432061|2431577|2431852|N|N|N|N|N| +2431943|AAAAAAAAHMLBFCAA|1946-05-02|556|2418|186|1946|4|5|2|2|1946|186|2418|Thursday|1946Q2|N|N|N|2431942|2432061|2431578|2431853|N|N|N|N|N| +2431944|AAAAAAAAIMLBFCAA|1946-05-03|556|2418|186|1946|5|5|3|2|1946|186|2418|Friday|1946Q2|N|Y|N|2431942|2432061|2431579|2431854|N|N|N|N|N| +2431945|AAAAAAAAJMLBFCAA|1946-05-04|556|2418|186|1946|6|5|4|2|1946|186|2418|Saturday|1946Q2|N|Y|N|2431942|2432061|2431580|2431855|N|N|N|N|N| +2431946|AAAAAAAAKMLBFCAA|1946-05-05|556|2418|186|1946|0|5|5|2|1946|186|2418|Sunday|1946Q2|N|N|N|2431942|2432061|2431581|2431856|N|N|N|N|N| +2431947|AAAAAAAALMLBFCAA|1946-05-06|556|2418|186|1946|1|5|6|2|1946|186|2418|Monday|1946Q2|N|N|N|2431942|2432061|2431582|2431857|N|N|N|N|N| +2431948|AAAAAAAAMMLBFCAA|1946-05-07|556|2419|186|1946|2|5|7|2|1946|186|2419|Tuesday|1946Q2|N|N|N|2431942|2432061|2431583|2431858|N|N|N|N|N| +2431949|AAAAAAAANMLBFCAA|1946-05-08|556|2419|186|1946|3|5|8|2|1946|186|2419|Wednesday|1946Q2|N|N|N|2431942|2432061|2431584|2431859|N|N|N|N|N| +2431950|AAAAAAAAOMLBFCAA|1946-05-09|556|2419|186|1946|4|5|9|2|1946|186|2419|Thursday|1946Q2|N|N|N|2431942|2432061|2431585|2431860|N|N|N|N|N| +2431951|AAAAAAAAPMLBFCAA|1946-05-10|556|2419|186|1946|5|5|10|2|1946|186|2419|Friday|1946Q2|N|Y|N|2431942|2432061|2431586|2431861|N|N|N|N|N| +2431952|AAAAAAAAANLBFCAA|1946-05-11|556|2419|186|1946|6|5|11|2|1946|186|2419|Saturday|1946Q2|N|Y|N|2431942|2432061|2431587|2431862|N|N|N|N|N| +2431953|AAAAAAAABNLBFCAA|1946-05-12|556|2419|186|1946|0|5|12|2|1946|186|2419|Sunday|1946Q2|N|N|N|2431942|2432061|2431588|2431863|N|N|N|N|N| +2431954|AAAAAAAACNLBFCAA|1946-05-13|556|2419|186|1946|1|5|13|2|1946|186|2419|Monday|1946Q2|N|N|N|2431942|2432061|2431589|2431864|N|N|N|N|N| +2431955|AAAAAAAADNLBFCAA|1946-05-14|556|2420|186|1946|2|5|14|2|1946|186|2420|Tuesday|1946Q2|N|N|N|2431942|2432061|2431590|2431865|N|N|N|N|N| +2431956|AAAAAAAAENLBFCAA|1946-05-15|556|2420|186|1946|3|5|15|2|1946|186|2420|Wednesday|1946Q2|N|N|N|2431942|2432061|2431591|2431866|N|N|N|N|N| +2431957|AAAAAAAAFNLBFCAA|1946-05-16|556|2420|186|1946|4|5|16|2|1946|186|2420|Thursday|1946Q2|N|N|N|2431942|2432061|2431592|2431867|N|N|N|N|N| +2431958|AAAAAAAAGNLBFCAA|1946-05-17|556|2420|186|1946|5|5|17|2|1946|186|2420|Friday|1946Q2|N|Y|N|2431942|2432061|2431593|2431868|N|N|N|N|N| +2431959|AAAAAAAAHNLBFCAA|1946-05-18|556|2420|186|1946|6|5|18|2|1946|186|2420|Saturday|1946Q2|N|Y|N|2431942|2432061|2431594|2431869|N|N|N|N|N| +2431960|AAAAAAAAINLBFCAA|1946-05-19|556|2420|186|1946|0|5|19|2|1946|186|2420|Sunday|1946Q2|N|N|N|2431942|2432061|2431595|2431870|N|N|N|N|N| +2431961|AAAAAAAAJNLBFCAA|1946-05-20|556|2420|186|1946|1|5|20|2|1946|186|2420|Monday|1946Q2|N|N|N|2431942|2432061|2431596|2431871|N|N|N|N|N| +2431962|AAAAAAAAKNLBFCAA|1946-05-21|556|2421|186|1946|2|5|21|2|1946|186|2421|Tuesday|1946Q2|N|N|N|2431942|2432061|2431597|2431872|N|N|N|N|N| +2431963|AAAAAAAALNLBFCAA|1946-05-22|556|2421|186|1946|3|5|22|2|1946|186|2421|Wednesday|1946Q2|N|N|N|2431942|2432061|2431598|2431873|N|N|N|N|N| +2431964|AAAAAAAAMNLBFCAA|1946-05-23|556|2421|186|1946|4|5|23|2|1946|186|2421|Thursday|1946Q2|N|N|N|2431942|2432061|2431599|2431874|N|N|N|N|N| +2431965|AAAAAAAANNLBFCAA|1946-05-24|556|2421|186|1946|5|5|24|2|1946|186|2421|Friday|1946Q2|N|Y|N|2431942|2432061|2431600|2431875|N|N|N|N|N| +2431966|AAAAAAAAONLBFCAA|1946-05-25|556|2421|186|1946|6|5|25|2|1946|186|2421|Saturday|1946Q2|N|Y|N|2431942|2432061|2431601|2431876|N|N|N|N|N| +2431967|AAAAAAAAPNLBFCAA|1946-05-26|556|2421|186|1946|0|5|26|2|1946|186|2421|Sunday|1946Q2|N|N|N|2431942|2432061|2431602|2431877|N|N|N|N|N| +2431968|AAAAAAAAAOLBFCAA|1946-05-27|556|2421|186|1946|1|5|27|2|1946|186|2421|Monday|1946Q2|N|N|N|2431942|2432061|2431603|2431878|N|N|N|N|N| +2431969|AAAAAAAABOLBFCAA|1946-05-28|556|2422|186|1946|2|5|28|2|1946|186|2422|Tuesday|1946Q2|N|N|N|2431942|2432061|2431604|2431879|N|N|N|N|N| +2431970|AAAAAAAACOLBFCAA|1946-05-29|556|2422|186|1946|3|5|29|2|1946|186|2422|Wednesday|1946Q2|N|N|N|2431942|2432061|2431605|2431880|N|N|N|N|N| +2431971|AAAAAAAADOLBFCAA|1946-05-30|556|2422|186|1946|4|5|30|2|1946|186|2422|Thursday|1946Q2|N|N|N|2431942|2432061|2431606|2431881|N|N|N|N|N| +2431972|AAAAAAAAEOLBFCAA|1946-05-31|556|2422|186|1946|5|5|31|2|1946|186|2422|Friday|1946Q2|N|Y|N|2431942|2432061|2431607|2431882|N|N|N|N|N| +2431973|AAAAAAAAFOLBFCAA|1946-06-01|557|2422|187|1946|6|6|1|2|1946|187|2422|Saturday|1946Q2|N|Y|N|2431973|2432123|2431608|2431883|N|N|N|N|N| +2431974|AAAAAAAAGOLBFCAA|1946-06-02|557|2422|187|1946|0|6|2|2|1946|187|2422|Sunday|1946Q2|N|N|N|2431973|2432123|2431609|2431884|N|N|N|N|N| +2431975|AAAAAAAAHOLBFCAA|1946-06-03|557|2422|187|1946|1|6|3|2|1946|187|2422|Monday|1946Q2|N|N|N|2431973|2432123|2431610|2431885|N|N|N|N|N| +2431976|AAAAAAAAIOLBFCAA|1946-06-04|557|2423|187|1946|2|6|4|2|1946|187|2423|Tuesday|1946Q2|N|N|N|2431973|2432123|2431611|2431886|N|N|N|N|N| +2431977|AAAAAAAAJOLBFCAA|1946-06-05|557|2423|187|1946|3|6|5|2|1946|187|2423|Wednesday|1946Q2|N|N|N|2431973|2432123|2431612|2431887|N|N|N|N|N| +2431978|AAAAAAAAKOLBFCAA|1946-06-06|557|2423|187|1946|4|6|6|2|1946|187|2423|Thursday|1946Q2|N|N|N|2431973|2432123|2431613|2431888|N|N|N|N|N| +2431979|AAAAAAAALOLBFCAA|1946-06-07|557|2423|187|1946|5|6|7|2|1946|187|2423|Friday|1946Q2|N|Y|N|2431973|2432123|2431614|2431889|N|N|N|N|N| +2431980|AAAAAAAAMOLBFCAA|1946-06-08|557|2423|187|1946|6|6|8|2|1946|187|2423|Saturday|1946Q2|N|Y|N|2431973|2432123|2431615|2431890|N|N|N|N|N| +2431981|AAAAAAAANOLBFCAA|1946-06-09|557|2423|187|1946|0|6|9|2|1946|187|2423|Sunday|1946Q2|N|N|N|2431973|2432123|2431616|2431891|N|N|N|N|N| +2431982|AAAAAAAAOOLBFCAA|1946-06-10|557|2423|187|1946|1|6|10|2|1946|187|2423|Monday|1946Q2|N|N|N|2431973|2432123|2431617|2431892|N|N|N|N|N| +2431983|AAAAAAAAPOLBFCAA|1946-06-11|557|2424|187|1946|2|6|11|2|1946|187|2424|Tuesday|1946Q2|N|N|N|2431973|2432123|2431618|2431893|N|N|N|N|N| +2431984|AAAAAAAAAPLBFCAA|1946-06-12|557|2424|187|1946|3|6|12|2|1946|187|2424|Wednesday|1946Q2|N|N|N|2431973|2432123|2431619|2431894|N|N|N|N|N| +2431985|AAAAAAAABPLBFCAA|1946-06-13|557|2424|187|1946|4|6|13|2|1946|187|2424|Thursday|1946Q2|N|N|N|2431973|2432123|2431620|2431895|N|N|N|N|N| +2431986|AAAAAAAACPLBFCAA|1946-06-14|557|2424|187|1946|5|6|14|2|1946|187|2424|Friday|1946Q2|N|Y|N|2431973|2432123|2431621|2431896|N|N|N|N|N| +2431987|AAAAAAAADPLBFCAA|1946-06-15|557|2424|187|1946|6|6|15|2|1946|187|2424|Saturday|1946Q2|N|Y|N|2431973|2432123|2431622|2431897|N|N|N|N|N| +2431988|AAAAAAAAEPLBFCAA|1946-06-16|557|2424|187|1946|0|6|16|2|1946|187|2424|Sunday|1946Q2|N|N|N|2431973|2432123|2431623|2431898|N|N|N|N|N| +2431989|AAAAAAAAFPLBFCAA|1946-06-17|557|2424|187|1946|1|6|17|2|1946|187|2424|Monday|1946Q2|N|N|N|2431973|2432123|2431624|2431899|N|N|N|N|N| +2431990|AAAAAAAAGPLBFCAA|1946-06-18|557|2425|187|1946|2|6|18|2|1946|187|2425|Tuesday|1946Q2|N|N|N|2431973|2432123|2431625|2431900|N|N|N|N|N| +2431991|AAAAAAAAHPLBFCAA|1946-06-19|557|2425|187|1946|3|6|19|2|1946|187|2425|Wednesday|1946Q2|N|N|N|2431973|2432123|2431626|2431901|N|N|N|N|N| +2431992|AAAAAAAAIPLBFCAA|1946-06-20|557|2425|187|1946|4|6|20|2|1946|187|2425|Thursday|1946Q2|N|N|N|2431973|2432123|2431627|2431902|N|N|N|N|N| +2431993|AAAAAAAAJPLBFCAA|1946-06-21|557|2425|187|1946|5|6|21|2|1946|187|2425|Friday|1946Q2|N|Y|N|2431973|2432123|2431628|2431903|N|N|N|N|N| +2431994|AAAAAAAAKPLBFCAA|1946-06-22|557|2425|187|1946|6|6|22|2|1946|187|2425|Saturday|1946Q2|N|Y|N|2431973|2432123|2431629|2431904|N|N|N|N|N| +2431995|AAAAAAAALPLBFCAA|1946-06-23|557|2425|187|1946|0|6|23|2|1946|187|2425|Sunday|1946Q2|N|N|N|2431973|2432123|2431630|2431905|N|N|N|N|N| +2431996|AAAAAAAAMPLBFCAA|1946-06-24|557|2425|187|1946|1|6|24|2|1946|187|2425|Monday|1946Q2|N|N|N|2431973|2432123|2431631|2431906|N|N|N|N|N| +2431997|AAAAAAAANPLBFCAA|1946-06-25|557|2426|187|1946|2|6|25|2|1946|187|2426|Tuesday|1946Q2|N|N|N|2431973|2432123|2431632|2431907|N|N|N|N|N| +2431998|AAAAAAAAOPLBFCAA|1946-06-26|557|2426|187|1946|3|6|26|2|1946|187|2426|Wednesday|1946Q2|N|N|N|2431973|2432123|2431633|2431908|N|N|N|N|N| +2431999|AAAAAAAAPPLBFCAA|1946-06-27|557|2426|187|1946|4|6|27|2|1946|187|2426|Thursday|1946Q2|N|N|N|2431973|2432123|2431634|2431909|N|N|N|N|N| +2432000|AAAAAAAAAAMBFCAA|1946-06-28|557|2426|187|1946|5|6|28|2|1946|187|2426|Friday|1946Q2|N|Y|N|2431973|2432123|2431635|2431910|N|N|N|N|N| +2432001|AAAAAAAABAMBFCAA|1946-06-29|557|2426|187|1946|6|6|29|2|1946|187|2426|Saturday|1946Q2|N|Y|N|2431973|2432123|2431636|2431911|N|N|N|N|N| +2432002|AAAAAAAACAMBFCAA|1946-06-30|557|2426|187|1946|0|6|30|2|1946|187|2426|Sunday|1946Q2|N|N|N|2431973|2432123|2431637|2431912|N|N|N|N|N| +2432003|AAAAAAAADAMBFCAA|1946-07-01|558|2426|187|1946|1|7|1|2|1946|187|2426|Monday|1946Q2|N|N|N|2432003|2432183|2431638|2431912|N|N|N|N|N| +2432004|AAAAAAAAEAMBFCAA|1946-07-02|558|2427|187|1946|2|7|2|3|1946|187|2427|Tuesday|1946Q3|N|N|N|2432003|2432183|2431639|2431913|N|N|N|N|N| +2432005|AAAAAAAAFAMBFCAA|1946-07-03|558|2427|187|1946|3|7|3|3|1946|187|2427|Wednesday|1946Q3|N|N|N|2432003|2432183|2431640|2431914|N|N|N|N|N| +2432006|AAAAAAAAGAMBFCAA|1946-07-04|558|2427|187|1946|4|7|4|3|1946|187|2427|Thursday|1946Q3|N|N|N|2432003|2432183|2431641|2431915|N|N|N|N|N| +2432007|AAAAAAAAHAMBFCAA|1946-07-05|558|2427|187|1946|5|7|5|3|1946|187|2427|Friday|1946Q3|Y|Y|N|2432003|2432183|2431642|2431916|N|N|N|N|N| +2432008|AAAAAAAAIAMBFCAA|1946-07-06|558|2427|187|1946|6|7|6|3|1946|187|2427|Saturday|1946Q3|N|Y|Y|2432003|2432183|2431643|2431917|N|N|N|N|N| +2432009|AAAAAAAAJAMBFCAA|1946-07-07|558|2427|187|1946|0|7|7|3|1946|187|2427|Sunday|1946Q3|N|N|N|2432003|2432183|2431644|2431918|N|N|N|N|N| +2432010|AAAAAAAAKAMBFCAA|1946-07-08|558|2427|187|1946|1|7|8|3|1946|187|2427|Monday|1946Q3|N|N|N|2432003|2432183|2431645|2431919|N|N|N|N|N| +2432011|AAAAAAAALAMBFCAA|1946-07-09|558|2428|187|1946|2|7|9|3|1946|187|2428|Tuesday|1946Q3|N|N|N|2432003|2432183|2431646|2431920|N|N|N|N|N| +2432012|AAAAAAAAMAMBFCAA|1946-07-10|558|2428|187|1946|3|7|10|3|1946|187|2428|Wednesday|1946Q3|N|N|N|2432003|2432183|2431647|2431921|N|N|N|N|N| +2432013|AAAAAAAANAMBFCAA|1946-07-11|558|2428|187|1946|4|7|11|3|1946|187|2428|Thursday|1946Q3|N|N|N|2432003|2432183|2431648|2431922|N|N|N|N|N| +2432014|AAAAAAAAOAMBFCAA|1946-07-12|558|2428|187|1946|5|7|12|3|1946|187|2428|Friday|1946Q3|N|Y|N|2432003|2432183|2431649|2431923|N|N|N|N|N| +2432015|AAAAAAAAPAMBFCAA|1946-07-13|558|2428|187|1946|6|7|13|3|1946|187|2428|Saturday|1946Q3|N|Y|N|2432003|2432183|2431650|2431924|N|N|N|N|N| +2432016|AAAAAAAAABMBFCAA|1946-07-14|558|2428|187|1946|0|7|14|3|1946|187|2428|Sunday|1946Q3|N|N|N|2432003|2432183|2431651|2431925|N|N|N|N|N| +2432017|AAAAAAAABBMBFCAA|1946-07-15|558|2428|187|1946|1|7|15|3|1946|187|2428|Monday|1946Q3|N|N|N|2432003|2432183|2431652|2431926|N|N|N|N|N| +2432018|AAAAAAAACBMBFCAA|1946-07-16|558|2429|187|1946|2|7|16|3|1946|187|2429|Tuesday|1946Q3|N|N|N|2432003|2432183|2431653|2431927|N|N|N|N|N| +2432019|AAAAAAAADBMBFCAA|1946-07-17|558|2429|187|1946|3|7|17|3|1946|187|2429|Wednesday|1946Q3|N|N|N|2432003|2432183|2431654|2431928|N|N|N|N|N| +2432020|AAAAAAAAEBMBFCAA|1946-07-18|558|2429|187|1946|4|7|18|3|1946|187|2429|Thursday|1946Q3|N|N|N|2432003|2432183|2431655|2431929|N|N|N|N|N| +2432021|AAAAAAAAFBMBFCAA|1946-07-19|558|2429|187|1946|5|7|19|3|1946|187|2429|Friday|1946Q3|N|Y|N|2432003|2432183|2431656|2431930|N|N|N|N|N| +2432022|AAAAAAAAGBMBFCAA|1946-07-20|558|2429|187|1946|6|7|20|3|1946|187|2429|Saturday|1946Q3|N|Y|N|2432003|2432183|2431657|2431931|N|N|N|N|N| +2432023|AAAAAAAAHBMBFCAA|1946-07-21|558|2429|187|1946|0|7|21|3|1946|187|2429|Sunday|1946Q3|N|N|N|2432003|2432183|2431658|2431932|N|N|N|N|N| +2432024|AAAAAAAAIBMBFCAA|1946-07-22|558|2429|187|1946|1|7|22|3|1946|187|2429|Monday|1946Q3|N|N|N|2432003|2432183|2431659|2431933|N|N|N|N|N| +2432025|AAAAAAAAJBMBFCAA|1946-07-23|558|2430|187|1946|2|7|23|3|1946|187|2430|Tuesday|1946Q3|N|N|N|2432003|2432183|2431660|2431934|N|N|N|N|N| +2432026|AAAAAAAAKBMBFCAA|1946-07-24|558|2430|187|1946|3|7|24|3|1946|187|2430|Wednesday|1946Q3|N|N|N|2432003|2432183|2431661|2431935|N|N|N|N|N| +2432027|AAAAAAAALBMBFCAA|1946-07-25|558|2430|187|1946|4|7|25|3|1946|187|2430|Thursday|1946Q3|N|N|N|2432003|2432183|2431662|2431936|N|N|N|N|N| +2432028|AAAAAAAAMBMBFCAA|1946-07-26|558|2430|187|1946|5|7|26|3|1946|187|2430|Friday|1946Q3|N|Y|N|2432003|2432183|2431663|2431937|N|N|N|N|N| +2432029|AAAAAAAANBMBFCAA|1946-07-27|558|2430|187|1946|6|7|27|3|1946|187|2430|Saturday|1946Q3|N|Y|N|2432003|2432183|2431664|2431938|N|N|N|N|N| +2432030|AAAAAAAAOBMBFCAA|1946-07-28|558|2430|187|1946|0|7|28|3|1946|187|2430|Sunday|1946Q3|N|N|N|2432003|2432183|2431665|2431939|N|N|N|N|N| +2432031|AAAAAAAAPBMBFCAA|1946-07-29|558|2430|187|1946|1|7|29|3|1946|187|2430|Monday|1946Q3|N|N|N|2432003|2432183|2431666|2431940|N|N|N|N|N| +2432032|AAAAAAAAACMBFCAA|1946-07-30|558|2431|187|1946|2|7|30|3|1946|187|2431|Tuesday|1946Q3|N|N|N|2432003|2432183|2431667|2431941|N|N|N|N|N| +2432033|AAAAAAAABCMBFCAA|1946-07-31|558|2431|187|1946|3|7|31|3|1946|187|2431|Wednesday|1946Q3|N|N|N|2432003|2432183|2431668|2431942|N|N|N|N|N| +2432034|AAAAAAAACCMBFCAA|1946-08-01|559|2431|187|1946|4|8|1|3|1946|187|2431|Thursday|1946Q3|N|N|N|2432034|2432245|2431669|2431943|N|N|N|N|N| +2432035|AAAAAAAADCMBFCAA|1946-08-02|559|2431|187|1946|5|8|2|3|1946|187|2431|Friday|1946Q3|N|Y|N|2432034|2432245|2431670|2431944|N|N|N|N|N| +2432036|AAAAAAAAECMBFCAA|1946-08-03|559|2431|187|1946|6|8|3|3|1946|187|2431|Saturday|1946Q3|N|Y|N|2432034|2432245|2431671|2431945|N|N|N|N|N| +2432037|AAAAAAAAFCMBFCAA|1946-08-04|559|2431|187|1946|0|8|4|3|1946|187|2431|Sunday|1946Q3|N|N|N|2432034|2432245|2431672|2431946|N|N|N|N|N| +2432038|AAAAAAAAGCMBFCAA|1946-08-05|559|2431|187|1946|1|8|5|3|1946|187|2431|Monday|1946Q3|N|N|N|2432034|2432245|2431673|2431947|N|N|N|N|N| +2432039|AAAAAAAAHCMBFCAA|1946-08-06|559|2432|187|1946|2|8|6|3|1946|187|2432|Tuesday|1946Q3|N|N|N|2432034|2432245|2431674|2431948|N|N|N|N|N| +2432040|AAAAAAAAICMBFCAA|1946-08-07|559|2432|187|1946|3|8|7|3|1946|187|2432|Wednesday|1946Q3|N|N|N|2432034|2432245|2431675|2431949|N|N|N|N|N| +2432041|AAAAAAAAJCMBFCAA|1946-08-08|559|2432|187|1946|4|8|8|3|1946|187|2432|Thursday|1946Q3|N|N|N|2432034|2432245|2431676|2431950|N|N|N|N|N| +2432042|AAAAAAAAKCMBFCAA|1946-08-09|559|2432|187|1946|5|8|9|3|1946|187|2432|Friday|1946Q3|N|Y|N|2432034|2432245|2431677|2431951|N|N|N|N|N| +2432043|AAAAAAAALCMBFCAA|1946-08-10|559|2432|187|1946|6|8|10|3|1946|187|2432|Saturday|1946Q3|N|Y|N|2432034|2432245|2431678|2431952|N|N|N|N|N| +2432044|AAAAAAAAMCMBFCAA|1946-08-11|559|2432|187|1946|0|8|11|3|1946|187|2432|Sunday|1946Q3|N|N|N|2432034|2432245|2431679|2431953|N|N|N|N|N| +2432045|AAAAAAAANCMBFCAA|1946-08-12|559|2432|187|1946|1|8|12|3|1946|187|2432|Monday|1946Q3|N|N|N|2432034|2432245|2431680|2431954|N|N|N|N|N| +2432046|AAAAAAAAOCMBFCAA|1946-08-13|559|2433|187|1946|2|8|13|3|1946|187|2433|Tuesday|1946Q3|N|N|N|2432034|2432245|2431681|2431955|N|N|N|N|N| +2432047|AAAAAAAAPCMBFCAA|1946-08-14|559|2433|187|1946|3|8|14|3|1946|187|2433|Wednesday|1946Q3|N|N|N|2432034|2432245|2431682|2431956|N|N|N|N|N| +2432048|AAAAAAAAADMBFCAA|1946-08-15|559|2433|187|1946|4|8|15|3|1946|187|2433|Thursday|1946Q3|N|N|N|2432034|2432245|2431683|2431957|N|N|N|N|N| +2432049|AAAAAAAABDMBFCAA|1946-08-16|559|2433|187|1946|5|8|16|3|1946|187|2433|Friday|1946Q3|N|Y|N|2432034|2432245|2431684|2431958|N|N|N|N|N| +2432050|AAAAAAAACDMBFCAA|1946-08-17|559|2433|187|1946|6|8|17|3|1946|187|2433|Saturday|1946Q3|N|Y|N|2432034|2432245|2431685|2431959|N|N|N|N|N| +2432051|AAAAAAAADDMBFCAA|1946-08-18|559|2433|187|1946|0|8|18|3|1946|187|2433|Sunday|1946Q3|N|N|N|2432034|2432245|2431686|2431960|N|N|N|N|N| +2432052|AAAAAAAAEDMBFCAA|1946-08-19|559|2433|187|1946|1|8|19|3|1946|187|2433|Monday|1946Q3|N|N|N|2432034|2432245|2431687|2431961|N|N|N|N|N| +2432053|AAAAAAAAFDMBFCAA|1946-08-20|559|2434|187|1946|2|8|20|3|1946|187|2434|Tuesday|1946Q3|N|N|N|2432034|2432245|2431688|2431962|N|N|N|N|N| +2432054|AAAAAAAAGDMBFCAA|1946-08-21|559|2434|187|1946|3|8|21|3|1946|187|2434|Wednesday|1946Q3|N|N|N|2432034|2432245|2431689|2431963|N|N|N|N|N| +2432055|AAAAAAAAHDMBFCAA|1946-08-22|559|2434|187|1946|4|8|22|3|1946|187|2434|Thursday|1946Q3|N|N|N|2432034|2432245|2431690|2431964|N|N|N|N|N| +2432056|AAAAAAAAIDMBFCAA|1946-08-23|559|2434|187|1946|5|8|23|3|1946|187|2434|Friday|1946Q3|N|Y|N|2432034|2432245|2431691|2431965|N|N|N|N|N| +2432057|AAAAAAAAJDMBFCAA|1946-08-24|559|2434|187|1946|6|8|24|3|1946|187|2434|Saturday|1946Q3|N|Y|N|2432034|2432245|2431692|2431966|N|N|N|N|N| +2432058|AAAAAAAAKDMBFCAA|1946-08-25|559|2434|187|1946|0|8|25|3|1946|187|2434|Sunday|1946Q3|N|N|N|2432034|2432245|2431693|2431967|N|N|N|N|N| +2432059|AAAAAAAALDMBFCAA|1946-08-26|559|2434|187|1946|1|8|26|3|1946|187|2434|Monday|1946Q3|N|N|N|2432034|2432245|2431694|2431968|N|N|N|N|N| +2432060|AAAAAAAAMDMBFCAA|1946-08-27|559|2435|187|1946|2|8|27|3|1946|187|2435|Tuesday|1946Q3|N|N|N|2432034|2432245|2431695|2431969|N|N|N|N|N| +2432061|AAAAAAAANDMBFCAA|1946-08-28|559|2435|187|1946|3|8|28|3|1946|187|2435|Wednesday|1946Q3|N|N|N|2432034|2432245|2431696|2431970|N|N|N|N|N| +2432062|AAAAAAAAODMBFCAA|1946-08-29|559|2435|187|1946|4|8|29|3|1946|187|2435|Thursday|1946Q3|N|N|N|2432034|2432245|2431697|2431971|N|N|N|N|N| +2432063|AAAAAAAAPDMBFCAA|1946-08-30|559|2435|187|1946|5|8|30|3|1946|187|2435|Friday|1946Q3|N|Y|N|2432034|2432245|2431698|2431972|N|N|N|N|N| +2432064|AAAAAAAAAEMBFCAA|1946-08-31|559|2435|187|1946|6|8|31|3|1946|187|2435|Saturday|1946Q3|N|Y|N|2432034|2432245|2431699|2431973|N|N|N|N|N| +2432065|AAAAAAAABEMBFCAA|1946-09-01|560|2435|188|1946|0|9|1|3|1946|188|2435|Sunday|1946Q3|N|N|N|2432065|2432307|2431700|2431974|N|N|N|N|N| +2432066|AAAAAAAACEMBFCAA|1946-09-02|560|2435|188|1946|1|9|2|3|1946|188|2435|Monday|1946Q3|N|N|N|2432065|2432307|2431701|2431975|N|N|N|N|N| +2432067|AAAAAAAADEMBFCAA|1946-09-03|560|2436|188|1946|2|9|3|3|1946|188|2436|Tuesday|1946Q3|N|N|N|2432065|2432307|2431702|2431976|N|N|N|N|N| +2432068|AAAAAAAAEEMBFCAA|1946-09-04|560|2436|188|1946|3|9|4|3|1946|188|2436|Wednesday|1946Q3|N|N|N|2432065|2432307|2431703|2431977|N|N|N|N|N| +2432069|AAAAAAAAFEMBFCAA|1946-09-05|560|2436|188|1946|4|9|5|3|1946|188|2436|Thursday|1946Q3|N|N|N|2432065|2432307|2431704|2431978|N|N|N|N|N| +2432070|AAAAAAAAGEMBFCAA|1946-09-06|560|2436|188|1946|5|9|6|3|1946|188|2436|Friday|1946Q3|N|Y|N|2432065|2432307|2431705|2431979|N|N|N|N|N| +2432071|AAAAAAAAHEMBFCAA|1946-09-07|560|2436|188|1946|6|9|7|3|1946|188|2436|Saturday|1946Q3|N|Y|N|2432065|2432307|2431706|2431980|N|N|N|N|N| +2432072|AAAAAAAAIEMBFCAA|1946-09-08|560|2436|188|1946|0|9|8|3|1946|188|2436|Sunday|1946Q3|N|N|N|2432065|2432307|2431707|2431981|N|N|N|N|N| +2432073|AAAAAAAAJEMBFCAA|1946-09-09|560|2436|188|1946|1|9|9|3|1946|188|2436|Monday|1946Q3|N|N|N|2432065|2432307|2431708|2431982|N|N|N|N|N| +2432074|AAAAAAAAKEMBFCAA|1946-09-10|560|2437|188|1946|2|9|10|3|1946|188|2437|Tuesday|1946Q3|N|N|N|2432065|2432307|2431709|2431983|N|N|N|N|N| +2432075|AAAAAAAALEMBFCAA|1946-09-11|560|2437|188|1946|3|9|11|3|1946|188|2437|Wednesday|1946Q3|N|N|N|2432065|2432307|2431710|2431984|N|N|N|N|N| +2432076|AAAAAAAAMEMBFCAA|1946-09-12|560|2437|188|1946|4|9|12|3|1946|188|2437|Thursday|1946Q3|N|N|N|2432065|2432307|2431711|2431985|N|N|N|N|N| +2432077|AAAAAAAANEMBFCAA|1946-09-13|560|2437|188|1946|5|9|13|3|1946|188|2437|Friday|1946Q3|N|Y|N|2432065|2432307|2431712|2431986|N|N|N|N|N| +2432078|AAAAAAAAOEMBFCAA|1946-09-14|560|2437|188|1946|6|9|14|3|1946|188|2437|Saturday|1946Q3|N|Y|N|2432065|2432307|2431713|2431987|N|N|N|N|N| +2432079|AAAAAAAAPEMBFCAA|1946-09-15|560|2437|188|1946|0|9|15|3|1946|188|2437|Sunday|1946Q3|N|N|N|2432065|2432307|2431714|2431988|N|N|N|N|N| +2432080|AAAAAAAAAFMBFCAA|1946-09-16|560|2437|188|1946|1|9|16|3|1946|188|2437|Monday|1946Q3|N|N|N|2432065|2432307|2431715|2431989|N|N|N|N|N| +2432081|AAAAAAAABFMBFCAA|1946-09-17|560|2438|188|1946|2|9|17|3|1946|188|2438|Tuesday|1946Q3|N|N|N|2432065|2432307|2431716|2431990|N|N|N|N|N| +2432082|AAAAAAAACFMBFCAA|1946-09-18|560|2438|188|1946|3|9|18|3|1946|188|2438|Wednesday|1946Q3|N|N|N|2432065|2432307|2431717|2431991|N|N|N|N|N| +2432083|AAAAAAAADFMBFCAA|1946-09-19|560|2438|188|1946|4|9|19|3|1946|188|2438|Thursday|1946Q3|N|N|N|2432065|2432307|2431718|2431992|N|N|N|N|N| +2432084|AAAAAAAAEFMBFCAA|1946-09-20|560|2438|188|1946|5|9|20|3|1946|188|2438|Friday|1946Q3|N|Y|N|2432065|2432307|2431719|2431993|N|N|N|N|N| +2432085|AAAAAAAAFFMBFCAA|1946-09-21|560|2438|188|1946|6|9|21|3|1946|188|2438|Saturday|1946Q3|N|Y|N|2432065|2432307|2431720|2431994|N|N|N|N|N| +2432086|AAAAAAAAGFMBFCAA|1946-09-22|560|2438|188|1946|0|9|22|3|1946|188|2438|Sunday|1946Q3|N|N|N|2432065|2432307|2431721|2431995|N|N|N|N|N| +2432087|AAAAAAAAHFMBFCAA|1946-09-23|560|2438|188|1946|1|9|23|3|1946|188|2438|Monday|1946Q3|N|N|N|2432065|2432307|2431722|2431996|N|N|N|N|N| +2432088|AAAAAAAAIFMBFCAA|1946-09-24|560|2439|188|1946|2|9|24|3|1946|188|2439|Tuesday|1946Q3|N|N|N|2432065|2432307|2431723|2431997|N|N|N|N|N| +2432089|AAAAAAAAJFMBFCAA|1946-09-25|560|2439|188|1946|3|9|25|3|1946|188|2439|Wednesday|1946Q3|N|N|N|2432065|2432307|2431724|2431998|N|N|N|N|N| +2432090|AAAAAAAAKFMBFCAA|1946-09-26|560|2439|188|1946|4|9|26|3|1946|188|2439|Thursday|1946Q3|N|N|N|2432065|2432307|2431725|2431999|N|N|N|N|N| +2432091|AAAAAAAALFMBFCAA|1946-09-27|560|2439|188|1946|5|9|27|3|1946|188|2439|Friday|1946Q3|N|Y|N|2432065|2432307|2431726|2432000|N|N|N|N|N| +2432092|AAAAAAAAMFMBFCAA|1946-09-28|560|2439|188|1946|6|9|28|3|1946|188|2439|Saturday|1946Q3|N|Y|N|2432065|2432307|2431727|2432001|N|N|N|N|N| +2432093|AAAAAAAANFMBFCAA|1946-09-29|560|2439|188|1946|0|9|29|3|1946|188|2439|Sunday|1946Q3|N|N|N|2432065|2432307|2431728|2432002|N|N|N|N|N| +2432094|AAAAAAAAOFMBFCAA|1946-09-30|560|2439|188|1946|1|9|30|3|1946|188|2439|Monday|1946Q3|N|N|N|2432065|2432307|2431729|2432003|N|N|N|N|N| +2432095|AAAAAAAAPFMBFCAA|1946-10-01|561|2440|188|1946|2|10|1|3|1946|188|2440|Tuesday|1946Q3|N|N|N|2432095|2432367|2431730|2432003|N|N|N|N|N| +2432096|AAAAAAAAAGMBFCAA|1946-10-02|561|2440|188|1946|3|10|2|4|1946|188|2440|Wednesday|1946Q4|N|N|N|2432095|2432367|2431731|2432004|N|N|N|N|N| +2432097|AAAAAAAABGMBFCAA|1946-10-03|561|2440|188|1946|4|10|3|4|1946|188|2440|Thursday|1946Q4|N|N|N|2432095|2432367|2431732|2432005|N|N|N|N|N| +2432098|AAAAAAAACGMBFCAA|1946-10-04|561|2440|188|1946|5|10|4|4|1946|188|2440|Friday|1946Q4|N|Y|N|2432095|2432367|2431733|2432006|N|N|N|N|N| +2432099|AAAAAAAADGMBFCAA|1946-10-05|561|2440|188|1946|6|10|5|4|1946|188|2440|Saturday|1946Q4|N|Y|N|2432095|2432367|2431734|2432007|N|N|N|N|N| +2432100|AAAAAAAAEGMBFCAA|1946-10-06|561|2440|188|1946|0|10|6|4|1946|188|2440|Sunday|1946Q4|N|N|N|2432095|2432367|2431735|2432008|N|N|N|N|N| +2432101|AAAAAAAAFGMBFCAA|1946-10-07|561|2440|188|1946|1|10|7|4|1946|188|2440|Monday|1946Q4|N|N|N|2432095|2432367|2431736|2432009|N|N|N|N|N| +2432102|AAAAAAAAGGMBFCAA|1946-10-08|561|2441|188|1946|2|10|8|4|1946|188|2441|Tuesday|1946Q4|N|N|N|2432095|2432367|2431737|2432010|N|N|N|N|N| +2432103|AAAAAAAAHGMBFCAA|1946-10-09|561|2441|188|1946|3|10|9|4|1946|188|2441|Wednesday|1946Q4|N|N|N|2432095|2432367|2431738|2432011|N|N|N|N|N| +2432104|AAAAAAAAIGMBFCAA|1946-10-10|561|2441|188|1946|4|10|10|4|1946|188|2441|Thursday|1946Q4|N|N|N|2432095|2432367|2431739|2432012|N|N|N|N|N| +2432105|AAAAAAAAJGMBFCAA|1946-10-11|561|2441|188|1946|5|10|11|4|1946|188|2441|Friday|1946Q4|N|Y|N|2432095|2432367|2431740|2432013|N|N|N|N|N| +2432106|AAAAAAAAKGMBFCAA|1946-10-12|561|2441|188|1946|6|10|12|4|1946|188|2441|Saturday|1946Q4|N|Y|N|2432095|2432367|2431741|2432014|N|N|N|N|N| +2432107|AAAAAAAALGMBFCAA|1946-10-13|561|2441|188|1946|0|10|13|4|1946|188|2441|Sunday|1946Q4|N|N|N|2432095|2432367|2431742|2432015|N|N|N|N|N| +2432108|AAAAAAAAMGMBFCAA|1946-10-14|561|2441|188|1946|1|10|14|4|1946|188|2441|Monday|1946Q4|N|N|N|2432095|2432367|2431743|2432016|N|N|N|N|N| +2432109|AAAAAAAANGMBFCAA|1946-10-15|561|2442|188|1946|2|10|15|4|1946|188|2442|Tuesday|1946Q4|N|N|N|2432095|2432367|2431744|2432017|N|N|N|N|N| +2432110|AAAAAAAAOGMBFCAA|1946-10-16|561|2442|188|1946|3|10|16|4|1946|188|2442|Wednesday|1946Q4|N|N|N|2432095|2432367|2431745|2432018|N|N|N|N|N| +2432111|AAAAAAAAPGMBFCAA|1946-10-17|561|2442|188|1946|4|10|17|4|1946|188|2442|Thursday|1946Q4|N|N|N|2432095|2432367|2431746|2432019|N|N|N|N|N| +2432112|AAAAAAAAAHMBFCAA|1946-10-18|561|2442|188|1946|5|10|18|4|1946|188|2442|Friday|1946Q4|N|Y|N|2432095|2432367|2431747|2432020|N|N|N|N|N| +2432113|AAAAAAAABHMBFCAA|1946-10-19|561|2442|188|1946|6|10|19|4|1946|188|2442|Saturday|1946Q4|N|Y|N|2432095|2432367|2431748|2432021|N|N|N|N|N| +2432114|AAAAAAAACHMBFCAA|1946-10-20|561|2442|188|1946|0|10|20|4|1946|188|2442|Sunday|1946Q4|N|N|N|2432095|2432367|2431749|2432022|N|N|N|N|N| +2432115|AAAAAAAADHMBFCAA|1946-10-21|561|2442|188|1946|1|10|21|4|1946|188|2442|Monday|1946Q4|N|N|N|2432095|2432367|2431750|2432023|N|N|N|N|N| +2432116|AAAAAAAAEHMBFCAA|1946-10-22|561|2443|188|1946|2|10|22|4|1946|188|2443|Tuesday|1946Q4|N|N|N|2432095|2432367|2431751|2432024|N|N|N|N|N| +2432117|AAAAAAAAFHMBFCAA|1946-10-23|561|2443|188|1946|3|10|23|4|1946|188|2443|Wednesday|1946Q4|N|N|N|2432095|2432367|2431752|2432025|N|N|N|N|N| +2432118|AAAAAAAAGHMBFCAA|1946-10-24|561|2443|188|1946|4|10|24|4|1946|188|2443|Thursday|1946Q4|N|N|N|2432095|2432367|2431753|2432026|N|N|N|N|N| +2432119|AAAAAAAAHHMBFCAA|1946-10-25|561|2443|188|1946|5|10|25|4|1946|188|2443|Friday|1946Q4|N|Y|N|2432095|2432367|2431754|2432027|N|N|N|N|N| +2432120|AAAAAAAAIHMBFCAA|1946-10-26|561|2443|188|1946|6|10|26|4|1946|188|2443|Saturday|1946Q4|N|Y|N|2432095|2432367|2431755|2432028|N|N|N|N|N| +2432121|AAAAAAAAJHMBFCAA|1946-10-27|561|2443|188|1946|0|10|27|4|1946|188|2443|Sunday|1946Q4|N|N|N|2432095|2432367|2431756|2432029|N|N|N|N|N| +2432122|AAAAAAAAKHMBFCAA|1946-10-28|561|2443|188|1946|1|10|28|4|1946|188|2443|Monday|1946Q4|N|N|N|2432095|2432367|2431757|2432030|N|N|N|N|N| +2432123|AAAAAAAALHMBFCAA|1946-10-29|561|2444|188|1946|2|10|29|4|1946|188|2444|Tuesday|1946Q4|N|N|N|2432095|2432367|2431758|2432031|N|N|N|N|N| +2432124|AAAAAAAAMHMBFCAA|1946-10-30|561|2444|188|1946|3|10|30|4|1946|188|2444|Wednesday|1946Q4|N|N|N|2432095|2432367|2431759|2432032|N|N|N|N|N| +2432125|AAAAAAAANHMBFCAA|1946-10-31|561|2444|188|1946|4|10|31|4|1946|188|2444|Thursday|1946Q4|N|N|N|2432095|2432367|2431760|2432033|N|N|N|N|N| +2432126|AAAAAAAAOHMBFCAA|1946-11-01|562|2444|188|1946|5|11|1|4|1946|188|2444|Friday|1946Q4|N|Y|N|2432126|2432429|2431761|2432034|N|N|N|N|N| +2432127|AAAAAAAAPHMBFCAA|1946-11-02|562|2444|188|1946|6|11|2|4|1946|188|2444|Saturday|1946Q4|N|Y|N|2432126|2432429|2431762|2432035|N|N|N|N|N| +2432128|AAAAAAAAAIMBFCAA|1946-11-03|562|2444|188|1946|0|11|3|4|1946|188|2444|Sunday|1946Q4|N|N|N|2432126|2432429|2431763|2432036|N|N|N|N|N| +2432129|AAAAAAAABIMBFCAA|1946-11-04|562|2444|188|1946|1|11|4|4|1946|188|2444|Monday|1946Q4|N|N|N|2432126|2432429|2431764|2432037|N|N|N|N|N| +2432130|AAAAAAAACIMBFCAA|1946-11-05|562|2445|188|1946|2|11|5|4|1946|188|2445|Tuesday|1946Q4|N|N|N|2432126|2432429|2431765|2432038|N|N|N|N|N| +2432131|AAAAAAAADIMBFCAA|1946-11-06|562|2445|188|1946|3|11|6|4|1946|188|2445|Wednesday|1946Q4|N|N|N|2432126|2432429|2431766|2432039|N|N|N|N|N| +2432132|AAAAAAAAEIMBFCAA|1946-11-07|562|2445|188|1946|4|11|7|4|1946|188|2445|Thursday|1946Q4|N|N|N|2432126|2432429|2431767|2432040|N|N|N|N|N| +2432133|AAAAAAAAFIMBFCAA|1946-11-08|562|2445|188|1946|5|11|8|4|1946|188|2445|Friday|1946Q4|N|Y|N|2432126|2432429|2431768|2432041|N|N|N|N|N| +2432134|AAAAAAAAGIMBFCAA|1946-11-09|562|2445|188|1946|6|11|9|4|1946|188|2445|Saturday|1946Q4|N|Y|N|2432126|2432429|2431769|2432042|N|N|N|N|N| +2432135|AAAAAAAAHIMBFCAA|1946-11-10|562|2445|188|1946|0|11|10|4|1946|188|2445|Sunday|1946Q4|N|N|N|2432126|2432429|2431770|2432043|N|N|N|N|N| +2432136|AAAAAAAAIIMBFCAA|1946-11-11|562|2445|188|1946|1|11|11|4|1946|188|2445|Monday|1946Q4|N|N|N|2432126|2432429|2431771|2432044|N|N|N|N|N| +2432137|AAAAAAAAJIMBFCAA|1946-11-12|562|2446|188|1946|2|11|12|4|1946|188|2446|Tuesday|1946Q4|N|N|N|2432126|2432429|2431772|2432045|N|N|N|N|N| +2432138|AAAAAAAAKIMBFCAA|1946-11-13|562|2446|188|1946|3|11|13|4|1946|188|2446|Wednesday|1946Q4|N|N|N|2432126|2432429|2431773|2432046|N|N|N|N|N| +2432139|AAAAAAAALIMBFCAA|1946-11-14|562|2446|188|1946|4|11|14|4|1946|188|2446|Thursday|1946Q4|N|N|N|2432126|2432429|2431774|2432047|N|N|N|N|N| +2432140|AAAAAAAAMIMBFCAA|1946-11-15|562|2446|188|1946|5|11|15|4|1946|188|2446|Friday|1946Q4|N|Y|N|2432126|2432429|2431775|2432048|N|N|N|N|N| +2432141|AAAAAAAANIMBFCAA|1946-11-16|562|2446|188|1946|6|11|16|4|1946|188|2446|Saturday|1946Q4|N|Y|N|2432126|2432429|2431776|2432049|N|N|N|N|N| +2432142|AAAAAAAAOIMBFCAA|1946-11-17|562|2446|188|1946|0|11|17|4|1946|188|2446|Sunday|1946Q4|N|N|N|2432126|2432429|2431777|2432050|N|N|N|N|N| +2432143|AAAAAAAAPIMBFCAA|1946-11-18|562|2446|188|1946|1|11|18|4|1946|188|2446|Monday|1946Q4|N|N|N|2432126|2432429|2431778|2432051|N|N|N|N|N| +2432144|AAAAAAAAAJMBFCAA|1946-11-19|562|2447|188|1946|2|11|19|4|1946|188|2447|Tuesday|1946Q4|N|N|N|2432126|2432429|2431779|2432052|N|N|N|N|N| +2432145|AAAAAAAABJMBFCAA|1946-11-20|562|2447|188|1946|3|11|20|4|1946|188|2447|Wednesday|1946Q4|N|N|N|2432126|2432429|2431780|2432053|N|N|N|N|N| +2432146|AAAAAAAACJMBFCAA|1946-11-21|562|2447|188|1946|4|11|21|4|1946|188|2447|Thursday|1946Q4|N|N|N|2432126|2432429|2431781|2432054|N|N|N|N|N| +2432147|AAAAAAAADJMBFCAA|1946-11-22|562|2447|188|1946|5|11|22|4|1946|188|2447|Friday|1946Q4|N|Y|N|2432126|2432429|2431782|2432055|N|N|N|N|N| +2432148|AAAAAAAAEJMBFCAA|1946-11-23|562|2447|188|1946|6|11|23|4|1946|188|2447|Saturday|1946Q4|N|Y|N|2432126|2432429|2431783|2432056|N|N|N|N|N| +2432149|AAAAAAAAFJMBFCAA|1946-11-24|562|2447|188|1946|0|11|24|4|1946|188|2447|Sunday|1946Q4|N|N|N|2432126|2432429|2431784|2432057|N|N|N|N|N| +2432150|AAAAAAAAGJMBFCAA|1946-11-25|562|2447|188|1946|1|11|25|4|1946|188|2447|Monday|1946Q4|N|N|N|2432126|2432429|2431785|2432058|N|N|N|N|N| +2432151|AAAAAAAAHJMBFCAA|1946-11-26|562|2448|188|1946|2|11|26|4|1946|188|2448|Tuesday|1946Q4|N|N|N|2432126|2432429|2431786|2432059|N|N|N|N|N| +2432152|AAAAAAAAIJMBFCAA|1946-11-27|562|2448|188|1946|3|11|27|4|1946|188|2448|Wednesday|1946Q4|N|N|N|2432126|2432429|2431787|2432060|N|N|N|N|N| +2432153|AAAAAAAAJJMBFCAA|1946-11-28|562|2448|188|1946|4|11|28|4|1946|188|2448|Thursday|1946Q4|N|N|N|2432126|2432429|2431788|2432061|N|N|N|N|N| +2432154|AAAAAAAAKJMBFCAA|1946-11-29|562|2448|188|1946|5|11|29|4|1946|188|2448|Friday|1946Q4|N|Y|N|2432126|2432429|2431789|2432062|N|N|N|N|N| +2432155|AAAAAAAALJMBFCAA|1946-11-30|562|2448|188|1946|6|11|30|4|1946|188|2448|Saturday|1946Q4|N|Y|N|2432126|2432429|2431790|2432063|N|N|N|N|N| +2432156|AAAAAAAAMJMBFCAA|1946-12-01|563|2448|189|1946|0|12|1|4|1946|189|2448|Sunday|1946Q4|N|N|N|2432156|2432489|2431791|2432064|N|N|N|N|N| +2432157|AAAAAAAANJMBFCAA|1946-12-02|563|2448|189|1946|1|12|2|4|1946|189|2448|Monday|1946Q4|N|N|N|2432156|2432489|2431792|2432065|N|N|N|N|N| +2432158|AAAAAAAAOJMBFCAA|1946-12-03|563|2449|189|1946|2|12|3|4|1946|189|2449|Tuesday|1946Q4|N|N|N|2432156|2432489|2431793|2432066|N|N|N|N|N| +2432159|AAAAAAAAPJMBFCAA|1946-12-04|563|2449|189|1946|3|12|4|4|1946|189|2449|Wednesday|1946Q4|N|N|N|2432156|2432489|2431794|2432067|N|N|N|N|N| +2432160|AAAAAAAAAKMBFCAA|1946-12-05|563|2449|189|1946|4|12|5|4|1946|189|2449|Thursday|1946Q4|N|N|N|2432156|2432489|2431795|2432068|N|N|N|N|N| +2432161|AAAAAAAABKMBFCAA|1946-12-06|563|2449|189|1946|5|12|6|4|1946|189|2449|Friday|1946Q4|N|Y|N|2432156|2432489|2431796|2432069|N|N|N|N|N| +2432162|AAAAAAAACKMBFCAA|1946-12-07|563|2449|189|1946|6|12|7|4|1946|189|2449|Saturday|1946Q4|N|Y|N|2432156|2432489|2431797|2432070|N|N|N|N|N| +2432163|AAAAAAAADKMBFCAA|1946-12-08|563|2449|189|1946|0|12|8|4|1946|189|2449|Sunday|1946Q4|N|N|N|2432156|2432489|2431798|2432071|N|N|N|N|N| +2432164|AAAAAAAAEKMBFCAA|1946-12-09|563|2449|189|1946|1|12|9|4|1946|189|2449|Monday|1946Q4|N|N|N|2432156|2432489|2431799|2432072|N|N|N|N|N| +2432165|AAAAAAAAFKMBFCAA|1946-12-10|563|2450|189|1946|2|12|10|4|1946|189|2450|Tuesday|1946Q4|N|N|N|2432156|2432489|2431800|2432073|N|N|N|N|N| +2432166|AAAAAAAAGKMBFCAA|1946-12-11|563|2450|189|1946|3|12|11|4|1946|189|2450|Wednesday|1946Q4|N|N|N|2432156|2432489|2431801|2432074|N|N|N|N|N| +2432167|AAAAAAAAHKMBFCAA|1946-12-12|563|2450|189|1946|4|12|12|4|1946|189|2450|Thursday|1946Q4|N|N|N|2432156|2432489|2431802|2432075|N|N|N|N|N| +2432168|AAAAAAAAIKMBFCAA|1946-12-13|563|2450|189|1946|5|12|13|4|1946|189|2450|Friday|1946Q4|N|Y|N|2432156|2432489|2431803|2432076|N|N|N|N|N| +2432169|AAAAAAAAJKMBFCAA|1946-12-14|563|2450|189|1946|6|12|14|4|1946|189|2450|Saturday|1946Q4|N|Y|N|2432156|2432489|2431804|2432077|N|N|N|N|N| +2432170|AAAAAAAAKKMBFCAA|1946-12-15|563|2450|189|1946|0|12|15|4|1946|189|2450|Sunday|1946Q4|N|N|N|2432156|2432489|2431805|2432078|N|N|N|N|N| +2432171|AAAAAAAALKMBFCAA|1946-12-16|563|2450|189|1946|1|12|16|4|1946|189|2450|Monday|1946Q4|N|N|N|2432156|2432489|2431806|2432079|N|N|N|N|N| +2432172|AAAAAAAAMKMBFCAA|1946-12-17|563|2451|189|1946|2|12|17|4|1946|189|2451|Tuesday|1946Q4|N|N|N|2432156|2432489|2431807|2432080|N|N|N|N|N| +2432173|AAAAAAAANKMBFCAA|1946-12-18|563|2451|189|1946|3|12|18|4|1946|189|2451|Wednesday|1946Q4|N|N|N|2432156|2432489|2431808|2432081|N|N|N|N|N| +2432174|AAAAAAAAOKMBFCAA|1946-12-19|563|2451|189|1946|4|12|19|4|1946|189|2451|Thursday|1946Q4|N|N|N|2432156|2432489|2431809|2432082|N|N|N|N|N| +2432175|AAAAAAAAPKMBFCAA|1946-12-20|563|2451|189|1946|5|12|20|4|1946|189|2451|Friday|1946Q4|N|Y|N|2432156|2432489|2431810|2432083|N|N|N|N|N| +2432176|AAAAAAAAALMBFCAA|1946-12-21|563|2451|189|1946|6|12|21|4|1946|189|2451|Saturday|1946Q4|N|Y|N|2432156|2432489|2431811|2432084|N|N|N|N|N| +2432177|AAAAAAAABLMBFCAA|1946-12-22|563|2451|189|1946|0|12|22|4|1946|189|2451|Sunday|1946Q4|N|N|N|2432156|2432489|2431812|2432085|N|N|N|N|N| +2432178|AAAAAAAACLMBFCAA|1946-12-23|563|2451|189|1946|1|12|23|4|1946|189|2451|Monday|1946Q4|N|N|N|2432156|2432489|2431813|2432086|N|N|N|N|N| +2432179|AAAAAAAADLMBFCAA|1946-12-24|563|2452|189|1946|2|12|24|4|1946|189|2452|Tuesday|1946Q4|N|N|N|2432156|2432489|2431814|2432087|N|N|N|N|N| +2432180|AAAAAAAAELMBFCAA|1946-12-25|563|2452|189|1946|3|12|25|4|1946|189|2452|Wednesday|1946Q4|N|N|N|2432156|2432489|2431815|2432088|N|N|N|N|N| +2432181|AAAAAAAAFLMBFCAA|1946-12-26|563|2452|189|1946|4|12|26|4|1946|189|2452|Thursday|1946Q4|Y|N|N|2432156|2432489|2431816|2432089|N|N|N|N|N| +2432182|AAAAAAAAGLMBFCAA|1946-12-27|563|2452|189|1946|5|12|27|4|1946|189|2452|Friday|1946Q4|N|Y|Y|2432156|2432489|2431817|2432090|N|N|N|N|N| +2432183|AAAAAAAAHLMBFCAA|1946-12-28|563|2452|189|1946|6|12|28|4|1946|189|2452|Saturday|1946Q4|N|Y|N|2432156|2432489|2431818|2432091|N|N|N|N|N| +2432184|AAAAAAAAILMBFCAA|1946-12-29|563|2452|189|1946|0|12|29|4|1946|189|2452|Sunday|1946Q4|N|N|N|2432156|2432489|2431819|2432092|N|N|N|N|N| +2432185|AAAAAAAAJLMBFCAA|1946-12-30|563|2452|189|1946|1|12|30|4|1946|189|2452|Monday|1946Q4|N|N|N|2432156|2432489|2431820|2432093|N|N|N|N|N| +2432186|AAAAAAAAKLMBFCAA|1946-12-31|563|2453|189|1946|2|12|31|4|1946|189|2453|Tuesday|1946Q4|N|N|N|2432156|2432489|2431821|2432094|N|N|N|N|N| +2432187|AAAAAAAALLMBFCAA|1947-01-01|564|2453|189|1947|3|1|1|1|1947|189|2453|Wednesday|1947Q1|Y|N|N|2432187|2432186|2431822|2432095|N|N|N|N|N| +2432188|AAAAAAAAMLMBFCAA|1947-01-02|564|2453|189|1947|4|1|2|1|1947|189|2453|Thursday|1947Q1|N|N|Y|2432187|2432186|2431823|2432096|N|N|N|N|N| +2432189|AAAAAAAANLMBFCAA|1947-01-03|564|2453|189|1947|5|1|3|1|1947|189|2453|Friday|1947Q1|N|Y|N|2432187|2432186|2431824|2432097|N|N|N|N|N| +2432190|AAAAAAAAOLMBFCAA|1947-01-04|564|2453|189|1947|6|1|4|1|1947|189|2453|Saturday|1947Q1|N|Y|N|2432187|2432186|2431825|2432098|N|N|N|N|N| +2432191|AAAAAAAAPLMBFCAA|1947-01-05|564|2453|189|1947|0|1|5|1|1947|189|2453|Sunday|1947Q1|N|N|N|2432187|2432186|2431826|2432099|N|N|N|N|N| +2432192|AAAAAAAAAMMBFCAA|1947-01-06|564|2453|189|1947|1|1|6|1|1947|189|2453|Monday|1947Q1|N|N|N|2432187|2432186|2431827|2432100|N|N|N|N|N| +2432193|AAAAAAAABMMBFCAA|1947-01-07|564|2454|189|1947|2|1|7|1|1947|189|2454|Tuesday|1947Q1|N|N|N|2432187|2432186|2431828|2432101|N|N|N|N|N| +2432194|AAAAAAAACMMBFCAA|1947-01-08|564|2454|189|1947|3|1|8|1|1947|189|2454|Wednesday|1947Q1|N|N|N|2432187|2432186|2431829|2432102|N|N|N|N|N| +2432195|AAAAAAAADMMBFCAA|1947-01-09|564|2454|189|1947|4|1|9|1|1947|189|2454|Thursday|1947Q1|N|N|N|2432187|2432186|2431830|2432103|N|N|N|N|N| +2432196|AAAAAAAAEMMBFCAA|1947-01-10|564|2454|189|1947|5|1|10|1|1947|189|2454|Friday|1947Q1|N|Y|N|2432187|2432186|2431831|2432104|N|N|N|N|N| +2432197|AAAAAAAAFMMBFCAA|1947-01-11|564|2454|189|1947|6|1|11|1|1947|189|2454|Saturday|1947Q1|N|Y|N|2432187|2432186|2431832|2432105|N|N|N|N|N| +2432198|AAAAAAAAGMMBFCAA|1947-01-12|564|2454|189|1947|0|1|12|1|1947|189|2454|Sunday|1947Q1|N|N|N|2432187|2432186|2431833|2432106|N|N|N|N|N| +2432199|AAAAAAAAHMMBFCAA|1947-01-13|564|2454|189|1947|1|1|13|1|1947|189|2454|Monday|1947Q1|N|N|N|2432187|2432186|2431834|2432107|N|N|N|N|N| +2432200|AAAAAAAAIMMBFCAA|1947-01-14|564|2455|189|1947|2|1|14|1|1947|189|2455|Tuesday|1947Q1|N|N|N|2432187|2432186|2431835|2432108|N|N|N|N|N| +2432201|AAAAAAAAJMMBFCAA|1947-01-15|564|2455|189|1947|3|1|15|1|1947|189|2455|Wednesday|1947Q1|N|N|N|2432187|2432186|2431836|2432109|N|N|N|N|N| +2432202|AAAAAAAAKMMBFCAA|1947-01-16|564|2455|189|1947|4|1|16|1|1947|189|2455|Thursday|1947Q1|N|N|N|2432187|2432186|2431837|2432110|N|N|N|N|N| +2432203|AAAAAAAALMMBFCAA|1947-01-17|564|2455|189|1947|5|1|17|1|1947|189|2455|Friday|1947Q1|N|Y|N|2432187|2432186|2431838|2432111|N|N|N|N|N| +2432204|AAAAAAAAMMMBFCAA|1947-01-18|564|2455|189|1947|6|1|18|1|1947|189|2455|Saturday|1947Q1|N|Y|N|2432187|2432186|2431839|2432112|N|N|N|N|N| +2432205|AAAAAAAANMMBFCAA|1947-01-19|564|2455|189|1947|0|1|19|1|1947|189|2455|Sunday|1947Q1|N|N|N|2432187|2432186|2431840|2432113|N|N|N|N|N| +2432206|AAAAAAAAOMMBFCAA|1947-01-20|564|2455|189|1947|1|1|20|1|1947|189|2455|Monday|1947Q1|N|N|N|2432187|2432186|2431841|2432114|N|N|N|N|N| +2432207|AAAAAAAAPMMBFCAA|1947-01-21|564|2456|189|1947|2|1|21|1|1947|189|2456|Tuesday|1947Q1|N|N|N|2432187|2432186|2431842|2432115|N|N|N|N|N| +2432208|AAAAAAAAANMBFCAA|1947-01-22|564|2456|189|1947|3|1|22|1|1947|189|2456|Wednesday|1947Q1|N|N|N|2432187|2432186|2431843|2432116|N|N|N|N|N| +2432209|AAAAAAAABNMBFCAA|1947-01-23|564|2456|189|1947|4|1|23|1|1947|189|2456|Thursday|1947Q1|N|N|N|2432187|2432186|2431844|2432117|N|N|N|N|N| +2432210|AAAAAAAACNMBFCAA|1947-01-24|564|2456|189|1947|5|1|24|1|1947|189|2456|Friday|1947Q1|N|Y|N|2432187|2432186|2431845|2432118|N|N|N|N|N| +2432211|AAAAAAAADNMBFCAA|1947-01-25|564|2456|189|1947|6|1|25|1|1947|189|2456|Saturday|1947Q1|N|Y|N|2432187|2432186|2431846|2432119|N|N|N|N|N| +2432212|AAAAAAAAENMBFCAA|1947-01-26|564|2456|189|1947|0|1|26|1|1947|189|2456|Sunday|1947Q1|N|N|N|2432187|2432186|2431847|2432120|N|N|N|N|N| +2432213|AAAAAAAAFNMBFCAA|1947-01-27|564|2456|189|1947|1|1|27|1|1947|189|2456|Monday|1947Q1|N|N|N|2432187|2432186|2431848|2432121|N|N|N|N|N| +2432214|AAAAAAAAGNMBFCAA|1947-01-28|564|2457|189|1947|2|1|28|1|1947|189|2457|Tuesday|1947Q1|N|N|N|2432187|2432186|2431849|2432122|N|N|N|N|N| +2432215|AAAAAAAAHNMBFCAA|1947-01-29|564|2457|189|1947|3|1|29|1|1947|189|2457|Wednesday|1947Q1|N|N|N|2432187|2432186|2431850|2432123|N|N|N|N|N| +2432216|AAAAAAAAINMBFCAA|1947-01-30|564|2457|189|1947|4|1|30|1|1947|189|2457|Thursday|1947Q1|N|N|N|2432187|2432186|2431851|2432124|N|N|N|N|N| +2432217|AAAAAAAAJNMBFCAA|1947-01-31|564|2457|189|1947|5|1|31|1|1947|189|2457|Friday|1947Q1|N|Y|N|2432187|2432186|2431852|2432125|N|N|N|N|N| +2432218|AAAAAAAAKNMBFCAA|1947-02-01|565|2457|189|1947|6|2|1|1|1947|189|2457|Saturday|1947Q1|N|Y|N|2432218|2432248|2431853|2432126|N|N|N|N|N| +2432219|AAAAAAAALNMBFCAA|1947-02-02|565|2457|189|1947|0|2|2|1|1947|189|2457|Sunday|1947Q1|N|N|N|2432218|2432248|2431854|2432127|N|N|N|N|N| +2432220|AAAAAAAAMNMBFCAA|1947-02-03|565|2457|189|1947|1|2|3|1|1947|189|2457|Monday|1947Q1|N|N|N|2432218|2432248|2431855|2432128|N|N|N|N|N| +2432221|AAAAAAAANNMBFCAA|1947-02-04|565|2458|189|1947|2|2|4|1|1947|189|2458|Tuesday|1947Q1|N|N|N|2432218|2432248|2431856|2432129|N|N|N|N|N| +2432222|AAAAAAAAONMBFCAA|1947-02-05|565|2458|189|1947|3|2|5|1|1947|189|2458|Wednesday|1947Q1|N|N|N|2432218|2432248|2431857|2432130|N|N|N|N|N| +2432223|AAAAAAAAPNMBFCAA|1947-02-06|565|2458|189|1947|4|2|6|1|1947|189|2458|Thursday|1947Q1|N|N|N|2432218|2432248|2431858|2432131|N|N|N|N|N| +2432224|AAAAAAAAAOMBFCAA|1947-02-07|565|2458|189|1947|5|2|7|1|1947|189|2458|Friday|1947Q1|N|Y|N|2432218|2432248|2431859|2432132|N|N|N|N|N| +2432225|AAAAAAAABOMBFCAA|1947-02-08|565|2458|189|1947|6|2|8|1|1947|189|2458|Saturday|1947Q1|N|Y|N|2432218|2432248|2431860|2432133|N|N|N|N|N| +2432226|AAAAAAAACOMBFCAA|1947-02-09|565|2458|189|1947|0|2|9|1|1947|189|2458|Sunday|1947Q1|N|N|N|2432218|2432248|2431861|2432134|N|N|N|N|N| +2432227|AAAAAAAADOMBFCAA|1947-02-10|565|2458|189|1947|1|2|10|1|1947|189|2458|Monday|1947Q1|N|N|N|2432218|2432248|2431862|2432135|N|N|N|N|N| +2432228|AAAAAAAAEOMBFCAA|1947-02-11|565|2459|189|1947|2|2|11|1|1947|189|2459|Tuesday|1947Q1|N|N|N|2432218|2432248|2431863|2432136|N|N|N|N|N| +2432229|AAAAAAAAFOMBFCAA|1947-02-12|565|2459|189|1947|3|2|12|1|1947|189|2459|Wednesday|1947Q1|N|N|N|2432218|2432248|2431864|2432137|N|N|N|N|N| +2432230|AAAAAAAAGOMBFCAA|1947-02-13|565|2459|189|1947|4|2|13|1|1947|189|2459|Thursday|1947Q1|N|N|N|2432218|2432248|2431865|2432138|N|N|N|N|N| +2432231|AAAAAAAAHOMBFCAA|1947-02-14|565|2459|189|1947|5|2|14|1|1947|189|2459|Friday|1947Q1|N|Y|N|2432218|2432248|2431866|2432139|N|N|N|N|N| +2432232|AAAAAAAAIOMBFCAA|1947-02-15|565|2459|189|1947|6|2|15|1|1947|189|2459|Saturday|1947Q1|N|Y|N|2432218|2432248|2431867|2432140|N|N|N|N|N| +2432233|AAAAAAAAJOMBFCAA|1947-02-16|565|2459|189|1947|0|2|16|1|1947|189|2459|Sunday|1947Q1|N|N|N|2432218|2432248|2431868|2432141|N|N|N|N|N| +2432234|AAAAAAAAKOMBFCAA|1947-02-17|565|2459|189|1947|1|2|17|1|1947|189|2459|Monday|1947Q1|N|N|N|2432218|2432248|2431869|2432142|N|N|N|N|N| +2432235|AAAAAAAALOMBFCAA|1947-02-18|565|2460|189|1947|2|2|18|1|1947|189|2460|Tuesday|1947Q1|N|N|N|2432218|2432248|2431870|2432143|N|N|N|N|N| +2432236|AAAAAAAAMOMBFCAA|1947-02-19|565|2460|189|1947|3|2|19|1|1947|189|2460|Wednesday|1947Q1|N|N|N|2432218|2432248|2431871|2432144|N|N|N|N|N| +2432237|AAAAAAAANOMBFCAA|1947-02-20|565|2460|189|1947|4|2|20|1|1947|189|2460|Thursday|1947Q1|N|N|N|2432218|2432248|2431872|2432145|N|N|N|N|N| +2432238|AAAAAAAAOOMBFCAA|1947-02-21|565|2460|189|1947|5|2|21|1|1947|189|2460|Friday|1947Q1|N|Y|N|2432218|2432248|2431873|2432146|N|N|N|N|N| +2432239|AAAAAAAAPOMBFCAA|1947-02-22|565|2460|189|1947|6|2|22|1|1947|189|2460|Saturday|1947Q1|N|Y|N|2432218|2432248|2431874|2432147|N|N|N|N|N| +2432240|AAAAAAAAAPMBFCAA|1947-02-23|565|2460|189|1947|0|2|23|1|1947|189|2460|Sunday|1947Q1|N|N|N|2432218|2432248|2431875|2432148|N|N|N|N|N| +2432241|AAAAAAAABPMBFCAA|1947-02-24|565|2460|189|1947|1|2|24|1|1947|189|2460|Monday|1947Q1|N|N|N|2432218|2432248|2431876|2432149|N|N|N|N|N| +2432242|AAAAAAAACPMBFCAA|1947-02-25|565|2461|189|1947|2|2|25|1|1947|189|2461|Tuesday|1947Q1|N|N|N|2432218|2432248|2431877|2432150|N|N|N|N|N| +2432243|AAAAAAAADPMBFCAA|1947-02-26|565|2461|189|1947|3|2|26|1|1947|189|2461|Wednesday|1947Q1|N|N|N|2432218|2432248|2431878|2432151|N|N|N|N|N| +2432244|AAAAAAAAEPMBFCAA|1947-02-27|565|2461|189|1947|4|2|27|1|1947|189|2461|Thursday|1947Q1|N|N|N|2432218|2432248|2431879|2432152|N|N|N|N|N| +2432245|AAAAAAAAFPMBFCAA|1947-02-28|565|2461|189|1947|5|2|28|1|1947|189|2461|Friday|1947Q1|N|Y|N|2432218|2432248|2431880|2432153|N|N|N|N|N| +2432246|AAAAAAAAGPMBFCAA|1947-03-01|566|2461|190|1947|6|3|1|1|1947|190|2461|Saturday|1947Q1|N|Y|N|2432246|2432304|2431881|2432154|N|N|N|N|N| +2432247|AAAAAAAAHPMBFCAA|1947-03-02|566|2461|190|1947|0|3|2|1|1947|190|2461|Sunday|1947Q1|N|N|N|2432246|2432304|2431882|2432155|N|N|N|N|N| +2432248|AAAAAAAAIPMBFCAA|1947-03-03|566|2461|190|1947|1|3|3|1|1947|190|2461|Monday|1947Q1|N|N|N|2432246|2432304|2431883|2432156|N|N|N|N|N| +2432249|AAAAAAAAJPMBFCAA|1947-03-04|566|2462|190|1947|2|3|4|1|1947|190|2462|Tuesday|1947Q1|N|N|N|2432246|2432304|2431884|2432157|N|N|N|N|N| +2432250|AAAAAAAAKPMBFCAA|1947-03-05|566|2462|190|1947|3|3|5|1|1947|190|2462|Wednesday|1947Q1|N|N|N|2432246|2432304|2431885|2432158|N|N|N|N|N| +2432251|AAAAAAAALPMBFCAA|1947-03-06|566|2462|190|1947|4|3|6|1|1947|190|2462|Thursday|1947Q1|N|N|N|2432246|2432304|2431886|2432159|N|N|N|N|N| +2432252|AAAAAAAAMPMBFCAA|1947-03-07|566|2462|190|1947|5|3|7|1|1947|190|2462|Friday|1947Q1|N|Y|N|2432246|2432304|2431887|2432160|N|N|N|N|N| +2432253|AAAAAAAANPMBFCAA|1947-03-08|566|2462|190|1947|6|3|8|1|1947|190|2462|Saturday|1947Q1|N|Y|N|2432246|2432304|2431888|2432161|N|N|N|N|N| +2432254|AAAAAAAAOPMBFCAA|1947-03-09|566|2462|190|1947|0|3|9|1|1947|190|2462|Sunday|1947Q1|N|N|N|2432246|2432304|2431889|2432162|N|N|N|N|N| +2432255|AAAAAAAAPPMBFCAA|1947-03-10|566|2462|190|1947|1|3|10|1|1947|190|2462|Monday|1947Q1|N|N|N|2432246|2432304|2431890|2432163|N|N|N|N|N| +2432256|AAAAAAAAAANBFCAA|1947-03-11|566|2463|190|1947|2|3|11|1|1947|190|2463|Tuesday|1947Q1|N|N|N|2432246|2432304|2431891|2432164|N|N|N|N|N| +2432257|AAAAAAAABANBFCAA|1947-03-12|566|2463|190|1947|3|3|12|1|1947|190|2463|Wednesday|1947Q1|N|N|N|2432246|2432304|2431892|2432165|N|N|N|N|N| +2432258|AAAAAAAACANBFCAA|1947-03-13|566|2463|190|1947|4|3|13|1|1947|190|2463|Thursday|1947Q1|N|N|N|2432246|2432304|2431893|2432166|N|N|N|N|N| +2432259|AAAAAAAADANBFCAA|1947-03-14|566|2463|190|1947|5|3|14|1|1947|190|2463|Friday|1947Q1|N|Y|N|2432246|2432304|2431894|2432167|N|N|N|N|N| +2432260|AAAAAAAAEANBFCAA|1947-03-15|566|2463|190|1947|6|3|15|1|1947|190|2463|Saturday|1947Q1|N|Y|N|2432246|2432304|2431895|2432168|N|N|N|N|N| +2432261|AAAAAAAAFANBFCAA|1947-03-16|566|2463|190|1947|0|3|16|1|1947|190|2463|Sunday|1947Q1|N|N|N|2432246|2432304|2431896|2432169|N|N|N|N|N| +2432262|AAAAAAAAGANBFCAA|1947-03-17|566|2463|190|1947|1|3|17|1|1947|190|2463|Monday|1947Q1|N|N|N|2432246|2432304|2431897|2432170|N|N|N|N|N| +2432263|AAAAAAAAHANBFCAA|1947-03-18|566|2464|190|1947|2|3|18|1|1947|190|2464|Tuesday|1947Q1|N|N|N|2432246|2432304|2431898|2432171|N|N|N|N|N| +2432264|AAAAAAAAIANBFCAA|1947-03-19|566|2464|190|1947|3|3|19|1|1947|190|2464|Wednesday|1947Q1|N|N|N|2432246|2432304|2431899|2432172|N|N|N|N|N| +2432265|AAAAAAAAJANBFCAA|1947-03-20|566|2464|190|1947|4|3|20|1|1947|190|2464|Thursday|1947Q1|N|N|N|2432246|2432304|2431900|2432173|N|N|N|N|N| +2432266|AAAAAAAAKANBFCAA|1947-03-21|566|2464|190|1947|5|3|21|1|1947|190|2464|Friday|1947Q1|N|Y|N|2432246|2432304|2431901|2432174|N|N|N|N|N| +2432267|AAAAAAAALANBFCAA|1947-03-22|566|2464|190|1947|6|3|22|1|1947|190|2464|Saturday|1947Q1|N|Y|N|2432246|2432304|2431902|2432175|N|N|N|N|N| +2432268|AAAAAAAAMANBFCAA|1947-03-23|566|2464|190|1947|0|3|23|1|1947|190|2464|Sunday|1947Q1|N|N|N|2432246|2432304|2431903|2432176|N|N|N|N|N| +2432269|AAAAAAAANANBFCAA|1947-03-24|566|2464|190|1947|1|3|24|1|1947|190|2464|Monday|1947Q1|N|N|N|2432246|2432304|2431904|2432177|N|N|N|N|N| +2432270|AAAAAAAAOANBFCAA|1947-03-25|566|2465|190|1947|2|3|25|1|1947|190|2465|Tuesday|1947Q1|N|N|N|2432246|2432304|2431905|2432178|N|N|N|N|N| +2432271|AAAAAAAAPANBFCAA|1947-03-26|566|2465|190|1947|3|3|26|1|1947|190|2465|Wednesday|1947Q1|N|N|N|2432246|2432304|2431906|2432179|N|N|N|N|N| +2432272|AAAAAAAAABNBFCAA|1947-03-27|566|2465|190|1947|4|3|27|1|1947|190|2465|Thursday|1947Q1|N|N|N|2432246|2432304|2431907|2432180|N|N|N|N|N| +2432273|AAAAAAAABBNBFCAA|1947-03-28|566|2465|190|1947|5|3|28|1|1947|190|2465|Friday|1947Q1|N|Y|N|2432246|2432304|2431908|2432181|N|N|N|N|N| +2432274|AAAAAAAACBNBFCAA|1947-03-29|566|2465|190|1947|6|3|29|1|1947|190|2465|Saturday|1947Q1|N|Y|N|2432246|2432304|2431909|2432182|N|N|N|N|N| +2432275|AAAAAAAADBNBFCAA|1947-03-30|566|2465|190|1947|0|3|30|1|1947|190|2465|Sunday|1947Q1|N|N|N|2432246|2432304|2431910|2432183|N|N|N|N|N| +2432276|AAAAAAAAEBNBFCAA|1947-03-31|566|2465|190|1947|1|3|31|1|1947|190|2465|Monday|1947Q1|N|N|N|2432246|2432304|2431911|2432184|N|N|N|N|N| +2432277|AAAAAAAAFBNBFCAA|1947-04-01|567|2466|190|1947|2|4|1|1|1947|190|2466|Tuesday|1947Q1|N|N|N|2432277|2432366|2431912|2432187|N|N|N|N|N| +2432278|AAAAAAAAGBNBFCAA|1947-04-02|567|2466|190|1947|3|4|2|2|1947|190|2466|Wednesday|1947Q2|N|N|N|2432277|2432366|2431913|2432188|N|N|N|N|N| +2432279|AAAAAAAAHBNBFCAA|1947-04-03|567|2466|190|1947|4|4|3|2|1947|190|2466|Thursday|1947Q2|N|N|N|2432277|2432366|2431914|2432189|N|N|N|N|N| +2432280|AAAAAAAAIBNBFCAA|1947-04-04|567|2466|190|1947|5|4|4|2|1947|190|2466|Friday|1947Q2|N|Y|N|2432277|2432366|2431915|2432190|N|N|N|N|N| +2432281|AAAAAAAAJBNBFCAA|1947-04-05|567|2466|190|1947|6|4|5|2|1947|190|2466|Saturday|1947Q2|N|Y|N|2432277|2432366|2431916|2432191|N|N|N|N|N| +2432282|AAAAAAAAKBNBFCAA|1947-04-06|567|2466|190|1947|0|4|6|2|1947|190|2466|Sunday|1947Q2|N|N|N|2432277|2432366|2431917|2432192|N|N|N|N|N| +2432283|AAAAAAAALBNBFCAA|1947-04-07|567|2466|190|1947|1|4|7|2|1947|190|2466|Monday|1947Q2|N|N|N|2432277|2432366|2431918|2432193|N|N|N|N|N| +2432284|AAAAAAAAMBNBFCAA|1947-04-08|567|2467|190|1947|2|4|8|2|1947|190|2467|Tuesday|1947Q2|N|N|N|2432277|2432366|2431919|2432194|N|N|N|N|N| +2432285|AAAAAAAANBNBFCAA|1947-04-09|567|2467|190|1947|3|4|9|2|1947|190|2467|Wednesday|1947Q2|N|N|N|2432277|2432366|2431920|2432195|N|N|N|N|N| +2432286|AAAAAAAAOBNBFCAA|1947-04-10|567|2467|190|1947|4|4|10|2|1947|190|2467|Thursday|1947Q2|N|N|N|2432277|2432366|2431921|2432196|N|N|N|N|N| +2432287|AAAAAAAAPBNBFCAA|1947-04-11|567|2467|190|1947|5|4|11|2|1947|190|2467|Friday|1947Q2|N|Y|N|2432277|2432366|2431922|2432197|N|N|N|N|N| +2432288|AAAAAAAAACNBFCAA|1947-04-12|567|2467|190|1947|6|4|12|2|1947|190|2467|Saturday|1947Q2|N|Y|N|2432277|2432366|2431923|2432198|N|N|N|N|N| +2432289|AAAAAAAABCNBFCAA|1947-04-13|567|2467|190|1947|0|4|13|2|1947|190|2467|Sunday|1947Q2|N|N|N|2432277|2432366|2431924|2432199|N|N|N|N|N| +2432290|AAAAAAAACCNBFCAA|1947-04-14|567|2467|190|1947|1|4|14|2|1947|190|2467|Monday|1947Q2|N|N|N|2432277|2432366|2431925|2432200|N|N|N|N|N| +2432291|AAAAAAAADCNBFCAA|1947-04-15|567|2468|190|1947|2|4|15|2|1947|190|2468|Tuesday|1947Q2|N|N|N|2432277|2432366|2431926|2432201|N|N|N|N|N| +2432292|AAAAAAAAECNBFCAA|1947-04-16|567|2468|190|1947|3|4|16|2|1947|190|2468|Wednesday|1947Q2|N|N|N|2432277|2432366|2431927|2432202|N|N|N|N|N| +2432293|AAAAAAAAFCNBFCAA|1947-04-17|567|2468|190|1947|4|4|17|2|1947|190|2468|Thursday|1947Q2|N|N|N|2432277|2432366|2431928|2432203|N|N|N|N|N| +2432294|AAAAAAAAGCNBFCAA|1947-04-18|567|2468|190|1947|5|4|18|2|1947|190|2468|Friday|1947Q2|N|Y|N|2432277|2432366|2431929|2432204|N|N|N|N|N| +2432295|AAAAAAAAHCNBFCAA|1947-04-19|567|2468|190|1947|6|4|19|2|1947|190|2468|Saturday|1947Q2|N|Y|N|2432277|2432366|2431930|2432205|N|N|N|N|N| +2432296|AAAAAAAAICNBFCAA|1947-04-20|567|2468|190|1947|0|4|20|2|1947|190|2468|Sunday|1947Q2|N|N|N|2432277|2432366|2431931|2432206|N|N|N|N|N| +2432297|AAAAAAAAJCNBFCAA|1947-04-21|567|2468|190|1947|1|4|21|2|1947|190|2468|Monday|1947Q2|N|N|N|2432277|2432366|2431932|2432207|N|N|N|N|N| +2432298|AAAAAAAAKCNBFCAA|1947-04-22|567|2469|190|1947|2|4|22|2|1947|190|2469|Tuesday|1947Q2|N|N|N|2432277|2432366|2431933|2432208|N|N|N|N|N| +2432299|AAAAAAAALCNBFCAA|1947-04-23|567|2469|190|1947|3|4|23|2|1947|190|2469|Wednesday|1947Q2|N|N|N|2432277|2432366|2431934|2432209|N|N|N|N|N| +2432300|AAAAAAAAMCNBFCAA|1947-04-24|567|2469|190|1947|4|4|24|2|1947|190|2469|Thursday|1947Q2|N|N|N|2432277|2432366|2431935|2432210|N|N|N|N|N| +2432301|AAAAAAAANCNBFCAA|1947-04-25|567|2469|190|1947|5|4|25|2|1947|190|2469|Friday|1947Q2|N|Y|N|2432277|2432366|2431936|2432211|N|N|N|N|N| +2432302|AAAAAAAAOCNBFCAA|1947-04-26|567|2469|190|1947|6|4|26|2|1947|190|2469|Saturday|1947Q2|N|Y|N|2432277|2432366|2431937|2432212|N|N|N|N|N| +2432303|AAAAAAAAPCNBFCAA|1947-04-27|567|2469|190|1947|0|4|27|2|1947|190|2469|Sunday|1947Q2|N|N|N|2432277|2432366|2431938|2432213|N|N|N|N|N| +2432304|AAAAAAAAADNBFCAA|1947-04-28|567|2469|190|1947|1|4|28|2|1947|190|2469|Monday|1947Q2|N|N|N|2432277|2432366|2431939|2432214|N|N|N|N|N| +2432305|AAAAAAAABDNBFCAA|1947-04-29|567|2470|190|1947|2|4|29|2|1947|190|2470|Tuesday|1947Q2|N|N|N|2432277|2432366|2431940|2432215|N|N|N|N|N| +2432306|AAAAAAAACDNBFCAA|1947-04-30|567|2470|190|1947|3|4|30|2|1947|190|2470|Wednesday|1947Q2|N|N|N|2432277|2432366|2431941|2432216|N|N|N|N|N| +2432307|AAAAAAAADDNBFCAA|1947-05-01|568|2470|190|1947|4|5|1|2|1947|190|2470|Thursday|1947Q2|N|N|N|2432307|2432426|2431942|2432217|N|N|N|N|N| +2432308|AAAAAAAAEDNBFCAA|1947-05-02|568|2470|190|1947|5|5|2|2|1947|190|2470|Friday|1947Q2|N|Y|N|2432307|2432426|2431943|2432218|N|N|N|N|N| +2432309|AAAAAAAAFDNBFCAA|1947-05-03|568|2470|190|1947|6|5|3|2|1947|190|2470|Saturday|1947Q2|N|Y|N|2432307|2432426|2431944|2432219|N|N|N|N|N| +2432310|AAAAAAAAGDNBFCAA|1947-05-04|568|2470|190|1947|0|5|4|2|1947|190|2470|Sunday|1947Q2|N|N|N|2432307|2432426|2431945|2432220|N|N|N|N|N| +2432311|AAAAAAAAHDNBFCAA|1947-05-05|568|2470|190|1947|1|5|5|2|1947|190|2470|Monday|1947Q2|N|N|N|2432307|2432426|2431946|2432221|N|N|N|N|N| +2432312|AAAAAAAAIDNBFCAA|1947-05-06|568|2471|190|1947|2|5|6|2|1947|190|2471|Tuesday|1947Q2|N|N|N|2432307|2432426|2431947|2432222|N|N|N|N|N| +2432313|AAAAAAAAJDNBFCAA|1947-05-07|568|2471|190|1947|3|5|7|2|1947|190|2471|Wednesday|1947Q2|N|N|N|2432307|2432426|2431948|2432223|N|N|N|N|N| +2432314|AAAAAAAAKDNBFCAA|1947-05-08|568|2471|190|1947|4|5|8|2|1947|190|2471|Thursday|1947Q2|N|N|N|2432307|2432426|2431949|2432224|N|N|N|N|N| +2432315|AAAAAAAALDNBFCAA|1947-05-09|568|2471|190|1947|5|5|9|2|1947|190|2471|Friday|1947Q2|N|Y|N|2432307|2432426|2431950|2432225|N|N|N|N|N| +2432316|AAAAAAAAMDNBFCAA|1947-05-10|568|2471|190|1947|6|5|10|2|1947|190|2471|Saturday|1947Q2|N|Y|N|2432307|2432426|2431951|2432226|N|N|N|N|N| +2432317|AAAAAAAANDNBFCAA|1947-05-11|568|2471|190|1947|0|5|11|2|1947|190|2471|Sunday|1947Q2|N|N|N|2432307|2432426|2431952|2432227|N|N|N|N|N| +2432318|AAAAAAAAODNBFCAA|1947-05-12|568|2471|190|1947|1|5|12|2|1947|190|2471|Monday|1947Q2|N|N|N|2432307|2432426|2431953|2432228|N|N|N|N|N| +2432319|AAAAAAAAPDNBFCAA|1947-05-13|568|2472|190|1947|2|5|13|2|1947|190|2472|Tuesday|1947Q2|N|N|N|2432307|2432426|2431954|2432229|N|N|N|N|N| +2432320|AAAAAAAAAENBFCAA|1947-05-14|568|2472|190|1947|3|5|14|2|1947|190|2472|Wednesday|1947Q2|N|N|N|2432307|2432426|2431955|2432230|N|N|N|N|N| +2432321|AAAAAAAABENBFCAA|1947-05-15|568|2472|190|1947|4|5|15|2|1947|190|2472|Thursday|1947Q2|N|N|N|2432307|2432426|2431956|2432231|N|N|N|N|N| +2432322|AAAAAAAACENBFCAA|1947-05-16|568|2472|190|1947|5|5|16|2|1947|190|2472|Friday|1947Q2|N|Y|N|2432307|2432426|2431957|2432232|N|N|N|N|N| +2432323|AAAAAAAADENBFCAA|1947-05-17|568|2472|190|1947|6|5|17|2|1947|190|2472|Saturday|1947Q2|N|Y|N|2432307|2432426|2431958|2432233|N|N|N|N|N| +2432324|AAAAAAAAEENBFCAA|1947-05-18|568|2472|190|1947|0|5|18|2|1947|190|2472|Sunday|1947Q2|N|N|N|2432307|2432426|2431959|2432234|N|N|N|N|N| +2432325|AAAAAAAAFENBFCAA|1947-05-19|568|2472|190|1947|1|5|19|2|1947|190|2472|Monday|1947Q2|N|N|N|2432307|2432426|2431960|2432235|N|N|N|N|N| +2432326|AAAAAAAAGENBFCAA|1947-05-20|568|2473|190|1947|2|5|20|2|1947|190|2473|Tuesday|1947Q2|N|N|N|2432307|2432426|2431961|2432236|N|N|N|N|N| +2432327|AAAAAAAAHENBFCAA|1947-05-21|568|2473|190|1947|3|5|21|2|1947|190|2473|Wednesday|1947Q2|N|N|N|2432307|2432426|2431962|2432237|N|N|N|N|N| +2432328|AAAAAAAAIENBFCAA|1947-05-22|568|2473|190|1947|4|5|22|2|1947|190|2473|Thursday|1947Q2|N|N|N|2432307|2432426|2431963|2432238|N|N|N|N|N| +2432329|AAAAAAAAJENBFCAA|1947-05-23|568|2473|190|1947|5|5|23|2|1947|190|2473|Friday|1947Q2|N|Y|N|2432307|2432426|2431964|2432239|N|N|N|N|N| +2432330|AAAAAAAAKENBFCAA|1947-05-24|568|2473|190|1947|6|5|24|2|1947|190|2473|Saturday|1947Q2|N|Y|N|2432307|2432426|2431965|2432240|N|N|N|N|N| +2432331|AAAAAAAALENBFCAA|1947-05-25|568|2473|190|1947|0|5|25|2|1947|190|2473|Sunday|1947Q2|N|N|N|2432307|2432426|2431966|2432241|N|N|N|N|N| +2432332|AAAAAAAAMENBFCAA|1947-05-26|568|2473|190|1947|1|5|26|2|1947|190|2473|Monday|1947Q2|N|N|N|2432307|2432426|2431967|2432242|N|N|N|N|N| +2432333|AAAAAAAANENBFCAA|1947-05-27|568|2474|190|1947|2|5|27|2|1947|190|2474|Tuesday|1947Q2|N|N|N|2432307|2432426|2431968|2432243|N|N|N|N|N| +2432334|AAAAAAAAOENBFCAA|1947-05-28|568|2474|190|1947|3|5|28|2|1947|190|2474|Wednesday|1947Q2|N|N|N|2432307|2432426|2431969|2432244|N|N|N|N|N| +2432335|AAAAAAAAPENBFCAA|1947-05-29|568|2474|190|1947|4|5|29|2|1947|190|2474|Thursday|1947Q2|N|N|N|2432307|2432426|2431970|2432245|N|N|N|N|N| +2432336|AAAAAAAAAFNBFCAA|1947-05-30|568|2474|190|1947|5|5|30|2|1947|190|2474|Friday|1947Q2|N|Y|N|2432307|2432426|2431971|2432246|N|N|N|N|N| +2432337|AAAAAAAABFNBFCAA|1947-05-31|568|2474|190|1947|6|5|31|2|1947|190|2474|Saturday|1947Q2|N|Y|N|2432307|2432426|2431972|2432247|N|N|N|N|N| +2432338|AAAAAAAACFNBFCAA|1947-06-01|569|2474|191|1947|0|6|1|2|1947|191|2474|Sunday|1947Q2|N|N|N|2432338|2432488|2431973|2432248|N|N|N|N|N| +2432339|AAAAAAAADFNBFCAA|1947-06-02|569|2474|191|1947|1|6|2|2|1947|191|2474|Monday|1947Q2|N|N|N|2432338|2432488|2431974|2432249|N|N|N|N|N| +2432340|AAAAAAAAEFNBFCAA|1947-06-03|569|2475|191|1947|2|6|3|2|1947|191|2475|Tuesday|1947Q2|N|N|N|2432338|2432488|2431975|2432250|N|N|N|N|N| +2432341|AAAAAAAAFFNBFCAA|1947-06-04|569|2475|191|1947|3|6|4|2|1947|191|2475|Wednesday|1947Q2|N|N|N|2432338|2432488|2431976|2432251|N|N|N|N|N| +2432342|AAAAAAAAGFNBFCAA|1947-06-05|569|2475|191|1947|4|6|5|2|1947|191|2475|Thursday|1947Q2|N|N|N|2432338|2432488|2431977|2432252|N|N|N|N|N| +2432343|AAAAAAAAHFNBFCAA|1947-06-06|569|2475|191|1947|5|6|6|2|1947|191|2475|Friday|1947Q2|N|Y|N|2432338|2432488|2431978|2432253|N|N|N|N|N| +2432344|AAAAAAAAIFNBFCAA|1947-06-07|569|2475|191|1947|6|6|7|2|1947|191|2475|Saturday|1947Q2|N|Y|N|2432338|2432488|2431979|2432254|N|N|N|N|N| +2432345|AAAAAAAAJFNBFCAA|1947-06-08|569|2475|191|1947|0|6|8|2|1947|191|2475|Sunday|1947Q2|N|N|N|2432338|2432488|2431980|2432255|N|N|N|N|N| +2432346|AAAAAAAAKFNBFCAA|1947-06-09|569|2475|191|1947|1|6|9|2|1947|191|2475|Monday|1947Q2|N|N|N|2432338|2432488|2431981|2432256|N|N|N|N|N| +2432347|AAAAAAAALFNBFCAA|1947-06-10|569|2476|191|1947|2|6|10|2|1947|191|2476|Tuesday|1947Q2|N|N|N|2432338|2432488|2431982|2432257|N|N|N|N|N| +2432348|AAAAAAAAMFNBFCAA|1947-06-11|569|2476|191|1947|3|6|11|2|1947|191|2476|Wednesday|1947Q2|N|N|N|2432338|2432488|2431983|2432258|N|N|N|N|N| +2432349|AAAAAAAANFNBFCAA|1947-06-12|569|2476|191|1947|4|6|12|2|1947|191|2476|Thursday|1947Q2|N|N|N|2432338|2432488|2431984|2432259|N|N|N|N|N| +2432350|AAAAAAAAOFNBFCAA|1947-06-13|569|2476|191|1947|5|6|13|2|1947|191|2476|Friday|1947Q2|N|Y|N|2432338|2432488|2431985|2432260|N|N|N|N|N| +2432351|AAAAAAAAPFNBFCAA|1947-06-14|569|2476|191|1947|6|6|14|2|1947|191|2476|Saturday|1947Q2|N|Y|N|2432338|2432488|2431986|2432261|N|N|N|N|N| +2432352|AAAAAAAAAGNBFCAA|1947-06-15|569|2476|191|1947|0|6|15|2|1947|191|2476|Sunday|1947Q2|N|N|N|2432338|2432488|2431987|2432262|N|N|N|N|N| +2432353|AAAAAAAABGNBFCAA|1947-06-16|569|2476|191|1947|1|6|16|2|1947|191|2476|Monday|1947Q2|N|N|N|2432338|2432488|2431988|2432263|N|N|N|N|N| +2432354|AAAAAAAACGNBFCAA|1947-06-17|569|2477|191|1947|2|6|17|2|1947|191|2477|Tuesday|1947Q2|N|N|N|2432338|2432488|2431989|2432264|N|N|N|N|N| +2432355|AAAAAAAADGNBFCAA|1947-06-18|569|2477|191|1947|3|6|18|2|1947|191|2477|Wednesday|1947Q2|N|N|N|2432338|2432488|2431990|2432265|N|N|N|N|N| +2432356|AAAAAAAAEGNBFCAA|1947-06-19|569|2477|191|1947|4|6|19|2|1947|191|2477|Thursday|1947Q2|N|N|N|2432338|2432488|2431991|2432266|N|N|N|N|N| +2432357|AAAAAAAAFGNBFCAA|1947-06-20|569|2477|191|1947|5|6|20|2|1947|191|2477|Friday|1947Q2|N|Y|N|2432338|2432488|2431992|2432267|N|N|N|N|N| +2432358|AAAAAAAAGGNBFCAA|1947-06-21|569|2477|191|1947|6|6|21|2|1947|191|2477|Saturday|1947Q2|N|Y|N|2432338|2432488|2431993|2432268|N|N|N|N|N| +2432359|AAAAAAAAHGNBFCAA|1947-06-22|569|2477|191|1947|0|6|22|2|1947|191|2477|Sunday|1947Q2|N|N|N|2432338|2432488|2431994|2432269|N|N|N|N|N| +2432360|AAAAAAAAIGNBFCAA|1947-06-23|569|2477|191|1947|1|6|23|2|1947|191|2477|Monday|1947Q2|N|N|N|2432338|2432488|2431995|2432270|N|N|N|N|N| +2432361|AAAAAAAAJGNBFCAA|1947-06-24|569|2478|191|1947|2|6|24|2|1947|191|2478|Tuesday|1947Q2|N|N|N|2432338|2432488|2431996|2432271|N|N|N|N|N| +2432362|AAAAAAAAKGNBFCAA|1947-06-25|569|2478|191|1947|3|6|25|2|1947|191|2478|Wednesday|1947Q2|N|N|N|2432338|2432488|2431997|2432272|N|N|N|N|N| +2432363|AAAAAAAALGNBFCAA|1947-06-26|569|2478|191|1947|4|6|26|2|1947|191|2478|Thursday|1947Q2|N|N|N|2432338|2432488|2431998|2432273|N|N|N|N|N| +2432364|AAAAAAAAMGNBFCAA|1947-06-27|569|2478|191|1947|5|6|27|2|1947|191|2478|Friday|1947Q2|N|Y|N|2432338|2432488|2431999|2432274|N|N|N|N|N| +2432365|AAAAAAAANGNBFCAA|1947-06-28|569|2478|191|1947|6|6|28|2|1947|191|2478|Saturday|1947Q2|N|Y|N|2432338|2432488|2432000|2432275|N|N|N|N|N| +2432366|AAAAAAAAOGNBFCAA|1947-06-29|569|2478|191|1947|0|6|29|2|1947|191|2478|Sunday|1947Q2|N|N|N|2432338|2432488|2432001|2432276|N|N|N|N|N| +2432367|AAAAAAAAPGNBFCAA|1947-06-30|569|2478|191|1947|1|6|30|2|1947|191|2478|Monday|1947Q2|N|N|N|2432338|2432488|2432002|2432277|N|N|N|N|N| +2432368|AAAAAAAAAHNBFCAA|1947-07-01|570|2479|191|1947|2|7|1|2|1947|191|2479|Tuesday|1947Q2|N|N|N|2432368|2432548|2432003|2432277|N|N|N|N|N| +2432369|AAAAAAAABHNBFCAA|1947-07-02|570|2479|191|1947|3|7|2|3|1947|191|2479|Wednesday|1947Q3|N|N|N|2432368|2432548|2432004|2432278|N|N|N|N|N| +2432370|AAAAAAAACHNBFCAA|1947-07-03|570|2479|191|1947|4|7|3|3|1947|191|2479|Thursday|1947Q3|N|N|N|2432368|2432548|2432005|2432279|N|N|N|N|N| +2432371|AAAAAAAADHNBFCAA|1947-07-04|570|2479|191|1947|5|7|4|3|1947|191|2479|Friday|1947Q3|N|Y|N|2432368|2432548|2432006|2432280|N|N|N|N|N| +2432372|AAAAAAAAEHNBFCAA|1947-07-05|570|2479|191|1947|6|7|5|3|1947|191|2479|Saturday|1947Q3|Y|Y|N|2432368|2432548|2432007|2432281|N|N|N|N|N| +2432373|AAAAAAAAFHNBFCAA|1947-07-06|570|2479|191|1947|0|7|6|3|1947|191|2479|Sunday|1947Q3|N|N|Y|2432368|2432548|2432008|2432282|N|N|N|N|N| +2432374|AAAAAAAAGHNBFCAA|1947-07-07|570|2479|191|1947|1|7|7|3|1947|191|2479|Monday|1947Q3|N|N|N|2432368|2432548|2432009|2432283|N|N|N|N|N| +2432375|AAAAAAAAHHNBFCAA|1947-07-08|570|2480|191|1947|2|7|8|3|1947|191|2480|Tuesday|1947Q3|N|N|N|2432368|2432548|2432010|2432284|N|N|N|N|N| +2432376|AAAAAAAAIHNBFCAA|1947-07-09|570|2480|191|1947|3|7|9|3|1947|191|2480|Wednesday|1947Q3|N|N|N|2432368|2432548|2432011|2432285|N|N|N|N|N| +2432377|AAAAAAAAJHNBFCAA|1947-07-10|570|2480|191|1947|4|7|10|3|1947|191|2480|Thursday|1947Q3|N|N|N|2432368|2432548|2432012|2432286|N|N|N|N|N| +2432378|AAAAAAAAKHNBFCAA|1947-07-11|570|2480|191|1947|5|7|11|3|1947|191|2480|Friday|1947Q3|N|Y|N|2432368|2432548|2432013|2432287|N|N|N|N|N| +2432379|AAAAAAAALHNBFCAA|1947-07-12|570|2480|191|1947|6|7|12|3|1947|191|2480|Saturday|1947Q3|N|Y|N|2432368|2432548|2432014|2432288|N|N|N|N|N| +2432380|AAAAAAAAMHNBFCAA|1947-07-13|570|2480|191|1947|0|7|13|3|1947|191|2480|Sunday|1947Q3|N|N|N|2432368|2432548|2432015|2432289|N|N|N|N|N| +2432381|AAAAAAAANHNBFCAA|1947-07-14|570|2480|191|1947|1|7|14|3|1947|191|2480|Monday|1947Q3|N|N|N|2432368|2432548|2432016|2432290|N|N|N|N|N| +2432382|AAAAAAAAOHNBFCAA|1947-07-15|570|2481|191|1947|2|7|15|3|1947|191|2481|Tuesday|1947Q3|N|N|N|2432368|2432548|2432017|2432291|N|N|N|N|N| +2432383|AAAAAAAAPHNBFCAA|1947-07-16|570|2481|191|1947|3|7|16|3|1947|191|2481|Wednesday|1947Q3|N|N|N|2432368|2432548|2432018|2432292|N|N|N|N|N| +2432384|AAAAAAAAAINBFCAA|1947-07-17|570|2481|191|1947|4|7|17|3|1947|191|2481|Thursday|1947Q3|N|N|N|2432368|2432548|2432019|2432293|N|N|N|N|N| +2432385|AAAAAAAABINBFCAA|1947-07-18|570|2481|191|1947|5|7|18|3|1947|191|2481|Friday|1947Q3|N|Y|N|2432368|2432548|2432020|2432294|N|N|N|N|N| +2432386|AAAAAAAACINBFCAA|1947-07-19|570|2481|191|1947|6|7|19|3|1947|191|2481|Saturday|1947Q3|N|Y|N|2432368|2432548|2432021|2432295|N|N|N|N|N| +2432387|AAAAAAAADINBFCAA|1947-07-20|570|2481|191|1947|0|7|20|3|1947|191|2481|Sunday|1947Q3|N|N|N|2432368|2432548|2432022|2432296|N|N|N|N|N| +2432388|AAAAAAAAEINBFCAA|1947-07-21|570|2481|191|1947|1|7|21|3|1947|191|2481|Monday|1947Q3|N|N|N|2432368|2432548|2432023|2432297|N|N|N|N|N| +2432389|AAAAAAAAFINBFCAA|1947-07-22|570|2482|191|1947|2|7|22|3|1947|191|2482|Tuesday|1947Q3|N|N|N|2432368|2432548|2432024|2432298|N|N|N|N|N| +2432390|AAAAAAAAGINBFCAA|1947-07-23|570|2482|191|1947|3|7|23|3|1947|191|2482|Wednesday|1947Q3|N|N|N|2432368|2432548|2432025|2432299|N|N|N|N|N| +2432391|AAAAAAAAHINBFCAA|1947-07-24|570|2482|191|1947|4|7|24|3|1947|191|2482|Thursday|1947Q3|N|N|N|2432368|2432548|2432026|2432300|N|N|N|N|N| +2432392|AAAAAAAAIINBFCAA|1947-07-25|570|2482|191|1947|5|7|25|3|1947|191|2482|Friday|1947Q3|N|Y|N|2432368|2432548|2432027|2432301|N|N|N|N|N| +2432393|AAAAAAAAJINBFCAA|1947-07-26|570|2482|191|1947|6|7|26|3|1947|191|2482|Saturday|1947Q3|N|Y|N|2432368|2432548|2432028|2432302|N|N|N|N|N| +2432394|AAAAAAAAKINBFCAA|1947-07-27|570|2482|191|1947|0|7|27|3|1947|191|2482|Sunday|1947Q3|N|N|N|2432368|2432548|2432029|2432303|N|N|N|N|N| +2432395|AAAAAAAALINBFCAA|1947-07-28|570|2482|191|1947|1|7|28|3|1947|191|2482|Monday|1947Q3|N|N|N|2432368|2432548|2432030|2432304|N|N|N|N|N| +2432396|AAAAAAAAMINBFCAA|1947-07-29|570|2483|191|1947|2|7|29|3|1947|191|2483|Tuesday|1947Q3|N|N|N|2432368|2432548|2432031|2432305|N|N|N|N|N| +2432397|AAAAAAAANINBFCAA|1947-07-30|570|2483|191|1947|3|7|30|3|1947|191|2483|Wednesday|1947Q3|N|N|N|2432368|2432548|2432032|2432306|N|N|N|N|N| +2432398|AAAAAAAAOINBFCAA|1947-07-31|570|2483|191|1947|4|7|31|3|1947|191|2483|Thursday|1947Q3|N|N|N|2432368|2432548|2432033|2432307|N|N|N|N|N| +2432399|AAAAAAAAPINBFCAA|1947-08-01|571|2483|191|1947|5|8|1|3|1947|191|2483|Friday|1947Q3|N|Y|N|2432399|2432610|2432034|2432308|N|N|N|N|N| +2432400|AAAAAAAAAJNBFCAA|1947-08-02|571|2483|191|1947|6|8|2|3|1947|191|2483|Saturday|1947Q3|N|Y|N|2432399|2432610|2432035|2432309|N|N|N|N|N| +2432401|AAAAAAAABJNBFCAA|1947-08-03|571|2483|191|1947|0|8|3|3|1947|191|2483|Sunday|1947Q3|N|N|N|2432399|2432610|2432036|2432310|N|N|N|N|N| +2432402|AAAAAAAACJNBFCAA|1947-08-04|571|2483|191|1947|1|8|4|3|1947|191|2483|Monday|1947Q3|N|N|N|2432399|2432610|2432037|2432311|N|N|N|N|N| +2432403|AAAAAAAADJNBFCAA|1947-08-05|571|2484|191|1947|2|8|5|3|1947|191|2484|Tuesday|1947Q3|N|N|N|2432399|2432610|2432038|2432312|N|N|N|N|N| +2432404|AAAAAAAAEJNBFCAA|1947-08-06|571|2484|191|1947|3|8|6|3|1947|191|2484|Wednesday|1947Q3|N|N|N|2432399|2432610|2432039|2432313|N|N|N|N|N| +2432405|AAAAAAAAFJNBFCAA|1947-08-07|571|2484|191|1947|4|8|7|3|1947|191|2484|Thursday|1947Q3|N|N|N|2432399|2432610|2432040|2432314|N|N|N|N|N| +2432406|AAAAAAAAGJNBFCAA|1947-08-08|571|2484|191|1947|5|8|8|3|1947|191|2484|Friday|1947Q3|N|Y|N|2432399|2432610|2432041|2432315|N|N|N|N|N| +2432407|AAAAAAAAHJNBFCAA|1947-08-09|571|2484|191|1947|6|8|9|3|1947|191|2484|Saturday|1947Q3|N|Y|N|2432399|2432610|2432042|2432316|N|N|N|N|N| +2432408|AAAAAAAAIJNBFCAA|1947-08-10|571|2484|191|1947|0|8|10|3|1947|191|2484|Sunday|1947Q3|N|N|N|2432399|2432610|2432043|2432317|N|N|N|N|N| +2432409|AAAAAAAAJJNBFCAA|1947-08-11|571|2484|191|1947|1|8|11|3|1947|191|2484|Monday|1947Q3|N|N|N|2432399|2432610|2432044|2432318|N|N|N|N|N| +2432410|AAAAAAAAKJNBFCAA|1947-08-12|571|2485|191|1947|2|8|12|3|1947|191|2485|Tuesday|1947Q3|N|N|N|2432399|2432610|2432045|2432319|N|N|N|N|N| +2432411|AAAAAAAALJNBFCAA|1947-08-13|571|2485|191|1947|3|8|13|3|1947|191|2485|Wednesday|1947Q3|N|N|N|2432399|2432610|2432046|2432320|N|N|N|N|N| +2432412|AAAAAAAAMJNBFCAA|1947-08-14|571|2485|191|1947|4|8|14|3|1947|191|2485|Thursday|1947Q3|N|N|N|2432399|2432610|2432047|2432321|N|N|N|N|N| +2432413|AAAAAAAANJNBFCAA|1947-08-15|571|2485|191|1947|5|8|15|3|1947|191|2485|Friday|1947Q3|N|Y|N|2432399|2432610|2432048|2432322|N|N|N|N|N| +2432414|AAAAAAAAOJNBFCAA|1947-08-16|571|2485|191|1947|6|8|16|3|1947|191|2485|Saturday|1947Q3|N|Y|N|2432399|2432610|2432049|2432323|N|N|N|N|N| +2432415|AAAAAAAAPJNBFCAA|1947-08-17|571|2485|191|1947|0|8|17|3|1947|191|2485|Sunday|1947Q3|N|N|N|2432399|2432610|2432050|2432324|N|N|N|N|N| +2432416|AAAAAAAAAKNBFCAA|1947-08-18|571|2485|191|1947|1|8|18|3|1947|191|2485|Monday|1947Q3|N|N|N|2432399|2432610|2432051|2432325|N|N|N|N|N| +2432417|AAAAAAAABKNBFCAA|1947-08-19|571|2486|191|1947|2|8|19|3|1947|191|2486|Tuesday|1947Q3|N|N|N|2432399|2432610|2432052|2432326|N|N|N|N|N| +2432418|AAAAAAAACKNBFCAA|1947-08-20|571|2486|191|1947|3|8|20|3|1947|191|2486|Wednesday|1947Q3|N|N|N|2432399|2432610|2432053|2432327|N|N|N|N|N| +2432419|AAAAAAAADKNBFCAA|1947-08-21|571|2486|191|1947|4|8|21|3|1947|191|2486|Thursday|1947Q3|N|N|N|2432399|2432610|2432054|2432328|N|N|N|N|N| +2432420|AAAAAAAAEKNBFCAA|1947-08-22|571|2486|191|1947|5|8|22|3|1947|191|2486|Friday|1947Q3|N|Y|N|2432399|2432610|2432055|2432329|N|N|N|N|N| +2432421|AAAAAAAAFKNBFCAA|1947-08-23|571|2486|191|1947|6|8|23|3|1947|191|2486|Saturday|1947Q3|N|Y|N|2432399|2432610|2432056|2432330|N|N|N|N|N| +2432422|AAAAAAAAGKNBFCAA|1947-08-24|571|2486|191|1947|0|8|24|3|1947|191|2486|Sunday|1947Q3|N|N|N|2432399|2432610|2432057|2432331|N|N|N|N|N| +2432423|AAAAAAAAHKNBFCAA|1947-08-25|571|2486|191|1947|1|8|25|3|1947|191|2486|Monday|1947Q3|N|N|N|2432399|2432610|2432058|2432332|N|N|N|N|N| +2432424|AAAAAAAAIKNBFCAA|1947-08-26|571|2487|191|1947|2|8|26|3|1947|191|2487|Tuesday|1947Q3|N|N|N|2432399|2432610|2432059|2432333|N|N|N|N|N| +2432425|AAAAAAAAJKNBFCAA|1947-08-27|571|2487|191|1947|3|8|27|3|1947|191|2487|Wednesday|1947Q3|N|N|N|2432399|2432610|2432060|2432334|N|N|N|N|N| +2432426|AAAAAAAAKKNBFCAA|1947-08-28|571|2487|191|1947|4|8|28|3|1947|191|2487|Thursday|1947Q3|N|N|N|2432399|2432610|2432061|2432335|N|N|N|N|N| +2432427|AAAAAAAALKNBFCAA|1947-08-29|571|2487|191|1947|5|8|29|3|1947|191|2487|Friday|1947Q3|N|Y|N|2432399|2432610|2432062|2432336|N|N|N|N|N| +2432428|AAAAAAAAMKNBFCAA|1947-08-30|571|2487|191|1947|6|8|30|3|1947|191|2487|Saturday|1947Q3|N|Y|N|2432399|2432610|2432063|2432337|N|N|N|N|N| +2432429|AAAAAAAANKNBFCAA|1947-08-31|571|2487|191|1947|0|8|31|3|1947|191|2487|Sunday|1947Q3|N|N|N|2432399|2432610|2432064|2432338|N|N|N|N|N| +2432430|AAAAAAAAOKNBFCAA|1947-09-01|572|2487|192|1947|1|9|1|3|1947|192|2487|Monday|1947Q3|N|N|N|2432430|2432672|2432065|2432339|N|N|N|N|N| +2432431|AAAAAAAAPKNBFCAA|1947-09-02|572|2488|192|1947|2|9|2|3|1947|192|2488|Tuesday|1947Q3|N|N|N|2432430|2432672|2432066|2432340|N|N|N|N|N| +2432432|AAAAAAAAALNBFCAA|1947-09-03|572|2488|192|1947|3|9|3|3|1947|192|2488|Wednesday|1947Q3|N|N|N|2432430|2432672|2432067|2432341|N|N|N|N|N| +2432433|AAAAAAAABLNBFCAA|1947-09-04|572|2488|192|1947|4|9|4|3|1947|192|2488|Thursday|1947Q3|N|N|N|2432430|2432672|2432068|2432342|N|N|N|N|N| +2432434|AAAAAAAACLNBFCAA|1947-09-05|572|2488|192|1947|5|9|5|3|1947|192|2488|Friday|1947Q3|N|Y|N|2432430|2432672|2432069|2432343|N|N|N|N|N| +2432435|AAAAAAAADLNBFCAA|1947-09-06|572|2488|192|1947|6|9|6|3|1947|192|2488|Saturday|1947Q3|N|Y|N|2432430|2432672|2432070|2432344|N|N|N|N|N| +2432436|AAAAAAAAELNBFCAA|1947-09-07|572|2488|192|1947|0|9|7|3|1947|192|2488|Sunday|1947Q3|N|N|N|2432430|2432672|2432071|2432345|N|N|N|N|N| +2432437|AAAAAAAAFLNBFCAA|1947-09-08|572|2488|192|1947|1|9|8|3|1947|192|2488|Monday|1947Q3|N|N|N|2432430|2432672|2432072|2432346|N|N|N|N|N| +2432438|AAAAAAAAGLNBFCAA|1947-09-09|572|2489|192|1947|2|9|9|3|1947|192|2489|Tuesday|1947Q3|N|N|N|2432430|2432672|2432073|2432347|N|N|N|N|N| +2432439|AAAAAAAAHLNBFCAA|1947-09-10|572|2489|192|1947|3|9|10|3|1947|192|2489|Wednesday|1947Q3|N|N|N|2432430|2432672|2432074|2432348|N|N|N|N|N| +2432440|AAAAAAAAILNBFCAA|1947-09-11|572|2489|192|1947|4|9|11|3|1947|192|2489|Thursday|1947Q3|N|N|N|2432430|2432672|2432075|2432349|N|N|N|N|N| +2432441|AAAAAAAAJLNBFCAA|1947-09-12|572|2489|192|1947|5|9|12|3|1947|192|2489|Friday|1947Q3|N|Y|N|2432430|2432672|2432076|2432350|N|N|N|N|N| +2432442|AAAAAAAAKLNBFCAA|1947-09-13|572|2489|192|1947|6|9|13|3|1947|192|2489|Saturday|1947Q3|N|Y|N|2432430|2432672|2432077|2432351|N|N|N|N|N| +2432443|AAAAAAAALLNBFCAA|1947-09-14|572|2489|192|1947|0|9|14|3|1947|192|2489|Sunday|1947Q3|N|N|N|2432430|2432672|2432078|2432352|N|N|N|N|N| +2432444|AAAAAAAAMLNBFCAA|1947-09-15|572|2489|192|1947|1|9|15|3|1947|192|2489|Monday|1947Q3|N|N|N|2432430|2432672|2432079|2432353|N|N|N|N|N| +2432445|AAAAAAAANLNBFCAA|1947-09-16|572|2490|192|1947|2|9|16|3|1947|192|2490|Tuesday|1947Q3|N|N|N|2432430|2432672|2432080|2432354|N|N|N|N|N| +2432446|AAAAAAAAOLNBFCAA|1947-09-17|572|2490|192|1947|3|9|17|3|1947|192|2490|Wednesday|1947Q3|N|N|N|2432430|2432672|2432081|2432355|N|N|N|N|N| +2432447|AAAAAAAAPLNBFCAA|1947-09-18|572|2490|192|1947|4|9|18|3|1947|192|2490|Thursday|1947Q3|N|N|N|2432430|2432672|2432082|2432356|N|N|N|N|N| +2432448|AAAAAAAAAMNBFCAA|1947-09-19|572|2490|192|1947|5|9|19|3|1947|192|2490|Friday|1947Q3|N|Y|N|2432430|2432672|2432083|2432357|N|N|N|N|N| +2432449|AAAAAAAABMNBFCAA|1947-09-20|572|2490|192|1947|6|9|20|3|1947|192|2490|Saturday|1947Q3|N|Y|N|2432430|2432672|2432084|2432358|N|N|N|N|N| +2432450|AAAAAAAACMNBFCAA|1947-09-21|572|2490|192|1947|0|9|21|3|1947|192|2490|Sunday|1947Q3|N|N|N|2432430|2432672|2432085|2432359|N|N|N|N|N| +2432451|AAAAAAAADMNBFCAA|1947-09-22|572|2490|192|1947|1|9|22|3|1947|192|2490|Monday|1947Q3|N|N|N|2432430|2432672|2432086|2432360|N|N|N|N|N| +2432452|AAAAAAAAEMNBFCAA|1947-09-23|572|2491|192|1947|2|9|23|3|1947|192|2491|Tuesday|1947Q3|N|N|N|2432430|2432672|2432087|2432361|N|N|N|N|N| +2432453|AAAAAAAAFMNBFCAA|1947-09-24|572|2491|192|1947|3|9|24|3|1947|192|2491|Wednesday|1947Q3|N|N|N|2432430|2432672|2432088|2432362|N|N|N|N|N| +2432454|AAAAAAAAGMNBFCAA|1947-09-25|572|2491|192|1947|4|9|25|3|1947|192|2491|Thursday|1947Q3|N|N|N|2432430|2432672|2432089|2432363|N|N|N|N|N| +2432455|AAAAAAAAHMNBFCAA|1947-09-26|572|2491|192|1947|5|9|26|3|1947|192|2491|Friday|1947Q3|N|Y|N|2432430|2432672|2432090|2432364|N|N|N|N|N| +2432456|AAAAAAAAIMNBFCAA|1947-09-27|572|2491|192|1947|6|9|27|3|1947|192|2491|Saturday|1947Q3|N|Y|N|2432430|2432672|2432091|2432365|N|N|N|N|N| +2432457|AAAAAAAAJMNBFCAA|1947-09-28|572|2491|192|1947|0|9|28|3|1947|192|2491|Sunday|1947Q3|N|N|N|2432430|2432672|2432092|2432366|N|N|N|N|N| +2432458|AAAAAAAAKMNBFCAA|1947-09-29|572|2491|192|1947|1|9|29|3|1947|192|2491|Monday|1947Q3|N|N|N|2432430|2432672|2432093|2432367|N|N|N|N|N| +2432459|AAAAAAAALMNBFCAA|1947-09-30|572|2492|192|1947|2|9|30|3|1947|192|2492|Tuesday|1947Q3|N|N|N|2432430|2432672|2432094|2432368|N|N|N|N|N| +2432460|AAAAAAAAMMNBFCAA|1947-10-01|573|2492|192|1947|3|10|1|3|1947|192|2492|Wednesday|1947Q3|N|N|N|2432460|2432732|2432095|2432368|N|N|N|N|N| +2432461|AAAAAAAANMNBFCAA|1947-10-02|573|2492|192|1947|4|10|2|4|1947|192|2492|Thursday|1947Q4|N|N|N|2432460|2432732|2432096|2432369|N|N|N|N|N| +2432462|AAAAAAAAOMNBFCAA|1947-10-03|573|2492|192|1947|5|10|3|4|1947|192|2492|Friday|1947Q4|N|Y|N|2432460|2432732|2432097|2432370|N|N|N|N|N| +2432463|AAAAAAAAPMNBFCAA|1947-10-04|573|2492|192|1947|6|10|4|4|1947|192|2492|Saturday|1947Q4|N|Y|N|2432460|2432732|2432098|2432371|N|N|N|N|N| +2432464|AAAAAAAAANNBFCAA|1947-10-05|573|2492|192|1947|0|10|5|4|1947|192|2492|Sunday|1947Q4|N|N|N|2432460|2432732|2432099|2432372|N|N|N|N|N| +2432465|AAAAAAAABNNBFCAA|1947-10-06|573|2492|192|1947|1|10|6|4|1947|192|2492|Monday|1947Q4|N|N|N|2432460|2432732|2432100|2432373|N|N|N|N|N| +2432466|AAAAAAAACNNBFCAA|1947-10-07|573|2493|192|1947|2|10|7|4|1947|192|2493|Tuesday|1947Q4|N|N|N|2432460|2432732|2432101|2432374|N|N|N|N|N| +2432467|AAAAAAAADNNBFCAA|1947-10-08|573|2493|192|1947|3|10|8|4|1947|192|2493|Wednesday|1947Q4|N|N|N|2432460|2432732|2432102|2432375|N|N|N|N|N| +2432468|AAAAAAAAENNBFCAA|1947-10-09|573|2493|192|1947|4|10|9|4|1947|192|2493|Thursday|1947Q4|N|N|N|2432460|2432732|2432103|2432376|N|N|N|N|N| +2432469|AAAAAAAAFNNBFCAA|1947-10-10|573|2493|192|1947|5|10|10|4|1947|192|2493|Friday|1947Q4|N|Y|N|2432460|2432732|2432104|2432377|N|N|N|N|N| +2432470|AAAAAAAAGNNBFCAA|1947-10-11|573|2493|192|1947|6|10|11|4|1947|192|2493|Saturday|1947Q4|N|Y|N|2432460|2432732|2432105|2432378|N|N|N|N|N| +2432471|AAAAAAAAHNNBFCAA|1947-10-12|573|2493|192|1947|0|10|12|4|1947|192|2493|Sunday|1947Q4|N|N|N|2432460|2432732|2432106|2432379|N|N|N|N|N| +2432472|AAAAAAAAINNBFCAA|1947-10-13|573|2493|192|1947|1|10|13|4|1947|192|2493|Monday|1947Q4|N|N|N|2432460|2432732|2432107|2432380|N|N|N|N|N| +2432473|AAAAAAAAJNNBFCAA|1947-10-14|573|2494|192|1947|2|10|14|4|1947|192|2494|Tuesday|1947Q4|N|N|N|2432460|2432732|2432108|2432381|N|N|N|N|N| +2432474|AAAAAAAAKNNBFCAA|1947-10-15|573|2494|192|1947|3|10|15|4|1947|192|2494|Wednesday|1947Q4|N|N|N|2432460|2432732|2432109|2432382|N|N|N|N|N| +2432475|AAAAAAAALNNBFCAA|1947-10-16|573|2494|192|1947|4|10|16|4|1947|192|2494|Thursday|1947Q4|N|N|N|2432460|2432732|2432110|2432383|N|N|N|N|N| +2432476|AAAAAAAAMNNBFCAA|1947-10-17|573|2494|192|1947|5|10|17|4|1947|192|2494|Friday|1947Q4|N|Y|N|2432460|2432732|2432111|2432384|N|N|N|N|N| +2432477|AAAAAAAANNNBFCAA|1947-10-18|573|2494|192|1947|6|10|18|4|1947|192|2494|Saturday|1947Q4|N|Y|N|2432460|2432732|2432112|2432385|N|N|N|N|N| +2432478|AAAAAAAAONNBFCAA|1947-10-19|573|2494|192|1947|0|10|19|4|1947|192|2494|Sunday|1947Q4|N|N|N|2432460|2432732|2432113|2432386|N|N|N|N|N| +2432479|AAAAAAAAPNNBFCAA|1947-10-20|573|2494|192|1947|1|10|20|4|1947|192|2494|Monday|1947Q4|N|N|N|2432460|2432732|2432114|2432387|N|N|N|N|N| +2432480|AAAAAAAAAONBFCAA|1947-10-21|573|2495|192|1947|2|10|21|4|1947|192|2495|Tuesday|1947Q4|N|N|N|2432460|2432732|2432115|2432388|N|N|N|N|N| +2432481|AAAAAAAABONBFCAA|1947-10-22|573|2495|192|1947|3|10|22|4|1947|192|2495|Wednesday|1947Q4|N|N|N|2432460|2432732|2432116|2432389|N|N|N|N|N| +2432482|AAAAAAAACONBFCAA|1947-10-23|573|2495|192|1947|4|10|23|4|1947|192|2495|Thursday|1947Q4|N|N|N|2432460|2432732|2432117|2432390|N|N|N|N|N| +2432483|AAAAAAAADONBFCAA|1947-10-24|573|2495|192|1947|5|10|24|4|1947|192|2495|Friday|1947Q4|N|Y|N|2432460|2432732|2432118|2432391|N|N|N|N|N| +2432484|AAAAAAAAEONBFCAA|1947-10-25|573|2495|192|1947|6|10|25|4|1947|192|2495|Saturday|1947Q4|N|Y|N|2432460|2432732|2432119|2432392|N|N|N|N|N| +2432485|AAAAAAAAFONBFCAA|1947-10-26|573|2495|192|1947|0|10|26|4|1947|192|2495|Sunday|1947Q4|N|N|N|2432460|2432732|2432120|2432393|N|N|N|N|N| +2432486|AAAAAAAAGONBFCAA|1947-10-27|573|2495|192|1947|1|10|27|4|1947|192|2495|Monday|1947Q4|N|N|N|2432460|2432732|2432121|2432394|N|N|N|N|N| +2432487|AAAAAAAAHONBFCAA|1947-10-28|573|2496|192|1947|2|10|28|4|1947|192|2496|Tuesday|1947Q4|N|N|N|2432460|2432732|2432122|2432395|N|N|N|N|N| +2432488|AAAAAAAAIONBFCAA|1947-10-29|573|2496|192|1947|3|10|29|4|1947|192|2496|Wednesday|1947Q4|N|N|N|2432460|2432732|2432123|2432396|N|N|N|N|N| +2432489|AAAAAAAAJONBFCAA|1947-10-30|573|2496|192|1947|4|10|30|4|1947|192|2496|Thursday|1947Q4|N|N|N|2432460|2432732|2432124|2432397|N|N|N|N|N| +2432490|AAAAAAAAKONBFCAA|1947-10-31|573|2496|192|1947|5|10|31|4|1947|192|2496|Friday|1947Q4|N|Y|N|2432460|2432732|2432125|2432398|N|N|N|N|N| +2432491|AAAAAAAALONBFCAA|1947-11-01|574|2496|192|1947|6|11|1|4|1947|192|2496|Saturday|1947Q4|N|Y|N|2432491|2432794|2432126|2432399|N|N|N|N|N| +2432492|AAAAAAAAMONBFCAA|1947-11-02|574|2496|192|1947|0|11|2|4|1947|192|2496|Sunday|1947Q4|N|N|N|2432491|2432794|2432127|2432400|N|N|N|N|N| +2432493|AAAAAAAANONBFCAA|1947-11-03|574|2496|192|1947|1|11|3|4|1947|192|2496|Monday|1947Q4|N|N|N|2432491|2432794|2432128|2432401|N|N|N|N|N| +2432494|AAAAAAAAOONBFCAA|1947-11-04|574|2497|192|1947|2|11|4|4|1947|192|2497|Tuesday|1947Q4|N|N|N|2432491|2432794|2432129|2432402|N|N|N|N|N| +2432495|AAAAAAAAPONBFCAA|1947-11-05|574|2497|192|1947|3|11|5|4|1947|192|2497|Wednesday|1947Q4|N|N|N|2432491|2432794|2432130|2432403|N|N|N|N|N| +2432496|AAAAAAAAAPNBFCAA|1947-11-06|574|2497|192|1947|4|11|6|4|1947|192|2497|Thursday|1947Q4|N|N|N|2432491|2432794|2432131|2432404|N|N|N|N|N| +2432497|AAAAAAAABPNBFCAA|1947-11-07|574|2497|192|1947|5|11|7|4|1947|192|2497|Friday|1947Q4|N|Y|N|2432491|2432794|2432132|2432405|N|N|N|N|N| +2432498|AAAAAAAACPNBFCAA|1947-11-08|574|2497|192|1947|6|11|8|4|1947|192|2497|Saturday|1947Q4|N|Y|N|2432491|2432794|2432133|2432406|N|N|N|N|N| +2432499|AAAAAAAADPNBFCAA|1947-11-09|574|2497|192|1947|0|11|9|4|1947|192|2497|Sunday|1947Q4|N|N|N|2432491|2432794|2432134|2432407|N|N|N|N|N| +2432500|AAAAAAAAEPNBFCAA|1947-11-10|574|2497|192|1947|1|11|10|4|1947|192|2497|Monday|1947Q4|N|N|N|2432491|2432794|2432135|2432408|N|N|N|N|N| +2432501|AAAAAAAAFPNBFCAA|1947-11-11|574|2498|192|1947|2|11|11|4|1947|192|2498|Tuesday|1947Q4|N|N|N|2432491|2432794|2432136|2432409|N|N|N|N|N| +2432502|AAAAAAAAGPNBFCAA|1947-11-12|574|2498|192|1947|3|11|12|4|1947|192|2498|Wednesday|1947Q4|N|N|N|2432491|2432794|2432137|2432410|N|N|N|N|N| +2432503|AAAAAAAAHPNBFCAA|1947-11-13|574|2498|192|1947|4|11|13|4|1947|192|2498|Thursday|1947Q4|N|N|N|2432491|2432794|2432138|2432411|N|N|N|N|N| +2432504|AAAAAAAAIPNBFCAA|1947-11-14|574|2498|192|1947|5|11|14|4|1947|192|2498|Friday|1947Q4|N|Y|N|2432491|2432794|2432139|2432412|N|N|N|N|N| +2432505|AAAAAAAAJPNBFCAA|1947-11-15|574|2498|192|1947|6|11|15|4|1947|192|2498|Saturday|1947Q4|N|Y|N|2432491|2432794|2432140|2432413|N|N|N|N|N| +2432506|AAAAAAAAKPNBFCAA|1947-11-16|574|2498|192|1947|0|11|16|4|1947|192|2498|Sunday|1947Q4|N|N|N|2432491|2432794|2432141|2432414|N|N|N|N|N| +2432507|AAAAAAAALPNBFCAA|1947-11-17|574|2498|192|1947|1|11|17|4|1947|192|2498|Monday|1947Q4|N|N|N|2432491|2432794|2432142|2432415|N|N|N|N|N| +2432508|AAAAAAAAMPNBFCAA|1947-11-18|574|2499|192|1947|2|11|18|4|1947|192|2499|Tuesday|1947Q4|N|N|N|2432491|2432794|2432143|2432416|N|N|N|N|N| +2432509|AAAAAAAANPNBFCAA|1947-11-19|574|2499|192|1947|3|11|19|4|1947|192|2499|Wednesday|1947Q4|N|N|N|2432491|2432794|2432144|2432417|N|N|N|N|N| +2432510|AAAAAAAAOPNBFCAA|1947-11-20|574|2499|192|1947|4|11|20|4|1947|192|2499|Thursday|1947Q4|N|N|N|2432491|2432794|2432145|2432418|N|N|N|N|N| +2432511|AAAAAAAAPPNBFCAA|1947-11-21|574|2499|192|1947|5|11|21|4|1947|192|2499|Friday|1947Q4|N|Y|N|2432491|2432794|2432146|2432419|N|N|N|N|N| +2432512|AAAAAAAAAAOBFCAA|1947-11-22|574|2499|192|1947|6|11|22|4|1947|192|2499|Saturday|1947Q4|N|Y|N|2432491|2432794|2432147|2432420|N|N|N|N|N| +2432513|AAAAAAAABAOBFCAA|1947-11-23|574|2499|192|1947|0|11|23|4|1947|192|2499|Sunday|1947Q4|N|N|N|2432491|2432794|2432148|2432421|N|N|N|N|N| +2432514|AAAAAAAACAOBFCAA|1947-11-24|574|2499|192|1947|1|11|24|4|1947|192|2499|Monday|1947Q4|N|N|N|2432491|2432794|2432149|2432422|N|N|N|N|N| +2432515|AAAAAAAADAOBFCAA|1947-11-25|574|2500|192|1947|2|11|25|4|1947|192|2500|Tuesday|1947Q4|N|N|N|2432491|2432794|2432150|2432423|N|N|N|N|N| +2432516|AAAAAAAAEAOBFCAA|1947-11-26|574|2500|192|1947|3|11|26|4|1947|192|2500|Wednesday|1947Q4|N|N|N|2432491|2432794|2432151|2432424|N|N|N|N|N| +2432517|AAAAAAAAFAOBFCAA|1947-11-27|574|2500|192|1947|4|11|27|4|1947|192|2500|Thursday|1947Q4|N|N|N|2432491|2432794|2432152|2432425|N|N|N|N|N| +2432518|AAAAAAAAGAOBFCAA|1947-11-28|574|2500|192|1947|5|11|28|4|1947|192|2500|Friday|1947Q4|N|Y|N|2432491|2432794|2432153|2432426|N|N|N|N|N| +2432519|AAAAAAAAHAOBFCAA|1947-11-29|574|2500|192|1947|6|11|29|4|1947|192|2500|Saturday|1947Q4|N|Y|N|2432491|2432794|2432154|2432427|N|N|N|N|N| +2432520|AAAAAAAAIAOBFCAA|1947-11-30|574|2500|192|1947|0|11|30|4|1947|192|2500|Sunday|1947Q4|N|N|N|2432491|2432794|2432155|2432428|N|N|N|N|N| +2432521|AAAAAAAAJAOBFCAA|1947-12-01|575|2500|193|1947|1|12|1|4|1947|193|2500|Monday|1947Q4|N|N|N|2432521|2432854|2432156|2432429|N|N|N|N|N| +2432522|AAAAAAAAKAOBFCAA|1947-12-02|575|2501|193|1947|2|12|2|4|1947|193|2501|Tuesday|1947Q4|N|N|N|2432521|2432854|2432157|2432430|N|N|N|N|N| +2432523|AAAAAAAALAOBFCAA|1947-12-03|575|2501|193|1947|3|12|3|4|1947|193|2501|Wednesday|1947Q4|N|N|N|2432521|2432854|2432158|2432431|N|N|N|N|N| +2432524|AAAAAAAAMAOBFCAA|1947-12-04|575|2501|193|1947|4|12|4|4|1947|193|2501|Thursday|1947Q4|N|N|N|2432521|2432854|2432159|2432432|N|N|N|N|N| +2432525|AAAAAAAANAOBFCAA|1947-12-05|575|2501|193|1947|5|12|5|4|1947|193|2501|Friday|1947Q4|N|Y|N|2432521|2432854|2432160|2432433|N|N|N|N|N| +2432526|AAAAAAAAOAOBFCAA|1947-12-06|575|2501|193|1947|6|12|6|4|1947|193|2501|Saturday|1947Q4|N|Y|N|2432521|2432854|2432161|2432434|N|N|N|N|N| +2432527|AAAAAAAAPAOBFCAA|1947-12-07|575|2501|193|1947|0|12|7|4|1947|193|2501|Sunday|1947Q4|N|N|N|2432521|2432854|2432162|2432435|N|N|N|N|N| +2432528|AAAAAAAAABOBFCAA|1947-12-08|575|2501|193|1947|1|12|8|4|1947|193|2501|Monday|1947Q4|N|N|N|2432521|2432854|2432163|2432436|N|N|N|N|N| +2432529|AAAAAAAABBOBFCAA|1947-12-09|575|2502|193|1947|2|12|9|4|1947|193|2502|Tuesday|1947Q4|N|N|N|2432521|2432854|2432164|2432437|N|N|N|N|N| +2432530|AAAAAAAACBOBFCAA|1947-12-10|575|2502|193|1947|3|12|10|4|1947|193|2502|Wednesday|1947Q4|N|N|N|2432521|2432854|2432165|2432438|N|N|N|N|N| +2432531|AAAAAAAADBOBFCAA|1947-12-11|575|2502|193|1947|4|12|11|4|1947|193|2502|Thursday|1947Q4|N|N|N|2432521|2432854|2432166|2432439|N|N|N|N|N| +2432532|AAAAAAAAEBOBFCAA|1947-12-12|575|2502|193|1947|5|12|12|4|1947|193|2502|Friday|1947Q4|N|Y|N|2432521|2432854|2432167|2432440|N|N|N|N|N| +2432533|AAAAAAAAFBOBFCAA|1947-12-13|575|2502|193|1947|6|12|13|4|1947|193|2502|Saturday|1947Q4|N|Y|N|2432521|2432854|2432168|2432441|N|N|N|N|N| +2432534|AAAAAAAAGBOBFCAA|1947-12-14|575|2502|193|1947|0|12|14|4|1947|193|2502|Sunday|1947Q4|N|N|N|2432521|2432854|2432169|2432442|N|N|N|N|N| +2432535|AAAAAAAAHBOBFCAA|1947-12-15|575|2502|193|1947|1|12|15|4|1947|193|2502|Monday|1947Q4|N|N|N|2432521|2432854|2432170|2432443|N|N|N|N|N| +2432536|AAAAAAAAIBOBFCAA|1947-12-16|575|2503|193|1947|2|12|16|4|1947|193|2503|Tuesday|1947Q4|N|N|N|2432521|2432854|2432171|2432444|N|N|N|N|N| +2432537|AAAAAAAAJBOBFCAA|1947-12-17|575|2503|193|1947|3|12|17|4|1947|193|2503|Wednesday|1947Q4|N|N|N|2432521|2432854|2432172|2432445|N|N|N|N|N| +2432538|AAAAAAAAKBOBFCAA|1947-12-18|575|2503|193|1947|4|12|18|4|1947|193|2503|Thursday|1947Q4|N|N|N|2432521|2432854|2432173|2432446|N|N|N|N|N| +2432539|AAAAAAAALBOBFCAA|1947-12-19|575|2503|193|1947|5|12|19|4|1947|193|2503|Friday|1947Q4|N|Y|N|2432521|2432854|2432174|2432447|N|N|N|N|N| +2432540|AAAAAAAAMBOBFCAA|1947-12-20|575|2503|193|1947|6|12|20|4|1947|193|2503|Saturday|1947Q4|N|Y|N|2432521|2432854|2432175|2432448|N|N|N|N|N| +2432541|AAAAAAAANBOBFCAA|1947-12-21|575|2503|193|1947|0|12|21|4|1947|193|2503|Sunday|1947Q4|N|N|N|2432521|2432854|2432176|2432449|N|N|N|N|N| +2432542|AAAAAAAAOBOBFCAA|1947-12-22|575|2503|193|1947|1|12|22|4|1947|193|2503|Monday|1947Q4|N|N|N|2432521|2432854|2432177|2432450|N|N|N|N|N| +2432543|AAAAAAAAPBOBFCAA|1947-12-23|575|2504|193|1947|2|12|23|4|1947|193|2504|Tuesday|1947Q4|N|N|N|2432521|2432854|2432178|2432451|N|N|N|N|N| +2432544|AAAAAAAAACOBFCAA|1947-12-24|575|2504|193|1947|3|12|24|4|1947|193|2504|Wednesday|1947Q4|N|N|N|2432521|2432854|2432179|2432452|N|N|N|N|N| +2432545|AAAAAAAABCOBFCAA|1947-12-25|575|2504|193|1947|4|12|25|4|1947|193|2504|Thursday|1947Q4|N|N|N|2432521|2432854|2432180|2432453|N|N|N|N|N| +2432546|AAAAAAAACCOBFCAA|1947-12-26|575|2504|193|1947|5|12|26|4|1947|193|2504|Friday|1947Q4|Y|Y|N|2432521|2432854|2432181|2432454|N|N|N|N|N| +2432547|AAAAAAAADCOBFCAA|1947-12-27|575|2504|193|1947|6|12|27|4|1947|193|2504|Saturday|1947Q4|N|Y|Y|2432521|2432854|2432182|2432455|N|N|N|N|N| +2432548|AAAAAAAAECOBFCAA|1947-12-28|575|2504|193|1947|0|12|28|4|1947|193|2504|Sunday|1947Q4|N|N|N|2432521|2432854|2432183|2432456|N|N|N|N|N| +2432549|AAAAAAAAFCOBFCAA|1947-12-29|575|2504|193|1947|1|12|29|4|1947|193|2504|Monday|1947Q4|N|N|N|2432521|2432854|2432184|2432457|N|N|N|N|N| +2432550|AAAAAAAAGCOBFCAA|1947-12-30|575|2505|193|1947|2|12|30|4|1947|193|2505|Tuesday|1947Q4|N|N|N|2432521|2432854|2432185|2432458|N|N|N|N|N| +2432551|AAAAAAAAHCOBFCAA|1947-12-31|575|2505|193|1947|3|12|31|4|1947|193|2505|Wednesday|1947Q4|N|N|N|2432521|2432854|2432186|2432459|N|N|N|N|N| +2432552|AAAAAAAAICOBFCAA|1948-01-01|576|2505|193|1948|4|1|1|1|1948|193|2505|Thursday|1948Q1|Y|N|N|2432552|2432551|2432187|2432460|N|N|N|N|N| +2432553|AAAAAAAAJCOBFCAA|1948-01-02|576|2505|193|1948|5|1|2|1|1948|193|2505|Friday|1948Q1|N|Y|Y|2432552|2432551|2432188|2432461|N|N|N|N|N| +2432554|AAAAAAAAKCOBFCAA|1948-01-03|576|2505|193|1948|6|1|3|1|1948|193|2505|Saturday|1948Q1|N|Y|N|2432552|2432551|2432189|2432462|N|N|N|N|N| +2432555|AAAAAAAALCOBFCAA|1948-01-04|576|2505|193|1948|0|1|4|1|1948|193|2505|Sunday|1948Q1|N|N|N|2432552|2432551|2432190|2432463|N|N|N|N|N| +2432556|AAAAAAAAMCOBFCAA|1948-01-05|576|2505|193|1948|1|1|5|1|1948|193|2505|Monday|1948Q1|N|N|N|2432552|2432551|2432191|2432464|N|N|N|N|N| +2432557|AAAAAAAANCOBFCAA|1948-01-06|576|2506|193|1948|2|1|6|1|1948|193|2506|Tuesday|1948Q1|N|N|N|2432552|2432551|2432192|2432465|N|N|N|N|N| +2432558|AAAAAAAAOCOBFCAA|1948-01-07|576|2506|193|1948|3|1|7|1|1948|193|2506|Wednesday|1948Q1|N|N|N|2432552|2432551|2432193|2432466|N|N|N|N|N| +2432559|AAAAAAAAPCOBFCAA|1948-01-08|576|2506|193|1948|4|1|8|1|1948|193|2506|Thursday|1948Q1|N|N|N|2432552|2432551|2432194|2432467|N|N|N|N|N| +2432560|AAAAAAAAADOBFCAA|1948-01-09|576|2506|193|1948|5|1|9|1|1948|193|2506|Friday|1948Q1|N|Y|N|2432552|2432551|2432195|2432468|N|N|N|N|N| +2432561|AAAAAAAABDOBFCAA|1948-01-10|576|2506|193|1948|6|1|10|1|1948|193|2506|Saturday|1948Q1|N|Y|N|2432552|2432551|2432196|2432469|N|N|N|N|N| +2432562|AAAAAAAACDOBFCAA|1948-01-11|576|2506|193|1948|0|1|11|1|1948|193|2506|Sunday|1948Q1|N|N|N|2432552|2432551|2432197|2432470|N|N|N|N|N| +2432563|AAAAAAAADDOBFCAA|1948-01-12|576|2506|193|1948|1|1|12|1|1948|193|2506|Monday|1948Q1|N|N|N|2432552|2432551|2432198|2432471|N|N|N|N|N| +2432564|AAAAAAAAEDOBFCAA|1948-01-13|576|2507|193|1948|2|1|13|1|1948|193|2507|Tuesday|1948Q1|N|N|N|2432552|2432551|2432199|2432472|N|N|N|N|N| +2432565|AAAAAAAAFDOBFCAA|1948-01-14|576|2507|193|1948|3|1|14|1|1948|193|2507|Wednesday|1948Q1|N|N|N|2432552|2432551|2432200|2432473|N|N|N|N|N| +2432566|AAAAAAAAGDOBFCAA|1948-01-15|576|2507|193|1948|4|1|15|1|1948|193|2507|Thursday|1948Q1|N|N|N|2432552|2432551|2432201|2432474|N|N|N|N|N| +2432567|AAAAAAAAHDOBFCAA|1948-01-16|576|2507|193|1948|5|1|16|1|1948|193|2507|Friday|1948Q1|N|Y|N|2432552|2432551|2432202|2432475|N|N|N|N|N| +2432568|AAAAAAAAIDOBFCAA|1948-01-17|576|2507|193|1948|6|1|17|1|1948|193|2507|Saturday|1948Q1|N|Y|N|2432552|2432551|2432203|2432476|N|N|N|N|N| +2432569|AAAAAAAAJDOBFCAA|1948-01-18|576|2507|193|1948|0|1|18|1|1948|193|2507|Sunday|1948Q1|N|N|N|2432552|2432551|2432204|2432477|N|N|N|N|N| +2432570|AAAAAAAAKDOBFCAA|1948-01-19|576|2507|193|1948|1|1|19|1|1948|193|2507|Monday|1948Q1|N|N|N|2432552|2432551|2432205|2432478|N|N|N|N|N| +2432571|AAAAAAAALDOBFCAA|1948-01-20|576|2508|193|1948|2|1|20|1|1948|193|2508|Tuesday|1948Q1|N|N|N|2432552|2432551|2432206|2432479|N|N|N|N|N| +2432572|AAAAAAAAMDOBFCAA|1948-01-21|576|2508|193|1948|3|1|21|1|1948|193|2508|Wednesday|1948Q1|N|N|N|2432552|2432551|2432207|2432480|N|N|N|N|N| +2432573|AAAAAAAANDOBFCAA|1948-01-22|576|2508|193|1948|4|1|22|1|1948|193|2508|Thursday|1948Q1|N|N|N|2432552|2432551|2432208|2432481|N|N|N|N|N| +2432574|AAAAAAAAODOBFCAA|1948-01-23|576|2508|193|1948|5|1|23|1|1948|193|2508|Friday|1948Q1|N|Y|N|2432552|2432551|2432209|2432482|N|N|N|N|N| +2432575|AAAAAAAAPDOBFCAA|1948-01-24|576|2508|193|1948|6|1|24|1|1948|193|2508|Saturday|1948Q1|N|Y|N|2432552|2432551|2432210|2432483|N|N|N|N|N| +2432576|AAAAAAAAAEOBFCAA|1948-01-25|576|2508|193|1948|0|1|25|1|1948|193|2508|Sunday|1948Q1|N|N|N|2432552|2432551|2432211|2432484|N|N|N|N|N| +2432577|AAAAAAAABEOBFCAA|1948-01-26|576|2508|193|1948|1|1|26|1|1948|193|2508|Monday|1948Q1|N|N|N|2432552|2432551|2432212|2432485|N|N|N|N|N| +2432578|AAAAAAAACEOBFCAA|1948-01-27|576|2509|193|1948|2|1|27|1|1948|193|2509|Tuesday|1948Q1|N|N|N|2432552|2432551|2432213|2432486|N|N|N|N|N| +2432579|AAAAAAAADEOBFCAA|1948-01-28|576|2509|193|1948|3|1|28|1|1948|193|2509|Wednesday|1948Q1|N|N|N|2432552|2432551|2432214|2432487|N|N|N|N|N| +2432580|AAAAAAAAEEOBFCAA|1948-01-29|576|2509|193|1948|4|1|29|1|1948|193|2509|Thursday|1948Q1|N|N|N|2432552|2432551|2432215|2432488|N|N|N|N|N| +2432581|AAAAAAAAFEOBFCAA|1948-01-30|576|2509|193|1948|5|1|30|1|1948|193|2509|Friday|1948Q1|N|Y|N|2432552|2432551|2432216|2432489|N|N|N|N|N| +2432582|AAAAAAAAGEOBFCAA|1948-01-31|576|2509|193|1948|6|1|31|1|1948|193|2509|Saturday|1948Q1|N|Y|N|2432552|2432551|2432217|2432490|N|N|N|N|N| +2432583|AAAAAAAAHEOBFCAA|1948-02-01|577|2509|193|1948|0|2|1|1|1948|193|2509|Sunday|1948Q1|N|N|N|2432583|2432613|2432218|2432491|N|N|N|N|N| +2432584|AAAAAAAAIEOBFCAA|1948-02-02|577|2509|193|1948|1|2|2|1|1948|193|2509|Monday|1948Q1|N|N|N|2432583|2432613|2432219|2432492|N|N|N|N|N| +2432585|AAAAAAAAJEOBFCAA|1948-02-03|577|2510|193|1948|2|2|3|1|1948|193|2510|Tuesday|1948Q1|N|N|N|2432583|2432613|2432220|2432493|N|N|N|N|N| +2432586|AAAAAAAAKEOBFCAA|1948-02-04|577|2510|193|1948|3|2|4|1|1948|193|2510|Wednesday|1948Q1|N|N|N|2432583|2432613|2432221|2432494|N|N|N|N|N| +2432587|AAAAAAAALEOBFCAA|1948-02-05|577|2510|193|1948|4|2|5|1|1948|193|2510|Thursday|1948Q1|N|N|N|2432583|2432613|2432222|2432495|N|N|N|N|N| +2432588|AAAAAAAAMEOBFCAA|1948-02-06|577|2510|193|1948|5|2|6|1|1948|193|2510|Friday|1948Q1|N|Y|N|2432583|2432613|2432223|2432496|N|N|N|N|N| +2432589|AAAAAAAANEOBFCAA|1948-02-07|577|2510|193|1948|6|2|7|1|1948|193|2510|Saturday|1948Q1|N|Y|N|2432583|2432613|2432224|2432497|N|N|N|N|N| +2432590|AAAAAAAAOEOBFCAA|1948-02-08|577|2510|193|1948|0|2|8|1|1948|193|2510|Sunday|1948Q1|N|N|N|2432583|2432613|2432225|2432498|N|N|N|N|N| +2432591|AAAAAAAAPEOBFCAA|1948-02-09|577|2510|193|1948|1|2|9|1|1948|193|2510|Monday|1948Q1|N|N|N|2432583|2432613|2432226|2432499|N|N|N|N|N| +2432592|AAAAAAAAAFOBFCAA|1948-02-10|577|2511|193|1948|2|2|10|1|1948|193|2511|Tuesday|1948Q1|N|N|N|2432583|2432613|2432227|2432500|N|N|N|N|N| +2432593|AAAAAAAABFOBFCAA|1948-02-11|577|2511|193|1948|3|2|11|1|1948|193|2511|Wednesday|1948Q1|N|N|N|2432583|2432613|2432228|2432501|N|N|N|N|N| +2432594|AAAAAAAACFOBFCAA|1948-02-12|577|2511|193|1948|4|2|12|1|1948|193|2511|Thursday|1948Q1|N|N|N|2432583|2432613|2432229|2432502|N|N|N|N|N| +2432595|AAAAAAAADFOBFCAA|1948-02-13|577|2511|193|1948|5|2|13|1|1948|193|2511|Friday|1948Q1|N|Y|N|2432583|2432613|2432230|2432503|N|N|N|N|N| +2432596|AAAAAAAAEFOBFCAA|1948-02-14|577|2511|193|1948|6|2|14|1|1948|193|2511|Saturday|1948Q1|N|Y|N|2432583|2432613|2432231|2432504|N|N|N|N|N| +2432597|AAAAAAAAFFOBFCAA|1948-02-15|577|2511|193|1948|0|2|15|1|1948|193|2511|Sunday|1948Q1|N|N|N|2432583|2432613|2432232|2432505|N|N|N|N|N| +2432598|AAAAAAAAGFOBFCAA|1948-02-16|577|2511|193|1948|1|2|16|1|1948|193|2511|Monday|1948Q1|N|N|N|2432583|2432613|2432233|2432506|N|N|N|N|N| +2432599|AAAAAAAAHFOBFCAA|1948-02-17|577|2512|193|1948|2|2|17|1|1948|193|2512|Tuesday|1948Q1|N|N|N|2432583|2432613|2432234|2432507|N|N|N|N|N| +2432600|AAAAAAAAIFOBFCAA|1948-02-18|577|2512|193|1948|3|2|18|1|1948|193|2512|Wednesday|1948Q1|N|N|N|2432583|2432613|2432235|2432508|N|N|N|N|N| +2432601|AAAAAAAAJFOBFCAA|1948-02-19|577|2512|193|1948|4|2|19|1|1948|193|2512|Thursday|1948Q1|N|N|N|2432583|2432613|2432236|2432509|N|N|N|N|N| +2432602|AAAAAAAAKFOBFCAA|1948-02-20|577|2512|193|1948|5|2|20|1|1948|193|2512|Friday|1948Q1|N|Y|N|2432583|2432613|2432237|2432510|N|N|N|N|N| +2432603|AAAAAAAALFOBFCAA|1948-02-21|577|2512|193|1948|6|2|21|1|1948|193|2512|Saturday|1948Q1|N|Y|N|2432583|2432613|2432238|2432511|N|N|N|N|N| +2432604|AAAAAAAAMFOBFCAA|1948-02-22|577|2512|193|1948|0|2|22|1|1948|193|2512|Sunday|1948Q1|N|N|N|2432583|2432613|2432239|2432512|N|N|N|N|N| +2432605|AAAAAAAANFOBFCAA|1948-02-23|577|2512|193|1948|1|2|23|1|1948|193|2512|Monday|1948Q1|N|N|N|2432583|2432613|2432240|2432513|N|N|N|N|N| +2432606|AAAAAAAAOFOBFCAA|1948-02-24|577|2513|193|1948|2|2|24|1|1948|193|2513|Tuesday|1948Q1|N|N|N|2432583|2432613|2432241|2432514|N|N|N|N|N| +2432607|AAAAAAAAPFOBFCAA|1948-02-25|577|2513|193|1948|3|2|25|1|1948|193|2513|Wednesday|1948Q1|N|N|N|2432583|2432613|2432242|2432515|N|N|N|N|N| +2432608|AAAAAAAAAGOBFCAA|1948-02-26|577|2513|193|1948|4|2|26|1|1948|193|2513|Thursday|1948Q1|N|N|N|2432583|2432613|2432243|2432516|N|N|N|N|N| +2432609|AAAAAAAABGOBFCAA|1948-02-27|577|2513|193|1948|5|2|27|1|1948|193|2513|Friday|1948Q1|N|Y|N|2432583|2432613|2432244|2432517|N|N|N|N|N| +2432610|AAAAAAAACGOBFCAA|1948-02-28|577|2513|193|1948|6|2|28|1|1948|193|2513|Saturday|1948Q1|N|Y|N|2432583|2432613|2432245|2432518|N|N|N|N|N| +2432611|AAAAAAAADGOBFCAA|1948-02-29|577|2513|193|1948|0|2|29|1|1948|193|2513|Sunday|1948Q1|N|N|N|2432583|2432613|2432245|2432519|N|N|N|N|N| +2432612|AAAAAAAAEGOBFCAA|1948-03-01|578|2513|194|1948|1|3|1|1|1948|194|2513|Monday|1948Q1|N|N|N|2432612|2432671|2432246|2432520|N|N|N|N|N| +2432613|AAAAAAAAFGOBFCAA|1948-03-02|578|2514|194|1948|2|3|2|1|1948|194|2514|Tuesday|1948Q1|N|N|N|2432612|2432671|2432247|2432521|N|N|N|N|N| +2432614|AAAAAAAAGGOBFCAA|1948-03-03|578|2514|194|1948|3|3|3|1|1948|194|2514|Wednesday|1948Q1|N|N|N|2432612|2432671|2432248|2432522|N|N|N|N|N| +2432615|AAAAAAAAHGOBFCAA|1948-03-04|578|2514|194|1948|4|3|4|1|1948|194|2514|Thursday|1948Q1|N|N|N|2432612|2432671|2432249|2432523|N|N|N|N|N| +2432616|AAAAAAAAIGOBFCAA|1948-03-05|578|2514|194|1948|5|3|5|1|1948|194|2514|Friday|1948Q1|N|Y|N|2432612|2432671|2432250|2432524|N|N|N|N|N| +2432617|AAAAAAAAJGOBFCAA|1948-03-06|578|2514|194|1948|6|3|6|1|1948|194|2514|Saturday|1948Q1|N|Y|N|2432612|2432671|2432251|2432525|N|N|N|N|N| +2432618|AAAAAAAAKGOBFCAA|1948-03-07|578|2514|194|1948|0|3|7|1|1948|194|2514|Sunday|1948Q1|N|N|N|2432612|2432671|2432252|2432526|N|N|N|N|N| +2432619|AAAAAAAALGOBFCAA|1948-03-08|578|2514|194|1948|1|3|8|1|1948|194|2514|Monday|1948Q1|N|N|N|2432612|2432671|2432253|2432527|N|N|N|N|N| +2432620|AAAAAAAAMGOBFCAA|1948-03-09|578|2515|194|1948|2|3|9|1|1948|194|2515|Tuesday|1948Q1|N|N|N|2432612|2432671|2432254|2432528|N|N|N|N|N| +2432621|AAAAAAAANGOBFCAA|1948-03-10|578|2515|194|1948|3|3|10|1|1948|194|2515|Wednesday|1948Q1|N|N|N|2432612|2432671|2432255|2432529|N|N|N|N|N| +2432622|AAAAAAAAOGOBFCAA|1948-03-11|578|2515|194|1948|4|3|11|1|1948|194|2515|Thursday|1948Q1|N|N|N|2432612|2432671|2432256|2432530|N|N|N|N|N| +2432623|AAAAAAAAPGOBFCAA|1948-03-12|578|2515|194|1948|5|3|12|1|1948|194|2515|Friday|1948Q1|N|Y|N|2432612|2432671|2432257|2432531|N|N|N|N|N| +2432624|AAAAAAAAAHOBFCAA|1948-03-13|578|2515|194|1948|6|3|13|1|1948|194|2515|Saturday|1948Q1|N|Y|N|2432612|2432671|2432258|2432532|N|N|N|N|N| +2432625|AAAAAAAABHOBFCAA|1948-03-14|578|2515|194|1948|0|3|14|1|1948|194|2515|Sunday|1948Q1|N|N|N|2432612|2432671|2432259|2432533|N|N|N|N|N| +2432626|AAAAAAAACHOBFCAA|1948-03-15|578|2515|194|1948|1|3|15|1|1948|194|2515|Monday|1948Q1|N|N|N|2432612|2432671|2432260|2432534|N|N|N|N|N| +2432627|AAAAAAAADHOBFCAA|1948-03-16|578|2516|194|1948|2|3|16|1|1948|194|2516|Tuesday|1948Q1|N|N|N|2432612|2432671|2432261|2432535|N|N|N|N|N| +2432628|AAAAAAAAEHOBFCAA|1948-03-17|578|2516|194|1948|3|3|17|1|1948|194|2516|Wednesday|1948Q1|N|N|N|2432612|2432671|2432262|2432536|N|N|N|N|N| +2432629|AAAAAAAAFHOBFCAA|1948-03-18|578|2516|194|1948|4|3|18|1|1948|194|2516|Thursday|1948Q1|N|N|N|2432612|2432671|2432263|2432537|N|N|N|N|N| +2432630|AAAAAAAAGHOBFCAA|1948-03-19|578|2516|194|1948|5|3|19|1|1948|194|2516|Friday|1948Q1|N|Y|N|2432612|2432671|2432264|2432538|N|N|N|N|N| +2432631|AAAAAAAAHHOBFCAA|1948-03-20|578|2516|194|1948|6|3|20|1|1948|194|2516|Saturday|1948Q1|N|Y|N|2432612|2432671|2432265|2432539|N|N|N|N|N| +2432632|AAAAAAAAIHOBFCAA|1948-03-21|578|2516|194|1948|0|3|21|1|1948|194|2516|Sunday|1948Q1|N|N|N|2432612|2432671|2432266|2432540|N|N|N|N|N| +2432633|AAAAAAAAJHOBFCAA|1948-03-22|578|2516|194|1948|1|3|22|1|1948|194|2516|Monday|1948Q1|N|N|N|2432612|2432671|2432267|2432541|N|N|N|N|N| +2432634|AAAAAAAAKHOBFCAA|1948-03-23|578|2517|194|1948|2|3|23|1|1948|194|2517|Tuesday|1948Q1|N|N|N|2432612|2432671|2432268|2432542|N|N|N|N|N| +2432635|AAAAAAAALHOBFCAA|1948-03-24|578|2517|194|1948|3|3|24|1|1948|194|2517|Wednesday|1948Q1|N|N|N|2432612|2432671|2432269|2432543|N|N|N|N|N| +2432636|AAAAAAAAMHOBFCAA|1948-03-25|578|2517|194|1948|4|3|25|1|1948|194|2517|Thursday|1948Q1|N|N|N|2432612|2432671|2432270|2432544|N|N|N|N|N| +2432637|AAAAAAAANHOBFCAA|1948-03-26|578|2517|194|1948|5|3|26|1|1948|194|2517|Friday|1948Q1|N|Y|N|2432612|2432671|2432271|2432545|N|N|N|N|N| +2432638|AAAAAAAAOHOBFCAA|1948-03-27|578|2517|194|1948|6|3|27|1|1948|194|2517|Saturday|1948Q1|N|Y|N|2432612|2432671|2432272|2432546|N|N|N|N|N| +2432639|AAAAAAAAPHOBFCAA|1948-03-28|578|2517|194|1948|0|3|28|1|1948|194|2517|Sunday|1948Q1|N|N|N|2432612|2432671|2432273|2432547|N|N|N|N|N| +2432640|AAAAAAAAAIOBFCAA|1948-03-29|578|2517|194|1948|1|3|29|1|1948|194|2517|Monday|1948Q1|N|N|N|2432612|2432671|2432274|2432548|N|N|N|N|N| +2432641|AAAAAAAABIOBFCAA|1948-03-30|578|2518|194|1948|2|3|30|1|1948|194|2518|Tuesday|1948Q1|N|N|N|2432612|2432671|2432275|2432549|N|N|N|N|N| +2432642|AAAAAAAACIOBFCAA|1948-03-31|578|2518|194|1948|3|3|31|1|1948|194|2518|Wednesday|1948Q1|N|N|N|2432612|2432671|2432276|2432550|N|N|N|N|N| +2432643|AAAAAAAADIOBFCAA|1948-04-01|579|2518|194|1948|4|4|1|2|1948|194|2518|Thursday|1948Q2|N|N|N|2432643|2432733|2432277|2432552|N|N|N|N|N| +2432644|AAAAAAAAEIOBFCAA|1948-04-02|579|2518|194|1948|5|4|2|2|1948|194|2518|Friday|1948Q2|N|Y|N|2432643|2432733|2432278|2432553|N|N|N|N|N| +2432645|AAAAAAAAFIOBFCAA|1948-04-03|579|2518|194|1948|6|4|3|2|1948|194|2518|Saturday|1948Q2|N|Y|N|2432643|2432733|2432279|2432554|N|N|N|N|N| +2432646|AAAAAAAAGIOBFCAA|1948-04-04|579|2518|194|1948|0|4|4|2|1948|194|2518|Sunday|1948Q2|N|N|N|2432643|2432733|2432280|2432555|N|N|N|N|N| +2432647|AAAAAAAAHIOBFCAA|1948-04-05|579|2518|194|1948|1|4|5|2|1948|194|2518|Monday|1948Q2|N|N|N|2432643|2432733|2432281|2432556|N|N|N|N|N| +2432648|AAAAAAAAIIOBFCAA|1948-04-06|579|2519|194|1948|2|4|6|2|1948|194|2519|Tuesday|1948Q2|N|N|N|2432643|2432733|2432282|2432557|N|N|N|N|N| +2432649|AAAAAAAAJIOBFCAA|1948-04-07|579|2519|194|1948|3|4|7|2|1948|194|2519|Wednesday|1948Q2|N|N|N|2432643|2432733|2432283|2432558|N|N|N|N|N| +2432650|AAAAAAAAKIOBFCAA|1948-04-08|579|2519|194|1948|4|4|8|2|1948|194|2519|Thursday|1948Q2|N|N|N|2432643|2432733|2432284|2432559|N|N|N|N|N| +2432651|AAAAAAAALIOBFCAA|1948-04-09|579|2519|194|1948|5|4|9|2|1948|194|2519|Friday|1948Q2|N|Y|N|2432643|2432733|2432285|2432560|N|N|N|N|N| +2432652|AAAAAAAAMIOBFCAA|1948-04-10|579|2519|194|1948|6|4|10|2|1948|194|2519|Saturday|1948Q2|N|Y|N|2432643|2432733|2432286|2432561|N|N|N|N|N| +2432653|AAAAAAAANIOBFCAA|1948-04-11|579|2519|194|1948|0|4|11|2|1948|194|2519|Sunday|1948Q2|N|N|N|2432643|2432733|2432287|2432562|N|N|N|N|N| +2432654|AAAAAAAAOIOBFCAA|1948-04-12|579|2519|194|1948|1|4|12|2|1948|194|2519|Monday|1948Q2|N|N|N|2432643|2432733|2432288|2432563|N|N|N|N|N| +2432655|AAAAAAAAPIOBFCAA|1948-04-13|579|2520|194|1948|2|4|13|2|1948|194|2520|Tuesday|1948Q2|N|N|N|2432643|2432733|2432289|2432564|N|N|N|N|N| +2432656|AAAAAAAAAJOBFCAA|1948-04-14|579|2520|194|1948|3|4|14|2|1948|194|2520|Wednesday|1948Q2|N|N|N|2432643|2432733|2432290|2432565|N|N|N|N|N| +2432657|AAAAAAAABJOBFCAA|1948-04-15|579|2520|194|1948|4|4|15|2|1948|194|2520|Thursday|1948Q2|N|N|N|2432643|2432733|2432291|2432566|N|N|N|N|N| +2432658|AAAAAAAACJOBFCAA|1948-04-16|579|2520|194|1948|5|4|16|2|1948|194|2520|Friday|1948Q2|N|Y|N|2432643|2432733|2432292|2432567|N|N|N|N|N| +2432659|AAAAAAAADJOBFCAA|1948-04-17|579|2520|194|1948|6|4|17|2|1948|194|2520|Saturday|1948Q2|N|Y|N|2432643|2432733|2432293|2432568|N|N|N|N|N| +2432660|AAAAAAAAEJOBFCAA|1948-04-18|579|2520|194|1948|0|4|18|2|1948|194|2520|Sunday|1948Q2|N|N|N|2432643|2432733|2432294|2432569|N|N|N|N|N| +2432661|AAAAAAAAFJOBFCAA|1948-04-19|579|2520|194|1948|1|4|19|2|1948|194|2520|Monday|1948Q2|N|N|N|2432643|2432733|2432295|2432570|N|N|N|N|N| +2432662|AAAAAAAAGJOBFCAA|1948-04-20|579|2521|194|1948|2|4|20|2|1948|194|2521|Tuesday|1948Q2|N|N|N|2432643|2432733|2432296|2432571|N|N|N|N|N| +2432663|AAAAAAAAHJOBFCAA|1948-04-21|579|2521|194|1948|3|4|21|2|1948|194|2521|Wednesday|1948Q2|N|N|N|2432643|2432733|2432297|2432572|N|N|N|N|N| +2432664|AAAAAAAAIJOBFCAA|1948-04-22|579|2521|194|1948|4|4|22|2|1948|194|2521|Thursday|1948Q2|N|N|N|2432643|2432733|2432298|2432573|N|N|N|N|N| +2432665|AAAAAAAAJJOBFCAA|1948-04-23|579|2521|194|1948|5|4|23|2|1948|194|2521|Friday|1948Q2|N|Y|N|2432643|2432733|2432299|2432574|N|N|N|N|N| +2432666|AAAAAAAAKJOBFCAA|1948-04-24|579|2521|194|1948|6|4|24|2|1948|194|2521|Saturday|1948Q2|N|Y|N|2432643|2432733|2432300|2432575|N|N|N|N|N| +2432667|AAAAAAAALJOBFCAA|1948-04-25|579|2521|194|1948|0|4|25|2|1948|194|2521|Sunday|1948Q2|N|N|N|2432643|2432733|2432301|2432576|N|N|N|N|N| +2432668|AAAAAAAAMJOBFCAA|1948-04-26|579|2521|194|1948|1|4|26|2|1948|194|2521|Monday|1948Q2|N|N|N|2432643|2432733|2432302|2432577|N|N|N|N|N| +2432669|AAAAAAAANJOBFCAA|1948-04-27|579|2522|194|1948|2|4|27|2|1948|194|2522|Tuesday|1948Q2|N|N|N|2432643|2432733|2432303|2432578|N|N|N|N|N| +2432670|AAAAAAAAOJOBFCAA|1948-04-28|579|2522|194|1948|3|4|28|2|1948|194|2522|Wednesday|1948Q2|N|N|N|2432643|2432733|2432304|2432579|N|N|N|N|N| +2432671|AAAAAAAAPJOBFCAA|1948-04-29|579|2522|194|1948|4|4|29|2|1948|194|2522|Thursday|1948Q2|N|N|N|2432643|2432733|2432305|2432580|N|N|N|N|N| +2432672|AAAAAAAAAKOBFCAA|1948-04-30|579|2522|194|1948|5|4|30|2|1948|194|2522|Friday|1948Q2|N|Y|N|2432643|2432733|2432306|2432581|N|N|N|N|N| +2432673|AAAAAAAABKOBFCAA|1948-05-01|580|2522|194|1948|6|5|1|2|1948|194|2522|Saturday|1948Q2|N|Y|N|2432673|2432793|2432307|2432582|N|N|N|N|N| +2432674|AAAAAAAACKOBFCAA|1948-05-02|580|2522|194|1948|0|5|2|2|1948|194|2522|Sunday|1948Q2|N|N|N|2432673|2432793|2432308|2432583|N|N|N|N|N| +2432675|AAAAAAAADKOBFCAA|1948-05-03|580|2522|194|1948|1|5|3|2|1948|194|2522|Monday|1948Q2|N|N|N|2432673|2432793|2432309|2432584|N|N|N|N|N| +2432676|AAAAAAAAEKOBFCAA|1948-05-04|580|2523|194|1948|2|5|4|2|1948|194|2523|Tuesday|1948Q2|N|N|N|2432673|2432793|2432310|2432585|N|N|N|N|N| +2432677|AAAAAAAAFKOBFCAA|1948-05-05|580|2523|194|1948|3|5|5|2|1948|194|2523|Wednesday|1948Q2|N|N|N|2432673|2432793|2432311|2432586|N|N|N|N|N| +2432678|AAAAAAAAGKOBFCAA|1948-05-06|580|2523|194|1948|4|5|6|2|1948|194|2523|Thursday|1948Q2|N|N|N|2432673|2432793|2432312|2432587|N|N|N|N|N| +2432679|AAAAAAAAHKOBFCAA|1948-05-07|580|2523|194|1948|5|5|7|2|1948|194|2523|Friday|1948Q2|N|Y|N|2432673|2432793|2432313|2432588|N|N|N|N|N| +2432680|AAAAAAAAIKOBFCAA|1948-05-08|580|2523|194|1948|6|5|8|2|1948|194|2523|Saturday|1948Q2|N|Y|N|2432673|2432793|2432314|2432589|N|N|N|N|N| +2432681|AAAAAAAAJKOBFCAA|1948-05-09|580|2523|194|1948|0|5|9|2|1948|194|2523|Sunday|1948Q2|N|N|N|2432673|2432793|2432315|2432590|N|N|N|N|N| +2432682|AAAAAAAAKKOBFCAA|1948-05-10|580|2523|194|1948|1|5|10|2|1948|194|2523|Monday|1948Q2|N|N|N|2432673|2432793|2432316|2432591|N|N|N|N|N| +2432683|AAAAAAAALKOBFCAA|1948-05-11|580|2524|194|1948|2|5|11|2|1948|194|2524|Tuesday|1948Q2|N|N|N|2432673|2432793|2432317|2432592|N|N|N|N|N| +2432684|AAAAAAAAMKOBFCAA|1948-05-12|580|2524|194|1948|3|5|12|2|1948|194|2524|Wednesday|1948Q2|N|N|N|2432673|2432793|2432318|2432593|N|N|N|N|N| +2432685|AAAAAAAANKOBFCAA|1948-05-13|580|2524|194|1948|4|5|13|2|1948|194|2524|Thursday|1948Q2|N|N|N|2432673|2432793|2432319|2432594|N|N|N|N|N| +2432686|AAAAAAAAOKOBFCAA|1948-05-14|580|2524|194|1948|5|5|14|2|1948|194|2524|Friday|1948Q2|N|Y|N|2432673|2432793|2432320|2432595|N|N|N|N|N| +2432687|AAAAAAAAPKOBFCAA|1948-05-15|580|2524|194|1948|6|5|15|2|1948|194|2524|Saturday|1948Q2|N|Y|N|2432673|2432793|2432321|2432596|N|N|N|N|N| +2432688|AAAAAAAAALOBFCAA|1948-05-16|580|2524|194|1948|0|5|16|2|1948|194|2524|Sunday|1948Q2|N|N|N|2432673|2432793|2432322|2432597|N|N|N|N|N| +2432689|AAAAAAAABLOBFCAA|1948-05-17|580|2524|194|1948|1|5|17|2|1948|194|2524|Monday|1948Q2|N|N|N|2432673|2432793|2432323|2432598|N|N|N|N|N| +2432690|AAAAAAAACLOBFCAA|1948-05-18|580|2525|194|1948|2|5|18|2|1948|194|2525|Tuesday|1948Q2|N|N|N|2432673|2432793|2432324|2432599|N|N|N|N|N| +2432691|AAAAAAAADLOBFCAA|1948-05-19|580|2525|194|1948|3|5|19|2|1948|194|2525|Wednesday|1948Q2|N|N|N|2432673|2432793|2432325|2432600|N|N|N|N|N| +2432692|AAAAAAAAELOBFCAA|1948-05-20|580|2525|194|1948|4|5|20|2|1948|194|2525|Thursday|1948Q2|N|N|N|2432673|2432793|2432326|2432601|N|N|N|N|N| +2432693|AAAAAAAAFLOBFCAA|1948-05-21|580|2525|194|1948|5|5|21|2|1948|194|2525|Friday|1948Q2|N|Y|N|2432673|2432793|2432327|2432602|N|N|N|N|N| +2432694|AAAAAAAAGLOBFCAA|1948-05-22|580|2525|194|1948|6|5|22|2|1948|194|2525|Saturday|1948Q2|N|Y|N|2432673|2432793|2432328|2432603|N|N|N|N|N| +2432695|AAAAAAAAHLOBFCAA|1948-05-23|580|2525|194|1948|0|5|23|2|1948|194|2525|Sunday|1948Q2|N|N|N|2432673|2432793|2432329|2432604|N|N|N|N|N| +2432696|AAAAAAAAILOBFCAA|1948-05-24|580|2525|194|1948|1|5|24|2|1948|194|2525|Monday|1948Q2|N|N|N|2432673|2432793|2432330|2432605|N|N|N|N|N| +2432697|AAAAAAAAJLOBFCAA|1948-05-25|580|2526|194|1948|2|5|25|2|1948|194|2526|Tuesday|1948Q2|N|N|N|2432673|2432793|2432331|2432606|N|N|N|N|N| +2432698|AAAAAAAAKLOBFCAA|1948-05-26|580|2526|194|1948|3|5|26|2|1948|194|2526|Wednesday|1948Q2|N|N|N|2432673|2432793|2432332|2432607|N|N|N|N|N| +2432699|AAAAAAAALLOBFCAA|1948-05-27|580|2526|194|1948|4|5|27|2|1948|194|2526|Thursday|1948Q2|N|N|N|2432673|2432793|2432333|2432608|N|N|N|N|N| +2432700|AAAAAAAAMLOBFCAA|1948-05-28|580|2526|194|1948|5|5|28|2|1948|194|2526|Friday|1948Q2|N|Y|N|2432673|2432793|2432334|2432609|N|N|N|N|N| +2432701|AAAAAAAANLOBFCAA|1948-05-29|580|2526|194|1948|6|5|29|2|1948|194|2526|Saturday|1948Q2|N|Y|N|2432673|2432793|2432335|2432610|N|N|N|N|N| +2432702|AAAAAAAAOLOBFCAA|1948-05-30|580|2526|194|1948|0|5|30|2|1948|194|2526|Sunday|1948Q2|N|N|N|2432673|2432793|2432336|2432611|N|N|N|N|N| +2432703|AAAAAAAAPLOBFCAA|1948-05-31|580|2526|194|1948|1|5|31|2|1948|194|2526|Monday|1948Q2|N|N|N|2432673|2432793|2432337|2432612|N|N|N|N|N| +2432704|AAAAAAAAAMOBFCAA|1948-06-01|581|2527|195|1948|2|6|1|2|1948|195|2527|Tuesday|1948Q2|N|N|N|2432704|2432855|2432338|2432613|N|N|N|N|N| +2432705|AAAAAAAABMOBFCAA|1948-06-02|581|2527|195|1948|3|6|2|2|1948|195|2527|Wednesday|1948Q2|N|N|N|2432704|2432855|2432339|2432614|N|N|N|N|N| +2432706|AAAAAAAACMOBFCAA|1948-06-03|581|2527|195|1948|4|6|3|2|1948|195|2527|Thursday|1948Q2|N|N|N|2432704|2432855|2432340|2432615|N|N|N|N|N| +2432707|AAAAAAAADMOBFCAA|1948-06-04|581|2527|195|1948|5|6|4|2|1948|195|2527|Friday|1948Q2|N|Y|N|2432704|2432855|2432341|2432616|N|N|N|N|N| +2432708|AAAAAAAAEMOBFCAA|1948-06-05|581|2527|195|1948|6|6|5|2|1948|195|2527|Saturday|1948Q2|N|Y|N|2432704|2432855|2432342|2432617|N|N|N|N|N| +2432709|AAAAAAAAFMOBFCAA|1948-06-06|581|2527|195|1948|0|6|6|2|1948|195|2527|Sunday|1948Q2|N|N|N|2432704|2432855|2432343|2432618|N|N|N|N|N| +2432710|AAAAAAAAGMOBFCAA|1948-06-07|581|2527|195|1948|1|6|7|2|1948|195|2527|Monday|1948Q2|N|N|N|2432704|2432855|2432344|2432619|N|N|N|N|N| +2432711|AAAAAAAAHMOBFCAA|1948-06-08|581|2528|195|1948|2|6|8|2|1948|195|2528|Tuesday|1948Q2|N|N|N|2432704|2432855|2432345|2432620|N|N|N|N|N| +2432712|AAAAAAAAIMOBFCAA|1948-06-09|581|2528|195|1948|3|6|9|2|1948|195|2528|Wednesday|1948Q2|N|N|N|2432704|2432855|2432346|2432621|N|N|N|N|N| +2432713|AAAAAAAAJMOBFCAA|1948-06-10|581|2528|195|1948|4|6|10|2|1948|195|2528|Thursday|1948Q2|N|N|N|2432704|2432855|2432347|2432622|N|N|N|N|N| +2432714|AAAAAAAAKMOBFCAA|1948-06-11|581|2528|195|1948|5|6|11|2|1948|195|2528|Friday|1948Q2|N|Y|N|2432704|2432855|2432348|2432623|N|N|N|N|N| +2432715|AAAAAAAALMOBFCAA|1948-06-12|581|2528|195|1948|6|6|12|2|1948|195|2528|Saturday|1948Q2|N|Y|N|2432704|2432855|2432349|2432624|N|N|N|N|N| +2432716|AAAAAAAAMMOBFCAA|1948-06-13|581|2528|195|1948|0|6|13|2|1948|195|2528|Sunday|1948Q2|N|N|N|2432704|2432855|2432350|2432625|N|N|N|N|N| +2432717|AAAAAAAANMOBFCAA|1948-06-14|581|2528|195|1948|1|6|14|2|1948|195|2528|Monday|1948Q2|N|N|N|2432704|2432855|2432351|2432626|N|N|N|N|N| +2432718|AAAAAAAAOMOBFCAA|1948-06-15|581|2529|195|1948|2|6|15|2|1948|195|2529|Tuesday|1948Q2|N|N|N|2432704|2432855|2432352|2432627|N|N|N|N|N| +2432719|AAAAAAAAPMOBFCAA|1948-06-16|581|2529|195|1948|3|6|16|2|1948|195|2529|Wednesday|1948Q2|N|N|N|2432704|2432855|2432353|2432628|N|N|N|N|N| +2432720|AAAAAAAAANOBFCAA|1948-06-17|581|2529|195|1948|4|6|17|2|1948|195|2529|Thursday|1948Q2|N|N|N|2432704|2432855|2432354|2432629|N|N|N|N|N| +2432721|AAAAAAAABNOBFCAA|1948-06-18|581|2529|195|1948|5|6|18|2|1948|195|2529|Friday|1948Q2|N|Y|N|2432704|2432855|2432355|2432630|N|N|N|N|N| +2432722|AAAAAAAACNOBFCAA|1948-06-19|581|2529|195|1948|6|6|19|2|1948|195|2529|Saturday|1948Q2|N|Y|N|2432704|2432855|2432356|2432631|N|N|N|N|N| +2432723|AAAAAAAADNOBFCAA|1948-06-20|581|2529|195|1948|0|6|20|2|1948|195|2529|Sunday|1948Q2|N|N|N|2432704|2432855|2432357|2432632|N|N|N|N|N| +2432724|AAAAAAAAENOBFCAA|1948-06-21|581|2529|195|1948|1|6|21|2|1948|195|2529|Monday|1948Q2|N|N|N|2432704|2432855|2432358|2432633|N|N|N|N|N| +2432725|AAAAAAAAFNOBFCAA|1948-06-22|581|2530|195|1948|2|6|22|2|1948|195|2530|Tuesday|1948Q2|N|N|N|2432704|2432855|2432359|2432634|N|N|N|N|N| +2432726|AAAAAAAAGNOBFCAA|1948-06-23|581|2530|195|1948|3|6|23|2|1948|195|2530|Wednesday|1948Q2|N|N|N|2432704|2432855|2432360|2432635|N|N|N|N|N| +2432727|AAAAAAAAHNOBFCAA|1948-06-24|581|2530|195|1948|4|6|24|2|1948|195|2530|Thursday|1948Q2|N|N|N|2432704|2432855|2432361|2432636|N|N|N|N|N| +2432728|AAAAAAAAINOBFCAA|1948-06-25|581|2530|195|1948|5|6|25|2|1948|195|2530|Friday|1948Q2|N|Y|N|2432704|2432855|2432362|2432637|N|N|N|N|N| +2432729|AAAAAAAAJNOBFCAA|1948-06-26|581|2530|195|1948|6|6|26|2|1948|195|2530|Saturday|1948Q2|N|Y|N|2432704|2432855|2432363|2432638|N|N|N|N|N| +2432730|AAAAAAAAKNOBFCAA|1948-06-27|581|2530|195|1948|0|6|27|2|1948|195|2530|Sunday|1948Q2|N|N|N|2432704|2432855|2432364|2432639|N|N|N|N|N| +2432731|AAAAAAAALNOBFCAA|1948-06-28|581|2530|195|1948|1|6|28|2|1948|195|2530|Monday|1948Q2|N|N|N|2432704|2432855|2432365|2432640|N|N|N|N|N| +2432732|AAAAAAAAMNOBFCAA|1948-06-29|581|2531|195|1948|2|6|29|2|1948|195|2531|Tuesday|1948Q2|N|N|N|2432704|2432855|2432366|2432641|N|N|N|N|N| +2432733|AAAAAAAANNOBFCAA|1948-06-30|581|2531|195|1948|3|6|30|2|1948|195|2531|Wednesday|1948Q2|N|N|N|2432704|2432855|2432367|2432642|N|N|N|N|N| +2432734|AAAAAAAAONOBFCAA|1948-07-01|582|2531|195|1948|4|7|1|3|1948|195|2531|Thursday|1948Q3|N|N|N|2432734|2432915|2432368|2432643|N|N|N|N|N| +2432735|AAAAAAAAPNOBFCAA|1948-07-02|582|2531|195|1948|5|7|2|3|1948|195|2531|Friday|1948Q3|N|Y|N|2432734|2432915|2432369|2432644|N|N|N|N|N| +2432736|AAAAAAAAAOOBFCAA|1948-07-03|582|2531|195|1948|6|7|3|3|1948|195|2531|Saturday|1948Q3|N|Y|N|2432734|2432915|2432370|2432645|N|N|N|N|N| +2432737|AAAAAAAABOOBFCAA|1948-07-04|582|2531|195|1948|0|7|4|3|1948|195|2531|Sunday|1948Q3|Y|N|N|2432734|2432915|2432371|2432646|N|N|N|N|N| +2432738|AAAAAAAACOOBFCAA|1948-07-05|582|2531|195|1948|1|7|5|3|1948|195|2531|Monday|1948Q3|N|N|Y|2432734|2432915|2432372|2432647|N|N|N|N|N| +2432739|AAAAAAAADOOBFCAA|1948-07-06|582|2532|195|1948|2|7|6|3|1948|195|2532|Tuesday|1948Q3|N|N|N|2432734|2432915|2432373|2432648|N|N|N|N|N| +2432740|AAAAAAAAEOOBFCAA|1948-07-07|582|2532|195|1948|3|7|7|3|1948|195|2532|Wednesday|1948Q3|N|N|N|2432734|2432915|2432374|2432649|N|N|N|N|N| +2432741|AAAAAAAAFOOBFCAA|1948-07-08|582|2532|195|1948|4|7|8|3|1948|195|2532|Thursday|1948Q3|N|N|N|2432734|2432915|2432375|2432650|N|N|N|N|N| +2432742|AAAAAAAAGOOBFCAA|1948-07-09|582|2532|195|1948|5|7|9|3|1948|195|2532|Friday|1948Q3|N|Y|N|2432734|2432915|2432376|2432651|N|N|N|N|N| +2432743|AAAAAAAAHOOBFCAA|1948-07-10|582|2532|195|1948|6|7|10|3|1948|195|2532|Saturday|1948Q3|N|Y|N|2432734|2432915|2432377|2432652|N|N|N|N|N| +2432744|AAAAAAAAIOOBFCAA|1948-07-11|582|2532|195|1948|0|7|11|3|1948|195|2532|Sunday|1948Q3|N|N|N|2432734|2432915|2432378|2432653|N|N|N|N|N| +2432745|AAAAAAAAJOOBFCAA|1948-07-12|582|2532|195|1948|1|7|12|3|1948|195|2532|Monday|1948Q3|N|N|N|2432734|2432915|2432379|2432654|N|N|N|N|N| +2432746|AAAAAAAAKOOBFCAA|1948-07-13|582|2533|195|1948|2|7|13|3|1948|195|2533|Tuesday|1948Q3|N|N|N|2432734|2432915|2432380|2432655|N|N|N|N|N| +2432747|AAAAAAAALOOBFCAA|1948-07-14|582|2533|195|1948|3|7|14|3|1948|195|2533|Wednesday|1948Q3|N|N|N|2432734|2432915|2432381|2432656|N|N|N|N|N| +2432748|AAAAAAAAMOOBFCAA|1948-07-15|582|2533|195|1948|4|7|15|3|1948|195|2533|Thursday|1948Q3|N|N|N|2432734|2432915|2432382|2432657|N|N|N|N|N| +2432749|AAAAAAAANOOBFCAA|1948-07-16|582|2533|195|1948|5|7|16|3|1948|195|2533|Friday|1948Q3|N|Y|N|2432734|2432915|2432383|2432658|N|N|N|N|N| +2432750|AAAAAAAAOOOBFCAA|1948-07-17|582|2533|195|1948|6|7|17|3|1948|195|2533|Saturday|1948Q3|N|Y|N|2432734|2432915|2432384|2432659|N|N|N|N|N| +2432751|AAAAAAAAPOOBFCAA|1948-07-18|582|2533|195|1948|0|7|18|3|1948|195|2533|Sunday|1948Q3|N|N|N|2432734|2432915|2432385|2432660|N|N|N|N|N| +2432752|AAAAAAAAAPOBFCAA|1948-07-19|582|2533|195|1948|1|7|19|3|1948|195|2533|Monday|1948Q3|N|N|N|2432734|2432915|2432386|2432661|N|N|N|N|N| +2432753|AAAAAAAABPOBFCAA|1948-07-20|582|2534|195|1948|2|7|20|3|1948|195|2534|Tuesday|1948Q3|N|N|N|2432734|2432915|2432387|2432662|N|N|N|N|N| +2432754|AAAAAAAACPOBFCAA|1948-07-21|582|2534|195|1948|3|7|21|3|1948|195|2534|Wednesday|1948Q3|N|N|N|2432734|2432915|2432388|2432663|N|N|N|N|N| +2432755|AAAAAAAADPOBFCAA|1948-07-22|582|2534|195|1948|4|7|22|3|1948|195|2534|Thursday|1948Q3|N|N|N|2432734|2432915|2432389|2432664|N|N|N|N|N| +2432756|AAAAAAAAEPOBFCAA|1948-07-23|582|2534|195|1948|5|7|23|3|1948|195|2534|Friday|1948Q3|N|Y|N|2432734|2432915|2432390|2432665|N|N|N|N|N| +2432757|AAAAAAAAFPOBFCAA|1948-07-24|582|2534|195|1948|6|7|24|3|1948|195|2534|Saturday|1948Q3|N|Y|N|2432734|2432915|2432391|2432666|N|N|N|N|N| +2432758|AAAAAAAAGPOBFCAA|1948-07-25|582|2534|195|1948|0|7|25|3|1948|195|2534|Sunday|1948Q3|N|N|N|2432734|2432915|2432392|2432667|N|N|N|N|N| +2432759|AAAAAAAAHPOBFCAA|1948-07-26|582|2534|195|1948|1|7|26|3|1948|195|2534|Monday|1948Q3|N|N|N|2432734|2432915|2432393|2432668|N|N|N|N|N| +2432760|AAAAAAAAIPOBFCAA|1948-07-27|582|2535|195|1948|2|7|27|3|1948|195|2535|Tuesday|1948Q3|N|N|N|2432734|2432915|2432394|2432669|N|N|N|N|N| +2432761|AAAAAAAAJPOBFCAA|1948-07-28|582|2535|195|1948|3|7|28|3|1948|195|2535|Wednesday|1948Q3|N|N|N|2432734|2432915|2432395|2432670|N|N|N|N|N| +2432762|AAAAAAAAKPOBFCAA|1948-07-29|582|2535|195|1948|4|7|29|3|1948|195|2535|Thursday|1948Q3|N|N|N|2432734|2432915|2432396|2432671|N|N|N|N|N| +2432763|AAAAAAAALPOBFCAA|1948-07-30|582|2535|195|1948|5|7|30|3|1948|195|2535|Friday|1948Q3|N|Y|N|2432734|2432915|2432397|2432672|N|N|N|N|N| +2432764|AAAAAAAAMPOBFCAA|1948-07-31|582|2535|195|1948|6|7|31|3|1948|195|2535|Saturday|1948Q3|N|Y|N|2432734|2432915|2432398|2432673|N|N|N|N|N| +2432765|AAAAAAAANPOBFCAA|1948-08-01|583|2535|195|1948|0|8|1|3|1948|195|2535|Sunday|1948Q3|N|N|N|2432765|2432977|2432399|2432674|N|N|N|N|N| +2432766|AAAAAAAAOPOBFCAA|1948-08-02|583|2535|195|1948|1|8|2|3|1948|195|2535|Monday|1948Q3|N|N|N|2432765|2432977|2432400|2432675|N|N|N|N|N| +2432767|AAAAAAAAPPOBFCAA|1948-08-03|583|2536|195|1948|2|8|3|3|1948|195|2536|Tuesday|1948Q3|N|N|N|2432765|2432977|2432401|2432676|N|N|N|N|N| +2432768|AAAAAAAAAAPBFCAA|1948-08-04|583|2536|195|1948|3|8|4|3|1948|195|2536|Wednesday|1948Q3|N|N|N|2432765|2432977|2432402|2432677|N|N|N|N|N| +2432769|AAAAAAAABAPBFCAA|1948-08-05|583|2536|195|1948|4|8|5|3|1948|195|2536|Thursday|1948Q3|N|N|N|2432765|2432977|2432403|2432678|N|N|N|N|N| +2432770|AAAAAAAACAPBFCAA|1948-08-06|583|2536|195|1948|5|8|6|3|1948|195|2536|Friday|1948Q3|N|Y|N|2432765|2432977|2432404|2432679|N|N|N|N|N| +2432771|AAAAAAAADAPBFCAA|1948-08-07|583|2536|195|1948|6|8|7|3|1948|195|2536|Saturday|1948Q3|N|Y|N|2432765|2432977|2432405|2432680|N|N|N|N|N| +2432772|AAAAAAAAEAPBFCAA|1948-08-08|583|2536|195|1948|0|8|8|3|1948|195|2536|Sunday|1948Q3|N|N|N|2432765|2432977|2432406|2432681|N|N|N|N|N| +2432773|AAAAAAAAFAPBFCAA|1948-08-09|583|2536|195|1948|1|8|9|3|1948|195|2536|Monday|1948Q3|N|N|N|2432765|2432977|2432407|2432682|N|N|N|N|N| +2432774|AAAAAAAAGAPBFCAA|1948-08-10|583|2537|195|1948|2|8|10|3|1948|195|2537|Tuesday|1948Q3|N|N|N|2432765|2432977|2432408|2432683|N|N|N|N|N| +2432775|AAAAAAAAHAPBFCAA|1948-08-11|583|2537|195|1948|3|8|11|3|1948|195|2537|Wednesday|1948Q3|N|N|N|2432765|2432977|2432409|2432684|N|N|N|N|N| +2432776|AAAAAAAAIAPBFCAA|1948-08-12|583|2537|195|1948|4|8|12|3|1948|195|2537|Thursday|1948Q3|N|N|N|2432765|2432977|2432410|2432685|N|N|N|N|N| +2432777|AAAAAAAAJAPBFCAA|1948-08-13|583|2537|195|1948|5|8|13|3|1948|195|2537|Friday|1948Q3|N|Y|N|2432765|2432977|2432411|2432686|N|N|N|N|N| +2432778|AAAAAAAAKAPBFCAA|1948-08-14|583|2537|195|1948|6|8|14|3|1948|195|2537|Saturday|1948Q3|N|Y|N|2432765|2432977|2432412|2432687|N|N|N|N|N| +2432779|AAAAAAAALAPBFCAA|1948-08-15|583|2537|195|1948|0|8|15|3|1948|195|2537|Sunday|1948Q3|N|N|N|2432765|2432977|2432413|2432688|N|N|N|N|N| +2432780|AAAAAAAAMAPBFCAA|1948-08-16|583|2537|195|1948|1|8|16|3|1948|195|2537|Monday|1948Q3|N|N|N|2432765|2432977|2432414|2432689|N|N|N|N|N| +2432781|AAAAAAAANAPBFCAA|1948-08-17|583|2538|195|1948|2|8|17|3|1948|195|2538|Tuesday|1948Q3|N|N|N|2432765|2432977|2432415|2432690|N|N|N|N|N| +2432782|AAAAAAAAOAPBFCAA|1948-08-18|583|2538|195|1948|3|8|18|3|1948|195|2538|Wednesday|1948Q3|N|N|N|2432765|2432977|2432416|2432691|N|N|N|N|N| +2432783|AAAAAAAAPAPBFCAA|1948-08-19|583|2538|195|1948|4|8|19|3|1948|195|2538|Thursday|1948Q3|N|N|N|2432765|2432977|2432417|2432692|N|N|N|N|N| +2432784|AAAAAAAAABPBFCAA|1948-08-20|583|2538|195|1948|5|8|20|3|1948|195|2538|Friday|1948Q3|N|Y|N|2432765|2432977|2432418|2432693|N|N|N|N|N| +2432785|AAAAAAAABBPBFCAA|1948-08-21|583|2538|195|1948|6|8|21|3|1948|195|2538|Saturday|1948Q3|N|Y|N|2432765|2432977|2432419|2432694|N|N|N|N|N| +2432786|AAAAAAAACBPBFCAA|1948-08-22|583|2538|195|1948|0|8|22|3|1948|195|2538|Sunday|1948Q3|N|N|N|2432765|2432977|2432420|2432695|N|N|N|N|N| +2432787|AAAAAAAADBPBFCAA|1948-08-23|583|2538|195|1948|1|8|23|3|1948|195|2538|Monday|1948Q3|N|N|N|2432765|2432977|2432421|2432696|N|N|N|N|N| +2432788|AAAAAAAAEBPBFCAA|1948-08-24|583|2539|195|1948|2|8|24|3|1948|195|2539|Tuesday|1948Q3|N|N|N|2432765|2432977|2432422|2432697|N|N|N|N|N| +2432789|AAAAAAAAFBPBFCAA|1948-08-25|583|2539|195|1948|3|8|25|3|1948|195|2539|Wednesday|1948Q3|N|N|N|2432765|2432977|2432423|2432698|N|N|N|N|N| +2432790|AAAAAAAAGBPBFCAA|1948-08-26|583|2539|195|1948|4|8|26|3|1948|195|2539|Thursday|1948Q3|N|N|N|2432765|2432977|2432424|2432699|N|N|N|N|N| +2432791|AAAAAAAAHBPBFCAA|1948-08-27|583|2539|195|1948|5|8|27|3|1948|195|2539|Friday|1948Q3|N|Y|N|2432765|2432977|2432425|2432700|N|N|N|N|N| +2432792|AAAAAAAAIBPBFCAA|1948-08-28|583|2539|195|1948|6|8|28|3|1948|195|2539|Saturday|1948Q3|N|Y|N|2432765|2432977|2432426|2432701|N|N|N|N|N| +2432793|AAAAAAAAJBPBFCAA|1948-08-29|583|2539|195|1948|0|8|29|3|1948|195|2539|Sunday|1948Q3|N|N|N|2432765|2432977|2432427|2432702|N|N|N|N|N| +2432794|AAAAAAAAKBPBFCAA|1948-08-30|583|2539|195|1948|1|8|30|3|1948|195|2539|Monday|1948Q3|N|N|N|2432765|2432977|2432428|2432703|N|N|N|N|N| +2432795|AAAAAAAALBPBFCAA|1948-08-31|583|2540|195|1948|2|8|31|3|1948|195|2540|Tuesday|1948Q3|N|N|N|2432765|2432977|2432429|2432704|N|N|N|N|N| +2432796|AAAAAAAAMBPBFCAA|1948-09-01|584|2540|196|1948|3|9|1|3|1948|196|2540|Wednesday|1948Q3|N|N|N|2432796|2433039|2432430|2432705|N|N|N|N|N| +2432797|AAAAAAAANBPBFCAA|1948-09-02|584|2540|196|1948|4|9|2|3|1948|196|2540|Thursday|1948Q3|N|N|N|2432796|2433039|2432431|2432706|N|N|N|N|N| +2432798|AAAAAAAAOBPBFCAA|1948-09-03|584|2540|196|1948|5|9|3|3|1948|196|2540|Friday|1948Q3|N|Y|N|2432796|2433039|2432432|2432707|N|N|N|N|N| +2432799|AAAAAAAAPBPBFCAA|1948-09-04|584|2540|196|1948|6|9|4|3|1948|196|2540|Saturday|1948Q3|N|Y|N|2432796|2433039|2432433|2432708|N|N|N|N|N| +2432800|AAAAAAAAACPBFCAA|1948-09-05|584|2540|196|1948|0|9|5|3|1948|196|2540|Sunday|1948Q3|N|N|N|2432796|2433039|2432434|2432709|N|N|N|N|N| +2432801|AAAAAAAABCPBFCAA|1948-09-06|584|2540|196|1948|1|9|6|3|1948|196|2540|Monday|1948Q3|N|N|N|2432796|2433039|2432435|2432710|N|N|N|N|N| +2432802|AAAAAAAACCPBFCAA|1948-09-07|584|2541|196|1948|2|9|7|3|1948|196|2541|Tuesday|1948Q3|N|N|N|2432796|2433039|2432436|2432711|N|N|N|N|N| +2432803|AAAAAAAADCPBFCAA|1948-09-08|584|2541|196|1948|3|9|8|3|1948|196|2541|Wednesday|1948Q3|N|N|N|2432796|2433039|2432437|2432712|N|N|N|N|N| +2432804|AAAAAAAAECPBFCAA|1948-09-09|584|2541|196|1948|4|9|9|3|1948|196|2541|Thursday|1948Q3|N|N|N|2432796|2433039|2432438|2432713|N|N|N|N|N| +2432805|AAAAAAAAFCPBFCAA|1948-09-10|584|2541|196|1948|5|9|10|3|1948|196|2541|Friday|1948Q3|N|Y|N|2432796|2433039|2432439|2432714|N|N|N|N|N| +2432806|AAAAAAAAGCPBFCAA|1948-09-11|584|2541|196|1948|6|9|11|3|1948|196|2541|Saturday|1948Q3|N|Y|N|2432796|2433039|2432440|2432715|N|N|N|N|N| +2432807|AAAAAAAAHCPBFCAA|1948-09-12|584|2541|196|1948|0|9|12|3|1948|196|2541|Sunday|1948Q3|N|N|N|2432796|2433039|2432441|2432716|N|N|N|N|N| +2432808|AAAAAAAAICPBFCAA|1948-09-13|584|2541|196|1948|1|9|13|3|1948|196|2541|Monday|1948Q3|N|N|N|2432796|2433039|2432442|2432717|N|N|N|N|N| +2432809|AAAAAAAAJCPBFCAA|1948-09-14|584|2542|196|1948|2|9|14|3|1948|196|2542|Tuesday|1948Q3|N|N|N|2432796|2433039|2432443|2432718|N|N|N|N|N| +2432810|AAAAAAAAKCPBFCAA|1948-09-15|584|2542|196|1948|3|9|15|3|1948|196|2542|Wednesday|1948Q3|N|N|N|2432796|2433039|2432444|2432719|N|N|N|N|N| +2432811|AAAAAAAALCPBFCAA|1948-09-16|584|2542|196|1948|4|9|16|3|1948|196|2542|Thursday|1948Q3|N|N|N|2432796|2433039|2432445|2432720|N|N|N|N|N| +2432812|AAAAAAAAMCPBFCAA|1948-09-17|584|2542|196|1948|5|9|17|3|1948|196|2542|Friday|1948Q3|N|Y|N|2432796|2433039|2432446|2432721|N|N|N|N|N| +2432813|AAAAAAAANCPBFCAA|1948-09-18|584|2542|196|1948|6|9|18|3|1948|196|2542|Saturday|1948Q3|N|Y|N|2432796|2433039|2432447|2432722|N|N|N|N|N| +2432814|AAAAAAAAOCPBFCAA|1948-09-19|584|2542|196|1948|0|9|19|3|1948|196|2542|Sunday|1948Q3|N|N|N|2432796|2433039|2432448|2432723|N|N|N|N|N| +2432815|AAAAAAAAPCPBFCAA|1948-09-20|584|2542|196|1948|1|9|20|3|1948|196|2542|Monday|1948Q3|N|N|N|2432796|2433039|2432449|2432724|N|N|N|N|N| +2432816|AAAAAAAAADPBFCAA|1948-09-21|584|2543|196|1948|2|9|21|3|1948|196|2543|Tuesday|1948Q3|N|N|N|2432796|2433039|2432450|2432725|N|N|N|N|N| +2432817|AAAAAAAABDPBFCAA|1948-09-22|584|2543|196|1948|3|9|22|3|1948|196|2543|Wednesday|1948Q3|N|N|N|2432796|2433039|2432451|2432726|N|N|N|N|N| +2432818|AAAAAAAACDPBFCAA|1948-09-23|584|2543|196|1948|4|9|23|3|1948|196|2543|Thursday|1948Q3|N|N|N|2432796|2433039|2432452|2432727|N|N|N|N|N| +2432819|AAAAAAAADDPBFCAA|1948-09-24|584|2543|196|1948|5|9|24|3|1948|196|2543|Friday|1948Q3|N|Y|N|2432796|2433039|2432453|2432728|N|N|N|N|N| +2432820|AAAAAAAAEDPBFCAA|1948-09-25|584|2543|196|1948|6|9|25|3|1948|196|2543|Saturday|1948Q3|N|Y|N|2432796|2433039|2432454|2432729|N|N|N|N|N| +2432821|AAAAAAAAFDPBFCAA|1948-09-26|584|2543|196|1948|0|9|26|3|1948|196|2543|Sunday|1948Q3|N|N|N|2432796|2433039|2432455|2432730|N|N|N|N|N| +2432822|AAAAAAAAGDPBFCAA|1948-09-27|584|2543|196|1948|1|9|27|3|1948|196|2543|Monday|1948Q3|N|N|N|2432796|2433039|2432456|2432731|N|N|N|N|N| +2432823|AAAAAAAAHDPBFCAA|1948-09-28|584|2544|196|1948|2|9|28|3|1948|196|2544|Tuesday|1948Q3|N|N|N|2432796|2433039|2432457|2432732|N|N|N|N|N| +2432824|AAAAAAAAIDPBFCAA|1948-09-29|584|2544|196|1948|3|9|29|3|1948|196|2544|Wednesday|1948Q3|N|N|N|2432796|2433039|2432458|2432733|N|N|N|N|N| +2432825|AAAAAAAAJDPBFCAA|1948-09-30|584|2544|196|1948|4|9|30|3|1948|196|2544|Thursday|1948Q3|N|N|N|2432796|2433039|2432459|2432734|N|N|N|N|N| +2432826|AAAAAAAAKDPBFCAA|1948-10-01|585|2544|196|1948|5|10|1|4|1948|196|2544|Friday|1948Q4|N|Y|N|2432826|2433099|2432460|2432734|N|N|N|N|N| +2432827|AAAAAAAALDPBFCAA|1948-10-02|585|2544|196|1948|6|10|2|4|1948|196|2544|Saturday|1948Q4|N|Y|N|2432826|2433099|2432461|2432735|N|N|N|N|N| +2432828|AAAAAAAAMDPBFCAA|1948-10-03|585|2544|196|1948|0|10|3|4|1948|196|2544|Sunday|1948Q4|N|N|N|2432826|2433099|2432462|2432736|N|N|N|N|N| +2432829|AAAAAAAANDPBFCAA|1948-10-04|585|2544|196|1948|1|10|4|4|1948|196|2544|Monday|1948Q4|N|N|N|2432826|2433099|2432463|2432737|N|N|N|N|N| +2432830|AAAAAAAAODPBFCAA|1948-10-05|585|2545|196|1948|2|10|5|4|1948|196|2545|Tuesday|1948Q4|N|N|N|2432826|2433099|2432464|2432738|N|N|N|N|N| +2432831|AAAAAAAAPDPBFCAA|1948-10-06|585|2545|196|1948|3|10|6|4|1948|196|2545|Wednesday|1948Q4|N|N|N|2432826|2433099|2432465|2432739|N|N|N|N|N| +2432832|AAAAAAAAAEPBFCAA|1948-10-07|585|2545|196|1948|4|10|7|4|1948|196|2545|Thursday|1948Q4|N|N|N|2432826|2433099|2432466|2432740|N|N|N|N|N| +2432833|AAAAAAAABEPBFCAA|1948-10-08|585|2545|196|1948|5|10|8|4|1948|196|2545|Friday|1948Q4|N|Y|N|2432826|2433099|2432467|2432741|N|N|N|N|N| +2432834|AAAAAAAACEPBFCAA|1948-10-09|585|2545|196|1948|6|10|9|4|1948|196|2545|Saturday|1948Q4|N|Y|N|2432826|2433099|2432468|2432742|N|N|N|N|N| +2432835|AAAAAAAADEPBFCAA|1948-10-10|585|2545|196|1948|0|10|10|4|1948|196|2545|Sunday|1948Q4|N|N|N|2432826|2433099|2432469|2432743|N|N|N|N|N| +2432836|AAAAAAAAEEPBFCAA|1948-10-11|585|2545|196|1948|1|10|11|4|1948|196|2545|Monday|1948Q4|N|N|N|2432826|2433099|2432470|2432744|N|N|N|N|N| +2432837|AAAAAAAAFEPBFCAA|1948-10-12|585|2546|196|1948|2|10|12|4|1948|196|2546|Tuesday|1948Q4|N|N|N|2432826|2433099|2432471|2432745|N|N|N|N|N| +2432838|AAAAAAAAGEPBFCAA|1948-10-13|585|2546|196|1948|3|10|13|4|1948|196|2546|Wednesday|1948Q4|N|N|N|2432826|2433099|2432472|2432746|N|N|N|N|N| +2432839|AAAAAAAAHEPBFCAA|1948-10-14|585|2546|196|1948|4|10|14|4|1948|196|2546|Thursday|1948Q4|N|N|N|2432826|2433099|2432473|2432747|N|N|N|N|N| +2432840|AAAAAAAAIEPBFCAA|1948-10-15|585|2546|196|1948|5|10|15|4|1948|196|2546|Friday|1948Q4|N|Y|N|2432826|2433099|2432474|2432748|N|N|N|N|N| +2432841|AAAAAAAAJEPBFCAA|1948-10-16|585|2546|196|1948|6|10|16|4|1948|196|2546|Saturday|1948Q4|N|Y|N|2432826|2433099|2432475|2432749|N|N|N|N|N| +2432842|AAAAAAAAKEPBFCAA|1948-10-17|585|2546|196|1948|0|10|17|4|1948|196|2546|Sunday|1948Q4|N|N|N|2432826|2433099|2432476|2432750|N|N|N|N|N| +2432843|AAAAAAAALEPBFCAA|1948-10-18|585|2546|196|1948|1|10|18|4|1948|196|2546|Monday|1948Q4|N|N|N|2432826|2433099|2432477|2432751|N|N|N|N|N| +2432844|AAAAAAAAMEPBFCAA|1948-10-19|585|2547|196|1948|2|10|19|4|1948|196|2547|Tuesday|1948Q4|N|N|N|2432826|2433099|2432478|2432752|N|N|N|N|N| +2432845|AAAAAAAANEPBFCAA|1948-10-20|585|2547|196|1948|3|10|20|4|1948|196|2547|Wednesday|1948Q4|N|N|N|2432826|2433099|2432479|2432753|N|N|N|N|N| +2432846|AAAAAAAAOEPBFCAA|1948-10-21|585|2547|196|1948|4|10|21|4|1948|196|2547|Thursday|1948Q4|N|N|N|2432826|2433099|2432480|2432754|N|N|N|N|N| +2432847|AAAAAAAAPEPBFCAA|1948-10-22|585|2547|196|1948|5|10|22|4|1948|196|2547|Friday|1948Q4|N|Y|N|2432826|2433099|2432481|2432755|N|N|N|N|N| +2432848|AAAAAAAAAFPBFCAA|1948-10-23|585|2547|196|1948|6|10|23|4|1948|196|2547|Saturday|1948Q4|N|Y|N|2432826|2433099|2432482|2432756|N|N|N|N|N| +2432849|AAAAAAAABFPBFCAA|1948-10-24|585|2547|196|1948|0|10|24|4|1948|196|2547|Sunday|1948Q4|N|N|N|2432826|2433099|2432483|2432757|N|N|N|N|N| +2432850|AAAAAAAACFPBFCAA|1948-10-25|585|2547|196|1948|1|10|25|4|1948|196|2547|Monday|1948Q4|N|N|N|2432826|2433099|2432484|2432758|N|N|N|N|N| +2432851|AAAAAAAADFPBFCAA|1948-10-26|585|2548|196|1948|2|10|26|4|1948|196|2548|Tuesday|1948Q4|N|N|N|2432826|2433099|2432485|2432759|N|N|N|N|N| +2432852|AAAAAAAAEFPBFCAA|1948-10-27|585|2548|196|1948|3|10|27|4|1948|196|2548|Wednesday|1948Q4|N|N|N|2432826|2433099|2432486|2432760|N|N|N|N|N| +2432853|AAAAAAAAFFPBFCAA|1948-10-28|585|2548|196|1948|4|10|28|4|1948|196|2548|Thursday|1948Q4|N|N|N|2432826|2433099|2432487|2432761|N|N|N|N|N| +2432854|AAAAAAAAGFPBFCAA|1948-10-29|585|2548|196|1948|5|10|29|4|1948|196|2548|Friday|1948Q4|N|Y|N|2432826|2433099|2432488|2432762|N|N|N|N|N| +2432855|AAAAAAAAHFPBFCAA|1948-10-30|585|2548|196|1948|6|10|30|4|1948|196|2548|Saturday|1948Q4|N|Y|N|2432826|2433099|2432489|2432763|N|N|N|N|N| +2432856|AAAAAAAAIFPBFCAA|1948-10-31|585|2548|196|1948|0|10|31|4|1948|196|2548|Sunday|1948Q4|N|N|N|2432826|2433099|2432490|2432764|N|N|N|N|N| +2432857|AAAAAAAAJFPBFCAA|1948-11-01|586|2548|196|1948|1|11|1|4|1948|196|2548|Monday|1948Q4|N|N|N|2432857|2433161|2432491|2432765|N|N|N|N|N| +2432858|AAAAAAAAKFPBFCAA|1948-11-02|586|2549|196|1948|2|11|2|4|1948|196|2549|Tuesday|1948Q4|N|N|N|2432857|2433161|2432492|2432766|N|N|N|N|N| +2432859|AAAAAAAALFPBFCAA|1948-11-03|586|2549|196|1948|3|11|3|4|1948|196|2549|Wednesday|1948Q4|N|N|N|2432857|2433161|2432493|2432767|N|N|N|N|N| +2432860|AAAAAAAAMFPBFCAA|1948-11-04|586|2549|196|1948|4|11|4|4|1948|196|2549|Thursday|1948Q4|N|N|N|2432857|2433161|2432494|2432768|N|N|N|N|N| +2432861|AAAAAAAANFPBFCAA|1948-11-05|586|2549|196|1948|5|11|5|4|1948|196|2549|Friday|1948Q4|N|Y|N|2432857|2433161|2432495|2432769|N|N|N|N|N| +2432862|AAAAAAAAOFPBFCAA|1948-11-06|586|2549|196|1948|6|11|6|4|1948|196|2549|Saturday|1948Q4|N|Y|N|2432857|2433161|2432496|2432770|N|N|N|N|N| +2432863|AAAAAAAAPFPBFCAA|1948-11-07|586|2549|196|1948|0|11|7|4|1948|196|2549|Sunday|1948Q4|N|N|N|2432857|2433161|2432497|2432771|N|N|N|N|N| +2432864|AAAAAAAAAGPBFCAA|1948-11-08|586|2549|196|1948|1|11|8|4|1948|196|2549|Monday|1948Q4|N|N|N|2432857|2433161|2432498|2432772|N|N|N|N|N| +2432865|AAAAAAAABGPBFCAA|1948-11-09|586|2550|196|1948|2|11|9|4|1948|196|2550|Tuesday|1948Q4|N|N|N|2432857|2433161|2432499|2432773|N|N|N|N|N| +2432866|AAAAAAAACGPBFCAA|1948-11-10|586|2550|196|1948|3|11|10|4|1948|196|2550|Wednesday|1948Q4|N|N|N|2432857|2433161|2432500|2432774|N|N|N|N|N| +2432867|AAAAAAAADGPBFCAA|1948-11-11|586|2550|196|1948|4|11|11|4|1948|196|2550|Thursday|1948Q4|N|N|N|2432857|2433161|2432501|2432775|N|N|N|N|N| +2432868|AAAAAAAAEGPBFCAA|1948-11-12|586|2550|196|1948|5|11|12|4|1948|196|2550|Friday|1948Q4|N|Y|N|2432857|2433161|2432502|2432776|N|N|N|N|N| +2432869|AAAAAAAAFGPBFCAA|1948-11-13|586|2550|196|1948|6|11|13|4|1948|196|2550|Saturday|1948Q4|N|Y|N|2432857|2433161|2432503|2432777|N|N|N|N|N| +2432870|AAAAAAAAGGPBFCAA|1948-11-14|586|2550|196|1948|0|11|14|4|1948|196|2550|Sunday|1948Q4|N|N|N|2432857|2433161|2432504|2432778|N|N|N|N|N| +2432871|AAAAAAAAHGPBFCAA|1948-11-15|586|2550|196|1948|1|11|15|4|1948|196|2550|Monday|1948Q4|N|N|N|2432857|2433161|2432505|2432779|N|N|N|N|N| +2432872|AAAAAAAAIGPBFCAA|1948-11-16|586|2551|196|1948|2|11|16|4|1948|196|2551|Tuesday|1948Q4|N|N|N|2432857|2433161|2432506|2432780|N|N|N|N|N| +2432873|AAAAAAAAJGPBFCAA|1948-11-17|586|2551|196|1948|3|11|17|4|1948|196|2551|Wednesday|1948Q4|N|N|N|2432857|2433161|2432507|2432781|N|N|N|N|N| +2432874|AAAAAAAAKGPBFCAA|1948-11-18|586|2551|196|1948|4|11|18|4|1948|196|2551|Thursday|1948Q4|N|N|N|2432857|2433161|2432508|2432782|N|N|N|N|N| +2432875|AAAAAAAALGPBFCAA|1948-11-19|586|2551|196|1948|5|11|19|4|1948|196|2551|Friday|1948Q4|N|Y|N|2432857|2433161|2432509|2432783|N|N|N|N|N| +2432876|AAAAAAAAMGPBFCAA|1948-11-20|586|2551|196|1948|6|11|20|4|1948|196|2551|Saturday|1948Q4|N|Y|N|2432857|2433161|2432510|2432784|N|N|N|N|N| +2432877|AAAAAAAANGPBFCAA|1948-11-21|586|2551|196|1948|0|11|21|4|1948|196|2551|Sunday|1948Q4|N|N|N|2432857|2433161|2432511|2432785|N|N|N|N|N| +2432878|AAAAAAAAOGPBFCAA|1948-11-22|586|2551|196|1948|1|11|22|4|1948|196|2551|Monday|1948Q4|N|N|N|2432857|2433161|2432512|2432786|N|N|N|N|N| +2432879|AAAAAAAAPGPBFCAA|1948-11-23|586|2552|196|1948|2|11|23|4|1948|196|2552|Tuesday|1948Q4|N|N|N|2432857|2433161|2432513|2432787|N|N|N|N|N| +2432880|AAAAAAAAAHPBFCAA|1948-11-24|586|2552|196|1948|3|11|24|4|1948|196|2552|Wednesday|1948Q4|N|N|N|2432857|2433161|2432514|2432788|N|N|N|N|N| +2432881|AAAAAAAABHPBFCAA|1948-11-25|586|2552|196|1948|4|11|25|4|1948|196|2552|Thursday|1948Q4|N|N|N|2432857|2433161|2432515|2432789|N|N|N|N|N| +2432882|AAAAAAAACHPBFCAA|1948-11-26|586|2552|196|1948|5|11|26|4|1948|196|2552|Friday|1948Q4|N|Y|N|2432857|2433161|2432516|2432790|N|N|N|N|N| +2432883|AAAAAAAADHPBFCAA|1948-11-27|586|2552|196|1948|6|11|27|4|1948|196|2552|Saturday|1948Q4|N|Y|N|2432857|2433161|2432517|2432791|N|N|N|N|N| +2432884|AAAAAAAAEHPBFCAA|1948-11-28|586|2552|196|1948|0|11|28|4|1948|196|2552|Sunday|1948Q4|N|N|N|2432857|2433161|2432518|2432792|N|N|N|N|N| +2432885|AAAAAAAAFHPBFCAA|1948-11-29|586|2552|196|1948|1|11|29|4|1948|196|2552|Monday|1948Q4|N|N|N|2432857|2433161|2432519|2432793|N|N|N|N|N| +2432886|AAAAAAAAGHPBFCAA|1948-11-30|586|2553|196|1948|2|11|30|4|1948|196|2553|Tuesday|1948Q4|N|N|N|2432857|2433161|2432520|2432794|N|N|N|N|N| +2432887|AAAAAAAAHHPBFCAA|1948-12-01|587|2553|197|1948|3|12|1|4|1948|197|2553|Wednesday|1948Q4|N|N|N|2432887|2433221|2432521|2432795|N|N|N|N|N| +2432888|AAAAAAAAIHPBFCAA|1948-12-02|587|2553|197|1948|4|12|2|4|1948|197|2553|Thursday|1948Q4|N|N|N|2432887|2433221|2432522|2432796|N|N|N|N|N| +2432889|AAAAAAAAJHPBFCAA|1948-12-03|587|2553|197|1948|5|12|3|4|1948|197|2553|Friday|1948Q4|N|Y|N|2432887|2433221|2432523|2432797|N|N|N|N|N| +2432890|AAAAAAAAKHPBFCAA|1948-12-04|587|2553|197|1948|6|12|4|4|1948|197|2553|Saturday|1948Q4|N|Y|N|2432887|2433221|2432524|2432798|N|N|N|N|N| +2432891|AAAAAAAALHPBFCAA|1948-12-05|587|2553|197|1948|0|12|5|4|1948|197|2553|Sunday|1948Q4|N|N|N|2432887|2433221|2432525|2432799|N|N|N|N|N| +2432892|AAAAAAAAMHPBFCAA|1948-12-06|587|2553|197|1948|1|12|6|4|1948|197|2553|Monday|1948Q4|N|N|N|2432887|2433221|2432526|2432800|N|N|N|N|N| +2432893|AAAAAAAANHPBFCAA|1948-12-07|587|2554|197|1948|2|12|7|4|1948|197|2554|Tuesday|1948Q4|N|N|N|2432887|2433221|2432527|2432801|N|N|N|N|N| +2432894|AAAAAAAAOHPBFCAA|1948-12-08|587|2554|197|1948|3|12|8|4|1948|197|2554|Wednesday|1948Q4|N|N|N|2432887|2433221|2432528|2432802|N|N|N|N|N| +2432895|AAAAAAAAPHPBFCAA|1948-12-09|587|2554|197|1948|4|12|9|4|1948|197|2554|Thursday|1948Q4|N|N|N|2432887|2433221|2432529|2432803|N|N|N|N|N| +2432896|AAAAAAAAAIPBFCAA|1948-12-10|587|2554|197|1948|5|12|10|4|1948|197|2554|Friday|1948Q4|N|Y|N|2432887|2433221|2432530|2432804|N|N|N|N|N| +2432897|AAAAAAAABIPBFCAA|1948-12-11|587|2554|197|1948|6|12|11|4|1948|197|2554|Saturday|1948Q4|N|Y|N|2432887|2433221|2432531|2432805|N|N|N|N|N| +2432898|AAAAAAAACIPBFCAA|1948-12-12|587|2554|197|1948|0|12|12|4|1948|197|2554|Sunday|1948Q4|N|N|N|2432887|2433221|2432532|2432806|N|N|N|N|N| +2432899|AAAAAAAADIPBFCAA|1948-12-13|587|2554|197|1948|1|12|13|4|1948|197|2554|Monday|1948Q4|N|N|N|2432887|2433221|2432533|2432807|N|N|N|N|N| +2432900|AAAAAAAAEIPBFCAA|1948-12-14|587|2555|197|1948|2|12|14|4|1948|197|2555|Tuesday|1948Q4|N|N|N|2432887|2433221|2432534|2432808|N|N|N|N|N| +2432901|AAAAAAAAFIPBFCAA|1948-12-15|587|2555|197|1948|3|12|15|4|1948|197|2555|Wednesday|1948Q4|N|N|N|2432887|2433221|2432535|2432809|N|N|N|N|N| +2432902|AAAAAAAAGIPBFCAA|1948-12-16|587|2555|197|1948|4|12|16|4|1948|197|2555|Thursday|1948Q4|N|N|N|2432887|2433221|2432536|2432810|N|N|N|N|N| +2432903|AAAAAAAAHIPBFCAA|1948-12-17|587|2555|197|1948|5|12|17|4|1948|197|2555|Friday|1948Q4|N|Y|N|2432887|2433221|2432537|2432811|N|N|N|N|N| +2432904|AAAAAAAAIIPBFCAA|1948-12-18|587|2555|197|1948|6|12|18|4|1948|197|2555|Saturday|1948Q4|N|Y|N|2432887|2433221|2432538|2432812|N|N|N|N|N| +2432905|AAAAAAAAJIPBFCAA|1948-12-19|587|2555|197|1948|0|12|19|4|1948|197|2555|Sunday|1948Q4|N|N|N|2432887|2433221|2432539|2432813|N|N|N|N|N| +2432906|AAAAAAAAKIPBFCAA|1948-12-20|587|2555|197|1948|1|12|20|4|1948|197|2555|Monday|1948Q4|N|N|N|2432887|2433221|2432540|2432814|N|N|N|N|N| +2432907|AAAAAAAALIPBFCAA|1948-12-21|587|2556|197|1948|2|12|21|4|1948|197|2556|Tuesday|1948Q4|N|N|N|2432887|2433221|2432541|2432815|N|N|N|N|N| +2432908|AAAAAAAAMIPBFCAA|1948-12-22|587|2556|197|1948|3|12|22|4|1948|197|2556|Wednesday|1948Q4|N|N|N|2432887|2433221|2432542|2432816|N|N|N|N|N| +2432909|AAAAAAAANIPBFCAA|1948-12-23|587|2556|197|1948|4|12|23|4|1948|197|2556|Thursday|1948Q4|N|N|N|2432887|2433221|2432543|2432817|N|N|N|N|N| +2432910|AAAAAAAAOIPBFCAA|1948-12-24|587|2556|197|1948|5|12|24|4|1948|197|2556|Friday|1948Q4|N|Y|N|2432887|2433221|2432544|2432818|N|N|N|N|N| +2432911|AAAAAAAAPIPBFCAA|1948-12-25|587|2556|197|1948|6|12|25|4|1948|197|2556|Saturday|1948Q4|Y|Y|N|2432887|2433221|2432545|2432819|N|N|N|N|N| +2432912|AAAAAAAAAJPBFCAA|1948-12-26|587|2556|197|1948|0|12|26|4|1948|197|2556|Sunday|1948Q4|N|N|Y|2432887|2433221|2432546|2432820|N|N|N|N|N| +2432913|AAAAAAAABJPBFCAA|1948-12-27|587|2556|197|1948|1|12|27|4|1948|197|2556|Monday|1948Q4|N|N|N|2432887|2433221|2432547|2432821|N|N|N|N|N| +2432914|AAAAAAAACJPBFCAA|1948-12-28|587|2557|197|1948|2|12|28|4|1948|197|2557|Tuesday|1948Q4|N|N|N|2432887|2433221|2432548|2432822|N|N|N|N|N| +2432915|AAAAAAAADJPBFCAA|1948-12-29|587|2557|197|1948|3|12|29|4|1948|197|2557|Wednesday|1948Q4|N|N|N|2432887|2433221|2432549|2432823|N|N|N|N|N| +2432916|AAAAAAAAEJPBFCAA|1948-12-30|587|2557|197|1948|4|12|30|4|1948|197|2557|Thursday|1948Q4|N|N|N|2432887|2433221|2432550|2432824|N|N|N|N|N| +2432917|AAAAAAAAFJPBFCAA|1948-12-31|587|2557|197|1948|5|12|31|4|1948|197|2557|Friday|1948Q4|Y|Y|N|2432887|2433221|2432551|2432825|N|N|N|N|N| +2432918|AAAAAAAAGJPBFCAA|1949-01-01|588|2557|197|1949|6|1|1|1|1949|197|2557|Saturday|1949Q1|Y|Y|Y|2432918|2432917|2432552|2432826|N|N|N|N|N| +2432919|AAAAAAAAHJPBFCAA|1949-01-02|588|2557|197|1949|0|1|2|1|1949|197|2557|Sunday|1949Q1|N|N|Y|2432918|2432917|2432553|2432827|N|N|N|N|N| +2432920|AAAAAAAAIJPBFCAA|1949-01-03|588|2557|197|1949|1|1|3|1|1949|197|2557|Monday|1949Q1|N|N|N|2432918|2432917|2432554|2432828|N|N|N|N|N| +2432921|AAAAAAAAJJPBFCAA|1949-01-04|588|2558|197|1949|2|1|4|1|1949|197|2558|Tuesday|1949Q1|N|N|N|2432918|2432917|2432555|2432829|N|N|N|N|N| +2432922|AAAAAAAAKJPBFCAA|1949-01-05|588|2558|197|1949|3|1|5|1|1949|197|2558|Wednesday|1949Q1|N|N|N|2432918|2432917|2432556|2432830|N|N|N|N|N| +2432923|AAAAAAAALJPBFCAA|1949-01-06|588|2558|197|1949|4|1|6|1|1949|197|2558|Thursday|1949Q1|N|N|N|2432918|2432917|2432557|2432831|N|N|N|N|N| +2432924|AAAAAAAAMJPBFCAA|1949-01-07|588|2558|197|1949|5|1|7|1|1949|197|2558|Friday|1949Q1|N|Y|N|2432918|2432917|2432558|2432832|N|N|N|N|N| +2432925|AAAAAAAANJPBFCAA|1949-01-08|588|2558|197|1949|6|1|8|1|1949|197|2558|Saturday|1949Q1|N|Y|N|2432918|2432917|2432559|2432833|N|N|N|N|N| +2432926|AAAAAAAAOJPBFCAA|1949-01-09|588|2558|197|1949|0|1|9|1|1949|197|2558|Sunday|1949Q1|N|N|N|2432918|2432917|2432560|2432834|N|N|N|N|N| +2432927|AAAAAAAAPJPBFCAA|1949-01-10|588|2558|197|1949|1|1|10|1|1949|197|2558|Monday|1949Q1|N|N|N|2432918|2432917|2432561|2432835|N|N|N|N|N| +2432928|AAAAAAAAAKPBFCAA|1949-01-11|588|2559|197|1949|2|1|11|1|1949|197|2559|Tuesday|1949Q1|N|N|N|2432918|2432917|2432562|2432836|N|N|N|N|N| +2432929|AAAAAAAABKPBFCAA|1949-01-12|588|2559|197|1949|3|1|12|1|1949|197|2559|Wednesday|1949Q1|N|N|N|2432918|2432917|2432563|2432837|N|N|N|N|N| +2432930|AAAAAAAACKPBFCAA|1949-01-13|588|2559|197|1949|4|1|13|1|1949|197|2559|Thursday|1949Q1|N|N|N|2432918|2432917|2432564|2432838|N|N|N|N|N| +2432931|AAAAAAAADKPBFCAA|1949-01-14|588|2559|197|1949|5|1|14|1|1949|197|2559|Friday|1949Q1|N|Y|N|2432918|2432917|2432565|2432839|N|N|N|N|N| +2432932|AAAAAAAAEKPBFCAA|1949-01-15|588|2559|197|1949|6|1|15|1|1949|197|2559|Saturday|1949Q1|N|Y|N|2432918|2432917|2432566|2432840|N|N|N|N|N| +2432933|AAAAAAAAFKPBFCAA|1949-01-16|588|2559|197|1949|0|1|16|1|1949|197|2559|Sunday|1949Q1|N|N|N|2432918|2432917|2432567|2432841|N|N|N|N|N| +2432934|AAAAAAAAGKPBFCAA|1949-01-17|588|2559|197|1949|1|1|17|1|1949|197|2559|Monday|1949Q1|N|N|N|2432918|2432917|2432568|2432842|N|N|N|N|N| +2432935|AAAAAAAAHKPBFCAA|1949-01-18|588|2560|197|1949|2|1|18|1|1949|197|2560|Tuesday|1949Q1|N|N|N|2432918|2432917|2432569|2432843|N|N|N|N|N| +2432936|AAAAAAAAIKPBFCAA|1949-01-19|588|2560|197|1949|3|1|19|1|1949|197|2560|Wednesday|1949Q1|N|N|N|2432918|2432917|2432570|2432844|N|N|N|N|N| +2432937|AAAAAAAAJKPBFCAA|1949-01-20|588|2560|197|1949|4|1|20|1|1949|197|2560|Thursday|1949Q1|N|N|N|2432918|2432917|2432571|2432845|N|N|N|N|N| +2432938|AAAAAAAAKKPBFCAA|1949-01-21|588|2560|197|1949|5|1|21|1|1949|197|2560|Friday|1949Q1|N|Y|N|2432918|2432917|2432572|2432846|N|N|N|N|N| +2432939|AAAAAAAALKPBFCAA|1949-01-22|588|2560|197|1949|6|1|22|1|1949|197|2560|Saturday|1949Q1|N|Y|N|2432918|2432917|2432573|2432847|N|N|N|N|N| +2432940|AAAAAAAAMKPBFCAA|1949-01-23|588|2560|197|1949|0|1|23|1|1949|197|2560|Sunday|1949Q1|N|N|N|2432918|2432917|2432574|2432848|N|N|N|N|N| +2432941|AAAAAAAANKPBFCAA|1949-01-24|588|2560|197|1949|1|1|24|1|1949|197|2560|Monday|1949Q1|N|N|N|2432918|2432917|2432575|2432849|N|N|N|N|N| +2432942|AAAAAAAAOKPBFCAA|1949-01-25|588|2561|197|1949|2|1|25|1|1949|197|2561|Tuesday|1949Q1|N|N|N|2432918|2432917|2432576|2432850|N|N|N|N|N| +2432943|AAAAAAAAPKPBFCAA|1949-01-26|588|2561|197|1949|3|1|26|1|1949|197|2561|Wednesday|1949Q1|N|N|N|2432918|2432917|2432577|2432851|N|N|N|N|N| +2432944|AAAAAAAAALPBFCAA|1949-01-27|588|2561|197|1949|4|1|27|1|1949|197|2561|Thursday|1949Q1|N|N|N|2432918|2432917|2432578|2432852|N|N|N|N|N| +2432945|AAAAAAAABLPBFCAA|1949-01-28|588|2561|197|1949|5|1|28|1|1949|197|2561|Friday|1949Q1|N|Y|N|2432918|2432917|2432579|2432853|N|N|N|N|N| +2432946|AAAAAAAACLPBFCAA|1949-01-29|588|2561|197|1949|6|1|29|1|1949|197|2561|Saturday|1949Q1|N|Y|N|2432918|2432917|2432580|2432854|N|N|N|N|N| +2432947|AAAAAAAADLPBFCAA|1949-01-30|588|2561|197|1949|0|1|30|1|1949|197|2561|Sunday|1949Q1|N|N|N|2432918|2432917|2432581|2432855|N|N|N|N|N| +2432948|AAAAAAAAELPBFCAA|1949-01-31|588|2561|197|1949|1|1|31|1|1949|197|2561|Monday|1949Q1|N|N|N|2432918|2432917|2432582|2432856|N|N|N|N|N| +2432949|AAAAAAAAFLPBFCAA|1949-02-01|589|2562|197|1949|2|2|1|1|1949|197|2562|Tuesday|1949Q1|N|N|N|2432949|2432979|2432583|2432857|N|N|N|N|N| +2432950|AAAAAAAAGLPBFCAA|1949-02-02|589|2562|197|1949|3|2|2|1|1949|197|2562|Wednesday|1949Q1|N|N|N|2432949|2432979|2432584|2432858|N|N|N|N|N| +2432951|AAAAAAAAHLPBFCAA|1949-02-03|589|2562|197|1949|4|2|3|1|1949|197|2562|Thursday|1949Q1|N|N|N|2432949|2432979|2432585|2432859|N|N|N|N|N| +2432952|AAAAAAAAILPBFCAA|1949-02-04|589|2562|197|1949|5|2|4|1|1949|197|2562|Friday|1949Q1|N|Y|N|2432949|2432979|2432586|2432860|N|N|N|N|N| +2432953|AAAAAAAAJLPBFCAA|1949-02-05|589|2562|197|1949|6|2|5|1|1949|197|2562|Saturday|1949Q1|N|Y|N|2432949|2432979|2432587|2432861|N|N|N|N|N| +2432954|AAAAAAAAKLPBFCAA|1949-02-06|589|2562|197|1949|0|2|6|1|1949|197|2562|Sunday|1949Q1|N|N|N|2432949|2432979|2432588|2432862|N|N|N|N|N| +2432955|AAAAAAAALLPBFCAA|1949-02-07|589|2562|197|1949|1|2|7|1|1949|197|2562|Monday|1949Q1|N|N|N|2432949|2432979|2432589|2432863|N|N|N|N|N| +2432956|AAAAAAAAMLPBFCAA|1949-02-08|589|2563|197|1949|2|2|8|1|1949|197|2563|Tuesday|1949Q1|N|N|N|2432949|2432979|2432590|2432864|N|N|N|N|N| +2432957|AAAAAAAANLPBFCAA|1949-02-09|589|2563|197|1949|3|2|9|1|1949|197|2563|Wednesday|1949Q1|N|N|N|2432949|2432979|2432591|2432865|N|N|N|N|N| +2432958|AAAAAAAAOLPBFCAA|1949-02-10|589|2563|197|1949|4|2|10|1|1949|197|2563|Thursday|1949Q1|N|N|N|2432949|2432979|2432592|2432866|N|N|N|N|N| +2432959|AAAAAAAAPLPBFCAA|1949-02-11|589|2563|197|1949|5|2|11|1|1949|197|2563|Friday|1949Q1|N|Y|N|2432949|2432979|2432593|2432867|N|N|N|N|N| +2432960|AAAAAAAAAMPBFCAA|1949-02-12|589|2563|197|1949|6|2|12|1|1949|197|2563|Saturday|1949Q1|N|Y|N|2432949|2432979|2432594|2432868|N|N|N|N|N| +2432961|AAAAAAAABMPBFCAA|1949-02-13|589|2563|197|1949|0|2|13|1|1949|197|2563|Sunday|1949Q1|N|N|N|2432949|2432979|2432595|2432869|N|N|N|N|N| +2432962|AAAAAAAACMPBFCAA|1949-02-14|589|2563|197|1949|1|2|14|1|1949|197|2563|Monday|1949Q1|N|N|N|2432949|2432979|2432596|2432870|N|N|N|N|N| +2432963|AAAAAAAADMPBFCAA|1949-02-15|589|2564|197|1949|2|2|15|1|1949|197|2564|Tuesday|1949Q1|N|N|N|2432949|2432979|2432597|2432871|N|N|N|N|N| +2432964|AAAAAAAAEMPBFCAA|1949-02-16|589|2564|197|1949|3|2|16|1|1949|197|2564|Wednesday|1949Q1|N|N|N|2432949|2432979|2432598|2432872|N|N|N|N|N| +2432965|AAAAAAAAFMPBFCAA|1949-02-17|589|2564|197|1949|4|2|17|1|1949|197|2564|Thursday|1949Q1|N|N|N|2432949|2432979|2432599|2432873|N|N|N|N|N| +2432966|AAAAAAAAGMPBFCAA|1949-02-18|589|2564|197|1949|5|2|18|1|1949|197|2564|Friday|1949Q1|N|Y|N|2432949|2432979|2432600|2432874|N|N|N|N|N| +2432967|AAAAAAAAHMPBFCAA|1949-02-19|589|2564|197|1949|6|2|19|1|1949|197|2564|Saturday|1949Q1|N|Y|N|2432949|2432979|2432601|2432875|N|N|N|N|N| +2432968|AAAAAAAAIMPBFCAA|1949-02-20|589|2564|197|1949|0|2|20|1|1949|197|2564|Sunday|1949Q1|N|N|N|2432949|2432979|2432602|2432876|N|N|N|N|N| +2432969|AAAAAAAAJMPBFCAA|1949-02-21|589|2564|197|1949|1|2|21|1|1949|197|2564|Monday|1949Q1|N|N|N|2432949|2432979|2432603|2432877|N|N|N|N|N| +2432970|AAAAAAAAKMPBFCAA|1949-02-22|589|2565|197|1949|2|2|22|1|1949|197|2565|Tuesday|1949Q1|N|N|N|2432949|2432979|2432604|2432878|N|N|N|N|N| +2432971|AAAAAAAALMPBFCAA|1949-02-23|589|2565|197|1949|3|2|23|1|1949|197|2565|Wednesday|1949Q1|N|N|N|2432949|2432979|2432605|2432879|N|N|N|N|N| +2432972|AAAAAAAAMMPBFCAA|1949-02-24|589|2565|197|1949|4|2|24|1|1949|197|2565|Thursday|1949Q1|N|N|N|2432949|2432979|2432606|2432880|N|N|N|N|N| +2432973|AAAAAAAANMPBFCAA|1949-02-25|589|2565|197|1949|5|2|25|1|1949|197|2565|Friday|1949Q1|N|Y|N|2432949|2432979|2432607|2432881|N|N|N|N|N| +2432974|AAAAAAAAOMPBFCAA|1949-02-26|589|2565|197|1949|6|2|26|1|1949|197|2565|Saturday|1949Q1|N|Y|N|2432949|2432979|2432608|2432882|N|N|N|N|N| +2432975|AAAAAAAAPMPBFCAA|1949-02-27|589|2565|197|1949|0|2|27|1|1949|197|2565|Sunday|1949Q1|N|N|N|2432949|2432979|2432609|2432883|N|N|N|N|N| +2432976|AAAAAAAAANPBFCAA|1949-02-28|589|2565|197|1949|1|2|28|1|1949|197|2565|Monday|1949Q1|N|N|N|2432949|2432979|2432610|2432884|N|N|N|N|N| +2432977|AAAAAAAABNPBFCAA|1949-03-01|590|2566|198|1949|2|3|1|1|1949|198|2566|Tuesday|1949Q1|N|N|N|2432977|2433035|2432612|2432885|N|N|N|N|N| +2432978|AAAAAAAACNPBFCAA|1949-03-02|590|2566|198|1949|3|3|2|1|1949|198|2566|Wednesday|1949Q1|N|N|N|2432977|2433035|2432613|2432886|N|N|N|N|N| +2432979|AAAAAAAADNPBFCAA|1949-03-03|590|2566|198|1949|4|3|3|1|1949|198|2566|Thursday|1949Q1|N|N|N|2432977|2433035|2432614|2432887|N|N|N|N|N| +2432980|AAAAAAAAENPBFCAA|1949-03-04|590|2566|198|1949|5|3|4|1|1949|198|2566|Friday|1949Q1|N|Y|N|2432977|2433035|2432615|2432888|N|N|N|N|N| +2432981|AAAAAAAAFNPBFCAA|1949-03-05|590|2566|198|1949|6|3|5|1|1949|198|2566|Saturday|1949Q1|N|Y|N|2432977|2433035|2432616|2432889|N|N|N|N|N| +2432982|AAAAAAAAGNPBFCAA|1949-03-06|590|2566|198|1949|0|3|6|1|1949|198|2566|Sunday|1949Q1|N|N|N|2432977|2433035|2432617|2432890|N|N|N|N|N| +2432983|AAAAAAAAHNPBFCAA|1949-03-07|590|2566|198|1949|1|3|7|1|1949|198|2566|Monday|1949Q1|N|N|N|2432977|2433035|2432618|2432891|N|N|N|N|N| +2432984|AAAAAAAAINPBFCAA|1949-03-08|590|2567|198|1949|2|3|8|1|1949|198|2567|Tuesday|1949Q1|N|N|N|2432977|2433035|2432619|2432892|N|N|N|N|N| +2432985|AAAAAAAAJNPBFCAA|1949-03-09|590|2567|198|1949|3|3|9|1|1949|198|2567|Wednesday|1949Q1|N|N|N|2432977|2433035|2432620|2432893|N|N|N|N|N| +2432986|AAAAAAAAKNPBFCAA|1949-03-10|590|2567|198|1949|4|3|10|1|1949|198|2567|Thursday|1949Q1|N|N|N|2432977|2433035|2432621|2432894|N|N|N|N|N| +2432987|AAAAAAAALNPBFCAA|1949-03-11|590|2567|198|1949|5|3|11|1|1949|198|2567|Friday|1949Q1|N|Y|N|2432977|2433035|2432622|2432895|N|N|N|N|N| +2432988|AAAAAAAAMNPBFCAA|1949-03-12|590|2567|198|1949|6|3|12|1|1949|198|2567|Saturday|1949Q1|N|Y|N|2432977|2433035|2432623|2432896|N|N|N|N|N| +2432989|AAAAAAAANNPBFCAA|1949-03-13|590|2567|198|1949|0|3|13|1|1949|198|2567|Sunday|1949Q1|N|N|N|2432977|2433035|2432624|2432897|N|N|N|N|N| +2432990|AAAAAAAAONPBFCAA|1949-03-14|590|2567|198|1949|1|3|14|1|1949|198|2567|Monday|1949Q1|N|N|N|2432977|2433035|2432625|2432898|N|N|N|N|N| +2432991|AAAAAAAAPNPBFCAA|1949-03-15|590|2568|198|1949|2|3|15|1|1949|198|2568|Tuesday|1949Q1|N|N|N|2432977|2433035|2432626|2432899|N|N|N|N|N| +2432992|AAAAAAAAAOPBFCAA|1949-03-16|590|2568|198|1949|3|3|16|1|1949|198|2568|Wednesday|1949Q1|N|N|N|2432977|2433035|2432627|2432900|N|N|N|N|N| +2432993|AAAAAAAABOPBFCAA|1949-03-17|590|2568|198|1949|4|3|17|1|1949|198|2568|Thursday|1949Q1|N|N|N|2432977|2433035|2432628|2432901|N|N|N|N|N| +2432994|AAAAAAAACOPBFCAA|1949-03-18|590|2568|198|1949|5|3|18|1|1949|198|2568|Friday|1949Q1|N|Y|N|2432977|2433035|2432629|2432902|N|N|N|N|N| +2432995|AAAAAAAADOPBFCAA|1949-03-19|590|2568|198|1949|6|3|19|1|1949|198|2568|Saturday|1949Q1|N|Y|N|2432977|2433035|2432630|2432903|N|N|N|N|N| +2432996|AAAAAAAAEOPBFCAA|1949-03-20|590|2568|198|1949|0|3|20|1|1949|198|2568|Sunday|1949Q1|N|N|N|2432977|2433035|2432631|2432904|N|N|N|N|N| +2432997|AAAAAAAAFOPBFCAA|1949-03-21|590|2568|198|1949|1|3|21|1|1949|198|2568|Monday|1949Q1|N|N|N|2432977|2433035|2432632|2432905|N|N|N|N|N| +2432998|AAAAAAAAGOPBFCAA|1949-03-22|590|2569|198|1949|2|3|22|1|1949|198|2569|Tuesday|1949Q1|N|N|N|2432977|2433035|2432633|2432906|N|N|N|N|N| +2432999|AAAAAAAAHOPBFCAA|1949-03-23|590|2569|198|1949|3|3|23|1|1949|198|2569|Wednesday|1949Q1|N|N|N|2432977|2433035|2432634|2432907|N|N|N|N|N| +2433000|AAAAAAAAIOPBFCAA|1949-03-24|590|2569|198|1949|4|3|24|1|1949|198|2569|Thursday|1949Q1|N|N|N|2432977|2433035|2432635|2432908|N|N|N|N|N| +2433001|AAAAAAAAJOPBFCAA|1949-03-25|590|2569|198|1949|5|3|25|1|1949|198|2569|Friday|1949Q1|N|Y|N|2432977|2433035|2432636|2432909|N|N|N|N|N| +2433002|AAAAAAAAKOPBFCAA|1949-03-26|590|2569|198|1949|6|3|26|1|1949|198|2569|Saturday|1949Q1|N|Y|N|2432977|2433035|2432637|2432910|N|N|N|N|N| +2433003|AAAAAAAALOPBFCAA|1949-03-27|590|2569|198|1949|0|3|27|1|1949|198|2569|Sunday|1949Q1|N|N|N|2432977|2433035|2432638|2432911|N|N|N|N|N| +2433004|AAAAAAAAMOPBFCAA|1949-03-28|590|2569|198|1949|1|3|28|1|1949|198|2569|Monday|1949Q1|N|N|N|2432977|2433035|2432639|2432912|N|N|N|N|N| +2433005|AAAAAAAANOPBFCAA|1949-03-29|590|2570|198|1949|2|3|29|1|1949|198|2570|Tuesday|1949Q1|N|N|N|2432977|2433035|2432640|2432913|N|N|N|N|N| +2433006|AAAAAAAAOOPBFCAA|1949-03-30|590|2570|198|1949|3|3|30|1|1949|198|2570|Wednesday|1949Q1|N|N|N|2432977|2433035|2432641|2432914|N|N|N|N|N| +2433007|AAAAAAAAPOPBFCAA|1949-03-31|590|2570|198|1949|4|3|31|1|1949|198|2570|Thursday|1949Q1|N|N|N|2432977|2433035|2432642|2432915|N|N|N|N|N| +2433008|AAAAAAAAAPPBFCAA|1949-04-01|591|2570|198|1949|5|4|1|1|1949|198|2570|Friday|1949Q1|N|Y|N|2433008|2433097|2432643|2432918|N|N|N|N|N| +2433009|AAAAAAAABPPBFCAA|1949-04-02|591|2570|198|1949|6|4|2|2|1949|198|2570|Saturday|1949Q2|N|Y|N|2433008|2433097|2432644|2432919|N|N|N|N|N| +2433010|AAAAAAAACPPBFCAA|1949-04-03|591|2570|198|1949|0|4|3|2|1949|198|2570|Sunday|1949Q2|N|N|N|2433008|2433097|2432645|2432920|N|N|N|N|N| +2433011|AAAAAAAADPPBFCAA|1949-04-04|591|2570|198|1949|1|4|4|2|1949|198|2570|Monday|1949Q2|N|N|N|2433008|2433097|2432646|2432921|N|N|N|N|N| +2433012|AAAAAAAAEPPBFCAA|1949-04-05|591|2571|198|1949|2|4|5|2|1949|198|2571|Tuesday|1949Q2|N|N|N|2433008|2433097|2432647|2432922|N|N|N|N|N| +2433013|AAAAAAAAFPPBFCAA|1949-04-06|591|2571|198|1949|3|4|6|2|1949|198|2571|Wednesday|1949Q2|N|N|N|2433008|2433097|2432648|2432923|N|N|N|N|N| +2433014|AAAAAAAAGPPBFCAA|1949-04-07|591|2571|198|1949|4|4|7|2|1949|198|2571|Thursday|1949Q2|N|N|N|2433008|2433097|2432649|2432924|N|N|N|N|N| +2433015|AAAAAAAAHPPBFCAA|1949-04-08|591|2571|198|1949|5|4|8|2|1949|198|2571|Friday|1949Q2|N|Y|N|2433008|2433097|2432650|2432925|N|N|N|N|N| +2433016|AAAAAAAAIPPBFCAA|1949-04-09|591|2571|198|1949|6|4|9|2|1949|198|2571|Saturday|1949Q2|N|Y|N|2433008|2433097|2432651|2432926|N|N|N|N|N| +2433017|AAAAAAAAJPPBFCAA|1949-04-10|591|2571|198|1949|0|4|10|2|1949|198|2571|Sunday|1949Q2|N|N|N|2433008|2433097|2432652|2432927|N|N|N|N|N| +2433018|AAAAAAAAKPPBFCAA|1949-04-11|591|2571|198|1949|1|4|11|2|1949|198|2571|Monday|1949Q2|N|N|N|2433008|2433097|2432653|2432928|N|N|N|N|N| +2433019|AAAAAAAALPPBFCAA|1949-04-12|591|2572|198|1949|2|4|12|2|1949|198|2572|Tuesday|1949Q2|N|N|N|2433008|2433097|2432654|2432929|N|N|N|N|N| +2433020|AAAAAAAAMPPBFCAA|1949-04-13|591|2572|198|1949|3|4|13|2|1949|198|2572|Wednesday|1949Q2|N|N|N|2433008|2433097|2432655|2432930|N|N|N|N|N| +2433021|AAAAAAAANPPBFCAA|1949-04-14|591|2572|198|1949|4|4|14|2|1949|198|2572|Thursday|1949Q2|N|N|N|2433008|2433097|2432656|2432931|N|N|N|N|N| +2433022|AAAAAAAAOPPBFCAA|1949-04-15|591|2572|198|1949|5|4|15|2|1949|198|2572|Friday|1949Q2|N|Y|N|2433008|2433097|2432657|2432932|N|N|N|N|N| +2433023|AAAAAAAAPPPBFCAA|1949-04-16|591|2572|198|1949|6|4|16|2|1949|198|2572|Saturday|1949Q2|N|Y|N|2433008|2433097|2432658|2432933|N|N|N|N|N| +2433024|AAAAAAAAAAACFCAA|1949-04-17|591|2572|198|1949|0|4|17|2|1949|198|2572|Sunday|1949Q2|N|N|N|2433008|2433097|2432659|2432934|N|N|N|N|N| +2433025|AAAAAAAABAACFCAA|1949-04-18|591|2572|198|1949|1|4|18|2|1949|198|2572|Monday|1949Q2|N|N|N|2433008|2433097|2432660|2432935|N|N|N|N|N| +2433026|AAAAAAAACAACFCAA|1949-04-19|591|2573|198|1949|2|4|19|2|1949|198|2573|Tuesday|1949Q2|N|N|N|2433008|2433097|2432661|2432936|N|N|N|N|N| +2433027|AAAAAAAADAACFCAA|1949-04-20|591|2573|198|1949|3|4|20|2|1949|198|2573|Wednesday|1949Q2|N|N|N|2433008|2433097|2432662|2432937|N|N|N|N|N| +2433028|AAAAAAAAEAACFCAA|1949-04-21|591|2573|198|1949|4|4|21|2|1949|198|2573|Thursday|1949Q2|N|N|N|2433008|2433097|2432663|2432938|N|N|N|N|N| +2433029|AAAAAAAAFAACFCAA|1949-04-22|591|2573|198|1949|5|4|22|2|1949|198|2573|Friday|1949Q2|N|Y|N|2433008|2433097|2432664|2432939|N|N|N|N|N| +2433030|AAAAAAAAGAACFCAA|1949-04-23|591|2573|198|1949|6|4|23|2|1949|198|2573|Saturday|1949Q2|N|Y|N|2433008|2433097|2432665|2432940|N|N|N|N|N| +2433031|AAAAAAAAHAACFCAA|1949-04-24|591|2573|198|1949|0|4|24|2|1949|198|2573|Sunday|1949Q2|N|N|N|2433008|2433097|2432666|2432941|N|N|N|N|N| +2433032|AAAAAAAAIAACFCAA|1949-04-25|591|2573|198|1949|1|4|25|2|1949|198|2573|Monday|1949Q2|N|N|N|2433008|2433097|2432667|2432942|N|N|N|N|N| +2433033|AAAAAAAAJAACFCAA|1949-04-26|591|2574|198|1949|2|4|26|2|1949|198|2574|Tuesday|1949Q2|N|N|N|2433008|2433097|2432668|2432943|N|N|N|N|N| +2433034|AAAAAAAAKAACFCAA|1949-04-27|591|2574|198|1949|3|4|27|2|1949|198|2574|Wednesday|1949Q2|N|N|N|2433008|2433097|2432669|2432944|N|N|N|N|N| +2433035|AAAAAAAALAACFCAA|1949-04-28|591|2574|198|1949|4|4|28|2|1949|198|2574|Thursday|1949Q2|N|N|N|2433008|2433097|2432670|2432945|N|N|N|N|N| +2433036|AAAAAAAAMAACFCAA|1949-04-29|591|2574|198|1949|5|4|29|2|1949|198|2574|Friday|1949Q2|N|Y|N|2433008|2433097|2432671|2432946|N|N|N|N|N| +2433037|AAAAAAAANAACFCAA|1949-04-30|591|2574|198|1949|6|4|30|2|1949|198|2574|Saturday|1949Q2|N|Y|N|2433008|2433097|2432672|2432947|N|N|N|N|N| +2433038|AAAAAAAAOAACFCAA|1949-05-01|592|2574|198|1949|0|5|1|2|1949|198|2574|Sunday|1949Q2|N|N|N|2433038|2433157|2432673|2432948|N|N|N|N|N| +2433039|AAAAAAAAPAACFCAA|1949-05-02|592|2574|198|1949|1|5|2|2|1949|198|2574|Monday|1949Q2|N|N|N|2433038|2433157|2432674|2432949|N|N|N|N|N| +2433040|AAAAAAAAABACFCAA|1949-05-03|592|2575|198|1949|2|5|3|2|1949|198|2575|Tuesday|1949Q2|N|N|N|2433038|2433157|2432675|2432950|N|N|N|N|N| +2433041|AAAAAAAABBACFCAA|1949-05-04|592|2575|198|1949|3|5|4|2|1949|198|2575|Wednesday|1949Q2|N|N|N|2433038|2433157|2432676|2432951|N|N|N|N|N| +2433042|AAAAAAAACBACFCAA|1949-05-05|592|2575|198|1949|4|5|5|2|1949|198|2575|Thursday|1949Q2|N|N|N|2433038|2433157|2432677|2432952|N|N|N|N|N| +2433043|AAAAAAAADBACFCAA|1949-05-06|592|2575|198|1949|5|5|6|2|1949|198|2575|Friday|1949Q2|N|Y|N|2433038|2433157|2432678|2432953|N|N|N|N|N| +2433044|AAAAAAAAEBACFCAA|1949-05-07|592|2575|198|1949|6|5|7|2|1949|198|2575|Saturday|1949Q2|N|Y|N|2433038|2433157|2432679|2432954|N|N|N|N|N| +2433045|AAAAAAAAFBACFCAA|1949-05-08|592|2575|198|1949|0|5|8|2|1949|198|2575|Sunday|1949Q2|N|N|N|2433038|2433157|2432680|2432955|N|N|N|N|N| +2433046|AAAAAAAAGBACFCAA|1949-05-09|592|2575|198|1949|1|5|9|2|1949|198|2575|Monday|1949Q2|N|N|N|2433038|2433157|2432681|2432956|N|N|N|N|N| +2433047|AAAAAAAAHBACFCAA|1949-05-10|592|2576|198|1949|2|5|10|2|1949|198|2576|Tuesday|1949Q2|N|N|N|2433038|2433157|2432682|2432957|N|N|N|N|N| +2433048|AAAAAAAAIBACFCAA|1949-05-11|592|2576|198|1949|3|5|11|2|1949|198|2576|Wednesday|1949Q2|N|N|N|2433038|2433157|2432683|2432958|N|N|N|N|N| +2433049|AAAAAAAAJBACFCAA|1949-05-12|592|2576|198|1949|4|5|12|2|1949|198|2576|Thursday|1949Q2|N|N|N|2433038|2433157|2432684|2432959|N|N|N|N|N| +2433050|AAAAAAAAKBACFCAA|1949-05-13|592|2576|198|1949|5|5|13|2|1949|198|2576|Friday|1949Q2|N|Y|N|2433038|2433157|2432685|2432960|N|N|N|N|N| +2433051|AAAAAAAALBACFCAA|1949-05-14|592|2576|198|1949|6|5|14|2|1949|198|2576|Saturday|1949Q2|N|Y|N|2433038|2433157|2432686|2432961|N|N|N|N|N| +2433052|AAAAAAAAMBACFCAA|1949-05-15|592|2576|198|1949|0|5|15|2|1949|198|2576|Sunday|1949Q2|N|N|N|2433038|2433157|2432687|2432962|N|N|N|N|N| +2433053|AAAAAAAANBACFCAA|1949-05-16|592|2576|198|1949|1|5|16|2|1949|198|2576|Monday|1949Q2|N|N|N|2433038|2433157|2432688|2432963|N|N|N|N|N| +2433054|AAAAAAAAOBACFCAA|1949-05-17|592|2577|198|1949|2|5|17|2|1949|198|2577|Tuesday|1949Q2|N|N|N|2433038|2433157|2432689|2432964|N|N|N|N|N| +2433055|AAAAAAAAPBACFCAA|1949-05-18|592|2577|198|1949|3|5|18|2|1949|198|2577|Wednesday|1949Q2|N|N|N|2433038|2433157|2432690|2432965|N|N|N|N|N| +2433056|AAAAAAAAACACFCAA|1949-05-19|592|2577|198|1949|4|5|19|2|1949|198|2577|Thursday|1949Q2|N|N|N|2433038|2433157|2432691|2432966|N|N|N|N|N| +2433057|AAAAAAAABCACFCAA|1949-05-20|592|2577|198|1949|5|5|20|2|1949|198|2577|Friday|1949Q2|N|Y|N|2433038|2433157|2432692|2432967|N|N|N|N|N| +2433058|AAAAAAAACCACFCAA|1949-05-21|592|2577|198|1949|6|5|21|2|1949|198|2577|Saturday|1949Q2|N|Y|N|2433038|2433157|2432693|2432968|N|N|N|N|N| +2433059|AAAAAAAADCACFCAA|1949-05-22|592|2577|198|1949|0|5|22|2|1949|198|2577|Sunday|1949Q2|N|N|N|2433038|2433157|2432694|2432969|N|N|N|N|N| +2433060|AAAAAAAAECACFCAA|1949-05-23|592|2577|198|1949|1|5|23|2|1949|198|2577|Monday|1949Q2|N|N|N|2433038|2433157|2432695|2432970|N|N|N|N|N| +2433061|AAAAAAAAFCACFCAA|1949-05-24|592|2578|198|1949|2|5|24|2|1949|198|2578|Tuesday|1949Q2|N|N|N|2433038|2433157|2432696|2432971|N|N|N|N|N| +2433062|AAAAAAAAGCACFCAA|1949-05-25|592|2578|198|1949|3|5|25|2|1949|198|2578|Wednesday|1949Q2|N|N|N|2433038|2433157|2432697|2432972|N|N|N|N|N| +2433063|AAAAAAAAHCACFCAA|1949-05-26|592|2578|198|1949|4|5|26|2|1949|198|2578|Thursday|1949Q2|N|N|N|2433038|2433157|2432698|2432973|N|N|N|N|N| +2433064|AAAAAAAAICACFCAA|1949-05-27|592|2578|198|1949|5|5|27|2|1949|198|2578|Friday|1949Q2|N|Y|N|2433038|2433157|2432699|2432974|N|N|N|N|N| +2433065|AAAAAAAAJCACFCAA|1949-05-28|592|2578|198|1949|6|5|28|2|1949|198|2578|Saturday|1949Q2|N|Y|N|2433038|2433157|2432700|2432975|N|N|N|N|N| +2433066|AAAAAAAAKCACFCAA|1949-05-29|592|2578|198|1949|0|5|29|2|1949|198|2578|Sunday|1949Q2|N|N|N|2433038|2433157|2432701|2432976|N|N|N|N|N| +2433067|AAAAAAAALCACFCAA|1949-05-30|592|2578|198|1949|1|5|30|2|1949|198|2578|Monday|1949Q2|N|N|N|2433038|2433157|2432702|2432977|N|N|N|N|N| +2433068|AAAAAAAAMCACFCAA|1949-05-31|592|2579|198|1949|2|5|31|2|1949|198|2579|Tuesday|1949Q2|N|N|N|2433038|2433157|2432703|2432978|N|N|N|N|N| +2433069|AAAAAAAANCACFCAA|1949-06-01|593|2579|199|1949|3|6|1|2|1949|199|2579|Wednesday|1949Q2|N|N|N|2433069|2433219|2432704|2432979|N|N|N|N|N| +2433070|AAAAAAAAOCACFCAA|1949-06-02|593|2579|199|1949|4|6|2|2|1949|199|2579|Thursday|1949Q2|N|N|N|2433069|2433219|2432705|2432980|N|N|N|N|N| +2433071|AAAAAAAAPCACFCAA|1949-06-03|593|2579|199|1949|5|6|3|2|1949|199|2579|Friday|1949Q2|N|Y|N|2433069|2433219|2432706|2432981|N|N|N|N|N| +2433072|AAAAAAAAADACFCAA|1949-06-04|593|2579|199|1949|6|6|4|2|1949|199|2579|Saturday|1949Q2|N|Y|N|2433069|2433219|2432707|2432982|N|N|N|N|N| +2433073|AAAAAAAABDACFCAA|1949-06-05|593|2579|199|1949|0|6|5|2|1949|199|2579|Sunday|1949Q2|N|N|N|2433069|2433219|2432708|2432983|N|N|N|N|N| +2433074|AAAAAAAACDACFCAA|1949-06-06|593|2579|199|1949|1|6|6|2|1949|199|2579|Monday|1949Q2|N|N|N|2433069|2433219|2432709|2432984|N|N|N|N|N| +2433075|AAAAAAAADDACFCAA|1949-06-07|593|2580|199|1949|2|6|7|2|1949|199|2580|Tuesday|1949Q2|N|N|N|2433069|2433219|2432710|2432985|N|N|N|N|N| +2433076|AAAAAAAAEDACFCAA|1949-06-08|593|2580|199|1949|3|6|8|2|1949|199|2580|Wednesday|1949Q2|N|N|N|2433069|2433219|2432711|2432986|N|N|N|N|N| +2433077|AAAAAAAAFDACFCAA|1949-06-09|593|2580|199|1949|4|6|9|2|1949|199|2580|Thursday|1949Q2|N|N|N|2433069|2433219|2432712|2432987|N|N|N|N|N| +2433078|AAAAAAAAGDACFCAA|1949-06-10|593|2580|199|1949|5|6|10|2|1949|199|2580|Friday|1949Q2|N|Y|N|2433069|2433219|2432713|2432988|N|N|N|N|N| +2433079|AAAAAAAAHDACFCAA|1949-06-11|593|2580|199|1949|6|6|11|2|1949|199|2580|Saturday|1949Q2|N|Y|N|2433069|2433219|2432714|2432989|N|N|N|N|N| +2433080|AAAAAAAAIDACFCAA|1949-06-12|593|2580|199|1949|0|6|12|2|1949|199|2580|Sunday|1949Q2|N|N|N|2433069|2433219|2432715|2432990|N|N|N|N|N| +2433081|AAAAAAAAJDACFCAA|1949-06-13|593|2580|199|1949|1|6|13|2|1949|199|2580|Monday|1949Q2|N|N|N|2433069|2433219|2432716|2432991|N|N|N|N|N| +2433082|AAAAAAAAKDACFCAA|1949-06-14|593|2581|199|1949|2|6|14|2|1949|199|2581|Tuesday|1949Q2|N|N|N|2433069|2433219|2432717|2432992|N|N|N|N|N| +2433083|AAAAAAAALDACFCAA|1949-06-15|593|2581|199|1949|3|6|15|2|1949|199|2581|Wednesday|1949Q2|N|N|N|2433069|2433219|2432718|2432993|N|N|N|N|N| +2433084|AAAAAAAAMDACFCAA|1949-06-16|593|2581|199|1949|4|6|16|2|1949|199|2581|Thursday|1949Q2|N|N|N|2433069|2433219|2432719|2432994|N|N|N|N|N| +2433085|AAAAAAAANDACFCAA|1949-06-17|593|2581|199|1949|5|6|17|2|1949|199|2581|Friday|1949Q2|N|Y|N|2433069|2433219|2432720|2432995|N|N|N|N|N| +2433086|AAAAAAAAODACFCAA|1949-06-18|593|2581|199|1949|6|6|18|2|1949|199|2581|Saturday|1949Q2|N|Y|N|2433069|2433219|2432721|2432996|N|N|N|N|N| +2433087|AAAAAAAAPDACFCAA|1949-06-19|593|2581|199|1949|0|6|19|2|1949|199|2581|Sunday|1949Q2|N|N|N|2433069|2433219|2432722|2432997|N|N|N|N|N| +2433088|AAAAAAAAAEACFCAA|1949-06-20|593|2581|199|1949|1|6|20|2|1949|199|2581|Monday|1949Q2|N|N|N|2433069|2433219|2432723|2432998|N|N|N|N|N| +2433089|AAAAAAAABEACFCAA|1949-06-21|593|2582|199|1949|2|6|21|2|1949|199|2582|Tuesday|1949Q2|N|N|N|2433069|2433219|2432724|2432999|N|N|N|N|N| +2433090|AAAAAAAACEACFCAA|1949-06-22|593|2582|199|1949|3|6|22|2|1949|199|2582|Wednesday|1949Q2|N|N|N|2433069|2433219|2432725|2433000|N|N|N|N|N| +2433091|AAAAAAAADEACFCAA|1949-06-23|593|2582|199|1949|4|6|23|2|1949|199|2582|Thursday|1949Q2|N|N|N|2433069|2433219|2432726|2433001|N|N|N|N|N| +2433092|AAAAAAAAEEACFCAA|1949-06-24|593|2582|199|1949|5|6|24|2|1949|199|2582|Friday|1949Q2|N|Y|N|2433069|2433219|2432727|2433002|N|N|N|N|N| +2433093|AAAAAAAAFEACFCAA|1949-06-25|593|2582|199|1949|6|6|25|2|1949|199|2582|Saturday|1949Q2|N|Y|N|2433069|2433219|2432728|2433003|N|N|N|N|N| +2433094|AAAAAAAAGEACFCAA|1949-06-26|593|2582|199|1949|0|6|26|2|1949|199|2582|Sunday|1949Q2|N|N|N|2433069|2433219|2432729|2433004|N|N|N|N|N| +2433095|AAAAAAAAHEACFCAA|1949-06-27|593|2582|199|1949|1|6|27|2|1949|199|2582|Monday|1949Q2|N|N|N|2433069|2433219|2432730|2433005|N|N|N|N|N| +2433096|AAAAAAAAIEACFCAA|1949-06-28|593|2583|199|1949|2|6|28|2|1949|199|2583|Tuesday|1949Q2|N|N|N|2433069|2433219|2432731|2433006|N|N|N|N|N| +2433097|AAAAAAAAJEACFCAA|1949-06-29|593|2583|199|1949|3|6|29|2|1949|199|2583|Wednesday|1949Q2|N|N|N|2433069|2433219|2432732|2433007|N|N|N|N|N| +2433098|AAAAAAAAKEACFCAA|1949-06-30|593|2583|199|1949|4|6|30|2|1949|199|2583|Thursday|1949Q2|N|N|N|2433069|2433219|2432733|2433008|N|N|N|N|N| +2433099|AAAAAAAALEACFCAA|1949-07-01|594|2583|199|1949|5|7|1|2|1949|199|2583|Friday|1949Q2|N|Y|N|2433099|2433279|2432734|2433008|N|N|N|N|N| +2433100|AAAAAAAAMEACFCAA|1949-07-02|594|2583|199|1949|6|7|2|3|1949|199|2583|Saturday|1949Q3|N|Y|N|2433099|2433279|2432735|2433009|N|N|N|N|N| +2433101|AAAAAAAANEACFCAA|1949-07-03|594|2583|199|1949|0|7|3|3|1949|199|2583|Sunday|1949Q3|N|N|N|2433099|2433279|2432736|2433010|N|N|N|N|N| +2433102|AAAAAAAAOEACFCAA|1949-07-04|594|2583|199|1949|1|7|4|3|1949|199|2583|Monday|1949Q3|N|N|N|2433099|2433279|2432737|2433011|N|N|N|N|N| +2433103|AAAAAAAAPEACFCAA|1949-07-05|594|2584|199|1949|2|7|5|3|1949|199|2584|Tuesday|1949Q3|Y|N|N|2433099|2433279|2432738|2433012|N|N|N|N|N| +2433104|AAAAAAAAAFACFCAA|1949-07-06|594|2584|199|1949|3|7|6|3|1949|199|2584|Wednesday|1949Q3|N|N|Y|2433099|2433279|2432739|2433013|N|N|N|N|N| +2433105|AAAAAAAABFACFCAA|1949-07-07|594|2584|199|1949|4|7|7|3|1949|199|2584|Thursday|1949Q3|N|N|N|2433099|2433279|2432740|2433014|N|N|N|N|N| +2433106|AAAAAAAACFACFCAA|1949-07-08|594|2584|199|1949|5|7|8|3|1949|199|2584|Friday|1949Q3|N|Y|N|2433099|2433279|2432741|2433015|N|N|N|N|N| +2433107|AAAAAAAADFACFCAA|1949-07-09|594|2584|199|1949|6|7|9|3|1949|199|2584|Saturday|1949Q3|N|Y|N|2433099|2433279|2432742|2433016|N|N|N|N|N| +2433108|AAAAAAAAEFACFCAA|1949-07-10|594|2584|199|1949|0|7|10|3|1949|199|2584|Sunday|1949Q3|N|N|N|2433099|2433279|2432743|2433017|N|N|N|N|N| +2433109|AAAAAAAAFFACFCAA|1949-07-11|594|2584|199|1949|1|7|11|3|1949|199|2584|Monday|1949Q3|N|N|N|2433099|2433279|2432744|2433018|N|N|N|N|N| +2433110|AAAAAAAAGFACFCAA|1949-07-12|594|2585|199|1949|2|7|12|3|1949|199|2585|Tuesday|1949Q3|N|N|N|2433099|2433279|2432745|2433019|N|N|N|N|N| +2433111|AAAAAAAAHFACFCAA|1949-07-13|594|2585|199|1949|3|7|13|3|1949|199|2585|Wednesday|1949Q3|N|N|N|2433099|2433279|2432746|2433020|N|N|N|N|N| +2433112|AAAAAAAAIFACFCAA|1949-07-14|594|2585|199|1949|4|7|14|3|1949|199|2585|Thursday|1949Q3|N|N|N|2433099|2433279|2432747|2433021|N|N|N|N|N| +2433113|AAAAAAAAJFACFCAA|1949-07-15|594|2585|199|1949|5|7|15|3|1949|199|2585|Friday|1949Q3|N|Y|N|2433099|2433279|2432748|2433022|N|N|N|N|N| +2433114|AAAAAAAAKFACFCAA|1949-07-16|594|2585|199|1949|6|7|16|3|1949|199|2585|Saturday|1949Q3|N|Y|N|2433099|2433279|2432749|2433023|N|N|N|N|N| +2433115|AAAAAAAALFACFCAA|1949-07-17|594|2585|199|1949|0|7|17|3|1949|199|2585|Sunday|1949Q3|N|N|N|2433099|2433279|2432750|2433024|N|N|N|N|N| +2433116|AAAAAAAAMFACFCAA|1949-07-18|594|2585|199|1949|1|7|18|3|1949|199|2585|Monday|1949Q3|N|N|N|2433099|2433279|2432751|2433025|N|N|N|N|N| +2433117|AAAAAAAANFACFCAA|1949-07-19|594|2586|199|1949|2|7|19|3|1949|199|2586|Tuesday|1949Q3|N|N|N|2433099|2433279|2432752|2433026|N|N|N|N|N| +2433118|AAAAAAAAOFACFCAA|1949-07-20|594|2586|199|1949|3|7|20|3|1949|199|2586|Wednesday|1949Q3|N|N|N|2433099|2433279|2432753|2433027|N|N|N|N|N| +2433119|AAAAAAAAPFACFCAA|1949-07-21|594|2586|199|1949|4|7|21|3|1949|199|2586|Thursday|1949Q3|N|N|N|2433099|2433279|2432754|2433028|N|N|N|N|N| +2433120|AAAAAAAAAGACFCAA|1949-07-22|594|2586|199|1949|5|7|22|3|1949|199|2586|Friday|1949Q3|N|Y|N|2433099|2433279|2432755|2433029|N|N|N|N|N| +2433121|AAAAAAAABGACFCAA|1949-07-23|594|2586|199|1949|6|7|23|3|1949|199|2586|Saturday|1949Q3|N|Y|N|2433099|2433279|2432756|2433030|N|N|N|N|N| +2433122|AAAAAAAACGACFCAA|1949-07-24|594|2586|199|1949|0|7|24|3|1949|199|2586|Sunday|1949Q3|N|N|N|2433099|2433279|2432757|2433031|N|N|N|N|N| +2433123|AAAAAAAADGACFCAA|1949-07-25|594|2586|199|1949|1|7|25|3|1949|199|2586|Monday|1949Q3|N|N|N|2433099|2433279|2432758|2433032|N|N|N|N|N| +2433124|AAAAAAAAEGACFCAA|1949-07-26|594|2587|199|1949|2|7|26|3|1949|199|2587|Tuesday|1949Q3|N|N|N|2433099|2433279|2432759|2433033|N|N|N|N|N| +2433125|AAAAAAAAFGACFCAA|1949-07-27|594|2587|199|1949|3|7|27|3|1949|199|2587|Wednesday|1949Q3|N|N|N|2433099|2433279|2432760|2433034|N|N|N|N|N| +2433126|AAAAAAAAGGACFCAA|1949-07-28|594|2587|199|1949|4|7|28|3|1949|199|2587|Thursday|1949Q3|N|N|N|2433099|2433279|2432761|2433035|N|N|N|N|N| +2433127|AAAAAAAAHGACFCAA|1949-07-29|594|2587|199|1949|5|7|29|3|1949|199|2587|Friday|1949Q3|N|Y|N|2433099|2433279|2432762|2433036|N|N|N|N|N| +2433128|AAAAAAAAIGACFCAA|1949-07-30|594|2587|199|1949|6|7|30|3|1949|199|2587|Saturday|1949Q3|N|Y|N|2433099|2433279|2432763|2433037|N|N|N|N|N| +2433129|AAAAAAAAJGACFCAA|1949-07-31|594|2587|199|1949|0|7|31|3|1949|199|2587|Sunday|1949Q3|N|N|N|2433099|2433279|2432764|2433038|N|N|N|N|N| +2433130|AAAAAAAAKGACFCAA|1949-08-01|595|2587|199|1949|1|8|1|3|1949|199|2587|Monday|1949Q3|N|N|N|2433130|2433341|2432765|2433039|N|N|N|N|N| +2433131|AAAAAAAALGACFCAA|1949-08-02|595|2588|199|1949|2|8|2|3|1949|199|2588|Tuesday|1949Q3|N|N|N|2433130|2433341|2432766|2433040|N|N|N|N|N| +2433132|AAAAAAAAMGACFCAA|1949-08-03|595|2588|199|1949|3|8|3|3|1949|199|2588|Wednesday|1949Q3|N|N|N|2433130|2433341|2432767|2433041|N|N|N|N|N| +2433133|AAAAAAAANGACFCAA|1949-08-04|595|2588|199|1949|4|8|4|3|1949|199|2588|Thursday|1949Q3|N|N|N|2433130|2433341|2432768|2433042|N|N|N|N|N| +2433134|AAAAAAAAOGACFCAA|1949-08-05|595|2588|199|1949|5|8|5|3|1949|199|2588|Friday|1949Q3|N|Y|N|2433130|2433341|2432769|2433043|N|N|N|N|N| +2433135|AAAAAAAAPGACFCAA|1949-08-06|595|2588|199|1949|6|8|6|3|1949|199|2588|Saturday|1949Q3|N|Y|N|2433130|2433341|2432770|2433044|N|N|N|N|N| +2433136|AAAAAAAAAHACFCAA|1949-08-07|595|2588|199|1949|0|8|7|3|1949|199|2588|Sunday|1949Q3|N|N|N|2433130|2433341|2432771|2433045|N|N|N|N|N| +2433137|AAAAAAAABHACFCAA|1949-08-08|595|2588|199|1949|1|8|8|3|1949|199|2588|Monday|1949Q3|N|N|N|2433130|2433341|2432772|2433046|N|N|N|N|N| +2433138|AAAAAAAACHACFCAA|1949-08-09|595|2589|199|1949|2|8|9|3|1949|199|2589|Tuesday|1949Q3|N|N|N|2433130|2433341|2432773|2433047|N|N|N|N|N| +2433139|AAAAAAAADHACFCAA|1949-08-10|595|2589|199|1949|3|8|10|3|1949|199|2589|Wednesday|1949Q3|N|N|N|2433130|2433341|2432774|2433048|N|N|N|N|N| +2433140|AAAAAAAAEHACFCAA|1949-08-11|595|2589|199|1949|4|8|11|3|1949|199|2589|Thursday|1949Q3|N|N|N|2433130|2433341|2432775|2433049|N|N|N|N|N| +2433141|AAAAAAAAFHACFCAA|1949-08-12|595|2589|199|1949|5|8|12|3|1949|199|2589|Friday|1949Q3|N|Y|N|2433130|2433341|2432776|2433050|N|N|N|N|N| +2433142|AAAAAAAAGHACFCAA|1949-08-13|595|2589|199|1949|6|8|13|3|1949|199|2589|Saturday|1949Q3|N|Y|N|2433130|2433341|2432777|2433051|N|N|N|N|N| +2433143|AAAAAAAAHHACFCAA|1949-08-14|595|2589|199|1949|0|8|14|3|1949|199|2589|Sunday|1949Q3|N|N|N|2433130|2433341|2432778|2433052|N|N|N|N|N| +2433144|AAAAAAAAIHACFCAA|1949-08-15|595|2589|199|1949|1|8|15|3|1949|199|2589|Monday|1949Q3|N|N|N|2433130|2433341|2432779|2433053|N|N|N|N|N| +2433145|AAAAAAAAJHACFCAA|1949-08-16|595|2590|199|1949|2|8|16|3|1949|199|2590|Tuesday|1949Q3|N|N|N|2433130|2433341|2432780|2433054|N|N|N|N|N| +2433146|AAAAAAAAKHACFCAA|1949-08-17|595|2590|199|1949|3|8|17|3|1949|199|2590|Wednesday|1949Q3|N|N|N|2433130|2433341|2432781|2433055|N|N|N|N|N| +2433147|AAAAAAAALHACFCAA|1949-08-18|595|2590|199|1949|4|8|18|3|1949|199|2590|Thursday|1949Q3|N|N|N|2433130|2433341|2432782|2433056|N|N|N|N|N| +2433148|AAAAAAAAMHACFCAA|1949-08-19|595|2590|199|1949|5|8|19|3|1949|199|2590|Friday|1949Q3|N|Y|N|2433130|2433341|2432783|2433057|N|N|N|N|N| +2433149|AAAAAAAANHACFCAA|1949-08-20|595|2590|199|1949|6|8|20|3|1949|199|2590|Saturday|1949Q3|N|Y|N|2433130|2433341|2432784|2433058|N|N|N|N|N| +2433150|AAAAAAAAOHACFCAA|1949-08-21|595|2590|199|1949|0|8|21|3|1949|199|2590|Sunday|1949Q3|N|N|N|2433130|2433341|2432785|2433059|N|N|N|N|N| +2433151|AAAAAAAAPHACFCAA|1949-08-22|595|2590|199|1949|1|8|22|3|1949|199|2590|Monday|1949Q3|N|N|N|2433130|2433341|2432786|2433060|N|N|N|N|N| +2433152|AAAAAAAAAIACFCAA|1949-08-23|595|2591|199|1949|2|8|23|3|1949|199|2591|Tuesday|1949Q3|N|N|N|2433130|2433341|2432787|2433061|N|N|N|N|N| +2433153|AAAAAAAABIACFCAA|1949-08-24|595|2591|199|1949|3|8|24|3|1949|199|2591|Wednesday|1949Q3|N|N|N|2433130|2433341|2432788|2433062|N|N|N|N|N| +2433154|AAAAAAAACIACFCAA|1949-08-25|595|2591|199|1949|4|8|25|3|1949|199|2591|Thursday|1949Q3|N|N|N|2433130|2433341|2432789|2433063|N|N|N|N|N| +2433155|AAAAAAAADIACFCAA|1949-08-26|595|2591|199|1949|5|8|26|3|1949|199|2591|Friday|1949Q3|N|Y|N|2433130|2433341|2432790|2433064|N|N|N|N|N| +2433156|AAAAAAAAEIACFCAA|1949-08-27|595|2591|199|1949|6|8|27|3|1949|199|2591|Saturday|1949Q3|N|Y|N|2433130|2433341|2432791|2433065|N|N|N|N|N| +2433157|AAAAAAAAFIACFCAA|1949-08-28|595|2591|199|1949|0|8|28|3|1949|199|2591|Sunday|1949Q3|N|N|N|2433130|2433341|2432792|2433066|N|N|N|N|N| +2433158|AAAAAAAAGIACFCAA|1949-08-29|595|2591|199|1949|1|8|29|3|1949|199|2591|Monday|1949Q3|N|N|N|2433130|2433341|2432793|2433067|N|N|N|N|N| +2433159|AAAAAAAAHIACFCAA|1949-08-30|595|2592|199|1949|2|8|30|3|1949|199|2592|Tuesday|1949Q3|N|N|N|2433130|2433341|2432794|2433068|N|N|N|N|N| +2433160|AAAAAAAAIIACFCAA|1949-08-31|595|2592|199|1949|3|8|31|3|1949|199|2592|Wednesday|1949Q3|N|N|N|2433130|2433341|2432795|2433069|N|N|N|N|N| +2433161|AAAAAAAAJIACFCAA|1949-09-01|596|2592|200|1949|4|9|1|3|1949|200|2592|Thursday|1949Q3|N|N|N|2433161|2433403|2432796|2433070|N|N|N|N|N| +2433162|AAAAAAAAKIACFCAA|1949-09-02|596|2592|200|1949|5|9|2|3|1949|200|2592|Friday|1949Q3|N|Y|N|2433161|2433403|2432797|2433071|N|N|N|N|N| +2433163|AAAAAAAALIACFCAA|1949-09-03|596|2592|200|1949|6|9|3|3|1949|200|2592|Saturday|1949Q3|N|Y|N|2433161|2433403|2432798|2433072|N|N|N|N|N| +2433164|AAAAAAAAMIACFCAA|1949-09-04|596|2592|200|1949|0|9|4|3|1949|200|2592|Sunday|1949Q3|N|N|N|2433161|2433403|2432799|2433073|N|N|N|N|N| +2433165|AAAAAAAANIACFCAA|1949-09-05|596|2592|200|1949|1|9|5|3|1949|200|2592|Monday|1949Q3|N|N|N|2433161|2433403|2432800|2433074|N|N|N|N|N| +2433166|AAAAAAAAOIACFCAA|1949-09-06|596|2593|200|1949|2|9|6|3|1949|200|2593|Tuesday|1949Q3|N|N|N|2433161|2433403|2432801|2433075|N|N|N|N|N| +2433167|AAAAAAAAPIACFCAA|1949-09-07|596|2593|200|1949|3|9|7|3|1949|200|2593|Wednesday|1949Q3|N|N|N|2433161|2433403|2432802|2433076|N|N|N|N|N| +2433168|AAAAAAAAAJACFCAA|1949-09-08|596|2593|200|1949|4|9|8|3|1949|200|2593|Thursday|1949Q3|N|N|N|2433161|2433403|2432803|2433077|N|N|N|N|N| +2433169|AAAAAAAABJACFCAA|1949-09-09|596|2593|200|1949|5|9|9|3|1949|200|2593|Friday|1949Q3|N|Y|N|2433161|2433403|2432804|2433078|N|N|N|N|N| +2433170|AAAAAAAACJACFCAA|1949-09-10|596|2593|200|1949|6|9|10|3|1949|200|2593|Saturday|1949Q3|N|Y|N|2433161|2433403|2432805|2433079|N|N|N|N|N| +2433171|AAAAAAAADJACFCAA|1949-09-11|596|2593|200|1949|0|9|11|3|1949|200|2593|Sunday|1949Q3|N|N|N|2433161|2433403|2432806|2433080|N|N|N|N|N| +2433172|AAAAAAAAEJACFCAA|1949-09-12|596|2593|200|1949|1|9|12|3|1949|200|2593|Monday|1949Q3|N|N|N|2433161|2433403|2432807|2433081|N|N|N|N|N| +2433173|AAAAAAAAFJACFCAA|1949-09-13|596|2594|200|1949|2|9|13|3|1949|200|2594|Tuesday|1949Q3|N|N|N|2433161|2433403|2432808|2433082|N|N|N|N|N| +2433174|AAAAAAAAGJACFCAA|1949-09-14|596|2594|200|1949|3|9|14|3|1949|200|2594|Wednesday|1949Q3|N|N|N|2433161|2433403|2432809|2433083|N|N|N|N|N| +2433175|AAAAAAAAHJACFCAA|1949-09-15|596|2594|200|1949|4|9|15|3|1949|200|2594|Thursday|1949Q3|N|N|N|2433161|2433403|2432810|2433084|N|N|N|N|N| +2433176|AAAAAAAAIJACFCAA|1949-09-16|596|2594|200|1949|5|9|16|3|1949|200|2594|Friday|1949Q3|N|Y|N|2433161|2433403|2432811|2433085|N|N|N|N|N| +2433177|AAAAAAAAJJACFCAA|1949-09-17|596|2594|200|1949|6|9|17|3|1949|200|2594|Saturday|1949Q3|N|Y|N|2433161|2433403|2432812|2433086|N|N|N|N|N| +2433178|AAAAAAAAKJACFCAA|1949-09-18|596|2594|200|1949|0|9|18|3|1949|200|2594|Sunday|1949Q3|N|N|N|2433161|2433403|2432813|2433087|N|N|N|N|N| +2433179|AAAAAAAALJACFCAA|1949-09-19|596|2594|200|1949|1|9|19|3|1949|200|2594|Monday|1949Q3|N|N|N|2433161|2433403|2432814|2433088|N|N|N|N|N| +2433180|AAAAAAAAMJACFCAA|1949-09-20|596|2595|200|1949|2|9|20|3|1949|200|2595|Tuesday|1949Q3|N|N|N|2433161|2433403|2432815|2433089|N|N|N|N|N| +2433181|AAAAAAAANJACFCAA|1949-09-21|596|2595|200|1949|3|9|21|3|1949|200|2595|Wednesday|1949Q3|N|N|N|2433161|2433403|2432816|2433090|N|N|N|N|N| +2433182|AAAAAAAAOJACFCAA|1949-09-22|596|2595|200|1949|4|9|22|3|1949|200|2595|Thursday|1949Q3|N|N|N|2433161|2433403|2432817|2433091|N|N|N|N|N| +2433183|AAAAAAAAPJACFCAA|1949-09-23|596|2595|200|1949|5|9|23|3|1949|200|2595|Friday|1949Q3|N|Y|N|2433161|2433403|2432818|2433092|N|N|N|N|N| +2433184|AAAAAAAAAKACFCAA|1949-09-24|596|2595|200|1949|6|9|24|3|1949|200|2595|Saturday|1949Q3|N|Y|N|2433161|2433403|2432819|2433093|N|N|N|N|N| +2433185|AAAAAAAABKACFCAA|1949-09-25|596|2595|200|1949|0|9|25|3|1949|200|2595|Sunday|1949Q3|N|N|N|2433161|2433403|2432820|2433094|N|N|N|N|N| +2433186|AAAAAAAACKACFCAA|1949-09-26|596|2595|200|1949|1|9|26|3|1949|200|2595|Monday|1949Q3|N|N|N|2433161|2433403|2432821|2433095|N|N|N|N|N| +2433187|AAAAAAAADKACFCAA|1949-09-27|596|2596|200|1949|2|9|27|3|1949|200|2596|Tuesday|1949Q3|N|N|N|2433161|2433403|2432822|2433096|N|N|N|N|N| +2433188|AAAAAAAAEKACFCAA|1949-09-28|596|2596|200|1949|3|9|28|3|1949|200|2596|Wednesday|1949Q3|N|N|N|2433161|2433403|2432823|2433097|N|N|N|N|N| +2433189|AAAAAAAAFKACFCAA|1949-09-29|596|2596|200|1949|4|9|29|3|1949|200|2596|Thursday|1949Q3|N|N|N|2433161|2433403|2432824|2433098|N|N|N|N|N| +2433190|AAAAAAAAGKACFCAA|1949-09-30|596|2596|200|1949|5|9|30|3|1949|200|2596|Friday|1949Q3|N|Y|N|2433161|2433403|2432825|2433099|N|N|N|N|N| +2433191|AAAAAAAAHKACFCAA|1949-10-01|597|2596|200|1949|6|10|1|3|1949|200|2596|Saturday|1949Q3|N|Y|N|2433191|2433463|2432826|2433099|N|N|N|N|N| +2433192|AAAAAAAAIKACFCAA|1949-10-02|597|2596|200|1949|0|10|2|4|1949|200|2596|Sunday|1949Q4|N|N|N|2433191|2433463|2432827|2433100|N|N|N|N|N| +2433193|AAAAAAAAJKACFCAA|1949-10-03|597|2596|200|1949|1|10|3|4|1949|200|2596|Monday|1949Q4|N|N|N|2433191|2433463|2432828|2433101|N|N|N|N|N| +2433194|AAAAAAAAKKACFCAA|1949-10-04|597|2597|200|1949|2|10|4|4|1949|200|2597|Tuesday|1949Q4|N|N|N|2433191|2433463|2432829|2433102|N|N|N|N|N| +2433195|AAAAAAAALKACFCAA|1949-10-05|597|2597|200|1949|3|10|5|4|1949|200|2597|Wednesday|1949Q4|N|N|N|2433191|2433463|2432830|2433103|N|N|N|N|N| +2433196|AAAAAAAAMKACFCAA|1949-10-06|597|2597|200|1949|4|10|6|4|1949|200|2597|Thursday|1949Q4|N|N|N|2433191|2433463|2432831|2433104|N|N|N|N|N| +2433197|AAAAAAAANKACFCAA|1949-10-07|597|2597|200|1949|5|10|7|4|1949|200|2597|Friday|1949Q4|N|Y|N|2433191|2433463|2432832|2433105|N|N|N|N|N| +2433198|AAAAAAAAOKACFCAA|1949-10-08|597|2597|200|1949|6|10|8|4|1949|200|2597|Saturday|1949Q4|N|Y|N|2433191|2433463|2432833|2433106|N|N|N|N|N| +2433199|AAAAAAAAPKACFCAA|1949-10-09|597|2597|200|1949|0|10|9|4|1949|200|2597|Sunday|1949Q4|N|N|N|2433191|2433463|2432834|2433107|N|N|N|N|N| +2433200|AAAAAAAAALACFCAA|1949-10-10|597|2597|200|1949|1|10|10|4|1949|200|2597|Monday|1949Q4|N|N|N|2433191|2433463|2432835|2433108|N|N|N|N|N| +2433201|AAAAAAAABLACFCAA|1949-10-11|597|2598|200|1949|2|10|11|4|1949|200|2598|Tuesday|1949Q4|N|N|N|2433191|2433463|2432836|2433109|N|N|N|N|N| +2433202|AAAAAAAACLACFCAA|1949-10-12|597|2598|200|1949|3|10|12|4|1949|200|2598|Wednesday|1949Q4|N|N|N|2433191|2433463|2432837|2433110|N|N|N|N|N| +2433203|AAAAAAAADLACFCAA|1949-10-13|597|2598|200|1949|4|10|13|4|1949|200|2598|Thursday|1949Q4|N|N|N|2433191|2433463|2432838|2433111|N|N|N|N|N| +2433204|AAAAAAAAELACFCAA|1949-10-14|597|2598|200|1949|5|10|14|4|1949|200|2598|Friday|1949Q4|N|Y|N|2433191|2433463|2432839|2433112|N|N|N|N|N| +2433205|AAAAAAAAFLACFCAA|1949-10-15|597|2598|200|1949|6|10|15|4|1949|200|2598|Saturday|1949Q4|N|Y|N|2433191|2433463|2432840|2433113|N|N|N|N|N| +2433206|AAAAAAAAGLACFCAA|1949-10-16|597|2598|200|1949|0|10|16|4|1949|200|2598|Sunday|1949Q4|N|N|N|2433191|2433463|2432841|2433114|N|N|N|N|N| +2433207|AAAAAAAAHLACFCAA|1949-10-17|597|2598|200|1949|1|10|17|4|1949|200|2598|Monday|1949Q4|N|N|N|2433191|2433463|2432842|2433115|N|N|N|N|N| +2433208|AAAAAAAAILACFCAA|1949-10-18|597|2599|200|1949|2|10|18|4|1949|200|2599|Tuesday|1949Q4|N|N|N|2433191|2433463|2432843|2433116|N|N|N|N|N| +2433209|AAAAAAAAJLACFCAA|1949-10-19|597|2599|200|1949|3|10|19|4|1949|200|2599|Wednesday|1949Q4|N|N|N|2433191|2433463|2432844|2433117|N|N|N|N|N| +2433210|AAAAAAAAKLACFCAA|1949-10-20|597|2599|200|1949|4|10|20|4|1949|200|2599|Thursday|1949Q4|N|N|N|2433191|2433463|2432845|2433118|N|N|N|N|N| +2433211|AAAAAAAALLACFCAA|1949-10-21|597|2599|200|1949|5|10|21|4|1949|200|2599|Friday|1949Q4|N|Y|N|2433191|2433463|2432846|2433119|N|N|N|N|N| +2433212|AAAAAAAAMLACFCAA|1949-10-22|597|2599|200|1949|6|10|22|4|1949|200|2599|Saturday|1949Q4|N|Y|N|2433191|2433463|2432847|2433120|N|N|N|N|N| +2433213|AAAAAAAANLACFCAA|1949-10-23|597|2599|200|1949|0|10|23|4|1949|200|2599|Sunday|1949Q4|N|N|N|2433191|2433463|2432848|2433121|N|N|N|N|N| +2433214|AAAAAAAAOLACFCAA|1949-10-24|597|2599|200|1949|1|10|24|4|1949|200|2599|Monday|1949Q4|N|N|N|2433191|2433463|2432849|2433122|N|N|N|N|N| +2433215|AAAAAAAAPLACFCAA|1949-10-25|597|2600|200|1949|2|10|25|4|1949|200|2600|Tuesday|1949Q4|N|N|N|2433191|2433463|2432850|2433123|N|N|N|N|N| +2433216|AAAAAAAAAMACFCAA|1949-10-26|597|2600|200|1949|3|10|26|4|1949|200|2600|Wednesday|1949Q4|N|N|N|2433191|2433463|2432851|2433124|N|N|N|N|N| +2433217|AAAAAAAABMACFCAA|1949-10-27|597|2600|200|1949|4|10|27|4|1949|200|2600|Thursday|1949Q4|N|N|N|2433191|2433463|2432852|2433125|N|N|N|N|N| +2433218|AAAAAAAACMACFCAA|1949-10-28|597|2600|200|1949|5|10|28|4|1949|200|2600|Friday|1949Q4|N|Y|N|2433191|2433463|2432853|2433126|N|N|N|N|N| +2433219|AAAAAAAADMACFCAA|1949-10-29|597|2600|200|1949|6|10|29|4|1949|200|2600|Saturday|1949Q4|N|Y|N|2433191|2433463|2432854|2433127|N|N|N|N|N| +2433220|AAAAAAAAEMACFCAA|1949-10-30|597|2600|200|1949|0|10|30|4|1949|200|2600|Sunday|1949Q4|N|N|N|2433191|2433463|2432855|2433128|N|N|N|N|N| +2433221|AAAAAAAAFMACFCAA|1949-10-31|597|2600|200|1949|1|10|31|4|1949|200|2600|Monday|1949Q4|N|N|N|2433191|2433463|2432856|2433129|N|N|N|N|N| +2433222|AAAAAAAAGMACFCAA|1949-11-01|598|2601|200|1949|2|11|1|4|1949|200|2601|Tuesday|1949Q4|N|N|N|2433222|2433525|2432857|2433130|N|N|N|N|N| +2433223|AAAAAAAAHMACFCAA|1949-11-02|598|2601|200|1949|3|11|2|4|1949|200|2601|Wednesday|1949Q4|N|N|N|2433222|2433525|2432858|2433131|N|N|N|N|N| +2433224|AAAAAAAAIMACFCAA|1949-11-03|598|2601|200|1949|4|11|3|4|1949|200|2601|Thursday|1949Q4|N|N|N|2433222|2433525|2432859|2433132|N|N|N|N|N| +2433225|AAAAAAAAJMACFCAA|1949-11-04|598|2601|200|1949|5|11|4|4|1949|200|2601|Friday|1949Q4|N|Y|N|2433222|2433525|2432860|2433133|N|N|N|N|N| +2433226|AAAAAAAAKMACFCAA|1949-11-05|598|2601|200|1949|6|11|5|4|1949|200|2601|Saturday|1949Q4|N|Y|N|2433222|2433525|2432861|2433134|N|N|N|N|N| +2433227|AAAAAAAALMACFCAA|1949-11-06|598|2601|200|1949|0|11|6|4|1949|200|2601|Sunday|1949Q4|N|N|N|2433222|2433525|2432862|2433135|N|N|N|N|N| +2433228|AAAAAAAAMMACFCAA|1949-11-07|598|2601|200|1949|1|11|7|4|1949|200|2601|Monday|1949Q4|N|N|N|2433222|2433525|2432863|2433136|N|N|N|N|N| +2433229|AAAAAAAANMACFCAA|1949-11-08|598|2602|200|1949|2|11|8|4|1949|200|2602|Tuesday|1949Q4|N|N|N|2433222|2433525|2432864|2433137|N|N|N|N|N| +2433230|AAAAAAAAOMACFCAA|1949-11-09|598|2602|200|1949|3|11|9|4|1949|200|2602|Wednesday|1949Q4|N|N|N|2433222|2433525|2432865|2433138|N|N|N|N|N| +2433231|AAAAAAAAPMACFCAA|1949-11-10|598|2602|200|1949|4|11|10|4|1949|200|2602|Thursday|1949Q4|N|N|N|2433222|2433525|2432866|2433139|N|N|N|N|N| +2433232|AAAAAAAAANACFCAA|1949-11-11|598|2602|200|1949|5|11|11|4|1949|200|2602|Friday|1949Q4|N|Y|N|2433222|2433525|2432867|2433140|N|N|N|N|N| +2433233|AAAAAAAABNACFCAA|1949-11-12|598|2602|200|1949|6|11|12|4|1949|200|2602|Saturday|1949Q4|N|Y|N|2433222|2433525|2432868|2433141|N|N|N|N|N| +2433234|AAAAAAAACNACFCAA|1949-11-13|598|2602|200|1949|0|11|13|4|1949|200|2602|Sunday|1949Q4|N|N|N|2433222|2433525|2432869|2433142|N|N|N|N|N| +2433235|AAAAAAAADNACFCAA|1949-11-14|598|2602|200|1949|1|11|14|4|1949|200|2602|Monday|1949Q4|N|N|N|2433222|2433525|2432870|2433143|N|N|N|N|N| +2433236|AAAAAAAAENACFCAA|1949-11-15|598|2603|200|1949|2|11|15|4|1949|200|2603|Tuesday|1949Q4|N|N|N|2433222|2433525|2432871|2433144|N|N|N|N|N| +2433237|AAAAAAAAFNACFCAA|1949-11-16|598|2603|200|1949|3|11|16|4|1949|200|2603|Wednesday|1949Q4|N|N|N|2433222|2433525|2432872|2433145|N|N|N|N|N| +2433238|AAAAAAAAGNACFCAA|1949-11-17|598|2603|200|1949|4|11|17|4|1949|200|2603|Thursday|1949Q4|N|N|N|2433222|2433525|2432873|2433146|N|N|N|N|N| +2433239|AAAAAAAAHNACFCAA|1949-11-18|598|2603|200|1949|5|11|18|4|1949|200|2603|Friday|1949Q4|N|Y|N|2433222|2433525|2432874|2433147|N|N|N|N|N| +2433240|AAAAAAAAINACFCAA|1949-11-19|598|2603|200|1949|6|11|19|4|1949|200|2603|Saturday|1949Q4|N|Y|N|2433222|2433525|2432875|2433148|N|N|N|N|N| +2433241|AAAAAAAAJNACFCAA|1949-11-20|598|2603|200|1949|0|11|20|4|1949|200|2603|Sunday|1949Q4|N|N|N|2433222|2433525|2432876|2433149|N|N|N|N|N| +2433242|AAAAAAAAKNACFCAA|1949-11-21|598|2603|200|1949|1|11|21|4|1949|200|2603|Monday|1949Q4|N|N|N|2433222|2433525|2432877|2433150|N|N|N|N|N| +2433243|AAAAAAAALNACFCAA|1949-11-22|598|2604|200|1949|2|11|22|4|1949|200|2604|Tuesday|1949Q4|N|N|N|2433222|2433525|2432878|2433151|N|N|N|N|N| +2433244|AAAAAAAAMNACFCAA|1949-11-23|598|2604|200|1949|3|11|23|4|1949|200|2604|Wednesday|1949Q4|N|N|N|2433222|2433525|2432879|2433152|N|N|N|N|N| +2433245|AAAAAAAANNACFCAA|1949-11-24|598|2604|200|1949|4|11|24|4|1949|200|2604|Thursday|1949Q4|N|N|N|2433222|2433525|2432880|2433153|N|N|N|N|N| +2433246|AAAAAAAAONACFCAA|1949-11-25|598|2604|200|1949|5|11|25|4|1949|200|2604|Friday|1949Q4|N|Y|N|2433222|2433525|2432881|2433154|N|N|N|N|N| +2433247|AAAAAAAAPNACFCAA|1949-11-26|598|2604|200|1949|6|11|26|4|1949|200|2604|Saturday|1949Q4|N|Y|N|2433222|2433525|2432882|2433155|N|N|N|N|N| +2433248|AAAAAAAAAOACFCAA|1949-11-27|598|2604|200|1949|0|11|27|4|1949|200|2604|Sunday|1949Q4|N|N|N|2433222|2433525|2432883|2433156|N|N|N|N|N| +2433249|AAAAAAAABOACFCAA|1949-11-28|598|2604|200|1949|1|11|28|4|1949|200|2604|Monday|1949Q4|N|N|N|2433222|2433525|2432884|2433157|N|N|N|N|N| +2433250|AAAAAAAACOACFCAA|1949-11-29|598|2605|200|1949|2|11|29|4|1949|200|2605|Tuesday|1949Q4|N|N|N|2433222|2433525|2432885|2433158|N|N|N|N|N| +2433251|AAAAAAAADOACFCAA|1949-11-30|598|2605|200|1949|3|11|30|4|1949|200|2605|Wednesday|1949Q4|N|N|N|2433222|2433525|2432886|2433159|N|N|N|N|N| +2433252|AAAAAAAAEOACFCAA|1949-12-01|599|2605|201|1949|4|12|1|4|1949|201|2605|Thursday|1949Q4|N|N|N|2433252|2433585|2432887|2433160|N|N|N|N|N| +2433253|AAAAAAAAFOACFCAA|1949-12-02|599|2605|201|1949|5|12|2|4|1949|201|2605|Friday|1949Q4|N|Y|N|2433252|2433585|2432888|2433161|N|N|N|N|N| +2433254|AAAAAAAAGOACFCAA|1949-12-03|599|2605|201|1949|6|12|3|4|1949|201|2605|Saturday|1949Q4|N|Y|N|2433252|2433585|2432889|2433162|N|N|N|N|N| +2433255|AAAAAAAAHOACFCAA|1949-12-04|599|2605|201|1949|0|12|4|4|1949|201|2605|Sunday|1949Q4|N|N|N|2433252|2433585|2432890|2433163|N|N|N|N|N| +2433256|AAAAAAAAIOACFCAA|1949-12-05|599|2605|201|1949|1|12|5|4|1949|201|2605|Monday|1949Q4|N|N|N|2433252|2433585|2432891|2433164|N|N|N|N|N| +2433257|AAAAAAAAJOACFCAA|1949-12-06|599|2606|201|1949|2|12|6|4|1949|201|2606|Tuesday|1949Q4|N|N|N|2433252|2433585|2432892|2433165|N|N|N|N|N| +2433258|AAAAAAAAKOACFCAA|1949-12-07|599|2606|201|1949|3|12|7|4|1949|201|2606|Wednesday|1949Q4|N|N|N|2433252|2433585|2432893|2433166|N|N|N|N|N| +2433259|AAAAAAAALOACFCAA|1949-12-08|599|2606|201|1949|4|12|8|4|1949|201|2606|Thursday|1949Q4|N|N|N|2433252|2433585|2432894|2433167|N|N|N|N|N| +2433260|AAAAAAAAMOACFCAA|1949-12-09|599|2606|201|1949|5|12|9|4|1949|201|2606|Friday|1949Q4|N|Y|N|2433252|2433585|2432895|2433168|N|N|N|N|N| +2433261|AAAAAAAANOACFCAA|1949-12-10|599|2606|201|1949|6|12|10|4|1949|201|2606|Saturday|1949Q4|N|Y|N|2433252|2433585|2432896|2433169|N|N|N|N|N| +2433262|AAAAAAAAOOACFCAA|1949-12-11|599|2606|201|1949|0|12|11|4|1949|201|2606|Sunday|1949Q4|N|N|N|2433252|2433585|2432897|2433170|N|N|N|N|N| +2433263|AAAAAAAAPOACFCAA|1949-12-12|599|2606|201|1949|1|12|12|4|1949|201|2606|Monday|1949Q4|N|N|N|2433252|2433585|2432898|2433171|N|N|N|N|N| +2433264|AAAAAAAAAPACFCAA|1949-12-13|599|2607|201|1949|2|12|13|4|1949|201|2607|Tuesday|1949Q4|N|N|N|2433252|2433585|2432899|2433172|N|N|N|N|N| +2433265|AAAAAAAABPACFCAA|1949-12-14|599|2607|201|1949|3|12|14|4|1949|201|2607|Wednesday|1949Q4|N|N|N|2433252|2433585|2432900|2433173|N|N|N|N|N| +2433266|AAAAAAAACPACFCAA|1949-12-15|599|2607|201|1949|4|12|15|4|1949|201|2607|Thursday|1949Q4|N|N|N|2433252|2433585|2432901|2433174|N|N|N|N|N| +2433267|AAAAAAAADPACFCAA|1949-12-16|599|2607|201|1949|5|12|16|4|1949|201|2607|Friday|1949Q4|N|Y|N|2433252|2433585|2432902|2433175|N|N|N|N|N| +2433268|AAAAAAAAEPACFCAA|1949-12-17|599|2607|201|1949|6|12|17|4|1949|201|2607|Saturday|1949Q4|N|Y|N|2433252|2433585|2432903|2433176|N|N|N|N|N| +2433269|AAAAAAAAFPACFCAA|1949-12-18|599|2607|201|1949|0|12|18|4|1949|201|2607|Sunday|1949Q4|N|N|N|2433252|2433585|2432904|2433177|N|N|N|N|N| +2433270|AAAAAAAAGPACFCAA|1949-12-19|599|2607|201|1949|1|12|19|4|1949|201|2607|Monday|1949Q4|N|N|N|2433252|2433585|2432905|2433178|N|N|N|N|N| +2433271|AAAAAAAAHPACFCAA|1949-12-20|599|2608|201|1949|2|12|20|4|1949|201|2608|Tuesday|1949Q4|N|N|N|2433252|2433585|2432906|2433179|N|N|N|N|N| +2433272|AAAAAAAAIPACFCAA|1949-12-21|599|2608|201|1949|3|12|21|4|1949|201|2608|Wednesday|1949Q4|N|N|N|2433252|2433585|2432907|2433180|N|N|N|N|N| +2433273|AAAAAAAAJPACFCAA|1949-12-22|599|2608|201|1949|4|12|22|4|1949|201|2608|Thursday|1949Q4|N|N|N|2433252|2433585|2432908|2433181|N|N|N|N|N| +2433274|AAAAAAAAKPACFCAA|1949-12-23|599|2608|201|1949|5|12|23|4|1949|201|2608|Friday|1949Q4|N|Y|N|2433252|2433585|2432909|2433182|N|N|N|N|N| +2433275|AAAAAAAALPACFCAA|1949-12-24|599|2608|201|1949|6|12|24|4|1949|201|2608|Saturday|1949Q4|N|Y|N|2433252|2433585|2432910|2433183|N|N|N|N|N| +2433276|AAAAAAAAMPACFCAA|1949-12-25|599|2608|201|1949|0|12|25|4|1949|201|2608|Sunday|1949Q4|N|N|N|2433252|2433585|2432911|2433184|N|N|N|N|N| +2433277|AAAAAAAANPACFCAA|1949-12-26|599|2608|201|1949|1|12|26|4|1949|201|2608|Monday|1949Q4|Y|N|N|2433252|2433585|2432912|2433185|N|N|N|N|N| +2433278|AAAAAAAAOPACFCAA|1949-12-27|599|2609|201|1949|2|12|27|4|1949|201|2609|Tuesday|1949Q4|N|N|Y|2433252|2433585|2432913|2433186|N|N|N|N|N| +2433279|AAAAAAAAPPACFCAA|1949-12-28|599|2609|201|1949|3|12|28|4|1949|201|2609|Wednesday|1949Q4|N|N|N|2433252|2433585|2432914|2433187|N|N|N|N|N| +2433280|AAAAAAAAAABCFCAA|1949-12-29|599|2609|201|1949|4|12|29|4|1949|201|2609|Thursday|1949Q4|N|N|N|2433252|2433585|2432915|2433188|N|N|N|N|N| +2433281|AAAAAAAABABCFCAA|1949-12-30|599|2609|201|1949|5|12|30|4|1949|201|2609|Friday|1949Q4|N|Y|N|2433252|2433585|2432916|2433189|N|N|N|N|N| +2433282|AAAAAAAACABCFCAA|1949-12-31|599|2609|201|1949|6|12|31|4|1949|201|2609|Saturday|1949Q4|N|Y|N|2433252|2433585|2432917|2433190|N|N|N|N|N| +2433283|AAAAAAAADABCFCAA|1950-01-01|600|2609|201|1950|0|1|1|1|1950|201|2609|Sunday|1950Q1|Y|N|N|2433283|2433282|2432918|2433191|N|N|N|N|N| +2433284|AAAAAAAAEABCFCAA|1950-01-02|600|2609|201|1950|1|1|2|1|1950|201|2609|Monday|1950Q1|N|N|Y|2433283|2433282|2432919|2433192|N|N|N|N|N| +2433285|AAAAAAAAFABCFCAA|1950-01-03|600|2610|201|1950|2|1|3|1|1950|201|2610|Tuesday|1950Q1|N|N|N|2433283|2433282|2432920|2433193|N|N|N|N|N| +2433286|AAAAAAAAGABCFCAA|1950-01-04|600|2610|201|1950|3|1|4|1|1950|201|2610|Wednesday|1950Q1|N|N|N|2433283|2433282|2432921|2433194|N|N|N|N|N| +2433287|AAAAAAAAHABCFCAA|1950-01-05|600|2610|201|1950|4|1|5|1|1950|201|2610|Thursday|1950Q1|N|N|N|2433283|2433282|2432922|2433195|N|N|N|N|N| +2433288|AAAAAAAAIABCFCAA|1950-01-06|600|2610|201|1950|5|1|6|1|1950|201|2610|Friday|1950Q1|N|Y|N|2433283|2433282|2432923|2433196|N|N|N|N|N| +2433289|AAAAAAAAJABCFCAA|1950-01-07|600|2610|201|1950|6|1|7|1|1950|201|2610|Saturday|1950Q1|N|Y|N|2433283|2433282|2432924|2433197|N|N|N|N|N| +2433290|AAAAAAAAKABCFCAA|1950-01-08|600|2610|201|1950|0|1|8|1|1950|201|2610|Sunday|1950Q1|N|N|N|2433283|2433282|2432925|2433198|N|N|N|N|N| +2433291|AAAAAAAALABCFCAA|1950-01-09|600|2610|201|1950|1|1|9|1|1950|201|2610|Monday|1950Q1|N|N|N|2433283|2433282|2432926|2433199|N|N|N|N|N| +2433292|AAAAAAAAMABCFCAA|1950-01-10|600|2611|201|1950|2|1|10|1|1950|201|2611|Tuesday|1950Q1|N|N|N|2433283|2433282|2432927|2433200|N|N|N|N|N| +2433293|AAAAAAAANABCFCAA|1950-01-11|600|2611|201|1950|3|1|11|1|1950|201|2611|Wednesday|1950Q1|N|N|N|2433283|2433282|2432928|2433201|N|N|N|N|N| +2433294|AAAAAAAAOABCFCAA|1950-01-12|600|2611|201|1950|4|1|12|1|1950|201|2611|Thursday|1950Q1|N|N|N|2433283|2433282|2432929|2433202|N|N|N|N|N| +2433295|AAAAAAAAPABCFCAA|1950-01-13|600|2611|201|1950|5|1|13|1|1950|201|2611|Friday|1950Q1|N|Y|N|2433283|2433282|2432930|2433203|N|N|N|N|N| +2433296|AAAAAAAAABBCFCAA|1950-01-14|600|2611|201|1950|6|1|14|1|1950|201|2611|Saturday|1950Q1|N|Y|N|2433283|2433282|2432931|2433204|N|N|N|N|N| +2433297|AAAAAAAABBBCFCAA|1950-01-15|600|2611|201|1950|0|1|15|1|1950|201|2611|Sunday|1950Q1|N|N|N|2433283|2433282|2432932|2433205|N|N|N|N|N| +2433298|AAAAAAAACBBCFCAA|1950-01-16|600|2611|201|1950|1|1|16|1|1950|201|2611|Monday|1950Q1|N|N|N|2433283|2433282|2432933|2433206|N|N|N|N|N| +2433299|AAAAAAAADBBCFCAA|1950-01-17|600|2612|201|1950|2|1|17|1|1950|201|2612|Tuesday|1950Q1|N|N|N|2433283|2433282|2432934|2433207|N|N|N|N|N| +2433300|AAAAAAAAEBBCFCAA|1950-01-18|600|2612|201|1950|3|1|18|1|1950|201|2612|Wednesday|1950Q1|N|N|N|2433283|2433282|2432935|2433208|N|N|N|N|N| +2433301|AAAAAAAAFBBCFCAA|1950-01-19|600|2612|201|1950|4|1|19|1|1950|201|2612|Thursday|1950Q1|N|N|N|2433283|2433282|2432936|2433209|N|N|N|N|N| +2433302|AAAAAAAAGBBCFCAA|1950-01-20|600|2612|201|1950|5|1|20|1|1950|201|2612|Friday|1950Q1|N|Y|N|2433283|2433282|2432937|2433210|N|N|N|N|N| +2433303|AAAAAAAAHBBCFCAA|1950-01-21|600|2612|201|1950|6|1|21|1|1950|201|2612|Saturday|1950Q1|N|Y|N|2433283|2433282|2432938|2433211|N|N|N|N|N| +2433304|AAAAAAAAIBBCFCAA|1950-01-22|600|2612|201|1950|0|1|22|1|1950|201|2612|Sunday|1950Q1|N|N|N|2433283|2433282|2432939|2433212|N|N|N|N|N| +2433305|AAAAAAAAJBBCFCAA|1950-01-23|600|2612|201|1950|1|1|23|1|1950|201|2612|Monday|1950Q1|N|N|N|2433283|2433282|2432940|2433213|N|N|N|N|N| +2433306|AAAAAAAAKBBCFCAA|1950-01-24|600|2613|201|1950|2|1|24|1|1950|201|2613|Tuesday|1950Q1|N|N|N|2433283|2433282|2432941|2433214|N|N|N|N|N| +2433307|AAAAAAAALBBCFCAA|1950-01-25|600|2613|201|1950|3|1|25|1|1950|201|2613|Wednesday|1950Q1|N|N|N|2433283|2433282|2432942|2433215|N|N|N|N|N| +2433308|AAAAAAAAMBBCFCAA|1950-01-26|600|2613|201|1950|4|1|26|1|1950|201|2613|Thursday|1950Q1|N|N|N|2433283|2433282|2432943|2433216|N|N|N|N|N| +2433309|AAAAAAAANBBCFCAA|1950-01-27|600|2613|201|1950|5|1|27|1|1950|201|2613|Friday|1950Q1|N|Y|N|2433283|2433282|2432944|2433217|N|N|N|N|N| +2433310|AAAAAAAAOBBCFCAA|1950-01-28|600|2613|201|1950|6|1|28|1|1950|201|2613|Saturday|1950Q1|N|Y|N|2433283|2433282|2432945|2433218|N|N|N|N|N| +2433311|AAAAAAAAPBBCFCAA|1950-01-29|600|2613|201|1950|0|1|29|1|1950|201|2613|Sunday|1950Q1|N|N|N|2433283|2433282|2432946|2433219|N|N|N|N|N| +2433312|AAAAAAAAACBCFCAA|1950-01-30|600|2613|201|1950|1|1|30|1|1950|201|2613|Monday|1950Q1|N|N|N|2433283|2433282|2432947|2433220|N|N|N|N|N| +2433313|AAAAAAAABCBCFCAA|1950-01-31|600|2614|201|1950|2|1|31|1|1950|201|2614|Tuesday|1950Q1|N|N|N|2433283|2433282|2432948|2433221|N|N|N|N|N| +2433314|AAAAAAAACCBCFCAA|1950-02-01|601|2614|201|1950|3|2|1|1|1950|201|2614|Wednesday|1950Q1|N|N|N|2433314|2433344|2432949|2433222|N|N|N|N|N| +2433315|AAAAAAAADCBCFCAA|1950-02-02|601|2614|201|1950|4|2|2|1|1950|201|2614|Thursday|1950Q1|N|N|N|2433314|2433344|2432950|2433223|N|N|N|N|N| +2433316|AAAAAAAAECBCFCAA|1950-02-03|601|2614|201|1950|5|2|3|1|1950|201|2614|Friday|1950Q1|N|Y|N|2433314|2433344|2432951|2433224|N|N|N|N|N| +2433317|AAAAAAAAFCBCFCAA|1950-02-04|601|2614|201|1950|6|2|4|1|1950|201|2614|Saturday|1950Q1|N|Y|N|2433314|2433344|2432952|2433225|N|N|N|N|N| +2433318|AAAAAAAAGCBCFCAA|1950-02-05|601|2614|201|1950|0|2|5|1|1950|201|2614|Sunday|1950Q1|N|N|N|2433314|2433344|2432953|2433226|N|N|N|N|N| +2433319|AAAAAAAAHCBCFCAA|1950-02-06|601|2614|201|1950|1|2|6|1|1950|201|2614|Monday|1950Q1|N|N|N|2433314|2433344|2432954|2433227|N|N|N|N|N| +2433320|AAAAAAAAICBCFCAA|1950-02-07|601|2615|201|1950|2|2|7|1|1950|201|2615|Tuesday|1950Q1|N|N|N|2433314|2433344|2432955|2433228|N|N|N|N|N| +2433321|AAAAAAAAJCBCFCAA|1950-02-08|601|2615|201|1950|3|2|8|1|1950|201|2615|Wednesday|1950Q1|N|N|N|2433314|2433344|2432956|2433229|N|N|N|N|N| +2433322|AAAAAAAAKCBCFCAA|1950-02-09|601|2615|201|1950|4|2|9|1|1950|201|2615|Thursday|1950Q1|N|N|N|2433314|2433344|2432957|2433230|N|N|N|N|N| +2433323|AAAAAAAALCBCFCAA|1950-02-10|601|2615|201|1950|5|2|10|1|1950|201|2615|Friday|1950Q1|N|Y|N|2433314|2433344|2432958|2433231|N|N|N|N|N| +2433324|AAAAAAAAMCBCFCAA|1950-02-11|601|2615|201|1950|6|2|11|1|1950|201|2615|Saturday|1950Q1|N|Y|N|2433314|2433344|2432959|2433232|N|N|N|N|N| +2433325|AAAAAAAANCBCFCAA|1950-02-12|601|2615|201|1950|0|2|12|1|1950|201|2615|Sunday|1950Q1|N|N|N|2433314|2433344|2432960|2433233|N|N|N|N|N| +2433326|AAAAAAAAOCBCFCAA|1950-02-13|601|2615|201|1950|1|2|13|1|1950|201|2615|Monday|1950Q1|N|N|N|2433314|2433344|2432961|2433234|N|N|N|N|N| +2433327|AAAAAAAAPCBCFCAA|1950-02-14|601|2616|201|1950|2|2|14|1|1950|201|2616|Tuesday|1950Q1|N|N|N|2433314|2433344|2432962|2433235|N|N|N|N|N| +2433328|AAAAAAAAADBCFCAA|1950-02-15|601|2616|201|1950|3|2|15|1|1950|201|2616|Wednesday|1950Q1|N|N|N|2433314|2433344|2432963|2433236|N|N|N|N|N| +2433329|AAAAAAAABDBCFCAA|1950-02-16|601|2616|201|1950|4|2|16|1|1950|201|2616|Thursday|1950Q1|N|N|N|2433314|2433344|2432964|2433237|N|N|N|N|N| +2433330|AAAAAAAACDBCFCAA|1950-02-17|601|2616|201|1950|5|2|17|1|1950|201|2616|Friday|1950Q1|N|Y|N|2433314|2433344|2432965|2433238|N|N|N|N|N| +2433331|AAAAAAAADDBCFCAA|1950-02-18|601|2616|201|1950|6|2|18|1|1950|201|2616|Saturday|1950Q1|N|Y|N|2433314|2433344|2432966|2433239|N|N|N|N|N| +2433332|AAAAAAAAEDBCFCAA|1950-02-19|601|2616|201|1950|0|2|19|1|1950|201|2616|Sunday|1950Q1|N|N|N|2433314|2433344|2432967|2433240|N|N|N|N|N| +2433333|AAAAAAAAFDBCFCAA|1950-02-20|601|2616|201|1950|1|2|20|1|1950|201|2616|Monday|1950Q1|N|N|N|2433314|2433344|2432968|2433241|N|N|N|N|N| +2433334|AAAAAAAAGDBCFCAA|1950-02-21|601|2617|201|1950|2|2|21|1|1950|201|2617|Tuesday|1950Q1|N|N|N|2433314|2433344|2432969|2433242|N|N|N|N|N| +2433335|AAAAAAAAHDBCFCAA|1950-02-22|601|2617|201|1950|3|2|22|1|1950|201|2617|Wednesday|1950Q1|N|N|N|2433314|2433344|2432970|2433243|N|N|N|N|N| +2433336|AAAAAAAAIDBCFCAA|1950-02-23|601|2617|201|1950|4|2|23|1|1950|201|2617|Thursday|1950Q1|N|N|N|2433314|2433344|2432971|2433244|N|N|N|N|N| +2433337|AAAAAAAAJDBCFCAA|1950-02-24|601|2617|201|1950|5|2|24|1|1950|201|2617|Friday|1950Q1|N|Y|N|2433314|2433344|2432972|2433245|N|N|N|N|N| +2433338|AAAAAAAAKDBCFCAA|1950-02-25|601|2617|201|1950|6|2|25|1|1950|201|2617|Saturday|1950Q1|N|Y|N|2433314|2433344|2432973|2433246|N|N|N|N|N| +2433339|AAAAAAAALDBCFCAA|1950-02-26|601|2617|201|1950|0|2|26|1|1950|201|2617|Sunday|1950Q1|N|N|N|2433314|2433344|2432974|2433247|N|N|N|N|N| +2433340|AAAAAAAAMDBCFCAA|1950-02-27|601|2617|201|1950|1|2|27|1|1950|201|2617|Monday|1950Q1|N|N|N|2433314|2433344|2432975|2433248|N|N|N|N|N| +2433341|AAAAAAAANDBCFCAA|1950-02-28|601|2618|201|1950|2|2|28|1|1950|201|2618|Tuesday|1950Q1|N|N|N|2433314|2433344|2432976|2433249|N|N|N|N|N| +2433342|AAAAAAAAODBCFCAA|1950-03-01|602|2618|202|1950|3|3|1|1|1950|202|2618|Wednesday|1950Q1|N|N|N|2433342|2433400|2432977|2433250|N|N|N|N|N| +2433343|AAAAAAAAPDBCFCAA|1950-03-02|602|2618|202|1950|4|3|2|1|1950|202|2618|Thursday|1950Q1|N|N|N|2433342|2433400|2432978|2433251|N|N|N|N|N| +2433344|AAAAAAAAAEBCFCAA|1950-03-03|602|2618|202|1950|5|3|3|1|1950|202|2618|Friday|1950Q1|N|Y|N|2433342|2433400|2432979|2433252|N|N|N|N|N| +2433345|AAAAAAAABEBCFCAA|1950-03-04|602|2618|202|1950|6|3|4|1|1950|202|2618|Saturday|1950Q1|N|Y|N|2433342|2433400|2432980|2433253|N|N|N|N|N| +2433346|AAAAAAAACEBCFCAA|1950-03-05|602|2618|202|1950|0|3|5|1|1950|202|2618|Sunday|1950Q1|N|N|N|2433342|2433400|2432981|2433254|N|N|N|N|N| +2433347|AAAAAAAADEBCFCAA|1950-03-06|602|2618|202|1950|1|3|6|1|1950|202|2618|Monday|1950Q1|N|N|N|2433342|2433400|2432982|2433255|N|N|N|N|N| +2433348|AAAAAAAAEEBCFCAA|1950-03-07|602|2619|202|1950|2|3|7|1|1950|202|2619|Tuesday|1950Q1|N|N|N|2433342|2433400|2432983|2433256|N|N|N|N|N| +2433349|AAAAAAAAFEBCFCAA|1950-03-08|602|2619|202|1950|3|3|8|1|1950|202|2619|Wednesday|1950Q1|N|N|N|2433342|2433400|2432984|2433257|N|N|N|N|N| +2433350|AAAAAAAAGEBCFCAA|1950-03-09|602|2619|202|1950|4|3|9|1|1950|202|2619|Thursday|1950Q1|N|N|N|2433342|2433400|2432985|2433258|N|N|N|N|N| +2433351|AAAAAAAAHEBCFCAA|1950-03-10|602|2619|202|1950|5|3|10|1|1950|202|2619|Friday|1950Q1|N|Y|N|2433342|2433400|2432986|2433259|N|N|N|N|N| +2433352|AAAAAAAAIEBCFCAA|1950-03-11|602|2619|202|1950|6|3|11|1|1950|202|2619|Saturday|1950Q1|N|Y|N|2433342|2433400|2432987|2433260|N|N|N|N|N| +2433353|AAAAAAAAJEBCFCAA|1950-03-12|602|2619|202|1950|0|3|12|1|1950|202|2619|Sunday|1950Q1|N|N|N|2433342|2433400|2432988|2433261|N|N|N|N|N| +2433354|AAAAAAAAKEBCFCAA|1950-03-13|602|2619|202|1950|1|3|13|1|1950|202|2619|Monday|1950Q1|N|N|N|2433342|2433400|2432989|2433262|N|N|N|N|N| +2433355|AAAAAAAALEBCFCAA|1950-03-14|602|2620|202|1950|2|3|14|1|1950|202|2620|Tuesday|1950Q1|N|N|N|2433342|2433400|2432990|2433263|N|N|N|N|N| +2433356|AAAAAAAAMEBCFCAA|1950-03-15|602|2620|202|1950|3|3|15|1|1950|202|2620|Wednesday|1950Q1|N|N|N|2433342|2433400|2432991|2433264|N|N|N|N|N| +2433357|AAAAAAAANEBCFCAA|1950-03-16|602|2620|202|1950|4|3|16|1|1950|202|2620|Thursday|1950Q1|N|N|N|2433342|2433400|2432992|2433265|N|N|N|N|N| +2433358|AAAAAAAAOEBCFCAA|1950-03-17|602|2620|202|1950|5|3|17|1|1950|202|2620|Friday|1950Q1|N|Y|N|2433342|2433400|2432993|2433266|N|N|N|N|N| +2433359|AAAAAAAAPEBCFCAA|1950-03-18|602|2620|202|1950|6|3|18|1|1950|202|2620|Saturday|1950Q1|N|Y|N|2433342|2433400|2432994|2433267|N|N|N|N|N| +2433360|AAAAAAAAAFBCFCAA|1950-03-19|602|2620|202|1950|0|3|19|1|1950|202|2620|Sunday|1950Q1|N|N|N|2433342|2433400|2432995|2433268|N|N|N|N|N| +2433361|AAAAAAAABFBCFCAA|1950-03-20|602|2620|202|1950|1|3|20|1|1950|202|2620|Monday|1950Q1|N|N|N|2433342|2433400|2432996|2433269|N|N|N|N|N| +2433362|AAAAAAAACFBCFCAA|1950-03-21|602|2621|202|1950|2|3|21|1|1950|202|2621|Tuesday|1950Q1|N|N|N|2433342|2433400|2432997|2433270|N|N|N|N|N| +2433363|AAAAAAAADFBCFCAA|1950-03-22|602|2621|202|1950|3|3|22|1|1950|202|2621|Wednesday|1950Q1|N|N|N|2433342|2433400|2432998|2433271|N|N|N|N|N| +2433364|AAAAAAAAEFBCFCAA|1950-03-23|602|2621|202|1950|4|3|23|1|1950|202|2621|Thursday|1950Q1|N|N|N|2433342|2433400|2432999|2433272|N|N|N|N|N| +2433365|AAAAAAAAFFBCFCAA|1950-03-24|602|2621|202|1950|5|3|24|1|1950|202|2621|Friday|1950Q1|N|Y|N|2433342|2433400|2433000|2433273|N|N|N|N|N| +2433366|AAAAAAAAGFBCFCAA|1950-03-25|602|2621|202|1950|6|3|25|1|1950|202|2621|Saturday|1950Q1|N|Y|N|2433342|2433400|2433001|2433274|N|N|N|N|N| +2433367|AAAAAAAAHFBCFCAA|1950-03-26|602|2621|202|1950|0|3|26|1|1950|202|2621|Sunday|1950Q1|N|N|N|2433342|2433400|2433002|2433275|N|N|N|N|N| +2433368|AAAAAAAAIFBCFCAA|1950-03-27|602|2621|202|1950|1|3|27|1|1950|202|2621|Monday|1950Q1|N|N|N|2433342|2433400|2433003|2433276|N|N|N|N|N| +2433369|AAAAAAAAJFBCFCAA|1950-03-28|602|2622|202|1950|2|3|28|1|1950|202|2622|Tuesday|1950Q1|N|N|N|2433342|2433400|2433004|2433277|N|N|N|N|N| +2433370|AAAAAAAAKFBCFCAA|1950-03-29|602|2622|202|1950|3|3|29|1|1950|202|2622|Wednesday|1950Q1|N|N|N|2433342|2433400|2433005|2433278|N|N|N|N|N| +2433371|AAAAAAAALFBCFCAA|1950-03-30|602|2622|202|1950|4|3|30|1|1950|202|2622|Thursday|1950Q1|N|N|N|2433342|2433400|2433006|2433279|N|N|N|N|N| +2433372|AAAAAAAAMFBCFCAA|1950-03-31|602|2622|202|1950|5|3|31|1|1950|202|2622|Friday|1950Q1|N|Y|N|2433342|2433400|2433007|2433280|N|N|N|N|N| +2433373|AAAAAAAANFBCFCAA|1950-04-01|603|2622|202|1950|6|4|1|1|1950|202|2622|Saturday|1950Q1|N|Y|N|2433373|2433462|2433008|2433283|N|N|N|N|N| +2433374|AAAAAAAAOFBCFCAA|1950-04-02|603|2622|202|1950|0|4|2|2|1950|202|2622|Sunday|1950Q2|N|N|N|2433373|2433462|2433009|2433284|N|N|N|N|N| +2433375|AAAAAAAAPFBCFCAA|1950-04-03|603|2622|202|1950|1|4|3|2|1950|202|2622|Monday|1950Q2|N|N|N|2433373|2433462|2433010|2433285|N|N|N|N|N| +2433376|AAAAAAAAAGBCFCAA|1950-04-04|603|2623|202|1950|2|4|4|2|1950|202|2623|Tuesday|1950Q2|N|N|N|2433373|2433462|2433011|2433286|N|N|N|N|N| +2433377|AAAAAAAABGBCFCAA|1950-04-05|603|2623|202|1950|3|4|5|2|1950|202|2623|Wednesday|1950Q2|N|N|N|2433373|2433462|2433012|2433287|N|N|N|N|N| +2433378|AAAAAAAACGBCFCAA|1950-04-06|603|2623|202|1950|4|4|6|2|1950|202|2623|Thursday|1950Q2|N|N|N|2433373|2433462|2433013|2433288|N|N|N|N|N| +2433379|AAAAAAAADGBCFCAA|1950-04-07|603|2623|202|1950|5|4|7|2|1950|202|2623|Friday|1950Q2|N|Y|N|2433373|2433462|2433014|2433289|N|N|N|N|N| +2433380|AAAAAAAAEGBCFCAA|1950-04-08|603|2623|202|1950|6|4|8|2|1950|202|2623|Saturday|1950Q2|N|Y|N|2433373|2433462|2433015|2433290|N|N|N|N|N| +2433381|AAAAAAAAFGBCFCAA|1950-04-09|603|2623|202|1950|0|4|9|2|1950|202|2623|Sunday|1950Q2|N|N|N|2433373|2433462|2433016|2433291|N|N|N|N|N| +2433382|AAAAAAAAGGBCFCAA|1950-04-10|603|2623|202|1950|1|4|10|2|1950|202|2623|Monday|1950Q2|N|N|N|2433373|2433462|2433017|2433292|N|N|N|N|N| +2433383|AAAAAAAAHGBCFCAA|1950-04-11|603|2624|202|1950|2|4|11|2|1950|202|2624|Tuesday|1950Q2|N|N|N|2433373|2433462|2433018|2433293|N|N|N|N|N| +2433384|AAAAAAAAIGBCFCAA|1950-04-12|603|2624|202|1950|3|4|12|2|1950|202|2624|Wednesday|1950Q2|N|N|N|2433373|2433462|2433019|2433294|N|N|N|N|N| +2433385|AAAAAAAAJGBCFCAA|1950-04-13|603|2624|202|1950|4|4|13|2|1950|202|2624|Thursday|1950Q2|N|N|N|2433373|2433462|2433020|2433295|N|N|N|N|N| +2433386|AAAAAAAAKGBCFCAA|1950-04-14|603|2624|202|1950|5|4|14|2|1950|202|2624|Friday|1950Q2|N|Y|N|2433373|2433462|2433021|2433296|N|N|N|N|N| +2433387|AAAAAAAALGBCFCAA|1950-04-15|603|2624|202|1950|6|4|15|2|1950|202|2624|Saturday|1950Q2|N|Y|N|2433373|2433462|2433022|2433297|N|N|N|N|N| +2433388|AAAAAAAAMGBCFCAA|1950-04-16|603|2624|202|1950|0|4|16|2|1950|202|2624|Sunday|1950Q2|N|N|N|2433373|2433462|2433023|2433298|N|N|N|N|N| +2433389|AAAAAAAANGBCFCAA|1950-04-17|603|2624|202|1950|1|4|17|2|1950|202|2624|Monday|1950Q2|N|N|N|2433373|2433462|2433024|2433299|N|N|N|N|N| +2433390|AAAAAAAAOGBCFCAA|1950-04-18|603|2625|202|1950|2|4|18|2|1950|202|2625|Tuesday|1950Q2|N|N|N|2433373|2433462|2433025|2433300|N|N|N|N|N| +2433391|AAAAAAAAPGBCFCAA|1950-04-19|603|2625|202|1950|3|4|19|2|1950|202|2625|Wednesday|1950Q2|N|N|N|2433373|2433462|2433026|2433301|N|N|N|N|N| +2433392|AAAAAAAAAHBCFCAA|1950-04-20|603|2625|202|1950|4|4|20|2|1950|202|2625|Thursday|1950Q2|N|N|N|2433373|2433462|2433027|2433302|N|N|N|N|N| +2433393|AAAAAAAABHBCFCAA|1950-04-21|603|2625|202|1950|5|4|21|2|1950|202|2625|Friday|1950Q2|N|Y|N|2433373|2433462|2433028|2433303|N|N|N|N|N| +2433394|AAAAAAAACHBCFCAA|1950-04-22|603|2625|202|1950|6|4|22|2|1950|202|2625|Saturday|1950Q2|N|Y|N|2433373|2433462|2433029|2433304|N|N|N|N|N| +2433395|AAAAAAAADHBCFCAA|1950-04-23|603|2625|202|1950|0|4|23|2|1950|202|2625|Sunday|1950Q2|N|N|N|2433373|2433462|2433030|2433305|N|N|N|N|N| +2433396|AAAAAAAAEHBCFCAA|1950-04-24|603|2625|202|1950|1|4|24|2|1950|202|2625|Monday|1950Q2|N|N|N|2433373|2433462|2433031|2433306|N|N|N|N|N| +2433397|AAAAAAAAFHBCFCAA|1950-04-25|603|2626|202|1950|2|4|25|2|1950|202|2626|Tuesday|1950Q2|N|N|N|2433373|2433462|2433032|2433307|N|N|N|N|N| +2433398|AAAAAAAAGHBCFCAA|1950-04-26|603|2626|202|1950|3|4|26|2|1950|202|2626|Wednesday|1950Q2|N|N|N|2433373|2433462|2433033|2433308|N|N|N|N|N| +2433399|AAAAAAAAHHBCFCAA|1950-04-27|603|2626|202|1950|4|4|27|2|1950|202|2626|Thursday|1950Q2|N|N|N|2433373|2433462|2433034|2433309|N|N|N|N|N| +2433400|AAAAAAAAIHBCFCAA|1950-04-28|603|2626|202|1950|5|4|28|2|1950|202|2626|Friday|1950Q2|N|Y|N|2433373|2433462|2433035|2433310|N|N|N|N|N| +2433401|AAAAAAAAJHBCFCAA|1950-04-29|603|2626|202|1950|6|4|29|2|1950|202|2626|Saturday|1950Q2|N|Y|N|2433373|2433462|2433036|2433311|N|N|N|N|N| +2433402|AAAAAAAAKHBCFCAA|1950-04-30|603|2626|202|1950|0|4|30|2|1950|202|2626|Sunday|1950Q2|N|N|N|2433373|2433462|2433037|2433312|N|N|N|N|N| +2433403|AAAAAAAALHBCFCAA|1950-05-01|604|2626|202|1950|1|5|1|2|1950|202|2626|Monday|1950Q2|N|N|N|2433403|2433522|2433038|2433313|N|N|N|N|N| +2433404|AAAAAAAAMHBCFCAA|1950-05-02|604|2627|202|1950|2|5|2|2|1950|202|2627|Tuesday|1950Q2|N|N|N|2433403|2433522|2433039|2433314|N|N|N|N|N| +2433405|AAAAAAAANHBCFCAA|1950-05-03|604|2627|202|1950|3|5|3|2|1950|202|2627|Wednesday|1950Q2|N|N|N|2433403|2433522|2433040|2433315|N|N|N|N|N| +2433406|AAAAAAAAOHBCFCAA|1950-05-04|604|2627|202|1950|4|5|4|2|1950|202|2627|Thursday|1950Q2|N|N|N|2433403|2433522|2433041|2433316|N|N|N|N|N| +2433407|AAAAAAAAPHBCFCAA|1950-05-05|604|2627|202|1950|5|5|5|2|1950|202|2627|Friday|1950Q2|N|Y|N|2433403|2433522|2433042|2433317|N|N|N|N|N| +2433408|AAAAAAAAAIBCFCAA|1950-05-06|604|2627|202|1950|6|5|6|2|1950|202|2627|Saturday|1950Q2|N|Y|N|2433403|2433522|2433043|2433318|N|N|N|N|N| +2433409|AAAAAAAABIBCFCAA|1950-05-07|604|2627|202|1950|0|5|7|2|1950|202|2627|Sunday|1950Q2|N|N|N|2433403|2433522|2433044|2433319|N|N|N|N|N| +2433410|AAAAAAAACIBCFCAA|1950-05-08|604|2627|202|1950|1|5|8|2|1950|202|2627|Monday|1950Q2|N|N|N|2433403|2433522|2433045|2433320|N|N|N|N|N| +2433411|AAAAAAAADIBCFCAA|1950-05-09|604|2628|202|1950|2|5|9|2|1950|202|2628|Tuesday|1950Q2|N|N|N|2433403|2433522|2433046|2433321|N|N|N|N|N| +2433412|AAAAAAAAEIBCFCAA|1950-05-10|604|2628|202|1950|3|5|10|2|1950|202|2628|Wednesday|1950Q2|N|N|N|2433403|2433522|2433047|2433322|N|N|N|N|N| +2433413|AAAAAAAAFIBCFCAA|1950-05-11|604|2628|202|1950|4|5|11|2|1950|202|2628|Thursday|1950Q2|N|N|N|2433403|2433522|2433048|2433323|N|N|N|N|N| +2433414|AAAAAAAAGIBCFCAA|1950-05-12|604|2628|202|1950|5|5|12|2|1950|202|2628|Friday|1950Q2|N|Y|N|2433403|2433522|2433049|2433324|N|N|N|N|N| +2433415|AAAAAAAAHIBCFCAA|1950-05-13|604|2628|202|1950|6|5|13|2|1950|202|2628|Saturday|1950Q2|N|Y|N|2433403|2433522|2433050|2433325|N|N|N|N|N| +2433416|AAAAAAAAIIBCFCAA|1950-05-14|604|2628|202|1950|0|5|14|2|1950|202|2628|Sunday|1950Q2|N|N|N|2433403|2433522|2433051|2433326|N|N|N|N|N| +2433417|AAAAAAAAJIBCFCAA|1950-05-15|604|2628|202|1950|1|5|15|2|1950|202|2628|Monday|1950Q2|N|N|N|2433403|2433522|2433052|2433327|N|N|N|N|N| +2433418|AAAAAAAAKIBCFCAA|1950-05-16|604|2629|202|1950|2|5|16|2|1950|202|2629|Tuesday|1950Q2|N|N|N|2433403|2433522|2433053|2433328|N|N|N|N|N| +2433419|AAAAAAAALIBCFCAA|1950-05-17|604|2629|202|1950|3|5|17|2|1950|202|2629|Wednesday|1950Q2|N|N|N|2433403|2433522|2433054|2433329|N|N|N|N|N| +2433420|AAAAAAAAMIBCFCAA|1950-05-18|604|2629|202|1950|4|5|18|2|1950|202|2629|Thursday|1950Q2|N|N|N|2433403|2433522|2433055|2433330|N|N|N|N|N| +2433421|AAAAAAAANIBCFCAA|1950-05-19|604|2629|202|1950|5|5|19|2|1950|202|2629|Friday|1950Q2|N|Y|N|2433403|2433522|2433056|2433331|N|N|N|N|N| +2433422|AAAAAAAAOIBCFCAA|1950-05-20|604|2629|202|1950|6|5|20|2|1950|202|2629|Saturday|1950Q2|N|Y|N|2433403|2433522|2433057|2433332|N|N|N|N|N| +2433423|AAAAAAAAPIBCFCAA|1950-05-21|604|2629|202|1950|0|5|21|2|1950|202|2629|Sunday|1950Q2|N|N|N|2433403|2433522|2433058|2433333|N|N|N|N|N| +2433424|AAAAAAAAAJBCFCAA|1950-05-22|604|2629|202|1950|1|5|22|2|1950|202|2629|Monday|1950Q2|N|N|N|2433403|2433522|2433059|2433334|N|N|N|N|N| +2433425|AAAAAAAABJBCFCAA|1950-05-23|604|2630|202|1950|2|5|23|2|1950|202|2630|Tuesday|1950Q2|N|N|N|2433403|2433522|2433060|2433335|N|N|N|N|N| +2433426|AAAAAAAACJBCFCAA|1950-05-24|604|2630|202|1950|3|5|24|2|1950|202|2630|Wednesday|1950Q2|N|N|N|2433403|2433522|2433061|2433336|N|N|N|N|N| +2433427|AAAAAAAADJBCFCAA|1950-05-25|604|2630|202|1950|4|5|25|2|1950|202|2630|Thursday|1950Q2|N|N|N|2433403|2433522|2433062|2433337|N|N|N|N|N| +2433428|AAAAAAAAEJBCFCAA|1950-05-26|604|2630|202|1950|5|5|26|2|1950|202|2630|Friday|1950Q2|N|Y|N|2433403|2433522|2433063|2433338|N|N|N|N|N| +2433429|AAAAAAAAFJBCFCAA|1950-05-27|604|2630|202|1950|6|5|27|2|1950|202|2630|Saturday|1950Q2|N|Y|N|2433403|2433522|2433064|2433339|N|N|N|N|N| +2433430|AAAAAAAAGJBCFCAA|1950-05-28|604|2630|202|1950|0|5|28|2|1950|202|2630|Sunday|1950Q2|N|N|N|2433403|2433522|2433065|2433340|N|N|N|N|N| +2433431|AAAAAAAAHJBCFCAA|1950-05-29|604|2630|202|1950|1|5|29|2|1950|202|2630|Monday|1950Q2|N|N|N|2433403|2433522|2433066|2433341|N|N|N|N|N| +2433432|AAAAAAAAIJBCFCAA|1950-05-30|604|2631|202|1950|2|5|30|2|1950|202|2631|Tuesday|1950Q2|N|N|N|2433403|2433522|2433067|2433342|N|N|N|N|N| +2433433|AAAAAAAAJJBCFCAA|1950-05-31|604|2631|202|1950|3|5|31|2|1950|202|2631|Wednesday|1950Q2|N|N|N|2433403|2433522|2433068|2433343|N|N|N|N|N| +2433434|AAAAAAAAKJBCFCAA|1950-06-01|605|2631|203|1950|4|6|1|2|1950|203|2631|Thursday|1950Q2|N|N|N|2433434|2433584|2433069|2433344|N|N|N|N|N| +2433435|AAAAAAAALJBCFCAA|1950-06-02|605|2631|203|1950|5|6|2|2|1950|203|2631|Friday|1950Q2|N|Y|N|2433434|2433584|2433070|2433345|N|N|N|N|N| +2433436|AAAAAAAAMJBCFCAA|1950-06-03|605|2631|203|1950|6|6|3|2|1950|203|2631|Saturday|1950Q2|N|Y|N|2433434|2433584|2433071|2433346|N|N|N|N|N| +2433437|AAAAAAAANJBCFCAA|1950-06-04|605|2631|203|1950|0|6|4|2|1950|203|2631|Sunday|1950Q2|N|N|N|2433434|2433584|2433072|2433347|N|N|N|N|N| +2433438|AAAAAAAAOJBCFCAA|1950-06-05|605|2631|203|1950|1|6|5|2|1950|203|2631|Monday|1950Q2|N|N|N|2433434|2433584|2433073|2433348|N|N|N|N|N| +2433439|AAAAAAAAPJBCFCAA|1950-06-06|605|2632|203|1950|2|6|6|2|1950|203|2632|Tuesday|1950Q2|N|N|N|2433434|2433584|2433074|2433349|N|N|N|N|N| +2433440|AAAAAAAAAKBCFCAA|1950-06-07|605|2632|203|1950|3|6|7|2|1950|203|2632|Wednesday|1950Q2|N|N|N|2433434|2433584|2433075|2433350|N|N|N|N|N| +2433441|AAAAAAAABKBCFCAA|1950-06-08|605|2632|203|1950|4|6|8|2|1950|203|2632|Thursday|1950Q2|N|N|N|2433434|2433584|2433076|2433351|N|N|N|N|N| +2433442|AAAAAAAACKBCFCAA|1950-06-09|605|2632|203|1950|5|6|9|2|1950|203|2632|Friday|1950Q2|N|Y|N|2433434|2433584|2433077|2433352|N|N|N|N|N| +2433443|AAAAAAAADKBCFCAA|1950-06-10|605|2632|203|1950|6|6|10|2|1950|203|2632|Saturday|1950Q2|N|Y|N|2433434|2433584|2433078|2433353|N|N|N|N|N| +2433444|AAAAAAAAEKBCFCAA|1950-06-11|605|2632|203|1950|0|6|11|2|1950|203|2632|Sunday|1950Q2|N|N|N|2433434|2433584|2433079|2433354|N|N|N|N|N| +2433445|AAAAAAAAFKBCFCAA|1950-06-12|605|2632|203|1950|1|6|12|2|1950|203|2632|Monday|1950Q2|N|N|N|2433434|2433584|2433080|2433355|N|N|N|N|N| +2433446|AAAAAAAAGKBCFCAA|1950-06-13|605|2633|203|1950|2|6|13|2|1950|203|2633|Tuesday|1950Q2|N|N|N|2433434|2433584|2433081|2433356|N|N|N|N|N| +2433447|AAAAAAAAHKBCFCAA|1950-06-14|605|2633|203|1950|3|6|14|2|1950|203|2633|Wednesday|1950Q2|N|N|N|2433434|2433584|2433082|2433357|N|N|N|N|N| +2433448|AAAAAAAAIKBCFCAA|1950-06-15|605|2633|203|1950|4|6|15|2|1950|203|2633|Thursday|1950Q2|N|N|N|2433434|2433584|2433083|2433358|N|N|N|N|N| +2433449|AAAAAAAAJKBCFCAA|1950-06-16|605|2633|203|1950|5|6|16|2|1950|203|2633|Friday|1950Q2|N|Y|N|2433434|2433584|2433084|2433359|N|N|N|N|N| +2433450|AAAAAAAAKKBCFCAA|1950-06-17|605|2633|203|1950|6|6|17|2|1950|203|2633|Saturday|1950Q2|N|Y|N|2433434|2433584|2433085|2433360|N|N|N|N|N| +2433451|AAAAAAAALKBCFCAA|1950-06-18|605|2633|203|1950|0|6|18|2|1950|203|2633|Sunday|1950Q2|N|N|N|2433434|2433584|2433086|2433361|N|N|N|N|N| +2433452|AAAAAAAAMKBCFCAA|1950-06-19|605|2633|203|1950|1|6|19|2|1950|203|2633|Monday|1950Q2|N|N|N|2433434|2433584|2433087|2433362|N|N|N|N|N| +2433453|AAAAAAAANKBCFCAA|1950-06-20|605|2634|203|1950|2|6|20|2|1950|203|2634|Tuesday|1950Q2|N|N|N|2433434|2433584|2433088|2433363|N|N|N|N|N| +2433454|AAAAAAAAOKBCFCAA|1950-06-21|605|2634|203|1950|3|6|21|2|1950|203|2634|Wednesday|1950Q2|N|N|N|2433434|2433584|2433089|2433364|N|N|N|N|N| +2433455|AAAAAAAAPKBCFCAA|1950-06-22|605|2634|203|1950|4|6|22|2|1950|203|2634|Thursday|1950Q2|N|N|N|2433434|2433584|2433090|2433365|N|N|N|N|N| +2433456|AAAAAAAAALBCFCAA|1950-06-23|605|2634|203|1950|5|6|23|2|1950|203|2634|Friday|1950Q2|N|Y|N|2433434|2433584|2433091|2433366|N|N|N|N|N| +2433457|AAAAAAAABLBCFCAA|1950-06-24|605|2634|203|1950|6|6|24|2|1950|203|2634|Saturday|1950Q2|N|Y|N|2433434|2433584|2433092|2433367|N|N|N|N|N| +2433458|AAAAAAAACLBCFCAA|1950-06-25|605|2634|203|1950|0|6|25|2|1950|203|2634|Sunday|1950Q2|N|N|N|2433434|2433584|2433093|2433368|N|N|N|N|N| +2433459|AAAAAAAADLBCFCAA|1950-06-26|605|2634|203|1950|1|6|26|2|1950|203|2634|Monday|1950Q2|N|N|N|2433434|2433584|2433094|2433369|N|N|N|N|N| +2433460|AAAAAAAAELBCFCAA|1950-06-27|605|2635|203|1950|2|6|27|2|1950|203|2635|Tuesday|1950Q2|N|N|N|2433434|2433584|2433095|2433370|N|N|N|N|N| +2433461|AAAAAAAAFLBCFCAA|1950-06-28|605|2635|203|1950|3|6|28|2|1950|203|2635|Wednesday|1950Q2|N|N|N|2433434|2433584|2433096|2433371|N|N|N|N|N| +2433462|AAAAAAAAGLBCFCAA|1950-06-29|605|2635|203|1950|4|6|29|2|1950|203|2635|Thursday|1950Q2|N|N|N|2433434|2433584|2433097|2433372|N|N|N|N|N| +2433463|AAAAAAAAHLBCFCAA|1950-06-30|605|2635|203|1950|5|6|30|2|1950|203|2635|Friday|1950Q2|N|Y|N|2433434|2433584|2433098|2433373|N|N|N|N|N| +2433464|AAAAAAAAILBCFCAA|1950-07-01|606|2635|203|1950|6|7|1|2|1950|203|2635|Saturday|1950Q2|N|Y|N|2433464|2433644|2433099|2433373|N|N|N|N|N| +2433465|AAAAAAAAJLBCFCAA|1950-07-02|606|2635|203|1950|0|7|2|3|1950|203|2635|Sunday|1950Q3|N|N|N|2433464|2433644|2433100|2433374|N|N|N|N|N| +2433466|AAAAAAAAKLBCFCAA|1950-07-03|606|2635|203|1950|1|7|3|3|1950|203|2635|Monday|1950Q3|N|N|N|2433464|2433644|2433101|2433375|N|N|N|N|N| +2433467|AAAAAAAALLBCFCAA|1950-07-04|606|2636|203|1950|2|7|4|3|1950|203|2636|Tuesday|1950Q3|N|N|N|2433464|2433644|2433102|2433376|N|N|N|N|N| +2433468|AAAAAAAAMLBCFCAA|1950-07-05|606|2636|203|1950|3|7|5|3|1950|203|2636|Wednesday|1950Q3|Y|N|N|2433464|2433644|2433103|2433377|N|N|N|N|N| +2433469|AAAAAAAANLBCFCAA|1950-07-06|606|2636|203|1950|4|7|6|3|1950|203|2636|Thursday|1950Q3|N|N|Y|2433464|2433644|2433104|2433378|N|N|N|N|N| +2433470|AAAAAAAAOLBCFCAA|1950-07-07|606|2636|203|1950|5|7|7|3|1950|203|2636|Friday|1950Q3|N|Y|N|2433464|2433644|2433105|2433379|N|N|N|N|N| +2433471|AAAAAAAAPLBCFCAA|1950-07-08|606|2636|203|1950|6|7|8|3|1950|203|2636|Saturday|1950Q3|N|Y|N|2433464|2433644|2433106|2433380|N|N|N|N|N| +2433472|AAAAAAAAAMBCFCAA|1950-07-09|606|2636|203|1950|0|7|9|3|1950|203|2636|Sunday|1950Q3|N|N|N|2433464|2433644|2433107|2433381|N|N|N|N|N| +2433473|AAAAAAAABMBCFCAA|1950-07-10|606|2636|203|1950|1|7|10|3|1950|203|2636|Monday|1950Q3|N|N|N|2433464|2433644|2433108|2433382|N|N|N|N|N| +2433474|AAAAAAAACMBCFCAA|1950-07-11|606|2637|203|1950|2|7|11|3|1950|203|2637|Tuesday|1950Q3|N|N|N|2433464|2433644|2433109|2433383|N|N|N|N|N| +2433475|AAAAAAAADMBCFCAA|1950-07-12|606|2637|203|1950|3|7|12|3|1950|203|2637|Wednesday|1950Q3|N|N|N|2433464|2433644|2433110|2433384|N|N|N|N|N| +2433476|AAAAAAAAEMBCFCAA|1950-07-13|606|2637|203|1950|4|7|13|3|1950|203|2637|Thursday|1950Q3|N|N|N|2433464|2433644|2433111|2433385|N|N|N|N|N| +2433477|AAAAAAAAFMBCFCAA|1950-07-14|606|2637|203|1950|5|7|14|3|1950|203|2637|Friday|1950Q3|N|Y|N|2433464|2433644|2433112|2433386|N|N|N|N|N| +2433478|AAAAAAAAGMBCFCAA|1950-07-15|606|2637|203|1950|6|7|15|3|1950|203|2637|Saturday|1950Q3|N|Y|N|2433464|2433644|2433113|2433387|N|N|N|N|N| +2433479|AAAAAAAAHMBCFCAA|1950-07-16|606|2637|203|1950|0|7|16|3|1950|203|2637|Sunday|1950Q3|N|N|N|2433464|2433644|2433114|2433388|N|N|N|N|N| +2433480|AAAAAAAAIMBCFCAA|1950-07-17|606|2637|203|1950|1|7|17|3|1950|203|2637|Monday|1950Q3|N|N|N|2433464|2433644|2433115|2433389|N|N|N|N|N| +2433481|AAAAAAAAJMBCFCAA|1950-07-18|606|2638|203|1950|2|7|18|3|1950|203|2638|Tuesday|1950Q3|N|N|N|2433464|2433644|2433116|2433390|N|N|N|N|N| +2433482|AAAAAAAAKMBCFCAA|1950-07-19|606|2638|203|1950|3|7|19|3|1950|203|2638|Wednesday|1950Q3|N|N|N|2433464|2433644|2433117|2433391|N|N|N|N|N| +2433483|AAAAAAAALMBCFCAA|1950-07-20|606|2638|203|1950|4|7|20|3|1950|203|2638|Thursday|1950Q3|N|N|N|2433464|2433644|2433118|2433392|N|N|N|N|N| +2433484|AAAAAAAAMMBCFCAA|1950-07-21|606|2638|203|1950|5|7|21|3|1950|203|2638|Friday|1950Q3|N|Y|N|2433464|2433644|2433119|2433393|N|N|N|N|N| +2433485|AAAAAAAANMBCFCAA|1950-07-22|606|2638|203|1950|6|7|22|3|1950|203|2638|Saturday|1950Q3|N|Y|N|2433464|2433644|2433120|2433394|N|N|N|N|N| +2433486|AAAAAAAAOMBCFCAA|1950-07-23|606|2638|203|1950|0|7|23|3|1950|203|2638|Sunday|1950Q3|N|N|N|2433464|2433644|2433121|2433395|N|N|N|N|N| +2433487|AAAAAAAAPMBCFCAA|1950-07-24|606|2638|203|1950|1|7|24|3|1950|203|2638|Monday|1950Q3|N|N|N|2433464|2433644|2433122|2433396|N|N|N|N|N| +2433488|AAAAAAAAANBCFCAA|1950-07-25|606|2639|203|1950|2|7|25|3|1950|203|2639|Tuesday|1950Q3|N|N|N|2433464|2433644|2433123|2433397|N|N|N|N|N| +2433489|AAAAAAAABNBCFCAA|1950-07-26|606|2639|203|1950|3|7|26|3|1950|203|2639|Wednesday|1950Q3|N|N|N|2433464|2433644|2433124|2433398|N|N|N|N|N| +2433490|AAAAAAAACNBCFCAA|1950-07-27|606|2639|203|1950|4|7|27|3|1950|203|2639|Thursday|1950Q3|N|N|N|2433464|2433644|2433125|2433399|N|N|N|N|N| +2433491|AAAAAAAADNBCFCAA|1950-07-28|606|2639|203|1950|5|7|28|3|1950|203|2639|Friday|1950Q3|N|Y|N|2433464|2433644|2433126|2433400|N|N|N|N|N| +2433492|AAAAAAAAENBCFCAA|1950-07-29|606|2639|203|1950|6|7|29|3|1950|203|2639|Saturday|1950Q3|N|Y|N|2433464|2433644|2433127|2433401|N|N|N|N|N| +2433493|AAAAAAAAFNBCFCAA|1950-07-30|606|2639|203|1950|0|7|30|3|1950|203|2639|Sunday|1950Q3|N|N|N|2433464|2433644|2433128|2433402|N|N|N|N|N| +2433494|AAAAAAAAGNBCFCAA|1950-07-31|606|2639|203|1950|1|7|31|3|1950|203|2639|Monday|1950Q3|N|N|N|2433464|2433644|2433129|2433403|N|N|N|N|N| +2433495|AAAAAAAAHNBCFCAA|1950-08-01|607|2640|203|1950|2|8|1|3|1950|203|2640|Tuesday|1950Q3|N|N|N|2433495|2433706|2433130|2433404|N|N|N|N|N| +2433496|AAAAAAAAINBCFCAA|1950-08-02|607|2640|203|1950|3|8|2|3|1950|203|2640|Wednesday|1950Q3|N|N|N|2433495|2433706|2433131|2433405|N|N|N|N|N| +2433497|AAAAAAAAJNBCFCAA|1950-08-03|607|2640|203|1950|4|8|3|3|1950|203|2640|Thursday|1950Q3|N|N|N|2433495|2433706|2433132|2433406|N|N|N|N|N| +2433498|AAAAAAAAKNBCFCAA|1950-08-04|607|2640|203|1950|5|8|4|3|1950|203|2640|Friday|1950Q3|N|Y|N|2433495|2433706|2433133|2433407|N|N|N|N|N| +2433499|AAAAAAAALNBCFCAA|1950-08-05|607|2640|203|1950|6|8|5|3|1950|203|2640|Saturday|1950Q3|N|Y|N|2433495|2433706|2433134|2433408|N|N|N|N|N| +2433500|AAAAAAAAMNBCFCAA|1950-08-06|607|2640|203|1950|0|8|6|3|1950|203|2640|Sunday|1950Q3|N|N|N|2433495|2433706|2433135|2433409|N|N|N|N|N| +2433501|AAAAAAAANNBCFCAA|1950-08-07|607|2640|203|1950|1|8|7|3|1950|203|2640|Monday|1950Q3|N|N|N|2433495|2433706|2433136|2433410|N|N|N|N|N| +2433502|AAAAAAAAONBCFCAA|1950-08-08|607|2641|203|1950|2|8|8|3|1950|203|2641|Tuesday|1950Q3|N|N|N|2433495|2433706|2433137|2433411|N|N|N|N|N| +2433503|AAAAAAAAPNBCFCAA|1950-08-09|607|2641|203|1950|3|8|9|3|1950|203|2641|Wednesday|1950Q3|N|N|N|2433495|2433706|2433138|2433412|N|N|N|N|N| +2433504|AAAAAAAAAOBCFCAA|1950-08-10|607|2641|203|1950|4|8|10|3|1950|203|2641|Thursday|1950Q3|N|N|N|2433495|2433706|2433139|2433413|N|N|N|N|N| +2433505|AAAAAAAABOBCFCAA|1950-08-11|607|2641|203|1950|5|8|11|3|1950|203|2641|Friday|1950Q3|N|Y|N|2433495|2433706|2433140|2433414|N|N|N|N|N| +2433506|AAAAAAAACOBCFCAA|1950-08-12|607|2641|203|1950|6|8|12|3|1950|203|2641|Saturday|1950Q3|N|Y|N|2433495|2433706|2433141|2433415|N|N|N|N|N| +2433507|AAAAAAAADOBCFCAA|1950-08-13|607|2641|203|1950|0|8|13|3|1950|203|2641|Sunday|1950Q3|N|N|N|2433495|2433706|2433142|2433416|N|N|N|N|N| +2433508|AAAAAAAAEOBCFCAA|1950-08-14|607|2641|203|1950|1|8|14|3|1950|203|2641|Monday|1950Q3|N|N|N|2433495|2433706|2433143|2433417|N|N|N|N|N| +2433509|AAAAAAAAFOBCFCAA|1950-08-15|607|2642|203|1950|2|8|15|3|1950|203|2642|Tuesday|1950Q3|N|N|N|2433495|2433706|2433144|2433418|N|N|N|N|N| +2433510|AAAAAAAAGOBCFCAA|1950-08-16|607|2642|203|1950|3|8|16|3|1950|203|2642|Wednesday|1950Q3|N|N|N|2433495|2433706|2433145|2433419|N|N|N|N|N| +2433511|AAAAAAAAHOBCFCAA|1950-08-17|607|2642|203|1950|4|8|17|3|1950|203|2642|Thursday|1950Q3|N|N|N|2433495|2433706|2433146|2433420|N|N|N|N|N| +2433512|AAAAAAAAIOBCFCAA|1950-08-18|607|2642|203|1950|5|8|18|3|1950|203|2642|Friday|1950Q3|N|Y|N|2433495|2433706|2433147|2433421|N|N|N|N|N| +2433513|AAAAAAAAJOBCFCAA|1950-08-19|607|2642|203|1950|6|8|19|3|1950|203|2642|Saturday|1950Q3|N|Y|N|2433495|2433706|2433148|2433422|N|N|N|N|N| +2433514|AAAAAAAAKOBCFCAA|1950-08-20|607|2642|203|1950|0|8|20|3|1950|203|2642|Sunday|1950Q3|N|N|N|2433495|2433706|2433149|2433423|N|N|N|N|N| +2433515|AAAAAAAALOBCFCAA|1950-08-21|607|2642|203|1950|1|8|21|3|1950|203|2642|Monday|1950Q3|N|N|N|2433495|2433706|2433150|2433424|N|N|N|N|N| +2433516|AAAAAAAAMOBCFCAA|1950-08-22|607|2643|203|1950|2|8|22|3|1950|203|2643|Tuesday|1950Q3|N|N|N|2433495|2433706|2433151|2433425|N|N|N|N|N| +2433517|AAAAAAAANOBCFCAA|1950-08-23|607|2643|203|1950|3|8|23|3|1950|203|2643|Wednesday|1950Q3|N|N|N|2433495|2433706|2433152|2433426|N|N|N|N|N| +2433518|AAAAAAAAOOBCFCAA|1950-08-24|607|2643|203|1950|4|8|24|3|1950|203|2643|Thursday|1950Q3|N|N|N|2433495|2433706|2433153|2433427|N|N|N|N|N| +2433519|AAAAAAAAPOBCFCAA|1950-08-25|607|2643|203|1950|5|8|25|3|1950|203|2643|Friday|1950Q3|N|Y|N|2433495|2433706|2433154|2433428|N|N|N|N|N| +2433520|AAAAAAAAAPBCFCAA|1950-08-26|607|2643|203|1950|6|8|26|3|1950|203|2643|Saturday|1950Q3|N|Y|N|2433495|2433706|2433155|2433429|N|N|N|N|N| +2433521|AAAAAAAABPBCFCAA|1950-08-27|607|2643|203|1950|0|8|27|3|1950|203|2643|Sunday|1950Q3|N|N|N|2433495|2433706|2433156|2433430|N|N|N|N|N| +2433522|AAAAAAAACPBCFCAA|1950-08-28|607|2643|203|1950|1|8|28|3|1950|203|2643|Monday|1950Q3|N|N|N|2433495|2433706|2433157|2433431|N|N|N|N|N| +2433523|AAAAAAAADPBCFCAA|1950-08-29|607|2644|203|1950|2|8|29|3|1950|203|2644|Tuesday|1950Q3|N|N|N|2433495|2433706|2433158|2433432|N|N|N|N|N| +2433524|AAAAAAAAEPBCFCAA|1950-08-30|607|2644|203|1950|3|8|30|3|1950|203|2644|Wednesday|1950Q3|N|N|N|2433495|2433706|2433159|2433433|N|N|N|N|N| +2433525|AAAAAAAAFPBCFCAA|1950-08-31|607|2644|203|1950|4|8|31|3|1950|203|2644|Thursday|1950Q3|N|N|N|2433495|2433706|2433160|2433434|N|N|N|N|N| +2433526|AAAAAAAAGPBCFCAA|1950-09-01|608|2644|204|1950|5|9|1|3|1950|204|2644|Friday|1950Q3|N|Y|N|2433526|2433768|2433161|2433435|N|N|N|N|N| +2433527|AAAAAAAAHPBCFCAA|1950-09-02|608|2644|204|1950|6|9|2|3|1950|204|2644|Saturday|1950Q3|N|Y|N|2433526|2433768|2433162|2433436|N|N|N|N|N| +2433528|AAAAAAAAIPBCFCAA|1950-09-03|608|2644|204|1950|0|9|3|3|1950|204|2644|Sunday|1950Q3|N|N|N|2433526|2433768|2433163|2433437|N|N|N|N|N| +2433529|AAAAAAAAJPBCFCAA|1950-09-04|608|2644|204|1950|1|9|4|3|1950|204|2644|Monday|1950Q3|N|N|N|2433526|2433768|2433164|2433438|N|N|N|N|N| +2433530|AAAAAAAAKPBCFCAA|1950-09-05|608|2645|204|1950|2|9|5|3|1950|204|2645|Tuesday|1950Q3|N|N|N|2433526|2433768|2433165|2433439|N|N|N|N|N| +2433531|AAAAAAAALPBCFCAA|1950-09-06|608|2645|204|1950|3|9|6|3|1950|204|2645|Wednesday|1950Q3|N|N|N|2433526|2433768|2433166|2433440|N|N|N|N|N| +2433532|AAAAAAAAMPBCFCAA|1950-09-07|608|2645|204|1950|4|9|7|3|1950|204|2645|Thursday|1950Q3|N|N|N|2433526|2433768|2433167|2433441|N|N|N|N|N| +2433533|AAAAAAAANPBCFCAA|1950-09-08|608|2645|204|1950|5|9|8|3|1950|204|2645|Friday|1950Q3|N|Y|N|2433526|2433768|2433168|2433442|N|N|N|N|N| +2433534|AAAAAAAAOPBCFCAA|1950-09-09|608|2645|204|1950|6|9|9|3|1950|204|2645|Saturday|1950Q3|N|Y|N|2433526|2433768|2433169|2433443|N|N|N|N|N| +2433535|AAAAAAAAPPBCFCAA|1950-09-10|608|2645|204|1950|0|9|10|3|1950|204|2645|Sunday|1950Q3|N|N|N|2433526|2433768|2433170|2433444|N|N|N|N|N| +2433536|AAAAAAAAAACCFCAA|1950-09-11|608|2645|204|1950|1|9|11|3|1950|204|2645|Monday|1950Q3|N|N|N|2433526|2433768|2433171|2433445|N|N|N|N|N| +2433537|AAAAAAAABACCFCAA|1950-09-12|608|2646|204|1950|2|9|12|3|1950|204|2646|Tuesday|1950Q3|N|N|N|2433526|2433768|2433172|2433446|N|N|N|N|N| +2433538|AAAAAAAACACCFCAA|1950-09-13|608|2646|204|1950|3|9|13|3|1950|204|2646|Wednesday|1950Q3|N|N|N|2433526|2433768|2433173|2433447|N|N|N|N|N| +2433539|AAAAAAAADACCFCAA|1950-09-14|608|2646|204|1950|4|9|14|3|1950|204|2646|Thursday|1950Q3|N|N|N|2433526|2433768|2433174|2433448|N|N|N|N|N| +2433540|AAAAAAAAEACCFCAA|1950-09-15|608|2646|204|1950|5|9|15|3|1950|204|2646|Friday|1950Q3|N|Y|N|2433526|2433768|2433175|2433449|N|N|N|N|N| +2433541|AAAAAAAAFACCFCAA|1950-09-16|608|2646|204|1950|6|9|16|3|1950|204|2646|Saturday|1950Q3|N|Y|N|2433526|2433768|2433176|2433450|N|N|N|N|N| +2433542|AAAAAAAAGACCFCAA|1950-09-17|608|2646|204|1950|0|9|17|3|1950|204|2646|Sunday|1950Q3|N|N|N|2433526|2433768|2433177|2433451|N|N|N|N|N| +2433543|AAAAAAAAHACCFCAA|1950-09-18|608|2646|204|1950|1|9|18|3|1950|204|2646|Monday|1950Q3|N|N|N|2433526|2433768|2433178|2433452|N|N|N|N|N| +2433544|AAAAAAAAIACCFCAA|1950-09-19|608|2647|204|1950|2|9|19|3|1950|204|2647|Tuesday|1950Q3|N|N|N|2433526|2433768|2433179|2433453|N|N|N|N|N| +2433545|AAAAAAAAJACCFCAA|1950-09-20|608|2647|204|1950|3|9|20|3|1950|204|2647|Wednesday|1950Q3|N|N|N|2433526|2433768|2433180|2433454|N|N|N|N|N| +2433546|AAAAAAAAKACCFCAA|1950-09-21|608|2647|204|1950|4|9|21|3|1950|204|2647|Thursday|1950Q3|N|N|N|2433526|2433768|2433181|2433455|N|N|N|N|N| +2433547|AAAAAAAALACCFCAA|1950-09-22|608|2647|204|1950|5|9|22|3|1950|204|2647|Friday|1950Q3|N|Y|N|2433526|2433768|2433182|2433456|N|N|N|N|N| +2433548|AAAAAAAAMACCFCAA|1950-09-23|608|2647|204|1950|6|9|23|3|1950|204|2647|Saturday|1950Q3|N|Y|N|2433526|2433768|2433183|2433457|N|N|N|N|N| +2433549|AAAAAAAANACCFCAA|1950-09-24|608|2647|204|1950|0|9|24|3|1950|204|2647|Sunday|1950Q3|N|N|N|2433526|2433768|2433184|2433458|N|N|N|N|N| +2433550|AAAAAAAAOACCFCAA|1950-09-25|608|2647|204|1950|1|9|25|3|1950|204|2647|Monday|1950Q3|N|N|N|2433526|2433768|2433185|2433459|N|N|N|N|N| +2433551|AAAAAAAAPACCFCAA|1950-09-26|608|2648|204|1950|2|9|26|3|1950|204|2648|Tuesday|1950Q3|N|N|N|2433526|2433768|2433186|2433460|N|N|N|N|N| +2433552|AAAAAAAAABCCFCAA|1950-09-27|608|2648|204|1950|3|9|27|3|1950|204|2648|Wednesday|1950Q3|N|N|N|2433526|2433768|2433187|2433461|N|N|N|N|N| +2433553|AAAAAAAABBCCFCAA|1950-09-28|608|2648|204|1950|4|9|28|3|1950|204|2648|Thursday|1950Q3|N|N|N|2433526|2433768|2433188|2433462|N|N|N|N|N| +2433554|AAAAAAAACBCCFCAA|1950-09-29|608|2648|204|1950|5|9|29|3|1950|204|2648|Friday|1950Q3|N|Y|N|2433526|2433768|2433189|2433463|N|N|N|N|N| +2433555|AAAAAAAADBCCFCAA|1950-09-30|608|2648|204|1950|6|9|30|3|1950|204|2648|Saturday|1950Q3|N|Y|N|2433526|2433768|2433190|2433464|N|N|N|N|N| +2433556|AAAAAAAAEBCCFCAA|1950-10-01|609|2648|204|1950|0|10|1|3|1950|204|2648|Sunday|1950Q3|N|N|N|2433556|2433828|2433191|2433464|N|N|N|N|N| +2433557|AAAAAAAAFBCCFCAA|1950-10-02|609|2648|204|1950|1|10|2|4|1950|204|2648|Monday|1950Q4|N|N|N|2433556|2433828|2433192|2433465|N|N|N|N|N| +2433558|AAAAAAAAGBCCFCAA|1950-10-03|609|2649|204|1950|2|10|3|4|1950|204|2649|Tuesday|1950Q4|N|N|N|2433556|2433828|2433193|2433466|N|N|N|N|N| +2433559|AAAAAAAAHBCCFCAA|1950-10-04|609|2649|204|1950|3|10|4|4|1950|204|2649|Wednesday|1950Q4|N|N|N|2433556|2433828|2433194|2433467|N|N|N|N|N| +2433560|AAAAAAAAIBCCFCAA|1950-10-05|609|2649|204|1950|4|10|5|4|1950|204|2649|Thursday|1950Q4|N|N|N|2433556|2433828|2433195|2433468|N|N|N|N|N| +2433561|AAAAAAAAJBCCFCAA|1950-10-06|609|2649|204|1950|5|10|6|4|1950|204|2649|Friday|1950Q4|N|Y|N|2433556|2433828|2433196|2433469|N|N|N|N|N| +2433562|AAAAAAAAKBCCFCAA|1950-10-07|609|2649|204|1950|6|10|7|4|1950|204|2649|Saturday|1950Q4|N|Y|N|2433556|2433828|2433197|2433470|N|N|N|N|N| +2433563|AAAAAAAALBCCFCAA|1950-10-08|609|2649|204|1950|0|10|8|4|1950|204|2649|Sunday|1950Q4|N|N|N|2433556|2433828|2433198|2433471|N|N|N|N|N| +2433564|AAAAAAAAMBCCFCAA|1950-10-09|609|2649|204|1950|1|10|9|4|1950|204|2649|Monday|1950Q4|N|N|N|2433556|2433828|2433199|2433472|N|N|N|N|N| +2433565|AAAAAAAANBCCFCAA|1950-10-10|609|2650|204|1950|2|10|10|4|1950|204|2650|Tuesday|1950Q4|N|N|N|2433556|2433828|2433200|2433473|N|N|N|N|N| +2433566|AAAAAAAAOBCCFCAA|1950-10-11|609|2650|204|1950|3|10|11|4|1950|204|2650|Wednesday|1950Q4|N|N|N|2433556|2433828|2433201|2433474|N|N|N|N|N| +2433567|AAAAAAAAPBCCFCAA|1950-10-12|609|2650|204|1950|4|10|12|4|1950|204|2650|Thursday|1950Q4|N|N|N|2433556|2433828|2433202|2433475|N|N|N|N|N| +2433568|AAAAAAAAACCCFCAA|1950-10-13|609|2650|204|1950|5|10|13|4|1950|204|2650|Friday|1950Q4|N|Y|N|2433556|2433828|2433203|2433476|N|N|N|N|N| +2433569|AAAAAAAABCCCFCAA|1950-10-14|609|2650|204|1950|6|10|14|4|1950|204|2650|Saturday|1950Q4|N|Y|N|2433556|2433828|2433204|2433477|N|N|N|N|N| +2433570|AAAAAAAACCCCFCAA|1950-10-15|609|2650|204|1950|0|10|15|4|1950|204|2650|Sunday|1950Q4|N|N|N|2433556|2433828|2433205|2433478|N|N|N|N|N| +2433571|AAAAAAAADCCCFCAA|1950-10-16|609|2650|204|1950|1|10|16|4|1950|204|2650|Monday|1950Q4|N|N|N|2433556|2433828|2433206|2433479|N|N|N|N|N| +2433572|AAAAAAAAECCCFCAA|1950-10-17|609|2651|204|1950|2|10|17|4|1950|204|2651|Tuesday|1950Q4|N|N|N|2433556|2433828|2433207|2433480|N|N|N|N|N| +2433573|AAAAAAAAFCCCFCAA|1950-10-18|609|2651|204|1950|3|10|18|4|1950|204|2651|Wednesday|1950Q4|N|N|N|2433556|2433828|2433208|2433481|N|N|N|N|N| +2433574|AAAAAAAAGCCCFCAA|1950-10-19|609|2651|204|1950|4|10|19|4|1950|204|2651|Thursday|1950Q4|N|N|N|2433556|2433828|2433209|2433482|N|N|N|N|N| +2433575|AAAAAAAAHCCCFCAA|1950-10-20|609|2651|204|1950|5|10|20|4|1950|204|2651|Friday|1950Q4|N|Y|N|2433556|2433828|2433210|2433483|N|N|N|N|N| +2433576|AAAAAAAAICCCFCAA|1950-10-21|609|2651|204|1950|6|10|21|4|1950|204|2651|Saturday|1950Q4|N|Y|N|2433556|2433828|2433211|2433484|N|N|N|N|N| +2433577|AAAAAAAAJCCCFCAA|1950-10-22|609|2651|204|1950|0|10|22|4|1950|204|2651|Sunday|1950Q4|N|N|N|2433556|2433828|2433212|2433485|N|N|N|N|N| +2433578|AAAAAAAAKCCCFCAA|1950-10-23|609|2651|204|1950|1|10|23|4|1950|204|2651|Monday|1950Q4|N|N|N|2433556|2433828|2433213|2433486|N|N|N|N|N| +2433579|AAAAAAAALCCCFCAA|1950-10-24|609|2652|204|1950|2|10|24|4|1950|204|2652|Tuesday|1950Q4|N|N|N|2433556|2433828|2433214|2433487|N|N|N|N|N| +2433580|AAAAAAAAMCCCFCAA|1950-10-25|609|2652|204|1950|3|10|25|4|1950|204|2652|Wednesday|1950Q4|N|N|N|2433556|2433828|2433215|2433488|N|N|N|N|N| +2433581|AAAAAAAANCCCFCAA|1950-10-26|609|2652|204|1950|4|10|26|4|1950|204|2652|Thursday|1950Q4|N|N|N|2433556|2433828|2433216|2433489|N|N|N|N|N| +2433582|AAAAAAAAOCCCFCAA|1950-10-27|609|2652|204|1950|5|10|27|4|1950|204|2652|Friday|1950Q4|N|Y|N|2433556|2433828|2433217|2433490|N|N|N|N|N| +2433583|AAAAAAAAPCCCFCAA|1950-10-28|609|2652|204|1950|6|10|28|4|1950|204|2652|Saturday|1950Q4|N|Y|N|2433556|2433828|2433218|2433491|N|N|N|N|N| +2433584|AAAAAAAAADCCFCAA|1950-10-29|609|2652|204|1950|0|10|29|4|1950|204|2652|Sunday|1950Q4|N|N|N|2433556|2433828|2433219|2433492|N|N|N|N|N| +2433585|AAAAAAAABDCCFCAA|1950-10-30|609|2652|204|1950|1|10|30|4|1950|204|2652|Monday|1950Q4|N|N|N|2433556|2433828|2433220|2433493|N|N|N|N|N| +2433586|AAAAAAAACDCCFCAA|1950-10-31|609|2653|204|1950|2|10|31|4|1950|204|2653|Tuesday|1950Q4|N|N|N|2433556|2433828|2433221|2433494|N|N|N|N|N| +2433587|AAAAAAAADDCCFCAA|1950-11-01|610|2653|204|1950|3|11|1|4|1950|204|2653|Wednesday|1950Q4|N|N|N|2433587|2433890|2433222|2433495|N|N|N|N|N| +2433588|AAAAAAAAEDCCFCAA|1950-11-02|610|2653|204|1950|4|11|2|4|1950|204|2653|Thursday|1950Q4|N|N|N|2433587|2433890|2433223|2433496|N|N|N|N|N| +2433589|AAAAAAAAFDCCFCAA|1950-11-03|610|2653|204|1950|5|11|3|4|1950|204|2653|Friday|1950Q4|N|Y|N|2433587|2433890|2433224|2433497|N|N|N|N|N| +2433590|AAAAAAAAGDCCFCAA|1950-11-04|610|2653|204|1950|6|11|4|4|1950|204|2653|Saturday|1950Q4|N|Y|N|2433587|2433890|2433225|2433498|N|N|N|N|N| +2433591|AAAAAAAAHDCCFCAA|1950-11-05|610|2653|204|1950|0|11|5|4|1950|204|2653|Sunday|1950Q4|N|N|N|2433587|2433890|2433226|2433499|N|N|N|N|N| +2433592|AAAAAAAAIDCCFCAA|1950-11-06|610|2653|204|1950|1|11|6|4|1950|204|2653|Monday|1950Q4|N|N|N|2433587|2433890|2433227|2433500|N|N|N|N|N| +2433593|AAAAAAAAJDCCFCAA|1950-11-07|610|2654|204|1950|2|11|7|4|1950|204|2654|Tuesday|1950Q4|N|N|N|2433587|2433890|2433228|2433501|N|N|N|N|N| +2433594|AAAAAAAAKDCCFCAA|1950-11-08|610|2654|204|1950|3|11|8|4|1950|204|2654|Wednesday|1950Q4|N|N|N|2433587|2433890|2433229|2433502|N|N|N|N|N| +2433595|AAAAAAAALDCCFCAA|1950-11-09|610|2654|204|1950|4|11|9|4|1950|204|2654|Thursday|1950Q4|N|N|N|2433587|2433890|2433230|2433503|N|N|N|N|N| +2433596|AAAAAAAAMDCCFCAA|1950-11-10|610|2654|204|1950|5|11|10|4|1950|204|2654|Friday|1950Q4|N|Y|N|2433587|2433890|2433231|2433504|N|N|N|N|N| +2433597|AAAAAAAANDCCFCAA|1950-11-11|610|2654|204|1950|6|11|11|4|1950|204|2654|Saturday|1950Q4|N|Y|N|2433587|2433890|2433232|2433505|N|N|N|N|N| +2433598|AAAAAAAAODCCFCAA|1950-11-12|610|2654|204|1950|0|11|12|4|1950|204|2654|Sunday|1950Q4|N|N|N|2433587|2433890|2433233|2433506|N|N|N|N|N| +2433599|AAAAAAAAPDCCFCAA|1950-11-13|610|2654|204|1950|1|11|13|4|1950|204|2654|Monday|1950Q4|N|N|N|2433587|2433890|2433234|2433507|N|N|N|N|N| +2433600|AAAAAAAAAECCFCAA|1950-11-14|610|2655|204|1950|2|11|14|4|1950|204|2655|Tuesday|1950Q4|N|N|N|2433587|2433890|2433235|2433508|N|N|N|N|N| +2433601|AAAAAAAABECCFCAA|1950-11-15|610|2655|204|1950|3|11|15|4|1950|204|2655|Wednesday|1950Q4|N|N|N|2433587|2433890|2433236|2433509|N|N|N|N|N| +2433602|AAAAAAAACECCFCAA|1950-11-16|610|2655|204|1950|4|11|16|4|1950|204|2655|Thursday|1950Q4|N|N|N|2433587|2433890|2433237|2433510|N|N|N|N|N| +2433603|AAAAAAAADECCFCAA|1950-11-17|610|2655|204|1950|5|11|17|4|1950|204|2655|Friday|1950Q4|N|Y|N|2433587|2433890|2433238|2433511|N|N|N|N|N| +2433604|AAAAAAAAEECCFCAA|1950-11-18|610|2655|204|1950|6|11|18|4|1950|204|2655|Saturday|1950Q4|N|Y|N|2433587|2433890|2433239|2433512|N|N|N|N|N| +2433605|AAAAAAAAFECCFCAA|1950-11-19|610|2655|204|1950|0|11|19|4|1950|204|2655|Sunday|1950Q4|N|N|N|2433587|2433890|2433240|2433513|N|N|N|N|N| +2433606|AAAAAAAAGECCFCAA|1950-11-20|610|2655|204|1950|1|11|20|4|1950|204|2655|Monday|1950Q4|N|N|N|2433587|2433890|2433241|2433514|N|N|N|N|N| +2433607|AAAAAAAAHECCFCAA|1950-11-21|610|2656|204|1950|2|11|21|4|1950|204|2656|Tuesday|1950Q4|N|N|N|2433587|2433890|2433242|2433515|N|N|N|N|N| +2433608|AAAAAAAAIECCFCAA|1950-11-22|610|2656|204|1950|3|11|22|4|1950|204|2656|Wednesday|1950Q4|N|N|N|2433587|2433890|2433243|2433516|N|N|N|N|N| +2433609|AAAAAAAAJECCFCAA|1950-11-23|610|2656|204|1950|4|11|23|4|1950|204|2656|Thursday|1950Q4|N|N|N|2433587|2433890|2433244|2433517|N|N|N|N|N| +2433610|AAAAAAAAKECCFCAA|1950-11-24|610|2656|204|1950|5|11|24|4|1950|204|2656|Friday|1950Q4|N|Y|N|2433587|2433890|2433245|2433518|N|N|N|N|N| +2433611|AAAAAAAALECCFCAA|1950-11-25|610|2656|204|1950|6|11|25|4|1950|204|2656|Saturday|1950Q4|N|Y|N|2433587|2433890|2433246|2433519|N|N|N|N|N| +2433612|AAAAAAAAMECCFCAA|1950-11-26|610|2656|204|1950|0|11|26|4|1950|204|2656|Sunday|1950Q4|N|N|N|2433587|2433890|2433247|2433520|N|N|N|N|N| +2433613|AAAAAAAANECCFCAA|1950-11-27|610|2656|204|1950|1|11|27|4|1950|204|2656|Monday|1950Q4|N|N|N|2433587|2433890|2433248|2433521|N|N|N|N|N| +2433614|AAAAAAAAOECCFCAA|1950-11-28|610|2657|204|1950|2|11|28|4|1950|204|2657|Tuesday|1950Q4|N|N|N|2433587|2433890|2433249|2433522|N|N|N|N|N| +2433615|AAAAAAAAPECCFCAA|1950-11-29|610|2657|204|1950|3|11|29|4|1950|204|2657|Wednesday|1950Q4|N|N|N|2433587|2433890|2433250|2433523|N|N|N|N|N| +2433616|AAAAAAAAAFCCFCAA|1950-11-30|610|2657|204|1950|4|11|30|4|1950|204|2657|Thursday|1950Q4|N|N|N|2433587|2433890|2433251|2433524|N|N|N|N|N| +2433617|AAAAAAAABFCCFCAA|1950-12-01|611|2657|205|1950|5|12|1|4|1950|205|2657|Friday|1950Q4|N|Y|N|2433617|2433950|2433252|2433525|N|N|N|N|N| +2433618|AAAAAAAACFCCFCAA|1950-12-02|611|2657|205|1950|6|12|2|4|1950|205|2657|Saturday|1950Q4|N|Y|N|2433617|2433950|2433253|2433526|N|N|N|N|N| +2433619|AAAAAAAADFCCFCAA|1950-12-03|611|2657|205|1950|0|12|3|4|1950|205|2657|Sunday|1950Q4|N|N|N|2433617|2433950|2433254|2433527|N|N|N|N|N| +2433620|AAAAAAAAEFCCFCAA|1950-12-04|611|2657|205|1950|1|12|4|4|1950|205|2657|Monday|1950Q4|N|N|N|2433617|2433950|2433255|2433528|N|N|N|N|N| +2433621|AAAAAAAAFFCCFCAA|1950-12-05|611|2658|205|1950|2|12|5|4|1950|205|2658|Tuesday|1950Q4|N|N|N|2433617|2433950|2433256|2433529|N|N|N|N|N| +2433622|AAAAAAAAGFCCFCAA|1950-12-06|611|2658|205|1950|3|12|6|4|1950|205|2658|Wednesday|1950Q4|N|N|N|2433617|2433950|2433257|2433530|N|N|N|N|N| +2433623|AAAAAAAAHFCCFCAA|1950-12-07|611|2658|205|1950|4|12|7|4|1950|205|2658|Thursday|1950Q4|N|N|N|2433617|2433950|2433258|2433531|N|N|N|N|N| +2433624|AAAAAAAAIFCCFCAA|1950-12-08|611|2658|205|1950|5|12|8|4|1950|205|2658|Friday|1950Q4|N|Y|N|2433617|2433950|2433259|2433532|N|N|N|N|N| +2433625|AAAAAAAAJFCCFCAA|1950-12-09|611|2658|205|1950|6|12|9|4|1950|205|2658|Saturday|1950Q4|N|Y|N|2433617|2433950|2433260|2433533|N|N|N|N|N| +2433626|AAAAAAAAKFCCFCAA|1950-12-10|611|2658|205|1950|0|12|10|4|1950|205|2658|Sunday|1950Q4|N|N|N|2433617|2433950|2433261|2433534|N|N|N|N|N| +2433627|AAAAAAAALFCCFCAA|1950-12-11|611|2658|205|1950|1|12|11|4|1950|205|2658|Monday|1950Q4|N|N|N|2433617|2433950|2433262|2433535|N|N|N|N|N| +2433628|AAAAAAAAMFCCFCAA|1950-12-12|611|2659|205|1950|2|12|12|4|1950|205|2659|Tuesday|1950Q4|N|N|N|2433617|2433950|2433263|2433536|N|N|N|N|N| +2433629|AAAAAAAANFCCFCAA|1950-12-13|611|2659|205|1950|3|12|13|4|1950|205|2659|Wednesday|1950Q4|N|N|N|2433617|2433950|2433264|2433537|N|N|N|N|N| +2433630|AAAAAAAAOFCCFCAA|1950-12-14|611|2659|205|1950|4|12|14|4|1950|205|2659|Thursday|1950Q4|N|N|N|2433617|2433950|2433265|2433538|N|N|N|N|N| +2433631|AAAAAAAAPFCCFCAA|1950-12-15|611|2659|205|1950|5|12|15|4|1950|205|2659|Friday|1950Q4|N|Y|N|2433617|2433950|2433266|2433539|N|N|N|N|N| +2433632|AAAAAAAAAGCCFCAA|1950-12-16|611|2659|205|1950|6|12|16|4|1950|205|2659|Saturday|1950Q4|N|Y|N|2433617|2433950|2433267|2433540|N|N|N|N|N| +2433633|AAAAAAAABGCCFCAA|1950-12-17|611|2659|205|1950|0|12|17|4|1950|205|2659|Sunday|1950Q4|N|N|N|2433617|2433950|2433268|2433541|N|N|N|N|N| +2433634|AAAAAAAACGCCFCAA|1950-12-18|611|2659|205|1950|1|12|18|4|1950|205|2659|Monday|1950Q4|N|N|N|2433617|2433950|2433269|2433542|N|N|N|N|N| +2433635|AAAAAAAADGCCFCAA|1950-12-19|611|2660|205|1950|2|12|19|4|1950|205|2660|Tuesday|1950Q4|N|N|N|2433617|2433950|2433270|2433543|N|N|N|N|N| +2433636|AAAAAAAAEGCCFCAA|1950-12-20|611|2660|205|1950|3|12|20|4|1950|205|2660|Wednesday|1950Q4|N|N|N|2433617|2433950|2433271|2433544|N|N|N|N|N| +2433637|AAAAAAAAFGCCFCAA|1950-12-21|611|2660|205|1950|4|12|21|4|1950|205|2660|Thursday|1950Q4|N|N|N|2433617|2433950|2433272|2433545|N|N|N|N|N| +2433638|AAAAAAAAGGCCFCAA|1950-12-22|611|2660|205|1950|5|12|22|4|1950|205|2660|Friday|1950Q4|N|Y|N|2433617|2433950|2433273|2433546|N|N|N|N|N| +2433639|AAAAAAAAHGCCFCAA|1950-12-23|611|2660|205|1950|6|12|23|4|1950|205|2660|Saturday|1950Q4|N|Y|N|2433617|2433950|2433274|2433547|N|N|N|N|N| +2433640|AAAAAAAAIGCCFCAA|1950-12-24|611|2660|205|1950|0|12|24|4|1950|205|2660|Sunday|1950Q4|N|N|N|2433617|2433950|2433275|2433548|N|N|N|N|N| +2433641|AAAAAAAAJGCCFCAA|1950-12-25|611|2660|205|1950|1|12|25|4|1950|205|2660|Monday|1950Q4|N|N|N|2433617|2433950|2433276|2433549|N|N|N|N|N| +2433642|AAAAAAAAKGCCFCAA|1950-12-26|611|2661|205|1950|2|12|26|4|1950|205|2661|Tuesday|1950Q4|Y|N|N|2433617|2433950|2433277|2433550|N|N|N|N|N| +2433643|AAAAAAAALGCCFCAA|1950-12-27|611|2661|205|1950|3|12|27|4|1950|205|2661|Wednesday|1950Q4|N|N|Y|2433617|2433950|2433278|2433551|N|N|N|N|N| +2433644|AAAAAAAAMGCCFCAA|1950-12-28|611|2661|205|1950|4|12|28|4|1950|205|2661|Thursday|1950Q4|N|N|N|2433617|2433950|2433279|2433552|N|N|N|N|N| +2433645|AAAAAAAANGCCFCAA|1950-12-29|611|2661|205|1950|5|12|29|4|1950|205|2661|Friday|1950Q4|N|Y|N|2433617|2433950|2433280|2433553|N|N|N|N|N| +2433646|AAAAAAAAOGCCFCAA|1950-12-30|611|2661|205|1950|6|12|30|4|1950|205|2661|Saturday|1950Q4|N|Y|N|2433617|2433950|2433281|2433554|N|N|N|N|N| +2433647|AAAAAAAAPGCCFCAA|1950-12-31|611|2661|205|1950|0|12|31|4|1950|205|2661|Sunday|1950Q4|N|N|N|2433617|2433950|2433282|2433555|N|N|N|N|N| +2433648|AAAAAAAAAHCCFCAA|1951-01-01|612|2661|205|1951|1|1|1|1|1951|205|2661|Monday|1951Q1|Y|N|N|2433648|2433647|2433283|2433556|N|N|N|N|N| +2433649|AAAAAAAABHCCFCAA|1951-01-02|612|2662|205|1951|2|1|2|1|1951|205|2662|Tuesday|1951Q1|N|N|Y|2433648|2433647|2433284|2433557|N|N|N|N|N| +2433650|AAAAAAAACHCCFCAA|1951-01-03|612|2662|205|1951|3|1|3|1|1951|205|2662|Wednesday|1951Q1|N|N|N|2433648|2433647|2433285|2433558|N|N|N|N|N| +2433651|AAAAAAAADHCCFCAA|1951-01-04|612|2662|205|1951|4|1|4|1|1951|205|2662|Thursday|1951Q1|N|N|N|2433648|2433647|2433286|2433559|N|N|N|N|N| +2433652|AAAAAAAAEHCCFCAA|1951-01-05|612|2662|205|1951|5|1|5|1|1951|205|2662|Friday|1951Q1|N|Y|N|2433648|2433647|2433287|2433560|N|N|N|N|N| +2433653|AAAAAAAAFHCCFCAA|1951-01-06|612|2662|205|1951|6|1|6|1|1951|205|2662|Saturday|1951Q1|N|Y|N|2433648|2433647|2433288|2433561|N|N|N|N|N| +2433654|AAAAAAAAGHCCFCAA|1951-01-07|612|2662|205|1951|0|1|7|1|1951|205|2662|Sunday|1951Q1|N|N|N|2433648|2433647|2433289|2433562|N|N|N|N|N| +2433655|AAAAAAAAHHCCFCAA|1951-01-08|612|2662|205|1951|1|1|8|1|1951|205|2662|Monday|1951Q1|N|N|N|2433648|2433647|2433290|2433563|N|N|N|N|N| +2433656|AAAAAAAAIHCCFCAA|1951-01-09|612|2663|205|1951|2|1|9|1|1951|205|2663|Tuesday|1951Q1|N|N|N|2433648|2433647|2433291|2433564|N|N|N|N|N| +2433657|AAAAAAAAJHCCFCAA|1951-01-10|612|2663|205|1951|3|1|10|1|1951|205|2663|Wednesday|1951Q1|N|N|N|2433648|2433647|2433292|2433565|N|N|N|N|N| +2433658|AAAAAAAAKHCCFCAA|1951-01-11|612|2663|205|1951|4|1|11|1|1951|205|2663|Thursday|1951Q1|N|N|N|2433648|2433647|2433293|2433566|N|N|N|N|N| +2433659|AAAAAAAALHCCFCAA|1951-01-12|612|2663|205|1951|5|1|12|1|1951|205|2663|Friday|1951Q1|N|Y|N|2433648|2433647|2433294|2433567|N|N|N|N|N| +2433660|AAAAAAAAMHCCFCAA|1951-01-13|612|2663|205|1951|6|1|13|1|1951|205|2663|Saturday|1951Q1|N|Y|N|2433648|2433647|2433295|2433568|N|N|N|N|N| +2433661|AAAAAAAANHCCFCAA|1951-01-14|612|2663|205|1951|0|1|14|1|1951|205|2663|Sunday|1951Q1|N|N|N|2433648|2433647|2433296|2433569|N|N|N|N|N| +2433662|AAAAAAAAOHCCFCAA|1951-01-15|612|2663|205|1951|1|1|15|1|1951|205|2663|Monday|1951Q1|N|N|N|2433648|2433647|2433297|2433570|N|N|N|N|N| +2433663|AAAAAAAAPHCCFCAA|1951-01-16|612|2664|205|1951|2|1|16|1|1951|205|2664|Tuesday|1951Q1|N|N|N|2433648|2433647|2433298|2433571|N|N|N|N|N| +2433664|AAAAAAAAAICCFCAA|1951-01-17|612|2664|205|1951|3|1|17|1|1951|205|2664|Wednesday|1951Q1|N|N|N|2433648|2433647|2433299|2433572|N|N|N|N|N| +2433665|AAAAAAAABICCFCAA|1951-01-18|612|2664|205|1951|4|1|18|1|1951|205|2664|Thursday|1951Q1|N|N|N|2433648|2433647|2433300|2433573|N|N|N|N|N| +2433666|AAAAAAAACICCFCAA|1951-01-19|612|2664|205|1951|5|1|19|1|1951|205|2664|Friday|1951Q1|N|Y|N|2433648|2433647|2433301|2433574|N|N|N|N|N| +2433667|AAAAAAAADICCFCAA|1951-01-20|612|2664|205|1951|6|1|20|1|1951|205|2664|Saturday|1951Q1|N|Y|N|2433648|2433647|2433302|2433575|N|N|N|N|N| +2433668|AAAAAAAAEICCFCAA|1951-01-21|612|2664|205|1951|0|1|21|1|1951|205|2664|Sunday|1951Q1|N|N|N|2433648|2433647|2433303|2433576|N|N|N|N|N| +2433669|AAAAAAAAFICCFCAA|1951-01-22|612|2664|205|1951|1|1|22|1|1951|205|2664|Monday|1951Q1|N|N|N|2433648|2433647|2433304|2433577|N|N|N|N|N| +2433670|AAAAAAAAGICCFCAA|1951-01-23|612|2665|205|1951|2|1|23|1|1951|205|2665|Tuesday|1951Q1|N|N|N|2433648|2433647|2433305|2433578|N|N|N|N|N| +2433671|AAAAAAAAHICCFCAA|1951-01-24|612|2665|205|1951|3|1|24|1|1951|205|2665|Wednesday|1951Q1|N|N|N|2433648|2433647|2433306|2433579|N|N|N|N|N| +2433672|AAAAAAAAIICCFCAA|1951-01-25|612|2665|205|1951|4|1|25|1|1951|205|2665|Thursday|1951Q1|N|N|N|2433648|2433647|2433307|2433580|N|N|N|N|N| +2433673|AAAAAAAAJICCFCAA|1951-01-26|612|2665|205|1951|5|1|26|1|1951|205|2665|Friday|1951Q1|N|Y|N|2433648|2433647|2433308|2433581|N|N|N|N|N| +2433674|AAAAAAAAKICCFCAA|1951-01-27|612|2665|205|1951|6|1|27|1|1951|205|2665|Saturday|1951Q1|N|Y|N|2433648|2433647|2433309|2433582|N|N|N|N|N| +2433675|AAAAAAAALICCFCAA|1951-01-28|612|2665|205|1951|0|1|28|1|1951|205|2665|Sunday|1951Q1|N|N|N|2433648|2433647|2433310|2433583|N|N|N|N|N| +2433676|AAAAAAAAMICCFCAA|1951-01-29|612|2665|205|1951|1|1|29|1|1951|205|2665|Monday|1951Q1|N|N|N|2433648|2433647|2433311|2433584|N|N|N|N|N| +2433677|AAAAAAAANICCFCAA|1951-01-30|612|2666|205|1951|2|1|30|1|1951|205|2666|Tuesday|1951Q1|N|N|N|2433648|2433647|2433312|2433585|N|N|N|N|N| +2433678|AAAAAAAAOICCFCAA|1951-01-31|612|2666|205|1951|3|1|31|1|1951|205|2666|Wednesday|1951Q1|N|N|N|2433648|2433647|2433313|2433586|N|N|N|N|N| +2433679|AAAAAAAAPICCFCAA|1951-02-01|613|2666|205|1951|4|2|1|1|1951|205|2666|Thursday|1951Q1|N|N|N|2433679|2433709|2433314|2433587|N|N|N|N|N| +2433680|AAAAAAAAAJCCFCAA|1951-02-02|613|2666|205|1951|5|2|2|1|1951|205|2666|Friday|1951Q1|N|Y|N|2433679|2433709|2433315|2433588|N|N|N|N|N| +2433681|AAAAAAAABJCCFCAA|1951-02-03|613|2666|205|1951|6|2|3|1|1951|205|2666|Saturday|1951Q1|N|Y|N|2433679|2433709|2433316|2433589|N|N|N|N|N| +2433682|AAAAAAAACJCCFCAA|1951-02-04|613|2666|205|1951|0|2|4|1|1951|205|2666|Sunday|1951Q1|N|N|N|2433679|2433709|2433317|2433590|N|N|N|N|N| +2433683|AAAAAAAADJCCFCAA|1951-02-05|613|2666|205|1951|1|2|5|1|1951|205|2666|Monday|1951Q1|N|N|N|2433679|2433709|2433318|2433591|N|N|N|N|N| +2433684|AAAAAAAAEJCCFCAA|1951-02-06|613|2667|205|1951|2|2|6|1|1951|205|2667|Tuesday|1951Q1|N|N|N|2433679|2433709|2433319|2433592|N|N|N|N|N| +2433685|AAAAAAAAFJCCFCAA|1951-02-07|613|2667|205|1951|3|2|7|1|1951|205|2667|Wednesday|1951Q1|N|N|N|2433679|2433709|2433320|2433593|N|N|N|N|N| +2433686|AAAAAAAAGJCCFCAA|1951-02-08|613|2667|205|1951|4|2|8|1|1951|205|2667|Thursday|1951Q1|N|N|N|2433679|2433709|2433321|2433594|N|N|N|N|N| +2433687|AAAAAAAAHJCCFCAA|1951-02-09|613|2667|205|1951|5|2|9|1|1951|205|2667|Friday|1951Q1|N|Y|N|2433679|2433709|2433322|2433595|N|N|N|N|N| +2433688|AAAAAAAAIJCCFCAA|1951-02-10|613|2667|205|1951|6|2|10|1|1951|205|2667|Saturday|1951Q1|N|Y|N|2433679|2433709|2433323|2433596|N|N|N|N|N| +2433689|AAAAAAAAJJCCFCAA|1951-02-11|613|2667|205|1951|0|2|11|1|1951|205|2667|Sunday|1951Q1|N|N|N|2433679|2433709|2433324|2433597|N|N|N|N|N| +2433690|AAAAAAAAKJCCFCAA|1951-02-12|613|2667|205|1951|1|2|12|1|1951|205|2667|Monday|1951Q1|N|N|N|2433679|2433709|2433325|2433598|N|N|N|N|N| +2433691|AAAAAAAALJCCFCAA|1951-02-13|613|2668|205|1951|2|2|13|1|1951|205|2668|Tuesday|1951Q1|N|N|N|2433679|2433709|2433326|2433599|N|N|N|N|N| +2433692|AAAAAAAAMJCCFCAA|1951-02-14|613|2668|205|1951|3|2|14|1|1951|205|2668|Wednesday|1951Q1|N|N|N|2433679|2433709|2433327|2433600|N|N|N|N|N| +2433693|AAAAAAAANJCCFCAA|1951-02-15|613|2668|205|1951|4|2|15|1|1951|205|2668|Thursday|1951Q1|N|N|N|2433679|2433709|2433328|2433601|N|N|N|N|N| +2433694|AAAAAAAAOJCCFCAA|1951-02-16|613|2668|205|1951|5|2|16|1|1951|205|2668|Friday|1951Q1|N|Y|N|2433679|2433709|2433329|2433602|N|N|N|N|N| +2433695|AAAAAAAAPJCCFCAA|1951-02-17|613|2668|205|1951|6|2|17|1|1951|205|2668|Saturday|1951Q1|N|Y|N|2433679|2433709|2433330|2433603|N|N|N|N|N| +2433696|AAAAAAAAAKCCFCAA|1951-02-18|613|2668|205|1951|0|2|18|1|1951|205|2668|Sunday|1951Q1|N|N|N|2433679|2433709|2433331|2433604|N|N|N|N|N| +2433697|AAAAAAAABKCCFCAA|1951-02-19|613|2668|205|1951|1|2|19|1|1951|205|2668|Monday|1951Q1|N|N|N|2433679|2433709|2433332|2433605|N|N|N|N|N| +2433698|AAAAAAAACKCCFCAA|1951-02-20|613|2669|205|1951|2|2|20|1|1951|205|2669|Tuesday|1951Q1|N|N|N|2433679|2433709|2433333|2433606|N|N|N|N|N| +2433699|AAAAAAAADKCCFCAA|1951-02-21|613|2669|205|1951|3|2|21|1|1951|205|2669|Wednesday|1951Q1|N|N|N|2433679|2433709|2433334|2433607|N|N|N|N|N| +2433700|AAAAAAAAEKCCFCAA|1951-02-22|613|2669|205|1951|4|2|22|1|1951|205|2669|Thursday|1951Q1|N|N|N|2433679|2433709|2433335|2433608|N|N|N|N|N| +2433701|AAAAAAAAFKCCFCAA|1951-02-23|613|2669|205|1951|5|2|23|1|1951|205|2669|Friday|1951Q1|N|Y|N|2433679|2433709|2433336|2433609|N|N|N|N|N| +2433702|AAAAAAAAGKCCFCAA|1951-02-24|613|2669|205|1951|6|2|24|1|1951|205|2669|Saturday|1951Q1|N|Y|N|2433679|2433709|2433337|2433610|N|N|N|N|N| +2433703|AAAAAAAAHKCCFCAA|1951-02-25|613|2669|205|1951|0|2|25|1|1951|205|2669|Sunday|1951Q1|N|N|N|2433679|2433709|2433338|2433611|N|N|N|N|N| +2433704|AAAAAAAAIKCCFCAA|1951-02-26|613|2669|205|1951|1|2|26|1|1951|205|2669|Monday|1951Q1|N|N|N|2433679|2433709|2433339|2433612|N|N|N|N|N| +2433705|AAAAAAAAJKCCFCAA|1951-02-27|613|2670|205|1951|2|2|27|1|1951|205|2670|Tuesday|1951Q1|N|N|N|2433679|2433709|2433340|2433613|N|N|N|N|N| +2433706|AAAAAAAAKKCCFCAA|1951-02-28|613|2670|205|1951|3|2|28|1|1951|205|2670|Wednesday|1951Q1|N|N|N|2433679|2433709|2433341|2433614|N|N|N|N|N| +2433707|AAAAAAAALKCCFCAA|1951-03-01|614|2670|206|1951|4|3|1|1|1951|206|2670|Thursday|1951Q1|N|N|N|2433707|2433765|2433342|2433615|N|N|N|N|N| +2433708|AAAAAAAAMKCCFCAA|1951-03-02|614|2670|206|1951|5|3|2|1|1951|206|2670|Friday|1951Q1|N|Y|N|2433707|2433765|2433343|2433616|N|N|N|N|N| +2433709|AAAAAAAANKCCFCAA|1951-03-03|614|2670|206|1951|6|3|3|1|1951|206|2670|Saturday|1951Q1|N|Y|N|2433707|2433765|2433344|2433617|N|N|N|N|N| +2433710|AAAAAAAAOKCCFCAA|1951-03-04|614|2670|206|1951|0|3|4|1|1951|206|2670|Sunday|1951Q1|N|N|N|2433707|2433765|2433345|2433618|N|N|N|N|N| +2433711|AAAAAAAAPKCCFCAA|1951-03-05|614|2670|206|1951|1|3|5|1|1951|206|2670|Monday|1951Q1|N|N|N|2433707|2433765|2433346|2433619|N|N|N|N|N| +2433712|AAAAAAAAALCCFCAA|1951-03-06|614|2671|206|1951|2|3|6|1|1951|206|2671|Tuesday|1951Q1|N|N|N|2433707|2433765|2433347|2433620|N|N|N|N|N| +2433713|AAAAAAAABLCCFCAA|1951-03-07|614|2671|206|1951|3|3|7|1|1951|206|2671|Wednesday|1951Q1|N|N|N|2433707|2433765|2433348|2433621|N|N|N|N|N| +2433714|AAAAAAAACLCCFCAA|1951-03-08|614|2671|206|1951|4|3|8|1|1951|206|2671|Thursday|1951Q1|N|N|N|2433707|2433765|2433349|2433622|N|N|N|N|N| +2433715|AAAAAAAADLCCFCAA|1951-03-09|614|2671|206|1951|5|3|9|1|1951|206|2671|Friday|1951Q1|N|Y|N|2433707|2433765|2433350|2433623|N|N|N|N|N| +2433716|AAAAAAAAELCCFCAA|1951-03-10|614|2671|206|1951|6|3|10|1|1951|206|2671|Saturday|1951Q1|N|Y|N|2433707|2433765|2433351|2433624|N|N|N|N|N| +2433717|AAAAAAAAFLCCFCAA|1951-03-11|614|2671|206|1951|0|3|11|1|1951|206|2671|Sunday|1951Q1|N|N|N|2433707|2433765|2433352|2433625|N|N|N|N|N| +2433718|AAAAAAAAGLCCFCAA|1951-03-12|614|2671|206|1951|1|3|12|1|1951|206|2671|Monday|1951Q1|N|N|N|2433707|2433765|2433353|2433626|N|N|N|N|N| +2433719|AAAAAAAAHLCCFCAA|1951-03-13|614|2672|206|1951|2|3|13|1|1951|206|2672|Tuesday|1951Q1|N|N|N|2433707|2433765|2433354|2433627|N|N|N|N|N| +2433720|AAAAAAAAILCCFCAA|1951-03-14|614|2672|206|1951|3|3|14|1|1951|206|2672|Wednesday|1951Q1|N|N|N|2433707|2433765|2433355|2433628|N|N|N|N|N| +2433721|AAAAAAAAJLCCFCAA|1951-03-15|614|2672|206|1951|4|3|15|1|1951|206|2672|Thursday|1951Q1|N|N|N|2433707|2433765|2433356|2433629|N|N|N|N|N| +2433722|AAAAAAAAKLCCFCAA|1951-03-16|614|2672|206|1951|5|3|16|1|1951|206|2672|Friday|1951Q1|N|Y|N|2433707|2433765|2433357|2433630|N|N|N|N|N| +2433723|AAAAAAAALLCCFCAA|1951-03-17|614|2672|206|1951|6|3|17|1|1951|206|2672|Saturday|1951Q1|N|Y|N|2433707|2433765|2433358|2433631|N|N|N|N|N| +2433724|AAAAAAAAMLCCFCAA|1951-03-18|614|2672|206|1951|0|3|18|1|1951|206|2672|Sunday|1951Q1|N|N|N|2433707|2433765|2433359|2433632|N|N|N|N|N| +2433725|AAAAAAAANLCCFCAA|1951-03-19|614|2672|206|1951|1|3|19|1|1951|206|2672|Monday|1951Q1|N|N|N|2433707|2433765|2433360|2433633|N|N|N|N|N| +2433726|AAAAAAAAOLCCFCAA|1951-03-20|614|2673|206|1951|2|3|20|1|1951|206|2673|Tuesday|1951Q1|N|N|N|2433707|2433765|2433361|2433634|N|N|N|N|N| +2433727|AAAAAAAAPLCCFCAA|1951-03-21|614|2673|206|1951|3|3|21|1|1951|206|2673|Wednesday|1951Q1|N|N|N|2433707|2433765|2433362|2433635|N|N|N|N|N| +2433728|AAAAAAAAAMCCFCAA|1951-03-22|614|2673|206|1951|4|3|22|1|1951|206|2673|Thursday|1951Q1|N|N|N|2433707|2433765|2433363|2433636|N|N|N|N|N| +2433729|AAAAAAAABMCCFCAA|1951-03-23|614|2673|206|1951|5|3|23|1|1951|206|2673|Friday|1951Q1|N|Y|N|2433707|2433765|2433364|2433637|N|N|N|N|N| +2433730|AAAAAAAACMCCFCAA|1951-03-24|614|2673|206|1951|6|3|24|1|1951|206|2673|Saturday|1951Q1|N|Y|N|2433707|2433765|2433365|2433638|N|N|N|N|N| +2433731|AAAAAAAADMCCFCAA|1951-03-25|614|2673|206|1951|0|3|25|1|1951|206|2673|Sunday|1951Q1|N|N|N|2433707|2433765|2433366|2433639|N|N|N|N|N| +2433732|AAAAAAAAEMCCFCAA|1951-03-26|614|2673|206|1951|1|3|26|1|1951|206|2673|Monday|1951Q1|N|N|N|2433707|2433765|2433367|2433640|N|N|N|N|N| +2433733|AAAAAAAAFMCCFCAA|1951-03-27|614|2674|206|1951|2|3|27|1|1951|206|2674|Tuesday|1951Q1|N|N|N|2433707|2433765|2433368|2433641|N|N|N|N|N| +2433734|AAAAAAAAGMCCFCAA|1951-03-28|614|2674|206|1951|3|3|28|1|1951|206|2674|Wednesday|1951Q1|N|N|N|2433707|2433765|2433369|2433642|N|N|N|N|N| +2433735|AAAAAAAAHMCCFCAA|1951-03-29|614|2674|206|1951|4|3|29|1|1951|206|2674|Thursday|1951Q1|N|N|N|2433707|2433765|2433370|2433643|N|N|N|N|N| +2433736|AAAAAAAAIMCCFCAA|1951-03-30|614|2674|206|1951|5|3|30|1|1951|206|2674|Friday|1951Q1|N|Y|N|2433707|2433765|2433371|2433644|N|N|N|N|N| +2433737|AAAAAAAAJMCCFCAA|1951-03-31|614|2674|206|1951|6|3|31|1|1951|206|2674|Saturday|1951Q1|N|Y|N|2433707|2433765|2433372|2433645|N|N|N|N|N| +2433738|AAAAAAAAKMCCFCAA|1951-04-01|615|2674|206|1951|0|4|1|1|1951|206|2674|Sunday|1951Q1|N|N|N|2433738|2433827|2433373|2433648|N|N|N|N|N| +2433739|AAAAAAAALMCCFCAA|1951-04-02|615|2674|206|1951|1|4|2|2|1951|206|2674|Monday|1951Q2|N|N|N|2433738|2433827|2433374|2433649|N|N|N|N|N| +2433740|AAAAAAAAMMCCFCAA|1951-04-03|615|2675|206|1951|2|4|3|2|1951|206|2675|Tuesday|1951Q2|N|N|N|2433738|2433827|2433375|2433650|N|N|N|N|N| +2433741|AAAAAAAANMCCFCAA|1951-04-04|615|2675|206|1951|3|4|4|2|1951|206|2675|Wednesday|1951Q2|N|N|N|2433738|2433827|2433376|2433651|N|N|N|N|N| +2433742|AAAAAAAAOMCCFCAA|1951-04-05|615|2675|206|1951|4|4|5|2|1951|206|2675|Thursday|1951Q2|N|N|N|2433738|2433827|2433377|2433652|N|N|N|N|N| +2433743|AAAAAAAAPMCCFCAA|1951-04-06|615|2675|206|1951|5|4|6|2|1951|206|2675|Friday|1951Q2|N|Y|N|2433738|2433827|2433378|2433653|N|N|N|N|N| +2433744|AAAAAAAAANCCFCAA|1951-04-07|615|2675|206|1951|6|4|7|2|1951|206|2675|Saturday|1951Q2|N|Y|N|2433738|2433827|2433379|2433654|N|N|N|N|N| +2433745|AAAAAAAABNCCFCAA|1951-04-08|615|2675|206|1951|0|4|8|2|1951|206|2675|Sunday|1951Q2|N|N|N|2433738|2433827|2433380|2433655|N|N|N|N|N| +2433746|AAAAAAAACNCCFCAA|1951-04-09|615|2675|206|1951|1|4|9|2|1951|206|2675|Monday|1951Q2|N|N|N|2433738|2433827|2433381|2433656|N|N|N|N|N| +2433747|AAAAAAAADNCCFCAA|1951-04-10|615|2676|206|1951|2|4|10|2|1951|206|2676|Tuesday|1951Q2|N|N|N|2433738|2433827|2433382|2433657|N|N|N|N|N| +2433748|AAAAAAAAENCCFCAA|1951-04-11|615|2676|206|1951|3|4|11|2|1951|206|2676|Wednesday|1951Q2|N|N|N|2433738|2433827|2433383|2433658|N|N|N|N|N| +2433749|AAAAAAAAFNCCFCAA|1951-04-12|615|2676|206|1951|4|4|12|2|1951|206|2676|Thursday|1951Q2|N|N|N|2433738|2433827|2433384|2433659|N|N|N|N|N| +2433750|AAAAAAAAGNCCFCAA|1951-04-13|615|2676|206|1951|5|4|13|2|1951|206|2676|Friday|1951Q2|N|Y|N|2433738|2433827|2433385|2433660|N|N|N|N|N| +2433751|AAAAAAAAHNCCFCAA|1951-04-14|615|2676|206|1951|6|4|14|2|1951|206|2676|Saturday|1951Q2|N|Y|N|2433738|2433827|2433386|2433661|N|N|N|N|N| +2433752|AAAAAAAAINCCFCAA|1951-04-15|615|2676|206|1951|0|4|15|2|1951|206|2676|Sunday|1951Q2|N|N|N|2433738|2433827|2433387|2433662|N|N|N|N|N| +2433753|AAAAAAAAJNCCFCAA|1951-04-16|615|2676|206|1951|1|4|16|2|1951|206|2676|Monday|1951Q2|N|N|N|2433738|2433827|2433388|2433663|N|N|N|N|N| +2433754|AAAAAAAAKNCCFCAA|1951-04-17|615|2677|206|1951|2|4|17|2|1951|206|2677|Tuesday|1951Q2|N|N|N|2433738|2433827|2433389|2433664|N|N|N|N|N| +2433755|AAAAAAAALNCCFCAA|1951-04-18|615|2677|206|1951|3|4|18|2|1951|206|2677|Wednesday|1951Q2|N|N|N|2433738|2433827|2433390|2433665|N|N|N|N|N| +2433756|AAAAAAAAMNCCFCAA|1951-04-19|615|2677|206|1951|4|4|19|2|1951|206|2677|Thursday|1951Q2|N|N|N|2433738|2433827|2433391|2433666|N|N|N|N|N| +2433757|AAAAAAAANNCCFCAA|1951-04-20|615|2677|206|1951|5|4|20|2|1951|206|2677|Friday|1951Q2|N|Y|N|2433738|2433827|2433392|2433667|N|N|N|N|N| +2433758|AAAAAAAAONCCFCAA|1951-04-21|615|2677|206|1951|6|4|21|2|1951|206|2677|Saturday|1951Q2|N|Y|N|2433738|2433827|2433393|2433668|N|N|N|N|N| +2433759|AAAAAAAAPNCCFCAA|1951-04-22|615|2677|206|1951|0|4|22|2|1951|206|2677|Sunday|1951Q2|N|N|N|2433738|2433827|2433394|2433669|N|N|N|N|N| +2433760|AAAAAAAAAOCCFCAA|1951-04-23|615|2677|206|1951|1|4|23|2|1951|206|2677|Monday|1951Q2|N|N|N|2433738|2433827|2433395|2433670|N|N|N|N|N| +2433761|AAAAAAAABOCCFCAA|1951-04-24|615|2678|206|1951|2|4|24|2|1951|206|2678|Tuesday|1951Q2|N|N|N|2433738|2433827|2433396|2433671|N|N|N|N|N| +2433762|AAAAAAAACOCCFCAA|1951-04-25|615|2678|206|1951|3|4|25|2|1951|206|2678|Wednesday|1951Q2|N|N|N|2433738|2433827|2433397|2433672|N|N|N|N|N| +2433763|AAAAAAAADOCCFCAA|1951-04-26|615|2678|206|1951|4|4|26|2|1951|206|2678|Thursday|1951Q2|N|N|N|2433738|2433827|2433398|2433673|N|N|N|N|N| +2433764|AAAAAAAAEOCCFCAA|1951-04-27|615|2678|206|1951|5|4|27|2|1951|206|2678|Friday|1951Q2|N|Y|N|2433738|2433827|2433399|2433674|N|N|N|N|N| +2433765|AAAAAAAAFOCCFCAA|1951-04-28|615|2678|206|1951|6|4|28|2|1951|206|2678|Saturday|1951Q2|N|Y|N|2433738|2433827|2433400|2433675|N|N|N|N|N| +2433766|AAAAAAAAGOCCFCAA|1951-04-29|615|2678|206|1951|0|4|29|2|1951|206|2678|Sunday|1951Q2|N|N|N|2433738|2433827|2433401|2433676|N|N|N|N|N| +2433767|AAAAAAAAHOCCFCAA|1951-04-30|615|2678|206|1951|1|4|30|2|1951|206|2678|Monday|1951Q2|N|N|N|2433738|2433827|2433402|2433677|N|N|N|N|N| +2433768|AAAAAAAAIOCCFCAA|1951-05-01|616|2679|206|1951|2|5|1|2|1951|206|2679|Tuesday|1951Q2|N|N|N|2433768|2433887|2433403|2433678|N|N|N|N|N| +2433769|AAAAAAAAJOCCFCAA|1951-05-02|616|2679|206|1951|3|5|2|2|1951|206|2679|Wednesday|1951Q2|N|N|N|2433768|2433887|2433404|2433679|N|N|N|N|N| +2433770|AAAAAAAAKOCCFCAA|1951-05-03|616|2679|206|1951|4|5|3|2|1951|206|2679|Thursday|1951Q2|N|N|N|2433768|2433887|2433405|2433680|N|N|N|N|N| +2433771|AAAAAAAALOCCFCAA|1951-05-04|616|2679|206|1951|5|5|4|2|1951|206|2679|Friday|1951Q2|N|Y|N|2433768|2433887|2433406|2433681|N|N|N|N|N| +2433772|AAAAAAAAMOCCFCAA|1951-05-05|616|2679|206|1951|6|5|5|2|1951|206|2679|Saturday|1951Q2|N|Y|N|2433768|2433887|2433407|2433682|N|N|N|N|N| +2433773|AAAAAAAANOCCFCAA|1951-05-06|616|2679|206|1951|0|5|6|2|1951|206|2679|Sunday|1951Q2|N|N|N|2433768|2433887|2433408|2433683|N|N|N|N|N| +2433774|AAAAAAAAOOCCFCAA|1951-05-07|616|2679|206|1951|1|5|7|2|1951|206|2679|Monday|1951Q2|N|N|N|2433768|2433887|2433409|2433684|N|N|N|N|N| +2433775|AAAAAAAAPOCCFCAA|1951-05-08|616|2680|206|1951|2|5|8|2|1951|206|2680|Tuesday|1951Q2|N|N|N|2433768|2433887|2433410|2433685|N|N|N|N|N| +2433776|AAAAAAAAAPCCFCAA|1951-05-09|616|2680|206|1951|3|5|9|2|1951|206|2680|Wednesday|1951Q2|N|N|N|2433768|2433887|2433411|2433686|N|N|N|N|N| +2433777|AAAAAAAABPCCFCAA|1951-05-10|616|2680|206|1951|4|5|10|2|1951|206|2680|Thursday|1951Q2|N|N|N|2433768|2433887|2433412|2433687|N|N|N|N|N| +2433778|AAAAAAAACPCCFCAA|1951-05-11|616|2680|206|1951|5|5|11|2|1951|206|2680|Friday|1951Q2|N|Y|N|2433768|2433887|2433413|2433688|N|N|N|N|N| +2433779|AAAAAAAADPCCFCAA|1951-05-12|616|2680|206|1951|6|5|12|2|1951|206|2680|Saturday|1951Q2|N|Y|N|2433768|2433887|2433414|2433689|N|N|N|N|N| +2433780|AAAAAAAAEPCCFCAA|1951-05-13|616|2680|206|1951|0|5|13|2|1951|206|2680|Sunday|1951Q2|N|N|N|2433768|2433887|2433415|2433690|N|N|N|N|N| +2433781|AAAAAAAAFPCCFCAA|1951-05-14|616|2680|206|1951|1|5|14|2|1951|206|2680|Monday|1951Q2|N|N|N|2433768|2433887|2433416|2433691|N|N|N|N|N| +2433782|AAAAAAAAGPCCFCAA|1951-05-15|616|2681|206|1951|2|5|15|2|1951|206|2681|Tuesday|1951Q2|N|N|N|2433768|2433887|2433417|2433692|N|N|N|N|N| +2433783|AAAAAAAAHPCCFCAA|1951-05-16|616|2681|206|1951|3|5|16|2|1951|206|2681|Wednesday|1951Q2|N|N|N|2433768|2433887|2433418|2433693|N|N|N|N|N| +2433784|AAAAAAAAIPCCFCAA|1951-05-17|616|2681|206|1951|4|5|17|2|1951|206|2681|Thursday|1951Q2|N|N|N|2433768|2433887|2433419|2433694|N|N|N|N|N| +2433785|AAAAAAAAJPCCFCAA|1951-05-18|616|2681|206|1951|5|5|18|2|1951|206|2681|Friday|1951Q2|N|Y|N|2433768|2433887|2433420|2433695|N|N|N|N|N| +2433786|AAAAAAAAKPCCFCAA|1951-05-19|616|2681|206|1951|6|5|19|2|1951|206|2681|Saturday|1951Q2|N|Y|N|2433768|2433887|2433421|2433696|N|N|N|N|N| +2433787|AAAAAAAALPCCFCAA|1951-05-20|616|2681|206|1951|0|5|20|2|1951|206|2681|Sunday|1951Q2|N|N|N|2433768|2433887|2433422|2433697|N|N|N|N|N| +2433788|AAAAAAAAMPCCFCAA|1951-05-21|616|2681|206|1951|1|5|21|2|1951|206|2681|Monday|1951Q2|N|N|N|2433768|2433887|2433423|2433698|N|N|N|N|N| +2433789|AAAAAAAANPCCFCAA|1951-05-22|616|2682|206|1951|2|5|22|2|1951|206|2682|Tuesday|1951Q2|N|N|N|2433768|2433887|2433424|2433699|N|N|N|N|N| +2433790|AAAAAAAAOPCCFCAA|1951-05-23|616|2682|206|1951|3|5|23|2|1951|206|2682|Wednesday|1951Q2|N|N|N|2433768|2433887|2433425|2433700|N|N|N|N|N| +2433791|AAAAAAAAPPCCFCAA|1951-05-24|616|2682|206|1951|4|5|24|2|1951|206|2682|Thursday|1951Q2|N|N|N|2433768|2433887|2433426|2433701|N|N|N|N|N| +2433792|AAAAAAAAAADCFCAA|1951-05-25|616|2682|206|1951|5|5|25|2|1951|206|2682|Friday|1951Q2|N|Y|N|2433768|2433887|2433427|2433702|N|N|N|N|N| +2433793|AAAAAAAABADCFCAA|1951-05-26|616|2682|206|1951|6|5|26|2|1951|206|2682|Saturday|1951Q2|N|Y|N|2433768|2433887|2433428|2433703|N|N|N|N|N| +2433794|AAAAAAAACADCFCAA|1951-05-27|616|2682|206|1951|0|5|27|2|1951|206|2682|Sunday|1951Q2|N|N|N|2433768|2433887|2433429|2433704|N|N|N|N|N| +2433795|AAAAAAAADADCFCAA|1951-05-28|616|2682|206|1951|1|5|28|2|1951|206|2682|Monday|1951Q2|N|N|N|2433768|2433887|2433430|2433705|N|N|N|N|N| +2433796|AAAAAAAAEADCFCAA|1951-05-29|616|2683|206|1951|2|5|29|2|1951|206|2683|Tuesday|1951Q2|N|N|N|2433768|2433887|2433431|2433706|N|N|N|N|N| +2433797|AAAAAAAAFADCFCAA|1951-05-30|616|2683|206|1951|3|5|30|2|1951|206|2683|Wednesday|1951Q2|N|N|N|2433768|2433887|2433432|2433707|N|N|N|N|N| +2433798|AAAAAAAAGADCFCAA|1951-05-31|616|2683|206|1951|4|5|31|2|1951|206|2683|Thursday|1951Q2|N|N|N|2433768|2433887|2433433|2433708|N|N|N|N|N| +2433799|AAAAAAAAHADCFCAA|1951-06-01|617|2683|207|1951|5|6|1|2|1951|207|2683|Friday|1951Q2|N|Y|N|2433799|2433949|2433434|2433709|N|N|N|N|N| +2433800|AAAAAAAAIADCFCAA|1951-06-02|617|2683|207|1951|6|6|2|2|1951|207|2683|Saturday|1951Q2|N|Y|N|2433799|2433949|2433435|2433710|N|N|N|N|N| +2433801|AAAAAAAAJADCFCAA|1951-06-03|617|2683|207|1951|0|6|3|2|1951|207|2683|Sunday|1951Q2|N|N|N|2433799|2433949|2433436|2433711|N|N|N|N|N| +2433802|AAAAAAAAKADCFCAA|1951-06-04|617|2683|207|1951|1|6|4|2|1951|207|2683|Monday|1951Q2|N|N|N|2433799|2433949|2433437|2433712|N|N|N|N|N| +2433803|AAAAAAAALADCFCAA|1951-06-05|617|2684|207|1951|2|6|5|2|1951|207|2684|Tuesday|1951Q2|N|N|N|2433799|2433949|2433438|2433713|N|N|N|N|N| +2433804|AAAAAAAAMADCFCAA|1951-06-06|617|2684|207|1951|3|6|6|2|1951|207|2684|Wednesday|1951Q2|N|N|N|2433799|2433949|2433439|2433714|N|N|N|N|N| +2433805|AAAAAAAANADCFCAA|1951-06-07|617|2684|207|1951|4|6|7|2|1951|207|2684|Thursday|1951Q2|N|N|N|2433799|2433949|2433440|2433715|N|N|N|N|N| +2433806|AAAAAAAAOADCFCAA|1951-06-08|617|2684|207|1951|5|6|8|2|1951|207|2684|Friday|1951Q2|N|Y|N|2433799|2433949|2433441|2433716|N|N|N|N|N| +2433807|AAAAAAAAPADCFCAA|1951-06-09|617|2684|207|1951|6|6|9|2|1951|207|2684|Saturday|1951Q2|N|Y|N|2433799|2433949|2433442|2433717|N|N|N|N|N| +2433808|AAAAAAAAABDCFCAA|1951-06-10|617|2684|207|1951|0|6|10|2|1951|207|2684|Sunday|1951Q2|N|N|N|2433799|2433949|2433443|2433718|N|N|N|N|N| +2433809|AAAAAAAABBDCFCAA|1951-06-11|617|2684|207|1951|1|6|11|2|1951|207|2684|Monday|1951Q2|N|N|N|2433799|2433949|2433444|2433719|N|N|N|N|N| +2433810|AAAAAAAACBDCFCAA|1951-06-12|617|2685|207|1951|2|6|12|2|1951|207|2685|Tuesday|1951Q2|N|N|N|2433799|2433949|2433445|2433720|N|N|N|N|N| +2433811|AAAAAAAADBDCFCAA|1951-06-13|617|2685|207|1951|3|6|13|2|1951|207|2685|Wednesday|1951Q2|N|N|N|2433799|2433949|2433446|2433721|N|N|N|N|N| +2433812|AAAAAAAAEBDCFCAA|1951-06-14|617|2685|207|1951|4|6|14|2|1951|207|2685|Thursday|1951Q2|N|N|N|2433799|2433949|2433447|2433722|N|N|N|N|N| +2433813|AAAAAAAAFBDCFCAA|1951-06-15|617|2685|207|1951|5|6|15|2|1951|207|2685|Friday|1951Q2|N|Y|N|2433799|2433949|2433448|2433723|N|N|N|N|N| +2433814|AAAAAAAAGBDCFCAA|1951-06-16|617|2685|207|1951|6|6|16|2|1951|207|2685|Saturday|1951Q2|N|Y|N|2433799|2433949|2433449|2433724|N|N|N|N|N| +2433815|AAAAAAAAHBDCFCAA|1951-06-17|617|2685|207|1951|0|6|17|2|1951|207|2685|Sunday|1951Q2|N|N|N|2433799|2433949|2433450|2433725|N|N|N|N|N| +2433816|AAAAAAAAIBDCFCAA|1951-06-18|617|2685|207|1951|1|6|18|2|1951|207|2685|Monday|1951Q2|N|N|N|2433799|2433949|2433451|2433726|N|N|N|N|N| +2433817|AAAAAAAAJBDCFCAA|1951-06-19|617|2686|207|1951|2|6|19|2|1951|207|2686|Tuesday|1951Q2|N|N|N|2433799|2433949|2433452|2433727|N|N|N|N|N| +2433818|AAAAAAAAKBDCFCAA|1951-06-20|617|2686|207|1951|3|6|20|2|1951|207|2686|Wednesday|1951Q2|N|N|N|2433799|2433949|2433453|2433728|N|N|N|N|N| +2433819|AAAAAAAALBDCFCAA|1951-06-21|617|2686|207|1951|4|6|21|2|1951|207|2686|Thursday|1951Q2|N|N|N|2433799|2433949|2433454|2433729|N|N|N|N|N| +2433820|AAAAAAAAMBDCFCAA|1951-06-22|617|2686|207|1951|5|6|22|2|1951|207|2686|Friday|1951Q2|N|Y|N|2433799|2433949|2433455|2433730|N|N|N|N|N| +2433821|AAAAAAAANBDCFCAA|1951-06-23|617|2686|207|1951|6|6|23|2|1951|207|2686|Saturday|1951Q2|N|Y|N|2433799|2433949|2433456|2433731|N|N|N|N|N| +2433822|AAAAAAAAOBDCFCAA|1951-06-24|617|2686|207|1951|0|6|24|2|1951|207|2686|Sunday|1951Q2|N|N|N|2433799|2433949|2433457|2433732|N|N|N|N|N| +2433823|AAAAAAAAPBDCFCAA|1951-06-25|617|2686|207|1951|1|6|25|2|1951|207|2686|Monday|1951Q2|N|N|N|2433799|2433949|2433458|2433733|N|N|N|N|N| +2433824|AAAAAAAAACDCFCAA|1951-06-26|617|2687|207|1951|2|6|26|2|1951|207|2687|Tuesday|1951Q2|N|N|N|2433799|2433949|2433459|2433734|N|N|N|N|N| +2433825|AAAAAAAABCDCFCAA|1951-06-27|617|2687|207|1951|3|6|27|2|1951|207|2687|Wednesday|1951Q2|N|N|N|2433799|2433949|2433460|2433735|N|N|N|N|N| +2433826|AAAAAAAACCDCFCAA|1951-06-28|617|2687|207|1951|4|6|28|2|1951|207|2687|Thursday|1951Q2|N|N|N|2433799|2433949|2433461|2433736|N|N|N|N|N| +2433827|AAAAAAAADCDCFCAA|1951-06-29|617|2687|207|1951|5|6|29|2|1951|207|2687|Friday|1951Q2|N|Y|N|2433799|2433949|2433462|2433737|N|N|N|N|N| +2433828|AAAAAAAAECDCFCAA|1951-06-30|617|2687|207|1951|6|6|30|2|1951|207|2687|Saturday|1951Q2|N|Y|N|2433799|2433949|2433463|2433738|N|N|N|N|N| +2433829|AAAAAAAAFCDCFCAA|1951-07-01|618|2687|207|1951|0|7|1|2|1951|207|2687|Sunday|1951Q2|N|N|N|2433829|2434009|2433464|2433738|N|N|N|N|N| +2433830|AAAAAAAAGCDCFCAA|1951-07-02|618|2687|207|1951|1|7|2|3|1951|207|2687|Monday|1951Q3|N|N|N|2433829|2434009|2433465|2433739|N|N|N|N|N| +2433831|AAAAAAAAHCDCFCAA|1951-07-03|618|2688|207|1951|2|7|3|3|1951|207|2688|Tuesday|1951Q3|N|N|N|2433829|2434009|2433466|2433740|N|N|N|N|N| +2433832|AAAAAAAAICDCFCAA|1951-07-04|618|2688|207|1951|3|7|4|3|1951|207|2688|Wednesday|1951Q3|N|N|N|2433829|2434009|2433467|2433741|N|N|N|N|N| +2433833|AAAAAAAAJCDCFCAA|1951-07-05|618|2688|207|1951|4|7|5|3|1951|207|2688|Thursday|1951Q3|Y|N|N|2433829|2434009|2433468|2433742|N|N|N|N|N| +2433834|AAAAAAAAKCDCFCAA|1951-07-06|618|2688|207|1951|5|7|6|3|1951|207|2688|Friday|1951Q3|N|Y|Y|2433829|2434009|2433469|2433743|N|N|N|N|N| +2433835|AAAAAAAALCDCFCAA|1951-07-07|618|2688|207|1951|6|7|7|3|1951|207|2688|Saturday|1951Q3|N|Y|N|2433829|2434009|2433470|2433744|N|N|N|N|N| +2433836|AAAAAAAAMCDCFCAA|1951-07-08|618|2688|207|1951|0|7|8|3|1951|207|2688|Sunday|1951Q3|N|N|N|2433829|2434009|2433471|2433745|N|N|N|N|N| +2433837|AAAAAAAANCDCFCAA|1951-07-09|618|2688|207|1951|1|7|9|3|1951|207|2688|Monday|1951Q3|N|N|N|2433829|2434009|2433472|2433746|N|N|N|N|N| +2433838|AAAAAAAAOCDCFCAA|1951-07-10|618|2689|207|1951|2|7|10|3|1951|207|2689|Tuesday|1951Q3|N|N|N|2433829|2434009|2433473|2433747|N|N|N|N|N| +2433839|AAAAAAAAPCDCFCAA|1951-07-11|618|2689|207|1951|3|7|11|3|1951|207|2689|Wednesday|1951Q3|N|N|N|2433829|2434009|2433474|2433748|N|N|N|N|N| +2433840|AAAAAAAAADDCFCAA|1951-07-12|618|2689|207|1951|4|7|12|3|1951|207|2689|Thursday|1951Q3|N|N|N|2433829|2434009|2433475|2433749|N|N|N|N|N| +2433841|AAAAAAAABDDCFCAA|1951-07-13|618|2689|207|1951|5|7|13|3|1951|207|2689|Friday|1951Q3|N|Y|N|2433829|2434009|2433476|2433750|N|N|N|N|N| +2433842|AAAAAAAACDDCFCAA|1951-07-14|618|2689|207|1951|6|7|14|3|1951|207|2689|Saturday|1951Q3|N|Y|N|2433829|2434009|2433477|2433751|N|N|N|N|N| +2433843|AAAAAAAADDDCFCAA|1951-07-15|618|2689|207|1951|0|7|15|3|1951|207|2689|Sunday|1951Q3|N|N|N|2433829|2434009|2433478|2433752|N|N|N|N|N| +2433844|AAAAAAAAEDDCFCAA|1951-07-16|618|2689|207|1951|1|7|16|3|1951|207|2689|Monday|1951Q3|N|N|N|2433829|2434009|2433479|2433753|N|N|N|N|N| +2433845|AAAAAAAAFDDCFCAA|1951-07-17|618|2690|207|1951|2|7|17|3|1951|207|2690|Tuesday|1951Q3|N|N|N|2433829|2434009|2433480|2433754|N|N|N|N|N| +2433846|AAAAAAAAGDDCFCAA|1951-07-18|618|2690|207|1951|3|7|18|3|1951|207|2690|Wednesday|1951Q3|N|N|N|2433829|2434009|2433481|2433755|N|N|N|N|N| +2433847|AAAAAAAAHDDCFCAA|1951-07-19|618|2690|207|1951|4|7|19|3|1951|207|2690|Thursday|1951Q3|N|N|N|2433829|2434009|2433482|2433756|N|N|N|N|N| +2433848|AAAAAAAAIDDCFCAA|1951-07-20|618|2690|207|1951|5|7|20|3|1951|207|2690|Friday|1951Q3|N|Y|N|2433829|2434009|2433483|2433757|N|N|N|N|N| +2433849|AAAAAAAAJDDCFCAA|1951-07-21|618|2690|207|1951|6|7|21|3|1951|207|2690|Saturday|1951Q3|N|Y|N|2433829|2434009|2433484|2433758|N|N|N|N|N| +2433850|AAAAAAAAKDDCFCAA|1951-07-22|618|2690|207|1951|0|7|22|3|1951|207|2690|Sunday|1951Q3|N|N|N|2433829|2434009|2433485|2433759|N|N|N|N|N| +2433851|AAAAAAAALDDCFCAA|1951-07-23|618|2690|207|1951|1|7|23|3|1951|207|2690|Monday|1951Q3|N|N|N|2433829|2434009|2433486|2433760|N|N|N|N|N| +2433852|AAAAAAAAMDDCFCAA|1951-07-24|618|2691|207|1951|2|7|24|3|1951|207|2691|Tuesday|1951Q3|N|N|N|2433829|2434009|2433487|2433761|N|N|N|N|N| +2433853|AAAAAAAANDDCFCAA|1951-07-25|618|2691|207|1951|3|7|25|3|1951|207|2691|Wednesday|1951Q3|N|N|N|2433829|2434009|2433488|2433762|N|N|N|N|N| +2433854|AAAAAAAAODDCFCAA|1951-07-26|618|2691|207|1951|4|7|26|3|1951|207|2691|Thursday|1951Q3|N|N|N|2433829|2434009|2433489|2433763|N|N|N|N|N| +2433855|AAAAAAAAPDDCFCAA|1951-07-27|618|2691|207|1951|5|7|27|3|1951|207|2691|Friday|1951Q3|N|Y|N|2433829|2434009|2433490|2433764|N|N|N|N|N| +2433856|AAAAAAAAAEDCFCAA|1951-07-28|618|2691|207|1951|6|7|28|3|1951|207|2691|Saturday|1951Q3|N|Y|N|2433829|2434009|2433491|2433765|N|N|N|N|N| +2433857|AAAAAAAABEDCFCAA|1951-07-29|618|2691|207|1951|0|7|29|3|1951|207|2691|Sunday|1951Q3|N|N|N|2433829|2434009|2433492|2433766|N|N|N|N|N| +2433858|AAAAAAAACEDCFCAA|1951-07-30|618|2691|207|1951|1|7|30|3|1951|207|2691|Monday|1951Q3|N|N|N|2433829|2434009|2433493|2433767|N|N|N|N|N| +2433859|AAAAAAAADEDCFCAA|1951-07-31|618|2692|207|1951|2|7|31|3|1951|207|2692|Tuesday|1951Q3|N|N|N|2433829|2434009|2433494|2433768|N|N|N|N|N| +2433860|AAAAAAAAEEDCFCAA|1951-08-01|619|2692|207|1951|3|8|1|3|1951|207|2692|Wednesday|1951Q3|N|N|N|2433860|2434071|2433495|2433769|N|N|N|N|N| +2433861|AAAAAAAAFEDCFCAA|1951-08-02|619|2692|207|1951|4|8|2|3|1951|207|2692|Thursday|1951Q3|N|N|N|2433860|2434071|2433496|2433770|N|N|N|N|N| +2433862|AAAAAAAAGEDCFCAA|1951-08-03|619|2692|207|1951|5|8|3|3|1951|207|2692|Friday|1951Q3|N|Y|N|2433860|2434071|2433497|2433771|N|N|N|N|N| +2433863|AAAAAAAAHEDCFCAA|1951-08-04|619|2692|207|1951|6|8|4|3|1951|207|2692|Saturday|1951Q3|N|Y|N|2433860|2434071|2433498|2433772|N|N|N|N|N| +2433864|AAAAAAAAIEDCFCAA|1951-08-05|619|2692|207|1951|0|8|5|3|1951|207|2692|Sunday|1951Q3|N|N|N|2433860|2434071|2433499|2433773|N|N|N|N|N| +2433865|AAAAAAAAJEDCFCAA|1951-08-06|619|2692|207|1951|1|8|6|3|1951|207|2692|Monday|1951Q3|N|N|N|2433860|2434071|2433500|2433774|N|N|N|N|N| +2433866|AAAAAAAAKEDCFCAA|1951-08-07|619|2693|207|1951|2|8|7|3|1951|207|2693|Tuesday|1951Q3|N|N|N|2433860|2434071|2433501|2433775|N|N|N|N|N| +2433867|AAAAAAAALEDCFCAA|1951-08-08|619|2693|207|1951|3|8|8|3|1951|207|2693|Wednesday|1951Q3|N|N|N|2433860|2434071|2433502|2433776|N|N|N|N|N| +2433868|AAAAAAAAMEDCFCAA|1951-08-09|619|2693|207|1951|4|8|9|3|1951|207|2693|Thursday|1951Q3|N|N|N|2433860|2434071|2433503|2433777|N|N|N|N|N| +2433869|AAAAAAAANEDCFCAA|1951-08-10|619|2693|207|1951|5|8|10|3|1951|207|2693|Friday|1951Q3|N|Y|N|2433860|2434071|2433504|2433778|N|N|N|N|N| +2433870|AAAAAAAAOEDCFCAA|1951-08-11|619|2693|207|1951|6|8|11|3|1951|207|2693|Saturday|1951Q3|N|Y|N|2433860|2434071|2433505|2433779|N|N|N|N|N| +2433871|AAAAAAAAPEDCFCAA|1951-08-12|619|2693|207|1951|0|8|12|3|1951|207|2693|Sunday|1951Q3|N|N|N|2433860|2434071|2433506|2433780|N|N|N|N|N| +2433872|AAAAAAAAAFDCFCAA|1951-08-13|619|2693|207|1951|1|8|13|3|1951|207|2693|Monday|1951Q3|N|N|N|2433860|2434071|2433507|2433781|N|N|N|N|N| +2433873|AAAAAAAABFDCFCAA|1951-08-14|619|2694|207|1951|2|8|14|3|1951|207|2694|Tuesday|1951Q3|N|N|N|2433860|2434071|2433508|2433782|N|N|N|N|N| +2433874|AAAAAAAACFDCFCAA|1951-08-15|619|2694|207|1951|3|8|15|3|1951|207|2694|Wednesday|1951Q3|N|N|N|2433860|2434071|2433509|2433783|N|N|N|N|N| +2433875|AAAAAAAADFDCFCAA|1951-08-16|619|2694|207|1951|4|8|16|3|1951|207|2694|Thursday|1951Q3|N|N|N|2433860|2434071|2433510|2433784|N|N|N|N|N| +2433876|AAAAAAAAEFDCFCAA|1951-08-17|619|2694|207|1951|5|8|17|3|1951|207|2694|Friday|1951Q3|N|Y|N|2433860|2434071|2433511|2433785|N|N|N|N|N| +2433877|AAAAAAAAFFDCFCAA|1951-08-18|619|2694|207|1951|6|8|18|3|1951|207|2694|Saturday|1951Q3|N|Y|N|2433860|2434071|2433512|2433786|N|N|N|N|N| +2433878|AAAAAAAAGFDCFCAA|1951-08-19|619|2694|207|1951|0|8|19|3|1951|207|2694|Sunday|1951Q3|N|N|N|2433860|2434071|2433513|2433787|N|N|N|N|N| +2433879|AAAAAAAAHFDCFCAA|1951-08-20|619|2694|207|1951|1|8|20|3|1951|207|2694|Monday|1951Q3|N|N|N|2433860|2434071|2433514|2433788|N|N|N|N|N| +2433880|AAAAAAAAIFDCFCAA|1951-08-21|619|2695|207|1951|2|8|21|3|1951|207|2695|Tuesday|1951Q3|N|N|N|2433860|2434071|2433515|2433789|N|N|N|N|N| +2433881|AAAAAAAAJFDCFCAA|1951-08-22|619|2695|207|1951|3|8|22|3|1951|207|2695|Wednesday|1951Q3|N|N|N|2433860|2434071|2433516|2433790|N|N|N|N|N| +2433882|AAAAAAAAKFDCFCAA|1951-08-23|619|2695|207|1951|4|8|23|3|1951|207|2695|Thursday|1951Q3|N|N|N|2433860|2434071|2433517|2433791|N|N|N|N|N| +2433883|AAAAAAAALFDCFCAA|1951-08-24|619|2695|207|1951|5|8|24|3|1951|207|2695|Friday|1951Q3|N|Y|N|2433860|2434071|2433518|2433792|N|N|N|N|N| +2433884|AAAAAAAAMFDCFCAA|1951-08-25|619|2695|207|1951|6|8|25|3|1951|207|2695|Saturday|1951Q3|N|Y|N|2433860|2434071|2433519|2433793|N|N|N|N|N| +2433885|AAAAAAAANFDCFCAA|1951-08-26|619|2695|207|1951|0|8|26|3|1951|207|2695|Sunday|1951Q3|N|N|N|2433860|2434071|2433520|2433794|N|N|N|N|N| +2433886|AAAAAAAAOFDCFCAA|1951-08-27|619|2695|207|1951|1|8|27|3|1951|207|2695|Monday|1951Q3|N|N|N|2433860|2434071|2433521|2433795|N|N|N|N|N| +2433887|AAAAAAAAPFDCFCAA|1951-08-28|619|2696|207|1951|2|8|28|3|1951|207|2696|Tuesday|1951Q3|N|N|N|2433860|2434071|2433522|2433796|N|N|N|N|N| +2433888|AAAAAAAAAGDCFCAA|1951-08-29|619|2696|207|1951|3|8|29|3|1951|207|2696|Wednesday|1951Q3|N|N|N|2433860|2434071|2433523|2433797|N|N|N|N|N| +2433889|AAAAAAAABGDCFCAA|1951-08-30|619|2696|207|1951|4|8|30|3|1951|207|2696|Thursday|1951Q3|N|N|N|2433860|2434071|2433524|2433798|N|N|N|N|N| +2433890|AAAAAAAACGDCFCAA|1951-08-31|619|2696|207|1951|5|8|31|3|1951|207|2696|Friday|1951Q3|N|Y|N|2433860|2434071|2433525|2433799|N|N|N|N|N| +2433891|AAAAAAAADGDCFCAA|1951-09-01|620|2696|208|1951|6|9|1|3|1951|208|2696|Saturday|1951Q3|N|Y|N|2433891|2434133|2433526|2433800|N|N|N|N|N| +2433892|AAAAAAAAEGDCFCAA|1951-09-02|620|2696|208|1951|0|9|2|3|1951|208|2696|Sunday|1951Q3|N|N|N|2433891|2434133|2433527|2433801|N|N|N|N|N| +2433893|AAAAAAAAFGDCFCAA|1951-09-03|620|2696|208|1951|1|9|3|3|1951|208|2696|Monday|1951Q3|N|N|N|2433891|2434133|2433528|2433802|N|N|N|N|N| +2433894|AAAAAAAAGGDCFCAA|1951-09-04|620|2697|208|1951|2|9|4|3|1951|208|2697|Tuesday|1951Q3|N|N|N|2433891|2434133|2433529|2433803|N|N|N|N|N| +2433895|AAAAAAAAHGDCFCAA|1951-09-05|620|2697|208|1951|3|9|5|3|1951|208|2697|Wednesday|1951Q3|N|N|N|2433891|2434133|2433530|2433804|N|N|N|N|N| +2433896|AAAAAAAAIGDCFCAA|1951-09-06|620|2697|208|1951|4|9|6|3|1951|208|2697|Thursday|1951Q3|N|N|N|2433891|2434133|2433531|2433805|N|N|N|N|N| +2433897|AAAAAAAAJGDCFCAA|1951-09-07|620|2697|208|1951|5|9|7|3|1951|208|2697|Friday|1951Q3|N|Y|N|2433891|2434133|2433532|2433806|N|N|N|N|N| +2433898|AAAAAAAAKGDCFCAA|1951-09-08|620|2697|208|1951|6|9|8|3|1951|208|2697|Saturday|1951Q3|N|Y|N|2433891|2434133|2433533|2433807|N|N|N|N|N| +2433899|AAAAAAAALGDCFCAA|1951-09-09|620|2697|208|1951|0|9|9|3|1951|208|2697|Sunday|1951Q3|N|N|N|2433891|2434133|2433534|2433808|N|N|N|N|N| +2433900|AAAAAAAAMGDCFCAA|1951-09-10|620|2697|208|1951|1|9|10|3|1951|208|2697|Monday|1951Q3|N|N|N|2433891|2434133|2433535|2433809|N|N|N|N|N| +2433901|AAAAAAAANGDCFCAA|1951-09-11|620|2698|208|1951|2|9|11|3|1951|208|2698|Tuesday|1951Q3|N|N|N|2433891|2434133|2433536|2433810|N|N|N|N|N| +2433902|AAAAAAAAOGDCFCAA|1951-09-12|620|2698|208|1951|3|9|12|3|1951|208|2698|Wednesday|1951Q3|N|N|N|2433891|2434133|2433537|2433811|N|N|N|N|N| +2433903|AAAAAAAAPGDCFCAA|1951-09-13|620|2698|208|1951|4|9|13|3|1951|208|2698|Thursday|1951Q3|N|N|N|2433891|2434133|2433538|2433812|N|N|N|N|N| +2433904|AAAAAAAAAHDCFCAA|1951-09-14|620|2698|208|1951|5|9|14|3|1951|208|2698|Friday|1951Q3|N|Y|N|2433891|2434133|2433539|2433813|N|N|N|N|N| +2433905|AAAAAAAABHDCFCAA|1951-09-15|620|2698|208|1951|6|9|15|3|1951|208|2698|Saturday|1951Q3|N|Y|N|2433891|2434133|2433540|2433814|N|N|N|N|N| +2433906|AAAAAAAACHDCFCAA|1951-09-16|620|2698|208|1951|0|9|16|3|1951|208|2698|Sunday|1951Q3|N|N|N|2433891|2434133|2433541|2433815|N|N|N|N|N| +2433907|AAAAAAAADHDCFCAA|1951-09-17|620|2698|208|1951|1|9|17|3|1951|208|2698|Monday|1951Q3|N|N|N|2433891|2434133|2433542|2433816|N|N|N|N|N| +2433908|AAAAAAAAEHDCFCAA|1951-09-18|620|2699|208|1951|2|9|18|3|1951|208|2699|Tuesday|1951Q3|N|N|N|2433891|2434133|2433543|2433817|N|N|N|N|N| +2433909|AAAAAAAAFHDCFCAA|1951-09-19|620|2699|208|1951|3|9|19|3|1951|208|2699|Wednesday|1951Q3|N|N|N|2433891|2434133|2433544|2433818|N|N|N|N|N| +2433910|AAAAAAAAGHDCFCAA|1951-09-20|620|2699|208|1951|4|9|20|3|1951|208|2699|Thursday|1951Q3|N|N|N|2433891|2434133|2433545|2433819|N|N|N|N|N| +2433911|AAAAAAAAHHDCFCAA|1951-09-21|620|2699|208|1951|5|9|21|3|1951|208|2699|Friday|1951Q3|N|Y|N|2433891|2434133|2433546|2433820|N|N|N|N|N| +2433912|AAAAAAAAIHDCFCAA|1951-09-22|620|2699|208|1951|6|9|22|3|1951|208|2699|Saturday|1951Q3|N|Y|N|2433891|2434133|2433547|2433821|N|N|N|N|N| +2433913|AAAAAAAAJHDCFCAA|1951-09-23|620|2699|208|1951|0|9|23|3|1951|208|2699|Sunday|1951Q3|N|N|N|2433891|2434133|2433548|2433822|N|N|N|N|N| +2433914|AAAAAAAAKHDCFCAA|1951-09-24|620|2699|208|1951|1|9|24|3|1951|208|2699|Monday|1951Q3|N|N|N|2433891|2434133|2433549|2433823|N|N|N|N|N| +2433915|AAAAAAAALHDCFCAA|1951-09-25|620|2700|208|1951|2|9|25|3|1951|208|2700|Tuesday|1951Q3|N|N|N|2433891|2434133|2433550|2433824|N|N|N|N|N| +2433916|AAAAAAAAMHDCFCAA|1951-09-26|620|2700|208|1951|3|9|26|3|1951|208|2700|Wednesday|1951Q3|N|N|N|2433891|2434133|2433551|2433825|N|N|N|N|N| +2433917|AAAAAAAANHDCFCAA|1951-09-27|620|2700|208|1951|4|9|27|3|1951|208|2700|Thursday|1951Q3|N|N|N|2433891|2434133|2433552|2433826|N|N|N|N|N| +2433918|AAAAAAAAOHDCFCAA|1951-09-28|620|2700|208|1951|5|9|28|3|1951|208|2700|Friday|1951Q3|N|Y|N|2433891|2434133|2433553|2433827|N|N|N|N|N| +2433919|AAAAAAAAPHDCFCAA|1951-09-29|620|2700|208|1951|6|9|29|3|1951|208|2700|Saturday|1951Q3|N|Y|N|2433891|2434133|2433554|2433828|N|N|N|N|N| +2433920|AAAAAAAAAIDCFCAA|1951-09-30|620|2700|208|1951|0|9|30|3|1951|208|2700|Sunday|1951Q3|N|N|N|2433891|2434133|2433555|2433829|N|N|N|N|N| +2433921|AAAAAAAABIDCFCAA|1951-10-01|621|2700|208|1951|1|10|1|3|1951|208|2700|Monday|1951Q3|N|N|N|2433921|2434193|2433556|2433829|N|N|N|N|N| +2433922|AAAAAAAACIDCFCAA|1951-10-02|621|2701|208|1951|2|10|2|4|1951|208|2701|Tuesday|1951Q4|N|N|N|2433921|2434193|2433557|2433830|N|N|N|N|N| +2433923|AAAAAAAADIDCFCAA|1951-10-03|621|2701|208|1951|3|10|3|4|1951|208|2701|Wednesday|1951Q4|N|N|N|2433921|2434193|2433558|2433831|N|N|N|N|N| +2433924|AAAAAAAAEIDCFCAA|1951-10-04|621|2701|208|1951|4|10|4|4|1951|208|2701|Thursday|1951Q4|N|N|N|2433921|2434193|2433559|2433832|N|N|N|N|N| +2433925|AAAAAAAAFIDCFCAA|1951-10-05|621|2701|208|1951|5|10|5|4|1951|208|2701|Friday|1951Q4|N|Y|N|2433921|2434193|2433560|2433833|N|N|N|N|N| +2433926|AAAAAAAAGIDCFCAA|1951-10-06|621|2701|208|1951|6|10|6|4|1951|208|2701|Saturday|1951Q4|N|Y|N|2433921|2434193|2433561|2433834|N|N|N|N|N| +2433927|AAAAAAAAHIDCFCAA|1951-10-07|621|2701|208|1951|0|10|7|4|1951|208|2701|Sunday|1951Q4|N|N|N|2433921|2434193|2433562|2433835|N|N|N|N|N| +2433928|AAAAAAAAIIDCFCAA|1951-10-08|621|2701|208|1951|1|10|8|4|1951|208|2701|Monday|1951Q4|N|N|N|2433921|2434193|2433563|2433836|N|N|N|N|N| +2433929|AAAAAAAAJIDCFCAA|1951-10-09|621|2702|208|1951|2|10|9|4|1951|208|2702|Tuesday|1951Q4|N|N|N|2433921|2434193|2433564|2433837|N|N|N|N|N| +2433930|AAAAAAAAKIDCFCAA|1951-10-10|621|2702|208|1951|3|10|10|4|1951|208|2702|Wednesday|1951Q4|N|N|N|2433921|2434193|2433565|2433838|N|N|N|N|N| +2433931|AAAAAAAALIDCFCAA|1951-10-11|621|2702|208|1951|4|10|11|4|1951|208|2702|Thursday|1951Q4|N|N|N|2433921|2434193|2433566|2433839|N|N|N|N|N| +2433932|AAAAAAAAMIDCFCAA|1951-10-12|621|2702|208|1951|5|10|12|4|1951|208|2702|Friday|1951Q4|N|Y|N|2433921|2434193|2433567|2433840|N|N|N|N|N| +2433933|AAAAAAAANIDCFCAA|1951-10-13|621|2702|208|1951|6|10|13|4|1951|208|2702|Saturday|1951Q4|N|Y|N|2433921|2434193|2433568|2433841|N|N|N|N|N| +2433934|AAAAAAAAOIDCFCAA|1951-10-14|621|2702|208|1951|0|10|14|4|1951|208|2702|Sunday|1951Q4|N|N|N|2433921|2434193|2433569|2433842|N|N|N|N|N| +2433935|AAAAAAAAPIDCFCAA|1951-10-15|621|2702|208|1951|1|10|15|4|1951|208|2702|Monday|1951Q4|N|N|N|2433921|2434193|2433570|2433843|N|N|N|N|N| +2433936|AAAAAAAAAJDCFCAA|1951-10-16|621|2703|208|1951|2|10|16|4|1951|208|2703|Tuesday|1951Q4|N|N|N|2433921|2434193|2433571|2433844|N|N|N|N|N| +2433937|AAAAAAAABJDCFCAA|1951-10-17|621|2703|208|1951|3|10|17|4|1951|208|2703|Wednesday|1951Q4|N|N|N|2433921|2434193|2433572|2433845|N|N|N|N|N| +2433938|AAAAAAAACJDCFCAA|1951-10-18|621|2703|208|1951|4|10|18|4|1951|208|2703|Thursday|1951Q4|N|N|N|2433921|2434193|2433573|2433846|N|N|N|N|N| +2433939|AAAAAAAADJDCFCAA|1951-10-19|621|2703|208|1951|5|10|19|4|1951|208|2703|Friday|1951Q4|N|Y|N|2433921|2434193|2433574|2433847|N|N|N|N|N| +2433940|AAAAAAAAEJDCFCAA|1951-10-20|621|2703|208|1951|6|10|20|4|1951|208|2703|Saturday|1951Q4|N|Y|N|2433921|2434193|2433575|2433848|N|N|N|N|N| +2433941|AAAAAAAAFJDCFCAA|1951-10-21|621|2703|208|1951|0|10|21|4|1951|208|2703|Sunday|1951Q4|N|N|N|2433921|2434193|2433576|2433849|N|N|N|N|N| +2433942|AAAAAAAAGJDCFCAA|1951-10-22|621|2703|208|1951|1|10|22|4|1951|208|2703|Monday|1951Q4|N|N|N|2433921|2434193|2433577|2433850|N|N|N|N|N| +2433943|AAAAAAAAHJDCFCAA|1951-10-23|621|2704|208|1951|2|10|23|4|1951|208|2704|Tuesday|1951Q4|N|N|N|2433921|2434193|2433578|2433851|N|N|N|N|N| +2433944|AAAAAAAAIJDCFCAA|1951-10-24|621|2704|208|1951|3|10|24|4|1951|208|2704|Wednesday|1951Q4|N|N|N|2433921|2434193|2433579|2433852|N|N|N|N|N| +2433945|AAAAAAAAJJDCFCAA|1951-10-25|621|2704|208|1951|4|10|25|4|1951|208|2704|Thursday|1951Q4|N|N|N|2433921|2434193|2433580|2433853|N|N|N|N|N| +2433946|AAAAAAAAKJDCFCAA|1951-10-26|621|2704|208|1951|5|10|26|4|1951|208|2704|Friday|1951Q4|N|Y|N|2433921|2434193|2433581|2433854|N|N|N|N|N| +2433947|AAAAAAAALJDCFCAA|1951-10-27|621|2704|208|1951|6|10|27|4|1951|208|2704|Saturday|1951Q4|N|Y|N|2433921|2434193|2433582|2433855|N|N|N|N|N| +2433948|AAAAAAAAMJDCFCAA|1951-10-28|621|2704|208|1951|0|10|28|4|1951|208|2704|Sunday|1951Q4|N|N|N|2433921|2434193|2433583|2433856|N|N|N|N|N| +2433949|AAAAAAAANJDCFCAA|1951-10-29|621|2704|208|1951|1|10|29|4|1951|208|2704|Monday|1951Q4|N|N|N|2433921|2434193|2433584|2433857|N|N|N|N|N| +2433950|AAAAAAAAOJDCFCAA|1951-10-30|621|2705|208|1951|2|10|30|4|1951|208|2705|Tuesday|1951Q4|N|N|N|2433921|2434193|2433585|2433858|N|N|N|N|N| +2433951|AAAAAAAAPJDCFCAA|1951-10-31|621|2705|208|1951|3|10|31|4|1951|208|2705|Wednesday|1951Q4|N|N|N|2433921|2434193|2433586|2433859|N|N|N|N|N| +2433952|AAAAAAAAAKDCFCAA|1951-11-01|622|2705|208|1951|4|11|1|4|1951|208|2705|Thursday|1951Q4|N|N|N|2433952|2434255|2433587|2433860|N|N|N|N|N| +2433953|AAAAAAAABKDCFCAA|1951-11-02|622|2705|208|1951|5|11|2|4|1951|208|2705|Friday|1951Q4|N|Y|N|2433952|2434255|2433588|2433861|N|N|N|N|N| +2433954|AAAAAAAACKDCFCAA|1951-11-03|622|2705|208|1951|6|11|3|4|1951|208|2705|Saturday|1951Q4|N|Y|N|2433952|2434255|2433589|2433862|N|N|N|N|N| +2433955|AAAAAAAADKDCFCAA|1951-11-04|622|2705|208|1951|0|11|4|4|1951|208|2705|Sunday|1951Q4|N|N|N|2433952|2434255|2433590|2433863|N|N|N|N|N| +2433956|AAAAAAAAEKDCFCAA|1951-11-05|622|2705|208|1951|1|11|5|4|1951|208|2705|Monday|1951Q4|N|N|N|2433952|2434255|2433591|2433864|N|N|N|N|N| +2433957|AAAAAAAAFKDCFCAA|1951-11-06|622|2706|208|1951|2|11|6|4|1951|208|2706|Tuesday|1951Q4|N|N|N|2433952|2434255|2433592|2433865|N|N|N|N|N| +2433958|AAAAAAAAGKDCFCAA|1951-11-07|622|2706|208|1951|3|11|7|4|1951|208|2706|Wednesday|1951Q4|N|N|N|2433952|2434255|2433593|2433866|N|N|N|N|N| +2433959|AAAAAAAAHKDCFCAA|1951-11-08|622|2706|208|1951|4|11|8|4|1951|208|2706|Thursday|1951Q4|N|N|N|2433952|2434255|2433594|2433867|N|N|N|N|N| +2433960|AAAAAAAAIKDCFCAA|1951-11-09|622|2706|208|1951|5|11|9|4|1951|208|2706|Friday|1951Q4|N|Y|N|2433952|2434255|2433595|2433868|N|N|N|N|N| +2433961|AAAAAAAAJKDCFCAA|1951-11-10|622|2706|208|1951|6|11|10|4|1951|208|2706|Saturday|1951Q4|N|Y|N|2433952|2434255|2433596|2433869|N|N|N|N|N| +2433962|AAAAAAAAKKDCFCAA|1951-11-11|622|2706|208|1951|0|11|11|4|1951|208|2706|Sunday|1951Q4|N|N|N|2433952|2434255|2433597|2433870|N|N|N|N|N| +2433963|AAAAAAAALKDCFCAA|1951-11-12|622|2706|208|1951|1|11|12|4|1951|208|2706|Monday|1951Q4|N|N|N|2433952|2434255|2433598|2433871|N|N|N|N|N| +2433964|AAAAAAAAMKDCFCAA|1951-11-13|622|2707|208|1951|2|11|13|4|1951|208|2707|Tuesday|1951Q4|N|N|N|2433952|2434255|2433599|2433872|N|N|N|N|N| +2433965|AAAAAAAANKDCFCAA|1951-11-14|622|2707|208|1951|3|11|14|4|1951|208|2707|Wednesday|1951Q4|N|N|N|2433952|2434255|2433600|2433873|N|N|N|N|N| +2433966|AAAAAAAAOKDCFCAA|1951-11-15|622|2707|208|1951|4|11|15|4|1951|208|2707|Thursday|1951Q4|N|N|N|2433952|2434255|2433601|2433874|N|N|N|N|N| +2433967|AAAAAAAAPKDCFCAA|1951-11-16|622|2707|208|1951|5|11|16|4|1951|208|2707|Friday|1951Q4|N|Y|N|2433952|2434255|2433602|2433875|N|N|N|N|N| +2433968|AAAAAAAAALDCFCAA|1951-11-17|622|2707|208|1951|6|11|17|4|1951|208|2707|Saturday|1951Q4|N|Y|N|2433952|2434255|2433603|2433876|N|N|N|N|N| +2433969|AAAAAAAABLDCFCAA|1951-11-18|622|2707|208|1951|0|11|18|4|1951|208|2707|Sunday|1951Q4|N|N|N|2433952|2434255|2433604|2433877|N|N|N|N|N| +2433970|AAAAAAAACLDCFCAA|1951-11-19|622|2707|208|1951|1|11|19|4|1951|208|2707|Monday|1951Q4|N|N|N|2433952|2434255|2433605|2433878|N|N|N|N|N| +2433971|AAAAAAAADLDCFCAA|1951-11-20|622|2708|208|1951|2|11|20|4|1951|208|2708|Tuesday|1951Q4|N|N|N|2433952|2434255|2433606|2433879|N|N|N|N|N| +2433972|AAAAAAAAELDCFCAA|1951-11-21|622|2708|208|1951|3|11|21|4|1951|208|2708|Wednesday|1951Q4|N|N|N|2433952|2434255|2433607|2433880|N|N|N|N|N| +2433973|AAAAAAAAFLDCFCAA|1951-11-22|622|2708|208|1951|4|11|22|4|1951|208|2708|Thursday|1951Q4|N|N|N|2433952|2434255|2433608|2433881|N|N|N|N|N| +2433974|AAAAAAAAGLDCFCAA|1951-11-23|622|2708|208|1951|5|11|23|4|1951|208|2708|Friday|1951Q4|N|Y|N|2433952|2434255|2433609|2433882|N|N|N|N|N| +2433975|AAAAAAAAHLDCFCAA|1951-11-24|622|2708|208|1951|6|11|24|4|1951|208|2708|Saturday|1951Q4|N|Y|N|2433952|2434255|2433610|2433883|N|N|N|N|N| +2433976|AAAAAAAAILDCFCAA|1951-11-25|622|2708|208|1951|0|11|25|4|1951|208|2708|Sunday|1951Q4|N|N|N|2433952|2434255|2433611|2433884|N|N|N|N|N| +2433977|AAAAAAAAJLDCFCAA|1951-11-26|622|2708|208|1951|1|11|26|4|1951|208|2708|Monday|1951Q4|N|N|N|2433952|2434255|2433612|2433885|N|N|N|N|N| +2433978|AAAAAAAAKLDCFCAA|1951-11-27|622|2709|208|1951|2|11|27|4|1951|208|2709|Tuesday|1951Q4|N|N|N|2433952|2434255|2433613|2433886|N|N|N|N|N| +2433979|AAAAAAAALLDCFCAA|1951-11-28|622|2709|208|1951|3|11|28|4|1951|208|2709|Wednesday|1951Q4|N|N|N|2433952|2434255|2433614|2433887|N|N|N|N|N| +2433980|AAAAAAAAMLDCFCAA|1951-11-29|622|2709|208|1951|4|11|29|4|1951|208|2709|Thursday|1951Q4|N|N|N|2433952|2434255|2433615|2433888|N|N|N|N|N| +2433981|AAAAAAAANLDCFCAA|1951-11-30|622|2709|208|1951|5|11|30|4|1951|208|2709|Friday|1951Q4|N|Y|N|2433952|2434255|2433616|2433889|N|N|N|N|N| +2433982|AAAAAAAAOLDCFCAA|1951-12-01|623|2709|209|1951|6|12|1|4|1951|209|2709|Saturday|1951Q4|N|Y|N|2433982|2434315|2433617|2433890|N|N|N|N|N| +2433983|AAAAAAAAPLDCFCAA|1951-12-02|623|2709|209|1951|0|12|2|4|1951|209|2709|Sunday|1951Q4|N|N|N|2433982|2434315|2433618|2433891|N|N|N|N|N| +2433984|AAAAAAAAAMDCFCAA|1951-12-03|623|2709|209|1951|1|12|3|4|1951|209|2709|Monday|1951Q4|N|N|N|2433982|2434315|2433619|2433892|N|N|N|N|N| +2433985|AAAAAAAABMDCFCAA|1951-12-04|623|2710|209|1951|2|12|4|4|1951|209|2710|Tuesday|1951Q4|N|N|N|2433982|2434315|2433620|2433893|N|N|N|N|N| +2433986|AAAAAAAACMDCFCAA|1951-12-05|623|2710|209|1951|3|12|5|4|1951|209|2710|Wednesday|1951Q4|N|N|N|2433982|2434315|2433621|2433894|N|N|N|N|N| +2433987|AAAAAAAADMDCFCAA|1951-12-06|623|2710|209|1951|4|12|6|4|1951|209|2710|Thursday|1951Q4|N|N|N|2433982|2434315|2433622|2433895|N|N|N|N|N| +2433988|AAAAAAAAEMDCFCAA|1951-12-07|623|2710|209|1951|5|12|7|4|1951|209|2710|Friday|1951Q4|N|Y|N|2433982|2434315|2433623|2433896|N|N|N|N|N| +2433989|AAAAAAAAFMDCFCAA|1951-12-08|623|2710|209|1951|6|12|8|4|1951|209|2710|Saturday|1951Q4|N|Y|N|2433982|2434315|2433624|2433897|N|N|N|N|N| +2433990|AAAAAAAAGMDCFCAA|1951-12-09|623|2710|209|1951|0|12|9|4|1951|209|2710|Sunday|1951Q4|N|N|N|2433982|2434315|2433625|2433898|N|N|N|N|N| +2433991|AAAAAAAAHMDCFCAA|1951-12-10|623|2710|209|1951|1|12|10|4|1951|209|2710|Monday|1951Q4|N|N|N|2433982|2434315|2433626|2433899|N|N|N|N|N| +2433992|AAAAAAAAIMDCFCAA|1951-12-11|623|2711|209|1951|2|12|11|4|1951|209|2711|Tuesday|1951Q4|N|N|N|2433982|2434315|2433627|2433900|N|N|N|N|N| +2433993|AAAAAAAAJMDCFCAA|1951-12-12|623|2711|209|1951|3|12|12|4|1951|209|2711|Wednesday|1951Q4|N|N|N|2433982|2434315|2433628|2433901|N|N|N|N|N| +2433994|AAAAAAAAKMDCFCAA|1951-12-13|623|2711|209|1951|4|12|13|4|1951|209|2711|Thursday|1951Q4|N|N|N|2433982|2434315|2433629|2433902|N|N|N|N|N| +2433995|AAAAAAAALMDCFCAA|1951-12-14|623|2711|209|1951|5|12|14|4|1951|209|2711|Friday|1951Q4|N|Y|N|2433982|2434315|2433630|2433903|N|N|N|N|N| +2433996|AAAAAAAAMMDCFCAA|1951-12-15|623|2711|209|1951|6|12|15|4|1951|209|2711|Saturday|1951Q4|N|Y|N|2433982|2434315|2433631|2433904|N|N|N|N|N| +2433997|AAAAAAAANMDCFCAA|1951-12-16|623|2711|209|1951|0|12|16|4|1951|209|2711|Sunday|1951Q4|N|N|N|2433982|2434315|2433632|2433905|N|N|N|N|N| +2433998|AAAAAAAAOMDCFCAA|1951-12-17|623|2711|209|1951|1|12|17|4|1951|209|2711|Monday|1951Q4|N|N|N|2433982|2434315|2433633|2433906|N|N|N|N|N| +2433999|AAAAAAAAPMDCFCAA|1951-12-18|623|2712|209|1951|2|12|18|4|1951|209|2712|Tuesday|1951Q4|N|N|N|2433982|2434315|2433634|2433907|N|N|N|N|N| +2434000|AAAAAAAAANDCFCAA|1951-12-19|623|2712|209|1951|3|12|19|4|1951|209|2712|Wednesday|1951Q4|N|N|N|2433982|2434315|2433635|2433908|N|N|N|N|N| +2434001|AAAAAAAABNDCFCAA|1951-12-20|623|2712|209|1951|4|12|20|4|1951|209|2712|Thursday|1951Q4|N|N|N|2433982|2434315|2433636|2433909|N|N|N|N|N| +2434002|AAAAAAAACNDCFCAA|1951-12-21|623|2712|209|1951|5|12|21|4|1951|209|2712|Friday|1951Q4|N|Y|N|2433982|2434315|2433637|2433910|N|N|N|N|N| +2434003|AAAAAAAADNDCFCAA|1951-12-22|623|2712|209|1951|6|12|22|4|1951|209|2712|Saturday|1951Q4|N|Y|N|2433982|2434315|2433638|2433911|N|N|N|N|N| +2434004|AAAAAAAAENDCFCAA|1951-12-23|623|2712|209|1951|0|12|23|4|1951|209|2712|Sunday|1951Q4|N|N|N|2433982|2434315|2433639|2433912|N|N|N|N|N| +2434005|AAAAAAAAFNDCFCAA|1951-12-24|623|2712|209|1951|1|12|24|4|1951|209|2712|Monday|1951Q4|N|N|N|2433982|2434315|2433640|2433913|N|N|N|N|N| +2434006|AAAAAAAAGNDCFCAA|1951-12-25|623|2713|209|1951|2|12|25|4|1951|209|2713|Tuesday|1951Q4|N|N|N|2433982|2434315|2433641|2433914|N|N|N|N|N| +2434007|AAAAAAAAHNDCFCAA|1951-12-26|623|2713|209|1951|3|12|26|4|1951|209|2713|Wednesday|1951Q4|Y|N|N|2433982|2434315|2433642|2433915|N|N|N|N|N| +2434008|AAAAAAAAINDCFCAA|1951-12-27|623|2713|209|1951|4|12|27|4|1951|209|2713|Thursday|1951Q4|N|N|Y|2433982|2434315|2433643|2433916|N|N|N|N|N| +2434009|AAAAAAAAJNDCFCAA|1951-12-28|623|2713|209|1951|5|12|28|4|1951|209|2713|Friday|1951Q4|N|Y|N|2433982|2434315|2433644|2433917|N|N|N|N|N| +2434010|AAAAAAAAKNDCFCAA|1951-12-29|623|2713|209|1951|6|12|29|4|1951|209|2713|Saturday|1951Q4|N|Y|N|2433982|2434315|2433645|2433918|N|N|N|N|N| +2434011|AAAAAAAALNDCFCAA|1951-12-30|623|2713|209|1951|0|12|30|4|1951|209|2713|Sunday|1951Q4|N|N|N|2433982|2434315|2433646|2433919|N|N|N|N|N| +2434012|AAAAAAAAMNDCFCAA|1951-12-31|623|2713|209|1951|1|12|31|4|1951|209|2713|Monday|1951Q4|N|N|N|2433982|2434315|2433647|2433920|N|N|N|N|N| +2434013|AAAAAAAANNDCFCAA|1952-01-01|624|2714|209|1952|2|1|1|1|1952|209|2714|Tuesday|1952Q1|Y|N|N|2434013|2434012|2433648|2433921|N|N|N|N|N| +2434014|AAAAAAAAONDCFCAA|1952-01-02|624|2714|209|1952|3|1|2|1|1952|209|2714|Wednesday|1952Q1|N|N|Y|2434013|2434012|2433649|2433922|N|N|N|N|N| +2434015|AAAAAAAAPNDCFCAA|1952-01-03|624|2714|209|1952|4|1|3|1|1952|209|2714|Thursday|1952Q1|N|N|N|2434013|2434012|2433650|2433923|N|N|N|N|N| +2434016|AAAAAAAAAODCFCAA|1952-01-04|624|2714|209|1952|5|1|4|1|1952|209|2714|Friday|1952Q1|N|Y|N|2434013|2434012|2433651|2433924|N|N|N|N|N| +2434017|AAAAAAAABODCFCAA|1952-01-05|624|2714|209|1952|6|1|5|1|1952|209|2714|Saturday|1952Q1|N|Y|N|2434013|2434012|2433652|2433925|N|N|N|N|N| +2434018|AAAAAAAACODCFCAA|1952-01-06|624|2714|209|1952|0|1|6|1|1952|209|2714|Sunday|1952Q1|N|N|N|2434013|2434012|2433653|2433926|N|N|N|N|N| +2434019|AAAAAAAADODCFCAA|1952-01-07|624|2714|209|1952|1|1|7|1|1952|209|2714|Monday|1952Q1|N|N|N|2434013|2434012|2433654|2433927|N|N|N|N|N| +2434020|AAAAAAAAEODCFCAA|1952-01-08|624|2715|209|1952|2|1|8|1|1952|209|2715|Tuesday|1952Q1|N|N|N|2434013|2434012|2433655|2433928|N|N|N|N|N| +2434021|AAAAAAAAFODCFCAA|1952-01-09|624|2715|209|1952|3|1|9|1|1952|209|2715|Wednesday|1952Q1|N|N|N|2434013|2434012|2433656|2433929|N|N|N|N|N| +2434022|AAAAAAAAGODCFCAA|1952-01-10|624|2715|209|1952|4|1|10|1|1952|209|2715|Thursday|1952Q1|N|N|N|2434013|2434012|2433657|2433930|N|N|N|N|N| +2434023|AAAAAAAAHODCFCAA|1952-01-11|624|2715|209|1952|5|1|11|1|1952|209|2715|Friday|1952Q1|N|Y|N|2434013|2434012|2433658|2433931|N|N|N|N|N| +2434024|AAAAAAAAIODCFCAA|1952-01-12|624|2715|209|1952|6|1|12|1|1952|209|2715|Saturday|1952Q1|N|Y|N|2434013|2434012|2433659|2433932|N|N|N|N|N| +2434025|AAAAAAAAJODCFCAA|1952-01-13|624|2715|209|1952|0|1|13|1|1952|209|2715|Sunday|1952Q1|N|N|N|2434013|2434012|2433660|2433933|N|N|N|N|N| +2434026|AAAAAAAAKODCFCAA|1952-01-14|624|2715|209|1952|1|1|14|1|1952|209|2715|Monday|1952Q1|N|N|N|2434013|2434012|2433661|2433934|N|N|N|N|N| +2434027|AAAAAAAALODCFCAA|1952-01-15|624|2716|209|1952|2|1|15|1|1952|209|2716|Tuesday|1952Q1|N|N|N|2434013|2434012|2433662|2433935|N|N|N|N|N| +2434028|AAAAAAAAMODCFCAA|1952-01-16|624|2716|209|1952|3|1|16|1|1952|209|2716|Wednesday|1952Q1|N|N|N|2434013|2434012|2433663|2433936|N|N|N|N|N| +2434029|AAAAAAAANODCFCAA|1952-01-17|624|2716|209|1952|4|1|17|1|1952|209|2716|Thursday|1952Q1|N|N|N|2434013|2434012|2433664|2433937|N|N|N|N|N| +2434030|AAAAAAAAOODCFCAA|1952-01-18|624|2716|209|1952|5|1|18|1|1952|209|2716|Friday|1952Q1|N|Y|N|2434013|2434012|2433665|2433938|N|N|N|N|N| +2434031|AAAAAAAAPODCFCAA|1952-01-19|624|2716|209|1952|6|1|19|1|1952|209|2716|Saturday|1952Q1|N|Y|N|2434013|2434012|2433666|2433939|N|N|N|N|N| +2434032|AAAAAAAAAPDCFCAA|1952-01-20|624|2716|209|1952|0|1|20|1|1952|209|2716|Sunday|1952Q1|N|N|N|2434013|2434012|2433667|2433940|N|N|N|N|N| +2434033|AAAAAAAABPDCFCAA|1952-01-21|624|2716|209|1952|1|1|21|1|1952|209|2716|Monday|1952Q1|N|N|N|2434013|2434012|2433668|2433941|N|N|N|N|N| +2434034|AAAAAAAACPDCFCAA|1952-01-22|624|2717|209|1952|2|1|22|1|1952|209|2717|Tuesday|1952Q1|N|N|N|2434013|2434012|2433669|2433942|N|N|N|N|N| +2434035|AAAAAAAADPDCFCAA|1952-01-23|624|2717|209|1952|3|1|23|1|1952|209|2717|Wednesday|1952Q1|N|N|N|2434013|2434012|2433670|2433943|N|N|N|N|N| +2434036|AAAAAAAAEPDCFCAA|1952-01-24|624|2717|209|1952|4|1|24|1|1952|209|2717|Thursday|1952Q1|N|N|N|2434013|2434012|2433671|2433944|N|N|N|N|N| +2434037|AAAAAAAAFPDCFCAA|1952-01-25|624|2717|209|1952|5|1|25|1|1952|209|2717|Friday|1952Q1|N|Y|N|2434013|2434012|2433672|2433945|N|N|N|N|N| +2434038|AAAAAAAAGPDCFCAA|1952-01-26|624|2717|209|1952|6|1|26|1|1952|209|2717|Saturday|1952Q1|N|Y|N|2434013|2434012|2433673|2433946|N|N|N|N|N| +2434039|AAAAAAAAHPDCFCAA|1952-01-27|624|2717|209|1952|0|1|27|1|1952|209|2717|Sunday|1952Q1|N|N|N|2434013|2434012|2433674|2433947|N|N|N|N|N| +2434040|AAAAAAAAIPDCFCAA|1952-01-28|624|2717|209|1952|1|1|28|1|1952|209|2717|Monday|1952Q1|N|N|N|2434013|2434012|2433675|2433948|N|N|N|N|N| +2434041|AAAAAAAAJPDCFCAA|1952-01-29|624|2718|209|1952|2|1|29|1|1952|209|2718|Tuesday|1952Q1|N|N|N|2434013|2434012|2433676|2433949|N|N|N|N|N| +2434042|AAAAAAAAKPDCFCAA|1952-01-30|624|2718|209|1952|3|1|30|1|1952|209|2718|Wednesday|1952Q1|N|N|N|2434013|2434012|2433677|2433950|N|N|N|N|N| +2434043|AAAAAAAALPDCFCAA|1952-01-31|624|2718|209|1952|4|1|31|1|1952|209|2718|Thursday|1952Q1|N|N|N|2434013|2434012|2433678|2433951|N|N|N|N|N| +2434044|AAAAAAAAMPDCFCAA|1952-02-01|625|2718|209|1952|5|2|1|1|1952|209|2718|Friday|1952Q1|N|Y|N|2434044|2434074|2433679|2433952|N|N|N|N|N| +2434045|AAAAAAAANPDCFCAA|1952-02-02|625|2718|209|1952|6|2|2|1|1952|209|2718|Saturday|1952Q1|N|Y|N|2434044|2434074|2433680|2433953|N|N|N|N|N| +2434046|AAAAAAAAOPDCFCAA|1952-02-03|625|2718|209|1952|0|2|3|1|1952|209|2718|Sunday|1952Q1|N|N|N|2434044|2434074|2433681|2433954|N|N|N|N|N| +2434047|AAAAAAAAPPDCFCAA|1952-02-04|625|2718|209|1952|1|2|4|1|1952|209|2718|Monday|1952Q1|N|N|N|2434044|2434074|2433682|2433955|N|N|N|N|N| +2434048|AAAAAAAAAAECFCAA|1952-02-05|625|2719|209|1952|2|2|5|1|1952|209|2719|Tuesday|1952Q1|N|N|N|2434044|2434074|2433683|2433956|N|N|N|N|N| +2434049|AAAAAAAABAECFCAA|1952-02-06|625|2719|209|1952|3|2|6|1|1952|209|2719|Wednesday|1952Q1|N|N|N|2434044|2434074|2433684|2433957|N|N|N|N|N| +2434050|AAAAAAAACAECFCAA|1952-02-07|625|2719|209|1952|4|2|7|1|1952|209|2719|Thursday|1952Q1|N|N|N|2434044|2434074|2433685|2433958|N|N|N|N|N| +2434051|AAAAAAAADAECFCAA|1952-02-08|625|2719|209|1952|5|2|8|1|1952|209|2719|Friday|1952Q1|N|Y|N|2434044|2434074|2433686|2433959|N|N|N|N|N| +2434052|AAAAAAAAEAECFCAA|1952-02-09|625|2719|209|1952|6|2|9|1|1952|209|2719|Saturday|1952Q1|N|Y|N|2434044|2434074|2433687|2433960|N|N|N|N|N| +2434053|AAAAAAAAFAECFCAA|1952-02-10|625|2719|209|1952|0|2|10|1|1952|209|2719|Sunday|1952Q1|N|N|N|2434044|2434074|2433688|2433961|N|N|N|N|N| +2434054|AAAAAAAAGAECFCAA|1952-02-11|625|2719|209|1952|1|2|11|1|1952|209|2719|Monday|1952Q1|N|N|N|2434044|2434074|2433689|2433962|N|N|N|N|N| +2434055|AAAAAAAAHAECFCAA|1952-02-12|625|2720|209|1952|2|2|12|1|1952|209|2720|Tuesday|1952Q1|N|N|N|2434044|2434074|2433690|2433963|N|N|N|N|N| +2434056|AAAAAAAAIAECFCAA|1952-02-13|625|2720|209|1952|3|2|13|1|1952|209|2720|Wednesday|1952Q1|N|N|N|2434044|2434074|2433691|2433964|N|N|N|N|N| +2434057|AAAAAAAAJAECFCAA|1952-02-14|625|2720|209|1952|4|2|14|1|1952|209|2720|Thursday|1952Q1|N|N|N|2434044|2434074|2433692|2433965|N|N|N|N|N| +2434058|AAAAAAAAKAECFCAA|1952-02-15|625|2720|209|1952|5|2|15|1|1952|209|2720|Friday|1952Q1|N|Y|N|2434044|2434074|2433693|2433966|N|N|N|N|N| +2434059|AAAAAAAALAECFCAA|1952-02-16|625|2720|209|1952|6|2|16|1|1952|209|2720|Saturday|1952Q1|N|Y|N|2434044|2434074|2433694|2433967|N|N|N|N|N| +2434060|AAAAAAAAMAECFCAA|1952-02-17|625|2720|209|1952|0|2|17|1|1952|209|2720|Sunday|1952Q1|N|N|N|2434044|2434074|2433695|2433968|N|N|N|N|N| +2434061|AAAAAAAANAECFCAA|1952-02-18|625|2720|209|1952|1|2|18|1|1952|209|2720|Monday|1952Q1|N|N|N|2434044|2434074|2433696|2433969|N|N|N|N|N| +2434062|AAAAAAAAOAECFCAA|1952-02-19|625|2721|209|1952|2|2|19|1|1952|209|2721|Tuesday|1952Q1|N|N|N|2434044|2434074|2433697|2433970|N|N|N|N|N| +2434063|AAAAAAAAPAECFCAA|1952-02-20|625|2721|209|1952|3|2|20|1|1952|209|2721|Wednesday|1952Q1|N|N|N|2434044|2434074|2433698|2433971|N|N|N|N|N| +2434064|AAAAAAAAABECFCAA|1952-02-21|625|2721|209|1952|4|2|21|1|1952|209|2721|Thursday|1952Q1|N|N|N|2434044|2434074|2433699|2433972|N|N|N|N|N| +2434065|AAAAAAAABBECFCAA|1952-02-22|625|2721|209|1952|5|2|22|1|1952|209|2721|Friday|1952Q1|N|Y|N|2434044|2434074|2433700|2433973|N|N|N|N|N| +2434066|AAAAAAAACBECFCAA|1952-02-23|625|2721|209|1952|6|2|23|1|1952|209|2721|Saturday|1952Q1|N|Y|N|2434044|2434074|2433701|2433974|N|N|N|N|N| +2434067|AAAAAAAADBECFCAA|1952-02-24|625|2721|209|1952|0|2|24|1|1952|209|2721|Sunday|1952Q1|N|N|N|2434044|2434074|2433702|2433975|N|N|N|N|N| +2434068|AAAAAAAAEBECFCAA|1952-02-25|625|2721|209|1952|1|2|25|1|1952|209|2721|Monday|1952Q1|N|N|N|2434044|2434074|2433703|2433976|N|N|N|N|N| +2434069|AAAAAAAAFBECFCAA|1952-02-26|625|2722|209|1952|2|2|26|1|1952|209|2722|Tuesday|1952Q1|N|N|N|2434044|2434074|2433704|2433977|N|N|N|N|N| +2434070|AAAAAAAAGBECFCAA|1952-02-27|625|2722|209|1952|3|2|27|1|1952|209|2722|Wednesday|1952Q1|N|N|N|2434044|2434074|2433705|2433978|N|N|N|N|N| +2434071|AAAAAAAAHBECFCAA|1952-02-28|625|2722|209|1952|4|2|28|1|1952|209|2722|Thursday|1952Q1|N|N|N|2434044|2434074|2433706|2433979|N|N|N|N|N| +2434072|AAAAAAAAIBECFCAA|1952-02-29|625|2722|209|1952|5|2|29|1|1952|209|2722|Friday|1952Q1|N|Y|N|2434044|2434074|2433706|2433980|N|N|N|N|N| +2434073|AAAAAAAAJBECFCAA|1952-03-01|626|2722|210|1952|6|3|1|1|1952|210|2722|Saturday|1952Q1|N|Y|N|2434073|2434132|2433707|2433981|N|N|N|N|N| +2434074|AAAAAAAAKBECFCAA|1952-03-02|626|2722|210|1952|0|3|2|1|1952|210|2722|Sunday|1952Q1|N|N|N|2434073|2434132|2433708|2433982|N|N|N|N|N| +2434075|AAAAAAAALBECFCAA|1952-03-03|626|2722|210|1952|1|3|3|1|1952|210|2722|Monday|1952Q1|N|N|N|2434073|2434132|2433709|2433983|N|N|N|N|N| +2434076|AAAAAAAAMBECFCAA|1952-03-04|626|2723|210|1952|2|3|4|1|1952|210|2723|Tuesday|1952Q1|N|N|N|2434073|2434132|2433710|2433984|N|N|N|N|N| +2434077|AAAAAAAANBECFCAA|1952-03-05|626|2723|210|1952|3|3|5|1|1952|210|2723|Wednesday|1952Q1|N|N|N|2434073|2434132|2433711|2433985|N|N|N|N|N| +2434078|AAAAAAAAOBECFCAA|1952-03-06|626|2723|210|1952|4|3|6|1|1952|210|2723|Thursday|1952Q1|N|N|N|2434073|2434132|2433712|2433986|N|N|N|N|N| +2434079|AAAAAAAAPBECFCAA|1952-03-07|626|2723|210|1952|5|3|7|1|1952|210|2723|Friday|1952Q1|N|Y|N|2434073|2434132|2433713|2433987|N|N|N|N|N| +2434080|AAAAAAAAACECFCAA|1952-03-08|626|2723|210|1952|6|3|8|1|1952|210|2723|Saturday|1952Q1|N|Y|N|2434073|2434132|2433714|2433988|N|N|N|N|N| +2434081|AAAAAAAABCECFCAA|1952-03-09|626|2723|210|1952|0|3|9|1|1952|210|2723|Sunday|1952Q1|N|N|N|2434073|2434132|2433715|2433989|N|N|N|N|N| +2434082|AAAAAAAACCECFCAA|1952-03-10|626|2723|210|1952|1|3|10|1|1952|210|2723|Monday|1952Q1|N|N|N|2434073|2434132|2433716|2433990|N|N|N|N|N| +2434083|AAAAAAAADCECFCAA|1952-03-11|626|2724|210|1952|2|3|11|1|1952|210|2724|Tuesday|1952Q1|N|N|N|2434073|2434132|2433717|2433991|N|N|N|N|N| +2434084|AAAAAAAAECECFCAA|1952-03-12|626|2724|210|1952|3|3|12|1|1952|210|2724|Wednesday|1952Q1|N|N|N|2434073|2434132|2433718|2433992|N|N|N|N|N| +2434085|AAAAAAAAFCECFCAA|1952-03-13|626|2724|210|1952|4|3|13|1|1952|210|2724|Thursday|1952Q1|N|N|N|2434073|2434132|2433719|2433993|N|N|N|N|N| +2434086|AAAAAAAAGCECFCAA|1952-03-14|626|2724|210|1952|5|3|14|1|1952|210|2724|Friday|1952Q1|N|Y|N|2434073|2434132|2433720|2433994|N|N|N|N|N| +2434087|AAAAAAAAHCECFCAA|1952-03-15|626|2724|210|1952|6|3|15|1|1952|210|2724|Saturday|1952Q1|N|Y|N|2434073|2434132|2433721|2433995|N|N|N|N|N| +2434088|AAAAAAAAICECFCAA|1952-03-16|626|2724|210|1952|0|3|16|1|1952|210|2724|Sunday|1952Q1|N|N|N|2434073|2434132|2433722|2433996|N|N|N|N|N| +2434089|AAAAAAAAJCECFCAA|1952-03-17|626|2724|210|1952|1|3|17|1|1952|210|2724|Monday|1952Q1|N|N|N|2434073|2434132|2433723|2433997|N|N|N|N|N| +2434090|AAAAAAAAKCECFCAA|1952-03-18|626|2725|210|1952|2|3|18|1|1952|210|2725|Tuesday|1952Q1|N|N|N|2434073|2434132|2433724|2433998|N|N|N|N|N| +2434091|AAAAAAAALCECFCAA|1952-03-19|626|2725|210|1952|3|3|19|1|1952|210|2725|Wednesday|1952Q1|N|N|N|2434073|2434132|2433725|2433999|N|N|N|N|N| +2434092|AAAAAAAAMCECFCAA|1952-03-20|626|2725|210|1952|4|3|20|1|1952|210|2725|Thursday|1952Q1|N|N|N|2434073|2434132|2433726|2434000|N|N|N|N|N| +2434093|AAAAAAAANCECFCAA|1952-03-21|626|2725|210|1952|5|3|21|1|1952|210|2725|Friday|1952Q1|N|Y|N|2434073|2434132|2433727|2434001|N|N|N|N|N| +2434094|AAAAAAAAOCECFCAA|1952-03-22|626|2725|210|1952|6|3|22|1|1952|210|2725|Saturday|1952Q1|N|Y|N|2434073|2434132|2433728|2434002|N|N|N|N|N| +2434095|AAAAAAAAPCECFCAA|1952-03-23|626|2725|210|1952|0|3|23|1|1952|210|2725|Sunday|1952Q1|N|N|N|2434073|2434132|2433729|2434003|N|N|N|N|N| +2434096|AAAAAAAAADECFCAA|1952-03-24|626|2725|210|1952|1|3|24|1|1952|210|2725|Monday|1952Q1|N|N|N|2434073|2434132|2433730|2434004|N|N|N|N|N| +2434097|AAAAAAAABDECFCAA|1952-03-25|626|2726|210|1952|2|3|25|1|1952|210|2726|Tuesday|1952Q1|N|N|N|2434073|2434132|2433731|2434005|N|N|N|N|N| +2434098|AAAAAAAACDECFCAA|1952-03-26|626|2726|210|1952|3|3|26|1|1952|210|2726|Wednesday|1952Q1|N|N|N|2434073|2434132|2433732|2434006|N|N|N|N|N| +2434099|AAAAAAAADDECFCAA|1952-03-27|626|2726|210|1952|4|3|27|1|1952|210|2726|Thursday|1952Q1|N|N|N|2434073|2434132|2433733|2434007|N|N|N|N|N| +2434100|AAAAAAAAEDECFCAA|1952-03-28|626|2726|210|1952|5|3|28|1|1952|210|2726|Friday|1952Q1|N|Y|N|2434073|2434132|2433734|2434008|N|N|N|N|N| +2434101|AAAAAAAAFDECFCAA|1952-03-29|626|2726|210|1952|6|3|29|1|1952|210|2726|Saturday|1952Q1|N|Y|N|2434073|2434132|2433735|2434009|N|N|N|N|N| +2434102|AAAAAAAAGDECFCAA|1952-03-30|626|2726|210|1952|0|3|30|1|1952|210|2726|Sunday|1952Q1|N|N|N|2434073|2434132|2433736|2434010|N|N|N|N|N| +2434103|AAAAAAAAHDECFCAA|1952-03-31|626|2726|210|1952|1|3|31|1|1952|210|2726|Monday|1952Q1|N|N|N|2434073|2434132|2433737|2434011|N|N|N|N|N| +2434104|AAAAAAAAIDECFCAA|1952-04-01|627|2727|210|1952|2|4|1|2|1952|210|2727|Tuesday|1952Q2|N|N|N|2434104|2434194|2433738|2434013|N|N|N|N|N| +2434105|AAAAAAAAJDECFCAA|1952-04-02|627|2727|210|1952|3|4|2|2|1952|210|2727|Wednesday|1952Q2|N|N|N|2434104|2434194|2433739|2434014|N|N|N|N|N| +2434106|AAAAAAAAKDECFCAA|1952-04-03|627|2727|210|1952|4|4|3|2|1952|210|2727|Thursday|1952Q2|N|N|N|2434104|2434194|2433740|2434015|N|N|N|N|N| +2434107|AAAAAAAALDECFCAA|1952-04-04|627|2727|210|1952|5|4|4|2|1952|210|2727|Friday|1952Q2|N|Y|N|2434104|2434194|2433741|2434016|N|N|N|N|N| +2434108|AAAAAAAAMDECFCAA|1952-04-05|627|2727|210|1952|6|4|5|2|1952|210|2727|Saturday|1952Q2|N|Y|N|2434104|2434194|2433742|2434017|N|N|N|N|N| +2434109|AAAAAAAANDECFCAA|1952-04-06|627|2727|210|1952|0|4|6|2|1952|210|2727|Sunday|1952Q2|N|N|N|2434104|2434194|2433743|2434018|N|N|N|N|N| +2434110|AAAAAAAAODECFCAA|1952-04-07|627|2727|210|1952|1|4|7|2|1952|210|2727|Monday|1952Q2|N|N|N|2434104|2434194|2433744|2434019|N|N|N|N|N| +2434111|AAAAAAAAPDECFCAA|1952-04-08|627|2728|210|1952|2|4|8|2|1952|210|2728|Tuesday|1952Q2|N|N|N|2434104|2434194|2433745|2434020|N|N|N|N|N| +2434112|AAAAAAAAAEECFCAA|1952-04-09|627|2728|210|1952|3|4|9|2|1952|210|2728|Wednesday|1952Q2|N|N|N|2434104|2434194|2433746|2434021|N|N|N|N|N| +2434113|AAAAAAAABEECFCAA|1952-04-10|627|2728|210|1952|4|4|10|2|1952|210|2728|Thursday|1952Q2|N|N|N|2434104|2434194|2433747|2434022|N|N|N|N|N| +2434114|AAAAAAAACEECFCAA|1952-04-11|627|2728|210|1952|5|4|11|2|1952|210|2728|Friday|1952Q2|N|Y|N|2434104|2434194|2433748|2434023|N|N|N|N|N| +2434115|AAAAAAAADEECFCAA|1952-04-12|627|2728|210|1952|6|4|12|2|1952|210|2728|Saturday|1952Q2|N|Y|N|2434104|2434194|2433749|2434024|N|N|N|N|N| +2434116|AAAAAAAAEEECFCAA|1952-04-13|627|2728|210|1952|0|4|13|2|1952|210|2728|Sunday|1952Q2|N|N|N|2434104|2434194|2433750|2434025|N|N|N|N|N| +2434117|AAAAAAAAFEECFCAA|1952-04-14|627|2728|210|1952|1|4|14|2|1952|210|2728|Monday|1952Q2|N|N|N|2434104|2434194|2433751|2434026|N|N|N|N|N| +2434118|AAAAAAAAGEECFCAA|1952-04-15|627|2729|210|1952|2|4|15|2|1952|210|2729|Tuesday|1952Q2|N|N|N|2434104|2434194|2433752|2434027|N|N|N|N|N| +2434119|AAAAAAAAHEECFCAA|1952-04-16|627|2729|210|1952|3|4|16|2|1952|210|2729|Wednesday|1952Q2|N|N|N|2434104|2434194|2433753|2434028|N|N|N|N|N| +2434120|AAAAAAAAIEECFCAA|1952-04-17|627|2729|210|1952|4|4|17|2|1952|210|2729|Thursday|1952Q2|N|N|N|2434104|2434194|2433754|2434029|N|N|N|N|N| +2434121|AAAAAAAAJEECFCAA|1952-04-18|627|2729|210|1952|5|4|18|2|1952|210|2729|Friday|1952Q2|N|Y|N|2434104|2434194|2433755|2434030|N|N|N|N|N| +2434122|AAAAAAAAKEECFCAA|1952-04-19|627|2729|210|1952|6|4|19|2|1952|210|2729|Saturday|1952Q2|N|Y|N|2434104|2434194|2433756|2434031|N|N|N|N|N| +2434123|AAAAAAAALEECFCAA|1952-04-20|627|2729|210|1952|0|4|20|2|1952|210|2729|Sunday|1952Q2|N|N|N|2434104|2434194|2433757|2434032|N|N|N|N|N| +2434124|AAAAAAAAMEECFCAA|1952-04-21|627|2729|210|1952|1|4|21|2|1952|210|2729|Monday|1952Q2|N|N|N|2434104|2434194|2433758|2434033|N|N|N|N|N| +2434125|AAAAAAAANEECFCAA|1952-04-22|627|2730|210|1952|2|4|22|2|1952|210|2730|Tuesday|1952Q2|N|N|N|2434104|2434194|2433759|2434034|N|N|N|N|N| +2434126|AAAAAAAAOEECFCAA|1952-04-23|627|2730|210|1952|3|4|23|2|1952|210|2730|Wednesday|1952Q2|N|N|N|2434104|2434194|2433760|2434035|N|N|N|N|N| +2434127|AAAAAAAAPEECFCAA|1952-04-24|627|2730|210|1952|4|4|24|2|1952|210|2730|Thursday|1952Q2|N|N|N|2434104|2434194|2433761|2434036|N|N|N|N|N| +2434128|AAAAAAAAAFECFCAA|1952-04-25|627|2730|210|1952|5|4|25|2|1952|210|2730|Friday|1952Q2|N|Y|N|2434104|2434194|2433762|2434037|N|N|N|N|N| +2434129|AAAAAAAABFECFCAA|1952-04-26|627|2730|210|1952|6|4|26|2|1952|210|2730|Saturday|1952Q2|N|Y|N|2434104|2434194|2433763|2434038|N|N|N|N|N| +2434130|AAAAAAAACFECFCAA|1952-04-27|627|2730|210|1952|0|4|27|2|1952|210|2730|Sunday|1952Q2|N|N|N|2434104|2434194|2433764|2434039|N|N|N|N|N| +2434131|AAAAAAAADFECFCAA|1952-04-28|627|2730|210|1952|1|4|28|2|1952|210|2730|Monday|1952Q2|N|N|N|2434104|2434194|2433765|2434040|N|N|N|N|N| +2434132|AAAAAAAAEFECFCAA|1952-04-29|627|2731|210|1952|2|4|29|2|1952|210|2731|Tuesday|1952Q2|N|N|N|2434104|2434194|2433766|2434041|N|N|N|N|N| +2434133|AAAAAAAAFFECFCAA|1952-04-30|627|2731|210|1952|3|4|30|2|1952|210|2731|Wednesday|1952Q2|N|N|N|2434104|2434194|2433767|2434042|N|N|N|N|N| +2434134|AAAAAAAAGFECFCAA|1952-05-01|628|2731|210|1952|4|5|1|2|1952|210|2731|Thursday|1952Q2|N|N|N|2434134|2434254|2433768|2434043|N|N|N|N|N| +2434135|AAAAAAAAHFECFCAA|1952-05-02|628|2731|210|1952|5|5|2|2|1952|210|2731|Friday|1952Q2|N|Y|N|2434134|2434254|2433769|2434044|N|N|N|N|N| +2434136|AAAAAAAAIFECFCAA|1952-05-03|628|2731|210|1952|6|5|3|2|1952|210|2731|Saturday|1952Q2|N|Y|N|2434134|2434254|2433770|2434045|N|N|N|N|N| +2434137|AAAAAAAAJFECFCAA|1952-05-04|628|2731|210|1952|0|5|4|2|1952|210|2731|Sunday|1952Q2|N|N|N|2434134|2434254|2433771|2434046|N|N|N|N|N| +2434138|AAAAAAAAKFECFCAA|1952-05-05|628|2731|210|1952|1|5|5|2|1952|210|2731|Monday|1952Q2|N|N|N|2434134|2434254|2433772|2434047|N|N|N|N|N| +2434139|AAAAAAAALFECFCAA|1952-05-06|628|2732|210|1952|2|5|6|2|1952|210|2732|Tuesday|1952Q2|N|N|N|2434134|2434254|2433773|2434048|N|N|N|N|N| +2434140|AAAAAAAAMFECFCAA|1952-05-07|628|2732|210|1952|3|5|7|2|1952|210|2732|Wednesday|1952Q2|N|N|N|2434134|2434254|2433774|2434049|N|N|N|N|N| +2434141|AAAAAAAANFECFCAA|1952-05-08|628|2732|210|1952|4|5|8|2|1952|210|2732|Thursday|1952Q2|N|N|N|2434134|2434254|2433775|2434050|N|N|N|N|N| +2434142|AAAAAAAAOFECFCAA|1952-05-09|628|2732|210|1952|5|5|9|2|1952|210|2732|Friday|1952Q2|N|Y|N|2434134|2434254|2433776|2434051|N|N|N|N|N| +2434143|AAAAAAAAPFECFCAA|1952-05-10|628|2732|210|1952|6|5|10|2|1952|210|2732|Saturday|1952Q2|N|Y|N|2434134|2434254|2433777|2434052|N|N|N|N|N| +2434144|AAAAAAAAAGECFCAA|1952-05-11|628|2732|210|1952|0|5|11|2|1952|210|2732|Sunday|1952Q2|N|N|N|2434134|2434254|2433778|2434053|N|N|N|N|N| +2434145|AAAAAAAABGECFCAA|1952-05-12|628|2732|210|1952|1|5|12|2|1952|210|2732|Monday|1952Q2|N|N|N|2434134|2434254|2433779|2434054|N|N|N|N|N| +2434146|AAAAAAAACGECFCAA|1952-05-13|628|2733|210|1952|2|5|13|2|1952|210|2733|Tuesday|1952Q2|N|N|N|2434134|2434254|2433780|2434055|N|N|N|N|N| +2434147|AAAAAAAADGECFCAA|1952-05-14|628|2733|210|1952|3|5|14|2|1952|210|2733|Wednesday|1952Q2|N|N|N|2434134|2434254|2433781|2434056|N|N|N|N|N| +2434148|AAAAAAAAEGECFCAA|1952-05-15|628|2733|210|1952|4|5|15|2|1952|210|2733|Thursday|1952Q2|N|N|N|2434134|2434254|2433782|2434057|N|N|N|N|N| +2434149|AAAAAAAAFGECFCAA|1952-05-16|628|2733|210|1952|5|5|16|2|1952|210|2733|Friday|1952Q2|N|Y|N|2434134|2434254|2433783|2434058|N|N|N|N|N| +2434150|AAAAAAAAGGECFCAA|1952-05-17|628|2733|210|1952|6|5|17|2|1952|210|2733|Saturday|1952Q2|N|Y|N|2434134|2434254|2433784|2434059|N|N|N|N|N| +2434151|AAAAAAAAHGECFCAA|1952-05-18|628|2733|210|1952|0|5|18|2|1952|210|2733|Sunday|1952Q2|N|N|N|2434134|2434254|2433785|2434060|N|N|N|N|N| +2434152|AAAAAAAAIGECFCAA|1952-05-19|628|2733|210|1952|1|5|19|2|1952|210|2733|Monday|1952Q2|N|N|N|2434134|2434254|2433786|2434061|N|N|N|N|N| +2434153|AAAAAAAAJGECFCAA|1952-05-20|628|2734|210|1952|2|5|20|2|1952|210|2734|Tuesday|1952Q2|N|N|N|2434134|2434254|2433787|2434062|N|N|N|N|N| +2434154|AAAAAAAAKGECFCAA|1952-05-21|628|2734|210|1952|3|5|21|2|1952|210|2734|Wednesday|1952Q2|N|N|N|2434134|2434254|2433788|2434063|N|N|N|N|N| +2434155|AAAAAAAALGECFCAA|1952-05-22|628|2734|210|1952|4|5|22|2|1952|210|2734|Thursday|1952Q2|N|N|N|2434134|2434254|2433789|2434064|N|N|N|N|N| +2434156|AAAAAAAAMGECFCAA|1952-05-23|628|2734|210|1952|5|5|23|2|1952|210|2734|Friday|1952Q2|N|Y|N|2434134|2434254|2433790|2434065|N|N|N|N|N| +2434157|AAAAAAAANGECFCAA|1952-05-24|628|2734|210|1952|6|5|24|2|1952|210|2734|Saturday|1952Q2|N|Y|N|2434134|2434254|2433791|2434066|N|N|N|N|N| +2434158|AAAAAAAAOGECFCAA|1952-05-25|628|2734|210|1952|0|5|25|2|1952|210|2734|Sunday|1952Q2|N|N|N|2434134|2434254|2433792|2434067|N|N|N|N|N| +2434159|AAAAAAAAPGECFCAA|1952-05-26|628|2734|210|1952|1|5|26|2|1952|210|2734|Monday|1952Q2|N|N|N|2434134|2434254|2433793|2434068|N|N|N|N|N| +2434160|AAAAAAAAAHECFCAA|1952-05-27|628|2735|210|1952|2|5|27|2|1952|210|2735|Tuesday|1952Q2|N|N|N|2434134|2434254|2433794|2434069|N|N|N|N|N| +2434161|AAAAAAAABHECFCAA|1952-05-28|628|2735|210|1952|3|5|28|2|1952|210|2735|Wednesday|1952Q2|N|N|N|2434134|2434254|2433795|2434070|N|N|N|N|N| +2434162|AAAAAAAACHECFCAA|1952-05-29|628|2735|210|1952|4|5|29|2|1952|210|2735|Thursday|1952Q2|N|N|N|2434134|2434254|2433796|2434071|N|N|N|N|N| +2434163|AAAAAAAADHECFCAA|1952-05-30|628|2735|210|1952|5|5|30|2|1952|210|2735|Friday|1952Q2|N|Y|N|2434134|2434254|2433797|2434072|N|N|N|N|N| +2434164|AAAAAAAAEHECFCAA|1952-05-31|628|2735|210|1952|6|5|31|2|1952|210|2735|Saturday|1952Q2|N|Y|N|2434134|2434254|2433798|2434073|N|N|N|N|N| +2434165|AAAAAAAAFHECFCAA|1952-06-01|629|2735|211|1952|0|6|1|2|1952|211|2735|Sunday|1952Q2|N|N|N|2434165|2434316|2433799|2434074|N|N|N|N|N| +2434166|AAAAAAAAGHECFCAA|1952-06-02|629|2735|211|1952|1|6|2|2|1952|211|2735|Monday|1952Q2|N|N|N|2434165|2434316|2433800|2434075|N|N|N|N|N| +2434167|AAAAAAAAHHECFCAA|1952-06-03|629|2736|211|1952|2|6|3|2|1952|211|2736|Tuesday|1952Q2|N|N|N|2434165|2434316|2433801|2434076|N|N|N|N|N| +2434168|AAAAAAAAIHECFCAA|1952-06-04|629|2736|211|1952|3|6|4|2|1952|211|2736|Wednesday|1952Q2|N|N|N|2434165|2434316|2433802|2434077|N|N|N|N|N| +2434169|AAAAAAAAJHECFCAA|1952-06-05|629|2736|211|1952|4|6|5|2|1952|211|2736|Thursday|1952Q2|N|N|N|2434165|2434316|2433803|2434078|N|N|N|N|N| +2434170|AAAAAAAAKHECFCAA|1952-06-06|629|2736|211|1952|5|6|6|2|1952|211|2736|Friday|1952Q2|N|Y|N|2434165|2434316|2433804|2434079|N|N|N|N|N| +2434171|AAAAAAAALHECFCAA|1952-06-07|629|2736|211|1952|6|6|7|2|1952|211|2736|Saturday|1952Q2|N|Y|N|2434165|2434316|2433805|2434080|N|N|N|N|N| +2434172|AAAAAAAAMHECFCAA|1952-06-08|629|2736|211|1952|0|6|8|2|1952|211|2736|Sunday|1952Q2|N|N|N|2434165|2434316|2433806|2434081|N|N|N|N|N| +2434173|AAAAAAAANHECFCAA|1952-06-09|629|2736|211|1952|1|6|9|2|1952|211|2736|Monday|1952Q2|N|N|N|2434165|2434316|2433807|2434082|N|N|N|N|N| +2434174|AAAAAAAAOHECFCAA|1952-06-10|629|2737|211|1952|2|6|10|2|1952|211|2737|Tuesday|1952Q2|N|N|N|2434165|2434316|2433808|2434083|N|N|N|N|N| +2434175|AAAAAAAAPHECFCAA|1952-06-11|629|2737|211|1952|3|6|11|2|1952|211|2737|Wednesday|1952Q2|N|N|N|2434165|2434316|2433809|2434084|N|N|N|N|N| +2434176|AAAAAAAAAIECFCAA|1952-06-12|629|2737|211|1952|4|6|12|2|1952|211|2737|Thursday|1952Q2|N|N|N|2434165|2434316|2433810|2434085|N|N|N|N|N| +2434177|AAAAAAAABIECFCAA|1952-06-13|629|2737|211|1952|5|6|13|2|1952|211|2737|Friday|1952Q2|N|Y|N|2434165|2434316|2433811|2434086|N|N|N|N|N| +2434178|AAAAAAAACIECFCAA|1952-06-14|629|2737|211|1952|6|6|14|2|1952|211|2737|Saturday|1952Q2|N|Y|N|2434165|2434316|2433812|2434087|N|N|N|N|N| +2434179|AAAAAAAADIECFCAA|1952-06-15|629|2737|211|1952|0|6|15|2|1952|211|2737|Sunday|1952Q2|N|N|N|2434165|2434316|2433813|2434088|N|N|N|N|N| +2434180|AAAAAAAAEIECFCAA|1952-06-16|629|2737|211|1952|1|6|16|2|1952|211|2737|Monday|1952Q2|N|N|N|2434165|2434316|2433814|2434089|N|N|N|N|N| +2434181|AAAAAAAAFIECFCAA|1952-06-17|629|2738|211|1952|2|6|17|2|1952|211|2738|Tuesday|1952Q2|N|N|N|2434165|2434316|2433815|2434090|N|N|N|N|N| +2434182|AAAAAAAAGIECFCAA|1952-06-18|629|2738|211|1952|3|6|18|2|1952|211|2738|Wednesday|1952Q2|N|N|N|2434165|2434316|2433816|2434091|N|N|N|N|N| +2434183|AAAAAAAAHIECFCAA|1952-06-19|629|2738|211|1952|4|6|19|2|1952|211|2738|Thursday|1952Q2|N|N|N|2434165|2434316|2433817|2434092|N|N|N|N|N| +2434184|AAAAAAAAIIECFCAA|1952-06-20|629|2738|211|1952|5|6|20|2|1952|211|2738|Friday|1952Q2|N|Y|N|2434165|2434316|2433818|2434093|N|N|N|N|N| +2434185|AAAAAAAAJIECFCAA|1952-06-21|629|2738|211|1952|6|6|21|2|1952|211|2738|Saturday|1952Q2|N|Y|N|2434165|2434316|2433819|2434094|N|N|N|N|N| +2434186|AAAAAAAAKIECFCAA|1952-06-22|629|2738|211|1952|0|6|22|2|1952|211|2738|Sunday|1952Q2|N|N|N|2434165|2434316|2433820|2434095|N|N|N|N|N| +2434187|AAAAAAAALIECFCAA|1952-06-23|629|2738|211|1952|1|6|23|2|1952|211|2738|Monday|1952Q2|N|N|N|2434165|2434316|2433821|2434096|N|N|N|N|N| +2434188|AAAAAAAAMIECFCAA|1952-06-24|629|2739|211|1952|2|6|24|2|1952|211|2739|Tuesday|1952Q2|N|N|N|2434165|2434316|2433822|2434097|N|N|N|N|N| +2434189|AAAAAAAANIECFCAA|1952-06-25|629|2739|211|1952|3|6|25|2|1952|211|2739|Wednesday|1952Q2|N|N|N|2434165|2434316|2433823|2434098|N|N|N|N|N| +2434190|AAAAAAAAOIECFCAA|1952-06-26|629|2739|211|1952|4|6|26|2|1952|211|2739|Thursday|1952Q2|N|N|N|2434165|2434316|2433824|2434099|N|N|N|N|N| +2434191|AAAAAAAAPIECFCAA|1952-06-27|629|2739|211|1952|5|6|27|2|1952|211|2739|Friday|1952Q2|N|Y|N|2434165|2434316|2433825|2434100|N|N|N|N|N| +2434192|AAAAAAAAAJECFCAA|1952-06-28|629|2739|211|1952|6|6|28|2|1952|211|2739|Saturday|1952Q2|N|Y|N|2434165|2434316|2433826|2434101|N|N|N|N|N| +2434193|AAAAAAAABJECFCAA|1952-06-29|629|2739|211|1952|0|6|29|2|1952|211|2739|Sunday|1952Q2|N|N|N|2434165|2434316|2433827|2434102|N|N|N|N|N| +2434194|AAAAAAAACJECFCAA|1952-06-30|629|2739|211|1952|1|6|30|2|1952|211|2739|Monday|1952Q2|N|N|N|2434165|2434316|2433828|2434103|N|N|N|N|N| +2434195|AAAAAAAADJECFCAA|1952-07-01|630|2740|211|1952|2|7|1|3|1952|211|2740|Tuesday|1952Q3|N|N|N|2434195|2434376|2433829|2434104|N|N|N|N|N| +2434196|AAAAAAAAEJECFCAA|1952-07-02|630|2740|211|1952|3|7|2|3|1952|211|2740|Wednesday|1952Q3|N|N|N|2434195|2434376|2433830|2434105|N|N|N|N|N| +2434197|AAAAAAAAFJECFCAA|1952-07-03|630|2740|211|1952|4|7|3|3|1952|211|2740|Thursday|1952Q3|N|N|N|2434195|2434376|2433831|2434106|N|N|N|N|N| +2434198|AAAAAAAAGJECFCAA|1952-07-04|630|2740|211|1952|5|7|4|3|1952|211|2740|Friday|1952Q3|Y|Y|N|2434195|2434376|2433832|2434107|N|N|N|N|N| +2434199|AAAAAAAAHJECFCAA|1952-07-05|630|2740|211|1952|6|7|5|3|1952|211|2740|Saturday|1952Q3|N|Y|Y|2434195|2434376|2433833|2434108|N|N|N|N|N| +2434200|AAAAAAAAIJECFCAA|1952-07-06|630|2740|211|1952|0|7|6|3|1952|211|2740|Sunday|1952Q3|N|N|N|2434195|2434376|2433834|2434109|N|N|N|N|N| +2434201|AAAAAAAAJJECFCAA|1952-07-07|630|2740|211|1952|1|7|7|3|1952|211|2740|Monday|1952Q3|N|N|N|2434195|2434376|2433835|2434110|N|N|N|N|N| +2434202|AAAAAAAAKJECFCAA|1952-07-08|630|2741|211|1952|2|7|8|3|1952|211|2741|Tuesday|1952Q3|N|N|N|2434195|2434376|2433836|2434111|N|N|N|N|N| +2434203|AAAAAAAALJECFCAA|1952-07-09|630|2741|211|1952|3|7|9|3|1952|211|2741|Wednesday|1952Q3|N|N|N|2434195|2434376|2433837|2434112|N|N|N|N|N| +2434204|AAAAAAAAMJECFCAA|1952-07-10|630|2741|211|1952|4|7|10|3|1952|211|2741|Thursday|1952Q3|N|N|N|2434195|2434376|2433838|2434113|N|N|N|N|N| +2434205|AAAAAAAANJECFCAA|1952-07-11|630|2741|211|1952|5|7|11|3|1952|211|2741|Friday|1952Q3|N|Y|N|2434195|2434376|2433839|2434114|N|N|N|N|N| +2434206|AAAAAAAAOJECFCAA|1952-07-12|630|2741|211|1952|6|7|12|3|1952|211|2741|Saturday|1952Q3|N|Y|N|2434195|2434376|2433840|2434115|N|N|N|N|N| +2434207|AAAAAAAAPJECFCAA|1952-07-13|630|2741|211|1952|0|7|13|3|1952|211|2741|Sunday|1952Q3|N|N|N|2434195|2434376|2433841|2434116|N|N|N|N|N| +2434208|AAAAAAAAAKECFCAA|1952-07-14|630|2741|211|1952|1|7|14|3|1952|211|2741|Monday|1952Q3|N|N|N|2434195|2434376|2433842|2434117|N|N|N|N|N| +2434209|AAAAAAAABKECFCAA|1952-07-15|630|2742|211|1952|2|7|15|3|1952|211|2742|Tuesday|1952Q3|N|N|N|2434195|2434376|2433843|2434118|N|N|N|N|N| +2434210|AAAAAAAACKECFCAA|1952-07-16|630|2742|211|1952|3|7|16|3|1952|211|2742|Wednesday|1952Q3|N|N|N|2434195|2434376|2433844|2434119|N|N|N|N|N| +2434211|AAAAAAAADKECFCAA|1952-07-17|630|2742|211|1952|4|7|17|3|1952|211|2742|Thursday|1952Q3|N|N|N|2434195|2434376|2433845|2434120|N|N|N|N|N| +2434212|AAAAAAAAEKECFCAA|1952-07-18|630|2742|211|1952|5|7|18|3|1952|211|2742|Friday|1952Q3|N|Y|N|2434195|2434376|2433846|2434121|N|N|N|N|N| +2434213|AAAAAAAAFKECFCAA|1952-07-19|630|2742|211|1952|6|7|19|3|1952|211|2742|Saturday|1952Q3|N|Y|N|2434195|2434376|2433847|2434122|N|N|N|N|N| +2434214|AAAAAAAAGKECFCAA|1952-07-20|630|2742|211|1952|0|7|20|3|1952|211|2742|Sunday|1952Q3|N|N|N|2434195|2434376|2433848|2434123|N|N|N|N|N| +2434215|AAAAAAAAHKECFCAA|1952-07-21|630|2742|211|1952|1|7|21|3|1952|211|2742|Monday|1952Q3|N|N|N|2434195|2434376|2433849|2434124|N|N|N|N|N| +2434216|AAAAAAAAIKECFCAA|1952-07-22|630|2743|211|1952|2|7|22|3|1952|211|2743|Tuesday|1952Q3|N|N|N|2434195|2434376|2433850|2434125|N|N|N|N|N| +2434217|AAAAAAAAJKECFCAA|1952-07-23|630|2743|211|1952|3|7|23|3|1952|211|2743|Wednesday|1952Q3|N|N|N|2434195|2434376|2433851|2434126|N|N|N|N|N| +2434218|AAAAAAAAKKECFCAA|1952-07-24|630|2743|211|1952|4|7|24|3|1952|211|2743|Thursday|1952Q3|N|N|N|2434195|2434376|2433852|2434127|N|N|N|N|N| +2434219|AAAAAAAALKECFCAA|1952-07-25|630|2743|211|1952|5|7|25|3|1952|211|2743|Friday|1952Q3|N|Y|N|2434195|2434376|2433853|2434128|N|N|N|N|N| +2434220|AAAAAAAAMKECFCAA|1952-07-26|630|2743|211|1952|6|7|26|3|1952|211|2743|Saturday|1952Q3|N|Y|N|2434195|2434376|2433854|2434129|N|N|N|N|N| +2434221|AAAAAAAANKECFCAA|1952-07-27|630|2743|211|1952|0|7|27|3|1952|211|2743|Sunday|1952Q3|N|N|N|2434195|2434376|2433855|2434130|N|N|N|N|N| +2434222|AAAAAAAAOKECFCAA|1952-07-28|630|2743|211|1952|1|7|28|3|1952|211|2743|Monday|1952Q3|N|N|N|2434195|2434376|2433856|2434131|N|N|N|N|N| +2434223|AAAAAAAAPKECFCAA|1952-07-29|630|2744|211|1952|2|7|29|3|1952|211|2744|Tuesday|1952Q3|N|N|N|2434195|2434376|2433857|2434132|N|N|N|N|N| +2434224|AAAAAAAAALECFCAA|1952-07-30|630|2744|211|1952|3|7|30|3|1952|211|2744|Wednesday|1952Q3|N|N|N|2434195|2434376|2433858|2434133|N|N|N|N|N| +2434225|AAAAAAAABLECFCAA|1952-07-31|630|2744|211|1952|4|7|31|3|1952|211|2744|Thursday|1952Q3|N|N|N|2434195|2434376|2433859|2434134|N|N|N|N|N| +2434226|AAAAAAAACLECFCAA|1952-08-01|631|2744|211|1952|5|8|1|3|1952|211|2744|Friday|1952Q3|N|Y|N|2434226|2434438|2433860|2434135|N|N|N|N|N| +2434227|AAAAAAAADLECFCAA|1952-08-02|631|2744|211|1952|6|8|2|3|1952|211|2744|Saturday|1952Q3|N|Y|N|2434226|2434438|2433861|2434136|N|N|N|N|N| +2434228|AAAAAAAAELECFCAA|1952-08-03|631|2744|211|1952|0|8|3|3|1952|211|2744|Sunday|1952Q3|N|N|N|2434226|2434438|2433862|2434137|N|N|N|N|N| +2434229|AAAAAAAAFLECFCAA|1952-08-04|631|2744|211|1952|1|8|4|3|1952|211|2744|Monday|1952Q3|N|N|N|2434226|2434438|2433863|2434138|N|N|N|N|N| +2434230|AAAAAAAAGLECFCAA|1952-08-05|631|2745|211|1952|2|8|5|3|1952|211|2745|Tuesday|1952Q3|N|N|N|2434226|2434438|2433864|2434139|N|N|N|N|N| +2434231|AAAAAAAAHLECFCAA|1952-08-06|631|2745|211|1952|3|8|6|3|1952|211|2745|Wednesday|1952Q3|N|N|N|2434226|2434438|2433865|2434140|N|N|N|N|N| +2434232|AAAAAAAAILECFCAA|1952-08-07|631|2745|211|1952|4|8|7|3|1952|211|2745|Thursday|1952Q3|N|N|N|2434226|2434438|2433866|2434141|N|N|N|N|N| +2434233|AAAAAAAAJLECFCAA|1952-08-08|631|2745|211|1952|5|8|8|3|1952|211|2745|Friday|1952Q3|N|Y|N|2434226|2434438|2433867|2434142|N|N|N|N|N| +2434234|AAAAAAAAKLECFCAA|1952-08-09|631|2745|211|1952|6|8|9|3|1952|211|2745|Saturday|1952Q3|N|Y|N|2434226|2434438|2433868|2434143|N|N|N|N|N| +2434235|AAAAAAAALLECFCAA|1952-08-10|631|2745|211|1952|0|8|10|3|1952|211|2745|Sunday|1952Q3|N|N|N|2434226|2434438|2433869|2434144|N|N|N|N|N| +2434236|AAAAAAAAMLECFCAA|1952-08-11|631|2745|211|1952|1|8|11|3|1952|211|2745|Monday|1952Q3|N|N|N|2434226|2434438|2433870|2434145|N|N|N|N|N| +2434237|AAAAAAAANLECFCAA|1952-08-12|631|2746|211|1952|2|8|12|3|1952|211|2746|Tuesday|1952Q3|N|N|N|2434226|2434438|2433871|2434146|N|N|N|N|N| +2434238|AAAAAAAAOLECFCAA|1952-08-13|631|2746|211|1952|3|8|13|3|1952|211|2746|Wednesday|1952Q3|N|N|N|2434226|2434438|2433872|2434147|N|N|N|N|N| +2434239|AAAAAAAAPLECFCAA|1952-08-14|631|2746|211|1952|4|8|14|3|1952|211|2746|Thursday|1952Q3|N|N|N|2434226|2434438|2433873|2434148|N|N|N|N|N| +2434240|AAAAAAAAAMECFCAA|1952-08-15|631|2746|211|1952|5|8|15|3|1952|211|2746|Friday|1952Q3|N|Y|N|2434226|2434438|2433874|2434149|N|N|N|N|N| +2434241|AAAAAAAABMECFCAA|1952-08-16|631|2746|211|1952|6|8|16|3|1952|211|2746|Saturday|1952Q3|N|Y|N|2434226|2434438|2433875|2434150|N|N|N|N|N| +2434242|AAAAAAAACMECFCAA|1952-08-17|631|2746|211|1952|0|8|17|3|1952|211|2746|Sunday|1952Q3|N|N|N|2434226|2434438|2433876|2434151|N|N|N|N|N| +2434243|AAAAAAAADMECFCAA|1952-08-18|631|2746|211|1952|1|8|18|3|1952|211|2746|Monday|1952Q3|N|N|N|2434226|2434438|2433877|2434152|N|N|N|N|N| +2434244|AAAAAAAAEMECFCAA|1952-08-19|631|2747|211|1952|2|8|19|3|1952|211|2747|Tuesday|1952Q3|N|N|N|2434226|2434438|2433878|2434153|N|N|N|N|N| +2434245|AAAAAAAAFMECFCAA|1952-08-20|631|2747|211|1952|3|8|20|3|1952|211|2747|Wednesday|1952Q3|N|N|N|2434226|2434438|2433879|2434154|N|N|N|N|N| +2434246|AAAAAAAAGMECFCAA|1952-08-21|631|2747|211|1952|4|8|21|3|1952|211|2747|Thursday|1952Q3|N|N|N|2434226|2434438|2433880|2434155|N|N|N|N|N| +2434247|AAAAAAAAHMECFCAA|1952-08-22|631|2747|211|1952|5|8|22|3|1952|211|2747|Friday|1952Q3|N|Y|N|2434226|2434438|2433881|2434156|N|N|N|N|N| +2434248|AAAAAAAAIMECFCAA|1952-08-23|631|2747|211|1952|6|8|23|3|1952|211|2747|Saturday|1952Q3|N|Y|N|2434226|2434438|2433882|2434157|N|N|N|N|N| +2434249|AAAAAAAAJMECFCAA|1952-08-24|631|2747|211|1952|0|8|24|3|1952|211|2747|Sunday|1952Q3|N|N|N|2434226|2434438|2433883|2434158|N|N|N|N|N| +2434250|AAAAAAAAKMECFCAA|1952-08-25|631|2747|211|1952|1|8|25|3|1952|211|2747|Monday|1952Q3|N|N|N|2434226|2434438|2433884|2434159|N|N|N|N|N| +2434251|AAAAAAAALMECFCAA|1952-08-26|631|2748|211|1952|2|8|26|3|1952|211|2748|Tuesday|1952Q3|N|N|N|2434226|2434438|2433885|2434160|N|N|N|N|N| +2434252|AAAAAAAAMMECFCAA|1952-08-27|631|2748|211|1952|3|8|27|3|1952|211|2748|Wednesday|1952Q3|N|N|N|2434226|2434438|2433886|2434161|N|N|N|N|N| +2434253|AAAAAAAANMECFCAA|1952-08-28|631|2748|211|1952|4|8|28|3|1952|211|2748|Thursday|1952Q3|N|N|N|2434226|2434438|2433887|2434162|N|N|N|N|N| +2434254|AAAAAAAAOMECFCAA|1952-08-29|631|2748|211|1952|5|8|29|3|1952|211|2748|Friday|1952Q3|N|Y|N|2434226|2434438|2433888|2434163|N|N|N|N|N| +2434255|AAAAAAAAPMECFCAA|1952-08-30|631|2748|211|1952|6|8|30|3|1952|211|2748|Saturday|1952Q3|N|Y|N|2434226|2434438|2433889|2434164|N|N|N|N|N| +2434256|AAAAAAAAANECFCAA|1952-08-31|631|2748|211|1952|0|8|31|3|1952|211|2748|Sunday|1952Q3|N|N|N|2434226|2434438|2433890|2434165|N|N|N|N|N| +2434257|AAAAAAAABNECFCAA|1952-09-01|632|2748|212|1952|1|9|1|3|1952|212|2748|Monday|1952Q3|N|N|N|2434257|2434500|2433891|2434166|N|N|N|N|N| +2434258|AAAAAAAACNECFCAA|1952-09-02|632|2749|212|1952|2|9|2|3|1952|212|2749|Tuesday|1952Q3|N|N|N|2434257|2434500|2433892|2434167|N|N|N|N|N| +2434259|AAAAAAAADNECFCAA|1952-09-03|632|2749|212|1952|3|9|3|3|1952|212|2749|Wednesday|1952Q3|N|N|N|2434257|2434500|2433893|2434168|N|N|N|N|N| +2434260|AAAAAAAAENECFCAA|1952-09-04|632|2749|212|1952|4|9|4|3|1952|212|2749|Thursday|1952Q3|N|N|N|2434257|2434500|2433894|2434169|N|N|N|N|N| +2434261|AAAAAAAAFNECFCAA|1952-09-05|632|2749|212|1952|5|9|5|3|1952|212|2749|Friday|1952Q3|N|Y|N|2434257|2434500|2433895|2434170|N|N|N|N|N| +2434262|AAAAAAAAGNECFCAA|1952-09-06|632|2749|212|1952|6|9|6|3|1952|212|2749|Saturday|1952Q3|N|Y|N|2434257|2434500|2433896|2434171|N|N|N|N|N| +2434263|AAAAAAAAHNECFCAA|1952-09-07|632|2749|212|1952|0|9|7|3|1952|212|2749|Sunday|1952Q3|N|N|N|2434257|2434500|2433897|2434172|N|N|N|N|N| +2434264|AAAAAAAAINECFCAA|1952-09-08|632|2749|212|1952|1|9|8|3|1952|212|2749|Monday|1952Q3|N|N|N|2434257|2434500|2433898|2434173|N|N|N|N|N| +2434265|AAAAAAAAJNECFCAA|1952-09-09|632|2750|212|1952|2|9|9|3|1952|212|2750|Tuesday|1952Q3|N|N|N|2434257|2434500|2433899|2434174|N|N|N|N|N| +2434266|AAAAAAAAKNECFCAA|1952-09-10|632|2750|212|1952|3|9|10|3|1952|212|2750|Wednesday|1952Q3|N|N|N|2434257|2434500|2433900|2434175|N|N|N|N|N| +2434267|AAAAAAAALNECFCAA|1952-09-11|632|2750|212|1952|4|9|11|3|1952|212|2750|Thursday|1952Q3|N|N|N|2434257|2434500|2433901|2434176|N|N|N|N|N| +2434268|AAAAAAAAMNECFCAA|1952-09-12|632|2750|212|1952|5|9|12|3|1952|212|2750|Friday|1952Q3|N|Y|N|2434257|2434500|2433902|2434177|N|N|N|N|N| +2434269|AAAAAAAANNECFCAA|1952-09-13|632|2750|212|1952|6|9|13|3|1952|212|2750|Saturday|1952Q3|N|Y|N|2434257|2434500|2433903|2434178|N|N|N|N|N| +2434270|AAAAAAAAONECFCAA|1952-09-14|632|2750|212|1952|0|9|14|3|1952|212|2750|Sunday|1952Q3|N|N|N|2434257|2434500|2433904|2434179|N|N|N|N|N| +2434271|AAAAAAAAPNECFCAA|1952-09-15|632|2750|212|1952|1|9|15|3|1952|212|2750|Monday|1952Q3|N|N|N|2434257|2434500|2433905|2434180|N|N|N|N|N| +2434272|AAAAAAAAAOECFCAA|1952-09-16|632|2751|212|1952|2|9|16|3|1952|212|2751|Tuesday|1952Q3|N|N|N|2434257|2434500|2433906|2434181|N|N|N|N|N| +2434273|AAAAAAAABOECFCAA|1952-09-17|632|2751|212|1952|3|9|17|3|1952|212|2751|Wednesday|1952Q3|N|N|N|2434257|2434500|2433907|2434182|N|N|N|N|N| +2434274|AAAAAAAACOECFCAA|1952-09-18|632|2751|212|1952|4|9|18|3|1952|212|2751|Thursday|1952Q3|N|N|N|2434257|2434500|2433908|2434183|N|N|N|N|N| +2434275|AAAAAAAADOECFCAA|1952-09-19|632|2751|212|1952|5|9|19|3|1952|212|2751|Friday|1952Q3|N|Y|N|2434257|2434500|2433909|2434184|N|N|N|N|N| +2434276|AAAAAAAAEOECFCAA|1952-09-20|632|2751|212|1952|6|9|20|3|1952|212|2751|Saturday|1952Q3|N|Y|N|2434257|2434500|2433910|2434185|N|N|N|N|N| +2434277|AAAAAAAAFOECFCAA|1952-09-21|632|2751|212|1952|0|9|21|3|1952|212|2751|Sunday|1952Q3|N|N|N|2434257|2434500|2433911|2434186|N|N|N|N|N| +2434278|AAAAAAAAGOECFCAA|1952-09-22|632|2751|212|1952|1|9|22|3|1952|212|2751|Monday|1952Q3|N|N|N|2434257|2434500|2433912|2434187|N|N|N|N|N| +2434279|AAAAAAAAHOECFCAA|1952-09-23|632|2752|212|1952|2|9|23|3|1952|212|2752|Tuesday|1952Q3|N|N|N|2434257|2434500|2433913|2434188|N|N|N|N|N| +2434280|AAAAAAAAIOECFCAA|1952-09-24|632|2752|212|1952|3|9|24|3|1952|212|2752|Wednesday|1952Q3|N|N|N|2434257|2434500|2433914|2434189|N|N|N|N|N| +2434281|AAAAAAAAJOECFCAA|1952-09-25|632|2752|212|1952|4|9|25|3|1952|212|2752|Thursday|1952Q3|N|N|N|2434257|2434500|2433915|2434190|N|N|N|N|N| +2434282|AAAAAAAAKOECFCAA|1952-09-26|632|2752|212|1952|5|9|26|3|1952|212|2752|Friday|1952Q3|N|Y|N|2434257|2434500|2433916|2434191|N|N|N|N|N| +2434283|AAAAAAAALOECFCAA|1952-09-27|632|2752|212|1952|6|9|27|3|1952|212|2752|Saturday|1952Q3|N|Y|N|2434257|2434500|2433917|2434192|N|N|N|N|N| +2434284|AAAAAAAAMOECFCAA|1952-09-28|632|2752|212|1952|0|9|28|3|1952|212|2752|Sunday|1952Q3|N|N|N|2434257|2434500|2433918|2434193|N|N|N|N|N| +2434285|AAAAAAAANOECFCAA|1952-09-29|632|2752|212|1952|1|9|29|3|1952|212|2752|Monday|1952Q3|N|N|N|2434257|2434500|2433919|2434194|N|N|N|N|N| +2434286|AAAAAAAAOOECFCAA|1952-09-30|632|2753|212|1952|2|9|30|3|1952|212|2753|Tuesday|1952Q3|N|N|N|2434257|2434500|2433920|2434195|N|N|N|N|N| +2434287|AAAAAAAAPOECFCAA|1952-10-01|633|2753|212|1952|3|10|1|4|1952|212|2753|Wednesday|1952Q4|N|N|N|2434287|2434560|2433921|2434195|N|N|N|N|N| +2434288|AAAAAAAAAPECFCAA|1952-10-02|633|2753|212|1952|4|10|2|4|1952|212|2753|Thursday|1952Q4|N|N|N|2434287|2434560|2433922|2434196|N|N|N|N|N| +2434289|AAAAAAAABPECFCAA|1952-10-03|633|2753|212|1952|5|10|3|4|1952|212|2753|Friday|1952Q4|N|Y|N|2434287|2434560|2433923|2434197|N|N|N|N|N| +2434290|AAAAAAAACPECFCAA|1952-10-04|633|2753|212|1952|6|10|4|4|1952|212|2753|Saturday|1952Q4|N|Y|N|2434287|2434560|2433924|2434198|N|N|N|N|N| +2434291|AAAAAAAADPECFCAA|1952-10-05|633|2753|212|1952|0|10|5|4|1952|212|2753|Sunday|1952Q4|N|N|N|2434287|2434560|2433925|2434199|N|N|N|N|N| +2434292|AAAAAAAAEPECFCAA|1952-10-06|633|2753|212|1952|1|10|6|4|1952|212|2753|Monday|1952Q4|N|N|N|2434287|2434560|2433926|2434200|N|N|N|N|N| +2434293|AAAAAAAAFPECFCAA|1952-10-07|633|2754|212|1952|2|10|7|4|1952|212|2754|Tuesday|1952Q4|N|N|N|2434287|2434560|2433927|2434201|N|N|N|N|N| +2434294|AAAAAAAAGPECFCAA|1952-10-08|633|2754|212|1952|3|10|8|4|1952|212|2754|Wednesday|1952Q4|N|N|N|2434287|2434560|2433928|2434202|N|N|N|N|N| +2434295|AAAAAAAAHPECFCAA|1952-10-09|633|2754|212|1952|4|10|9|4|1952|212|2754|Thursday|1952Q4|N|N|N|2434287|2434560|2433929|2434203|N|N|N|N|N| +2434296|AAAAAAAAIPECFCAA|1952-10-10|633|2754|212|1952|5|10|10|4|1952|212|2754|Friday|1952Q4|N|Y|N|2434287|2434560|2433930|2434204|N|N|N|N|N| +2434297|AAAAAAAAJPECFCAA|1952-10-11|633|2754|212|1952|6|10|11|4|1952|212|2754|Saturday|1952Q4|N|Y|N|2434287|2434560|2433931|2434205|N|N|N|N|N| +2434298|AAAAAAAAKPECFCAA|1952-10-12|633|2754|212|1952|0|10|12|4|1952|212|2754|Sunday|1952Q4|N|N|N|2434287|2434560|2433932|2434206|N|N|N|N|N| +2434299|AAAAAAAALPECFCAA|1952-10-13|633|2754|212|1952|1|10|13|4|1952|212|2754|Monday|1952Q4|N|N|N|2434287|2434560|2433933|2434207|N|N|N|N|N| +2434300|AAAAAAAAMPECFCAA|1952-10-14|633|2755|212|1952|2|10|14|4|1952|212|2755|Tuesday|1952Q4|N|N|N|2434287|2434560|2433934|2434208|N|N|N|N|N| +2434301|AAAAAAAANPECFCAA|1952-10-15|633|2755|212|1952|3|10|15|4|1952|212|2755|Wednesday|1952Q4|N|N|N|2434287|2434560|2433935|2434209|N|N|N|N|N| +2434302|AAAAAAAAOPECFCAA|1952-10-16|633|2755|212|1952|4|10|16|4|1952|212|2755|Thursday|1952Q4|N|N|N|2434287|2434560|2433936|2434210|N|N|N|N|N| +2434303|AAAAAAAAPPECFCAA|1952-10-17|633|2755|212|1952|5|10|17|4|1952|212|2755|Friday|1952Q4|N|Y|N|2434287|2434560|2433937|2434211|N|N|N|N|N| +2434304|AAAAAAAAAAFCFCAA|1952-10-18|633|2755|212|1952|6|10|18|4|1952|212|2755|Saturday|1952Q4|N|Y|N|2434287|2434560|2433938|2434212|N|N|N|N|N| +2434305|AAAAAAAABAFCFCAA|1952-10-19|633|2755|212|1952|0|10|19|4|1952|212|2755|Sunday|1952Q4|N|N|N|2434287|2434560|2433939|2434213|N|N|N|N|N| +2434306|AAAAAAAACAFCFCAA|1952-10-20|633|2755|212|1952|1|10|20|4|1952|212|2755|Monday|1952Q4|N|N|N|2434287|2434560|2433940|2434214|N|N|N|N|N| +2434307|AAAAAAAADAFCFCAA|1952-10-21|633|2756|212|1952|2|10|21|4|1952|212|2756|Tuesday|1952Q4|N|N|N|2434287|2434560|2433941|2434215|N|N|N|N|N| +2434308|AAAAAAAAEAFCFCAA|1952-10-22|633|2756|212|1952|3|10|22|4|1952|212|2756|Wednesday|1952Q4|N|N|N|2434287|2434560|2433942|2434216|N|N|N|N|N| +2434309|AAAAAAAAFAFCFCAA|1952-10-23|633|2756|212|1952|4|10|23|4|1952|212|2756|Thursday|1952Q4|N|N|N|2434287|2434560|2433943|2434217|N|N|N|N|N| +2434310|AAAAAAAAGAFCFCAA|1952-10-24|633|2756|212|1952|5|10|24|4|1952|212|2756|Friday|1952Q4|N|Y|N|2434287|2434560|2433944|2434218|N|N|N|N|N| +2434311|AAAAAAAAHAFCFCAA|1952-10-25|633|2756|212|1952|6|10|25|4|1952|212|2756|Saturday|1952Q4|N|Y|N|2434287|2434560|2433945|2434219|N|N|N|N|N| +2434312|AAAAAAAAIAFCFCAA|1952-10-26|633|2756|212|1952|0|10|26|4|1952|212|2756|Sunday|1952Q4|N|N|N|2434287|2434560|2433946|2434220|N|N|N|N|N| +2434313|AAAAAAAAJAFCFCAA|1952-10-27|633|2756|212|1952|1|10|27|4|1952|212|2756|Monday|1952Q4|N|N|N|2434287|2434560|2433947|2434221|N|N|N|N|N| +2434314|AAAAAAAAKAFCFCAA|1952-10-28|633|2757|212|1952|2|10|28|4|1952|212|2757|Tuesday|1952Q4|N|N|N|2434287|2434560|2433948|2434222|N|N|N|N|N| +2434315|AAAAAAAALAFCFCAA|1952-10-29|633|2757|212|1952|3|10|29|4|1952|212|2757|Wednesday|1952Q4|N|N|N|2434287|2434560|2433949|2434223|N|N|N|N|N| +2434316|AAAAAAAAMAFCFCAA|1952-10-30|633|2757|212|1952|4|10|30|4|1952|212|2757|Thursday|1952Q4|N|N|N|2434287|2434560|2433950|2434224|N|N|N|N|N| +2434317|AAAAAAAANAFCFCAA|1952-10-31|633|2757|212|1952|5|10|31|4|1952|212|2757|Friday|1952Q4|N|Y|N|2434287|2434560|2433951|2434225|N|N|N|N|N| +2434318|AAAAAAAAOAFCFCAA|1952-11-01|634|2757|212|1952|6|11|1|4|1952|212|2757|Saturday|1952Q4|N|Y|N|2434318|2434622|2433952|2434226|N|N|N|N|N| +2434319|AAAAAAAAPAFCFCAA|1952-11-02|634|2757|212|1952|0|11|2|4|1952|212|2757|Sunday|1952Q4|N|N|N|2434318|2434622|2433953|2434227|N|N|N|N|N| +2434320|AAAAAAAAABFCFCAA|1952-11-03|634|2757|212|1952|1|11|3|4|1952|212|2757|Monday|1952Q4|N|N|N|2434318|2434622|2433954|2434228|N|N|N|N|N| +2434321|AAAAAAAABBFCFCAA|1952-11-04|634|2758|212|1952|2|11|4|4|1952|212|2758|Tuesday|1952Q4|N|N|N|2434318|2434622|2433955|2434229|N|N|N|N|N| +2434322|AAAAAAAACBFCFCAA|1952-11-05|634|2758|212|1952|3|11|5|4|1952|212|2758|Wednesday|1952Q4|N|N|N|2434318|2434622|2433956|2434230|N|N|N|N|N| +2434323|AAAAAAAADBFCFCAA|1952-11-06|634|2758|212|1952|4|11|6|4|1952|212|2758|Thursday|1952Q4|N|N|N|2434318|2434622|2433957|2434231|N|N|N|N|N| +2434324|AAAAAAAAEBFCFCAA|1952-11-07|634|2758|212|1952|5|11|7|4|1952|212|2758|Friday|1952Q4|N|Y|N|2434318|2434622|2433958|2434232|N|N|N|N|N| +2434325|AAAAAAAAFBFCFCAA|1952-11-08|634|2758|212|1952|6|11|8|4|1952|212|2758|Saturday|1952Q4|N|Y|N|2434318|2434622|2433959|2434233|N|N|N|N|N| +2434326|AAAAAAAAGBFCFCAA|1952-11-09|634|2758|212|1952|0|11|9|4|1952|212|2758|Sunday|1952Q4|N|N|N|2434318|2434622|2433960|2434234|N|N|N|N|N| +2434327|AAAAAAAAHBFCFCAA|1952-11-10|634|2758|212|1952|1|11|10|4|1952|212|2758|Monday|1952Q4|N|N|N|2434318|2434622|2433961|2434235|N|N|N|N|N| +2434328|AAAAAAAAIBFCFCAA|1952-11-11|634|2759|212|1952|2|11|11|4|1952|212|2759|Tuesday|1952Q4|N|N|N|2434318|2434622|2433962|2434236|N|N|N|N|N| +2434329|AAAAAAAAJBFCFCAA|1952-11-12|634|2759|212|1952|3|11|12|4|1952|212|2759|Wednesday|1952Q4|N|N|N|2434318|2434622|2433963|2434237|N|N|N|N|N| +2434330|AAAAAAAAKBFCFCAA|1952-11-13|634|2759|212|1952|4|11|13|4|1952|212|2759|Thursday|1952Q4|N|N|N|2434318|2434622|2433964|2434238|N|N|N|N|N| +2434331|AAAAAAAALBFCFCAA|1952-11-14|634|2759|212|1952|5|11|14|4|1952|212|2759|Friday|1952Q4|N|Y|N|2434318|2434622|2433965|2434239|N|N|N|N|N| +2434332|AAAAAAAAMBFCFCAA|1952-11-15|634|2759|212|1952|6|11|15|4|1952|212|2759|Saturday|1952Q4|N|Y|N|2434318|2434622|2433966|2434240|N|N|N|N|N| +2434333|AAAAAAAANBFCFCAA|1952-11-16|634|2759|212|1952|0|11|16|4|1952|212|2759|Sunday|1952Q4|N|N|N|2434318|2434622|2433967|2434241|N|N|N|N|N| +2434334|AAAAAAAAOBFCFCAA|1952-11-17|634|2759|212|1952|1|11|17|4|1952|212|2759|Monday|1952Q4|N|N|N|2434318|2434622|2433968|2434242|N|N|N|N|N| +2434335|AAAAAAAAPBFCFCAA|1952-11-18|634|2760|212|1952|2|11|18|4|1952|212|2760|Tuesday|1952Q4|N|N|N|2434318|2434622|2433969|2434243|N|N|N|N|N| +2434336|AAAAAAAAACFCFCAA|1952-11-19|634|2760|212|1952|3|11|19|4|1952|212|2760|Wednesday|1952Q4|N|N|N|2434318|2434622|2433970|2434244|N|N|N|N|N| +2434337|AAAAAAAABCFCFCAA|1952-11-20|634|2760|212|1952|4|11|20|4|1952|212|2760|Thursday|1952Q4|N|N|N|2434318|2434622|2433971|2434245|N|N|N|N|N| +2434338|AAAAAAAACCFCFCAA|1952-11-21|634|2760|212|1952|5|11|21|4|1952|212|2760|Friday|1952Q4|N|Y|N|2434318|2434622|2433972|2434246|N|N|N|N|N| +2434339|AAAAAAAADCFCFCAA|1952-11-22|634|2760|212|1952|6|11|22|4|1952|212|2760|Saturday|1952Q4|N|Y|N|2434318|2434622|2433973|2434247|N|N|N|N|N| +2434340|AAAAAAAAECFCFCAA|1952-11-23|634|2760|212|1952|0|11|23|4|1952|212|2760|Sunday|1952Q4|N|N|N|2434318|2434622|2433974|2434248|N|N|N|N|N| +2434341|AAAAAAAAFCFCFCAA|1952-11-24|634|2760|212|1952|1|11|24|4|1952|212|2760|Monday|1952Q4|N|N|N|2434318|2434622|2433975|2434249|N|N|N|N|N| +2434342|AAAAAAAAGCFCFCAA|1952-11-25|634|2761|212|1952|2|11|25|4|1952|212|2761|Tuesday|1952Q4|N|N|N|2434318|2434622|2433976|2434250|N|N|N|N|N| +2434343|AAAAAAAAHCFCFCAA|1952-11-26|634|2761|212|1952|3|11|26|4|1952|212|2761|Wednesday|1952Q4|N|N|N|2434318|2434622|2433977|2434251|N|N|N|N|N| +2434344|AAAAAAAAICFCFCAA|1952-11-27|634|2761|212|1952|4|11|27|4|1952|212|2761|Thursday|1952Q4|N|N|N|2434318|2434622|2433978|2434252|N|N|N|N|N| +2434345|AAAAAAAAJCFCFCAA|1952-11-28|634|2761|212|1952|5|11|28|4|1952|212|2761|Friday|1952Q4|N|Y|N|2434318|2434622|2433979|2434253|N|N|N|N|N| +2434346|AAAAAAAAKCFCFCAA|1952-11-29|634|2761|212|1952|6|11|29|4|1952|212|2761|Saturday|1952Q4|N|Y|N|2434318|2434622|2433980|2434254|N|N|N|N|N| +2434347|AAAAAAAALCFCFCAA|1952-11-30|634|2761|212|1952|0|11|30|4|1952|212|2761|Sunday|1952Q4|N|N|N|2434318|2434622|2433981|2434255|N|N|N|N|N| +2434348|AAAAAAAAMCFCFCAA|1952-12-01|635|2761|213|1952|1|12|1|4|1952|213|2761|Monday|1952Q4|N|N|N|2434348|2434682|2433982|2434256|N|N|N|N|N| +2434349|AAAAAAAANCFCFCAA|1952-12-02|635|2762|213|1952|2|12|2|4|1952|213|2762|Tuesday|1952Q4|N|N|N|2434348|2434682|2433983|2434257|N|N|N|N|N| +2434350|AAAAAAAAOCFCFCAA|1952-12-03|635|2762|213|1952|3|12|3|4|1952|213|2762|Wednesday|1952Q4|N|N|N|2434348|2434682|2433984|2434258|N|N|N|N|N| +2434351|AAAAAAAAPCFCFCAA|1952-12-04|635|2762|213|1952|4|12|4|4|1952|213|2762|Thursday|1952Q4|N|N|N|2434348|2434682|2433985|2434259|N|N|N|N|N| +2434352|AAAAAAAAADFCFCAA|1952-12-05|635|2762|213|1952|5|12|5|4|1952|213|2762|Friday|1952Q4|N|Y|N|2434348|2434682|2433986|2434260|N|N|N|N|N| +2434353|AAAAAAAABDFCFCAA|1952-12-06|635|2762|213|1952|6|12|6|4|1952|213|2762|Saturday|1952Q4|N|Y|N|2434348|2434682|2433987|2434261|N|N|N|N|N| +2434354|AAAAAAAACDFCFCAA|1952-12-07|635|2762|213|1952|0|12|7|4|1952|213|2762|Sunday|1952Q4|N|N|N|2434348|2434682|2433988|2434262|N|N|N|N|N| +2434355|AAAAAAAADDFCFCAA|1952-12-08|635|2762|213|1952|1|12|8|4|1952|213|2762|Monday|1952Q4|N|N|N|2434348|2434682|2433989|2434263|N|N|N|N|N| +2434356|AAAAAAAAEDFCFCAA|1952-12-09|635|2763|213|1952|2|12|9|4|1952|213|2763|Tuesday|1952Q4|N|N|N|2434348|2434682|2433990|2434264|N|N|N|N|N| +2434357|AAAAAAAAFDFCFCAA|1952-12-10|635|2763|213|1952|3|12|10|4|1952|213|2763|Wednesday|1952Q4|N|N|N|2434348|2434682|2433991|2434265|N|N|N|N|N| +2434358|AAAAAAAAGDFCFCAA|1952-12-11|635|2763|213|1952|4|12|11|4|1952|213|2763|Thursday|1952Q4|N|N|N|2434348|2434682|2433992|2434266|N|N|N|N|N| +2434359|AAAAAAAAHDFCFCAA|1952-12-12|635|2763|213|1952|5|12|12|4|1952|213|2763|Friday|1952Q4|N|Y|N|2434348|2434682|2433993|2434267|N|N|N|N|N| +2434360|AAAAAAAAIDFCFCAA|1952-12-13|635|2763|213|1952|6|12|13|4|1952|213|2763|Saturday|1952Q4|N|Y|N|2434348|2434682|2433994|2434268|N|N|N|N|N| +2434361|AAAAAAAAJDFCFCAA|1952-12-14|635|2763|213|1952|0|12|14|4|1952|213|2763|Sunday|1952Q4|N|N|N|2434348|2434682|2433995|2434269|N|N|N|N|N| +2434362|AAAAAAAAKDFCFCAA|1952-12-15|635|2763|213|1952|1|12|15|4|1952|213|2763|Monday|1952Q4|N|N|N|2434348|2434682|2433996|2434270|N|N|N|N|N| +2434363|AAAAAAAALDFCFCAA|1952-12-16|635|2764|213|1952|2|12|16|4|1952|213|2764|Tuesday|1952Q4|N|N|N|2434348|2434682|2433997|2434271|N|N|N|N|N| +2434364|AAAAAAAAMDFCFCAA|1952-12-17|635|2764|213|1952|3|12|17|4|1952|213|2764|Wednesday|1952Q4|N|N|N|2434348|2434682|2433998|2434272|N|N|N|N|N| +2434365|AAAAAAAANDFCFCAA|1952-12-18|635|2764|213|1952|4|12|18|4|1952|213|2764|Thursday|1952Q4|N|N|N|2434348|2434682|2433999|2434273|N|N|N|N|N| +2434366|AAAAAAAAODFCFCAA|1952-12-19|635|2764|213|1952|5|12|19|4|1952|213|2764|Friday|1952Q4|N|Y|N|2434348|2434682|2434000|2434274|N|N|N|N|N| +2434367|AAAAAAAAPDFCFCAA|1952-12-20|635|2764|213|1952|6|12|20|4|1952|213|2764|Saturday|1952Q4|N|Y|N|2434348|2434682|2434001|2434275|N|N|N|N|N| +2434368|AAAAAAAAAEFCFCAA|1952-12-21|635|2764|213|1952|0|12|21|4|1952|213|2764|Sunday|1952Q4|N|N|N|2434348|2434682|2434002|2434276|N|N|N|N|N| +2434369|AAAAAAAABEFCFCAA|1952-12-22|635|2764|213|1952|1|12|22|4|1952|213|2764|Monday|1952Q4|N|N|N|2434348|2434682|2434003|2434277|N|N|N|N|N| +2434370|AAAAAAAACEFCFCAA|1952-12-23|635|2765|213|1952|2|12|23|4|1952|213|2765|Tuesday|1952Q4|N|N|N|2434348|2434682|2434004|2434278|N|N|N|N|N| +2434371|AAAAAAAADEFCFCAA|1952-12-24|635|2765|213|1952|3|12|24|4|1952|213|2765|Wednesday|1952Q4|N|N|N|2434348|2434682|2434005|2434279|N|N|N|N|N| +2434372|AAAAAAAAEEFCFCAA|1952-12-25|635|2765|213|1952|4|12|25|4|1952|213|2765|Thursday|1952Q4|Y|N|N|2434348|2434682|2434006|2434280|N|N|N|N|N| +2434373|AAAAAAAAFEFCFCAA|1952-12-26|635|2765|213|1952|5|12|26|4|1952|213|2765|Friday|1952Q4|N|Y|Y|2434348|2434682|2434007|2434281|N|N|N|N|N| +2434374|AAAAAAAAGEFCFCAA|1952-12-27|635|2765|213|1952|6|12|27|4|1952|213|2765|Saturday|1952Q4|N|Y|N|2434348|2434682|2434008|2434282|N|N|N|N|N| +2434375|AAAAAAAAHEFCFCAA|1952-12-28|635|2765|213|1952|0|12|28|4|1952|213|2765|Sunday|1952Q4|N|N|N|2434348|2434682|2434009|2434283|N|N|N|N|N| +2434376|AAAAAAAAIEFCFCAA|1952-12-29|635|2765|213|1952|1|12|29|4|1952|213|2765|Monday|1952Q4|N|N|N|2434348|2434682|2434010|2434284|N|N|N|N|N| +2434377|AAAAAAAAJEFCFCAA|1952-12-30|635|2766|213|1952|2|12|30|4|1952|213|2766|Tuesday|1952Q4|N|N|N|2434348|2434682|2434011|2434285|N|N|N|N|N| +2434378|AAAAAAAAKEFCFCAA|1952-12-31|635|2766|213|1952|3|12|31|4|1952|213|2766|Wednesday|1952Q4|Y|N|N|2434348|2434682|2434012|2434286|N|N|N|N|N| +2434379|AAAAAAAALEFCFCAA|1953-01-01|636|2766|213|1953|4|1|1|1|1953|213|2766|Thursday|1953Q1|Y|N|Y|2434379|2434378|2434013|2434287|N|N|N|N|N| +2434380|AAAAAAAAMEFCFCAA|1953-01-02|636|2766|213|1953|5|1|2|1|1953|213|2766|Friday|1953Q1|N|Y|Y|2434379|2434378|2434014|2434288|N|N|N|N|N| +2434381|AAAAAAAANEFCFCAA|1953-01-03|636|2766|213|1953|6|1|3|1|1953|213|2766|Saturday|1953Q1|N|Y|N|2434379|2434378|2434015|2434289|N|N|N|N|N| +2434382|AAAAAAAAOEFCFCAA|1953-01-04|636|2766|213|1953|0|1|4|1|1953|213|2766|Sunday|1953Q1|N|N|N|2434379|2434378|2434016|2434290|N|N|N|N|N| +2434383|AAAAAAAAPEFCFCAA|1953-01-05|636|2766|213|1953|1|1|5|1|1953|213|2766|Monday|1953Q1|N|N|N|2434379|2434378|2434017|2434291|N|N|N|N|N| +2434384|AAAAAAAAAFFCFCAA|1953-01-06|636|2767|213|1953|2|1|6|1|1953|213|2767|Tuesday|1953Q1|N|N|N|2434379|2434378|2434018|2434292|N|N|N|N|N| +2434385|AAAAAAAABFFCFCAA|1953-01-07|636|2767|213|1953|3|1|7|1|1953|213|2767|Wednesday|1953Q1|N|N|N|2434379|2434378|2434019|2434293|N|N|N|N|N| +2434386|AAAAAAAACFFCFCAA|1953-01-08|636|2767|213|1953|4|1|8|1|1953|213|2767|Thursday|1953Q1|N|N|N|2434379|2434378|2434020|2434294|N|N|N|N|N| +2434387|AAAAAAAADFFCFCAA|1953-01-09|636|2767|213|1953|5|1|9|1|1953|213|2767|Friday|1953Q1|N|Y|N|2434379|2434378|2434021|2434295|N|N|N|N|N| +2434388|AAAAAAAAEFFCFCAA|1953-01-10|636|2767|213|1953|6|1|10|1|1953|213|2767|Saturday|1953Q1|N|Y|N|2434379|2434378|2434022|2434296|N|N|N|N|N| +2434389|AAAAAAAAFFFCFCAA|1953-01-11|636|2767|213|1953|0|1|11|1|1953|213|2767|Sunday|1953Q1|N|N|N|2434379|2434378|2434023|2434297|N|N|N|N|N| +2434390|AAAAAAAAGFFCFCAA|1953-01-12|636|2767|213|1953|1|1|12|1|1953|213|2767|Monday|1953Q1|N|N|N|2434379|2434378|2434024|2434298|N|N|N|N|N| +2434391|AAAAAAAAHFFCFCAA|1953-01-13|636|2768|213|1953|2|1|13|1|1953|213|2768|Tuesday|1953Q1|N|N|N|2434379|2434378|2434025|2434299|N|N|N|N|N| +2434392|AAAAAAAAIFFCFCAA|1953-01-14|636|2768|213|1953|3|1|14|1|1953|213|2768|Wednesday|1953Q1|N|N|N|2434379|2434378|2434026|2434300|N|N|N|N|N| +2434393|AAAAAAAAJFFCFCAA|1953-01-15|636|2768|213|1953|4|1|15|1|1953|213|2768|Thursday|1953Q1|N|N|N|2434379|2434378|2434027|2434301|N|N|N|N|N| +2434394|AAAAAAAAKFFCFCAA|1953-01-16|636|2768|213|1953|5|1|16|1|1953|213|2768|Friday|1953Q1|N|Y|N|2434379|2434378|2434028|2434302|N|N|N|N|N| +2434395|AAAAAAAALFFCFCAA|1953-01-17|636|2768|213|1953|6|1|17|1|1953|213|2768|Saturday|1953Q1|N|Y|N|2434379|2434378|2434029|2434303|N|N|N|N|N| +2434396|AAAAAAAAMFFCFCAA|1953-01-18|636|2768|213|1953|0|1|18|1|1953|213|2768|Sunday|1953Q1|N|N|N|2434379|2434378|2434030|2434304|N|N|N|N|N| +2434397|AAAAAAAANFFCFCAA|1953-01-19|636|2768|213|1953|1|1|19|1|1953|213|2768|Monday|1953Q1|N|N|N|2434379|2434378|2434031|2434305|N|N|N|N|N| +2434398|AAAAAAAAOFFCFCAA|1953-01-20|636|2769|213|1953|2|1|20|1|1953|213|2769|Tuesday|1953Q1|N|N|N|2434379|2434378|2434032|2434306|N|N|N|N|N| +2434399|AAAAAAAAPFFCFCAA|1953-01-21|636|2769|213|1953|3|1|21|1|1953|213|2769|Wednesday|1953Q1|N|N|N|2434379|2434378|2434033|2434307|N|N|N|N|N| +2434400|AAAAAAAAAGFCFCAA|1953-01-22|636|2769|213|1953|4|1|22|1|1953|213|2769|Thursday|1953Q1|N|N|N|2434379|2434378|2434034|2434308|N|N|N|N|N| +2434401|AAAAAAAABGFCFCAA|1953-01-23|636|2769|213|1953|5|1|23|1|1953|213|2769|Friday|1953Q1|N|Y|N|2434379|2434378|2434035|2434309|N|N|N|N|N| +2434402|AAAAAAAACGFCFCAA|1953-01-24|636|2769|213|1953|6|1|24|1|1953|213|2769|Saturday|1953Q1|N|Y|N|2434379|2434378|2434036|2434310|N|N|N|N|N| +2434403|AAAAAAAADGFCFCAA|1953-01-25|636|2769|213|1953|0|1|25|1|1953|213|2769|Sunday|1953Q1|N|N|N|2434379|2434378|2434037|2434311|N|N|N|N|N| +2434404|AAAAAAAAEGFCFCAA|1953-01-26|636|2769|213|1953|1|1|26|1|1953|213|2769|Monday|1953Q1|N|N|N|2434379|2434378|2434038|2434312|N|N|N|N|N| +2434405|AAAAAAAAFGFCFCAA|1953-01-27|636|2770|213|1953|2|1|27|1|1953|213|2770|Tuesday|1953Q1|N|N|N|2434379|2434378|2434039|2434313|N|N|N|N|N| +2434406|AAAAAAAAGGFCFCAA|1953-01-28|636|2770|213|1953|3|1|28|1|1953|213|2770|Wednesday|1953Q1|N|N|N|2434379|2434378|2434040|2434314|N|N|N|N|N| +2434407|AAAAAAAAHGFCFCAA|1953-01-29|636|2770|213|1953|4|1|29|1|1953|213|2770|Thursday|1953Q1|N|N|N|2434379|2434378|2434041|2434315|N|N|N|N|N| +2434408|AAAAAAAAIGFCFCAA|1953-01-30|636|2770|213|1953|5|1|30|1|1953|213|2770|Friday|1953Q1|N|Y|N|2434379|2434378|2434042|2434316|N|N|N|N|N| +2434409|AAAAAAAAJGFCFCAA|1953-01-31|636|2770|213|1953|6|1|31|1|1953|213|2770|Saturday|1953Q1|N|Y|N|2434379|2434378|2434043|2434317|N|N|N|N|N| +2434410|AAAAAAAAKGFCFCAA|1953-02-01|637|2770|213|1953|0|2|1|1|1953|213|2770|Sunday|1953Q1|N|N|N|2434410|2434440|2434044|2434318|N|N|N|N|N| +2434411|AAAAAAAALGFCFCAA|1953-02-02|637|2770|213|1953|1|2|2|1|1953|213|2770|Monday|1953Q1|N|N|N|2434410|2434440|2434045|2434319|N|N|N|N|N| +2434412|AAAAAAAAMGFCFCAA|1953-02-03|637|2771|213|1953|2|2|3|1|1953|213|2771|Tuesday|1953Q1|N|N|N|2434410|2434440|2434046|2434320|N|N|N|N|N| +2434413|AAAAAAAANGFCFCAA|1953-02-04|637|2771|213|1953|3|2|4|1|1953|213|2771|Wednesday|1953Q1|N|N|N|2434410|2434440|2434047|2434321|N|N|N|N|N| +2434414|AAAAAAAAOGFCFCAA|1953-02-05|637|2771|213|1953|4|2|5|1|1953|213|2771|Thursday|1953Q1|N|N|N|2434410|2434440|2434048|2434322|N|N|N|N|N| +2434415|AAAAAAAAPGFCFCAA|1953-02-06|637|2771|213|1953|5|2|6|1|1953|213|2771|Friday|1953Q1|N|Y|N|2434410|2434440|2434049|2434323|N|N|N|N|N| +2434416|AAAAAAAAAHFCFCAA|1953-02-07|637|2771|213|1953|6|2|7|1|1953|213|2771|Saturday|1953Q1|N|Y|N|2434410|2434440|2434050|2434324|N|N|N|N|N| +2434417|AAAAAAAABHFCFCAA|1953-02-08|637|2771|213|1953|0|2|8|1|1953|213|2771|Sunday|1953Q1|N|N|N|2434410|2434440|2434051|2434325|N|N|N|N|N| +2434418|AAAAAAAACHFCFCAA|1953-02-09|637|2771|213|1953|1|2|9|1|1953|213|2771|Monday|1953Q1|N|N|N|2434410|2434440|2434052|2434326|N|N|N|N|N| +2434419|AAAAAAAADHFCFCAA|1953-02-10|637|2772|213|1953|2|2|10|1|1953|213|2772|Tuesday|1953Q1|N|N|N|2434410|2434440|2434053|2434327|N|N|N|N|N| +2434420|AAAAAAAAEHFCFCAA|1953-02-11|637|2772|213|1953|3|2|11|1|1953|213|2772|Wednesday|1953Q1|N|N|N|2434410|2434440|2434054|2434328|N|N|N|N|N| +2434421|AAAAAAAAFHFCFCAA|1953-02-12|637|2772|213|1953|4|2|12|1|1953|213|2772|Thursday|1953Q1|N|N|N|2434410|2434440|2434055|2434329|N|N|N|N|N| +2434422|AAAAAAAAGHFCFCAA|1953-02-13|637|2772|213|1953|5|2|13|1|1953|213|2772|Friday|1953Q1|N|Y|N|2434410|2434440|2434056|2434330|N|N|N|N|N| +2434423|AAAAAAAAHHFCFCAA|1953-02-14|637|2772|213|1953|6|2|14|1|1953|213|2772|Saturday|1953Q1|N|Y|N|2434410|2434440|2434057|2434331|N|N|N|N|N| +2434424|AAAAAAAAIHFCFCAA|1953-02-15|637|2772|213|1953|0|2|15|1|1953|213|2772|Sunday|1953Q1|N|N|N|2434410|2434440|2434058|2434332|N|N|N|N|N| +2434425|AAAAAAAAJHFCFCAA|1953-02-16|637|2772|213|1953|1|2|16|1|1953|213|2772|Monday|1953Q1|N|N|N|2434410|2434440|2434059|2434333|N|N|N|N|N| +2434426|AAAAAAAAKHFCFCAA|1953-02-17|637|2773|213|1953|2|2|17|1|1953|213|2773|Tuesday|1953Q1|N|N|N|2434410|2434440|2434060|2434334|N|N|N|N|N| +2434427|AAAAAAAALHFCFCAA|1953-02-18|637|2773|213|1953|3|2|18|1|1953|213|2773|Wednesday|1953Q1|N|N|N|2434410|2434440|2434061|2434335|N|N|N|N|N| +2434428|AAAAAAAAMHFCFCAA|1953-02-19|637|2773|213|1953|4|2|19|1|1953|213|2773|Thursday|1953Q1|N|N|N|2434410|2434440|2434062|2434336|N|N|N|N|N| +2434429|AAAAAAAANHFCFCAA|1953-02-20|637|2773|213|1953|5|2|20|1|1953|213|2773|Friday|1953Q1|N|Y|N|2434410|2434440|2434063|2434337|N|N|N|N|N| +2434430|AAAAAAAAOHFCFCAA|1953-02-21|637|2773|213|1953|6|2|21|1|1953|213|2773|Saturday|1953Q1|N|Y|N|2434410|2434440|2434064|2434338|N|N|N|N|N| +2434431|AAAAAAAAPHFCFCAA|1953-02-22|637|2773|213|1953|0|2|22|1|1953|213|2773|Sunday|1953Q1|N|N|N|2434410|2434440|2434065|2434339|N|N|N|N|N| +2434432|AAAAAAAAAIFCFCAA|1953-02-23|637|2773|213|1953|1|2|23|1|1953|213|2773|Monday|1953Q1|N|N|N|2434410|2434440|2434066|2434340|N|N|N|N|N| +2434433|AAAAAAAABIFCFCAA|1953-02-24|637|2774|213|1953|2|2|24|1|1953|213|2774|Tuesday|1953Q1|N|N|N|2434410|2434440|2434067|2434341|N|N|N|N|N| +2434434|AAAAAAAACIFCFCAA|1953-02-25|637|2774|213|1953|3|2|25|1|1953|213|2774|Wednesday|1953Q1|N|N|N|2434410|2434440|2434068|2434342|N|N|N|N|N| +2434435|AAAAAAAADIFCFCAA|1953-02-26|637|2774|213|1953|4|2|26|1|1953|213|2774|Thursday|1953Q1|N|N|N|2434410|2434440|2434069|2434343|N|N|N|N|N| +2434436|AAAAAAAAEIFCFCAA|1953-02-27|637|2774|213|1953|5|2|27|1|1953|213|2774|Friday|1953Q1|N|Y|N|2434410|2434440|2434070|2434344|N|N|N|N|N| +2434437|AAAAAAAAFIFCFCAA|1953-02-28|637|2774|213|1953|6|2|28|1|1953|213|2774|Saturday|1953Q1|N|Y|N|2434410|2434440|2434071|2434345|N|N|N|N|N| +2434438|AAAAAAAAGIFCFCAA|1953-03-01|638|2774|214|1953|0|3|1|1|1953|214|2774|Sunday|1953Q1|N|N|N|2434438|2434496|2434073|2434346|N|N|N|N|N| +2434439|AAAAAAAAHIFCFCAA|1953-03-02|638|2774|214|1953|1|3|2|1|1953|214|2774|Monday|1953Q1|N|N|N|2434438|2434496|2434074|2434347|N|N|N|N|N| +2434440|AAAAAAAAIIFCFCAA|1953-03-03|638|2775|214|1953|2|3|3|1|1953|214|2775|Tuesday|1953Q1|N|N|N|2434438|2434496|2434075|2434348|N|N|N|N|N| +2434441|AAAAAAAAJIFCFCAA|1953-03-04|638|2775|214|1953|3|3|4|1|1953|214|2775|Wednesday|1953Q1|N|N|N|2434438|2434496|2434076|2434349|N|N|N|N|N| +2434442|AAAAAAAAKIFCFCAA|1953-03-05|638|2775|214|1953|4|3|5|1|1953|214|2775|Thursday|1953Q1|N|N|N|2434438|2434496|2434077|2434350|N|N|N|N|N| +2434443|AAAAAAAALIFCFCAA|1953-03-06|638|2775|214|1953|5|3|6|1|1953|214|2775|Friday|1953Q1|N|Y|N|2434438|2434496|2434078|2434351|N|N|N|N|N| +2434444|AAAAAAAAMIFCFCAA|1953-03-07|638|2775|214|1953|6|3|7|1|1953|214|2775|Saturday|1953Q1|N|Y|N|2434438|2434496|2434079|2434352|N|N|N|N|N| +2434445|AAAAAAAANIFCFCAA|1953-03-08|638|2775|214|1953|0|3|8|1|1953|214|2775|Sunday|1953Q1|N|N|N|2434438|2434496|2434080|2434353|N|N|N|N|N| +2434446|AAAAAAAAOIFCFCAA|1953-03-09|638|2775|214|1953|1|3|9|1|1953|214|2775|Monday|1953Q1|N|N|N|2434438|2434496|2434081|2434354|N|N|N|N|N| +2434447|AAAAAAAAPIFCFCAA|1953-03-10|638|2776|214|1953|2|3|10|1|1953|214|2776|Tuesday|1953Q1|N|N|N|2434438|2434496|2434082|2434355|N|N|N|N|N| +2434448|AAAAAAAAAJFCFCAA|1953-03-11|638|2776|214|1953|3|3|11|1|1953|214|2776|Wednesday|1953Q1|N|N|N|2434438|2434496|2434083|2434356|N|N|N|N|N| +2434449|AAAAAAAABJFCFCAA|1953-03-12|638|2776|214|1953|4|3|12|1|1953|214|2776|Thursday|1953Q1|N|N|N|2434438|2434496|2434084|2434357|N|N|N|N|N| +2434450|AAAAAAAACJFCFCAA|1953-03-13|638|2776|214|1953|5|3|13|1|1953|214|2776|Friday|1953Q1|N|Y|N|2434438|2434496|2434085|2434358|N|N|N|N|N| +2434451|AAAAAAAADJFCFCAA|1953-03-14|638|2776|214|1953|6|3|14|1|1953|214|2776|Saturday|1953Q1|N|Y|N|2434438|2434496|2434086|2434359|N|N|N|N|N| +2434452|AAAAAAAAEJFCFCAA|1953-03-15|638|2776|214|1953|0|3|15|1|1953|214|2776|Sunday|1953Q1|N|N|N|2434438|2434496|2434087|2434360|N|N|N|N|N| +2434453|AAAAAAAAFJFCFCAA|1953-03-16|638|2776|214|1953|1|3|16|1|1953|214|2776|Monday|1953Q1|N|N|N|2434438|2434496|2434088|2434361|N|N|N|N|N| +2434454|AAAAAAAAGJFCFCAA|1953-03-17|638|2777|214|1953|2|3|17|1|1953|214|2777|Tuesday|1953Q1|N|N|N|2434438|2434496|2434089|2434362|N|N|N|N|N| +2434455|AAAAAAAAHJFCFCAA|1953-03-18|638|2777|214|1953|3|3|18|1|1953|214|2777|Wednesday|1953Q1|N|N|N|2434438|2434496|2434090|2434363|N|N|N|N|N| +2434456|AAAAAAAAIJFCFCAA|1953-03-19|638|2777|214|1953|4|3|19|1|1953|214|2777|Thursday|1953Q1|N|N|N|2434438|2434496|2434091|2434364|N|N|N|N|N| +2434457|AAAAAAAAJJFCFCAA|1953-03-20|638|2777|214|1953|5|3|20|1|1953|214|2777|Friday|1953Q1|N|Y|N|2434438|2434496|2434092|2434365|N|N|N|N|N| +2434458|AAAAAAAAKJFCFCAA|1953-03-21|638|2777|214|1953|6|3|21|1|1953|214|2777|Saturday|1953Q1|N|Y|N|2434438|2434496|2434093|2434366|N|N|N|N|N| +2434459|AAAAAAAALJFCFCAA|1953-03-22|638|2777|214|1953|0|3|22|1|1953|214|2777|Sunday|1953Q1|N|N|N|2434438|2434496|2434094|2434367|N|N|N|N|N| +2434460|AAAAAAAAMJFCFCAA|1953-03-23|638|2777|214|1953|1|3|23|1|1953|214|2777|Monday|1953Q1|N|N|N|2434438|2434496|2434095|2434368|N|N|N|N|N| +2434461|AAAAAAAANJFCFCAA|1953-03-24|638|2778|214|1953|2|3|24|1|1953|214|2778|Tuesday|1953Q1|N|N|N|2434438|2434496|2434096|2434369|N|N|N|N|N| +2434462|AAAAAAAAOJFCFCAA|1953-03-25|638|2778|214|1953|3|3|25|1|1953|214|2778|Wednesday|1953Q1|N|N|N|2434438|2434496|2434097|2434370|N|N|N|N|N| +2434463|AAAAAAAAPJFCFCAA|1953-03-26|638|2778|214|1953|4|3|26|1|1953|214|2778|Thursday|1953Q1|N|N|N|2434438|2434496|2434098|2434371|N|N|N|N|N| +2434464|AAAAAAAAAKFCFCAA|1953-03-27|638|2778|214|1953|5|3|27|1|1953|214|2778|Friday|1953Q1|N|Y|N|2434438|2434496|2434099|2434372|N|N|N|N|N| +2434465|AAAAAAAABKFCFCAA|1953-03-28|638|2778|214|1953|6|3|28|1|1953|214|2778|Saturday|1953Q1|N|Y|N|2434438|2434496|2434100|2434373|N|N|N|N|N| +2434466|AAAAAAAACKFCFCAA|1953-03-29|638|2778|214|1953|0|3|29|1|1953|214|2778|Sunday|1953Q1|N|N|N|2434438|2434496|2434101|2434374|N|N|N|N|N| +2434467|AAAAAAAADKFCFCAA|1953-03-30|638|2778|214|1953|1|3|30|1|1953|214|2778|Monday|1953Q1|N|N|N|2434438|2434496|2434102|2434375|N|N|N|N|N| +2434468|AAAAAAAAEKFCFCAA|1953-03-31|638|2779|214|1953|2|3|31|1|1953|214|2779|Tuesday|1953Q1|N|N|N|2434438|2434496|2434103|2434376|N|N|N|N|N| +2434469|AAAAAAAAFKFCFCAA|1953-04-01|639|2779|214|1953|3|4|1|1|1953|214|2779|Wednesday|1953Q1|N|N|N|2434469|2434558|2434104|2434379|N|N|N|N|N| +2434470|AAAAAAAAGKFCFCAA|1953-04-02|639|2779|214|1953|4|4|2|2|1953|214|2779|Thursday|1953Q2|N|N|N|2434469|2434558|2434105|2434380|N|N|N|N|N| +2434471|AAAAAAAAHKFCFCAA|1953-04-03|639|2779|214|1953|5|4|3|2|1953|214|2779|Friday|1953Q2|N|Y|N|2434469|2434558|2434106|2434381|N|N|N|N|N| +2434472|AAAAAAAAIKFCFCAA|1953-04-04|639|2779|214|1953|6|4|4|2|1953|214|2779|Saturday|1953Q2|N|Y|N|2434469|2434558|2434107|2434382|N|N|N|N|N| +2434473|AAAAAAAAJKFCFCAA|1953-04-05|639|2779|214|1953|0|4|5|2|1953|214|2779|Sunday|1953Q2|N|N|N|2434469|2434558|2434108|2434383|N|N|N|N|N| +2434474|AAAAAAAAKKFCFCAA|1953-04-06|639|2779|214|1953|1|4|6|2|1953|214|2779|Monday|1953Q2|N|N|N|2434469|2434558|2434109|2434384|N|N|N|N|N| +2434475|AAAAAAAALKFCFCAA|1953-04-07|639|2780|214|1953|2|4|7|2|1953|214|2780|Tuesday|1953Q2|N|N|N|2434469|2434558|2434110|2434385|N|N|N|N|N| +2434476|AAAAAAAAMKFCFCAA|1953-04-08|639|2780|214|1953|3|4|8|2|1953|214|2780|Wednesday|1953Q2|N|N|N|2434469|2434558|2434111|2434386|N|N|N|N|N| +2434477|AAAAAAAANKFCFCAA|1953-04-09|639|2780|214|1953|4|4|9|2|1953|214|2780|Thursday|1953Q2|N|N|N|2434469|2434558|2434112|2434387|N|N|N|N|N| +2434478|AAAAAAAAOKFCFCAA|1953-04-10|639|2780|214|1953|5|4|10|2|1953|214|2780|Friday|1953Q2|N|Y|N|2434469|2434558|2434113|2434388|N|N|N|N|N| +2434479|AAAAAAAAPKFCFCAA|1953-04-11|639|2780|214|1953|6|4|11|2|1953|214|2780|Saturday|1953Q2|N|Y|N|2434469|2434558|2434114|2434389|N|N|N|N|N| +2434480|AAAAAAAAALFCFCAA|1953-04-12|639|2780|214|1953|0|4|12|2|1953|214|2780|Sunday|1953Q2|N|N|N|2434469|2434558|2434115|2434390|N|N|N|N|N| +2434481|AAAAAAAABLFCFCAA|1953-04-13|639|2780|214|1953|1|4|13|2|1953|214|2780|Monday|1953Q2|N|N|N|2434469|2434558|2434116|2434391|N|N|N|N|N| +2434482|AAAAAAAACLFCFCAA|1953-04-14|639|2781|214|1953|2|4|14|2|1953|214|2781|Tuesday|1953Q2|N|N|N|2434469|2434558|2434117|2434392|N|N|N|N|N| +2434483|AAAAAAAADLFCFCAA|1953-04-15|639|2781|214|1953|3|4|15|2|1953|214|2781|Wednesday|1953Q2|N|N|N|2434469|2434558|2434118|2434393|N|N|N|N|N| +2434484|AAAAAAAAELFCFCAA|1953-04-16|639|2781|214|1953|4|4|16|2|1953|214|2781|Thursday|1953Q2|N|N|N|2434469|2434558|2434119|2434394|N|N|N|N|N| +2434485|AAAAAAAAFLFCFCAA|1953-04-17|639|2781|214|1953|5|4|17|2|1953|214|2781|Friday|1953Q2|N|Y|N|2434469|2434558|2434120|2434395|N|N|N|N|N| +2434486|AAAAAAAAGLFCFCAA|1953-04-18|639|2781|214|1953|6|4|18|2|1953|214|2781|Saturday|1953Q2|N|Y|N|2434469|2434558|2434121|2434396|N|N|N|N|N| +2434487|AAAAAAAAHLFCFCAA|1953-04-19|639|2781|214|1953|0|4|19|2|1953|214|2781|Sunday|1953Q2|N|N|N|2434469|2434558|2434122|2434397|N|N|N|N|N| +2434488|AAAAAAAAILFCFCAA|1953-04-20|639|2781|214|1953|1|4|20|2|1953|214|2781|Monday|1953Q2|N|N|N|2434469|2434558|2434123|2434398|N|N|N|N|N| +2434489|AAAAAAAAJLFCFCAA|1953-04-21|639|2782|214|1953|2|4|21|2|1953|214|2782|Tuesday|1953Q2|N|N|N|2434469|2434558|2434124|2434399|N|N|N|N|N| +2434490|AAAAAAAAKLFCFCAA|1953-04-22|639|2782|214|1953|3|4|22|2|1953|214|2782|Wednesday|1953Q2|N|N|N|2434469|2434558|2434125|2434400|N|N|N|N|N| +2434491|AAAAAAAALLFCFCAA|1953-04-23|639|2782|214|1953|4|4|23|2|1953|214|2782|Thursday|1953Q2|N|N|N|2434469|2434558|2434126|2434401|N|N|N|N|N| +2434492|AAAAAAAAMLFCFCAA|1953-04-24|639|2782|214|1953|5|4|24|2|1953|214|2782|Friday|1953Q2|N|Y|N|2434469|2434558|2434127|2434402|N|N|N|N|N| +2434493|AAAAAAAANLFCFCAA|1953-04-25|639|2782|214|1953|6|4|25|2|1953|214|2782|Saturday|1953Q2|N|Y|N|2434469|2434558|2434128|2434403|N|N|N|N|N| +2434494|AAAAAAAAOLFCFCAA|1953-04-26|639|2782|214|1953|0|4|26|2|1953|214|2782|Sunday|1953Q2|N|N|N|2434469|2434558|2434129|2434404|N|N|N|N|N| +2434495|AAAAAAAAPLFCFCAA|1953-04-27|639|2782|214|1953|1|4|27|2|1953|214|2782|Monday|1953Q2|N|N|N|2434469|2434558|2434130|2434405|N|N|N|N|N| +2434496|AAAAAAAAAMFCFCAA|1953-04-28|639|2783|214|1953|2|4|28|2|1953|214|2783|Tuesday|1953Q2|N|N|N|2434469|2434558|2434131|2434406|N|N|N|N|N| +2434497|AAAAAAAABMFCFCAA|1953-04-29|639|2783|214|1953|3|4|29|2|1953|214|2783|Wednesday|1953Q2|N|N|N|2434469|2434558|2434132|2434407|N|N|N|N|N| +2434498|AAAAAAAACMFCFCAA|1953-04-30|639|2783|214|1953|4|4|30|2|1953|214|2783|Thursday|1953Q2|N|N|N|2434469|2434558|2434133|2434408|N|N|N|N|N| +2434499|AAAAAAAADMFCFCAA|1953-05-01|640|2783|214|1953|5|5|1|2|1953|214|2783|Friday|1953Q2|N|Y|N|2434499|2434618|2434134|2434409|N|N|N|N|N| +2434500|AAAAAAAAEMFCFCAA|1953-05-02|640|2783|214|1953|6|5|2|2|1953|214|2783|Saturday|1953Q2|N|Y|N|2434499|2434618|2434135|2434410|N|N|N|N|N| +2434501|AAAAAAAAFMFCFCAA|1953-05-03|640|2783|214|1953|0|5|3|2|1953|214|2783|Sunday|1953Q2|N|N|N|2434499|2434618|2434136|2434411|N|N|N|N|N| +2434502|AAAAAAAAGMFCFCAA|1953-05-04|640|2783|214|1953|1|5|4|2|1953|214|2783|Monday|1953Q2|N|N|N|2434499|2434618|2434137|2434412|N|N|N|N|N| +2434503|AAAAAAAAHMFCFCAA|1953-05-05|640|2784|214|1953|2|5|5|2|1953|214|2784|Tuesday|1953Q2|N|N|N|2434499|2434618|2434138|2434413|N|N|N|N|N| +2434504|AAAAAAAAIMFCFCAA|1953-05-06|640|2784|214|1953|3|5|6|2|1953|214|2784|Wednesday|1953Q2|N|N|N|2434499|2434618|2434139|2434414|N|N|N|N|N| +2434505|AAAAAAAAJMFCFCAA|1953-05-07|640|2784|214|1953|4|5|7|2|1953|214|2784|Thursday|1953Q2|N|N|N|2434499|2434618|2434140|2434415|N|N|N|N|N| +2434506|AAAAAAAAKMFCFCAA|1953-05-08|640|2784|214|1953|5|5|8|2|1953|214|2784|Friday|1953Q2|N|Y|N|2434499|2434618|2434141|2434416|N|N|N|N|N| +2434507|AAAAAAAALMFCFCAA|1953-05-09|640|2784|214|1953|6|5|9|2|1953|214|2784|Saturday|1953Q2|N|Y|N|2434499|2434618|2434142|2434417|N|N|N|N|N| +2434508|AAAAAAAAMMFCFCAA|1953-05-10|640|2784|214|1953|0|5|10|2|1953|214|2784|Sunday|1953Q2|N|N|N|2434499|2434618|2434143|2434418|N|N|N|N|N| +2434509|AAAAAAAANMFCFCAA|1953-05-11|640|2784|214|1953|1|5|11|2|1953|214|2784|Monday|1953Q2|N|N|N|2434499|2434618|2434144|2434419|N|N|N|N|N| +2434510|AAAAAAAAOMFCFCAA|1953-05-12|640|2785|214|1953|2|5|12|2|1953|214|2785|Tuesday|1953Q2|N|N|N|2434499|2434618|2434145|2434420|N|N|N|N|N| +2434511|AAAAAAAAPMFCFCAA|1953-05-13|640|2785|214|1953|3|5|13|2|1953|214|2785|Wednesday|1953Q2|N|N|N|2434499|2434618|2434146|2434421|N|N|N|N|N| +2434512|AAAAAAAAANFCFCAA|1953-05-14|640|2785|214|1953|4|5|14|2|1953|214|2785|Thursday|1953Q2|N|N|N|2434499|2434618|2434147|2434422|N|N|N|N|N| +2434513|AAAAAAAABNFCFCAA|1953-05-15|640|2785|214|1953|5|5|15|2|1953|214|2785|Friday|1953Q2|N|Y|N|2434499|2434618|2434148|2434423|N|N|N|N|N| +2434514|AAAAAAAACNFCFCAA|1953-05-16|640|2785|214|1953|6|5|16|2|1953|214|2785|Saturday|1953Q2|N|Y|N|2434499|2434618|2434149|2434424|N|N|N|N|N| +2434515|AAAAAAAADNFCFCAA|1953-05-17|640|2785|214|1953|0|5|17|2|1953|214|2785|Sunday|1953Q2|N|N|N|2434499|2434618|2434150|2434425|N|N|N|N|N| +2434516|AAAAAAAAENFCFCAA|1953-05-18|640|2785|214|1953|1|5|18|2|1953|214|2785|Monday|1953Q2|N|N|N|2434499|2434618|2434151|2434426|N|N|N|N|N| +2434517|AAAAAAAAFNFCFCAA|1953-05-19|640|2786|214|1953|2|5|19|2|1953|214|2786|Tuesday|1953Q2|N|N|N|2434499|2434618|2434152|2434427|N|N|N|N|N| +2434518|AAAAAAAAGNFCFCAA|1953-05-20|640|2786|214|1953|3|5|20|2|1953|214|2786|Wednesday|1953Q2|N|N|N|2434499|2434618|2434153|2434428|N|N|N|N|N| +2434519|AAAAAAAAHNFCFCAA|1953-05-21|640|2786|214|1953|4|5|21|2|1953|214|2786|Thursday|1953Q2|N|N|N|2434499|2434618|2434154|2434429|N|N|N|N|N| +2434520|AAAAAAAAINFCFCAA|1953-05-22|640|2786|214|1953|5|5|22|2|1953|214|2786|Friday|1953Q2|N|Y|N|2434499|2434618|2434155|2434430|N|N|N|N|N| +2434521|AAAAAAAAJNFCFCAA|1953-05-23|640|2786|214|1953|6|5|23|2|1953|214|2786|Saturday|1953Q2|N|Y|N|2434499|2434618|2434156|2434431|N|N|N|N|N| +2434522|AAAAAAAAKNFCFCAA|1953-05-24|640|2786|214|1953|0|5|24|2|1953|214|2786|Sunday|1953Q2|N|N|N|2434499|2434618|2434157|2434432|N|N|N|N|N| +2434523|AAAAAAAALNFCFCAA|1953-05-25|640|2786|214|1953|1|5|25|2|1953|214|2786|Monday|1953Q2|N|N|N|2434499|2434618|2434158|2434433|N|N|N|N|N| +2434524|AAAAAAAAMNFCFCAA|1953-05-26|640|2787|214|1953|2|5|26|2|1953|214|2787|Tuesday|1953Q2|N|N|N|2434499|2434618|2434159|2434434|N|N|N|N|N| +2434525|AAAAAAAANNFCFCAA|1953-05-27|640|2787|214|1953|3|5|27|2|1953|214|2787|Wednesday|1953Q2|N|N|N|2434499|2434618|2434160|2434435|N|N|N|N|N| +2434526|AAAAAAAAONFCFCAA|1953-05-28|640|2787|214|1953|4|5|28|2|1953|214|2787|Thursday|1953Q2|N|N|N|2434499|2434618|2434161|2434436|N|N|N|N|N| +2434527|AAAAAAAAPNFCFCAA|1953-05-29|640|2787|214|1953|5|5|29|2|1953|214|2787|Friday|1953Q2|N|Y|N|2434499|2434618|2434162|2434437|N|N|N|N|N| +2434528|AAAAAAAAAOFCFCAA|1953-05-30|640|2787|214|1953|6|5|30|2|1953|214|2787|Saturday|1953Q2|N|Y|N|2434499|2434618|2434163|2434438|N|N|N|N|N| +2434529|AAAAAAAABOFCFCAA|1953-05-31|640|2787|214|1953|0|5|31|2|1953|214|2787|Sunday|1953Q2|N|N|N|2434499|2434618|2434164|2434439|N|N|N|N|N| +2434530|AAAAAAAACOFCFCAA|1953-06-01|641|2787|215|1953|1|6|1|2|1953|215|2787|Monday|1953Q2|N|N|N|2434530|2434680|2434165|2434440|N|N|N|N|N| +2434531|AAAAAAAADOFCFCAA|1953-06-02|641|2788|215|1953|2|6|2|2|1953|215|2788|Tuesday|1953Q2|N|N|N|2434530|2434680|2434166|2434441|N|N|N|N|N| +2434532|AAAAAAAAEOFCFCAA|1953-06-03|641|2788|215|1953|3|6|3|2|1953|215|2788|Wednesday|1953Q2|N|N|N|2434530|2434680|2434167|2434442|N|N|N|N|N| +2434533|AAAAAAAAFOFCFCAA|1953-06-04|641|2788|215|1953|4|6|4|2|1953|215|2788|Thursday|1953Q2|N|N|N|2434530|2434680|2434168|2434443|N|N|N|N|N| +2434534|AAAAAAAAGOFCFCAA|1953-06-05|641|2788|215|1953|5|6|5|2|1953|215|2788|Friday|1953Q2|N|Y|N|2434530|2434680|2434169|2434444|N|N|N|N|N| +2434535|AAAAAAAAHOFCFCAA|1953-06-06|641|2788|215|1953|6|6|6|2|1953|215|2788|Saturday|1953Q2|N|Y|N|2434530|2434680|2434170|2434445|N|N|N|N|N| +2434536|AAAAAAAAIOFCFCAA|1953-06-07|641|2788|215|1953|0|6|7|2|1953|215|2788|Sunday|1953Q2|N|N|N|2434530|2434680|2434171|2434446|N|N|N|N|N| +2434537|AAAAAAAAJOFCFCAA|1953-06-08|641|2788|215|1953|1|6|8|2|1953|215|2788|Monday|1953Q2|N|N|N|2434530|2434680|2434172|2434447|N|N|N|N|N| +2434538|AAAAAAAAKOFCFCAA|1953-06-09|641|2789|215|1953|2|6|9|2|1953|215|2789|Tuesday|1953Q2|N|N|N|2434530|2434680|2434173|2434448|N|N|N|N|N| +2434539|AAAAAAAALOFCFCAA|1953-06-10|641|2789|215|1953|3|6|10|2|1953|215|2789|Wednesday|1953Q2|N|N|N|2434530|2434680|2434174|2434449|N|N|N|N|N| +2434540|AAAAAAAAMOFCFCAA|1953-06-11|641|2789|215|1953|4|6|11|2|1953|215|2789|Thursday|1953Q2|N|N|N|2434530|2434680|2434175|2434450|N|N|N|N|N| +2434541|AAAAAAAANOFCFCAA|1953-06-12|641|2789|215|1953|5|6|12|2|1953|215|2789|Friday|1953Q2|N|Y|N|2434530|2434680|2434176|2434451|N|N|N|N|N| +2434542|AAAAAAAAOOFCFCAA|1953-06-13|641|2789|215|1953|6|6|13|2|1953|215|2789|Saturday|1953Q2|N|Y|N|2434530|2434680|2434177|2434452|N|N|N|N|N| +2434543|AAAAAAAAPOFCFCAA|1953-06-14|641|2789|215|1953|0|6|14|2|1953|215|2789|Sunday|1953Q2|N|N|N|2434530|2434680|2434178|2434453|N|N|N|N|N| +2434544|AAAAAAAAAPFCFCAA|1953-06-15|641|2789|215|1953|1|6|15|2|1953|215|2789|Monday|1953Q2|N|N|N|2434530|2434680|2434179|2434454|N|N|N|N|N| +2434545|AAAAAAAABPFCFCAA|1953-06-16|641|2790|215|1953|2|6|16|2|1953|215|2790|Tuesday|1953Q2|N|N|N|2434530|2434680|2434180|2434455|N|N|N|N|N| +2434546|AAAAAAAACPFCFCAA|1953-06-17|641|2790|215|1953|3|6|17|2|1953|215|2790|Wednesday|1953Q2|N|N|N|2434530|2434680|2434181|2434456|N|N|N|N|N| +2434547|AAAAAAAADPFCFCAA|1953-06-18|641|2790|215|1953|4|6|18|2|1953|215|2790|Thursday|1953Q2|N|N|N|2434530|2434680|2434182|2434457|N|N|N|N|N| +2434548|AAAAAAAAEPFCFCAA|1953-06-19|641|2790|215|1953|5|6|19|2|1953|215|2790|Friday|1953Q2|N|Y|N|2434530|2434680|2434183|2434458|N|N|N|N|N| +2434549|AAAAAAAAFPFCFCAA|1953-06-20|641|2790|215|1953|6|6|20|2|1953|215|2790|Saturday|1953Q2|N|Y|N|2434530|2434680|2434184|2434459|N|N|N|N|N| +2434550|AAAAAAAAGPFCFCAA|1953-06-21|641|2790|215|1953|0|6|21|2|1953|215|2790|Sunday|1953Q2|N|N|N|2434530|2434680|2434185|2434460|N|N|N|N|N| +2434551|AAAAAAAAHPFCFCAA|1953-06-22|641|2790|215|1953|1|6|22|2|1953|215|2790|Monday|1953Q2|N|N|N|2434530|2434680|2434186|2434461|N|N|N|N|N| +2434552|AAAAAAAAIPFCFCAA|1953-06-23|641|2791|215|1953|2|6|23|2|1953|215|2791|Tuesday|1953Q2|N|N|N|2434530|2434680|2434187|2434462|N|N|N|N|N| +2434553|AAAAAAAAJPFCFCAA|1953-06-24|641|2791|215|1953|3|6|24|2|1953|215|2791|Wednesday|1953Q2|N|N|N|2434530|2434680|2434188|2434463|N|N|N|N|N| +2434554|AAAAAAAAKPFCFCAA|1953-06-25|641|2791|215|1953|4|6|25|2|1953|215|2791|Thursday|1953Q2|N|N|N|2434530|2434680|2434189|2434464|N|N|N|N|N| +2434555|AAAAAAAALPFCFCAA|1953-06-26|641|2791|215|1953|5|6|26|2|1953|215|2791|Friday|1953Q2|N|Y|N|2434530|2434680|2434190|2434465|N|N|N|N|N| +2434556|AAAAAAAAMPFCFCAA|1953-06-27|641|2791|215|1953|6|6|27|2|1953|215|2791|Saturday|1953Q2|N|Y|N|2434530|2434680|2434191|2434466|N|N|N|N|N| +2434557|AAAAAAAANPFCFCAA|1953-06-28|641|2791|215|1953|0|6|28|2|1953|215|2791|Sunday|1953Q2|N|N|N|2434530|2434680|2434192|2434467|N|N|N|N|N| +2434558|AAAAAAAAOPFCFCAA|1953-06-29|641|2791|215|1953|1|6|29|2|1953|215|2791|Monday|1953Q2|N|N|N|2434530|2434680|2434193|2434468|N|N|N|N|N| +2434559|AAAAAAAAPPFCFCAA|1953-06-30|641|2792|215|1953|2|6|30|2|1953|215|2792|Tuesday|1953Q2|N|N|N|2434530|2434680|2434194|2434469|N|N|N|N|N| +2434560|AAAAAAAAAAGCFCAA|1953-07-01|642|2792|215|1953|3|7|1|2|1953|215|2792|Wednesday|1953Q2|N|N|N|2434560|2434740|2434195|2434469|N|N|N|N|N| +2434561|AAAAAAAABAGCFCAA|1953-07-02|642|2792|215|1953|4|7|2|3|1953|215|2792|Thursday|1953Q3|N|N|N|2434560|2434740|2434196|2434470|N|N|N|N|N| +2434562|AAAAAAAACAGCFCAA|1953-07-03|642|2792|215|1953|5|7|3|3|1953|215|2792|Friday|1953Q3|N|Y|N|2434560|2434740|2434197|2434471|N|N|N|N|N| +2434563|AAAAAAAADAGCFCAA|1953-07-04|642|2792|215|1953|6|7|4|3|1953|215|2792|Saturday|1953Q3|N|Y|N|2434560|2434740|2434198|2434472|N|N|N|N|N| +2434564|AAAAAAAAEAGCFCAA|1953-07-05|642|2792|215|1953|0|7|5|3|1953|215|2792|Sunday|1953Q3|Y|N|N|2434560|2434740|2434199|2434473|N|N|N|N|N| +2434565|AAAAAAAAFAGCFCAA|1953-07-06|642|2792|215|1953|1|7|6|3|1953|215|2792|Monday|1953Q3|N|N|Y|2434560|2434740|2434200|2434474|N|N|N|N|N| +2434566|AAAAAAAAGAGCFCAA|1953-07-07|642|2793|215|1953|2|7|7|3|1953|215|2793|Tuesday|1953Q3|N|N|N|2434560|2434740|2434201|2434475|N|N|N|N|N| +2434567|AAAAAAAAHAGCFCAA|1953-07-08|642|2793|215|1953|3|7|8|3|1953|215|2793|Wednesday|1953Q3|N|N|N|2434560|2434740|2434202|2434476|N|N|N|N|N| +2434568|AAAAAAAAIAGCFCAA|1953-07-09|642|2793|215|1953|4|7|9|3|1953|215|2793|Thursday|1953Q3|N|N|N|2434560|2434740|2434203|2434477|N|N|N|N|N| +2434569|AAAAAAAAJAGCFCAA|1953-07-10|642|2793|215|1953|5|7|10|3|1953|215|2793|Friday|1953Q3|N|Y|N|2434560|2434740|2434204|2434478|N|N|N|N|N| +2434570|AAAAAAAAKAGCFCAA|1953-07-11|642|2793|215|1953|6|7|11|3|1953|215|2793|Saturday|1953Q3|N|Y|N|2434560|2434740|2434205|2434479|N|N|N|N|N| +2434571|AAAAAAAALAGCFCAA|1953-07-12|642|2793|215|1953|0|7|12|3|1953|215|2793|Sunday|1953Q3|N|N|N|2434560|2434740|2434206|2434480|N|N|N|N|N| +2434572|AAAAAAAAMAGCFCAA|1953-07-13|642|2793|215|1953|1|7|13|3|1953|215|2793|Monday|1953Q3|N|N|N|2434560|2434740|2434207|2434481|N|N|N|N|N| +2434573|AAAAAAAANAGCFCAA|1953-07-14|642|2794|215|1953|2|7|14|3|1953|215|2794|Tuesday|1953Q3|N|N|N|2434560|2434740|2434208|2434482|N|N|N|N|N| +2434574|AAAAAAAAOAGCFCAA|1953-07-15|642|2794|215|1953|3|7|15|3|1953|215|2794|Wednesday|1953Q3|N|N|N|2434560|2434740|2434209|2434483|N|N|N|N|N| +2434575|AAAAAAAAPAGCFCAA|1953-07-16|642|2794|215|1953|4|7|16|3|1953|215|2794|Thursday|1953Q3|N|N|N|2434560|2434740|2434210|2434484|N|N|N|N|N| +2434576|AAAAAAAAABGCFCAA|1953-07-17|642|2794|215|1953|5|7|17|3|1953|215|2794|Friday|1953Q3|N|Y|N|2434560|2434740|2434211|2434485|N|N|N|N|N| +2434577|AAAAAAAABBGCFCAA|1953-07-18|642|2794|215|1953|6|7|18|3|1953|215|2794|Saturday|1953Q3|N|Y|N|2434560|2434740|2434212|2434486|N|N|N|N|N| +2434578|AAAAAAAACBGCFCAA|1953-07-19|642|2794|215|1953|0|7|19|3|1953|215|2794|Sunday|1953Q3|N|N|N|2434560|2434740|2434213|2434487|N|N|N|N|N| +2434579|AAAAAAAADBGCFCAA|1953-07-20|642|2794|215|1953|1|7|20|3|1953|215|2794|Monday|1953Q3|N|N|N|2434560|2434740|2434214|2434488|N|N|N|N|N| +2434580|AAAAAAAAEBGCFCAA|1953-07-21|642|2795|215|1953|2|7|21|3|1953|215|2795|Tuesday|1953Q3|N|N|N|2434560|2434740|2434215|2434489|N|N|N|N|N| +2434581|AAAAAAAAFBGCFCAA|1953-07-22|642|2795|215|1953|3|7|22|3|1953|215|2795|Wednesday|1953Q3|N|N|N|2434560|2434740|2434216|2434490|N|N|N|N|N| +2434582|AAAAAAAAGBGCFCAA|1953-07-23|642|2795|215|1953|4|7|23|3|1953|215|2795|Thursday|1953Q3|N|N|N|2434560|2434740|2434217|2434491|N|N|N|N|N| +2434583|AAAAAAAAHBGCFCAA|1953-07-24|642|2795|215|1953|5|7|24|3|1953|215|2795|Friday|1953Q3|N|Y|N|2434560|2434740|2434218|2434492|N|N|N|N|N| +2434584|AAAAAAAAIBGCFCAA|1953-07-25|642|2795|215|1953|6|7|25|3|1953|215|2795|Saturday|1953Q3|N|Y|N|2434560|2434740|2434219|2434493|N|N|N|N|N| +2434585|AAAAAAAAJBGCFCAA|1953-07-26|642|2795|215|1953|0|7|26|3|1953|215|2795|Sunday|1953Q3|N|N|N|2434560|2434740|2434220|2434494|N|N|N|N|N| +2434586|AAAAAAAAKBGCFCAA|1953-07-27|642|2795|215|1953|1|7|27|3|1953|215|2795|Monday|1953Q3|N|N|N|2434560|2434740|2434221|2434495|N|N|N|N|N| +2434587|AAAAAAAALBGCFCAA|1953-07-28|642|2796|215|1953|2|7|28|3|1953|215|2796|Tuesday|1953Q3|N|N|N|2434560|2434740|2434222|2434496|N|N|N|N|N| +2434588|AAAAAAAAMBGCFCAA|1953-07-29|642|2796|215|1953|3|7|29|3|1953|215|2796|Wednesday|1953Q3|N|N|N|2434560|2434740|2434223|2434497|N|N|N|N|N| +2434589|AAAAAAAANBGCFCAA|1953-07-30|642|2796|215|1953|4|7|30|3|1953|215|2796|Thursday|1953Q3|N|N|N|2434560|2434740|2434224|2434498|N|N|N|N|N| +2434590|AAAAAAAAOBGCFCAA|1953-07-31|642|2796|215|1953|5|7|31|3|1953|215|2796|Friday|1953Q3|N|Y|N|2434560|2434740|2434225|2434499|N|N|N|N|N| +2434591|AAAAAAAAPBGCFCAA|1953-08-01|643|2796|215|1953|6|8|1|3|1953|215|2796|Saturday|1953Q3|N|Y|N|2434591|2434802|2434226|2434500|N|N|N|N|N| +2434592|AAAAAAAAACGCFCAA|1953-08-02|643|2796|215|1953|0|8|2|3|1953|215|2796|Sunday|1953Q3|N|N|N|2434591|2434802|2434227|2434501|N|N|N|N|N| +2434593|AAAAAAAABCGCFCAA|1953-08-03|643|2796|215|1953|1|8|3|3|1953|215|2796|Monday|1953Q3|N|N|N|2434591|2434802|2434228|2434502|N|N|N|N|N| +2434594|AAAAAAAACCGCFCAA|1953-08-04|643|2797|215|1953|2|8|4|3|1953|215|2797|Tuesday|1953Q3|N|N|N|2434591|2434802|2434229|2434503|N|N|N|N|N| +2434595|AAAAAAAADCGCFCAA|1953-08-05|643|2797|215|1953|3|8|5|3|1953|215|2797|Wednesday|1953Q3|N|N|N|2434591|2434802|2434230|2434504|N|N|N|N|N| +2434596|AAAAAAAAECGCFCAA|1953-08-06|643|2797|215|1953|4|8|6|3|1953|215|2797|Thursday|1953Q3|N|N|N|2434591|2434802|2434231|2434505|N|N|N|N|N| +2434597|AAAAAAAAFCGCFCAA|1953-08-07|643|2797|215|1953|5|8|7|3|1953|215|2797|Friday|1953Q3|N|Y|N|2434591|2434802|2434232|2434506|N|N|N|N|N| +2434598|AAAAAAAAGCGCFCAA|1953-08-08|643|2797|215|1953|6|8|8|3|1953|215|2797|Saturday|1953Q3|N|Y|N|2434591|2434802|2434233|2434507|N|N|N|N|N| +2434599|AAAAAAAAHCGCFCAA|1953-08-09|643|2797|215|1953|0|8|9|3|1953|215|2797|Sunday|1953Q3|N|N|N|2434591|2434802|2434234|2434508|N|N|N|N|N| +2434600|AAAAAAAAICGCFCAA|1953-08-10|643|2797|215|1953|1|8|10|3|1953|215|2797|Monday|1953Q3|N|N|N|2434591|2434802|2434235|2434509|N|N|N|N|N| +2434601|AAAAAAAAJCGCFCAA|1953-08-11|643|2798|215|1953|2|8|11|3|1953|215|2798|Tuesday|1953Q3|N|N|N|2434591|2434802|2434236|2434510|N|N|N|N|N| +2434602|AAAAAAAAKCGCFCAA|1953-08-12|643|2798|215|1953|3|8|12|3|1953|215|2798|Wednesday|1953Q3|N|N|N|2434591|2434802|2434237|2434511|N|N|N|N|N| +2434603|AAAAAAAALCGCFCAA|1953-08-13|643|2798|215|1953|4|8|13|3|1953|215|2798|Thursday|1953Q3|N|N|N|2434591|2434802|2434238|2434512|N|N|N|N|N| +2434604|AAAAAAAAMCGCFCAA|1953-08-14|643|2798|215|1953|5|8|14|3|1953|215|2798|Friday|1953Q3|N|Y|N|2434591|2434802|2434239|2434513|N|N|N|N|N| +2434605|AAAAAAAANCGCFCAA|1953-08-15|643|2798|215|1953|6|8|15|3|1953|215|2798|Saturday|1953Q3|N|Y|N|2434591|2434802|2434240|2434514|N|N|N|N|N| +2434606|AAAAAAAAOCGCFCAA|1953-08-16|643|2798|215|1953|0|8|16|3|1953|215|2798|Sunday|1953Q3|N|N|N|2434591|2434802|2434241|2434515|N|N|N|N|N| +2434607|AAAAAAAAPCGCFCAA|1953-08-17|643|2798|215|1953|1|8|17|3|1953|215|2798|Monday|1953Q3|N|N|N|2434591|2434802|2434242|2434516|N|N|N|N|N| +2434608|AAAAAAAAADGCFCAA|1953-08-18|643|2799|215|1953|2|8|18|3|1953|215|2799|Tuesday|1953Q3|N|N|N|2434591|2434802|2434243|2434517|N|N|N|N|N| +2434609|AAAAAAAABDGCFCAA|1953-08-19|643|2799|215|1953|3|8|19|3|1953|215|2799|Wednesday|1953Q3|N|N|N|2434591|2434802|2434244|2434518|N|N|N|N|N| +2434610|AAAAAAAACDGCFCAA|1953-08-20|643|2799|215|1953|4|8|20|3|1953|215|2799|Thursday|1953Q3|N|N|N|2434591|2434802|2434245|2434519|N|N|N|N|N| +2434611|AAAAAAAADDGCFCAA|1953-08-21|643|2799|215|1953|5|8|21|3|1953|215|2799|Friday|1953Q3|N|Y|N|2434591|2434802|2434246|2434520|N|N|N|N|N| +2434612|AAAAAAAAEDGCFCAA|1953-08-22|643|2799|215|1953|6|8|22|3|1953|215|2799|Saturday|1953Q3|N|Y|N|2434591|2434802|2434247|2434521|N|N|N|N|N| +2434613|AAAAAAAAFDGCFCAA|1953-08-23|643|2799|215|1953|0|8|23|3|1953|215|2799|Sunday|1953Q3|N|N|N|2434591|2434802|2434248|2434522|N|N|N|N|N| +2434614|AAAAAAAAGDGCFCAA|1953-08-24|643|2799|215|1953|1|8|24|3|1953|215|2799|Monday|1953Q3|N|N|N|2434591|2434802|2434249|2434523|N|N|N|N|N| +2434615|AAAAAAAAHDGCFCAA|1953-08-25|643|2800|215|1953|2|8|25|3|1953|215|2800|Tuesday|1953Q3|N|N|N|2434591|2434802|2434250|2434524|N|N|N|N|N| +2434616|AAAAAAAAIDGCFCAA|1953-08-26|643|2800|215|1953|3|8|26|3|1953|215|2800|Wednesday|1953Q3|N|N|N|2434591|2434802|2434251|2434525|N|N|N|N|N| +2434617|AAAAAAAAJDGCFCAA|1953-08-27|643|2800|215|1953|4|8|27|3|1953|215|2800|Thursday|1953Q3|N|N|N|2434591|2434802|2434252|2434526|N|N|N|N|N| +2434618|AAAAAAAAKDGCFCAA|1953-08-28|643|2800|215|1953|5|8|28|3|1953|215|2800|Friday|1953Q3|N|Y|N|2434591|2434802|2434253|2434527|N|N|N|N|N| +2434619|AAAAAAAALDGCFCAA|1953-08-29|643|2800|215|1953|6|8|29|3|1953|215|2800|Saturday|1953Q3|N|Y|N|2434591|2434802|2434254|2434528|N|N|N|N|N| +2434620|AAAAAAAAMDGCFCAA|1953-08-30|643|2800|215|1953|0|8|30|3|1953|215|2800|Sunday|1953Q3|N|N|N|2434591|2434802|2434255|2434529|N|N|N|N|N| +2434621|AAAAAAAANDGCFCAA|1953-08-31|643|2800|215|1953|1|8|31|3|1953|215|2800|Monday|1953Q3|N|N|N|2434591|2434802|2434256|2434530|N|N|N|N|N| +2434622|AAAAAAAAODGCFCAA|1953-09-01|644|2801|216|1953|2|9|1|3|1953|216|2801|Tuesday|1953Q3|N|N|N|2434622|2434864|2434257|2434531|N|N|N|N|N| +2434623|AAAAAAAAPDGCFCAA|1953-09-02|644|2801|216|1953|3|9|2|3|1953|216|2801|Wednesday|1953Q3|N|N|N|2434622|2434864|2434258|2434532|N|N|N|N|N| +2434624|AAAAAAAAAEGCFCAA|1953-09-03|644|2801|216|1953|4|9|3|3|1953|216|2801|Thursday|1953Q3|N|N|N|2434622|2434864|2434259|2434533|N|N|N|N|N| +2434625|AAAAAAAABEGCFCAA|1953-09-04|644|2801|216|1953|5|9|4|3|1953|216|2801|Friday|1953Q3|N|Y|N|2434622|2434864|2434260|2434534|N|N|N|N|N| +2434626|AAAAAAAACEGCFCAA|1953-09-05|644|2801|216|1953|6|9|5|3|1953|216|2801|Saturday|1953Q3|N|Y|N|2434622|2434864|2434261|2434535|N|N|N|N|N| +2434627|AAAAAAAADEGCFCAA|1953-09-06|644|2801|216|1953|0|9|6|3|1953|216|2801|Sunday|1953Q3|N|N|N|2434622|2434864|2434262|2434536|N|N|N|N|N| +2434628|AAAAAAAAEEGCFCAA|1953-09-07|644|2801|216|1953|1|9|7|3|1953|216|2801|Monday|1953Q3|N|N|N|2434622|2434864|2434263|2434537|N|N|N|N|N| +2434629|AAAAAAAAFEGCFCAA|1953-09-08|644|2802|216|1953|2|9|8|3|1953|216|2802|Tuesday|1953Q3|N|N|N|2434622|2434864|2434264|2434538|N|N|N|N|N| +2434630|AAAAAAAAGEGCFCAA|1953-09-09|644|2802|216|1953|3|9|9|3|1953|216|2802|Wednesday|1953Q3|N|N|N|2434622|2434864|2434265|2434539|N|N|N|N|N| +2434631|AAAAAAAAHEGCFCAA|1953-09-10|644|2802|216|1953|4|9|10|3|1953|216|2802|Thursday|1953Q3|N|N|N|2434622|2434864|2434266|2434540|N|N|N|N|N| +2434632|AAAAAAAAIEGCFCAA|1953-09-11|644|2802|216|1953|5|9|11|3|1953|216|2802|Friday|1953Q3|N|Y|N|2434622|2434864|2434267|2434541|N|N|N|N|N| +2434633|AAAAAAAAJEGCFCAA|1953-09-12|644|2802|216|1953|6|9|12|3|1953|216|2802|Saturday|1953Q3|N|Y|N|2434622|2434864|2434268|2434542|N|N|N|N|N| +2434634|AAAAAAAAKEGCFCAA|1953-09-13|644|2802|216|1953|0|9|13|3|1953|216|2802|Sunday|1953Q3|N|N|N|2434622|2434864|2434269|2434543|N|N|N|N|N| +2434635|AAAAAAAALEGCFCAA|1953-09-14|644|2802|216|1953|1|9|14|3|1953|216|2802|Monday|1953Q3|N|N|N|2434622|2434864|2434270|2434544|N|N|N|N|N| +2434636|AAAAAAAAMEGCFCAA|1953-09-15|644|2803|216|1953|2|9|15|3|1953|216|2803|Tuesday|1953Q3|N|N|N|2434622|2434864|2434271|2434545|N|N|N|N|N| +2434637|AAAAAAAANEGCFCAA|1953-09-16|644|2803|216|1953|3|9|16|3|1953|216|2803|Wednesday|1953Q3|N|N|N|2434622|2434864|2434272|2434546|N|N|N|N|N| +2434638|AAAAAAAAOEGCFCAA|1953-09-17|644|2803|216|1953|4|9|17|3|1953|216|2803|Thursday|1953Q3|N|N|N|2434622|2434864|2434273|2434547|N|N|N|N|N| +2434639|AAAAAAAAPEGCFCAA|1953-09-18|644|2803|216|1953|5|9|18|3|1953|216|2803|Friday|1953Q3|N|Y|N|2434622|2434864|2434274|2434548|N|N|N|N|N| +2434640|AAAAAAAAAFGCFCAA|1953-09-19|644|2803|216|1953|6|9|19|3|1953|216|2803|Saturday|1953Q3|N|Y|N|2434622|2434864|2434275|2434549|N|N|N|N|N| +2434641|AAAAAAAABFGCFCAA|1953-09-20|644|2803|216|1953|0|9|20|3|1953|216|2803|Sunday|1953Q3|N|N|N|2434622|2434864|2434276|2434550|N|N|N|N|N| +2434642|AAAAAAAACFGCFCAA|1953-09-21|644|2803|216|1953|1|9|21|3|1953|216|2803|Monday|1953Q3|N|N|N|2434622|2434864|2434277|2434551|N|N|N|N|N| +2434643|AAAAAAAADFGCFCAA|1953-09-22|644|2804|216|1953|2|9|22|3|1953|216|2804|Tuesday|1953Q3|N|N|N|2434622|2434864|2434278|2434552|N|N|N|N|N| +2434644|AAAAAAAAEFGCFCAA|1953-09-23|644|2804|216|1953|3|9|23|3|1953|216|2804|Wednesday|1953Q3|N|N|N|2434622|2434864|2434279|2434553|N|N|N|N|N| +2434645|AAAAAAAAFFGCFCAA|1953-09-24|644|2804|216|1953|4|9|24|3|1953|216|2804|Thursday|1953Q3|N|N|N|2434622|2434864|2434280|2434554|N|N|N|N|N| +2434646|AAAAAAAAGFGCFCAA|1953-09-25|644|2804|216|1953|5|9|25|3|1953|216|2804|Friday|1953Q3|N|Y|N|2434622|2434864|2434281|2434555|N|N|N|N|N| +2434647|AAAAAAAAHFGCFCAA|1953-09-26|644|2804|216|1953|6|9|26|3|1953|216|2804|Saturday|1953Q3|N|Y|N|2434622|2434864|2434282|2434556|N|N|N|N|N| +2434648|AAAAAAAAIFGCFCAA|1953-09-27|644|2804|216|1953|0|9|27|3|1953|216|2804|Sunday|1953Q3|N|N|N|2434622|2434864|2434283|2434557|N|N|N|N|N| +2434649|AAAAAAAAJFGCFCAA|1953-09-28|644|2804|216|1953|1|9|28|3|1953|216|2804|Monday|1953Q3|N|N|N|2434622|2434864|2434284|2434558|N|N|N|N|N| +2434650|AAAAAAAAKFGCFCAA|1953-09-29|644|2805|216|1953|2|9|29|3|1953|216|2805|Tuesday|1953Q3|N|N|N|2434622|2434864|2434285|2434559|N|N|N|N|N| +2434651|AAAAAAAALFGCFCAA|1953-09-30|644|2805|216|1953|3|9|30|3|1953|216|2805|Wednesday|1953Q3|N|N|N|2434622|2434864|2434286|2434560|N|N|N|N|N| +2434652|AAAAAAAAMFGCFCAA|1953-10-01|645|2805|216|1953|4|10|1|3|1953|216|2805|Thursday|1953Q3|N|N|N|2434652|2434924|2434287|2434560|N|N|N|N|N| +2434653|AAAAAAAANFGCFCAA|1953-10-02|645|2805|216|1953|5|10|2|4|1953|216|2805|Friday|1953Q4|N|Y|N|2434652|2434924|2434288|2434561|N|N|N|N|N| +2434654|AAAAAAAAOFGCFCAA|1953-10-03|645|2805|216|1953|6|10|3|4|1953|216|2805|Saturday|1953Q4|N|Y|N|2434652|2434924|2434289|2434562|N|N|N|N|N| +2434655|AAAAAAAAPFGCFCAA|1953-10-04|645|2805|216|1953|0|10|4|4|1953|216|2805|Sunday|1953Q4|N|N|N|2434652|2434924|2434290|2434563|N|N|N|N|N| +2434656|AAAAAAAAAGGCFCAA|1953-10-05|645|2805|216|1953|1|10|5|4|1953|216|2805|Monday|1953Q4|N|N|N|2434652|2434924|2434291|2434564|N|N|N|N|N| +2434657|AAAAAAAABGGCFCAA|1953-10-06|645|2806|216|1953|2|10|6|4|1953|216|2806|Tuesday|1953Q4|N|N|N|2434652|2434924|2434292|2434565|N|N|N|N|N| +2434658|AAAAAAAACGGCFCAA|1953-10-07|645|2806|216|1953|3|10|7|4|1953|216|2806|Wednesday|1953Q4|N|N|N|2434652|2434924|2434293|2434566|N|N|N|N|N| +2434659|AAAAAAAADGGCFCAA|1953-10-08|645|2806|216|1953|4|10|8|4|1953|216|2806|Thursday|1953Q4|N|N|N|2434652|2434924|2434294|2434567|N|N|N|N|N| +2434660|AAAAAAAAEGGCFCAA|1953-10-09|645|2806|216|1953|5|10|9|4|1953|216|2806|Friday|1953Q4|N|Y|N|2434652|2434924|2434295|2434568|N|N|N|N|N| +2434661|AAAAAAAAFGGCFCAA|1953-10-10|645|2806|216|1953|6|10|10|4|1953|216|2806|Saturday|1953Q4|N|Y|N|2434652|2434924|2434296|2434569|N|N|N|N|N| +2434662|AAAAAAAAGGGCFCAA|1953-10-11|645|2806|216|1953|0|10|11|4|1953|216|2806|Sunday|1953Q4|N|N|N|2434652|2434924|2434297|2434570|N|N|N|N|N| +2434663|AAAAAAAAHGGCFCAA|1953-10-12|645|2806|216|1953|1|10|12|4|1953|216|2806|Monday|1953Q4|N|N|N|2434652|2434924|2434298|2434571|N|N|N|N|N| +2434664|AAAAAAAAIGGCFCAA|1953-10-13|645|2807|216|1953|2|10|13|4|1953|216|2807|Tuesday|1953Q4|N|N|N|2434652|2434924|2434299|2434572|N|N|N|N|N| +2434665|AAAAAAAAJGGCFCAA|1953-10-14|645|2807|216|1953|3|10|14|4|1953|216|2807|Wednesday|1953Q4|N|N|N|2434652|2434924|2434300|2434573|N|N|N|N|N| +2434666|AAAAAAAAKGGCFCAA|1953-10-15|645|2807|216|1953|4|10|15|4|1953|216|2807|Thursday|1953Q4|N|N|N|2434652|2434924|2434301|2434574|N|N|N|N|N| +2434667|AAAAAAAALGGCFCAA|1953-10-16|645|2807|216|1953|5|10|16|4|1953|216|2807|Friday|1953Q4|N|Y|N|2434652|2434924|2434302|2434575|N|N|N|N|N| +2434668|AAAAAAAAMGGCFCAA|1953-10-17|645|2807|216|1953|6|10|17|4|1953|216|2807|Saturday|1953Q4|N|Y|N|2434652|2434924|2434303|2434576|N|N|N|N|N| +2434669|AAAAAAAANGGCFCAA|1953-10-18|645|2807|216|1953|0|10|18|4|1953|216|2807|Sunday|1953Q4|N|N|N|2434652|2434924|2434304|2434577|N|N|N|N|N| +2434670|AAAAAAAAOGGCFCAA|1953-10-19|645|2807|216|1953|1|10|19|4|1953|216|2807|Monday|1953Q4|N|N|N|2434652|2434924|2434305|2434578|N|N|N|N|N| +2434671|AAAAAAAAPGGCFCAA|1953-10-20|645|2808|216|1953|2|10|20|4|1953|216|2808|Tuesday|1953Q4|N|N|N|2434652|2434924|2434306|2434579|N|N|N|N|N| +2434672|AAAAAAAAAHGCFCAA|1953-10-21|645|2808|216|1953|3|10|21|4|1953|216|2808|Wednesday|1953Q4|N|N|N|2434652|2434924|2434307|2434580|N|N|N|N|N| +2434673|AAAAAAAABHGCFCAA|1953-10-22|645|2808|216|1953|4|10|22|4|1953|216|2808|Thursday|1953Q4|N|N|N|2434652|2434924|2434308|2434581|N|N|N|N|N| +2434674|AAAAAAAACHGCFCAA|1953-10-23|645|2808|216|1953|5|10|23|4|1953|216|2808|Friday|1953Q4|N|Y|N|2434652|2434924|2434309|2434582|N|N|N|N|N| +2434675|AAAAAAAADHGCFCAA|1953-10-24|645|2808|216|1953|6|10|24|4|1953|216|2808|Saturday|1953Q4|N|Y|N|2434652|2434924|2434310|2434583|N|N|N|N|N| +2434676|AAAAAAAAEHGCFCAA|1953-10-25|645|2808|216|1953|0|10|25|4|1953|216|2808|Sunday|1953Q4|N|N|N|2434652|2434924|2434311|2434584|N|N|N|N|N| +2434677|AAAAAAAAFHGCFCAA|1953-10-26|645|2808|216|1953|1|10|26|4|1953|216|2808|Monday|1953Q4|N|N|N|2434652|2434924|2434312|2434585|N|N|N|N|N| +2434678|AAAAAAAAGHGCFCAA|1953-10-27|645|2809|216|1953|2|10|27|4|1953|216|2809|Tuesday|1953Q4|N|N|N|2434652|2434924|2434313|2434586|N|N|N|N|N| +2434679|AAAAAAAAHHGCFCAA|1953-10-28|645|2809|216|1953|3|10|28|4|1953|216|2809|Wednesday|1953Q4|N|N|N|2434652|2434924|2434314|2434587|N|N|N|N|N| +2434680|AAAAAAAAIHGCFCAA|1953-10-29|645|2809|216|1953|4|10|29|4|1953|216|2809|Thursday|1953Q4|N|N|N|2434652|2434924|2434315|2434588|N|N|N|N|N| +2434681|AAAAAAAAJHGCFCAA|1953-10-30|645|2809|216|1953|5|10|30|4|1953|216|2809|Friday|1953Q4|N|Y|N|2434652|2434924|2434316|2434589|N|N|N|N|N| +2434682|AAAAAAAAKHGCFCAA|1953-10-31|645|2809|216|1953|6|10|31|4|1953|216|2809|Saturday|1953Q4|N|Y|N|2434652|2434924|2434317|2434590|N|N|N|N|N| +2434683|AAAAAAAALHGCFCAA|1953-11-01|646|2809|216|1953|0|11|1|4|1953|216|2809|Sunday|1953Q4|N|N|N|2434683|2434986|2434318|2434591|N|N|N|N|N| +2434684|AAAAAAAAMHGCFCAA|1953-11-02|646|2809|216|1953|1|11|2|4|1953|216|2809|Monday|1953Q4|N|N|N|2434683|2434986|2434319|2434592|N|N|N|N|N| +2434685|AAAAAAAANHGCFCAA|1953-11-03|646|2810|216|1953|2|11|3|4|1953|216|2810|Tuesday|1953Q4|N|N|N|2434683|2434986|2434320|2434593|N|N|N|N|N| +2434686|AAAAAAAAOHGCFCAA|1953-11-04|646|2810|216|1953|3|11|4|4|1953|216|2810|Wednesday|1953Q4|N|N|N|2434683|2434986|2434321|2434594|N|N|N|N|N| +2434687|AAAAAAAAPHGCFCAA|1953-11-05|646|2810|216|1953|4|11|5|4|1953|216|2810|Thursday|1953Q4|N|N|N|2434683|2434986|2434322|2434595|N|N|N|N|N| +2434688|AAAAAAAAAIGCFCAA|1953-11-06|646|2810|216|1953|5|11|6|4|1953|216|2810|Friday|1953Q4|N|Y|N|2434683|2434986|2434323|2434596|N|N|N|N|N| +2434689|AAAAAAAABIGCFCAA|1953-11-07|646|2810|216|1953|6|11|7|4|1953|216|2810|Saturday|1953Q4|N|Y|N|2434683|2434986|2434324|2434597|N|N|N|N|N| +2434690|AAAAAAAACIGCFCAA|1953-11-08|646|2810|216|1953|0|11|8|4|1953|216|2810|Sunday|1953Q4|N|N|N|2434683|2434986|2434325|2434598|N|N|N|N|N| +2434691|AAAAAAAADIGCFCAA|1953-11-09|646|2810|216|1953|1|11|9|4|1953|216|2810|Monday|1953Q4|N|N|N|2434683|2434986|2434326|2434599|N|N|N|N|N| +2434692|AAAAAAAAEIGCFCAA|1953-11-10|646|2811|216|1953|2|11|10|4|1953|216|2811|Tuesday|1953Q4|N|N|N|2434683|2434986|2434327|2434600|N|N|N|N|N| +2434693|AAAAAAAAFIGCFCAA|1953-11-11|646|2811|216|1953|3|11|11|4|1953|216|2811|Wednesday|1953Q4|N|N|N|2434683|2434986|2434328|2434601|N|N|N|N|N| +2434694|AAAAAAAAGIGCFCAA|1953-11-12|646|2811|216|1953|4|11|12|4|1953|216|2811|Thursday|1953Q4|N|N|N|2434683|2434986|2434329|2434602|N|N|N|N|N| +2434695|AAAAAAAAHIGCFCAA|1953-11-13|646|2811|216|1953|5|11|13|4|1953|216|2811|Friday|1953Q4|N|Y|N|2434683|2434986|2434330|2434603|N|N|N|N|N| +2434696|AAAAAAAAIIGCFCAA|1953-11-14|646|2811|216|1953|6|11|14|4|1953|216|2811|Saturday|1953Q4|N|Y|N|2434683|2434986|2434331|2434604|N|N|N|N|N| +2434697|AAAAAAAAJIGCFCAA|1953-11-15|646|2811|216|1953|0|11|15|4|1953|216|2811|Sunday|1953Q4|N|N|N|2434683|2434986|2434332|2434605|N|N|N|N|N| +2434698|AAAAAAAAKIGCFCAA|1953-11-16|646|2811|216|1953|1|11|16|4|1953|216|2811|Monday|1953Q4|N|N|N|2434683|2434986|2434333|2434606|N|N|N|N|N| +2434699|AAAAAAAALIGCFCAA|1953-11-17|646|2812|216|1953|2|11|17|4|1953|216|2812|Tuesday|1953Q4|N|N|N|2434683|2434986|2434334|2434607|N|N|N|N|N| +2434700|AAAAAAAAMIGCFCAA|1953-11-18|646|2812|216|1953|3|11|18|4|1953|216|2812|Wednesday|1953Q4|N|N|N|2434683|2434986|2434335|2434608|N|N|N|N|N| +2434701|AAAAAAAANIGCFCAA|1953-11-19|646|2812|216|1953|4|11|19|4|1953|216|2812|Thursday|1953Q4|N|N|N|2434683|2434986|2434336|2434609|N|N|N|N|N| +2434702|AAAAAAAAOIGCFCAA|1953-11-20|646|2812|216|1953|5|11|20|4|1953|216|2812|Friday|1953Q4|N|Y|N|2434683|2434986|2434337|2434610|N|N|N|N|N| +2434703|AAAAAAAAPIGCFCAA|1953-11-21|646|2812|216|1953|6|11|21|4|1953|216|2812|Saturday|1953Q4|N|Y|N|2434683|2434986|2434338|2434611|N|N|N|N|N| +2434704|AAAAAAAAAJGCFCAA|1953-11-22|646|2812|216|1953|0|11|22|4|1953|216|2812|Sunday|1953Q4|N|N|N|2434683|2434986|2434339|2434612|N|N|N|N|N| +2434705|AAAAAAAABJGCFCAA|1953-11-23|646|2812|216|1953|1|11|23|4|1953|216|2812|Monday|1953Q4|N|N|N|2434683|2434986|2434340|2434613|N|N|N|N|N| +2434706|AAAAAAAACJGCFCAA|1953-11-24|646|2813|216|1953|2|11|24|4|1953|216|2813|Tuesday|1953Q4|N|N|N|2434683|2434986|2434341|2434614|N|N|N|N|N| +2434707|AAAAAAAADJGCFCAA|1953-11-25|646|2813|216|1953|3|11|25|4|1953|216|2813|Wednesday|1953Q4|N|N|N|2434683|2434986|2434342|2434615|N|N|N|N|N| +2434708|AAAAAAAAEJGCFCAA|1953-11-26|646|2813|216|1953|4|11|26|4|1953|216|2813|Thursday|1953Q4|N|N|N|2434683|2434986|2434343|2434616|N|N|N|N|N| +2434709|AAAAAAAAFJGCFCAA|1953-11-27|646|2813|216|1953|5|11|27|4|1953|216|2813|Friday|1953Q4|N|Y|N|2434683|2434986|2434344|2434617|N|N|N|N|N| +2434710|AAAAAAAAGJGCFCAA|1953-11-28|646|2813|216|1953|6|11|28|4|1953|216|2813|Saturday|1953Q4|N|Y|N|2434683|2434986|2434345|2434618|N|N|N|N|N| +2434711|AAAAAAAAHJGCFCAA|1953-11-29|646|2813|216|1953|0|11|29|4|1953|216|2813|Sunday|1953Q4|N|N|N|2434683|2434986|2434346|2434619|N|N|N|N|N| +2434712|AAAAAAAAIJGCFCAA|1953-11-30|646|2813|216|1953|1|11|30|4|1953|216|2813|Monday|1953Q4|N|N|N|2434683|2434986|2434347|2434620|N|N|N|N|N| +2434713|AAAAAAAAJJGCFCAA|1953-12-01|647|2814|217|1953|2|12|1|4|1953|217|2814|Tuesday|1953Q4|N|N|N|2434713|2435046|2434348|2434621|N|N|N|N|N| +2434714|AAAAAAAAKJGCFCAA|1953-12-02|647|2814|217|1953|3|12|2|4|1953|217|2814|Wednesday|1953Q4|N|N|N|2434713|2435046|2434349|2434622|N|N|N|N|N| +2434715|AAAAAAAALJGCFCAA|1953-12-03|647|2814|217|1953|4|12|3|4|1953|217|2814|Thursday|1953Q4|N|N|N|2434713|2435046|2434350|2434623|N|N|N|N|N| +2434716|AAAAAAAAMJGCFCAA|1953-12-04|647|2814|217|1953|5|12|4|4|1953|217|2814|Friday|1953Q4|N|Y|N|2434713|2435046|2434351|2434624|N|N|N|N|N| +2434717|AAAAAAAANJGCFCAA|1953-12-05|647|2814|217|1953|6|12|5|4|1953|217|2814|Saturday|1953Q4|N|Y|N|2434713|2435046|2434352|2434625|N|N|N|N|N| +2434718|AAAAAAAAOJGCFCAA|1953-12-06|647|2814|217|1953|0|12|6|4|1953|217|2814|Sunday|1953Q4|N|N|N|2434713|2435046|2434353|2434626|N|N|N|N|N| +2434719|AAAAAAAAPJGCFCAA|1953-12-07|647|2814|217|1953|1|12|7|4|1953|217|2814|Monday|1953Q4|N|N|N|2434713|2435046|2434354|2434627|N|N|N|N|N| +2434720|AAAAAAAAAKGCFCAA|1953-12-08|647|2815|217|1953|2|12|8|4|1953|217|2815|Tuesday|1953Q4|N|N|N|2434713|2435046|2434355|2434628|N|N|N|N|N| +2434721|AAAAAAAABKGCFCAA|1953-12-09|647|2815|217|1953|3|12|9|4|1953|217|2815|Wednesday|1953Q4|N|N|N|2434713|2435046|2434356|2434629|N|N|N|N|N| +2434722|AAAAAAAACKGCFCAA|1953-12-10|647|2815|217|1953|4|12|10|4|1953|217|2815|Thursday|1953Q4|N|N|N|2434713|2435046|2434357|2434630|N|N|N|N|N| +2434723|AAAAAAAADKGCFCAA|1953-12-11|647|2815|217|1953|5|12|11|4|1953|217|2815|Friday|1953Q4|N|Y|N|2434713|2435046|2434358|2434631|N|N|N|N|N| +2434724|AAAAAAAAEKGCFCAA|1953-12-12|647|2815|217|1953|6|12|12|4|1953|217|2815|Saturday|1953Q4|N|Y|N|2434713|2435046|2434359|2434632|N|N|N|N|N| +2434725|AAAAAAAAFKGCFCAA|1953-12-13|647|2815|217|1953|0|12|13|4|1953|217|2815|Sunday|1953Q4|N|N|N|2434713|2435046|2434360|2434633|N|N|N|N|N| +2434726|AAAAAAAAGKGCFCAA|1953-12-14|647|2815|217|1953|1|12|14|4|1953|217|2815|Monday|1953Q4|N|N|N|2434713|2435046|2434361|2434634|N|N|N|N|N| +2434727|AAAAAAAAHKGCFCAA|1953-12-15|647|2816|217|1953|2|12|15|4|1953|217|2816|Tuesday|1953Q4|N|N|N|2434713|2435046|2434362|2434635|N|N|N|N|N| +2434728|AAAAAAAAIKGCFCAA|1953-12-16|647|2816|217|1953|3|12|16|4|1953|217|2816|Wednesday|1953Q4|N|N|N|2434713|2435046|2434363|2434636|N|N|N|N|N| +2434729|AAAAAAAAJKGCFCAA|1953-12-17|647|2816|217|1953|4|12|17|4|1953|217|2816|Thursday|1953Q4|N|N|N|2434713|2435046|2434364|2434637|N|N|N|N|N| +2434730|AAAAAAAAKKGCFCAA|1953-12-18|647|2816|217|1953|5|12|18|4|1953|217|2816|Friday|1953Q4|N|Y|N|2434713|2435046|2434365|2434638|N|N|N|N|N| +2434731|AAAAAAAALKGCFCAA|1953-12-19|647|2816|217|1953|6|12|19|4|1953|217|2816|Saturday|1953Q4|N|Y|N|2434713|2435046|2434366|2434639|N|N|N|N|N| +2434732|AAAAAAAAMKGCFCAA|1953-12-20|647|2816|217|1953|0|12|20|4|1953|217|2816|Sunday|1953Q4|N|N|N|2434713|2435046|2434367|2434640|N|N|N|N|N| +2434733|AAAAAAAANKGCFCAA|1953-12-21|647|2816|217|1953|1|12|21|4|1953|217|2816|Monday|1953Q4|N|N|N|2434713|2435046|2434368|2434641|N|N|N|N|N| +2434734|AAAAAAAAOKGCFCAA|1953-12-22|647|2817|217|1953|2|12|22|4|1953|217|2817|Tuesday|1953Q4|N|N|N|2434713|2435046|2434369|2434642|N|N|N|N|N| +2434735|AAAAAAAAPKGCFCAA|1953-12-23|647|2817|217|1953|3|12|23|4|1953|217|2817|Wednesday|1953Q4|N|N|N|2434713|2435046|2434370|2434643|N|N|N|N|N| +2434736|AAAAAAAAALGCFCAA|1953-12-24|647|2817|217|1953|4|12|24|4|1953|217|2817|Thursday|1953Q4|N|N|N|2434713|2435046|2434371|2434644|N|N|N|N|N| +2434737|AAAAAAAABLGCFCAA|1953-12-25|647|2817|217|1953|5|12|25|4|1953|217|2817|Friday|1953Q4|N|Y|N|2434713|2435046|2434372|2434645|N|N|N|N|N| +2434738|AAAAAAAACLGCFCAA|1953-12-26|647|2817|217|1953|6|12|26|4|1953|217|2817|Saturday|1953Q4|Y|Y|N|2434713|2435046|2434373|2434646|N|N|N|N|N| +2434739|AAAAAAAADLGCFCAA|1953-12-27|647|2817|217|1953|0|12|27|4|1953|217|2817|Sunday|1953Q4|N|N|Y|2434713|2435046|2434374|2434647|N|N|N|N|N| +2434740|AAAAAAAAELGCFCAA|1953-12-28|647|2817|217|1953|1|12|28|4|1953|217|2817|Monday|1953Q4|N|N|N|2434713|2435046|2434375|2434648|N|N|N|N|N| +2434741|AAAAAAAAFLGCFCAA|1953-12-29|647|2818|217|1953|2|12|29|4|1953|217|2818|Tuesday|1953Q4|N|N|N|2434713|2435046|2434376|2434649|N|N|N|N|N| +2434742|AAAAAAAAGLGCFCAA|1953-12-30|647|2818|217|1953|3|12|30|4|1953|217|2818|Wednesday|1953Q4|N|N|N|2434713|2435046|2434377|2434650|N|N|N|N|N| +2434743|AAAAAAAAHLGCFCAA|1953-12-31|647|2818|217|1953|4|12|31|4|1953|217|2818|Thursday|1953Q4|N|N|N|2434713|2435046|2434378|2434651|N|N|N|N|N| +2434744|AAAAAAAAILGCFCAA|1954-01-01|648|2818|217|1954|5|1|1|1|1954|217|2818|Friday|1954Q1|Y|Y|N|2434744|2434743|2434379|2434652|N|N|N|N|N| +2434745|AAAAAAAAJLGCFCAA|1954-01-02|648|2818|217|1954|6|1|2|1|1954|217|2818|Saturday|1954Q1|N|Y|Y|2434744|2434743|2434380|2434653|N|N|N|N|N| +2434746|AAAAAAAAKLGCFCAA|1954-01-03|648|2818|217|1954|0|1|3|1|1954|217|2818|Sunday|1954Q1|N|N|N|2434744|2434743|2434381|2434654|N|N|N|N|N| +2434747|AAAAAAAALLGCFCAA|1954-01-04|648|2818|217|1954|1|1|4|1|1954|217|2818|Monday|1954Q1|N|N|N|2434744|2434743|2434382|2434655|N|N|N|N|N| +2434748|AAAAAAAAMLGCFCAA|1954-01-05|648|2819|217|1954|2|1|5|1|1954|217|2819|Tuesday|1954Q1|N|N|N|2434744|2434743|2434383|2434656|N|N|N|N|N| +2434749|AAAAAAAANLGCFCAA|1954-01-06|648|2819|217|1954|3|1|6|1|1954|217|2819|Wednesday|1954Q1|N|N|N|2434744|2434743|2434384|2434657|N|N|N|N|N| +2434750|AAAAAAAAOLGCFCAA|1954-01-07|648|2819|217|1954|4|1|7|1|1954|217|2819|Thursday|1954Q1|N|N|N|2434744|2434743|2434385|2434658|N|N|N|N|N| +2434751|AAAAAAAAPLGCFCAA|1954-01-08|648|2819|217|1954|5|1|8|1|1954|217|2819|Friday|1954Q1|N|Y|N|2434744|2434743|2434386|2434659|N|N|N|N|N| +2434752|AAAAAAAAAMGCFCAA|1954-01-09|648|2819|217|1954|6|1|9|1|1954|217|2819|Saturday|1954Q1|N|Y|N|2434744|2434743|2434387|2434660|N|N|N|N|N| +2434753|AAAAAAAABMGCFCAA|1954-01-10|648|2819|217|1954|0|1|10|1|1954|217|2819|Sunday|1954Q1|N|N|N|2434744|2434743|2434388|2434661|N|N|N|N|N| +2434754|AAAAAAAACMGCFCAA|1954-01-11|648|2819|217|1954|1|1|11|1|1954|217|2819|Monday|1954Q1|N|N|N|2434744|2434743|2434389|2434662|N|N|N|N|N| +2434755|AAAAAAAADMGCFCAA|1954-01-12|648|2820|217|1954|2|1|12|1|1954|217|2820|Tuesday|1954Q1|N|N|N|2434744|2434743|2434390|2434663|N|N|N|N|N| +2434756|AAAAAAAAEMGCFCAA|1954-01-13|648|2820|217|1954|3|1|13|1|1954|217|2820|Wednesday|1954Q1|N|N|N|2434744|2434743|2434391|2434664|N|N|N|N|N| +2434757|AAAAAAAAFMGCFCAA|1954-01-14|648|2820|217|1954|4|1|14|1|1954|217|2820|Thursday|1954Q1|N|N|N|2434744|2434743|2434392|2434665|N|N|N|N|N| +2434758|AAAAAAAAGMGCFCAA|1954-01-15|648|2820|217|1954|5|1|15|1|1954|217|2820|Friday|1954Q1|N|Y|N|2434744|2434743|2434393|2434666|N|N|N|N|N| +2434759|AAAAAAAAHMGCFCAA|1954-01-16|648|2820|217|1954|6|1|16|1|1954|217|2820|Saturday|1954Q1|N|Y|N|2434744|2434743|2434394|2434667|N|N|N|N|N| +2434760|AAAAAAAAIMGCFCAA|1954-01-17|648|2820|217|1954|0|1|17|1|1954|217|2820|Sunday|1954Q1|N|N|N|2434744|2434743|2434395|2434668|N|N|N|N|N| +2434761|AAAAAAAAJMGCFCAA|1954-01-18|648|2820|217|1954|1|1|18|1|1954|217|2820|Monday|1954Q1|N|N|N|2434744|2434743|2434396|2434669|N|N|N|N|N| +2434762|AAAAAAAAKMGCFCAA|1954-01-19|648|2821|217|1954|2|1|19|1|1954|217|2821|Tuesday|1954Q1|N|N|N|2434744|2434743|2434397|2434670|N|N|N|N|N| +2434763|AAAAAAAALMGCFCAA|1954-01-20|648|2821|217|1954|3|1|20|1|1954|217|2821|Wednesday|1954Q1|N|N|N|2434744|2434743|2434398|2434671|N|N|N|N|N| +2434764|AAAAAAAAMMGCFCAA|1954-01-21|648|2821|217|1954|4|1|21|1|1954|217|2821|Thursday|1954Q1|N|N|N|2434744|2434743|2434399|2434672|N|N|N|N|N| +2434765|AAAAAAAANMGCFCAA|1954-01-22|648|2821|217|1954|5|1|22|1|1954|217|2821|Friday|1954Q1|N|Y|N|2434744|2434743|2434400|2434673|N|N|N|N|N| +2434766|AAAAAAAAOMGCFCAA|1954-01-23|648|2821|217|1954|6|1|23|1|1954|217|2821|Saturday|1954Q1|N|Y|N|2434744|2434743|2434401|2434674|N|N|N|N|N| +2434767|AAAAAAAAPMGCFCAA|1954-01-24|648|2821|217|1954|0|1|24|1|1954|217|2821|Sunday|1954Q1|N|N|N|2434744|2434743|2434402|2434675|N|N|N|N|N| +2434768|AAAAAAAAANGCFCAA|1954-01-25|648|2821|217|1954|1|1|25|1|1954|217|2821|Monday|1954Q1|N|N|N|2434744|2434743|2434403|2434676|N|N|N|N|N| +2434769|AAAAAAAABNGCFCAA|1954-01-26|648|2822|217|1954|2|1|26|1|1954|217|2822|Tuesday|1954Q1|N|N|N|2434744|2434743|2434404|2434677|N|N|N|N|N| +2434770|AAAAAAAACNGCFCAA|1954-01-27|648|2822|217|1954|3|1|27|1|1954|217|2822|Wednesday|1954Q1|N|N|N|2434744|2434743|2434405|2434678|N|N|N|N|N| +2434771|AAAAAAAADNGCFCAA|1954-01-28|648|2822|217|1954|4|1|28|1|1954|217|2822|Thursday|1954Q1|N|N|N|2434744|2434743|2434406|2434679|N|N|N|N|N| +2434772|AAAAAAAAENGCFCAA|1954-01-29|648|2822|217|1954|5|1|29|1|1954|217|2822|Friday|1954Q1|N|Y|N|2434744|2434743|2434407|2434680|N|N|N|N|N| +2434773|AAAAAAAAFNGCFCAA|1954-01-30|648|2822|217|1954|6|1|30|1|1954|217|2822|Saturday|1954Q1|N|Y|N|2434744|2434743|2434408|2434681|N|N|N|N|N| +2434774|AAAAAAAAGNGCFCAA|1954-01-31|648|2822|217|1954|0|1|31|1|1954|217|2822|Sunday|1954Q1|N|N|N|2434744|2434743|2434409|2434682|N|N|N|N|N| +2434775|AAAAAAAAHNGCFCAA|1954-02-01|649|2822|217|1954|1|2|1|1|1954|217|2822|Monday|1954Q1|N|N|N|2434775|2434805|2434410|2434683|N|N|N|N|N| +2434776|AAAAAAAAINGCFCAA|1954-02-02|649|2823|217|1954|2|2|2|1|1954|217|2823|Tuesday|1954Q1|N|N|N|2434775|2434805|2434411|2434684|N|N|N|N|N| +2434777|AAAAAAAAJNGCFCAA|1954-02-03|649|2823|217|1954|3|2|3|1|1954|217|2823|Wednesday|1954Q1|N|N|N|2434775|2434805|2434412|2434685|N|N|N|N|N| +2434778|AAAAAAAAKNGCFCAA|1954-02-04|649|2823|217|1954|4|2|4|1|1954|217|2823|Thursday|1954Q1|N|N|N|2434775|2434805|2434413|2434686|N|N|N|N|N| +2434779|AAAAAAAALNGCFCAA|1954-02-05|649|2823|217|1954|5|2|5|1|1954|217|2823|Friday|1954Q1|N|Y|N|2434775|2434805|2434414|2434687|N|N|N|N|N| +2434780|AAAAAAAAMNGCFCAA|1954-02-06|649|2823|217|1954|6|2|6|1|1954|217|2823|Saturday|1954Q1|N|Y|N|2434775|2434805|2434415|2434688|N|N|N|N|N| +2434781|AAAAAAAANNGCFCAA|1954-02-07|649|2823|217|1954|0|2|7|1|1954|217|2823|Sunday|1954Q1|N|N|N|2434775|2434805|2434416|2434689|N|N|N|N|N| +2434782|AAAAAAAAONGCFCAA|1954-02-08|649|2823|217|1954|1|2|8|1|1954|217|2823|Monday|1954Q1|N|N|N|2434775|2434805|2434417|2434690|N|N|N|N|N| +2434783|AAAAAAAAPNGCFCAA|1954-02-09|649|2824|217|1954|2|2|9|1|1954|217|2824|Tuesday|1954Q1|N|N|N|2434775|2434805|2434418|2434691|N|N|N|N|N| +2434784|AAAAAAAAAOGCFCAA|1954-02-10|649|2824|217|1954|3|2|10|1|1954|217|2824|Wednesday|1954Q1|N|N|N|2434775|2434805|2434419|2434692|N|N|N|N|N| +2434785|AAAAAAAABOGCFCAA|1954-02-11|649|2824|217|1954|4|2|11|1|1954|217|2824|Thursday|1954Q1|N|N|N|2434775|2434805|2434420|2434693|N|N|N|N|N| +2434786|AAAAAAAACOGCFCAA|1954-02-12|649|2824|217|1954|5|2|12|1|1954|217|2824|Friday|1954Q1|N|Y|N|2434775|2434805|2434421|2434694|N|N|N|N|N| +2434787|AAAAAAAADOGCFCAA|1954-02-13|649|2824|217|1954|6|2|13|1|1954|217|2824|Saturday|1954Q1|N|Y|N|2434775|2434805|2434422|2434695|N|N|N|N|N| +2434788|AAAAAAAAEOGCFCAA|1954-02-14|649|2824|217|1954|0|2|14|1|1954|217|2824|Sunday|1954Q1|N|N|N|2434775|2434805|2434423|2434696|N|N|N|N|N| +2434789|AAAAAAAAFOGCFCAA|1954-02-15|649|2824|217|1954|1|2|15|1|1954|217|2824|Monday|1954Q1|N|N|N|2434775|2434805|2434424|2434697|N|N|N|N|N| +2434790|AAAAAAAAGOGCFCAA|1954-02-16|649|2825|217|1954|2|2|16|1|1954|217|2825|Tuesday|1954Q1|N|N|N|2434775|2434805|2434425|2434698|N|N|N|N|N| +2434791|AAAAAAAAHOGCFCAA|1954-02-17|649|2825|217|1954|3|2|17|1|1954|217|2825|Wednesday|1954Q1|N|N|N|2434775|2434805|2434426|2434699|N|N|N|N|N| +2434792|AAAAAAAAIOGCFCAA|1954-02-18|649|2825|217|1954|4|2|18|1|1954|217|2825|Thursday|1954Q1|N|N|N|2434775|2434805|2434427|2434700|N|N|N|N|N| +2434793|AAAAAAAAJOGCFCAA|1954-02-19|649|2825|217|1954|5|2|19|1|1954|217|2825|Friday|1954Q1|N|Y|N|2434775|2434805|2434428|2434701|N|N|N|N|N| +2434794|AAAAAAAAKOGCFCAA|1954-02-20|649|2825|217|1954|6|2|20|1|1954|217|2825|Saturday|1954Q1|N|Y|N|2434775|2434805|2434429|2434702|N|N|N|N|N| +2434795|AAAAAAAALOGCFCAA|1954-02-21|649|2825|217|1954|0|2|21|1|1954|217|2825|Sunday|1954Q1|N|N|N|2434775|2434805|2434430|2434703|N|N|N|N|N| +2434796|AAAAAAAAMOGCFCAA|1954-02-22|649|2825|217|1954|1|2|22|1|1954|217|2825|Monday|1954Q1|N|N|N|2434775|2434805|2434431|2434704|N|N|N|N|N| +2434797|AAAAAAAANOGCFCAA|1954-02-23|649|2826|217|1954|2|2|23|1|1954|217|2826|Tuesday|1954Q1|N|N|N|2434775|2434805|2434432|2434705|N|N|N|N|N| +2434798|AAAAAAAAOOGCFCAA|1954-02-24|649|2826|217|1954|3|2|24|1|1954|217|2826|Wednesday|1954Q1|N|N|N|2434775|2434805|2434433|2434706|N|N|N|N|N| +2434799|AAAAAAAAPOGCFCAA|1954-02-25|649|2826|217|1954|4|2|25|1|1954|217|2826|Thursday|1954Q1|N|N|N|2434775|2434805|2434434|2434707|N|N|N|N|N| +2434800|AAAAAAAAAPGCFCAA|1954-02-26|649|2826|217|1954|5|2|26|1|1954|217|2826|Friday|1954Q1|N|Y|N|2434775|2434805|2434435|2434708|N|N|N|N|N| +2434801|AAAAAAAABPGCFCAA|1954-02-27|649|2826|217|1954|6|2|27|1|1954|217|2826|Saturday|1954Q1|N|Y|N|2434775|2434805|2434436|2434709|N|N|N|N|N| +2434802|AAAAAAAACPGCFCAA|1954-02-28|649|2826|217|1954|0|2|28|1|1954|217|2826|Sunday|1954Q1|N|N|N|2434775|2434805|2434437|2434710|N|N|N|N|N| +2434803|AAAAAAAADPGCFCAA|1954-03-01|650|2826|218|1954|1|3|1|1|1954|218|2826|Monday|1954Q1|N|N|N|2434803|2434861|2434438|2434711|N|N|N|N|N| +2434804|AAAAAAAAEPGCFCAA|1954-03-02|650|2827|218|1954|2|3|2|1|1954|218|2827|Tuesday|1954Q1|N|N|N|2434803|2434861|2434439|2434712|N|N|N|N|N| +2434805|AAAAAAAAFPGCFCAA|1954-03-03|650|2827|218|1954|3|3|3|1|1954|218|2827|Wednesday|1954Q1|N|N|N|2434803|2434861|2434440|2434713|N|N|N|N|N| +2434806|AAAAAAAAGPGCFCAA|1954-03-04|650|2827|218|1954|4|3|4|1|1954|218|2827|Thursday|1954Q1|N|N|N|2434803|2434861|2434441|2434714|N|N|N|N|N| +2434807|AAAAAAAAHPGCFCAA|1954-03-05|650|2827|218|1954|5|3|5|1|1954|218|2827|Friday|1954Q1|N|Y|N|2434803|2434861|2434442|2434715|N|N|N|N|N| +2434808|AAAAAAAAIPGCFCAA|1954-03-06|650|2827|218|1954|6|3|6|1|1954|218|2827|Saturday|1954Q1|N|Y|N|2434803|2434861|2434443|2434716|N|N|N|N|N| +2434809|AAAAAAAAJPGCFCAA|1954-03-07|650|2827|218|1954|0|3|7|1|1954|218|2827|Sunday|1954Q1|N|N|N|2434803|2434861|2434444|2434717|N|N|N|N|N| +2434810|AAAAAAAAKPGCFCAA|1954-03-08|650|2827|218|1954|1|3|8|1|1954|218|2827|Monday|1954Q1|N|N|N|2434803|2434861|2434445|2434718|N|N|N|N|N| +2434811|AAAAAAAALPGCFCAA|1954-03-09|650|2828|218|1954|2|3|9|1|1954|218|2828|Tuesday|1954Q1|N|N|N|2434803|2434861|2434446|2434719|N|N|N|N|N| +2434812|AAAAAAAAMPGCFCAA|1954-03-10|650|2828|218|1954|3|3|10|1|1954|218|2828|Wednesday|1954Q1|N|N|N|2434803|2434861|2434447|2434720|N|N|N|N|N| +2434813|AAAAAAAANPGCFCAA|1954-03-11|650|2828|218|1954|4|3|11|1|1954|218|2828|Thursday|1954Q1|N|N|N|2434803|2434861|2434448|2434721|N|N|N|N|N| +2434814|AAAAAAAAOPGCFCAA|1954-03-12|650|2828|218|1954|5|3|12|1|1954|218|2828|Friday|1954Q1|N|Y|N|2434803|2434861|2434449|2434722|N|N|N|N|N| +2434815|AAAAAAAAPPGCFCAA|1954-03-13|650|2828|218|1954|6|3|13|1|1954|218|2828|Saturday|1954Q1|N|Y|N|2434803|2434861|2434450|2434723|N|N|N|N|N| +2434816|AAAAAAAAAAHCFCAA|1954-03-14|650|2828|218|1954|0|3|14|1|1954|218|2828|Sunday|1954Q1|N|N|N|2434803|2434861|2434451|2434724|N|N|N|N|N| +2434817|AAAAAAAABAHCFCAA|1954-03-15|650|2828|218|1954|1|3|15|1|1954|218|2828|Monday|1954Q1|N|N|N|2434803|2434861|2434452|2434725|N|N|N|N|N| +2434818|AAAAAAAACAHCFCAA|1954-03-16|650|2829|218|1954|2|3|16|1|1954|218|2829|Tuesday|1954Q1|N|N|N|2434803|2434861|2434453|2434726|N|N|N|N|N| +2434819|AAAAAAAADAHCFCAA|1954-03-17|650|2829|218|1954|3|3|17|1|1954|218|2829|Wednesday|1954Q1|N|N|N|2434803|2434861|2434454|2434727|N|N|N|N|N| +2434820|AAAAAAAAEAHCFCAA|1954-03-18|650|2829|218|1954|4|3|18|1|1954|218|2829|Thursday|1954Q1|N|N|N|2434803|2434861|2434455|2434728|N|N|N|N|N| +2434821|AAAAAAAAFAHCFCAA|1954-03-19|650|2829|218|1954|5|3|19|1|1954|218|2829|Friday|1954Q1|N|Y|N|2434803|2434861|2434456|2434729|N|N|N|N|N| +2434822|AAAAAAAAGAHCFCAA|1954-03-20|650|2829|218|1954|6|3|20|1|1954|218|2829|Saturday|1954Q1|N|Y|N|2434803|2434861|2434457|2434730|N|N|N|N|N| +2434823|AAAAAAAAHAHCFCAA|1954-03-21|650|2829|218|1954|0|3|21|1|1954|218|2829|Sunday|1954Q1|N|N|N|2434803|2434861|2434458|2434731|N|N|N|N|N| +2434824|AAAAAAAAIAHCFCAA|1954-03-22|650|2829|218|1954|1|3|22|1|1954|218|2829|Monday|1954Q1|N|N|N|2434803|2434861|2434459|2434732|N|N|N|N|N| +2434825|AAAAAAAAJAHCFCAA|1954-03-23|650|2830|218|1954|2|3|23|1|1954|218|2830|Tuesday|1954Q1|N|N|N|2434803|2434861|2434460|2434733|N|N|N|N|N| +2434826|AAAAAAAAKAHCFCAA|1954-03-24|650|2830|218|1954|3|3|24|1|1954|218|2830|Wednesday|1954Q1|N|N|N|2434803|2434861|2434461|2434734|N|N|N|N|N| +2434827|AAAAAAAALAHCFCAA|1954-03-25|650|2830|218|1954|4|3|25|1|1954|218|2830|Thursday|1954Q1|N|N|N|2434803|2434861|2434462|2434735|N|N|N|N|N| +2434828|AAAAAAAAMAHCFCAA|1954-03-26|650|2830|218|1954|5|3|26|1|1954|218|2830|Friday|1954Q1|N|Y|N|2434803|2434861|2434463|2434736|N|N|N|N|N| +2434829|AAAAAAAANAHCFCAA|1954-03-27|650|2830|218|1954|6|3|27|1|1954|218|2830|Saturday|1954Q1|N|Y|N|2434803|2434861|2434464|2434737|N|N|N|N|N| +2434830|AAAAAAAAOAHCFCAA|1954-03-28|650|2830|218|1954|0|3|28|1|1954|218|2830|Sunday|1954Q1|N|N|N|2434803|2434861|2434465|2434738|N|N|N|N|N| +2434831|AAAAAAAAPAHCFCAA|1954-03-29|650|2830|218|1954|1|3|29|1|1954|218|2830|Monday|1954Q1|N|N|N|2434803|2434861|2434466|2434739|N|N|N|N|N| +2434832|AAAAAAAAABHCFCAA|1954-03-30|650|2831|218|1954|2|3|30|1|1954|218|2831|Tuesday|1954Q1|N|N|N|2434803|2434861|2434467|2434740|N|N|N|N|N| +2434833|AAAAAAAABBHCFCAA|1954-03-31|650|2831|218|1954|3|3|31|1|1954|218|2831|Wednesday|1954Q1|N|N|N|2434803|2434861|2434468|2434741|N|N|N|N|N| +2434834|AAAAAAAACBHCFCAA|1954-04-01|651|2831|218|1954|4|4|1|1|1954|218|2831|Thursday|1954Q1|N|N|N|2434834|2434923|2434469|2434744|N|N|N|N|N| +2434835|AAAAAAAADBHCFCAA|1954-04-02|651|2831|218|1954|5|4|2|2|1954|218|2831|Friday|1954Q2|N|Y|N|2434834|2434923|2434470|2434745|N|N|N|N|N| +2434836|AAAAAAAAEBHCFCAA|1954-04-03|651|2831|218|1954|6|4|3|2|1954|218|2831|Saturday|1954Q2|N|Y|N|2434834|2434923|2434471|2434746|N|N|N|N|N| +2434837|AAAAAAAAFBHCFCAA|1954-04-04|651|2831|218|1954|0|4|4|2|1954|218|2831|Sunday|1954Q2|N|N|N|2434834|2434923|2434472|2434747|N|N|N|N|N| +2434838|AAAAAAAAGBHCFCAA|1954-04-05|651|2831|218|1954|1|4|5|2|1954|218|2831|Monday|1954Q2|N|N|N|2434834|2434923|2434473|2434748|N|N|N|N|N| +2434839|AAAAAAAAHBHCFCAA|1954-04-06|651|2832|218|1954|2|4|6|2|1954|218|2832|Tuesday|1954Q2|N|N|N|2434834|2434923|2434474|2434749|N|N|N|N|N| +2434840|AAAAAAAAIBHCFCAA|1954-04-07|651|2832|218|1954|3|4|7|2|1954|218|2832|Wednesday|1954Q2|N|N|N|2434834|2434923|2434475|2434750|N|N|N|N|N| +2434841|AAAAAAAAJBHCFCAA|1954-04-08|651|2832|218|1954|4|4|8|2|1954|218|2832|Thursday|1954Q2|N|N|N|2434834|2434923|2434476|2434751|N|N|N|N|N| +2434842|AAAAAAAAKBHCFCAA|1954-04-09|651|2832|218|1954|5|4|9|2|1954|218|2832|Friday|1954Q2|N|Y|N|2434834|2434923|2434477|2434752|N|N|N|N|N| +2434843|AAAAAAAALBHCFCAA|1954-04-10|651|2832|218|1954|6|4|10|2|1954|218|2832|Saturday|1954Q2|N|Y|N|2434834|2434923|2434478|2434753|N|N|N|N|N| +2434844|AAAAAAAAMBHCFCAA|1954-04-11|651|2832|218|1954|0|4|11|2|1954|218|2832|Sunday|1954Q2|N|N|N|2434834|2434923|2434479|2434754|N|N|N|N|N| +2434845|AAAAAAAANBHCFCAA|1954-04-12|651|2832|218|1954|1|4|12|2|1954|218|2832|Monday|1954Q2|N|N|N|2434834|2434923|2434480|2434755|N|N|N|N|N| +2434846|AAAAAAAAOBHCFCAA|1954-04-13|651|2833|218|1954|2|4|13|2|1954|218|2833|Tuesday|1954Q2|N|N|N|2434834|2434923|2434481|2434756|N|N|N|N|N| +2434847|AAAAAAAAPBHCFCAA|1954-04-14|651|2833|218|1954|3|4|14|2|1954|218|2833|Wednesday|1954Q2|N|N|N|2434834|2434923|2434482|2434757|N|N|N|N|N| +2434848|AAAAAAAAACHCFCAA|1954-04-15|651|2833|218|1954|4|4|15|2|1954|218|2833|Thursday|1954Q2|N|N|N|2434834|2434923|2434483|2434758|N|N|N|N|N| +2434849|AAAAAAAABCHCFCAA|1954-04-16|651|2833|218|1954|5|4|16|2|1954|218|2833|Friday|1954Q2|N|Y|N|2434834|2434923|2434484|2434759|N|N|N|N|N| +2434850|AAAAAAAACCHCFCAA|1954-04-17|651|2833|218|1954|6|4|17|2|1954|218|2833|Saturday|1954Q2|N|Y|N|2434834|2434923|2434485|2434760|N|N|N|N|N| +2434851|AAAAAAAADCHCFCAA|1954-04-18|651|2833|218|1954|0|4|18|2|1954|218|2833|Sunday|1954Q2|N|N|N|2434834|2434923|2434486|2434761|N|N|N|N|N| +2434852|AAAAAAAAECHCFCAA|1954-04-19|651|2833|218|1954|1|4|19|2|1954|218|2833|Monday|1954Q2|N|N|N|2434834|2434923|2434487|2434762|N|N|N|N|N| +2434853|AAAAAAAAFCHCFCAA|1954-04-20|651|2834|218|1954|2|4|20|2|1954|218|2834|Tuesday|1954Q2|N|N|N|2434834|2434923|2434488|2434763|N|N|N|N|N| +2434854|AAAAAAAAGCHCFCAA|1954-04-21|651|2834|218|1954|3|4|21|2|1954|218|2834|Wednesday|1954Q2|N|N|N|2434834|2434923|2434489|2434764|N|N|N|N|N| +2434855|AAAAAAAAHCHCFCAA|1954-04-22|651|2834|218|1954|4|4|22|2|1954|218|2834|Thursday|1954Q2|N|N|N|2434834|2434923|2434490|2434765|N|N|N|N|N| +2434856|AAAAAAAAICHCFCAA|1954-04-23|651|2834|218|1954|5|4|23|2|1954|218|2834|Friday|1954Q2|N|Y|N|2434834|2434923|2434491|2434766|N|N|N|N|N| +2434857|AAAAAAAAJCHCFCAA|1954-04-24|651|2834|218|1954|6|4|24|2|1954|218|2834|Saturday|1954Q2|N|Y|N|2434834|2434923|2434492|2434767|N|N|N|N|N| +2434858|AAAAAAAAKCHCFCAA|1954-04-25|651|2834|218|1954|0|4|25|2|1954|218|2834|Sunday|1954Q2|N|N|N|2434834|2434923|2434493|2434768|N|N|N|N|N| +2434859|AAAAAAAALCHCFCAA|1954-04-26|651|2834|218|1954|1|4|26|2|1954|218|2834|Monday|1954Q2|N|N|N|2434834|2434923|2434494|2434769|N|N|N|N|N| +2434860|AAAAAAAAMCHCFCAA|1954-04-27|651|2835|218|1954|2|4|27|2|1954|218|2835|Tuesday|1954Q2|N|N|N|2434834|2434923|2434495|2434770|N|N|N|N|N| +2434861|AAAAAAAANCHCFCAA|1954-04-28|651|2835|218|1954|3|4|28|2|1954|218|2835|Wednesday|1954Q2|N|N|N|2434834|2434923|2434496|2434771|N|N|N|N|N| +2434862|AAAAAAAAOCHCFCAA|1954-04-29|651|2835|218|1954|4|4|29|2|1954|218|2835|Thursday|1954Q2|N|N|N|2434834|2434923|2434497|2434772|N|N|N|N|N| +2434863|AAAAAAAAPCHCFCAA|1954-04-30|651|2835|218|1954|5|4|30|2|1954|218|2835|Friday|1954Q2|N|Y|N|2434834|2434923|2434498|2434773|N|N|N|N|N| +2434864|AAAAAAAAADHCFCAA|1954-05-01|652|2835|218|1954|6|5|1|2|1954|218|2835|Saturday|1954Q2|N|Y|N|2434864|2434983|2434499|2434774|N|N|N|N|N| +2434865|AAAAAAAABDHCFCAA|1954-05-02|652|2835|218|1954|0|5|2|2|1954|218|2835|Sunday|1954Q2|N|N|N|2434864|2434983|2434500|2434775|N|N|N|N|N| +2434866|AAAAAAAACDHCFCAA|1954-05-03|652|2835|218|1954|1|5|3|2|1954|218|2835|Monday|1954Q2|N|N|N|2434864|2434983|2434501|2434776|N|N|N|N|N| +2434867|AAAAAAAADDHCFCAA|1954-05-04|652|2836|218|1954|2|5|4|2|1954|218|2836|Tuesday|1954Q2|N|N|N|2434864|2434983|2434502|2434777|N|N|N|N|N| +2434868|AAAAAAAAEDHCFCAA|1954-05-05|652|2836|218|1954|3|5|5|2|1954|218|2836|Wednesday|1954Q2|N|N|N|2434864|2434983|2434503|2434778|N|N|N|N|N| +2434869|AAAAAAAAFDHCFCAA|1954-05-06|652|2836|218|1954|4|5|6|2|1954|218|2836|Thursday|1954Q2|N|N|N|2434864|2434983|2434504|2434779|N|N|N|N|N| +2434870|AAAAAAAAGDHCFCAA|1954-05-07|652|2836|218|1954|5|5|7|2|1954|218|2836|Friday|1954Q2|N|Y|N|2434864|2434983|2434505|2434780|N|N|N|N|N| +2434871|AAAAAAAAHDHCFCAA|1954-05-08|652|2836|218|1954|6|5|8|2|1954|218|2836|Saturday|1954Q2|N|Y|N|2434864|2434983|2434506|2434781|N|N|N|N|N| +2434872|AAAAAAAAIDHCFCAA|1954-05-09|652|2836|218|1954|0|5|9|2|1954|218|2836|Sunday|1954Q2|N|N|N|2434864|2434983|2434507|2434782|N|N|N|N|N| +2434873|AAAAAAAAJDHCFCAA|1954-05-10|652|2836|218|1954|1|5|10|2|1954|218|2836|Monday|1954Q2|N|N|N|2434864|2434983|2434508|2434783|N|N|N|N|N| +2434874|AAAAAAAAKDHCFCAA|1954-05-11|652|2837|218|1954|2|5|11|2|1954|218|2837|Tuesday|1954Q2|N|N|N|2434864|2434983|2434509|2434784|N|N|N|N|N| +2434875|AAAAAAAALDHCFCAA|1954-05-12|652|2837|218|1954|3|5|12|2|1954|218|2837|Wednesday|1954Q2|N|N|N|2434864|2434983|2434510|2434785|N|N|N|N|N| +2434876|AAAAAAAAMDHCFCAA|1954-05-13|652|2837|218|1954|4|5|13|2|1954|218|2837|Thursday|1954Q2|N|N|N|2434864|2434983|2434511|2434786|N|N|N|N|N| +2434877|AAAAAAAANDHCFCAA|1954-05-14|652|2837|218|1954|5|5|14|2|1954|218|2837|Friday|1954Q2|N|Y|N|2434864|2434983|2434512|2434787|N|N|N|N|N| +2434878|AAAAAAAAODHCFCAA|1954-05-15|652|2837|218|1954|6|5|15|2|1954|218|2837|Saturday|1954Q2|N|Y|N|2434864|2434983|2434513|2434788|N|N|N|N|N| +2434879|AAAAAAAAPDHCFCAA|1954-05-16|652|2837|218|1954|0|5|16|2|1954|218|2837|Sunday|1954Q2|N|N|N|2434864|2434983|2434514|2434789|N|N|N|N|N| +2434880|AAAAAAAAAEHCFCAA|1954-05-17|652|2837|218|1954|1|5|17|2|1954|218|2837|Monday|1954Q2|N|N|N|2434864|2434983|2434515|2434790|N|N|N|N|N| +2434881|AAAAAAAABEHCFCAA|1954-05-18|652|2838|218|1954|2|5|18|2|1954|218|2838|Tuesday|1954Q2|N|N|N|2434864|2434983|2434516|2434791|N|N|N|N|N| +2434882|AAAAAAAACEHCFCAA|1954-05-19|652|2838|218|1954|3|5|19|2|1954|218|2838|Wednesday|1954Q2|N|N|N|2434864|2434983|2434517|2434792|N|N|N|N|N| +2434883|AAAAAAAADEHCFCAA|1954-05-20|652|2838|218|1954|4|5|20|2|1954|218|2838|Thursday|1954Q2|N|N|N|2434864|2434983|2434518|2434793|N|N|N|N|N| +2434884|AAAAAAAAEEHCFCAA|1954-05-21|652|2838|218|1954|5|5|21|2|1954|218|2838|Friday|1954Q2|N|Y|N|2434864|2434983|2434519|2434794|N|N|N|N|N| +2434885|AAAAAAAAFEHCFCAA|1954-05-22|652|2838|218|1954|6|5|22|2|1954|218|2838|Saturday|1954Q2|N|Y|N|2434864|2434983|2434520|2434795|N|N|N|N|N| +2434886|AAAAAAAAGEHCFCAA|1954-05-23|652|2838|218|1954|0|5|23|2|1954|218|2838|Sunday|1954Q2|N|N|N|2434864|2434983|2434521|2434796|N|N|N|N|N| +2434887|AAAAAAAAHEHCFCAA|1954-05-24|652|2838|218|1954|1|5|24|2|1954|218|2838|Monday|1954Q2|N|N|N|2434864|2434983|2434522|2434797|N|N|N|N|N| +2434888|AAAAAAAAIEHCFCAA|1954-05-25|652|2839|218|1954|2|5|25|2|1954|218|2839|Tuesday|1954Q2|N|N|N|2434864|2434983|2434523|2434798|N|N|N|N|N| +2434889|AAAAAAAAJEHCFCAA|1954-05-26|652|2839|218|1954|3|5|26|2|1954|218|2839|Wednesday|1954Q2|N|N|N|2434864|2434983|2434524|2434799|N|N|N|N|N| +2434890|AAAAAAAAKEHCFCAA|1954-05-27|652|2839|218|1954|4|5|27|2|1954|218|2839|Thursday|1954Q2|N|N|N|2434864|2434983|2434525|2434800|N|N|N|N|N| +2434891|AAAAAAAALEHCFCAA|1954-05-28|652|2839|218|1954|5|5|28|2|1954|218|2839|Friday|1954Q2|N|Y|N|2434864|2434983|2434526|2434801|N|N|N|N|N| +2434892|AAAAAAAAMEHCFCAA|1954-05-29|652|2839|218|1954|6|5|29|2|1954|218|2839|Saturday|1954Q2|N|Y|N|2434864|2434983|2434527|2434802|N|N|N|N|N| +2434893|AAAAAAAANEHCFCAA|1954-05-30|652|2839|218|1954|0|5|30|2|1954|218|2839|Sunday|1954Q2|N|N|N|2434864|2434983|2434528|2434803|N|N|N|N|N| +2434894|AAAAAAAAOEHCFCAA|1954-05-31|652|2839|218|1954|1|5|31|2|1954|218|2839|Monday|1954Q2|N|N|N|2434864|2434983|2434529|2434804|N|N|N|N|N| +2434895|AAAAAAAAPEHCFCAA|1954-06-01|653|2840|219|1954|2|6|1|2|1954|219|2840|Tuesday|1954Q2|N|N|N|2434895|2435045|2434530|2434805|N|N|N|N|N| +2434896|AAAAAAAAAFHCFCAA|1954-06-02|653|2840|219|1954|3|6|2|2|1954|219|2840|Wednesday|1954Q2|N|N|N|2434895|2435045|2434531|2434806|N|N|N|N|N| +2434897|AAAAAAAABFHCFCAA|1954-06-03|653|2840|219|1954|4|6|3|2|1954|219|2840|Thursday|1954Q2|N|N|N|2434895|2435045|2434532|2434807|N|N|N|N|N| +2434898|AAAAAAAACFHCFCAA|1954-06-04|653|2840|219|1954|5|6|4|2|1954|219|2840|Friday|1954Q2|N|Y|N|2434895|2435045|2434533|2434808|N|N|N|N|N| +2434899|AAAAAAAADFHCFCAA|1954-06-05|653|2840|219|1954|6|6|5|2|1954|219|2840|Saturday|1954Q2|N|Y|N|2434895|2435045|2434534|2434809|N|N|N|N|N| +2434900|AAAAAAAAEFHCFCAA|1954-06-06|653|2840|219|1954|0|6|6|2|1954|219|2840|Sunday|1954Q2|N|N|N|2434895|2435045|2434535|2434810|N|N|N|N|N| +2434901|AAAAAAAAFFHCFCAA|1954-06-07|653|2840|219|1954|1|6|7|2|1954|219|2840|Monday|1954Q2|N|N|N|2434895|2435045|2434536|2434811|N|N|N|N|N| +2434902|AAAAAAAAGFHCFCAA|1954-06-08|653|2841|219|1954|2|6|8|2|1954|219|2841|Tuesday|1954Q2|N|N|N|2434895|2435045|2434537|2434812|N|N|N|N|N| +2434903|AAAAAAAAHFHCFCAA|1954-06-09|653|2841|219|1954|3|6|9|2|1954|219|2841|Wednesday|1954Q2|N|N|N|2434895|2435045|2434538|2434813|N|N|N|N|N| +2434904|AAAAAAAAIFHCFCAA|1954-06-10|653|2841|219|1954|4|6|10|2|1954|219|2841|Thursday|1954Q2|N|N|N|2434895|2435045|2434539|2434814|N|N|N|N|N| +2434905|AAAAAAAAJFHCFCAA|1954-06-11|653|2841|219|1954|5|6|11|2|1954|219|2841|Friday|1954Q2|N|Y|N|2434895|2435045|2434540|2434815|N|N|N|N|N| +2434906|AAAAAAAAKFHCFCAA|1954-06-12|653|2841|219|1954|6|6|12|2|1954|219|2841|Saturday|1954Q2|N|Y|N|2434895|2435045|2434541|2434816|N|N|N|N|N| +2434907|AAAAAAAALFHCFCAA|1954-06-13|653|2841|219|1954|0|6|13|2|1954|219|2841|Sunday|1954Q2|N|N|N|2434895|2435045|2434542|2434817|N|N|N|N|N| +2434908|AAAAAAAAMFHCFCAA|1954-06-14|653|2841|219|1954|1|6|14|2|1954|219|2841|Monday|1954Q2|N|N|N|2434895|2435045|2434543|2434818|N|N|N|N|N| +2434909|AAAAAAAANFHCFCAA|1954-06-15|653|2842|219|1954|2|6|15|2|1954|219|2842|Tuesday|1954Q2|N|N|N|2434895|2435045|2434544|2434819|N|N|N|N|N| +2434910|AAAAAAAAOFHCFCAA|1954-06-16|653|2842|219|1954|3|6|16|2|1954|219|2842|Wednesday|1954Q2|N|N|N|2434895|2435045|2434545|2434820|N|N|N|N|N| +2434911|AAAAAAAAPFHCFCAA|1954-06-17|653|2842|219|1954|4|6|17|2|1954|219|2842|Thursday|1954Q2|N|N|N|2434895|2435045|2434546|2434821|N|N|N|N|N| +2434912|AAAAAAAAAGHCFCAA|1954-06-18|653|2842|219|1954|5|6|18|2|1954|219|2842|Friday|1954Q2|N|Y|N|2434895|2435045|2434547|2434822|N|N|N|N|N| +2434913|AAAAAAAABGHCFCAA|1954-06-19|653|2842|219|1954|6|6|19|2|1954|219|2842|Saturday|1954Q2|N|Y|N|2434895|2435045|2434548|2434823|N|N|N|N|N| +2434914|AAAAAAAACGHCFCAA|1954-06-20|653|2842|219|1954|0|6|20|2|1954|219|2842|Sunday|1954Q2|N|N|N|2434895|2435045|2434549|2434824|N|N|N|N|N| +2434915|AAAAAAAADGHCFCAA|1954-06-21|653|2842|219|1954|1|6|21|2|1954|219|2842|Monday|1954Q2|N|N|N|2434895|2435045|2434550|2434825|N|N|N|N|N| +2434916|AAAAAAAAEGHCFCAA|1954-06-22|653|2843|219|1954|2|6|22|2|1954|219|2843|Tuesday|1954Q2|N|N|N|2434895|2435045|2434551|2434826|N|N|N|N|N| +2434917|AAAAAAAAFGHCFCAA|1954-06-23|653|2843|219|1954|3|6|23|2|1954|219|2843|Wednesday|1954Q2|N|N|N|2434895|2435045|2434552|2434827|N|N|N|N|N| +2434918|AAAAAAAAGGHCFCAA|1954-06-24|653|2843|219|1954|4|6|24|2|1954|219|2843|Thursday|1954Q2|N|N|N|2434895|2435045|2434553|2434828|N|N|N|N|N| +2434919|AAAAAAAAHGHCFCAA|1954-06-25|653|2843|219|1954|5|6|25|2|1954|219|2843|Friday|1954Q2|N|Y|N|2434895|2435045|2434554|2434829|N|N|N|N|N| +2434920|AAAAAAAAIGHCFCAA|1954-06-26|653|2843|219|1954|6|6|26|2|1954|219|2843|Saturday|1954Q2|N|Y|N|2434895|2435045|2434555|2434830|N|N|N|N|N| +2434921|AAAAAAAAJGHCFCAA|1954-06-27|653|2843|219|1954|0|6|27|2|1954|219|2843|Sunday|1954Q2|N|N|N|2434895|2435045|2434556|2434831|N|N|N|N|N| +2434922|AAAAAAAAKGHCFCAA|1954-06-28|653|2843|219|1954|1|6|28|2|1954|219|2843|Monday|1954Q2|N|N|N|2434895|2435045|2434557|2434832|N|N|N|N|N| +2434923|AAAAAAAALGHCFCAA|1954-06-29|653|2844|219|1954|2|6|29|2|1954|219|2844|Tuesday|1954Q2|N|N|N|2434895|2435045|2434558|2434833|N|N|N|N|N| +2434924|AAAAAAAAMGHCFCAA|1954-06-30|653|2844|219|1954|3|6|30|2|1954|219|2844|Wednesday|1954Q2|N|N|N|2434895|2435045|2434559|2434834|N|N|N|N|N| +2434925|AAAAAAAANGHCFCAA|1954-07-01|654|2844|219|1954|4|7|1|2|1954|219|2844|Thursday|1954Q2|N|N|N|2434925|2435105|2434560|2434834|N|N|N|N|N| +2434926|AAAAAAAAOGHCFCAA|1954-07-02|654|2844|219|1954|5|7|2|3|1954|219|2844|Friday|1954Q3|N|Y|N|2434925|2435105|2434561|2434835|N|N|N|N|N| +2434927|AAAAAAAAPGHCFCAA|1954-07-03|654|2844|219|1954|6|7|3|3|1954|219|2844|Saturday|1954Q3|N|Y|N|2434925|2435105|2434562|2434836|N|N|N|N|N| +2434928|AAAAAAAAAHHCFCAA|1954-07-04|654|2844|219|1954|0|7|4|3|1954|219|2844|Sunday|1954Q3|N|N|N|2434925|2435105|2434563|2434837|N|N|N|N|N| +2434929|AAAAAAAABHHCFCAA|1954-07-05|654|2844|219|1954|1|7|5|3|1954|219|2844|Monday|1954Q3|Y|N|N|2434925|2435105|2434564|2434838|N|N|N|N|N| +2434930|AAAAAAAACHHCFCAA|1954-07-06|654|2845|219|1954|2|7|6|3|1954|219|2845|Tuesday|1954Q3|N|N|Y|2434925|2435105|2434565|2434839|N|N|N|N|N| +2434931|AAAAAAAADHHCFCAA|1954-07-07|654|2845|219|1954|3|7|7|3|1954|219|2845|Wednesday|1954Q3|N|N|N|2434925|2435105|2434566|2434840|N|N|N|N|N| +2434932|AAAAAAAAEHHCFCAA|1954-07-08|654|2845|219|1954|4|7|8|3|1954|219|2845|Thursday|1954Q3|N|N|N|2434925|2435105|2434567|2434841|N|N|N|N|N| +2434933|AAAAAAAAFHHCFCAA|1954-07-09|654|2845|219|1954|5|7|9|3|1954|219|2845|Friday|1954Q3|N|Y|N|2434925|2435105|2434568|2434842|N|N|N|N|N| +2434934|AAAAAAAAGHHCFCAA|1954-07-10|654|2845|219|1954|6|7|10|3|1954|219|2845|Saturday|1954Q3|N|Y|N|2434925|2435105|2434569|2434843|N|N|N|N|N| +2434935|AAAAAAAAHHHCFCAA|1954-07-11|654|2845|219|1954|0|7|11|3|1954|219|2845|Sunday|1954Q3|N|N|N|2434925|2435105|2434570|2434844|N|N|N|N|N| +2434936|AAAAAAAAIHHCFCAA|1954-07-12|654|2845|219|1954|1|7|12|3|1954|219|2845|Monday|1954Q3|N|N|N|2434925|2435105|2434571|2434845|N|N|N|N|N| +2434937|AAAAAAAAJHHCFCAA|1954-07-13|654|2846|219|1954|2|7|13|3|1954|219|2846|Tuesday|1954Q3|N|N|N|2434925|2435105|2434572|2434846|N|N|N|N|N| +2434938|AAAAAAAAKHHCFCAA|1954-07-14|654|2846|219|1954|3|7|14|3|1954|219|2846|Wednesday|1954Q3|N|N|N|2434925|2435105|2434573|2434847|N|N|N|N|N| +2434939|AAAAAAAALHHCFCAA|1954-07-15|654|2846|219|1954|4|7|15|3|1954|219|2846|Thursday|1954Q3|N|N|N|2434925|2435105|2434574|2434848|N|N|N|N|N| +2434940|AAAAAAAAMHHCFCAA|1954-07-16|654|2846|219|1954|5|7|16|3|1954|219|2846|Friday|1954Q3|N|Y|N|2434925|2435105|2434575|2434849|N|N|N|N|N| +2434941|AAAAAAAANHHCFCAA|1954-07-17|654|2846|219|1954|6|7|17|3|1954|219|2846|Saturday|1954Q3|N|Y|N|2434925|2435105|2434576|2434850|N|N|N|N|N| +2434942|AAAAAAAAOHHCFCAA|1954-07-18|654|2846|219|1954|0|7|18|3|1954|219|2846|Sunday|1954Q3|N|N|N|2434925|2435105|2434577|2434851|N|N|N|N|N| +2434943|AAAAAAAAPHHCFCAA|1954-07-19|654|2846|219|1954|1|7|19|3|1954|219|2846|Monday|1954Q3|N|N|N|2434925|2435105|2434578|2434852|N|N|N|N|N| +2434944|AAAAAAAAAIHCFCAA|1954-07-20|654|2847|219|1954|2|7|20|3|1954|219|2847|Tuesday|1954Q3|N|N|N|2434925|2435105|2434579|2434853|N|N|N|N|N| +2434945|AAAAAAAABIHCFCAA|1954-07-21|654|2847|219|1954|3|7|21|3|1954|219|2847|Wednesday|1954Q3|N|N|N|2434925|2435105|2434580|2434854|N|N|N|N|N| +2434946|AAAAAAAACIHCFCAA|1954-07-22|654|2847|219|1954|4|7|22|3|1954|219|2847|Thursday|1954Q3|N|N|N|2434925|2435105|2434581|2434855|N|N|N|N|N| +2434947|AAAAAAAADIHCFCAA|1954-07-23|654|2847|219|1954|5|7|23|3|1954|219|2847|Friday|1954Q3|N|Y|N|2434925|2435105|2434582|2434856|N|N|N|N|N| +2434948|AAAAAAAAEIHCFCAA|1954-07-24|654|2847|219|1954|6|7|24|3|1954|219|2847|Saturday|1954Q3|N|Y|N|2434925|2435105|2434583|2434857|N|N|N|N|N| +2434949|AAAAAAAAFIHCFCAA|1954-07-25|654|2847|219|1954|0|7|25|3|1954|219|2847|Sunday|1954Q3|N|N|N|2434925|2435105|2434584|2434858|N|N|N|N|N| +2434950|AAAAAAAAGIHCFCAA|1954-07-26|654|2847|219|1954|1|7|26|3|1954|219|2847|Monday|1954Q3|N|N|N|2434925|2435105|2434585|2434859|N|N|N|N|N| +2434951|AAAAAAAAHIHCFCAA|1954-07-27|654|2848|219|1954|2|7|27|3|1954|219|2848|Tuesday|1954Q3|N|N|N|2434925|2435105|2434586|2434860|N|N|N|N|N| +2434952|AAAAAAAAIIHCFCAA|1954-07-28|654|2848|219|1954|3|7|28|3|1954|219|2848|Wednesday|1954Q3|N|N|N|2434925|2435105|2434587|2434861|N|N|N|N|N| +2434953|AAAAAAAAJIHCFCAA|1954-07-29|654|2848|219|1954|4|7|29|3|1954|219|2848|Thursday|1954Q3|N|N|N|2434925|2435105|2434588|2434862|N|N|N|N|N| +2434954|AAAAAAAAKIHCFCAA|1954-07-30|654|2848|219|1954|5|7|30|3|1954|219|2848|Friday|1954Q3|N|Y|N|2434925|2435105|2434589|2434863|N|N|N|N|N| +2434955|AAAAAAAALIHCFCAA|1954-07-31|654|2848|219|1954|6|7|31|3|1954|219|2848|Saturday|1954Q3|N|Y|N|2434925|2435105|2434590|2434864|N|N|N|N|N| +2434956|AAAAAAAAMIHCFCAA|1954-08-01|655|2848|219|1954|0|8|1|3|1954|219|2848|Sunday|1954Q3|N|N|N|2434956|2435167|2434591|2434865|N|N|N|N|N| +2434957|AAAAAAAANIHCFCAA|1954-08-02|655|2848|219|1954|1|8|2|3|1954|219|2848|Monday|1954Q3|N|N|N|2434956|2435167|2434592|2434866|N|N|N|N|N| +2434958|AAAAAAAAOIHCFCAA|1954-08-03|655|2849|219|1954|2|8|3|3|1954|219|2849|Tuesday|1954Q3|N|N|N|2434956|2435167|2434593|2434867|N|N|N|N|N| +2434959|AAAAAAAAPIHCFCAA|1954-08-04|655|2849|219|1954|3|8|4|3|1954|219|2849|Wednesday|1954Q3|N|N|N|2434956|2435167|2434594|2434868|N|N|N|N|N| +2434960|AAAAAAAAAJHCFCAA|1954-08-05|655|2849|219|1954|4|8|5|3|1954|219|2849|Thursday|1954Q3|N|N|N|2434956|2435167|2434595|2434869|N|N|N|N|N| +2434961|AAAAAAAABJHCFCAA|1954-08-06|655|2849|219|1954|5|8|6|3|1954|219|2849|Friday|1954Q3|N|Y|N|2434956|2435167|2434596|2434870|N|N|N|N|N| +2434962|AAAAAAAACJHCFCAA|1954-08-07|655|2849|219|1954|6|8|7|3|1954|219|2849|Saturday|1954Q3|N|Y|N|2434956|2435167|2434597|2434871|N|N|N|N|N| +2434963|AAAAAAAADJHCFCAA|1954-08-08|655|2849|219|1954|0|8|8|3|1954|219|2849|Sunday|1954Q3|N|N|N|2434956|2435167|2434598|2434872|N|N|N|N|N| +2434964|AAAAAAAAEJHCFCAA|1954-08-09|655|2849|219|1954|1|8|9|3|1954|219|2849|Monday|1954Q3|N|N|N|2434956|2435167|2434599|2434873|N|N|N|N|N| +2434965|AAAAAAAAFJHCFCAA|1954-08-10|655|2850|219|1954|2|8|10|3|1954|219|2850|Tuesday|1954Q3|N|N|N|2434956|2435167|2434600|2434874|N|N|N|N|N| +2434966|AAAAAAAAGJHCFCAA|1954-08-11|655|2850|219|1954|3|8|11|3|1954|219|2850|Wednesday|1954Q3|N|N|N|2434956|2435167|2434601|2434875|N|N|N|N|N| +2434967|AAAAAAAAHJHCFCAA|1954-08-12|655|2850|219|1954|4|8|12|3|1954|219|2850|Thursday|1954Q3|N|N|N|2434956|2435167|2434602|2434876|N|N|N|N|N| +2434968|AAAAAAAAIJHCFCAA|1954-08-13|655|2850|219|1954|5|8|13|3|1954|219|2850|Friday|1954Q3|N|Y|N|2434956|2435167|2434603|2434877|N|N|N|N|N| +2434969|AAAAAAAAJJHCFCAA|1954-08-14|655|2850|219|1954|6|8|14|3|1954|219|2850|Saturday|1954Q3|N|Y|N|2434956|2435167|2434604|2434878|N|N|N|N|N| +2434970|AAAAAAAAKJHCFCAA|1954-08-15|655|2850|219|1954|0|8|15|3|1954|219|2850|Sunday|1954Q3|N|N|N|2434956|2435167|2434605|2434879|N|N|N|N|N| +2434971|AAAAAAAALJHCFCAA|1954-08-16|655|2850|219|1954|1|8|16|3|1954|219|2850|Monday|1954Q3|N|N|N|2434956|2435167|2434606|2434880|N|N|N|N|N| +2434972|AAAAAAAAMJHCFCAA|1954-08-17|655|2851|219|1954|2|8|17|3|1954|219|2851|Tuesday|1954Q3|N|N|N|2434956|2435167|2434607|2434881|N|N|N|N|N| +2434973|AAAAAAAANJHCFCAA|1954-08-18|655|2851|219|1954|3|8|18|3|1954|219|2851|Wednesday|1954Q3|N|N|N|2434956|2435167|2434608|2434882|N|N|N|N|N| +2434974|AAAAAAAAOJHCFCAA|1954-08-19|655|2851|219|1954|4|8|19|3|1954|219|2851|Thursday|1954Q3|N|N|N|2434956|2435167|2434609|2434883|N|N|N|N|N| +2434975|AAAAAAAAPJHCFCAA|1954-08-20|655|2851|219|1954|5|8|20|3|1954|219|2851|Friday|1954Q3|N|Y|N|2434956|2435167|2434610|2434884|N|N|N|N|N| +2434976|AAAAAAAAAKHCFCAA|1954-08-21|655|2851|219|1954|6|8|21|3|1954|219|2851|Saturday|1954Q3|N|Y|N|2434956|2435167|2434611|2434885|N|N|N|N|N| +2434977|AAAAAAAABKHCFCAA|1954-08-22|655|2851|219|1954|0|8|22|3|1954|219|2851|Sunday|1954Q3|N|N|N|2434956|2435167|2434612|2434886|N|N|N|N|N| +2434978|AAAAAAAACKHCFCAA|1954-08-23|655|2851|219|1954|1|8|23|3|1954|219|2851|Monday|1954Q3|N|N|N|2434956|2435167|2434613|2434887|N|N|N|N|N| +2434979|AAAAAAAADKHCFCAA|1954-08-24|655|2852|219|1954|2|8|24|3|1954|219|2852|Tuesday|1954Q3|N|N|N|2434956|2435167|2434614|2434888|N|N|N|N|N| +2434980|AAAAAAAAEKHCFCAA|1954-08-25|655|2852|219|1954|3|8|25|3|1954|219|2852|Wednesday|1954Q3|N|N|N|2434956|2435167|2434615|2434889|N|N|N|N|N| +2434981|AAAAAAAAFKHCFCAA|1954-08-26|655|2852|219|1954|4|8|26|3|1954|219|2852|Thursday|1954Q3|N|N|N|2434956|2435167|2434616|2434890|N|N|N|N|N| +2434982|AAAAAAAAGKHCFCAA|1954-08-27|655|2852|219|1954|5|8|27|3|1954|219|2852|Friday|1954Q3|N|Y|N|2434956|2435167|2434617|2434891|N|N|N|N|N| +2434983|AAAAAAAAHKHCFCAA|1954-08-28|655|2852|219|1954|6|8|28|3|1954|219|2852|Saturday|1954Q3|N|Y|N|2434956|2435167|2434618|2434892|N|N|N|N|N| +2434984|AAAAAAAAIKHCFCAA|1954-08-29|655|2852|219|1954|0|8|29|3|1954|219|2852|Sunday|1954Q3|N|N|N|2434956|2435167|2434619|2434893|N|N|N|N|N| +2434985|AAAAAAAAJKHCFCAA|1954-08-30|655|2852|219|1954|1|8|30|3|1954|219|2852|Monday|1954Q3|N|N|N|2434956|2435167|2434620|2434894|N|N|N|N|N| +2434986|AAAAAAAAKKHCFCAA|1954-08-31|655|2853|219|1954|2|8|31|3|1954|219|2853|Tuesday|1954Q3|N|N|N|2434956|2435167|2434621|2434895|N|N|N|N|N| +2434987|AAAAAAAALKHCFCAA|1954-09-01|656|2853|220|1954|3|9|1|3|1954|220|2853|Wednesday|1954Q3|N|N|N|2434987|2435229|2434622|2434896|N|N|N|N|N| +2434988|AAAAAAAAMKHCFCAA|1954-09-02|656|2853|220|1954|4|9|2|3|1954|220|2853|Thursday|1954Q3|N|N|N|2434987|2435229|2434623|2434897|N|N|N|N|N| +2434989|AAAAAAAANKHCFCAA|1954-09-03|656|2853|220|1954|5|9|3|3|1954|220|2853|Friday|1954Q3|N|Y|N|2434987|2435229|2434624|2434898|N|N|N|N|N| +2434990|AAAAAAAAOKHCFCAA|1954-09-04|656|2853|220|1954|6|9|4|3|1954|220|2853|Saturday|1954Q3|N|Y|N|2434987|2435229|2434625|2434899|N|N|N|N|N| +2434991|AAAAAAAAPKHCFCAA|1954-09-05|656|2853|220|1954|0|9|5|3|1954|220|2853|Sunday|1954Q3|N|N|N|2434987|2435229|2434626|2434900|N|N|N|N|N| +2434992|AAAAAAAAALHCFCAA|1954-09-06|656|2853|220|1954|1|9|6|3|1954|220|2853|Monday|1954Q3|N|N|N|2434987|2435229|2434627|2434901|N|N|N|N|N| +2434993|AAAAAAAABLHCFCAA|1954-09-07|656|2854|220|1954|2|9|7|3|1954|220|2854|Tuesday|1954Q3|N|N|N|2434987|2435229|2434628|2434902|N|N|N|N|N| +2434994|AAAAAAAACLHCFCAA|1954-09-08|656|2854|220|1954|3|9|8|3|1954|220|2854|Wednesday|1954Q3|N|N|N|2434987|2435229|2434629|2434903|N|N|N|N|N| +2434995|AAAAAAAADLHCFCAA|1954-09-09|656|2854|220|1954|4|9|9|3|1954|220|2854|Thursday|1954Q3|N|N|N|2434987|2435229|2434630|2434904|N|N|N|N|N| +2434996|AAAAAAAAELHCFCAA|1954-09-10|656|2854|220|1954|5|9|10|3|1954|220|2854|Friday|1954Q3|N|Y|N|2434987|2435229|2434631|2434905|N|N|N|N|N| +2434997|AAAAAAAAFLHCFCAA|1954-09-11|656|2854|220|1954|6|9|11|3|1954|220|2854|Saturday|1954Q3|N|Y|N|2434987|2435229|2434632|2434906|N|N|N|N|N| +2434998|AAAAAAAAGLHCFCAA|1954-09-12|656|2854|220|1954|0|9|12|3|1954|220|2854|Sunday|1954Q3|N|N|N|2434987|2435229|2434633|2434907|N|N|N|N|N| +2434999|AAAAAAAAHLHCFCAA|1954-09-13|656|2854|220|1954|1|9|13|3|1954|220|2854|Monday|1954Q3|N|N|N|2434987|2435229|2434634|2434908|N|N|N|N|N| +2435000|AAAAAAAAILHCFCAA|1954-09-14|656|2855|220|1954|2|9|14|3|1954|220|2855|Tuesday|1954Q3|N|N|N|2434987|2435229|2434635|2434909|N|N|N|N|N| +2435001|AAAAAAAAJLHCFCAA|1954-09-15|656|2855|220|1954|3|9|15|3|1954|220|2855|Wednesday|1954Q3|N|N|N|2434987|2435229|2434636|2434910|N|N|N|N|N| +2435002|AAAAAAAAKLHCFCAA|1954-09-16|656|2855|220|1954|4|9|16|3|1954|220|2855|Thursday|1954Q3|N|N|N|2434987|2435229|2434637|2434911|N|N|N|N|N| +2435003|AAAAAAAALLHCFCAA|1954-09-17|656|2855|220|1954|5|9|17|3|1954|220|2855|Friday|1954Q3|N|Y|N|2434987|2435229|2434638|2434912|N|N|N|N|N| +2435004|AAAAAAAAMLHCFCAA|1954-09-18|656|2855|220|1954|6|9|18|3|1954|220|2855|Saturday|1954Q3|N|Y|N|2434987|2435229|2434639|2434913|N|N|N|N|N| +2435005|AAAAAAAANLHCFCAA|1954-09-19|656|2855|220|1954|0|9|19|3|1954|220|2855|Sunday|1954Q3|N|N|N|2434987|2435229|2434640|2434914|N|N|N|N|N| +2435006|AAAAAAAAOLHCFCAA|1954-09-20|656|2855|220|1954|1|9|20|3|1954|220|2855|Monday|1954Q3|N|N|N|2434987|2435229|2434641|2434915|N|N|N|N|N| +2435007|AAAAAAAAPLHCFCAA|1954-09-21|656|2856|220|1954|2|9|21|3|1954|220|2856|Tuesday|1954Q3|N|N|N|2434987|2435229|2434642|2434916|N|N|N|N|N| +2435008|AAAAAAAAAMHCFCAA|1954-09-22|656|2856|220|1954|3|9|22|3|1954|220|2856|Wednesday|1954Q3|N|N|N|2434987|2435229|2434643|2434917|N|N|N|N|N| +2435009|AAAAAAAABMHCFCAA|1954-09-23|656|2856|220|1954|4|9|23|3|1954|220|2856|Thursday|1954Q3|N|N|N|2434987|2435229|2434644|2434918|N|N|N|N|N| +2435010|AAAAAAAACMHCFCAA|1954-09-24|656|2856|220|1954|5|9|24|3|1954|220|2856|Friday|1954Q3|N|Y|N|2434987|2435229|2434645|2434919|N|N|N|N|N| +2435011|AAAAAAAADMHCFCAA|1954-09-25|656|2856|220|1954|6|9|25|3|1954|220|2856|Saturday|1954Q3|N|Y|N|2434987|2435229|2434646|2434920|N|N|N|N|N| +2435012|AAAAAAAAEMHCFCAA|1954-09-26|656|2856|220|1954|0|9|26|3|1954|220|2856|Sunday|1954Q3|N|N|N|2434987|2435229|2434647|2434921|N|N|N|N|N| +2435013|AAAAAAAAFMHCFCAA|1954-09-27|656|2856|220|1954|1|9|27|3|1954|220|2856|Monday|1954Q3|N|N|N|2434987|2435229|2434648|2434922|N|N|N|N|N| +2435014|AAAAAAAAGMHCFCAA|1954-09-28|656|2857|220|1954|2|9|28|3|1954|220|2857|Tuesday|1954Q3|N|N|N|2434987|2435229|2434649|2434923|N|N|N|N|N| +2435015|AAAAAAAAHMHCFCAA|1954-09-29|656|2857|220|1954|3|9|29|3|1954|220|2857|Wednesday|1954Q3|N|N|N|2434987|2435229|2434650|2434924|N|N|N|N|N| +2435016|AAAAAAAAIMHCFCAA|1954-09-30|656|2857|220|1954|4|9|30|3|1954|220|2857|Thursday|1954Q3|N|N|N|2434987|2435229|2434651|2434925|N|N|N|N|N| +2435017|AAAAAAAAJMHCFCAA|1954-10-01|657|2857|220|1954|5|10|1|3|1954|220|2857|Friday|1954Q3|N|Y|N|2435017|2435289|2434652|2434925|N|N|N|N|N| +2435018|AAAAAAAAKMHCFCAA|1954-10-02|657|2857|220|1954|6|10|2|4|1954|220|2857|Saturday|1954Q4|N|Y|N|2435017|2435289|2434653|2434926|N|N|N|N|N| +2435019|AAAAAAAALMHCFCAA|1954-10-03|657|2857|220|1954|0|10|3|4|1954|220|2857|Sunday|1954Q4|N|N|N|2435017|2435289|2434654|2434927|N|N|N|N|N| +2435020|AAAAAAAAMMHCFCAA|1954-10-04|657|2857|220|1954|1|10|4|4|1954|220|2857|Monday|1954Q4|N|N|N|2435017|2435289|2434655|2434928|N|N|N|N|N| +2435021|AAAAAAAANMHCFCAA|1954-10-05|657|2858|220|1954|2|10|5|4|1954|220|2858|Tuesday|1954Q4|N|N|N|2435017|2435289|2434656|2434929|N|N|N|N|N| +2435022|AAAAAAAAOMHCFCAA|1954-10-06|657|2858|220|1954|3|10|6|4|1954|220|2858|Wednesday|1954Q4|N|N|N|2435017|2435289|2434657|2434930|N|N|N|N|N| +2435023|AAAAAAAAPMHCFCAA|1954-10-07|657|2858|220|1954|4|10|7|4|1954|220|2858|Thursday|1954Q4|N|N|N|2435017|2435289|2434658|2434931|N|N|N|N|N| +2435024|AAAAAAAAANHCFCAA|1954-10-08|657|2858|220|1954|5|10|8|4|1954|220|2858|Friday|1954Q4|N|Y|N|2435017|2435289|2434659|2434932|N|N|N|N|N| +2435025|AAAAAAAABNHCFCAA|1954-10-09|657|2858|220|1954|6|10|9|4|1954|220|2858|Saturday|1954Q4|N|Y|N|2435017|2435289|2434660|2434933|N|N|N|N|N| +2435026|AAAAAAAACNHCFCAA|1954-10-10|657|2858|220|1954|0|10|10|4|1954|220|2858|Sunday|1954Q4|N|N|N|2435017|2435289|2434661|2434934|N|N|N|N|N| +2435027|AAAAAAAADNHCFCAA|1954-10-11|657|2858|220|1954|1|10|11|4|1954|220|2858|Monday|1954Q4|N|N|N|2435017|2435289|2434662|2434935|N|N|N|N|N| +2435028|AAAAAAAAENHCFCAA|1954-10-12|657|2859|220|1954|2|10|12|4|1954|220|2859|Tuesday|1954Q4|N|N|N|2435017|2435289|2434663|2434936|N|N|N|N|N| +2435029|AAAAAAAAFNHCFCAA|1954-10-13|657|2859|220|1954|3|10|13|4|1954|220|2859|Wednesday|1954Q4|N|N|N|2435017|2435289|2434664|2434937|N|N|N|N|N| +2435030|AAAAAAAAGNHCFCAA|1954-10-14|657|2859|220|1954|4|10|14|4|1954|220|2859|Thursday|1954Q4|N|N|N|2435017|2435289|2434665|2434938|N|N|N|N|N| +2435031|AAAAAAAAHNHCFCAA|1954-10-15|657|2859|220|1954|5|10|15|4|1954|220|2859|Friday|1954Q4|N|Y|N|2435017|2435289|2434666|2434939|N|N|N|N|N| +2435032|AAAAAAAAINHCFCAA|1954-10-16|657|2859|220|1954|6|10|16|4|1954|220|2859|Saturday|1954Q4|N|Y|N|2435017|2435289|2434667|2434940|N|N|N|N|N| +2435033|AAAAAAAAJNHCFCAA|1954-10-17|657|2859|220|1954|0|10|17|4|1954|220|2859|Sunday|1954Q4|N|N|N|2435017|2435289|2434668|2434941|N|N|N|N|N| +2435034|AAAAAAAAKNHCFCAA|1954-10-18|657|2859|220|1954|1|10|18|4|1954|220|2859|Monday|1954Q4|N|N|N|2435017|2435289|2434669|2434942|N|N|N|N|N| +2435035|AAAAAAAALNHCFCAA|1954-10-19|657|2860|220|1954|2|10|19|4|1954|220|2860|Tuesday|1954Q4|N|N|N|2435017|2435289|2434670|2434943|N|N|N|N|N| +2435036|AAAAAAAAMNHCFCAA|1954-10-20|657|2860|220|1954|3|10|20|4|1954|220|2860|Wednesday|1954Q4|N|N|N|2435017|2435289|2434671|2434944|N|N|N|N|N| +2435037|AAAAAAAANNHCFCAA|1954-10-21|657|2860|220|1954|4|10|21|4|1954|220|2860|Thursday|1954Q4|N|N|N|2435017|2435289|2434672|2434945|N|N|N|N|N| +2435038|AAAAAAAAONHCFCAA|1954-10-22|657|2860|220|1954|5|10|22|4|1954|220|2860|Friday|1954Q4|N|Y|N|2435017|2435289|2434673|2434946|N|N|N|N|N| +2435039|AAAAAAAAPNHCFCAA|1954-10-23|657|2860|220|1954|6|10|23|4|1954|220|2860|Saturday|1954Q4|N|Y|N|2435017|2435289|2434674|2434947|N|N|N|N|N| +2435040|AAAAAAAAAOHCFCAA|1954-10-24|657|2860|220|1954|0|10|24|4|1954|220|2860|Sunday|1954Q4|N|N|N|2435017|2435289|2434675|2434948|N|N|N|N|N| +2435041|AAAAAAAABOHCFCAA|1954-10-25|657|2860|220|1954|1|10|25|4|1954|220|2860|Monday|1954Q4|N|N|N|2435017|2435289|2434676|2434949|N|N|N|N|N| +2435042|AAAAAAAACOHCFCAA|1954-10-26|657|2861|220|1954|2|10|26|4|1954|220|2861|Tuesday|1954Q4|N|N|N|2435017|2435289|2434677|2434950|N|N|N|N|N| +2435043|AAAAAAAADOHCFCAA|1954-10-27|657|2861|220|1954|3|10|27|4|1954|220|2861|Wednesday|1954Q4|N|N|N|2435017|2435289|2434678|2434951|N|N|N|N|N| +2435044|AAAAAAAAEOHCFCAA|1954-10-28|657|2861|220|1954|4|10|28|4|1954|220|2861|Thursday|1954Q4|N|N|N|2435017|2435289|2434679|2434952|N|N|N|N|N| +2435045|AAAAAAAAFOHCFCAA|1954-10-29|657|2861|220|1954|5|10|29|4|1954|220|2861|Friday|1954Q4|N|Y|N|2435017|2435289|2434680|2434953|N|N|N|N|N| +2435046|AAAAAAAAGOHCFCAA|1954-10-30|657|2861|220|1954|6|10|30|4|1954|220|2861|Saturday|1954Q4|N|Y|N|2435017|2435289|2434681|2434954|N|N|N|N|N| +2435047|AAAAAAAAHOHCFCAA|1954-10-31|657|2861|220|1954|0|10|31|4|1954|220|2861|Sunday|1954Q4|N|N|N|2435017|2435289|2434682|2434955|N|N|N|N|N| +2435048|AAAAAAAAIOHCFCAA|1954-11-01|658|2861|220|1954|1|11|1|4|1954|220|2861|Monday|1954Q4|N|N|N|2435048|2435351|2434683|2434956|N|N|N|N|N| +2435049|AAAAAAAAJOHCFCAA|1954-11-02|658|2862|220|1954|2|11|2|4|1954|220|2862|Tuesday|1954Q4|N|N|N|2435048|2435351|2434684|2434957|N|N|N|N|N| +2435050|AAAAAAAAKOHCFCAA|1954-11-03|658|2862|220|1954|3|11|3|4|1954|220|2862|Wednesday|1954Q4|N|N|N|2435048|2435351|2434685|2434958|N|N|N|N|N| +2435051|AAAAAAAALOHCFCAA|1954-11-04|658|2862|220|1954|4|11|4|4|1954|220|2862|Thursday|1954Q4|N|N|N|2435048|2435351|2434686|2434959|N|N|N|N|N| +2435052|AAAAAAAAMOHCFCAA|1954-11-05|658|2862|220|1954|5|11|5|4|1954|220|2862|Friday|1954Q4|N|Y|N|2435048|2435351|2434687|2434960|N|N|N|N|N| +2435053|AAAAAAAANOHCFCAA|1954-11-06|658|2862|220|1954|6|11|6|4|1954|220|2862|Saturday|1954Q4|N|Y|N|2435048|2435351|2434688|2434961|N|N|N|N|N| +2435054|AAAAAAAAOOHCFCAA|1954-11-07|658|2862|220|1954|0|11|7|4|1954|220|2862|Sunday|1954Q4|N|N|N|2435048|2435351|2434689|2434962|N|N|N|N|N| +2435055|AAAAAAAAPOHCFCAA|1954-11-08|658|2862|220|1954|1|11|8|4|1954|220|2862|Monday|1954Q4|N|N|N|2435048|2435351|2434690|2434963|N|N|N|N|N| +2435056|AAAAAAAAAPHCFCAA|1954-11-09|658|2863|220|1954|2|11|9|4|1954|220|2863|Tuesday|1954Q4|N|N|N|2435048|2435351|2434691|2434964|N|N|N|N|N| +2435057|AAAAAAAABPHCFCAA|1954-11-10|658|2863|220|1954|3|11|10|4|1954|220|2863|Wednesday|1954Q4|N|N|N|2435048|2435351|2434692|2434965|N|N|N|N|N| +2435058|AAAAAAAACPHCFCAA|1954-11-11|658|2863|220|1954|4|11|11|4|1954|220|2863|Thursday|1954Q4|N|N|N|2435048|2435351|2434693|2434966|N|N|N|N|N| +2435059|AAAAAAAADPHCFCAA|1954-11-12|658|2863|220|1954|5|11|12|4|1954|220|2863|Friday|1954Q4|N|Y|N|2435048|2435351|2434694|2434967|N|N|N|N|N| +2435060|AAAAAAAAEPHCFCAA|1954-11-13|658|2863|220|1954|6|11|13|4|1954|220|2863|Saturday|1954Q4|N|Y|N|2435048|2435351|2434695|2434968|N|N|N|N|N| +2435061|AAAAAAAAFPHCFCAA|1954-11-14|658|2863|220|1954|0|11|14|4|1954|220|2863|Sunday|1954Q4|N|N|N|2435048|2435351|2434696|2434969|N|N|N|N|N| +2435062|AAAAAAAAGPHCFCAA|1954-11-15|658|2863|220|1954|1|11|15|4|1954|220|2863|Monday|1954Q4|N|N|N|2435048|2435351|2434697|2434970|N|N|N|N|N| +2435063|AAAAAAAAHPHCFCAA|1954-11-16|658|2864|220|1954|2|11|16|4|1954|220|2864|Tuesday|1954Q4|N|N|N|2435048|2435351|2434698|2434971|N|N|N|N|N| +2435064|AAAAAAAAIPHCFCAA|1954-11-17|658|2864|220|1954|3|11|17|4|1954|220|2864|Wednesday|1954Q4|N|N|N|2435048|2435351|2434699|2434972|N|N|N|N|N| +2435065|AAAAAAAAJPHCFCAA|1954-11-18|658|2864|220|1954|4|11|18|4|1954|220|2864|Thursday|1954Q4|N|N|N|2435048|2435351|2434700|2434973|N|N|N|N|N| +2435066|AAAAAAAAKPHCFCAA|1954-11-19|658|2864|220|1954|5|11|19|4|1954|220|2864|Friday|1954Q4|N|Y|N|2435048|2435351|2434701|2434974|N|N|N|N|N| +2435067|AAAAAAAALPHCFCAA|1954-11-20|658|2864|220|1954|6|11|20|4|1954|220|2864|Saturday|1954Q4|N|Y|N|2435048|2435351|2434702|2434975|N|N|N|N|N| +2435068|AAAAAAAAMPHCFCAA|1954-11-21|658|2864|220|1954|0|11|21|4|1954|220|2864|Sunday|1954Q4|N|N|N|2435048|2435351|2434703|2434976|N|N|N|N|N| +2435069|AAAAAAAANPHCFCAA|1954-11-22|658|2864|220|1954|1|11|22|4|1954|220|2864|Monday|1954Q4|N|N|N|2435048|2435351|2434704|2434977|N|N|N|N|N| +2435070|AAAAAAAAOPHCFCAA|1954-11-23|658|2865|220|1954|2|11|23|4|1954|220|2865|Tuesday|1954Q4|N|N|N|2435048|2435351|2434705|2434978|N|N|N|N|N| +2435071|AAAAAAAAPPHCFCAA|1954-11-24|658|2865|220|1954|3|11|24|4|1954|220|2865|Wednesday|1954Q4|N|N|N|2435048|2435351|2434706|2434979|N|N|N|N|N| +2435072|AAAAAAAAAAICFCAA|1954-11-25|658|2865|220|1954|4|11|25|4|1954|220|2865|Thursday|1954Q4|N|N|N|2435048|2435351|2434707|2434980|N|N|N|N|N| +2435073|AAAAAAAABAICFCAA|1954-11-26|658|2865|220|1954|5|11|26|4|1954|220|2865|Friday|1954Q4|N|Y|N|2435048|2435351|2434708|2434981|N|N|N|N|N| +2435074|AAAAAAAACAICFCAA|1954-11-27|658|2865|220|1954|6|11|27|4|1954|220|2865|Saturday|1954Q4|N|Y|N|2435048|2435351|2434709|2434982|N|N|N|N|N| +2435075|AAAAAAAADAICFCAA|1954-11-28|658|2865|220|1954|0|11|28|4|1954|220|2865|Sunday|1954Q4|N|N|N|2435048|2435351|2434710|2434983|N|N|N|N|N| +2435076|AAAAAAAAEAICFCAA|1954-11-29|658|2865|220|1954|1|11|29|4|1954|220|2865|Monday|1954Q4|N|N|N|2435048|2435351|2434711|2434984|N|N|N|N|N| +2435077|AAAAAAAAFAICFCAA|1954-11-30|658|2866|220|1954|2|11|30|4|1954|220|2866|Tuesday|1954Q4|N|N|N|2435048|2435351|2434712|2434985|N|N|N|N|N| +2435078|AAAAAAAAGAICFCAA|1954-12-01|659|2866|221|1954|3|12|1|4|1954|221|2866|Wednesday|1954Q4|N|N|N|2435078|2435411|2434713|2434986|N|N|N|N|N| +2435079|AAAAAAAAHAICFCAA|1954-12-02|659|2866|221|1954|4|12|2|4|1954|221|2866|Thursday|1954Q4|N|N|N|2435078|2435411|2434714|2434987|N|N|N|N|N| +2435080|AAAAAAAAIAICFCAA|1954-12-03|659|2866|221|1954|5|12|3|4|1954|221|2866|Friday|1954Q4|N|Y|N|2435078|2435411|2434715|2434988|N|N|N|N|N| +2435081|AAAAAAAAJAICFCAA|1954-12-04|659|2866|221|1954|6|12|4|4|1954|221|2866|Saturday|1954Q4|N|Y|N|2435078|2435411|2434716|2434989|N|N|N|N|N| +2435082|AAAAAAAAKAICFCAA|1954-12-05|659|2866|221|1954|0|12|5|4|1954|221|2866|Sunday|1954Q4|N|N|N|2435078|2435411|2434717|2434990|N|N|N|N|N| +2435083|AAAAAAAALAICFCAA|1954-12-06|659|2866|221|1954|1|12|6|4|1954|221|2866|Monday|1954Q4|N|N|N|2435078|2435411|2434718|2434991|N|N|N|N|N| +2435084|AAAAAAAAMAICFCAA|1954-12-07|659|2867|221|1954|2|12|7|4|1954|221|2867|Tuesday|1954Q4|N|N|N|2435078|2435411|2434719|2434992|N|N|N|N|N| +2435085|AAAAAAAANAICFCAA|1954-12-08|659|2867|221|1954|3|12|8|4|1954|221|2867|Wednesday|1954Q4|N|N|N|2435078|2435411|2434720|2434993|N|N|N|N|N| +2435086|AAAAAAAAOAICFCAA|1954-12-09|659|2867|221|1954|4|12|9|4|1954|221|2867|Thursday|1954Q4|N|N|N|2435078|2435411|2434721|2434994|N|N|N|N|N| +2435087|AAAAAAAAPAICFCAA|1954-12-10|659|2867|221|1954|5|12|10|4|1954|221|2867|Friday|1954Q4|N|Y|N|2435078|2435411|2434722|2434995|N|N|N|N|N| +2435088|AAAAAAAAABICFCAA|1954-12-11|659|2867|221|1954|6|12|11|4|1954|221|2867|Saturday|1954Q4|N|Y|N|2435078|2435411|2434723|2434996|N|N|N|N|N| +2435089|AAAAAAAABBICFCAA|1954-12-12|659|2867|221|1954|0|12|12|4|1954|221|2867|Sunday|1954Q4|N|N|N|2435078|2435411|2434724|2434997|N|N|N|N|N| +2435090|AAAAAAAACBICFCAA|1954-12-13|659|2867|221|1954|1|12|13|4|1954|221|2867|Monday|1954Q4|N|N|N|2435078|2435411|2434725|2434998|N|N|N|N|N| +2435091|AAAAAAAADBICFCAA|1954-12-14|659|2868|221|1954|2|12|14|4|1954|221|2868|Tuesday|1954Q4|N|N|N|2435078|2435411|2434726|2434999|N|N|N|N|N| +2435092|AAAAAAAAEBICFCAA|1954-12-15|659|2868|221|1954|3|12|15|4|1954|221|2868|Wednesday|1954Q4|N|N|N|2435078|2435411|2434727|2435000|N|N|N|N|N| +2435093|AAAAAAAAFBICFCAA|1954-12-16|659|2868|221|1954|4|12|16|4|1954|221|2868|Thursday|1954Q4|N|N|N|2435078|2435411|2434728|2435001|N|N|N|N|N| +2435094|AAAAAAAAGBICFCAA|1954-12-17|659|2868|221|1954|5|12|17|4|1954|221|2868|Friday|1954Q4|N|Y|N|2435078|2435411|2434729|2435002|N|N|N|N|N| +2435095|AAAAAAAAHBICFCAA|1954-12-18|659|2868|221|1954|6|12|18|4|1954|221|2868|Saturday|1954Q4|N|Y|N|2435078|2435411|2434730|2435003|N|N|N|N|N| +2435096|AAAAAAAAIBICFCAA|1954-12-19|659|2868|221|1954|0|12|19|4|1954|221|2868|Sunday|1954Q4|N|N|N|2435078|2435411|2434731|2435004|N|N|N|N|N| +2435097|AAAAAAAAJBICFCAA|1954-12-20|659|2868|221|1954|1|12|20|4|1954|221|2868|Monday|1954Q4|N|N|N|2435078|2435411|2434732|2435005|N|N|N|N|N| +2435098|AAAAAAAAKBICFCAA|1954-12-21|659|2869|221|1954|2|12|21|4|1954|221|2869|Tuesday|1954Q4|N|N|N|2435078|2435411|2434733|2435006|N|N|N|N|N| +2435099|AAAAAAAALBICFCAA|1954-12-22|659|2869|221|1954|3|12|22|4|1954|221|2869|Wednesday|1954Q4|N|N|N|2435078|2435411|2434734|2435007|N|N|N|N|N| +2435100|AAAAAAAAMBICFCAA|1954-12-23|659|2869|221|1954|4|12|23|4|1954|221|2869|Thursday|1954Q4|N|N|N|2435078|2435411|2434735|2435008|N|N|N|N|N| +2435101|AAAAAAAANBICFCAA|1954-12-24|659|2869|221|1954|5|12|24|4|1954|221|2869|Friday|1954Q4|N|Y|N|2435078|2435411|2434736|2435009|N|N|N|N|N| +2435102|AAAAAAAAOBICFCAA|1954-12-25|659|2869|221|1954|6|12|25|4|1954|221|2869|Saturday|1954Q4|N|Y|N|2435078|2435411|2434737|2435010|N|N|N|N|N| +2435103|AAAAAAAAPBICFCAA|1954-12-26|659|2869|221|1954|0|12|26|4|1954|221|2869|Sunday|1954Q4|Y|N|N|2435078|2435411|2434738|2435011|N|N|N|N|N| +2435104|AAAAAAAAACICFCAA|1954-12-27|659|2869|221|1954|1|12|27|4|1954|221|2869|Monday|1954Q4|N|N|Y|2435078|2435411|2434739|2435012|N|N|N|N|N| +2435105|AAAAAAAABCICFCAA|1954-12-28|659|2870|221|1954|2|12|28|4|1954|221|2870|Tuesday|1954Q4|N|N|N|2435078|2435411|2434740|2435013|N|N|N|N|N| +2435106|AAAAAAAACCICFCAA|1954-12-29|659|2870|221|1954|3|12|29|4|1954|221|2870|Wednesday|1954Q4|N|N|N|2435078|2435411|2434741|2435014|N|N|N|N|N| +2435107|AAAAAAAADCICFCAA|1954-12-30|659|2870|221|1954|4|12|30|4|1954|221|2870|Thursday|1954Q4|N|N|N|2435078|2435411|2434742|2435015|N|N|N|N|N| +2435108|AAAAAAAAECICFCAA|1954-12-31|659|2870|221|1954|5|12|31|4|1954|221|2870|Friday|1954Q4|N|Y|N|2435078|2435411|2434743|2435016|N|N|N|N|N| +2435109|AAAAAAAAFCICFCAA|1955-01-01|660|2870|221|1955|6|1|1|1|1955|221|2870|Saturday|1955Q1|Y|Y|N|2435109|2435108|2434744|2435017|N|N|N|N|N| +2435110|AAAAAAAAGCICFCAA|1955-01-02|660|2870|221|1955|0|1|2|1|1955|221|2870|Sunday|1955Q1|N|N|Y|2435109|2435108|2434745|2435018|N|N|N|N|N| +2435111|AAAAAAAAHCICFCAA|1955-01-03|660|2870|221|1955|1|1|3|1|1955|221|2870|Monday|1955Q1|N|N|N|2435109|2435108|2434746|2435019|N|N|N|N|N| +2435112|AAAAAAAAICICFCAA|1955-01-04|660|2871|221|1955|2|1|4|1|1955|221|2871|Tuesday|1955Q1|N|N|N|2435109|2435108|2434747|2435020|N|N|N|N|N| +2435113|AAAAAAAAJCICFCAA|1955-01-05|660|2871|221|1955|3|1|5|1|1955|221|2871|Wednesday|1955Q1|N|N|N|2435109|2435108|2434748|2435021|N|N|N|N|N| +2435114|AAAAAAAAKCICFCAA|1955-01-06|660|2871|221|1955|4|1|6|1|1955|221|2871|Thursday|1955Q1|N|N|N|2435109|2435108|2434749|2435022|N|N|N|N|N| +2435115|AAAAAAAALCICFCAA|1955-01-07|660|2871|221|1955|5|1|7|1|1955|221|2871|Friday|1955Q1|N|Y|N|2435109|2435108|2434750|2435023|N|N|N|N|N| +2435116|AAAAAAAAMCICFCAA|1955-01-08|660|2871|221|1955|6|1|8|1|1955|221|2871|Saturday|1955Q1|N|Y|N|2435109|2435108|2434751|2435024|N|N|N|N|N| +2435117|AAAAAAAANCICFCAA|1955-01-09|660|2871|221|1955|0|1|9|1|1955|221|2871|Sunday|1955Q1|N|N|N|2435109|2435108|2434752|2435025|N|N|N|N|N| +2435118|AAAAAAAAOCICFCAA|1955-01-10|660|2871|221|1955|1|1|10|1|1955|221|2871|Monday|1955Q1|N|N|N|2435109|2435108|2434753|2435026|N|N|N|N|N| +2435119|AAAAAAAAPCICFCAA|1955-01-11|660|2872|221|1955|2|1|11|1|1955|221|2872|Tuesday|1955Q1|N|N|N|2435109|2435108|2434754|2435027|N|N|N|N|N| +2435120|AAAAAAAAADICFCAA|1955-01-12|660|2872|221|1955|3|1|12|1|1955|221|2872|Wednesday|1955Q1|N|N|N|2435109|2435108|2434755|2435028|N|N|N|N|N| +2435121|AAAAAAAABDICFCAA|1955-01-13|660|2872|221|1955|4|1|13|1|1955|221|2872|Thursday|1955Q1|N|N|N|2435109|2435108|2434756|2435029|N|N|N|N|N| +2435122|AAAAAAAACDICFCAA|1955-01-14|660|2872|221|1955|5|1|14|1|1955|221|2872|Friday|1955Q1|N|Y|N|2435109|2435108|2434757|2435030|N|N|N|N|N| +2435123|AAAAAAAADDICFCAA|1955-01-15|660|2872|221|1955|6|1|15|1|1955|221|2872|Saturday|1955Q1|N|Y|N|2435109|2435108|2434758|2435031|N|N|N|N|N| +2435124|AAAAAAAAEDICFCAA|1955-01-16|660|2872|221|1955|0|1|16|1|1955|221|2872|Sunday|1955Q1|N|N|N|2435109|2435108|2434759|2435032|N|N|N|N|N| +2435125|AAAAAAAAFDICFCAA|1955-01-17|660|2872|221|1955|1|1|17|1|1955|221|2872|Monday|1955Q1|N|N|N|2435109|2435108|2434760|2435033|N|N|N|N|N| +2435126|AAAAAAAAGDICFCAA|1955-01-18|660|2873|221|1955|2|1|18|1|1955|221|2873|Tuesday|1955Q1|N|N|N|2435109|2435108|2434761|2435034|N|N|N|N|N| +2435127|AAAAAAAAHDICFCAA|1955-01-19|660|2873|221|1955|3|1|19|1|1955|221|2873|Wednesday|1955Q1|N|N|N|2435109|2435108|2434762|2435035|N|N|N|N|N| +2435128|AAAAAAAAIDICFCAA|1955-01-20|660|2873|221|1955|4|1|20|1|1955|221|2873|Thursday|1955Q1|N|N|N|2435109|2435108|2434763|2435036|N|N|N|N|N| +2435129|AAAAAAAAJDICFCAA|1955-01-21|660|2873|221|1955|5|1|21|1|1955|221|2873|Friday|1955Q1|N|Y|N|2435109|2435108|2434764|2435037|N|N|N|N|N| +2435130|AAAAAAAAKDICFCAA|1955-01-22|660|2873|221|1955|6|1|22|1|1955|221|2873|Saturday|1955Q1|N|Y|N|2435109|2435108|2434765|2435038|N|N|N|N|N| +2435131|AAAAAAAALDICFCAA|1955-01-23|660|2873|221|1955|0|1|23|1|1955|221|2873|Sunday|1955Q1|N|N|N|2435109|2435108|2434766|2435039|N|N|N|N|N| +2435132|AAAAAAAAMDICFCAA|1955-01-24|660|2873|221|1955|1|1|24|1|1955|221|2873|Monday|1955Q1|N|N|N|2435109|2435108|2434767|2435040|N|N|N|N|N| +2435133|AAAAAAAANDICFCAA|1955-01-25|660|2874|221|1955|2|1|25|1|1955|221|2874|Tuesday|1955Q1|N|N|N|2435109|2435108|2434768|2435041|N|N|N|N|N| +2435134|AAAAAAAAODICFCAA|1955-01-26|660|2874|221|1955|3|1|26|1|1955|221|2874|Wednesday|1955Q1|N|N|N|2435109|2435108|2434769|2435042|N|N|N|N|N| +2435135|AAAAAAAAPDICFCAA|1955-01-27|660|2874|221|1955|4|1|27|1|1955|221|2874|Thursday|1955Q1|N|N|N|2435109|2435108|2434770|2435043|N|N|N|N|N| +2435136|AAAAAAAAAEICFCAA|1955-01-28|660|2874|221|1955|5|1|28|1|1955|221|2874|Friday|1955Q1|N|Y|N|2435109|2435108|2434771|2435044|N|N|N|N|N| +2435137|AAAAAAAABEICFCAA|1955-01-29|660|2874|221|1955|6|1|29|1|1955|221|2874|Saturday|1955Q1|N|Y|N|2435109|2435108|2434772|2435045|N|N|N|N|N| +2435138|AAAAAAAACEICFCAA|1955-01-30|660|2874|221|1955|0|1|30|1|1955|221|2874|Sunday|1955Q1|N|N|N|2435109|2435108|2434773|2435046|N|N|N|N|N| +2435139|AAAAAAAADEICFCAA|1955-01-31|660|2874|221|1955|1|1|31|1|1955|221|2874|Monday|1955Q1|N|N|N|2435109|2435108|2434774|2435047|N|N|N|N|N| +2435140|AAAAAAAAEEICFCAA|1955-02-01|661|2875|221|1955|2|2|1|1|1955|221|2875|Tuesday|1955Q1|N|N|N|2435140|2435170|2434775|2435048|N|N|N|N|N| +2435141|AAAAAAAAFEICFCAA|1955-02-02|661|2875|221|1955|3|2|2|1|1955|221|2875|Wednesday|1955Q1|N|N|N|2435140|2435170|2434776|2435049|N|N|N|N|N| +2435142|AAAAAAAAGEICFCAA|1955-02-03|661|2875|221|1955|4|2|3|1|1955|221|2875|Thursday|1955Q1|N|N|N|2435140|2435170|2434777|2435050|N|N|N|N|N| +2435143|AAAAAAAAHEICFCAA|1955-02-04|661|2875|221|1955|5|2|4|1|1955|221|2875|Friday|1955Q1|N|Y|N|2435140|2435170|2434778|2435051|N|N|N|N|N| +2435144|AAAAAAAAIEICFCAA|1955-02-05|661|2875|221|1955|6|2|5|1|1955|221|2875|Saturday|1955Q1|N|Y|N|2435140|2435170|2434779|2435052|N|N|N|N|N| +2435145|AAAAAAAAJEICFCAA|1955-02-06|661|2875|221|1955|0|2|6|1|1955|221|2875|Sunday|1955Q1|N|N|N|2435140|2435170|2434780|2435053|N|N|N|N|N| +2435146|AAAAAAAAKEICFCAA|1955-02-07|661|2875|221|1955|1|2|7|1|1955|221|2875|Monday|1955Q1|N|N|N|2435140|2435170|2434781|2435054|N|N|N|N|N| +2435147|AAAAAAAALEICFCAA|1955-02-08|661|2876|221|1955|2|2|8|1|1955|221|2876|Tuesday|1955Q1|N|N|N|2435140|2435170|2434782|2435055|N|N|N|N|N| +2435148|AAAAAAAAMEICFCAA|1955-02-09|661|2876|221|1955|3|2|9|1|1955|221|2876|Wednesday|1955Q1|N|N|N|2435140|2435170|2434783|2435056|N|N|N|N|N| +2435149|AAAAAAAANEICFCAA|1955-02-10|661|2876|221|1955|4|2|10|1|1955|221|2876|Thursday|1955Q1|N|N|N|2435140|2435170|2434784|2435057|N|N|N|N|N| +2435150|AAAAAAAAOEICFCAA|1955-02-11|661|2876|221|1955|5|2|11|1|1955|221|2876|Friday|1955Q1|N|Y|N|2435140|2435170|2434785|2435058|N|N|N|N|N| +2435151|AAAAAAAAPEICFCAA|1955-02-12|661|2876|221|1955|6|2|12|1|1955|221|2876|Saturday|1955Q1|N|Y|N|2435140|2435170|2434786|2435059|N|N|N|N|N| +2435152|AAAAAAAAAFICFCAA|1955-02-13|661|2876|221|1955|0|2|13|1|1955|221|2876|Sunday|1955Q1|N|N|N|2435140|2435170|2434787|2435060|N|N|N|N|N| +2435153|AAAAAAAABFICFCAA|1955-02-14|661|2876|221|1955|1|2|14|1|1955|221|2876|Monday|1955Q1|N|N|N|2435140|2435170|2434788|2435061|N|N|N|N|N| +2435154|AAAAAAAACFICFCAA|1955-02-15|661|2877|221|1955|2|2|15|1|1955|221|2877|Tuesday|1955Q1|N|N|N|2435140|2435170|2434789|2435062|N|N|N|N|N| +2435155|AAAAAAAADFICFCAA|1955-02-16|661|2877|221|1955|3|2|16|1|1955|221|2877|Wednesday|1955Q1|N|N|N|2435140|2435170|2434790|2435063|N|N|N|N|N| +2435156|AAAAAAAAEFICFCAA|1955-02-17|661|2877|221|1955|4|2|17|1|1955|221|2877|Thursday|1955Q1|N|N|N|2435140|2435170|2434791|2435064|N|N|N|N|N| +2435157|AAAAAAAAFFICFCAA|1955-02-18|661|2877|221|1955|5|2|18|1|1955|221|2877|Friday|1955Q1|N|Y|N|2435140|2435170|2434792|2435065|N|N|N|N|N| +2435158|AAAAAAAAGFICFCAA|1955-02-19|661|2877|221|1955|6|2|19|1|1955|221|2877|Saturday|1955Q1|N|Y|N|2435140|2435170|2434793|2435066|N|N|N|N|N| +2435159|AAAAAAAAHFICFCAA|1955-02-20|661|2877|221|1955|0|2|20|1|1955|221|2877|Sunday|1955Q1|N|N|N|2435140|2435170|2434794|2435067|N|N|N|N|N| +2435160|AAAAAAAAIFICFCAA|1955-02-21|661|2877|221|1955|1|2|21|1|1955|221|2877|Monday|1955Q1|N|N|N|2435140|2435170|2434795|2435068|N|N|N|N|N| +2435161|AAAAAAAAJFICFCAA|1955-02-22|661|2878|221|1955|2|2|22|1|1955|221|2878|Tuesday|1955Q1|N|N|N|2435140|2435170|2434796|2435069|N|N|N|N|N| +2435162|AAAAAAAAKFICFCAA|1955-02-23|661|2878|221|1955|3|2|23|1|1955|221|2878|Wednesday|1955Q1|N|N|N|2435140|2435170|2434797|2435070|N|N|N|N|N| +2435163|AAAAAAAALFICFCAA|1955-02-24|661|2878|221|1955|4|2|24|1|1955|221|2878|Thursday|1955Q1|N|N|N|2435140|2435170|2434798|2435071|N|N|N|N|N| +2435164|AAAAAAAAMFICFCAA|1955-02-25|661|2878|221|1955|5|2|25|1|1955|221|2878|Friday|1955Q1|N|Y|N|2435140|2435170|2434799|2435072|N|N|N|N|N| +2435165|AAAAAAAANFICFCAA|1955-02-26|661|2878|221|1955|6|2|26|1|1955|221|2878|Saturday|1955Q1|N|Y|N|2435140|2435170|2434800|2435073|N|N|N|N|N| +2435166|AAAAAAAAOFICFCAA|1955-02-27|661|2878|221|1955|0|2|27|1|1955|221|2878|Sunday|1955Q1|N|N|N|2435140|2435170|2434801|2435074|N|N|N|N|N| +2435167|AAAAAAAAPFICFCAA|1955-02-28|661|2878|221|1955|1|2|28|1|1955|221|2878|Monday|1955Q1|N|N|N|2435140|2435170|2434802|2435075|N|N|N|N|N| +2435168|AAAAAAAAAGICFCAA|1955-03-01|662|2879|222|1955|2|3|1|1|1955|222|2879|Tuesday|1955Q1|N|N|N|2435168|2435226|2434803|2435076|N|N|N|N|N| +2435169|AAAAAAAABGICFCAA|1955-03-02|662|2879|222|1955|3|3|2|1|1955|222|2879|Wednesday|1955Q1|N|N|N|2435168|2435226|2434804|2435077|N|N|N|N|N| +2435170|AAAAAAAACGICFCAA|1955-03-03|662|2879|222|1955|4|3|3|1|1955|222|2879|Thursday|1955Q1|N|N|N|2435168|2435226|2434805|2435078|N|N|N|N|N| +2435171|AAAAAAAADGICFCAA|1955-03-04|662|2879|222|1955|5|3|4|1|1955|222|2879|Friday|1955Q1|N|Y|N|2435168|2435226|2434806|2435079|N|N|N|N|N| +2435172|AAAAAAAAEGICFCAA|1955-03-05|662|2879|222|1955|6|3|5|1|1955|222|2879|Saturday|1955Q1|N|Y|N|2435168|2435226|2434807|2435080|N|N|N|N|N| +2435173|AAAAAAAAFGICFCAA|1955-03-06|662|2879|222|1955|0|3|6|1|1955|222|2879|Sunday|1955Q1|N|N|N|2435168|2435226|2434808|2435081|N|N|N|N|N| +2435174|AAAAAAAAGGICFCAA|1955-03-07|662|2879|222|1955|1|3|7|1|1955|222|2879|Monday|1955Q1|N|N|N|2435168|2435226|2434809|2435082|N|N|N|N|N| +2435175|AAAAAAAAHGICFCAA|1955-03-08|662|2880|222|1955|2|3|8|1|1955|222|2880|Tuesday|1955Q1|N|N|N|2435168|2435226|2434810|2435083|N|N|N|N|N| +2435176|AAAAAAAAIGICFCAA|1955-03-09|662|2880|222|1955|3|3|9|1|1955|222|2880|Wednesday|1955Q1|N|N|N|2435168|2435226|2434811|2435084|N|N|N|N|N| +2435177|AAAAAAAAJGICFCAA|1955-03-10|662|2880|222|1955|4|3|10|1|1955|222|2880|Thursday|1955Q1|N|N|N|2435168|2435226|2434812|2435085|N|N|N|N|N| +2435178|AAAAAAAAKGICFCAA|1955-03-11|662|2880|222|1955|5|3|11|1|1955|222|2880|Friday|1955Q1|N|Y|N|2435168|2435226|2434813|2435086|N|N|N|N|N| +2435179|AAAAAAAALGICFCAA|1955-03-12|662|2880|222|1955|6|3|12|1|1955|222|2880|Saturday|1955Q1|N|Y|N|2435168|2435226|2434814|2435087|N|N|N|N|N| +2435180|AAAAAAAAMGICFCAA|1955-03-13|662|2880|222|1955|0|3|13|1|1955|222|2880|Sunday|1955Q1|N|N|N|2435168|2435226|2434815|2435088|N|N|N|N|N| +2435181|AAAAAAAANGICFCAA|1955-03-14|662|2880|222|1955|1|3|14|1|1955|222|2880|Monday|1955Q1|N|N|N|2435168|2435226|2434816|2435089|N|N|N|N|N| +2435182|AAAAAAAAOGICFCAA|1955-03-15|662|2881|222|1955|2|3|15|1|1955|222|2881|Tuesday|1955Q1|N|N|N|2435168|2435226|2434817|2435090|N|N|N|N|N| +2435183|AAAAAAAAPGICFCAA|1955-03-16|662|2881|222|1955|3|3|16|1|1955|222|2881|Wednesday|1955Q1|N|N|N|2435168|2435226|2434818|2435091|N|N|N|N|N| +2435184|AAAAAAAAAHICFCAA|1955-03-17|662|2881|222|1955|4|3|17|1|1955|222|2881|Thursday|1955Q1|N|N|N|2435168|2435226|2434819|2435092|N|N|N|N|N| +2435185|AAAAAAAABHICFCAA|1955-03-18|662|2881|222|1955|5|3|18|1|1955|222|2881|Friday|1955Q1|N|Y|N|2435168|2435226|2434820|2435093|N|N|N|N|N| +2435186|AAAAAAAACHICFCAA|1955-03-19|662|2881|222|1955|6|3|19|1|1955|222|2881|Saturday|1955Q1|N|Y|N|2435168|2435226|2434821|2435094|N|N|N|N|N| +2435187|AAAAAAAADHICFCAA|1955-03-20|662|2881|222|1955|0|3|20|1|1955|222|2881|Sunday|1955Q1|N|N|N|2435168|2435226|2434822|2435095|N|N|N|N|N| +2435188|AAAAAAAAEHICFCAA|1955-03-21|662|2881|222|1955|1|3|21|1|1955|222|2881|Monday|1955Q1|N|N|N|2435168|2435226|2434823|2435096|N|N|N|N|N| +2435189|AAAAAAAAFHICFCAA|1955-03-22|662|2882|222|1955|2|3|22|1|1955|222|2882|Tuesday|1955Q1|N|N|N|2435168|2435226|2434824|2435097|N|N|N|N|N| +2435190|AAAAAAAAGHICFCAA|1955-03-23|662|2882|222|1955|3|3|23|1|1955|222|2882|Wednesday|1955Q1|N|N|N|2435168|2435226|2434825|2435098|N|N|N|N|N| +2435191|AAAAAAAAHHICFCAA|1955-03-24|662|2882|222|1955|4|3|24|1|1955|222|2882|Thursday|1955Q1|N|N|N|2435168|2435226|2434826|2435099|N|N|N|N|N| +2435192|AAAAAAAAIHICFCAA|1955-03-25|662|2882|222|1955|5|3|25|1|1955|222|2882|Friday|1955Q1|N|Y|N|2435168|2435226|2434827|2435100|N|N|N|N|N| +2435193|AAAAAAAAJHICFCAA|1955-03-26|662|2882|222|1955|6|3|26|1|1955|222|2882|Saturday|1955Q1|N|Y|N|2435168|2435226|2434828|2435101|N|N|N|N|N| +2435194|AAAAAAAAKHICFCAA|1955-03-27|662|2882|222|1955|0|3|27|1|1955|222|2882|Sunday|1955Q1|N|N|N|2435168|2435226|2434829|2435102|N|N|N|N|N| +2435195|AAAAAAAALHICFCAA|1955-03-28|662|2882|222|1955|1|3|28|1|1955|222|2882|Monday|1955Q1|N|N|N|2435168|2435226|2434830|2435103|N|N|N|N|N| +2435196|AAAAAAAAMHICFCAA|1955-03-29|662|2883|222|1955|2|3|29|1|1955|222|2883|Tuesday|1955Q1|N|N|N|2435168|2435226|2434831|2435104|N|N|N|N|N| +2435197|AAAAAAAANHICFCAA|1955-03-30|662|2883|222|1955|3|3|30|1|1955|222|2883|Wednesday|1955Q1|N|N|N|2435168|2435226|2434832|2435105|N|N|N|N|N| +2435198|AAAAAAAAOHICFCAA|1955-03-31|662|2883|222|1955|4|3|31|1|1955|222|2883|Thursday|1955Q1|N|N|N|2435168|2435226|2434833|2435106|N|N|N|N|N| +2435199|AAAAAAAAPHICFCAA|1955-04-01|663|2883|222|1955|5|4|1|1|1955|222|2883|Friday|1955Q1|N|Y|N|2435199|2435288|2434834|2435109|N|N|N|N|N| +2435200|AAAAAAAAAIICFCAA|1955-04-02|663|2883|222|1955|6|4|2|2|1955|222|2883|Saturday|1955Q2|N|Y|N|2435199|2435288|2434835|2435110|N|N|N|N|N| +2435201|AAAAAAAABIICFCAA|1955-04-03|663|2883|222|1955|0|4|3|2|1955|222|2883|Sunday|1955Q2|N|N|N|2435199|2435288|2434836|2435111|N|N|N|N|N| +2435202|AAAAAAAACIICFCAA|1955-04-04|663|2883|222|1955|1|4|4|2|1955|222|2883|Monday|1955Q2|N|N|N|2435199|2435288|2434837|2435112|N|N|N|N|N| +2435203|AAAAAAAADIICFCAA|1955-04-05|663|2884|222|1955|2|4|5|2|1955|222|2884|Tuesday|1955Q2|N|N|N|2435199|2435288|2434838|2435113|N|N|N|N|N| +2435204|AAAAAAAAEIICFCAA|1955-04-06|663|2884|222|1955|3|4|6|2|1955|222|2884|Wednesday|1955Q2|N|N|N|2435199|2435288|2434839|2435114|N|N|N|N|N| +2435205|AAAAAAAAFIICFCAA|1955-04-07|663|2884|222|1955|4|4|7|2|1955|222|2884|Thursday|1955Q2|N|N|N|2435199|2435288|2434840|2435115|N|N|N|N|N| +2435206|AAAAAAAAGIICFCAA|1955-04-08|663|2884|222|1955|5|4|8|2|1955|222|2884|Friday|1955Q2|N|Y|N|2435199|2435288|2434841|2435116|N|N|N|N|N| +2435207|AAAAAAAAHIICFCAA|1955-04-09|663|2884|222|1955|6|4|9|2|1955|222|2884|Saturday|1955Q2|N|Y|N|2435199|2435288|2434842|2435117|N|N|N|N|N| +2435208|AAAAAAAAIIICFCAA|1955-04-10|663|2884|222|1955|0|4|10|2|1955|222|2884|Sunday|1955Q2|N|N|N|2435199|2435288|2434843|2435118|N|N|N|N|N| +2435209|AAAAAAAAJIICFCAA|1955-04-11|663|2884|222|1955|1|4|11|2|1955|222|2884|Monday|1955Q2|N|N|N|2435199|2435288|2434844|2435119|N|N|N|N|N| +2435210|AAAAAAAAKIICFCAA|1955-04-12|663|2885|222|1955|2|4|12|2|1955|222|2885|Tuesday|1955Q2|N|N|N|2435199|2435288|2434845|2435120|N|N|N|N|N| +2435211|AAAAAAAALIICFCAA|1955-04-13|663|2885|222|1955|3|4|13|2|1955|222|2885|Wednesday|1955Q2|N|N|N|2435199|2435288|2434846|2435121|N|N|N|N|N| +2435212|AAAAAAAAMIICFCAA|1955-04-14|663|2885|222|1955|4|4|14|2|1955|222|2885|Thursday|1955Q2|N|N|N|2435199|2435288|2434847|2435122|N|N|N|N|N| +2435213|AAAAAAAANIICFCAA|1955-04-15|663|2885|222|1955|5|4|15|2|1955|222|2885|Friday|1955Q2|N|Y|N|2435199|2435288|2434848|2435123|N|N|N|N|N| +2435214|AAAAAAAAOIICFCAA|1955-04-16|663|2885|222|1955|6|4|16|2|1955|222|2885|Saturday|1955Q2|N|Y|N|2435199|2435288|2434849|2435124|N|N|N|N|N| +2435215|AAAAAAAAPIICFCAA|1955-04-17|663|2885|222|1955|0|4|17|2|1955|222|2885|Sunday|1955Q2|N|N|N|2435199|2435288|2434850|2435125|N|N|N|N|N| +2435216|AAAAAAAAAJICFCAA|1955-04-18|663|2885|222|1955|1|4|18|2|1955|222|2885|Monday|1955Q2|N|N|N|2435199|2435288|2434851|2435126|N|N|N|N|N| +2435217|AAAAAAAABJICFCAA|1955-04-19|663|2886|222|1955|2|4|19|2|1955|222|2886|Tuesday|1955Q2|N|N|N|2435199|2435288|2434852|2435127|N|N|N|N|N| +2435218|AAAAAAAACJICFCAA|1955-04-20|663|2886|222|1955|3|4|20|2|1955|222|2886|Wednesday|1955Q2|N|N|N|2435199|2435288|2434853|2435128|N|N|N|N|N| +2435219|AAAAAAAADJICFCAA|1955-04-21|663|2886|222|1955|4|4|21|2|1955|222|2886|Thursday|1955Q2|N|N|N|2435199|2435288|2434854|2435129|N|N|N|N|N| +2435220|AAAAAAAAEJICFCAA|1955-04-22|663|2886|222|1955|5|4|22|2|1955|222|2886|Friday|1955Q2|N|Y|N|2435199|2435288|2434855|2435130|N|N|N|N|N| +2435221|AAAAAAAAFJICFCAA|1955-04-23|663|2886|222|1955|6|4|23|2|1955|222|2886|Saturday|1955Q2|N|Y|N|2435199|2435288|2434856|2435131|N|N|N|N|N| +2435222|AAAAAAAAGJICFCAA|1955-04-24|663|2886|222|1955|0|4|24|2|1955|222|2886|Sunday|1955Q2|N|N|N|2435199|2435288|2434857|2435132|N|N|N|N|N| +2435223|AAAAAAAAHJICFCAA|1955-04-25|663|2886|222|1955|1|4|25|2|1955|222|2886|Monday|1955Q2|N|N|N|2435199|2435288|2434858|2435133|N|N|N|N|N| +2435224|AAAAAAAAIJICFCAA|1955-04-26|663|2887|222|1955|2|4|26|2|1955|222|2887|Tuesday|1955Q2|N|N|N|2435199|2435288|2434859|2435134|N|N|N|N|N| +2435225|AAAAAAAAJJICFCAA|1955-04-27|663|2887|222|1955|3|4|27|2|1955|222|2887|Wednesday|1955Q2|N|N|N|2435199|2435288|2434860|2435135|N|N|N|N|N| +2435226|AAAAAAAAKJICFCAA|1955-04-28|663|2887|222|1955|4|4|28|2|1955|222|2887|Thursday|1955Q2|N|N|N|2435199|2435288|2434861|2435136|N|N|N|N|N| +2435227|AAAAAAAALJICFCAA|1955-04-29|663|2887|222|1955|5|4|29|2|1955|222|2887|Friday|1955Q2|N|Y|N|2435199|2435288|2434862|2435137|N|N|N|N|N| +2435228|AAAAAAAAMJICFCAA|1955-04-30|663|2887|222|1955|6|4|30|2|1955|222|2887|Saturday|1955Q2|N|Y|N|2435199|2435288|2434863|2435138|N|N|N|N|N| +2435229|AAAAAAAANJICFCAA|1955-05-01|664|2887|222|1955|0|5|1|2|1955|222|2887|Sunday|1955Q2|N|N|N|2435229|2435348|2434864|2435139|N|N|N|N|N| +2435230|AAAAAAAAOJICFCAA|1955-05-02|664|2887|222|1955|1|5|2|2|1955|222|2887|Monday|1955Q2|N|N|N|2435229|2435348|2434865|2435140|N|N|N|N|N| +2435231|AAAAAAAAPJICFCAA|1955-05-03|664|2888|222|1955|2|5|3|2|1955|222|2888|Tuesday|1955Q2|N|N|N|2435229|2435348|2434866|2435141|N|N|N|N|N| +2435232|AAAAAAAAAKICFCAA|1955-05-04|664|2888|222|1955|3|5|4|2|1955|222|2888|Wednesday|1955Q2|N|N|N|2435229|2435348|2434867|2435142|N|N|N|N|N| +2435233|AAAAAAAABKICFCAA|1955-05-05|664|2888|222|1955|4|5|5|2|1955|222|2888|Thursday|1955Q2|N|N|N|2435229|2435348|2434868|2435143|N|N|N|N|N| +2435234|AAAAAAAACKICFCAA|1955-05-06|664|2888|222|1955|5|5|6|2|1955|222|2888|Friday|1955Q2|N|Y|N|2435229|2435348|2434869|2435144|N|N|N|N|N| +2435235|AAAAAAAADKICFCAA|1955-05-07|664|2888|222|1955|6|5|7|2|1955|222|2888|Saturday|1955Q2|N|Y|N|2435229|2435348|2434870|2435145|N|N|N|N|N| +2435236|AAAAAAAAEKICFCAA|1955-05-08|664|2888|222|1955|0|5|8|2|1955|222|2888|Sunday|1955Q2|N|N|N|2435229|2435348|2434871|2435146|N|N|N|N|N| +2435237|AAAAAAAAFKICFCAA|1955-05-09|664|2888|222|1955|1|5|9|2|1955|222|2888|Monday|1955Q2|N|N|N|2435229|2435348|2434872|2435147|N|N|N|N|N| +2435238|AAAAAAAAGKICFCAA|1955-05-10|664|2889|222|1955|2|5|10|2|1955|222|2889|Tuesday|1955Q2|N|N|N|2435229|2435348|2434873|2435148|N|N|N|N|N| +2435239|AAAAAAAAHKICFCAA|1955-05-11|664|2889|222|1955|3|5|11|2|1955|222|2889|Wednesday|1955Q2|N|N|N|2435229|2435348|2434874|2435149|N|N|N|N|N| +2435240|AAAAAAAAIKICFCAA|1955-05-12|664|2889|222|1955|4|5|12|2|1955|222|2889|Thursday|1955Q2|N|N|N|2435229|2435348|2434875|2435150|N|N|N|N|N| +2435241|AAAAAAAAJKICFCAA|1955-05-13|664|2889|222|1955|5|5|13|2|1955|222|2889|Friday|1955Q2|N|Y|N|2435229|2435348|2434876|2435151|N|N|N|N|N| +2435242|AAAAAAAAKKICFCAA|1955-05-14|664|2889|222|1955|6|5|14|2|1955|222|2889|Saturday|1955Q2|N|Y|N|2435229|2435348|2434877|2435152|N|N|N|N|N| +2435243|AAAAAAAALKICFCAA|1955-05-15|664|2889|222|1955|0|5|15|2|1955|222|2889|Sunday|1955Q2|N|N|N|2435229|2435348|2434878|2435153|N|N|N|N|N| +2435244|AAAAAAAAMKICFCAA|1955-05-16|664|2889|222|1955|1|5|16|2|1955|222|2889|Monday|1955Q2|N|N|N|2435229|2435348|2434879|2435154|N|N|N|N|N| +2435245|AAAAAAAANKICFCAA|1955-05-17|664|2890|222|1955|2|5|17|2|1955|222|2890|Tuesday|1955Q2|N|N|N|2435229|2435348|2434880|2435155|N|N|N|N|N| +2435246|AAAAAAAAOKICFCAA|1955-05-18|664|2890|222|1955|3|5|18|2|1955|222|2890|Wednesday|1955Q2|N|N|N|2435229|2435348|2434881|2435156|N|N|N|N|N| +2435247|AAAAAAAAPKICFCAA|1955-05-19|664|2890|222|1955|4|5|19|2|1955|222|2890|Thursday|1955Q2|N|N|N|2435229|2435348|2434882|2435157|N|N|N|N|N| +2435248|AAAAAAAAALICFCAA|1955-05-20|664|2890|222|1955|5|5|20|2|1955|222|2890|Friday|1955Q2|N|Y|N|2435229|2435348|2434883|2435158|N|N|N|N|N| +2435249|AAAAAAAABLICFCAA|1955-05-21|664|2890|222|1955|6|5|21|2|1955|222|2890|Saturday|1955Q2|N|Y|N|2435229|2435348|2434884|2435159|N|N|N|N|N| +2435250|AAAAAAAACLICFCAA|1955-05-22|664|2890|222|1955|0|5|22|2|1955|222|2890|Sunday|1955Q2|N|N|N|2435229|2435348|2434885|2435160|N|N|N|N|N| +2435251|AAAAAAAADLICFCAA|1955-05-23|664|2890|222|1955|1|5|23|2|1955|222|2890|Monday|1955Q2|N|N|N|2435229|2435348|2434886|2435161|N|N|N|N|N| +2435252|AAAAAAAAELICFCAA|1955-05-24|664|2891|222|1955|2|5|24|2|1955|222|2891|Tuesday|1955Q2|N|N|N|2435229|2435348|2434887|2435162|N|N|N|N|N| +2435253|AAAAAAAAFLICFCAA|1955-05-25|664|2891|222|1955|3|5|25|2|1955|222|2891|Wednesday|1955Q2|N|N|N|2435229|2435348|2434888|2435163|N|N|N|N|N| +2435254|AAAAAAAAGLICFCAA|1955-05-26|664|2891|222|1955|4|5|26|2|1955|222|2891|Thursday|1955Q2|N|N|N|2435229|2435348|2434889|2435164|N|N|N|N|N| +2435255|AAAAAAAAHLICFCAA|1955-05-27|664|2891|222|1955|5|5|27|2|1955|222|2891|Friday|1955Q2|N|Y|N|2435229|2435348|2434890|2435165|N|N|N|N|N| +2435256|AAAAAAAAILICFCAA|1955-05-28|664|2891|222|1955|6|5|28|2|1955|222|2891|Saturday|1955Q2|N|Y|N|2435229|2435348|2434891|2435166|N|N|N|N|N| +2435257|AAAAAAAAJLICFCAA|1955-05-29|664|2891|222|1955|0|5|29|2|1955|222|2891|Sunday|1955Q2|N|N|N|2435229|2435348|2434892|2435167|N|N|N|N|N| +2435258|AAAAAAAAKLICFCAA|1955-05-30|664|2891|222|1955|1|5|30|2|1955|222|2891|Monday|1955Q2|N|N|N|2435229|2435348|2434893|2435168|N|N|N|N|N| +2435259|AAAAAAAALLICFCAA|1955-05-31|664|2892|222|1955|2|5|31|2|1955|222|2892|Tuesday|1955Q2|N|N|N|2435229|2435348|2434894|2435169|N|N|N|N|N| +2435260|AAAAAAAAMLICFCAA|1955-06-01|665|2892|223|1955|3|6|1|2|1955|223|2892|Wednesday|1955Q2|N|N|N|2435260|2435410|2434895|2435170|N|N|N|N|N| +2435261|AAAAAAAANLICFCAA|1955-06-02|665|2892|223|1955|4|6|2|2|1955|223|2892|Thursday|1955Q2|N|N|N|2435260|2435410|2434896|2435171|N|N|N|N|N| +2435262|AAAAAAAAOLICFCAA|1955-06-03|665|2892|223|1955|5|6|3|2|1955|223|2892|Friday|1955Q2|N|Y|N|2435260|2435410|2434897|2435172|N|N|N|N|N| +2435263|AAAAAAAAPLICFCAA|1955-06-04|665|2892|223|1955|6|6|4|2|1955|223|2892|Saturday|1955Q2|N|Y|N|2435260|2435410|2434898|2435173|N|N|N|N|N| +2435264|AAAAAAAAAMICFCAA|1955-06-05|665|2892|223|1955|0|6|5|2|1955|223|2892|Sunday|1955Q2|N|N|N|2435260|2435410|2434899|2435174|N|N|N|N|N| +2435265|AAAAAAAABMICFCAA|1955-06-06|665|2892|223|1955|1|6|6|2|1955|223|2892|Monday|1955Q2|N|N|N|2435260|2435410|2434900|2435175|N|N|N|N|N| +2435266|AAAAAAAACMICFCAA|1955-06-07|665|2893|223|1955|2|6|7|2|1955|223|2893|Tuesday|1955Q2|N|N|N|2435260|2435410|2434901|2435176|N|N|N|N|N| +2435267|AAAAAAAADMICFCAA|1955-06-08|665|2893|223|1955|3|6|8|2|1955|223|2893|Wednesday|1955Q2|N|N|N|2435260|2435410|2434902|2435177|N|N|N|N|N| +2435268|AAAAAAAAEMICFCAA|1955-06-09|665|2893|223|1955|4|6|9|2|1955|223|2893|Thursday|1955Q2|N|N|N|2435260|2435410|2434903|2435178|N|N|N|N|N| +2435269|AAAAAAAAFMICFCAA|1955-06-10|665|2893|223|1955|5|6|10|2|1955|223|2893|Friday|1955Q2|N|Y|N|2435260|2435410|2434904|2435179|N|N|N|N|N| +2435270|AAAAAAAAGMICFCAA|1955-06-11|665|2893|223|1955|6|6|11|2|1955|223|2893|Saturday|1955Q2|N|Y|N|2435260|2435410|2434905|2435180|N|N|N|N|N| +2435271|AAAAAAAAHMICFCAA|1955-06-12|665|2893|223|1955|0|6|12|2|1955|223|2893|Sunday|1955Q2|N|N|N|2435260|2435410|2434906|2435181|N|N|N|N|N| +2435272|AAAAAAAAIMICFCAA|1955-06-13|665|2893|223|1955|1|6|13|2|1955|223|2893|Monday|1955Q2|N|N|N|2435260|2435410|2434907|2435182|N|N|N|N|N| +2435273|AAAAAAAAJMICFCAA|1955-06-14|665|2894|223|1955|2|6|14|2|1955|223|2894|Tuesday|1955Q2|N|N|N|2435260|2435410|2434908|2435183|N|N|N|N|N| +2435274|AAAAAAAAKMICFCAA|1955-06-15|665|2894|223|1955|3|6|15|2|1955|223|2894|Wednesday|1955Q2|N|N|N|2435260|2435410|2434909|2435184|N|N|N|N|N| +2435275|AAAAAAAALMICFCAA|1955-06-16|665|2894|223|1955|4|6|16|2|1955|223|2894|Thursday|1955Q2|N|N|N|2435260|2435410|2434910|2435185|N|N|N|N|N| +2435276|AAAAAAAAMMICFCAA|1955-06-17|665|2894|223|1955|5|6|17|2|1955|223|2894|Friday|1955Q2|N|Y|N|2435260|2435410|2434911|2435186|N|N|N|N|N| +2435277|AAAAAAAANMICFCAA|1955-06-18|665|2894|223|1955|6|6|18|2|1955|223|2894|Saturday|1955Q2|N|Y|N|2435260|2435410|2434912|2435187|N|N|N|N|N| +2435278|AAAAAAAAOMICFCAA|1955-06-19|665|2894|223|1955|0|6|19|2|1955|223|2894|Sunday|1955Q2|N|N|N|2435260|2435410|2434913|2435188|N|N|N|N|N| +2435279|AAAAAAAAPMICFCAA|1955-06-20|665|2894|223|1955|1|6|20|2|1955|223|2894|Monday|1955Q2|N|N|N|2435260|2435410|2434914|2435189|N|N|N|N|N| +2435280|AAAAAAAAANICFCAA|1955-06-21|665|2895|223|1955|2|6|21|2|1955|223|2895|Tuesday|1955Q2|N|N|N|2435260|2435410|2434915|2435190|N|N|N|N|N| +2435281|AAAAAAAABNICFCAA|1955-06-22|665|2895|223|1955|3|6|22|2|1955|223|2895|Wednesday|1955Q2|N|N|N|2435260|2435410|2434916|2435191|N|N|N|N|N| +2435282|AAAAAAAACNICFCAA|1955-06-23|665|2895|223|1955|4|6|23|2|1955|223|2895|Thursday|1955Q2|N|N|N|2435260|2435410|2434917|2435192|N|N|N|N|N| +2435283|AAAAAAAADNICFCAA|1955-06-24|665|2895|223|1955|5|6|24|2|1955|223|2895|Friday|1955Q2|N|Y|N|2435260|2435410|2434918|2435193|N|N|N|N|N| +2435284|AAAAAAAAENICFCAA|1955-06-25|665|2895|223|1955|6|6|25|2|1955|223|2895|Saturday|1955Q2|N|Y|N|2435260|2435410|2434919|2435194|N|N|N|N|N| +2435285|AAAAAAAAFNICFCAA|1955-06-26|665|2895|223|1955|0|6|26|2|1955|223|2895|Sunday|1955Q2|N|N|N|2435260|2435410|2434920|2435195|N|N|N|N|N| +2435286|AAAAAAAAGNICFCAA|1955-06-27|665|2895|223|1955|1|6|27|2|1955|223|2895|Monday|1955Q2|N|N|N|2435260|2435410|2434921|2435196|N|N|N|N|N| +2435287|AAAAAAAAHNICFCAA|1955-06-28|665|2896|223|1955|2|6|28|2|1955|223|2896|Tuesday|1955Q2|N|N|N|2435260|2435410|2434922|2435197|N|N|N|N|N| +2435288|AAAAAAAAINICFCAA|1955-06-29|665|2896|223|1955|3|6|29|2|1955|223|2896|Wednesday|1955Q2|N|N|N|2435260|2435410|2434923|2435198|N|N|N|N|N| +2435289|AAAAAAAAJNICFCAA|1955-06-30|665|2896|223|1955|4|6|30|2|1955|223|2896|Thursday|1955Q2|N|N|N|2435260|2435410|2434924|2435199|N|N|N|N|N| +2435290|AAAAAAAAKNICFCAA|1955-07-01|666|2896|223|1955|5|7|1|2|1955|223|2896|Friday|1955Q2|N|Y|N|2435290|2435470|2434925|2435199|N|N|N|N|N| +2435291|AAAAAAAALNICFCAA|1955-07-02|666|2896|223|1955|6|7|2|3|1955|223|2896|Saturday|1955Q3|N|Y|N|2435290|2435470|2434926|2435200|N|N|N|N|N| +2435292|AAAAAAAAMNICFCAA|1955-07-03|666|2896|223|1955|0|7|3|3|1955|223|2896|Sunday|1955Q3|N|N|N|2435290|2435470|2434927|2435201|N|N|N|N|N| +2435293|AAAAAAAANNICFCAA|1955-07-04|666|2896|223|1955|1|7|4|3|1955|223|2896|Monday|1955Q3|N|N|N|2435290|2435470|2434928|2435202|N|N|N|N|N| +2435294|AAAAAAAAONICFCAA|1955-07-05|666|2897|223|1955|2|7|5|3|1955|223|2897|Tuesday|1955Q3|Y|N|N|2435290|2435470|2434929|2435203|N|N|N|N|N| +2435295|AAAAAAAAPNICFCAA|1955-07-06|666|2897|223|1955|3|7|6|3|1955|223|2897|Wednesday|1955Q3|N|N|Y|2435290|2435470|2434930|2435204|N|N|N|N|N| +2435296|AAAAAAAAAOICFCAA|1955-07-07|666|2897|223|1955|4|7|7|3|1955|223|2897|Thursday|1955Q3|N|N|N|2435290|2435470|2434931|2435205|N|N|N|N|N| +2435297|AAAAAAAABOICFCAA|1955-07-08|666|2897|223|1955|5|7|8|3|1955|223|2897|Friday|1955Q3|N|Y|N|2435290|2435470|2434932|2435206|N|N|N|N|N| +2435298|AAAAAAAACOICFCAA|1955-07-09|666|2897|223|1955|6|7|9|3|1955|223|2897|Saturday|1955Q3|N|Y|N|2435290|2435470|2434933|2435207|N|N|N|N|N| +2435299|AAAAAAAADOICFCAA|1955-07-10|666|2897|223|1955|0|7|10|3|1955|223|2897|Sunday|1955Q3|N|N|N|2435290|2435470|2434934|2435208|N|N|N|N|N| +2435300|AAAAAAAAEOICFCAA|1955-07-11|666|2897|223|1955|1|7|11|3|1955|223|2897|Monday|1955Q3|N|N|N|2435290|2435470|2434935|2435209|N|N|N|N|N| +2435301|AAAAAAAAFOICFCAA|1955-07-12|666|2898|223|1955|2|7|12|3|1955|223|2898|Tuesday|1955Q3|N|N|N|2435290|2435470|2434936|2435210|N|N|N|N|N| +2435302|AAAAAAAAGOICFCAA|1955-07-13|666|2898|223|1955|3|7|13|3|1955|223|2898|Wednesday|1955Q3|N|N|N|2435290|2435470|2434937|2435211|N|N|N|N|N| +2435303|AAAAAAAAHOICFCAA|1955-07-14|666|2898|223|1955|4|7|14|3|1955|223|2898|Thursday|1955Q3|N|N|N|2435290|2435470|2434938|2435212|N|N|N|N|N| +2435304|AAAAAAAAIOICFCAA|1955-07-15|666|2898|223|1955|5|7|15|3|1955|223|2898|Friday|1955Q3|N|Y|N|2435290|2435470|2434939|2435213|N|N|N|N|N| +2435305|AAAAAAAAJOICFCAA|1955-07-16|666|2898|223|1955|6|7|16|3|1955|223|2898|Saturday|1955Q3|N|Y|N|2435290|2435470|2434940|2435214|N|N|N|N|N| +2435306|AAAAAAAAKOICFCAA|1955-07-17|666|2898|223|1955|0|7|17|3|1955|223|2898|Sunday|1955Q3|N|N|N|2435290|2435470|2434941|2435215|N|N|N|N|N| +2435307|AAAAAAAALOICFCAA|1955-07-18|666|2898|223|1955|1|7|18|3|1955|223|2898|Monday|1955Q3|N|N|N|2435290|2435470|2434942|2435216|N|N|N|N|N| +2435308|AAAAAAAAMOICFCAA|1955-07-19|666|2899|223|1955|2|7|19|3|1955|223|2899|Tuesday|1955Q3|N|N|N|2435290|2435470|2434943|2435217|N|N|N|N|N| +2435309|AAAAAAAANOICFCAA|1955-07-20|666|2899|223|1955|3|7|20|3|1955|223|2899|Wednesday|1955Q3|N|N|N|2435290|2435470|2434944|2435218|N|N|N|N|N| +2435310|AAAAAAAAOOICFCAA|1955-07-21|666|2899|223|1955|4|7|21|3|1955|223|2899|Thursday|1955Q3|N|N|N|2435290|2435470|2434945|2435219|N|N|N|N|N| +2435311|AAAAAAAAPOICFCAA|1955-07-22|666|2899|223|1955|5|7|22|3|1955|223|2899|Friday|1955Q3|N|Y|N|2435290|2435470|2434946|2435220|N|N|N|N|N| +2435312|AAAAAAAAAPICFCAA|1955-07-23|666|2899|223|1955|6|7|23|3|1955|223|2899|Saturday|1955Q3|N|Y|N|2435290|2435470|2434947|2435221|N|N|N|N|N| +2435313|AAAAAAAABPICFCAA|1955-07-24|666|2899|223|1955|0|7|24|3|1955|223|2899|Sunday|1955Q3|N|N|N|2435290|2435470|2434948|2435222|N|N|N|N|N| +2435314|AAAAAAAACPICFCAA|1955-07-25|666|2899|223|1955|1|7|25|3|1955|223|2899|Monday|1955Q3|N|N|N|2435290|2435470|2434949|2435223|N|N|N|N|N| +2435315|AAAAAAAADPICFCAA|1955-07-26|666|2900|223|1955|2|7|26|3|1955|223|2900|Tuesday|1955Q3|N|N|N|2435290|2435470|2434950|2435224|N|N|N|N|N| +2435316|AAAAAAAAEPICFCAA|1955-07-27|666|2900|223|1955|3|7|27|3|1955|223|2900|Wednesday|1955Q3|N|N|N|2435290|2435470|2434951|2435225|N|N|N|N|N| +2435317|AAAAAAAAFPICFCAA|1955-07-28|666|2900|223|1955|4|7|28|3|1955|223|2900|Thursday|1955Q3|N|N|N|2435290|2435470|2434952|2435226|N|N|N|N|N| +2435318|AAAAAAAAGPICFCAA|1955-07-29|666|2900|223|1955|5|7|29|3|1955|223|2900|Friday|1955Q3|N|Y|N|2435290|2435470|2434953|2435227|N|N|N|N|N| +2435319|AAAAAAAAHPICFCAA|1955-07-30|666|2900|223|1955|6|7|30|3|1955|223|2900|Saturday|1955Q3|N|Y|N|2435290|2435470|2434954|2435228|N|N|N|N|N| +2435320|AAAAAAAAIPICFCAA|1955-07-31|666|2900|223|1955|0|7|31|3|1955|223|2900|Sunday|1955Q3|N|N|N|2435290|2435470|2434955|2435229|N|N|N|N|N| +2435321|AAAAAAAAJPICFCAA|1955-08-01|667|2900|223|1955|1|8|1|3|1955|223|2900|Monday|1955Q3|N|N|N|2435321|2435532|2434956|2435230|N|N|N|N|N| +2435322|AAAAAAAAKPICFCAA|1955-08-02|667|2901|223|1955|2|8|2|3|1955|223|2901|Tuesday|1955Q3|N|N|N|2435321|2435532|2434957|2435231|N|N|N|N|N| +2435323|AAAAAAAALPICFCAA|1955-08-03|667|2901|223|1955|3|8|3|3|1955|223|2901|Wednesday|1955Q3|N|N|N|2435321|2435532|2434958|2435232|N|N|N|N|N| +2435324|AAAAAAAAMPICFCAA|1955-08-04|667|2901|223|1955|4|8|4|3|1955|223|2901|Thursday|1955Q3|N|N|N|2435321|2435532|2434959|2435233|N|N|N|N|N| +2435325|AAAAAAAANPICFCAA|1955-08-05|667|2901|223|1955|5|8|5|3|1955|223|2901|Friday|1955Q3|N|Y|N|2435321|2435532|2434960|2435234|N|N|N|N|N| +2435326|AAAAAAAAOPICFCAA|1955-08-06|667|2901|223|1955|6|8|6|3|1955|223|2901|Saturday|1955Q3|N|Y|N|2435321|2435532|2434961|2435235|N|N|N|N|N| +2435327|AAAAAAAAPPICFCAA|1955-08-07|667|2901|223|1955|0|8|7|3|1955|223|2901|Sunday|1955Q3|N|N|N|2435321|2435532|2434962|2435236|N|N|N|N|N| +2435328|AAAAAAAAAAJCFCAA|1955-08-08|667|2901|223|1955|1|8|8|3|1955|223|2901|Monday|1955Q3|N|N|N|2435321|2435532|2434963|2435237|N|N|N|N|N| +2435329|AAAAAAAABAJCFCAA|1955-08-09|667|2902|223|1955|2|8|9|3|1955|223|2902|Tuesday|1955Q3|N|N|N|2435321|2435532|2434964|2435238|N|N|N|N|N| +2435330|AAAAAAAACAJCFCAA|1955-08-10|667|2902|223|1955|3|8|10|3|1955|223|2902|Wednesday|1955Q3|N|N|N|2435321|2435532|2434965|2435239|N|N|N|N|N| +2435331|AAAAAAAADAJCFCAA|1955-08-11|667|2902|223|1955|4|8|11|3|1955|223|2902|Thursday|1955Q3|N|N|N|2435321|2435532|2434966|2435240|N|N|N|N|N| +2435332|AAAAAAAAEAJCFCAA|1955-08-12|667|2902|223|1955|5|8|12|3|1955|223|2902|Friday|1955Q3|N|Y|N|2435321|2435532|2434967|2435241|N|N|N|N|N| +2435333|AAAAAAAAFAJCFCAA|1955-08-13|667|2902|223|1955|6|8|13|3|1955|223|2902|Saturday|1955Q3|N|Y|N|2435321|2435532|2434968|2435242|N|N|N|N|N| +2435334|AAAAAAAAGAJCFCAA|1955-08-14|667|2902|223|1955|0|8|14|3|1955|223|2902|Sunday|1955Q3|N|N|N|2435321|2435532|2434969|2435243|N|N|N|N|N| +2435335|AAAAAAAAHAJCFCAA|1955-08-15|667|2902|223|1955|1|8|15|3|1955|223|2902|Monday|1955Q3|N|N|N|2435321|2435532|2434970|2435244|N|N|N|N|N| +2435336|AAAAAAAAIAJCFCAA|1955-08-16|667|2903|223|1955|2|8|16|3|1955|223|2903|Tuesday|1955Q3|N|N|N|2435321|2435532|2434971|2435245|N|N|N|N|N| +2435337|AAAAAAAAJAJCFCAA|1955-08-17|667|2903|223|1955|3|8|17|3|1955|223|2903|Wednesday|1955Q3|N|N|N|2435321|2435532|2434972|2435246|N|N|N|N|N| +2435338|AAAAAAAAKAJCFCAA|1955-08-18|667|2903|223|1955|4|8|18|3|1955|223|2903|Thursday|1955Q3|N|N|N|2435321|2435532|2434973|2435247|N|N|N|N|N| +2435339|AAAAAAAALAJCFCAA|1955-08-19|667|2903|223|1955|5|8|19|3|1955|223|2903|Friday|1955Q3|N|Y|N|2435321|2435532|2434974|2435248|N|N|N|N|N| +2435340|AAAAAAAAMAJCFCAA|1955-08-20|667|2903|223|1955|6|8|20|3|1955|223|2903|Saturday|1955Q3|N|Y|N|2435321|2435532|2434975|2435249|N|N|N|N|N| +2435341|AAAAAAAANAJCFCAA|1955-08-21|667|2903|223|1955|0|8|21|3|1955|223|2903|Sunday|1955Q3|N|N|N|2435321|2435532|2434976|2435250|N|N|N|N|N| +2435342|AAAAAAAAOAJCFCAA|1955-08-22|667|2903|223|1955|1|8|22|3|1955|223|2903|Monday|1955Q3|N|N|N|2435321|2435532|2434977|2435251|N|N|N|N|N| +2435343|AAAAAAAAPAJCFCAA|1955-08-23|667|2904|223|1955|2|8|23|3|1955|223|2904|Tuesday|1955Q3|N|N|N|2435321|2435532|2434978|2435252|N|N|N|N|N| +2435344|AAAAAAAAABJCFCAA|1955-08-24|667|2904|223|1955|3|8|24|3|1955|223|2904|Wednesday|1955Q3|N|N|N|2435321|2435532|2434979|2435253|N|N|N|N|N| +2435345|AAAAAAAABBJCFCAA|1955-08-25|667|2904|223|1955|4|8|25|3|1955|223|2904|Thursday|1955Q3|N|N|N|2435321|2435532|2434980|2435254|N|N|N|N|N| +2435346|AAAAAAAACBJCFCAA|1955-08-26|667|2904|223|1955|5|8|26|3|1955|223|2904|Friday|1955Q3|N|Y|N|2435321|2435532|2434981|2435255|N|N|N|N|N| +2435347|AAAAAAAADBJCFCAA|1955-08-27|667|2904|223|1955|6|8|27|3|1955|223|2904|Saturday|1955Q3|N|Y|N|2435321|2435532|2434982|2435256|N|N|N|N|N| +2435348|AAAAAAAAEBJCFCAA|1955-08-28|667|2904|223|1955|0|8|28|3|1955|223|2904|Sunday|1955Q3|N|N|N|2435321|2435532|2434983|2435257|N|N|N|N|N| +2435349|AAAAAAAAFBJCFCAA|1955-08-29|667|2904|223|1955|1|8|29|3|1955|223|2904|Monday|1955Q3|N|N|N|2435321|2435532|2434984|2435258|N|N|N|N|N| +2435350|AAAAAAAAGBJCFCAA|1955-08-30|667|2905|223|1955|2|8|30|3|1955|223|2905|Tuesday|1955Q3|N|N|N|2435321|2435532|2434985|2435259|N|N|N|N|N| +2435351|AAAAAAAAHBJCFCAA|1955-08-31|667|2905|223|1955|3|8|31|3|1955|223|2905|Wednesday|1955Q3|N|N|N|2435321|2435532|2434986|2435260|N|N|N|N|N| +2435352|AAAAAAAAIBJCFCAA|1955-09-01|668|2905|224|1955|4|9|1|3|1955|224|2905|Thursday|1955Q3|N|N|N|2435352|2435594|2434987|2435261|N|N|N|N|N| +2435353|AAAAAAAAJBJCFCAA|1955-09-02|668|2905|224|1955|5|9|2|3|1955|224|2905|Friday|1955Q3|N|Y|N|2435352|2435594|2434988|2435262|N|N|N|N|N| +2435354|AAAAAAAAKBJCFCAA|1955-09-03|668|2905|224|1955|6|9|3|3|1955|224|2905|Saturday|1955Q3|N|Y|N|2435352|2435594|2434989|2435263|N|N|N|N|N| +2435355|AAAAAAAALBJCFCAA|1955-09-04|668|2905|224|1955|0|9|4|3|1955|224|2905|Sunday|1955Q3|N|N|N|2435352|2435594|2434990|2435264|N|N|N|N|N| +2435356|AAAAAAAAMBJCFCAA|1955-09-05|668|2905|224|1955|1|9|5|3|1955|224|2905|Monday|1955Q3|N|N|N|2435352|2435594|2434991|2435265|N|N|N|N|N| +2435357|AAAAAAAANBJCFCAA|1955-09-06|668|2906|224|1955|2|9|6|3|1955|224|2906|Tuesday|1955Q3|N|N|N|2435352|2435594|2434992|2435266|N|N|N|N|N| +2435358|AAAAAAAAOBJCFCAA|1955-09-07|668|2906|224|1955|3|9|7|3|1955|224|2906|Wednesday|1955Q3|N|N|N|2435352|2435594|2434993|2435267|N|N|N|N|N| +2435359|AAAAAAAAPBJCFCAA|1955-09-08|668|2906|224|1955|4|9|8|3|1955|224|2906|Thursday|1955Q3|N|N|N|2435352|2435594|2434994|2435268|N|N|N|N|N| +2435360|AAAAAAAAACJCFCAA|1955-09-09|668|2906|224|1955|5|9|9|3|1955|224|2906|Friday|1955Q3|N|Y|N|2435352|2435594|2434995|2435269|N|N|N|N|N| +2435361|AAAAAAAABCJCFCAA|1955-09-10|668|2906|224|1955|6|9|10|3|1955|224|2906|Saturday|1955Q3|N|Y|N|2435352|2435594|2434996|2435270|N|N|N|N|N| +2435362|AAAAAAAACCJCFCAA|1955-09-11|668|2906|224|1955|0|9|11|3|1955|224|2906|Sunday|1955Q3|N|N|N|2435352|2435594|2434997|2435271|N|N|N|N|N| +2435363|AAAAAAAADCJCFCAA|1955-09-12|668|2906|224|1955|1|9|12|3|1955|224|2906|Monday|1955Q3|N|N|N|2435352|2435594|2434998|2435272|N|N|N|N|N| +2435364|AAAAAAAAECJCFCAA|1955-09-13|668|2907|224|1955|2|9|13|3|1955|224|2907|Tuesday|1955Q3|N|N|N|2435352|2435594|2434999|2435273|N|N|N|N|N| +2435365|AAAAAAAAFCJCFCAA|1955-09-14|668|2907|224|1955|3|9|14|3|1955|224|2907|Wednesday|1955Q3|N|N|N|2435352|2435594|2435000|2435274|N|N|N|N|N| +2435366|AAAAAAAAGCJCFCAA|1955-09-15|668|2907|224|1955|4|9|15|3|1955|224|2907|Thursday|1955Q3|N|N|N|2435352|2435594|2435001|2435275|N|N|N|N|N| +2435367|AAAAAAAAHCJCFCAA|1955-09-16|668|2907|224|1955|5|9|16|3|1955|224|2907|Friday|1955Q3|N|Y|N|2435352|2435594|2435002|2435276|N|N|N|N|N| +2435368|AAAAAAAAICJCFCAA|1955-09-17|668|2907|224|1955|6|9|17|3|1955|224|2907|Saturday|1955Q3|N|Y|N|2435352|2435594|2435003|2435277|N|N|N|N|N| +2435369|AAAAAAAAJCJCFCAA|1955-09-18|668|2907|224|1955|0|9|18|3|1955|224|2907|Sunday|1955Q3|N|N|N|2435352|2435594|2435004|2435278|N|N|N|N|N| +2435370|AAAAAAAAKCJCFCAA|1955-09-19|668|2907|224|1955|1|9|19|3|1955|224|2907|Monday|1955Q3|N|N|N|2435352|2435594|2435005|2435279|N|N|N|N|N| +2435371|AAAAAAAALCJCFCAA|1955-09-20|668|2908|224|1955|2|9|20|3|1955|224|2908|Tuesday|1955Q3|N|N|N|2435352|2435594|2435006|2435280|N|N|N|N|N| +2435372|AAAAAAAAMCJCFCAA|1955-09-21|668|2908|224|1955|3|9|21|3|1955|224|2908|Wednesday|1955Q3|N|N|N|2435352|2435594|2435007|2435281|N|N|N|N|N| +2435373|AAAAAAAANCJCFCAA|1955-09-22|668|2908|224|1955|4|9|22|3|1955|224|2908|Thursday|1955Q3|N|N|N|2435352|2435594|2435008|2435282|N|N|N|N|N| +2435374|AAAAAAAAOCJCFCAA|1955-09-23|668|2908|224|1955|5|9|23|3|1955|224|2908|Friday|1955Q3|N|Y|N|2435352|2435594|2435009|2435283|N|N|N|N|N| +2435375|AAAAAAAAPCJCFCAA|1955-09-24|668|2908|224|1955|6|9|24|3|1955|224|2908|Saturday|1955Q3|N|Y|N|2435352|2435594|2435010|2435284|N|N|N|N|N| +2435376|AAAAAAAAADJCFCAA|1955-09-25|668|2908|224|1955|0|9|25|3|1955|224|2908|Sunday|1955Q3|N|N|N|2435352|2435594|2435011|2435285|N|N|N|N|N| +2435377|AAAAAAAABDJCFCAA|1955-09-26|668|2908|224|1955|1|9|26|3|1955|224|2908|Monday|1955Q3|N|N|N|2435352|2435594|2435012|2435286|N|N|N|N|N| +2435378|AAAAAAAACDJCFCAA|1955-09-27|668|2909|224|1955|2|9|27|3|1955|224|2909|Tuesday|1955Q3|N|N|N|2435352|2435594|2435013|2435287|N|N|N|N|N| +2435379|AAAAAAAADDJCFCAA|1955-09-28|668|2909|224|1955|3|9|28|3|1955|224|2909|Wednesday|1955Q3|N|N|N|2435352|2435594|2435014|2435288|N|N|N|N|N| +2435380|AAAAAAAAEDJCFCAA|1955-09-29|668|2909|224|1955|4|9|29|3|1955|224|2909|Thursday|1955Q3|N|N|N|2435352|2435594|2435015|2435289|N|N|N|N|N| +2435381|AAAAAAAAFDJCFCAA|1955-09-30|668|2909|224|1955|5|9|30|3|1955|224|2909|Friday|1955Q3|N|Y|N|2435352|2435594|2435016|2435290|N|N|N|N|N| +2435382|AAAAAAAAGDJCFCAA|1955-10-01|669|2909|224|1955|6|10|1|3|1955|224|2909|Saturday|1955Q3|N|Y|N|2435382|2435654|2435017|2435290|N|N|N|N|N| +2435383|AAAAAAAAHDJCFCAA|1955-10-02|669|2909|224|1955|0|10|2|4|1955|224|2909|Sunday|1955Q4|N|N|N|2435382|2435654|2435018|2435291|N|N|N|N|N| +2435384|AAAAAAAAIDJCFCAA|1955-10-03|669|2909|224|1955|1|10|3|4|1955|224|2909|Monday|1955Q4|N|N|N|2435382|2435654|2435019|2435292|N|N|N|N|N| +2435385|AAAAAAAAJDJCFCAA|1955-10-04|669|2910|224|1955|2|10|4|4|1955|224|2910|Tuesday|1955Q4|N|N|N|2435382|2435654|2435020|2435293|N|N|N|N|N| +2435386|AAAAAAAAKDJCFCAA|1955-10-05|669|2910|224|1955|3|10|5|4|1955|224|2910|Wednesday|1955Q4|N|N|N|2435382|2435654|2435021|2435294|N|N|N|N|N| +2435387|AAAAAAAALDJCFCAA|1955-10-06|669|2910|224|1955|4|10|6|4|1955|224|2910|Thursday|1955Q4|N|N|N|2435382|2435654|2435022|2435295|N|N|N|N|N| +2435388|AAAAAAAAMDJCFCAA|1955-10-07|669|2910|224|1955|5|10|7|4|1955|224|2910|Friday|1955Q4|N|Y|N|2435382|2435654|2435023|2435296|N|N|N|N|N| +2435389|AAAAAAAANDJCFCAA|1955-10-08|669|2910|224|1955|6|10|8|4|1955|224|2910|Saturday|1955Q4|N|Y|N|2435382|2435654|2435024|2435297|N|N|N|N|N| +2435390|AAAAAAAAODJCFCAA|1955-10-09|669|2910|224|1955|0|10|9|4|1955|224|2910|Sunday|1955Q4|N|N|N|2435382|2435654|2435025|2435298|N|N|N|N|N| +2435391|AAAAAAAAPDJCFCAA|1955-10-10|669|2910|224|1955|1|10|10|4|1955|224|2910|Monday|1955Q4|N|N|N|2435382|2435654|2435026|2435299|N|N|N|N|N| +2435392|AAAAAAAAAEJCFCAA|1955-10-11|669|2911|224|1955|2|10|11|4|1955|224|2911|Tuesday|1955Q4|N|N|N|2435382|2435654|2435027|2435300|N|N|N|N|N| +2435393|AAAAAAAABEJCFCAA|1955-10-12|669|2911|224|1955|3|10|12|4|1955|224|2911|Wednesday|1955Q4|N|N|N|2435382|2435654|2435028|2435301|N|N|N|N|N| +2435394|AAAAAAAACEJCFCAA|1955-10-13|669|2911|224|1955|4|10|13|4|1955|224|2911|Thursday|1955Q4|N|N|N|2435382|2435654|2435029|2435302|N|N|N|N|N| +2435395|AAAAAAAADEJCFCAA|1955-10-14|669|2911|224|1955|5|10|14|4|1955|224|2911|Friday|1955Q4|N|Y|N|2435382|2435654|2435030|2435303|N|N|N|N|N| +2435396|AAAAAAAAEEJCFCAA|1955-10-15|669|2911|224|1955|6|10|15|4|1955|224|2911|Saturday|1955Q4|N|Y|N|2435382|2435654|2435031|2435304|N|N|N|N|N| +2435397|AAAAAAAAFEJCFCAA|1955-10-16|669|2911|224|1955|0|10|16|4|1955|224|2911|Sunday|1955Q4|N|N|N|2435382|2435654|2435032|2435305|N|N|N|N|N| +2435398|AAAAAAAAGEJCFCAA|1955-10-17|669|2911|224|1955|1|10|17|4|1955|224|2911|Monday|1955Q4|N|N|N|2435382|2435654|2435033|2435306|N|N|N|N|N| +2435399|AAAAAAAAHEJCFCAA|1955-10-18|669|2912|224|1955|2|10|18|4|1955|224|2912|Tuesday|1955Q4|N|N|N|2435382|2435654|2435034|2435307|N|N|N|N|N| +2435400|AAAAAAAAIEJCFCAA|1955-10-19|669|2912|224|1955|3|10|19|4|1955|224|2912|Wednesday|1955Q4|N|N|N|2435382|2435654|2435035|2435308|N|N|N|N|N| +2435401|AAAAAAAAJEJCFCAA|1955-10-20|669|2912|224|1955|4|10|20|4|1955|224|2912|Thursday|1955Q4|N|N|N|2435382|2435654|2435036|2435309|N|N|N|N|N| +2435402|AAAAAAAAKEJCFCAA|1955-10-21|669|2912|224|1955|5|10|21|4|1955|224|2912|Friday|1955Q4|N|Y|N|2435382|2435654|2435037|2435310|N|N|N|N|N| +2435403|AAAAAAAALEJCFCAA|1955-10-22|669|2912|224|1955|6|10|22|4|1955|224|2912|Saturday|1955Q4|N|Y|N|2435382|2435654|2435038|2435311|N|N|N|N|N| +2435404|AAAAAAAAMEJCFCAA|1955-10-23|669|2912|224|1955|0|10|23|4|1955|224|2912|Sunday|1955Q4|N|N|N|2435382|2435654|2435039|2435312|N|N|N|N|N| +2435405|AAAAAAAANEJCFCAA|1955-10-24|669|2912|224|1955|1|10|24|4|1955|224|2912|Monday|1955Q4|N|N|N|2435382|2435654|2435040|2435313|N|N|N|N|N| +2435406|AAAAAAAAOEJCFCAA|1955-10-25|669|2913|224|1955|2|10|25|4|1955|224|2913|Tuesday|1955Q4|N|N|N|2435382|2435654|2435041|2435314|N|N|N|N|N| +2435407|AAAAAAAAPEJCFCAA|1955-10-26|669|2913|224|1955|3|10|26|4|1955|224|2913|Wednesday|1955Q4|N|N|N|2435382|2435654|2435042|2435315|N|N|N|N|N| +2435408|AAAAAAAAAFJCFCAA|1955-10-27|669|2913|224|1955|4|10|27|4|1955|224|2913|Thursday|1955Q4|N|N|N|2435382|2435654|2435043|2435316|N|N|N|N|N| +2435409|AAAAAAAABFJCFCAA|1955-10-28|669|2913|224|1955|5|10|28|4|1955|224|2913|Friday|1955Q4|N|Y|N|2435382|2435654|2435044|2435317|N|N|N|N|N| +2435410|AAAAAAAACFJCFCAA|1955-10-29|669|2913|224|1955|6|10|29|4|1955|224|2913|Saturday|1955Q4|N|Y|N|2435382|2435654|2435045|2435318|N|N|N|N|N| +2435411|AAAAAAAADFJCFCAA|1955-10-30|669|2913|224|1955|0|10|30|4|1955|224|2913|Sunday|1955Q4|N|N|N|2435382|2435654|2435046|2435319|N|N|N|N|N| +2435412|AAAAAAAAEFJCFCAA|1955-10-31|669|2913|224|1955|1|10|31|4|1955|224|2913|Monday|1955Q4|N|N|N|2435382|2435654|2435047|2435320|N|N|N|N|N| +2435413|AAAAAAAAFFJCFCAA|1955-11-01|670|2914|224|1955|2|11|1|4|1955|224|2914|Tuesday|1955Q4|N|N|N|2435413|2435716|2435048|2435321|N|N|N|N|N| +2435414|AAAAAAAAGFJCFCAA|1955-11-02|670|2914|224|1955|3|11|2|4|1955|224|2914|Wednesday|1955Q4|N|N|N|2435413|2435716|2435049|2435322|N|N|N|N|N| +2435415|AAAAAAAAHFJCFCAA|1955-11-03|670|2914|224|1955|4|11|3|4|1955|224|2914|Thursday|1955Q4|N|N|N|2435413|2435716|2435050|2435323|N|N|N|N|N| +2435416|AAAAAAAAIFJCFCAA|1955-11-04|670|2914|224|1955|5|11|4|4|1955|224|2914|Friday|1955Q4|N|Y|N|2435413|2435716|2435051|2435324|N|N|N|N|N| +2435417|AAAAAAAAJFJCFCAA|1955-11-05|670|2914|224|1955|6|11|5|4|1955|224|2914|Saturday|1955Q4|N|Y|N|2435413|2435716|2435052|2435325|N|N|N|N|N| +2435418|AAAAAAAAKFJCFCAA|1955-11-06|670|2914|224|1955|0|11|6|4|1955|224|2914|Sunday|1955Q4|N|N|N|2435413|2435716|2435053|2435326|N|N|N|N|N| +2435419|AAAAAAAALFJCFCAA|1955-11-07|670|2914|224|1955|1|11|7|4|1955|224|2914|Monday|1955Q4|N|N|N|2435413|2435716|2435054|2435327|N|N|N|N|N| +2435420|AAAAAAAAMFJCFCAA|1955-11-08|670|2915|224|1955|2|11|8|4|1955|224|2915|Tuesday|1955Q4|N|N|N|2435413|2435716|2435055|2435328|N|N|N|N|N| +2435421|AAAAAAAANFJCFCAA|1955-11-09|670|2915|224|1955|3|11|9|4|1955|224|2915|Wednesday|1955Q4|N|N|N|2435413|2435716|2435056|2435329|N|N|N|N|N| +2435422|AAAAAAAAOFJCFCAA|1955-11-10|670|2915|224|1955|4|11|10|4|1955|224|2915|Thursday|1955Q4|N|N|N|2435413|2435716|2435057|2435330|N|N|N|N|N| +2435423|AAAAAAAAPFJCFCAA|1955-11-11|670|2915|224|1955|5|11|11|4|1955|224|2915|Friday|1955Q4|N|Y|N|2435413|2435716|2435058|2435331|N|N|N|N|N| +2435424|AAAAAAAAAGJCFCAA|1955-11-12|670|2915|224|1955|6|11|12|4|1955|224|2915|Saturday|1955Q4|N|Y|N|2435413|2435716|2435059|2435332|N|N|N|N|N| +2435425|AAAAAAAABGJCFCAA|1955-11-13|670|2915|224|1955|0|11|13|4|1955|224|2915|Sunday|1955Q4|N|N|N|2435413|2435716|2435060|2435333|N|N|N|N|N| +2435426|AAAAAAAACGJCFCAA|1955-11-14|670|2915|224|1955|1|11|14|4|1955|224|2915|Monday|1955Q4|N|N|N|2435413|2435716|2435061|2435334|N|N|N|N|N| +2435427|AAAAAAAADGJCFCAA|1955-11-15|670|2916|224|1955|2|11|15|4|1955|224|2916|Tuesday|1955Q4|N|N|N|2435413|2435716|2435062|2435335|N|N|N|N|N| +2435428|AAAAAAAAEGJCFCAA|1955-11-16|670|2916|224|1955|3|11|16|4|1955|224|2916|Wednesday|1955Q4|N|N|N|2435413|2435716|2435063|2435336|N|N|N|N|N| +2435429|AAAAAAAAFGJCFCAA|1955-11-17|670|2916|224|1955|4|11|17|4|1955|224|2916|Thursday|1955Q4|N|N|N|2435413|2435716|2435064|2435337|N|N|N|N|N| +2435430|AAAAAAAAGGJCFCAA|1955-11-18|670|2916|224|1955|5|11|18|4|1955|224|2916|Friday|1955Q4|N|Y|N|2435413|2435716|2435065|2435338|N|N|N|N|N| +2435431|AAAAAAAAHGJCFCAA|1955-11-19|670|2916|224|1955|6|11|19|4|1955|224|2916|Saturday|1955Q4|N|Y|N|2435413|2435716|2435066|2435339|N|N|N|N|N| +2435432|AAAAAAAAIGJCFCAA|1955-11-20|670|2916|224|1955|0|11|20|4|1955|224|2916|Sunday|1955Q4|N|N|N|2435413|2435716|2435067|2435340|N|N|N|N|N| +2435433|AAAAAAAAJGJCFCAA|1955-11-21|670|2916|224|1955|1|11|21|4|1955|224|2916|Monday|1955Q4|N|N|N|2435413|2435716|2435068|2435341|N|N|N|N|N| +2435434|AAAAAAAAKGJCFCAA|1955-11-22|670|2917|224|1955|2|11|22|4|1955|224|2917|Tuesday|1955Q4|N|N|N|2435413|2435716|2435069|2435342|N|N|N|N|N| +2435435|AAAAAAAALGJCFCAA|1955-11-23|670|2917|224|1955|3|11|23|4|1955|224|2917|Wednesday|1955Q4|N|N|N|2435413|2435716|2435070|2435343|N|N|N|N|N| +2435436|AAAAAAAAMGJCFCAA|1955-11-24|670|2917|224|1955|4|11|24|4|1955|224|2917|Thursday|1955Q4|N|N|N|2435413|2435716|2435071|2435344|N|N|N|N|N| +2435437|AAAAAAAANGJCFCAA|1955-11-25|670|2917|224|1955|5|11|25|4|1955|224|2917|Friday|1955Q4|N|Y|N|2435413|2435716|2435072|2435345|N|N|N|N|N| +2435438|AAAAAAAAOGJCFCAA|1955-11-26|670|2917|224|1955|6|11|26|4|1955|224|2917|Saturday|1955Q4|N|Y|N|2435413|2435716|2435073|2435346|N|N|N|N|N| +2435439|AAAAAAAAPGJCFCAA|1955-11-27|670|2917|224|1955|0|11|27|4|1955|224|2917|Sunday|1955Q4|N|N|N|2435413|2435716|2435074|2435347|N|N|N|N|N| +2435440|AAAAAAAAAHJCFCAA|1955-11-28|670|2917|224|1955|1|11|28|4|1955|224|2917|Monday|1955Q4|N|N|N|2435413|2435716|2435075|2435348|N|N|N|N|N| +2435441|AAAAAAAABHJCFCAA|1955-11-29|670|2918|224|1955|2|11|29|4|1955|224|2918|Tuesday|1955Q4|N|N|N|2435413|2435716|2435076|2435349|N|N|N|N|N| +2435442|AAAAAAAACHJCFCAA|1955-11-30|670|2918|224|1955|3|11|30|4|1955|224|2918|Wednesday|1955Q4|N|N|N|2435413|2435716|2435077|2435350|N|N|N|N|N| +2435443|AAAAAAAADHJCFCAA|1955-12-01|671|2918|225|1955|4|12|1|4|1955|225|2918|Thursday|1955Q4|N|N|N|2435443|2435776|2435078|2435351|N|N|N|N|N| +2435444|AAAAAAAAEHJCFCAA|1955-12-02|671|2918|225|1955|5|12|2|4|1955|225|2918|Friday|1955Q4|N|Y|N|2435443|2435776|2435079|2435352|N|N|N|N|N| +2435445|AAAAAAAAFHJCFCAA|1955-12-03|671|2918|225|1955|6|12|3|4|1955|225|2918|Saturday|1955Q4|N|Y|N|2435443|2435776|2435080|2435353|N|N|N|N|N| +2435446|AAAAAAAAGHJCFCAA|1955-12-04|671|2918|225|1955|0|12|4|4|1955|225|2918|Sunday|1955Q4|N|N|N|2435443|2435776|2435081|2435354|N|N|N|N|N| +2435447|AAAAAAAAHHJCFCAA|1955-12-05|671|2918|225|1955|1|12|5|4|1955|225|2918|Monday|1955Q4|N|N|N|2435443|2435776|2435082|2435355|N|N|N|N|N| +2435448|AAAAAAAAIHJCFCAA|1955-12-06|671|2919|225|1955|2|12|6|4|1955|225|2919|Tuesday|1955Q4|N|N|N|2435443|2435776|2435083|2435356|N|N|N|N|N| +2435449|AAAAAAAAJHJCFCAA|1955-12-07|671|2919|225|1955|3|12|7|4|1955|225|2919|Wednesday|1955Q4|N|N|N|2435443|2435776|2435084|2435357|N|N|N|N|N| +2435450|AAAAAAAAKHJCFCAA|1955-12-08|671|2919|225|1955|4|12|8|4|1955|225|2919|Thursday|1955Q4|N|N|N|2435443|2435776|2435085|2435358|N|N|N|N|N| +2435451|AAAAAAAALHJCFCAA|1955-12-09|671|2919|225|1955|5|12|9|4|1955|225|2919|Friday|1955Q4|N|Y|N|2435443|2435776|2435086|2435359|N|N|N|N|N| +2435452|AAAAAAAAMHJCFCAA|1955-12-10|671|2919|225|1955|6|12|10|4|1955|225|2919|Saturday|1955Q4|N|Y|N|2435443|2435776|2435087|2435360|N|N|N|N|N| +2435453|AAAAAAAANHJCFCAA|1955-12-11|671|2919|225|1955|0|12|11|4|1955|225|2919|Sunday|1955Q4|N|N|N|2435443|2435776|2435088|2435361|N|N|N|N|N| +2435454|AAAAAAAAOHJCFCAA|1955-12-12|671|2919|225|1955|1|12|12|4|1955|225|2919|Monday|1955Q4|N|N|N|2435443|2435776|2435089|2435362|N|N|N|N|N| +2435455|AAAAAAAAPHJCFCAA|1955-12-13|671|2920|225|1955|2|12|13|4|1955|225|2920|Tuesday|1955Q4|N|N|N|2435443|2435776|2435090|2435363|N|N|N|N|N| +2435456|AAAAAAAAAIJCFCAA|1955-12-14|671|2920|225|1955|3|12|14|4|1955|225|2920|Wednesday|1955Q4|N|N|N|2435443|2435776|2435091|2435364|N|N|N|N|N| +2435457|AAAAAAAABIJCFCAA|1955-12-15|671|2920|225|1955|4|12|15|4|1955|225|2920|Thursday|1955Q4|N|N|N|2435443|2435776|2435092|2435365|N|N|N|N|N| +2435458|AAAAAAAACIJCFCAA|1955-12-16|671|2920|225|1955|5|12|16|4|1955|225|2920|Friday|1955Q4|N|Y|N|2435443|2435776|2435093|2435366|N|N|N|N|N| +2435459|AAAAAAAADIJCFCAA|1955-12-17|671|2920|225|1955|6|12|17|4|1955|225|2920|Saturday|1955Q4|N|Y|N|2435443|2435776|2435094|2435367|N|N|N|N|N| +2435460|AAAAAAAAEIJCFCAA|1955-12-18|671|2920|225|1955|0|12|18|4|1955|225|2920|Sunday|1955Q4|N|N|N|2435443|2435776|2435095|2435368|N|N|N|N|N| +2435461|AAAAAAAAFIJCFCAA|1955-12-19|671|2920|225|1955|1|12|19|4|1955|225|2920|Monday|1955Q4|N|N|N|2435443|2435776|2435096|2435369|N|N|N|N|N| +2435462|AAAAAAAAGIJCFCAA|1955-12-20|671|2921|225|1955|2|12|20|4|1955|225|2921|Tuesday|1955Q4|N|N|N|2435443|2435776|2435097|2435370|N|N|N|N|N| +2435463|AAAAAAAAHIJCFCAA|1955-12-21|671|2921|225|1955|3|12|21|4|1955|225|2921|Wednesday|1955Q4|N|N|N|2435443|2435776|2435098|2435371|N|N|N|N|N| +2435464|AAAAAAAAIIJCFCAA|1955-12-22|671|2921|225|1955|4|12|22|4|1955|225|2921|Thursday|1955Q4|N|N|N|2435443|2435776|2435099|2435372|N|N|N|N|N| +2435465|AAAAAAAAJIJCFCAA|1955-12-23|671|2921|225|1955|5|12|23|4|1955|225|2921|Friday|1955Q4|N|Y|N|2435443|2435776|2435100|2435373|N|N|N|N|N| +2435466|AAAAAAAAKIJCFCAA|1955-12-24|671|2921|225|1955|6|12|24|4|1955|225|2921|Saturday|1955Q4|N|Y|N|2435443|2435776|2435101|2435374|N|N|N|N|N| +2435467|AAAAAAAALIJCFCAA|1955-12-25|671|2921|225|1955|0|12|25|4|1955|225|2921|Sunday|1955Q4|N|N|N|2435443|2435776|2435102|2435375|N|N|N|N|N| +2435468|AAAAAAAAMIJCFCAA|1955-12-26|671|2921|225|1955|1|12|26|4|1955|225|2921|Monday|1955Q4|Y|N|N|2435443|2435776|2435103|2435376|N|N|N|N|N| +2435469|AAAAAAAANIJCFCAA|1955-12-27|671|2922|225|1955|2|12|27|4|1955|225|2922|Tuesday|1955Q4|N|N|Y|2435443|2435776|2435104|2435377|N|N|N|N|N| +2435470|AAAAAAAAOIJCFCAA|1955-12-28|671|2922|225|1955|3|12|28|4|1955|225|2922|Wednesday|1955Q4|N|N|N|2435443|2435776|2435105|2435378|N|N|N|N|N| +2435471|AAAAAAAAPIJCFCAA|1955-12-29|671|2922|225|1955|4|12|29|4|1955|225|2922|Thursday|1955Q4|N|N|N|2435443|2435776|2435106|2435379|N|N|N|N|N| +2435472|AAAAAAAAAJJCFCAA|1955-12-30|671|2922|225|1955|5|12|30|4|1955|225|2922|Friday|1955Q4|N|Y|N|2435443|2435776|2435107|2435380|N|N|N|N|N| +2435473|AAAAAAAABJJCFCAA|1955-12-31|671|2922|225|1955|6|12|31|4|1955|225|2922|Saturday|1955Q4|N|Y|N|2435443|2435776|2435108|2435381|N|N|N|N|N| +2435474|AAAAAAAACJJCFCAA|1956-01-01|672|2922|225|1956|0|1|1|1|1956|225|2922|Sunday|1956Q1|Y|N|N|2435474|2435473|2435109|2435382|N|N|N|N|N| +2435475|AAAAAAAADJJCFCAA|1956-01-02|672|2922|225|1956|1|1|2|1|1956|225|2922|Monday|1956Q1|N|N|Y|2435474|2435473|2435110|2435383|N|N|N|N|N| +2435476|AAAAAAAAEJJCFCAA|1956-01-03|672|2923|225|1956|2|1|3|1|1956|225|2923|Tuesday|1956Q1|N|N|N|2435474|2435473|2435111|2435384|N|N|N|N|N| +2435477|AAAAAAAAFJJCFCAA|1956-01-04|672|2923|225|1956|3|1|4|1|1956|225|2923|Wednesday|1956Q1|N|N|N|2435474|2435473|2435112|2435385|N|N|N|N|N| +2435478|AAAAAAAAGJJCFCAA|1956-01-05|672|2923|225|1956|4|1|5|1|1956|225|2923|Thursday|1956Q1|N|N|N|2435474|2435473|2435113|2435386|N|N|N|N|N| +2435479|AAAAAAAAHJJCFCAA|1956-01-06|672|2923|225|1956|5|1|6|1|1956|225|2923|Friday|1956Q1|N|Y|N|2435474|2435473|2435114|2435387|N|N|N|N|N| +2435480|AAAAAAAAIJJCFCAA|1956-01-07|672|2923|225|1956|6|1|7|1|1956|225|2923|Saturday|1956Q1|N|Y|N|2435474|2435473|2435115|2435388|N|N|N|N|N| +2435481|AAAAAAAAJJJCFCAA|1956-01-08|672|2923|225|1956|0|1|8|1|1956|225|2923|Sunday|1956Q1|N|N|N|2435474|2435473|2435116|2435389|N|N|N|N|N| +2435482|AAAAAAAAKJJCFCAA|1956-01-09|672|2923|225|1956|1|1|9|1|1956|225|2923|Monday|1956Q1|N|N|N|2435474|2435473|2435117|2435390|N|N|N|N|N| +2435483|AAAAAAAALJJCFCAA|1956-01-10|672|2924|225|1956|2|1|10|1|1956|225|2924|Tuesday|1956Q1|N|N|N|2435474|2435473|2435118|2435391|N|N|N|N|N| +2435484|AAAAAAAAMJJCFCAA|1956-01-11|672|2924|225|1956|3|1|11|1|1956|225|2924|Wednesday|1956Q1|N|N|N|2435474|2435473|2435119|2435392|N|N|N|N|N| +2435485|AAAAAAAANJJCFCAA|1956-01-12|672|2924|225|1956|4|1|12|1|1956|225|2924|Thursday|1956Q1|N|N|N|2435474|2435473|2435120|2435393|N|N|N|N|N| +2435486|AAAAAAAAOJJCFCAA|1956-01-13|672|2924|225|1956|5|1|13|1|1956|225|2924|Friday|1956Q1|N|Y|N|2435474|2435473|2435121|2435394|N|N|N|N|N| +2435487|AAAAAAAAPJJCFCAA|1956-01-14|672|2924|225|1956|6|1|14|1|1956|225|2924|Saturday|1956Q1|N|Y|N|2435474|2435473|2435122|2435395|N|N|N|N|N| +2435488|AAAAAAAAAKJCFCAA|1956-01-15|672|2924|225|1956|0|1|15|1|1956|225|2924|Sunday|1956Q1|N|N|N|2435474|2435473|2435123|2435396|N|N|N|N|N| +2435489|AAAAAAAABKJCFCAA|1956-01-16|672|2924|225|1956|1|1|16|1|1956|225|2924|Monday|1956Q1|N|N|N|2435474|2435473|2435124|2435397|N|N|N|N|N| +2435490|AAAAAAAACKJCFCAA|1956-01-17|672|2925|225|1956|2|1|17|1|1956|225|2925|Tuesday|1956Q1|N|N|N|2435474|2435473|2435125|2435398|N|N|N|N|N| +2435491|AAAAAAAADKJCFCAA|1956-01-18|672|2925|225|1956|3|1|18|1|1956|225|2925|Wednesday|1956Q1|N|N|N|2435474|2435473|2435126|2435399|N|N|N|N|N| +2435492|AAAAAAAAEKJCFCAA|1956-01-19|672|2925|225|1956|4|1|19|1|1956|225|2925|Thursday|1956Q1|N|N|N|2435474|2435473|2435127|2435400|N|N|N|N|N| +2435493|AAAAAAAAFKJCFCAA|1956-01-20|672|2925|225|1956|5|1|20|1|1956|225|2925|Friday|1956Q1|N|Y|N|2435474|2435473|2435128|2435401|N|N|N|N|N| +2435494|AAAAAAAAGKJCFCAA|1956-01-21|672|2925|225|1956|6|1|21|1|1956|225|2925|Saturday|1956Q1|N|Y|N|2435474|2435473|2435129|2435402|N|N|N|N|N| +2435495|AAAAAAAAHKJCFCAA|1956-01-22|672|2925|225|1956|0|1|22|1|1956|225|2925|Sunday|1956Q1|N|N|N|2435474|2435473|2435130|2435403|N|N|N|N|N| +2435496|AAAAAAAAIKJCFCAA|1956-01-23|672|2925|225|1956|1|1|23|1|1956|225|2925|Monday|1956Q1|N|N|N|2435474|2435473|2435131|2435404|N|N|N|N|N| +2435497|AAAAAAAAJKJCFCAA|1956-01-24|672|2926|225|1956|2|1|24|1|1956|225|2926|Tuesday|1956Q1|N|N|N|2435474|2435473|2435132|2435405|N|N|N|N|N| +2435498|AAAAAAAAKKJCFCAA|1956-01-25|672|2926|225|1956|3|1|25|1|1956|225|2926|Wednesday|1956Q1|N|N|N|2435474|2435473|2435133|2435406|N|N|N|N|N| +2435499|AAAAAAAALKJCFCAA|1956-01-26|672|2926|225|1956|4|1|26|1|1956|225|2926|Thursday|1956Q1|N|N|N|2435474|2435473|2435134|2435407|N|N|N|N|N| +2435500|AAAAAAAAMKJCFCAA|1956-01-27|672|2926|225|1956|5|1|27|1|1956|225|2926|Friday|1956Q1|N|Y|N|2435474|2435473|2435135|2435408|N|N|N|N|N| +2435501|AAAAAAAANKJCFCAA|1956-01-28|672|2926|225|1956|6|1|28|1|1956|225|2926|Saturday|1956Q1|N|Y|N|2435474|2435473|2435136|2435409|N|N|N|N|N| +2435502|AAAAAAAAOKJCFCAA|1956-01-29|672|2926|225|1956|0|1|29|1|1956|225|2926|Sunday|1956Q1|N|N|N|2435474|2435473|2435137|2435410|N|N|N|N|N| +2435503|AAAAAAAAPKJCFCAA|1956-01-30|672|2926|225|1956|1|1|30|1|1956|225|2926|Monday|1956Q1|N|N|N|2435474|2435473|2435138|2435411|N|N|N|N|N| +2435504|AAAAAAAAALJCFCAA|1956-01-31|672|2927|225|1956|2|1|31|1|1956|225|2927|Tuesday|1956Q1|N|N|N|2435474|2435473|2435139|2435412|N|N|N|N|N| +2435505|AAAAAAAABLJCFCAA|1956-02-01|673|2927|225|1956|3|2|1|1|1956|225|2927|Wednesday|1956Q1|N|N|N|2435505|2435535|2435140|2435413|N|N|N|N|N| +2435506|AAAAAAAACLJCFCAA|1956-02-02|673|2927|225|1956|4|2|2|1|1956|225|2927|Thursday|1956Q1|N|N|N|2435505|2435535|2435141|2435414|N|N|N|N|N| +2435507|AAAAAAAADLJCFCAA|1956-02-03|673|2927|225|1956|5|2|3|1|1956|225|2927|Friday|1956Q1|N|Y|N|2435505|2435535|2435142|2435415|N|N|N|N|N| +2435508|AAAAAAAAELJCFCAA|1956-02-04|673|2927|225|1956|6|2|4|1|1956|225|2927|Saturday|1956Q1|N|Y|N|2435505|2435535|2435143|2435416|N|N|N|N|N| +2435509|AAAAAAAAFLJCFCAA|1956-02-05|673|2927|225|1956|0|2|5|1|1956|225|2927|Sunday|1956Q1|N|N|N|2435505|2435535|2435144|2435417|N|N|N|N|N| +2435510|AAAAAAAAGLJCFCAA|1956-02-06|673|2927|225|1956|1|2|6|1|1956|225|2927|Monday|1956Q1|N|N|N|2435505|2435535|2435145|2435418|N|N|N|N|N| +2435511|AAAAAAAAHLJCFCAA|1956-02-07|673|2928|225|1956|2|2|7|1|1956|225|2928|Tuesday|1956Q1|N|N|N|2435505|2435535|2435146|2435419|N|N|N|N|N| +2435512|AAAAAAAAILJCFCAA|1956-02-08|673|2928|225|1956|3|2|8|1|1956|225|2928|Wednesday|1956Q1|N|N|N|2435505|2435535|2435147|2435420|N|N|N|N|N| +2435513|AAAAAAAAJLJCFCAA|1956-02-09|673|2928|225|1956|4|2|9|1|1956|225|2928|Thursday|1956Q1|N|N|N|2435505|2435535|2435148|2435421|N|N|N|N|N| +2435514|AAAAAAAAKLJCFCAA|1956-02-10|673|2928|225|1956|5|2|10|1|1956|225|2928|Friday|1956Q1|N|Y|N|2435505|2435535|2435149|2435422|N|N|N|N|N| +2435515|AAAAAAAALLJCFCAA|1956-02-11|673|2928|225|1956|6|2|11|1|1956|225|2928|Saturday|1956Q1|N|Y|N|2435505|2435535|2435150|2435423|N|N|N|N|N| +2435516|AAAAAAAAMLJCFCAA|1956-02-12|673|2928|225|1956|0|2|12|1|1956|225|2928|Sunday|1956Q1|N|N|N|2435505|2435535|2435151|2435424|N|N|N|N|N| +2435517|AAAAAAAANLJCFCAA|1956-02-13|673|2928|225|1956|1|2|13|1|1956|225|2928|Monday|1956Q1|N|N|N|2435505|2435535|2435152|2435425|N|N|N|N|N| +2435518|AAAAAAAAOLJCFCAA|1956-02-14|673|2929|225|1956|2|2|14|1|1956|225|2929|Tuesday|1956Q1|N|N|N|2435505|2435535|2435153|2435426|N|N|N|N|N| +2435519|AAAAAAAAPLJCFCAA|1956-02-15|673|2929|225|1956|3|2|15|1|1956|225|2929|Wednesday|1956Q1|N|N|N|2435505|2435535|2435154|2435427|N|N|N|N|N| +2435520|AAAAAAAAAMJCFCAA|1956-02-16|673|2929|225|1956|4|2|16|1|1956|225|2929|Thursday|1956Q1|N|N|N|2435505|2435535|2435155|2435428|N|N|N|N|N| +2435521|AAAAAAAABMJCFCAA|1956-02-17|673|2929|225|1956|5|2|17|1|1956|225|2929|Friday|1956Q1|N|Y|N|2435505|2435535|2435156|2435429|N|N|N|N|N| +2435522|AAAAAAAACMJCFCAA|1956-02-18|673|2929|225|1956|6|2|18|1|1956|225|2929|Saturday|1956Q1|N|Y|N|2435505|2435535|2435157|2435430|N|N|N|N|N| +2435523|AAAAAAAADMJCFCAA|1956-02-19|673|2929|225|1956|0|2|19|1|1956|225|2929|Sunday|1956Q1|N|N|N|2435505|2435535|2435158|2435431|N|N|N|N|N| +2435524|AAAAAAAAEMJCFCAA|1956-02-20|673|2929|225|1956|1|2|20|1|1956|225|2929|Monday|1956Q1|N|N|N|2435505|2435535|2435159|2435432|N|N|N|N|N| +2435525|AAAAAAAAFMJCFCAA|1956-02-21|673|2930|225|1956|2|2|21|1|1956|225|2930|Tuesday|1956Q1|N|N|N|2435505|2435535|2435160|2435433|N|N|N|N|N| +2435526|AAAAAAAAGMJCFCAA|1956-02-22|673|2930|225|1956|3|2|22|1|1956|225|2930|Wednesday|1956Q1|N|N|N|2435505|2435535|2435161|2435434|N|N|N|N|N| +2435527|AAAAAAAAHMJCFCAA|1956-02-23|673|2930|225|1956|4|2|23|1|1956|225|2930|Thursday|1956Q1|N|N|N|2435505|2435535|2435162|2435435|N|N|N|N|N| +2435528|AAAAAAAAIMJCFCAA|1956-02-24|673|2930|225|1956|5|2|24|1|1956|225|2930|Friday|1956Q1|N|Y|N|2435505|2435535|2435163|2435436|N|N|N|N|N| +2435529|AAAAAAAAJMJCFCAA|1956-02-25|673|2930|225|1956|6|2|25|1|1956|225|2930|Saturday|1956Q1|N|Y|N|2435505|2435535|2435164|2435437|N|N|N|N|N| +2435530|AAAAAAAAKMJCFCAA|1956-02-26|673|2930|225|1956|0|2|26|1|1956|225|2930|Sunday|1956Q1|N|N|N|2435505|2435535|2435165|2435438|N|N|N|N|N| +2435531|AAAAAAAALMJCFCAA|1956-02-27|673|2930|225|1956|1|2|27|1|1956|225|2930|Monday|1956Q1|N|N|N|2435505|2435535|2435166|2435439|N|N|N|N|N| +2435532|AAAAAAAAMMJCFCAA|1956-02-28|673|2931|225|1956|2|2|28|1|1956|225|2931|Tuesday|1956Q1|N|N|N|2435505|2435535|2435167|2435440|N|N|N|N|N| +2435533|AAAAAAAANMJCFCAA|1956-02-29|673|2931|225|1956|3|2|29|1|1956|225|2931|Wednesday|1956Q1|N|N|N|2435505|2435535|2435167|2435441|N|N|N|N|N| +2435534|AAAAAAAAOMJCFCAA|1956-03-01|674|2931|226|1956|4|3|1|1|1956|226|2931|Thursday|1956Q1|N|N|N|2435534|2435593|2435168|2435442|N|N|N|N|N| +2435535|AAAAAAAAPMJCFCAA|1956-03-02|674|2931|226|1956|5|3|2|1|1956|226|2931|Friday|1956Q1|N|Y|N|2435534|2435593|2435169|2435443|N|N|N|N|N| +2435536|AAAAAAAAANJCFCAA|1956-03-03|674|2931|226|1956|6|3|3|1|1956|226|2931|Saturday|1956Q1|N|Y|N|2435534|2435593|2435170|2435444|N|N|N|N|N| +2435537|AAAAAAAABNJCFCAA|1956-03-04|674|2931|226|1956|0|3|4|1|1956|226|2931|Sunday|1956Q1|N|N|N|2435534|2435593|2435171|2435445|N|N|N|N|N| +2435538|AAAAAAAACNJCFCAA|1956-03-05|674|2931|226|1956|1|3|5|1|1956|226|2931|Monday|1956Q1|N|N|N|2435534|2435593|2435172|2435446|N|N|N|N|N| +2435539|AAAAAAAADNJCFCAA|1956-03-06|674|2932|226|1956|2|3|6|1|1956|226|2932|Tuesday|1956Q1|N|N|N|2435534|2435593|2435173|2435447|N|N|N|N|N| +2435540|AAAAAAAAENJCFCAA|1956-03-07|674|2932|226|1956|3|3|7|1|1956|226|2932|Wednesday|1956Q1|N|N|N|2435534|2435593|2435174|2435448|N|N|N|N|N| +2435541|AAAAAAAAFNJCFCAA|1956-03-08|674|2932|226|1956|4|3|8|1|1956|226|2932|Thursday|1956Q1|N|N|N|2435534|2435593|2435175|2435449|N|N|N|N|N| +2435542|AAAAAAAAGNJCFCAA|1956-03-09|674|2932|226|1956|5|3|9|1|1956|226|2932|Friday|1956Q1|N|Y|N|2435534|2435593|2435176|2435450|N|N|N|N|N| +2435543|AAAAAAAAHNJCFCAA|1956-03-10|674|2932|226|1956|6|3|10|1|1956|226|2932|Saturday|1956Q1|N|Y|N|2435534|2435593|2435177|2435451|N|N|N|N|N| +2435544|AAAAAAAAINJCFCAA|1956-03-11|674|2932|226|1956|0|3|11|1|1956|226|2932|Sunday|1956Q1|N|N|N|2435534|2435593|2435178|2435452|N|N|N|N|N| +2435545|AAAAAAAAJNJCFCAA|1956-03-12|674|2932|226|1956|1|3|12|1|1956|226|2932|Monday|1956Q1|N|N|N|2435534|2435593|2435179|2435453|N|N|N|N|N| +2435546|AAAAAAAAKNJCFCAA|1956-03-13|674|2933|226|1956|2|3|13|1|1956|226|2933|Tuesday|1956Q1|N|N|N|2435534|2435593|2435180|2435454|N|N|N|N|N| +2435547|AAAAAAAALNJCFCAA|1956-03-14|674|2933|226|1956|3|3|14|1|1956|226|2933|Wednesday|1956Q1|N|N|N|2435534|2435593|2435181|2435455|N|N|N|N|N| +2435548|AAAAAAAAMNJCFCAA|1956-03-15|674|2933|226|1956|4|3|15|1|1956|226|2933|Thursday|1956Q1|N|N|N|2435534|2435593|2435182|2435456|N|N|N|N|N| +2435549|AAAAAAAANNJCFCAA|1956-03-16|674|2933|226|1956|5|3|16|1|1956|226|2933|Friday|1956Q1|N|Y|N|2435534|2435593|2435183|2435457|N|N|N|N|N| +2435550|AAAAAAAAONJCFCAA|1956-03-17|674|2933|226|1956|6|3|17|1|1956|226|2933|Saturday|1956Q1|N|Y|N|2435534|2435593|2435184|2435458|N|N|N|N|N| +2435551|AAAAAAAAPNJCFCAA|1956-03-18|674|2933|226|1956|0|3|18|1|1956|226|2933|Sunday|1956Q1|N|N|N|2435534|2435593|2435185|2435459|N|N|N|N|N| +2435552|AAAAAAAAAOJCFCAA|1956-03-19|674|2933|226|1956|1|3|19|1|1956|226|2933|Monday|1956Q1|N|N|N|2435534|2435593|2435186|2435460|N|N|N|N|N| +2435553|AAAAAAAABOJCFCAA|1956-03-20|674|2934|226|1956|2|3|20|1|1956|226|2934|Tuesday|1956Q1|N|N|N|2435534|2435593|2435187|2435461|N|N|N|N|N| +2435554|AAAAAAAACOJCFCAA|1956-03-21|674|2934|226|1956|3|3|21|1|1956|226|2934|Wednesday|1956Q1|N|N|N|2435534|2435593|2435188|2435462|N|N|N|N|N| +2435555|AAAAAAAADOJCFCAA|1956-03-22|674|2934|226|1956|4|3|22|1|1956|226|2934|Thursday|1956Q1|N|N|N|2435534|2435593|2435189|2435463|N|N|N|N|N| +2435556|AAAAAAAAEOJCFCAA|1956-03-23|674|2934|226|1956|5|3|23|1|1956|226|2934|Friday|1956Q1|N|Y|N|2435534|2435593|2435190|2435464|N|N|N|N|N| +2435557|AAAAAAAAFOJCFCAA|1956-03-24|674|2934|226|1956|6|3|24|1|1956|226|2934|Saturday|1956Q1|N|Y|N|2435534|2435593|2435191|2435465|N|N|N|N|N| +2435558|AAAAAAAAGOJCFCAA|1956-03-25|674|2934|226|1956|0|3|25|1|1956|226|2934|Sunday|1956Q1|N|N|N|2435534|2435593|2435192|2435466|N|N|N|N|N| +2435559|AAAAAAAAHOJCFCAA|1956-03-26|674|2934|226|1956|1|3|26|1|1956|226|2934|Monday|1956Q1|N|N|N|2435534|2435593|2435193|2435467|N|N|N|N|N| +2435560|AAAAAAAAIOJCFCAA|1956-03-27|674|2935|226|1956|2|3|27|1|1956|226|2935|Tuesday|1956Q1|N|N|N|2435534|2435593|2435194|2435468|N|N|N|N|N| +2435561|AAAAAAAAJOJCFCAA|1956-03-28|674|2935|226|1956|3|3|28|1|1956|226|2935|Wednesday|1956Q1|N|N|N|2435534|2435593|2435195|2435469|N|N|N|N|N| +2435562|AAAAAAAAKOJCFCAA|1956-03-29|674|2935|226|1956|4|3|29|1|1956|226|2935|Thursday|1956Q1|N|N|N|2435534|2435593|2435196|2435470|N|N|N|N|N| +2435563|AAAAAAAALOJCFCAA|1956-03-30|674|2935|226|1956|5|3|30|1|1956|226|2935|Friday|1956Q1|N|Y|N|2435534|2435593|2435197|2435471|N|N|N|N|N| +2435564|AAAAAAAAMOJCFCAA|1956-03-31|674|2935|226|1956|6|3|31|1|1956|226|2935|Saturday|1956Q1|N|Y|N|2435534|2435593|2435198|2435472|N|N|N|N|N| +2435565|AAAAAAAANOJCFCAA|1956-04-01|675|2935|226|1956|0|4|1|2|1956|226|2935|Sunday|1956Q2|N|N|N|2435565|2435655|2435199|2435474|N|N|N|N|N| +2435566|AAAAAAAAOOJCFCAA|1956-04-02|675|2935|226|1956|1|4|2|2|1956|226|2935|Monday|1956Q2|N|N|N|2435565|2435655|2435200|2435475|N|N|N|N|N| +2435567|AAAAAAAAPOJCFCAA|1956-04-03|675|2936|226|1956|2|4|3|2|1956|226|2936|Tuesday|1956Q2|N|N|N|2435565|2435655|2435201|2435476|N|N|N|N|N| +2435568|AAAAAAAAAPJCFCAA|1956-04-04|675|2936|226|1956|3|4|4|2|1956|226|2936|Wednesday|1956Q2|N|N|N|2435565|2435655|2435202|2435477|N|N|N|N|N| +2435569|AAAAAAAABPJCFCAA|1956-04-05|675|2936|226|1956|4|4|5|2|1956|226|2936|Thursday|1956Q2|N|N|N|2435565|2435655|2435203|2435478|N|N|N|N|N| +2435570|AAAAAAAACPJCFCAA|1956-04-06|675|2936|226|1956|5|4|6|2|1956|226|2936|Friday|1956Q2|N|Y|N|2435565|2435655|2435204|2435479|N|N|N|N|N| +2435571|AAAAAAAADPJCFCAA|1956-04-07|675|2936|226|1956|6|4|7|2|1956|226|2936|Saturday|1956Q2|N|Y|N|2435565|2435655|2435205|2435480|N|N|N|N|N| +2435572|AAAAAAAAEPJCFCAA|1956-04-08|675|2936|226|1956|0|4|8|2|1956|226|2936|Sunday|1956Q2|N|N|N|2435565|2435655|2435206|2435481|N|N|N|N|N| +2435573|AAAAAAAAFPJCFCAA|1956-04-09|675|2936|226|1956|1|4|9|2|1956|226|2936|Monday|1956Q2|N|N|N|2435565|2435655|2435207|2435482|N|N|N|N|N| +2435574|AAAAAAAAGPJCFCAA|1956-04-10|675|2937|226|1956|2|4|10|2|1956|226|2937|Tuesday|1956Q2|N|N|N|2435565|2435655|2435208|2435483|N|N|N|N|N| +2435575|AAAAAAAAHPJCFCAA|1956-04-11|675|2937|226|1956|3|4|11|2|1956|226|2937|Wednesday|1956Q2|N|N|N|2435565|2435655|2435209|2435484|N|N|N|N|N| +2435576|AAAAAAAAIPJCFCAA|1956-04-12|675|2937|226|1956|4|4|12|2|1956|226|2937|Thursday|1956Q2|N|N|N|2435565|2435655|2435210|2435485|N|N|N|N|N| +2435577|AAAAAAAAJPJCFCAA|1956-04-13|675|2937|226|1956|5|4|13|2|1956|226|2937|Friday|1956Q2|N|Y|N|2435565|2435655|2435211|2435486|N|N|N|N|N| +2435578|AAAAAAAAKPJCFCAA|1956-04-14|675|2937|226|1956|6|4|14|2|1956|226|2937|Saturday|1956Q2|N|Y|N|2435565|2435655|2435212|2435487|N|N|N|N|N| +2435579|AAAAAAAALPJCFCAA|1956-04-15|675|2937|226|1956|0|4|15|2|1956|226|2937|Sunday|1956Q2|N|N|N|2435565|2435655|2435213|2435488|N|N|N|N|N| +2435580|AAAAAAAAMPJCFCAA|1956-04-16|675|2937|226|1956|1|4|16|2|1956|226|2937|Monday|1956Q2|N|N|N|2435565|2435655|2435214|2435489|N|N|N|N|N| +2435581|AAAAAAAANPJCFCAA|1956-04-17|675|2938|226|1956|2|4|17|2|1956|226|2938|Tuesday|1956Q2|N|N|N|2435565|2435655|2435215|2435490|N|N|N|N|N| +2435582|AAAAAAAAOPJCFCAA|1956-04-18|675|2938|226|1956|3|4|18|2|1956|226|2938|Wednesday|1956Q2|N|N|N|2435565|2435655|2435216|2435491|N|N|N|N|N| +2435583|AAAAAAAAPPJCFCAA|1956-04-19|675|2938|226|1956|4|4|19|2|1956|226|2938|Thursday|1956Q2|N|N|N|2435565|2435655|2435217|2435492|N|N|N|N|N| +2435584|AAAAAAAAAAKCFCAA|1956-04-20|675|2938|226|1956|5|4|20|2|1956|226|2938|Friday|1956Q2|N|Y|N|2435565|2435655|2435218|2435493|N|N|N|N|N| +2435585|AAAAAAAABAKCFCAA|1956-04-21|675|2938|226|1956|6|4|21|2|1956|226|2938|Saturday|1956Q2|N|Y|N|2435565|2435655|2435219|2435494|N|N|N|N|N| +2435586|AAAAAAAACAKCFCAA|1956-04-22|675|2938|226|1956|0|4|22|2|1956|226|2938|Sunday|1956Q2|N|N|N|2435565|2435655|2435220|2435495|N|N|N|N|N| +2435587|AAAAAAAADAKCFCAA|1956-04-23|675|2938|226|1956|1|4|23|2|1956|226|2938|Monday|1956Q2|N|N|N|2435565|2435655|2435221|2435496|N|N|N|N|N| +2435588|AAAAAAAAEAKCFCAA|1956-04-24|675|2939|226|1956|2|4|24|2|1956|226|2939|Tuesday|1956Q2|N|N|N|2435565|2435655|2435222|2435497|N|N|N|N|N| +2435589|AAAAAAAAFAKCFCAA|1956-04-25|675|2939|226|1956|3|4|25|2|1956|226|2939|Wednesday|1956Q2|N|N|N|2435565|2435655|2435223|2435498|N|N|N|N|N| +2435590|AAAAAAAAGAKCFCAA|1956-04-26|675|2939|226|1956|4|4|26|2|1956|226|2939|Thursday|1956Q2|N|N|N|2435565|2435655|2435224|2435499|N|N|N|N|N| +2435591|AAAAAAAAHAKCFCAA|1956-04-27|675|2939|226|1956|5|4|27|2|1956|226|2939|Friday|1956Q2|N|Y|N|2435565|2435655|2435225|2435500|N|N|N|N|N| +2435592|AAAAAAAAIAKCFCAA|1956-04-28|675|2939|226|1956|6|4|28|2|1956|226|2939|Saturday|1956Q2|N|Y|N|2435565|2435655|2435226|2435501|N|N|N|N|N| +2435593|AAAAAAAAJAKCFCAA|1956-04-29|675|2939|226|1956|0|4|29|2|1956|226|2939|Sunday|1956Q2|N|N|N|2435565|2435655|2435227|2435502|N|N|N|N|N| +2435594|AAAAAAAAKAKCFCAA|1956-04-30|675|2939|226|1956|1|4|30|2|1956|226|2939|Monday|1956Q2|N|N|N|2435565|2435655|2435228|2435503|N|N|N|N|N| +2435595|AAAAAAAALAKCFCAA|1956-05-01|676|2940|226|1956|2|5|1|2|1956|226|2940|Tuesday|1956Q2|N|N|N|2435595|2435715|2435229|2435504|N|N|N|N|N| +2435596|AAAAAAAAMAKCFCAA|1956-05-02|676|2940|226|1956|3|5|2|2|1956|226|2940|Wednesday|1956Q2|N|N|N|2435595|2435715|2435230|2435505|N|N|N|N|N| +2435597|AAAAAAAANAKCFCAA|1956-05-03|676|2940|226|1956|4|5|3|2|1956|226|2940|Thursday|1956Q2|N|N|N|2435595|2435715|2435231|2435506|N|N|N|N|N| +2435598|AAAAAAAAOAKCFCAA|1956-05-04|676|2940|226|1956|5|5|4|2|1956|226|2940|Friday|1956Q2|N|Y|N|2435595|2435715|2435232|2435507|N|N|N|N|N| +2435599|AAAAAAAAPAKCFCAA|1956-05-05|676|2940|226|1956|6|5|5|2|1956|226|2940|Saturday|1956Q2|N|Y|N|2435595|2435715|2435233|2435508|N|N|N|N|N| +2435600|AAAAAAAAABKCFCAA|1956-05-06|676|2940|226|1956|0|5|6|2|1956|226|2940|Sunday|1956Q2|N|N|N|2435595|2435715|2435234|2435509|N|N|N|N|N| +2435601|AAAAAAAABBKCFCAA|1956-05-07|676|2940|226|1956|1|5|7|2|1956|226|2940|Monday|1956Q2|N|N|N|2435595|2435715|2435235|2435510|N|N|N|N|N| +2435602|AAAAAAAACBKCFCAA|1956-05-08|676|2941|226|1956|2|5|8|2|1956|226|2941|Tuesday|1956Q2|N|N|N|2435595|2435715|2435236|2435511|N|N|N|N|N| +2435603|AAAAAAAADBKCFCAA|1956-05-09|676|2941|226|1956|3|5|9|2|1956|226|2941|Wednesday|1956Q2|N|N|N|2435595|2435715|2435237|2435512|N|N|N|N|N| +2435604|AAAAAAAAEBKCFCAA|1956-05-10|676|2941|226|1956|4|5|10|2|1956|226|2941|Thursday|1956Q2|N|N|N|2435595|2435715|2435238|2435513|N|N|N|N|N| +2435605|AAAAAAAAFBKCFCAA|1956-05-11|676|2941|226|1956|5|5|11|2|1956|226|2941|Friday|1956Q2|N|Y|N|2435595|2435715|2435239|2435514|N|N|N|N|N| +2435606|AAAAAAAAGBKCFCAA|1956-05-12|676|2941|226|1956|6|5|12|2|1956|226|2941|Saturday|1956Q2|N|Y|N|2435595|2435715|2435240|2435515|N|N|N|N|N| +2435607|AAAAAAAAHBKCFCAA|1956-05-13|676|2941|226|1956|0|5|13|2|1956|226|2941|Sunday|1956Q2|N|N|N|2435595|2435715|2435241|2435516|N|N|N|N|N| +2435608|AAAAAAAAIBKCFCAA|1956-05-14|676|2941|226|1956|1|5|14|2|1956|226|2941|Monday|1956Q2|N|N|N|2435595|2435715|2435242|2435517|N|N|N|N|N| +2435609|AAAAAAAAJBKCFCAA|1956-05-15|676|2942|226|1956|2|5|15|2|1956|226|2942|Tuesday|1956Q2|N|N|N|2435595|2435715|2435243|2435518|N|N|N|N|N| +2435610|AAAAAAAAKBKCFCAA|1956-05-16|676|2942|226|1956|3|5|16|2|1956|226|2942|Wednesday|1956Q2|N|N|N|2435595|2435715|2435244|2435519|N|N|N|N|N| +2435611|AAAAAAAALBKCFCAA|1956-05-17|676|2942|226|1956|4|5|17|2|1956|226|2942|Thursday|1956Q2|N|N|N|2435595|2435715|2435245|2435520|N|N|N|N|N| +2435612|AAAAAAAAMBKCFCAA|1956-05-18|676|2942|226|1956|5|5|18|2|1956|226|2942|Friday|1956Q2|N|Y|N|2435595|2435715|2435246|2435521|N|N|N|N|N| +2435613|AAAAAAAANBKCFCAA|1956-05-19|676|2942|226|1956|6|5|19|2|1956|226|2942|Saturday|1956Q2|N|Y|N|2435595|2435715|2435247|2435522|N|N|N|N|N| +2435614|AAAAAAAAOBKCFCAA|1956-05-20|676|2942|226|1956|0|5|20|2|1956|226|2942|Sunday|1956Q2|N|N|N|2435595|2435715|2435248|2435523|N|N|N|N|N| +2435615|AAAAAAAAPBKCFCAA|1956-05-21|676|2942|226|1956|1|5|21|2|1956|226|2942|Monday|1956Q2|N|N|N|2435595|2435715|2435249|2435524|N|N|N|N|N| +2435616|AAAAAAAAACKCFCAA|1956-05-22|676|2943|226|1956|2|5|22|2|1956|226|2943|Tuesday|1956Q2|N|N|N|2435595|2435715|2435250|2435525|N|N|N|N|N| +2435617|AAAAAAAABCKCFCAA|1956-05-23|676|2943|226|1956|3|5|23|2|1956|226|2943|Wednesday|1956Q2|N|N|N|2435595|2435715|2435251|2435526|N|N|N|N|N| +2435618|AAAAAAAACCKCFCAA|1956-05-24|676|2943|226|1956|4|5|24|2|1956|226|2943|Thursday|1956Q2|N|N|N|2435595|2435715|2435252|2435527|N|N|N|N|N| +2435619|AAAAAAAADCKCFCAA|1956-05-25|676|2943|226|1956|5|5|25|2|1956|226|2943|Friday|1956Q2|N|Y|N|2435595|2435715|2435253|2435528|N|N|N|N|N| +2435620|AAAAAAAAECKCFCAA|1956-05-26|676|2943|226|1956|6|5|26|2|1956|226|2943|Saturday|1956Q2|N|Y|N|2435595|2435715|2435254|2435529|N|N|N|N|N| +2435621|AAAAAAAAFCKCFCAA|1956-05-27|676|2943|226|1956|0|5|27|2|1956|226|2943|Sunday|1956Q2|N|N|N|2435595|2435715|2435255|2435530|N|N|N|N|N| +2435622|AAAAAAAAGCKCFCAA|1956-05-28|676|2943|226|1956|1|5|28|2|1956|226|2943|Monday|1956Q2|N|N|N|2435595|2435715|2435256|2435531|N|N|N|N|N| +2435623|AAAAAAAAHCKCFCAA|1956-05-29|676|2944|226|1956|2|5|29|2|1956|226|2944|Tuesday|1956Q2|N|N|N|2435595|2435715|2435257|2435532|N|N|N|N|N| +2435624|AAAAAAAAICKCFCAA|1956-05-30|676|2944|226|1956|3|5|30|2|1956|226|2944|Wednesday|1956Q2|N|N|N|2435595|2435715|2435258|2435533|N|N|N|N|N| +2435625|AAAAAAAAJCKCFCAA|1956-05-31|676|2944|226|1956|4|5|31|2|1956|226|2944|Thursday|1956Q2|N|N|N|2435595|2435715|2435259|2435534|N|N|N|N|N| +2435626|AAAAAAAAKCKCFCAA|1956-06-01|677|2944|227|1956|5|6|1|2|1956|227|2944|Friday|1956Q2|N|Y|N|2435626|2435777|2435260|2435535|N|N|N|N|N| +2435627|AAAAAAAALCKCFCAA|1956-06-02|677|2944|227|1956|6|6|2|2|1956|227|2944|Saturday|1956Q2|N|Y|N|2435626|2435777|2435261|2435536|N|N|N|N|N| +2435628|AAAAAAAAMCKCFCAA|1956-06-03|677|2944|227|1956|0|6|3|2|1956|227|2944|Sunday|1956Q2|N|N|N|2435626|2435777|2435262|2435537|N|N|N|N|N| +2435629|AAAAAAAANCKCFCAA|1956-06-04|677|2944|227|1956|1|6|4|2|1956|227|2944|Monday|1956Q2|N|N|N|2435626|2435777|2435263|2435538|N|N|N|N|N| +2435630|AAAAAAAAOCKCFCAA|1956-06-05|677|2945|227|1956|2|6|5|2|1956|227|2945|Tuesday|1956Q2|N|N|N|2435626|2435777|2435264|2435539|N|N|N|N|N| +2435631|AAAAAAAAPCKCFCAA|1956-06-06|677|2945|227|1956|3|6|6|2|1956|227|2945|Wednesday|1956Q2|N|N|N|2435626|2435777|2435265|2435540|N|N|N|N|N| +2435632|AAAAAAAAADKCFCAA|1956-06-07|677|2945|227|1956|4|6|7|2|1956|227|2945|Thursday|1956Q2|N|N|N|2435626|2435777|2435266|2435541|N|N|N|N|N| +2435633|AAAAAAAABDKCFCAA|1956-06-08|677|2945|227|1956|5|6|8|2|1956|227|2945|Friday|1956Q2|N|Y|N|2435626|2435777|2435267|2435542|N|N|N|N|N| +2435634|AAAAAAAACDKCFCAA|1956-06-09|677|2945|227|1956|6|6|9|2|1956|227|2945|Saturday|1956Q2|N|Y|N|2435626|2435777|2435268|2435543|N|N|N|N|N| +2435635|AAAAAAAADDKCFCAA|1956-06-10|677|2945|227|1956|0|6|10|2|1956|227|2945|Sunday|1956Q2|N|N|N|2435626|2435777|2435269|2435544|N|N|N|N|N| +2435636|AAAAAAAAEDKCFCAA|1956-06-11|677|2945|227|1956|1|6|11|2|1956|227|2945|Monday|1956Q2|N|N|N|2435626|2435777|2435270|2435545|N|N|N|N|N| +2435637|AAAAAAAAFDKCFCAA|1956-06-12|677|2946|227|1956|2|6|12|2|1956|227|2946|Tuesday|1956Q2|N|N|N|2435626|2435777|2435271|2435546|N|N|N|N|N| +2435638|AAAAAAAAGDKCFCAA|1956-06-13|677|2946|227|1956|3|6|13|2|1956|227|2946|Wednesday|1956Q2|N|N|N|2435626|2435777|2435272|2435547|N|N|N|N|N| +2435639|AAAAAAAAHDKCFCAA|1956-06-14|677|2946|227|1956|4|6|14|2|1956|227|2946|Thursday|1956Q2|N|N|N|2435626|2435777|2435273|2435548|N|N|N|N|N| +2435640|AAAAAAAAIDKCFCAA|1956-06-15|677|2946|227|1956|5|6|15|2|1956|227|2946|Friday|1956Q2|N|Y|N|2435626|2435777|2435274|2435549|N|N|N|N|N| +2435641|AAAAAAAAJDKCFCAA|1956-06-16|677|2946|227|1956|6|6|16|2|1956|227|2946|Saturday|1956Q2|N|Y|N|2435626|2435777|2435275|2435550|N|N|N|N|N| +2435642|AAAAAAAAKDKCFCAA|1956-06-17|677|2946|227|1956|0|6|17|2|1956|227|2946|Sunday|1956Q2|N|N|N|2435626|2435777|2435276|2435551|N|N|N|N|N| +2435643|AAAAAAAALDKCFCAA|1956-06-18|677|2946|227|1956|1|6|18|2|1956|227|2946|Monday|1956Q2|N|N|N|2435626|2435777|2435277|2435552|N|N|N|N|N| +2435644|AAAAAAAAMDKCFCAA|1956-06-19|677|2947|227|1956|2|6|19|2|1956|227|2947|Tuesday|1956Q2|N|N|N|2435626|2435777|2435278|2435553|N|N|N|N|N| +2435645|AAAAAAAANDKCFCAA|1956-06-20|677|2947|227|1956|3|6|20|2|1956|227|2947|Wednesday|1956Q2|N|N|N|2435626|2435777|2435279|2435554|N|N|N|N|N| +2435646|AAAAAAAAODKCFCAA|1956-06-21|677|2947|227|1956|4|6|21|2|1956|227|2947|Thursday|1956Q2|N|N|N|2435626|2435777|2435280|2435555|N|N|N|N|N| +2435647|AAAAAAAAPDKCFCAA|1956-06-22|677|2947|227|1956|5|6|22|2|1956|227|2947|Friday|1956Q2|N|Y|N|2435626|2435777|2435281|2435556|N|N|N|N|N| +2435648|AAAAAAAAAEKCFCAA|1956-06-23|677|2947|227|1956|6|6|23|2|1956|227|2947|Saturday|1956Q2|N|Y|N|2435626|2435777|2435282|2435557|N|N|N|N|N| +2435649|AAAAAAAABEKCFCAA|1956-06-24|677|2947|227|1956|0|6|24|2|1956|227|2947|Sunday|1956Q2|N|N|N|2435626|2435777|2435283|2435558|N|N|N|N|N| +2435650|AAAAAAAACEKCFCAA|1956-06-25|677|2947|227|1956|1|6|25|2|1956|227|2947|Monday|1956Q2|N|N|N|2435626|2435777|2435284|2435559|N|N|N|N|N| +2435651|AAAAAAAADEKCFCAA|1956-06-26|677|2948|227|1956|2|6|26|2|1956|227|2948|Tuesday|1956Q2|N|N|N|2435626|2435777|2435285|2435560|N|N|N|N|N| +2435652|AAAAAAAAEEKCFCAA|1956-06-27|677|2948|227|1956|3|6|27|2|1956|227|2948|Wednesday|1956Q2|N|N|N|2435626|2435777|2435286|2435561|N|N|N|N|N| +2435653|AAAAAAAAFEKCFCAA|1956-06-28|677|2948|227|1956|4|6|28|2|1956|227|2948|Thursday|1956Q2|N|N|N|2435626|2435777|2435287|2435562|N|N|N|N|N| +2435654|AAAAAAAAGEKCFCAA|1956-06-29|677|2948|227|1956|5|6|29|2|1956|227|2948|Friday|1956Q2|N|Y|N|2435626|2435777|2435288|2435563|N|N|N|N|N| +2435655|AAAAAAAAHEKCFCAA|1956-06-30|677|2948|227|1956|6|6|30|2|1956|227|2948|Saturday|1956Q2|N|Y|N|2435626|2435777|2435289|2435564|N|N|N|N|N| +2435656|AAAAAAAAIEKCFCAA|1956-07-01|678|2948|227|1956|0|7|1|3|1956|227|2948|Sunday|1956Q3|N|N|N|2435656|2435837|2435290|2435565|N|N|N|N|N| +2435657|AAAAAAAAJEKCFCAA|1956-07-02|678|2948|227|1956|1|7|2|3|1956|227|2948|Monday|1956Q3|N|N|N|2435656|2435837|2435291|2435566|N|N|N|N|N| +2435658|AAAAAAAAKEKCFCAA|1956-07-03|678|2949|227|1956|2|7|3|3|1956|227|2949|Tuesday|1956Q3|N|N|N|2435656|2435837|2435292|2435567|N|N|N|N|N| +2435659|AAAAAAAALEKCFCAA|1956-07-04|678|2949|227|1956|3|7|4|3|1956|227|2949|Wednesday|1956Q3|Y|N|N|2435656|2435837|2435293|2435568|N|N|N|N|N| +2435660|AAAAAAAAMEKCFCAA|1956-07-05|678|2949|227|1956|4|7|5|3|1956|227|2949|Thursday|1956Q3|N|N|Y|2435656|2435837|2435294|2435569|N|N|N|N|N| +2435661|AAAAAAAANEKCFCAA|1956-07-06|678|2949|227|1956|5|7|6|3|1956|227|2949|Friday|1956Q3|N|Y|N|2435656|2435837|2435295|2435570|N|N|N|N|N| +2435662|AAAAAAAAOEKCFCAA|1956-07-07|678|2949|227|1956|6|7|7|3|1956|227|2949|Saturday|1956Q3|N|Y|N|2435656|2435837|2435296|2435571|N|N|N|N|N| +2435663|AAAAAAAAPEKCFCAA|1956-07-08|678|2949|227|1956|0|7|8|3|1956|227|2949|Sunday|1956Q3|N|N|N|2435656|2435837|2435297|2435572|N|N|N|N|N| +2435664|AAAAAAAAAFKCFCAA|1956-07-09|678|2949|227|1956|1|7|9|3|1956|227|2949|Monday|1956Q3|N|N|N|2435656|2435837|2435298|2435573|N|N|N|N|N| +2435665|AAAAAAAABFKCFCAA|1956-07-10|678|2950|227|1956|2|7|10|3|1956|227|2950|Tuesday|1956Q3|N|N|N|2435656|2435837|2435299|2435574|N|N|N|N|N| +2435666|AAAAAAAACFKCFCAA|1956-07-11|678|2950|227|1956|3|7|11|3|1956|227|2950|Wednesday|1956Q3|N|N|N|2435656|2435837|2435300|2435575|N|N|N|N|N| +2435667|AAAAAAAADFKCFCAA|1956-07-12|678|2950|227|1956|4|7|12|3|1956|227|2950|Thursday|1956Q3|N|N|N|2435656|2435837|2435301|2435576|N|N|N|N|N| +2435668|AAAAAAAAEFKCFCAA|1956-07-13|678|2950|227|1956|5|7|13|3|1956|227|2950|Friday|1956Q3|N|Y|N|2435656|2435837|2435302|2435577|N|N|N|N|N| +2435669|AAAAAAAAFFKCFCAA|1956-07-14|678|2950|227|1956|6|7|14|3|1956|227|2950|Saturday|1956Q3|N|Y|N|2435656|2435837|2435303|2435578|N|N|N|N|N| +2435670|AAAAAAAAGFKCFCAA|1956-07-15|678|2950|227|1956|0|7|15|3|1956|227|2950|Sunday|1956Q3|N|N|N|2435656|2435837|2435304|2435579|N|N|N|N|N| +2435671|AAAAAAAAHFKCFCAA|1956-07-16|678|2950|227|1956|1|7|16|3|1956|227|2950|Monday|1956Q3|N|N|N|2435656|2435837|2435305|2435580|N|N|N|N|N| +2435672|AAAAAAAAIFKCFCAA|1956-07-17|678|2951|227|1956|2|7|17|3|1956|227|2951|Tuesday|1956Q3|N|N|N|2435656|2435837|2435306|2435581|N|N|N|N|N| +2435673|AAAAAAAAJFKCFCAA|1956-07-18|678|2951|227|1956|3|7|18|3|1956|227|2951|Wednesday|1956Q3|N|N|N|2435656|2435837|2435307|2435582|N|N|N|N|N| +2435674|AAAAAAAAKFKCFCAA|1956-07-19|678|2951|227|1956|4|7|19|3|1956|227|2951|Thursday|1956Q3|N|N|N|2435656|2435837|2435308|2435583|N|N|N|N|N| +2435675|AAAAAAAALFKCFCAA|1956-07-20|678|2951|227|1956|5|7|20|3|1956|227|2951|Friday|1956Q3|N|Y|N|2435656|2435837|2435309|2435584|N|N|N|N|N| +2435676|AAAAAAAAMFKCFCAA|1956-07-21|678|2951|227|1956|6|7|21|3|1956|227|2951|Saturday|1956Q3|N|Y|N|2435656|2435837|2435310|2435585|N|N|N|N|N| +2435677|AAAAAAAANFKCFCAA|1956-07-22|678|2951|227|1956|0|7|22|3|1956|227|2951|Sunday|1956Q3|N|N|N|2435656|2435837|2435311|2435586|N|N|N|N|N| +2435678|AAAAAAAAOFKCFCAA|1956-07-23|678|2951|227|1956|1|7|23|3|1956|227|2951|Monday|1956Q3|N|N|N|2435656|2435837|2435312|2435587|N|N|N|N|N| +2435679|AAAAAAAAPFKCFCAA|1956-07-24|678|2952|227|1956|2|7|24|3|1956|227|2952|Tuesday|1956Q3|N|N|N|2435656|2435837|2435313|2435588|N|N|N|N|N| +2435680|AAAAAAAAAGKCFCAA|1956-07-25|678|2952|227|1956|3|7|25|3|1956|227|2952|Wednesday|1956Q3|N|N|N|2435656|2435837|2435314|2435589|N|N|N|N|N| +2435681|AAAAAAAABGKCFCAA|1956-07-26|678|2952|227|1956|4|7|26|3|1956|227|2952|Thursday|1956Q3|N|N|N|2435656|2435837|2435315|2435590|N|N|N|N|N| +2435682|AAAAAAAACGKCFCAA|1956-07-27|678|2952|227|1956|5|7|27|3|1956|227|2952|Friday|1956Q3|N|Y|N|2435656|2435837|2435316|2435591|N|N|N|N|N| +2435683|AAAAAAAADGKCFCAA|1956-07-28|678|2952|227|1956|6|7|28|3|1956|227|2952|Saturday|1956Q3|N|Y|N|2435656|2435837|2435317|2435592|N|N|N|N|N| +2435684|AAAAAAAAEGKCFCAA|1956-07-29|678|2952|227|1956|0|7|29|3|1956|227|2952|Sunday|1956Q3|N|N|N|2435656|2435837|2435318|2435593|N|N|N|N|N| +2435685|AAAAAAAAFGKCFCAA|1956-07-30|678|2952|227|1956|1|7|30|3|1956|227|2952|Monday|1956Q3|N|N|N|2435656|2435837|2435319|2435594|N|N|N|N|N| +2435686|AAAAAAAAGGKCFCAA|1956-07-31|678|2953|227|1956|2|7|31|3|1956|227|2953|Tuesday|1956Q3|N|N|N|2435656|2435837|2435320|2435595|N|N|N|N|N| +2435687|AAAAAAAAHGKCFCAA|1956-08-01|679|2953|227|1956|3|8|1|3|1956|227|2953|Wednesday|1956Q3|N|N|N|2435687|2435899|2435321|2435596|N|N|N|N|N| +2435688|AAAAAAAAIGKCFCAA|1956-08-02|679|2953|227|1956|4|8|2|3|1956|227|2953|Thursday|1956Q3|N|N|N|2435687|2435899|2435322|2435597|N|N|N|N|N| +2435689|AAAAAAAAJGKCFCAA|1956-08-03|679|2953|227|1956|5|8|3|3|1956|227|2953|Friday|1956Q3|N|Y|N|2435687|2435899|2435323|2435598|N|N|N|N|N| +2435690|AAAAAAAAKGKCFCAA|1956-08-04|679|2953|227|1956|6|8|4|3|1956|227|2953|Saturday|1956Q3|N|Y|N|2435687|2435899|2435324|2435599|N|N|N|N|N| +2435691|AAAAAAAALGKCFCAA|1956-08-05|679|2953|227|1956|0|8|5|3|1956|227|2953|Sunday|1956Q3|N|N|N|2435687|2435899|2435325|2435600|N|N|N|N|N| +2435692|AAAAAAAAMGKCFCAA|1956-08-06|679|2953|227|1956|1|8|6|3|1956|227|2953|Monday|1956Q3|N|N|N|2435687|2435899|2435326|2435601|N|N|N|N|N| +2435693|AAAAAAAANGKCFCAA|1956-08-07|679|2954|227|1956|2|8|7|3|1956|227|2954|Tuesday|1956Q3|N|N|N|2435687|2435899|2435327|2435602|N|N|N|N|N| +2435694|AAAAAAAAOGKCFCAA|1956-08-08|679|2954|227|1956|3|8|8|3|1956|227|2954|Wednesday|1956Q3|N|N|N|2435687|2435899|2435328|2435603|N|N|N|N|N| +2435695|AAAAAAAAPGKCFCAA|1956-08-09|679|2954|227|1956|4|8|9|3|1956|227|2954|Thursday|1956Q3|N|N|N|2435687|2435899|2435329|2435604|N|N|N|N|N| +2435696|AAAAAAAAAHKCFCAA|1956-08-10|679|2954|227|1956|5|8|10|3|1956|227|2954|Friday|1956Q3|N|Y|N|2435687|2435899|2435330|2435605|N|N|N|N|N| +2435697|AAAAAAAABHKCFCAA|1956-08-11|679|2954|227|1956|6|8|11|3|1956|227|2954|Saturday|1956Q3|N|Y|N|2435687|2435899|2435331|2435606|N|N|N|N|N| +2435698|AAAAAAAACHKCFCAA|1956-08-12|679|2954|227|1956|0|8|12|3|1956|227|2954|Sunday|1956Q3|N|N|N|2435687|2435899|2435332|2435607|N|N|N|N|N| +2435699|AAAAAAAADHKCFCAA|1956-08-13|679|2954|227|1956|1|8|13|3|1956|227|2954|Monday|1956Q3|N|N|N|2435687|2435899|2435333|2435608|N|N|N|N|N| +2435700|AAAAAAAAEHKCFCAA|1956-08-14|679|2955|227|1956|2|8|14|3|1956|227|2955|Tuesday|1956Q3|N|N|N|2435687|2435899|2435334|2435609|N|N|N|N|N| +2435701|AAAAAAAAFHKCFCAA|1956-08-15|679|2955|227|1956|3|8|15|3|1956|227|2955|Wednesday|1956Q3|N|N|N|2435687|2435899|2435335|2435610|N|N|N|N|N| +2435702|AAAAAAAAGHKCFCAA|1956-08-16|679|2955|227|1956|4|8|16|3|1956|227|2955|Thursday|1956Q3|N|N|N|2435687|2435899|2435336|2435611|N|N|N|N|N| +2435703|AAAAAAAAHHKCFCAA|1956-08-17|679|2955|227|1956|5|8|17|3|1956|227|2955|Friday|1956Q3|N|Y|N|2435687|2435899|2435337|2435612|N|N|N|N|N| +2435704|AAAAAAAAIHKCFCAA|1956-08-18|679|2955|227|1956|6|8|18|3|1956|227|2955|Saturday|1956Q3|N|Y|N|2435687|2435899|2435338|2435613|N|N|N|N|N| +2435705|AAAAAAAAJHKCFCAA|1956-08-19|679|2955|227|1956|0|8|19|3|1956|227|2955|Sunday|1956Q3|N|N|N|2435687|2435899|2435339|2435614|N|N|N|N|N| +2435706|AAAAAAAAKHKCFCAA|1956-08-20|679|2955|227|1956|1|8|20|3|1956|227|2955|Monday|1956Q3|N|N|N|2435687|2435899|2435340|2435615|N|N|N|N|N| +2435707|AAAAAAAALHKCFCAA|1956-08-21|679|2956|227|1956|2|8|21|3|1956|227|2956|Tuesday|1956Q3|N|N|N|2435687|2435899|2435341|2435616|N|N|N|N|N| +2435708|AAAAAAAAMHKCFCAA|1956-08-22|679|2956|227|1956|3|8|22|3|1956|227|2956|Wednesday|1956Q3|N|N|N|2435687|2435899|2435342|2435617|N|N|N|N|N| +2435709|AAAAAAAANHKCFCAA|1956-08-23|679|2956|227|1956|4|8|23|3|1956|227|2956|Thursday|1956Q3|N|N|N|2435687|2435899|2435343|2435618|N|N|N|N|N| +2435710|AAAAAAAAOHKCFCAA|1956-08-24|679|2956|227|1956|5|8|24|3|1956|227|2956|Friday|1956Q3|N|Y|N|2435687|2435899|2435344|2435619|N|N|N|N|N| +2435711|AAAAAAAAPHKCFCAA|1956-08-25|679|2956|227|1956|6|8|25|3|1956|227|2956|Saturday|1956Q3|N|Y|N|2435687|2435899|2435345|2435620|N|N|N|N|N| +2435712|AAAAAAAAAIKCFCAA|1956-08-26|679|2956|227|1956|0|8|26|3|1956|227|2956|Sunday|1956Q3|N|N|N|2435687|2435899|2435346|2435621|N|N|N|N|N| +2435713|AAAAAAAABIKCFCAA|1956-08-27|679|2956|227|1956|1|8|27|3|1956|227|2956|Monday|1956Q3|N|N|N|2435687|2435899|2435347|2435622|N|N|N|N|N| +2435714|AAAAAAAACIKCFCAA|1956-08-28|679|2957|227|1956|2|8|28|3|1956|227|2957|Tuesday|1956Q3|N|N|N|2435687|2435899|2435348|2435623|N|N|N|N|N| +2435715|AAAAAAAADIKCFCAA|1956-08-29|679|2957|227|1956|3|8|29|3|1956|227|2957|Wednesday|1956Q3|N|N|N|2435687|2435899|2435349|2435624|N|N|N|N|N| +2435716|AAAAAAAAEIKCFCAA|1956-08-30|679|2957|227|1956|4|8|30|3|1956|227|2957|Thursday|1956Q3|N|N|N|2435687|2435899|2435350|2435625|N|N|N|N|N| +2435717|AAAAAAAAFIKCFCAA|1956-08-31|679|2957|227|1956|5|8|31|3|1956|227|2957|Friday|1956Q3|N|Y|N|2435687|2435899|2435351|2435626|N|N|N|N|N| +2435718|AAAAAAAAGIKCFCAA|1956-09-01|680|2957|228|1956|6|9|1|3|1956|228|2957|Saturday|1956Q3|N|Y|N|2435718|2435961|2435352|2435627|N|N|N|N|N| +2435719|AAAAAAAAHIKCFCAA|1956-09-02|680|2957|228|1956|0|9|2|3|1956|228|2957|Sunday|1956Q3|N|N|N|2435718|2435961|2435353|2435628|N|N|N|N|N| +2435720|AAAAAAAAIIKCFCAA|1956-09-03|680|2957|228|1956|1|9|3|3|1956|228|2957|Monday|1956Q3|N|N|N|2435718|2435961|2435354|2435629|N|N|N|N|N| +2435721|AAAAAAAAJIKCFCAA|1956-09-04|680|2958|228|1956|2|9|4|3|1956|228|2958|Tuesday|1956Q3|N|N|N|2435718|2435961|2435355|2435630|N|N|N|N|N| +2435722|AAAAAAAAKIKCFCAA|1956-09-05|680|2958|228|1956|3|9|5|3|1956|228|2958|Wednesday|1956Q3|N|N|N|2435718|2435961|2435356|2435631|N|N|N|N|N| +2435723|AAAAAAAALIKCFCAA|1956-09-06|680|2958|228|1956|4|9|6|3|1956|228|2958|Thursday|1956Q3|N|N|N|2435718|2435961|2435357|2435632|N|N|N|N|N| +2435724|AAAAAAAAMIKCFCAA|1956-09-07|680|2958|228|1956|5|9|7|3|1956|228|2958|Friday|1956Q3|N|Y|N|2435718|2435961|2435358|2435633|N|N|N|N|N| +2435725|AAAAAAAANIKCFCAA|1956-09-08|680|2958|228|1956|6|9|8|3|1956|228|2958|Saturday|1956Q3|N|Y|N|2435718|2435961|2435359|2435634|N|N|N|N|N| +2435726|AAAAAAAAOIKCFCAA|1956-09-09|680|2958|228|1956|0|9|9|3|1956|228|2958|Sunday|1956Q3|N|N|N|2435718|2435961|2435360|2435635|N|N|N|N|N| +2435727|AAAAAAAAPIKCFCAA|1956-09-10|680|2958|228|1956|1|9|10|3|1956|228|2958|Monday|1956Q3|N|N|N|2435718|2435961|2435361|2435636|N|N|N|N|N| +2435728|AAAAAAAAAJKCFCAA|1956-09-11|680|2959|228|1956|2|9|11|3|1956|228|2959|Tuesday|1956Q3|N|N|N|2435718|2435961|2435362|2435637|N|N|N|N|N| +2435729|AAAAAAAABJKCFCAA|1956-09-12|680|2959|228|1956|3|9|12|3|1956|228|2959|Wednesday|1956Q3|N|N|N|2435718|2435961|2435363|2435638|N|N|N|N|N| +2435730|AAAAAAAACJKCFCAA|1956-09-13|680|2959|228|1956|4|9|13|3|1956|228|2959|Thursday|1956Q3|N|N|N|2435718|2435961|2435364|2435639|N|N|N|N|N| +2435731|AAAAAAAADJKCFCAA|1956-09-14|680|2959|228|1956|5|9|14|3|1956|228|2959|Friday|1956Q3|N|Y|N|2435718|2435961|2435365|2435640|N|N|N|N|N| +2435732|AAAAAAAAEJKCFCAA|1956-09-15|680|2959|228|1956|6|9|15|3|1956|228|2959|Saturday|1956Q3|N|Y|N|2435718|2435961|2435366|2435641|N|N|N|N|N| +2435733|AAAAAAAAFJKCFCAA|1956-09-16|680|2959|228|1956|0|9|16|3|1956|228|2959|Sunday|1956Q3|N|N|N|2435718|2435961|2435367|2435642|N|N|N|N|N| +2435734|AAAAAAAAGJKCFCAA|1956-09-17|680|2959|228|1956|1|9|17|3|1956|228|2959|Monday|1956Q3|N|N|N|2435718|2435961|2435368|2435643|N|N|N|N|N| +2435735|AAAAAAAAHJKCFCAA|1956-09-18|680|2960|228|1956|2|9|18|3|1956|228|2960|Tuesday|1956Q3|N|N|N|2435718|2435961|2435369|2435644|N|N|N|N|N| +2435736|AAAAAAAAIJKCFCAA|1956-09-19|680|2960|228|1956|3|9|19|3|1956|228|2960|Wednesday|1956Q3|N|N|N|2435718|2435961|2435370|2435645|N|N|N|N|N| +2435737|AAAAAAAAJJKCFCAA|1956-09-20|680|2960|228|1956|4|9|20|3|1956|228|2960|Thursday|1956Q3|N|N|N|2435718|2435961|2435371|2435646|N|N|N|N|N| +2435738|AAAAAAAAKJKCFCAA|1956-09-21|680|2960|228|1956|5|9|21|3|1956|228|2960|Friday|1956Q3|N|Y|N|2435718|2435961|2435372|2435647|N|N|N|N|N| +2435739|AAAAAAAALJKCFCAA|1956-09-22|680|2960|228|1956|6|9|22|3|1956|228|2960|Saturday|1956Q3|N|Y|N|2435718|2435961|2435373|2435648|N|N|N|N|N| +2435740|AAAAAAAAMJKCFCAA|1956-09-23|680|2960|228|1956|0|9|23|3|1956|228|2960|Sunday|1956Q3|N|N|N|2435718|2435961|2435374|2435649|N|N|N|N|N| +2435741|AAAAAAAANJKCFCAA|1956-09-24|680|2960|228|1956|1|9|24|3|1956|228|2960|Monday|1956Q3|N|N|N|2435718|2435961|2435375|2435650|N|N|N|N|N| +2435742|AAAAAAAAOJKCFCAA|1956-09-25|680|2961|228|1956|2|9|25|3|1956|228|2961|Tuesday|1956Q3|N|N|N|2435718|2435961|2435376|2435651|N|N|N|N|N| +2435743|AAAAAAAAPJKCFCAA|1956-09-26|680|2961|228|1956|3|9|26|3|1956|228|2961|Wednesday|1956Q3|N|N|N|2435718|2435961|2435377|2435652|N|N|N|N|N| +2435744|AAAAAAAAAKKCFCAA|1956-09-27|680|2961|228|1956|4|9|27|3|1956|228|2961|Thursday|1956Q3|N|N|N|2435718|2435961|2435378|2435653|N|N|N|N|N| +2435745|AAAAAAAABKKCFCAA|1956-09-28|680|2961|228|1956|5|9|28|3|1956|228|2961|Friday|1956Q3|N|Y|N|2435718|2435961|2435379|2435654|N|N|N|N|N| +2435746|AAAAAAAACKKCFCAA|1956-09-29|680|2961|228|1956|6|9|29|3|1956|228|2961|Saturday|1956Q3|N|Y|N|2435718|2435961|2435380|2435655|N|N|N|N|N| +2435747|AAAAAAAADKKCFCAA|1956-09-30|680|2961|228|1956|0|9|30|3|1956|228|2961|Sunday|1956Q3|N|N|N|2435718|2435961|2435381|2435656|N|N|N|N|N| +2435748|AAAAAAAAEKKCFCAA|1956-10-01|681|2961|228|1956|1|10|1|4|1956|228|2961|Monday|1956Q4|N|N|N|2435748|2436021|2435382|2435656|N|N|N|N|N| +2435749|AAAAAAAAFKKCFCAA|1956-10-02|681|2962|228|1956|2|10|2|4|1956|228|2962|Tuesday|1956Q4|N|N|N|2435748|2436021|2435383|2435657|N|N|N|N|N| +2435750|AAAAAAAAGKKCFCAA|1956-10-03|681|2962|228|1956|3|10|3|4|1956|228|2962|Wednesday|1956Q4|N|N|N|2435748|2436021|2435384|2435658|N|N|N|N|N| +2435751|AAAAAAAAHKKCFCAA|1956-10-04|681|2962|228|1956|4|10|4|4|1956|228|2962|Thursday|1956Q4|N|N|N|2435748|2436021|2435385|2435659|N|N|N|N|N| +2435752|AAAAAAAAIKKCFCAA|1956-10-05|681|2962|228|1956|5|10|5|4|1956|228|2962|Friday|1956Q4|N|Y|N|2435748|2436021|2435386|2435660|N|N|N|N|N| +2435753|AAAAAAAAJKKCFCAA|1956-10-06|681|2962|228|1956|6|10|6|4|1956|228|2962|Saturday|1956Q4|N|Y|N|2435748|2436021|2435387|2435661|N|N|N|N|N| +2435754|AAAAAAAAKKKCFCAA|1956-10-07|681|2962|228|1956|0|10|7|4|1956|228|2962|Sunday|1956Q4|N|N|N|2435748|2436021|2435388|2435662|N|N|N|N|N| +2435755|AAAAAAAALKKCFCAA|1956-10-08|681|2962|228|1956|1|10|8|4|1956|228|2962|Monday|1956Q4|N|N|N|2435748|2436021|2435389|2435663|N|N|N|N|N| +2435756|AAAAAAAAMKKCFCAA|1956-10-09|681|2963|228|1956|2|10|9|4|1956|228|2963|Tuesday|1956Q4|N|N|N|2435748|2436021|2435390|2435664|N|N|N|N|N| +2435757|AAAAAAAANKKCFCAA|1956-10-10|681|2963|228|1956|3|10|10|4|1956|228|2963|Wednesday|1956Q4|N|N|N|2435748|2436021|2435391|2435665|N|N|N|N|N| +2435758|AAAAAAAAOKKCFCAA|1956-10-11|681|2963|228|1956|4|10|11|4|1956|228|2963|Thursday|1956Q4|N|N|N|2435748|2436021|2435392|2435666|N|N|N|N|N| +2435759|AAAAAAAAPKKCFCAA|1956-10-12|681|2963|228|1956|5|10|12|4|1956|228|2963|Friday|1956Q4|N|Y|N|2435748|2436021|2435393|2435667|N|N|N|N|N| +2435760|AAAAAAAAALKCFCAA|1956-10-13|681|2963|228|1956|6|10|13|4|1956|228|2963|Saturday|1956Q4|N|Y|N|2435748|2436021|2435394|2435668|N|N|N|N|N| +2435761|AAAAAAAABLKCFCAA|1956-10-14|681|2963|228|1956|0|10|14|4|1956|228|2963|Sunday|1956Q4|N|N|N|2435748|2436021|2435395|2435669|N|N|N|N|N| +2435762|AAAAAAAACLKCFCAA|1956-10-15|681|2963|228|1956|1|10|15|4|1956|228|2963|Monday|1956Q4|N|N|N|2435748|2436021|2435396|2435670|N|N|N|N|N| +2435763|AAAAAAAADLKCFCAA|1956-10-16|681|2964|228|1956|2|10|16|4|1956|228|2964|Tuesday|1956Q4|N|N|N|2435748|2436021|2435397|2435671|N|N|N|N|N| +2435764|AAAAAAAAELKCFCAA|1956-10-17|681|2964|228|1956|3|10|17|4|1956|228|2964|Wednesday|1956Q4|N|N|N|2435748|2436021|2435398|2435672|N|N|N|N|N| +2435765|AAAAAAAAFLKCFCAA|1956-10-18|681|2964|228|1956|4|10|18|4|1956|228|2964|Thursday|1956Q4|N|N|N|2435748|2436021|2435399|2435673|N|N|N|N|N| +2435766|AAAAAAAAGLKCFCAA|1956-10-19|681|2964|228|1956|5|10|19|4|1956|228|2964|Friday|1956Q4|N|Y|N|2435748|2436021|2435400|2435674|N|N|N|N|N| +2435767|AAAAAAAAHLKCFCAA|1956-10-20|681|2964|228|1956|6|10|20|4|1956|228|2964|Saturday|1956Q4|N|Y|N|2435748|2436021|2435401|2435675|N|N|N|N|N| +2435768|AAAAAAAAILKCFCAA|1956-10-21|681|2964|228|1956|0|10|21|4|1956|228|2964|Sunday|1956Q4|N|N|N|2435748|2436021|2435402|2435676|N|N|N|N|N| +2435769|AAAAAAAAJLKCFCAA|1956-10-22|681|2964|228|1956|1|10|22|4|1956|228|2964|Monday|1956Q4|N|N|N|2435748|2436021|2435403|2435677|N|N|N|N|N| +2435770|AAAAAAAAKLKCFCAA|1956-10-23|681|2965|228|1956|2|10|23|4|1956|228|2965|Tuesday|1956Q4|N|N|N|2435748|2436021|2435404|2435678|N|N|N|N|N| +2435771|AAAAAAAALLKCFCAA|1956-10-24|681|2965|228|1956|3|10|24|4|1956|228|2965|Wednesday|1956Q4|N|N|N|2435748|2436021|2435405|2435679|N|N|N|N|N| +2435772|AAAAAAAAMLKCFCAA|1956-10-25|681|2965|228|1956|4|10|25|4|1956|228|2965|Thursday|1956Q4|N|N|N|2435748|2436021|2435406|2435680|N|N|N|N|N| +2435773|AAAAAAAANLKCFCAA|1956-10-26|681|2965|228|1956|5|10|26|4|1956|228|2965|Friday|1956Q4|N|Y|N|2435748|2436021|2435407|2435681|N|N|N|N|N| +2435774|AAAAAAAAOLKCFCAA|1956-10-27|681|2965|228|1956|6|10|27|4|1956|228|2965|Saturday|1956Q4|N|Y|N|2435748|2436021|2435408|2435682|N|N|N|N|N| +2435775|AAAAAAAAPLKCFCAA|1956-10-28|681|2965|228|1956|0|10|28|4|1956|228|2965|Sunday|1956Q4|N|N|N|2435748|2436021|2435409|2435683|N|N|N|N|N| +2435776|AAAAAAAAAMKCFCAA|1956-10-29|681|2965|228|1956|1|10|29|4|1956|228|2965|Monday|1956Q4|N|N|N|2435748|2436021|2435410|2435684|N|N|N|N|N| +2435777|AAAAAAAABMKCFCAA|1956-10-30|681|2966|228|1956|2|10|30|4|1956|228|2966|Tuesday|1956Q4|N|N|N|2435748|2436021|2435411|2435685|N|N|N|N|N| +2435778|AAAAAAAACMKCFCAA|1956-10-31|681|2966|228|1956|3|10|31|4|1956|228|2966|Wednesday|1956Q4|N|N|N|2435748|2436021|2435412|2435686|N|N|N|N|N| +2435779|AAAAAAAADMKCFCAA|1956-11-01|682|2966|228|1956|4|11|1|4|1956|228|2966|Thursday|1956Q4|N|N|N|2435779|2436083|2435413|2435687|N|N|N|N|N| +2435780|AAAAAAAAEMKCFCAA|1956-11-02|682|2966|228|1956|5|11|2|4|1956|228|2966|Friday|1956Q4|N|Y|N|2435779|2436083|2435414|2435688|N|N|N|N|N| +2435781|AAAAAAAAFMKCFCAA|1956-11-03|682|2966|228|1956|6|11|3|4|1956|228|2966|Saturday|1956Q4|N|Y|N|2435779|2436083|2435415|2435689|N|N|N|N|N| +2435782|AAAAAAAAGMKCFCAA|1956-11-04|682|2966|228|1956|0|11|4|4|1956|228|2966|Sunday|1956Q4|N|N|N|2435779|2436083|2435416|2435690|N|N|N|N|N| +2435783|AAAAAAAAHMKCFCAA|1956-11-05|682|2966|228|1956|1|11|5|4|1956|228|2966|Monday|1956Q4|N|N|N|2435779|2436083|2435417|2435691|N|N|N|N|N| +2435784|AAAAAAAAIMKCFCAA|1956-11-06|682|2967|228|1956|2|11|6|4|1956|228|2967|Tuesday|1956Q4|N|N|N|2435779|2436083|2435418|2435692|N|N|N|N|N| +2435785|AAAAAAAAJMKCFCAA|1956-11-07|682|2967|228|1956|3|11|7|4|1956|228|2967|Wednesday|1956Q4|N|N|N|2435779|2436083|2435419|2435693|N|N|N|N|N| +2435786|AAAAAAAAKMKCFCAA|1956-11-08|682|2967|228|1956|4|11|8|4|1956|228|2967|Thursday|1956Q4|N|N|N|2435779|2436083|2435420|2435694|N|N|N|N|N| +2435787|AAAAAAAALMKCFCAA|1956-11-09|682|2967|228|1956|5|11|9|4|1956|228|2967|Friday|1956Q4|N|Y|N|2435779|2436083|2435421|2435695|N|N|N|N|N| +2435788|AAAAAAAAMMKCFCAA|1956-11-10|682|2967|228|1956|6|11|10|4|1956|228|2967|Saturday|1956Q4|N|Y|N|2435779|2436083|2435422|2435696|N|N|N|N|N| +2435789|AAAAAAAANMKCFCAA|1956-11-11|682|2967|228|1956|0|11|11|4|1956|228|2967|Sunday|1956Q4|N|N|N|2435779|2436083|2435423|2435697|N|N|N|N|N| +2435790|AAAAAAAAOMKCFCAA|1956-11-12|682|2967|228|1956|1|11|12|4|1956|228|2967|Monday|1956Q4|N|N|N|2435779|2436083|2435424|2435698|N|N|N|N|N| +2435791|AAAAAAAAPMKCFCAA|1956-11-13|682|2968|228|1956|2|11|13|4|1956|228|2968|Tuesday|1956Q4|N|N|N|2435779|2436083|2435425|2435699|N|N|N|N|N| +2435792|AAAAAAAAANKCFCAA|1956-11-14|682|2968|228|1956|3|11|14|4|1956|228|2968|Wednesday|1956Q4|N|N|N|2435779|2436083|2435426|2435700|N|N|N|N|N| +2435793|AAAAAAAABNKCFCAA|1956-11-15|682|2968|228|1956|4|11|15|4|1956|228|2968|Thursday|1956Q4|N|N|N|2435779|2436083|2435427|2435701|N|N|N|N|N| +2435794|AAAAAAAACNKCFCAA|1956-11-16|682|2968|228|1956|5|11|16|4|1956|228|2968|Friday|1956Q4|N|Y|N|2435779|2436083|2435428|2435702|N|N|N|N|N| +2435795|AAAAAAAADNKCFCAA|1956-11-17|682|2968|228|1956|6|11|17|4|1956|228|2968|Saturday|1956Q4|N|Y|N|2435779|2436083|2435429|2435703|N|N|N|N|N| +2435796|AAAAAAAAENKCFCAA|1956-11-18|682|2968|228|1956|0|11|18|4|1956|228|2968|Sunday|1956Q4|N|N|N|2435779|2436083|2435430|2435704|N|N|N|N|N| +2435797|AAAAAAAAFNKCFCAA|1956-11-19|682|2968|228|1956|1|11|19|4|1956|228|2968|Monday|1956Q4|N|N|N|2435779|2436083|2435431|2435705|N|N|N|N|N| +2435798|AAAAAAAAGNKCFCAA|1956-11-20|682|2969|228|1956|2|11|20|4|1956|228|2969|Tuesday|1956Q4|N|N|N|2435779|2436083|2435432|2435706|N|N|N|N|N| +2435799|AAAAAAAAHNKCFCAA|1956-11-21|682|2969|228|1956|3|11|21|4|1956|228|2969|Wednesday|1956Q4|N|N|N|2435779|2436083|2435433|2435707|N|N|N|N|N| +2435800|AAAAAAAAINKCFCAA|1956-11-22|682|2969|228|1956|4|11|22|4|1956|228|2969|Thursday|1956Q4|N|N|N|2435779|2436083|2435434|2435708|N|N|N|N|N| +2435801|AAAAAAAAJNKCFCAA|1956-11-23|682|2969|228|1956|5|11|23|4|1956|228|2969|Friday|1956Q4|N|Y|N|2435779|2436083|2435435|2435709|N|N|N|N|N| +2435802|AAAAAAAAKNKCFCAA|1956-11-24|682|2969|228|1956|6|11|24|4|1956|228|2969|Saturday|1956Q4|N|Y|N|2435779|2436083|2435436|2435710|N|N|N|N|N| +2435803|AAAAAAAALNKCFCAA|1956-11-25|682|2969|228|1956|0|11|25|4|1956|228|2969|Sunday|1956Q4|N|N|N|2435779|2436083|2435437|2435711|N|N|N|N|N| +2435804|AAAAAAAAMNKCFCAA|1956-11-26|682|2969|228|1956|1|11|26|4|1956|228|2969|Monday|1956Q4|N|N|N|2435779|2436083|2435438|2435712|N|N|N|N|N| +2435805|AAAAAAAANNKCFCAA|1956-11-27|682|2970|228|1956|2|11|27|4|1956|228|2970|Tuesday|1956Q4|N|N|N|2435779|2436083|2435439|2435713|N|N|N|N|N| +2435806|AAAAAAAAONKCFCAA|1956-11-28|682|2970|228|1956|3|11|28|4|1956|228|2970|Wednesday|1956Q4|N|N|N|2435779|2436083|2435440|2435714|N|N|N|N|N| +2435807|AAAAAAAAPNKCFCAA|1956-11-29|682|2970|228|1956|4|11|29|4|1956|228|2970|Thursday|1956Q4|N|N|N|2435779|2436083|2435441|2435715|N|N|N|N|N| +2435808|AAAAAAAAAOKCFCAA|1956-11-30|682|2970|228|1956|5|11|30|4|1956|228|2970|Friday|1956Q4|N|Y|N|2435779|2436083|2435442|2435716|N|N|N|N|N| +2435809|AAAAAAAABOKCFCAA|1956-12-01|683|2970|229|1956|6|12|1|4|1956|229|2970|Saturday|1956Q4|N|Y|N|2435809|2436143|2435443|2435717|N|N|N|N|N| +2435810|AAAAAAAACOKCFCAA|1956-12-02|683|2970|229|1956|0|12|2|4|1956|229|2970|Sunday|1956Q4|N|N|N|2435809|2436143|2435444|2435718|N|N|N|N|N| +2435811|AAAAAAAADOKCFCAA|1956-12-03|683|2970|229|1956|1|12|3|4|1956|229|2970|Monday|1956Q4|N|N|N|2435809|2436143|2435445|2435719|N|N|N|N|N| +2435812|AAAAAAAAEOKCFCAA|1956-12-04|683|2971|229|1956|2|12|4|4|1956|229|2971|Tuesday|1956Q4|N|N|N|2435809|2436143|2435446|2435720|N|N|N|N|N| +2435813|AAAAAAAAFOKCFCAA|1956-12-05|683|2971|229|1956|3|12|5|4|1956|229|2971|Wednesday|1956Q4|N|N|N|2435809|2436143|2435447|2435721|N|N|N|N|N| +2435814|AAAAAAAAGOKCFCAA|1956-12-06|683|2971|229|1956|4|12|6|4|1956|229|2971|Thursday|1956Q4|N|N|N|2435809|2436143|2435448|2435722|N|N|N|N|N| +2435815|AAAAAAAAHOKCFCAA|1956-12-07|683|2971|229|1956|5|12|7|4|1956|229|2971|Friday|1956Q4|N|Y|N|2435809|2436143|2435449|2435723|N|N|N|N|N| +2435816|AAAAAAAAIOKCFCAA|1956-12-08|683|2971|229|1956|6|12|8|4|1956|229|2971|Saturday|1956Q4|N|Y|N|2435809|2436143|2435450|2435724|N|N|N|N|N| +2435817|AAAAAAAAJOKCFCAA|1956-12-09|683|2971|229|1956|0|12|9|4|1956|229|2971|Sunday|1956Q4|N|N|N|2435809|2436143|2435451|2435725|N|N|N|N|N| +2435818|AAAAAAAAKOKCFCAA|1956-12-10|683|2971|229|1956|1|12|10|4|1956|229|2971|Monday|1956Q4|N|N|N|2435809|2436143|2435452|2435726|N|N|N|N|N| +2435819|AAAAAAAALOKCFCAA|1956-12-11|683|2972|229|1956|2|12|11|4|1956|229|2972|Tuesday|1956Q4|N|N|N|2435809|2436143|2435453|2435727|N|N|N|N|N| +2435820|AAAAAAAAMOKCFCAA|1956-12-12|683|2972|229|1956|3|12|12|4|1956|229|2972|Wednesday|1956Q4|N|N|N|2435809|2436143|2435454|2435728|N|N|N|N|N| +2435821|AAAAAAAANOKCFCAA|1956-12-13|683|2972|229|1956|4|12|13|4|1956|229|2972|Thursday|1956Q4|N|N|N|2435809|2436143|2435455|2435729|N|N|N|N|N| +2435822|AAAAAAAAOOKCFCAA|1956-12-14|683|2972|229|1956|5|12|14|4|1956|229|2972|Friday|1956Q4|N|Y|N|2435809|2436143|2435456|2435730|N|N|N|N|N| +2435823|AAAAAAAAPOKCFCAA|1956-12-15|683|2972|229|1956|6|12|15|4|1956|229|2972|Saturday|1956Q4|N|Y|N|2435809|2436143|2435457|2435731|N|N|N|N|N| +2435824|AAAAAAAAAPKCFCAA|1956-12-16|683|2972|229|1956|0|12|16|4|1956|229|2972|Sunday|1956Q4|N|N|N|2435809|2436143|2435458|2435732|N|N|N|N|N| +2435825|AAAAAAAABPKCFCAA|1956-12-17|683|2972|229|1956|1|12|17|4|1956|229|2972|Monday|1956Q4|N|N|N|2435809|2436143|2435459|2435733|N|N|N|N|N| +2435826|AAAAAAAACPKCFCAA|1956-12-18|683|2973|229|1956|2|12|18|4|1956|229|2973|Tuesday|1956Q4|N|N|N|2435809|2436143|2435460|2435734|N|N|N|N|N| +2435827|AAAAAAAADPKCFCAA|1956-12-19|683|2973|229|1956|3|12|19|4|1956|229|2973|Wednesday|1956Q4|N|N|N|2435809|2436143|2435461|2435735|N|N|N|N|N| +2435828|AAAAAAAAEPKCFCAA|1956-12-20|683|2973|229|1956|4|12|20|4|1956|229|2973|Thursday|1956Q4|N|N|N|2435809|2436143|2435462|2435736|N|N|N|N|N| +2435829|AAAAAAAAFPKCFCAA|1956-12-21|683|2973|229|1956|5|12|21|4|1956|229|2973|Friday|1956Q4|N|Y|N|2435809|2436143|2435463|2435737|N|N|N|N|N| +2435830|AAAAAAAAGPKCFCAA|1956-12-22|683|2973|229|1956|6|12|22|4|1956|229|2973|Saturday|1956Q4|N|Y|N|2435809|2436143|2435464|2435738|N|N|N|N|N| +2435831|AAAAAAAAHPKCFCAA|1956-12-23|683|2973|229|1956|0|12|23|4|1956|229|2973|Sunday|1956Q4|N|N|N|2435809|2436143|2435465|2435739|N|N|N|N|N| +2435832|AAAAAAAAIPKCFCAA|1956-12-24|683|2973|229|1956|1|12|24|4|1956|229|2973|Monday|1956Q4|N|N|N|2435809|2436143|2435466|2435740|N|N|N|N|N| +2435833|AAAAAAAAJPKCFCAA|1956-12-25|683|2974|229|1956|2|12|25|4|1956|229|2974|Tuesday|1956Q4|Y|N|N|2435809|2436143|2435467|2435741|N|N|N|N|N| +2435834|AAAAAAAAKPKCFCAA|1956-12-26|683|2974|229|1956|3|12|26|4|1956|229|2974|Wednesday|1956Q4|N|N|Y|2435809|2436143|2435468|2435742|N|N|N|N|N| +2435835|AAAAAAAALPKCFCAA|1956-12-27|683|2974|229|1956|4|12|27|4|1956|229|2974|Thursday|1956Q4|N|N|N|2435809|2436143|2435469|2435743|N|N|N|N|N| +2435836|AAAAAAAAMPKCFCAA|1956-12-28|683|2974|229|1956|5|12|28|4|1956|229|2974|Friday|1956Q4|N|Y|N|2435809|2436143|2435470|2435744|N|N|N|N|N| +2435837|AAAAAAAANPKCFCAA|1956-12-29|683|2974|229|1956|6|12|29|4|1956|229|2974|Saturday|1956Q4|N|Y|N|2435809|2436143|2435471|2435745|N|N|N|N|N| +2435838|AAAAAAAAOPKCFCAA|1956-12-30|683|2974|229|1956|0|12|30|4|1956|229|2974|Sunday|1956Q4|N|N|N|2435809|2436143|2435472|2435746|N|N|N|N|N| +2435839|AAAAAAAAPPKCFCAA|1956-12-31|683|2974|229|1956|1|12|31|4|1956|229|2974|Monday|1956Q4|Y|N|N|2435809|2436143|2435473|2435747|N|N|N|N|N| +2435840|AAAAAAAAAALCFCAA|1957-01-01|684|2975|229|1957|2|1|1|1|1957|229|2975|Tuesday|1957Q1|Y|N|Y|2435840|2435839|2435474|2435748|N|N|N|N|N| +2435841|AAAAAAAABALCFCAA|1957-01-02|684|2975|229|1957|3|1|2|1|1957|229|2975|Wednesday|1957Q1|N|N|Y|2435840|2435839|2435475|2435749|N|N|N|N|N| +2435842|AAAAAAAACALCFCAA|1957-01-03|684|2975|229|1957|4|1|3|1|1957|229|2975|Thursday|1957Q1|N|N|N|2435840|2435839|2435476|2435750|N|N|N|N|N| +2435843|AAAAAAAADALCFCAA|1957-01-04|684|2975|229|1957|5|1|4|1|1957|229|2975|Friday|1957Q1|N|Y|N|2435840|2435839|2435477|2435751|N|N|N|N|N| +2435844|AAAAAAAAEALCFCAA|1957-01-05|684|2975|229|1957|6|1|5|1|1957|229|2975|Saturday|1957Q1|N|Y|N|2435840|2435839|2435478|2435752|N|N|N|N|N| +2435845|AAAAAAAAFALCFCAA|1957-01-06|684|2975|229|1957|0|1|6|1|1957|229|2975|Sunday|1957Q1|N|N|N|2435840|2435839|2435479|2435753|N|N|N|N|N| +2435846|AAAAAAAAGALCFCAA|1957-01-07|684|2975|229|1957|1|1|7|1|1957|229|2975|Monday|1957Q1|N|N|N|2435840|2435839|2435480|2435754|N|N|N|N|N| +2435847|AAAAAAAAHALCFCAA|1957-01-08|684|2976|229|1957|2|1|8|1|1957|229|2976|Tuesday|1957Q1|N|N|N|2435840|2435839|2435481|2435755|N|N|N|N|N| +2435848|AAAAAAAAIALCFCAA|1957-01-09|684|2976|229|1957|3|1|9|1|1957|229|2976|Wednesday|1957Q1|N|N|N|2435840|2435839|2435482|2435756|N|N|N|N|N| +2435849|AAAAAAAAJALCFCAA|1957-01-10|684|2976|229|1957|4|1|10|1|1957|229|2976|Thursday|1957Q1|N|N|N|2435840|2435839|2435483|2435757|N|N|N|N|N| +2435850|AAAAAAAAKALCFCAA|1957-01-11|684|2976|229|1957|5|1|11|1|1957|229|2976|Friday|1957Q1|N|Y|N|2435840|2435839|2435484|2435758|N|N|N|N|N| +2435851|AAAAAAAALALCFCAA|1957-01-12|684|2976|229|1957|6|1|12|1|1957|229|2976|Saturday|1957Q1|N|Y|N|2435840|2435839|2435485|2435759|N|N|N|N|N| +2435852|AAAAAAAAMALCFCAA|1957-01-13|684|2976|229|1957|0|1|13|1|1957|229|2976|Sunday|1957Q1|N|N|N|2435840|2435839|2435486|2435760|N|N|N|N|N| +2435853|AAAAAAAANALCFCAA|1957-01-14|684|2976|229|1957|1|1|14|1|1957|229|2976|Monday|1957Q1|N|N|N|2435840|2435839|2435487|2435761|N|N|N|N|N| +2435854|AAAAAAAAOALCFCAA|1957-01-15|684|2977|229|1957|2|1|15|1|1957|229|2977|Tuesday|1957Q1|N|N|N|2435840|2435839|2435488|2435762|N|N|N|N|N| +2435855|AAAAAAAAPALCFCAA|1957-01-16|684|2977|229|1957|3|1|16|1|1957|229|2977|Wednesday|1957Q1|N|N|N|2435840|2435839|2435489|2435763|N|N|N|N|N| +2435856|AAAAAAAAABLCFCAA|1957-01-17|684|2977|229|1957|4|1|17|1|1957|229|2977|Thursday|1957Q1|N|N|N|2435840|2435839|2435490|2435764|N|N|N|N|N| +2435857|AAAAAAAABBLCFCAA|1957-01-18|684|2977|229|1957|5|1|18|1|1957|229|2977|Friday|1957Q1|N|Y|N|2435840|2435839|2435491|2435765|N|N|N|N|N| +2435858|AAAAAAAACBLCFCAA|1957-01-19|684|2977|229|1957|6|1|19|1|1957|229|2977|Saturday|1957Q1|N|Y|N|2435840|2435839|2435492|2435766|N|N|N|N|N| +2435859|AAAAAAAADBLCFCAA|1957-01-20|684|2977|229|1957|0|1|20|1|1957|229|2977|Sunday|1957Q1|N|N|N|2435840|2435839|2435493|2435767|N|N|N|N|N| +2435860|AAAAAAAAEBLCFCAA|1957-01-21|684|2977|229|1957|1|1|21|1|1957|229|2977|Monday|1957Q1|N|N|N|2435840|2435839|2435494|2435768|N|N|N|N|N| +2435861|AAAAAAAAFBLCFCAA|1957-01-22|684|2978|229|1957|2|1|22|1|1957|229|2978|Tuesday|1957Q1|N|N|N|2435840|2435839|2435495|2435769|N|N|N|N|N| +2435862|AAAAAAAAGBLCFCAA|1957-01-23|684|2978|229|1957|3|1|23|1|1957|229|2978|Wednesday|1957Q1|N|N|N|2435840|2435839|2435496|2435770|N|N|N|N|N| +2435863|AAAAAAAAHBLCFCAA|1957-01-24|684|2978|229|1957|4|1|24|1|1957|229|2978|Thursday|1957Q1|N|N|N|2435840|2435839|2435497|2435771|N|N|N|N|N| +2435864|AAAAAAAAIBLCFCAA|1957-01-25|684|2978|229|1957|5|1|25|1|1957|229|2978|Friday|1957Q1|N|Y|N|2435840|2435839|2435498|2435772|N|N|N|N|N| +2435865|AAAAAAAAJBLCFCAA|1957-01-26|684|2978|229|1957|6|1|26|1|1957|229|2978|Saturday|1957Q1|N|Y|N|2435840|2435839|2435499|2435773|N|N|N|N|N| +2435866|AAAAAAAAKBLCFCAA|1957-01-27|684|2978|229|1957|0|1|27|1|1957|229|2978|Sunday|1957Q1|N|N|N|2435840|2435839|2435500|2435774|N|N|N|N|N| +2435867|AAAAAAAALBLCFCAA|1957-01-28|684|2978|229|1957|1|1|28|1|1957|229|2978|Monday|1957Q1|N|N|N|2435840|2435839|2435501|2435775|N|N|N|N|N| +2435868|AAAAAAAAMBLCFCAA|1957-01-29|684|2979|229|1957|2|1|29|1|1957|229|2979|Tuesday|1957Q1|N|N|N|2435840|2435839|2435502|2435776|N|N|N|N|N| +2435869|AAAAAAAANBLCFCAA|1957-01-30|684|2979|229|1957|3|1|30|1|1957|229|2979|Wednesday|1957Q1|N|N|N|2435840|2435839|2435503|2435777|N|N|N|N|N| +2435870|AAAAAAAAOBLCFCAA|1957-01-31|684|2979|229|1957|4|1|31|1|1957|229|2979|Thursday|1957Q1|N|N|N|2435840|2435839|2435504|2435778|N|N|N|N|N| +2435871|AAAAAAAAPBLCFCAA|1957-02-01|685|2979|229|1957|5|2|1|1|1957|229|2979|Friday|1957Q1|N|Y|N|2435871|2435901|2435505|2435779|N|N|N|N|N| +2435872|AAAAAAAAACLCFCAA|1957-02-02|685|2979|229|1957|6|2|2|1|1957|229|2979|Saturday|1957Q1|N|Y|N|2435871|2435901|2435506|2435780|N|N|N|N|N| +2435873|AAAAAAAABCLCFCAA|1957-02-03|685|2979|229|1957|0|2|3|1|1957|229|2979|Sunday|1957Q1|N|N|N|2435871|2435901|2435507|2435781|N|N|N|N|N| +2435874|AAAAAAAACCLCFCAA|1957-02-04|685|2979|229|1957|1|2|4|1|1957|229|2979|Monday|1957Q1|N|N|N|2435871|2435901|2435508|2435782|N|N|N|N|N| +2435875|AAAAAAAADCLCFCAA|1957-02-05|685|2980|229|1957|2|2|5|1|1957|229|2980|Tuesday|1957Q1|N|N|N|2435871|2435901|2435509|2435783|N|N|N|N|N| +2435876|AAAAAAAAECLCFCAA|1957-02-06|685|2980|229|1957|3|2|6|1|1957|229|2980|Wednesday|1957Q1|N|N|N|2435871|2435901|2435510|2435784|N|N|N|N|N| +2435877|AAAAAAAAFCLCFCAA|1957-02-07|685|2980|229|1957|4|2|7|1|1957|229|2980|Thursday|1957Q1|N|N|N|2435871|2435901|2435511|2435785|N|N|N|N|N| +2435878|AAAAAAAAGCLCFCAA|1957-02-08|685|2980|229|1957|5|2|8|1|1957|229|2980|Friday|1957Q1|N|Y|N|2435871|2435901|2435512|2435786|N|N|N|N|N| +2435879|AAAAAAAAHCLCFCAA|1957-02-09|685|2980|229|1957|6|2|9|1|1957|229|2980|Saturday|1957Q1|N|Y|N|2435871|2435901|2435513|2435787|N|N|N|N|N| +2435880|AAAAAAAAICLCFCAA|1957-02-10|685|2980|229|1957|0|2|10|1|1957|229|2980|Sunday|1957Q1|N|N|N|2435871|2435901|2435514|2435788|N|N|N|N|N| +2435881|AAAAAAAAJCLCFCAA|1957-02-11|685|2980|229|1957|1|2|11|1|1957|229|2980|Monday|1957Q1|N|N|N|2435871|2435901|2435515|2435789|N|N|N|N|N| +2435882|AAAAAAAAKCLCFCAA|1957-02-12|685|2981|229|1957|2|2|12|1|1957|229|2981|Tuesday|1957Q1|N|N|N|2435871|2435901|2435516|2435790|N|N|N|N|N| +2435883|AAAAAAAALCLCFCAA|1957-02-13|685|2981|229|1957|3|2|13|1|1957|229|2981|Wednesday|1957Q1|N|N|N|2435871|2435901|2435517|2435791|N|N|N|N|N| +2435884|AAAAAAAAMCLCFCAA|1957-02-14|685|2981|229|1957|4|2|14|1|1957|229|2981|Thursday|1957Q1|N|N|N|2435871|2435901|2435518|2435792|N|N|N|N|N| +2435885|AAAAAAAANCLCFCAA|1957-02-15|685|2981|229|1957|5|2|15|1|1957|229|2981|Friday|1957Q1|N|Y|N|2435871|2435901|2435519|2435793|N|N|N|N|N| +2435886|AAAAAAAAOCLCFCAA|1957-02-16|685|2981|229|1957|6|2|16|1|1957|229|2981|Saturday|1957Q1|N|Y|N|2435871|2435901|2435520|2435794|N|N|N|N|N| +2435887|AAAAAAAAPCLCFCAA|1957-02-17|685|2981|229|1957|0|2|17|1|1957|229|2981|Sunday|1957Q1|N|N|N|2435871|2435901|2435521|2435795|N|N|N|N|N| +2435888|AAAAAAAAADLCFCAA|1957-02-18|685|2981|229|1957|1|2|18|1|1957|229|2981|Monday|1957Q1|N|N|N|2435871|2435901|2435522|2435796|N|N|N|N|N| +2435889|AAAAAAAABDLCFCAA|1957-02-19|685|2982|229|1957|2|2|19|1|1957|229|2982|Tuesday|1957Q1|N|N|N|2435871|2435901|2435523|2435797|N|N|N|N|N| +2435890|AAAAAAAACDLCFCAA|1957-02-20|685|2982|229|1957|3|2|20|1|1957|229|2982|Wednesday|1957Q1|N|N|N|2435871|2435901|2435524|2435798|N|N|N|N|N| +2435891|AAAAAAAADDLCFCAA|1957-02-21|685|2982|229|1957|4|2|21|1|1957|229|2982|Thursday|1957Q1|N|N|N|2435871|2435901|2435525|2435799|N|N|N|N|N| +2435892|AAAAAAAAEDLCFCAA|1957-02-22|685|2982|229|1957|5|2|22|1|1957|229|2982|Friday|1957Q1|N|Y|N|2435871|2435901|2435526|2435800|N|N|N|N|N| +2435893|AAAAAAAAFDLCFCAA|1957-02-23|685|2982|229|1957|6|2|23|1|1957|229|2982|Saturday|1957Q1|N|Y|N|2435871|2435901|2435527|2435801|N|N|N|N|N| +2435894|AAAAAAAAGDLCFCAA|1957-02-24|685|2982|229|1957|0|2|24|1|1957|229|2982|Sunday|1957Q1|N|N|N|2435871|2435901|2435528|2435802|N|N|N|N|N| +2435895|AAAAAAAAHDLCFCAA|1957-02-25|685|2982|229|1957|1|2|25|1|1957|229|2982|Monday|1957Q1|N|N|N|2435871|2435901|2435529|2435803|N|N|N|N|N| +2435896|AAAAAAAAIDLCFCAA|1957-02-26|685|2983|229|1957|2|2|26|1|1957|229|2983|Tuesday|1957Q1|N|N|N|2435871|2435901|2435530|2435804|N|N|N|N|N| +2435897|AAAAAAAAJDLCFCAA|1957-02-27|685|2983|229|1957|3|2|27|1|1957|229|2983|Wednesday|1957Q1|N|N|N|2435871|2435901|2435531|2435805|N|N|N|N|N| +2435898|AAAAAAAAKDLCFCAA|1957-02-28|685|2983|229|1957|4|2|28|1|1957|229|2983|Thursday|1957Q1|N|N|N|2435871|2435901|2435532|2435806|N|N|N|N|N| +2435899|AAAAAAAALDLCFCAA|1957-03-01|686|2983|230|1957|5|3|1|1|1957|230|2983|Friday|1957Q1|N|Y|N|2435899|2435957|2435534|2435807|N|N|N|N|N| +2435900|AAAAAAAAMDLCFCAA|1957-03-02|686|2983|230|1957|6|3|2|1|1957|230|2983|Saturday|1957Q1|N|Y|N|2435899|2435957|2435535|2435808|N|N|N|N|N| +2435901|AAAAAAAANDLCFCAA|1957-03-03|686|2983|230|1957|0|3|3|1|1957|230|2983|Sunday|1957Q1|N|N|N|2435899|2435957|2435536|2435809|N|N|N|N|N| +2435902|AAAAAAAAODLCFCAA|1957-03-04|686|2983|230|1957|1|3|4|1|1957|230|2983|Monday|1957Q1|N|N|N|2435899|2435957|2435537|2435810|N|N|N|N|N| +2435903|AAAAAAAAPDLCFCAA|1957-03-05|686|2984|230|1957|2|3|5|1|1957|230|2984|Tuesday|1957Q1|N|N|N|2435899|2435957|2435538|2435811|N|N|N|N|N| +2435904|AAAAAAAAAELCFCAA|1957-03-06|686|2984|230|1957|3|3|6|1|1957|230|2984|Wednesday|1957Q1|N|N|N|2435899|2435957|2435539|2435812|N|N|N|N|N| +2435905|AAAAAAAABELCFCAA|1957-03-07|686|2984|230|1957|4|3|7|1|1957|230|2984|Thursday|1957Q1|N|N|N|2435899|2435957|2435540|2435813|N|N|N|N|N| +2435906|AAAAAAAACELCFCAA|1957-03-08|686|2984|230|1957|5|3|8|1|1957|230|2984|Friday|1957Q1|N|Y|N|2435899|2435957|2435541|2435814|N|N|N|N|N| +2435907|AAAAAAAADELCFCAA|1957-03-09|686|2984|230|1957|6|3|9|1|1957|230|2984|Saturday|1957Q1|N|Y|N|2435899|2435957|2435542|2435815|N|N|N|N|N| +2435908|AAAAAAAAEELCFCAA|1957-03-10|686|2984|230|1957|0|3|10|1|1957|230|2984|Sunday|1957Q1|N|N|N|2435899|2435957|2435543|2435816|N|N|N|N|N| +2435909|AAAAAAAAFELCFCAA|1957-03-11|686|2984|230|1957|1|3|11|1|1957|230|2984|Monday|1957Q1|N|N|N|2435899|2435957|2435544|2435817|N|N|N|N|N| +2435910|AAAAAAAAGELCFCAA|1957-03-12|686|2985|230|1957|2|3|12|1|1957|230|2985|Tuesday|1957Q1|N|N|N|2435899|2435957|2435545|2435818|N|N|N|N|N| +2435911|AAAAAAAAHELCFCAA|1957-03-13|686|2985|230|1957|3|3|13|1|1957|230|2985|Wednesday|1957Q1|N|N|N|2435899|2435957|2435546|2435819|N|N|N|N|N| +2435912|AAAAAAAAIELCFCAA|1957-03-14|686|2985|230|1957|4|3|14|1|1957|230|2985|Thursday|1957Q1|N|N|N|2435899|2435957|2435547|2435820|N|N|N|N|N| +2435913|AAAAAAAAJELCFCAA|1957-03-15|686|2985|230|1957|5|3|15|1|1957|230|2985|Friday|1957Q1|N|Y|N|2435899|2435957|2435548|2435821|N|N|N|N|N| +2435914|AAAAAAAAKELCFCAA|1957-03-16|686|2985|230|1957|6|3|16|1|1957|230|2985|Saturday|1957Q1|N|Y|N|2435899|2435957|2435549|2435822|N|N|N|N|N| +2435915|AAAAAAAALELCFCAA|1957-03-17|686|2985|230|1957|0|3|17|1|1957|230|2985|Sunday|1957Q1|N|N|N|2435899|2435957|2435550|2435823|N|N|N|N|N| +2435916|AAAAAAAAMELCFCAA|1957-03-18|686|2985|230|1957|1|3|18|1|1957|230|2985|Monday|1957Q1|N|N|N|2435899|2435957|2435551|2435824|N|N|N|N|N| +2435917|AAAAAAAANELCFCAA|1957-03-19|686|2986|230|1957|2|3|19|1|1957|230|2986|Tuesday|1957Q1|N|N|N|2435899|2435957|2435552|2435825|N|N|N|N|N| +2435918|AAAAAAAAOELCFCAA|1957-03-20|686|2986|230|1957|3|3|20|1|1957|230|2986|Wednesday|1957Q1|N|N|N|2435899|2435957|2435553|2435826|N|N|N|N|N| +2435919|AAAAAAAAPELCFCAA|1957-03-21|686|2986|230|1957|4|3|21|1|1957|230|2986|Thursday|1957Q1|N|N|N|2435899|2435957|2435554|2435827|N|N|N|N|N| +2435920|AAAAAAAAAFLCFCAA|1957-03-22|686|2986|230|1957|5|3|22|1|1957|230|2986|Friday|1957Q1|N|Y|N|2435899|2435957|2435555|2435828|N|N|N|N|N| +2435921|AAAAAAAABFLCFCAA|1957-03-23|686|2986|230|1957|6|3|23|1|1957|230|2986|Saturday|1957Q1|N|Y|N|2435899|2435957|2435556|2435829|N|N|N|N|N| +2435922|AAAAAAAACFLCFCAA|1957-03-24|686|2986|230|1957|0|3|24|1|1957|230|2986|Sunday|1957Q1|N|N|N|2435899|2435957|2435557|2435830|N|N|N|N|N| +2435923|AAAAAAAADFLCFCAA|1957-03-25|686|2986|230|1957|1|3|25|1|1957|230|2986|Monday|1957Q1|N|N|N|2435899|2435957|2435558|2435831|N|N|N|N|N| +2435924|AAAAAAAAEFLCFCAA|1957-03-26|686|2987|230|1957|2|3|26|1|1957|230|2987|Tuesday|1957Q1|N|N|N|2435899|2435957|2435559|2435832|N|N|N|N|N| +2435925|AAAAAAAAFFLCFCAA|1957-03-27|686|2987|230|1957|3|3|27|1|1957|230|2987|Wednesday|1957Q1|N|N|N|2435899|2435957|2435560|2435833|N|N|N|N|N| +2435926|AAAAAAAAGFLCFCAA|1957-03-28|686|2987|230|1957|4|3|28|1|1957|230|2987|Thursday|1957Q1|N|N|N|2435899|2435957|2435561|2435834|N|N|N|N|N| +2435927|AAAAAAAAHFLCFCAA|1957-03-29|686|2987|230|1957|5|3|29|1|1957|230|2987|Friday|1957Q1|N|Y|N|2435899|2435957|2435562|2435835|N|N|N|N|N| +2435928|AAAAAAAAIFLCFCAA|1957-03-30|686|2987|230|1957|6|3|30|1|1957|230|2987|Saturday|1957Q1|N|Y|N|2435899|2435957|2435563|2435836|N|N|N|N|N| +2435929|AAAAAAAAJFLCFCAA|1957-03-31|686|2987|230|1957|0|3|31|1|1957|230|2987|Sunday|1957Q1|N|N|N|2435899|2435957|2435564|2435837|N|N|N|N|N| +2435930|AAAAAAAAKFLCFCAA|1957-04-01|687|2987|230|1957|1|4|1|1|1957|230|2987|Monday|1957Q1|N|N|N|2435930|2436019|2435565|2435840|N|N|N|N|N| +2435931|AAAAAAAALFLCFCAA|1957-04-02|687|2988|230|1957|2|4|2|2|1957|230|2988|Tuesday|1957Q2|N|N|N|2435930|2436019|2435566|2435841|N|N|N|N|N| +2435932|AAAAAAAAMFLCFCAA|1957-04-03|687|2988|230|1957|3|4|3|2|1957|230|2988|Wednesday|1957Q2|N|N|N|2435930|2436019|2435567|2435842|N|N|N|N|N| +2435933|AAAAAAAANFLCFCAA|1957-04-04|687|2988|230|1957|4|4|4|2|1957|230|2988|Thursday|1957Q2|N|N|N|2435930|2436019|2435568|2435843|N|N|N|N|N| +2435934|AAAAAAAAOFLCFCAA|1957-04-05|687|2988|230|1957|5|4|5|2|1957|230|2988|Friday|1957Q2|N|Y|N|2435930|2436019|2435569|2435844|N|N|N|N|N| +2435935|AAAAAAAAPFLCFCAA|1957-04-06|687|2988|230|1957|6|4|6|2|1957|230|2988|Saturday|1957Q2|N|Y|N|2435930|2436019|2435570|2435845|N|N|N|N|N| +2435936|AAAAAAAAAGLCFCAA|1957-04-07|687|2988|230|1957|0|4|7|2|1957|230|2988|Sunday|1957Q2|N|N|N|2435930|2436019|2435571|2435846|N|N|N|N|N| +2435937|AAAAAAAABGLCFCAA|1957-04-08|687|2988|230|1957|1|4|8|2|1957|230|2988|Monday|1957Q2|N|N|N|2435930|2436019|2435572|2435847|N|N|N|N|N| +2435938|AAAAAAAACGLCFCAA|1957-04-09|687|2989|230|1957|2|4|9|2|1957|230|2989|Tuesday|1957Q2|N|N|N|2435930|2436019|2435573|2435848|N|N|N|N|N| +2435939|AAAAAAAADGLCFCAA|1957-04-10|687|2989|230|1957|3|4|10|2|1957|230|2989|Wednesday|1957Q2|N|N|N|2435930|2436019|2435574|2435849|N|N|N|N|N| +2435940|AAAAAAAAEGLCFCAA|1957-04-11|687|2989|230|1957|4|4|11|2|1957|230|2989|Thursday|1957Q2|N|N|N|2435930|2436019|2435575|2435850|N|N|N|N|N| +2435941|AAAAAAAAFGLCFCAA|1957-04-12|687|2989|230|1957|5|4|12|2|1957|230|2989|Friday|1957Q2|N|Y|N|2435930|2436019|2435576|2435851|N|N|N|N|N| +2435942|AAAAAAAAGGLCFCAA|1957-04-13|687|2989|230|1957|6|4|13|2|1957|230|2989|Saturday|1957Q2|N|Y|N|2435930|2436019|2435577|2435852|N|N|N|N|N| +2435943|AAAAAAAAHGLCFCAA|1957-04-14|687|2989|230|1957|0|4|14|2|1957|230|2989|Sunday|1957Q2|N|N|N|2435930|2436019|2435578|2435853|N|N|N|N|N| +2435944|AAAAAAAAIGLCFCAA|1957-04-15|687|2989|230|1957|1|4|15|2|1957|230|2989|Monday|1957Q2|N|N|N|2435930|2436019|2435579|2435854|N|N|N|N|N| +2435945|AAAAAAAAJGLCFCAA|1957-04-16|687|2990|230|1957|2|4|16|2|1957|230|2990|Tuesday|1957Q2|N|N|N|2435930|2436019|2435580|2435855|N|N|N|N|N| +2435946|AAAAAAAAKGLCFCAA|1957-04-17|687|2990|230|1957|3|4|17|2|1957|230|2990|Wednesday|1957Q2|N|N|N|2435930|2436019|2435581|2435856|N|N|N|N|N| +2435947|AAAAAAAALGLCFCAA|1957-04-18|687|2990|230|1957|4|4|18|2|1957|230|2990|Thursday|1957Q2|N|N|N|2435930|2436019|2435582|2435857|N|N|N|N|N| +2435948|AAAAAAAAMGLCFCAA|1957-04-19|687|2990|230|1957|5|4|19|2|1957|230|2990|Friday|1957Q2|N|Y|N|2435930|2436019|2435583|2435858|N|N|N|N|N| +2435949|AAAAAAAANGLCFCAA|1957-04-20|687|2990|230|1957|6|4|20|2|1957|230|2990|Saturday|1957Q2|N|Y|N|2435930|2436019|2435584|2435859|N|N|N|N|N| +2435950|AAAAAAAAOGLCFCAA|1957-04-21|687|2990|230|1957|0|4|21|2|1957|230|2990|Sunday|1957Q2|N|N|N|2435930|2436019|2435585|2435860|N|N|N|N|N| +2435951|AAAAAAAAPGLCFCAA|1957-04-22|687|2990|230|1957|1|4|22|2|1957|230|2990|Monday|1957Q2|N|N|N|2435930|2436019|2435586|2435861|N|N|N|N|N| +2435952|AAAAAAAAAHLCFCAA|1957-04-23|687|2991|230|1957|2|4|23|2|1957|230|2991|Tuesday|1957Q2|N|N|N|2435930|2436019|2435587|2435862|N|N|N|N|N| +2435953|AAAAAAAABHLCFCAA|1957-04-24|687|2991|230|1957|3|4|24|2|1957|230|2991|Wednesday|1957Q2|N|N|N|2435930|2436019|2435588|2435863|N|N|N|N|N| +2435954|AAAAAAAACHLCFCAA|1957-04-25|687|2991|230|1957|4|4|25|2|1957|230|2991|Thursday|1957Q2|N|N|N|2435930|2436019|2435589|2435864|N|N|N|N|N| +2435955|AAAAAAAADHLCFCAA|1957-04-26|687|2991|230|1957|5|4|26|2|1957|230|2991|Friday|1957Q2|N|Y|N|2435930|2436019|2435590|2435865|N|N|N|N|N| +2435956|AAAAAAAAEHLCFCAA|1957-04-27|687|2991|230|1957|6|4|27|2|1957|230|2991|Saturday|1957Q2|N|Y|N|2435930|2436019|2435591|2435866|N|N|N|N|N| +2435957|AAAAAAAAFHLCFCAA|1957-04-28|687|2991|230|1957|0|4|28|2|1957|230|2991|Sunday|1957Q2|N|N|N|2435930|2436019|2435592|2435867|N|N|N|N|N| +2435958|AAAAAAAAGHLCFCAA|1957-04-29|687|2991|230|1957|1|4|29|2|1957|230|2991|Monday|1957Q2|N|N|N|2435930|2436019|2435593|2435868|N|N|N|N|N| +2435959|AAAAAAAAHHLCFCAA|1957-04-30|687|2992|230|1957|2|4|30|2|1957|230|2992|Tuesday|1957Q2|N|N|N|2435930|2436019|2435594|2435869|N|N|N|N|N| +2435960|AAAAAAAAIHLCFCAA|1957-05-01|688|2992|230|1957|3|5|1|2|1957|230|2992|Wednesday|1957Q2|N|N|N|2435960|2436079|2435595|2435870|N|N|N|N|N| +2435961|AAAAAAAAJHLCFCAA|1957-05-02|688|2992|230|1957|4|5|2|2|1957|230|2992|Thursday|1957Q2|N|N|N|2435960|2436079|2435596|2435871|N|N|N|N|N| +2435962|AAAAAAAAKHLCFCAA|1957-05-03|688|2992|230|1957|5|5|3|2|1957|230|2992|Friday|1957Q2|N|Y|N|2435960|2436079|2435597|2435872|N|N|N|N|N| +2435963|AAAAAAAALHLCFCAA|1957-05-04|688|2992|230|1957|6|5|4|2|1957|230|2992|Saturday|1957Q2|N|Y|N|2435960|2436079|2435598|2435873|N|N|N|N|N| +2435964|AAAAAAAAMHLCFCAA|1957-05-05|688|2992|230|1957|0|5|5|2|1957|230|2992|Sunday|1957Q2|N|N|N|2435960|2436079|2435599|2435874|N|N|N|N|N| +2435965|AAAAAAAANHLCFCAA|1957-05-06|688|2992|230|1957|1|5|6|2|1957|230|2992|Monday|1957Q2|N|N|N|2435960|2436079|2435600|2435875|N|N|N|N|N| +2435966|AAAAAAAAOHLCFCAA|1957-05-07|688|2993|230|1957|2|5|7|2|1957|230|2993|Tuesday|1957Q2|N|N|N|2435960|2436079|2435601|2435876|N|N|N|N|N| +2435967|AAAAAAAAPHLCFCAA|1957-05-08|688|2993|230|1957|3|5|8|2|1957|230|2993|Wednesday|1957Q2|N|N|N|2435960|2436079|2435602|2435877|N|N|N|N|N| +2435968|AAAAAAAAAILCFCAA|1957-05-09|688|2993|230|1957|4|5|9|2|1957|230|2993|Thursday|1957Q2|N|N|N|2435960|2436079|2435603|2435878|N|N|N|N|N| +2435969|AAAAAAAABILCFCAA|1957-05-10|688|2993|230|1957|5|5|10|2|1957|230|2993|Friday|1957Q2|N|Y|N|2435960|2436079|2435604|2435879|N|N|N|N|N| +2435970|AAAAAAAACILCFCAA|1957-05-11|688|2993|230|1957|6|5|11|2|1957|230|2993|Saturday|1957Q2|N|Y|N|2435960|2436079|2435605|2435880|N|N|N|N|N| +2435971|AAAAAAAADILCFCAA|1957-05-12|688|2993|230|1957|0|5|12|2|1957|230|2993|Sunday|1957Q2|N|N|N|2435960|2436079|2435606|2435881|N|N|N|N|N| +2435972|AAAAAAAAEILCFCAA|1957-05-13|688|2993|230|1957|1|5|13|2|1957|230|2993|Monday|1957Q2|N|N|N|2435960|2436079|2435607|2435882|N|N|N|N|N| +2435973|AAAAAAAAFILCFCAA|1957-05-14|688|2994|230|1957|2|5|14|2|1957|230|2994|Tuesday|1957Q2|N|N|N|2435960|2436079|2435608|2435883|N|N|N|N|N| +2435974|AAAAAAAAGILCFCAA|1957-05-15|688|2994|230|1957|3|5|15|2|1957|230|2994|Wednesday|1957Q2|N|N|N|2435960|2436079|2435609|2435884|N|N|N|N|N| +2435975|AAAAAAAAHILCFCAA|1957-05-16|688|2994|230|1957|4|5|16|2|1957|230|2994|Thursday|1957Q2|N|N|N|2435960|2436079|2435610|2435885|N|N|N|N|N| +2435976|AAAAAAAAIILCFCAA|1957-05-17|688|2994|230|1957|5|5|17|2|1957|230|2994|Friday|1957Q2|N|Y|N|2435960|2436079|2435611|2435886|N|N|N|N|N| +2435977|AAAAAAAAJILCFCAA|1957-05-18|688|2994|230|1957|6|5|18|2|1957|230|2994|Saturday|1957Q2|N|Y|N|2435960|2436079|2435612|2435887|N|N|N|N|N| +2435978|AAAAAAAAKILCFCAA|1957-05-19|688|2994|230|1957|0|5|19|2|1957|230|2994|Sunday|1957Q2|N|N|N|2435960|2436079|2435613|2435888|N|N|N|N|N| +2435979|AAAAAAAALILCFCAA|1957-05-20|688|2994|230|1957|1|5|20|2|1957|230|2994|Monday|1957Q2|N|N|N|2435960|2436079|2435614|2435889|N|N|N|N|N| +2435980|AAAAAAAAMILCFCAA|1957-05-21|688|2995|230|1957|2|5|21|2|1957|230|2995|Tuesday|1957Q2|N|N|N|2435960|2436079|2435615|2435890|N|N|N|N|N| +2435981|AAAAAAAANILCFCAA|1957-05-22|688|2995|230|1957|3|5|22|2|1957|230|2995|Wednesday|1957Q2|N|N|N|2435960|2436079|2435616|2435891|N|N|N|N|N| +2435982|AAAAAAAAOILCFCAA|1957-05-23|688|2995|230|1957|4|5|23|2|1957|230|2995|Thursday|1957Q2|N|N|N|2435960|2436079|2435617|2435892|N|N|N|N|N| +2435983|AAAAAAAAPILCFCAA|1957-05-24|688|2995|230|1957|5|5|24|2|1957|230|2995|Friday|1957Q2|N|Y|N|2435960|2436079|2435618|2435893|N|N|N|N|N| +2435984|AAAAAAAAAJLCFCAA|1957-05-25|688|2995|230|1957|6|5|25|2|1957|230|2995|Saturday|1957Q2|N|Y|N|2435960|2436079|2435619|2435894|N|N|N|N|N| +2435985|AAAAAAAABJLCFCAA|1957-05-26|688|2995|230|1957|0|5|26|2|1957|230|2995|Sunday|1957Q2|N|N|N|2435960|2436079|2435620|2435895|N|N|N|N|N| +2435986|AAAAAAAACJLCFCAA|1957-05-27|688|2995|230|1957|1|5|27|2|1957|230|2995|Monday|1957Q2|N|N|N|2435960|2436079|2435621|2435896|N|N|N|N|N| +2435987|AAAAAAAADJLCFCAA|1957-05-28|688|2996|230|1957|2|5|28|2|1957|230|2996|Tuesday|1957Q2|N|N|N|2435960|2436079|2435622|2435897|N|N|N|N|N| +2435988|AAAAAAAAEJLCFCAA|1957-05-29|688|2996|230|1957|3|5|29|2|1957|230|2996|Wednesday|1957Q2|N|N|N|2435960|2436079|2435623|2435898|N|N|N|N|N| +2435989|AAAAAAAAFJLCFCAA|1957-05-30|688|2996|230|1957|4|5|30|2|1957|230|2996|Thursday|1957Q2|N|N|N|2435960|2436079|2435624|2435899|N|N|N|N|N| +2435990|AAAAAAAAGJLCFCAA|1957-05-31|688|2996|230|1957|5|5|31|2|1957|230|2996|Friday|1957Q2|N|Y|N|2435960|2436079|2435625|2435900|N|N|N|N|N| +2435991|AAAAAAAAHJLCFCAA|1957-06-01|689|2996|231|1957|6|6|1|2|1957|231|2996|Saturday|1957Q2|N|Y|N|2435991|2436141|2435626|2435901|N|N|N|N|N| +2435992|AAAAAAAAIJLCFCAA|1957-06-02|689|2996|231|1957|0|6|2|2|1957|231|2996|Sunday|1957Q2|N|N|N|2435991|2436141|2435627|2435902|N|N|N|N|N| +2435993|AAAAAAAAJJLCFCAA|1957-06-03|689|2996|231|1957|1|6|3|2|1957|231|2996|Monday|1957Q2|N|N|N|2435991|2436141|2435628|2435903|N|N|N|N|N| +2435994|AAAAAAAAKJLCFCAA|1957-06-04|689|2997|231|1957|2|6|4|2|1957|231|2997|Tuesday|1957Q2|N|N|N|2435991|2436141|2435629|2435904|N|N|N|N|N| +2435995|AAAAAAAALJLCFCAA|1957-06-05|689|2997|231|1957|3|6|5|2|1957|231|2997|Wednesday|1957Q2|N|N|N|2435991|2436141|2435630|2435905|N|N|N|N|N| +2435996|AAAAAAAAMJLCFCAA|1957-06-06|689|2997|231|1957|4|6|6|2|1957|231|2997|Thursday|1957Q2|N|N|N|2435991|2436141|2435631|2435906|N|N|N|N|N| +2435997|AAAAAAAANJLCFCAA|1957-06-07|689|2997|231|1957|5|6|7|2|1957|231|2997|Friday|1957Q2|N|Y|N|2435991|2436141|2435632|2435907|N|N|N|N|N| +2435998|AAAAAAAAOJLCFCAA|1957-06-08|689|2997|231|1957|6|6|8|2|1957|231|2997|Saturday|1957Q2|N|Y|N|2435991|2436141|2435633|2435908|N|N|N|N|N| +2435999|AAAAAAAAPJLCFCAA|1957-06-09|689|2997|231|1957|0|6|9|2|1957|231|2997|Sunday|1957Q2|N|N|N|2435991|2436141|2435634|2435909|N|N|N|N|N| +2436000|AAAAAAAAAKLCFCAA|1957-06-10|689|2997|231|1957|1|6|10|2|1957|231|2997|Monday|1957Q2|N|N|N|2435991|2436141|2435635|2435910|N|N|N|N|N| +2436001|AAAAAAAABKLCFCAA|1957-06-11|689|2998|231|1957|2|6|11|2|1957|231|2998|Tuesday|1957Q2|N|N|N|2435991|2436141|2435636|2435911|N|N|N|N|N| +2436002|AAAAAAAACKLCFCAA|1957-06-12|689|2998|231|1957|3|6|12|2|1957|231|2998|Wednesday|1957Q2|N|N|N|2435991|2436141|2435637|2435912|N|N|N|N|N| +2436003|AAAAAAAADKLCFCAA|1957-06-13|689|2998|231|1957|4|6|13|2|1957|231|2998|Thursday|1957Q2|N|N|N|2435991|2436141|2435638|2435913|N|N|N|N|N| +2436004|AAAAAAAAEKLCFCAA|1957-06-14|689|2998|231|1957|5|6|14|2|1957|231|2998|Friday|1957Q2|N|Y|N|2435991|2436141|2435639|2435914|N|N|N|N|N| +2436005|AAAAAAAAFKLCFCAA|1957-06-15|689|2998|231|1957|6|6|15|2|1957|231|2998|Saturday|1957Q2|N|Y|N|2435991|2436141|2435640|2435915|N|N|N|N|N| +2436006|AAAAAAAAGKLCFCAA|1957-06-16|689|2998|231|1957|0|6|16|2|1957|231|2998|Sunday|1957Q2|N|N|N|2435991|2436141|2435641|2435916|N|N|N|N|N| +2436007|AAAAAAAAHKLCFCAA|1957-06-17|689|2998|231|1957|1|6|17|2|1957|231|2998|Monday|1957Q2|N|N|N|2435991|2436141|2435642|2435917|N|N|N|N|N| +2436008|AAAAAAAAIKLCFCAA|1957-06-18|689|2999|231|1957|2|6|18|2|1957|231|2999|Tuesday|1957Q2|N|N|N|2435991|2436141|2435643|2435918|N|N|N|N|N| +2436009|AAAAAAAAJKLCFCAA|1957-06-19|689|2999|231|1957|3|6|19|2|1957|231|2999|Wednesday|1957Q2|N|N|N|2435991|2436141|2435644|2435919|N|N|N|N|N| +2436010|AAAAAAAAKKLCFCAA|1957-06-20|689|2999|231|1957|4|6|20|2|1957|231|2999|Thursday|1957Q2|N|N|N|2435991|2436141|2435645|2435920|N|N|N|N|N| +2436011|AAAAAAAALKLCFCAA|1957-06-21|689|2999|231|1957|5|6|21|2|1957|231|2999|Friday|1957Q2|N|Y|N|2435991|2436141|2435646|2435921|N|N|N|N|N| +2436012|AAAAAAAAMKLCFCAA|1957-06-22|689|2999|231|1957|6|6|22|2|1957|231|2999|Saturday|1957Q2|N|Y|N|2435991|2436141|2435647|2435922|N|N|N|N|N| +2436013|AAAAAAAANKLCFCAA|1957-06-23|689|2999|231|1957|0|6|23|2|1957|231|2999|Sunday|1957Q2|N|N|N|2435991|2436141|2435648|2435923|N|N|N|N|N| +2436014|AAAAAAAAOKLCFCAA|1957-06-24|689|2999|231|1957|1|6|24|2|1957|231|2999|Monday|1957Q2|N|N|N|2435991|2436141|2435649|2435924|N|N|N|N|N| +2436015|AAAAAAAAPKLCFCAA|1957-06-25|689|3000|231|1957|2|6|25|2|1957|231|3000|Tuesday|1957Q2|N|N|N|2435991|2436141|2435650|2435925|N|N|N|N|N| +2436016|AAAAAAAAALLCFCAA|1957-06-26|689|3000|231|1957|3|6|26|2|1957|231|3000|Wednesday|1957Q2|N|N|N|2435991|2436141|2435651|2435926|N|N|N|N|N| +2436017|AAAAAAAABLLCFCAA|1957-06-27|689|3000|231|1957|4|6|27|2|1957|231|3000|Thursday|1957Q2|N|N|N|2435991|2436141|2435652|2435927|N|N|N|N|N| +2436018|AAAAAAAACLLCFCAA|1957-06-28|689|3000|231|1957|5|6|28|2|1957|231|3000|Friday|1957Q2|N|Y|N|2435991|2436141|2435653|2435928|N|N|N|N|N| +2436019|AAAAAAAADLLCFCAA|1957-06-29|689|3000|231|1957|6|6|29|2|1957|231|3000|Saturday|1957Q2|N|Y|N|2435991|2436141|2435654|2435929|N|N|N|N|N| +2436020|AAAAAAAAELLCFCAA|1957-06-30|689|3000|231|1957|0|6|30|2|1957|231|3000|Sunday|1957Q2|N|N|N|2435991|2436141|2435655|2435930|N|N|N|N|N| +2436021|AAAAAAAAFLLCFCAA|1957-07-01|690|3000|231|1957|1|7|1|2|1957|231|3000|Monday|1957Q2|N|N|N|2436021|2436201|2435656|2435930|N|N|N|N|N| +2436022|AAAAAAAAGLLCFCAA|1957-07-02|690|3001|231|1957|2|7|2|3|1957|231|3001|Tuesday|1957Q3|N|N|N|2436021|2436201|2435657|2435931|N|N|N|N|N| +2436023|AAAAAAAAHLLCFCAA|1957-07-03|690|3001|231|1957|3|7|3|3|1957|231|3001|Wednesday|1957Q3|N|N|N|2436021|2436201|2435658|2435932|N|N|N|N|N| +2436024|AAAAAAAAILLCFCAA|1957-07-04|690|3001|231|1957|4|7|4|3|1957|231|3001|Thursday|1957Q3|N|N|N|2436021|2436201|2435659|2435933|N|N|N|N|N| +2436025|AAAAAAAAJLLCFCAA|1957-07-05|690|3001|231|1957|5|7|5|3|1957|231|3001|Friday|1957Q3|Y|Y|N|2436021|2436201|2435660|2435934|N|N|N|N|N| +2436026|AAAAAAAAKLLCFCAA|1957-07-06|690|3001|231|1957|6|7|6|3|1957|231|3001|Saturday|1957Q3|N|Y|Y|2436021|2436201|2435661|2435935|N|N|N|N|N| +2436027|AAAAAAAALLLCFCAA|1957-07-07|690|3001|231|1957|0|7|7|3|1957|231|3001|Sunday|1957Q3|N|N|N|2436021|2436201|2435662|2435936|N|N|N|N|N| +2436028|AAAAAAAAMLLCFCAA|1957-07-08|690|3001|231|1957|1|7|8|3|1957|231|3001|Monday|1957Q3|N|N|N|2436021|2436201|2435663|2435937|N|N|N|N|N| +2436029|AAAAAAAANLLCFCAA|1957-07-09|690|3002|231|1957|2|7|9|3|1957|231|3002|Tuesday|1957Q3|N|N|N|2436021|2436201|2435664|2435938|N|N|N|N|N| +2436030|AAAAAAAAOLLCFCAA|1957-07-10|690|3002|231|1957|3|7|10|3|1957|231|3002|Wednesday|1957Q3|N|N|N|2436021|2436201|2435665|2435939|N|N|N|N|N| +2436031|AAAAAAAAPLLCFCAA|1957-07-11|690|3002|231|1957|4|7|11|3|1957|231|3002|Thursday|1957Q3|N|N|N|2436021|2436201|2435666|2435940|N|N|N|N|N| +2436032|AAAAAAAAAMLCFCAA|1957-07-12|690|3002|231|1957|5|7|12|3|1957|231|3002|Friday|1957Q3|N|Y|N|2436021|2436201|2435667|2435941|N|N|N|N|N| +2436033|AAAAAAAABMLCFCAA|1957-07-13|690|3002|231|1957|6|7|13|3|1957|231|3002|Saturday|1957Q3|N|Y|N|2436021|2436201|2435668|2435942|N|N|N|N|N| +2436034|AAAAAAAACMLCFCAA|1957-07-14|690|3002|231|1957|0|7|14|3|1957|231|3002|Sunday|1957Q3|N|N|N|2436021|2436201|2435669|2435943|N|N|N|N|N| +2436035|AAAAAAAADMLCFCAA|1957-07-15|690|3002|231|1957|1|7|15|3|1957|231|3002|Monday|1957Q3|N|N|N|2436021|2436201|2435670|2435944|N|N|N|N|N| +2436036|AAAAAAAAEMLCFCAA|1957-07-16|690|3003|231|1957|2|7|16|3|1957|231|3003|Tuesday|1957Q3|N|N|N|2436021|2436201|2435671|2435945|N|N|N|N|N| +2436037|AAAAAAAAFMLCFCAA|1957-07-17|690|3003|231|1957|3|7|17|3|1957|231|3003|Wednesday|1957Q3|N|N|N|2436021|2436201|2435672|2435946|N|N|N|N|N| +2436038|AAAAAAAAGMLCFCAA|1957-07-18|690|3003|231|1957|4|7|18|3|1957|231|3003|Thursday|1957Q3|N|N|N|2436021|2436201|2435673|2435947|N|N|N|N|N| +2436039|AAAAAAAAHMLCFCAA|1957-07-19|690|3003|231|1957|5|7|19|3|1957|231|3003|Friday|1957Q3|N|Y|N|2436021|2436201|2435674|2435948|N|N|N|N|N| +2436040|AAAAAAAAIMLCFCAA|1957-07-20|690|3003|231|1957|6|7|20|3|1957|231|3003|Saturday|1957Q3|N|Y|N|2436021|2436201|2435675|2435949|N|N|N|N|N| +2436041|AAAAAAAAJMLCFCAA|1957-07-21|690|3003|231|1957|0|7|21|3|1957|231|3003|Sunday|1957Q3|N|N|N|2436021|2436201|2435676|2435950|N|N|N|N|N| +2436042|AAAAAAAAKMLCFCAA|1957-07-22|690|3003|231|1957|1|7|22|3|1957|231|3003|Monday|1957Q3|N|N|N|2436021|2436201|2435677|2435951|N|N|N|N|N| +2436043|AAAAAAAALMLCFCAA|1957-07-23|690|3004|231|1957|2|7|23|3|1957|231|3004|Tuesday|1957Q3|N|N|N|2436021|2436201|2435678|2435952|N|N|N|N|N| +2436044|AAAAAAAAMMLCFCAA|1957-07-24|690|3004|231|1957|3|7|24|3|1957|231|3004|Wednesday|1957Q3|N|N|N|2436021|2436201|2435679|2435953|N|N|N|N|N| +2436045|AAAAAAAANMLCFCAA|1957-07-25|690|3004|231|1957|4|7|25|3|1957|231|3004|Thursday|1957Q3|N|N|N|2436021|2436201|2435680|2435954|N|N|N|N|N| +2436046|AAAAAAAAOMLCFCAA|1957-07-26|690|3004|231|1957|5|7|26|3|1957|231|3004|Friday|1957Q3|N|Y|N|2436021|2436201|2435681|2435955|N|N|N|N|N| +2436047|AAAAAAAAPMLCFCAA|1957-07-27|690|3004|231|1957|6|7|27|3|1957|231|3004|Saturday|1957Q3|N|Y|N|2436021|2436201|2435682|2435956|N|N|N|N|N| +2436048|AAAAAAAAANLCFCAA|1957-07-28|690|3004|231|1957|0|7|28|3|1957|231|3004|Sunday|1957Q3|N|N|N|2436021|2436201|2435683|2435957|N|N|N|N|N| +2436049|AAAAAAAABNLCFCAA|1957-07-29|690|3004|231|1957|1|7|29|3|1957|231|3004|Monday|1957Q3|N|N|N|2436021|2436201|2435684|2435958|N|N|N|N|N| +2436050|AAAAAAAACNLCFCAA|1957-07-30|690|3005|231|1957|2|7|30|3|1957|231|3005|Tuesday|1957Q3|N|N|N|2436021|2436201|2435685|2435959|N|N|N|N|N| +2436051|AAAAAAAADNLCFCAA|1957-07-31|690|3005|231|1957|3|7|31|3|1957|231|3005|Wednesday|1957Q3|N|N|N|2436021|2436201|2435686|2435960|N|N|N|N|N| +2436052|AAAAAAAAENLCFCAA|1957-08-01|691|3005|231|1957|4|8|1|3|1957|231|3005|Thursday|1957Q3|N|N|N|2436052|2436263|2435687|2435961|N|N|N|N|N| +2436053|AAAAAAAAFNLCFCAA|1957-08-02|691|3005|231|1957|5|8|2|3|1957|231|3005|Friday|1957Q3|N|Y|N|2436052|2436263|2435688|2435962|N|N|N|N|N| +2436054|AAAAAAAAGNLCFCAA|1957-08-03|691|3005|231|1957|6|8|3|3|1957|231|3005|Saturday|1957Q3|N|Y|N|2436052|2436263|2435689|2435963|N|N|N|N|N| +2436055|AAAAAAAAHNLCFCAA|1957-08-04|691|3005|231|1957|0|8|4|3|1957|231|3005|Sunday|1957Q3|N|N|N|2436052|2436263|2435690|2435964|N|N|N|N|N| +2436056|AAAAAAAAINLCFCAA|1957-08-05|691|3005|231|1957|1|8|5|3|1957|231|3005|Monday|1957Q3|N|N|N|2436052|2436263|2435691|2435965|N|N|N|N|N| +2436057|AAAAAAAAJNLCFCAA|1957-08-06|691|3006|231|1957|2|8|6|3|1957|231|3006|Tuesday|1957Q3|N|N|N|2436052|2436263|2435692|2435966|N|N|N|N|N| +2436058|AAAAAAAAKNLCFCAA|1957-08-07|691|3006|231|1957|3|8|7|3|1957|231|3006|Wednesday|1957Q3|N|N|N|2436052|2436263|2435693|2435967|N|N|N|N|N| +2436059|AAAAAAAALNLCFCAA|1957-08-08|691|3006|231|1957|4|8|8|3|1957|231|3006|Thursday|1957Q3|N|N|N|2436052|2436263|2435694|2435968|N|N|N|N|N| +2436060|AAAAAAAAMNLCFCAA|1957-08-09|691|3006|231|1957|5|8|9|3|1957|231|3006|Friday|1957Q3|N|Y|N|2436052|2436263|2435695|2435969|N|N|N|N|N| +2436061|AAAAAAAANNLCFCAA|1957-08-10|691|3006|231|1957|6|8|10|3|1957|231|3006|Saturday|1957Q3|N|Y|N|2436052|2436263|2435696|2435970|N|N|N|N|N| +2436062|AAAAAAAAONLCFCAA|1957-08-11|691|3006|231|1957|0|8|11|3|1957|231|3006|Sunday|1957Q3|N|N|N|2436052|2436263|2435697|2435971|N|N|N|N|N| +2436063|AAAAAAAAPNLCFCAA|1957-08-12|691|3006|231|1957|1|8|12|3|1957|231|3006|Monday|1957Q3|N|N|N|2436052|2436263|2435698|2435972|N|N|N|N|N| +2436064|AAAAAAAAAOLCFCAA|1957-08-13|691|3007|231|1957|2|8|13|3|1957|231|3007|Tuesday|1957Q3|N|N|N|2436052|2436263|2435699|2435973|N|N|N|N|N| +2436065|AAAAAAAABOLCFCAA|1957-08-14|691|3007|231|1957|3|8|14|3|1957|231|3007|Wednesday|1957Q3|N|N|N|2436052|2436263|2435700|2435974|N|N|N|N|N| +2436066|AAAAAAAACOLCFCAA|1957-08-15|691|3007|231|1957|4|8|15|3|1957|231|3007|Thursday|1957Q3|N|N|N|2436052|2436263|2435701|2435975|N|N|N|N|N| +2436067|AAAAAAAADOLCFCAA|1957-08-16|691|3007|231|1957|5|8|16|3|1957|231|3007|Friday|1957Q3|N|Y|N|2436052|2436263|2435702|2435976|N|N|N|N|N| +2436068|AAAAAAAAEOLCFCAA|1957-08-17|691|3007|231|1957|6|8|17|3|1957|231|3007|Saturday|1957Q3|N|Y|N|2436052|2436263|2435703|2435977|N|N|N|N|N| +2436069|AAAAAAAAFOLCFCAA|1957-08-18|691|3007|231|1957|0|8|18|3|1957|231|3007|Sunday|1957Q3|N|N|N|2436052|2436263|2435704|2435978|N|N|N|N|N| +2436070|AAAAAAAAGOLCFCAA|1957-08-19|691|3007|231|1957|1|8|19|3|1957|231|3007|Monday|1957Q3|N|N|N|2436052|2436263|2435705|2435979|N|N|N|N|N| +2436071|AAAAAAAAHOLCFCAA|1957-08-20|691|3008|231|1957|2|8|20|3|1957|231|3008|Tuesday|1957Q3|N|N|N|2436052|2436263|2435706|2435980|N|N|N|N|N| +2436072|AAAAAAAAIOLCFCAA|1957-08-21|691|3008|231|1957|3|8|21|3|1957|231|3008|Wednesday|1957Q3|N|N|N|2436052|2436263|2435707|2435981|N|N|N|N|N| +2436073|AAAAAAAAJOLCFCAA|1957-08-22|691|3008|231|1957|4|8|22|3|1957|231|3008|Thursday|1957Q3|N|N|N|2436052|2436263|2435708|2435982|N|N|N|N|N| +2436074|AAAAAAAAKOLCFCAA|1957-08-23|691|3008|231|1957|5|8|23|3|1957|231|3008|Friday|1957Q3|N|Y|N|2436052|2436263|2435709|2435983|N|N|N|N|N| +2436075|AAAAAAAALOLCFCAA|1957-08-24|691|3008|231|1957|6|8|24|3|1957|231|3008|Saturday|1957Q3|N|Y|N|2436052|2436263|2435710|2435984|N|N|N|N|N| +2436076|AAAAAAAAMOLCFCAA|1957-08-25|691|3008|231|1957|0|8|25|3|1957|231|3008|Sunday|1957Q3|N|N|N|2436052|2436263|2435711|2435985|N|N|N|N|N| +2436077|AAAAAAAANOLCFCAA|1957-08-26|691|3008|231|1957|1|8|26|3|1957|231|3008|Monday|1957Q3|N|N|N|2436052|2436263|2435712|2435986|N|N|N|N|N| +2436078|AAAAAAAAOOLCFCAA|1957-08-27|691|3009|231|1957|2|8|27|3|1957|231|3009|Tuesday|1957Q3|N|N|N|2436052|2436263|2435713|2435987|N|N|N|N|N| +2436079|AAAAAAAAPOLCFCAA|1957-08-28|691|3009|231|1957|3|8|28|3|1957|231|3009|Wednesday|1957Q3|N|N|N|2436052|2436263|2435714|2435988|N|N|N|N|N| +2436080|AAAAAAAAAPLCFCAA|1957-08-29|691|3009|231|1957|4|8|29|3|1957|231|3009|Thursday|1957Q3|N|N|N|2436052|2436263|2435715|2435989|N|N|N|N|N| +2436081|AAAAAAAABPLCFCAA|1957-08-30|691|3009|231|1957|5|8|30|3|1957|231|3009|Friday|1957Q3|N|Y|N|2436052|2436263|2435716|2435990|N|N|N|N|N| +2436082|AAAAAAAACPLCFCAA|1957-08-31|691|3009|231|1957|6|8|31|3|1957|231|3009|Saturday|1957Q3|N|Y|N|2436052|2436263|2435717|2435991|N|N|N|N|N| +2436083|AAAAAAAADPLCFCAA|1957-09-01|692|3009|232|1957|0|9|1|3|1957|232|3009|Sunday|1957Q3|N|N|N|2436083|2436325|2435718|2435992|N|N|N|N|N| +2436084|AAAAAAAAEPLCFCAA|1957-09-02|692|3009|232|1957|1|9|2|3|1957|232|3009|Monday|1957Q3|N|N|N|2436083|2436325|2435719|2435993|N|N|N|N|N| +2436085|AAAAAAAAFPLCFCAA|1957-09-03|692|3010|232|1957|2|9|3|3|1957|232|3010|Tuesday|1957Q3|N|N|N|2436083|2436325|2435720|2435994|N|N|N|N|N| +2436086|AAAAAAAAGPLCFCAA|1957-09-04|692|3010|232|1957|3|9|4|3|1957|232|3010|Wednesday|1957Q3|N|N|N|2436083|2436325|2435721|2435995|N|N|N|N|N| +2436087|AAAAAAAAHPLCFCAA|1957-09-05|692|3010|232|1957|4|9|5|3|1957|232|3010|Thursday|1957Q3|N|N|N|2436083|2436325|2435722|2435996|N|N|N|N|N| +2436088|AAAAAAAAIPLCFCAA|1957-09-06|692|3010|232|1957|5|9|6|3|1957|232|3010|Friday|1957Q3|N|Y|N|2436083|2436325|2435723|2435997|N|N|N|N|N| +2436089|AAAAAAAAJPLCFCAA|1957-09-07|692|3010|232|1957|6|9|7|3|1957|232|3010|Saturday|1957Q3|N|Y|N|2436083|2436325|2435724|2435998|N|N|N|N|N| +2436090|AAAAAAAAKPLCFCAA|1957-09-08|692|3010|232|1957|0|9|8|3|1957|232|3010|Sunday|1957Q3|N|N|N|2436083|2436325|2435725|2435999|N|N|N|N|N| +2436091|AAAAAAAALPLCFCAA|1957-09-09|692|3010|232|1957|1|9|9|3|1957|232|3010|Monday|1957Q3|N|N|N|2436083|2436325|2435726|2436000|N|N|N|N|N| +2436092|AAAAAAAAMPLCFCAA|1957-09-10|692|3011|232|1957|2|9|10|3|1957|232|3011|Tuesday|1957Q3|N|N|N|2436083|2436325|2435727|2436001|N|N|N|N|N| +2436093|AAAAAAAANPLCFCAA|1957-09-11|692|3011|232|1957|3|9|11|3|1957|232|3011|Wednesday|1957Q3|N|N|N|2436083|2436325|2435728|2436002|N|N|N|N|N| +2436094|AAAAAAAAOPLCFCAA|1957-09-12|692|3011|232|1957|4|9|12|3|1957|232|3011|Thursday|1957Q3|N|N|N|2436083|2436325|2435729|2436003|N|N|N|N|N| +2436095|AAAAAAAAPPLCFCAA|1957-09-13|692|3011|232|1957|5|9|13|3|1957|232|3011|Friday|1957Q3|N|Y|N|2436083|2436325|2435730|2436004|N|N|N|N|N| +2436096|AAAAAAAAAAMCFCAA|1957-09-14|692|3011|232|1957|6|9|14|3|1957|232|3011|Saturday|1957Q3|N|Y|N|2436083|2436325|2435731|2436005|N|N|N|N|N| +2436097|AAAAAAAABAMCFCAA|1957-09-15|692|3011|232|1957|0|9|15|3|1957|232|3011|Sunday|1957Q3|N|N|N|2436083|2436325|2435732|2436006|N|N|N|N|N| +2436098|AAAAAAAACAMCFCAA|1957-09-16|692|3011|232|1957|1|9|16|3|1957|232|3011|Monday|1957Q3|N|N|N|2436083|2436325|2435733|2436007|N|N|N|N|N| +2436099|AAAAAAAADAMCFCAA|1957-09-17|692|3012|232|1957|2|9|17|3|1957|232|3012|Tuesday|1957Q3|N|N|N|2436083|2436325|2435734|2436008|N|N|N|N|N| +2436100|AAAAAAAAEAMCFCAA|1957-09-18|692|3012|232|1957|3|9|18|3|1957|232|3012|Wednesday|1957Q3|N|N|N|2436083|2436325|2435735|2436009|N|N|N|N|N| +2436101|AAAAAAAAFAMCFCAA|1957-09-19|692|3012|232|1957|4|9|19|3|1957|232|3012|Thursday|1957Q3|N|N|N|2436083|2436325|2435736|2436010|N|N|N|N|N| +2436102|AAAAAAAAGAMCFCAA|1957-09-20|692|3012|232|1957|5|9|20|3|1957|232|3012|Friday|1957Q3|N|Y|N|2436083|2436325|2435737|2436011|N|N|N|N|N| +2436103|AAAAAAAAHAMCFCAA|1957-09-21|692|3012|232|1957|6|9|21|3|1957|232|3012|Saturday|1957Q3|N|Y|N|2436083|2436325|2435738|2436012|N|N|N|N|N| +2436104|AAAAAAAAIAMCFCAA|1957-09-22|692|3012|232|1957|0|9|22|3|1957|232|3012|Sunday|1957Q3|N|N|N|2436083|2436325|2435739|2436013|N|N|N|N|N| +2436105|AAAAAAAAJAMCFCAA|1957-09-23|692|3012|232|1957|1|9|23|3|1957|232|3012|Monday|1957Q3|N|N|N|2436083|2436325|2435740|2436014|N|N|N|N|N| +2436106|AAAAAAAAKAMCFCAA|1957-09-24|692|3013|232|1957|2|9|24|3|1957|232|3013|Tuesday|1957Q3|N|N|N|2436083|2436325|2435741|2436015|N|N|N|N|N| +2436107|AAAAAAAALAMCFCAA|1957-09-25|692|3013|232|1957|3|9|25|3|1957|232|3013|Wednesday|1957Q3|N|N|N|2436083|2436325|2435742|2436016|N|N|N|N|N| +2436108|AAAAAAAAMAMCFCAA|1957-09-26|692|3013|232|1957|4|9|26|3|1957|232|3013|Thursday|1957Q3|N|N|N|2436083|2436325|2435743|2436017|N|N|N|N|N| +2436109|AAAAAAAANAMCFCAA|1957-09-27|692|3013|232|1957|5|9|27|3|1957|232|3013|Friday|1957Q3|N|Y|N|2436083|2436325|2435744|2436018|N|N|N|N|N| +2436110|AAAAAAAAOAMCFCAA|1957-09-28|692|3013|232|1957|6|9|28|3|1957|232|3013|Saturday|1957Q3|N|Y|N|2436083|2436325|2435745|2436019|N|N|N|N|N| +2436111|AAAAAAAAPAMCFCAA|1957-09-29|692|3013|232|1957|0|9|29|3|1957|232|3013|Sunday|1957Q3|N|N|N|2436083|2436325|2435746|2436020|N|N|N|N|N| +2436112|AAAAAAAAABMCFCAA|1957-09-30|692|3013|232|1957|1|9|30|3|1957|232|3013|Monday|1957Q3|N|N|N|2436083|2436325|2435747|2436021|N|N|N|N|N| +2436113|AAAAAAAABBMCFCAA|1957-10-01|693|3014|232|1957|2|10|1|3|1957|232|3014|Tuesday|1957Q3|N|N|N|2436113|2436385|2435748|2436021|N|N|N|N|N| +2436114|AAAAAAAACBMCFCAA|1957-10-02|693|3014|232|1957|3|10|2|4|1957|232|3014|Wednesday|1957Q4|N|N|N|2436113|2436385|2435749|2436022|N|N|N|N|N| +2436115|AAAAAAAADBMCFCAA|1957-10-03|693|3014|232|1957|4|10|3|4|1957|232|3014|Thursday|1957Q4|N|N|N|2436113|2436385|2435750|2436023|N|N|N|N|N| +2436116|AAAAAAAAEBMCFCAA|1957-10-04|693|3014|232|1957|5|10|4|4|1957|232|3014|Friday|1957Q4|N|Y|N|2436113|2436385|2435751|2436024|N|N|N|N|N| +2436117|AAAAAAAAFBMCFCAA|1957-10-05|693|3014|232|1957|6|10|5|4|1957|232|3014|Saturday|1957Q4|N|Y|N|2436113|2436385|2435752|2436025|N|N|N|N|N| +2436118|AAAAAAAAGBMCFCAA|1957-10-06|693|3014|232|1957|0|10|6|4|1957|232|3014|Sunday|1957Q4|N|N|N|2436113|2436385|2435753|2436026|N|N|N|N|N| +2436119|AAAAAAAAHBMCFCAA|1957-10-07|693|3014|232|1957|1|10|7|4|1957|232|3014|Monday|1957Q4|N|N|N|2436113|2436385|2435754|2436027|N|N|N|N|N| +2436120|AAAAAAAAIBMCFCAA|1957-10-08|693|3015|232|1957|2|10|8|4|1957|232|3015|Tuesday|1957Q4|N|N|N|2436113|2436385|2435755|2436028|N|N|N|N|N| +2436121|AAAAAAAAJBMCFCAA|1957-10-09|693|3015|232|1957|3|10|9|4|1957|232|3015|Wednesday|1957Q4|N|N|N|2436113|2436385|2435756|2436029|N|N|N|N|N| +2436122|AAAAAAAAKBMCFCAA|1957-10-10|693|3015|232|1957|4|10|10|4|1957|232|3015|Thursday|1957Q4|N|N|N|2436113|2436385|2435757|2436030|N|N|N|N|N| +2436123|AAAAAAAALBMCFCAA|1957-10-11|693|3015|232|1957|5|10|11|4|1957|232|3015|Friday|1957Q4|N|Y|N|2436113|2436385|2435758|2436031|N|N|N|N|N| +2436124|AAAAAAAAMBMCFCAA|1957-10-12|693|3015|232|1957|6|10|12|4|1957|232|3015|Saturday|1957Q4|N|Y|N|2436113|2436385|2435759|2436032|N|N|N|N|N| +2436125|AAAAAAAANBMCFCAA|1957-10-13|693|3015|232|1957|0|10|13|4|1957|232|3015|Sunday|1957Q4|N|N|N|2436113|2436385|2435760|2436033|N|N|N|N|N| +2436126|AAAAAAAAOBMCFCAA|1957-10-14|693|3015|232|1957|1|10|14|4|1957|232|3015|Monday|1957Q4|N|N|N|2436113|2436385|2435761|2436034|N|N|N|N|N| +2436127|AAAAAAAAPBMCFCAA|1957-10-15|693|3016|232|1957|2|10|15|4|1957|232|3016|Tuesday|1957Q4|N|N|N|2436113|2436385|2435762|2436035|N|N|N|N|N| +2436128|AAAAAAAAACMCFCAA|1957-10-16|693|3016|232|1957|3|10|16|4|1957|232|3016|Wednesday|1957Q4|N|N|N|2436113|2436385|2435763|2436036|N|N|N|N|N| +2436129|AAAAAAAABCMCFCAA|1957-10-17|693|3016|232|1957|4|10|17|4|1957|232|3016|Thursday|1957Q4|N|N|N|2436113|2436385|2435764|2436037|N|N|N|N|N| +2436130|AAAAAAAACCMCFCAA|1957-10-18|693|3016|232|1957|5|10|18|4|1957|232|3016|Friday|1957Q4|N|Y|N|2436113|2436385|2435765|2436038|N|N|N|N|N| +2436131|AAAAAAAADCMCFCAA|1957-10-19|693|3016|232|1957|6|10|19|4|1957|232|3016|Saturday|1957Q4|N|Y|N|2436113|2436385|2435766|2436039|N|N|N|N|N| +2436132|AAAAAAAAECMCFCAA|1957-10-20|693|3016|232|1957|0|10|20|4|1957|232|3016|Sunday|1957Q4|N|N|N|2436113|2436385|2435767|2436040|N|N|N|N|N| +2436133|AAAAAAAAFCMCFCAA|1957-10-21|693|3016|232|1957|1|10|21|4|1957|232|3016|Monday|1957Q4|N|N|N|2436113|2436385|2435768|2436041|N|N|N|N|N| +2436134|AAAAAAAAGCMCFCAA|1957-10-22|693|3017|232|1957|2|10|22|4|1957|232|3017|Tuesday|1957Q4|N|N|N|2436113|2436385|2435769|2436042|N|N|N|N|N| +2436135|AAAAAAAAHCMCFCAA|1957-10-23|693|3017|232|1957|3|10|23|4|1957|232|3017|Wednesday|1957Q4|N|N|N|2436113|2436385|2435770|2436043|N|N|N|N|N| +2436136|AAAAAAAAICMCFCAA|1957-10-24|693|3017|232|1957|4|10|24|4|1957|232|3017|Thursday|1957Q4|N|N|N|2436113|2436385|2435771|2436044|N|N|N|N|N| +2436137|AAAAAAAAJCMCFCAA|1957-10-25|693|3017|232|1957|5|10|25|4|1957|232|3017|Friday|1957Q4|N|Y|N|2436113|2436385|2435772|2436045|N|N|N|N|N| +2436138|AAAAAAAAKCMCFCAA|1957-10-26|693|3017|232|1957|6|10|26|4|1957|232|3017|Saturday|1957Q4|N|Y|N|2436113|2436385|2435773|2436046|N|N|N|N|N| +2436139|AAAAAAAALCMCFCAA|1957-10-27|693|3017|232|1957|0|10|27|4|1957|232|3017|Sunday|1957Q4|N|N|N|2436113|2436385|2435774|2436047|N|N|N|N|N| +2436140|AAAAAAAAMCMCFCAA|1957-10-28|693|3017|232|1957|1|10|28|4|1957|232|3017|Monday|1957Q4|N|N|N|2436113|2436385|2435775|2436048|N|N|N|N|N| +2436141|AAAAAAAANCMCFCAA|1957-10-29|693|3018|232|1957|2|10|29|4|1957|232|3018|Tuesday|1957Q4|N|N|N|2436113|2436385|2435776|2436049|N|N|N|N|N| +2436142|AAAAAAAAOCMCFCAA|1957-10-30|693|3018|232|1957|3|10|30|4|1957|232|3018|Wednesday|1957Q4|N|N|N|2436113|2436385|2435777|2436050|N|N|N|N|N| +2436143|AAAAAAAAPCMCFCAA|1957-10-31|693|3018|232|1957|4|10|31|4|1957|232|3018|Thursday|1957Q4|N|N|N|2436113|2436385|2435778|2436051|N|N|N|N|N| +2436144|AAAAAAAAADMCFCAA|1957-11-01|694|3018|232|1957|5|11|1|4|1957|232|3018|Friday|1957Q4|N|Y|N|2436144|2436447|2435779|2436052|N|N|N|N|N| +2436145|AAAAAAAABDMCFCAA|1957-11-02|694|3018|232|1957|6|11|2|4|1957|232|3018|Saturday|1957Q4|N|Y|N|2436144|2436447|2435780|2436053|N|N|N|N|N| +2436146|AAAAAAAACDMCFCAA|1957-11-03|694|3018|232|1957|0|11|3|4|1957|232|3018|Sunday|1957Q4|N|N|N|2436144|2436447|2435781|2436054|N|N|N|N|N| +2436147|AAAAAAAADDMCFCAA|1957-11-04|694|3018|232|1957|1|11|4|4|1957|232|3018|Monday|1957Q4|N|N|N|2436144|2436447|2435782|2436055|N|N|N|N|N| +2436148|AAAAAAAAEDMCFCAA|1957-11-05|694|3019|232|1957|2|11|5|4|1957|232|3019|Tuesday|1957Q4|N|N|N|2436144|2436447|2435783|2436056|N|N|N|N|N| +2436149|AAAAAAAAFDMCFCAA|1957-11-06|694|3019|232|1957|3|11|6|4|1957|232|3019|Wednesday|1957Q4|N|N|N|2436144|2436447|2435784|2436057|N|N|N|N|N| +2436150|AAAAAAAAGDMCFCAA|1957-11-07|694|3019|232|1957|4|11|7|4|1957|232|3019|Thursday|1957Q4|N|N|N|2436144|2436447|2435785|2436058|N|N|N|N|N| +2436151|AAAAAAAAHDMCFCAA|1957-11-08|694|3019|232|1957|5|11|8|4|1957|232|3019|Friday|1957Q4|N|Y|N|2436144|2436447|2435786|2436059|N|N|N|N|N| +2436152|AAAAAAAAIDMCFCAA|1957-11-09|694|3019|232|1957|6|11|9|4|1957|232|3019|Saturday|1957Q4|N|Y|N|2436144|2436447|2435787|2436060|N|N|N|N|N| +2436153|AAAAAAAAJDMCFCAA|1957-11-10|694|3019|232|1957|0|11|10|4|1957|232|3019|Sunday|1957Q4|N|N|N|2436144|2436447|2435788|2436061|N|N|N|N|N| +2436154|AAAAAAAAKDMCFCAA|1957-11-11|694|3019|232|1957|1|11|11|4|1957|232|3019|Monday|1957Q4|N|N|N|2436144|2436447|2435789|2436062|N|N|N|N|N| +2436155|AAAAAAAALDMCFCAA|1957-11-12|694|3020|232|1957|2|11|12|4|1957|232|3020|Tuesday|1957Q4|N|N|N|2436144|2436447|2435790|2436063|N|N|N|N|N| +2436156|AAAAAAAAMDMCFCAA|1957-11-13|694|3020|232|1957|3|11|13|4|1957|232|3020|Wednesday|1957Q4|N|N|N|2436144|2436447|2435791|2436064|N|N|N|N|N| +2436157|AAAAAAAANDMCFCAA|1957-11-14|694|3020|232|1957|4|11|14|4|1957|232|3020|Thursday|1957Q4|N|N|N|2436144|2436447|2435792|2436065|N|N|N|N|N| +2436158|AAAAAAAAODMCFCAA|1957-11-15|694|3020|232|1957|5|11|15|4|1957|232|3020|Friday|1957Q4|N|Y|N|2436144|2436447|2435793|2436066|N|N|N|N|N| +2436159|AAAAAAAAPDMCFCAA|1957-11-16|694|3020|232|1957|6|11|16|4|1957|232|3020|Saturday|1957Q4|N|Y|N|2436144|2436447|2435794|2436067|N|N|N|N|N| +2436160|AAAAAAAAAEMCFCAA|1957-11-17|694|3020|232|1957|0|11|17|4|1957|232|3020|Sunday|1957Q4|N|N|N|2436144|2436447|2435795|2436068|N|N|N|N|N| +2436161|AAAAAAAABEMCFCAA|1957-11-18|694|3020|232|1957|1|11|18|4|1957|232|3020|Monday|1957Q4|N|N|N|2436144|2436447|2435796|2436069|N|N|N|N|N| +2436162|AAAAAAAACEMCFCAA|1957-11-19|694|3021|232|1957|2|11|19|4|1957|232|3021|Tuesday|1957Q4|N|N|N|2436144|2436447|2435797|2436070|N|N|N|N|N| +2436163|AAAAAAAADEMCFCAA|1957-11-20|694|3021|232|1957|3|11|20|4|1957|232|3021|Wednesday|1957Q4|N|N|N|2436144|2436447|2435798|2436071|N|N|N|N|N| +2436164|AAAAAAAAEEMCFCAA|1957-11-21|694|3021|232|1957|4|11|21|4|1957|232|3021|Thursday|1957Q4|N|N|N|2436144|2436447|2435799|2436072|N|N|N|N|N| +2436165|AAAAAAAAFEMCFCAA|1957-11-22|694|3021|232|1957|5|11|22|4|1957|232|3021|Friday|1957Q4|N|Y|N|2436144|2436447|2435800|2436073|N|N|N|N|N| +2436166|AAAAAAAAGEMCFCAA|1957-11-23|694|3021|232|1957|6|11|23|4|1957|232|3021|Saturday|1957Q4|N|Y|N|2436144|2436447|2435801|2436074|N|N|N|N|N| +2436167|AAAAAAAAHEMCFCAA|1957-11-24|694|3021|232|1957|0|11|24|4|1957|232|3021|Sunday|1957Q4|N|N|N|2436144|2436447|2435802|2436075|N|N|N|N|N| +2436168|AAAAAAAAIEMCFCAA|1957-11-25|694|3021|232|1957|1|11|25|4|1957|232|3021|Monday|1957Q4|N|N|N|2436144|2436447|2435803|2436076|N|N|N|N|N| +2436169|AAAAAAAAJEMCFCAA|1957-11-26|694|3022|232|1957|2|11|26|4|1957|232|3022|Tuesday|1957Q4|N|N|N|2436144|2436447|2435804|2436077|N|N|N|N|N| +2436170|AAAAAAAAKEMCFCAA|1957-11-27|694|3022|232|1957|3|11|27|4|1957|232|3022|Wednesday|1957Q4|N|N|N|2436144|2436447|2435805|2436078|N|N|N|N|N| +2436171|AAAAAAAALEMCFCAA|1957-11-28|694|3022|232|1957|4|11|28|4|1957|232|3022|Thursday|1957Q4|N|N|N|2436144|2436447|2435806|2436079|N|N|N|N|N| +2436172|AAAAAAAAMEMCFCAA|1957-11-29|694|3022|232|1957|5|11|29|4|1957|232|3022|Friday|1957Q4|N|Y|N|2436144|2436447|2435807|2436080|N|N|N|N|N| +2436173|AAAAAAAANEMCFCAA|1957-11-30|694|3022|232|1957|6|11|30|4|1957|232|3022|Saturday|1957Q4|N|Y|N|2436144|2436447|2435808|2436081|N|N|N|N|N| +2436174|AAAAAAAAOEMCFCAA|1957-12-01|695|3022|233|1957|0|12|1|4|1957|233|3022|Sunday|1957Q4|N|N|N|2436174|2436507|2435809|2436082|N|N|N|N|N| +2436175|AAAAAAAAPEMCFCAA|1957-12-02|695|3022|233|1957|1|12|2|4|1957|233|3022|Monday|1957Q4|N|N|N|2436174|2436507|2435810|2436083|N|N|N|N|N| +2436176|AAAAAAAAAFMCFCAA|1957-12-03|695|3023|233|1957|2|12|3|4|1957|233|3023|Tuesday|1957Q4|N|N|N|2436174|2436507|2435811|2436084|N|N|N|N|N| +2436177|AAAAAAAABFMCFCAA|1957-12-04|695|3023|233|1957|3|12|4|4|1957|233|3023|Wednesday|1957Q4|N|N|N|2436174|2436507|2435812|2436085|N|N|N|N|N| +2436178|AAAAAAAACFMCFCAA|1957-12-05|695|3023|233|1957|4|12|5|4|1957|233|3023|Thursday|1957Q4|N|N|N|2436174|2436507|2435813|2436086|N|N|N|N|N| +2436179|AAAAAAAADFMCFCAA|1957-12-06|695|3023|233|1957|5|12|6|4|1957|233|3023|Friday|1957Q4|N|Y|N|2436174|2436507|2435814|2436087|N|N|N|N|N| +2436180|AAAAAAAAEFMCFCAA|1957-12-07|695|3023|233|1957|6|12|7|4|1957|233|3023|Saturday|1957Q4|N|Y|N|2436174|2436507|2435815|2436088|N|N|N|N|N| +2436181|AAAAAAAAFFMCFCAA|1957-12-08|695|3023|233|1957|0|12|8|4|1957|233|3023|Sunday|1957Q4|N|N|N|2436174|2436507|2435816|2436089|N|N|N|N|N| +2436182|AAAAAAAAGFMCFCAA|1957-12-09|695|3023|233|1957|1|12|9|4|1957|233|3023|Monday|1957Q4|N|N|N|2436174|2436507|2435817|2436090|N|N|N|N|N| +2436183|AAAAAAAAHFMCFCAA|1957-12-10|695|3024|233|1957|2|12|10|4|1957|233|3024|Tuesday|1957Q4|N|N|N|2436174|2436507|2435818|2436091|N|N|N|N|N| +2436184|AAAAAAAAIFMCFCAA|1957-12-11|695|3024|233|1957|3|12|11|4|1957|233|3024|Wednesday|1957Q4|N|N|N|2436174|2436507|2435819|2436092|N|N|N|N|N| +2436185|AAAAAAAAJFMCFCAA|1957-12-12|695|3024|233|1957|4|12|12|4|1957|233|3024|Thursday|1957Q4|N|N|N|2436174|2436507|2435820|2436093|N|N|N|N|N| +2436186|AAAAAAAAKFMCFCAA|1957-12-13|695|3024|233|1957|5|12|13|4|1957|233|3024|Friday|1957Q4|N|Y|N|2436174|2436507|2435821|2436094|N|N|N|N|N| +2436187|AAAAAAAALFMCFCAA|1957-12-14|695|3024|233|1957|6|12|14|4|1957|233|3024|Saturday|1957Q4|N|Y|N|2436174|2436507|2435822|2436095|N|N|N|N|N| +2436188|AAAAAAAAMFMCFCAA|1957-12-15|695|3024|233|1957|0|12|15|4|1957|233|3024|Sunday|1957Q4|N|N|N|2436174|2436507|2435823|2436096|N|N|N|N|N| +2436189|AAAAAAAANFMCFCAA|1957-12-16|695|3024|233|1957|1|12|16|4|1957|233|3024|Monday|1957Q4|N|N|N|2436174|2436507|2435824|2436097|N|N|N|N|N| +2436190|AAAAAAAAOFMCFCAA|1957-12-17|695|3025|233|1957|2|12|17|4|1957|233|3025|Tuesday|1957Q4|N|N|N|2436174|2436507|2435825|2436098|N|N|N|N|N| +2436191|AAAAAAAAPFMCFCAA|1957-12-18|695|3025|233|1957|3|12|18|4|1957|233|3025|Wednesday|1957Q4|N|N|N|2436174|2436507|2435826|2436099|N|N|N|N|N| +2436192|AAAAAAAAAGMCFCAA|1957-12-19|695|3025|233|1957|4|12|19|4|1957|233|3025|Thursday|1957Q4|N|N|N|2436174|2436507|2435827|2436100|N|N|N|N|N| +2436193|AAAAAAAABGMCFCAA|1957-12-20|695|3025|233|1957|5|12|20|4|1957|233|3025|Friday|1957Q4|N|Y|N|2436174|2436507|2435828|2436101|N|N|N|N|N| +2436194|AAAAAAAACGMCFCAA|1957-12-21|695|3025|233|1957|6|12|21|4|1957|233|3025|Saturday|1957Q4|N|Y|N|2436174|2436507|2435829|2436102|N|N|N|N|N| +2436195|AAAAAAAADGMCFCAA|1957-12-22|695|3025|233|1957|0|12|22|4|1957|233|3025|Sunday|1957Q4|N|N|N|2436174|2436507|2435830|2436103|N|N|N|N|N| +2436196|AAAAAAAAEGMCFCAA|1957-12-23|695|3025|233|1957|1|12|23|4|1957|233|3025|Monday|1957Q4|N|N|N|2436174|2436507|2435831|2436104|N|N|N|N|N| +2436197|AAAAAAAAFGMCFCAA|1957-12-24|695|3026|233|1957|2|12|24|4|1957|233|3026|Tuesday|1957Q4|N|N|N|2436174|2436507|2435832|2436105|N|N|N|N|N| +2436198|AAAAAAAAGGMCFCAA|1957-12-25|695|3026|233|1957|3|12|25|4|1957|233|3026|Wednesday|1957Q4|N|N|N|2436174|2436507|2435833|2436106|N|N|N|N|N| +2436199|AAAAAAAAHGMCFCAA|1957-12-26|695|3026|233|1957|4|12|26|4|1957|233|3026|Thursday|1957Q4|Y|N|N|2436174|2436507|2435834|2436107|N|N|N|N|N| +2436200|AAAAAAAAIGMCFCAA|1957-12-27|695|3026|233|1957|5|12|27|4|1957|233|3026|Friday|1957Q4|N|Y|Y|2436174|2436507|2435835|2436108|N|N|N|N|N| +2436201|AAAAAAAAJGMCFCAA|1957-12-28|695|3026|233|1957|6|12|28|4|1957|233|3026|Saturday|1957Q4|N|Y|N|2436174|2436507|2435836|2436109|N|N|N|N|N| +2436202|AAAAAAAAKGMCFCAA|1957-12-29|695|3026|233|1957|0|12|29|4|1957|233|3026|Sunday|1957Q4|N|N|N|2436174|2436507|2435837|2436110|N|N|N|N|N| +2436203|AAAAAAAALGMCFCAA|1957-12-30|695|3026|233|1957|1|12|30|4|1957|233|3026|Monday|1957Q4|N|N|N|2436174|2436507|2435838|2436111|N|N|N|N|N| +2436204|AAAAAAAAMGMCFCAA|1957-12-31|695|3027|233|1957|2|12|31|4|1957|233|3027|Tuesday|1957Q4|N|N|N|2436174|2436507|2435839|2436112|N|N|N|N|N| +2436205|AAAAAAAANGMCFCAA|1958-01-01|696|3027|233|1958|3|1|1|1|1958|233|3027|Wednesday|1958Q1|Y|N|N|2436205|2436204|2435840|2436113|N|N|N|N|N| +2436206|AAAAAAAAOGMCFCAA|1958-01-02|696|3027|233|1958|4|1|2|1|1958|233|3027|Thursday|1958Q1|N|N|Y|2436205|2436204|2435841|2436114|N|N|N|N|N| +2436207|AAAAAAAAPGMCFCAA|1958-01-03|696|3027|233|1958|5|1|3|1|1958|233|3027|Friday|1958Q1|N|Y|N|2436205|2436204|2435842|2436115|N|N|N|N|N| +2436208|AAAAAAAAAHMCFCAA|1958-01-04|696|3027|233|1958|6|1|4|1|1958|233|3027|Saturday|1958Q1|N|Y|N|2436205|2436204|2435843|2436116|N|N|N|N|N| +2436209|AAAAAAAABHMCFCAA|1958-01-05|696|3027|233|1958|0|1|5|1|1958|233|3027|Sunday|1958Q1|N|N|N|2436205|2436204|2435844|2436117|N|N|N|N|N| +2436210|AAAAAAAACHMCFCAA|1958-01-06|696|3027|233|1958|1|1|6|1|1958|233|3027|Monday|1958Q1|N|N|N|2436205|2436204|2435845|2436118|N|N|N|N|N| +2436211|AAAAAAAADHMCFCAA|1958-01-07|696|3028|233|1958|2|1|7|1|1958|233|3028|Tuesday|1958Q1|N|N|N|2436205|2436204|2435846|2436119|N|N|N|N|N| +2436212|AAAAAAAAEHMCFCAA|1958-01-08|696|3028|233|1958|3|1|8|1|1958|233|3028|Wednesday|1958Q1|N|N|N|2436205|2436204|2435847|2436120|N|N|N|N|N| +2436213|AAAAAAAAFHMCFCAA|1958-01-09|696|3028|233|1958|4|1|9|1|1958|233|3028|Thursday|1958Q1|N|N|N|2436205|2436204|2435848|2436121|N|N|N|N|N| +2436214|AAAAAAAAGHMCFCAA|1958-01-10|696|3028|233|1958|5|1|10|1|1958|233|3028|Friday|1958Q1|N|Y|N|2436205|2436204|2435849|2436122|N|N|N|N|N| +2436215|AAAAAAAAHHMCFCAA|1958-01-11|696|3028|233|1958|6|1|11|1|1958|233|3028|Saturday|1958Q1|N|Y|N|2436205|2436204|2435850|2436123|N|N|N|N|N| +2436216|AAAAAAAAIHMCFCAA|1958-01-12|696|3028|233|1958|0|1|12|1|1958|233|3028|Sunday|1958Q1|N|N|N|2436205|2436204|2435851|2436124|N|N|N|N|N| +2436217|AAAAAAAAJHMCFCAA|1958-01-13|696|3028|233|1958|1|1|13|1|1958|233|3028|Monday|1958Q1|N|N|N|2436205|2436204|2435852|2436125|N|N|N|N|N| +2436218|AAAAAAAAKHMCFCAA|1958-01-14|696|3029|233|1958|2|1|14|1|1958|233|3029|Tuesday|1958Q1|N|N|N|2436205|2436204|2435853|2436126|N|N|N|N|N| +2436219|AAAAAAAALHMCFCAA|1958-01-15|696|3029|233|1958|3|1|15|1|1958|233|3029|Wednesday|1958Q1|N|N|N|2436205|2436204|2435854|2436127|N|N|N|N|N| +2436220|AAAAAAAAMHMCFCAA|1958-01-16|696|3029|233|1958|4|1|16|1|1958|233|3029|Thursday|1958Q1|N|N|N|2436205|2436204|2435855|2436128|N|N|N|N|N| +2436221|AAAAAAAANHMCFCAA|1958-01-17|696|3029|233|1958|5|1|17|1|1958|233|3029|Friday|1958Q1|N|Y|N|2436205|2436204|2435856|2436129|N|N|N|N|N| +2436222|AAAAAAAAOHMCFCAA|1958-01-18|696|3029|233|1958|6|1|18|1|1958|233|3029|Saturday|1958Q1|N|Y|N|2436205|2436204|2435857|2436130|N|N|N|N|N| +2436223|AAAAAAAAPHMCFCAA|1958-01-19|696|3029|233|1958|0|1|19|1|1958|233|3029|Sunday|1958Q1|N|N|N|2436205|2436204|2435858|2436131|N|N|N|N|N| +2436224|AAAAAAAAAIMCFCAA|1958-01-20|696|3029|233|1958|1|1|20|1|1958|233|3029|Monday|1958Q1|N|N|N|2436205|2436204|2435859|2436132|N|N|N|N|N| +2436225|AAAAAAAABIMCFCAA|1958-01-21|696|3030|233|1958|2|1|21|1|1958|233|3030|Tuesday|1958Q1|N|N|N|2436205|2436204|2435860|2436133|N|N|N|N|N| +2436226|AAAAAAAACIMCFCAA|1958-01-22|696|3030|233|1958|3|1|22|1|1958|233|3030|Wednesday|1958Q1|N|N|N|2436205|2436204|2435861|2436134|N|N|N|N|N| +2436227|AAAAAAAADIMCFCAA|1958-01-23|696|3030|233|1958|4|1|23|1|1958|233|3030|Thursday|1958Q1|N|N|N|2436205|2436204|2435862|2436135|N|N|N|N|N| +2436228|AAAAAAAAEIMCFCAA|1958-01-24|696|3030|233|1958|5|1|24|1|1958|233|3030|Friday|1958Q1|N|Y|N|2436205|2436204|2435863|2436136|N|N|N|N|N| +2436229|AAAAAAAAFIMCFCAA|1958-01-25|696|3030|233|1958|6|1|25|1|1958|233|3030|Saturday|1958Q1|N|Y|N|2436205|2436204|2435864|2436137|N|N|N|N|N| +2436230|AAAAAAAAGIMCFCAA|1958-01-26|696|3030|233|1958|0|1|26|1|1958|233|3030|Sunday|1958Q1|N|N|N|2436205|2436204|2435865|2436138|N|N|N|N|N| +2436231|AAAAAAAAHIMCFCAA|1958-01-27|696|3030|233|1958|1|1|27|1|1958|233|3030|Monday|1958Q1|N|N|N|2436205|2436204|2435866|2436139|N|N|N|N|N| +2436232|AAAAAAAAIIMCFCAA|1958-01-28|696|3031|233|1958|2|1|28|1|1958|233|3031|Tuesday|1958Q1|N|N|N|2436205|2436204|2435867|2436140|N|N|N|N|N| +2436233|AAAAAAAAJIMCFCAA|1958-01-29|696|3031|233|1958|3|1|29|1|1958|233|3031|Wednesday|1958Q1|N|N|N|2436205|2436204|2435868|2436141|N|N|N|N|N| +2436234|AAAAAAAAKIMCFCAA|1958-01-30|696|3031|233|1958|4|1|30|1|1958|233|3031|Thursday|1958Q1|N|N|N|2436205|2436204|2435869|2436142|N|N|N|N|N| +2436235|AAAAAAAALIMCFCAA|1958-01-31|696|3031|233|1958|5|1|31|1|1958|233|3031|Friday|1958Q1|N|Y|N|2436205|2436204|2435870|2436143|N|N|N|N|N| +2436236|AAAAAAAAMIMCFCAA|1958-02-01|697|3031|233|1958|6|2|1|1|1958|233|3031|Saturday|1958Q1|N|Y|N|2436236|2436266|2435871|2436144|N|N|N|N|N| +2436237|AAAAAAAANIMCFCAA|1958-02-02|697|3031|233|1958|0|2|2|1|1958|233|3031|Sunday|1958Q1|N|N|N|2436236|2436266|2435872|2436145|N|N|N|N|N| +2436238|AAAAAAAAOIMCFCAA|1958-02-03|697|3031|233|1958|1|2|3|1|1958|233|3031|Monday|1958Q1|N|N|N|2436236|2436266|2435873|2436146|N|N|N|N|N| +2436239|AAAAAAAAPIMCFCAA|1958-02-04|697|3032|233|1958|2|2|4|1|1958|233|3032|Tuesday|1958Q1|N|N|N|2436236|2436266|2435874|2436147|N|N|N|N|N| +2436240|AAAAAAAAAJMCFCAA|1958-02-05|697|3032|233|1958|3|2|5|1|1958|233|3032|Wednesday|1958Q1|N|N|N|2436236|2436266|2435875|2436148|N|N|N|N|N| +2436241|AAAAAAAABJMCFCAA|1958-02-06|697|3032|233|1958|4|2|6|1|1958|233|3032|Thursday|1958Q1|N|N|N|2436236|2436266|2435876|2436149|N|N|N|N|N| +2436242|AAAAAAAACJMCFCAA|1958-02-07|697|3032|233|1958|5|2|7|1|1958|233|3032|Friday|1958Q1|N|Y|N|2436236|2436266|2435877|2436150|N|N|N|N|N| +2436243|AAAAAAAADJMCFCAA|1958-02-08|697|3032|233|1958|6|2|8|1|1958|233|3032|Saturday|1958Q1|N|Y|N|2436236|2436266|2435878|2436151|N|N|N|N|N| +2436244|AAAAAAAAEJMCFCAA|1958-02-09|697|3032|233|1958|0|2|9|1|1958|233|3032|Sunday|1958Q1|N|N|N|2436236|2436266|2435879|2436152|N|N|N|N|N| +2436245|AAAAAAAAFJMCFCAA|1958-02-10|697|3032|233|1958|1|2|10|1|1958|233|3032|Monday|1958Q1|N|N|N|2436236|2436266|2435880|2436153|N|N|N|N|N| +2436246|AAAAAAAAGJMCFCAA|1958-02-11|697|3033|233|1958|2|2|11|1|1958|233|3033|Tuesday|1958Q1|N|N|N|2436236|2436266|2435881|2436154|N|N|N|N|N| +2436247|AAAAAAAAHJMCFCAA|1958-02-12|697|3033|233|1958|3|2|12|1|1958|233|3033|Wednesday|1958Q1|N|N|N|2436236|2436266|2435882|2436155|N|N|N|N|N| +2436248|AAAAAAAAIJMCFCAA|1958-02-13|697|3033|233|1958|4|2|13|1|1958|233|3033|Thursday|1958Q1|N|N|N|2436236|2436266|2435883|2436156|N|N|N|N|N| +2436249|AAAAAAAAJJMCFCAA|1958-02-14|697|3033|233|1958|5|2|14|1|1958|233|3033|Friday|1958Q1|N|Y|N|2436236|2436266|2435884|2436157|N|N|N|N|N| +2436250|AAAAAAAAKJMCFCAA|1958-02-15|697|3033|233|1958|6|2|15|1|1958|233|3033|Saturday|1958Q1|N|Y|N|2436236|2436266|2435885|2436158|N|N|N|N|N| +2436251|AAAAAAAALJMCFCAA|1958-02-16|697|3033|233|1958|0|2|16|1|1958|233|3033|Sunday|1958Q1|N|N|N|2436236|2436266|2435886|2436159|N|N|N|N|N| +2436252|AAAAAAAAMJMCFCAA|1958-02-17|697|3033|233|1958|1|2|17|1|1958|233|3033|Monday|1958Q1|N|N|N|2436236|2436266|2435887|2436160|N|N|N|N|N| +2436253|AAAAAAAANJMCFCAA|1958-02-18|697|3034|233|1958|2|2|18|1|1958|233|3034|Tuesday|1958Q1|N|N|N|2436236|2436266|2435888|2436161|N|N|N|N|N| +2436254|AAAAAAAAOJMCFCAA|1958-02-19|697|3034|233|1958|3|2|19|1|1958|233|3034|Wednesday|1958Q1|N|N|N|2436236|2436266|2435889|2436162|N|N|N|N|N| +2436255|AAAAAAAAPJMCFCAA|1958-02-20|697|3034|233|1958|4|2|20|1|1958|233|3034|Thursday|1958Q1|N|N|N|2436236|2436266|2435890|2436163|N|N|N|N|N| +2436256|AAAAAAAAAKMCFCAA|1958-02-21|697|3034|233|1958|5|2|21|1|1958|233|3034|Friday|1958Q1|N|Y|N|2436236|2436266|2435891|2436164|N|N|N|N|N| +2436257|AAAAAAAABKMCFCAA|1958-02-22|697|3034|233|1958|6|2|22|1|1958|233|3034|Saturday|1958Q1|N|Y|N|2436236|2436266|2435892|2436165|N|N|N|N|N| +2436258|AAAAAAAACKMCFCAA|1958-02-23|697|3034|233|1958|0|2|23|1|1958|233|3034|Sunday|1958Q1|N|N|N|2436236|2436266|2435893|2436166|N|N|N|N|N| +2436259|AAAAAAAADKMCFCAA|1958-02-24|697|3034|233|1958|1|2|24|1|1958|233|3034|Monday|1958Q1|N|N|N|2436236|2436266|2435894|2436167|N|N|N|N|N| +2436260|AAAAAAAAEKMCFCAA|1958-02-25|697|3035|233|1958|2|2|25|1|1958|233|3035|Tuesday|1958Q1|N|N|N|2436236|2436266|2435895|2436168|N|N|N|N|N| +2436261|AAAAAAAAFKMCFCAA|1958-02-26|697|3035|233|1958|3|2|26|1|1958|233|3035|Wednesday|1958Q1|N|N|N|2436236|2436266|2435896|2436169|N|N|N|N|N| +2436262|AAAAAAAAGKMCFCAA|1958-02-27|697|3035|233|1958|4|2|27|1|1958|233|3035|Thursday|1958Q1|N|N|N|2436236|2436266|2435897|2436170|N|N|N|N|N| +2436263|AAAAAAAAHKMCFCAA|1958-02-28|697|3035|233|1958|5|2|28|1|1958|233|3035|Friday|1958Q1|N|Y|N|2436236|2436266|2435898|2436171|N|N|N|N|N| +2436264|AAAAAAAAIKMCFCAA|1958-03-01|698|3035|234|1958|6|3|1|1|1958|234|3035|Saturday|1958Q1|N|Y|N|2436264|2436322|2435899|2436172|N|N|N|N|N| +2436265|AAAAAAAAJKMCFCAA|1958-03-02|698|3035|234|1958|0|3|2|1|1958|234|3035|Sunday|1958Q1|N|N|N|2436264|2436322|2435900|2436173|N|N|N|N|N| +2436266|AAAAAAAAKKMCFCAA|1958-03-03|698|3035|234|1958|1|3|3|1|1958|234|3035|Monday|1958Q1|N|N|N|2436264|2436322|2435901|2436174|N|N|N|N|N| +2436267|AAAAAAAALKMCFCAA|1958-03-04|698|3036|234|1958|2|3|4|1|1958|234|3036|Tuesday|1958Q1|N|N|N|2436264|2436322|2435902|2436175|N|N|N|N|N| +2436268|AAAAAAAAMKMCFCAA|1958-03-05|698|3036|234|1958|3|3|5|1|1958|234|3036|Wednesday|1958Q1|N|N|N|2436264|2436322|2435903|2436176|N|N|N|N|N| +2436269|AAAAAAAANKMCFCAA|1958-03-06|698|3036|234|1958|4|3|6|1|1958|234|3036|Thursday|1958Q1|N|N|N|2436264|2436322|2435904|2436177|N|N|N|N|N| +2436270|AAAAAAAAOKMCFCAA|1958-03-07|698|3036|234|1958|5|3|7|1|1958|234|3036|Friday|1958Q1|N|Y|N|2436264|2436322|2435905|2436178|N|N|N|N|N| +2436271|AAAAAAAAPKMCFCAA|1958-03-08|698|3036|234|1958|6|3|8|1|1958|234|3036|Saturday|1958Q1|N|Y|N|2436264|2436322|2435906|2436179|N|N|N|N|N| +2436272|AAAAAAAAALMCFCAA|1958-03-09|698|3036|234|1958|0|3|9|1|1958|234|3036|Sunday|1958Q1|N|N|N|2436264|2436322|2435907|2436180|N|N|N|N|N| +2436273|AAAAAAAABLMCFCAA|1958-03-10|698|3036|234|1958|1|3|10|1|1958|234|3036|Monday|1958Q1|N|N|N|2436264|2436322|2435908|2436181|N|N|N|N|N| +2436274|AAAAAAAACLMCFCAA|1958-03-11|698|3037|234|1958|2|3|11|1|1958|234|3037|Tuesday|1958Q1|N|N|N|2436264|2436322|2435909|2436182|N|N|N|N|N| +2436275|AAAAAAAADLMCFCAA|1958-03-12|698|3037|234|1958|3|3|12|1|1958|234|3037|Wednesday|1958Q1|N|N|N|2436264|2436322|2435910|2436183|N|N|N|N|N| +2436276|AAAAAAAAELMCFCAA|1958-03-13|698|3037|234|1958|4|3|13|1|1958|234|3037|Thursday|1958Q1|N|N|N|2436264|2436322|2435911|2436184|N|N|N|N|N| +2436277|AAAAAAAAFLMCFCAA|1958-03-14|698|3037|234|1958|5|3|14|1|1958|234|3037|Friday|1958Q1|N|Y|N|2436264|2436322|2435912|2436185|N|N|N|N|N| +2436278|AAAAAAAAGLMCFCAA|1958-03-15|698|3037|234|1958|6|3|15|1|1958|234|3037|Saturday|1958Q1|N|Y|N|2436264|2436322|2435913|2436186|N|N|N|N|N| +2436279|AAAAAAAAHLMCFCAA|1958-03-16|698|3037|234|1958|0|3|16|1|1958|234|3037|Sunday|1958Q1|N|N|N|2436264|2436322|2435914|2436187|N|N|N|N|N| +2436280|AAAAAAAAILMCFCAA|1958-03-17|698|3037|234|1958|1|3|17|1|1958|234|3037|Monday|1958Q1|N|N|N|2436264|2436322|2435915|2436188|N|N|N|N|N| +2436281|AAAAAAAAJLMCFCAA|1958-03-18|698|3038|234|1958|2|3|18|1|1958|234|3038|Tuesday|1958Q1|N|N|N|2436264|2436322|2435916|2436189|N|N|N|N|N| +2436282|AAAAAAAAKLMCFCAA|1958-03-19|698|3038|234|1958|3|3|19|1|1958|234|3038|Wednesday|1958Q1|N|N|N|2436264|2436322|2435917|2436190|N|N|N|N|N| +2436283|AAAAAAAALLMCFCAA|1958-03-20|698|3038|234|1958|4|3|20|1|1958|234|3038|Thursday|1958Q1|N|N|N|2436264|2436322|2435918|2436191|N|N|N|N|N| +2436284|AAAAAAAAMLMCFCAA|1958-03-21|698|3038|234|1958|5|3|21|1|1958|234|3038|Friday|1958Q1|N|Y|N|2436264|2436322|2435919|2436192|N|N|N|N|N| +2436285|AAAAAAAANLMCFCAA|1958-03-22|698|3038|234|1958|6|3|22|1|1958|234|3038|Saturday|1958Q1|N|Y|N|2436264|2436322|2435920|2436193|N|N|N|N|N| +2436286|AAAAAAAAOLMCFCAA|1958-03-23|698|3038|234|1958|0|3|23|1|1958|234|3038|Sunday|1958Q1|N|N|N|2436264|2436322|2435921|2436194|N|N|N|N|N| +2436287|AAAAAAAAPLMCFCAA|1958-03-24|698|3038|234|1958|1|3|24|1|1958|234|3038|Monday|1958Q1|N|N|N|2436264|2436322|2435922|2436195|N|N|N|N|N| +2436288|AAAAAAAAAMMCFCAA|1958-03-25|698|3039|234|1958|2|3|25|1|1958|234|3039|Tuesday|1958Q1|N|N|N|2436264|2436322|2435923|2436196|N|N|N|N|N| +2436289|AAAAAAAABMMCFCAA|1958-03-26|698|3039|234|1958|3|3|26|1|1958|234|3039|Wednesday|1958Q1|N|N|N|2436264|2436322|2435924|2436197|N|N|N|N|N| +2436290|AAAAAAAACMMCFCAA|1958-03-27|698|3039|234|1958|4|3|27|1|1958|234|3039|Thursday|1958Q1|N|N|N|2436264|2436322|2435925|2436198|N|N|N|N|N| +2436291|AAAAAAAADMMCFCAA|1958-03-28|698|3039|234|1958|5|3|28|1|1958|234|3039|Friday|1958Q1|N|Y|N|2436264|2436322|2435926|2436199|N|N|N|N|N| +2436292|AAAAAAAAEMMCFCAA|1958-03-29|698|3039|234|1958|6|3|29|1|1958|234|3039|Saturday|1958Q1|N|Y|N|2436264|2436322|2435927|2436200|N|N|N|N|N| +2436293|AAAAAAAAFMMCFCAA|1958-03-30|698|3039|234|1958|0|3|30|1|1958|234|3039|Sunday|1958Q1|N|N|N|2436264|2436322|2435928|2436201|N|N|N|N|N| +2436294|AAAAAAAAGMMCFCAA|1958-03-31|698|3039|234|1958|1|3|31|1|1958|234|3039|Monday|1958Q1|N|N|N|2436264|2436322|2435929|2436202|N|N|N|N|N| +2436295|AAAAAAAAHMMCFCAA|1958-04-01|699|3040|234|1958|2|4|1|1|1958|234|3040|Tuesday|1958Q1|N|N|N|2436295|2436384|2435930|2436205|N|N|N|N|N| +2436296|AAAAAAAAIMMCFCAA|1958-04-02|699|3040|234|1958|3|4|2|2|1958|234|3040|Wednesday|1958Q2|N|N|N|2436295|2436384|2435931|2436206|N|N|N|N|N| +2436297|AAAAAAAAJMMCFCAA|1958-04-03|699|3040|234|1958|4|4|3|2|1958|234|3040|Thursday|1958Q2|N|N|N|2436295|2436384|2435932|2436207|N|N|N|N|N| +2436298|AAAAAAAAKMMCFCAA|1958-04-04|699|3040|234|1958|5|4|4|2|1958|234|3040|Friday|1958Q2|N|Y|N|2436295|2436384|2435933|2436208|N|N|N|N|N| +2436299|AAAAAAAALMMCFCAA|1958-04-05|699|3040|234|1958|6|4|5|2|1958|234|3040|Saturday|1958Q2|N|Y|N|2436295|2436384|2435934|2436209|N|N|N|N|N| +2436300|AAAAAAAAMMMCFCAA|1958-04-06|699|3040|234|1958|0|4|6|2|1958|234|3040|Sunday|1958Q2|N|N|N|2436295|2436384|2435935|2436210|N|N|N|N|N| +2436301|AAAAAAAANMMCFCAA|1958-04-07|699|3040|234|1958|1|4|7|2|1958|234|3040|Monday|1958Q2|N|N|N|2436295|2436384|2435936|2436211|N|N|N|N|N| +2436302|AAAAAAAAOMMCFCAA|1958-04-08|699|3041|234|1958|2|4|8|2|1958|234|3041|Tuesday|1958Q2|N|N|N|2436295|2436384|2435937|2436212|N|N|N|N|N| +2436303|AAAAAAAAPMMCFCAA|1958-04-09|699|3041|234|1958|3|4|9|2|1958|234|3041|Wednesday|1958Q2|N|N|N|2436295|2436384|2435938|2436213|N|N|N|N|N| +2436304|AAAAAAAAANMCFCAA|1958-04-10|699|3041|234|1958|4|4|10|2|1958|234|3041|Thursday|1958Q2|N|N|N|2436295|2436384|2435939|2436214|N|N|N|N|N| +2436305|AAAAAAAABNMCFCAA|1958-04-11|699|3041|234|1958|5|4|11|2|1958|234|3041|Friday|1958Q2|N|Y|N|2436295|2436384|2435940|2436215|N|N|N|N|N| +2436306|AAAAAAAACNMCFCAA|1958-04-12|699|3041|234|1958|6|4|12|2|1958|234|3041|Saturday|1958Q2|N|Y|N|2436295|2436384|2435941|2436216|N|N|N|N|N| +2436307|AAAAAAAADNMCFCAA|1958-04-13|699|3041|234|1958|0|4|13|2|1958|234|3041|Sunday|1958Q2|N|N|N|2436295|2436384|2435942|2436217|N|N|N|N|N| +2436308|AAAAAAAAENMCFCAA|1958-04-14|699|3041|234|1958|1|4|14|2|1958|234|3041|Monday|1958Q2|N|N|N|2436295|2436384|2435943|2436218|N|N|N|N|N| +2436309|AAAAAAAAFNMCFCAA|1958-04-15|699|3042|234|1958|2|4|15|2|1958|234|3042|Tuesday|1958Q2|N|N|N|2436295|2436384|2435944|2436219|N|N|N|N|N| +2436310|AAAAAAAAGNMCFCAA|1958-04-16|699|3042|234|1958|3|4|16|2|1958|234|3042|Wednesday|1958Q2|N|N|N|2436295|2436384|2435945|2436220|N|N|N|N|N| +2436311|AAAAAAAAHNMCFCAA|1958-04-17|699|3042|234|1958|4|4|17|2|1958|234|3042|Thursday|1958Q2|N|N|N|2436295|2436384|2435946|2436221|N|N|N|N|N| +2436312|AAAAAAAAINMCFCAA|1958-04-18|699|3042|234|1958|5|4|18|2|1958|234|3042|Friday|1958Q2|N|Y|N|2436295|2436384|2435947|2436222|N|N|N|N|N| +2436313|AAAAAAAAJNMCFCAA|1958-04-19|699|3042|234|1958|6|4|19|2|1958|234|3042|Saturday|1958Q2|N|Y|N|2436295|2436384|2435948|2436223|N|N|N|N|N| +2436314|AAAAAAAAKNMCFCAA|1958-04-20|699|3042|234|1958|0|4|20|2|1958|234|3042|Sunday|1958Q2|N|N|N|2436295|2436384|2435949|2436224|N|N|N|N|N| +2436315|AAAAAAAALNMCFCAA|1958-04-21|699|3042|234|1958|1|4|21|2|1958|234|3042|Monday|1958Q2|N|N|N|2436295|2436384|2435950|2436225|N|N|N|N|N| +2436316|AAAAAAAAMNMCFCAA|1958-04-22|699|3043|234|1958|2|4|22|2|1958|234|3043|Tuesday|1958Q2|N|N|N|2436295|2436384|2435951|2436226|N|N|N|N|N| +2436317|AAAAAAAANNMCFCAA|1958-04-23|699|3043|234|1958|3|4|23|2|1958|234|3043|Wednesday|1958Q2|N|N|N|2436295|2436384|2435952|2436227|N|N|N|N|N| +2436318|AAAAAAAAONMCFCAA|1958-04-24|699|3043|234|1958|4|4|24|2|1958|234|3043|Thursday|1958Q2|N|N|N|2436295|2436384|2435953|2436228|N|N|N|N|N| +2436319|AAAAAAAAPNMCFCAA|1958-04-25|699|3043|234|1958|5|4|25|2|1958|234|3043|Friday|1958Q2|N|Y|N|2436295|2436384|2435954|2436229|N|N|N|N|N| +2436320|AAAAAAAAAOMCFCAA|1958-04-26|699|3043|234|1958|6|4|26|2|1958|234|3043|Saturday|1958Q2|N|Y|N|2436295|2436384|2435955|2436230|N|N|N|N|N| +2436321|AAAAAAAABOMCFCAA|1958-04-27|699|3043|234|1958|0|4|27|2|1958|234|3043|Sunday|1958Q2|N|N|N|2436295|2436384|2435956|2436231|N|N|N|N|N| +2436322|AAAAAAAACOMCFCAA|1958-04-28|699|3043|234|1958|1|4|28|2|1958|234|3043|Monday|1958Q2|N|N|N|2436295|2436384|2435957|2436232|N|N|N|N|N| +2436323|AAAAAAAADOMCFCAA|1958-04-29|699|3044|234|1958|2|4|29|2|1958|234|3044|Tuesday|1958Q2|N|N|N|2436295|2436384|2435958|2436233|N|N|N|N|N| +2436324|AAAAAAAAEOMCFCAA|1958-04-30|699|3044|234|1958|3|4|30|2|1958|234|3044|Wednesday|1958Q2|N|N|N|2436295|2436384|2435959|2436234|N|N|N|N|N| +2436325|AAAAAAAAFOMCFCAA|1958-05-01|700|3044|234|1958|4|5|1|2|1958|234|3044|Thursday|1958Q2|N|N|N|2436325|2436444|2435960|2436235|N|N|N|N|N| +2436326|AAAAAAAAGOMCFCAA|1958-05-02|700|3044|234|1958|5|5|2|2|1958|234|3044|Friday|1958Q2|N|Y|N|2436325|2436444|2435961|2436236|N|N|N|N|N| +2436327|AAAAAAAAHOMCFCAA|1958-05-03|700|3044|234|1958|6|5|3|2|1958|234|3044|Saturday|1958Q2|N|Y|N|2436325|2436444|2435962|2436237|N|N|N|N|N| +2436328|AAAAAAAAIOMCFCAA|1958-05-04|700|3044|234|1958|0|5|4|2|1958|234|3044|Sunday|1958Q2|N|N|N|2436325|2436444|2435963|2436238|N|N|N|N|N| +2436329|AAAAAAAAJOMCFCAA|1958-05-05|700|3044|234|1958|1|5|5|2|1958|234|3044|Monday|1958Q2|N|N|N|2436325|2436444|2435964|2436239|N|N|N|N|N| +2436330|AAAAAAAAKOMCFCAA|1958-05-06|700|3045|234|1958|2|5|6|2|1958|234|3045|Tuesday|1958Q2|N|N|N|2436325|2436444|2435965|2436240|N|N|N|N|N| +2436331|AAAAAAAALOMCFCAA|1958-05-07|700|3045|234|1958|3|5|7|2|1958|234|3045|Wednesday|1958Q2|N|N|N|2436325|2436444|2435966|2436241|N|N|N|N|N| +2436332|AAAAAAAAMOMCFCAA|1958-05-08|700|3045|234|1958|4|5|8|2|1958|234|3045|Thursday|1958Q2|N|N|N|2436325|2436444|2435967|2436242|N|N|N|N|N| +2436333|AAAAAAAANOMCFCAA|1958-05-09|700|3045|234|1958|5|5|9|2|1958|234|3045|Friday|1958Q2|N|Y|N|2436325|2436444|2435968|2436243|N|N|N|N|N| +2436334|AAAAAAAAOOMCFCAA|1958-05-10|700|3045|234|1958|6|5|10|2|1958|234|3045|Saturday|1958Q2|N|Y|N|2436325|2436444|2435969|2436244|N|N|N|N|N| +2436335|AAAAAAAAPOMCFCAA|1958-05-11|700|3045|234|1958|0|5|11|2|1958|234|3045|Sunday|1958Q2|N|N|N|2436325|2436444|2435970|2436245|N|N|N|N|N| +2436336|AAAAAAAAAPMCFCAA|1958-05-12|700|3045|234|1958|1|5|12|2|1958|234|3045|Monday|1958Q2|N|N|N|2436325|2436444|2435971|2436246|N|N|N|N|N| +2436337|AAAAAAAABPMCFCAA|1958-05-13|700|3046|234|1958|2|5|13|2|1958|234|3046|Tuesday|1958Q2|N|N|N|2436325|2436444|2435972|2436247|N|N|N|N|N| +2436338|AAAAAAAACPMCFCAA|1958-05-14|700|3046|234|1958|3|5|14|2|1958|234|3046|Wednesday|1958Q2|N|N|N|2436325|2436444|2435973|2436248|N|N|N|N|N| +2436339|AAAAAAAADPMCFCAA|1958-05-15|700|3046|234|1958|4|5|15|2|1958|234|3046|Thursday|1958Q2|N|N|N|2436325|2436444|2435974|2436249|N|N|N|N|N| +2436340|AAAAAAAAEPMCFCAA|1958-05-16|700|3046|234|1958|5|5|16|2|1958|234|3046|Friday|1958Q2|N|Y|N|2436325|2436444|2435975|2436250|N|N|N|N|N| +2436341|AAAAAAAAFPMCFCAA|1958-05-17|700|3046|234|1958|6|5|17|2|1958|234|3046|Saturday|1958Q2|N|Y|N|2436325|2436444|2435976|2436251|N|N|N|N|N| +2436342|AAAAAAAAGPMCFCAA|1958-05-18|700|3046|234|1958|0|5|18|2|1958|234|3046|Sunday|1958Q2|N|N|N|2436325|2436444|2435977|2436252|N|N|N|N|N| +2436343|AAAAAAAAHPMCFCAA|1958-05-19|700|3046|234|1958|1|5|19|2|1958|234|3046|Monday|1958Q2|N|N|N|2436325|2436444|2435978|2436253|N|N|N|N|N| +2436344|AAAAAAAAIPMCFCAA|1958-05-20|700|3047|234|1958|2|5|20|2|1958|234|3047|Tuesday|1958Q2|N|N|N|2436325|2436444|2435979|2436254|N|N|N|N|N| +2436345|AAAAAAAAJPMCFCAA|1958-05-21|700|3047|234|1958|3|5|21|2|1958|234|3047|Wednesday|1958Q2|N|N|N|2436325|2436444|2435980|2436255|N|N|N|N|N| +2436346|AAAAAAAAKPMCFCAA|1958-05-22|700|3047|234|1958|4|5|22|2|1958|234|3047|Thursday|1958Q2|N|N|N|2436325|2436444|2435981|2436256|N|N|N|N|N| +2436347|AAAAAAAALPMCFCAA|1958-05-23|700|3047|234|1958|5|5|23|2|1958|234|3047|Friday|1958Q2|N|Y|N|2436325|2436444|2435982|2436257|N|N|N|N|N| +2436348|AAAAAAAAMPMCFCAA|1958-05-24|700|3047|234|1958|6|5|24|2|1958|234|3047|Saturday|1958Q2|N|Y|N|2436325|2436444|2435983|2436258|N|N|N|N|N| +2436349|AAAAAAAANPMCFCAA|1958-05-25|700|3047|234|1958|0|5|25|2|1958|234|3047|Sunday|1958Q2|N|N|N|2436325|2436444|2435984|2436259|N|N|N|N|N| +2436350|AAAAAAAAOPMCFCAA|1958-05-26|700|3047|234|1958|1|5|26|2|1958|234|3047|Monday|1958Q2|N|N|N|2436325|2436444|2435985|2436260|N|N|N|N|N| +2436351|AAAAAAAAPPMCFCAA|1958-05-27|700|3048|234|1958|2|5|27|2|1958|234|3048|Tuesday|1958Q2|N|N|N|2436325|2436444|2435986|2436261|N|N|N|N|N| +2436352|AAAAAAAAAANCFCAA|1958-05-28|700|3048|234|1958|3|5|28|2|1958|234|3048|Wednesday|1958Q2|N|N|N|2436325|2436444|2435987|2436262|N|N|N|N|N| +2436353|AAAAAAAABANCFCAA|1958-05-29|700|3048|234|1958|4|5|29|2|1958|234|3048|Thursday|1958Q2|N|N|N|2436325|2436444|2435988|2436263|N|N|N|N|N| +2436354|AAAAAAAACANCFCAA|1958-05-30|700|3048|234|1958|5|5|30|2|1958|234|3048|Friday|1958Q2|N|Y|N|2436325|2436444|2435989|2436264|N|N|N|N|N| +2436355|AAAAAAAADANCFCAA|1958-05-31|700|3048|234|1958|6|5|31|2|1958|234|3048|Saturday|1958Q2|N|Y|N|2436325|2436444|2435990|2436265|N|N|N|N|N| +2436356|AAAAAAAAEANCFCAA|1958-06-01|701|3048|235|1958|0|6|1|2|1958|235|3048|Sunday|1958Q2|N|N|N|2436356|2436506|2435991|2436266|N|N|N|N|N| +2436357|AAAAAAAAFANCFCAA|1958-06-02|701|3048|235|1958|1|6|2|2|1958|235|3048|Monday|1958Q2|N|N|N|2436356|2436506|2435992|2436267|N|N|N|N|N| +2436358|AAAAAAAAGANCFCAA|1958-06-03|701|3049|235|1958|2|6|3|2|1958|235|3049|Tuesday|1958Q2|N|N|N|2436356|2436506|2435993|2436268|N|N|N|N|N| +2436359|AAAAAAAAHANCFCAA|1958-06-04|701|3049|235|1958|3|6|4|2|1958|235|3049|Wednesday|1958Q2|N|N|N|2436356|2436506|2435994|2436269|N|N|N|N|N| +2436360|AAAAAAAAIANCFCAA|1958-06-05|701|3049|235|1958|4|6|5|2|1958|235|3049|Thursday|1958Q2|N|N|N|2436356|2436506|2435995|2436270|N|N|N|N|N| +2436361|AAAAAAAAJANCFCAA|1958-06-06|701|3049|235|1958|5|6|6|2|1958|235|3049|Friday|1958Q2|N|Y|N|2436356|2436506|2435996|2436271|N|N|N|N|N| +2436362|AAAAAAAAKANCFCAA|1958-06-07|701|3049|235|1958|6|6|7|2|1958|235|3049|Saturday|1958Q2|N|Y|N|2436356|2436506|2435997|2436272|N|N|N|N|N| +2436363|AAAAAAAALANCFCAA|1958-06-08|701|3049|235|1958|0|6|8|2|1958|235|3049|Sunday|1958Q2|N|N|N|2436356|2436506|2435998|2436273|N|N|N|N|N| +2436364|AAAAAAAAMANCFCAA|1958-06-09|701|3049|235|1958|1|6|9|2|1958|235|3049|Monday|1958Q2|N|N|N|2436356|2436506|2435999|2436274|N|N|N|N|N| +2436365|AAAAAAAANANCFCAA|1958-06-10|701|3050|235|1958|2|6|10|2|1958|235|3050|Tuesday|1958Q2|N|N|N|2436356|2436506|2436000|2436275|N|N|N|N|N| +2436366|AAAAAAAAOANCFCAA|1958-06-11|701|3050|235|1958|3|6|11|2|1958|235|3050|Wednesday|1958Q2|N|N|N|2436356|2436506|2436001|2436276|N|N|N|N|N| +2436367|AAAAAAAAPANCFCAA|1958-06-12|701|3050|235|1958|4|6|12|2|1958|235|3050|Thursday|1958Q2|N|N|N|2436356|2436506|2436002|2436277|N|N|N|N|N| +2436368|AAAAAAAAABNCFCAA|1958-06-13|701|3050|235|1958|5|6|13|2|1958|235|3050|Friday|1958Q2|N|Y|N|2436356|2436506|2436003|2436278|N|N|N|N|N| +2436369|AAAAAAAABBNCFCAA|1958-06-14|701|3050|235|1958|6|6|14|2|1958|235|3050|Saturday|1958Q2|N|Y|N|2436356|2436506|2436004|2436279|N|N|N|N|N| +2436370|AAAAAAAACBNCFCAA|1958-06-15|701|3050|235|1958|0|6|15|2|1958|235|3050|Sunday|1958Q2|N|N|N|2436356|2436506|2436005|2436280|N|N|N|N|N| +2436371|AAAAAAAADBNCFCAA|1958-06-16|701|3050|235|1958|1|6|16|2|1958|235|3050|Monday|1958Q2|N|N|N|2436356|2436506|2436006|2436281|N|N|N|N|N| +2436372|AAAAAAAAEBNCFCAA|1958-06-17|701|3051|235|1958|2|6|17|2|1958|235|3051|Tuesday|1958Q2|N|N|N|2436356|2436506|2436007|2436282|N|N|N|N|N| +2436373|AAAAAAAAFBNCFCAA|1958-06-18|701|3051|235|1958|3|6|18|2|1958|235|3051|Wednesday|1958Q2|N|N|N|2436356|2436506|2436008|2436283|N|N|N|N|N| +2436374|AAAAAAAAGBNCFCAA|1958-06-19|701|3051|235|1958|4|6|19|2|1958|235|3051|Thursday|1958Q2|N|N|N|2436356|2436506|2436009|2436284|N|N|N|N|N| +2436375|AAAAAAAAHBNCFCAA|1958-06-20|701|3051|235|1958|5|6|20|2|1958|235|3051|Friday|1958Q2|N|Y|N|2436356|2436506|2436010|2436285|N|N|N|N|N| +2436376|AAAAAAAAIBNCFCAA|1958-06-21|701|3051|235|1958|6|6|21|2|1958|235|3051|Saturday|1958Q2|N|Y|N|2436356|2436506|2436011|2436286|N|N|N|N|N| +2436377|AAAAAAAAJBNCFCAA|1958-06-22|701|3051|235|1958|0|6|22|2|1958|235|3051|Sunday|1958Q2|N|N|N|2436356|2436506|2436012|2436287|N|N|N|N|N| +2436378|AAAAAAAAKBNCFCAA|1958-06-23|701|3051|235|1958|1|6|23|2|1958|235|3051|Monday|1958Q2|N|N|N|2436356|2436506|2436013|2436288|N|N|N|N|N| +2436379|AAAAAAAALBNCFCAA|1958-06-24|701|3052|235|1958|2|6|24|2|1958|235|3052|Tuesday|1958Q2|N|N|N|2436356|2436506|2436014|2436289|N|N|N|N|N| +2436380|AAAAAAAAMBNCFCAA|1958-06-25|701|3052|235|1958|3|6|25|2|1958|235|3052|Wednesday|1958Q2|N|N|N|2436356|2436506|2436015|2436290|N|N|N|N|N| +2436381|AAAAAAAANBNCFCAA|1958-06-26|701|3052|235|1958|4|6|26|2|1958|235|3052|Thursday|1958Q2|N|N|N|2436356|2436506|2436016|2436291|N|N|N|N|N| +2436382|AAAAAAAAOBNCFCAA|1958-06-27|701|3052|235|1958|5|6|27|2|1958|235|3052|Friday|1958Q2|N|Y|N|2436356|2436506|2436017|2436292|N|N|N|N|N| +2436383|AAAAAAAAPBNCFCAA|1958-06-28|701|3052|235|1958|6|6|28|2|1958|235|3052|Saturday|1958Q2|N|Y|N|2436356|2436506|2436018|2436293|N|N|N|N|N| +2436384|AAAAAAAAACNCFCAA|1958-06-29|701|3052|235|1958|0|6|29|2|1958|235|3052|Sunday|1958Q2|N|N|N|2436356|2436506|2436019|2436294|N|N|N|N|N| +2436385|AAAAAAAABCNCFCAA|1958-06-30|701|3052|235|1958|1|6|30|2|1958|235|3052|Monday|1958Q2|N|N|N|2436356|2436506|2436020|2436295|N|N|N|N|N| +2436386|AAAAAAAACCNCFCAA|1958-07-01|702|3053|235|1958|2|7|1|2|1958|235|3053|Tuesday|1958Q2|N|N|N|2436386|2436566|2436021|2436295|N|N|N|N|N| +2436387|AAAAAAAADCNCFCAA|1958-07-02|702|3053|235|1958|3|7|2|3|1958|235|3053|Wednesday|1958Q3|N|N|N|2436386|2436566|2436022|2436296|N|N|N|N|N| +2436388|AAAAAAAAECNCFCAA|1958-07-03|702|3053|235|1958|4|7|3|3|1958|235|3053|Thursday|1958Q3|N|N|N|2436386|2436566|2436023|2436297|N|N|N|N|N| +2436389|AAAAAAAAFCNCFCAA|1958-07-04|702|3053|235|1958|5|7|4|3|1958|235|3053|Friday|1958Q3|N|Y|N|2436386|2436566|2436024|2436298|N|N|N|N|N| +2436390|AAAAAAAAGCNCFCAA|1958-07-05|702|3053|235|1958|6|7|5|3|1958|235|3053|Saturday|1958Q3|Y|Y|N|2436386|2436566|2436025|2436299|N|N|N|N|N| +2436391|AAAAAAAAHCNCFCAA|1958-07-06|702|3053|235|1958|0|7|6|3|1958|235|3053|Sunday|1958Q3|N|N|Y|2436386|2436566|2436026|2436300|N|N|N|N|N| +2436392|AAAAAAAAICNCFCAA|1958-07-07|702|3053|235|1958|1|7|7|3|1958|235|3053|Monday|1958Q3|N|N|N|2436386|2436566|2436027|2436301|N|N|N|N|N| +2436393|AAAAAAAAJCNCFCAA|1958-07-08|702|3054|235|1958|2|7|8|3|1958|235|3054|Tuesday|1958Q3|N|N|N|2436386|2436566|2436028|2436302|N|N|N|N|N| +2436394|AAAAAAAAKCNCFCAA|1958-07-09|702|3054|235|1958|3|7|9|3|1958|235|3054|Wednesday|1958Q3|N|N|N|2436386|2436566|2436029|2436303|N|N|N|N|N| +2436395|AAAAAAAALCNCFCAA|1958-07-10|702|3054|235|1958|4|7|10|3|1958|235|3054|Thursday|1958Q3|N|N|N|2436386|2436566|2436030|2436304|N|N|N|N|N| +2436396|AAAAAAAAMCNCFCAA|1958-07-11|702|3054|235|1958|5|7|11|3|1958|235|3054|Friday|1958Q3|N|Y|N|2436386|2436566|2436031|2436305|N|N|N|N|N| +2436397|AAAAAAAANCNCFCAA|1958-07-12|702|3054|235|1958|6|7|12|3|1958|235|3054|Saturday|1958Q3|N|Y|N|2436386|2436566|2436032|2436306|N|N|N|N|N| +2436398|AAAAAAAAOCNCFCAA|1958-07-13|702|3054|235|1958|0|7|13|3|1958|235|3054|Sunday|1958Q3|N|N|N|2436386|2436566|2436033|2436307|N|N|N|N|N| +2436399|AAAAAAAAPCNCFCAA|1958-07-14|702|3054|235|1958|1|7|14|3|1958|235|3054|Monday|1958Q3|N|N|N|2436386|2436566|2436034|2436308|N|N|N|N|N| +2436400|AAAAAAAAADNCFCAA|1958-07-15|702|3055|235|1958|2|7|15|3|1958|235|3055|Tuesday|1958Q3|N|N|N|2436386|2436566|2436035|2436309|N|N|N|N|N| +2436401|AAAAAAAABDNCFCAA|1958-07-16|702|3055|235|1958|3|7|16|3|1958|235|3055|Wednesday|1958Q3|N|N|N|2436386|2436566|2436036|2436310|N|N|N|N|N| +2436402|AAAAAAAACDNCFCAA|1958-07-17|702|3055|235|1958|4|7|17|3|1958|235|3055|Thursday|1958Q3|N|N|N|2436386|2436566|2436037|2436311|N|N|N|N|N| +2436403|AAAAAAAADDNCFCAA|1958-07-18|702|3055|235|1958|5|7|18|3|1958|235|3055|Friday|1958Q3|N|Y|N|2436386|2436566|2436038|2436312|N|N|N|N|N| +2436404|AAAAAAAAEDNCFCAA|1958-07-19|702|3055|235|1958|6|7|19|3|1958|235|3055|Saturday|1958Q3|N|Y|N|2436386|2436566|2436039|2436313|N|N|N|N|N| +2436405|AAAAAAAAFDNCFCAA|1958-07-20|702|3055|235|1958|0|7|20|3|1958|235|3055|Sunday|1958Q3|N|N|N|2436386|2436566|2436040|2436314|N|N|N|N|N| +2436406|AAAAAAAAGDNCFCAA|1958-07-21|702|3055|235|1958|1|7|21|3|1958|235|3055|Monday|1958Q3|N|N|N|2436386|2436566|2436041|2436315|N|N|N|N|N| +2436407|AAAAAAAAHDNCFCAA|1958-07-22|702|3056|235|1958|2|7|22|3|1958|235|3056|Tuesday|1958Q3|N|N|N|2436386|2436566|2436042|2436316|N|N|N|N|N| +2436408|AAAAAAAAIDNCFCAA|1958-07-23|702|3056|235|1958|3|7|23|3|1958|235|3056|Wednesday|1958Q3|N|N|N|2436386|2436566|2436043|2436317|N|N|N|N|N| +2436409|AAAAAAAAJDNCFCAA|1958-07-24|702|3056|235|1958|4|7|24|3|1958|235|3056|Thursday|1958Q3|N|N|N|2436386|2436566|2436044|2436318|N|N|N|N|N| +2436410|AAAAAAAAKDNCFCAA|1958-07-25|702|3056|235|1958|5|7|25|3|1958|235|3056|Friday|1958Q3|N|Y|N|2436386|2436566|2436045|2436319|N|N|N|N|N| +2436411|AAAAAAAALDNCFCAA|1958-07-26|702|3056|235|1958|6|7|26|3|1958|235|3056|Saturday|1958Q3|N|Y|N|2436386|2436566|2436046|2436320|N|N|N|N|N| +2436412|AAAAAAAAMDNCFCAA|1958-07-27|702|3056|235|1958|0|7|27|3|1958|235|3056|Sunday|1958Q3|N|N|N|2436386|2436566|2436047|2436321|N|N|N|N|N| +2436413|AAAAAAAANDNCFCAA|1958-07-28|702|3056|235|1958|1|7|28|3|1958|235|3056|Monday|1958Q3|N|N|N|2436386|2436566|2436048|2436322|N|N|N|N|N| +2436414|AAAAAAAAODNCFCAA|1958-07-29|702|3057|235|1958|2|7|29|3|1958|235|3057|Tuesday|1958Q3|N|N|N|2436386|2436566|2436049|2436323|N|N|N|N|N| +2436415|AAAAAAAAPDNCFCAA|1958-07-30|702|3057|235|1958|3|7|30|3|1958|235|3057|Wednesday|1958Q3|N|N|N|2436386|2436566|2436050|2436324|N|N|N|N|N| +2436416|AAAAAAAAAENCFCAA|1958-07-31|702|3057|235|1958|4|7|31|3|1958|235|3057|Thursday|1958Q3|N|N|N|2436386|2436566|2436051|2436325|N|N|N|N|N| +2436417|AAAAAAAABENCFCAA|1958-08-01|703|3057|235|1958|5|8|1|3|1958|235|3057|Friday|1958Q3|N|Y|N|2436417|2436628|2436052|2436326|N|N|N|N|N| +2436418|AAAAAAAACENCFCAA|1958-08-02|703|3057|235|1958|6|8|2|3|1958|235|3057|Saturday|1958Q3|N|Y|N|2436417|2436628|2436053|2436327|N|N|N|N|N| +2436419|AAAAAAAADENCFCAA|1958-08-03|703|3057|235|1958|0|8|3|3|1958|235|3057|Sunday|1958Q3|N|N|N|2436417|2436628|2436054|2436328|N|N|N|N|N| +2436420|AAAAAAAAEENCFCAA|1958-08-04|703|3057|235|1958|1|8|4|3|1958|235|3057|Monday|1958Q3|N|N|N|2436417|2436628|2436055|2436329|N|N|N|N|N| +2436421|AAAAAAAAFENCFCAA|1958-08-05|703|3058|235|1958|2|8|5|3|1958|235|3058|Tuesday|1958Q3|N|N|N|2436417|2436628|2436056|2436330|N|N|N|N|N| +2436422|AAAAAAAAGENCFCAA|1958-08-06|703|3058|235|1958|3|8|6|3|1958|235|3058|Wednesday|1958Q3|N|N|N|2436417|2436628|2436057|2436331|N|N|N|N|N| +2436423|AAAAAAAAHENCFCAA|1958-08-07|703|3058|235|1958|4|8|7|3|1958|235|3058|Thursday|1958Q3|N|N|N|2436417|2436628|2436058|2436332|N|N|N|N|N| +2436424|AAAAAAAAIENCFCAA|1958-08-08|703|3058|235|1958|5|8|8|3|1958|235|3058|Friday|1958Q3|N|Y|N|2436417|2436628|2436059|2436333|N|N|N|N|N| +2436425|AAAAAAAAJENCFCAA|1958-08-09|703|3058|235|1958|6|8|9|3|1958|235|3058|Saturday|1958Q3|N|Y|N|2436417|2436628|2436060|2436334|N|N|N|N|N| +2436426|AAAAAAAAKENCFCAA|1958-08-10|703|3058|235|1958|0|8|10|3|1958|235|3058|Sunday|1958Q3|N|N|N|2436417|2436628|2436061|2436335|N|N|N|N|N| +2436427|AAAAAAAALENCFCAA|1958-08-11|703|3058|235|1958|1|8|11|3|1958|235|3058|Monday|1958Q3|N|N|N|2436417|2436628|2436062|2436336|N|N|N|N|N| +2436428|AAAAAAAAMENCFCAA|1958-08-12|703|3059|235|1958|2|8|12|3|1958|235|3059|Tuesday|1958Q3|N|N|N|2436417|2436628|2436063|2436337|N|N|N|N|N| +2436429|AAAAAAAANENCFCAA|1958-08-13|703|3059|235|1958|3|8|13|3|1958|235|3059|Wednesday|1958Q3|N|N|N|2436417|2436628|2436064|2436338|N|N|N|N|N| +2436430|AAAAAAAAOENCFCAA|1958-08-14|703|3059|235|1958|4|8|14|3|1958|235|3059|Thursday|1958Q3|N|N|N|2436417|2436628|2436065|2436339|N|N|N|N|N| +2436431|AAAAAAAAPENCFCAA|1958-08-15|703|3059|235|1958|5|8|15|3|1958|235|3059|Friday|1958Q3|N|Y|N|2436417|2436628|2436066|2436340|N|N|N|N|N| +2436432|AAAAAAAAAFNCFCAA|1958-08-16|703|3059|235|1958|6|8|16|3|1958|235|3059|Saturday|1958Q3|N|Y|N|2436417|2436628|2436067|2436341|N|N|N|N|N| +2436433|AAAAAAAABFNCFCAA|1958-08-17|703|3059|235|1958|0|8|17|3|1958|235|3059|Sunday|1958Q3|N|N|N|2436417|2436628|2436068|2436342|N|N|N|N|N| +2436434|AAAAAAAACFNCFCAA|1958-08-18|703|3059|235|1958|1|8|18|3|1958|235|3059|Monday|1958Q3|N|N|N|2436417|2436628|2436069|2436343|N|N|N|N|N| +2436435|AAAAAAAADFNCFCAA|1958-08-19|703|3060|235|1958|2|8|19|3|1958|235|3060|Tuesday|1958Q3|N|N|N|2436417|2436628|2436070|2436344|N|N|N|N|N| +2436436|AAAAAAAAEFNCFCAA|1958-08-20|703|3060|235|1958|3|8|20|3|1958|235|3060|Wednesday|1958Q3|N|N|N|2436417|2436628|2436071|2436345|N|N|N|N|N| +2436437|AAAAAAAAFFNCFCAA|1958-08-21|703|3060|235|1958|4|8|21|3|1958|235|3060|Thursday|1958Q3|N|N|N|2436417|2436628|2436072|2436346|N|N|N|N|N| +2436438|AAAAAAAAGFNCFCAA|1958-08-22|703|3060|235|1958|5|8|22|3|1958|235|3060|Friday|1958Q3|N|Y|N|2436417|2436628|2436073|2436347|N|N|N|N|N| +2436439|AAAAAAAAHFNCFCAA|1958-08-23|703|3060|235|1958|6|8|23|3|1958|235|3060|Saturday|1958Q3|N|Y|N|2436417|2436628|2436074|2436348|N|N|N|N|N| +2436440|AAAAAAAAIFNCFCAA|1958-08-24|703|3060|235|1958|0|8|24|3|1958|235|3060|Sunday|1958Q3|N|N|N|2436417|2436628|2436075|2436349|N|N|N|N|N| +2436441|AAAAAAAAJFNCFCAA|1958-08-25|703|3060|235|1958|1|8|25|3|1958|235|3060|Monday|1958Q3|N|N|N|2436417|2436628|2436076|2436350|N|N|N|N|N| +2436442|AAAAAAAAKFNCFCAA|1958-08-26|703|3061|235|1958|2|8|26|3|1958|235|3061|Tuesday|1958Q3|N|N|N|2436417|2436628|2436077|2436351|N|N|N|N|N| +2436443|AAAAAAAALFNCFCAA|1958-08-27|703|3061|235|1958|3|8|27|3|1958|235|3061|Wednesday|1958Q3|N|N|N|2436417|2436628|2436078|2436352|N|N|N|N|N| +2436444|AAAAAAAAMFNCFCAA|1958-08-28|703|3061|235|1958|4|8|28|3|1958|235|3061|Thursday|1958Q3|N|N|N|2436417|2436628|2436079|2436353|N|N|N|N|N| +2436445|AAAAAAAANFNCFCAA|1958-08-29|703|3061|235|1958|5|8|29|3|1958|235|3061|Friday|1958Q3|N|Y|N|2436417|2436628|2436080|2436354|N|N|N|N|N| +2436446|AAAAAAAAOFNCFCAA|1958-08-30|703|3061|235|1958|6|8|30|3|1958|235|3061|Saturday|1958Q3|N|Y|N|2436417|2436628|2436081|2436355|N|N|N|N|N| +2436447|AAAAAAAAPFNCFCAA|1958-08-31|703|3061|235|1958|0|8|31|3|1958|235|3061|Sunday|1958Q3|N|N|N|2436417|2436628|2436082|2436356|N|N|N|N|N| +2436448|AAAAAAAAAGNCFCAA|1958-09-01|704|3061|236|1958|1|9|1|3|1958|236|3061|Monday|1958Q3|N|N|N|2436448|2436690|2436083|2436357|N|N|N|N|N| +2436449|AAAAAAAABGNCFCAA|1958-09-02|704|3062|236|1958|2|9|2|3|1958|236|3062|Tuesday|1958Q3|N|N|N|2436448|2436690|2436084|2436358|N|N|N|N|N| +2436450|AAAAAAAACGNCFCAA|1958-09-03|704|3062|236|1958|3|9|3|3|1958|236|3062|Wednesday|1958Q3|N|N|N|2436448|2436690|2436085|2436359|N|N|N|N|N| +2436451|AAAAAAAADGNCFCAA|1958-09-04|704|3062|236|1958|4|9|4|3|1958|236|3062|Thursday|1958Q3|N|N|N|2436448|2436690|2436086|2436360|N|N|N|N|N| +2436452|AAAAAAAAEGNCFCAA|1958-09-05|704|3062|236|1958|5|9|5|3|1958|236|3062|Friday|1958Q3|N|Y|N|2436448|2436690|2436087|2436361|N|N|N|N|N| +2436453|AAAAAAAAFGNCFCAA|1958-09-06|704|3062|236|1958|6|9|6|3|1958|236|3062|Saturday|1958Q3|N|Y|N|2436448|2436690|2436088|2436362|N|N|N|N|N| +2436454|AAAAAAAAGGNCFCAA|1958-09-07|704|3062|236|1958|0|9|7|3|1958|236|3062|Sunday|1958Q3|N|N|N|2436448|2436690|2436089|2436363|N|N|N|N|N| +2436455|AAAAAAAAHGNCFCAA|1958-09-08|704|3062|236|1958|1|9|8|3|1958|236|3062|Monday|1958Q3|N|N|N|2436448|2436690|2436090|2436364|N|N|N|N|N| +2436456|AAAAAAAAIGNCFCAA|1958-09-09|704|3063|236|1958|2|9|9|3|1958|236|3063|Tuesday|1958Q3|N|N|N|2436448|2436690|2436091|2436365|N|N|N|N|N| +2436457|AAAAAAAAJGNCFCAA|1958-09-10|704|3063|236|1958|3|9|10|3|1958|236|3063|Wednesday|1958Q3|N|N|N|2436448|2436690|2436092|2436366|N|N|N|N|N| +2436458|AAAAAAAAKGNCFCAA|1958-09-11|704|3063|236|1958|4|9|11|3|1958|236|3063|Thursday|1958Q3|N|N|N|2436448|2436690|2436093|2436367|N|N|N|N|N| +2436459|AAAAAAAALGNCFCAA|1958-09-12|704|3063|236|1958|5|9|12|3|1958|236|3063|Friday|1958Q3|N|Y|N|2436448|2436690|2436094|2436368|N|N|N|N|N| +2436460|AAAAAAAAMGNCFCAA|1958-09-13|704|3063|236|1958|6|9|13|3|1958|236|3063|Saturday|1958Q3|N|Y|N|2436448|2436690|2436095|2436369|N|N|N|N|N| +2436461|AAAAAAAANGNCFCAA|1958-09-14|704|3063|236|1958|0|9|14|3|1958|236|3063|Sunday|1958Q3|N|N|N|2436448|2436690|2436096|2436370|N|N|N|N|N| +2436462|AAAAAAAAOGNCFCAA|1958-09-15|704|3063|236|1958|1|9|15|3|1958|236|3063|Monday|1958Q3|N|N|N|2436448|2436690|2436097|2436371|N|N|N|N|N| +2436463|AAAAAAAAPGNCFCAA|1958-09-16|704|3064|236|1958|2|9|16|3|1958|236|3064|Tuesday|1958Q3|N|N|N|2436448|2436690|2436098|2436372|N|N|N|N|N| +2436464|AAAAAAAAAHNCFCAA|1958-09-17|704|3064|236|1958|3|9|17|3|1958|236|3064|Wednesday|1958Q3|N|N|N|2436448|2436690|2436099|2436373|N|N|N|N|N| +2436465|AAAAAAAABHNCFCAA|1958-09-18|704|3064|236|1958|4|9|18|3|1958|236|3064|Thursday|1958Q3|N|N|N|2436448|2436690|2436100|2436374|N|N|N|N|N| +2436466|AAAAAAAACHNCFCAA|1958-09-19|704|3064|236|1958|5|9|19|3|1958|236|3064|Friday|1958Q3|N|Y|N|2436448|2436690|2436101|2436375|N|N|N|N|N| +2436467|AAAAAAAADHNCFCAA|1958-09-20|704|3064|236|1958|6|9|20|3|1958|236|3064|Saturday|1958Q3|N|Y|N|2436448|2436690|2436102|2436376|N|N|N|N|N| +2436468|AAAAAAAAEHNCFCAA|1958-09-21|704|3064|236|1958|0|9|21|3|1958|236|3064|Sunday|1958Q3|N|N|N|2436448|2436690|2436103|2436377|N|N|N|N|N| +2436469|AAAAAAAAFHNCFCAA|1958-09-22|704|3064|236|1958|1|9|22|3|1958|236|3064|Monday|1958Q3|N|N|N|2436448|2436690|2436104|2436378|N|N|N|N|N| +2436470|AAAAAAAAGHNCFCAA|1958-09-23|704|3065|236|1958|2|9|23|3|1958|236|3065|Tuesday|1958Q3|N|N|N|2436448|2436690|2436105|2436379|N|N|N|N|N| +2436471|AAAAAAAAHHNCFCAA|1958-09-24|704|3065|236|1958|3|9|24|3|1958|236|3065|Wednesday|1958Q3|N|N|N|2436448|2436690|2436106|2436380|N|N|N|N|N| +2436472|AAAAAAAAIHNCFCAA|1958-09-25|704|3065|236|1958|4|9|25|3|1958|236|3065|Thursday|1958Q3|N|N|N|2436448|2436690|2436107|2436381|N|N|N|N|N| +2436473|AAAAAAAAJHNCFCAA|1958-09-26|704|3065|236|1958|5|9|26|3|1958|236|3065|Friday|1958Q3|N|Y|N|2436448|2436690|2436108|2436382|N|N|N|N|N| +2436474|AAAAAAAAKHNCFCAA|1958-09-27|704|3065|236|1958|6|9|27|3|1958|236|3065|Saturday|1958Q3|N|Y|N|2436448|2436690|2436109|2436383|N|N|N|N|N| +2436475|AAAAAAAALHNCFCAA|1958-09-28|704|3065|236|1958|0|9|28|3|1958|236|3065|Sunday|1958Q3|N|N|N|2436448|2436690|2436110|2436384|N|N|N|N|N| +2436476|AAAAAAAAMHNCFCAA|1958-09-29|704|3065|236|1958|1|9|29|3|1958|236|3065|Monday|1958Q3|N|N|N|2436448|2436690|2436111|2436385|N|N|N|N|N| +2436477|AAAAAAAANHNCFCAA|1958-09-30|704|3066|236|1958|2|9|30|3|1958|236|3066|Tuesday|1958Q3|N|N|N|2436448|2436690|2436112|2436386|N|N|N|N|N| +2436478|AAAAAAAAOHNCFCAA|1958-10-01|705|3066|236|1958|3|10|1|3|1958|236|3066|Wednesday|1958Q3|N|N|N|2436478|2436750|2436113|2436386|N|N|N|N|N| +2436479|AAAAAAAAPHNCFCAA|1958-10-02|705|3066|236|1958|4|10|2|4|1958|236|3066|Thursday|1958Q4|N|N|N|2436478|2436750|2436114|2436387|N|N|N|N|N| +2436480|AAAAAAAAAINCFCAA|1958-10-03|705|3066|236|1958|5|10|3|4|1958|236|3066|Friday|1958Q4|N|Y|N|2436478|2436750|2436115|2436388|N|N|N|N|N| +2436481|AAAAAAAABINCFCAA|1958-10-04|705|3066|236|1958|6|10|4|4|1958|236|3066|Saturday|1958Q4|N|Y|N|2436478|2436750|2436116|2436389|N|N|N|N|N| +2436482|AAAAAAAACINCFCAA|1958-10-05|705|3066|236|1958|0|10|5|4|1958|236|3066|Sunday|1958Q4|N|N|N|2436478|2436750|2436117|2436390|N|N|N|N|N| +2436483|AAAAAAAADINCFCAA|1958-10-06|705|3066|236|1958|1|10|6|4|1958|236|3066|Monday|1958Q4|N|N|N|2436478|2436750|2436118|2436391|N|N|N|N|N| +2436484|AAAAAAAAEINCFCAA|1958-10-07|705|3067|236|1958|2|10|7|4|1958|236|3067|Tuesday|1958Q4|N|N|N|2436478|2436750|2436119|2436392|N|N|N|N|N| +2436485|AAAAAAAAFINCFCAA|1958-10-08|705|3067|236|1958|3|10|8|4|1958|236|3067|Wednesday|1958Q4|N|N|N|2436478|2436750|2436120|2436393|N|N|N|N|N| +2436486|AAAAAAAAGINCFCAA|1958-10-09|705|3067|236|1958|4|10|9|4|1958|236|3067|Thursday|1958Q4|N|N|N|2436478|2436750|2436121|2436394|N|N|N|N|N| +2436487|AAAAAAAAHINCFCAA|1958-10-10|705|3067|236|1958|5|10|10|4|1958|236|3067|Friday|1958Q4|N|Y|N|2436478|2436750|2436122|2436395|N|N|N|N|N| +2436488|AAAAAAAAIINCFCAA|1958-10-11|705|3067|236|1958|6|10|11|4|1958|236|3067|Saturday|1958Q4|N|Y|N|2436478|2436750|2436123|2436396|N|N|N|N|N| +2436489|AAAAAAAAJINCFCAA|1958-10-12|705|3067|236|1958|0|10|12|4|1958|236|3067|Sunday|1958Q4|N|N|N|2436478|2436750|2436124|2436397|N|N|N|N|N| +2436490|AAAAAAAAKINCFCAA|1958-10-13|705|3067|236|1958|1|10|13|4|1958|236|3067|Monday|1958Q4|N|N|N|2436478|2436750|2436125|2436398|N|N|N|N|N| +2436491|AAAAAAAALINCFCAA|1958-10-14|705|3068|236|1958|2|10|14|4|1958|236|3068|Tuesday|1958Q4|N|N|N|2436478|2436750|2436126|2436399|N|N|N|N|N| +2436492|AAAAAAAAMINCFCAA|1958-10-15|705|3068|236|1958|3|10|15|4|1958|236|3068|Wednesday|1958Q4|N|N|N|2436478|2436750|2436127|2436400|N|N|N|N|N| +2436493|AAAAAAAANINCFCAA|1958-10-16|705|3068|236|1958|4|10|16|4|1958|236|3068|Thursday|1958Q4|N|N|N|2436478|2436750|2436128|2436401|N|N|N|N|N| +2436494|AAAAAAAAOINCFCAA|1958-10-17|705|3068|236|1958|5|10|17|4|1958|236|3068|Friday|1958Q4|N|Y|N|2436478|2436750|2436129|2436402|N|N|N|N|N| +2436495|AAAAAAAAPINCFCAA|1958-10-18|705|3068|236|1958|6|10|18|4|1958|236|3068|Saturday|1958Q4|N|Y|N|2436478|2436750|2436130|2436403|N|N|N|N|N| +2436496|AAAAAAAAAJNCFCAA|1958-10-19|705|3068|236|1958|0|10|19|4|1958|236|3068|Sunday|1958Q4|N|N|N|2436478|2436750|2436131|2436404|N|N|N|N|N| +2436497|AAAAAAAABJNCFCAA|1958-10-20|705|3068|236|1958|1|10|20|4|1958|236|3068|Monday|1958Q4|N|N|N|2436478|2436750|2436132|2436405|N|N|N|N|N| +2436498|AAAAAAAACJNCFCAA|1958-10-21|705|3069|236|1958|2|10|21|4|1958|236|3069|Tuesday|1958Q4|N|N|N|2436478|2436750|2436133|2436406|N|N|N|N|N| +2436499|AAAAAAAADJNCFCAA|1958-10-22|705|3069|236|1958|3|10|22|4|1958|236|3069|Wednesday|1958Q4|N|N|N|2436478|2436750|2436134|2436407|N|N|N|N|N| +2436500|AAAAAAAAEJNCFCAA|1958-10-23|705|3069|236|1958|4|10|23|4|1958|236|3069|Thursday|1958Q4|N|N|N|2436478|2436750|2436135|2436408|N|N|N|N|N| +2436501|AAAAAAAAFJNCFCAA|1958-10-24|705|3069|236|1958|5|10|24|4|1958|236|3069|Friday|1958Q4|N|Y|N|2436478|2436750|2436136|2436409|N|N|N|N|N| +2436502|AAAAAAAAGJNCFCAA|1958-10-25|705|3069|236|1958|6|10|25|4|1958|236|3069|Saturday|1958Q4|N|Y|N|2436478|2436750|2436137|2436410|N|N|N|N|N| +2436503|AAAAAAAAHJNCFCAA|1958-10-26|705|3069|236|1958|0|10|26|4|1958|236|3069|Sunday|1958Q4|N|N|N|2436478|2436750|2436138|2436411|N|N|N|N|N| +2436504|AAAAAAAAIJNCFCAA|1958-10-27|705|3069|236|1958|1|10|27|4|1958|236|3069|Monday|1958Q4|N|N|N|2436478|2436750|2436139|2436412|N|N|N|N|N| +2436505|AAAAAAAAJJNCFCAA|1958-10-28|705|3070|236|1958|2|10|28|4|1958|236|3070|Tuesday|1958Q4|N|N|N|2436478|2436750|2436140|2436413|N|N|N|N|N| +2436506|AAAAAAAAKJNCFCAA|1958-10-29|705|3070|236|1958|3|10|29|4|1958|236|3070|Wednesday|1958Q4|N|N|N|2436478|2436750|2436141|2436414|N|N|N|N|N| +2436507|AAAAAAAALJNCFCAA|1958-10-30|705|3070|236|1958|4|10|30|4|1958|236|3070|Thursday|1958Q4|N|N|N|2436478|2436750|2436142|2436415|N|N|N|N|N| +2436508|AAAAAAAAMJNCFCAA|1958-10-31|705|3070|236|1958|5|10|31|4|1958|236|3070|Friday|1958Q4|N|Y|N|2436478|2436750|2436143|2436416|N|N|N|N|N| +2436509|AAAAAAAANJNCFCAA|1958-11-01|706|3070|236|1958|6|11|1|4|1958|236|3070|Saturday|1958Q4|N|Y|N|2436509|2436812|2436144|2436417|N|N|N|N|N| +2436510|AAAAAAAAOJNCFCAA|1958-11-02|706|3070|236|1958|0|11|2|4|1958|236|3070|Sunday|1958Q4|N|N|N|2436509|2436812|2436145|2436418|N|N|N|N|N| +2436511|AAAAAAAAPJNCFCAA|1958-11-03|706|3070|236|1958|1|11|3|4|1958|236|3070|Monday|1958Q4|N|N|N|2436509|2436812|2436146|2436419|N|N|N|N|N| +2436512|AAAAAAAAAKNCFCAA|1958-11-04|706|3071|236|1958|2|11|4|4|1958|236|3071|Tuesday|1958Q4|N|N|N|2436509|2436812|2436147|2436420|N|N|N|N|N| +2436513|AAAAAAAABKNCFCAA|1958-11-05|706|3071|236|1958|3|11|5|4|1958|236|3071|Wednesday|1958Q4|N|N|N|2436509|2436812|2436148|2436421|N|N|N|N|N| +2436514|AAAAAAAACKNCFCAA|1958-11-06|706|3071|236|1958|4|11|6|4|1958|236|3071|Thursday|1958Q4|N|N|N|2436509|2436812|2436149|2436422|N|N|N|N|N| +2436515|AAAAAAAADKNCFCAA|1958-11-07|706|3071|236|1958|5|11|7|4|1958|236|3071|Friday|1958Q4|N|Y|N|2436509|2436812|2436150|2436423|N|N|N|N|N| +2436516|AAAAAAAAEKNCFCAA|1958-11-08|706|3071|236|1958|6|11|8|4|1958|236|3071|Saturday|1958Q4|N|Y|N|2436509|2436812|2436151|2436424|N|N|N|N|N| +2436517|AAAAAAAAFKNCFCAA|1958-11-09|706|3071|236|1958|0|11|9|4|1958|236|3071|Sunday|1958Q4|N|N|N|2436509|2436812|2436152|2436425|N|N|N|N|N| +2436518|AAAAAAAAGKNCFCAA|1958-11-10|706|3071|236|1958|1|11|10|4|1958|236|3071|Monday|1958Q4|N|N|N|2436509|2436812|2436153|2436426|N|N|N|N|N| +2436519|AAAAAAAAHKNCFCAA|1958-11-11|706|3072|236|1958|2|11|11|4|1958|236|3072|Tuesday|1958Q4|N|N|N|2436509|2436812|2436154|2436427|N|N|N|N|N| +2436520|AAAAAAAAIKNCFCAA|1958-11-12|706|3072|236|1958|3|11|12|4|1958|236|3072|Wednesday|1958Q4|N|N|N|2436509|2436812|2436155|2436428|N|N|N|N|N| +2436521|AAAAAAAAJKNCFCAA|1958-11-13|706|3072|236|1958|4|11|13|4|1958|236|3072|Thursday|1958Q4|N|N|N|2436509|2436812|2436156|2436429|N|N|N|N|N| +2436522|AAAAAAAAKKNCFCAA|1958-11-14|706|3072|236|1958|5|11|14|4|1958|236|3072|Friday|1958Q4|N|Y|N|2436509|2436812|2436157|2436430|N|N|N|N|N| +2436523|AAAAAAAALKNCFCAA|1958-11-15|706|3072|236|1958|6|11|15|4|1958|236|3072|Saturday|1958Q4|N|Y|N|2436509|2436812|2436158|2436431|N|N|N|N|N| +2436524|AAAAAAAAMKNCFCAA|1958-11-16|706|3072|236|1958|0|11|16|4|1958|236|3072|Sunday|1958Q4|N|N|N|2436509|2436812|2436159|2436432|N|N|N|N|N| +2436525|AAAAAAAANKNCFCAA|1958-11-17|706|3072|236|1958|1|11|17|4|1958|236|3072|Monday|1958Q4|N|N|N|2436509|2436812|2436160|2436433|N|N|N|N|N| +2436526|AAAAAAAAOKNCFCAA|1958-11-18|706|3073|236|1958|2|11|18|4|1958|236|3073|Tuesday|1958Q4|N|N|N|2436509|2436812|2436161|2436434|N|N|N|N|N| +2436527|AAAAAAAAPKNCFCAA|1958-11-19|706|3073|236|1958|3|11|19|4|1958|236|3073|Wednesday|1958Q4|N|N|N|2436509|2436812|2436162|2436435|N|N|N|N|N| +2436528|AAAAAAAAALNCFCAA|1958-11-20|706|3073|236|1958|4|11|20|4|1958|236|3073|Thursday|1958Q4|N|N|N|2436509|2436812|2436163|2436436|N|N|N|N|N| +2436529|AAAAAAAABLNCFCAA|1958-11-21|706|3073|236|1958|5|11|21|4|1958|236|3073|Friday|1958Q4|N|Y|N|2436509|2436812|2436164|2436437|N|N|N|N|N| +2436530|AAAAAAAACLNCFCAA|1958-11-22|706|3073|236|1958|6|11|22|4|1958|236|3073|Saturday|1958Q4|N|Y|N|2436509|2436812|2436165|2436438|N|N|N|N|N| +2436531|AAAAAAAADLNCFCAA|1958-11-23|706|3073|236|1958|0|11|23|4|1958|236|3073|Sunday|1958Q4|N|N|N|2436509|2436812|2436166|2436439|N|N|N|N|N| +2436532|AAAAAAAAELNCFCAA|1958-11-24|706|3073|236|1958|1|11|24|4|1958|236|3073|Monday|1958Q4|N|N|N|2436509|2436812|2436167|2436440|N|N|N|N|N| +2436533|AAAAAAAAFLNCFCAA|1958-11-25|706|3074|236|1958|2|11|25|4|1958|236|3074|Tuesday|1958Q4|N|N|N|2436509|2436812|2436168|2436441|N|N|N|N|N| +2436534|AAAAAAAAGLNCFCAA|1958-11-26|706|3074|236|1958|3|11|26|4|1958|236|3074|Wednesday|1958Q4|N|N|N|2436509|2436812|2436169|2436442|N|N|N|N|N| +2436535|AAAAAAAAHLNCFCAA|1958-11-27|706|3074|236|1958|4|11|27|4|1958|236|3074|Thursday|1958Q4|N|N|N|2436509|2436812|2436170|2436443|N|N|N|N|N| +2436536|AAAAAAAAILNCFCAA|1958-11-28|706|3074|236|1958|5|11|28|4|1958|236|3074|Friday|1958Q4|N|Y|N|2436509|2436812|2436171|2436444|N|N|N|N|N| +2436537|AAAAAAAAJLNCFCAA|1958-11-29|706|3074|236|1958|6|11|29|4|1958|236|3074|Saturday|1958Q4|N|Y|N|2436509|2436812|2436172|2436445|N|N|N|N|N| +2436538|AAAAAAAAKLNCFCAA|1958-11-30|706|3074|236|1958|0|11|30|4|1958|236|3074|Sunday|1958Q4|N|N|N|2436509|2436812|2436173|2436446|N|N|N|N|N| +2436539|AAAAAAAALLNCFCAA|1958-12-01|707|3074|237|1958|1|12|1|4|1958|237|3074|Monday|1958Q4|N|N|N|2436539|2436872|2436174|2436447|N|N|N|N|N| +2436540|AAAAAAAAMLNCFCAA|1958-12-02|707|3075|237|1958|2|12|2|4|1958|237|3075|Tuesday|1958Q4|N|N|N|2436539|2436872|2436175|2436448|N|N|N|N|N| +2436541|AAAAAAAANLNCFCAA|1958-12-03|707|3075|237|1958|3|12|3|4|1958|237|3075|Wednesday|1958Q4|N|N|N|2436539|2436872|2436176|2436449|N|N|N|N|N| +2436542|AAAAAAAAOLNCFCAA|1958-12-04|707|3075|237|1958|4|12|4|4|1958|237|3075|Thursday|1958Q4|N|N|N|2436539|2436872|2436177|2436450|N|N|N|N|N| +2436543|AAAAAAAAPLNCFCAA|1958-12-05|707|3075|237|1958|5|12|5|4|1958|237|3075|Friday|1958Q4|N|Y|N|2436539|2436872|2436178|2436451|N|N|N|N|N| +2436544|AAAAAAAAAMNCFCAA|1958-12-06|707|3075|237|1958|6|12|6|4|1958|237|3075|Saturday|1958Q4|N|Y|N|2436539|2436872|2436179|2436452|N|N|N|N|N| +2436545|AAAAAAAABMNCFCAA|1958-12-07|707|3075|237|1958|0|12|7|4|1958|237|3075|Sunday|1958Q4|N|N|N|2436539|2436872|2436180|2436453|N|N|N|N|N| +2436546|AAAAAAAACMNCFCAA|1958-12-08|707|3075|237|1958|1|12|8|4|1958|237|3075|Monday|1958Q4|N|N|N|2436539|2436872|2436181|2436454|N|N|N|N|N| +2436547|AAAAAAAADMNCFCAA|1958-12-09|707|3076|237|1958|2|12|9|4|1958|237|3076|Tuesday|1958Q4|N|N|N|2436539|2436872|2436182|2436455|N|N|N|N|N| +2436548|AAAAAAAAEMNCFCAA|1958-12-10|707|3076|237|1958|3|12|10|4|1958|237|3076|Wednesday|1958Q4|N|N|N|2436539|2436872|2436183|2436456|N|N|N|N|N| +2436549|AAAAAAAAFMNCFCAA|1958-12-11|707|3076|237|1958|4|12|11|4|1958|237|3076|Thursday|1958Q4|N|N|N|2436539|2436872|2436184|2436457|N|N|N|N|N| +2436550|AAAAAAAAGMNCFCAA|1958-12-12|707|3076|237|1958|5|12|12|4|1958|237|3076|Friday|1958Q4|N|Y|N|2436539|2436872|2436185|2436458|N|N|N|N|N| +2436551|AAAAAAAAHMNCFCAA|1958-12-13|707|3076|237|1958|6|12|13|4|1958|237|3076|Saturday|1958Q4|N|Y|N|2436539|2436872|2436186|2436459|N|N|N|N|N| +2436552|AAAAAAAAIMNCFCAA|1958-12-14|707|3076|237|1958|0|12|14|4|1958|237|3076|Sunday|1958Q4|N|N|N|2436539|2436872|2436187|2436460|N|N|N|N|N| +2436553|AAAAAAAAJMNCFCAA|1958-12-15|707|3076|237|1958|1|12|15|4|1958|237|3076|Monday|1958Q4|N|N|N|2436539|2436872|2436188|2436461|N|N|N|N|N| +2436554|AAAAAAAAKMNCFCAA|1958-12-16|707|3077|237|1958|2|12|16|4|1958|237|3077|Tuesday|1958Q4|N|N|N|2436539|2436872|2436189|2436462|N|N|N|N|N| +2436555|AAAAAAAALMNCFCAA|1958-12-17|707|3077|237|1958|3|12|17|4|1958|237|3077|Wednesday|1958Q4|N|N|N|2436539|2436872|2436190|2436463|N|N|N|N|N| +2436556|AAAAAAAAMMNCFCAA|1958-12-18|707|3077|237|1958|4|12|18|4|1958|237|3077|Thursday|1958Q4|N|N|N|2436539|2436872|2436191|2436464|N|N|N|N|N| +2436557|AAAAAAAANMNCFCAA|1958-12-19|707|3077|237|1958|5|12|19|4|1958|237|3077|Friday|1958Q4|N|Y|N|2436539|2436872|2436192|2436465|N|N|N|N|N| +2436558|AAAAAAAAOMNCFCAA|1958-12-20|707|3077|237|1958|6|12|20|4|1958|237|3077|Saturday|1958Q4|N|Y|N|2436539|2436872|2436193|2436466|N|N|N|N|N| +2436559|AAAAAAAAPMNCFCAA|1958-12-21|707|3077|237|1958|0|12|21|4|1958|237|3077|Sunday|1958Q4|N|N|N|2436539|2436872|2436194|2436467|N|N|N|N|N| +2436560|AAAAAAAAANNCFCAA|1958-12-22|707|3077|237|1958|1|12|22|4|1958|237|3077|Monday|1958Q4|N|N|N|2436539|2436872|2436195|2436468|N|N|N|N|N| +2436561|AAAAAAAABNNCFCAA|1958-12-23|707|3078|237|1958|2|12|23|4|1958|237|3078|Tuesday|1958Q4|N|N|N|2436539|2436872|2436196|2436469|N|N|N|N|N| +2436562|AAAAAAAACNNCFCAA|1958-12-24|707|3078|237|1958|3|12|24|4|1958|237|3078|Wednesday|1958Q4|N|N|N|2436539|2436872|2436197|2436470|N|N|N|N|N| +2436563|AAAAAAAADNNCFCAA|1958-12-25|707|3078|237|1958|4|12|25|4|1958|237|3078|Thursday|1958Q4|N|N|N|2436539|2436872|2436198|2436471|N|N|N|N|N| +2436564|AAAAAAAAENNCFCAA|1958-12-26|707|3078|237|1958|5|12|26|4|1958|237|3078|Friday|1958Q4|Y|Y|N|2436539|2436872|2436199|2436472|N|N|N|N|N| +2436565|AAAAAAAAFNNCFCAA|1958-12-27|707|3078|237|1958|6|12|27|4|1958|237|3078|Saturday|1958Q4|N|Y|Y|2436539|2436872|2436200|2436473|N|N|N|N|N| +2436566|AAAAAAAAGNNCFCAA|1958-12-28|707|3078|237|1958|0|12|28|4|1958|237|3078|Sunday|1958Q4|N|N|N|2436539|2436872|2436201|2436474|N|N|N|N|N| +2436567|AAAAAAAAHNNCFCAA|1958-12-29|707|3078|237|1958|1|12|29|4|1958|237|3078|Monday|1958Q4|N|N|N|2436539|2436872|2436202|2436475|N|N|N|N|N| +2436568|AAAAAAAAINNCFCAA|1958-12-30|707|3079|237|1958|2|12|30|4|1958|237|3079|Tuesday|1958Q4|N|N|N|2436539|2436872|2436203|2436476|N|N|N|N|N| +2436569|AAAAAAAAJNNCFCAA|1958-12-31|707|3079|237|1958|3|12|31|4|1958|237|3079|Wednesday|1958Q4|N|N|N|2436539|2436872|2436204|2436477|N|N|N|N|N| +2436570|AAAAAAAAKNNCFCAA|1959-01-01|708|3079|237|1959|4|1|1|1|1959|237|3079|Thursday|1959Q1|Y|N|N|2436570|2436569|2436205|2436478|N|N|N|N|N| +2436571|AAAAAAAALNNCFCAA|1959-01-02|708|3079|237|1959|5|1|2|1|1959|237|3079|Friday|1959Q1|N|Y|Y|2436570|2436569|2436206|2436479|N|N|N|N|N| +2436572|AAAAAAAAMNNCFCAA|1959-01-03|708|3079|237|1959|6|1|3|1|1959|237|3079|Saturday|1959Q1|N|Y|N|2436570|2436569|2436207|2436480|N|N|N|N|N| +2436573|AAAAAAAANNNCFCAA|1959-01-04|708|3079|237|1959|0|1|4|1|1959|237|3079|Sunday|1959Q1|N|N|N|2436570|2436569|2436208|2436481|N|N|N|N|N| +2436574|AAAAAAAAONNCFCAA|1959-01-05|708|3079|237|1959|1|1|5|1|1959|237|3079|Monday|1959Q1|N|N|N|2436570|2436569|2436209|2436482|N|N|N|N|N| +2436575|AAAAAAAAPNNCFCAA|1959-01-06|708|3080|237|1959|2|1|6|1|1959|237|3080|Tuesday|1959Q1|N|N|N|2436570|2436569|2436210|2436483|N|N|N|N|N| +2436576|AAAAAAAAAONCFCAA|1959-01-07|708|3080|237|1959|3|1|7|1|1959|237|3080|Wednesday|1959Q1|N|N|N|2436570|2436569|2436211|2436484|N|N|N|N|N| +2436577|AAAAAAAABONCFCAA|1959-01-08|708|3080|237|1959|4|1|8|1|1959|237|3080|Thursday|1959Q1|N|N|N|2436570|2436569|2436212|2436485|N|N|N|N|N| +2436578|AAAAAAAACONCFCAA|1959-01-09|708|3080|237|1959|5|1|9|1|1959|237|3080|Friday|1959Q1|N|Y|N|2436570|2436569|2436213|2436486|N|N|N|N|N| +2436579|AAAAAAAADONCFCAA|1959-01-10|708|3080|237|1959|6|1|10|1|1959|237|3080|Saturday|1959Q1|N|Y|N|2436570|2436569|2436214|2436487|N|N|N|N|N| +2436580|AAAAAAAAEONCFCAA|1959-01-11|708|3080|237|1959|0|1|11|1|1959|237|3080|Sunday|1959Q1|N|N|N|2436570|2436569|2436215|2436488|N|N|N|N|N| +2436581|AAAAAAAAFONCFCAA|1959-01-12|708|3080|237|1959|1|1|12|1|1959|237|3080|Monday|1959Q1|N|N|N|2436570|2436569|2436216|2436489|N|N|N|N|N| +2436582|AAAAAAAAGONCFCAA|1959-01-13|708|3081|237|1959|2|1|13|1|1959|237|3081|Tuesday|1959Q1|N|N|N|2436570|2436569|2436217|2436490|N|N|N|N|N| +2436583|AAAAAAAAHONCFCAA|1959-01-14|708|3081|237|1959|3|1|14|1|1959|237|3081|Wednesday|1959Q1|N|N|N|2436570|2436569|2436218|2436491|N|N|N|N|N| +2436584|AAAAAAAAIONCFCAA|1959-01-15|708|3081|237|1959|4|1|15|1|1959|237|3081|Thursday|1959Q1|N|N|N|2436570|2436569|2436219|2436492|N|N|N|N|N| +2436585|AAAAAAAAJONCFCAA|1959-01-16|708|3081|237|1959|5|1|16|1|1959|237|3081|Friday|1959Q1|N|Y|N|2436570|2436569|2436220|2436493|N|N|N|N|N| +2436586|AAAAAAAAKONCFCAA|1959-01-17|708|3081|237|1959|6|1|17|1|1959|237|3081|Saturday|1959Q1|N|Y|N|2436570|2436569|2436221|2436494|N|N|N|N|N| +2436587|AAAAAAAALONCFCAA|1959-01-18|708|3081|237|1959|0|1|18|1|1959|237|3081|Sunday|1959Q1|N|N|N|2436570|2436569|2436222|2436495|N|N|N|N|N| +2436588|AAAAAAAAMONCFCAA|1959-01-19|708|3081|237|1959|1|1|19|1|1959|237|3081|Monday|1959Q1|N|N|N|2436570|2436569|2436223|2436496|N|N|N|N|N| +2436589|AAAAAAAANONCFCAA|1959-01-20|708|3082|237|1959|2|1|20|1|1959|237|3082|Tuesday|1959Q1|N|N|N|2436570|2436569|2436224|2436497|N|N|N|N|N| +2436590|AAAAAAAAOONCFCAA|1959-01-21|708|3082|237|1959|3|1|21|1|1959|237|3082|Wednesday|1959Q1|N|N|N|2436570|2436569|2436225|2436498|N|N|N|N|N| +2436591|AAAAAAAAPONCFCAA|1959-01-22|708|3082|237|1959|4|1|22|1|1959|237|3082|Thursday|1959Q1|N|N|N|2436570|2436569|2436226|2436499|N|N|N|N|N| +2436592|AAAAAAAAAPNCFCAA|1959-01-23|708|3082|237|1959|5|1|23|1|1959|237|3082|Friday|1959Q1|N|Y|N|2436570|2436569|2436227|2436500|N|N|N|N|N| +2436593|AAAAAAAABPNCFCAA|1959-01-24|708|3082|237|1959|6|1|24|1|1959|237|3082|Saturday|1959Q1|N|Y|N|2436570|2436569|2436228|2436501|N|N|N|N|N| +2436594|AAAAAAAACPNCFCAA|1959-01-25|708|3082|237|1959|0|1|25|1|1959|237|3082|Sunday|1959Q1|N|N|N|2436570|2436569|2436229|2436502|N|N|N|N|N| +2436595|AAAAAAAADPNCFCAA|1959-01-26|708|3082|237|1959|1|1|26|1|1959|237|3082|Monday|1959Q1|N|N|N|2436570|2436569|2436230|2436503|N|N|N|N|N| +2436596|AAAAAAAAEPNCFCAA|1959-01-27|708|3083|237|1959|2|1|27|1|1959|237|3083|Tuesday|1959Q1|N|N|N|2436570|2436569|2436231|2436504|N|N|N|N|N| +2436597|AAAAAAAAFPNCFCAA|1959-01-28|708|3083|237|1959|3|1|28|1|1959|237|3083|Wednesday|1959Q1|N|N|N|2436570|2436569|2436232|2436505|N|N|N|N|N| +2436598|AAAAAAAAGPNCFCAA|1959-01-29|708|3083|237|1959|4|1|29|1|1959|237|3083|Thursday|1959Q1|N|N|N|2436570|2436569|2436233|2436506|N|N|N|N|N| +2436599|AAAAAAAAHPNCFCAA|1959-01-30|708|3083|237|1959|5|1|30|1|1959|237|3083|Friday|1959Q1|N|Y|N|2436570|2436569|2436234|2436507|N|N|N|N|N| +2436600|AAAAAAAAIPNCFCAA|1959-01-31|708|3083|237|1959|6|1|31|1|1959|237|3083|Saturday|1959Q1|N|Y|N|2436570|2436569|2436235|2436508|N|N|N|N|N| +2436601|AAAAAAAAJPNCFCAA|1959-02-01|709|3083|237|1959|0|2|1|1|1959|237|3083|Sunday|1959Q1|N|N|N|2436601|2436631|2436236|2436509|N|N|N|N|N| +2436602|AAAAAAAAKPNCFCAA|1959-02-02|709|3083|237|1959|1|2|2|1|1959|237|3083|Monday|1959Q1|N|N|N|2436601|2436631|2436237|2436510|N|N|N|N|N| +2436603|AAAAAAAALPNCFCAA|1959-02-03|709|3084|237|1959|2|2|3|1|1959|237|3084|Tuesday|1959Q1|N|N|N|2436601|2436631|2436238|2436511|N|N|N|N|N| +2436604|AAAAAAAAMPNCFCAA|1959-02-04|709|3084|237|1959|3|2|4|1|1959|237|3084|Wednesday|1959Q1|N|N|N|2436601|2436631|2436239|2436512|N|N|N|N|N| +2436605|AAAAAAAANPNCFCAA|1959-02-05|709|3084|237|1959|4|2|5|1|1959|237|3084|Thursday|1959Q1|N|N|N|2436601|2436631|2436240|2436513|N|N|N|N|N| +2436606|AAAAAAAAOPNCFCAA|1959-02-06|709|3084|237|1959|5|2|6|1|1959|237|3084|Friday|1959Q1|N|Y|N|2436601|2436631|2436241|2436514|N|N|N|N|N| +2436607|AAAAAAAAPPNCFCAA|1959-02-07|709|3084|237|1959|6|2|7|1|1959|237|3084|Saturday|1959Q1|N|Y|N|2436601|2436631|2436242|2436515|N|N|N|N|N| +2436608|AAAAAAAAAAOCFCAA|1959-02-08|709|3084|237|1959|0|2|8|1|1959|237|3084|Sunday|1959Q1|N|N|N|2436601|2436631|2436243|2436516|N|N|N|N|N| +2436609|AAAAAAAABAOCFCAA|1959-02-09|709|3084|237|1959|1|2|9|1|1959|237|3084|Monday|1959Q1|N|N|N|2436601|2436631|2436244|2436517|N|N|N|N|N| +2436610|AAAAAAAACAOCFCAA|1959-02-10|709|3085|237|1959|2|2|10|1|1959|237|3085|Tuesday|1959Q1|N|N|N|2436601|2436631|2436245|2436518|N|N|N|N|N| +2436611|AAAAAAAADAOCFCAA|1959-02-11|709|3085|237|1959|3|2|11|1|1959|237|3085|Wednesday|1959Q1|N|N|N|2436601|2436631|2436246|2436519|N|N|N|N|N| +2436612|AAAAAAAAEAOCFCAA|1959-02-12|709|3085|237|1959|4|2|12|1|1959|237|3085|Thursday|1959Q1|N|N|N|2436601|2436631|2436247|2436520|N|N|N|N|N| +2436613|AAAAAAAAFAOCFCAA|1959-02-13|709|3085|237|1959|5|2|13|1|1959|237|3085|Friday|1959Q1|N|Y|N|2436601|2436631|2436248|2436521|N|N|N|N|N| +2436614|AAAAAAAAGAOCFCAA|1959-02-14|709|3085|237|1959|6|2|14|1|1959|237|3085|Saturday|1959Q1|N|Y|N|2436601|2436631|2436249|2436522|N|N|N|N|N| +2436615|AAAAAAAAHAOCFCAA|1959-02-15|709|3085|237|1959|0|2|15|1|1959|237|3085|Sunday|1959Q1|N|N|N|2436601|2436631|2436250|2436523|N|N|N|N|N| +2436616|AAAAAAAAIAOCFCAA|1959-02-16|709|3085|237|1959|1|2|16|1|1959|237|3085|Monday|1959Q1|N|N|N|2436601|2436631|2436251|2436524|N|N|N|N|N| +2436617|AAAAAAAAJAOCFCAA|1959-02-17|709|3086|237|1959|2|2|17|1|1959|237|3086|Tuesday|1959Q1|N|N|N|2436601|2436631|2436252|2436525|N|N|N|N|N| +2436618|AAAAAAAAKAOCFCAA|1959-02-18|709|3086|237|1959|3|2|18|1|1959|237|3086|Wednesday|1959Q1|N|N|N|2436601|2436631|2436253|2436526|N|N|N|N|N| +2436619|AAAAAAAALAOCFCAA|1959-02-19|709|3086|237|1959|4|2|19|1|1959|237|3086|Thursday|1959Q1|N|N|N|2436601|2436631|2436254|2436527|N|N|N|N|N| +2436620|AAAAAAAAMAOCFCAA|1959-02-20|709|3086|237|1959|5|2|20|1|1959|237|3086|Friday|1959Q1|N|Y|N|2436601|2436631|2436255|2436528|N|N|N|N|N| +2436621|AAAAAAAANAOCFCAA|1959-02-21|709|3086|237|1959|6|2|21|1|1959|237|3086|Saturday|1959Q1|N|Y|N|2436601|2436631|2436256|2436529|N|N|N|N|N| +2436622|AAAAAAAAOAOCFCAA|1959-02-22|709|3086|237|1959|0|2|22|1|1959|237|3086|Sunday|1959Q1|N|N|N|2436601|2436631|2436257|2436530|N|N|N|N|N| +2436623|AAAAAAAAPAOCFCAA|1959-02-23|709|3086|237|1959|1|2|23|1|1959|237|3086|Monday|1959Q1|N|N|N|2436601|2436631|2436258|2436531|N|N|N|N|N| +2436624|AAAAAAAAABOCFCAA|1959-02-24|709|3087|237|1959|2|2|24|1|1959|237|3087|Tuesday|1959Q1|N|N|N|2436601|2436631|2436259|2436532|N|N|N|N|N| +2436625|AAAAAAAABBOCFCAA|1959-02-25|709|3087|237|1959|3|2|25|1|1959|237|3087|Wednesday|1959Q1|N|N|N|2436601|2436631|2436260|2436533|N|N|N|N|N| +2436626|AAAAAAAACBOCFCAA|1959-02-26|709|3087|237|1959|4|2|26|1|1959|237|3087|Thursday|1959Q1|N|N|N|2436601|2436631|2436261|2436534|N|N|N|N|N| +2436627|AAAAAAAADBOCFCAA|1959-02-27|709|3087|237|1959|5|2|27|1|1959|237|3087|Friday|1959Q1|N|Y|N|2436601|2436631|2436262|2436535|N|N|N|N|N| +2436628|AAAAAAAAEBOCFCAA|1959-02-28|709|3087|237|1959|6|2|28|1|1959|237|3087|Saturday|1959Q1|N|Y|N|2436601|2436631|2436263|2436536|N|N|N|N|N| +2436629|AAAAAAAAFBOCFCAA|1959-03-01|710|3087|238|1959|0|3|1|1|1959|238|3087|Sunday|1959Q1|N|N|N|2436629|2436687|2436264|2436537|N|N|N|N|N| +2436630|AAAAAAAAGBOCFCAA|1959-03-02|710|3087|238|1959|1|3|2|1|1959|238|3087|Monday|1959Q1|N|N|N|2436629|2436687|2436265|2436538|N|N|N|N|N| +2436631|AAAAAAAAHBOCFCAA|1959-03-03|710|3088|238|1959|2|3|3|1|1959|238|3088|Tuesday|1959Q1|N|N|N|2436629|2436687|2436266|2436539|N|N|N|N|N| +2436632|AAAAAAAAIBOCFCAA|1959-03-04|710|3088|238|1959|3|3|4|1|1959|238|3088|Wednesday|1959Q1|N|N|N|2436629|2436687|2436267|2436540|N|N|N|N|N| +2436633|AAAAAAAAJBOCFCAA|1959-03-05|710|3088|238|1959|4|3|5|1|1959|238|3088|Thursday|1959Q1|N|N|N|2436629|2436687|2436268|2436541|N|N|N|N|N| +2436634|AAAAAAAAKBOCFCAA|1959-03-06|710|3088|238|1959|5|3|6|1|1959|238|3088|Friday|1959Q1|N|Y|N|2436629|2436687|2436269|2436542|N|N|N|N|N| +2436635|AAAAAAAALBOCFCAA|1959-03-07|710|3088|238|1959|6|3|7|1|1959|238|3088|Saturday|1959Q1|N|Y|N|2436629|2436687|2436270|2436543|N|N|N|N|N| +2436636|AAAAAAAAMBOCFCAA|1959-03-08|710|3088|238|1959|0|3|8|1|1959|238|3088|Sunday|1959Q1|N|N|N|2436629|2436687|2436271|2436544|N|N|N|N|N| +2436637|AAAAAAAANBOCFCAA|1959-03-09|710|3088|238|1959|1|3|9|1|1959|238|3088|Monday|1959Q1|N|N|N|2436629|2436687|2436272|2436545|N|N|N|N|N| +2436638|AAAAAAAAOBOCFCAA|1959-03-10|710|3089|238|1959|2|3|10|1|1959|238|3089|Tuesday|1959Q1|N|N|N|2436629|2436687|2436273|2436546|N|N|N|N|N| +2436639|AAAAAAAAPBOCFCAA|1959-03-11|710|3089|238|1959|3|3|11|1|1959|238|3089|Wednesday|1959Q1|N|N|N|2436629|2436687|2436274|2436547|N|N|N|N|N| +2436640|AAAAAAAAACOCFCAA|1959-03-12|710|3089|238|1959|4|3|12|1|1959|238|3089|Thursday|1959Q1|N|N|N|2436629|2436687|2436275|2436548|N|N|N|N|N| +2436641|AAAAAAAABCOCFCAA|1959-03-13|710|3089|238|1959|5|3|13|1|1959|238|3089|Friday|1959Q1|N|Y|N|2436629|2436687|2436276|2436549|N|N|N|N|N| +2436642|AAAAAAAACCOCFCAA|1959-03-14|710|3089|238|1959|6|3|14|1|1959|238|3089|Saturday|1959Q1|N|Y|N|2436629|2436687|2436277|2436550|N|N|N|N|N| +2436643|AAAAAAAADCOCFCAA|1959-03-15|710|3089|238|1959|0|3|15|1|1959|238|3089|Sunday|1959Q1|N|N|N|2436629|2436687|2436278|2436551|N|N|N|N|N| +2436644|AAAAAAAAECOCFCAA|1959-03-16|710|3089|238|1959|1|3|16|1|1959|238|3089|Monday|1959Q1|N|N|N|2436629|2436687|2436279|2436552|N|N|N|N|N| +2436645|AAAAAAAAFCOCFCAA|1959-03-17|710|3090|238|1959|2|3|17|1|1959|238|3090|Tuesday|1959Q1|N|N|N|2436629|2436687|2436280|2436553|N|N|N|N|N| +2436646|AAAAAAAAGCOCFCAA|1959-03-18|710|3090|238|1959|3|3|18|1|1959|238|3090|Wednesday|1959Q1|N|N|N|2436629|2436687|2436281|2436554|N|N|N|N|N| +2436647|AAAAAAAAHCOCFCAA|1959-03-19|710|3090|238|1959|4|3|19|1|1959|238|3090|Thursday|1959Q1|N|N|N|2436629|2436687|2436282|2436555|N|N|N|N|N| +2436648|AAAAAAAAICOCFCAA|1959-03-20|710|3090|238|1959|5|3|20|1|1959|238|3090|Friday|1959Q1|N|Y|N|2436629|2436687|2436283|2436556|N|N|N|N|N| +2436649|AAAAAAAAJCOCFCAA|1959-03-21|710|3090|238|1959|6|3|21|1|1959|238|3090|Saturday|1959Q1|N|Y|N|2436629|2436687|2436284|2436557|N|N|N|N|N| +2436650|AAAAAAAAKCOCFCAA|1959-03-22|710|3090|238|1959|0|3|22|1|1959|238|3090|Sunday|1959Q1|N|N|N|2436629|2436687|2436285|2436558|N|N|N|N|N| +2436651|AAAAAAAALCOCFCAA|1959-03-23|710|3090|238|1959|1|3|23|1|1959|238|3090|Monday|1959Q1|N|N|N|2436629|2436687|2436286|2436559|N|N|N|N|N| +2436652|AAAAAAAAMCOCFCAA|1959-03-24|710|3091|238|1959|2|3|24|1|1959|238|3091|Tuesday|1959Q1|N|N|N|2436629|2436687|2436287|2436560|N|N|N|N|N| +2436653|AAAAAAAANCOCFCAA|1959-03-25|710|3091|238|1959|3|3|25|1|1959|238|3091|Wednesday|1959Q1|N|N|N|2436629|2436687|2436288|2436561|N|N|N|N|N| +2436654|AAAAAAAAOCOCFCAA|1959-03-26|710|3091|238|1959|4|3|26|1|1959|238|3091|Thursday|1959Q1|N|N|N|2436629|2436687|2436289|2436562|N|N|N|N|N| +2436655|AAAAAAAAPCOCFCAA|1959-03-27|710|3091|238|1959|5|3|27|1|1959|238|3091|Friday|1959Q1|N|Y|N|2436629|2436687|2436290|2436563|N|N|N|N|N| +2436656|AAAAAAAAADOCFCAA|1959-03-28|710|3091|238|1959|6|3|28|1|1959|238|3091|Saturday|1959Q1|N|Y|N|2436629|2436687|2436291|2436564|N|N|N|N|N| +2436657|AAAAAAAABDOCFCAA|1959-03-29|710|3091|238|1959|0|3|29|1|1959|238|3091|Sunday|1959Q1|N|N|N|2436629|2436687|2436292|2436565|N|N|N|N|N| +2436658|AAAAAAAACDOCFCAA|1959-03-30|710|3091|238|1959|1|3|30|1|1959|238|3091|Monday|1959Q1|N|N|N|2436629|2436687|2436293|2436566|N|N|N|N|N| +2436659|AAAAAAAADDOCFCAA|1959-03-31|710|3092|238|1959|2|3|31|1|1959|238|3092|Tuesday|1959Q1|N|N|N|2436629|2436687|2436294|2436567|N|N|N|N|N| +2436660|AAAAAAAAEDOCFCAA|1959-04-01|711|3092|238|1959|3|4|1|1|1959|238|3092|Wednesday|1959Q1|N|N|N|2436660|2436749|2436295|2436570|N|N|N|N|N| +2436661|AAAAAAAAFDOCFCAA|1959-04-02|711|3092|238|1959|4|4|2|2|1959|238|3092|Thursday|1959Q2|N|N|N|2436660|2436749|2436296|2436571|N|N|N|N|N| +2436662|AAAAAAAAGDOCFCAA|1959-04-03|711|3092|238|1959|5|4|3|2|1959|238|3092|Friday|1959Q2|N|Y|N|2436660|2436749|2436297|2436572|N|N|N|N|N| +2436663|AAAAAAAAHDOCFCAA|1959-04-04|711|3092|238|1959|6|4|4|2|1959|238|3092|Saturday|1959Q2|N|Y|N|2436660|2436749|2436298|2436573|N|N|N|N|N| +2436664|AAAAAAAAIDOCFCAA|1959-04-05|711|3092|238|1959|0|4|5|2|1959|238|3092|Sunday|1959Q2|N|N|N|2436660|2436749|2436299|2436574|N|N|N|N|N| +2436665|AAAAAAAAJDOCFCAA|1959-04-06|711|3092|238|1959|1|4|6|2|1959|238|3092|Monday|1959Q2|N|N|N|2436660|2436749|2436300|2436575|N|N|N|N|N| +2436666|AAAAAAAAKDOCFCAA|1959-04-07|711|3093|238|1959|2|4|7|2|1959|238|3093|Tuesday|1959Q2|N|N|N|2436660|2436749|2436301|2436576|N|N|N|N|N| +2436667|AAAAAAAALDOCFCAA|1959-04-08|711|3093|238|1959|3|4|8|2|1959|238|3093|Wednesday|1959Q2|N|N|N|2436660|2436749|2436302|2436577|N|N|N|N|N| +2436668|AAAAAAAAMDOCFCAA|1959-04-09|711|3093|238|1959|4|4|9|2|1959|238|3093|Thursday|1959Q2|N|N|N|2436660|2436749|2436303|2436578|N|N|N|N|N| +2436669|AAAAAAAANDOCFCAA|1959-04-10|711|3093|238|1959|5|4|10|2|1959|238|3093|Friday|1959Q2|N|Y|N|2436660|2436749|2436304|2436579|N|N|N|N|N| +2436670|AAAAAAAAODOCFCAA|1959-04-11|711|3093|238|1959|6|4|11|2|1959|238|3093|Saturday|1959Q2|N|Y|N|2436660|2436749|2436305|2436580|N|N|N|N|N| +2436671|AAAAAAAAPDOCFCAA|1959-04-12|711|3093|238|1959|0|4|12|2|1959|238|3093|Sunday|1959Q2|N|N|N|2436660|2436749|2436306|2436581|N|N|N|N|N| +2436672|AAAAAAAAAEOCFCAA|1959-04-13|711|3093|238|1959|1|4|13|2|1959|238|3093|Monday|1959Q2|N|N|N|2436660|2436749|2436307|2436582|N|N|N|N|N| +2436673|AAAAAAAABEOCFCAA|1959-04-14|711|3094|238|1959|2|4|14|2|1959|238|3094|Tuesday|1959Q2|N|N|N|2436660|2436749|2436308|2436583|N|N|N|N|N| +2436674|AAAAAAAACEOCFCAA|1959-04-15|711|3094|238|1959|3|4|15|2|1959|238|3094|Wednesday|1959Q2|N|N|N|2436660|2436749|2436309|2436584|N|N|N|N|N| +2436675|AAAAAAAADEOCFCAA|1959-04-16|711|3094|238|1959|4|4|16|2|1959|238|3094|Thursday|1959Q2|N|N|N|2436660|2436749|2436310|2436585|N|N|N|N|N| +2436676|AAAAAAAAEEOCFCAA|1959-04-17|711|3094|238|1959|5|4|17|2|1959|238|3094|Friday|1959Q2|N|Y|N|2436660|2436749|2436311|2436586|N|N|N|N|N| +2436677|AAAAAAAAFEOCFCAA|1959-04-18|711|3094|238|1959|6|4|18|2|1959|238|3094|Saturday|1959Q2|N|Y|N|2436660|2436749|2436312|2436587|N|N|N|N|N| +2436678|AAAAAAAAGEOCFCAA|1959-04-19|711|3094|238|1959|0|4|19|2|1959|238|3094|Sunday|1959Q2|N|N|N|2436660|2436749|2436313|2436588|N|N|N|N|N| +2436679|AAAAAAAAHEOCFCAA|1959-04-20|711|3094|238|1959|1|4|20|2|1959|238|3094|Monday|1959Q2|N|N|N|2436660|2436749|2436314|2436589|N|N|N|N|N| +2436680|AAAAAAAAIEOCFCAA|1959-04-21|711|3095|238|1959|2|4|21|2|1959|238|3095|Tuesday|1959Q2|N|N|N|2436660|2436749|2436315|2436590|N|N|N|N|N| +2436681|AAAAAAAAJEOCFCAA|1959-04-22|711|3095|238|1959|3|4|22|2|1959|238|3095|Wednesday|1959Q2|N|N|N|2436660|2436749|2436316|2436591|N|N|N|N|N| +2436682|AAAAAAAAKEOCFCAA|1959-04-23|711|3095|238|1959|4|4|23|2|1959|238|3095|Thursday|1959Q2|N|N|N|2436660|2436749|2436317|2436592|N|N|N|N|N| +2436683|AAAAAAAALEOCFCAA|1959-04-24|711|3095|238|1959|5|4|24|2|1959|238|3095|Friday|1959Q2|N|Y|N|2436660|2436749|2436318|2436593|N|N|N|N|N| +2436684|AAAAAAAAMEOCFCAA|1959-04-25|711|3095|238|1959|6|4|25|2|1959|238|3095|Saturday|1959Q2|N|Y|N|2436660|2436749|2436319|2436594|N|N|N|N|N| +2436685|AAAAAAAANEOCFCAA|1959-04-26|711|3095|238|1959|0|4|26|2|1959|238|3095|Sunday|1959Q2|N|N|N|2436660|2436749|2436320|2436595|N|N|N|N|N| +2436686|AAAAAAAAOEOCFCAA|1959-04-27|711|3095|238|1959|1|4|27|2|1959|238|3095|Monday|1959Q2|N|N|N|2436660|2436749|2436321|2436596|N|N|N|N|N| +2436687|AAAAAAAAPEOCFCAA|1959-04-28|711|3096|238|1959|2|4|28|2|1959|238|3096|Tuesday|1959Q2|N|N|N|2436660|2436749|2436322|2436597|N|N|N|N|N| +2436688|AAAAAAAAAFOCFCAA|1959-04-29|711|3096|238|1959|3|4|29|2|1959|238|3096|Wednesday|1959Q2|N|N|N|2436660|2436749|2436323|2436598|N|N|N|N|N| +2436689|AAAAAAAABFOCFCAA|1959-04-30|711|3096|238|1959|4|4|30|2|1959|238|3096|Thursday|1959Q2|N|N|N|2436660|2436749|2436324|2436599|N|N|N|N|N| +2436690|AAAAAAAACFOCFCAA|1959-05-01|712|3096|238|1959|5|5|1|2|1959|238|3096|Friday|1959Q2|N|Y|N|2436690|2436809|2436325|2436600|N|N|N|N|N| +2436691|AAAAAAAADFOCFCAA|1959-05-02|712|3096|238|1959|6|5|2|2|1959|238|3096|Saturday|1959Q2|N|Y|N|2436690|2436809|2436326|2436601|N|N|N|N|N| +2436692|AAAAAAAAEFOCFCAA|1959-05-03|712|3096|238|1959|0|5|3|2|1959|238|3096|Sunday|1959Q2|N|N|N|2436690|2436809|2436327|2436602|N|N|N|N|N| +2436693|AAAAAAAAFFOCFCAA|1959-05-04|712|3096|238|1959|1|5|4|2|1959|238|3096|Monday|1959Q2|N|N|N|2436690|2436809|2436328|2436603|N|N|N|N|N| +2436694|AAAAAAAAGFOCFCAA|1959-05-05|712|3097|238|1959|2|5|5|2|1959|238|3097|Tuesday|1959Q2|N|N|N|2436690|2436809|2436329|2436604|N|N|N|N|N| +2436695|AAAAAAAAHFOCFCAA|1959-05-06|712|3097|238|1959|3|5|6|2|1959|238|3097|Wednesday|1959Q2|N|N|N|2436690|2436809|2436330|2436605|N|N|N|N|N| +2436696|AAAAAAAAIFOCFCAA|1959-05-07|712|3097|238|1959|4|5|7|2|1959|238|3097|Thursday|1959Q2|N|N|N|2436690|2436809|2436331|2436606|N|N|N|N|N| +2436697|AAAAAAAAJFOCFCAA|1959-05-08|712|3097|238|1959|5|5|8|2|1959|238|3097|Friday|1959Q2|N|Y|N|2436690|2436809|2436332|2436607|N|N|N|N|N| +2436698|AAAAAAAAKFOCFCAA|1959-05-09|712|3097|238|1959|6|5|9|2|1959|238|3097|Saturday|1959Q2|N|Y|N|2436690|2436809|2436333|2436608|N|N|N|N|N| +2436699|AAAAAAAALFOCFCAA|1959-05-10|712|3097|238|1959|0|5|10|2|1959|238|3097|Sunday|1959Q2|N|N|N|2436690|2436809|2436334|2436609|N|N|N|N|N| +2436700|AAAAAAAAMFOCFCAA|1959-05-11|712|3097|238|1959|1|5|11|2|1959|238|3097|Monday|1959Q2|N|N|N|2436690|2436809|2436335|2436610|N|N|N|N|N| +2436701|AAAAAAAANFOCFCAA|1959-05-12|712|3098|238|1959|2|5|12|2|1959|238|3098|Tuesday|1959Q2|N|N|N|2436690|2436809|2436336|2436611|N|N|N|N|N| +2436702|AAAAAAAAOFOCFCAA|1959-05-13|712|3098|238|1959|3|5|13|2|1959|238|3098|Wednesday|1959Q2|N|N|N|2436690|2436809|2436337|2436612|N|N|N|N|N| +2436703|AAAAAAAAPFOCFCAA|1959-05-14|712|3098|238|1959|4|5|14|2|1959|238|3098|Thursday|1959Q2|N|N|N|2436690|2436809|2436338|2436613|N|N|N|N|N| +2436704|AAAAAAAAAGOCFCAA|1959-05-15|712|3098|238|1959|5|5|15|2|1959|238|3098|Friday|1959Q2|N|Y|N|2436690|2436809|2436339|2436614|N|N|N|N|N| +2436705|AAAAAAAABGOCFCAA|1959-05-16|712|3098|238|1959|6|5|16|2|1959|238|3098|Saturday|1959Q2|N|Y|N|2436690|2436809|2436340|2436615|N|N|N|N|N| +2436706|AAAAAAAACGOCFCAA|1959-05-17|712|3098|238|1959|0|5|17|2|1959|238|3098|Sunday|1959Q2|N|N|N|2436690|2436809|2436341|2436616|N|N|N|N|N| +2436707|AAAAAAAADGOCFCAA|1959-05-18|712|3098|238|1959|1|5|18|2|1959|238|3098|Monday|1959Q2|N|N|N|2436690|2436809|2436342|2436617|N|N|N|N|N| +2436708|AAAAAAAAEGOCFCAA|1959-05-19|712|3099|238|1959|2|5|19|2|1959|238|3099|Tuesday|1959Q2|N|N|N|2436690|2436809|2436343|2436618|N|N|N|N|N| +2436709|AAAAAAAAFGOCFCAA|1959-05-20|712|3099|238|1959|3|5|20|2|1959|238|3099|Wednesday|1959Q2|N|N|N|2436690|2436809|2436344|2436619|N|N|N|N|N| +2436710|AAAAAAAAGGOCFCAA|1959-05-21|712|3099|238|1959|4|5|21|2|1959|238|3099|Thursday|1959Q2|N|N|N|2436690|2436809|2436345|2436620|N|N|N|N|N| +2436711|AAAAAAAAHGOCFCAA|1959-05-22|712|3099|238|1959|5|5|22|2|1959|238|3099|Friday|1959Q2|N|Y|N|2436690|2436809|2436346|2436621|N|N|N|N|N| +2436712|AAAAAAAAIGOCFCAA|1959-05-23|712|3099|238|1959|6|5|23|2|1959|238|3099|Saturday|1959Q2|N|Y|N|2436690|2436809|2436347|2436622|N|N|N|N|N| +2436713|AAAAAAAAJGOCFCAA|1959-05-24|712|3099|238|1959|0|5|24|2|1959|238|3099|Sunday|1959Q2|N|N|N|2436690|2436809|2436348|2436623|N|N|N|N|N| +2436714|AAAAAAAAKGOCFCAA|1959-05-25|712|3099|238|1959|1|5|25|2|1959|238|3099|Monday|1959Q2|N|N|N|2436690|2436809|2436349|2436624|N|N|N|N|N| +2436715|AAAAAAAALGOCFCAA|1959-05-26|712|3100|238|1959|2|5|26|2|1959|238|3100|Tuesday|1959Q2|N|N|N|2436690|2436809|2436350|2436625|N|N|N|N|N| +2436716|AAAAAAAAMGOCFCAA|1959-05-27|712|3100|238|1959|3|5|27|2|1959|238|3100|Wednesday|1959Q2|N|N|N|2436690|2436809|2436351|2436626|N|N|N|N|N| +2436717|AAAAAAAANGOCFCAA|1959-05-28|712|3100|238|1959|4|5|28|2|1959|238|3100|Thursday|1959Q2|N|N|N|2436690|2436809|2436352|2436627|N|N|N|N|N| +2436718|AAAAAAAAOGOCFCAA|1959-05-29|712|3100|238|1959|5|5|29|2|1959|238|3100|Friday|1959Q2|N|Y|N|2436690|2436809|2436353|2436628|N|N|N|N|N| +2436719|AAAAAAAAPGOCFCAA|1959-05-30|712|3100|238|1959|6|5|30|2|1959|238|3100|Saturday|1959Q2|N|Y|N|2436690|2436809|2436354|2436629|N|N|N|N|N| +2436720|AAAAAAAAAHOCFCAA|1959-05-31|712|3100|238|1959|0|5|31|2|1959|238|3100|Sunday|1959Q2|N|N|N|2436690|2436809|2436355|2436630|N|N|N|N|N| +2436721|AAAAAAAABHOCFCAA|1959-06-01|713|3100|239|1959|1|6|1|2|1959|239|3100|Monday|1959Q2|N|N|N|2436721|2436871|2436356|2436631|N|N|N|N|N| +2436722|AAAAAAAACHOCFCAA|1959-06-02|713|3101|239|1959|2|6|2|2|1959|239|3101|Tuesday|1959Q2|N|N|N|2436721|2436871|2436357|2436632|N|N|N|N|N| +2436723|AAAAAAAADHOCFCAA|1959-06-03|713|3101|239|1959|3|6|3|2|1959|239|3101|Wednesday|1959Q2|N|N|N|2436721|2436871|2436358|2436633|N|N|N|N|N| +2436724|AAAAAAAAEHOCFCAA|1959-06-04|713|3101|239|1959|4|6|4|2|1959|239|3101|Thursday|1959Q2|N|N|N|2436721|2436871|2436359|2436634|N|N|N|N|N| +2436725|AAAAAAAAFHOCFCAA|1959-06-05|713|3101|239|1959|5|6|5|2|1959|239|3101|Friday|1959Q2|N|Y|N|2436721|2436871|2436360|2436635|N|N|N|N|N| +2436726|AAAAAAAAGHOCFCAA|1959-06-06|713|3101|239|1959|6|6|6|2|1959|239|3101|Saturday|1959Q2|N|Y|N|2436721|2436871|2436361|2436636|N|N|N|N|N| +2436727|AAAAAAAAHHOCFCAA|1959-06-07|713|3101|239|1959|0|6|7|2|1959|239|3101|Sunday|1959Q2|N|N|N|2436721|2436871|2436362|2436637|N|N|N|N|N| +2436728|AAAAAAAAIHOCFCAA|1959-06-08|713|3101|239|1959|1|6|8|2|1959|239|3101|Monday|1959Q2|N|N|N|2436721|2436871|2436363|2436638|N|N|N|N|N| +2436729|AAAAAAAAJHOCFCAA|1959-06-09|713|3102|239|1959|2|6|9|2|1959|239|3102|Tuesday|1959Q2|N|N|N|2436721|2436871|2436364|2436639|N|N|N|N|N| +2436730|AAAAAAAAKHOCFCAA|1959-06-10|713|3102|239|1959|3|6|10|2|1959|239|3102|Wednesday|1959Q2|N|N|N|2436721|2436871|2436365|2436640|N|N|N|N|N| +2436731|AAAAAAAALHOCFCAA|1959-06-11|713|3102|239|1959|4|6|11|2|1959|239|3102|Thursday|1959Q2|N|N|N|2436721|2436871|2436366|2436641|N|N|N|N|N| +2436732|AAAAAAAAMHOCFCAA|1959-06-12|713|3102|239|1959|5|6|12|2|1959|239|3102|Friday|1959Q2|N|Y|N|2436721|2436871|2436367|2436642|N|N|N|N|N| +2436733|AAAAAAAANHOCFCAA|1959-06-13|713|3102|239|1959|6|6|13|2|1959|239|3102|Saturday|1959Q2|N|Y|N|2436721|2436871|2436368|2436643|N|N|N|N|N| +2436734|AAAAAAAAOHOCFCAA|1959-06-14|713|3102|239|1959|0|6|14|2|1959|239|3102|Sunday|1959Q2|N|N|N|2436721|2436871|2436369|2436644|N|N|N|N|N| +2436735|AAAAAAAAPHOCFCAA|1959-06-15|713|3102|239|1959|1|6|15|2|1959|239|3102|Monday|1959Q2|N|N|N|2436721|2436871|2436370|2436645|N|N|N|N|N| +2436736|AAAAAAAAAIOCFCAA|1959-06-16|713|3103|239|1959|2|6|16|2|1959|239|3103|Tuesday|1959Q2|N|N|N|2436721|2436871|2436371|2436646|N|N|N|N|N| +2436737|AAAAAAAABIOCFCAA|1959-06-17|713|3103|239|1959|3|6|17|2|1959|239|3103|Wednesday|1959Q2|N|N|N|2436721|2436871|2436372|2436647|N|N|N|N|N| +2436738|AAAAAAAACIOCFCAA|1959-06-18|713|3103|239|1959|4|6|18|2|1959|239|3103|Thursday|1959Q2|N|N|N|2436721|2436871|2436373|2436648|N|N|N|N|N| +2436739|AAAAAAAADIOCFCAA|1959-06-19|713|3103|239|1959|5|6|19|2|1959|239|3103|Friday|1959Q2|N|Y|N|2436721|2436871|2436374|2436649|N|N|N|N|N| +2436740|AAAAAAAAEIOCFCAA|1959-06-20|713|3103|239|1959|6|6|20|2|1959|239|3103|Saturday|1959Q2|N|Y|N|2436721|2436871|2436375|2436650|N|N|N|N|N| +2436741|AAAAAAAAFIOCFCAA|1959-06-21|713|3103|239|1959|0|6|21|2|1959|239|3103|Sunday|1959Q2|N|N|N|2436721|2436871|2436376|2436651|N|N|N|N|N| +2436742|AAAAAAAAGIOCFCAA|1959-06-22|713|3103|239|1959|1|6|22|2|1959|239|3103|Monday|1959Q2|N|N|N|2436721|2436871|2436377|2436652|N|N|N|N|N| +2436743|AAAAAAAAHIOCFCAA|1959-06-23|713|3104|239|1959|2|6|23|2|1959|239|3104|Tuesday|1959Q2|N|N|N|2436721|2436871|2436378|2436653|N|N|N|N|N| +2436744|AAAAAAAAIIOCFCAA|1959-06-24|713|3104|239|1959|3|6|24|2|1959|239|3104|Wednesday|1959Q2|N|N|N|2436721|2436871|2436379|2436654|N|N|N|N|N| +2436745|AAAAAAAAJIOCFCAA|1959-06-25|713|3104|239|1959|4|6|25|2|1959|239|3104|Thursday|1959Q2|N|N|N|2436721|2436871|2436380|2436655|N|N|N|N|N| +2436746|AAAAAAAAKIOCFCAA|1959-06-26|713|3104|239|1959|5|6|26|2|1959|239|3104|Friday|1959Q2|N|Y|N|2436721|2436871|2436381|2436656|N|N|N|N|N| +2436747|AAAAAAAALIOCFCAA|1959-06-27|713|3104|239|1959|6|6|27|2|1959|239|3104|Saturday|1959Q2|N|Y|N|2436721|2436871|2436382|2436657|N|N|N|N|N| +2436748|AAAAAAAAMIOCFCAA|1959-06-28|713|3104|239|1959|0|6|28|2|1959|239|3104|Sunday|1959Q2|N|N|N|2436721|2436871|2436383|2436658|N|N|N|N|N| +2436749|AAAAAAAANIOCFCAA|1959-06-29|713|3104|239|1959|1|6|29|2|1959|239|3104|Monday|1959Q2|N|N|N|2436721|2436871|2436384|2436659|N|N|N|N|N| +2436750|AAAAAAAAOIOCFCAA|1959-06-30|713|3105|239|1959|2|6|30|2|1959|239|3105|Tuesday|1959Q2|N|N|N|2436721|2436871|2436385|2436660|N|N|N|N|N| +2436751|AAAAAAAAPIOCFCAA|1959-07-01|714|3105|239|1959|3|7|1|2|1959|239|3105|Wednesday|1959Q2|N|N|N|2436751|2436931|2436386|2436660|N|N|N|N|N| +2436752|AAAAAAAAAJOCFCAA|1959-07-02|714|3105|239|1959|4|7|2|3|1959|239|3105|Thursday|1959Q3|N|N|N|2436751|2436931|2436387|2436661|N|N|N|N|N| +2436753|AAAAAAAABJOCFCAA|1959-07-03|714|3105|239|1959|5|7|3|3|1959|239|3105|Friday|1959Q3|N|Y|N|2436751|2436931|2436388|2436662|N|N|N|N|N| +2436754|AAAAAAAACJOCFCAA|1959-07-04|714|3105|239|1959|6|7|4|3|1959|239|3105|Saturday|1959Q3|N|Y|N|2436751|2436931|2436389|2436663|N|N|N|N|N| +2436755|AAAAAAAADJOCFCAA|1959-07-05|714|3105|239|1959|0|7|5|3|1959|239|3105|Sunday|1959Q3|Y|N|N|2436751|2436931|2436390|2436664|N|N|N|N|N| +2436756|AAAAAAAAEJOCFCAA|1959-07-06|714|3105|239|1959|1|7|6|3|1959|239|3105|Monday|1959Q3|N|N|Y|2436751|2436931|2436391|2436665|N|N|N|N|N| +2436757|AAAAAAAAFJOCFCAA|1959-07-07|714|3106|239|1959|2|7|7|3|1959|239|3106|Tuesday|1959Q3|N|N|N|2436751|2436931|2436392|2436666|N|N|N|N|N| +2436758|AAAAAAAAGJOCFCAA|1959-07-08|714|3106|239|1959|3|7|8|3|1959|239|3106|Wednesday|1959Q3|N|N|N|2436751|2436931|2436393|2436667|N|N|N|N|N| +2436759|AAAAAAAAHJOCFCAA|1959-07-09|714|3106|239|1959|4|7|9|3|1959|239|3106|Thursday|1959Q3|N|N|N|2436751|2436931|2436394|2436668|N|N|N|N|N| +2436760|AAAAAAAAIJOCFCAA|1959-07-10|714|3106|239|1959|5|7|10|3|1959|239|3106|Friday|1959Q3|N|Y|N|2436751|2436931|2436395|2436669|N|N|N|N|N| +2436761|AAAAAAAAJJOCFCAA|1959-07-11|714|3106|239|1959|6|7|11|3|1959|239|3106|Saturday|1959Q3|N|Y|N|2436751|2436931|2436396|2436670|N|N|N|N|N| +2436762|AAAAAAAAKJOCFCAA|1959-07-12|714|3106|239|1959|0|7|12|3|1959|239|3106|Sunday|1959Q3|N|N|N|2436751|2436931|2436397|2436671|N|N|N|N|N| +2436763|AAAAAAAALJOCFCAA|1959-07-13|714|3106|239|1959|1|7|13|3|1959|239|3106|Monday|1959Q3|N|N|N|2436751|2436931|2436398|2436672|N|N|N|N|N| +2436764|AAAAAAAAMJOCFCAA|1959-07-14|714|3107|239|1959|2|7|14|3|1959|239|3107|Tuesday|1959Q3|N|N|N|2436751|2436931|2436399|2436673|N|N|N|N|N| +2436765|AAAAAAAANJOCFCAA|1959-07-15|714|3107|239|1959|3|7|15|3|1959|239|3107|Wednesday|1959Q3|N|N|N|2436751|2436931|2436400|2436674|N|N|N|N|N| +2436766|AAAAAAAAOJOCFCAA|1959-07-16|714|3107|239|1959|4|7|16|3|1959|239|3107|Thursday|1959Q3|N|N|N|2436751|2436931|2436401|2436675|N|N|N|N|N| +2436767|AAAAAAAAPJOCFCAA|1959-07-17|714|3107|239|1959|5|7|17|3|1959|239|3107|Friday|1959Q3|N|Y|N|2436751|2436931|2436402|2436676|N|N|N|N|N| +2436768|AAAAAAAAAKOCFCAA|1959-07-18|714|3107|239|1959|6|7|18|3|1959|239|3107|Saturday|1959Q3|N|Y|N|2436751|2436931|2436403|2436677|N|N|N|N|N| +2436769|AAAAAAAABKOCFCAA|1959-07-19|714|3107|239|1959|0|7|19|3|1959|239|3107|Sunday|1959Q3|N|N|N|2436751|2436931|2436404|2436678|N|N|N|N|N| +2436770|AAAAAAAACKOCFCAA|1959-07-20|714|3107|239|1959|1|7|20|3|1959|239|3107|Monday|1959Q3|N|N|N|2436751|2436931|2436405|2436679|N|N|N|N|N| +2436771|AAAAAAAADKOCFCAA|1959-07-21|714|3108|239|1959|2|7|21|3|1959|239|3108|Tuesday|1959Q3|N|N|N|2436751|2436931|2436406|2436680|N|N|N|N|N| +2436772|AAAAAAAAEKOCFCAA|1959-07-22|714|3108|239|1959|3|7|22|3|1959|239|3108|Wednesday|1959Q3|N|N|N|2436751|2436931|2436407|2436681|N|N|N|N|N| +2436773|AAAAAAAAFKOCFCAA|1959-07-23|714|3108|239|1959|4|7|23|3|1959|239|3108|Thursday|1959Q3|N|N|N|2436751|2436931|2436408|2436682|N|N|N|N|N| +2436774|AAAAAAAAGKOCFCAA|1959-07-24|714|3108|239|1959|5|7|24|3|1959|239|3108|Friday|1959Q3|N|Y|N|2436751|2436931|2436409|2436683|N|N|N|N|N| +2436775|AAAAAAAAHKOCFCAA|1959-07-25|714|3108|239|1959|6|7|25|3|1959|239|3108|Saturday|1959Q3|N|Y|N|2436751|2436931|2436410|2436684|N|N|N|N|N| +2436776|AAAAAAAAIKOCFCAA|1959-07-26|714|3108|239|1959|0|7|26|3|1959|239|3108|Sunday|1959Q3|N|N|N|2436751|2436931|2436411|2436685|N|N|N|N|N| +2436777|AAAAAAAAJKOCFCAA|1959-07-27|714|3108|239|1959|1|7|27|3|1959|239|3108|Monday|1959Q3|N|N|N|2436751|2436931|2436412|2436686|N|N|N|N|N| +2436778|AAAAAAAAKKOCFCAA|1959-07-28|714|3109|239|1959|2|7|28|3|1959|239|3109|Tuesday|1959Q3|N|N|N|2436751|2436931|2436413|2436687|N|N|N|N|N| +2436779|AAAAAAAALKOCFCAA|1959-07-29|714|3109|239|1959|3|7|29|3|1959|239|3109|Wednesday|1959Q3|N|N|N|2436751|2436931|2436414|2436688|N|N|N|N|N| +2436780|AAAAAAAAMKOCFCAA|1959-07-30|714|3109|239|1959|4|7|30|3|1959|239|3109|Thursday|1959Q3|N|N|N|2436751|2436931|2436415|2436689|N|N|N|N|N| +2436781|AAAAAAAANKOCFCAA|1959-07-31|714|3109|239|1959|5|7|31|3|1959|239|3109|Friday|1959Q3|N|Y|N|2436751|2436931|2436416|2436690|N|N|N|N|N| +2436782|AAAAAAAAOKOCFCAA|1959-08-01|715|3109|239|1959|6|8|1|3|1959|239|3109|Saturday|1959Q3|N|Y|N|2436782|2436993|2436417|2436691|N|N|N|N|N| +2436783|AAAAAAAAPKOCFCAA|1959-08-02|715|3109|239|1959|0|8|2|3|1959|239|3109|Sunday|1959Q3|N|N|N|2436782|2436993|2436418|2436692|N|N|N|N|N| +2436784|AAAAAAAAALOCFCAA|1959-08-03|715|3109|239|1959|1|8|3|3|1959|239|3109|Monday|1959Q3|N|N|N|2436782|2436993|2436419|2436693|N|N|N|N|N| +2436785|AAAAAAAABLOCFCAA|1959-08-04|715|3110|239|1959|2|8|4|3|1959|239|3110|Tuesday|1959Q3|N|N|N|2436782|2436993|2436420|2436694|N|N|N|N|N| +2436786|AAAAAAAACLOCFCAA|1959-08-05|715|3110|239|1959|3|8|5|3|1959|239|3110|Wednesday|1959Q3|N|N|N|2436782|2436993|2436421|2436695|N|N|N|N|N| +2436787|AAAAAAAADLOCFCAA|1959-08-06|715|3110|239|1959|4|8|6|3|1959|239|3110|Thursday|1959Q3|N|N|N|2436782|2436993|2436422|2436696|N|N|N|N|N| +2436788|AAAAAAAAELOCFCAA|1959-08-07|715|3110|239|1959|5|8|7|3|1959|239|3110|Friday|1959Q3|N|Y|N|2436782|2436993|2436423|2436697|N|N|N|N|N| +2436789|AAAAAAAAFLOCFCAA|1959-08-08|715|3110|239|1959|6|8|8|3|1959|239|3110|Saturday|1959Q3|N|Y|N|2436782|2436993|2436424|2436698|N|N|N|N|N| +2436790|AAAAAAAAGLOCFCAA|1959-08-09|715|3110|239|1959|0|8|9|3|1959|239|3110|Sunday|1959Q3|N|N|N|2436782|2436993|2436425|2436699|N|N|N|N|N| +2436791|AAAAAAAAHLOCFCAA|1959-08-10|715|3110|239|1959|1|8|10|3|1959|239|3110|Monday|1959Q3|N|N|N|2436782|2436993|2436426|2436700|N|N|N|N|N| +2436792|AAAAAAAAILOCFCAA|1959-08-11|715|3111|239|1959|2|8|11|3|1959|239|3111|Tuesday|1959Q3|N|N|N|2436782|2436993|2436427|2436701|N|N|N|N|N| +2436793|AAAAAAAAJLOCFCAA|1959-08-12|715|3111|239|1959|3|8|12|3|1959|239|3111|Wednesday|1959Q3|N|N|N|2436782|2436993|2436428|2436702|N|N|N|N|N| +2436794|AAAAAAAAKLOCFCAA|1959-08-13|715|3111|239|1959|4|8|13|3|1959|239|3111|Thursday|1959Q3|N|N|N|2436782|2436993|2436429|2436703|N|N|N|N|N| +2436795|AAAAAAAALLOCFCAA|1959-08-14|715|3111|239|1959|5|8|14|3|1959|239|3111|Friday|1959Q3|N|Y|N|2436782|2436993|2436430|2436704|N|N|N|N|N| +2436796|AAAAAAAAMLOCFCAA|1959-08-15|715|3111|239|1959|6|8|15|3|1959|239|3111|Saturday|1959Q3|N|Y|N|2436782|2436993|2436431|2436705|N|N|N|N|N| +2436797|AAAAAAAANLOCFCAA|1959-08-16|715|3111|239|1959|0|8|16|3|1959|239|3111|Sunday|1959Q3|N|N|N|2436782|2436993|2436432|2436706|N|N|N|N|N| +2436798|AAAAAAAAOLOCFCAA|1959-08-17|715|3111|239|1959|1|8|17|3|1959|239|3111|Monday|1959Q3|N|N|N|2436782|2436993|2436433|2436707|N|N|N|N|N| +2436799|AAAAAAAAPLOCFCAA|1959-08-18|715|3112|239|1959|2|8|18|3|1959|239|3112|Tuesday|1959Q3|N|N|N|2436782|2436993|2436434|2436708|N|N|N|N|N| +2436800|AAAAAAAAAMOCFCAA|1959-08-19|715|3112|239|1959|3|8|19|3|1959|239|3112|Wednesday|1959Q3|N|N|N|2436782|2436993|2436435|2436709|N|N|N|N|N| +2436801|AAAAAAAABMOCFCAA|1959-08-20|715|3112|239|1959|4|8|20|3|1959|239|3112|Thursday|1959Q3|N|N|N|2436782|2436993|2436436|2436710|N|N|N|N|N| +2436802|AAAAAAAACMOCFCAA|1959-08-21|715|3112|239|1959|5|8|21|3|1959|239|3112|Friday|1959Q3|N|Y|N|2436782|2436993|2436437|2436711|N|N|N|N|N| +2436803|AAAAAAAADMOCFCAA|1959-08-22|715|3112|239|1959|6|8|22|3|1959|239|3112|Saturday|1959Q3|N|Y|N|2436782|2436993|2436438|2436712|N|N|N|N|N| +2436804|AAAAAAAAEMOCFCAA|1959-08-23|715|3112|239|1959|0|8|23|3|1959|239|3112|Sunday|1959Q3|N|N|N|2436782|2436993|2436439|2436713|N|N|N|N|N| +2436805|AAAAAAAAFMOCFCAA|1959-08-24|715|3112|239|1959|1|8|24|3|1959|239|3112|Monday|1959Q3|N|N|N|2436782|2436993|2436440|2436714|N|N|N|N|N| +2436806|AAAAAAAAGMOCFCAA|1959-08-25|715|3113|239|1959|2|8|25|3|1959|239|3113|Tuesday|1959Q3|N|N|N|2436782|2436993|2436441|2436715|N|N|N|N|N| +2436807|AAAAAAAAHMOCFCAA|1959-08-26|715|3113|239|1959|3|8|26|3|1959|239|3113|Wednesday|1959Q3|N|N|N|2436782|2436993|2436442|2436716|N|N|N|N|N| +2436808|AAAAAAAAIMOCFCAA|1959-08-27|715|3113|239|1959|4|8|27|3|1959|239|3113|Thursday|1959Q3|N|N|N|2436782|2436993|2436443|2436717|N|N|N|N|N| +2436809|AAAAAAAAJMOCFCAA|1959-08-28|715|3113|239|1959|5|8|28|3|1959|239|3113|Friday|1959Q3|N|Y|N|2436782|2436993|2436444|2436718|N|N|N|N|N| +2436810|AAAAAAAAKMOCFCAA|1959-08-29|715|3113|239|1959|6|8|29|3|1959|239|3113|Saturday|1959Q3|N|Y|N|2436782|2436993|2436445|2436719|N|N|N|N|N| +2436811|AAAAAAAALMOCFCAA|1959-08-30|715|3113|239|1959|0|8|30|3|1959|239|3113|Sunday|1959Q3|N|N|N|2436782|2436993|2436446|2436720|N|N|N|N|N| +2436812|AAAAAAAAMMOCFCAA|1959-08-31|715|3113|239|1959|1|8|31|3|1959|239|3113|Monday|1959Q3|N|N|N|2436782|2436993|2436447|2436721|N|N|N|N|N| +2436813|AAAAAAAANMOCFCAA|1959-09-01|716|3114|240|1959|2|9|1|3|1959|240|3114|Tuesday|1959Q3|N|N|N|2436813|2437055|2436448|2436722|N|N|N|N|N| +2436814|AAAAAAAAOMOCFCAA|1959-09-02|716|3114|240|1959|3|9|2|3|1959|240|3114|Wednesday|1959Q3|N|N|N|2436813|2437055|2436449|2436723|N|N|N|N|N| +2436815|AAAAAAAAPMOCFCAA|1959-09-03|716|3114|240|1959|4|9|3|3|1959|240|3114|Thursday|1959Q3|N|N|N|2436813|2437055|2436450|2436724|N|N|N|N|N| +2436816|AAAAAAAAANOCFCAA|1959-09-04|716|3114|240|1959|5|9|4|3|1959|240|3114|Friday|1959Q3|N|Y|N|2436813|2437055|2436451|2436725|N|N|N|N|N| +2436817|AAAAAAAABNOCFCAA|1959-09-05|716|3114|240|1959|6|9|5|3|1959|240|3114|Saturday|1959Q3|N|Y|N|2436813|2437055|2436452|2436726|N|N|N|N|N| +2436818|AAAAAAAACNOCFCAA|1959-09-06|716|3114|240|1959|0|9|6|3|1959|240|3114|Sunday|1959Q3|N|N|N|2436813|2437055|2436453|2436727|N|N|N|N|N| +2436819|AAAAAAAADNOCFCAA|1959-09-07|716|3114|240|1959|1|9|7|3|1959|240|3114|Monday|1959Q3|N|N|N|2436813|2437055|2436454|2436728|N|N|N|N|N| +2436820|AAAAAAAAENOCFCAA|1959-09-08|716|3115|240|1959|2|9|8|3|1959|240|3115|Tuesday|1959Q3|N|N|N|2436813|2437055|2436455|2436729|N|N|N|N|N| +2436821|AAAAAAAAFNOCFCAA|1959-09-09|716|3115|240|1959|3|9|9|3|1959|240|3115|Wednesday|1959Q3|N|N|N|2436813|2437055|2436456|2436730|N|N|N|N|N| +2436822|AAAAAAAAGNOCFCAA|1959-09-10|716|3115|240|1959|4|9|10|3|1959|240|3115|Thursday|1959Q3|N|N|N|2436813|2437055|2436457|2436731|N|N|N|N|N| +2436823|AAAAAAAAHNOCFCAA|1959-09-11|716|3115|240|1959|5|9|11|3|1959|240|3115|Friday|1959Q3|N|Y|N|2436813|2437055|2436458|2436732|N|N|N|N|N| +2436824|AAAAAAAAINOCFCAA|1959-09-12|716|3115|240|1959|6|9|12|3|1959|240|3115|Saturday|1959Q3|N|Y|N|2436813|2437055|2436459|2436733|N|N|N|N|N| +2436825|AAAAAAAAJNOCFCAA|1959-09-13|716|3115|240|1959|0|9|13|3|1959|240|3115|Sunday|1959Q3|N|N|N|2436813|2437055|2436460|2436734|N|N|N|N|N| +2436826|AAAAAAAAKNOCFCAA|1959-09-14|716|3115|240|1959|1|9|14|3|1959|240|3115|Monday|1959Q3|N|N|N|2436813|2437055|2436461|2436735|N|N|N|N|N| +2436827|AAAAAAAALNOCFCAA|1959-09-15|716|3116|240|1959|2|9|15|3|1959|240|3116|Tuesday|1959Q3|N|N|N|2436813|2437055|2436462|2436736|N|N|N|N|N| +2436828|AAAAAAAAMNOCFCAA|1959-09-16|716|3116|240|1959|3|9|16|3|1959|240|3116|Wednesday|1959Q3|N|N|N|2436813|2437055|2436463|2436737|N|N|N|N|N| +2436829|AAAAAAAANNOCFCAA|1959-09-17|716|3116|240|1959|4|9|17|3|1959|240|3116|Thursday|1959Q3|N|N|N|2436813|2437055|2436464|2436738|N|N|N|N|N| +2436830|AAAAAAAAONOCFCAA|1959-09-18|716|3116|240|1959|5|9|18|3|1959|240|3116|Friday|1959Q3|N|Y|N|2436813|2437055|2436465|2436739|N|N|N|N|N| +2436831|AAAAAAAAPNOCFCAA|1959-09-19|716|3116|240|1959|6|9|19|3|1959|240|3116|Saturday|1959Q3|N|Y|N|2436813|2437055|2436466|2436740|N|N|N|N|N| +2436832|AAAAAAAAAOOCFCAA|1959-09-20|716|3116|240|1959|0|9|20|3|1959|240|3116|Sunday|1959Q3|N|N|N|2436813|2437055|2436467|2436741|N|N|N|N|N| +2436833|AAAAAAAABOOCFCAA|1959-09-21|716|3116|240|1959|1|9|21|3|1959|240|3116|Monday|1959Q3|N|N|N|2436813|2437055|2436468|2436742|N|N|N|N|N| +2436834|AAAAAAAACOOCFCAA|1959-09-22|716|3117|240|1959|2|9|22|3|1959|240|3117|Tuesday|1959Q3|N|N|N|2436813|2437055|2436469|2436743|N|N|N|N|N| +2436835|AAAAAAAADOOCFCAA|1959-09-23|716|3117|240|1959|3|9|23|3|1959|240|3117|Wednesday|1959Q3|N|N|N|2436813|2437055|2436470|2436744|N|N|N|N|N| +2436836|AAAAAAAAEOOCFCAA|1959-09-24|716|3117|240|1959|4|9|24|3|1959|240|3117|Thursday|1959Q3|N|N|N|2436813|2437055|2436471|2436745|N|N|N|N|N| +2436837|AAAAAAAAFOOCFCAA|1959-09-25|716|3117|240|1959|5|9|25|3|1959|240|3117|Friday|1959Q3|N|Y|N|2436813|2437055|2436472|2436746|N|N|N|N|N| +2436838|AAAAAAAAGOOCFCAA|1959-09-26|716|3117|240|1959|6|9|26|3|1959|240|3117|Saturday|1959Q3|N|Y|N|2436813|2437055|2436473|2436747|N|N|N|N|N| +2436839|AAAAAAAAHOOCFCAA|1959-09-27|716|3117|240|1959|0|9|27|3|1959|240|3117|Sunday|1959Q3|N|N|N|2436813|2437055|2436474|2436748|N|N|N|N|N| +2436840|AAAAAAAAIOOCFCAA|1959-09-28|716|3117|240|1959|1|9|28|3|1959|240|3117|Monday|1959Q3|N|N|N|2436813|2437055|2436475|2436749|N|N|N|N|N| +2436841|AAAAAAAAJOOCFCAA|1959-09-29|716|3118|240|1959|2|9|29|3|1959|240|3118|Tuesday|1959Q3|N|N|N|2436813|2437055|2436476|2436750|N|N|N|N|N| +2436842|AAAAAAAAKOOCFCAA|1959-09-30|716|3118|240|1959|3|9|30|3|1959|240|3118|Wednesday|1959Q3|N|N|N|2436813|2437055|2436477|2436751|N|N|N|N|N| +2436843|AAAAAAAALOOCFCAA|1959-10-01|717|3118|240|1959|4|10|1|3|1959|240|3118|Thursday|1959Q3|N|N|N|2436843|2437115|2436478|2436751|N|N|N|N|N| +2436844|AAAAAAAAMOOCFCAA|1959-10-02|717|3118|240|1959|5|10|2|4|1959|240|3118|Friday|1959Q4|N|Y|N|2436843|2437115|2436479|2436752|N|N|N|N|N| +2436845|AAAAAAAANOOCFCAA|1959-10-03|717|3118|240|1959|6|10|3|4|1959|240|3118|Saturday|1959Q4|N|Y|N|2436843|2437115|2436480|2436753|N|N|N|N|N| +2436846|AAAAAAAAOOOCFCAA|1959-10-04|717|3118|240|1959|0|10|4|4|1959|240|3118|Sunday|1959Q4|N|N|N|2436843|2437115|2436481|2436754|N|N|N|N|N| +2436847|AAAAAAAAPOOCFCAA|1959-10-05|717|3118|240|1959|1|10|5|4|1959|240|3118|Monday|1959Q4|N|N|N|2436843|2437115|2436482|2436755|N|N|N|N|N| +2436848|AAAAAAAAAPOCFCAA|1959-10-06|717|3119|240|1959|2|10|6|4|1959|240|3119|Tuesday|1959Q4|N|N|N|2436843|2437115|2436483|2436756|N|N|N|N|N| +2436849|AAAAAAAABPOCFCAA|1959-10-07|717|3119|240|1959|3|10|7|4|1959|240|3119|Wednesday|1959Q4|N|N|N|2436843|2437115|2436484|2436757|N|N|N|N|N| +2436850|AAAAAAAACPOCFCAA|1959-10-08|717|3119|240|1959|4|10|8|4|1959|240|3119|Thursday|1959Q4|N|N|N|2436843|2437115|2436485|2436758|N|N|N|N|N| +2436851|AAAAAAAADPOCFCAA|1959-10-09|717|3119|240|1959|5|10|9|4|1959|240|3119|Friday|1959Q4|N|Y|N|2436843|2437115|2436486|2436759|N|N|N|N|N| +2436852|AAAAAAAAEPOCFCAA|1959-10-10|717|3119|240|1959|6|10|10|4|1959|240|3119|Saturday|1959Q4|N|Y|N|2436843|2437115|2436487|2436760|N|N|N|N|N| +2436853|AAAAAAAAFPOCFCAA|1959-10-11|717|3119|240|1959|0|10|11|4|1959|240|3119|Sunday|1959Q4|N|N|N|2436843|2437115|2436488|2436761|N|N|N|N|N| +2436854|AAAAAAAAGPOCFCAA|1959-10-12|717|3119|240|1959|1|10|12|4|1959|240|3119|Monday|1959Q4|N|N|N|2436843|2437115|2436489|2436762|N|N|N|N|N| +2436855|AAAAAAAAHPOCFCAA|1959-10-13|717|3120|240|1959|2|10|13|4|1959|240|3120|Tuesday|1959Q4|N|N|N|2436843|2437115|2436490|2436763|N|N|N|N|N| +2436856|AAAAAAAAIPOCFCAA|1959-10-14|717|3120|240|1959|3|10|14|4|1959|240|3120|Wednesday|1959Q4|N|N|N|2436843|2437115|2436491|2436764|N|N|N|N|N| +2436857|AAAAAAAAJPOCFCAA|1959-10-15|717|3120|240|1959|4|10|15|4|1959|240|3120|Thursday|1959Q4|N|N|N|2436843|2437115|2436492|2436765|N|N|N|N|N| +2436858|AAAAAAAAKPOCFCAA|1959-10-16|717|3120|240|1959|5|10|16|4|1959|240|3120|Friday|1959Q4|N|Y|N|2436843|2437115|2436493|2436766|N|N|N|N|N| +2436859|AAAAAAAALPOCFCAA|1959-10-17|717|3120|240|1959|6|10|17|4|1959|240|3120|Saturday|1959Q4|N|Y|N|2436843|2437115|2436494|2436767|N|N|N|N|N| +2436860|AAAAAAAAMPOCFCAA|1959-10-18|717|3120|240|1959|0|10|18|4|1959|240|3120|Sunday|1959Q4|N|N|N|2436843|2437115|2436495|2436768|N|N|N|N|N| +2436861|AAAAAAAANPOCFCAA|1959-10-19|717|3120|240|1959|1|10|19|4|1959|240|3120|Monday|1959Q4|N|N|N|2436843|2437115|2436496|2436769|N|N|N|N|N| +2436862|AAAAAAAAOPOCFCAA|1959-10-20|717|3121|240|1959|2|10|20|4|1959|240|3121|Tuesday|1959Q4|N|N|N|2436843|2437115|2436497|2436770|N|N|N|N|N| +2436863|AAAAAAAAPPOCFCAA|1959-10-21|717|3121|240|1959|3|10|21|4|1959|240|3121|Wednesday|1959Q4|N|N|N|2436843|2437115|2436498|2436771|N|N|N|N|N| +2436864|AAAAAAAAAAPCFCAA|1959-10-22|717|3121|240|1959|4|10|22|4|1959|240|3121|Thursday|1959Q4|N|N|N|2436843|2437115|2436499|2436772|N|N|N|N|N| +2436865|AAAAAAAABAPCFCAA|1959-10-23|717|3121|240|1959|5|10|23|4|1959|240|3121|Friday|1959Q4|N|Y|N|2436843|2437115|2436500|2436773|N|N|N|N|N| +2436866|AAAAAAAACAPCFCAA|1959-10-24|717|3121|240|1959|6|10|24|4|1959|240|3121|Saturday|1959Q4|N|Y|N|2436843|2437115|2436501|2436774|N|N|N|N|N| +2436867|AAAAAAAADAPCFCAA|1959-10-25|717|3121|240|1959|0|10|25|4|1959|240|3121|Sunday|1959Q4|N|N|N|2436843|2437115|2436502|2436775|N|N|N|N|N| +2436868|AAAAAAAAEAPCFCAA|1959-10-26|717|3121|240|1959|1|10|26|4|1959|240|3121|Monday|1959Q4|N|N|N|2436843|2437115|2436503|2436776|N|N|N|N|N| +2436869|AAAAAAAAFAPCFCAA|1959-10-27|717|3122|240|1959|2|10|27|4|1959|240|3122|Tuesday|1959Q4|N|N|N|2436843|2437115|2436504|2436777|N|N|N|N|N| +2436870|AAAAAAAAGAPCFCAA|1959-10-28|717|3122|240|1959|3|10|28|4|1959|240|3122|Wednesday|1959Q4|N|N|N|2436843|2437115|2436505|2436778|N|N|N|N|N| +2436871|AAAAAAAAHAPCFCAA|1959-10-29|717|3122|240|1959|4|10|29|4|1959|240|3122|Thursday|1959Q4|N|N|N|2436843|2437115|2436506|2436779|N|N|N|N|N| +2436872|AAAAAAAAIAPCFCAA|1959-10-30|717|3122|240|1959|5|10|30|4|1959|240|3122|Friday|1959Q4|N|Y|N|2436843|2437115|2436507|2436780|N|N|N|N|N| +2436873|AAAAAAAAJAPCFCAA|1959-10-31|717|3122|240|1959|6|10|31|4|1959|240|3122|Saturday|1959Q4|N|Y|N|2436843|2437115|2436508|2436781|N|N|N|N|N| +2436874|AAAAAAAAKAPCFCAA|1959-11-01|718|3122|240|1959|0|11|1|4|1959|240|3122|Sunday|1959Q4|N|N|N|2436874|2437177|2436509|2436782|N|N|N|N|N| +2436875|AAAAAAAALAPCFCAA|1959-11-02|718|3122|240|1959|1|11|2|4|1959|240|3122|Monday|1959Q4|N|N|N|2436874|2437177|2436510|2436783|N|N|N|N|N| +2436876|AAAAAAAAMAPCFCAA|1959-11-03|718|3123|240|1959|2|11|3|4|1959|240|3123|Tuesday|1959Q4|N|N|N|2436874|2437177|2436511|2436784|N|N|N|N|N| +2436877|AAAAAAAANAPCFCAA|1959-11-04|718|3123|240|1959|3|11|4|4|1959|240|3123|Wednesday|1959Q4|N|N|N|2436874|2437177|2436512|2436785|N|N|N|N|N| +2436878|AAAAAAAAOAPCFCAA|1959-11-05|718|3123|240|1959|4|11|5|4|1959|240|3123|Thursday|1959Q4|N|N|N|2436874|2437177|2436513|2436786|N|N|N|N|N| +2436879|AAAAAAAAPAPCFCAA|1959-11-06|718|3123|240|1959|5|11|6|4|1959|240|3123|Friday|1959Q4|N|Y|N|2436874|2437177|2436514|2436787|N|N|N|N|N| +2436880|AAAAAAAAABPCFCAA|1959-11-07|718|3123|240|1959|6|11|7|4|1959|240|3123|Saturday|1959Q4|N|Y|N|2436874|2437177|2436515|2436788|N|N|N|N|N| +2436881|AAAAAAAABBPCFCAA|1959-11-08|718|3123|240|1959|0|11|8|4|1959|240|3123|Sunday|1959Q4|N|N|N|2436874|2437177|2436516|2436789|N|N|N|N|N| +2436882|AAAAAAAACBPCFCAA|1959-11-09|718|3123|240|1959|1|11|9|4|1959|240|3123|Monday|1959Q4|N|N|N|2436874|2437177|2436517|2436790|N|N|N|N|N| +2436883|AAAAAAAADBPCFCAA|1959-11-10|718|3124|240|1959|2|11|10|4|1959|240|3124|Tuesday|1959Q4|N|N|N|2436874|2437177|2436518|2436791|N|N|N|N|N| +2436884|AAAAAAAAEBPCFCAA|1959-11-11|718|3124|240|1959|3|11|11|4|1959|240|3124|Wednesday|1959Q4|N|N|N|2436874|2437177|2436519|2436792|N|N|N|N|N| +2436885|AAAAAAAAFBPCFCAA|1959-11-12|718|3124|240|1959|4|11|12|4|1959|240|3124|Thursday|1959Q4|N|N|N|2436874|2437177|2436520|2436793|N|N|N|N|N| +2436886|AAAAAAAAGBPCFCAA|1959-11-13|718|3124|240|1959|5|11|13|4|1959|240|3124|Friday|1959Q4|N|Y|N|2436874|2437177|2436521|2436794|N|N|N|N|N| +2436887|AAAAAAAAHBPCFCAA|1959-11-14|718|3124|240|1959|6|11|14|4|1959|240|3124|Saturday|1959Q4|N|Y|N|2436874|2437177|2436522|2436795|N|N|N|N|N| +2436888|AAAAAAAAIBPCFCAA|1959-11-15|718|3124|240|1959|0|11|15|4|1959|240|3124|Sunday|1959Q4|N|N|N|2436874|2437177|2436523|2436796|N|N|N|N|N| +2436889|AAAAAAAAJBPCFCAA|1959-11-16|718|3124|240|1959|1|11|16|4|1959|240|3124|Monday|1959Q4|N|N|N|2436874|2437177|2436524|2436797|N|N|N|N|N| +2436890|AAAAAAAAKBPCFCAA|1959-11-17|718|3125|240|1959|2|11|17|4|1959|240|3125|Tuesday|1959Q4|N|N|N|2436874|2437177|2436525|2436798|N|N|N|N|N| +2436891|AAAAAAAALBPCFCAA|1959-11-18|718|3125|240|1959|3|11|18|4|1959|240|3125|Wednesday|1959Q4|N|N|N|2436874|2437177|2436526|2436799|N|N|N|N|N| +2436892|AAAAAAAAMBPCFCAA|1959-11-19|718|3125|240|1959|4|11|19|4|1959|240|3125|Thursday|1959Q4|N|N|N|2436874|2437177|2436527|2436800|N|N|N|N|N| +2436893|AAAAAAAANBPCFCAA|1959-11-20|718|3125|240|1959|5|11|20|4|1959|240|3125|Friday|1959Q4|N|Y|N|2436874|2437177|2436528|2436801|N|N|N|N|N| +2436894|AAAAAAAAOBPCFCAA|1959-11-21|718|3125|240|1959|6|11|21|4|1959|240|3125|Saturday|1959Q4|N|Y|N|2436874|2437177|2436529|2436802|N|N|N|N|N| +2436895|AAAAAAAAPBPCFCAA|1959-11-22|718|3125|240|1959|0|11|22|4|1959|240|3125|Sunday|1959Q4|N|N|N|2436874|2437177|2436530|2436803|N|N|N|N|N| +2436896|AAAAAAAAACPCFCAA|1959-11-23|718|3125|240|1959|1|11|23|4|1959|240|3125|Monday|1959Q4|N|N|N|2436874|2437177|2436531|2436804|N|N|N|N|N| +2436897|AAAAAAAABCPCFCAA|1959-11-24|718|3126|240|1959|2|11|24|4|1959|240|3126|Tuesday|1959Q4|N|N|N|2436874|2437177|2436532|2436805|N|N|N|N|N| +2436898|AAAAAAAACCPCFCAA|1959-11-25|718|3126|240|1959|3|11|25|4|1959|240|3126|Wednesday|1959Q4|N|N|N|2436874|2437177|2436533|2436806|N|N|N|N|N| +2436899|AAAAAAAADCPCFCAA|1959-11-26|718|3126|240|1959|4|11|26|4|1959|240|3126|Thursday|1959Q4|N|N|N|2436874|2437177|2436534|2436807|N|N|N|N|N| +2436900|AAAAAAAAECPCFCAA|1959-11-27|718|3126|240|1959|5|11|27|4|1959|240|3126|Friday|1959Q4|N|Y|N|2436874|2437177|2436535|2436808|N|N|N|N|N| +2436901|AAAAAAAAFCPCFCAA|1959-11-28|718|3126|240|1959|6|11|28|4|1959|240|3126|Saturday|1959Q4|N|Y|N|2436874|2437177|2436536|2436809|N|N|N|N|N| +2436902|AAAAAAAAGCPCFCAA|1959-11-29|718|3126|240|1959|0|11|29|4|1959|240|3126|Sunday|1959Q4|N|N|N|2436874|2437177|2436537|2436810|N|N|N|N|N| +2436903|AAAAAAAAHCPCFCAA|1959-11-30|718|3126|240|1959|1|11|30|4|1959|240|3126|Monday|1959Q4|N|N|N|2436874|2437177|2436538|2436811|N|N|N|N|N| +2436904|AAAAAAAAICPCFCAA|1959-12-01|719|3127|241|1959|2|12|1|4|1959|241|3127|Tuesday|1959Q4|N|N|N|2436904|2437237|2436539|2436812|N|N|N|N|N| +2436905|AAAAAAAAJCPCFCAA|1959-12-02|719|3127|241|1959|3|12|2|4|1959|241|3127|Wednesday|1959Q4|N|N|N|2436904|2437237|2436540|2436813|N|N|N|N|N| +2436906|AAAAAAAAKCPCFCAA|1959-12-03|719|3127|241|1959|4|12|3|4|1959|241|3127|Thursday|1959Q4|N|N|N|2436904|2437237|2436541|2436814|N|N|N|N|N| +2436907|AAAAAAAALCPCFCAA|1959-12-04|719|3127|241|1959|5|12|4|4|1959|241|3127|Friday|1959Q4|N|Y|N|2436904|2437237|2436542|2436815|N|N|N|N|N| +2436908|AAAAAAAAMCPCFCAA|1959-12-05|719|3127|241|1959|6|12|5|4|1959|241|3127|Saturday|1959Q4|N|Y|N|2436904|2437237|2436543|2436816|N|N|N|N|N| +2436909|AAAAAAAANCPCFCAA|1959-12-06|719|3127|241|1959|0|12|6|4|1959|241|3127|Sunday|1959Q4|N|N|N|2436904|2437237|2436544|2436817|N|N|N|N|N| +2436910|AAAAAAAAOCPCFCAA|1959-12-07|719|3127|241|1959|1|12|7|4|1959|241|3127|Monday|1959Q4|N|N|N|2436904|2437237|2436545|2436818|N|N|N|N|N| +2436911|AAAAAAAAPCPCFCAA|1959-12-08|719|3128|241|1959|2|12|8|4|1959|241|3128|Tuesday|1959Q4|N|N|N|2436904|2437237|2436546|2436819|N|N|N|N|N| +2436912|AAAAAAAAADPCFCAA|1959-12-09|719|3128|241|1959|3|12|9|4|1959|241|3128|Wednesday|1959Q4|N|N|N|2436904|2437237|2436547|2436820|N|N|N|N|N| +2436913|AAAAAAAABDPCFCAA|1959-12-10|719|3128|241|1959|4|12|10|4|1959|241|3128|Thursday|1959Q4|N|N|N|2436904|2437237|2436548|2436821|N|N|N|N|N| +2436914|AAAAAAAACDPCFCAA|1959-12-11|719|3128|241|1959|5|12|11|4|1959|241|3128|Friday|1959Q4|N|Y|N|2436904|2437237|2436549|2436822|N|N|N|N|N| +2436915|AAAAAAAADDPCFCAA|1959-12-12|719|3128|241|1959|6|12|12|4|1959|241|3128|Saturday|1959Q4|N|Y|N|2436904|2437237|2436550|2436823|N|N|N|N|N| +2436916|AAAAAAAAEDPCFCAA|1959-12-13|719|3128|241|1959|0|12|13|4|1959|241|3128|Sunday|1959Q4|N|N|N|2436904|2437237|2436551|2436824|N|N|N|N|N| +2436917|AAAAAAAAFDPCFCAA|1959-12-14|719|3128|241|1959|1|12|14|4|1959|241|3128|Monday|1959Q4|N|N|N|2436904|2437237|2436552|2436825|N|N|N|N|N| +2436918|AAAAAAAAGDPCFCAA|1959-12-15|719|3129|241|1959|2|12|15|4|1959|241|3129|Tuesday|1959Q4|N|N|N|2436904|2437237|2436553|2436826|N|N|N|N|N| +2436919|AAAAAAAAHDPCFCAA|1959-12-16|719|3129|241|1959|3|12|16|4|1959|241|3129|Wednesday|1959Q4|N|N|N|2436904|2437237|2436554|2436827|N|N|N|N|N| +2436920|AAAAAAAAIDPCFCAA|1959-12-17|719|3129|241|1959|4|12|17|4|1959|241|3129|Thursday|1959Q4|N|N|N|2436904|2437237|2436555|2436828|N|N|N|N|N| +2436921|AAAAAAAAJDPCFCAA|1959-12-18|719|3129|241|1959|5|12|18|4|1959|241|3129|Friday|1959Q4|N|Y|N|2436904|2437237|2436556|2436829|N|N|N|N|N| +2436922|AAAAAAAAKDPCFCAA|1959-12-19|719|3129|241|1959|6|12|19|4|1959|241|3129|Saturday|1959Q4|N|Y|N|2436904|2437237|2436557|2436830|N|N|N|N|N| +2436923|AAAAAAAALDPCFCAA|1959-12-20|719|3129|241|1959|0|12|20|4|1959|241|3129|Sunday|1959Q4|N|N|N|2436904|2437237|2436558|2436831|N|N|N|N|N| +2436924|AAAAAAAAMDPCFCAA|1959-12-21|719|3129|241|1959|1|12|21|4|1959|241|3129|Monday|1959Q4|N|N|N|2436904|2437237|2436559|2436832|N|N|N|N|N| +2436925|AAAAAAAANDPCFCAA|1959-12-22|719|3130|241|1959|2|12|22|4|1959|241|3130|Tuesday|1959Q4|N|N|N|2436904|2437237|2436560|2436833|N|N|N|N|N| +2436926|AAAAAAAAODPCFCAA|1959-12-23|719|3130|241|1959|3|12|23|4|1959|241|3130|Wednesday|1959Q4|N|N|N|2436904|2437237|2436561|2436834|N|N|N|N|N| +2436927|AAAAAAAAPDPCFCAA|1959-12-24|719|3130|241|1959|4|12|24|4|1959|241|3130|Thursday|1959Q4|N|N|N|2436904|2437237|2436562|2436835|N|N|N|N|N| +2436928|AAAAAAAAAEPCFCAA|1959-12-25|719|3130|241|1959|5|12|25|4|1959|241|3130|Friday|1959Q4|N|Y|N|2436904|2437237|2436563|2436836|N|N|N|N|N| +2436929|AAAAAAAABEPCFCAA|1959-12-26|719|3130|241|1959|6|12|26|4|1959|241|3130|Saturday|1959Q4|Y|Y|N|2436904|2437237|2436564|2436837|N|N|N|N|N| +2436930|AAAAAAAACEPCFCAA|1959-12-27|719|3130|241|1959|0|12|27|4|1959|241|3130|Sunday|1959Q4|N|N|Y|2436904|2437237|2436565|2436838|N|N|N|N|N| +2436931|AAAAAAAADEPCFCAA|1959-12-28|719|3130|241|1959|1|12|28|4|1959|241|3130|Monday|1959Q4|N|N|N|2436904|2437237|2436566|2436839|N|N|N|N|N| +2436932|AAAAAAAAEEPCFCAA|1959-12-29|719|3131|241|1959|2|12|29|4|1959|241|3131|Tuesday|1959Q4|N|N|N|2436904|2437237|2436567|2436840|N|N|N|N|N| +2436933|AAAAAAAAFEPCFCAA|1959-12-30|719|3131|241|1959|3|12|30|4|1959|241|3131|Wednesday|1959Q4|N|N|N|2436904|2437237|2436568|2436841|N|N|N|N|N| +2436934|AAAAAAAAGEPCFCAA|1959-12-31|719|3131|241|1959|4|12|31|4|1959|241|3131|Thursday|1959Q4|N|N|N|2436904|2437237|2436569|2436842|N|N|N|N|N| +2436935|AAAAAAAAHEPCFCAA|1960-01-01|720|3131|241|1960|5|1|1|1|1960|241|3131|Friday|1960Q1|Y|Y|N|2436935|2436934|2436570|2436843|N|N|N|N|N| +2436936|AAAAAAAAIEPCFCAA|1960-01-02|720|3131|241|1960|6|1|2|1|1960|241|3131|Saturday|1960Q1|N|Y|Y|2436935|2436934|2436571|2436844|N|N|N|N|N| +2436937|AAAAAAAAJEPCFCAA|1960-01-03|720|3131|241|1960|0|1|3|1|1960|241|3131|Sunday|1960Q1|N|N|N|2436935|2436934|2436572|2436845|N|N|N|N|N| +2436938|AAAAAAAAKEPCFCAA|1960-01-04|720|3131|241|1960|1|1|4|1|1960|241|3131|Monday|1960Q1|N|N|N|2436935|2436934|2436573|2436846|N|N|N|N|N| +2436939|AAAAAAAALEPCFCAA|1960-01-05|720|3132|241|1960|2|1|5|1|1960|241|3132|Tuesday|1960Q1|N|N|N|2436935|2436934|2436574|2436847|N|N|N|N|N| +2436940|AAAAAAAAMEPCFCAA|1960-01-06|720|3132|241|1960|3|1|6|1|1960|241|3132|Wednesday|1960Q1|N|N|N|2436935|2436934|2436575|2436848|N|N|N|N|N| +2436941|AAAAAAAANEPCFCAA|1960-01-07|720|3132|241|1960|4|1|7|1|1960|241|3132|Thursday|1960Q1|N|N|N|2436935|2436934|2436576|2436849|N|N|N|N|N| +2436942|AAAAAAAAOEPCFCAA|1960-01-08|720|3132|241|1960|5|1|8|1|1960|241|3132|Friday|1960Q1|N|Y|N|2436935|2436934|2436577|2436850|N|N|N|N|N| +2436943|AAAAAAAAPEPCFCAA|1960-01-09|720|3132|241|1960|6|1|9|1|1960|241|3132|Saturday|1960Q1|N|Y|N|2436935|2436934|2436578|2436851|N|N|N|N|N| +2436944|AAAAAAAAAFPCFCAA|1960-01-10|720|3132|241|1960|0|1|10|1|1960|241|3132|Sunday|1960Q1|N|N|N|2436935|2436934|2436579|2436852|N|N|N|N|N| +2436945|AAAAAAAABFPCFCAA|1960-01-11|720|3132|241|1960|1|1|11|1|1960|241|3132|Monday|1960Q1|N|N|N|2436935|2436934|2436580|2436853|N|N|N|N|N| +2436946|AAAAAAAACFPCFCAA|1960-01-12|720|3133|241|1960|2|1|12|1|1960|241|3133|Tuesday|1960Q1|N|N|N|2436935|2436934|2436581|2436854|N|N|N|N|N| +2436947|AAAAAAAADFPCFCAA|1960-01-13|720|3133|241|1960|3|1|13|1|1960|241|3133|Wednesday|1960Q1|N|N|N|2436935|2436934|2436582|2436855|N|N|N|N|N| +2436948|AAAAAAAAEFPCFCAA|1960-01-14|720|3133|241|1960|4|1|14|1|1960|241|3133|Thursday|1960Q1|N|N|N|2436935|2436934|2436583|2436856|N|N|N|N|N| +2436949|AAAAAAAAFFPCFCAA|1960-01-15|720|3133|241|1960|5|1|15|1|1960|241|3133|Friday|1960Q1|N|Y|N|2436935|2436934|2436584|2436857|N|N|N|N|N| +2436950|AAAAAAAAGFPCFCAA|1960-01-16|720|3133|241|1960|6|1|16|1|1960|241|3133|Saturday|1960Q1|N|Y|N|2436935|2436934|2436585|2436858|N|N|N|N|N| +2436951|AAAAAAAAHFPCFCAA|1960-01-17|720|3133|241|1960|0|1|17|1|1960|241|3133|Sunday|1960Q1|N|N|N|2436935|2436934|2436586|2436859|N|N|N|N|N| +2436952|AAAAAAAAIFPCFCAA|1960-01-18|720|3133|241|1960|1|1|18|1|1960|241|3133|Monday|1960Q1|N|N|N|2436935|2436934|2436587|2436860|N|N|N|N|N| +2436953|AAAAAAAAJFPCFCAA|1960-01-19|720|3134|241|1960|2|1|19|1|1960|241|3134|Tuesday|1960Q1|N|N|N|2436935|2436934|2436588|2436861|N|N|N|N|N| +2436954|AAAAAAAAKFPCFCAA|1960-01-20|720|3134|241|1960|3|1|20|1|1960|241|3134|Wednesday|1960Q1|N|N|N|2436935|2436934|2436589|2436862|N|N|N|N|N| +2436955|AAAAAAAALFPCFCAA|1960-01-21|720|3134|241|1960|4|1|21|1|1960|241|3134|Thursday|1960Q1|N|N|N|2436935|2436934|2436590|2436863|N|N|N|N|N| +2436956|AAAAAAAAMFPCFCAA|1960-01-22|720|3134|241|1960|5|1|22|1|1960|241|3134|Friday|1960Q1|N|Y|N|2436935|2436934|2436591|2436864|N|N|N|N|N| +2436957|AAAAAAAANFPCFCAA|1960-01-23|720|3134|241|1960|6|1|23|1|1960|241|3134|Saturday|1960Q1|N|Y|N|2436935|2436934|2436592|2436865|N|N|N|N|N| +2436958|AAAAAAAAOFPCFCAA|1960-01-24|720|3134|241|1960|0|1|24|1|1960|241|3134|Sunday|1960Q1|N|N|N|2436935|2436934|2436593|2436866|N|N|N|N|N| +2436959|AAAAAAAAPFPCFCAA|1960-01-25|720|3134|241|1960|1|1|25|1|1960|241|3134|Monday|1960Q1|N|N|N|2436935|2436934|2436594|2436867|N|N|N|N|N| +2436960|AAAAAAAAAGPCFCAA|1960-01-26|720|3135|241|1960|2|1|26|1|1960|241|3135|Tuesday|1960Q1|N|N|N|2436935|2436934|2436595|2436868|N|N|N|N|N| +2436961|AAAAAAAABGPCFCAA|1960-01-27|720|3135|241|1960|3|1|27|1|1960|241|3135|Wednesday|1960Q1|N|N|N|2436935|2436934|2436596|2436869|N|N|N|N|N| +2436962|AAAAAAAACGPCFCAA|1960-01-28|720|3135|241|1960|4|1|28|1|1960|241|3135|Thursday|1960Q1|N|N|N|2436935|2436934|2436597|2436870|N|N|N|N|N| +2436963|AAAAAAAADGPCFCAA|1960-01-29|720|3135|241|1960|5|1|29|1|1960|241|3135|Friday|1960Q1|N|Y|N|2436935|2436934|2436598|2436871|N|N|N|N|N| +2436964|AAAAAAAAEGPCFCAA|1960-01-30|720|3135|241|1960|6|1|30|1|1960|241|3135|Saturday|1960Q1|N|Y|N|2436935|2436934|2436599|2436872|N|N|N|N|N| +2436965|AAAAAAAAFGPCFCAA|1960-01-31|720|3135|241|1960|0|1|31|1|1960|241|3135|Sunday|1960Q1|N|N|N|2436935|2436934|2436600|2436873|N|N|N|N|N| +2436966|AAAAAAAAGGPCFCAA|1960-02-01|721|3135|241|1960|1|2|1|1|1960|241|3135|Monday|1960Q1|N|N|N|2436966|2436996|2436601|2436874|N|N|N|N|N| +2436967|AAAAAAAAHGPCFCAA|1960-02-02|721|3136|241|1960|2|2|2|1|1960|241|3136|Tuesday|1960Q1|N|N|N|2436966|2436996|2436602|2436875|N|N|N|N|N| +2436968|AAAAAAAAIGPCFCAA|1960-02-03|721|3136|241|1960|3|2|3|1|1960|241|3136|Wednesday|1960Q1|N|N|N|2436966|2436996|2436603|2436876|N|N|N|N|N| +2436969|AAAAAAAAJGPCFCAA|1960-02-04|721|3136|241|1960|4|2|4|1|1960|241|3136|Thursday|1960Q1|N|N|N|2436966|2436996|2436604|2436877|N|N|N|N|N| +2436970|AAAAAAAAKGPCFCAA|1960-02-05|721|3136|241|1960|5|2|5|1|1960|241|3136|Friday|1960Q1|N|Y|N|2436966|2436996|2436605|2436878|N|N|N|N|N| +2436971|AAAAAAAALGPCFCAA|1960-02-06|721|3136|241|1960|6|2|6|1|1960|241|3136|Saturday|1960Q1|N|Y|N|2436966|2436996|2436606|2436879|N|N|N|N|N| +2436972|AAAAAAAAMGPCFCAA|1960-02-07|721|3136|241|1960|0|2|7|1|1960|241|3136|Sunday|1960Q1|N|N|N|2436966|2436996|2436607|2436880|N|N|N|N|N| +2436973|AAAAAAAANGPCFCAA|1960-02-08|721|3136|241|1960|1|2|8|1|1960|241|3136|Monday|1960Q1|N|N|N|2436966|2436996|2436608|2436881|N|N|N|N|N| +2436974|AAAAAAAAOGPCFCAA|1960-02-09|721|3137|241|1960|2|2|9|1|1960|241|3137|Tuesday|1960Q1|N|N|N|2436966|2436996|2436609|2436882|N|N|N|N|N| +2436975|AAAAAAAAPGPCFCAA|1960-02-10|721|3137|241|1960|3|2|10|1|1960|241|3137|Wednesday|1960Q1|N|N|N|2436966|2436996|2436610|2436883|N|N|N|N|N| +2436976|AAAAAAAAAHPCFCAA|1960-02-11|721|3137|241|1960|4|2|11|1|1960|241|3137|Thursday|1960Q1|N|N|N|2436966|2436996|2436611|2436884|N|N|N|N|N| +2436977|AAAAAAAABHPCFCAA|1960-02-12|721|3137|241|1960|5|2|12|1|1960|241|3137|Friday|1960Q1|N|Y|N|2436966|2436996|2436612|2436885|N|N|N|N|N| +2436978|AAAAAAAACHPCFCAA|1960-02-13|721|3137|241|1960|6|2|13|1|1960|241|3137|Saturday|1960Q1|N|Y|N|2436966|2436996|2436613|2436886|N|N|N|N|N| +2436979|AAAAAAAADHPCFCAA|1960-02-14|721|3137|241|1960|0|2|14|1|1960|241|3137|Sunday|1960Q1|N|N|N|2436966|2436996|2436614|2436887|N|N|N|N|N| +2436980|AAAAAAAAEHPCFCAA|1960-02-15|721|3137|241|1960|1|2|15|1|1960|241|3137|Monday|1960Q1|N|N|N|2436966|2436996|2436615|2436888|N|N|N|N|N| +2436981|AAAAAAAAFHPCFCAA|1960-02-16|721|3138|241|1960|2|2|16|1|1960|241|3138|Tuesday|1960Q1|N|N|N|2436966|2436996|2436616|2436889|N|N|N|N|N| +2436982|AAAAAAAAGHPCFCAA|1960-02-17|721|3138|241|1960|3|2|17|1|1960|241|3138|Wednesday|1960Q1|N|N|N|2436966|2436996|2436617|2436890|N|N|N|N|N| +2436983|AAAAAAAAHHPCFCAA|1960-02-18|721|3138|241|1960|4|2|18|1|1960|241|3138|Thursday|1960Q1|N|N|N|2436966|2436996|2436618|2436891|N|N|N|N|N| +2436984|AAAAAAAAIHPCFCAA|1960-02-19|721|3138|241|1960|5|2|19|1|1960|241|3138|Friday|1960Q1|N|Y|N|2436966|2436996|2436619|2436892|N|N|N|N|N| +2436985|AAAAAAAAJHPCFCAA|1960-02-20|721|3138|241|1960|6|2|20|1|1960|241|3138|Saturday|1960Q1|N|Y|N|2436966|2436996|2436620|2436893|N|N|N|N|N| +2436986|AAAAAAAAKHPCFCAA|1960-02-21|721|3138|241|1960|0|2|21|1|1960|241|3138|Sunday|1960Q1|N|N|N|2436966|2436996|2436621|2436894|N|N|N|N|N| +2436987|AAAAAAAALHPCFCAA|1960-02-22|721|3138|241|1960|1|2|22|1|1960|241|3138|Monday|1960Q1|N|N|N|2436966|2436996|2436622|2436895|N|N|N|N|N| +2436988|AAAAAAAAMHPCFCAA|1960-02-23|721|3139|241|1960|2|2|23|1|1960|241|3139|Tuesday|1960Q1|N|N|N|2436966|2436996|2436623|2436896|N|N|N|N|N| +2436989|AAAAAAAANHPCFCAA|1960-02-24|721|3139|241|1960|3|2|24|1|1960|241|3139|Wednesday|1960Q1|N|N|N|2436966|2436996|2436624|2436897|N|N|N|N|N| +2436990|AAAAAAAAOHPCFCAA|1960-02-25|721|3139|241|1960|4|2|25|1|1960|241|3139|Thursday|1960Q1|N|N|N|2436966|2436996|2436625|2436898|N|N|N|N|N| +2436991|AAAAAAAAPHPCFCAA|1960-02-26|721|3139|241|1960|5|2|26|1|1960|241|3139|Friday|1960Q1|N|Y|N|2436966|2436996|2436626|2436899|N|N|N|N|N| +2436992|AAAAAAAAAIPCFCAA|1960-02-27|721|3139|241|1960|6|2|27|1|1960|241|3139|Saturday|1960Q1|N|Y|N|2436966|2436996|2436627|2436900|N|N|N|N|N| +2436993|AAAAAAAABIPCFCAA|1960-02-28|721|3139|241|1960|0|2|28|1|1960|241|3139|Sunday|1960Q1|N|N|N|2436966|2436996|2436628|2436901|N|N|N|N|N| +2436994|AAAAAAAACIPCFCAA|1960-02-29|721|3139|241|1960|1|2|29|1|1960|241|3139|Monday|1960Q1|N|N|N|2436966|2436996|2436628|2436902|N|N|N|N|N| +2436995|AAAAAAAADIPCFCAA|1960-03-01|722|3140|242|1960|2|3|1|1|1960|242|3140|Tuesday|1960Q1|N|N|N|2436995|2437054|2436629|2436903|N|N|N|N|N| +2436996|AAAAAAAAEIPCFCAA|1960-03-02|722|3140|242|1960|3|3|2|1|1960|242|3140|Wednesday|1960Q1|N|N|N|2436995|2437054|2436630|2436904|N|N|N|N|N| +2436997|AAAAAAAAFIPCFCAA|1960-03-03|722|3140|242|1960|4|3|3|1|1960|242|3140|Thursday|1960Q1|N|N|N|2436995|2437054|2436631|2436905|N|N|N|N|N| +2436998|AAAAAAAAGIPCFCAA|1960-03-04|722|3140|242|1960|5|3|4|1|1960|242|3140|Friday|1960Q1|N|Y|N|2436995|2437054|2436632|2436906|N|N|N|N|N| +2436999|AAAAAAAAHIPCFCAA|1960-03-05|722|3140|242|1960|6|3|5|1|1960|242|3140|Saturday|1960Q1|N|Y|N|2436995|2437054|2436633|2436907|N|N|N|N|N| +2437000|AAAAAAAAIIPCFCAA|1960-03-06|722|3140|242|1960|0|3|6|1|1960|242|3140|Sunday|1960Q1|N|N|N|2436995|2437054|2436634|2436908|N|N|N|N|N| +2437001|AAAAAAAAJIPCFCAA|1960-03-07|722|3140|242|1960|1|3|7|1|1960|242|3140|Monday|1960Q1|N|N|N|2436995|2437054|2436635|2436909|N|N|N|N|N| +2437002|AAAAAAAAKIPCFCAA|1960-03-08|722|3141|242|1960|2|3|8|1|1960|242|3141|Tuesday|1960Q1|N|N|N|2436995|2437054|2436636|2436910|N|N|N|N|N| +2437003|AAAAAAAALIPCFCAA|1960-03-09|722|3141|242|1960|3|3|9|1|1960|242|3141|Wednesday|1960Q1|N|N|N|2436995|2437054|2436637|2436911|N|N|N|N|N| +2437004|AAAAAAAAMIPCFCAA|1960-03-10|722|3141|242|1960|4|3|10|1|1960|242|3141|Thursday|1960Q1|N|N|N|2436995|2437054|2436638|2436912|N|N|N|N|N| +2437005|AAAAAAAANIPCFCAA|1960-03-11|722|3141|242|1960|5|3|11|1|1960|242|3141|Friday|1960Q1|N|Y|N|2436995|2437054|2436639|2436913|N|N|N|N|N| +2437006|AAAAAAAAOIPCFCAA|1960-03-12|722|3141|242|1960|6|3|12|1|1960|242|3141|Saturday|1960Q1|N|Y|N|2436995|2437054|2436640|2436914|N|N|N|N|N| +2437007|AAAAAAAAPIPCFCAA|1960-03-13|722|3141|242|1960|0|3|13|1|1960|242|3141|Sunday|1960Q1|N|N|N|2436995|2437054|2436641|2436915|N|N|N|N|N| +2437008|AAAAAAAAAJPCFCAA|1960-03-14|722|3141|242|1960|1|3|14|1|1960|242|3141|Monday|1960Q1|N|N|N|2436995|2437054|2436642|2436916|N|N|N|N|N| +2437009|AAAAAAAABJPCFCAA|1960-03-15|722|3142|242|1960|2|3|15|1|1960|242|3142|Tuesday|1960Q1|N|N|N|2436995|2437054|2436643|2436917|N|N|N|N|N| +2437010|AAAAAAAACJPCFCAA|1960-03-16|722|3142|242|1960|3|3|16|1|1960|242|3142|Wednesday|1960Q1|N|N|N|2436995|2437054|2436644|2436918|N|N|N|N|N| +2437011|AAAAAAAADJPCFCAA|1960-03-17|722|3142|242|1960|4|3|17|1|1960|242|3142|Thursday|1960Q1|N|N|N|2436995|2437054|2436645|2436919|N|N|N|N|N| +2437012|AAAAAAAAEJPCFCAA|1960-03-18|722|3142|242|1960|5|3|18|1|1960|242|3142|Friday|1960Q1|N|Y|N|2436995|2437054|2436646|2436920|N|N|N|N|N| +2437013|AAAAAAAAFJPCFCAA|1960-03-19|722|3142|242|1960|6|3|19|1|1960|242|3142|Saturday|1960Q1|N|Y|N|2436995|2437054|2436647|2436921|N|N|N|N|N| +2437014|AAAAAAAAGJPCFCAA|1960-03-20|722|3142|242|1960|0|3|20|1|1960|242|3142|Sunday|1960Q1|N|N|N|2436995|2437054|2436648|2436922|N|N|N|N|N| +2437015|AAAAAAAAHJPCFCAA|1960-03-21|722|3142|242|1960|1|3|21|1|1960|242|3142|Monday|1960Q1|N|N|N|2436995|2437054|2436649|2436923|N|N|N|N|N| +2437016|AAAAAAAAIJPCFCAA|1960-03-22|722|3143|242|1960|2|3|22|1|1960|242|3143|Tuesday|1960Q1|N|N|N|2436995|2437054|2436650|2436924|N|N|N|N|N| +2437017|AAAAAAAAJJPCFCAA|1960-03-23|722|3143|242|1960|3|3|23|1|1960|242|3143|Wednesday|1960Q1|N|N|N|2436995|2437054|2436651|2436925|N|N|N|N|N| +2437018|AAAAAAAAKJPCFCAA|1960-03-24|722|3143|242|1960|4|3|24|1|1960|242|3143|Thursday|1960Q1|N|N|N|2436995|2437054|2436652|2436926|N|N|N|N|N| +2437019|AAAAAAAALJPCFCAA|1960-03-25|722|3143|242|1960|5|3|25|1|1960|242|3143|Friday|1960Q1|N|Y|N|2436995|2437054|2436653|2436927|N|N|N|N|N| +2437020|AAAAAAAAMJPCFCAA|1960-03-26|722|3143|242|1960|6|3|26|1|1960|242|3143|Saturday|1960Q1|N|Y|N|2436995|2437054|2436654|2436928|N|N|N|N|N| +2437021|AAAAAAAANJPCFCAA|1960-03-27|722|3143|242|1960|0|3|27|1|1960|242|3143|Sunday|1960Q1|N|N|N|2436995|2437054|2436655|2436929|N|N|N|N|N| +2437022|AAAAAAAAOJPCFCAA|1960-03-28|722|3143|242|1960|1|3|28|1|1960|242|3143|Monday|1960Q1|N|N|N|2436995|2437054|2436656|2436930|N|N|N|N|N| +2437023|AAAAAAAAPJPCFCAA|1960-03-29|722|3144|242|1960|2|3|29|1|1960|242|3144|Tuesday|1960Q1|N|N|N|2436995|2437054|2436657|2436931|N|N|N|N|N| +2437024|AAAAAAAAAKPCFCAA|1960-03-30|722|3144|242|1960|3|3|30|1|1960|242|3144|Wednesday|1960Q1|N|N|N|2436995|2437054|2436658|2436932|N|N|N|N|N| +2437025|AAAAAAAABKPCFCAA|1960-03-31|722|3144|242|1960|4|3|31|1|1960|242|3144|Thursday|1960Q1|N|N|N|2436995|2437054|2436659|2436933|N|N|N|N|N| +2437026|AAAAAAAACKPCFCAA|1960-04-01|723|3144|242|1960|5|4|1|2|1960|242|3144|Friday|1960Q2|N|Y|N|2437026|2437116|2436660|2436935|N|N|N|N|N| +2437027|AAAAAAAADKPCFCAA|1960-04-02|723|3144|242|1960|6|4|2|2|1960|242|3144|Saturday|1960Q2|N|Y|N|2437026|2437116|2436661|2436936|N|N|N|N|N| +2437028|AAAAAAAAEKPCFCAA|1960-04-03|723|3144|242|1960|0|4|3|2|1960|242|3144|Sunday|1960Q2|N|N|N|2437026|2437116|2436662|2436937|N|N|N|N|N| +2437029|AAAAAAAAFKPCFCAA|1960-04-04|723|3144|242|1960|1|4|4|2|1960|242|3144|Monday|1960Q2|N|N|N|2437026|2437116|2436663|2436938|N|N|N|N|N| +2437030|AAAAAAAAGKPCFCAA|1960-04-05|723|3145|242|1960|2|4|5|2|1960|242|3145|Tuesday|1960Q2|N|N|N|2437026|2437116|2436664|2436939|N|N|N|N|N| +2437031|AAAAAAAAHKPCFCAA|1960-04-06|723|3145|242|1960|3|4|6|2|1960|242|3145|Wednesday|1960Q2|N|N|N|2437026|2437116|2436665|2436940|N|N|N|N|N| +2437032|AAAAAAAAIKPCFCAA|1960-04-07|723|3145|242|1960|4|4|7|2|1960|242|3145|Thursday|1960Q2|N|N|N|2437026|2437116|2436666|2436941|N|N|N|N|N| +2437033|AAAAAAAAJKPCFCAA|1960-04-08|723|3145|242|1960|5|4|8|2|1960|242|3145|Friday|1960Q2|N|Y|N|2437026|2437116|2436667|2436942|N|N|N|N|N| +2437034|AAAAAAAAKKPCFCAA|1960-04-09|723|3145|242|1960|6|4|9|2|1960|242|3145|Saturday|1960Q2|N|Y|N|2437026|2437116|2436668|2436943|N|N|N|N|N| +2437035|AAAAAAAALKPCFCAA|1960-04-10|723|3145|242|1960|0|4|10|2|1960|242|3145|Sunday|1960Q2|N|N|N|2437026|2437116|2436669|2436944|N|N|N|N|N| +2437036|AAAAAAAAMKPCFCAA|1960-04-11|723|3145|242|1960|1|4|11|2|1960|242|3145|Monday|1960Q2|N|N|N|2437026|2437116|2436670|2436945|N|N|N|N|N| +2437037|AAAAAAAANKPCFCAA|1960-04-12|723|3146|242|1960|2|4|12|2|1960|242|3146|Tuesday|1960Q2|N|N|N|2437026|2437116|2436671|2436946|N|N|N|N|N| +2437038|AAAAAAAAOKPCFCAA|1960-04-13|723|3146|242|1960|3|4|13|2|1960|242|3146|Wednesday|1960Q2|N|N|N|2437026|2437116|2436672|2436947|N|N|N|N|N| +2437039|AAAAAAAAPKPCFCAA|1960-04-14|723|3146|242|1960|4|4|14|2|1960|242|3146|Thursday|1960Q2|N|N|N|2437026|2437116|2436673|2436948|N|N|N|N|N| +2437040|AAAAAAAAALPCFCAA|1960-04-15|723|3146|242|1960|5|4|15|2|1960|242|3146|Friday|1960Q2|N|Y|N|2437026|2437116|2436674|2436949|N|N|N|N|N| +2437041|AAAAAAAABLPCFCAA|1960-04-16|723|3146|242|1960|6|4|16|2|1960|242|3146|Saturday|1960Q2|N|Y|N|2437026|2437116|2436675|2436950|N|N|N|N|N| +2437042|AAAAAAAACLPCFCAA|1960-04-17|723|3146|242|1960|0|4|17|2|1960|242|3146|Sunday|1960Q2|N|N|N|2437026|2437116|2436676|2436951|N|N|N|N|N| +2437043|AAAAAAAADLPCFCAA|1960-04-18|723|3146|242|1960|1|4|18|2|1960|242|3146|Monday|1960Q2|N|N|N|2437026|2437116|2436677|2436952|N|N|N|N|N| +2437044|AAAAAAAAELPCFCAA|1960-04-19|723|3147|242|1960|2|4|19|2|1960|242|3147|Tuesday|1960Q2|N|N|N|2437026|2437116|2436678|2436953|N|N|N|N|N| +2437045|AAAAAAAAFLPCFCAA|1960-04-20|723|3147|242|1960|3|4|20|2|1960|242|3147|Wednesday|1960Q2|N|N|N|2437026|2437116|2436679|2436954|N|N|N|N|N| +2437046|AAAAAAAAGLPCFCAA|1960-04-21|723|3147|242|1960|4|4|21|2|1960|242|3147|Thursday|1960Q2|N|N|N|2437026|2437116|2436680|2436955|N|N|N|N|N| +2437047|AAAAAAAAHLPCFCAA|1960-04-22|723|3147|242|1960|5|4|22|2|1960|242|3147|Friday|1960Q2|N|Y|N|2437026|2437116|2436681|2436956|N|N|N|N|N| +2437048|AAAAAAAAILPCFCAA|1960-04-23|723|3147|242|1960|6|4|23|2|1960|242|3147|Saturday|1960Q2|N|Y|N|2437026|2437116|2436682|2436957|N|N|N|N|N| +2437049|AAAAAAAAJLPCFCAA|1960-04-24|723|3147|242|1960|0|4|24|2|1960|242|3147|Sunday|1960Q2|N|N|N|2437026|2437116|2436683|2436958|N|N|N|N|N| +2437050|AAAAAAAAKLPCFCAA|1960-04-25|723|3147|242|1960|1|4|25|2|1960|242|3147|Monday|1960Q2|N|N|N|2437026|2437116|2436684|2436959|N|N|N|N|N| +2437051|AAAAAAAALLPCFCAA|1960-04-26|723|3148|242|1960|2|4|26|2|1960|242|3148|Tuesday|1960Q2|N|N|N|2437026|2437116|2436685|2436960|N|N|N|N|N| +2437052|AAAAAAAAMLPCFCAA|1960-04-27|723|3148|242|1960|3|4|27|2|1960|242|3148|Wednesday|1960Q2|N|N|N|2437026|2437116|2436686|2436961|N|N|N|N|N| +2437053|AAAAAAAANLPCFCAA|1960-04-28|723|3148|242|1960|4|4|28|2|1960|242|3148|Thursday|1960Q2|N|N|N|2437026|2437116|2436687|2436962|N|N|N|N|N| +2437054|AAAAAAAAOLPCFCAA|1960-04-29|723|3148|242|1960|5|4|29|2|1960|242|3148|Friday|1960Q2|N|Y|N|2437026|2437116|2436688|2436963|N|N|N|N|N| +2437055|AAAAAAAAPLPCFCAA|1960-04-30|723|3148|242|1960|6|4|30|2|1960|242|3148|Saturday|1960Q2|N|Y|N|2437026|2437116|2436689|2436964|N|N|N|N|N| +2437056|AAAAAAAAAMPCFCAA|1960-05-01|724|3148|242|1960|0|5|1|2|1960|242|3148|Sunday|1960Q2|N|N|N|2437056|2437176|2436690|2436965|N|N|N|N|N| +2437057|AAAAAAAABMPCFCAA|1960-05-02|724|3148|242|1960|1|5|2|2|1960|242|3148|Monday|1960Q2|N|N|N|2437056|2437176|2436691|2436966|N|N|N|N|N| +2437058|AAAAAAAACMPCFCAA|1960-05-03|724|3149|242|1960|2|5|3|2|1960|242|3149|Tuesday|1960Q2|N|N|N|2437056|2437176|2436692|2436967|N|N|N|N|N| +2437059|AAAAAAAADMPCFCAA|1960-05-04|724|3149|242|1960|3|5|4|2|1960|242|3149|Wednesday|1960Q2|N|N|N|2437056|2437176|2436693|2436968|N|N|N|N|N| +2437060|AAAAAAAAEMPCFCAA|1960-05-05|724|3149|242|1960|4|5|5|2|1960|242|3149|Thursday|1960Q2|N|N|N|2437056|2437176|2436694|2436969|N|N|N|N|N| +2437061|AAAAAAAAFMPCFCAA|1960-05-06|724|3149|242|1960|5|5|6|2|1960|242|3149|Friday|1960Q2|N|Y|N|2437056|2437176|2436695|2436970|N|N|N|N|N| +2437062|AAAAAAAAGMPCFCAA|1960-05-07|724|3149|242|1960|6|5|7|2|1960|242|3149|Saturday|1960Q2|N|Y|N|2437056|2437176|2436696|2436971|N|N|N|N|N| +2437063|AAAAAAAAHMPCFCAA|1960-05-08|724|3149|242|1960|0|5|8|2|1960|242|3149|Sunday|1960Q2|N|N|N|2437056|2437176|2436697|2436972|N|N|N|N|N| +2437064|AAAAAAAAIMPCFCAA|1960-05-09|724|3149|242|1960|1|5|9|2|1960|242|3149|Monday|1960Q2|N|N|N|2437056|2437176|2436698|2436973|N|N|N|N|N| +2437065|AAAAAAAAJMPCFCAA|1960-05-10|724|3150|242|1960|2|5|10|2|1960|242|3150|Tuesday|1960Q2|N|N|N|2437056|2437176|2436699|2436974|N|N|N|N|N| +2437066|AAAAAAAAKMPCFCAA|1960-05-11|724|3150|242|1960|3|5|11|2|1960|242|3150|Wednesday|1960Q2|N|N|N|2437056|2437176|2436700|2436975|N|N|N|N|N| +2437067|AAAAAAAALMPCFCAA|1960-05-12|724|3150|242|1960|4|5|12|2|1960|242|3150|Thursday|1960Q2|N|N|N|2437056|2437176|2436701|2436976|N|N|N|N|N| +2437068|AAAAAAAAMMPCFCAA|1960-05-13|724|3150|242|1960|5|5|13|2|1960|242|3150|Friday|1960Q2|N|Y|N|2437056|2437176|2436702|2436977|N|N|N|N|N| +2437069|AAAAAAAANMPCFCAA|1960-05-14|724|3150|242|1960|6|5|14|2|1960|242|3150|Saturday|1960Q2|N|Y|N|2437056|2437176|2436703|2436978|N|N|N|N|N| +2437070|AAAAAAAAOMPCFCAA|1960-05-15|724|3150|242|1960|0|5|15|2|1960|242|3150|Sunday|1960Q2|N|N|N|2437056|2437176|2436704|2436979|N|N|N|N|N| +2437071|AAAAAAAAPMPCFCAA|1960-05-16|724|3150|242|1960|1|5|16|2|1960|242|3150|Monday|1960Q2|N|N|N|2437056|2437176|2436705|2436980|N|N|N|N|N| +2437072|AAAAAAAAANPCFCAA|1960-05-17|724|3151|242|1960|2|5|17|2|1960|242|3151|Tuesday|1960Q2|N|N|N|2437056|2437176|2436706|2436981|N|N|N|N|N| +2437073|AAAAAAAABNPCFCAA|1960-05-18|724|3151|242|1960|3|5|18|2|1960|242|3151|Wednesday|1960Q2|N|N|N|2437056|2437176|2436707|2436982|N|N|N|N|N| +2437074|AAAAAAAACNPCFCAA|1960-05-19|724|3151|242|1960|4|5|19|2|1960|242|3151|Thursday|1960Q2|N|N|N|2437056|2437176|2436708|2436983|N|N|N|N|N| +2437075|AAAAAAAADNPCFCAA|1960-05-20|724|3151|242|1960|5|5|20|2|1960|242|3151|Friday|1960Q2|N|Y|N|2437056|2437176|2436709|2436984|N|N|N|N|N| +2437076|AAAAAAAAENPCFCAA|1960-05-21|724|3151|242|1960|6|5|21|2|1960|242|3151|Saturday|1960Q2|N|Y|N|2437056|2437176|2436710|2436985|N|N|N|N|N| +2437077|AAAAAAAAFNPCFCAA|1960-05-22|724|3151|242|1960|0|5|22|2|1960|242|3151|Sunday|1960Q2|N|N|N|2437056|2437176|2436711|2436986|N|N|N|N|N| +2437078|AAAAAAAAGNPCFCAA|1960-05-23|724|3151|242|1960|1|5|23|2|1960|242|3151|Monday|1960Q2|N|N|N|2437056|2437176|2436712|2436987|N|N|N|N|N| +2437079|AAAAAAAAHNPCFCAA|1960-05-24|724|3152|242|1960|2|5|24|2|1960|242|3152|Tuesday|1960Q2|N|N|N|2437056|2437176|2436713|2436988|N|N|N|N|N| +2437080|AAAAAAAAINPCFCAA|1960-05-25|724|3152|242|1960|3|5|25|2|1960|242|3152|Wednesday|1960Q2|N|N|N|2437056|2437176|2436714|2436989|N|N|N|N|N| +2437081|AAAAAAAAJNPCFCAA|1960-05-26|724|3152|242|1960|4|5|26|2|1960|242|3152|Thursday|1960Q2|N|N|N|2437056|2437176|2436715|2436990|N|N|N|N|N| +2437082|AAAAAAAAKNPCFCAA|1960-05-27|724|3152|242|1960|5|5|27|2|1960|242|3152|Friday|1960Q2|N|Y|N|2437056|2437176|2436716|2436991|N|N|N|N|N| +2437083|AAAAAAAALNPCFCAA|1960-05-28|724|3152|242|1960|6|5|28|2|1960|242|3152|Saturday|1960Q2|N|Y|N|2437056|2437176|2436717|2436992|N|N|N|N|N| +2437084|AAAAAAAAMNPCFCAA|1960-05-29|724|3152|242|1960|0|5|29|2|1960|242|3152|Sunday|1960Q2|N|N|N|2437056|2437176|2436718|2436993|N|N|N|N|N| +2437085|AAAAAAAANNPCFCAA|1960-05-30|724|3152|242|1960|1|5|30|2|1960|242|3152|Monday|1960Q2|N|N|N|2437056|2437176|2436719|2436994|N|N|N|N|N| +2437086|AAAAAAAAONPCFCAA|1960-05-31|724|3153|242|1960|2|5|31|2|1960|242|3153|Tuesday|1960Q2|N|N|N|2437056|2437176|2436720|2436995|N|N|N|N|N| +2437087|AAAAAAAAPNPCFCAA|1960-06-01|725|3153|243|1960|3|6|1|2|1960|243|3153|Wednesday|1960Q2|N|N|N|2437087|2437238|2436721|2436996|N|N|N|N|N| +2437088|AAAAAAAAAOPCFCAA|1960-06-02|725|3153|243|1960|4|6|2|2|1960|243|3153|Thursday|1960Q2|N|N|N|2437087|2437238|2436722|2436997|N|N|N|N|N| +2437089|AAAAAAAABOPCFCAA|1960-06-03|725|3153|243|1960|5|6|3|2|1960|243|3153|Friday|1960Q2|N|Y|N|2437087|2437238|2436723|2436998|N|N|N|N|N| +2437090|AAAAAAAACOPCFCAA|1960-06-04|725|3153|243|1960|6|6|4|2|1960|243|3153|Saturday|1960Q2|N|Y|N|2437087|2437238|2436724|2436999|N|N|N|N|N| +2437091|AAAAAAAADOPCFCAA|1960-06-05|725|3153|243|1960|0|6|5|2|1960|243|3153|Sunday|1960Q2|N|N|N|2437087|2437238|2436725|2437000|N|N|N|N|N| +2437092|AAAAAAAAEOPCFCAA|1960-06-06|725|3153|243|1960|1|6|6|2|1960|243|3153|Monday|1960Q2|N|N|N|2437087|2437238|2436726|2437001|N|N|N|N|N| +2437093|AAAAAAAAFOPCFCAA|1960-06-07|725|3154|243|1960|2|6|7|2|1960|243|3154|Tuesday|1960Q2|N|N|N|2437087|2437238|2436727|2437002|N|N|N|N|N| +2437094|AAAAAAAAGOPCFCAA|1960-06-08|725|3154|243|1960|3|6|8|2|1960|243|3154|Wednesday|1960Q2|N|N|N|2437087|2437238|2436728|2437003|N|N|N|N|N| +2437095|AAAAAAAAHOPCFCAA|1960-06-09|725|3154|243|1960|4|6|9|2|1960|243|3154|Thursday|1960Q2|N|N|N|2437087|2437238|2436729|2437004|N|N|N|N|N| +2437096|AAAAAAAAIOPCFCAA|1960-06-10|725|3154|243|1960|5|6|10|2|1960|243|3154|Friday|1960Q2|N|Y|N|2437087|2437238|2436730|2437005|N|N|N|N|N| +2437097|AAAAAAAAJOPCFCAA|1960-06-11|725|3154|243|1960|6|6|11|2|1960|243|3154|Saturday|1960Q2|N|Y|N|2437087|2437238|2436731|2437006|N|N|N|N|N| +2437098|AAAAAAAAKOPCFCAA|1960-06-12|725|3154|243|1960|0|6|12|2|1960|243|3154|Sunday|1960Q2|N|N|N|2437087|2437238|2436732|2437007|N|N|N|N|N| +2437099|AAAAAAAALOPCFCAA|1960-06-13|725|3154|243|1960|1|6|13|2|1960|243|3154|Monday|1960Q2|N|N|N|2437087|2437238|2436733|2437008|N|N|N|N|N| +2437100|AAAAAAAAMOPCFCAA|1960-06-14|725|3155|243|1960|2|6|14|2|1960|243|3155|Tuesday|1960Q2|N|N|N|2437087|2437238|2436734|2437009|N|N|N|N|N| +2437101|AAAAAAAANOPCFCAA|1960-06-15|725|3155|243|1960|3|6|15|2|1960|243|3155|Wednesday|1960Q2|N|N|N|2437087|2437238|2436735|2437010|N|N|N|N|N| +2437102|AAAAAAAAOOPCFCAA|1960-06-16|725|3155|243|1960|4|6|16|2|1960|243|3155|Thursday|1960Q2|N|N|N|2437087|2437238|2436736|2437011|N|N|N|N|N| +2437103|AAAAAAAAPOPCFCAA|1960-06-17|725|3155|243|1960|5|6|17|2|1960|243|3155|Friday|1960Q2|N|Y|N|2437087|2437238|2436737|2437012|N|N|N|N|N| +2437104|AAAAAAAAAPPCFCAA|1960-06-18|725|3155|243|1960|6|6|18|2|1960|243|3155|Saturday|1960Q2|N|Y|N|2437087|2437238|2436738|2437013|N|N|N|N|N| +2437105|AAAAAAAABPPCFCAA|1960-06-19|725|3155|243|1960|0|6|19|2|1960|243|3155|Sunday|1960Q2|N|N|N|2437087|2437238|2436739|2437014|N|N|N|N|N| +2437106|AAAAAAAACPPCFCAA|1960-06-20|725|3155|243|1960|1|6|20|2|1960|243|3155|Monday|1960Q2|N|N|N|2437087|2437238|2436740|2437015|N|N|N|N|N| +2437107|AAAAAAAADPPCFCAA|1960-06-21|725|3156|243|1960|2|6|21|2|1960|243|3156|Tuesday|1960Q2|N|N|N|2437087|2437238|2436741|2437016|N|N|N|N|N| +2437108|AAAAAAAAEPPCFCAA|1960-06-22|725|3156|243|1960|3|6|22|2|1960|243|3156|Wednesday|1960Q2|N|N|N|2437087|2437238|2436742|2437017|N|N|N|N|N| +2437109|AAAAAAAAFPPCFCAA|1960-06-23|725|3156|243|1960|4|6|23|2|1960|243|3156|Thursday|1960Q2|N|N|N|2437087|2437238|2436743|2437018|N|N|N|N|N| +2437110|AAAAAAAAGPPCFCAA|1960-06-24|725|3156|243|1960|5|6|24|2|1960|243|3156|Friday|1960Q2|N|Y|N|2437087|2437238|2436744|2437019|N|N|N|N|N| +2437111|AAAAAAAAHPPCFCAA|1960-06-25|725|3156|243|1960|6|6|25|2|1960|243|3156|Saturday|1960Q2|N|Y|N|2437087|2437238|2436745|2437020|N|N|N|N|N| +2437112|AAAAAAAAIPPCFCAA|1960-06-26|725|3156|243|1960|0|6|26|2|1960|243|3156|Sunday|1960Q2|N|N|N|2437087|2437238|2436746|2437021|N|N|N|N|N| +2437113|AAAAAAAAJPPCFCAA|1960-06-27|725|3156|243|1960|1|6|27|2|1960|243|3156|Monday|1960Q2|N|N|N|2437087|2437238|2436747|2437022|N|N|N|N|N| +2437114|AAAAAAAAKPPCFCAA|1960-06-28|725|3157|243|1960|2|6|28|2|1960|243|3157|Tuesday|1960Q2|N|N|N|2437087|2437238|2436748|2437023|N|N|N|N|N| +2437115|AAAAAAAALPPCFCAA|1960-06-29|725|3157|243|1960|3|6|29|2|1960|243|3157|Wednesday|1960Q2|N|N|N|2437087|2437238|2436749|2437024|N|N|N|N|N| +2437116|AAAAAAAAMPPCFCAA|1960-06-30|725|3157|243|1960|4|6|30|2|1960|243|3157|Thursday|1960Q2|N|N|N|2437087|2437238|2436750|2437025|N|N|N|N|N| +2437117|AAAAAAAANPPCFCAA|1960-07-01|726|3157|243|1960|5|7|1|3|1960|243|3157|Friday|1960Q3|N|Y|N|2437117|2437298|2436751|2437026|N|N|N|N|N| +2437118|AAAAAAAAOPPCFCAA|1960-07-02|726|3157|243|1960|6|7|2|3|1960|243|3157|Saturday|1960Q3|N|Y|N|2437117|2437298|2436752|2437027|N|N|N|N|N| +2437119|AAAAAAAAPPPCFCAA|1960-07-03|726|3157|243|1960|0|7|3|3|1960|243|3157|Sunday|1960Q3|N|N|N|2437117|2437298|2436753|2437028|N|N|N|N|N| +2437120|AAAAAAAAAAADFCAA|1960-07-04|726|3157|243|1960|1|7|4|3|1960|243|3157|Monday|1960Q3|Y|N|N|2437117|2437298|2436754|2437029|N|N|N|N|N| +2437121|AAAAAAAABAADFCAA|1960-07-05|726|3158|243|1960|2|7|5|3|1960|243|3158|Tuesday|1960Q3|N|N|Y|2437117|2437298|2436755|2437030|N|N|N|N|N| +2437122|AAAAAAAACAADFCAA|1960-07-06|726|3158|243|1960|3|7|6|3|1960|243|3158|Wednesday|1960Q3|N|N|N|2437117|2437298|2436756|2437031|N|N|N|N|N| +2437123|AAAAAAAADAADFCAA|1960-07-07|726|3158|243|1960|4|7|7|3|1960|243|3158|Thursday|1960Q3|N|N|N|2437117|2437298|2436757|2437032|N|N|N|N|N| +2437124|AAAAAAAAEAADFCAA|1960-07-08|726|3158|243|1960|5|7|8|3|1960|243|3158|Friday|1960Q3|N|Y|N|2437117|2437298|2436758|2437033|N|N|N|N|N| +2437125|AAAAAAAAFAADFCAA|1960-07-09|726|3158|243|1960|6|7|9|3|1960|243|3158|Saturday|1960Q3|N|Y|N|2437117|2437298|2436759|2437034|N|N|N|N|N| +2437126|AAAAAAAAGAADFCAA|1960-07-10|726|3158|243|1960|0|7|10|3|1960|243|3158|Sunday|1960Q3|N|N|N|2437117|2437298|2436760|2437035|N|N|N|N|N| +2437127|AAAAAAAAHAADFCAA|1960-07-11|726|3158|243|1960|1|7|11|3|1960|243|3158|Monday|1960Q3|N|N|N|2437117|2437298|2436761|2437036|N|N|N|N|N| +2437128|AAAAAAAAIAADFCAA|1960-07-12|726|3159|243|1960|2|7|12|3|1960|243|3159|Tuesday|1960Q3|N|N|N|2437117|2437298|2436762|2437037|N|N|N|N|N| +2437129|AAAAAAAAJAADFCAA|1960-07-13|726|3159|243|1960|3|7|13|3|1960|243|3159|Wednesday|1960Q3|N|N|N|2437117|2437298|2436763|2437038|N|N|N|N|N| +2437130|AAAAAAAAKAADFCAA|1960-07-14|726|3159|243|1960|4|7|14|3|1960|243|3159|Thursday|1960Q3|N|N|N|2437117|2437298|2436764|2437039|N|N|N|N|N| +2437131|AAAAAAAALAADFCAA|1960-07-15|726|3159|243|1960|5|7|15|3|1960|243|3159|Friday|1960Q3|N|Y|N|2437117|2437298|2436765|2437040|N|N|N|N|N| +2437132|AAAAAAAAMAADFCAA|1960-07-16|726|3159|243|1960|6|7|16|3|1960|243|3159|Saturday|1960Q3|N|Y|N|2437117|2437298|2436766|2437041|N|N|N|N|N| +2437133|AAAAAAAANAADFCAA|1960-07-17|726|3159|243|1960|0|7|17|3|1960|243|3159|Sunday|1960Q3|N|N|N|2437117|2437298|2436767|2437042|N|N|N|N|N| +2437134|AAAAAAAAOAADFCAA|1960-07-18|726|3159|243|1960|1|7|18|3|1960|243|3159|Monday|1960Q3|N|N|N|2437117|2437298|2436768|2437043|N|N|N|N|N| +2437135|AAAAAAAAPAADFCAA|1960-07-19|726|3160|243|1960|2|7|19|3|1960|243|3160|Tuesday|1960Q3|N|N|N|2437117|2437298|2436769|2437044|N|N|N|N|N| +2437136|AAAAAAAAABADFCAA|1960-07-20|726|3160|243|1960|3|7|20|3|1960|243|3160|Wednesday|1960Q3|N|N|N|2437117|2437298|2436770|2437045|N|N|N|N|N| +2437137|AAAAAAAABBADFCAA|1960-07-21|726|3160|243|1960|4|7|21|3|1960|243|3160|Thursday|1960Q3|N|N|N|2437117|2437298|2436771|2437046|N|N|N|N|N| +2437138|AAAAAAAACBADFCAA|1960-07-22|726|3160|243|1960|5|7|22|3|1960|243|3160|Friday|1960Q3|N|Y|N|2437117|2437298|2436772|2437047|N|N|N|N|N| +2437139|AAAAAAAADBADFCAA|1960-07-23|726|3160|243|1960|6|7|23|3|1960|243|3160|Saturday|1960Q3|N|Y|N|2437117|2437298|2436773|2437048|N|N|N|N|N| +2437140|AAAAAAAAEBADFCAA|1960-07-24|726|3160|243|1960|0|7|24|3|1960|243|3160|Sunday|1960Q3|N|N|N|2437117|2437298|2436774|2437049|N|N|N|N|N| +2437141|AAAAAAAAFBADFCAA|1960-07-25|726|3160|243|1960|1|7|25|3|1960|243|3160|Monday|1960Q3|N|N|N|2437117|2437298|2436775|2437050|N|N|N|N|N| +2437142|AAAAAAAAGBADFCAA|1960-07-26|726|3161|243|1960|2|7|26|3|1960|243|3161|Tuesday|1960Q3|N|N|N|2437117|2437298|2436776|2437051|N|N|N|N|N| +2437143|AAAAAAAAHBADFCAA|1960-07-27|726|3161|243|1960|3|7|27|3|1960|243|3161|Wednesday|1960Q3|N|N|N|2437117|2437298|2436777|2437052|N|N|N|N|N| +2437144|AAAAAAAAIBADFCAA|1960-07-28|726|3161|243|1960|4|7|28|3|1960|243|3161|Thursday|1960Q3|N|N|N|2437117|2437298|2436778|2437053|N|N|N|N|N| +2437145|AAAAAAAAJBADFCAA|1960-07-29|726|3161|243|1960|5|7|29|3|1960|243|3161|Friday|1960Q3|N|Y|N|2437117|2437298|2436779|2437054|N|N|N|N|N| +2437146|AAAAAAAAKBADFCAA|1960-07-30|726|3161|243|1960|6|7|30|3|1960|243|3161|Saturday|1960Q3|N|Y|N|2437117|2437298|2436780|2437055|N|N|N|N|N| +2437147|AAAAAAAALBADFCAA|1960-07-31|726|3161|243|1960|0|7|31|3|1960|243|3161|Sunday|1960Q3|N|N|N|2437117|2437298|2436781|2437056|N|N|N|N|N| +2437148|AAAAAAAAMBADFCAA|1960-08-01|727|3161|243|1960|1|8|1|3|1960|243|3161|Monday|1960Q3|N|N|N|2437148|2437360|2436782|2437057|N|N|N|N|N| +2437149|AAAAAAAANBADFCAA|1960-08-02|727|3162|243|1960|2|8|2|3|1960|243|3162|Tuesday|1960Q3|N|N|N|2437148|2437360|2436783|2437058|N|N|N|N|N| +2437150|AAAAAAAAOBADFCAA|1960-08-03|727|3162|243|1960|3|8|3|3|1960|243|3162|Wednesday|1960Q3|N|N|N|2437148|2437360|2436784|2437059|N|N|N|N|N| +2437151|AAAAAAAAPBADFCAA|1960-08-04|727|3162|243|1960|4|8|4|3|1960|243|3162|Thursday|1960Q3|N|N|N|2437148|2437360|2436785|2437060|N|N|N|N|N| +2437152|AAAAAAAAACADFCAA|1960-08-05|727|3162|243|1960|5|8|5|3|1960|243|3162|Friday|1960Q3|N|Y|N|2437148|2437360|2436786|2437061|N|N|N|N|N| +2437153|AAAAAAAABCADFCAA|1960-08-06|727|3162|243|1960|6|8|6|3|1960|243|3162|Saturday|1960Q3|N|Y|N|2437148|2437360|2436787|2437062|N|N|N|N|N| +2437154|AAAAAAAACCADFCAA|1960-08-07|727|3162|243|1960|0|8|7|3|1960|243|3162|Sunday|1960Q3|N|N|N|2437148|2437360|2436788|2437063|N|N|N|N|N| +2437155|AAAAAAAADCADFCAA|1960-08-08|727|3162|243|1960|1|8|8|3|1960|243|3162|Monday|1960Q3|N|N|N|2437148|2437360|2436789|2437064|N|N|N|N|N| +2437156|AAAAAAAAECADFCAA|1960-08-09|727|3163|243|1960|2|8|9|3|1960|243|3163|Tuesday|1960Q3|N|N|N|2437148|2437360|2436790|2437065|N|N|N|N|N| +2437157|AAAAAAAAFCADFCAA|1960-08-10|727|3163|243|1960|3|8|10|3|1960|243|3163|Wednesday|1960Q3|N|N|N|2437148|2437360|2436791|2437066|N|N|N|N|N| +2437158|AAAAAAAAGCADFCAA|1960-08-11|727|3163|243|1960|4|8|11|3|1960|243|3163|Thursday|1960Q3|N|N|N|2437148|2437360|2436792|2437067|N|N|N|N|N| +2437159|AAAAAAAAHCADFCAA|1960-08-12|727|3163|243|1960|5|8|12|3|1960|243|3163|Friday|1960Q3|N|Y|N|2437148|2437360|2436793|2437068|N|N|N|N|N| +2437160|AAAAAAAAICADFCAA|1960-08-13|727|3163|243|1960|6|8|13|3|1960|243|3163|Saturday|1960Q3|N|Y|N|2437148|2437360|2436794|2437069|N|N|N|N|N| +2437161|AAAAAAAAJCADFCAA|1960-08-14|727|3163|243|1960|0|8|14|3|1960|243|3163|Sunday|1960Q3|N|N|N|2437148|2437360|2436795|2437070|N|N|N|N|N| +2437162|AAAAAAAAKCADFCAA|1960-08-15|727|3163|243|1960|1|8|15|3|1960|243|3163|Monday|1960Q3|N|N|N|2437148|2437360|2436796|2437071|N|N|N|N|N| +2437163|AAAAAAAALCADFCAA|1960-08-16|727|3164|243|1960|2|8|16|3|1960|243|3164|Tuesday|1960Q3|N|N|N|2437148|2437360|2436797|2437072|N|N|N|N|N| +2437164|AAAAAAAAMCADFCAA|1960-08-17|727|3164|243|1960|3|8|17|3|1960|243|3164|Wednesday|1960Q3|N|N|N|2437148|2437360|2436798|2437073|N|N|N|N|N| +2437165|AAAAAAAANCADFCAA|1960-08-18|727|3164|243|1960|4|8|18|3|1960|243|3164|Thursday|1960Q3|N|N|N|2437148|2437360|2436799|2437074|N|N|N|N|N| +2437166|AAAAAAAAOCADFCAA|1960-08-19|727|3164|243|1960|5|8|19|3|1960|243|3164|Friday|1960Q3|N|Y|N|2437148|2437360|2436800|2437075|N|N|N|N|N| +2437167|AAAAAAAAPCADFCAA|1960-08-20|727|3164|243|1960|6|8|20|3|1960|243|3164|Saturday|1960Q3|N|Y|N|2437148|2437360|2436801|2437076|N|N|N|N|N| +2437168|AAAAAAAAADADFCAA|1960-08-21|727|3164|243|1960|0|8|21|3|1960|243|3164|Sunday|1960Q3|N|N|N|2437148|2437360|2436802|2437077|N|N|N|N|N| +2437169|AAAAAAAABDADFCAA|1960-08-22|727|3164|243|1960|1|8|22|3|1960|243|3164|Monday|1960Q3|N|N|N|2437148|2437360|2436803|2437078|N|N|N|N|N| +2437170|AAAAAAAACDADFCAA|1960-08-23|727|3165|243|1960|2|8|23|3|1960|243|3165|Tuesday|1960Q3|N|N|N|2437148|2437360|2436804|2437079|N|N|N|N|N| +2437171|AAAAAAAADDADFCAA|1960-08-24|727|3165|243|1960|3|8|24|3|1960|243|3165|Wednesday|1960Q3|N|N|N|2437148|2437360|2436805|2437080|N|N|N|N|N| +2437172|AAAAAAAAEDADFCAA|1960-08-25|727|3165|243|1960|4|8|25|3|1960|243|3165|Thursday|1960Q3|N|N|N|2437148|2437360|2436806|2437081|N|N|N|N|N| +2437173|AAAAAAAAFDADFCAA|1960-08-26|727|3165|243|1960|5|8|26|3|1960|243|3165|Friday|1960Q3|N|Y|N|2437148|2437360|2436807|2437082|N|N|N|N|N| +2437174|AAAAAAAAGDADFCAA|1960-08-27|727|3165|243|1960|6|8|27|3|1960|243|3165|Saturday|1960Q3|N|Y|N|2437148|2437360|2436808|2437083|N|N|N|N|N| +2437175|AAAAAAAAHDADFCAA|1960-08-28|727|3165|243|1960|0|8|28|3|1960|243|3165|Sunday|1960Q3|N|N|N|2437148|2437360|2436809|2437084|N|N|N|N|N| +2437176|AAAAAAAAIDADFCAA|1960-08-29|727|3165|243|1960|1|8|29|3|1960|243|3165|Monday|1960Q3|N|N|N|2437148|2437360|2436810|2437085|N|N|N|N|N| +2437177|AAAAAAAAJDADFCAA|1960-08-30|727|3166|243|1960|2|8|30|3|1960|243|3166|Tuesday|1960Q3|N|N|N|2437148|2437360|2436811|2437086|N|N|N|N|N| +2437178|AAAAAAAAKDADFCAA|1960-08-31|727|3166|243|1960|3|8|31|3|1960|243|3166|Wednesday|1960Q3|N|N|N|2437148|2437360|2436812|2437087|N|N|N|N|N| +2437179|AAAAAAAALDADFCAA|1960-09-01|728|3166|244|1960|4|9|1|3|1960|244|3166|Thursday|1960Q3|N|N|N|2437179|2437422|2436813|2437088|N|N|N|N|N| +2437180|AAAAAAAAMDADFCAA|1960-09-02|728|3166|244|1960|5|9|2|3|1960|244|3166|Friday|1960Q3|N|Y|N|2437179|2437422|2436814|2437089|N|N|N|N|N| +2437181|AAAAAAAANDADFCAA|1960-09-03|728|3166|244|1960|6|9|3|3|1960|244|3166|Saturday|1960Q3|N|Y|N|2437179|2437422|2436815|2437090|N|N|N|N|N| +2437182|AAAAAAAAODADFCAA|1960-09-04|728|3166|244|1960|0|9|4|3|1960|244|3166|Sunday|1960Q3|N|N|N|2437179|2437422|2436816|2437091|N|N|N|N|N| +2437183|AAAAAAAAPDADFCAA|1960-09-05|728|3166|244|1960|1|9|5|3|1960|244|3166|Monday|1960Q3|N|N|N|2437179|2437422|2436817|2437092|N|N|N|N|N| +2437184|AAAAAAAAAEADFCAA|1960-09-06|728|3167|244|1960|2|9|6|3|1960|244|3167|Tuesday|1960Q3|N|N|N|2437179|2437422|2436818|2437093|N|N|N|N|N| +2437185|AAAAAAAABEADFCAA|1960-09-07|728|3167|244|1960|3|9|7|3|1960|244|3167|Wednesday|1960Q3|N|N|N|2437179|2437422|2436819|2437094|N|N|N|N|N| +2437186|AAAAAAAACEADFCAA|1960-09-08|728|3167|244|1960|4|9|8|3|1960|244|3167|Thursday|1960Q3|N|N|N|2437179|2437422|2436820|2437095|N|N|N|N|N| +2437187|AAAAAAAADEADFCAA|1960-09-09|728|3167|244|1960|5|9|9|3|1960|244|3167|Friday|1960Q3|N|Y|N|2437179|2437422|2436821|2437096|N|N|N|N|N| +2437188|AAAAAAAAEEADFCAA|1960-09-10|728|3167|244|1960|6|9|10|3|1960|244|3167|Saturday|1960Q3|N|Y|N|2437179|2437422|2436822|2437097|N|N|N|N|N| +2437189|AAAAAAAAFEADFCAA|1960-09-11|728|3167|244|1960|0|9|11|3|1960|244|3167|Sunday|1960Q3|N|N|N|2437179|2437422|2436823|2437098|N|N|N|N|N| +2437190|AAAAAAAAGEADFCAA|1960-09-12|728|3167|244|1960|1|9|12|3|1960|244|3167|Monday|1960Q3|N|N|N|2437179|2437422|2436824|2437099|N|N|N|N|N| +2437191|AAAAAAAAHEADFCAA|1960-09-13|728|3168|244|1960|2|9|13|3|1960|244|3168|Tuesday|1960Q3|N|N|N|2437179|2437422|2436825|2437100|N|N|N|N|N| +2437192|AAAAAAAAIEADFCAA|1960-09-14|728|3168|244|1960|3|9|14|3|1960|244|3168|Wednesday|1960Q3|N|N|N|2437179|2437422|2436826|2437101|N|N|N|N|N| +2437193|AAAAAAAAJEADFCAA|1960-09-15|728|3168|244|1960|4|9|15|3|1960|244|3168|Thursday|1960Q3|N|N|N|2437179|2437422|2436827|2437102|N|N|N|N|N| +2437194|AAAAAAAAKEADFCAA|1960-09-16|728|3168|244|1960|5|9|16|3|1960|244|3168|Friday|1960Q3|N|Y|N|2437179|2437422|2436828|2437103|N|N|N|N|N| +2437195|AAAAAAAALEADFCAA|1960-09-17|728|3168|244|1960|6|9|17|3|1960|244|3168|Saturday|1960Q3|N|Y|N|2437179|2437422|2436829|2437104|N|N|N|N|N| +2437196|AAAAAAAAMEADFCAA|1960-09-18|728|3168|244|1960|0|9|18|3|1960|244|3168|Sunday|1960Q3|N|N|N|2437179|2437422|2436830|2437105|N|N|N|N|N| +2437197|AAAAAAAANEADFCAA|1960-09-19|728|3168|244|1960|1|9|19|3|1960|244|3168|Monday|1960Q3|N|N|N|2437179|2437422|2436831|2437106|N|N|N|N|N| +2437198|AAAAAAAAOEADFCAA|1960-09-20|728|3169|244|1960|2|9|20|3|1960|244|3169|Tuesday|1960Q3|N|N|N|2437179|2437422|2436832|2437107|N|N|N|N|N| +2437199|AAAAAAAAPEADFCAA|1960-09-21|728|3169|244|1960|3|9|21|3|1960|244|3169|Wednesday|1960Q3|N|N|N|2437179|2437422|2436833|2437108|N|N|N|N|N| +2437200|AAAAAAAAAFADFCAA|1960-09-22|728|3169|244|1960|4|9|22|3|1960|244|3169|Thursday|1960Q3|N|N|N|2437179|2437422|2436834|2437109|N|N|N|N|N| +2437201|AAAAAAAABFADFCAA|1960-09-23|728|3169|244|1960|5|9|23|3|1960|244|3169|Friday|1960Q3|N|Y|N|2437179|2437422|2436835|2437110|N|N|N|N|N| +2437202|AAAAAAAACFADFCAA|1960-09-24|728|3169|244|1960|6|9|24|3|1960|244|3169|Saturday|1960Q3|N|Y|N|2437179|2437422|2436836|2437111|N|N|N|N|N| +2437203|AAAAAAAADFADFCAA|1960-09-25|728|3169|244|1960|0|9|25|3|1960|244|3169|Sunday|1960Q3|N|N|N|2437179|2437422|2436837|2437112|N|N|N|N|N| +2437204|AAAAAAAAEFADFCAA|1960-09-26|728|3169|244|1960|1|9|26|3|1960|244|3169|Monday|1960Q3|N|N|N|2437179|2437422|2436838|2437113|N|N|N|N|N| +2437205|AAAAAAAAFFADFCAA|1960-09-27|728|3170|244|1960|2|9|27|3|1960|244|3170|Tuesday|1960Q3|N|N|N|2437179|2437422|2436839|2437114|N|N|N|N|N| +2437206|AAAAAAAAGFADFCAA|1960-09-28|728|3170|244|1960|3|9|28|3|1960|244|3170|Wednesday|1960Q3|N|N|N|2437179|2437422|2436840|2437115|N|N|N|N|N| +2437207|AAAAAAAAHFADFCAA|1960-09-29|728|3170|244|1960|4|9|29|3|1960|244|3170|Thursday|1960Q3|N|N|N|2437179|2437422|2436841|2437116|N|N|N|N|N| +2437208|AAAAAAAAIFADFCAA|1960-09-30|728|3170|244|1960|5|9|30|3|1960|244|3170|Friday|1960Q3|N|Y|N|2437179|2437422|2436842|2437117|N|N|N|N|N| +2437209|AAAAAAAAJFADFCAA|1960-10-01|729|3170|244|1960|6|10|1|4|1960|244|3170|Saturday|1960Q4|N|Y|N|2437209|2437482|2436843|2437117|N|N|N|N|N| +2437210|AAAAAAAAKFADFCAA|1960-10-02|729|3170|244|1960|0|10|2|4|1960|244|3170|Sunday|1960Q4|N|N|N|2437209|2437482|2436844|2437118|N|N|N|N|N| +2437211|AAAAAAAALFADFCAA|1960-10-03|729|3170|244|1960|1|10|3|4|1960|244|3170|Monday|1960Q4|N|N|N|2437209|2437482|2436845|2437119|N|N|N|N|N| +2437212|AAAAAAAAMFADFCAA|1960-10-04|729|3171|244|1960|2|10|4|4|1960|244|3171|Tuesday|1960Q4|N|N|N|2437209|2437482|2436846|2437120|N|N|N|N|N| +2437213|AAAAAAAANFADFCAA|1960-10-05|729|3171|244|1960|3|10|5|4|1960|244|3171|Wednesday|1960Q4|N|N|N|2437209|2437482|2436847|2437121|N|N|N|N|N| +2437214|AAAAAAAAOFADFCAA|1960-10-06|729|3171|244|1960|4|10|6|4|1960|244|3171|Thursday|1960Q4|N|N|N|2437209|2437482|2436848|2437122|N|N|N|N|N| +2437215|AAAAAAAAPFADFCAA|1960-10-07|729|3171|244|1960|5|10|7|4|1960|244|3171|Friday|1960Q4|N|Y|N|2437209|2437482|2436849|2437123|N|N|N|N|N| +2437216|AAAAAAAAAGADFCAA|1960-10-08|729|3171|244|1960|6|10|8|4|1960|244|3171|Saturday|1960Q4|N|Y|N|2437209|2437482|2436850|2437124|N|N|N|N|N| +2437217|AAAAAAAABGADFCAA|1960-10-09|729|3171|244|1960|0|10|9|4|1960|244|3171|Sunday|1960Q4|N|N|N|2437209|2437482|2436851|2437125|N|N|N|N|N| +2437218|AAAAAAAACGADFCAA|1960-10-10|729|3171|244|1960|1|10|10|4|1960|244|3171|Monday|1960Q4|N|N|N|2437209|2437482|2436852|2437126|N|N|N|N|N| +2437219|AAAAAAAADGADFCAA|1960-10-11|729|3172|244|1960|2|10|11|4|1960|244|3172|Tuesday|1960Q4|N|N|N|2437209|2437482|2436853|2437127|N|N|N|N|N| +2437220|AAAAAAAAEGADFCAA|1960-10-12|729|3172|244|1960|3|10|12|4|1960|244|3172|Wednesday|1960Q4|N|N|N|2437209|2437482|2436854|2437128|N|N|N|N|N| +2437221|AAAAAAAAFGADFCAA|1960-10-13|729|3172|244|1960|4|10|13|4|1960|244|3172|Thursday|1960Q4|N|N|N|2437209|2437482|2436855|2437129|N|N|N|N|N| +2437222|AAAAAAAAGGADFCAA|1960-10-14|729|3172|244|1960|5|10|14|4|1960|244|3172|Friday|1960Q4|N|Y|N|2437209|2437482|2436856|2437130|N|N|N|N|N| +2437223|AAAAAAAAHGADFCAA|1960-10-15|729|3172|244|1960|6|10|15|4|1960|244|3172|Saturday|1960Q4|N|Y|N|2437209|2437482|2436857|2437131|N|N|N|N|N| +2437224|AAAAAAAAIGADFCAA|1960-10-16|729|3172|244|1960|0|10|16|4|1960|244|3172|Sunday|1960Q4|N|N|N|2437209|2437482|2436858|2437132|N|N|N|N|N| +2437225|AAAAAAAAJGADFCAA|1960-10-17|729|3172|244|1960|1|10|17|4|1960|244|3172|Monday|1960Q4|N|N|N|2437209|2437482|2436859|2437133|N|N|N|N|N| +2437226|AAAAAAAAKGADFCAA|1960-10-18|729|3173|244|1960|2|10|18|4|1960|244|3173|Tuesday|1960Q4|N|N|N|2437209|2437482|2436860|2437134|N|N|N|N|N| +2437227|AAAAAAAALGADFCAA|1960-10-19|729|3173|244|1960|3|10|19|4|1960|244|3173|Wednesday|1960Q4|N|N|N|2437209|2437482|2436861|2437135|N|N|N|N|N| +2437228|AAAAAAAAMGADFCAA|1960-10-20|729|3173|244|1960|4|10|20|4|1960|244|3173|Thursday|1960Q4|N|N|N|2437209|2437482|2436862|2437136|N|N|N|N|N| +2437229|AAAAAAAANGADFCAA|1960-10-21|729|3173|244|1960|5|10|21|4|1960|244|3173|Friday|1960Q4|N|Y|N|2437209|2437482|2436863|2437137|N|N|N|N|N| +2437230|AAAAAAAAOGADFCAA|1960-10-22|729|3173|244|1960|6|10|22|4|1960|244|3173|Saturday|1960Q4|N|Y|N|2437209|2437482|2436864|2437138|N|N|N|N|N| +2437231|AAAAAAAAPGADFCAA|1960-10-23|729|3173|244|1960|0|10|23|4|1960|244|3173|Sunday|1960Q4|N|N|N|2437209|2437482|2436865|2437139|N|N|N|N|N| +2437232|AAAAAAAAAHADFCAA|1960-10-24|729|3173|244|1960|1|10|24|4|1960|244|3173|Monday|1960Q4|N|N|N|2437209|2437482|2436866|2437140|N|N|N|N|N| +2437233|AAAAAAAABHADFCAA|1960-10-25|729|3174|244|1960|2|10|25|4|1960|244|3174|Tuesday|1960Q4|N|N|N|2437209|2437482|2436867|2437141|N|N|N|N|N| +2437234|AAAAAAAACHADFCAA|1960-10-26|729|3174|244|1960|3|10|26|4|1960|244|3174|Wednesday|1960Q4|N|N|N|2437209|2437482|2436868|2437142|N|N|N|N|N| +2437235|AAAAAAAADHADFCAA|1960-10-27|729|3174|244|1960|4|10|27|4|1960|244|3174|Thursday|1960Q4|N|N|N|2437209|2437482|2436869|2437143|N|N|N|N|N| +2437236|AAAAAAAAEHADFCAA|1960-10-28|729|3174|244|1960|5|10|28|4|1960|244|3174|Friday|1960Q4|N|Y|N|2437209|2437482|2436870|2437144|N|N|N|N|N| +2437237|AAAAAAAAFHADFCAA|1960-10-29|729|3174|244|1960|6|10|29|4|1960|244|3174|Saturday|1960Q4|N|Y|N|2437209|2437482|2436871|2437145|N|N|N|N|N| +2437238|AAAAAAAAGHADFCAA|1960-10-30|729|3174|244|1960|0|10|30|4|1960|244|3174|Sunday|1960Q4|N|N|N|2437209|2437482|2436872|2437146|N|N|N|N|N| +2437239|AAAAAAAAHHADFCAA|1960-10-31|729|3174|244|1960|1|10|31|4|1960|244|3174|Monday|1960Q4|N|N|N|2437209|2437482|2436873|2437147|N|N|N|N|N| +2437240|AAAAAAAAIHADFCAA|1960-11-01|730|3175|244|1960|2|11|1|4|1960|244|3175|Tuesday|1960Q4|N|N|N|2437240|2437544|2436874|2437148|N|N|N|N|N| +2437241|AAAAAAAAJHADFCAA|1960-11-02|730|3175|244|1960|3|11|2|4|1960|244|3175|Wednesday|1960Q4|N|N|N|2437240|2437544|2436875|2437149|N|N|N|N|N| +2437242|AAAAAAAAKHADFCAA|1960-11-03|730|3175|244|1960|4|11|3|4|1960|244|3175|Thursday|1960Q4|N|N|N|2437240|2437544|2436876|2437150|N|N|N|N|N| +2437243|AAAAAAAALHADFCAA|1960-11-04|730|3175|244|1960|5|11|4|4|1960|244|3175|Friday|1960Q4|N|Y|N|2437240|2437544|2436877|2437151|N|N|N|N|N| +2437244|AAAAAAAAMHADFCAA|1960-11-05|730|3175|244|1960|6|11|5|4|1960|244|3175|Saturday|1960Q4|N|Y|N|2437240|2437544|2436878|2437152|N|N|N|N|N| +2437245|AAAAAAAANHADFCAA|1960-11-06|730|3175|244|1960|0|11|6|4|1960|244|3175|Sunday|1960Q4|N|N|N|2437240|2437544|2436879|2437153|N|N|N|N|N| +2437246|AAAAAAAAOHADFCAA|1960-11-07|730|3175|244|1960|1|11|7|4|1960|244|3175|Monday|1960Q4|N|N|N|2437240|2437544|2436880|2437154|N|N|N|N|N| +2437247|AAAAAAAAPHADFCAA|1960-11-08|730|3176|244|1960|2|11|8|4|1960|244|3176|Tuesday|1960Q4|N|N|N|2437240|2437544|2436881|2437155|N|N|N|N|N| +2437248|AAAAAAAAAIADFCAA|1960-11-09|730|3176|244|1960|3|11|9|4|1960|244|3176|Wednesday|1960Q4|N|N|N|2437240|2437544|2436882|2437156|N|N|N|N|N| +2437249|AAAAAAAABIADFCAA|1960-11-10|730|3176|244|1960|4|11|10|4|1960|244|3176|Thursday|1960Q4|N|N|N|2437240|2437544|2436883|2437157|N|N|N|N|N| +2437250|AAAAAAAACIADFCAA|1960-11-11|730|3176|244|1960|5|11|11|4|1960|244|3176|Friday|1960Q4|N|Y|N|2437240|2437544|2436884|2437158|N|N|N|N|N| +2437251|AAAAAAAADIADFCAA|1960-11-12|730|3176|244|1960|6|11|12|4|1960|244|3176|Saturday|1960Q4|N|Y|N|2437240|2437544|2436885|2437159|N|N|N|N|N| +2437252|AAAAAAAAEIADFCAA|1960-11-13|730|3176|244|1960|0|11|13|4|1960|244|3176|Sunday|1960Q4|N|N|N|2437240|2437544|2436886|2437160|N|N|N|N|N| +2437253|AAAAAAAAFIADFCAA|1960-11-14|730|3176|244|1960|1|11|14|4|1960|244|3176|Monday|1960Q4|N|N|N|2437240|2437544|2436887|2437161|N|N|N|N|N| +2437254|AAAAAAAAGIADFCAA|1960-11-15|730|3177|244|1960|2|11|15|4|1960|244|3177|Tuesday|1960Q4|N|N|N|2437240|2437544|2436888|2437162|N|N|N|N|N| +2437255|AAAAAAAAHIADFCAA|1960-11-16|730|3177|244|1960|3|11|16|4|1960|244|3177|Wednesday|1960Q4|N|N|N|2437240|2437544|2436889|2437163|N|N|N|N|N| +2437256|AAAAAAAAIIADFCAA|1960-11-17|730|3177|244|1960|4|11|17|4|1960|244|3177|Thursday|1960Q4|N|N|N|2437240|2437544|2436890|2437164|N|N|N|N|N| +2437257|AAAAAAAAJIADFCAA|1960-11-18|730|3177|244|1960|5|11|18|4|1960|244|3177|Friday|1960Q4|N|Y|N|2437240|2437544|2436891|2437165|N|N|N|N|N| +2437258|AAAAAAAAKIADFCAA|1960-11-19|730|3177|244|1960|6|11|19|4|1960|244|3177|Saturday|1960Q4|N|Y|N|2437240|2437544|2436892|2437166|N|N|N|N|N| +2437259|AAAAAAAALIADFCAA|1960-11-20|730|3177|244|1960|0|11|20|4|1960|244|3177|Sunday|1960Q4|N|N|N|2437240|2437544|2436893|2437167|N|N|N|N|N| +2437260|AAAAAAAAMIADFCAA|1960-11-21|730|3177|244|1960|1|11|21|4|1960|244|3177|Monday|1960Q4|N|N|N|2437240|2437544|2436894|2437168|N|N|N|N|N| +2437261|AAAAAAAANIADFCAA|1960-11-22|730|3178|244|1960|2|11|22|4|1960|244|3178|Tuesday|1960Q4|N|N|N|2437240|2437544|2436895|2437169|N|N|N|N|N| +2437262|AAAAAAAAOIADFCAA|1960-11-23|730|3178|244|1960|3|11|23|4|1960|244|3178|Wednesday|1960Q4|N|N|N|2437240|2437544|2436896|2437170|N|N|N|N|N| +2437263|AAAAAAAAPIADFCAA|1960-11-24|730|3178|244|1960|4|11|24|4|1960|244|3178|Thursday|1960Q4|N|N|N|2437240|2437544|2436897|2437171|N|N|N|N|N| +2437264|AAAAAAAAAJADFCAA|1960-11-25|730|3178|244|1960|5|11|25|4|1960|244|3178|Friday|1960Q4|N|Y|N|2437240|2437544|2436898|2437172|N|N|N|N|N| +2437265|AAAAAAAABJADFCAA|1960-11-26|730|3178|244|1960|6|11|26|4|1960|244|3178|Saturday|1960Q4|N|Y|N|2437240|2437544|2436899|2437173|N|N|N|N|N| +2437266|AAAAAAAACJADFCAA|1960-11-27|730|3178|244|1960|0|11|27|4|1960|244|3178|Sunday|1960Q4|N|N|N|2437240|2437544|2436900|2437174|N|N|N|N|N| +2437267|AAAAAAAADJADFCAA|1960-11-28|730|3178|244|1960|1|11|28|4|1960|244|3178|Monday|1960Q4|N|N|N|2437240|2437544|2436901|2437175|N|N|N|N|N| +2437268|AAAAAAAAEJADFCAA|1960-11-29|730|3179|244|1960|2|11|29|4|1960|244|3179|Tuesday|1960Q4|N|N|N|2437240|2437544|2436902|2437176|N|N|N|N|N| +2437269|AAAAAAAAFJADFCAA|1960-11-30|730|3179|244|1960|3|11|30|4|1960|244|3179|Wednesday|1960Q4|N|N|N|2437240|2437544|2436903|2437177|N|N|N|N|N| +2437270|AAAAAAAAGJADFCAA|1960-12-01|731|3179|245|1960|4|12|1|4|1960|245|3179|Thursday|1960Q4|N|N|N|2437270|2437604|2436904|2437178|N|N|N|N|N| +2437271|AAAAAAAAHJADFCAA|1960-12-02|731|3179|245|1960|5|12|2|4|1960|245|3179|Friday|1960Q4|N|Y|N|2437270|2437604|2436905|2437179|N|N|N|N|N| +2437272|AAAAAAAAIJADFCAA|1960-12-03|731|3179|245|1960|6|12|3|4|1960|245|3179|Saturday|1960Q4|N|Y|N|2437270|2437604|2436906|2437180|N|N|N|N|N| +2437273|AAAAAAAAJJADFCAA|1960-12-04|731|3179|245|1960|0|12|4|4|1960|245|3179|Sunday|1960Q4|N|N|N|2437270|2437604|2436907|2437181|N|N|N|N|N| +2437274|AAAAAAAAKJADFCAA|1960-12-05|731|3179|245|1960|1|12|5|4|1960|245|3179|Monday|1960Q4|N|N|N|2437270|2437604|2436908|2437182|N|N|N|N|N| +2437275|AAAAAAAALJADFCAA|1960-12-06|731|3180|245|1960|2|12|6|4|1960|245|3180|Tuesday|1960Q4|N|N|N|2437270|2437604|2436909|2437183|N|N|N|N|N| +2437276|AAAAAAAAMJADFCAA|1960-12-07|731|3180|245|1960|3|12|7|4|1960|245|3180|Wednesday|1960Q4|N|N|N|2437270|2437604|2436910|2437184|N|N|N|N|N| +2437277|AAAAAAAANJADFCAA|1960-12-08|731|3180|245|1960|4|12|8|4|1960|245|3180|Thursday|1960Q4|N|N|N|2437270|2437604|2436911|2437185|N|N|N|N|N| +2437278|AAAAAAAAOJADFCAA|1960-12-09|731|3180|245|1960|5|12|9|4|1960|245|3180|Friday|1960Q4|N|Y|N|2437270|2437604|2436912|2437186|N|N|N|N|N| +2437279|AAAAAAAAPJADFCAA|1960-12-10|731|3180|245|1960|6|12|10|4|1960|245|3180|Saturday|1960Q4|N|Y|N|2437270|2437604|2436913|2437187|N|N|N|N|N| +2437280|AAAAAAAAAKADFCAA|1960-12-11|731|3180|245|1960|0|12|11|4|1960|245|3180|Sunday|1960Q4|N|N|N|2437270|2437604|2436914|2437188|N|N|N|N|N| +2437281|AAAAAAAABKADFCAA|1960-12-12|731|3180|245|1960|1|12|12|4|1960|245|3180|Monday|1960Q4|N|N|N|2437270|2437604|2436915|2437189|N|N|N|N|N| +2437282|AAAAAAAACKADFCAA|1960-12-13|731|3181|245|1960|2|12|13|4|1960|245|3181|Tuesday|1960Q4|N|N|N|2437270|2437604|2436916|2437190|N|N|N|N|N| +2437283|AAAAAAAADKADFCAA|1960-12-14|731|3181|245|1960|3|12|14|4|1960|245|3181|Wednesday|1960Q4|N|N|N|2437270|2437604|2436917|2437191|N|N|N|N|N| +2437284|AAAAAAAAEKADFCAA|1960-12-15|731|3181|245|1960|4|12|15|4|1960|245|3181|Thursday|1960Q4|N|N|N|2437270|2437604|2436918|2437192|N|N|N|N|N| +2437285|AAAAAAAAFKADFCAA|1960-12-16|731|3181|245|1960|5|12|16|4|1960|245|3181|Friday|1960Q4|N|Y|N|2437270|2437604|2436919|2437193|N|N|N|N|N| +2437286|AAAAAAAAGKADFCAA|1960-12-17|731|3181|245|1960|6|12|17|4|1960|245|3181|Saturday|1960Q4|N|Y|N|2437270|2437604|2436920|2437194|N|N|N|N|N| +2437287|AAAAAAAAHKADFCAA|1960-12-18|731|3181|245|1960|0|12|18|4|1960|245|3181|Sunday|1960Q4|N|N|N|2437270|2437604|2436921|2437195|N|N|N|N|N| +2437288|AAAAAAAAIKADFCAA|1960-12-19|731|3181|245|1960|1|12|19|4|1960|245|3181|Monday|1960Q4|N|N|N|2437270|2437604|2436922|2437196|N|N|N|N|N| +2437289|AAAAAAAAJKADFCAA|1960-12-20|731|3182|245|1960|2|12|20|4|1960|245|3182|Tuesday|1960Q4|N|N|N|2437270|2437604|2436923|2437197|N|N|N|N|N| +2437290|AAAAAAAAKKADFCAA|1960-12-21|731|3182|245|1960|3|12|21|4|1960|245|3182|Wednesday|1960Q4|N|N|N|2437270|2437604|2436924|2437198|N|N|N|N|N| +2437291|AAAAAAAALKADFCAA|1960-12-22|731|3182|245|1960|4|12|22|4|1960|245|3182|Thursday|1960Q4|N|N|N|2437270|2437604|2436925|2437199|N|N|N|N|N| +2437292|AAAAAAAAMKADFCAA|1960-12-23|731|3182|245|1960|5|12|23|4|1960|245|3182|Friday|1960Q4|N|Y|N|2437270|2437604|2436926|2437200|N|N|N|N|N| +2437293|AAAAAAAANKADFCAA|1960-12-24|731|3182|245|1960|6|12|24|4|1960|245|3182|Saturday|1960Q4|N|Y|N|2437270|2437604|2436927|2437201|N|N|N|N|N| +2437294|AAAAAAAAOKADFCAA|1960-12-25|731|3182|245|1960|0|12|25|4|1960|245|3182|Sunday|1960Q4|Y|N|N|2437270|2437604|2436928|2437202|N|N|N|N|N| +2437295|AAAAAAAAPKADFCAA|1960-12-26|731|3182|245|1960|1|12|26|4|1960|245|3182|Monday|1960Q4|N|N|Y|2437270|2437604|2436929|2437203|N|N|N|N|N| +2437296|AAAAAAAAALADFCAA|1960-12-27|731|3183|245|1960|2|12|27|4|1960|245|3183|Tuesday|1960Q4|N|N|N|2437270|2437604|2436930|2437204|N|N|N|N|N| +2437297|AAAAAAAABLADFCAA|1960-12-28|731|3183|245|1960|3|12|28|4|1960|245|3183|Wednesday|1960Q4|N|N|N|2437270|2437604|2436931|2437205|N|N|N|N|N| +2437298|AAAAAAAACLADFCAA|1960-12-29|731|3183|245|1960|4|12|29|4|1960|245|3183|Thursday|1960Q4|N|N|N|2437270|2437604|2436932|2437206|N|N|N|N|N| +2437299|AAAAAAAADLADFCAA|1960-12-30|731|3183|245|1960|5|12|30|4|1960|245|3183|Friday|1960Q4|N|Y|N|2437270|2437604|2436933|2437207|N|N|N|N|N| +2437300|AAAAAAAAELADFCAA|1960-12-31|731|3183|245|1960|6|12|31|4|1960|245|3183|Saturday|1960Q4|Y|Y|N|2437270|2437604|2436934|2437208|N|N|N|N|N| +2437301|AAAAAAAAFLADFCAA|1961-01-01|732|3183|245|1961|0|1|1|1|1961|245|3183|Sunday|1961Q1|Y|N|Y|2437301|2437300|2436935|2437209|N|N|N|N|N| +2437302|AAAAAAAAGLADFCAA|1961-01-02|732|3183|245|1961|1|1|2|1|1961|245|3183|Monday|1961Q1|N|N|Y|2437301|2437300|2436936|2437210|N|N|N|N|N| +2437303|AAAAAAAAHLADFCAA|1961-01-03|732|3184|245|1961|2|1|3|1|1961|245|3184|Tuesday|1961Q1|N|N|N|2437301|2437300|2436937|2437211|N|N|N|N|N| +2437304|AAAAAAAAILADFCAA|1961-01-04|732|3184|245|1961|3|1|4|1|1961|245|3184|Wednesday|1961Q1|N|N|N|2437301|2437300|2436938|2437212|N|N|N|N|N| +2437305|AAAAAAAAJLADFCAA|1961-01-05|732|3184|245|1961|4|1|5|1|1961|245|3184|Thursday|1961Q1|N|N|N|2437301|2437300|2436939|2437213|N|N|N|N|N| +2437306|AAAAAAAAKLADFCAA|1961-01-06|732|3184|245|1961|5|1|6|1|1961|245|3184|Friday|1961Q1|N|Y|N|2437301|2437300|2436940|2437214|N|N|N|N|N| +2437307|AAAAAAAALLADFCAA|1961-01-07|732|3184|245|1961|6|1|7|1|1961|245|3184|Saturday|1961Q1|N|Y|N|2437301|2437300|2436941|2437215|N|N|N|N|N| +2437308|AAAAAAAAMLADFCAA|1961-01-08|732|3184|245|1961|0|1|8|1|1961|245|3184|Sunday|1961Q1|N|N|N|2437301|2437300|2436942|2437216|N|N|N|N|N| +2437309|AAAAAAAANLADFCAA|1961-01-09|732|3184|245|1961|1|1|9|1|1961|245|3184|Monday|1961Q1|N|N|N|2437301|2437300|2436943|2437217|N|N|N|N|N| +2437310|AAAAAAAAOLADFCAA|1961-01-10|732|3185|245|1961|2|1|10|1|1961|245|3185|Tuesday|1961Q1|N|N|N|2437301|2437300|2436944|2437218|N|N|N|N|N| +2437311|AAAAAAAAPLADFCAA|1961-01-11|732|3185|245|1961|3|1|11|1|1961|245|3185|Wednesday|1961Q1|N|N|N|2437301|2437300|2436945|2437219|N|N|N|N|N| +2437312|AAAAAAAAAMADFCAA|1961-01-12|732|3185|245|1961|4|1|12|1|1961|245|3185|Thursday|1961Q1|N|N|N|2437301|2437300|2436946|2437220|N|N|N|N|N| +2437313|AAAAAAAABMADFCAA|1961-01-13|732|3185|245|1961|5|1|13|1|1961|245|3185|Friday|1961Q1|N|Y|N|2437301|2437300|2436947|2437221|N|N|N|N|N| +2437314|AAAAAAAACMADFCAA|1961-01-14|732|3185|245|1961|6|1|14|1|1961|245|3185|Saturday|1961Q1|N|Y|N|2437301|2437300|2436948|2437222|N|N|N|N|N| +2437315|AAAAAAAADMADFCAA|1961-01-15|732|3185|245|1961|0|1|15|1|1961|245|3185|Sunday|1961Q1|N|N|N|2437301|2437300|2436949|2437223|N|N|N|N|N| +2437316|AAAAAAAAEMADFCAA|1961-01-16|732|3185|245|1961|1|1|16|1|1961|245|3185|Monday|1961Q1|N|N|N|2437301|2437300|2436950|2437224|N|N|N|N|N| +2437317|AAAAAAAAFMADFCAA|1961-01-17|732|3186|245|1961|2|1|17|1|1961|245|3186|Tuesday|1961Q1|N|N|N|2437301|2437300|2436951|2437225|N|N|N|N|N| +2437318|AAAAAAAAGMADFCAA|1961-01-18|732|3186|245|1961|3|1|18|1|1961|245|3186|Wednesday|1961Q1|N|N|N|2437301|2437300|2436952|2437226|N|N|N|N|N| +2437319|AAAAAAAAHMADFCAA|1961-01-19|732|3186|245|1961|4|1|19|1|1961|245|3186|Thursday|1961Q1|N|N|N|2437301|2437300|2436953|2437227|N|N|N|N|N| +2437320|AAAAAAAAIMADFCAA|1961-01-20|732|3186|245|1961|5|1|20|1|1961|245|3186|Friday|1961Q1|N|Y|N|2437301|2437300|2436954|2437228|N|N|N|N|N| +2437321|AAAAAAAAJMADFCAA|1961-01-21|732|3186|245|1961|6|1|21|1|1961|245|3186|Saturday|1961Q1|N|Y|N|2437301|2437300|2436955|2437229|N|N|N|N|N| +2437322|AAAAAAAAKMADFCAA|1961-01-22|732|3186|245|1961|0|1|22|1|1961|245|3186|Sunday|1961Q1|N|N|N|2437301|2437300|2436956|2437230|N|N|N|N|N| +2437323|AAAAAAAALMADFCAA|1961-01-23|732|3186|245|1961|1|1|23|1|1961|245|3186|Monday|1961Q1|N|N|N|2437301|2437300|2436957|2437231|N|N|N|N|N| +2437324|AAAAAAAAMMADFCAA|1961-01-24|732|3187|245|1961|2|1|24|1|1961|245|3187|Tuesday|1961Q1|N|N|N|2437301|2437300|2436958|2437232|N|N|N|N|N| +2437325|AAAAAAAANMADFCAA|1961-01-25|732|3187|245|1961|3|1|25|1|1961|245|3187|Wednesday|1961Q1|N|N|N|2437301|2437300|2436959|2437233|N|N|N|N|N| +2437326|AAAAAAAAOMADFCAA|1961-01-26|732|3187|245|1961|4|1|26|1|1961|245|3187|Thursday|1961Q1|N|N|N|2437301|2437300|2436960|2437234|N|N|N|N|N| +2437327|AAAAAAAAPMADFCAA|1961-01-27|732|3187|245|1961|5|1|27|1|1961|245|3187|Friday|1961Q1|N|Y|N|2437301|2437300|2436961|2437235|N|N|N|N|N| +2437328|AAAAAAAAANADFCAA|1961-01-28|732|3187|245|1961|6|1|28|1|1961|245|3187|Saturday|1961Q1|N|Y|N|2437301|2437300|2436962|2437236|N|N|N|N|N| +2437329|AAAAAAAABNADFCAA|1961-01-29|732|3187|245|1961|0|1|29|1|1961|245|3187|Sunday|1961Q1|N|N|N|2437301|2437300|2436963|2437237|N|N|N|N|N| +2437330|AAAAAAAACNADFCAA|1961-01-30|732|3187|245|1961|1|1|30|1|1961|245|3187|Monday|1961Q1|N|N|N|2437301|2437300|2436964|2437238|N|N|N|N|N| +2437331|AAAAAAAADNADFCAA|1961-01-31|732|3188|245|1961|2|1|31|1|1961|245|3188|Tuesday|1961Q1|N|N|N|2437301|2437300|2436965|2437239|N|N|N|N|N| +2437332|AAAAAAAAENADFCAA|1961-02-01|733|3188|245|1961|3|2|1|1|1961|245|3188|Wednesday|1961Q1|N|N|N|2437332|2437362|2436966|2437240|N|N|N|N|N| +2437333|AAAAAAAAFNADFCAA|1961-02-02|733|3188|245|1961|4|2|2|1|1961|245|3188|Thursday|1961Q1|N|N|N|2437332|2437362|2436967|2437241|N|N|N|N|N| +2437334|AAAAAAAAGNADFCAA|1961-02-03|733|3188|245|1961|5|2|3|1|1961|245|3188|Friday|1961Q1|N|Y|N|2437332|2437362|2436968|2437242|N|N|N|N|N| +2437335|AAAAAAAAHNADFCAA|1961-02-04|733|3188|245|1961|6|2|4|1|1961|245|3188|Saturday|1961Q1|N|Y|N|2437332|2437362|2436969|2437243|N|N|N|N|N| +2437336|AAAAAAAAINADFCAA|1961-02-05|733|3188|245|1961|0|2|5|1|1961|245|3188|Sunday|1961Q1|N|N|N|2437332|2437362|2436970|2437244|N|N|N|N|N| +2437337|AAAAAAAAJNADFCAA|1961-02-06|733|3188|245|1961|1|2|6|1|1961|245|3188|Monday|1961Q1|N|N|N|2437332|2437362|2436971|2437245|N|N|N|N|N| +2437338|AAAAAAAAKNADFCAA|1961-02-07|733|3189|245|1961|2|2|7|1|1961|245|3189|Tuesday|1961Q1|N|N|N|2437332|2437362|2436972|2437246|N|N|N|N|N| +2437339|AAAAAAAALNADFCAA|1961-02-08|733|3189|245|1961|3|2|8|1|1961|245|3189|Wednesday|1961Q1|N|N|N|2437332|2437362|2436973|2437247|N|N|N|N|N| +2437340|AAAAAAAAMNADFCAA|1961-02-09|733|3189|245|1961|4|2|9|1|1961|245|3189|Thursday|1961Q1|N|N|N|2437332|2437362|2436974|2437248|N|N|N|N|N| +2437341|AAAAAAAANNADFCAA|1961-02-10|733|3189|245|1961|5|2|10|1|1961|245|3189|Friday|1961Q1|N|Y|N|2437332|2437362|2436975|2437249|N|N|N|N|N| +2437342|AAAAAAAAONADFCAA|1961-02-11|733|3189|245|1961|6|2|11|1|1961|245|3189|Saturday|1961Q1|N|Y|N|2437332|2437362|2436976|2437250|N|N|N|N|N| +2437343|AAAAAAAAPNADFCAA|1961-02-12|733|3189|245|1961|0|2|12|1|1961|245|3189|Sunday|1961Q1|N|N|N|2437332|2437362|2436977|2437251|N|N|N|N|N| +2437344|AAAAAAAAAOADFCAA|1961-02-13|733|3189|245|1961|1|2|13|1|1961|245|3189|Monday|1961Q1|N|N|N|2437332|2437362|2436978|2437252|N|N|N|N|N| +2437345|AAAAAAAABOADFCAA|1961-02-14|733|3190|245|1961|2|2|14|1|1961|245|3190|Tuesday|1961Q1|N|N|N|2437332|2437362|2436979|2437253|N|N|N|N|N| +2437346|AAAAAAAACOADFCAA|1961-02-15|733|3190|245|1961|3|2|15|1|1961|245|3190|Wednesday|1961Q1|N|N|N|2437332|2437362|2436980|2437254|N|N|N|N|N| +2437347|AAAAAAAADOADFCAA|1961-02-16|733|3190|245|1961|4|2|16|1|1961|245|3190|Thursday|1961Q1|N|N|N|2437332|2437362|2436981|2437255|N|N|N|N|N| +2437348|AAAAAAAAEOADFCAA|1961-02-17|733|3190|245|1961|5|2|17|1|1961|245|3190|Friday|1961Q1|N|Y|N|2437332|2437362|2436982|2437256|N|N|N|N|N| +2437349|AAAAAAAAFOADFCAA|1961-02-18|733|3190|245|1961|6|2|18|1|1961|245|3190|Saturday|1961Q1|N|Y|N|2437332|2437362|2436983|2437257|N|N|N|N|N| +2437350|AAAAAAAAGOADFCAA|1961-02-19|733|3190|245|1961|0|2|19|1|1961|245|3190|Sunday|1961Q1|N|N|N|2437332|2437362|2436984|2437258|N|N|N|N|N| +2437351|AAAAAAAAHOADFCAA|1961-02-20|733|3190|245|1961|1|2|20|1|1961|245|3190|Monday|1961Q1|N|N|N|2437332|2437362|2436985|2437259|N|N|N|N|N| +2437352|AAAAAAAAIOADFCAA|1961-02-21|733|3191|245|1961|2|2|21|1|1961|245|3191|Tuesday|1961Q1|N|N|N|2437332|2437362|2436986|2437260|N|N|N|N|N| +2437353|AAAAAAAAJOADFCAA|1961-02-22|733|3191|245|1961|3|2|22|1|1961|245|3191|Wednesday|1961Q1|N|N|N|2437332|2437362|2436987|2437261|N|N|N|N|N| +2437354|AAAAAAAAKOADFCAA|1961-02-23|733|3191|245|1961|4|2|23|1|1961|245|3191|Thursday|1961Q1|N|N|N|2437332|2437362|2436988|2437262|N|N|N|N|N| +2437355|AAAAAAAALOADFCAA|1961-02-24|733|3191|245|1961|5|2|24|1|1961|245|3191|Friday|1961Q1|N|Y|N|2437332|2437362|2436989|2437263|N|N|N|N|N| +2437356|AAAAAAAAMOADFCAA|1961-02-25|733|3191|245|1961|6|2|25|1|1961|245|3191|Saturday|1961Q1|N|Y|N|2437332|2437362|2436990|2437264|N|N|N|N|N| +2437357|AAAAAAAANOADFCAA|1961-02-26|733|3191|245|1961|0|2|26|1|1961|245|3191|Sunday|1961Q1|N|N|N|2437332|2437362|2436991|2437265|N|N|N|N|N| +2437358|AAAAAAAAOOADFCAA|1961-02-27|733|3191|245|1961|1|2|27|1|1961|245|3191|Monday|1961Q1|N|N|N|2437332|2437362|2436992|2437266|N|N|N|N|N| +2437359|AAAAAAAAPOADFCAA|1961-02-28|733|3192|245|1961|2|2|28|1|1961|245|3192|Tuesday|1961Q1|N|N|N|2437332|2437362|2436993|2437267|N|N|N|N|N| +2437360|AAAAAAAAAPADFCAA|1961-03-01|734|3192|246|1961|3|3|1|1|1961|246|3192|Wednesday|1961Q1|N|N|N|2437360|2437418|2436995|2437268|N|N|N|N|N| +2437361|AAAAAAAABPADFCAA|1961-03-02|734|3192|246|1961|4|3|2|1|1961|246|3192|Thursday|1961Q1|N|N|N|2437360|2437418|2436996|2437269|N|N|N|N|N| +2437362|AAAAAAAACPADFCAA|1961-03-03|734|3192|246|1961|5|3|3|1|1961|246|3192|Friday|1961Q1|N|Y|N|2437360|2437418|2436997|2437270|N|N|N|N|N| +2437363|AAAAAAAADPADFCAA|1961-03-04|734|3192|246|1961|6|3|4|1|1961|246|3192|Saturday|1961Q1|N|Y|N|2437360|2437418|2436998|2437271|N|N|N|N|N| +2437364|AAAAAAAAEPADFCAA|1961-03-05|734|3192|246|1961|0|3|5|1|1961|246|3192|Sunday|1961Q1|N|N|N|2437360|2437418|2436999|2437272|N|N|N|N|N| +2437365|AAAAAAAAFPADFCAA|1961-03-06|734|3192|246|1961|1|3|6|1|1961|246|3192|Monday|1961Q1|N|N|N|2437360|2437418|2437000|2437273|N|N|N|N|N| +2437366|AAAAAAAAGPADFCAA|1961-03-07|734|3193|246|1961|2|3|7|1|1961|246|3193|Tuesday|1961Q1|N|N|N|2437360|2437418|2437001|2437274|N|N|N|N|N| +2437367|AAAAAAAAHPADFCAA|1961-03-08|734|3193|246|1961|3|3|8|1|1961|246|3193|Wednesday|1961Q1|N|N|N|2437360|2437418|2437002|2437275|N|N|N|N|N| +2437368|AAAAAAAAIPADFCAA|1961-03-09|734|3193|246|1961|4|3|9|1|1961|246|3193|Thursday|1961Q1|N|N|N|2437360|2437418|2437003|2437276|N|N|N|N|N| +2437369|AAAAAAAAJPADFCAA|1961-03-10|734|3193|246|1961|5|3|10|1|1961|246|3193|Friday|1961Q1|N|Y|N|2437360|2437418|2437004|2437277|N|N|N|N|N| +2437370|AAAAAAAAKPADFCAA|1961-03-11|734|3193|246|1961|6|3|11|1|1961|246|3193|Saturday|1961Q1|N|Y|N|2437360|2437418|2437005|2437278|N|N|N|N|N| +2437371|AAAAAAAALPADFCAA|1961-03-12|734|3193|246|1961|0|3|12|1|1961|246|3193|Sunday|1961Q1|N|N|N|2437360|2437418|2437006|2437279|N|N|N|N|N| +2437372|AAAAAAAAMPADFCAA|1961-03-13|734|3193|246|1961|1|3|13|1|1961|246|3193|Monday|1961Q1|N|N|N|2437360|2437418|2437007|2437280|N|N|N|N|N| +2437373|AAAAAAAANPADFCAA|1961-03-14|734|3194|246|1961|2|3|14|1|1961|246|3194|Tuesday|1961Q1|N|N|N|2437360|2437418|2437008|2437281|N|N|N|N|N| +2437374|AAAAAAAAOPADFCAA|1961-03-15|734|3194|246|1961|3|3|15|1|1961|246|3194|Wednesday|1961Q1|N|N|N|2437360|2437418|2437009|2437282|N|N|N|N|N| +2437375|AAAAAAAAPPADFCAA|1961-03-16|734|3194|246|1961|4|3|16|1|1961|246|3194|Thursday|1961Q1|N|N|N|2437360|2437418|2437010|2437283|N|N|N|N|N| +2437376|AAAAAAAAAABDFCAA|1961-03-17|734|3194|246|1961|5|3|17|1|1961|246|3194|Friday|1961Q1|N|Y|N|2437360|2437418|2437011|2437284|N|N|N|N|N| +2437377|AAAAAAAABABDFCAA|1961-03-18|734|3194|246|1961|6|3|18|1|1961|246|3194|Saturday|1961Q1|N|Y|N|2437360|2437418|2437012|2437285|N|N|N|N|N| +2437378|AAAAAAAACABDFCAA|1961-03-19|734|3194|246|1961|0|3|19|1|1961|246|3194|Sunday|1961Q1|N|N|N|2437360|2437418|2437013|2437286|N|N|N|N|N| +2437379|AAAAAAAADABDFCAA|1961-03-20|734|3194|246|1961|1|3|20|1|1961|246|3194|Monday|1961Q1|N|N|N|2437360|2437418|2437014|2437287|N|N|N|N|N| +2437380|AAAAAAAAEABDFCAA|1961-03-21|734|3195|246|1961|2|3|21|1|1961|246|3195|Tuesday|1961Q1|N|N|N|2437360|2437418|2437015|2437288|N|N|N|N|N| +2437381|AAAAAAAAFABDFCAA|1961-03-22|734|3195|246|1961|3|3|22|1|1961|246|3195|Wednesday|1961Q1|N|N|N|2437360|2437418|2437016|2437289|N|N|N|N|N| +2437382|AAAAAAAAGABDFCAA|1961-03-23|734|3195|246|1961|4|3|23|1|1961|246|3195|Thursday|1961Q1|N|N|N|2437360|2437418|2437017|2437290|N|N|N|N|N| +2437383|AAAAAAAAHABDFCAA|1961-03-24|734|3195|246|1961|5|3|24|1|1961|246|3195|Friday|1961Q1|N|Y|N|2437360|2437418|2437018|2437291|N|N|N|N|N| +2437384|AAAAAAAAIABDFCAA|1961-03-25|734|3195|246|1961|6|3|25|1|1961|246|3195|Saturday|1961Q1|N|Y|N|2437360|2437418|2437019|2437292|N|N|N|N|N| +2437385|AAAAAAAAJABDFCAA|1961-03-26|734|3195|246|1961|0|3|26|1|1961|246|3195|Sunday|1961Q1|N|N|N|2437360|2437418|2437020|2437293|N|N|N|N|N| +2437386|AAAAAAAAKABDFCAA|1961-03-27|734|3195|246|1961|1|3|27|1|1961|246|3195|Monday|1961Q1|N|N|N|2437360|2437418|2437021|2437294|N|N|N|N|N| +2437387|AAAAAAAALABDFCAA|1961-03-28|734|3196|246|1961|2|3|28|1|1961|246|3196|Tuesday|1961Q1|N|N|N|2437360|2437418|2437022|2437295|N|N|N|N|N| +2437388|AAAAAAAAMABDFCAA|1961-03-29|734|3196|246|1961|3|3|29|1|1961|246|3196|Wednesday|1961Q1|N|N|N|2437360|2437418|2437023|2437296|N|N|N|N|N| +2437389|AAAAAAAANABDFCAA|1961-03-30|734|3196|246|1961|4|3|30|1|1961|246|3196|Thursday|1961Q1|N|N|N|2437360|2437418|2437024|2437297|N|N|N|N|N| +2437390|AAAAAAAAOABDFCAA|1961-03-31|734|3196|246|1961|5|3|31|1|1961|246|3196|Friday|1961Q1|N|Y|N|2437360|2437418|2437025|2437298|N|N|N|N|N| +2437391|AAAAAAAAPABDFCAA|1961-04-01|735|3196|246|1961|6|4|1|1|1961|246|3196|Saturday|1961Q1|N|Y|N|2437391|2437480|2437026|2437301|N|N|N|N|N| +2437392|AAAAAAAAABBDFCAA|1961-04-02|735|3196|246|1961|0|4|2|2|1961|246|3196|Sunday|1961Q2|N|N|N|2437391|2437480|2437027|2437302|N|N|N|N|N| +2437393|AAAAAAAABBBDFCAA|1961-04-03|735|3196|246|1961|1|4|3|2|1961|246|3196|Monday|1961Q2|N|N|N|2437391|2437480|2437028|2437303|N|N|N|N|N| +2437394|AAAAAAAACBBDFCAA|1961-04-04|735|3197|246|1961|2|4|4|2|1961|246|3197|Tuesday|1961Q2|N|N|N|2437391|2437480|2437029|2437304|N|N|N|N|N| +2437395|AAAAAAAADBBDFCAA|1961-04-05|735|3197|246|1961|3|4|5|2|1961|246|3197|Wednesday|1961Q2|N|N|N|2437391|2437480|2437030|2437305|N|N|N|N|N| +2437396|AAAAAAAAEBBDFCAA|1961-04-06|735|3197|246|1961|4|4|6|2|1961|246|3197|Thursday|1961Q2|N|N|N|2437391|2437480|2437031|2437306|N|N|N|N|N| +2437397|AAAAAAAAFBBDFCAA|1961-04-07|735|3197|246|1961|5|4|7|2|1961|246|3197|Friday|1961Q2|N|Y|N|2437391|2437480|2437032|2437307|N|N|N|N|N| +2437398|AAAAAAAAGBBDFCAA|1961-04-08|735|3197|246|1961|6|4|8|2|1961|246|3197|Saturday|1961Q2|N|Y|N|2437391|2437480|2437033|2437308|N|N|N|N|N| +2437399|AAAAAAAAHBBDFCAA|1961-04-09|735|3197|246|1961|0|4|9|2|1961|246|3197|Sunday|1961Q2|N|N|N|2437391|2437480|2437034|2437309|N|N|N|N|N| +2437400|AAAAAAAAIBBDFCAA|1961-04-10|735|3197|246|1961|1|4|10|2|1961|246|3197|Monday|1961Q2|N|N|N|2437391|2437480|2437035|2437310|N|N|N|N|N| +2437401|AAAAAAAAJBBDFCAA|1961-04-11|735|3198|246|1961|2|4|11|2|1961|246|3198|Tuesday|1961Q2|N|N|N|2437391|2437480|2437036|2437311|N|N|N|N|N| +2437402|AAAAAAAAKBBDFCAA|1961-04-12|735|3198|246|1961|3|4|12|2|1961|246|3198|Wednesday|1961Q2|N|N|N|2437391|2437480|2437037|2437312|N|N|N|N|N| +2437403|AAAAAAAALBBDFCAA|1961-04-13|735|3198|246|1961|4|4|13|2|1961|246|3198|Thursday|1961Q2|N|N|N|2437391|2437480|2437038|2437313|N|N|N|N|N| +2437404|AAAAAAAAMBBDFCAA|1961-04-14|735|3198|246|1961|5|4|14|2|1961|246|3198|Friday|1961Q2|N|Y|N|2437391|2437480|2437039|2437314|N|N|N|N|N| +2437405|AAAAAAAANBBDFCAA|1961-04-15|735|3198|246|1961|6|4|15|2|1961|246|3198|Saturday|1961Q2|N|Y|N|2437391|2437480|2437040|2437315|N|N|N|N|N| +2437406|AAAAAAAAOBBDFCAA|1961-04-16|735|3198|246|1961|0|4|16|2|1961|246|3198|Sunday|1961Q2|N|N|N|2437391|2437480|2437041|2437316|N|N|N|N|N| +2437407|AAAAAAAAPBBDFCAA|1961-04-17|735|3198|246|1961|1|4|17|2|1961|246|3198|Monday|1961Q2|N|N|N|2437391|2437480|2437042|2437317|N|N|N|N|N| +2437408|AAAAAAAAACBDFCAA|1961-04-18|735|3199|246|1961|2|4|18|2|1961|246|3199|Tuesday|1961Q2|N|N|N|2437391|2437480|2437043|2437318|N|N|N|N|N| +2437409|AAAAAAAABCBDFCAA|1961-04-19|735|3199|246|1961|3|4|19|2|1961|246|3199|Wednesday|1961Q2|N|N|N|2437391|2437480|2437044|2437319|N|N|N|N|N| +2437410|AAAAAAAACCBDFCAA|1961-04-20|735|3199|246|1961|4|4|20|2|1961|246|3199|Thursday|1961Q2|N|N|N|2437391|2437480|2437045|2437320|N|N|N|N|N| +2437411|AAAAAAAADCBDFCAA|1961-04-21|735|3199|246|1961|5|4|21|2|1961|246|3199|Friday|1961Q2|N|Y|N|2437391|2437480|2437046|2437321|N|N|N|N|N| +2437412|AAAAAAAAECBDFCAA|1961-04-22|735|3199|246|1961|6|4|22|2|1961|246|3199|Saturday|1961Q2|N|Y|N|2437391|2437480|2437047|2437322|N|N|N|N|N| +2437413|AAAAAAAAFCBDFCAA|1961-04-23|735|3199|246|1961|0|4|23|2|1961|246|3199|Sunday|1961Q2|N|N|N|2437391|2437480|2437048|2437323|N|N|N|N|N| +2437414|AAAAAAAAGCBDFCAA|1961-04-24|735|3199|246|1961|1|4|24|2|1961|246|3199|Monday|1961Q2|N|N|N|2437391|2437480|2437049|2437324|N|N|N|N|N| +2437415|AAAAAAAAHCBDFCAA|1961-04-25|735|3200|246|1961|2|4|25|2|1961|246|3200|Tuesday|1961Q2|N|N|N|2437391|2437480|2437050|2437325|N|N|N|N|N| +2437416|AAAAAAAAICBDFCAA|1961-04-26|735|3200|246|1961|3|4|26|2|1961|246|3200|Wednesday|1961Q2|N|N|N|2437391|2437480|2437051|2437326|N|N|N|N|N| +2437417|AAAAAAAAJCBDFCAA|1961-04-27|735|3200|246|1961|4|4|27|2|1961|246|3200|Thursday|1961Q2|N|N|N|2437391|2437480|2437052|2437327|N|N|N|N|N| +2437418|AAAAAAAAKCBDFCAA|1961-04-28|735|3200|246|1961|5|4|28|2|1961|246|3200|Friday|1961Q2|N|Y|N|2437391|2437480|2437053|2437328|N|N|N|N|N| +2437419|AAAAAAAALCBDFCAA|1961-04-29|735|3200|246|1961|6|4|29|2|1961|246|3200|Saturday|1961Q2|N|Y|N|2437391|2437480|2437054|2437329|N|N|N|N|N| +2437420|AAAAAAAAMCBDFCAA|1961-04-30|735|3200|246|1961|0|4|30|2|1961|246|3200|Sunday|1961Q2|N|N|N|2437391|2437480|2437055|2437330|N|N|N|N|N| +2437421|AAAAAAAANCBDFCAA|1961-05-01|736|3200|246|1961|1|5|1|2|1961|246|3200|Monday|1961Q2|N|N|N|2437421|2437540|2437056|2437331|N|N|N|N|N| +2437422|AAAAAAAAOCBDFCAA|1961-05-02|736|3201|246|1961|2|5|2|2|1961|246|3201|Tuesday|1961Q2|N|N|N|2437421|2437540|2437057|2437332|N|N|N|N|N| +2437423|AAAAAAAAPCBDFCAA|1961-05-03|736|3201|246|1961|3|5|3|2|1961|246|3201|Wednesday|1961Q2|N|N|N|2437421|2437540|2437058|2437333|N|N|N|N|N| +2437424|AAAAAAAAADBDFCAA|1961-05-04|736|3201|246|1961|4|5|4|2|1961|246|3201|Thursday|1961Q2|N|N|N|2437421|2437540|2437059|2437334|N|N|N|N|N| +2437425|AAAAAAAABDBDFCAA|1961-05-05|736|3201|246|1961|5|5|5|2|1961|246|3201|Friday|1961Q2|N|Y|N|2437421|2437540|2437060|2437335|N|N|N|N|N| +2437426|AAAAAAAACDBDFCAA|1961-05-06|736|3201|246|1961|6|5|6|2|1961|246|3201|Saturday|1961Q2|N|Y|N|2437421|2437540|2437061|2437336|N|N|N|N|N| +2437427|AAAAAAAADDBDFCAA|1961-05-07|736|3201|246|1961|0|5|7|2|1961|246|3201|Sunday|1961Q2|N|N|N|2437421|2437540|2437062|2437337|N|N|N|N|N| +2437428|AAAAAAAAEDBDFCAA|1961-05-08|736|3201|246|1961|1|5|8|2|1961|246|3201|Monday|1961Q2|N|N|N|2437421|2437540|2437063|2437338|N|N|N|N|N| +2437429|AAAAAAAAFDBDFCAA|1961-05-09|736|3202|246|1961|2|5|9|2|1961|246|3202|Tuesday|1961Q2|N|N|N|2437421|2437540|2437064|2437339|N|N|N|N|N| +2437430|AAAAAAAAGDBDFCAA|1961-05-10|736|3202|246|1961|3|5|10|2|1961|246|3202|Wednesday|1961Q2|N|N|N|2437421|2437540|2437065|2437340|N|N|N|N|N| +2437431|AAAAAAAAHDBDFCAA|1961-05-11|736|3202|246|1961|4|5|11|2|1961|246|3202|Thursday|1961Q2|N|N|N|2437421|2437540|2437066|2437341|N|N|N|N|N| +2437432|AAAAAAAAIDBDFCAA|1961-05-12|736|3202|246|1961|5|5|12|2|1961|246|3202|Friday|1961Q2|N|Y|N|2437421|2437540|2437067|2437342|N|N|N|N|N| +2437433|AAAAAAAAJDBDFCAA|1961-05-13|736|3202|246|1961|6|5|13|2|1961|246|3202|Saturday|1961Q2|N|Y|N|2437421|2437540|2437068|2437343|N|N|N|N|N| +2437434|AAAAAAAAKDBDFCAA|1961-05-14|736|3202|246|1961|0|5|14|2|1961|246|3202|Sunday|1961Q2|N|N|N|2437421|2437540|2437069|2437344|N|N|N|N|N| +2437435|AAAAAAAALDBDFCAA|1961-05-15|736|3202|246|1961|1|5|15|2|1961|246|3202|Monday|1961Q2|N|N|N|2437421|2437540|2437070|2437345|N|N|N|N|N| +2437436|AAAAAAAAMDBDFCAA|1961-05-16|736|3203|246|1961|2|5|16|2|1961|246|3203|Tuesday|1961Q2|N|N|N|2437421|2437540|2437071|2437346|N|N|N|N|N| +2437437|AAAAAAAANDBDFCAA|1961-05-17|736|3203|246|1961|3|5|17|2|1961|246|3203|Wednesday|1961Q2|N|N|N|2437421|2437540|2437072|2437347|N|N|N|N|N| +2437438|AAAAAAAAODBDFCAA|1961-05-18|736|3203|246|1961|4|5|18|2|1961|246|3203|Thursday|1961Q2|N|N|N|2437421|2437540|2437073|2437348|N|N|N|N|N| +2437439|AAAAAAAAPDBDFCAA|1961-05-19|736|3203|246|1961|5|5|19|2|1961|246|3203|Friday|1961Q2|N|Y|N|2437421|2437540|2437074|2437349|N|N|N|N|N| +2437440|AAAAAAAAAEBDFCAA|1961-05-20|736|3203|246|1961|6|5|20|2|1961|246|3203|Saturday|1961Q2|N|Y|N|2437421|2437540|2437075|2437350|N|N|N|N|N| +2437441|AAAAAAAABEBDFCAA|1961-05-21|736|3203|246|1961|0|5|21|2|1961|246|3203|Sunday|1961Q2|N|N|N|2437421|2437540|2437076|2437351|N|N|N|N|N| +2437442|AAAAAAAACEBDFCAA|1961-05-22|736|3203|246|1961|1|5|22|2|1961|246|3203|Monday|1961Q2|N|N|N|2437421|2437540|2437077|2437352|N|N|N|N|N| +2437443|AAAAAAAADEBDFCAA|1961-05-23|736|3204|246|1961|2|5|23|2|1961|246|3204|Tuesday|1961Q2|N|N|N|2437421|2437540|2437078|2437353|N|N|N|N|N| +2437444|AAAAAAAAEEBDFCAA|1961-05-24|736|3204|246|1961|3|5|24|2|1961|246|3204|Wednesday|1961Q2|N|N|N|2437421|2437540|2437079|2437354|N|N|N|N|N| +2437445|AAAAAAAAFEBDFCAA|1961-05-25|736|3204|246|1961|4|5|25|2|1961|246|3204|Thursday|1961Q2|N|N|N|2437421|2437540|2437080|2437355|N|N|N|N|N| +2437446|AAAAAAAAGEBDFCAA|1961-05-26|736|3204|246|1961|5|5|26|2|1961|246|3204|Friday|1961Q2|N|Y|N|2437421|2437540|2437081|2437356|N|N|N|N|N| +2437447|AAAAAAAAHEBDFCAA|1961-05-27|736|3204|246|1961|6|5|27|2|1961|246|3204|Saturday|1961Q2|N|Y|N|2437421|2437540|2437082|2437357|N|N|N|N|N| +2437448|AAAAAAAAIEBDFCAA|1961-05-28|736|3204|246|1961|0|5|28|2|1961|246|3204|Sunday|1961Q2|N|N|N|2437421|2437540|2437083|2437358|N|N|N|N|N| +2437449|AAAAAAAAJEBDFCAA|1961-05-29|736|3204|246|1961|1|5|29|2|1961|246|3204|Monday|1961Q2|N|N|N|2437421|2437540|2437084|2437359|N|N|N|N|N| +2437450|AAAAAAAAKEBDFCAA|1961-05-30|736|3205|246|1961|2|5|30|2|1961|246|3205|Tuesday|1961Q2|N|N|N|2437421|2437540|2437085|2437360|N|N|N|N|N| +2437451|AAAAAAAALEBDFCAA|1961-05-31|736|3205|246|1961|3|5|31|2|1961|246|3205|Wednesday|1961Q2|N|N|N|2437421|2437540|2437086|2437361|N|N|N|N|N| +2437452|AAAAAAAAMEBDFCAA|1961-06-01|737|3205|247|1961|4|6|1|2|1961|247|3205|Thursday|1961Q2|N|N|N|2437452|2437602|2437087|2437362|N|N|N|N|N| +2437453|AAAAAAAANEBDFCAA|1961-06-02|737|3205|247|1961|5|6|2|2|1961|247|3205|Friday|1961Q2|N|Y|N|2437452|2437602|2437088|2437363|N|N|N|N|N| +2437454|AAAAAAAAOEBDFCAA|1961-06-03|737|3205|247|1961|6|6|3|2|1961|247|3205|Saturday|1961Q2|N|Y|N|2437452|2437602|2437089|2437364|N|N|N|N|N| +2437455|AAAAAAAAPEBDFCAA|1961-06-04|737|3205|247|1961|0|6|4|2|1961|247|3205|Sunday|1961Q2|N|N|N|2437452|2437602|2437090|2437365|N|N|N|N|N| +2437456|AAAAAAAAAFBDFCAA|1961-06-05|737|3205|247|1961|1|6|5|2|1961|247|3205|Monday|1961Q2|N|N|N|2437452|2437602|2437091|2437366|N|N|N|N|N| +2437457|AAAAAAAABFBDFCAA|1961-06-06|737|3206|247|1961|2|6|6|2|1961|247|3206|Tuesday|1961Q2|N|N|N|2437452|2437602|2437092|2437367|N|N|N|N|N| +2437458|AAAAAAAACFBDFCAA|1961-06-07|737|3206|247|1961|3|6|7|2|1961|247|3206|Wednesday|1961Q2|N|N|N|2437452|2437602|2437093|2437368|N|N|N|N|N| +2437459|AAAAAAAADFBDFCAA|1961-06-08|737|3206|247|1961|4|6|8|2|1961|247|3206|Thursday|1961Q2|N|N|N|2437452|2437602|2437094|2437369|N|N|N|N|N| +2437460|AAAAAAAAEFBDFCAA|1961-06-09|737|3206|247|1961|5|6|9|2|1961|247|3206|Friday|1961Q2|N|Y|N|2437452|2437602|2437095|2437370|N|N|N|N|N| +2437461|AAAAAAAAFFBDFCAA|1961-06-10|737|3206|247|1961|6|6|10|2|1961|247|3206|Saturday|1961Q2|N|Y|N|2437452|2437602|2437096|2437371|N|N|N|N|N| +2437462|AAAAAAAAGFBDFCAA|1961-06-11|737|3206|247|1961|0|6|11|2|1961|247|3206|Sunday|1961Q2|N|N|N|2437452|2437602|2437097|2437372|N|N|N|N|N| +2437463|AAAAAAAAHFBDFCAA|1961-06-12|737|3206|247|1961|1|6|12|2|1961|247|3206|Monday|1961Q2|N|N|N|2437452|2437602|2437098|2437373|N|N|N|N|N| +2437464|AAAAAAAAIFBDFCAA|1961-06-13|737|3207|247|1961|2|6|13|2|1961|247|3207|Tuesday|1961Q2|N|N|N|2437452|2437602|2437099|2437374|N|N|N|N|N| +2437465|AAAAAAAAJFBDFCAA|1961-06-14|737|3207|247|1961|3|6|14|2|1961|247|3207|Wednesday|1961Q2|N|N|N|2437452|2437602|2437100|2437375|N|N|N|N|N| +2437466|AAAAAAAAKFBDFCAA|1961-06-15|737|3207|247|1961|4|6|15|2|1961|247|3207|Thursday|1961Q2|N|N|N|2437452|2437602|2437101|2437376|N|N|N|N|N| +2437467|AAAAAAAALFBDFCAA|1961-06-16|737|3207|247|1961|5|6|16|2|1961|247|3207|Friday|1961Q2|N|Y|N|2437452|2437602|2437102|2437377|N|N|N|N|N| +2437468|AAAAAAAAMFBDFCAA|1961-06-17|737|3207|247|1961|6|6|17|2|1961|247|3207|Saturday|1961Q2|N|Y|N|2437452|2437602|2437103|2437378|N|N|N|N|N| +2437469|AAAAAAAANFBDFCAA|1961-06-18|737|3207|247|1961|0|6|18|2|1961|247|3207|Sunday|1961Q2|N|N|N|2437452|2437602|2437104|2437379|N|N|N|N|N| +2437470|AAAAAAAAOFBDFCAA|1961-06-19|737|3207|247|1961|1|6|19|2|1961|247|3207|Monday|1961Q2|N|N|N|2437452|2437602|2437105|2437380|N|N|N|N|N| +2437471|AAAAAAAAPFBDFCAA|1961-06-20|737|3208|247|1961|2|6|20|2|1961|247|3208|Tuesday|1961Q2|N|N|N|2437452|2437602|2437106|2437381|N|N|N|N|N| +2437472|AAAAAAAAAGBDFCAA|1961-06-21|737|3208|247|1961|3|6|21|2|1961|247|3208|Wednesday|1961Q2|N|N|N|2437452|2437602|2437107|2437382|N|N|N|N|N| +2437473|AAAAAAAABGBDFCAA|1961-06-22|737|3208|247|1961|4|6|22|2|1961|247|3208|Thursday|1961Q2|N|N|N|2437452|2437602|2437108|2437383|N|N|N|N|N| +2437474|AAAAAAAACGBDFCAA|1961-06-23|737|3208|247|1961|5|6|23|2|1961|247|3208|Friday|1961Q2|N|Y|N|2437452|2437602|2437109|2437384|N|N|N|N|N| +2437475|AAAAAAAADGBDFCAA|1961-06-24|737|3208|247|1961|6|6|24|2|1961|247|3208|Saturday|1961Q2|N|Y|N|2437452|2437602|2437110|2437385|N|N|N|N|N| +2437476|AAAAAAAAEGBDFCAA|1961-06-25|737|3208|247|1961|0|6|25|2|1961|247|3208|Sunday|1961Q2|N|N|N|2437452|2437602|2437111|2437386|N|N|N|N|N| +2437477|AAAAAAAAFGBDFCAA|1961-06-26|737|3208|247|1961|1|6|26|2|1961|247|3208|Monday|1961Q2|N|N|N|2437452|2437602|2437112|2437387|N|N|N|N|N| +2437478|AAAAAAAAGGBDFCAA|1961-06-27|737|3209|247|1961|2|6|27|2|1961|247|3209|Tuesday|1961Q2|N|N|N|2437452|2437602|2437113|2437388|N|N|N|N|N| +2437479|AAAAAAAAHGBDFCAA|1961-06-28|737|3209|247|1961|3|6|28|2|1961|247|3209|Wednesday|1961Q2|N|N|N|2437452|2437602|2437114|2437389|N|N|N|N|N| +2437480|AAAAAAAAIGBDFCAA|1961-06-29|737|3209|247|1961|4|6|29|2|1961|247|3209|Thursday|1961Q2|N|N|N|2437452|2437602|2437115|2437390|N|N|N|N|N| +2437481|AAAAAAAAJGBDFCAA|1961-06-30|737|3209|247|1961|5|6|30|2|1961|247|3209|Friday|1961Q2|N|Y|N|2437452|2437602|2437116|2437391|N|N|N|N|N| +2437482|AAAAAAAAKGBDFCAA|1961-07-01|738|3209|247|1961|6|7|1|2|1961|247|3209|Saturday|1961Q2|N|Y|N|2437482|2437662|2437117|2437391|N|N|N|N|N| +2437483|AAAAAAAALGBDFCAA|1961-07-02|738|3209|247|1961|0|7|2|3|1961|247|3209|Sunday|1961Q3|N|N|N|2437482|2437662|2437118|2437392|N|N|N|N|N| +2437484|AAAAAAAAMGBDFCAA|1961-07-03|738|3209|247|1961|1|7|3|3|1961|247|3209|Monday|1961Q3|N|N|N|2437482|2437662|2437119|2437393|N|N|N|N|N| +2437485|AAAAAAAANGBDFCAA|1961-07-04|738|3210|247|1961|2|7|4|3|1961|247|3210|Tuesday|1961Q3|N|N|N|2437482|2437662|2437120|2437394|N|N|N|N|N| +2437486|AAAAAAAAOGBDFCAA|1961-07-05|738|3210|247|1961|3|7|5|3|1961|247|3210|Wednesday|1961Q3|Y|N|N|2437482|2437662|2437121|2437395|N|N|N|N|N| +2437487|AAAAAAAAPGBDFCAA|1961-07-06|738|3210|247|1961|4|7|6|3|1961|247|3210|Thursday|1961Q3|N|N|Y|2437482|2437662|2437122|2437396|N|N|N|N|N| +2437488|AAAAAAAAAHBDFCAA|1961-07-07|738|3210|247|1961|5|7|7|3|1961|247|3210|Friday|1961Q3|N|Y|N|2437482|2437662|2437123|2437397|N|N|N|N|N| +2437489|AAAAAAAABHBDFCAA|1961-07-08|738|3210|247|1961|6|7|8|3|1961|247|3210|Saturday|1961Q3|N|Y|N|2437482|2437662|2437124|2437398|N|N|N|N|N| +2437490|AAAAAAAACHBDFCAA|1961-07-09|738|3210|247|1961|0|7|9|3|1961|247|3210|Sunday|1961Q3|N|N|N|2437482|2437662|2437125|2437399|N|N|N|N|N| +2437491|AAAAAAAADHBDFCAA|1961-07-10|738|3210|247|1961|1|7|10|3|1961|247|3210|Monday|1961Q3|N|N|N|2437482|2437662|2437126|2437400|N|N|N|N|N| +2437492|AAAAAAAAEHBDFCAA|1961-07-11|738|3211|247|1961|2|7|11|3|1961|247|3211|Tuesday|1961Q3|N|N|N|2437482|2437662|2437127|2437401|N|N|N|N|N| +2437493|AAAAAAAAFHBDFCAA|1961-07-12|738|3211|247|1961|3|7|12|3|1961|247|3211|Wednesday|1961Q3|N|N|N|2437482|2437662|2437128|2437402|N|N|N|N|N| +2437494|AAAAAAAAGHBDFCAA|1961-07-13|738|3211|247|1961|4|7|13|3|1961|247|3211|Thursday|1961Q3|N|N|N|2437482|2437662|2437129|2437403|N|N|N|N|N| +2437495|AAAAAAAAHHBDFCAA|1961-07-14|738|3211|247|1961|5|7|14|3|1961|247|3211|Friday|1961Q3|N|Y|N|2437482|2437662|2437130|2437404|N|N|N|N|N| +2437496|AAAAAAAAIHBDFCAA|1961-07-15|738|3211|247|1961|6|7|15|3|1961|247|3211|Saturday|1961Q3|N|Y|N|2437482|2437662|2437131|2437405|N|N|N|N|N| +2437497|AAAAAAAAJHBDFCAA|1961-07-16|738|3211|247|1961|0|7|16|3|1961|247|3211|Sunday|1961Q3|N|N|N|2437482|2437662|2437132|2437406|N|N|N|N|N| +2437498|AAAAAAAAKHBDFCAA|1961-07-17|738|3211|247|1961|1|7|17|3|1961|247|3211|Monday|1961Q3|N|N|N|2437482|2437662|2437133|2437407|N|N|N|N|N| +2437499|AAAAAAAALHBDFCAA|1961-07-18|738|3212|247|1961|2|7|18|3|1961|247|3212|Tuesday|1961Q3|N|N|N|2437482|2437662|2437134|2437408|N|N|N|N|N| +2437500|AAAAAAAAMHBDFCAA|1961-07-19|738|3212|247|1961|3|7|19|3|1961|247|3212|Wednesday|1961Q3|N|N|N|2437482|2437662|2437135|2437409|N|N|N|N|N| +2437501|AAAAAAAANHBDFCAA|1961-07-20|738|3212|247|1961|4|7|20|3|1961|247|3212|Thursday|1961Q3|N|N|N|2437482|2437662|2437136|2437410|N|N|N|N|N| +2437502|AAAAAAAAOHBDFCAA|1961-07-21|738|3212|247|1961|5|7|21|3|1961|247|3212|Friday|1961Q3|N|Y|N|2437482|2437662|2437137|2437411|N|N|N|N|N| +2437503|AAAAAAAAPHBDFCAA|1961-07-22|738|3212|247|1961|6|7|22|3|1961|247|3212|Saturday|1961Q3|N|Y|N|2437482|2437662|2437138|2437412|N|N|N|N|N| +2437504|AAAAAAAAAIBDFCAA|1961-07-23|738|3212|247|1961|0|7|23|3|1961|247|3212|Sunday|1961Q3|N|N|N|2437482|2437662|2437139|2437413|N|N|N|N|N| +2437505|AAAAAAAABIBDFCAA|1961-07-24|738|3212|247|1961|1|7|24|3|1961|247|3212|Monday|1961Q3|N|N|N|2437482|2437662|2437140|2437414|N|N|N|N|N| +2437506|AAAAAAAACIBDFCAA|1961-07-25|738|3213|247|1961|2|7|25|3|1961|247|3213|Tuesday|1961Q3|N|N|N|2437482|2437662|2437141|2437415|N|N|N|N|N| +2437507|AAAAAAAADIBDFCAA|1961-07-26|738|3213|247|1961|3|7|26|3|1961|247|3213|Wednesday|1961Q3|N|N|N|2437482|2437662|2437142|2437416|N|N|N|N|N| +2437508|AAAAAAAAEIBDFCAA|1961-07-27|738|3213|247|1961|4|7|27|3|1961|247|3213|Thursday|1961Q3|N|N|N|2437482|2437662|2437143|2437417|N|N|N|N|N| +2437509|AAAAAAAAFIBDFCAA|1961-07-28|738|3213|247|1961|5|7|28|3|1961|247|3213|Friday|1961Q3|N|Y|N|2437482|2437662|2437144|2437418|N|N|N|N|N| +2437510|AAAAAAAAGIBDFCAA|1961-07-29|738|3213|247|1961|6|7|29|3|1961|247|3213|Saturday|1961Q3|N|Y|N|2437482|2437662|2437145|2437419|N|N|N|N|N| +2437511|AAAAAAAAHIBDFCAA|1961-07-30|738|3213|247|1961|0|7|30|3|1961|247|3213|Sunday|1961Q3|N|N|N|2437482|2437662|2437146|2437420|N|N|N|N|N| +2437512|AAAAAAAAIIBDFCAA|1961-07-31|738|3213|247|1961|1|7|31|3|1961|247|3213|Monday|1961Q3|N|N|N|2437482|2437662|2437147|2437421|N|N|N|N|N| +2437513|AAAAAAAAJIBDFCAA|1961-08-01|739|3214|247|1961|2|8|1|3|1961|247|3214|Tuesday|1961Q3|N|N|N|2437513|2437724|2437148|2437422|N|N|N|N|N| +2437514|AAAAAAAAKIBDFCAA|1961-08-02|739|3214|247|1961|3|8|2|3|1961|247|3214|Wednesday|1961Q3|N|N|N|2437513|2437724|2437149|2437423|N|N|N|N|N| +2437515|AAAAAAAALIBDFCAA|1961-08-03|739|3214|247|1961|4|8|3|3|1961|247|3214|Thursday|1961Q3|N|N|N|2437513|2437724|2437150|2437424|N|N|N|N|N| +2437516|AAAAAAAAMIBDFCAA|1961-08-04|739|3214|247|1961|5|8|4|3|1961|247|3214|Friday|1961Q3|N|Y|N|2437513|2437724|2437151|2437425|N|N|N|N|N| +2437517|AAAAAAAANIBDFCAA|1961-08-05|739|3214|247|1961|6|8|5|3|1961|247|3214|Saturday|1961Q3|N|Y|N|2437513|2437724|2437152|2437426|N|N|N|N|N| +2437518|AAAAAAAAOIBDFCAA|1961-08-06|739|3214|247|1961|0|8|6|3|1961|247|3214|Sunday|1961Q3|N|N|N|2437513|2437724|2437153|2437427|N|N|N|N|N| +2437519|AAAAAAAAPIBDFCAA|1961-08-07|739|3214|247|1961|1|8|7|3|1961|247|3214|Monday|1961Q3|N|N|N|2437513|2437724|2437154|2437428|N|N|N|N|N| +2437520|AAAAAAAAAJBDFCAA|1961-08-08|739|3215|247|1961|2|8|8|3|1961|247|3215|Tuesday|1961Q3|N|N|N|2437513|2437724|2437155|2437429|N|N|N|N|N| +2437521|AAAAAAAABJBDFCAA|1961-08-09|739|3215|247|1961|3|8|9|3|1961|247|3215|Wednesday|1961Q3|N|N|N|2437513|2437724|2437156|2437430|N|N|N|N|N| +2437522|AAAAAAAACJBDFCAA|1961-08-10|739|3215|247|1961|4|8|10|3|1961|247|3215|Thursday|1961Q3|N|N|N|2437513|2437724|2437157|2437431|N|N|N|N|N| +2437523|AAAAAAAADJBDFCAA|1961-08-11|739|3215|247|1961|5|8|11|3|1961|247|3215|Friday|1961Q3|N|Y|N|2437513|2437724|2437158|2437432|N|N|N|N|N| +2437524|AAAAAAAAEJBDFCAA|1961-08-12|739|3215|247|1961|6|8|12|3|1961|247|3215|Saturday|1961Q3|N|Y|N|2437513|2437724|2437159|2437433|N|N|N|N|N| +2437525|AAAAAAAAFJBDFCAA|1961-08-13|739|3215|247|1961|0|8|13|3|1961|247|3215|Sunday|1961Q3|N|N|N|2437513|2437724|2437160|2437434|N|N|N|N|N| +2437526|AAAAAAAAGJBDFCAA|1961-08-14|739|3215|247|1961|1|8|14|3|1961|247|3215|Monday|1961Q3|N|N|N|2437513|2437724|2437161|2437435|N|N|N|N|N| +2437527|AAAAAAAAHJBDFCAA|1961-08-15|739|3216|247|1961|2|8|15|3|1961|247|3216|Tuesday|1961Q3|N|N|N|2437513|2437724|2437162|2437436|N|N|N|N|N| +2437528|AAAAAAAAIJBDFCAA|1961-08-16|739|3216|247|1961|3|8|16|3|1961|247|3216|Wednesday|1961Q3|N|N|N|2437513|2437724|2437163|2437437|N|N|N|N|N| +2437529|AAAAAAAAJJBDFCAA|1961-08-17|739|3216|247|1961|4|8|17|3|1961|247|3216|Thursday|1961Q3|N|N|N|2437513|2437724|2437164|2437438|N|N|N|N|N| +2437530|AAAAAAAAKJBDFCAA|1961-08-18|739|3216|247|1961|5|8|18|3|1961|247|3216|Friday|1961Q3|N|Y|N|2437513|2437724|2437165|2437439|N|N|N|N|N| +2437531|AAAAAAAALJBDFCAA|1961-08-19|739|3216|247|1961|6|8|19|3|1961|247|3216|Saturday|1961Q3|N|Y|N|2437513|2437724|2437166|2437440|N|N|N|N|N| +2437532|AAAAAAAAMJBDFCAA|1961-08-20|739|3216|247|1961|0|8|20|3|1961|247|3216|Sunday|1961Q3|N|N|N|2437513|2437724|2437167|2437441|N|N|N|N|N| +2437533|AAAAAAAANJBDFCAA|1961-08-21|739|3216|247|1961|1|8|21|3|1961|247|3216|Monday|1961Q3|N|N|N|2437513|2437724|2437168|2437442|N|N|N|N|N| +2437534|AAAAAAAAOJBDFCAA|1961-08-22|739|3217|247|1961|2|8|22|3|1961|247|3217|Tuesday|1961Q3|N|N|N|2437513|2437724|2437169|2437443|N|N|N|N|N| +2437535|AAAAAAAAPJBDFCAA|1961-08-23|739|3217|247|1961|3|8|23|3|1961|247|3217|Wednesday|1961Q3|N|N|N|2437513|2437724|2437170|2437444|N|N|N|N|N| +2437536|AAAAAAAAAKBDFCAA|1961-08-24|739|3217|247|1961|4|8|24|3|1961|247|3217|Thursday|1961Q3|N|N|N|2437513|2437724|2437171|2437445|N|N|N|N|N| +2437537|AAAAAAAABKBDFCAA|1961-08-25|739|3217|247|1961|5|8|25|3|1961|247|3217|Friday|1961Q3|N|Y|N|2437513|2437724|2437172|2437446|N|N|N|N|N| +2437538|AAAAAAAACKBDFCAA|1961-08-26|739|3217|247|1961|6|8|26|3|1961|247|3217|Saturday|1961Q3|N|Y|N|2437513|2437724|2437173|2437447|N|N|N|N|N| +2437539|AAAAAAAADKBDFCAA|1961-08-27|739|3217|247|1961|0|8|27|3|1961|247|3217|Sunday|1961Q3|N|N|N|2437513|2437724|2437174|2437448|N|N|N|N|N| +2437540|AAAAAAAAEKBDFCAA|1961-08-28|739|3217|247|1961|1|8|28|3|1961|247|3217|Monday|1961Q3|N|N|N|2437513|2437724|2437175|2437449|N|N|N|N|N| +2437541|AAAAAAAAFKBDFCAA|1961-08-29|739|3218|247|1961|2|8|29|3|1961|247|3218|Tuesday|1961Q3|N|N|N|2437513|2437724|2437176|2437450|N|N|N|N|N| +2437542|AAAAAAAAGKBDFCAA|1961-08-30|739|3218|247|1961|3|8|30|3|1961|247|3218|Wednesday|1961Q3|N|N|N|2437513|2437724|2437177|2437451|N|N|N|N|N| +2437543|AAAAAAAAHKBDFCAA|1961-08-31|739|3218|247|1961|4|8|31|3|1961|247|3218|Thursday|1961Q3|N|N|N|2437513|2437724|2437178|2437452|N|N|N|N|N| +2437544|AAAAAAAAIKBDFCAA|1961-09-01|740|3218|248|1961|5|9|1|3|1961|248|3218|Friday|1961Q3|N|Y|N|2437544|2437786|2437179|2437453|N|N|N|N|N| +2437545|AAAAAAAAJKBDFCAA|1961-09-02|740|3218|248|1961|6|9|2|3|1961|248|3218|Saturday|1961Q3|N|Y|N|2437544|2437786|2437180|2437454|N|N|N|N|N| +2437546|AAAAAAAAKKBDFCAA|1961-09-03|740|3218|248|1961|0|9|3|3|1961|248|3218|Sunday|1961Q3|N|N|N|2437544|2437786|2437181|2437455|N|N|N|N|N| +2437547|AAAAAAAALKBDFCAA|1961-09-04|740|3218|248|1961|1|9|4|3|1961|248|3218|Monday|1961Q3|N|N|N|2437544|2437786|2437182|2437456|N|N|N|N|N| +2437548|AAAAAAAAMKBDFCAA|1961-09-05|740|3219|248|1961|2|9|5|3|1961|248|3219|Tuesday|1961Q3|N|N|N|2437544|2437786|2437183|2437457|N|N|N|N|N| +2437549|AAAAAAAANKBDFCAA|1961-09-06|740|3219|248|1961|3|9|6|3|1961|248|3219|Wednesday|1961Q3|N|N|N|2437544|2437786|2437184|2437458|N|N|N|N|N| +2437550|AAAAAAAAOKBDFCAA|1961-09-07|740|3219|248|1961|4|9|7|3|1961|248|3219|Thursday|1961Q3|N|N|N|2437544|2437786|2437185|2437459|N|N|N|N|N| +2437551|AAAAAAAAPKBDFCAA|1961-09-08|740|3219|248|1961|5|9|8|3|1961|248|3219|Friday|1961Q3|N|Y|N|2437544|2437786|2437186|2437460|N|N|N|N|N| +2437552|AAAAAAAAALBDFCAA|1961-09-09|740|3219|248|1961|6|9|9|3|1961|248|3219|Saturday|1961Q3|N|Y|N|2437544|2437786|2437187|2437461|N|N|N|N|N| +2437553|AAAAAAAABLBDFCAA|1961-09-10|740|3219|248|1961|0|9|10|3|1961|248|3219|Sunday|1961Q3|N|N|N|2437544|2437786|2437188|2437462|N|N|N|N|N| +2437554|AAAAAAAACLBDFCAA|1961-09-11|740|3219|248|1961|1|9|11|3|1961|248|3219|Monday|1961Q3|N|N|N|2437544|2437786|2437189|2437463|N|N|N|N|N| +2437555|AAAAAAAADLBDFCAA|1961-09-12|740|3220|248|1961|2|9|12|3|1961|248|3220|Tuesday|1961Q3|N|N|N|2437544|2437786|2437190|2437464|N|N|N|N|N| +2437556|AAAAAAAAELBDFCAA|1961-09-13|740|3220|248|1961|3|9|13|3|1961|248|3220|Wednesday|1961Q3|N|N|N|2437544|2437786|2437191|2437465|N|N|N|N|N| +2437557|AAAAAAAAFLBDFCAA|1961-09-14|740|3220|248|1961|4|9|14|3|1961|248|3220|Thursday|1961Q3|N|N|N|2437544|2437786|2437192|2437466|N|N|N|N|N| +2437558|AAAAAAAAGLBDFCAA|1961-09-15|740|3220|248|1961|5|9|15|3|1961|248|3220|Friday|1961Q3|N|Y|N|2437544|2437786|2437193|2437467|N|N|N|N|N| +2437559|AAAAAAAAHLBDFCAA|1961-09-16|740|3220|248|1961|6|9|16|3|1961|248|3220|Saturday|1961Q3|N|Y|N|2437544|2437786|2437194|2437468|N|N|N|N|N| +2437560|AAAAAAAAILBDFCAA|1961-09-17|740|3220|248|1961|0|9|17|3|1961|248|3220|Sunday|1961Q3|N|N|N|2437544|2437786|2437195|2437469|N|N|N|N|N| +2437561|AAAAAAAAJLBDFCAA|1961-09-18|740|3220|248|1961|1|9|18|3|1961|248|3220|Monday|1961Q3|N|N|N|2437544|2437786|2437196|2437470|N|N|N|N|N| +2437562|AAAAAAAAKLBDFCAA|1961-09-19|740|3221|248|1961|2|9|19|3|1961|248|3221|Tuesday|1961Q3|N|N|N|2437544|2437786|2437197|2437471|N|N|N|N|N| +2437563|AAAAAAAALLBDFCAA|1961-09-20|740|3221|248|1961|3|9|20|3|1961|248|3221|Wednesday|1961Q3|N|N|N|2437544|2437786|2437198|2437472|N|N|N|N|N| +2437564|AAAAAAAAMLBDFCAA|1961-09-21|740|3221|248|1961|4|9|21|3|1961|248|3221|Thursday|1961Q3|N|N|N|2437544|2437786|2437199|2437473|N|N|N|N|N| +2437565|AAAAAAAANLBDFCAA|1961-09-22|740|3221|248|1961|5|9|22|3|1961|248|3221|Friday|1961Q3|N|Y|N|2437544|2437786|2437200|2437474|N|N|N|N|N| +2437566|AAAAAAAAOLBDFCAA|1961-09-23|740|3221|248|1961|6|9|23|3|1961|248|3221|Saturday|1961Q3|N|Y|N|2437544|2437786|2437201|2437475|N|N|N|N|N| +2437567|AAAAAAAAPLBDFCAA|1961-09-24|740|3221|248|1961|0|9|24|3|1961|248|3221|Sunday|1961Q3|N|N|N|2437544|2437786|2437202|2437476|N|N|N|N|N| +2437568|AAAAAAAAAMBDFCAA|1961-09-25|740|3221|248|1961|1|9|25|3|1961|248|3221|Monday|1961Q3|N|N|N|2437544|2437786|2437203|2437477|N|N|N|N|N| +2437569|AAAAAAAABMBDFCAA|1961-09-26|740|3222|248|1961|2|9|26|3|1961|248|3222|Tuesday|1961Q3|N|N|N|2437544|2437786|2437204|2437478|N|N|N|N|N| +2437570|AAAAAAAACMBDFCAA|1961-09-27|740|3222|248|1961|3|9|27|3|1961|248|3222|Wednesday|1961Q3|N|N|N|2437544|2437786|2437205|2437479|N|N|N|N|N| +2437571|AAAAAAAADMBDFCAA|1961-09-28|740|3222|248|1961|4|9|28|3|1961|248|3222|Thursday|1961Q3|N|N|N|2437544|2437786|2437206|2437480|N|N|N|N|N| +2437572|AAAAAAAAEMBDFCAA|1961-09-29|740|3222|248|1961|5|9|29|3|1961|248|3222|Friday|1961Q3|N|Y|N|2437544|2437786|2437207|2437481|N|N|N|N|N| +2437573|AAAAAAAAFMBDFCAA|1961-09-30|740|3222|248|1961|6|9|30|3|1961|248|3222|Saturday|1961Q3|N|Y|N|2437544|2437786|2437208|2437482|N|N|N|N|N| +2437574|AAAAAAAAGMBDFCAA|1961-10-01|741|3222|248|1961|0|10|1|3|1961|248|3222|Sunday|1961Q3|N|N|N|2437574|2437846|2437209|2437482|N|N|N|N|N| +2437575|AAAAAAAAHMBDFCAA|1961-10-02|741|3222|248|1961|1|10|2|4|1961|248|3222|Monday|1961Q4|N|N|N|2437574|2437846|2437210|2437483|N|N|N|N|N| +2437576|AAAAAAAAIMBDFCAA|1961-10-03|741|3223|248|1961|2|10|3|4|1961|248|3223|Tuesday|1961Q4|N|N|N|2437574|2437846|2437211|2437484|N|N|N|N|N| +2437577|AAAAAAAAJMBDFCAA|1961-10-04|741|3223|248|1961|3|10|4|4|1961|248|3223|Wednesday|1961Q4|N|N|N|2437574|2437846|2437212|2437485|N|N|N|N|N| +2437578|AAAAAAAAKMBDFCAA|1961-10-05|741|3223|248|1961|4|10|5|4|1961|248|3223|Thursday|1961Q4|N|N|N|2437574|2437846|2437213|2437486|N|N|N|N|N| +2437579|AAAAAAAALMBDFCAA|1961-10-06|741|3223|248|1961|5|10|6|4|1961|248|3223|Friday|1961Q4|N|Y|N|2437574|2437846|2437214|2437487|N|N|N|N|N| +2437580|AAAAAAAAMMBDFCAA|1961-10-07|741|3223|248|1961|6|10|7|4|1961|248|3223|Saturday|1961Q4|N|Y|N|2437574|2437846|2437215|2437488|N|N|N|N|N| +2437581|AAAAAAAANMBDFCAA|1961-10-08|741|3223|248|1961|0|10|8|4|1961|248|3223|Sunday|1961Q4|N|N|N|2437574|2437846|2437216|2437489|N|N|N|N|N| +2437582|AAAAAAAAOMBDFCAA|1961-10-09|741|3223|248|1961|1|10|9|4|1961|248|3223|Monday|1961Q4|N|N|N|2437574|2437846|2437217|2437490|N|N|N|N|N| +2437583|AAAAAAAAPMBDFCAA|1961-10-10|741|3224|248|1961|2|10|10|4|1961|248|3224|Tuesday|1961Q4|N|N|N|2437574|2437846|2437218|2437491|N|N|N|N|N| +2437584|AAAAAAAAANBDFCAA|1961-10-11|741|3224|248|1961|3|10|11|4|1961|248|3224|Wednesday|1961Q4|N|N|N|2437574|2437846|2437219|2437492|N|N|N|N|N| +2437585|AAAAAAAABNBDFCAA|1961-10-12|741|3224|248|1961|4|10|12|4|1961|248|3224|Thursday|1961Q4|N|N|N|2437574|2437846|2437220|2437493|N|N|N|N|N| +2437586|AAAAAAAACNBDFCAA|1961-10-13|741|3224|248|1961|5|10|13|4|1961|248|3224|Friday|1961Q4|N|Y|N|2437574|2437846|2437221|2437494|N|N|N|N|N| +2437587|AAAAAAAADNBDFCAA|1961-10-14|741|3224|248|1961|6|10|14|4|1961|248|3224|Saturday|1961Q4|N|Y|N|2437574|2437846|2437222|2437495|N|N|N|N|N| +2437588|AAAAAAAAENBDFCAA|1961-10-15|741|3224|248|1961|0|10|15|4|1961|248|3224|Sunday|1961Q4|N|N|N|2437574|2437846|2437223|2437496|N|N|N|N|N| +2437589|AAAAAAAAFNBDFCAA|1961-10-16|741|3224|248|1961|1|10|16|4|1961|248|3224|Monday|1961Q4|N|N|N|2437574|2437846|2437224|2437497|N|N|N|N|N| +2437590|AAAAAAAAGNBDFCAA|1961-10-17|741|3225|248|1961|2|10|17|4|1961|248|3225|Tuesday|1961Q4|N|N|N|2437574|2437846|2437225|2437498|N|N|N|N|N| +2437591|AAAAAAAAHNBDFCAA|1961-10-18|741|3225|248|1961|3|10|18|4|1961|248|3225|Wednesday|1961Q4|N|N|N|2437574|2437846|2437226|2437499|N|N|N|N|N| +2437592|AAAAAAAAINBDFCAA|1961-10-19|741|3225|248|1961|4|10|19|4|1961|248|3225|Thursday|1961Q4|N|N|N|2437574|2437846|2437227|2437500|N|N|N|N|N| +2437593|AAAAAAAAJNBDFCAA|1961-10-20|741|3225|248|1961|5|10|20|4|1961|248|3225|Friday|1961Q4|N|Y|N|2437574|2437846|2437228|2437501|N|N|N|N|N| +2437594|AAAAAAAAKNBDFCAA|1961-10-21|741|3225|248|1961|6|10|21|4|1961|248|3225|Saturday|1961Q4|N|Y|N|2437574|2437846|2437229|2437502|N|N|N|N|N| +2437595|AAAAAAAALNBDFCAA|1961-10-22|741|3225|248|1961|0|10|22|4|1961|248|3225|Sunday|1961Q4|N|N|N|2437574|2437846|2437230|2437503|N|N|N|N|N| +2437596|AAAAAAAAMNBDFCAA|1961-10-23|741|3225|248|1961|1|10|23|4|1961|248|3225|Monday|1961Q4|N|N|N|2437574|2437846|2437231|2437504|N|N|N|N|N| +2437597|AAAAAAAANNBDFCAA|1961-10-24|741|3226|248|1961|2|10|24|4|1961|248|3226|Tuesday|1961Q4|N|N|N|2437574|2437846|2437232|2437505|N|N|N|N|N| +2437598|AAAAAAAAONBDFCAA|1961-10-25|741|3226|248|1961|3|10|25|4|1961|248|3226|Wednesday|1961Q4|N|N|N|2437574|2437846|2437233|2437506|N|N|N|N|N| +2437599|AAAAAAAAPNBDFCAA|1961-10-26|741|3226|248|1961|4|10|26|4|1961|248|3226|Thursday|1961Q4|N|N|N|2437574|2437846|2437234|2437507|N|N|N|N|N| +2437600|AAAAAAAAAOBDFCAA|1961-10-27|741|3226|248|1961|5|10|27|4|1961|248|3226|Friday|1961Q4|N|Y|N|2437574|2437846|2437235|2437508|N|N|N|N|N| +2437601|AAAAAAAABOBDFCAA|1961-10-28|741|3226|248|1961|6|10|28|4|1961|248|3226|Saturday|1961Q4|N|Y|N|2437574|2437846|2437236|2437509|N|N|N|N|N| +2437602|AAAAAAAACOBDFCAA|1961-10-29|741|3226|248|1961|0|10|29|4|1961|248|3226|Sunday|1961Q4|N|N|N|2437574|2437846|2437237|2437510|N|N|N|N|N| +2437603|AAAAAAAADOBDFCAA|1961-10-30|741|3226|248|1961|1|10|30|4|1961|248|3226|Monday|1961Q4|N|N|N|2437574|2437846|2437238|2437511|N|N|N|N|N| +2437604|AAAAAAAAEOBDFCAA|1961-10-31|741|3227|248|1961|2|10|31|4|1961|248|3227|Tuesday|1961Q4|N|N|N|2437574|2437846|2437239|2437512|N|N|N|N|N| +2437605|AAAAAAAAFOBDFCAA|1961-11-01|742|3227|248|1961|3|11|1|4|1961|248|3227|Wednesday|1961Q4|N|N|N|2437605|2437908|2437240|2437513|N|N|N|N|N| +2437606|AAAAAAAAGOBDFCAA|1961-11-02|742|3227|248|1961|4|11|2|4|1961|248|3227|Thursday|1961Q4|N|N|N|2437605|2437908|2437241|2437514|N|N|N|N|N| +2437607|AAAAAAAAHOBDFCAA|1961-11-03|742|3227|248|1961|5|11|3|4|1961|248|3227|Friday|1961Q4|N|Y|N|2437605|2437908|2437242|2437515|N|N|N|N|N| +2437608|AAAAAAAAIOBDFCAA|1961-11-04|742|3227|248|1961|6|11|4|4|1961|248|3227|Saturday|1961Q4|N|Y|N|2437605|2437908|2437243|2437516|N|N|N|N|N| +2437609|AAAAAAAAJOBDFCAA|1961-11-05|742|3227|248|1961|0|11|5|4|1961|248|3227|Sunday|1961Q4|N|N|N|2437605|2437908|2437244|2437517|N|N|N|N|N| +2437610|AAAAAAAAKOBDFCAA|1961-11-06|742|3227|248|1961|1|11|6|4|1961|248|3227|Monday|1961Q4|N|N|N|2437605|2437908|2437245|2437518|N|N|N|N|N| +2437611|AAAAAAAALOBDFCAA|1961-11-07|742|3228|248|1961|2|11|7|4|1961|248|3228|Tuesday|1961Q4|N|N|N|2437605|2437908|2437246|2437519|N|N|N|N|N| +2437612|AAAAAAAAMOBDFCAA|1961-11-08|742|3228|248|1961|3|11|8|4|1961|248|3228|Wednesday|1961Q4|N|N|N|2437605|2437908|2437247|2437520|N|N|N|N|N| +2437613|AAAAAAAANOBDFCAA|1961-11-09|742|3228|248|1961|4|11|9|4|1961|248|3228|Thursday|1961Q4|N|N|N|2437605|2437908|2437248|2437521|N|N|N|N|N| +2437614|AAAAAAAAOOBDFCAA|1961-11-10|742|3228|248|1961|5|11|10|4|1961|248|3228|Friday|1961Q4|N|Y|N|2437605|2437908|2437249|2437522|N|N|N|N|N| +2437615|AAAAAAAAPOBDFCAA|1961-11-11|742|3228|248|1961|6|11|11|4|1961|248|3228|Saturday|1961Q4|N|Y|N|2437605|2437908|2437250|2437523|N|N|N|N|N| +2437616|AAAAAAAAAPBDFCAA|1961-11-12|742|3228|248|1961|0|11|12|4|1961|248|3228|Sunday|1961Q4|N|N|N|2437605|2437908|2437251|2437524|N|N|N|N|N| +2437617|AAAAAAAABPBDFCAA|1961-11-13|742|3228|248|1961|1|11|13|4|1961|248|3228|Monday|1961Q4|N|N|N|2437605|2437908|2437252|2437525|N|N|N|N|N| +2437618|AAAAAAAACPBDFCAA|1961-11-14|742|3229|248|1961|2|11|14|4|1961|248|3229|Tuesday|1961Q4|N|N|N|2437605|2437908|2437253|2437526|N|N|N|N|N| +2437619|AAAAAAAADPBDFCAA|1961-11-15|742|3229|248|1961|3|11|15|4|1961|248|3229|Wednesday|1961Q4|N|N|N|2437605|2437908|2437254|2437527|N|N|N|N|N| +2437620|AAAAAAAAEPBDFCAA|1961-11-16|742|3229|248|1961|4|11|16|4|1961|248|3229|Thursday|1961Q4|N|N|N|2437605|2437908|2437255|2437528|N|N|N|N|N| +2437621|AAAAAAAAFPBDFCAA|1961-11-17|742|3229|248|1961|5|11|17|4|1961|248|3229|Friday|1961Q4|N|Y|N|2437605|2437908|2437256|2437529|N|N|N|N|N| +2437622|AAAAAAAAGPBDFCAA|1961-11-18|742|3229|248|1961|6|11|18|4|1961|248|3229|Saturday|1961Q4|N|Y|N|2437605|2437908|2437257|2437530|N|N|N|N|N| +2437623|AAAAAAAAHPBDFCAA|1961-11-19|742|3229|248|1961|0|11|19|4|1961|248|3229|Sunday|1961Q4|N|N|N|2437605|2437908|2437258|2437531|N|N|N|N|N| +2437624|AAAAAAAAIPBDFCAA|1961-11-20|742|3229|248|1961|1|11|20|4|1961|248|3229|Monday|1961Q4|N|N|N|2437605|2437908|2437259|2437532|N|N|N|N|N| +2437625|AAAAAAAAJPBDFCAA|1961-11-21|742|3230|248|1961|2|11|21|4|1961|248|3230|Tuesday|1961Q4|N|N|N|2437605|2437908|2437260|2437533|N|N|N|N|N| +2437626|AAAAAAAAKPBDFCAA|1961-11-22|742|3230|248|1961|3|11|22|4|1961|248|3230|Wednesday|1961Q4|N|N|N|2437605|2437908|2437261|2437534|N|N|N|N|N| +2437627|AAAAAAAALPBDFCAA|1961-11-23|742|3230|248|1961|4|11|23|4|1961|248|3230|Thursday|1961Q4|N|N|N|2437605|2437908|2437262|2437535|N|N|N|N|N| +2437628|AAAAAAAAMPBDFCAA|1961-11-24|742|3230|248|1961|5|11|24|4|1961|248|3230|Friday|1961Q4|N|Y|N|2437605|2437908|2437263|2437536|N|N|N|N|N| +2437629|AAAAAAAANPBDFCAA|1961-11-25|742|3230|248|1961|6|11|25|4|1961|248|3230|Saturday|1961Q4|N|Y|N|2437605|2437908|2437264|2437537|N|N|N|N|N| +2437630|AAAAAAAAOPBDFCAA|1961-11-26|742|3230|248|1961|0|11|26|4|1961|248|3230|Sunday|1961Q4|N|N|N|2437605|2437908|2437265|2437538|N|N|N|N|N| +2437631|AAAAAAAAPPBDFCAA|1961-11-27|742|3230|248|1961|1|11|27|4|1961|248|3230|Monday|1961Q4|N|N|N|2437605|2437908|2437266|2437539|N|N|N|N|N| +2437632|AAAAAAAAAACDFCAA|1961-11-28|742|3231|248|1961|2|11|28|4|1961|248|3231|Tuesday|1961Q4|N|N|N|2437605|2437908|2437267|2437540|N|N|N|N|N| +2437633|AAAAAAAABACDFCAA|1961-11-29|742|3231|248|1961|3|11|29|4|1961|248|3231|Wednesday|1961Q4|N|N|N|2437605|2437908|2437268|2437541|N|N|N|N|N| +2437634|AAAAAAAACACDFCAA|1961-11-30|742|3231|248|1961|4|11|30|4|1961|248|3231|Thursday|1961Q4|N|N|N|2437605|2437908|2437269|2437542|N|N|N|N|N| +2437635|AAAAAAAADACDFCAA|1961-12-01|743|3231|249|1961|5|12|1|4|1961|249|3231|Friday|1961Q4|N|Y|N|2437635|2437968|2437270|2437543|N|N|N|N|N| +2437636|AAAAAAAAEACDFCAA|1961-12-02|743|3231|249|1961|6|12|2|4|1961|249|3231|Saturday|1961Q4|N|Y|N|2437635|2437968|2437271|2437544|N|N|N|N|N| +2437637|AAAAAAAAFACDFCAA|1961-12-03|743|3231|249|1961|0|12|3|4|1961|249|3231|Sunday|1961Q4|N|N|N|2437635|2437968|2437272|2437545|N|N|N|N|N| +2437638|AAAAAAAAGACDFCAA|1961-12-04|743|3231|249|1961|1|12|4|4|1961|249|3231|Monday|1961Q4|N|N|N|2437635|2437968|2437273|2437546|N|N|N|N|N| +2437639|AAAAAAAAHACDFCAA|1961-12-05|743|3232|249|1961|2|12|5|4|1961|249|3232|Tuesday|1961Q4|N|N|N|2437635|2437968|2437274|2437547|N|N|N|N|N| +2437640|AAAAAAAAIACDFCAA|1961-12-06|743|3232|249|1961|3|12|6|4|1961|249|3232|Wednesday|1961Q4|N|N|N|2437635|2437968|2437275|2437548|N|N|N|N|N| +2437641|AAAAAAAAJACDFCAA|1961-12-07|743|3232|249|1961|4|12|7|4|1961|249|3232|Thursday|1961Q4|N|N|N|2437635|2437968|2437276|2437549|N|N|N|N|N| +2437642|AAAAAAAAKACDFCAA|1961-12-08|743|3232|249|1961|5|12|8|4|1961|249|3232|Friday|1961Q4|N|Y|N|2437635|2437968|2437277|2437550|N|N|N|N|N| +2437643|AAAAAAAALACDFCAA|1961-12-09|743|3232|249|1961|6|12|9|4|1961|249|3232|Saturday|1961Q4|N|Y|N|2437635|2437968|2437278|2437551|N|N|N|N|N| +2437644|AAAAAAAAMACDFCAA|1961-12-10|743|3232|249|1961|0|12|10|4|1961|249|3232|Sunday|1961Q4|N|N|N|2437635|2437968|2437279|2437552|N|N|N|N|N| +2437645|AAAAAAAANACDFCAA|1961-12-11|743|3232|249|1961|1|12|11|4|1961|249|3232|Monday|1961Q4|N|N|N|2437635|2437968|2437280|2437553|N|N|N|N|N| +2437646|AAAAAAAAOACDFCAA|1961-12-12|743|3233|249|1961|2|12|12|4|1961|249|3233|Tuesday|1961Q4|N|N|N|2437635|2437968|2437281|2437554|N|N|N|N|N| +2437647|AAAAAAAAPACDFCAA|1961-12-13|743|3233|249|1961|3|12|13|4|1961|249|3233|Wednesday|1961Q4|N|N|N|2437635|2437968|2437282|2437555|N|N|N|N|N| +2437648|AAAAAAAAABCDFCAA|1961-12-14|743|3233|249|1961|4|12|14|4|1961|249|3233|Thursday|1961Q4|N|N|N|2437635|2437968|2437283|2437556|N|N|N|N|N| +2437649|AAAAAAAABBCDFCAA|1961-12-15|743|3233|249|1961|5|12|15|4|1961|249|3233|Friday|1961Q4|N|Y|N|2437635|2437968|2437284|2437557|N|N|N|N|N| +2437650|AAAAAAAACBCDFCAA|1961-12-16|743|3233|249|1961|6|12|16|4|1961|249|3233|Saturday|1961Q4|N|Y|N|2437635|2437968|2437285|2437558|N|N|N|N|N| +2437651|AAAAAAAADBCDFCAA|1961-12-17|743|3233|249|1961|0|12|17|4|1961|249|3233|Sunday|1961Q4|N|N|N|2437635|2437968|2437286|2437559|N|N|N|N|N| +2437652|AAAAAAAAEBCDFCAA|1961-12-18|743|3233|249|1961|1|12|18|4|1961|249|3233|Monday|1961Q4|N|N|N|2437635|2437968|2437287|2437560|N|N|N|N|N| +2437653|AAAAAAAAFBCDFCAA|1961-12-19|743|3234|249|1961|2|12|19|4|1961|249|3234|Tuesday|1961Q4|N|N|N|2437635|2437968|2437288|2437561|N|N|N|N|N| +2437654|AAAAAAAAGBCDFCAA|1961-12-20|743|3234|249|1961|3|12|20|4|1961|249|3234|Wednesday|1961Q4|N|N|N|2437635|2437968|2437289|2437562|N|N|N|N|N| +2437655|AAAAAAAAHBCDFCAA|1961-12-21|743|3234|249|1961|4|12|21|4|1961|249|3234|Thursday|1961Q4|N|N|N|2437635|2437968|2437290|2437563|N|N|N|N|N| +2437656|AAAAAAAAIBCDFCAA|1961-12-22|743|3234|249|1961|5|12|22|4|1961|249|3234|Friday|1961Q4|N|Y|N|2437635|2437968|2437291|2437564|N|N|N|N|N| +2437657|AAAAAAAAJBCDFCAA|1961-12-23|743|3234|249|1961|6|12|23|4|1961|249|3234|Saturday|1961Q4|N|Y|N|2437635|2437968|2437292|2437565|N|N|N|N|N| +2437658|AAAAAAAAKBCDFCAA|1961-12-24|743|3234|249|1961|0|12|24|4|1961|249|3234|Sunday|1961Q4|N|N|N|2437635|2437968|2437293|2437566|N|N|N|N|N| +2437659|AAAAAAAALBCDFCAA|1961-12-25|743|3234|249|1961|1|12|25|4|1961|249|3234|Monday|1961Q4|N|N|N|2437635|2437968|2437294|2437567|N|N|N|N|N| +2437660|AAAAAAAAMBCDFCAA|1961-12-26|743|3235|249|1961|2|12|26|4|1961|249|3235|Tuesday|1961Q4|Y|N|N|2437635|2437968|2437295|2437568|N|N|N|N|N| +2437661|AAAAAAAANBCDFCAA|1961-12-27|743|3235|249|1961|3|12|27|4|1961|249|3235|Wednesday|1961Q4|N|N|Y|2437635|2437968|2437296|2437569|N|N|N|N|N| +2437662|AAAAAAAAOBCDFCAA|1961-12-28|743|3235|249|1961|4|12|28|4|1961|249|3235|Thursday|1961Q4|N|N|N|2437635|2437968|2437297|2437570|N|N|N|N|N| +2437663|AAAAAAAAPBCDFCAA|1961-12-29|743|3235|249|1961|5|12|29|4|1961|249|3235|Friday|1961Q4|N|Y|N|2437635|2437968|2437298|2437571|N|N|N|N|N| +2437664|AAAAAAAAACCDFCAA|1961-12-30|743|3235|249|1961|6|12|30|4|1961|249|3235|Saturday|1961Q4|N|Y|N|2437635|2437968|2437299|2437572|N|N|N|N|N| +2437665|AAAAAAAABCCDFCAA|1961-12-31|743|3235|249|1961|0|12|31|4|1961|249|3235|Sunday|1961Q4|N|N|N|2437635|2437968|2437300|2437573|N|N|N|N|N| +2437666|AAAAAAAACCCDFCAA|1962-01-01|744|3235|249|1962|1|1|1|1|1962|249|3235|Monday|1962Q1|Y|N|N|2437666|2437665|2437301|2437574|N|N|N|N|N| +2437667|AAAAAAAADCCDFCAA|1962-01-02|744|3236|249|1962|2|1|2|1|1962|249|3236|Tuesday|1962Q1|N|N|Y|2437666|2437665|2437302|2437575|N|N|N|N|N| +2437668|AAAAAAAAECCDFCAA|1962-01-03|744|3236|249|1962|3|1|3|1|1962|249|3236|Wednesday|1962Q1|N|N|N|2437666|2437665|2437303|2437576|N|N|N|N|N| +2437669|AAAAAAAAFCCDFCAA|1962-01-04|744|3236|249|1962|4|1|4|1|1962|249|3236|Thursday|1962Q1|N|N|N|2437666|2437665|2437304|2437577|N|N|N|N|N| +2437670|AAAAAAAAGCCDFCAA|1962-01-05|744|3236|249|1962|5|1|5|1|1962|249|3236|Friday|1962Q1|N|Y|N|2437666|2437665|2437305|2437578|N|N|N|N|N| +2437671|AAAAAAAAHCCDFCAA|1962-01-06|744|3236|249|1962|6|1|6|1|1962|249|3236|Saturday|1962Q1|N|Y|N|2437666|2437665|2437306|2437579|N|N|N|N|N| +2437672|AAAAAAAAICCDFCAA|1962-01-07|744|3236|249|1962|0|1|7|1|1962|249|3236|Sunday|1962Q1|N|N|N|2437666|2437665|2437307|2437580|N|N|N|N|N| +2437673|AAAAAAAAJCCDFCAA|1962-01-08|744|3236|249|1962|1|1|8|1|1962|249|3236|Monday|1962Q1|N|N|N|2437666|2437665|2437308|2437581|N|N|N|N|N| +2437674|AAAAAAAAKCCDFCAA|1962-01-09|744|3237|249|1962|2|1|9|1|1962|249|3237|Tuesday|1962Q1|N|N|N|2437666|2437665|2437309|2437582|N|N|N|N|N| +2437675|AAAAAAAALCCDFCAA|1962-01-10|744|3237|249|1962|3|1|10|1|1962|249|3237|Wednesday|1962Q1|N|N|N|2437666|2437665|2437310|2437583|N|N|N|N|N| +2437676|AAAAAAAAMCCDFCAA|1962-01-11|744|3237|249|1962|4|1|11|1|1962|249|3237|Thursday|1962Q1|N|N|N|2437666|2437665|2437311|2437584|N|N|N|N|N| +2437677|AAAAAAAANCCDFCAA|1962-01-12|744|3237|249|1962|5|1|12|1|1962|249|3237|Friday|1962Q1|N|Y|N|2437666|2437665|2437312|2437585|N|N|N|N|N| +2437678|AAAAAAAAOCCDFCAA|1962-01-13|744|3237|249|1962|6|1|13|1|1962|249|3237|Saturday|1962Q1|N|Y|N|2437666|2437665|2437313|2437586|N|N|N|N|N| +2437679|AAAAAAAAPCCDFCAA|1962-01-14|744|3237|249|1962|0|1|14|1|1962|249|3237|Sunday|1962Q1|N|N|N|2437666|2437665|2437314|2437587|N|N|N|N|N| +2437680|AAAAAAAAADCDFCAA|1962-01-15|744|3237|249|1962|1|1|15|1|1962|249|3237|Monday|1962Q1|N|N|N|2437666|2437665|2437315|2437588|N|N|N|N|N| +2437681|AAAAAAAABDCDFCAA|1962-01-16|744|3238|249|1962|2|1|16|1|1962|249|3238|Tuesday|1962Q1|N|N|N|2437666|2437665|2437316|2437589|N|N|N|N|N| +2437682|AAAAAAAACDCDFCAA|1962-01-17|744|3238|249|1962|3|1|17|1|1962|249|3238|Wednesday|1962Q1|N|N|N|2437666|2437665|2437317|2437590|N|N|N|N|N| +2437683|AAAAAAAADDCDFCAA|1962-01-18|744|3238|249|1962|4|1|18|1|1962|249|3238|Thursday|1962Q1|N|N|N|2437666|2437665|2437318|2437591|N|N|N|N|N| +2437684|AAAAAAAAEDCDFCAA|1962-01-19|744|3238|249|1962|5|1|19|1|1962|249|3238|Friday|1962Q1|N|Y|N|2437666|2437665|2437319|2437592|N|N|N|N|N| +2437685|AAAAAAAAFDCDFCAA|1962-01-20|744|3238|249|1962|6|1|20|1|1962|249|3238|Saturday|1962Q1|N|Y|N|2437666|2437665|2437320|2437593|N|N|N|N|N| +2437686|AAAAAAAAGDCDFCAA|1962-01-21|744|3238|249|1962|0|1|21|1|1962|249|3238|Sunday|1962Q1|N|N|N|2437666|2437665|2437321|2437594|N|N|N|N|N| +2437687|AAAAAAAAHDCDFCAA|1962-01-22|744|3238|249|1962|1|1|22|1|1962|249|3238|Monday|1962Q1|N|N|N|2437666|2437665|2437322|2437595|N|N|N|N|N| +2437688|AAAAAAAAIDCDFCAA|1962-01-23|744|3239|249|1962|2|1|23|1|1962|249|3239|Tuesday|1962Q1|N|N|N|2437666|2437665|2437323|2437596|N|N|N|N|N| +2437689|AAAAAAAAJDCDFCAA|1962-01-24|744|3239|249|1962|3|1|24|1|1962|249|3239|Wednesday|1962Q1|N|N|N|2437666|2437665|2437324|2437597|N|N|N|N|N| +2437690|AAAAAAAAKDCDFCAA|1962-01-25|744|3239|249|1962|4|1|25|1|1962|249|3239|Thursday|1962Q1|N|N|N|2437666|2437665|2437325|2437598|N|N|N|N|N| +2437691|AAAAAAAALDCDFCAA|1962-01-26|744|3239|249|1962|5|1|26|1|1962|249|3239|Friday|1962Q1|N|Y|N|2437666|2437665|2437326|2437599|N|N|N|N|N| +2437692|AAAAAAAAMDCDFCAA|1962-01-27|744|3239|249|1962|6|1|27|1|1962|249|3239|Saturday|1962Q1|N|Y|N|2437666|2437665|2437327|2437600|N|N|N|N|N| +2437693|AAAAAAAANDCDFCAA|1962-01-28|744|3239|249|1962|0|1|28|1|1962|249|3239|Sunday|1962Q1|N|N|N|2437666|2437665|2437328|2437601|N|N|N|N|N| +2437694|AAAAAAAAODCDFCAA|1962-01-29|744|3239|249|1962|1|1|29|1|1962|249|3239|Monday|1962Q1|N|N|N|2437666|2437665|2437329|2437602|N|N|N|N|N| +2437695|AAAAAAAAPDCDFCAA|1962-01-30|744|3240|249|1962|2|1|30|1|1962|249|3240|Tuesday|1962Q1|N|N|N|2437666|2437665|2437330|2437603|N|N|N|N|N| +2437696|AAAAAAAAAECDFCAA|1962-01-31|744|3240|249|1962|3|1|31|1|1962|249|3240|Wednesday|1962Q1|N|N|N|2437666|2437665|2437331|2437604|N|N|N|N|N| +2437697|AAAAAAAABECDFCAA|1962-02-01|745|3240|249|1962|4|2|1|1|1962|249|3240|Thursday|1962Q1|N|N|N|2437697|2437727|2437332|2437605|N|N|N|N|N| +2437698|AAAAAAAACECDFCAA|1962-02-02|745|3240|249|1962|5|2|2|1|1962|249|3240|Friday|1962Q1|N|Y|N|2437697|2437727|2437333|2437606|N|N|N|N|N| +2437699|AAAAAAAADECDFCAA|1962-02-03|745|3240|249|1962|6|2|3|1|1962|249|3240|Saturday|1962Q1|N|Y|N|2437697|2437727|2437334|2437607|N|N|N|N|N| +2437700|AAAAAAAAEECDFCAA|1962-02-04|745|3240|249|1962|0|2|4|1|1962|249|3240|Sunday|1962Q1|N|N|N|2437697|2437727|2437335|2437608|N|N|N|N|N| +2437701|AAAAAAAAFECDFCAA|1962-02-05|745|3240|249|1962|1|2|5|1|1962|249|3240|Monday|1962Q1|N|N|N|2437697|2437727|2437336|2437609|N|N|N|N|N| +2437702|AAAAAAAAGECDFCAA|1962-02-06|745|3241|249|1962|2|2|6|1|1962|249|3241|Tuesday|1962Q1|N|N|N|2437697|2437727|2437337|2437610|N|N|N|N|N| +2437703|AAAAAAAAHECDFCAA|1962-02-07|745|3241|249|1962|3|2|7|1|1962|249|3241|Wednesday|1962Q1|N|N|N|2437697|2437727|2437338|2437611|N|N|N|N|N| +2437704|AAAAAAAAIECDFCAA|1962-02-08|745|3241|249|1962|4|2|8|1|1962|249|3241|Thursday|1962Q1|N|N|N|2437697|2437727|2437339|2437612|N|N|N|N|N| +2437705|AAAAAAAAJECDFCAA|1962-02-09|745|3241|249|1962|5|2|9|1|1962|249|3241|Friday|1962Q1|N|Y|N|2437697|2437727|2437340|2437613|N|N|N|N|N| +2437706|AAAAAAAAKECDFCAA|1962-02-10|745|3241|249|1962|6|2|10|1|1962|249|3241|Saturday|1962Q1|N|Y|N|2437697|2437727|2437341|2437614|N|N|N|N|N| +2437707|AAAAAAAALECDFCAA|1962-02-11|745|3241|249|1962|0|2|11|1|1962|249|3241|Sunday|1962Q1|N|N|N|2437697|2437727|2437342|2437615|N|N|N|N|N| +2437708|AAAAAAAAMECDFCAA|1962-02-12|745|3241|249|1962|1|2|12|1|1962|249|3241|Monday|1962Q1|N|N|N|2437697|2437727|2437343|2437616|N|N|N|N|N| +2437709|AAAAAAAANECDFCAA|1962-02-13|745|3242|249|1962|2|2|13|1|1962|249|3242|Tuesday|1962Q1|N|N|N|2437697|2437727|2437344|2437617|N|N|N|N|N| +2437710|AAAAAAAAOECDFCAA|1962-02-14|745|3242|249|1962|3|2|14|1|1962|249|3242|Wednesday|1962Q1|N|N|N|2437697|2437727|2437345|2437618|N|N|N|N|N| +2437711|AAAAAAAAPECDFCAA|1962-02-15|745|3242|249|1962|4|2|15|1|1962|249|3242|Thursday|1962Q1|N|N|N|2437697|2437727|2437346|2437619|N|N|N|N|N| +2437712|AAAAAAAAAFCDFCAA|1962-02-16|745|3242|249|1962|5|2|16|1|1962|249|3242|Friday|1962Q1|N|Y|N|2437697|2437727|2437347|2437620|N|N|N|N|N| +2437713|AAAAAAAABFCDFCAA|1962-02-17|745|3242|249|1962|6|2|17|1|1962|249|3242|Saturday|1962Q1|N|Y|N|2437697|2437727|2437348|2437621|N|N|N|N|N| +2437714|AAAAAAAACFCDFCAA|1962-02-18|745|3242|249|1962|0|2|18|1|1962|249|3242|Sunday|1962Q1|N|N|N|2437697|2437727|2437349|2437622|N|N|N|N|N| +2437715|AAAAAAAADFCDFCAA|1962-02-19|745|3242|249|1962|1|2|19|1|1962|249|3242|Monday|1962Q1|N|N|N|2437697|2437727|2437350|2437623|N|N|N|N|N| +2437716|AAAAAAAAEFCDFCAA|1962-02-20|745|3243|249|1962|2|2|20|1|1962|249|3243|Tuesday|1962Q1|N|N|N|2437697|2437727|2437351|2437624|N|N|N|N|N| +2437717|AAAAAAAAFFCDFCAA|1962-02-21|745|3243|249|1962|3|2|21|1|1962|249|3243|Wednesday|1962Q1|N|N|N|2437697|2437727|2437352|2437625|N|N|N|N|N| +2437718|AAAAAAAAGFCDFCAA|1962-02-22|745|3243|249|1962|4|2|22|1|1962|249|3243|Thursday|1962Q1|N|N|N|2437697|2437727|2437353|2437626|N|N|N|N|N| +2437719|AAAAAAAAHFCDFCAA|1962-02-23|745|3243|249|1962|5|2|23|1|1962|249|3243|Friday|1962Q1|N|Y|N|2437697|2437727|2437354|2437627|N|N|N|N|N| +2437720|AAAAAAAAIFCDFCAA|1962-02-24|745|3243|249|1962|6|2|24|1|1962|249|3243|Saturday|1962Q1|N|Y|N|2437697|2437727|2437355|2437628|N|N|N|N|N| +2437721|AAAAAAAAJFCDFCAA|1962-02-25|745|3243|249|1962|0|2|25|1|1962|249|3243|Sunday|1962Q1|N|N|N|2437697|2437727|2437356|2437629|N|N|N|N|N| +2437722|AAAAAAAAKFCDFCAA|1962-02-26|745|3243|249|1962|1|2|26|1|1962|249|3243|Monday|1962Q1|N|N|N|2437697|2437727|2437357|2437630|N|N|N|N|N| +2437723|AAAAAAAALFCDFCAA|1962-02-27|745|3244|249|1962|2|2|27|1|1962|249|3244|Tuesday|1962Q1|N|N|N|2437697|2437727|2437358|2437631|N|N|N|N|N| +2437724|AAAAAAAAMFCDFCAA|1962-02-28|745|3244|249|1962|3|2|28|1|1962|249|3244|Wednesday|1962Q1|N|N|N|2437697|2437727|2437359|2437632|N|N|N|N|N| +2437725|AAAAAAAANFCDFCAA|1962-03-01|746|3244|250|1962|4|3|1|1|1962|250|3244|Thursday|1962Q1|N|N|N|2437725|2437783|2437360|2437633|N|N|N|N|N| +2437726|AAAAAAAAOFCDFCAA|1962-03-02|746|3244|250|1962|5|3|2|1|1962|250|3244|Friday|1962Q1|N|Y|N|2437725|2437783|2437361|2437634|N|N|N|N|N| +2437727|AAAAAAAAPFCDFCAA|1962-03-03|746|3244|250|1962|6|3|3|1|1962|250|3244|Saturday|1962Q1|N|Y|N|2437725|2437783|2437362|2437635|N|N|N|N|N| +2437728|AAAAAAAAAGCDFCAA|1962-03-04|746|3244|250|1962|0|3|4|1|1962|250|3244|Sunday|1962Q1|N|N|N|2437725|2437783|2437363|2437636|N|N|N|N|N| +2437729|AAAAAAAABGCDFCAA|1962-03-05|746|3244|250|1962|1|3|5|1|1962|250|3244|Monday|1962Q1|N|N|N|2437725|2437783|2437364|2437637|N|N|N|N|N| +2437730|AAAAAAAACGCDFCAA|1962-03-06|746|3245|250|1962|2|3|6|1|1962|250|3245|Tuesday|1962Q1|N|N|N|2437725|2437783|2437365|2437638|N|N|N|N|N| +2437731|AAAAAAAADGCDFCAA|1962-03-07|746|3245|250|1962|3|3|7|1|1962|250|3245|Wednesday|1962Q1|N|N|N|2437725|2437783|2437366|2437639|N|N|N|N|N| +2437732|AAAAAAAAEGCDFCAA|1962-03-08|746|3245|250|1962|4|3|8|1|1962|250|3245|Thursday|1962Q1|N|N|N|2437725|2437783|2437367|2437640|N|N|N|N|N| +2437733|AAAAAAAAFGCDFCAA|1962-03-09|746|3245|250|1962|5|3|9|1|1962|250|3245|Friday|1962Q1|N|Y|N|2437725|2437783|2437368|2437641|N|N|N|N|N| +2437734|AAAAAAAAGGCDFCAA|1962-03-10|746|3245|250|1962|6|3|10|1|1962|250|3245|Saturday|1962Q1|N|Y|N|2437725|2437783|2437369|2437642|N|N|N|N|N| +2437735|AAAAAAAAHGCDFCAA|1962-03-11|746|3245|250|1962|0|3|11|1|1962|250|3245|Sunday|1962Q1|N|N|N|2437725|2437783|2437370|2437643|N|N|N|N|N| +2437736|AAAAAAAAIGCDFCAA|1962-03-12|746|3245|250|1962|1|3|12|1|1962|250|3245|Monday|1962Q1|N|N|N|2437725|2437783|2437371|2437644|N|N|N|N|N| +2437737|AAAAAAAAJGCDFCAA|1962-03-13|746|3246|250|1962|2|3|13|1|1962|250|3246|Tuesday|1962Q1|N|N|N|2437725|2437783|2437372|2437645|N|N|N|N|N| +2437738|AAAAAAAAKGCDFCAA|1962-03-14|746|3246|250|1962|3|3|14|1|1962|250|3246|Wednesday|1962Q1|N|N|N|2437725|2437783|2437373|2437646|N|N|N|N|N| +2437739|AAAAAAAALGCDFCAA|1962-03-15|746|3246|250|1962|4|3|15|1|1962|250|3246|Thursday|1962Q1|N|N|N|2437725|2437783|2437374|2437647|N|N|N|N|N| +2437740|AAAAAAAAMGCDFCAA|1962-03-16|746|3246|250|1962|5|3|16|1|1962|250|3246|Friday|1962Q1|N|Y|N|2437725|2437783|2437375|2437648|N|N|N|N|N| +2437741|AAAAAAAANGCDFCAA|1962-03-17|746|3246|250|1962|6|3|17|1|1962|250|3246|Saturday|1962Q1|N|Y|N|2437725|2437783|2437376|2437649|N|N|N|N|N| +2437742|AAAAAAAAOGCDFCAA|1962-03-18|746|3246|250|1962|0|3|18|1|1962|250|3246|Sunday|1962Q1|N|N|N|2437725|2437783|2437377|2437650|N|N|N|N|N| +2437743|AAAAAAAAPGCDFCAA|1962-03-19|746|3246|250|1962|1|3|19|1|1962|250|3246|Monday|1962Q1|N|N|N|2437725|2437783|2437378|2437651|N|N|N|N|N| +2437744|AAAAAAAAAHCDFCAA|1962-03-20|746|3247|250|1962|2|3|20|1|1962|250|3247|Tuesday|1962Q1|N|N|N|2437725|2437783|2437379|2437652|N|N|N|N|N| +2437745|AAAAAAAABHCDFCAA|1962-03-21|746|3247|250|1962|3|3|21|1|1962|250|3247|Wednesday|1962Q1|N|N|N|2437725|2437783|2437380|2437653|N|N|N|N|N| +2437746|AAAAAAAACHCDFCAA|1962-03-22|746|3247|250|1962|4|3|22|1|1962|250|3247|Thursday|1962Q1|N|N|N|2437725|2437783|2437381|2437654|N|N|N|N|N| +2437747|AAAAAAAADHCDFCAA|1962-03-23|746|3247|250|1962|5|3|23|1|1962|250|3247|Friday|1962Q1|N|Y|N|2437725|2437783|2437382|2437655|N|N|N|N|N| +2437748|AAAAAAAAEHCDFCAA|1962-03-24|746|3247|250|1962|6|3|24|1|1962|250|3247|Saturday|1962Q1|N|Y|N|2437725|2437783|2437383|2437656|N|N|N|N|N| +2437749|AAAAAAAAFHCDFCAA|1962-03-25|746|3247|250|1962|0|3|25|1|1962|250|3247|Sunday|1962Q1|N|N|N|2437725|2437783|2437384|2437657|N|N|N|N|N| +2437750|AAAAAAAAGHCDFCAA|1962-03-26|746|3247|250|1962|1|3|26|1|1962|250|3247|Monday|1962Q1|N|N|N|2437725|2437783|2437385|2437658|N|N|N|N|N| +2437751|AAAAAAAAHHCDFCAA|1962-03-27|746|3248|250|1962|2|3|27|1|1962|250|3248|Tuesday|1962Q1|N|N|N|2437725|2437783|2437386|2437659|N|N|N|N|N| +2437752|AAAAAAAAIHCDFCAA|1962-03-28|746|3248|250|1962|3|3|28|1|1962|250|3248|Wednesday|1962Q1|N|N|N|2437725|2437783|2437387|2437660|N|N|N|N|N| +2437753|AAAAAAAAJHCDFCAA|1962-03-29|746|3248|250|1962|4|3|29|1|1962|250|3248|Thursday|1962Q1|N|N|N|2437725|2437783|2437388|2437661|N|N|N|N|N| +2437754|AAAAAAAAKHCDFCAA|1962-03-30|746|3248|250|1962|5|3|30|1|1962|250|3248|Friday|1962Q1|N|Y|N|2437725|2437783|2437389|2437662|N|N|N|N|N| +2437755|AAAAAAAALHCDFCAA|1962-03-31|746|3248|250|1962|6|3|31|1|1962|250|3248|Saturday|1962Q1|N|Y|N|2437725|2437783|2437390|2437663|N|N|N|N|N| +2437756|AAAAAAAAMHCDFCAA|1962-04-01|747|3248|250|1962|0|4|1|1|1962|250|3248|Sunday|1962Q1|N|N|N|2437756|2437845|2437391|2437666|N|N|N|N|N| +2437757|AAAAAAAANHCDFCAA|1962-04-02|747|3248|250|1962|1|4|2|2|1962|250|3248|Monday|1962Q2|N|N|N|2437756|2437845|2437392|2437667|N|N|N|N|N| +2437758|AAAAAAAAOHCDFCAA|1962-04-03|747|3249|250|1962|2|4|3|2|1962|250|3249|Tuesday|1962Q2|N|N|N|2437756|2437845|2437393|2437668|N|N|N|N|N| +2437759|AAAAAAAAPHCDFCAA|1962-04-04|747|3249|250|1962|3|4|4|2|1962|250|3249|Wednesday|1962Q2|N|N|N|2437756|2437845|2437394|2437669|N|N|N|N|N| +2437760|AAAAAAAAAICDFCAA|1962-04-05|747|3249|250|1962|4|4|5|2|1962|250|3249|Thursday|1962Q2|N|N|N|2437756|2437845|2437395|2437670|N|N|N|N|N| +2437761|AAAAAAAABICDFCAA|1962-04-06|747|3249|250|1962|5|4|6|2|1962|250|3249|Friday|1962Q2|N|Y|N|2437756|2437845|2437396|2437671|N|N|N|N|N| +2437762|AAAAAAAACICDFCAA|1962-04-07|747|3249|250|1962|6|4|7|2|1962|250|3249|Saturday|1962Q2|N|Y|N|2437756|2437845|2437397|2437672|N|N|N|N|N| +2437763|AAAAAAAADICDFCAA|1962-04-08|747|3249|250|1962|0|4|8|2|1962|250|3249|Sunday|1962Q2|N|N|N|2437756|2437845|2437398|2437673|N|N|N|N|N| +2437764|AAAAAAAAEICDFCAA|1962-04-09|747|3249|250|1962|1|4|9|2|1962|250|3249|Monday|1962Q2|N|N|N|2437756|2437845|2437399|2437674|N|N|N|N|N| +2437765|AAAAAAAAFICDFCAA|1962-04-10|747|3250|250|1962|2|4|10|2|1962|250|3250|Tuesday|1962Q2|N|N|N|2437756|2437845|2437400|2437675|N|N|N|N|N| +2437766|AAAAAAAAGICDFCAA|1962-04-11|747|3250|250|1962|3|4|11|2|1962|250|3250|Wednesday|1962Q2|N|N|N|2437756|2437845|2437401|2437676|N|N|N|N|N| +2437767|AAAAAAAAHICDFCAA|1962-04-12|747|3250|250|1962|4|4|12|2|1962|250|3250|Thursday|1962Q2|N|N|N|2437756|2437845|2437402|2437677|N|N|N|N|N| +2437768|AAAAAAAAIICDFCAA|1962-04-13|747|3250|250|1962|5|4|13|2|1962|250|3250|Friday|1962Q2|N|Y|N|2437756|2437845|2437403|2437678|N|N|N|N|N| +2437769|AAAAAAAAJICDFCAA|1962-04-14|747|3250|250|1962|6|4|14|2|1962|250|3250|Saturday|1962Q2|N|Y|N|2437756|2437845|2437404|2437679|N|N|N|N|N| +2437770|AAAAAAAAKICDFCAA|1962-04-15|747|3250|250|1962|0|4|15|2|1962|250|3250|Sunday|1962Q2|N|N|N|2437756|2437845|2437405|2437680|N|N|N|N|N| +2437771|AAAAAAAALICDFCAA|1962-04-16|747|3250|250|1962|1|4|16|2|1962|250|3250|Monday|1962Q2|N|N|N|2437756|2437845|2437406|2437681|N|N|N|N|N| +2437772|AAAAAAAAMICDFCAA|1962-04-17|747|3251|250|1962|2|4|17|2|1962|250|3251|Tuesday|1962Q2|N|N|N|2437756|2437845|2437407|2437682|N|N|N|N|N| +2437773|AAAAAAAANICDFCAA|1962-04-18|747|3251|250|1962|3|4|18|2|1962|250|3251|Wednesday|1962Q2|N|N|N|2437756|2437845|2437408|2437683|N|N|N|N|N| +2437774|AAAAAAAAOICDFCAA|1962-04-19|747|3251|250|1962|4|4|19|2|1962|250|3251|Thursday|1962Q2|N|N|N|2437756|2437845|2437409|2437684|N|N|N|N|N| +2437775|AAAAAAAAPICDFCAA|1962-04-20|747|3251|250|1962|5|4|20|2|1962|250|3251|Friday|1962Q2|N|Y|N|2437756|2437845|2437410|2437685|N|N|N|N|N| +2437776|AAAAAAAAAJCDFCAA|1962-04-21|747|3251|250|1962|6|4|21|2|1962|250|3251|Saturday|1962Q2|N|Y|N|2437756|2437845|2437411|2437686|N|N|N|N|N| +2437777|AAAAAAAABJCDFCAA|1962-04-22|747|3251|250|1962|0|4|22|2|1962|250|3251|Sunday|1962Q2|N|N|N|2437756|2437845|2437412|2437687|N|N|N|N|N| +2437778|AAAAAAAACJCDFCAA|1962-04-23|747|3251|250|1962|1|4|23|2|1962|250|3251|Monday|1962Q2|N|N|N|2437756|2437845|2437413|2437688|N|N|N|N|N| +2437779|AAAAAAAADJCDFCAA|1962-04-24|747|3252|250|1962|2|4|24|2|1962|250|3252|Tuesday|1962Q2|N|N|N|2437756|2437845|2437414|2437689|N|N|N|N|N| +2437780|AAAAAAAAEJCDFCAA|1962-04-25|747|3252|250|1962|3|4|25|2|1962|250|3252|Wednesday|1962Q2|N|N|N|2437756|2437845|2437415|2437690|N|N|N|N|N| +2437781|AAAAAAAAFJCDFCAA|1962-04-26|747|3252|250|1962|4|4|26|2|1962|250|3252|Thursday|1962Q2|N|N|N|2437756|2437845|2437416|2437691|N|N|N|N|N| +2437782|AAAAAAAAGJCDFCAA|1962-04-27|747|3252|250|1962|5|4|27|2|1962|250|3252|Friday|1962Q2|N|Y|N|2437756|2437845|2437417|2437692|N|N|N|N|N| +2437783|AAAAAAAAHJCDFCAA|1962-04-28|747|3252|250|1962|6|4|28|2|1962|250|3252|Saturday|1962Q2|N|Y|N|2437756|2437845|2437418|2437693|N|N|N|N|N| +2437784|AAAAAAAAIJCDFCAA|1962-04-29|747|3252|250|1962|0|4|29|2|1962|250|3252|Sunday|1962Q2|N|N|N|2437756|2437845|2437419|2437694|N|N|N|N|N| +2437785|AAAAAAAAJJCDFCAA|1962-04-30|747|3252|250|1962|1|4|30|2|1962|250|3252|Monday|1962Q2|N|N|N|2437756|2437845|2437420|2437695|N|N|N|N|N| +2437786|AAAAAAAAKJCDFCAA|1962-05-01|748|3253|250|1962|2|5|1|2|1962|250|3253|Tuesday|1962Q2|N|N|N|2437786|2437905|2437421|2437696|N|N|N|N|N| +2437787|AAAAAAAALJCDFCAA|1962-05-02|748|3253|250|1962|3|5|2|2|1962|250|3253|Wednesday|1962Q2|N|N|N|2437786|2437905|2437422|2437697|N|N|N|N|N| +2437788|AAAAAAAAMJCDFCAA|1962-05-03|748|3253|250|1962|4|5|3|2|1962|250|3253|Thursday|1962Q2|N|N|N|2437786|2437905|2437423|2437698|N|N|N|N|N| +2437789|AAAAAAAANJCDFCAA|1962-05-04|748|3253|250|1962|5|5|4|2|1962|250|3253|Friday|1962Q2|N|Y|N|2437786|2437905|2437424|2437699|N|N|N|N|N| +2437790|AAAAAAAAOJCDFCAA|1962-05-05|748|3253|250|1962|6|5|5|2|1962|250|3253|Saturday|1962Q2|N|Y|N|2437786|2437905|2437425|2437700|N|N|N|N|N| +2437791|AAAAAAAAPJCDFCAA|1962-05-06|748|3253|250|1962|0|5|6|2|1962|250|3253|Sunday|1962Q2|N|N|N|2437786|2437905|2437426|2437701|N|N|N|N|N| +2437792|AAAAAAAAAKCDFCAA|1962-05-07|748|3253|250|1962|1|5|7|2|1962|250|3253|Monday|1962Q2|N|N|N|2437786|2437905|2437427|2437702|N|N|N|N|N| +2437793|AAAAAAAABKCDFCAA|1962-05-08|748|3254|250|1962|2|5|8|2|1962|250|3254|Tuesday|1962Q2|N|N|N|2437786|2437905|2437428|2437703|N|N|N|N|N| +2437794|AAAAAAAACKCDFCAA|1962-05-09|748|3254|250|1962|3|5|9|2|1962|250|3254|Wednesday|1962Q2|N|N|N|2437786|2437905|2437429|2437704|N|N|N|N|N| +2437795|AAAAAAAADKCDFCAA|1962-05-10|748|3254|250|1962|4|5|10|2|1962|250|3254|Thursday|1962Q2|N|N|N|2437786|2437905|2437430|2437705|N|N|N|N|N| +2437796|AAAAAAAAEKCDFCAA|1962-05-11|748|3254|250|1962|5|5|11|2|1962|250|3254|Friday|1962Q2|N|Y|N|2437786|2437905|2437431|2437706|N|N|N|N|N| +2437797|AAAAAAAAFKCDFCAA|1962-05-12|748|3254|250|1962|6|5|12|2|1962|250|3254|Saturday|1962Q2|N|Y|N|2437786|2437905|2437432|2437707|N|N|N|N|N| +2437798|AAAAAAAAGKCDFCAA|1962-05-13|748|3254|250|1962|0|5|13|2|1962|250|3254|Sunday|1962Q2|N|N|N|2437786|2437905|2437433|2437708|N|N|N|N|N| +2437799|AAAAAAAAHKCDFCAA|1962-05-14|748|3254|250|1962|1|5|14|2|1962|250|3254|Monday|1962Q2|N|N|N|2437786|2437905|2437434|2437709|N|N|N|N|N| +2437800|AAAAAAAAIKCDFCAA|1962-05-15|748|3255|250|1962|2|5|15|2|1962|250|3255|Tuesday|1962Q2|N|N|N|2437786|2437905|2437435|2437710|N|N|N|N|N| +2437801|AAAAAAAAJKCDFCAA|1962-05-16|748|3255|250|1962|3|5|16|2|1962|250|3255|Wednesday|1962Q2|N|N|N|2437786|2437905|2437436|2437711|N|N|N|N|N| +2437802|AAAAAAAAKKCDFCAA|1962-05-17|748|3255|250|1962|4|5|17|2|1962|250|3255|Thursday|1962Q2|N|N|N|2437786|2437905|2437437|2437712|N|N|N|N|N| +2437803|AAAAAAAALKCDFCAA|1962-05-18|748|3255|250|1962|5|5|18|2|1962|250|3255|Friday|1962Q2|N|Y|N|2437786|2437905|2437438|2437713|N|N|N|N|N| +2437804|AAAAAAAAMKCDFCAA|1962-05-19|748|3255|250|1962|6|5|19|2|1962|250|3255|Saturday|1962Q2|N|Y|N|2437786|2437905|2437439|2437714|N|N|N|N|N| +2437805|AAAAAAAANKCDFCAA|1962-05-20|748|3255|250|1962|0|5|20|2|1962|250|3255|Sunday|1962Q2|N|N|N|2437786|2437905|2437440|2437715|N|N|N|N|N| +2437806|AAAAAAAAOKCDFCAA|1962-05-21|748|3255|250|1962|1|5|21|2|1962|250|3255|Monday|1962Q2|N|N|N|2437786|2437905|2437441|2437716|N|N|N|N|N| +2437807|AAAAAAAAPKCDFCAA|1962-05-22|748|3256|250|1962|2|5|22|2|1962|250|3256|Tuesday|1962Q2|N|N|N|2437786|2437905|2437442|2437717|N|N|N|N|N| +2437808|AAAAAAAAALCDFCAA|1962-05-23|748|3256|250|1962|3|5|23|2|1962|250|3256|Wednesday|1962Q2|N|N|N|2437786|2437905|2437443|2437718|N|N|N|N|N| +2437809|AAAAAAAABLCDFCAA|1962-05-24|748|3256|250|1962|4|5|24|2|1962|250|3256|Thursday|1962Q2|N|N|N|2437786|2437905|2437444|2437719|N|N|N|N|N| +2437810|AAAAAAAACLCDFCAA|1962-05-25|748|3256|250|1962|5|5|25|2|1962|250|3256|Friday|1962Q2|N|Y|N|2437786|2437905|2437445|2437720|N|N|N|N|N| +2437811|AAAAAAAADLCDFCAA|1962-05-26|748|3256|250|1962|6|5|26|2|1962|250|3256|Saturday|1962Q2|N|Y|N|2437786|2437905|2437446|2437721|N|N|N|N|N| +2437812|AAAAAAAAELCDFCAA|1962-05-27|748|3256|250|1962|0|5|27|2|1962|250|3256|Sunday|1962Q2|N|N|N|2437786|2437905|2437447|2437722|N|N|N|N|N| +2437813|AAAAAAAAFLCDFCAA|1962-05-28|748|3256|250|1962|1|5|28|2|1962|250|3256|Monday|1962Q2|N|N|N|2437786|2437905|2437448|2437723|N|N|N|N|N| +2437814|AAAAAAAAGLCDFCAA|1962-05-29|748|3257|250|1962|2|5|29|2|1962|250|3257|Tuesday|1962Q2|N|N|N|2437786|2437905|2437449|2437724|N|N|N|N|N| +2437815|AAAAAAAAHLCDFCAA|1962-05-30|748|3257|250|1962|3|5|30|2|1962|250|3257|Wednesday|1962Q2|N|N|N|2437786|2437905|2437450|2437725|N|N|N|N|N| +2437816|AAAAAAAAILCDFCAA|1962-05-31|748|3257|250|1962|4|5|31|2|1962|250|3257|Thursday|1962Q2|N|N|N|2437786|2437905|2437451|2437726|N|N|N|N|N| +2437817|AAAAAAAAJLCDFCAA|1962-06-01|749|3257|251|1962|5|6|1|2|1962|251|3257|Friday|1962Q2|N|Y|N|2437817|2437967|2437452|2437727|N|N|N|N|N| +2437818|AAAAAAAAKLCDFCAA|1962-06-02|749|3257|251|1962|6|6|2|2|1962|251|3257|Saturday|1962Q2|N|Y|N|2437817|2437967|2437453|2437728|N|N|N|N|N| +2437819|AAAAAAAALLCDFCAA|1962-06-03|749|3257|251|1962|0|6|3|2|1962|251|3257|Sunday|1962Q2|N|N|N|2437817|2437967|2437454|2437729|N|N|N|N|N| +2437820|AAAAAAAAMLCDFCAA|1962-06-04|749|3257|251|1962|1|6|4|2|1962|251|3257|Monday|1962Q2|N|N|N|2437817|2437967|2437455|2437730|N|N|N|N|N| +2437821|AAAAAAAANLCDFCAA|1962-06-05|749|3258|251|1962|2|6|5|2|1962|251|3258|Tuesday|1962Q2|N|N|N|2437817|2437967|2437456|2437731|N|N|N|N|N| +2437822|AAAAAAAAOLCDFCAA|1962-06-06|749|3258|251|1962|3|6|6|2|1962|251|3258|Wednesday|1962Q2|N|N|N|2437817|2437967|2437457|2437732|N|N|N|N|N| +2437823|AAAAAAAAPLCDFCAA|1962-06-07|749|3258|251|1962|4|6|7|2|1962|251|3258|Thursday|1962Q2|N|N|N|2437817|2437967|2437458|2437733|N|N|N|N|N| +2437824|AAAAAAAAAMCDFCAA|1962-06-08|749|3258|251|1962|5|6|8|2|1962|251|3258|Friday|1962Q2|N|Y|N|2437817|2437967|2437459|2437734|N|N|N|N|N| +2437825|AAAAAAAABMCDFCAA|1962-06-09|749|3258|251|1962|6|6|9|2|1962|251|3258|Saturday|1962Q2|N|Y|N|2437817|2437967|2437460|2437735|N|N|N|N|N| +2437826|AAAAAAAACMCDFCAA|1962-06-10|749|3258|251|1962|0|6|10|2|1962|251|3258|Sunday|1962Q2|N|N|N|2437817|2437967|2437461|2437736|N|N|N|N|N| +2437827|AAAAAAAADMCDFCAA|1962-06-11|749|3258|251|1962|1|6|11|2|1962|251|3258|Monday|1962Q2|N|N|N|2437817|2437967|2437462|2437737|N|N|N|N|N| +2437828|AAAAAAAAEMCDFCAA|1962-06-12|749|3259|251|1962|2|6|12|2|1962|251|3259|Tuesday|1962Q2|N|N|N|2437817|2437967|2437463|2437738|N|N|N|N|N| +2437829|AAAAAAAAFMCDFCAA|1962-06-13|749|3259|251|1962|3|6|13|2|1962|251|3259|Wednesday|1962Q2|N|N|N|2437817|2437967|2437464|2437739|N|N|N|N|N| +2437830|AAAAAAAAGMCDFCAA|1962-06-14|749|3259|251|1962|4|6|14|2|1962|251|3259|Thursday|1962Q2|N|N|N|2437817|2437967|2437465|2437740|N|N|N|N|N| +2437831|AAAAAAAAHMCDFCAA|1962-06-15|749|3259|251|1962|5|6|15|2|1962|251|3259|Friday|1962Q2|N|Y|N|2437817|2437967|2437466|2437741|N|N|N|N|N| +2437832|AAAAAAAAIMCDFCAA|1962-06-16|749|3259|251|1962|6|6|16|2|1962|251|3259|Saturday|1962Q2|N|Y|N|2437817|2437967|2437467|2437742|N|N|N|N|N| +2437833|AAAAAAAAJMCDFCAA|1962-06-17|749|3259|251|1962|0|6|17|2|1962|251|3259|Sunday|1962Q2|N|N|N|2437817|2437967|2437468|2437743|N|N|N|N|N| +2437834|AAAAAAAAKMCDFCAA|1962-06-18|749|3259|251|1962|1|6|18|2|1962|251|3259|Monday|1962Q2|N|N|N|2437817|2437967|2437469|2437744|N|N|N|N|N| +2437835|AAAAAAAALMCDFCAA|1962-06-19|749|3260|251|1962|2|6|19|2|1962|251|3260|Tuesday|1962Q2|N|N|N|2437817|2437967|2437470|2437745|N|N|N|N|N| +2437836|AAAAAAAAMMCDFCAA|1962-06-20|749|3260|251|1962|3|6|20|2|1962|251|3260|Wednesday|1962Q2|N|N|N|2437817|2437967|2437471|2437746|N|N|N|N|N| +2437837|AAAAAAAANMCDFCAA|1962-06-21|749|3260|251|1962|4|6|21|2|1962|251|3260|Thursday|1962Q2|N|N|N|2437817|2437967|2437472|2437747|N|N|N|N|N| +2437838|AAAAAAAAOMCDFCAA|1962-06-22|749|3260|251|1962|5|6|22|2|1962|251|3260|Friday|1962Q2|N|Y|N|2437817|2437967|2437473|2437748|N|N|N|N|N| +2437839|AAAAAAAAPMCDFCAA|1962-06-23|749|3260|251|1962|6|6|23|2|1962|251|3260|Saturday|1962Q2|N|Y|N|2437817|2437967|2437474|2437749|N|N|N|N|N| +2437840|AAAAAAAAANCDFCAA|1962-06-24|749|3260|251|1962|0|6|24|2|1962|251|3260|Sunday|1962Q2|N|N|N|2437817|2437967|2437475|2437750|N|N|N|N|N| +2437841|AAAAAAAABNCDFCAA|1962-06-25|749|3260|251|1962|1|6|25|2|1962|251|3260|Monday|1962Q2|N|N|N|2437817|2437967|2437476|2437751|N|N|N|N|N| +2437842|AAAAAAAACNCDFCAA|1962-06-26|749|3261|251|1962|2|6|26|2|1962|251|3261|Tuesday|1962Q2|N|N|N|2437817|2437967|2437477|2437752|N|N|N|N|N| +2437843|AAAAAAAADNCDFCAA|1962-06-27|749|3261|251|1962|3|6|27|2|1962|251|3261|Wednesday|1962Q2|N|N|N|2437817|2437967|2437478|2437753|N|N|N|N|N| +2437844|AAAAAAAAENCDFCAA|1962-06-28|749|3261|251|1962|4|6|28|2|1962|251|3261|Thursday|1962Q2|N|N|N|2437817|2437967|2437479|2437754|N|N|N|N|N| +2437845|AAAAAAAAFNCDFCAA|1962-06-29|749|3261|251|1962|5|6|29|2|1962|251|3261|Friday|1962Q2|N|Y|N|2437817|2437967|2437480|2437755|N|N|N|N|N| +2437846|AAAAAAAAGNCDFCAA|1962-06-30|749|3261|251|1962|6|6|30|2|1962|251|3261|Saturday|1962Q2|N|Y|N|2437817|2437967|2437481|2437756|N|N|N|N|N| +2437847|AAAAAAAAHNCDFCAA|1962-07-01|750|3261|251|1962|0|7|1|2|1962|251|3261|Sunday|1962Q2|N|N|N|2437847|2438027|2437482|2437756|N|N|N|N|N| +2437848|AAAAAAAAINCDFCAA|1962-07-02|750|3261|251|1962|1|7|2|3|1962|251|3261|Monday|1962Q3|N|N|N|2437847|2438027|2437483|2437757|N|N|N|N|N| +2437849|AAAAAAAAJNCDFCAA|1962-07-03|750|3262|251|1962|2|7|3|3|1962|251|3262|Tuesday|1962Q3|N|N|N|2437847|2438027|2437484|2437758|N|N|N|N|N| +2437850|AAAAAAAAKNCDFCAA|1962-07-04|750|3262|251|1962|3|7|4|3|1962|251|3262|Wednesday|1962Q3|N|N|N|2437847|2438027|2437485|2437759|N|N|N|N|N| +2437851|AAAAAAAALNCDFCAA|1962-07-05|750|3262|251|1962|4|7|5|3|1962|251|3262|Thursday|1962Q3|Y|N|N|2437847|2438027|2437486|2437760|N|N|N|N|N| +2437852|AAAAAAAAMNCDFCAA|1962-07-06|750|3262|251|1962|5|7|6|3|1962|251|3262|Friday|1962Q3|N|Y|Y|2437847|2438027|2437487|2437761|N|N|N|N|N| +2437853|AAAAAAAANNCDFCAA|1962-07-07|750|3262|251|1962|6|7|7|3|1962|251|3262|Saturday|1962Q3|N|Y|N|2437847|2438027|2437488|2437762|N|N|N|N|N| +2437854|AAAAAAAAONCDFCAA|1962-07-08|750|3262|251|1962|0|7|8|3|1962|251|3262|Sunday|1962Q3|N|N|N|2437847|2438027|2437489|2437763|N|N|N|N|N| +2437855|AAAAAAAAPNCDFCAA|1962-07-09|750|3262|251|1962|1|7|9|3|1962|251|3262|Monday|1962Q3|N|N|N|2437847|2438027|2437490|2437764|N|N|N|N|N| +2437856|AAAAAAAAAOCDFCAA|1962-07-10|750|3263|251|1962|2|7|10|3|1962|251|3263|Tuesday|1962Q3|N|N|N|2437847|2438027|2437491|2437765|N|N|N|N|N| +2437857|AAAAAAAABOCDFCAA|1962-07-11|750|3263|251|1962|3|7|11|3|1962|251|3263|Wednesday|1962Q3|N|N|N|2437847|2438027|2437492|2437766|N|N|N|N|N| +2437858|AAAAAAAACOCDFCAA|1962-07-12|750|3263|251|1962|4|7|12|3|1962|251|3263|Thursday|1962Q3|N|N|N|2437847|2438027|2437493|2437767|N|N|N|N|N| +2437859|AAAAAAAADOCDFCAA|1962-07-13|750|3263|251|1962|5|7|13|3|1962|251|3263|Friday|1962Q3|N|Y|N|2437847|2438027|2437494|2437768|N|N|N|N|N| +2437860|AAAAAAAAEOCDFCAA|1962-07-14|750|3263|251|1962|6|7|14|3|1962|251|3263|Saturday|1962Q3|N|Y|N|2437847|2438027|2437495|2437769|N|N|N|N|N| +2437861|AAAAAAAAFOCDFCAA|1962-07-15|750|3263|251|1962|0|7|15|3|1962|251|3263|Sunday|1962Q3|N|N|N|2437847|2438027|2437496|2437770|N|N|N|N|N| +2437862|AAAAAAAAGOCDFCAA|1962-07-16|750|3263|251|1962|1|7|16|3|1962|251|3263|Monday|1962Q3|N|N|N|2437847|2438027|2437497|2437771|N|N|N|N|N| +2437863|AAAAAAAAHOCDFCAA|1962-07-17|750|3264|251|1962|2|7|17|3|1962|251|3264|Tuesday|1962Q3|N|N|N|2437847|2438027|2437498|2437772|N|N|N|N|N| +2437864|AAAAAAAAIOCDFCAA|1962-07-18|750|3264|251|1962|3|7|18|3|1962|251|3264|Wednesday|1962Q3|N|N|N|2437847|2438027|2437499|2437773|N|N|N|N|N| +2437865|AAAAAAAAJOCDFCAA|1962-07-19|750|3264|251|1962|4|7|19|3|1962|251|3264|Thursday|1962Q3|N|N|N|2437847|2438027|2437500|2437774|N|N|N|N|N| +2437866|AAAAAAAAKOCDFCAA|1962-07-20|750|3264|251|1962|5|7|20|3|1962|251|3264|Friday|1962Q3|N|Y|N|2437847|2438027|2437501|2437775|N|N|N|N|N| +2437867|AAAAAAAALOCDFCAA|1962-07-21|750|3264|251|1962|6|7|21|3|1962|251|3264|Saturday|1962Q3|N|Y|N|2437847|2438027|2437502|2437776|N|N|N|N|N| +2437868|AAAAAAAAMOCDFCAA|1962-07-22|750|3264|251|1962|0|7|22|3|1962|251|3264|Sunday|1962Q3|N|N|N|2437847|2438027|2437503|2437777|N|N|N|N|N| +2437869|AAAAAAAANOCDFCAA|1962-07-23|750|3264|251|1962|1|7|23|3|1962|251|3264|Monday|1962Q3|N|N|N|2437847|2438027|2437504|2437778|N|N|N|N|N| +2437870|AAAAAAAAOOCDFCAA|1962-07-24|750|3265|251|1962|2|7|24|3|1962|251|3265|Tuesday|1962Q3|N|N|N|2437847|2438027|2437505|2437779|N|N|N|N|N| +2437871|AAAAAAAAPOCDFCAA|1962-07-25|750|3265|251|1962|3|7|25|3|1962|251|3265|Wednesday|1962Q3|N|N|N|2437847|2438027|2437506|2437780|N|N|N|N|N| +2437872|AAAAAAAAAPCDFCAA|1962-07-26|750|3265|251|1962|4|7|26|3|1962|251|3265|Thursday|1962Q3|N|N|N|2437847|2438027|2437507|2437781|N|N|N|N|N| +2437873|AAAAAAAABPCDFCAA|1962-07-27|750|3265|251|1962|5|7|27|3|1962|251|3265|Friday|1962Q3|N|Y|N|2437847|2438027|2437508|2437782|N|N|N|N|N| +2437874|AAAAAAAACPCDFCAA|1962-07-28|750|3265|251|1962|6|7|28|3|1962|251|3265|Saturday|1962Q3|N|Y|N|2437847|2438027|2437509|2437783|N|N|N|N|N| +2437875|AAAAAAAADPCDFCAA|1962-07-29|750|3265|251|1962|0|7|29|3|1962|251|3265|Sunday|1962Q3|N|N|N|2437847|2438027|2437510|2437784|N|N|N|N|N| +2437876|AAAAAAAAEPCDFCAA|1962-07-30|750|3265|251|1962|1|7|30|3|1962|251|3265|Monday|1962Q3|N|N|N|2437847|2438027|2437511|2437785|N|N|N|N|N| +2437877|AAAAAAAAFPCDFCAA|1962-07-31|750|3266|251|1962|2|7|31|3|1962|251|3266|Tuesday|1962Q3|N|N|N|2437847|2438027|2437512|2437786|N|N|N|N|N| +2437878|AAAAAAAAGPCDFCAA|1962-08-01|751|3266|251|1962|3|8|1|3|1962|251|3266|Wednesday|1962Q3|N|N|N|2437878|2438089|2437513|2437787|N|N|N|N|N| +2437879|AAAAAAAAHPCDFCAA|1962-08-02|751|3266|251|1962|4|8|2|3|1962|251|3266|Thursday|1962Q3|N|N|N|2437878|2438089|2437514|2437788|N|N|N|N|N| +2437880|AAAAAAAAIPCDFCAA|1962-08-03|751|3266|251|1962|5|8|3|3|1962|251|3266|Friday|1962Q3|N|Y|N|2437878|2438089|2437515|2437789|N|N|N|N|N| +2437881|AAAAAAAAJPCDFCAA|1962-08-04|751|3266|251|1962|6|8|4|3|1962|251|3266|Saturday|1962Q3|N|Y|N|2437878|2438089|2437516|2437790|N|N|N|N|N| +2437882|AAAAAAAAKPCDFCAA|1962-08-05|751|3266|251|1962|0|8|5|3|1962|251|3266|Sunday|1962Q3|N|N|N|2437878|2438089|2437517|2437791|N|N|N|N|N| +2437883|AAAAAAAALPCDFCAA|1962-08-06|751|3266|251|1962|1|8|6|3|1962|251|3266|Monday|1962Q3|N|N|N|2437878|2438089|2437518|2437792|N|N|N|N|N| +2437884|AAAAAAAAMPCDFCAA|1962-08-07|751|3267|251|1962|2|8|7|3|1962|251|3267|Tuesday|1962Q3|N|N|N|2437878|2438089|2437519|2437793|N|N|N|N|N| +2437885|AAAAAAAANPCDFCAA|1962-08-08|751|3267|251|1962|3|8|8|3|1962|251|3267|Wednesday|1962Q3|N|N|N|2437878|2438089|2437520|2437794|N|N|N|N|N| +2437886|AAAAAAAAOPCDFCAA|1962-08-09|751|3267|251|1962|4|8|9|3|1962|251|3267|Thursday|1962Q3|N|N|N|2437878|2438089|2437521|2437795|N|N|N|N|N| +2437887|AAAAAAAAPPCDFCAA|1962-08-10|751|3267|251|1962|5|8|10|3|1962|251|3267|Friday|1962Q3|N|Y|N|2437878|2438089|2437522|2437796|N|N|N|N|N| +2437888|AAAAAAAAAADDFCAA|1962-08-11|751|3267|251|1962|6|8|11|3|1962|251|3267|Saturday|1962Q3|N|Y|N|2437878|2438089|2437523|2437797|N|N|N|N|N| +2437889|AAAAAAAABADDFCAA|1962-08-12|751|3267|251|1962|0|8|12|3|1962|251|3267|Sunday|1962Q3|N|N|N|2437878|2438089|2437524|2437798|N|N|N|N|N| +2437890|AAAAAAAACADDFCAA|1962-08-13|751|3267|251|1962|1|8|13|3|1962|251|3267|Monday|1962Q3|N|N|N|2437878|2438089|2437525|2437799|N|N|N|N|N| +2437891|AAAAAAAADADDFCAA|1962-08-14|751|3268|251|1962|2|8|14|3|1962|251|3268|Tuesday|1962Q3|N|N|N|2437878|2438089|2437526|2437800|N|N|N|N|N| +2437892|AAAAAAAAEADDFCAA|1962-08-15|751|3268|251|1962|3|8|15|3|1962|251|3268|Wednesday|1962Q3|N|N|N|2437878|2438089|2437527|2437801|N|N|N|N|N| +2437893|AAAAAAAAFADDFCAA|1962-08-16|751|3268|251|1962|4|8|16|3|1962|251|3268|Thursday|1962Q3|N|N|N|2437878|2438089|2437528|2437802|N|N|N|N|N| +2437894|AAAAAAAAGADDFCAA|1962-08-17|751|3268|251|1962|5|8|17|3|1962|251|3268|Friday|1962Q3|N|Y|N|2437878|2438089|2437529|2437803|N|N|N|N|N| +2437895|AAAAAAAAHADDFCAA|1962-08-18|751|3268|251|1962|6|8|18|3|1962|251|3268|Saturday|1962Q3|N|Y|N|2437878|2438089|2437530|2437804|N|N|N|N|N| +2437896|AAAAAAAAIADDFCAA|1962-08-19|751|3268|251|1962|0|8|19|3|1962|251|3268|Sunday|1962Q3|N|N|N|2437878|2438089|2437531|2437805|N|N|N|N|N| +2437897|AAAAAAAAJADDFCAA|1962-08-20|751|3268|251|1962|1|8|20|3|1962|251|3268|Monday|1962Q3|N|N|N|2437878|2438089|2437532|2437806|N|N|N|N|N| +2437898|AAAAAAAAKADDFCAA|1962-08-21|751|3269|251|1962|2|8|21|3|1962|251|3269|Tuesday|1962Q3|N|N|N|2437878|2438089|2437533|2437807|N|N|N|N|N| +2437899|AAAAAAAALADDFCAA|1962-08-22|751|3269|251|1962|3|8|22|3|1962|251|3269|Wednesday|1962Q3|N|N|N|2437878|2438089|2437534|2437808|N|N|N|N|N| +2437900|AAAAAAAAMADDFCAA|1962-08-23|751|3269|251|1962|4|8|23|3|1962|251|3269|Thursday|1962Q3|N|N|N|2437878|2438089|2437535|2437809|N|N|N|N|N| +2437901|AAAAAAAANADDFCAA|1962-08-24|751|3269|251|1962|5|8|24|3|1962|251|3269|Friday|1962Q3|N|Y|N|2437878|2438089|2437536|2437810|N|N|N|N|N| +2437902|AAAAAAAAOADDFCAA|1962-08-25|751|3269|251|1962|6|8|25|3|1962|251|3269|Saturday|1962Q3|N|Y|N|2437878|2438089|2437537|2437811|N|N|N|N|N| +2437903|AAAAAAAAPADDFCAA|1962-08-26|751|3269|251|1962|0|8|26|3|1962|251|3269|Sunday|1962Q3|N|N|N|2437878|2438089|2437538|2437812|N|N|N|N|N| +2437904|AAAAAAAAABDDFCAA|1962-08-27|751|3269|251|1962|1|8|27|3|1962|251|3269|Monday|1962Q3|N|N|N|2437878|2438089|2437539|2437813|N|N|N|N|N| +2437905|AAAAAAAABBDDFCAA|1962-08-28|751|3270|251|1962|2|8|28|3|1962|251|3270|Tuesday|1962Q3|N|N|N|2437878|2438089|2437540|2437814|N|N|N|N|N| +2437906|AAAAAAAACBDDFCAA|1962-08-29|751|3270|251|1962|3|8|29|3|1962|251|3270|Wednesday|1962Q3|N|N|N|2437878|2438089|2437541|2437815|N|N|N|N|N| +2437907|AAAAAAAADBDDFCAA|1962-08-30|751|3270|251|1962|4|8|30|3|1962|251|3270|Thursday|1962Q3|N|N|N|2437878|2438089|2437542|2437816|N|N|N|N|N| +2437908|AAAAAAAAEBDDFCAA|1962-08-31|751|3270|251|1962|5|8|31|3|1962|251|3270|Friday|1962Q3|N|Y|N|2437878|2438089|2437543|2437817|N|N|N|N|N| +2437909|AAAAAAAAFBDDFCAA|1962-09-01|752|3270|252|1962|6|9|1|3|1962|252|3270|Saturday|1962Q3|N|Y|N|2437909|2438151|2437544|2437818|N|N|N|N|N| +2437910|AAAAAAAAGBDDFCAA|1962-09-02|752|3270|252|1962|0|9|2|3|1962|252|3270|Sunday|1962Q3|N|N|N|2437909|2438151|2437545|2437819|N|N|N|N|N| +2437911|AAAAAAAAHBDDFCAA|1962-09-03|752|3270|252|1962|1|9|3|3|1962|252|3270|Monday|1962Q3|N|N|N|2437909|2438151|2437546|2437820|N|N|N|N|N| +2437912|AAAAAAAAIBDDFCAA|1962-09-04|752|3271|252|1962|2|9|4|3|1962|252|3271|Tuesday|1962Q3|N|N|N|2437909|2438151|2437547|2437821|N|N|N|N|N| +2437913|AAAAAAAAJBDDFCAA|1962-09-05|752|3271|252|1962|3|9|5|3|1962|252|3271|Wednesday|1962Q3|N|N|N|2437909|2438151|2437548|2437822|N|N|N|N|N| +2437914|AAAAAAAAKBDDFCAA|1962-09-06|752|3271|252|1962|4|9|6|3|1962|252|3271|Thursday|1962Q3|N|N|N|2437909|2438151|2437549|2437823|N|N|N|N|N| +2437915|AAAAAAAALBDDFCAA|1962-09-07|752|3271|252|1962|5|9|7|3|1962|252|3271|Friday|1962Q3|N|Y|N|2437909|2438151|2437550|2437824|N|N|N|N|N| +2437916|AAAAAAAAMBDDFCAA|1962-09-08|752|3271|252|1962|6|9|8|3|1962|252|3271|Saturday|1962Q3|N|Y|N|2437909|2438151|2437551|2437825|N|N|N|N|N| +2437917|AAAAAAAANBDDFCAA|1962-09-09|752|3271|252|1962|0|9|9|3|1962|252|3271|Sunday|1962Q3|N|N|N|2437909|2438151|2437552|2437826|N|N|N|N|N| +2437918|AAAAAAAAOBDDFCAA|1962-09-10|752|3271|252|1962|1|9|10|3|1962|252|3271|Monday|1962Q3|N|N|N|2437909|2438151|2437553|2437827|N|N|N|N|N| +2437919|AAAAAAAAPBDDFCAA|1962-09-11|752|3272|252|1962|2|9|11|3|1962|252|3272|Tuesday|1962Q3|N|N|N|2437909|2438151|2437554|2437828|N|N|N|N|N| +2437920|AAAAAAAAACDDFCAA|1962-09-12|752|3272|252|1962|3|9|12|3|1962|252|3272|Wednesday|1962Q3|N|N|N|2437909|2438151|2437555|2437829|N|N|N|N|N| +2437921|AAAAAAAABCDDFCAA|1962-09-13|752|3272|252|1962|4|9|13|3|1962|252|3272|Thursday|1962Q3|N|N|N|2437909|2438151|2437556|2437830|N|N|N|N|N| +2437922|AAAAAAAACCDDFCAA|1962-09-14|752|3272|252|1962|5|9|14|3|1962|252|3272|Friday|1962Q3|N|Y|N|2437909|2438151|2437557|2437831|N|N|N|N|N| +2437923|AAAAAAAADCDDFCAA|1962-09-15|752|3272|252|1962|6|9|15|3|1962|252|3272|Saturday|1962Q3|N|Y|N|2437909|2438151|2437558|2437832|N|N|N|N|N| +2437924|AAAAAAAAECDDFCAA|1962-09-16|752|3272|252|1962|0|9|16|3|1962|252|3272|Sunday|1962Q3|N|N|N|2437909|2438151|2437559|2437833|N|N|N|N|N| +2437925|AAAAAAAAFCDDFCAA|1962-09-17|752|3272|252|1962|1|9|17|3|1962|252|3272|Monday|1962Q3|N|N|N|2437909|2438151|2437560|2437834|N|N|N|N|N| +2437926|AAAAAAAAGCDDFCAA|1962-09-18|752|3273|252|1962|2|9|18|3|1962|252|3273|Tuesday|1962Q3|N|N|N|2437909|2438151|2437561|2437835|N|N|N|N|N| +2437927|AAAAAAAAHCDDFCAA|1962-09-19|752|3273|252|1962|3|9|19|3|1962|252|3273|Wednesday|1962Q3|N|N|N|2437909|2438151|2437562|2437836|N|N|N|N|N| +2437928|AAAAAAAAICDDFCAA|1962-09-20|752|3273|252|1962|4|9|20|3|1962|252|3273|Thursday|1962Q3|N|N|N|2437909|2438151|2437563|2437837|N|N|N|N|N| +2437929|AAAAAAAAJCDDFCAA|1962-09-21|752|3273|252|1962|5|9|21|3|1962|252|3273|Friday|1962Q3|N|Y|N|2437909|2438151|2437564|2437838|N|N|N|N|N| +2437930|AAAAAAAAKCDDFCAA|1962-09-22|752|3273|252|1962|6|9|22|3|1962|252|3273|Saturday|1962Q3|N|Y|N|2437909|2438151|2437565|2437839|N|N|N|N|N| +2437931|AAAAAAAALCDDFCAA|1962-09-23|752|3273|252|1962|0|9|23|3|1962|252|3273|Sunday|1962Q3|N|N|N|2437909|2438151|2437566|2437840|N|N|N|N|N| +2437932|AAAAAAAAMCDDFCAA|1962-09-24|752|3273|252|1962|1|9|24|3|1962|252|3273|Monday|1962Q3|N|N|N|2437909|2438151|2437567|2437841|N|N|N|N|N| +2437933|AAAAAAAANCDDFCAA|1962-09-25|752|3274|252|1962|2|9|25|3|1962|252|3274|Tuesday|1962Q3|N|N|N|2437909|2438151|2437568|2437842|N|N|N|N|N| +2437934|AAAAAAAAOCDDFCAA|1962-09-26|752|3274|252|1962|3|9|26|3|1962|252|3274|Wednesday|1962Q3|N|N|N|2437909|2438151|2437569|2437843|N|N|N|N|N| +2437935|AAAAAAAAPCDDFCAA|1962-09-27|752|3274|252|1962|4|9|27|3|1962|252|3274|Thursday|1962Q3|N|N|N|2437909|2438151|2437570|2437844|N|N|N|N|N| +2437936|AAAAAAAAADDDFCAA|1962-09-28|752|3274|252|1962|5|9|28|3|1962|252|3274|Friday|1962Q3|N|Y|N|2437909|2438151|2437571|2437845|N|N|N|N|N| +2437937|AAAAAAAABDDDFCAA|1962-09-29|752|3274|252|1962|6|9|29|3|1962|252|3274|Saturday|1962Q3|N|Y|N|2437909|2438151|2437572|2437846|N|N|N|N|N| +2437938|AAAAAAAACDDDFCAA|1962-09-30|752|3274|252|1962|0|9|30|3|1962|252|3274|Sunday|1962Q3|N|N|N|2437909|2438151|2437573|2437847|N|N|N|N|N| +2437939|AAAAAAAADDDDFCAA|1962-10-01|753|3274|252|1962|1|10|1|3|1962|252|3274|Monday|1962Q3|N|N|N|2437939|2438211|2437574|2437847|N|N|N|N|N| +2437940|AAAAAAAAEDDDFCAA|1962-10-02|753|3275|252|1962|2|10|2|4|1962|252|3275|Tuesday|1962Q4|N|N|N|2437939|2438211|2437575|2437848|N|N|N|N|N| +2437941|AAAAAAAAFDDDFCAA|1962-10-03|753|3275|252|1962|3|10|3|4|1962|252|3275|Wednesday|1962Q4|N|N|N|2437939|2438211|2437576|2437849|N|N|N|N|N| +2437942|AAAAAAAAGDDDFCAA|1962-10-04|753|3275|252|1962|4|10|4|4|1962|252|3275|Thursday|1962Q4|N|N|N|2437939|2438211|2437577|2437850|N|N|N|N|N| +2437943|AAAAAAAAHDDDFCAA|1962-10-05|753|3275|252|1962|5|10|5|4|1962|252|3275|Friday|1962Q4|N|Y|N|2437939|2438211|2437578|2437851|N|N|N|N|N| +2437944|AAAAAAAAIDDDFCAA|1962-10-06|753|3275|252|1962|6|10|6|4|1962|252|3275|Saturday|1962Q4|N|Y|N|2437939|2438211|2437579|2437852|N|N|N|N|N| +2437945|AAAAAAAAJDDDFCAA|1962-10-07|753|3275|252|1962|0|10|7|4|1962|252|3275|Sunday|1962Q4|N|N|N|2437939|2438211|2437580|2437853|N|N|N|N|N| +2437946|AAAAAAAAKDDDFCAA|1962-10-08|753|3275|252|1962|1|10|8|4|1962|252|3275|Monday|1962Q4|N|N|N|2437939|2438211|2437581|2437854|N|N|N|N|N| +2437947|AAAAAAAALDDDFCAA|1962-10-09|753|3276|252|1962|2|10|9|4|1962|252|3276|Tuesday|1962Q4|N|N|N|2437939|2438211|2437582|2437855|N|N|N|N|N| +2437948|AAAAAAAAMDDDFCAA|1962-10-10|753|3276|252|1962|3|10|10|4|1962|252|3276|Wednesday|1962Q4|N|N|N|2437939|2438211|2437583|2437856|N|N|N|N|N| +2437949|AAAAAAAANDDDFCAA|1962-10-11|753|3276|252|1962|4|10|11|4|1962|252|3276|Thursday|1962Q4|N|N|N|2437939|2438211|2437584|2437857|N|N|N|N|N| +2437950|AAAAAAAAODDDFCAA|1962-10-12|753|3276|252|1962|5|10|12|4|1962|252|3276|Friday|1962Q4|N|Y|N|2437939|2438211|2437585|2437858|N|N|N|N|N| +2437951|AAAAAAAAPDDDFCAA|1962-10-13|753|3276|252|1962|6|10|13|4|1962|252|3276|Saturday|1962Q4|N|Y|N|2437939|2438211|2437586|2437859|N|N|N|N|N| +2437952|AAAAAAAAAEDDFCAA|1962-10-14|753|3276|252|1962|0|10|14|4|1962|252|3276|Sunday|1962Q4|N|N|N|2437939|2438211|2437587|2437860|N|N|N|N|N| +2437953|AAAAAAAABEDDFCAA|1962-10-15|753|3276|252|1962|1|10|15|4|1962|252|3276|Monday|1962Q4|N|N|N|2437939|2438211|2437588|2437861|N|N|N|N|N| +2437954|AAAAAAAACEDDFCAA|1962-10-16|753|3277|252|1962|2|10|16|4|1962|252|3277|Tuesday|1962Q4|N|N|N|2437939|2438211|2437589|2437862|N|N|N|N|N| +2437955|AAAAAAAADEDDFCAA|1962-10-17|753|3277|252|1962|3|10|17|4|1962|252|3277|Wednesday|1962Q4|N|N|N|2437939|2438211|2437590|2437863|N|N|N|N|N| +2437956|AAAAAAAAEEDDFCAA|1962-10-18|753|3277|252|1962|4|10|18|4|1962|252|3277|Thursday|1962Q4|N|N|N|2437939|2438211|2437591|2437864|N|N|N|N|N| +2437957|AAAAAAAAFEDDFCAA|1962-10-19|753|3277|252|1962|5|10|19|4|1962|252|3277|Friday|1962Q4|N|Y|N|2437939|2438211|2437592|2437865|N|N|N|N|N| +2437958|AAAAAAAAGEDDFCAA|1962-10-20|753|3277|252|1962|6|10|20|4|1962|252|3277|Saturday|1962Q4|N|Y|N|2437939|2438211|2437593|2437866|N|N|N|N|N| +2437959|AAAAAAAAHEDDFCAA|1962-10-21|753|3277|252|1962|0|10|21|4|1962|252|3277|Sunday|1962Q4|N|N|N|2437939|2438211|2437594|2437867|N|N|N|N|N| +2437960|AAAAAAAAIEDDFCAA|1962-10-22|753|3277|252|1962|1|10|22|4|1962|252|3277|Monday|1962Q4|N|N|N|2437939|2438211|2437595|2437868|N|N|N|N|N| +2437961|AAAAAAAAJEDDFCAA|1962-10-23|753|3278|252|1962|2|10|23|4|1962|252|3278|Tuesday|1962Q4|N|N|N|2437939|2438211|2437596|2437869|N|N|N|N|N| +2437962|AAAAAAAAKEDDFCAA|1962-10-24|753|3278|252|1962|3|10|24|4|1962|252|3278|Wednesday|1962Q4|N|N|N|2437939|2438211|2437597|2437870|N|N|N|N|N| +2437963|AAAAAAAALEDDFCAA|1962-10-25|753|3278|252|1962|4|10|25|4|1962|252|3278|Thursday|1962Q4|N|N|N|2437939|2438211|2437598|2437871|N|N|N|N|N| +2437964|AAAAAAAAMEDDFCAA|1962-10-26|753|3278|252|1962|5|10|26|4|1962|252|3278|Friday|1962Q4|N|Y|N|2437939|2438211|2437599|2437872|N|N|N|N|N| +2437965|AAAAAAAANEDDFCAA|1962-10-27|753|3278|252|1962|6|10|27|4|1962|252|3278|Saturday|1962Q4|N|Y|N|2437939|2438211|2437600|2437873|N|N|N|N|N| +2437966|AAAAAAAAOEDDFCAA|1962-10-28|753|3278|252|1962|0|10|28|4|1962|252|3278|Sunday|1962Q4|N|N|N|2437939|2438211|2437601|2437874|N|N|N|N|N| +2437967|AAAAAAAAPEDDFCAA|1962-10-29|753|3278|252|1962|1|10|29|4|1962|252|3278|Monday|1962Q4|N|N|N|2437939|2438211|2437602|2437875|N|N|N|N|N| +2437968|AAAAAAAAAFDDFCAA|1962-10-30|753|3279|252|1962|2|10|30|4|1962|252|3279|Tuesday|1962Q4|N|N|N|2437939|2438211|2437603|2437876|N|N|N|N|N| +2437969|AAAAAAAABFDDFCAA|1962-10-31|753|3279|252|1962|3|10|31|4|1962|252|3279|Wednesday|1962Q4|N|N|N|2437939|2438211|2437604|2437877|N|N|N|N|N| +2437970|AAAAAAAACFDDFCAA|1962-11-01|754|3279|252|1962|4|11|1|4|1962|252|3279|Thursday|1962Q4|N|N|N|2437970|2438273|2437605|2437878|N|N|N|N|N| +2437971|AAAAAAAADFDDFCAA|1962-11-02|754|3279|252|1962|5|11|2|4|1962|252|3279|Friday|1962Q4|N|Y|N|2437970|2438273|2437606|2437879|N|N|N|N|N| +2437972|AAAAAAAAEFDDFCAA|1962-11-03|754|3279|252|1962|6|11|3|4|1962|252|3279|Saturday|1962Q4|N|Y|N|2437970|2438273|2437607|2437880|N|N|N|N|N| +2437973|AAAAAAAAFFDDFCAA|1962-11-04|754|3279|252|1962|0|11|4|4|1962|252|3279|Sunday|1962Q4|N|N|N|2437970|2438273|2437608|2437881|N|N|N|N|N| +2437974|AAAAAAAAGFDDFCAA|1962-11-05|754|3279|252|1962|1|11|5|4|1962|252|3279|Monday|1962Q4|N|N|N|2437970|2438273|2437609|2437882|N|N|N|N|N| +2437975|AAAAAAAAHFDDFCAA|1962-11-06|754|3280|252|1962|2|11|6|4|1962|252|3280|Tuesday|1962Q4|N|N|N|2437970|2438273|2437610|2437883|N|N|N|N|N| +2437976|AAAAAAAAIFDDFCAA|1962-11-07|754|3280|252|1962|3|11|7|4|1962|252|3280|Wednesday|1962Q4|N|N|N|2437970|2438273|2437611|2437884|N|N|N|N|N| +2437977|AAAAAAAAJFDDFCAA|1962-11-08|754|3280|252|1962|4|11|8|4|1962|252|3280|Thursday|1962Q4|N|N|N|2437970|2438273|2437612|2437885|N|N|N|N|N| +2437978|AAAAAAAAKFDDFCAA|1962-11-09|754|3280|252|1962|5|11|9|4|1962|252|3280|Friday|1962Q4|N|Y|N|2437970|2438273|2437613|2437886|N|N|N|N|N| +2437979|AAAAAAAALFDDFCAA|1962-11-10|754|3280|252|1962|6|11|10|4|1962|252|3280|Saturday|1962Q4|N|Y|N|2437970|2438273|2437614|2437887|N|N|N|N|N| +2437980|AAAAAAAAMFDDFCAA|1962-11-11|754|3280|252|1962|0|11|11|4|1962|252|3280|Sunday|1962Q4|N|N|N|2437970|2438273|2437615|2437888|N|N|N|N|N| +2437981|AAAAAAAANFDDFCAA|1962-11-12|754|3280|252|1962|1|11|12|4|1962|252|3280|Monday|1962Q4|N|N|N|2437970|2438273|2437616|2437889|N|N|N|N|N| +2437982|AAAAAAAAOFDDFCAA|1962-11-13|754|3281|252|1962|2|11|13|4|1962|252|3281|Tuesday|1962Q4|N|N|N|2437970|2438273|2437617|2437890|N|N|N|N|N| +2437983|AAAAAAAAPFDDFCAA|1962-11-14|754|3281|252|1962|3|11|14|4|1962|252|3281|Wednesday|1962Q4|N|N|N|2437970|2438273|2437618|2437891|N|N|N|N|N| +2437984|AAAAAAAAAGDDFCAA|1962-11-15|754|3281|252|1962|4|11|15|4|1962|252|3281|Thursday|1962Q4|N|N|N|2437970|2438273|2437619|2437892|N|N|N|N|N| +2437985|AAAAAAAABGDDFCAA|1962-11-16|754|3281|252|1962|5|11|16|4|1962|252|3281|Friday|1962Q4|N|Y|N|2437970|2438273|2437620|2437893|N|N|N|N|N| +2437986|AAAAAAAACGDDFCAA|1962-11-17|754|3281|252|1962|6|11|17|4|1962|252|3281|Saturday|1962Q4|N|Y|N|2437970|2438273|2437621|2437894|N|N|N|N|N| +2437987|AAAAAAAADGDDFCAA|1962-11-18|754|3281|252|1962|0|11|18|4|1962|252|3281|Sunday|1962Q4|N|N|N|2437970|2438273|2437622|2437895|N|N|N|N|N| +2437988|AAAAAAAAEGDDFCAA|1962-11-19|754|3281|252|1962|1|11|19|4|1962|252|3281|Monday|1962Q4|N|N|N|2437970|2438273|2437623|2437896|N|N|N|N|N| +2437989|AAAAAAAAFGDDFCAA|1962-11-20|754|3282|252|1962|2|11|20|4|1962|252|3282|Tuesday|1962Q4|N|N|N|2437970|2438273|2437624|2437897|N|N|N|N|N| +2437990|AAAAAAAAGGDDFCAA|1962-11-21|754|3282|252|1962|3|11|21|4|1962|252|3282|Wednesday|1962Q4|N|N|N|2437970|2438273|2437625|2437898|N|N|N|N|N| +2437991|AAAAAAAAHGDDFCAA|1962-11-22|754|3282|252|1962|4|11|22|4|1962|252|3282|Thursday|1962Q4|N|N|N|2437970|2438273|2437626|2437899|N|N|N|N|N| +2437992|AAAAAAAAIGDDFCAA|1962-11-23|754|3282|252|1962|5|11|23|4|1962|252|3282|Friday|1962Q4|N|Y|N|2437970|2438273|2437627|2437900|N|N|N|N|N| +2437993|AAAAAAAAJGDDFCAA|1962-11-24|754|3282|252|1962|6|11|24|4|1962|252|3282|Saturday|1962Q4|N|Y|N|2437970|2438273|2437628|2437901|N|N|N|N|N| +2437994|AAAAAAAAKGDDFCAA|1962-11-25|754|3282|252|1962|0|11|25|4|1962|252|3282|Sunday|1962Q4|N|N|N|2437970|2438273|2437629|2437902|N|N|N|N|N| +2437995|AAAAAAAALGDDFCAA|1962-11-26|754|3282|252|1962|1|11|26|4|1962|252|3282|Monday|1962Q4|N|N|N|2437970|2438273|2437630|2437903|N|N|N|N|N| +2437996|AAAAAAAAMGDDFCAA|1962-11-27|754|3283|252|1962|2|11|27|4|1962|252|3283|Tuesday|1962Q4|N|N|N|2437970|2438273|2437631|2437904|N|N|N|N|N| +2437997|AAAAAAAANGDDFCAA|1962-11-28|754|3283|252|1962|3|11|28|4|1962|252|3283|Wednesday|1962Q4|N|N|N|2437970|2438273|2437632|2437905|N|N|N|N|N| +2437998|AAAAAAAAOGDDFCAA|1962-11-29|754|3283|252|1962|4|11|29|4|1962|252|3283|Thursday|1962Q4|N|N|N|2437970|2438273|2437633|2437906|N|N|N|N|N| +2437999|AAAAAAAAPGDDFCAA|1962-11-30|754|3283|252|1962|5|11|30|4|1962|252|3283|Friday|1962Q4|N|Y|N|2437970|2438273|2437634|2437907|N|N|N|N|N| +2438000|AAAAAAAAAHDDFCAA|1962-12-01|755|3283|253|1962|6|12|1|4|1962|253|3283|Saturday|1962Q4|N|Y|N|2438000|2438333|2437635|2437908|N|N|N|N|N| +2438001|AAAAAAAABHDDFCAA|1962-12-02|755|3283|253|1962|0|12|2|4|1962|253|3283|Sunday|1962Q4|N|N|N|2438000|2438333|2437636|2437909|N|N|N|N|N| +2438002|AAAAAAAACHDDFCAA|1962-12-03|755|3283|253|1962|1|12|3|4|1962|253|3283|Monday|1962Q4|N|N|N|2438000|2438333|2437637|2437910|N|N|N|N|N| +2438003|AAAAAAAADHDDFCAA|1962-12-04|755|3284|253|1962|2|12|4|4|1962|253|3284|Tuesday|1962Q4|N|N|N|2438000|2438333|2437638|2437911|N|N|N|N|N| +2438004|AAAAAAAAEHDDFCAA|1962-12-05|755|3284|253|1962|3|12|5|4|1962|253|3284|Wednesday|1962Q4|N|N|N|2438000|2438333|2437639|2437912|N|N|N|N|N| +2438005|AAAAAAAAFHDDFCAA|1962-12-06|755|3284|253|1962|4|12|6|4|1962|253|3284|Thursday|1962Q4|N|N|N|2438000|2438333|2437640|2437913|N|N|N|N|N| +2438006|AAAAAAAAGHDDFCAA|1962-12-07|755|3284|253|1962|5|12|7|4|1962|253|3284|Friday|1962Q4|N|Y|N|2438000|2438333|2437641|2437914|N|N|N|N|N| +2438007|AAAAAAAAHHDDFCAA|1962-12-08|755|3284|253|1962|6|12|8|4|1962|253|3284|Saturday|1962Q4|N|Y|N|2438000|2438333|2437642|2437915|N|N|N|N|N| +2438008|AAAAAAAAIHDDFCAA|1962-12-09|755|3284|253|1962|0|12|9|4|1962|253|3284|Sunday|1962Q4|N|N|N|2438000|2438333|2437643|2437916|N|N|N|N|N| +2438009|AAAAAAAAJHDDFCAA|1962-12-10|755|3284|253|1962|1|12|10|4|1962|253|3284|Monday|1962Q4|N|N|N|2438000|2438333|2437644|2437917|N|N|N|N|N| +2438010|AAAAAAAAKHDDFCAA|1962-12-11|755|3285|253|1962|2|12|11|4|1962|253|3285|Tuesday|1962Q4|N|N|N|2438000|2438333|2437645|2437918|N|N|N|N|N| +2438011|AAAAAAAALHDDFCAA|1962-12-12|755|3285|253|1962|3|12|12|4|1962|253|3285|Wednesday|1962Q4|N|N|N|2438000|2438333|2437646|2437919|N|N|N|N|N| +2438012|AAAAAAAAMHDDFCAA|1962-12-13|755|3285|253|1962|4|12|13|4|1962|253|3285|Thursday|1962Q4|N|N|N|2438000|2438333|2437647|2437920|N|N|N|N|N| +2438013|AAAAAAAANHDDFCAA|1962-12-14|755|3285|253|1962|5|12|14|4|1962|253|3285|Friday|1962Q4|N|Y|N|2438000|2438333|2437648|2437921|N|N|N|N|N| +2438014|AAAAAAAAOHDDFCAA|1962-12-15|755|3285|253|1962|6|12|15|4|1962|253|3285|Saturday|1962Q4|N|Y|N|2438000|2438333|2437649|2437922|N|N|N|N|N| +2438015|AAAAAAAAPHDDFCAA|1962-12-16|755|3285|253|1962|0|12|16|4|1962|253|3285|Sunday|1962Q4|N|N|N|2438000|2438333|2437650|2437923|N|N|N|N|N| +2438016|AAAAAAAAAIDDFCAA|1962-12-17|755|3285|253|1962|1|12|17|4|1962|253|3285|Monday|1962Q4|N|N|N|2438000|2438333|2437651|2437924|N|N|N|N|N| +2438017|AAAAAAAABIDDFCAA|1962-12-18|755|3286|253|1962|2|12|18|4|1962|253|3286|Tuesday|1962Q4|N|N|N|2438000|2438333|2437652|2437925|N|N|N|N|N| +2438018|AAAAAAAACIDDFCAA|1962-12-19|755|3286|253|1962|3|12|19|4|1962|253|3286|Wednesday|1962Q4|N|N|N|2438000|2438333|2437653|2437926|N|N|N|N|N| +2438019|AAAAAAAADIDDFCAA|1962-12-20|755|3286|253|1962|4|12|20|4|1962|253|3286|Thursday|1962Q4|N|N|N|2438000|2438333|2437654|2437927|N|N|N|N|N| +2438020|AAAAAAAAEIDDFCAA|1962-12-21|755|3286|253|1962|5|12|21|4|1962|253|3286|Friday|1962Q4|N|Y|N|2438000|2438333|2437655|2437928|N|N|N|N|N| +2438021|AAAAAAAAFIDDFCAA|1962-12-22|755|3286|253|1962|6|12|22|4|1962|253|3286|Saturday|1962Q4|N|Y|N|2438000|2438333|2437656|2437929|N|N|N|N|N| +2438022|AAAAAAAAGIDDFCAA|1962-12-23|755|3286|253|1962|0|12|23|4|1962|253|3286|Sunday|1962Q4|N|N|N|2438000|2438333|2437657|2437930|N|N|N|N|N| +2438023|AAAAAAAAHIDDFCAA|1962-12-24|755|3286|253|1962|1|12|24|4|1962|253|3286|Monday|1962Q4|N|N|N|2438000|2438333|2437658|2437931|N|N|N|N|N| +2438024|AAAAAAAAIIDDFCAA|1962-12-25|755|3287|253|1962|2|12|25|4|1962|253|3287|Tuesday|1962Q4|N|N|N|2438000|2438333|2437659|2437932|N|N|N|N|N| +2438025|AAAAAAAAJIDDFCAA|1962-12-26|755|3287|253|1962|3|12|26|4|1962|253|3287|Wednesday|1962Q4|Y|N|N|2438000|2438333|2437660|2437933|N|N|N|N|N| +2438026|AAAAAAAAKIDDFCAA|1962-12-27|755|3287|253|1962|4|12|27|4|1962|253|3287|Thursday|1962Q4|N|N|Y|2438000|2438333|2437661|2437934|N|N|N|N|N| +2438027|AAAAAAAALIDDFCAA|1962-12-28|755|3287|253|1962|5|12|28|4|1962|253|3287|Friday|1962Q4|N|Y|N|2438000|2438333|2437662|2437935|N|N|N|N|N| +2438028|AAAAAAAAMIDDFCAA|1962-12-29|755|3287|253|1962|6|12|29|4|1962|253|3287|Saturday|1962Q4|N|Y|N|2438000|2438333|2437663|2437936|N|N|N|N|N| +2438029|AAAAAAAANIDDFCAA|1962-12-30|755|3287|253|1962|0|12|30|4|1962|253|3287|Sunday|1962Q4|N|N|N|2438000|2438333|2437664|2437937|N|N|N|N|N| +2438030|AAAAAAAAOIDDFCAA|1962-12-31|755|3287|253|1962|1|12|31|4|1962|253|3287|Monday|1962Q4|N|N|N|2438000|2438333|2437665|2437938|N|N|N|N|N| +2438031|AAAAAAAAPIDDFCAA|1963-01-01|756|3288|253|1963|2|1|1|1|1963|253|3288|Tuesday|1963Q1|Y|N|N|2438031|2438030|2437666|2437939|N|N|N|N|N| +2438032|AAAAAAAAAJDDFCAA|1963-01-02|756|3288|253|1963|3|1|2|1|1963|253|3288|Wednesday|1963Q1|N|N|Y|2438031|2438030|2437667|2437940|N|N|N|N|N| +2438033|AAAAAAAABJDDFCAA|1963-01-03|756|3288|253|1963|4|1|3|1|1963|253|3288|Thursday|1963Q1|N|N|N|2438031|2438030|2437668|2437941|N|N|N|N|N| +2438034|AAAAAAAACJDDFCAA|1963-01-04|756|3288|253|1963|5|1|4|1|1963|253|3288|Friday|1963Q1|N|Y|N|2438031|2438030|2437669|2437942|N|N|N|N|N| +2438035|AAAAAAAADJDDFCAA|1963-01-05|756|3288|253|1963|6|1|5|1|1963|253|3288|Saturday|1963Q1|N|Y|N|2438031|2438030|2437670|2437943|N|N|N|N|N| +2438036|AAAAAAAAEJDDFCAA|1963-01-06|756|3288|253|1963|0|1|6|1|1963|253|3288|Sunday|1963Q1|N|N|N|2438031|2438030|2437671|2437944|N|N|N|N|N| +2438037|AAAAAAAAFJDDFCAA|1963-01-07|756|3288|253|1963|1|1|7|1|1963|253|3288|Monday|1963Q1|N|N|N|2438031|2438030|2437672|2437945|N|N|N|N|N| +2438038|AAAAAAAAGJDDFCAA|1963-01-08|756|3289|253|1963|2|1|8|1|1963|253|3289|Tuesday|1963Q1|N|N|N|2438031|2438030|2437673|2437946|N|N|N|N|N| +2438039|AAAAAAAAHJDDFCAA|1963-01-09|756|3289|253|1963|3|1|9|1|1963|253|3289|Wednesday|1963Q1|N|N|N|2438031|2438030|2437674|2437947|N|N|N|N|N| +2438040|AAAAAAAAIJDDFCAA|1963-01-10|756|3289|253|1963|4|1|10|1|1963|253|3289|Thursday|1963Q1|N|N|N|2438031|2438030|2437675|2437948|N|N|N|N|N| +2438041|AAAAAAAAJJDDFCAA|1963-01-11|756|3289|253|1963|5|1|11|1|1963|253|3289|Friday|1963Q1|N|Y|N|2438031|2438030|2437676|2437949|N|N|N|N|N| +2438042|AAAAAAAAKJDDFCAA|1963-01-12|756|3289|253|1963|6|1|12|1|1963|253|3289|Saturday|1963Q1|N|Y|N|2438031|2438030|2437677|2437950|N|N|N|N|N| +2438043|AAAAAAAALJDDFCAA|1963-01-13|756|3289|253|1963|0|1|13|1|1963|253|3289|Sunday|1963Q1|N|N|N|2438031|2438030|2437678|2437951|N|N|N|N|N| +2438044|AAAAAAAAMJDDFCAA|1963-01-14|756|3289|253|1963|1|1|14|1|1963|253|3289|Monday|1963Q1|N|N|N|2438031|2438030|2437679|2437952|N|N|N|N|N| +2438045|AAAAAAAANJDDFCAA|1963-01-15|756|3290|253|1963|2|1|15|1|1963|253|3290|Tuesday|1963Q1|N|N|N|2438031|2438030|2437680|2437953|N|N|N|N|N| +2438046|AAAAAAAAOJDDFCAA|1963-01-16|756|3290|253|1963|3|1|16|1|1963|253|3290|Wednesday|1963Q1|N|N|N|2438031|2438030|2437681|2437954|N|N|N|N|N| +2438047|AAAAAAAAPJDDFCAA|1963-01-17|756|3290|253|1963|4|1|17|1|1963|253|3290|Thursday|1963Q1|N|N|N|2438031|2438030|2437682|2437955|N|N|N|N|N| +2438048|AAAAAAAAAKDDFCAA|1963-01-18|756|3290|253|1963|5|1|18|1|1963|253|3290|Friday|1963Q1|N|Y|N|2438031|2438030|2437683|2437956|N|N|N|N|N| +2438049|AAAAAAAABKDDFCAA|1963-01-19|756|3290|253|1963|6|1|19|1|1963|253|3290|Saturday|1963Q1|N|Y|N|2438031|2438030|2437684|2437957|N|N|N|N|N| +2438050|AAAAAAAACKDDFCAA|1963-01-20|756|3290|253|1963|0|1|20|1|1963|253|3290|Sunday|1963Q1|N|N|N|2438031|2438030|2437685|2437958|N|N|N|N|N| +2438051|AAAAAAAADKDDFCAA|1963-01-21|756|3290|253|1963|1|1|21|1|1963|253|3290|Monday|1963Q1|N|N|N|2438031|2438030|2437686|2437959|N|N|N|N|N| +2438052|AAAAAAAAEKDDFCAA|1963-01-22|756|3291|253|1963|2|1|22|1|1963|253|3291|Tuesday|1963Q1|N|N|N|2438031|2438030|2437687|2437960|N|N|N|N|N| +2438053|AAAAAAAAFKDDFCAA|1963-01-23|756|3291|253|1963|3|1|23|1|1963|253|3291|Wednesday|1963Q1|N|N|N|2438031|2438030|2437688|2437961|N|N|N|N|N| +2438054|AAAAAAAAGKDDFCAA|1963-01-24|756|3291|253|1963|4|1|24|1|1963|253|3291|Thursday|1963Q1|N|N|N|2438031|2438030|2437689|2437962|N|N|N|N|N| +2438055|AAAAAAAAHKDDFCAA|1963-01-25|756|3291|253|1963|5|1|25|1|1963|253|3291|Friday|1963Q1|N|Y|N|2438031|2438030|2437690|2437963|N|N|N|N|N| +2438056|AAAAAAAAIKDDFCAA|1963-01-26|756|3291|253|1963|6|1|26|1|1963|253|3291|Saturday|1963Q1|N|Y|N|2438031|2438030|2437691|2437964|N|N|N|N|N| +2438057|AAAAAAAAJKDDFCAA|1963-01-27|756|3291|253|1963|0|1|27|1|1963|253|3291|Sunday|1963Q1|N|N|N|2438031|2438030|2437692|2437965|N|N|N|N|N| +2438058|AAAAAAAAKKDDFCAA|1963-01-28|756|3291|253|1963|1|1|28|1|1963|253|3291|Monday|1963Q1|N|N|N|2438031|2438030|2437693|2437966|N|N|N|N|N| +2438059|AAAAAAAALKDDFCAA|1963-01-29|756|3292|253|1963|2|1|29|1|1963|253|3292|Tuesday|1963Q1|N|N|N|2438031|2438030|2437694|2437967|N|N|N|N|N| +2438060|AAAAAAAAMKDDFCAA|1963-01-30|756|3292|253|1963|3|1|30|1|1963|253|3292|Wednesday|1963Q1|N|N|N|2438031|2438030|2437695|2437968|N|N|N|N|N| +2438061|AAAAAAAANKDDFCAA|1963-01-31|756|3292|253|1963|4|1|31|1|1963|253|3292|Thursday|1963Q1|N|N|N|2438031|2438030|2437696|2437969|N|N|N|N|N| +2438062|AAAAAAAAOKDDFCAA|1963-02-01|757|3292|253|1963|5|2|1|1|1963|253|3292|Friday|1963Q1|N|Y|N|2438062|2438092|2437697|2437970|N|N|N|N|N| +2438063|AAAAAAAAPKDDFCAA|1963-02-02|757|3292|253|1963|6|2|2|1|1963|253|3292|Saturday|1963Q1|N|Y|N|2438062|2438092|2437698|2437971|N|N|N|N|N| +2438064|AAAAAAAAALDDFCAA|1963-02-03|757|3292|253|1963|0|2|3|1|1963|253|3292|Sunday|1963Q1|N|N|N|2438062|2438092|2437699|2437972|N|N|N|N|N| +2438065|AAAAAAAABLDDFCAA|1963-02-04|757|3292|253|1963|1|2|4|1|1963|253|3292|Monday|1963Q1|N|N|N|2438062|2438092|2437700|2437973|N|N|N|N|N| +2438066|AAAAAAAACLDDFCAA|1963-02-05|757|3293|253|1963|2|2|5|1|1963|253|3293|Tuesday|1963Q1|N|N|N|2438062|2438092|2437701|2437974|N|N|N|N|N| +2438067|AAAAAAAADLDDFCAA|1963-02-06|757|3293|253|1963|3|2|6|1|1963|253|3293|Wednesday|1963Q1|N|N|N|2438062|2438092|2437702|2437975|N|N|N|N|N| +2438068|AAAAAAAAELDDFCAA|1963-02-07|757|3293|253|1963|4|2|7|1|1963|253|3293|Thursday|1963Q1|N|N|N|2438062|2438092|2437703|2437976|N|N|N|N|N| +2438069|AAAAAAAAFLDDFCAA|1963-02-08|757|3293|253|1963|5|2|8|1|1963|253|3293|Friday|1963Q1|N|Y|N|2438062|2438092|2437704|2437977|N|N|N|N|N| +2438070|AAAAAAAAGLDDFCAA|1963-02-09|757|3293|253|1963|6|2|9|1|1963|253|3293|Saturday|1963Q1|N|Y|N|2438062|2438092|2437705|2437978|N|N|N|N|N| +2438071|AAAAAAAAHLDDFCAA|1963-02-10|757|3293|253|1963|0|2|10|1|1963|253|3293|Sunday|1963Q1|N|N|N|2438062|2438092|2437706|2437979|N|N|N|N|N| +2438072|AAAAAAAAILDDFCAA|1963-02-11|757|3293|253|1963|1|2|11|1|1963|253|3293|Monday|1963Q1|N|N|N|2438062|2438092|2437707|2437980|N|N|N|N|N| +2438073|AAAAAAAAJLDDFCAA|1963-02-12|757|3294|253|1963|2|2|12|1|1963|253|3294|Tuesday|1963Q1|N|N|N|2438062|2438092|2437708|2437981|N|N|N|N|N| +2438074|AAAAAAAAKLDDFCAA|1963-02-13|757|3294|253|1963|3|2|13|1|1963|253|3294|Wednesday|1963Q1|N|N|N|2438062|2438092|2437709|2437982|N|N|N|N|N| +2438075|AAAAAAAALLDDFCAA|1963-02-14|757|3294|253|1963|4|2|14|1|1963|253|3294|Thursday|1963Q1|N|N|N|2438062|2438092|2437710|2437983|N|N|N|N|N| +2438076|AAAAAAAAMLDDFCAA|1963-02-15|757|3294|253|1963|5|2|15|1|1963|253|3294|Friday|1963Q1|N|Y|N|2438062|2438092|2437711|2437984|N|N|N|N|N| +2438077|AAAAAAAANLDDFCAA|1963-02-16|757|3294|253|1963|6|2|16|1|1963|253|3294|Saturday|1963Q1|N|Y|N|2438062|2438092|2437712|2437985|N|N|N|N|N| +2438078|AAAAAAAAOLDDFCAA|1963-02-17|757|3294|253|1963|0|2|17|1|1963|253|3294|Sunday|1963Q1|N|N|N|2438062|2438092|2437713|2437986|N|N|N|N|N| +2438079|AAAAAAAAPLDDFCAA|1963-02-18|757|3294|253|1963|1|2|18|1|1963|253|3294|Monday|1963Q1|N|N|N|2438062|2438092|2437714|2437987|N|N|N|N|N| +2438080|AAAAAAAAAMDDFCAA|1963-02-19|757|3295|253|1963|2|2|19|1|1963|253|3295|Tuesday|1963Q1|N|N|N|2438062|2438092|2437715|2437988|N|N|N|N|N| +2438081|AAAAAAAABMDDFCAA|1963-02-20|757|3295|253|1963|3|2|20|1|1963|253|3295|Wednesday|1963Q1|N|N|N|2438062|2438092|2437716|2437989|N|N|N|N|N| +2438082|AAAAAAAACMDDFCAA|1963-02-21|757|3295|253|1963|4|2|21|1|1963|253|3295|Thursday|1963Q1|N|N|N|2438062|2438092|2437717|2437990|N|N|N|N|N| +2438083|AAAAAAAADMDDFCAA|1963-02-22|757|3295|253|1963|5|2|22|1|1963|253|3295|Friday|1963Q1|N|Y|N|2438062|2438092|2437718|2437991|N|N|N|N|N| +2438084|AAAAAAAAEMDDFCAA|1963-02-23|757|3295|253|1963|6|2|23|1|1963|253|3295|Saturday|1963Q1|N|Y|N|2438062|2438092|2437719|2437992|N|N|N|N|N| +2438085|AAAAAAAAFMDDFCAA|1963-02-24|757|3295|253|1963|0|2|24|1|1963|253|3295|Sunday|1963Q1|N|N|N|2438062|2438092|2437720|2437993|N|N|N|N|N| +2438086|AAAAAAAAGMDDFCAA|1963-02-25|757|3295|253|1963|1|2|25|1|1963|253|3295|Monday|1963Q1|N|N|N|2438062|2438092|2437721|2437994|N|N|N|N|N| +2438087|AAAAAAAAHMDDFCAA|1963-02-26|757|3296|253|1963|2|2|26|1|1963|253|3296|Tuesday|1963Q1|N|N|N|2438062|2438092|2437722|2437995|N|N|N|N|N| +2438088|AAAAAAAAIMDDFCAA|1963-02-27|757|3296|253|1963|3|2|27|1|1963|253|3296|Wednesday|1963Q1|N|N|N|2438062|2438092|2437723|2437996|N|N|N|N|N| +2438089|AAAAAAAAJMDDFCAA|1963-02-28|757|3296|253|1963|4|2|28|1|1963|253|3296|Thursday|1963Q1|N|N|N|2438062|2438092|2437724|2437997|N|N|N|N|N| +2438090|AAAAAAAAKMDDFCAA|1963-03-01|758|3296|254|1963|5|3|1|1|1963|254|3296|Friday|1963Q1|N|Y|N|2438090|2438148|2437725|2437998|N|N|N|N|N| +2438091|AAAAAAAALMDDFCAA|1963-03-02|758|3296|254|1963|6|3|2|1|1963|254|3296|Saturday|1963Q1|N|Y|N|2438090|2438148|2437726|2437999|N|N|N|N|N| +2438092|AAAAAAAAMMDDFCAA|1963-03-03|758|3296|254|1963|0|3|3|1|1963|254|3296|Sunday|1963Q1|N|N|N|2438090|2438148|2437727|2438000|N|N|N|N|N| +2438093|AAAAAAAANMDDFCAA|1963-03-04|758|3296|254|1963|1|3|4|1|1963|254|3296|Monday|1963Q1|N|N|N|2438090|2438148|2437728|2438001|N|N|N|N|N| +2438094|AAAAAAAAOMDDFCAA|1963-03-05|758|3297|254|1963|2|3|5|1|1963|254|3297|Tuesday|1963Q1|N|N|N|2438090|2438148|2437729|2438002|N|N|N|N|N| +2438095|AAAAAAAAPMDDFCAA|1963-03-06|758|3297|254|1963|3|3|6|1|1963|254|3297|Wednesday|1963Q1|N|N|N|2438090|2438148|2437730|2438003|N|N|N|N|N| +2438096|AAAAAAAAANDDFCAA|1963-03-07|758|3297|254|1963|4|3|7|1|1963|254|3297|Thursday|1963Q1|N|N|N|2438090|2438148|2437731|2438004|N|N|N|N|N| +2438097|AAAAAAAABNDDFCAA|1963-03-08|758|3297|254|1963|5|3|8|1|1963|254|3297|Friday|1963Q1|N|Y|N|2438090|2438148|2437732|2438005|N|N|N|N|N| +2438098|AAAAAAAACNDDFCAA|1963-03-09|758|3297|254|1963|6|3|9|1|1963|254|3297|Saturday|1963Q1|N|Y|N|2438090|2438148|2437733|2438006|N|N|N|N|N| +2438099|AAAAAAAADNDDFCAA|1963-03-10|758|3297|254|1963|0|3|10|1|1963|254|3297|Sunday|1963Q1|N|N|N|2438090|2438148|2437734|2438007|N|N|N|N|N| +2438100|AAAAAAAAENDDFCAA|1963-03-11|758|3297|254|1963|1|3|11|1|1963|254|3297|Monday|1963Q1|N|N|N|2438090|2438148|2437735|2438008|N|N|N|N|N| +2438101|AAAAAAAAFNDDFCAA|1963-03-12|758|3298|254|1963|2|3|12|1|1963|254|3298|Tuesday|1963Q1|N|N|N|2438090|2438148|2437736|2438009|N|N|N|N|N| +2438102|AAAAAAAAGNDDFCAA|1963-03-13|758|3298|254|1963|3|3|13|1|1963|254|3298|Wednesday|1963Q1|N|N|N|2438090|2438148|2437737|2438010|N|N|N|N|N| +2438103|AAAAAAAAHNDDFCAA|1963-03-14|758|3298|254|1963|4|3|14|1|1963|254|3298|Thursday|1963Q1|N|N|N|2438090|2438148|2437738|2438011|N|N|N|N|N| +2438104|AAAAAAAAINDDFCAA|1963-03-15|758|3298|254|1963|5|3|15|1|1963|254|3298|Friday|1963Q1|N|Y|N|2438090|2438148|2437739|2438012|N|N|N|N|N| +2438105|AAAAAAAAJNDDFCAA|1963-03-16|758|3298|254|1963|6|3|16|1|1963|254|3298|Saturday|1963Q1|N|Y|N|2438090|2438148|2437740|2438013|N|N|N|N|N| +2438106|AAAAAAAAKNDDFCAA|1963-03-17|758|3298|254|1963|0|3|17|1|1963|254|3298|Sunday|1963Q1|N|N|N|2438090|2438148|2437741|2438014|N|N|N|N|N| +2438107|AAAAAAAALNDDFCAA|1963-03-18|758|3298|254|1963|1|3|18|1|1963|254|3298|Monday|1963Q1|N|N|N|2438090|2438148|2437742|2438015|N|N|N|N|N| +2438108|AAAAAAAAMNDDFCAA|1963-03-19|758|3299|254|1963|2|3|19|1|1963|254|3299|Tuesday|1963Q1|N|N|N|2438090|2438148|2437743|2438016|N|N|N|N|N| +2438109|AAAAAAAANNDDFCAA|1963-03-20|758|3299|254|1963|3|3|20|1|1963|254|3299|Wednesday|1963Q1|N|N|N|2438090|2438148|2437744|2438017|N|N|N|N|N| +2438110|AAAAAAAAONDDFCAA|1963-03-21|758|3299|254|1963|4|3|21|1|1963|254|3299|Thursday|1963Q1|N|N|N|2438090|2438148|2437745|2438018|N|N|N|N|N| +2438111|AAAAAAAAPNDDFCAA|1963-03-22|758|3299|254|1963|5|3|22|1|1963|254|3299|Friday|1963Q1|N|Y|N|2438090|2438148|2437746|2438019|N|N|N|N|N| +2438112|AAAAAAAAAODDFCAA|1963-03-23|758|3299|254|1963|6|3|23|1|1963|254|3299|Saturday|1963Q1|N|Y|N|2438090|2438148|2437747|2438020|N|N|N|N|N| +2438113|AAAAAAAABODDFCAA|1963-03-24|758|3299|254|1963|0|3|24|1|1963|254|3299|Sunday|1963Q1|N|N|N|2438090|2438148|2437748|2438021|N|N|N|N|N| +2438114|AAAAAAAACODDFCAA|1963-03-25|758|3299|254|1963|1|3|25|1|1963|254|3299|Monday|1963Q1|N|N|N|2438090|2438148|2437749|2438022|N|N|N|N|N| +2438115|AAAAAAAADODDFCAA|1963-03-26|758|3300|254|1963|2|3|26|1|1963|254|3300|Tuesday|1963Q1|N|N|N|2438090|2438148|2437750|2438023|N|N|N|N|N| +2438116|AAAAAAAAEODDFCAA|1963-03-27|758|3300|254|1963|3|3|27|1|1963|254|3300|Wednesday|1963Q1|N|N|N|2438090|2438148|2437751|2438024|N|N|N|N|N| +2438117|AAAAAAAAFODDFCAA|1963-03-28|758|3300|254|1963|4|3|28|1|1963|254|3300|Thursday|1963Q1|N|N|N|2438090|2438148|2437752|2438025|N|N|N|N|N| +2438118|AAAAAAAAGODDFCAA|1963-03-29|758|3300|254|1963|5|3|29|1|1963|254|3300|Friday|1963Q1|N|Y|N|2438090|2438148|2437753|2438026|N|N|N|N|N| +2438119|AAAAAAAAHODDFCAA|1963-03-30|758|3300|254|1963|6|3|30|1|1963|254|3300|Saturday|1963Q1|N|Y|N|2438090|2438148|2437754|2438027|N|N|N|N|N| +2438120|AAAAAAAAIODDFCAA|1963-03-31|758|3300|254|1963|0|3|31|1|1963|254|3300|Sunday|1963Q1|N|N|N|2438090|2438148|2437755|2438028|N|N|N|N|N| +2438121|AAAAAAAAJODDFCAA|1963-04-01|759|3300|254|1963|1|4|1|1|1963|254|3300|Monday|1963Q1|N|N|N|2438121|2438210|2437756|2438031|N|N|N|N|N| +2438122|AAAAAAAAKODDFCAA|1963-04-02|759|3301|254|1963|2|4|2|2|1963|254|3301|Tuesday|1963Q2|N|N|N|2438121|2438210|2437757|2438032|N|N|N|N|N| +2438123|AAAAAAAALODDFCAA|1963-04-03|759|3301|254|1963|3|4|3|2|1963|254|3301|Wednesday|1963Q2|N|N|N|2438121|2438210|2437758|2438033|N|N|N|N|N| +2438124|AAAAAAAAMODDFCAA|1963-04-04|759|3301|254|1963|4|4|4|2|1963|254|3301|Thursday|1963Q2|N|N|N|2438121|2438210|2437759|2438034|N|N|N|N|N| +2438125|AAAAAAAANODDFCAA|1963-04-05|759|3301|254|1963|5|4|5|2|1963|254|3301|Friday|1963Q2|N|Y|N|2438121|2438210|2437760|2438035|N|N|N|N|N| +2438126|AAAAAAAAOODDFCAA|1963-04-06|759|3301|254|1963|6|4|6|2|1963|254|3301|Saturday|1963Q2|N|Y|N|2438121|2438210|2437761|2438036|N|N|N|N|N| +2438127|AAAAAAAAPODDFCAA|1963-04-07|759|3301|254|1963|0|4|7|2|1963|254|3301|Sunday|1963Q2|N|N|N|2438121|2438210|2437762|2438037|N|N|N|N|N| +2438128|AAAAAAAAAPDDFCAA|1963-04-08|759|3301|254|1963|1|4|8|2|1963|254|3301|Monday|1963Q2|N|N|N|2438121|2438210|2437763|2438038|N|N|N|N|N| +2438129|AAAAAAAABPDDFCAA|1963-04-09|759|3302|254|1963|2|4|9|2|1963|254|3302|Tuesday|1963Q2|N|N|N|2438121|2438210|2437764|2438039|N|N|N|N|N| +2438130|AAAAAAAACPDDFCAA|1963-04-10|759|3302|254|1963|3|4|10|2|1963|254|3302|Wednesday|1963Q2|N|N|N|2438121|2438210|2437765|2438040|N|N|N|N|N| +2438131|AAAAAAAADPDDFCAA|1963-04-11|759|3302|254|1963|4|4|11|2|1963|254|3302|Thursday|1963Q2|N|N|N|2438121|2438210|2437766|2438041|N|N|N|N|N| +2438132|AAAAAAAAEPDDFCAA|1963-04-12|759|3302|254|1963|5|4|12|2|1963|254|3302|Friday|1963Q2|N|Y|N|2438121|2438210|2437767|2438042|N|N|N|N|N| +2438133|AAAAAAAAFPDDFCAA|1963-04-13|759|3302|254|1963|6|4|13|2|1963|254|3302|Saturday|1963Q2|N|Y|N|2438121|2438210|2437768|2438043|N|N|N|N|N| +2438134|AAAAAAAAGPDDFCAA|1963-04-14|759|3302|254|1963|0|4|14|2|1963|254|3302|Sunday|1963Q2|N|N|N|2438121|2438210|2437769|2438044|N|N|N|N|N| +2438135|AAAAAAAAHPDDFCAA|1963-04-15|759|3302|254|1963|1|4|15|2|1963|254|3302|Monday|1963Q2|N|N|N|2438121|2438210|2437770|2438045|N|N|N|N|N| +2438136|AAAAAAAAIPDDFCAA|1963-04-16|759|3303|254|1963|2|4|16|2|1963|254|3303|Tuesday|1963Q2|N|N|N|2438121|2438210|2437771|2438046|N|N|N|N|N| +2438137|AAAAAAAAJPDDFCAA|1963-04-17|759|3303|254|1963|3|4|17|2|1963|254|3303|Wednesday|1963Q2|N|N|N|2438121|2438210|2437772|2438047|N|N|N|N|N| +2438138|AAAAAAAAKPDDFCAA|1963-04-18|759|3303|254|1963|4|4|18|2|1963|254|3303|Thursday|1963Q2|N|N|N|2438121|2438210|2437773|2438048|N|N|N|N|N| +2438139|AAAAAAAALPDDFCAA|1963-04-19|759|3303|254|1963|5|4|19|2|1963|254|3303|Friday|1963Q2|N|Y|N|2438121|2438210|2437774|2438049|N|N|N|N|N| +2438140|AAAAAAAAMPDDFCAA|1963-04-20|759|3303|254|1963|6|4|20|2|1963|254|3303|Saturday|1963Q2|N|Y|N|2438121|2438210|2437775|2438050|N|N|N|N|N| +2438141|AAAAAAAANPDDFCAA|1963-04-21|759|3303|254|1963|0|4|21|2|1963|254|3303|Sunday|1963Q2|N|N|N|2438121|2438210|2437776|2438051|N|N|N|N|N| +2438142|AAAAAAAAOPDDFCAA|1963-04-22|759|3303|254|1963|1|4|22|2|1963|254|3303|Monday|1963Q2|N|N|N|2438121|2438210|2437777|2438052|N|N|N|N|N| +2438143|AAAAAAAAPPDDFCAA|1963-04-23|759|3304|254|1963|2|4|23|2|1963|254|3304|Tuesday|1963Q2|N|N|N|2438121|2438210|2437778|2438053|N|N|N|N|N| +2438144|AAAAAAAAAAEDFCAA|1963-04-24|759|3304|254|1963|3|4|24|2|1963|254|3304|Wednesday|1963Q2|N|N|N|2438121|2438210|2437779|2438054|N|N|N|N|N| +2438145|AAAAAAAABAEDFCAA|1963-04-25|759|3304|254|1963|4|4|25|2|1963|254|3304|Thursday|1963Q2|N|N|N|2438121|2438210|2437780|2438055|N|N|N|N|N| +2438146|AAAAAAAACAEDFCAA|1963-04-26|759|3304|254|1963|5|4|26|2|1963|254|3304|Friday|1963Q2|N|Y|N|2438121|2438210|2437781|2438056|N|N|N|N|N| +2438147|AAAAAAAADAEDFCAA|1963-04-27|759|3304|254|1963|6|4|27|2|1963|254|3304|Saturday|1963Q2|N|Y|N|2438121|2438210|2437782|2438057|N|N|N|N|N| +2438148|AAAAAAAAEAEDFCAA|1963-04-28|759|3304|254|1963|0|4|28|2|1963|254|3304|Sunday|1963Q2|N|N|N|2438121|2438210|2437783|2438058|N|N|N|N|N| +2438149|AAAAAAAAFAEDFCAA|1963-04-29|759|3304|254|1963|1|4|29|2|1963|254|3304|Monday|1963Q2|N|N|N|2438121|2438210|2437784|2438059|N|N|N|N|N| +2438150|AAAAAAAAGAEDFCAA|1963-04-30|759|3305|254|1963|2|4|30|2|1963|254|3305|Tuesday|1963Q2|N|N|N|2438121|2438210|2437785|2438060|N|N|N|N|N| +2438151|AAAAAAAAHAEDFCAA|1963-05-01|760|3305|254|1963|3|5|1|2|1963|254|3305|Wednesday|1963Q2|N|N|N|2438151|2438270|2437786|2438061|N|N|N|N|N| +2438152|AAAAAAAAIAEDFCAA|1963-05-02|760|3305|254|1963|4|5|2|2|1963|254|3305|Thursday|1963Q2|N|N|N|2438151|2438270|2437787|2438062|N|N|N|N|N| +2438153|AAAAAAAAJAEDFCAA|1963-05-03|760|3305|254|1963|5|5|3|2|1963|254|3305|Friday|1963Q2|N|Y|N|2438151|2438270|2437788|2438063|N|N|N|N|N| +2438154|AAAAAAAAKAEDFCAA|1963-05-04|760|3305|254|1963|6|5|4|2|1963|254|3305|Saturday|1963Q2|N|Y|N|2438151|2438270|2437789|2438064|N|N|N|N|N| +2438155|AAAAAAAALAEDFCAA|1963-05-05|760|3305|254|1963|0|5|5|2|1963|254|3305|Sunday|1963Q2|N|N|N|2438151|2438270|2437790|2438065|N|N|N|N|N| +2438156|AAAAAAAAMAEDFCAA|1963-05-06|760|3305|254|1963|1|5|6|2|1963|254|3305|Monday|1963Q2|N|N|N|2438151|2438270|2437791|2438066|N|N|N|N|N| +2438157|AAAAAAAANAEDFCAA|1963-05-07|760|3306|254|1963|2|5|7|2|1963|254|3306|Tuesday|1963Q2|N|N|N|2438151|2438270|2437792|2438067|N|N|N|N|N| +2438158|AAAAAAAAOAEDFCAA|1963-05-08|760|3306|254|1963|3|5|8|2|1963|254|3306|Wednesday|1963Q2|N|N|N|2438151|2438270|2437793|2438068|N|N|N|N|N| +2438159|AAAAAAAAPAEDFCAA|1963-05-09|760|3306|254|1963|4|5|9|2|1963|254|3306|Thursday|1963Q2|N|N|N|2438151|2438270|2437794|2438069|N|N|N|N|N| +2438160|AAAAAAAAABEDFCAA|1963-05-10|760|3306|254|1963|5|5|10|2|1963|254|3306|Friday|1963Q2|N|Y|N|2438151|2438270|2437795|2438070|N|N|N|N|N| +2438161|AAAAAAAABBEDFCAA|1963-05-11|760|3306|254|1963|6|5|11|2|1963|254|3306|Saturday|1963Q2|N|Y|N|2438151|2438270|2437796|2438071|N|N|N|N|N| +2438162|AAAAAAAACBEDFCAA|1963-05-12|760|3306|254|1963|0|5|12|2|1963|254|3306|Sunday|1963Q2|N|N|N|2438151|2438270|2437797|2438072|N|N|N|N|N| +2438163|AAAAAAAADBEDFCAA|1963-05-13|760|3306|254|1963|1|5|13|2|1963|254|3306|Monday|1963Q2|N|N|N|2438151|2438270|2437798|2438073|N|N|N|N|N| +2438164|AAAAAAAAEBEDFCAA|1963-05-14|760|3307|254|1963|2|5|14|2|1963|254|3307|Tuesday|1963Q2|N|N|N|2438151|2438270|2437799|2438074|N|N|N|N|N| +2438165|AAAAAAAAFBEDFCAA|1963-05-15|760|3307|254|1963|3|5|15|2|1963|254|3307|Wednesday|1963Q2|N|N|N|2438151|2438270|2437800|2438075|N|N|N|N|N| +2438166|AAAAAAAAGBEDFCAA|1963-05-16|760|3307|254|1963|4|5|16|2|1963|254|3307|Thursday|1963Q2|N|N|N|2438151|2438270|2437801|2438076|N|N|N|N|N| +2438167|AAAAAAAAHBEDFCAA|1963-05-17|760|3307|254|1963|5|5|17|2|1963|254|3307|Friday|1963Q2|N|Y|N|2438151|2438270|2437802|2438077|N|N|N|N|N| +2438168|AAAAAAAAIBEDFCAA|1963-05-18|760|3307|254|1963|6|5|18|2|1963|254|3307|Saturday|1963Q2|N|Y|N|2438151|2438270|2437803|2438078|N|N|N|N|N| +2438169|AAAAAAAAJBEDFCAA|1963-05-19|760|3307|254|1963|0|5|19|2|1963|254|3307|Sunday|1963Q2|N|N|N|2438151|2438270|2437804|2438079|N|N|N|N|N| +2438170|AAAAAAAAKBEDFCAA|1963-05-20|760|3307|254|1963|1|5|20|2|1963|254|3307|Monday|1963Q2|N|N|N|2438151|2438270|2437805|2438080|N|N|N|N|N| +2438171|AAAAAAAALBEDFCAA|1963-05-21|760|3308|254|1963|2|5|21|2|1963|254|3308|Tuesday|1963Q2|N|N|N|2438151|2438270|2437806|2438081|N|N|N|N|N| +2438172|AAAAAAAAMBEDFCAA|1963-05-22|760|3308|254|1963|3|5|22|2|1963|254|3308|Wednesday|1963Q2|N|N|N|2438151|2438270|2437807|2438082|N|N|N|N|N| +2438173|AAAAAAAANBEDFCAA|1963-05-23|760|3308|254|1963|4|5|23|2|1963|254|3308|Thursday|1963Q2|N|N|N|2438151|2438270|2437808|2438083|N|N|N|N|N| +2438174|AAAAAAAAOBEDFCAA|1963-05-24|760|3308|254|1963|5|5|24|2|1963|254|3308|Friday|1963Q2|N|Y|N|2438151|2438270|2437809|2438084|N|N|N|N|N| +2438175|AAAAAAAAPBEDFCAA|1963-05-25|760|3308|254|1963|6|5|25|2|1963|254|3308|Saturday|1963Q2|N|Y|N|2438151|2438270|2437810|2438085|N|N|N|N|N| +2438176|AAAAAAAAACEDFCAA|1963-05-26|760|3308|254|1963|0|5|26|2|1963|254|3308|Sunday|1963Q2|N|N|N|2438151|2438270|2437811|2438086|N|N|N|N|N| +2438177|AAAAAAAABCEDFCAA|1963-05-27|760|3308|254|1963|1|5|27|2|1963|254|3308|Monday|1963Q2|N|N|N|2438151|2438270|2437812|2438087|N|N|N|N|N| +2438178|AAAAAAAACCEDFCAA|1963-05-28|760|3309|254|1963|2|5|28|2|1963|254|3309|Tuesday|1963Q2|N|N|N|2438151|2438270|2437813|2438088|N|N|N|N|N| +2438179|AAAAAAAADCEDFCAA|1963-05-29|760|3309|254|1963|3|5|29|2|1963|254|3309|Wednesday|1963Q2|N|N|N|2438151|2438270|2437814|2438089|N|N|N|N|N| +2438180|AAAAAAAAECEDFCAA|1963-05-30|760|3309|254|1963|4|5|30|2|1963|254|3309|Thursday|1963Q2|N|N|N|2438151|2438270|2437815|2438090|N|N|N|N|N| +2438181|AAAAAAAAFCEDFCAA|1963-05-31|760|3309|254|1963|5|5|31|2|1963|254|3309|Friday|1963Q2|N|Y|N|2438151|2438270|2437816|2438091|N|N|N|N|N| +2438182|AAAAAAAAGCEDFCAA|1963-06-01|761|3309|255|1963|6|6|1|2|1963|255|3309|Saturday|1963Q2|N|Y|N|2438182|2438332|2437817|2438092|N|N|N|N|N| +2438183|AAAAAAAAHCEDFCAA|1963-06-02|761|3309|255|1963|0|6|2|2|1963|255|3309|Sunday|1963Q2|N|N|N|2438182|2438332|2437818|2438093|N|N|N|N|N| +2438184|AAAAAAAAICEDFCAA|1963-06-03|761|3309|255|1963|1|6|3|2|1963|255|3309|Monday|1963Q2|N|N|N|2438182|2438332|2437819|2438094|N|N|N|N|N| +2438185|AAAAAAAAJCEDFCAA|1963-06-04|761|3310|255|1963|2|6|4|2|1963|255|3310|Tuesday|1963Q2|N|N|N|2438182|2438332|2437820|2438095|N|N|N|N|N| +2438186|AAAAAAAAKCEDFCAA|1963-06-05|761|3310|255|1963|3|6|5|2|1963|255|3310|Wednesday|1963Q2|N|N|N|2438182|2438332|2437821|2438096|N|N|N|N|N| +2438187|AAAAAAAALCEDFCAA|1963-06-06|761|3310|255|1963|4|6|6|2|1963|255|3310|Thursday|1963Q2|N|N|N|2438182|2438332|2437822|2438097|N|N|N|N|N| +2438188|AAAAAAAAMCEDFCAA|1963-06-07|761|3310|255|1963|5|6|7|2|1963|255|3310|Friday|1963Q2|N|Y|N|2438182|2438332|2437823|2438098|N|N|N|N|N| +2438189|AAAAAAAANCEDFCAA|1963-06-08|761|3310|255|1963|6|6|8|2|1963|255|3310|Saturday|1963Q2|N|Y|N|2438182|2438332|2437824|2438099|N|N|N|N|N| +2438190|AAAAAAAAOCEDFCAA|1963-06-09|761|3310|255|1963|0|6|9|2|1963|255|3310|Sunday|1963Q2|N|N|N|2438182|2438332|2437825|2438100|N|N|N|N|N| +2438191|AAAAAAAAPCEDFCAA|1963-06-10|761|3310|255|1963|1|6|10|2|1963|255|3310|Monday|1963Q2|N|N|N|2438182|2438332|2437826|2438101|N|N|N|N|N| +2438192|AAAAAAAAADEDFCAA|1963-06-11|761|3311|255|1963|2|6|11|2|1963|255|3311|Tuesday|1963Q2|N|N|N|2438182|2438332|2437827|2438102|N|N|N|N|N| +2438193|AAAAAAAABDEDFCAA|1963-06-12|761|3311|255|1963|3|6|12|2|1963|255|3311|Wednesday|1963Q2|N|N|N|2438182|2438332|2437828|2438103|N|N|N|N|N| +2438194|AAAAAAAACDEDFCAA|1963-06-13|761|3311|255|1963|4|6|13|2|1963|255|3311|Thursday|1963Q2|N|N|N|2438182|2438332|2437829|2438104|N|N|N|N|N| +2438195|AAAAAAAADDEDFCAA|1963-06-14|761|3311|255|1963|5|6|14|2|1963|255|3311|Friday|1963Q2|N|Y|N|2438182|2438332|2437830|2438105|N|N|N|N|N| +2438196|AAAAAAAAEDEDFCAA|1963-06-15|761|3311|255|1963|6|6|15|2|1963|255|3311|Saturday|1963Q2|N|Y|N|2438182|2438332|2437831|2438106|N|N|N|N|N| +2438197|AAAAAAAAFDEDFCAA|1963-06-16|761|3311|255|1963|0|6|16|2|1963|255|3311|Sunday|1963Q2|N|N|N|2438182|2438332|2437832|2438107|N|N|N|N|N| +2438198|AAAAAAAAGDEDFCAA|1963-06-17|761|3311|255|1963|1|6|17|2|1963|255|3311|Monday|1963Q2|N|N|N|2438182|2438332|2437833|2438108|N|N|N|N|N| +2438199|AAAAAAAAHDEDFCAA|1963-06-18|761|3312|255|1963|2|6|18|2|1963|255|3312|Tuesday|1963Q2|N|N|N|2438182|2438332|2437834|2438109|N|N|N|N|N| +2438200|AAAAAAAAIDEDFCAA|1963-06-19|761|3312|255|1963|3|6|19|2|1963|255|3312|Wednesday|1963Q2|N|N|N|2438182|2438332|2437835|2438110|N|N|N|N|N| +2438201|AAAAAAAAJDEDFCAA|1963-06-20|761|3312|255|1963|4|6|20|2|1963|255|3312|Thursday|1963Q2|N|N|N|2438182|2438332|2437836|2438111|N|N|N|N|N| +2438202|AAAAAAAAKDEDFCAA|1963-06-21|761|3312|255|1963|5|6|21|2|1963|255|3312|Friday|1963Q2|N|Y|N|2438182|2438332|2437837|2438112|N|N|N|N|N| +2438203|AAAAAAAALDEDFCAA|1963-06-22|761|3312|255|1963|6|6|22|2|1963|255|3312|Saturday|1963Q2|N|Y|N|2438182|2438332|2437838|2438113|N|N|N|N|N| +2438204|AAAAAAAAMDEDFCAA|1963-06-23|761|3312|255|1963|0|6|23|2|1963|255|3312|Sunday|1963Q2|N|N|N|2438182|2438332|2437839|2438114|N|N|N|N|N| +2438205|AAAAAAAANDEDFCAA|1963-06-24|761|3312|255|1963|1|6|24|2|1963|255|3312|Monday|1963Q2|N|N|N|2438182|2438332|2437840|2438115|N|N|N|N|N| +2438206|AAAAAAAAODEDFCAA|1963-06-25|761|3313|255|1963|2|6|25|2|1963|255|3313|Tuesday|1963Q2|N|N|N|2438182|2438332|2437841|2438116|N|N|N|N|N| +2438207|AAAAAAAAPDEDFCAA|1963-06-26|761|3313|255|1963|3|6|26|2|1963|255|3313|Wednesday|1963Q2|N|N|N|2438182|2438332|2437842|2438117|N|N|N|N|N| +2438208|AAAAAAAAAEEDFCAA|1963-06-27|761|3313|255|1963|4|6|27|2|1963|255|3313|Thursday|1963Q2|N|N|N|2438182|2438332|2437843|2438118|N|N|N|N|N| +2438209|AAAAAAAABEEDFCAA|1963-06-28|761|3313|255|1963|5|6|28|2|1963|255|3313|Friday|1963Q2|N|Y|N|2438182|2438332|2437844|2438119|N|N|N|N|N| +2438210|AAAAAAAACEEDFCAA|1963-06-29|761|3313|255|1963|6|6|29|2|1963|255|3313|Saturday|1963Q2|N|Y|N|2438182|2438332|2437845|2438120|N|N|N|N|N| +2438211|AAAAAAAADEEDFCAA|1963-06-30|761|3313|255|1963|0|6|30|2|1963|255|3313|Sunday|1963Q2|N|N|N|2438182|2438332|2437846|2438121|N|N|N|N|N| +2438212|AAAAAAAAEEEDFCAA|1963-07-01|762|3313|255|1963|1|7|1|2|1963|255|3313|Monday|1963Q2|N|N|N|2438212|2438392|2437847|2438121|N|N|N|N|N| +2438213|AAAAAAAAFEEDFCAA|1963-07-02|762|3314|255|1963|2|7|2|3|1963|255|3314|Tuesday|1963Q3|N|N|N|2438212|2438392|2437848|2438122|N|N|N|N|N| +2438214|AAAAAAAAGEEDFCAA|1963-07-03|762|3314|255|1963|3|7|3|3|1963|255|3314|Wednesday|1963Q3|N|N|N|2438212|2438392|2437849|2438123|N|N|N|N|N| +2438215|AAAAAAAAHEEDFCAA|1963-07-04|762|3314|255|1963|4|7|4|3|1963|255|3314|Thursday|1963Q3|N|N|N|2438212|2438392|2437850|2438124|N|N|N|N|N| +2438216|AAAAAAAAIEEDFCAA|1963-07-05|762|3314|255|1963|5|7|5|3|1963|255|3314|Friday|1963Q3|Y|Y|N|2438212|2438392|2437851|2438125|N|N|N|N|N| +2438217|AAAAAAAAJEEDFCAA|1963-07-06|762|3314|255|1963|6|7|6|3|1963|255|3314|Saturday|1963Q3|N|Y|Y|2438212|2438392|2437852|2438126|N|N|N|N|N| +2438218|AAAAAAAAKEEDFCAA|1963-07-07|762|3314|255|1963|0|7|7|3|1963|255|3314|Sunday|1963Q3|N|N|N|2438212|2438392|2437853|2438127|N|N|N|N|N| +2438219|AAAAAAAALEEDFCAA|1963-07-08|762|3314|255|1963|1|7|8|3|1963|255|3314|Monday|1963Q3|N|N|N|2438212|2438392|2437854|2438128|N|N|N|N|N| +2438220|AAAAAAAAMEEDFCAA|1963-07-09|762|3315|255|1963|2|7|9|3|1963|255|3315|Tuesday|1963Q3|N|N|N|2438212|2438392|2437855|2438129|N|N|N|N|N| +2438221|AAAAAAAANEEDFCAA|1963-07-10|762|3315|255|1963|3|7|10|3|1963|255|3315|Wednesday|1963Q3|N|N|N|2438212|2438392|2437856|2438130|N|N|N|N|N| +2438222|AAAAAAAAOEEDFCAA|1963-07-11|762|3315|255|1963|4|7|11|3|1963|255|3315|Thursday|1963Q3|N|N|N|2438212|2438392|2437857|2438131|N|N|N|N|N| +2438223|AAAAAAAAPEEDFCAA|1963-07-12|762|3315|255|1963|5|7|12|3|1963|255|3315|Friday|1963Q3|N|Y|N|2438212|2438392|2437858|2438132|N|N|N|N|N| +2438224|AAAAAAAAAFEDFCAA|1963-07-13|762|3315|255|1963|6|7|13|3|1963|255|3315|Saturday|1963Q3|N|Y|N|2438212|2438392|2437859|2438133|N|N|N|N|N| +2438225|AAAAAAAABFEDFCAA|1963-07-14|762|3315|255|1963|0|7|14|3|1963|255|3315|Sunday|1963Q3|N|N|N|2438212|2438392|2437860|2438134|N|N|N|N|N| +2438226|AAAAAAAACFEDFCAA|1963-07-15|762|3315|255|1963|1|7|15|3|1963|255|3315|Monday|1963Q3|N|N|N|2438212|2438392|2437861|2438135|N|N|N|N|N| +2438227|AAAAAAAADFEDFCAA|1963-07-16|762|3316|255|1963|2|7|16|3|1963|255|3316|Tuesday|1963Q3|N|N|N|2438212|2438392|2437862|2438136|N|N|N|N|N| +2438228|AAAAAAAAEFEDFCAA|1963-07-17|762|3316|255|1963|3|7|17|3|1963|255|3316|Wednesday|1963Q3|N|N|N|2438212|2438392|2437863|2438137|N|N|N|N|N| +2438229|AAAAAAAAFFEDFCAA|1963-07-18|762|3316|255|1963|4|7|18|3|1963|255|3316|Thursday|1963Q3|N|N|N|2438212|2438392|2437864|2438138|N|N|N|N|N| +2438230|AAAAAAAAGFEDFCAA|1963-07-19|762|3316|255|1963|5|7|19|3|1963|255|3316|Friday|1963Q3|N|Y|N|2438212|2438392|2437865|2438139|N|N|N|N|N| +2438231|AAAAAAAAHFEDFCAA|1963-07-20|762|3316|255|1963|6|7|20|3|1963|255|3316|Saturday|1963Q3|N|Y|N|2438212|2438392|2437866|2438140|N|N|N|N|N| +2438232|AAAAAAAAIFEDFCAA|1963-07-21|762|3316|255|1963|0|7|21|3|1963|255|3316|Sunday|1963Q3|N|N|N|2438212|2438392|2437867|2438141|N|N|N|N|N| +2438233|AAAAAAAAJFEDFCAA|1963-07-22|762|3316|255|1963|1|7|22|3|1963|255|3316|Monday|1963Q3|N|N|N|2438212|2438392|2437868|2438142|N|N|N|N|N| +2438234|AAAAAAAAKFEDFCAA|1963-07-23|762|3317|255|1963|2|7|23|3|1963|255|3317|Tuesday|1963Q3|N|N|N|2438212|2438392|2437869|2438143|N|N|N|N|N| +2438235|AAAAAAAALFEDFCAA|1963-07-24|762|3317|255|1963|3|7|24|3|1963|255|3317|Wednesday|1963Q3|N|N|N|2438212|2438392|2437870|2438144|N|N|N|N|N| +2438236|AAAAAAAAMFEDFCAA|1963-07-25|762|3317|255|1963|4|7|25|3|1963|255|3317|Thursday|1963Q3|N|N|N|2438212|2438392|2437871|2438145|N|N|N|N|N| +2438237|AAAAAAAANFEDFCAA|1963-07-26|762|3317|255|1963|5|7|26|3|1963|255|3317|Friday|1963Q3|N|Y|N|2438212|2438392|2437872|2438146|N|N|N|N|N| +2438238|AAAAAAAAOFEDFCAA|1963-07-27|762|3317|255|1963|6|7|27|3|1963|255|3317|Saturday|1963Q3|N|Y|N|2438212|2438392|2437873|2438147|N|N|N|N|N| +2438239|AAAAAAAAPFEDFCAA|1963-07-28|762|3317|255|1963|0|7|28|3|1963|255|3317|Sunday|1963Q3|N|N|N|2438212|2438392|2437874|2438148|N|N|N|N|N| +2438240|AAAAAAAAAGEDFCAA|1963-07-29|762|3317|255|1963|1|7|29|3|1963|255|3317|Monday|1963Q3|N|N|N|2438212|2438392|2437875|2438149|N|N|N|N|N| +2438241|AAAAAAAABGEDFCAA|1963-07-30|762|3318|255|1963|2|7|30|3|1963|255|3318|Tuesday|1963Q3|N|N|N|2438212|2438392|2437876|2438150|N|N|N|N|N| +2438242|AAAAAAAACGEDFCAA|1963-07-31|762|3318|255|1963|3|7|31|3|1963|255|3318|Wednesday|1963Q3|N|N|N|2438212|2438392|2437877|2438151|N|N|N|N|N| +2438243|AAAAAAAADGEDFCAA|1963-08-01|763|3318|255|1963|4|8|1|3|1963|255|3318|Thursday|1963Q3|N|N|N|2438243|2438454|2437878|2438152|N|N|N|N|N| +2438244|AAAAAAAAEGEDFCAA|1963-08-02|763|3318|255|1963|5|8|2|3|1963|255|3318|Friday|1963Q3|N|Y|N|2438243|2438454|2437879|2438153|N|N|N|N|N| +2438245|AAAAAAAAFGEDFCAA|1963-08-03|763|3318|255|1963|6|8|3|3|1963|255|3318|Saturday|1963Q3|N|Y|N|2438243|2438454|2437880|2438154|N|N|N|N|N| +2438246|AAAAAAAAGGEDFCAA|1963-08-04|763|3318|255|1963|0|8|4|3|1963|255|3318|Sunday|1963Q3|N|N|N|2438243|2438454|2437881|2438155|N|N|N|N|N| +2438247|AAAAAAAAHGEDFCAA|1963-08-05|763|3318|255|1963|1|8|5|3|1963|255|3318|Monday|1963Q3|N|N|N|2438243|2438454|2437882|2438156|N|N|N|N|N| +2438248|AAAAAAAAIGEDFCAA|1963-08-06|763|3319|255|1963|2|8|6|3|1963|255|3319|Tuesday|1963Q3|N|N|N|2438243|2438454|2437883|2438157|N|N|N|N|N| +2438249|AAAAAAAAJGEDFCAA|1963-08-07|763|3319|255|1963|3|8|7|3|1963|255|3319|Wednesday|1963Q3|N|N|N|2438243|2438454|2437884|2438158|N|N|N|N|N| +2438250|AAAAAAAAKGEDFCAA|1963-08-08|763|3319|255|1963|4|8|8|3|1963|255|3319|Thursday|1963Q3|N|N|N|2438243|2438454|2437885|2438159|N|N|N|N|N| +2438251|AAAAAAAALGEDFCAA|1963-08-09|763|3319|255|1963|5|8|9|3|1963|255|3319|Friday|1963Q3|N|Y|N|2438243|2438454|2437886|2438160|N|N|N|N|N| +2438252|AAAAAAAAMGEDFCAA|1963-08-10|763|3319|255|1963|6|8|10|3|1963|255|3319|Saturday|1963Q3|N|Y|N|2438243|2438454|2437887|2438161|N|N|N|N|N| +2438253|AAAAAAAANGEDFCAA|1963-08-11|763|3319|255|1963|0|8|11|3|1963|255|3319|Sunday|1963Q3|N|N|N|2438243|2438454|2437888|2438162|N|N|N|N|N| +2438254|AAAAAAAAOGEDFCAA|1963-08-12|763|3319|255|1963|1|8|12|3|1963|255|3319|Monday|1963Q3|N|N|N|2438243|2438454|2437889|2438163|N|N|N|N|N| +2438255|AAAAAAAAPGEDFCAA|1963-08-13|763|3320|255|1963|2|8|13|3|1963|255|3320|Tuesday|1963Q3|N|N|N|2438243|2438454|2437890|2438164|N|N|N|N|N| +2438256|AAAAAAAAAHEDFCAA|1963-08-14|763|3320|255|1963|3|8|14|3|1963|255|3320|Wednesday|1963Q3|N|N|N|2438243|2438454|2437891|2438165|N|N|N|N|N| +2438257|AAAAAAAABHEDFCAA|1963-08-15|763|3320|255|1963|4|8|15|3|1963|255|3320|Thursday|1963Q3|N|N|N|2438243|2438454|2437892|2438166|N|N|N|N|N| +2438258|AAAAAAAACHEDFCAA|1963-08-16|763|3320|255|1963|5|8|16|3|1963|255|3320|Friday|1963Q3|N|Y|N|2438243|2438454|2437893|2438167|N|N|N|N|N| +2438259|AAAAAAAADHEDFCAA|1963-08-17|763|3320|255|1963|6|8|17|3|1963|255|3320|Saturday|1963Q3|N|Y|N|2438243|2438454|2437894|2438168|N|N|N|N|N| +2438260|AAAAAAAAEHEDFCAA|1963-08-18|763|3320|255|1963|0|8|18|3|1963|255|3320|Sunday|1963Q3|N|N|N|2438243|2438454|2437895|2438169|N|N|N|N|N| +2438261|AAAAAAAAFHEDFCAA|1963-08-19|763|3320|255|1963|1|8|19|3|1963|255|3320|Monday|1963Q3|N|N|N|2438243|2438454|2437896|2438170|N|N|N|N|N| +2438262|AAAAAAAAGHEDFCAA|1963-08-20|763|3321|255|1963|2|8|20|3|1963|255|3321|Tuesday|1963Q3|N|N|N|2438243|2438454|2437897|2438171|N|N|N|N|N| +2438263|AAAAAAAAHHEDFCAA|1963-08-21|763|3321|255|1963|3|8|21|3|1963|255|3321|Wednesday|1963Q3|N|N|N|2438243|2438454|2437898|2438172|N|N|N|N|N| +2438264|AAAAAAAAIHEDFCAA|1963-08-22|763|3321|255|1963|4|8|22|3|1963|255|3321|Thursday|1963Q3|N|N|N|2438243|2438454|2437899|2438173|N|N|N|N|N| +2438265|AAAAAAAAJHEDFCAA|1963-08-23|763|3321|255|1963|5|8|23|3|1963|255|3321|Friday|1963Q3|N|Y|N|2438243|2438454|2437900|2438174|N|N|N|N|N| +2438266|AAAAAAAAKHEDFCAA|1963-08-24|763|3321|255|1963|6|8|24|3|1963|255|3321|Saturday|1963Q3|N|Y|N|2438243|2438454|2437901|2438175|N|N|N|N|N| +2438267|AAAAAAAALHEDFCAA|1963-08-25|763|3321|255|1963|0|8|25|3|1963|255|3321|Sunday|1963Q3|N|N|N|2438243|2438454|2437902|2438176|N|N|N|N|N| +2438268|AAAAAAAAMHEDFCAA|1963-08-26|763|3321|255|1963|1|8|26|3|1963|255|3321|Monday|1963Q3|N|N|N|2438243|2438454|2437903|2438177|N|N|N|N|N| +2438269|AAAAAAAANHEDFCAA|1963-08-27|763|3322|255|1963|2|8|27|3|1963|255|3322|Tuesday|1963Q3|N|N|N|2438243|2438454|2437904|2438178|N|N|N|N|N| +2438270|AAAAAAAAOHEDFCAA|1963-08-28|763|3322|255|1963|3|8|28|3|1963|255|3322|Wednesday|1963Q3|N|N|N|2438243|2438454|2437905|2438179|N|N|N|N|N| +2438271|AAAAAAAAPHEDFCAA|1963-08-29|763|3322|255|1963|4|8|29|3|1963|255|3322|Thursday|1963Q3|N|N|N|2438243|2438454|2437906|2438180|N|N|N|N|N| +2438272|AAAAAAAAAIEDFCAA|1963-08-30|763|3322|255|1963|5|8|30|3|1963|255|3322|Friday|1963Q3|N|Y|N|2438243|2438454|2437907|2438181|N|N|N|N|N| +2438273|AAAAAAAABIEDFCAA|1963-08-31|763|3322|255|1963|6|8|31|3|1963|255|3322|Saturday|1963Q3|N|Y|N|2438243|2438454|2437908|2438182|N|N|N|N|N| +2438274|AAAAAAAACIEDFCAA|1963-09-01|764|3322|256|1963|0|9|1|3|1963|256|3322|Sunday|1963Q3|N|N|N|2438274|2438516|2437909|2438183|N|N|N|N|N| +2438275|AAAAAAAADIEDFCAA|1963-09-02|764|3322|256|1963|1|9|2|3|1963|256|3322|Monday|1963Q3|N|N|N|2438274|2438516|2437910|2438184|N|N|N|N|N| +2438276|AAAAAAAAEIEDFCAA|1963-09-03|764|3323|256|1963|2|9|3|3|1963|256|3323|Tuesday|1963Q3|N|N|N|2438274|2438516|2437911|2438185|N|N|N|N|N| +2438277|AAAAAAAAFIEDFCAA|1963-09-04|764|3323|256|1963|3|9|4|3|1963|256|3323|Wednesday|1963Q3|N|N|N|2438274|2438516|2437912|2438186|N|N|N|N|N| +2438278|AAAAAAAAGIEDFCAA|1963-09-05|764|3323|256|1963|4|9|5|3|1963|256|3323|Thursday|1963Q3|N|N|N|2438274|2438516|2437913|2438187|N|N|N|N|N| +2438279|AAAAAAAAHIEDFCAA|1963-09-06|764|3323|256|1963|5|9|6|3|1963|256|3323|Friday|1963Q3|N|Y|N|2438274|2438516|2437914|2438188|N|N|N|N|N| +2438280|AAAAAAAAIIEDFCAA|1963-09-07|764|3323|256|1963|6|9|7|3|1963|256|3323|Saturday|1963Q3|N|Y|N|2438274|2438516|2437915|2438189|N|N|N|N|N| +2438281|AAAAAAAAJIEDFCAA|1963-09-08|764|3323|256|1963|0|9|8|3|1963|256|3323|Sunday|1963Q3|N|N|N|2438274|2438516|2437916|2438190|N|N|N|N|N| +2438282|AAAAAAAAKIEDFCAA|1963-09-09|764|3323|256|1963|1|9|9|3|1963|256|3323|Monday|1963Q3|N|N|N|2438274|2438516|2437917|2438191|N|N|N|N|N| +2438283|AAAAAAAALIEDFCAA|1963-09-10|764|3324|256|1963|2|9|10|3|1963|256|3324|Tuesday|1963Q3|N|N|N|2438274|2438516|2437918|2438192|N|N|N|N|N| +2438284|AAAAAAAAMIEDFCAA|1963-09-11|764|3324|256|1963|3|9|11|3|1963|256|3324|Wednesday|1963Q3|N|N|N|2438274|2438516|2437919|2438193|N|N|N|N|N| +2438285|AAAAAAAANIEDFCAA|1963-09-12|764|3324|256|1963|4|9|12|3|1963|256|3324|Thursday|1963Q3|N|N|N|2438274|2438516|2437920|2438194|N|N|N|N|N| +2438286|AAAAAAAAOIEDFCAA|1963-09-13|764|3324|256|1963|5|9|13|3|1963|256|3324|Friday|1963Q3|N|Y|N|2438274|2438516|2437921|2438195|N|N|N|N|N| +2438287|AAAAAAAAPIEDFCAA|1963-09-14|764|3324|256|1963|6|9|14|3|1963|256|3324|Saturday|1963Q3|N|Y|N|2438274|2438516|2437922|2438196|N|N|N|N|N| +2438288|AAAAAAAAAJEDFCAA|1963-09-15|764|3324|256|1963|0|9|15|3|1963|256|3324|Sunday|1963Q3|N|N|N|2438274|2438516|2437923|2438197|N|N|N|N|N| +2438289|AAAAAAAABJEDFCAA|1963-09-16|764|3324|256|1963|1|9|16|3|1963|256|3324|Monday|1963Q3|N|N|N|2438274|2438516|2437924|2438198|N|N|N|N|N| +2438290|AAAAAAAACJEDFCAA|1963-09-17|764|3325|256|1963|2|9|17|3|1963|256|3325|Tuesday|1963Q3|N|N|N|2438274|2438516|2437925|2438199|N|N|N|N|N| +2438291|AAAAAAAADJEDFCAA|1963-09-18|764|3325|256|1963|3|9|18|3|1963|256|3325|Wednesday|1963Q3|N|N|N|2438274|2438516|2437926|2438200|N|N|N|N|N| +2438292|AAAAAAAAEJEDFCAA|1963-09-19|764|3325|256|1963|4|9|19|3|1963|256|3325|Thursday|1963Q3|N|N|N|2438274|2438516|2437927|2438201|N|N|N|N|N| +2438293|AAAAAAAAFJEDFCAA|1963-09-20|764|3325|256|1963|5|9|20|3|1963|256|3325|Friday|1963Q3|N|Y|N|2438274|2438516|2437928|2438202|N|N|N|N|N| +2438294|AAAAAAAAGJEDFCAA|1963-09-21|764|3325|256|1963|6|9|21|3|1963|256|3325|Saturday|1963Q3|N|Y|N|2438274|2438516|2437929|2438203|N|N|N|N|N| +2438295|AAAAAAAAHJEDFCAA|1963-09-22|764|3325|256|1963|0|9|22|3|1963|256|3325|Sunday|1963Q3|N|N|N|2438274|2438516|2437930|2438204|N|N|N|N|N| +2438296|AAAAAAAAIJEDFCAA|1963-09-23|764|3325|256|1963|1|9|23|3|1963|256|3325|Monday|1963Q3|N|N|N|2438274|2438516|2437931|2438205|N|N|N|N|N| +2438297|AAAAAAAAJJEDFCAA|1963-09-24|764|3326|256|1963|2|9|24|3|1963|256|3326|Tuesday|1963Q3|N|N|N|2438274|2438516|2437932|2438206|N|N|N|N|N| +2438298|AAAAAAAAKJEDFCAA|1963-09-25|764|3326|256|1963|3|9|25|3|1963|256|3326|Wednesday|1963Q3|N|N|N|2438274|2438516|2437933|2438207|N|N|N|N|N| +2438299|AAAAAAAALJEDFCAA|1963-09-26|764|3326|256|1963|4|9|26|3|1963|256|3326|Thursday|1963Q3|N|N|N|2438274|2438516|2437934|2438208|N|N|N|N|N| +2438300|AAAAAAAAMJEDFCAA|1963-09-27|764|3326|256|1963|5|9|27|3|1963|256|3326|Friday|1963Q3|N|Y|N|2438274|2438516|2437935|2438209|N|N|N|N|N| +2438301|AAAAAAAANJEDFCAA|1963-09-28|764|3326|256|1963|6|9|28|3|1963|256|3326|Saturday|1963Q3|N|Y|N|2438274|2438516|2437936|2438210|N|N|N|N|N| +2438302|AAAAAAAAOJEDFCAA|1963-09-29|764|3326|256|1963|0|9|29|3|1963|256|3326|Sunday|1963Q3|N|N|N|2438274|2438516|2437937|2438211|N|N|N|N|N| +2438303|AAAAAAAAPJEDFCAA|1963-09-30|764|3326|256|1963|1|9|30|3|1963|256|3326|Monday|1963Q3|N|N|N|2438274|2438516|2437938|2438212|N|N|N|N|N| +2438304|AAAAAAAAAKEDFCAA|1963-10-01|765|3327|256|1963|2|10|1|3|1963|256|3327|Tuesday|1963Q3|N|N|N|2438304|2438576|2437939|2438212|N|N|N|N|N| +2438305|AAAAAAAABKEDFCAA|1963-10-02|765|3327|256|1963|3|10|2|4|1963|256|3327|Wednesday|1963Q4|N|N|N|2438304|2438576|2437940|2438213|N|N|N|N|N| +2438306|AAAAAAAACKEDFCAA|1963-10-03|765|3327|256|1963|4|10|3|4|1963|256|3327|Thursday|1963Q4|N|N|N|2438304|2438576|2437941|2438214|N|N|N|N|N| +2438307|AAAAAAAADKEDFCAA|1963-10-04|765|3327|256|1963|5|10|4|4|1963|256|3327|Friday|1963Q4|N|Y|N|2438304|2438576|2437942|2438215|N|N|N|N|N| +2438308|AAAAAAAAEKEDFCAA|1963-10-05|765|3327|256|1963|6|10|5|4|1963|256|3327|Saturday|1963Q4|N|Y|N|2438304|2438576|2437943|2438216|N|N|N|N|N| +2438309|AAAAAAAAFKEDFCAA|1963-10-06|765|3327|256|1963|0|10|6|4|1963|256|3327|Sunday|1963Q4|N|N|N|2438304|2438576|2437944|2438217|N|N|N|N|N| +2438310|AAAAAAAAGKEDFCAA|1963-10-07|765|3327|256|1963|1|10|7|4|1963|256|3327|Monday|1963Q4|N|N|N|2438304|2438576|2437945|2438218|N|N|N|N|N| +2438311|AAAAAAAAHKEDFCAA|1963-10-08|765|3328|256|1963|2|10|8|4|1963|256|3328|Tuesday|1963Q4|N|N|N|2438304|2438576|2437946|2438219|N|N|N|N|N| +2438312|AAAAAAAAIKEDFCAA|1963-10-09|765|3328|256|1963|3|10|9|4|1963|256|3328|Wednesday|1963Q4|N|N|N|2438304|2438576|2437947|2438220|N|N|N|N|N| +2438313|AAAAAAAAJKEDFCAA|1963-10-10|765|3328|256|1963|4|10|10|4|1963|256|3328|Thursday|1963Q4|N|N|N|2438304|2438576|2437948|2438221|N|N|N|N|N| +2438314|AAAAAAAAKKEDFCAA|1963-10-11|765|3328|256|1963|5|10|11|4|1963|256|3328|Friday|1963Q4|N|Y|N|2438304|2438576|2437949|2438222|N|N|N|N|N| +2438315|AAAAAAAALKEDFCAA|1963-10-12|765|3328|256|1963|6|10|12|4|1963|256|3328|Saturday|1963Q4|N|Y|N|2438304|2438576|2437950|2438223|N|N|N|N|N| +2438316|AAAAAAAAMKEDFCAA|1963-10-13|765|3328|256|1963|0|10|13|4|1963|256|3328|Sunday|1963Q4|N|N|N|2438304|2438576|2437951|2438224|N|N|N|N|N| +2438317|AAAAAAAANKEDFCAA|1963-10-14|765|3328|256|1963|1|10|14|4|1963|256|3328|Monday|1963Q4|N|N|N|2438304|2438576|2437952|2438225|N|N|N|N|N| +2438318|AAAAAAAAOKEDFCAA|1963-10-15|765|3329|256|1963|2|10|15|4|1963|256|3329|Tuesday|1963Q4|N|N|N|2438304|2438576|2437953|2438226|N|N|N|N|N| +2438319|AAAAAAAAPKEDFCAA|1963-10-16|765|3329|256|1963|3|10|16|4|1963|256|3329|Wednesday|1963Q4|N|N|N|2438304|2438576|2437954|2438227|N|N|N|N|N| +2438320|AAAAAAAAALEDFCAA|1963-10-17|765|3329|256|1963|4|10|17|4|1963|256|3329|Thursday|1963Q4|N|N|N|2438304|2438576|2437955|2438228|N|N|N|N|N| +2438321|AAAAAAAABLEDFCAA|1963-10-18|765|3329|256|1963|5|10|18|4|1963|256|3329|Friday|1963Q4|N|Y|N|2438304|2438576|2437956|2438229|N|N|N|N|N| +2438322|AAAAAAAACLEDFCAA|1963-10-19|765|3329|256|1963|6|10|19|4|1963|256|3329|Saturday|1963Q4|N|Y|N|2438304|2438576|2437957|2438230|N|N|N|N|N| +2438323|AAAAAAAADLEDFCAA|1963-10-20|765|3329|256|1963|0|10|20|4|1963|256|3329|Sunday|1963Q4|N|N|N|2438304|2438576|2437958|2438231|N|N|N|N|N| +2438324|AAAAAAAAELEDFCAA|1963-10-21|765|3329|256|1963|1|10|21|4|1963|256|3329|Monday|1963Q4|N|N|N|2438304|2438576|2437959|2438232|N|N|N|N|N| +2438325|AAAAAAAAFLEDFCAA|1963-10-22|765|3330|256|1963|2|10|22|4|1963|256|3330|Tuesday|1963Q4|N|N|N|2438304|2438576|2437960|2438233|N|N|N|N|N| +2438326|AAAAAAAAGLEDFCAA|1963-10-23|765|3330|256|1963|3|10|23|4|1963|256|3330|Wednesday|1963Q4|N|N|N|2438304|2438576|2437961|2438234|N|N|N|N|N| +2438327|AAAAAAAAHLEDFCAA|1963-10-24|765|3330|256|1963|4|10|24|4|1963|256|3330|Thursday|1963Q4|N|N|N|2438304|2438576|2437962|2438235|N|N|N|N|N| +2438328|AAAAAAAAILEDFCAA|1963-10-25|765|3330|256|1963|5|10|25|4|1963|256|3330|Friday|1963Q4|N|Y|N|2438304|2438576|2437963|2438236|N|N|N|N|N| +2438329|AAAAAAAAJLEDFCAA|1963-10-26|765|3330|256|1963|6|10|26|4|1963|256|3330|Saturday|1963Q4|N|Y|N|2438304|2438576|2437964|2438237|N|N|N|N|N| +2438330|AAAAAAAAKLEDFCAA|1963-10-27|765|3330|256|1963|0|10|27|4|1963|256|3330|Sunday|1963Q4|N|N|N|2438304|2438576|2437965|2438238|N|N|N|N|N| +2438331|AAAAAAAALLEDFCAA|1963-10-28|765|3330|256|1963|1|10|28|4|1963|256|3330|Monday|1963Q4|N|N|N|2438304|2438576|2437966|2438239|N|N|N|N|N| +2438332|AAAAAAAAMLEDFCAA|1963-10-29|765|3331|256|1963|2|10|29|4|1963|256|3331|Tuesday|1963Q4|N|N|N|2438304|2438576|2437967|2438240|N|N|N|N|N| +2438333|AAAAAAAANLEDFCAA|1963-10-30|765|3331|256|1963|3|10|30|4|1963|256|3331|Wednesday|1963Q4|N|N|N|2438304|2438576|2437968|2438241|N|N|N|N|N| +2438334|AAAAAAAAOLEDFCAA|1963-10-31|765|3331|256|1963|4|10|31|4|1963|256|3331|Thursday|1963Q4|N|N|N|2438304|2438576|2437969|2438242|N|N|N|N|N| +2438335|AAAAAAAAPLEDFCAA|1963-11-01|766|3331|256|1963|5|11|1|4|1963|256|3331|Friday|1963Q4|N|Y|N|2438335|2438638|2437970|2438243|N|N|N|N|N| +2438336|AAAAAAAAAMEDFCAA|1963-11-02|766|3331|256|1963|6|11|2|4|1963|256|3331|Saturday|1963Q4|N|Y|N|2438335|2438638|2437971|2438244|N|N|N|N|N| +2438337|AAAAAAAABMEDFCAA|1963-11-03|766|3331|256|1963|0|11|3|4|1963|256|3331|Sunday|1963Q4|N|N|N|2438335|2438638|2437972|2438245|N|N|N|N|N| +2438338|AAAAAAAACMEDFCAA|1963-11-04|766|3331|256|1963|1|11|4|4|1963|256|3331|Monday|1963Q4|N|N|N|2438335|2438638|2437973|2438246|N|N|N|N|N| +2438339|AAAAAAAADMEDFCAA|1963-11-05|766|3332|256|1963|2|11|5|4|1963|256|3332|Tuesday|1963Q4|N|N|N|2438335|2438638|2437974|2438247|N|N|N|N|N| +2438340|AAAAAAAAEMEDFCAA|1963-11-06|766|3332|256|1963|3|11|6|4|1963|256|3332|Wednesday|1963Q4|N|N|N|2438335|2438638|2437975|2438248|N|N|N|N|N| +2438341|AAAAAAAAFMEDFCAA|1963-11-07|766|3332|256|1963|4|11|7|4|1963|256|3332|Thursday|1963Q4|N|N|N|2438335|2438638|2437976|2438249|N|N|N|N|N| +2438342|AAAAAAAAGMEDFCAA|1963-11-08|766|3332|256|1963|5|11|8|4|1963|256|3332|Friday|1963Q4|N|Y|N|2438335|2438638|2437977|2438250|N|N|N|N|N| +2438343|AAAAAAAAHMEDFCAA|1963-11-09|766|3332|256|1963|6|11|9|4|1963|256|3332|Saturday|1963Q4|N|Y|N|2438335|2438638|2437978|2438251|N|N|N|N|N| +2438344|AAAAAAAAIMEDFCAA|1963-11-10|766|3332|256|1963|0|11|10|4|1963|256|3332|Sunday|1963Q4|N|N|N|2438335|2438638|2437979|2438252|N|N|N|N|N| +2438345|AAAAAAAAJMEDFCAA|1963-11-11|766|3332|256|1963|1|11|11|4|1963|256|3332|Monday|1963Q4|N|N|N|2438335|2438638|2437980|2438253|N|N|N|N|N| +2438346|AAAAAAAAKMEDFCAA|1963-11-12|766|3333|256|1963|2|11|12|4|1963|256|3333|Tuesday|1963Q4|N|N|N|2438335|2438638|2437981|2438254|N|N|N|N|N| +2438347|AAAAAAAALMEDFCAA|1963-11-13|766|3333|256|1963|3|11|13|4|1963|256|3333|Wednesday|1963Q4|N|N|N|2438335|2438638|2437982|2438255|N|N|N|N|N| +2438348|AAAAAAAAMMEDFCAA|1963-11-14|766|3333|256|1963|4|11|14|4|1963|256|3333|Thursday|1963Q4|N|N|N|2438335|2438638|2437983|2438256|N|N|N|N|N| +2438349|AAAAAAAANMEDFCAA|1963-11-15|766|3333|256|1963|5|11|15|4|1963|256|3333|Friday|1963Q4|N|Y|N|2438335|2438638|2437984|2438257|N|N|N|N|N| +2438350|AAAAAAAAOMEDFCAA|1963-11-16|766|3333|256|1963|6|11|16|4|1963|256|3333|Saturday|1963Q4|N|Y|N|2438335|2438638|2437985|2438258|N|N|N|N|N| +2438351|AAAAAAAAPMEDFCAA|1963-11-17|766|3333|256|1963|0|11|17|4|1963|256|3333|Sunday|1963Q4|N|N|N|2438335|2438638|2437986|2438259|N|N|N|N|N| +2438352|AAAAAAAAANEDFCAA|1963-11-18|766|3333|256|1963|1|11|18|4|1963|256|3333|Monday|1963Q4|N|N|N|2438335|2438638|2437987|2438260|N|N|N|N|N| +2438353|AAAAAAAABNEDFCAA|1963-11-19|766|3334|256|1963|2|11|19|4|1963|256|3334|Tuesday|1963Q4|N|N|N|2438335|2438638|2437988|2438261|N|N|N|N|N| +2438354|AAAAAAAACNEDFCAA|1963-11-20|766|3334|256|1963|3|11|20|4|1963|256|3334|Wednesday|1963Q4|N|N|N|2438335|2438638|2437989|2438262|N|N|N|N|N| +2438355|AAAAAAAADNEDFCAA|1963-11-21|766|3334|256|1963|4|11|21|4|1963|256|3334|Thursday|1963Q4|N|N|N|2438335|2438638|2437990|2438263|N|N|N|N|N| +2438356|AAAAAAAAENEDFCAA|1963-11-22|766|3334|256|1963|5|11|22|4|1963|256|3334|Friday|1963Q4|N|Y|N|2438335|2438638|2437991|2438264|N|N|N|N|N| +2438357|AAAAAAAAFNEDFCAA|1963-11-23|766|3334|256|1963|6|11|23|4|1963|256|3334|Saturday|1963Q4|N|Y|N|2438335|2438638|2437992|2438265|N|N|N|N|N| +2438358|AAAAAAAAGNEDFCAA|1963-11-24|766|3334|256|1963|0|11|24|4|1963|256|3334|Sunday|1963Q4|N|N|N|2438335|2438638|2437993|2438266|N|N|N|N|N| +2438359|AAAAAAAAHNEDFCAA|1963-11-25|766|3334|256|1963|1|11|25|4|1963|256|3334|Monday|1963Q4|N|N|N|2438335|2438638|2437994|2438267|N|N|N|N|N| +2438360|AAAAAAAAINEDFCAA|1963-11-26|766|3335|256|1963|2|11|26|4|1963|256|3335|Tuesday|1963Q4|N|N|N|2438335|2438638|2437995|2438268|N|N|N|N|N| +2438361|AAAAAAAAJNEDFCAA|1963-11-27|766|3335|256|1963|3|11|27|4|1963|256|3335|Wednesday|1963Q4|N|N|N|2438335|2438638|2437996|2438269|N|N|N|N|N| +2438362|AAAAAAAAKNEDFCAA|1963-11-28|766|3335|256|1963|4|11|28|4|1963|256|3335|Thursday|1963Q4|N|N|N|2438335|2438638|2437997|2438270|N|N|N|N|N| +2438363|AAAAAAAALNEDFCAA|1963-11-29|766|3335|256|1963|5|11|29|4|1963|256|3335|Friday|1963Q4|N|Y|N|2438335|2438638|2437998|2438271|N|N|N|N|N| +2438364|AAAAAAAAMNEDFCAA|1963-11-30|766|3335|256|1963|6|11|30|4|1963|256|3335|Saturday|1963Q4|N|Y|N|2438335|2438638|2437999|2438272|N|N|N|N|N| +2438365|AAAAAAAANNEDFCAA|1963-12-01|767|3335|257|1963|0|12|1|4|1963|257|3335|Sunday|1963Q4|N|N|N|2438365|2438698|2438000|2438273|N|N|N|N|N| +2438366|AAAAAAAAONEDFCAA|1963-12-02|767|3335|257|1963|1|12|2|4|1963|257|3335|Monday|1963Q4|N|N|N|2438365|2438698|2438001|2438274|N|N|N|N|N| +2438367|AAAAAAAAPNEDFCAA|1963-12-03|767|3336|257|1963|2|12|3|4|1963|257|3336|Tuesday|1963Q4|N|N|N|2438365|2438698|2438002|2438275|N|N|N|N|N| +2438368|AAAAAAAAAOEDFCAA|1963-12-04|767|3336|257|1963|3|12|4|4|1963|257|3336|Wednesday|1963Q4|N|N|N|2438365|2438698|2438003|2438276|N|N|N|N|N| +2438369|AAAAAAAABOEDFCAA|1963-12-05|767|3336|257|1963|4|12|5|4|1963|257|3336|Thursday|1963Q4|N|N|N|2438365|2438698|2438004|2438277|N|N|N|N|N| +2438370|AAAAAAAACOEDFCAA|1963-12-06|767|3336|257|1963|5|12|6|4|1963|257|3336|Friday|1963Q4|N|Y|N|2438365|2438698|2438005|2438278|N|N|N|N|N| +2438371|AAAAAAAADOEDFCAA|1963-12-07|767|3336|257|1963|6|12|7|4|1963|257|3336|Saturday|1963Q4|N|Y|N|2438365|2438698|2438006|2438279|N|N|N|N|N| +2438372|AAAAAAAAEOEDFCAA|1963-12-08|767|3336|257|1963|0|12|8|4|1963|257|3336|Sunday|1963Q4|N|N|N|2438365|2438698|2438007|2438280|N|N|N|N|N| +2438373|AAAAAAAAFOEDFCAA|1963-12-09|767|3336|257|1963|1|12|9|4|1963|257|3336|Monday|1963Q4|N|N|N|2438365|2438698|2438008|2438281|N|N|N|N|N| +2438374|AAAAAAAAGOEDFCAA|1963-12-10|767|3337|257|1963|2|12|10|4|1963|257|3337|Tuesday|1963Q4|N|N|N|2438365|2438698|2438009|2438282|N|N|N|N|N| +2438375|AAAAAAAAHOEDFCAA|1963-12-11|767|3337|257|1963|3|12|11|4|1963|257|3337|Wednesday|1963Q4|N|N|N|2438365|2438698|2438010|2438283|N|N|N|N|N| +2438376|AAAAAAAAIOEDFCAA|1963-12-12|767|3337|257|1963|4|12|12|4|1963|257|3337|Thursday|1963Q4|N|N|N|2438365|2438698|2438011|2438284|N|N|N|N|N| +2438377|AAAAAAAAJOEDFCAA|1963-12-13|767|3337|257|1963|5|12|13|4|1963|257|3337|Friday|1963Q4|N|Y|N|2438365|2438698|2438012|2438285|N|N|N|N|N| +2438378|AAAAAAAAKOEDFCAA|1963-12-14|767|3337|257|1963|6|12|14|4|1963|257|3337|Saturday|1963Q4|N|Y|N|2438365|2438698|2438013|2438286|N|N|N|N|N| +2438379|AAAAAAAALOEDFCAA|1963-12-15|767|3337|257|1963|0|12|15|4|1963|257|3337|Sunday|1963Q4|N|N|N|2438365|2438698|2438014|2438287|N|N|N|N|N| +2438380|AAAAAAAAMOEDFCAA|1963-12-16|767|3337|257|1963|1|12|16|4|1963|257|3337|Monday|1963Q4|N|N|N|2438365|2438698|2438015|2438288|N|N|N|N|N| +2438381|AAAAAAAANOEDFCAA|1963-12-17|767|3338|257|1963|2|12|17|4|1963|257|3338|Tuesday|1963Q4|N|N|N|2438365|2438698|2438016|2438289|N|N|N|N|N| +2438382|AAAAAAAAOOEDFCAA|1963-12-18|767|3338|257|1963|3|12|18|4|1963|257|3338|Wednesday|1963Q4|N|N|N|2438365|2438698|2438017|2438290|N|N|N|N|N| +2438383|AAAAAAAAPOEDFCAA|1963-12-19|767|3338|257|1963|4|12|19|4|1963|257|3338|Thursday|1963Q4|N|N|N|2438365|2438698|2438018|2438291|N|N|N|N|N| +2438384|AAAAAAAAAPEDFCAA|1963-12-20|767|3338|257|1963|5|12|20|4|1963|257|3338|Friday|1963Q4|N|Y|N|2438365|2438698|2438019|2438292|N|N|N|N|N| +2438385|AAAAAAAABPEDFCAA|1963-12-21|767|3338|257|1963|6|12|21|4|1963|257|3338|Saturday|1963Q4|N|Y|N|2438365|2438698|2438020|2438293|N|N|N|N|N| +2438386|AAAAAAAACPEDFCAA|1963-12-22|767|3338|257|1963|0|12|22|4|1963|257|3338|Sunday|1963Q4|N|N|N|2438365|2438698|2438021|2438294|N|N|N|N|N| +2438387|AAAAAAAADPEDFCAA|1963-12-23|767|3338|257|1963|1|12|23|4|1963|257|3338|Monday|1963Q4|N|N|N|2438365|2438698|2438022|2438295|N|N|N|N|N| +2438388|AAAAAAAAEPEDFCAA|1963-12-24|767|3339|257|1963|2|12|24|4|1963|257|3339|Tuesday|1963Q4|N|N|N|2438365|2438698|2438023|2438296|N|N|N|N|N| +2438389|AAAAAAAAFPEDFCAA|1963-12-25|767|3339|257|1963|3|12|25|4|1963|257|3339|Wednesday|1963Q4|N|N|N|2438365|2438698|2438024|2438297|N|N|N|N|N| +2438390|AAAAAAAAGPEDFCAA|1963-12-26|767|3339|257|1963|4|12|26|4|1963|257|3339|Thursday|1963Q4|Y|N|N|2438365|2438698|2438025|2438298|N|N|N|N|N| +2438391|AAAAAAAAHPEDFCAA|1963-12-27|767|3339|257|1963|5|12|27|4|1963|257|3339|Friday|1963Q4|N|Y|Y|2438365|2438698|2438026|2438299|N|N|N|N|N| +2438392|AAAAAAAAIPEDFCAA|1963-12-28|767|3339|257|1963|6|12|28|4|1963|257|3339|Saturday|1963Q4|N|Y|N|2438365|2438698|2438027|2438300|N|N|N|N|N| +2438393|AAAAAAAAJPEDFCAA|1963-12-29|767|3339|257|1963|0|12|29|4|1963|257|3339|Sunday|1963Q4|N|N|N|2438365|2438698|2438028|2438301|N|N|N|N|N| +2438394|AAAAAAAAKPEDFCAA|1963-12-30|767|3339|257|1963|1|12|30|4|1963|257|3339|Monday|1963Q4|N|N|N|2438365|2438698|2438029|2438302|N|N|N|N|N| +2438395|AAAAAAAALPEDFCAA|1963-12-31|767|3340|257|1963|2|12|31|4|1963|257|3340|Tuesday|1963Q4|N|N|N|2438365|2438698|2438030|2438303|N|N|N|N|N| +2438396|AAAAAAAAMPEDFCAA|1964-01-01|768|3340|257|1964|3|1|1|1|1964|257|3340|Wednesday|1964Q1|Y|N|N|2438396|2438395|2438031|2438304|N|N|N|N|N| +2438397|AAAAAAAANPEDFCAA|1964-01-02|768|3340|257|1964|4|1|2|1|1964|257|3340|Thursday|1964Q1|N|N|Y|2438396|2438395|2438032|2438305|N|N|N|N|N| +2438398|AAAAAAAAOPEDFCAA|1964-01-03|768|3340|257|1964|5|1|3|1|1964|257|3340|Friday|1964Q1|N|Y|N|2438396|2438395|2438033|2438306|N|N|N|N|N| +2438399|AAAAAAAAPPEDFCAA|1964-01-04|768|3340|257|1964|6|1|4|1|1964|257|3340|Saturday|1964Q1|N|Y|N|2438396|2438395|2438034|2438307|N|N|N|N|N| +2438400|AAAAAAAAAAFDFCAA|1964-01-05|768|3340|257|1964|0|1|5|1|1964|257|3340|Sunday|1964Q1|N|N|N|2438396|2438395|2438035|2438308|N|N|N|N|N| +2438401|AAAAAAAABAFDFCAA|1964-01-06|768|3340|257|1964|1|1|6|1|1964|257|3340|Monday|1964Q1|N|N|N|2438396|2438395|2438036|2438309|N|N|N|N|N| +2438402|AAAAAAAACAFDFCAA|1964-01-07|768|3341|257|1964|2|1|7|1|1964|257|3341|Tuesday|1964Q1|N|N|N|2438396|2438395|2438037|2438310|N|N|N|N|N| +2438403|AAAAAAAADAFDFCAA|1964-01-08|768|3341|257|1964|3|1|8|1|1964|257|3341|Wednesday|1964Q1|N|N|N|2438396|2438395|2438038|2438311|N|N|N|N|N| +2438404|AAAAAAAAEAFDFCAA|1964-01-09|768|3341|257|1964|4|1|9|1|1964|257|3341|Thursday|1964Q1|N|N|N|2438396|2438395|2438039|2438312|N|N|N|N|N| +2438405|AAAAAAAAFAFDFCAA|1964-01-10|768|3341|257|1964|5|1|10|1|1964|257|3341|Friday|1964Q1|N|Y|N|2438396|2438395|2438040|2438313|N|N|N|N|N| +2438406|AAAAAAAAGAFDFCAA|1964-01-11|768|3341|257|1964|6|1|11|1|1964|257|3341|Saturday|1964Q1|N|Y|N|2438396|2438395|2438041|2438314|N|N|N|N|N| +2438407|AAAAAAAAHAFDFCAA|1964-01-12|768|3341|257|1964|0|1|12|1|1964|257|3341|Sunday|1964Q1|N|N|N|2438396|2438395|2438042|2438315|N|N|N|N|N| +2438408|AAAAAAAAIAFDFCAA|1964-01-13|768|3341|257|1964|1|1|13|1|1964|257|3341|Monday|1964Q1|N|N|N|2438396|2438395|2438043|2438316|N|N|N|N|N| +2438409|AAAAAAAAJAFDFCAA|1964-01-14|768|3342|257|1964|2|1|14|1|1964|257|3342|Tuesday|1964Q1|N|N|N|2438396|2438395|2438044|2438317|N|N|N|N|N| +2438410|AAAAAAAAKAFDFCAA|1964-01-15|768|3342|257|1964|3|1|15|1|1964|257|3342|Wednesday|1964Q1|N|N|N|2438396|2438395|2438045|2438318|N|N|N|N|N| +2438411|AAAAAAAALAFDFCAA|1964-01-16|768|3342|257|1964|4|1|16|1|1964|257|3342|Thursday|1964Q1|N|N|N|2438396|2438395|2438046|2438319|N|N|N|N|N| +2438412|AAAAAAAAMAFDFCAA|1964-01-17|768|3342|257|1964|5|1|17|1|1964|257|3342|Friday|1964Q1|N|Y|N|2438396|2438395|2438047|2438320|N|N|N|N|N| +2438413|AAAAAAAANAFDFCAA|1964-01-18|768|3342|257|1964|6|1|18|1|1964|257|3342|Saturday|1964Q1|N|Y|N|2438396|2438395|2438048|2438321|N|N|N|N|N| +2438414|AAAAAAAAOAFDFCAA|1964-01-19|768|3342|257|1964|0|1|19|1|1964|257|3342|Sunday|1964Q1|N|N|N|2438396|2438395|2438049|2438322|N|N|N|N|N| +2438415|AAAAAAAAPAFDFCAA|1964-01-20|768|3342|257|1964|1|1|20|1|1964|257|3342|Monday|1964Q1|N|N|N|2438396|2438395|2438050|2438323|N|N|N|N|N| +2438416|AAAAAAAAABFDFCAA|1964-01-21|768|3343|257|1964|2|1|21|1|1964|257|3343|Tuesday|1964Q1|N|N|N|2438396|2438395|2438051|2438324|N|N|N|N|N| +2438417|AAAAAAAABBFDFCAA|1964-01-22|768|3343|257|1964|3|1|22|1|1964|257|3343|Wednesday|1964Q1|N|N|N|2438396|2438395|2438052|2438325|N|N|N|N|N| +2438418|AAAAAAAACBFDFCAA|1964-01-23|768|3343|257|1964|4|1|23|1|1964|257|3343|Thursday|1964Q1|N|N|N|2438396|2438395|2438053|2438326|N|N|N|N|N| +2438419|AAAAAAAADBFDFCAA|1964-01-24|768|3343|257|1964|5|1|24|1|1964|257|3343|Friday|1964Q1|N|Y|N|2438396|2438395|2438054|2438327|N|N|N|N|N| +2438420|AAAAAAAAEBFDFCAA|1964-01-25|768|3343|257|1964|6|1|25|1|1964|257|3343|Saturday|1964Q1|N|Y|N|2438396|2438395|2438055|2438328|N|N|N|N|N| +2438421|AAAAAAAAFBFDFCAA|1964-01-26|768|3343|257|1964|0|1|26|1|1964|257|3343|Sunday|1964Q1|N|N|N|2438396|2438395|2438056|2438329|N|N|N|N|N| +2438422|AAAAAAAAGBFDFCAA|1964-01-27|768|3343|257|1964|1|1|27|1|1964|257|3343|Monday|1964Q1|N|N|N|2438396|2438395|2438057|2438330|N|N|N|N|N| +2438423|AAAAAAAAHBFDFCAA|1964-01-28|768|3344|257|1964|2|1|28|1|1964|257|3344|Tuesday|1964Q1|N|N|N|2438396|2438395|2438058|2438331|N|N|N|N|N| +2438424|AAAAAAAAIBFDFCAA|1964-01-29|768|3344|257|1964|3|1|29|1|1964|257|3344|Wednesday|1964Q1|N|N|N|2438396|2438395|2438059|2438332|N|N|N|N|N| +2438425|AAAAAAAAJBFDFCAA|1964-01-30|768|3344|257|1964|4|1|30|1|1964|257|3344|Thursday|1964Q1|N|N|N|2438396|2438395|2438060|2438333|N|N|N|N|N| +2438426|AAAAAAAAKBFDFCAA|1964-01-31|768|3344|257|1964|5|1|31|1|1964|257|3344|Friday|1964Q1|N|Y|N|2438396|2438395|2438061|2438334|N|N|N|N|N| +2438427|AAAAAAAALBFDFCAA|1964-02-01|769|3344|257|1964|6|2|1|1|1964|257|3344|Saturday|1964Q1|N|Y|N|2438427|2438457|2438062|2438335|N|N|N|N|N| +2438428|AAAAAAAAMBFDFCAA|1964-02-02|769|3344|257|1964|0|2|2|1|1964|257|3344|Sunday|1964Q1|N|N|N|2438427|2438457|2438063|2438336|N|N|N|N|N| +2438429|AAAAAAAANBFDFCAA|1964-02-03|769|3344|257|1964|1|2|3|1|1964|257|3344|Monday|1964Q1|N|N|N|2438427|2438457|2438064|2438337|N|N|N|N|N| +2438430|AAAAAAAAOBFDFCAA|1964-02-04|769|3345|257|1964|2|2|4|1|1964|257|3345|Tuesday|1964Q1|N|N|N|2438427|2438457|2438065|2438338|N|N|N|N|N| +2438431|AAAAAAAAPBFDFCAA|1964-02-05|769|3345|257|1964|3|2|5|1|1964|257|3345|Wednesday|1964Q1|N|N|N|2438427|2438457|2438066|2438339|N|N|N|N|N| +2438432|AAAAAAAAACFDFCAA|1964-02-06|769|3345|257|1964|4|2|6|1|1964|257|3345|Thursday|1964Q1|N|N|N|2438427|2438457|2438067|2438340|N|N|N|N|N| +2438433|AAAAAAAABCFDFCAA|1964-02-07|769|3345|257|1964|5|2|7|1|1964|257|3345|Friday|1964Q1|N|Y|N|2438427|2438457|2438068|2438341|N|N|N|N|N| +2438434|AAAAAAAACCFDFCAA|1964-02-08|769|3345|257|1964|6|2|8|1|1964|257|3345|Saturday|1964Q1|N|Y|N|2438427|2438457|2438069|2438342|N|N|N|N|N| +2438435|AAAAAAAADCFDFCAA|1964-02-09|769|3345|257|1964|0|2|9|1|1964|257|3345|Sunday|1964Q1|N|N|N|2438427|2438457|2438070|2438343|N|N|N|N|N| +2438436|AAAAAAAAECFDFCAA|1964-02-10|769|3345|257|1964|1|2|10|1|1964|257|3345|Monday|1964Q1|N|N|N|2438427|2438457|2438071|2438344|N|N|N|N|N| +2438437|AAAAAAAAFCFDFCAA|1964-02-11|769|3346|257|1964|2|2|11|1|1964|257|3346|Tuesday|1964Q1|N|N|N|2438427|2438457|2438072|2438345|N|N|N|N|N| +2438438|AAAAAAAAGCFDFCAA|1964-02-12|769|3346|257|1964|3|2|12|1|1964|257|3346|Wednesday|1964Q1|N|N|N|2438427|2438457|2438073|2438346|N|N|N|N|N| +2438439|AAAAAAAAHCFDFCAA|1964-02-13|769|3346|257|1964|4|2|13|1|1964|257|3346|Thursday|1964Q1|N|N|N|2438427|2438457|2438074|2438347|N|N|N|N|N| +2438440|AAAAAAAAICFDFCAA|1964-02-14|769|3346|257|1964|5|2|14|1|1964|257|3346|Friday|1964Q1|N|Y|N|2438427|2438457|2438075|2438348|N|N|N|N|N| +2438441|AAAAAAAAJCFDFCAA|1964-02-15|769|3346|257|1964|6|2|15|1|1964|257|3346|Saturday|1964Q1|N|Y|N|2438427|2438457|2438076|2438349|N|N|N|N|N| +2438442|AAAAAAAAKCFDFCAA|1964-02-16|769|3346|257|1964|0|2|16|1|1964|257|3346|Sunday|1964Q1|N|N|N|2438427|2438457|2438077|2438350|N|N|N|N|N| +2438443|AAAAAAAALCFDFCAA|1964-02-17|769|3346|257|1964|1|2|17|1|1964|257|3346|Monday|1964Q1|N|N|N|2438427|2438457|2438078|2438351|N|N|N|N|N| +2438444|AAAAAAAAMCFDFCAA|1964-02-18|769|3347|257|1964|2|2|18|1|1964|257|3347|Tuesday|1964Q1|N|N|N|2438427|2438457|2438079|2438352|N|N|N|N|N| +2438445|AAAAAAAANCFDFCAA|1964-02-19|769|3347|257|1964|3|2|19|1|1964|257|3347|Wednesday|1964Q1|N|N|N|2438427|2438457|2438080|2438353|N|N|N|N|N| +2438446|AAAAAAAAOCFDFCAA|1964-02-20|769|3347|257|1964|4|2|20|1|1964|257|3347|Thursday|1964Q1|N|N|N|2438427|2438457|2438081|2438354|N|N|N|N|N| +2438447|AAAAAAAAPCFDFCAA|1964-02-21|769|3347|257|1964|5|2|21|1|1964|257|3347|Friday|1964Q1|N|Y|N|2438427|2438457|2438082|2438355|N|N|N|N|N| +2438448|AAAAAAAAADFDFCAA|1964-02-22|769|3347|257|1964|6|2|22|1|1964|257|3347|Saturday|1964Q1|N|Y|N|2438427|2438457|2438083|2438356|N|N|N|N|N| +2438449|AAAAAAAABDFDFCAA|1964-02-23|769|3347|257|1964|0|2|23|1|1964|257|3347|Sunday|1964Q1|N|N|N|2438427|2438457|2438084|2438357|N|N|N|N|N| +2438450|AAAAAAAACDFDFCAA|1964-02-24|769|3347|257|1964|1|2|24|1|1964|257|3347|Monday|1964Q1|N|N|N|2438427|2438457|2438085|2438358|N|N|N|N|N| +2438451|AAAAAAAADDFDFCAA|1964-02-25|769|3348|257|1964|2|2|25|1|1964|257|3348|Tuesday|1964Q1|N|N|N|2438427|2438457|2438086|2438359|N|N|N|N|N| +2438452|AAAAAAAAEDFDFCAA|1964-02-26|769|3348|257|1964|3|2|26|1|1964|257|3348|Wednesday|1964Q1|N|N|N|2438427|2438457|2438087|2438360|N|N|N|N|N| +2438453|AAAAAAAAFDFDFCAA|1964-02-27|769|3348|257|1964|4|2|27|1|1964|257|3348|Thursday|1964Q1|N|N|N|2438427|2438457|2438088|2438361|N|N|N|N|N| +2438454|AAAAAAAAGDFDFCAA|1964-02-28|769|3348|257|1964|5|2|28|1|1964|257|3348|Friday|1964Q1|N|Y|N|2438427|2438457|2438089|2438362|N|N|N|N|N| +2438455|AAAAAAAAHDFDFCAA|1964-02-29|769|3348|257|1964|6|2|29|1|1964|257|3348|Saturday|1964Q1|N|Y|N|2438427|2438457|2438089|2438363|N|N|N|N|N| +2438456|AAAAAAAAIDFDFCAA|1964-03-01|770|3348|258|1964|0|3|1|1|1964|258|3348|Sunday|1964Q1|N|N|N|2438456|2438515|2438090|2438364|N|N|N|N|N| +2438457|AAAAAAAAJDFDFCAA|1964-03-02|770|3348|258|1964|1|3|2|1|1964|258|3348|Monday|1964Q1|N|N|N|2438456|2438515|2438091|2438365|N|N|N|N|N| +2438458|AAAAAAAAKDFDFCAA|1964-03-03|770|3349|258|1964|2|3|3|1|1964|258|3349|Tuesday|1964Q1|N|N|N|2438456|2438515|2438092|2438366|N|N|N|N|N| +2438459|AAAAAAAALDFDFCAA|1964-03-04|770|3349|258|1964|3|3|4|1|1964|258|3349|Wednesday|1964Q1|N|N|N|2438456|2438515|2438093|2438367|N|N|N|N|N| +2438460|AAAAAAAAMDFDFCAA|1964-03-05|770|3349|258|1964|4|3|5|1|1964|258|3349|Thursday|1964Q1|N|N|N|2438456|2438515|2438094|2438368|N|N|N|N|N| +2438461|AAAAAAAANDFDFCAA|1964-03-06|770|3349|258|1964|5|3|6|1|1964|258|3349|Friday|1964Q1|N|Y|N|2438456|2438515|2438095|2438369|N|N|N|N|N| +2438462|AAAAAAAAODFDFCAA|1964-03-07|770|3349|258|1964|6|3|7|1|1964|258|3349|Saturday|1964Q1|N|Y|N|2438456|2438515|2438096|2438370|N|N|N|N|N| +2438463|AAAAAAAAPDFDFCAA|1964-03-08|770|3349|258|1964|0|3|8|1|1964|258|3349|Sunday|1964Q1|N|N|N|2438456|2438515|2438097|2438371|N|N|N|N|N| +2438464|AAAAAAAAAEFDFCAA|1964-03-09|770|3349|258|1964|1|3|9|1|1964|258|3349|Monday|1964Q1|N|N|N|2438456|2438515|2438098|2438372|N|N|N|N|N| +2438465|AAAAAAAABEFDFCAA|1964-03-10|770|3350|258|1964|2|3|10|1|1964|258|3350|Tuesday|1964Q1|N|N|N|2438456|2438515|2438099|2438373|N|N|N|N|N| +2438466|AAAAAAAACEFDFCAA|1964-03-11|770|3350|258|1964|3|3|11|1|1964|258|3350|Wednesday|1964Q1|N|N|N|2438456|2438515|2438100|2438374|N|N|N|N|N| +2438467|AAAAAAAADEFDFCAA|1964-03-12|770|3350|258|1964|4|3|12|1|1964|258|3350|Thursday|1964Q1|N|N|N|2438456|2438515|2438101|2438375|N|N|N|N|N| +2438468|AAAAAAAAEEFDFCAA|1964-03-13|770|3350|258|1964|5|3|13|1|1964|258|3350|Friday|1964Q1|N|Y|N|2438456|2438515|2438102|2438376|N|N|N|N|N| +2438469|AAAAAAAAFEFDFCAA|1964-03-14|770|3350|258|1964|6|3|14|1|1964|258|3350|Saturday|1964Q1|N|Y|N|2438456|2438515|2438103|2438377|N|N|N|N|N| +2438470|AAAAAAAAGEFDFCAA|1964-03-15|770|3350|258|1964|0|3|15|1|1964|258|3350|Sunday|1964Q1|N|N|N|2438456|2438515|2438104|2438378|N|N|N|N|N| +2438471|AAAAAAAAHEFDFCAA|1964-03-16|770|3350|258|1964|1|3|16|1|1964|258|3350|Monday|1964Q1|N|N|N|2438456|2438515|2438105|2438379|N|N|N|N|N| +2438472|AAAAAAAAIEFDFCAA|1964-03-17|770|3351|258|1964|2|3|17|1|1964|258|3351|Tuesday|1964Q1|N|N|N|2438456|2438515|2438106|2438380|N|N|N|N|N| +2438473|AAAAAAAAJEFDFCAA|1964-03-18|770|3351|258|1964|3|3|18|1|1964|258|3351|Wednesday|1964Q1|N|N|N|2438456|2438515|2438107|2438381|N|N|N|N|N| +2438474|AAAAAAAAKEFDFCAA|1964-03-19|770|3351|258|1964|4|3|19|1|1964|258|3351|Thursday|1964Q1|N|N|N|2438456|2438515|2438108|2438382|N|N|N|N|N| +2438475|AAAAAAAALEFDFCAA|1964-03-20|770|3351|258|1964|5|3|20|1|1964|258|3351|Friday|1964Q1|N|Y|N|2438456|2438515|2438109|2438383|N|N|N|N|N| +2438476|AAAAAAAAMEFDFCAA|1964-03-21|770|3351|258|1964|6|3|21|1|1964|258|3351|Saturday|1964Q1|N|Y|N|2438456|2438515|2438110|2438384|N|N|N|N|N| +2438477|AAAAAAAANEFDFCAA|1964-03-22|770|3351|258|1964|0|3|22|1|1964|258|3351|Sunday|1964Q1|N|N|N|2438456|2438515|2438111|2438385|N|N|N|N|N| +2438478|AAAAAAAAOEFDFCAA|1964-03-23|770|3351|258|1964|1|3|23|1|1964|258|3351|Monday|1964Q1|N|N|N|2438456|2438515|2438112|2438386|N|N|N|N|N| +2438479|AAAAAAAAPEFDFCAA|1964-03-24|770|3352|258|1964|2|3|24|1|1964|258|3352|Tuesday|1964Q1|N|N|N|2438456|2438515|2438113|2438387|N|N|N|N|N| +2438480|AAAAAAAAAFFDFCAA|1964-03-25|770|3352|258|1964|3|3|25|1|1964|258|3352|Wednesday|1964Q1|N|N|N|2438456|2438515|2438114|2438388|N|N|N|N|N| +2438481|AAAAAAAABFFDFCAA|1964-03-26|770|3352|258|1964|4|3|26|1|1964|258|3352|Thursday|1964Q1|N|N|N|2438456|2438515|2438115|2438389|N|N|N|N|N| +2438482|AAAAAAAACFFDFCAA|1964-03-27|770|3352|258|1964|5|3|27|1|1964|258|3352|Friday|1964Q1|N|Y|N|2438456|2438515|2438116|2438390|N|N|N|N|N| +2438483|AAAAAAAADFFDFCAA|1964-03-28|770|3352|258|1964|6|3|28|1|1964|258|3352|Saturday|1964Q1|N|Y|N|2438456|2438515|2438117|2438391|N|N|N|N|N| +2438484|AAAAAAAAEFFDFCAA|1964-03-29|770|3352|258|1964|0|3|29|1|1964|258|3352|Sunday|1964Q1|N|N|N|2438456|2438515|2438118|2438392|N|N|N|N|N| +2438485|AAAAAAAAFFFDFCAA|1964-03-30|770|3352|258|1964|1|3|30|1|1964|258|3352|Monday|1964Q1|N|N|N|2438456|2438515|2438119|2438393|N|N|N|N|N| +2438486|AAAAAAAAGFFDFCAA|1964-03-31|770|3353|258|1964|2|3|31|1|1964|258|3353|Tuesday|1964Q1|N|N|N|2438456|2438515|2438120|2438394|N|N|N|N|N| +2438487|AAAAAAAAHFFDFCAA|1964-04-01|771|3353|258|1964|3|4|1|2|1964|258|3353|Wednesday|1964Q2|N|N|N|2438487|2438577|2438121|2438396|N|N|N|N|N| +2438488|AAAAAAAAIFFDFCAA|1964-04-02|771|3353|258|1964|4|4|2|2|1964|258|3353|Thursday|1964Q2|N|N|N|2438487|2438577|2438122|2438397|N|N|N|N|N| +2438489|AAAAAAAAJFFDFCAA|1964-04-03|771|3353|258|1964|5|4|3|2|1964|258|3353|Friday|1964Q2|N|Y|N|2438487|2438577|2438123|2438398|N|N|N|N|N| +2438490|AAAAAAAAKFFDFCAA|1964-04-04|771|3353|258|1964|6|4|4|2|1964|258|3353|Saturday|1964Q2|N|Y|N|2438487|2438577|2438124|2438399|N|N|N|N|N| +2438491|AAAAAAAALFFDFCAA|1964-04-05|771|3353|258|1964|0|4|5|2|1964|258|3353|Sunday|1964Q2|N|N|N|2438487|2438577|2438125|2438400|N|N|N|N|N| +2438492|AAAAAAAAMFFDFCAA|1964-04-06|771|3353|258|1964|1|4|6|2|1964|258|3353|Monday|1964Q2|N|N|N|2438487|2438577|2438126|2438401|N|N|N|N|N| +2438493|AAAAAAAANFFDFCAA|1964-04-07|771|3354|258|1964|2|4|7|2|1964|258|3354|Tuesday|1964Q2|N|N|N|2438487|2438577|2438127|2438402|N|N|N|N|N| +2438494|AAAAAAAAOFFDFCAA|1964-04-08|771|3354|258|1964|3|4|8|2|1964|258|3354|Wednesday|1964Q2|N|N|N|2438487|2438577|2438128|2438403|N|N|N|N|N| +2438495|AAAAAAAAPFFDFCAA|1964-04-09|771|3354|258|1964|4|4|9|2|1964|258|3354|Thursday|1964Q2|N|N|N|2438487|2438577|2438129|2438404|N|N|N|N|N| +2438496|AAAAAAAAAGFDFCAA|1964-04-10|771|3354|258|1964|5|4|10|2|1964|258|3354|Friday|1964Q2|N|Y|N|2438487|2438577|2438130|2438405|N|N|N|N|N| +2438497|AAAAAAAABGFDFCAA|1964-04-11|771|3354|258|1964|6|4|11|2|1964|258|3354|Saturday|1964Q2|N|Y|N|2438487|2438577|2438131|2438406|N|N|N|N|N| +2438498|AAAAAAAACGFDFCAA|1964-04-12|771|3354|258|1964|0|4|12|2|1964|258|3354|Sunday|1964Q2|N|N|N|2438487|2438577|2438132|2438407|N|N|N|N|N| +2438499|AAAAAAAADGFDFCAA|1964-04-13|771|3354|258|1964|1|4|13|2|1964|258|3354|Monday|1964Q2|N|N|N|2438487|2438577|2438133|2438408|N|N|N|N|N| +2438500|AAAAAAAAEGFDFCAA|1964-04-14|771|3355|258|1964|2|4|14|2|1964|258|3355|Tuesday|1964Q2|N|N|N|2438487|2438577|2438134|2438409|N|N|N|N|N| +2438501|AAAAAAAAFGFDFCAA|1964-04-15|771|3355|258|1964|3|4|15|2|1964|258|3355|Wednesday|1964Q2|N|N|N|2438487|2438577|2438135|2438410|N|N|N|N|N| +2438502|AAAAAAAAGGFDFCAA|1964-04-16|771|3355|258|1964|4|4|16|2|1964|258|3355|Thursday|1964Q2|N|N|N|2438487|2438577|2438136|2438411|N|N|N|N|N| +2438503|AAAAAAAAHGFDFCAA|1964-04-17|771|3355|258|1964|5|4|17|2|1964|258|3355|Friday|1964Q2|N|Y|N|2438487|2438577|2438137|2438412|N|N|N|N|N| +2438504|AAAAAAAAIGFDFCAA|1964-04-18|771|3355|258|1964|6|4|18|2|1964|258|3355|Saturday|1964Q2|N|Y|N|2438487|2438577|2438138|2438413|N|N|N|N|N| +2438505|AAAAAAAAJGFDFCAA|1964-04-19|771|3355|258|1964|0|4|19|2|1964|258|3355|Sunday|1964Q2|N|N|N|2438487|2438577|2438139|2438414|N|N|N|N|N| +2438506|AAAAAAAAKGFDFCAA|1964-04-20|771|3355|258|1964|1|4|20|2|1964|258|3355|Monday|1964Q2|N|N|N|2438487|2438577|2438140|2438415|N|N|N|N|N| +2438507|AAAAAAAALGFDFCAA|1964-04-21|771|3356|258|1964|2|4|21|2|1964|258|3356|Tuesday|1964Q2|N|N|N|2438487|2438577|2438141|2438416|N|N|N|N|N| +2438508|AAAAAAAAMGFDFCAA|1964-04-22|771|3356|258|1964|3|4|22|2|1964|258|3356|Wednesday|1964Q2|N|N|N|2438487|2438577|2438142|2438417|N|N|N|N|N| +2438509|AAAAAAAANGFDFCAA|1964-04-23|771|3356|258|1964|4|4|23|2|1964|258|3356|Thursday|1964Q2|N|N|N|2438487|2438577|2438143|2438418|N|N|N|N|N| +2438510|AAAAAAAAOGFDFCAA|1964-04-24|771|3356|258|1964|5|4|24|2|1964|258|3356|Friday|1964Q2|N|Y|N|2438487|2438577|2438144|2438419|N|N|N|N|N| +2438511|AAAAAAAAPGFDFCAA|1964-04-25|771|3356|258|1964|6|4|25|2|1964|258|3356|Saturday|1964Q2|N|Y|N|2438487|2438577|2438145|2438420|N|N|N|N|N| +2438512|AAAAAAAAAHFDFCAA|1964-04-26|771|3356|258|1964|0|4|26|2|1964|258|3356|Sunday|1964Q2|N|N|N|2438487|2438577|2438146|2438421|N|N|N|N|N| +2438513|AAAAAAAABHFDFCAA|1964-04-27|771|3356|258|1964|1|4|27|2|1964|258|3356|Monday|1964Q2|N|N|N|2438487|2438577|2438147|2438422|N|N|N|N|N| +2438514|AAAAAAAACHFDFCAA|1964-04-28|771|3357|258|1964|2|4|28|2|1964|258|3357|Tuesday|1964Q2|N|N|N|2438487|2438577|2438148|2438423|N|N|N|N|N| +2438515|AAAAAAAADHFDFCAA|1964-04-29|771|3357|258|1964|3|4|29|2|1964|258|3357|Wednesday|1964Q2|N|N|N|2438487|2438577|2438149|2438424|N|N|N|N|N| +2438516|AAAAAAAAEHFDFCAA|1964-04-30|771|3357|258|1964|4|4|30|2|1964|258|3357|Thursday|1964Q2|N|N|N|2438487|2438577|2438150|2438425|N|N|N|N|N| +2438517|AAAAAAAAFHFDFCAA|1964-05-01|772|3357|258|1964|5|5|1|2|1964|258|3357|Friday|1964Q2|N|Y|N|2438517|2438637|2438151|2438426|N|N|N|N|N| +2438518|AAAAAAAAGHFDFCAA|1964-05-02|772|3357|258|1964|6|5|2|2|1964|258|3357|Saturday|1964Q2|N|Y|N|2438517|2438637|2438152|2438427|N|N|N|N|N| +2438519|AAAAAAAAHHFDFCAA|1964-05-03|772|3357|258|1964|0|5|3|2|1964|258|3357|Sunday|1964Q2|N|N|N|2438517|2438637|2438153|2438428|N|N|N|N|N| +2438520|AAAAAAAAIHFDFCAA|1964-05-04|772|3357|258|1964|1|5|4|2|1964|258|3357|Monday|1964Q2|N|N|N|2438517|2438637|2438154|2438429|N|N|N|N|N| +2438521|AAAAAAAAJHFDFCAA|1964-05-05|772|3358|258|1964|2|5|5|2|1964|258|3358|Tuesday|1964Q2|N|N|N|2438517|2438637|2438155|2438430|N|N|N|N|N| +2438522|AAAAAAAAKHFDFCAA|1964-05-06|772|3358|258|1964|3|5|6|2|1964|258|3358|Wednesday|1964Q2|N|N|N|2438517|2438637|2438156|2438431|N|N|N|N|N| +2438523|AAAAAAAALHFDFCAA|1964-05-07|772|3358|258|1964|4|5|7|2|1964|258|3358|Thursday|1964Q2|N|N|N|2438517|2438637|2438157|2438432|N|N|N|N|N| +2438524|AAAAAAAAMHFDFCAA|1964-05-08|772|3358|258|1964|5|5|8|2|1964|258|3358|Friday|1964Q2|N|Y|N|2438517|2438637|2438158|2438433|N|N|N|N|N| +2438525|AAAAAAAANHFDFCAA|1964-05-09|772|3358|258|1964|6|5|9|2|1964|258|3358|Saturday|1964Q2|N|Y|N|2438517|2438637|2438159|2438434|N|N|N|N|N| +2438526|AAAAAAAAOHFDFCAA|1964-05-10|772|3358|258|1964|0|5|10|2|1964|258|3358|Sunday|1964Q2|N|N|N|2438517|2438637|2438160|2438435|N|N|N|N|N| +2438527|AAAAAAAAPHFDFCAA|1964-05-11|772|3358|258|1964|1|5|11|2|1964|258|3358|Monday|1964Q2|N|N|N|2438517|2438637|2438161|2438436|N|N|N|N|N| +2438528|AAAAAAAAAIFDFCAA|1964-05-12|772|3359|258|1964|2|5|12|2|1964|258|3359|Tuesday|1964Q2|N|N|N|2438517|2438637|2438162|2438437|N|N|N|N|N| +2438529|AAAAAAAABIFDFCAA|1964-05-13|772|3359|258|1964|3|5|13|2|1964|258|3359|Wednesday|1964Q2|N|N|N|2438517|2438637|2438163|2438438|N|N|N|N|N| +2438530|AAAAAAAACIFDFCAA|1964-05-14|772|3359|258|1964|4|5|14|2|1964|258|3359|Thursday|1964Q2|N|N|N|2438517|2438637|2438164|2438439|N|N|N|N|N| +2438531|AAAAAAAADIFDFCAA|1964-05-15|772|3359|258|1964|5|5|15|2|1964|258|3359|Friday|1964Q2|N|Y|N|2438517|2438637|2438165|2438440|N|N|N|N|N| +2438532|AAAAAAAAEIFDFCAA|1964-05-16|772|3359|258|1964|6|5|16|2|1964|258|3359|Saturday|1964Q2|N|Y|N|2438517|2438637|2438166|2438441|N|N|N|N|N| +2438533|AAAAAAAAFIFDFCAA|1964-05-17|772|3359|258|1964|0|5|17|2|1964|258|3359|Sunday|1964Q2|N|N|N|2438517|2438637|2438167|2438442|N|N|N|N|N| +2438534|AAAAAAAAGIFDFCAA|1964-05-18|772|3359|258|1964|1|5|18|2|1964|258|3359|Monday|1964Q2|N|N|N|2438517|2438637|2438168|2438443|N|N|N|N|N| +2438535|AAAAAAAAHIFDFCAA|1964-05-19|772|3360|258|1964|2|5|19|2|1964|258|3360|Tuesday|1964Q2|N|N|N|2438517|2438637|2438169|2438444|N|N|N|N|N| +2438536|AAAAAAAAIIFDFCAA|1964-05-20|772|3360|258|1964|3|5|20|2|1964|258|3360|Wednesday|1964Q2|N|N|N|2438517|2438637|2438170|2438445|N|N|N|N|N| +2438537|AAAAAAAAJIFDFCAA|1964-05-21|772|3360|258|1964|4|5|21|2|1964|258|3360|Thursday|1964Q2|N|N|N|2438517|2438637|2438171|2438446|N|N|N|N|N| +2438538|AAAAAAAAKIFDFCAA|1964-05-22|772|3360|258|1964|5|5|22|2|1964|258|3360|Friday|1964Q2|N|Y|N|2438517|2438637|2438172|2438447|N|N|N|N|N| +2438539|AAAAAAAALIFDFCAA|1964-05-23|772|3360|258|1964|6|5|23|2|1964|258|3360|Saturday|1964Q2|N|Y|N|2438517|2438637|2438173|2438448|N|N|N|N|N| +2438540|AAAAAAAAMIFDFCAA|1964-05-24|772|3360|258|1964|0|5|24|2|1964|258|3360|Sunday|1964Q2|N|N|N|2438517|2438637|2438174|2438449|N|N|N|N|N| +2438541|AAAAAAAANIFDFCAA|1964-05-25|772|3360|258|1964|1|5|25|2|1964|258|3360|Monday|1964Q2|N|N|N|2438517|2438637|2438175|2438450|N|N|N|N|N| +2438542|AAAAAAAAOIFDFCAA|1964-05-26|772|3361|258|1964|2|5|26|2|1964|258|3361|Tuesday|1964Q2|N|N|N|2438517|2438637|2438176|2438451|N|N|N|N|N| +2438543|AAAAAAAAPIFDFCAA|1964-05-27|772|3361|258|1964|3|5|27|2|1964|258|3361|Wednesday|1964Q2|N|N|N|2438517|2438637|2438177|2438452|N|N|N|N|N| +2438544|AAAAAAAAAJFDFCAA|1964-05-28|772|3361|258|1964|4|5|28|2|1964|258|3361|Thursday|1964Q2|N|N|N|2438517|2438637|2438178|2438453|N|N|N|N|N| +2438545|AAAAAAAABJFDFCAA|1964-05-29|772|3361|258|1964|5|5|29|2|1964|258|3361|Friday|1964Q2|N|Y|N|2438517|2438637|2438179|2438454|N|N|N|N|N| +2438546|AAAAAAAACJFDFCAA|1964-05-30|772|3361|258|1964|6|5|30|2|1964|258|3361|Saturday|1964Q2|N|Y|N|2438517|2438637|2438180|2438455|N|N|N|N|N| +2438547|AAAAAAAADJFDFCAA|1964-05-31|772|3361|258|1964|0|5|31|2|1964|258|3361|Sunday|1964Q2|N|N|N|2438517|2438637|2438181|2438456|N|N|N|N|N| +2438548|AAAAAAAAEJFDFCAA|1964-06-01|773|3361|259|1964|1|6|1|2|1964|259|3361|Monday|1964Q2|N|N|N|2438548|2438699|2438182|2438457|N|N|N|N|N| +2438549|AAAAAAAAFJFDFCAA|1964-06-02|773|3362|259|1964|2|6|2|2|1964|259|3362|Tuesday|1964Q2|N|N|N|2438548|2438699|2438183|2438458|N|N|N|N|N| +2438550|AAAAAAAAGJFDFCAA|1964-06-03|773|3362|259|1964|3|6|3|2|1964|259|3362|Wednesday|1964Q2|N|N|N|2438548|2438699|2438184|2438459|N|N|N|N|N| +2438551|AAAAAAAAHJFDFCAA|1964-06-04|773|3362|259|1964|4|6|4|2|1964|259|3362|Thursday|1964Q2|N|N|N|2438548|2438699|2438185|2438460|N|N|N|N|N| +2438552|AAAAAAAAIJFDFCAA|1964-06-05|773|3362|259|1964|5|6|5|2|1964|259|3362|Friday|1964Q2|N|Y|N|2438548|2438699|2438186|2438461|N|N|N|N|N| +2438553|AAAAAAAAJJFDFCAA|1964-06-06|773|3362|259|1964|6|6|6|2|1964|259|3362|Saturday|1964Q2|N|Y|N|2438548|2438699|2438187|2438462|N|N|N|N|N| +2438554|AAAAAAAAKJFDFCAA|1964-06-07|773|3362|259|1964|0|6|7|2|1964|259|3362|Sunday|1964Q2|N|N|N|2438548|2438699|2438188|2438463|N|N|N|N|N| +2438555|AAAAAAAALJFDFCAA|1964-06-08|773|3362|259|1964|1|6|8|2|1964|259|3362|Monday|1964Q2|N|N|N|2438548|2438699|2438189|2438464|N|N|N|N|N| +2438556|AAAAAAAAMJFDFCAA|1964-06-09|773|3363|259|1964|2|6|9|2|1964|259|3363|Tuesday|1964Q2|N|N|N|2438548|2438699|2438190|2438465|N|N|N|N|N| +2438557|AAAAAAAANJFDFCAA|1964-06-10|773|3363|259|1964|3|6|10|2|1964|259|3363|Wednesday|1964Q2|N|N|N|2438548|2438699|2438191|2438466|N|N|N|N|N| +2438558|AAAAAAAAOJFDFCAA|1964-06-11|773|3363|259|1964|4|6|11|2|1964|259|3363|Thursday|1964Q2|N|N|N|2438548|2438699|2438192|2438467|N|N|N|N|N| +2438559|AAAAAAAAPJFDFCAA|1964-06-12|773|3363|259|1964|5|6|12|2|1964|259|3363|Friday|1964Q2|N|Y|N|2438548|2438699|2438193|2438468|N|N|N|N|N| +2438560|AAAAAAAAAKFDFCAA|1964-06-13|773|3363|259|1964|6|6|13|2|1964|259|3363|Saturday|1964Q2|N|Y|N|2438548|2438699|2438194|2438469|N|N|N|N|N| +2438561|AAAAAAAABKFDFCAA|1964-06-14|773|3363|259|1964|0|6|14|2|1964|259|3363|Sunday|1964Q2|N|N|N|2438548|2438699|2438195|2438470|N|N|N|N|N| +2438562|AAAAAAAACKFDFCAA|1964-06-15|773|3363|259|1964|1|6|15|2|1964|259|3363|Monday|1964Q2|N|N|N|2438548|2438699|2438196|2438471|N|N|N|N|N| +2438563|AAAAAAAADKFDFCAA|1964-06-16|773|3364|259|1964|2|6|16|2|1964|259|3364|Tuesday|1964Q2|N|N|N|2438548|2438699|2438197|2438472|N|N|N|N|N| +2438564|AAAAAAAAEKFDFCAA|1964-06-17|773|3364|259|1964|3|6|17|2|1964|259|3364|Wednesday|1964Q2|N|N|N|2438548|2438699|2438198|2438473|N|N|N|N|N| +2438565|AAAAAAAAFKFDFCAA|1964-06-18|773|3364|259|1964|4|6|18|2|1964|259|3364|Thursday|1964Q2|N|N|N|2438548|2438699|2438199|2438474|N|N|N|N|N| +2438566|AAAAAAAAGKFDFCAA|1964-06-19|773|3364|259|1964|5|6|19|2|1964|259|3364|Friday|1964Q2|N|Y|N|2438548|2438699|2438200|2438475|N|N|N|N|N| +2438567|AAAAAAAAHKFDFCAA|1964-06-20|773|3364|259|1964|6|6|20|2|1964|259|3364|Saturday|1964Q2|N|Y|N|2438548|2438699|2438201|2438476|N|N|N|N|N| +2438568|AAAAAAAAIKFDFCAA|1964-06-21|773|3364|259|1964|0|6|21|2|1964|259|3364|Sunday|1964Q2|N|N|N|2438548|2438699|2438202|2438477|N|N|N|N|N| +2438569|AAAAAAAAJKFDFCAA|1964-06-22|773|3364|259|1964|1|6|22|2|1964|259|3364|Monday|1964Q2|N|N|N|2438548|2438699|2438203|2438478|N|N|N|N|N| +2438570|AAAAAAAAKKFDFCAA|1964-06-23|773|3365|259|1964|2|6|23|2|1964|259|3365|Tuesday|1964Q2|N|N|N|2438548|2438699|2438204|2438479|N|N|N|N|N| +2438571|AAAAAAAALKFDFCAA|1964-06-24|773|3365|259|1964|3|6|24|2|1964|259|3365|Wednesday|1964Q2|N|N|N|2438548|2438699|2438205|2438480|N|N|N|N|N| +2438572|AAAAAAAAMKFDFCAA|1964-06-25|773|3365|259|1964|4|6|25|2|1964|259|3365|Thursday|1964Q2|N|N|N|2438548|2438699|2438206|2438481|N|N|N|N|N| +2438573|AAAAAAAANKFDFCAA|1964-06-26|773|3365|259|1964|5|6|26|2|1964|259|3365|Friday|1964Q2|N|Y|N|2438548|2438699|2438207|2438482|N|N|N|N|N| +2438574|AAAAAAAAOKFDFCAA|1964-06-27|773|3365|259|1964|6|6|27|2|1964|259|3365|Saturday|1964Q2|N|Y|N|2438548|2438699|2438208|2438483|N|N|N|N|N| +2438575|AAAAAAAAPKFDFCAA|1964-06-28|773|3365|259|1964|0|6|28|2|1964|259|3365|Sunday|1964Q2|N|N|N|2438548|2438699|2438209|2438484|N|N|N|N|N| +2438576|AAAAAAAAALFDFCAA|1964-06-29|773|3365|259|1964|1|6|29|2|1964|259|3365|Monday|1964Q2|N|N|N|2438548|2438699|2438210|2438485|N|N|N|N|N| +2438577|AAAAAAAABLFDFCAA|1964-06-30|773|3366|259|1964|2|6|30|2|1964|259|3366|Tuesday|1964Q2|N|N|N|2438548|2438699|2438211|2438486|N|N|N|N|N| +2438578|AAAAAAAACLFDFCAA|1964-07-01|774|3366|259|1964|3|7|1|3|1964|259|3366|Wednesday|1964Q3|N|N|N|2438578|2438759|2438212|2438487|N|N|N|N|N| +2438579|AAAAAAAADLFDFCAA|1964-07-02|774|3366|259|1964|4|7|2|3|1964|259|3366|Thursday|1964Q3|N|N|N|2438578|2438759|2438213|2438488|N|N|N|N|N| +2438580|AAAAAAAAELFDFCAA|1964-07-03|774|3366|259|1964|5|7|3|3|1964|259|3366|Friday|1964Q3|N|Y|N|2438578|2438759|2438214|2438489|N|N|N|N|N| +2438581|AAAAAAAAFLFDFCAA|1964-07-04|774|3366|259|1964|6|7|4|3|1964|259|3366|Saturday|1964Q3|Y|Y|N|2438578|2438759|2438215|2438490|N|N|N|N|N| +2438582|AAAAAAAAGLFDFCAA|1964-07-05|774|3366|259|1964|0|7|5|3|1964|259|3366|Sunday|1964Q3|N|N|Y|2438578|2438759|2438216|2438491|N|N|N|N|N| +2438583|AAAAAAAAHLFDFCAA|1964-07-06|774|3366|259|1964|1|7|6|3|1964|259|3366|Monday|1964Q3|N|N|N|2438578|2438759|2438217|2438492|N|N|N|N|N| +2438584|AAAAAAAAILFDFCAA|1964-07-07|774|3367|259|1964|2|7|7|3|1964|259|3367|Tuesday|1964Q3|N|N|N|2438578|2438759|2438218|2438493|N|N|N|N|N| +2438585|AAAAAAAAJLFDFCAA|1964-07-08|774|3367|259|1964|3|7|8|3|1964|259|3367|Wednesday|1964Q3|N|N|N|2438578|2438759|2438219|2438494|N|N|N|N|N| +2438586|AAAAAAAAKLFDFCAA|1964-07-09|774|3367|259|1964|4|7|9|3|1964|259|3367|Thursday|1964Q3|N|N|N|2438578|2438759|2438220|2438495|N|N|N|N|N| +2438587|AAAAAAAALLFDFCAA|1964-07-10|774|3367|259|1964|5|7|10|3|1964|259|3367|Friday|1964Q3|N|Y|N|2438578|2438759|2438221|2438496|N|N|N|N|N| +2438588|AAAAAAAAMLFDFCAA|1964-07-11|774|3367|259|1964|6|7|11|3|1964|259|3367|Saturday|1964Q3|N|Y|N|2438578|2438759|2438222|2438497|N|N|N|N|N| +2438589|AAAAAAAANLFDFCAA|1964-07-12|774|3367|259|1964|0|7|12|3|1964|259|3367|Sunday|1964Q3|N|N|N|2438578|2438759|2438223|2438498|N|N|N|N|N| +2438590|AAAAAAAAOLFDFCAA|1964-07-13|774|3367|259|1964|1|7|13|3|1964|259|3367|Monday|1964Q3|N|N|N|2438578|2438759|2438224|2438499|N|N|N|N|N| +2438591|AAAAAAAAPLFDFCAA|1964-07-14|774|3368|259|1964|2|7|14|3|1964|259|3368|Tuesday|1964Q3|N|N|N|2438578|2438759|2438225|2438500|N|N|N|N|N| +2438592|AAAAAAAAAMFDFCAA|1964-07-15|774|3368|259|1964|3|7|15|3|1964|259|3368|Wednesday|1964Q3|N|N|N|2438578|2438759|2438226|2438501|N|N|N|N|N| +2438593|AAAAAAAABMFDFCAA|1964-07-16|774|3368|259|1964|4|7|16|3|1964|259|3368|Thursday|1964Q3|N|N|N|2438578|2438759|2438227|2438502|N|N|N|N|N| +2438594|AAAAAAAACMFDFCAA|1964-07-17|774|3368|259|1964|5|7|17|3|1964|259|3368|Friday|1964Q3|N|Y|N|2438578|2438759|2438228|2438503|N|N|N|N|N| +2438595|AAAAAAAADMFDFCAA|1964-07-18|774|3368|259|1964|6|7|18|3|1964|259|3368|Saturday|1964Q3|N|Y|N|2438578|2438759|2438229|2438504|N|N|N|N|N| +2438596|AAAAAAAAEMFDFCAA|1964-07-19|774|3368|259|1964|0|7|19|3|1964|259|3368|Sunday|1964Q3|N|N|N|2438578|2438759|2438230|2438505|N|N|N|N|N| +2438597|AAAAAAAAFMFDFCAA|1964-07-20|774|3368|259|1964|1|7|20|3|1964|259|3368|Monday|1964Q3|N|N|N|2438578|2438759|2438231|2438506|N|N|N|N|N| +2438598|AAAAAAAAGMFDFCAA|1964-07-21|774|3369|259|1964|2|7|21|3|1964|259|3369|Tuesday|1964Q3|N|N|N|2438578|2438759|2438232|2438507|N|N|N|N|N| +2438599|AAAAAAAAHMFDFCAA|1964-07-22|774|3369|259|1964|3|7|22|3|1964|259|3369|Wednesday|1964Q3|N|N|N|2438578|2438759|2438233|2438508|N|N|N|N|N| +2438600|AAAAAAAAIMFDFCAA|1964-07-23|774|3369|259|1964|4|7|23|3|1964|259|3369|Thursday|1964Q3|N|N|N|2438578|2438759|2438234|2438509|N|N|N|N|N| +2438601|AAAAAAAAJMFDFCAA|1964-07-24|774|3369|259|1964|5|7|24|3|1964|259|3369|Friday|1964Q3|N|Y|N|2438578|2438759|2438235|2438510|N|N|N|N|N| +2438602|AAAAAAAAKMFDFCAA|1964-07-25|774|3369|259|1964|6|7|25|3|1964|259|3369|Saturday|1964Q3|N|Y|N|2438578|2438759|2438236|2438511|N|N|N|N|N| +2438603|AAAAAAAALMFDFCAA|1964-07-26|774|3369|259|1964|0|7|26|3|1964|259|3369|Sunday|1964Q3|N|N|N|2438578|2438759|2438237|2438512|N|N|N|N|N| +2438604|AAAAAAAAMMFDFCAA|1964-07-27|774|3369|259|1964|1|7|27|3|1964|259|3369|Monday|1964Q3|N|N|N|2438578|2438759|2438238|2438513|N|N|N|N|N| +2438605|AAAAAAAANMFDFCAA|1964-07-28|774|3370|259|1964|2|7|28|3|1964|259|3370|Tuesday|1964Q3|N|N|N|2438578|2438759|2438239|2438514|N|N|N|N|N| +2438606|AAAAAAAAOMFDFCAA|1964-07-29|774|3370|259|1964|3|7|29|3|1964|259|3370|Wednesday|1964Q3|N|N|N|2438578|2438759|2438240|2438515|N|N|N|N|N| +2438607|AAAAAAAAPMFDFCAA|1964-07-30|774|3370|259|1964|4|7|30|3|1964|259|3370|Thursday|1964Q3|N|N|N|2438578|2438759|2438241|2438516|N|N|N|N|N| +2438608|AAAAAAAAANFDFCAA|1964-07-31|774|3370|259|1964|5|7|31|3|1964|259|3370|Friday|1964Q3|N|Y|N|2438578|2438759|2438242|2438517|N|N|N|N|N| +2438609|AAAAAAAABNFDFCAA|1964-08-01|775|3370|259|1964|6|8|1|3|1964|259|3370|Saturday|1964Q3|N|Y|N|2438609|2438821|2438243|2438518|N|N|N|N|N| +2438610|AAAAAAAACNFDFCAA|1964-08-02|775|3370|259|1964|0|8|2|3|1964|259|3370|Sunday|1964Q3|N|N|N|2438609|2438821|2438244|2438519|N|N|N|N|N| +2438611|AAAAAAAADNFDFCAA|1964-08-03|775|3370|259|1964|1|8|3|3|1964|259|3370|Monday|1964Q3|N|N|N|2438609|2438821|2438245|2438520|N|N|N|N|N| +2438612|AAAAAAAAENFDFCAA|1964-08-04|775|3371|259|1964|2|8|4|3|1964|259|3371|Tuesday|1964Q3|N|N|N|2438609|2438821|2438246|2438521|N|N|N|N|N| +2438613|AAAAAAAAFNFDFCAA|1964-08-05|775|3371|259|1964|3|8|5|3|1964|259|3371|Wednesday|1964Q3|N|N|N|2438609|2438821|2438247|2438522|N|N|N|N|N| +2438614|AAAAAAAAGNFDFCAA|1964-08-06|775|3371|259|1964|4|8|6|3|1964|259|3371|Thursday|1964Q3|N|N|N|2438609|2438821|2438248|2438523|N|N|N|N|N| +2438615|AAAAAAAAHNFDFCAA|1964-08-07|775|3371|259|1964|5|8|7|3|1964|259|3371|Friday|1964Q3|N|Y|N|2438609|2438821|2438249|2438524|N|N|N|N|N| +2438616|AAAAAAAAINFDFCAA|1964-08-08|775|3371|259|1964|6|8|8|3|1964|259|3371|Saturday|1964Q3|N|Y|N|2438609|2438821|2438250|2438525|N|N|N|N|N| +2438617|AAAAAAAAJNFDFCAA|1964-08-09|775|3371|259|1964|0|8|9|3|1964|259|3371|Sunday|1964Q3|N|N|N|2438609|2438821|2438251|2438526|N|N|N|N|N| +2438618|AAAAAAAAKNFDFCAA|1964-08-10|775|3371|259|1964|1|8|10|3|1964|259|3371|Monday|1964Q3|N|N|N|2438609|2438821|2438252|2438527|N|N|N|N|N| +2438619|AAAAAAAALNFDFCAA|1964-08-11|775|3372|259|1964|2|8|11|3|1964|259|3372|Tuesday|1964Q3|N|N|N|2438609|2438821|2438253|2438528|N|N|N|N|N| +2438620|AAAAAAAAMNFDFCAA|1964-08-12|775|3372|259|1964|3|8|12|3|1964|259|3372|Wednesday|1964Q3|N|N|N|2438609|2438821|2438254|2438529|N|N|N|N|N| +2438621|AAAAAAAANNFDFCAA|1964-08-13|775|3372|259|1964|4|8|13|3|1964|259|3372|Thursday|1964Q3|N|N|N|2438609|2438821|2438255|2438530|N|N|N|N|N| +2438622|AAAAAAAAONFDFCAA|1964-08-14|775|3372|259|1964|5|8|14|3|1964|259|3372|Friday|1964Q3|N|Y|N|2438609|2438821|2438256|2438531|N|N|N|N|N| +2438623|AAAAAAAAPNFDFCAA|1964-08-15|775|3372|259|1964|6|8|15|3|1964|259|3372|Saturday|1964Q3|N|Y|N|2438609|2438821|2438257|2438532|N|N|N|N|N| +2438624|AAAAAAAAAOFDFCAA|1964-08-16|775|3372|259|1964|0|8|16|3|1964|259|3372|Sunday|1964Q3|N|N|N|2438609|2438821|2438258|2438533|N|N|N|N|N| +2438625|AAAAAAAABOFDFCAA|1964-08-17|775|3372|259|1964|1|8|17|3|1964|259|3372|Monday|1964Q3|N|N|N|2438609|2438821|2438259|2438534|N|N|N|N|N| +2438626|AAAAAAAACOFDFCAA|1964-08-18|775|3373|259|1964|2|8|18|3|1964|259|3373|Tuesday|1964Q3|N|N|N|2438609|2438821|2438260|2438535|N|N|N|N|N| +2438627|AAAAAAAADOFDFCAA|1964-08-19|775|3373|259|1964|3|8|19|3|1964|259|3373|Wednesday|1964Q3|N|N|N|2438609|2438821|2438261|2438536|N|N|N|N|N| +2438628|AAAAAAAAEOFDFCAA|1964-08-20|775|3373|259|1964|4|8|20|3|1964|259|3373|Thursday|1964Q3|N|N|N|2438609|2438821|2438262|2438537|N|N|N|N|N| +2438629|AAAAAAAAFOFDFCAA|1964-08-21|775|3373|259|1964|5|8|21|3|1964|259|3373|Friday|1964Q3|N|Y|N|2438609|2438821|2438263|2438538|N|N|N|N|N| +2438630|AAAAAAAAGOFDFCAA|1964-08-22|775|3373|259|1964|6|8|22|3|1964|259|3373|Saturday|1964Q3|N|Y|N|2438609|2438821|2438264|2438539|N|N|N|N|N| +2438631|AAAAAAAAHOFDFCAA|1964-08-23|775|3373|259|1964|0|8|23|3|1964|259|3373|Sunday|1964Q3|N|N|N|2438609|2438821|2438265|2438540|N|N|N|N|N| +2438632|AAAAAAAAIOFDFCAA|1964-08-24|775|3373|259|1964|1|8|24|3|1964|259|3373|Monday|1964Q3|N|N|N|2438609|2438821|2438266|2438541|N|N|N|N|N| +2438633|AAAAAAAAJOFDFCAA|1964-08-25|775|3374|259|1964|2|8|25|3|1964|259|3374|Tuesday|1964Q3|N|N|N|2438609|2438821|2438267|2438542|N|N|N|N|N| +2438634|AAAAAAAAKOFDFCAA|1964-08-26|775|3374|259|1964|3|8|26|3|1964|259|3374|Wednesday|1964Q3|N|N|N|2438609|2438821|2438268|2438543|N|N|N|N|N| +2438635|AAAAAAAALOFDFCAA|1964-08-27|775|3374|259|1964|4|8|27|3|1964|259|3374|Thursday|1964Q3|N|N|N|2438609|2438821|2438269|2438544|N|N|N|N|N| +2438636|AAAAAAAAMOFDFCAA|1964-08-28|775|3374|259|1964|5|8|28|3|1964|259|3374|Friday|1964Q3|N|Y|N|2438609|2438821|2438270|2438545|N|N|N|N|N| +2438637|AAAAAAAANOFDFCAA|1964-08-29|775|3374|259|1964|6|8|29|3|1964|259|3374|Saturday|1964Q3|N|Y|N|2438609|2438821|2438271|2438546|N|N|N|N|N| +2438638|AAAAAAAAOOFDFCAA|1964-08-30|775|3374|259|1964|0|8|30|3|1964|259|3374|Sunday|1964Q3|N|N|N|2438609|2438821|2438272|2438547|N|N|N|N|N| +2438639|AAAAAAAAPOFDFCAA|1964-08-31|775|3374|259|1964|1|8|31|3|1964|259|3374|Monday|1964Q3|N|N|N|2438609|2438821|2438273|2438548|N|N|N|N|N| +2438640|AAAAAAAAAPFDFCAA|1964-09-01|776|3375|260|1964|2|9|1|3|1964|260|3375|Tuesday|1964Q3|N|N|N|2438640|2438883|2438274|2438549|N|N|N|N|N| +2438641|AAAAAAAABPFDFCAA|1964-09-02|776|3375|260|1964|3|9|2|3|1964|260|3375|Wednesday|1964Q3|N|N|N|2438640|2438883|2438275|2438550|N|N|N|N|N| +2438642|AAAAAAAACPFDFCAA|1964-09-03|776|3375|260|1964|4|9|3|3|1964|260|3375|Thursday|1964Q3|N|N|N|2438640|2438883|2438276|2438551|N|N|N|N|N| +2438643|AAAAAAAADPFDFCAA|1964-09-04|776|3375|260|1964|5|9|4|3|1964|260|3375|Friday|1964Q3|N|Y|N|2438640|2438883|2438277|2438552|N|N|N|N|N| +2438644|AAAAAAAAEPFDFCAA|1964-09-05|776|3375|260|1964|6|9|5|3|1964|260|3375|Saturday|1964Q3|N|Y|N|2438640|2438883|2438278|2438553|N|N|N|N|N| +2438645|AAAAAAAAFPFDFCAA|1964-09-06|776|3375|260|1964|0|9|6|3|1964|260|3375|Sunday|1964Q3|N|N|N|2438640|2438883|2438279|2438554|N|N|N|N|N| +2438646|AAAAAAAAGPFDFCAA|1964-09-07|776|3375|260|1964|1|9|7|3|1964|260|3375|Monday|1964Q3|N|N|N|2438640|2438883|2438280|2438555|N|N|N|N|N| +2438647|AAAAAAAAHPFDFCAA|1964-09-08|776|3376|260|1964|2|9|8|3|1964|260|3376|Tuesday|1964Q3|N|N|N|2438640|2438883|2438281|2438556|N|N|N|N|N| +2438648|AAAAAAAAIPFDFCAA|1964-09-09|776|3376|260|1964|3|9|9|3|1964|260|3376|Wednesday|1964Q3|N|N|N|2438640|2438883|2438282|2438557|N|N|N|N|N| +2438649|AAAAAAAAJPFDFCAA|1964-09-10|776|3376|260|1964|4|9|10|3|1964|260|3376|Thursday|1964Q3|N|N|N|2438640|2438883|2438283|2438558|N|N|N|N|N| +2438650|AAAAAAAAKPFDFCAA|1964-09-11|776|3376|260|1964|5|9|11|3|1964|260|3376|Friday|1964Q3|N|Y|N|2438640|2438883|2438284|2438559|N|N|N|N|N| +2438651|AAAAAAAALPFDFCAA|1964-09-12|776|3376|260|1964|6|9|12|3|1964|260|3376|Saturday|1964Q3|N|Y|N|2438640|2438883|2438285|2438560|N|N|N|N|N| +2438652|AAAAAAAAMPFDFCAA|1964-09-13|776|3376|260|1964|0|9|13|3|1964|260|3376|Sunday|1964Q3|N|N|N|2438640|2438883|2438286|2438561|N|N|N|N|N| +2438653|AAAAAAAANPFDFCAA|1964-09-14|776|3376|260|1964|1|9|14|3|1964|260|3376|Monday|1964Q3|N|N|N|2438640|2438883|2438287|2438562|N|N|N|N|N| +2438654|AAAAAAAAOPFDFCAA|1964-09-15|776|3377|260|1964|2|9|15|3|1964|260|3377|Tuesday|1964Q3|N|N|N|2438640|2438883|2438288|2438563|N|N|N|N|N| +2438655|AAAAAAAAPPFDFCAA|1964-09-16|776|3377|260|1964|3|9|16|3|1964|260|3377|Wednesday|1964Q3|N|N|N|2438640|2438883|2438289|2438564|N|N|N|N|N| +2438656|AAAAAAAAAAGDFCAA|1964-09-17|776|3377|260|1964|4|9|17|3|1964|260|3377|Thursday|1964Q3|N|N|N|2438640|2438883|2438290|2438565|N|N|N|N|N| +2438657|AAAAAAAABAGDFCAA|1964-09-18|776|3377|260|1964|5|9|18|3|1964|260|3377|Friday|1964Q3|N|Y|N|2438640|2438883|2438291|2438566|N|N|N|N|N| +2438658|AAAAAAAACAGDFCAA|1964-09-19|776|3377|260|1964|6|9|19|3|1964|260|3377|Saturday|1964Q3|N|Y|N|2438640|2438883|2438292|2438567|N|N|N|N|N| +2438659|AAAAAAAADAGDFCAA|1964-09-20|776|3377|260|1964|0|9|20|3|1964|260|3377|Sunday|1964Q3|N|N|N|2438640|2438883|2438293|2438568|N|N|N|N|N| +2438660|AAAAAAAAEAGDFCAA|1964-09-21|776|3377|260|1964|1|9|21|3|1964|260|3377|Monday|1964Q3|N|N|N|2438640|2438883|2438294|2438569|N|N|N|N|N| +2438661|AAAAAAAAFAGDFCAA|1964-09-22|776|3378|260|1964|2|9|22|3|1964|260|3378|Tuesday|1964Q3|N|N|N|2438640|2438883|2438295|2438570|N|N|N|N|N| +2438662|AAAAAAAAGAGDFCAA|1964-09-23|776|3378|260|1964|3|9|23|3|1964|260|3378|Wednesday|1964Q3|N|N|N|2438640|2438883|2438296|2438571|N|N|N|N|N| +2438663|AAAAAAAAHAGDFCAA|1964-09-24|776|3378|260|1964|4|9|24|3|1964|260|3378|Thursday|1964Q3|N|N|N|2438640|2438883|2438297|2438572|N|N|N|N|N| +2438664|AAAAAAAAIAGDFCAA|1964-09-25|776|3378|260|1964|5|9|25|3|1964|260|3378|Friday|1964Q3|N|Y|N|2438640|2438883|2438298|2438573|N|N|N|N|N| +2438665|AAAAAAAAJAGDFCAA|1964-09-26|776|3378|260|1964|6|9|26|3|1964|260|3378|Saturday|1964Q3|N|Y|N|2438640|2438883|2438299|2438574|N|N|N|N|N| +2438666|AAAAAAAAKAGDFCAA|1964-09-27|776|3378|260|1964|0|9|27|3|1964|260|3378|Sunday|1964Q3|N|N|N|2438640|2438883|2438300|2438575|N|N|N|N|N| +2438667|AAAAAAAALAGDFCAA|1964-09-28|776|3378|260|1964|1|9|28|3|1964|260|3378|Monday|1964Q3|N|N|N|2438640|2438883|2438301|2438576|N|N|N|N|N| +2438668|AAAAAAAAMAGDFCAA|1964-09-29|776|3379|260|1964|2|9|29|3|1964|260|3379|Tuesday|1964Q3|N|N|N|2438640|2438883|2438302|2438577|N|N|N|N|N| +2438669|AAAAAAAANAGDFCAA|1964-09-30|776|3379|260|1964|3|9|30|3|1964|260|3379|Wednesday|1964Q3|N|N|N|2438640|2438883|2438303|2438578|N|N|N|N|N| +2438670|AAAAAAAAOAGDFCAA|1964-10-01|777|3379|260|1964|4|10|1|4|1964|260|3379|Thursday|1964Q4|N|N|N|2438670|2438943|2438304|2438578|N|N|N|N|N| +2438671|AAAAAAAAPAGDFCAA|1964-10-02|777|3379|260|1964|5|10|2|4|1964|260|3379|Friday|1964Q4|N|Y|N|2438670|2438943|2438305|2438579|N|N|N|N|N| +2438672|AAAAAAAAABGDFCAA|1964-10-03|777|3379|260|1964|6|10|3|4|1964|260|3379|Saturday|1964Q4|N|Y|N|2438670|2438943|2438306|2438580|N|N|N|N|N| +2438673|AAAAAAAABBGDFCAA|1964-10-04|777|3379|260|1964|0|10|4|4|1964|260|3379|Sunday|1964Q4|N|N|N|2438670|2438943|2438307|2438581|N|N|N|N|N| +2438674|AAAAAAAACBGDFCAA|1964-10-05|777|3379|260|1964|1|10|5|4|1964|260|3379|Monday|1964Q4|N|N|N|2438670|2438943|2438308|2438582|N|N|N|N|N| +2438675|AAAAAAAADBGDFCAA|1964-10-06|777|3380|260|1964|2|10|6|4|1964|260|3380|Tuesday|1964Q4|N|N|N|2438670|2438943|2438309|2438583|N|N|N|N|N| +2438676|AAAAAAAAEBGDFCAA|1964-10-07|777|3380|260|1964|3|10|7|4|1964|260|3380|Wednesday|1964Q4|N|N|N|2438670|2438943|2438310|2438584|N|N|N|N|N| +2438677|AAAAAAAAFBGDFCAA|1964-10-08|777|3380|260|1964|4|10|8|4|1964|260|3380|Thursday|1964Q4|N|N|N|2438670|2438943|2438311|2438585|N|N|N|N|N| +2438678|AAAAAAAAGBGDFCAA|1964-10-09|777|3380|260|1964|5|10|9|4|1964|260|3380|Friday|1964Q4|N|Y|N|2438670|2438943|2438312|2438586|N|N|N|N|N| +2438679|AAAAAAAAHBGDFCAA|1964-10-10|777|3380|260|1964|6|10|10|4|1964|260|3380|Saturday|1964Q4|N|Y|N|2438670|2438943|2438313|2438587|N|N|N|N|N| +2438680|AAAAAAAAIBGDFCAA|1964-10-11|777|3380|260|1964|0|10|11|4|1964|260|3380|Sunday|1964Q4|N|N|N|2438670|2438943|2438314|2438588|N|N|N|N|N| +2438681|AAAAAAAAJBGDFCAA|1964-10-12|777|3380|260|1964|1|10|12|4|1964|260|3380|Monday|1964Q4|N|N|N|2438670|2438943|2438315|2438589|N|N|N|N|N| +2438682|AAAAAAAAKBGDFCAA|1964-10-13|777|3381|260|1964|2|10|13|4|1964|260|3381|Tuesday|1964Q4|N|N|N|2438670|2438943|2438316|2438590|N|N|N|N|N| +2438683|AAAAAAAALBGDFCAA|1964-10-14|777|3381|260|1964|3|10|14|4|1964|260|3381|Wednesday|1964Q4|N|N|N|2438670|2438943|2438317|2438591|N|N|N|N|N| +2438684|AAAAAAAAMBGDFCAA|1964-10-15|777|3381|260|1964|4|10|15|4|1964|260|3381|Thursday|1964Q4|N|N|N|2438670|2438943|2438318|2438592|N|N|N|N|N| +2438685|AAAAAAAANBGDFCAA|1964-10-16|777|3381|260|1964|5|10|16|4|1964|260|3381|Friday|1964Q4|N|Y|N|2438670|2438943|2438319|2438593|N|N|N|N|N| +2438686|AAAAAAAAOBGDFCAA|1964-10-17|777|3381|260|1964|6|10|17|4|1964|260|3381|Saturday|1964Q4|N|Y|N|2438670|2438943|2438320|2438594|N|N|N|N|N| +2438687|AAAAAAAAPBGDFCAA|1964-10-18|777|3381|260|1964|0|10|18|4|1964|260|3381|Sunday|1964Q4|N|N|N|2438670|2438943|2438321|2438595|N|N|N|N|N| +2438688|AAAAAAAAACGDFCAA|1964-10-19|777|3381|260|1964|1|10|19|4|1964|260|3381|Monday|1964Q4|N|N|N|2438670|2438943|2438322|2438596|N|N|N|N|N| +2438689|AAAAAAAABCGDFCAA|1964-10-20|777|3382|260|1964|2|10|20|4|1964|260|3382|Tuesday|1964Q4|N|N|N|2438670|2438943|2438323|2438597|N|N|N|N|N| +2438690|AAAAAAAACCGDFCAA|1964-10-21|777|3382|260|1964|3|10|21|4|1964|260|3382|Wednesday|1964Q4|N|N|N|2438670|2438943|2438324|2438598|N|N|N|N|N| +2438691|AAAAAAAADCGDFCAA|1964-10-22|777|3382|260|1964|4|10|22|4|1964|260|3382|Thursday|1964Q4|N|N|N|2438670|2438943|2438325|2438599|N|N|N|N|N| +2438692|AAAAAAAAECGDFCAA|1964-10-23|777|3382|260|1964|5|10|23|4|1964|260|3382|Friday|1964Q4|N|Y|N|2438670|2438943|2438326|2438600|N|N|N|N|N| +2438693|AAAAAAAAFCGDFCAA|1964-10-24|777|3382|260|1964|6|10|24|4|1964|260|3382|Saturday|1964Q4|N|Y|N|2438670|2438943|2438327|2438601|N|N|N|N|N| +2438694|AAAAAAAAGCGDFCAA|1964-10-25|777|3382|260|1964|0|10|25|4|1964|260|3382|Sunday|1964Q4|N|N|N|2438670|2438943|2438328|2438602|N|N|N|N|N| +2438695|AAAAAAAAHCGDFCAA|1964-10-26|777|3382|260|1964|1|10|26|4|1964|260|3382|Monday|1964Q4|N|N|N|2438670|2438943|2438329|2438603|N|N|N|N|N| +2438696|AAAAAAAAICGDFCAA|1964-10-27|777|3383|260|1964|2|10|27|4|1964|260|3383|Tuesday|1964Q4|N|N|N|2438670|2438943|2438330|2438604|N|N|N|N|N| +2438697|AAAAAAAAJCGDFCAA|1964-10-28|777|3383|260|1964|3|10|28|4|1964|260|3383|Wednesday|1964Q4|N|N|N|2438670|2438943|2438331|2438605|N|N|N|N|N| +2438698|AAAAAAAAKCGDFCAA|1964-10-29|777|3383|260|1964|4|10|29|4|1964|260|3383|Thursday|1964Q4|N|N|N|2438670|2438943|2438332|2438606|N|N|N|N|N| +2438699|AAAAAAAALCGDFCAA|1964-10-30|777|3383|260|1964|5|10|30|4|1964|260|3383|Friday|1964Q4|N|Y|N|2438670|2438943|2438333|2438607|N|N|N|N|N| +2438700|AAAAAAAAMCGDFCAA|1964-10-31|777|3383|260|1964|6|10|31|4|1964|260|3383|Saturday|1964Q4|N|Y|N|2438670|2438943|2438334|2438608|N|N|N|N|N| +2438701|AAAAAAAANCGDFCAA|1964-11-01|778|3383|260|1964|0|11|1|4|1964|260|3383|Sunday|1964Q4|N|N|N|2438701|2439005|2438335|2438609|N|N|N|N|N| +2438702|AAAAAAAAOCGDFCAA|1964-11-02|778|3383|260|1964|1|11|2|4|1964|260|3383|Monday|1964Q4|N|N|N|2438701|2439005|2438336|2438610|N|N|N|N|N| +2438703|AAAAAAAAPCGDFCAA|1964-11-03|778|3384|260|1964|2|11|3|4|1964|260|3384|Tuesday|1964Q4|N|N|N|2438701|2439005|2438337|2438611|N|N|N|N|N| +2438704|AAAAAAAAADGDFCAA|1964-11-04|778|3384|260|1964|3|11|4|4|1964|260|3384|Wednesday|1964Q4|N|N|N|2438701|2439005|2438338|2438612|N|N|N|N|N| +2438705|AAAAAAAABDGDFCAA|1964-11-05|778|3384|260|1964|4|11|5|4|1964|260|3384|Thursday|1964Q4|N|N|N|2438701|2439005|2438339|2438613|N|N|N|N|N| +2438706|AAAAAAAACDGDFCAA|1964-11-06|778|3384|260|1964|5|11|6|4|1964|260|3384|Friday|1964Q4|N|Y|N|2438701|2439005|2438340|2438614|N|N|N|N|N| +2438707|AAAAAAAADDGDFCAA|1964-11-07|778|3384|260|1964|6|11|7|4|1964|260|3384|Saturday|1964Q4|N|Y|N|2438701|2439005|2438341|2438615|N|N|N|N|N| +2438708|AAAAAAAAEDGDFCAA|1964-11-08|778|3384|260|1964|0|11|8|4|1964|260|3384|Sunday|1964Q4|N|N|N|2438701|2439005|2438342|2438616|N|N|N|N|N| +2438709|AAAAAAAAFDGDFCAA|1964-11-09|778|3384|260|1964|1|11|9|4|1964|260|3384|Monday|1964Q4|N|N|N|2438701|2439005|2438343|2438617|N|N|N|N|N| +2438710|AAAAAAAAGDGDFCAA|1964-11-10|778|3385|260|1964|2|11|10|4|1964|260|3385|Tuesday|1964Q4|N|N|N|2438701|2439005|2438344|2438618|N|N|N|N|N| +2438711|AAAAAAAAHDGDFCAA|1964-11-11|778|3385|260|1964|3|11|11|4|1964|260|3385|Wednesday|1964Q4|N|N|N|2438701|2439005|2438345|2438619|N|N|N|N|N| +2438712|AAAAAAAAIDGDFCAA|1964-11-12|778|3385|260|1964|4|11|12|4|1964|260|3385|Thursday|1964Q4|N|N|N|2438701|2439005|2438346|2438620|N|N|N|N|N| +2438713|AAAAAAAAJDGDFCAA|1964-11-13|778|3385|260|1964|5|11|13|4|1964|260|3385|Friday|1964Q4|N|Y|N|2438701|2439005|2438347|2438621|N|N|N|N|N| +2438714|AAAAAAAAKDGDFCAA|1964-11-14|778|3385|260|1964|6|11|14|4|1964|260|3385|Saturday|1964Q4|N|Y|N|2438701|2439005|2438348|2438622|N|N|N|N|N| +2438715|AAAAAAAALDGDFCAA|1964-11-15|778|3385|260|1964|0|11|15|4|1964|260|3385|Sunday|1964Q4|N|N|N|2438701|2439005|2438349|2438623|N|N|N|N|N| +2438716|AAAAAAAAMDGDFCAA|1964-11-16|778|3385|260|1964|1|11|16|4|1964|260|3385|Monday|1964Q4|N|N|N|2438701|2439005|2438350|2438624|N|N|N|N|N| +2438717|AAAAAAAANDGDFCAA|1964-11-17|778|3386|260|1964|2|11|17|4|1964|260|3386|Tuesday|1964Q4|N|N|N|2438701|2439005|2438351|2438625|N|N|N|N|N| +2438718|AAAAAAAAODGDFCAA|1964-11-18|778|3386|260|1964|3|11|18|4|1964|260|3386|Wednesday|1964Q4|N|N|N|2438701|2439005|2438352|2438626|N|N|N|N|N| +2438719|AAAAAAAAPDGDFCAA|1964-11-19|778|3386|260|1964|4|11|19|4|1964|260|3386|Thursday|1964Q4|N|N|N|2438701|2439005|2438353|2438627|N|N|N|N|N| +2438720|AAAAAAAAAEGDFCAA|1964-11-20|778|3386|260|1964|5|11|20|4|1964|260|3386|Friday|1964Q4|N|Y|N|2438701|2439005|2438354|2438628|N|N|N|N|N| +2438721|AAAAAAAABEGDFCAA|1964-11-21|778|3386|260|1964|6|11|21|4|1964|260|3386|Saturday|1964Q4|N|Y|N|2438701|2439005|2438355|2438629|N|N|N|N|N| +2438722|AAAAAAAACEGDFCAA|1964-11-22|778|3386|260|1964|0|11|22|4|1964|260|3386|Sunday|1964Q4|N|N|N|2438701|2439005|2438356|2438630|N|N|N|N|N| +2438723|AAAAAAAADEGDFCAA|1964-11-23|778|3386|260|1964|1|11|23|4|1964|260|3386|Monday|1964Q4|N|N|N|2438701|2439005|2438357|2438631|N|N|N|N|N| +2438724|AAAAAAAAEEGDFCAA|1964-11-24|778|3387|260|1964|2|11|24|4|1964|260|3387|Tuesday|1964Q4|N|N|N|2438701|2439005|2438358|2438632|N|N|N|N|N| +2438725|AAAAAAAAFEGDFCAA|1964-11-25|778|3387|260|1964|3|11|25|4|1964|260|3387|Wednesday|1964Q4|N|N|N|2438701|2439005|2438359|2438633|N|N|N|N|N| +2438726|AAAAAAAAGEGDFCAA|1964-11-26|778|3387|260|1964|4|11|26|4|1964|260|3387|Thursday|1964Q4|N|N|N|2438701|2439005|2438360|2438634|N|N|N|N|N| +2438727|AAAAAAAAHEGDFCAA|1964-11-27|778|3387|260|1964|5|11|27|4|1964|260|3387|Friday|1964Q4|N|Y|N|2438701|2439005|2438361|2438635|N|N|N|N|N| +2438728|AAAAAAAAIEGDFCAA|1964-11-28|778|3387|260|1964|6|11|28|4|1964|260|3387|Saturday|1964Q4|N|Y|N|2438701|2439005|2438362|2438636|N|N|N|N|N| +2438729|AAAAAAAAJEGDFCAA|1964-11-29|778|3387|260|1964|0|11|29|4|1964|260|3387|Sunday|1964Q4|N|N|N|2438701|2439005|2438363|2438637|N|N|N|N|N| +2438730|AAAAAAAAKEGDFCAA|1964-11-30|778|3387|260|1964|1|11|30|4|1964|260|3387|Monday|1964Q4|N|N|N|2438701|2439005|2438364|2438638|N|N|N|N|N| +2438731|AAAAAAAALEGDFCAA|1964-12-01|779|3388|261|1964|2|12|1|4|1964|261|3388|Tuesday|1964Q4|N|N|N|2438731|2439065|2438365|2438639|N|N|N|N|N| +2438732|AAAAAAAAMEGDFCAA|1964-12-02|779|3388|261|1964|3|12|2|4|1964|261|3388|Wednesday|1964Q4|N|N|N|2438731|2439065|2438366|2438640|N|N|N|N|N| +2438733|AAAAAAAANEGDFCAA|1964-12-03|779|3388|261|1964|4|12|3|4|1964|261|3388|Thursday|1964Q4|N|N|N|2438731|2439065|2438367|2438641|N|N|N|N|N| +2438734|AAAAAAAAOEGDFCAA|1964-12-04|779|3388|261|1964|5|12|4|4|1964|261|3388|Friday|1964Q4|N|Y|N|2438731|2439065|2438368|2438642|N|N|N|N|N| +2438735|AAAAAAAAPEGDFCAA|1964-12-05|779|3388|261|1964|6|12|5|4|1964|261|3388|Saturday|1964Q4|N|Y|N|2438731|2439065|2438369|2438643|N|N|N|N|N| +2438736|AAAAAAAAAFGDFCAA|1964-12-06|779|3388|261|1964|0|12|6|4|1964|261|3388|Sunday|1964Q4|N|N|N|2438731|2439065|2438370|2438644|N|N|N|N|N| +2438737|AAAAAAAABFGDFCAA|1964-12-07|779|3388|261|1964|1|12|7|4|1964|261|3388|Monday|1964Q4|N|N|N|2438731|2439065|2438371|2438645|N|N|N|N|N| +2438738|AAAAAAAACFGDFCAA|1964-12-08|779|3389|261|1964|2|12|8|4|1964|261|3389|Tuesday|1964Q4|N|N|N|2438731|2439065|2438372|2438646|N|N|N|N|N| +2438739|AAAAAAAADFGDFCAA|1964-12-09|779|3389|261|1964|3|12|9|4|1964|261|3389|Wednesday|1964Q4|N|N|N|2438731|2439065|2438373|2438647|N|N|N|N|N| +2438740|AAAAAAAAEFGDFCAA|1964-12-10|779|3389|261|1964|4|12|10|4|1964|261|3389|Thursday|1964Q4|N|N|N|2438731|2439065|2438374|2438648|N|N|N|N|N| +2438741|AAAAAAAAFFGDFCAA|1964-12-11|779|3389|261|1964|5|12|11|4|1964|261|3389|Friday|1964Q4|N|Y|N|2438731|2439065|2438375|2438649|N|N|N|N|N| +2438742|AAAAAAAAGFGDFCAA|1964-12-12|779|3389|261|1964|6|12|12|4|1964|261|3389|Saturday|1964Q4|N|Y|N|2438731|2439065|2438376|2438650|N|N|N|N|N| +2438743|AAAAAAAAHFGDFCAA|1964-12-13|779|3389|261|1964|0|12|13|4|1964|261|3389|Sunday|1964Q4|N|N|N|2438731|2439065|2438377|2438651|N|N|N|N|N| +2438744|AAAAAAAAIFGDFCAA|1964-12-14|779|3389|261|1964|1|12|14|4|1964|261|3389|Monday|1964Q4|N|N|N|2438731|2439065|2438378|2438652|N|N|N|N|N| +2438745|AAAAAAAAJFGDFCAA|1964-12-15|779|3390|261|1964|2|12|15|4|1964|261|3390|Tuesday|1964Q4|N|N|N|2438731|2439065|2438379|2438653|N|N|N|N|N| +2438746|AAAAAAAAKFGDFCAA|1964-12-16|779|3390|261|1964|3|12|16|4|1964|261|3390|Wednesday|1964Q4|N|N|N|2438731|2439065|2438380|2438654|N|N|N|N|N| +2438747|AAAAAAAALFGDFCAA|1964-12-17|779|3390|261|1964|4|12|17|4|1964|261|3390|Thursday|1964Q4|N|N|N|2438731|2439065|2438381|2438655|N|N|N|N|N| +2438748|AAAAAAAAMFGDFCAA|1964-12-18|779|3390|261|1964|5|12|18|4|1964|261|3390|Friday|1964Q4|N|Y|N|2438731|2439065|2438382|2438656|N|N|N|N|N| +2438749|AAAAAAAANFGDFCAA|1964-12-19|779|3390|261|1964|6|12|19|4|1964|261|3390|Saturday|1964Q4|N|Y|N|2438731|2439065|2438383|2438657|N|N|N|N|N| +2438750|AAAAAAAAOFGDFCAA|1964-12-20|779|3390|261|1964|0|12|20|4|1964|261|3390|Sunday|1964Q4|N|N|N|2438731|2439065|2438384|2438658|N|N|N|N|N| +2438751|AAAAAAAAPFGDFCAA|1964-12-21|779|3390|261|1964|1|12|21|4|1964|261|3390|Monday|1964Q4|N|N|N|2438731|2439065|2438385|2438659|N|N|N|N|N| +2438752|AAAAAAAAAGGDFCAA|1964-12-22|779|3391|261|1964|2|12|22|4|1964|261|3391|Tuesday|1964Q4|N|N|N|2438731|2439065|2438386|2438660|N|N|N|N|N| +2438753|AAAAAAAABGGDFCAA|1964-12-23|779|3391|261|1964|3|12|23|4|1964|261|3391|Wednesday|1964Q4|N|N|N|2438731|2439065|2438387|2438661|N|N|N|N|N| +2438754|AAAAAAAACGGDFCAA|1964-12-24|779|3391|261|1964|4|12|24|4|1964|261|3391|Thursday|1964Q4|N|N|N|2438731|2439065|2438388|2438662|N|N|N|N|N| +2438755|AAAAAAAADGGDFCAA|1964-12-25|779|3391|261|1964|5|12|25|4|1964|261|3391|Friday|1964Q4|Y|Y|N|2438731|2439065|2438389|2438663|N|N|N|N|N| +2438756|AAAAAAAAEGGDFCAA|1964-12-26|779|3391|261|1964|6|12|26|4|1964|261|3391|Saturday|1964Q4|N|Y|Y|2438731|2439065|2438390|2438664|N|N|N|N|N| +2438757|AAAAAAAAFGGDFCAA|1964-12-27|779|3391|261|1964|0|12|27|4|1964|261|3391|Sunday|1964Q4|N|N|N|2438731|2439065|2438391|2438665|N|N|N|N|N| +2438758|AAAAAAAAGGGDFCAA|1964-12-28|779|3391|261|1964|1|12|28|4|1964|261|3391|Monday|1964Q4|N|N|N|2438731|2439065|2438392|2438666|N|N|N|N|N| +2438759|AAAAAAAAHGGDFCAA|1964-12-29|779|3392|261|1964|2|12|29|4|1964|261|3392|Tuesday|1964Q4|N|N|N|2438731|2439065|2438393|2438667|N|N|N|N|N| +2438760|AAAAAAAAIGGDFCAA|1964-12-30|779|3392|261|1964|3|12|30|4|1964|261|3392|Wednesday|1964Q4|N|N|N|2438731|2439065|2438394|2438668|N|N|N|N|N| +2438761|AAAAAAAAJGGDFCAA|1964-12-31|779|3392|261|1964|4|12|31|4|1964|261|3392|Thursday|1964Q4|Y|N|N|2438731|2439065|2438395|2438669|N|N|N|N|N| +2438762|AAAAAAAAKGGDFCAA|1965-01-01|780|3392|261|1965|5|1|1|1|1965|261|3392|Friday|1965Q1|Y|Y|Y|2438762|2438761|2438396|2438670|N|N|N|N|N| +2438763|AAAAAAAALGGDFCAA|1965-01-02|780|3392|261|1965|6|1|2|1|1965|261|3392|Saturday|1965Q1|N|Y|Y|2438762|2438761|2438397|2438671|N|N|N|N|N| +2438764|AAAAAAAAMGGDFCAA|1965-01-03|780|3392|261|1965|0|1|3|1|1965|261|3392|Sunday|1965Q1|N|N|N|2438762|2438761|2438398|2438672|N|N|N|N|N| +2438765|AAAAAAAANGGDFCAA|1965-01-04|780|3392|261|1965|1|1|4|1|1965|261|3392|Monday|1965Q1|N|N|N|2438762|2438761|2438399|2438673|N|N|N|N|N| +2438766|AAAAAAAAOGGDFCAA|1965-01-05|780|3393|261|1965|2|1|5|1|1965|261|3393|Tuesday|1965Q1|N|N|N|2438762|2438761|2438400|2438674|N|N|N|N|N| +2438767|AAAAAAAAPGGDFCAA|1965-01-06|780|3393|261|1965|3|1|6|1|1965|261|3393|Wednesday|1965Q1|N|N|N|2438762|2438761|2438401|2438675|N|N|N|N|N| +2438768|AAAAAAAAAHGDFCAA|1965-01-07|780|3393|261|1965|4|1|7|1|1965|261|3393|Thursday|1965Q1|N|N|N|2438762|2438761|2438402|2438676|N|N|N|N|N| +2438769|AAAAAAAABHGDFCAA|1965-01-08|780|3393|261|1965|5|1|8|1|1965|261|3393|Friday|1965Q1|N|Y|N|2438762|2438761|2438403|2438677|N|N|N|N|N| +2438770|AAAAAAAACHGDFCAA|1965-01-09|780|3393|261|1965|6|1|9|1|1965|261|3393|Saturday|1965Q1|N|Y|N|2438762|2438761|2438404|2438678|N|N|N|N|N| +2438771|AAAAAAAADHGDFCAA|1965-01-10|780|3393|261|1965|0|1|10|1|1965|261|3393|Sunday|1965Q1|N|N|N|2438762|2438761|2438405|2438679|N|N|N|N|N| +2438772|AAAAAAAAEHGDFCAA|1965-01-11|780|3393|261|1965|1|1|11|1|1965|261|3393|Monday|1965Q1|N|N|N|2438762|2438761|2438406|2438680|N|N|N|N|N| +2438773|AAAAAAAAFHGDFCAA|1965-01-12|780|3394|261|1965|2|1|12|1|1965|261|3394|Tuesday|1965Q1|N|N|N|2438762|2438761|2438407|2438681|N|N|N|N|N| +2438774|AAAAAAAAGHGDFCAA|1965-01-13|780|3394|261|1965|3|1|13|1|1965|261|3394|Wednesday|1965Q1|N|N|N|2438762|2438761|2438408|2438682|N|N|N|N|N| +2438775|AAAAAAAAHHGDFCAA|1965-01-14|780|3394|261|1965|4|1|14|1|1965|261|3394|Thursday|1965Q1|N|N|N|2438762|2438761|2438409|2438683|N|N|N|N|N| +2438776|AAAAAAAAIHGDFCAA|1965-01-15|780|3394|261|1965|5|1|15|1|1965|261|3394|Friday|1965Q1|N|Y|N|2438762|2438761|2438410|2438684|N|N|N|N|N| +2438777|AAAAAAAAJHGDFCAA|1965-01-16|780|3394|261|1965|6|1|16|1|1965|261|3394|Saturday|1965Q1|N|Y|N|2438762|2438761|2438411|2438685|N|N|N|N|N| +2438778|AAAAAAAAKHGDFCAA|1965-01-17|780|3394|261|1965|0|1|17|1|1965|261|3394|Sunday|1965Q1|N|N|N|2438762|2438761|2438412|2438686|N|N|N|N|N| +2438779|AAAAAAAALHGDFCAA|1965-01-18|780|3394|261|1965|1|1|18|1|1965|261|3394|Monday|1965Q1|N|N|N|2438762|2438761|2438413|2438687|N|N|N|N|N| +2438780|AAAAAAAAMHGDFCAA|1965-01-19|780|3395|261|1965|2|1|19|1|1965|261|3395|Tuesday|1965Q1|N|N|N|2438762|2438761|2438414|2438688|N|N|N|N|N| +2438781|AAAAAAAANHGDFCAA|1965-01-20|780|3395|261|1965|3|1|20|1|1965|261|3395|Wednesday|1965Q1|N|N|N|2438762|2438761|2438415|2438689|N|N|N|N|N| +2438782|AAAAAAAAOHGDFCAA|1965-01-21|780|3395|261|1965|4|1|21|1|1965|261|3395|Thursday|1965Q1|N|N|N|2438762|2438761|2438416|2438690|N|N|N|N|N| +2438783|AAAAAAAAPHGDFCAA|1965-01-22|780|3395|261|1965|5|1|22|1|1965|261|3395|Friday|1965Q1|N|Y|N|2438762|2438761|2438417|2438691|N|N|N|N|N| +2438784|AAAAAAAAAIGDFCAA|1965-01-23|780|3395|261|1965|6|1|23|1|1965|261|3395|Saturday|1965Q1|N|Y|N|2438762|2438761|2438418|2438692|N|N|N|N|N| +2438785|AAAAAAAABIGDFCAA|1965-01-24|780|3395|261|1965|0|1|24|1|1965|261|3395|Sunday|1965Q1|N|N|N|2438762|2438761|2438419|2438693|N|N|N|N|N| +2438786|AAAAAAAACIGDFCAA|1965-01-25|780|3395|261|1965|1|1|25|1|1965|261|3395|Monday|1965Q1|N|N|N|2438762|2438761|2438420|2438694|N|N|N|N|N| +2438787|AAAAAAAADIGDFCAA|1965-01-26|780|3396|261|1965|2|1|26|1|1965|261|3396|Tuesday|1965Q1|N|N|N|2438762|2438761|2438421|2438695|N|N|N|N|N| +2438788|AAAAAAAAEIGDFCAA|1965-01-27|780|3396|261|1965|3|1|27|1|1965|261|3396|Wednesday|1965Q1|N|N|N|2438762|2438761|2438422|2438696|N|N|N|N|N| +2438789|AAAAAAAAFIGDFCAA|1965-01-28|780|3396|261|1965|4|1|28|1|1965|261|3396|Thursday|1965Q1|N|N|N|2438762|2438761|2438423|2438697|N|N|N|N|N| +2438790|AAAAAAAAGIGDFCAA|1965-01-29|780|3396|261|1965|5|1|29|1|1965|261|3396|Friday|1965Q1|N|Y|N|2438762|2438761|2438424|2438698|N|N|N|N|N| +2438791|AAAAAAAAHIGDFCAA|1965-01-30|780|3396|261|1965|6|1|30|1|1965|261|3396|Saturday|1965Q1|N|Y|N|2438762|2438761|2438425|2438699|N|N|N|N|N| +2438792|AAAAAAAAIIGDFCAA|1965-01-31|780|3396|261|1965|0|1|31|1|1965|261|3396|Sunday|1965Q1|N|N|N|2438762|2438761|2438426|2438700|N|N|N|N|N| +2438793|AAAAAAAAJIGDFCAA|1965-02-01|781|3396|261|1965|1|2|1|1|1965|261|3396|Monday|1965Q1|N|N|N|2438793|2438823|2438427|2438701|N|N|N|N|N| +2438794|AAAAAAAAKIGDFCAA|1965-02-02|781|3397|261|1965|2|2|2|1|1965|261|3397|Tuesday|1965Q1|N|N|N|2438793|2438823|2438428|2438702|N|N|N|N|N| +2438795|AAAAAAAALIGDFCAA|1965-02-03|781|3397|261|1965|3|2|3|1|1965|261|3397|Wednesday|1965Q1|N|N|N|2438793|2438823|2438429|2438703|N|N|N|N|N| +2438796|AAAAAAAAMIGDFCAA|1965-02-04|781|3397|261|1965|4|2|4|1|1965|261|3397|Thursday|1965Q1|N|N|N|2438793|2438823|2438430|2438704|N|N|N|N|N| +2438797|AAAAAAAANIGDFCAA|1965-02-05|781|3397|261|1965|5|2|5|1|1965|261|3397|Friday|1965Q1|N|Y|N|2438793|2438823|2438431|2438705|N|N|N|N|N| +2438798|AAAAAAAAOIGDFCAA|1965-02-06|781|3397|261|1965|6|2|6|1|1965|261|3397|Saturday|1965Q1|N|Y|N|2438793|2438823|2438432|2438706|N|N|N|N|N| +2438799|AAAAAAAAPIGDFCAA|1965-02-07|781|3397|261|1965|0|2|7|1|1965|261|3397|Sunday|1965Q1|N|N|N|2438793|2438823|2438433|2438707|N|N|N|N|N| +2438800|AAAAAAAAAJGDFCAA|1965-02-08|781|3397|261|1965|1|2|8|1|1965|261|3397|Monday|1965Q1|N|N|N|2438793|2438823|2438434|2438708|N|N|N|N|N| +2438801|AAAAAAAABJGDFCAA|1965-02-09|781|3398|261|1965|2|2|9|1|1965|261|3398|Tuesday|1965Q1|N|N|N|2438793|2438823|2438435|2438709|N|N|N|N|N| +2438802|AAAAAAAACJGDFCAA|1965-02-10|781|3398|261|1965|3|2|10|1|1965|261|3398|Wednesday|1965Q1|N|N|N|2438793|2438823|2438436|2438710|N|N|N|N|N| +2438803|AAAAAAAADJGDFCAA|1965-02-11|781|3398|261|1965|4|2|11|1|1965|261|3398|Thursday|1965Q1|N|N|N|2438793|2438823|2438437|2438711|N|N|N|N|N| +2438804|AAAAAAAAEJGDFCAA|1965-02-12|781|3398|261|1965|5|2|12|1|1965|261|3398|Friday|1965Q1|N|Y|N|2438793|2438823|2438438|2438712|N|N|N|N|N| +2438805|AAAAAAAAFJGDFCAA|1965-02-13|781|3398|261|1965|6|2|13|1|1965|261|3398|Saturday|1965Q1|N|Y|N|2438793|2438823|2438439|2438713|N|N|N|N|N| +2438806|AAAAAAAAGJGDFCAA|1965-02-14|781|3398|261|1965|0|2|14|1|1965|261|3398|Sunday|1965Q1|N|N|N|2438793|2438823|2438440|2438714|N|N|N|N|N| +2438807|AAAAAAAAHJGDFCAA|1965-02-15|781|3398|261|1965|1|2|15|1|1965|261|3398|Monday|1965Q1|N|N|N|2438793|2438823|2438441|2438715|N|N|N|N|N| +2438808|AAAAAAAAIJGDFCAA|1965-02-16|781|3399|261|1965|2|2|16|1|1965|261|3399|Tuesday|1965Q1|N|N|N|2438793|2438823|2438442|2438716|N|N|N|N|N| +2438809|AAAAAAAAJJGDFCAA|1965-02-17|781|3399|261|1965|3|2|17|1|1965|261|3399|Wednesday|1965Q1|N|N|N|2438793|2438823|2438443|2438717|N|N|N|N|N| +2438810|AAAAAAAAKJGDFCAA|1965-02-18|781|3399|261|1965|4|2|18|1|1965|261|3399|Thursday|1965Q1|N|N|N|2438793|2438823|2438444|2438718|N|N|N|N|N| +2438811|AAAAAAAALJGDFCAA|1965-02-19|781|3399|261|1965|5|2|19|1|1965|261|3399|Friday|1965Q1|N|Y|N|2438793|2438823|2438445|2438719|N|N|N|N|N| +2438812|AAAAAAAAMJGDFCAA|1965-02-20|781|3399|261|1965|6|2|20|1|1965|261|3399|Saturday|1965Q1|N|Y|N|2438793|2438823|2438446|2438720|N|N|N|N|N| +2438813|AAAAAAAANJGDFCAA|1965-02-21|781|3399|261|1965|0|2|21|1|1965|261|3399|Sunday|1965Q1|N|N|N|2438793|2438823|2438447|2438721|N|N|N|N|N| +2438814|AAAAAAAAOJGDFCAA|1965-02-22|781|3399|261|1965|1|2|22|1|1965|261|3399|Monday|1965Q1|N|N|N|2438793|2438823|2438448|2438722|N|N|N|N|N| +2438815|AAAAAAAAPJGDFCAA|1965-02-23|781|3400|261|1965|2|2|23|1|1965|261|3400|Tuesday|1965Q1|N|N|N|2438793|2438823|2438449|2438723|N|N|N|N|N| +2438816|AAAAAAAAAKGDFCAA|1965-02-24|781|3400|261|1965|3|2|24|1|1965|261|3400|Wednesday|1965Q1|N|N|N|2438793|2438823|2438450|2438724|N|N|N|N|N| +2438817|AAAAAAAABKGDFCAA|1965-02-25|781|3400|261|1965|4|2|25|1|1965|261|3400|Thursday|1965Q1|N|N|N|2438793|2438823|2438451|2438725|N|N|N|N|N| +2438818|AAAAAAAACKGDFCAA|1965-02-26|781|3400|261|1965|5|2|26|1|1965|261|3400|Friday|1965Q1|N|Y|N|2438793|2438823|2438452|2438726|N|N|N|N|N| +2438819|AAAAAAAADKGDFCAA|1965-02-27|781|3400|261|1965|6|2|27|1|1965|261|3400|Saturday|1965Q1|N|Y|N|2438793|2438823|2438453|2438727|N|N|N|N|N| +2438820|AAAAAAAAEKGDFCAA|1965-02-28|781|3400|261|1965|0|2|28|1|1965|261|3400|Sunday|1965Q1|N|N|N|2438793|2438823|2438454|2438728|N|N|N|N|N| +2438821|AAAAAAAAFKGDFCAA|1965-03-01|782|3400|262|1965|1|3|1|1|1965|262|3400|Monday|1965Q1|N|N|N|2438821|2438879|2438456|2438729|N|N|N|N|N| +2438822|AAAAAAAAGKGDFCAA|1965-03-02|782|3401|262|1965|2|3|2|1|1965|262|3401|Tuesday|1965Q1|N|N|N|2438821|2438879|2438457|2438730|N|N|N|N|N| +2438823|AAAAAAAAHKGDFCAA|1965-03-03|782|3401|262|1965|3|3|3|1|1965|262|3401|Wednesday|1965Q1|N|N|N|2438821|2438879|2438458|2438731|N|N|N|N|N| +2438824|AAAAAAAAIKGDFCAA|1965-03-04|782|3401|262|1965|4|3|4|1|1965|262|3401|Thursday|1965Q1|N|N|N|2438821|2438879|2438459|2438732|N|N|N|N|N| +2438825|AAAAAAAAJKGDFCAA|1965-03-05|782|3401|262|1965|5|3|5|1|1965|262|3401|Friday|1965Q1|N|Y|N|2438821|2438879|2438460|2438733|N|N|N|N|N| +2438826|AAAAAAAAKKGDFCAA|1965-03-06|782|3401|262|1965|6|3|6|1|1965|262|3401|Saturday|1965Q1|N|Y|N|2438821|2438879|2438461|2438734|N|N|N|N|N| +2438827|AAAAAAAALKGDFCAA|1965-03-07|782|3401|262|1965|0|3|7|1|1965|262|3401|Sunday|1965Q1|N|N|N|2438821|2438879|2438462|2438735|N|N|N|N|N| +2438828|AAAAAAAAMKGDFCAA|1965-03-08|782|3401|262|1965|1|3|8|1|1965|262|3401|Monday|1965Q1|N|N|N|2438821|2438879|2438463|2438736|N|N|N|N|N| +2438829|AAAAAAAANKGDFCAA|1965-03-09|782|3402|262|1965|2|3|9|1|1965|262|3402|Tuesday|1965Q1|N|N|N|2438821|2438879|2438464|2438737|N|N|N|N|N| +2438830|AAAAAAAAOKGDFCAA|1965-03-10|782|3402|262|1965|3|3|10|1|1965|262|3402|Wednesday|1965Q1|N|N|N|2438821|2438879|2438465|2438738|N|N|N|N|N| +2438831|AAAAAAAAPKGDFCAA|1965-03-11|782|3402|262|1965|4|3|11|1|1965|262|3402|Thursday|1965Q1|N|N|N|2438821|2438879|2438466|2438739|N|N|N|N|N| +2438832|AAAAAAAAALGDFCAA|1965-03-12|782|3402|262|1965|5|3|12|1|1965|262|3402|Friday|1965Q1|N|Y|N|2438821|2438879|2438467|2438740|N|N|N|N|N| +2438833|AAAAAAAABLGDFCAA|1965-03-13|782|3402|262|1965|6|3|13|1|1965|262|3402|Saturday|1965Q1|N|Y|N|2438821|2438879|2438468|2438741|N|N|N|N|N| +2438834|AAAAAAAACLGDFCAA|1965-03-14|782|3402|262|1965|0|3|14|1|1965|262|3402|Sunday|1965Q1|N|N|N|2438821|2438879|2438469|2438742|N|N|N|N|N| +2438835|AAAAAAAADLGDFCAA|1965-03-15|782|3402|262|1965|1|3|15|1|1965|262|3402|Monday|1965Q1|N|N|N|2438821|2438879|2438470|2438743|N|N|N|N|N| +2438836|AAAAAAAAELGDFCAA|1965-03-16|782|3403|262|1965|2|3|16|1|1965|262|3403|Tuesday|1965Q1|N|N|N|2438821|2438879|2438471|2438744|N|N|N|N|N| +2438837|AAAAAAAAFLGDFCAA|1965-03-17|782|3403|262|1965|3|3|17|1|1965|262|3403|Wednesday|1965Q1|N|N|N|2438821|2438879|2438472|2438745|N|N|N|N|N| +2438838|AAAAAAAAGLGDFCAA|1965-03-18|782|3403|262|1965|4|3|18|1|1965|262|3403|Thursday|1965Q1|N|N|N|2438821|2438879|2438473|2438746|N|N|N|N|N| +2438839|AAAAAAAAHLGDFCAA|1965-03-19|782|3403|262|1965|5|3|19|1|1965|262|3403|Friday|1965Q1|N|Y|N|2438821|2438879|2438474|2438747|N|N|N|N|N| +2438840|AAAAAAAAILGDFCAA|1965-03-20|782|3403|262|1965|6|3|20|1|1965|262|3403|Saturday|1965Q1|N|Y|N|2438821|2438879|2438475|2438748|N|N|N|N|N| +2438841|AAAAAAAAJLGDFCAA|1965-03-21|782|3403|262|1965|0|3|21|1|1965|262|3403|Sunday|1965Q1|N|N|N|2438821|2438879|2438476|2438749|N|N|N|N|N| +2438842|AAAAAAAAKLGDFCAA|1965-03-22|782|3403|262|1965|1|3|22|1|1965|262|3403|Monday|1965Q1|N|N|N|2438821|2438879|2438477|2438750|N|N|N|N|N| +2438843|AAAAAAAALLGDFCAA|1965-03-23|782|3404|262|1965|2|3|23|1|1965|262|3404|Tuesday|1965Q1|N|N|N|2438821|2438879|2438478|2438751|N|N|N|N|N| +2438844|AAAAAAAAMLGDFCAA|1965-03-24|782|3404|262|1965|3|3|24|1|1965|262|3404|Wednesday|1965Q1|N|N|N|2438821|2438879|2438479|2438752|N|N|N|N|N| +2438845|AAAAAAAANLGDFCAA|1965-03-25|782|3404|262|1965|4|3|25|1|1965|262|3404|Thursday|1965Q1|N|N|N|2438821|2438879|2438480|2438753|N|N|N|N|N| +2438846|AAAAAAAAOLGDFCAA|1965-03-26|782|3404|262|1965|5|3|26|1|1965|262|3404|Friday|1965Q1|N|Y|N|2438821|2438879|2438481|2438754|N|N|N|N|N| +2438847|AAAAAAAAPLGDFCAA|1965-03-27|782|3404|262|1965|6|3|27|1|1965|262|3404|Saturday|1965Q1|N|Y|N|2438821|2438879|2438482|2438755|N|N|N|N|N| +2438848|AAAAAAAAAMGDFCAA|1965-03-28|782|3404|262|1965|0|3|28|1|1965|262|3404|Sunday|1965Q1|N|N|N|2438821|2438879|2438483|2438756|N|N|N|N|N| +2438849|AAAAAAAABMGDFCAA|1965-03-29|782|3404|262|1965|1|3|29|1|1965|262|3404|Monday|1965Q1|N|N|N|2438821|2438879|2438484|2438757|N|N|N|N|N| +2438850|AAAAAAAACMGDFCAA|1965-03-30|782|3405|262|1965|2|3|30|1|1965|262|3405|Tuesday|1965Q1|N|N|N|2438821|2438879|2438485|2438758|N|N|N|N|N| +2438851|AAAAAAAADMGDFCAA|1965-03-31|782|3405|262|1965|3|3|31|1|1965|262|3405|Wednesday|1965Q1|N|N|N|2438821|2438879|2438486|2438759|N|N|N|N|N| +2438852|AAAAAAAAEMGDFCAA|1965-04-01|783|3405|262|1965|4|4|1|1|1965|262|3405|Thursday|1965Q1|N|N|N|2438852|2438941|2438487|2438762|N|N|N|N|N| +2438853|AAAAAAAAFMGDFCAA|1965-04-02|783|3405|262|1965|5|4|2|2|1965|262|3405|Friday|1965Q2|N|Y|N|2438852|2438941|2438488|2438763|N|N|N|N|N| +2438854|AAAAAAAAGMGDFCAA|1965-04-03|783|3405|262|1965|6|4|3|2|1965|262|3405|Saturday|1965Q2|N|Y|N|2438852|2438941|2438489|2438764|N|N|N|N|N| +2438855|AAAAAAAAHMGDFCAA|1965-04-04|783|3405|262|1965|0|4|4|2|1965|262|3405|Sunday|1965Q2|N|N|N|2438852|2438941|2438490|2438765|N|N|N|N|N| +2438856|AAAAAAAAIMGDFCAA|1965-04-05|783|3405|262|1965|1|4|5|2|1965|262|3405|Monday|1965Q2|N|N|N|2438852|2438941|2438491|2438766|N|N|N|N|N| +2438857|AAAAAAAAJMGDFCAA|1965-04-06|783|3406|262|1965|2|4|6|2|1965|262|3406|Tuesday|1965Q2|N|N|N|2438852|2438941|2438492|2438767|N|N|N|N|N| +2438858|AAAAAAAAKMGDFCAA|1965-04-07|783|3406|262|1965|3|4|7|2|1965|262|3406|Wednesday|1965Q2|N|N|N|2438852|2438941|2438493|2438768|N|N|N|N|N| +2438859|AAAAAAAALMGDFCAA|1965-04-08|783|3406|262|1965|4|4|8|2|1965|262|3406|Thursday|1965Q2|N|N|N|2438852|2438941|2438494|2438769|N|N|N|N|N| +2438860|AAAAAAAAMMGDFCAA|1965-04-09|783|3406|262|1965|5|4|9|2|1965|262|3406|Friday|1965Q2|N|Y|N|2438852|2438941|2438495|2438770|N|N|N|N|N| +2438861|AAAAAAAANMGDFCAA|1965-04-10|783|3406|262|1965|6|4|10|2|1965|262|3406|Saturday|1965Q2|N|Y|N|2438852|2438941|2438496|2438771|N|N|N|N|N| +2438862|AAAAAAAAOMGDFCAA|1965-04-11|783|3406|262|1965|0|4|11|2|1965|262|3406|Sunday|1965Q2|N|N|N|2438852|2438941|2438497|2438772|N|N|N|N|N| +2438863|AAAAAAAAPMGDFCAA|1965-04-12|783|3406|262|1965|1|4|12|2|1965|262|3406|Monday|1965Q2|N|N|N|2438852|2438941|2438498|2438773|N|N|N|N|N| +2438864|AAAAAAAAANGDFCAA|1965-04-13|783|3407|262|1965|2|4|13|2|1965|262|3407|Tuesday|1965Q2|N|N|N|2438852|2438941|2438499|2438774|N|N|N|N|N| +2438865|AAAAAAAABNGDFCAA|1965-04-14|783|3407|262|1965|3|4|14|2|1965|262|3407|Wednesday|1965Q2|N|N|N|2438852|2438941|2438500|2438775|N|N|N|N|N| +2438866|AAAAAAAACNGDFCAA|1965-04-15|783|3407|262|1965|4|4|15|2|1965|262|3407|Thursday|1965Q2|N|N|N|2438852|2438941|2438501|2438776|N|N|N|N|N| +2438867|AAAAAAAADNGDFCAA|1965-04-16|783|3407|262|1965|5|4|16|2|1965|262|3407|Friday|1965Q2|N|Y|N|2438852|2438941|2438502|2438777|N|N|N|N|N| +2438868|AAAAAAAAENGDFCAA|1965-04-17|783|3407|262|1965|6|4|17|2|1965|262|3407|Saturday|1965Q2|N|Y|N|2438852|2438941|2438503|2438778|N|N|N|N|N| +2438869|AAAAAAAAFNGDFCAA|1965-04-18|783|3407|262|1965|0|4|18|2|1965|262|3407|Sunday|1965Q2|N|N|N|2438852|2438941|2438504|2438779|N|N|N|N|N| +2438870|AAAAAAAAGNGDFCAA|1965-04-19|783|3407|262|1965|1|4|19|2|1965|262|3407|Monday|1965Q2|N|N|N|2438852|2438941|2438505|2438780|N|N|N|N|N| +2438871|AAAAAAAAHNGDFCAA|1965-04-20|783|3408|262|1965|2|4|20|2|1965|262|3408|Tuesday|1965Q2|N|N|N|2438852|2438941|2438506|2438781|N|N|N|N|N| +2438872|AAAAAAAAINGDFCAA|1965-04-21|783|3408|262|1965|3|4|21|2|1965|262|3408|Wednesday|1965Q2|N|N|N|2438852|2438941|2438507|2438782|N|N|N|N|N| +2438873|AAAAAAAAJNGDFCAA|1965-04-22|783|3408|262|1965|4|4|22|2|1965|262|3408|Thursday|1965Q2|N|N|N|2438852|2438941|2438508|2438783|N|N|N|N|N| +2438874|AAAAAAAAKNGDFCAA|1965-04-23|783|3408|262|1965|5|4|23|2|1965|262|3408|Friday|1965Q2|N|Y|N|2438852|2438941|2438509|2438784|N|N|N|N|N| +2438875|AAAAAAAALNGDFCAA|1965-04-24|783|3408|262|1965|6|4|24|2|1965|262|3408|Saturday|1965Q2|N|Y|N|2438852|2438941|2438510|2438785|N|N|N|N|N| +2438876|AAAAAAAAMNGDFCAA|1965-04-25|783|3408|262|1965|0|4|25|2|1965|262|3408|Sunday|1965Q2|N|N|N|2438852|2438941|2438511|2438786|N|N|N|N|N| +2438877|AAAAAAAANNGDFCAA|1965-04-26|783|3408|262|1965|1|4|26|2|1965|262|3408|Monday|1965Q2|N|N|N|2438852|2438941|2438512|2438787|N|N|N|N|N| +2438878|AAAAAAAAONGDFCAA|1965-04-27|783|3409|262|1965|2|4|27|2|1965|262|3409|Tuesday|1965Q2|N|N|N|2438852|2438941|2438513|2438788|N|N|N|N|N| +2438879|AAAAAAAAPNGDFCAA|1965-04-28|783|3409|262|1965|3|4|28|2|1965|262|3409|Wednesday|1965Q2|N|N|N|2438852|2438941|2438514|2438789|N|N|N|N|N| +2438880|AAAAAAAAAOGDFCAA|1965-04-29|783|3409|262|1965|4|4|29|2|1965|262|3409|Thursday|1965Q2|N|N|N|2438852|2438941|2438515|2438790|N|N|N|N|N| +2438881|AAAAAAAABOGDFCAA|1965-04-30|783|3409|262|1965|5|4|30|2|1965|262|3409|Friday|1965Q2|N|Y|N|2438852|2438941|2438516|2438791|N|N|N|N|N| +2438882|AAAAAAAACOGDFCAA|1965-05-01|784|3409|262|1965|6|5|1|2|1965|262|3409|Saturday|1965Q2|N|Y|N|2438882|2439001|2438517|2438792|N|N|N|N|N| +2438883|AAAAAAAADOGDFCAA|1965-05-02|784|3409|262|1965|0|5|2|2|1965|262|3409|Sunday|1965Q2|N|N|N|2438882|2439001|2438518|2438793|N|N|N|N|N| +2438884|AAAAAAAAEOGDFCAA|1965-05-03|784|3409|262|1965|1|5|3|2|1965|262|3409|Monday|1965Q2|N|N|N|2438882|2439001|2438519|2438794|N|N|N|N|N| +2438885|AAAAAAAAFOGDFCAA|1965-05-04|784|3410|262|1965|2|5|4|2|1965|262|3410|Tuesday|1965Q2|N|N|N|2438882|2439001|2438520|2438795|N|N|N|N|N| +2438886|AAAAAAAAGOGDFCAA|1965-05-05|784|3410|262|1965|3|5|5|2|1965|262|3410|Wednesday|1965Q2|N|N|N|2438882|2439001|2438521|2438796|N|N|N|N|N| +2438887|AAAAAAAAHOGDFCAA|1965-05-06|784|3410|262|1965|4|5|6|2|1965|262|3410|Thursday|1965Q2|N|N|N|2438882|2439001|2438522|2438797|N|N|N|N|N| +2438888|AAAAAAAAIOGDFCAA|1965-05-07|784|3410|262|1965|5|5|7|2|1965|262|3410|Friday|1965Q2|N|Y|N|2438882|2439001|2438523|2438798|N|N|N|N|N| +2438889|AAAAAAAAJOGDFCAA|1965-05-08|784|3410|262|1965|6|5|8|2|1965|262|3410|Saturday|1965Q2|N|Y|N|2438882|2439001|2438524|2438799|N|N|N|N|N| +2438890|AAAAAAAAKOGDFCAA|1965-05-09|784|3410|262|1965|0|5|9|2|1965|262|3410|Sunday|1965Q2|N|N|N|2438882|2439001|2438525|2438800|N|N|N|N|N| +2438891|AAAAAAAALOGDFCAA|1965-05-10|784|3410|262|1965|1|5|10|2|1965|262|3410|Monday|1965Q2|N|N|N|2438882|2439001|2438526|2438801|N|N|N|N|N| +2438892|AAAAAAAAMOGDFCAA|1965-05-11|784|3411|262|1965|2|5|11|2|1965|262|3411|Tuesday|1965Q2|N|N|N|2438882|2439001|2438527|2438802|N|N|N|N|N| +2438893|AAAAAAAANOGDFCAA|1965-05-12|784|3411|262|1965|3|5|12|2|1965|262|3411|Wednesday|1965Q2|N|N|N|2438882|2439001|2438528|2438803|N|N|N|N|N| +2438894|AAAAAAAAOOGDFCAA|1965-05-13|784|3411|262|1965|4|5|13|2|1965|262|3411|Thursday|1965Q2|N|N|N|2438882|2439001|2438529|2438804|N|N|N|N|N| +2438895|AAAAAAAAPOGDFCAA|1965-05-14|784|3411|262|1965|5|5|14|2|1965|262|3411|Friday|1965Q2|N|Y|N|2438882|2439001|2438530|2438805|N|N|N|N|N| +2438896|AAAAAAAAAPGDFCAA|1965-05-15|784|3411|262|1965|6|5|15|2|1965|262|3411|Saturday|1965Q2|N|Y|N|2438882|2439001|2438531|2438806|N|N|N|N|N| +2438897|AAAAAAAABPGDFCAA|1965-05-16|784|3411|262|1965|0|5|16|2|1965|262|3411|Sunday|1965Q2|N|N|N|2438882|2439001|2438532|2438807|N|N|N|N|N| +2438898|AAAAAAAACPGDFCAA|1965-05-17|784|3411|262|1965|1|5|17|2|1965|262|3411|Monday|1965Q2|N|N|N|2438882|2439001|2438533|2438808|N|N|N|N|N| +2438899|AAAAAAAADPGDFCAA|1965-05-18|784|3412|262|1965|2|5|18|2|1965|262|3412|Tuesday|1965Q2|N|N|N|2438882|2439001|2438534|2438809|N|N|N|N|N| +2438900|AAAAAAAAEPGDFCAA|1965-05-19|784|3412|262|1965|3|5|19|2|1965|262|3412|Wednesday|1965Q2|N|N|N|2438882|2439001|2438535|2438810|N|N|N|N|N| +2438901|AAAAAAAAFPGDFCAA|1965-05-20|784|3412|262|1965|4|5|20|2|1965|262|3412|Thursday|1965Q2|N|N|N|2438882|2439001|2438536|2438811|N|N|N|N|N| +2438902|AAAAAAAAGPGDFCAA|1965-05-21|784|3412|262|1965|5|5|21|2|1965|262|3412|Friday|1965Q2|N|Y|N|2438882|2439001|2438537|2438812|N|N|N|N|N| +2438903|AAAAAAAAHPGDFCAA|1965-05-22|784|3412|262|1965|6|5|22|2|1965|262|3412|Saturday|1965Q2|N|Y|N|2438882|2439001|2438538|2438813|N|N|N|N|N| +2438904|AAAAAAAAIPGDFCAA|1965-05-23|784|3412|262|1965|0|5|23|2|1965|262|3412|Sunday|1965Q2|N|N|N|2438882|2439001|2438539|2438814|N|N|N|N|N| +2438905|AAAAAAAAJPGDFCAA|1965-05-24|784|3412|262|1965|1|5|24|2|1965|262|3412|Monday|1965Q2|N|N|N|2438882|2439001|2438540|2438815|N|N|N|N|N| +2438906|AAAAAAAAKPGDFCAA|1965-05-25|784|3413|262|1965|2|5|25|2|1965|262|3413|Tuesday|1965Q2|N|N|N|2438882|2439001|2438541|2438816|N|N|N|N|N| +2438907|AAAAAAAALPGDFCAA|1965-05-26|784|3413|262|1965|3|5|26|2|1965|262|3413|Wednesday|1965Q2|N|N|N|2438882|2439001|2438542|2438817|N|N|N|N|N| +2438908|AAAAAAAAMPGDFCAA|1965-05-27|784|3413|262|1965|4|5|27|2|1965|262|3413|Thursday|1965Q2|N|N|N|2438882|2439001|2438543|2438818|N|N|N|N|N| +2438909|AAAAAAAANPGDFCAA|1965-05-28|784|3413|262|1965|5|5|28|2|1965|262|3413|Friday|1965Q2|N|Y|N|2438882|2439001|2438544|2438819|N|N|N|N|N| +2438910|AAAAAAAAOPGDFCAA|1965-05-29|784|3413|262|1965|6|5|29|2|1965|262|3413|Saturday|1965Q2|N|Y|N|2438882|2439001|2438545|2438820|N|N|N|N|N| +2438911|AAAAAAAAPPGDFCAA|1965-05-30|784|3413|262|1965|0|5|30|2|1965|262|3413|Sunday|1965Q2|N|N|N|2438882|2439001|2438546|2438821|N|N|N|N|N| +2438912|AAAAAAAAAAHDFCAA|1965-05-31|784|3413|262|1965|1|5|31|2|1965|262|3413|Monday|1965Q2|N|N|N|2438882|2439001|2438547|2438822|N|N|N|N|N| +2438913|AAAAAAAABAHDFCAA|1965-06-01|785|3414|263|1965|2|6|1|2|1965|263|3414|Tuesday|1965Q2|N|N|N|2438913|2439063|2438548|2438823|N|N|N|N|N| +2438914|AAAAAAAACAHDFCAA|1965-06-02|785|3414|263|1965|3|6|2|2|1965|263|3414|Wednesday|1965Q2|N|N|N|2438913|2439063|2438549|2438824|N|N|N|N|N| +2438915|AAAAAAAADAHDFCAA|1965-06-03|785|3414|263|1965|4|6|3|2|1965|263|3414|Thursday|1965Q2|N|N|N|2438913|2439063|2438550|2438825|N|N|N|N|N| +2438916|AAAAAAAAEAHDFCAA|1965-06-04|785|3414|263|1965|5|6|4|2|1965|263|3414|Friday|1965Q2|N|Y|N|2438913|2439063|2438551|2438826|N|N|N|N|N| +2438917|AAAAAAAAFAHDFCAA|1965-06-05|785|3414|263|1965|6|6|5|2|1965|263|3414|Saturday|1965Q2|N|Y|N|2438913|2439063|2438552|2438827|N|N|N|N|N| +2438918|AAAAAAAAGAHDFCAA|1965-06-06|785|3414|263|1965|0|6|6|2|1965|263|3414|Sunday|1965Q2|N|N|N|2438913|2439063|2438553|2438828|N|N|N|N|N| +2438919|AAAAAAAAHAHDFCAA|1965-06-07|785|3414|263|1965|1|6|7|2|1965|263|3414|Monday|1965Q2|N|N|N|2438913|2439063|2438554|2438829|N|N|N|N|N| +2438920|AAAAAAAAIAHDFCAA|1965-06-08|785|3415|263|1965|2|6|8|2|1965|263|3415|Tuesday|1965Q2|N|N|N|2438913|2439063|2438555|2438830|N|N|N|N|N| +2438921|AAAAAAAAJAHDFCAA|1965-06-09|785|3415|263|1965|3|6|9|2|1965|263|3415|Wednesday|1965Q2|N|N|N|2438913|2439063|2438556|2438831|N|N|N|N|N| +2438922|AAAAAAAAKAHDFCAA|1965-06-10|785|3415|263|1965|4|6|10|2|1965|263|3415|Thursday|1965Q2|N|N|N|2438913|2439063|2438557|2438832|N|N|N|N|N| +2438923|AAAAAAAALAHDFCAA|1965-06-11|785|3415|263|1965|5|6|11|2|1965|263|3415|Friday|1965Q2|N|Y|N|2438913|2439063|2438558|2438833|N|N|N|N|N| +2438924|AAAAAAAAMAHDFCAA|1965-06-12|785|3415|263|1965|6|6|12|2|1965|263|3415|Saturday|1965Q2|N|Y|N|2438913|2439063|2438559|2438834|N|N|N|N|N| +2438925|AAAAAAAANAHDFCAA|1965-06-13|785|3415|263|1965|0|6|13|2|1965|263|3415|Sunday|1965Q2|N|N|N|2438913|2439063|2438560|2438835|N|N|N|N|N| +2438926|AAAAAAAAOAHDFCAA|1965-06-14|785|3415|263|1965|1|6|14|2|1965|263|3415|Monday|1965Q2|N|N|N|2438913|2439063|2438561|2438836|N|N|N|N|N| +2438927|AAAAAAAAPAHDFCAA|1965-06-15|785|3416|263|1965|2|6|15|2|1965|263|3416|Tuesday|1965Q2|N|N|N|2438913|2439063|2438562|2438837|N|N|N|N|N| +2438928|AAAAAAAAABHDFCAA|1965-06-16|785|3416|263|1965|3|6|16|2|1965|263|3416|Wednesday|1965Q2|N|N|N|2438913|2439063|2438563|2438838|N|N|N|N|N| +2438929|AAAAAAAABBHDFCAA|1965-06-17|785|3416|263|1965|4|6|17|2|1965|263|3416|Thursday|1965Q2|N|N|N|2438913|2439063|2438564|2438839|N|N|N|N|N| +2438930|AAAAAAAACBHDFCAA|1965-06-18|785|3416|263|1965|5|6|18|2|1965|263|3416|Friday|1965Q2|N|Y|N|2438913|2439063|2438565|2438840|N|N|N|N|N| +2438931|AAAAAAAADBHDFCAA|1965-06-19|785|3416|263|1965|6|6|19|2|1965|263|3416|Saturday|1965Q2|N|Y|N|2438913|2439063|2438566|2438841|N|N|N|N|N| +2438932|AAAAAAAAEBHDFCAA|1965-06-20|785|3416|263|1965|0|6|20|2|1965|263|3416|Sunday|1965Q2|N|N|N|2438913|2439063|2438567|2438842|N|N|N|N|N| +2438933|AAAAAAAAFBHDFCAA|1965-06-21|785|3416|263|1965|1|6|21|2|1965|263|3416|Monday|1965Q2|N|N|N|2438913|2439063|2438568|2438843|N|N|N|N|N| +2438934|AAAAAAAAGBHDFCAA|1965-06-22|785|3417|263|1965|2|6|22|2|1965|263|3417|Tuesday|1965Q2|N|N|N|2438913|2439063|2438569|2438844|N|N|N|N|N| +2438935|AAAAAAAAHBHDFCAA|1965-06-23|785|3417|263|1965|3|6|23|2|1965|263|3417|Wednesday|1965Q2|N|N|N|2438913|2439063|2438570|2438845|N|N|N|N|N| +2438936|AAAAAAAAIBHDFCAA|1965-06-24|785|3417|263|1965|4|6|24|2|1965|263|3417|Thursday|1965Q2|N|N|N|2438913|2439063|2438571|2438846|N|N|N|N|N| +2438937|AAAAAAAAJBHDFCAA|1965-06-25|785|3417|263|1965|5|6|25|2|1965|263|3417|Friday|1965Q2|N|Y|N|2438913|2439063|2438572|2438847|N|N|N|N|N| +2438938|AAAAAAAAKBHDFCAA|1965-06-26|785|3417|263|1965|6|6|26|2|1965|263|3417|Saturday|1965Q2|N|Y|N|2438913|2439063|2438573|2438848|N|N|N|N|N| +2438939|AAAAAAAALBHDFCAA|1965-06-27|785|3417|263|1965|0|6|27|2|1965|263|3417|Sunday|1965Q2|N|N|N|2438913|2439063|2438574|2438849|N|N|N|N|N| +2438940|AAAAAAAAMBHDFCAA|1965-06-28|785|3417|263|1965|1|6|28|2|1965|263|3417|Monday|1965Q2|N|N|N|2438913|2439063|2438575|2438850|N|N|N|N|N| +2438941|AAAAAAAANBHDFCAA|1965-06-29|785|3418|263|1965|2|6|29|2|1965|263|3418|Tuesday|1965Q2|N|N|N|2438913|2439063|2438576|2438851|N|N|N|N|N| +2438942|AAAAAAAAOBHDFCAA|1965-06-30|785|3418|263|1965|3|6|30|2|1965|263|3418|Wednesday|1965Q2|N|N|N|2438913|2439063|2438577|2438852|N|N|N|N|N| +2438943|AAAAAAAAPBHDFCAA|1965-07-01|786|3418|263|1965|4|7|1|2|1965|263|3418|Thursday|1965Q2|N|N|N|2438943|2439123|2438578|2438852|N|N|N|N|N| +2438944|AAAAAAAAACHDFCAA|1965-07-02|786|3418|263|1965|5|7|2|3|1965|263|3418|Friday|1965Q3|N|Y|N|2438943|2439123|2438579|2438853|N|N|N|N|N| +2438945|AAAAAAAABCHDFCAA|1965-07-03|786|3418|263|1965|6|7|3|3|1965|263|3418|Saturday|1965Q3|N|Y|N|2438943|2439123|2438580|2438854|N|N|N|N|N| +2438946|AAAAAAAACCHDFCAA|1965-07-04|786|3418|263|1965|0|7|4|3|1965|263|3418|Sunday|1965Q3|N|N|N|2438943|2439123|2438581|2438855|N|N|N|N|N| +2438947|AAAAAAAADCHDFCAA|1965-07-05|786|3418|263|1965|1|7|5|3|1965|263|3418|Monday|1965Q3|Y|N|N|2438943|2439123|2438582|2438856|N|N|N|N|N| +2438948|AAAAAAAAECHDFCAA|1965-07-06|786|3419|263|1965|2|7|6|3|1965|263|3419|Tuesday|1965Q3|N|N|Y|2438943|2439123|2438583|2438857|N|N|N|N|N| +2438949|AAAAAAAAFCHDFCAA|1965-07-07|786|3419|263|1965|3|7|7|3|1965|263|3419|Wednesday|1965Q3|N|N|N|2438943|2439123|2438584|2438858|N|N|N|N|N| +2438950|AAAAAAAAGCHDFCAA|1965-07-08|786|3419|263|1965|4|7|8|3|1965|263|3419|Thursday|1965Q3|N|N|N|2438943|2439123|2438585|2438859|N|N|N|N|N| +2438951|AAAAAAAAHCHDFCAA|1965-07-09|786|3419|263|1965|5|7|9|3|1965|263|3419|Friday|1965Q3|N|Y|N|2438943|2439123|2438586|2438860|N|N|N|N|N| +2438952|AAAAAAAAICHDFCAA|1965-07-10|786|3419|263|1965|6|7|10|3|1965|263|3419|Saturday|1965Q3|N|Y|N|2438943|2439123|2438587|2438861|N|N|N|N|N| +2438953|AAAAAAAAJCHDFCAA|1965-07-11|786|3419|263|1965|0|7|11|3|1965|263|3419|Sunday|1965Q3|N|N|N|2438943|2439123|2438588|2438862|N|N|N|N|N| +2438954|AAAAAAAAKCHDFCAA|1965-07-12|786|3419|263|1965|1|7|12|3|1965|263|3419|Monday|1965Q3|N|N|N|2438943|2439123|2438589|2438863|N|N|N|N|N| +2438955|AAAAAAAALCHDFCAA|1965-07-13|786|3420|263|1965|2|7|13|3|1965|263|3420|Tuesday|1965Q3|N|N|N|2438943|2439123|2438590|2438864|N|N|N|N|N| +2438956|AAAAAAAAMCHDFCAA|1965-07-14|786|3420|263|1965|3|7|14|3|1965|263|3420|Wednesday|1965Q3|N|N|N|2438943|2439123|2438591|2438865|N|N|N|N|N| +2438957|AAAAAAAANCHDFCAA|1965-07-15|786|3420|263|1965|4|7|15|3|1965|263|3420|Thursday|1965Q3|N|N|N|2438943|2439123|2438592|2438866|N|N|N|N|N| +2438958|AAAAAAAAOCHDFCAA|1965-07-16|786|3420|263|1965|5|7|16|3|1965|263|3420|Friday|1965Q3|N|Y|N|2438943|2439123|2438593|2438867|N|N|N|N|N| +2438959|AAAAAAAAPCHDFCAA|1965-07-17|786|3420|263|1965|6|7|17|3|1965|263|3420|Saturday|1965Q3|N|Y|N|2438943|2439123|2438594|2438868|N|N|N|N|N| +2438960|AAAAAAAAADHDFCAA|1965-07-18|786|3420|263|1965|0|7|18|3|1965|263|3420|Sunday|1965Q3|N|N|N|2438943|2439123|2438595|2438869|N|N|N|N|N| +2438961|AAAAAAAABDHDFCAA|1965-07-19|786|3420|263|1965|1|7|19|3|1965|263|3420|Monday|1965Q3|N|N|N|2438943|2439123|2438596|2438870|N|N|N|N|N| +2438962|AAAAAAAACDHDFCAA|1965-07-20|786|3421|263|1965|2|7|20|3|1965|263|3421|Tuesday|1965Q3|N|N|N|2438943|2439123|2438597|2438871|N|N|N|N|N| +2438963|AAAAAAAADDHDFCAA|1965-07-21|786|3421|263|1965|3|7|21|3|1965|263|3421|Wednesday|1965Q3|N|N|N|2438943|2439123|2438598|2438872|N|N|N|N|N| +2438964|AAAAAAAAEDHDFCAA|1965-07-22|786|3421|263|1965|4|7|22|3|1965|263|3421|Thursday|1965Q3|N|N|N|2438943|2439123|2438599|2438873|N|N|N|N|N| +2438965|AAAAAAAAFDHDFCAA|1965-07-23|786|3421|263|1965|5|7|23|3|1965|263|3421|Friday|1965Q3|N|Y|N|2438943|2439123|2438600|2438874|N|N|N|N|N| +2438966|AAAAAAAAGDHDFCAA|1965-07-24|786|3421|263|1965|6|7|24|3|1965|263|3421|Saturday|1965Q3|N|Y|N|2438943|2439123|2438601|2438875|N|N|N|N|N| +2438967|AAAAAAAAHDHDFCAA|1965-07-25|786|3421|263|1965|0|7|25|3|1965|263|3421|Sunday|1965Q3|N|N|N|2438943|2439123|2438602|2438876|N|N|N|N|N| +2438968|AAAAAAAAIDHDFCAA|1965-07-26|786|3421|263|1965|1|7|26|3|1965|263|3421|Monday|1965Q3|N|N|N|2438943|2439123|2438603|2438877|N|N|N|N|N| +2438969|AAAAAAAAJDHDFCAA|1965-07-27|786|3422|263|1965|2|7|27|3|1965|263|3422|Tuesday|1965Q3|N|N|N|2438943|2439123|2438604|2438878|N|N|N|N|N| +2438970|AAAAAAAAKDHDFCAA|1965-07-28|786|3422|263|1965|3|7|28|3|1965|263|3422|Wednesday|1965Q3|N|N|N|2438943|2439123|2438605|2438879|N|N|N|N|N| +2438971|AAAAAAAALDHDFCAA|1965-07-29|786|3422|263|1965|4|7|29|3|1965|263|3422|Thursday|1965Q3|N|N|N|2438943|2439123|2438606|2438880|N|N|N|N|N| +2438972|AAAAAAAAMDHDFCAA|1965-07-30|786|3422|263|1965|5|7|30|3|1965|263|3422|Friday|1965Q3|N|Y|N|2438943|2439123|2438607|2438881|N|N|N|N|N| +2438973|AAAAAAAANDHDFCAA|1965-07-31|786|3422|263|1965|6|7|31|3|1965|263|3422|Saturday|1965Q3|N|Y|N|2438943|2439123|2438608|2438882|N|N|N|N|N| +2438974|AAAAAAAAODHDFCAA|1965-08-01|787|3422|263|1965|0|8|1|3|1965|263|3422|Sunday|1965Q3|N|N|N|2438974|2439185|2438609|2438883|N|N|N|N|N| +2438975|AAAAAAAAPDHDFCAA|1965-08-02|787|3422|263|1965|1|8|2|3|1965|263|3422|Monday|1965Q3|N|N|N|2438974|2439185|2438610|2438884|N|N|N|N|N| +2438976|AAAAAAAAAEHDFCAA|1965-08-03|787|3423|263|1965|2|8|3|3|1965|263|3423|Tuesday|1965Q3|N|N|N|2438974|2439185|2438611|2438885|N|N|N|N|N| +2438977|AAAAAAAABEHDFCAA|1965-08-04|787|3423|263|1965|3|8|4|3|1965|263|3423|Wednesday|1965Q3|N|N|N|2438974|2439185|2438612|2438886|N|N|N|N|N| +2438978|AAAAAAAACEHDFCAA|1965-08-05|787|3423|263|1965|4|8|5|3|1965|263|3423|Thursday|1965Q3|N|N|N|2438974|2439185|2438613|2438887|N|N|N|N|N| +2438979|AAAAAAAADEHDFCAA|1965-08-06|787|3423|263|1965|5|8|6|3|1965|263|3423|Friday|1965Q3|N|Y|N|2438974|2439185|2438614|2438888|N|N|N|N|N| +2438980|AAAAAAAAEEHDFCAA|1965-08-07|787|3423|263|1965|6|8|7|3|1965|263|3423|Saturday|1965Q3|N|Y|N|2438974|2439185|2438615|2438889|N|N|N|N|N| +2438981|AAAAAAAAFEHDFCAA|1965-08-08|787|3423|263|1965|0|8|8|3|1965|263|3423|Sunday|1965Q3|N|N|N|2438974|2439185|2438616|2438890|N|N|N|N|N| +2438982|AAAAAAAAGEHDFCAA|1965-08-09|787|3423|263|1965|1|8|9|3|1965|263|3423|Monday|1965Q3|N|N|N|2438974|2439185|2438617|2438891|N|N|N|N|N| +2438983|AAAAAAAAHEHDFCAA|1965-08-10|787|3424|263|1965|2|8|10|3|1965|263|3424|Tuesday|1965Q3|N|N|N|2438974|2439185|2438618|2438892|N|N|N|N|N| +2438984|AAAAAAAAIEHDFCAA|1965-08-11|787|3424|263|1965|3|8|11|3|1965|263|3424|Wednesday|1965Q3|N|N|N|2438974|2439185|2438619|2438893|N|N|N|N|N| +2438985|AAAAAAAAJEHDFCAA|1965-08-12|787|3424|263|1965|4|8|12|3|1965|263|3424|Thursday|1965Q3|N|N|N|2438974|2439185|2438620|2438894|N|N|N|N|N| +2438986|AAAAAAAAKEHDFCAA|1965-08-13|787|3424|263|1965|5|8|13|3|1965|263|3424|Friday|1965Q3|N|Y|N|2438974|2439185|2438621|2438895|N|N|N|N|N| +2438987|AAAAAAAALEHDFCAA|1965-08-14|787|3424|263|1965|6|8|14|3|1965|263|3424|Saturday|1965Q3|N|Y|N|2438974|2439185|2438622|2438896|N|N|N|N|N| +2438988|AAAAAAAAMEHDFCAA|1965-08-15|787|3424|263|1965|0|8|15|3|1965|263|3424|Sunday|1965Q3|N|N|N|2438974|2439185|2438623|2438897|N|N|N|N|N| +2438989|AAAAAAAANEHDFCAA|1965-08-16|787|3424|263|1965|1|8|16|3|1965|263|3424|Monday|1965Q3|N|N|N|2438974|2439185|2438624|2438898|N|N|N|N|N| +2438990|AAAAAAAAOEHDFCAA|1965-08-17|787|3425|263|1965|2|8|17|3|1965|263|3425|Tuesday|1965Q3|N|N|N|2438974|2439185|2438625|2438899|N|N|N|N|N| +2438991|AAAAAAAAPEHDFCAA|1965-08-18|787|3425|263|1965|3|8|18|3|1965|263|3425|Wednesday|1965Q3|N|N|N|2438974|2439185|2438626|2438900|N|N|N|N|N| +2438992|AAAAAAAAAFHDFCAA|1965-08-19|787|3425|263|1965|4|8|19|3|1965|263|3425|Thursday|1965Q3|N|N|N|2438974|2439185|2438627|2438901|N|N|N|N|N| +2438993|AAAAAAAABFHDFCAA|1965-08-20|787|3425|263|1965|5|8|20|3|1965|263|3425|Friday|1965Q3|N|Y|N|2438974|2439185|2438628|2438902|N|N|N|N|N| +2438994|AAAAAAAACFHDFCAA|1965-08-21|787|3425|263|1965|6|8|21|3|1965|263|3425|Saturday|1965Q3|N|Y|N|2438974|2439185|2438629|2438903|N|N|N|N|N| +2438995|AAAAAAAADFHDFCAA|1965-08-22|787|3425|263|1965|0|8|22|3|1965|263|3425|Sunday|1965Q3|N|N|N|2438974|2439185|2438630|2438904|N|N|N|N|N| +2438996|AAAAAAAAEFHDFCAA|1965-08-23|787|3425|263|1965|1|8|23|3|1965|263|3425|Monday|1965Q3|N|N|N|2438974|2439185|2438631|2438905|N|N|N|N|N| +2438997|AAAAAAAAFFHDFCAA|1965-08-24|787|3426|263|1965|2|8|24|3|1965|263|3426|Tuesday|1965Q3|N|N|N|2438974|2439185|2438632|2438906|N|N|N|N|N| +2438998|AAAAAAAAGFHDFCAA|1965-08-25|787|3426|263|1965|3|8|25|3|1965|263|3426|Wednesday|1965Q3|N|N|N|2438974|2439185|2438633|2438907|N|N|N|N|N| +2438999|AAAAAAAAHFHDFCAA|1965-08-26|787|3426|263|1965|4|8|26|3|1965|263|3426|Thursday|1965Q3|N|N|N|2438974|2439185|2438634|2438908|N|N|N|N|N| +2439000|AAAAAAAAIFHDFCAA|1965-08-27|787|3426|263|1965|5|8|27|3|1965|263|3426|Friday|1965Q3|N|Y|N|2438974|2439185|2438635|2438909|N|N|N|N|N| +2439001|AAAAAAAAJFHDFCAA|1965-08-28|787|3426|263|1965|6|8|28|3|1965|263|3426|Saturday|1965Q3|N|Y|N|2438974|2439185|2438636|2438910|N|N|N|N|N| +2439002|AAAAAAAAKFHDFCAA|1965-08-29|787|3426|263|1965|0|8|29|3|1965|263|3426|Sunday|1965Q3|N|N|N|2438974|2439185|2438637|2438911|N|N|N|N|N| +2439003|AAAAAAAALFHDFCAA|1965-08-30|787|3426|263|1965|1|8|30|3|1965|263|3426|Monday|1965Q3|N|N|N|2438974|2439185|2438638|2438912|N|N|N|N|N| +2439004|AAAAAAAAMFHDFCAA|1965-08-31|787|3427|263|1965|2|8|31|3|1965|263|3427|Tuesday|1965Q3|N|N|N|2438974|2439185|2438639|2438913|N|N|N|N|N| +2439005|AAAAAAAANFHDFCAA|1965-09-01|788|3427|264|1965|3|9|1|3|1965|264|3427|Wednesday|1965Q3|N|N|N|2439005|2439247|2438640|2438914|N|N|N|N|N| +2439006|AAAAAAAAOFHDFCAA|1965-09-02|788|3427|264|1965|4|9|2|3|1965|264|3427|Thursday|1965Q3|N|N|N|2439005|2439247|2438641|2438915|N|N|N|N|N| +2439007|AAAAAAAAPFHDFCAA|1965-09-03|788|3427|264|1965|5|9|3|3|1965|264|3427|Friday|1965Q3|N|Y|N|2439005|2439247|2438642|2438916|N|N|N|N|N| +2439008|AAAAAAAAAGHDFCAA|1965-09-04|788|3427|264|1965|6|9|4|3|1965|264|3427|Saturday|1965Q3|N|Y|N|2439005|2439247|2438643|2438917|N|N|N|N|N| +2439009|AAAAAAAABGHDFCAA|1965-09-05|788|3427|264|1965|0|9|5|3|1965|264|3427|Sunday|1965Q3|N|N|N|2439005|2439247|2438644|2438918|N|N|N|N|N| +2439010|AAAAAAAACGHDFCAA|1965-09-06|788|3427|264|1965|1|9|6|3|1965|264|3427|Monday|1965Q3|N|N|N|2439005|2439247|2438645|2438919|N|N|N|N|N| +2439011|AAAAAAAADGHDFCAA|1965-09-07|788|3428|264|1965|2|9|7|3|1965|264|3428|Tuesday|1965Q3|N|N|N|2439005|2439247|2438646|2438920|N|N|N|N|N| +2439012|AAAAAAAAEGHDFCAA|1965-09-08|788|3428|264|1965|3|9|8|3|1965|264|3428|Wednesday|1965Q3|N|N|N|2439005|2439247|2438647|2438921|N|N|N|N|N| +2439013|AAAAAAAAFGHDFCAA|1965-09-09|788|3428|264|1965|4|9|9|3|1965|264|3428|Thursday|1965Q3|N|N|N|2439005|2439247|2438648|2438922|N|N|N|N|N| +2439014|AAAAAAAAGGHDFCAA|1965-09-10|788|3428|264|1965|5|9|10|3|1965|264|3428|Friday|1965Q3|N|Y|N|2439005|2439247|2438649|2438923|N|N|N|N|N| +2439015|AAAAAAAAHGHDFCAA|1965-09-11|788|3428|264|1965|6|9|11|3|1965|264|3428|Saturday|1965Q3|N|Y|N|2439005|2439247|2438650|2438924|N|N|N|N|N| +2439016|AAAAAAAAIGHDFCAA|1965-09-12|788|3428|264|1965|0|9|12|3|1965|264|3428|Sunday|1965Q3|N|N|N|2439005|2439247|2438651|2438925|N|N|N|N|N| +2439017|AAAAAAAAJGHDFCAA|1965-09-13|788|3428|264|1965|1|9|13|3|1965|264|3428|Monday|1965Q3|N|N|N|2439005|2439247|2438652|2438926|N|N|N|N|N| +2439018|AAAAAAAAKGHDFCAA|1965-09-14|788|3429|264|1965|2|9|14|3|1965|264|3429|Tuesday|1965Q3|N|N|N|2439005|2439247|2438653|2438927|N|N|N|N|N| +2439019|AAAAAAAALGHDFCAA|1965-09-15|788|3429|264|1965|3|9|15|3|1965|264|3429|Wednesday|1965Q3|N|N|N|2439005|2439247|2438654|2438928|N|N|N|N|N| +2439020|AAAAAAAAMGHDFCAA|1965-09-16|788|3429|264|1965|4|9|16|3|1965|264|3429|Thursday|1965Q3|N|N|N|2439005|2439247|2438655|2438929|N|N|N|N|N| +2439021|AAAAAAAANGHDFCAA|1965-09-17|788|3429|264|1965|5|9|17|3|1965|264|3429|Friday|1965Q3|N|Y|N|2439005|2439247|2438656|2438930|N|N|N|N|N| +2439022|AAAAAAAAOGHDFCAA|1965-09-18|788|3429|264|1965|6|9|18|3|1965|264|3429|Saturday|1965Q3|N|Y|N|2439005|2439247|2438657|2438931|N|N|N|N|N| +2439023|AAAAAAAAPGHDFCAA|1965-09-19|788|3429|264|1965|0|9|19|3|1965|264|3429|Sunday|1965Q3|N|N|N|2439005|2439247|2438658|2438932|N|N|N|N|N| +2439024|AAAAAAAAAHHDFCAA|1965-09-20|788|3429|264|1965|1|9|20|3|1965|264|3429|Monday|1965Q3|N|N|N|2439005|2439247|2438659|2438933|N|N|N|N|N| +2439025|AAAAAAAABHHDFCAA|1965-09-21|788|3430|264|1965|2|9|21|3|1965|264|3430|Tuesday|1965Q3|N|N|N|2439005|2439247|2438660|2438934|N|N|N|N|N| +2439026|AAAAAAAACHHDFCAA|1965-09-22|788|3430|264|1965|3|9|22|3|1965|264|3430|Wednesday|1965Q3|N|N|N|2439005|2439247|2438661|2438935|N|N|N|N|N| +2439027|AAAAAAAADHHDFCAA|1965-09-23|788|3430|264|1965|4|9|23|3|1965|264|3430|Thursday|1965Q3|N|N|N|2439005|2439247|2438662|2438936|N|N|N|N|N| +2439028|AAAAAAAAEHHDFCAA|1965-09-24|788|3430|264|1965|5|9|24|3|1965|264|3430|Friday|1965Q3|N|Y|N|2439005|2439247|2438663|2438937|N|N|N|N|N| +2439029|AAAAAAAAFHHDFCAA|1965-09-25|788|3430|264|1965|6|9|25|3|1965|264|3430|Saturday|1965Q3|N|Y|N|2439005|2439247|2438664|2438938|N|N|N|N|N| +2439030|AAAAAAAAGHHDFCAA|1965-09-26|788|3430|264|1965|0|9|26|3|1965|264|3430|Sunday|1965Q3|N|N|N|2439005|2439247|2438665|2438939|N|N|N|N|N| +2439031|AAAAAAAAHHHDFCAA|1965-09-27|788|3430|264|1965|1|9|27|3|1965|264|3430|Monday|1965Q3|N|N|N|2439005|2439247|2438666|2438940|N|N|N|N|N| +2439032|AAAAAAAAIHHDFCAA|1965-09-28|788|3431|264|1965|2|9|28|3|1965|264|3431|Tuesday|1965Q3|N|N|N|2439005|2439247|2438667|2438941|N|N|N|N|N| +2439033|AAAAAAAAJHHDFCAA|1965-09-29|788|3431|264|1965|3|9|29|3|1965|264|3431|Wednesday|1965Q3|N|N|N|2439005|2439247|2438668|2438942|N|N|N|N|N| +2439034|AAAAAAAAKHHDFCAA|1965-09-30|788|3431|264|1965|4|9|30|3|1965|264|3431|Thursday|1965Q3|N|N|N|2439005|2439247|2438669|2438943|N|N|N|N|N| +2439035|AAAAAAAALHHDFCAA|1965-10-01|789|3431|264|1965|5|10|1|3|1965|264|3431|Friday|1965Q3|N|Y|N|2439035|2439307|2438670|2438943|N|N|N|N|N| +2439036|AAAAAAAAMHHDFCAA|1965-10-02|789|3431|264|1965|6|10|2|4|1965|264|3431|Saturday|1965Q4|N|Y|N|2439035|2439307|2438671|2438944|N|N|N|N|N| +2439037|AAAAAAAANHHDFCAA|1965-10-03|789|3431|264|1965|0|10|3|4|1965|264|3431|Sunday|1965Q4|N|N|N|2439035|2439307|2438672|2438945|N|N|N|N|N| +2439038|AAAAAAAAOHHDFCAA|1965-10-04|789|3431|264|1965|1|10|4|4|1965|264|3431|Monday|1965Q4|N|N|N|2439035|2439307|2438673|2438946|N|N|N|N|N| +2439039|AAAAAAAAPHHDFCAA|1965-10-05|789|3432|264|1965|2|10|5|4|1965|264|3432|Tuesday|1965Q4|N|N|N|2439035|2439307|2438674|2438947|N|N|N|N|N| +2439040|AAAAAAAAAIHDFCAA|1965-10-06|789|3432|264|1965|3|10|6|4|1965|264|3432|Wednesday|1965Q4|N|N|N|2439035|2439307|2438675|2438948|N|N|N|N|N| +2439041|AAAAAAAABIHDFCAA|1965-10-07|789|3432|264|1965|4|10|7|4|1965|264|3432|Thursday|1965Q4|N|N|N|2439035|2439307|2438676|2438949|N|N|N|N|N| +2439042|AAAAAAAACIHDFCAA|1965-10-08|789|3432|264|1965|5|10|8|4|1965|264|3432|Friday|1965Q4|N|Y|N|2439035|2439307|2438677|2438950|N|N|N|N|N| +2439043|AAAAAAAADIHDFCAA|1965-10-09|789|3432|264|1965|6|10|9|4|1965|264|3432|Saturday|1965Q4|N|Y|N|2439035|2439307|2438678|2438951|N|N|N|N|N| +2439044|AAAAAAAAEIHDFCAA|1965-10-10|789|3432|264|1965|0|10|10|4|1965|264|3432|Sunday|1965Q4|N|N|N|2439035|2439307|2438679|2438952|N|N|N|N|N| +2439045|AAAAAAAAFIHDFCAA|1965-10-11|789|3432|264|1965|1|10|11|4|1965|264|3432|Monday|1965Q4|N|N|N|2439035|2439307|2438680|2438953|N|N|N|N|N| +2439046|AAAAAAAAGIHDFCAA|1965-10-12|789|3433|264|1965|2|10|12|4|1965|264|3433|Tuesday|1965Q4|N|N|N|2439035|2439307|2438681|2438954|N|N|N|N|N| +2439047|AAAAAAAAHIHDFCAA|1965-10-13|789|3433|264|1965|3|10|13|4|1965|264|3433|Wednesday|1965Q4|N|N|N|2439035|2439307|2438682|2438955|N|N|N|N|N| +2439048|AAAAAAAAIIHDFCAA|1965-10-14|789|3433|264|1965|4|10|14|4|1965|264|3433|Thursday|1965Q4|N|N|N|2439035|2439307|2438683|2438956|N|N|N|N|N| +2439049|AAAAAAAAJIHDFCAA|1965-10-15|789|3433|264|1965|5|10|15|4|1965|264|3433|Friday|1965Q4|N|Y|N|2439035|2439307|2438684|2438957|N|N|N|N|N| +2439050|AAAAAAAAKIHDFCAA|1965-10-16|789|3433|264|1965|6|10|16|4|1965|264|3433|Saturday|1965Q4|N|Y|N|2439035|2439307|2438685|2438958|N|N|N|N|N| +2439051|AAAAAAAALIHDFCAA|1965-10-17|789|3433|264|1965|0|10|17|4|1965|264|3433|Sunday|1965Q4|N|N|N|2439035|2439307|2438686|2438959|N|N|N|N|N| +2439052|AAAAAAAAMIHDFCAA|1965-10-18|789|3433|264|1965|1|10|18|4|1965|264|3433|Monday|1965Q4|N|N|N|2439035|2439307|2438687|2438960|N|N|N|N|N| +2439053|AAAAAAAANIHDFCAA|1965-10-19|789|3434|264|1965|2|10|19|4|1965|264|3434|Tuesday|1965Q4|N|N|N|2439035|2439307|2438688|2438961|N|N|N|N|N| +2439054|AAAAAAAAOIHDFCAA|1965-10-20|789|3434|264|1965|3|10|20|4|1965|264|3434|Wednesday|1965Q4|N|N|N|2439035|2439307|2438689|2438962|N|N|N|N|N| +2439055|AAAAAAAAPIHDFCAA|1965-10-21|789|3434|264|1965|4|10|21|4|1965|264|3434|Thursday|1965Q4|N|N|N|2439035|2439307|2438690|2438963|N|N|N|N|N| +2439056|AAAAAAAAAJHDFCAA|1965-10-22|789|3434|264|1965|5|10|22|4|1965|264|3434|Friday|1965Q4|N|Y|N|2439035|2439307|2438691|2438964|N|N|N|N|N| +2439057|AAAAAAAABJHDFCAA|1965-10-23|789|3434|264|1965|6|10|23|4|1965|264|3434|Saturday|1965Q4|N|Y|N|2439035|2439307|2438692|2438965|N|N|N|N|N| +2439058|AAAAAAAACJHDFCAA|1965-10-24|789|3434|264|1965|0|10|24|4|1965|264|3434|Sunday|1965Q4|N|N|N|2439035|2439307|2438693|2438966|N|N|N|N|N| +2439059|AAAAAAAADJHDFCAA|1965-10-25|789|3434|264|1965|1|10|25|4|1965|264|3434|Monday|1965Q4|N|N|N|2439035|2439307|2438694|2438967|N|N|N|N|N| +2439060|AAAAAAAAEJHDFCAA|1965-10-26|789|3435|264|1965|2|10|26|4|1965|264|3435|Tuesday|1965Q4|N|N|N|2439035|2439307|2438695|2438968|N|N|N|N|N| +2439061|AAAAAAAAFJHDFCAA|1965-10-27|789|3435|264|1965|3|10|27|4|1965|264|3435|Wednesday|1965Q4|N|N|N|2439035|2439307|2438696|2438969|N|N|N|N|N| +2439062|AAAAAAAAGJHDFCAA|1965-10-28|789|3435|264|1965|4|10|28|4|1965|264|3435|Thursday|1965Q4|N|N|N|2439035|2439307|2438697|2438970|N|N|N|N|N| +2439063|AAAAAAAAHJHDFCAA|1965-10-29|789|3435|264|1965|5|10|29|4|1965|264|3435|Friday|1965Q4|N|Y|N|2439035|2439307|2438698|2438971|N|N|N|N|N| +2439064|AAAAAAAAIJHDFCAA|1965-10-30|789|3435|264|1965|6|10|30|4|1965|264|3435|Saturday|1965Q4|N|Y|N|2439035|2439307|2438699|2438972|N|N|N|N|N| +2439065|AAAAAAAAJJHDFCAA|1965-10-31|789|3435|264|1965|0|10|31|4|1965|264|3435|Sunday|1965Q4|N|N|N|2439035|2439307|2438700|2438973|N|N|N|N|N| +2439066|AAAAAAAAKJHDFCAA|1965-11-01|790|3435|264|1965|1|11|1|4|1965|264|3435|Monday|1965Q4|N|N|N|2439066|2439369|2438701|2438974|N|N|N|N|N| +2439067|AAAAAAAALJHDFCAA|1965-11-02|790|3436|264|1965|2|11|2|4|1965|264|3436|Tuesday|1965Q4|N|N|N|2439066|2439369|2438702|2438975|N|N|N|N|N| +2439068|AAAAAAAAMJHDFCAA|1965-11-03|790|3436|264|1965|3|11|3|4|1965|264|3436|Wednesday|1965Q4|N|N|N|2439066|2439369|2438703|2438976|N|N|N|N|N| +2439069|AAAAAAAANJHDFCAA|1965-11-04|790|3436|264|1965|4|11|4|4|1965|264|3436|Thursday|1965Q4|N|N|N|2439066|2439369|2438704|2438977|N|N|N|N|N| +2439070|AAAAAAAAOJHDFCAA|1965-11-05|790|3436|264|1965|5|11|5|4|1965|264|3436|Friday|1965Q4|N|Y|N|2439066|2439369|2438705|2438978|N|N|N|N|N| +2439071|AAAAAAAAPJHDFCAA|1965-11-06|790|3436|264|1965|6|11|6|4|1965|264|3436|Saturday|1965Q4|N|Y|N|2439066|2439369|2438706|2438979|N|N|N|N|N| +2439072|AAAAAAAAAKHDFCAA|1965-11-07|790|3436|264|1965|0|11|7|4|1965|264|3436|Sunday|1965Q4|N|N|N|2439066|2439369|2438707|2438980|N|N|N|N|N| +2439073|AAAAAAAABKHDFCAA|1965-11-08|790|3436|264|1965|1|11|8|4|1965|264|3436|Monday|1965Q4|N|N|N|2439066|2439369|2438708|2438981|N|N|N|N|N| +2439074|AAAAAAAACKHDFCAA|1965-11-09|790|3437|264|1965|2|11|9|4|1965|264|3437|Tuesday|1965Q4|N|N|N|2439066|2439369|2438709|2438982|N|N|N|N|N| +2439075|AAAAAAAADKHDFCAA|1965-11-10|790|3437|264|1965|3|11|10|4|1965|264|3437|Wednesday|1965Q4|N|N|N|2439066|2439369|2438710|2438983|N|N|N|N|N| +2439076|AAAAAAAAEKHDFCAA|1965-11-11|790|3437|264|1965|4|11|11|4|1965|264|3437|Thursday|1965Q4|N|N|N|2439066|2439369|2438711|2438984|N|N|N|N|N| +2439077|AAAAAAAAFKHDFCAA|1965-11-12|790|3437|264|1965|5|11|12|4|1965|264|3437|Friday|1965Q4|N|Y|N|2439066|2439369|2438712|2438985|N|N|N|N|N| +2439078|AAAAAAAAGKHDFCAA|1965-11-13|790|3437|264|1965|6|11|13|4|1965|264|3437|Saturday|1965Q4|N|Y|N|2439066|2439369|2438713|2438986|N|N|N|N|N| +2439079|AAAAAAAAHKHDFCAA|1965-11-14|790|3437|264|1965|0|11|14|4|1965|264|3437|Sunday|1965Q4|N|N|N|2439066|2439369|2438714|2438987|N|N|N|N|N| +2439080|AAAAAAAAIKHDFCAA|1965-11-15|790|3437|264|1965|1|11|15|4|1965|264|3437|Monday|1965Q4|N|N|N|2439066|2439369|2438715|2438988|N|N|N|N|N| +2439081|AAAAAAAAJKHDFCAA|1965-11-16|790|3438|264|1965|2|11|16|4|1965|264|3438|Tuesday|1965Q4|N|N|N|2439066|2439369|2438716|2438989|N|N|N|N|N| +2439082|AAAAAAAAKKHDFCAA|1965-11-17|790|3438|264|1965|3|11|17|4|1965|264|3438|Wednesday|1965Q4|N|N|N|2439066|2439369|2438717|2438990|N|N|N|N|N| +2439083|AAAAAAAALKHDFCAA|1965-11-18|790|3438|264|1965|4|11|18|4|1965|264|3438|Thursday|1965Q4|N|N|N|2439066|2439369|2438718|2438991|N|N|N|N|N| +2439084|AAAAAAAAMKHDFCAA|1965-11-19|790|3438|264|1965|5|11|19|4|1965|264|3438|Friday|1965Q4|N|Y|N|2439066|2439369|2438719|2438992|N|N|N|N|N| +2439085|AAAAAAAANKHDFCAA|1965-11-20|790|3438|264|1965|6|11|20|4|1965|264|3438|Saturday|1965Q4|N|Y|N|2439066|2439369|2438720|2438993|N|N|N|N|N| +2439086|AAAAAAAAOKHDFCAA|1965-11-21|790|3438|264|1965|0|11|21|4|1965|264|3438|Sunday|1965Q4|N|N|N|2439066|2439369|2438721|2438994|N|N|N|N|N| +2439087|AAAAAAAAPKHDFCAA|1965-11-22|790|3438|264|1965|1|11|22|4|1965|264|3438|Monday|1965Q4|N|N|N|2439066|2439369|2438722|2438995|N|N|N|N|N| +2439088|AAAAAAAAALHDFCAA|1965-11-23|790|3439|264|1965|2|11|23|4|1965|264|3439|Tuesday|1965Q4|N|N|N|2439066|2439369|2438723|2438996|N|N|N|N|N| +2439089|AAAAAAAABLHDFCAA|1965-11-24|790|3439|264|1965|3|11|24|4|1965|264|3439|Wednesday|1965Q4|N|N|N|2439066|2439369|2438724|2438997|N|N|N|N|N| +2439090|AAAAAAAACLHDFCAA|1965-11-25|790|3439|264|1965|4|11|25|4|1965|264|3439|Thursday|1965Q4|N|N|N|2439066|2439369|2438725|2438998|N|N|N|N|N| +2439091|AAAAAAAADLHDFCAA|1965-11-26|790|3439|264|1965|5|11|26|4|1965|264|3439|Friday|1965Q4|N|Y|N|2439066|2439369|2438726|2438999|N|N|N|N|N| +2439092|AAAAAAAAELHDFCAA|1965-11-27|790|3439|264|1965|6|11|27|4|1965|264|3439|Saturday|1965Q4|N|Y|N|2439066|2439369|2438727|2439000|N|N|N|N|N| +2439093|AAAAAAAAFLHDFCAA|1965-11-28|790|3439|264|1965|0|11|28|4|1965|264|3439|Sunday|1965Q4|N|N|N|2439066|2439369|2438728|2439001|N|N|N|N|N| +2439094|AAAAAAAAGLHDFCAA|1965-11-29|790|3439|264|1965|1|11|29|4|1965|264|3439|Monday|1965Q4|N|N|N|2439066|2439369|2438729|2439002|N|N|N|N|N| +2439095|AAAAAAAAHLHDFCAA|1965-11-30|790|3440|264|1965|2|11|30|4|1965|264|3440|Tuesday|1965Q4|N|N|N|2439066|2439369|2438730|2439003|N|N|N|N|N| +2439096|AAAAAAAAILHDFCAA|1965-12-01|791|3440|265|1965|3|12|1|4|1965|265|3440|Wednesday|1965Q4|N|N|N|2439096|2439429|2438731|2439004|N|N|N|N|N| +2439097|AAAAAAAAJLHDFCAA|1965-12-02|791|3440|265|1965|4|12|2|4|1965|265|3440|Thursday|1965Q4|N|N|N|2439096|2439429|2438732|2439005|N|N|N|N|N| +2439098|AAAAAAAAKLHDFCAA|1965-12-03|791|3440|265|1965|5|12|3|4|1965|265|3440|Friday|1965Q4|N|Y|N|2439096|2439429|2438733|2439006|N|N|N|N|N| +2439099|AAAAAAAALLHDFCAA|1965-12-04|791|3440|265|1965|6|12|4|4|1965|265|3440|Saturday|1965Q4|N|Y|N|2439096|2439429|2438734|2439007|N|N|N|N|N| +2439100|AAAAAAAAMLHDFCAA|1965-12-05|791|3440|265|1965|0|12|5|4|1965|265|3440|Sunday|1965Q4|N|N|N|2439096|2439429|2438735|2439008|N|N|N|N|N| +2439101|AAAAAAAANLHDFCAA|1965-12-06|791|3440|265|1965|1|12|6|4|1965|265|3440|Monday|1965Q4|N|N|N|2439096|2439429|2438736|2439009|N|N|N|N|N| +2439102|AAAAAAAAOLHDFCAA|1965-12-07|791|3441|265|1965|2|12|7|4|1965|265|3441|Tuesday|1965Q4|N|N|N|2439096|2439429|2438737|2439010|N|N|N|N|N| +2439103|AAAAAAAAPLHDFCAA|1965-12-08|791|3441|265|1965|3|12|8|4|1965|265|3441|Wednesday|1965Q4|N|N|N|2439096|2439429|2438738|2439011|N|N|N|N|N| +2439104|AAAAAAAAAMHDFCAA|1965-12-09|791|3441|265|1965|4|12|9|4|1965|265|3441|Thursday|1965Q4|N|N|N|2439096|2439429|2438739|2439012|N|N|N|N|N| +2439105|AAAAAAAABMHDFCAA|1965-12-10|791|3441|265|1965|5|12|10|4|1965|265|3441|Friday|1965Q4|N|Y|N|2439096|2439429|2438740|2439013|N|N|N|N|N| +2439106|AAAAAAAACMHDFCAA|1965-12-11|791|3441|265|1965|6|12|11|4|1965|265|3441|Saturday|1965Q4|N|Y|N|2439096|2439429|2438741|2439014|N|N|N|N|N| +2439107|AAAAAAAADMHDFCAA|1965-12-12|791|3441|265|1965|0|12|12|4|1965|265|3441|Sunday|1965Q4|N|N|N|2439096|2439429|2438742|2439015|N|N|N|N|N| +2439108|AAAAAAAAEMHDFCAA|1965-12-13|791|3441|265|1965|1|12|13|4|1965|265|3441|Monday|1965Q4|N|N|N|2439096|2439429|2438743|2439016|N|N|N|N|N| +2439109|AAAAAAAAFMHDFCAA|1965-12-14|791|3442|265|1965|2|12|14|4|1965|265|3442|Tuesday|1965Q4|N|N|N|2439096|2439429|2438744|2439017|N|N|N|N|N| +2439110|AAAAAAAAGMHDFCAA|1965-12-15|791|3442|265|1965|3|12|15|4|1965|265|3442|Wednesday|1965Q4|N|N|N|2439096|2439429|2438745|2439018|N|N|N|N|N| +2439111|AAAAAAAAHMHDFCAA|1965-12-16|791|3442|265|1965|4|12|16|4|1965|265|3442|Thursday|1965Q4|N|N|N|2439096|2439429|2438746|2439019|N|N|N|N|N| +2439112|AAAAAAAAIMHDFCAA|1965-12-17|791|3442|265|1965|5|12|17|4|1965|265|3442|Friday|1965Q4|N|Y|N|2439096|2439429|2438747|2439020|N|N|N|N|N| +2439113|AAAAAAAAJMHDFCAA|1965-12-18|791|3442|265|1965|6|12|18|4|1965|265|3442|Saturday|1965Q4|N|Y|N|2439096|2439429|2438748|2439021|N|N|N|N|N| +2439114|AAAAAAAAKMHDFCAA|1965-12-19|791|3442|265|1965|0|12|19|4|1965|265|3442|Sunday|1965Q4|N|N|N|2439096|2439429|2438749|2439022|N|N|N|N|N| +2439115|AAAAAAAALMHDFCAA|1965-12-20|791|3442|265|1965|1|12|20|4|1965|265|3442|Monday|1965Q4|N|N|N|2439096|2439429|2438750|2439023|N|N|N|N|N| +2439116|AAAAAAAAMMHDFCAA|1965-12-21|791|3443|265|1965|2|12|21|4|1965|265|3443|Tuesday|1965Q4|N|N|N|2439096|2439429|2438751|2439024|N|N|N|N|N| +2439117|AAAAAAAANMHDFCAA|1965-12-22|791|3443|265|1965|3|12|22|4|1965|265|3443|Wednesday|1965Q4|N|N|N|2439096|2439429|2438752|2439025|N|N|N|N|N| +2439118|AAAAAAAAOMHDFCAA|1965-12-23|791|3443|265|1965|4|12|23|4|1965|265|3443|Thursday|1965Q4|N|N|N|2439096|2439429|2438753|2439026|N|N|N|N|N| +2439119|AAAAAAAAPMHDFCAA|1965-12-24|791|3443|265|1965|5|12|24|4|1965|265|3443|Friday|1965Q4|N|Y|N|2439096|2439429|2438754|2439027|N|N|N|N|N| +2439120|AAAAAAAAANHDFCAA|1965-12-25|791|3443|265|1965|6|12|25|4|1965|265|3443|Saturday|1965Q4|N|Y|N|2439096|2439429|2438755|2439028|N|N|N|N|N| +2439121|AAAAAAAABNHDFCAA|1965-12-26|791|3443|265|1965|0|12|26|4|1965|265|3443|Sunday|1965Q4|Y|N|N|2439096|2439429|2438756|2439029|N|N|N|N|N| +2439122|AAAAAAAACNHDFCAA|1965-12-27|791|3443|265|1965|1|12|27|4|1965|265|3443|Monday|1965Q4|N|N|Y|2439096|2439429|2438757|2439030|N|N|N|N|N| +2439123|AAAAAAAADNHDFCAA|1965-12-28|791|3444|265|1965|2|12|28|4|1965|265|3444|Tuesday|1965Q4|N|N|N|2439096|2439429|2438758|2439031|N|N|N|N|N| +2439124|AAAAAAAAENHDFCAA|1965-12-29|791|3444|265|1965|3|12|29|4|1965|265|3444|Wednesday|1965Q4|N|N|N|2439096|2439429|2438759|2439032|N|N|N|N|N| +2439125|AAAAAAAAFNHDFCAA|1965-12-30|791|3444|265|1965|4|12|30|4|1965|265|3444|Thursday|1965Q4|N|N|N|2439096|2439429|2438760|2439033|N|N|N|N|N| +2439126|AAAAAAAAGNHDFCAA|1965-12-31|791|3444|265|1965|5|12|31|4|1965|265|3444|Friday|1965Q4|N|Y|N|2439096|2439429|2438761|2439034|N|N|N|N|N| +2439127|AAAAAAAAHNHDFCAA|1966-01-01|792|3444|265|1966|6|1|1|1|1966|265|3444|Saturday|1966Q1|Y|Y|N|2439127|2439126|2438762|2439035|N|N|N|N|N| +2439128|AAAAAAAAINHDFCAA|1966-01-02|792|3444|265|1966|0|1|2|1|1966|265|3444|Sunday|1966Q1|N|N|Y|2439127|2439126|2438763|2439036|N|N|N|N|N| +2439129|AAAAAAAAJNHDFCAA|1966-01-03|792|3444|265|1966|1|1|3|1|1966|265|3444|Monday|1966Q1|N|N|N|2439127|2439126|2438764|2439037|N|N|N|N|N| +2439130|AAAAAAAAKNHDFCAA|1966-01-04|792|3445|265|1966|2|1|4|1|1966|265|3445|Tuesday|1966Q1|N|N|N|2439127|2439126|2438765|2439038|N|N|N|N|N| +2439131|AAAAAAAALNHDFCAA|1966-01-05|792|3445|265|1966|3|1|5|1|1966|265|3445|Wednesday|1966Q1|N|N|N|2439127|2439126|2438766|2439039|N|N|N|N|N| +2439132|AAAAAAAAMNHDFCAA|1966-01-06|792|3445|265|1966|4|1|6|1|1966|265|3445|Thursday|1966Q1|N|N|N|2439127|2439126|2438767|2439040|N|N|N|N|N| +2439133|AAAAAAAANNHDFCAA|1966-01-07|792|3445|265|1966|5|1|7|1|1966|265|3445|Friday|1966Q1|N|Y|N|2439127|2439126|2438768|2439041|N|N|N|N|N| +2439134|AAAAAAAAONHDFCAA|1966-01-08|792|3445|265|1966|6|1|8|1|1966|265|3445|Saturday|1966Q1|N|Y|N|2439127|2439126|2438769|2439042|N|N|N|N|N| +2439135|AAAAAAAAPNHDFCAA|1966-01-09|792|3445|265|1966|0|1|9|1|1966|265|3445|Sunday|1966Q1|N|N|N|2439127|2439126|2438770|2439043|N|N|N|N|N| +2439136|AAAAAAAAAOHDFCAA|1966-01-10|792|3445|265|1966|1|1|10|1|1966|265|3445|Monday|1966Q1|N|N|N|2439127|2439126|2438771|2439044|N|N|N|N|N| +2439137|AAAAAAAABOHDFCAA|1966-01-11|792|3446|265|1966|2|1|11|1|1966|265|3446|Tuesday|1966Q1|N|N|N|2439127|2439126|2438772|2439045|N|N|N|N|N| +2439138|AAAAAAAACOHDFCAA|1966-01-12|792|3446|265|1966|3|1|12|1|1966|265|3446|Wednesday|1966Q1|N|N|N|2439127|2439126|2438773|2439046|N|N|N|N|N| +2439139|AAAAAAAADOHDFCAA|1966-01-13|792|3446|265|1966|4|1|13|1|1966|265|3446|Thursday|1966Q1|N|N|N|2439127|2439126|2438774|2439047|N|N|N|N|N| +2439140|AAAAAAAAEOHDFCAA|1966-01-14|792|3446|265|1966|5|1|14|1|1966|265|3446|Friday|1966Q1|N|Y|N|2439127|2439126|2438775|2439048|N|N|N|N|N| +2439141|AAAAAAAAFOHDFCAA|1966-01-15|792|3446|265|1966|6|1|15|1|1966|265|3446|Saturday|1966Q1|N|Y|N|2439127|2439126|2438776|2439049|N|N|N|N|N| +2439142|AAAAAAAAGOHDFCAA|1966-01-16|792|3446|265|1966|0|1|16|1|1966|265|3446|Sunday|1966Q1|N|N|N|2439127|2439126|2438777|2439050|N|N|N|N|N| +2439143|AAAAAAAAHOHDFCAA|1966-01-17|792|3446|265|1966|1|1|17|1|1966|265|3446|Monday|1966Q1|N|N|N|2439127|2439126|2438778|2439051|N|N|N|N|N| +2439144|AAAAAAAAIOHDFCAA|1966-01-18|792|3447|265|1966|2|1|18|1|1966|265|3447|Tuesday|1966Q1|N|N|N|2439127|2439126|2438779|2439052|N|N|N|N|N| +2439145|AAAAAAAAJOHDFCAA|1966-01-19|792|3447|265|1966|3|1|19|1|1966|265|3447|Wednesday|1966Q1|N|N|N|2439127|2439126|2438780|2439053|N|N|N|N|N| +2439146|AAAAAAAAKOHDFCAA|1966-01-20|792|3447|265|1966|4|1|20|1|1966|265|3447|Thursday|1966Q1|N|N|N|2439127|2439126|2438781|2439054|N|N|N|N|N| +2439147|AAAAAAAALOHDFCAA|1966-01-21|792|3447|265|1966|5|1|21|1|1966|265|3447|Friday|1966Q1|N|Y|N|2439127|2439126|2438782|2439055|N|N|N|N|N| +2439148|AAAAAAAAMOHDFCAA|1966-01-22|792|3447|265|1966|6|1|22|1|1966|265|3447|Saturday|1966Q1|N|Y|N|2439127|2439126|2438783|2439056|N|N|N|N|N| +2439149|AAAAAAAANOHDFCAA|1966-01-23|792|3447|265|1966|0|1|23|1|1966|265|3447|Sunday|1966Q1|N|N|N|2439127|2439126|2438784|2439057|N|N|N|N|N| +2439150|AAAAAAAAOOHDFCAA|1966-01-24|792|3447|265|1966|1|1|24|1|1966|265|3447|Monday|1966Q1|N|N|N|2439127|2439126|2438785|2439058|N|N|N|N|N| +2439151|AAAAAAAAPOHDFCAA|1966-01-25|792|3448|265|1966|2|1|25|1|1966|265|3448|Tuesday|1966Q1|N|N|N|2439127|2439126|2438786|2439059|N|N|N|N|N| +2439152|AAAAAAAAAPHDFCAA|1966-01-26|792|3448|265|1966|3|1|26|1|1966|265|3448|Wednesday|1966Q1|N|N|N|2439127|2439126|2438787|2439060|N|N|N|N|N| +2439153|AAAAAAAABPHDFCAA|1966-01-27|792|3448|265|1966|4|1|27|1|1966|265|3448|Thursday|1966Q1|N|N|N|2439127|2439126|2438788|2439061|N|N|N|N|N| +2439154|AAAAAAAACPHDFCAA|1966-01-28|792|3448|265|1966|5|1|28|1|1966|265|3448|Friday|1966Q1|N|Y|N|2439127|2439126|2438789|2439062|N|N|N|N|N| +2439155|AAAAAAAADPHDFCAA|1966-01-29|792|3448|265|1966|6|1|29|1|1966|265|3448|Saturday|1966Q1|N|Y|N|2439127|2439126|2438790|2439063|N|N|N|N|N| +2439156|AAAAAAAAEPHDFCAA|1966-01-30|792|3448|265|1966|0|1|30|1|1966|265|3448|Sunday|1966Q1|N|N|N|2439127|2439126|2438791|2439064|N|N|N|N|N| +2439157|AAAAAAAAFPHDFCAA|1966-01-31|792|3448|265|1966|1|1|31|1|1966|265|3448|Monday|1966Q1|N|N|N|2439127|2439126|2438792|2439065|N|N|N|N|N| +2439158|AAAAAAAAGPHDFCAA|1966-02-01|793|3449|265|1966|2|2|1|1|1966|265|3449|Tuesday|1966Q1|N|N|N|2439158|2439188|2438793|2439066|N|N|N|N|N| +2439159|AAAAAAAAHPHDFCAA|1966-02-02|793|3449|265|1966|3|2|2|1|1966|265|3449|Wednesday|1966Q1|N|N|N|2439158|2439188|2438794|2439067|N|N|N|N|N| +2439160|AAAAAAAAIPHDFCAA|1966-02-03|793|3449|265|1966|4|2|3|1|1966|265|3449|Thursday|1966Q1|N|N|N|2439158|2439188|2438795|2439068|N|N|N|N|N| +2439161|AAAAAAAAJPHDFCAA|1966-02-04|793|3449|265|1966|5|2|4|1|1966|265|3449|Friday|1966Q1|N|Y|N|2439158|2439188|2438796|2439069|N|N|N|N|N| +2439162|AAAAAAAAKPHDFCAA|1966-02-05|793|3449|265|1966|6|2|5|1|1966|265|3449|Saturday|1966Q1|N|Y|N|2439158|2439188|2438797|2439070|N|N|N|N|N| +2439163|AAAAAAAALPHDFCAA|1966-02-06|793|3449|265|1966|0|2|6|1|1966|265|3449|Sunday|1966Q1|N|N|N|2439158|2439188|2438798|2439071|N|N|N|N|N| +2439164|AAAAAAAAMPHDFCAA|1966-02-07|793|3449|265|1966|1|2|7|1|1966|265|3449|Monday|1966Q1|N|N|N|2439158|2439188|2438799|2439072|N|N|N|N|N| +2439165|AAAAAAAANPHDFCAA|1966-02-08|793|3450|265|1966|2|2|8|1|1966|265|3450|Tuesday|1966Q1|N|N|N|2439158|2439188|2438800|2439073|N|N|N|N|N| +2439166|AAAAAAAAOPHDFCAA|1966-02-09|793|3450|265|1966|3|2|9|1|1966|265|3450|Wednesday|1966Q1|N|N|N|2439158|2439188|2438801|2439074|N|N|N|N|N| +2439167|AAAAAAAAPPHDFCAA|1966-02-10|793|3450|265|1966|4|2|10|1|1966|265|3450|Thursday|1966Q1|N|N|N|2439158|2439188|2438802|2439075|N|N|N|N|N| +2439168|AAAAAAAAAAIDFCAA|1966-02-11|793|3450|265|1966|5|2|11|1|1966|265|3450|Friday|1966Q1|N|Y|N|2439158|2439188|2438803|2439076|N|N|N|N|N| +2439169|AAAAAAAABAIDFCAA|1966-02-12|793|3450|265|1966|6|2|12|1|1966|265|3450|Saturday|1966Q1|N|Y|N|2439158|2439188|2438804|2439077|N|N|N|N|N| +2439170|AAAAAAAACAIDFCAA|1966-02-13|793|3450|265|1966|0|2|13|1|1966|265|3450|Sunday|1966Q1|N|N|N|2439158|2439188|2438805|2439078|N|N|N|N|N| +2439171|AAAAAAAADAIDFCAA|1966-02-14|793|3450|265|1966|1|2|14|1|1966|265|3450|Monday|1966Q1|N|N|N|2439158|2439188|2438806|2439079|N|N|N|N|N| +2439172|AAAAAAAAEAIDFCAA|1966-02-15|793|3451|265|1966|2|2|15|1|1966|265|3451|Tuesday|1966Q1|N|N|N|2439158|2439188|2438807|2439080|N|N|N|N|N| +2439173|AAAAAAAAFAIDFCAA|1966-02-16|793|3451|265|1966|3|2|16|1|1966|265|3451|Wednesday|1966Q1|N|N|N|2439158|2439188|2438808|2439081|N|N|N|N|N| +2439174|AAAAAAAAGAIDFCAA|1966-02-17|793|3451|265|1966|4|2|17|1|1966|265|3451|Thursday|1966Q1|N|N|N|2439158|2439188|2438809|2439082|N|N|N|N|N| +2439175|AAAAAAAAHAIDFCAA|1966-02-18|793|3451|265|1966|5|2|18|1|1966|265|3451|Friday|1966Q1|N|Y|N|2439158|2439188|2438810|2439083|N|N|N|N|N| +2439176|AAAAAAAAIAIDFCAA|1966-02-19|793|3451|265|1966|6|2|19|1|1966|265|3451|Saturday|1966Q1|N|Y|N|2439158|2439188|2438811|2439084|N|N|N|N|N| +2439177|AAAAAAAAJAIDFCAA|1966-02-20|793|3451|265|1966|0|2|20|1|1966|265|3451|Sunday|1966Q1|N|N|N|2439158|2439188|2438812|2439085|N|N|N|N|N| +2439178|AAAAAAAAKAIDFCAA|1966-02-21|793|3451|265|1966|1|2|21|1|1966|265|3451|Monday|1966Q1|N|N|N|2439158|2439188|2438813|2439086|N|N|N|N|N| +2439179|AAAAAAAALAIDFCAA|1966-02-22|793|3452|265|1966|2|2|22|1|1966|265|3452|Tuesday|1966Q1|N|N|N|2439158|2439188|2438814|2439087|N|N|N|N|N| +2439180|AAAAAAAAMAIDFCAA|1966-02-23|793|3452|265|1966|3|2|23|1|1966|265|3452|Wednesday|1966Q1|N|N|N|2439158|2439188|2438815|2439088|N|N|N|N|N| +2439181|AAAAAAAANAIDFCAA|1966-02-24|793|3452|265|1966|4|2|24|1|1966|265|3452|Thursday|1966Q1|N|N|N|2439158|2439188|2438816|2439089|N|N|N|N|N| +2439182|AAAAAAAAOAIDFCAA|1966-02-25|793|3452|265|1966|5|2|25|1|1966|265|3452|Friday|1966Q1|N|Y|N|2439158|2439188|2438817|2439090|N|N|N|N|N| +2439183|AAAAAAAAPAIDFCAA|1966-02-26|793|3452|265|1966|6|2|26|1|1966|265|3452|Saturday|1966Q1|N|Y|N|2439158|2439188|2438818|2439091|N|N|N|N|N| +2439184|AAAAAAAAABIDFCAA|1966-02-27|793|3452|265|1966|0|2|27|1|1966|265|3452|Sunday|1966Q1|N|N|N|2439158|2439188|2438819|2439092|N|N|N|N|N| +2439185|AAAAAAAABBIDFCAA|1966-02-28|793|3452|265|1966|1|2|28|1|1966|265|3452|Monday|1966Q1|N|N|N|2439158|2439188|2438820|2439093|N|N|N|N|N| +2439186|AAAAAAAACBIDFCAA|1966-03-01|794|3453|266|1966|2|3|1|1|1966|266|3453|Tuesday|1966Q1|N|N|N|2439186|2439244|2438821|2439094|N|N|N|N|N| +2439187|AAAAAAAADBIDFCAA|1966-03-02|794|3453|266|1966|3|3|2|1|1966|266|3453|Wednesday|1966Q1|N|N|N|2439186|2439244|2438822|2439095|N|N|N|N|N| +2439188|AAAAAAAAEBIDFCAA|1966-03-03|794|3453|266|1966|4|3|3|1|1966|266|3453|Thursday|1966Q1|N|N|N|2439186|2439244|2438823|2439096|N|N|N|N|N| +2439189|AAAAAAAAFBIDFCAA|1966-03-04|794|3453|266|1966|5|3|4|1|1966|266|3453|Friday|1966Q1|N|Y|N|2439186|2439244|2438824|2439097|N|N|N|N|N| +2439190|AAAAAAAAGBIDFCAA|1966-03-05|794|3453|266|1966|6|3|5|1|1966|266|3453|Saturday|1966Q1|N|Y|N|2439186|2439244|2438825|2439098|N|N|N|N|N| +2439191|AAAAAAAAHBIDFCAA|1966-03-06|794|3453|266|1966|0|3|6|1|1966|266|3453|Sunday|1966Q1|N|N|N|2439186|2439244|2438826|2439099|N|N|N|N|N| +2439192|AAAAAAAAIBIDFCAA|1966-03-07|794|3453|266|1966|1|3|7|1|1966|266|3453|Monday|1966Q1|N|N|N|2439186|2439244|2438827|2439100|N|N|N|N|N| +2439193|AAAAAAAAJBIDFCAA|1966-03-08|794|3454|266|1966|2|3|8|1|1966|266|3454|Tuesday|1966Q1|N|N|N|2439186|2439244|2438828|2439101|N|N|N|N|N| +2439194|AAAAAAAAKBIDFCAA|1966-03-09|794|3454|266|1966|3|3|9|1|1966|266|3454|Wednesday|1966Q1|N|N|N|2439186|2439244|2438829|2439102|N|N|N|N|N| +2439195|AAAAAAAALBIDFCAA|1966-03-10|794|3454|266|1966|4|3|10|1|1966|266|3454|Thursday|1966Q1|N|N|N|2439186|2439244|2438830|2439103|N|N|N|N|N| +2439196|AAAAAAAAMBIDFCAA|1966-03-11|794|3454|266|1966|5|3|11|1|1966|266|3454|Friday|1966Q1|N|Y|N|2439186|2439244|2438831|2439104|N|N|N|N|N| +2439197|AAAAAAAANBIDFCAA|1966-03-12|794|3454|266|1966|6|3|12|1|1966|266|3454|Saturday|1966Q1|N|Y|N|2439186|2439244|2438832|2439105|N|N|N|N|N| +2439198|AAAAAAAAOBIDFCAA|1966-03-13|794|3454|266|1966|0|3|13|1|1966|266|3454|Sunday|1966Q1|N|N|N|2439186|2439244|2438833|2439106|N|N|N|N|N| +2439199|AAAAAAAAPBIDFCAA|1966-03-14|794|3454|266|1966|1|3|14|1|1966|266|3454|Monday|1966Q1|N|N|N|2439186|2439244|2438834|2439107|N|N|N|N|N| +2439200|AAAAAAAAACIDFCAA|1966-03-15|794|3455|266|1966|2|3|15|1|1966|266|3455|Tuesday|1966Q1|N|N|N|2439186|2439244|2438835|2439108|N|N|N|N|N| +2439201|AAAAAAAABCIDFCAA|1966-03-16|794|3455|266|1966|3|3|16|1|1966|266|3455|Wednesday|1966Q1|N|N|N|2439186|2439244|2438836|2439109|N|N|N|N|N| +2439202|AAAAAAAACCIDFCAA|1966-03-17|794|3455|266|1966|4|3|17|1|1966|266|3455|Thursday|1966Q1|N|N|N|2439186|2439244|2438837|2439110|N|N|N|N|N| +2439203|AAAAAAAADCIDFCAA|1966-03-18|794|3455|266|1966|5|3|18|1|1966|266|3455|Friday|1966Q1|N|Y|N|2439186|2439244|2438838|2439111|N|N|N|N|N| +2439204|AAAAAAAAECIDFCAA|1966-03-19|794|3455|266|1966|6|3|19|1|1966|266|3455|Saturday|1966Q1|N|Y|N|2439186|2439244|2438839|2439112|N|N|N|N|N| +2439205|AAAAAAAAFCIDFCAA|1966-03-20|794|3455|266|1966|0|3|20|1|1966|266|3455|Sunday|1966Q1|N|N|N|2439186|2439244|2438840|2439113|N|N|N|N|N| +2439206|AAAAAAAAGCIDFCAA|1966-03-21|794|3455|266|1966|1|3|21|1|1966|266|3455|Monday|1966Q1|N|N|N|2439186|2439244|2438841|2439114|N|N|N|N|N| +2439207|AAAAAAAAHCIDFCAA|1966-03-22|794|3456|266|1966|2|3|22|1|1966|266|3456|Tuesday|1966Q1|N|N|N|2439186|2439244|2438842|2439115|N|N|N|N|N| +2439208|AAAAAAAAICIDFCAA|1966-03-23|794|3456|266|1966|3|3|23|1|1966|266|3456|Wednesday|1966Q1|N|N|N|2439186|2439244|2438843|2439116|N|N|N|N|N| +2439209|AAAAAAAAJCIDFCAA|1966-03-24|794|3456|266|1966|4|3|24|1|1966|266|3456|Thursday|1966Q1|N|N|N|2439186|2439244|2438844|2439117|N|N|N|N|N| +2439210|AAAAAAAAKCIDFCAA|1966-03-25|794|3456|266|1966|5|3|25|1|1966|266|3456|Friday|1966Q1|N|Y|N|2439186|2439244|2438845|2439118|N|N|N|N|N| +2439211|AAAAAAAALCIDFCAA|1966-03-26|794|3456|266|1966|6|3|26|1|1966|266|3456|Saturday|1966Q1|N|Y|N|2439186|2439244|2438846|2439119|N|N|N|N|N| +2439212|AAAAAAAAMCIDFCAA|1966-03-27|794|3456|266|1966|0|3|27|1|1966|266|3456|Sunday|1966Q1|N|N|N|2439186|2439244|2438847|2439120|N|N|N|N|N| +2439213|AAAAAAAANCIDFCAA|1966-03-28|794|3456|266|1966|1|3|28|1|1966|266|3456|Monday|1966Q1|N|N|N|2439186|2439244|2438848|2439121|N|N|N|N|N| +2439214|AAAAAAAAOCIDFCAA|1966-03-29|794|3457|266|1966|2|3|29|1|1966|266|3457|Tuesday|1966Q1|N|N|N|2439186|2439244|2438849|2439122|N|N|N|N|N| +2439215|AAAAAAAAPCIDFCAA|1966-03-30|794|3457|266|1966|3|3|30|1|1966|266|3457|Wednesday|1966Q1|N|N|N|2439186|2439244|2438850|2439123|N|N|N|N|N| +2439216|AAAAAAAAADIDFCAA|1966-03-31|794|3457|266|1966|4|3|31|1|1966|266|3457|Thursday|1966Q1|N|N|N|2439186|2439244|2438851|2439124|N|N|N|N|N| +2439217|AAAAAAAABDIDFCAA|1966-04-01|795|3457|266|1966|5|4|1|1|1966|266|3457|Friday|1966Q1|N|Y|N|2439217|2439306|2438852|2439127|N|N|N|N|N| +2439218|AAAAAAAACDIDFCAA|1966-04-02|795|3457|266|1966|6|4|2|2|1966|266|3457|Saturday|1966Q2|N|Y|N|2439217|2439306|2438853|2439128|N|N|N|N|N| +2439219|AAAAAAAADDIDFCAA|1966-04-03|795|3457|266|1966|0|4|3|2|1966|266|3457|Sunday|1966Q2|N|N|N|2439217|2439306|2438854|2439129|N|N|N|N|N| +2439220|AAAAAAAAEDIDFCAA|1966-04-04|795|3457|266|1966|1|4|4|2|1966|266|3457|Monday|1966Q2|N|N|N|2439217|2439306|2438855|2439130|N|N|N|N|N| +2439221|AAAAAAAAFDIDFCAA|1966-04-05|795|3458|266|1966|2|4|5|2|1966|266|3458|Tuesday|1966Q2|N|N|N|2439217|2439306|2438856|2439131|N|N|N|N|N| +2439222|AAAAAAAAGDIDFCAA|1966-04-06|795|3458|266|1966|3|4|6|2|1966|266|3458|Wednesday|1966Q2|N|N|N|2439217|2439306|2438857|2439132|N|N|N|N|N| +2439223|AAAAAAAAHDIDFCAA|1966-04-07|795|3458|266|1966|4|4|7|2|1966|266|3458|Thursday|1966Q2|N|N|N|2439217|2439306|2438858|2439133|N|N|N|N|N| +2439224|AAAAAAAAIDIDFCAA|1966-04-08|795|3458|266|1966|5|4|8|2|1966|266|3458|Friday|1966Q2|N|Y|N|2439217|2439306|2438859|2439134|N|N|N|N|N| +2439225|AAAAAAAAJDIDFCAA|1966-04-09|795|3458|266|1966|6|4|9|2|1966|266|3458|Saturday|1966Q2|N|Y|N|2439217|2439306|2438860|2439135|N|N|N|N|N| +2439226|AAAAAAAAKDIDFCAA|1966-04-10|795|3458|266|1966|0|4|10|2|1966|266|3458|Sunday|1966Q2|N|N|N|2439217|2439306|2438861|2439136|N|N|N|N|N| +2439227|AAAAAAAALDIDFCAA|1966-04-11|795|3458|266|1966|1|4|11|2|1966|266|3458|Monday|1966Q2|N|N|N|2439217|2439306|2438862|2439137|N|N|N|N|N| +2439228|AAAAAAAAMDIDFCAA|1966-04-12|795|3459|266|1966|2|4|12|2|1966|266|3459|Tuesday|1966Q2|N|N|N|2439217|2439306|2438863|2439138|N|N|N|N|N| +2439229|AAAAAAAANDIDFCAA|1966-04-13|795|3459|266|1966|3|4|13|2|1966|266|3459|Wednesday|1966Q2|N|N|N|2439217|2439306|2438864|2439139|N|N|N|N|N| +2439230|AAAAAAAAODIDFCAA|1966-04-14|795|3459|266|1966|4|4|14|2|1966|266|3459|Thursday|1966Q2|N|N|N|2439217|2439306|2438865|2439140|N|N|N|N|N| +2439231|AAAAAAAAPDIDFCAA|1966-04-15|795|3459|266|1966|5|4|15|2|1966|266|3459|Friday|1966Q2|N|Y|N|2439217|2439306|2438866|2439141|N|N|N|N|N| +2439232|AAAAAAAAAEIDFCAA|1966-04-16|795|3459|266|1966|6|4|16|2|1966|266|3459|Saturday|1966Q2|N|Y|N|2439217|2439306|2438867|2439142|N|N|N|N|N| +2439233|AAAAAAAABEIDFCAA|1966-04-17|795|3459|266|1966|0|4|17|2|1966|266|3459|Sunday|1966Q2|N|N|N|2439217|2439306|2438868|2439143|N|N|N|N|N| +2439234|AAAAAAAACEIDFCAA|1966-04-18|795|3459|266|1966|1|4|18|2|1966|266|3459|Monday|1966Q2|N|N|N|2439217|2439306|2438869|2439144|N|N|N|N|N| +2439235|AAAAAAAADEIDFCAA|1966-04-19|795|3460|266|1966|2|4|19|2|1966|266|3460|Tuesday|1966Q2|N|N|N|2439217|2439306|2438870|2439145|N|N|N|N|N| +2439236|AAAAAAAAEEIDFCAA|1966-04-20|795|3460|266|1966|3|4|20|2|1966|266|3460|Wednesday|1966Q2|N|N|N|2439217|2439306|2438871|2439146|N|N|N|N|N| +2439237|AAAAAAAAFEIDFCAA|1966-04-21|795|3460|266|1966|4|4|21|2|1966|266|3460|Thursday|1966Q2|N|N|N|2439217|2439306|2438872|2439147|N|N|N|N|N| +2439238|AAAAAAAAGEIDFCAA|1966-04-22|795|3460|266|1966|5|4|22|2|1966|266|3460|Friday|1966Q2|N|Y|N|2439217|2439306|2438873|2439148|N|N|N|N|N| +2439239|AAAAAAAAHEIDFCAA|1966-04-23|795|3460|266|1966|6|4|23|2|1966|266|3460|Saturday|1966Q2|N|Y|N|2439217|2439306|2438874|2439149|N|N|N|N|N| +2439240|AAAAAAAAIEIDFCAA|1966-04-24|795|3460|266|1966|0|4|24|2|1966|266|3460|Sunday|1966Q2|N|N|N|2439217|2439306|2438875|2439150|N|N|N|N|N| +2439241|AAAAAAAAJEIDFCAA|1966-04-25|795|3460|266|1966|1|4|25|2|1966|266|3460|Monday|1966Q2|N|N|N|2439217|2439306|2438876|2439151|N|N|N|N|N| +2439242|AAAAAAAAKEIDFCAA|1966-04-26|795|3461|266|1966|2|4|26|2|1966|266|3461|Tuesday|1966Q2|N|N|N|2439217|2439306|2438877|2439152|N|N|N|N|N| +2439243|AAAAAAAALEIDFCAA|1966-04-27|795|3461|266|1966|3|4|27|2|1966|266|3461|Wednesday|1966Q2|N|N|N|2439217|2439306|2438878|2439153|N|N|N|N|N| +2439244|AAAAAAAAMEIDFCAA|1966-04-28|795|3461|266|1966|4|4|28|2|1966|266|3461|Thursday|1966Q2|N|N|N|2439217|2439306|2438879|2439154|N|N|N|N|N| +2439245|AAAAAAAANEIDFCAA|1966-04-29|795|3461|266|1966|5|4|29|2|1966|266|3461|Friday|1966Q2|N|Y|N|2439217|2439306|2438880|2439155|N|N|N|N|N| +2439246|AAAAAAAAOEIDFCAA|1966-04-30|795|3461|266|1966|6|4|30|2|1966|266|3461|Saturday|1966Q2|N|Y|N|2439217|2439306|2438881|2439156|N|N|N|N|N| +2439247|AAAAAAAAPEIDFCAA|1966-05-01|796|3461|266|1966|0|5|1|2|1966|266|3461|Sunday|1966Q2|N|N|N|2439247|2439366|2438882|2439157|N|N|N|N|N| +2439248|AAAAAAAAAFIDFCAA|1966-05-02|796|3461|266|1966|1|5|2|2|1966|266|3461|Monday|1966Q2|N|N|N|2439247|2439366|2438883|2439158|N|N|N|N|N| +2439249|AAAAAAAABFIDFCAA|1966-05-03|796|3462|266|1966|2|5|3|2|1966|266|3462|Tuesday|1966Q2|N|N|N|2439247|2439366|2438884|2439159|N|N|N|N|N| +2439250|AAAAAAAACFIDFCAA|1966-05-04|796|3462|266|1966|3|5|4|2|1966|266|3462|Wednesday|1966Q2|N|N|N|2439247|2439366|2438885|2439160|N|N|N|N|N| +2439251|AAAAAAAADFIDFCAA|1966-05-05|796|3462|266|1966|4|5|5|2|1966|266|3462|Thursday|1966Q2|N|N|N|2439247|2439366|2438886|2439161|N|N|N|N|N| +2439252|AAAAAAAAEFIDFCAA|1966-05-06|796|3462|266|1966|5|5|6|2|1966|266|3462|Friday|1966Q2|N|Y|N|2439247|2439366|2438887|2439162|N|N|N|N|N| +2439253|AAAAAAAAFFIDFCAA|1966-05-07|796|3462|266|1966|6|5|7|2|1966|266|3462|Saturday|1966Q2|N|Y|N|2439247|2439366|2438888|2439163|N|N|N|N|N| +2439254|AAAAAAAAGFIDFCAA|1966-05-08|796|3462|266|1966|0|5|8|2|1966|266|3462|Sunday|1966Q2|N|N|N|2439247|2439366|2438889|2439164|N|N|N|N|N| +2439255|AAAAAAAAHFIDFCAA|1966-05-09|796|3462|266|1966|1|5|9|2|1966|266|3462|Monday|1966Q2|N|N|N|2439247|2439366|2438890|2439165|N|N|N|N|N| +2439256|AAAAAAAAIFIDFCAA|1966-05-10|796|3463|266|1966|2|5|10|2|1966|266|3463|Tuesday|1966Q2|N|N|N|2439247|2439366|2438891|2439166|N|N|N|N|N| +2439257|AAAAAAAAJFIDFCAA|1966-05-11|796|3463|266|1966|3|5|11|2|1966|266|3463|Wednesday|1966Q2|N|N|N|2439247|2439366|2438892|2439167|N|N|N|N|N| +2439258|AAAAAAAAKFIDFCAA|1966-05-12|796|3463|266|1966|4|5|12|2|1966|266|3463|Thursday|1966Q2|N|N|N|2439247|2439366|2438893|2439168|N|N|N|N|N| +2439259|AAAAAAAALFIDFCAA|1966-05-13|796|3463|266|1966|5|5|13|2|1966|266|3463|Friday|1966Q2|N|Y|N|2439247|2439366|2438894|2439169|N|N|N|N|N| +2439260|AAAAAAAAMFIDFCAA|1966-05-14|796|3463|266|1966|6|5|14|2|1966|266|3463|Saturday|1966Q2|N|Y|N|2439247|2439366|2438895|2439170|N|N|N|N|N| +2439261|AAAAAAAANFIDFCAA|1966-05-15|796|3463|266|1966|0|5|15|2|1966|266|3463|Sunday|1966Q2|N|N|N|2439247|2439366|2438896|2439171|N|N|N|N|N| +2439262|AAAAAAAAOFIDFCAA|1966-05-16|796|3463|266|1966|1|5|16|2|1966|266|3463|Monday|1966Q2|N|N|N|2439247|2439366|2438897|2439172|N|N|N|N|N| +2439263|AAAAAAAAPFIDFCAA|1966-05-17|796|3464|266|1966|2|5|17|2|1966|266|3464|Tuesday|1966Q2|N|N|N|2439247|2439366|2438898|2439173|N|N|N|N|N| +2439264|AAAAAAAAAGIDFCAA|1966-05-18|796|3464|266|1966|3|5|18|2|1966|266|3464|Wednesday|1966Q2|N|N|N|2439247|2439366|2438899|2439174|N|N|N|N|N| +2439265|AAAAAAAABGIDFCAA|1966-05-19|796|3464|266|1966|4|5|19|2|1966|266|3464|Thursday|1966Q2|N|N|N|2439247|2439366|2438900|2439175|N|N|N|N|N| +2439266|AAAAAAAACGIDFCAA|1966-05-20|796|3464|266|1966|5|5|20|2|1966|266|3464|Friday|1966Q2|N|Y|N|2439247|2439366|2438901|2439176|N|N|N|N|N| +2439267|AAAAAAAADGIDFCAA|1966-05-21|796|3464|266|1966|6|5|21|2|1966|266|3464|Saturday|1966Q2|N|Y|N|2439247|2439366|2438902|2439177|N|N|N|N|N| +2439268|AAAAAAAAEGIDFCAA|1966-05-22|796|3464|266|1966|0|5|22|2|1966|266|3464|Sunday|1966Q2|N|N|N|2439247|2439366|2438903|2439178|N|N|N|N|N| +2439269|AAAAAAAAFGIDFCAA|1966-05-23|796|3464|266|1966|1|5|23|2|1966|266|3464|Monday|1966Q2|N|N|N|2439247|2439366|2438904|2439179|N|N|N|N|N| +2439270|AAAAAAAAGGIDFCAA|1966-05-24|796|3465|266|1966|2|5|24|2|1966|266|3465|Tuesday|1966Q2|N|N|N|2439247|2439366|2438905|2439180|N|N|N|N|N| +2439271|AAAAAAAAHGIDFCAA|1966-05-25|796|3465|266|1966|3|5|25|2|1966|266|3465|Wednesday|1966Q2|N|N|N|2439247|2439366|2438906|2439181|N|N|N|N|N| +2439272|AAAAAAAAIGIDFCAA|1966-05-26|796|3465|266|1966|4|5|26|2|1966|266|3465|Thursday|1966Q2|N|N|N|2439247|2439366|2438907|2439182|N|N|N|N|N| +2439273|AAAAAAAAJGIDFCAA|1966-05-27|796|3465|266|1966|5|5|27|2|1966|266|3465|Friday|1966Q2|N|Y|N|2439247|2439366|2438908|2439183|N|N|N|N|N| +2439274|AAAAAAAAKGIDFCAA|1966-05-28|796|3465|266|1966|6|5|28|2|1966|266|3465|Saturday|1966Q2|N|Y|N|2439247|2439366|2438909|2439184|N|N|N|N|N| +2439275|AAAAAAAALGIDFCAA|1966-05-29|796|3465|266|1966|0|5|29|2|1966|266|3465|Sunday|1966Q2|N|N|N|2439247|2439366|2438910|2439185|N|N|N|N|N| +2439276|AAAAAAAAMGIDFCAA|1966-05-30|796|3465|266|1966|1|5|30|2|1966|266|3465|Monday|1966Q2|N|N|N|2439247|2439366|2438911|2439186|N|N|N|N|N| +2439277|AAAAAAAANGIDFCAA|1966-05-31|796|3466|266|1966|2|5|31|2|1966|266|3466|Tuesday|1966Q2|N|N|N|2439247|2439366|2438912|2439187|N|N|N|N|N| +2439278|AAAAAAAAOGIDFCAA|1966-06-01|797|3466|267|1966|3|6|1|2|1966|267|3466|Wednesday|1966Q2|N|N|N|2439278|2439428|2438913|2439188|N|N|N|N|N| +2439279|AAAAAAAAPGIDFCAA|1966-06-02|797|3466|267|1966|4|6|2|2|1966|267|3466|Thursday|1966Q2|N|N|N|2439278|2439428|2438914|2439189|N|N|N|N|N| +2439280|AAAAAAAAAHIDFCAA|1966-06-03|797|3466|267|1966|5|6|3|2|1966|267|3466|Friday|1966Q2|N|Y|N|2439278|2439428|2438915|2439190|N|N|N|N|N| +2439281|AAAAAAAABHIDFCAA|1966-06-04|797|3466|267|1966|6|6|4|2|1966|267|3466|Saturday|1966Q2|N|Y|N|2439278|2439428|2438916|2439191|N|N|N|N|N| +2439282|AAAAAAAACHIDFCAA|1966-06-05|797|3466|267|1966|0|6|5|2|1966|267|3466|Sunday|1966Q2|N|N|N|2439278|2439428|2438917|2439192|N|N|N|N|N| +2439283|AAAAAAAADHIDFCAA|1966-06-06|797|3466|267|1966|1|6|6|2|1966|267|3466|Monday|1966Q2|N|N|N|2439278|2439428|2438918|2439193|N|N|N|N|N| +2439284|AAAAAAAAEHIDFCAA|1966-06-07|797|3467|267|1966|2|6|7|2|1966|267|3467|Tuesday|1966Q2|N|N|N|2439278|2439428|2438919|2439194|N|N|N|N|N| +2439285|AAAAAAAAFHIDFCAA|1966-06-08|797|3467|267|1966|3|6|8|2|1966|267|3467|Wednesday|1966Q2|N|N|N|2439278|2439428|2438920|2439195|N|N|N|N|N| +2439286|AAAAAAAAGHIDFCAA|1966-06-09|797|3467|267|1966|4|6|9|2|1966|267|3467|Thursday|1966Q2|N|N|N|2439278|2439428|2438921|2439196|N|N|N|N|N| +2439287|AAAAAAAAHHIDFCAA|1966-06-10|797|3467|267|1966|5|6|10|2|1966|267|3467|Friday|1966Q2|N|Y|N|2439278|2439428|2438922|2439197|N|N|N|N|N| +2439288|AAAAAAAAIHIDFCAA|1966-06-11|797|3467|267|1966|6|6|11|2|1966|267|3467|Saturday|1966Q2|N|Y|N|2439278|2439428|2438923|2439198|N|N|N|N|N| +2439289|AAAAAAAAJHIDFCAA|1966-06-12|797|3467|267|1966|0|6|12|2|1966|267|3467|Sunday|1966Q2|N|N|N|2439278|2439428|2438924|2439199|N|N|N|N|N| +2439290|AAAAAAAAKHIDFCAA|1966-06-13|797|3467|267|1966|1|6|13|2|1966|267|3467|Monday|1966Q2|N|N|N|2439278|2439428|2438925|2439200|N|N|N|N|N| +2439291|AAAAAAAALHIDFCAA|1966-06-14|797|3468|267|1966|2|6|14|2|1966|267|3468|Tuesday|1966Q2|N|N|N|2439278|2439428|2438926|2439201|N|N|N|N|N| +2439292|AAAAAAAAMHIDFCAA|1966-06-15|797|3468|267|1966|3|6|15|2|1966|267|3468|Wednesday|1966Q2|N|N|N|2439278|2439428|2438927|2439202|N|N|N|N|N| +2439293|AAAAAAAANHIDFCAA|1966-06-16|797|3468|267|1966|4|6|16|2|1966|267|3468|Thursday|1966Q2|N|N|N|2439278|2439428|2438928|2439203|N|N|N|N|N| +2439294|AAAAAAAAOHIDFCAA|1966-06-17|797|3468|267|1966|5|6|17|2|1966|267|3468|Friday|1966Q2|N|Y|N|2439278|2439428|2438929|2439204|N|N|N|N|N| +2439295|AAAAAAAAPHIDFCAA|1966-06-18|797|3468|267|1966|6|6|18|2|1966|267|3468|Saturday|1966Q2|N|Y|N|2439278|2439428|2438930|2439205|N|N|N|N|N| +2439296|AAAAAAAAAIIDFCAA|1966-06-19|797|3468|267|1966|0|6|19|2|1966|267|3468|Sunday|1966Q2|N|N|N|2439278|2439428|2438931|2439206|N|N|N|N|N| +2439297|AAAAAAAABIIDFCAA|1966-06-20|797|3468|267|1966|1|6|20|2|1966|267|3468|Monday|1966Q2|N|N|N|2439278|2439428|2438932|2439207|N|N|N|N|N| +2439298|AAAAAAAACIIDFCAA|1966-06-21|797|3469|267|1966|2|6|21|2|1966|267|3469|Tuesday|1966Q2|N|N|N|2439278|2439428|2438933|2439208|N|N|N|N|N| +2439299|AAAAAAAADIIDFCAA|1966-06-22|797|3469|267|1966|3|6|22|2|1966|267|3469|Wednesday|1966Q2|N|N|N|2439278|2439428|2438934|2439209|N|N|N|N|N| +2439300|AAAAAAAAEIIDFCAA|1966-06-23|797|3469|267|1966|4|6|23|2|1966|267|3469|Thursday|1966Q2|N|N|N|2439278|2439428|2438935|2439210|N|N|N|N|N| +2439301|AAAAAAAAFIIDFCAA|1966-06-24|797|3469|267|1966|5|6|24|2|1966|267|3469|Friday|1966Q2|N|Y|N|2439278|2439428|2438936|2439211|N|N|N|N|N| +2439302|AAAAAAAAGIIDFCAA|1966-06-25|797|3469|267|1966|6|6|25|2|1966|267|3469|Saturday|1966Q2|N|Y|N|2439278|2439428|2438937|2439212|N|N|N|N|N| +2439303|AAAAAAAAHIIDFCAA|1966-06-26|797|3469|267|1966|0|6|26|2|1966|267|3469|Sunday|1966Q2|N|N|N|2439278|2439428|2438938|2439213|N|N|N|N|N| +2439304|AAAAAAAAIIIDFCAA|1966-06-27|797|3469|267|1966|1|6|27|2|1966|267|3469|Monday|1966Q2|N|N|N|2439278|2439428|2438939|2439214|N|N|N|N|N| +2439305|AAAAAAAAJIIDFCAA|1966-06-28|797|3470|267|1966|2|6|28|2|1966|267|3470|Tuesday|1966Q2|N|N|N|2439278|2439428|2438940|2439215|N|N|N|N|N| +2439306|AAAAAAAAKIIDFCAA|1966-06-29|797|3470|267|1966|3|6|29|2|1966|267|3470|Wednesday|1966Q2|N|N|N|2439278|2439428|2438941|2439216|N|N|N|N|N| +2439307|AAAAAAAALIIDFCAA|1966-06-30|797|3470|267|1966|4|6|30|2|1966|267|3470|Thursday|1966Q2|N|N|N|2439278|2439428|2438942|2439217|N|N|N|N|N| +2439308|AAAAAAAAMIIDFCAA|1966-07-01|798|3470|267|1966|5|7|1|2|1966|267|3470|Friday|1966Q2|N|Y|N|2439308|2439488|2438943|2439217|N|N|N|N|N| +2439309|AAAAAAAANIIDFCAA|1966-07-02|798|3470|267|1966|6|7|2|3|1966|267|3470|Saturday|1966Q3|N|Y|N|2439308|2439488|2438944|2439218|N|N|N|N|N| +2439310|AAAAAAAAOIIDFCAA|1966-07-03|798|3470|267|1966|0|7|3|3|1966|267|3470|Sunday|1966Q3|N|N|N|2439308|2439488|2438945|2439219|N|N|N|N|N| +2439311|AAAAAAAAPIIDFCAA|1966-07-04|798|3470|267|1966|1|7|4|3|1966|267|3470|Monday|1966Q3|N|N|N|2439308|2439488|2438946|2439220|N|N|N|N|N| +2439312|AAAAAAAAAJIDFCAA|1966-07-05|798|3471|267|1966|2|7|5|3|1966|267|3471|Tuesday|1966Q3|Y|N|N|2439308|2439488|2438947|2439221|N|N|N|N|N| +2439313|AAAAAAAABJIDFCAA|1966-07-06|798|3471|267|1966|3|7|6|3|1966|267|3471|Wednesday|1966Q3|N|N|Y|2439308|2439488|2438948|2439222|N|N|N|N|N| +2439314|AAAAAAAACJIDFCAA|1966-07-07|798|3471|267|1966|4|7|7|3|1966|267|3471|Thursday|1966Q3|N|N|N|2439308|2439488|2438949|2439223|N|N|N|N|N| +2439315|AAAAAAAADJIDFCAA|1966-07-08|798|3471|267|1966|5|7|8|3|1966|267|3471|Friday|1966Q3|N|Y|N|2439308|2439488|2438950|2439224|N|N|N|N|N| +2439316|AAAAAAAAEJIDFCAA|1966-07-09|798|3471|267|1966|6|7|9|3|1966|267|3471|Saturday|1966Q3|N|Y|N|2439308|2439488|2438951|2439225|N|N|N|N|N| +2439317|AAAAAAAAFJIDFCAA|1966-07-10|798|3471|267|1966|0|7|10|3|1966|267|3471|Sunday|1966Q3|N|N|N|2439308|2439488|2438952|2439226|N|N|N|N|N| +2439318|AAAAAAAAGJIDFCAA|1966-07-11|798|3471|267|1966|1|7|11|3|1966|267|3471|Monday|1966Q3|N|N|N|2439308|2439488|2438953|2439227|N|N|N|N|N| +2439319|AAAAAAAAHJIDFCAA|1966-07-12|798|3472|267|1966|2|7|12|3|1966|267|3472|Tuesday|1966Q3|N|N|N|2439308|2439488|2438954|2439228|N|N|N|N|N| +2439320|AAAAAAAAIJIDFCAA|1966-07-13|798|3472|267|1966|3|7|13|3|1966|267|3472|Wednesday|1966Q3|N|N|N|2439308|2439488|2438955|2439229|N|N|N|N|N| +2439321|AAAAAAAAJJIDFCAA|1966-07-14|798|3472|267|1966|4|7|14|3|1966|267|3472|Thursday|1966Q3|N|N|N|2439308|2439488|2438956|2439230|N|N|N|N|N| +2439322|AAAAAAAAKJIDFCAA|1966-07-15|798|3472|267|1966|5|7|15|3|1966|267|3472|Friday|1966Q3|N|Y|N|2439308|2439488|2438957|2439231|N|N|N|N|N| +2439323|AAAAAAAALJIDFCAA|1966-07-16|798|3472|267|1966|6|7|16|3|1966|267|3472|Saturday|1966Q3|N|Y|N|2439308|2439488|2438958|2439232|N|N|N|N|N| +2439324|AAAAAAAAMJIDFCAA|1966-07-17|798|3472|267|1966|0|7|17|3|1966|267|3472|Sunday|1966Q3|N|N|N|2439308|2439488|2438959|2439233|N|N|N|N|N| +2439325|AAAAAAAANJIDFCAA|1966-07-18|798|3472|267|1966|1|7|18|3|1966|267|3472|Monday|1966Q3|N|N|N|2439308|2439488|2438960|2439234|N|N|N|N|N| +2439326|AAAAAAAAOJIDFCAA|1966-07-19|798|3473|267|1966|2|7|19|3|1966|267|3473|Tuesday|1966Q3|N|N|N|2439308|2439488|2438961|2439235|N|N|N|N|N| +2439327|AAAAAAAAPJIDFCAA|1966-07-20|798|3473|267|1966|3|7|20|3|1966|267|3473|Wednesday|1966Q3|N|N|N|2439308|2439488|2438962|2439236|N|N|N|N|N| +2439328|AAAAAAAAAKIDFCAA|1966-07-21|798|3473|267|1966|4|7|21|3|1966|267|3473|Thursday|1966Q3|N|N|N|2439308|2439488|2438963|2439237|N|N|N|N|N| +2439329|AAAAAAAABKIDFCAA|1966-07-22|798|3473|267|1966|5|7|22|3|1966|267|3473|Friday|1966Q3|N|Y|N|2439308|2439488|2438964|2439238|N|N|N|N|N| +2439330|AAAAAAAACKIDFCAA|1966-07-23|798|3473|267|1966|6|7|23|3|1966|267|3473|Saturday|1966Q3|N|Y|N|2439308|2439488|2438965|2439239|N|N|N|N|N| +2439331|AAAAAAAADKIDFCAA|1966-07-24|798|3473|267|1966|0|7|24|3|1966|267|3473|Sunday|1966Q3|N|N|N|2439308|2439488|2438966|2439240|N|N|N|N|N| +2439332|AAAAAAAAEKIDFCAA|1966-07-25|798|3473|267|1966|1|7|25|3|1966|267|3473|Monday|1966Q3|N|N|N|2439308|2439488|2438967|2439241|N|N|N|N|N| +2439333|AAAAAAAAFKIDFCAA|1966-07-26|798|3474|267|1966|2|7|26|3|1966|267|3474|Tuesday|1966Q3|N|N|N|2439308|2439488|2438968|2439242|N|N|N|N|N| +2439334|AAAAAAAAGKIDFCAA|1966-07-27|798|3474|267|1966|3|7|27|3|1966|267|3474|Wednesday|1966Q3|N|N|N|2439308|2439488|2438969|2439243|N|N|N|N|N| +2439335|AAAAAAAAHKIDFCAA|1966-07-28|798|3474|267|1966|4|7|28|3|1966|267|3474|Thursday|1966Q3|N|N|N|2439308|2439488|2438970|2439244|N|N|N|N|N| +2439336|AAAAAAAAIKIDFCAA|1966-07-29|798|3474|267|1966|5|7|29|3|1966|267|3474|Friday|1966Q3|N|Y|N|2439308|2439488|2438971|2439245|N|N|N|N|N| +2439337|AAAAAAAAJKIDFCAA|1966-07-30|798|3474|267|1966|6|7|30|3|1966|267|3474|Saturday|1966Q3|N|Y|N|2439308|2439488|2438972|2439246|N|N|N|N|N| +2439338|AAAAAAAAKKIDFCAA|1966-07-31|798|3474|267|1966|0|7|31|3|1966|267|3474|Sunday|1966Q3|N|N|N|2439308|2439488|2438973|2439247|N|N|N|N|N| +2439339|AAAAAAAALKIDFCAA|1966-08-01|799|3474|267|1966|1|8|1|3|1966|267|3474|Monday|1966Q3|N|N|N|2439339|2439550|2438974|2439248|N|N|N|N|N| +2439340|AAAAAAAAMKIDFCAA|1966-08-02|799|3475|267|1966|2|8|2|3|1966|267|3475|Tuesday|1966Q3|N|N|N|2439339|2439550|2438975|2439249|N|N|N|N|N| +2439341|AAAAAAAANKIDFCAA|1966-08-03|799|3475|267|1966|3|8|3|3|1966|267|3475|Wednesday|1966Q3|N|N|N|2439339|2439550|2438976|2439250|N|N|N|N|N| +2439342|AAAAAAAAOKIDFCAA|1966-08-04|799|3475|267|1966|4|8|4|3|1966|267|3475|Thursday|1966Q3|N|N|N|2439339|2439550|2438977|2439251|N|N|N|N|N| +2439343|AAAAAAAAPKIDFCAA|1966-08-05|799|3475|267|1966|5|8|5|3|1966|267|3475|Friday|1966Q3|N|Y|N|2439339|2439550|2438978|2439252|N|N|N|N|N| +2439344|AAAAAAAAALIDFCAA|1966-08-06|799|3475|267|1966|6|8|6|3|1966|267|3475|Saturday|1966Q3|N|Y|N|2439339|2439550|2438979|2439253|N|N|N|N|N| +2439345|AAAAAAAABLIDFCAA|1966-08-07|799|3475|267|1966|0|8|7|3|1966|267|3475|Sunday|1966Q3|N|N|N|2439339|2439550|2438980|2439254|N|N|N|N|N| +2439346|AAAAAAAACLIDFCAA|1966-08-08|799|3475|267|1966|1|8|8|3|1966|267|3475|Monday|1966Q3|N|N|N|2439339|2439550|2438981|2439255|N|N|N|N|N| +2439347|AAAAAAAADLIDFCAA|1966-08-09|799|3476|267|1966|2|8|9|3|1966|267|3476|Tuesday|1966Q3|N|N|N|2439339|2439550|2438982|2439256|N|N|N|N|N| +2439348|AAAAAAAAELIDFCAA|1966-08-10|799|3476|267|1966|3|8|10|3|1966|267|3476|Wednesday|1966Q3|N|N|N|2439339|2439550|2438983|2439257|N|N|N|N|N| +2439349|AAAAAAAAFLIDFCAA|1966-08-11|799|3476|267|1966|4|8|11|3|1966|267|3476|Thursday|1966Q3|N|N|N|2439339|2439550|2438984|2439258|N|N|N|N|N| +2439350|AAAAAAAAGLIDFCAA|1966-08-12|799|3476|267|1966|5|8|12|3|1966|267|3476|Friday|1966Q3|N|Y|N|2439339|2439550|2438985|2439259|N|N|N|N|N| +2439351|AAAAAAAAHLIDFCAA|1966-08-13|799|3476|267|1966|6|8|13|3|1966|267|3476|Saturday|1966Q3|N|Y|N|2439339|2439550|2438986|2439260|N|N|N|N|N| +2439352|AAAAAAAAILIDFCAA|1966-08-14|799|3476|267|1966|0|8|14|3|1966|267|3476|Sunday|1966Q3|N|N|N|2439339|2439550|2438987|2439261|N|N|N|N|N| +2439353|AAAAAAAAJLIDFCAA|1966-08-15|799|3476|267|1966|1|8|15|3|1966|267|3476|Monday|1966Q3|N|N|N|2439339|2439550|2438988|2439262|N|N|N|N|N| +2439354|AAAAAAAAKLIDFCAA|1966-08-16|799|3477|267|1966|2|8|16|3|1966|267|3477|Tuesday|1966Q3|N|N|N|2439339|2439550|2438989|2439263|N|N|N|N|N| +2439355|AAAAAAAALLIDFCAA|1966-08-17|799|3477|267|1966|3|8|17|3|1966|267|3477|Wednesday|1966Q3|N|N|N|2439339|2439550|2438990|2439264|N|N|N|N|N| +2439356|AAAAAAAAMLIDFCAA|1966-08-18|799|3477|267|1966|4|8|18|3|1966|267|3477|Thursday|1966Q3|N|N|N|2439339|2439550|2438991|2439265|N|N|N|N|N| +2439357|AAAAAAAANLIDFCAA|1966-08-19|799|3477|267|1966|5|8|19|3|1966|267|3477|Friday|1966Q3|N|Y|N|2439339|2439550|2438992|2439266|N|N|N|N|N| +2439358|AAAAAAAAOLIDFCAA|1966-08-20|799|3477|267|1966|6|8|20|3|1966|267|3477|Saturday|1966Q3|N|Y|N|2439339|2439550|2438993|2439267|N|N|N|N|N| +2439359|AAAAAAAAPLIDFCAA|1966-08-21|799|3477|267|1966|0|8|21|3|1966|267|3477|Sunday|1966Q3|N|N|N|2439339|2439550|2438994|2439268|N|N|N|N|N| +2439360|AAAAAAAAAMIDFCAA|1966-08-22|799|3477|267|1966|1|8|22|3|1966|267|3477|Monday|1966Q3|N|N|N|2439339|2439550|2438995|2439269|N|N|N|N|N| +2439361|AAAAAAAABMIDFCAA|1966-08-23|799|3478|267|1966|2|8|23|3|1966|267|3478|Tuesday|1966Q3|N|N|N|2439339|2439550|2438996|2439270|N|N|N|N|N| +2439362|AAAAAAAACMIDFCAA|1966-08-24|799|3478|267|1966|3|8|24|3|1966|267|3478|Wednesday|1966Q3|N|N|N|2439339|2439550|2438997|2439271|N|N|N|N|N| +2439363|AAAAAAAADMIDFCAA|1966-08-25|799|3478|267|1966|4|8|25|3|1966|267|3478|Thursday|1966Q3|N|N|N|2439339|2439550|2438998|2439272|N|N|N|N|N| +2439364|AAAAAAAAEMIDFCAA|1966-08-26|799|3478|267|1966|5|8|26|3|1966|267|3478|Friday|1966Q3|N|Y|N|2439339|2439550|2438999|2439273|N|N|N|N|N| +2439365|AAAAAAAAFMIDFCAA|1966-08-27|799|3478|267|1966|6|8|27|3|1966|267|3478|Saturday|1966Q3|N|Y|N|2439339|2439550|2439000|2439274|N|N|N|N|N| +2439366|AAAAAAAAGMIDFCAA|1966-08-28|799|3478|267|1966|0|8|28|3|1966|267|3478|Sunday|1966Q3|N|N|N|2439339|2439550|2439001|2439275|N|N|N|N|N| +2439367|AAAAAAAAHMIDFCAA|1966-08-29|799|3478|267|1966|1|8|29|3|1966|267|3478|Monday|1966Q3|N|N|N|2439339|2439550|2439002|2439276|N|N|N|N|N| +2439368|AAAAAAAAIMIDFCAA|1966-08-30|799|3479|267|1966|2|8|30|3|1966|267|3479|Tuesday|1966Q3|N|N|N|2439339|2439550|2439003|2439277|N|N|N|N|N| +2439369|AAAAAAAAJMIDFCAA|1966-08-31|799|3479|267|1966|3|8|31|3|1966|267|3479|Wednesday|1966Q3|N|N|N|2439339|2439550|2439004|2439278|N|N|N|N|N| +2439370|AAAAAAAAKMIDFCAA|1966-09-01|800|3479|268|1966|4|9|1|3|1966|268|3479|Thursday|1966Q3|N|N|N|2439370|2439612|2439005|2439279|N|N|N|N|N| +2439371|AAAAAAAALMIDFCAA|1966-09-02|800|3479|268|1966|5|9|2|3|1966|268|3479|Friday|1966Q3|N|Y|N|2439370|2439612|2439006|2439280|N|N|N|N|N| +2439372|AAAAAAAAMMIDFCAA|1966-09-03|800|3479|268|1966|6|9|3|3|1966|268|3479|Saturday|1966Q3|N|Y|N|2439370|2439612|2439007|2439281|N|N|N|N|N| +2439373|AAAAAAAANMIDFCAA|1966-09-04|800|3479|268|1966|0|9|4|3|1966|268|3479|Sunday|1966Q3|N|N|N|2439370|2439612|2439008|2439282|N|N|N|N|N| +2439374|AAAAAAAAOMIDFCAA|1966-09-05|800|3479|268|1966|1|9|5|3|1966|268|3479|Monday|1966Q3|N|N|N|2439370|2439612|2439009|2439283|N|N|N|N|N| +2439375|AAAAAAAAPMIDFCAA|1966-09-06|800|3480|268|1966|2|9|6|3|1966|268|3480|Tuesday|1966Q3|N|N|N|2439370|2439612|2439010|2439284|N|N|N|N|N| +2439376|AAAAAAAAANIDFCAA|1966-09-07|800|3480|268|1966|3|9|7|3|1966|268|3480|Wednesday|1966Q3|N|N|N|2439370|2439612|2439011|2439285|N|N|N|N|N| +2439377|AAAAAAAABNIDFCAA|1966-09-08|800|3480|268|1966|4|9|8|3|1966|268|3480|Thursday|1966Q3|N|N|N|2439370|2439612|2439012|2439286|N|N|N|N|N| +2439378|AAAAAAAACNIDFCAA|1966-09-09|800|3480|268|1966|5|9|9|3|1966|268|3480|Friday|1966Q3|N|Y|N|2439370|2439612|2439013|2439287|N|N|N|N|N| +2439379|AAAAAAAADNIDFCAA|1966-09-10|800|3480|268|1966|6|9|10|3|1966|268|3480|Saturday|1966Q3|N|Y|N|2439370|2439612|2439014|2439288|N|N|N|N|N| +2439380|AAAAAAAAENIDFCAA|1966-09-11|800|3480|268|1966|0|9|11|3|1966|268|3480|Sunday|1966Q3|N|N|N|2439370|2439612|2439015|2439289|N|N|N|N|N| +2439381|AAAAAAAAFNIDFCAA|1966-09-12|800|3480|268|1966|1|9|12|3|1966|268|3480|Monday|1966Q3|N|N|N|2439370|2439612|2439016|2439290|N|N|N|N|N| +2439382|AAAAAAAAGNIDFCAA|1966-09-13|800|3481|268|1966|2|9|13|3|1966|268|3481|Tuesday|1966Q3|N|N|N|2439370|2439612|2439017|2439291|N|N|N|N|N| +2439383|AAAAAAAAHNIDFCAA|1966-09-14|800|3481|268|1966|3|9|14|3|1966|268|3481|Wednesday|1966Q3|N|N|N|2439370|2439612|2439018|2439292|N|N|N|N|N| +2439384|AAAAAAAAINIDFCAA|1966-09-15|800|3481|268|1966|4|9|15|3|1966|268|3481|Thursday|1966Q3|N|N|N|2439370|2439612|2439019|2439293|N|N|N|N|N| +2439385|AAAAAAAAJNIDFCAA|1966-09-16|800|3481|268|1966|5|9|16|3|1966|268|3481|Friday|1966Q3|N|Y|N|2439370|2439612|2439020|2439294|N|N|N|N|N| +2439386|AAAAAAAAKNIDFCAA|1966-09-17|800|3481|268|1966|6|9|17|3|1966|268|3481|Saturday|1966Q3|N|Y|N|2439370|2439612|2439021|2439295|N|N|N|N|N| +2439387|AAAAAAAALNIDFCAA|1966-09-18|800|3481|268|1966|0|9|18|3|1966|268|3481|Sunday|1966Q3|N|N|N|2439370|2439612|2439022|2439296|N|N|N|N|N| +2439388|AAAAAAAAMNIDFCAA|1966-09-19|800|3481|268|1966|1|9|19|3|1966|268|3481|Monday|1966Q3|N|N|N|2439370|2439612|2439023|2439297|N|N|N|N|N| +2439389|AAAAAAAANNIDFCAA|1966-09-20|800|3482|268|1966|2|9|20|3|1966|268|3482|Tuesday|1966Q3|N|N|N|2439370|2439612|2439024|2439298|N|N|N|N|N| +2439390|AAAAAAAAONIDFCAA|1966-09-21|800|3482|268|1966|3|9|21|3|1966|268|3482|Wednesday|1966Q3|N|N|N|2439370|2439612|2439025|2439299|N|N|N|N|N| +2439391|AAAAAAAAPNIDFCAA|1966-09-22|800|3482|268|1966|4|9|22|3|1966|268|3482|Thursday|1966Q3|N|N|N|2439370|2439612|2439026|2439300|N|N|N|N|N| +2439392|AAAAAAAAAOIDFCAA|1966-09-23|800|3482|268|1966|5|9|23|3|1966|268|3482|Friday|1966Q3|N|Y|N|2439370|2439612|2439027|2439301|N|N|N|N|N| +2439393|AAAAAAAABOIDFCAA|1966-09-24|800|3482|268|1966|6|9|24|3|1966|268|3482|Saturday|1966Q3|N|Y|N|2439370|2439612|2439028|2439302|N|N|N|N|N| +2439394|AAAAAAAACOIDFCAA|1966-09-25|800|3482|268|1966|0|9|25|3|1966|268|3482|Sunday|1966Q3|N|N|N|2439370|2439612|2439029|2439303|N|N|N|N|N| +2439395|AAAAAAAADOIDFCAA|1966-09-26|800|3482|268|1966|1|9|26|3|1966|268|3482|Monday|1966Q3|N|N|N|2439370|2439612|2439030|2439304|N|N|N|N|N| +2439396|AAAAAAAAEOIDFCAA|1966-09-27|800|3483|268|1966|2|9|27|3|1966|268|3483|Tuesday|1966Q3|N|N|N|2439370|2439612|2439031|2439305|N|N|N|N|N| +2439397|AAAAAAAAFOIDFCAA|1966-09-28|800|3483|268|1966|3|9|28|3|1966|268|3483|Wednesday|1966Q3|N|N|N|2439370|2439612|2439032|2439306|N|N|N|N|N| +2439398|AAAAAAAAGOIDFCAA|1966-09-29|800|3483|268|1966|4|9|29|3|1966|268|3483|Thursday|1966Q3|N|N|N|2439370|2439612|2439033|2439307|N|N|N|N|N| +2439399|AAAAAAAAHOIDFCAA|1966-09-30|800|3483|268|1966|5|9|30|3|1966|268|3483|Friday|1966Q3|N|Y|N|2439370|2439612|2439034|2439308|N|N|N|N|N| +2439400|AAAAAAAAIOIDFCAA|1966-10-01|801|3483|268|1966|6|10|1|3|1966|268|3483|Saturday|1966Q3|N|Y|N|2439400|2439672|2439035|2439308|N|N|N|N|N| +2439401|AAAAAAAAJOIDFCAA|1966-10-02|801|3483|268|1966|0|10|2|4|1966|268|3483|Sunday|1966Q4|N|N|N|2439400|2439672|2439036|2439309|N|N|N|N|N| +2439402|AAAAAAAAKOIDFCAA|1966-10-03|801|3483|268|1966|1|10|3|4|1966|268|3483|Monday|1966Q4|N|N|N|2439400|2439672|2439037|2439310|N|N|N|N|N| +2439403|AAAAAAAALOIDFCAA|1966-10-04|801|3484|268|1966|2|10|4|4|1966|268|3484|Tuesday|1966Q4|N|N|N|2439400|2439672|2439038|2439311|N|N|N|N|N| +2439404|AAAAAAAAMOIDFCAA|1966-10-05|801|3484|268|1966|3|10|5|4|1966|268|3484|Wednesday|1966Q4|N|N|N|2439400|2439672|2439039|2439312|N|N|N|N|N| +2439405|AAAAAAAANOIDFCAA|1966-10-06|801|3484|268|1966|4|10|6|4|1966|268|3484|Thursday|1966Q4|N|N|N|2439400|2439672|2439040|2439313|N|N|N|N|N| +2439406|AAAAAAAAOOIDFCAA|1966-10-07|801|3484|268|1966|5|10|7|4|1966|268|3484|Friday|1966Q4|N|Y|N|2439400|2439672|2439041|2439314|N|N|N|N|N| +2439407|AAAAAAAAPOIDFCAA|1966-10-08|801|3484|268|1966|6|10|8|4|1966|268|3484|Saturday|1966Q4|N|Y|N|2439400|2439672|2439042|2439315|N|N|N|N|N| +2439408|AAAAAAAAAPIDFCAA|1966-10-09|801|3484|268|1966|0|10|9|4|1966|268|3484|Sunday|1966Q4|N|N|N|2439400|2439672|2439043|2439316|N|N|N|N|N| +2439409|AAAAAAAABPIDFCAA|1966-10-10|801|3484|268|1966|1|10|10|4|1966|268|3484|Monday|1966Q4|N|N|N|2439400|2439672|2439044|2439317|N|N|N|N|N| +2439410|AAAAAAAACPIDFCAA|1966-10-11|801|3485|268|1966|2|10|11|4|1966|268|3485|Tuesday|1966Q4|N|N|N|2439400|2439672|2439045|2439318|N|N|N|N|N| +2439411|AAAAAAAADPIDFCAA|1966-10-12|801|3485|268|1966|3|10|12|4|1966|268|3485|Wednesday|1966Q4|N|N|N|2439400|2439672|2439046|2439319|N|N|N|N|N| +2439412|AAAAAAAAEPIDFCAA|1966-10-13|801|3485|268|1966|4|10|13|4|1966|268|3485|Thursday|1966Q4|N|N|N|2439400|2439672|2439047|2439320|N|N|N|N|N| +2439413|AAAAAAAAFPIDFCAA|1966-10-14|801|3485|268|1966|5|10|14|4|1966|268|3485|Friday|1966Q4|N|Y|N|2439400|2439672|2439048|2439321|N|N|N|N|N| +2439414|AAAAAAAAGPIDFCAA|1966-10-15|801|3485|268|1966|6|10|15|4|1966|268|3485|Saturday|1966Q4|N|Y|N|2439400|2439672|2439049|2439322|N|N|N|N|N| +2439415|AAAAAAAAHPIDFCAA|1966-10-16|801|3485|268|1966|0|10|16|4|1966|268|3485|Sunday|1966Q4|N|N|N|2439400|2439672|2439050|2439323|N|N|N|N|N| +2439416|AAAAAAAAIPIDFCAA|1966-10-17|801|3485|268|1966|1|10|17|4|1966|268|3485|Monday|1966Q4|N|N|N|2439400|2439672|2439051|2439324|N|N|N|N|N| +2439417|AAAAAAAAJPIDFCAA|1966-10-18|801|3486|268|1966|2|10|18|4|1966|268|3486|Tuesday|1966Q4|N|N|N|2439400|2439672|2439052|2439325|N|N|N|N|N| +2439418|AAAAAAAAKPIDFCAA|1966-10-19|801|3486|268|1966|3|10|19|4|1966|268|3486|Wednesday|1966Q4|N|N|N|2439400|2439672|2439053|2439326|N|N|N|N|N| +2439419|AAAAAAAALPIDFCAA|1966-10-20|801|3486|268|1966|4|10|20|4|1966|268|3486|Thursday|1966Q4|N|N|N|2439400|2439672|2439054|2439327|N|N|N|N|N| +2439420|AAAAAAAAMPIDFCAA|1966-10-21|801|3486|268|1966|5|10|21|4|1966|268|3486|Friday|1966Q4|N|Y|N|2439400|2439672|2439055|2439328|N|N|N|N|N| +2439421|AAAAAAAANPIDFCAA|1966-10-22|801|3486|268|1966|6|10|22|4|1966|268|3486|Saturday|1966Q4|N|Y|N|2439400|2439672|2439056|2439329|N|N|N|N|N| +2439422|AAAAAAAAOPIDFCAA|1966-10-23|801|3486|268|1966|0|10|23|4|1966|268|3486|Sunday|1966Q4|N|N|N|2439400|2439672|2439057|2439330|N|N|N|N|N| +2439423|AAAAAAAAPPIDFCAA|1966-10-24|801|3486|268|1966|1|10|24|4|1966|268|3486|Monday|1966Q4|N|N|N|2439400|2439672|2439058|2439331|N|N|N|N|N| +2439424|AAAAAAAAAAJDFCAA|1966-10-25|801|3487|268|1966|2|10|25|4|1966|268|3487|Tuesday|1966Q4|N|N|N|2439400|2439672|2439059|2439332|N|N|N|N|N| +2439425|AAAAAAAABAJDFCAA|1966-10-26|801|3487|268|1966|3|10|26|4|1966|268|3487|Wednesday|1966Q4|N|N|N|2439400|2439672|2439060|2439333|N|N|N|N|N| +2439426|AAAAAAAACAJDFCAA|1966-10-27|801|3487|268|1966|4|10|27|4|1966|268|3487|Thursday|1966Q4|N|N|N|2439400|2439672|2439061|2439334|N|N|N|N|N| +2439427|AAAAAAAADAJDFCAA|1966-10-28|801|3487|268|1966|5|10|28|4|1966|268|3487|Friday|1966Q4|N|Y|N|2439400|2439672|2439062|2439335|N|N|N|N|N| +2439428|AAAAAAAAEAJDFCAA|1966-10-29|801|3487|268|1966|6|10|29|4|1966|268|3487|Saturday|1966Q4|N|Y|N|2439400|2439672|2439063|2439336|N|N|N|N|N| +2439429|AAAAAAAAFAJDFCAA|1966-10-30|801|3487|268|1966|0|10|30|4|1966|268|3487|Sunday|1966Q4|N|N|N|2439400|2439672|2439064|2439337|N|N|N|N|N| +2439430|AAAAAAAAGAJDFCAA|1966-10-31|801|3487|268|1966|1|10|31|4|1966|268|3487|Monday|1966Q4|N|N|N|2439400|2439672|2439065|2439338|N|N|N|N|N| +2439431|AAAAAAAAHAJDFCAA|1966-11-01|802|3488|268|1966|2|11|1|4|1966|268|3488|Tuesday|1966Q4|N|N|N|2439431|2439734|2439066|2439339|N|N|N|N|N| +2439432|AAAAAAAAIAJDFCAA|1966-11-02|802|3488|268|1966|3|11|2|4|1966|268|3488|Wednesday|1966Q4|N|N|N|2439431|2439734|2439067|2439340|N|N|N|N|N| +2439433|AAAAAAAAJAJDFCAA|1966-11-03|802|3488|268|1966|4|11|3|4|1966|268|3488|Thursday|1966Q4|N|N|N|2439431|2439734|2439068|2439341|N|N|N|N|N| +2439434|AAAAAAAAKAJDFCAA|1966-11-04|802|3488|268|1966|5|11|4|4|1966|268|3488|Friday|1966Q4|N|Y|N|2439431|2439734|2439069|2439342|N|N|N|N|N| +2439435|AAAAAAAALAJDFCAA|1966-11-05|802|3488|268|1966|6|11|5|4|1966|268|3488|Saturday|1966Q4|N|Y|N|2439431|2439734|2439070|2439343|N|N|N|N|N| +2439436|AAAAAAAAMAJDFCAA|1966-11-06|802|3488|268|1966|0|11|6|4|1966|268|3488|Sunday|1966Q4|N|N|N|2439431|2439734|2439071|2439344|N|N|N|N|N| +2439437|AAAAAAAANAJDFCAA|1966-11-07|802|3488|268|1966|1|11|7|4|1966|268|3488|Monday|1966Q4|N|N|N|2439431|2439734|2439072|2439345|N|N|N|N|N| +2439438|AAAAAAAAOAJDFCAA|1966-11-08|802|3489|268|1966|2|11|8|4|1966|268|3489|Tuesday|1966Q4|N|N|N|2439431|2439734|2439073|2439346|N|N|N|N|N| +2439439|AAAAAAAAPAJDFCAA|1966-11-09|802|3489|268|1966|3|11|9|4|1966|268|3489|Wednesday|1966Q4|N|N|N|2439431|2439734|2439074|2439347|N|N|N|N|N| +2439440|AAAAAAAAABJDFCAA|1966-11-10|802|3489|268|1966|4|11|10|4|1966|268|3489|Thursday|1966Q4|N|N|N|2439431|2439734|2439075|2439348|N|N|N|N|N| +2439441|AAAAAAAABBJDFCAA|1966-11-11|802|3489|268|1966|5|11|11|4|1966|268|3489|Friday|1966Q4|N|Y|N|2439431|2439734|2439076|2439349|N|N|N|N|N| +2439442|AAAAAAAACBJDFCAA|1966-11-12|802|3489|268|1966|6|11|12|4|1966|268|3489|Saturday|1966Q4|N|Y|N|2439431|2439734|2439077|2439350|N|N|N|N|N| +2439443|AAAAAAAADBJDFCAA|1966-11-13|802|3489|268|1966|0|11|13|4|1966|268|3489|Sunday|1966Q4|N|N|N|2439431|2439734|2439078|2439351|N|N|N|N|N| +2439444|AAAAAAAAEBJDFCAA|1966-11-14|802|3489|268|1966|1|11|14|4|1966|268|3489|Monday|1966Q4|N|N|N|2439431|2439734|2439079|2439352|N|N|N|N|N| +2439445|AAAAAAAAFBJDFCAA|1966-11-15|802|3490|268|1966|2|11|15|4|1966|268|3490|Tuesday|1966Q4|N|N|N|2439431|2439734|2439080|2439353|N|N|N|N|N| +2439446|AAAAAAAAGBJDFCAA|1966-11-16|802|3490|268|1966|3|11|16|4|1966|268|3490|Wednesday|1966Q4|N|N|N|2439431|2439734|2439081|2439354|N|N|N|N|N| +2439447|AAAAAAAAHBJDFCAA|1966-11-17|802|3490|268|1966|4|11|17|4|1966|268|3490|Thursday|1966Q4|N|N|N|2439431|2439734|2439082|2439355|N|N|N|N|N| +2439448|AAAAAAAAIBJDFCAA|1966-11-18|802|3490|268|1966|5|11|18|4|1966|268|3490|Friday|1966Q4|N|Y|N|2439431|2439734|2439083|2439356|N|N|N|N|N| +2439449|AAAAAAAAJBJDFCAA|1966-11-19|802|3490|268|1966|6|11|19|4|1966|268|3490|Saturday|1966Q4|N|Y|N|2439431|2439734|2439084|2439357|N|N|N|N|N| +2439450|AAAAAAAAKBJDFCAA|1966-11-20|802|3490|268|1966|0|11|20|4|1966|268|3490|Sunday|1966Q4|N|N|N|2439431|2439734|2439085|2439358|N|N|N|N|N| +2439451|AAAAAAAALBJDFCAA|1966-11-21|802|3490|268|1966|1|11|21|4|1966|268|3490|Monday|1966Q4|N|N|N|2439431|2439734|2439086|2439359|N|N|N|N|N| +2439452|AAAAAAAAMBJDFCAA|1966-11-22|802|3491|268|1966|2|11|22|4|1966|268|3491|Tuesday|1966Q4|N|N|N|2439431|2439734|2439087|2439360|N|N|N|N|N| +2439453|AAAAAAAANBJDFCAA|1966-11-23|802|3491|268|1966|3|11|23|4|1966|268|3491|Wednesday|1966Q4|N|N|N|2439431|2439734|2439088|2439361|N|N|N|N|N| +2439454|AAAAAAAAOBJDFCAA|1966-11-24|802|3491|268|1966|4|11|24|4|1966|268|3491|Thursday|1966Q4|N|N|N|2439431|2439734|2439089|2439362|N|N|N|N|N| +2439455|AAAAAAAAPBJDFCAA|1966-11-25|802|3491|268|1966|5|11|25|4|1966|268|3491|Friday|1966Q4|N|Y|N|2439431|2439734|2439090|2439363|N|N|N|N|N| +2439456|AAAAAAAAACJDFCAA|1966-11-26|802|3491|268|1966|6|11|26|4|1966|268|3491|Saturday|1966Q4|N|Y|N|2439431|2439734|2439091|2439364|N|N|N|N|N| +2439457|AAAAAAAABCJDFCAA|1966-11-27|802|3491|268|1966|0|11|27|4|1966|268|3491|Sunday|1966Q4|N|N|N|2439431|2439734|2439092|2439365|N|N|N|N|N| +2439458|AAAAAAAACCJDFCAA|1966-11-28|802|3491|268|1966|1|11|28|4|1966|268|3491|Monday|1966Q4|N|N|N|2439431|2439734|2439093|2439366|N|N|N|N|N| +2439459|AAAAAAAADCJDFCAA|1966-11-29|802|3492|268|1966|2|11|29|4|1966|268|3492|Tuesday|1966Q4|N|N|N|2439431|2439734|2439094|2439367|N|N|N|N|N| +2439460|AAAAAAAAECJDFCAA|1966-11-30|802|3492|268|1966|3|11|30|4|1966|268|3492|Wednesday|1966Q4|N|N|N|2439431|2439734|2439095|2439368|N|N|N|N|N| +2439461|AAAAAAAAFCJDFCAA|1966-12-01|803|3492|269|1966|4|12|1|4|1966|269|3492|Thursday|1966Q4|N|N|N|2439461|2439794|2439096|2439369|N|N|N|N|N| +2439462|AAAAAAAAGCJDFCAA|1966-12-02|803|3492|269|1966|5|12|2|4|1966|269|3492|Friday|1966Q4|N|Y|N|2439461|2439794|2439097|2439370|N|N|N|N|N| +2439463|AAAAAAAAHCJDFCAA|1966-12-03|803|3492|269|1966|6|12|3|4|1966|269|3492|Saturday|1966Q4|N|Y|N|2439461|2439794|2439098|2439371|N|N|N|N|N| +2439464|AAAAAAAAICJDFCAA|1966-12-04|803|3492|269|1966|0|12|4|4|1966|269|3492|Sunday|1966Q4|N|N|N|2439461|2439794|2439099|2439372|N|N|N|N|N| +2439465|AAAAAAAAJCJDFCAA|1966-12-05|803|3492|269|1966|1|12|5|4|1966|269|3492|Monday|1966Q4|N|N|N|2439461|2439794|2439100|2439373|N|N|N|N|N| +2439466|AAAAAAAAKCJDFCAA|1966-12-06|803|3493|269|1966|2|12|6|4|1966|269|3493|Tuesday|1966Q4|N|N|N|2439461|2439794|2439101|2439374|N|N|N|N|N| +2439467|AAAAAAAALCJDFCAA|1966-12-07|803|3493|269|1966|3|12|7|4|1966|269|3493|Wednesday|1966Q4|N|N|N|2439461|2439794|2439102|2439375|N|N|N|N|N| +2439468|AAAAAAAAMCJDFCAA|1966-12-08|803|3493|269|1966|4|12|8|4|1966|269|3493|Thursday|1966Q4|N|N|N|2439461|2439794|2439103|2439376|N|N|N|N|N| +2439469|AAAAAAAANCJDFCAA|1966-12-09|803|3493|269|1966|5|12|9|4|1966|269|3493|Friday|1966Q4|N|Y|N|2439461|2439794|2439104|2439377|N|N|N|N|N| +2439470|AAAAAAAAOCJDFCAA|1966-12-10|803|3493|269|1966|6|12|10|4|1966|269|3493|Saturday|1966Q4|N|Y|N|2439461|2439794|2439105|2439378|N|N|N|N|N| +2439471|AAAAAAAAPCJDFCAA|1966-12-11|803|3493|269|1966|0|12|11|4|1966|269|3493|Sunday|1966Q4|N|N|N|2439461|2439794|2439106|2439379|N|N|N|N|N| +2439472|AAAAAAAAADJDFCAA|1966-12-12|803|3493|269|1966|1|12|12|4|1966|269|3493|Monday|1966Q4|N|N|N|2439461|2439794|2439107|2439380|N|N|N|N|N| +2439473|AAAAAAAABDJDFCAA|1966-12-13|803|3494|269|1966|2|12|13|4|1966|269|3494|Tuesday|1966Q4|N|N|N|2439461|2439794|2439108|2439381|N|N|N|N|N| +2439474|AAAAAAAACDJDFCAA|1966-12-14|803|3494|269|1966|3|12|14|4|1966|269|3494|Wednesday|1966Q4|N|N|N|2439461|2439794|2439109|2439382|N|N|N|N|N| +2439475|AAAAAAAADDJDFCAA|1966-12-15|803|3494|269|1966|4|12|15|4|1966|269|3494|Thursday|1966Q4|N|N|N|2439461|2439794|2439110|2439383|N|N|N|N|N| +2439476|AAAAAAAAEDJDFCAA|1966-12-16|803|3494|269|1966|5|12|16|4|1966|269|3494|Friday|1966Q4|N|Y|N|2439461|2439794|2439111|2439384|N|N|N|N|N| +2439477|AAAAAAAAFDJDFCAA|1966-12-17|803|3494|269|1966|6|12|17|4|1966|269|3494|Saturday|1966Q4|N|Y|N|2439461|2439794|2439112|2439385|N|N|N|N|N| +2439478|AAAAAAAAGDJDFCAA|1966-12-18|803|3494|269|1966|0|12|18|4|1966|269|3494|Sunday|1966Q4|N|N|N|2439461|2439794|2439113|2439386|N|N|N|N|N| +2439479|AAAAAAAAHDJDFCAA|1966-12-19|803|3494|269|1966|1|12|19|4|1966|269|3494|Monday|1966Q4|N|N|N|2439461|2439794|2439114|2439387|N|N|N|N|N| +2439480|AAAAAAAAIDJDFCAA|1966-12-20|803|3495|269|1966|2|12|20|4|1966|269|3495|Tuesday|1966Q4|N|N|N|2439461|2439794|2439115|2439388|N|N|N|N|N| +2439481|AAAAAAAAJDJDFCAA|1966-12-21|803|3495|269|1966|3|12|21|4|1966|269|3495|Wednesday|1966Q4|N|N|N|2439461|2439794|2439116|2439389|N|N|N|N|N| +2439482|AAAAAAAAKDJDFCAA|1966-12-22|803|3495|269|1966|4|12|22|4|1966|269|3495|Thursday|1966Q4|N|N|N|2439461|2439794|2439117|2439390|N|N|N|N|N| +2439483|AAAAAAAALDJDFCAA|1966-12-23|803|3495|269|1966|5|12|23|4|1966|269|3495|Friday|1966Q4|N|Y|N|2439461|2439794|2439118|2439391|N|N|N|N|N| +2439484|AAAAAAAAMDJDFCAA|1966-12-24|803|3495|269|1966|6|12|24|4|1966|269|3495|Saturday|1966Q4|N|Y|N|2439461|2439794|2439119|2439392|N|N|N|N|N| +2439485|AAAAAAAANDJDFCAA|1966-12-25|803|3495|269|1966|0|12|25|4|1966|269|3495|Sunday|1966Q4|N|N|N|2439461|2439794|2439120|2439393|N|N|N|N|N| +2439486|AAAAAAAAODJDFCAA|1966-12-26|803|3495|269|1966|1|12|26|4|1966|269|3495|Monday|1966Q4|Y|N|N|2439461|2439794|2439121|2439394|N|N|N|N|N| +2439487|AAAAAAAAPDJDFCAA|1966-12-27|803|3496|269|1966|2|12|27|4|1966|269|3496|Tuesday|1966Q4|N|N|Y|2439461|2439794|2439122|2439395|N|N|N|N|N| +2439488|AAAAAAAAAEJDFCAA|1966-12-28|803|3496|269|1966|3|12|28|4|1966|269|3496|Wednesday|1966Q4|N|N|N|2439461|2439794|2439123|2439396|N|N|N|N|N| +2439489|AAAAAAAABEJDFCAA|1966-12-29|803|3496|269|1966|4|12|29|4|1966|269|3496|Thursday|1966Q4|N|N|N|2439461|2439794|2439124|2439397|N|N|N|N|N| +2439490|AAAAAAAACEJDFCAA|1966-12-30|803|3496|269|1966|5|12|30|4|1966|269|3496|Friday|1966Q4|N|Y|N|2439461|2439794|2439125|2439398|N|N|N|N|N| +2439491|AAAAAAAADEJDFCAA|1966-12-31|803|3496|269|1966|6|12|31|4|1966|269|3496|Saturday|1966Q4|N|Y|N|2439461|2439794|2439126|2439399|N|N|N|N|N| +2439492|AAAAAAAAEEJDFCAA|1967-01-01|804|3496|269|1967|0|1|1|1|1967|269|3496|Sunday|1967Q1|Y|N|N|2439492|2439491|2439127|2439400|N|N|N|N|N| +2439493|AAAAAAAAFEJDFCAA|1967-01-02|804|3496|269|1967|1|1|2|1|1967|269|3496|Monday|1967Q1|N|N|Y|2439492|2439491|2439128|2439401|N|N|N|N|N| +2439494|AAAAAAAAGEJDFCAA|1967-01-03|804|3497|269|1967|2|1|3|1|1967|269|3497|Tuesday|1967Q1|N|N|N|2439492|2439491|2439129|2439402|N|N|N|N|N| +2439495|AAAAAAAAHEJDFCAA|1967-01-04|804|3497|269|1967|3|1|4|1|1967|269|3497|Wednesday|1967Q1|N|N|N|2439492|2439491|2439130|2439403|N|N|N|N|N| +2439496|AAAAAAAAIEJDFCAA|1967-01-05|804|3497|269|1967|4|1|5|1|1967|269|3497|Thursday|1967Q1|N|N|N|2439492|2439491|2439131|2439404|N|N|N|N|N| +2439497|AAAAAAAAJEJDFCAA|1967-01-06|804|3497|269|1967|5|1|6|1|1967|269|3497|Friday|1967Q1|N|Y|N|2439492|2439491|2439132|2439405|N|N|N|N|N| +2439498|AAAAAAAAKEJDFCAA|1967-01-07|804|3497|269|1967|6|1|7|1|1967|269|3497|Saturday|1967Q1|N|Y|N|2439492|2439491|2439133|2439406|N|N|N|N|N| +2439499|AAAAAAAALEJDFCAA|1967-01-08|804|3497|269|1967|0|1|8|1|1967|269|3497|Sunday|1967Q1|N|N|N|2439492|2439491|2439134|2439407|N|N|N|N|N| +2439500|AAAAAAAAMEJDFCAA|1967-01-09|804|3497|269|1967|1|1|9|1|1967|269|3497|Monday|1967Q1|N|N|N|2439492|2439491|2439135|2439408|N|N|N|N|N| +2439501|AAAAAAAANEJDFCAA|1967-01-10|804|3498|269|1967|2|1|10|1|1967|269|3498|Tuesday|1967Q1|N|N|N|2439492|2439491|2439136|2439409|N|N|N|N|N| +2439502|AAAAAAAAOEJDFCAA|1967-01-11|804|3498|269|1967|3|1|11|1|1967|269|3498|Wednesday|1967Q1|N|N|N|2439492|2439491|2439137|2439410|N|N|N|N|N| +2439503|AAAAAAAAPEJDFCAA|1967-01-12|804|3498|269|1967|4|1|12|1|1967|269|3498|Thursday|1967Q1|N|N|N|2439492|2439491|2439138|2439411|N|N|N|N|N| +2439504|AAAAAAAAAFJDFCAA|1967-01-13|804|3498|269|1967|5|1|13|1|1967|269|3498|Friday|1967Q1|N|Y|N|2439492|2439491|2439139|2439412|N|N|N|N|N| +2439505|AAAAAAAABFJDFCAA|1967-01-14|804|3498|269|1967|6|1|14|1|1967|269|3498|Saturday|1967Q1|N|Y|N|2439492|2439491|2439140|2439413|N|N|N|N|N| +2439506|AAAAAAAACFJDFCAA|1967-01-15|804|3498|269|1967|0|1|15|1|1967|269|3498|Sunday|1967Q1|N|N|N|2439492|2439491|2439141|2439414|N|N|N|N|N| +2439507|AAAAAAAADFJDFCAA|1967-01-16|804|3498|269|1967|1|1|16|1|1967|269|3498|Monday|1967Q1|N|N|N|2439492|2439491|2439142|2439415|N|N|N|N|N| +2439508|AAAAAAAAEFJDFCAA|1967-01-17|804|3499|269|1967|2|1|17|1|1967|269|3499|Tuesday|1967Q1|N|N|N|2439492|2439491|2439143|2439416|N|N|N|N|N| +2439509|AAAAAAAAFFJDFCAA|1967-01-18|804|3499|269|1967|3|1|18|1|1967|269|3499|Wednesday|1967Q1|N|N|N|2439492|2439491|2439144|2439417|N|N|N|N|N| +2439510|AAAAAAAAGFJDFCAA|1967-01-19|804|3499|269|1967|4|1|19|1|1967|269|3499|Thursday|1967Q1|N|N|N|2439492|2439491|2439145|2439418|N|N|N|N|N| +2439511|AAAAAAAAHFJDFCAA|1967-01-20|804|3499|269|1967|5|1|20|1|1967|269|3499|Friday|1967Q1|N|Y|N|2439492|2439491|2439146|2439419|N|N|N|N|N| +2439512|AAAAAAAAIFJDFCAA|1967-01-21|804|3499|269|1967|6|1|21|1|1967|269|3499|Saturday|1967Q1|N|Y|N|2439492|2439491|2439147|2439420|N|N|N|N|N| +2439513|AAAAAAAAJFJDFCAA|1967-01-22|804|3499|269|1967|0|1|22|1|1967|269|3499|Sunday|1967Q1|N|N|N|2439492|2439491|2439148|2439421|N|N|N|N|N| +2439514|AAAAAAAAKFJDFCAA|1967-01-23|804|3499|269|1967|1|1|23|1|1967|269|3499|Monday|1967Q1|N|N|N|2439492|2439491|2439149|2439422|N|N|N|N|N| +2439515|AAAAAAAALFJDFCAA|1967-01-24|804|3500|269|1967|2|1|24|1|1967|269|3500|Tuesday|1967Q1|N|N|N|2439492|2439491|2439150|2439423|N|N|N|N|N| +2439516|AAAAAAAAMFJDFCAA|1967-01-25|804|3500|269|1967|3|1|25|1|1967|269|3500|Wednesday|1967Q1|N|N|N|2439492|2439491|2439151|2439424|N|N|N|N|N| +2439517|AAAAAAAANFJDFCAA|1967-01-26|804|3500|269|1967|4|1|26|1|1967|269|3500|Thursday|1967Q1|N|N|N|2439492|2439491|2439152|2439425|N|N|N|N|N| +2439518|AAAAAAAAOFJDFCAA|1967-01-27|804|3500|269|1967|5|1|27|1|1967|269|3500|Friday|1967Q1|N|Y|N|2439492|2439491|2439153|2439426|N|N|N|N|N| +2439519|AAAAAAAAPFJDFCAA|1967-01-28|804|3500|269|1967|6|1|28|1|1967|269|3500|Saturday|1967Q1|N|Y|N|2439492|2439491|2439154|2439427|N|N|N|N|N| +2439520|AAAAAAAAAGJDFCAA|1967-01-29|804|3500|269|1967|0|1|29|1|1967|269|3500|Sunday|1967Q1|N|N|N|2439492|2439491|2439155|2439428|N|N|N|N|N| +2439521|AAAAAAAABGJDFCAA|1967-01-30|804|3500|269|1967|1|1|30|1|1967|269|3500|Monday|1967Q1|N|N|N|2439492|2439491|2439156|2439429|N|N|N|N|N| +2439522|AAAAAAAACGJDFCAA|1967-01-31|804|3501|269|1967|2|1|31|1|1967|269|3501|Tuesday|1967Q1|N|N|N|2439492|2439491|2439157|2439430|N|N|N|N|N| +2439523|AAAAAAAADGJDFCAA|1967-02-01|805|3501|269|1967|3|2|1|1|1967|269|3501|Wednesday|1967Q1|N|N|N|2439523|2439553|2439158|2439431|N|N|N|N|N| +2439524|AAAAAAAAEGJDFCAA|1967-02-02|805|3501|269|1967|4|2|2|1|1967|269|3501|Thursday|1967Q1|N|N|N|2439523|2439553|2439159|2439432|N|N|N|N|N| +2439525|AAAAAAAAFGJDFCAA|1967-02-03|805|3501|269|1967|5|2|3|1|1967|269|3501|Friday|1967Q1|N|Y|N|2439523|2439553|2439160|2439433|N|N|N|N|N| +2439526|AAAAAAAAGGJDFCAA|1967-02-04|805|3501|269|1967|6|2|4|1|1967|269|3501|Saturday|1967Q1|N|Y|N|2439523|2439553|2439161|2439434|N|N|N|N|N| +2439527|AAAAAAAAHGJDFCAA|1967-02-05|805|3501|269|1967|0|2|5|1|1967|269|3501|Sunday|1967Q1|N|N|N|2439523|2439553|2439162|2439435|N|N|N|N|N| +2439528|AAAAAAAAIGJDFCAA|1967-02-06|805|3501|269|1967|1|2|6|1|1967|269|3501|Monday|1967Q1|N|N|N|2439523|2439553|2439163|2439436|N|N|N|N|N| +2439529|AAAAAAAAJGJDFCAA|1967-02-07|805|3502|269|1967|2|2|7|1|1967|269|3502|Tuesday|1967Q1|N|N|N|2439523|2439553|2439164|2439437|N|N|N|N|N| +2439530|AAAAAAAAKGJDFCAA|1967-02-08|805|3502|269|1967|3|2|8|1|1967|269|3502|Wednesday|1967Q1|N|N|N|2439523|2439553|2439165|2439438|N|N|N|N|N| +2439531|AAAAAAAALGJDFCAA|1967-02-09|805|3502|269|1967|4|2|9|1|1967|269|3502|Thursday|1967Q1|N|N|N|2439523|2439553|2439166|2439439|N|N|N|N|N| +2439532|AAAAAAAAMGJDFCAA|1967-02-10|805|3502|269|1967|5|2|10|1|1967|269|3502|Friday|1967Q1|N|Y|N|2439523|2439553|2439167|2439440|N|N|N|N|N| +2439533|AAAAAAAANGJDFCAA|1967-02-11|805|3502|269|1967|6|2|11|1|1967|269|3502|Saturday|1967Q1|N|Y|N|2439523|2439553|2439168|2439441|N|N|N|N|N| +2439534|AAAAAAAAOGJDFCAA|1967-02-12|805|3502|269|1967|0|2|12|1|1967|269|3502|Sunday|1967Q1|N|N|N|2439523|2439553|2439169|2439442|N|N|N|N|N| +2439535|AAAAAAAAPGJDFCAA|1967-02-13|805|3502|269|1967|1|2|13|1|1967|269|3502|Monday|1967Q1|N|N|N|2439523|2439553|2439170|2439443|N|N|N|N|N| +2439536|AAAAAAAAAHJDFCAA|1967-02-14|805|3503|269|1967|2|2|14|1|1967|269|3503|Tuesday|1967Q1|N|N|N|2439523|2439553|2439171|2439444|N|N|N|N|N| +2439537|AAAAAAAABHJDFCAA|1967-02-15|805|3503|269|1967|3|2|15|1|1967|269|3503|Wednesday|1967Q1|N|N|N|2439523|2439553|2439172|2439445|N|N|N|N|N| +2439538|AAAAAAAACHJDFCAA|1967-02-16|805|3503|269|1967|4|2|16|1|1967|269|3503|Thursday|1967Q1|N|N|N|2439523|2439553|2439173|2439446|N|N|N|N|N| +2439539|AAAAAAAADHJDFCAA|1967-02-17|805|3503|269|1967|5|2|17|1|1967|269|3503|Friday|1967Q1|N|Y|N|2439523|2439553|2439174|2439447|N|N|N|N|N| +2439540|AAAAAAAAEHJDFCAA|1967-02-18|805|3503|269|1967|6|2|18|1|1967|269|3503|Saturday|1967Q1|N|Y|N|2439523|2439553|2439175|2439448|N|N|N|N|N| +2439541|AAAAAAAAFHJDFCAA|1967-02-19|805|3503|269|1967|0|2|19|1|1967|269|3503|Sunday|1967Q1|N|N|N|2439523|2439553|2439176|2439449|N|N|N|N|N| +2439542|AAAAAAAAGHJDFCAA|1967-02-20|805|3503|269|1967|1|2|20|1|1967|269|3503|Monday|1967Q1|N|N|N|2439523|2439553|2439177|2439450|N|N|N|N|N| +2439543|AAAAAAAAHHJDFCAA|1967-02-21|805|3504|269|1967|2|2|21|1|1967|269|3504|Tuesday|1967Q1|N|N|N|2439523|2439553|2439178|2439451|N|N|N|N|N| +2439544|AAAAAAAAIHJDFCAA|1967-02-22|805|3504|269|1967|3|2|22|1|1967|269|3504|Wednesday|1967Q1|N|N|N|2439523|2439553|2439179|2439452|N|N|N|N|N| +2439545|AAAAAAAAJHJDFCAA|1967-02-23|805|3504|269|1967|4|2|23|1|1967|269|3504|Thursday|1967Q1|N|N|N|2439523|2439553|2439180|2439453|N|N|N|N|N| +2439546|AAAAAAAAKHJDFCAA|1967-02-24|805|3504|269|1967|5|2|24|1|1967|269|3504|Friday|1967Q1|N|Y|N|2439523|2439553|2439181|2439454|N|N|N|N|N| +2439547|AAAAAAAALHJDFCAA|1967-02-25|805|3504|269|1967|6|2|25|1|1967|269|3504|Saturday|1967Q1|N|Y|N|2439523|2439553|2439182|2439455|N|N|N|N|N| +2439548|AAAAAAAAMHJDFCAA|1967-02-26|805|3504|269|1967|0|2|26|1|1967|269|3504|Sunday|1967Q1|N|N|N|2439523|2439553|2439183|2439456|N|N|N|N|N| +2439549|AAAAAAAANHJDFCAA|1967-02-27|805|3504|269|1967|1|2|27|1|1967|269|3504|Monday|1967Q1|N|N|N|2439523|2439553|2439184|2439457|N|N|N|N|N| +2439550|AAAAAAAAOHJDFCAA|1967-02-28|805|3505|269|1967|2|2|28|1|1967|269|3505|Tuesday|1967Q1|N|N|N|2439523|2439553|2439185|2439458|N|N|N|N|N| +2439551|AAAAAAAAPHJDFCAA|1967-03-01|806|3505|270|1967|3|3|1|1|1967|270|3505|Wednesday|1967Q1|N|N|N|2439551|2439609|2439186|2439459|N|N|N|N|N| +2439552|AAAAAAAAAIJDFCAA|1967-03-02|806|3505|270|1967|4|3|2|1|1967|270|3505|Thursday|1967Q1|N|N|N|2439551|2439609|2439187|2439460|N|N|N|N|N| +2439553|AAAAAAAABIJDFCAA|1967-03-03|806|3505|270|1967|5|3|3|1|1967|270|3505|Friday|1967Q1|N|Y|N|2439551|2439609|2439188|2439461|N|N|N|N|N| +2439554|AAAAAAAACIJDFCAA|1967-03-04|806|3505|270|1967|6|3|4|1|1967|270|3505|Saturday|1967Q1|N|Y|N|2439551|2439609|2439189|2439462|N|N|N|N|N| +2439555|AAAAAAAADIJDFCAA|1967-03-05|806|3505|270|1967|0|3|5|1|1967|270|3505|Sunday|1967Q1|N|N|N|2439551|2439609|2439190|2439463|N|N|N|N|N| +2439556|AAAAAAAAEIJDFCAA|1967-03-06|806|3505|270|1967|1|3|6|1|1967|270|3505|Monday|1967Q1|N|N|N|2439551|2439609|2439191|2439464|N|N|N|N|N| +2439557|AAAAAAAAFIJDFCAA|1967-03-07|806|3506|270|1967|2|3|7|1|1967|270|3506|Tuesday|1967Q1|N|N|N|2439551|2439609|2439192|2439465|N|N|N|N|N| +2439558|AAAAAAAAGIJDFCAA|1967-03-08|806|3506|270|1967|3|3|8|1|1967|270|3506|Wednesday|1967Q1|N|N|N|2439551|2439609|2439193|2439466|N|N|N|N|N| +2439559|AAAAAAAAHIJDFCAA|1967-03-09|806|3506|270|1967|4|3|9|1|1967|270|3506|Thursday|1967Q1|N|N|N|2439551|2439609|2439194|2439467|N|N|N|N|N| +2439560|AAAAAAAAIIJDFCAA|1967-03-10|806|3506|270|1967|5|3|10|1|1967|270|3506|Friday|1967Q1|N|Y|N|2439551|2439609|2439195|2439468|N|N|N|N|N| +2439561|AAAAAAAAJIJDFCAA|1967-03-11|806|3506|270|1967|6|3|11|1|1967|270|3506|Saturday|1967Q1|N|Y|N|2439551|2439609|2439196|2439469|N|N|N|N|N| +2439562|AAAAAAAAKIJDFCAA|1967-03-12|806|3506|270|1967|0|3|12|1|1967|270|3506|Sunday|1967Q1|N|N|N|2439551|2439609|2439197|2439470|N|N|N|N|N| +2439563|AAAAAAAALIJDFCAA|1967-03-13|806|3506|270|1967|1|3|13|1|1967|270|3506|Monday|1967Q1|N|N|N|2439551|2439609|2439198|2439471|N|N|N|N|N| +2439564|AAAAAAAAMIJDFCAA|1967-03-14|806|3507|270|1967|2|3|14|1|1967|270|3507|Tuesday|1967Q1|N|N|N|2439551|2439609|2439199|2439472|N|N|N|N|N| +2439565|AAAAAAAANIJDFCAA|1967-03-15|806|3507|270|1967|3|3|15|1|1967|270|3507|Wednesday|1967Q1|N|N|N|2439551|2439609|2439200|2439473|N|N|N|N|N| +2439566|AAAAAAAAOIJDFCAA|1967-03-16|806|3507|270|1967|4|3|16|1|1967|270|3507|Thursday|1967Q1|N|N|N|2439551|2439609|2439201|2439474|N|N|N|N|N| +2439567|AAAAAAAAPIJDFCAA|1967-03-17|806|3507|270|1967|5|3|17|1|1967|270|3507|Friday|1967Q1|N|Y|N|2439551|2439609|2439202|2439475|N|N|N|N|N| +2439568|AAAAAAAAAJJDFCAA|1967-03-18|806|3507|270|1967|6|3|18|1|1967|270|3507|Saturday|1967Q1|N|Y|N|2439551|2439609|2439203|2439476|N|N|N|N|N| +2439569|AAAAAAAABJJDFCAA|1967-03-19|806|3507|270|1967|0|3|19|1|1967|270|3507|Sunday|1967Q1|N|N|N|2439551|2439609|2439204|2439477|N|N|N|N|N| +2439570|AAAAAAAACJJDFCAA|1967-03-20|806|3507|270|1967|1|3|20|1|1967|270|3507|Monday|1967Q1|N|N|N|2439551|2439609|2439205|2439478|N|N|N|N|N| +2439571|AAAAAAAADJJDFCAA|1967-03-21|806|3508|270|1967|2|3|21|1|1967|270|3508|Tuesday|1967Q1|N|N|N|2439551|2439609|2439206|2439479|N|N|N|N|N| +2439572|AAAAAAAAEJJDFCAA|1967-03-22|806|3508|270|1967|3|3|22|1|1967|270|3508|Wednesday|1967Q1|N|N|N|2439551|2439609|2439207|2439480|N|N|N|N|N| +2439573|AAAAAAAAFJJDFCAA|1967-03-23|806|3508|270|1967|4|3|23|1|1967|270|3508|Thursday|1967Q1|N|N|N|2439551|2439609|2439208|2439481|N|N|N|N|N| +2439574|AAAAAAAAGJJDFCAA|1967-03-24|806|3508|270|1967|5|3|24|1|1967|270|3508|Friday|1967Q1|N|Y|N|2439551|2439609|2439209|2439482|N|N|N|N|N| +2439575|AAAAAAAAHJJDFCAA|1967-03-25|806|3508|270|1967|6|3|25|1|1967|270|3508|Saturday|1967Q1|N|Y|N|2439551|2439609|2439210|2439483|N|N|N|N|N| +2439576|AAAAAAAAIJJDFCAA|1967-03-26|806|3508|270|1967|0|3|26|1|1967|270|3508|Sunday|1967Q1|N|N|N|2439551|2439609|2439211|2439484|N|N|N|N|N| +2439577|AAAAAAAAJJJDFCAA|1967-03-27|806|3508|270|1967|1|3|27|1|1967|270|3508|Monday|1967Q1|N|N|N|2439551|2439609|2439212|2439485|N|N|N|N|N| +2439578|AAAAAAAAKJJDFCAA|1967-03-28|806|3509|270|1967|2|3|28|1|1967|270|3509|Tuesday|1967Q1|N|N|N|2439551|2439609|2439213|2439486|N|N|N|N|N| +2439579|AAAAAAAALJJDFCAA|1967-03-29|806|3509|270|1967|3|3|29|1|1967|270|3509|Wednesday|1967Q1|N|N|N|2439551|2439609|2439214|2439487|N|N|N|N|N| +2439580|AAAAAAAAMJJDFCAA|1967-03-30|806|3509|270|1967|4|3|30|1|1967|270|3509|Thursday|1967Q1|N|N|N|2439551|2439609|2439215|2439488|N|N|N|N|N| +2439581|AAAAAAAANJJDFCAA|1967-03-31|806|3509|270|1967|5|3|31|1|1967|270|3509|Friday|1967Q1|N|Y|N|2439551|2439609|2439216|2439489|N|N|N|N|N| +2439582|AAAAAAAAOJJDFCAA|1967-04-01|807|3509|270|1967|6|4|1|1|1967|270|3509|Saturday|1967Q1|N|Y|N|2439582|2439671|2439217|2439492|N|N|N|N|N| +2439583|AAAAAAAAPJJDFCAA|1967-04-02|807|3509|270|1967|0|4|2|2|1967|270|3509|Sunday|1967Q2|N|N|N|2439582|2439671|2439218|2439493|N|N|N|N|N| +2439584|AAAAAAAAAKJDFCAA|1967-04-03|807|3509|270|1967|1|4|3|2|1967|270|3509|Monday|1967Q2|N|N|N|2439582|2439671|2439219|2439494|N|N|N|N|N| +2439585|AAAAAAAABKJDFCAA|1967-04-04|807|3510|270|1967|2|4|4|2|1967|270|3510|Tuesday|1967Q2|N|N|N|2439582|2439671|2439220|2439495|N|N|N|N|N| +2439586|AAAAAAAACKJDFCAA|1967-04-05|807|3510|270|1967|3|4|5|2|1967|270|3510|Wednesday|1967Q2|N|N|N|2439582|2439671|2439221|2439496|N|N|N|N|N| +2439587|AAAAAAAADKJDFCAA|1967-04-06|807|3510|270|1967|4|4|6|2|1967|270|3510|Thursday|1967Q2|N|N|N|2439582|2439671|2439222|2439497|N|N|N|N|N| +2439588|AAAAAAAAEKJDFCAA|1967-04-07|807|3510|270|1967|5|4|7|2|1967|270|3510|Friday|1967Q2|N|Y|N|2439582|2439671|2439223|2439498|N|N|N|N|N| +2439589|AAAAAAAAFKJDFCAA|1967-04-08|807|3510|270|1967|6|4|8|2|1967|270|3510|Saturday|1967Q2|N|Y|N|2439582|2439671|2439224|2439499|N|N|N|N|N| +2439590|AAAAAAAAGKJDFCAA|1967-04-09|807|3510|270|1967|0|4|9|2|1967|270|3510|Sunday|1967Q2|N|N|N|2439582|2439671|2439225|2439500|N|N|N|N|N| +2439591|AAAAAAAAHKJDFCAA|1967-04-10|807|3510|270|1967|1|4|10|2|1967|270|3510|Monday|1967Q2|N|N|N|2439582|2439671|2439226|2439501|N|N|N|N|N| +2439592|AAAAAAAAIKJDFCAA|1967-04-11|807|3511|270|1967|2|4|11|2|1967|270|3511|Tuesday|1967Q2|N|N|N|2439582|2439671|2439227|2439502|N|N|N|N|N| +2439593|AAAAAAAAJKJDFCAA|1967-04-12|807|3511|270|1967|3|4|12|2|1967|270|3511|Wednesday|1967Q2|N|N|N|2439582|2439671|2439228|2439503|N|N|N|N|N| +2439594|AAAAAAAAKKJDFCAA|1967-04-13|807|3511|270|1967|4|4|13|2|1967|270|3511|Thursday|1967Q2|N|N|N|2439582|2439671|2439229|2439504|N|N|N|N|N| +2439595|AAAAAAAALKJDFCAA|1967-04-14|807|3511|270|1967|5|4|14|2|1967|270|3511|Friday|1967Q2|N|Y|N|2439582|2439671|2439230|2439505|N|N|N|N|N| +2439596|AAAAAAAAMKJDFCAA|1967-04-15|807|3511|270|1967|6|4|15|2|1967|270|3511|Saturday|1967Q2|N|Y|N|2439582|2439671|2439231|2439506|N|N|N|N|N| +2439597|AAAAAAAANKJDFCAA|1967-04-16|807|3511|270|1967|0|4|16|2|1967|270|3511|Sunday|1967Q2|N|N|N|2439582|2439671|2439232|2439507|N|N|N|N|N| +2439598|AAAAAAAAOKJDFCAA|1967-04-17|807|3511|270|1967|1|4|17|2|1967|270|3511|Monday|1967Q2|N|N|N|2439582|2439671|2439233|2439508|N|N|N|N|N| +2439599|AAAAAAAAPKJDFCAA|1967-04-18|807|3512|270|1967|2|4|18|2|1967|270|3512|Tuesday|1967Q2|N|N|N|2439582|2439671|2439234|2439509|N|N|N|N|N| +2439600|AAAAAAAAALJDFCAA|1967-04-19|807|3512|270|1967|3|4|19|2|1967|270|3512|Wednesday|1967Q2|N|N|N|2439582|2439671|2439235|2439510|N|N|N|N|N| +2439601|AAAAAAAABLJDFCAA|1967-04-20|807|3512|270|1967|4|4|20|2|1967|270|3512|Thursday|1967Q2|N|N|N|2439582|2439671|2439236|2439511|N|N|N|N|N| +2439602|AAAAAAAACLJDFCAA|1967-04-21|807|3512|270|1967|5|4|21|2|1967|270|3512|Friday|1967Q2|N|Y|N|2439582|2439671|2439237|2439512|N|N|N|N|N| +2439603|AAAAAAAADLJDFCAA|1967-04-22|807|3512|270|1967|6|4|22|2|1967|270|3512|Saturday|1967Q2|N|Y|N|2439582|2439671|2439238|2439513|N|N|N|N|N| +2439604|AAAAAAAAELJDFCAA|1967-04-23|807|3512|270|1967|0|4|23|2|1967|270|3512|Sunday|1967Q2|N|N|N|2439582|2439671|2439239|2439514|N|N|N|N|N| +2439605|AAAAAAAAFLJDFCAA|1967-04-24|807|3512|270|1967|1|4|24|2|1967|270|3512|Monday|1967Q2|N|N|N|2439582|2439671|2439240|2439515|N|N|N|N|N| +2439606|AAAAAAAAGLJDFCAA|1967-04-25|807|3513|270|1967|2|4|25|2|1967|270|3513|Tuesday|1967Q2|N|N|N|2439582|2439671|2439241|2439516|N|N|N|N|N| +2439607|AAAAAAAAHLJDFCAA|1967-04-26|807|3513|270|1967|3|4|26|2|1967|270|3513|Wednesday|1967Q2|N|N|N|2439582|2439671|2439242|2439517|N|N|N|N|N| +2439608|AAAAAAAAILJDFCAA|1967-04-27|807|3513|270|1967|4|4|27|2|1967|270|3513|Thursday|1967Q2|N|N|N|2439582|2439671|2439243|2439518|N|N|N|N|N| +2439609|AAAAAAAAJLJDFCAA|1967-04-28|807|3513|270|1967|5|4|28|2|1967|270|3513|Friday|1967Q2|N|Y|N|2439582|2439671|2439244|2439519|N|N|N|N|N| +2439610|AAAAAAAAKLJDFCAA|1967-04-29|807|3513|270|1967|6|4|29|2|1967|270|3513|Saturday|1967Q2|N|Y|N|2439582|2439671|2439245|2439520|N|N|N|N|N| +2439611|AAAAAAAALLJDFCAA|1967-04-30|807|3513|270|1967|0|4|30|2|1967|270|3513|Sunday|1967Q2|N|N|N|2439582|2439671|2439246|2439521|N|N|N|N|N| +2439612|AAAAAAAAMLJDFCAA|1967-05-01|808|3513|270|1967|1|5|1|2|1967|270|3513|Monday|1967Q2|N|N|N|2439612|2439731|2439247|2439522|N|N|N|N|N| +2439613|AAAAAAAANLJDFCAA|1967-05-02|808|3514|270|1967|2|5|2|2|1967|270|3514|Tuesday|1967Q2|N|N|N|2439612|2439731|2439248|2439523|N|N|N|N|N| +2439614|AAAAAAAAOLJDFCAA|1967-05-03|808|3514|270|1967|3|5|3|2|1967|270|3514|Wednesday|1967Q2|N|N|N|2439612|2439731|2439249|2439524|N|N|N|N|N| +2439615|AAAAAAAAPLJDFCAA|1967-05-04|808|3514|270|1967|4|5|4|2|1967|270|3514|Thursday|1967Q2|N|N|N|2439612|2439731|2439250|2439525|N|N|N|N|N| +2439616|AAAAAAAAAMJDFCAA|1967-05-05|808|3514|270|1967|5|5|5|2|1967|270|3514|Friday|1967Q2|N|Y|N|2439612|2439731|2439251|2439526|N|N|N|N|N| +2439617|AAAAAAAABMJDFCAA|1967-05-06|808|3514|270|1967|6|5|6|2|1967|270|3514|Saturday|1967Q2|N|Y|N|2439612|2439731|2439252|2439527|N|N|N|N|N| +2439618|AAAAAAAACMJDFCAA|1967-05-07|808|3514|270|1967|0|5|7|2|1967|270|3514|Sunday|1967Q2|N|N|N|2439612|2439731|2439253|2439528|N|N|N|N|N| +2439619|AAAAAAAADMJDFCAA|1967-05-08|808|3514|270|1967|1|5|8|2|1967|270|3514|Monday|1967Q2|N|N|N|2439612|2439731|2439254|2439529|N|N|N|N|N| +2439620|AAAAAAAAEMJDFCAA|1967-05-09|808|3515|270|1967|2|5|9|2|1967|270|3515|Tuesday|1967Q2|N|N|N|2439612|2439731|2439255|2439530|N|N|N|N|N| +2439621|AAAAAAAAFMJDFCAA|1967-05-10|808|3515|270|1967|3|5|10|2|1967|270|3515|Wednesday|1967Q2|N|N|N|2439612|2439731|2439256|2439531|N|N|N|N|N| +2439622|AAAAAAAAGMJDFCAA|1967-05-11|808|3515|270|1967|4|5|11|2|1967|270|3515|Thursday|1967Q2|N|N|N|2439612|2439731|2439257|2439532|N|N|N|N|N| +2439623|AAAAAAAAHMJDFCAA|1967-05-12|808|3515|270|1967|5|5|12|2|1967|270|3515|Friday|1967Q2|N|Y|N|2439612|2439731|2439258|2439533|N|N|N|N|N| +2439624|AAAAAAAAIMJDFCAA|1967-05-13|808|3515|270|1967|6|5|13|2|1967|270|3515|Saturday|1967Q2|N|Y|N|2439612|2439731|2439259|2439534|N|N|N|N|N| +2439625|AAAAAAAAJMJDFCAA|1967-05-14|808|3515|270|1967|0|5|14|2|1967|270|3515|Sunday|1967Q2|N|N|N|2439612|2439731|2439260|2439535|N|N|N|N|N| +2439626|AAAAAAAAKMJDFCAA|1967-05-15|808|3515|270|1967|1|5|15|2|1967|270|3515|Monday|1967Q2|N|N|N|2439612|2439731|2439261|2439536|N|N|N|N|N| +2439627|AAAAAAAALMJDFCAA|1967-05-16|808|3516|270|1967|2|5|16|2|1967|270|3516|Tuesday|1967Q2|N|N|N|2439612|2439731|2439262|2439537|N|N|N|N|N| +2439628|AAAAAAAAMMJDFCAA|1967-05-17|808|3516|270|1967|3|5|17|2|1967|270|3516|Wednesday|1967Q2|N|N|N|2439612|2439731|2439263|2439538|N|N|N|N|N| +2439629|AAAAAAAANMJDFCAA|1967-05-18|808|3516|270|1967|4|5|18|2|1967|270|3516|Thursday|1967Q2|N|N|N|2439612|2439731|2439264|2439539|N|N|N|N|N| +2439630|AAAAAAAAOMJDFCAA|1967-05-19|808|3516|270|1967|5|5|19|2|1967|270|3516|Friday|1967Q2|N|Y|N|2439612|2439731|2439265|2439540|N|N|N|N|N| +2439631|AAAAAAAAPMJDFCAA|1967-05-20|808|3516|270|1967|6|5|20|2|1967|270|3516|Saturday|1967Q2|N|Y|N|2439612|2439731|2439266|2439541|N|N|N|N|N| +2439632|AAAAAAAAANJDFCAA|1967-05-21|808|3516|270|1967|0|5|21|2|1967|270|3516|Sunday|1967Q2|N|N|N|2439612|2439731|2439267|2439542|N|N|N|N|N| +2439633|AAAAAAAABNJDFCAA|1967-05-22|808|3516|270|1967|1|5|22|2|1967|270|3516|Monday|1967Q2|N|N|N|2439612|2439731|2439268|2439543|N|N|N|N|N| +2439634|AAAAAAAACNJDFCAA|1967-05-23|808|3517|270|1967|2|5|23|2|1967|270|3517|Tuesday|1967Q2|N|N|N|2439612|2439731|2439269|2439544|N|N|N|N|N| +2439635|AAAAAAAADNJDFCAA|1967-05-24|808|3517|270|1967|3|5|24|2|1967|270|3517|Wednesday|1967Q2|N|N|N|2439612|2439731|2439270|2439545|N|N|N|N|N| +2439636|AAAAAAAAENJDFCAA|1967-05-25|808|3517|270|1967|4|5|25|2|1967|270|3517|Thursday|1967Q2|N|N|N|2439612|2439731|2439271|2439546|N|N|N|N|N| +2439637|AAAAAAAAFNJDFCAA|1967-05-26|808|3517|270|1967|5|5|26|2|1967|270|3517|Friday|1967Q2|N|Y|N|2439612|2439731|2439272|2439547|N|N|N|N|N| +2439638|AAAAAAAAGNJDFCAA|1967-05-27|808|3517|270|1967|6|5|27|2|1967|270|3517|Saturday|1967Q2|N|Y|N|2439612|2439731|2439273|2439548|N|N|N|N|N| +2439639|AAAAAAAAHNJDFCAA|1967-05-28|808|3517|270|1967|0|5|28|2|1967|270|3517|Sunday|1967Q2|N|N|N|2439612|2439731|2439274|2439549|N|N|N|N|N| +2439640|AAAAAAAAINJDFCAA|1967-05-29|808|3517|270|1967|1|5|29|2|1967|270|3517|Monday|1967Q2|N|N|N|2439612|2439731|2439275|2439550|N|N|N|N|N| +2439641|AAAAAAAAJNJDFCAA|1967-05-30|808|3518|270|1967|2|5|30|2|1967|270|3518|Tuesday|1967Q2|N|N|N|2439612|2439731|2439276|2439551|N|N|N|N|N| +2439642|AAAAAAAAKNJDFCAA|1967-05-31|808|3518|270|1967|3|5|31|2|1967|270|3518|Wednesday|1967Q2|N|N|N|2439612|2439731|2439277|2439552|N|N|N|N|N| +2439643|AAAAAAAALNJDFCAA|1967-06-01|809|3518|271|1967|4|6|1|2|1967|271|3518|Thursday|1967Q2|N|N|N|2439643|2439793|2439278|2439553|N|N|N|N|N| +2439644|AAAAAAAAMNJDFCAA|1967-06-02|809|3518|271|1967|5|6|2|2|1967|271|3518|Friday|1967Q2|N|Y|N|2439643|2439793|2439279|2439554|N|N|N|N|N| +2439645|AAAAAAAANNJDFCAA|1967-06-03|809|3518|271|1967|6|6|3|2|1967|271|3518|Saturday|1967Q2|N|Y|N|2439643|2439793|2439280|2439555|N|N|N|N|N| +2439646|AAAAAAAAONJDFCAA|1967-06-04|809|3518|271|1967|0|6|4|2|1967|271|3518|Sunday|1967Q2|N|N|N|2439643|2439793|2439281|2439556|N|N|N|N|N| +2439647|AAAAAAAAPNJDFCAA|1967-06-05|809|3518|271|1967|1|6|5|2|1967|271|3518|Monday|1967Q2|N|N|N|2439643|2439793|2439282|2439557|N|N|N|N|N| +2439648|AAAAAAAAAOJDFCAA|1967-06-06|809|3519|271|1967|2|6|6|2|1967|271|3519|Tuesday|1967Q2|N|N|N|2439643|2439793|2439283|2439558|N|N|N|N|N| +2439649|AAAAAAAABOJDFCAA|1967-06-07|809|3519|271|1967|3|6|7|2|1967|271|3519|Wednesday|1967Q2|N|N|N|2439643|2439793|2439284|2439559|N|N|N|N|N| +2439650|AAAAAAAACOJDFCAA|1967-06-08|809|3519|271|1967|4|6|8|2|1967|271|3519|Thursday|1967Q2|N|N|N|2439643|2439793|2439285|2439560|N|N|N|N|N| +2439651|AAAAAAAADOJDFCAA|1967-06-09|809|3519|271|1967|5|6|9|2|1967|271|3519|Friday|1967Q2|N|Y|N|2439643|2439793|2439286|2439561|N|N|N|N|N| +2439652|AAAAAAAAEOJDFCAA|1967-06-10|809|3519|271|1967|6|6|10|2|1967|271|3519|Saturday|1967Q2|N|Y|N|2439643|2439793|2439287|2439562|N|N|N|N|N| +2439653|AAAAAAAAFOJDFCAA|1967-06-11|809|3519|271|1967|0|6|11|2|1967|271|3519|Sunday|1967Q2|N|N|N|2439643|2439793|2439288|2439563|N|N|N|N|N| +2439654|AAAAAAAAGOJDFCAA|1967-06-12|809|3519|271|1967|1|6|12|2|1967|271|3519|Monday|1967Q2|N|N|N|2439643|2439793|2439289|2439564|N|N|N|N|N| +2439655|AAAAAAAAHOJDFCAA|1967-06-13|809|3520|271|1967|2|6|13|2|1967|271|3520|Tuesday|1967Q2|N|N|N|2439643|2439793|2439290|2439565|N|N|N|N|N| +2439656|AAAAAAAAIOJDFCAA|1967-06-14|809|3520|271|1967|3|6|14|2|1967|271|3520|Wednesday|1967Q2|N|N|N|2439643|2439793|2439291|2439566|N|N|N|N|N| +2439657|AAAAAAAAJOJDFCAA|1967-06-15|809|3520|271|1967|4|6|15|2|1967|271|3520|Thursday|1967Q2|N|N|N|2439643|2439793|2439292|2439567|N|N|N|N|N| +2439658|AAAAAAAAKOJDFCAA|1967-06-16|809|3520|271|1967|5|6|16|2|1967|271|3520|Friday|1967Q2|N|Y|N|2439643|2439793|2439293|2439568|N|N|N|N|N| +2439659|AAAAAAAALOJDFCAA|1967-06-17|809|3520|271|1967|6|6|17|2|1967|271|3520|Saturday|1967Q2|N|Y|N|2439643|2439793|2439294|2439569|N|N|N|N|N| +2439660|AAAAAAAAMOJDFCAA|1967-06-18|809|3520|271|1967|0|6|18|2|1967|271|3520|Sunday|1967Q2|N|N|N|2439643|2439793|2439295|2439570|N|N|N|N|N| +2439661|AAAAAAAANOJDFCAA|1967-06-19|809|3520|271|1967|1|6|19|2|1967|271|3520|Monday|1967Q2|N|N|N|2439643|2439793|2439296|2439571|N|N|N|N|N| +2439662|AAAAAAAAOOJDFCAA|1967-06-20|809|3521|271|1967|2|6|20|2|1967|271|3521|Tuesday|1967Q2|N|N|N|2439643|2439793|2439297|2439572|N|N|N|N|N| +2439663|AAAAAAAAPOJDFCAA|1967-06-21|809|3521|271|1967|3|6|21|2|1967|271|3521|Wednesday|1967Q2|N|N|N|2439643|2439793|2439298|2439573|N|N|N|N|N| +2439664|AAAAAAAAAPJDFCAA|1967-06-22|809|3521|271|1967|4|6|22|2|1967|271|3521|Thursday|1967Q2|N|N|N|2439643|2439793|2439299|2439574|N|N|N|N|N| +2439665|AAAAAAAABPJDFCAA|1967-06-23|809|3521|271|1967|5|6|23|2|1967|271|3521|Friday|1967Q2|N|Y|N|2439643|2439793|2439300|2439575|N|N|N|N|N| +2439666|AAAAAAAACPJDFCAA|1967-06-24|809|3521|271|1967|6|6|24|2|1967|271|3521|Saturday|1967Q2|N|Y|N|2439643|2439793|2439301|2439576|N|N|N|N|N| +2439667|AAAAAAAADPJDFCAA|1967-06-25|809|3521|271|1967|0|6|25|2|1967|271|3521|Sunday|1967Q2|N|N|N|2439643|2439793|2439302|2439577|N|N|N|N|N| +2439668|AAAAAAAAEPJDFCAA|1967-06-26|809|3521|271|1967|1|6|26|2|1967|271|3521|Monday|1967Q2|N|N|N|2439643|2439793|2439303|2439578|N|N|N|N|N| +2439669|AAAAAAAAFPJDFCAA|1967-06-27|809|3522|271|1967|2|6|27|2|1967|271|3522|Tuesday|1967Q2|N|N|N|2439643|2439793|2439304|2439579|N|N|N|N|N| +2439670|AAAAAAAAGPJDFCAA|1967-06-28|809|3522|271|1967|3|6|28|2|1967|271|3522|Wednesday|1967Q2|N|N|N|2439643|2439793|2439305|2439580|N|N|N|N|N| +2439671|AAAAAAAAHPJDFCAA|1967-06-29|809|3522|271|1967|4|6|29|2|1967|271|3522|Thursday|1967Q2|N|N|N|2439643|2439793|2439306|2439581|N|N|N|N|N| +2439672|AAAAAAAAIPJDFCAA|1967-06-30|809|3522|271|1967|5|6|30|2|1967|271|3522|Friday|1967Q2|N|Y|N|2439643|2439793|2439307|2439582|N|N|N|N|N| +2439673|AAAAAAAAJPJDFCAA|1967-07-01|810|3522|271|1967|6|7|1|2|1967|271|3522|Saturday|1967Q2|N|Y|N|2439673|2439853|2439308|2439582|N|N|N|N|N| +2439674|AAAAAAAAKPJDFCAA|1967-07-02|810|3522|271|1967|0|7|2|3|1967|271|3522|Sunday|1967Q3|N|N|N|2439673|2439853|2439309|2439583|N|N|N|N|N| +2439675|AAAAAAAALPJDFCAA|1967-07-03|810|3522|271|1967|1|7|3|3|1967|271|3522|Monday|1967Q3|N|N|N|2439673|2439853|2439310|2439584|N|N|N|N|N| +2439676|AAAAAAAAMPJDFCAA|1967-07-04|810|3523|271|1967|2|7|4|3|1967|271|3523|Tuesday|1967Q3|N|N|N|2439673|2439853|2439311|2439585|N|N|N|N|N| +2439677|AAAAAAAANPJDFCAA|1967-07-05|810|3523|271|1967|3|7|5|3|1967|271|3523|Wednesday|1967Q3|Y|N|N|2439673|2439853|2439312|2439586|N|N|N|N|N| +2439678|AAAAAAAAOPJDFCAA|1967-07-06|810|3523|271|1967|4|7|6|3|1967|271|3523|Thursday|1967Q3|N|N|Y|2439673|2439853|2439313|2439587|N|N|N|N|N| +2439679|AAAAAAAAPPJDFCAA|1967-07-07|810|3523|271|1967|5|7|7|3|1967|271|3523|Friday|1967Q3|N|Y|N|2439673|2439853|2439314|2439588|N|N|N|N|N| +2439680|AAAAAAAAAAKDFCAA|1967-07-08|810|3523|271|1967|6|7|8|3|1967|271|3523|Saturday|1967Q3|N|Y|N|2439673|2439853|2439315|2439589|N|N|N|N|N| +2439681|AAAAAAAABAKDFCAA|1967-07-09|810|3523|271|1967|0|7|9|3|1967|271|3523|Sunday|1967Q3|N|N|N|2439673|2439853|2439316|2439590|N|N|N|N|N| +2439682|AAAAAAAACAKDFCAA|1967-07-10|810|3523|271|1967|1|7|10|3|1967|271|3523|Monday|1967Q3|N|N|N|2439673|2439853|2439317|2439591|N|N|N|N|N| +2439683|AAAAAAAADAKDFCAA|1967-07-11|810|3524|271|1967|2|7|11|3|1967|271|3524|Tuesday|1967Q3|N|N|N|2439673|2439853|2439318|2439592|N|N|N|N|N| +2439684|AAAAAAAAEAKDFCAA|1967-07-12|810|3524|271|1967|3|7|12|3|1967|271|3524|Wednesday|1967Q3|N|N|N|2439673|2439853|2439319|2439593|N|N|N|N|N| +2439685|AAAAAAAAFAKDFCAA|1967-07-13|810|3524|271|1967|4|7|13|3|1967|271|3524|Thursday|1967Q3|N|N|N|2439673|2439853|2439320|2439594|N|N|N|N|N| +2439686|AAAAAAAAGAKDFCAA|1967-07-14|810|3524|271|1967|5|7|14|3|1967|271|3524|Friday|1967Q3|N|Y|N|2439673|2439853|2439321|2439595|N|N|N|N|N| +2439687|AAAAAAAAHAKDFCAA|1967-07-15|810|3524|271|1967|6|7|15|3|1967|271|3524|Saturday|1967Q3|N|Y|N|2439673|2439853|2439322|2439596|N|N|N|N|N| +2439688|AAAAAAAAIAKDFCAA|1967-07-16|810|3524|271|1967|0|7|16|3|1967|271|3524|Sunday|1967Q3|N|N|N|2439673|2439853|2439323|2439597|N|N|N|N|N| +2439689|AAAAAAAAJAKDFCAA|1967-07-17|810|3524|271|1967|1|7|17|3|1967|271|3524|Monday|1967Q3|N|N|N|2439673|2439853|2439324|2439598|N|N|N|N|N| +2439690|AAAAAAAAKAKDFCAA|1967-07-18|810|3525|271|1967|2|7|18|3|1967|271|3525|Tuesday|1967Q3|N|N|N|2439673|2439853|2439325|2439599|N|N|N|N|N| +2439691|AAAAAAAALAKDFCAA|1967-07-19|810|3525|271|1967|3|7|19|3|1967|271|3525|Wednesday|1967Q3|N|N|N|2439673|2439853|2439326|2439600|N|N|N|N|N| +2439692|AAAAAAAAMAKDFCAA|1967-07-20|810|3525|271|1967|4|7|20|3|1967|271|3525|Thursday|1967Q3|N|N|N|2439673|2439853|2439327|2439601|N|N|N|N|N| +2439693|AAAAAAAANAKDFCAA|1967-07-21|810|3525|271|1967|5|7|21|3|1967|271|3525|Friday|1967Q3|N|Y|N|2439673|2439853|2439328|2439602|N|N|N|N|N| +2439694|AAAAAAAAOAKDFCAA|1967-07-22|810|3525|271|1967|6|7|22|3|1967|271|3525|Saturday|1967Q3|N|Y|N|2439673|2439853|2439329|2439603|N|N|N|N|N| +2439695|AAAAAAAAPAKDFCAA|1967-07-23|810|3525|271|1967|0|7|23|3|1967|271|3525|Sunday|1967Q3|N|N|N|2439673|2439853|2439330|2439604|N|N|N|N|N| +2439696|AAAAAAAAABKDFCAA|1967-07-24|810|3525|271|1967|1|7|24|3|1967|271|3525|Monday|1967Q3|N|N|N|2439673|2439853|2439331|2439605|N|N|N|N|N| +2439697|AAAAAAAABBKDFCAA|1967-07-25|810|3526|271|1967|2|7|25|3|1967|271|3526|Tuesday|1967Q3|N|N|N|2439673|2439853|2439332|2439606|N|N|N|N|N| +2439698|AAAAAAAACBKDFCAA|1967-07-26|810|3526|271|1967|3|7|26|3|1967|271|3526|Wednesday|1967Q3|N|N|N|2439673|2439853|2439333|2439607|N|N|N|N|N| +2439699|AAAAAAAADBKDFCAA|1967-07-27|810|3526|271|1967|4|7|27|3|1967|271|3526|Thursday|1967Q3|N|N|N|2439673|2439853|2439334|2439608|N|N|N|N|N| +2439700|AAAAAAAAEBKDFCAA|1967-07-28|810|3526|271|1967|5|7|28|3|1967|271|3526|Friday|1967Q3|N|Y|N|2439673|2439853|2439335|2439609|N|N|N|N|N| +2439701|AAAAAAAAFBKDFCAA|1967-07-29|810|3526|271|1967|6|7|29|3|1967|271|3526|Saturday|1967Q3|N|Y|N|2439673|2439853|2439336|2439610|N|N|N|N|N| +2439702|AAAAAAAAGBKDFCAA|1967-07-30|810|3526|271|1967|0|7|30|3|1967|271|3526|Sunday|1967Q3|N|N|N|2439673|2439853|2439337|2439611|N|N|N|N|N| +2439703|AAAAAAAAHBKDFCAA|1967-07-31|810|3526|271|1967|1|7|31|3|1967|271|3526|Monday|1967Q3|N|N|N|2439673|2439853|2439338|2439612|N|N|N|N|N| +2439704|AAAAAAAAIBKDFCAA|1967-08-01|811|3527|271|1967|2|8|1|3|1967|271|3527|Tuesday|1967Q3|N|N|N|2439704|2439915|2439339|2439613|N|N|N|N|N| +2439705|AAAAAAAAJBKDFCAA|1967-08-02|811|3527|271|1967|3|8|2|3|1967|271|3527|Wednesday|1967Q3|N|N|N|2439704|2439915|2439340|2439614|N|N|N|N|N| +2439706|AAAAAAAAKBKDFCAA|1967-08-03|811|3527|271|1967|4|8|3|3|1967|271|3527|Thursday|1967Q3|N|N|N|2439704|2439915|2439341|2439615|N|N|N|N|N| +2439707|AAAAAAAALBKDFCAA|1967-08-04|811|3527|271|1967|5|8|4|3|1967|271|3527|Friday|1967Q3|N|Y|N|2439704|2439915|2439342|2439616|N|N|N|N|N| +2439708|AAAAAAAAMBKDFCAA|1967-08-05|811|3527|271|1967|6|8|5|3|1967|271|3527|Saturday|1967Q3|N|Y|N|2439704|2439915|2439343|2439617|N|N|N|N|N| +2439709|AAAAAAAANBKDFCAA|1967-08-06|811|3527|271|1967|0|8|6|3|1967|271|3527|Sunday|1967Q3|N|N|N|2439704|2439915|2439344|2439618|N|N|N|N|N| +2439710|AAAAAAAAOBKDFCAA|1967-08-07|811|3527|271|1967|1|8|7|3|1967|271|3527|Monday|1967Q3|N|N|N|2439704|2439915|2439345|2439619|N|N|N|N|N| +2439711|AAAAAAAAPBKDFCAA|1967-08-08|811|3528|271|1967|2|8|8|3|1967|271|3528|Tuesday|1967Q3|N|N|N|2439704|2439915|2439346|2439620|N|N|N|N|N| +2439712|AAAAAAAAACKDFCAA|1967-08-09|811|3528|271|1967|3|8|9|3|1967|271|3528|Wednesday|1967Q3|N|N|N|2439704|2439915|2439347|2439621|N|N|N|N|N| +2439713|AAAAAAAABCKDFCAA|1967-08-10|811|3528|271|1967|4|8|10|3|1967|271|3528|Thursday|1967Q3|N|N|N|2439704|2439915|2439348|2439622|N|N|N|N|N| +2439714|AAAAAAAACCKDFCAA|1967-08-11|811|3528|271|1967|5|8|11|3|1967|271|3528|Friday|1967Q3|N|Y|N|2439704|2439915|2439349|2439623|N|N|N|N|N| +2439715|AAAAAAAADCKDFCAA|1967-08-12|811|3528|271|1967|6|8|12|3|1967|271|3528|Saturday|1967Q3|N|Y|N|2439704|2439915|2439350|2439624|N|N|N|N|N| +2439716|AAAAAAAAECKDFCAA|1967-08-13|811|3528|271|1967|0|8|13|3|1967|271|3528|Sunday|1967Q3|N|N|N|2439704|2439915|2439351|2439625|N|N|N|N|N| +2439717|AAAAAAAAFCKDFCAA|1967-08-14|811|3528|271|1967|1|8|14|3|1967|271|3528|Monday|1967Q3|N|N|N|2439704|2439915|2439352|2439626|N|N|N|N|N| +2439718|AAAAAAAAGCKDFCAA|1967-08-15|811|3529|271|1967|2|8|15|3|1967|271|3529|Tuesday|1967Q3|N|N|N|2439704|2439915|2439353|2439627|N|N|N|N|N| +2439719|AAAAAAAAHCKDFCAA|1967-08-16|811|3529|271|1967|3|8|16|3|1967|271|3529|Wednesday|1967Q3|N|N|N|2439704|2439915|2439354|2439628|N|N|N|N|N| +2439720|AAAAAAAAICKDFCAA|1967-08-17|811|3529|271|1967|4|8|17|3|1967|271|3529|Thursday|1967Q3|N|N|N|2439704|2439915|2439355|2439629|N|N|N|N|N| +2439721|AAAAAAAAJCKDFCAA|1967-08-18|811|3529|271|1967|5|8|18|3|1967|271|3529|Friday|1967Q3|N|Y|N|2439704|2439915|2439356|2439630|N|N|N|N|N| +2439722|AAAAAAAAKCKDFCAA|1967-08-19|811|3529|271|1967|6|8|19|3|1967|271|3529|Saturday|1967Q3|N|Y|N|2439704|2439915|2439357|2439631|N|N|N|N|N| +2439723|AAAAAAAALCKDFCAA|1967-08-20|811|3529|271|1967|0|8|20|3|1967|271|3529|Sunday|1967Q3|N|N|N|2439704|2439915|2439358|2439632|N|N|N|N|N| +2439724|AAAAAAAAMCKDFCAA|1967-08-21|811|3529|271|1967|1|8|21|3|1967|271|3529|Monday|1967Q3|N|N|N|2439704|2439915|2439359|2439633|N|N|N|N|N| +2439725|AAAAAAAANCKDFCAA|1967-08-22|811|3530|271|1967|2|8|22|3|1967|271|3530|Tuesday|1967Q3|N|N|N|2439704|2439915|2439360|2439634|N|N|N|N|N| +2439726|AAAAAAAAOCKDFCAA|1967-08-23|811|3530|271|1967|3|8|23|3|1967|271|3530|Wednesday|1967Q3|N|N|N|2439704|2439915|2439361|2439635|N|N|N|N|N| +2439727|AAAAAAAAPCKDFCAA|1967-08-24|811|3530|271|1967|4|8|24|3|1967|271|3530|Thursday|1967Q3|N|N|N|2439704|2439915|2439362|2439636|N|N|N|N|N| +2439728|AAAAAAAAADKDFCAA|1967-08-25|811|3530|271|1967|5|8|25|3|1967|271|3530|Friday|1967Q3|N|Y|N|2439704|2439915|2439363|2439637|N|N|N|N|N| +2439729|AAAAAAAABDKDFCAA|1967-08-26|811|3530|271|1967|6|8|26|3|1967|271|3530|Saturday|1967Q3|N|Y|N|2439704|2439915|2439364|2439638|N|N|N|N|N| +2439730|AAAAAAAACDKDFCAA|1967-08-27|811|3530|271|1967|0|8|27|3|1967|271|3530|Sunday|1967Q3|N|N|N|2439704|2439915|2439365|2439639|N|N|N|N|N| +2439731|AAAAAAAADDKDFCAA|1967-08-28|811|3530|271|1967|1|8|28|3|1967|271|3530|Monday|1967Q3|N|N|N|2439704|2439915|2439366|2439640|N|N|N|N|N| +2439732|AAAAAAAAEDKDFCAA|1967-08-29|811|3531|271|1967|2|8|29|3|1967|271|3531|Tuesday|1967Q3|N|N|N|2439704|2439915|2439367|2439641|N|N|N|N|N| +2439733|AAAAAAAAFDKDFCAA|1967-08-30|811|3531|271|1967|3|8|30|3|1967|271|3531|Wednesday|1967Q3|N|N|N|2439704|2439915|2439368|2439642|N|N|N|N|N| +2439734|AAAAAAAAGDKDFCAA|1967-08-31|811|3531|271|1967|4|8|31|3|1967|271|3531|Thursday|1967Q3|N|N|N|2439704|2439915|2439369|2439643|N|N|N|N|N| +2439735|AAAAAAAAHDKDFCAA|1967-09-01|812|3531|272|1967|5|9|1|3|1967|272|3531|Friday|1967Q3|N|Y|N|2439735|2439977|2439370|2439644|N|N|N|N|N| +2439736|AAAAAAAAIDKDFCAA|1967-09-02|812|3531|272|1967|6|9|2|3|1967|272|3531|Saturday|1967Q3|N|Y|N|2439735|2439977|2439371|2439645|N|N|N|N|N| +2439737|AAAAAAAAJDKDFCAA|1967-09-03|812|3531|272|1967|0|9|3|3|1967|272|3531|Sunday|1967Q3|N|N|N|2439735|2439977|2439372|2439646|N|N|N|N|N| +2439738|AAAAAAAAKDKDFCAA|1967-09-04|812|3531|272|1967|1|9|4|3|1967|272|3531|Monday|1967Q3|N|N|N|2439735|2439977|2439373|2439647|N|N|N|N|N| +2439739|AAAAAAAALDKDFCAA|1967-09-05|812|3532|272|1967|2|9|5|3|1967|272|3532|Tuesday|1967Q3|N|N|N|2439735|2439977|2439374|2439648|N|N|N|N|N| +2439740|AAAAAAAAMDKDFCAA|1967-09-06|812|3532|272|1967|3|9|6|3|1967|272|3532|Wednesday|1967Q3|N|N|N|2439735|2439977|2439375|2439649|N|N|N|N|N| +2439741|AAAAAAAANDKDFCAA|1967-09-07|812|3532|272|1967|4|9|7|3|1967|272|3532|Thursday|1967Q3|N|N|N|2439735|2439977|2439376|2439650|N|N|N|N|N| +2439742|AAAAAAAAODKDFCAA|1967-09-08|812|3532|272|1967|5|9|8|3|1967|272|3532|Friday|1967Q3|N|Y|N|2439735|2439977|2439377|2439651|N|N|N|N|N| +2439743|AAAAAAAAPDKDFCAA|1967-09-09|812|3532|272|1967|6|9|9|3|1967|272|3532|Saturday|1967Q3|N|Y|N|2439735|2439977|2439378|2439652|N|N|N|N|N| +2439744|AAAAAAAAAEKDFCAA|1967-09-10|812|3532|272|1967|0|9|10|3|1967|272|3532|Sunday|1967Q3|N|N|N|2439735|2439977|2439379|2439653|N|N|N|N|N| +2439745|AAAAAAAABEKDFCAA|1967-09-11|812|3532|272|1967|1|9|11|3|1967|272|3532|Monday|1967Q3|N|N|N|2439735|2439977|2439380|2439654|N|N|N|N|N| +2439746|AAAAAAAACEKDFCAA|1967-09-12|812|3533|272|1967|2|9|12|3|1967|272|3533|Tuesday|1967Q3|N|N|N|2439735|2439977|2439381|2439655|N|N|N|N|N| +2439747|AAAAAAAADEKDFCAA|1967-09-13|812|3533|272|1967|3|9|13|3|1967|272|3533|Wednesday|1967Q3|N|N|N|2439735|2439977|2439382|2439656|N|N|N|N|N| +2439748|AAAAAAAAEEKDFCAA|1967-09-14|812|3533|272|1967|4|9|14|3|1967|272|3533|Thursday|1967Q3|N|N|N|2439735|2439977|2439383|2439657|N|N|N|N|N| +2439749|AAAAAAAAFEKDFCAA|1967-09-15|812|3533|272|1967|5|9|15|3|1967|272|3533|Friday|1967Q3|N|Y|N|2439735|2439977|2439384|2439658|N|N|N|N|N| +2439750|AAAAAAAAGEKDFCAA|1967-09-16|812|3533|272|1967|6|9|16|3|1967|272|3533|Saturday|1967Q3|N|Y|N|2439735|2439977|2439385|2439659|N|N|N|N|N| +2439751|AAAAAAAAHEKDFCAA|1967-09-17|812|3533|272|1967|0|9|17|3|1967|272|3533|Sunday|1967Q3|N|N|N|2439735|2439977|2439386|2439660|N|N|N|N|N| +2439752|AAAAAAAAIEKDFCAA|1967-09-18|812|3533|272|1967|1|9|18|3|1967|272|3533|Monday|1967Q3|N|N|N|2439735|2439977|2439387|2439661|N|N|N|N|N| +2439753|AAAAAAAAJEKDFCAA|1967-09-19|812|3534|272|1967|2|9|19|3|1967|272|3534|Tuesday|1967Q3|N|N|N|2439735|2439977|2439388|2439662|N|N|N|N|N| +2439754|AAAAAAAAKEKDFCAA|1967-09-20|812|3534|272|1967|3|9|20|3|1967|272|3534|Wednesday|1967Q3|N|N|N|2439735|2439977|2439389|2439663|N|N|N|N|N| +2439755|AAAAAAAALEKDFCAA|1967-09-21|812|3534|272|1967|4|9|21|3|1967|272|3534|Thursday|1967Q3|N|N|N|2439735|2439977|2439390|2439664|N|N|N|N|N| +2439756|AAAAAAAAMEKDFCAA|1967-09-22|812|3534|272|1967|5|9|22|3|1967|272|3534|Friday|1967Q3|N|Y|N|2439735|2439977|2439391|2439665|N|N|N|N|N| +2439757|AAAAAAAANEKDFCAA|1967-09-23|812|3534|272|1967|6|9|23|3|1967|272|3534|Saturday|1967Q3|N|Y|N|2439735|2439977|2439392|2439666|N|N|N|N|N| +2439758|AAAAAAAAOEKDFCAA|1967-09-24|812|3534|272|1967|0|9|24|3|1967|272|3534|Sunday|1967Q3|N|N|N|2439735|2439977|2439393|2439667|N|N|N|N|N| +2439759|AAAAAAAAPEKDFCAA|1967-09-25|812|3534|272|1967|1|9|25|3|1967|272|3534|Monday|1967Q3|N|N|N|2439735|2439977|2439394|2439668|N|N|N|N|N| +2439760|AAAAAAAAAFKDFCAA|1967-09-26|812|3535|272|1967|2|9|26|3|1967|272|3535|Tuesday|1967Q3|N|N|N|2439735|2439977|2439395|2439669|N|N|N|N|N| +2439761|AAAAAAAABFKDFCAA|1967-09-27|812|3535|272|1967|3|9|27|3|1967|272|3535|Wednesday|1967Q3|N|N|N|2439735|2439977|2439396|2439670|N|N|N|N|N| +2439762|AAAAAAAACFKDFCAA|1967-09-28|812|3535|272|1967|4|9|28|3|1967|272|3535|Thursday|1967Q3|N|N|N|2439735|2439977|2439397|2439671|N|N|N|N|N| +2439763|AAAAAAAADFKDFCAA|1967-09-29|812|3535|272|1967|5|9|29|3|1967|272|3535|Friday|1967Q3|N|Y|N|2439735|2439977|2439398|2439672|N|N|N|N|N| +2439764|AAAAAAAAEFKDFCAA|1967-09-30|812|3535|272|1967|6|9|30|3|1967|272|3535|Saturday|1967Q3|N|Y|N|2439735|2439977|2439399|2439673|N|N|N|N|N| +2439765|AAAAAAAAFFKDFCAA|1967-10-01|813|3535|272|1967|0|10|1|3|1967|272|3535|Sunday|1967Q3|N|N|N|2439765|2440037|2439400|2439673|N|N|N|N|N| +2439766|AAAAAAAAGFKDFCAA|1967-10-02|813|3535|272|1967|1|10|2|4|1967|272|3535|Monday|1967Q4|N|N|N|2439765|2440037|2439401|2439674|N|N|N|N|N| +2439767|AAAAAAAAHFKDFCAA|1967-10-03|813|3536|272|1967|2|10|3|4|1967|272|3536|Tuesday|1967Q4|N|N|N|2439765|2440037|2439402|2439675|N|N|N|N|N| +2439768|AAAAAAAAIFKDFCAA|1967-10-04|813|3536|272|1967|3|10|4|4|1967|272|3536|Wednesday|1967Q4|N|N|N|2439765|2440037|2439403|2439676|N|N|N|N|N| +2439769|AAAAAAAAJFKDFCAA|1967-10-05|813|3536|272|1967|4|10|5|4|1967|272|3536|Thursday|1967Q4|N|N|N|2439765|2440037|2439404|2439677|N|N|N|N|N| +2439770|AAAAAAAAKFKDFCAA|1967-10-06|813|3536|272|1967|5|10|6|4|1967|272|3536|Friday|1967Q4|N|Y|N|2439765|2440037|2439405|2439678|N|N|N|N|N| +2439771|AAAAAAAALFKDFCAA|1967-10-07|813|3536|272|1967|6|10|7|4|1967|272|3536|Saturday|1967Q4|N|Y|N|2439765|2440037|2439406|2439679|N|N|N|N|N| +2439772|AAAAAAAAMFKDFCAA|1967-10-08|813|3536|272|1967|0|10|8|4|1967|272|3536|Sunday|1967Q4|N|N|N|2439765|2440037|2439407|2439680|N|N|N|N|N| +2439773|AAAAAAAANFKDFCAA|1967-10-09|813|3536|272|1967|1|10|9|4|1967|272|3536|Monday|1967Q4|N|N|N|2439765|2440037|2439408|2439681|N|N|N|N|N| +2439774|AAAAAAAAOFKDFCAA|1967-10-10|813|3537|272|1967|2|10|10|4|1967|272|3537|Tuesday|1967Q4|N|N|N|2439765|2440037|2439409|2439682|N|N|N|N|N| +2439775|AAAAAAAAPFKDFCAA|1967-10-11|813|3537|272|1967|3|10|11|4|1967|272|3537|Wednesday|1967Q4|N|N|N|2439765|2440037|2439410|2439683|N|N|N|N|N| +2439776|AAAAAAAAAGKDFCAA|1967-10-12|813|3537|272|1967|4|10|12|4|1967|272|3537|Thursday|1967Q4|N|N|N|2439765|2440037|2439411|2439684|N|N|N|N|N| +2439777|AAAAAAAABGKDFCAA|1967-10-13|813|3537|272|1967|5|10|13|4|1967|272|3537|Friday|1967Q4|N|Y|N|2439765|2440037|2439412|2439685|N|N|N|N|N| +2439778|AAAAAAAACGKDFCAA|1967-10-14|813|3537|272|1967|6|10|14|4|1967|272|3537|Saturday|1967Q4|N|Y|N|2439765|2440037|2439413|2439686|N|N|N|N|N| +2439779|AAAAAAAADGKDFCAA|1967-10-15|813|3537|272|1967|0|10|15|4|1967|272|3537|Sunday|1967Q4|N|N|N|2439765|2440037|2439414|2439687|N|N|N|N|N| +2439780|AAAAAAAAEGKDFCAA|1967-10-16|813|3537|272|1967|1|10|16|4|1967|272|3537|Monday|1967Q4|N|N|N|2439765|2440037|2439415|2439688|N|N|N|N|N| +2439781|AAAAAAAAFGKDFCAA|1967-10-17|813|3538|272|1967|2|10|17|4|1967|272|3538|Tuesday|1967Q4|N|N|N|2439765|2440037|2439416|2439689|N|N|N|N|N| +2439782|AAAAAAAAGGKDFCAA|1967-10-18|813|3538|272|1967|3|10|18|4|1967|272|3538|Wednesday|1967Q4|N|N|N|2439765|2440037|2439417|2439690|N|N|N|N|N| +2439783|AAAAAAAAHGKDFCAA|1967-10-19|813|3538|272|1967|4|10|19|4|1967|272|3538|Thursday|1967Q4|N|N|N|2439765|2440037|2439418|2439691|N|N|N|N|N| +2439784|AAAAAAAAIGKDFCAA|1967-10-20|813|3538|272|1967|5|10|20|4|1967|272|3538|Friday|1967Q4|N|Y|N|2439765|2440037|2439419|2439692|N|N|N|N|N| +2439785|AAAAAAAAJGKDFCAA|1967-10-21|813|3538|272|1967|6|10|21|4|1967|272|3538|Saturday|1967Q4|N|Y|N|2439765|2440037|2439420|2439693|N|N|N|N|N| +2439786|AAAAAAAAKGKDFCAA|1967-10-22|813|3538|272|1967|0|10|22|4|1967|272|3538|Sunday|1967Q4|N|N|N|2439765|2440037|2439421|2439694|N|N|N|N|N| +2439787|AAAAAAAALGKDFCAA|1967-10-23|813|3538|272|1967|1|10|23|4|1967|272|3538|Monday|1967Q4|N|N|N|2439765|2440037|2439422|2439695|N|N|N|N|N| +2439788|AAAAAAAAMGKDFCAA|1967-10-24|813|3539|272|1967|2|10|24|4|1967|272|3539|Tuesday|1967Q4|N|N|N|2439765|2440037|2439423|2439696|N|N|N|N|N| +2439789|AAAAAAAANGKDFCAA|1967-10-25|813|3539|272|1967|3|10|25|4|1967|272|3539|Wednesday|1967Q4|N|N|N|2439765|2440037|2439424|2439697|N|N|N|N|N| +2439790|AAAAAAAAOGKDFCAA|1967-10-26|813|3539|272|1967|4|10|26|4|1967|272|3539|Thursday|1967Q4|N|N|N|2439765|2440037|2439425|2439698|N|N|N|N|N| +2439791|AAAAAAAAPGKDFCAA|1967-10-27|813|3539|272|1967|5|10|27|4|1967|272|3539|Friday|1967Q4|N|Y|N|2439765|2440037|2439426|2439699|N|N|N|N|N| +2439792|AAAAAAAAAHKDFCAA|1967-10-28|813|3539|272|1967|6|10|28|4|1967|272|3539|Saturday|1967Q4|N|Y|N|2439765|2440037|2439427|2439700|N|N|N|N|N| +2439793|AAAAAAAABHKDFCAA|1967-10-29|813|3539|272|1967|0|10|29|4|1967|272|3539|Sunday|1967Q4|N|N|N|2439765|2440037|2439428|2439701|N|N|N|N|N| +2439794|AAAAAAAACHKDFCAA|1967-10-30|813|3539|272|1967|1|10|30|4|1967|272|3539|Monday|1967Q4|N|N|N|2439765|2440037|2439429|2439702|N|N|N|N|N| +2439795|AAAAAAAADHKDFCAA|1967-10-31|813|3540|272|1967|2|10|31|4|1967|272|3540|Tuesday|1967Q4|N|N|N|2439765|2440037|2439430|2439703|N|N|N|N|N| +2439796|AAAAAAAAEHKDFCAA|1967-11-01|814|3540|272|1967|3|11|1|4|1967|272|3540|Wednesday|1967Q4|N|N|N|2439796|2440099|2439431|2439704|N|N|N|N|N| +2439797|AAAAAAAAFHKDFCAA|1967-11-02|814|3540|272|1967|4|11|2|4|1967|272|3540|Thursday|1967Q4|N|N|N|2439796|2440099|2439432|2439705|N|N|N|N|N| +2439798|AAAAAAAAGHKDFCAA|1967-11-03|814|3540|272|1967|5|11|3|4|1967|272|3540|Friday|1967Q4|N|Y|N|2439796|2440099|2439433|2439706|N|N|N|N|N| +2439799|AAAAAAAAHHKDFCAA|1967-11-04|814|3540|272|1967|6|11|4|4|1967|272|3540|Saturday|1967Q4|N|Y|N|2439796|2440099|2439434|2439707|N|N|N|N|N| +2439800|AAAAAAAAIHKDFCAA|1967-11-05|814|3540|272|1967|0|11|5|4|1967|272|3540|Sunday|1967Q4|N|N|N|2439796|2440099|2439435|2439708|N|N|N|N|N| +2439801|AAAAAAAAJHKDFCAA|1967-11-06|814|3540|272|1967|1|11|6|4|1967|272|3540|Monday|1967Q4|N|N|N|2439796|2440099|2439436|2439709|N|N|N|N|N| +2439802|AAAAAAAAKHKDFCAA|1967-11-07|814|3541|272|1967|2|11|7|4|1967|272|3541|Tuesday|1967Q4|N|N|N|2439796|2440099|2439437|2439710|N|N|N|N|N| +2439803|AAAAAAAALHKDFCAA|1967-11-08|814|3541|272|1967|3|11|8|4|1967|272|3541|Wednesday|1967Q4|N|N|N|2439796|2440099|2439438|2439711|N|N|N|N|N| +2439804|AAAAAAAAMHKDFCAA|1967-11-09|814|3541|272|1967|4|11|9|4|1967|272|3541|Thursday|1967Q4|N|N|N|2439796|2440099|2439439|2439712|N|N|N|N|N| +2439805|AAAAAAAANHKDFCAA|1967-11-10|814|3541|272|1967|5|11|10|4|1967|272|3541|Friday|1967Q4|N|Y|N|2439796|2440099|2439440|2439713|N|N|N|N|N| +2439806|AAAAAAAAOHKDFCAA|1967-11-11|814|3541|272|1967|6|11|11|4|1967|272|3541|Saturday|1967Q4|N|Y|N|2439796|2440099|2439441|2439714|N|N|N|N|N| +2439807|AAAAAAAAPHKDFCAA|1967-11-12|814|3541|272|1967|0|11|12|4|1967|272|3541|Sunday|1967Q4|N|N|N|2439796|2440099|2439442|2439715|N|N|N|N|N| +2439808|AAAAAAAAAIKDFCAA|1967-11-13|814|3541|272|1967|1|11|13|4|1967|272|3541|Monday|1967Q4|N|N|N|2439796|2440099|2439443|2439716|N|N|N|N|N| +2439809|AAAAAAAABIKDFCAA|1967-11-14|814|3542|272|1967|2|11|14|4|1967|272|3542|Tuesday|1967Q4|N|N|N|2439796|2440099|2439444|2439717|N|N|N|N|N| +2439810|AAAAAAAACIKDFCAA|1967-11-15|814|3542|272|1967|3|11|15|4|1967|272|3542|Wednesday|1967Q4|N|N|N|2439796|2440099|2439445|2439718|N|N|N|N|N| +2439811|AAAAAAAADIKDFCAA|1967-11-16|814|3542|272|1967|4|11|16|4|1967|272|3542|Thursday|1967Q4|N|N|N|2439796|2440099|2439446|2439719|N|N|N|N|N| +2439812|AAAAAAAAEIKDFCAA|1967-11-17|814|3542|272|1967|5|11|17|4|1967|272|3542|Friday|1967Q4|N|Y|N|2439796|2440099|2439447|2439720|N|N|N|N|N| +2439813|AAAAAAAAFIKDFCAA|1967-11-18|814|3542|272|1967|6|11|18|4|1967|272|3542|Saturday|1967Q4|N|Y|N|2439796|2440099|2439448|2439721|N|N|N|N|N| +2439814|AAAAAAAAGIKDFCAA|1967-11-19|814|3542|272|1967|0|11|19|4|1967|272|3542|Sunday|1967Q4|N|N|N|2439796|2440099|2439449|2439722|N|N|N|N|N| +2439815|AAAAAAAAHIKDFCAA|1967-11-20|814|3542|272|1967|1|11|20|4|1967|272|3542|Monday|1967Q4|N|N|N|2439796|2440099|2439450|2439723|N|N|N|N|N| +2439816|AAAAAAAAIIKDFCAA|1967-11-21|814|3543|272|1967|2|11|21|4|1967|272|3543|Tuesday|1967Q4|N|N|N|2439796|2440099|2439451|2439724|N|N|N|N|N| +2439817|AAAAAAAAJIKDFCAA|1967-11-22|814|3543|272|1967|3|11|22|4|1967|272|3543|Wednesday|1967Q4|N|N|N|2439796|2440099|2439452|2439725|N|N|N|N|N| +2439818|AAAAAAAAKIKDFCAA|1967-11-23|814|3543|272|1967|4|11|23|4|1967|272|3543|Thursday|1967Q4|N|N|N|2439796|2440099|2439453|2439726|N|N|N|N|N| +2439819|AAAAAAAALIKDFCAA|1967-11-24|814|3543|272|1967|5|11|24|4|1967|272|3543|Friday|1967Q4|N|Y|N|2439796|2440099|2439454|2439727|N|N|N|N|N| +2439820|AAAAAAAAMIKDFCAA|1967-11-25|814|3543|272|1967|6|11|25|4|1967|272|3543|Saturday|1967Q4|N|Y|N|2439796|2440099|2439455|2439728|N|N|N|N|N| +2439821|AAAAAAAANIKDFCAA|1967-11-26|814|3543|272|1967|0|11|26|4|1967|272|3543|Sunday|1967Q4|N|N|N|2439796|2440099|2439456|2439729|N|N|N|N|N| +2439822|AAAAAAAAOIKDFCAA|1967-11-27|814|3543|272|1967|1|11|27|4|1967|272|3543|Monday|1967Q4|N|N|N|2439796|2440099|2439457|2439730|N|N|N|N|N| +2439823|AAAAAAAAPIKDFCAA|1967-11-28|814|3544|272|1967|2|11|28|4|1967|272|3544|Tuesday|1967Q4|N|N|N|2439796|2440099|2439458|2439731|N|N|N|N|N| +2439824|AAAAAAAAAJKDFCAA|1967-11-29|814|3544|272|1967|3|11|29|4|1967|272|3544|Wednesday|1967Q4|N|N|N|2439796|2440099|2439459|2439732|N|N|N|N|N| +2439825|AAAAAAAABJKDFCAA|1967-11-30|814|3544|272|1967|4|11|30|4|1967|272|3544|Thursday|1967Q4|N|N|N|2439796|2440099|2439460|2439733|N|N|N|N|N| +2439826|AAAAAAAACJKDFCAA|1967-12-01|815|3544|273|1967|5|12|1|4|1967|273|3544|Friday|1967Q4|N|Y|N|2439826|2440159|2439461|2439734|N|N|N|N|N| +2439827|AAAAAAAADJKDFCAA|1967-12-02|815|3544|273|1967|6|12|2|4|1967|273|3544|Saturday|1967Q4|N|Y|N|2439826|2440159|2439462|2439735|N|N|N|N|N| +2439828|AAAAAAAAEJKDFCAA|1967-12-03|815|3544|273|1967|0|12|3|4|1967|273|3544|Sunday|1967Q4|N|N|N|2439826|2440159|2439463|2439736|N|N|N|N|N| +2439829|AAAAAAAAFJKDFCAA|1967-12-04|815|3544|273|1967|1|12|4|4|1967|273|3544|Monday|1967Q4|N|N|N|2439826|2440159|2439464|2439737|N|N|N|N|N| +2439830|AAAAAAAAGJKDFCAA|1967-12-05|815|3545|273|1967|2|12|5|4|1967|273|3545|Tuesday|1967Q4|N|N|N|2439826|2440159|2439465|2439738|N|N|N|N|N| +2439831|AAAAAAAAHJKDFCAA|1967-12-06|815|3545|273|1967|3|12|6|4|1967|273|3545|Wednesday|1967Q4|N|N|N|2439826|2440159|2439466|2439739|N|N|N|N|N| +2439832|AAAAAAAAIJKDFCAA|1967-12-07|815|3545|273|1967|4|12|7|4|1967|273|3545|Thursday|1967Q4|N|N|N|2439826|2440159|2439467|2439740|N|N|N|N|N| +2439833|AAAAAAAAJJKDFCAA|1967-12-08|815|3545|273|1967|5|12|8|4|1967|273|3545|Friday|1967Q4|N|Y|N|2439826|2440159|2439468|2439741|N|N|N|N|N| +2439834|AAAAAAAAKJKDFCAA|1967-12-09|815|3545|273|1967|6|12|9|4|1967|273|3545|Saturday|1967Q4|N|Y|N|2439826|2440159|2439469|2439742|N|N|N|N|N| +2439835|AAAAAAAALJKDFCAA|1967-12-10|815|3545|273|1967|0|12|10|4|1967|273|3545|Sunday|1967Q4|N|N|N|2439826|2440159|2439470|2439743|N|N|N|N|N| +2439836|AAAAAAAAMJKDFCAA|1967-12-11|815|3545|273|1967|1|12|11|4|1967|273|3545|Monday|1967Q4|N|N|N|2439826|2440159|2439471|2439744|N|N|N|N|N| +2439837|AAAAAAAANJKDFCAA|1967-12-12|815|3546|273|1967|2|12|12|4|1967|273|3546|Tuesday|1967Q4|N|N|N|2439826|2440159|2439472|2439745|N|N|N|N|N| +2439838|AAAAAAAAOJKDFCAA|1967-12-13|815|3546|273|1967|3|12|13|4|1967|273|3546|Wednesday|1967Q4|N|N|N|2439826|2440159|2439473|2439746|N|N|N|N|N| +2439839|AAAAAAAAPJKDFCAA|1967-12-14|815|3546|273|1967|4|12|14|4|1967|273|3546|Thursday|1967Q4|N|N|N|2439826|2440159|2439474|2439747|N|N|N|N|N| +2439840|AAAAAAAAAKKDFCAA|1967-12-15|815|3546|273|1967|5|12|15|4|1967|273|3546|Friday|1967Q4|N|Y|N|2439826|2440159|2439475|2439748|N|N|N|N|N| +2439841|AAAAAAAABKKDFCAA|1967-12-16|815|3546|273|1967|6|12|16|4|1967|273|3546|Saturday|1967Q4|N|Y|N|2439826|2440159|2439476|2439749|N|N|N|N|N| +2439842|AAAAAAAACKKDFCAA|1967-12-17|815|3546|273|1967|0|12|17|4|1967|273|3546|Sunday|1967Q4|N|N|N|2439826|2440159|2439477|2439750|N|N|N|N|N| +2439843|AAAAAAAADKKDFCAA|1967-12-18|815|3546|273|1967|1|12|18|4|1967|273|3546|Monday|1967Q4|N|N|N|2439826|2440159|2439478|2439751|N|N|N|N|N| +2439844|AAAAAAAAEKKDFCAA|1967-12-19|815|3547|273|1967|2|12|19|4|1967|273|3547|Tuesday|1967Q4|N|N|N|2439826|2440159|2439479|2439752|N|N|N|N|N| +2439845|AAAAAAAAFKKDFCAA|1967-12-20|815|3547|273|1967|3|12|20|4|1967|273|3547|Wednesday|1967Q4|N|N|N|2439826|2440159|2439480|2439753|N|N|N|N|N| +2439846|AAAAAAAAGKKDFCAA|1967-12-21|815|3547|273|1967|4|12|21|4|1967|273|3547|Thursday|1967Q4|N|N|N|2439826|2440159|2439481|2439754|N|N|N|N|N| +2439847|AAAAAAAAHKKDFCAA|1967-12-22|815|3547|273|1967|5|12|22|4|1967|273|3547|Friday|1967Q4|N|Y|N|2439826|2440159|2439482|2439755|N|N|N|N|N| +2439848|AAAAAAAAIKKDFCAA|1967-12-23|815|3547|273|1967|6|12|23|4|1967|273|3547|Saturday|1967Q4|N|Y|N|2439826|2440159|2439483|2439756|N|N|N|N|N| +2439849|AAAAAAAAJKKDFCAA|1967-12-24|815|3547|273|1967|0|12|24|4|1967|273|3547|Sunday|1967Q4|N|N|N|2439826|2440159|2439484|2439757|N|N|N|N|N| +2439850|AAAAAAAAKKKDFCAA|1967-12-25|815|3547|273|1967|1|12|25|4|1967|273|3547|Monday|1967Q4|N|N|N|2439826|2440159|2439485|2439758|N|N|N|N|N| +2439851|AAAAAAAALKKDFCAA|1967-12-26|815|3548|273|1967|2|12|26|4|1967|273|3548|Tuesday|1967Q4|Y|N|N|2439826|2440159|2439486|2439759|N|N|N|N|N| +2439852|AAAAAAAAMKKDFCAA|1967-12-27|815|3548|273|1967|3|12|27|4|1967|273|3548|Wednesday|1967Q4|N|N|Y|2439826|2440159|2439487|2439760|N|N|N|N|N| +2439853|AAAAAAAANKKDFCAA|1967-12-28|815|3548|273|1967|4|12|28|4|1967|273|3548|Thursday|1967Q4|N|N|N|2439826|2440159|2439488|2439761|N|N|N|N|N| +2439854|AAAAAAAAOKKDFCAA|1967-12-29|815|3548|273|1967|5|12|29|4|1967|273|3548|Friday|1967Q4|N|Y|N|2439826|2440159|2439489|2439762|N|N|N|N|N| +2439855|AAAAAAAAPKKDFCAA|1967-12-30|815|3548|273|1967|6|12|30|4|1967|273|3548|Saturday|1967Q4|N|Y|N|2439826|2440159|2439490|2439763|N|N|N|N|N| +2439856|AAAAAAAAALKDFCAA|1967-12-31|815|3548|273|1967|0|12|31|4|1967|273|3548|Sunday|1967Q4|N|N|N|2439826|2440159|2439491|2439764|N|N|N|N|N| +2439857|AAAAAAAABLKDFCAA|1968-01-01|816|3548|273|1968|1|1|1|1|1968|273|3548|Monday|1968Q1|Y|N|N|2439857|2439856|2439492|2439765|N|N|N|N|N| +2439858|AAAAAAAACLKDFCAA|1968-01-02|816|3549|273|1968|2|1|2|1|1968|273|3549|Tuesday|1968Q1|N|N|Y|2439857|2439856|2439493|2439766|N|N|N|N|N| +2439859|AAAAAAAADLKDFCAA|1968-01-03|816|3549|273|1968|3|1|3|1|1968|273|3549|Wednesday|1968Q1|N|N|N|2439857|2439856|2439494|2439767|N|N|N|N|N| +2439860|AAAAAAAAELKDFCAA|1968-01-04|816|3549|273|1968|4|1|4|1|1968|273|3549|Thursday|1968Q1|N|N|N|2439857|2439856|2439495|2439768|N|N|N|N|N| +2439861|AAAAAAAAFLKDFCAA|1968-01-05|816|3549|273|1968|5|1|5|1|1968|273|3549|Friday|1968Q1|N|Y|N|2439857|2439856|2439496|2439769|N|N|N|N|N| +2439862|AAAAAAAAGLKDFCAA|1968-01-06|816|3549|273|1968|6|1|6|1|1968|273|3549|Saturday|1968Q1|N|Y|N|2439857|2439856|2439497|2439770|N|N|N|N|N| +2439863|AAAAAAAAHLKDFCAA|1968-01-07|816|3549|273|1968|0|1|7|1|1968|273|3549|Sunday|1968Q1|N|N|N|2439857|2439856|2439498|2439771|N|N|N|N|N| +2439864|AAAAAAAAILKDFCAA|1968-01-08|816|3549|273|1968|1|1|8|1|1968|273|3549|Monday|1968Q1|N|N|N|2439857|2439856|2439499|2439772|N|N|N|N|N| +2439865|AAAAAAAAJLKDFCAA|1968-01-09|816|3550|273|1968|2|1|9|1|1968|273|3550|Tuesday|1968Q1|N|N|N|2439857|2439856|2439500|2439773|N|N|N|N|N| +2439866|AAAAAAAAKLKDFCAA|1968-01-10|816|3550|273|1968|3|1|10|1|1968|273|3550|Wednesday|1968Q1|N|N|N|2439857|2439856|2439501|2439774|N|N|N|N|N| +2439867|AAAAAAAALLKDFCAA|1968-01-11|816|3550|273|1968|4|1|11|1|1968|273|3550|Thursday|1968Q1|N|N|N|2439857|2439856|2439502|2439775|N|N|N|N|N| +2439868|AAAAAAAAMLKDFCAA|1968-01-12|816|3550|273|1968|5|1|12|1|1968|273|3550|Friday|1968Q1|N|Y|N|2439857|2439856|2439503|2439776|N|N|N|N|N| +2439869|AAAAAAAANLKDFCAA|1968-01-13|816|3550|273|1968|6|1|13|1|1968|273|3550|Saturday|1968Q1|N|Y|N|2439857|2439856|2439504|2439777|N|N|N|N|N| +2439870|AAAAAAAAOLKDFCAA|1968-01-14|816|3550|273|1968|0|1|14|1|1968|273|3550|Sunday|1968Q1|N|N|N|2439857|2439856|2439505|2439778|N|N|N|N|N| +2439871|AAAAAAAAPLKDFCAA|1968-01-15|816|3550|273|1968|1|1|15|1|1968|273|3550|Monday|1968Q1|N|N|N|2439857|2439856|2439506|2439779|N|N|N|N|N| +2439872|AAAAAAAAAMKDFCAA|1968-01-16|816|3551|273|1968|2|1|16|1|1968|273|3551|Tuesday|1968Q1|N|N|N|2439857|2439856|2439507|2439780|N|N|N|N|N| +2439873|AAAAAAAABMKDFCAA|1968-01-17|816|3551|273|1968|3|1|17|1|1968|273|3551|Wednesday|1968Q1|N|N|N|2439857|2439856|2439508|2439781|N|N|N|N|N| +2439874|AAAAAAAACMKDFCAA|1968-01-18|816|3551|273|1968|4|1|18|1|1968|273|3551|Thursday|1968Q1|N|N|N|2439857|2439856|2439509|2439782|N|N|N|N|N| +2439875|AAAAAAAADMKDFCAA|1968-01-19|816|3551|273|1968|5|1|19|1|1968|273|3551|Friday|1968Q1|N|Y|N|2439857|2439856|2439510|2439783|N|N|N|N|N| +2439876|AAAAAAAAEMKDFCAA|1968-01-20|816|3551|273|1968|6|1|20|1|1968|273|3551|Saturday|1968Q1|N|Y|N|2439857|2439856|2439511|2439784|N|N|N|N|N| +2439877|AAAAAAAAFMKDFCAA|1968-01-21|816|3551|273|1968|0|1|21|1|1968|273|3551|Sunday|1968Q1|N|N|N|2439857|2439856|2439512|2439785|N|N|N|N|N| +2439878|AAAAAAAAGMKDFCAA|1968-01-22|816|3551|273|1968|1|1|22|1|1968|273|3551|Monday|1968Q1|N|N|N|2439857|2439856|2439513|2439786|N|N|N|N|N| +2439879|AAAAAAAAHMKDFCAA|1968-01-23|816|3552|273|1968|2|1|23|1|1968|273|3552|Tuesday|1968Q1|N|N|N|2439857|2439856|2439514|2439787|N|N|N|N|N| +2439880|AAAAAAAAIMKDFCAA|1968-01-24|816|3552|273|1968|3|1|24|1|1968|273|3552|Wednesday|1968Q1|N|N|N|2439857|2439856|2439515|2439788|N|N|N|N|N| +2439881|AAAAAAAAJMKDFCAA|1968-01-25|816|3552|273|1968|4|1|25|1|1968|273|3552|Thursday|1968Q1|N|N|N|2439857|2439856|2439516|2439789|N|N|N|N|N| +2439882|AAAAAAAAKMKDFCAA|1968-01-26|816|3552|273|1968|5|1|26|1|1968|273|3552|Friday|1968Q1|N|Y|N|2439857|2439856|2439517|2439790|N|N|N|N|N| +2439883|AAAAAAAALMKDFCAA|1968-01-27|816|3552|273|1968|6|1|27|1|1968|273|3552|Saturday|1968Q1|N|Y|N|2439857|2439856|2439518|2439791|N|N|N|N|N| +2439884|AAAAAAAAMMKDFCAA|1968-01-28|816|3552|273|1968|0|1|28|1|1968|273|3552|Sunday|1968Q1|N|N|N|2439857|2439856|2439519|2439792|N|N|N|N|N| +2439885|AAAAAAAANMKDFCAA|1968-01-29|816|3552|273|1968|1|1|29|1|1968|273|3552|Monday|1968Q1|N|N|N|2439857|2439856|2439520|2439793|N|N|N|N|N| +2439886|AAAAAAAAOMKDFCAA|1968-01-30|816|3553|273|1968|2|1|30|1|1968|273|3553|Tuesday|1968Q1|N|N|N|2439857|2439856|2439521|2439794|N|N|N|N|N| +2439887|AAAAAAAAPMKDFCAA|1968-01-31|816|3553|273|1968|3|1|31|1|1968|273|3553|Wednesday|1968Q1|N|N|N|2439857|2439856|2439522|2439795|N|N|N|N|N| +2439888|AAAAAAAAANKDFCAA|1968-02-01|817|3553|273|1968|4|2|1|1|1968|273|3553|Thursday|1968Q1|N|N|N|2439888|2439918|2439523|2439796|N|N|N|N|N| +2439889|AAAAAAAABNKDFCAA|1968-02-02|817|3553|273|1968|5|2|2|1|1968|273|3553|Friday|1968Q1|N|Y|N|2439888|2439918|2439524|2439797|N|N|N|N|N| +2439890|AAAAAAAACNKDFCAA|1968-02-03|817|3553|273|1968|6|2|3|1|1968|273|3553|Saturday|1968Q1|N|Y|N|2439888|2439918|2439525|2439798|N|N|N|N|N| +2439891|AAAAAAAADNKDFCAA|1968-02-04|817|3553|273|1968|0|2|4|1|1968|273|3553|Sunday|1968Q1|N|N|N|2439888|2439918|2439526|2439799|N|N|N|N|N| +2439892|AAAAAAAAENKDFCAA|1968-02-05|817|3553|273|1968|1|2|5|1|1968|273|3553|Monday|1968Q1|N|N|N|2439888|2439918|2439527|2439800|N|N|N|N|N| +2439893|AAAAAAAAFNKDFCAA|1968-02-06|817|3554|273|1968|2|2|6|1|1968|273|3554|Tuesday|1968Q1|N|N|N|2439888|2439918|2439528|2439801|N|N|N|N|N| +2439894|AAAAAAAAGNKDFCAA|1968-02-07|817|3554|273|1968|3|2|7|1|1968|273|3554|Wednesday|1968Q1|N|N|N|2439888|2439918|2439529|2439802|N|N|N|N|N| +2439895|AAAAAAAAHNKDFCAA|1968-02-08|817|3554|273|1968|4|2|8|1|1968|273|3554|Thursday|1968Q1|N|N|N|2439888|2439918|2439530|2439803|N|N|N|N|N| +2439896|AAAAAAAAINKDFCAA|1968-02-09|817|3554|273|1968|5|2|9|1|1968|273|3554|Friday|1968Q1|N|Y|N|2439888|2439918|2439531|2439804|N|N|N|N|N| +2439897|AAAAAAAAJNKDFCAA|1968-02-10|817|3554|273|1968|6|2|10|1|1968|273|3554|Saturday|1968Q1|N|Y|N|2439888|2439918|2439532|2439805|N|N|N|N|N| +2439898|AAAAAAAAKNKDFCAA|1968-02-11|817|3554|273|1968|0|2|11|1|1968|273|3554|Sunday|1968Q1|N|N|N|2439888|2439918|2439533|2439806|N|N|N|N|N| +2439899|AAAAAAAALNKDFCAA|1968-02-12|817|3554|273|1968|1|2|12|1|1968|273|3554|Monday|1968Q1|N|N|N|2439888|2439918|2439534|2439807|N|N|N|N|N| +2439900|AAAAAAAAMNKDFCAA|1968-02-13|817|3555|273|1968|2|2|13|1|1968|273|3555|Tuesday|1968Q1|N|N|N|2439888|2439918|2439535|2439808|N|N|N|N|N| +2439901|AAAAAAAANNKDFCAA|1968-02-14|817|3555|273|1968|3|2|14|1|1968|273|3555|Wednesday|1968Q1|N|N|N|2439888|2439918|2439536|2439809|N|N|N|N|N| +2439902|AAAAAAAAONKDFCAA|1968-02-15|817|3555|273|1968|4|2|15|1|1968|273|3555|Thursday|1968Q1|N|N|N|2439888|2439918|2439537|2439810|N|N|N|N|N| +2439903|AAAAAAAAPNKDFCAA|1968-02-16|817|3555|273|1968|5|2|16|1|1968|273|3555|Friday|1968Q1|N|Y|N|2439888|2439918|2439538|2439811|N|N|N|N|N| +2439904|AAAAAAAAAOKDFCAA|1968-02-17|817|3555|273|1968|6|2|17|1|1968|273|3555|Saturday|1968Q1|N|Y|N|2439888|2439918|2439539|2439812|N|N|N|N|N| +2439905|AAAAAAAABOKDFCAA|1968-02-18|817|3555|273|1968|0|2|18|1|1968|273|3555|Sunday|1968Q1|N|N|N|2439888|2439918|2439540|2439813|N|N|N|N|N| +2439906|AAAAAAAACOKDFCAA|1968-02-19|817|3555|273|1968|1|2|19|1|1968|273|3555|Monday|1968Q1|N|N|N|2439888|2439918|2439541|2439814|N|N|N|N|N| +2439907|AAAAAAAADOKDFCAA|1968-02-20|817|3556|273|1968|2|2|20|1|1968|273|3556|Tuesday|1968Q1|N|N|N|2439888|2439918|2439542|2439815|N|N|N|N|N| +2439908|AAAAAAAAEOKDFCAA|1968-02-21|817|3556|273|1968|3|2|21|1|1968|273|3556|Wednesday|1968Q1|N|N|N|2439888|2439918|2439543|2439816|N|N|N|N|N| +2439909|AAAAAAAAFOKDFCAA|1968-02-22|817|3556|273|1968|4|2|22|1|1968|273|3556|Thursday|1968Q1|N|N|N|2439888|2439918|2439544|2439817|N|N|N|N|N| +2439910|AAAAAAAAGOKDFCAA|1968-02-23|817|3556|273|1968|5|2|23|1|1968|273|3556|Friday|1968Q1|N|Y|N|2439888|2439918|2439545|2439818|N|N|N|N|N| +2439911|AAAAAAAAHOKDFCAA|1968-02-24|817|3556|273|1968|6|2|24|1|1968|273|3556|Saturday|1968Q1|N|Y|N|2439888|2439918|2439546|2439819|N|N|N|N|N| +2439912|AAAAAAAAIOKDFCAA|1968-02-25|817|3556|273|1968|0|2|25|1|1968|273|3556|Sunday|1968Q1|N|N|N|2439888|2439918|2439547|2439820|N|N|N|N|N| +2439913|AAAAAAAAJOKDFCAA|1968-02-26|817|3556|273|1968|1|2|26|1|1968|273|3556|Monday|1968Q1|N|N|N|2439888|2439918|2439548|2439821|N|N|N|N|N| +2439914|AAAAAAAAKOKDFCAA|1968-02-27|817|3557|273|1968|2|2|27|1|1968|273|3557|Tuesday|1968Q1|N|N|N|2439888|2439918|2439549|2439822|N|N|N|N|N| +2439915|AAAAAAAALOKDFCAA|1968-02-28|817|3557|273|1968|3|2|28|1|1968|273|3557|Wednesday|1968Q1|N|N|N|2439888|2439918|2439550|2439823|N|N|N|N|N| +2439916|AAAAAAAAMOKDFCAA|1968-02-29|817|3557|273|1968|4|2|29|1|1968|273|3557|Thursday|1968Q1|N|N|N|2439888|2439918|2439550|2439824|N|N|N|N|N| +2439917|AAAAAAAANOKDFCAA|1968-03-01|818|3557|274|1968|5|3|1|1|1968|274|3557|Friday|1968Q1|N|Y|N|2439917|2439976|2439551|2439825|N|N|N|N|N| +2439918|AAAAAAAAOOKDFCAA|1968-03-02|818|3557|274|1968|6|3|2|1|1968|274|3557|Saturday|1968Q1|N|Y|N|2439917|2439976|2439552|2439826|N|N|N|N|N| +2439919|AAAAAAAAPOKDFCAA|1968-03-03|818|3557|274|1968|0|3|3|1|1968|274|3557|Sunday|1968Q1|N|N|N|2439917|2439976|2439553|2439827|N|N|N|N|N| +2439920|AAAAAAAAAPKDFCAA|1968-03-04|818|3557|274|1968|1|3|4|1|1968|274|3557|Monday|1968Q1|N|N|N|2439917|2439976|2439554|2439828|N|N|N|N|N| +2439921|AAAAAAAABPKDFCAA|1968-03-05|818|3558|274|1968|2|3|5|1|1968|274|3558|Tuesday|1968Q1|N|N|N|2439917|2439976|2439555|2439829|N|N|N|N|N| +2439922|AAAAAAAACPKDFCAA|1968-03-06|818|3558|274|1968|3|3|6|1|1968|274|3558|Wednesday|1968Q1|N|N|N|2439917|2439976|2439556|2439830|N|N|N|N|N| +2439923|AAAAAAAADPKDFCAA|1968-03-07|818|3558|274|1968|4|3|7|1|1968|274|3558|Thursday|1968Q1|N|N|N|2439917|2439976|2439557|2439831|N|N|N|N|N| +2439924|AAAAAAAAEPKDFCAA|1968-03-08|818|3558|274|1968|5|3|8|1|1968|274|3558|Friday|1968Q1|N|Y|N|2439917|2439976|2439558|2439832|N|N|N|N|N| +2439925|AAAAAAAAFPKDFCAA|1968-03-09|818|3558|274|1968|6|3|9|1|1968|274|3558|Saturday|1968Q1|N|Y|N|2439917|2439976|2439559|2439833|N|N|N|N|N| +2439926|AAAAAAAAGPKDFCAA|1968-03-10|818|3558|274|1968|0|3|10|1|1968|274|3558|Sunday|1968Q1|N|N|N|2439917|2439976|2439560|2439834|N|N|N|N|N| +2439927|AAAAAAAAHPKDFCAA|1968-03-11|818|3558|274|1968|1|3|11|1|1968|274|3558|Monday|1968Q1|N|N|N|2439917|2439976|2439561|2439835|N|N|N|N|N| +2439928|AAAAAAAAIPKDFCAA|1968-03-12|818|3559|274|1968|2|3|12|1|1968|274|3559|Tuesday|1968Q1|N|N|N|2439917|2439976|2439562|2439836|N|N|N|N|N| +2439929|AAAAAAAAJPKDFCAA|1968-03-13|818|3559|274|1968|3|3|13|1|1968|274|3559|Wednesday|1968Q1|N|N|N|2439917|2439976|2439563|2439837|N|N|N|N|N| +2439930|AAAAAAAAKPKDFCAA|1968-03-14|818|3559|274|1968|4|3|14|1|1968|274|3559|Thursday|1968Q1|N|N|N|2439917|2439976|2439564|2439838|N|N|N|N|N| +2439931|AAAAAAAALPKDFCAA|1968-03-15|818|3559|274|1968|5|3|15|1|1968|274|3559|Friday|1968Q1|N|Y|N|2439917|2439976|2439565|2439839|N|N|N|N|N| +2439932|AAAAAAAAMPKDFCAA|1968-03-16|818|3559|274|1968|6|3|16|1|1968|274|3559|Saturday|1968Q1|N|Y|N|2439917|2439976|2439566|2439840|N|N|N|N|N| +2439933|AAAAAAAANPKDFCAA|1968-03-17|818|3559|274|1968|0|3|17|1|1968|274|3559|Sunday|1968Q1|N|N|N|2439917|2439976|2439567|2439841|N|N|N|N|N| +2439934|AAAAAAAAOPKDFCAA|1968-03-18|818|3559|274|1968|1|3|18|1|1968|274|3559|Monday|1968Q1|N|N|N|2439917|2439976|2439568|2439842|N|N|N|N|N| +2439935|AAAAAAAAPPKDFCAA|1968-03-19|818|3560|274|1968|2|3|19|1|1968|274|3560|Tuesday|1968Q1|N|N|N|2439917|2439976|2439569|2439843|N|N|N|N|N| +2439936|AAAAAAAAAALDFCAA|1968-03-20|818|3560|274|1968|3|3|20|1|1968|274|3560|Wednesday|1968Q1|N|N|N|2439917|2439976|2439570|2439844|N|N|N|N|N| +2439937|AAAAAAAABALDFCAA|1968-03-21|818|3560|274|1968|4|3|21|1|1968|274|3560|Thursday|1968Q1|N|N|N|2439917|2439976|2439571|2439845|N|N|N|N|N| +2439938|AAAAAAAACALDFCAA|1968-03-22|818|3560|274|1968|5|3|22|1|1968|274|3560|Friday|1968Q1|N|Y|N|2439917|2439976|2439572|2439846|N|N|N|N|N| +2439939|AAAAAAAADALDFCAA|1968-03-23|818|3560|274|1968|6|3|23|1|1968|274|3560|Saturday|1968Q1|N|Y|N|2439917|2439976|2439573|2439847|N|N|N|N|N| +2439940|AAAAAAAAEALDFCAA|1968-03-24|818|3560|274|1968|0|3|24|1|1968|274|3560|Sunday|1968Q1|N|N|N|2439917|2439976|2439574|2439848|N|N|N|N|N| +2439941|AAAAAAAAFALDFCAA|1968-03-25|818|3560|274|1968|1|3|25|1|1968|274|3560|Monday|1968Q1|N|N|N|2439917|2439976|2439575|2439849|N|N|N|N|N| +2439942|AAAAAAAAGALDFCAA|1968-03-26|818|3561|274|1968|2|3|26|1|1968|274|3561|Tuesday|1968Q1|N|N|N|2439917|2439976|2439576|2439850|N|N|N|N|N| +2439943|AAAAAAAAHALDFCAA|1968-03-27|818|3561|274|1968|3|3|27|1|1968|274|3561|Wednesday|1968Q1|N|N|N|2439917|2439976|2439577|2439851|N|N|N|N|N| +2439944|AAAAAAAAIALDFCAA|1968-03-28|818|3561|274|1968|4|3|28|1|1968|274|3561|Thursday|1968Q1|N|N|N|2439917|2439976|2439578|2439852|N|N|N|N|N| +2439945|AAAAAAAAJALDFCAA|1968-03-29|818|3561|274|1968|5|3|29|1|1968|274|3561|Friday|1968Q1|N|Y|N|2439917|2439976|2439579|2439853|N|N|N|N|N| +2439946|AAAAAAAAKALDFCAA|1968-03-30|818|3561|274|1968|6|3|30|1|1968|274|3561|Saturday|1968Q1|N|Y|N|2439917|2439976|2439580|2439854|N|N|N|N|N| +2439947|AAAAAAAALALDFCAA|1968-03-31|818|3561|274|1968|0|3|31|1|1968|274|3561|Sunday|1968Q1|N|N|N|2439917|2439976|2439581|2439855|N|N|N|N|N| +2439948|AAAAAAAAMALDFCAA|1968-04-01|819|3561|274|1968|1|4|1|2|1968|274|3561|Monday|1968Q2|N|N|N|2439948|2440038|2439582|2439857|N|N|N|N|N| +2439949|AAAAAAAANALDFCAA|1968-04-02|819|3562|274|1968|2|4|2|2|1968|274|3562|Tuesday|1968Q2|N|N|N|2439948|2440038|2439583|2439858|N|N|N|N|N| +2439950|AAAAAAAAOALDFCAA|1968-04-03|819|3562|274|1968|3|4|3|2|1968|274|3562|Wednesday|1968Q2|N|N|N|2439948|2440038|2439584|2439859|N|N|N|N|N| +2439951|AAAAAAAAPALDFCAA|1968-04-04|819|3562|274|1968|4|4|4|2|1968|274|3562|Thursday|1968Q2|N|N|N|2439948|2440038|2439585|2439860|N|N|N|N|N| +2439952|AAAAAAAAABLDFCAA|1968-04-05|819|3562|274|1968|5|4|5|2|1968|274|3562|Friday|1968Q2|N|Y|N|2439948|2440038|2439586|2439861|N|N|N|N|N| +2439953|AAAAAAAABBLDFCAA|1968-04-06|819|3562|274|1968|6|4|6|2|1968|274|3562|Saturday|1968Q2|N|Y|N|2439948|2440038|2439587|2439862|N|N|N|N|N| +2439954|AAAAAAAACBLDFCAA|1968-04-07|819|3562|274|1968|0|4|7|2|1968|274|3562|Sunday|1968Q2|N|N|N|2439948|2440038|2439588|2439863|N|N|N|N|N| +2439955|AAAAAAAADBLDFCAA|1968-04-08|819|3562|274|1968|1|4|8|2|1968|274|3562|Monday|1968Q2|N|N|N|2439948|2440038|2439589|2439864|N|N|N|N|N| +2439956|AAAAAAAAEBLDFCAA|1968-04-09|819|3563|274|1968|2|4|9|2|1968|274|3563|Tuesday|1968Q2|N|N|N|2439948|2440038|2439590|2439865|N|N|N|N|N| +2439957|AAAAAAAAFBLDFCAA|1968-04-10|819|3563|274|1968|3|4|10|2|1968|274|3563|Wednesday|1968Q2|N|N|N|2439948|2440038|2439591|2439866|N|N|N|N|N| +2439958|AAAAAAAAGBLDFCAA|1968-04-11|819|3563|274|1968|4|4|11|2|1968|274|3563|Thursday|1968Q2|N|N|N|2439948|2440038|2439592|2439867|N|N|N|N|N| +2439959|AAAAAAAAHBLDFCAA|1968-04-12|819|3563|274|1968|5|4|12|2|1968|274|3563|Friday|1968Q2|N|Y|N|2439948|2440038|2439593|2439868|N|N|N|N|N| +2439960|AAAAAAAAIBLDFCAA|1968-04-13|819|3563|274|1968|6|4|13|2|1968|274|3563|Saturday|1968Q2|N|Y|N|2439948|2440038|2439594|2439869|N|N|N|N|N| +2439961|AAAAAAAAJBLDFCAA|1968-04-14|819|3563|274|1968|0|4|14|2|1968|274|3563|Sunday|1968Q2|N|N|N|2439948|2440038|2439595|2439870|N|N|N|N|N| +2439962|AAAAAAAAKBLDFCAA|1968-04-15|819|3563|274|1968|1|4|15|2|1968|274|3563|Monday|1968Q2|N|N|N|2439948|2440038|2439596|2439871|N|N|N|N|N| +2439963|AAAAAAAALBLDFCAA|1968-04-16|819|3564|274|1968|2|4|16|2|1968|274|3564|Tuesday|1968Q2|N|N|N|2439948|2440038|2439597|2439872|N|N|N|N|N| +2439964|AAAAAAAAMBLDFCAA|1968-04-17|819|3564|274|1968|3|4|17|2|1968|274|3564|Wednesday|1968Q2|N|N|N|2439948|2440038|2439598|2439873|N|N|N|N|N| +2439965|AAAAAAAANBLDFCAA|1968-04-18|819|3564|274|1968|4|4|18|2|1968|274|3564|Thursday|1968Q2|N|N|N|2439948|2440038|2439599|2439874|N|N|N|N|N| +2439966|AAAAAAAAOBLDFCAA|1968-04-19|819|3564|274|1968|5|4|19|2|1968|274|3564|Friday|1968Q2|N|Y|N|2439948|2440038|2439600|2439875|N|N|N|N|N| +2439967|AAAAAAAAPBLDFCAA|1968-04-20|819|3564|274|1968|6|4|20|2|1968|274|3564|Saturday|1968Q2|N|Y|N|2439948|2440038|2439601|2439876|N|N|N|N|N| +2439968|AAAAAAAAACLDFCAA|1968-04-21|819|3564|274|1968|0|4|21|2|1968|274|3564|Sunday|1968Q2|N|N|N|2439948|2440038|2439602|2439877|N|N|N|N|N| +2439969|AAAAAAAABCLDFCAA|1968-04-22|819|3564|274|1968|1|4|22|2|1968|274|3564|Monday|1968Q2|N|N|N|2439948|2440038|2439603|2439878|N|N|N|N|N| +2439970|AAAAAAAACCLDFCAA|1968-04-23|819|3565|274|1968|2|4|23|2|1968|274|3565|Tuesday|1968Q2|N|N|N|2439948|2440038|2439604|2439879|N|N|N|N|N| +2439971|AAAAAAAADCLDFCAA|1968-04-24|819|3565|274|1968|3|4|24|2|1968|274|3565|Wednesday|1968Q2|N|N|N|2439948|2440038|2439605|2439880|N|N|N|N|N| +2439972|AAAAAAAAECLDFCAA|1968-04-25|819|3565|274|1968|4|4|25|2|1968|274|3565|Thursday|1968Q2|N|N|N|2439948|2440038|2439606|2439881|N|N|N|N|N| +2439973|AAAAAAAAFCLDFCAA|1968-04-26|819|3565|274|1968|5|4|26|2|1968|274|3565|Friday|1968Q2|N|Y|N|2439948|2440038|2439607|2439882|N|N|N|N|N| +2439974|AAAAAAAAGCLDFCAA|1968-04-27|819|3565|274|1968|6|4|27|2|1968|274|3565|Saturday|1968Q2|N|Y|N|2439948|2440038|2439608|2439883|N|N|N|N|N| +2439975|AAAAAAAAHCLDFCAA|1968-04-28|819|3565|274|1968|0|4|28|2|1968|274|3565|Sunday|1968Q2|N|N|N|2439948|2440038|2439609|2439884|N|N|N|N|N| +2439976|AAAAAAAAICLDFCAA|1968-04-29|819|3565|274|1968|1|4|29|2|1968|274|3565|Monday|1968Q2|N|N|N|2439948|2440038|2439610|2439885|N|N|N|N|N| +2439977|AAAAAAAAJCLDFCAA|1968-04-30|819|3566|274|1968|2|4|30|2|1968|274|3566|Tuesday|1968Q2|N|N|N|2439948|2440038|2439611|2439886|N|N|N|N|N| +2439978|AAAAAAAAKCLDFCAA|1968-05-01|820|3566|274|1968|3|5|1|2|1968|274|3566|Wednesday|1968Q2|N|N|N|2439978|2440098|2439612|2439887|N|N|N|N|N| +2439979|AAAAAAAALCLDFCAA|1968-05-02|820|3566|274|1968|4|5|2|2|1968|274|3566|Thursday|1968Q2|N|N|N|2439978|2440098|2439613|2439888|N|N|N|N|N| +2439980|AAAAAAAAMCLDFCAA|1968-05-03|820|3566|274|1968|5|5|3|2|1968|274|3566|Friday|1968Q2|N|Y|N|2439978|2440098|2439614|2439889|N|N|N|N|N| +2439981|AAAAAAAANCLDFCAA|1968-05-04|820|3566|274|1968|6|5|4|2|1968|274|3566|Saturday|1968Q2|N|Y|N|2439978|2440098|2439615|2439890|N|N|N|N|N| +2439982|AAAAAAAAOCLDFCAA|1968-05-05|820|3566|274|1968|0|5|5|2|1968|274|3566|Sunday|1968Q2|N|N|N|2439978|2440098|2439616|2439891|N|N|N|N|N| +2439983|AAAAAAAAPCLDFCAA|1968-05-06|820|3566|274|1968|1|5|6|2|1968|274|3566|Monday|1968Q2|N|N|N|2439978|2440098|2439617|2439892|N|N|N|N|N| +2439984|AAAAAAAAADLDFCAA|1968-05-07|820|3567|274|1968|2|5|7|2|1968|274|3567|Tuesday|1968Q2|N|N|N|2439978|2440098|2439618|2439893|N|N|N|N|N| +2439985|AAAAAAAABDLDFCAA|1968-05-08|820|3567|274|1968|3|5|8|2|1968|274|3567|Wednesday|1968Q2|N|N|N|2439978|2440098|2439619|2439894|N|N|N|N|N| +2439986|AAAAAAAACDLDFCAA|1968-05-09|820|3567|274|1968|4|5|9|2|1968|274|3567|Thursday|1968Q2|N|N|N|2439978|2440098|2439620|2439895|N|N|N|N|N| +2439987|AAAAAAAADDLDFCAA|1968-05-10|820|3567|274|1968|5|5|10|2|1968|274|3567|Friday|1968Q2|N|Y|N|2439978|2440098|2439621|2439896|N|N|N|N|N| +2439988|AAAAAAAAEDLDFCAA|1968-05-11|820|3567|274|1968|6|5|11|2|1968|274|3567|Saturday|1968Q2|N|Y|N|2439978|2440098|2439622|2439897|N|N|N|N|N| +2439989|AAAAAAAAFDLDFCAA|1968-05-12|820|3567|274|1968|0|5|12|2|1968|274|3567|Sunday|1968Q2|N|N|N|2439978|2440098|2439623|2439898|N|N|N|N|N| +2439990|AAAAAAAAGDLDFCAA|1968-05-13|820|3567|274|1968|1|5|13|2|1968|274|3567|Monday|1968Q2|N|N|N|2439978|2440098|2439624|2439899|N|N|N|N|N| +2439991|AAAAAAAAHDLDFCAA|1968-05-14|820|3568|274|1968|2|5|14|2|1968|274|3568|Tuesday|1968Q2|N|N|N|2439978|2440098|2439625|2439900|N|N|N|N|N| +2439992|AAAAAAAAIDLDFCAA|1968-05-15|820|3568|274|1968|3|5|15|2|1968|274|3568|Wednesday|1968Q2|N|N|N|2439978|2440098|2439626|2439901|N|N|N|N|N| +2439993|AAAAAAAAJDLDFCAA|1968-05-16|820|3568|274|1968|4|5|16|2|1968|274|3568|Thursday|1968Q2|N|N|N|2439978|2440098|2439627|2439902|N|N|N|N|N| +2439994|AAAAAAAAKDLDFCAA|1968-05-17|820|3568|274|1968|5|5|17|2|1968|274|3568|Friday|1968Q2|N|Y|N|2439978|2440098|2439628|2439903|N|N|N|N|N| +2439995|AAAAAAAALDLDFCAA|1968-05-18|820|3568|274|1968|6|5|18|2|1968|274|3568|Saturday|1968Q2|N|Y|N|2439978|2440098|2439629|2439904|N|N|N|N|N| +2439996|AAAAAAAAMDLDFCAA|1968-05-19|820|3568|274|1968|0|5|19|2|1968|274|3568|Sunday|1968Q2|N|N|N|2439978|2440098|2439630|2439905|N|N|N|N|N| +2439997|AAAAAAAANDLDFCAA|1968-05-20|820|3568|274|1968|1|5|20|2|1968|274|3568|Monday|1968Q2|N|N|N|2439978|2440098|2439631|2439906|N|N|N|N|N| +2439998|AAAAAAAAODLDFCAA|1968-05-21|820|3569|274|1968|2|5|21|2|1968|274|3569|Tuesday|1968Q2|N|N|N|2439978|2440098|2439632|2439907|N|N|N|N|N| +2439999|AAAAAAAAPDLDFCAA|1968-05-22|820|3569|274|1968|3|5|22|2|1968|274|3569|Wednesday|1968Q2|N|N|N|2439978|2440098|2439633|2439908|N|N|N|N|N| +2440000|AAAAAAAAAELDFCAA|1968-05-23|820|3569|274|1968|4|5|23|2|1968|274|3569|Thursday|1968Q2|N|N|N|2439978|2440098|2439634|2439909|N|N|N|N|N| +2440001|AAAAAAAABELDFCAA|1968-05-24|820|3569|274|1968|5|5|24|2|1968|274|3569|Friday|1968Q2|N|Y|N|2439978|2440098|2439635|2439910|N|N|N|N|N| +2440002|AAAAAAAACELDFCAA|1968-05-25|820|3569|274|1968|6|5|25|2|1968|274|3569|Saturday|1968Q2|N|Y|N|2439978|2440098|2439636|2439911|N|N|N|N|N| +2440003|AAAAAAAADELDFCAA|1968-05-26|820|3569|274|1968|0|5|26|2|1968|274|3569|Sunday|1968Q2|N|N|N|2439978|2440098|2439637|2439912|N|N|N|N|N| +2440004|AAAAAAAAEELDFCAA|1968-05-27|820|3569|274|1968|1|5|27|2|1968|274|3569|Monday|1968Q2|N|N|N|2439978|2440098|2439638|2439913|N|N|N|N|N| +2440005|AAAAAAAAFELDFCAA|1968-05-28|820|3570|274|1968|2|5|28|2|1968|274|3570|Tuesday|1968Q2|N|N|N|2439978|2440098|2439639|2439914|N|N|N|N|N| +2440006|AAAAAAAAGELDFCAA|1968-05-29|820|3570|274|1968|3|5|29|2|1968|274|3570|Wednesday|1968Q2|N|N|N|2439978|2440098|2439640|2439915|N|N|N|N|N| +2440007|AAAAAAAAHELDFCAA|1968-05-30|820|3570|274|1968|4|5|30|2|1968|274|3570|Thursday|1968Q2|N|N|N|2439978|2440098|2439641|2439916|N|N|N|N|N| +2440008|AAAAAAAAIELDFCAA|1968-05-31|820|3570|274|1968|5|5|31|2|1968|274|3570|Friday|1968Q2|N|Y|N|2439978|2440098|2439642|2439917|N|N|N|N|N| +2440009|AAAAAAAAJELDFCAA|1968-06-01|821|3570|275|1968|6|6|1|2|1968|275|3570|Saturday|1968Q2|N|Y|N|2440009|2440160|2439643|2439918|N|N|N|N|N| +2440010|AAAAAAAAKELDFCAA|1968-06-02|821|3570|275|1968|0|6|2|2|1968|275|3570|Sunday|1968Q2|N|N|N|2440009|2440160|2439644|2439919|N|N|N|N|N| +2440011|AAAAAAAALELDFCAA|1968-06-03|821|3570|275|1968|1|6|3|2|1968|275|3570|Monday|1968Q2|N|N|N|2440009|2440160|2439645|2439920|N|N|N|N|N| +2440012|AAAAAAAAMELDFCAA|1968-06-04|821|3571|275|1968|2|6|4|2|1968|275|3571|Tuesday|1968Q2|N|N|N|2440009|2440160|2439646|2439921|N|N|N|N|N| +2440013|AAAAAAAANELDFCAA|1968-06-05|821|3571|275|1968|3|6|5|2|1968|275|3571|Wednesday|1968Q2|N|N|N|2440009|2440160|2439647|2439922|N|N|N|N|N| +2440014|AAAAAAAAOELDFCAA|1968-06-06|821|3571|275|1968|4|6|6|2|1968|275|3571|Thursday|1968Q2|N|N|N|2440009|2440160|2439648|2439923|N|N|N|N|N| +2440015|AAAAAAAAPELDFCAA|1968-06-07|821|3571|275|1968|5|6|7|2|1968|275|3571|Friday|1968Q2|N|Y|N|2440009|2440160|2439649|2439924|N|N|N|N|N| +2440016|AAAAAAAAAFLDFCAA|1968-06-08|821|3571|275|1968|6|6|8|2|1968|275|3571|Saturday|1968Q2|N|Y|N|2440009|2440160|2439650|2439925|N|N|N|N|N| +2440017|AAAAAAAABFLDFCAA|1968-06-09|821|3571|275|1968|0|6|9|2|1968|275|3571|Sunday|1968Q2|N|N|N|2440009|2440160|2439651|2439926|N|N|N|N|N| +2440018|AAAAAAAACFLDFCAA|1968-06-10|821|3571|275|1968|1|6|10|2|1968|275|3571|Monday|1968Q2|N|N|N|2440009|2440160|2439652|2439927|N|N|N|N|N| +2440019|AAAAAAAADFLDFCAA|1968-06-11|821|3572|275|1968|2|6|11|2|1968|275|3572|Tuesday|1968Q2|N|N|N|2440009|2440160|2439653|2439928|N|N|N|N|N| +2440020|AAAAAAAAEFLDFCAA|1968-06-12|821|3572|275|1968|3|6|12|2|1968|275|3572|Wednesday|1968Q2|N|N|N|2440009|2440160|2439654|2439929|N|N|N|N|N| +2440021|AAAAAAAAFFLDFCAA|1968-06-13|821|3572|275|1968|4|6|13|2|1968|275|3572|Thursday|1968Q2|N|N|N|2440009|2440160|2439655|2439930|N|N|N|N|N| +2440022|AAAAAAAAGFLDFCAA|1968-06-14|821|3572|275|1968|5|6|14|2|1968|275|3572|Friday|1968Q2|N|Y|N|2440009|2440160|2439656|2439931|N|N|N|N|N| +2440023|AAAAAAAAHFLDFCAA|1968-06-15|821|3572|275|1968|6|6|15|2|1968|275|3572|Saturday|1968Q2|N|Y|N|2440009|2440160|2439657|2439932|N|N|N|N|N| +2440024|AAAAAAAAIFLDFCAA|1968-06-16|821|3572|275|1968|0|6|16|2|1968|275|3572|Sunday|1968Q2|N|N|N|2440009|2440160|2439658|2439933|N|N|N|N|N| +2440025|AAAAAAAAJFLDFCAA|1968-06-17|821|3572|275|1968|1|6|17|2|1968|275|3572|Monday|1968Q2|N|N|N|2440009|2440160|2439659|2439934|N|N|N|N|N| +2440026|AAAAAAAAKFLDFCAA|1968-06-18|821|3573|275|1968|2|6|18|2|1968|275|3573|Tuesday|1968Q2|N|N|N|2440009|2440160|2439660|2439935|N|N|N|N|N| +2440027|AAAAAAAALFLDFCAA|1968-06-19|821|3573|275|1968|3|6|19|2|1968|275|3573|Wednesday|1968Q2|N|N|N|2440009|2440160|2439661|2439936|N|N|N|N|N| +2440028|AAAAAAAAMFLDFCAA|1968-06-20|821|3573|275|1968|4|6|20|2|1968|275|3573|Thursday|1968Q2|N|N|N|2440009|2440160|2439662|2439937|N|N|N|N|N| +2440029|AAAAAAAANFLDFCAA|1968-06-21|821|3573|275|1968|5|6|21|2|1968|275|3573|Friday|1968Q2|N|Y|N|2440009|2440160|2439663|2439938|N|N|N|N|N| +2440030|AAAAAAAAOFLDFCAA|1968-06-22|821|3573|275|1968|6|6|22|2|1968|275|3573|Saturday|1968Q2|N|Y|N|2440009|2440160|2439664|2439939|N|N|N|N|N| +2440031|AAAAAAAAPFLDFCAA|1968-06-23|821|3573|275|1968|0|6|23|2|1968|275|3573|Sunday|1968Q2|N|N|N|2440009|2440160|2439665|2439940|N|N|N|N|N| +2440032|AAAAAAAAAGLDFCAA|1968-06-24|821|3573|275|1968|1|6|24|2|1968|275|3573|Monday|1968Q2|N|N|N|2440009|2440160|2439666|2439941|N|N|N|N|N| +2440033|AAAAAAAABGLDFCAA|1968-06-25|821|3574|275|1968|2|6|25|2|1968|275|3574|Tuesday|1968Q2|N|N|N|2440009|2440160|2439667|2439942|N|N|N|N|N| +2440034|AAAAAAAACGLDFCAA|1968-06-26|821|3574|275|1968|3|6|26|2|1968|275|3574|Wednesday|1968Q2|N|N|N|2440009|2440160|2439668|2439943|N|N|N|N|N| +2440035|AAAAAAAADGLDFCAA|1968-06-27|821|3574|275|1968|4|6|27|2|1968|275|3574|Thursday|1968Q2|N|N|N|2440009|2440160|2439669|2439944|N|N|N|N|N| +2440036|AAAAAAAAEGLDFCAA|1968-06-28|821|3574|275|1968|5|6|28|2|1968|275|3574|Friday|1968Q2|N|Y|N|2440009|2440160|2439670|2439945|N|N|N|N|N| +2440037|AAAAAAAAFGLDFCAA|1968-06-29|821|3574|275|1968|6|6|29|2|1968|275|3574|Saturday|1968Q2|N|Y|N|2440009|2440160|2439671|2439946|N|N|N|N|N| +2440038|AAAAAAAAGGLDFCAA|1968-06-30|821|3574|275|1968|0|6|30|2|1968|275|3574|Sunday|1968Q2|N|N|N|2440009|2440160|2439672|2439947|N|N|N|N|N| +2440039|AAAAAAAAHGLDFCAA|1968-07-01|822|3574|275|1968|1|7|1|3|1968|275|3574|Monday|1968Q3|N|N|N|2440039|2440220|2439673|2439948|N|N|N|N|N| +2440040|AAAAAAAAIGLDFCAA|1968-07-02|822|3575|275|1968|2|7|2|3|1968|275|3575|Tuesday|1968Q3|N|N|N|2440039|2440220|2439674|2439949|N|N|N|N|N| +2440041|AAAAAAAAJGLDFCAA|1968-07-03|822|3575|275|1968|3|7|3|3|1968|275|3575|Wednesday|1968Q3|N|N|N|2440039|2440220|2439675|2439950|N|N|N|N|N| +2440042|AAAAAAAAKGLDFCAA|1968-07-04|822|3575|275|1968|4|7|4|3|1968|275|3575|Thursday|1968Q3|Y|N|N|2440039|2440220|2439676|2439951|N|N|N|N|N| +2440043|AAAAAAAALGLDFCAA|1968-07-05|822|3575|275|1968|5|7|5|3|1968|275|3575|Friday|1968Q3|N|Y|Y|2440039|2440220|2439677|2439952|N|N|N|N|N| +2440044|AAAAAAAAMGLDFCAA|1968-07-06|822|3575|275|1968|6|7|6|3|1968|275|3575|Saturday|1968Q3|N|Y|N|2440039|2440220|2439678|2439953|N|N|N|N|N| +2440045|AAAAAAAANGLDFCAA|1968-07-07|822|3575|275|1968|0|7|7|3|1968|275|3575|Sunday|1968Q3|N|N|N|2440039|2440220|2439679|2439954|N|N|N|N|N| +2440046|AAAAAAAAOGLDFCAA|1968-07-08|822|3575|275|1968|1|7|8|3|1968|275|3575|Monday|1968Q3|N|N|N|2440039|2440220|2439680|2439955|N|N|N|N|N| +2440047|AAAAAAAAPGLDFCAA|1968-07-09|822|3576|275|1968|2|7|9|3|1968|275|3576|Tuesday|1968Q3|N|N|N|2440039|2440220|2439681|2439956|N|N|N|N|N| +2440048|AAAAAAAAAHLDFCAA|1968-07-10|822|3576|275|1968|3|7|10|3|1968|275|3576|Wednesday|1968Q3|N|N|N|2440039|2440220|2439682|2439957|N|N|N|N|N| +2440049|AAAAAAAABHLDFCAA|1968-07-11|822|3576|275|1968|4|7|11|3|1968|275|3576|Thursday|1968Q3|N|N|N|2440039|2440220|2439683|2439958|N|N|N|N|N| +2440050|AAAAAAAACHLDFCAA|1968-07-12|822|3576|275|1968|5|7|12|3|1968|275|3576|Friday|1968Q3|N|Y|N|2440039|2440220|2439684|2439959|N|N|N|N|N| +2440051|AAAAAAAADHLDFCAA|1968-07-13|822|3576|275|1968|6|7|13|3|1968|275|3576|Saturday|1968Q3|N|Y|N|2440039|2440220|2439685|2439960|N|N|N|N|N| +2440052|AAAAAAAAEHLDFCAA|1968-07-14|822|3576|275|1968|0|7|14|3|1968|275|3576|Sunday|1968Q3|N|N|N|2440039|2440220|2439686|2439961|N|N|N|N|N| +2440053|AAAAAAAAFHLDFCAA|1968-07-15|822|3576|275|1968|1|7|15|3|1968|275|3576|Monday|1968Q3|N|N|N|2440039|2440220|2439687|2439962|N|N|N|N|N| +2440054|AAAAAAAAGHLDFCAA|1968-07-16|822|3577|275|1968|2|7|16|3|1968|275|3577|Tuesday|1968Q3|N|N|N|2440039|2440220|2439688|2439963|N|N|N|N|N| +2440055|AAAAAAAAHHLDFCAA|1968-07-17|822|3577|275|1968|3|7|17|3|1968|275|3577|Wednesday|1968Q3|N|N|N|2440039|2440220|2439689|2439964|N|N|N|N|N| +2440056|AAAAAAAAIHLDFCAA|1968-07-18|822|3577|275|1968|4|7|18|3|1968|275|3577|Thursday|1968Q3|N|N|N|2440039|2440220|2439690|2439965|N|N|N|N|N| +2440057|AAAAAAAAJHLDFCAA|1968-07-19|822|3577|275|1968|5|7|19|3|1968|275|3577|Friday|1968Q3|N|Y|N|2440039|2440220|2439691|2439966|N|N|N|N|N| +2440058|AAAAAAAAKHLDFCAA|1968-07-20|822|3577|275|1968|6|7|20|3|1968|275|3577|Saturday|1968Q3|N|Y|N|2440039|2440220|2439692|2439967|N|N|N|N|N| +2440059|AAAAAAAALHLDFCAA|1968-07-21|822|3577|275|1968|0|7|21|3|1968|275|3577|Sunday|1968Q3|N|N|N|2440039|2440220|2439693|2439968|N|N|N|N|N| +2440060|AAAAAAAAMHLDFCAA|1968-07-22|822|3577|275|1968|1|7|22|3|1968|275|3577|Monday|1968Q3|N|N|N|2440039|2440220|2439694|2439969|N|N|N|N|N| +2440061|AAAAAAAANHLDFCAA|1968-07-23|822|3578|275|1968|2|7|23|3|1968|275|3578|Tuesday|1968Q3|N|N|N|2440039|2440220|2439695|2439970|N|N|N|N|N| +2440062|AAAAAAAAOHLDFCAA|1968-07-24|822|3578|275|1968|3|7|24|3|1968|275|3578|Wednesday|1968Q3|N|N|N|2440039|2440220|2439696|2439971|N|N|N|N|N| +2440063|AAAAAAAAPHLDFCAA|1968-07-25|822|3578|275|1968|4|7|25|3|1968|275|3578|Thursday|1968Q3|N|N|N|2440039|2440220|2439697|2439972|N|N|N|N|N| +2440064|AAAAAAAAAILDFCAA|1968-07-26|822|3578|275|1968|5|7|26|3|1968|275|3578|Friday|1968Q3|N|Y|N|2440039|2440220|2439698|2439973|N|N|N|N|N| +2440065|AAAAAAAABILDFCAA|1968-07-27|822|3578|275|1968|6|7|27|3|1968|275|3578|Saturday|1968Q3|N|Y|N|2440039|2440220|2439699|2439974|N|N|N|N|N| +2440066|AAAAAAAACILDFCAA|1968-07-28|822|3578|275|1968|0|7|28|3|1968|275|3578|Sunday|1968Q3|N|N|N|2440039|2440220|2439700|2439975|N|N|N|N|N| +2440067|AAAAAAAADILDFCAA|1968-07-29|822|3578|275|1968|1|7|29|3|1968|275|3578|Monday|1968Q3|N|N|N|2440039|2440220|2439701|2439976|N|N|N|N|N| +2440068|AAAAAAAAEILDFCAA|1968-07-30|822|3579|275|1968|2|7|30|3|1968|275|3579|Tuesday|1968Q3|N|N|N|2440039|2440220|2439702|2439977|N|N|N|N|N| +2440069|AAAAAAAAFILDFCAA|1968-07-31|822|3579|275|1968|3|7|31|3|1968|275|3579|Wednesday|1968Q3|N|N|N|2440039|2440220|2439703|2439978|N|N|N|N|N| +2440070|AAAAAAAAGILDFCAA|1968-08-01|823|3579|275|1968|4|8|1|3|1968|275|3579|Thursday|1968Q3|N|N|N|2440070|2440282|2439704|2439979|N|N|N|N|N| +2440071|AAAAAAAAHILDFCAA|1968-08-02|823|3579|275|1968|5|8|2|3|1968|275|3579|Friday|1968Q3|N|Y|N|2440070|2440282|2439705|2439980|N|N|N|N|N| +2440072|AAAAAAAAIILDFCAA|1968-08-03|823|3579|275|1968|6|8|3|3|1968|275|3579|Saturday|1968Q3|N|Y|N|2440070|2440282|2439706|2439981|N|N|N|N|N| +2440073|AAAAAAAAJILDFCAA|1968-08-04|823|3579|275|1968|0|8|4|3|1968|275|3579|Sunday|1968Q3|N|N|N|2440070|2440282|2439707|2439982|N|N|N|N|N| +2440074|AAAAAAAAKILDFCAA|1968-08-05|823|3579|275|1968|1|8|5|3|1968|275|3579|Monday|1968Q3|N|N|N|2440070|2440282|2439708|2439983|N|N|N|N|N| +2440075|AAAAAAAALILDFCAA|1968-08-06|823|3580|275|1968|2|8|6|3|1968|275|3580|Tuesday|1968Q3|N|N|N|2440070|2440282|2439709|2439984|N|N|N|N|N| +2440076|AAAAAAAAMILDFCAA|1968-08-07|823|3580|275|1968|3|8|7|3|1968|275|3580|Wednesday|1968Q3|N|N|N|2440070|2440282|2439710|2439985|N|N|N|N|N| +2440077|AAAAAAAANILDFCAA|1968-08-08|823|3580|275|1968|4|8|8|3|1968|275|3580|Thursday|1968Q3|N|N|N|2440070|2440282|2439711|2439986|N|N|N|N|N| +2440078|AAAAAAAAOILDFCAA|1968-08-09|823|3580|275|1968|5|8|9|3|1968|275|3580|Friday|1968Q3|N|Y|N|2440070|2440282|2439712|2439987|N|N|N|N|N| +2440079|AAAAAAAAPILDFCAA|1968-08-10|823|3580|275|1968|6|8|10|3|1968|275|3580|Saturday|1968Q3|N|Y|N|2440070|2440282|2439713|2439988|N|N|N|N|N| +2440080|AAAAAAAAAJLDFCAA|1968-08-11|823|3580|275|1968|0|8|11|3|1968|275|3580|Sunday|1968Q3|N|N|N|2440070|2440282|2439714|2439989|N|N|N|N|N| +2440081|AAAAAAAABJLDFCAA|1968-08-12|823|3580|275|1968|1|8|12|3|1968|275|3580|Monday|1968Q3|N|N|N|2440070|2440282|2439715|2439990|N|N|N|N|N| +2440082|AAAAAAAACJLDFCAA|1968-08-13|823|3581|275|1968|2|8|13|3|1968|275|3581|Tuesday|1968Q3|N|N|N|2440070|2440282|2439716|2439991|N|N|N|N|N| +2440083|AAAAAAAADJLDFCAA|1968-08-14|823|3581|275|1968|3|8|14|3|1968|275|3581|Wednesday|1968Q3|N|N|N|2440070|2440282|2439717|2439992|N|N|N|N|N| +2440084|AAAAAAAAEJLDFCAA|1968-08-15|823|3581|275|1968|4|8|15|3|1968|275|3581|Thursday|1968Q3|N|N|N|2440070|2440282|2439718|2439993|N|N|N|N|N| +2440085|AAAAAAAAFJLDFCAA|1968-08-16|823|3581|275|1968|5|8|16|3|1968|275|3581|Friday|1968Q3|N|Y|N|2440070|2440282|2439719|2439994|N|N|N|N|N| +2440086|AAAAAAAAGJLDFCAA|1968-08-17|823|3581|275|1968|6|8|17|3|1968|275|3581|Saturday|1968Q3|N|Y|N|2440070|2440282|2439720|2439995|N|N|N|N|N| +2440087|AAAAAAAAHJLDFCAA|1968-08-18|823|3581|275|1968|0|8|18|3|1968|275|3581|Sunday|1968Q3|N|N|N|2440070|2440282|2439721|2439996|N|N|N|N|N| +2440088|AAAAAAAAIJLDFCAA|1968-08-19|823|3581|275|1968|1|8|19|3|1968|275|3581|Monday|1968Q3|N|N|N|2440070|2440282|2439722|2439997|N|N|N|N|N| +2440089|AAAAAAAAJJLDFCAA|1968-08-20|823|3582|275|1968|2|8|20|3|1968|275|3582|Tuesday|1968Q3|N|N|N|2440070|2440282|2439723|2439998|N|N|N|N|N| +2440090|AAAAAAAAKJLDFCAA|1968-08-21|823|3582|275|1968|3|8|21|3|1968|275|3582|Wednesday|1968Q3|N|N|N|2440070|2440282|2439724|2439999|N|N|N|N|N| +2440091|AAAAAAAALJLDFCAA|1968-08-22|823|3582|275|1968|4|8|22|3|1968|275|3582|Thursday|1968Q3|N|N|N|2440070|2440282|2439725|2440000|N|N|N|N|N| +2440092|AAAAAAAAMJLDFCAA|1968-08-23|823|3582|275|1968|5|8|23|3|1968|275|3582|Friday|1968Q3|N|Y|N|2440070|2440282|2439726|2440001|N|N|N|N|N| +2440093|AAAAAAAANJLDFCAA|1968-08-24|823|3582|275|1968|6|8|24|3|1968|275|3582|Saturday|1968Q3|N|Y|N|2440070|2440282|2439727|2440002|N|N|N|N|N| +2440094|AAAAAAAAOJLDFCAA|1968-08-25|823|3582|275|1968|0|8|25|3|1968|275|3582|Sunday|1968Q3|N|N|N|2440070|2440282|2439728|2440003|N|N|N|N|N| +2440095|AAAAAAAAPJLDFCAA|1968-08-26|823|3582|275|1968|1|8|26|3|1968|275|3582|Monday|1968Q3|N|N|N|2440070|2440282|2439729|2440004|N|N|N|N|N| +2440096|AAAAAAAAAKLDFCAA|1968-08-27|823|3583|275|1968|2|8|27|3|1968|275|3583|Tuesday|1968Q3|N|N|N|2440070|2440282|2439730|2440005|N|N|N|N|N| +2440097|AAAAAAAABKLDFCAA|1968-08-28|823|3583|275|1968|3|8|28|3|1968|275|3583|Wednesday|1968Q3|N|N|N|2440070|2440282|2439731|2440006|N|N|N|N|N| +2440098|AAAAAAAACKLDFCAA|1968-08-29|823|3583|275|1968|4|8|29|3|1968|275|3583|Thursday|1968Q3|N|N|N|2440070|2440282|2439732|2440007|N|N|N|N|N| +2440099|AAAAAAAADKLDFCAA|1968-08-30|823|3583|275|1968|5|8|30|3|1968|275|3583|Friday|1968Q3|N|Y|N|2440070|2440282|2439733|2440008|N|N|N|N|N| +2440100|AAAAAAAAEKLDFCAA|1968-08-31|823|3583|275|1968|6|8|31|3|1968|275|3583|Saturday|1968Q3|N|Y|N|2440070|2440282|2439734|2440009|N|N|N|N|N| +2440101|AAAAAAAAFKLDFCAA|1968-09-01|824|3583|276|1968|0|9|1|3|1968|276|3583|Sunday|1968Q3|N|N|N|2440101|2440344|2439735|2440010|N|N|N|N|N| +2440102|AAAAAAAAGKLDFCAA|1968-09-02|824|3583|276|1968|1|9|2|3|1968|276|3583|Monday|1968Q3|N|N|N|2440101|2440344|2439736|2440011|N|N|N|N|N| +2440103|AAAAAAAAHKLDFCAA|1968-09-03|824|3584|276|1968|2|9|3|3|1968|276|3584|Tuesday|1968Q3|N|N|N|2440101|2440344|2439737|2440012|N|N|N|N|N| +2440104|AAAAAAAAIKLDFCAA|1968-09-04|824|3584|276|1968|3|9|4|3|1968|276|3584|Wednesday|1968Q3|N|N|N|2440101|2440344|2439738|2440013|N|N|N|N|N| +2440105|AAAAAAAAJKLDFCAA|1968-09-05|824|3584|276|1968|4|9|5|3|1968|276|3584|Thursday|1968Q3|N|N|N|2440101|2440344|2439739|2440014|N|N|N|N|N| +2440106|AAAAAAAAKKLDFCAA|1968-09-06|824|3584|276|1968|5|9|6|3|1968|276|3584|Friday|1968Q3|N|Y|N|2440101|2440344|2439740|2440015|N|N|N|N|N| +2440107|AAAAAAAALKLDFCAA|1968-09-07|824|3584|276|1968|6|9|7|3|1968|276|3584|Saturday|1968Q3|N|Y|N|2440101|2440344|2439741|2440016|N|N|N|N|N| +2440108|AAAAAAAAMKLDFCAA|1968-09-08|824|3584|276|1968|0|9|8|3|1968|276|3584|Sunday|1968Q3|N|N|N|2440101|2440344|2439742|2440017|N|N|N|N|N| +2440109|AAAAAAAANKLDFCAA|1968-09-09|824|3584|276|1968|1|9|9|3|1968|276|3584|Monday|1968Q3|N|N|N|2440101|2440344|2439743|2440018|N|N|N|N|N| +2440110|AAAAAAAAOKLDFCAA|1968-09-10|824|3585|276|1968|2|9|10|3|1968|276|3585|Tuesday|1968Q3|N|N|N|2440101|2440344|2439744|2440019|N|N|N|N|N| +2440111|AAAAAAAAPKLDFCAA|1968-09-11|824|3585|276|1968|3|9|11|3|1968|276|3585|Wednesday|1968Q3|N|N|N|2440101|2440344|2439745|2440020|N|N|N|N|N| +2440112|AAAAAAAAALLDFCAA|1968-09-12|824|3585|276|1968|4|9|12|3|1968|276|3585|Thursday|1968Q3|N|N|N|2440101|2440344|2439746|2440021|N|N|N|N|N| +2440113|AAAAAAAABLLDFCAA|1968-09-13|824|3585|276|1968|5|9|13|3|1968|276|3585|Friday|1968Q3|N|Y|N|2440101|2440344|2439747|2440022|N|N|N|N|N| +2440114|AAAAAAAACLLDFCAA|1968-09-14|824|3585|276|1968|6|9|14|3|1968|276|3585|Saturday|1968Q3|N|Y|N|2440101|2440344|2439748|2440023|N|N|N|N|N| +2440115|AAAAAAAADLLDFCAA|1968-09-15|824|3585|276|1968|0|9|15|3|1968|276|3585|Sunday|1968Q3|N|N|N|2440101|2440344|2439749|2440024|N|N|N|N|N| +2440116|AAAAAAAAELLDFCAA|1968-09-16|824|3585|276|1968|1|9|16|3|1968|276|3585|Monday|1968Q3|N|N|N|2440101|2440344|2439750|2440025|N|N|N|N|N| +2440117|AAAAAAAAFLLDFCAA|1968-09-17|824|3586|276|1968|2|9|17|3|1968|276|3586|Tuesday|1968Q3|N|N|N|2440101|2440344|2439751|2440026|N|N|N|N|N| +2440118|AAAAAAAAGLLDFCAA|1968-09-18|824|3586|276|1968|3|9|18|3|1968|276|3586|Wednesday|1968Q3|N|N|N|2440101|2440344|2439752|2440027|N|N|N|N|N| +2440119|AAAAAAAAHLLDFCAA|1968-09-19|824|3586|276|1968|4|9|19|3|1968|276|3586|Thursday|1968Q3|N|N|N|2440101|2440344|2439753|2440028|N|N|N|N|N| +2440120|AAAAAAAAILLDFCAA|1968-09-20|824|3586|276|1968|5|9|20|3|1968|276|3586|Friday|1968Q3|N|Y|N|2440101|2440344|2439754|2440029|N|N|N|N|N| +2440121|AAAAAAAAJLLDFCAA|1968-09-21|824|3586|276|1968|6|9|21|3|1968|276|3586|Saturday|1968Q3|N|Y|N|2440101|2440344|2439755|2440030|N|N|N|N|N| +2440122|AAAAAAAAKLLDFCAA|1968-09-22|824|3586|276|1968|0|9|22|3|1968|276|3586|Sunday|1968Q3|N|N|N|2440101|2440344|2439756|2440031|N|N|N|N|N| +2440123|AAAAAAAALLLDFCAA|1968-09-23|824|3586|276|1968|1|9|23|3|1968|276|3586|Monday|1968Q3|N|N|N|2440101|2440344|2439757|2440032|N|N|N|N|N| +2440124|AAAAAAAAMLLDFCAA|1968-09-24|824|3587|276|1968|2|9|24|3|1968|276|3587|Tuesday|1968Q3|N|N|N|2440101|2440344|2439758|2440033|N|N|N|N|N| +2440125|AAAAAAAANLLDFCAA|1968-09-25|824|3587|276|1968|3|9|25|3|1968|276|3587|Wednesday|1968Q3|N|N|N|2440101|2440344|2439759|2440034|N|N|N|N|N| +2440126|AAAAAAAAOLLDFCAA|1968-09-26|824|3587|276|1968|4|9|26|3|1968|276|3587|Thursday|1968Q3|N|N|N|2440101|2440344|2439760|2440035|N|N|N|N|N| +2440127|AAAAAAAAPLLDFCAA|1968-09-27|824|3587|276|1968|5|9|27|3|1968|276|3587|Friday|1968Q3|N|Y|N|2440101|2440344|2439761|2440036|N|N|N|N|N| +2440128|AAAAAAAAAMLDFCAA|1968-09-28|824|3587|276|1968|6|9|28|3|1968|276|3587|Saturday|1968Q3|N|Y|N|2440101|2440344|2439762|2440037|N|N|N|N|N| +2440129|AAAAAAAABMLDFCAA|1968-09-29|824|3587|276|1968|0|9|29|3|1968|276|3587|Sunday|1968Q3|N|N|N|2440101|2440344|2439763|2440038|N|N|N|N|N| +2440130|AAAAAAAACMLDFCAA|1968-09-30|824|3587|276|1968|1|9|30|3|1968|276|3587|Monday|1968Q3|N|N|N|2440101|2440344|2439764|2440039|N|N|N|N|N| +2440131|AAAAAAAADMLDFCAA|1968-10-01|825|3588|276|1968|2|10|1|4|1968|276|3588|Tuesday|1968Q4|N|N|N|2440131|2440404|2439765|2440039|N|N|N|N|N| +2440132|AAAAAAAAEMLDFCAA|1968-10-02|825|3588|276|1968|3|10|2|4|1968|276|3588|Wednesday|1968Q4|N|N|N|2440131|2440404|2439766|2440040|N|N|N|N|N| +2440133|AAAAAAAAFMLDFCAA|1968-10-03|825|3588|276|1968|4|10|3|4|1968|276|3588|Thursday|1968Q4|N|N|N|2440131|2440404|2439767|2440041|N|N|N|N|N| +2440134|AAAAAAAAGMLDFCAA|1968-10-04|825|3588|276|1968|5|10|4|4|1968|276|3588|Friday|1968Q4|N|Y|N|2440131|2440404|2439768|2440042|N|N|N|N|N| +2440135|AAAAAAAAHMLDFCAA|1968-10-05|825|3588|276|1968|6|10|5|4|1968|276|3588|Saturday|1968Q4|N|Y|N|2440131|2440404|2439769|2440043|N|N|N|N|N| +2440136|AAAAAAAAIMLDFCAA|1968-10-06|825|3588|276|1968|0|10|6|4|1968|276|3588|Sunday|1968Q4|N|N|N|2440131|2440404|2439770|2440044|N|N|N|N|N| +2440137|AAAAAAAAJMLDFCAA|1968-10-07|825|3588|276|1968|1|10|7|4|1968|276|3588|Monday|1968Q4|N|N|N|2440131|2440404|2439771|2440045|N|N|N|N|N| +2440138|AAAAAAAAKMLDFCAA|1968-10-08|825|3589|276|1968|2|10|8|4|1968|276|3589|Tuesday|1968Q4|N|N|N|2440131|2440404|2439772|2440046|N|N|N|N|N| +2440139|AAAAAAAALMLDFCAA|1968-10-09|825|3589|276|1968|3|10|9|4|1968|276|3589|Wednesday|1968Q4|N|N|N|2440131|2440404|2439773|2440047|N|N|N|N|N| +2440140|AAAAAAAAMMLDFCAA|1968-10-10|825|3589|276|1968|4|10|10|4|1968|276|3589|Thursday|1968Q4|N|N|N|2440131|2440404|2439774|2440048|N|N|N|N|N| +2440141|AAAAAAAANMLDFCAA|1968-10-11|825|3589|276|1968|5|10|11|4|1968|276|3589|Friday|1968Q4|N|Y|N|2440131|2440404|2439775|2440049|N|N|N|N|N| +2440142|AAAAAAAAOMLDFCAA|1968-10-12|825|3589|276|1968|6|10|12|4|1968|276|3589|Saturday|1968Q4|N|Y|N|2440131|2440404|2439776|2440050|N|N|N|N|N| +2440143|AAAAAAAAPMLDFCAA|1968-10-13|825|3589|276|1968|0|10|13|4|1968|276|3589|Sunday|1968Q4|N|N|N|2440131|2440404|2439777|2440051|N|N|N|N|N| +2440144|AAAAAAAAANLDFCAA|1968-10-14|825|3589|276|1968|1|10|14|4|1968|276|3589|Monday|1968Q4|N|N|N|2440131|2440404|2439778|2440052|N|N|N|N|N| +2440145|AAAAAAAABNLDFCAA|1968-10-15|825|3590|276|1968|2|10|15|4|1968|276|3590|Tuesday|1968Q4|N|N|N|2440131|2440404|2439779|2440053|N|N|N|N|N| +2440146|AAAAAAAACNLDFCAA|1968-10-16|825|3590|276|1968|3|10|16|4|1968|276|3590|Wednesday|1968Q4|N|N|N|2440131|2440404|2439780|2440054|N|N|N|N|N| +2440147|AAAAAAAADNLDFCAA|1968-10-17|825|3590|276|1968|4|10|17|4|1968|276|3590|Thursday|1968Q4|N|N|N|2440131|2440404|2439781|2440055|N|N|N|N|N| +2440148|AAAAAAAAENLDFCAA|1968-10-18|825|3590|276|1968|5|10|18|4|1968|276|3590|Friday|1968Q4|N|Y|N|2440131|2440404|2439782|2440056|N|N|N|N|N| +2440149|AAAAAAAAFNLDFCAA|1968-10-19|825|3590|276|1968|6|10|19|4|1968|276|3590|Saturday|1968Q4|N|Y|N|2440131|2440404|2439783|2440057|N|N|N|N|N| +2440150|AAAAAAAAGNLDFCAA|1968-10-20|825|3590|276|1968|0|10|20|4|1968|276|3590|Sunday|1968Q4|N|N|N|2440131|2440404|2439784|2440058|N|N|N|N|N| +2440151|AAAAAAAAHNLDFCAA|1968-10-21|825|3590|276|1968|1|10|21|4|1968|276|3590|Monday|1968Q4|N|N|N|2440131|2440404|2439785|2440059|N|N|N|N|N| +2440152|AAAAAAAAINLDFCAA|1968-10-22|825|3591|276|1968|2|10|22|4|1968|276|3591|Tuesday|1968Q4|N|N|N|2440131|2440404|2439786|2440060|N|N|N|N|N| +2440153|AAAAAAAAJNLDFCAA|1968-10-23|825|3591|276|1968|3|10|23|4|1968|276|3591|Wednesday|1968Q4|N|N|N|2440131|2440404|2439787|2440061|N|N|N|N|N| +2440154|AAAAAAAAKNLDFCAA|1968-10-24|825|3591|276|1968|4|10|24|4|1968|276|3591|Thursday|1968Q4|N|N|N|2440131|2440404|2439788|2440062|N|N|N|N|N| +2440155|AAAAAAAALNLDFCAA|1968-10-25|825|3591|276|1968|5|10|25|4|1968|276|3591|Friday|1968Q4|N|Y|N|2440131|2440404|2439789|2440063|N|N|N|N|N| +2440156|AAAAAAAAMNLDFCAA|1968-10-26|825|3591|276|1968|6|10|26|4|1968|276|3591|Saturday|1968Q4|N|Y|N|2440131|2440404|2439790|2440064|N|N|N|N|N| +2440157|AAAAAAAANNLDFCAA|1968-10-27|825|3591|276|1968|0|10|27|4|1968|276|3591|Sunday|1968Q4|N|N|N|2440131|2440404|2439791|2440065|N|N|N|N|N| +2440158|AAAAAAAAONLDFCAA|1968-10-28|825|3591|276|1968|1|10|28|4|1968|276|3591|Monday|1968Q4|N|N|N|2440131|2440404|2439792|2440066|N|N|N|N|N| +2440159|AAAAAAAAPNLDFCAA|1968-10-29|825|3592|276|1968|2|10|29|4|1968|276|3592|Tuesday|1968Q4|N|N|N|2440131|2440404|2439793|2440067|N|N|N|N|N| +2440160|AAAAAAAAAOLDFCAA|1968-10-30|825|3592|276|1968|3|10|30|4|1968|276|3592|Wednesday|1968Q4|N|N|N|2440131|2440404|2439794|2440068|N|N|N|N|N| +2440161|AAAAAAAABOLDFCAA|1968-10-31|825|3592|276|1968|4|10|31|4|1968|276|3592|Thursday|1968Q4|N|N|N|2440131|2440404|2439795|2440069|N|N|N|N|N| +2440162|AAAAAAAACOLDFCAA|1968-11-01|826|3592|276|1968|5|11|1|4|1968|276|3592|Friday|1968Q4|N|Y|N|2440162|2440466|2439796|2440070|N|N|N|N|N| +2440163|AAAAAAAADOLDFCAA|1968-11-02|826|3592|276|1968|6|11|2|4|1968|276|3592|Saturday|1968Q4|N|Y|N|2440162|2440466|2439797|2440071|N|N|N|N|N| +2440164|AAAAAAAAEOLDFCAA|1968-11-03|826|3592|276|1968|0|11|3|4|1968|276|3592|Sunday|1968Q4|N|N|N|2440162|2440466|2439798|2440072|N|N|N|N|N| +2440165|AAAAAAAAFOLDFCAA|1968-11-04|826|3592|276|1968|1|11|4|4|1968|276|3592|Monday|1968Q4|N|N|N|2440162|2440466|2439799|2440073|N|N|N|N|N| +2440166|AAAAAAAAGOLDFCAA|1968-11-05|826|3593|276|1968|2|11|5|4|1968|276|3593|Tuesday|1968Q4|N|N|N|2440162|2440466|2439800|2440074|N|N|N|N|N| +2440167|AAAAAAAAHOLDFCAA|1968-11-06|826|3593|276|1968|3|11|6|4|1968|276|3593|Wednesday|1968Q4|N|N|N|2440162|2440466|2439801|2440075|N|N|N|N|N| +2440168|AAAAAAAAIOLDFCAA|1968-11-07|826|3593|276|1968|4|11|7|4|1968|276|3593|Thursday|1968Q4|N|N|N|2440162|2440466|2439802|2440076|N|N|N|N|N| +2440169|AAAAAAAAJOLDFCAA|1968-11-08|826|3593|276|1968|5|11|8|4|1968|276|3593|Friday|1968Q4|N|Y|N|2440162|2440466|2439803|2440077|N|N|N|N|N| +2440170|AAAAAAAAKOLDFCAA|1968-11-09|826|3593|276|1968|6|11|9|4|1968|276|3593|Saturday|1968Q4|N|Y|N|2440162|2440466|2439804|2440078|N|N|N|N|N| +2440171|AAAAAAAALOLDFCAA|1968-11-10|826|3593|276|1968|0|11|10|4|1968|276|3593|Sunday|1968Q4|N|N|N|2440162|2440466|2439805|2440079|N|N|N|N|N| +2440172|AAAAAAAAMOLDFCAA|1968-11-11|826|3593|276|1968|1|11|11|4|1968|276|3593|Monday|1968Q4|N|N|N|2440162|2440466|2439806|2440080|N|N|N|N|N| +2440173|AAAAAAAANOLDFCAA|1968-11-12|826|3594|276|1968|2|11|12|4|1968|276|3594|Tuesday|1968Q4|N|N|N|2440162|2440466|2439807|2440081|N|N|N|N|N| +2440174|AAAAAAAAOOLDFCAA|1968-11-13|826|3594|276|1968|3|11|13|4|1968|276|3594|Wednesday|1968Q4|N|N|N|2440162|2440466|2439808|2440082|N|N|N|N|N| +2440175|AAAAAAAAPOLDFCAA|1968-11-14|826|3594|276|1968|4|11|14|4|1968|276|3594|Thursday|1968Q4|N|N|N|2440162|2440466|2439809|2440083|N|N|N|N|N| +2440176|AAAAAAAAAPLDFCAA|1968-11-15|826|3594|276|1968|5|11|15|4|1968|276|3594|Friday|1968Q4|N|Y|N|2440162|2440466|2439810|2440084|N|N|N|N|N| +2440177|AAAAAAAABPLDFCAA|1968-11-16|826|3594|276|1968|6|11|16|4|1968|276|3594|Saturday|1968Q4|N|Y|N|2440162|2440466|2439811|2440085|N|N|N|N|N| +2440178|AAAAAAAACPLDFCAA|1968-11-17|826|3594|276|1968|0|11|17|4|1968|276|3594|Sunday|1968Q4|N|N|N|2440162|2440466|2439812|2440086|N|N|N|N|N| +2440179|AAAAAAAADPLDFCAA|1968-11-18|826|3594|276|1968|1|11|18|4|1968|276|3594|Monday|1968Q4|N|N|N|2440162|2440466|2439813|2440087|N|N|N|N|N| +2440180|AAAAAAAAEPLDFCAA|1968-11-19|826|3595|276|1968|2|11|19|4|1968|276|3595|Tuesday|1968Q4|N|N|N|2440162|2440466|2439814|2440088|N|N|N|N|N| +2440181|AAAAAAAAFPLDFCAA|1968-11-20|826|3595|276|1968|3|11|20|4|1968|276|3595|Wednesday|1968Q4|N|N|N|2440162|2440466|2439815|2440089|N|N|N|N|N| +2440182|AAAAAAAAGPLDFCAA|1968-11-21|826|3595|276|1968|4|11|21|4|1968|276|3595|Thursday|1968Q4|N|N|N|2440162|2440466|2439816|2440090|N|N|N|N|N| +2440183|AAAAAAAAHPLDFCAA|1968-11-22|826|3595|276|1968|5|11|22|4|1968|276|3595|Friday|1968Q4|N|Y|N|2440162|2440466|2439817|2440091|N|N|N|N|N| +2440184|AAAAAAAAIPLDFCAA|1968-11-23|826|3595|276|1968|6|11|23|4|1968|276|3595|Saturday|1968Q4|N|Y|N|2440162|2440466|2439818|2440092|N|N|N|N|N| +2440185|AAAAAAAAJPLDFCAA|1968-11-24|826|3595|276|1968|0|11|24|4|1968|276|3595|Sunday|1968Q4|N|N|N|2440162|2440466|2439819|2440093|N|N|N|N|N| +2440186|AAAAAAAAKPLDFCAA|1968-11-25|826|3595|276|1968|1|11|25|4|1968|276|3595|Monday|1968Q4|N|N|N|2440162|2440466|2439820|2440094|N|N|N|N|N| +2440187|AAAAAAAALPLDFCAA|1968-11-26|826|3596|276|1968|2|11|26|4|1968|276|3596|Tuesday|1968Q4|N|N|N|2440162|2440466|2439821|2440095|N|N|N|N|N| +2440188|AAAAAAAAMPLDFCAA|1968-11-27|826|3596|276|1968|3|11|27|4|1968|276|3596|Wednesday|1968Q4|N|N|N|2440162|2440466|2439822|2440096|N|N|N|N|N| +2440189|AAAAAAAANPLDFCAA|1968-11-28|826|3596|276|1968|4|11|28|4|1968|276|3596|Thursday|1968Q4|N|N|N|2440162|2440466|2439823|2440097|N|N|N|N|N| +2440190|AAAAAAAAOPLDFCAA|1968-11-29|826|3596|276|1968|5|11|29|4|1968|276|3596|Friday|1968Q4|N|Y|N|2440162|2440466|2439824|2440098|N|N|N|N|N| +2440191|AAAAAAAAPPLDFCAA|1968-11-30|826|3596|276|1968|6|11|30|4|1968|276|3596|Saturday|1968Q4|N|Y|N|2440162|2440466|2439825|2440099|N|N|N|N|N| +2440192|AAAAAAAAAAMDFCAA|1968-12-01|827|3596|277|1968|0|12|1|4|1968|277|3596|Sunday|1968Q4|N|N|N|2440192|2440526|2439826|2440100|N|N|N|N|N| +2440193|AAAAAAAABAMDFCAA|1968-12-02|827|3596|277|1968|1|12|2|4|1968|277|3596|Monday|1968Q4|N|N|N|2440192|2440526|2439827|2440101|N|N|N|N|N| +2440194|AAAAAAAACAMDFCAA|1968-12-03|827|3597|277|1968|2|12|3|4|1968|277|3597|Tuesday|1968Q4|N|N|N|2440192|2440526|2439828|2440102|N|N|N|N|N| +2440195|AAAAAAAADAMDFCAA|1968-12-04|827|3597|277|1968|3|12|4|4|1968|277|3597|Wednesday|1968Q4|N|N|N|2440192|2440526|2439829|2440103|N|N|N|N|N| +2440196|AAAAAAAAEAMDFCAA|1968-12-05|827|3597|277|1968|4|12|5|4|1968|277|3597|Thursday|1968Q4|N|N|N|2440192|2440526|2439830|2440104|N|N|N|N|N| +2440197|AAAAAAAAFAMDFCAA|1968-12-06|827|3597|277|1968|5|12|6|4|1968|277|3597|Friday|1968Q4|N|Y|N|2440192|2440526|2439831|2440105|N|N|N|N|N| +2440198|AAAAAAAAGAMDFCAA|1968-12-07|827|3597|277|1968|6|12|7|4|1968|277|3597|Saturday|1968Q4|N|Y|N|2440192|2440526|2439832|2440106|N|N|N|N|N| +2440199|AAAAAAAAHAMDFCAA|1968-12-08|827|3597|277|1968|0|12|8|4|1968|277|3597|Sunday|1968Q4|N|N|N|2440192|2440526|2439833|2440107|N|N|N|N|N| +2440200|AAAAAAAAIAMDFCAA|1968-12-09|827|3597|277|1968|1|12|9|4|1968|277|3597|Monday|1968Q4|N|N|N|2440192|2440526|2439834|2440108|N|N|N|N|N| +2440201|AAAAAAAAJAMDFCAA|1968-12-10|827|3598|277|1968|2|12|10|4|1968|277|3598|Tuesday|1968Q4|N|N|N|2440192|2440526|2439835|2440109|N|N|N|N|N| +2440202|AAAAAAAAKAMDFCAA|1968-12-11|827|3598|277|1968|3|12|11|4|1968|277|3598|Wednesday|1968Q4|N|N|N|2440192|2440526|2439836|2440110|N|N|N|N|N| +2440203|AAAAAAAALAMDFCAA|1968-12-12|827|3598|277|1968|4|12|12|4|1968|277|3598|Thursday|1968Q4|N|N|N|2440192|2440526|2439837|2440111|N|N|N|N|N| +2440204|AAAAAAAAMAMDFCAA|1968-12-13|827|3598|277|1968|5|12|13|4|1968|277|3598|Friday|1968Q4|N|Y|N|2440192|2440526|2439838|2440112|N|N|N|N|N| +2440205|AAAAAAAANAMDFCAA|1968-12-14|827|3598|277|1968|6|12|14|4|1968|277|3598|Saturday|1968Q4|N|Y|N|2440192|2440526|2439839|2440113|N|N|N|N|N| +2440206|AAAAAAAAOAMDFCAA|1968-12-15|827|3598|277|1968|0|12|15|4|1968|277|3598|Sunday|1968Q4|N|N|N|2440192|2440526|2439840|2440114|N|N|N|N|N| +2440207|AAAAAAAAPAMDFCAA|1968-12-16|827|3598|277|1968|1|12|16|4|1968|277|3598|Monday|1968Q4|N|N|N|2440192|2440526|2439841|2440115|N|N|N|N|N| +2440208|AAAAAAAAABMDFCAA|1968-12-17|827|3599|277|1968|2|12|17|4|1968|277|3599|Tuesday|1968Q4|N|N|N|2440192|2440526|2439842|2440116|N|N|N|N|N| +2440209|AAAAAAAABBMDFCAA|1968-12-18|827|3599|277|1968|3|12|18|4|1968|277|3599|Wednesday|1968Q4|N|N|N|2440192|2440526|2439843|2440117|N|N|N|N|N| +2440210|AAAAAAAACBMDFCAA|1968-12-19|827|3599|277|1968|4|12|19|4|1968|277|3599|Thursday|1968Q4|N|N|N|2440192|2440526|2439844|2440118|N|N|N|N|N| +2440211|AAAAAAAADBMDFCAA|1968-12-20|827|3599|277|1968|5|12|20|4|1968|277|3599|Friday|1968Q4|N|Y|N|2440192|2440526|2439845|2440119|N|N|N|N|N| +2440212|AAAAAAAAEBMDFCAA|1968-12-21|827|3599|277|1968|6|12|21|4|1968|277|3599|Saturday|1968Q4|N|Y|N|2440192|2440526|2439846|2440120|N|N|N|N|N| +2440213|AAAAAAAAFBMDFCAA|1968-12-22|827|3599|277|1968|0|12|22|4|1968|277|3599|Sunday|1968Q4|N|N|N|2440192|2440526|2439847|2440121|N|N|N|N|N| +2440214|AAAAAAAAGBMDFCAA|1968-12-23|827|3599|277|1968|1|12|23|4|1968|277|3599|Monday|1968Q4|N|N|N|2440192|2440526|2439848|2440122|N|N|N|N|N| +2440215|AAAAAAAAHBMDFCAA|1968-12-24|827|3600|277|1968|2|12|24|4|1968|277|3600|Tuesday|1968Q4|N|N|N|2440192|2440526|2439849|2440123|N|N|N|N|N| +2440216|AAAAAAAAIBMDFCAA|1968-12-25|827|3600|277|1968|3|12|25|4|1968|277|3600|Wednesday|1968Q4|Y|N|N|2440192|2440526|2439850|2440124|N|N|N|N|N| +2440217|AAAAAAAAJBMDFCAA|1968-12-26|827|3600|277|1968|4|12|26|4|1968|277|3600|Thursday|1968Q4|N|N|Y|2440192|2440526|2439851|2440125|N|N|N|N|N| +2440218|AAAAAAAAKBMDFCAA|1968-12-27|827|3600|277|1968|5|12|27|4|1968|277|3600|Friday|1968Q4|N|Y|N|2440192|2440526|2439852|2440126|N|N|N|N|N| +2440219|AAAAAAAALBMDFCAA|1968-12-28|827|3600|277|1968|6|12|28|4|1968|277|3600|Saturday|1968Q4|N|Y|N|2440192|2440526|2439853|2440127|N|N|N|N|N| +2440220|AAAAAAAAMBMDFCAA|1968-12-29|827|3600|277|1968|0|12|29|4|1968|277|3600|Sunday|1968Q4|N|N|N|2440192|2440526|2439854|2440128|N|N|N|N|N| +2440221|AAAAAAAANBMDFCAA|1968-12-30|827|3600|277|1968|1|12|30|4|1968|277|3600|Monday|1968Q4|N|N|N|2440192|2440526|2439855|2440129|N|N|N|N|N| +2440222|AAAAAAAAOBMDFCAA|1968-12-31|827|3601|277|1968|2|12|31|4|1968|277|3601|Tuesday|1968Q4|Y|N|N|2440192|2440526|2439856|2440130|N|N|N|N|N| +2440223|AAAAAAAAPBMDFCAA|1969-01-01|828|3601|277|1969|3|1|1|1|1969|277|3601|Wednesday|1969Q1|Y|N|Y|2440223|2440222|2439857|2440131|N|N|N|N|N| +2440224|AAAAAAAAACMDFCAA|1969-01-02|828|3601|277|1969|4|1|2|1|1969|277|3601|Thursday|1969Q1|N|N|Y|2440223|2440222|2439858|2440132|N|N|N|N|N| +2440225|AAAAAAAABCMDFCAA|1969-01-03|828|3601|277|1969|5|1|3|1|1969|277|3601|Friday|1969Q1|N|Y|N|2440223|2440222|2439859|2440133|N|N|N|N|N| +2440226|AAAAAAAACCMDFCAA|1969-01-04|828|3601|277|1969|6|1|4|1|1969|277|3601|Saturday|1969Q1|N|Y|N|2440223|2440222|2439860|2440134|N|N|N|N|N| +2440227|AAAAAAAADCMDFCAA|1969-01-05|828|3601|277|1969|0|1|5|1|1969|277|3601|Sunday|1969Q1|N|N|N|2440223|2440222|2439861|2440135|N|N|N|N|N| +2440228|AAAAAAAAECMDFCAA|1969-01-06|828|3601|277|1969|1|1|6|1|1969|277|3601|Monday|1969Q1|N|N|N|2440223|2440222|2439862|2440136|N|N|N|N|N| +2440229|AAAAAAAAFCMDFCAA|1969-01-07|828|3602|277|1969|2|1|7|1|1969|277|3602|Tuesday|1969Q1|N|N|N|2440223|2440222|2439863|2440137|N|N|N|N|N| +2440230|AAAAAAAAGCMDFCAA|1969-01-08|828|3602|277|1969|3|1|8|1|1969|277|3602|Wednesday|1969Q1|N|N|N|2440223|2440222|2439864|2440138|N|N|N|N|N| +2440231|AAAAAAAAHCMDFCAA|1969-01-09|828|3602|277|1969|4|1|9|1|1969|277|3602|Thursday|1969Q1|N|N|N|2440223|2440222|2439865|2440139|N|N|N|N|N| +2440232|AAAAAAAAICMDFCAA|1969-01-10|828|3602|277|1969|5|1|10|1|1969|277|3602|Friday|1969Q1|N|Y|N|2440223|2440222|2439866|2440140|N|N|N|N|N| +2440233|AAAAAAAAJCMDFCAA|1969-01-11|828|3602|277|1969|6|1|11|1|1969|277|3602|Saturday|1969Q1|N|Y|N|2440223|2440222|2439867|2440141|N|N|N|N|N| +2440234|AAAAAAAAKCMDFCAA|1969-01-12|828|3602|277|1969|0|1|12|1|1969|277|3602|Sunday|1969Q1|N|N|N|2440223|2440222|2439868|2440142|N|N|N|N|N| +2440235|AAAAAAAALCMDFCAA|1969-01-13|828|3602|277|1969|1|1|13|1|1969|277|3602|Monday|1969Q1|N|N|N|2440223|2440222|2439869|2440143|N|N|N|N|N| +2440236|AAAAAAAAMCMDFCAA|1969-01-14|828|3603|277|1969|2|1|14|1|1969|277|3603|Tuesday|1969Q1|N|N|N|2440223|2440222|2439870|2440144|N|N|N|N|N| +2440237|AAAAAAAANCMDFCAA|1969-01-15|828|3603|277|1969|3|1|15|1|1969|277|3603|Wednesday|1969Q1|N|N|N|2440223|2440222|2439871|2440145|N|N|N|N|N| +2440238|AAAAAAAAOCMDFCAA|1969-01-16|828|3603|277|1969|4|1|16|1|1969|277|3603|Thursday|1969Q1|N|N|N|2440223|2440222|2439872|2440146|N|N|N|N|N| +2440239|AAAAAAAAPCMDFCAA|1969-01-17|828|3603|277|1969|5|1|17|1|1969|277|3603|Friday|1969Q1|N|Y|N|2440223|2440222|2439873|2440147|N|N|N|N|N| +2440240|AAAAAAAAADMDFCAA|1969-01-18|828|3603|277|1969|6|1|18|1|1969|277|3603|Saturday|1969Q1|N|Y|N|2440223|2440222|2439874|2440148|N|N|N|N|N| +2440241|AAAAAAAABDMDFCAA|1969-01-19|828|3603|277|1969|0|1|19|1|1969|277|3603|Sunday|1969Q1|N|N|N|2440223|2440222|2439875|2440149|N|N|N|N|N| +2440242|AAAAAAAACDMDFCAA|1969-01-20|828|3603|277|1969|1|1|20|1|1969|277|3603|Monday|1969Q1|N|N|N|2440223|2440222|2439876|2440150|N|N|N|N|N| +2440243|AAAAAAAADDMDFCAA|1969-01-21|828|3604|277|1969|2|1|21|1|1969|277|3604|Tuesday|1969Q1|N|N|N|2440223|2440222|2439877|2440151|N|N|N|N|N| +2440244|AAAAAAAAEDMDFCAA|1969-01-22|828|3604|277|1969|3|1|22|1|1969|277|3604|Wednesday|1969Q1|N|N|N|2440223|2440222|2439878|2440152|N|N|N|N|N| +2440245|AAAAAAAAFDMDFCAA|1969-01-23|828|3604|277|1969|4|1|23|1|1969|277|3604|Thursday|1969Q1|N|N|N|2440223|2440222|2439879|2440153|N|N|N|N|N| +2440246|AAAAAAAAGDMDFCAA|1969-01-24|828|3604|277|1969|5|1|24|1|1969|277|3604|Friday|1969Q1|N|Y|N|2440223|2440222|2439880|2440154|N|N|N|N|N| +2440247|AAAAAAAAHDMDFCAA|1969-01-25|828|3604|277|1969|6|1|25|1|1969|277|3604|Saturday|1969Q1|N|Y|N|2440223|2440222|2439881|2440155|N|N|N|N|N| +2440248|AAAAAAAAIDMDFCAA|1969-01-26|828|3604|277|1969|0|1|26|1|1969|277|3604|Sunday|1969Q1|N|N|N|2440223|2440222|2439882|2440156|N|N|N|N|N| +2440249|AAAAAAAAJDMDFCAA|1969-01-27|828|3604|277|1969|1|1|27|1|1969|277|3604|Monday|1969Q1|N|N|N|2440223|2440222|2439883|2440157|N|N|N|N|N| +2440250|AAAAAAAAKDMDFCAA|1969-01-28|828|3605|277|1969|2|1|28|1|1969|277|3605|Tuesday|1969Q1|N|N|N|2440223|2440222|2439884|2440158|N|N|N|N|N| +2440251|AAAAAAAALDMDFCAA|1969-01-29|828|3605|277|1969|3|1|29|1|1969|277|3605|Wednesday|1969Q1|N|N|N|2440223|2440222|2439885|2440159|N|N|N|N|N| +2440252|AAAAAAAAMDMDFCAA|1969-01-30|828|3605|277|1969|4|1|30|1|1969|277|3605|Thursday|1969Q1|N|N|N|2440223|2440222|2439886|2440160|N|N|N|N|N| +2440253|AAAAAAAANDMDFCAA|1969-01-31|828|3605|277|1969|5|1|31|1|1969|277|3605|Friday|1969Q1|N|Y|N|2440223|2440222|2439887|2440161|N|N|N|N|N| +2440254|AAAAAAAAODMDFCAA|1969-02-01|829|3605|277|1969|6|2|1|1|1969|277|3605|Saturday|1969Q1|N|Y|N|2440254|2440284|2439888|2440162|N|N|N|N|N| +2440255|AAAAAAAAPDMDFCAA|1969-02-02|829|3605|277|1969|0|2|2|1|1969|277|3605|Sunday|1969Q1|N|N|N|2440254|2440284|2439889|2440163|N|N|N|N|N| +2440256|AAAAAAAAAEMDFCAA|1969-02-03|829|3605|277|1969|1|2|3|1|1969|277|3605|Monday|1969Q1|N|N|N|2440254|2440284|2439890|2440164|N|N|N|N|N| +2440257|AAAAAAAABEMDFCAA|1969-02-04|829|3606|277|1969|2|2|4|1|1969|277|3606|Tuesday|1969Q1|N|N|N|2440254|2440284|2439891|2440165|N|N|N|N|N| +2440258|AAAAAAAACEMDFCAA|1969-02-05|829|3606|277|1969|3|2|5|1|1969|277|3606|Wednesday|1969Q1|N|N|N|2440254|2440284|2439892|2440166|N|N|N|N|N| +2440259|AAAAAAAADEMDFCAA|1969-02-06|829|3606|277|1969|4|2|6|1|1969|277|3606|Thursday|1969Q1|N|N|N|2440254|2440284|2439893|2440167|N|N|N|N|N| +2440260|AAAAAAAAEEMDFCAA|1969-02-07|829|3606|277|1969|5|2|7|1|1969|277|3606|Friday|1969Q1|N|Y|N|2440254|2440284|2439894|2440168|N|N|N|N|N| +2440261|AAAAAAAAFEMDFCAA|1969-02-08|829|3606|277|1969|6|2|8|1|1969|277|3606|Saturday|1969Q1|N|Y|N|2440254|2440284|2439895|2440169|N|N|N|N|N| +2440262|AAAAAAAAGEMDFCAA|1969-02-09|829|3606|277|1969|0|2|9|1|1969|277|3606|Sunday|1969Q1|N|N|N|2440254|2440284|2439896|2440170|N|N|N|N|N| +2440263|AAAAAAAAHEMDFCAA|1969-02-10|829|3606|277|1969|1|2|10|1|1969|277|3606|Monday|1969Q1|N|N|N|2440254|2440284|2439897|2440171|N|N|N|N|N| +2440264|AAAAAAAAIEMDFCAA|1969-02-11|829|3607|277|1969|2|2|11|1|1969|277|3607|Tuesday|1969Q1|N|N|N|2440254|2440284|2439898|2440172|N|N|N|N|N| +2440265|AAAAAAAAJEMDFCAA|1969-02-12|829|3607|277|1969|3|2|12|1|1969|277|3607|Wednesday|1969Q1|N|N|N|2440254|2440284|2439899|2440173|N|N|N|N|N| +2440266|AAAAAAAAKEMDFCAA|1969-02-13|829|3607|277|1969|4|2|13|1|1969|277|3607|Thursday|1969Q1|N|N|N|2440254|2440284|2439900|2440174|N|N|N|N|N| +2440267|AAAAAAAALEMDFCAA|1969-02-14|829|3607|277|1969|5|2|14|1|1969|277|3607|Friday|1969Q1|N|Y|N|2440254|2440284|2439901|2440175|N|N|N|N|N| +2440268|AAAAAAAAMEMDFCAA|1969-02-15|829|3607|277|1969|6|2|15|1|1969|277|3607|Saturday|1969Q1|N|Y|N|2440254|2440284|2439902|2440176|N|N|N|N|N| +2440269|AAAAAAAANEMDFCAA|1969-02-16|829|3607|277|1969|0|2|16|1|1969|277|3607|Sunday|1969Q1|N|N|N|2440254|2440284|2439903|2440177|N|N|N|N|N| +2440270|AAAAAAAAOEMDFCAA|1969-02-17|829|3607|277|1969|1|2|17|1|1969|277|3607|Monday|1969Q1|N|N|N|2440254|2440284|2439904|2440178|N|N|N|N|N| +2440271|AAAAAAAAPEMDFCAA|1969-02-18|829|3608|277|1969|2|2|18|1|1969|277|3608|Tuesday|1969Q1|N|N|N|2440254|2440284|2439905|2440179|N|N|N|N|N| +2440272|AAAAAAAAAFMDFCAA|1969-02-19|829|3608|277|1969|3|2|19|1|1969|277|3608|Wednesday|1969Q1|N|N|N|2440254|2440284|2439906|2440180|N|N|N|N|N| +2440273|AAAAAAAABFMDFCAA|1969-02-20|829|3608|277|1969|4|2|20|1|1969|277|3608|Thursday|1969Q1|N|N|N|2440254|2440284|2439907|2440181|N|N|N|N|N| +2440274|AAAAAAAACFMDFCAA|1969-02-21|829|3608|277|1969|5|2|21|1|1969|277|3608|Friday|1969Q1|N|Y|N|2440254|2440284|2439908|2440182|N|N|N|N|N| +2440275|AAAAAAAADFMDFCAA|1969-02-22|829|3608|277|1969|6|2|22|1|1969|277|3608|Saturday|1969Q1|N|Y|N|2440254|2440284|2439909|2440183|N|N|N|N|N| +2440276|AAAAAAAAEFMDFCAA|1969-02-23|829|3608|277|1969|0|2|23|1|1969|277|3608|Sunday|1969Q1|N|N|N|2440254|2440284|2439910|2440184|N|N|N|N|N| +2440277|AAAAAAAAFFMDFCAA|1969-02-24|829|3608|277|1969|1|2|24|1|1969|277|3608|Monday|1969Q1|N|N|N|2440254|2440284|2439911|2440185|N|N|N|N|N| +2440278|AAAAAAAAGFMDFCAA|1969-02-25|829|3609|277|1969|2|2|25|1|1969|277|3609|Tuesday|1969Q1|N|N|N|2440254|2440284|2439912|2440186|N|N|N|N|N| +2440279|AAAAAAAAHFMDFCAA|1969-02-26|829|3609|277|1969|3|2|26|1|1969|277|3609|Wednesday|1969Q1|N|N|N|2440254|2440284|2439913|2440187|N|N|N|N|N| +2440280|AAAAAAAAIFMDFCAA|1969-02-27|829|3609|277|1969|4|2|27|1|1969|277|3609|Thursday|1969Q1|N|N|N|2440254|2440284|2439914|2440188|N|N|N|N|N| +2440281|AAAAAAAAJFMDFCAA|1969-02-28|829|3609|277|1969|5|2|28|1|1969|277|3609|Friday|1969Q1|N|Y|N|2440254|2440284|2439915|2440189|N|N|N|N|N| +2440282|AAAAAAAAKFMDFCAA|1969-03-01|830|3609|278|1969|6|3|1|1|1969|278|3609|Saturday|1969Q1|N|Y|N|2440282|2440340|2439917|2440190|N|N|N|N|N| +2440283|AAAAAAAALFMDFCAA|1969-03-02|830|3609|278|1969|0|3|2|1|1969|278|3609|Sunday|1969Q1|N|N|N|2440282|2440340|2439918|2440191|N|N|N|N|N| +2440284|AAAAAAAAMFMDFCAA|1969-03-03|830|3609|278|1969|1|3|3|1|1969|278|3609|Monday|1969Q1|N|N|N|2440282|2440340|2439919|2440192|N|N|N|N|N| +2440285|AAAAAAAANFMDFCAA|1969-03-04|830|3610|278|1969|2|3|4|1|1969|278|3610|Tuesday|1969Q1|N|N|N|2440282|2440340|2439920|2440193|N|N|N|N|N| +2440286|AAAAAAAAOFMDFCAA|1969-03-05|830|3610|278|1969|3|3|5|1|1969|278|3610|Wednesday|1969Q1|N|N|N|2440282|2440340|2439921|2440194|N|N|N|N|N| +2440287|AAAAAAAAPFMDFCAA|1969-03-06|830|3610|278|1969|4|3|6|1|1969|278|3610|Thursday|1969Q1|N|N|N|2440282|2440340|2439922|2440195|N|N|N|N|N| +2440288|AAAAAAAAAGMDFCAA|1969-03-07|830|3610|278|1969|5|3|7|1|1969|278|3610|Friday|1969Q1|N|Y|N|2440282|2440340|2439923|2440196|N|N|N|N|N| +2440289|AAAAAAAABGMDFCAA|1969-03-08|830|3610|278|1969|6|3|8|1|1969|278|3610|Saturday|1969Q1|N|Y|N|2440282|2440340|2439924|2440197|N|N|N|N|N| +2440290|AAAAAAAACGMDFCAA|1969-03-09|830|3610|278|1969|0|3|9|1|1969|278|3610|Sunday|1969Q1|N|N|N|2440282|2440340|2439925|2440198|N|N|N|N|N| +2440291|AAAAAAAADGMDFCAA|1969-03-10|830|3610|278|1969|1|3|10|1|1969|278|3610|Monday|1969Q1|N|N|N|2440282|2440340|2439926|2440199|N|N|N|N|N| +2440292|AAAAAAAAEGMDFCAA|1969-03-11|830|3611|278|1969|2|3|11|1|1969|278|3611|Tuesday|1969Q1|N|N|N|2440282|2440340|2439927|2440200|N|N|N|N|N| +2440293|AAAAAAAAFGMDFCAA|1969-03-12|830|3611|278|1969|3|3|12|1|1969|278|3611|Wednesday|1969Q1|N|N|N|2440282|2440340|2439928|2440201|N|N|N|N|N| +2440294|AAAAAAAAGGMDFCAA|1969-03-13|830|3611|278|1969|4|3|13|1|1969|278|3611|Thursday|1969Q1|N|N|N|2440282|2440340|2439929|2440202|N|N|N|N|N| +2440295|AAAAAAAAHGMDFCAA|1969-03-14|830|3611|278|1969|5|3|14|1|1969|278|3611|Friday|1969Q1|N|Y|N|2440282|2440340|2439930|2440203|N|N|N|N|N| +2440296|AAAAAAAAIGMDFCAA|1969-03-15|830|3611|278|1969|6|3|15|1|1969|278|3611|Saturday|1969Q1|N|Y|N|2440282|2440340|2439931|2440204|N|N|N|N|N| +2440297|AAAAAAAAJGMDFCAA|1969-03-16|830|3611|278|1969|0|3|16|1|1969|278|3611|Sunday|1969Q1|N|N|N|2440282|2440340|2439932|2440205|N|N|N|N|N| +2440298|AAAAAAAAKGMDFCAA|1969-03-17|830|3611|278|1969|1|3|17|1|1969|278|3611|Monday|1969Q1|N|N|N|2440282|2440340|2439933|2440206|N|N|N|N|N| +2440299|AAAAAAAALGMDFCAA|1969-03-18|830|3612|278|1969|2|3|18|1|1969|278|3612|Tuesday|1969Q1|N|N|N|2440282|2440340|2439934|2440207|N|N|N|N|N| +2440300|AAAAAAAAMGMDFCAA|1969-03-19|830|3612|278|1969|3|3|19|1|1969|278|3612|Wednesday|1969Q1|N|N|N|2440282|2440340|2439935|2440208|N|N|N|N|N| +2440301|AAAAAAAANGMDFCAA|1969-03-20|830|3612|278|1969|4|3|20|1|1969|278|3612|Thursday|1969Q1|N|N|N|2440282|2440340|2439936|2440209|N|N|N|N|N| +2440302|AAAAAAAAOGMDFCAA|1969-03-21|830|3612|278|1969|5|3|21|1|1969|278|3612|Friday|1969Q1|N|Y|N|2440282|2440340|2439937|2440210|N|N|N|N|N| +2440303|AAAAAAAAPGMDFCAA|1969-03-22|830|3612|278|1969|6|3|22|1|1969|278|3612|Saturday|1969Q1|N|Y|N|2440282|2440340|2439938|2440211|N|N|N|N|N| +2440304|AAAAAAAAAHMDFCAA|1969-03-23|830|3612|278|1969|0|3|23|1|1969|278|3612|Sunday|1969Q1|N|N|N|2440282|2440340|2439939|2440212|N|N|N|N|N| +2440305|AAAAAAAABHMDFCAA|1969-03-24|830|3612|278|1969|1|3|24|1|1969|278|3612|Monday|1969Q1|N|N|N|2440282|2440340|2439940|2440213|N|N|N|N|N| +2440306|AAAAAAAACHMDFCAA|1969-03-25|830|3613|278|1969|2|3|25|1|1969|278|3613|Tuesday|1969Q1|N|N|N|2440282|2440340|2439941|2440214|N|N|N|N|N| +2440307|AAAAAAAADHMDFCAA|1969-03-26|830|3613|278|1969|3|3|26|1|1969|278|3613|Wednesday|1969Q1|N|N|N|2440282|2440340|2439942|2440215|N|N|N|N|N| +2440308|AAAAAAAAEHMDFCAA|1969-03-27|830|3613|278|1969|4|3|27|1|1969|278|3613|Thursday|1969Q1|N|N|N|2440282|2440340|2439943|2440216|N|N|N|N|N| +2440309|AAAAAAAAFHMDFCAA|1969-03-28|830|3613|278|1969|5|3|28|1|1969|278|3613|Friday|1969Q1|N|Y|N|2440282|2440340|2439944|2440217|N|N|N|N|N| +2440310|AAAAAAAAGHMDFCAA|1969-03-29|830|3613|278|1969|6|3|29|1|1969|278|3613|Saturday|1969Q1|N|Y|N|2440282|2440340|2439945|2440218|N|N|N|N|N| +2440311|AAAAAAAAHHMDFCAA|1969-03-30|830|3613|278|1969|0|3|30|1|1969|278|3613|Sunday|1969Q1|N|N|N|2440282|2440340|2439946|2440219|N|N|N|N|N| +2440312|AAAAAAAAIHMDFCAA|1969-03-31|830|3613|278|1969|1|3|31|1|1969|278|3613|Monday|1969Q1|N|N|N|2440282|2440340|2439947|2440220|N|N|N|N|N| +2440313|AAAAAAAAJHMDFCAA|1969-04-01|831|3614|278|1969|2|4|1|1|1969|278|3614|Tuesday|1969Q1|N|N|N|2440313|2440402|2439948|2440223|N|N|N|N|N| +2440314|AAAAAAAAKHMDFCAA|1969-04-02|831|3614|278|1969|3|4|2|2|1969|278|3614|Wednesday|1969Q2|N|N|N|2440313|2440402|2439949|2440224|N|N|N|N|N| +2440315|AAAAAAAALHMDFCAA|1969-04-03|831|3614|278|1969|4|4|3|2|1969|278|3614|Thursday|1969Q2|N|N|N|2440313|2440402|2439950|2440225|N|N|N|N|N| +2440316|AAAAAAAAMHMDFCAA|1969-04-04|831|3614|278|1969|5|4|4|2|1969|278|3614|Friday|1969Q2|N|Y|N|2440313|2440402|2439951|2440226|N|N|N|N|N| +2440317|AAAAAAAANHMDFCAA|1969-04-05|831|3614|278|1969|6|4|5|2|1969|278|3614|Saturday|1969Q2|N|Y|N|2440313|2440402|2439952|2440227|N|N|N|N|N| +2440318|AAAAAAAAOHMDFCAA|1969-04-06|831|3614|278|1969|0|4|6|2|1969|278|3614|Sunday|1969Q2|N|N|N|2440313|2440402|2439953|2440228|N|N|N|N|N| +2440319|AAAAAAAAPHMDFCAA|1969-04-07|831|3614|278|1969|1|4|7|2|1969|278|3614|Monday|1969Q2|N|N|N|2440313|2440402|2439954|2440229|N|N|N|N|N| +2440320|AAAAAAAAAIMDFCAA|1969-04-08|831|3615|278|1969|2|4|8|2|1969|278|3615|Tuesday|1969Q2|N|N|N|2440313|2440402|2439955|2440230|N|N|N|N|N| +2440321|AAAAAAAABIMDFCAA|1969-04-09|831|3615|278|1969|3|4|9|2|1969|278|3615|Wednesday|1969Q2|N|N|N|2440313|2440402|2439956|2440231|N|N|N|N|N| +2440322|AAAAAAAACIMDFCAA|1969-04-10|831|3615|278|1969|4|4|10|2|1969|278|3615|Thursday|1969Q2|N|N|N|2440313|2440402|2439957|2440232|N|N|N|N|N| +2440323|AAAAAAAADIMDFCAA|1969-04-11|831|3615|278|1969|5|4|11|2|1969|278|3615|Friday|1969Q2|N|Y|N|2440313|2440402|2439958|2440233|N|N|N|N|N| +2440324|AAAAAAAAEIMDFCAA|1969-04-12|831|3615|278|1969|6|4|12|2|1969|278|3615|Saturday|1969Q2|N|Y|N|2440313|2440402|2439959|2440234|N|N|N|N|N| +2440325|AAAAAAAAFIMDFCAA|1969-04-13|831|3615|278|1969|0|4|13|2|1969|278|3615|Sunday|1969Q2|N|N|N|2440313|2440402|2439960|2440235|N|N|N|N|N| +2440326|AAAAAAAAGIMDFCAA|1969-04-14|831|3615|278|1969|1|4|14|2|1969|278|3615|Monday|1969Q2|N|N|N|2440313|2440402|2439961|2440236|N|N|N|N|N| +2440327|AAAAAAAAHIMDFCAA|1969-04-15|831|3616|278|1969|2|4|15|2|1969|278|3616|Tuesday|1969Q2|N|N|N|2440313|2440402|2439962|2440237|N|N|N|N|N| +2440328|AAAAAAAAIIMDFCAA|1969-04-16|831|3616|278|1969|3|4|16|2|1969|278|3616|Wednesday|1969Q2|N|N|N|2440313|2440402|2439963|2440238|N|N|N|N|N| +2440329|AAAAAAAAJIMDFCAA|1969-04-17|831|3616|278|1969|4|4|17|2|1969|278|3616|Thursday|1969Q2|N|N|N|2440313|2440402|2439964|2440239|N|N|N|N|N| +2440330|AAAAAAAAKIMDFCAA|1969-04-18|831|3616|278|1969|5|4|18|2|1969|278|3616|Friday|1969Q2|N|Y|N|2440313|2440402|2439965|2440240|N|N|N|N|N| +2440331|AAAAAAAALIMDFCAA|1969-04-19|831|3616|278|1969|6|4|19|2|1969|278|3616|Saturday|1969Q2|N|Y|N|2440313|2440402|2439966|2440241|N|N|N|N|N| +2440332|AAAAAAAAMIMDFCAA|1969-04-20|831|3616|278|1969|0|4|20|2|1969|278|3616|Sunday|1969Q2|N|N|N|2440313|2440402|2439967|2440242|N|N|N|N|N| +2440333|AAAAAAAANIMDFCAA|1969-04-21|831|3616|278|1969|1|4|21|2|1969|278|3616|Monday|1969Q2|N|N|N|2440313|2440402|2439968|2440243|N|N|N|N|N| +2440334|AAAAAAAAOIMDFCAA|1969-04-22|831|3617|278|1969|2|4|22|2|1969|278|3617|Tuesday|1969Q2|N|N|N|2440313|2440402|2439969|2440244|N|N|N|N|N| +2440335|AAAAAAAAPIMDFCAA|1969-04-23|831|3617|278|1969|3|4|23|2|1969|278|3617|Wednesday|1969Q2|N|N|N|2440313|2440402|2439970|2440245|N|N|N|N|N| +2440336|AAAAAAAAAJMDFCAA|1969-04-24|831|3617|278|1969|4|4|24|2|1969|278|3617|Thursday|1969Q2|N|N|N|2440313|2440402|2439971|2440246|N|N|N|N|N| +2440337|AAAAAAAABJMDFCAA|1969-04-25|831|3617|278|1969|5|4|25|2|1969|278|3617|Friday|1969Q2|N|Y|N|2440313|2440402|2439972|2440247|N|N|N|N|N| +2440338|AAAAAAAACJMDFCAA|1969-04-26|831|3617|278|1969|6|4|26|2|1969|278|3617|Saturday|1969Q2|N|Y|N|2440313|2440402|2439973|2440248|N|N|N|N|N| +2440339|AAAAAAAADJMDFCAA|1969-04-27|831|3617|278|1969|0|4|27|2|1969|278|3617|Sunday|1969Q2|N|N|N|2440313|2440402|2439974|2440249|N|N|N|N|N| +2440340|AAAAAAAAEJMDFCAA|1969-04-28|831|3617|278|1969|1|4|28|2|1969|278|3617|Monday|1969Q2|N|N|N|2440313|2440402|2439975|2440250|N|N|N|N|N| +2440341|AAAAAAAAFJMDFCAA|1969-04-29|831|3618|278|1969|2|4|29|2|1969|278|3618|Tuesday|1969Q2|N|N|N|2440313|2440402|2439976|2440251|N|N|N|N|N| +2440342|AAAAAAAAGJMDFCAA|1969-04-30|831|3618|278|1969|3|4|30|2|1969|278|3618|Wednesday|1969Q2|N|N|N|2440313|2440402|2439977|2440252|N|N|N|N|N| +2440343|AAAAAAAAHJMDFCAA|1969-05-01|832|3618|278|1969|4|5|1|2|1969|278|3618|Thursday|1969Q2|N|N|N|2440343|2440462|2439978|2440253|N|N|N|N|N| +2440344|AAAAAAAAIJMDFCAA|1969-05-02|832|3618|278|1969|5|5|2|2|1969|278|3618|Friday|1969Q2|N|Y|N|2440343|2440462|2439979|2440254|N|N|N|N|N| +2440345|AAAAAAAAJJMDFCAA|1969-05-03|832|3618|278|1969|6|5|3|2|1969|278|3618|Saturday|1969Q2|N|Y|N|2440343|2440462|2439980|2440255|N|N|N|N|N| +2440346|AAAAAAAAKJMDFCAA|1969-05-04|832|3618|278|1969|0|5|4|2|1969|278|3618|Sunday|1969Q2|N|N|N|2440343|2440462|2439981|2440256|N|N|N|N|N| +2440347|AAAAAAAALJMDFCAA|1969-05-05|832|3618|278|1969|1|5|5|2|1969|278|3618|Monday|1969Q2|N|N|N|2440343|2440462|2439982|2440257|N|N|N|N|N| +2440348|AAAAAAAAMJMDFCAA|1969-05-06|832|3619|278|1969|2|5|6|2|1969|278|3619|Tuesday|1969Q2|N|N|N|2440343|2440462|2439983|2440258|N|N|N|N|N| +2440349|AAAAAAAANJMDFCAA|1969-05-07|832|3619|278|1969|3|5|7|2|1969|278|3619|Wednesday|1969Q2|N|N|N|2440343|2440462|2439984|2440259|N|N|N|N|N| +2440350|AAAAAAAAOJMDFCAA|1969-05-08|832|3619|278|1969|4|5|8|2|1969|278|3619|Thursday|1969Q2|N|N|N|2440343|2440462|2439985|2440260|N|N|N|N|N| +2440351|AAAAAAAAPJMDFCAA|1969-05-09|832|3619|278|1969|5|5|9|2|1969|278|3619|Friday|1969Q2|N|Y|N|2440343|2440462|2439986|2440261|N|N|N|N|N| +2440352|AAAAAAAAAKMDFCAA|1969-05-10|832|3619|278|1969|6|5|10|2|1969|278|3619|Saturday|1969Q2|N|Y|N|2440343|2440462|2439987|2440262|N|N|N|N|N| +2440353|AAAAAAAABKMDFCAA|1969-05-11|832|3619|278|1969|0|5|11|2|1969|278|3619|Sunday|1969Q2|N|N|N|2440343|2440462|2439988|2440263|N|N|N|N|N| +2440354|AAAAAAAACKMDFCAA|1969-05-12|832|3619|278|1969|1|5|12|2|1969|278|3619|Monday|1969Q2|N|N|N|2440343|2440462|2439989|2440264|N|N|N|N|N| +2440355|AAAAAAAADKMDFCAA|1969-05-13|832|3620|278|1969|2|5|13|2|1969|278|3620|Tuesday|1969Q2|N|N|N|2440343|2440462|2439990|2440265|N|N|N|N|N| +2440356|AAAAAAAAEKMDFCAA|1969-05-14|832|3620|278|1969|3|5|14|2|1969|278|3620|Wednesday|1969Q2|N|N|N|2440343|2440462|2439991|2440266|N|N|N|N|N| +2440357|AAAAAAAAFKMDFCAA|1969-05-15|832|3620|278|1969|4|5|15|2|1969|278|3620|Thursday|1969Q2|N|N|N|2440343|2440462|2439992|2440267|N|N|N|N|N| +2440358|AAAAAAAAGKMDFCAA|1969-05-16|832|3620|278|1969|5|5|16|2|1969|278|3620|Friday|1969Q2|N|Y|N|2440343|2440462|2439993|2440268|N|N|N|N|N| +2440359|AAAAAAAAHKMDFCAA|1969-05-17|832|3620|278|1969|6|5|17|2|1969|278|3620|Saturday|1969Q2|N|Y|N|2440343|2440462|2439994|2440269|N|N|N|N|N| +2440360|AAAAAAAAIKMDFCAA|1969-05-18|832|3620|278|1969|0|5|18|2|1969|278|3620|Sunday|1969Q2|N|N|N|2440343|2440462|2439995|2440270|N|N|N|N|N| +2440361|AAAAAAAAJKMDFCAA|1969-05-19|832|3620|278|1969|1|5|19|2|1969|278|3620|Monday|1969Q2|N|N|N|2440343|2440462|2439996|2440271|N|N|N|N|N| +2440362|AAAAAAAAKKMDFCAA|1969-05-20|832|3621|278|1969|2|5|20|2|1969|278|3621|Tuesday|1969Q2|N|N|N|2440343|2440462|2439997|2440272|N|N|N|N|N| +2440363|AAAAAAAALKMDFCAA|1969-05-21|832|3621|278|1969|3|5|21|2|1969|278|3621|Wednesday|1969Q2|N|N|N|2440343|2440462|2439998|2440273|N|N|N|N|N| +2440364|AAAAAAAAMKMDFCAA|1969-05-22|832|3621|278|1969|4|5|22|2|1969|278|3621|Thursday|1969Q2|N|N|N|2440343|2440462|2439999|2440274|N|N|N|N|N| +2440365|AAAAAAAANKMDFCAA|1969-05-23|832|3621|278|1969|5|5|23|2|1969|278|3621|Friday|1969Q2|N|Y|N|2440343|2440462|2440000|2440275|N|N|N|N|N| +2440366|AAAAAAAAOKMDFCAA|1969-05-24|832|3621|278|1969|6|5|24|2|1969|278|3621|Saturday|1969Q2|N|Y|N|2440343|2440462|2440001|2440276|N|N|N|N|N| +2440367|AAAAAAAAPKMDFCAA|1969-05-25|832|3621|278|1969|0|5|25|2|1969|278|3621|Sunday|1969Q2|N|N|N|2440343|2440462|2440002|2440277|N|N|N|N|N| +2440368|AAAAAAAAALMDFCAA|1969-05-26|832|3621|278|1969|1|5|26|2|1969|278|3621|Monday|1969Q2|N|N|N|2440343|2440462|2440003|2440278|N|N|N|N|N| +2440369|AAAAAAAABLMDFCAA|1969-05-27|832|3622|278|1969|2|5|27|2|1969|278|3622|Tuesday|1969Q2|N|N|N|2440343|2440462|2440004|2440279|N|N|N|N|N| +2440370|AAAAAAAACLMDFCAA|1969-05-28|832|3622|278|1969|3|5|28|2|1969|278|3622|Wednesday|1969Q2|N|N|N|2440343|2440462|2440005|2440280|N|N|N|N|N| +2440371|AAAAAAAADLMDFCAA|1969-05-29|832|3622|278|1969|4|5|29|2|1969|278|3622|Thursday|1969Q2|N|N|N|2440343|2440462|2440006|2440281|N|N|N|N|N| +2440372|AAAAAAAAELMDFCAA|1969-05-30|832|3622|278|1969|5|5|30|2|1969|278|3622|Friday|1969Q2|N|Y|N|2440343|2440462|2440007|2440282|N|N|N|N|N| +2440373|AAAAAAAAFLMDFCAA|1969-05-31|832|3622|278|1969|6|5|31|2|1969|278|3622|Saturday|1969Q2|N|Y|N|2440343|2440462|2440008|2440283|N|N|N|N|N| +2440374|AAAAAAAAGLMDFCAA|1969-06-01|833|3622|279|1969|0|6|1|2|1969|279|3622|Sunday|1969Q2|N|N|N|2440374|2440524|2440009|2440284|N|N|N|N|N| +2440375|AAAAAAAAHLMDFCAA|1969-06-02|833|3622|279|1969|1|6|2|2|1969|279|3622|Monday|1969Q2|N|N|N|2440374|2440524|2440010|2440285|N|N|N|N|N| +2440376|AAAAAAAAILMDFCAA|1969-06-03|833|3623|279|1969|2|6|3|2|1969|279|3623|Tuesday|1969Q2|N|N|N|2440374|2440524|2440011|2440286|N|N|N|N|N| +2440377|AAAAAAAAJLMDFCAA|1969-06-04|833|3623|279|1969|3|6|4|2|1969|279|3623|Wednesday|1969Q2|N|N|N|2440374|2440524|2440012|2440287|N|N|N|N|N| +2440378|AAAAAAAAKLMDFCAA|1969-06-05|833|3623|279|1969|4|6|5|2|1969|279|3623|Thursday|1969Q2|N|N|N|2440374|2440524|2440013|2440288|N|N|N|N|N| +2440379|AAAAAAAALLMDFCAA|1969-06-06|833|3623|279|1969|5|6|6|2|1969|279|3623|Friday|1969Q2|N|Y|N|2440374|2440524|2440014|2440289|N|N|N|N|N| +2440380|AAAAAAAAMLMDFCAA|1969-06-07|833|3623|279|1969|6|6|7|2|1969|279|3623|Saturday|1969Q2|N|Y|N|2440374|2440524|2440015|2440290|N|N|N|N|N| +2440381|AAAAAAAANLMDFCAA|1969-06-08|833|3623|279|1969|0|6|8|2|1969|279|3623|Sunday|1969Q2|N|N|N|2440374|2440524|2440016|2440291|N|N|N|N|N| +2440382|AAAAAAAAOLMDFCAA|1969-06-09|833|3623|279|1969|1|6|9|2|1969|279|3623|Monday|1969Q2|N|N|N|2440374|2440524|2440017|2440292|N|N|N|N|N| +2440383|AAAAAAAAPLMDFCAA|1969-06-10|833|3624|279|1969|2|6|10|2|1969|279|3624|Tuesday|1969Q2|N|N|N|2440374|2440524|2440018|2440293|N|N|N|N|N| +2440384|AAAAAAAAAMMDFCAA|1969-06-11|833|3624|279|1969|3|6|11|2|1969|279|3624|Wednesday|1969Q2|N|N|N|2440374|2440524|2440019|2440294|N|N|N|N|N| +2440385|AAAAAAAABMMDFCAA|1969-06-12|833|3624|279|1969|4|6|12|2|1969|279|3624|Thursday|1969Q2|N|N|N|2440374|2440524|2440020|2440295|N|N|N|N|N| +2440386|AAAAAAAACMMDFCAA|1969-06-13|833|3624|279|1969|5|6|13|2|1969|279|3624|Friday|1969Q2|N|Y|N|2440374|2440524|2440021|2440296|N|N|N|N|N| +2440387|AAAAAAAADMMDFCAA|1969-06-14|833|3624|279|1969|6|6|14|2|1969|279|3624|Saturday|1969Q2|N|Y|N|2440374|2440524|2440022|2440297|N|N|N|N|N| +2440388|AAAAAAAAEMMDFCAA|1969-06-15|833|3624|279|1969|0|6|15|2|1969|279|3624|Sunday|1969Q2|N|N|N|2440374|2440524|2440023|2440298|N|N|N|N|N| +2440389|AAAAAAAAFMMDFCAA|1969-06-16|833|3624|279|1969|1|6|16|2|1969|279|3624|Monday|1969Q2|N|N|N|2440374|2440524|2440024|2440299|N|N|N|N|N| +2440390|AAAAAAAAGMMDFCAA|1969-06-17|833|3625|279|1969|2|6|17|2|1969|279|3625|Tuesday|1969Q2|N|N|N|2440374|2440524|2440025|2440300|N|N|N|N|N| +2440391|AAAAAAAAHMMDFCAA|1969-06-18|833|3625|279|1969|3|6|18|2|1969|279|3625|Wednesday|1969Q2|N|N|N|2440374|2440524|2440026|2440301|N|N|N|N|N| +2440392|AAAAAAAAIMMDFCAA|1969-06-19|833|3625|279|1969|4|6|19|2|1969|279|3625|Thursday|1969Q2|N|N|N|2440374|2440524|2440027|2440302|N|N|N|N|N| +2440393|AAAAAAAAJMMDFCAA|1969-06-20|833|3625|279|1969|5|6|20|2|1969|279|3625|Friday|1969Q2|N|Y|N|2440374|2440524|2440028|2440303|N|N|N|N|N| +2440394|AAAAAAAAKMMDFCAA|1969-06-21|833|3625|279|1969|6|6|21|2|1969|279|3625|Saturday|1969Q2|N|Y|N|2440374|2440524|2440029|2440304|N|N|N|N|N| +2440395|AAAAAAAALMMDFCAA|1969-06-22|833|3625|279|1969|0|6|22|2|1969|279|3625|Sunday|1969Q2|N|N|N|2440374|2440524|2440030|2440305|N|N|N|N|N| +2440396|AAAAAAAAMMMDFCAA|1969-06-23|833|3625|279|1969|1|6|23|2|1969|279|3625|Monday|1969Q2|N|N|N|2440374|2440524|2440031|2440306|N|N|N|N|N| +2440397|AAAAAAAANMMDFCAA|1969-06-24|833|3626|279|1969|2|6|24|2|1969|279|3626|Tuesday|1969Q2|N|N|N|2440374|2440524|2440032|2440307|N|N|N|N|N| +2440398|AAAAAAAAOMMDFCAA|1969-06-25|833|3626|279|1969|3|6|25|2|1969|279|3626|Wednesday|1969Q2|N|N|N|2440374|2440524|2440033|2440308|N|N|N|N|N| +2440399|AAAAAAAAPMMDFCAA|1969-06-26|833|3626|279|1969|4|6|26|2|1969|279|3626|Thursday|1969Q2|N|N|N|2440374|2440524|2440034|2440309|N|N|N|N|N| +2440400|AAAAAAAAANMDFCAA|1969-06-27|833|3626|279|1969|5|6|27|2|1969|279|3626|Friday|1969Q2|N|Y|N|2440374|2440524|2440035|2440310|N|N|N|N|N| +2440401|AAAAAAAABNMDFCAA|1969-06-28|833|3626|279|1969|6|6|28|2|1969|279|3626|Saturday|1969Q2|N|Y|N|2440374|2440524|2440036|2440311|N|N|N|N|N| +2440402|AAAAAAAACNMDFCAA|1969-06-29|833|3626|279|1969|0|6|29|2|1969|279|3626|Sunday|1969Q2|N|N|N|2440374|2440524|2440037|2440312|N|N|N|N|N| +2440403|AAAAAAAADNMDFCAA|1969-06-30|833|3626|279|1969|1|6|30|2|1969|279|3626|Monday|1969Q2|N|N|N|2440374|2440524|2440038|2440313|N|N|N|N|N| +2440404|AAAAAAAAENMDFCAA|1969-07-01|834|3627|279|1969|2|7|1|2|1969|279|3627|Tuesday|1969Q2|N|N|N|2440404|2440584|2440039|2440313|N|N|N|N|N| +2440405|AAAAAAAAFNMDFCAA|1969-07-02|834|3627|279|1969|3|7|2|3|1969|279|3627|Wednesday|1969Q3|N|N|N|2440404|2440584|2440040|2440314|N|N|N|N|N| +2440406|AAAAAAAAGNMDFCAA|1969-07-03|834|3627|279|1969|4|7|3|3|1969|279|3627|Thursday|1969Q3|N|N|N|2440404|2440584|2440041|2440315|N|N|N|N|N| +2440407|AAAAAAAAHNMDFCAA|1969-07-04|834|3627|279|1969|5|7|4|3|1969|279|3627|Friday|1969Q3|N|Y|N|2440404|2440584|2440042|2440316|N|N|N|N|N| +2440408|AAAAAAAAINMDFCAA|1969-07-05|834|3627|279|1969|6|7|5|3|1969|279|3627|Saturday|1969Q3|Y|Y|N|2440404|2440584|2440043|2440317|N|N|N|N|N| +2440409|AAAAAAAAJNMDFCAA|1969-07-06|834|3627|279|1969|0|7|6|3|1969|279|3627|Sunday|1969Q3|N|N|Y|2440404|2440584|2440044|2440318|N|N|N|N|N| +2440410|AAAAAAAAKNMDFCAA|1969-07-07|834|3627|279|1969|1|7|7|3|1969|279|3627|Monday|1969Q3|N|N|N|2440404|2440584|2440045|2440319|N|N|N|N|N| +2440411|AAAAAAAALNMDFCAA|1969-07-08|834|3628|279|1969|2|7|8|3|1969|279|3628|Tuesday|1969Q3|N|N|N|2440404|2440584|2440046|2440320|N|N|N|N|N| +2440412|AAAAAAAAMNMDFCAA|1969-07-09|834|3628|279|1969|3|7|9|3|1969|279|3628|Wednesday|1969Q3|N|N|N|2440404|2440584|2440047|2440321|N|N|N|N|N| +2440413|AAAAAAAANNMDFCAA|1969-07-10|834|3628|279|1969|4|7|10|3|1969|279|3628|Thursday|1969Q3|N|N|N|2440404|2440584|2440048|2440322|N|N|N|N|N| +2440414|AAAAAAAAONMDFCAA|1969-07-11|834|3628|279|1969|5|7|11|3|1969|279|3628|Friday|1969Q3|N|Y|N|2440404|2440584|2440049|2440323|N|N|N|N|N| +2440415|AAAAAAAAPNMDFCAA|1969-07-12|834|3628|279|1969|6|7|12|3|1969|279|3628|Saturday|1969Q3|N|Y|N|2440404|2440584|2440050|2440324|N|N|N|N|N| +2440416|AAAAAAAAAOMDFCAA|1969-07-13|834|3628|279|1969|0|7|13|3|1969|279|3628|Sunday|1969Q3|N|N|N|2440404|2440584|2440051|2440325|N|N|N|N|N| +2440417|AAAAAAAABOMDFCAA|1969-07-14|834|3628|279|1969|1|7|14|3|1969|279|3628|Monday|1969Q3|N|N|N|2440404|2440584|2440052|2440326|N|N|N|N|N| +2440418|AAAAAAAACOMDFCAA|1969-07-15|834|3629|279|1969|2|7|15|3|1969|279|3629|Tuesday|1969Q3|N|N|N|2440404|2440584|2440053|2440327|N|N|N|N|N| +2440419|AAAAAAAADOMDFCAA|1969-07-16|834|3629|279|1969|3|7|16|3|1969|279|3629|Wednesday|1969Q3|N|N|N|2440404|2440584|2440054|2440328|N|N|N|N|N| +2440420|AAAAAAAAEOMDFCAA|1969-07-17|834|3629|279|1969|4|7|17|3|1969|279|3629|Thursday|1969Q3|N|N|N|2440404|2440584|2440055|2440329|N|N|N|N|N| +2440421|AAAAAAAAFOMDFCAA|1969-07-18|834|3629|279|1969|5|7|18|3|1969|279|3629|Friday|1969Q3|N|Y|N|2440404|2440584|2440056|2440330|N|N|N|N|N| +2440422|AAAAAAAAGOMDFCAA|1969-07-19|834|3629|279|1969|6|7|19|3|1969|279|3629|Saturday|1969Q3|N|Y|N|2440404|2440584|2440057|2440331|N|N|N|N|N| +2440423|AAAAAAAAHOMDFCAA|1969-07-20|834|3629|279|1969|0|7|20|3|1969|279|3629|Sunday|1969Q3|N|N|N|2440404|2440584|2440058|2440332|N|N|N|N|N| +2440424|AAAAAAAAIOMDFCAA|1969-07-21|834|3629|279|1969|1|7|21|3|1969|279|3629|Monday|1969Q3|N|N|N|2440404|2440584|2440059|2440333|N|N|N|N|N| +2440425|AAAAAAAAJOMDFCAA|1969-07-22|834|3630|279|1969|2|7|22|3|1969|279|3630|Tuesday|1969Q3|N|N|N|2440404|2440584|2440060|2440334|N|N|N|N|N| +2440426|AAAAAAAAKOMDFCAA|1969-07-23|834|3630|279|1969|3|7|23|3|1969|279|3630|Wednesday|1969Q3|N|N|N|2440404|2440584|2440061|2440335|N|N|N|N|N| +2440427|AAAAAAAALOMDFCAA|1969-07-24|834|3630|279|1969|4|7|24|3|1969|279|3630|Thursday|1969Q3|N|N|N|2440404|2440584|2440062|2440336|N|N|N|N|N| +2440428|AAAAAAAAMOMDFCAA|1969-07-25|834|3630|279|1969|5|7|25|3|1969|279|3630|Friday|1969Q3|N|Y|N|2440404|2440584|2440063|2440337|N|N|N|N|N| +2440429|AAAAAAAANOMDFCAA|1969-07-26|834|3630|279|1969|6|7|26|3|1969|279|3630|Saturday|1969Q3|N|Y|N|2440404|2440584|2440064|2440338|N|N|N|N|N| +2440430|AAAAAAAAOOMDFCAA|1969-07-27|834|3630|279|1969|0|7|27|3|1969|279|3630|Sunday|1969Q3|N|N|N|2440404|2440584|2440065|2440339|N|N|N|N|N| +2440431|AAAAAAAAPOMDFCAA|1969-07-28|834|3630|279|1969|1|7|28|3|1969|279|3630|Monday|1969Q3|N|N|N|2440404|2440584|2440066|2440340|N|N|N|N|N| +2440432|AAAAAAAAAPMDFCAA|1969-07-29|834|3631|279|1969|2|7|29|3|1969|279|3631|Tuesday|1969Q3|N|N|N|2440404|2440584|2440067|2440341|N|N|N|N|N| +2440433|AAAAAAAABPMDFCAA|1969-07-30|834|3631|279|1969|3|7|30|3|1969|279|3631|Wednesday|1969Q3|N|N|N|2440404|2440584|2440068|2440342|N|N|N|N|N| +2440434|AAAAAAAACPMDFCAA|1969-07-31|834|3631|279|1969|4|7|31|3|1969|279|3631|Thursday|1969Q3|N|N|N|2440404|2440584|2440069|2440343|N|N|N|N|N| +2440435|AAAAAAAADPMDFCAA|1969-08-01|835|3631|279|1969|5|8|1|3|1969|279|3631|Friday|1969Q3|N|Y|N|2440435|2440646|2440070|2440344|N|N|N|N|N| +2440436|AAAAAAAAEPMDFCAA|1969-08-02|835|3631|279|1969|6|8|2|3|1969|279|3631|Saturday|1969Q3|N|Y|N|2440435|2440646|2440071|2440345|N|N|N|N|N| +2440437|AAAAAAAAFPMDFCAA|1969-08-03|835|3631|279|1969|0|8|3|3|1969|279|3631|Sunday|1969Q3|N|N|N|2440435|2440646|2440072|2440346|N|N|N|N|N| +2440438|AAAAAAAAGPMDFCAA|1969-08-04|835|3631|279|1969|1|8|4|3|1969|279|3631|Monday|1969Q3|N|N|N|2440435|2440646|2440073|2440347|N|N|N|N|N| +2440439|AAAAAAAAHPMDFCAA|1969-08-05|835|3632|279|1969|2|8|5|3|1969|279|3632|Tuesday|1969Q3|N|N|N|2440435|2440646|2440074|2440348|N|N|N|N|N| +2440440|AAAAAAAAIPMDFCAA|1969-08-06|835|3632|279|1969|3|8|6|3|1969|279|3632|Wednesday|1969Q3|N|N|N|2440435|2440646|2440075|2440349|N|N|N|N|N| +2440441|AAAAAAAAJPMDFCAA|1969-08-07|835|3632|279|1969|4|8|7|3|1969|279|3632|Thursday|1969Q3|N|N|N|2440435|2440646|2440076|2440350|N|N|N|N|N| +2440442|AAAAAAAAKPMDFCAA|1969-08-08|835|3632|279|1969|5|8|8|3|1969|279|3632|Friday|1969Q3|N|Y|N|2440435|2440646|2440077|2440351|N|N|N|N|N| +2440443|AAAAAAAALPMDFCAA|1969-08-09|835|3632|279|1969|6|8|9|3|1969|279|3632|Saturday|1969Q3|N|Y|N|2440435|2440646|2440078|2440352|N|N|N|N|N| +2440444|AAAAAAAAMPMDFCAA|1969-08-10|835|3632|279|1969|0|8|10|3|1969|279|3632|Sunday|1969Q3|N|N|N|2440435|2440646|2440079|2440353|N|N|N|N|N| +2440445|AAAAAAAANPMDFCAA|1969-08-11|835|3632|279|1969|1|8|11|3|1969|279|3632|Monday|1969Q3|N|N|N|2440435|2440646|2440080|2440354|N|N|N|N|N| +2440446|AAAAAAAAOPMDFCAA|1969-08-12|835|3633|279|1969|2|8|12|3|1969|279|3633|Tuesday|1969Q3|N|N|N|2440435|2440646|2440081|2440355|N|N|N|N|N| +2440447|AAAAAAAAPPMDFCAA|1969-08-13|835|3633|279|1969|3|8|13|3|1969|279|3633|Wednesday|1969Q3|N|N|N|2440435|2440646|2440082|2440356|N|N|N|N|N| +2440448|AAAAAAAAAANDFCAA|1969-08-14|835|3633|279|1969|4|8|14|3|1969|279|3633|Thursday|1969Q3|N|N|N|2440435|2440646|2440083|2440357|N|N|N|N|N| +2440449|AAAAAAAABANDFCAA|1969-08-15|835|3633|279|1969|5|8|15|3|1969|279|3633|Friday|1969Q3|N|Y|N|2440435|2440646|2440084|2440358|N|N|N|N|N| +2440450|AAAAAAAACANDFCAA|1969-08-16|835|3633|279|1969|6|8|16|3|1969|279|3633|Saturday|1969Q3|N|Y|N|2440435|2440646|2440085|2440359|N|N|N|N|N| +2440451|AAAAAAAADANDFCAA|1969-08-17|835|3633|279|1969|0|8|17|3|1969|279|3633|Sunday|1969Q3|N|N|N|2440435|2440646|2440086|2440360|N|N|N|N|N| +2440452|AAAAAAAAEANDFCAA|1969-08-18|835|3633|279|1969|1|8|18|3|1969|279|3633|Monday|1969Q3|N|N|N|2440435|2440646|2440087|2440361|N|N|N|N|N| +2440453|AAAAAAAAFANDFCAA|1969-08-19|835|3634|279|1969|2|8|19|3|1969|279|3634|Tuesday|1969Q3|N|N|N|2440435|2440646|2440088|2440362|N|N|N|N|N| +2440454|AAAAAAAAGANDFCAA|1969-08-20|835|3634|279|1969|3|8|20|3|1969|279|3634|Wednesday|1969Q3|N|N|N|2440435|2440646|2440089|2440363|N|N|N|N|N| +2440455|AAAAAAAAHANDFCAA|1969-08-21|835|3634|279|1969|4|8|21|3|1969|279|3634|Thursday|1969Q3|N|N|N|2440435|2440646|2440090|2440364|N|N|N|N|N| +2440456|AAAAAAAAIANDFCAA|1969-08-22|835|3634|279|1969|5|8|22|3|1969|279|3634|Friday|1969Q3|N|Y|N|2440435|2440646|2440091|2440365|N|N|N|N|N| +2440457|AAAAAAAAJANDFCAA|1969-08-23|835|3634|279|1969|6|8|23|3|1969|279|3634|Saturday|1969Q3|N|Y|N|2440435|2440646|2440092|2440366|N|N|N|N|N| +2440458|AAAAAAAAKANDFCAA|1969-08-24|835|3634|279|1969|0|8|24|3|1969|279|3634|Sunday|1969Q3|N|N|N|2440435|2440646|2440093|2440367|N|N|N|N|N| +2440459|AAAAAAAALANDFCAA|1969-08-25|835|3634|279|1969|1|8|25|3|1969|279|3634|Monday|1969Q3|N|N|N|2440435|2440646|2440094|2440368|N|N|N|N|N| +2440460|AAAAAAAAMANDFCAA|1969-08-26|835|3635|279|1969|2|8|26|3|1969|279|3635|Tuesday|1969Q3|N|N|N|2440435|2440646|2440095|2440369|N|N|N|N|N| +2440461|AAAAAAAANANDFCAA|1969-08-27|835|3635|279|1969|3|8|27|3|1969|279|3635|Wednesday|1969Q3|N|N|N|2440435|2440646|2440096|2440370|N|N|N|N|N| +2440462|AAAAAAAAOANDFCAA|1969-08-28|835|3635|279|1969|4|8|28|3|1969|279|3635|Thursday|1969Q3|N|N|N|2440435|2440646|2440097|2440371|N|N|N|N|N| +2440463|AAAAAAAAPANDFCAA|1969-08-29|835|3635|279|1969|5|8|29|3|1969|279|3635|Friday|1969Q3|N|Y|N|2440435|2440646|2440098|2440372|N|N|N|N|N| +2440464|AAAAAAAAABNDFCAA|1969-08-30|835|3635|279|1969|6|8|30|3|1969|279|3635|Saturday|1969Q3|N|Y|N|2440435|2440646|2440099|2440373|N|N|N|N|N| +2440465|AAAAAAAABBNDFCAA|1969-08-31|835|3635|279|1969|0|8|31|3|1969|279|3635|Sunday|1969Q3|N|N|N|2440435|2440646|2440100|2440374|N|N|N|N|N| +2440466|AAAAAAAACBNDFCAA|1969-09-01|836|3635|280|1969|1|9|1|3|1969|280|3635|Monday|1969Q3|N|N|N|2440466|2440708|2440101|2440375|N|N|N|N|N| +2440467|AAAAAAAADBNDFCAA|1969-09-02|836|3636|280|1969|2|9|2|3|1969|280|3636|Tuesday|1969Q3|N|N|N|2440466|2440708|2440102|2440376|N|N|N|N|N| +2440468|AAAAAAAAEBNDFCAA|1969-09-03|836|3636|280|1969|3|9|3|3|1969|280|3636|Wednesday|1969Q3|N|N|N|2440466|2440708|2440103|2440377|N|N|N|N|N| +2440469|AAAAAAAAFBNDFCAA|1969-09-04|836|3636|280|1969|4|9|4|3|1969|280|3636|Thursday|1969Q3|N|N|N|2440466|2440708|2440104|2440378|N|N|N|N|N| +2440470|AAAAAAAAGBNDFCAA|1969-09-05|836|3636|280|1969|5|9|5|3|1969|280|3636|Friday|1969Q3|N|Y|N|2440466|2440708|2440105|2440379|N|N|N|N|N| +2440471|AAAAAAAAHBNDFCAA|1969-09-06|836|3636|280|1969|6|9|6|3|1969|280|3636|Saturday|1969Q3|N|Y|N|2440466|2440708|2440106|2440380|N|N|N|N|N| +2440472|AAAAAAAAIBNDFCAA|1969-09-07|836|3636|280|1969|0|9|7|3|1969|280|3636|Sunday|1969Q3|N|N|N|2440466|2440708|2440107|2440381|N|N|N|N|N| +2440473|AAAAAAAAJBNDFCAA|1969-09-08|836|3636|280|1969|1|9|8|3|1969|280|3636|Monday|1969Q3|N|N|N|2440466|2440708|2440108|2440382|N|N|N|N|N| +2440474|AAAAAAAAKBNDFCAA|1969-09-09|836|3637|280|1969|2|9|9|3|1969|280|3637|Tuesday|1969Q3|N|N|N|2440466|2440708|2440109|2440383|N|N|N|N|N| +2440475|AAAAAAAALBNDFCAA|1969-09-10|836|3637|280|1969|3|9|10|3|1969|280|3637|Wednesday|1969Q3|N|N|N|2440466|2440708|2440110|2440384|N|N|N|N|N| +2440476|AAAAAAAAMBNDFCAA|1969-09-11|836|3637|280|1969|4|9|11|3|1969|280|3637|Thursday|1969Q3|N|N|N|2440466|2440708|2440111|2440385|N|N|N|N|N| +2440477|AAAAAAAANBNDFCAA|1969-09-12|836|3637|280|1969|5|9|12|3|1969|280|3637|Friday|1969Q3|N|Y|N|2440466|2440708|2440112|2440386|N|N|N|N|N| +2440478|AAAAAAAAOBNDFCAA|1969-09-13|836|3637|280|1969|6|9|13|3|1969|280|3637|Saturday|1969Q3|N|Y|N|2440466|2440708|2440113|2440387|N|N|N|N|N| +2440479|AAAAAAAAPBNDFCAA|1969-09-14|836|3637|280|1969|0|9|14|3|1969|280|3637|Sunday|1969Q3|N|N|N|2440466|2440708|2440114|2440388|N|N|N|N|N| +2440480|AAAAAAAAACNDFCAA|1969-09-15|836|3637|280|1969|1|9|15|3|1969|280|3637|Monday|1969Q3|N|N|N|2440466|2440708|2440115|2440389|N|N|N|N|N| +2440481|AAAAAAAABCNDFCAA|1969-09-16|836|3638|280|1969|2|9|16|3|1969|280|3638|Tuesday|1969Q3|N|N|N|2440466|2440708|2440116|2440390|N|N|N|N|N| +2440482|AAAAAAAACCNDFCAA|1969-09-17|836|3638|280|1969|3|9|17|3|1969|280|3638|Wednesday|1969Q3|N|N|N|2440466|2440708|2440117|2440391|N|N|N|N|N| +2440483|AAAAAAAADCNDFCAA|1969-09-18|836|3638|280|1969|4|9|18|3|1969|280|3638|Thursday|1969Q3|N|N|N|2440466|2440708|2440118|2440392|N|N|N|N|N| +2440484|AAAAAAAAECNDFCAA|1969-09-19|836|3638|280|1969|5|9|19|3|1969|280|3638|Friday|1969Q3|N|Y|N|2440466|2440708|2440119|2440393|N|N|N|N|N| +2440485|AAAAAAAAFCNDFCAA|1969-09-20|836|3638|280|1969|6|9|20|3|1969|280|3638|Saturday|1969Q3|N|Y|N|2440466|2440708|2440120|2440394|N|N|N|N|N| +2440486|AAAAAAAAGCNDFCAA|1969-09-21|836|3638|280|1969|0|9|21|3|1969|280|3638|Sunday|1969Q3|N|N|N|2440466|2440708|2440121|2440395|N|N|N|N|N| +2440487|AAAAAAAAHCNDFCAA|1969-09-22|836|3638|280|1969|1|9|22|3|1969|280|3638|Monday|1969Q3|N|N|N|2440466|2440708|2440122|2440396|N|N|N|N|N| +2440488|AAAAAAAAICNDFCAA|1969-09-23|836|3639|280|1969|2|9|23|3|1969|280|3639|Tuesday|1969Q3|N|N|N|2440466|2440708|2440123|2440397|N|N|N|N|N| +2440489|AAAAAAAAJCNDFCAA|1969-09-24|836|3639|280|1969|3|9|24|3|1969|280|3639|Wednesday|1969Q3|N|N|N|2440466|2440708|2440124|2440398|N|N|N|N|N| +2440490|AAAAAAAAKCNDFCAA|1969-09-25|836|3639|280|1969|4|9|25|3|1969|280|3639|Thursday|1969Q3|N|N|N|2440466|2440708|2440125|2440399|N|N|N|N|N| +2440491|AAAAAAAALCNDFCAA|1969-09-26|836|3639|280|1969|5|9|26|3|1969|280|3639|Friday|1969Q3|N|Y|N|2440466|2440708|2440126|2440400|N|N|N|N|N| +2440492|AAAAAAAAMCNDFCAA|1969-09-27|836|3639|280|1969|6|9|27|3|1969|280|3639|Saturday|1969Q3|N|Y|N|2440466|2440708|2440127|2440401|N|N|N|N|N| +2440493|AAAAAAAANCNDFCAA|1969-09-28|836|3639|280|1969|0|9|28|3|1969|280|3639|Sunday|1969Q3|N|N|N|2440466|2440708|2440128|2440402|N|N|N|N|N| +2440494|AAAAAAAAOCNDFCAA|1969-09-29|836|3639|280|1969|1|9|29|3|1969|280|3639|Monday|1969Q3|N|N|N|2440466|2440708|2440129|2440403|N|N|N|N|N| +2440495|AAAAAAAAPCNDFCAA|1969-09-30|836|3640|280|1969|2|9|30|3|1969|280|3640|Tuesday|1969Q3|N|N|N|2440466|2440708|2440130|2440404|N|N|N|N|N| +2440496|AAAAAAAAADNDFCAA|1969-10-01|837|3640|280|1969|3|10|1|3|1969|280|3640|Wednesday|1969Q3|N|N|N|2440496|2440768|2440131|2440404|N|N|N|N|N| +2440497|AAAAAAAABDNDFCAA|1969-10-02|837|3640|280|1969|4|10|2|4|1969|280|3640|Thursday|1969Q4|N|N|N|2440496|2440768|2440132|2440405|N|N|N|N|N| +2440498|AAAAAAAACDNDFCAA|1969-10-03|837|3640|280|1969|5|10|3|4|1969|280|3640|Friday|1969Q4|N|Y|N|2440496|2440768|2440133|2440406|N|N|N|N|N| +2440499|AAAAAAAADDNDFCAA|1969-10-04|837|3640|280|1969|6|10|4|4|1969|280|3640|Saturday|1969Q4|N|Y|N|2440496|2440768|2440134|2440407|N|N|N|N|N| +2440500|AAAAAAAAEDNDFCAA|1969-10-05|837|3640|280|1969|0|10|5|4|1969|280|3640|Sunday|1969Q4|N|N|N|2440496|2440768|2440135|2440408|N|N|N|N|N| +2440501|AAAAAAAAFDNDFCAA|1969-10-06|837|3640|280|1969|1|10|6|4|1969|280|3640|Monday|1969Q4|N|N|N|2440496|2440768|2440136|2440409|N|N|N|N|N| +2440502|AAAAAAAAGDNDFCAA|1969-10-07|837|3641|280|1969|2|10|7|4|1969|280|3641|Tuesday|1969Q4|N|N|N|2440496|2440768|2440137|2440410|N|N|N|N|N| +2440503|AAAAAAAAHDNDFCAA|1969-10-08|837|3641|280|1969|3|10|8|4|1969|280|3641|Wednesday|1969Q4|N|N|N|2440496|2440768|2440138|2440411|N|N|N|N|N| +2440504|AAAAAAAAIDNDFCAA|1969-10-09|837|3641|280|1969|4|10|9|4|1969|280|3641|Thursday|1969Q4|N|N|N|2440496|2440768|2440139|2440412|N|N|N|N|N| +2440505|AAAAAAAAJDNDFCAA|1969-10-10|837|3641|280|1969|5|10|10|4|1969|280|3641|Friday|1969Q4|N|Y|N|2440496|2440768|2440140|2440413|N|N|N|N|N| +2440506|AAAAAAAAKDNDFCAA|1969-10-11|837|3641|280|1969|6|10|11|4|1969|280|3641|Saturday|1969Q4|N|Y|N|2440496|2440768|2440141|2440414|N|N|N|N|N| +2440507|AAAAAAAALDNDFCAA|1969-10-12|837|3641|280|1969|0|10|12|4|1969|280|3641|Sunday|1969Q4|N|N|N|2440496|2440768|2440142|2440415|N|N|N|N|N| +2440508|AAAAAAAAMDNDFCAA|1969-10-13|837|3641|280|1969|1|10|13|4|1969|280|3641|Monday|1969Q4|N|N|N|2440496|2440768|2440143|2440416|N|N|N|N|N| +2440509|AAAAAAAANDNDFCAA|1969-10-14|837|3642|280|1969|2|10|14|4|1969|280|3642|Tuesday|1969Q4|N|N|N|2440496|2440768|2440144|2440417|N|N|N|N|N| +2440510|AAAAAAAAODNDFCAA|1969-10-15|837|3642|280|1969|3|10|15|4|1969|280|3642|Wednesday|1969Q4|N|N|N|2440496|2440768|2440145|2440418|N|N|N|N|N| +2440511|AAAAAAAAPDNDFCAA|1969-10-16|837|3642|280|1969|4|10|16|4|1969|280|3642|Thursday|1969Q4|N|N|N|2440496|2440768|2440146|2440419|N|N|N|N|N| +2440512|AAAAAAAAAENDFCAA|1969-10-17|837|3642|280|1969|5|10|17|4|1969|280|3642|Friday|1969Q4|N|Y|N|2440496|2440768|2440147|2440420|N|N|N|N|N| +2440513|AAAAAAAABENDFCAA|1969-10-18|837|3642|280|1969|6|10|18|4|1969|280|3642|Saturday|1969Q4|N|Y|N|2440496|2440768|2440148|2440421|N|N|N|N|N| +2440514|AAAAAAAACENDFCAA|1969-10-19|837|3642|280|1969|0|10|19|4|1969|280|3642|Sunday|1969Q4|N|N|N|2440496|2440768|2440149|2440422|N|N|N|N|N| +2440515|AAAAAAAADENDFCAA|1969-10-20|837|3642|280|1969|1|10|20|4|1969|280|3642|Monday|1969Q4|N|N|N|2440496|2440768|2440150|2440423|N|N|N|N|N| +2440516|AAAAAAAAEENDFCAA|1969-10-21|837|3643|280|1969|2|10|21|4|1969|280|3643|Tuesday|1969Q4|N|N|N|2440496|2440768|2440151|2440424|N|N|N|N|N| +2440517|AAAAAAAAFENDFCAA|1969-10-22|837|3643|280|1969|3|10|22|4|1969|280|3643|Wednesday|1969Q4|N|N|N|2440496|2440768|2440152|2440425|N|N|N|N|N| +2440518|AAAAAAAAGENDFCAA|1969-10-23|837|3643|280|1969|4|10|23|4|1969|280|3643|Thursday|1969Q4|N|N|N|2440496|2440768|2440153|2440426|N|N|N|N|N| +2440519|AAAAAAAAHENDFCAA|1969-10-24|837|3643|280|1969|5|10|24|4|1969|280|3643|Friday|1969Q4|N|Y|N|2440496|2440768|2440154|2440427|N|N|N|N|N| +2440520|AAAAAAAAIENDFCAA|1969-10-25|837|3643|280|1969|6|10|25|4|1969|280|3643|Saturday|1969Q4|N|Y|N|2440496|2440768|2440155|2440428|N|N|N|N|N| +2440521|AAAAAAAAJENDFCAA|1969-10-26|837|3643|280|1969|0|10|26|4|1969|280|3643|Sunday|1969Q4|N|N|N|2440496|2440768|2440156|2440429|N|N|N|N|N| +2440522|AAAAAAAAKENDFCAA|1969-10-27|837|3643|280|1969|1|10|27|4|1969|280|3643|Monday|1969Q4|N|N|N|2440496|2440768|2440157|2440430|N|N|N|N|N| +2440523|AAAAAAAALENDFCAA|1969-10-28|837|3644|280|1969|2|10|28|4|1969|280|3644|Tuesday|1969Q4|N|N|N|2440496|2440768|2440158|2440431|N|N|N|N|N| +2440524|AAAAAAAAMENDFCAA|1969-10-29|837|3644|280|1969|3|10|29|4|1969|280|3644|Wednesday|1969Q4|N|N|N|2440496|2440768|2440159|2440432|N|N|N|N|N| +2440525|AAAAAAAANENDFCAA|1969-10-30|837|3644|280|1969|4|10|30|4|1969|280|3644|Thursday|1969Q4|N|N|N|2440496|2440768|2440160|2440433|N|N|N|N|N| +2440526|AAAAAAAAOENDFCAA|1969-10-31|837|3644|280|1969|5|10|31|4|1969|280|3644|Friday|1969Q4|N|Y|N|2440496|2440768|2440161|2440434|N|N|N|N|N| +2440527|AAAAAAAAPENDFCAA|1969-11-01|838|3644|280|1969|6|11|1|4|1969|280|3644|Saturday|1969Q4|N|Y|N|2440527|2440830|2440162|2440435|N|N|N|N|N| +2440528|AAAAAAAAAFNDFCAA|1969-11-02|838|3644|280|1969|0|11|2|4|1969|280|3644|Sunday|1969Q4|N|N|N|2440527|2440830|2440163|2440436|N|N|N|N|N| +2440529|AAAAAAAABFNDFCAA|1969-11-03|838|3644|280|1969|1|11|3|4|1969|280|3644|Monday|1969Q4|N|N|N|2440527|2440830|2440164|2440437|N|N|N|N|N| +2440530|AAAAAAAACFNDFCAA|1969-11-04|838|3645|280|1969|2|11|4|4|1969|280|3645|Tuesday|1969Q4|N|N|N|2440527|2440830|2440165|2440438|N|N|N|N|N| +2440531|AAAAAAAADFNDFCAA|1969-11-05|838|3645|280|1969|3|11|5|4|1969|280|3645|Wednesday|1969Q4|N|N|N|2440527|2440830|2440166|2440439|N|N|N|N|N| +2440532|AAAAAAAAEFNDFCAA|1969-11-06|838|3645|280|1969|4|11|6|4|1969|280|3645|Thursday|1969Q4|N|N|N|2440527|2440830|2440167|2440440|N|N|N|N|N| +2440533|AAAAAAAAFFNDFCAA|1969-11-07|838|3645|280|1969|5|11|7|4|1969|280|3645|Friday|1969Q4|N|Y|N|2440527|2440830|2440168|2440441|N|N|N|N|N| +2440534|AAAAAAAAGFNDFCAA|1969-11-08|838|3645|280|1969|6|11|8|4|1969|280|3645|Saturday|1969Q4|N|Y|N|2440527|2440830|2440169|2440442|N|N|N|N|N| +2440535|AAAAAAAAHFNDFCAA|1969-11-09|838|3645|280|1969|0|11|9|4|1969|280|3645|Sunday|1969Q4|N|N|N|2440527|2440830|2440170|2440443|N|N|N|N|N| +2440536|AAAAAAAAIFNDFCAA|1969-11-10|838|3645|280|1969|1|11|10|4|1969|280|3645|Monday|1969Q4|N|N|N|2440527|2440830|2440171|2440444|N|N|N|N|N| +2440537|AAAAAAAAJFNDFCAA|1969-11-11|838|3646|280|1969|2|11|11|4|1969|280|3646|Tuesday|1969Q4|N|N|N|2440527|2440830|2440172|2440445|N|N|N|N|N| +2440538|AAAAAAAAKFNDFCAA|1969-11-12|838|3646|280|1969|3|11|12|4|1969|280|3646|Wednesday|1969Q4|N|N|N|2440527|2440830|2440173|2440446|N|N|N|N|N| +2440539|AAAAAAAALFNDFCAA|1969-11-13|838|3646|280|1969|4|11|13|4|1969|280|3646|Thursday|1969Q4|N|N|N|2440527|2440830|2440174|2440447|N|N|N|N|N| +2440540|AAAAAAAAMFNDFCAA|1969-11-14|838|3646|280|1969|5|11|14|4|1969|280|3646|Friday|1969Q4|N|Y|N|2440527|2440830|2440175|2440448|N|N|N|N|N| +2440541|AAAAAAAANFNDFCAA|1969-11-15|838|3646|280|1969|6|11|15|4|1969|280|3646|Saturday|1969Q4|N|Y|N|2440527|2440830|2440176|2440449|N|N|N|N|N| +2440542|AAAAAAAAOFNDFCAA|1969-11-16|838|3646|280|1969|0|11|16|4|1969|280|3646|Sunday|1969Q4|N|N|N|2440527|2440830|2440177|2440450|N|N|N|N|N| +2440543|AAAAAAAAPFNDFCAA|1969-11-17|838|3646|280|1969|1|11|17|4|1969|280|3646|Monday|1969Q4|N|N|N|2440527|2440830|2440178|2440451|N|N|N|N|N| +2440544|AAAAAAAAAGNDFCAA|1969-11-18|838|3647|280|1969|2|11|18|4|1969|280|3647|Tuesday|1969Q4|N|N|N|2440527|2440830|2440179|2440452|N|N|N|N|N| +2440545|AAAAAAAABGNDFCAA|1969-11-19|838|3647|280|1969|3|11|19|4|1969|280|3647|Wednesday|1969Q4|N|N|N|2440527|2440830|2440180|2440453|N|N|N|N|N| +2440546|AAAAAAAACGNDFCAA|1969-11-20|838|3647|280|1969|4|11|20|4|1969|280|3647|Thursday|1969Q4|N|N|N|2440527|2440830|2440181|2440454|N|N|N|N|N| +2440547|AAAAAAAADGNDFCAA|1969-11-21|838|3647|280|1969|5|11|21|4|1969|280|3647|Friday|1969Q4|N|Y|N|2440527|2440830|2440182|2440455|N|N|N|N|N| +2440548|AAAAAAAAEGNDFCAA|1969-11-22|838|3647|280|1969|6|11|22|4|1969|280|3647|Saturday|1969Q4|N|Y|N|2440527|2440830|2440183|2440456|N|N|N|N|N| +2440549|AAAAAAAAFGNDFCAA|1969-11-23|838|3647|280|1969|0|11|23|4|1969|280|3647|Sunday|1969Q4|N|N|N|2440527|2440830|2440184|2440457|N|N|N|N|N| +2440550|AAAAAAAAGGNDFCAA|1969-11-24|838|3647|280|1969|1|11|24|4|1969|280|3647|Monday|1969Q4|N|N|N|2440527|2440830|2440185|2440458|N|N|N|N|N| +2440551|AAAAAAAAHGNDFCAA|1969-11-25|838|3648|280|1969|2|11|25|4|1969|280|3648|Tuesday|1969Q4|N|N|N|2440527|2440830|2440186|2440459|N|N|N|N|N| +2440552|AAAAAAAAIGNDFCAA|1969-11-26|838|3648|280|1969|3|11|26|4|1969|280|3648|Wednesday|1969Q4|N|N|N|2440527|2440830|2440187|2440460|N|N|N|N|N| +2440553|AAAAAAAAJGNDFCAA|1969-11-27|838|3648|280|1969|4|11|27|4|1969|280|3648|Thursday|1969Q4|N|N|N|2440527|2440830|2440188|2440461|N|N|N|N|N| +2440554|AAAAAAAAKGNDFCAA|1969-11-28|838|3648|280|1969|5|11|28|4|1969|280|3648|Friday|1969Q4|N|Y|N|2440527|2440830|2440189|2440462|N|N|N|N|N| +2440555|AAAAAAAALGNDFCAA|1969-11-29|838|3648|280|1969|6|11|29|4|1969|280|3648|Saturday|1969Q4|N|Y|N|2440527|2440830|2440190|2440463|N|N|N|N|N| +2440556|AAAAAAAAMGNDFCAA|1969-11-30|838|3648|280|1969|0|11|30|4|1969|280|3648|Sunday|1969Q4|N|N|N|2440527|2440830|2440191|2440464|N|N|N|N|N| +2440557|AAAAAAAANGNDFCAA|1969-12-01|839|3648|281|1969|1|12|1|4|1969|281|3648|Monday|1969Q4|N|N|N|2440557|2440890|2440192|2440465|N|N|N|N|N| +2440558|AAAAAAAAOGNDFCAA|1969-12-02|839|3649|281|1969|2|12|2|4|1969|281|3649|Tuesday|1969Q4|N|N|N|2440557|2440890|2440193|2440466|N|N|N|N|N| +2440559|AAAAAAAAPGNDFCAA|1969-12-03|839|3649|281|1969|3|12|3|4|1969|281|3649|Wednesday|1969Q4|N|N|N|2440557|2440890|2440194|2440467|N|N|N|N|N| +2440560|AAAAAAAAAHNDFCAA|1969-12-04|839|3649|281|1969|4|12|4|4|1969|281|3649|Thursday|1969Q4|N|N|N|2440557|2440890|2440195|2440468|N|N|N|N|N| +2440561|AAAAAAAABHNDFCAA|1969-12-05|839|3649|281|1969|5|12|5|4|1969|281|3649|Friday|1969Q4|N|Y|N|2440557|2440890|2440196|2440469|N|N|N|N|N| +2440562|AAAAAAAACHNDFCAA|1969-12-06|839|3649|281|1969|6|12|6|4|1969|281|3649|Saturday|1969Q4|N|Y|N|2440557|2440890|2440197|2440470|N|N|N|N|N| +2440563|AAAAAAAADHNDFCAA|1969-12-07|839|3649|281|1969|0|12|7|4|1969|281|3649|Sunday|1969Q4|N|N|N|2440557|2440890|2440198|2440471|N|N|N|N|N| +2440564|AAAAAAAAEHNDFCAA|1969-12-08|839|3649|281|1969|1|12|8|4|1969|281|3649|Monday|1969Q4|N|N|N|2440557|2440890|2440199|2440472|N|N|N|N|N| +2440565|AAAAAAAAFHNDFCAA|1969-12-09|839|3650|281|1969|2|12|9|4|1969|281|3650|Tuesday|1969Q4|N|N|N|2440557|2440890|2440200|2440473|N|N|N|N|N| +2440566|AAAAAAAAGHNDFCAA|1969-12-10|839|3650|281|1969|3|12|10|4|1969|281|3650|Wednesday|1969Q4|N|N|N|2440557|2440890|2440201|2440474|N|N|N|N|N| +2440567|AAAAAAAAHHNDFCAA|1969-12-11|839|3650|281|1969|4|12|11|4|1969|281|3650|Thursday|1969Q4|N|N|N|2440557|2440890|2440202|2440475|N|N|N|N|N| +2440568|AAAAAAAAIHNDFCAA|1969-12-12|839|3650|281|1969|5|12|12|4|1969|281|3650|Friday|1969Q4|N|Y|N|2440557|2440890|2440203|2440476|N|N|N|N|N| +2440569|AAAAAAAAJHNDFCAA|1969-12-13|839|3650|281|1969|6|12|13|4|1969|281|3650|Saturday|1969Q4|N|Y|N|2440557|2440890|2440204|2440477|N|N|N|N|N| +2440570|AAAAAAAAKHNDFCAA|1969-12-14|839|3650|281|1969|0|12|14|4|1969|281|3650|Sunday|1969Q4|N|N|N|2440557|2440890|2440205|2440478|N|N|N|N|N| +2440571|AAAAAAAALHNDFCAA|1969-12-15|839|3650|281|1969|1|12|15|4|1969|281|3650|Monday|1969Q4|N|N|N|2440557|2440890|2440206|2440479|N|N|N|N|N| +2440572|AAAAAAAAMHNDFCAA|1969-12-16|839|3651|281|1969|2|12|16|4|1969|281|3651|Tuesday|1969Q4|N|N|N|2440557|2440890|2440207|2440480|N|N|N|N|N| +2440573|AAAAAAAANHNDFCAA|1969-12-17|839|3651|281|1969|3|12|17|4|1969|281|3651|Wednesday|1969Q4|N|N|N|2440557|2440890|2440208|2440481|N|N|N|N|N| +2440574|AAAAAAAAOHNDFCAA|1969-12-18|839|3651|281|1969|4|12|18|4|1969|281|3651|Thursday|1969Q4|N|N|N|2440557|2440890|2440209|2440482|N|N|N|N|N| +2440575|AAAAAAAAPHNDFCAA|1969-12-19|839|3651|281|1969|5|12|19|4|1969|281|3651|Friday|1969Q4|N|Y|N|2440557|2440890|2440210|2440483|N|N|N|N|N| +2440576|AAAAAAAAAINDFCAA|1969-12-20|839|3651|281|1969|6|12|20|4|1969|281|3651|Saturday|1969Q4|N|Y|N|2440557|2440890|2440211|2440484|N|N|N|N|N| +2440577|AAAAAAAABINDFCAA|1969-12-21|839|3651|281|1969|0|12|21|4|1969|281|3651|Sunday|1969Q4|N|N|N|2440557|2440890|2440212|2440485|N|N|N|N|N| +2440578|AAAAAAAACINDFCAA|1969-12-22|839|3651|281|1969|1|12|22|4|1969|281|3651|Monday|1969Q4|N|N|N|2440557|2440890|2440213|2440486|N|N|N|N|N| +2440579|AAAAAAAADINDFCAA|1969-12-23|839|3652|281|1969|2|12|23|4|1969|281|3652|Tuesday|1969Q4|N|N|N|2440557|2440890|2440214|2440487|N|N|N|N|N| +2440580|AAAAAAAAEINDFCAA|1969-12-24|839|3652|281|1969|3|12|24|4|1969|281|3652|Wednesday|1969Q4|N|N|N|2440557|2440890|2440215|2440488|N|N|N|N|N| +2440581|AAAAAAAAFINDFCAA|1969-12-25|839|3652|281|1969|4|12|25|4|1969|281|3652|Thursday|1969Q4|N|N|N|2440557|2440890|2440216|2440489|N|N|N|N|N| +2440582|AAAAAAAAGINDFCAA|1969-12-26|839|3652|281|1969|5|12|26|4|1969|281|3652|Friday|1969Q4|Y|Y|N|2440557|2440890|2440217|2440490|N|N|N|N|N| +2440583|AAAAAAAAHINDFCAA|1969-12-27|839|3652|281|1969|6|12|27|4|1969|281|3652|Saturday|1969Q4|N|Y|Y|2440557|2440890|2440218|2440491|N|N|N|N|N| +2440584|AAAAAAAAIINDFCAA|1969-12-28|839|3652|281|1969|0|12|28|4|1969|281|3652|Sunday|1969Q4|N|N|N|2440557|2440890|2440219|2440492|N|N|N|N|N| +2440585|AAAAAAAAJINDFCAA|1969-12-29|839|3652|281|1969|1|12|29|4|1969|281|3652|Monday|1969Q4|N|N|N|2440557|2440890|2440220|2440493|N|N|N|N|N| +2440586|AAAAAAAAKINDFCAA|1969-12-30|839|3653|281|1969|2|12|30|4|1969|281|3653|Tuesday|1969Q4|N|N|N|2440557|2440890|2440221|2440494|N|N|N|N|N| +2440587|AAAAAAAALINDFCAA|1969-12-31|839|3653|281|1969|3|12|31|4|1969|281|3653|Wednesday|1969Q4|N|N|N|2440557|2440890|2440222|2440495|N|N|N|N|N| +2440588|AAAAAAAAMINDFCAA|1970-01-01|840|3653|281|1970|4|1|1|1|1970|281|3653|Thursday|1970Q1|Y|N|N|2440588|2440587|2440223|2440496|N|N|N|N|N| +2440589|AAAAAAAANINDFCAA|1970-01-02|840|3653|281|1970|5|1|2|1|1970|281|3653|Friday|1970Q1|N|Y|Y|2440588|2440587|2440224|2440497|N|N|N|N|N| +2440590|AAAAAAAAOINDFCAA|1970-01-03|840|3653|281|1970|6|1|3|1|1970|281|3653|Saturday|1970Q1|N|Y|N|2440588|2440587|2440225|2440498|N|N|N|N|N| +2440591|AAAAAAAAPINDFCAA|1970-01-04|840|3653|281|1970|0|1|4|1|1970|281|3653|Sunday|1970Q1|N|N|N|2440588|2440587|2440226|2440499|N|N|N|N|N| +2440592|AAAAAAAAAJNDFCAA|1970-01-05|840|3653|281|1970|1|1|5|1|1970|281|3653|Monday|1970Q1|N|N|N|2440588|2440587|2440227|2440500|N|N|N|N|N| +2440593|AAAAAAAABJNDFCAA|1970-01-06|840|3654|281|1970|2|1|6|1|1970|281|3654|Tuesday|1970Q1|N|N|N|2440588|2440587|2440228|2440501|N|N|N|N|N| +2440594|AAAAAAAACJNDFCAA|1970-01-07|840|3654|281|1970|3|1|7|1|1970|281|3654|Wednesday|1970Q1|N|N|N|2440588|2440587|2440229|2440502|N|N|N|N|N| +2440595|AAAAAAAADJNDFCAA|1970-01-08|840|3654|281|1970|4|1|8|1|1970|281|3654|Thursday|1970Q1|N|N|N|2440588|2440587|2440230|2440503|N|N|N|N|N| +2440596|AAAAAAAAEJNDFCAA|1970-01-09|840|3654|281|1970|5|1|9|1|1970|281|3654|Friday|1970Q1|N|Y|N|2440588|2440587|2440231|2440504|N|N|N|N|N| +2440597|AAAAAAAAFJNDFCAA|1970-01-10|840|3654|281|1970|6|1|10|1|1970|281|3654|Saturday|1970Q1|N|Y|N|2440588|2440587|2440232|2440505|N|N|N|N|N| +2440598|AAAAAAAAGJNDFCAA|1970-01-11|840|3654|281|1970|0|1|11|1|1970|281|3654|Sunday|1970Q1|N|N|N|2440588|2440587|2440233|2440506|N|N|N|N|N| +2440599|AAAAAAAAHJNDFCAA|1970-01-12|840|3654|281|1970|1|1|12|1|1970|281|3654|Monday|1970Q1|N|N|N|2440588|2440587|2440234|2440507|N|N|N|N|N| +2440600|AAAAAAAAIJNDFCAA|1970-01-13|840|3655|281|1970|2|1|13|1|1970|281|3655|Tuesday|1970Q1|N|N|N|2440588|2440587|2440235|2440508|N|N|N|N|N| +2440601|AAAAAAAAJJNDFCAA|1970-01-14|840|3655|281|1970|3|1|14|1|1970|281|3655|Wednesday|1970Q1|N|N|N|2440588|2440587|2440236|2440509|N|N|N|N|N| +2440602|AAAAAAAAKJNDFCAA|1970-01-15|840|3655|281|1970|4|1|15|1|1970|281|3655|Thursday|1970Q1|N|N|N|2440588|2440587|2440237|2440510|N|N|N|N|N| +2440603|AAAAAAAALJNDFCAA|1970-01-16|840|3655|281|1970|5|1|16|1|1970|281|3655|Friday|1970Q1|N|Y|N|2440588|2440587|2440238|2440511|N|N|N|N|N| +2440604|AAAAAAAAMJNDFCAA|1970-01-17|840|3655|281|1970|6|1|17|1|1970|281|3655|Saturday|1970Q1|N|Y|N|2440588|2440587|2440239|2440512|N|N|N|N|N| +2440605|AAAAAAAANJNDFCAA|1970-01-18|840|3655|281|1970|0|1|18|1|1970|281|3655|Sunday|1970Q1|N|N|N|2440588|2440587|2440240|2440513|N|N|N|N|N| +2440606|AAAAAAAAOJNDFCAA|1970-01-19|840|3655|281|1970|1|1|19|1|1970|281|3655|Monday|1970Q1|N|N|N|2440588|2440587|2440241|2440514|N|N|N|N|N| +2440607|AAAAAAAAPJNDFCAA|1970-01-20|840|3656|281|1970|2|1|20|1|1970|281|3656|Tuesday|1970Q1|N|N|N|2440588|2440587|2440242|2440515|N|N|N|N|N| +2440608|AAAAAAAAAKNDFCAA|1970-01-21|840|3656|281|1970|3|1|21|1|1970|281|3656|Wednesday|1970Q1|N|N|N|2440588|2440587|2440243|2440516|N|N|N|N|N| +2440609|AAAAAAAABKNDFCAA|1970-01-22|840|3656|281|1970|4|1|22|1|1970|281|3656|Thursday|1970Q1|N|N|N|2440588|2440587|2440244|2440517|N|N|N|N|N| +2440610|AAAAAAAACKNDFCAA|1970-01-23|840|3656|281|1970|5|1|23|1|1970|281|3656|Friday|1970Q1|N|Y|N|2440588|2440587|2440245|2440518|N|N|N|N|N| +2440611|AAAAAAAADKNDFCAA|1970-01-24|840|3656|281|1970|6|1|24|1|1970|281|3656|Saturday|1970Q1|N|Y|N|2440588|2440587|2440246|2440519|N|N|N|N|N| +2440612|AAAAAAAAEKNDFCAA|1970-01-25|840|3656|281|1970|0|1|25|1|1970|281|3656|Sunday|1970Q1|N|N|N|2440588|2440587|2440247|2440520|N|N|N|N|N| +2440613|AAAAAAAAFKNDFCAA|1970-01-26|840|3656|281|1970|1|1|26|1|1970|281|3656|Monday|1970Q1|N|N|N|2440588|2440587|2440248|2440521|N|N|N|N|N| +2440614|AAAAAAAAGKNDFCAA|1970-01-27|840|3657|281|1970|2|1|27|1|1970|281|3657|Tuesday|1970Q1|N|N|N|2440588|2440587|2440249|2440522|N|N|N|N|N| +2440615|AAAAAAAAHKNDFCAA|1970-01-28|840|3657|281|1970|3|1|28|1|1970|281|3657|Wednesday|1970Q1|N|N|N|2440588|2440587|2440250|2440523|N|N|N|N|N| +2440616|AAAAAAAAIKNDFCAA|1970-01-29|840|3657|281|1970|4|1|29|1|1970|281|3657|Thursday|1970Q1|N|N|N|2440588|2440587|2440251|2440524|N|N|N|N|N| +2440617|AAAAAAAAJKNDFCAA|1970-01-30|840|3657|281|1970|5|1|30|1|1970|281|3657|Friday|1970Q1|N|Y|N|2440588|2440587|2440252|2440525|N|N|N|N|N| +2440618|AAAAAAAAKKNDFCAA|1970-01-31|840|3657|281|1970|6|1|31|1|1970|281|3657|Saturday|1970Q1|N|Y|N|2440588|2440587|2440253|2440526|N|N|N|N|N| +2440619|AAAAAAAALKNDFCAA|1970-02-01|841|3657|281|1970|0|2|1|1|1970|281|3657|Sunday|1970Q1|N|N|N|2440619|2440649|2440254|2440527|N|N|N|N|N| +2440620|AAAAAAAAMKNDFCAA|1970-02-02|841|3657|281|1970|1|2|2|1|1970|281|3657|Monday|1970Q1|N|N|N|2440619|2440649|2440255|2440528|N|N|N|N|N| +2440621|AAAAAAAANKNDFCAA|1970-02-03|841|3658|281|1970|2|2|3|1|1970|281|3658|Tuesday|1970Q1|N|N|N|2440619|2440649|2440256|2440529|N|N|N|N|N| +2440622|AAAAAAAAOKNDFCAA|1970-02-04|841|3658|281|1970|3|2|4|1|1970|281|3658|Wednesday|1970Q1|N|N|N|2440619|2440649|2440257|2440530|N|N|N|N|N| +2440623|AAAAAAAAPKNDFCAA|1970-02-05|841|3658|281|1970|4|2|5|1|1970|281|3658|Thursday|1970Q1|N|N|N|2440619|2440649|2440258|2440531|N|N|N|N|N| +2440624|AAAAAAAAALNDFCAA|1970-02-06|841|3658|281|1970|5|2|6|1|1970|281|3658|Friday|1970Q1|N|Y|N|2440619|2440649|2440259|2440532|N|N|N|N|N| +2440625|AAAAAAAABLNDFCAA|1970-02-07|841|3658|281|1970|6|2|7|1|1970|281|3658|Saturday|1970Q1|N|Y|N|2440619|2440649|2440260|2440533|N|N|N|N|N| +2440626|AAAAAAAACLNDFCAA|1970-02-08|841|3658|281|1970|0|2|8|1|1970|281|3658|Sunday|1970Q1|N|N|N|2440619|2440649|2440261|2440534|N|N|N|N|N| +2440627|AAAAAAAADLNDFCAA|1970-02-09|841|3658|281|1970|1|2|9|1|1970|281|3658|Monday|1970Q1|N|N|N|2440619|2440649|2440262|2440535|N|N|N|N|N| +2440628|AAAAAAAAELNDFCAA|1970-02-10|841|3659|281|1970|2|2|10|1|1970|281|3659|Tuesday|1970Q1|N|N|N|2440619|2440649|2440263|2440536|N|N|N|N|N| +2440629|AAAAAAAAFLNDFCAA|1970-02-11|841|3659|281|1970|3|2|11|1|1970|281|3659|Wednesday|1970Q1|N|N|N|2440619|2440649|2440264|2440537|N|N|N|N|N| +2440630|AAAAAAAAGLNDFCAA|1970-02-12|841|3659|281|1970|4|2|12|1|1970|281|3659|Thursday|1970Q1|N|N|N|2440619|2440649|2440265|2440538|N|N|N|N|N| +2440631|AAAAAAAAHLNDFCAA|1970-02-13|841|3659|281|1970|5|2|13|1|1970|281|3659|Friday|1970Q1|N|Y|N|2440619|2440649|2440266|2440539|N|N|N|N|N| +2440632|AAAAAAAAILNDFCAA|1970-02-14|841|3659|281|1970|6|2|14|1|1970|281|3659|Saturday|1970Q1|N|Y|N|2440619|2440649|2440267|2440540|N|N|N|N|N| +2440633|AAAAAAAAJLNDFCAA|1970-02-15|841|3659|281|1970|0|2|15|1|1970|281|3659|Sunday|1970Q1|N|N|N|2440619|2440649|2440268|2440541|N|N|N|N|N| +2440634|AAAAAAAAKLNDFCAA|1970-02-16|841|3659|281|1970|1|2|16|1|1970|281|3659|Monday|1970Q1|N|N|N|2440619|2440649|2440269|2440542|N|N|N|N|N| +2440635|AAAAAAAALLNDFCAA|1970-02-17|841|3660|281|1970|2|2|17|1|1970|281|3660|Tuesday|1970Q1|N|N|N|2440619|2440649|2440270|2440543|N|N|N|N|N| +2440636|AAAAAAAAMLNDFCAA|1970-02-18|841|3660|281|1970|3|2|18|1|1970|281|3660|Wednesday|1970Q1|N|N|N|2440619|2440649|2440271|2440544|N|N|N|N|N| +2440637|AAAAAAAANLNDFCAA|1970-02-19|841|3660|281|1970|4|2|19|1|1970|281|3660|Thursday|1970Q1|N|N|N|2440619|2440649|2440272|2440545|N|N|N|N|N| +2440638|AAAAAAAAOLNDFCAA|1970-02-20|841|3660|281|1970|5|2|20|1|1970|281|3660|Friday|1970Q1|N|Y|N|2440619|2440649|2440273|2440546|N|N|N|N|N| +2440639|AAAAAAAAPLNDFCAA|1970-02-21|841|3660|281|1970|6|2|21|1|1970|281|3660|Saturday|1970Q1|N|Y|N|2440619|2440649|2440274|2440547|N|N|N|N|N| +2440640|AAAAAAAAAMNDFCAA|1970-02-22|841|3660|281|1970|0|2|22|1|1970|281|3660|Sunday|1970Q1|N|N|N|2440619|2440649|2440275|2440548|N|N|N|N|N| +2440641|AAAAAAAABMNDFCAA|1970-02-23|841|3660|281|1970|1|2|23|1|1970|281|3660|Monday|1970Q1|N|N|N|2440619|2440649|2440276|2440549|N|N|N|N|N| +2440642|AAAAAAAACMNDFCAA|1970-02-24|841|3661|281|1970|2|2|24|1|1970|281|3661|Tuesday|1970Q1|N|N|N|2440619|2440649|2440277|2440550|N|N|N|N|N| +2440643|AAAAAAAADMNDFCAA|1970-02-25|841|3661|281|1970|3|2|25|1|1970|281|3661|Wednesday|1970Q1|N|N|N|2440619|2440649|2440278|2440551|N|N|N|N|N| +2440644|AAAAAAAAEMNDFCAA|1970-02-26|841|3661|281|1970|4|2|26|1|1970|281|3661|Thursday|1970Q1|N|N|N|2440619|2440649|2440279|2440552|N|N|N|N|N| +2440645|AAAAAAAAFMNDFCAA|1970-02-27|841|3661|281|1970|5|2|27|1|1970|281|3661|Friday|1970Q1|N|Y|N|2440619|2440649|2440280|2440553|N|N|N|N|N| +2440646|AAAAAAAAGMNDFCAA|1970-02-28|841|3661|281|1970|6|2|28|1|1970|281|3661|Saturday|1970Q1|N|Y|N|2440619|2440649|2440281|2440554|N|N|N|N|N| +2440647|AAAAAAAAHMNDFCAA|1970-03-01|842|3661|282|1970|0|3|1|1|1970|282|3661|Sunday|1970Q1|N|N|N|2440647|2440705|2440282|2440555|N|N|N|N|N| +2440648|AAAAAAAAIMNDFCAA|1970-03-02|842|3661|282|1970|1|3|2|1|1970|282|3661|Monday|1970Q1|N|N|N|2440647|2440705|2440283|2440556|N|N|N|N|N| +2440649|AAAAAAAAJMNDFCAA|1970-03-03|842|3662|282|1970|2|3|3|1|1970|282|3662|Tuesday|1970Q1|N|N|N|2440647|2440705|2440284|2440557|N|N|N|N|N| +2440650|AAAAAAAAKMNDFCAA|1970-03-04|842|3662|282|1970|3|3|4|1|1970|282|3662|Wednesday|1970Q1|N|N|N|2440647|2440705|2440285|2440558|N|N|N|N|N| +2440651|AAAAAAAALMNDFCAA|1970-03-05|842|3662|282|1970|4|3|5|1|1970|282|3662|Thursday|1970Q1|N|N|N|2440647|2440705|2440286|2440559|N|N|N|N|N| +2440652|AAAAAAAAMMNDFCAA|1970-03-06|842|3662|282|1970|5|3|6|1|1970|282|3662|Friday|1970Q1|N|Y|N|2440647|2440705|2440287|2440560|N|N|N|N|N| +2440653|AAAAAAAANMNDFCAA|1970-03-07|842|3662|282|1970|6|3|7|1|1970|282|3662|Saturday|1970Q1|N|Y|N|2440647|2440705|2440288|2440561|N|N|N|N|N| +2440654|AAAAAAAAOMNDFCAA|1970-03-08|842|3662|282|1970|0|3|8|1|1970|282|3662|Sunday|1970Q1|N|N|N|2440647|2440705|2440289|2440562|N|N|N|N|N| +2440655|AAAAAAAAPMNDFCAA|1970-03-09|842|3662|282|1970|1|3|9|1|1970|282|3662|Monday|1970Q1|N|N|N|2440647|2440705|2440290|2440563|N|N|N|N|N| +2440656|AAAAAAAAANNDFCAA|1970-03-10|842|3663|282|1970|2|3|10|1|1970|282|3663|Tuesday|1970Q1|N|N|N|2440647|2440705|2440291|2440564|N|N|N|N|N| +2440657|AAAAAAAABNNDFCAA|1970-03-11|842|3663|282|1970|3|3|11|1|1970|282|3663|Wednesday|1970Q1|N|N|N|2440647|2440705|2440292|2440565|N|N|N|N|N| +2440658|AAAAAAAACNNDFCAA|1970-03-12|842|3663|282|1970|4|3|12|1|1970|282|3663|Thursday|1970Q1|N|N|N|2440647|2440705|2440293|2440566|N|N|N|N|N| +2440659|AAAAAAAADNNDFCAA|1970-03-13|842|3663|282|1970|5|3|13|1|1970|282|3663|Friday|1970Q1|N|Y|N|2440647|2440705|2440294|2440567|N|N|N|N|N| +2440660|AAAAAAAAENNDFCAA|1970-03-14|842|3663|282|1970|6|3|14|1|1970|282|3663|Saturday|1970Q1|N|Y|N|2440647|2440705|2440295|2440568|N|N|N|N|N| +2440661|AAAAAAAAFNNDFCAA|1970-03-15|842|3663|282|1970|0|3|15|1|1970|282|3663|Sunday|1970Q1|N|N|N|2440647|2440705|2440296|2440569|N|N|N|N|N| +2440662|AAAAAAAAGNNDFCAA|1970-03-16|842|3663|282|1970|1|3|16|1|1970|282|3663|Monday|1970Q1|N|N|N|2440647|2440705|2440297|2440570|N|N|N|N|N| +2440663|AAAAAAAAHNNDFCAA|1970-03-17|842|3664|282|1970|2|3|17|1|1970|282|3664|Tuesday|1970Q1|N|N|N|2440647|2440705|2440298|2440571|N|N|N|N|N| +2440664|AAAAAAAAINNDFCAA|1970-03-18|842|3664|282|1970|3|3|18|1|1970|282|3664|Wednesday|1970Q1|N|N|N|2440647|2440705|2440299|2440572|N|N|N|N|N| +2440665|AAAAAAAAJNNDFCAA|1970-03-19|842|3664|282|1970|4|3|19|1|1970|282|3664|Thursday|1970Q1|N|N|N|2440647|2440705|2440300|2440573|N|N|N|N|N| +2440666|AAAAAAAAKNNDFCAA|1970-03-20|842|3664|282|1970|5|3|20|1|1970|282|3664|Friday|1970Q1|N|Y|N|2440647|2440705|2440301|2440574|N|N|N|N|N| +2440667|AAAAAAAALNNDFCAA|1970-03-21|842|3664|282|1970|6|3|21|1|1970|282|3664|Saturday|1970Q1|N|Y|N|2440647|2440705|2440302|2440575|N|N|N|N|N| +2440668|AAAAAAAAMNNDFCAA|1970-03-22|842|3664|282|1970|0|3|22|1|1970|282|3664|Sunday|1970Q1|N|N|N|2440647|2440705|2440303|2440576|N|N|N|N|N| +2440669|AAAAAAAANNNDFCAA|1970-03-23|842|3664|282|1970|1|3|23|1|1970|282|3664|Monday|1970Q1|N|N|N|2440647|2440705|2440304|2440577|N|N|N|N|N| +2440670|AAAAAAAAONNDFCAA|1970-03-24|842|3665|282|1970|2|3|24|1|1970|282|3665|Tuesday|1970Q1|N|N|N|2440647|2440705|2440305|2440578|N|N|N|N|N| +2440671|AAAAAAAAPNNDFCAA|1970-03-25|842|3665|282|1970|3|3|25|1|1970|282|3665|Wednesday|1970Q1|N|N|N|2440647|2440705|2440306|2440579|N|N|N|N|N| +2440672|AAAAAAAAAONDFCAA|1970-03-26|842|3665|282|1970|4|3|26|1|1970|282|3665|Thursday|1970Q1|N|N|N|2440647|2440705|2440307|2440580|N|N|N|N|N| +2440673|AAAAAAAABONDFCAA|1970-03-27|842|3665|282|1970|5|3|27|1|1970|282|3665|Friday|1970Q1|N|Y|N|2440647|2440705|2440308|2440581|N|N|N|N|N| +2440674|AAAAAAAACONDFCAA|1970-03-28|842|3665|282|1970|6|3|28|1|1970|282|3665|Saturday|1970Q1|N|Y|N|2440647|2440705|2440309|2440582|N|N|N|N|N| +2440675|AAAAAAAADONDFCAA|1970-03-29|842|3665|282|1970|0|3|29|1|1970|282|3665|Sunday|1970Q1|N|N|N|2440647|2440705|2440310|2440583|N|N|N|N|N| +2440676|AAAAAAAAEONDFCAA|1970-03-30|842|3665|282|1970|1|3|30|1|1970|282|3665|Monday|1970Q1|N|N|N|2440647|2440705|2440311|2440584|N|N|N|N|N| +2440677|AAAAAAAAFONDFCAA|1970-03-31|842|3666|282|1970|2|3|31|1|1970|282|3666|Tuesday|1970Q1|N|N|N|2440647|2440705|2440312|2440585|N|N|N|N|N| +2440678|AAAAAAAAGONDFCAA|1970-04-01|843|3666|282|1970|3|4|1|1|1970|282|3666|Wednesday|1970Q1|N|N|N|2440678|2440767|2440313|2440588|N|N|N|N|N| +2440679|AAAAAAAAHONDFCAA|1970-04-02|843|3666|282|1970|4|4|2|2|1970|282|3666|Thursday|1970Q2|N|N|N|2440678|2440767|2440314|2440589|N|N|N|N|N| +2440680|AAAAAAAAIONDFCAA|1970-04-03|843|3666|282|1970|5|4|3|2|1970|282|3666|Friday|1970Q2|N|Y|N|2440678|2440767|2440315|2440590|N|N|N|N|N| +2440681|AAAAAAAAJONDFCAA|1970-04-04|843|3666|282|1970|6|4|4|2|1970|282|3666|Saturday|1970Q2|N|Y|N|2440678|2440767|2440316|2440591|N|N|N|N|N| +2440682|AAAAAAAAKONDFCAA|1970-04-05|843|3666|282|1970|0|4|5|2|1970|282|3666|Sunday|1970Q2|N|N|N|2440678|2440767|2440317|2440592|N|N|N|N|N| +2440683|AAAAAAAALONDFCAA|1970-04-06|843|3666|282|1970|1|4|6|2|1970|282|3666|Monday|1970Q2|N|N|N|2440678|2440767|2440318|2440593|N|N|N|N|N| +2440684|AAAAAAAAMONDFCAA|1970-04-07|843|3667|282|1970|2|4|7|2|1970|282|3667|Tuesday|1970Q2|N|N|N|2440678|2440767|2440319|2440594|N|N|N|N|N| +2440685|AAAAAAAANONDFCAA|1970-04-08|843|3667|282|1970|3|4|8|2|1970|282|3667|Wednesday|1970Q2|N|N|N|2440678|2440767|2440320|2440595|N|N|N|N|N| +2440686|AAAAAAAAOONDFCAA|1970-04-09|843|3667|282|1970|4|4|9|2|1970|282|3667|Thursday|1970Q2|N|N|N|2440678|2440767|2440321|2440596|N|N|N|N|N| +2440687|AAAAAAAAPONDFCAA|1970-04-10|843|3667|282|1970|5|4|10|2|1970|282|3667|Friday|1970Q2|N|Y|N|2440678|2440767|2440322|2440597|N|N|N|N|N| +2440688|AAAAAAAAAPNDFCAA|1970-04-11|843|3667|282|1970|6|4|11|2|1970|282|3667|Saturday|1970Q2|N|Y|N|2440678|2440767|2440323|2440598|N|N|N|N|N| +2440689|AAAAAAAABPNDFCAA|1970-04-12|843|3667|282|1970|0|4|12|2|1970|282|3667|Sunday|1970Q2|N|N|N|2440678|2440767|2440324|2440599|N|N|N|N|N| +2440690|AAAAAAAACPNDFCAA|1970-04-13|843|3667|282|1970|1|4|13|2|1970|282|3667|Monday|1970Q2|N|N|N|2440678|2440767|2440325|2440600|N|N|N|N|N| +2440691|AAAAAAAADPNDFCAA|1970-04-14|843|3668|282|1970|2|4|14|2|1970|282|3668|Tuesday|1970Q2|N|N|N|2440678|2440767|2440326|2440601|N|N|N|N|N| +2440692|AAAAAAAAEPNDFCAA|1970-04-15|843|3668|282|1970|3|4|15|2|1970|282|3668|Wednesday|1970Q2|N|N|N|2440678|2440767|2440327|2440602|N|N|N|N|N| +2440693|AAAAAAAAFPNDFCAA|1970-04-16|843|3668|282|1970|4|4|16|2|1970|282|3668|Thursday|1970Q2|N|N|N|2440678|2440767|2440328|2440603|N|N|N|N|N| +2440694|AAAAAAAAGPNDFCAA|1970-04-17|843|3668|282|1970|5|4|17|2|1970|282|3668|Friday|1970Q2|N|Y|N|2440678|2440767|2440329|2440604|N|N|N|N|N| +2440695|AAAAAAAAHPNDFCAA|1970-04-18|843|3668|282|1970|6|4|18|2|1970|282|3668|Saturday|1970Q2|N|Y|N|2440678|2440767|2440330|2440605|N|N|N|N|N| +2440696|AAAAAAAAIPNDFCAA|1970-04-19|843|3668|282|1970|0|4|19|2|1970|282|3668|Sunday|1970Q2|N|N|N|2440678|2440767|2440331|2440606|N|N|N|N|N| +2440697|AAAAAAAAJPNDFCAA|1970-04-20|843|3668|282|1970|1|4|20|2|1970|282|3668|Monday|1970Q2|N|N|N|2440678|2440767|2440332|2440607|N|N|N|N|N| +2440698|AAAAAAAAKPNDFCAA|1970-04-21|843|3669|282|1970|2|4|21|2|1970|282|3669|Tuesday|1970Q2|N|N|N|2440678|2440767|2440333|2440608|N|N|N|N|N| +2440699|AAAAAAAALPNDFCAA|1970-04-22|843|3669|282|1970|3|4|22|2|1970|282|3669|Wednesday|1970Q2|N|N|N|2440678|2440767|2440334|2440609|N|N|N|N|N| +2440700|AAAAAAAAMPNDFCAA|1970-04-23|843|3669|282|1970|4|4|23|2|1970|282|3669|Thursday|1970Q2|N|N|N|2440678|2440767|2440335|2440610|N|N|N|N|N| +2440701|AAAAAAAANPNDFCAA|1970-04-24|843|3669|282|1970|5|4|24|2|1970|282|3669|Friday|1970Q2|N|Y|N|2440678|2440767|2440336|2440611|N|N|N|N|N| +2440702|AAAAAAAAOPNDFCAA|1970-04-25|843|3669|282|1970|6|4|25|2|1970|282|3669|Saturday|1970Q2|N|Y|N|2440678|2440767|2440337|2440612|N|N|N|N|N| +2440703|AAAAAAAAPPNDFCAA|1970-04-26|843|3669|282|1970|0|4|26|2|1970|282|3669|Sunday|1970Q2|N|N|N|2440678|2440767|2440338|2440613|N|N|N|N|N| +2440704|AAAAAAAAAAODFCAA|1970-04-27|843|3669|282|1970|1|4|27|2|1970|282|3669|Monday|1970Q2|N|N|N|2440678|2440767|2440339|2440614|N|N|N|N|N| +2440705|AAAAAAAABAODFCAA|1970-04-28|843|3670|282|1970|2|4|28|2|1970|282|3670|Tuesday|1970Q2|N|N|N|2440678|2440767|2440340|2440615|N|N|N|N|N| +2440706|AAAAAAAACAODFCAA|1970-04-29|843|3670|282|1970|3|4|29|2|1970|282|3670|Wednesday|1970Q2|N|N|N|2440678|2440767|2440341|2440616|N|N|N|N|N| +2440707|AAAAAAAADAODFCAA|1970-04-30|843|3670|282|1970|4|4|30|2|1970|282|3670|Thursday|1970Q2|N|N|N|2440678|2440767|2440342|2440617|N|N|N|N|N| +2440708|AAAAAAAAEAODFCAA|1970-05-01|844|3670|282|1970|5|5|1|2|1970|282|3670|Friday|1970Q2|N|Y|N|2440708|2440827|2440343|2440618|N|N|N|N|N| +2440709|AAAAAAAAFAODFCAA|1970-05-02|844|3670|282|1970|6|5|2|2|1970|282|3670|Saturday|1970Q2|N|Y|N|2440708|2440827|2440344|2440619|N|N|N|N|N| +2440710|AAAAAAAAGAODFCAA|1970-05-03|844|3670|282|1970|0|5|3|2|1970|282|3670|Sunday|1970Q2|N|N|N|2440708|2440827|2440345|2440620|N|N|N|N|N| +2440711|AAAAAAAAHAODFCAA|1970-05-04|844|3670|282|1970|1|5|4|2|1970|282|3670|Monday|1970Q2|N|N|N|2440708|2440827|2440346|2440621|N|N|N|N|N| +2440712|AAAAAAAAIAODFCAA|1970-05-05|844|3671|282|1970|2|5|5|2|1970|282|3671|Tuesday|1970Q2|N|N|N|2440708|2440827|2440347|2440622|N|N|N|N|N| +2440713|AAAAAAAAJAODFCAA|1970-05-06|844|3671|282|1970|3|5|6|2|1970|282|3671|Wednesday|1970Q2|N|N|N|2440708|2440827|2440348|2440623|N|N|N|N|N| +2440714|AAAAAAAAKAODFCAA|1970-05-07|844|3671|282|1970|4|5|7|2|1970|282|3671|Thursday|1970Q2|N|N|N|2440708|2440827|2440349|2440624|N|N|N|N|N| +2440715|AAAAAAAALAODFCAA|1970-05-08|844|3671|282|1970|5|5|8|2|1970|282|3671|Friday|1970Q2|N|Y|N|2440708|2440827|2440350|2440625|N|N|N|N|N| +2440716|AAAAAAAAMAODFCAA|1970-05-09|844|3671|282|1970|6|5|9|2|1970|282|3671|Saturday|1970Q2|N|Y|N|2440708|2440827|2440351|2440626|N|N|N|N|N| +2440717|AAAAAAAANAODFCAA|1970-05-10|844|3671|282|1970|0|5|10|2|1970|282|3671|Sunday|1970Q2|N|N|N|2440708|2440827|2440352|2440627|N|N|N|N|N| +2440718|AAAAAAAAOAODFCAA|1970-05-11|844|3671|282|1970|1|5|11|2|1970|282|3671|Monday|1970Q2|N|N|N|2440708|2440827|2440353|2440628|N|N|N|N|N| +2440719|AAAAAAAAPAODFCAA|1970-05-12|844|3672|282|1970|2|5|12|2|1970|282|3672|Tuesday|1970Q2|N|N|N|2440708|2440827|2440354|2440629|N|N|N|N|N| +2440720|AAAAAAAAABODFCAA|1970-05-13|844|3672|282|1970|3|5|13|2|1970|282|3672|Wednesday|1970Q2|N|N|N|2440708|2440827|2440355|2440630|N|N|N|N|N| +2440721|AAAAAAAABBODFCAA|1970-05-14|844|3672|282|1970|4|5|14|2|1970|282|3672|Thursday|1970Q2|N|N|N|2440708|2440827|2440356|2440631|N|N|N|N|N| +2440722|AAAAAAAACBODFCAA|1970-05-15|844|3672|282|1970|5|5|15|2|1970|282|3672|Friday|1970Q2|N|Y|N|2440708|2440827|2440357|2440632|N|N|N|N|N| +2440723|AAAAAAAADBODFCAA|1970-05-16|844|3672|282|1970|6|5|16|2|1970|282|3672|Saturday|1970Q2|N|Y|N|2440708|2440827|2440358|2440633|N|N|N|N|N| +2440724|AAAAAAAAEBODFCAA|1970-05-17|844|3672|282|1970|0|5|17|2|1970|282|3672|Sunday|1970Q2|N|N|N|2440708|2440827|2440359|2440634|N|N|N|N|N| +2440725|AAAAAAAAFBODFCAA|1970-05-18|844|3672|282|1970|1|5|18|2|1970|282|3672|Monday|1970Q2|N|N|N|2440708|2440827|2440360|2440635|N|N|N|N|N| +2440726|AAAAAAAAGBODFCAA|1970-05-19|844|3673|282|1970|2|5|19|2|1970|282|3673|Tuesday|1970Q2|N|N|N|2440708|2440827|2440361|2440636|N|N|N|N|N| +2440727|AAAAAAAAHBODFCAA|1970-05-20|844|3673|282|1970|3|5|20|2|1970|282|3673|Wednesday|1970Q2|N|N|N|2440708|2440827|2440362|2440637|N|N|N|N|N| +2440728|AAAAAAAAIBODFCAA|1970-05-21|844|3673|282|1970|4|5|21|2|1970|282|3673|Thursday|1970Q2|N|N|N|2440708|2440827|2440363|2440638|N|N|N|N|N| +2440729|AAAAAAAAJBODFCAA|1970-05-22|844|3673|282|1970|5|5|22|2|1970|282|3673|Friday|1970Q2|N|Y|N|2440708|2440827|2440364|2440639|N|N|N|N|N| +2440730|AAAAAAAAKBODFCAA|1970-05-23|844|3673|282|1970|6|5|23|2|1970|282|3673|Saturday|1970Q2|N|Y|N|2440708|2440827|2440365|2440640|N|N|N|N|N| +2440731|AAAAAAAALBODFCAA|1970-05-24|844|3673|282|1970|0|5|24|2|1970|282|3673|Sunday|1970Q2|N|N|N|2440708|2440827|2440366|2440641|N|N|N|N|N| +2440732|AAAAAAAAMBODFCAA|1970-05-25|844|3673|282|1970|1|5|25|2|1970|282|3673|Monday|1970Q2|N|N|N|2440708|2440827|2440367|2440642|N|N|N|N|N| +2440733|AAAAAAAANBODFCAA|1970-05-26|844|3674|282|1970|2|5|26|2|1970|282|3674|Tuesday|1970Q2|N|N|N|2440708|2440827|2440368|2440643|N|N|N|N|N| +2440734|AAAAAAAAOBODFCAA|1970-05-27|844|3674|282|1970|3|5|27|2|1970|282|3674|Wednesday|1970Q2|N|N|N|2440708|2440827|2440369|2440644|N|N|N|N|N| +2440735|AAAAAAAAPBODFCAA|1970-05-28|844|3674|282|1970|4|5|28|2|1970|282|3674|Thursday|1970Q2|N|N|N|2440708|2440827|2440370|2440645|N|N|N|N|N| +2440736|AAAAAAAAACODFCAA|1970-05-29|844|3674|282|1970|5|5|29|2|1970|282|3674|Friday|1970Q2|N|Y|N|2440708|2440827|2440371|2440646|N|N|N|N|N| +2440737|AAAAAAAABCODFCAA|1970-05-30|844|3674|282|1970|6|5|30|2|1970|282|3674|Saturday|1970Q2|N|Y|N|2440708|2440827|2440372|2440647|N|N|N|N|N| +2440738|AAAAAAAACCODFCAA|1970-05-31|844|3674|282|1970|0|5|31|2|1970|282|3674|Sunday|1970Q2|N|N|N|2440708|2440827|2440373|2440648|N|N|N|N|N| +2440739|AAAAAAAADCODFCAA|1970-06-01|845|3674|283|1970|1|6|1|2|1970|283|3674|Monday|1970Q2|N|N|N|2440739|2440889|2440374|2440649|N|N|N|N|N| +2440740|AAAAAAAAECODFCAA|1970-06-02|845|3675|283|1970|2|6|2|2|1970|283|3675|Tuesday|1970Q2|N|N|N|2440739|2440889|2440375|2440650|N|N|N|N|N| +2440741|AAAAAAAAFCODFCAA|1970-06-03|845|3675|283|1970|3|6|3|2|1970|283|3675|Wednesday|1970Q2|N|N|N|2440739|2440889|2440376|2440651|N|N|N|N|N| +2440742|AAAAAAAAGCODFCAA|1970-06-04|845|3675|283|1970|4|6|4|2|1970|283|3675|Thursday|1970Q2|N|N|N|2440739|2440889|2440377|2440652|N|N|N|N|N| +2440743|AAAAAAAAHCODFCAA|1970-06-05|845|3675|283|1970|5|6|5|2|1970|283|3675|Friday|1970Q2|N|Y|N|2440739|2440889|2440378|2440653|N|N|N|N|N| +2440744|AAAAAAAAICODFCAA|1970-06-06|845|3675|283|1970|6|6|6|2|1970|283|3675|Saturday|1970Q2|N|Y|N|2440739|2440889|2440379|2440654|N|N|N|N|N| +2440745|AAAAAAAAJCODFCAA|1970-06-07|845|3675|283|1970|0|6|7|2|1970|283|3675|Sunday|1970Q2|N|N|N|2440739|2440889|2440380|2440655|N|N|N|N|N| +2440746|AAAAAAAAKCODFCAA|1970-06-08|845|3675|283|1970|1|6|8|2|1970|283|3675|Monday|1970Q2|N|N|N|2440739|2440889|2440381|2440656|N|N|N|N|N| +2440747|AAAAAAAALCODFCAA|1970-06-09|845|3676|283|1970|2|6|9|2|1970|283|3676|Tuesday|1970Q2|N|N|N|2440739|2440889|2440382|2440657|N|N|N|N|N| +2440748|AAAAAAAAMCODFCAA|1970-06-10|845|3676|283|1970|3|6|10|2|1970|283|3676|Wednesday|1970Q2|N|N|N|2440739|2440889|2440383|2440658|N|N|N|N|N| +2440749|AAAAAAAANCODFCAA|1970-06-11|845|3676|283|1970|4|6|11|2|1970|283|3676|Thursday|1970Q2|N|N|N|2440739|2440889|2440384|2440659|N|N|N|N|N| +2440750|AAAAAAAAOCODFCAA|1970-06-12|845|3676|283|1970|5|6|12|2|1970|283|3676|Friday|1970Q2|N|Y|N|2440739|2440889|2440385|2440660|N|N|N|N|N| +2440751|AAAAAAAAPCODFCAA|1970-06-13|845|3676|283|1970|6|6|13|2|1970|283|3676|Saturday|1970Q2|N|Y|N|2440739|2440889|2440386|2440661|N|N|N|N|N| +2440752|AAAAAAAAADODFCAA|1970-06-14|845|3676|283|1970|0|6|14|2|1970|283|3676|Sunday|1970Q2|N|N|N|2440739|2440889|2440387|2440662|N|N|N|N|N| +2440753|AAAAAAAABDODFCAA|1970-06-15|845|3676|283|1970|1|6|15|2|1970|283|3676|Monday|1970Q2|N|N|N|2440739|2440889|2440388|2440663|N|N|N|N|N| +2440754|AAAAAAAACDODFCAA|1970-06-16|845|3677|283|1970|2|6|16|2|1970|283|3677|Tuesday|1970Q2|N|N|N|2440739|2440889|2440389|2440664|N|N|N|N|N| +2440755|AAAAAAAADDODFCAA|1970-06-17|845|3677|283|1970|3|6|17|2|1970|283|3677|Wednesday|1970Q2|N|N|N|2440739|2440889|2440390|2440665|N|N|N|N|N| +2440756|AAAAAAAAEDODFCAA|1970-06-18|845|3677|283|1970|4|6|18|2|1970|283|3677|Thursday|1970Q2|N|N|N|2440739|2440889|2440391|2440666|N|N|N|N|N| +2440757|AAAAAAAAFDODFCAA|1970-06-19|845|3677|283|1970|5|6|19|2|1970|283|3677|Friday|1970Q2|N|Y|N|2440739|2440889|2440392|2440667|N|N|N|N|N| +2440758|AAAAAAAAGDODFCAA|1970-06-20|845|3677|283|1970|6|6|20|2|1970|283|3677|Saturday|1970Q2|N|Y|N|2440739|2440889|2440393|2440668|N|N|N|N|N| +2440759|AAAAAAAAHDODFCAA|1970-06-21|845|3677|283|1970|0|6|21|2|1970|283|3677|Sunday|1970Q2|N|N|N|2440739|2440889|2440394|2440669|N|N|N|N|N| +2440760|AAAAAAAAIDODFCAA|1970-06-22|845|3677|283|1970|1|6|22|2|1970|283|3677|Monday|1970Q2|N|N|N|2440739|2440889|2440395|2440670|N|N|N|N|N| +2440761|AAAAAAAAJDODFCAA|1970-06-23|845|3678|283|1970|2|6|23|2|1970|283|3678|Tuesday|1970Q2|N|N|N|2440739|2440889|2440396|2440671|N|N|N|N|N| +2440762|AAAAAAAAKDODFCAA|1970-06-24|845|3678|283|1970|3|6|24|2|1970|283|3678|Wednesday|1970Q2|N|N|N|2440739|2440889|2440397|2440672|N|N|N|N|N| +2440763|AAAAAAAALDODFCAA|1970-06-25|845|3678|283|1970|4|6|25|2|1970|283|3678|Thursday|1970Q2|N|N|N|2440739|2440889|2440398|2440673|N|N|N|N|N| +2440764|AAAAAAAAMDODFCAA|1970-06-26|845|3678|283|1970|5|6|26|2|1970|283|3678|Friday|1970Q2|N|Y|N|2440739|2440889|2440399|2440674|N|N|N|N|N| +2440765|AAAAAAAANDODFCAA|1970-06-27|845|3678|283|1970|6|6|27|2|1970|283|3678|Saturday|1970Q2|N|Y|N|2440739|2440889|2440400|2440675|N|N|N|N|N| +2440766|AAAAAAAAODODFCAA|1970-06-28|845|3678|283|1970|0|6|28|2|1970|283|3678|Sunday|1970Q2|N|N|N|2440739|2440889|2440401|2440676|N|N|N|N|N| +2440767|AAAAAAAAPDODFCAA|1970-06-29|845|3678|283|1970|1|6|29|2|1970|283|3678|Monday|1970Q2|N|N|N|2440739|2440889|2440402|2440677|N|N|N|N|N| +2440768|AAAAAAAAAEODFCAA|1970-06-30|845|3679|283|1970|2|6|30|2|1970|283|3679|Tuesday|1970Q2|N|N|N|2440739|2440889|2440403|2440678|N|N|N|N|N| +2440769|AAAAAAAABEODFCAA|1970-07-01|846|3679|283|1970|3|7|1|2|1970|283|3679|Wednesday|1970Q2|N|N|N|2440769|2440949|2440404|2440678|N|N|N|N|N| +2440770|AAAAAAAACEODFCAA|1970-07-02|846|3679|283|1970|4|7|2|3|1970|283|3679|Thursday|1970Q3|N|N|N|2440769|2440949|2440405|2440679|N|N|N|N|N| +2440771|AAAAAAAADEODFCAA|1970-07-03|846|3679|283|1970|5|7|3|3|1970|283|3679|Friday|1970Q3|N|Y|N|2440769|2440949|2440406|2440680|N|N|N|N|N| +2440772|AAAAAAAAEEODFCAA|1970-07-04|846|3679|283|1970|6|7|4|3|1970|283|3679|Saturday|1970Q3|N|Y|N|2440769|2440949|2440407|2440681|N|N|N|N|N| +2440773|AAAAAAAAFEODFCAA|1970-07-05|846|3679|283|1970|0|7|5|3|1970|283|3679|Sunday|1970Q3|Y|N|N|2440769|2440949|2440408|2440682|N|N|N|N|N| +2440774|AAAAAAAAGEODFCAA|1970-07-06|846|3679|283|1970|1|7|6|3|1970|283|3679|Monday|1970Q3|N|N|Y|2440769|2440949|2440409|2440683|N|N|N|N|N| +2440775|AAAAAAAAHEODFCAA|1970-07-07|846|3680|283|1970|2|7|7|3|1970|283|3680|Tuesday|1970Q3|N|N|N|2440769|2440949|2440410|2440684|N|N|N|N|N| +2440776|AAAAAAAAIEODFCAA|1970-07-08|846|3680|283|1970|3|7|8|3|1970|283|3680|Wednesday|1970Q3|N|N|N|2440769|2440949|2440411|2440685|N|N|N|N|N| +2440777|AAAAAAAAJEODFCAA|1970-07-09|846|3680|283|1970|4|7|9|3|1970|283|3680|Thursday|1970Q3|N|N|N|2440769|2440949|2440412|2440686|N|N|N|N|N| +2440778|AAAAAAAAKEODFCAA|1970-07-10|846|3680|283|1970|5|7|10|3|1970|283|3680|Friday|1970Q3|N|Y|N|2440769|2440949|2440413|2440687|N|N|N|N|N| +2440779|AAAAAAAALEODFCAA|1970-07-11|846|3680|283|1970|6|7|11|3|1970|283|3680|Saturday|1970Q3|N|Y|N|2440769|2440949|2440414|2440688|N|N|N|N|N| +2440780|AAAAAAAAMEODFCAA|1970-07-12|846|3680|283|1970|0|7|12|3|1970|283|3680|Sunday|1970Q3|N|N|N|2440769|2440949|2440415|2440689|N|N|N|N|N| +2440781|AAAAAAAANEODFCAA|1970-07-13|846|3680|283|1970|1|7|13|3|1970|283|3680|Monday|1970Q3|N|N|N|2440769|2440949|2440416|2440690|N|N|N|N|N| +2440782|AAAAAAAAOEODFCAA|1970-07-14|846|3681|283|1970|2|7|14|3|1970|283|3681|Tuesday|1970Q3|N|N|N|2440769|2440949|2440417|2440691|N|N|N|N|N| +2440783|AAAAAAAAPEODFCAA|1970-07-15|846|3681|283|1970|3|7|15|3|1970|283|3681|Wednesday|1970Q3|N|N|N|2440769|2440949|2440418|2440692|N|N|N|N|N| +2440784|AAAAAAAAAFODFCAA|1970-07-16|846|3681|283|1970|4|7|16|3|1970|283|3681|Thursday|1970Q3|N|N|N|2440769|2440949|2440419|2440693|N|N|N|N|N| +2440785|AAAAAAAABFODFCAA|1970-07-17|846|3681|283|1970|5|7|17|3|1970|283|3681|Friday|1970Q3|N|Y|N|2440769|2440949|2440420|2440694|N|N|N|N|N| +2440786|AAAAAAAACFODFCAA|1970-07-18|846|3681|283|1970|6|7|18|3|1970|283|3681|Saturday|1970Q3|N|Y|N|2440769|2440949|2440421|2440695|N|N|N|N|N| +2440787|AAAAAAAADFODFCAA|1970-07-19|846|3681|283|1970|0|7|19|3|1970|283|3681|Sunday|1970Q3|N|N|N|2440769|2440949|2440422|2440696|N|N|N|N|N| +2440788|AAAAAAAAEFODFCAA|1970-07-20|846|3681|283|1970|1|7|20|3|1970|283|3681|Monday|1970Q3|N|N|N|2440769|2440949|2440423|2440697|N|N|N|N|N| +2440789|AAAAAAAAFFODFCAA|1970-07-21|846|3682|283|1970|2|7|21|3|1970|283|3682|Tuesday|1970Q3|N|N|N|2440769|2440949|2440424|2440698|N|N|N|N|N| +2440790|AAAAAAAAGFODFCAA|1970-07-22|846|3682|283|1970|3|7|22|3|1970|283|3682|Wednesday|1970Q3|N|N|N|2440769|2440949|2440425|2440699|N|N|N|N|N| +2440791|AAAAAAAAHFODFCAA|1970-07-23|846|3682|283|1970|4|7|23|3|1970|283|3682|Thursday|1970Q3|N|N|N|2440769|2440949|2440426|2440700|N|N|N|N|N| +2440792|AAAAAAAAIFODFCAA|1970-07-24|846|3682|283|1970|5|7|24|3|1970|283|3682|Friday|1970Q3|N|Y|N|2440769|2440949|2440427|2440701|N|N|N|N|N| +2440793|AAAAAAAAJFODFCAA|1970-07-25|846|3682|283|1970|6|7|25|3|1970|283|3682|Saturday|1970Q3|N|Y|N|2440769|2440949|2440428|2440702|N|N|N|N|N| +2440794|AAAAAAAAKFODFCAA|1970-07-26|846|3682|283|1970|0|7|26|3|1970|283|3682|Sunday|1970Q3|N|N|N|2440769|2440949|2440429|2440703|N|N|N|N|N| +2440795|AAAAAAAALFODFCAA|1970-07-27|846|3682|283|1970|1|7|27|3|1970|283|3682|Monday|1970Q3|N|N|N|2440769|2440949|2440430|2440704|N|N|N|N|N| +2440796|AAAAAAAAMFODFCAA|1970-07-28|846|3683|283|1970|2|7|28|3|1970|283|3683|Tuesday|1970Q3|N|N|N|2440769|2440949|2440431|2440705|N|N|N|N|N| +2440797|AAAAAAAANFODFCAA|1970-07-29|846|3683|283|1970|3|7|29|3|1970|283|3683|Wednesday|1970Q3|N|N|N|2440769|2440949|2440432|2440706|N|N|N|N|N| +2440798|AAAAAAAAOFODFCAA|1970-07-30|846|3683|283|1970|4|7|30|3|1970|283|3683|Thursday|1970Q3|N|N|N|2440769|2440949|2440433|2440707|N|N|N|N|N| +2440799|AAAAAAAAPFODFCAA|1970-07-31|846|3683|283|1970|5|7|31|3|1970|283|3683|Friday|1970Q3|N|Y|N|2440769|2440949|2440434|2440708|N|N|N|N|N| +2440800|AAAAAAAAAGODFCAA|1970-08-01|847|3683|283|1970|6|8|1|3|1970|283|3683|Saturday|1970Q3|N|Y|N|2440800|2441011|2440435|2440709|N|N|N|N|N| +2440801|AAAAAAAABGODFCAA|1970-08-02|847|3683|283|1970|0|8|2|3|1970|283|3683|Sunday|1970Q3|N|N|N|2440800|2441011|2440436|2440710|N|N|N|N|N| +2440802|AAAAAAAACGODFCAA|1970-08-03|847|3683|283|1970|1|8|3|3|1970|283|3683|Monday|1970Q3|N|N|N|2440800|2441011|2440437|2440711|N|N|N|N|N| +2440803|AAAAAAAADGODFCAA|1970-08-04|847|3684|283|1970|2|8|4|3|1970|283|3684|Tuesday|1970Q3|N|N|N|2440800|2441011|2440438|2440712|N|N|N|N|N| +2440804|AAAAAAAAEGODFCAA|1970-08-05|847|3684|283|1970|3|8|5|3|1970|283|3684|Wednesday|1970Q3|N|N|N|2440800|2441011|2440439|2440713|N|N|N|N|N| +2440805|AAAAAAAAFGODFCAA|1970-08-06|847|3684|283|1970|4|8|6|3|1970|283|3684|Thursday|1970Q3|N|N|N|2440800|2441011|2440440|2440714|N|N|N|N|N| +2440806|AAAAAAAAGGODFCAA|1970-08-07|847|3684|283|1970|5|8|7|3|1970|283|3684|Friday|1970Q3|N|Y|N|2440800|2441011|2440441|2440715|N|N|N|N|N| +2440807|AAAAAAAAHGODFCAA|1970-08-08|847|3684|283|1970|6|8|8|3|1970|283|3684|Saturday|1970Q3|N|Y|N|2440800|2441011|2440442|2440716|N|N|N|N|N| +2440808|AAAAAAAAIGODFCAA|1970-08-09|847|3684|283|1970|0|8|9|3|1970|283|3684|Sunday|1970Q3|N|N|N|2440800|2441011|2440443|2440717|N|N|N|N|N| +2440809|AAAAAAAAJGODFCAA|1970-08-10|847|3684|283|1970|1|8|10|3|1970|283|3684|Monday|1970Q3|N|N|N|2440800|2441011|2440444|2440718|N|N|N|N|N| +2440810|AAAAAAAAKGODFCAA|1970-08-11|847|3685|283|1970|2|8|11|3|1970|283|3685|Tuesday|1970Q3|N|N|N|2440800|2441011|2440445|2440719|N|N|N|N|N| +2440811|AAAAAAAALGODFCAA|1970-08-12|847|3685|283|1970|3|8|12|3|1970|283|3685|Wednesday|1970Q3|N|N|N|2440800|2441011|2440446|2440720|N|N|N|N|N| +2440812|AAAAAAAAMGODFCAA|1970-08-13|847|3685|283|1970|4|8|13|3|1970|283|3685|Thursday|1970Q3|N|N|N|2440800|2441011|2440447|2440721|N|N|N|N|N| +2440813|AAAAAAAANGODFCAA|1970-08-14|847|3685|283|1970|5|8|14|3|1970|283|3685|Friday|1970Q3|N|Y|N|2440800|2441011|2440448|2440722|N|N|N|N|N| +2440814|AAAAAAAAOGODFCAA|1970-08-15|847|3685|283|1970|6|8|15|3|1970|283|3685|Saturday|1970Q3|N|Y|N|2440800|2441011|2440449|2440723|N|N|N|N|N| +2440815|AAAAAAAAPGODFCAA|1970-08-16|847|3685|283|1970|0|8|16|3|1970|283|3685|Sunday|1970Q3|N|N|N|2440800|2441011|2440450|2440724|N|N|N|N|N| +2440816|AAAAAAAAAHODFCAA|1970-08-17|847|3685|283|1970|1|8|17|3|1970|283|3685|Monday|1970Q3|N|N|N|2440800|2441011|2440451|2440725|N|N|N|N|N| +2440817|AAAAAAAABHODFCAA|1970-08-18|847|3686|283|1970|2|8|18|3|1970|283|3686|Tuesday|1970Q3|N|N|N|2440800|2441011|2440452|2440726|N|N|N|N|N| +2440818|AAAAAAAACHODFCAA|1970-08-19|847|3686|283|1970|3|8|19|3|1970|283|3686|Wednesday|1970Q3|N|N|N|2440800|2441011|2440453|2440727|N|N|N|N|N| +2440819|AAAAAAAADHODFCAA|1970-08-20|847|3686|283|1970|4|8|20|3|1970|283|3686|Thursday|1970Q3|N|N|N|2440800|2441011|2440454|2440728|N|N|N|N|N| +2440820|AAAAAAAAEHODFCAA|1970-08-21|847|3686|283|1970|5|8|21|3|1970|283|3686|Friday|1970Q3|N|Y|N|2440800|2441011|2440455|2440729|N|N|N|N|N| +2440821|AAAAAAAAFHODFCAA|1970-08-22|847|3686|283|1970|6|8|22|3|1970|283|3686|Saturday|1970Q3|N|Y|N|2440800|2441011|2440456|2440730|N|N|N|N|N| +2440822|AAAAAAAAGHODFCAA|1970-08-23|847|3686|283|1970|0|8|23|3|1970|283|3686|Sunday|1970Q3|N|N|N|2440800|2441011|2440457|2440731|N|N|N|N|N| +2440823|AAAAAAAAHHODFCAA|1970-08-24|847|3686|283|1970|1|8|24|3|1970|283|3686|Monday|1970Q3|N|N|N|2440800|2441011|2440458|2440732|N|N|N|N|N| +2440824|AAAAAAAAIHODFCAA|1970-08-25|847|3687|283|1970|2|8|25|3|1970|283|3687|Tuesday|1970Q3|N|N|N|2440800|2441011|2440459|2440733|N|N|N|N|N| +2440825|AAAAAAAAJHODFCAA|1970-08-26|847|3687|283|1970|3|8|26|3|1970|283|3687|Wednesday|1970Q3|N|N|N|2440800|2441011|2440460|2440734|N|N|N|N|N| +2440826|AAAAAAAAKHODFCAA|1970-08-27|847|3687|283|1970|4|8|27|3|1970|283|3687|Thursday|1970Q3|N|N|N|2440800|2441011|2440461|2440735|N|N|N|N|N| +2440827|AAAAAAAALHODFCAA|1970-08-28|847|3687|283|1970|5|8|28|3|1970|283|3687|Friday|1970Q3|N|Y|N|2440800|2441011|2440462|2440736|N|N|N|N|N| +2440828|AAAAAAAAMHODFCAA|1970-08-29|847|3687|283|1970|6|8|29|3|1970|283|3687|Saturday|1970Q3|N|Y|N|2440800|2441011|2440463|2440737|N|N|N|N|N| +2440829|AAAAAAAANHODFCAA|1970-08-30|847|3687|283|1970|0|8|30|3|1970|283|3687|Sunday|1970Q3|N|N|N|2440800|2441011|2440464|2440738|N|N|N|N|N| +2440830|AAAAAAAAOHODFCAA|1970-08-31|847|3687|283|1970|1|8|31|3|1970|283|3687|Monday|1970Q3|N|N|N|2440800|2441011|2440465|2440739|N|N|N|N|N| +2440831|AAAAAAAAPHODFCAA|1970-09-01|848|3688|284|1970|2|9|1|3|1970|284|3688|Tuesday|1970Q3|N|N|N|2440831|2441073|2440466|2440740|N|N|N|N|N| +2440832|AAAAAAAAAIODFCAA|1970-09-02|848|3688|284|1970|3|9|2|3|1970|284|3688|Wednesday|1970Q3|N|N|N|2440831|2441073|2440467|2440741|N|N|N|N|N| +2440833|AAAAAAAABIODFCAA|1970-09-03|848|3688|284|1970|4|9|3|3|1970|284|3688|Thursday|1970Q3|N|N|N|2440831|2441073|2440468|2440742|N|N|N|N|N| +2440834|AAAAAAAACIODFCAA|1970-09-04|848|3688|284|1970|5|9|4|3|1970|284|3688|Friday|1970Q3|N|Y|N|2440831|2441073|2440469|2440743|N|N|N|N|N| +2440835|AAAAAAAADIODFCAA|1970-09-05|848|3688|284|1970|6|9|5|3|1970|284|3688|Saturday|1970Q3|N|Y|N|2440831|2441073|2440470|2440744|N|N|N|N|N| +2440836|AAAAAAAAEIODFCAA|1970-09-06|848|3688|284|1970|0|9|6|3|1970|284|3688|Sunday|1970Q3|N|N|N|2440831|2441073|2440471|2440745|N|N|N|N|N| +2440837|AAAAAAAAFIODFCAA|1970-09-07|848|3688|284|1970|1|9|7|3|1970|284|3688|Monday|1970Q3|N|N|N|2440831|2441073|2440472|2440746|N|N|N|N|N| +2440838|AAAAAAAAGIODFCAA|1970-09-08|848|3689|284|1970|2|9|8|3|1970|284|3689|Tuesday|1970Q3|N|N|N|2440831|2441073|2440473|2440747|N|N|N|N|N| +2440839|AAAAAAAAHIODFCAA|1970-09-09|848|3689|284|1970|3|9|9|3|1970|284|3689|Wednesday|1970Q3|N|N|N|2440831|2441073|2440474|2440748|N|N|N|N|N| +2440840|AAAAAAAAIIODFCAA|1970-09-10|848|3689|284|1970|4|9|10|3|1970|284|3689|Thursday|1970Q3|N|N|N|2440831|2441073|2440475|2440749|N|N|N|N|N| +2440841|AAAAAAAAJIODFCAA|1970-09-11|848|3689|284|1970|5|9|11|3|1970|284|3689|Friday|1970Q3|N|Y|N|2440831|2441073|2440476|2440750|N|N|N|N|N| +2440842|AAAAAAAAKIODFCAA|1970-09-12|848|3689|284|1970|6|9|12|3|1970|284|3689|Saturday|1970Q3|N|Y|N|2440831|2441073|2440477|2440751|N|N|N|N|N| +2440843|AAAAAAAALIODFCAA|1970-09-13|848|3689|284|1970|0|9|13|3|1970|284|3689|Sunday|1970Q3|N|N|N|2440831|2441073|2440478|2440752|N|N|N|N|N| +2440844|AAAAAAAAMIODFCAA|1970-09-14|848|3689|284|1970|1|9|14|3|1970|284|3689|Monday|1970Q3|N|N|N|2440831|2441073|2440479|2440753|N|N|N|N|N| +2440845|AAAAAAAANIODFCAA|1970-09-15|848|3690|284|1970|2|9|15|3|1970|284|3690|Tuesday|1970Q3|N|N|N|2440831|2441073|2440480|2440754|N|N|N|N|N| +2440846|AAAAAAAAOIODFCAA|1970-09-16|848|3690|284|1970|3|9|16|3|1970|284|3690|Wednesday|1970Q3|N|N|N|2440831|2441073|2440481|2440755|N|N|N|N|N| +2440847|AAAAAAAAPIODFCAA|1970-09-17|848|3690|284|1970|4|9|17|3|1970|284|3690|Thursday|1970Q3|N|N|N|2440831|2441073|2440482|2440756|N|N|N|N|N| +2440848|AAAAAAAAAJODFCAA|1970-09-18|848|3690|284|1970|5|9|18|3|1970|284|3690|Friday|1970Q3|N|Y|N|2440831|2441073|2440483|2440757|N|N|N|N|N| +2440849|AAAAAAAABJODFCAA|1970-09-19|848|3690|284|1970|6|9|19|3|1970|284|3690|Saturday|1970Q3|N|Y|N|2440831|2441073|2440484|2440758|N|N|N|N|N| +2440850|AAAAAAAACJODFCAA|1970-09-20|848|3690|284|1970|0|9|20|3|1970|284|3690|Sunday|1970Q3|N|N|N|2440831|2441073|2440485|2440759|N|N|N|N|N| +2440851|AAAAAAAADJODFCAA|1970-09-21|848|3690|284|1970|1|9|21|3|1970|284|3690|Monday|1970Q3|N|N|N|2440831|2441073|2440486|2440760|N|N|N|N|N| +2440852|AAAAAAAAEJODFCAA|1970-09-22|848|3691|284|1970|2|9|22|3|1970|284|3691|Tuesday|1970Q3|N|N|N|2440831|2441073|2440487|2440761|N|N|N|N|N| +2440853|AAAAAAAAFJODFCAA|1970-09-23|848|3691|284|1970|3|9|23|3|1970|284|3691|Wednesday|1970Q3|N|N|N|2440831|2441073|2440488|2440762|N|N|N|N|N| +2440854|AAAAAAAAGJODFCAA|1970-09-24|848|3691|284|1970|4|9|24|3|1970|284|3691|Thursday|1970Q3|N|N|N|2440831|2441073|2440489|2440763|N|N|N|N|N| +2440855|AAAAAAAAHJODFCAA|1970-09-25|848|3691|284|1970|5|9|25|3|1970|284|3691|Friday|1970Q3|N|Y|N|2440831|2441073|2440490|2440764|N|N|N|N|N| +2440856|AAAAAAAAIJODFCAA|1970-09-26|848|3691|284|1970|6|9|26|3|1970|284|3691|Saturday|1970Q3|N|Y|N|2440831|2441073|2440491|2440765|N|N|N|N|N| +2440857|AAAAAAAAJJODFCAA|1970-09-27|848|3691|284|1970|0|9|27|3|1970|284|3691|Sunday|1970Q3|N|N|N|2440831|2441073|2440492|2440766|N|N|N|N|N| +2440858|AAAAAAAAKJODFCAA|1970-09-28|848|3691|284|1970|1|9|28|3|1970|284|3691|Monday|1970Q3|N|N|N|2440831|2441073|2440493|2440767|N|N|N|N|N| +2440859|AAAAAAAALJODFCAA|1970-09-29|848|3692|284|1970|2|9|29|3|1970|284|3692|Tuesday|1970Q3|N|N|N|2440831|2441073|2440494|2440768|N|N|N|N|N| +2440860|AAAAAAAAMJODFCAA|1970-09-30|848|3692|284|1970|3|9|30|3|1970|284|3692|Wednesday|1970Q3|N|N|N|2440831|2441073|2440495|2440769|N|N|N|N|N| +2440861|AAAAAAAANJODFCAA|1970-10-01|849|3692|284|1970|4|10|1|3|1970|284|3692|Thursday|1970Q3|N|N|N|2440861|2441133|2440496|2440769|N|N|N|N|N| +2440862|AAAAAAAAOJODFCAA|1970-10-02|849|3692|284|1970|5|10|2|4|1970|284|3692|Friday|1970Q4|N|Y|N|2440861|2441133|2440497|2440770|N|N|N|N|N| +2440863|AAAAAAAAPJODFCAA|1970-10-03|849|3692|284|1970|6|10|3|4|1970|284|3692|Saturday|1970Q4|N|Y|N|2440861|2441133|2440498|2440771|N|N|N|N|N| +2440864|AAAAAAAAAKODFCAA|1970-10-04|849|3692|284|1970|0|10|4|4|1970|284|3692|Sunday|1970Q4|N|N|N|2440861|2441133|2440499|2440772|N|N|N|N|N| +2440865|AAAAAAAABKODFCAA|1970-10-05|849|3692|284|1970|1|10|5|4|1970|284|3692|Monday|1970Q4|N|N|N|2440861|2441133|2440500|2440773|N|N|N|N|N| +2440866|AAAAAAAACKODFCAA|1970-10-06|849|3693|284|1970|2|10|6|4|1970|284|3693|Tuesday|1970Q4|N|N|N|2440861|2441133|2440501|2440774|N|N|N|N|N| +2440867|AAAAAAAADKODFCAA|1970-10-07|849|3693|284|1970|3|10|7|4|1970|284|3693|Wednesday|1970Q4|N|N|N|2440861|2441133|2440502|2440775|N|N|N|N|N| +2440868|AAAAAAAAEKODFCAA|1970-10-08|849|3693|284|1970|4|10|8|4|1970|284|3693|Thursday|1970Q4|N|N|N|2440861|2441133|2440503|2440776|N|N|N|N|N| +2440869|AAAAAAAAFKODFCAA|1970-10-09|849|3693|284|1970|5|10|9|4|1970|284|3693|Friday|1970Q4|N|Y|N|2440861|2441133|2440504|2440777|N|N|N|N|N| +2440870|AAAAAAAAGKODFCAA|1970-10-10|849|3693|284|1970|6|10|10|4|1970|284|3693|Saturday|1970Q4|N|Y|N|2440861|2441133|2440505|2440778|N|N|N|N|N| +2440871|AAAAAAAAHKODFCAA|1970-10-11|849|3693|284|1970|0|10|11|4|1970|284|3693|Sunday|1970Q4|N|N|N|2440861|2441133|2440506|2440779|N|N|N|N|N| +2440872|AAAAAAAAIKODFCAA|1970-10-12|849|3693|284|1970|1|10|12|4|1970|284|3693|Monday|1970Q4|N|N|N|2440861|2441133|2440507|2440780|N|N|N|N|N| +2440873|AAAAAAAAJKODFCAA|1970-10-13|849|3694|284|1970|2|10|13|4|1970|284|3694|Tuesday|1970Q4|N|N|N|2440861|2441133|2440508|2440781|N|N|N|N|N| +2440874|AAAAAAAAKKODFCAA|1970-10-14|849|3694|284|1970|3|10|14|4|1970|284|3694|Wednesday|1970Q4|N|N|N|2440861|2441133|2440509|2440782|N|N|N|N|N| +2440875|AAAAAAAALKODFCAA|1970-10-15|849|3694|284|1970|4|10|15|4|1970|284|3694|Thursday|1970Q4|N|N|N|2440861|2441133|2440510|2440783|N|N|N|N|N| +2440876|AAAAAAAAMKODFCAA|1970-10-16|849|3694|284|1970|5|10|16|4|1970|284|3694|Friday|1970Q4|N|Y|N|2440861|2441133|2440511|2440784|N|N|N|N|N| +2440877|AAAAAAAANKODFCAA|1970-10-17|849|3694|284|1970|6|10|17|4|1970|284|3694|Saturday|1970Q4|N|Y|N|2440861|2441133|2440512|2440785|N|N|N|N|N| +2440878|AAAAAAAAOKODFCAA|1970-10-18|849|3694|284|1970|0|10|18|4|1970|284|3694|Sunday|1970Q4|N|N|N|2440861|2441133|2440513|2440786|N|N|N|N|N| +2440879|AAAAAAAAPKODFCAA|1970-10-19|849|3694|284|1970|1|10|19|4|1970|284|3694|Monday|1970Q4|N|N|N|2440861|2441133|2440514|2440787|N|N|N|N|N| +2440880|AAAAAAAAALODFCAA|1970-10-20|849|3695|284|1970|2|10|20|4|1970|284|3695|Tuesday|1970Q4|N|N|N|2440861|2441133|2440515|2440788|N|N|N|N|N| +2440881|AAAAAAAABLODFCAA|1970-10-21|849|3695|284|1970|3|10|21|4|1970|284|3695|Wednesday|1970Q4|N|N|N|2440861|2441133|2440516|2440789|N|N|N|N|N| +2440882|AAAAAAAACLODFCAA|1970-10-22|849|3695|284|1970|4|10|22|4|1970|284|3695|Thursday|1970Q4|N|N|N|2440861|2441133|2440517|2440790|N|N|N|N|N| +2440883|AAAAAAAADLODFCAA|1970-10-23|849|3695|284|1970|5|10|23|4|1970|284|3695|Friday|1970Q4|N|Y|N|2440861|2441133|2440518|2440791|N|N|N|N|N| +2440884|AAAAAAAAELODFCAA|1970-10-24|849|3695|284|1970|6|10|24|4|1970|284|3695|Saturday|1970Q4|N|Y|N|2440861|2441133|2440519|2440792|N|N|N|N|N| +2440885|AAAAAAAAFLODFCAA|1970-10-25|849|3695|284|1970|0|10|25|4|1970|284|3695|Sunday|1970Q4|N|N|N|2440861|2441133|2440520|2440793|N|N|N|N|N| +2440886|AAAAAAAAGLODFCAA|1970-10-26|849|3695|284|1970|1|10|26|4|1970|284|3695|Monday|1970Q4|N|N|N|2440861|2441133|2440521|2440794|N|N|N|N|N| +2440887|AAAAAAAAHLODFCAA|1970-10-27|849|3696|284|1970|2|10|27|4|1970|284|3696|Tuesday|1970Q4|N|N|N|2440861|2441133|2440522|2440795|N|N|N|N|N| +2440888|AAAAAAAAILODFCAA|1970-10-28|849|3696|284|1970|3|10|28|4|1970|284|3696|Wednesday|1970Q4|N|N|N|2440861|2441133|2440523|2440796|N|N|N|N|N| +2440889|AAAAAAAAJLODFCAA|1970-10-29|849|3696|284|1970|4|10|29|4|1970|284|3696|Thursday|1970Q4|N|N|N|2440861|2441133|2440524|2440797|N|N|N|N|N| +2440890|AAAAAAAAKLODFCAA|1970-10-30|849|3696|284|1970|5|10|30|4|1970|284|3696|Friday|1970Q4|N|Y|N|2440861|2441133|2440525|2440798|N|N|N|N|N| +2440891|AAAAAAAALLODFCAA|1970-10-31|849|3696|284|1970|6|10|31|4|1970|284|3696|Saturday|1970Q4|N|Y|N|2440861|2441133|2440526|2440799|N|N|N|N|N| +2440892|AAAAAAAAMLODFCAA|1970-11-01|850|3696|284|1970|0|11|1|4|1970|284|3696|Sunday|1970Q4|N|N|N|2440892|2441195|2440527|2440800|N|N|N|N|N| +2440893|AAAAAAAANLODFCAA|1970-11-02|850|3696|284|1970|1|11|2|4|1970|284|3696|Monday|1970Q4|N|N|N|2440892|2441195|2440528|2440801|N|N|N|N|N| +2440894|AAAAAAAAOLODFCAA|1970-11-03|850|3697|284|1970|2|11|3|4|1970|284|3697|Tuesday|1970Q4|N|N|N|2440892|2441195|2440529|2440802|N|N|N|N|N| +2440895|AAAAAAAAPLODFCAA|1970-11-04|850|3697|284|1970|3|11|4|4|1970|284|3697|Wednesday|1970Q4|N|N|N|2440892|2441195|2440530|2440803|N|N|N|N|N| +2440896|AAAAAAAAAMODFCAA|1970-11-05|850|3697|284|1970|4|11|5|4|1970|284|3697|Thursday|1970Q4|N|N|N|2440892|2441195|2440531|2440804|N|N|N|N|N| +2440897|AAAAAAAABMODFCAA|1970-11-06|850|3697|284|1970|5|11|6|4|1970|284|3697|Friday|1970Q4|N|Y|N|2440892|2441195|2440532|2440805|N|N|N|N|N| +2440898|AAAAAAAACMODFCAA|1970-11-07|850|3697|284|1970|6|11|7|4|1970|284|3697|Saturday|1970Q4|N|Y|N|2440892|2441195|2440533|2440806|N|N|N|N|N| +2440899|AAAAAAAADMODFCAA|1970-11-08|850|3697|284|1970|0|11|8|4|1970|284|3697|Sunday|1970Q4|N|N|N|2440892|2441195|2440534|2440807|N|N|N|N|N| +2440900|AAAAAAAAEMODFCAA|1970-11-09|850|3697|284|1970|1|11|9|4|1970|284|3697|Monday|1970Q4|N|N|N|2440892|2441195|2440535|2440808|N|N|N|N|N| +2440901|AAAAAAAAFMODFCAA|1970-11-10|850|3698|284|1970|2|11|10|4|1970|284|3698|Tuesday|1970Q4|N|N|N|2440892|2441195|2440536|2440809|N|N|N|N|N| +2440902|AAAAAAAAGMODFCAA|1970-11-11|850|3698|284|1970|3|11|11|4|1970|284|3698|Wednesday|1970Q4|N|N|N|2440892|2441195|2440537|2440810|N|N|N|N|N| +2440903|AAAAAAAAHMODFCAA|1970-11-12|850|3698|284|1970|4|11|12|4|1970|284|3698|Thursday|1970Q4|N|N|N|2440892|2441195|2440538|2440811|N|N|N|N|N| +2440904|AAAAAAAAIMODFCAA|1970-11-13|850|3698|284|1970|5|11|13|4|1970|284|3698|Friday|1970Q4|N|Y|N|2440892|2441195|2440539|2440812|N|N|N|N|N| +2440905|AAAAAAAAJMODFCAA|1970-11-14|850|3698|284|1970|6|11|14|4|1970|284|3698|Saturday|1970Q4|N|Y|N|2440892|2441195|2440540|2440813|N|N|N|N|N| +2440906|AAAAAAAAKMODFCAA|1970-11-15|850|3698|284|1970|0|11|15|4|1970|284|3698|Sunday|1970Q4|N|N|N|2440892|2441195|2440541|2440814|N|N|N|N|N| +2440907|AAAAAAAALMODFCAA|1970-11-16|850|3698|284|1970|1|11|16|4|1970|284|3698|Monday|1970Q4|N|N|N|2440892|2441195|2440542|2440815|N|N|N|N|N| +2440908|AAAAAAAAMMODFCAA|1970-11-17|850|3699|284|1970|2|11|17|4|1970|284|3699|Tuesday|1970Q4|N|N|N|2440892|2441195|2440543|2440816|N|N|N|N|N| +2440909|AAAAAAAANMODFCAA|1970-11-18|850|3699|284|1970|3|11|18|4|1970|284|3699|Wednesday|1970Q4|N|N|N|2440892|2441195|2440544|2440817|N|N|N|N|N| +2440910|AAAAAAAAOMODFCAA|1970-11-19|850|3699|284|1970|4|11|19|4|1970|284|3699|Thursday|1970Q4|N|N|N|2440892|2441195|2440545|2440818|N|N|N|N|N| +2440911|AAAAAAAAPMODFCAA|1970-11-20|850|3699|284|1970|5|11|20|4|1970|284|3699|Friday|1970Q4|N|Y|N|2440892|2441195|2440546|2440819|N|N|N|N|N| +2440912|AAAAAAAAANODFCAA|1970-11-21|850|3699|284|1970|6|11|21|4|1970|284|3699|Saturday|1970Q4|N|Y|N|2440892|2441195|2440547|2440820|N|N|N|N|N| +2440913|AAAAAAAABNODFCAA|1970-11-22|850|3699|284|1970|0|11|22|4|1970|284|3699|Sunday|1970Q4|N|N|N|2440892|2441195|2440548|2440821|N|N|N|N|N| +2440914|AAAAAAAACNODFCAA|1970-11-23|850|3699|284|1970|1|11|23|4|1970|284|3699|Monday|1970Q4|N|N|N|2440892|2441195|2440549|2440822|N|N|N|N|N| +2440915|AAAAAAAADNODFCAA|1970-11-24|850|3700|284|1970|2|11|24|4|1970|284|3700|Tuesday|1970Q4|N|N|N|2440892|2441195|2440550|2440823|N|N|N|N|N| +2440916|AAAAAAAAENODFCAA|1970-11-25|850|3700|284|1970|3|11|25|4|1970|284|3700|Wednesday|1970Q4|N|N|N|2440892|2441195|2440551|2440824|N|N|N|N|N| +2440917|AAAAAAAAFNODFCAA|1970-11-26|850|3700|284|1970|4|11|26|4|1970|284|3700|Thursday|1970Q4|N|N|N|2440892|2441195|2440552|2440825|N|N|N|N|N| +2440918|AAAAAAAAGNODFCAA|1970-11-27|850|3700|284|1970|5|11|27|4|1970|284|3700|Friday|1970Q4|N|Y|N|2440892|2441195|2440553|2440826|N|N|N|N|N| +2440919|AAAAAAAAHNODFCAA|1970-11-28|850|3700|284|1970|6|11|28|4|1970|284|3700|Saturday|1970Q4|N|Y|N|2440892|2441195|2440554|2440827|N|N|N|N|N| +2440920|AAAAAAAAINODFCAA|1970-11-29|850|3700|284|1970|0|11|29|4|1970|284|3700|Sunday|1970Q4|N|N|N|2440892|2441195|2440555|2440828|N|N|N|N|N| +2440921|AAAAAAAAJNODFCAA|1970-11-30|850|3700|284|1970|1|11|30|4|1970|284|3700|Monday|1970Q4|N|N|N|2440892|2441195|2440556|2440829|N|N|N|N|N| +2440922|AAAAAAAAKNODFCAA|1970-12-01|851|3701|285|1970|2|12|1|4|1970|285|3701|Tuesday|1970Q4|N|N|N|2440922|2441255|2440557|2440830|N|N|N|N|N| +2440923|AAAAAAAALNODFCAA|1970-12-02|851|3701|285|1970|3|12|2|4|1970|285|3701|Wednesday|1970Q4|N|N|N|2440922|2441255|2440558|2440831|N|N|N|N|N| +2440924|AAAAAAAAMNODFCAA|1970-12-03|851|3701|285|1970|4|12|3|4|1970|285|3701|Thursday|1970Q4|N|N|N|2440922|2441255|2440559|2440832|N|N|N|N|N| +2440925|AAAAAAAANNODFCAA|1970-12-04|851|3701|285|1970|5|12|4|4|1970|285|3701|Friday|1970Q4|N|Y|N|2440922|2441255|2440560|2440833|N|N|N|N|N| +2440926|AAAAAAAAONODFCAA|1970-12-05|851|3701|285|1970|6|12|5|4|1970|285|3701|Saturday|1970Q4|N|Y|N|2440922|2441255|2440561|2440834|N|N|N|N|N| +2440927|AAAAAAAAPNODFCAA|1970-12-06|851|3701|285|1970|0|12|6|4|1970|285|3701|Sunday|1970Q4|N|N|N|2440922|2441255|2440562|2440835|N|N|N|N|N| +2440928|AAAAAAAAAOODFCAA|1970-12-07|851|3701|285|1970|1|12|7|4|1970|285|3701|Monday|1970Q4|N|N|N|2440922|2441255|2440563|2440836|N|N|N|N|N| +2440929|AAAAAAAABOODFCAA|1970-12-08|851|3702|285|1970|2|12|8|4|1970|285|3702|Tuesday|1970Q4|N|N|N|2440922|2441255|2440564|2440837|N|N|N|N|N| +2440930|AAAAAAAACOODFCAA|1970-12-09|851|3702|285|1970|3|12|9|4|1970|285|3702|Wednesday|1970Q4|N|N|N|2440922|2441255|2440565|2440838|N|N|N|N|N| +2440931|AAAAAAAADOODFCAA|1970-12-10|851|3702|285|1970|4|12|10|4|1970|285|3702|Thursday|1970Q4|N|N|N|2440922|2441255|2440566|2440839|N|N|N|N|N| +2440932|AAAAAAAAEOODFCAA|1970-12-11|851|3702|285|1970|5|12|11|4|1970|285|3702|Friday|1970Q4|N|Y|N|2440922|2441255|2440567|2440840|N|N|N|N|N| +2440933|AAAAAAAAFOODFCAA|1970-12-12|851|3702|285|1970|6|12|12|4|1970|285|3702|Saturday|1970Q4|N|Y|N|2440922|2441255|2440568|2440841|N|N|N|N|N| +2440934|AAAAAAAAGOODFCAA|1970-12-13|851|3702|285|1970|0|12|13|4|1970|285|3702|Sunday|1970Q4|N|N|N|2440922|2441255|2440569|2440842|N|N|N|N|N| +2440935|AAAAAAAAHOODFCAA|1970-12-14|851|3702|285|1970|1|12|14|4|1970|285|3702|Monday|1970Q4|N|N|N|2440922|2441255|2440570|2440843|N|N|N|N|N| +2440936|AAAAAAAAIOODFCAA|1970-12-15|851|3703|285|1970|2|12|15|4|1970|285|3703|Tuesday|1970Q4|N|N|N|2440922|2441255|2440571|2440844|N|N|N|N|N| +2440937|AAAAAAAAJOODFCAA|1970-12-16|851|3703|285|1970|3|12|16|4|1970|285|3703|Wednesday|1970Q4|N|N|N|2440922|2441255|2440572|2440845|N|N|N|N|N| +2440938|AAAAAAAAKOODFCAA|1970-12-17|851|3703|285|1970|4|12|17|4|1970|285|3703|Thursday|1970Q4|N|N|N|2440922|2441255|2440573|2440846|N|N|N|N|N| +2440939|AAAAAAAALOODFCAA|1970-12-18|851|3703|285|1970|5|12|18|4|1970|285|3703|Friday|1970Q4|N|Y|N|2440922|2441255|2440574|2440847|N|N|N|N|N| +2440940|AAAAAAAAMOODFCAA|1970-12-19|851|3703|285|1970|6|12|19|4|1970|285|3703|Saturday|1970Q4|N|Y|N|2440922|2441255|2440575|2440848|N|N|N|N|N| +2440941|AAAAAAAANOODFCAA|1970-12-20|851|3703|285|1970|0|12|20|4|1970|285|3703|Sunday|1970Q4|N|N|N|2440922|2441255|2440576|2440849|N|N|N|N|N| +2440942|AAAAAAAAOOODFCAA|1970-12-21|851|3703|285|1970|1|12|21|4|1970|285|3703|Monday|1970Q4|N|N|N|2440922|2441255|2440577|2440850|N|N|N|N|N| +2440943|AAAAAAAAPOODFCAA|1970-12-22|851|3704|285|1970|2|12|22|4|1970|285|3704|Tuesday|1970Q4|N|N|N|2440922|2441255|2440578|2440851|N|N|N|N|N| +2440944|AAAAAAAAAPODFCAA|1970-12-23|851|3704|285|1970|3|12|23|4|1970|285|3704|Wednesday|1970Q4|N|N|N|2440922|2441255|2440579|2440852|N|N|N|N|N| +2440945|AAAAAAAABPODFCAA|1970-12-24|851|3704|285|1970|4|12|24|4|1970|285|3704|Thursday|1970Q4|N|N|N|2440922|2441255|2440580|2440853|N|N|N|N|N| +2440946|AAAAAAAACPODFCAA|1970-12-25|851|3704|285|1970|5|12|25|4|1970|285|3704|Friday|1970Q4|N|Y|N|2440922|2441255|2440581|2440854|N|N|N|N|N| +2440947|AAAAAAAADPODFCAA|1970-12-26|851|3704|285|1970|6|12|26|4|1970|285|3704|Saturday|1970Q4|Y|Y|N|2440922|2441255|2440582|2440855|N|N|N|N|N| +2440948|AAAAAAAAEPODFCAA|1970-12-27|851|3704|285|1970|0|12|27|4|1970|285|3704|Sunday|1970Q4|N|N|Y|2440922|2441255|2440583|2440856|N|N|N|N|N| +2440949|AAAAAAAAFPODFCAA|1970-12-28|851|3704|285|1970|1|12|28|4|1970|285|3704|Monday|1970Q4|N|N|N|2440922|2441255|2440584|2440857|N|N|N|N|N| +2440950|AAAAAAAAGPODFCAA|1970-12-29|851|3705|285|1970|2|12|29|4|1970|285|3705|Tuesday|1970Q4|N|N|N|2440922|2441255|2440585|2440858|N|N|N|N|N| +2440951|AAAAAAAAHPODFCAA|1970-12-30|851|3705|285|1970|3|12|30|4|1970|285|3705|Wednesday|1970Q4|N|N|N|2440922|2441255|2440586|2440859|N|N|N|N|N| +2440952|AAAAAAAAIPODFCAA|1970-12-31|851|3705|285|1970|4|12|31|4|1970|285|3705|Thursday|1970Q4|N|N|N|2440922|2441255|2440587|2440860|N|N|N|N|N| +2440953|AAAAAAAAJPODFCAA|1971-01-01|852|3705|285|1971|5|1|1|1|1971|285|3705|Friday|1971Q1|Y|Y|N|2440953|2440952|2440588|2440861|N|N|N|N|N| +2440954|AAAAAAAAKPODFCAA|1971-01-02|852|3705|285|1971|6|1|2|1|1971|285|3705|Saturday|1971Q1|N|Y|Y|2440953|2440952|2440589|2440862|N|N|N|N|N| +2440955|AAAAAAAALPODFCAA|1971-01-03|852|3705|285|1971|0|1|3|1|1971|285|3705|Sunday|1971Q1|N|N|N|2440953|2440952|2440590|2440863|N|N|N|N|N| +2440956|AAAAAAAAMPODFCAA|1971-01-04|852|3705|285|1971|1|1|4|1|1971|285|3705|Monday|1971Q1|N|N|N|2440953|2440952|2440591|2440864|N|N|N|N|N| +2440957|AAAAAAAANPODFCAA|1971-01-05|852|3706|285|1971|2|1|5|1|1971|285|3706|Tuesday|1971Q1|N|N|N|2440953|2440952|2440592|2440865|N|N|N|N|N| +2440958|AAAAAAAAOPODFCAA|1971-01-06|852|3706|285|1971|3|1|6|1|1971|285|3706|Wednesday|1971Q1|N|N|N|2440953|2440952|2440593|2440866|N|N|N|N|N| +2440959|AAAAAAAAPPODFCAA|1971-01-07|852|3706|285|1971|4|1|7|1|1971|285|3706|Thursday|1971Q1|N|N|N|2440953|2440952|2440594|2440867|N|N|N|N|N| +2440960|AAAAAAAAAAPDFCAA|1971-01-08|852|3706|285|1971|5|1|8|1|1971|285|3706|Friday|1971Q1|N|Y|N|2440953|2440952|2440595|2440868|N|N|N|N|N| +2440961|AAAAAAAABAPDFCAA|1971-01-09|852|3706|285|1971|6|1|9|1|1971|285|3706|Saturday|1971Q1|N|Y|N|2440953|2440952|2440596|2440869|N|N|N|N|N| +2440962|AAAAAAAACAPDFCAA|1971-01-10|852|3706|285|1971|0|1|10|1|1971|285|3706|Sunday|1971Q1|N|N|N|2440953|2440952|2440597|2440870|N|N|N|N|N| +2440963|AAAAAAAADAPDFCAA|1971-01-11|852|3706|285|1971|1|1|11|1|1971|285|3706|Monday|1971Q1|N|N|N|2440953|2440952|2440598|2440871|N|N|N|N|N| +2440964|AAAAAAAAEAPDFCAA|1971-01-12|852|3707|285|1971|2|1|12|1|1971|285|3707|Tuesday|1971Q1|N|N|N|2440953|2440952|2440599|2440872|N|N|N|N|N| +2440965|AAAAAAAAFAPDFCAA|1971-01-13|852|3707|285|1971|3|1|13|1|1971|285|3707|Wednesday|1971Q1|N|N|N|2440953|2440952|2440600|2440873|N|N|N|N|N| +2440966|AAAAAAAAGAPDFCAA|1971-01-14|852|3707|285|1971|4|1|14|1|1971|285|3707|Thursday|1971Q1|N|N|N|2440953|2440952|2440601|2440874|N|N|N|N|N| +2440967|AAAAAAAAHAPDFCAA|1971-01-15|852|3707|285|1971|5|1|15|1|1971|285|3707|Friday|1971Q1|N|Y|N|2440953|2440952|2440602|2440875|N|N|N|N|N| +2440968|AAAAAAAAIAPDFCAA|1971-01-16|852|3707|285|1971|6|1|16|1|1971|285|3707|Saturday|1971Q1|N|Y|N|2440953|2440952|2440603|2440876|N|N|N|N|N| +2440969|AAAAAAAAJAPDFCAA|1971-01-17|852|3707|285|1971|0|1|17|1|1971|285|3707|Sunday|1971Q1|N|N|N|2440953|2440952|2440604|2440877|N|N|N|N|N| +2440970|AAAAAAAAKAPDFCAA|1971-01-18|852|3707|285|1971|1|1|18|1|1971|285|3707|Monday|1971Q1|N|N|N|2440953|2440952|2440605|2440878|N|N|N|N|N| +2440971|AAAAAAAALAPDFCAA|1971-01-19|852|3708|285|1971|2|1|19|1|1971|285|3708|Tuesday|1971Q1|N|N|N|2440953|2440952|2440606|2440879|N|N|N|N|N| +2440972|AAAAAAAAMAPDFCAA|1971-01-20|852|3708|285|1971|3|1|20|1|1971|285|3708|Wednesday|1971Q1|N|N|N|2440953|2440952|2440607|2440880|N|N|N|N|N| +2440973|AAAAAAAANAPDFCAA|1971-01-21|852|3708|285|1971|4|1|21|1|1971|285|3708|Thursday|1971Q1|N|N|N|2440953|2440952|2440608|2440881|N|N|N|N|N| +2440974|AAAAAAAAOAPDFCAA|1971-01-22|852|3708|285|1971|5|1|22|1|1971|285|3708|Friday|1971Q1|N|Y|N|2440953|2440952|2440609|2440882|N|N|N|N|N| +2440975|AAAAAAAAPAPDFCAA|1971-01-23|852|3708|285|1971|6|1|23|1|1971|285|3708|Saturday|1971Q1|N|Y|N|2440953|2440952|2440610|2440883|N|N|N|N|N| +2440976|AAAAAAAAABPDFCAA|1971-01-24|852|3708|285|1971|0|1|24|1|1971|285|3708|Sunday|1971Q1|N|N|N|2440953|2440952|2440611|2440884|N|N|N|N|N| +2440977|AAAAAAAABBPDFCAA|1971-01-25|852|3708|285|1971|1|1|25|1|1971|285|3708|Monday|1971Q1|N|N|N|2440953|2440952|2440612|2440885|N|N|N|N|N| +2440978|AAAAAAAACBPDFCAA|1971-01-26|852|3709|285|1971|2|1|26|1|1971|285|3709|Tuesday|1971Q1|N|N|N|2440953|2440952|2440613|2440886|N|N|N|N|N| +2440979|AAAAAAAADBPDFCAA|1971-01-27|852|3709|285|1971|3|1|27|1|1971|285|3709|Wednesday|1971Q1|N|N|N|2440953|2440952|2440614|2440887|N|N|N|N|N| +2440980|AAAAAAAAEBPDFCAA|1971-01-28|852|3709|285|1971|4|1|28|1|1971|285|3709|Thursday|1971Q1|N|N|N|2440953|2440952|2440615|2440888|N|N|N|N|N| +2440981|AAAAAAAAFBPDFCAA|1971-01-29|852|3709|285|1971|5|1|29|1|1971|285|3709|Friday|1971Q1|N|Y|N|2440953|2440952|2440616|2440889|N|N|N|N|N| +2440982|AAAAAAAAGBPDFCAA|1971-01-30|852|3709|285|1971|6|1|30|1|1971|285|3709|Saturday|1971Q1|N|Y|N|2440953|2440952|2440617|2440890|N|N|N|N|N| +2440983|AAAAAAAAHBPDFCAA|1971-01-31|852|3709|285|1971|0|1|31|1|1971|285|3709|Sunday|1971Q1|N|N|N|2440953|2440952|2440618|2440891|N|N|N|N|N| +2440984|AAAAAAAAIBPDFCAA|1971-02-01|853|3709|285|1971|1|2|1|1|1971|285|3709|Monday|1971Q1|N|N|N|2440984|2441014|2440619|2440892|N|N|N|N|N| +2440985|AAAAAAAAJBPDFCAA|1971-02-02|853|3710|285|1971|2|2|2|1|1971|285|3710|Tuesday|1971Q1|N|N|N|2440984|2441014|2440620|2440893|N|N|N|N|N| +2440986|AAAAAAAAKBPDFCAA|1971-02-03|853|3710|285|1971|3|2|3|1|1971|285|3710|Wednesday|1971Q1|N|N|N|2440984|2441014|2440621|2440894|N|N|N|N|N| +2440987|AAAAAAAALBPDFCAA|1971-02-04|853|3710|285|1971|4|2|4|1|1971|285|3710|Thursday|1971Q1|N|N|N|2440984|2441014|2440622|2440895|N|N|N|N|N| +2440988|AAAAAAAAMBPDFCAA|1971-02-05|853|3710|285|1971|5|2|5|1|1971|285|3710|Friday|1971Q1|N|Y|N|2440984|2441014|2440623|2440896|N|N|N|N|N| +2440989|AAAAAAAANBPDFCAA|1971-02-06|853|3710|285|1971|6|2|6|1|1971|285|3710|Saturday|1971Q1|N|Y|N|2440984|2441014|2440624|2440897|N|N|N|N|N| +2440990|AAAAAAAAOBPDFCAA|1971-02-07|853|3710|285|1971|0|2|7|1|1971|285|3710|Sunday|1971Q1|N|N|N|2440984|2441014|2440625|2440898|N|N|N|N|N| +2440991|AAAAAAAAPBPDFCAA|1971-02-08|853|3710|285|1971|1|2|8|1|1971|285|3710|Monday|1971Q1|N|N|N|2440984|2441014|2440626|2440899|N|N|N|N|N| +2440992|AAAAAAAAACPDFCAA|1971-02-09|853|3711|285|1971|2|2|9|1|1971|285|3711|Tuesday|1971Q1|N|N|N|2440984|2441014|2440627|2440900|N|N|N|N|N| +2440993|AAAAAAAABCPDFCAA|1971-02-10|853|3711|285|1971|3|2|10|1|1971|285|3711|Wednesday|1971Q1|N|N|N|2440984|2441014|2440628|2440901|N|N|N|N|N| +2440994|AAAAAAAACCPDFCAA|1971-02-11|853|3711|285|1971|4|2|11|1|1971|285|3711|Thursday|1971Q1|N|N|N|2440984|2441014|2440629|2440902|N|N|N|N|N| +2440995|AAAAAAAADCPDFCAA|1971-02-12|853|3711|285|1971|5|2|12|1|1971|285|3711|Friday|1971Q1|N|Y|N|2440984|2441014|2440630|2440903|N|N|N|N|N| +2440996|AAAAAAAAECPDFCAA|1971-02-13|853|3711|285|1971|6|2|13|1|1971|285|3711|Saturday|1971Q1|N|Y|N|2440984|2441014|2440631|2440904|N|N|N|N|N| +2440997|AAAAAAAAFCPDFCAA|1971-02-14|853|3711|285|1971|0|2|14|1|1971|285|3711|Sunday|1971Q1|N|N|N|2440984|2441014|2440632|2440905|N|N|N|N|N| +2440998|AAAAAAAAGCPDFCAA|1971-02-15|853|3711|285|1971|1|2|15|1|1971|285|3711|Monday|1971Q1|N|N|N|2440984|2441014|2440633|2440906|N|N|N|N|N| +2440999|AAAAAAAAHCPDFCAA|1971-02-16|853|3712|285|1971|2|2|16|1|1971|285|3712|Tuesday|1971Q1|N|N|N|2440984|2441014|2440634|2440907|N|N|N|N|N| +2441000|AAAAAAAAICPDFCAA|1971-02-17|853|3712|285|1971|3|2|17|1|1971|285|3712|Wednesday|1971Q1|N|N|N|2440984|2441014|2440635|2440908|N|N|N|N|N| +2441001|AAAAAAAAJCPDFCAA|1971-02-18|853|3712|285|1971|4|2|18|1|1971|285|3712|Thursday|1971Q1|N|N|N|2440984|2441014|2440636|2440909|N|N|N|N|N| +2441002|AAAAAAAAKCPDFCAA|1971-02-19|853|3712|285|1971|5|2|19|1|1971|285|3712|Friday|1971Q1|N|Y|N|2440984|2441014|2440637|2440910|N|N|N|N|N| +2441003|AAAAAAAALCPDFCAA|1971-02-20|853|3712|285|1971|6|2|20|1|1971|285|3712|Saturday|1971Q1|N|Y|N|2440984|2441014|2440638|2440911|N|N|N|N|N| +2441004|AAAAAAAAMCPDFCAA|1971-02-21|853|3712|285|1971|0|2|21|1|1971|285|3712|Sunday|1971Q1|N|N|N|2440984|2441014|2440639|2440912|N|N|N|N|N| +2441005|AAAAAAAANCPDFCAA|1971-02-22|853|3712|285|1971|1|2|22|1|1971|285|3712|Monday|1971Q1|N|N|N|2440984|2441014|2440640|2440913|N|N|N|N|N| +2441006|AAAAAAAAOCPDFCAA|1971-02-23|853|3713|285|1971|2|2|23|1|1971|285|3713|Tuesday|1971Q1|N|N|N|2440984|2441014|2440641|2440914|N|N|N|N|N| +2441007|AAAAAAAAPCPDFCAA|1971-02-24|853|3713|285|1971|3|2|24|1|1971|285|3713|Wednesday|1971Q1|N|N|N|2440984|2441014|2440642|2440915|N|N|N|N|N| +2441008|AAAAAAAAADPDFCAA|1971-02-25|853|3713|285|1971|4|2|25|1|1971|285|3713|Thursday|1971Q1|N|N|N|2440984|2441014|2440643|2440916|N|N|N|N|N| +2441009|AAAAAAAABDPDFCAA|1971-02-26|853|3713|285|1971|5|2|26|1|1971|285|3713|Friday|1971Q1|N|Y|N|2440984|2441014|2440644|2440917|N|N|N|N|N| +2441010|AAAAAAAACDPDFCAA|1971-02-27|853|3713|285|1971|6|2|27|1|1971|285|3713|Saturday|1971Q1|N|Y|N|2440984|2441014|2440645|2440918|N|N|N|N|N| +2441011|AAAAAAAADDPDFCAA|1971-02-28|853|3713|285|1971|0|2|28|1|1971|285|3713|Sunday|1971Q1|N|N|N|2440984|2441014|2440646|2440919|N|N|N|N|N| +2441012|AAAAAAAAEDPDFCAA|1971-03-01|854|3713|286|1971|1|3|1|1|1971|286|3713|Monday|1971Q1|N|N|N|2441012|2441070|2440647|2440920|N|N|N|N|N| +2441013|AAAAAAAAFDPDFCAA|1971-03-02|854|3714|286|1971|2|3|2|1|1971|286|3714|Tuesday|1971Q1|N|N|N|2441012|2441070|2440648|2440921|N|N|N|N|N| +2441014|AAAAAAAAGDPDFCAA|1971-03-03|854|3714|286|1971|3|3|3|1|1971|286|3714|Wednesday|1971Q1|N|N|N|2441012|2441070|2440649|2440922|N|N|N|N|N| +2441015|AAAAAAAAHDPDFCAA|1971-03-04|854|3714|286|1971|4|3|4|1|1971|286|3714|Thursday|1971Q1|N|N|N|2441012|2441070|2440650|2440923|N|N|N|N|N| +2441016|AAAAAAAAIDPDFCAA|1971-03-05|854|3714|286|1971|5|3|5|1|1971|286|3714|Friday|1971Q1|N|Y|N|2441012|2441070|2440651|2440924|N|N|N|N|N| +2441017|AAAAAAAAJDPDFCAA|1971-03-06|854|3714|286|1971|6|3|6|1|1971|286|3714|Saturday|1971Q1|N|Y|N|2441012|2441070|2440652|2440925|N|N|N|N|N| +2441018|AAAAAAAAKDPDFCAA|1971-03-07|854|3714|286|1971|0|3|7|1|1971|286|3714|Sunday|1971Q1|N|N|N|2441012|2441070|2440653|2440926|N|N|N|N|N| +2441019|AAAAAAAALDPDFCAA|1971-03-08|854|3714|286|1971|1|3|8|1|1971|286|3714|Monday|1971Q1|N|N|N|2441012|2441070|2440654|2440927|N|N|N|N|N| +2441020|AAAAAAAAMDPDFCAA|1971-03-09|854|3715|286|1971|2|3|9|1|1971|286|3715|Tuesday|1971Q1|N|N|N|2441012|2441070|2440655|2440928|N|N|N|N|N| +2441021|AAAAAAAANDPDFCAA|1971-03-10|854|3715|286|1971|3|3|10|1|1971|286|3715|Wednesday|1971Q1|N|N|N|2441012|2441070|2440656|2440929|N|N|N|N|N| +2441022|AAAAAAAAODPDFCAA|1971-03-11|854|3715|286|1971|4|3|11|1|1971|286|3715|Thursday|1971Q1|N|N|N|2441012|2441070|2440657|2440930|N|N|N|N|N| +2441023|AAAAAAAAPDPDFCAA|1971-03-12|854|3715|286|1971|5|3|12|1|1971|286|3715|Friday|1971Q1|N|Y|N|2441012|2441070|2440658|2440931|N|N|N|N|N| +2441024|AAAAAAAAAEPDFCAA|1971-03-13|854|3715|286|1971|6|3|13|1|1971|286|3715|Saturday|1971Q1|N|Y|N|2441012|2441070|2440659|2440932|N|N|N|N|N| +2441025|AAAAAAAABEPDFCAA|1971-03-14|854|3715|286|1971|0|3|14|1|1971|286|3715|Sunday|1971Q1|N|N|N|2441012|2441070|2440660|2440933|N|N|N|N|N| +2441026|AAAAAAAACEPDFCAA|1971-03-15|854|3715|286|1971|1|3|15|1|1971|286|3715|Monday|1971Q1|N|N|N|2441012|2441070|2440661|2440934|N|N|N|N|N| +2441027|AAAAAAAADEPDFCAA|1971-03-16|854|3716|286|1971|2|3|16|1|1971|286|3716|Tuesday|1971Q1|N|N|N|2441012|2441070|2440662|2440935|N|N|N|N|N| +2441028|AAAAAAAAEEPDFCAA|1971-03-17|854|3716|286|1971|3|3|17|1|1971|286|3716|Wednesday|1971Q1|N|N|N|2441012|2441070|2440663|2440936|N|N|N|N|N| +2441029|AAAAAAAAFEPDFCAA|1971-03-18|854|3716|286|1971|4|3|18|1|1971|286|3716|Thursday|1971Q1|N|N|N|2441012|2441070|2440664|2440937|N|N|N|N|N| +2441030|AAAAAAAAGEPDFCAA|1971-03-19|854|3716|286|1971|5|3|19|1|1971|286|3716|Friday|1971Q1|N|Y|N|2441012|2441070|2440665|2440938|N|N|N|N|N| +2441031|AAAAAAAAHEPDFCAA|1971-03-20|854|3716|286|1971|6|3|20|1|1971|286|3716|Saturday|1971Q1|N|Y|N|2441012|2441070|2440666|2440939|N|N|N|N|N| +2441032|AAAAAAAAIEPDFCAA|1971-03-21|854|3716|286|1971|0|3|21|1|1971|286|3716|Sunday|1971Q1|N|N|N|2441012|2441070|2440667|2440940|N|N|N|N|N| +2441033|AAAAAAAAJEPDFCAA|1971-03-22|854|3716|286|1971|1|3|22|1|1971|286|3716|Monday|1971Q1|N|N|N|2441012|2441070|2440668|2440941|N|N|N|N|N| +2441034|AAAAAAAAKEPDFCAA|1971-03-23|854|3717|286|1971|2|3|23|1|1971|286|3717|Tuesday|1971Q1|N|N|N|2441012|2441070|2440669|2440942|N|N|N|N|N| +2441035|AAAAAAAALEPDFCAA|1971-03-24|854|3717|286|1971|3|3|24|1|1971|286|3717|Wednesday|1971Q1|N|N|N|2441012|2441070|2440670|2440943|N|N|N|N|N| +2441036|AAAAAAAAMEPDFCAA|1971-03-25|854|3717|286|1971|4|3|25|1|1971|286|3717|Thursday|1971Q1|N|N|N|2441012|2441070|2440671|2440944|N|N|N|N|N| +2441037|AAAAAAAANEPDFCAA|1971-03-26|854|3717|286|1971|5|3|26|1|1971|286|3717|Friday|1971Q1|N|Y|N|2441012|2441070|2440672|2440945|N|N|N|N|N| +2441038|AAAAAAAAOEPDFCAA|1971-03-27|854|3717|286|1971|6|3|27|1|1971|286|3717|Saturday|1971Q1|N|Y|N|2441012|2441070|2440673|2440946|N|N|N|N|N| +2441039|AAAAAAAAPEPDFCAA|1971-03-28|854|3717|286|1971|0|3|28|1|1971|286|3717|Sunday|1971Q1|N|N|N|2441012|2441070|2440674|2440947|N|N|N|N|N| +2441040|AAAAAAAAAFPDFCAA|1971-03-29|854|3717|286|1971|1|3|29|1|1971|286|3717|Monday|1971Q1|N|N|N|2441012|2441070|2440675|2440948|N|N|N|N|N| +2441041|AAAAAAAABFPDFCAA|1971-03-30|854|3718|286|1971|2|3|30|1|1971|286|3718|Tuesday|1971Q1|N|N|N|2441012|2441070|2440676|2440949|N|N|N|N|N| +2441042|AAAAAAAACFPDFCAA|1971-03-31|854|3718|286|1971|3|3|31|1|1971|286|3718|Wednesday|1971Q1|N|N|N|2441012|2441070|2440677|2440950|N|N|N|N|N| +2441043|AAAAAAAADFPDFCAA|1971-04-01|855|3718|286|1971|4|4|1|1|1971|286|3718|Thursday|1971Q1|N|N|N|2441043|2441132|2440678|2440953|N|N|N|N|N| +2441044|AAAAAAAAEFPDFCAA|1971-04-02|855|3718|286|1971|5|4|2|2|1971|286|3718|Friday|1971Q2|N|Y|N|2441043|2441132|2440679|2440954|N|N|N|N|N| +2441045|AAAAAAAAFFPDFCAA|1971-04-03|855|3718|286|1971|6|4|3|2|1971|286|3718|Saturday|1971Q2|N|Y|N|2441043|2441132|2440680|2440955|N|N|N|N|N| +2441046|AAAAAAAAGFPDFCAA|1971-04-04|855|3718|286|1971|0|4|4|2|1971|286|3718|Sunday|1971Q2|N|N|N|2441043|2441132|2440681|2440956|N|N|N|N|N| +2441047|AAAAAAAAHFPDFCAA|1971-04-05|855|3718|286|1971|1|4|5|2|1971|286|3718|Monday|1971Q2|N|N|N|2441043|2441132|2440682|2440957|N|N|N|N|N| +2441048|AAAAAAAAIFPDFCAA|1971-04-06|855|3719|286|1971|2|4|6|2|1971|286|3719|Tuesday|1971Q2|N|N|N|2441043|2441132|2440683|2440958|N|N|N|N|N| +2441049|AAAAAAAAJFPDFCAA|1971-04-07|855|3719|286|1971|3|4|7|2|1971|286|3719|Wednesday|1971Q2|N|N|N|2441043|2441132|2440684|2440959|N|N|N|N|N| +2441050|AAAAAAAAKFPDFCAA|1971-04-08|855|3719|286|1971|4|4|8|2|1971|286|3719|Thursday|1971Q2|N|N|N|2441043|2441132|2440685|2440960|N|N|N|N|N| +2441051|AAAAAAAALFPDFCAA|1971-04-09|855|3719|286|1971|5|4|9|2|1971|286|3719|Friday|1971Q2|N|Y|N|2441043|2441132|2440686|2440961|N|N|N|N|N| +2441052|AAAAAAAAMFPDFCAA|1971-04-10|855|3719|286|1971|6|4|10|2|1971|286|3719|Saturday|1971Q2|N|Y|N|2441043|2441132|2440687|2440962|N|N|N|N|N| +2441053|AAAAAAAANFPDFCAA|1971-04-11|855|3719|286|1971|0|4|11|2|1971|286|3719|Sunday|1971Q2|N|N|N|2441043|2441132|2440688|2440963|N|N|N|N|N| +2441054|AAAAAAAAOFPDFCAA|1971-04-12|855|3719|286|1971|1|4|12|2|1971|286|3719|Monday|1971Q2|N|N|N|2441043|2441132|2440689|2440964|N|N|N|N|N| +2441055|AAAAAAAAPFPDFCAA|1971-04-13|855|3720|286|1971|2|4|13|2|1971|286|3720|Tuesday|1971Q2|N|N|N|2441043|2441132|2440690|2440965|N|N|N|N|N| +2441056|AAAAAAAAAGPDFCAA|1971-04-14|855|3720|286|1971|3|4|14|2|1971|286|3720|Wednesday|1971Q2|N|N|N|2441043|2441132|2440691|2440966|N|N|N|N|N| +2441057|AAAAAAAABGPDFCAA|1971-04-15|855|3720|286|1971|4|4|15|2|1971|286|3720|Thursday|1971Q2|N|N|N|2441043|2441132|2440692|2440967|N|N|N|N|N| +2441058|AAAAAAAACGPDFCAA|1971-04-16|855|3720|286|1971|5|4|16|2|1971|286|3720|Friday|1971Q2|N|Y|N|2441043|2441132|2440693|2440968|N|N|N|N|N| +2441059|AAAAAAAADGPDFCAA|1971-04-17|855|3720|286|1971|6|4|17|2|1971|286|3720|Saturday|1971Q2|N|Y|N|2441043|2441132|2440694|2440969|N|N|N|N|N| +2441060|AAAAAAAAEGPDFCAA|1971-04-18|855|3720|286|1971|0|4|18|2|1971|286|3720|Sunday|1971Q2|N|N|N|2441043|2441132|2440695|2440970|N|N|N|N|N| +2441061|AAAAAAAAFGPDFCAA|1971-04-19|855|3720|286|1971|1|4|19|2|1971|286|3720|Monday|1971Q2|N|N|N|2441043|2441132|2440696|2440971|N|N|N|N|N| +2441062|AAAAAAAAGGPDFCAA|1971-04-20|855|3721|286|1971|2|4|20|2|1971|286|3721|Tuesday|1971Q2|N|N|N|2441043|2441132|2440697|2440972|N|N|N|N|N| +2441063|AAAAAAAAHGPDFCAA|1971-04-21|855|3721|286|1971|3|4|21|2|1971|286|3721|Wednesday|1971Q2|N|N|N|2441043|2441132|2440698|2440973|N|N|N|N|N| +2441064|AAAAAAAAIGPDFCAA|1971-04-22|855|3721|286|1971|4|4|22|2|1971|286|3721|Thursday|1971Q2|N|N|N|2441043|2441132|2440699|2440974|N|N|N|N|N| +2441065|AAAAAAAAJGPDFCAA|1971-04-23|855|3721|286|1971|5|4|23|2|1971|286|3721|Friday|1971Q2|N|Y|N|2441043|2441132|2440700|2440975|N|N|N|N|N| +2441066|AAAAAAAAKGPDFCAA|1971-04-24|855|3721|286|1971|6|4|24|2|1971|286|3721|Saturday|1971Q2|N|Y|N|2441043|2441132|2440701|2440976|N|N|N|N|N| +2441067|AAAAAAAALGPDFCAA|1971-04-25|855|3721|286|1971|0|4|25|2|1971|286|3721|Sunday|1971Q2|N|N|N|2441043|2441132|2440702|2440977|N|N|N|N|N| +2441068|AAAAAAAAMGPDFCAA|1971-04-26|855|3721|286|1971|1|4|26|2|1971|286|3721|Monday|1971Q2|N|N|N|2441043|2441132|2440703|2440978|N|N|N|N|N| +2441069|AAAAAAAANGPDFCAA|1971-04-27|855|3722|286|1971|2|4|27|2|1971|286|3722|Tuesday|1971Q2|N|N|N|2441043|2441132|2440704|2440979|N|N|N|N|N| +2441070|AAAAAAAAOGPDFCAA|1971-04-28|855|3722|286|1971|3|4|28|2|1971|286|3722|Wednesday|1971Q2|N|N|N|2441043|2441132|2440705|2440980|N|N|N|N|N| +2441071|AAAAAAAAPGPDFCAA|1971-04-29|855|3722|286|1971|4|4|29|2|1971|286|3722|Thursday|1971Q2|N|N|N|2441043|2441132|2440706|2440981|N|N|N|N|N| +2441072|AAAAAAAAAHPDFCAA|1971-04-30|855|3722|286|1971|5|4|30|2|1971|286|3722|Friday|1971Q2|N|Y|N|2441043|2441132|2440707|2440982|N|N|N|N|N| +2441073|AAAAAAAABHPDFCAA|1971-05-01|856|3722|286|1971|6|5|1|2|1971|286|3722|Saturday|1971Q2|N|Y|N|2441073|2441192|2440708|2440983|N|N|N|N|N| +2441074|AAAAAAAACHPDFCAA|1971-05-02|856|3722|286|1971|0|5|2|2|1971|286|3722|Sunday|1971Q2|N|N|N|2441073|2441192|2440709|2440984|N|N|N|N|N| +2441075|AAAAAAAADHPDFCAA|1971-05-03|856|3722|286|1971|1|5|3|2|1971|286|3722|Monday|1971Q2|N|N|N|2441073|2441192|2440710|2440985|N|N|N|N|N| +2441076|AAAAAAAAEHPDFCAA|1971-05-04|856|3723|286|1971|2|5|4|2|1971|286|3723|Tuesday|1971Q2|N|N|N|2441073|2441192|2440711|2440986|N|N|N|N|N| +2441077|AAAAAAAAFHPDFCAA|1971-05-05|856|3723|286|1971|3|5|5|2|1971|286|3723|Wednesday|1971Q2|N|N|N|2441073|2441192|2440712|2440987|N|N|N|N|N| +2441078|AAAAAAAAGHPDFCAA|1971-05-06|856|3723|286|1971|4|5|6|2|1971|286|3723|Thursday|1971Q2|N|N|N|2441073|2441192|2440713|2440988|N|N|N|N|N| +2441079|AAAAAAAAHHPDFCAA|1971-05-07|856|3723|286|1971|5|5|7|2|1971|286|3723|Friday|1971Q2|N|Y|N|2441073|2441192|2440714|2440989|N|N|N|N|N| +2441080|AAAAAAAAIHPDFCAA|1971-05-08|856|3723|286|1971|6|5|8|2|1971|286|3723|Saturday|1971Q2|N|Y|N|2441073|2441192|2440715|2440990|N|N|N|N|N| +2441081|AAAAAAAAJHPDFCAA|1971-05-09|856|3723|286|1971|0|5|9|2|1971|286|3723|Sunday|1971Q2|N|N|N|2441073|2441192|2440716|2440991|N|N|N|N|N| +2441082|AAAAAAAAKHPDFCAA|1971-05-10|856|3723|286|1971|1|5|10|2|1971|286|3723|Monday|1971Q2|N|N|N|2441073|2441192|2440717|2440992|N|N|N|N|N| +2441083|AAAAAAAALHPDFCAA|1971-05-11|856|3724|286|1971|2|5|11|2|1971|286|3724|Tuesday|1971Q2|N|N|N|2441073|2441192|2440718|2440993|N|N|N|N|N| +2441084|AAAAAAAAMHPDFCAA|1971-05-12|856|3724|286|1971|3|5|12|2|1971|286|3724|Wednesday|1971Q2|N|N|N|2441073|2441192|2440719|2440994|N|N|N|N|N| +2441085|AAAAAAAANHPDFCAA|1971-05-13|856|3724|286|1971|4|5|13|2|1971|286|3724|Thursday|1971Q2|N|N|N|2441073|2441192|2440720|2440995|N|N|N|N|N| +2441086|AAAAAAAAOHPDFCAA|1971-05-14|856|3724|286|1971|5|5|14|2|1971|286|3724|Friday|1971Q2|N|Y|N|2441073|2441192|2440721|2440996|N|N|N|N|N| +2441087|AAAAAAAAPHPDFCAA|1971-05-15|856|3724|286|1971|6|5|15|2|1971|286|3724|Saturday|1971Q2|N|Y|N|2441073|2441192|2440722|2440997|N|N|N|N|N| +2441088|AAAAAAAAAIPDFCAA|1971-05-16|856|3724|286|1971|0|5|16|2|1971|286|3724|Sunday|1971Q2|N|N|N|2441073|2441192|2440723|2440998|N|N|N|N|N| +2441089|AAAAAAAABIPDFCAA|1971-05-17|856|3724|286|1971|1|5|17|2|1971|286|3724|Monday|1971Q2|N|N|N|2441073|2441192|2440724|2440999|N|N|N|N|N| +2441090|AAAAAAAACIPDFCAA|1971-05-18|856|3725|286|1971|2|5|18|2|1971|286|3725|Tuesday|1971Q2|N|N|N|2441073|2441192|2440725|2441000|N|N|N|N|N| +2441091|AAAAAAAADIPDFCAA|1971-05-19|856|3725|286|1971|3|5|19|2|1971|286|3725|Wednesday|1971Q2|N|N|N|2441073|2441192|2440726|2441001|N|N|N|N|N| +2441092|AAAAAAAAEIPDFCAA|1971-05-20|856|3725|286|1971|4|5|20|2|1971|286|3725|Thursday|1971Q2|N|N|N|2441073|2441192|2440727|2441002|N|N|N|N|N| +2441093|AAAAAAAAFIPDFCAA|1971-05-21|856|3725|286|1971|5|5|21|2|1971|286|3725|Friday|1971Q2|N|Y|N|2441073|2441192|2440728|2441003|N|N|N|N|N| +2441094|AAAAAAAAGIPDFCAA|1971-05-22|856|3725|286|1971|6|5|22|2|1971|286|3725|Saturday|1971Q2|N|Y|N|2441073|2441192|2440729|2441004|N|N|N|N|N| +2441095|AAAAAAAAHIPDFCAA|1971-05-23|856|3725|286|1971|0|5|23|2|1971|286|3725|Sunday|1971Q2|N|N|N|2441073|2441192|2440730|2441005|N|N|N|N|N| +2441096|AAAAAAAAIIPDFCAA|1971-05-24|856|3725|286|1971|1|5|24|2|1971|286|3725|Monday|1971Q2|N|N|N|2441073|2441192|2440731|2441006|N|N|N|N|N| +2441097|AAAAAAAAJIPDFCAA|1971-05-25|856|3726|286|1971|2|5|25|2|1971|286|3726|Tuesday|1971Q2|N|N|N|2441073|2441192|2440732|2441007|N|N|N|N|N| +2441098|AAAAAAAAKIPDFCAA|1971-05-26|856|3726|286|1971|3|5|26|2|1971|286|3726|Wednesday|1971Q2|N|N|N|2441073|2441192|2440733|2441008|N|N|N|N|N| +2441099|AAAAAAAALIPDFCAA|1971-05-27|856|3726|286|1971|4|5|27|2|1971|286|3726|Thursday|1971Q2|N|N|N|2441073|2441192|2440734|2441009|N|N|N|N|N| +2441100|AAAAAAAAMIPDFCAA|1971-05-28|856|3726|286|1971|5|5|28|2|1971|286|3726|Friday|1971Q2|N|Y|N|2441073|2441192|2440735|2441010|N|N|N|N|N| +2441101|AAAAAAAANIPDFCAA|1971-05-29|856|3726|286|1971|6|5|29|2|1971|286|3726|Saturday|1971Q2|N|Y|N|2441073|2441192|2440736|2441011|N|N|N|N|N| +2441102|AAAAAAAAOIPDFCAA|1971-05-30|856|3726|286|1971|0|5|30|2|1971|286|3726|Sunday|1971Q2|N|N|N|2441073|2441192|2440737|2441012|N|N|N|N|N| +2441103|AAAAAAAAPIPDFCAA|1971-05-31|856|3726|286|1971|1|5|31|2|1971|286|3726|Monday|1971Q2|N|N|N|2441073|2441192|2440738|2441013|N|N|N|N|N| +2441104|AAAAAAAAAJPDFCAA|1971-06-01|857|3727|287|1971|2|6|1|2|1971|287|3727|Tuesday|1971Q2|N|N|N|2441104|2441254|2440739|2441014|N|N|N|N|N| +2441105|AAAAAAAABJPDFCAA|1971-06-02|857|3727|287|1971|3|6|2|2|1971|287|3727|Wednesday|1971Q2|N|N|N|2441104|2441254|2440740|2441015|N|N|N|N|N| +2441106|AAAAAAAACJPDFCAA|1971-06-03|857|3727|287|1971|4|6|3|2|1971|287|3727|Thursday|1971Q2|N|N|N|2441104|2441254|2440741|2441016|N|N|N|N|N| +2441107|AAAAAAAADJPDFCAA|1971-06-04|857|3727|287|1971|5|6|4|2|1971|287|3727|Friday|1971Q2|N|Y|N|2441104|2441254|2440742|2441017|N|N|N|N|N| +2441108|AAAAAAAAEJPDFCAA|1971-06-05|857|3727|287|1971|6|6|5|2|1971|287|3727|Saturday|1971Q2|N|Y|N|2441104|2441254|2440743|2441018|N|N|N|N|N| +2441109|AAAAAAAAFJPDFCAA|1971-06-06|857|3727|287|1971|0|6|6|2|1971|287|3727|Sunday|1971Q2|N|N|N|2441104|2441254|2440744|2441019|N|N|N|N|N| +2441110|AAAAAAAAGJPDFCAA|1971-06-07|857|3727|287|1971|1|6|7|2|1971|287|3727|Monday|1971Q2|N|N|N|2441104|2441254|2440745|2441020|N|N|N|N|N| +2441111|AAAAAAAAHJPDFCAA|1971-06-08|857|3728|287|1971|2|6|8|2|1971|287|3728|Tuesday|1971Q2|N|N|N|2441104|2441254|2440746|2441021|N|N|N|N|N| +2441112|AAAAAAAAIJPDFCAA|1971-06-09|857|3728|287|1971|3|6|9|2|1971|287|3728|Wednesday|1971Q2|N|N|N|2441104|2441254|2440747|2441022|N|N|N|N|N| +2441113|AAAAAAAAJJPDFCAA|1971-06-10|857|3728|287|1971|4|6|10|2|1971|287|3728|Thursday|1971Q2|N|N|N|2441104|2441254|2440748|2441023|N|N|N|N|N| +2441114|AAAAAAAAKJPDFCAA|1971-06-11|857|3728|287|1971|5|6|11|2|1971|287|3728|Friday|1971Q2|N|Y|N|2441104|2441254|2440749|2441024|N|N|N|N|N| +2441115|AAAAAAAALJPDFCAA|1971-06-12|857|3728|287|1971|6|6|12|2|1971|287|3728|Saturday|1971Q2|N|Y|N|2441104|2441254|2440750|2441025|N|N|N|N|N| +2441116|AAAAAAAAMJPDFCAA|1971-06-13|857|3728|287|1971|0|6|13|2|1971|287|3728|Sunday|1971Q2|N|N|N|2441104|2441254|2440751|2441026|N|N|N|N|N| +2441117|AAAAAAAANJPDFCAA|1971-06-14|857|3728|287|1971|1|6|14|2|1971|287|3728|Monday|1971Q2|N|N|N|2441104|2441254|2440752|2441027|N|N|N|N|N| +2441118|AAAAAAAAOJPDFCAA|1971-06-15|857|3729|287|1971|2|6|15|2|1971|287|3729|Tuesday|1971Q2|N|N|N|2441104|2441254|2440753|2441028|N|N|N|N|N| +2441119|AAAAAAAAPJPDFCAA|1971-06-16|857|3729|287|1971|3|6|16|2|1971|287|3729|Wednesday|1971Q2|N|N|N|2441104|2441254|2440754|2441029|N|N|N|N|N| +2441120|AAAAAAAAAKPDFCAA|1971-06-17|857|3729|287|1971|4|6|17|2|1971|287|3729|Thursday|1971Q2|N|N|N|2441104|2441254|2440755|2441030|N|N|N|N|N| +2441121|AAAAAAAABKPDFCAA|1971-06-18|857|3729|287|1971|5|6|18|2|1971|287|3729|Friday|1971Q2|N|Y|N|2441104|2441254|2440756|2441031|N|N|N|N|N| +2441122|AAAAAAAACKPDFCAA|1971-06-19|857|3729|287|1971|6|6|19|2|1971|287|3729|Saturday|1971Q2|N|Y|N|2441104|2441254|2440757|2441032|N|N|N|N|N| +2441123|AAAAAAAADKPDFCAA|1971-06-20|857|3729|287|1971|0|6|20|2|1971|287|3729|Sunday|1971Q2|N|N|N|2441104|2441254|2440758|2441033|N|N|N|N|N| +2441124|AAAAAAAAEKPDFCAA|1971-06-21|857|3729|287|1971|1|6|21|2|1971|287|3729|Monday|1971Q2|N|N|N|2441104|2441254|2440759|2441034|N|N|N|N|N| +2441125|AAAAAAAAFKPDFCAA|1971-06-22|857|3730|287|1971|2|6|22|2|1971|287|3730|Tuesday|1971Q2|N|N|N|2441104|2441254|2440760|2441035|N|N|N|N|N| +2441126|AAAAAAAAGKPDFCAA|1971-06-23|857|3730|287|1971|3|6|23|2|1971|287|3730|Wednesday|1971Q2|N|N|N|2441104|2441254|2440761|2441036|N|N|N|N|N| +2441127|AAAAAAAAHKPDFCAA|1971-06-24|857|3730|287|1971|4|6|24|2|1971|287|3730|Thursday|1971Q2|N|N|N|2441104|2441254|2440762|2441037|N|N|N|N|N| +2441128|AAAAAAAAIKPDFCAA|1971-06-25|857|3730|287|1971|5|6|25|2|1971|287|3730|Friday|1971Q2|N|Y|N|2441104|2441254|2440763|2441038|N|N|N|N|N| +2441129|AAAAAAAAJKPDFCAA|1971-06-26|857|3730|287|1971|6|6|26|2|1971|287|3730|Saturday|1971Q2|N|Y|N|2441104|2441254|2440764|2441039|N|N|N|N|N| +2441130|AAAAAAAAKKPDFCAA|1971-06-27|857|3730|287|1971|0|6|27|2|1971|287|3730|Sunday|1971Q2|N|N|N|2441104|2441254|2440765|2441040|N|N|N|N|N| +2441131|AAAAAAAALKPDFCAA|1971-06-28|857|3730|287|1971|1|6|28|2|1971|287|3730|Monday|1971Q2|N|N|N|2441104|2441254|2440766|2441041|N|N|N|N|N| +2441132|AAAAAAAAMKPDFCAA|1971-06-29|857|3731|287|1971|2|6|29|2|1971|287|3731|Tuesday|1971Q2|N|N|N|2441104|2441254|2440767|2441042|N|N|N|N|N| +2441133|AAAAAAAANKPDFCAA|1971-06-30|857|3731|287|1971|3|6|30|2|1971|287|3731|Wednesday|1971Q2|N|N|N|2441104|2441254|2440768|2441043|N|N|N|N|N| +2441134|AAAAAAAAOKPDFCAA|1971-07-01|858|3731|287|1971|4|7|1|2|1971|287|3731|Thursday|1971Q2|N|N|N|2441134|2441314|2440769|2441043|N|N|N|N|N| +2441135|AAAAAAAAPKPDFCAA|1971-07-02|858|3731|287|1971|5|7|2|3|1971|287|3731|Friday|1971Q3|N|Y|N|2441134|2441314|2440770|2441044|N|N|N|N|N| +2441136|AAAAAAAAALPDFCAA|1971-07-03|858|3731|287|1971|6|7|3|3|1971|287|3731|Saturday|1971Q3|N|Y|N|2441134|2441314|2440771|2441045|N|N|N|N|N| +2441137|AAAAAAAABLPDFCAA|1971-07-04|858|3731|287|1971|0|7|4|3|1971|287|3731|Sunday|1971Q3|N|N|N|2441134|2441314|2440772|2441046|N|N|N|N|N| +2441138|AAAAAAAACLPDFCAA|1971-07-05|858|3731|287|1971|1|7|5|3|1971|287|3731|Monday|1971Q3|Y|N|N|2441134|2441314|2440773|2441047|N|N|N|N|N| +2441139|AAAAAAAADLPDFCAA|1971-07-06|858|3732|287|1971|2|7|6|3|1971|287|3732|Tuesday|1971Q3|N|N|Y|2441134|2441314|2440774|2441048|N|N|N|N|N| +2441140|AAAAAAAAELPDFCAA|1971-07-07|858|3732|287|1971|3|7|7|3|1971|287|3732|Wednesday|1971Q3|N|N|N|2441134|2441314|2440775|2441049|N|N|N|N|N| +2441141|AAAAAAAAFLPDFCAA|1971-07-08|858|3732|287|1971|4|7|8|3|1971|287|3732|Thursday|1971Q3|N|N|N|2441134|2441314|2440776|2441050|N|N|N|N|N| +2441142|AAAAAAAAGLPDFCAA|1971-07-09|858|3732|287|1971|5|7|9|3|1971|287|3732|Friday|1971Q3|N|Y|N|2441134|2441314|2440777|2441051|N|N|N|N|N| +2441143|AAAAAAAAHLPDFCAA|1971-07-10|858|3732|287|1971|6|7|10|3|1971|287|3732|Saturday|1971Q3|N|Y|N|2441134|2441314|2440778|2441052|N|N|N|N|N| +2441144|AAAAAAAAILPDFCAA|1971-07-11|858|3732|287|1971|0|7|11|3|1971|287|3732|Sunday|1971Q3|N|N|N|2441134|2441314|2440779|2441053|N|N|N|N|N| +2441145|AAAAAAAAJLPDFCAA|1971-07-12|858|3732|287|1971|1|7|12|3|1971|287|3732|Monday|1971Q3|N|N|N|2441134|2441314|2440780|2441054|N|N|N|N|N| +2441146|AAAAAAAAKLPDFCAA|1971-07-13|858|3733|287|1971|2|7|13|3|1971|287|3733|Tuesday|1971Q3|N|N|N|2441134|2441314|2440781|2441055|N|N|N|N|N| +2441147|AAAAAAAALLPDFCAA|1971-07-14|858|3733|287|1971|3|7|14|3|1971|287|3733|Wednesday|1971Q3|N|N|N|2441134|2441314|2440782|2441056|N|N|N|N|N| +2441148|AAAAAAAAMLPDFCAA|1971-07-15|858|3733|287|1971|4|7|15|3|1971|287|3733|Thursday|1971Q3|N|N|N|2441134|2441314|2440783|2441057|N|N|N|N|N| +2441149|AAAAAAAANLPDFCAA|1971-07-16|858|3733|287|1971|5|7|16|3|1971|287|3733|Friday|1971Q3|N|Y|N|2441134|2441314|2440784|2441058|N|N|N|N|N| +2441150|AAAAAAAAOLPDFCAA|1971-07-17|858|3733|287|1971|6|7|17|3|1971|287|3733|Saturday|1971Q3|N|Y|N|2441134|2441314|2440785|2441059|N|N|N|N|N| +2441151|AAAAAAAAPLPDFCAA|1971-07-18|858|3733|287|1971|0|7|18|3|1971|287|3733|Sunday|1971Q3|N|N|N|2441134|2441314|2440786|2441060|N|N|N|N|N| +2441152|AAAAAAAAAMPDFCAA|1971-07-19|858|3733|287|1971|1|7|19|3|1971|287|3733|Monday|1971Q3|N|N|N|2441134|2441314|2440787|2441061|N|N|N|N|N| +2441153|AAAAAAAABMPDFCAA|1971-07-20|858|3734|287|1971|2|7|20|3|1971|287|3734|Tuesday|1971Q3|N|N|N|2441134|2441314|2440788|2441062|N|N|N|N|N| +2441154|AAAAAAAACMPDFCAA|1971-07-21|858|3734|287|1971|3|7|21|3|1971|287|3734|Wednesday|1971Q3|N|N|N|2441134|2441314|2440789|2441063|N|N|N|N|N| +2441155|AAAAAAAADMPDFCAA|1971-07-22|858|3734|287|1971|4|7|22|3|1971|287|3734|Thursday|1971Q3|N|N|N|2441134|2441314|2440790|2441064|N|N|N|N|N| +2441156|AAAAAAAAEMPDFCAA|1971-07-23|858|3734|287|1971|5|7|23|3|1971|287|3734|Friday|1971Q3|N|Y|N|2441134|2441314|2440791|2441065|N|N|N|N|N| +2441157|AAAAAAAAFMPDFCAA|1971-07-24|858|3734|287|1971|6|7|24|3|1971|287|3734|Saturday|1971Q3|N|Y|N|2441134|2441314|2440792|2441066|N|N|N|N|N| +2441158|AAAAAAAAGMPDFCAA|1971-07-25|858|3734|287|1971|0|7|25|3|1971|287|3734|Sunday|1971Q3|N|N|N|2441134|2441314|2440793|2441067|N|N|N|N|N| +2441159|AAAAAAAAHMPDFCAA|1971-07-26|858|3734|287|1971|1|7|26|3|1971|287|3734|Monday|1971Q3|N|N|N|2441134|2441314|2440794|2441068|N|N|N|N|N| +2441160|AAAAAAAAIMPDFCAA|1971-07-27|858|3735|287|1971|2|7|27|3|1971|287|3735|Tuesday|1971Q3|N|N|N|2441134|2441314|2440795|2441069|N|N|N|N|N| +2441161|AAAAAAAAJMPDFCAA|1971-07-28|858|3735|287|1971|3|7|28|3|1971|287|3735|Wednesday|1971Q3|N|N|N|2441134|2441314|2440796|2441070|N|N|N|N|N| +2441162|AAAAAAAAKMPDFCAA|1971-07-29|858|3735|287|1971|4|7|29|3|1971|287|3735|Thursday|1971Q3|N|N|N|2441134|2441314|2440797|2441071|N|N|N|N|N| +2441163|AAAAAAAALMPDFCAA|1971-07-30|858|3735|287|1971|5|7|30|3|1971|287|3735|Friday|1971Q3|N|Y|N|2441134|2441314|2440798|2441072|N|N|N|N|N| +2441164|AAAAAAAAMMPDFCAA|1971-07-31|858|3735|287|1971|6|7|31|3|1971|287|3735|Saturday|1971Q3|N|Y|N|2441134|2441314|2440799|2441073|N|N|N|N|N| +2441165|AAAAAAAANMPDFCAA|1971-08-01|859|3735|287|1971|0|8|1|3|1971|287|3735|Sunday|1971Q3|N|N|N|2441165|2441376|2440800|2441074|N|N|N|N|N| +2441166|AAAAAAAAOMPDFCAA|1971-08-02|859|3735|287|1971|1|8|2|3|1971|287|3735|Monday|1971Q3|N|N|N|2441165|2441376|2440801|2441075|N|N|N|N|N| +2441167|AAAAAAAAPMPDFCAA|1971-08-03|859|3736|287|1971|2|8|3|3|1971|287|3736|Tuesday|1971Q3|N|N|N|2441165|2441376|2440802|2441076|N|N|N|N|N| +2441168|AAAAAAAAANPDFCAA|1971-08-04|859|3736|287|1971|3|8|4|3|1971|287|3736|Wednesday|1971Q3|N|N|N|2441165|2441376|2440803|2441077|N|N|N|N|N| +2441169|AAAAAAAABNPDFCAA|1971-08-05|859|3736|287|1971|4|8|5|3|1971|287|3736|Thursday|1971Q3|N|N|N|2441165|2441376|2440804|2441078|N|N|N|N|N| +2441170|AAAAAAAACNPDFCAA|1971-08-06|859|3736|287|1971|5|8|6|3|1971|287|3736|Friday|1971Q3|N|Y|N|2441165|2441376|2440805|2441079|N|N|N|N|N| +2441171|AAAAAAAADNPDFCAA|1971-08-07|859|3736|287|1971|6|8|7|3|1971|287|3736|Saturday|1971Q3|N|Y|N|2441165|2441376|2440806|2441080|N|N|N|N|N| +2441172|AAAAAAAAENPDFCAA|1971-08-08|859|3736|287|1971|0|8|8|3|1971|287|3736|Sunday|1971Q3|N|N|N|2441165|2441376|2440807|2441081|N|N|N|N|N| +2441173|AAAAAAAAFNPDFCAA|1971-08-09|859|3736|287|1971|1|8|9|3|1971|287|3736|Monday|1971Q3|N|N|N|2441165|2441376|2440808|2441082|N|N|N|N|N| +2441174|AAAAAAAAGNPDFCAA|1971-08-10|859|3737|287|1971|2|8|10|3|1971|287|3737|Tuesday|1971Q3|N|N|N|2441165|2441376|2440809|2441083|N|N|N|N|N| +2441175|AAAAAAAAHNPDFCAA|1971-08-11|859|3737|287|1971|3|8|11|3|1971|287|3737|Wednesday|1971Q3|N|N|N|2441165|2441376|2440810|2441084|N|N|N|N|N| +2441176|AAAAAAAAINPDFCAA|1971-08-12|859|3737|287|1971|4|8|12|3|1971|287|3737|Thursday|1971Q3|N|N|N|2441165|2441376|2440811|2441085|N|N|N|N|N| +2441177|AAAAAAAAJNPDFCAA|1971-08-13|859|3737|287|1971|5|8|13|3|1971|287|3737|Friday|1971Q3|N|Y|N|2441165|2441376|2440812|2441086|N|N|N|N|N| +2441178|AAAAAAAAKNPDFCAA|1971-08-14|859|3737|287|1971|6|8|14|3|1971|287|3737|Saturday|1971Q3|N|Y|N|2441165|2441376|2440813|2441087|N|N|N|N|N| +2441179|AAAAAAAALNPDFCAA|1971-08-15|859|3737|287|1971|0|8|15|3|1971|287|3737|Sunday|1971Q3|N|N|N|2441165|2441376|2440814|2441088|N|N|N|N|N| +2441180|AAAAAAAAMNPDFCAA|1971-08-16|859|3737|287|1971|1|8|16|3|1971|287|3737|Monday|1971Q3|N|N|N|2441165|2441376|2440815|2441089|N|N|N|N|N| +2441181|AAAAAAAANNPDFCAA|1971-08-17|859|3738|287|1971|2|8|17|3|1971|287|3738|Tuesday|1971Q3|N|N|N|2441165|2441376|2440816|2441090|N|N|N|N|N| +2441182|AAAAAAAAONPDFCAA|1971-08-18|859|3738|287|1971|3|8|18|3|1971|287|3738|Wednesday|1971Q3|N|N|N|2441165|2441376|2440817|2441091|N|N|N|N|N| +2441183|AAAAAAAAPNPDFCAA|1971-08-19|859|3738|287|1971|4|8|19|3|1971|287|3738|Thursday|1971Q3|N|N|N|2441165|2441376|2440818|2441092|N|N|N|N|N| +2441184|AAAAAAAAAOPDFCAA|1971-08-20|859|3738|287|1971|5|8|20|3|1971|287|3738|Friday|1971Q3|N|Y|N|2441165|2441376|2440819|2441093|N|N|N|N|N| +2441185|AAAAAAAABOPDFCAA|1971-08-21|859|3738|287|1971|6|8|21|3|1971|287|3738|Saturday|1971Q3|N|Y|N|2441165|2441376|2440820|2441094|N|N|N|N|N| +2441186|AAAAAAAACOPDFCAA|1971-08-22|859|3738|287|1971|0|8|22|3|1971|287|3738|Sunday|1971Q3|N|N|N|2441165|2441376|2440821|2441095|N|N|N|N|N| +2441187|AAAAAAAADOPDFCAA|1971-08-23|859|3738|287|1971|1|8|23|3|1971|287|3738|Monday|1971Q3|N|N|N|2441165|2441376|2440822|2441096|N|N|N|N|N| +2441188|AAAAAAAAEOPDFCAA|1971-08-24|859|3739|287|1971|2|8|24|3|1971|287|3739|Tuesday|1971Q3|N|N|N|2441165|2441376|2440823|2441097|N|N|N|N|N| +2441189|AAAAAAAAFOPDFCAA|1971-08-25|859|3739|287|1971|3|8|25|3|1971|287|3739|Wednesday|1971Q3|N|N|N|2441165|2441376|2440824|2441098|N|N|N|N|N| +2441190|AAAAAAAAGOPDFCAA|1971-08-26|859|3739|287|1971|4|8|26|3|1971|287|3739|Thursday|1971Q3|N|N|N|2441165|2441376|2440825|2441099|N|N|N|N|N| +2441191|AAAAAAAAHOPDFCAA|1971-08-27|859|3739|287|1971|5|8|27|3|1971|287|3739|Friday|1971Q3|N|Y|N|2441165|2441376|2440826|2441100|N|N|N|N|N| +2441192|AAAAAAAAIOPDFCAA|1971-08-28|859|3739|287|1971|6|8|28|3|1971|287|3739|Saturday|1971Q3|N|Y|N|2441165|2441376|2440827|2441101|N|N|N|N|N| +2441193|AAAAAAAAJOPDFCAA|1971-08-29|859|3739|287|1971|0|8|29|3|1971|287|3739|Sunday|1971Q3|N|N|N|2441165|2441376|2440828|2441102|N|N|N|N|N| +2441194|AAAAAAAAKOPDFCAA|1971-08-30|859|3739|287|1971|1|8|30|3|1971|287|3739|Monday|1971Q3|N|N|N|2441165|2441376|2440829|2441103|N|N|N|N|N| +2441195|AAAAAAAALOPDFCAA|1971-08-31|859|3740|287|1971|2|8|31|3|1971|287|3740|Tuesday|1971Q3|N|N|N|2441165|2441376|2440830|2441104|N|N|N|N|N| +2441196|AAAAAAAAMOPDFCAA|1971-09-01|860|3740|288|1971|3|9|1|3|1971|288|3740|Wednesday|1971Q3|N|N|N|2441196|2441438|2440831|2441105|N|N|N|N|N| +2441197|AAAAAAAANOPDFCAA|1971-09-02|860|3740|288|1971|4|9|2|3|1971|288|3740|Thursday|1971Q3|N|N|N|2441196|2441438|2440832|2441106|N|N|N|N|N| +2441198|AAAAAAAAOOPDFCAA|1971-09-03|860|3740|288|1971|5|9|3|3|1971|288|3740|Friday|1971Q3|N|Y|N|2441196|2441438|2440833|2441107|N|N|N|N|N| +2441199|AAAAAAAAPOPDFCAA|1971-09-04|860|3740|288|1971|6|9|4|3|1971|288|3740|Saturday|1971Q3|N|Y|N|2441196|2441438|2440834|2441108|N|N|N|N|N| +2441200|AAAAAAAAAPPDFCAA|1971-09-05|860|3740|288|1971|0|9|5|3|1971|288|3740|Sunday|1971Q3|N|N|N|2441196|2441438|2440835|2441109|N|N|N|N|N| +2441201|AAAAAAAABPPDFCAA|1971-09-06|860|3740|288|1971|1|9|6|3|1971|288|3740|Monday|1971Q3|N|N|N|2441196|2441438|2440836|2441110|N|N|N|N|N| +2441202|AAAAAAAACPPDFCAA|1971-09-07|860|3741|288|1971|2|9|7|3|1971|288|3741|Tuesday|1971Q3|N|N|N|2441196|2441438|2440837|2441111|N|N|N|N|N| +2441203|AAAAAAAADPPDFCAA|1971-09-08|860|3741|288|1971|3|9|8|3|1971|288|3741|Wednesday|1971Q3|N|N|N|2441196|2441438|2440838|2441112|N|N|N|N|N| +2441204|AAAAAAAAEPPDFCAA|1971-09-09|860|3741|288|1971|4|9|9|3|1971|288|3741|Thursday|1971Q3|N|N|N|2441196|2441438|2440839|2441113|N|N|N|N|N| +2441205|AAAAAAAAFPPDFCAA|1971-09-10|860|3741|288|1971|5|9|10|3|1971|288|3741|Friday|1971Q3|N|Y|N|2441196|2441438|2440840|2441114|N|N|N|N|N| +2441206|AAAAAAAAGPPDFCAA|1971-09-11|860|3741|288|1971|6|9|11|3|1971|288|3741|Saturday|1971Q3|N|Y|N|2441196|2441438|2440841|2441115|N|N|N|N|N| +2441207|AAAAAAAAHPPDFCAA|1971-09-12|860|3741|288|1971|0|9|12|3|1971|288|3741|Sunday|1971Q3|N|N|N|2441196|2441438|2440842|2441116|N|N|N|N|N| +2441208|AAAAAAAAIPPDFCAA|1971-09-13|860|3741|288|1971|1|9|13|3|1971|288|3741|Monday|1971Q3|N|N|N|2441196|2441438|2440843|2441117|N|N|N|N|N| +2441209|AAAAAAAAJPPDFCAA|1971-09-14|860|3742|288|1971|2|9|14|3|1971|288|3742|Tuesday|1971Q3|N|N|N|2441196|2441438|2440844|2441118|N|N|N|N|N| +2441210|AAAAAAAAKPPDFCAA|1971-09-15|860|3742|288|1971|3|9|15|3|1971|288|3742|Wednesday|1971Q3|N|N|N|2441196|2441438|2440845|2441119|N|N|N|N|N| +2441211|AAAAAAAALPPDFCAA|1971-09-16|860|3742|288|1971|4|9|16|3|1971|288|3742|Thursday|1971Q3|N|N|N|2441196|2441438|2440846|2441120|N|N|N|N|N| +2441212|AAAAAAAAMPPDFCAA|1971-09-17|860|3742|288|1971|5|9|17|3|1971|288|3742|Friday|1971Q3|N|Y|N|2441196|2441438|2440847|2441121|N|N|N|N|N| +2441213|AAAAAAAANPPDFCAA|1971-09-18|860|3742|288|1971|6|9|18|3|1971|288|3742|Saturday|1971Q3|N|Y|N|2441196|2441438|2440848|2441122|N|N|N|N|N| +2441214|AAAAAAAAOPPDFCAA|1971-09-19|860|3742|288|1971|0|9|19|3|1971|288|3742|Sunday|1971Q3|N|N|N|2441196|2441438|2440849|2441123|N|N|N|N|N| +2441215|AAAAAAAAPPPDFCAA|1971-09-20|860|3742|288|1971|1|9|20|3|1971|288|3742|Monday|1971Q3|N|N|N|2441196|2441438|2440850|2441124|N|N|N|N|N| +2441216|AAAAAAAAAAAEFCAA|1971-09-21|860|3743|288|1971|2|9|21|3|1971|288|3743|Tuesday|1971Q3|N|N|N|2441196|2441438|2440851|2441125|N|N|N|N|N| +2441217|AAAAAAAABAAEFCAA|1971-09-22|860|3743|288|1971|3|9|22|3|1971|288|3743|Wednesday|1971Q3|N|N|N|2441196|2441438|2440852|2441126|N|N|N|N|N| +2441218|AAAAAAAACAAEFCAA|1971-09-23|860|3743|288|1971|4|9|23|3|1971|288|3743|Thursday|1971Q3|N|N|N|2441196|2441438|2440853|2441127|N|N|N|N|N| +2441219|AAAAAAAADAAEFCAA|1971-09-24|860|3743|288|1971|5|9|24|3|1971|288|3743|Friday|1971Q3|N|Y|N|2441196|2441438|2440854|2441128|N|N|N|N|N| +2441220|AAAAAAAAEAAEFCAA|1971-09-25|860|3743|288|1971|6|9|25|3|1971|288|3743|Saturday|1971Q3|N|Y|N|2441196|2441438|2440855|2441129|N|N|N|N|N| +2441221|AAAAAAAAFAAEFCAA|1971-09-26|860|3743|288|1971|0|9|26|3|1971|288|3743|Sunday|1971Q3|N|N|N|2441196|2441438|2440856|2441130|N|N|N|N|N| +2441222|AAAAAAAAGAAEFCAA|1971-09-27|860|3743|288|1971|1|9|27|3|1971|288|3743|Monday|1971Q3|N|N|N|2441196|2441438|2440857|2441131|N|N|N|N|N| +2441223|AAAAAAAAHAAEFCAA|1971-09-28|860|3744|288|1971|2|9|28|3|1971|288|3744|Tuesday|1971Q3|N|N|N|2441196|2441438|2440858|2441132|N|N|N|N|N| +2441224|AAAAAAAAIAAEFCAA|1971-09-29|860|3744|288|1971|3|9|29|3|1971|288|3744|Wednesday|1971Q3|N|N|N|2441196|2441438|2440859|2441133|N|N|N|N|N| +2441225|AAAAAAAAJAAEFCAA|1971-09-30|860|3744|288|1971|4|9|30|3|1971|288|3744|Thursday|1971Q3|N|N|N|2441196|2441438|2440860|2441134|N|N|N|N|N| +2441226|AAAAAAAAKAAEFCAA|1971-10-01|861|3744|288|1971|5|10|1|3|1971|288|3744|Friday|1971Q3|N|Y|N|2441226|2441498|2440861|2441134|N|N|N|N|N| +2441227|AAAAAAAALAAEFCAA|1971-10-02|861|3744|288|1971|6|10|2|4|1971|288|3744|Saturday|1971Q4|N|Y|N|2441226|2441498|2440862|2441135|N|N|N|N|N| +2441228|AAAAAAAAMAAEFCAA|1971-10-03|861|3744|288|1971|0|10|3|4|1971|288|3744|Sunday|1971Q4|N|N|N|2441226|2441498|2440863|2441136|N|N|N|N|N| +2441229|AAAAAAAANAAEFCAA|1971-10-04|861|3744|288|1971|1|10|4|4|1971|288|3744|Monday|1971Q4|N|N|N|2441226|2441498|2440864|2441137|N|N|N|N|N| +2441230|AAAAAAAAOAAEFCAA|1971-10-05|861|3745|288|1971|2|10|5|4|1971|288|3745|Tuesday|1971Q4|N|N|N|2441226|2441498|2440865|2441138|N|N|N|N|N| +2441231|AAAAAAAAPAAEFCAA|1971-10-06|861|3745|288|1971|3|10|6|4|1971|288|3745|Wednesday|1971Q4|N|N|N|2441226|2441498|2440866|2441139|N|N|N|N|N| +2441232|AAAAAAAAABAEFCAA|1971-10-07|861|3745|288|1971|4|10|7|4|1971|288|3745|Thursday|1971Q4|N|N|N|2441226|2441498|2440867|2441140|N|N|N|N|N| +2441233|AAAAAAAABBAEFCAA|1971-10-08|861|3745|288|1971|5|10|8|4|1971|288|3745|Friday|1971Q4|N|Y|N|2441226|2441498|2440868|2441141|N|N|N|N|N| +2441234|AAAAAAAACBAEFCAA|1971-10-09|861|3745|288|1971|6|10|9|4|1971|288|3745|Saturday|1971Q4|N|Y|N|2441226|2441498|2440869|2441142|N|N|N|N|N| +2441235|AAAAAAAADBAEFCAA|1971-10-10|861|3745|288|1971|0|10|10|4|1971|288|3745|Sunday|1971Q4|N|N|N|2441226|2441498|2440870|2441143|N|N|N|N|N| +2441236|AAAAAAAAEBAEFCAA|1971-10-11|861|3745|288|1971|1|10|11|4|1971|288|3745|Monday|1971Q4|N|N|N|2441226|2441498|2440871|2441144|N|N|N|N|N| +2441237|AAAAAAAAFBAEFCAA|1971-10-12|861|3746|288|1971|2|10|12|4|1971|288|3746|Tuesday|1971Q4|N|N|N|2441226|2441498|2440872|2441145|N|N|N|N|N| +2441238|AAAAAAAAGBAEFCAA|1971-10-13|861|3746|288|1971|3|10|13|4|1971|288|3746|Wednesday|1971Q4|N|N|N|2441226|2441498|2440873|2441146|N|N|N|N|N| +2441239|AAAAAAAAHBAEFCAA|1971-10-14|861|3746|288|1971|4|10|14|4|1971|288|3746|Thursday|1971Q4|N|N|N|2441226|2441498|2440874|2441147|N|N|N|N|N| +2441240|AAAAAAAAIBAEFCAA|1971-10-15|861|3746|288|1971|5|10|15|4|1971|288|3746|Friday|1971Q4|N|Y|N|2441226|2441498|2440875|2441148|N|N|N|N|N| +2441241|AAAAAAAAJBAEFCAA|1971-10-16|861|3746|288|1971|6|10|16|4|1971|288|3746|Saturday|1971Q4|N|Y|N|2441226|2441498|2440876|2441149|N|N|N|N|N| +2441242|AAAAAAAAKBAEFCAA|1971-10-17|861|3746|288|1971|0|10|17|4|1971|288|3746|Sunday|1971Q4|N|N|N|2441226|2441498|2440877|2441150|N|N|N|N|N| +2441243|AAAAAAAALBAEFCAA|1971-10-18|861|3746|288|1971|1|10|18|4|1971|288|3746|Monday|1971Q4|N|N|N|2441226|2441498|2440878|2441151|N|N|N|N|N| +2441244|AAAAAAAAMBAEFCAA|1971-10-19|861|3747|288|1971|2|10|19|4|1971|288|3747|Tuesday|1971Q4|N|N|N|2441226|2441498|2440879|2441152|N|N|N|N|N| +2441245|AAAAAAAANBAEFCAA|1971-10-20|861|3747|288|1971|3|10|20|4|1971|288|3747|Wednesday|1971Q4|N|N|N|2441226|2441498|2440880|2441153|N|N|N|N|N| +2441246|AAAAAAAAOBAEFCAA|1971-10-21|861|3747|288|1971|4|10|21|4|1971|288|3747|Thursday|1971Q4|N|N|N|2441226|2441498|2440881|2441154|N|N|N|N|N| +2441247|AAAAAAAAPBAEFCAA|1971-10-22|861|3747|288|1971|5|10|22|4|1971|288|3747|Friday|1971Q4|N|Y|N|2441226|2441498|2440882|2441155|N|N|N|N|N| +2441248|AAAAAAAAACAEFCAA|1971-10-23|861|3747|288|1971|6|10|23|4|1971|288|3747|Saturday|1971Q4|N|Y|N|2441226|2441498|2440883|2441156|N|N|N|N|N| +2441249|AAAAAAAABCAEFCAA|1971-10-24|861|3747|288|1971|0|10|24|4|1971|288|3747|Sunday|1971Q4|N|N|N|2441226|2441498|2440884|2441157|N|N|N|N|N| +2441250|AAAAAAAACCAEFCAA|1971-10-25|861|3747|288|1971|1|10|25|4|1971|288|3747|Monday|1971Q4|N|N|N|2441226|2441498|2440885|2441158|N|N|N|N|N| +2441251|AAAAAAAADCAEFCAA|1971-10-26|861|3748|288|1971|2|10|26|4|1971|288|3748|Tuesday|1971Q4|N|N|N|2441226|2441498|2440886|2441159|N|N|N|N|N| +2441252|AAAAAAAAECAEFCAA|1971-10-27|861|3748|288|1971|3|10|27|4|1971|288|3748|Wednesday|1971Q4|N|N|N|2441226|2441498|2440887|2441160|N|N|N|N|N| +2441253|AAAAAAAAFCAEFCAA|1971-10-28|861|3748|288|1971|4|10|28|4|1971|288|3748|Thursday|1971Q4|N|N|N|2441226|2441498|2440888|2441161|N|N|N|N|N| +2441254|AAAAAAAAGCAEFCAA|1971-10-29|861|3748|288|1971|5|10|29|4|1971|288|3748|Friday|1971Q4|N|Y|N|2441226|2441498|2440889|2441162|N|N|N|N|N| +2441255|AAAAAAAAHCAEFCAA|1971-10-30|861|3748|288|1971|6|10|30|4|1971|288|3748|Saturday|1971Q4|N|Y|N|2441226|2441498|2440890|2441163|N|N|N|N|N| +2441256|AAAAAAAAICAEFCAA|1971-10-31|861|3748|288|1971|0|10|31|4|1971|288|3748|Sunday|1971Q4|N|N|N|2441226|2441498|2440891|2441164|N|N|N|N|N| +2441257|AAAAAAAAJCAEFCAA|1971-11-01|862|3748|288|1971|1|11|1|4|1971|288|3748|Monday|1971Q4|N|N|N|2441257|2441560|2440892|2441165|N|N|N|N|N| +2441258|AAAAAAAAKCAEFCAA|1971-11-02|862|3749|288|1971|2|11|2|4|1971|288|3749|Tuesday|1971Q4|N|N|N|2441257|2441560|2440893|2441166|N|N|N|N|N| +2441259|AAAAAAAALCAEFCAA|1971-11-03|862|3749|288|1971|3|11|3|4|1971|288|3749|Wednesday|1971Q4|N|N|N|2441257|2441560|2440894|2441167|N|N|N|N|N| +2441260|AAAAAAAAMCAEFCAA|1971-11-04|862|3749|288|1971|4|11|4|4|1971|288|3749|Thursday|1971Q4|N|N|N|2441257|2441560|2440895|2441168|N|N|N|N|N| +2441261|AAAAAAAANCAEFCAA|1971-11-05|862|3749|288|1971|5|11|5|4|1971|288|3749|Friday|1971Q4|N|Y|N|2441257|2441560|2440896|2441169|N|N|N|N|N| +2441262|AAAAAAAAOCAEFCAA|1971-11-06|862|3749|288|1971|6|11|6|4|1971|288|3749|Saturday|1971Q4|N|Y|N|2441257|2441560|2440897|2441170|N|N|N|N|N| +2441263|AAAAAAAAPCAEFCAA|1971-11-07|862|3749|288|1971|0|11|7|4|1971|288|3749|Sunday|1971Q4|N|N|N|2441257|2441560|2440898|2441171|N|N|N|N|N| +2441264|AAAAAAAAADAEFCAA|1971-11-08|862|3749|288|1971|1|11|8|4|1971|288|3749|Monday|1971Q4|N|N|N|2441257|2441560|2440899|2441172|N|N|N|N|N| +2441265|AAAAAAAABDAEFCAA|1971-11-09|862|3750|288|1971|2|11|9|4|1971|288|3750|Tuesday|1971Q4|N|N|N|2441257|2441560|2440900|2441173|N|N|N|N|N| +2441266|AAAAAAAACDAEFCAA|1971-11-10|862|3750|288|1971|3|11|10|4|1971|288|3750|Wednesday|1971Q4|N|N|N|2441257|2441560|2440901|2441174|N|N|N|N|N| +2441267|AAAAAAAADDAEFCAA|1971-11-11|862|3750|288|1971|4|11|11|4|1971|288|3750|Thursday|1971Q4|N|N|N|2441257|2441560|2440902|2441175|N|N|N|N|N| +2441268|AAAAAAAAEDAEFCAA|1971-11-12|862|3750|288|1971|5|11|12|4|1971|288|3750|Friday|1971Q4|N|Y|N|2441257|2441560|2440903|2441176|N|N|N|N|N| +2441269|AAAAAAAAFDAEFCAA|1971-11-13|862|3750|288|1971|6|11|13|4|1971|288|3750|Saturday|1971Q4|N|Y|N|2441257|2441560|2440904|2441177|N|N|N|N|N| +2441270|AAAAAAAAGDAEFCAA|1971-11-14|862|3750|288|1971|0|11|14|4|1971|288|3750|Sunday|1971Q4|N|N|N|2441257|2441560|2440905|2441178|N|N|N|N|N| +2441271|AAAAAAAAHDAEFCAA|1971-11-15|862|3750|288|1971|1|11|15|4|1971|288|3750|Monday|1971Q4|N|N|N|2441257|2441560|2440906|2441179|N|N|N|N|N| +2441272|AAAAAAAAIDAEFCAA|1971-11-16|862|3751|288|1971|2|11|16|4|1971|288|3751|Tuesday|1971Q4|N|N|N|2441257|2441560|2440907|2441180|N|N|N|N|N| +2441273|AAAAAAAAJDAEFCAA|1971-11-17|862|3751|288|1971|3|11|17|4|1971|288|3751|Wednesday|1971Q4|N|N|N|2441257|2441560|2440908|2441181|N|N|N|N|N| +2441274|AAAAAAAAKDAEFCAA|1971-11-18|862|3751|288|1971|4|11|18|4|1971|288|3751|Thursday|1971Q4|N|N|N|2441257|2441560|2440909|2441182|N|N|N|N|N| +2441275|AAAAAAAALDAEFCAA|1971-11-19|862|3751|288|1971|5|11|19|4|1971|288|3751|Friday|1971Q4|N|Y|N|2441257|2441560|2440910|2441183|N|N|N|N|N| +2441276|AAAAAAAAMDAEFCAA|1971-11-20|862|3751|288|1971|6|11|20|4|1971|288|3751|Saturday|1971Q4|N|Y|N|2441257|2441560|2440911|2441184|N|N|N|N|N| +2441277|AAAAAAAANDAEFCAA|1971-11-21|862|3751|288|1971|0|11|21|4|1971|288|3751|Sunday|1971Q4|N|N|N|2441257|2441560|2440912|2441185|N|N|N|N|N| +2441278|AAAAAAAAODAEFCAA|1971-11-22|862|3751|288|1971|1|11|22|4|1971|288|3751|Monday|1971Q4|N|N|N|2441257|2441560|2440913|2441186|N|N|N|N|N| +2441279|AAAAAAAAPDAEFCAA|1971-11-23|862|3752|288|1971|2|11|23|4|1971|288|3752|Tuesday|1971Q4|N|N|N|2441257|2441560|2440914|2441187|N|N|N|N|N| +2441280|AAAAAAAAAEAEFCAA|1971-11-24|862|3752|288|1971|3|11|24|4|1971|288|3752|Wednesday|1971Q4|N|N|N|2441257|2441560|2440915|2441188|N|N|N|N|N| +2441281|AAAAAAAABEAEFCAA|1971-11-25|862|3752|288|1971|4|11|25|4|1971|288|3752|Thursday|1971Q4|N|N|N|2441257|2441560|2440916|2441189|N|N|N|N|N| +2441282|AAAAAAAACEAEFCAA|1971-11-26|862|3752|288|1971|5|11|26|4|1971|288|3752|Friday|1971Q4|N|Y|N|2441257|2441560|2440917|2441190|N|N|N|N|N| +2441283|AAAAAAAADEAEFCAA|1971-11-27|862|3752|288|1971|6|11|27|4|1971|288|3752|Saturday|1971Q4|N|Y|N|2441257|2441560|2440918|2441191|N|N|N|N|N| +2441284|AAAAAAAAEEAEFCAA|1971-11-28|862|3752|288|1971|0|11|28|4|1971|288|3752|Sunday|1971Q4|N|N|N|2441257|2441560|2440919|2441192|N|N|N|N|N| +2441285|AAAAAAAAFEAEFCAA|1971-11-29|862|3752|288|1971|1|11|29|4|1971|288|3752|Monday|1971Q4|N|N|N|2441257|2441560|2440920|2441193|N|N|N|N|N| +2441286|AAAAAAAAGEAEFCAA|1971-11-30|862|3753|288|1971|2|11|30|4|1971|288|3753|Tuesday|1971Q4|N|N|N|2441257|2441560|2440921|2441194|N|N|N|N|N| +2441287|AAAAAAAAHEAEFCAA|1971-12-01|863|3753|289|1971|3|12|1|4|1971|289|3753|Wednesday|1971Q4|N|N|N|2441287|2441620|2440922|2441195|N|N|N|N|N| +2441288|AAAAAAAAIEAEFCAA|1971-12-02|863|3753|289|1971|4|12|2|4|1971|289|3753|Thursday|1971Q4|N|N|N|2441287|2441620|2440923|2441196|N|N|N|N|N| +2441289|AAAAAAAAJEAEFCAA|1971-12-03|863|3753|289|1971|5|12|3|4|1971|289|3753|Friday|1971Q4|N|Y|N|2441287|2441620|2440924|2441197|N|N|N|N|N| +2441290|AAAAAAAAKEAEFCAA|1971-12-04|863|3753|289|1971|6|12|4|4|1971|289|3753|Saturday|1971Q4|N|Y|N|2441287|2441620|2440925|2441198|N|N|N|N|N| +2441291|AAAAAAAALEAEFCAA|1971-12-05|863|3753|289|1971|0|12|5|4|1971|289|3753|Sunday|1971Q4|N|N|N|2441287|2441620|2440926|2441199|N|N|N|N|N| +2441292|AAAAAAAAMEAEFCAA|1971-12-06|863|3753|289|1971|1|12|6|4|1971|289|3753|Monday|1971Q4|N|N|N|2441287|2441620|2440927|2441200|N|N|N|N|N| +2441293|AAAAAAAANEAEFCAA|1971-12-07|863|3754|289|1971|2|12|7|4|1971|289|3754|Tuesday|1971Q4|N|N|N|2441287|2441620|2440928|2441201|N|N|N|N|N| +2441294|AAAAAAAAOEAEFCAA|1971-12-08|863|3754|289|1971|3|12|8|4|1971|289|3754|Wednesday|1971Q4|N|N|N|2441287|2441620|2440929|2441202|N|N|N|N|N| +2441295|AAAAAAAAPEAEFCAA|1971-12-09|863|3754|289|1971|4|12|9|4|1971|289|3754|Thursday|1971Q4|N|N|N|2441287|2441620|2440930|2441203|N|N|N|N|N| +2441296|AAAAAAAAAFAEFCAA|1971-12-10|863|3754|289|1971|5|12|10|4|1971|289|3754|Friday|1971Q4|N|Y|N|2441287|2441620|2440931|2441204|N|N|N|N|N| +2441297|AAAAAAAABFAEFCAA|1971-12-11|863|3754|289|1971|6|12|11|4|1971|289|3754|Saturday|1971Q4|N|Y|N|2441287|2441620|2440932|2441205|N|N|N|N|N| +2441298|AAAAAAAACFAEFCAA|1971-12-12|863|3754|289|1971|0|12|12|4|1971|289|3754|Sunday|1971Q4|N|N|N|2441287|2441620|2440933|2441206|N|N|N|N|N| +2441299|AAAAAAAADFAEFCAA|1971-12-13|863|3754|289|1971|1|12|13|4|1971|289|3754|Monday|1971Q4|N|N|N|2441287|2441620|2440934|2441207|N|N|N|N|N| +2441300|AAAAAAAAEFAEFCAA|1971-12-14|863|3755|289|1971|2|12|14|4|1971|289|3755|Tuesday|1971Q4|N|N|N|2441287|2441620|2440935|2441208|N|N|N|N|N| +2441301|AAAAAAAAFFAEFCAA|1971-12-15|863|3755|289|1971|3|12|15|4|1971|289|3755|Wednesday|1971Q4|N|N|N|2441287|2441620|2440936|2441209|N|N|N|N|N| +2441302|AAAAAAAAGFAEFCAA|1971-12-16|863|3755|289|1971|4|12|16|4|1971|289|3755|Thursday|1971Q4|N|N|N|2441287|2441620|2440937|2441210|N|N|N|N|N| +2441303|AAAAAAAAHFAEFCAA|1971-12-17|863|3755|289|1971|5|12|17|4|1971|289|3755|Friday|1971Q4|N|Y|N|2441287|2441620|2440938|2441211|N|N|N|N|N| +2441304|AAAAAAAAIFAEFCAA|1971-12-18|863|3755|289|1971|6|12|18|4|1971|289|3755|Saturday|1971Q4|N|Y|N|2441287|2441620|2440939|2441212|N|N|N|N|N| +2441305|AAAAAAAAJFAEFCAA|1971-12-19|863|3755|289|1971|0|12|19|4|1971|289|3755|Sunday|1971Q4|N|N|N|2441287|2441620|2440940|2441213|N|N|N|N|N| +2441306|AAAAAAAAKFAEFCAA|1971-12-20|863|3755|289|1971|1|12|20|4|1971|289|3755|Monday|1971Q4|N|N|N|2441287|2441620|2440941|2441214|N|N|N|N|N| +2441307|AAAAAAAALFAEFCAA|1971-12-21|863|3756|289|1971|2|12|21|4|1971|289|3756|Tuesday|1971Q4|N|N|N|2441287|2441620|2440942|2441215|N|N|N|N|N| +2441308|AAAAAAAAMFAEFCAA|1971-12-22|863|3756|289|1971|3|12|22|4|1971|289|3756|Wednesday|1971Q4|N|N|N|2441287|2441620|2440943|2441216|N|N|N|N|N| +2441309|AAAAAAAANFAEFCAA|1971-12-23|863|3756|289|1971|4|12|23|4|1971|289|3756|Thursday|1971Q4|N|N|N|2441287|2441620|2440944|2441217|N|N|N|N|N| +2441310|AAAAAAAAOFAEFCAA|1971-12-24|863|3756|289|1971|5|12|24|4|1971|289|3756|Friday|1971Q4|N|Y|N|2441287|2441620|2440945|2441218|N|N|N|N|N| +2441311|AAAAAAAAPFAEFCAA|1971-12-25|863|3756|289|1971|6|12|25|4|1971|289|3756|Saturday|1971Q4|N|Y|N|2441287|2441620|2440946|2441219|N|N|N|N|N| +2441312|AAAAAAAAAGAEFCAA|1971-12-26|863|3756|289|1971|0|12|26|4|1971|289|3756|Sunday|1971Q4|Y|N|N|2441287|2441620|2440947|2441220|N|N|N|N|N| +2441313|AAAAAAAABGAEFCAA|1971-12-27|863|3756|289|1971|1|12|27|4|1971|289|3756|Monday|1971Q4|N|N|Y|2441287|2441620|2440948|2441221|N|N|N|N|N| +2441314|AAAAAAAACGAEFCAA|1971-12-28|863|3757|289|1971|2|12|28|4|1971|289|3757|Tuesday|1971Q4|N|N|N|2441287|2441620|2440949|2441222|N|N|N|N|N| +2441315|AAAAAAAADGAEFCAA|1971-12-29|863|3757|289|1971|3|12|29|4|1971|289|3757|Wednesday|1971Q4|N|N|N|2441287|2441620|2440950|2441223|N|N|N|N|N| +2441316|AAAAAAAAEGAEFCAA|1971-12-30|863|3757|289|1971|4|12|30|4|1971|289|3757|Thursday|1971Q4|N|N|N|2441287|2441620|2440951|2441224|N|N|N|N|N| +2441317|AAAAAAAAFGAEFCAA|1971-12-31|863|3757|289|1971|5|12|31|4|1971|289|3757|Friday|1971Q4|N|Y|N|2441287|2441620|2440952|2441225|N|N|N|N|N| +2441318|AAAAAAAAGGAEFCAA|1972-01-01|864|3757|289|1972|6|1|1|1|1972|289|3757|Saturday|1972Q1|Y|Y|N|2441318|2441317|2440953|2441226|N|N|N|N|N| +2441319|AAAAAAAAHGAEFCAA|1972-01-02|864|3757|289|1972|0|1|2|1|1972|289|3757|Sunday|1972Q1|N|N|Y|2441318|2441317|2440954|2441227|N|N|N|N|N| +2441320|AAAAAAAAIGAEFCAA|1972-01-03|864|3757|289|1972|1|1|3|1|1972|289|3757|Monday|1972Q1|N|N|N|2441318|2441317|2440955|2441228|N|N|N|N|N| +2441321|AAAAAAAAJGAEFCAA|1972-01-04|864|3758|289|1972|2|1|4|1|1972|289|3758|Tuesday|1972Q1|N|N|N|2441318|2441317|2440956|2441229|N|N|N|N|N| +2441322|AAAAAAAAKGAEFCAA|1972-01-05|864|3758|289|1972|3|1|5|1|1972|289|3758|Wednesday|1972Q1|N|N|N|2441318|2441317|2440957|2441230|N|N|N|N|N| +2441323|AAAAAAAALGAEFCAA|1972-01-06|864|3758|289|1972|4|1|6|1|1972|289|3758|Thursday|1972Q1|N|N|N|2441318|2441317|2440958|2441231|N|N|N|N|N| +2441324|AAAAAAAAMGAEFCAA|1972-01-07|864|3758|289|1972|5|1|7|1|1972|289|3758|Friday|1972Q1|N|Y|N|2441318|2441317|2440959|2441232|N|N|N|N|N| +2441325|AAAAAAAANGAEFCAA|1972-01-08|864|3758|289|1972|6|1|8|1|1972|289|3758|Saturday|1972Q1|N|Y|N|2441318|2441317|2440960|2441233|N|N|N|N|N| +2441326|AAAAAAAAOGAEFCAA|1972-01-09|864|3758|289|1972|0|1|9|1|1972|289|3758|Sunday|1972Q1|N|N|N|2441318|2441317|2440961|2441234|N|N|N|N|N| +2441327|AAAAAAAAPGAEFCAA|1972-01-10|864|3758|289|1972|1|1|10|1|1972|289|3758|Monday|1972Q1|N|N|N|2441318|2441317|2440962|2441235|N|N|N|N|N| +2441328|AAAAAAAAAHAEFCAA|1972-01-11|864|3759|289|1972|2|1|11|1|1972|289|3759|Tuesday|1972Q1|N|N|N|2441318|2441317|2440963|2441236|N|N|N|N|N| +2441329|AAAAAAAABHAEFCAA|1972-01-12|864|3759|289|1972|3|1|12|1|1972|289|3759|Wednesday|1972Q1|N|N|N|2441318|2441317|2440964|2441237|N|N|N|N|N| +2441330|AAAAAAAACHAEFCAA|1972-01-13|864|3759|289|1972|4|1|13|1|1972|289|3759|Thursday|1972Q1|N|N|N|2441318|2441317|2440965|2441238|N|N|N|N|N| +2441331|AAAAAAAADHAEFCAA|1972-01-14|864|3759|289|1972|5|1|14|1|1972|289|3759|Friday|1972Q1|N|Y|N|2441318|2441317|2440966|2441239|N|N|N|N|N| +2441332|AAAAAAAAEHAEFCAA|1972-01-15|864|3759|289|1972|6|1|15|1|1972|289|3759|Saturday|1972Q1|N|Y|N|2441318|2441317|2440967|2441240|N|N|N|N|N| +2441333|AAAAAAAAFHAEFCAA|1972-01-16|864|3759|289|1972|0|1|16|1|1972|289|3759|Sunday|1972Q1|N|N|N|2441318|2441317|2440968|2441241|N|N|N|N|N| +2441334|AAAAAAAAGHAEFCAA|1972-01-17|864|3759|289|1972|1|1|17|1|1972|289|3759|Monday|1972Q1|N|N|N|2441318|2441317|2440969|2441242|N|N|N|N|N| +2441335|AAAAAAAAHHAEFCAA|1972-01-18|864|3760|289|1972|2|1|18|1|1972|289|3760|Tuesday|1972Q1|N|N|N|2441318|2441317|2440970|2441243|N|N|N|N|N| +2441336|AAAAAAAAIHAEFCAA|1972-01-19|864|3760|289|1972|3|1|19|1|1972|289|3760|Wednesday|1972Q1|N|N|N|2441318|2441317|2440971|2441244|N|N|N|N|N| +2441337|AAAAAAAAJHAEFCAA|1972-01-20|864|3760|289|1972|4|1|20|1|1972|289|3760|Thursday|1972Q1|N|N|N|2441318|2441317|2440972|2441245|N|N|N|N|N| +2441338|AAAAAAAAKHAEFCAA|1972-01-21|864|3760|289|1972|5|1|21|1|1972|289|3760|Friday|1972Q1|N|Y|N|2441318|2441317|2440973|2441246|N|N|N|N|N| +2441339|AAAAAAAALHAEFCAA|1972-01-22|864|3760|289|1972|6|1|22|1|1972|289|3760|Saturday|1972Q1|N|Y|N|2441318|2441317|2440974|2441247|N|N|N|N|N| +2441340|AAAAAAAAMHAEFCAA|1972-01-23|864|3760|289|1972|0|1|23|1|1972|289|3760|Sunday|1972Q1|N|N|N|2441318|2441317|2440975|2441248|N|N|N|N|N| +2441341|AAAAAAAANHAEFCAA|1972-01-24|864|3760|289|1972|1|1|24|1|1972|289|3760|Monday|1972Q1|N|N|N|2441318|2441317|2440976|2441249|N|N|N|N|N| +2441342|AAAAAAAAOHAEFCAA|1972-01-25|864|3761|289|1972|2|1|25|1|1972|289|3761|Tuesday|1972Q1|N|N|N|2441318|2441317|2440977|2441250|N|N|N|N|N| +2441343|AAAAAAAAPHAEFCAA|1972-01-26|864|3761|289|1972|3|1|26|1|1972|289|3761|Wednesday|1972Q1|N|N|N|2441318|2441317|2440978|2441251|N|N|N|N|N| +2441344|AAAAAAAAAIAEFCAA|1972-01-27|864|3761|289|1972|4|1|27|1|1972|289|3761|Thursday|1972Q1|N|N|N|2441318|2441317|2440979|2441252|N|N|N|N|N| +2441345|AAAAAAAABIAEFCAA|1972-01-28|864|3761|289|1972|5|1|28|1|1972|289|3761|Friday|1972Q1|N|Y|N|2441318|2441317|2440980|2441253|N|N|N|N|N| +2441346|AAAAAAAACIAEFCAA|1972-01-29|864|3761|289|1972|6|1|29|1|1972|289|3761|Saturday|1972Q1|N|Y|N|2441318|2441317|2440981|2441254|N|N|N|N|N| +2441347|AAAAAAAADIAEFCAA|1972-01-30|864|3761|289|1972|0|1|30|1|1972|289|3761|Sunday|1972Q1|N|N|N|2441318|2441317|2440982|2441255|N|N|N|N|N| +2441348|AAAAAAAAEIAEFCAA|1972-01-31|864|3761|289|1972|1|1|31|1|1972|289|3761|Monday|1972Q1|N|N|N|2441318|2441317|2440983|2441256|N|N|N|N|N| +2441349|AAAAAAAAFIAEFCAA|1972-02-01|865|3762|289|1972|2|2|1|1|1972|289|3762|Tuesday|1972Q1|N|N|N|2441349|2441379|2440984|2441257|N|N|N|N|N| +2441350|AAAAAAAAGIAEFCAA|1972-02-02|865|3762|289|1972|3|2|2|1|1972|289|3762|Wednesday|1972Q1|N|N|N|2441349|2441379|2440985|2441258|N|N|N|N|N| +2441351|AAAAAAAAHIAEFCAA|1972-02-03|865|3762|289|1972|4|2|3|1|1972|289|3762|Thursday|1972Q1|N|N|N|2441349|2441379|2440986|2441259|N|N|N|N|N| +2441352|AAAAAAAAIIAEFCAA|1972-02-04|865|3762|289|1972|5|2|4|1|1972|289|3762|Friday|1972Q1|N|Y|N|2441349|2441379|2440987|2441260|N|N|N|N|N| +2441353|AAAAAAAAJIAEFCAA|1972-02-05|865|3762|289|1972|6|2|5|1|1972|289|3762|Saturday|1972Q1|N|Y|N|2441349|2441379|2440988|2441261|N|N|N|N|N| +2441354|AAAAAAAAKIAEFCAA|1972-02-06|865|3762|289|1972|0|2|6|1|1972|289|3762|Sunday|1972Q1|N|N|N|2441349|2441379|2440989|2441262|N|N|N|N|N| +2441355|AAAAAAAALIAEFCAA|1972-02-07|865|3762|289|1972|1|2|7|1|1972|289|3762|Monday|1972Q1|N|N|N|2441349|2441379|2440990|2441263|N|N|N|N|N| +2441356|AAAAAAAAMIAEFCAA|1972-02-08|865|3763|289|1972|2|2|8|1|1972|289|3763|Tuesday|1972Q1|N|N|N|2441349|2441379|2440991|2441264|N|N|N|N|N| +2441357|AAAAAAAANIAEFCAA|1972-02-09|865|3763|289|1972|3|2|9|1|1972|289|3763|Wednesday|1972Q1|N|N|N|2441349|2441379|2440992|2441265|N|N|N|N|N| +2441358|AAAAAAAAOIAEFCAA|1972-02-10|865|3763|289|1972|4|2|10|1|1972|289|3763|Thursday|1972Q1|N|N|N|2441349|2441379|2440993|2441266|N|N|N|N|N| +2441359|AAAAAAAAPIAEFCAA|1972-02-11|865|3763|289|1972|5|2|11|1|1972|289|3763|Friday|1972Q1|N|Y|N|2441349|2441379|2440994|2441267|N|N|N|N|N| +2441360|AAAAAAAAAJAEFCAA|1972-02-12|865|3763|289|1972|6|2|12|1|1972|289|3763|Saturday|1972Q1|N|Y|N|2441349|2441379|2440995|2441268|N|N|N|N|N| +2441361|AAAAAAAABJAEFCAA|1972-02-13|865|3763|289|1972|0|2|13|1|1972|289|3763|Sunday|1972Q1|N|N|N|2441349|2441379|2440996|2441269|N|N|N|N|N| +2441362|AAAAAAAACJAEFCAA|1972-02-14|865|3763|289|1972|1|2|14|1|1972|289|3763|Monday|1972Q1|N|N|N|2441349|2441379|2440997|2441270|N|N|N|N|N| +2441363|AAAAAAAADJAEFCAA|1972-02-15|865|3764|289|1972|2|2|15|1|1972|289|3764|Tuesday|1972Q1|N|N|N|2441349|2441379|2440998|2441271|N|N|N|N|N| +2441364|AAAAAAAAEJAEFCAA|1972-02-16|865|3764|289|1972|3|2|16|1|1972|289|3764|Wednesday|1972Q1|N|N|N|2441349|2441379|2440999|2441272|N|N|N|N|N| +2441365|AAAAAAAAFJAEFCAA|1972-02-17|865|3764|289|1972|4|2|17|1|1972|289|3764|Thursday|1972Q1|N|N|N|2441349|2441379|2441000|2441273|N|N|N|N|N| +2441366|AAAAAAAAGJAEFCAA|1972-02-18|865|3764|289|1972|5|2|18|1|1972|289|3764|Friday|1972Q1|N|Y|N|2441349|2441379|2441001|2441274|N|N|N|N|N| +2441367|AAAAAAAAHJAEFCAA|1972-02-19|865|3764|289|1972|6|2|19|1|1972|289|3764|Saturday|1972Q1|N|Y|N|2441349|2441379|2441002|2441275|N|N|N|N|N| +2441368|AAAAAAAAIJAEFCAA|1972-02-20|865|3764|289|1972|0|2|20|1|1972|289|3764|Sunday|1972Q1|N|N|N|2441349|2441379|2441003|2441276|N|N|N|N|N| +2441369|AAAAAAAAJJAEFCAA|1972-02-21|865|3764|289|1972|1|2|21|1|1972|289|3764|Monday|1972Q1|N|N|N|2441349|2441379|2441004|2441277|N|N|N|N|N| +2441370|AAAAAAAAKJAEFCAA|1972-02-22|865|3765|289|1972|2|2|22|1|1972|289|3765|Tuesday|1972Q1|N|N|N|2441349|2441379|2441005|2441278|N|N|N|N|N| +2441371|AAAAAAAALJAEFCAA|1972-02-23|865|3765|289|1972|3|2|23|1|1972|289|3765|Wednesday|1972Q1|N|N|N|2441349|2441379|2441006|2441279|N|N|N|N|N| +2441372|AAAAAAAAMJAEFCAA|1972-02-24|865|3765|289|1972|4|2|24|1|1972|289|3765|Thursday|1972Q1|N|N|N|2441349|2441379|2441007|2441280|N|N|N|N|N| +2441373|AAAAAAAANJAEFCAA|1972-02-25|865|3765|289|1972|5|2|25|1|1972|289|3765|Friday|1972Q1|N|Y|N|2441349|2441379|2441008|2441281|N|N|N|N|N| +2441374|AAAAAAAAOJAEFCAA|1972-02-26|865|3765|289|1972|6|2|26|1|1972|289|3765|Saturday|1972Q1|N|Y|N|2441349|2441379|2441009|2441282|N|N|N|N|N| +2441375|AAAAAAAAPJAEFCAA|1972-02-27|865|3765|289|1972|0|2|27|1|1972|289|3765|Sunday|1972Q1|N|N|N|2441349|2441379|2441010|2441283|N|N|N|N|N| +2441376|AAAAAAAAAKAEFCAA|1972-02-28|865|3765|289|1972|1|2|28|1|1972|289|3765|Monday|1972Q1|N|N|N|2441349|2441379|2441011|2441284|N|N|N|N|N| +2441377|AAAAAAAABKAEFCAA|1972-02-29|865|3766|289|1972|2|2|29|1|1972|289|3766|Tuesday|1972Q1|N|N|N|2441349|2441379|2441011|2441285|N|N|N|N|N| +2441378|AAAAAAAACKAEFCAA|1972-03-01|866|3766|290|1972|3|3|1|1|1972|290|3766|Wednesday|1972Q1|N|N|N|2441378|2441437|2441012|2441286|N|N|N|N|N| +2441379|AAAAAAAADKAEFCAA|1972-03-02|866|3766|290|1972|4|3|2|1|1972|290|3766|Thursday|1972Q1|N|N|N|2441378|2441437|2441013|2441287|N|N|N|N|N| +2441380|AAAAAAAAEKAEFCAA|1972-03-03|866|3766|290|1972|5|3|3|1|1972|290|3766|Friday|1972Q1|N|Y|N|2441378|2441437|2441014|2441288|N|N|N|N|N| +2441381|AAAAAAAAFKAEFCAA|1972-03-04|866|3766|290|1972|6|3|4|1|1972|290|3766|Saturday|1972Q1|N|Y|N|2441378|2441437|2441015|2441289|N|N|N|N|N| +2441382|AAAAAAAAGKAEFCAA|1972-03-05|866|3766|290|1972|0|3|5|1|1972|290|3766|Sunday|1972Q1|N|N|N|2441378|2441437|2441016|2441290|N|N|N|N|N| +2441383|AAAAAAAAHKAEFCAA|1972-03-06|866|3766|290|1972|1|3|6|1|1972|290|3766|Monday|1972Q1|N|N|N|2441378|2441437|2441017|2441291|N|N|N|N|N| +2441384|AAAAAAAAIKAEFCAA|1972-03-07|866|3767|290|1972|2|3|7|1|1972|290|3767|Tuesday|1972Q1|N|N|N|2441378|2441437|2441018|2441292|N|N|N|N|N| +2441385|AAAAAAAAJKAEFCAA|1972-03-08|866|3767|290|1972|3|3|8|1|1972|290|3767|Wednesday|1972Q1|N|N|N|2441378|2441437|2441019|2441293|N|N|N|N|N| +2441386|AAAAAAAAKKAEFCAA|1972-03-09|866|3767|290|1972|4|3|9|1|1972|290|3767|Thursday|1972Q1|N|N|N|2441378|2441437|2441020|2441294|N|N|N|N|N| +2441387|AAAAAAAALKAEFCAA|1972-03-10|866|3767|290|1972|5|3|10|1|1972|290|3767|Friday|1972Q1|N|Y|N|2441378|2441437|2441021|2441295|N|N|N|N|N| +2441388|AAAAAAAAMKAEFCAA|1972-03-11|866|3767|290|1972|6|3|11|1|1972|290|3767|Saturday|1972Q1|N|Y|N|2441378|2441437|2441022|2441296|N|N|N|N|N| +2441389|AAAAAAAANKAEFCAA|1972-03-12|866|3767|290|1972|0|3|12|1|1972|290|3767|Sunday|1972Q1|N|N|N|2441378|2441437|2441023|2441297|N|N|N|N|N| +2441390|AAAAAAAAOKAEFCAA|1972-03-13|866|3767|290|1972|1|3|13|1|1972|290|3767|Monday|1972Q1|N|N|N|2441378|2441437|2441024|2441298|N|N|N|N|N| +2441391|AAAAAAAAPKAEFCAA|1972-03-14|866|3768|290|1972|2|3|14|1|1972|290|3768|Tuesday|1972Q1|N|N|N|2441378|2441437|2441025|2441299|N|N|N|N|N| +2441392|AAAAAAAAALAEFCAA|1972-03-15|866|3768|290|1972|3|3|15|1|1972|290|3768|Wednesday|1972Q1|N|N|N|2441378|2441437|2441026|2441300|N|N|N|N|N| +2441393|AAAAAAAABLAEFCAA|1972-03-16|866|3768|290|1972|4|3|16|1|1972|290|3768|Thursday|1972Q1|N|N|N|2441378|2441437|2441027|2441301|N|N|N|N|N| +2441394|AAAAAAAACLAEFCAA|1972-03-17|866|3768|290|1972|5|3|17|1|1972|290|3768|Friday|1972Q1|N|Y|N|2441378|2441437|2441028|2441302|N|N|N|N|N| +2441395|AAAAAAAADLAEFCAA|1972-03-18|866|3768|290|1972|6|3|18|1|1972|290|3768|Saturday|1972Q1|N|Y|N|2441378|2441437|2441029|2441303|N|N|N|N|N| +2441396|AAAAAAAAELAEFCAA|1972-03-19|866|3768|290|1972|0|3|19|1|1972|290|3768|Sunday|1972Q1|N|N|N|2441378|2441437|2441030|2441304|N|N|N|N|N| +2441397|AAAAAAAAFLAEFCAA|1972-03-20|866|3768|290|1972|1|3|20|1|1972|290|3768|Monday|1972Q1|N|N|N|2441378|2441437|2441031|2441305|N|N|N|N|N| +2441398|AAAAAAAAGLAEFCAA|1972-03-21|866|3769|290|1972|2|3|21|1|1972|290|3769|Tuesday|1972Q1|N|N|N|2441378|2441437|2441032|2441306|N|N|N|N|N| +2441399|AAAAAAAAHLAEFCAA|1972-03-22|866|3769|290|1972|3|3|22|1|1972|290|3769|Wednesday|1972Q1|N|N|N|2441378|2441437|2441033|2441307|N|N|N|N|N| +2441400|AAAAAAAAILAEFCAA|1972-03-23|866|3769|290|1972|4|3|23|1|1972|290|3769|Thursday|1972Q1|N|N|N|2441378|2441437|2441034|2441308|N|N|N|N|N| +2441401|AAAAAAAAJLAEFCAA|1972-03-24|866|3769|290|1972|5|3|24|1|1972|290|3769|Friday|1972Q1|N|Y|N|2441378|2441437|2441035|2441309|N|N|N|N|N| +2441402|AAAAAAAAKLAEFCAA|1972-03-25|866|3769|290|1972|6|3|25|1|1972|290|3769|Saturday|1972Q1|N|Y|N|2441378|2441437|2441036|2441310|N|N|N|N|N| +2441403|AAAAAAAALLAEFCAA|1972-03-26|866|3769|290|1972|0|3|26|1|1972|290|3769|Sunday|1972Q1|N|N|N|2441378|2441437|2441037|2441311|N|N|N|N|N| +2441404|AAAAAAAAMLAEFCAA|1972-03-27|866|3769|290|1972|1|3|27|1|1972|290|3769|Monday|1972Q1|N|N|N|2441378|2441437|2441038|2441312|N|N|N|N|N| +2441405|AAAAAAAANLAEFCAA|1972-03-28|866|3770|290|1972|2|3|28|1|1972|290|3770|Tuesday|1972Q1|N|N|N|2441378|2441437|2441039|2441313|N|N|N|N|N| +2441406|AAAAAAAAOLAEFCAA|1972-03-29|866|3770|290|1972|3|3|29|1|1972|290|3770|Wednesday|1972Q1|N|N|N|2441378|2441437|2441040|2441314|N|N|N|N|N| +2441407|AAAAAAAAPLAEFCAA|1972-03-30|866|3770|290|1972|4|3|30|1|1972|290|3770|Thursday|1972Q1|N|N|N|2441378|2441437|2441041|2441315|N|N|N|N|N| +2441408|AAAAAAAAAMAEFCAA|1972-03-31|866|3770|290|1972|5|3|31|1|1972|290|3770|Friday|1972Q1|N|Y|N|2441378|2441437|2441042|2441316|N|N|N|N|N| +2441409|AAAAAAAABMAEFCAA|1972-04-01|867|3770|290|1972|6|4|1|2|1972|290|3770|Saturday|1972Q2|N|Y|N|2441409|2441499|2441043|2441318|N|N|N|N|N| +2441410|AAAAAAAACMAEFCAA|1972-04-02|867|3770|290|1972|0|4|2|2|1972|290|3770|Sunday|1972Q2|N|N|N|2441409|2441499|2441044|2441319|N|N|N|N|N| +2441411|AAAAAAAADMAEFCAA|1972-04-03|867|3770|290|1972|1|4|3|2|1972|290|3770|Monday|1972Q2|N|N|N|2441409|2441499|2441045|2441320|N|N|N|N|N| +2441412|AAAAAAAAEMAEFCAA|1972-04-04|867|3771|290|1972|2|4|4|2|1972|290|3771|Tuesday|1972Q2|N|N|N|2441409|2441499|2441046|2441321|N|N|N|N|N| +2441413|AAAAAAAAFMAEFCAA|1972-04-05|867|3771|290|1972|3|4|5|2|1972|290|3771|Wednesday|1972Q2|N|N|N|2441409|2441499|2441047|2441322|N|N|N|N|N| +2441414|AAAAAAAAGMAEFCAA|1972-04-06|867|3771|290|1972|4|4|6|2|1972|290|3771|Thursday|1972Q2|N|N|N|2441409|2441499|2441048|2441323|N|N|N|N|N| +2441415|AAAAAAAAHMAEFCAA|1972-04-07|867|3771|290|1972|5|4|7|2|1972|290|3771|Friday|1972Q2|N|Y|N|2441409|2441499|2441049|2441324|N|N|N|N|N| +2441416|AAAAAAAAIMAEFCAA|1972-04-08|867|3771|290|1972|6|4|8|2|1972|290|3771|Saturday|1972Q2|N|Y|N|2441409|2441499|2441050|2441325|N|N|N|N|N| +2441417|AAAAAAAAJMAEFCAA|1972-04-09|867|3771|290|1972|0|4|9|2|1972|290|3771|Sunday|1972Q2|N|N|N|2441409|2441499|2441051|2441326|N|N|N|N|N| +2441418|AAAAAAAAKMAEFCAA|1972-04-10|867|3771|290|1972|1|4|10|2|1972|290|3771|Monday|1972Q2|N|N|N|2441409|2441499|2441052|2441327|N|N|N|N|N| +2441419|AAAAAAAALMAEFCAA|1972-04-11|867|3772|290|1972|2|4|11|2|1972|290|3772|Tuesday|1972Q2|N|N|N|2441409|2441499|2441053|2441328|N|N|N|N|N| +2441420|AAAAAAAAMMAEFCAA|1972-04-12|867|3772|290|1972|3|4|12|2|1972|290|3772|Wednesday|1972Q2|N|N|N|2441409|2441499|2441054|2441329|N|N|N|N|N| +2441421|AAAAAAAANMAEFCAA|1972-04-13|867|3772|290|1972|4|4|13|2|1972|290|3772|Thursday|1972Q2|N|N|N|2441409|2441499|2441055|2441330|N|N|N|N|N| +2441422|AAAAAAAAOMAEFCAA|1972-04-14|867|3772|290|1972|5|4|14|2|1972|290|3772|Friday|1972Q2|N|Y|N|2441409|2441499|2441056|2441331|N|N|N|N|N| +2441423|AAAAAAAAPMAEFCAA|1972-04-15|867|3772|290|1972|6|4|15|2|1972|290|3772|Saturday|1972Q2|N|Y|N|2441409|2441499|2441057|2441332|N|N|N|N|N| +2441424|AAAAAAAAANAEFCAA|1972-04-16|867|3772|290|1972|0|4|16|2|1972|290|3772|Sunday|1972Q2|N|N|N|2441409|2441499|2441058|2441333|N|N|N|N|N| +2441425|AAAAAAAABNAEFCAA|1972-04-17|867|3772|290|1972|1|4|17|2|1972|290|3772|Monday|1972Q2|N|N|N|2441409|2441499|2441059|2441334|N|N|N|N|N| +2441426|AAAAAAAACNAEFCAA|1972-04-18|867|3773|290|1972|2|4|18|2|1972|290|3773|Tuesday|1972Q2|N|N|N|2441409|2441499|2441060|2441335|N|N|N|N|N| +2441427|AAAAAAAADNAEFCAA|1972-04-19|867|3773|290|1972|3|4|19|2|1972|290|3773|Wednesday|1972Q2|N|N|N|2441409|2441499|2441061|2441336|N|N|N|N|N| +2441428|AAAAAAAAENAEFCAA|1972-04-20|867|3773|290|1972|4|4|20|2|1972|290|3773|Thursday|1972Q2|N|N|N|2441409|2441499|2441062|2441337|N|N|N|N|N| +2441429|AAAAAAAAFNAEFCAA|1972-04-21|867|3773|290|1972|5|4|21|2|1972|290|3773|Friday|1972Q2|N|Y|N|2441409|2441499|2441063|2441338|N|N|N|N|N| +2441430|AAAAAAAAGNAEFCAA|1972-04-22|867|3773|290|1972|6|4|22|2|1972|290|3773|Saturday|1972Q2|N|Y|N|2441409|2441499|2441064|2441339|N|N|N|N|N| +2441431|AAAAAAAAHNAEFCAA|1972-04-23|867|3773|290|1972|0|4|23|2|1972|290|3773|Sunday|1972Q2|N|N|N|2441409|2441499|2441065|2441340|N|N|N|N|N| +2441432|AAAAAAAAINAEFCAA|1972-04-24|867|3773|290|1972|1|4|24|2|1972|290|3773|Monday|1972Q2|N|N|N|2441409|2441499|2441066|2441341|N|N|N|N|N| +2441433|AAAAAAAAJNAEFCAA|1972-04-25|867|3774|290|1972|2|4|25|2|1972|290|3774|Tuesday|1972Q2|N|N|N|2441409|2441499|2441067|2441342|N|N|N|N|N| +2441434|AAAAAAAAKNAEFCAA|1972-04-26|867|3774|290|1972|3|4|26|2|1972|290|3774|Wednesday|1972Q2|N|N|N|2441409|2441499|2441068|2441343|N|N|N|N|N| +2441435|AAAAAAAALNAEFCAA|1972-04-27|867|3774|290|1972|4|4|27|2|1972|290|3774|Thursday|1972Q2|N|N|N|2441409|2441499|2441069|2441344|N|N|N|N|N| +2441436|AAAAAAAAMNAEFCAA|1972-04-28|867|3774|290|1972|5|4|28|2|1972|290|3774|Friday|1972Q2|N|Y|N|2441409|2441499|2441070|2441345|N|N|N|N|N| +2441437|AAAAAAAANNAEFCAA|1972-04-29|867|3774|290|1972|6|4|29|2|1972|290|3774|Saturday|1972Q2|N|Y|N|2441409|2441499|2441071|2441346|N|N|N|N|N| +2441438|AAAAAAAAONAEFCAA|1972-04-30|867|3774|290|1972|0|4|30|2|1972|290|3774|Sunday|1972Q2|N|N|N|2441409|2441499|2441072|2441347|N|N|N|N|N| +2441439|AAAAAAAAPNAEFCAA|1972-05-01|868|3774|290|1972|1|5|1|2|1972|290|3774|Monday|1972Q2|N|N|N|2441439|2441559|2441073|2441348|N|N|N|N|N| +2441440|AAAAAAAAAOAEFCAA|1972-05-02|868|3775|290|1972|2|5|2|2|1972|290|3775|Tuesday|1972Q2|N|N|N|2441439|2441559|2441074|2441349|N|N|N|N|N| +2441441|AAAAAAAABOAEFCAA|1972-05-03|868|3775|290|1972|3|5|3|2|1972|290|3775|Wednesday|1972Q2|N|N|N|2441439|2441559|2441075|2441350|N|N|N|N|N| +2441442|AAAAAAAACOAEFCAA|1972-05-04|868|3775|290|1972|4|5|4|2|1972|290|3775|Thursday|1972Q2|N|N|N|2441439|2441559|2441076|2441351|N|N|N|N|N| +2441443|AAAAAAAADOAEFCAA|1972-05-05|868|3775|290|1972|5|5|5|2|1972|290|3775|Friday|1972Q2|N|Y|N|2441439|2441559|2441077|2441352|N|N|N|N|N| +2441444|AAAAAAAAEOAEFCAA|1972-05-06|868|3775|290|1972|6|5|6|2|1972|290|3775|Saturday|1972Q2|N|Y|N|2441439|2441559|2441078|2441353|N|N|N|N|N| +2441445|AAAAAAAAFOAEFCAA|1972-05-07|868|3775|290|1972|0|5|7|2|1972|290|3775|Sunday|1972Q2|N|N|N|2441439|2441559|2441079|2441354|N|N|N|N|N| +2441446|AAAAAAAAGOAEFCAA|1972-05-08|868|3775|290|1972|1|5|8|2|1972|290|3775|Monday|1972Q2|N|N|N|2441439|2441559|2441080|2441355|N|N|N|N|N| +2441447|AAAAAAAAHOAEFCAA|1972-05-09|868|3776|290|1972|2|5|9|2|1972|290|3776|Tuesday|1972Q2|N|N|N|2441439|2441559|2441081|2441356|N|N|N|N|N| +2441448|AAAAAAAAIOAEFCAA|1972-05-10|868|3776|290|1972|3|5|10|2|1972|290|3776|Wednesday|1972Q2|N|N|N|2441439|2441559|2441082|2441357|N|N|N|N|N| +2441449|AAAAAAAAJOAEFCAA|1972-05-11|868|3776|290|1972|4|5|11|2|1972|290|3776|Thursday|1972Q2|N|N|N|2441439|2441559|2441083|2441358|N|N|N|N|N| +2441450|AAAAAAAAKOAEFCAA|1972-05-12|868|3776|290|1972|5|5|12|2|1972|290|3776|Friday|1972Q2|N|Y|N|2441439|2441559|2441084|2441359|N|N|N|N|N| +2441451|AAAAAAAALOAEFCAA|1972-05-13|868|3776|290|1972|6|5|13|2|1972|290|3776|Saturday|1972Q2|N|Y|N|2441439|2441559|2441085|2441360|N|N|N|N|N| +2441452|AAAAAAAAMOAEFCAA|1972-05-14|868|3776|290|1972|0|5|14|2|1972|290|3776|Sunday|1972Q2|N|N|N|2441439|2441559|2441086|2441361|N|N|N|N|N| +2441453|AAAAAAAANOAEFCAA|1972-05-15|868|3776|290|1972|1|5|15|2|1972|290|3776|Monday|1972Q2|N|N|N|2441439|2441559|2441087|2441362|N|N|N|N|N| +2441454|AAAAAAAAOOAEFCAA|1972-05-16|868|3777|290|1972|2|5|16|2|1972|290|3777|Tuesday|1972Q2|N|N|N|2441439|2441559|2441088|2441363|N|N|N|N|N| +2441455|AAAAAAAAPOAEFCAA|1972-05-17|868|3777|290|1972|3|5|17|2|1972|290|3777|Wednesday|1972Q2|N|N|N|2441439|2441559|2441089|2441364|N|N|N|N|N| +2441456|AAAAAAAAAPAEFCAA|1972-05-18|868|3777|290|1972|4|5|18|2|1972|290|3777|Thursday|1972Q2|N|N|N|2441439|2441559|2441090|2441365|N|N|N|N|N| +2441457|AAAAAAAABPAEFCAA|1972-05-19|868|3777|290|1972|5|5|19|2|1972|290|3777|Friday|1972Q2|N|Y|N|2441439|2441559|2441091|2441366|N|N|N|N|N| +2441458|AAAAAAAACPAEFCAA|1972-05-20|868|3777|290|1972|6|5|20|2|1972|290|3777|Saturday|1972Q2|N|Y|N|2441439|2441559|2441092|2441367|N|N|N|N|N| +2441459|AAAAAAAADPAEFCAA|1972-05-21|868|3777|290|1972|0|5|21|2|1972|290|3777|Sunday|1972Q2|N|N|N|2441439|2441559|2441093|2441368|N|N|N|N|N| +2441460|AAAAAAAAEPAEFCAA|1972-05-22|868|3777|290|1972|1|5|22|2|1972|290|3777|Monday|1972Q2|N|N|N|2441439|2441559|2441094|2441369|N|N|N|N|N| +2441461|AAAAAAAAFPAEFCAA|1972-05-23|868|3778|290|1972|2|5|23|2|1972|290|3778|Tuesday|1972Q2|N|N|N|2441439|2441559|2441095|2441370|N|N|N|N|N| +2441462|AAAAAAAAGPAEFCAA|1972-05-24|868|3778|290|1972|3|5|24|2|1972|290|3778|Wednesday|1972Q2|N|N|N|2441439|2441559|2441096|2441371|N|N|N|N|N| +2441463|AAAAAAAAHPAEFCAA|1972-05-25|868|3778|290|1972|4|5|25|2|1972|290|3778|Thursday|1972Q2|N|N|N|2441439|2441559|2441097|2441372|N|N|N|N|N| +2441464|AAAAAAAAIPAEFCAA|1972-05-26|868|3778|290|1972|5|5|26|2|1972|290|3778|Friday|1972Q2|N|Y|N|2441439|2441559|2441098|2441373|N|N|N|N|N| +2441465|AAAAAAAAJPAEFCAA|1972-05-27|868|3778|290|1972|6|5|27|2|1972|290|3778|Saturday|1972Q2|N|Y|N|2441439|2441559|2441099|2441374|N|N|N|N|N| +2441466|AAAAAAAAKPAEFCAA|1972-05-28|868|3778|290|1972|0|5|28|2|1972|290|3778|Sunday|1972Q2|N|N|N|2441439|2441559|2441100|2441375|N|N|N|N|N| +2441467|AAAAAAAALPAEFCAA|1972-05-29|868|3778|290|1972|1|5|29|2|1972|290|3778|Monday|1972Q2|N|N|N|2441439|2441559|2441101|2441376|N|N|N|N|N| +2441468|AAAAAAAAMPAEFCAA|1972-05-30|868|3779|290|1972|2|5|30|2|1972|290|3779|Tuesday|1972Q2|N|N|N|2441439|2441559|2441102|2441377|N|N|N|N|N| +2441469|AAAAAAAANPAEFCAA|1972-05-31|868|3779|290|1972|3|5|31|2|1972|290|3779|Wednesday|1972Q2|N|N|N|2441439|2441559|2441103|2441378|N|N|N|N|N| +2441470|AAAAAAAAOPAEFCAA|1972-06-01|869|3779|291|1972|4|6|1|2|1972|291|3779|Thursday|1972Q2|N|N|N|2441470|2441621|2441104|2441379|N|N|N|N|N| +2441471|AAAAAAAAPPAEFCAA|1972-06-02|869|3779|291|1972|5|6|2|2|1972|291|3779|Friday|1972Q2|N|Y|N|2441470|2441621|2441105|2441380|N|N|N|N|N| +2441472|AAAAAAAAAABEFCAA|1972-06-03|869|3779|291|1972|6|6|3|2|1972|291|3779|Saturday|1972Q2|N|Y|N|2441470|2441621|2441106|2441381|N|N|N|N|N| +2441473|AAAAAAAABABEFCAA|1972-06-04|869|3779|291|1972|0|6|4|2|1972|291|3779|Sunday|1972Q2|N|N|N|2441470|2441621|2441107|2441382|N|N|N|N|N| +2441474|AAAAAAAACABEFCAA|1972-06-05|869|3779|291|1972|1|6|5|2|1972|291|3779|Monday|1972Q2|N|N|N|2441470|2441621|2441108|2441383|N|N|N|N|N| +2441475|AAAAAAAADABEFCAA|1972-06-06|869|3780|291|1972|2|6|6|2|1972|291|3780|Tuesday|1972Q2|N|N|N|2441470|2441621|2441109|2441384|N|N|N|N|N| +2441476|AAAAAAAAEABEFCAA|1972-06-07|869|3780|291|1972|3|6|7|2|1972|291|3780|Wednesday|1972Q2|N|N|N|2441470|2441621|2441110|2441385|N|N|N|N|N| +2441477|AAAAAAAAFABEFCAA|1972-06-08|869|3780|291|1972|4|6|8|2|1972|291|3780|Thursday|1972Q2|N|N|N|2441470|2441621|2441111|2441386|N|N|N|N|N| +2441478|AAAAAAAAGABEFCAA|1972-06-09|869|3780|291|1972|5|6|9|2|1972|291|3780|Friday|1972Q2|N|Y|N|2441470|2441621|2441112|2441387|N|N|N|N|N| +2441479|AAAAAAAAHABEFCAA|1972-06-10|869|3780|291|1972|6|6|10|2|1972|291|3780|Saturday|1972Q2|N|Y|N|2441470|2441621|2441113|2441388|N|N|N|N|N| +2441480|AAAAAAAAIABEFCAA|1972-06-11|869|3780|291|1972|0|6|11|2|1972|291|3780|Sunday|1972Q2|N|N|N|2441470|2441621|2441114|2441389|N|N|N|N|N| +2441481|AAAAAAAAJABEFCAA|1972-06-12|869|3780|291|1972|1|6|12|2|1972|291|3780|Monday|1972Q2|N|N|N|2441470|2441621|2441115|2441390|N|N|N|N|N| +2441482|AAAAAAAAKABEFCAA|1972-06-13|869|3781|291|1972|2|6|13|2|1972|291|3781|Tuesday|1972Q2|N|N|N|2441470|2441621|2441116|2441391|N|N|N|N|N| +2441483|AAAAAAAALABEFCAA|1972-06-14|869|3781|291|1972|3|6|14|2|1972|291|3781|Wednesday|1972Q2|N|N|N|2441470|2441621|2441117|2441392|N|N|N|N|N| +2441484|AAAAAAAAMABEFCAA|1972-06-15|869|3781|291|1972|4|6|15|2|1972|291|3781|Thursday|1972Q2|N|N|N|2441470|2441621|2441118|2441393|N|N|N|N|N| +2441485|AAAAAAAANABEFCAA|1972-06-16|869|3781|291|1972|5|6|16|2|1972|291|3781|Friday|1972Q2|N|Y|N|2441470|2441621|2441119|2441394|N|N|N|N|N| +2441486|AAAAAAAAOABEFCAA|1972-06-17|869|3781|291|1972|6|6|17|2|1972|291|3781|Saturday|1972Q2|N|Y|N|2441470|2441621|2441120|2441395|N|N|N|N|N| +2441487|AAAAAAAAPABEFCAA|1972-06-18|869|3781|291|1972|0|6|18|2|1972|291|3781|Sunday|1972Q2|N|N|N|2441470|2441621|2441121|2441396|N|N|N|N|N| +2441488|AAAAAAAAABBEFCAA|1972-06-19|869|3781|291|1972|1|6|19|2|1972|291|3781|Monday|1972Q2|N|N|N|2441470|2441621|2441122|2441397|N|N|N|N|N| +2441489|AAAAAAAABBBEFCAA|1972-06-20|869|3782|291|1972|2|6|20|2|1972|291|3782|Tuesday|1972Q2|N|N|N|2441470|2441621|2441123|2441398|N|N|N|N|N| +2441490|AAAAAAAACBBEFCAA|1972-06-21|869|3782|291|1972|3|6|21|2|1972|291|3782|Wednesday|1972Q2|N|N|N|2441470|2441621|2441124|2441399|N|N|N|N|N| +2441491|AAAAAAAADBBEFCAA|1972-06-22|869|3782|291|1972|4|6|22|2|1972|291|3782|Thursday|1972Q2|N|N|N|2441470|2441621|2441125|2441400|N|N|N|N|N| +2441492|AAAAAAAAEBBEFCAA|1972-06-23|869|3782|291|1972|5|6|23|2|1972|291|3782|Friday|1972Q2|N|Y|N|2441470|2441621|2441126|2441401|N|N|N|N|N| +2441493|AAAAAAAAFBBEFCAA|1972-06-24|869|3782|291|1972|6|6|24|2|1972|291|3782|Saturday|1972Q2|N|Y|N|2441470|2441621|2441127|2441402|N|N|N|N|N| +2441494|AAAAAAAAGBBEFCAA|1972-06-25|869|3782|291|1972|0|6|25|2|1972|291|3782|Sunday|1972Q2|N|N|N|2441470|2441621|2441128|2441403|N|N|N|N|N| +2441495|AAAAAAAAHBBEFCAA|1972-06-26|869|3782|291|1972|1|6|26|2|1972|291|3782|Monday|1972Q2|N|N|N|2441470|2441621|2441129|2441404|N|N|N|N|N| +2441496|AAAAAAAAIBBEFCAA|1972-06-27|869|3783|291|1972|2|6|27|2|1972|291|3783|Tuesday|1972Q2|N|N|N|2441470|2441621|2441130|2441405|N|N|N|N|N| +2441497|AAAAAAAAJBBEFCAA|1972-06-28|869|3783|291|1972|3|6|28|2|1972|291|3783|Wednesday|1972Q2|N|N|N|2441470|2441621|2441131|2441406|N|N|N|N|N| +2441498|AAAAAAAAKBBEFCAA|1972-06-29|869|3783|291|1972|4|6|29|2|1972|291|3783|Thursday|1972Q2|N|N|N|2441470|2441621|2441132|2441407|N|N|N|N|N| +2441499|AAAAAAAALBBEFCAA|1972-06-30|869|3783|291|1972|5|6|30|2|1972|291|3783|Friday|1972Q2|N|Y|N|2441470|2441621|2441133|2441408|N|N|N|N|N| +2441500|AAAAAAAAMBBEFCAA|1972-07-01|870|3783|291|1972|6|7|1|3|1972|291|3783|Saturday|1972Q3|N|Y|N|2441500|2441681|2441134|2441409|N|N|N|N|N| +2441501|AAAAAAAANBBEFCAA|1972-07-02|870|3783|291|1972|0|7|2|3|1972|291|3783|Sunday|1972Q3|N|N|N|2441500|2441681|2441135|2441410|N|N|N|N|N| +2441502|AAAAAAAAOBBEFCAA|1972-07-03|870|3783|291|1972|1|7|3|3|1972|291|3783|Monday|1972Q3|N|N|N|2441500|2441681|2441136|2441411|N|N|N|N|N| +2441503|AAAAAAAAPBBEFCAA|1972-07-04|870|3784|291|1972|2|7|4|3|1972|291|3784|Tuesday|1972Q3|Y|N|N|2441500|2441681|2441137|2441412|N|N|N|N|N| +2441504|AAAAAAAAACBEFCAA|1972-07-05|870|3784|291|1972|3|7|5|3|1972|291|3784|Wednesday|1972Q3|N|N|Y|2441500|2441681|2441138|2441413|N|N|N|N|N| +2441505|AAAAAAAABCBEFCAA|1972-07-06|870|3784|291|1972|4|7|6|3|1972|291|3784|Thursday|1972Q3|N|N|N|2441500|2441681|2441139|2441414|N|N|N|N|N| +2441506|AAAAAAAACCBEFCAA|1972-07-07|870|3784|291|1972|5|7|7|3|1972|291|3784|Friday|1972Q3|N|Y|N|2441500|2441681|2441140|2441415|N|N|N|N|N| +2441507|AAAAAAAADCBEFCAA|1972-07-08|870|3784|291|1972|6|7|8|3|1972|291|3784|Saturday|1972Q3|N|Y|N|2441500|2441681|2441141|2441416|N|N|N|N|N| +2441508|AAAAAAAAECBEFCAA|1972-07-09|870|3784|291|1972|0|7|9|3|1972|291|3784|Sunday|1972Q3|N|N|N|2441500|2441681|2441142|2441417|N|N|N|N|N| +2441509|AAAAAAAAFCBEFCAA|1972-07-10|870|3784|291|1972|1|7|10|3|1972|291|3784|Monday|1972Q3|N|N|N|2441500|2441681|2441143|2441418|N|N|N|N|N| +2441510|AAAAAAAAGCBEFCAA|1972-07-11|870|3785|291|1972|2|7|11|3|1972|291|3785|Tuesday|1972Q3|N|N|N|2441500|2441681|2441144|2441419|N|N|N|N|N| +2441511|AAAAAAAAHCBEFCAA|1972-07-12|870|3785|291|1972|3|7|12|3|1972|291|3785|Wednesday|1972Q3|N|N|N|2441500|2441681|2441145|2441420|N|N|N|N|N| +2441512|AAAAAAAAICBEFCAA|1972-07-13|870|3785|291|1972|4|7|13|3|1972|291|3785|Thursday|1972Q3|N|N|N|2441500|2441681|2441146|2441421|N|N|N|N|N| +2441513|AAAAAAAAJCBEFCAA|1972-07-14|870|3785|291|1972|5|7|14|3|1972|291|3785|Friday|1972Q3|N|Y|N|2441500|2441681|2441147|2441422|N|N|N|N|N| +2441514|AAAAAAAAKCBEFCAA|1972-07-15|870|3785|291|1972|6|7|15|3|1972|291|3785|Saturday|1972Q3|N|Y|N|2441500|2441681|2441148|2441423|N|N|N|N|N| +2441515|AAAAAAAALCBEFCAA|1972-07-16|870|3785|291|1972|0|7|16|3|1972|291|3785|Sunday|1972Q3|N|N|N|2441500|2441681|2441149|2441424|N|N|N|N|N| +2441516|AAAAAAAAMCBEFCAA|1972-07-17|870|3785|291|1972|1|7|17|3|1972|291|3785|Monday|1972Q3|N|N|N|2441500|2441681|2441150|2441425|N|N|N|N|N| +2441517|AAAAAAAANCBEFCAA|1972-07-18|870|3786|291|1972|2|7|18|3|1972|291|3786|Tuesday|1972Q3|N|N|N|2441500|2441681|2441151|2441426|N|N|N|N|N| +2441518|AAAAAAAAOCBEFCAA|1972-07-19|870|3786|291|1972|3|7|19|3|1972|291|3786|Wednesday|1972Q3|N|N|N|2441500|2441681|2441152|2441427|N|N|N|N|N| +2441519|AAAAAAAAPCBEFCAA|1972-07-20|870|3786|291|1972|4|7|20|3|1972|291|3786|Thursday|1972Q3|N|N|N|2441500|2441681|2441153|2441428|N|N|N|N|N| +2441520|AAAAAAAAADBEFCAA|1972-07-21|870|3786|291|1972|5|7|21|3|1972|291|3786|Friday|1972Q3|N|Y|N|2441500|2441681|2441154|2441429|N|N|N|N|N| +2441521|AAAAAAAABDBEFCAA|1972-07-22|870|3786|291|1972|6|7|22|3|1972|291|3786|Saturday|1972Q3|N|Y|N|2441500|2441681|2441155|2441430|N|N|N|N|N| +2441522|AAAAAAAACDBEFCAA|1972-07-23|870|3786|291|1972|0|7|23|3|1972|291|3786|Sunday|1972Q3|N|N|N|2441500|2441681|2441156|2441431|N|N|N|N|N| +2441523|AAAAAAAADDBEFCAA|1972-07-24|870|3786|291|1972|1|7|24|3|1972|291|3786|Monday|1972Q3|N|N|N|2441500|2441681|2441157|2441432|N|N|N|N|N| +2441524|AAAAAAAAEDBEFCAA|1972-07-25|870|3787|291|1972|2|7|25|3|1972|291|3787|Tuesday|1972Q3|N|N|N|2441500|2441681|2441158|2441433|N|N|N|N|N| +2441525|AAAAAAAAFDBEFCAA|1972-07-26|870|3787|291|1972|3|7|26|3|1972|291|3787|Wednesday|1972Q3|N|N|N|2441500|2441681|2441159|2441434|N|N|N|N|N| +2441526|AAAAAAAAGDBEFCAA|1972-07-27|870|3787|291|1972|4|7|27|3|1972|291|3787|Thursday|1972Q3|N|N|N|2441500|2441681|2441160|2441435|N|N|N|N|N| +2441527|AAAAAAAAHDBEFCAA|1972-07-28|870|3787|291|1972|5|7|28|3|1972|291|3787|Friday|1972Q3|N|Y|N|2441500|2441681|2441161|2441436|N|N|N|N|N| +2441528|AAAAAAAAIDBEFCAA|1972-07-29|870|3787|291|1972|6|7|29|3|1972|291|3787|Saturday|1972Q3|N|Y|N|2441500|2441681|2441162|2441437|N|N|N|N|N| +2441529|AAAAAAAAJDBEFCAA|1972-07-30|870|3787|291|1972|0|7|30|3|1972|291|3787|Sunday|1972Q3|N|N|N|2441500|2441681|2441163|2441438|N|N|N|N|N| +2441530|AAAAAAAAKDBEFCAA|1972-07-31|870|3787|291|1972|1|7|31|3|1972|291|3787|Monday|1972Q3|N|N|N|2441500|2441681|2441164|2441439|N|N|N|N|N| +2441531|AAAAAAAALDBEFCAA|1972-08-01|871|3788|291|1972|2|8|1|3|1972|291|3788|Tuesday|1972Q3|N|N|N|2441531|2441743|2441165|2441440|N|N|N|N|N| +2441532|AAAAAAAAMDBEFCAA|1972-08-02|871|3788|291|1972|3|8|2|3|1972|291|3788|Wednesday|1972Q3|N|N|N|2441531|2441743|2441166|2441441|N|N|N|N|N| +2441533|AAAAAAAANDBEFCAA|1972-08-03|871|3788|291|1972|4|8|3|3|1972|291|3788|Thursday|1972Q3|N|N|N|2441531|2441743|2441167|2441442|N|N|N|N|N| +2441534|AAAAAAAAODBEFCAA|1972-08-04|871|3788|291|1972|5|8|4|3|1972|291|3788|Friday|1972Q3|N|Y|N|2441531|2441743|2441168|2441443|N|N|N|N|N| +2441535|AAAAAAAAPDBEFCAA|1972-08-05|871|3788|291|1972|6|8|5|3|1972|291|3788|Saturday|1972Q3|N|Y|N|2441531|2441743|2441169|2441444|N|N|N|N|N| +2441536|AAAAAAAAAEBEFCAA|1972-08-06|871|3788|291|1972|0|8|6|3|1972|291|3788|Sunday|1972Q3|N|N|N|2441531|2441743|2441170|2441445|N|N|N|N|N| +2441537|AAAAAAAABEBEFCAA|1972-08-07|871|3788|291|1972|1|8|7|3|1972|291|3788|Monday|1972Q3|N|N|N|2441531|2441743|2441171|2441446|N|N|N|N|N| +2441538|AAAAAAAACEBEFCAA|1972-08-08|871|3789|291|1972|2|8|8|3|1972|291|3789|Tuesday|1972Q3|N|N|N|2441531|2441743|2441172|2441447|N|N|N|N|N| +2441539|AAAAAAAADEBEFCAA|1972-08-09|871|3789|291|1972|3|8|9|3|1972|291|3789|Wednesday|1972Q3|N|N|N|2441531|2441743|2441173|2441448|N|N|N|N|N| +2441540|AAAAAAAAEEBEFCAA|1972-08-10|871|3789|291|1972|4|8|10|3|1972|291|3789|Thursday|1972Q3|N|N|N|2441531|2441743|2441174|2441449|N|N|N|N|N| +2441541|AAAAAAAAFEBEFCAA|1972-08-11|871|3789|291|1972|5|8|11|3|1972|291|3789|Friday|1972Q3|N|Y|N|2441531|2441743|2441175|2441450|N|N|N|N|N| +2441542|AAAAAAAAGEBEFCAA|1972-08-12|871|3789|291|1972|6|8|12|3|1972|291|3789|Saturday|1972Q3|N|Y|N|2441531|2441743|2441176|2441451|N|N|N|N|N| +2441543|AAAAAAAAHEBEFCAA|1972-08-13|871|3789|291|1972|0|8|13|3|1972|291|3789|Sunday|1972Q3|N|N|N|2441531|2441743|2441177|2441452|N|N|N|N|N| +2441544|AAAAAAAAIEBEFCAA|1972-08-14|871|3789|291|1972|1|8|14|3|1972|291|3789|Monday|1972Q3|N|N|N|2441531|2441743|2441178|2441453|N|N|N|N|N| +2441545|AAAAAAAAJEBEFCAA|1972-08-15|871|3790|291|1972|2|8|15|3|1972|291|3790|Tuesday|1972Q3|N|N|N|2441531|2441743|2441179|2441454|N|N|N|N|N| +2441546|AAAAAAAAKEBEFCAA|1972-08-16|871|3790|291|1972|3|8|16|3|1972|291|3790|Wednesday|1972Q3|N|N|N|2441531|2441743|2441180|2441455|N|N|N|N|N| +2441547|AAAAAAAALEBEFCAA|1972-08-17|871|3790|291|1972|4|8|17|3|1972|291|3790|Thursday|1972Q3|N|N|N|2441531|2441743|2441181|2441456|N|N|N|N|N| +2441548|AAAAAAAAMEBEFCAA|1972-08-18|871|3790|291|1972|5|8|18|3|1972|291|3790|Friday|1972Q3|N|Y|N|2441531|2441743|2441182|2441457|N|N|N|N|N| +2441549|AAAAAAAANEBEFCAA|1972-08-19|871|3790|291|1972|6|8|19|3|1972|291|3790|Saturday|1972Q3|N|Y|N|2441531|2441743|2441183|2441458|N|N|N|N|N| +2441550|AAAAAAAAOEBEFCAA|1972-08-20|871|3790|291|1972|0|8|20|3|1972|291|3790|Sunday|1972Q3|N|N|N|2441531|2441743|2441184|2441459|N|N|N|N|N| +2441551|AAAAAAAAPEBEFCAA|1972-08-21|871|3790|291|1972|1|8|21|3|1972|291|3790|Monday|1972Q3|N|N|N|2441531|2441743|2441185|2441460|N|N|N|N|N| +2441552|AAAAAAAAAFBEFCAA|1972-08-22|871|3791|291|1972|2|8|22|3|1972|291|3791|Tuesday|1972Q3|N|N|N|2441531|2441743|2441186|2441461|N|N|N|N|N| +2441553|AAAAAAAABFBEFCAA|1972-08-23|871|3791|291|1972|3|8|23|3|1972|291|3791|Wednesday|1972Q3|N|N|N|2441531|2441743|2441187|2441462|N|N|N|N|N| +2441554|AAAAAAAACFBEFCAA|1972-08-24|871|3791|291|1972|4|8|24|3|1972|291|3791|Thursday|1972Q3|N|N|N|2441531|2441743|2441188|2441463|N|N|N|N|N| +2441555|AAAAAAAADFBEFCAA|1972-08-25|871|3791|291|1972|5|8|25|3|1972|291|3791|Friday|1972Q3|N|Y|N|2441531|2441743|2441189|2441464|N|N|N|N|N| +2441556|AAAAAAAAEFBEFCAA|1972-08-26|871|3791|291|1972|6|8|26|3|1972|291|3791|Saturday|1972Q3|N|Y|N|2441531|2441743|2441190|2441465|N|N|N|N|N| +2441557|AAAAAAAAFFBEFCAA|1972-08-27|871|3791|291|1972|0|8|27|3|1972|291|3791|Sunday|1972Q3|N|N|N|2441531|2441743|2441191|2441466|N|N|N|N|N| +2441558|AAAAAAAAGFBEFCAA|1972-08-28|871|3791|291|1972|1|8|28|3|1972|291|3791|Monday|1972Q3|N|N|N|2441531|2441743|2441192|2441467|N|N|N|N|N| +2441559|AAAAAAAAHFBEFCAA|1972-08-29|871|3792|291|1972|2|8|29|3|1972|291|3792|Tuesday|1972Q3|N|N|N|2441531|2441743|2441193|2441468|N|N|N|N|N| +2441560|AAAAAAAAIFBEFCAA|1972-08-30|871|3792|291|1972|3|8|30|3|1972|291|3792|Wednesday|1972Q3|N|N|N|2441531|2441743|2441194|2441469|N|N|N|N|N| +2441561|AAAAAAAAJFBEFCAA|1972-08-31|871|3792|291|1972|4|8|31|3|1972|291|3792|Thursday|1972Q3|N|N|N|2441531|2441743|2441195|2441470|N|N|N|N|N| +2441562|AAAAAAAAKFBEFCAA|1972-09-01|872|3792|292|1972|5|9|1|3|1972|292|3792|Friday|1972Q3|N|Y|N|2441562|2441805|2441196|2441471|N|N|N|N|N| +2441563|AAAAAAAALFBEFCAA|1972-09-02|872|3792|292|1972|6|9|2|3|1972|292|3792|Saturday|1972Q3|N|Y|N|2441562|2441805|2441197|2441472|N|N|N|N|N| +2441564|AAAAAAAAMFBEFCAA|1972-09-03|872|3792|292|1972|0|9|3|3|1972|292|3792|Sunday|1972Q3|N|N|N|2441562|2441805|2441198|2441473|N|N|N|N|N| +2441565|AAAAAAAANFBEFCAA|1972-09-04|872|3792|292|1972|1|9|4|3|1972|292|3792|Monday|1972Q3|N|N|N|2441562|2441805|2441199|2441474|N|N|N|N|N| +2441566|AAAAAAAAOFBEFCAA|1972-09-05|872|3793|292|1972|2|9|5|3|1972|292|3793|Tuesday|1972Q3|N|N|N|2441562|2441805|2441200|2441475|N|N|N|N|N| +2441567|AAAAAAAAPFBEFCAA|1972-09-06|872|3793|292|1972|3|9|6|3|1972|292|3793|Wednesday|1972Q3|N|N|N|2441562|2441805|2441201|2441476|N|N|N|N|N| +2441568|AAAAAAAAAGBEFCAA|1972-09-07|872|3793|292|1972|4|9|7|3|1972|292|3793|Thursday|1972Q3|N|N|N|2441562|2441805|2441202|2441477|N|N|N|N|N| +2441569|AAAAAAAABGBEFCAA|1972-09-08|872|3793|292|1972|5|9|8|3|1972|292|3793|Friday|1972Q3|N|Y|N|2441562|2441805|2441203|2441478|N|N|N|N|N| +2441570|AAAAAAAACGBEFCAA|1972-09-09|872|3793|292|1972|6|9|9|3|1972|292|3793|Saturday|1972Q3|N|Y|N|2441562|2441805|2441204|2441479|N|N|N|N|N| +2441571|AAAAAAAADGBEFCAA|1972-09-10|872|3793|292|1972|0|9|10|3|1972|292|3793|Sunday|1972Q3|N|N|N|2441562|2441805|2441205|2441480|N|N|N|N|N| +2441572|AAAAAAAAEGBEFCAA|1972-09-11|872|3793|292|1972|1|9|11|3|1972|292|3793|Monday|1972Q3|N|N|N|2441562|2441805|2441206|2441481|N|N|N|N|N| +2441573|AAAAAAAAFGBEFCAA|1972-09-12|872|3794|292|1972|2|9|12|3|1972|292|3794|Tuesday|1972Q3|N|N|N|2441562|2441805|2441207|2441482|N|N|N|N|N| +2441574|AAAAAAAAGGBEFCAA|1972-09-13|872|3794|292|1972|3|9|13|3|1972|292|3794|Wednesday|1972Q3|N|N|N|2441562|2441805|2441208|2441483|N|N|N|N|N| +2441575|AAAAAAAAHGBEFCAA|1972-09-14|872|3794|292|1972|4|9|14|3|1972|292|3794|Thursday|1972Q3|N|N|N|2441562|2441805|2441209|2441484|N|N|N|N|N| +2441576|AAAAAAAAIGBEFCAA|1972-09-15|872|3794|292|1972|5|9|15|3|1972|292|3794|Friday|1972Q3|N|Y|N|2441562|2441805|2441210|2441485|N|N|N|N|N| +2441577|AAAAAAAAJGBEFCAA|1972-09-16|872|3794|292|1972|6|9|16|3|1972|292|3794|Saturday|1972Q3|N|Y|N|2441562|2441805|2441211|2441486|N|N|N|N|N| +2441578|AAAAAAAAKGBEFCAA|1972-09-17|872|3794|292|1972|0|9|17|3|1972|292|3794|Sunday|1972Q3|N|N|N|2441562|2441805|2441212|2441487|N|N|N|N|N| +2441579|AAAAAAAALGBEFCAA|1972-09-18|872|3794|292|1972|1|9|18|3|1972|292|3794|Monday|1972Q3|N|N|N|2441562|2441805|2441213|2441488|N|N|N|N|N| +2441580|AAAAAAAAMGBEFCAA|1972-09-19|872|3795|292|1972|2|9|19|3|1972|292|3795|Tuesday|1972Q3|N|N|N|2441562|2441805|2441214|2441489|N|N|N|N|N| +2441581|AAAAAAAANGBEFCAA|1972-09-20|872|3795|292|1972|3|9|20|3|1972|292|3795|Wednesday|1972Q3|N|N|N|2441562|2441805|2441215|2441490|N|N|N|N|N| +2441582|AAAAAAAAOGBEFCAA|1972-09-21|872|3795|292|1972|4|9|21|3|1972|292|3795|Thursday|1972Q3|N|N|N|2441562|2441805|2441216|2441491|N|N|N|N|N| +2441583|AAAAAAAAPGBEFCAA|1972-09-22|872|3795|292|1972|5|9|22|3|1972|292|3795|Friday|1972Q3|N|Y|N|2441562|2441805|2441217|2441492|N|N|N|N|N| +2441584|AAAAAAAAAHBEFCAA|1972-09-23|872|3795|292|1972|6|9|23|3|1972|292|3795|Saturday|1972Q3|N|Y|N|2441562|2441805|2441218|2441493|N|N|N|N|N| +2441585|AAAAAAAABHBEFCAA|1972-09-24|872|3795|292|1972|0|9|24|3|1972|292|3795|Sunday|1972Q3|N|N|N|2441562|2441805|2441219|2441494|N|N|N|N|N| +2441586|AAAAAAAACHBEFCAA|1972-09-25|872|3795|292|1972|1|9|25|3|1972|292|3795|Monday|1972Q3|N|N|N|2441562|2441805|2441220|2441495|N|N|N|N|N| +2441587|AAAAAAAADHBEFCAA|1972-09-26|872|3796|292|1972|2|9|26|3|1972|292|3796|Tuesday|1972Q3|N|N|N|2441562|2441805|2441221|2441496|N|N|N|N|N| +2441588|AAAAAAAAEHBEFCAA|1972-09-27|872|3796|292|1972|3|9|27|3|1972|292|3796|Wednesday|1972Q3|N|N|N|2441562|2441805|2441222|2441497|N|N|N|N|N| +2441589|AAAAAAAAFHBEFCAA|1972-09-28|872|3796|292|1972|4|9|28|3|1972|292|3796|Thursday|1972Q3|N|N|N|2441562|2441805|2441223|2441498|N|N|N|N|N| +2441590|AAAAAAAAGHBEFCAA|1972-09-29|872|3796|292|1972|5|9|29|3|1972|292|3796|Friday|1972Q3|N|Y|N|2441562|2441805|2441224|2441499|N|N|N|N|N| +2441591|AAAAAAAAHHBEFCAA|1972-09-30|872|3796|292|1972|6|9|30|3|1972|292|3796|Saturday|1972Q3|N|Y|N|2441562|2441805|2441225|2441500|N|N|N|N|N| +2441592|AAAAAAAAIHBEFCAA|1972-10-01|873|3796|292|1972|0|10|1|4|1972|292|3796|Sunday|1972Q4|N|N|N|2441592|2441865|2441226|2441500|N|N|N|N|N| +2441593|AAAAAAAAJHBEFCAA|1972-10-02|873|3796|292|1972|1|10|2|4|1972|292|3796|Monday|1972Q4|N|N|N|2441592|2441865|2441227|2441501|N|N|N|N|N| +2441594|AAAAAAAAKHBEFCAA|1972-10-03|873|3797|292|1972|2|10|3|4|1972|292|3797|Tuesday|1972Q4|N|N|N|2441592|2441865|2441228|2441502|N|N|N|N|N| +2441595|AAAAAAAALHBEFCAA|1972-10-04|873|3797|292|1972|3|10|4|4|1972|292|3797|Wednesday|1972Q4|N|N|N|2441592|2441865|2441229|2441503|N|N|N|N|N| +2441596|AAAAAAAAMHBEFCAA|1972-10-05|873|3797|292|1972|4|10|5|4|1972|292|3797|Thursday|1972Q4|N|N|N|2441592|2441865|2441230|2441504|N|N|N|N|N| +2441597|AAAAAAAANHBEFCAA|1972-10-06|873|3797|292|1972|5|10|6|4|1972|292|3797|Friday|1972Q4|N|Y|N|2441592|2441865|2441231|2441505|N|N|N|N|N| +2441598|AAAAAAAAOHBEFCAA|1972-10-07|873|3797|292|1972|6|10|7|4|1972|292|3797|Saturday|1972Q4|N|Y|N|2441592|2441865|2441232|2441506|N|N|N|N|N| +2441599|AAAAAAAAPHBEFCAA|1972-10-08|873|3797|292|1972|0|10|8|4|1972|292|3797|Sunday|1972Q4|N|N|N|2441592|2441865|2441233|2441507|N|N|N|N|N| +2441600|AAAAAAAAAIBEFCAA|1972-10-09|873|3797|292|1972|1|10|9|4|1972|292|3797|Monday|1972Q4|N|N|N|2441592|2441865|2441234|2441508|N|N|N|N|N| +2441601|AAAAAAAABIBEFCAA|1972-10-10|873|3798|292|1972|2|10|10|4|1972|292|3798|Tuesday|1972Q4|N|N|N|2441592|2441865|2441235|2441509|N|N|N|N|N| +2441602|AAAAAAAACIBEFCAA|1972-10-11|873|3798|292|1972|3|10|11|4|1972|292|3798|Wednesday|1972Q4|N|N|N|2441592|2441865|2441236|2441510|N|N|N|N|N| +2441603|AAAAAAAADIBEFCAA|1972-10-12|873|3798|292|1972|4|10|12|4|1972|292|3798|Thursday|1972Q4|N|N|N|2441592|2441865|2441237|2441511|N|N|N|N|N| +2441604|AAAAAAAAEIBEFCAA|1972-10-13|873|3798|292|1972|5|10|13|4|1972|292|3798|Friday|1972Q4|N|Y|N|2441592|2441865|2441238|2441512|N|N|N|N|N| +2441605|AAAAAAAAFIBEFCAA|1972-10-14|873|3798|292|1972|6|10|14|4|1972|292|3798|Saturday|1972Q4|N|Y|N|2441592|2441865|2441239|2441513|N|N|N|N|N| +2441606|AAAAAAAAGIBEFCAA|1972-10-15|873|3798|292|1972|0|10|15|4|1972|292|3798|Sunday|1972Q4|N|N|N|2441592|2441865|2441240|2441514|N|N|N|N|N| +2441607|AAAAAAAAHIBEFCAA|1972-10-16|873|3798|292|1972|1|10|16|4|1972|292|3798|Monday|1972Q4|N|N|N|2441592|2441865|2441241|2441515|N|N|N|N|N| +2441608|AAAAAAAAIIBEFCAA|1972-10-17|873|3799|292|1972|2|10|17|4|1972|292|3799|Tuesday|1972Q4|N|N|N|2441592|2441865|2441242|2441516|N|N|N|N|N| +2441609|AAAAAAAAJIBEFCAA|1972-10-18|873|3799|292|1972|3|10|18|4|1972|292|3799|Wednesday|1972Q4|N|N|N|2441592|2441865|2441243|2441517|N|N|N|N|N| +2441610|AAAAAAAAKIBEFCAA|1972-10-19|873|3799|292|1972|4|10|19|4|1972|292|3799|Thursday|1972Q4|N|N|N|2441592|2441865|2441244|2441518|N|N|N|N|N| +2441611|AAAAAAAALIBEFCAA|1972-10-20|873|3799|292|1972|5|10|20|4|1972|292|3799|Friday|1972Q4|N|Y|N|2441592|2441865|2441245|2441519|N|N|N|N|N| +2441612|AAAAAAAAMIBEFCAA|1972-10-21|873|3799|292|1972|6|10|21|4|1972|292|3799|Saturday|1972Q4|N|Y|N|2441592|2441865|2441246|2441520|N|N|N|N|N| +2441613|AAAAAAAANIBEFCAA|1972-10-22|873|3799|292|1972|0|10|22|4|1972|292|3799|Sunday|1972Q4|N|N|N|2441592|2441865|2441247|2441521|N|N|N|N|N| +2441614|AAAAAAAAOIBEFCAA|1972-10-23|873|3799|292|1972|1|10|23|4|1972|292|3799|Monday|1972Q4|N|N|N|2441592|2441865|2441248|2441522|N|N|N|N|N| +2441615|AAAAAAAAPIBEFCAA|1972-10-24|873|3800|292|1972|2|10|24|4|1972|292|3800|Tuesday|1972Q4|N|N|N|2441592|2441865|2441249|2441523|N|N|N|N|N| +2441616|AAAAAAAAAJBEFCAA|1972-10-25|873|3800|292|1972|3|10|25|4|1972|292|3800|Wednesday|1972Q4|N|N|N|2441592|2441865|2441250|2441524|N|N|N|N|N| +2441617|AAAAAAAABJBEFCAA|1972-10-26|873|3800|292|1972|4|10|26|4|1972|292|3800|Thursday|1972Q4|N|N|N|2441592|2441865|2441251|2441525|N|N|N|N|N| +2441618|AAAAAAAACJBEFCAA|1972-10-27|873|3800|292|1972|5|10|27|4|1972|292|3800|Friday|1972Q4|N|Y|N|2441592|2441865|2441252|2441526|N|N|N|N|N| +2441619|AAAAAAAADJBEFCAA|1972-10-28|873|3800|292|1972|6|10|28|4|1972|292|3800|Saturday|1972Q4|N|Y|N|2441592|2441865|2441253|2441527|N|N|N|N|N| +2441620|AAAAAAAAEJBEFCAA|1972-10-29|873|3800|292|1972|0|10|29|4|1972|292|3800|Sunday|1972Q4|N|N|N|2441592|2441865|2441254|2441528|N|N|N|N|N| +2441621|AAAAAAAAFJBEFCAA|1972-10-30|873|3800|292|1972|1|10|30|4|1972|292|3800|Monday|1972Q4|N|N|N|2441592|2441865|2441255|2441529|N|N|N|N|N| +2441622|AAAAAAAAGJBEFCAA|1972-10-31|873|3801|292|1972|2|10|31|4|1972|292|3801|Tuesday|1972Q4|N|N|N|2441592|2441865|2441256|2441530|N|N|N|N|N| +2441623|AAAAAAAAHJBEFCAA|1972-11-01|874|3801|292|1972|3|11|1|4|1972|292|3801|Wednesday|1972Q4|N|N|N|2441623|2441927|2441257|2441531|N|N|N|N|N| +2441624|AAAAAAAAIJBEFCAA|1972-11-02|874|3801|292|1972|4|11|2|4|1972|292|3801|Thursday|1972Q4|N|N|N|2441623|2441927|2441258|2441532|N|N|N|N|N| +2441625|AAAAAAAAJJBEFCAA|1972-11-03|874|3801|292|1972|5|11|3|4|1972|292|3801|Friday|1972Q4|N|Y|N|2441623|2441927|2441259|2441533|N|N|N|N|N| +2441626|AAAAAAAAKJBEFCAA|1972-11-04|874|3801|292|1972|6|11|4|4|1972|292|3801|Saturday|1972Q4|N|Y|N|2441623|2441927|2441260|2441534|N|N|N|N|N| +2441627|AAAAAAAALJBEFCAA|1972-11-05|874|3801|292|1972|0|11|5|4|1972|292|3801|Sunday|1972Q4|N|N|N|2441623|2441927|2441261|2441535|N|N|N|N|N| +2441628|AAAAAAAAMJBEFCAA|1972-11-06|874|3801|292|1972|1|11|6|4|1972|292|3801|Monday|1972Q4|N|N|N|2441623|2441927|2441262|2441536|N|N|N|N|N| +2441629|AAAAAAAANJBEFCAA|1972-11-07|874|3802|292|1972|2|11|7|4|1972|292|3802|Tuesday|1972Q4|N|N|N|2441623|2441927|2441263|2441537|N|N|N|N|N| +2441630|AAAAAAAAOJBEFCAA|1972-11-08|874|3802|292|1972|3|11|8|4|1972|292|3802|Wednesday|1972Q4|N|N|N|2441623|2441927|2441264|2441538|N|N|N|N|N| +2441631|AAAAAAAAPJBEFCAA|1972-11-09|874|3802|292|1972|4|11|9|4|1972|292|3802|Thursday|1972Q4|N|N|N|2441623|2441927|2441265|2441539|N|N|N|N|N| +2441632|AAAAAAAAAKBEFCAA|1972-11-10|874|3802|292|1972|5|11|10|4|1972|292|3802|Friday|1972Q4|N|Y|N|2441623|2441927|2441266|2441540|N|N|N|N|N| +2441633|AAAAAAAABKBEFCAA|1972-11-11|874|3802|292|1972|6|11|11|4|1972|292|3802|Saturday|1972Q4|N|Y|N|2441623|2441927|2441267|2441541|N|N|N|N|N| +2441634|AAAAAAAACKBEFCAA|1972-11-12|874|3802|292|1972|0|11|12|4|1972|292|3802|Sunday|1972Q4|N|N|N|2441623|2441927|2441268|2441542|N|N|N|N|N| +2441635|AAAAAAAADKBEFCAA|1972-11-13|874|3802|292|1972|1|11|13|4|1972|292|3802|Monday|1972Q4|N|N|N|2441623|2441927|2441269|2441543|N|N|N|N|N| +2441636|AAAAAAAAEKBEFCAA|1972-11-14|874|3803|292|1972|2|11|14|4|1972|292|3803|Tuesday|1972Q4|N|N|N|2441623|2441927|2441270|2441544|N|N|N|N|N| +2441637|AAAAAAAAFKBEFCAA|1972-11-15|874|3803|292|1972|3|11|15|4|1972|292|3803|Wednesday|1972Q4|N|N|N|2441623|2441927|2441271|2441545|N|N|N|N|N| +2441638|AAAAAAAAGKBEFCAA|1972-11-16|874|3803|292|1972|4|11|16|4|1972|292|3803|Thursday|1972Q4|N|N|N|2441623|2441927|2441272|2441546|N|N|N|N|N| +2441639|AAAAAAAAHKBEFCAA|1972-11-17|874|3803|292|1972|5|11|17|4|1972|292|3803|Friday|1972Q4|N|Y|N|2441623|2441927|2441273|2441547|N|N|N|N|N| +2441640|AAAAAAAAIKBEFCAA|1972-11-18|874|3803|292|1972|6|11|18|4|1972|292|3803|Saturday|1972Q4|N|Y|N|2441623|2441927|2441274|2441548|N|N|N|N|N| +2441641|AAAAAAAAJKBEFCAA|1972-11-19|874|3803|292|1972|0|11|19|4|1972|292|3803|Sunday|1972Q4|N|N|N|2441623|2441927|2441275|2441549|N|N|N|N|N| +2441642|AAAAAAAAKKBEFCAA|1972-11-20|874|3803|292|1972|1|11|20|4|1972|292|3803|Monday|1972Q4|N|N|N|2441623|2441927|2441276|2441550|N|N|N|N|N| +2441643|AAAAAAAALKBEFCAA|1972-11-21|874|3804|292|1972|2|11|21|4|1972|292|3804|Tuesday|1972Q4|N|N|N|2441623|2441927|2441277|2441551|N|N|N|N|N| +2441644|AAAAAAAAMKBEFCAA|1972-11-22|874|3804|292|1972|3|11|22|4|1972|292|3804|Wednesday|1972Q4|N|N|N|2441623|2441927|2441278|2441552|N|N|N|N|N| +2441645|AAAAAAAANKBEFCAA|1972-11-23|874|3804|292|1972|4|11|23|4|1972|292|3804|Thursday|1972Q4|N|N|N|2441623|2441927|2441279|2441553|N|N|N|N|N| +2441646|AAAAAAAAOKBEFCAA|1972-11-24|874|3804|292|1972|5|11|24|4|1972|292|3804|Friday|1972Q4|N|Y|N|2441623|2441927|2441280|2441554|N|N|N|N|N| +2441647|AAAAAAAAPKBEFCAA|1972-11-25|874|3804|292|1972|6|11|25|4|1972|292|3804|Saturday|1972Q4|N|Y|N|2441623|2441927|2441281|2441555|N|N|N|N|N| +2441648|AAAAAAAAALBEFCAA|1972-11-26|874|3804|292|1972|0|11|26|4|1972|292|3804|Sunday|1972Q4|N|N|N|2441623|2441927|2441282|2441556|N|N|N|N|N| +2441649|AAAAAAAABLBEFCAA|1972-11-27|874|3804|292|1972|1|11|27|4|1972|292|3804|Monday|1972Q4|N|N|N|2441623|2441927|2441283|2441557|N|N|N|N|N| +2441650|AAAAAAAACLBEFCAA|1972-11-28|874|3805|292|1972|2|11|28|4|1972|292|3805|Tuesday|1972Q4|N|N|N|2441623|2441927|2441284|2441558|N|N|N|N|N| +2441651|AAAAAAAADLBEFCAA|1972-11-29|874|3805|292|1972|3|11|29|4|1972|292|3805|Wednesday|1972Q4|N|N|N|2441623|2441927|2441285|2441559|N|N|N|N|N| +2441652|AAAAAAAAELBEFCAA|1972-11-30|874|3805|292|1972|4|11|30|4|1972|292|3805|Thursday|1972Q4|N|N|N|2441623|2441927|2441286|2441560|N|N|N|N|N| +2441653|AAAAAAAAFLBEFCAA|1972-12-01|875|3805|293|1972|5|12|1|4|1972|293|3805|Friday|1972Q4|N|Y|N|2441653|2441987|2441287|2441561|N|N|N|N|N| +2441654|AAAAAAAAGLBEFCAA|1972-12-02|875|3805|293|1972|6|12|2|4|1972|293|3805|Saturday|1972Q4|N|Y|N|2441653|2441987|2441288|2441562|N|N|N|N|N| +2441655|AAAAAAAAHLBEFCAA|1972-12-03|875|3805|293|1972|0|12|3|4|1972|293|3805|Sunday|1972Q4|N|N|N|2441653|2441987|2441289|2441563|N|N|N|N|N| +2441656|AAAAAAAAILBEFCAA|1972-12-04|875|3805|293|1972|1|12|4|4|1972|293|3805|Monday|1972Q4|N|N|N|2441653|2441987|2441290|2441564|N|N|N|N|N| +2441657|AAAAAAAAJLBEFCAA|1972-12-05|875|3806|293|1972|2|12|5|4|1972|293|3806|Tuesday|1972Q4|N|N|N|2441653|2441987|2441291|2441565|N|N|N|N|N| +2441658|AAAAAAAAKLBEFCAA|1972-12-06|875|3806|293|1972|3|12|6|4|1972|293|3806|Wednesday|1972Q4|N|N|N|2441653|2441987|2441292|2441566|N|N|N|N|N| +2441659|AAAAAAAALLBEFCAA|1972-12-07|875|3806|293|1972|4|12|7|4|1972|293|3806|Thursday|1972Q4|N|N|N|2441653|2441987|2441293|2441567|N|N|N|N|N| +2441660|AAAAAAAAMLBEFCAA|1972-12-08|875|3806|293|1972|5|12|8|4|1972|293|3806|Friday|1972Q4|N|Y|N|2441653|2441987|2441294|2441568|N|N|N|N|N| +2441661|AAAAAAAANLBEFCAA|1972-12-09|875|3806|293|1972|6|12|9|4|1972|293|3806|Saturday|1972Q4|N|Y|N|2441653|2441987|2441295|2441569|N|N|N|N|N| +2441662|AAAAAAAAOLBEFCAA|1972-12-10|875|3806|293|1972|0|12|10|4|1972|293|3806|Sunday|1972Q4|N|N|N|2441653|2441987|2441296|2441570|N|N|N|N|N| +2441663|AAAAAAAAPLBEFCAA|1972-12-11|875|3806|293|1972|1|12|11|4|1972|293|3806|Monday|1972Q4|N|N|N|2441653|2441987|2441297|2441571|N|N|N|N|N| +2441664|AAAAAAAAAMBEFCAA|1972-12-12|875|3807|293|1972|2|12|12|4|1972|293|3807|Tuesday|1972Q4|N|N|N|2441653|2441987|2441298|2441572|N|N|N|N|N| +2441665|AAAAAAAABMBEFCAA|1972-12-13|875|3807|293|1972|3|12|13|4|1972|293|3807|Wednesday|1972Q4|N|N|N|2441653|2441987|2441299|2441573|N|N|N|N|N| +2441666|AAAAAAAACMBEFCAA|1972-12-14|875|3807|293|1972|4|12|14|4|1972|293|3807|Thursday|1972Q4|N|N|N|2441653|2441987|2441300|2441574|N|N|N|N|N| +2441667|AAAAAAAADMBEFCAA|1972-12-15|875|3807|293|1972|5|12|15|4|1972|293|3807|Friday|1972Q4|N|Y|N|2441653|2441987|2441301|2441575|N|N|N|N|N| +2441668|AAAAAAAAEMBEFCAA|1972-12-16|875|3807|293|1972|6|12|16|4|1972|293|3807|Saturday|1972Q4|N|Y|N|2441653|2441987|2441302|2441576|N|N|N|N|N| +2441669|AAAAAAAAFMBEFCAA|1972-12-17|875|3807|293|1972|0|12|17|4|1972|293|3807|Sunday|1972Q4|N|N|N|2441653|2441987|2441303|2441577|N|N|N|N|N| +2441670|AAAAAAAAGMBEFCAA|1972-12-18|875|3807|293|1972|1|12|18|4|1972|293|3807|Monday|1972Q4|N|N|N|2441653|2441987|2441304|2441578|N|N|N|N|N| +2441671|AAAAAAAAHMBEFCAA|1972-12-19|875|3808|293|1972|2|12|19|4|1972|293|3808|Tuesday|1972Q4|N|N|N|2441653|2441987|2441305|2441579|N|N|N|N|N| +2441672|AAAAAAAAIMBEFCAA|1972-12-20|875|3808|293|1972|3|12|20|4|1972|293|3808|Wednesday|1972Q4|N|N|N|2441653|2441987|2441306|2441580|N|N|N|N|N| +2441673|AAAAAAAAJMBEFCAA|1972-12-21|875|3808|293|1972|4|12|21|4|1972|293|3808|Thursday|1972Q4|N|N|N|2441653|2441987|2441307|2441581|N|N|N|N|N| +2441674|AAAAAAAAKMBEFCAA|1972-12-22|875|3808|293|1972|5|12|22|4|1972|293|3808|Friday|1972Q4|N|Y|N|2441653|2441987|2441308|2441582|N|N|N|N|N| +2441675|AAAAAAAALMBEFCAA|1972-12-23|875|3808|293|1972|6|12|23|4|1972|293|3808|Saturday|1972Q4|N|Y|N|2441653|2441987|2441309|2441583|N|N|N|N|N| +2441676|AAAAAAAAMMBEFCAA|1972-12-24|875|3808|293|1972|0|12|24|4|1972|293|3808|Sunday|1972Q4|N|N|N|2441653|2441987|2441310|2441584|N|N|N|N|N| +2441677|AAAAAAAANMBEFCAA|1972-12-25|875|3808|293|1972|1|12|25|4|1972|293|3808|Monday|1972Q4|Y|N|N|2441653|2441987|2441311|2441585|N|N|N|N|N| +2441678|AAAAAAAAOMBEFCAA|1972-12-26|875|3809|293|1972|2|12|26|4|1972|293|3809|Tuesday|1972Q4|N|N|Y|2441653|2441987|2441312|2441586|N|N|N|N|N| +2441679|AAAAAAAAPMBEFCAA|1972-12-27|875|3809|293|1972|3|12|27|4|1972|293|3809|Wednesday|1972Q4|N|N|N|2441653|2441987|2441313|2441587|N|N|N|N|N| +2441680|AAAAAAAAANBEFCAA|1972-12-28|875|3809|293|1972|4|12|28|4|1972|293|3809|Thursday|1972Q4|N|N|N|2441653|2441987|2441314|2441588|N|N|N|N|N| +2441681|AAAAAAAABNBEFCAA|1972-12-29|875|3809|293|1972|5|12|29|4|1972|293|3809|Friday|1972Q4|N|Y|N|2441653|2441987|2441315|2441589|N|N|N|N|N| +2441682|AAAAAAAACNBEFCAA|1972-12-30|875|3809|293|1972|6|12|30|4|1972|293|3809|Saturday|1972Q4|N|Y|N|2441653|2441987|2441316|2441590|N|N|N|N|N| +2441683|AAAAAAAADNBEFCAA|1972-12-31|875|3809|293|1972|0|12|31|4|1972|293|3809|Sunday|1972Q4|Y|N|N|2441653|2441987|2441317|2441591|N|N|N|N|N| +2441684|AAAAAAAAENBEFCAA|1973-01-01|876|3809|293|1973|1|1|1|1|1973|293|3809|Monday|1973Q1|Y|N|Y|2441684|2441683|2441318|2441592|N|N|N|N|N| +2441685|AAAAAAAAFNBEFCAA|1973-01-02|876|3810|293|1973|2|1|2|1|1973|293|3810|Tuesday|1973Q1|N|N|Y|2441684|2441683|2441319|2441593|N|N|N|N|N| +2441686|AAAAAAAAGNBEFCAA|1973-01-03|876|3810|293|1973|3|1|3|1|1973|293|3810|Wednesday|1973Q1|N|N|N|2441684|2441683|2441320|2441594|N|N|N|N|N| +2441687|AAAAAAAAHNBEFCAA|1973-01-04|876|3810|293|1973|4|1|4|1|1973|293|3810|Thursday|1973Q1|N|N|N|2441684|2441683|2441321|2441595|N|N|N|N|N| +2441688|AAAAAAAAINBEFCAA|1973-01-05|876|3810|293|1973|5|1|5|1|1973|293|3810|Friday|1973Q1|N|Y|N|2441684|2441683|2441322|2441596|N|N|N|N|N| +2441689|AAAAAAAAJNBEFCAA|1973-01-06|876|3810|293|1973|6|1|6|1|1973|293|3810|Saturday|1973Q1|N|Y|N|2441684|2441683|2441323|2441597|N|N|N|N|N| +2441690|AAAAAAAAKNBEFCAA|1973-01-07|876|3810|293|1973|0|1|7|1|1973|293|3810|Sunday|1973Q1|N|N|N|2441684|2441683|2441324|2441598|N|N|N|N|N| +2441691|AAAAAAAALNBEFCAA|1973-01-08|876|3810|293|1973|1|1|8|1|1973|293|3810|Monday|1973Q1|N|N|N|2441684|2441683|2441325|2441599|N|N|N|N|N| +2441692|AAAAAAAAMNBEFCAA|1973-01-09|876|3811|293|1973|2|1|9|1|1973|293|3811|Tuesday|1973Q1|N|N|N|2441684|2441683|2441326|2441600|N|N|N|N|N| +2441693|AAAAAAAANNBEFCAA|1973-01-10|876|3811|293|1973|3|1|10|1|1973|293|3811|Wednesday|1973Q1|N|N|N|2441684|2441683|2441327|2441601|N|N|N|N|N| +2441694|AAAAAAAAONBEFCAA|1973-01-11|876|3811|293|1973|4|1|11|1|1973|293|3811|Thursday|1973Q1|N|N|N|2441684|2441683|2441328|2441602|N|N|N|N|N| +2441695|AAAAAAAAPNBEFCAA|1973-01-12|876|3811|293|1973|5|1|12|1|1973|293|3811|Friday|1973Q1|N|Y|N|2441684|2441683|2441329|2441603|N|N|N|N|N| +2441696|AAAAAAAAAOBEFCAA|1973-01-13|876|3811|293|1973|6|1|13|1|1973|293|3811|Saturday|1973Q1|N|Y|N|2441684|2441683|2441330|2441604|N|N|N|N|N| +2441697|AAAAAAAABOBEFCAA|1973-01-14|876|3811|293|1973|0|1|14|1|1973|293|3811|Sunday|1973Q1|N|N|N|2441684|2441683|2441331|2441605|N|N|N|N|N| +2441698|AAAAAAAACOBEFCAA|1973-01-15|876|3811|293|1973|1|1|15|1|1973|293|3811|Monday|1973Q1|N|N|N|2441684|2441683|2441332|2441606|N|N|N|N|N| +2441699|AAAAAAAADOBEFCAA|1973-01-16|876|3812|293|1973|2|1|16|1|1973|293|3812|Tuesday|1973Q1|N|N|N|2441684|2441683|2441333|2441607|N|N|N|N|N| +2441700|AAAAAAAAEOBEFCAA|1973-01-17|876|3812|293|1973|3|1|17|1|1973|293|3812|Wednesday|1973Q1|N|N|N|2441684|2441683|2441334|2441608|N|N|N|N|N| +2441701|AAAAAAAAFOBEFCAA|1973-01-18|876|3812|293|1973|4|1|18|1|1973|293|3812|Thursday|1973Q1|N|N|N|2441684|2441683|2441335|2441609|N|N|N|N|N| +2441702|AAAAAAAAGOBEFCAA|1973-01-19|876|3812|293|1973|5|1|19|1|1973|293|3812|Friday|1973Q1|N|Y|N|2441684|2441683|2441336|2441610|N|N|N|N|N| +2441703|AAAAAAAAHOBEFCAA|1973-01-20|876|3812|293|1973|6|1|20|1|1973|293|3812|Saturday|1973Q1|N|Y|N|2441684|2441683|2441337|2441611|N|N|N|N|N| +2441704|AAAAAAAAIOBEFCAA|1973-01-21|876|3812|293|1973|0|1|21|1|1973|293|3812|Sunday|1973Q1|N|N|N|2441684|2441683|2441338|2441612|N|N|N|N|N| +2441705|AAAAAAAAJOBEFCAA|1973-01-22|876|3812|293|1973|1|1|22|1|1973|293|3812|Monday|1973Q1|N|N|N|2441684|2441683|2441339|2441613|N|N|N|N|N| +2441706|AAAAAAAAKOBEFCAA|1973-01-23|876|3813|293|1973|2|1|23|1|1973|293|3813|Tuesday|1973Q1|N|N|N|2441684|2441683|2441340|2441614|N|N|N|N|N| +2441707|AAAAAAAALOBEFCAA|1973-01-24|876|3813|293|1973|3|1|24|1|1973|293|3813|Wednesday|1973Q1|N|N|N|2441684|2441683|2441341|2441615|N|N|N|N|N| +2441708|AAAAAAAAMOBEFCAA|1973-01-25|876|3813|293|1973|4|1|25|1|1973|293|3813|Thursday|1973Q1|N|N|N|2441684|2441683|2441342|2441616|N|N|N|N|N| +2441709|AAAAAAAANOBEFCAA|1973-01-26|876|3813|293|1973|5|1|26|1|1973|293|3813|Friday|1973Q1|N|Y|N|2441684|2441683|2441343|2441617|N|N|N|N|N| +2441710|AAAAAAAAOOBEFCAA|1973-01-27|876|3813|293|1973|6|1|27|1|1973|293|3813|Saturday|1973Q1|N|Y|N|2441684|2441683|2441344|2441618|N|N|N|N|N| +2441711|AAAAAAAAPOBEFCAA|1973-01-28|876|3813|293|1973|0|1|28|1|1973|293|3813|Sunday|1973Q1|N|N|N|2441684|2441683|2441345|2441619|N|N|N|N|N| +2441712|AAAAAAAAAPBEFCAA|1973-01-29|876|3813|293|1973|1|1|29|1|1973|293|3813|Monday|1973Q1|N|N|N|2441684|2441683|2441346|2441620|N|N|N|N|N| +2441713|AAAAAAAABPBEFCAA|1973-01-30|876|3814|293|1973|2|1|30|1|1973|293|3814|Tuesday|1973Q1|N|N|N|2441684|2441683|2441347|2441621|N|N|N|N|N| +2441714|AAAAAAAACPBEFCAA|1973-01-31|876|3814|293|1973|3|1|31|1|1973|293|3814|Wednesday|1973Q1|N|N|N|2441684|2441683|2441348|2441622|N|N|N|N|N| +2441715|AAAAAAAADPBEFCAA|1973-02-01|877|3814|293|1973|4|2|1|1|1973|293|3814|Thursday|1973Q1|N|N|N|2441715|2441745|2441349|2441623|N|N|N|N|N| +2441716|AAAAAAAAEPBEFCAA|1973-02-02|877|3814|293|1973|5|2|2|1|1973|293|3814|Friday|1973Q1|N|Y|N|2441715|2441745|2441350|2441624|N|N|N|N|N| +2441717|AAAAAAAAFPBEFCAA|1973-02-03|877|3814|293|1973|6|2|3|1|1973|293|3814|Saturday|1973Q1|N|Y|N|2441715|2441745|2441351|2441625|N|N|N|N|N| +2441718|AAAAAAAAGPBEFCAA|1973-02-04|877|3814|293|1973|0|2|4|1|1973|293|3814|Sunday|1973Q1|N|N|N|2441715|2441745|2441352|2441626|N|N|N|N|N| +2441719|AAAAAAAAHPBEFCAA|1973-02-05|877|3814|293|1973|1|2|5|1|1973|293|3814|Monday|1973Q1|N|N|N|2441715|2441745|2441353|2441627|N|N|N|N|N| +2441720|AAAAAAAAIPBEFCAA|1973-02-06|877|3815|293|1973|2|2|6|1|1973|293|3815|Tuesday|1973Q1|N|N|N|2441715|2441745|2441354|2441628|N|N|N|N|N| +2441721|AAAAAAAAJPBEFCAA|1973-02-07|877|3815|293|1973|3|2|7|1|1973|293|3815|Wednesday|1973Q1|N|N|N|2441715|2441745|2441355|2441629|N|N|N|N|N| +2441722|AAAAAAAAKPBEFCAA|1973-02-08|877|3815|293|1973|4|2|8|1|1973|293|3815|Thursday|1973Q1|N|N|N|2441715|2441745|2441356|2441630|N|N|N|N|N| +2441723|AAAAAAAALPBEFCAA|1973-02-09|877|3815|293|1973|5|2|9|1|1973|293|3815|Friday|1973Q1|N|Y|N|2441715|2441745|2441357|2441631|N|N|N|N|N| +2441724|AAAAAAAAMPBEFCAA|1973-02-10|877|3815|293|1973|6|2|10|1|1973|293|3815|Saturday|1973Q1|N|Y|N|2441715|2441745|2441358|2441632|N|N|N|N|N| +2441725|AAAAAAAANPBEFCAA|1973-02-11|877|3815|293|1973|0|2|11|1|1973|293|3815|Sunday|1973Q1|N|N|N|2441715|2441745|2441359|2441633|N|N|N|N|N| +2441726|AAAAAAAAOPBEFCAA|1973-02-12|877|3815|293|1973|1|2|12|1|1973|293|3815|Monday|1973Q1|N|N|N|2441715|2441745|2441360|2441634|N|N|N|N|N| +2441727|AAAAAAAAPPBEFCAA|1973-02-13|877|3816|293|1973|2|2|13|1|1973|293|3816|Tuesday|1973Q1|N|N|N|2441715|2441745|2441361|2441635|N|N|N|N|N| +2441728|AAAAAAAAAACEFCAA|1973-02-14|877|3816|293|1973|3|2|14|1|1973|293|3816|Wednesday|1973Q1|N|N|N|2441715|2441745|2441362|2441636|N|N|N|N|N| +2441729|AAAAAAAABACEFCAA|1973-02-15|877|3816|293|1973|4|2|15|1|1973|293|3816|Thursday|1973Q1|N|N|N|2441715|2441745|2441363|2441637|N|N|N|N|N| +2441730|AAAAAAAACACEFCAA|1973-02-16|877|3816|293|1973|5|2|16|1|1973|293|3816|Friday|1973Q1|N|Y|N|2441715|2441745|2441364|2441638|N|N|N|N|N| +2441731|AAAAAAAADACEFCAA|1973-02-17|877|3816|293|1973|6|2|17|1|1973|293|3816|Saturday|1973Q1|N|Y|N|2441715|2441745|2441365|2441639|N|N|N|N|N| +2441732|AAAAAAAAEACEFCAA|1973-02-18|877|3816|293|1973|0|2|18|1|1973|293|3816|Sunday|1973Q1|N|N|N|2441715|2441745|2441366|2441640|N|N|N|N|N| +2441733|AAAAAAAAFACEFCAA|1973-02-19|877|3816|293|1973|1|2|19|1|1973|293|3816|Monday|1973Q1|N|N|N|2441715|2441745|2441367|2441641|N|N|N|N|N| +2441734|AAAAAAAAGACEFCAA|1973-02-20|877|3817|293|1973|2|2|20|1|1973|293|3817|Tuesday|1973Q1|N|N|N|2441715|2441745|2441368|2441642|N|N|N|N|N| +2441735|AAAAAAAAHACEFCAA|1973-02-21|877|3817|293|1973|3|2|21|1|1973|293|3817|Wednesday|1973Q1|N|N|N|2441715|2441745|2441369|2441643|N|N|N|N|N| +2441736|AAAAAAAAIACEFCAA|1973-02-22|877|3817|293|1973|4|2|22|1|1973|293|3817|Thursday|1973Q1|N|N|N|2441715|2441745|2441370|2441644|N|N|N|N|N| +2441737|AAAAAAAAJACEFCAA|1973-02-23|877|3817|293|1973|5|2|23|1|1973|293|3817|Friday|1973Q1|N|Y|N|2441715|2441745|2441371|2441645|N|N|N|N|N| +2441738|AAAAAAAAKACEFCAA|1973-02-24|877|3817|293|1973|6|2|24|1|1973|293|3817|Saturday|1973Q1|N|Y|N|2441715|2441745|2441372|2441646|N|N|N|N|N| +2441739|AAAAAAAALACEFCAA|1973-02-25|877|3817|293|1973|0|2|25|1|1973|293|3817|Sunday|1973Q1|N|N|N|2441715|2441745|2441373|2441647|N|N|N|N|N| +2441740|AAAAAAAAMACEFCAA|1973-02-26|877|3817|293|1973|1|2|26|1|1973|293|3817|Monday|1973Q1|N|N|N|2441715|2441745|2441374|2441648|N|N|N|N|N| +2441741|AAAAAAAANACEFCAA|1973-02-27|877|3818|293|1973|2|2|27|1|1973|293|3818|Tuesday|1973Q1|N|N|N|2441715|2441745|2441375|2441649|N|N|N|N|N| +2441742|AAAAAAAAOACEFCAA|1973-02-28|877|3818|293|1973|3|2|28|1|1973|293|3818|Wednesday|1973Q1|N|N|N|2441715|2441745|2441376|2441650|N|N|N|N|N| +2441743|AAAAAAAAPACEFCAA|1973-03-01|878|3818|294|1973|4|3|1|1|1973|294|3818|Thursday|1973Q1|N|N|N|2441743|2441801|2441378|2441651|N|N|N|N|N| +2441744|AAAAAAAAABCEFCAA|1973-03-02|878|3818|294|1973|5|3|2|1|1973|294|3818|Friday|1973Q1|N|Y|N|2441743|2441801|2441379|2441652|N|N|N|N|N| +2441745|AAAAAAAABBCEFCAA|1973-03-03|878|3818|294|1973|6|3|3|1|1973|294|3818|Saturday|1973Q1|N|Y|N|2441743|2441801|2441380|2441653|N|N|N|N|N| +2441746|AAAAAAAACBCEFCAA|1973-03-04|878|3818|294|1973|0|3|4|1|1973|294|3818|Sunday|1973Q1|N|N|N|2441743|2441801|2441381|2441654|N|N|N|N|N| +2441747|AAAAAAAADBCEFCAA|1973-03-05|878|3818|294|1973|1|3|5|1|1973|294|3818|Monday|1973Q1|N|N|N|2441743|2441801|2441382|2441655|N|N|N|N|N| +2441748|AAAAAAAAEBCEFCAA|1973-03-06|878|3819|294|1973|2|3|6|1|1973|294|3819|Tuesday|1973Q1|N|N|N|2441743|2441801|2441383|2441656|N|N|N|N|N| +2441749|AAAAAAAAFBCEFCAA|1973-03-07|878|3819|294|1973|3|3|7|1|1973|294|3819|Wednesday|1973Q1|N|N|N|2441743|2441801|2441384|2441657|N|N|N|N|N| +2441750|AAAAAAAAGBCEFCAA|1973-03-08|878|3819|294|1973|4|3|8|1|1973|294|3819|Thursday|1973Q1|N|N|N|2441743|2441801|2441385|2441658|N|N|N|N|N| +2441751|AAAAAAAAHBCEFCAA|1973-03-09|878|3819|294|1973|5|3|9|1|1973|294|3819|Friday|1973Q1|N|Y|N|2441743|2441801|2441386|2441659|N|N|N|N|N| +2441752|AAAAAAAAIBCEFCAA|1973-03-10|878|3819|294|1973|6|3|10|1|1973|294|3819|Saturday|1973Q1|N|Y|N|2441743|2441801|2441387|2441660|N|N|N|N|N| +2441753|AAAAAAAAJBCEFCAA|1973-03-11|878|3819|294|1973|0|3|11|1|1973|294|3819|Sunday|1973Q1|N|N|N|2441743|2441801|2441388|2441661|N|N|N|N|N| +2441754|AAAAAAAAKBCEFCAA|1973-03-12|878|3819|294|1973|1|3|12|1|1973|294|3819|Monday|1973Q1|N|N|N|2441743|2441801|2441389|2441662|N|N|N|N|N| +2441755|AAAAAAAALBCEFCAA|1973-03-13|878|3820|294|1973|2|3|13|1|1973|294|3820|Tuesday|1973Q1|N|N|N|2441743|2441801|2441390|2441663|N|N|N|N|N| +2441756|AAAAAAAAMBCEFCAA|1973-03-14|878|3820|294|1973|3|3|14|1|1973|294|3820|Wednesday|1973Q1|N|N|N|2441743|2441801|2441391|2441664|N|N|N|N|N| +2441757|AAAAAAAANBCEFCAA|1973-03-15|878|3820|294|1973|4|3|15|1|1973|294|3820|Thursday|1973Q1|N|N|N|2441743|2441801|2441392|2441665|N|N|N|N|N| +2441758|AAAAAAAAOBCEFCAA|1973-03-16|878|3820|294|1973|5|3|16|1|1973|294|3820|Friday|1973Q1|N|Y|N|2441743|2441801|2441393|2441666|N|N|N|N|N| +2441759|AAAAAAAAPBCEFCAA|1973-03-17|878|3820|294|1973|6|3|17|1|1973|294|3820|Saturday|1973Q1|N|Y|N|2441743|2441801|2441394|2441667|N|N|N|N|N| +2441760|AAAAAAAAACCEFCAA|1973-03-18|878|3820|294|1973|0|3|18|1|1973|294|3820|Sunday|1973Q1|N|N|N|2441743|2441801|2441395|2441668|N|N|N|N|N| +2441761|AAAAAAAABCCEFCAA|1973-03-19|878|3820|294|1973|1|3|19|1|1973|294|3820|Monday|1973Q1|N|N|N|2441743|2441801|2441396|2441669|N|N|N|N|N| +2441762|AAAAAAAACCCEFCAA|1973-03-20|878|3821|294|1973|2|3|20|1|1973|294|3821|Tuesday|1973Q1|N|N|N|2441743|2441801|2441397|2441670|N|N|N|N|N| +2441763|AAAAAAAADCCEFCAA|1973-03-21|878|3821|294|1973|3|3|21|1|1973|294|3821|Wednesday|1973Q1|N|N|N|2441743|2441801|2441398|2441671|N|N|N|N|N| +2441764|AAAAAAAAECCEFCAA|1973-03-22|878|3821|294|1973|4|3|22|1|1973|294|3821|Thursday|1973Q1|N|N|N|2441743|2441801|2441399|2441672|N|N|N|N|N| +2441765|AAAAAAAAFCCEFCAA|1973-03-23|878|3821|294|1973|5|3|23|1|1973|294|3821|Friday|1973Q1|N|Y|N|2441743|2441801|2441400|2441673|N|N|N|N|N| +2441766|AAAAAAAAGCCEFCAA|1973-03-24|878|3821|294|1973|6|3|24|1|1973|294|3821|Saturday|1973Q1|N|Y|N|2441743|2441801|2441401|2441674|N|N|N|N|N| +2441767|AAAAAAAAHCCEFCAA|1973-03-25|878|3821|294|1973|0|3|25|1|1973|294|3821|Sunday|1973Q1|N|N|N|2441743|2441801|2441402|2441675|N|N|N|N|N| +2441768|AAAAAAAAICCEFCAA|1973-03-26|878|3821|294|1973|1|3|26|1|1973|294|3821|Monday|1973Q1|N|N|N|2441743|2441801|2441403|2441676|N|N|N|N|N| +2441769|AAAAAAAAJCCEFCAA|1973-03-27|878|3822|294|1973|2|3|27|1|1973|294|3822|Tuesday|1973Q1|N|N|N|2441743|2441801|2441404|2441677|N|N|N|N|N| +2441770|AAAAAAAAKCCEFCAA|1973-03-28|878|3822|294|1973|3|3|28|1|1973|294|3822|Wednesday|1973Q1|N|N|N|2441743|2441801|2441405|2441678|N|N|N|N|N| +2441771|AAAAAAAALCCEFCAA|1973-03-29|878|3822|294|1973|4|3|29|1|1973|294|3822|Thursday|1973Q1|N|N|N|2441743|2441801|2441406|2441679|N|N|N|N|N| +2441772|AAAAAAAAMCCEFCAA|1973-03-30|878|3822|294|1973|5|3|30|1|1973|294|3822|Friday|1973Q1|N|Y|N|2441743|2441801|2441407|2441680|N|N|N|N|N| +2441773|AAAAAAAANCCEFCAA|1973-03-31|878|3822|294|1973|6|3|31|1|1973|294|3822|Saturday|1973Q1|N|Y|N|2441743|2441801|2441408|2441681|N|N|N|N|N| +2441774|AAAAAAAAOCCEFCAA|1973-04-01|879|3822|294|1973|0|4|1|1|1973|294|3822|Sunday|1973Q1|N|N|N|2441774|2441863|2441409|2441684|N|N|N|N|N| +2441775|AAAAAAAAPCCEFCAA|1973-04-02|879|3822|294|1973|1|4|2|2|1973|294|3822|Monday|1973Q2|N|N|N|2441774|2441863|2441410|2441685|N|N|N|N|N| +2441776|AAAAAAAAADCEFCAA|1973-04-03|879|3823|294|1973|2|4|3|2|1973|294|3823|Tuesday|1973Q2|N|N|N|2441774|2441863|2441411|2441686|N|N|N|N|N| +2441777|AAAAAAAABDCEFCAA|1973-04-04|879|3823|294|1973|3|4|4|2|1973|294|3823|Wednesday|1973Q2|N|N|N|2441774|2441863|2441412|2441687|N|N|N|N|N| +2441778|AAAAAAAACDCEFCAA|1973-04-05|879|3823|294|1973|4|4|5|2|1973|294|3823|Thursday|1973Q2|N|N|N|2441774|2441863|2441413|2441688|N|N|N|N|N| +2441779|AAAAAAAADDCEFCAA|1973-04-06|879|3823|294|1973|5|4|6|2|1973|294|3823|Friday|1973Q2|N|Y|N|2441774|2441863|2441414|2441689|N|N|N|N|N| +2441780|AAAAAAAAEDCEFCAA|1973-04-07|879|3823|294|1973|6|4|7|2|1973|294|3823|Saturday|1973Q2|N|Y|N|2441774|2441863|2441415|2441690|N|N|N|N|N| +2441781|AAAAAAAAFDCEFCAA|1973-04-08|879|3823|294|1973|0|4|8|2|1973|294|3823|Sunday|1973Q2|N|N|N|2441774|2441863|2441416|2441691|N|N|N|N|N| +2441782|AAAAAAAAGDCEFCAA|1973-04-09|879|3823|294|1973|1|4|9|2|1973|294|3823|Monday|1973Q2|N|N|N|2441774|2441863|2441417|2441692|N|N|N|N|N| +2441783|AAAAAAAAHDCEFCAA|1973-04-10|879|3824|294|1973|2|4|10|2|1973|294|3824|Tuesday|1973Q2|N|N|N|2441774|2441863|2441418|2441693|N|N|N|N|N| +2441784|AAAAAAAAIDCEFCAA|1973-04-11|879|3824|294|1973|3|4|11|2|1973|294|3824|Wednesday|1973Q2|N|N|N|2441774|2441863|2441419|2441694|N|N|N|N|N| +2441785|AAAAAAAAJDCEFCAA|1973-04-12|879|3824|294|1973|4|4|12|2|1973|294|3824|Thursday|1973Q2|N|N|N|2441774|2441863|2441420|2441695|N|N|N|N|N| +2441786|AAAAAAAAKDCEFCAA|1973-04-13|879|3824|294|1973|5|4|13|2|1973|294|3824|Friday|1973Q2|N|Y|N|2441774|2441863|2441421|2441696|N|N|N|N|N| +2441787|AAAAAAAALDCEFCAA|1973-04-14|879|3824|294|1973|6|4|14|2|1973|294|3824|Saturday|1973Q2|N|Y|N|2441774|2441863|2441422|2441697|N|N|N|N|N| +2441788|AAAAAAAAMDCEFCAA|1973-04-15|879|3824|294|1973|0|4|15|2|1973|294|3824|Sunday|1973Q2|N|N|N|2441774|2441863|2441423|2441698|N|N|N|N|N| +2441789|AAAAAAAANDCEFCAA|1973-04-16|879|3824|294|1973|1|4|16|2|1973|294|3824|Monday|1973Q2|N|N|N|2441774|2441863|2441424|2441699|N|N|N|N|N| +2441790|AAAAAAAAODCEFCAA|1973-04-17|879|3825|294|1973|2|4|17|2|1973|294|3825|Tuesday|1973Q2|N|N|N|2441774|2441863|2441425|2441700|N|N|N|N|N| +2441791|AAAAAAAAPDCEFCAA|1973-04-18|879|3825|294|1973|3|4|18|2|1973|294|3825|Wednesday|1973Q2|N|N|N|2441774|2441863|2441426|2441701|N|N|N|N|N| +2441792|AAAAAAAAAECEFCAA|1973-04-19|879|3825|294|1973|4|4|19|2|1973|294|3825|Thursday|1973Q2|N|N|N|2441774|2441863|2441427|2441702|N|N|N|N|N| +2441793|AAAAAAAABECEFCAA|1973-04-20|879|3825|294|1973|5|4|20|2|1973|294|3825|Friday|1973Q2|N|Y|N|2441774|2441863|2441428|2441703|N|N|N|N|N| +2441794|AAAAAAAACECEFCAA|1973-04-21|879|3825|294|1973|6|4|21|2|1973|294|3825|Saturday|1973Q2|N|Y|N|2441774|2441863|2441429|2441704|N|N|N|N|N| +2441795|AAAAAAAADECEFCAA|1973-04-22|879|3825|294|1973|0|4|22|2|1973|294|3825|Sunday|1973Q2|N|N|N|2441774|2441863|2441430|2441705|N|N|N|N|N| +2441796|AAAAAAAAEECEFCAA|1973-04-23|879|3825|294|1973|1|4|23|2|1973|294|3825|Monday|1973Q2|N|N|N|2441774|2441863|2441431|2441706|N|N|N|N|N| +2441797|AAAAAAAAFECEFCAA|1973-04-24|879|3826|294|1973|2|4|24|2|1973|294|3826|Tuesday|1973Q2|N|N|N|2441774|2441863|2441432|2441707|N|N|N|N|N| +2441798|AAAAAAAAGECEFCAA|1973-04-25|879|3826|294|1973|3|4|25|2|1973|294|3826|Wednesday|1973Q2|N|N|N|2441774|2441863|2441433|2441708|N|N|N|N|N| +2441799|AAAAAAAAHECEFCAA|1973-04-26|879|3826|294|1973|4|4|26|2|1973|294|3826|Thursday|1973Q2|N|N|N|2441774|2441863|2441434|2441709|N|N|N|N|N| +2441800|AAAAAAAAIECEFCAA|1973-04-27|879|3826|294|1973|5|4|27|2|1973|294|3826|Friday|1973Q2|N|Y|N|2441774|2441863|2441435|2441710|N|N|N|N|N| +2441801|AAAAAAAAJECEFCAA|1973-04-28|879|3826|294|1973|6|4|28|2|1973|294|3826|Saturday|1973Q2|N|Y|N|2441774|2441863|2441436|2441711|N|N|N|N|N| +2441802|AAAAAAAAKECEFCAA|1973-04-29|879|3826|294|1973|0|4|29|2|1973|294|3826|Sunday|1973Q2|N|N|N|2441774|2441863|2441437|2441712|N|N|N|N|N| +2441803|AAAAAAAALECEFCAA|1973-04-30|879|3826|294|1973|1|4|30|2|1973|294|3826|Monday|1973Q2|N|N|N|2441774|2441863|2441438|2441713|N|N|N|N|N| +2441804|AAAAAAAAMECEFCAA|1973-05-01|880|3827|294|1973|2|5|1|2|1973|294|3827|Tuesday|1973Q2|N|N|N|2441804|2441923|2441439|2441714|N|N|N|N|N| +2441805|AAAAAAAANECEFCAA|1973-05-02|880|3827|294|1973|3|5|2|2|1973|294|3827|Wednesday|1973Q2|N|N|N|2441804|2441923|2441440|2441715|N|N|N|N|N| +2441806|AAAAAAAAOECEFCAA|1973-05-03|880|3827|294|1973|4|5|3|2|1973|294|3827|Thursday|1973Q2|N|N|N|2441804|2441923|2441441|2441716|N|N|N|N|N| +2441807|AAAAAAAAPECEFCAA|1973-05-04|880|3827|294|1973|5|5|4|2|1973|294|3827|Friday|1973Q2|N|Y|N|2441804|2441923|2441442|2441717|N|N|N|N|N| +2441808|AAAAAAAAAFCEFCAA|1973-05-05|880|3827|294|1973|6|5|5|2|1973|294|3827|Saturday|1973Q2|N|Y|N|2441804|2441923|2441443|2441718|N|N|N|N|N| +2441809|AAAAAAAABFCEFCAA|1973-05-06|880|3827|294|1973|0|5|6|2|1973|294|3827|Sunday|1973Q2|N|N|N|2441804|2441923|2441444|2441719|N|N|N|N|N| +2441810|AAAAAAAACFCEFCAA|1973-05-07|880|3827|294|1973|1|5|7|2|1973|294|3827|Monday|1973Q2|N|N|N|2441804|2441923|2441445|2441720|N|N|N|N|N| +2441811|AAAAAAAADFCEFCAA|1973-05-08|880|3828|294|1973|2|5|8|2|1973|294|3828|Tuesday|1973Q2|N|N|N|2441804|2441923|2441446|2441721|N|N|N|N|N| +2441812|AAAAAAAAEFCEFCAA|1973-05-09|880|3828|294|1973|3|5|9|2|1973|294|3828|Wednesday|1973Q2|N|N|N|2441804|2441923|2441447|2441722|N|N|N|N|N| +2441813|AAAAAAAAFFCEFCAA|1973-05-10|880|3828|294|1973|4|5|10|2|1973|294|3828|Thursday|1973Q2|N|N|N|2441804|2441923|2441448|2441723|N|N|N|N|N| +2441814|AAAAAAAAGFCEFCAA|1973-05-11|880|3828|294|1973|5|5|11|2|1973|294|3828|Friday|1973Q2|N|Y|N|2441804|2441923|2441449|2441724|N|N|N|N|N| +2441815|AAAAAAAAHFCEFCAA|1973-05-12|880|3828|294|1973|6|5|12|2|1973|294|3828|Saturday|1973Q2|N|Y|N|2441804|2441923|2441450|2441725|N|N|N|N|N| +2441816|AAAAAAAAIFCEFCAA|1973-05-13|880|3828|294|1973|0|5|13|2|1973|294|3828|Sunday|1973Q2|N|N|N|2441804|2441923|2441451|2441726|N|N|N|N|N| +2441817|AAAAAAAAJFCEFCAA|1973-05-14|880|3828|294|1973|1|5|14|2|1973|294|3828|Monday|1973Q2|N|N|N|2441804|2441923|2441452|2441727|N|N|N|N|N| +2441818|AAAAAAAAKFCEFCAA|1973-05-15|880|3829|294|1973|2|5|15|2|1973|294|3829|Tuesday|1973Q2|N|N|N|2441804|2441923|2441453|2441728|N|N|N|N|N| +2441819|AAAAAAAALFCEFCAA|1973-05-16|880|3829|294|1973|3|5|16|2|1973|294|3829|Wednesday|1973Q2|N|N|N|2441804|2441923|2441454|2441729|N|N|N|N|N| +2441820|AAAAAAAAMFCEFCAA|1973-05-17|880|3829|294|1973|4|5|17|2|1973|294|3829|Thursday|1973Q2|N|N|N|2441804|2441923|2441455|2441730|N|N|N|N|N| +2441821|AAAAAAAANFCEFCAA|1973-05-18|880|3829|294|1973|5|5|18|2|1973|294|3829|Friday|1973Q2|N|Y|N|2441804|2441923|2441456|2441731|N|N|N|N|N| +2441822|AAAAAAAAOFCEFCAA|1973-05-19|880|3829|294|1973|6|5|19|2|1973|294|3829|Saturday|1973Q2|N|Y|N|2441804|2441923|2441457|2441732|N|N|N|N|N| +2441823|AAAAAAAAPFCEFCAA|1973-05-20|880|3829|294|1973|0|5|20|2|1973|294|3829|Sunday|1973Q2|N|N|N|2441804|2441923|2441458|2441733|N|N|N|N|N| +2441824|AAAAAAAAAGCEFCAA|1973-05-21|880|3829|294|1973|1|5|21|2|1973|294|3829|Monday|1973Q2|N|N|N|2441804|2441923|2441459|2441734|N|N|N|N|N| +2441825|AAAAAAAABGCEFCAA|1973-05-22|880|3830|294|1973|2|5|22|2|1973|294|3830|Tuesday|1973Q2|N|N|N|2441804|2441923|2441460|2441735|N|N|N|N|N| +2441826|AAAAAAAACGCEFCAA|1973-05-23|880|3830|294|1973|3|5|23|2|1973|294|3830|Wednesday|1973Q2|N|N|N|2441804|2441923|2441461|2441736|N|N|N|N|N| +2441827|AAAAAAAADGCEFCAA|1973-05-24|880|3830|294|1973|4|5|24|2|1973|294|3830|Thursday|1973Q2|N|N|N|2441804|2441923|2441462|2441737|N|N|N|N|N| +2441828|AAAAAAAAEGCEFCAA|1973-05-25|880|3830|294|1973|5|5|25|2|1973|294|3830|Friday|1973Q2|N|Y|N|2441804|2441923|2441463|2441738|N|N|N|N|N| +2441829|AAAAAAAAFGCEFCAA|1973-05-26|880|3830|294|1973|6|5|26|2|1973|294|3830|Saturday|1973Q2|N|Y|N|2441804|2441923|2441464|2441739|N|N|N|N|N| +2441830|AAAAAAAAGGCEFCAA|1973-05-27|880|3830|294|1973|0|5|27|2|1973|294|3830|Sunday|1973Q2|N|N|N|2441804|2441923|2441465|2441740|N|N|N|N|N| +2441831|AAAAAAAAHGCEFCAA|1973-05-28|880|3830|294|1973|1|5|28|2|1973|294|3830|Monday|1973Q2|N|N|N|2441804|2441923|2441466|2441741|N|N|N|N|N| +2441832|AAAAAAAAIGCEFCAA|1973-05-29|880|3831|294|1973|2|5|29|2|1973|294|3831|Tuesday|1973Q2|N|N|N|2441804|2441923|2441467|2441742|N|N|N|N|N| +2441833|AAAAAAAAJGCEFCAA|1973-05-30|880|3831|294|1973|3|5|30|2|1973|294|3831|Wednesday|1973Q2|N|N|N|2441804|2441923|2441468|2441743|N|N|N|N|N| +2441834|AAAAAAAAKGCEFCAA|1973-05-31|880|3831|294|1973|4|5|31|2|1973|294|3831|Thursday|1973Q2|N|N|N|2441804|2441923|2441469|2441744|N|N|N|N|N| +2441835|AAAAAAAALGCEFCAA|1973-06-01|881|3831|295|1973|5|6|1|2|1973|295|3831|Friday|1973Q2|N|Y|N|2441835|2441985|2441470|2441745|N|N|N|N|N| +2441836|AAAAAAAAMGCEFCAA|1973-06-02|881|3831|295|1973|6|6|2|2|1973|295|3831|Saturday|1973Q2|N|Y|N|2441835|2441985|2441471|2441746|N|N|N|N|N| +2441837|AAAAAAAANGCEFCAA|1973-06-03|881|3831|295|1973|0|6|3|2|1973|295|3831|Sunday|1973Q2|N|N|N|2441835|2441985|2441472|2441747|N|N|N|N|N| +2441838|AAAAAAAAOGCEFCAA|1973-06-04|881|3831|295|1973|1|6|4|2|1973|295|3831|Monday|1973Q2|N|N|N|2441835|2441985|2441473|2441748|N|N|N|N|N| +2441839|AAAAAAAAPGCEFCAA|1973-06-05|881|3832|295|1973|2|6|5|2|1973|295|3832|Tuesday|1973Q2|N|N|N|2441835|2441985|2441474|2441749|N|N|N|N|N| +2441840|AAAAAAAAAHCEFCAA|1973-06-06|881|3832|295|1973|3|6|6|2|1973|295|3832|Wednesday|1973Q2|N|N|N|2441835|2441985|2441475|2441750|N|N|N|N|N| +2441841|AAAAAAAABHCEFCAA|1973-06-07|881|3832|295|1973|4|6|7|2|1973|295|3832|Thursday|1973Q2|N|N|N|2441835|2441985|2441476|2441751|N|N|N|N|N| +2441842|AAAAAAAACHCEFCAA|1973-06-08|881|3832|295|1973|5|6|8|2|1973|295|3832|Friday|1973Q2|N|Y|N|2441835|2441985|2441477|2441752|N|N|N|N|N| +2441843|AAAAAAAADHCEFCAA|1973-06-09|881|3832|295|1973|6|6|9|2|1973|295|3832|Saturday|1973Q2|N|Y|N|2441835|2441985|2441478|2441753|N|N|N|N|N| +2441844|AAAAAAAAEHCEFCAA|1973-06-10|881|3832|295|1973|0|6|10|2|1973|295|3832|Sunday|1973Q2|N|N|N|2441835|2441985|2441479|2441754|N|N|N|N|N| +2441845|AAAAAAAAFHCEFCAA|1973-06-11|881|3832|295|1973|1|6|11|2|1973|295|3832|Monday|1973Q2|N|N|N|2441835|2441985|2441480|2441755|N|N|N|N|N| +2441846|AAAAAAAAGHCEFCAA|1973-06-12|881|3833|295|1973|2|6|12|2|1973|295|3833|Tuesday|1973Q2|N|N|N|2441835|2441985|2441481|2441756|N|N|N|N|N| +2441847|AAAAAAAAHHCEFCAA|1973-06-13|881|3833|295|1973|3|6|13|2|1973|295|3833|Wednesday|1973Q2|N|N|N|2441835|2441985|2441482|2441757|N|N|N|N|N| +2441848|AAAAAAAAIHCEFCAA|1973-06-14|881|3833|295|1973|4|6|14|2|1973|295|3833|Thursday|1973Q2|N|N|N|2441835|2441985|2441483|2441758|N|N|N|N|N| +2441849|AAAAAAAAJHCEFCAA|1973-06-15|881|3833|295|1973|5|6|15|2|1973|295|3833|Friday|1973Q2|N|Y|N|2441835|2441985|2441484|2441759|N|N|N|N|N| +2441850|AAAAAAAAKHCEFCAA|1973-06-16|881|3833|295|1973|6|6|16|2|1973|295|3833|Saturday|1973Q2|N|Y|N|2441835|2441985|2441485|2441760|N|N|N|N|N| +2441851|AAAAAAAALHCEFCAA|1973-06-17|881|3833|295|1973|0|6|17|2|1973|295|3833|Sunday|1973Q2|N|N|N|2441835|2441985|2441486|2441761|N|N|N|N|N| +2441852|AAAAAAAAMHCEFCAA|1973-06-18|881|3833|295|1973|1|6|18|2|1973|295|3833|Monday|1973Q2|N|N|N|2441835|2441985|2441487|2441762|N|N|N|N|N| +2441853|AAAAAAAANHCEFCAA|1973-06-19|881|3834|295|1973|2|6|19|2|1973|295|3834|Tuesday|1973Q2|N|N|N|2441835|2441985|2441488|2441763|N|N|N|N|N| +2441854|AAAAAAAAOHCEFCAA|1973-06-20|881|3834|295|1973|3|6|20|2|1973|295|3834|Wednesday|1973Q2|N|N|N|2441835|2441985|2441489|2441764|N|N|N|N|N| +2441855|AAAAAAAAPHCEFCAA|1973-06-21|881|3834|295|1973|4|6|21|2|1973|295|3834|Thursday|1973Q2|N|N|N|2441835|2441985|2441490|2441765|N|N|N|N|N| +2441856|AAAAAAAAAICEFCAA|1973-06-22|881|3834|295|1973|5|6|22|2|1973|295|3834|Friday|1973Q2|N|Y|N|2441835|2441985|2441491|2441766|N|N|N|N|N| +2441857|AAAAAAAABICEFCAA|1973-06-23|881|3834|295|1973|6|6|23|2|1973|295|3834|Saturday|1973Q2|N|Y|N|2441835|2441985|2441492|2441767|N|N|N|N|N| +2441858|AAAAAAAACICEFCAA|1973-06-24|881|3834|295|1973|0|6|24|2|1973|295|3834|Sunday|1973Q2|N|N|N|2441835|2441985|2441493|2441768|N|N|N|N|N| +2441859|AAAAAAAADICEFCAA|1973-06-25|881|3834|295|1973|1|6|25|2|1973|295|3834|Monday|1973Q2|N|N|N|2441835|2441985|2441494|2441769|N|N|N|N|N| +2441860|AAAAAAAAEICEFCAA|1973-06-26|881|3835|295|1973|2|6|26|2|1973|295|3835|Tuesday|1973Q2|N|N|N|2441835|2441985|2441495|2441770|N|N|N|N|N| +2441861|AAAAAAAAFICEFCAA|1973-06-27|881|3835|295|1973|3|6|27|2|1973|295|3835|Wednesday|1973Q2|N|N|N|2441835|2441985|2441496|2441771|N|N|N|N|N| +2441862|AAAAAAAAGICEFCAA|1973-06-28|881|3835|295|1973|4|6|28|2|1973|295|3835|Thursday|1973Q2|N|N|N|2441835|2441985|2441497|2441772|N|N|N|N|N| +2441863|AAAAAAAAHICEFCAA|1973-06-29|881|3835|295|1973|5|6|29|2|1973|295|3835|Friday|1973Q2|N|Y|N|2441835|2441985|2441498|2441773|N|N|N|N|N| +2441864|AAAAAAAAIICEFCAA|1973-06-30|881|3835|295|1973|6|6|30|2|1973|295|3835|Saturday|1973Q2|N|Y|N|2441835|2441985|2441499|2441774|N|N|N|N|N| +2441865|AAAAAAAAJICEFCAA|1973-07-01|882|3835|295|1973|0|7|1|2|1973|295|3835|Sunday|1973Q2|N|N|N|2441865|2442045|2441500|2441774|N|N|N|N|N| +2441866|AAAAAAAAKICEFCAA|1973-07-02|882|3835|295|1973|1|7|2|3|1973|295|3835|Monday|1973Q3|N|N|N|2441865|2442045|2441501|2441775|N|N|N|N|N| +2441867|AAAAAAAALICEFCAA|1973-07-03|882|3836|295|1973|2|7|3|3|1973|295|3836|Tuesday|1973Q3|N|N|N|2441865|2442045|2441502|2441776|N|N|N|N|N| +2441868|AAAAAAAAMICEFCAA|1973-07-04|882|3836|295|1973|3|7|4|3|1973|295|3836|Wednesday|1973Q3|N|N|N|2441865|2442045|2441503|2441777|N|N|N|N|N| +2441869|AAAAAAAANICEFCAA|1973-07-05|882|3836|295|1973|4|7|5|3|1973|295|3836|Thursday|1973Q3|Y|N|N|2441865|2442045|2441504|2441778|N|N|N|N|N| +2441870|AAAAAAAAOICEFCAA|1973-07-06|882|3836|295|1973|5|7|6|3|1973|295|3836|Friday|1973Q3|N|Y|Y|2441865|2442045|2441505|2441779|N|N|N|N|N| +2441871|AAAAAAAAPICEFCAA|1973-07-07|882|3836|295|1973|6|7|7|3|1973|295|3836|Saturday|1973Q3|N|Y|N|2441865|2442045|2441506|2441780|N|N|N|N|N| +2441872|AAAAAAAAAJCEFCAA|1973-07-08|882|3836|295|1973|0|7|8|3|1973|295|3836|Sunday|1973Q3|N|N|N|2441865|2442045|2441507|2441781|N|N|N|N|N| +2441873|AAAAAAAABJCEFCAA|1973-07-09|882|3836|295|1973|1|7|9|3|1973|295|3836|Monday|1973Q3|N|N|N|2441865|2442045|2441508|2441782|N|N|N|N|N| +2441874|AAAAAAAACJCEFCAA|1973-07-10|882|3837|295|1973|2|7|10|3|1973|295|3837|Tuesday|1973Q3|N|N|N|2441865|2442045|2441509|2441783|N|N|N|N|N| +2441875|AAAAAAAADJCEFCAA|1973-07-11|882|3837|295|1973|3|7|11|3|1973|295|3837|Wednesday|1973Q3|N|N|N|2441865|2442045|2441510|2441784|N|N|N|N|N| +2441876|AAAAAAAAEJCEFCAA|1973-07-12|882|3837|295|1973|4|7|12|3|1973|295|3837|Thursday|1973Q3|N|N|N|2441865|2442045|2441511|2441785|N|N|N|N|N| +2441877|AAAAAAAAFJCEFCAA|1973-07-13|882|3837|295|1973|5|7|13|3|1973|295|3837|Friday|1973Q3|N|Y|N|2441865|2442045|2441512|2441786|N|N|N|N|N| +2441878|AAAAAAAAGJCEFCAA|1973-07-14|882|3837|295|1973|6|7|14|3|1973|295|3837|Saturday|1973Q3|N|Y|N|2441865|2442045|2441513|2441787|N|N|N|N|N| +2441879|AAAAAAAAHJCEFCAA|1973-07-15|882|3837|295|1973|0|7|15|3|1973|295|3837|Sunday|1973Q3|N|N|N|2441865|2442045|2441514|2441788|N|N|N|N|N| +2441880|AAAAAAAAIJCEFCAA|1973-07-16|882|3837|295|1973|1|7|16|3|1973|295|3837|Monday|1973Q3|N|N|N|2441865|2442045|2441515|2441789|N|N|N|N|N| +2441881|AAAAAAAAJJCEFCAA|1973-07-17|882|3838|295|1973|2|7|17|3|1973|295|3838|Tuesday|1973Q3|N|N|N|2441865|2442045|2441516|2441790|N|N|N|N|N| +2441882|AAAAAAAAKJCEFCAA|1973-07-18|882|3838|295|1973|3|7|18|3|1973|295|3838|Wednesday|1973Q3|N|N|N|2441865|2442045|2441517|2441791|N|N|N|N|N| +2441883|AAAAAAAALJCEFCAA|1973-07-19|882|3838|295|1973|4|7|19|3|1973|295|3838|Thursday|1973Q3|N|N|N|2441865|2442045|2441518|2441792|N|N|N|N|N| +2441884|AAAAAAAAMJCEFCAA|1973-07-20|882|3838|295|1973|5|7|20|3|1973|295|3838|Friday|1973Q3|N|Y|N|2441865|2442045|2441519|2441793|N|N|N|N|N| +2441885|AAAAAAAANJCEFCAA|1973-07-21|882|3838|295|1973|6|7|21|3|1973|295|3838|Saturday|1973Q3|N|Y|N|2441865|2442045|2441520|2441794|N|N|N|N|N| +2441886|AAAAAAAAOJCEFCAA|1973-07-22|882|3838|295|1973|0|7|22|3|1973|295|3838|Sunday|1973Q3|N|N|N|2441865|2442045|2441521|2441795|N|N|N|N|N| +2441887|AAAAAAAAPJCEFCAA|1973-07-23|882|3838|295|1973|1|7|23|3|1973|295|3838|Monday|1973Q3|N|N|N|2441865|2442045|2441522|2441796|N|N|N|N|N| +2441888|AAAAAAAAAKCEFCAA|1973-07-24|882|3839|295|1973|2|7|24|3|1973|295|3839|Tuesday|1973Q3|N|N|N|2441865|2442045|2441523|2441797|N|N|N|N|N| +2441889|AAAAAAAABKCEFCAA|1973-07-25|882|3839|295|1973|3|7|25|3|1973|295|3839|Wednesday|1973Q3|N|N|N|2441865|2442045|2441524|2441798|N|N|N|N|N| +2441890|AAAAAAAACKCEFCAA|1973-07-26|882|3839|295|1973|4|7|26|3|1973|295|3839|Thursday|1973Q3|N|N|N|2441865|2442045|2441525|2441799|N|N|N|N|N| +2441891|AAAAAAAADKCEFCAA|1973-07-27|882|3839|295|1973|5|7|27|3|1973|295|3839|Friday|1973Q3|N|Y|N|2441865|2442045|2441526|2441800|N|N|N|N|N| +2441892|AAAAAAAAEKCEFCAA|1973-07-28|882|3839|295|1973|6|7|28|3|1973|295|3839|Saturday|1973Q3|N|Y|N|2441865|2442045|2441527|2441801|N|N|N|N|N| +2441893|AAAAAAAAFKCEFCAA|1973-07-29|882|3839|295|1973|0|7|29|3|1973|295|3839|Sunday|1973Q3|N|N|N|2441865|2442045|2441528|2441802|N|N|N|N|N| +2441894|AAAAAAAAGKCEFCAA|1973-07-30|882|3839|295|1973|1|7|30|3|1973|295|3839|Monday|1973Q3|N|N|N|2441865|2442045|2441529|2441803|N|N|N|N|N| +2441895|AAAAAAAAHKCEFCAA|1973-07-31|882|3840|295|1973|2|7|31|3|1973|295|3840|Tuesday|1973Q3|N|N|N|2441865|2442045|2441530|2441804|N|N|N|N|N| +2441896|AAAAAAAAIKCEFCAA|1973-08-01|883|3840|295|1973|3|8|1|3|1973|295|3840|Wednesday|1973Q3|N|N|N|2441896|2442107|2441531|2441805|N|N|N|N|N| +2441897|AAAAAAAAJKCEFCAA|1973-08-02|883|3840|295|1973|4|8|2|3|1973|295|3840|Thursday|1973Q3|N|N|N|2441896|2442107|2441532|2441806|N|N|N|N|N| +2441898|AAAAAAAAKKCEFCAA|1973-08-03|883|3840|295|1973|5|8|3|3|1973|295|3840|Friday|1973Q3|N|Y|N|2441896|2442107|2441533|2441807|N|N|N|N|N| +2441899|AAAAAAAALKCEFCAA|1973-08-04|883|3840|295|1973|6|8|4|3|1973|295|3840|Saturday|1973Q3|N|Y|N|2441896|2442107|2441534|2441808|N|N|N|N|N| +2441900|AAAAAAAAMKCEFCAA|1973-08-05|883|3840|295|1973|0|8|5|3|1973|295|3840|Sunday|1973Q3|N|N|N|2441896|2442107|2441535|2441809|N|N|N|N|N| +2441901|AAAAAAAANKCEFCAA|1973-08-06|883|3840|295|1973|1|8|6|3|1973|295|3840|Monday|1973Q3|N|N|N|2441896|2442107|2441536|2441810|N|N|N|N|N| +2441902|AAAAAAAAOKCEFCAA|1973-08-07|883|3841|295|1973|2|8|7|3|1973|295|3841|Tuesday|1973Q3|N|N|N|2441896|2442107|2441537|2441811|N|N|N|N|N| +2441903|AAAAAAAAPKCEFCAA|1973-08-08|883|3841|295|1973|3|8|8|3|1973|295|3841|Wednesday|1973Q3|N|N|N|2441896|2442107|2441538|2441812|N|N|N|N|N| +2441904|AAAAAAAAALCEFCAA|1973-08-09|883|3841|295|1973|4|8|9|3|1973|295|3841|Thursday|1973Q3|N|N|N|2441896|2442107|2441539|2441813|N|N|N|N|N| +2441905|AAAAAAAABLCEFCAA|1973-08-10|883|3841|295|1973|5|8|10|3|1973|295|3841|Friday|1973Q3|N|Y|N|2441896|2442107|2441540|2441814|N|N|N|N|N| +2441906|AAAAAAAACLCEFCAA|1973-08-11|883|3841|295|1973|6|8|11|3|1973|295|3841|Saturday|1973Q3|N|Y|N|2441896|2442107|2441541|2441815|N|N|N|N|N| +2441907|AAAAAAAADLCEFCAA|1973-08-12|883|3841|295|1973|0|8|12|3|1973|295|3841|Sunday|1973Q3|N|N|N|2441896|2442107|2441542|2441816|N|N|N|N|N| +2441908|AAAAAAAAELCEFCAA|1973-08-13|883|3841|295|1973|1|8|13|3|1973|295|3841|Monday|1973Q3|N|N|N|2441896|2442107|2441543|2441817|N|N|N|N|N| +2441909|AAAAAAAAFLCEFCAA|1973-08-14|883|3842|295|1973|2|8|14|3|1973|295|3842|Tuesday|1973Q3|N|N|N|2441896|2442107|2441544|2441818|N|N|N|N|N| +2441910|AAAAAAAAGLCEFCAA|1973-08-15|883|3842|295|1973|3|8|15|3|1973|295|3842|Wednesday|1973Q3|N|N|N|2441896|2442107|2441545|2441819|N|N|N|N|N| +2441911|AAAAAAAAHLCEFCAA|1973-08-16|883|3842|295|1973|4|8|16|3|1973|295|3842|Thursday|1973Q3|N|N|N|2441896|2442107|2441546|2441820|N|N|N|N|N| +2441912|AAAAAAAAILCEFCAA|1973-08-17|883|3842|295|1973|5|8|17|3|1973|295|3842|Friday|1973Q3|N|Y|N|2441896|2442107|2441547|2441821|N|N|N|N|N| +2441913|AAAAAAAAJLCEFCAA|1973-08-18|883|3842|295|1973|6|8|18|3|1973|295|3842|Saturday|1973Q3|N|Y|N|2441896|2442107|2441548|2441822|N|N|N|N|N| +2441914|AAAAAAAAKLCEFCAA|1973-08-19|883|3842|295|1973|0|8|19|3|1973|295|3842|Sunday|1973Q3|N|N|N|2441896|2442107|2441549|2441823|N|N|N|N|N| +2441915|AAAAAAAALLCEFCAA|1973-08-20|883|3842|295|1973|1|8|20|3|1973|295|3842|Monday|1973Q3|N|N|N|2441896|2442107|2441550|2441824|N|N|N|N|N| +2441916|AAAAAAAAMLCEFCAA|1973-08-21|883|3843|295|1973|2|8|21|3|1973|295|3843|Tuesday|1973Q3|N|N|N|2441896|2442107|2441551|2441825|N|N|N|N|N| +2441917|AAAAAAAANLCEFCAA|1973-08-22|883|3843|295|1973|3|8|22|3|1973|295|3843|Wednesday|1973Q3|N|N|N|2441896|2442107|2441552|2441826|N|N|N|N|N| +2441918|AAAAAAAAOLCEFCAA|1973-08-23|883|3843|295|1973|4|8|23|3|1973|295|3843|Thursday|1973Q3|N|N|N|2441896|2442107|2441553|2441827|N|N|N|N|N| +2441919|AAAAAAAAPLCEFCAA|1973-08-24|883|3843|295|1973|5|8|24|3|1973|295|3843|Friday|1973Q3|N|Y|N|2441896|2442107|2441554|2441828|N|N|N|N|N| +2441920|AAAAAAAAAMCEFCAA|1973-08-25|883|3843|295|1973|6|8|25|3|1973|295|3843|Saturday|1973Q3|N|Y|N|2441896|2442107|2441555|2441829|N|N|N|N|N| +2441921|AAAAAAAABMCEFCAA|1973-08-26|883|3843|295|1973|0|8|26|3|1973|295|3843|Sunday|1973Q3|N|N|N|2441896|2442107|2441556|2441830|N|N|N|N|N| +2441922|AAAAAAAACMCEFCAA|1973-08-27|883|3843|295|1973|1|8|27|3|1973|295|3843|Monday|1973Q3|N|N|N|2441896|2442107|2441557|2441831|N|N|N|N|N| +2441923|AAAAAAAADMCEFCAA|1973-08-28|883|3844|295|1973|2|8|28|3|1973|295|3844|Tuesday|1973Q3|N|N|N|2441896|2442107|2441558|2441832|N|N|N|N|N| +2441924|AAAAAAAAEMCEFCAA|1973-08-29|883|3844|295|1973|3|8|29|3|1973|295|3844|Wednesday|1973Q3|N|N|N|2441896|2442107|2441559|2441833|N|N|N|N|N| +2441925|AAAAAAAAFMCEFCAA|1973-08-30|883|3844|295|1973|4|8|30|3|1973|295|3844|Thursday|1973Q3|N|N|N|2441896|2442107|2441560|2441834|N|N|N|N|N| +2441926|AAAAAAAAGMCEFCAA|1973-08-31|883|3844|295|1973|5|8|31|3|1973|295|3844|Friday|1973Q3|N|Y|N|2441896|2442107|2441561|2441835|N|N|N|N|N| +2441927|AAAAAAAAHMCEFCAA|1973-09-01|884|3844|296|1973|6|9|1|3|1973|296|3844|Saturday|1973Q3|N|Y|N|2441927|2442169|2441562|2441836|N|N|N|N|N| +2441928|AAAAAAAAIMCEFCAA|1973-09-02|884|3844|296|1973|0|9|2|3|1973|296|3844|Sunday|1973Q3|N|N|N|2441927|2442169|2441563|2441837|N|N|N|N|N| +2441929|AAAAAAAAJMCEFCAA|1973-09-03|884|3844|296|1973|1|9|3|3|1973|296|3844|Monday|1973Q3|N|N|N|2441927|2442169|2441564|2441838|N|N|N|N|N| +2441930|AAAAAAAAKMCEFCAA|1973-09-04|884|3845|296|1973|2|9|4|3|1973|296|3845|Tuesday|1973Q3|N|N|N|2441927|2442169|2441565|2441839|N|N|N|N|N| +2441931|AAAAAAAALMCEFCAA|1973-09-05|884|3845|296|1973|3|9|5|3|1973|296|3845|Wednesday|1973Q3|N|N|N|2441927|2442169|2441566|2441840|N|N|N|N|N| +2441932|AAAAAAAAMMCEFCAA|1973-09-06|884|3845|296|1973|4|9|6|3|1973|296|3845|Thursday|1973Q3|N|N|N|2441927|2442169|2441567|2441841|N|N|N|N|N| +2441933|AAAAAAAANMCEFCAA|1973-09-07|884|3845|296|1973|5|9|7|3|1973|296|3845|Friday|1973Q3|N|Y|N|2441927|2442169|2441568|2441842|N|N|N|N|N| +2441934|AAAAAAAAOMCEFCAA|1973-09-08|884|3845|296|1973|6|9|8|3|1973|296|3845|Saturday|1973Q3|N|Y|N|2441927|2442169|2441569|2441843|N|N|N|N|N| +2441935|AAAAAAAAPMCEFCAA|1973-09-09|884|3845|296|1973|0|9|9|3|1973|296|3845|Sunday|1973Q3|N|N|N|2441927|2442169|2441570|2441844|N|N|N|N|N| +2441936|AAAAAAAAANCEFCAA|1973-09-10|884|3845|296|1973|1|9|10|3|1973|296|3845|Monday|1973Q3|N|N|N|2441927|2442169|2441571|2441845|N|N|N|N|N| +2441937|AAAAAAAABNCEFCAA|1973-09-11|884|3846|296|1973|2|9|11|3|1973|296|3846|Tuesday|1973Q3|N|N|N|2441927|2442169|2441572|2441846|N|N|N|N|N| +2441938|AAAAAAAACNCEFCAA|1973-09-12|884|3846|296|1973|3|9|12|3|1973|296|3846|Wednesday|1973Q3|N|N|N|2441927|2442169|2441573|2441847|N|N|N|N|N| +2441939|AAAAAAAADNCEFCAA|1973-09-13|884|3846|296|1973|4|9|13|3|1973|296|3846|Thursday|1973Q3|N|N|N|2441927|2442169|2441574|2441848|N|N|N|N|N| +2441940|AAAAAAAAENCEFCAA|1973-09-14|884|3846|296|1973|5|9|14|3|1973|296|3846|Friday|1973Q3|N|Y|N|2441927|2442169|2441575|2441849|N|N|N|N|N| +2441941|AAAAAAAAFNCEFCAA|1973-09-15|884|3846|296|1973|6|9|15|3|1973|296|3846|Saturday|1973Q3|N|Y|N|2441927|2442169|2441576|2441850|N|N|N|N|N| +2441942|AAAAAAAAGNCEFCAA|1973-09-16|884|3846|296|1973|0|9|16|3|1973|296|3846|Sunday|1973Q3|N|N|N|2441927|2442169|2441577|2441851|N|N|N|N|N| +2441943|AAAAAAAAHNCEFCAA|1973-09-17|884|3846|296|1973|1|9|17|3|1973|296|3846|Monday|1973Q3|N|N|N|2441927|2442169|2441578|2441852|N|N|N|N|N| +2441944|AAAAAAAAINCEFCAA|1973-09-18|884|3847|296|1973|2|9|18|3|1973|296|3847|Tuesday|1973Q3|N|N|N|2441927|2442169|2441579|2441853|N|N|N|N|N| +2441945|AAAAAAAAJNCEFCAA|1973-09-19|884|3847|296|1973|3|9|19|3|1973|296|3847|Wednesday|1973Q3|N|N|N|2441927|2442169|2441580|2441854|N|N|N|N|N| +2441946|AAAAAAAAKNCEFCAA|1973-09-20|884|3847|296|1973|4|9|20|3|1973|296|3847|Thursday|1973Q3|N|N|N|2441927|2442169|2441581|2441855|N|N|N|N|N| +2441947|AAAAAAAALNCEFCAA|1973-09-21|884|3847|296|1973|5|9|21|3|1973|296|3847|Friday|1973Q3|N|Y|N|2441927|2442169|2441582|2441856|N|N|N|N|N| +2441948|AAAAAAAAMNCEFCAA|1973-09-22|884|3847|296|1973|6|9|22|3|1973|296|3847|Saturday|1973Q3|N|Y|N|2441927|2442169|2441583|2441857|N|N|N|N|N| +2441949|AAAAAAAANNCEFCAA|1973-09-23|884|3847|296|1973|0|9|23|3|1973|296|3847|Sunday|1973Q3|N|N|N|2441927|2442169|2441584|2441858|N|N|N|N|N| +2441950|AAAAAAAAONCEFCAA|1973-09-24|884|3847|296|1973|1|9|24|3|1973|296|3847|Monday|1973Q3|N|N|N|2441927|2442169|2441585|2441859|N|N|N|N|N| +2441951|AAAAAAAAPNCEFCAA|1973-09-25|884|3848|296|1973|2|9|25|3|1973|296|3848|Tuesday|1973Q3|N|N|N|2441927|2442169|2441586|2441860|N|N|N|N|N| +2441952|AAAAAAAAAOCEFCAA|1973-09-26|884|3848|296|1973|3|9|26|3|1973|296|3848|Wednesday|1973Q3|N|N|N|2441927|2442169|2441587|2441861|N|N|N|N|N| +2441953|AAAAAAAABOCEFCAA|1973-09-27|884|3848|296|1973|4|9|27|3|1973|296|3848|Thursday|1973Q3|N|N|N|2441927|2442169|2441588|2441862|N|N|N|N|N| +2441954|AAAAAAAACOCEFCAA|1973-09-28|884|3848|296|1973|5|9|28|3|1973|296|3848|Friday|1973Q3|N|Y|N|2441927|2442169|2441589|2441863|N|N|N|N|N| +2441955|AAAAAAAADOCEFCAA|1973-09-29|884|3848|296|1973|6|9|29|3|1973|296|3848|Saturday|1973Q3|N|Y|N|2441927|2442169|2441590|2441864|N|N|N|N|N| +2441956|AAAAAAAAEOCEFCAA|1973-09-30|884|3848|296|1973|0|9|30|3|1973|296|3848|Sunday|1973Q3|N|N|N|2441927|2442169|2441591|2441865|N|N|N|N|N| +2441957|AAAAAAAAFOCEFCAA|1973-10-01|885|3848|296|1973|1|10|1|3|1973|296|3848|Monday|1973Q3|N|N|N|2441957|2442229|2441592|2441865|N|N|N|N|N| +2441958|AAAAAAAAGOCEFCAA|1973-10-02|885|3849|296|1973|2|10|2|4|1973|296|3849|Tuesday|1973Q4|N|N|N|2441957|2442229|2441593|2441866|N|N|N|N|N| +2441959|AAAAAAAAHOCEFCAA|1973-10-03|885|3849|296|1973|3|10|3|4|1973|296|3849|Wednesday|1973Q4|N|N|N|2441957|2442229|2441594|2441867|N|N|N|N|N| +2441960|AAAAAAAAIOCEFCAA|1973-10-04|885|3849|296|1973|4|10|4|4|1973|296|3849|Thursday|1973Q4|N|N|N|2441957|2442229|2441595|2441868|N|N|N|N|N| +2441961|AAAAAAAAJOCEFCAA|1973-10-05|885|3849|296|1973|5|10|5|4|1973|296|3849|Friday|1973Q4|N|Y|N|2441957|2442229|2441596|2441869|N|N|N|N|N| +2441962|AAAAAAAAKOCEFCAA|1973-10-06|885|3849|296|1973|6|10|6|4|1973|296|3849|Saturday|1973Q4|N|Y|N|2441957|2442229|2441597|2441870|N|N|N|N|N| +2441963|AAAAAAAALOCEFCAA|1973-10-07|885|3849|296|1973|0|10|7|4|1973|296|3849|Sunday|1973Q4|N|N|N|2441957|2442229|2441598|2441871|N|N|N|N|N| +2441964|AAAAAAAAMOCEFCAA|1973-10-08|885|3849|296|1973|1|10|8|4|1973|296|3849|Monday|1973Q4|N|N|N|2441957|2442229|2441599|2441872|N|N|N|N|N| +2441965|AAAAAAAANOCEFCAA|1973-10-09|885|3850|296|1973|2|10|9|4|1973|296|3850|Tuesday|1973Q4|N|N|N|2441957|2442229|2441600|2441873|N|N|N|N|N| +2441966|AAAAAAAAOOCEFCAA|1973-10-10|885|3850|296|1973|3|10|10|4|1973|296|3850|Wednesday|1973Q4|N|N|N|2441957|2442229|2441601|2441874|N|N|N|N|N| +2441967|AAAAAAAAPOCEFCAA|1973-10-11|885|3850|296|1973|4|10|11|4|1973|296|3850|Thursday|1973Q4|N|N|N|2441957|2442229|2441602|2441875|N|N|N|N|N| +2441968|AAAAAAAAAPCEFCAA|1973-10-12|885|3850|296|1973|5|10|12|4|1973|296|3850|Friday|1973Q4|N|Y|N|2441957|2442229|2441603|2441876|N|N|N|N|N| +2441969|AAAAAAAABPCEFCAA|1973-10-13|885|3850|296|1973|6|10|13|4|1973|296|3850|Saturday|1973Q4|N|Y|N|2441957|2442229|2441604|2441877|N|N|N|N|N| +2441970|AAAAAAAACPCEFCAA|1973-10-14|885|3850|296|1973|0|10|14|4|1973|296|3850|Sunday|1973Q4|N|N|N|2441957|2442229|2441605|2441878|N|N|N|N|N| +2441971|AAAAAAAADPCEFCAA|1973-10-15|885|3850|296|1973|1|10|15|4|1973|296|3850|Monday|1973Q4|N|N|N|2441957|2442229|2441606|2441879|N|N|N|N|N| +2441972|AAAAAAAAEPCEFCAA|1973-10-16|885|3851|296|1973|2|10|16|4|1973|296|3851|Tuesday|1973Q4|N|N|N|2441957|2442229|2441607|2441880|N|N|N|N|N| +2441973|AAAAAAAAFPCEFCAA|1973-10-17|885|3851|296|1973|3|10|17|4|1973|296|3851|Wednesday|1973Q4|N|N|N|2441957|2442229|2441608|2441881|N|N|N|N|N| +2441974|AAAAAAAAGPCEFCAA|1973-10-18|885|3851|296|1973|4|10|18|4|1973|296|3851|Thursday|1973Q4|N|N|N|2441957|2442229|2441609|2441882|N|N|N|N|N| +2441975|AAAAAAAAHPCEFCAA|1973-10-19|885|3851|296|1973|5|10|19|4|1973|296|3851|Friday|1973Q4|N|Y|N|2441957|2442229|2441610|2441883|N|N|N|N|N| +2441976|AAAAAAAAIPCEFCAA|1973-10-20|885|3851|296|1973|6|10|20|4|1973|296|3851|Saturday|1973Q4|N|Y|N|2441957|2442229|2441611|2441884|N|N|N|N|N| +2441977|AAAAAAAAJPCEFCAA|1973-10-21|885|3851|296|1973|0|10|21|4|1973|296|3851|Sunday|1973Q4|N|N|N|2441957|2442229|2441612|2441885|N|N|N|N|N| +2441978|AAAAAAAAKPCEFCAA|1973-10-22|885|3851|296|1973|1|10|22|4|1973|296|3851|Monday|1973Q4|N|N|N|2441957|2442229|2441613|2441886|N|N|N|N|N| +2441979|AAAAAAAALPCEFCAA|1973-10-23|885|3852|296|1973|2|10|23|4|1973|296|3852|Tuesday|1973Q4|N|N|N|2441957|2442229|2441614|2441887|N|N|N|N|N| +2441980|AAAAAAAAMPCEFCAA|1973-10-24|885|3852|296|1973|3|10|24|4|1973|296|3852|Wednesday|1973Q4|N|N|N|2441957|2442229|2441615|2441888|N|N|N|N|N| +2441981|AAAAAAAANPCEFCAA|1973-10-25|885|3852|296|1973|4|10|25|4|1973|296|3852|Thursday|1973Q4|N|N|N|2441957|2442229|2441616|2441889|N|N|N|N|N| +2441982|AAAAAAAAOPCEFCAA|1973-10-26|885|3852|296|1973|5|10|26|4|1973|296|3852|Friday|1973Q4|N|Y|N|2441957|2442229|2441617|2441890|N|N|N|N|N| +2441983|AAAAAAAAPPCEFCAA|1973-10-27|885|3852|296|1973|6|10|27|4|1973|296|3852|Saturday|1973Q4|N|Y|N|2441957|2442229|2441618|2441891|N|N|N|N|N| +2441984|AAAAAAAAAADEFCAA|1973-10-28|885|3852|296|1973|0|10|28|4|1973|296|3852|Sunday|1973Q4|N|N|N|2441957|2442229|2441619|2441892|N|N|N|N|N| +2441985|AAAAAAAABADEFCAA|1973-10-29|885|3852|296|1973|1|10|29|4|1973|296|3852|Monday|1973Q4|N|N|N|2441957|2442229|2441620|2441893|N|N|N|N|N| +2441986|AAAAAAAACADEFCAA|1973-10-30|885|3853|296|1973|2|10|30|4|1973|296|3853|Tuesday|1973Q4|N|N|N|2441957|2442229|2441621|2441894|N|N|N|N|N| +2441987|AAAAAAAADADEFCAA|1973-10-31|885|3853|296|1973|3|10|31|4|1973|296|3853|Wednesday|1973Q4|N|N|N|2441957|2442229|2441622|2441895|N|N|N|N|N| +2441988|AAAAAAAAEADEFCAA|1973-11-01|886|3853|296|1973|4|11|1|4|1973|296|3853|Thursday|1973Q4|N|N|N|2441988|2442291|2441623|2441896|N|N|N|N|N| +2441989|AAAAAAAAFADEFCAA|1973-11-02|886|3853|296|1973|5|11|2|4|1973|296|3853|Friday|1973Q4|N|Y|N|2441988|2442291|2441624|2441897|N|N|N|N|N| +2441990|AAAAAAAAGADEFCAA|1973-11-03|886|3853|296|1973|6|11|3|4|1973|296|3853|Saturday|1973Q4|N|Y|N|2441988|2442291|2441625|2441898|N|N|N|N|N| +2441991|AAAAAAAAHADEFCAA|1973-11-04|886|3853|296|1973|0|11|4|4|1973|296|3853|Sunday|1973Q4|N|N|N|2441988|2442291|2441626|2441899|N|N|N|N|N| +2441992|AAAAAAAAIADEFCAA|1973-11-05|886|3853|296|1973|1|11|5|4|1973|296|3853|Monday|1973Q4|N|N|N|2441988|2442291|2441627|2441900|N|N|N|N|N| +2441993|AAAAAAAAJADEFCAA|1973-11-06|886|3854|296|1973|2|11|6|4|1973|296|3854|Tuesday|1973Q4|N|N|N|2441988|2442291|2441628|2441901|N|N|N|N|N| +2441994|AAAAAAAAKADEFCAA|1973-11-07|886|3854|296|1973|3|11|7|4|1973|296|3854|Wednesday|1973Q4|N|N|N|2441988|2442291|2441629|2441902|N|N|N|N|N| +2441995|AAAAAAAALADEFCAA|1973-11-08|886|3854|296|1973|4|11|8|4|1973|296|3854|Thursday|1973Q4|N|N|N|2441988|2442291|2441630|2441903|N|N|N|N|N| +2441996|AAAAAAAAMADEFCAA|1973-11-09|886|3854|296|1973|5|11|9|4|1973|296|3854|Friday|1973Q4|N|Y|N|2441988|2442291|2441631|2441904|N|N|N|N|N| +2441997|AAAAAAAANADEFCAA|1973-11-10|886|3854|296|1973|6|11|10|4|1973|296|3854|Saturday|1973Q4|N|Y|N|2441988|2442291|2441632|2441905|N|N|N|N|N| +2441998|AAAAAAAAOADEFCAA|1973-11-11|886|3854|296|1973|0|11|11|4|1973|296|3854|Sunday|1973Q4|N|N|N|2441988|2442291|2441633|2441906|N|N|N|N|N| +2441999|AAAAAAAAPADEFCAA|1973-11-12|886|3854|296|1973|1|11|12|4|1973|296|3854|Monday|1973Q4|N|N|N|2441988|2442291|2441634|2441907|N|N|N|N|N| +2442000|AAAAAAAAABDEFCAA|1973-11-13|886|3855|296|1973|2|11|13|4|1973|296|3855|Tuesday|1973Q4|N|N|N|2441988|2442291|2441635|2441908|N|N|N|N|N| +2442001|AAAAAAAABBDEFCAA|1973-11-14|886|3855|296|1973|3|11|14|4|1973|296|3855|Wednesday|1973Q4|N|N|N|2441988|2442291|2441636|2441909|N|N|N|N|N| +2442002|AAAAAAAACBDEFCAA|1973-11-15|886|3855|296|1973|4|11|15|4|1973|296|3855|Thursday|1973Q4|N|N|N|2441988|2442291|2441637|2441910|N|N|N|N|N| +2442003|AAAAAAAADBDEFCAA|1973-11-16|886|3855|296|1973|5|11|16|4|1973|296|3855|Friday|1973Q4|N|Y|N|2441988|2442291|2441638|2441911|N|N|N|N|N| +2442004|AAAAAAAAEBDEFCAA|1973-11-17|886|3855|296|1973|6|11|17|4|1973|296|3855|Saturday|1973Q4|N|Y|N|2441988|2442291|2441639|2441912|N|N|N|N|N| +2442005|AAAAAAAAFBDEFCAA|1973-11-18|886|3855|296|1973|0|11|18|4|1973|296|3855|Sunday|1973Q4|N|N|N|2441988|2442291|2441640|2441913|N|N|N|N|N| +2442006|AAAAAAAAGBDEFCAA|1973-11-19|886|3855|296|1973|1|11|19|4|1973|296|3855|Monday|1973Q4|N|N|N|2441988|2442291|2441641|2441914|N|N|N|N|N| +2442007|AAAAAAAAHBDEFCAA|1973-11-20|886|3856|296|1973|2|11|20|4|1973|296|3856|Tuesday|1973Q4|N|N|N|2441988|2442291|2441642|2441915|N|N|N|N|N| +2442008|AAAAAAAAIBDEFCAA|1973-11-21|886|3856|296|1973|3|11|21|4|1973|296|3856|Wednesday|1973Q4|N|N|N|2441988|2442291|2441643|2441916|N|N|N|N|N| +2442009|AAAAAAAAJBDEFCAA|1973-11-22|886|3856|296|1973|4|11|22|4|1973|296|3856|Thursday|1973Q4|N|N|N|2441988|2442291|2441644|2441917|N|N|N|N|N| +2442010|AAAAAAAAKBDEFCAA|1973-11-23|886|3856|296|1973|5|11|23|4|1973|296|3856|Friday|1973Q4|N|Y|N|2441988|2442291|2441645|2441918|N|N|N|N|N| +2442011|AAAAAAAALBDEFCAA|1973-11-24|886|3856|296|1973|6|11|24|4|1973|296|3856|Saturday|1973Q4|N|Y|N|2441988|2442291|2441646|2441919|N|N|N|N|N| +2442012|AAAAAAAAMBDEFCAA|1973-11-25|886|3856|296|1973|0|11|25|4|1973|296|3856|Sunday|1973Q4|N|N|N|2441988|2442291|2441647|2441920|N|N|N|N|N| +2442013|AAAAAAAANBDEFCAA|1973-11-26|886|3856|296|1973|1|11|26|4|1973|296|3856|Monday|1973Q4|N|N|N|2441988|2442291|2441648|2441921|N|N|N|N|N| +2442014|AAAAAAAAOBDEFCAA|1973-11-27|886|3857|296|1973|2|11|27|4|1973|296|3857|Tuesday|1973Q4|N|N|N|2441988|2442291|2441649|2441922|N|N|N|N|N| +2442015|AAAAAAAAPBDEFCAA|1973-11-28|886|3857|296|1973|3|11|28|4|1973|296|3857|Wednesday|1973Q4|N|N|N|2441988|2442291|2441650|2441923|N|N|N|N|N| +2442016|AAAAAAAAACDEFCAA|1973-11-29|886|3857|296|1973|4|11|29|4|1973|296|3857|Thursday|1973Q4|N|N|N|2441988|2442291|2441651|2441924|N|N|N|N|N| +2442017|AAAAAAAABCDEFCAA|1973-11-30|886|3857|296|1973|5|11|30|4|1973|296|3857|Friday|1973Q4|N|Y|N|2441988|2442291|2441652|2441925|N|N|N|N|N| +2442018|AAAAAAAACCDEFCAA|1973-12-01|887|3857|297|1973|6|12|1|4|1973|297|3857|Saturday|1973Q4|N|Y|N|2442018|2442351|2441653|2441926|N|N|N|N|N| +2442019|AAAAAAAADCDEFCAA|1973-12-02|887|3857|297|1973|0|12|2|4|1973|297|3857|Sunday|1973Q4|N|N|N|2442018|2442351|2441654|2441927|N|N|N|N|N| +2442020|AAAAAAAAECDEFCAA|1973-12-03|887|3857|297|1973|1|12|3|4|1973|297|3857|Monday|1973Q4|N|N|N|2442018|2442351|2441655|2441928|N|N|N|N|N| +2442021|AAAAAAAAFCDEFCAA|1973-12-04|887|3858|297|1973|2|12|4|4|1973|297|3858|Tuesday|1973Q4|N|N|N|2442018|2442351|2441656|2441929|N|N|N|N|N| +2442022|AAAAAAAAGCDEFCAA|1973-12-05|887|3858|297|1973|3|12|5|4|1973|297|3858|Wednesday|1973Q4|N|N|N|2442018|2442351|2441657|2441930|N|N|N|N|N| +2442023|AAAAAAAAHCDEFCAA|1973-12-06|887|3858|297|1973|4|12|6|4|1973|297|3858|Thursday|1973Q4|N|N|N|2442018|2442351|2441658|2441931|N|N|N|N|N| +2442024|AAAAAAAAICDEFCAA|1973-12-07|887|3858|297|1973|5|12|7|4|1973|297|3858|Friday|1973Q4|N|Y|N|2442018|2442351|2441659|2441932|N|N|N|N|N| +2442025|AAAAAAAAJCDEFCAA|1973-12-08|887|3858|297|1973|6|12|8|4|1973|297|3858|Saturday|1973Q4|N|Y|N|2442018|2442351|2441660|2441933|N|N|N|N|N| +2442026|AAAAAAAAKCDEFCAA|1973-12-09|887|3858|297|1973|0|12|9|4|1973|297|3858|Sunday|1973Q4|N|N|N|2442018|2442351|2441661|2441934|N|N|N|N|N| +2442027|AAAAAAAALCDEFCAA|1973-12-10|887|3858|297|1973|1|12|10|4|1973|297|3858|Monday|1973Q4|N|N|N|2442018|2442351|2441662|2441935|N|N|N|N|N| +2442028|AAAAAAAAMCDEFCAA|1973-12-11|887|3859|297|1973|2|12|11|4|1973|297|3859|Tuesday|1973Q4|N|N|N|2442018|2442351|2441663|2441936|N|N|N|N|N| +2442029|AAAAAAAANCDEFCAA|1973-12-12|887|3859|297|1973|3|12|12|4|1973|297|3859|Wednesday|1973Q4|N|N|N|2442018|2442351|2441664|2441937|N|N|N|N|N| +2442030|AAAAAAAAOCDEFCAA|1973-12-13|887|3859|297|1973|4|12|13|4|1973|297|3859|Thursday|1973Q4|N|N|N|2442018|2442351|2441665|2441938|N|N|N|N|N| +2442031|AAAAAAAAPCDEFCAA|1973-12-14|887|3859|297|1973|5|12|14|4|1973|297|3859|Friday|1973Q4|N|Y|N|2442018|2442351|2441666|2441939|N|N|N|N|N| +2442032|AAAAAAAAADDEFCAA|1973-12-15|887|3859|297|1973|6|12|15|4|1973|297|3859|Saturday|1973Q4|N|Y|N|2442018|2442351|2441667|2441940|N|N|N|N|N| +2442033|AAAAAAAABDDEFCAA|1973-12-16|887|3859|297|1973|0|12|16|4|1973|297|3859|Sunday|1973Q4|N|N|N|2442018|2442351|2441668|2441941|N|N|N|N|N| +2442034|AAAAAAAACDDEFCAA|1973-12-17|887|3859|297|1973|1|12|17|4|1973|297|3859|Monday|1973Q4|N|N|N|2442018|2442351|2441669|2441942|N|N|N|N|N| +2442035|AAAAAAAADDDEFCAA|1973-12-18|887|3860|297|1973|2|12|18|4|1973|297|3860|Tuesday|1973Q4|N|N|N|2442018|2442351|2441670|2441943|N|N|N|N|N| +2442036|AAAAAAAAEDDEFCAA|1973-12-19|887|3860|297|1973|3|12|19|4|1973|297|3860|Wednesday|1973Q4|N|N|N|2442018|2442351|2441671|2441944|N|N|N|N|N| +2442037|AAAAAAAAFDDEFCAA|1973-12-20|887|3860|297|1973|4|12|20|4|1973|297|3860|Thursday|1973Q4|N|N|N|2442018|2442351|2441672|2441945|N|N|N|N|N| +2442038|AAAAAAAAGDDEFCAA|1973-12-21|887|3860|297|1973|5|12|21|4|1973|297|3860|Friday|1973Q4|N|Y|N|2442018|2442351|2441673|2441946|N|N|N|N|N| +2442039|AAAAAAAAHDDEFCAA|1973-12-22|887|3860|297|1973|6|12|22|4|1973|297|3860|Saturday|1973Q4|N|Y|N|2442018|2442351|2441674|2441947|N|N|N|N|N| +2442040|AAAAAAAAIDDEFCAA|1973-12-23|887|3860|297|1973|0|12|23|4|1973|297|3860|Sunday|1973Q4|N|N|N|2442018|2442351|2441675|2441948|N|N|N|N|N| +2442041|AAAAAAAAJDDEFCAA|1973-12-24|887|3860|297|1973|1|12|24|4|1973|297|3860|Monday|1973Q4|N|N|N|2442018|2442351|2441676|2441949|N|N|N|N|N| +2442042|AAAAAAAAKDDEFCAA|1973-12-25|887|3861|297|1973|2|12|25|4|1973|297|3861|Tuesday|1973Q4|N|N|N|2442018|2442351|2441677|2441950|N|N|N|N|N| +2442043|AAAAAAAALDDEFCAA|1973-12-26|887|3861|297|1973|3|12|26|4|1973|297|3861|Wednesday|1973Q4|Y|N|N|2442018|2442351|2441678|2441951|N|N|N|N|N| +2442044|AAAAAAAAMDDEFCAA|1973-12-27|887|3861|297|1973|4|12|27|4|1973|297|3861|Thursday|1973Q4|N|N|Y|2442018|2442351|2441679|2441952|N|N|N|N|N| +2442045|AAAAAAAANDDEFCAA|1973-12-28|887|3861|297|1973|5|12|28|4|1973|297|3861|Friday|1973Q4|N|Y|N|2442018|2442351|2441680|2441953|N|N|N|N|N| +2442046|AAAAAAAAODDEFCAA|1973-12-29|887|3861|297|1973|6|12|29|4|1973|297|3861|Saturday|1973Q4|N|Y|N|2442018|2442351|2441681|2441954|N|N|N|N|N| +2442047|AAAAAAAAPDDEFCAA|1973-12-30|887|3861|297|1973|0|12|30|4|1973|297|3861|Sunday|1973Q4|N|N|N|2442018|2442351|2441682|2441955|N|N|N|N|N| +2442048|AAAAAAAAAEDEFCAA|1973-12-31|887|3861|297|1973|1|12|31|4|1973|297|3861|Monday|1973Q4|N|N|N|2442018|2442351|2441683|2441956|N|N|N|N|N| +2442049|AAAAAAAABEDEFCAA|1974-01-01|888|3862|297|1974|2|1|1|1|1974|297|3862|Tuesday|1974Q1|Y|N|N|2442049|2442048|2441684|2441957|N|N|N|N|N| +2442050|AAAAAAAACEDEFCAA|1974-01-02|888|3862|297|1974|3|1|2|1|1974|297|3862|Wednesday|1974Q1|N|N|Y|2442049|2442048|2441685|2441958|N|N|N|N|N| +2442051|AAAAAAAADEDEFCAA|1974-01-03|888|3862|297|1974|4|1|3|1|1974|297|3862|Thursday|1974Q1|N|N|N|2442049|2442048|2441686|2441959|N|N|N|N|N| +2442052|AAAAAAAAEEDEFCAA|1974-01-04|888|3862|297|1974|5|1|4|1|1974|297|3862|Friday|1974Q1|N|Y|N|2442049|2442048|2441687|2441960|N|N|N|N|N| +2442053|AAAAAAAAFEDEFCAA|1974-01-05|888|3862|297|1974|6|1|5|1|1974|297|3862|Saturday|1974Q1|N|Y|N|2442049|2442048|2441688|2441961|N|N|N|N|N| +2442054|AAAAAAAAGEDEFCAA|1974-01-06|888|3862|297|1974|0|1|6|1|1974|297|3862|Sunday|1974Q1|N|N|N|2442049|2442048|2441689|2441962|N|N|N|N|N| +2442055|AAAAAAAAHEDEFCAA|1974-01-07|888|3862|297|1974|1|1|7|1|1974|297|3862|Monday|1974Q1|N|N|N|2442049|2442048|2441690|2441963|N|N|N|N|N| +2442056|AAAAAAAAIEDEFCAA|1974-01-08|888|3863|297|1974|2|1|8|1|1974|297|3863|Tuesday|1974Q1|N|N|N|2442049|2442048|2441691|2441964|N|N|N|N|N| +2442057|AAAAAAAAJEDEFCAA|1974-01-09|888|3863|297|1974|3|1|9|1|1974|297|3863|Wednesday|1974Q1|N|N|N|2442049|2442048|2441692|2441965|N|N|N|N|N| +2442058|AAAAAAAAKEDEFCAA|1974-01-10|888|3863|297|1974|4|1|10|1|1974|297|3863|Thursday|1974Q1|N|N|N|2442049|2442048|2441693|2441966|N|N|N|N|N| +2442059|AAAAAAAALEDEFCAA|1974-01-11|888|3863|297|1974|5|1|11|1|1974|297|3863|Friday|1974Q1|N|Y|N|2442049|2442048|2441694|2441967|N|N|N|N|N| +2442060|AAAAAAAAMEDEFCAA|1974-01-12|888|3863|297|1974|6|1|12|1|1974|297|3863|Saturday|1974Q1|N|Y|N|2442049|2442048|2441695|2441968|N|N|N|N|N| +2442061|AAAAAAAANEDEFCAA|1974-01-13|888|3863|297|1974|0|1|13|1|1974|297|3863|Sunday|1974Q1|N|N|N|2442049|2442048|2441696|2441969|N|N|N|N|N| +2442062|AAAAAAAAOEDEFCAA|1974-01-14|888|3863|297|1974|1|1|14|1|1974|297|3863|Monday|1974Q1|N|N|N|2442049|2442048|2441697|2441970|N|N|N|N|N| +2442063|AAAAAAAAPEDEFCAA|1974-01-15|888|3864|297|1974|2|1|15|1|1974|297|3864|Tuesday|1974Q1|N|N|N|2442049|2442048|2441698|2441971|N|N|N|N|N| +2442064|AAAAAAAAAFDEFCAA|1974-01-16|888|3864|297|1974|3|1|16|1|1974|297|3864|Wednesday|1974Q1|N|N|N|2442049|2442048|2441699|2441972|N|N|N|N|N| +2442065|AAAAAAAABFDEFCAA|1974-01-17|888|3864|297|1974|4|1|17|1|1974|297|3864|Thursday|1974Q1|N|N|N|2442049|2442048|2441700|2441973|N|N|N|N|N| +2442066|AAAAAAAACFDEFCAA|1974-01-18|888|3864|297|1974|5|1|18|1|1974|297|3864|Friday|1974Q1|N|Y|N|2442049|2442048|2441701|2441974|N|N|N|N|N| +2442067|AAAAAAAADFDEFCAA|1974-01-19|888|3864|297|1974|6|1|19|1|1974|297|3864|Saturday|1974Q1|N|Y|N|2442049|2442048|2441702|2441975|N|N|N|N|N| +2442068|AAAAAAAAEFDEFCAA|1974-01-20|888|3864|297|1974|0|1|20|1|1974|297|3864|Sunday|1974Q1|N|N|N|2442049|2442048|2441703|2441976|N|N|N|N|N| +2442069|AAAAAAAAFFDEFCAA|1974-01-21|888|3864|297|1974|1|1|21|1|1974|297|3864|Monday|1974Q1|N|N|N|2442049|2442048|2441704|2441977|N|N|N|N|N| +2442070|AAAAAAAAGFDEFCAA|1974-01-22|888|3865|297|1974|2|1|22|1|1974|297|3865|Tuesday|1974Q1|N|N|N|2442049|2442048|2441705|2441978|N|N|N|N|N| +2442071|AAAAAAAAHFDEFCAA|1974-01-23|888|3865|297|1974|3|1|23|1|1974|297|3865|Wednesday|1974Q1|N|N|N|2442049|2442048|2441706|2441979|N|N|N|N|N| +2442072|AAAAAAAAIFDEFCAA|1974-01-24|888|3865|297|1974|4|1|24|1|1974|297|3865|Thursday|1974Q1|N|N|N|2442049|2442048|2441707|2441980|N|N|N|N|N| +2442073|AAAAAAAAJFDEFCAA|1974-01-25|888|3865|297|1974|5|1|25|1|1974|297|3865|Friday|1974Q1|N|Y|N|2442049|2442048|2441708|2441981|N|N|N|N|N| +2442074|AAAAAAAAKFDEFCAA|1974-01-26|888|3865|297|1974|6|1|26|1|1974|297|3865|Saturday|1974Q1|N|Y|N|2442049|2442048|2441709|2441982|N|N|N|N|N| +2442075|AAAAAAAALFDEFCAA|1974-01-27|888|3865|297|1974|0|1|27|1|1974|297|3865|Sunday|1974Q1|N|N|N|2442049|2442048|2441710|2441983|N|N|N|N|N| +2442076|AAAAAAAAMFDEFCAA|1974-01-28|888|3865|297|1974|1|1|28|1|1974|297|3865|Monday|1974Q1|N|N|N|2442049|2442048|2441711|2441984|N|N|N|N|N| +2442077|AAAAAAAANFDEFCAA|1974-01-29|888|3866|297|1974|2|1|29|1|1974|297|3866|Tuesday|1974Q1|N|N|N|2442049|2442048|2441712|2441985|N|N|N|N|N| +2442078|AAAAAAAAOFDEFCAA|1974-01-30|888|3866|297|1974|3|1|30|1|1974|297|3866|Wednesday|1974Q1|N|N|N|2442049|2442048|2441713|2441986|N|N|N|N|N| +2442079|AAAAAAAAPFDEFCAA|1974-01-31|888|3866|297|1974|4|1|31|1|1974|297|3866|Thursday|1974Q1|N|N|N|2442049|2442048|2441714|2441987|N|N|N|N|N| +2442080|AAAAAAAAAGDEFCAA|1974-02-01|889|3866|297|1974|5|2|1|1|1974|297|3866|Friday|1974Q1|N|Y|N|2442080|2442110|2441715|2441988|N|N|N|N|N| +2442081|AAAAAAAABGDEFCAA|1974-02-02|889|3866|297|1974|6|2|2|1|1974|297|3866|Saturday|1974Q1|N|Y|N|2442080|2442110|2441716|2441989|N|N|N|N|N| +2442082|AAAAAAAACGDEFCAA|1974-02-03|889|3866|297|1974|0|2|3|1|1974|297|3866|Sunday|1974Q1|N|N|N|2442080|2442110|2441717|2441990|N|N|N|N|N| +2442083|AAAAAAAADGDEFCAA|1974-02-04|889|3866|297|1974|1|2|4|1|1974|297|3866|Monday|1974Q1|N|N|N|2442080|2442110|2441718|2441991|N|N|N|N|N| +2442084|AAAAAAAAEGDEFCAA|1974-02-05|889|3867|297|1974|2|2|5|1|1974|297|3867|Tuesday|1974Q1|N|N|N|2442080|2442110|2441719|2441992|N|N|N|N|N| +2442085|AAAAAAAAFGDEFCAA|1974-02-06|889|3867|297|1974|3|2|6|1|1974|297|3867|Wednesday|1974Q1|N|N|N|2442080|2442110|2441720|2441993|N|N|N|N|N| +2442086|AAAAAAAAGGDEFCAA|1974-02-07|889|3867|297|1974|4|2|7|1|1974|297|3867|Thursday|1974Q1|N|N|N|2442080|2442110|2441721|2441994|N|N|N|N|N| +2442087|AAAAAAAAHGDEFCAA|1974-02-08|889|3867|297|1974|5|2|8|1|1974|297|3867|Friday|1974Q1|N|Y|N|2442080|2442110|2441722|2441995|N|N|N|N|N| +2442088|AAAAAAAAIGDEFCAA|1974-02-09|889|3867|297|1974|6|2|9|1|1974|297|3867|Saturday|1974Q1|N|Y|N|2442080|2442110|2441723|2441996|N|N|N|N|N| +2442089|AAAAAAAAJGDEFCAA|1974-02-10|889|3867|297|1974|0|2|10|1|1974|297|3867|Sunday|1974Q1|N|N|N|2442080|2442110|2441724|2441997|N|N|N|N|N| +2442090|AAAAAAAAKGDEFCAA|1974-02-11|889|3867|297|1974|1|2|11|1|1974|297|3867|Monday|1974Q1|N|N|N|2442080|2442110|2441725|2441998|N|N|N|N|N| +2442091|AAAAAAAALGDEFCAA|1974-02-12|889|3868|297|1974|2|2|12|1|1974|297|3868|Tuesday|1974Q1|N|N|N|2442080|2442110|2441726|2441999|N|N|N|N|N| +2442092|AAAAAAAAMGDEFCAA|1974-02-13|889|3868|297|1974|3|2|13|1|1974|297|3868|Wednesday|1974Q1|N|N|N|2442080|2442110|2441727|2442000|N|N|N|N|N| +2442093|AAAAAAAANGDEFCAA|1974-02-14|889|3868|297|1974|4|2|14|1|1974|297|3868|Thursday|1974Q1|N|N|N|2442080|2442110|2441728|2442001|N|N|N|N|N| +2442094|AAAAAAAAOGDEFCAA|1974-02-15|889|3868|297|1974|5|2|15|1|1974|297|3868|Friday|1974Q1|N|Y|N|2442080|2442110|2441729|2442002|N|N|N|N|N| +2442095|AAAAAAAAPGDEFCAA|1974-02-16|889|3868|297|1974|6|2|16|1|1974|297|3868|Saturday|1974Q1|N|Y|N|2442080|2442110|2441730|2442003|N|N|N|N|N| +2442096|AAAAAAAAAHDEFCAA|1974-02-17|889|3868|297|1974|0|2|17|1|1974|297|3868|Sunday|1974Q1|N|N|N|2442080|2442110|2441731|2442004|N|N|N|N|N| +2442097|AAAAAAAABHDEFCAA|1974-02-18|889|3868|297|1974|1|2|18|1|1974|297|3868|Monday|1974Q1|N|N|N|2442080|2442110|2441732|2442005|N|N|N|N|N| +2442098|AAAAAAAACHDEFCAA|1974-02-19|889|3869|297|1974|2|2|19|1|1974|297|3869|Tuesday|1974Q1|N|N|N|2442080|2442110|2441733|2442006|N|N|N|N|N| +2442099|AAAAAAAADHDEFCAA|1974-02-20|889|3869|297|1974|3|2|20|1|1974|297|3869|Wednesday|1974Q1|N|N|N|2442080|2442110|2441734|2442007|N|N|N|N|N| +2442100|AAAAAAAAEHDEFCAA|1974-02-21|889|3869|297|1974|4|2|21|1|1974|297|3869|Thursday|1974Q1|N|N|N|2442080|2442110|2441735|2442008|N|N|N|N|N| +2442101|AAAAAAAAFHDEFCAA|1974-02-22|889|3869|297|1974|5|2|22|1|1974|297|3869|Friday|1974Q1|N|Y|N|2442080|2442110|2441736|2442009|N|N|N|N|N| +2442102|AAAAAAAAGHDEFCAA|1974-02-23|889|3869|297|1974|6|2|23|1|1974|297|3869|Saturday|1974Q1|N|Y|N|2442080|2442110|2441737|2442010|N|N|N|N|N| +2442103|AAAAAAAAHHDEFCAA|1974-02-24|889|3869|297|1974|0|2|24|1|1974|297|3869|Sunday|1974Q1|N|N|N|2442080|2442110|2441738|2442011|N|N|N|N|N| +2442104|AAAAAAAAIHDEFCAA|1974-02-25|889|3869|297|1974|1|2|25|1|1974|297|3869|Monday|1974Q1|N|N|N|2442080|2442110|2441739|2442012|N|N|N|N|N| +2442105|AAAAAAAAJHDEFCAA|1974-02-26|889|3870|297|1974|2|2|26|1|1974|297|3870|Tuesday|1974Q1|N|N|N|2442080|2442110|2441740|2442013|N|N|N|N|N| +2442106|AAAAAAAAKHDEFCAA|1974-02-27|889|3870|297|1974|3|2|27|1|1974|297|3870|Wednesday|1974Q1|N|N|N|2442080|2442110|2441741|2442014|N|N|N|N|N| +2442107|AAAAAAAALHDEFCAA|1974-02-28|889|3870|297|1974|4|2|28|1|1974|297|3870|Thursday|1974Q1|N|N|N|2442080|2442110|2441742|2442015|N|N|N|N|N| +2442108|AAAAAAAAMHDEFCAA|1974-03-01|890|3870|298|1974|5|3|1|1|1974|298|3870|Friday|1974Q1|N|Y|N|2442108|2442166|2441743|2442016|N|N|N|N|N| +2442109|AAAAAAAANHDEFCAA|1974-03-02|890|3870|298|1974|6|3|2|1|1974|298|3870|Saturday|1974Q1|N|Y|N|2442108|2442166|2441744|2442017|N|N|N|N|N| +2442110|AAAAAAAAOHDEFCAA|1974-03-03|890|3870|298|1974|0|3|3|1|1974|298|3870|Sunday|1974Q1|N|N|N|2442108|2442166|2441745|2442018|N|N|N|N|N| +2442111|AAAAAAAAPHDEFCAA|1974-03-04|890|3870|298|1974|1|3|4|1|1974|298|3870|Monday|1974Q1|N|N|N|2442108|2442166|2441746|2442019|N|N|N|N|N| +2442112|AAAAAAAAAIDEFCAA|1974-03-05|890|3871|298|1974|2|3|5|1|1974|298|3871|Tuesday|1974Q1|N|N|N|2442108|2442166|2441747|2442020|N|N|N|N|N| +2442113|AAAAAAAABIDEFCAA|1974-03-06|890|3871|298|1974|3|3|6|1|1974|298|3871|Wednesday|1974Q1|N|N|N|2442108|2442166|2441748|2442021|N|N|N|N|N| +2442114|AAAAAAAACIDEFCAA|1974-03-07|890|3871|298|1974|4|3|7|1|1974|298|3871|Thursday|1974Q1|N|N|N|2442108|2442166|2441749|2442022|N|N|N|N|N| +2442115|AAAAAAAADIDEFCAA|1974-03-08|890|3871|298|1974|5|3|8|1|1974|298|3871|Friday|1974Q1|N|Y|N|2442108|2442166|2441750|2442023|N|N|N|N|N| +2442116|AAAAAAAAEIDEFCAA|1974-03-09|890|3871|298|1974|6|3|9|1|1974|298|3871|Saturday|1974Q1|N|Y|N|2442108|2442166|2441751|2442024|N|N|N|N|N| +2442117|AAAAAAAAFIDEFCAA|1974-03-10|890|3871|298|1974|0|3|10|1|1974|298|3871|Sunday|1974Q1|N|N|N|2442108|2442166|2441752|2442025|N|N|N|N|N| +2442118|AAAAAAAAGIDEFCAA|1974-03-11|890|3871|298|1974|1|3|11|1|1974|298|3871|Monday|1974Q1|N|N|N|2442108|2442166|2441753|2442026|N|N|N|N|N| +2442119|AAAAAAAAHIDEFCAA|1974-03-12|890|3872|298|1974|2|3|12|1|1974|298|3872|Tuesday|1974Q1|N|N|N|2442108|2442166|2441754|2442027|N|N|N|N|N| +2442120|AAAAAAAAIIDEFCAA|1974-03-13|890|3872|298|1974|3|3|13|1|1974|298|3872|Wednesday|1974Q1|N|N|N|2442108|2442166|2441755|2442028|N|N|N|N|N| +2442121|AAAAAAAAJIDEFCAA|1974-03-14|890|3872|298|1974|4|3|14|1|1974|298|3872|Thursday|1974Q1|N|N|N|2442108|2442166|2441756|2442029|N|N|N|N|N| +2442122|AAAAAAAAKIDEFCAA|1974-03-15|890|3872|298|1974|5|3|15|1|1974|298|3872|Friday|1974Q1|N|Y|N|2442108|2442166|2441757|2442030|N|N|N|N|N| +2442123|AAAAAAAALIDEFCAA|1974-03-16|890|3872|298|1974|6|3|16|1|1974|298|3872|Saturday|1974Q1|N|Y|N|2442108|2442166|2441758|2442031|N|N|N|N|N| +2442124|AAAAAAAAMIDEFCAA|1974-03-17|890|3872|298|1974|0|3|17|1|1974|298|3872|Sunday|1974Q1|N|N|N|2442108|2442166|2441759|2442032|N|N|N|N|N| +2442125|AAAAAAAANIDEFCAA|1974-03-18|890|3872|298|1974|1|3|18|1|1974|298|3872|Monday|1974Q1|N|N|N|2442108|2442166|2441760|2442033|N|N|N|N|N| +2442126|AAAAAAAAOIDEFCAA|1974-03-19|890|3873|298|1974|2|3|19|1|1974|298|3873|Tuesday|1974Q1|N|N|N|2442108|2442166|2441761|2442034|N|N|N|N|N| +2442127|AAAAAAAAPIDEFCAA|1974-03-20|890|3873|298|1974|3|3|20|1|1974|298|3873|Wednesday|1974Q1|N|N|N|2442108|2442166|2441762|2442035|N|N|N|N|N| +2442128|AAAAAAAAAJDEFCAA|1974-03-21|890|3873|298|1974|4|3|21|1|1974|298|3873|Thursday|1974Q1|N|N|N|2442108|2442166|2441763|2442036|N|N|N|N|N| +2442129|AAAAAAAABJDEFCAA|1974-03-22|890|3873|298|1974|5|3|22|1|1974|298|3873|Friday|1974Q1|N|Y|N|2442108|2442166|2441764|2442037|N|N|N|N|N| +2442130|AAAAAAAACJDEFCAA|1974-03-23|890|3873|298|1974|6|3|23|1|1974|298|3873|Saturday|1974Q1|N|Y|N|2442108|2442166|2441765|2442038|N|N|N|N|N| +2442131|AAAAAAAADJDEFCAA|1974-03-24|890|3873|298|1974|0|3|24|1|1974|298|3873|Sunday|1974Q1|N|N|N|2442108|2442166|2441766|2442039|N|N|N|N|N| +2442132|AAAAAAAAEJDEFCAA|1974-03-25|890|3873|298|1974|1|3|25|1|1974|298|3873|Monday|1974Q1|N|N|N|2442108|2442166|2441767|2442040|N|N|N|N|N| +2442133|AAAAAAAAFJDEFCAA|1974-03-26|890|3874|298|1974|2|3|26|1|1974|298|3874|Tuesday|1974Q1|N|N|N|2442108|2442166|2441768|2442041|N|N|N|N|N| +2442134|AAAAAAAAGJDEFCAA|1974-03-27|890|3874|298|1974|3|3|27|1|1974|298|3874|Wednesday|1974Q1|N|N|N|2442108|2442166|2441769|2442042|N|N|N|N|N| +2442135|AAAAAAAAHJDEFCAA|1974-03-28|890|3874|298|1974|4|3|28|1|1974|298|3874|Thursday|1974Q1|N|N|N|2442108|2442166|2441770|2442043|N|N|N|N|N| +2442136|AAAAAAAAIJDEFCAA|1974-03-29|890|3874|298|1974|5|3|29|1|1974|298|3874|Friday|1974Q1|N|Y|N|2442108|2442166|2441771|2442044|N|N|N|N|N| +2442137|AAAAAAAAJJDEFCAA|1974-03-30|890|3874|298|1974|6|3|30|1|1974|298|3874|Saturday|1974Q1|N|Y|N|2442108|2442166|2441772|2442045|N|N|N|N|N| +2442138|AAAAAAAAKJDEFCAA|1974-03-31|890|3874|298|1974|0|3|31|1|1974|298|3874|Sunday|1974Q1|N|N|N|2442108|2442166|2441773|2442046|N|N|N|N|N| +2442139|AAAAAAAALJDEFCAA|1974-04-01|891|3874|298|1974|1|4|1|1|1974|298|3874|Monday|1974Q1|N|N|N|2442139|2442228|2441774|2442049|N|N|N|N|N| +2442140|AAAAAAAAMJDEFCAA|1974-04-02|891|3875|298|1974|2|4|2|2|1974|298|3875|Tuesday|1974Q2|N|N|N|2442139|2442228|2441775|2442050|N|N|N|N|N| +2442141|AAAAAAAANJDEFCAA|1974-04-03|891|3875|298|1974|3|4|3|2|1974|298|3875|Wednesday|1974Q2|N|N|N|2442139|2442228|2441776|2442051|N|N|N|N|N| +2442142|AAAAAAAAOJDEFCAA|1974-04-04|891|3875|298|1974|4|4|4|2|1974|298|3875|Thursday|1974Q2|N|N|N|2442139|2442228|2441777|2442052|N|N|N|N|N| +2442143|AAAAAAAAPJDEFCAA|1974-04-05|891|3875|298|1974|5|4|5|2|1974|298|3875|Friday|1974Q2|N|Y|N|2442139|2442228|2441778|2442053|N|N|N|N|N| +2442144|AAAAAAAAAKDEFCAA|1974-04-06|891|3875|298|1974|6|4|6|2|1974|298|3875|Saturday|1974Q2|N|Y|N|2442139|2442228|2441779|2442054|N|N|N|N|N| +2442145|AAAAAAAABKDEFCAA|1974-04-07|891|3875|298|1974|0|4|7|2|1974|298|3875|Sunday|1974Q2|N|N|N|2442139|2442228|2441780|2442055|N|N|N|N|N| +2442146|AAAAAAAACKDEFCAA|1974-04-08|891|3875|298|1974|1|4|8|2|1974|298|3875|Monday|1974Q2|N|N|N|2442139|2442228|2441781|2442056|N|N|N|N|N| +2442147|AAAAAAAADKDEFCAA|1974-04-09|891|3876|298|1974|2|4|9|2|1974|298|3876|Tuesday|1974Q2|N|N|N|2442139|2442228|2441782|2442057|N|N|N|N|N| +2442148|AAAAAAAAEKDEFCAA|1974-04-10|891|3876|298|1974|3|4|10|2|1974|298|3876|Wednesday|1974Q2|N|N|N|2442139|2442228|2441783|2442058|N|N|N|N|N| +2442149|AAAAAAAAFKDEFCAA|1974-04-11|891|3876|298|1974|4|4|11|2|1974|298|3876|Thursday|1974Q2|N|N|N|2442139|2442228|2441784|2442059|N|N|N|N|N| +2442150|AAAAAAAAGKDEFCAA|1974-04-12|891|3876|298|1974|5|4|12|2|1974|298|3876|Friday|1974Q2|N|Y|N|2442139|2442228|2441785|2442060|N|N|N|N|N| +2442151|AAAAAAAAHKDEFCAA|1974-04-13|891|3876|298|1974|6|4|13|2|1974|298|3876|Saturday|1974Q2|N|Y|N|2442139|2442228|2441786|2442061|N|N|N|N|N| +2442152|AAAAAAAAIKDEFCAA|1974-04-14|891|3876|298|1974|0|4|14|2|1974|298|3876|Sunday|1974Q2|N|N|N|2442139|2442228|2441787|2442062|N|N|N|N|N| +2442153|AAAAAAAAJKDEFCAA|1974-04-15|891|3876|298|1974|1|4|15|2|1974|298|3876|Monday|1974Q2|N|N|N|2442139|2442228|2441788|2442063|N|N|N|N|N| +2442154|AAAAAAAAKKDEFCAA|1974-04-16|891|3877|298|1974|2|4|16|2|1974|298|3877|Tuesday|1974Q2|N|N|N|2442139|2442228|2441789|2442064|N|N|N|N|N| +2442155|AAAAAAAALKDEFCAA|1974-04-17|891|3877|298|1974|3|4|17|2|1974|298|3877|Wednesday|1974Q2|N|N|N|2442139|2442228|2441790|2442065|N|N|N|N|N| +2442156|AAAAAAAAMKDEFCAA|1974-04-18|891|3877|298|1974|4|4|18|2|1974|298|3877|Thursday|1974Q2|N|N|N|2442139|2442228|2441791|2442066|N|N|N|N|N| +2442157|AAAAAAAANKDEFCAA|1974-04-19|891|3877|298|1974|5|4|19|2|1974|298|3877|Friday|1974Q2|N|Y|N|2442139|2442228|2441792|2442067|N|N|N|N|N| +2442158|AAAAAAAAOKDEFCAA|1974-04-20|891|3877|298|1974|6|4|20|2|1974|298|3877|Saturday|1974Q2|N|Y|N|2442139|2442228|2441793|2442068|N|N|N|N|N| +2442159|AAAAAAAAPKDEFCAA|1974-04-21|891|3877|298|1974|0|4|21|2|1974|298|3877|Sunday|1974Q2|N|N|N|2442139|2442228|2441794|2442069|N|N|N|N|N| +2442160|AAAAAAAAALDEFCAA|1974-04-22|891|3877|298|1974|1|4|22|2|1974|298|3877|Monday|1974Q2|N|N|N|2442139|2442228|2441795|2442070|N|N|N|N|N| +2442161|AAAAAAAABLDEFCAA|1974-04-23|891|3878|298|1974|2|4|23|2|1974|298|3878|Tuesday|1974Q2|N|N|N|2442139|2442228|2441796|2442071|N|N|N|N|N| +2442162|AAAAAAAACLDEFCAA|1974-04-24|891|3878|298|1974|3|4|24|2|1974|298|3878|Wednesday|1974Q2|N|N|N|2442139|2442228|2441797|2442072|N|N|N|N|N| +2442163|AAAAAAAADLDEFCAA|1974-04-25|891|3878|298|1974|4|4|25|2|1974|298|3878|Thursday|1974Q2|N|N|N|2442139|2442228|2441798|2442073|N|N|N|N|N| +2442164|AAAAAAAAELDEFCAA|1974-04-26|891|3878|298|1974|5|4|26|2|1974|298|3878|Friday|1974Q2|N|Y|N|2442139|2442228|2441799|2442074|N|N|N|N|N| +2442165|AAAAAAAAFLDEFCAA|1974-04-27|891|3878|298|1974|6|4|27|2|1974|298|3878|Saturday|1974Q2|N|Y|N|2442139|2442228|2441800|2442075|N|N|N|N|N| +2442166|AAAAAAAAGLDEFCAA|1974-04-28|891|3878|298|1974|0|4|28|2|1974|298|3878|Sunday|1974Q2|N|N|N|2442139|2442228|2441801|2442076|N|N|N|N|N| +2442167|AAAAAAAAHLDEFCAA|1974-04-29|891|3878|298|1974|1|4|29|2|1974|298|3878|Monday|1974Q2|N|N|N|2442139|2442228|2441802|2442077|N|N|N|N|N| +2442168|AAAAAAAAILDEFCAA|1974-04-30|891|3879|298|1974|2|4|30|2|1974|298|3879|Tuesday|1974Q2|N|N|N|2442139|2442228|2441803|2442078|N|N|N|N|N| +2442169|AAAAAAAAJLDEFCAA|1974-05-01|892|3879|298|1974|3|5|1|2|1974|298|3879|Wednesday|1974Q2|N|N|N|2442169|2442288|2441804|2442079|N|N|N|N|N| +2442170|AAAAAAAAKLDEFCAA|1974-05-02|892|3879|298|1974|4|5|2|2|1974|298|3879|Thursday|1974Q2|N|N|N|2442169|2442288|2441805|2442080|N|N|N|N|N| +2442171|AAAAAAAALLDEFCAA|1974-05-03|892|3879|298|1974|5|5|3|2|1974|298|3879|Friday|1974Q2|N|Y|N|2442169|2442288|2441806|2442081|N|N|N|N|N| +2442172|AAAAAAAAMLDEFCAA|1974-05-04|892|3879|298|1974|6|5|4|2|1974|298|3879|Saturday|1974Q2|N|Y|N|2442169|2442288|2441807|2442082|N|N|N|N|N| +2442173|AAAAAAAANLDEFCAA|1974-05-05|892|3879|298|1974|0|5|5|2|1974|298|3879|Sunday|1974Q2|N|N|N|2442169|2442288|2441808|2442083|N|N|N|N|N| +2442174|AAAAAAAAOLDEFCAA|1974-05-06|892|3879|298|1974|1|5|6|2|1974|298|3879|Monday|1974Q2|N|N|N|2442169|2442288|2441809|2442084|N|N|N|N|N| +2442175|AAAAAAAAPLDEFCAA|1974-05-07|892|3880|298|1974|2|5|7|2|1974|298|3880|Tuesday|1974Q2|N|N|N|2442169|2442288|2441810|2442085|N|N|N|N|N| +2442176|AAAAAAAAAMDEFCAA|1974-05-08|892|3880|298|1974|3|5|8|2|1974|298|3880|Wednesday|1974Q2|N|N|N|2442169|2442288|2441811|2442086|N|N|N|N|N| +2442177|AAAAAAAABMDEFCAA|1974-05-09|892|3880|298|1974|4|5|9|2|1974|298|3880|Thursday|1974Q2|N|N|N|2442169|2442288|2441812|2442087|N|N|N|N|N| +2442178|AAAAAAAACMDEFCAA|1974-05-10|892|3880|298|1974|5|5|10|2|1974|298|3880|Friday|1974Q2|N|Y|N|2442169|2442288|2441813|2442088|N|N|N|N|N| +2442179|AAAAAAAADMDEFCAA|1974-05-11|892|3880|298|1974|6|5|11|2|1974|298|3880|Saturday|1974Q2|N|Y|N|2442169|2442288|2441814|2442089|N|N|N|N|N| +2442180|AAAAAAAAEMDEFCAA|1974-05-12|892|3880|298|1974|0|5|12|2|1974|298|3880|Sunday|1974Q2|N|N|N|2442169|2442288|2441815|2442090|N|N|N|N|N| +2442181|AAAAAAAAFMDEFCAA|1974-05-13|892|3880|298|1974|1|5|13|2|1974|298|3880|Monday|1974Q2|N|N|N|2442169|2442288|2441816|2442091|N|N|N|N|N| +2442182|AAAAAAAAGMDEFCAA|1974-05-14|892|3881|298|1974|2|5|14|2|1974|298|3881|Tuesday|1974Q2|N|N|N|2442169|2442288|2441817|2442092|N|N|N|N|N| +2442183|AAAAAAAAHMDEFCAA|1974-05-15|892|3881|298|1974|3|5|15|2|1974|298|3881|Wednesday|1974Q2|N|N|N|2442169|2442288|2441818|2442093|N|N|N|N|N| +2442184|AAAAAAAAIMDEFCAA|1974-05-16|892|3881|298|1974|4|5|16|2|1974|298|3881|Thursday|1974Q2|N|N|N|2442169|2442288|2441819|2442094|N|N|N|N|N| +2442185|AAAAAAAAJMDEFCAA|1974-05-17|892|3881|298|1974|5|5|17|2|1974|298|3881|Friday|1974Q2|N|Y|N|2442169|2442288|2441820|2442095|N|N|N|N|N| +2442186|AAAAAAAAKMDEFCAA|1974-05-18|892|3881|298|1974|6|5|18|2|1974|298|3881|Saturday|1974Q2|N|Y|N|2442169|2442288|2441821|2442096|N|N|N|N|N| +2442187|AAAAAAAALMDEFCAA|1974-05-19|892|3881|298|1974|0|5|19|2|1974|298|3881|Sunday|1974Q2|N|N|N|2442169|2442288|2441822|2442097|N|N|N|N|N| +2442188|AAAAAAAAMMDEFCAA|1974-05-20|892|3881|298|1974|1|5|20|2|1974|298|3881|Monday|1974Q2|N|N|N|2442169|2442288|2441823|2442098|N|N|N|N|N| +2442189|AAAAAAAANMDEFCAA|1974-05-21|892|3882|298|1974|2|5|21|2|1974|298|3882|Tuesday|1974Q2|N|N|N|2442169|2442288|2441824|2442099|N|N|N|N|N| +2442190|AAAAAAAAOMDEFCAA|1974-05-22|892|3882|298|1974|3|5|22|2|1974|298|3882|Wednesday|1974Q2|N|N|N|2442169|2442288|2441825|2442100|N|N|N|N|N| +2442191|AAAAAAAAPMDEFCAA|1974-05-23|892|3882|298|1974|4|5|23|2|1974|298|3882|Thursday|1974Q2|N|N|N|2442169|2442288|2441826|2442101|N|N|N|N|N| +2442192|AAAAAAAAANDEFCAA|1974-05-24|892|3882|298|1974|5|5|24|2|1974|298|3882|Friday|1974Q2|N|Y|N|2442169|2442288|2441827|2442102|N|N|N|N|N| +2442193|AAAAAAAABNDEFCAA|1974-05-25|892|3882|298|1974|6|5|25|2|1974|298|3882|Saturday|1974Q2|N|Y|N|2442169|2442288|2441828|2442103|N|N|N|N|N| +2442194|AAAAAAAACNDEFCAA|1974-05-26|892|3882|298|1974|0|5|26|2|1974|298|3882|Sunday|1974Q2|N|N|N|2442169|2442288|2441829|2442104|N|N|N|N|N| +2442195|AAAAAAAADNDEFCAA|1974-05-27|892|3882|298|1974|1|5|27|2|1974|298|3882|Monday|1974Q2|N|N|N|2442169|2442288|2441830|2442105|N|N|N|N|N| +2442196|AAAAAAAAENDEFCAA|1974-05-28|892|3883|298|1974|2|5|28|2|1974|298|3883|Tuesday|1974Q2|N|N|N|2442169|2442288|2441831|2442106|N|N|N|N|N| +2442197|AAAAAAAAFNDEFCAA|1974-05-29|892|3883|298|1974|3|5|29|2|1974|298|3883|Wednesday|1974Q2|N|N|N|2442169|2442288|2441832|2442107|N|N|N|N|N| +2442198|AAAAAAAAGNDEFCAA|1974-05-30|892|3883|298|1974|4|5|30|2|1974|298|3883|Thursday|1974Q2|N|N|N|2442169|2442288|2441833|2442108|N|N|N|N|N| +2442199|AAAAAAAAHNDEFCAA|1974-05-31|892|3883|298|1974|5|5|31|2|1974|298|3883|Friday|1974Q2|N|Y|N|2442169|2442288|2441834|2442109|N|N|N|N|N| +2442200|AAAAAAAAINDEFCAA|1974-06-01|893|3883|299|1974|6|6|1|2|1974|299|3883|Saturday|1974Q2|N|Y|N|2442200|2442350|2441835|2442110|N|N|N|N|N| +2442201|AAAAAAAAJNDEFCAA|1974-06-02|893|3883|299|1974|0|6|2|2|1974|299|3883|Sunday|1974Q2|N|N|N|2442200|2442350|2441836|2442111|N|N|N|N|N| +2442202|AAAAAAAAKNDEFCAA|1974-06-03|893|3883|299|1974|1|6|3|2|1974|299|3883|Monday|1974Q2|N|N|N|2442200|2442350|2441837|2442112|N|N|N|N|N| +2442203|AAAAAAAALNDEFCAA|1974-06-04|893|3884|299|1974|2|6|4|2|1974|299|3884|Tuesday|1974Q2|N|N|N|2442200|2442350|2441838|2442113|N|N|N|N|N| +2442204|AAAAAAAAMNDEFCAA|1974-06-05|893|3884|299|1974|3|6|5|2|1974|299|3884|Wednesday|1974Q2|N|N|N|2442200|2442350|2441839|2442114|N|N|N|N|N| +2442205|AAAAAAAANNDEFCAA|1974-06-06|893|3884|299|1974|4|6|6|2|1974|299|3884|Thursday|1974Q2|N|N|N|2442200|2442350|2441840|2442115|N|N|N|N|N| +2442206|AAAAAAAAONDEFCAA|1974-06-07|893|3884|299|1974|5|6|7|2|1974|299|3884|Friday|1974Q2|N|Y|N|2442200|2442350|2441841|2442116|N|N|N|N|N| +2442207|AAAAAAAAPNDEFCAA|1974-06-08|893|3884|299|1974|6|6|8|2|1974|299|3884|Saturday|1974Q2|N|Y|N|2442200|2442350|2441842|2442117|N|N|N|N|N| +2442208|AAAAAAAAAODEFCAA|1974-06-09|893|3884|299|1974|0|6|9|2|1974|299|3884|Sunday|1974Q2|N|N|N|2442200|2442350|2441843|2442118|N|N|N|N|N| +2442209|AAAAAAAABODEFCAA|1974-06-10|893|3884|299|1974|1|6|10|2|1974|299|3884|Monday|1974Q2|N|N|N|2442200|2442350|2441844|2442119|N|N|N|N|N| +2442210|AAAAAAAACODEFCAA|1974-06-11|893|3885|299|1974|2|6|11|2|1974|299|3885|Tuesday|1974Q2|N|N|N|2442200|2442350|2441845|2442120|N|N|N|N|N| +2442211|AAAAAAAADODEFCAA|1974-06-12|893|3885|299|1974|3|6|12|2|1974|299|3885|Wednesday|1974Q2|N|N|N|2442200|2442350|2441846|2442121|N|N|N|N|N| +2442212|AAAAAAAAEODEFCAA|1974-06-13|893|3885|299|1974|4|6|13|2|1974|299|3885|Thursday|1974Q2|N|N|N|2442200|2442350|2441847|2442122|N|N|N|N|N| +2442213|AAAAAAAAFODEFCAA|1974-06-14|893|3885|299|1974|5|6|14|2|1974|299|3885|Friday|1974Q2|N|Y|N|2442200|2442350|2441848|2442123|N|N|N|N|N| +2442214|AAAAAAAAGODEFCAA|1974-06-15|893|3885|299|1974|6|6|15|2|1974|299|3885|Saturday|1974Q2|N|Y|N|2442200|2442350|2441849|2442124|N|N|N|N|N| +2442215|AAAAAAAAHODEFCAA|1974-06-16|893|3885|299|1974|0|6|16|2|1974|299|3885|Sunday|1974Q2|N|N|N|2442200|2442350|2441850|2442125|N|N|N|N|N| +2442216|AAAAAAAAIODEFCAA|1974-06-17|893|3885|299|1974|1|6|17|2|1974|299|3885|Monday|1974Q2|N|N|N|2442200|2442350|2441851|2442126|N|N|N|N|N| +2442217|AAAAAAAAJODEFCAA|1974-06-18|893|3886|299|1974|2|6|18|2|1974|299|3886|Tuesday|1974Q2|N|N|N|2442200|2442350|2441852|2442127|N|N|N|N|N| +2442218|AAAAAAAAKODEFCAA|1974-06-19|893|3886|299|1974|3|6|19|2|1974|299|3886|Wednesday|1974Q2|N|N|N|2442200|2442350|2441853|2442128|N|N|N|N|N| +2442219|AAAAAAAALODEFCAA|1974-06-20|893|3886|299|1974|4|6|20|2|1974|299|3886|Thursday|1974Q2|N|N|N|2442200|2442350|2441854|2442129|N|N|N|N|N| +2442220|AAAAAAAAMODEFCAA|1974-06-21|893|3886|299|1974|5|6|21|2|1974|299|3886|Friday|1974Q2|N|Y|N|2442200|2442350|2441855|2442130|N|N|N|N|N| +2442221|AAAAAAAANODEFCAA|1974-06-22|893|3886|299|1974|6|6|22|2|1974|299|3886|Saturday|1974Q2|N|Y|N|2442200|2442350|2441856|2442131|N|N|N|N|N| +2442222|AAAAAAAAOODEFCAA|1974-06-23|893|3886|299|1974|0|6|23|2|1974|299|3886|Sunday|1974Q2|N|N|N|2442200|2442350|2441857|2442132|N|N|N|N|N| +2442223|AAAAAAAAPODEFCAA|1974-06-24|893|3886|299|1974|1|6|24|2|1974|299|3886|Monday|1974Q2|N|N|N|2442200|2442350|2441858|2442133|N|N|N|N|N| +2442224|AAAAAAAAAPDEFCAA|1974-06-25|893|3887|299|1974|2|6|25|2|1974|299|3887|Tuesday|1974Q2|N|N|N|2442200|2442350|2441859|2442134|N|N|N|N|N| +2442225|AAAAAAAABPDEFCAA|1974-06-26|893|3887|299|1974|3|6|26|2|1974|299|3887|Wednesday|1974Q2|N|N|N|2442200|2442350|2441860|2442135|N|N|N|N|N| +2442226|AAAAAAAACPDEFCAA|1974-06-27|893|3887|299|1974|4|6|27|2|1974|299|3887|Thursday|1974Q2|N|N|N|2442200|2442350|2441861|2442136|N|N|N|N|N| +2442227|AAAAAAAADPDEFCAA|1974-06-28|893|3887|299|1974|5|6|28|2|1974|299|3887|Friday|1974Q2|N|Y|N|2442200|2442350|2441862|2442137|N|N|N|N|N| +2442228|AAAAAAAAEPDEFCAA|1974-06-29|893|3887|299|1974|6|6|29|2|1974|299|3887|Saturday|1974Q2|N|Y|N|2442200|2442350|2441863|2442138|N|N|N|N|N| +2442229|AAAAAAAAFPDEFCAA|1974-06-30|893|3887|299|1974|0|6|30|2|1974|299|3887|Sunday|1974Q2|N|N|N|2442200|2442350|2441864|2442139|N|N|N|N|N| +2442230|AAAAAAAAGPDEFCAA|1974-07-01|894|3887|299|1974|1|7|1|2|1974|299|3887|Monday|1974Q2|N|N|N|2442230|2442410|2441865|2442139|N|N|N|N|N| +2442231|AAAAAAAAHPDEFCAA|1974-07-02|894|3888|299|1974|2|7|2|3|1974|299|3888|Tuesday|1974Q3|N|N|N|2442230|2442410|2441866|2442140|N|N|N|N|N| +2442232|AAAAAAAAIPDEFCAA|1974-07-03|894|3888|299|1974|3|7|3|3|1974|299|3888|Wednesday|1974Q3|N|N|N|2442230|2442410|2441867|2442141|N|N|N|N|N| +2442233|AAAAAAAAJPDEFCAA|1974-07-04|894|3888|299|1974|4|7|4|3|1974|299|3888|Thursday|1974Q3|N|N|N|2442230|2442410|2441868|2442142|N|N|N|N|N| +2442234|AAAAAAAAKPDEFCAA|1974-07-05|894|3888|299|1974|5|7|5|3|1974|299|3888|Friday|1974Q3|Y|Y|N|2442230|2442410|2441869|2442143|N|N|N|N|N| +2442235|AAAAAAAALPDEFCAA|1974-07-06|894|3888|299|1974|6|7|6|3|1974|299|3888|Saturday|1974Q3|N|Y|Y|2442230|2442410|2441870|2442144|N|N|N|N|N| +2442236|AAAAAAAAMPDEFCAA|1974-07-07|894|3888|299|1974|0|7|7|3|1974|299|3888|Sunday|1974Q3|N|N|N|2442230|2442410|2441871|2442145|N|N|N|N|N| +2442237|AAAAAAAANPDEFCAA|1974-07-08|894|3888|299|1974|1|7|8|3|1974|299|3888|Monday|1974Q3|N|N|N|2442230|2442410|2441872|2442146|N|N|N|N|N| +2442238|AAAAAAAAOPDEFCAA|1974-07-09|894|3889|299|1974|2|7|9|3|1974|299|3889|Tuesday|1974Q3|N|N|N|2442230|2442410|2441873|2442147|N|N|N|N|N| +2442239|AAAAAAAAPPDEFCAA|1974-07-10|894|3889|299|1974|3|7|10|3|1974|299|3889|Wednesday|1974Q3|N|N|N|2442230|2442410|2441874|2442148|N|N|N|N|N| +2442240|AAAAAAAAAAEEFCAA|1974-07-11|894|3889|299|1974|4|7|11|3|1974|299|3889|Thursday|1974Q3|N|N|N|2442230|2442410|2441875|2442149|N|N|N|N|N| +2442241|AAAAAAAABAEEFCAA|1974-07-12|894|3889|299|1974|5|7|12|3|1974|299|3889|Friday|1974Q3|N|Y|N|2442230|2442410|2441876|2442150|N|N|N|N|N| +2442242|AAAAAAAACAEEFCAA|1974-07-13|894|3889|299|1974|6|7|13|3|1974|299|3889|Saturday|1974Q3|N|Y|N|2442230|2442410|2441877|2442151|N|N|N|N|N| +2442243|AAAAAAAADAEEFCAA|1974-07-14|894|3889|299|1974|0|7|14|3|1974|299|3889|Sunday|1974Q3|N|N|N|2442230|2442410|2441878|2442152|N|N|N|N|N| +2442244|AAAAAAAAEAEEFCAA|1974-07-15|894|3889|299|1974|1|7|15|3|1974|299|3889|Monday|1974Q3|N|N|N|2442230|2442410|2441879|2442153|N|N|N|N|N| +2442245|AAAAAAAAFAEEFCAA|1974-07-16|894|3890|299|1974|2|7|16|3|1974|299|3890|Tuesday|1974Q3|N|N|N|2442230|2442410|2441880|2442154|N|N|N|N|N| +2442246|AAAAAAAAGAEEFCAA|1974-07-17|894|3890|299|1974|3|7|17|3|1974|299|3890|Wednesday|1974Q3|N|N|N|2442230|2442410|2441881|2442155|N|N|N|N|N| +2442247|AAAAAAAAHAEEFCAA|1974-07-18|894|3890|299|1974|4|7|18|3|1974|299|3890|Thursday|1974Q3|N|N|N|2442230|2442410|2441882|2442156|N|N|N|N|N| +2442248|AAAAAAAAIAEEFCAA|1974-07-19|894|3890|299|1974|5|7|19|3|1974|299|3890|Friday|1974Q3|N|Y|N|2442230|2442410|2441883|2442157|N|N|N|N|N| +2442249|AAAAAAAAJAEEFCAA|1974-07-20|894|3890|299|1974|6|7|20|3|1974|299|3890|Saturday|1974Q3|N|Y|N|2442230|2442410|2441884|2442158|N|N|N|N|N| +2442250|AAAAAAAAKAEEFCAA|1974-07-21|894|3890|299|1974|0|7|21|3|1974|299|3890|Sunday|1974Q3|N|N|N|2442230|2442410|2441885|2442159|N|N|N|N|N| +2442251|AAAAAAAALAEEFCAA|1974-07-22|894|3890|299|1974|1|7|22|3|1974|299|3890|Monday|1974Q3|N|N|N|2442230|2442410|2441886|2442160|N|N|N|N|N| +2442252|AAAAAAAAMAEEFCAA|1974-07-23|894|3891|299|1974|2|7|23|3|1974|299|3891|Tuesday|1974Q3|N|N|N|2442230|2442410|2441887|2442161|N|N|N|N|N| +2442253|AAAAAAAANAEEFCAA|1974-07-24|894|3891|299|1974|3|7|24|3|1974|299|3891|Wednesday|1974Q3|N|N|N|2442230|2442410|2441888|2442162|N|N|N|N|N| +2442254|AAAAAAAAOAEEFCAA|1974-07-25|894|3891|299|1974|4|7|25|3|1974|299|3891|Thursday|1974Q3|N|N|N|2442230|2442410|2441889|2442163|N|N|N|N|N| +2442255|AAAAAAAAPAEEFCAA|1974-07-26|894|3891|299|1974|5|7|26|3|1974|299|3891|Friday|1974Q3|N|Y|N|2442230|2442410|2441890|2442164|N|N|N|N|N| +2442256|AAAAAAAAABEEFCAA|1974-07-27|894|3891|299|1974|6|7|27|3|1974|299|3891|Saturday|1974Q3|N|Y|N|2442230|2442410|2441891|2442165|N|N|N|N|N| +2442257|AAAAAAAABBEEFCAA|1974-07-28|894|3891|299|1974|0|7|28|3|1974|299|3891|Sunday|1974Q3|N|N|N|2442230|2442410|2441892|2442166|N|N|N|N|N| +2442258|AAAAAAAACBEEFCAA|1974-07-29|894|3891|299|1974|1|7|29|3|1974|299|3891|Monday|1974Q3|N|N|N|2442230|2442410|2441893|2442167|N|N|N|N|N| +2442259|AAAAAAAADBEEFCAA|1974-07-30|894|3892|299|1974|2|7|30|3|1974|299|3892|Tuesday|1974Q3|N|N|N|2442230|2442410|2441894|2442168|N|N|N|N|N| +2442260|AAAAAAAAEBEEFCAA|1974-07-31|894|3892|299|1974|3|7|31|3|1974|299|3892|Wednesday|1974Q3|N|N|N|2442230|2442410|2441895|2442169|N|N|N|N|N| +2442261|AAAAAAAAFBEEFCAA|1974-08-01|895|3892|299|1974|4|8|1|3|1974|299|3892|Thursday|1974Q3|N|N|N|2442261|2442472|2441896|2442170|N|N|N|N|N| +2442262|AAAAAAAAGBEEFCAA|1974-08-02|895|3892|299|1974|5|8|2|3|1974|299|3892|Friday|1974Q3|N|Y|N|2442261|2442472|2441897|2442171|N|N|N|N|N| +2442263|AAAAAAAAHBEEFCAA|1974-08-03|895|3892|299|1974|6|8|3|3|1974|299|3892|Saturday|1974Q3|N|Y|N|2442261|2442472|2441898|2442172|N|N|N|N|N| +2442264|AAAAAAAAIBEEFCAA|1974-08-04|895|3892|299|1974|0|8|4|3|1974|299|3892|Sunday|1974Q3|N|N|N|2442261|2442472|2441899|2442173|N|N|N|N|N| +2442265|AAAAAAAAJBEEFCAA|1974-08-05|895|3892|299|1974|1|8|5|3|1974|299|3892|Monday|1974Q3|N|N|N|2442261|2442472|2441900|2442174|N|N|N|N|N| +2442266|AAAAAAAAKBEEFCAA|1974-08-06|895|3893|299|1974|2|8|6|3|1974|299|3893|Tuesday|1974Q3|N|N|N|2442261|2442472|2441901|2442175|N|N|N|N|N| +2442267|AAAAAAAALBEEFCAA|1974-08-07|895|3893|299|1974|3|8|7|3|1974|299|3893|Wednesday|1974Q3|N|N|N|2442261|2442472|2441902|2442176|N|N|N|N|N| +2442268|AAAAAAAAMBEEFCAA|1974-08-08|895|3893|299|1974|4|8|8|3|1974|299|3893|Thursday|1974Q3|N|N|N|2442261|2442472|2441903|2442177|N|N|N|N|N| +2442269|AAAAAAAANBEEFCAA|1974-08-09|895|3893|299|1974|5|8|9|3|1974|299|3893|Friday|1974Q3|N|Y|N|2442261|2442472|2441904|2442178|N|N|N|N|N| +2442270|AAAAAAAAOBEEFCAA|1974-08-10|895|3893|299|1974|6|8|10|3|1974|299|3893|Saturday|1974Q3|N|Y|N|2442261|2442472|2441905|2442179|N|N|N|N|N| +2442271|AAAAAAAAPBEEFCAA|1974-08-11|895|3893|299|1974|0|8|11|3|1974|299|3893|Sunday|1974Q3|N|N|N|2442261|2442472|2441906|2442180|N|N|N|N|N| +2442272|AAAAAAAAACEEFCAA|1974-08-12|895|3893|299|1974|1|8|12|3|1974|299|3893|Monday|1974Q3|N|N|N|2442261|2442472|2441907|2442181|N|N|N|N|N| +2442273|AAAAAAAABCEEFCAA|1974-08-13|895|3894|299|1974|2|8|13|3|1974|299|3894|Tuesday|1974Q3|N|N|N|2442261|2442472|2441908|2442182|N|N|N|N|N| +2442274|AAAAAAAACCEEFCAA|1974-08-14|895|3894|299|1974|3|8|14|3|1974|299|3894|Wednesday|1974Q3|N|N|N|2442261|2442472|2441909|2442183|N|N|N|N|N| +2442275|AAAAAAAADCEEFCAA|1974-08-15|895|3894|299|1974|4|8|15|3|1974|299|3894|Thursday|1974Q3|N|N|N|2442261|2442472|2441910|2442184|N|N|N|N|N| +2442276|AAAAAAAAECEEFCAA|1974-08-16|895|3894|299|1974|5|8|16|3|1974|299|3894|Friday|1974Q3|N|Y|N|2442261|2442472|2441911|2442185|N|N|N|N|N| +2442277|AAAAAAAAFCEEFCAA|1974-08-17|895|3894|299|1974|6|8|17|3|1974|299|3894|Saturday|1974Q3|N|Y|N|2442261|2442472|2441912|2442186|N|N|N|N|N| +2442278|AAAAAAAAGCEEFCAA|1974-08-18|895|3894|299|1974|0|8|18|3|1974|299|3894|Sunday|1974Q3|N|N|N|2442261|2442472|2441913|2442187|N|N|N|N|N| +2442279|AAAAAAAAHCEEFCAA|1974-08-19|895|3894|299|1974|1|8|19|3|1974|299|3894|Monday|1974Q3|N|N|N|2442261|2442472|2441914|2442188|N|N|N|N|N| +2442280|AAAAAAAAICEEFCAA|1974-08-20|895|3895|299|1974|2|8|20|3|1974|299|3895|Tuesday|1974Q3|N|N|N|2442261|2442472|2441915|2442189|N|N|N|N|N| +2442281|AAAAAAAAJCEEFCAA|1974-08-21|895|3895|299|1974|3|8|21|3|1974|299|3895|Wednesday|1974Q3|N|N|N|2442261|2442472|2441916|2442190|N|N|N|N|N| +2442282|AAAAAAAAKCEEFCAA|1974-08-22|895|3895|299|1974|4|8|22|3|1974|299|3895|Thursday|1974Q3|N|N|N|2442261|2442472|2441917|2442191|N|N|N|N|N| +2442283|AAAAAAAALCEEFCAA|1974-08-23|895|3895|299|1974|5|8|23|3|1974|299|3895|Friday|1974Q3|N|Y|N|2442261|2442472|2441918|2442192|N|N|N|N|N| +2442284|AAAAAAAAMCEEFCAA|1974-08-24|895|3895|299|1974|6|8|24|3|1974|299|3895|Saturday|1974Q3|N|Y|N|2442261|2442472|2441919|2442193|N|N|N|N|N| +2442285|AAAAAAAANCEEFCAA|1974-08-25|895|3895|299|1974|0|8|25|3|1974|299|3895|Sunday|1974Q3|N|N|N|2442261|2442472|2441920|2442194|N|N|N|N|N| +2442286|AAAAAAAAOCEEFCAA|1974-08-26|895|3895|299|1974|1|8|26|3|1974|299|3895|Monday|1974Q3|N|N|N|2442261|2442472|2441921|2442195|N|N|N|N|N| +2442287|AAAAAAAAPCEEFCAA|1974-08-27|895|3896|299|1974|2|8|27|3|1974|299|3896|Tuesday|1974Q3|N|N|N|2442261|2442472|2441922|2442196|N|N|N|N|N| +2442288|AAAAAAAAADEEFCAA|1974-08-28|895|3896|299|1974|3|8|28|3|1974|299|3896|Wednesday|1974Q3|N|N|N|2442261|2442472|2441923|2442197|N|N|N|N|N| +2442289|AAAAAAAABDEEFCAA|1974-08-29|895|3896|299|1974|4|8|29|3|1974|299|3896|Thursday|1974Q3|N|N|N|2442261|2442472|2441924|2442198|N|N|N|N|N| +2442290|AAAAAAAACDEEFCAA|1974-08-30|895|3896|299|1974|5|8|30|3|1974|299|3896|Friday|1974Q3|N|Y|N|2442261|2442472|2441925|2442199|N|N|N|N|N| +2442291|AAAAAAAADDEEFCAA|1974-08-31|895|3896|299|1974|6|8|31|3|1974|299|3896|Saturday|1974Q3|N|Y|N|2442261|2442472|2441926|2442200|N|N|N|N|N| +2442292|AAAAAAAAEDEEFCAA|1974-09-01|896|3896|300|1974|0|9|1|3|1974|300|3896|Sunday|1974Q3|N|N|N|2442292|2442534|2441927|2442201|N|N|N|N|N| +2442293|AAAAAAAAFDEEFCAA|1974-09-02|896|3896|300|1974|1|9|2|3|1974|300|3896|Monday|1974Q3|N|N|N|2442292|2442534|2441928|2442202|N|N|N|N|N| +2442294|AAAAAAAAGDEEFCAA|1974-09-03|896|3897|300|1974|2|9|3|3|1974|300|3897|Tuesday|1974Q3|N|N|N|2442292|2442534|2441929|2442203|N|N|N|N|N| +2442295|AAAAAAAAHDEEFCAA|1974-09-04|896|3897|300|1974|3|9|4|3|1974|300|3897|Wednesday|1974Q3|N|N|N|2442292|2442534|2441930|2442204|N|N|N|N|N| +2442296|AAAAAAAAIDEEFCAA|1974-09-05|896|3897|300|1974|4|9|5|3|1974|300|3897|Thursday|1974Q3|N|N|N|2442292|2442534|2441931|2442205|N|N|N|N|N| +2442297|AAAAAAAAJDEEFCAA|1974-09-06|896|3897|300|1974|5|9|6|3|1974|300|3897|Friday|1974Q3|N|Y|N|2442292|2442534|2441932|2442206|N|N|N|N|N| +2442298|AAAAAAAAKDEEFCAA|1974-09-07|896|3897|300|1974|6|9|7|3|1974|300|3897|Saturday|1974Q3|N|Y|N|2442292|2442534|2441933|2442207|N|N|N|N|N| +2442299|AAAAAAAALDEEFCAA|1974-09-08|896|3897|300|1974|0|9|8|3|1974|300|3897|Sunday|1974Q3|N|N|N|2442292|2442534|2441934|2442208|N|N|N|N|N| +2442300|AAAAAAAAMDEEFCAA|1974-09-09|896|3897|300|1974|1|9|9|3|1974|300|3897|Monday|1974Q3|N|N|N|2442292|2442534|2441935|2442209|N|N|N|N|N| +2442301|AAAAAAAANDEEFCAA|1974-09-10|896|3898|300|1974|2|9|10|3|1974|300|3898|Tuesday|1974Q3|N|N|N|2442292|2442534|2441936|2442210|N|N|N|N|N| +2442302|AAAAAAAAODEEFCAA|1974-09-11|896|3898|300|1974|3|9|11|3|1974|300|3898|Wednesday|1974Q3|N|N|N|2442292|2442534|2441937|2442211|N|N|N|N|N| +2442303|AAAAAAAAPDEEFCAA|1974-09-12|896|3898|300|1974|4|9|12|3|1974|300|3898|Thursday|1974Q3|N|N|N|2442292|2442534|2441938|2442212|N|N|N|N|N| +2442304|AAAAAAAAAEEEFCAA|1974-09-13|896|3898|300|1974|5|9|13|3|1974|300|3898|Friday|1974Q3|N|Y|N|2442292|2442534|2441939|2442213|N|N|N|N|N| +2442305|AAAAAAAABEEEFCAA|1974-09-14|896|3898|300|1974|6|9|14|3|1974|300|3898|Saturday|1974Q3|N|Y|N|2442292|2442534|2441940|2442214|N|N|N|N|N| +2442306|AAAAAAAACEEEFCAA|1974-09-15|896|3898|300|1974|0|9|15|3|1974|300|3898|Sunday|1974Q3|N|N|N|2442292|2442534|2441941|2442215|N|N|N|N|N| +2442307|AAAAAAAADEEEFCAA|1974-09-16|896|3898|300|1974|1|9|16|3|1974|300|3898|Monday|1974Q3|N|N|N|2442292|2442534|2441942|2442216|N|N|N|N|N| +2442308|AAAAAAAAEEEEFCAA|1974-09-17|896|3899|300|1974|2|9|17|3|1974|300|3899|Tuesday|1974Q3|N|N|N|2442292|2442534|2441943|2442217|N|N|N|N|N| +2442309|AAAAAAAAFEEEFCAA|1974-09-18|896|3899|300|1974|3|9|18|3|1974|300|3899|Wednesday|1974Q3|N|N|N|2442292|2442534|2441944|2442218|N|N|N|N|N| +2442310|AAAAAAAAGEEEFCAA|1974-09-19|896|3899|300|1974|4|9|19|3|1974|300|3899|Thursday|1974Q3|N|N|N|2442292|2442534|2441945|2442219|N|N|N|N|N| +2442311|AAAAAAAAHEEEFCAA|1974-09-20|896|3899|300|1974|5|9|20|3|1974|300|3899|Friday|1974Q3|N|Y|N|2442292|2442534|2441946|2442220|N|N|N|N|N| +2442312|AAAAAAAAIEEEFCAA|1974-09-21|896|3899|300|1974|6|9|21|3|1974|300|3899|Saturday|1974Q3|N|Y|N|2442292|2442534|2441947|2442221|N|N|N|N|N| +2442313|AAAAAAAAJEEEFCAA|1974-09-22|896|3899|300|1974|0|9|22|3|1974|300|3899|Sunday|1974Q3|N|N|N|2442292|2442534|2441948|2442222|N|N|N|N|N| +2442314|AAAAAAAAKEEEFCAA|1974-09-23|896|3899|300|1974|1|9|23|3|1974|300|3899|Monday|1974Q3|N|N|N|2442292|2442534|2441949|2442223|N|N|N|N|N| +2442315|AAAAAAAALEEEFCAA|1974-09-24|896|3900|300|1974|2|9|24|3|1974|300|3900|Tuesday|1974Q3|N|N|N|2442292|2442534|2441950|2442224|N|N|N|N|N| +2442316|AAAAAAAAMEEEFCAA|1974-09-25|896|3900|300|1974|3|9|25|3|1974|300|3900|Wednesday|1974Q3|N|N|N|2442292|2442534|2441951|2442225|N|N|N|N|N| +2442317|AAAAAAAANEEEFCAA|1974-09-26|896|3900|300|1974|4|9|26|3|1974|300|3900|Thursday|1974Q3|N|N|N|2442292|2442534|2441952|2442226|N|N|N|N|N| +2442318|AAAAAAAAOEEEFCAA|1974-09-27|896|3900|300|1974|5|9|27|3|1974|300|3900|Friday|1974Q3|N|Y|N|2442292|2442534|2441953|2442227|N|N|N|N|N| +2442319|AAAAAAAAPEEEFCAA|1974-09-28|896|3900|300|1974|6|9|28|3|1974|300|3900|Saturday|1974Q3|N|Y|N|2442292|2442534|2441954|2442228|N|N|N|N|N| +2442320|AAAAAAAAAFEEFCAA|1974-09-29|896|3900|300|1974|0|9|29|3|1974|300|3900|Sunday|1974Q3|N|N|N|2442292|2442534|2441955|2442229|N|N|N|N|N| +2442321|AAAAAAAABFEEFCAA|1974-09-30|896|3900|300|1974|1|9|30|3|1974|300|3900|Monday|1974Q3|N|N|N|2442292|2442534|2441956|2442230|N|N|N|N|N| +2442322|AAAAAAAACFEEFCAA|1974-10-01|897|3901|300|1974|2|10|1|3|1974|300|3901|Tuesday|1974Q3|N|N|N|2442322|2442594|2441957|2442230|N|N|N|N|N| +2442323|AAAAAAAADFEEFCAA|1974-10-02|897|3901|300|1974|3|10|2|4|1974|300|3901|Wednesday|1974Q4|N|N|N|2442322|2442594|2441958|2442231|N|N|N|N|N| +2442324|AAAAAAAAEFEEFCAA|1974-10-03|897|3901|300|1974|4|10|3|4|1974|300|3901|Thursday|1974Q4|N|N|N|2442322|2442594|2441959|2442232|N|N|N|N|N| +2442325|AAAAAAAAFFEEFCAA|1974-10-04|897|3901|300|1974|5|10|4|4|1974|300|3901|Friday|1974Q4|N|Y|N|2442322|2442594|2441960|2442233|N|N|N|N|N| +2442326|AAAAAAAAGFEEFCAA|1974-10-05|897|3901|300|1974|6|10|5|4|1974|300|3901|Saturday|1974Q4|N|Y|N|2442322|2442594|2441961|2442234|N|N|N|N|N| +2442327|AAAAAAAAHFEEFCAA|1974-10-06|897|3901|300|1974|0|10|6|4|1974|300|3901|Sunday|1974Q4|N|N|N|2442322|2442594|2441962|2442235|N|N|N|N|N| +2442328|AAAAAAAAIFEEFCAA|1974-10-07|897|3901|300|1974|1|10|7|4|1974|300|3901|Monday|1974Q4|N|N|N|2442322|2442594|2441963|2442236|N|N|N|N|N| +2442329|AAAAAAAAJFEEFCAA|1974-10-08|897|3902|300|1974|2|10|8|4|1974|300|3902|Tuesday|1974Q4|N|N|N|2442322|2442594|2441964|2442237|N|N|N|N|N| +2442330|AAAAAAAAKFEEFCAA|1974-10-09|897|3902|300|1974|3|10|9|4|1974|300|3902|Wednesday|1974Q4|N|N|N|2442322|2442594|2441965|2442238|N|N|N|N|N| +2442331|AAAAAAAALFEEFCAA|1974-10-10|897|3902|300|1974|4|10|10|4|1974|300|3902|Thursday|1974Q4|N|N|N|2442322|2442594|2441966|2442239|N|N|N|N|N| +2442332|AAAAAAAAMFEEFCAA|1974-10-11|897|3902|300|1974|5|10|11|4|1974|300|3902|Friday|1974Q4|N|Y|N|2442322|2442594|2441967|2442240|N|N|N|N|N| +2442333|AAAAAAAANFEEFCAA|1974-10-12|897|3902|300|1974|6|10|12|4|1974|300|3902|Saturday|1974Q4|N|Y|N|2442322|2442594|2441968|2442241|N|N|N|N|N| +2442334|AAAAAAAAOFEEFCAA|1974-10-13|897|3902|300|1974|0|10|13|4|1974|300|3902|Sunday|1974Q4|N|N|N|2442322|2442594|2441969|2442242|N|N|N|N|N| +2442335|AAAAAAAAPFEEFCAA|1974-10-14|897|3902|300|1974|1|10|14|4|1974|300|3902|Monday|1974Q4|N|N|N|2442322|2442594|2441970|2442243|N|N|N|N|N| +2442336|AAAAAAAAAGEEFCAA|1974-10-15|897|3903|300|1974|2|10|15|4|1974|300|3903|Tuesday|1974Q4|N|N|N|2442322|2442594|2441971|2442244|N|N|N|N|N| +2442337|AAAAAAAABGEEFCAA|1974-10-16|897|3903|300|1974|3|10|16|4|1974|300|3903|Wednesday|1974Q4|N|N|N|2442322|2442594|2441972|2442245|N|N|N|N|N| +2442338|AAAAAAAACGEEFCAA|1974-10-17|897|3903|300|1974|4|10|17|4|1974|300|3903|Thursday|1974Q4|N|N|N|2442322|2442594|2441973|2442246|N|N|N|N|N| +2442339|AAAAAAAADGEEFCAA|1974-10-18|897|3903|300|1974|5|10|18|4|1974|300|3903|Friday|1974Q4|N|Y|N|2442322|2442594|2441974|2442247|N|N|N|N|N| +2442340|AAAAAAAAEGEEFCAA|1974-10-19|897|3903|300|1974|6|10|19|4|1974|300|3903|Saturday|1974Q4|N|Y|N|2442322|2442594|2441975|2442248|N|N|N|N|N| +2442341|AAAAAAAAFGEEFCAA|1974-10-20|897|3903|300|1974|0|10|20|4|1974|300|3903|Sunday|1974Q4|N|N|N|2442322|2442594|2441976|2442249|N|N|N|N|N| +2442342|AAAAAAAAGGEEFCAA|1974-10-21|897|3903|300|1974|1|10|21|4|1974|300|3903|Monday|1974Q4|N|N|N|2442322|2442594|2441977|2442250|N|N|N|N|N| +2442343|AAAAAAAAHGEEFCAA|1974-10-22|897|3904|300|1974|2|10|22|4|1974|300|3904|Tuesday|1974Q4|N|N|N|2442322|2442594|2441978|2442251|N|N|N|N|N| +2442344|AAAAAAAAIGEEFCAA|1974-10-23|897|3904|300|1974|3|10|23|4|1974|300|3904|Wednesday|1974Q4|N|N|N|2442322|2442594|2441979|2442252|N|N|N|N|N| +2442345|AAAAAAAAJGEEFCAA|1974-10-24|897|3904|300|1974|4|10|24|4|1974|300|3904|Thursday|1974Q4|N|N|N|2442322|2442594|2441980|2442253|N|N|N|N|N| +2442346|AAAAAAAAKGEEFCAA|1974-10-25|897|3904|300|1974|5|10|25|4|1974|300|3904|Friday|1974Q4|N|Y|N|2442322|2442594|2441981|2442254|N|N|N|N|N| +2442347|AAAAAAAALGEEFCAA|1974-10-26|897|3904|300|1974|6|10|26|4|1974|300|3904|Saturday|1974Q4|N|Y|N|2442322|2442594|2441982|2442255|N|N|N|N|N| +2442348|AAAAAAAAMGEEFCAA|1974-10-27|897|3904|300|1974|0|10|27|4|1974|300|3904|Sunday|1974Q4|N|N|N|2442322|2442594|2441983|2442256|N|N|N|N|N| +2442349|AAAAAAAANGEEFCAA|1974-10-28|897|3904|300|1974|1|10|28|4|1974|300|3904|Monday|1974Q4|N|N|N|2442322|2442594|2441984|2442257|N|N|N|N|N| +2442350|AAAAAAAAOGEEFCAA|1974-10-29|897|3905|300|1974|2|10|29|4|1974|300|3905|Tuesday|1974Q4|N|N|N|2442322|2442594|2441985|2442258|N|N|N|N|N| +2442351|AAAAAAAAPGEEFCAA|1974-10-30|897|3905|300|1974|3|10|30|4|1974|300|3905|Wednesday|1974Q4|N|N|N|2442322|2442594|2441986|2442259|N|N|N|N|N| +2442352|AAAAAAAAAHEEFCAA|1974-10-31|897|3905|300|1974|4|10|31|4|1974|300|3905|Thursday|1974Q4|N|N|N|2442322|2442594|2441987|2442260|N|N|N|N|N| +2442353|AAAAAAAABHEEFCAA|1974-11-01|898|3905|300|1974|5|11|1|4|1974|300|3905|Friday|1974Q4|N|Y|N|2442353|2442656|2441988|2442261|N|N|N|N|N| +2442354|AAAAAAAACHEEFCAA|1974-11-02|898|3905|300|1974|6|11|2|4|1974|300|3905|Saturday|1974Q4|N|Y|N|2442353|2442656|2441989|2442262|N|N|N|N|N| +2442355|AAAAAAAADHEEFCAA|1974-11-03|898|3905|300|1974|0|11|3|4|1974|300|3905|Sunday|1974Q4|N|N|N|2442353|2442656|2441990|2442263|N|N|N|N|N| +2442356|AAAAAAAAEHEEFCAA|1974-11-04|898|3905|300|1974|1|11|4|4|1974|300|3905|Monday|1974Q4|N|N|N|2442353|2442656|2441991|2442264|N|N|N|N|N| +2442357|AAAAAAAAFHEEFCAA|1974-11-05|898|3906|300|1974|2|11|5|4|1974|300|3906|Tuesday|1974Q4|N|N|N|2442353|2442656|2441992|2442265|N|N|N|N|N| +2442358|AAAAAAAAGHEEFCAA|1974-11-06|898|3906|300|1974|3|11|6|4|1974|300|3906|Wednesday|1974Q4|N|N|N|2442353|2442656|2441993|2442266|N|N|N|N|N| +2442359|AAAAAAAAHHEEFCAA|1974-11-07|898|3906|300|1974|4|11|7|4|1974|300|3906|Thursday|1974Q4|N|N|N|2442353|2442656|2441994|2442267|N|N|N|N|N| +2442360|AAAAAAAAIHEEFCAA|1974-11-08|898|3906|300|1974|5|11|8|4|1974|300|3906|Friday|1974Q4|N|Y|N|2442353|2442656|2441995|2442268|N|N|N|N|N| +2442361|AAAAAAAAJHEEFCAA|1974-11-09|898|3906|300|1974|6|11|9|4|1974|300|3906|Saturday|1974Q4|N|Y|N|2442353|2442656|2441996|2442269|N|N|N|N|N| +2442362|AAAAAAAAKHEEFCAA|1974-11-10|898|3906|300|1974|0|11|10|4|1974|300|3906|Sunday|1974Q4|N|N|N|2442353|2442656|2441997|2442270|N|N|N|N|N| +2442363|AAAAAAAALHEEFCAA|1974-11-11|898|3906|300|1974|1|11|11|4|1974|300|3906|Monday|1974Q4|N|N|N|2442353|2442656|2441998|2442271|N|N|N|N|N| +2442364|AAAAAAAAMHEEFCAA|1974-11-12|898|3907|300|1974|2|11|12|4|1974|300|3907|Tuesday|1974Q4|N|N|N|2442353|2442656|2441999|2442272|N|N|N|N|N| +2442365|AAAAAAAANHEEFCAA|1974-11-13|898|3907|300|1974|3|11|13|4|1974|300|3907|Wednesday|1974Q4|N|N|N|2442353|2442656|2442000|2442273|N|N|N|N|N| +2442366|AAAAAAAAOHEEFCAA|1974-11-14|898|3907|300|1974|4|11|14|4|1974|300|3907|Thursday|1974Q4|N|N|N|2442353|2442656|2442001|2442274|N|N|N|N|N| +2442367|AAAAAAAAPHEEFCAA|1974-11-15|898|3907|300|1974|5|11|15|4|1974|300|3907|Friday|1974Q4|N|Y|N|2442353|2442656|2442002|2442275|N|N|N|N|N| +2442368|AAAAAAAAAIEEFCAA|1974-11-16|898|3907|300|1974|6|11|16|4|1974|300|3907|Saturday|1974Q4|N|Y|N|2442353|2442656|2442003|2442276|N|N|N|N|N| +2442369|AAAAAAAABIEEFCAA|1974-11-17|898|3907|300|1974|0|11|17|4|1974|300|3907|Sunday|1974Q4|N|N|N|2442353|2442656|2442004|2442277|N|N|N|N|N| +2442370|AAAAAAAACIEEFCAA|1974-11-18|898|3907|300|1974|1|11|18|4|1974|300|3907|Monday|1974Q4|N|N|N|2442353|2442656|2442005|2442278|N|N|N|N|N| +2442371|AAAAAAAADIEEFCAA|1974-11-19|898|3908|300|1974|2|11|19|4|1974|300|3908|Tuesday|1974Q4|N|N|N|2442353|2442656|2442006|2442279|N|N|N|N|N| +2442372|AAAAAAAAEIEEFCAA|1974-11-20|898|3908|300|1974|3|11|20|4|1974|300|3908|Wednesday|1974Q4|N|N|N|2442353|2442656|2442007|2442280|N|N|N|N|N| +2442373|AAAAAAAAFIEEFCAA|1974-11-21|898|3908|300|1974|4|11|21|4|1974|300|3908|Thursday|1974Q4|N|N|N|2442353|2442656|2442008|2442281|N|N|N|N|N| +2442374|AAAAAAAAGIEEFCAA|1974-11-22|898|3908|300|1974|5|11|22|4|1974|300|3908|Friday|1974Q4|N|Y|N|2442353|2442656|2442009|2442282|N|N|N|N|N| +2442375|AAAAAAAAHIEEFCAA|1974-11-23|898|3908|300|1974|6|11|23|4|1974|300|3908|Saturday|1974Q4|N|Y|N|2442353|2442656|2442010|2442283|N|N|N|N|N| +2442376|AAAAAAAAIIEEFCAA|1974-11-24|898|3908|300|1974|0|11|24|4|1974|300|3908|Sunday|1974Q4|N|N|N|2442353|2442656|2442011|2442284|N|N|N|N|N| +2442377|AAAAAAAAJIEEFCAA|1974-11-25|898|3908|300|1974|1|11|25|4|1974|300|3908|Monday|1974Q4|N|N|N|2442353|2442656|2442012|2442285|N|N|N|N|N| +2442378|AAAAAAAAKIEEFCAA|1974-11-26|898|3909|300|1974|2|11|26|4|1974|300|3909|Tuesday|1974Q4|N|N|N|2442353|2442656|2442013|2442286|N|N|N|N|N| +2442379|AAAAAAAALIEEFCAA|1974-11-27|898|3909|300|1974|3|11|27|4|1974|300|3909|Wednesday|1974Q4|N|N|N|2442353|2442656|2442014|2442287|N|N|N|N|N| +2442380|AAAAAAAAMIEEFCAA|1974-11-28|898|3909|300|1974|4|11|28|4|1974|300|3909|Thursday|1974Q4|N|N|N|2442353|2442656|2442015|2442288|N|N|N|N|N| +2442381|AAAAAAAANIEEFCAA|1974-11-29|898|3909|300|1974|5|11|29|4|1974|300|3909|Friday|1974Q4|N|Y|N|2442353|2442656|2442016|2442289|N|N|N|N|N| +2442382|AAAAAAAAOIEEFCAA|1974-11-30|898|3909|300|1974|6|11|30|4|1974|300|3909|Saturday|1974Q4|N|Y|N|2442353|2442656|2442017|2442290|N|N|N|N|N| +2442383|AAAAAAAAPIEEFCAA|1974-12-01|899|3909|301|1974|0|12|1|4|1974|301|3909|Sunday|1974Q4|N|N|N|2442383|2442716|2442018|2442291|N|N|N|N|N| +2442384|AAAAAAAAAJEEFCAA|1974-12-02|899|3909|301|1974|1|12|2|4|1974|301|3909|Monday|1974Q4|N|N|N|2442383|2442716|2442019|2442292|N|N|N|N|N| +2442385|AAAAAAAABJEEFCAA|1974-12-03|899|3910|301|1974|2|12|3|4|1974|301|3910|Tuesday|1974Q4|N|N|N|2442383|2442716|2442020|2442293|N|N|N|N|N| +2442386|AAAAAAAACJEEFCAA|1974-12-04|899|3910|301|1974|3|12|4|4|1974|301|3910|Wednesday|1974Q4|N|N|N|2442383|2442716|2442021|2442294|N|N|N|N|N| +2442387|AAAAAAAADJEEFCAA|1974-12-05|899|3910|301|1974|4|12|5|4|1974|301|3910|Thursday|1974Q4|N|N|N|2442383|2442716|2442022|2442295|N|N|N|N|N| +2442388|AAAAAAAAEJEEFCAA|1974-12-06|899|3910|301|1974|5|12|6|4|1974|301|3910|Friday|1974Q4|N|Y|N|2442383|2442716|2442023|2442296|N|N|N|N|N| +2442389|AAAAAAAAFJEEFCAA|1974-12-07|899|3910|301|1974|6|12|7|4|1974|301|3910|Saturday|1974Q4|N|Y|N|2442383|2442716|2442024|2442297|N|N|N|N|N| +2442390|AAAAAAAAGJEEFCAA|1974-12-08|899|3910|301|1974|0|12|8|4|1974|301|3910|Sunday|1974Q4|N|N|N|2442383|2442716|2442025|2442298|N|N|N|N|N| +2442391|AAAAAAAAHJEEFCAA|1974-12-09|899|3910|301|1974|1|12|9|4|1974|301|3910|Monday|1974Q4|N|N|N|2442383|2442716|2442026|2442299|N|N|N|N|N| +2442392|AAAAAAAAIJEEFCAA|1974-12-10|899|3911|301|1974|2|12|10|4|1974|301|3911|Tuesday|1974Q4|N|N|N|2442383|2442716|2442027|2442300|N|N|N|N|N| +2442393|AAAAAAAAJJEEFCAA|1974-12-11|899|3911|301|1974|3|12|11|4|1974|301|3911|Wednesday|1974Q4|N|N|N|2442383|2442716|2442028|2442301|N|N|N|N|N| +2442394|AAAAAAAAKJEEFCAA|1974-12-12|899|3911|301|1974|4|12|12|4|1974|301|3911|Thursday|1974Q4|N|N|N|2442383|2442716|2442029|2442302|N|N|N|N|N| +2442395|AAAAAAAALJEEFCAA|1974-12-13|899|3911|301|1974|5|12|13|4|1974|301|3911|Friday|1974Q4|N|Y|N|2442383|2442716|2442030|2442303|N|N|N|N|N| +2442396|AAAAAAAAMJEEFCAA|1974-12-14|899|3911|301|1974|6|12|14|4|1974|301|3911|Saturday|1974Q4|N|Y|N|2442383|2442716|2442031|2442304|N|N|N|N|N| +2442397|AAAAAAAANJEEFCAA|1974-12-15|899|3911|301|1974|0|12|15|4|1974|301|3911|Sunday|1974Q4|N|N|N|2442383|2442716|2442032|2442305|N|N|N|N|N| +2442398|AAAAAAAAOJEEFCAA|1974-12-16|899|3911|301|1974|1|12|16|4|1974|301|3911|Monday|1974Q4|N|N|N|2442383|2442716|2442033|2442306|N|N|N|N|N| +2442399|AAAAAAAAPJEEFCAA|1974-12-17|899|3912|301|1974|2|12|17|4|1974|301|3912|Tuesday|1974Q4|N|N|N|2442383|2442716|2442034|2442307|N|N|N|N|N| +2442400|AAAAAAAAAKEEFCAA|1974-12-18|899|3912|301|1974|3|12|18|4|1974|301|3912|Wednesday|1974Q4|N|N|N|2442383|2442716|2442035|2442308|N|N|N|N|N| +2442401|AAAAAAAABKEEFCAA|1974-12-19|899|3912|301|1974|4|12|19|4|1974|301|3912|Thursday|1974Q4|N|N|N|2442383|2442716|2442036|2442309|N|N|N|N|N| +2442402|AAAAAAAACKEEFCAA|1974-12-20|899|3912|301|1974|5|12|20|4|1974|301|3912|Friday|1974Q4|N|Y|N|2442383|2442716|2442037|2442310|N|N|N|N|N| +2442403|AAAAAAAADKEEFCAA|1974-12-21|899|3912|301|1974|6|12|21|4|1974|301|3912|Saturday|1974Q4|N|Y|N|2442383|2442716|2442038|2442311|N|N|N|N|N| +2442404|AAAAAAAAEKEEFCAA|1974-12-22|899|3912|301|1974|0|12|22|4|1974|301|3912|Sunday|1974Q4|N|N|N|2442383|2442716|2442039|2442312|N|N|N|N|N| +2442405|AAAAAAAAFKEEFCAA|1974-12-23|899|3912|301|1974|1|12|23|4|1974|301|3912|Monday|1974Q4|N|N|N|2442383|2442716|2442040|2442313|N|N|N|N|N| +2442406|AAAAAAAAGKEEFCAA|1974-12-24|899|3913|301|1974|2|12|24|4|1974|301|3913|Tuesday|1974Q4|N|N|N|2442383|2442716|2442041|2442314|N|N|N|N|N| +2442407|AAAAAAAAHKEEFCAA|1974-12-25|899|3913|301|1974|3|12|25|4|1974|301|3913|Wednesday|1974Q4|N|N|N|2442383|2442716|2442042|2442315|N|N|N|N|N| +2442408|AAAAAAAAIKEEFCAA|1974-12-26|899|3913|301|1974|4|12|26|4|1974|301|3913|Thursday|1974Q4|Y|N|N|2442383|2442716|2442043|2442316|N|N|N|N|N| +2442409|AAAAAAAAJKEEFCAA|1974-12-27|899|3913|301|1974|5|12|27|4|1974|301|3913|Friday|1974Q4|N|Y|Y|2442383|2442716|2442044|2442317|N|N|N|N|N| +2442410|AAAAAAAAKKEEFCAA|1974-12-28|899|3913|301|1974|6|12|28|4|1974|301|3913|Saturday|1974Q4|N|Y|N|2442383|2442716|2442045|2442318|N|N|N|N|N| +2442411|AAAAAAAALKEEFCAA|1974-12-29|899|3913|301|1974|0|12|29|4|1974|301|3913|Sunday|1974Q4|N|N|N|2442383|2442716|2442046|2442319|N|N|N|N|N| +2442412|AAAAAAAAMKEEFCAA|1974-12-30|899|3913|301|1974|1|12|30|4|1974|301|3913|Monday|1974Q4|N|N|N|2442383|2442716|2442047|2442320|N|N|N|N|N| +2442413|AAAAAAAANKEEFCAA|1974-12-31|899|3914|301|1974|2|12|31|4|1974|301|3914|Tuesday|1974Q4|N|N|N|2442383|2442716|2442048|2442321|N|N|N|N|N| +2442414|AAAAAAAAOKEEFCAA|1975-01-01|900|3914|301|1975|3|1|1|1|1975|301|3914|Wednesday|1975Q1|Y|N|N|2442414|2442413|2442049|2442322|N|N|N|N|N| +2442415|AAAAAAAAPKEEFCAA|1975-01-02|900|3914|301|1975|4|1|2|1|1975|301|3914|Thursday|1975Q1|N|N|Y|2442414|2442413|2442050|2442323|N|N|N|N|N| +2442416|AAAAAAAAALEEFCAA|1975-01-03|900|3914|301|1975|5|1|3|1|1975|301|3914|Friday|1975Q1|N|Y|N|2442414|2442413|2442051|2442324|N|N|N|N|N| +2442417|AAAAAAAABLEEFCAA|1975-01-04|900|3914|301|1975|6|1|4|1|1975|301|3914|Saturday|1975Q1|N|Y|N|2442414|2442413|2442052|2442325|N|N|N|N|N| +2442418|AAAAAAAACLEEFCAA|1975-01-05|900|3914|301|1975|0|1|5|1|1975|301|3914|Sunday|1975Q1|N|N|N|2442414|2442413|2442053|2442326|N|N|N|N|N| +2442419|AAAAAAAADLEEFCAA|1975-01-06|900|3914|301|1975|1|1|6|1|1975|301|3914|Monday|1975Q1|N|N|N|2442414|2442413|2442054|2442327|N|N|N|N|N| +2442420|AAAAAAAAELEEFCAA|1975-01-07|900|3915|301|1975|2|1|7|1|1975|301|3915|Tuesday|1975Q1|N|N|N|2442414|2442413|2442055|2442328|N|N|N|N|N| +2442421|AAAAAAAAFLEEFCAA|1975-01-08|900|3915|301|1975|3|1|8|1|1975|301|3915|Wednesday|1975Q1|N|N|N|2442414|2442413|2442056|2442329|N|N|N|N|N| +2442422|AAAAAAAAGLEEFCAA|1975-01-09|900|3915|301|1975|4|1|9|1|1975|301|3915|Thursday|1975Q1|N|N|N|2442414|2442413|2442057|2442330|N|N|N|N|N| +2442423|AAAAAAAAHLEEFCAA|1975-01-10|900|3915|301|1975|5|1|10|1|1975|301|3915|Friday|1975Q1|N|Y|N|2442414|2442413|2442058|2442331|N|N|N|N|N| +2442424|AAAAAAAAILEEFCAA|1975-01-11|900|3915|301|1975|6|1|11|1|1975|301|3915|Saturday|1975Q1|N|Y|N|2442414|2442413|2442059|2442332|N|N|N|N|N| +2442425|AAAAAAAAJLEEFCAA|1975-01-12|900|3915|301|1975|0|1|12|1|1975|301|3915|Sunday|1975Q1|N|N|N|2442414|2442413|2442060|2442333|N|N|N|N|N| +2442426|AAAAAAAAKLEEFCAA|1975-01-13|900|3915|301|1975|1|1|13|1|1975|301|3915|Monday|1975Q1|N|N|N|2442414|2442413|2442061|2442334|N|N|N|N|N| +2442427|AAAAAAAALLEEFCAA|1975-01-14|900|3916|301|1975|2|1|14|1|1975|301|3916|Tuesday|1975Q1|N|N|N|2442414|2442413|2442062|2442335|N|N|N|N|N| +2442428|AAAAAAAAMLEEFCAA|1975-01-15|900|3916|301|1975|3|1|15|1|1975|301|3916|Wednesday|1975Q1|N|N|N|2442414|2442413|2442063|2442336|N|N|N|N|N| +2442429|AAAAAAAANLEEFCAA|1975-01-16|900|3916|301|1975|4|1|16|1|1975|301|3916|Thursday|1975Q1|N|N|N|2442414|2442413|2442064|2442337|N|N|N|N|N| +2442430|AAAAAAAAOLEEFCAA|1975-01-17|900|3916|301|1975|5|1|17|1|1975|301|3916|Friday|1975Q1|N|Y|N|2442414|2442413|2442065|2442338|N|N|N|N|N| +2442431|AAAAAAAAPLEEFCAA|1975-01-18|900|3916|301|1975|6|1|18|1|1975|301|3916|Saturday|1975Q1|N|Y|N|2442414|2442413|2442066|2442339|N|N|N|N|N| +2442432|AAAAAAAAAMEEFCAA|1975-01-19|900|3916|301|1975|0|1|19|1|1975|301|3916|Sunday|1975Q1|N|N|N|2442414|2442413|2442067|2442340|N|N|N|N|N| +2442433|AAAAAAAABMEEFCAA|1975-01-20|900|3916|301|1975|1|1|20|1|1975|301|3916|Monday|1975Q1|N|N|N|2442414|2442413|2442068|2442341|N|N|N|N|N| +2442434|AAAAAAAACMEEFCAA|1975-01-21|900|3917|301|1975|2|1|21|1|1975|301|3917|Tuesday|1975Q1|N|N|N|2442414|2442413|2442069|2442342|N|N|N|N|N| +2442435|AAAAAAAADMEEFCAA|1975-01-22|900|3917|301|1975|3|1|22|1|1975|301|3917|Wednesday|1975Q1|N|N|N|2442414|2442413|2442070|2442343|N|N|N|N|N| +2442436|AAAAAAAAEMEEFCAA|1975-01-23|900|3917|301|1975|4|1|23|1|1975|301|3917|Thursday|1975Q1|N|N|N|2442414|2442413|2442071|2442344|N|N|N|N|N| +2442437|AAAAAAAAFMEEFCAA|1975-01-24|900|3917|301|1975|5|1|24|1|1975|301|3917|Friday|1975Q1|N|Y|N|2442414|2442413|2442072|2442345|N|N|N|N|N| +2442438|AAAAAAAAGMEEFCAA|1975-01-25|900|3917|301|1975|6|1|25|1|1975|301|3917|Saturday|1975Q1|N|Y|N|2442414|2442413|2442073|2442346|N|N|N|N|N| +2442439|AAAAAAAAHMEEFCAA|1975-01-26|900|3917|301|1975|0|1|26|1|1975|301|3917|Sunday|1975Q1|N|N|N|2442414|2442413|2442074|2442347|N|N|N|N|N| +2442440|AAAAAAAAIMEEFCAA|1975-01-27|900|3917|301|1975|1|1|27|1|1975|301|3917|Monday|1975Q1|N|N|N|2442414|2442413|2442075|2442348|N|N|N|N|N| +2442441|AAAAAAAAJMEEFCAA|1975-01-28|900|3918|301|1975|2|1|28|1|1975|301|3918|Tuesday|1975Q1|N|N|N|2442414|2442413|2442076|2442349|N|N|N|N|N| +2442442|AAAAAAAAKMEEFCAA|1975-01-29|900|3918|301|1975|3|1|29|1|1975|301|3918|Wednesday|1975Q1|N|N|N|2442414|2442413|2442077|2442350|N|N|N|N|N| +2442443|AAAAAAAALMEEFCAA|1975-01-30|900|3918|301|1975|4|1|30|1|1975|301|3918|Thursday|1975Q1|N|N|N|2442414|2442413|2442078|2442351|N|N|N|N|N| +2442444|AAAAAAAAMMEEFCAA|1975-01-31|900|3918|301|1975|5|1|31|1|1975|301|3918|Friday|1975Q1|N|Y|N|2442414|2442413|2442079|2442352|N|N|N|N|N| +2442445|AAAAAAAANMEEFCAA|1975-02-01|901|3918|301|1975|6|2|1|1|1975|301|3918|Saturday|1975Q1|N|Y|N|2442445|2442475|2442080|2442353|N|N|N|N|N| +2442446|AAAAAAAAOMEEFCAA|1975-02-02|901|3918|301|1975|0|2|2|1|1975|301|3918|Sunday|1975Q1|N|N|N|2442445|2442475|2442081|2442354|N|N|N|N|N| +2442447|AAAAAAAAPMEEFCAA|1975-02-03|901|3918|301|1975|1|2|3|1|1975|301|3918|Monday|1975Q1|N|N|N|2442445|2442475|2442082|2442355|N|N|N|N|N| +2442448|AAAAAAAAANEEFCAA|1975-02-04|901|3919|301|1975|2|2|4|1|1975|301|3919|Tuesday|1975Q1|N|N|N|2442445|2442475|2442083|2442356|N|N|N|N|N| +2442449|AAAAAAAABNEEFCAA|1975-02-05|901|3919|301|1975|3|2|5|1|1975|301|3919|Wednesday|1975Q1|N|N|N|2442445|2442475|2442084|2442357|N|N|N|N|N| +2442450|AAAAAAAACNEEFCAA|1975-02-06|901|3919|301|1975|4|2|6|1|1975|301|3919|Thursday|1975Q1|N|N|N|2442445|2442475|2442085|2442358|N|N|N|N|N| +2442451|AAAAAAAADNEEFCAA|1975-02-07|901|3919|301|1975|5|2|7|1|1975|301|3919|Friday|1975Q1|N|Y|N|2442445|2442475|2442086|2442359|N|N|N|N|N| +2442452|AAAAAAAAENEEFCAA|1975-02-08|901|3919|301|1975|6|2|8|1|1975|301|3919|Saturday|1975Q1|N|Y|N|2442445|2442475|2442087|2442360|N|N|N|N|N| +2442453|AAAAAAAAFNEEFCAA|1975-02-09|901|3919|301|1975|0|2|9|1|1975|301|3919|Sunday|1975Q1|N|N|N|2442445|2442475|2442088|2442361|N|N|N|N|N| +2442454|AAAAAAAAGNEEFCAA|1975-02-10|901|3919|301|1975|1|2|10|1|1975|301|3919|Monday|1975Q1|N|N|N|2442445|2442475|2442089|2442362|N|N|N|N|N| +2442455|AAAAAAAAHNEEFCAA|1975-02-11|901|3920|301|1975|2|2|11|1|1975|301|3920|Tuesday|1975Q1|N|N|N|2442445|2442475|2442090|2442363|N|N|N|N|N| +2442456|AAAAAAAAINEEFCAA|1975-02-12|901|3920|301|1975|3|2|12|1|1975|301|3920|Wednesday|1975Q1|N|N|N|2442445|2442475|2442091|2442364|N|N|N|N|N| +2442457|AAAAAAAAJNEEFCAA|1975-02-13|901|3920|301|1975|4|2|13|1|1975|301|3920|Thursday|1975Q1|N|N|N|2442445|2442475|2442092|2442365|N|N|N|N|N| +2442458|AAAAAAAAKNEEFCAA|1975-02-14|901|3920|301|1975|5|2|14|1|1975|301|3920|Friday|1975Q1|N|Y|N|2442445|2442475|2442093|2442366|N|N|N|N|N| +2442459|AAAAAAAALNEEFCAA|1975-02-15|901|3920|301|1975|6|2|15|1|1975|301|3920|Saturday|1975Q1|N|Y|N|2442445|2442475|2442094|2442367|N|N|N|N|N| +2442460|AAAAAAAAMNEEFCAA|1975-02-16|901|3920|301|1975|0|2|16|1|1975|301|3920|Sunday|1975Q1|N|N|N|2442445|2442475|2442095|2442368|N|N|N|N|N| +2442461|AAAAAAAANNEEFCAA|1975-02-17|901|3920|301|1975|1|2|17|1|1975|301|3920|Monday|1975Q1|N|N|N|2442445|2442475|2442096|2442369|N|N|N|N|N| +2442462|AAAAAAAAONEEFCAA|1975-02-18|901|3921|301|1975|2|2|18|1|1975|301|3921|Tuesday|1975Q1|N|N|N|2442445|2442475|2442097|2442370|N|N|N|N|N| +2442463|AAAAAAAAPNEEFCAA|1975-02-19|901|3921|301|1975|3|2|19|1|1975|301|3921|Wednesday|1975Q1|N|N|N|2442445|2442475|2442098|2442371|N|N|N|N|N| +2442464|AAAAAAAAAOEEFCAA|1975-02-20|901|3921|301|1975|4|2|20|1|1975|301|3921|Thursday|1975Q1|N|N|N|2442445|2442475|2442099|2442372|N|N|N|N|N| +2442465|AAAAAAAABOEEFCAA|1975-02-21|901|3921|301|1975|5|2|21|1|1975|301|3921|Friday|1975Q1|N|Y|N|2442445|2442475|2442100|2442373|N|N|N|N|N| +2442466|AAAAAAAACOEEFCAA|1975-02-22|901|3921|301|1975|6|2|22|1|1975|301|3921|Saturday|1975Q1|N|Y|N|2442445|2442475|2442101|2442374|N|N|N|N|N| +2442467|AAAAAAAADOEEFCAA|1975-02-23|901|3921|301|1975|0|2|23|1|1975|301|3921|Sunday|1975Q1|N|N|N|2442445|2442475|2442102|2442375|N|N|N|N|N| +2442468|AAAAAAAAEOEEFCAA|1975-02-24|901|3921|301|1975|1|2|24|1|1975|301|3921|Monday|1975Q1|N|N|N|2442445|2442475|2442103|2442376|N|N|N|N|N| +2442469|AAAAAAAAFOEEFCAA|1975-02-25|901|3922|301|1975|2|2|25|1|1975|301|3922|Tuesday|1975Q1|N|N|N|2442445|2442475|2442104|2442377|N|N|N|N|N| +2442470|AAAAAAAAGOEEFCAA|1975-02-26|901|3922|301|1975|3|2|26|1|1975|301|3922|Wednesday|1975Q1|N|N|N|2442445|2442475|2442105|2442378|N|N|N|N|N| +2442471|AAAAAAAAHOEEFCAA|1975-02-27|901|3922|301|1975|4|2|27|1|1975|301|3922|Thursday|1975Q1|N|N|N|2442445|2442475|2442106|2442379|N|N|N|N|N| +2442472|AAAAAAAAIOEEFCAA|1975-02-28|901|3922|301|1975|5|2|28|1|1975|301|3922|Friday|1975Q1|N|Y|N|2442445|2442475|2442107|2442380|N|N|N|N|N| +2442473|AAAAAAAAJOEEFCAA|1975-03-01|902|3922|302|1975|6|3|1|1|1975|302|3922|Saturday|1975Q1|N|Y|N|2442473|2442531|2442108|2442381|N|N|N|N|N| +2442474|AAAAAAAAKOEEFCAA|1975-03-02|902|3922|302|1975|0|3|2|1|1975|302|3922|Sunday|1975Q1|N|N|N|2442473|2442531|2442109|2442382|N|N|N|N|N| +2442475|AAAAAAAALOEEFCAA|1975-03-03|902|3922|302|1975|1|3|3|1|1975|302|3922|Monday|1975Q1|N|N|N|2442473|2442531|2442110|2442383|N|N|N|N|N| +2442476|AAAAAAAAMOEEFCAA|1975-03-04|902|3923|302|1975|2|3|4|1|1975|302|3923|Tuesday|1975Q1|N|N|N|2442473|2442531|2442111|2442384|N|N|N|N|N| +2442477|AAAAAAAANOEEFCAA|1975-03-05|902|3923|302|1975|3|3|5|1|1975|302|3923|Wednesday|1975Q1|N|N|N|2442473|2442531|2442112|2442385|N|N|N|N|N| +2442478|AAAAAAAAOOEEFCAA|1975-03-06|902|3923|302|1975|4|3|6|1|1975|302|3923|Thursday|1975Q1|N|N|N|2442473|2442531|2442113|2442386|N|N|N|N|N| +2442479|AAAAAAAAPOEEFCAA|1975-03-07|902|3923|302|1975|5|3|7|1|1975|302|3923|Friday|1975Q1|N|Y|N|2442473|2442531|2442114|2442387|N|N|N|N|N| +2442480|AAAAAAAAAPEEFCAA|1975-03-08|902|3923|302|1975|6|3|8|1|1975|302|3923|Saturday|1975Q1|N|Y|N|2442473|2442531|2442115|2442388|N|N|N|N|N| +2442481|AAAAAAAABPEEFCAA|1975-03-09|902|3923|302|1975|0|3|9|1|1975|302|3923|Sunday|1975Q1|N|N|N|2442473|2442531|2442116|2442389|N|N|N|N|N| +2442482|AAAAAAAACPEEFCAA|1975-03-10|902|3923|302|1975|1|3|10|1|1975|302|3923|Monday|1975Q1|N|N|N|2442473|2442531|2442117|2442390|N|N|N|N|N| +2442483|AAAAAAAADPEEFCAA|1975-03-11|902|3924|302|1975|2|3|11|1|1975|302|3924|Tuesday|1975Q1|N|N|N|2442473|2442531|2442118|2442391|N|N|N|N|N| +2442484|AAAAAAAAEPEEFCAA|1975-03-12|902|3924|302|1975|3|3|12|1|1975|302|3924|Wednesday|1975Q1|N|N|N|2442473|2442531|2442119|2442392|N|N|N|N|N| +2442485|AAAAAAAAFPEEFCAA|1975-03-13|902|3924|302|1975|4|3|13|1|1975|302|3924|Thursday|1975Q1|N|N|N|2442473|2442531|2442120|2442393|N|N|N|N|N| +2442486|AAAAAAAAGPEEFCAA|1975-03-14|902|3924|302|1975|5|3|14|1|1975|302|3924|Friday|1975Q1|N|Y|N|2442473|2442531|2442121|2442394|N|N|N|N|N| +2442487|AAAAAAAAHPEEFCAA|1975-03-15|902|3924|302|1975|6|3|15|1|1975|302|3924|Saturday|1975Q1|N|Y|N|2442473|2442531|2442122|2442395|N|N|N|N|N| +2442488|AAAAAAAAIPEEFCAA|1975-03-16|902|3924|302|1975|0|3|16|1|1975|302|3924|Sunday|1975Q1|N|N|N|2442473|2442531|2442123|2442396|N|N|N|N|N| +2442489|AAAAAAAAJPEEFCAA|1975-03-17|902|3924|302|1975|1|3|17|1|1975|302|3924|Monday|1975Q1|N|N|N|2442473|2442531|2442124|2442397|N|N|N|N|N| +2442490|AAAAAAAAKPEEFCAA|1975-03-18|902|3925|302|1975|2|3|18|1|1975|302|3925|Tuesday|1975Q1|N|N|N|2442473|2442531|2442125|2442398|N|N|N|N|N| +2442491|AAAAAAAALPEEFCAA|1975-03-19|902|3925|302|1975|3|3|19|1|1975|302|3925|Wednesday|1975Q1|N|N|N|2442473|2442531|2442126|2442399|N|N|N|N|N| +2442492|AAAAAAAAMPEEFCAA|1975-03-20|902|3925|302|1975|4|3|20|1|1975|302|3925|Thursday|1975Q1|N|N|N|2442473|2442531|2442127|2442400|N|N|N|N|N| +2442493|AAAAAAAANPEEFCAA|1975-03-21|902|3925|302|1975|5|3|21|1|1975|302|3925|Friday|1975Q1|N|Y|N|2442473|2442531|2442128|2442401|N|N|N|N|N| +2442494|AAAAAAAAOPEEFCAA|1975-03-22|902|3925|302|1975|6|3|22|1|1975|302|3925|Saturday|1975Q1|N|Y|N|2442473|2442531|2442129|2442402|N|N|N|N|N| +2442495|AAAAAAAAPPEEFCAA|1975-03-23|902|3925|302|1975|0|3|23|1|1975|302|3925|Sunday|1975Q1|N|N|N|2442473|2442531|2442130|2442403|N|N|N|N|N| +2442496|AAAAAAAAAAFEFCAA|1975-03-24|902|3925|302|1975|1|3|24|1|1975|302|3925|Monday|1975Q1|N|N|N|2442473|2442531|2442131|2442404|N|N|N|N|N| +2442497|AAAAAAAABAFEFCAA|1975-03-25|902|3926|302|1975|2|3|25|1|1975|302|3926|Tuesday|1975Q1|N|N|N|2442473|2442531|2442132|2442405|N|N|N|N|N| +2442498|AAAAAAAACAFEFCAA|1975-03-26|902|3926|302|1975|3|3|26|1|1975|302|3926|Wednesday|1975Q1|N|N|N|2442473|2442531|2442133|2442406|N|N|N|N|N| +2442499|AAAAAAAADAFEFCAA|1975-03-27|902|3926|302|1975|4|3|27|1|1975|302|3926|Thursday|1975Q1|N|N|N|2442473|2442531|2442134|2442407|N|N|N|N|N| +2442500|AAAAAAAAEAFEFCAA|1975-03-28|902|3926|302|1975|5|3|28|1|1975|302|3926|Friday|1975Q1|N|Y|N|2442473|2442531|2442135|2442408|N|N|N|N|N| +2442501|AAAAAAAAFAFEFCAA|1975-03-29|902|3926|302|1975|6|3|29|1|1975|302|3926|Saturday|1975Q1|N|Y|N|2442473|2442531|2442136|2442409|N|N|N|N|N| +2442502|AAAAAAAAGAFEFCAA|1975-03-30|902|3926|302|1975|0|3|30|1|1975|302|3926|Sunday|1975Q1|N|N|N|2442473|2442531|2442137|2442410|N|N|N|N|N| +2442503|AAAAAAAAHAFEFCAA|1975-03-31|902|3926|302|1975|1|3|31|1|1975|302|3926|Monday|1975Q1|N|N|N|2442473|2442531|2442138|2442411|N|N|N|N|N| +2442504|AAAAAAAAIAFEFCAA|1975-04-01|903|3927|302|1975|2|4|1|1|1975|302|3927|Tuesday|1975Q1|N|N|N|2442504|2442593|2442139|2442414|N|N|N|N|N| +2442505|AAAAAAAAJAFEFCAA|1975-04-02|903|3927|302|1975|3|4|2|2|1975|302|3927|Wednesday|1975Q2|N|N|N|2442504|2442593|2442140|2442415|N|N|N|N|N| +2442506|AAAAAAAAKAFEFCAA|1975-04-03|903|3927|302|1975|4|4|3|2|1975|302|3927|Thursday|1975Q2|N|N|N|2442504|2442593|2442141|2442416|N|N|N|N|N| +2442507|AAAAAAAALAFEFCAA|1975-04-04|903|3927|302|1975|5|4|4|2|1975|302|3927|Friday|1975Q2|N|Y|N|2442504|2442593|2442142|2442417|N|N|N|N|N| +2442508|AAAAAAAAMAFEFCAA|1975-04-05|903|3927|302|1975|6|4|5|2|1975|302|3927|Saturday|1975Q2|N|Y|N|2442504|2442593|2442143|2442418|N|N|N|N|N| +2442509|AAAAAAAANAFEFCAA|1975-04-06|903|3927|302|1975|0|4|6|2|1975|302|3927|Sunday|1975Q2|N|N|N|2442504|2442593|2442144|2442419|N|N|N|N|N| +2442510|AAAAAAAAOAFEFCAA|1975-04-07|903|3927|302|1975|1|4|7|2|1975|302|3927|Monday|1975Q2|N|N|N|2442504|2442593|2442145|2442420|N|N|N|N|N| +2442511|AAAAAAAAPAFEFCAA|1975-04-08|903|3928|302|1975|2|4|8|2|1975|302|3928|Tuesday|1975Q2|N|N|N|2442504|2442593|2442146|2442421|N|N|N|N|N| +2442512|AAAAAAAAABFEFCAA|1975-04-09|903|3928|302|1975|3|4|9|2|1975|302|3928|Wednesday|1975Q2|N|N|N|2442504|2442593|2442147|2442422|N|N|N|N|N| +2442513|AAAAAAAABBFEFCAA|1975-04-10|903|3928|302|1975|4|4|10|2|1975|302|3928|Thursday|1975Q2|N|N|N|2442504|2442593|2442148|2442423|N|N|N|N|N| +2442514|AAAAAAAACBFEFCAA|1975-04-11|903|3928|302|1975|5|4|11|2|1975|302|3928|Friday|1975Q2|N|Y|N|2442504|2442593|2442149|2442424|N|N|N|N|N| +2442515|AAAAAAAADBFEFCAA|1975-04-12|903|3928|302|1975|6|4|12|2|1975|302|3928|Saturday|1975Q2|N|Y|N|2442504|2442593|2442150|2442425|N|N|N|N|N| +2442516|AAAAAAAAEBFEFCAA|1975-04-13|903|3928|302|1975|0|4|13|2|1975|302|3928|Sunday|1975Q2|N|N|N|2442504|2442593|2442151|2442426|N|N|N|N|N| +2442517|AAAAAAAAFBFEFCAA|1975-04-14|903|3928|302|1975|1|4|14|2|1975|302|3928|Monday|1975Q2|N|N|N|2442504|2442593|2442152|2442427|N|N|N|N|N| +2442518|AAAAAAAAGBFEFCAA|1975-04-15|903|3929|302|1975|2|4|15|2|1975|302|3929|Tuesday|1975Q2|N|N|N|2442504|2442593|2442153|2442428|N|N|N|N|N| +2442519|AAAAAAAAHBFEFCAA|1975-04-16|903|3929|302|1975|3|4|16|2|1975|302|3929|Wednesday|1975Q2|N|N|N|2442504|2442593|2442154|2442429|N|N|N|N|N| +2442520|AAAAAAAAIBFEFCAA|1975-04-17|903|3929|302|1975|4|4|17|2|1975|302|3929|Thursday|1975Q2|N|N|N|2442504|2442593|2442155|2442430|N|N|N|N|N| +2442521|AAAAAAAAJBFEFCAA|1975-04-18|903|3929|302|1975|5|4|18|2|1975|302|3929|Friday|1975Q2|N|Y|N|2442504|2442593|2442156|2442431|N|N|N|N|N| +2442522|AAAAAAAAKBFEFCAA|1975-04-19|903|3929|302|1975|6|4|19|2|1975|302|3929|Saturday|1975Q2|N|Y|N|2442504|2442593|2442157|2442432|N|N|N|N|N| +2442523|AAAAAAAALBFEFCAA|1975-04-20|903|3929|302|1975|0|4|20|2|1975|302|3929|Sunday|1975Q2|N|N|N|2442504|2442593|2442158|2442433|N|N|N|N|N| +2442524|AAAAAAAAMBFEFCAA|1975-04-21|903|3929|302|1975|1|4|21|2|1975|302|3929|Monday|1975Q2|N|N|N|2442504|2442593|2442159|2442434|N|N|N|N|N| +2442525|AAAAAAAANBFEFCAA|1975-04-22|903|3930|302|1975|2|4|22|2|1975|302|3930|Tuesday|1975Q2|N|N|N|2442504|2442593|2442160|2442435|N|N|N|N|N| +2442526|AAAAAAAAOBFEFCAA|1975-04-23|903|3930|302|1975|3|4|23|2|1975|302|3930|Wednesday|1975Q2|N|N|N|2442504|2442593|2442161|2442436|N|N|N|N|N| +2442527|AAAAAAAAPBFEFCAA|1975-04-24|903|3930|302|1975|4|4|24|2|1975|302|3930|Thursday|1975Q2|N|N|N|2442504|2442593|2442162|2442437|N|N|N|N|N| +2442528|AAAAAAAAACFEFCAA|1975-04-25|903|3930|302|1975|5|4|25|2|1975|302|3930|Friday|1975Q2|N|Y|N|2442504|2442593|2442163|2442438|N|N|N|N|N| +2442529|AAAAAAAABCFEFCAA|1975-04-26|903|3930|302|1975|6|4|26|2|1975|302|3930|Saturday|1975Q2|N|Y|N|2442504|2442593|2442164|2442439|N|N|N|N|N| +2442530|AAAAAAAACCFEFCAA|1975-04-27|903|3930|302|1975|0|4|27|2|1975|302|3930|Sunday|1975Q2|N|N|N|2442504|2442593|2442165|2442440|N|N|N|N|N| +2442531|AAAAAAAADCFEFCAA|1975-04-28|903|3930|302|1975|1|4|28|2|1975|302|3930|Monday|1975Q2|N|N|N|2442504|2442593|2442166|2442441|N|N|N|N|N| +2442532|AAAAAAAAECFEFCAA|1975-04-29|903|3931|302|1975|2|4|29|2|1975|302|3931|Tuesday|1975Q2|N|N|N|2442504|2442593|2442167|2442442|N|N|N|N|N| +2442533|AAAAAAAAFCFEFCAA|1975-04-30|903|3931|302|1975|3|4|30|2|1975|302|3931|Wednesday|1975Q2|N|N|N|2442504|2442593|2442168|2442443|N|N|N|N|N| +2442534|AAAAAAAAGCFEFCAA|1975-05-01|904|3931|302|1975|4|5|1|2|1975|302|3931|Thursday|1975Q2|N|N|N|2442534|2442653|2442169|2442444|N|N|N|N|N| +2442535|AAAAAAAAHCFEFCAA|1975-05-02|904|3931|302|1975|5|5|2|2|1975|302|3931|Friday|1975Q2|N|Y|N|2442534|2442653|2442170|2442445|N|N|N|N|N| +2442536|AAAAAAAAICFEFCAA|1975-05-03|904|3931|302|1975|6|5|3|2|1975|302|3931|Saturday|1975Q2|N|Y|N|2442534|2442653|2442171|2442446|N|N|N|N|N| +2442537|AAAAAAAAJCFEFCAA|1975-05-04|904|3931|302|1975|0|5|4|2|1975|302|3931|Sunday|1975Q2|N|N|N|2442534|2442653|2442172|2442447|N|N|N|N|N| +2442538|AAAAAAAAKCFEFCAA|1975-05-05|904|3931|302|1975|1|5|5|2|1975|302|3931|Monday|1975Q2|N|N|N|2442534|2442653|2442173|2442448|N|N|N|N|N| +2442539|AAAAAAAALCFEFCAA|1975-05-06|904|3932|302|1975|2|5|6|2|1975|302|3932|Tuesday|1975Q2|N|N|N|2442534|2442653|2442174|2442449|N|N|N|N|N| +2442540|AAAAAAAAMCFEFCAA|1975-05-07|904|3932|302|1975|3|5|7|2|1975|302|3932|Wednesday|1975Q2|N|N|N|2442534|2442653|2442175|2442450|N|N|N|N|N| +2442541|AAAAAAAANCFEFCAA|1975-05-08|904|3932|302|1975|4|5|8|2|1975|302|3932|Thursday|1975Q2|N|N|N|2442534|2442653|2442176|2442451|N|N|N|N|N| +2442542|AAAAAAAAOCFEFCAA|1975-05-09|904|3932|302|1975|5|5|9|2|1975|302|3932|Friday|1975Q2|N|Y|N|2442534|2442653|2442177|2442452|N|N|N|N|N| +2442543|AAAAAAAAPCFEFCAA|1975-05-10|904|3932|302|1975|6|5|10|2|1975|302|3932|Saturday|1975Q2|N|Y|N|2442534|2442653|2442178|2442453|N|N|N|N|N| +2442544|AAAAAAAAADFEFCAA|1975-05-11|904|3932|302|1975|0|5|11|2|1975|302|3932|Sunday|1975Q2|N|N|N|2442534|2442653|2442179|2442454|N|N|N|N|N| +2442545|AAAAAAAABDFEFCAA|1975-05-12|904|3932|302|1975|1|5|12|2|1975|302|3932|Monday|1975Q2|N|N|N|2442534|2442653|2442180|2442455|N|N|N|N|N| +2442546|AAAAAAAACDFEFCAA|1975-05-13|904|3933|302|1975|2|5|13|2|1975|302|3933|Tuesday|1975Q2|N|N|N|2442534|2442653|2442181|2442456|N|N|N|N|N| +2442547|AAAAAAAADDFEFCAA|1975-05-14|904|3933|302|1975|3|5|14|2|1975|302|3933|Wednesday|1975Q2|N|N|N|2442534|2442653|2442182|2442457|N|N|N|N|N| +2442548|AAAAAAAAEDFEFCAA|1975-05-15|904|3933|302|1975|4|5|15|2|1975|302|3933|Thursday|1975Q2|N|N|N|2442534|2442653|2442183|2442458|N|N|N|N|N| +2442549|AAAAAAAAFDFEFCAA|1975-05-16|904|3933|302|1975|5|5|16|2|1975|302|3933|Friday|1975Q2|N|Y|N|2442534|2442653|2442184|2442459|N|N|N|N|N| +2442550|AAAAAAAAGDFEFCAA|1975-05-17|904|3933|302|1975|6|5|17|2|1975|302|3933|Saturday|1975Q2|N|Y|N|2442534|2442653|2442185|2442460|N|N|N|N|N| +2442551|AAAAAAAAHDFEFCAA|1975-05-18|904|3933|302|1975|0|5|18|2|1975|302|3933|Sunday|1975Q2|N|N|N|2442534|2442653|2442186|2442461|N|N|N|N|N| +2442552|AAAAAAAAIDFEFCAA|1975-05-19|904|3933|302|1975|1|5|19|2|1975|302|3933|Monday|1975Q2|N|N|N|2442534|2442653|2442187|2442462|N|N|N|N|N| +2442553|AAAAAAAAJDFEFCAA|1975-05-20|904|3934|302|1975|2|5|20|2|1975|302|3934|Tuesday|1975Q2|N|N|N|2442534|2442653|2442188|2442463|N|N|N|N|N| +2442554|AAAAAAAAKDFEFCAA|1975-05-21|904|3934|302|1975|3|5|21|2|1975|302|3934|Wednesday|1975Q2|N|N|N|2442534|2442653|2442189|2442464|N|N|N|N|N| +2442555|AAAAAAAALDFEFCAA|1975-05-22|904|3934|302|1975|4|5|22|2|1975|302|3934|Thursday|1975Q2|N|N|N|2442534|2442653|2442190|2442465|N|N|N|N|N| +2442556|AAAAAAAAMDFEFCAA|1975-05-23|904|3934|302|1975|5|5|23|2|1975|302|3934|Friday|1975Q2|N|Y|N|2442534|2442653|2442191|2442466|N|N|N|N|N| +2442557|AAAAAAAANDFEFCAA|1975-05-24|904|3934|302|1975|6|5|24|2|1975|302|3934|Saturday|1975Q2|N|Y|N|2442534|2442653|2442192|2442467|N|N|N|N|N| +2442558|AAAAAAAAODFEFCAA|1975-05-25|904|3934|302|1975|0|5|25|2|1975|302|3934|Sunday|1975Q2|N|N|N|2442534|2442653|2442193|2442468|N|N|N|N|N| +2442559|AAAAAAAAPDFEFCAA|1975-05-26|904|3934|302|1975|1|5|26|2|1975|302|3934|Monday|1975Q2|N|N|N|2442534|2442653|2442194|2442469|N|N|N|N|N| +2442560|AAAAAAAAAEFEFCAA|1975-05-27|904|3935|302|1975|2|5|27|2|1975|302|3935|Tuesday|1975Q2|N|N|N|2442534|2442653|2442195|2442470|N|N|N|N|N| +2442561|AAAAAAAABEFEFCAA|1975-05-28|904|3935|302|1975|3|5|28|2|1975|302|3935|Wednesday|1975Q2|N|N|N|2442534|2442653|2442196|2442471|N|N|N|N|N| +2442562|AAAAAAAACEFEFCAA|1975-05-29|904|3935|302|1975|4|5|29|2|1975|302|3935|Thursday|1975Q2|N|N|N|2442534|2442653|2442197|2442472|N|N|N|N|N| +2442563|AAAAAAAADEFEFCAA|1975-05-30|904|3935|302|1975|5|5|30|2|1975|302|3935|Friday|1975Q2|N|Y|N|2442534|2442653|2442198|2442473|N|N|N|N|N| +2442564|AAAAAAAAEEFEFCAA|1975-05-31|904|3935|302|1975|6|5|31|2|1975|302|3935|Saturday|1975Q2|N|Y|N|2442534|2442653|2442199|2442474|N|N|N|N|N| +2442565|AAAAAAAAFEFEFCAA|1975-06-01|905|3935|303|1975|0|6|1|2|1975|303|3935|Sunday|1975Q2|N|N|N|2442565|2442715|2442200|2442475|N|N|N|N|N| +2442566|AAAAAAAAGEFEFCAA|1975-06-02|905|3935|303|1975|1|6|2|2|1975|303|3935|Monday|1975Q2|N|N|N|2442565|2442715|2442201|2442476|N|N|N|N|N| +2442567|AAAAAAAAHEFEFCAA|1975-06-03|905|3936|303|1975|2|6|3|2|1975|303|3936|Tuesday|1975Q2|N|N|N|2442565|2442715|2442202|2442477|N|N|N|N|N| +2442568|AAAAAAAAIEFEFCAA|1975-06-04|905|3936|303|1975|3|6|4|2|1975|303|3936|Wednesday|1975Q2|N|N|N|2442565|2442715|2442203|2442478|N|N|N|N|N| +2442569|AAAAAAAAJEFEFCAA|1975-06-05|905|3936|303|1975|4|6|5|2|1975|303|3936|Thursday|1975Q2|N|N|N|2442565|2442715|2442204|2442479|N|N|N|N|N| +2442570|AAAAAAAAKEFEFCAA|1975-06-06|905|3936|303|1975|5|6|6|2|1975|303|3936|Friday|1975Q2|N|Y|N|2442565|2442715|2442205|2442480|N|N|N|N|N| +2442571|AAAAAAAALEFEFCAA|1975-06-07|905|3936|303|1975|6|6|7|2|1975|303|3936|Saturday|1975Q2|N|Y|N|2442565|2442715|2442206|2442481|N|N|N|N|N| +2442572|AAAAAAAAMEFEFCAA|1975-06-08|905|3936|303|1975|0|6|8|2|1975|303|3936|Sunday|1975Q2|N|N|N|2442565|2442715|2442207|2442482|N|N|N|N|N| +2442573|AAAAAAAANEFEFCAA|1975-06-09|905|3936|303|1975|1|6|9|2|1975|303|3936|Monday|1975Q2|N|N|N|2442565|2442715|2442208|2442483|N|N|N|N|N| +2442574|AAAAAAAAOEFEFCAA|1975-06-10|905|3937|303|1975|2|6|10|2|1975|303|3937|Tuesday|1975Q2|N|N|N|2442565|2442715|2442209|2442484|N|N|N|N|N| +2442575|AAAAAAAAPEFEFCAA|1975-06-11|905|3937|303|1975|3|6|11|2|1975|303|3937|Wednesday|1975Q2|N|N|N|2442565|2442715|2442210|2442485|N|N|N|N|N| +2442576|AAAAAAAAAFFEFCAA|1975-06-12|905|3937|303|1975|4|6|12|2|1975|303|3937|Thursday|1975Q2|N|N|N|2442565|2442715|2442211|2442486|N|N|N|N|N| +2442577|AAAAAAAABFFEFCAA|1975-06-13|905|3937|303|1975|5|6|13|2|1975|303|3937|Friday|1975Q2|N|Y|N|2442565|2442715|2442212|2442487|N|N|N|N|N| +2442578|AAAAAAAACFFEFCAA|1975-06-14|905|3937|303|1975|6|6|14|2|1975|303|3937|Saturday|1975Q2|N|Y|N|2442565|2442715|2442213|2442488|N|N|N|N|N| +2442579|AAAAAAAADFFEFCAA|1975-06-15|905|3937|303|1975|0|6|15|2|1975|303|3937|Sunday|1975Q2|N|N|N|2442565|2442715|2442214|2442489|N|N|N|N|N| +2442580|AAAAAAAAEFFEFCAA|1975-06-16|905|3937|303|1975|1|6|16|2|1975|303|3937|Monday|1975Q2|N|N|N|2442565|2442715|2442215|2442490|N|N|N|N|N| +2442581|AAAAAAAAFFFEFCAA|1975-06-17|905|3938|303|1975|2|6|17|2|1975|303|3938|Tuesday|1975Q2|N|N|N|2442565|2442715|2442216|2442491|N|N|N|N|N| +2442582|AAAAAAAAGFFEFCAA|1975-06-18|905|3938|303|1975|3|6|18|2|1975|303|3938|Wednesday|1975Q2|N|N|N|2442565|2442715|2442217|2442492|N|N|N|N|N| +2442583|AAAAAAAAHFFEFCAA|1975-06-19|905|3938|303|1975|4|6|19|2|1975|303|3938|Thursday|1975Q2|N|N|N|2442565|2442715|2442218|2442493|N|N|N|N|N| +2442584|AAAAAAAAIFFEFCAA|1975-06-20|905|3938|303|1975|5|6|20|2|1975|303|3938|Friday|1975Q2|N|Y|N|2442565|2442715|2442219|2442494|N|N|N|N|N| +2442585|AAAAAAAAJFFEFCAA|1975-06-21|905|3938|303|1975|6|6|21|2|1975|303|3938|Saturday|1975Q2|N|Y|N|2442565|2442715|2442220|2442495|N|N|N|N|N| +2442586|AAAAAAAAKFFEFCAA|1975-06-22|905|3938|303|1975|0|6|22|2|1975|303|3938|Sunday|1975Q2|N|N|N|2442565|2442715|2442221|2442496|N|N|N|N|N| +2442587|AAAAAAAALFFEFCAA|1975-06-23|905|3938|303|1975|1|6|23|2|1975|303|3938|Monday|1975Q2|N|N|N|2442565|2442715|2442222|2442497|N|N|N|N|N| +2442588|AAAAAAAAMFFEFCAA|1975-06-24|905|3939|303|1975|2|6|24|2|1975|303|3939|Tuesday|1975Q2|N|N|N|2442565|2442715|2442223|2442498|N|N|N|N|N| +2442589|AAAAAAAANFFEFCAA|1975-06-25|905|3939|303|1975|3|6|25|2|1975|303|3939|Wednesday|1975Q2|N|N|N|2442565|2442715|2442224|2442499|N|N|N|N|N| +2442590|AAAAAAAAOFFEFCAA|1975-06-26|905|3939|303|1975|4|6|26|2|1975|303|3939|Thursday|1975Q2|N|N|N|2442565|2442715|2442225|2442500|N|N|N|N|N| +2442591|AAAAAAAAPFFEFCAA|1975-06-27|905|3939|303|1975|5|6|27|2|1975|303|3939|Friday|1975Q2|N|Y|N|2442565|2442715|2442226|2442501|N|N|N|N|N| +2442592|AAAAAAAAAGFEFCAA|1975-06-28|905|3939|303|1975|6|6|28|2|1975|303|3939|Saturday|1975Q2|N|Y|N|2442565|2442715|2442227|2442502|N|N|N|N|N| +2442593|AAAAAAAABGFEFCAA|1975-06-29|905|3939|303|1975|0|6|29|2|1975|303|3939|Sunday|1975Q2|N|N|N|2442565|2442715|2442228|2442503|N|N|N|N|N| +2442594|AAAAAAAACGFEFCAA|1975-06-30|905|3939|303|1975|1|6|30|2|1975|303|3939|Monday|1975Q2|N|N|N|2442565|2442715|2442229|2442504|N|N|N|N|N| +2442595|AAAAAAAADGFEFCAA|1975-07-01|906|3940|303|1975|2|7|1|2|1975|303|3940|Tuesday|1975Q2|N|N|N|2442595|2442775|2442230|2442504|N|N|N|N|N| +2442596|AAAAAAAAEGFEFCAA|1975-07-02|906|3940|303|1975|3|7|2|3|1975|303|3940|Wednesday|1975Q3|N|N|N|2442595|2442775|2442231|2442505|N|N|N|N|N| +2442597|AAAAAAAAFGFEFCAA|1975-07-03|906|3940|303|1975|4|7|3|3|1975|303|3940|Thursday|1975Q3|N|N|N|2442595|2442775|2442232|2442506|N|N|N|N|N| +2442598|AAAAAAAAGGFEFCAA|1975-07-04|906|3940|303|1975|5|7|4|3|1975|303|3940|Friday|1975Q3|N|Y|N|2442595|2442775|2442233|2442507|N|N|N|N|N| +2442599|AAAAAAAAHGFEFCAA|1975-07-05|906|3940|303|1975|6|7|5|3|1975|303|3940|Saturday|1975Q3|Y|Y|N|2442595|2442775|2442234|2442508|N|N|N|N|N| +2442600|AAAAAAAAIGFEFCAA|1975-07-06|906|3940|303|1975|0|7|6|3|1975|303|3940|Sunday|1975Q3|N|N|Y|2442595|2442775|2442235|2442509|N|N|N|N|N| +2442601|AAAAAAAAJGFEFCAA|1975-07-07|906|3940|303|1975|1|7|7|3|1975|303|3940|Monday|1975Q3|N|N|N|2442595|2442775|2442236|2442510|N|N|N|N|N| +2442602|AAAAAAAAKGFEFCAA|1975-07-08|906|3941|303|1975|2|7|8|3|1975|303|3941|Tuesday|1975Q3|N|N|N|2442595|2442775|2442237|2442511|N|N|N|N|N| +2442603|AAAAAAAALGFEFCAA|1975-07-09|906|3941|303|1975|3|7|9|3|1975|303|3941|Wednesday|1975Q3|N|N|N|2442595|2442775|2442238|2442512|N|N|N|N|N| +2442604|AAAAAAAAMGFEFCAA|1975-07-10|906|3941|303|1975|4|7|10|3|1975|303|3941|Thursday|1975Q3|N|N|N|2442595|2442775|2442239|2442513|N|N|N|N|N| +2442605|AAAAAAAANGFEFCAA|1975-07-11|906|3941|303|1975|5|7|11|3|1975|303|3941|Friday|1975Q3|N|Y|N|2442595|2442775|2442240|2442514|N|N|N|N|N| +2442606|AAAAAAAAOGFEFCAA|1975-07-12|906|3941|303|1975|6|7|12|3|1975|303|3941|Saturday|1975Q3|N|Y|N|2442595|2442775|2442241|2442515|N|N|N|N|N| +2442607|AAAAAAAAPGFEFCAA|1975-07-13|906|3941|303|1975|0|7|13|3|1975|303|3941|Sunday|1975Q3|N|N|N|2442595|2442775|2442242|2442516|N|N|N|N|N| +2442608|AAAAAAAAAHFEFCAA|1975-07-14|906|3941|303|1975|1|7|14|3|1975|303|3941|Monday|1975Q3|N|N|N|2442595|2442775|2442243|2442517|N|N|N|N|N| +2442609|AAAAAAAABHFEFCAA|1975-07-15|906|3942|303|1975|2|7|15|3|1975|303|3942|Tuesday|1975Q3|N|N|N|2442595|2442775|2442244|2442518|N|N|N|N|N| +2442610|AAAAAAAACHFEFCAA|1975-07-16|906|3942|303|1975|3|7|16|3|1975|303|3942|Wednesday|1975Q3|N|N|N|2442595|2442775|2442245|2442519|N|N|N|N|N| +2442611|AAAAAAAADHFEFCAA|1975-07-17|906|3942|303|1975|4|7|17|3|1975|303|3942|Thursday|1975Q3|N|N|N|2442595|2442775|2442246|2442520|N|N|N|N|N| +2442612|AAAAAAAAEHFEFCAA|1975-07-18|906|3942|303|1975|5|7|18|3|1975|303|3942|Friday|1975Q3|N|Y|N|2442595|2442775|2442247|2442521|N|N|N|N|N| +2442613|AAAAAAAAFHFEFCAA|1975-07-19|906|3942|303|1975|6|7|19|3|1975|303|3942|Saturday|1975Q3|N|Y|N|2442595|2442775|2442248|2442522|N|N|N|N|N| +2442614|AAAAAAAAGHFEFCAA|1975-07-20|906|3942|303|1975|0|7|20|3|1975|303|3942|Sunday|1975Q3|N|N|N|2442595|2442775|2442249|2442523|N|N|N|N|N| +2442615|AAAAAAAAHHFEFCAA|1975-07-21|906|3942|303|1975|1|7|21|3|1975|303|3942|Monday|1975Q3|N|N|N|2442595|2442775|2442250|2442524|N|N|N|N|N| +2442616|AAAAAAAAIHFEFCAA|1975-07-22|906|3943|303|1975|2|7|22|3|1975|303|3943|Tuesday|1975Q3|N|N|N|2442595|2442775|2442251|2442525|N|N|N|N|N| +2442617|AAAAAAAAJHFEFCAA|1975-07-23|906|3943|303|1975|3|7|23|3|1975|303|3943|Wednesday|1975Q3|N|N|N|2442595|2442775|2442252|2442526|N|N|N|N|N| +2442618|AAAAAAAAKHFEFCAA|1975-07-24|906|3943|303|1975|4|7|24|3|1975|303|3943|Thursday|1975Q3|N|N|N|2442595|2442775|2442253|2442527|N|N|N|N|N| +2442619|AAAAAAAALHFEFCAA|1975-07-25|906|3943|303|1975|5|7|25|3|1975|303|3943|Friday|1975Q3|N|Y|N|2442595|2442775|2442254|2442528|N|N|N|N|N| +2442620|AAAAAAAAMHFEFCAA|1975-07-26|906|3943|303|1975|6|7|26|3|1975|303|3943|Saturday|1975Q3|N|Y|N|2442595|2442775|2442255|2442529|N|N|N|N|N| +2442621|AAAAAAAANHFEFCAA|1975-07-27|906|3943|303|1975|0|7|27|3|1975|303|3943|Sunday|1975Q3|N|N|N|2442595|2442775|2442256|2442530|N|N|N|N|N| +2442622|AAAAAAAAOHFEFCAA|1975-07-28|906|3943|303|1975|1|7|28|3|1975|303|3943|Monday|1975Q3|N|N|N|2442595|2442775|2442257|2442531|N|N|N|N|N| +2442623|AAAAAAAAPHFEFCAA|1975-07-29|906|3944|303|1975|2|7|29|3|1975|303|3944|Tuesday|1975Q3|N|N|N|2442595|2442775|2442258|2442532|N|N|N|N|N| +2442624|AAAAAAAAAIFEFCAA|1975-07-30|906|3944|303|1975|3|7|30|3|1975|303|3944|Wednesday|1975Q3|N|N|N|2442595|2442775|2442259|2442533|N|N|N|N|N| +2442625|AAAAAAAABIFEFCAA|1975-07-31|906|3944|303|1975|4|7|31|3|1975|303|3944|Thursday|1975Q3|N|N|N|2442595|2442775|2442260|2442534|N|N|N|N|N| +2442626|AAAAAAAACIFEFCAA|1975-08-01|907|3944|303|1975|5|8|1|3|1975|303|3944|Friday|1975Q3|N|Y|N|2442626|2442837|2442261|2442535|N|N|N|N|N| +2442627|AAAAAAAADIFEFCAA|1975-08-02|907|3944|303|1975|6|8|2|3|1975|303|3944|Saturday|1975Q3|N|Y|N|2442626|2442837|2442262|2442536|N|N|N|N|N| +2442628|AAAAAAAAEIFEFCAA|1975-08-03|907|3944|303|1975|0|8|3|3|1975|303|3944|Sunday|1975Q3|N|N|N|2442626|2442837|2442263|2442537|N|N|N|N|N| +2442629|AAAAAAAAFIFEFCAA|1975-08-04|907|3944|303|1975|1|8|4|3|1975|303|3944|Monday|1975Q3|N|N|N|2442626|2442837|2442264|2442538|N|N|N|N|N| +2442630|AAAAAAAAGIFEFCAA|1975-08-05|907|3945|303|1975|2|8|5|3|1975|303|3945|Tuesday|1975Q3|N|N|N|2442626|2442837|2442265|2442539|N|N|N|N|N| +2442631|AAAAAAAAHIFEFCAA|1975-08-06|907|3945|303|1975|3|8|6|3|1975|303|3945|Wednesday|1975Q3|N|N|N|2442626|2442837|2442266|2442540|N|N|N|N|N| +2442632|AAAAAAAAIIFEFCAA|1975-08-07|907|3945|303|1975|4|8|7|3|1975|303|3945|Thursday|1975Q3|N|N|N|2442626|2442837|2442267|2442541|N|N|N|N|N| +2442633|AAAAAAAAJIFEFCAA|1975-08-08|907|3945|303|1975|5|8|8|3|1975|303|3945|Friday|1975Q3|N|Y|N|2442626|2442837|2442268|2442542|N|N|N|N|N| +2442634|AAAAAAAAKIFEFCAA|1975-08-09|907|3945|303|1975|6|8|9|3|1975|303|3945|Saturday|1975Q3|N|Y|N|2442626|2442837|2442269|2442543|N|N|N|N|N| +2442635|AAAAAAAALIFEFCAA|1975-08-10|907|3945|303|1975|0|8|10|3|1975|303|3945|Sunday|1975Q3|N|N|N|2442626|2442837|2442270|2442544|N|N|N|N|N| +2442636|AAAAAAAAMIFEFCAA|1975-08-11|907|3945|303|1975|1|8|11|3|1975|303|3945|Monday|1975Q3|N|N|N|2442626|2442837|2442271|2442545|N|N|N|N|N| +2442637|AAAAAAAANIFEFCAA|1975-08-12|907|3946|303|1975|2|8|12|3|1975|303|3946|Tuesday|1975Q3|N|N|N|2442626|2442837|2442272|2442546|N|N|N|N|N| +2442638|AAAAAAAAOIFEFCAA|1975-08-13|907|3946|303|1975|3|8|13|3|1975|303|3946|Wednesday|1975Q3|N|N|N|2442626|2442837|2442273|2442547|N|N|N|N|N| +2442639|AAAAAAAAPIFEFCAA|1975-08-14|907|3946|303|1975|4|8|14|3|1975|303|3946|Thursday|1975Q3|N|N|N|2442626|2442837|2442274|2442548|N|N|N|N|N| +2442640|AAAAAAAAAJFEFCAA|1975-08-15|907|3946|303|1975|5|8|15|3|1975|303|3946|Friday|1975Q3|N|Y|N|2442626|2442837|2442275|2442549|N|N|N|N|N| +2442641|AAAAAAAABJFEFCAA|1975-08-16|907|3946|303|1975|6|8|16|3|1975|303|3946|Saturday|1975Q3|N|Y|N|2442626|2442837|2442276|2442550|N|N|N|N|N| +2442642|AAAAAAAACJFEFCAA|1975-08-17|907|3946|303|1975|0|8|17|3|1975|303|3946|Sunday|1975Q3|N|N|N|2442626|2442837|2442277|2442551|N|N|N|N|N| +2442643|AAAAAAAADJFEFCAA|1975-08-18|907|3946|303|1975|1|8|18|3|1975|303|3946|Monday|1975Q3|N|N|N|2442626|2442837|2442278|2442552|N|N|N|N|N| +2442644|AAAAAAAAEJFEFCAA|1975-08-19|907|3947|303|1975|2|8|19|3|1975|303|3947|Tuesday|1975Q3|N|N|N|2442626|2442837|2442279|2442553|N|N|N|N|N| +2442645|AAAAAAAAFJFEFCAA|1975-08-20|907|3947|303|1975|3|8|20|3|1975|303|3947|Wednesday|1975Q3|N|N|N|2442626|2442837|2442280|2442554|N|N|N|N|N| +2442646|AAAAAAAAGJFEFCAA|1975-08-21|907|3947|303|1975|4|8|21|3|1975|303|3947|Thursday|1975Q3|N|N|N|2442626|2442837|2442281|2442555|N|N|N|N|N| +2442647|AAAAAAAAHJFEFCAA|1975-08-22|907|3947|303|1975|5|8|22|3|1975|303|3947|Friday|1975Q3|N|Y|N|2442626|2442837|2442282|2442556|N|N|N|N|N| +2442648|AAAAAAAAIJFEFCAA|1975-08-23|907|3947|303|1975|6|8|23|3|1975|303|3947|Saturday|1975Q3|N|Y|N|2442626|2442837|2442283|2442557|N|N|N|N|N| +2442649|AAAAAAAAJJFEFCAA|1975-08-24|907|3947|303|1975|0|8|24|3|1975|303|3947|Sunday|1975Q3|N|N|N|2442626|2442837|2442284|2442558|N|N|N|N|N| +2442650|AAAAAAAAKJFEFCAA|1975-08-25|907|3947|303|1975|1|8|25|3|1975|303|3947|Monday|1975Q3|N|N|N|2442626|2442837|2442285|2442559|N|N|N|N|N| +2442651|AAAAAAAALJFEFCAA|1975-08-26|907|3948|303|1975|2|8|26|3|1975|303|3948|Tuesday|1975Q3|N|N|N|2442626|2442837|2442286|2442560|N|N|N|N|N| +2442652|AAAAAAAAMJFEFCAA|1975-08-27|907|3948|303|1975|3|8|27|3|1975|303|3948|Wednesday|1975Q3|N|N|N|2442626|2442837|2442287|2442561|N|N|N|N|N| +2442653|AAAAAAAANJFEFCAA|1975-08-28|907|3948|303|1975|4|8|28|3|1975|303|3948|Thursday|1975Q3|N|N|N|2442626|2442837|2442288|2442562|N|N|N|N|N| +2442654|AAAAAAAAOJFEFCAA|1975-08-29|907|3948|303|1975|5|8|29|3|1975|303|3948|Friday|1975Q3|N|Y|N|2442626|2442837|2442289|2442563|N|N|N|N|N| +2442655|AAAAAAAAPJFEFCAA|1975-08-30|907|3948|303|1975|6|8|30|3|1975|303|3948|Saturday|1975Q3|N|Y|N|2442626|2442837|2442290|2442564|N|N|N|N|N| +2442656|AAAAAAAAAKFEFCAA|1975-08-31|907|3948|303|1975|0|8|31|3|1975|303|3948|Sunday|1975Q3|N|N|N|2442626|2442837|2442291|2442565|N|N|N|N|N| +2442657|AAAAAAAABKFEFCAA|1975-09-01|908|3948|304|1975|1|9|1|3|1975|304|3948|Monday|1975Q3|N|N|N|2442657|2442899|2442292|2442566|N|N|N|N|N| +2442658|AAAAAAAACKFEFCAA|1975-09-02|908|3949|304|1975|2|9|2|3|1975|304|3949|Tuesday|1975Q3|N|N|N|2442657|2442899|2442293|2442567|N|N|N|N|N| +2442659|AAAAAAAADKFEFCAA|1975-09-03|908|3949|304|1975|3|9|3|3|1975|304|3949|Wednesday|1975Q3|N|N|N|2442657|2442899|2442294|2442568|N|N|N|N|N| +2442660|AAAAAAAAEKFEFCAA|1975-09-04|908|3949|304|1975|4|9|4|3|1975|304|3949|Thursday|1975Q3|N|N|N|2442657|2442899|2442295|2442569|N|N|N|N|N| +2442661|AAAAAAAAFKFEFCAA|1975-09-05|908|3949|304|1975|5|9|5|3|1975|304|3949|Friday|1975Q3|N|Y|N|2442657|2442899|2442296|2442570|N|N|N|N|N| +2442662|AAAAAAAAGKFEFCAA|1975-09-06|908|3949|304|1975|6|9|6|3|1975|304|3949|Saturday|1975Q3|N|Y|N|2442657|2442899|2442297|2442571|N|N|N|N|N| +2442663|AAAAAAAAHKFEFCAA|1975-09-07|908|3949|304|1975|0|9|7|3|1975|304|3949|Sunday|1975Q3|N|N|N|2442657|2442899|2442298|2442572|N|N|N|N|N| +2442664|AAAAAAAAIKFEFCAA|1975-09-08|908|3949|304|1975|1|9|8|3|1975|304|3949|Monday|1975Q3|N|N|N|2442657|2442899|2442299|2442573|N|N|N|N|N| +2442665|AAAAAAAAJKFEFCAA|1975-09-09|908|3950|304|1975|2|9|9|3|1975|304|3950|Tuesday|1975Q3|N|N|N|2442657|2442899|2442300|2442574|N|N|N|N|N| +2442666|AAAAAAAAKKFEFCAA|1975-09-10|908|3950|304|1975|3|9|10|3|1975|304|3950|Wednesday|1975Q3|N|N|N|2442657|2442899|2442301|2442575|N|N|N|N|N| +2442667|AAAAAAAALKFEFCAA|1975-09-11|908|3950|304|1975|4|9|11|3|1975|304|3950|Thursday|1975Q3|N|N|N|2442657|2442899|2442302|2442576|N|N|N|N|N| +2442668|AAAAAAAAMKFEFCAA|1975-09-12|908|3950|304|1975|5|9|12|3|1975|304|3950|Friday|1975Q3|N|Y|N|2442657|2442899|2442303|2442577|N|N|N|N|N| +2442669|AAAAAAAANKFEFCAA|1975-09-13|908|3950|304|1975|6|9|13|3|1975|304|3950|Saturday|1975Q3|N|Y|N|2442657|2442899|2442304|2442578|N|N|N|N|N| +2442670|AAAAAAAAOKFEFCAA|1975-09-14|908|3950|304|1975|0|9|14|3|1975|304|3950|Sunday|1975Q3|N|N|N|2442657|2442899|2442305|2442579|N|N|N|N|N| +2442671|AAAAAAAAPKFEFCAA|1975-09-15|908|3950|304|1975|1|9|15|3|1975|304|3950|Monday|1975Q3|N|N|N|2442657|2442899|2442306|2442580|N|N|N|N|N| +2442672|AAAAAAAAALFEFCAA|1975-09-16|908|3951|304|1975|2|9|16|3|1975|304|3951|Tuesday|1975Q3|N|N|N|2442657|2442899|2442307|2442581|N|N|N|N|N| +2442673|AAAAAAAABLFEFCAA|1975-09-17|908|3951|304|1975|3|9|17|3|1975|304|3951|Wednesday|1975Q3|N|N|N|2442657|2442899|2442308|2442582|N|N|N|N|N| +2442674|AAAAAAAACLFEFCAA|1975-09-18|908|3951|304|1975|4|9|18|3|1975|304|3951|Thursday|1975Q3|N|N|N|2442657|2442899|2442309|2442583|N|N|N|N|N| +2442675|AAAAAAAADLFEFCAA|1975-09-19|908|3951|304|1975|5|9|19|3|1975|304|3951|Friday|1975Q3|N|Y|N|2442657|2442899|2442310|2442584|N|N|N|N|N| +2442676|AAAAAAAAELFEFCAA|1975-09-20|908|3951|304|1975|6|9|20|3|1975|304|3951|Saturday|1975Q3|N|Y|N|2442657|2442899|2442311|2442585|N|N|N|N|N| +2442677|AAAAAAAAFLFEFCAA|1975-09-21|908|3951|304|1975|0|9|21|3|1975|304|3951|Sunday|1975Q3|N|N|N|2442657|2442899|2442312|2442586|N|N|N|N|N| +2442678|AAAAAAAAGLFEFCAA|1975-09-22|908|3951|304|1975|1|9|22|3|1975|304|3951|Monday|1975Q3|N|N|N|2442657|2442899|2442313|2442587|N|N|N|N|N| +2442679|AAAAAAAAHLFEFCAA|1975-09-23|908|3952|304|1975|2|9|23|3|1975|304|3952|Tuesday|1975Q3|N|N|N|2442657|2442899|2442314|2442588|N|N|N|N|N| +2442680|AAAAAAAAILFEFCAA|1975-09-24|908|3952|304|1975|3|9|24|3|1975|304|3952|Wednesday|1975Q3|N|N|N|2442657|2442899|2442315|2442589|N|N|N|N|N| +2442681|AAAAAAAAJLFEFCAA|1975-09-25|908|3952|304|1975|4|9|25|3|1975|304|3952|Thursday|1975Q3|N|N|N|2442657|2442899|2442316|2442590|N|N|N|N|N| +2442682|AAAAAAAAKLFEFCAA|1975-09-26|908|3952|304|1975|5|9|26|3|1975|304|3952|Friday|1975Q3|N|Y|N|2442657|2442899|2442317|2442591|N|N|N|N|N| +2442683|AAAAAAAALLFEFCAA|1975-09-27|908|3952|304|1975|6|9|27|3|1975|304|3952|Saturday|1975Q3|N|Y|N|2442657|2442899|2442318|2442592|N|N|N|N|N| +2442684|AAAAAAAAMLFEFCAA|1975-09-28|908|3952|304|1975|0|9|28|3|1975|304|3952|Sunday|1975Q3|N|N|N|2442657|2442899|2442319|2442593|N|N|N|N|N| +2442685|AAAAAAAANLFEFCAA|1975-09-29|908|3952|304|1975|1|9|29|3|1975|304|3952|Monday|1975Q3|N|N|N|2442657|2442899|2442320|2442594|N|N|N|N|N| +2442686|AAAAAAAAOLFEFCAA|1975-09-30|908|3953|304|1975|2|9|30|3|1975|304|3953|Tuesday|1975Q3|N|N|N|2442657|2442899|2442321|2442595|N|N|N|N|N| +2442687|AAAAAAAAPLFEFCAA|1975-10-01|909|3953|304|1975|3|10|1|3|1975|304|3953|Wednesday|1975Q3|N|N|N|2442687|2442959|2442322|2442595|N|N|N|N|N| +2442688|AAAAAAAAAMFEFCAA|1975-10-02|909|3953|304|1975|4|10|2|4|1975|304|3953|Thursday|1975Q4|N|N|N|2442687|2442959|2442323|2442596|N|N|N|N|N| +2442689|AAAAAAAABMFEFCAA|1975-10-03|909|3953|304|1975|5|10|3|4|1975|304|3953|Friday|1975Q4|N|Y|N|2442687|2442959|2442324|2442597|N|N|N|N|N| +2442690|AAAAAAAACMFEFCAA|1975-10-04|909|3953|304|1975|6|10|4|4|1975|304|3953|Saturday|1975Q4|N|Y|N|2442687|2442959|2442325|2442598|N|N|N|N|N| +2442691|AAAAAAAADMFEFCAA|1975-10-05|909|3953|304|1975|0|10|5|4|1975|304|3953|Sunday|1975Q4|N|N|N|2442687|2442959|2442326|2442599|N|N|N|N|N| +2442692|AAAAAAAAEMFEFCAA|1975-10-06|909|3953|304|1975|1|10|6|4|1975|304|3953|Monday|1975Q4|N|N|N|2442687|2442959|2442327|2442600|N|N|N|N|N| +2442693|AAAAAAAAFMFEFCAA|1975-10-07|909|3954|304|1975|2|10|7|4|1975|304|3954|Tuesday|1975Q4|N|N|N|2442687|2442959|2442328|2442601|N|N|N|N|N| +2442694|AAAAAAAAGMFEFCAA|1975-10-08|909|3954|304|1975|3|10|8|4|1975|304|3954|Wednesday|1975Q4|N|N|N|2442687|2442959|2442329|2442602|N|N|N|N|N| +2442695|AAAAAAAAHMFEFCAA|1975-10-09|909|3954|304|1975|4|10|9|4|1975|304|3954|Thursday|1975Q4|N|N|N|2442687|2442959|2442330|2442603|N|N|N|N|N| +2442696|AAAAAAAAIMFEFCAA|1975-10-10|909|3954|304|1975|5|10|10|4|1975|304|3954|Friday|1975Q4|N|Y|N|2442687|2442959|2442331|2442604|N|N|N|N|N| +2442697|AAAAAAAAJMFEFCAA|1975-10-11|909|3954|304|1975|6|10|11|4|1975|304|3954|Saturday|1975Q4|N|Y|N|2442687|2442959|2442332|2442605|N|N|N|N|N| +2442698|AAAAAAAAKMFEFCAA|1975-10-12|909|3954|304|1975|0|10|12|4|1975|304|3954|Sunday|1975Q4|N|N|N|2442687|2442959|2442333|2442606|N|N|N|N|N| +2442699|AAAAAAAALMFEFCAA|1975-10-13|909|3954|304|1975|1|10|13|4|1975|304|3954|Monday|1975Q4|N|N|N|2442687|2442959|2442334|2442607|N|N|N|N|N| +2442700|AAAAAAAAMMFEFCAA|1975-10-14|909|3955|304|1975|2|10|14|4|1975|304|3955|Tuesday|1975Q4|N|N|N|2442687|2442959|2442335|2442608|N|N|N|N|N| +2442701|AAAAAAAANMFEFCAA|1975-10-15|909|3955|304|1975|3|10|15|4|1975|304|3955|Wednesday|1975Q4|N|N|N|2442687|2442959|2442336|2442609|N|N|N|N|N| +2442702|AAAAAAAAOMFEFCAA|1975-10-16|909|3955|304|1975|4|10|16|4|1975|304|3955|Thursday|1975Q4|N|N|N|2442687|2442959|2442337|2442610|N|N|N|N|N| +2442703|AAAAAAAAPMFEFCAA|1975-10-17|909|3955|304|1975|5|10|17|4|1975|304|3955|Friday|1975Q4|N|Y|N|2442687|2442959|2442338|2442611|N|N|N|N|N| +2442704|AAAAAAAAANFEFCAA|1975-10-18|909|3955|304|1975|6|10|18|4|1975|304|3955|Saturday|1975Q4|N|Y|N|2442687|2442959|2442339|2442612|N|N|N|N|N| +2442705|AAAAAAAABNFEFCAA|1975-10-19|909|3955|304|1975|0|10|19|4|1975|304|3955|Sunday|1975Q4|N|N|N|2442687|2442959|2442340|2442613|N|N|N|N|N| +2442706|AAAAAAAACNFEFCAA|1975-10-20|909|3955|304|1975|1|10|20|4|1975|304|3955|Monday|1975Q4|N|N|N|2442687|2442959|2442341|2442614|N|N|N|N|N| +2442707|AAAAAAAADNFEFCAA|1975-10-21|909|3956|304|1975|2|10|21|4|1975|304|3956|Tuesday|1975Q4|N|N|N|2442687|2442959|2442342|2442615|N|N|N|N|N| +2442708|AAAAAAAAENFEFCAA|1975-10-22|909|3956|304|1975|3|10|22|4|1975|304|3956|Wednesday|1975Q4|N|N|N|2442687|2442959|2442343|2442616|N|N|N|N|N| +2442709|AAAAAAAAFNFEFCAA|1975-10-23|909|3956|304|1975|4|10|23|4|1975|304|3956|Thursday|1975Q4|N|N|N|2442687|2442959|2442344|2442617|N|N|N|N|N| +2442710|AAAAAAAAGNFEFCAA|1975-10-24|909|3956|304|1975|5|10|24|4|1975|304|3956|Friday|1975Q4|N|Y|N|2442687|2442959|2442345|2442618|N|N|N|N|N| +2442711|AAAAAAAAHNFEFCAA|1975-10-25|909|3956|304|1975|6|10|25|4|1975|304|3956|Saturday|1975Q4|N|Y|N|2442687|2442959|2442346|2442619|N|N|N|N|N| +2442712|AAAAAAAAINFEFCAA|1975-10-26|909|3956|304|1975|0|10|26|4|1975|304|3956|Sunday|1975Q4|N|N|N|2442687|2442959|2442347|2442620|N|N|N|N|N| +2442713|AAAAAAAAJNFEFCAA|1975-10-27|909|3956|304|1975|1|10|27|4|1975|304|3956|Monday|1975Q4|N|N|N|2442687|2442959|2442348|2442621|N|N|N|N|N| +2442714|AAAAAAAAKNFEFCAA|1975-10-28|909|3957|304|1975|2|10|28|4|1975|304|3957|Tuesday|1975Q4|N|N|N|2442687|2442959|2442349|2442622|N|N|N|N|N| +2442715|AAAAAAAALNFEFCAA|1975-10-29|909|3957|304|1975|3|10|29|4|1975|304|3957|Wednesday|1975Q4|N|N|N|2442687|2442959|2442350|2442623|N|N|N|N|N| +2442716|AAAAAAAAMNFEFCAA|1975-10-30|909|3957|304|1975|4|10|30|4|1975|304|3957|Thursday|1975Q4|N|N|N|2442687|2442959|2442351|2442624|N|N|N|N|N| +2442717|AAAAAAAANNFEFCAA|1975-10-31|909|3957|304|1975|5|10|31|4|1975|304|3957|Friday|1975Q4|N|Y|N|2442687|2442959|2442352|2442625|N|N|N|N|N| +2442718|AAAAAAAAONFEFCAA|1975-11-01|910|3957|304|1975|6|11|1|4|1975|304|3957|Saturday|1975Q4|N|Y|N|2442718|2443021|2442353|2442626|N|N|N|N|N| +2442719|AAAAAAAAPNFEFCAA|1975-11-02|910|3957|304|1975|0|11|2|4|1975|304|3957|Sunday|1975Q4|N|N|N|2442718|2443021|2442354|2442627|N|N|N|N|N| +2442720|AAAAAAAAAOFEFCAA|1975-11-03|910|3957|304|1975|1|11|3|4|1975|304|3957|Monday|1975Q4|N|N|N|2442718|2443021|2442355|2442628|N|N|N|N|N| +2442721|AAAAAAAABOFEFCAA|1975-11-04|910|3958|304|1975|2|11|4|4|1975|304|3958|Tuesday|1975Q4|N|N|N|2442718|2443021|2442356|2442629|N|N|N|N|N| +2442722|AAAAAAAACOFEFCAA|1975-11-05|910|3958|304|1975|3|11|5|4|1975|304|3958|Wednesday|1975Q4|N|N|N|2442718|2443021|2442357|2442630|N|N|N|N|N| +2442723|AAAAAAAADOFEFCAA|1975-11-06|910|3958|304|1975|4|11|6|4|1975|304|3958|Thursday|1975Q4|N|N|N|2442718|2443021|2442358|2442631|N|N|N|N|N| +2442724|AAAAAAAAEOFEFCAA|1975-11-07|910|3958|304|1975|5|11|7|4|1975|304|3958|Friday|1975Q4|N|Y|N|2442718|2443021|2442359|2442632|N|N|N|N|N| +2442725|AAAAAAAAFOFEFCAA|1975-11-08|910|3958|304|1975|6|11|8|4|1975|304|3958|Saturday|1975Q4|N|Y|N|2442718|2443021|2442360|2442633|N|N|N|N|N| +2442726|AAAAAAAAGOFEFCAA|1975-11-09|910|3958|304|1975|0|11|9|4|1975|304|3958|Sunday|1975Q4|N|N|N|2442718|2443021|2442361|2442634|N|N|N|N|N| +2442727|AAAAAAAAHOFEFCAA|1975-11-10|910|3958|304|1975|1|11|10|4|1975|304|3958|Monday|1975Q4|N|N|N|2442718|2443021|2442362|2442635|N|N|N|N|N| +2442728|AAAAAAAAIOFEFCAA|1975-11-11|910|3959|304|1975|2|11|11|4|1975|304|3959|Tuesday|1975Q4|N|N|N|2442718|2443021|2442363|2442636|N|N|N|N|N| +2442729|AAAAAAAAJOFEFCAA|1975-11-12|910|3959|304|1975|3|11|12|4|1975|304|3959|Wednesday|1975Q4|N|N|N|2442718|2443021|2442364|2442637|N|N|N|N|N| +2442730|AAAAAAAAKOFEFCAA|1975-11-13|910|3959|304|1975|4|11|13|4|1975|304|3959|Thursday|1975Q4|N|N|N|2442718|2443021|2442365|2442638|N|N|N|N|N| +2442731|AAAAAAAALOFEFCAA|1975-11-14|910|3959|304|1975|5|11|14|4|1975|304|3959|Friday|1975Q4|N|Y|N|2442718|2443021|2442366|2442639|N|N|N|N|N| +2442732|AAAAAAAAMOFEFCAA|1975-11-15|910|3959|304|1975|6|11|15|4|1975|304|3959|Saturday|1975Q4|N|Y|N|2442718|2443021|2442367|2442640|N|N|N|N|N| +2442733|AAAAAAAANOFEFCAA|1975-11-16|910|3959|304|1975|0|11|16|4|1975|304|3959|Sunday|1975Q4|N|N|N|2442718|2443021|2442368|2442641|N|N|N|N|N| +2442734|AAAAAAAAOOFEFCAA|1975-11-17|910|3959|304|1975|1|11|17|4|1975|304|3959|Monday|1975Q4|N|N|N|2442718|2443021|2442369|2442642|N|N|N|N|N| +2442735|AAAAAAAAPOFEFCAA|1975-11-18|910|3960|304|1975|2|11|18|4|1975|304|3960|Tuesday|1975Q4|N|N|N|2442718|2443021|2442370|2442643|N|N|N|N|N| +2442736|AAAAAAAAAPFEFCAA|1975-11-19|910|3960|304|1975|3|11|19|4|1975|304|3960|Wednesday|1975Q4|N|N|N|2442718|2443021|2442371|2442644|N|N|N|N|N| +2442737|AAAAAAAABPFEFCAA|1975-11-20|910|3960|304|1975|4|11|20|4|1975|304|3960|Thursday|1975Q4|N|N|N|2442718|2443021|2442372|2442645|N|N|N|N|N| +2442738|AAAAAAAACPFEFCAA|1975-11-21|910|3960|304|1975|5|11|21|4|1975|304|3960|Friday|1975Q4|N|Y|N|2442718|2443021|2442373|2442646|N|N|N|N|N| +2442739|AAAAAAAADPFEFCAA|1975-11-22|910|3960|304|1975|6|11|22|4|1975|304|3960|Saturday|1975Q4|N|Y|N|2442718|2443021|2442374|2442647|N|N|N|N|N| +2442740|AAAAAAAAEPFEFCAA|1975-11-23|910|3960|304|1975|0|11|23|4|1975|304|3960|Sunday|1975Q4|N|N|N|2442718|2443021|2442375|2442648|N|N|N|N|N| +2442741|AAAAAAAAFPFEFCAA|1975-11-24|910|3960|304|1975|1|11|24|4|1975|304|3960|Monday|1975Q4|N|N|N|2442718|2443021|2442376|2442649|N|N|N|N|N| +2442742|AAAAAAAAGPFEFCAA|1975-11-25|910|3961|304|1975|2|11|25|4|1975|304|3961|Tuesday|1975Q4|N|N|N|2442718|2443021|2442377|2442650|N|N|N|N|N| +2442743|AAAAAAAAHPFEFCAA|1975-11-26|910|3961|304|1975|3|11|26|4|1975|304|3961|Wednesday|1975Q4|N|N|N|2442718|2443021|2442378|2442651|N|N|N|N|N| +2442744|AAAAAAAAIPFEFCAA|1975-11-27|910|3961|304|1975|4|11|27|4|1975|304|3961|Thursday|1975Q4|N|N|N|2442718|2443021|2442379|2442652|N|N|N|N|N| +2442745|AAAAAAAAJPFEFCAA|1975-11-28|910|3961|304|1975|5|11|28|4|1975|304|3961|Friday|1975Q4|N|Y|N|2442718|2443021|2442380|2442653|N|N|N|N|N| +2442746|AAAAAAAAKPFEFCAA|1975-11-29|910|3961|304|1975|6|11|29|4|1975|304|3961|Saturday|1975Q4|N|Y|N|2442718|2443021|2442381|2442654|N|N|N|N|N| +2442747|AAAAAAAALPFEFCAA|1975-11-30|910|3961|304|1975|0|11|30|4|1975|304|3961|Sunday|1975Q4|N|N|N|2442718|2443021|2442382|2442655|N|N|N|N|N| +2442748|AAAAAAAAMPFEFCAA|1975-12-01|911|3961|305|1975|1|12|1|4|1975|305|3961|Monday|1975Q4|N|N|N|2442748|2443081|2442383|2442656|N|N|N|N|N| +2442749|AAAAAAAANPFEFCAA|1975-12-02|911|3962|305|1975|2|12|2|4|1975|305|3962|Tuesday|1975Q4|N|N|N|2442748|2443081|2442384|2442657|N|N|N|N|N| +2442750|AAAAAAAAOPFEFCAA|1975-12-03|911|3962|305|1975|3|12|3|4|1975|305|3962|Wednesday|1975Q4|N|N|N|2442748|2443081|2442385|2442658|N|N|N|N|N| +2442751|AAAAAAAAPPFEFCAA|1975-12-04|911|3962|305|1975|4|12|4|4|1975|305|3962|Thursday|1975Q4|N|N|N|2442748|2443081|2442386|2442659|N|N|N|N|N| +2442752|AAAAAAAAAAGEFCAA|1975-12-05|911|3962|305|1975|5|12|5|4|1975|305|3962|Friday|1975Q4|N|Y|N|2442748|2443081|2442387|2442660|N|N|N|N|N| +2442753|AAAAAAAABAGEFCAA|1975-12-06|911|3962|305|1975|6|12|6|4|1975|305|3962|Saturday|1975Q4|N|Y|N|2442748|2443081|2442388|2442661|N|N|N|N|N| +2442754|AAAAAAAACAGEFCAA|1975-12-07|911|3962|305|1975|0|12|7|4|1975|305|3962|Sunday|1975Q4|N|N|N|2442748|2443081|2442389|2442662|N|N|N|N|N| +2442755|AAAAAAAADAGEFCAA|1975-12-08|911|3962|305|1975|1|12|8|4|1975|305|3962|Monday|1975Q4|N|N|N|2442748|2443081|2442390|2442663|N|N|N|N|N| +2442756|AAAAAAAAEAGEFCAA|1975-12-09|911|3963|305|1975|2|12|9|4|1975|305|3963|Tuesday|1975Q4|N|N|N|2442748|2443081|2442391|2442664|N|N|N|N|N| +2442757|AAAAAAAAFAGEFCAA|1975-12-10|911|3963|305|1975|3|12|10|4|1975|305|3963|Wednesday|1975Q4|N|N|N|2442748|2443081|2442392|2442665|N|N|N|N|N| +2442758|AAAAAAAAGAGEFCAA|1975-12-11|911|3963|305|1975|4|12|11|4|1975|305|3963|Thursday|1975Q4|N|N|N|2442748|2443081|2442393|2442666|N|N|N|N|N| +2442759|AAAAAAAAHAGEFCAA|1975-12-12|911|3963|305|1975|5|12|12|4|1975|305|3963|Friday|1975Q4|N|Y|N|2442748|2443081|2442394|2442667|N|N|N|N|N| +2442760|AAAAAAAAIAGEFCAA|1975-12-13|911|3963|305|1975|6|12|13|4|1975|305|3963|Saturday|1975Q4|N|Y|N|2442748|2443081|2442395|2442668|N|N|N|N|N| +2442761|AAAAAAAAJAGEFCAA|1975-12-14|911|3963|305|1975|0|12|14|4|1975|305|3963|Sunday|1975Q4|N|N|N|2442748|2443081|2442396|2442669|N|N|N|N|N| +2442762|AAAAAAAAKAGEFCAA|1975-12-15|911|3963|305|1975|1|12|15|4|1975|305|3963|Monday|1975Q4|N|N|N|2442748|2443081|2442397|2442670|N|N|N|N|N| +2442763|AAAAAAAALAGEFCAA|1975-12-16|911|3964|305|1975|2|12|16|4|1975|305|3964|Tuesday|1975Q4|N|N|N|2442748|2443081|2442398|2442671|N|N|N|N|N| +2442764|AAAAAAAAMAGEFCAA|1975-12-17|911|3964|305|1975|3|12|17|4|1975|305|3964|Wednesday|1975Q4|N|N|N|2442748|2443081|2442399|2442672|N|N|N|N|N| +2442765|AAAAAAAANAGEFCAA|1975-12-18|911|3964|305|1975|4|12|18|4|1975|305|3964|Thursday|1975Q4|N|N|N|2442748|2443081|2442400|2442673|N|N|N|N|N| +2442766|AAAAAAAAOAGEFCAA|1975-12-19|911|3964|305|1975|5|12|19|4|1975|305|3964|Friday|1975Q4|N|Y|N|2442748|2443081|2442401|2442674|N|N|N|N|N| +2442767|AAAAAAAAPAGEFCAA|1975-12-20|911|3964|305|1975|6|12|20|4|1975|305|3964|Saturday|1975Q4|N|Y|N|2442748|2443081|2442402|2442675|N|N|N|N|N| +2442768|AAAAAAAAABGEFCAA|1975-12-21|911|3964|305|1975|0|12|21|4|1975|305|3964|Sunday|1975Q4|N|N|N|2442748|2443081|2442403|2442676|N|N|N|N|N| +2442769|AAAAAAAABBGEFCAA|1975-12-22|911|3964|305|1975|1|12|22|4|1975|305|3964|Monday|1975Q4|N|N|N|2442748|2443081|2442404|2442677|N|N|N|N|N| +2442770|AAAAAAAACBGEFCAA|1975-12-23|911|3965|305|1975|2|12|23|4|1975|305|3965|Tuesday|1975Q4|N|N|N|2442748|2443081|2442405|2442678|N|N|N|N|N| +2442771|AAAAAAAADBGEFCAA|1975-12-24|911|3965|305|1975|3|12|24|4|1975|305|3965|Wednesday|1975Q4|N|N|N|2442748|2443081|2442406|2442679|N|N|N|N|N| +2442772|AAAAAAAAEBGEFCAA|1975-12-25|911|3965|305|1975|4|12|25|4|1975|305|3965|Thursday|1975Q4|N|N|N|2442748|2443081|2442407|2442680|N|N|N|N|N| +2442773|AAAAAAAAFBGEFCAA|1975-12-26|911|3965|305|1975|5|12|26|4|1975|305|3965|Friday|1975Q4|Y|Y|N|2442748|2443081|2442408|2442681|N|N|N|N|N| +2442774|AAAAAAAAGBGEFCAA|1975-12-27|911|3965|305|1975|6|12|27|4|1975|305|3965|Saturday|1975Q4|N|Y|Y|2442748|2443081|2442409|2442682|N|N|N|N|N| +2442775|AAAAAAAAHBGEFCAA|1975-12-28|911|3965|305|1975|0|12|28|4|1975|305|3965|Sunday|1975Q4|N|N|N|2442748|2443081|2442410|2442683|N|N|N|N|N| +2442776|AAAAAAAAIBGEFCAA|1975-12-29|911|3965|305|1975|1|12|29|4|1975|305|3965|Monday|1975Q4|N|N|N|2442748|2443081|2442411|2442684|N|N|N|N|N| +2442777|AAAAAAAAJBGEFCAA|1975-12-30|911|3966|305|1975|2|12|30|4|1975|305|3966|Tuesday|1975Q4|N|N|N|2442748|2443081|2442412|2442685|N|N|N|N|N| +2442778|AAAAAAAAKBGEFCAA|1975-12-31|911|3966|305|1975|3|12|31|4|1975|305|3966|Wednesday|1975Q4|N|N|N|2442748|2443081|2442413|2442686|N|N|N|N|N| +2442779|AAAAAAAALBGEFCAA|1976-01-01|912|3966|305|1976|4|1|1|1|1976|305|3966|Thursday|1976Q1|Y|N|N|2442779|2442778|2442414|2442687|N|N|N|N|N| +2442780|AAAAAAAAMBGEFCAA|1976-01-02|912|3966|305|1976|5|1|2|1|1976|305|3966|Friday|1976Q1|N|Y|Y|2442779|2442778|2442415|2442688|N|N|N|N|N| +2442781|AAAAAAAANBGEFCAA|1976-01-03|912|3966|305|1976|6|1|3|1|1976|305|3966|Saturday|1976Q1|N|Y|N|2442779|2442778|2442416|2442689|N|N|N|N|N| +2442782|AAAAAAAAOBGEFCAA|1976-01-04|912|3966|305|1976|0|1|4|1|1976|305|3966|Sunday|1976Q1|N|N|N|2442779|2442778|2442417|2442690|N|N|N|N|N| +2442783|AAAAAAAAPBGEFCAA|1976-01-05|912|3966|305|1976|1|1|5|1|1976|305|3966|Monday|1976Q1|N|N|N|2442779|2442778|2442418|2442691|N|N|N|N|N| +2442784|AAAAAAAAACGEFCAA|1976-01-06|912|3967|305|1976|2|1|6|1|1976|305|3967|Tuesday|1976Q1|N|N|N|2442779|2442778|2442419|2442692|N|N|N|N|N| +2442785|AAAAAAAABCGEFCAA|1976-01-07|912|3967|305|1976|3|1|7|1|1976|305|3967|Wednesday|1976Q1|N|N|N|2442779|2442778|2442420|2442693|N|N|N|N|N| +2442786|AAAAAAAACCGEFCAA|1976-01-08|912|3967|305|1976|4|1|8|1|1976|305|3967|Thursday|1976Q1|N|N|N|2442779|2442778|2442421|2442694|N|N|N|N|N| +2442787|AAAAAAAADCGEFCAA|1976-01-09|912|3967|305|1976|5|1|9|1|1976|305|3967|Friday|1976Q1|N|Y|N|2442779|2442778|2442422|2442695|N|N|N|N|N| +2442788|AAAAAAAAECGEFCAA|1976-01-10|912|3967|305|1976|6|1|10|1|1976|305|3967|Saturday|1976Q1|N|Y|N|2442779|2442778|2442423|2442696|N|N|N|N|N| +2442789|AAAAAAAAFCGEFCAA|1976-01-11|912|3967|305|1976|0|1|11|1|1976|305|3967|Sunday|1976Q1|N|N|N|2442779|2442778|2442424|2442697|N|N|N|N|N| +2442790|AAAAAAAAGCGEFCAA|1976-01-12|912|3967|305|1976|1|1|12|1|1976|305|3967|Monday|1976Q1|N|N|N|2442779|2442778|2442425|2442698|N|N|N|N|N| +2442791|AAAAAAAAHCGEFCAA|1976-01-13|912|3968|305|1976|2|1|13|1|1976|305|3968|Tuesday|1976Q1|N|N|N|2442779|2442778|2442426|2442699|N|N|N|N|N| +2442792|AAAAAAAAICGEFCAA|1976-01-14|912|3968|305|1976|3|1|14|1|1976|305|3968|Wednesday|1976Q1|N|N|N|2442779|2442778|2442427|2442700|N|N|N|N|N| +2442793|AAAAAAAAJCGEFCAA|1976-01-15|912|3968|305|1976|4|1|15|1|1976|305|3968|Thursday|1976Q1|N|N|N|2442779|2442778|2442428|2442701|N|N|N|N|N| +2442794|AAAAAAAAKCGEFCAA|1976-01-16|912|3968|305|1976|5|1|16|1|1976|305|3968|Friday|1976Q1|N|Y|N|2442779|2442778|2442429|2442702|N|N|N|N|N| +2442795|AAAAAAAALCGEFCAA|1976-01-17|912|3968|305|1976|6|1|17|1|1976|305|3968|Saturday|1976Q1|N|Y|N|2442779|2442778|2442430|2442703|N|N|N|N|N| +2442796|AAAAAAAAMCGEFCAA|1976-01-18|912|3968|305|1976|0|1|18|1|1976|305|3968|Sunday|1976Q1|N|N|N|2442779|2442778|2442431|2442704|N|N|N|N|N| +2442797|AAAAAAAANCGEFCAA|1976-01-19|912|3968|305|1976|1|1|19|1|1976|305|3968|Monday|1976Q1|N|N|N|2442779|2442778|2442432|2442705|N|N|N|N|N| +2442798|AAAAAAAAOCGEFCAA|1976-01-20|912|3969|305|1976|2|1|20|1|1976|305|3969|Tuesday|1976Q1|N|N|N|2442779|2442778|2442433|2442706|N|N|N|N|N| +2442799|AAAAAAAAPCGEFCAA|1976-01-21|912|3969|305|1976|3|1|21|1|1976|305|3969|Wednesday|1976Q1|N|N|N|2442779|2442778|2442434|2442707|N|N|N|N|N| +2442800|AAAAAAAAADGEFCAA|1976-01-22|912|3969|305|1976|4|1|22|1|1976|305|3969|Thursday|1976Q1|N|N|N|2442779|2442778|2442435|2442708|N|N|N|N|N| +2442801|AAAAAAAABDGEFCAA|1976-01-23|912|3969|305|1976|5|1|23|1|1976|305|3969|Friday|1976Q1|N|Y|N|2442779|2442778|2442436|2442709|N|N|N|N|N| +2442802|AAAAAAAACDGEFCAA|1976-01-24|912|3969|305|1976|6|1|24|1|1976|305|3969|Saturday|1976Q1|N|Y|N|2442779|2442778|2442437|2442710|N|N|N|N|N| +2442803|AAAAAAAADDGEFCAA|1976-01-25|912|3969|305|1976|0|1|25|1|1976|305|3969|Sunday|1976Q1|N|N|N|2442779|2442778|2442438|2442711|N|N|N|N|N| +2442804|AAAAAAAAEDGEFCAA|1976-01-26|912|3969|305|1976|1|1|26|1|1976|305|3969|Monday|1976Q1|N|N|N|2442779|2442778|2442439|2442712|N|N|N|N|N| +2442805|AAAAAAAAFDGEFCAA|1976-01-27|912|3970|305|1976|2|1|27|1|1976|305|3970|Tuesday|1976Q1|N|N|N|2442779|2442778|2442440|2442713|N|N|N|N|N| +2442806|AAAAAAAAGDGEFCAA|1976-01-28|912|3970|305|1976|3|1|28|1|1976|305|3970|Wednesday|1976Q1|N|N|N|2442779|2442778|2442441|2442714|N|N|N|N|N| +2442807|AAAAAAAAHDGEFCAA|1976-01-29|912|3970|305|1976|4|1|29|1|1976|305|3970|Thursday|1976Q1|N|N|N|2442779|2442778|2442442|2442715|N|N|N|N|N| +2442808|AAAAAAAAIDGEFCAA|1976-01-30|912|3970|305|1976|5|1|30|1|1976|305|3970|Friday|1976Q1|N|Y|N|2442779|2442778|2442443|2442716|N|N|N|N|N| +2442809|AAAAAAAAJDGEFCAA|1976-01-31|912|3970|305|1976|6|1|31|1|1976|305|3970|Saturday|1976Q1|N|Y|N|2442779|2442778|2442444|2442717|N|N|N|N|N| +2442810|AAAAAAAAKDGEFCAA|1976-02-01|913|3970|305|1976|0|2|1|1|1976|305|3970|Sunday|1976Q1|N|N|N|2442810|2442840|2442445|2442718|N|N|N|N|N| +2442811|AAAAAAAALDGEFCAA|1976-02-02|913|3970|305|1976|1|2|2|1|1976|305|3970|Monday|1976Q1|N|N|N|2442810|2442840|2442446|2442719|N|N|N|N|N| +2442812|AAAAAAAAMDGEFCAA|1976-02-03|913|3971|305|1976|2|2|3|1|1976|305|3971|Tuesday|1976Q1|N|N|N|2442810|2442840|2442447|2442720|N|N|N|N|N| +2442813|AAAAAAAANDGEFCAA|1976-02-04|913|3971|305|1976|3|2|4|1|1976|305|3971|Wednesday|1976Q1|N|N|N|2442810|2442840|2442448|2442721|N|N|N|N|N| +2442814|AAAAAAAAODGEFCAA|1976-02-05|913|3971|305|1976|4|2|5|1|1976|305|3971|Thursday|1976Q1|N|N|N|2442810|2442840|2442449|2442722|N|N|N|N|N| +2442815|AAAAAAAAPDGEFCAA|1976-02-06|913|3971|305|1976|5|2|6|1|1976|305|3971|Friday|1976Q1|N|Y|N|2442810|2442840|2442450|2442723|N|N|N|N|N| +2442816|AAAAAAAAAEGEFCAA|1976-02-07|913|3971|305|1976|6|2|7|1|1976|305|3971|Saturday|1976Q1|N|Y|N|2442810|2442840|2442451|2442724|N|N|N|N|N| +2442817|AAAAAAAABEGEFCAA|1976-02-08|913|3971|305|1976|0|2|8|1|1976|305|3971|Sunday|1976Q1|N|N|N|2442810|2442840|2442452|2442725|N|N|N|N|N| +2442818|AAAAAAAACEGEFCAA|1976-02-09|913|3971|305|1976|1|2|9|1|1976|305|3971|Monday|1976Q1|N|N|N|2442810|2442840|2442453|2442726|N|N|N|N|N| +2442819|AAAAAAAADEGEFCAA|1976-02-10|913|3972|305|1976|2|2|10|1|1976|305|3972|Tuesday|1976Q1|N|N|N|2442810|2442840|2442454|2442727|N|N|N|N|N| +2442820|AAAAAAAAEEGEFCAA|1976-02-11|913|3972|305|1976|3|2|11|1|1976|305|3972|Wednesday|1976Q1|N|N|N|2442810|2442840|2442455|2442728|N|N|N|N|N| +2442821|AAAAAAAAFEGEFCAA|1976-02-12|913|3972|305|1976|4|2|12|1|1976|305|3972|Thursday|1976Q1|N|N|N|2442810|2442840|2442456|2442729|N|N|N|N|N| +2442822|AAAAAAAAGEGEFCAA|1976-02-13|913|3972|305|1976|5|2|13|1|1976|305|3972|Friday|1976Q1|N|Y|N|2442810|2442840|2442457|2442730|N|N|N|N|N| +2442823|AAAAAAAAHEGEFCAA|1976-02-14|913|3972|305|1976|6|2|14|1|1976|305|3972|Saturday|1976Q1|N|Y|N|2442810|2442840|2442458|2442731|N|N|N|N|N| +2442824|AAAAAAAAIEGEFCAA|1976-02-15|913|3972|305|1976|0|2|15|1|1976|305|3972|Sunday|1976Q1|N|N|N|2442810|2442840|2442459|2442732|N|N|N|N|N| +2442825|AAAAAAAAJEGEFCAA|1976-02-16|913|3972|305|1976|1|2|16|1|1976|305|3972|Monday|1976Q1|N|N|N|2442810|2442840|2442460|2442733|N|N|N|N|N| +2442826|AAAAAAAAKEGEFCAA|1976-02-17|913|3973|305|1976|2|2|17|1|1976|305|3973|Tuesday|1976Q1|N|N|N|2442810|2442840|2442461|2442734|N|N|N|N|N| +2442827|AAAAAAAALEGEFCAA|1976-02-18|913|3973|305|1976|3|2|18|1|1976|305|3973|Wednesday|1976Q1|N|N|N|2442810|2442840|2442462|2442735|N|N|N|N|N| +2442828|AAAAAAAAMEGEFCAA|1976-02-19|913|3973|305|1976|4|2|19|1|1976|305|3973|Thursday|1976Q1|N|N|N|2442810|2442840|2442463|2442736|N|N|N|N|N| +2442829|AAAAAAAANEGEFCAA|1976-02-20|913|3973|305|1976|5|2|20|1|1976|305|3973|Friday|1976Q1|N|Y|N|2442810|2442840|2442464|2442737|N|N|N|N|N| +2442830|AAAAAAAAOEGEFCAA|1976-02-21|913|3973|305|1976|6|2|21|1|1976|305|3973|Saturday|1976Q1|N|Y|N|2442810|2442840|2442465|2442738|N|N|N|N|N| +2442831|AAAAAAAAPEGEFCAA|1976-02-22|913|3973|305|1976|0|2|22|1|1976|305|3973|Sunday|1976Q1|N|N|N|2442810|2442840|2442466|2442739|N|N|N|N|N| +2442832|AAAAAAAAAFGEFCAA|1976-02-23|913|3973|305|1976|1|2|23|1|1976|305|3973|Monday|1976Q1|N|N|N|2442810|2442840|2442467|2442740|N|N|N|N|N| +2442833|AAAAAAAABFGEFCAA|1976-02-24|913|3974|305|1976|2|2|24|1|1976|305|3974|Tuesday|1976Q1|N|N|N|2442810|2442840|2442468|2442741|N|N|N|N|N| +2442834|AAAAAAAACFGEFCAA|1976-02-25|913|3974|305|1976|3|2|25|1|1976|305|3974|Wednesday|1976Q1|N|N|N|2442810|2442840|2442469|2442742|N|N|N|N|N| +2442835|AAAAAAAADFGEFCAA|1976-02-26|913|3974|305|1976|4|2|26|1|1976|305|3974|Thursday|1976Q1|N|N|N|2442810|2442840|2442470|2442743|N|N|N|N|N| +2442836|AAAAAAAAEFGEFCAA|1976-02-27|913|3974|305|1976|5|2|27|1|1976|305|3974|Friday|1976Q1|N|Y|N|2442810|2442840|2442471|2442744|N|N|N|N|N| +2442837|AAAAAAAAFFGEFCAA|1976-02-28|913|3974|305|1976|6|2|28|1|1976|305|3974|Saturday|1976Q1|N|Y|N|2442810|2442840|2442472|2442745|N|N|N|N|N| +2442838|AAAAAAAAGFGEFCAA|1976-02-29|913|3974|305|1976|0|2|29|1|1976|305|3974|Sunday|1976Q1|N|N|N|2442810|2442840|2442472|2442746|N|N|N|N|N| +2442839|AAAAAAAAHFGEFCAA|1976-03-01|914|3974|306|1976|1|3|1|1|1976|306|3974|Monday|1976Q1|N|N|N|2442839|2442898|2442473|2442747|N|N|N|N|N| +2442840|AAAAAAAAIFGEFCAA|1976-03-02|914|3975|306|1976|2|3|2|1|1976|306|3975|Tuesday|1976Q1|N|N|N|2442839|2442898|2442474|2442748|N|N|N|N|N| +2442841|AAAAAAAAJFGEFCAA|1976-03-03|914|3975|306|1976|3|3|3|1|1976|306|3975|Wednesday|1976Q1|N|N|N|2442839|2442898|2442475|2442749|N|N|N|N|N| +2442842|AAAAAAAAKFGEFCAA|1976-03-04|914|3975|306|1976|4|3|4|1|1976|306|3975|Thursday|1976Q1|N|N|N|2442839|2442898|2442476|2442750|N|N|N|N|N| +2442843|AAAAAAAALFGEFCAA|1976-03-05|914|3975|306|1976|5|3|5|1|1976|306|3975|Friday|1976Q1|N|Y|N|2442839|2442898|2442477|2442751|N|N|N|N|N| +2442844|AAAAAAAAMFGEFCAA|1976-03-06|914|3975|306|1976|6|3|6|1|1976|306|3975|Saturday|1976Q1|N|Y|N|2442839|2442898|2442478|2442752|N|N|N|N|N| +2442845|AAAAAAAANFGEFCAA|1976-03-07|914|3975|306|1976|0|3|7|1|1976|306|3975|Sunday|1976Q1|N|N|N|2442839|2442898|2442479|2442753|N|N|N|N|N| +2442846|AAAAAAAAOFGEFCAA|1976-03-08|914|3975|306|1976|1|3|8|1|1976|306|3975|Monday|1976Q1|N|N|N|2442839|2442898|2442480|2442754|N|N|N|N|N| +2442847|AAAAAAAAPFGEFCAA|1976-03-09|914|3976|306|1976|2|3|9|1|1976|306|3976|Tuesday|1976Q1|N|N|N|2442839|2442898|2442481|2442755|N|N|N|N|N| +2442848|AAAAAAAAAGGEFCAA|1976-03-10|914|3976|306|1976|3|3|10|1|1976|306|3976|Wednesday|1976Q1|N|N|N|2442839|2442898|2442482|2442756|N|N|N|N|N| +2442849|AAAAAAAABGGEFCAA|1976-03-11|914|3976|306|1976|4|3|11|1|1976|306|3976|Thursday|1976Q1|N|N|N|2442839|2442898|2442483|2442757|N|N|N|N|N| +2442850|AAAAAAAACGGEFCAA|1976-03-12|914|3976|306|1976|5|3|12|1|1976|306|3976|Friday|1976Q1|N|Y|N|2442839|2442898|2442484|2442758|N|N|N|N|N| +2442851|AAAAAAAADGGEFCAA|1976-03-13|914|3976|306|1976|6|3|13|1|1976|306|3976|Saturday|1976Q1|N|Y|N|2442839|2442898|2442485|2442759|N|N|N|N|N| +2442852|AAAAAAAAEGGEFCAA|1976-03-14|914|3976|306|1976|0|3|14|1|1976|306|3976|Sunday|1976Q1|N|N|N|2442839|2442898|2442486|2442760|N|N|N|N|N| +2442853|AAAAAAAAFGGEFCAA|1976-03-15|914|3976|306|1976|1|3|15|1|1976|306|3976|Monday|1976Q1|N|N|N|2442839|2442898|2442487|2442761|N|N|N|N|N| +2442854|AAAAAAAAGGGEFCAA|1976-03-16|914|3977|306|1976|2|3|16|1|1976|306|3977|Tuesday|1976Q1|N|N|N|2442839|2442898|2442488|2442762|N|N|N|N|N| +2442855|AAAAAAAAHGGEFCAA|1976-03-17|914|3977|306|1976|3|3|17|1|1976|306|3977|Wednesday|1976Q1|N|N|N|2442839|2442898|2442489|2442763|N|N|N|N|N| +2442856|AAAAAAAAIGGEFCAA|1976-03-18|914|3977|306|1976|4|3|18|1|1976|306|3977|Thursday|1976Q1|N|N|N|2442839|2442898|2442490|2442764|N|N|N|N|N| +2442857|AAAAAAAAJGGEFCAA|1976-03-19|914|3977|306|1976|5|3|19|1|1976|306|3977|Friday|1976Q1|N|Y|N|2442839|2442898|2442491|2442765|N|N|N|N|N| +2442858|AAAAAAAAKGGEFCAA|1976-03-20|914|3977|306|1976|6|3|20|1|1976|306|3977|Saturday|1976Q1|N|Y|N|2442839|2442898|2442492|2442766|N|N|N|N|N| +2442859|AAAAAAAALGGEFCAA|1976-03-21|914|3977|306|1976|0|3|21|1|1976|306|3977|Sunday|1976Q1|N|N|N|2442839|2442898|2442493|2442767|N|N|N|N|N| +2442860|AAAAAAAAMGGEFCAA|1976-03-22|914|3977|306|1976|1|3|22|1|1976|306|3977|Monday|1976Q1|N|N|N|2442839|2442898|2442494|2442768|N|N|N|N|N| +2442861|AAAAAAAANGGEFCAA|1976-03-23|914|3978|306|1976|2|3|23|1|1976|306|3978|Tuesday|1976Q1|N|N|N|2442839|2442898|2442495|2442769|N|N|N|N|N| +2442862|AAAAAAAAOGGEFCAA|1976-03-24|914|3978|306|1976|3|3|24|1|1976|306|3978|Wednesday|1976Q1|N|N|N|2442839|2442898|2442496|2442770|N|N|N|N|N| +2442863|AAAAAAAAPGGEFCAA|1976-03-25|914|3978|306|1976|4|3|25|1|1976|306|3978|Thursday|1976Q1|N|N|N|2442839|2442898|2442497|2442771|N|N|N|N|N| +2442864|AAAAAAAAAHGEFCAA|1976-03-26|914|3978|306|1976|5|3|26|1|1976|306|3978|Friday|1976Q1|N|Y|N|2442839|2442898|2442498|2442772|N|N|N|N|N| +2442865|AAAAAAAABHGEFCAA|1976-03-27|914|3978|306|1976|6|3|27|1|1976|306|3978|Saturday|1976Q1|N|Y|N|2442839|2442898|2442499|2442773|N|N|N|N|N| +2442866|AAAAAAAACHGEFCAA|1976-03-28|914|3978|306|1976|0|3|28|1|1976|306|3978|Sunday|1976Q1|N|N|N|2442839|2442898|2442500|2442774|N|N|N|N|N| +2442867|AAAAAAAADHGEFCAA|1976-03-29|914|3978|306|1976|1|3|29|1|1976|306|3978|Monday|1976Q1|N|N|N|2442839|2442898|2442501|2442775|N|N|N|N|N| +2442868|AAAAAAAAEHGEFCAA|1976-03-30|914|3979|306|1976|2|3|30|1|1976|306|3979|Tuesday|1976Q1|N|N|N|2442839|2442898|2442502|2442776|N|N|N|N|N| +2442869|AAAAAAAAFHGEFCAA|1976-03-31|914|3979|306|1976|3|3|31|1|1976|306|3979|Wednesday|1976Q1|N|N|N|2442839|2442898|2442503|2442777|N|N|N|N|N| +2442870|AAAAAAAAGHGEFCAA|1976-04-01|915|3979|306|1976|4|4|1|2|1976|306|3979|Thursday|1976Q2|N|N|N|2442870|2442960|2442504|2442779|N|N|N|N|N| +2442871|AAAAAAAAHHGEFCAA|1976-04-02|915|3979|306|1976|5|4|2|2|1976|306|3979|Friday|1976Q2|N|Y|N|2442870|2442960|2442505|2442780|N|N|N|N|N| +2442872|AAAAAAAAIHGEFCAA|1976-04-03|915|3979|306|1976|6|4|3|2|1976|306|3979|Saturday|1976Q2|N|Y|N|2442870|2442960|2442506|2442781|N|N|N|N|N| +2442873|AAAAAAAAJHGEFCAA|1976-04-04|915|3979|306|1976|0|4|4|2|1976|306|3979|Sunday|1976Q2|N|N|N|2442870|2442960|2442507|2442782|N|N|N|N|N| +2442874|AAAAAAAAKHGEFCAA|1976-04-05|915|3979|306|1976|1|4|5|2|1976|306|3979|Monday|1976Q2|N|N|N|2442870|2442960|2442508|2442783|N|N|N|N|N| +2442875|AAAAAAAALHGEFCAA|1976-04-06|915|3980|306|1976|2|4|6|2|1976|306|3980|Tuesday|1976Q2|N|N|N|2442870|2442960|2442509|2442784|N|N|N|N|N| +2442876|AAAAAAAAMHGEFCAA|1976-04-07|915|3980|306|1976|3|4|7|2|1976|306|3980|Wednesday|1976Q2|N|N|N|2442870|2442960|2442510|2442785|N|N|N|N|N| +2442877|AAAAAAAANHGEFCAA|1976-04-08|915|3980|306|1976|4|4|8|2|1976|306|3980|Thursday|1976Q2|N|N|N|2442870|2442960|2442511|2442786|N|N|N|N|N| +2442878|AAAAAAAAOHGEFCAA|1976-04-09|915|3980|306|1976|5|4|9|2|1976|306|3980|Friday|1976Q2|N|Y|N|2442870|2442960|2442512|2442787|N|N|N|N|N| +2442879|AAAAAAAAPHGEFCAA|1976-04-10|915|3980|306|1976|6|4|10|2|1976|306|3980|Saturday|1976Q2|N|Y|N|2442870|2442960|2442513|2442788|N|N|N|N|N| +2442880|AAAAAAAAAIGEFCAA|1976-04-11|915|3980|306|1976|0|4|11|2|1976|306|3980|Sunday|1976Q2|N|N|N|2442870|2442960|2442514|2442789|N|N|N|N|N| +2442881|AAAAAAAABIGEFCAA|1976-04-12|915|3980|306|1976|1|4|12|2|1976|306|3980|Monday|1976Q2|N|N|N|2442870|2442960|2442515|2442790|N|N|N|N|N| +2442882|AAAAAAAACIGEFCAA|1976-04-13|915|3981|306|1976|2|4|13|2|1976|306|3981|Tuesday|1976Q2|N|N|N|2442870|2442960|2442516|2442791|N|N|N|N|N| +2442883|AAAAAAAADIGEFCAA|1976-04-14|915|3981|306|1976|3|4|14|2|1976|306|3981|Wednesday|1976Q2|N|N|N|2442870|2442960|2442517|2442792|N|N|N|N|N| +2442884|AAAAAAAAEIGEFCAA|1976-04-15|915|3981|306|1976|4|4|15|2|1976|306|3981|Thursday|1976Q2|N|N|N|2442870|2442960|2442518|2442793|N|N|N|N|N| +2442885|AAAAAAAAFIGEFCAA|1976-04-16|915|3981|306|1976|5|4|16|2|1976|306|3981|Friday|1976Q2|N|Y|N|2442870|2442960|2442519|2442794|N|N|N|N|N| +2442886|AAAAAAAAGIGEFCAA|1976-04-17|915|3981|306|1976|6|4|17|2|1976|306|3981|Saturday|1976Q2|N|Y|N|2442870|2442960|2442520|2442795|N|N|N|N|N| +2442887|AAAAAAAAHIGEFCAA|1976-04-18|915|3981|306|1976|0|4|18|2|1976|306|3981|Sunday|1976Q2|N|N|N|2442870|2442960|2442521|2442796|N|N|N|N|N| +2442888|AAAAAAAAIIGEFCAA|1976-04-19|915|3981|306|1976|1|4|19|2|1976|306|3981|Monday|1976Q2|N|N|N|2442870|2442960|2442522|2442797|N|N|N|N|N| +2442889|AAAAAAAAJIGEFCAA|1976-04-20|915|3982|306|1976|2|4|20|2|1976|306|3982|Tuesday|1976Q2|N|N|N|2442870|2442960|2442523|2442798|N|N|N|N|N| +2442890|AAAAAAAAKIGEFCAA|1976-04-21|915|3982|306|1976|3|4|21|2|1976|306|3982|Wednesday|1976Q2|N|N|N|2442870|2442960|2442524|2442799|N|N|N|N|N| +2442891|AAAAAAAALIGEFCAA|1976-04-22|915|3982|306|1976|4|4|22|2|1976|306|3982|Thursday|1976Q2|N|N|N|2442870|2442960|2442525|2442800|N|N|N|N|N| +2442892|AAAAAAAAMIGEFCAA|1976-04-23|915|3982|306|1976|5|4|23|2|1976|306|3982|Friday|1976Q2|N|Y|N|2442870|2442960|2442526|2442801|N|N|N|N|N| +2442893|AAAAAAAANIGEFCAA|1976-04-24|915|3982|306|1976|6|4|24|2|1976|306|3982|Saturday|1976Q2|N|Y|N|2442870|2442960|2442527|2442802|N|N|N|N|N| +2442894|AAAAAAAAOIGEFCAA|1976-04-25|915|3982|306|1976|0|4|25|2|1976|306|3982|Sunday|1976Q2|N|N|N|2442870|2442960|2442528|2442803|N|N|N|N|N| +2442895|AAAAAAAAPIGEFCAA|1976-04-26|915|3982|306|1976|1|4|26|2|1976|306|3982|Monday|1976Q2|N|N|N|2442870|2442960|2442529|2442804|N|N|N|N|N| +2442896|AAAAAAAAAJGEFCAA|1976-04-27|915|3983|306|1976|2|4|27|2|1976|306|3983|Tuesday|1976Q2|N|N|N|2442870|2442960|2442530|2442805|N|N|N|N|N| +2442897|AAAAAAAABJGEFCAA|1976-04-28|915|3983|306|1976|3|4|28|2|1976|306|3983|Wednesday|1976Q2|N|N|N|2442870|2442960|2442531|2442806|N|N|N|N|N| +2442898|AAAAAAAACJGEFCAA|1976-04-29|915|3983|306|1976|4|4|29|2|1976|306|3983|Thursday|1976Q2|N|N|N|2442870|2442960|2442532|2442807|N|N|N|N|N| +2442899|AAAAAAAADJGEFCAA|1976-04-30|915|3983|306|1976|5|4|30|2|1976|306|3983|Friday|1976Q2|N|Y|N|2442870|2442960|2442533|2442808|N|N|N|N|N| +2442900|AAAAAAAAEJGEFCAA|1976-05-01|916|3983|306|1976|6|5|1|2|1976|306|3983|Saturday|1976Q2|N|Y|N|2442900|2443020|2442534|2442809|N|N|N|N|N| +2442901|AAAAAAAAFJGEFCAA|1976-05-02|916|3983|306|1976|0|5|2|2|1976|306|3983|Sunday|1976Q2|N|N|N|2442900|2443020|2442535|2442810|N|N|N|N|N| +2442902|AAAAAAAAGJGEFCAA|1976-05-03|916|3983|306|1976|1|5|3|2|1976|306|3983|Monday|1976Q2|N|N|N|2442900|2443020|2442536|2442811|N|N|N|N|N| +2442903|AAAAAAAAHJGEFCAA|1976-05-04|916|3984|306|1976|2|5|4|2|1976|306|3984|Tuesday|1976Q2|N|N|N|2442900|2443020|2442537|2442812|N|N|N|N|N| +2442904|AAAAAAAAIJGEFCAA|1976-05-05|916|3984|306|1976|3|5|5|2|1976|306|3984|Wednesday|1976Q2|N|N|N|2442900|2443020|2442538|2442813|N|N|N|N|N| +2442905|AAAAAAAAJJGEFCAA|1976-05-06|916|3984|306|1976|4|5|6|2|1976|306|3984|Thursday|1976Q2|N|N|N|2442900|2443020|2442539|2442814|N|N|N|N|N| +2442906|AAAAAAAAKJGEFCAA|1976-05-07|916|3984|306|1976|5|5|7|2|1976|306|3984|Friday|1976Q2|N|Y|N|2442900|2443020|2442540|2442815|N|N|N|N|N| +2442907|AAAAAAAALJGEFCAA|1976-05-08|916|3984|306|1976|6|5|8|2|1976|306|3984|Saturday|1976Q2|N|Y|N|2442900|2443020|2442541|2442816|N|N|N|N|N| +2442908|AAAAAAAAMJGEFCAA|1976-05-09|916|3984|306|1976|0|5|9|2|1976|306|3984|Sunday|1976Q2|N|N|N|2442900|2443020|2442542|2442817|N|N|N|N|N| +2442909|AAAAAAAANJGEFCAA|1976-05-10|916|3984|306|1976|1|5|10|2|1976|306|3984|Monday|1976Q2|N|N|N|2442900|2443020|2442543|2442818|N|N|N|N|N| +2442910|AAAAAAAAOJGEFCAA|1976-05-11|916|3985|306|1976|2|5|11|2|1976|306|3985|Tuesday|1976Q2|N|N|N|2442900|2443020|2442544|2442819|N|N|N|N|N| +2442911|AAAAAAAAPJGEFCAA|1976-05-12|916|3985|306|1976|3|5|12|2|1976|306|3985|Wednesday|1976Q2|N|N|N|2442900|2443020|2442545|2442820|N|N|N|N|N| +2442912|AAAAAAAAAKGEFCAA|1976-05-13|916|3985|306|1976|4|5|13|2|1976|306|3985|Thursday|1976Q2|N|N|N|2442900|2443020|2442546|2442821|N|N|N|N|N| +2442913|AAAAAAAABKGEFCAA|1976-05-14|916|3985|306|1976|5|5|14|2|1976|306|3985|Friday|1976Q2|N|Y|N|2442900|2443020|2442547|2442822|N|N|N|N|N| +2442914|AAAAAAAACKGEFCAA|1976-05-15|916|3985|306|1976|6|5|15|2|1976|306|3985|Saturday|1976Q2|N|Y|N|2442900|2443020|2442548|2442823|N|N|N|N|N| +2442915|AAAAAAAADKGEFCAA|1976-05-16|916|3985|306|1976|0|5|16|2|1976|306|3985|Sunday|1976Q2|N|N|N|2442900|2443020|2442549|2442824|N|N|N|N|N| +2442916|AAAAAAAAEKGEFCAA|1976-05-17|916|3985|306|1976|1|5|17|2|1976|306|3985|Monday|1976Q2|N|N|N|2442900|2443020|2442550|2442825|N|N|N|N|N| +2442917|AAAAAAAAFKGEFCAA|1976-05-18|916|3986|306|1976|2|5|18|2|1976|306|3986|Tuesday|1976Q2|N|N|N|2442900|2443020|2442551|2442826|N|N|N|N|N| +2442918|AAAAAAAAGKGEFCAA|1976-05-19|916|3986|306|1976|3|5|19|2|1976|306|3986|Wednesday|1976Q2|N|N|N|2442900|2443020|2442552|2442827|N|N|N|N|N| +2442919|AAAAAAAAHKGEFCAA|1976-05-20|916|3986|306|1976|4|5|20|2|1976|306|3986|Thursday|1976Q2|N|N|N|2442900|2443020|2442553|2442828|N|N|N|N|N| +2442920|AAAAAAAAIKGEFCAA|1976-05-21|916|3986|306|1976|5|5|21|2|1976|306|3986|Friday|1976Q2|N|Y|N|2442900|2443020|2442554|2442829|N|N|N|N|N| +2442921|AAAAAAAAJKGEFCAA|1976-05-22|916|3986|306|1976|6|5|22|2|1976|306|3986|Saturday|1976Q2|N|Y|N|2442900|2443020|2442555|2442830|N|N|N|N|N| +2442922|AAAAAAAAKKGEFCAA|1976-05-23|916|3986|306|1976|0|5|23|2|1976|306|3986|Sunday|1976Q2|N|N|N|2442900|2443020|2442556|2442831|N|N|N|N|N| +2442923|AAAAAAAALKGEFCAA|1976-05-24|916|3986|306|1976|1|5|24|2|1976|306|3986|Monday|1976Q2|N|N|N|2442900|2443020|2442557|2442832|N|N|N|N|N| +2442924|AAAAAAAAMKGEFCAA|1976-05-25|916|3987|306|1976|2|5|25|2|1976|306|3987|Tuesday|1976Q2|N|N|N|2442900|2443020|2442558|2442833|N|N|N|N|N| +2442925|AAAAAAAANKGEFCAA|1976-05-26|916|3987|306|1976|3|5|26|2|1976|306|3987|Wednesday|1976Q2|N|N|N|2442900|2443020|2442559|2442834|N|N|N|N|N| +2442926|AAAAAAAAOKGEFCAA|1976-05-27|916|3987|306|1976|4|5|27|2|1976|306|3987|Thursday|1976Q2|N|N|N|2442900|2443020|2442560|2442835|N|N|N|N|N| +2442927|AAAAAAAAPKGEFCAA|1976-05-28|916|3987|306|1976|5|5|28|2|1976|306|3987|Friday|1976Q2|N|Y|N|2442900|2443020|2442561|2442836|N|N|N|N|N| +2442928|AAAAAAAAALGEFCAA|1976-05-29|916|3987|306|1976|6|5|29|2|1976|306|3987|Saturday|1976Q2|N|Y|N|2442900|2443020|2442562|2442837|N|N|N|N|N| +2442929|AAAAAAAABLGEFCAA|1976-05-30|916|3987|306|1976|0|5|30|2|1976|306|3987|Sunday|1976Q2|N|N|N|2442900|2443020|2442563|2442838|N|N|N|N|N| +2442930|AAAAAAAACLGEFCAA|1976-05-31|916|3987|306|1976|1|5|31|2|1976|306|3987|Monday|1976Q2|N|N|N|2442900|2443020|2442564|2442839|N|N|N|N|N| +2442931|AAAAAAAADLGEFCAA|1976-06-01|917|3988|307|1976|2|6|1|2|1976|307|3988|Tuesday|1976Q2|N|N|N|2442931|2443082|2442565|2442840|N|N|N|N|N| +2442932|AAAAAAAAELGEFCAA|1976-06-02|917|3988|307|1976|3|6|2|2|1976|307|3988|Wednesday|1976Q2|N|N|N|2442931|2443082|2442566|2442841|N|N|N|N|N| +2442933|AAAAAAAAFLGEFCAA|1976-06-03|917|3988|307|1976|4|6|3|2|1976|307|3988|Thursday|1976Q2|N|N|N|2442931|2443082|2442567|2442842|N|N|N|N|N| +2442934|AAAAAAAAGLGEFCAA|1976-06-04|917|3988|307|1976|5|6|4|2|1976|307|3988|Friday|1976Q2|N|Y|N|2442931|2443082|2442568|2442843|N|N|N|N|N| +2442935|AAAAAAAAHLGEFCAA|1976-06-05|917|3988|307|1976|6|6|5|2|1976|307|3988|Saturday|1976Q2|N|Y|N|2442931|2443082|2442569|2442844|N|N|N|N|N| +2442936|AAAAAAAAILGEFCAA|1976-06-06|917|3988|307|1976|0|6|6|2|1976|307|3988|Sunday|1976Q2|N|N|N|2442931|2443082|2442570|2442845|N|N|N|N|N| +2442937|AAAAAAAAJLGEFCAA|1976-06-07|917|3988|307|1976|1|6|7|2|1976|307|3988|Monday|1976Q2|N|N|N|2442931|2443082|2442571|2442846|N|N|N|N|N| +2442938|AAAAAAAAKLGEFCAA|1976-06-08|917|3989|307|1976|2|6|8|2|1976|307|3989|Tuesday|1976Q2|N|N|N|2442931|2443082|2442572|2442847|N|N|N|N|N| +2442939|AAAAAAAALLGEFCAA|1976-06-09|917|3989|307|1976|3|6|9|2|1976|307|3989|Wednesday|1976Q2|N|N|N|2442931|2443082|2442573|2442848|N|N|N|N|N| +2442940|AAAAAAAAMLGEFCAA|1976-06-10|917|3989|307|1976|4|6|10|2|1976|307|3989|Thursday|1976Q2|N|N|N|2442931|2443082|2442574|2442849|N|N|N|N|N| +2442941|AAAAAAAANLGEFCAA|1976-06-11|917|3989|307|1976|5|6|11|2|1976|307|3989|Friday|1976Q2|N|Y|N|2442931|2443082|2442575|2442850|N|N|N|N|N| +2442942|AAAAAAAAOLGEFCAA|1976-06-12|917|3989|307|1976|6|6|12|2|1976|307|3989|Saturday|1976Q2|N|Y|N|2442931|2443082|2442576|2442851|N|N|N|N|N| +2442943|AAAAAAAAPLGEFCAA|1976-06-13|917|3989|307|1976|0|6|13|2|1976|307|3989|Sunday|1976Q2|N|N|N|2442931|2443082|2442577|2442852|N|N|N|N|N| +2442944|AAAAAAAAAMGEFCAA|1976-06-14|917|3989|307|1976|1|6|14|2|1976|307|3989|Monday|1976Q2|N|N|N|2442931|2443082|2442578|2442853|N|N|N|N|N| +2442945|AAAAAAAABMGEFCAA|1976-06-15|917|3990|307|1976|2|6|15|2|1976|307|3990|Tuesday|1976Q2|N|N|N|2442931|2443082|2442579|2442854|N|N|N|N|N| +2442946|AAAAAAAACMGEFCAA|1976-06-16|917|3990|307|1976|3|6|16|2|1976|307|3990|Wednesday|1976Q2|N|N|N|2442931|2443082|2442580|2442855|N|N|N|N|N| +2442947|AAAAAAAADMGEFCAA|1976-06-17|917|3990|307|1976|4|6|17|2|1976|307|3990|Thursday|1976Q2|N|N|N|2442931|2443082|2442581|2442856|N|N|N|N|N| +2442948|AAAAAAAAEMGEFCAA|1976-06-18|917|3990|307|1976|5|6|18|2|1976|307|3990|Friday|1976Q2|N|Y|N|2442931|2443082|2442582|2442857|N|N|N|N|N| +2442949|AAAAAAAAFMGEFCAA|1976-06-19|917|3990|307|1976|6|6|19|2|1976|307|3990|Saturday|1976Q2|N|Y|N|2442931|2443082|2442583|2442858|N|N|N|N|N| +2442950|AAAAAAAAGMGEFCAA|1976-06-20|917|3990|307|1976|0|6|20|2|1976|307|3990|Sunday|1976Q2|N|N|N|2442931|2443082|2442584|2442859|N|N|N|N|N| +2442951|AAAAAAAAHMGEFCAA|1976-06-21|917|3990|307|1976|1|6|21|2|1976|307|3990|Monday|1976Q2|N|N|N|2442931|2443082|2442585|2442860|N|N|N|N|N| +2442952|AAAAAAAAIMGEFCAA|1976-06-22|917|3991|307|1976|2|6|22|2|1976|307|3991|Tuesday|1976Q2|N|N|N|2442931|2443082|2442586|2442861|N|N|N|N|N| +2442953|AAAAAAAAJMGEFCAA|1976-06-23|917|3991|307|1976|3|6|23|2|1976|307|3991|Wednesday|1976Q2|N|N|N|2442931|2443082|2442587|2442862|N|N|N|N|N| +2442954|AAAAAAAAKMGEFCAA|1976-06-24|917|3991|307|1976|4|6|24|2|1976|307|3991|Thursday|1976Q2|N|N|N|2442931|2443082|2442588|2442863|N|N|N|N|N| +2442955|AAAAAAAALMGEFCAA|1976-06-25|917|3991|307|1976|5|6|25|2|1976|307|3991|Friday|1976Q2|N|Y|N|2442931|2443082|2442589|2442864|N|N|N|N|N| +2442956|AAAAAAAAMMGEFCAA|1976-06-26|917|3991|307|1976|6|6|26|2|1976|307|3991|Saturday|1976Q2|N|Y|N|2442931|2443082|2442590|2442865|N|N|N|N|N| +2442957|AAAAAAAANMGEFCAA|1976-06-27|917|3991|307|1976|0|6|27|2|1976|307|3991|Sunday|1976Q2|N|N|N|2442931|2443082|2442591|2442866|N|N|N|N|N| +2442958|AAAAAAAAOMGEFCAA|1976-06-28|917|3991|307|1976|1|6|28|2|1976|307|3991|Monday|1976Q2|N|N|N|2442931|2443082|2442592|2442867|N|N|N|N|N| +2442959|AAAAAAAAPMGEFCAA|1976-06-29|917|3992|307|1976|2|6|29|2|1976|307|3992|Tuesday|1976Q2|N|N|N|2442931|2443082|2442593|2442868|N|N|N|N|N| +2442960|AAAAAAAAANGEFCAA|1976-06-30|917|3992|307|1976|3|6|30|2|1976|307|3992|Wednesday|1976Q2|N|N|N|2442931|2443082|2442594|2442869|N|N|N|N|N| +2442961|AAAAAAAABNGEFCAA|1976-07-01|918|3992|307|1976|4|7|1|3|1976|307|3992|Thursday|1976Q3|N|N|N|2442961|2443142|2442595|2442870|N|N|N|N|N| +2442962|AAAAAAAACNGEFCAA|1976-07-02|918|3992|307|1976|5|7|2|3|1976|307|3992|Friday|1976Q3|N|Y|N|2442961|2443142|2442596|2442871|N|N|N|N|N| +2442963|AAAAAAAADNGEFCAA|1976-07-03|918|3992|307|1976|6|7|3|3|1976|307|3992|Saturday|1976Q3|N|Y|N|2442961|2443142|2442597|2442872|N|N|N|N|N| +2442964|AAAAAAAAENGEFCAA|1976-07-04|918|3992|307|1976|0|7|4|3|1976|307|3992|Sunday|1976Q3|Y|N|N|2442961|2443142|2442598|2442873|N|N|N|N|N| +2442965|AAAAAAAAFNGEFCAA|1976-07-05|918|3992|307|1976|1|7|5|3|1976|307|3992|Monday|1976Q3|N|N|Y|2442961|2443142|2442599|2442874|N|N|N|N|N| +2442966|AAAAAAAAGNGEFCAA|1976-07-06|918|3993|307|1976|2|7|6|3|1976|307|3993|Tuesday|1976Q3|N|N|N|2442961|2443142|2442600|2442875|N|N|N|N|N| +2442967|AAAAAAAAHNGEFCAA|1976-07-07|918|3993|307|1976|3|7|7|3|1976|307|3993|Wednesday|1976Q3|N|N|N|2442961|2443142|2442601|2442876|N|N|N|N|N| +2442968|AAAAAAAAINGEFCAA|1976-07-08|918|3993|307|1976|4|7|8|3|1976|307|3993|Thursday|1976Q3|N|N|N|2442961|2443142|2442602|2442877|N|N|N|N|N| +2442969|AAAAAAAAJNGEFCAA|1976-07-09|918|3993|307|1976|5|7|9|3|1976|307|3993|Friday|1976Q3|N|Y|N|2442961|2443142|2442603|2442878|N|N|N|N|N| +2442970|AAAAAAAAKNGEFCAA|1976-07-10|918|3993|307|1976|6|7|10|3|1976|307|3993|Saturday|1976Q3|N|Y|N|2442961|2443142|2442604|2442879|N|N|N|N|N| +2442971|AAAAAAAALNGEFCAA|1976-07-11|918|3993|307|1976|0|7|11|3|1976|307|3993|Sunday|1976Q3|N|N|N|2442961|2443142|2442605|2442880|N|N|N|N|N| +2442972|AAAAAAAAMNGEFCAA|1976-07-12|918|3993|307|1976|1|7|12|3|1976|307|3993|Monday|1976Q3|N|N|N|2442961|2443142|2442606|2442881|N|N|N|N|N| +2442973|AAAAAAAANNGEFCAA|1976-07-13|918|3994|307|1976|2|7|13|3|1976|307|3994|Tuesday|1976Q3|N|N|N|2442961|2443142|2442607|2442882|N|N|N|N|N| +2442974|AAAAAAAAONGEFCAA|1976-07-14|918|3994|307|1976|3|7|14|3|1976|307|3994|Wednesday|1976Q3|N|N|N|2442961|2443142|2442608|2442883|N|N|N|N|N| +2442975|AAAAAAAAPNGEFCAA|1976-07-15|918|3994|307|1976|4|7|15|3|1976|307|3994|Thursday|1976Q3|N|N|N|2442961|2443142|2442609|2442884|N|N|N|N|N| +2442976|AAAAAAAAAOGEFCAA|1976-07-16|918|3994|307|1976|5|7|16|3|1976|307|3994|Friday|1976Q3|N|Y|N|2442961|2443142|2442610|2442885|N|N|N|N|N| +2442977|AAAAAAAABOGEFCAA|1976-07-17|918|3994|307|1976|6|7|17|3|1976|307|3994|Saturday|1976Q3|N|Y|N|2442961|2443142|2442611|2442886|N|N|N|N|N| +2442978|AAAAAAAACOGEFCAA|1976-07-18|918|3994|307|1976|0|7|18|3|1976|307|3994|Sunday|1976Q3|N|N|N|2442961|2443142|2442612|2442887|N|N|N|N|N| +2442979|AAAAAAAADOGEFCAA|1976-07-19|918|3994|307|1976|1|7|19|3|1976|307|3994|Monday|1976Q3|N|N|N|2442961|2443142|2442613|2442888|N|N|N|N|N| +2442980|AAAAAAAAEOGEFCAA|1976-07-20|918|3995|307|1976|2|7|20|3|1976|307|3995|Tuesday|1976Q3|N|N|N|2442961|2443142|2442614|2442889|N|N|N|N|N| +2442981|AAAAAAAAFOGEFCAA|1976-07-21|918|3995|307|1976|3|7|21|3|1976|307|3995|Wednesday|1976Q3|N|N|N|2442961|2443142|2442615|2442890|N|N|N|N|N| +2442982|AAAAAAAAGOGEFCAA|1976-07-22|918|3995|307|1976|4|7|22|3|1976|307|3995|Thursday|1976Q3|N|N|N|2442961|2443142|2442616|2442891|N|N|N|N|N| +2442983|AAAAAAAAHOGEFCAA|1976-07-23|918|3995|307|1976|5|7|23|3|1976|307|3995|Friday|1976Q3|N|Y|N|2442961|2443142|2442617|2442892|N|N|N|N|N| +2442984|AAAAAAAAIOGEFCAA|1976-07-24|918|3995|307|1976|6|7|24|3|1976|307|3995|Saturday|1976Q3|N|Y|N|2442961|2443142|2442618|2442893|N|N|N|N|N| +2442985|AAAAAAAAJOGEFCAA|1976-07-25|918|3995|307|1976|0|7|25|3|1976|307|3995|Sunday|1976Q3|N|N|N|2442961|2443142|2442619|2442894|N|N|N|N|N| +2442986|AAAAAAAAKOGEFCAA|1976-07-26|918|3995|307|1976|1|7|26|3|1976|307|3995|Monday|1976Q3|N|N|N|2442961|2443142|2442620|2442895|N|N|N|N|N| +2442987|AAAAAAAALOGEFCAA|1976-07-27|918|3996|307|1976|2|7|27|3|1976|307|3996|Tuesday|1976Q3|N|N|N|2442961|2443142|2442621|2442896|N|N|N|N|N| +2442988|AAAAAAAAMOGEFCAA|1976-07-28|918|3996|307|1976|3|7|28|3|1976|307|3996|Wednesday|1976Q3|N|N|N|2442961|2443142|2442622|2442897|N|N|N|N|N| +2442989|AAAAAAAANOGEFCAA|1976-07-29|918|3996|307|1976|4|7|29|3|1976|307|3996|Thursday|1976Q3|N|N|N|2442961|2443142|2442623|2442898|N|N|N|N|N| +2442990|AAAAAAAAOOGEFCAA|1976-07-30|918|3996|307|1976|5|7|30|3|1976|307|3996|Friday|1976Q3|N|Y|N|2442961|2443142|2442624|2442899|N|N|N|N|N| +2442991|AAAAAAAAPOGEFCAA|1976-07-31|918|3996|307|1976|6|7|31|3|1976|307|3996|Saturday|1976Q3|N|Y|N|2442961|2443142|2442625|2442900|N|N|N|N|N| +2442992|AAAAAAAAAPGEFCAA|1976-08-01|919|3996|307|1976|0|8|1|3|1976|307|3996|Sunday|1976Q3|N|N|N|2442992|2443204|2442626|2442901|N|N|N|N|N| +2442993|AAAAAAAABPGEFCAA|1976-08-02|919|3996|307|1976|1|8|2|3|1976|307|3996|Monday|1976Q3|N|N|N|2442992|2443204|2442627|2442902|N|N|N|N|N| +2442994|AAAAAAAACPGEFCAA|1976-08-03|919|3997|307|1976|2|8|3|3|1976|307|3997|Tuesday|1976Q3|N|N|N|2442992|2443204|2442628|2442903|N|N|N|N|N| +2442995|AAAAAAAADPGEFCAA|1976-08-04|919|3997|307|1976|3|8|4|3|1976|307|3997|Wednesday|1976Q3|N|N|N|2442992|2443204|2442629|2442904|N|N|N|N|N| +2442996|AAAAAAAAEPGEFCAA|1976-08-05|919|3997|307|1976|4|8|5|3|1976|307|3997|Thursday|1976Q3|N|N|N|2442992|2443204|2442630|2442905|N|N|N|N|N| +2442997|AAAAAAAAFPGEFCAA|1976-08-06|919|3997|307|1976|5|8|6|3|1976|307|3997|Friday|1976Q3|N|Y|N|2442992|2443204|2442631|2442906|N|N|N|N|N| +2442998|AAAAAAAAGPGEFCAA|1976-08-07|919|3997|307|1976|6|8|7|3|1976|307|3997|Saturday|1976Q3|N|Y|N|2442992|2443204|2442632|2442907|N|N|N|N|N| +2442999|AAAAAAAAHPGEFCAA|1976-08-08|919|3997|307|1976|0|8|8|3|1976|307|3997|Sunday|1976Q3|N|N|N|2442992|2443204|2442633|2442908|N|N|N|N|N| +2443000|AAAAAAAAIPGEFCAA|1976-08-09|919|3997|307|1976|1|8|9|3|1976|307|3997|Monday|1976Q3|N|N|N|2442992|2443204|2442634|2442909|N|N|N|N|N| +2443001|AAAAAAAAJPGEFCAA|1976-08-10|919|3998|307|1976|2|8|10|3|1976|307|3998|Tuesday|1976Q3|N|N|N|2442992|2443204|2442635|2442910|N|N|N|N|N| +2443002|AAAAAAAAKPGEFCAA|1976-08-11|919|3998|307|1976|3|8|11|3|1976|307|3998|Wednesday|1976Q3|N|N|N|2442992|2443204|2442636|2442911|N|N|N|N|N| +2443003|AAAAAAAALPGEFCAA|1976-08-12|919|3998|307|1976|4|8|12|3|1976|307|3998|Thursday|1976Q3|N|N|N|2442992|2443204|2442637|2442912|N|N|N|N|N| +2443004|AAAAAAAAMPGEFCAA|1976-08-13|919|3998|307|1976|5|8|13|3|1976|307|3998|Friday|1976Q3|N|Y|N|2442992|2443204|2442638|2442913|N|N|N|N|N| +2443005|AAAAAAAANPGEFCAA|1976-08-14|919|3998|307|1976|6|8|14|3|1976|307|3998|Saturday|1976Q3|N|Y|N|2442992|2443204|2442639|2442914|N|N|N|N|N| +2443006|AAAAAAAAOPGEFCAA|1976-08-15|919|3998|307|1976|0|8|15|3|1976|307|3998|Sunday|1976Q3|N|N|N|2442992|2443204|2442640|2442915|N|N|N|N|N| +2443007|AAAAAAAAPPGEFCAA|1976-08-16|919|3998|307|1976|1|8|16|3|1976|307|3998|Monday|1976Q3|N|N|N|2442992|2443204|2442641|2442916|N|N|N|N|N| +2443008|AAAAAAAAAAHEFCAA|1976-08-17|919|3999|307|1976|2|8|17|3|1976|307|3999|Tuesday|1976Q3|N|N|N|2442992|2443204|2442642|2442917|N|N|N|N|N| +2443009|AAAAAAAABAHEFCAA|1976-08-18|919|3999|307|1976|3|8|18|3|1976|307|3999|Wednesday|1976Q3|N|N|N|2442992|2443204|2442643|2442918|N|N|N|N|N| +2443010|AAAAAAAACAHEFCAA|1976-08-19|919|3999|307|1976|4|8|19|3|1976|307|3999|Thursday|1976Q3|N|N|N|2442992|2443204|2442644|2442919|N|N|N|N|N| +2443011|AAAAAAAADAHEFCAA|1976-08-20|919|3999|307|1976|5|8|20|3|1976|307|3999|Friday|1976Q3|N|Y|N|2442992|2443204|2442645|2442920|N|N|N|N|N| +2443012|AAAAAAAAEAHEFCAA|1976-08-21|919|3999|307|1976|6|8|21|3|1976|307|3999|Saturday|1976Q3|N|Y|N|2442992|2443204|2442646|2442921|N|N|N|N|N| +2443013|AAAAAAAAFAHEFCAA|1976-08-22|919|3999|307|1976|0|8|22|3|1976|307|3999|Sunday|1976Q3|N|N|N|2442992|2443204|2442647|2442922|N|N|N|N|N| +2443014|AAAAAAAAGAHEFCAA|1976-08-23|919|3999|307|1976|1|8|23|3|1976|307|3999|Monday|1976Q3|N|N|N|2442992|2443204|2442648|2442923|N|N|N|N|N| +2443015|AAAAAAAAHAHEFCAA|1976-08-24|919|4000|307|1976|2|8|24|3|1976|307|4000|Tuesday|1976Q3|N|N|N|2442992|2443204|2442649|2442924|N|N|N|N|N| +2443016|AAAAAAAAIAHEFCAA|1976-08-25|919|4000|307|1976|3|8|25|3|1976|307|4000|Wednesday|1976Q3|N|N|N|2442992|2443204|2442650|2442925|N|N|N|N|N| +2443017|AAAAAAAAJAHEFCAA|1976-08-26|919|4000|307|1976|4|8|26|3|1976|307|4000|Thursday|1976Q3|N|N|N|2442992|2443204|2442651|2442926|N|N|N|N|N| +2443018|AAAAAAAAKAHEFCAA|1976-08-27|919|4000|307|1976|5|8|27|3|1976|307|4000|Friday|1976Q3|N|Y|N|2442992|2443204|2442652|2442927|N|N|N|N|N| +2443019|AAAAAAAALAHEFCAA|1976-08-28|919|4000|307|1976|6|8|28|3|1976|307|4000|Saturday|1976Q3|N|Y|N|2442992|2443204|2442653|2442928|N|N|N|N|N| +2443020|AAAAAAAAMAHEFCAA|1976-08-29|919|4000|307|1976|0|8|29|3|1976|307|4000|Sunday|1976Q3|N|N|N|2442992|2443204|2442654|2442929|N|N|N|N|N| +2443021|AAAAAAAANAHEFCAA|1976-08-30|919|4000|307|1976|1|8|30|3|1976|307|4000|Monday|1976Q3|N|N|N|2442992|2443204|2442655|2442930|N|N|N|N|N| +2443022|AAAAAAAAOAHEFCAA|1976-08-31|919|4001|307|1976|2|8|31|3|1976|307|4001|Tuesday|1976Q3|N|N|N|2442992|2443204|2442656|2442931|N|N|N|N|N| +2443023|AAAAAAAAPAHEFCAA|1976-09-01|920|4001|308|1976|3|9|1|3|1976|308|4001|Wednesday|1976Q3|N|N|N|2443023|2443266|2442657|2442932|N|N|N|N|N| +2443024|AAAAAAAAABHEFCAA|1976-09-02|920|4001|308|1976|4|9|2|3|1976|308|4001|Thursday|1976Q3|N|N|N|2443023|2443266|2442658|2442933|N|N|N|N|N| +2443025|AAAAAAAABBHEFCAA|1976-09-03|920|4001|308|1976|5|9|3|3|1976|308|4001|Friday|1976Q3|N|Y|N|2443023|2443266|2442659|2442934|N|N|N|N|N| +2443026|AAAAAAAACBHEFCAA|1976-09-04|920|4001|308|1976|6|9|4|3|1976|308|4001|Saturday|1976Q3|N|Y|N|2443023|2443266|2442660|2442935|N|N|N|N|N| +2443027|AAAAAAAADBHEFCAA|1976-09-05|920|4001|308|1976|0|9|5|3|1976|308|4001|Sunday|1976Q3|N|N|N|2443023|2443266|2442661|2442936|N|N|N|N|N| +2443028|AAAAAAAAEBHEFCAA|1976-09-06|920|4001|308|1976|1|9|6|3|1976|308|4001|Monday|1976Q3|N|N|N|2443023|2443266|2442662|2442937|N|N|N|N|N| +2443029|AAAAAAAAFBHEFCAA|1976-09-07|920|4002|308|1976|2|9|7|3|1976|308|4002|Tuesday|1976Q3|N|N|N|2443023|2443266|2442663|2442938|N|N|N|N|N| +2443030|AAAAAAAAGBHEFCAA|1976-09-08|920|4002|308|1976|3|9|8|3|1976|308|4002|Wednesday|1976Q3|N|N|N|2443023|2443266|2442664|2442939|N|N|N|N|N| +2443031|AAAAAAAAHBHEFCAA|1976-09-09|920|4002|308|1976|4|9|9|3|1976|308|4002|Thursday|1976Q3|N|N|N|2443023|2443266|2442665|2442940|N|N|N|N|N| +2443032|AAAAAAAAIBHEFCAA|1976-09-10|920|4002|308|1976|5|9|10|3|1976|308|4002|Friday|1976Q3|N|Y|N|2443023|2443266|2442666|2442941|N|N|N|N|N| +2443033|AAAAAAAAJBHEFCAA|1976-09-11|920|4002|308|1976|6|9|11|3|1976|308|4002|Saturday|1976Q3|N|Y|N|2443023|2443266|2442667|2442942|N|N|N|N|N| +2443034|AAAAAAAAKBHEFCAA|1976-09-12|920|4002|308|1976|0|9|12|3|1976|308|4002|Sunday|1976Q3|N|N|N|2443023|2443266|2442668|2442943|N|N|N|N|N| +2443035|AAAAAAAALBHEFCAA|1976-09-13|920|4002|308|1976|1|9|13|3|1976|308|4002|Monday|1976Q3|N|N|N|2443023|2443266|2442669|2442944|N|N|N|N|N| +2443036|AAAAAAAAMBHEFCAA|1976-09-14|920|4003|308|1976|2|9|14|3|1976|308|4003|Tuesday|1976Q3|N|N|N|2443023|2443266|2442670|2442945|N|N|N|N|N| +2443037|AAAAAAAANBHEFCAA|1976-09-15|920|4003|308|1976|3|9|15|3|1976|308|4003|Wednesday|1976Q3|N|N|N|2443023|2443266|2442671|2442946|N|N|N|N|N| +2443038|AAAAAAAAOBHEFCAA|1976-09-16|920|4003|308|1976|4|9|16|3|1976|308|4003|Thursday|1976Q3|N|N|N|2443023|2443266|2442672|2442947|N|N|N|N|N| +2443039|AAAAAAAAPBHEFCAA|1976-09-17|920|4003|308|1976|5|9|17|3|1976|308|4003|Friday|1976Q3|N|Y|N|2443023|2443266|2442673|2442948|N|N|N|N|N| +2443040|AAAAAAAAACHEFCAA|1976-09-18|920|4003|308|1976|6|9|18|3|1976|308|4003|Saturday|1976Q3|N|Y|N|2443023|2443266|2442674|2442949|N|N|N|N|N| +2443041|AAAAAAAABCHEFCAA|1976-09-19|920|4003|308|1976|0|9|19|3|1976|308|4003|Sunday|1976Q3|N|N|N|2443023|2443266|2442675|2442950|N|N|N|N|N| +2443042|AAAAAAAACCHEFCAA|1976-09-20|920|4003|308|1976|1|9|20|3|1976|308|4003|Monday|1976Q3|N|N|N|2443023|2443266|2442676|2442951|N|N|N|N|N| +2443043|AAAAAAAADCHEFCAA|1976-09-21|920|4004|308|1976|2|9|21|3|1976|308|4004|Tuesday|1976Q3|N|N|N|2443023|2443266|2442677|2442952|N|N|N|N|N| +2443044|AAAAAAAAECHEFCAA|1976-09-22|920|4004|308|1976|3|9|22|3|1976|308|4004|Wednesday|1976Q3|N|N|N|2443023|2443266|2442678|2442953|N|N|N|N|N| +2443045|AAAAAAAAFCHEFCAA|1976-09-23|920|4004|308|1976|4|9|23|3|1976|308|4004|Thursday|1976Q3|N|N|N|2443023|2443266|2442679|2442954|N|N|N|N|N| +2443046|AAAAAAAAGCHEFCAA|1976-09-24|920|4004|308|1976|5|9|24|3|1976|308|4004|Friday|1976Q3|N|Y|N|2443023|2443266|2442680|2442955|N|N|N|N|N| +2443047|AAAAAAAAHCHEFCAA|1976-09-25|920|4004|308|1976|6|9|25|3|1976|308|4004|Saturday|1976Q3|N|Y|N|2443023|2443266|2442681|2442956|N|N|N|N|N| +2443048|AAAAAAAAICHEFCAA|1976-09-26|920|4004|308|1976|0|9|26|3|1976|308|4004|Sunday|1976Q3|N|N|N|2443023|2443266|2442682|2442957|N|N|N|N|N| +2443049|AAAAAAAAJCHEFCAA|1976-09-27|920|4004|308|1976|1|9|27|3|1976|308|4004|Monday|1976Q3|N|N|N|2443023|2443266|2442683|2442958|N|N|N|N|N| +2443050|AAAAAAAAKCHEFCAA|1976-09-28|920|4005|308|1976|2|9|28|3|1976|308|4005|Tuesday|1976Q3|N|N|N|2443023|2443266|2442684|2442959|N|N|N|N|N| +2443051|AAAAAAAALCHEFCAA|1976-09-29|920|4005|308|1976|3|9|29|3|1976|308|4005|Wednesday|1976Q3|N|N|N|2443023|2443266|2442685|2442960|N|N|N|N|N| +2443052|AAAAAAAAMCHEFCAA|1976-09-30|920|4005|308|1976|4|9|30|3|1976|308|4005|Thursday|1976Q3|N|N|N|2443023|2443266|2442686|2442961|N|N|N|N|N| +2443053|AAAAAAAANCHEFCAA|1976-10-01|921|4005|308|1976|5|10|1|4|1976|308|4005|Friday|1976Q4|N|Y|N|2443053|2443326|2442687|2442961|N|N|N|N|N| +2443054|AAAAAAAAOCHEFCAA|1976-10-02|921|4005|308|1976|6|10|2|4|1976|308|4005|Saturday|1976Q4|N|Y|N|2443053|2443326|2442688|2442962|N|N|N|N|N| +2443055|AAAAAAAAPCHEFCAA|1976-10-03|921|4005|308|1976|0|10|3|4|1976|308|4005|Sunday|1976Q4|N|N|N|2443053|2443326|2442689|2442963|N|N|N|N|N| +2443056|AAAAAAAAADHEFCAA|1976-10-04|921|4005|308|1976|1|10|4|4|1976|308|4005|Monday|1976Q4|N|N|N|2443053|2443326|2442690|2442964|N|N|N|N|N| +2443057|AAAAAAAABDHEFCAA|1976-10-05|921|4006|308|1976|2|10|5|4|1976|308|4006|Tuesday|1976Q4|N|N|N|2443053|2443326|2442691|2442965|N|N|N|N|N| +2443058|AAAAAAAACDHEFCAA|1976-10-06|921|4006|308|1976|3|10|6|4|1976|308|4006|Wednesday|1976Q4|N|N|N|2443053|2443326|2442692|2442966|N|N|N|N|N| +2443059|AAAAAAAADDHEFCAA|1976-10-07|921|4006|308|1976|4|10|7|4|1976|308|4006|Thursday|1976Q4|N|N|N|2443053|2443326|2442693|2442967|N|N|N|N|N| +2443060|AAAAAAAAEDHEFCAA|1976-10-08|921|4006|308|1976|5|10|8|4|1976|308|4006|Friday|1976Q4|N|Y|N|2443053|2443326|2442694|2442968|N|N|N|N|N| +2443061|AAAAAAAAFDHEFCAA|1976-10-09|921|4006|308|1976|6|10|9|4|1976|308|4006|Saturday|1976Q4|N|Y|N|2443053|2443326|2442695|2442969|N|N|N|N|N| +2443062|AAAAAAAAGDHEFCAA|1976-10-10|921|4006|308|1976|0|10|10|4|1976|308|4006|Sunday|1976Q4|N|N|N|2443053|2443326|2442696|2442970|N|N|N|N|N| +2443063|AAAAAAAAHDHEFCAA|1976-10-11|921|4006|308|1976|1|10|11|4|1976|308|4006|Monday|1976Q4|N|N|N|2443053|2443326|2442697|2442971|N|N|N|N|N| +2443064|AAAAAAAAIDHEFCAA|1976-10-12|921|4007|308|1976|2|10|12|4|1976|308|4007|Tuesday|1976Q4|N|N|N|2443053|2443326|2442698|2442972|N|N|N|N|N| +2443065|AAAAAAAAJDHEFCAA|1976-10-13|921|4007|308|1976|3|10|13|4|1976|308|4007|Wednesday|1976Q4|N|N|N|2443053|2443326|2442699|2442973|N|N|N|N|N| +2443066|AAAAAAAAKDHEFCAA|1976-10-14|921|4007|308|1976|4|10|14|4|1976|308|4007|Thursday|1976Q4|N|N|N|2443053|2443326|2442700|2442974|N|N|N|N|N| +2443067|AAAAAAAALDHEFCAA|1976-10-15|921|4007|308|1976|5|10|15|4|1976|308|4007|Friday|1976Q4|N|Y|N|2443053|2443326|2442701|2442975|N|N|N|N|N| +2443068|AAAAAAAAMDHEFCAA|1976-10-16|921|4007|308|1976|6|10|16|4|1976|308|4007|Saturday|1976Q4|N|Y|N|2443053|2443326|2442702|2442976|N|N|N|N|N| +2443069|AAAAAAAANDHEFCAA|1976-10-17|921|4007|308|1976|0|10|17|4|1976|308|4007|Sunday|1976Q4|N|N|N|2443053|2443326|2442703|2442977|N|N|N|N|N| +2443070|AAAAAAAAODHEFCAA|1976-10-18|921|4007|308|1976|1|10|18|4|1976|308|4007|Monday|1976Q4|N|N|N|2443053|2443326|2442704|2442978|N|N|N|N|N| +2443071|AAAAAAAAPDHEFCAA|1976-10-19|921|4008|308|1976|2|10|19|4|1976|308|4008|Tuesday|1976Q4|N|N|N|2443053|2443326|2442705|2442979|N|N|N|N|N| +2443072|AAAAAAAAAEHEFCAA|1976-10-20|921|4008|308|1976|3|10|20|4|1976|308|4008|Wednesday|1976Q4|N|N|N|2443053|2443326|2442706|2442980|N|N|N|N|N| +2443073|AAAAAAAABEHEFCAA|1976-10-21|921|4008|308|1976|4|10|21|4|1976|308|4008|Thursday|1976Q4|N|N|N|2443053|2443326|2442707|2442981|N|N|N|N|N| +2443074|AAAAAAAACEHEFCAA|1976-10-22|921|4008|308|1976|5|10|22|4|1976|308|4008|Friday|1976Q4|N|Y|N|2443053|2443326|2442708|2442982|N|N|N|N|N| +2443075|AAAAAAAADEHEFCAA|1976-10-23|921|4008|308|1976|6|10|23|4|1976|308|4008|Saturday|1976Q4|N|Y|N|2443053|2443326|2442709|2442983|N|N|N|N|N| +2443076|AAAAAAAAEEHEFCAA|1976-10-24|921|4008|308|1976|0|10|24|4|1976|308|4008|Sunday|1976Q4|N|N|N|2443053|2443326|2442710|2442984|N|N|N|N|N| +2443077|AAAAAAAAFEHEFCAA|1976-10-25|921|4008|308|1976|1|10|25|4|1976|308|4008|Monday|1976Q4|N|N|N|2443053|2443326|2442711|2442985|N|N|N|N|N| +2443078|AAAAAAAAGEHEFCAA|1976-10-26|921|4009|308|1976|2|10|26|4|1976|308|4009|Tuesday|1976Q4|N|N|N|2443053|2443326|2442712|2442986|N|N|N|N|N| +2443079|AAAAAAAAHEHEFCAA|1976-10-27|921|4009|308|1976|3|10|27|4|1976|308|4009|Wednesday|1976Q4|N|N|N|2443053|2443326|2442713|2442987|N|N|N|N|N| +2443080|AAAAAAAAIEHEFCAA|1976-10-28|921|4009|308|1976|4|10|28|4|1976|308|4009|Thursday|1976Q4|N|N|N|2443053|2443326|2442714|2442988|N|N|N|N|N| +2443081|AAAAAAAAJEHEFCAA|1976-10-29|921|4009|308|1976|5|10|29|4|1976|308|4009|Friday|1976Q4|N|Y|N|2443053|2443326|2442715|2442989|N|N|N|N|N| +2443082|AAAAAAAAKEHEFCAA|1976-10-30|921|4009|308|1976|6|10|30|4|1976|308|4009|Saturday|1976Q4|N|Y|N|2443053|2443326|2442716|2442990|N|N|N|N|N| +2443083|AAAAAAAALEHEFCAA|1976-10-31|921|4009|308|1976|0|10|31|4|1976|308|4009|Sunday|1976Q4|N|N|N|2443053|2443326|2442717|2442991|N|N|N|N|N| +2443084|AAAAAAAAMEHEFCAA|1976-11-01|922|4009|308|1976|1|11|1|4|1976|308|4009|Monday|1976Q4|N|N|N|2443084|2443388|2442718|2442992|N|N|N|N|N| +2443085|AAAAAAAANEHEFCAA|1976-11-02|922|4010|308|1976|2|11|2|4|1976|308|4010|Tuesday|1976Q4|N|N|N|2443084|2443388|2442719|2442993|N|N|N|N|N| +2443086|AAAAAAAAOEHEFCAA|1976-11-03|922|4010|308|1976|3|11|3|4|1976|308|4010|Wednesday|1976Q4|N|N|N|2443084|2443388|2442720|2442994|N|N|N|N|N| +2443087|AAAAAAAAPEHEFCAA|1976-11-04|922|4010|308|1976|4|11|4|4|1976|308|4010|Thursday|1976Q4|N|N|N|2443084|2443388|2442721|2442995|N|N|N|N|N| +2443088|AAAAAAAAAFHEFCAA|1976-11-05|922|4010|308|1976|5|11|5|4|1976|308|4010|Friday|1976Q4|N|Y|N|2443084|2443388|2442722|2442996|N|N|N|N|N| +2443089|AAAAAAAABFHEFCAA|1976-11-06|922|4010|308|1976|6|11|6|4|1976|308|4010|Saturday|1976Q4|N|Y|N|2443084|2443388|2442723|2442997|N|N|N|N|N| +2443090|AAAAAAAACFHEFCAA|1976-11-07|922|4010|308|1976|0|11|7|4|1976|308|4010|Sunday|1976Q4|N|N|N|2443084|2443388|2442724|2442998|N|N|N|N|N| +2443091|AAAAAAAADFHEFCAA|1976-11-08|922|4010|308|1976|1|11|8|4|1976|308|4010|Monday|1976Q4|N|N|N|2443084|2443388|2442725|2442999|N|N|N|N|N| +2443092|AAAAAAAAEFHEFCAA|1976-11-09|922|4011|308|1976|2|11|9|4|1976|308|4011|Tuesday|1976Q4|N|N|N|2443084|2443388|2442726|2443000|N|N|N|N|N| +2443093|AAAAAAAAFFHEFCAA|1976-11-10|922|4011|308|1976|3|11|10|4|1976|308|4011|Wednesday|1976Q4|N|N|N|2443084|2443388|2442727|2443001|N|N|N|N|N| +2443094|AAAAAAAAGFHEFCAA|1976-11-11|922|4011|308|1976|4|11|11|4|1976|308|4011|Thursday|1976Q4|N|N|N|2443084|2443388|2442728|2443002|N|N|N|N|N| +2443095|AAAAAAAAHFHEFCAA|1976-11-12|922|4011|308|1976|5|11|12|4|1976|308|4011|Friday|1976Q4|N|Y|N|2443084|2443388|2442729|2443003|N|N|N|N|N| +2443096|AAAAAAAAIFHEFCAA|1976-11-13|922|4011|308|1976|6|11|13|4|1976|308|4011|Saturday|1976Q4|N|Y|N|2443084|2443388|2442730|2443004|N|N|N|N|N| +2443097|AAAAAAAAJFHEFCAA|1976-11-14|922|4011|308|1976|0|11|14|4|1976|308|4011|Sunday|1976Q4|N|N|N|2443084|2443388|2442731|2443005|N|N|N|N|N| +2443098|AAAAAAAAKFHEFCAA|1976-11-15|922|4011|308|1976|1|11|15|4|1976|308|4011|Monday|1976Q4|N|N|N|2443084|2443388|2442732|2443006|N|N|N|N|N| +2443099|AAAAAAAALFHEFCAA|1976-11-16|922|4012|308|1976|2|11|16|4|1976|308|4012|Tuesday|1976Q4|N|N|N|2443084|2443388|2442733|2443007|N|N|N|N|N| +2443100|AAAAAAAAMFHEFCAA|1976-11-17|922|4012|308|1976|3|11|17|4|1976|308|4012|Wednesday|1976Q4|N|N|N|2443084|2443388|2442734|2443008|N|N|N|N|N| +2443101|AAAAAAAANFHEFCAA|1976-11-18|922|4012|308|1976|4|11|18|4|1976|308|4012|Thursday|1976Q4|N|N|N|2443084|2443388|2442735|2443009|N|N|N|N|N| +2443102|AAAAAAAAOFHEFCAA|1976-11-19|922|4012|308|1976|5|11|19|4|1976|308|4012|Friday|1976Q4|N|Y|N|2443084|2443388|2442736|2443010|N|N|N|N|N| +2443103|AAAAAAAAPFHEFCAA|1976-11-20|922|4012|308|1976|6|11|20|4|1976|308|4012|Saturday|1976Q4|N|Y|N|2443084|2443388|2442737|2443011|N|N|N|N|N| +2443104|AAAAAAAAAGHEFCAA|1976-11-21|922|4012|308|1976|0|11|21|4|1976|308|4012|Sunday|1976Q4|N|N|N|2443084|2443388|2442738|2443012|N|N|N|N|N| +2443105|AAAAAAAABGHEFCAA|1976-11-22|922|4012|308|1976|1|11|22|4|1976|308|4012|Monday|1976Q4|N|N|N|2443084|2443388|2442739|2443013|N|N|N|N|N| +2443106|AAAAAAAACGHEFCAA|1976-11-23|922|4013|308|1976|2|11|23|4|1976|308|4013|Tuesday|1976Q4|N|N|N|2443084|2443388|2442740|2443014|N|N|N|N|N| +2443107|AAAAAAAADGHEFCAA|1976-11-24|922|4013|308|1976|3|11|24|4|1976|308|4013|Wednesday|1976Q4|N|N|N|2443084|2443388|2442741|2443015|N|N|N|N|N| +2443108|AAAAAAAAEGHEFCAA|1976-11-25|922|4013|308|1976|4|11|25|4|1976|308|4013|Thursday|1976Q4|N|N|N|2443084|2443388|2442742|2443016|N|N|N|N|N| +2443109|AAAAAAAAFGHEFCAA|1976-11-26|922|4013|308|1976|5|11|26|4|1976|308|4013|Friday|1976Q4|N|Y|N|2443084|2443388|2442743|2443017|N|N|N|N|N| +2443110|AAAAAAAAGGHEFCAA|1976-11-27|922|4013|308|1976|6|11|27|4|1976|308|4013|Saturday|1976Q4|N|Y|N|2443084|2443388|2442744|2443018|N|N|N|N|N| +2443111|AAAAAAAAHGHEFCAA|1976-11-28|922|4013|308|1976|0|11|28|4|1976|308|4013|Sunday|1976Q4|N|N|N|2443084|2443388|2442745|2443019|N|N|N|N|N| +2443112|AAAAAAAAIGHEFCAA|1976-11-29|922|4013|308|1976|1|11|29|4|1976|308|4013|Monday|1976Q4|N|N|N|2443084|2443388|2442746|2443020|N|N|N|N|N| +2443113|AAAAAAAAJGHEFCAA|1976-11-30|922|4014|308|1976|2|11|30|4|1976|308|4014|Tuesday|1976Q4|N|N|N|2443084|2443388|2442747|2443021|N|N|N|N|N| +2443114|AAAAAAAAKGHEFCAA|1976-12-01|923|4014|309|1976|3|12|1|4|1976|309|4014|Wednesday|1976Q4|N|N|N|2443114|2443448|2442748|2443022|N|N|N|N|N| +2443115|AAAAAAAALGHEFCAA|1976-12-02|923|4014|309|1976|4|12|2|4|1976|309|4014|Thursday|1976Q4|N|N|N|2443114|2443448|2442749|2443023|N|N|N|N|N| +2443116|AAAAAAAAMGHEFCAA|1976-12-03|923|4014|309|1976|5|12|3|4|1976|309|4014|Friday|1976Q4|N|Y|N|2443114|2443448|2442750|2443024|N|N|N|N|N| +2443117|AAAAAAAANGHEFCAA|1976-12-04|923|4014|309|1976|6|12|4|4|1976|309|4014|Saturday|1976Q4|N|Y|N|2443114|2443448|2442751|2443025|N|N|N|N|N| +2443118|AAAAAAAAOGHEFCAA|1976-12-05|923|4014|309|1976|0|12|5|4|1976|309|4014|Sunday|1976Q4|N|N|N|2443114|2443448|2442752|2443026|N|N|N|N|N| +2443119|AAAAAAAAPGHEFCAA|1976-12-06|923|4014|309|1976|1|12|6|4|1976|309|4014|Monday|1976Q4|N|N|N|2443114|2443448|2442753|2443027|N|N|N|N|N| +2443120|AAAAAAAAAHHEFCAA|1976-12-07|923|4015|309|1976|2|12|7|4|1976|309|4015|Tuesday|1976Q4|N|N|N|2443114|2443448|2442754|2443028|N|N|N|N|N| +2443121|AAAAAAAABHHEFCAA|1976-12-08|923|4015|309|1976|3|12|8|4|1976|309|4015|Wednesday|1976Q4|N|N|N|2443114|2443448|2442755|2443029|N|N|N|N|N| +2443122|AAAAAAAACHHEFCAA|1976-12-09|923|4015|309|1976|4|12|9|4|1976|309|4015|Thursday|1976Q4|N|N|N|2443114|2443448|2442756|2443030|N|N|N|N|N| +2443123|AAAAAAAADHHEFCAA|1976-12-10|923|4015|309|1976|5|12|10|4|1976|309|4015|Friday|1976Q4|N|Y|N|2443114|2443448|2442757|2443031|N|N|N|N|N| +2443124|AAAAAAAAEHHEFCAA|1976-12-11|923|4015|309|1976|6|12|11|4|1976|309|4015|Saturday|1976Q4|N|Y|N|2443114|2443448|2442758|2443032|N|N|N|N|N| +2443125|AAAAAAAAFHHEFCAA|1976-12-12|923|4015|309|1976|0|12|12|4|1976|309|4015|Sunday|1976Q4|N|N|N|2443114|2443448|2442759|2443033|N|N|N|N|N| +2443126|AAAAAAAAGHHEFCAA|1976-12-13|923|4015|309|1976|1|12|13|4|1976|309|4015|Monday|1976Q4|N|N|N|2443114|2443448|2442760|2443034|N|N|N|N|N| +2443127|AAAAAAAAHHHEFCAA|1976-12-14|923|4016|309|1976|2|12|14|4|1976|309|4016|Tuesday|1976Q4|N|N|N|2443114|2443448|2442761|2443035|N|N|N|N|N| +2443128|AAAAAAAAIHHEFCAA|1976-12-15|923|4016|309|1976|3|12|15|4|1976|309|4016|Wednesday|1976Q4|N|N|N|2443114|2443448|2442762|2443036|N|N|N|N|N| +2443129|AAAAAAAAJHHEFCAA|1976-12-16|923|4016|309|1976|4|12|16|4|1976|309|4016|Thursday|1976Q4|N|N|N|2443114|2443448|2442763|2443037|N|N|N|N|N| +2443130|AAAAAAAAKHHEFCAA|1976-12-17|923|4016|309|1976|5|12|17|4|1976|309|4016|Friday|1976Q4|N|Y|N|2443114|2443448|2442764|2443038|N|N|N|N|N| +2443131|AAAAAAAALHHEFCAA|1976-12-18|923|4016|309|1976|6|12|18|4|1976|309|4016|Saturday|1976Q4|N|Y|N|2443114|2443448|2442765|2443039|N|N|N|N|N| +2443132|AAAAAAAAMHHEFCAA|1976-12-19|923|4016|309|1976|0|12|19|4|1976|309|4016|Sunday|1976Q4|N|N|N|2443114|2443448|2442766|2443040|N|N|N|N|N| +2443133|AAAAAAAANHHEFCAA|1976-12-20|923|4016|309|1976|1|12|20|4|1976|309|4016|Monday|1976Q4|N|N|N|2443114|2443448|2442767|2443041|N|N|N|N|N| +2443134|AAAAAAAAOHHEFCAA|1976-12-21|923|4017|309|1976|2|12|21|4|1976|309|4017|Tuesday|1976Q4|N|N|N|2443114|2443448|2442768|2443042|N|N|N|N|N| +2443135|AAAAAAAAPHHEFCAA|1976-12-22|923|4017|309|1976|3|12|22|4|1976|309|4017|Wednesday|1976Q4|N|N|N|2443114|2443448|2442769|2443043|N|N|N|N|N| +2443136|AAAAAAAAAIHEFCAA|1976-12-23|923|4017|309|1976|4|12|23|4|1976|309|4017|Thursday|1976Q4|N|N|N|2443114|2443448|2442770|2443044|N|N|N|N|N| +2443137|AAAAAAAABIHEFCAA|1976-12-24|923|4017|309|1976|5|12|24|4|1976|309|4017|Friday|1976Q4|N|Y|N|2443114|2443448|2442771|2443045|N|N|N|N|N| +2443138|AAAAAAAACIHEFCAA|1976-12-25|923|4017|309|1976|6|12|25|4|1976|309|4017|Saturday|1976Q4|Y|Y|N|2443114|2443448|2442772|2443046|N|N|N|N|N| +2443139|AAAAAAAADIHEFCAA|1976-12-26|923|4017|309|1976|0|12|26|4|1976|309|4017|Sunday|1976Q4|N|N|Y|2443114|2443448|2442773|2443047|N|N|N|N|N| +2443140|AAAAAAAAEIHEFCAA|1976-12-27|923|4017|309|1976|1|12|27|4|1976|309|4017|Monday|1976Q4|N|N|N|2443114|2443448|2442774|2443048|N|N|N|N|N| +2443141|AAAAAAAAFIHEFCAA|1976-12-28|923|4018|309|1976|2|12|28|4|1976|309|4018|Tuesday|1976Q4|N|N|N|2443114|2443448|2442775|2443049|N|N|N|N|N| +2443142|AAAAAAAAGIHEFCAA|1976-12-29|923|4018|309|1976|3|12|29|4|1976|309|4018|Wednesday|1976Q4|N|N|N|2443114|2443448|2442776|2443050|N|N|N|N|N| +2443143|AAAAAAAAHIHEFCAA|1976-12-30|923|4018|309|1976|4|12|30|4|1976|309|4018|Thursday|1976Q4|N|N|N|2443114|2443448|2442777|2443051|N|N|N|N|N| +2443144|AAAAAAAAIIHEFCAA|1976-12-31|923|4018|309|1976|5|12|31|4|1976|309|4018|Friday|1976Q4|Y|Y|N|2443114|2443448|2442778|2443052|N|N|N|N|N| +2443145|AAAAAAAAJIHEFCAA|1977-01-01|924|4018|309|1977|6|1|1|1|1977|309|4018|Saturday|1977Q1|Y|Y|Y|2443145|2443144|2442779|2443053|N|N|N|N|N| +2443146|AAAAAAAAKIHEFCAA|1977-01-02|924|4018|309|1977|0|1|2|1|1977|309|4018|Sunday|1977Q1|N|N|Y|2443145|2443144|2442780|2443054|N|N|N|N|N| +2443147|AAAAAAAALIHEFCAA|1977-01-03|924|4018|309|1977|1|1|3|1|1977|309|4018|Monday|1977Q1|N|N|N|2443145|2443144|2442781|2443055|N|N|N|N|N| +2443148|AAAAAAAAMIHEFCAA|1977-01-04|924|4019|309|1977|2|1|4|1|1977|309|4019|Tuesday|1977Q1|N|N|N|2443145|2443144|2442782|2443056|N|N|N|N|N| +2443149|AAAAAAAANIHEFCAA|1977-01-05|924|4019|309|1977|3|1|5|1|1977|309|4019|Wednesday|1977Q1|N|N|N|2443145|2443144|2442783|2443057|N|N|N|N|N| +2443150|AAAAAAAAOIHEFCAA|1977-01-06|924|4019|309|1977|4|1|6|1|1977|309|4019|Thursday|1977Q1|N|N|N|2443145|2443144|2442784|2443058|N|N|N|N|N| +2443151|AAAAAAAAPIHEFCAA|1977-01-07|924|4019|309|1977|5|1|7|1|1977|309|4019|Friday|1977Q1|N|Y|N|2443145|2443144|2442785|2443059|N|N|N|N|N| +2443152|AAAAAAAAAJHEFCAA|1977-01-08|924|4019|309|1977|6|1|8|1|1977|309|4019|Saturday|1977Q1|N|Y|N|2443145|2443144|2442786|2443060|N|N|N|N|N| +2443153|AAAAAAAABJHEFCAA|1977-01-09|924|4019|309|1977|0|1|9|1|1977|309|4019|Sunday|1977Q1|N|N|N|2443145|2443144|2442787|2443061|N|N|N|N|N| +2443154|AAAAAAAACJHEFCAA|1977-01-10|924|4019|309|1977|1|1|10|1|1977|309|4019|Monday|1977Q1|N|N|N|2443145|2443144|2442788|2443062|N|N|N|N|N| +2443155|AAAAAAAADJHEFCAA|1977-01-11|924|4020|309|1977|2|1|11|1|1977|309|4020|Tuesday|1977Q1|N|N|N|2443145|2443144|2442789|2443063|N|N|N|N|N| +2443156|AAAAAAAAEJHEFCAA|1977-01-12|924|4020|309|1977|3|1|12|1|1977|309|4020|Wednesday|1977Q1|N|N|N|2443145|2443144|2442790|2443064|N|N|N|N|N| +2443157|AAAAAAAAFJHEFCAA|1977-01-13|924|4020|309|1977|4|1|13|1|1977|309|4020|Thursday|1977Q1|N|N|N|2443145|2443144|2442791|2443065|N|N|N|N|N| +2443158|AAAAAAAAGJHEFCAA|1977-01-14|924|4020|309|1977|5|1|14|1|1977|309|4020|Friday|1977Q1|N|Y|N|2443145|2443144|2442792|2443066|N|N|N|N|N| +2443159|AAAAAAAAHJHEFCAA|1977-01-15|924|4020|309|1977|6|1|15|1|1977|309|4020|Saturday|1977Q1|N|Y|N|2443145|2443144|2442793|2443067|N|N|N|N|N| +2443160|AAAAAAAAIJHEFCAA|1977-01-16|924|4020|309|1977|0|1|16|1|1977|309|4020|Sunday|1977Q1|N|N|N|2443145|2443144|2442794|2443068|N|N|N|N|N| +2443161|AAAAAAAAJJHEFCAA|1977-01-17|924|4020|309|1977|1|1|17|1|1977|309|4020|Monday|1977Q1|N|N|N|2443145|2443144|2442795|2443069|N|N|N|N|N| +2443162|AAAAAAAAKJHEFCAA|1977-01-18|924|4021|309|1977|2|1|18|1|1977|309|4021|Tuesday|1977Q1|N|N|N|2443145|2443144|2442796|2443070|N|N|N|N|N| +2443163|AAAAAAAALJHEFCAA|1977-01-19|924|4021|309|1977|3|1|19|1|1977|309|4021|Wednesday|1977Q1|N|N|N|2443145|2443144|2442797|2443071|N|N|N|N|N| +2443164|AAAAAAAAMJHEFCAA|1977-01-20|924|4021|309|1977|4|1|20|1|1977|309|4021|Thursday|1977Q1|N|N|N|2443145|2443144|2442798|2443072|N|N|N|N|N| +2443165|AAAAAAAANJHEFCAA|1977-01-21|924|4021|309|1977|5|1|21|1|1977|309|4021|Friday|1977Q1|N|Y|N|2443145|2443144|2442799|2443073|N|N|N|N|N| +2443166|AAAAAAAAOJHEFCAA|1977-01-22|924|4021|309|1977|6|1|22|1|1977|309|4021|Saturday|1977Q1|N|Y|N|2443145|2443144|2442800|2443074|N|N|N|N|N| +2443167|AAAAAAAAPJHEFCAA|1977-01-23|924|4021|309|1977|0|1|23|1|1977|309|4021|Sunday|1977Q1|N|N|N|2443145|2443144|2442801|2443075|N|N|N|N|N| +2443168|AAAAAAAAAKHEFCAA|1977-01-24|924|4021|309|1977|1|1|24|1|1977|309|4021|Monday|1977Q1|N|N|N|2443145|2443144|2442802|2443076|N|N|N|N|N| +2443169|AAAAAAAABKHEFCAA|1977-01-25|924|4022|309|1977|2|1|25|1|1977|309|4022|Tuesday|1977Q1|N|N|N|2443145|2443144|2442803|2443077|N|N|N|N|N| +2443170|AAAAAAAACKHEFCAA|1977-01-26|924|4022|309|1977|3|1|26|1|1977|309|4022|Wednesday|1977Q1|N|N|N|2443145|2443144|2442804|2443078|N|N|N|N|N| +2443171|AAAAAAAADKHEFCAA|1977-01-27|924|4022|309|1977|4|1|27|1|1977|309|4022|Thursday|1977Q1|N|N|N|2443145|2443144|2442805|2443079|N|N|N|N|N| +2443172|AAAAAAAAEKHEFCAA|1977-01-28|924|4022|309|1977|5|1|28|1|1977|309|4022|Friday|1977Q1|N|Y|N|2443145|2443144|2442806|2443080|N|N|N|N|N| +2443173|AAAAAAAAFKHEFCAA|1977-01-29|924|4022|309|1977|6|1|29|1|1977|309|4022|Saturday|1977Q1|N|Y|N|2443145|2443144|2442807|2443081|N|N|N|N|N| +2443174|AAAAAAAAGKHEFCAA|1977-01-30|924|4022|309|1977|0|1|30|1|1977|309|4022|Sunday|1977Q1|N|N|N|2443145|2443144|2442808|2443082|N|N|N|N|N| +2443175|AAAAAAAAHKHEFCAA|1977-01-31|924|4022|309|1977|1|1|31|1|1977|309|4022|Monday|1977Q1|N|N|N|2443145|2443144|2442809|2443083|N|N|N|N|N| +2443176|AAAAAAAAIKHEFCAA|1977-02-01|925|4023|309|1977|2|2|1|1|1977|309|4023|Tuesday|1977Q1|N|N|N|2443176|2443206|2442810|2443084|N|N|N|N|N| +2443177|AAAAAAAAJKHEFCAA|1977-02-02|925|4023|309|1977|3|2|2|1|1977|309|4023|Wednesday|1977Q1|N|N|N|2443176|2443206|2442811|2443085|N|N|N|N|N| +2443178|AAAAAAAAKKHEFCAA|1977-02-03|925|4023|309|1977|4|2|3|1|1977|309|4023|Thursday|1977Q1|N|N|N|2443176|2443206|2442812|2443086|N|N|N|N|N| +2443179|AAAAAAAALKHEFCAA|1977-02-04|925|4023|309|1977|5|2|4|1|1977|309|4023|Friday|1977Q1|N|Y|N|2443176|2443206|2442813|2443087|N|N|N|N|N| +2443180|AAAAAAAAMKHEFCAA|1977-02-05|925|4023|309|1977|6|2|5|1|1977|309|4023|Saturday|1977Q1|N|Y|N|2443176|2443206|2442814|2443088|N|N|N|N|N| +2443181|AAAAAAAANKHEFCAA|1977-02-06|925|4023|309|1977|0|2|6|1|1977|309|4023|Sunday|1977Q1|N|N|N|2443176|2443206|2442815|2443089|N|N|N|N|N| +2443182|AAAAAAAAOKHEFCAA|1977-02-07|925|4023|309|1977|1|2|7|1|1977|309|4023|Monday|1977Q1|N|N|N|2443176|2443206|2442816|2443090|N|N|N|N|N| +2443183|AAAAAAAAPKHEFCAA|1977-02-08|925|4024|309|1977|2|2|8|1|1977|309|4024|Tuesday|1977Q1|N|N|N|2443176|2443206|2442817|2443091|N|N|N|N|N| +2443184|AAAAAAAAALHEFCAA|1977-02-09|925|4024|309|1977|3|2|9|1|1977|309|4024|Wednesday|1977Q1|N|N|N|2443176|2443206|2442818|2443092|N|N|N|N|N| +2443185|AAAAAAAABLHEFCAA|1977-02-10|925|4024|309|1977|4|2|10|1|1977|309|4024|Thursday|1977Q1|N|N|N|2443176|2443206|2442819|2443093|N|N|N|N|N| +2443186|AAAAAAAACLHEFCAA|1977-02-11|925|4024|309|1977|5|2|11|1|1977|309|4024|Friday|1977Q1|N|Y|N|2443176|2443206|2442820|2443094|N|N|N|N|N| +2443187|AAAAAAAADLHEFCAA|1977-02-12|925|4024|309|1977|6|2|12|1|1977|309|4024|Saturday|1977Q1|N|Y|N|2443176|2443206|2442821|2443095|N|N|N|N|N| +2443188|AAAAAAAAELHEFCAA|1977-02-13|925|4024|309|1977|0|2|13|1|1977|309|4024|Sunday|1977Q1|N|N|N|2443176|2443206|2442822|2443096|N|N|N|N|N| +2443189|AAAAAAAAFLHEFCAA|1977-02-14|925|4024|309|1977|1|2|14|1|1977|309|4024|Monday|1977Q1|N|N|N|2443176|2443206|2442823|2443097|N|N|N|N|N| +2443190|AAAAAAAAGLHEFCAA|1977-02-15|925|4025|309|1977|2|2|15|1|1977|309|4025|Tuesday|1977Q1|N|N|N|2443176|2443206|2442824|2443098|N|N|N|N|N| +2443191|AAAAAAAAHLHEFCAA|1977-02-16|925|4025|309|1977|3|2|16|1|1977|309|4025|Wednesday|1977Q1|N|N|N|2443176|2443206|2442825|2443099|N|N|N|N|N| +2443192|AAAAAAAAILHEFCAA|1977-02-17|925|4025|309|1977|4|2|17|1|1977|309|4025|Thursday|1977Q1|N|N|N|2443176|2443206|2442826|2443100|N|N|N|N|N| +2443193|AAAAAAAAJLHEFCAA|1977-02-18|925|4025|309|1977|5|2|18|1|1977|309|4025|Friday|1977Q1|N|Y|N|2443176|2443206|2442827|2443101|N|N|N|N|N| +2443194|AAAAAAAAKLHEFCAA|1977-02-19|925|4025|309|1977|6|2|19|1|1977|309|4025|Saturday|1977Q1|N|Y|N|2443176|2443206|2442828|2443102|N|N|N|N|N| +2443195|AAAAAAAALLHEFCAA|1977-02-20|925|4025|309|1977|0|2|20|1|1977|309|4025|Sunday|1977Q1|N|N|N|2443176|2443206|2442829|2443103|N|N|N|N|N| +2443196|AAAAAAAAMLHEFCAA|1977-02-21|925|4025|309|1977|1|2|21|1|1977|309|4025|Monday|1977Q1|N|N|N|2443176|2443206|2442830|2443104|N|N|N|N|N| +2443197|AAAAAAAANLHEFCAA|1977-02-22|925|4026|309|1977|2|2|22|1|1977|309|4026|Tuesday|1977Q1|N|N|N|2443176|2443206|2442831|2443105|N|N|N|N|N| +2443198|AAAAAAAAOLHEFCAA|1977-02-23|925|4026|309|1977|3|2|23|1|1977|309|4026|Wednesday|1977Q1|N|N|N|2443176|2443206|2442832|2443106|N|N|N|N|N| +2443199|AAAAAAAAPLHEFCAA|1977-02-24|925|4026|309|1977|4|2|24|1|1977|309|4026|Thursday|1977Q1|N|N|N|2443176|2443206|2442833|2443107|N|N|N|N|N| +2443200|AAAAAAAAAMHEFCAA|1977-02-25|925|4026|309|1977|5|2|25|1|1977|309|4026|Friday|1977Q1|N|Y|N|2443176|2443206|2442834|2443108|N|N|N|N|N| +2443201|AAAAAAAABMHEFCAA|1977-02-26|925|4026|309|1977|6|2|26|1|1977|309|4026|Saturday|1977Q1|N|Y|N|2443176|2443206|2442835|2443109|N|N|N|N|N| +2443202|AAAAAAAACMHEFCAA|1977-02-27|925|4026|309|1977|0|2|27|1|1977|309|4026|Sunday|1977Q1|N|N|N|2443176|2443206|2442836|2443110|N|N|N|N|N| +2443203|AAAAAAAADMHEFCAA|1977-02-28|925|4026|309|1977|1|2|28|1|1977|309|4026|Monday|1977Q1|N|N|N|2443176|2443206|2442837|2443111|N|N|N|N|N| +2443204|AAAAAAAAEMHEFCAA|1977-03-01|926|4027|310|1977|2|3|1|1|1977|310|4027|Tuesday|1977Q1|N|N|N|2443204|2443262|2442839|2443112|N|N|N|N|N| +2443205|AAAAAAAAFMHEFCAA|1977-03-02|926|4027|310|1977|3|3|2|1|1977|310|4027|Wednesday|1977Q1|N|N|N|2443204|2443262|2442840|2443113|N|N|N|N|N| +2443206|AAAAAAAAGMHEFCAA|1977-03-03|926|4027|310|1977|4|3|3|1|1977|310|4027|Thursday|1977Q1|N|N|N|2443204|2443262|2442841|2443114|N|N|N|N|N| +2443207|AAAAAAAAHMHEFCAA|1977-03-04|926|4027|310|1977|5|3|4|1|1977|310|4027|Friday|1977Q1|N|Y|N|2443204|2443262|2442842|2443115|N|N|N|N|N| +2443208|AAAAAAAAIMHEFCAA|1977-03-05|926|4027|310|1977|6|3|5|1|1977|310|4027|Saturday|1977Q1|N|Y|N|2443204|2443262|2442843|2443116|N|N|N|N|N| +2443209|AAAAAAAAJMHEFCAA|1977-03-06|926|4027|310|1977|0|3|6|1|1977|310|4027|Sunday|1977Q1|N|N|N|2443204|2443262|2442844|2443117|N|N|N|N|N| +2443210|AAAAAAAAKMHEFCAA|1977-03-07|926|4027|310|1977|1|3|7|1|1977|310|4027|Monday|1977Q1|N|N|N|2443204|2443262|2442845|2443118|N|N|N|N|N| +2443211|AAAAAAAALMHEFCAA|1977-03-08|926|4028|310|1977|2|3|8|1|1977|310|4028|Tuesday|1977Q1|N|N|N|2443204|2443262|2442846|2443119|N|N|N|N|N| +2443212|AAAAAAAAMMHEFCAA|1977-03-09|926|4028|310|1977|3|3|9|1|1977|310|4028|Wednesday|1977Q1|N|N|N|2443204|2443262|2442847|2443120|N|N|N|N|N| +2443213|AAAAAAAANMHEFCAA|1977-03-10|926|4028|310|1977|4|3|10|1|1977|310|4028|Thursday|1977Q1|N|N|N|2443204|2443262|2442848|2443121|N|N|N|N|N| +2443214|AAAAAAAAOMHEFCAA|1977-03-11|926|4028|310|1977|5|3|11|1|1977|310|4028|Friday|1977Q1|N|Y|N|2443204|2443262|2442849|2443122|N|N|N|N|N| +2443215|AAAAAAAAPMHEFCAA|1977-03-12|926|4028|310|1977|6|3|12|1|1977|310|4028|Saturday|1977Q1|N|Y|N|2443204|2443262|2442850|2443123|N|N|N|N|N| +2443216|AAAAAAAAANHEFCAA|1977-03-13|926|4028|310|1977|0|3|13|1|1977|310|4028|Sunday|1977Q1|N|N|N|2443204|2443262|2442851|2443124|N|N|N|N|N| +2443217|AAAAAAAABNHEFCAA|1977-03-14|926|4028|310|1977|1|3|14|1|1977|310|4028|Monday|1977Q1|N|N|N|2443204|2443262|2442852|2443125|N|N|N|N|N| +2443218|AAAAAAAACNHEFCAA|1977-03-15|926|4029|310|1977|2|3|15|1|1977|310|4029|Tuesday|1977Q1|N|N|N|2443204|2443262|2442853|2443126|N|N|N|N|N| +2443219|AAAAAAAADNHEFCAA|1977-03-16|926|4029|310|1977|3|3|16|1|1977|310|4029|Wednesday|1977Q1|N|N|N|2443204|2443262|2442854|2443127|N|N|N|N|N| +2443220|AAAAAAAAENHEFCAA|1977-03-17|926|4029|310|1977|4|3|17|1|1977|310|4029|Thursday|1977Q1|N|N|N|2443204|2443262|2442855|2443128|N|N|N|N|N| +2443221|AAAAAAAAFNHEFCAA|1977-03-18|926|4029|310|1977|5|3|18|1|1977|310|4029|Friday|1977Q1|N|Y|N|2443204|2443262|2442856|2443129|N|N|N|N|N| +2443222|AAAAAAAAGNHEFCAA|1977-03-19|926|4029|310|1977|6|3|19|1|1977|310|4029|Saturday|1977Q1|N|Y|N|2443204|2443262|2442857|2443130|N|N|N|N|N| +2443223|AAAAAAAAHNHEFCAA|1977-03-20|926|4029|310|1977|0|3|20|1|1977|310|4029|Sunday|1977Q1|N|N|N|2443204|2443262|2442858|2443131|N|N|N|N|N| +2443224|AAAAAAAAINHEFCAA|1977-03-21|926|4029|310|1977|1|3|21|1|1977|310|4029|Monday|1977Q1|N|N|N|2443204|2443262|2442859|2443132|N|N|N|N|N| +2443225|AAAAAAAAJNHEFCAA|1977-03-22|926|4030|310|1977|2|3|22|1|1977|310|4030|Tuesday|1977Q1|N|N|N|2443204|2443262|2442860|2443133|N|N|N|N|N| +2443226|AAAAAAAAKNHEFCAA|1977-03-23|926|4030|310|1977|3|3|23|1|1977|310|4030|Wednesday|1977Q1|N|N|N|2443204|2443262|2442861|2443134|N|N|N|N|N| +2443227|AAAAAAAALNHEFCAA|1977-03-24|926|4030|310|1977|4|3|24|1|1977|310|4030|Thursday|1977Q1|N|N|N|2443204|2443262|2442862|2443135|N|N|N|N|N| +2443228|AAAAAAAAMNHEFCAA|1977-03-25|926|4030|310|1977|5|3|25|1|1977|310|4030|Friday|1977Q1|N|Y|N|2443204|2443262|2442863|2443136|N|N|N|N|N| +2443229|AAAAAAAANNHEFCAA|1977-03-26|926|4030|310|1977|6|3|26|1|1977|310|4030|Saturday|1977Q1|N|Y|N|2443204|2443262|2442864|2443137|N|N|N|N|N| +2443230|AAAAAAAAONHEFCAA|1977-03-27|926|4030|310|1977|0|3|27|1|1977|310|4030|Sunday|1977Q1|N|N|N|2443204|2443262|2442865|2443138|N|N|N|N|N| +2443231|AAAAAAAAPNHEFCAA|1977-03-28|926|4030|310|1977|1|3|28|1|1977|310|4030|Monday|1977Q1|N|N|N|2443204|2443262|2442866|2443139|N|N|N|N|N| +2443232|AAAAAAAAAOHEFCAA|1977-03-29|926|4031|310|1977|2|3|29|1|1977|310|4031|Tuesday|1977Q1|N|N|N|2443204|2443262|2442867|2443140|N|N|N|N|N| +2443233|AAAAAAAABOHEFCAA|1977-03-30|926|4031|310|1977|3|3|30|1|1977|310|4031|Wednesday|1977Q1|N|N|N|2443204|2443262|2442868|2443141|N|N|N|N|N| +2443234|AAAAAAAACOHEFCAA|1977-03-31|926|4031|310|1977|4|3|31|1|1977|310|4031|Thursday|1977Q1|N|N|N|2443204|2443262|2442869|2443142|N|N|N|N|N| +2443235|AAAAAAAADOHEFCAA|1977-04-01|927|4031|310|1977|5|4|1|1|1977|310|4031|Friday|1977Q1|N|Y|N|2443235|2443324|2442870|2443145|N|N|N|N|N| +2443236|AAAAAAAAEOHEFCAA|1977-04-02|927|4031|310|1977|6|4|2|2|1977|310|4031|Saturday|1977Q2|N|Y|N|2443235|2443324|2442871|2443146|N|N|N|N|N| +2443237|AAAAAAAAFOHEFCAA|1977-04-03|927|4031|310|1977|0|4|3|2|1977|310|4031|Sunday|1977Q2|N|N|N|2443235|2443324|2442872|2443147|N|N|N|N|N| +2443238|AAAAAAAAGOHEFCAA|1977-04-04|927|4031|310|1977|1|4|4|2|1977|310|4031|Monday|1977Q2|N|N|N|2443235|2443324|2442873|2443148|N|N|N|N|N| +2443239|AAAAAAAAHOHEFCAA|1977-04-05|927|4032|310|1977|2|4|5|2|1977|310|4032|Tuesday|1977Q2|N|N|N|2443235|2443324|2442874|2443149|N|N|N|N|N| +2443240|AAAAAAAAIOHEFCAA|1977-04-06|927|4032|310|1977|3|4|6|2|1977|310|4032|Wednesday|1977Q2|N|N|N|2443235|2443324|2442875|2443150|N|N|N|N|N| +2443241|AAAAAAAAJOHEFCAA|1977-04-07|927|4032|310|1977|4|4|7|2|1977|310|4032|Thursday|1977Q2|N|N|N|2443235|2443324|2442876|2443151|N|N|N|N|N| +2443242|AAAAAAAAKOHEFCAA|1977-04-08|927|4032|310|1977|5|4|8|2|1977|310|4032|Friday|1977Q2|N|Y|N|2443235|2443324|2442877|2443152|N|N|N|N|N| +2443243|AAAAAAAALOHEFCAA|1977-04-09|927|4032|310|1977|6|4|9|2|1977|310|4032|Saturday|1977Q2|N|Y|N|2443235|2443324|2442878|2443153|N|N|N|N|N| +2443244|AAAAAAAAMOHEFCAA|1977-04-10|927|4032|310|1977|0|4|10|2|1977|310|4032|Sunday|1977Q2|N|N|N|2443235|2443324|2442879|2443154|N|N|N|N|N| +2443245|AAAAAAAANOHEFCAA|1977-04-11|927|4032|310|1977|1|4|11|2|1977|310|4032|Monday|1977Q2|N|N|N|2443235|2443324|2442880|2443155|N|N|N|N|N| +2443246|AAAAAAAAOOHEFCAA|1977-04-12|927|4033|310|1977|2|4|12|2|1977|310|4033|Tuesday|1977Q2|N|N|N|2443235|2443324|2442881|2443156|N|N|N|N|N| +2443247|AAAAAAAAPOHEFCAA|1977-04-13|927|4033|310|1977|3|4|13|2|1977|310|4033|Wednesday|1977Q2|N|N|N|2443235|2443324|2442882|2443157|N|N|N|N|N| +2443248|AAAAAAAAAPHEFCAA|1977-04-14|927|4033|310|1977|4|4|14|2|1977|310|4033|Thursday|1977Q2|N|N|N|2443235|2443324|2442883|2443158|N|N|N|N|N| +2443249|AAAAAAAABPHEFCAA|1977-04-15|927|4033|310|1977|5|4|15|2|1977|310|4033|Friday|1977Q2|N|Y|N|2443235|2443324|2442884|2443159|N|N|N|N|N| +2443250|AAAAAAAACPHEFCAA|1977-04-16|927|4033|310|1977|6|4|16|2|1977|310|4033|Saturday|1977Q2|N|Y|N|2443235|2443324|2442885|2443160|N|N|N|N|N| +2443251|AAAAAAAADPHEFCAA|1977-04-17|927|4033|310|1977|0|4|17|2|1977|310|4033|Sunday|1977Q2|N|N|N|2443235|2443324|2442886|2443161|N|N|N|N|N| +2443252|AAAAAAAAEPHEFCAA|1977-04-18|927|4033|310|1977|1|4|18|2|1977|310|4033|Monday|1977Q2|N|N|N|2443235|2443324|2442887|2443162|N|N|N|N|N| +2443253|AAAAAAAAFPHEFCAA|1977-04-19|927|4034|310|1977|2|4|19|2|1977|310|4034|Tuesday|1977Q2|N|N|N|2443235|2443324|2442888|2443163|N|N|N|N|N| +2443254|AAAAAAAAGPHEFCAA|1977-04-20|927|4034|310|1977|3|4|20|2|1977|310|4034|Wednesday|1977Q2|N|N|N|2443235|2443324|2442889|2443164|N|N|N|N|N| +2443255|AAAAAAAAHPHEFCAA|1977-04-21|927|4034|310|1977|4|4|21|2|1977|310|4034|Thursday|1977Q2|N|N|N|2443235|2443324|2442890|2443165|N|N|N|N|N| +2443256|AAAAAAAAIPHEFCAA|1977-04-22|927|4034|310|1977|5|4|22|2|1977|310|4034|Friday|1977Q2|N|Y|N|2443235|2443324|2442891|2443166|N|N|N|N|N| +2443257|AAAAAAAAJPHEFCAA|1977-04-23|927|4034|310|1977|6|4|23|2|1977|310|4034|Saturday|1977Q2|N|Y|N|2443235|2443324|2442892|2443167|N|N|N|N|N| +2443258|AAAAAAAAKPHEFCAA|1977-04-24|927|4034|310|1977|0|4|24|2|1977|310|4034|Sunday|1977Q2|N|N|N|2443235|2443324|2442893|2443168|N|N|N|N|N| +2443259|AAAAAAAALPHEFCAA|1977-04-25|927|4034|310|1977|1|4|25|2|1977|310|4034|Monday|1977Q2|N|N|N|2443235|2443324|2442894|2443169|N|N|N|N|N| +2443260|AAAAAAAAMPHEFCAA|1977-04-26|927|4035|310|1977|2|4|26|2|1977|310|4035|Tuesday|1977Q2|N|N|N|2443235|2443324|2442895|2443170|N|N|N|N|N| +2443261|AAAAAAAANPHEFCAA|1977-04-27|927|4035|310|1977|3|4|27|2|1977|310|4035|Wednesday|1977Q2|N|N|N|2443235|2443324|2442896|2443171|N|N|N|N|N| +2443262|AAAAAAAAOPHEFCAA|1977-04-28|927|4035|310|1977|4|4|28|2|1977|310|4035|Thursday|1977Q2|N|N|N|2443235|2443324|2442897|2443172|N|N|N|N|N| +2443263|AAAAAAAAPPHEFCAA|1977-04-29|927|4035|310|1977|5|4|29|2|1977|310|4035|Friday|1977Q2|N|Y|N|2443235|2443324|2442898|2443173|N|N|N|N|N| +2443264|AAAAAAAAAAIEFCAA|1977-04-30|927|4035|310|1977|6|4|30|2|1977|310|4035|Saturday|1977Q2|N|Y|N|2443235|2443324|2442899|2443174|N|N|N|N|N| +2443265|AAAAAAAABAIEFCAA|1977-05-01|928|4035|310|1977|0|5|1|2|1977|310|4035|Sunday|1977Q2|N|N|N|2443265|2443384|2442900|2443175|N|N|N|N|N| +2443266|AAAAAAAACAIEFCAA|1977-05-02|928|4035|310|1977|1|5|2|2|1977|310|4035|Monday|1977Q2|N|N|N|2443265|2443384|2442901|2443176|N|N|N|N|N| +2443267|AAAAAAAADAIEFCAA|1977-05-03|928|4036|310|1977|2|5|3|2|1977|310|4036|Tuesday|1977Q2|N|N|N|2443265|2443384|2442902|2443177|N|N|N|N|N| +2443268|AAAAAAAAEAIEFCAA|1977-05-04|928|4036|310|1977|3|5|4|2|1977|310|4036|Wednesday|1977Q2|N|N|N|2443265|2443384|2442903|2443178|N|N|N|N|N| +2443269|AAAAAAAAFAIEFCAA|1977-05-05|928|4036|310|1977|4|5|5|2|1977|310|4036|Thursday|1977Q2|N|N|N|2443265|2443384|2442904|2443179|N|N|N|N|N| +2443270|AAAAAAAAGAIEFCAA|1977-05-06|928|4036|310|1977|5|5|6|2|1977|310|4036|Friday|1977Q2|N|Y|N|2443265|2443384|2442905|2443180|N|N|N|N|N| +2443271|AAAAAAAAHAIEFCAA|1977-05-07|928|4036|310|1977|6|5|7|2|1977|310|4036|Saturday|1977Q2|N|Y|N|2443265|2443384|2442906|2443181|N|N|N|N|N| +2443272|AAAAAAAAIAIEFCAA|1977-05-08|928|4036|310|1977|0|5|8|2|1977|310|4036|Sunday|1977Q2|N|N|N|2443265|2443384|2442907|2443182|N|N|N|N|N| +2443273|AAAAAAAAJAIEFCAA|1977-05-09|928|4036|310|1977|1|5|9|2|1977|310|4036|Monday|1977Q2|N|N|N|2443265|2443384|2442908|2443183|N|N|N|N|N| +2443274|AAAAAAAAKAIEFCAA|1977-05-10|928|4037|310|1977|2|5|10|2|1977|310|4037|Tuesday|1977Q2|N|N|N|2443265|2443384|2442909|2443184|N|N|N|N|N| +2443275|AAAAAAAALAIEFCAA|1977-05-11|928|4037|310|1977|3|5|11|2|1977|310|4037|Wednesday|1977Q2|N|N|N|2443265|2443384|2442910|2443185|N|N|N|N|N| +2443276|AAAAAAAAMAIEFCAA|1977-05-12|928|4037|310|1977|4|5|12|2|1977|310|4037|Thursday|1977Q2|N|N|N|2443265|2443384|2442911|2443186|N|N|N|N|N| +2443277|AAAAAAAANAIEFCAA|1977-05-13|928|4037|310|1977|5|5|13|2|1977|310|4037|Friday|1977Q2|N|Y|N|2443265|2443384|2442912|2443187|N|N|N|N|N| +2443278|AAAAAAAAOAIEFCAA|1977-05-14|928|4037|310|1977|6|5|14|2|1977|310|4037|Saturday|1977Q2|N|Y|N|2443265|2443384|2442913|2443188|N|N|N|N|N| +2443279|AAAAAAAAPAIEFCAA|1977-05-15|928|4037|310|1977|0|5|15|2|1977|310|4037|Sunday|1977Q2|N|N|N|2443265|2443384|2442914|2443189|N|N|N|N|N| +2443280|AAAAAAAAABIEFCAA|1977-05-16|928|4037|310|1977|1|5|16|2|1977|310|4037|Monday|1977Q2|N|N|N|2443265|2443384|2442915|2443190|N|N|N|N|N| +2443281|AAAAAAAABBIEFCAA|1977-05-17|928|4038|310|1977|2|5|17|2|1977|310|4038|Tuesday|1977Q2|N|N|N|2443265|2443384|2442916|2443191|N|N|N|N|N| +2443282|AAAAAAAACBIEFCAA|1977-05-18|928|4038|310|1977|3|5|18|2|1977|310|4038|Wednesday|1977Q2|N|N|N|2443265|2443384|2442917|2443192|N|N|N|N|N| +2443283|AAAAAAAADBIEFCAA|1977-05-19|928|4038|310|1977|4|5|19|2|1977|310|4038|Thursday|1977Q2|N|N|N|2443265|2443384|2442918|2443193|N|N|N|N|N| +2443284|AAAAAAAAEBIEFCAA|1977-05-20|928|4038|310|1977|5|5|20|2|1977|310|4038|Friday|1977Q2|N|Y|N|2443265|2443384|2442919|2443194|N|N|N|N|N| +2443285|AAAAAAAAFBIEFCAA|1977-05-21|928|4038|310|1977|6|5|21|2|1977|310|4038|Saturday|1977Q2|N|Y|N|2443265|2443384|2442920|2443195|N|N|N|N|N| +2443286|AAAAAAAAGBIEFCAA|1977-05-22|928|4038|310|1977|0|5|22|2|1977|310|4038|Sunday|1977Q2|N|N|N|2443265|2443384|2442921|2443196|N|N|N|N|N| +2443287|AAAAAAAAHBIEFCAA|1977-05-23|928|4038|310|1977|1|5|23|2|1977|310|4038|Monday|1977Q2|N|N|N|2443265|2443384|2442922|2443197|N|N|N|N|N| +2443288|AAAAAAAAIBIEFCAA|1977-05-24|928|4039|310|1977|2|5|24|2|1977|310|4039|Tuesday|1977Q2|N|N|N|2443265|2443384|2442923|2443198|N|N|N|N|N| +2443289|AAAAAAAAJBIEFCAA|1977-05-25|928|4039|310|1977|3|5|25|2|1977|310|4039|Wednesday|1977Q2|N|N|N|2443265|2443384|2442924|2443199|N|N|N|N|N| +2443290|AAAAAAAAKBIEFCAA|1977-05-26|928|4039|310|1977|4|5|26|2|1977|310|4039|Thursday|1977Q2|N|N|N|2443265|2443384|2442925|2443200|N|N|N|N|N| +2443291|AAAAAAAALBIEFCAA|1977-05-27|928|4039|310|1977|5|5|27|2|1977|310|4039|Friday|1977Q2|N|Y|N|2443265|2443384|2442926|2443201|N|N|N|N|N| +2443292|AAAAAAAAMBIEFCAA|1977-05-28|928|4039|310|1977|6|5|28|2|1977|310|4039|Saturday|1977Q2|N|Y|N|2443265|2443384|2442927|2443202|N|N|N|N|N| +2443293|AAAAAAAANBIEFCAA|1977-05-29|928|4039|310|1977|0|5|29|2|1977|310|4039|Sunday|1977Q2|N|N|N|2443265|2443384|2442928|2443203|N|N|N|N|N| +2443294|AAAAAAAAOBIEFCAA|1977-05-30|928|4039|310|1977|1|5|30|2|1977|310|4039|Monday|1977Q2|N|N|N|2443265|2443384|2442929|2443204|N|N|N|N|N| +2443295|AAAAAAAAPBIEFCAA|1977-05-31|928|4040|310|1977|2|5|31|2|1977|310|4040|Tuesday|1977Q2|N|N|N|2443265|2443384|2442930|2443205|N|N|N|N|N| +2443296|AAAAAAAAACIEFCAA|1977-06-01|929|4040|311|1977|3|6|1|2|1977|311|4040|Wednesday|1977Q2|N|N|N|2443296|2443446|2442931|2443206|N|N|N|N|N| +2443297|AAAAAAAABCIEFCAA|1977-06-02|929|4040|311|1977|4|6|2|2|1977|311|4040|Thursday|1977Q2|N|N|N|2443296|2443446|2442932|2443207|N|N|N|N|N| +2443298|AAAAAAAACCIEFCAA|1977-06-03|929|4040|311|1977|5|6|3|2|1977|311|4040|Friday|1977Q2|N|Y|N|2443296|2443446|2442933|2443208|N|N|N|N|N| +2443299|AAAAAAAADCIEFCAA|1977-06-04|929|4040|311|1977|6|6|4|2|1977|311|4040|Saturday|1977Q2|N|Y|N|2443296|2443446|2442934|2443209|N|N|N|N|N| +2443300|AAAAAAAAECIEFCAA|1977-06-05|929|4040|311|1977|0|6|5|2|1977|311|4040|Sunday|1977Q2|N|N|N|2443296|2443446|2442935|2443210|N|N|N|N|N| +2443301|AAAAAAAAFCIEFCAA|1977-06-06|929|4040|311|1977|1|6|6|2|1977|311|4040|Monday|1977Q2|N|N|N|2443296|2443446|2442936|2443211|N|N|N|N|N| +2443302|AAAAAAAAGCIEFCAA|1977-06-07|929|4041|311|1977|2|6|7|2|1977|311|4041|Tuesday|1977Q2|N|N|N|2443296|2443446|2442937|2443212|N|N|N|N|N| +2443303|AAAAAAAAHCIEFCAA|1977-06-08|929|4041|311|1977|3|6|8|2|1977|311|4041|Wednesday|1977Q2|N|N|N|2443296|2443446|2442938|2443213|N|N|N|N|N| +2443304|AAAAAAAAICIEFCAA|1977-06-09|929|4041|311|1977|4|6|9|2|1977|311|4041|Thursday|1977Q2|N|N|N|2443296|2443446|2442939|2443214|N|N|N|N|N| +2443305|AAAAAAAAJCIEFCAA|1977-06-10|929|4041|311|1977|5|6|10|2|1977|311|4041|Friday|1977Q2|N|Y|N|2443296|2443446|2442940|2443215|N|N|N|N|N| +2443306|AAAAAAAAKCIEFCAA|1977-06-11|929|4041|311|1977|6|6|11|2|1977|311|4041|Saturday|1977Q2|N|Y|N|2443296|2443446|2442941|2443216|N|N|N|N|N| +2443307|AAAAAAAALCIEFCAA|1977-06-12|929|4041|311|1977|0|6|12|2|1977|311|4041|Sunday|1977Q2|N|N|N|2443296|2443446|2442942|2443217|N|N|N|N|N| +2443308|AAAAAAAAMCIEFCAA|1977-06-13|929|4041|311|1977|1|6|13|2|1977|311|4041|Monday|1977Q2|N|N|N|2443296|2443446|2442943|2443218|N|N|N|N|N| +2443309|AAAAAAAANCIEFCAA|1977-06-14|929|4042|311|1977|2|6|14|2|1977|311|4042|Tuesday|1977Q2|N|N|N|2443296|2443446|2442944|2443219|N|N|N|N|N| +2443310|AAAAAAAAOCIEFCAA|1977-06-15|929|4042|311|1977|3|6|15|2|1977|311|4042|Wednesday|1977Q2|N|N|N|2443296|2443446|2442945|2443220|N|N|N|N|N| +2443311|AAAAAAAAPCIEFCAA|1977-06-16|929|4042|311|1977|4|6|16|2|1977|311|4042|Thursday|1977Q2|N|N|N|2443296|2443446|2442946|2443221|N|N|N|N|N| +2443312|AAAAAAAAADIEFCAA|1977-06-17|929|4042|311|1977|5|6|17|2|1977|311|4042|Friday|1977Q2|N|Y|N|2443296|2443446|2442947|2443222|N|N|N|N|N| +2443313|AAAAAAAABDIEFCAA|1977-06-18|929|4042|311|1977|6|6|18|2|1977|311|4042|Saturday|1977Q2|N|Y|N|2443296|2443446|2442948|2443223|N|N|N|N|N| +2443314|AAAAAAAACDIEFCAA|1977-06-19|929|4042|311|1977|0|6|19|2|1977|311|4042|Sunday|1977Q2|N|N|N|2443296|2443446|2442949|2443224|N|N|N|N|N| +2443315|AAAAAAAADDIEFCAA|1977-06-20|929|4042|311|1977|1|6|20|2|1977|311|4042|Monday|1977Q2|N|N|N|2443296|2443446|2442950|2443225|N|N|N|N|N| +2443316|AAAAAAAAEDIEFCAA|1977-06-21|929|4043|311|1977|2|6|21|2|1977|311|4043|Tuesday|1977Q2|N|N|N|2443296|2443446|2442951|2443226|N|N|N|N|N| +2443317|AAAAAAAAFDIEFCAA|1977-06-22|929|4043|311|1977|3|6|22|2|1977|311|4043|Wednesday|1977Q2|N|N|N|2443296|2443446|2442952|2443227|N|N|N|N|N| +2443318|AAAAAAAAGDIEFCAA|1977-06-23|929|4043|311|1977|4|6|23|2|1977|311|4043|Thursday|1977Q2|N|N|N|2443296|2443446|2442953|2443228|N|N|N|N|N| +2443319|AAAAAAAAHDIEFCAA|1977-06-24|929|4043|311|1977|5|6|24|2|1977|311|4043|Friday|1977Q2|N|Y|N|2443296|2443446|2442954|2443229|N|N|N|N|N| +2443320|AAAAAAAAIDIEFCAA|1977-06-25|929|4043|311|1977|6|6|25|2|1977|311|4043|Saturday|1977Q2|N|Y|N|2443296|2443446|2442955|2443230|N|N|N|N|N| +2443321|AAAAAAAAJDIEFCAA|1977-06-26|929|4043|311|1977|0|6|26|2|1977|311|4043|Sunday|1977Q2|N|N|N|2443296|2443446|2442956|2443231|N|N|N|N|N| +2443322|AAAAAAAAKDIEFCAA|1977-06-27|929|4043|311|1977|1|6|27|2|1977|311|4043|Monday|1977Q2|N|N|N|2443296|2443446|2442957|2443232|N|N|N|N|N| +2443323|AAAAAAAALDIEFCAA|1977-06-28|929|4044|311|1977|2|6|28|2|1977|311|4044|Tuesday|1977Q2|N|N|N|2443296|2443446|2442958|2443233|N|N|N|N|N| +2443324|AAAAAAAAMDIEFCAA|1977-06-29|929|4044|311|1977|3|6|29|2|1977|311|4044|Wednesday|1977Q2|N|N|N|2443296|2443446|2442959|2443234|N|N|N|N|N| +2443325|AAAAAAAANDIEFCAA|1977-06-30|929|4044|311|1977|4|6|30|2|1977|311|4044|Thursday|1977Q2|N|N|N|2443296|2443446|2442960|2443235|N|N|N|N|N| +2443326|AAAAAAAAODIEFCAA|1977-07-01|930|4044|311|1977|5|7|1|2|1977|311|4044|Friday|1977Q2|N|Y|N|2443326|2443506|2442961|2443235|N|N|N|N|N| +2443327|AAAAAAAAPDIEFCAA|1977-07-02|930|4044|311|1977|6|7|2|3|1977|311|4044|Saturday|1977Q3|N|Y|N|2443326|2443506|2442962|2443236|N|N|N|N|N| +2443328|AAAAAAAAAEIEFCAA|1977-07-03|930|4044|311|1977|0|7|3|3|1977|311|4044|Sunday|1977Q3|N|N|N|2443326|2443506|2442963|2443237|N|N|N|N|N| +2443329|AAAAAAAABEIEFCAA|1977-07-04|930|4044|311|1977|1|7|4|3|1977|311|4044|Monday|1977Q3|N|N|N|2443326|2443506|2442964|2443238|N|N|N|N|N| +2443330|AAAAAAAACEIEFCAA|1977-07-05|930|4045|311|1977|2|7|5|3|1977|311|4045|Tuesday|1977Q3|Y|N|N|2443326|2443506|2442965|2443239|N|N|N|N|N| +2443331|AAAAAAAADEIEFCAA|1977-07-06|930|4045|311|1977|3|7|6|3|1977|311|4045|Wednesday|1977Q3|N|N|Y|2443326|2443506|2442966|2443240|N|N|N|N|N| +2443332|AAAAAAAAEEIEFCAA|1977-07-07|930|4045|311|1977|4|7|7|3|1977|311|4045|Thursday|1977Q3|N|N|N|2443326|2443506|2442967|2443241|N|N|N|N|N| +2443333|AAAAAAAAFEIEFCAA|1977-07-08|930|4045|311|1977|5|7|8|3|1977|311|4045|Friday|1977Q3|N|Y|N|2443326|2443506|2442968|2443242|N|N|N|N|N| +2443334|AAAAAAAAGEIEFCAA|1977-07-09|930|4045|311|1977|6|7|9|3|1977|311|4045|Saturday|1977Q3|N|Y|N|2443326|2443506|2442969|2443243|N|N|N|N|N| +2443335|AAAAAAAAHEIEFCAA|1977-07-10|930|4045|311|1977|0|7|10|3|1977|311|4045|Sunday|1977Q3|N|N|N|2443326|2443506|2442970|2443244|N|N|N|N|N| +2443336|AAAAAAAAIEIEFCAA|1977-07-11|930|4045|311|1977|1|7|11|3|1977|311|4045|Monday|1977Q3|N|N|N|2443326|2443506|2442971|2443245|N|N|N|N|N| +2443337|AAAAAAAAJEIEFCAA|1977-07-12|930|4046|311|1977|2|7|12|3|1977|311|4046|Tuesday|1977Q3|N|N|N|2443326|2443506|2442972|2443246|N|N|N|N|N| +2443338|AAAAAAAAKEIEFCAA|1977-07-13|930|4046|311|1977|3|7|13|3|1977|311|4046|Wednesday|1977Q3|N|N|N|2443326|2443506|2442973|2443247|N|N|N|N|N| +2443339|AAAAAAAALEIEFCAA|1977-07-14|930|4046|311|1977|4|7|14|3|1977|311|4046|Thursday|1977Q3|N|N|N|2443326|2443506|2442974|2443248|N|N|N|N|N| +2443340|AAAAAAAAMEIEFCAA|1977-07-15|930|4046|311|1977|5|7|15|3|1977|311|4046|Friday|1977Q3|N|Y|N|2443326|2443506|2442975|2443249|N|N|N|N|N| +2443341|AAAAAAAANEIEFCAA|1977-07-16|930|4046|311|1977|6|7|16|3|1977|311|4046|Saturday|1977Q3|N|Y|N|2443326|2443506|2442976|2443250|N|N|N|N|N| +2443342|AAAAAAAAOEIEFCAA|1977-07-17|930|4046|311|1977|0|7|17|3|1977|311|4046|Sunday|1977Q3|N|N|N|2443326|2443506|2442977|2443251|N|N|N|N|N| +2443343|AAAAAAAAPEIEFCAA|1977-07-18|930|4046|311|1977|1|7|18|3|1977|311|4046|Monday|1977Q3|N|N|N|2443326|2443506|2442978|2443252|N|N|N|N|N| +2443344|AAAAAAAAAFIEFCAA|1977-07-19|930|4047|311|1977|2|7|19|3|1977|311|4047|Tuesday|1977Q3|N|N|N|2443326|2443506|2442979|2443253|N|N|N|N|N| +2443345|AAAAAAAABFIEFCAA|1977-07-20|930|4047|311|1977|3|7|20|3|1977|311|4047|Wednesday|1977Q3|N|N|N|2443326|2443506|2442980|2443254|N|N|N|N|N| +2443346|AAAAAAAACFIEFCAA|1977-07-21|930|4047|311|1977|4|7|21|3|1977|311|4047|Thursday|1977Q3|N|N|N|2443326|2443506|2442981|2443255|N|N|N|N|N| +2443347|AAAAAAAADFIEFCAA|1977-07-22|930|4047|311|1977|5|7|22|3|1977|311|4047|Friday|1977Q3|N|Y|N|2443326|2443506|2442982|2443256|N|N|N|N|N| +2443348|AAAAAAAAEFIEFCAA|1977-07-23|930|4047|311|1977|6|7|23|3|1977|311|4047|Saturday|1977Q3|N|Y|N|2443326|2443506|2442983|2443257|N|N|N|N|N| +2443349|AAAAAAAAFFIEFCAA|1977-07-24|930|4047|311|1977|0|7|24|3|1977|311|4047|Sunday|1977Q3|N|N|N|2443326|2443506|2442984|2443258|N|N|N|N|N| +2443350|AAAAAAAAGFIEFCAA|1977-07-25|930|4047|311|1977|1|7|25|3|1977|311|4047|Monday|1977Q3|N|N|N|2443326|2443506|2442985|2443259|N|N|N|N|N| +2443351|AAAAAAAAHFIEFCAA|1977-07-26|930|4048|311|1977|2|7|26|3|1977|311|4048|Tuesday|1977Q3|N|N|N|2443326|2443506|2442986|2443260|N|N|N|N|N| +2443352|AAAAAAAAIFIEFCAA|1977-07-27|930|4048|311|1977|3|7|27|3|1977|311|4048|Wednesday|1977Q3|N|N|N|2443326|2443506|2442987|2443261|N|N|N|N|N| +2443353|AAAAAAAAJFIEFCAA|1977-07-28|930|4048|311|1977|4|7|28|3|1977|311|4048|Thursday|1977Q3|N|N|N|2443326|2443506|2442988|2443262|N|N|N|N|N| +2443354|AAAAAAAAKFIEFCAA|1977-07-29|930|4048|311|1977|5|7|29|3|1977|311|4048|Friday|1977Q3|N|Y|N|2443326|2443506|2442989|2443263|N|N|N|N|N| +2443355|AAAAAAAALFIEFCAA|1977-07-30|930|4048|311|1977|6|7|30|3|1977|311|4048|Saturday|1977Q3|N|Y|N|2443326|2443506|2442990|2443264|N|N|N|N|N| +2443356|AAAAAAAAMFIEFCAA|1977-07-31|930|4048|311|1977|0|7|31|3|1977|311|4048|Sunday|1977Q3|N|N|N|2443326|2443506|2442991|2443265|N|N|N|N|N| +2443357|AAAAAAAANFIEFCAA|1977-08-01|931|4048|311|1977|1|8|1|3|1977|311|4048|Monday|1977Q3|N|N|N|2443357|2443568|2442992|2443266|N|N|N|N|N| +2443358|AAAAAAAAOFIEFCAA|1977-08-02|931|4049|311|1977|2|8|2|3|1977|311|4049|Tuesday|1977Q3|N|N|N|2443357|2443568|2442993|2443267|N|N|N|N|N| +2443359|AAAAAAAAPFIEFCAA|1977-08-03|931|4049|311|1977|3|8|3|3|1977|311|4049|Wednesday|1977Q3|N|N|N|2443357|2443568|2442994|2443268|N|N|N|N|N| +2443360|AAAAAAAAAGIEFCAA|1977-08-04|931|4049|311|1977|4|8|4|3|1977|311|4049|Thursday|1977Q3|N|N|N|2443357|2443568|2442995|2443269|N|N|N|N|N| +2443361|AAAAAAAABGIEFCAA|1977-08-05|931|4049|311|1977|5|8|5|3|1977|311|4049|Friday|1977Q3|N|Y|N|2443357|2443568|2442996|2443270|N|N|N|N|N| +2443362|AAAAAAAACGIEFCAA|1977-08-06|931|4049|311|1977|6|8|6|3|1977|311|4049|Saturday|1977Q3|N|Y|N|2443357|2443568|2442997|2443271|N|N|N|N|N| +2443363|AAAAAAAADGIEFCAA|1977-08-07|931|4049|311|1977|0|8|7|3|1977|311|4049|Sunday|1977Q3|N|N|N|2443357|2443568|2442998|2443272|N|N|N|N|N| +2443364|AAAAAAAAEGIEFCAA|1977-08-08|931|4049|311|1977|1|8|8|3|1977|311|4049|Monday|1977Q3|N|N|N|2443357|2443568|2442999|2443273|N|N|N|N|N| +2443365|AAAAAAAAFGIEFCAA|1977-08-09|931|4050|311|1977|2|8|9|3|1977|311|4050|Tuesday|1977Q3|N|N|N|2443357|2443568|2443000|2443274|N|N|N|N|N| +2443366|AAAAAAAAGGIEFCAA|1977-08-10|931|4050|311|1977|3|8|10|3|1977|311|4050|Wednesday|1977Q3|N|N|N|2443357|2443568|2443001|2443275|N|N|N|N|N| +2443367|AAAAAAAAHGIEFCAA|1977-08-11|931|4050|311|1977|4|8|11|3|1977|311|4050|Thursday|1977Q3|N|N|N|2443357|2443568|2443002|2443276|N|N|N|N|N| +2443368|AAAAAAAAIGIEFCAA|1977-08-12|931|4050|311|1977|5|8|12|3|1977|311|4050|Friday|1977Q3|N|Y|N|2443357|2443568|2443003|2443277|N|N|N|N|N| +2443369|AAAAAAAAJGIEFCAA|1977-08-13|931|4050|311|1977|6|8|13|3|1977|311|4050|Saturday|1977Q3|N|Y|N|2443357|2443568|2443004|2443278|N|N|N|N|N| +2443370|AAAAAAAAKGIEFCAA|1977-08-14|931|4050|311|1977|0|8|14|3|1977|311|4050|Sunday|1977Q3|N|N|N|2443357|2443568|2443005|2443279|N|N|N|N|N| +2443371|AAAAAAAALGIEFCAA|1977-08-15|931|4050|311|1977|1|8|15|3|1977|311|4050|Monday|1977Q3|N|N|N|2443357|2443568|2443006|2443280|N|N|N|N|N| +2443372|AAAAAAAAMGIEFCAA|1977-08-16|931|4051|311|1977|2|8|16|3|1977|311|4051|Tuesday|1977Q3|N|N|N|2443357|2443568|2443007|2443281|N|N|N|N|N| +2443373|AAAAAAAANGIEFCAA|1977-08-17|931|4051|311|1977|3|8|17|3|1977|311|4051|Wednesday|1977Q3|N|N|N|2443357|2443568|2443008|2443282|N|N|N|N|N| +2443374|AAAAAAAAOGIEFCAA|1977-08-18|931|4051|311|1977|4|8|18|3|1977|311|4051|Thursday|1977Q3|N|N|N|2443357|2443568|2443009|2443283|N|N|N|N|N| +2443375|AAAAAAAAPGIEFCAA|1977-08-19|931|4051|311|1977|5|8|19|3|1977|311|4051|Friday|1977Q3|N|Y|N|2443357|2443568|2443010|2443284|N|N|N|N|N| +2443376|AAAAAAAAAHIEFCAA|1977-08-20|931|4051|311|1977|6|8|20|3|1977|311|4051|Saturday|1977Q3|N|Y|N|2443357|2443568|2443011|2443285|N|N|N|N|N| +2443377|AAAAAAAABHIEFCAA|1977-08-21|931|4051|311|1977|0|8|21|3|1977|311|4051|Sunday|1977Q3|N|N|N|2443357|2443568|2443012|2443286|N|N|N|N|N| +2443378|AAAAAAAACHIEFCAA|1977-08-22|931|4051|311|1977|1|8|22|3|1977|311|4051|Monday|1977Q3|N|N|N|2443357|2443568|2443013|2443287|N|N|N|N|N| +2443379|AAAAAAAADHIEFCAA|1977-08-23|931|4052|311|1977|2|8|23|3|1977|311|4052|Tuesday|1977Q3|N|N|N|2443357|2443568|2443014|2443288|N|N|N|N|N| +2443380|AAAAAAAAEHIEFCAA|1977-08-24|931|4052|311|1977|3|8|24|3|1977|311|4052|Wednesday|1977Q3|N|N|N|2443357|2443568|2443015|2443289|N|N|N|N|N| +2443381|AAAAAAAAFHIEFCAA|1977-08-25|931|4052|311|1977|4|8|25|3|1977|311|4052|Thursday|1977Q3|N|N|N|2443357|2443568|2443016|2443290|N|N|N|N|N| +2443382|AAAAAAAAGHIEFCAA|1977-08-26|931|4052|311|1977|5|8|26|3|1977|311|4052|Friday|1977Q3|N|Y|N|2443357|2443568|2443017|2443291|N|N|N|N|N| +2443383|AAAAAAAAHHIEFCAA|1977-08-27|931|4052|311|1977|6|8|27|3|1977|311|4052|Saturday|1977Q3|N|Y|N|2443357|2443568|2443018|2443292|N|N|N|N|N| +2443384|AAAAAAAAIHIEFCAA|1977-08-28|931|4052|311|1977|0|8|28|3|1977|311|4052|Sunday|1977Q3|N|N|N|2443357|2443568|2443019|2443293|N|N|N|N|N| +2443385|AAAAAAAAJHIEFCAA|1977-08-29|931|4052|311|1977|1|8|29|3|1977|311|4052|Monday|1977Q3|N|N|N|2443357|2443568|2443020|2443294|N|N|N|N|N| +2443386|AAAAAAAAKHIEFCAA|1977-08-30|931|4053|311|1977|2|8|30|3|1977|311|4053|Tuesday|1977Q3|N|N|N|2443357|2443568|2443021|2443295|N|N|N|N|N| +2443387|AAAAAAAALHIEFCAA|1977-08-31|931|4053|311|1977|3|8|31|3|1977|311|4053|Wednesday|1977Q3|N|N|N|2443357|2443568|2443022|2443296|N|N|N|N|N| +2443388|AAAAAAAAMHIEFCAA|1977-09-01|932|4053|312|1977|4|9|1|3|1977|312|4053|Thursday|1977Q3|N|N|N|2443388|2443630|2443023|2443297|N|N|N|N|N| +2443389|AAAAAAAANHIEFCAA|1977-09-02|932|4053|312|1977|5|9|2|3|1977|312|4053|Friday|1977Q3|N|Y|N|2443388|2443630|2443024|2443298|N|N|N|N|N| +2443390|AAAAAAAAOHIEFCAA|1977-09-03|932|4053|312|1977|6|9|3|3|1977|312|4053|Saturday|1977Q3|N|Y|N|2443388|2443630|2443025|2443299|N|N|N|N|N| +2443391|AAAAAAAAPHIEFCAA|1977-09-04|932|4053|312|1977|0|9|4|3|1977|312|4053|Sunday|1977Q3|N|N|N|2443388|2443630|2443026|2443300|N|N|N|N|N| +2443392|AAAAAAAAAIIEFCAA|1977-09-05|932|4053|312|1977|1|9|5|3|1977|312|4053|Monday|1977Q3|N|N|N|2443388|2443630|2443027|2443301|N|N|N|N|N| +2443393|AAAAAAAABIIEFCAA|1977-09-06|932|4054|312|1977|2|9|6|3|1977|312|4054|Tuesday|1977Q3|N|N|N|2443388|2443630|2443028|2443302|N|N|N|N|N| +2443394|AAAAAAAACIIEFCAA|1977-09-07|932|4054|312|1977|3|9|7|3|1977|312|4054|Wednesday|1977Q3|N|N|N|2443388|2443630|2443029|2443303|N|N|N|N|N| +2443395|AAAAAAAADIIEFCAA|1977-09-08|932|4054|312|1977|4|9|8|3|1977|312|4054|Thursday|1977Q3|N|N|N|2443388|2443630|2443030|2443304|N|N|N|N|N| +2443396|AAAAAAAAEIIEFCAA|1977-09-09|932|4054|312|1977|5|9|9|3|1977|312|4054|Friday|1977Q3|N|Y|N|2443388|2443630|2443031|2443305|N|N|N|N|N| +2443397|AAAAAAAAFIIEFCAA|1977-09-10|932|4054|312|1977|6|9|10|3|1977|312|4054|Saturday|1977Q3|N|Y|N|2443388|2443630|2443032|2443306|N|N|N|N|N| +2443398|AAAAAAAAGIIEFCAA|1977-09-11|932|4054|312|1977|0|9|11|3|1977|312|4054|Sunday|1977Q3|N|N|N|2443388|2443630|2443033|2443307|N|N|N|N|N| +2443399|AAAAAAAAHIIEFCAA|1977-09-12|932|4054|312|1977|1|9|12|3|1977|312|4054|Monday|1977Q3|N|N|N|2443388|2443630|2443034|2443308|N|N|N|N|N| +2443400|AAAAAAAAIIIEFCAA|1977-09-13|932|4055|312|1977|2|9|13|3|1977|312|4055|Tuesday|1977Q3|N|N|N|2443388|2443630|2443035|2443309|N|N|N|N|N| +2443401|AAAAAAAAJIIEFCAA|1977-09-14|932|4055|312|1977|3|9|14|3|1977|312|4055|Wednesday|1977Q3|N|N|N|2443388|2443630|2443036|2443310|N|N|N|N|N| +2443402|AAAAAAAAKIIEFCAA|1977-09-15|932|4055|312|1977|4|9|15|3|1977|312|4055|Thursday|1977Q3|N|N|N|2443388|2443630|2443037|2443311|N|N|N|N|N| +2443403|AAAAAAAALIIEFCAA|1977-09-16|932|4055|312|1977|5|9|16|3|1977|312|4055|Friday|1977Q3|N|Y|N|2443388|2443630|2443038|2443312|N|N|N|N|N| +2443404|AAAAAAAAMIIEFCAA|1977-09-17|932|4055|312|1977|6|9|17|3|1977|312|4055|Saturday|1977Q3|N|Y|N|2443388|2443630|2443039|2443313|N|N|N|N|N| +2443405|AAAAAAAANIIEFCAA|1977-09-18|932|4055|312|1977|0|9|18|3|1977|312|4055|Sunday|1977Q3|N|N|N|2443388|2443630|2443040|2443314|N|N|N|N|N| +2443406|AAAAAAAAOIIEFCAA|1977-09-19|932|4055|312|1977|1|9|19|3|1977|312|4055|Monday|1977Q3|N|N|N|2443388|2443630|2443041|2443315|N|N|N|N|N| +2443407|AAAAAAAAPIIEFCAA|1977-09-20|932|4056|312|1977|2|9|20|3|1977|312|4056|Tuesday|1977Q3|N|N|N|2443388|2443630|2443042|2443316|N|N|N|N|N| +2443408|AAAAAAAAAJIEFCAA|1977-09-21|932|4056|312|1977|3|9|21|3|1977|312|4056|Wednesday|1977Q3|N|N|N|2443388|2443630|2443043|2443317|N|N|N|N|N| +2443409|AAAAAAAABJIEFCAA|1977-09-22|932|4056|312|1977|4|9|22|3|1977|312|4056|Thursday|1977Q3|N|N|N|2443388|2443630|2443044|2443318|N|N|N|N|N| +2443410|AAAAAAAACJIEFCAA|1977-09-23|932|4056|312|1977|5|9|23|3|1977|312|4056|Friday|1977Q3|N|Y|N|2443388|2443630|2443045|2443319|N|N|N|N|N| +2443411|AAAAAAAADJIEFCAA|1977-09-24|932|4056|312|1977|6|9|24|3|1977|312|4056|Saturday|1977Q3|N|Y|N|2443388|2443630|2443046|2443320|N|N|N|N|N| +2443412|AAAAAAAAEJIEFCAA|1977-09-25|932|4056|312|1977|0|9|25|3|1977|312|4056|Sunday|1977Q3|N|N|N|2443388|2443630|2443047|2443321|N|N|N|N|N| +2443413|AAAAAAAAFJIEFCAA|1977-09-26|932|4056|312|1977|1|9|26|3|1977|312|4056|Monday|1977Q3|N|N|N|2443388|2443630|2443048|2443322|N|N|N|N|N| +2443414|AAAAAAAAGJIEFCAA|1977-09-27|932|4057|312|1977|2|9|27|3|1977|312|4057|Tuesday|1977Q3|N|N|N|2443388|2443630|2443049|2443323|N|N|N|N|N| +2443415|AAAAAAAAHJIEFCAA|1977-09-28|932|4057|312|1977|3|9|28|3|1977|312|4057|Wednesday|1977Q3|N|N|N|2443388|2443630|2443050|2443324|N|N|N|N|N| +2443416|AAAAAAAAIJIEFCAA|1977-09-29|932|4057|312|1977|4|9|29|3|1977|312|4057|Thursday|1977Q3|N|N|N|2443388|2443630|2443051|2443325|N|N|N|N|N| +2443417|AAAAAAAAJJIEFCAA|1977-09-30|932|4057|312|1977|5|9|30|3|1977|312|4057|Friday|1977Q3|N|Y|N|2443388|2443630|2443052|2443326|N|N|N|N|N| +2443418|AAAAAAAAKJIEFCAA|1977-10-01|933|4057|312|1977|6|10|1|3|1977|312|4057|Saturday|1977Q3|N|Y|N|2443418|2443690|2443053|2443326|N|N|N|N|N| +2443419|AAAAAAAALJIEFCAA|1977-10-02|933|4057|312|1977|0|10|2|4|1977|312|4057|Sunday|1977Q4|N|N|N|2443418|2443690|2443054|2443327|N|N|N|N|N| +2443420|AAAAAAAAMJIEFCAA|1977-10-03|933|4057|312|1977|1|10|3|4|1977|312|4057|Monday|1977Q4|N|N|N|2443418|2443690|2443055|2443328|N|N|N|N|N| +2443421|AAAAAAAANJIEFCAA|1977-10-04|933|4058|312|1977|2|10|4|4|1977|312|4058|Tuesday|1977Q4|N|N|N|2443418|2443690|2443056|2443329|N|N|N|N|N| +2443422|AAAAAAAAOJIEFCAA|1977-10-05|933|4058|312|1977|3|10|5|4|1977|312|4058|Wednesday|1977Q4|N|N|N|2443418|2443690|2443057|2443330|N|N|N|N|N| +2443423|AAAAAAAAPJIEFCAA|1977-10-06|933|4058|312|1977|4|10|6|4|1977|312|4058|Thursday|1977Q4|N|N|N|2443418|2443690|2443058|2443331|N|N|N|N|N| +2443424|AAAAAAAAAKIEFCAA|1977-10-07|933|4058|312|1977|5|10|7|4|1977|312|4058|Friday|1977Q4|N|Y|N|2443418|2443690|2443059|2443332|N|N|N|N|N| +2443425|AAAAAAAABKIEFCAA|1977-10-08|933|4058|312|1977|6|10|8|4|1977|312|4058|Saturday|1977Q4|N|Y|N|2443418|2443690|2443060|2443333|N|N|N|N|N| +2443426|AAAAAAAACKIEFCAA|1977-10-09|933|4058|312|1977|0|10|9|4|1977|312|4058|Sunday|1977Q4|N|N|N|2443418|2443690|2443061|2443334|N|N|N|N|N| +2443427|AAAAAAAADKIEFCAA|1977-10-10|933|4058|312|1977|1|10|10|4|1977|312|4058|Monday|1977Q4|N|N|N|2443418|2443690|2443062|2443335|N|N|N|N|N| +2443428|AAAAAAAAEKIEFCAA|1977-10-11|933|4059|312|1977|2|10|11|4|1977|312|4059|Tuesday|1977Q4|N|N|N|2443418|2443690|2443063|2443336|N|N|N|N|N| +2443429|AAAAAAAAFKIEFCAA|1977-10-12|933|4059|312|1977|3|10|12|4|1977|312|4059|Wednesday|1977Q4|N|N|N|2443418|2443690|2443064|2443337|N|N|N|N|N| +2443430|AAAAAAAAGKIEFCAA|1977-10-13|933|4059|312|1977|4|10|13|4|1977|312|4059|Thursday|1977Q4|N|N|N|2443418|2443690|2443065|2443338|N|N|N|N|N| +2443431|AAAAAAAAHKIEFCAA|1977-10-14|933|4059|312|1977|5|10|14|4|1977|312|4059|Friday|1977Q4|N|Y|N|2443418|2443690|2443066|2443339|N|N|N|N|N| +2443432|AAAAAAAAIKIEFCAA|1977-10-15|933|4059|312|1977|6|10|15|4|1977|312|4059|Saturday|1977Q4|N|Y|N|2443418|2443690|2443067|2443340|N|N|N|N|N| +2443433|AAAAAAAAJKIEFCAA|1977-10-16|933|4059|312|1977|0|10|16|4|1977|312|4059|Sunday|1977Q4|N|N|N|2443418|2443690|2443068|2443341|N|N|N|N|N| +2443434|AAAAAAAAKKIEFCAA|1977-10-17|933|4059|312|1977|1|10|17|4|1977|312|4059|Monday|1977Q4|N|N|N|2443418|2443690|2443069|2443342|N|N|N|N|N| +2443435|AAAAAAAALKIEFCAA|1977-10-18|933|4060|312|1977|2|10|18|4|1977|312|4060|Tuesday|1977Q4|N|N|N|2443418|2443690|2443070|2443343|N|N|N|N|N| +2443436|AAAAAAAAMKIEFCAA|1977-10-19|933|4060|312|1977|3|10|19|4|1977|312|4060|Wednesday|1977Q4|N|N|N|2443418|2443690|2443071|2443344|N|N|N|N|N| +2443437|AAAAAAAANKIEFCAA|1977-10-20|933|4060|312|1977|4|10|20|4|1977|312|4060|Thursday|1977Q4|N|N|N|2443418|2443690|2443072|2443345|N|N|N|N|N| +2443438|AAAAAAAAOKIEFCAA|1977-10-21|933|4060|312|1977|5|10|21|4|1977|312|4060|Friday|1977Q4|N|Y|N|2443418|2443690|2443073|2443346|N|N|N|N|N| +2443439|AAAAAAAAPKIEFCAA|1977-10-22|933|4060|312|1977|6|10|22|4|1977|312|4060|Saturday|1977Q4|N|Y|N|2443418|2443690|2443074|2443347|N|N|N|N|N| +2443440|AAAAAAAAALIEFCAA|1977-10-23|933|4060|312|1977|0|10|23|4|1977|312|4060|Sunday|1977Q4|N|N|N|2443418|2443690|2443075|2443348|N|N|N|N|N| +2443441|AAAAAAAABLIEFCAA|1977-10-24|933|4060|312|1977|1|10|24|4|1977|312|4060|Monday|1977Q4|N|N|N|2443418|2443690|2443076|2443349|N|N|N|N|N| +2443442|AAAAAAAACLIEFCAA|1977-10-25|933|4061|312|1977|2|10|25|4|1977|312|4061|Tuesday|1977Q4|N|N|N|2443418|2443690|2443077|2443350|N|N|N|N|N| +2443443|AAAAAAAADLIEFCAA|1977-10-26|933|4061|312|1977|3|10|26|4|1977|312|4061|Wednesday|1977Q4|N|N|N|2443418|2443690|2443078|2443351|N|N|N|N|N| +2443444|AAAAAAAAELIEFCAA|1977-10-27|933|4061|312|1977|4|10|27|4|1977|312|4061|Thursday|1977Q4|N|N|N|2443418|2443690|2443079|2443352|N|N|N|N|N| +2443445|AAAAAAAAFLIEFCAA|1977-10-28|933|4061|312|1977|5|10|28|4|1977|312|4061|Friday|1977Q4|N|Y|N|2443418|2443690|2443080|2443353|N|N|N|N|N| +2443446|AAAAAAAAGLIEFCAA|1977-10-29|933|4061|312|1977|6|10|29|4|1977|312|4061|Saturday|1977Q4|N|Y|N|2443418|2443690|2443081|2443354|N|N|N|N|N| +2443447|AAAAAAAAHLIEFCAA|1977-10-30|933|4061|312|1977|0|10|30|4|1977|312|4061|Sunday|1977Q4|N|N|N|2443418|2443690|2443082|2443355|N|N|N|N|N| +2443448|AAAAAAAAILIEFCAA|1977-10-31|933|4061|312|1977|1|10|31|4|1977|312|4061|Monday|1977Q4|N|N|N|2443418|2443690|2443083|2443356|N|N|N|N|N| +2443449|AAAAAAAAJLIEFCAA|1977-11-01|934|4062|312|1977|2|11|1|4|1977|312|4062|Tuesday|1977Q4|N|N|N|2443449|2443752|2443084|2443357|N|N|N|N|N| +2443450|AAAAAAAAKLIEFCAA|1977-11-02|934|4062|312|1977|3|11|2|4|1977|312|4062|Wednesday|1977Q4|N|N|N|2443449|2443752|2443085|2443358|N|N|N|N|N| +2443451|AAAAAAAALLIEFCAA|1977-11-03|934|4062|312|1977|4|11|3|4|1977|312|4062|Thursday|1977Q4|N|N|N|2443449|2443752|2443086|2443359|N|N|N|N|N| +2443452|AAAAAAAAMLIEFCAA|1977-11-04|934|4062|312|1977|5|11|4|4|1977|312|4062|Friday|1977Q4|N|Y|N|2443449|2443752|2443087|2443360|N|N|N|N|N| +2443453|AAAAAAAANLIEFCAA|1977-11-05|934|4062|312|1977|6|11|5|4|1977|312|4062|Saturday|1977Q4|N|Y|N|2443449|2443752|2443088|2443361|N|N|N|N|N| +2443454|AAAAAAAAOLIEFCAA|1977-11-06|934|4062|312|1977|0|11|6|4|1977|312|4062|Sunday|1977Q4|N|N|N|2443449|2443752|2443089|2443362|N|N|N|N|N| +2443455|AAAAAAAAPLIEFCAA|1977-11-07|934|4062|312|1977|1|11|7|4|1977|312|4062|Monday|1977Q4|N|N|N|2443449|2443752|2443090|2443363|N|N|N|N|N| +2443456|AAAAAAAAAMIEFCAA|1977-11-08|934|4063|312|1977|2|11|8|4|1977|312|4063|Tuesday|1977Q4|N|N|N|2443449|2443752|2443091|2443364|N|N|N|N|N| +2443457|AAAAAAAABMIEFCAA|1977-11-09|934|4063|312|1977|3|11|9|4|1977|312|4063|Wednesday|1977Q4|N|N|N|2443449|2443752|2443092|2443365|N|N|N|N|N| +2443458|AAAAAAAACMIEFCAA|1977-11-10|934|4063|312|1977|4|11|10|4|1977|312|4063|Thursday|1977Q4|N|N|N|2443449|2443752|2443093|2443366|N|N|N|N|N| +2443459|AAAAAAAADMIEFCAA|1977-11-11|934|4063|312|1977|5|11|11|4|1977|312|4063|Friday|1977Q4|N|Y|N|2443449|2443752|2443094|2443367|N|N|N|N|N| +2443460|AAAAAAAAEMIEFCAA|1977-11-12|934|4063|312|1977|6|11|12|4|1977|312|4063|Saturday|1977Q4|N|Y|N|2443449|2443752|2443095|2443368|N|N|N|N|N| +2443461|AAAAAAAAFMIEFCAA|1977-11-13|934|4063|312|1977|0|11|13|4|1977|312|4063|Sunday|1977Q4|N|N|N|2443449|2443752|2443096|2443369|N|N|N|N|N| +2443462|AAAAAAAAGMIEFCAA|1977-11-14|934|4063|312|1977|1|11|14|4|1977|312|4063|Monday|1977Q4|N|N|N|2443449|2443752|2443097|2443370|N|N|N|N|N| +2443463|AAAAAAAAHMIEFCAA|1977-11-15|934|4064|312|1977|2|11|15|4|1977|312|4064|Tuesday|1977Q4|N|N|N|2443449|2443752|2443098|2443371|N|N|N|N|N| +2443464|AAAAAAAAIMIEFCAA|1977-11-16|934|4064|312|1977|3|11|16|4|1977|312|4064|Wednesday|1977Q4|N|N|N|2443449|2443752|2443099|2443372|N|N|N|N|N| +2443465|AAAAAAAAJMIEFCAA|1977-11-17|934|4064|312|1977|4|11|17|4|1977|312|4064|Thursday|1977Q4|N|N|N|2443449|2443752|2443100|2443373|N|N|N|N|N| +2443466|AAAAAAAAKMIEFCAA|1977-11-18|934|4064|312|1977|5|11|18|4|1977|312|4064|Friday|1977Q4|N|Y|N|2443449|2443752|2443101|2443374|N|N|N|N|N| +2443467|AAAAAAAALMIEFCAA|1977-11-19|934|4064|312|1977|6|11|19|4|1977|312|4064|Saturday|1977Q4|N|Y|N|2443449|2443752|2443102|2443375|N|N|N|N|N| +2443468|AAAAAAAAMMIEFCAA|1977-11-20|934|4064|312|1977|0|11|20|4|1977|312|4064|Sunday|1977Q4|N|N|N|2443449|2443752|2443103|2443376|N|N|N|N|N| +2443469|AAAAAAAANMIEFCAA|1977-11-21|934|4064|312|1977|1|11|21|4|1977|312|4064|Monday|1977Q4|N|N|N|2443449|2443752|2443104|2443377|N|N|N|N|N| +2443470|AAAAAAAAOMIEFCAA|1977-11-22|934|4065|312|1977|2|11|22|4|1977|312|4065|Tuesday|1977Q4|N|N|N|2443449|2443752|2443105|2443378|N|N|N|N|N| +2443471|AAAAAAAAPMIEFCAA|1977-11-23|934|4065|312|1977|3|11|23|4|1977|312|4065|Wednesday|1977Q4|N|N|N|2443449|2443752|2443106|2443379|N|N|N|N|N| +2443472|AAAAAAAAANIEFCAA|1977-11-24|934|4065|312|1977|4|11|24|4|1977|312|4065|Thursday|1977Q4|N|N|N|2443449|2443752|2443107|2443380|N|N|N|N|N| +2443473|AAAAAAAABNIEFCAA|1977-11-25|934|4065|312|1977|5|11|25|4|1977|312|4065|Friday|1977Q4|N|Y|N|2443449|2443752|2443108|2443381|N|N|N|N|N| +2443474|AAAAAAAACNIEFCAA|1977-11-26|934|4065|312|1977|6|11|26|4|1977|312|4065|Saturday|1977Q4|N|Y|N|2443449|2443752|2443109|2443382|N|N|N|N|N| +2443475|AAAAAAAADNIEFCAA|1977-11-27|934|4065|312|1977|0|11|27|4|1977|312|4065|Sunday|1977Q4|N|N|N|2443449|2443752|2443110|2443383|N|N|N|N|N| +2443476|AAAAAAAAENIEFCAA|1977-11-28|934|4065|312|1977|1|11|28|4|1977|312|4065|Monday|1977Q4|N|N|N|2443449|2443752|2443111|2443384|N|N|N|N|N| +2443477|AAAAAAAAFNIEFCAA|1977-11-29|934|4066|312|1977|2|11|29|4|1977|312|4066|Tuesday|1977Q4|N|N|N|2443449|2443752|2443112|2443385|N|N|N|N|N| +2443478|AAAAAAAAGNIEFCAA|1977-11-30|934|4066|312|1977|3|11|30|4|1977|312|4066|Wednesday|1977Q4|N|N|N|2443449|2443752|2443113|2443386|N|N|N|N|N| +2443479|AAAAAAAAHNIEFCAA|1977-12-01|935|4066|313|1977|4|12|1|4|1977|313|4066|Thursday|1977Q4|N|N|N|2443479|2443812|2443114|2443387|N|N|N|N|N| +2443480|AAAAAAAAINIEFCAA|1977-12-02|935|4066|313|1977|5|12|2|4|1977|313|4066|Friday|1977Q4|N|Y|N|2443479|2443812|2443115|2443388|N|N|N|N|N| +2443481|AAAAAAAAJNIEFCAA|1977-12-03|935|4066|313|1977|6|12|3|4|1977|313|4066|Saturday|1977Q4|N|Y|N|2443479|2443812|2443116|2443389|N|N|N|N|N| +2443482|AAAAAAAAKNIEFCAA|1977-12-04|935|4066|313|1977|0|12|4|4|1977|313|4066|Sunday|1977Q4|N|N|N|2443479|2443812|2443117|2443390|N|N|N|N|N| +2443483|AAAAAAAALNIEFCAA|1977-12-05|935|4066|313|1977|1|12|5|4|1977|313|4066|Monday|1977Q4|N|N|N|2443479|2443812|2443118|2443391|N|N|N|N|N| +2443484|AAAAAAAAMNIEFCAA|1977-12-06|935|4067|313|1977|2|12|6|4|1977|313|4067|Tuesday|1977Q4|N|N|N|2443479|2443812|2443119|2443392|N|N|N|N|N| +2443485|AAAAAAAANNIEFCAA|1977-12-07|935|4067|313|1977|3|12|7|4|1977|313|4067|Wednesday|1977Q4|N|N|N|2443479|2443812|2443120|2443393|N|N|N|N|N| +2443486|AAAAAAAAONIEFCAA|1977-12-08|935|4067|313|1977|4|12|8|4|1977|313|4067|Thursday|1977Q4|N|N|N|2443479|2443812|2443121|2443394|N|N|N|N|N| +2443487|AAAAAAAAPNIEFCAA|1977-12-09|935|4067|313|1977|5|12|9|4|1977|313|4067|Friday|1977Q4|N|Y|N|2443479|2443812|2443122|2443395|N|N|N|N|N| +2443488|AAAAAAAAAOIEFCAA|1977-12-10|935|4067|313|1977|6|12|10|4|1977|313|4067|Saturday|1977Q4|N|Y|N|2443479|2443812|2443123|2443396|N|N|N|N|N| +2443489|AAAAAAAABOIEFCAA|1977-12-11|935|4067|313|1977|0|12|11|4|1977|313|4067|Sunday|1977Q4|N|N|N|2443479|2443812|2443124|2443397|N|N|N|N|N| +2443490|AAAAAAAACOIEFCAA|1977-12-12|935|4067|313|1977|1|12|12|4|1977|313|4067|Monday|1977Q4|N|N|N|2443479|2443812|2443125|2443398|N|N|N|N|N| +2443491|AAAAAAAADOIEFCAA|1977-12-13|935|4068|313|1977|2|12|13|4|1977|313|4068|Tuesday|1977Q4|N|N|N|2443479|2443812|2443126|2443399|N|N|N|N|N| +2443492|AAAAAAAAEOIEFCAA|1977-12-14|935|4068|313|1977|3|12|14|4|1977|313|4068|Wednesday|1977Q4|N|N|N|2443479|2443812|2443127|2443400|N|N|N|N|N| +2443493|AAAAAAAAFOIEFCAA|1977-12-15|935|4068|313|1977|4|12|15|4|1977|313|4068|Thursday|1977Q4|N|N|N|2443479|2443812|2443128|2443401|N|N|N|N|N| +2443494|AAAAAAAAGOIEFCAA|1977-12-16|935|4068|313|1977|5|12|16|4|1977|313|4068|Friday|1977Q4|N|Y|N|2443479|2443812|2443129|2443402|N|N|N|N|N| +2443495|AAAAAAAAHOIEFCAA|1977-12-17|935|4068|313|1977|6|12|17|4|1977|313|4068|Saturday|1977Q4|N|Y|N|2443479|2443812|2443130|2443403|N|N|N|N|N| +2443496|AAAAAAAAIOIEFCAA|1977-12-18|935|4068|313|1977|0|12|18|4|1977|313|4068|Sunday|1977Q4|N|N|N|2443479|2443812|2443131|2443404|N|N|N|N|N| +2443497|AAAAAAAAJOIEFCAA|1977-12-19|935|4068|313|1977|1|12|19|4|1977|313|4068|Monday|1977Q4|N|N|N|2443479|2443812|2443132|2443405|N|N|N|N|N| +2443498|AAAAAAAAKOIEFCAA|1977-12-20|935|4069|313|1977|2|12|20|4|1977|313|4069|Tuesday|1977Q4|N|N|N|2443479|2443812|2443133|2443406|N|N|N|N|N| +2443499|AAAAAAAALOIEFCAA|1977-12-21|935|4069|313|1977|3|12|21|4|1977|313|4069|Wednesday|1977Q4|N|N|N|2443479|2443812|2443134|2443407|N|N|N|N|N| +2443500|AAAAAAAAMOIEFCAA|1977-12-22|935|4069|313|1977|4|12|22|4|1977|313|4069|Thursday|1977Q4|N|N|N|2443479|2443812|2443135|2443408|N|N|N|N|N| +2443501|AAAAAAAANOIEFCAA|1977-12-23|935|4069|313|1977|5|12|23|4|1977|313|4069|Friday|1977Q4|N|Y|N|2443479|2443812|2443136|2443409|N|N|N|N|N| +2443502|AAAAAAAAOOIEFCAA|1977-12-24|935|4069|313|1977|6|12|24|4|1977|313|4069|Saturday|1977Q4|N|Y|N|2443479|2443812|2443137|2443410|N|N|N|N|N| +2443503|AAAAAAAAPOIEFCAA|1977-12-25|935|4069|313|1977|0|12|25|4|1977|313|4069|Sunday|1977Q4|N|N|N|2443479|2443812|2443138|2443411|N|N|N|N|N| +2443504|AAAAAAAAAPIEFCAA|1977-12-26|935|4069|313|1977|1|12|26|4|1977|313|4069|Monday|1977Q4|Y|N|N|2443479|2443812|2443139|2443412|N|N|N|N|N| +2443505|AAAAAAAABPIEFCAA|1977-12-27|935|4070|313|1977|2|12|27|4|1977|313|4070|Tuesday|1977Q4|N|N|Y|2443479|2443812|2443140|2443413|N|N|N|N|N| +2443506|AAAAAAAACPIEFCAA|1977-12-28|935|4070|313|1977|3|12|28|4|1977|313|4070|Wednesday|1977Q4|N|N|N|2443479|2443812|2443141|2443414|N|N|N|N|N| +2443507|AAAAAAAADPIEFCAA|1977-12-29|935|4070|313|1977|4|12|29|4|1977|313|4070|Thursday|1977Q4|N|N|N|2443479|2443812|2443142|2443415|N|N|N|N|N| +2443508|AAAAAAAAEPIEFCAA|1977-12-30|935|4070|313|1977|5|12|30|4|1977|313|4070|Friday|1977Q4|N|Y|N|2443479|2443812|2443143|2443416|N|N|N|N|N| +2443509|AAAAAAAAFPIEFCAA|1977-12-31|935|4070|313|1977|6|12|31|4|1977|313|4070|Saturday|1977Q4|N|Y|N|2443479|2443812|2443144|2443417|N|N|N|N|N| +2443510|AAAAAAAAGPIEFCAA|1978-01-01|936|4070|313|1978|0|1|1|1|1978|313|4070|Sunday|1978Q1|Y|N|N|2443510|2443509|2443145|2443418|N|N|N|N|N| +2443511|AAAAAAAAHPIEFCAA|1978-01-02|936|4070|313|1978|1|1|2|1|1978|313|4070|Monday|1978Q1|N|N|Y|2443510|2443509|2443146|2443419|N|N|N|N|N| +2443512|AAAAAAAAIPIEFCAA|1978-01-03|936|4071|313|1978|2|1|3|1|1978|313|4071|Tuesday|1978Q1|N|N|N|2443510|2443509|2443147|2443420|N|N|N|N|N| +2443513|AAAAAAAAJPIEFCAA|1978-01-04|936|4071|313|1978|3|1|4|1|1978|313|4071|Wednesday|1978Q1|N|N|N|2443510|2443509|2443148|2443421|N|N|N|N|N| +2443514|AAAAAAAAKPIEFCAA|1978-01-05|936|4071|313|1978|4|1|5|1|1978|313|4071|Thursday|1978Q1|N|N|N|2443510|2443509|2443149|2443422|N|N|N|N|N| +2443515|AAAAAAAALPIEFCAA|1978-01-06|936|4071|313|1978|5|1|6|1|1978|313|4071|Friday|1978Q1|N|Y|N|2443510|2443509|2443150|2443423|N|N|N|N|N| +2443516|AAAAAAAAMPIEFCAA|1978-01-07|936|4071|313|1978|6|1|7|1|1978|313|4071|Saturday|1978Q1|N|Y|N|2443510|2443509|2443151|2443424|N|N|N|N|N| +2443517|AAAAAAAANPIEFCAA|1978-01-08|936|4071|313|1978|0|1|8|1|1978|313|4071|Sunday|1978Q1|N|N|N|2443510|2443509|2443152|2443425|N|N|N|N|N| +2443518|AAAAAAAAOPIEFCAA|1978-01-09|936|4071|313|1978|1|1|9|1|1978|313|4071|Monday|1978Q1|N|N|N|2443510|2443509|2443153|2443426|N|N|N|N|N| +2443519|AAAAAAAAPPIEFCAA|1978-01-10|936|4072|313|1978|2|1|10|1|1978|313|4072|Tuesday|1978Q1|N|N|N|2443510|2443509|2443154|2443427|N|N|N|N|N| +2443520|AAAAAAAAAAJEFCAA|1978-01-11|936|4072|313|1978|3|1|11|1|1978|313|4072|Wednesday|1978Q1|N|N|N|2443510|2443509|2443155|2443428|N|N|N|N|N| +2443521|AAAAAAAABAJEFCAA|1978-01-12|936|4072|313|1978|4|1|12|1|1978|313|4072|Thursday|1978Q1|N|N|N|2443510|2443509|2443156|2443429|N|N|N|N|N| +2443522|AAAAAAAACAJEFCAA|1978-01-13|936|4072|313|1978|5|1|13|1|1978|313|4072|Friday|1978Q1|N|Y|N|2443510|2443509|2443157|2443430|N|N|N|N|N| +2443523|AAAAAAAADAJEFCAA|1978-01-14|936|4072|313|1978|6|1|14|1|1978|313|4072|Saturday|1978Q1|N|Y|N|2443510|2443509|2443158|2443431|N|N|N|N|N| +2443524|AAAAAAAAEAJEFCAA|1978-01-15|936|4072|313|1978|0|1|15|1|1978|313|4072|Sunday|1978Q1|N|N|N|2443510|2443509|2443159|2443432|N|N|N|N|N| +2443525|AAAAAAAAFAJEFCAA|1978-01-16|936|4072|313|1978|1|1|16|1|1978|313|4072|Monday|1978Q1|N|N|N|2443510|2443509|2443160|2443433|N|N|N|N|N| +2443526|AAAAAAAAGAJEFCAA|1978-01-17|936|4073|313|1978|2|1|17|1|1978|313|4073|Tuesday|1978Q1|N|N|N|2443510|2443509|2443161|2443434|N|N|N|N|N| +2443527|AAAAAAAAHAJEFCAA|1978-01-18|936|4073|313|1978|3|1|18|1|1978|313|4073|Wednesday|1978Q1|N|N|N|2443510|2443509|2443162|2443435|N|N|N|N|N| +2443528|AAAAAAAAIAJEFCAA|1978-01-19|936|4073|313|1978|4|1|19|1|1978|313|4073|Thursday|1978Q1|N|N|N|2443510|2443509|2443163|2443436|N|N|N|N|N| +2443529|AAAAAAAAJAJEFCAA|1978-01-20|936|4073|313|1978|5|1|20|1|1978|313|4073|Friday|1978Q1|N|Y|N|2443510|2443509|2443164|2443437|N|N|N|N|N| +2443530|AAAAAAAAKAJEFCAA|1978-01-21|936|4073|313|1978|6|1|21|1|1978|313|4073|Saturday|1978Q1|N|Y|N|2443510|2443509|2443165|2443438|N|N|N|N|N| +2443531|AAAAAAAALAJEFCAA|1978-01-22|936|4073|313|1978|0|1|22|1|1978|313|4073|Sunday|1978Q1|N|N|N|2443510|2443509|2443166|2443439|N|N|N|N|N| +2443532|AAAAAAAAMAJEFCAA|1978-01-23|936|4073|313|1978|1|1|23|1|1978|313|4073|Monday|1978Q1|N|N|N|2443510|2443509|2443167|2443440|N|N|N|N|N| +2443533|AAAAAAAANAJEFCAA|1978-01-24|936|4074|313|1978|2|1|24|1|1978|313|4074|Tuesday|1978Q1|N|N|N|2443510|2443509|2443168|2443441|N|N|N|N|N| +2443534|AAAAAAAAOAJEFCAA|1978-01-25|936|4074|313|1978|3|1|25|1|1978|313|4074|Wednesday|1978Q1|N|N|N|2443510|2443509|2443169|2443442|N|N|N|N|N| +2443535|AAAAAAAAPAJEFCAA|1978-01-26|936|4074|313|1978|4|1|26|1|1978|313|4074|Thursday|1978Q1|N|N|N|2443510|2443509|2443170|2443443|N|N|N|N|N| +2443536|AAAAAAAAABJEFCAA|1978-01-27|936|4074|313|1978|5|1|27|1|1978|313|4074|Friday|1978Q1|N|Y|N|2443510|2443509|2443171|2443444|N|N|N|N|N| +2443537|AAAAAAAABBJEFCAA|1978-01-28|936|4074|313|1978|6|1|28|1|1978|313|4074|Saturday|1978Q1|N|Y|N|2443510|2443509|2443172|2443445|N|N|N|N|N| +2443538|AAAAAAAACBJEFCAA|1978-01-29|936|4074|313|1978|0|1|29|1|1978|313|4074|Sunday|1978Q1|N|N|N|2443510|2443509|2443173|2443446|N|N|N|N|N| +2443539|AAAAAAAADBJEFCAA|1978-01-30|936|4074|313|1978|1|1|30|1|1978|313|4074|Monday|1978Q1|N|N|N|2443510|2443509|2443174|2443447|N|N|N|N|N| +2443540|AAAAAAAAEBJEFCAA|1978-01-31|936|4075|313|1978|2|1|31|1|1978|313|4075|Tuesday|1978Q1|N|N|N|2443510|2443509|2443175|2443448|N|N|N|N|N| +2443541|AAAAAAAAFBJEFCAA|1978-02-01|937|4075|313|1978|3|2|1|1|1978|313|4075|Wednesday|1978Q1|N|N|N|2443541|2443571|2443176|2443449|N|N|N|N|N| +2443542|AAAAAAAAGBJEFCAA|1978-02-02|937|4075|313|1978|4|2|2|1|1978|313|4075|Thursday|1978Q1|N|N|N|2443541|2443571|2443177|2443450|N|N|N|N|N| +2443543|AAAAAAAAHBJEFCAA|1978-02-03|937|4075|313|1978|5|2|3|1|1978|313|4075|Friday|1978Q1|N|Y|N|2443541|2443571|2443178|2443451|N|N|N|N|N| +2443544|AAAAAAAAIBJEFCAA|1978-02-04|937|4075|313|1978|6|2|4|1|1978|313|4075|Saturday|1978Q1|N|Y|N|2443541|2443571|2443179|2443452|N|N|N|N|N| +2443545|AAAAAAAAJBJEFCAA|1978-02-05|937|4075|313|1978|0|2|5|1|1978|313|4075|Sunday|1978Q1|N|N|N|2443541|2443571|2443180|2443453|N|N|N|N|N| +2443546|AAAAAAAAKBJEFCAA|1978-02-06|937|4075|313|1978|1|2|6|1|1978|313|4075|Monday|1978Q1|N|N|N|2443541|2443571|2443181|2443454|N|N|N|N|N| +2443547|AAAAAAAALBJEFCAA|1978-02-07|937|4076|313|1978|2|2|7|1|1978|313|4076|Tuesday|1978Q1|N|N|N|2443541|2443571|2443182|2443455|N|N|N|N|N| +2443548|AAAAAAAAMBJEFCAA|1978-02-08|937|4076|313|1978|3|2|8|1|1978|313|4076|Wednesday|1978Q1|N|N|N|2443541|2443571|2443183|2443456|N|N|N|N|N| +2443549|AAAAAAAANBJEFCAA|1978-02-09|937|4076|313|1978|4|2|9|1|1978|313|4076|Thursday|1978Q1|N|N|N|2443541|2443571|2443184|2443457|N|N|N|N|N| +2443550|AAAAAAAAOBJEFCAA|1978-02-10|937|4076|313|1978|5|2|10|1|1978|313|4076|Friday|1978Q1|N|Y|N|2443541|2443571|2443185|2443458|N|N|N|N|N| +2443551|AAAAAAAAPBJEFCAA|1978-02-11|937|4076|313|1978|6|2|11|1|1978|313|4076|Saturday|1978Q1|N|Y|N|2443541|2443571|2443186|2443459|N|N|N|N|N| +2443552|AAAAAAAAACJEFCAA|1978-02-12|937|4076|313|1978|0|2|12|1|1978|313|4076|Sunday|1978Q1|N|N|N|2443541|2443571|2443187|2443460|N|N|N|N|N| +2443553|AAAAAAAABCJEFCAA|1978-02-13|937|4076|313|1978|1|2|13|1|1978|313|4076|Monday|1978Q1|N|N|N|2443541|2443571|2443188|2443461|N|N|N|N|N| +2443554|AAAAAAAACCJEFCAA|1978-02-14|937|4077|313|1978|2|2|14|1|1978|313|4077|Tuesday|1978Q1|N|N|N|2443541|2443571|2443189|2443462|N|N|N|N|N| +2443555|AAAAAAAADCJEFCAA|1978-02-15|937|4077|313|1978|3|2|15|1|1978|313|4077|Wednesday|1978Q1|N|N|N|2443541|2443571|2443190|2443463|N|N|N|N|N| +2443556|AAAAAAAAECJEFCAA|1978-02-16|937|4077|313|1978|4|2|16|1|1978|313|4077|Thursday|1978Q1|N|N|N|2443541|2443571|2443191|2443464|N|N|N|N|N| +2443557|AAAAAAAAFCJEFCAA|1978-02-17|937|4077|313|1978|5|2|17|1|1978|313|4077|Friday|1978Q1|N|Y|N|2443541|2443571|2443192|2443465|N|N|N|N|N| +2443558|AAAAAAAAGCJEFCAA|1978-02-18|937|4077|313|1978|6|2|18|1|1978|313|4077|Saturday|1978Q1|N|Y|N|2443541|2443571|2443193|2443466|N|N|N|N|N| +2443559|AAAAAAAAHCJEFCAA|1978-02-19|937|4077|313|1978|0|2|19|1|1978|313|4077|Sunday|1978Q1|N|N|N|2443541|2443571|2443194|2443467|N|N|N|N|N| +2443560|AAAAAAAAICJEFCAA|1978-02-20|937|4077|313|1978|1|2|20|1|1978|313|4077|Monday|1978Q1|N|N|N|2443541|2443571|2443195|2443468|N|N|N|N|N| +2443561|AAAAAAAAJCJEFCAA|1978-02-21|937|4078|313|1978|2|2|21|1|1978|313|4078|Tuesday|1978Q1|N|N|N|2443541|2443571|2443196|2443469|N|N|N|N|N| +2443562|AAAAAAAAKCJEFCAA|1978-02-22|937|4078|313|1978|3|2|22|1|1978|313|4078|Wednesday|1978Q1|N|N|N|2443541|2443571|2443197|2443470|N|N|N|N|N| +2443563|AAAAAAAALCJEFCAA|1978-02-23|937|4078|313|1978|4|2|23|1|1978|313|4078|Thursday|1978Q1|N|N|N|2443541|2443571|2443198|2443471|N|N|N|N|N| +2443564|AAAAAAAAMCJEFCAA|1978-02-24|937|4078|313|1978|5|2|24|1|1978|313|4078|Friday|1978Q1|N|Y|N|2443541|2443571|2443199|2443472|N|N|N|N|N| +2443565|AAAAAAAANCJEFCAA|1978-02-25|937|4078|313|1978|6|2|25|1|1978|313|4078|Saturday|1978Q1|N|Y|N|2443541|2443571|2443200|2443473|N|N|N|N|N| +2443566|AAAAAAAAOCJEFCAA|1978-02-26|937|4078|313|1978|0|2|26|1|1978|313|4078|Sunday|1978Q1|N|N|N|2443541|2443571|2443201|2443474|N|N|N|N|N| +2443567|AAAAAAAAPCJEFCAA|1978-02-27|937|4078|313|1978|1|2|27|1|1978|313|4078|Monday|1978Q1|N|N|N|2443541|2443571|2443202|2443475|N|N|N|N|N| +2443568|AAAAAAAAADJEFCAA|1978-02-28|937|4079|313|1978|2|2|28|1|1978|313|4079|Tuesday|1978Q1|N|N|N|2443541|2443571|2443203|2443476|N|N|N|N|N| +2443569|AAAAAAAABDJEFCAA|1978-03-01|938|4079|314|1978|3|3|1|1|1978|314|4079|Wednesday|1978Q1|N|N|N|2443569|2443627|2443204|2443477|N|N|N|N|N| +2443570|AAAAAAAACDJEFCAA|1978-03-02|938|4079|314|1978|4|3|2|1|1978|314|4079|Thursday|1978Q1|N|N|N|2443569|2443627|2443205|2443478|N|N|N|N|N| +2443571|AAAAAAAADDJEFCAA|1978-03-03|938|4079|314|1978|5|3|3|1|1978|314|4079|Friday|1978Q1|N|Y|N|2443569|2443627|2443206|2443479|N|N|N|N|N| +2443572|AAAAAAAAEDJEFCAA|1978-03-04|938|4079|314|1978|6|3|4|1|1978|314|4079|Saturday|1978Q1|N|Y|N|2443569|2443627|2443207|2443480|N|N|N|N|N| +2443573|AAAAAAAAFDJEFCAA|1978-03-05|938|4079|314|1978|0|3|5|1|1978|314|4079|Sunday|1978Q1|N|N|N|2443569|2443627|2443208|2443481|N|N|N|N|N| +2443574|AAAAAAAAGDJEFCAA|1978-03-06|938|4079|314|1978|1|3|6|1|1978|314|4079|Monday|1978Q1|N|N|N|2443569|2443627|2443209|2443482|N|N|N|N|N| +2443575|AAAAAAAAHDJEFCAA|1978-03-07|938|4080|314|1978|2|3|7|1|1978|314|4080|Tuesday|1978Q1|N|N|N|2443569|2443627|2443210|2443483|N|N|N|N|N| +2443576|AAAAAAAAIDJEFCAA|1978-03-08|938|4080|314|1978|3|3|8|1|1978|314|4080|Wednesday|1978Q1|N|N|N|2443569|2443627|2443211|2443484|N|N|N|N|N| +2443577|AAAAAAAAJDJEFCAA|1978-03-09|938|4080|314|1978|4|3|9|1|1978|314|4080|Thursday|1978Q1|N|N|N|2443569|2443627|2443212|2443485|N|N|N|N|N| +2443578|AAAAAAAAKDJEFCAA|1978-03-10|938|4080|314|1978|5|3|10|1|1978|314|4080|Friday|1978Q1|N|Y|N|2443569|2443627|2443213|2443486|N|N|N|N|N| +2443579|AAAAAAAALDJEFCAA|1978-03-11|938|4080|314|1978|6|3|11|1|1978|314|4080|Saturday|1978Q1|N|Y|N|2443569|2443627|2443214|2443487|N|N|N|N|N| +2443580|AAAAAAAAMDJEFCAA|1978-03-12|938|4080|314|1978|0|3|12|1|1978|314|4080|Sunday|1978Q1|N|N|N|2443569|2443627|2443215|2443488|N|N|N|N|N| +2443581|AAAAAAAANDJEFCAA|1978-03-13|938|4080|314|1978|1|3|13|1|1978|314|4080|Monday|1978Q1|N|N|N|2443569|2443627|2443216|2443489|N|N|N|N|N| +2443582|AAAAAAAAODJEFCAA|1978-03-14|938|4081|314|1978|2|3|14|1|1978|314|4081|Tuesday|1978Q1|N|N|N|2443569|2443627|2443217|2443490|N|N|N|N|N| +2443583|AAAAAAAAPDJEFCAA|1978-03-15|938|4081|314|1978|3|3|15|1|1978|314|4081|Wednesday|1978Q1|N|N|N|2443569|2443627|2443218|2443491|N|N|N|N|N| +2443584|AAAAAAAAAEJEFCAA|1978-03-16|938|4081|314|1978|4|3|16|1|1978|314|4081|Thursday|1978Q1|N|N|N|2443569|2443627|2443219|2443492|N|N|N|N|N| +2443585|AAAAAAAABEJEFCAA|1978-03-17|938|4081|314|1978|5|3|17|1|1978|314|4081|Friday|1978Q1|N|Y|N|2443569|2443627|2443220|2443493|N|N|N|N|N| +2443586|AAAAAAAACEJEFCAA|1978-03-18|938|4081|314|1978|6|3|18|1|1978|314|4081|Saturday|1978Q1|N|Y|N|2443569|2443627|2443221|2443494|N|N|N|N|N| +2443587|AAAAAAAADEJEFCAA|1978-03-19|938|4081|314|1978|0|3|19|1|1978|314|4081|Sunday|1978Q1|N|N|N|2443569|2443627|2443222|2443495|N|N|N|N|N| +2443588|AAAAAAAAEEJEFCAA|1978-03-20|938|4081|314|1978|1|3|20|1|1978|314|4081|Monday|1978Q1|N|N|N|2443569|2443627|2443223|2443496|N|N|N|N|N| +2443589|AAAAAAAAFEJEFCAA|1978-03-21|938|4082|314|1978|2|3|21|1|1978|314|4082|Tuesday|1978Q1|N|N|N|2443569|2443627|2443224|2443497|N|N|N|N|N| +2443590|AAAAAAAAGEJEFCAA|1978-03-22|938|4082|314|1978|3|3|22|1|1978|314|4082|Wednesday|1978Q1|N|N|N|2443569|2443627|2443225|2443498|N|N|N|N|N| +2443591|AAAAAAAAHEJEFCAA|1978-03-23|938|4082|314|1978|4|3|23|1|1978|314|4082|Thursday|1978Q1|N|N|N|2443569|2443627|2443226|2443499|N|N|N|N|N| +2443592|AAAAAAAAIEJEFCAA|1978-03-24|938|4082|314|1978|5|3|24|1|1978|314|4082|Friday|1978Q1|N|Y|N|2443569|2443627|2443227|2443500|N|N|N|N|N| +2443593|AAAAAAAAJEJEFCAA|1978-03-25|938|4082|314|1978|6|3|25|1|1978|314|4082|Saturday|1978Q1|N|Y|N|2443569|2443627|2443228|2443501|N|N|N|N|N| +2443594|AAAAAAAAKEJEFCAA|1978-03-26|938|4082|314|1978|0|3|26|1|1978|314|4082|Sunday|1978Q1|N|N|N|2443569|2443627|2443229|2443502|N|N|N|N|N| +2443595|AAAAAAAALEJEFCAA|1978-03-27|938|4082|314|1978|1|3|27|1|1978|314|4082|Monday|1978Q1|N|N|N|2443569|2443627|2443230|2443503|N|N|N|N|N| +2443596|AAAAAAAAMEJEFCAA|1978-03-28|938|4083|314|1978|2|3|28|1|1978|314|4083|Tuesday|1978Q1|N|N|N|2443569|2443627|2443231|2443504|N|N|N|N|N| +2443597|AAAAAAAANEJEFCAA|1978-03-29|938|4083|314|1978|3|3|29|1|1978|314|4083|Wednesday|1978Q1|N|N|N|2443569|2443627|2443232|2443505|N|N|N|N|N| +2443598|AAAAAAAAOEJEFCAA|1978-03-30|938|4083|314|1978|4|3|30|1|1978|314|4083|Thursday|1978Q1|N|N|N|2443569|2443627|2443233|2443506|N|N|N|N|N| +2443599|AAAAAAAAPEJEFCAA|1978-03-31|938|4083|314|1978|5|3|31|1|1978|314|4083|Friday|1978Q1|N|Y|N|2443569|2443627|2443234|2443507|N|N|N|N|N| +2443600|AAAAAAAAAFJEFCAA|1978-04-01|939|4083|314|1978|6|4|1|1|1978|314|4083|Saturday|1978Q1|N|Y|N|2443600|2443689|2443235|2443510|N|N|N|N|N| +2443601|AAAAAAAABFJEFCAA|1978-04-02|939|4083|314|1978|0|4|2|2|1978|314|4083|Sunday|1978Q2|N|N|N|2443600|2443689|2443236|2443511|N|N|N|N|N| +2443602|AAAAAAAACFJEFCAA|1978-04-03|939|4083|314|1978|1|4|3|2|1978|314|4083|Monday|1978Q2|N|N|N|2443600|2443689|2443237|2443512|N|N|N|N|N| +2443603|AAAAAAAADFJEFCAA|1978-04-04|939|4084|314|1978|2|4|4|2|1978|314|4084|Tuesday|1978Q2|N|N|N|2443600|2443689|2443238|2443513|N|N|N|N|N| +2443604|AAAAAAAAEFJEFCAA|1978-04-05|939|4084|314|1978|3|4|5|2|1978|314|4084|Wednesday|1978Q2|N|N|N|2443600|2443689|2443239|2443514|N|N|N|N|N| +2443605|AAAAAAAAFFJEFCAA|1978-04-06|939|4084|314|1978|4|4|6|2|1978|314|4084|Thursday|1978Q2|N|N|N|2443600|2443689|2443240|2443515|N|N|N|N|N| +2443606|AAAAAAAAGFJEFCAA|1978-04-07|939|4084|314|1978|5|4|7|2|1978|314|4084|Friday|1978Q2|N|Y|N|2443600|2443689|2443241|2443516|N|N|N|N|N| +2443607|AAAAAAAAHFJEFCAA|1978-04-08|939|4084|314|1978|6|4|8|2|1978|314|4084|Saturday|1978Q2|N|Y|N|2443600|2443689|2443242|2443517|N|N|N|N|N| +2443608|AAAAAAAAIFJEFCAA|1978-04-09|939|4084|314|1978|0|4|9|2|1978|314|4084|Sunday|1978Q2|N|N|N|2443600|2443689|2443243|2443518|N|N|N|N|N| +2443609|AAAAAAAAJFJEFCAA|1978-04-10|939|4084|314|1978|1|4|10|2|1978|314|4084|Monday|1978Q2|N|N|N|2443600|2443689|2443244|2443519|N|N|N|N|N| +2443610|AAAAAAAAKFJEFCAA|1978-04-11|939|4085|314|1978|2|4|11|2|1978|314|4085|Tuesday|1978Q2|N|N|N|2443600|2443689|2443245|2443520|N|N|N|N|N| +2443611|AAAAAAAALFJEFCAA|1978-04-12|939|4085|314|1978|3|4|12|2|1978|314|4085|Wednesday|1978Q2|N|N|N|2443600|2443689|2443246|2443521|N|N|N|N|N| +2443612|AAAAAAAAMFJEFCAA|1978-04-13|939|4085|314|1978|4|4|13|2|1978|314|4085|Thursday|1978Q2|N|N|N|2443600|2443689|2443247|2443522|N|N|N|N|N| +2443613|AAAAAAAANFJEFCAA|1978-04-14|939|4085|314|1978|5|4|14|2|1978|314|4085|Friday|1978Q2|N|Y|N|2443600|2443689|2443248|2443523|N|N|N|N|N| +2443614|AAAAAAAAOFJEFCAA|1978-04-15|939|4085|314|1978|6|4|15|2|1978|314|4085|Saturday|1978Q2|N|Y|N|2443600|2443689|2443249|2443524|N|N|N|N|N| +2443615|AAAAAAAAPFJEFCAA|1978-04-16|939|4085|314|1978|0|4|16|2|1978|314|4085|Sunday|1978Q2|N|N|N|2443600|2443689|2443250|2443525|N|N|N|N|N| +2443616|AAAAAAAAAGJEFCAA|1978-04-17|939|4085|314|1978|1|4|17|2|1978|314|4085|Monday|1978Q2|N|N|N|2443600|2443689|2443251|2443526|N|N|N|N|N| +2443617|AAAAAAAABGJEFCAA|1978-04-18|939|4086|314|1978|2|4|18|2|1978|314|4086|Tuesday|1978Q2|N|N|N|2443600|2443689|2443252|2443527|N|N|N|N|N| +2443618|AAAAAAAACGJEFCAA|1978-04-19|939|4086|314|1978|3|4|19|2|1978|314|4086|Wednesday|1978Q2|N|N|N|2443600|2443689|2443253|2443528|N|N|N|N|N| +2443619|AAAAAAAADGJEFCAA|1978-04-20|939|4086|314|1978|4|4|20|2|1978|314|4086|Thursday|1978Q2|N|N|N|2443600|2443689|2443254|2443529|N|N|N|N|N| +2443620|AAAAAAAAEGJEFCAA|1978-04-21|939|4086|314|1978|5|4|21|2|1978|314|4086|Friday|1978Q2|N|Y|N|2443600|2443689|2443255|2443530|N|N|N|N|N| +2443621|AAAAAAAAFGJEFCAA|1978-04-22|939|4086|314|1978|6|4|22|2|1978|314|4086|Saturday|1978Q2|N|Y|N|2443600|2443689|2443256|2443531|N|N|N|N|N| +2443622|AAAAAAAAGGJEFCAA|1978-04-23|939|4086|314|1978|0|4|23|2|1978|314|4086|Sunday|1978Q2|N|N|N|2443600|2443689|2443257|2443532|N|N|N|N|N| +2443623|AAAAAAAAHGJEFCAA|1978-04-24|939|4086|314|1978|1|4|24|2|1978|314|4086|Monday|1978Q2|N|N|N|2443600|2443689|2443258|2443533|N|N|N|N|N| +2443624|AAAAAAAAIGJEFCAA|1978-04-25|939|4087|314|1978|2|4|25|2|1978|314|4087|Tuesday|1978Q2|N|N|N|2443600|2443689|2443259|2443534|N|N|N|N|N| +2443625|AAAAAAAAJGJEFCAA|1978-04-26|939|4087|314|1978|3|4|26|2|1978|314|4087|Wednesday|1978Q2|N|N|N|2443600|2443689|2443260|2443535|N|N|N|N|N| +2443626|AAAAAAAAKGJEFCAA|1978-04-27|939|4087|314|1978|4|4|27|2|1978|314|4087|Thursday|1978Q2|N|N|N|2443600|2443689|2443261|2443536|N|N|N|N|N| +2443627|AAAAAAAALGJEFCAA|1978-04-28|939|4087|314|1978|5|4|28|2|1978|314|4087|Friday|1978Q2|N|Y|N|2443600|2443689|2443262|2443537|N|N|N|N|N| +2443628|AAAAAAAAMGJEFCAA|1978-04-29|939|4087|314|1978|6|4|29|2|1978|314|4087|Saturday|1978Q2|N|Y|N|2443600|2443689|2443263|2443538|N|N|N|N|N| +2443629|AAAAAAAANGJEFCAA|1978-04-30|939|4087|314|1978|0|4|30|2|1978|314|4087|Sunday|1978Q2|N|N|N|2443600|2443689|2443264|2443539|N|N|N|N|N| +2443630|AAAAAAAAOGJEFCAA|1978-05-01|940|4087|314|1978|1|5|1|2|1978|314|4087|Monday|1978Q2|N|N|N|2443630|2443749|2443265|2443540|N|N|N|N|N| +2443631|AAAAAAAAPGJEFCAA|1978-05-02|940|4088|314|1978|2|5|2|2|1978|314|4088|Tuesday|1978Q2|N|N|N|2443630|2443749|2443266|2443541|N|N|N|N|N| +2443632|AAAAAAAAAHJEFCAA|1978-05-03|940|4088|314|1978|3|5|3|2|1978|314|4088|Wednesday|1978Q2|N|N|N|2443630|2443749|2443267|2443542|N|N|N|N|N| +2443633|AAAAAAAABHJEFCAA|1978-05-04|940|4088|314|1978|4|5|4|2|1978|314|4088|Thursday|1978Q2|N|N|N|2443630|2443749|2443268|2443543|N|N|N|N|N| +2443634|AAAAAAAACHJEFCAA|1978-05-05|940|4088|314|1978|5|5|5|2|1978|314|4088|Friday|1978Q2|N|Y|N|2443630|2443749|2443269|2443544|N|N|N|N|N| +2443635|AAAAAAAADHJEFCAA|1978-05-06|940|4088|314|1978|6|5|6|2|1978|314|4088|Saturday|1978Q2|N|Y|N|2443630|2443749|2443270|2443545|N|N|N|N|N| +2443636|AAAAAAAAEHJEFCAA|1978-05-07|940|4088|314|1978|0|5|7|2|1978|314|4088|Sunday|1978Q2|N|N|N|2443630|2443749|2443271|2443546|N|N|N|N|N| +2443637|AAAAAAAAFHJEFCAA|1978-05-08|940|4088|314|1978|1|5|8|2|1978|314|4088|Monday|1978Q2|N|N|N|2443630|2443749|2443272|2443547|N|N|N|N|N| +2443638|AAAAAAAAGHJEFCAA|1978-05-09|940|4089|314|1978|2|5|9|2|1978|314|4089|Tuesday|1978Q2|N|N|N|2443630|2443749|2443273|2443548|N|N|N|N|N| +2443639|AAAAAAAAHHJEFCAA|1978-05-10|940|4089|314|1978|3|5|10|2|1978|314|4089|Wednesday|1978Q2|N|N|N|2443630|2443749|2443274|2443549|N|N|N|N|N| +2443640|AAAAAAAAIHJEFCAA|1978-05-11|940|4089|314|1978|4|5|11|2|1978|314|4089|Thursday|1978Q2|N|N|N|2443630|2443749|2443275|2443550|N|N|N|N|N| +2443641|AAAAAAAAJHJEFCAA|1978-05-12|940|4089|314|1978|5|5|12|2|1978|314|4089|Friday|1978Q2|N|Y|N|2443630|2443749|2443276|2443551|N|N|N|N|N| +2443642|AAAAAAAAKHJEFCAA|1978-05-13|940|4089|314|1978|6|5|13|2|1978|314|4089|Saturday|1978Q2|N|Y|N|2443630|2443749|2443277|2443552|N|N|N|N|N| +2443643|AAAAAAAALHJEFCAA|1978-05-14|940|4089|314|1978|0|5|14|2|1978|314|4089|Sunday|1978Q2|N|N|N|2443630|2443749|2443278|2443553|N|N|N|N|N| +2443644|AAAAAAAAMHJEFCAA|1978-05-15|940|4089|314|1978|1|5|15|2|1978|314|4089|Monday|1978Q2|N|N|N|2443630|2443749|2443279|2443554|N|N|N|N|N| +2443645|AAAAAAAANHJEFCAA|1978-05-16|940|4090|314|1978|2|5|16|2|1978|314|4090|Tuesday|1978Q2|N|N|N|2443630|2443749|2443280|2443555|N|N|N|N|N| +2443646|AAAAAAAAOHJEFCAA|1978-05-17|940|4090|314|1978|3|5|17|2|1978|314|4090|Wednesday|1978Q2|N|N|N|2443630|2443749|2443281|2443556|N|N|N|N|N| +2443647|AAAAAAAAPHJEFCAA|1978-05-18|940|4090|314|1978|4|5|18|2|1978|314|4090|Thursday|1978Q2|N|N|N|2443630|2443749|2443282|2443557|N|N|N|N|N| +2443648|AAAAAAAAAIJEFCAA|1978-05-19|940|4090|314|1978|5|5|19|2|1978|314|4090|Friday|1978Q2|N|Y|N|2443630|2443749|2443283|2443558|N|N|N|N|N| +2443649|AAAAAAAABIJEFCAA|1978-05-20|940|4090|314|1978|6|5|20|2|1978|314|4090|Saturday|1978Q2|N|Y|N|2443630|2443749|2443284|2443559|N|N|N|N|N| +2443650|AAAAAAAACIJEFCAA|1978-05-21|940|4090|314|1978|0|5|21|2|1978|314|4090|Sunday|1978Q2|N|N|N|2443630|2443749|2443285|2443560|N|N|N|N|N| +2443651|AAAAAAAADIJEFCAA|1978-05-22|940|4090|314|1978|1|5|22|2|1978|314|4090|Monday|1978Q2|N|N|N|2443630|2443749|2443286|2443561|N|N|N|N|N| +2443652|AAAAAAAAEIJEFCAA|1978-05-23|940|4091|314|1978|2|5|23|2|1978|314|4091|Tuesday|1978Q2|N|N|N|2443630|2443749|2443287|2443562|N|N|N|N|N| +2443653|AAAAAAAAFIJEFCAA|1978-05-24|940|4091|314|1978|3|5|24|2|1978|314|4091|Wednesday|1978Q2|N|N|N|2443630|2443749|2443288|2443563|N|N|N|N|N| +2443654|AAAAAAAAGIJEFCAA|1978-05-25|940|4091|314|1978|4|5|25|2|1978|314|4091|Thursday|1978Q2|N|N|N|2443630|2443749|2443289|2443564|N|N|N|N|N| +2443655|AAAAAAAAHIJEFCAA|1978-05-26|940|4091|314|1978|5|5|26|2|1978|314|4091|Friday|1978Q2|N|Y|N|2443630|2443749|2443290|2443565|N|N|N|N|N| +2443656|AAAAAAAAIIJEFCAA|1978-05-27|940|4091|314|1978|6|5|27|2|1978|314|4091|Saturday|1978Q2|N|Y|N|2443630|2443749|2443291|2443566|N|N|N|N|N| +2443657|AAAAAAAAJIJEFCAA|1978-05-28|940|4091|314|1978|0|5|28|2|1978|314|4091|Sunday|1978Q2|N|N|N|2443630|2443749|2443292|2443567|N|N|N|N|N| +2443658|AAAAAAAAKIJEFCAA|1978-05-29|940|4091|314|1978|1|5|29|2|1978|314|4091|Monday|1978Q2|N|N|N|2443630|2443749|2443293|2443568|N|N|N|N|N| +2443659|AAAAAAAALIJEFCAA|1978-05-30|940|4092|314|1978|2|5|30|2|1978|314|4092|Tuesday|1978Q2|N|N|N|2443630|2443749|2443294|2443569|N|N|N|N|N| +2443660|AAAAAAAAMIJEFCAA|1978-05-31|940|4092|314|1978|3|5|31|2|1978|314|4092|Wednesday|1978Q2|N|N|N|2443630|2443749|2443295|2443570|N|N|N|N|N| +2443661|AAAAAAAANIJEFCAA|1978-06-01|941|4092|315|1978|4|6|1|2|1978|315|4092|Thursday|1978Q2|N|N|N|2443661|2443811|2443296|2443571|N|N|N|N|N| +2443662|AAAAAAAAOIJEFCAA|1978-06-02|941|4092|315|1978|5|6|2|2|1978|315|4092|Friday|1978Q2|N|Y|N|2443661|2443811|2443297|2443572|N|N|N|N|N| +2443663|AAAAAAAAPIJEFCAA|1978-06-03|941|4092|315|1978|6|6|3|2|1978|315|4092|Saturday|1978Q2|N|Y|N|2443661|2443811|2443298|2443573|N|N|N|N|N| +2443664|AAAAAAAAAJJEFCAA|1978-06-04|941|4092|315|1978|0|6|4|2|1978|315|4092|Sunday|1978Q2|N|N|N|2443661|2443811|2443299|2443574|N|N|N|N|N| +2443665|AAAAAAAABJJEFCAA|1978-06-05|941|4092|315|1978|1|6|5|2|1978|315|4092|Monday|1978Q2|N|N|N|2443661|2443811|2443300|2443575|N|N|N|N|N| +2443666|AAAAAAAACJJEFCAA|1978-06-06|941|4093|315|1978|2|6|6|2|1978|315|4093|Tuesday|1978Q2|N|N|N|2443661|2443811|2443301|2443576|N|N|N|N|N| +2443667|AAAAAAAADJJEFCAA|1978-06-07|941|4093|315|1978|3|6|7|2|1978|315|4093|Wednesday|1978Q2|N|N|N|2443661|2443811|2443302|2443577|N|N|N|N|N| +2443668|AAAAAAAAEJJEFCAA|1978-06-08|941|4093|315|1978|4|6|8|2|1978|315|4093|Thursday|1978Q2|N|N|N|2443661|2443811|2443303|2443578|N|N|N|N|N| +2443669|AAAAAAAAFJJEFCAA|1978-06-09|941|4093|315|1978|5|6|9|2|1978|315|4093|Friday|1978Q2|N|Y|N|2443661|2443811|2443304|2443579|N|N|N|N|N| +2443670|AAAAAAAAGJJEFCAA|1978-06-10|941|4093|315|1978|6|6|10|2|1978|315|4093|Saturday|1978Q2|N|Y|N|2443661|2443811|2443305|2443580|N|N|N|N|N| +2443671|AAAAAAAAHJJEFCAA|1978-06-11|941|4093|315|1978|0|6|11|2|1978|315|4093|Sunday|1978Q2|N|N|N|2443661|2443811|2443306|2443581|N|N|N|N|N| +2443672|AAAAAAAAIJJEFCAA|1978-06-12|941|4093|315|1978|1|6|12|2|1978|315|4093|Monday|1978Q2|N|N|N|2443661|2443811|2443307|2443582|N|N|N|N|N| +2443673|AAAAAAAAJJJEFCAA|1978-06-13|941|4094|315|1978|2|6|13|2|1978|315|4094|Tuesday|1978Q2|N|N|N|2443661|2443811|2443308|2443583|N|N|N|N|N| +2443674|AAAAAAAAKJJEFCAA|1978-06-14|941|4094|315|1978|3|6|14|2|1978|315|4094|Wednesday|1978Q2|N|N|N|2443661|2443811|2443309|2443584|N|N|N|N|N| +2443675|AAAAAAAALJJEFCAA|1978-06-15|941|4094|315|1978|4|6|15|2|1978|315|4094|Thursday|1978Q2|N|N|N|2443661|2443811|2443310|2443585|N|N|N|N|N| +2443676|AAAAAAAAMJJEFCAA|1978-06-16|941|4094|315|1978|5|6|16|2|1978|315|4094|Friday|1978Q2|N|Y|N|2443661|2443811|2443311|2443586|N|N|N|N|N| +2443677|AAAAAAAANJJEFCAA|1978-06-17|941|4094|315|1978|6|6|17|2|1978|315|4094|Saturday|1978Q2|N|Y|N|2443661|2443811|2443312|2443587|N|N|N|N|N| +2443678|AAAAAAAAOJJEFCAA|1978-06-18|941|4094|315|1978|0|6|18|2|1978|315|4094|Sunday|1978Q2|N|N|N|2443661|2443811|2443313|2443588|N|N|N|N|N| +2443679|AAAAAAAAPJJEFCAA|1978-06-19|941|4094|315|1978|1|6|19|2|1978|315|4094|Monday|1978Q2|N|N|N|2443661|2443811|2443314|2443589|N|N|N|N|N| +2443680|AAAAAAAAAKJEFCAA|1978-06-20|941|4095|315|1978|2|6|20|2|1978|315|4095|Tuesday|1978Q2|N|N|N|2443661|2443811|2443315|2443590|N|N|N|N|N| +2443681|AAAAAAAABKJEFCAA|1978-06-21|941|4095|315|1978|3|6|21|2|1978|315|4095|Wednesday|1978Q2|N|N|N|2443661|2443811|2443316|2443591|N|N|N|N|N| +2443682|AAAAAAAACKJEFCAA|1978-06-22|941|4095|315|1978|4|6|22|2|1978|315|4095|Thursday|1978Q2|N|N|N|2443661|2443811|2443317|2443592|N|N|N|N|N| +2443683|AAAAAAAADKJEFCAA|1978-06-23|941|4095|315|1978|5|6|23|2|1978|315|4095|Friday|1978Q2|N|Y|N|2443661|2443811|2443318|2443593|N|N|N|N|N| +2443684|AAAAAAAAEKJEFCAA|1978-06-24|941|4095|315|1978|6|6|24|2|1978|315|4095|Saturday|1978Q2|N|Y|N|2443661|2443811|2443319|2443594|N|N|N|N|N| +2443685|AAAAAAAAFKJEFCAA|1978-06-25|941|4095|315|1978|0|6|25|2|1978|315|4095|Sunday|1978Q2|N|N|N|2443661|2443811|2443320|2443595|N|N|N|N|N| +2443686|AAAAAAAAGKJEFCAA|1978-06-26|941|4095|315|1978|1|6|26|2|1978|315|4095|Monday|1978Q2|N|N|N|2443661|2443811|2443321|2443596|N|N|N|N|N| +2443687|AAAAAAAAHKJEFCAA|1978-06-27|941|4096|315|1978|2|6|27|2|1978|315|4096|Tuesday|1978Q2|N|N|N|2443661|2443811|2443322|2443597|N|N|N|N|N| +2443688|AAAAAAAAIKJEFCAA|1978-06-28|941|4096|315|1978|3|6|28|2|1978|315|4096|Wednesday|1978Q2|N|N|N|2443661|2443811|2443323|2443598|N|N|N|N|N| +2443689|AAAAAAAAJKJEFCAA|1978-06-29|941|4096|315|1978|4|6|29|2|1978|315|4096|Thursday|1978Q2|N|N|N|2443661|2443811|2443324|2443599|N|N|N|N|N| +2443690|AAAAAAAAKKJEFCAA|1978-06-30|941|4096|315|1978|5|6|30|2|1978|315|4096|Friday|1978Q2|N|Y|N|2443661|2443811|2443325|2443600|N|N|N|N|N| +2443691|AAAAAAAALKJEFCAA|1978-07-01|942|4096|315|1978|6|7|1|2|1978|315|4096|Saturday|1978Q2|N|Y|N|2443691|2443871|2443326|2443600|N|N|N|N|N| +2443692|AAAAAAAAMKJEFCAA|1978-07-02|942|4096|315|1978|0|7|2|3|1978|315|4096|Sunday|1978Q3|N|N|N|2443691|2443871|2443327|2443601|N|N|N|N|N| +2443693|AAAAAAAANKJEFCAA|1978-07-03|942|4096|315|1978|1|7|3|3|1978|315|4096|Monday|1978Q3|N|N|N|2443691|2443871|2443328|2443602|N|N|N|N|N| +2443694|AAAAAAAAOKJEFCAA|1978-07-04|942|4097|315|1978|2|7|4|3|1978|315|4097|Tuesday|1978Q3|N|N|N|2443691|2443871|2443329|2443603|N|N|N|N|N| +2443695|AAAAAAAAPKJEFCAA|1978-07-05|942|4097|315|1978|3|7|5|3|1978|315|4097|Wednesday|1978Q3|Y|N|N|2443691|2443871|2443330|2443604|N|N|N|N|N| +2443696|AAAAAAAAALJEFCAA|1978-07-06|942|4097|315|1978|4|7|6|3|1978|315|4097|Thursday|1978Q3|N|N|Y|2443691|2443871|2443331|2443605|N|N|N|N|N| +2443697|AAAAAAAABLJEFCAA|1978-07-07|942|4097|315|1978|5|7|7|3|1978|315|4097|Friday|1978Q3|N|Y|N|2443691|2443871|2443332|2443606|N|N|N|N|N| +2443698|AAAAAAAACLJEFCAA|1978-07-08|942|4097|315|1978|6|7|8|3|1978|315|4097|Saturday|1978Q3|N|Y|N|2443691|2443871|2443333|2443607|N|N|N|N|N| +2443699|AAAAAAAADLJEFCAA|1978-07-09|942|4097|315|1978|0|7|9|3|1978|315|4097|Sunday|1978Q3|N|N|N|2443691|2443871|2443334|2443608|N|N|N|N|N| +2443700|AAAAAAAAELJEFCAA|1978-07-10|942|4097|315|1978|1|7|10|3|1978|315|4097|Monday|1978Q3|N|N|N|2443691|2443871|2443335|2443609|N|N|N|N|N| +2443701|AAAAAAAAFLJEFCAA|1978-07-11|942|4098|315|1978|2|7|11|3|1978|315|4098|Tuesday|1978Q3|N|N|N|2443691|2443871|2443336|2443610|N|N|N|N|N| +2443702|AAAAAAAAGLJEFCAA|1978-07-12|942|4098|315|1978|3|7|12|3|1978|315|4098|Wednesday|1978Q3|N|N|N|2443691|2443871|2443337|2443611|N|N|N|N|N| +2443703|AAAAAAAAHLJEFCAA|1978-07-13|942|4098|315|1978|4|7|13|3|1978|315|4098|Thursday|1978Q3|N|N|N|2443691|2443871|2443338|2443612|N|N|N|N|N| +2443704|AAAAAAAAILJEFCAA|1978-07-14|942|4098|315|1978|5|7|14|3|1978|315|4098|Friday|1978Q3|N|Y|N|2443691|2443871|2443339|2443613|N|N|N|N|N| +2443705|AAAAAAAAJLJEFCAA|1978-07-15|942|4098|315|1978|6|7|15|3|1978|315|4098|Saturday|1978Q3|N|Y|N|2443691|2443871|2443340|2443614|N|N|N|N|N| +2443706|AAAAAAAAKLJEFCAA|1978-07-16|942|4098|315|1978|0|7|16|3|1978|315|4098|Sunday|1978Q3|N|N|N|2443691|2443871|2443341|2443615|N|N|N|N|N| +2443707|AAAAAAAALLJEFCAA|1978-07-17|942|4098|315|1978|1|7|17|3|1978|315|4098|Monday|1978Q3|N|N|N|2443691|2443871|2443342|2443616|N|N|N|N|N| +2443708|AAAAAAAAMLJEFCAA|1978-07-18|942|4099|315|1978|2|7|18|3|1978|315|4099|Tuesday|1978Q3|N|N|N|2443691|2443871|2443343|2443617|N|N|N|N|N| +2443709|AAAAAAAANLJEFCAA|1978-07-19|942|4099|315|1978|3|7|19|3|1978|315|4099|Wednesday|1978Q3|N|N|N|2443691|2443871|2443344|2443618|N|N|N|N|N| +2443710|AAAAAAAAOLJEFCAA|1978-07-20|942|4099|315|1978|4|7|20|3|1978|315|4099|Thursday|1978Q3|N|N|N|2443691|2443871|2443345|2443619|N|N|N|N|N| +2443711|AAAAAAAAPLJEFCAA|1978-07-21|942|4099|315|1978|5|7|21|3|1978|315|4099|Friday|1978Q3|N|Y|N|2443691|2443871|2443346|2443620|N|N|N|N|N| +2443712|AAAAAAAAAMJEFCAA|1978-07-22|942|4099|315|1978|6|7|22|3|1978|315|4099|Saturday|1978Q3|N|Y|N|2443691|2443871|2443347|2443621|N|N|N|N|N| +2443713|AAAAAAAABMJEFCAA|1978-07-23|942|4099|315|1978|0|7|23|3|1978|315|4099|Sunday|1978Q3|N|N|N|2443691|2443871|2443348|2443622|N|N|N|N|N| +2443714|AAAAAAAACMJEFCAA|1978-07-24|942|4099|315|1978|1|7|24|3|1978|315|4099|Monday|1978Q3|N|N|N|2443691|2443871|2443349|2443623|N|N|N|N|N| +2443715|AAAAAAAADMJEFCAA|1978-07-25|942|4100|315|1978|2|7|25|3|1978|315|4100|Tuesday|1978Q3|N|N|N|2443691|2443871|2443350|2443624|N|N|N|N|N| +2443716|AAAAAAAAEMJEFCAA|1978-07-26|942|4100|315|1978|3|7|26|3|1978|315|4100|Wednesday|1978Q3|N|N|N|2443691|2443871|2443351|2443625|N|N|N|N|N| +2443717|AAAAAAAAFMJEFCAA|1978-07-27|942|4100|315|1978|4|7|27|3|1978|315|4100|Thursday|1978Q3|N|N|N|2443691|2443871|2443352|2443626|N|N|N|N|N| +2443718|AAAAAAAAGMJEFCAA|1978-07-28|942|4100|315|1978|5|7|28|3|1978|315|4100|Friday|1978Q3|N|Y|N|2443691|2443871|2443353|2443627|N|N|N|N|N| +2443719|AAAAAAAAHMJEFCAA|1978-07-29|942|4100|315|1978|6|7|29|3|1978|315|4100|Saturday|1978Q3|N|Y|N|2443691|2443871|2443354|2443628|N|N|N|N|N| +2443720|AAAAAAAAIMJEFCAA|1978-07-30|942|4100|315|1978|0|7|30|3|1978|315|4100|Sunday|1978Q3|N|N|N|2443691|2443871|2443355|2443629|N|N|N|N|N| +2443721|AAAAAAAAJMJEFCAA|1978-07-31|942|4100|315|1978|1|7|31|3|1978|315|4100|Monday|1978Q3|N|N|N|2443691|2443871|2443356|2443630|N|N|N|N|N| +2443722|AAAAAAAAKMJEFCAA|1978-08-01|943|4101|315|1978|2|8|1|3|1978|315|4101|Tuesday|1978Q3|N|N|N|2443722|2443933|2443357|2443631|N|N|N|N|N| +2443723|AAAAAAAALMJEFCAA|1978-08-02|943|4101|315|1978|3|8|2|3|1978|315|4101|Wednesday|1978Q3|N|N|N|2443722|2443933|2443358|2443632|N|N|N|N|N| +2443724|AAAAAAAAMMJEFCAA|1978-08-03|943|4101|315|1978|4|8|3|3|1978|315|4101|Thursday|1978Q3|N|N|N|2443722|2443933|2443359|2443633|N|N|N|N|N| +2443725|AAAAAAAANMJEFCAA|1978-08-04|943|4101|315|1978|5|8|4|3|1978|315|4101|Friday|1978Q3|N|Y|N|2443722|2443933|2443360|2443634|N|N|N|N|N| +2443726|AAAAAAAAOMJEFCAA|1978-08-05|943|4101|315|1978|6|8|5|3|1978|315|4101|Saturday|1978Q3|N|Y|N|2443722|2443933|2443361|2443635|N|N|N|N|N| +2443727|AAAAAAAAPMJEFCAA|1978-08-06|943|4101|315|1978|0|8|6|3|1978|315|4101|Sunday|1978Q3|N|N|N|2443722|2443933|2443362|2443636|N|N|N|N|N| +2443728|AAAAAAAAANJEFCAA|1978-08-07|943|4101|315|1978|1|8|7|3|1978|315|4101|Monday|1978Q3|N|N|N|2443722|2443933|2443363|2443637|N|N|N|N|N| +2443729|AAAAAAAABNJEFCAA|1978-08-08|943|4102|315|1978|2|8|8|3|1978|315|4102|Tuesday|1978Q3|N|N|N|2443722|2443933|2443364|2443638|N|N|N|N|N| +2443730|AAAAAAAACNJEFCAA|1978-08-09|943|4102|315|1978|3|8|9|3|1978|315|4102|Wednesday|1978Q3|N|N|N|2443722|2443933|2443365|2443639|N|N|N|N|N| +2443731|AAAAAAAADNJEFCAA|1978-08-10|943|4102|315|1978|4|8|10|3|1978|315|4102|Thursday|1978Q3|N|N|N|2443722|2443933|2443366|2443640|N|N|N|N|N| +2443732|AAAAAAAAENJEFCAA|1978-08-11|943|4102|315|1978|5|8|11|3|1978|315|4102|Friday|1978Q3|N|Y|N|2443722|2443933|2443367|2443641|N|N|N|N|N| +2443733|AAAAAAAAFNJEFCAA|1978-08-12|943|4102|315|1978|6|8|12|3|1978|315|4102|Saturday|1978Q3|N|Y|N|2443722|2443933|2443368|2443642|N|N|N|N|N| +2443734|AAAAAAAAGNJEFCAA|1978-08-13|943|4102|315|1978|0|8|13|3|1978|315|4102|Sunday|1978Q3|N|N|N|2443722|2443933|2443369|2443643|N|N|N|N|N| +2443735|AAAAAAAAHNJEFCAA|1978-08-14|943|4102|315|1978|1|8|14|3|1978|315|4102|Monday|1978Q3|N|N|N|2443722|2443933|2443370|2443644|N|N|N|N|N| +2443736|AAAAAAAAINJEFCAA|1978-08-15|943|4103|315|1978|2|8|15|3|1978|315|4103|Tuesday|1978Q3|N|N|N|2443722|2443933|2443371|2443645|N|N|N|N|N| +2443737|AAAAAAAAJNJEFCAA|1978-08-16|943|4103|315|1978|3|8|16|3|1978|315|4103|Wednesday|1978Q3|N|N|N|2443722|2443933|2443372|2443646|N|N|N|N|N| +2443738|AAAAAAAAKNJEFCAA|1978-08-17|943|4103|315|1978|4|8|17|3|1978|315|4103|Thursday|1978Q3|N|N|N|2443722|2443933|2443373|2443647|N|N|N|N|N| +2443739|AAAAAAAALNJEFCAA|1978-08-18|943|4103|315|1978|5|8|18|3|1978|315|4103|Friday|1978Q3|N|Y|N|2443722|2443933|2443374|2443648|N|N|N|N|N| +2443740|AAAAAAAAMNJEFCAA|1978-08-19|943|4103|315|1978|6|8|19|3|1978|315|4103|Saturday|1978Q3|N|Y|N|2443722|2443933|2443375|2443649|N|N|N|N|N| +2443741|AAAAAAAANNJEFCAA|1978-08-20|943|4103|315|1978|0|8|20|3|1978|315|4103|Sunday|1978Q3|N|N|N|2443722|2443933|2443376|2443650|N|N|N|N|N| +2443742|AAAAAAAAONJEFCAA|1978-08-21|943|4103|315|1978|1|8|21|3|1978|315|4103|Monday|1978Q3|N|N|N|2443722|2443933|2443377|2443651|N|N|N|N|N| +2443743|AAAAAAAAPNJEFCAA|1978-08-22|943|4104|315|1978|2|8|22|3|1978|315|4104|Tuesday|1978Q3|N|N|N|2443722|2443933|2443378|2443652|N|N|N|N|N| +2443744|AAAAAAAAAOJEFCAA|1978-08-23|943|4104|315|1978|3|8|23|3|1978|315|4104|Wednesday|1978Q3|N|N|N|2443722|2443933|2443379|2443653|N|N|N|N|N| +2443745|AAAAAAAABOJEFCAA|1978-08-24|943|4104|315|1978|4|8|24|3|1978|315|4104|Thursday|1978Q3|N|N|N|2443722|2443933|2443380|2443654|N|N|N|N|N| +2443746|AAAAAAAACOJEFCAA|1978-08-25|943|4104|315|1978|5|8|25|3|1978|315|4104|Friday|1978Q3|N|Y|N|2443722|2443933|2443381|2443655|N|N|N|N|N| +2443747|AAAAAAAADOJEFCAA|1978-08-26|943|4104|315|1978|6|8|26|3|1978|315|4104|Saturday|1978Q3|N|Y|N|2443722|2443933|2443382|2443656|N|N|N|N|N| +2443748|AAAAAAAAEOJEFCAA|1978-08-27|943|4104|315|1978|0|8|27|3|1978|315|4104|Sunday|1978Q3|N|N|N|2443722|2443933|2443383|2443657|N|N|N|N|N| +2443749|AAAAAAAAFOJEFCAA|1978-08-28|943|4104|315|1978|1|8|28|3|1978|315|4104|Monday|1978Q3|N|N|N|2443722|2443933|2443384|2443658|N|N|N|N|N| +2443750|AAAAAAAAGOJEFCAA|1978-08-29|943|4105|315|1978|2|8|29|3|1978|315|4105|Tuesday|1978Q3|N|N|N|2443722|2443933|2443385|2443659|N|N|N|N|N| +2443751|AAAAAAAAHOJEFCAA|1978-08-30|943|4105|315|1978|3|8|30|3|1978|315|4105|Wednesday|1978Q3|N|N|N|2443722|2443933|2443386|2443660|N|N|N|N|N| +2443752|AAAAAAAAIOJEFCAA|1978-08-31|943|4105|315|1978|4|8|31|3|1978|315|4105|Thursday|1978Q3|N|N|N|2443722|2443933|2443387|2443661|N|N|N|N|N| +2443753|AAAAAAAAJOJEFCAA|1978-09-01|944|4105|316|1978|5|9|1|3|1978|316|4105|Friday|1978Q3|N|Y|N|2443753|2443995|2443388|2443662|N|N|N|N|N| +2443754|AAAAAAAAKOJEFCAA|1978-09-02|944|4105|316|1978|6|9|2|3|1978|316|4105|Saturday|1978Q3|N|Y|N|2443753|2443995|2443389|2443663|N|N|N|N|N| +2443755|AAAAAAAALOJEFCAA|1978-09-03|944|4105|316|1978|0|9|3|3|1978|316|4105|Sunday|1978Q3|N|N|N|2443753|2443995|2443390|2443664|N|N|N|N|N| +2443756|AAAAAAAAMOJEFCAA|1978-09-04|944|4105|316|1978|1|9|4|3|1978|316|4105|Monday|1978Q3|N|N|N|2443753|2443995|2443391|2443665|N|N|N|N|N| +2443757|AAAAAAAANOJEFCAA|1978-09-05|944|4106|316|1978|2|9|5|3|1978|316|4106|Tuesday|1978Q3|N|N|N|2443753|2443995|2443392|2443666|N|N|N|N|N| +2443758|AAAAAAAAOOJEFCAA|1978-09-06|944|4106|316|1978|3|9|6|3|1978|316|4106|Wednesday|1978Q3|N|N|N|2443753|2443995|2443393|2443667|N|N|N|N|N| +2443759|AAAAAAAAPOJEFCAA|1978-09-07|944|4106|316|1978|4|9|7|3|1978|316|4106|Thursday|1978Q3|N|N|N|2443753|2443995|2443394|2443668|N|N|N|N|N| +2443760|AAAAAAAAAPJEFCAA|1978-09-08|944|4106|316|1978|5|9|8|3|1978|316|4106|Friday|1978Q3|N|Y|N|2443753|2443995|2443395|2443669|N|N|N|N|N| +2443761|AAAAAAAABPJEFCAA|1978-09-09|944|4106|316|1978|6|9|9|3|1978|316|4106|Saturday|1978Q3|N|Y|N|2443753|2443995|2443396|2443670|N|N|N|N|N| +2443762|AAAAAAAACPJEFCAA|1978-09-10|944|4106|316|1978|0|9|10|3|1978|316|4106|Sunday|1978Q3|N|N|N|2443753|2443995|2443397|2443671|N|N|N|N|N| +2443763|AAAAAAAADPJEFCAA|1978-09-11|944|4106|316|1978|1|9|11|3|1978|316|4106|Monday|1978Q3|N|N|N|2443753|2443995|2443398|2443672|N|N|N|N|N| +2443764|AAAAAAAAEPJEFCAA|1978-09-12|944|4107|316|1978|2|9|12|3|1978|316|4107|Tuesday|1978Q3|N|N|N|2443753|2443995|2443399|2443673|N|N|N|N|N| +2443765|AAAAAAAAFPJEFCAA|1978-09-13|944|4107|316|1978|3|9|13|3|1978|316|4107|Wednesday|1978Q3|N|N|N|2443753|2443995|2443400|2443674|N|N|N|N|N| +2443766|AAAAAAAAGPJEFCAA|1978-09-14|944|4107|316|1978|4|9|14|3|1978|316|4107|Thursday|1978Q3|N|N|N|2443753|2443995|2443401|2443675|N|N|N|N|N| +2443767|AAAAAAAAHPJEFCAA|1978-09-15|944|4107|316|1978|5|9|15|3|1978|316|4107|Friday|1978Q3|N|Y|N|2443753|2443995|2443402|2443676|N|N|N|N|N| +2443768|AAAAAAAAIPJEFCAA|1978-09-16|944|4107|316|1978|6|9|16|3|1978|316|4107|Saturday|1978Q3|N|Y|N|2443753|2443995|2443403|2443677|N|N|N|N|N| +2443769|AAAAAAAAJPJEFCAA|1978-09-17|944|4107|316|1978|0|9|17|3|1978|316|4107|Sunday|1978Q3|N|N|N|2443753|2443995|2443404|2443678|N|N|N|N|N| +2443770|AAAAAAAAKPJEFCAA|1978-09-18|944|4107|316|1978|1|9|18|3|1978|316|4107|Monday|1978Q3|N|N|N|2443753|2443995|2443405|2443679|N|N|N|N|N| +2443771|AAAAAAAALPJEFCAA|1978-09-19|944|4108|316|1978|2|9|19|3|1978|316|4108|Tuesday|1978Q3|N|N|N|2443753|2443995|2443406|2443680|N|N|N|N|N| +2443772|AAAAAAAAMPJEFCAA|1978-09-20|944|4108|316|1978|3|9|20|3|1978|316|4108|Wednesday|1978Q3|N|N|N|2443753|2443995|2443407|2443681|N|N|N|N|N| +2443773|AAAAAAAANPJEFCAA|1978-09-21|944|4108|316|1978|4|9|21|3|1978|316|4108|Thursday|1978Q3|N|N|N|2443753|2443995|2443408|2443682|N|N|N|N|N| +2443774|AAAAAAAAOPJEFCAA|1978-09-22|944|4108|316|1978|5|9|22|3|1978|316|4108|Friday|1978Q3|N|Y|N|2443753|2443995|2443409|2443683|N|N|N|N|N| +2443775|AAAAAAAAPPJEFCAA|1978-09-23|944|4108|316|1978|6|9|23|3|1978|316|4108|Saturday|1978Q3|N|Y|N|2443753|2443995|2443410|2443684|N|N|N|N|N| +2443776|AAAAAAAAAAKEFCAA|1978-09-24|944|4108|316|1978|0|9|24|3|1978|316|4108|Sunday|1978Q3|N|N|N|2443753|2443995|2443411|2443685|N|N|N|N|N| +2443777|AAAAAAAABAKEFCAA|1978-09-25|944|4108|316|1978|1|9|25|3|1978|316|4108|Monday|1978Q3|N|N|N|2443753|2443995|2443412|2443686|N|N|N|N|N| +2443778|AAAAAAAACAKEFCAA|1978-09-26|944|4109|316|1978|2|9|26|3|1978|316|4109|Tuesday|1978Q3|N|N|N|2443753|2443995|2443413|2443687|N|N|N|N|N| +2443779|AAAAAAAADAKEFCAA|1978-09-27|944|4109|316|1978|3|9|27|3|1978|316|4109|Wednesday|1978Q3|N|N|N|2443753|2443995|2443414|2443688|N|N|N|N|N| +2443780|AAAAAAAAEAKEFCAA|1978-09-28|944|4109|316|1978|4|9|28|3|1978|316|4109|Thursday|1978Q3|N|N|N|2443753|2443995|2443415|2443689|N|N|N|N|N| +2443781|AAAAAAAAFAKEFCAA|1978-09-29|944|4109|316|1978|5|9|29|3|1978|316|4109|Friday|1978Q3|N|Y|N|2443753|2443995|2443416|2443690|N|N|N|N|N| +2443782|AAAAAAAAGAKEFCAA|1978-09-30|944|4109|316|1978|6|9|30|3|1978|316|4109|Saturday|1978Q3|N|Y|N|2443753|2443995|2443417|2443691|N|N|N|N|N| +2443783|AAAAAAAAHAKEFCAA|1978-10-01|945|4109|316|1978|0|10|1|3|1978|316|4109|Sunday|1978Q3|N|N|N|2443783|2444055|2443418|2443691|N|N|N|N|N| +2443784|AAAAAAAAIAKEFCAA|1978-10-02|945|4109|316|1978|1|10|2|4|1978|316|4109|Monday|1978Q4|N|N|N|2443783|2444055|2443419|2443692|N|N|N|N|N| +2443785|AAAAAAAAJAKEFCAA|1978-10-03|945|4110|316|1978|2|10|3|4|1978|316|4110|Tuesday|1978Q4|N|N|N|2443783|2444055|2443420|2443693|N|N|N|N|N| +2443786|AAAAAAAAKAKEFCAA|1978-10-04|945|4110|316|1978|3|10|4|4|1978|316|4110|Wednesday|1978Q4|N|N|N|2443783|2444055|2443421|2443694|N|N|N|N|N| +2443787|AAAAAAAALAKEFCAA|1978-10-05|945|4110|316|1978|4|10|5|4|1978|316|4110|Thursday|1978Q4|N|N|N|2443783|2444055|2443422|2443695|N|N|N|N|N| +2443788|AAAAAAAAMAKEFCAA|1978-10-06|945|4110|316|1978|5|10|6|4|1978|316|4110|Friday|1978Q4|N|Y|N|2443783|2444055|2443423|2443696|N|N|N|N|N| +2443789|AAAAAAAANAKEFCAA|1978-10-07|945|4110|316|1978|6|10|7|4|1978|316|4110|Saturday|1978Q4|N|Y|N|2443783|2444055|2443424|2443697|N|N|N|N|N| +2443790|AAAAAAAAOAKEFCAA|1978-10-08|945|4110|316|1978|0|10|8|4|1978|316|4110|Sunday|1978Q4|N|N|N|2443783|2444055|2443425|2443698|N|N|N|N|N| +2443791|AAAAAAAAPAKEFCAA|1978-10-09|945|4110|316|1978|1|10|9|4|1978|316|4110|Monday|1978Q4|N|N|N|2443783|2444055|2443426|2443699|N|N|N|N|N| +2443792|AAAAAAAAABKEFCAA|1978-10-10|945|4111|316|1978|2|10|10|4|1978|316|4111|Tuesday|1978Q4|N|N|N|2443783|2444055|2443427|2443700|N|N|N|N|N| +2443793|AAAAAAAABBKEFCAA|1978-10-11|945|4111|316|1978|3|10|11|4|1978|316|4111|Wednesday|1978Q4|N|N|N|2443783|2444055|2443428|2443701|N|N|N|N|N| +2443794|AAAAAAAACBKEFCAA|1978-10-12|945|4111|316|1978|4|10|12|4|1978|316|4111|Thursday|1978Q4|N|N|N|2443783|2444055|2443429|2443702|N|N|N|N|N| +2443795|AAAAAAAADBKEFCAA|1978-10-13|945|4111|316|1978|5|10|13|4|1978|316|4111|Friday|1978Q4|N|Y|N|2443783|2444055|2443430|2443703|N|N|N|N|N| +2443796|AAAAAAAAEBKEFCAA|1978-10-14|945|4111|316|1978|6|10|14|4|1978|316|4111|Saturday|1978Q4|N|Y|N|2443783|2444055|2443431|2443704|N|N|N|N|N| +2443797|AAAAAAAAFBKEFCAA|1978-10-15|945|4111|316|1978|0|10|15|4|1978|316|4111|Sunday|1978Q4|N|N|N|2443783|2444055|2443432|2443705|N|N|N|N|N| +2443798|AAAAAAAAGBKEFCAA|1978-10-16|945|4111|316|1978|1|10|16|4|1978|316|4111|Monday|1978Q4|N|N|N|2443783|2444055|2443433|2443706|N|N|N|N|N| +2443799|AAAAAAAAHBKEFCAA|1978-10-17|945|4112|316|1978|2|10|17|4|1978|316|4112|Tuesday|1978Q4|N|N|N|2443783|2444055|2443434|2443707|N|N|N|N|N| +2443800|AAAAAAAAIBKEFCAA|1978-10-18|945|4112|316|1978|3|10|18|4|1978|316|4112|Wednesday|1978Q4|N|N|N|2443783|2444055|2443435|2443708|N|N|N|N|N| +2443801|AAAAAAAAJBKEFCAA|1978-10-19|945|4112|316|1978|4|10|19|4|1978|316|4112|Thursday|1978Q4|N|N|N|2443783|2444055|2443436|2443709|N|N|N|N|N| +2443802|AAAAAAAAKBKEFCAA|1978-10-20|945|4112|316|1978|5|10|20|4|1978|316|4112|Friday|1978Q4|N|Y|N|2443783|2444055|2443437|2443710|N|N|N|N|N| +2443803|AAAAAAAALBKEFCAA|1978-10-21|945|4112|316|1978|6|10|21|4|1978|316|4112|Saturday|1978Q4|N|Y|N|2443783|2444055|2443438|2443711|N|N|N|N|N| +2443804|AAAAAAAAMBKEFCAA|1978-10-22|945|4112|316|1978|0|10|22|4|1978|316|4112|Sunday|1978Q4|N|N|N|2443783|2444055|2443439|2443712|N|N|N|N|N| +2443805|AAAAAAAANBKEFCAA|1978-10-23|945|4112|316|1978|1|10|23|4|1978|316|4112|Monday|1978Q4|N|N|N|2443783|2444055|2443440|2443713|N|N|N|N|N| +2443806|AAAAAAAAOBKEFCAA|1978-10-24|945|4113|316|1978|2|10|24|4|1978|316|4113|Tuesday|1978Q4|N|N|N|2443783|2444055|2443441|2443714|N|N|N|N|N| +2443807|AAAAAAAAPBKEFCAA|1978-10-25|945|4113|316|1978|3|10|25|4|1978|316|4113|Wednesday|1978Q4|N|N|N|2443783|2444055|2443442|2443715|N|N|N|N|N| +2443808|AAAAAAAAACKEFCAA|1978-10-26|945|4113|316|1978|4|10|26|4|1978|316|4113|Thursday|1978Q4|N|N|N|2443783|2444055|2443443|2443716|N|N|N|N|N| +2443809|AAAAAAAABCKEFCAA|1978-10-27|945|4113|316|1978|5|10|27|4|1978|316|4113|Friday|1978Q4|N|Y|N|2443783|2444055|2443444|2443717|N|N|N|N|N| +2443810|AAAAAAAACCKEFCAA|1978-10-28|945|4113|316|1978|6|10|28|4|1978|316|4113|Saturday|1978Q4|N|Y|N|2443783|2444055|2443445|2443718|N|N|N|N|N| +2443811|AAAAAAAADCKEFCAA|1978-10-29|945|4113|316|1978|0|10|29|4|1978|316|4113|Sunday|1978Q4|N|N|N|2443783|2444055|2443446|2443719|N|N|N|N|N| +2443812|AAAAAAAAECKEFCAA|1978-10-30|945|4113|316|1978|1|10|30|4|1978|316|4113|Monday|1978Q4|N|N|N|2443783|2444055|2443447|2443720|N|N|N|N|N| +2443813|AAAAAAAAFCKEFCAA|1978-10-31|945|4114|316|1978|2|10|31|4|1978|316|4114|Tuesday|1978Q4|N|N|N|2443783|2444055|2443448|2443721|N|N|N|N|N| +2443814|AAAAAAAAGCKEFCAA|1978-11-01|946|4114|316|1978|3|11|1|4|1978|316|4114|Wednesday|1978Q4|N|N|N|2443814|2444117|2443449|2443722|N|N|N|N|N| +2443815|AAAAAAAAHCKEFCAA|1978-11-02|946|4114|316|1978|4|11|2|4|1978|316|4114|Thursday|1978Q4|N|N|N|2443814|2444117|2443450|2443723|N|N|N|N|N| +2443816|AAAAAAAAICKEFCAA|1978-11-03|946|4114|316|1978|5|11|3|4|1978|316|4114|Friday|1978Q4|N|Y|N|2443814|2444117|2443451|2443724|N|N|N|N|N| +2443817|AAAAAAAAJCKEFCAA|1978-11-04|946|4114|316|1978|6|11|4|4|1978|316|4114|Saturday|1978Q4|N|Y|N|2443814|2444117|2443452|2443725|N|N|N|N|N| +2443818|AAAAAAAAKCKEFCAA|1978-11-05|946|4114|316|1978|0|11|5|4|1978|316|4114|Sunday|1978Q4|N|N|N|2443814|2444117|2443453|2443726|N|N|N|N|N| +2443819|AAAAAAAALCKEFCAA|1978-11-06|946|4114|316|1978|1|11|6|4|1978|316|4114|Monday|1978Q4|N|N|N|2443814|2444117|2443454|2443727|N|N|N|N|N| +2443820|AAAAAAAAMCKEFCAA|1978-11-07|946|4115|316|1978|2|11|7|4|1978|316|4115|Tuesday|1978Q4|N|N|N|2443814|2444117|2443455|2443728|N|N|N|N|N| +2443821|AAAAAAAANCKEFCAA|1978-11-08|946|4115|316|1978|3|11|8|4|1978|316|4115|Wednesday|1978Q4|N|N|N|2443814|2444117|2443456|2443729|N|N|N|N|N| +2443822|AAAAAAAAOCKEFCAA|1978-11-09|946|4115|316|1978|4|11|9|4|1978|316|4115|Thursday|1978Q4|N|N|N|2443814|2444117|2443457|2443730|N|N|N|N|N| +2443823|AAAAAAAAPCKEFCAA|1978-11-10|946|4115|316|1978|5|11|10|4|1978|316|4115|Friday|1978Q4|N|Y|N|2443814|2444117|2443458|2443731|N|N|N|N|N| +2443824|AAAAAAAAADKEFCAA|1978-11-11|946|4115|316|1978|6|11|11|4|1978|316|4115|Saturday|1978Q4|N|Y|N|2443814|2444117|2443459|2443732|N|N|N|N|N| +2443825|AAAAAAAABDKEFCAA|1978-11-12|946|4115|316|1978|0|11|12|4|1978|316|4115|Sunday|1978Q4|N|N|N|2443814|2444117|2443460|2443733|N|N|N|N|N| +2443826|AAAAAAAACDKEFCAA|1978-11-13|946|4115|316|1978|1|11|13|4|1978|316|4115|Monday|1978Q4|N|N|N|2443814|2444117|2443461|2443734|N|N|N|N|N| +2443827|AAAAAAAADDKEFCAA|1978-11-14|946|4116|316|1978|2|11|14|4|1978|316|4116|Tuesday|1978Q4|N|N|N|2443814|2444117|2443462|2443735|N|N|N|N|N| +2443828|AAAAAAAAEDKEFCAA|1978-11-15|946|4116|316|1978|3|11|15|4|1978|316|4116|Wednesday|1978Q4|N|N|N|2443814|2444117|2443463|2443736|N|N|N|N|N| +2443829|AAAAAAAAFDKEFCAA|1978-11-16|946|4116|316|1978|4|11|16|4|1978|316|4116|Thursday|1978Q4|N|N|N|2443814|2444117|2443464|2443737|N|N|N|N|N| +2443830|AAAAAAAAGDKEFCAA|1978-11-17|946|4116|316|1978|5|11|17|4|1978|316|4116|Friday|1978Q4|N|Y|N|2443814|2444117|2443465|2443738|N|N|N|N|N| +2443831|AAAAAAAAHDKEFCAA|1978-11-18|946|4116|316|1978|6|11|18|4|1978|316|4116|Saturday|1978Q4|N|Y|N|2443814|2444117|2443466|2443739|N|N|N|N|N| +2443832|AAAAAAAAIDKEFCAA|1978-11-19|946|4116|316|1978|0|11|19|4|1978|316|4116|Sunday|1978Q4|N|N|N|2443814|2444117|2443467|2443740|N|N|N|N|N| +2443833|AAAAAAAAJDKEFCAA|1978-11-20|946|4116|316|1978|1|11|20|4|1978|316|4116|Monday|1978Q4|N|N|N|2443814|2444117|2443468|2443741|N|N|N|N|N| +2443834|AAAAAAAAKDKEFCAA|1978-11-21|946|4117|316|1978|2|11|21|4|1978|316|4117|Tuesday|1978Q4|N|N|N|2443814|2444117|2443469|2443742|N|N|N|N|N| +2443835|AAAAAAAALDKEFCAA|1978-11-22|946|4117|316|1978|3|11|22|4|1978|316|4117|Wednesday|1978Q4|N|N|N|2443814|2444117|2443470|2443743|N|N|N|N|N| +2443836|AAAAAAAAMDKEFCAA|1978-11-23|946|4117|316|1978|4|11|23|4|1978|316|4117|Thursday|1978Q4|N|N|N|2443814|2444117|2443471|2443744|N|N|N|N|N| +2443837|AAAAAAAANDKEFCAA|1978-11-24|946|4117|316|1978|5|11|24|4|1978|316|4117|Friday|1978Q4|N|Y|N|2443814|2444117|2443472|2443745|N|N|N|N|N| +2443838|AAAAAAAAODKEFCAA|1978-11-25|946|4117|316|1978|6|11|25|4|1978|316|4117|Saturday|1978Q4|N|Y|N|2443814|2444117|2443473|2443746|N|N|N|N|N| +2443839|AAAAAAAAPDKEFCAA|1978-11-26|946|4117|316|1978|0|11|26|4|1978|316|4117|Sunday|1978Q4|N|N|N|2443814|2444117|2443474|2443747|N|N|N|N|N| +2443840|AAAAAAAAAEKEFCAA|1978-11-27|946|4117|316|1978|1|11|27|4|1978|316|4117|Monday|1978Q4|N|N|N|2443814|2444117|2443475|2443748|N|N|N|N|N| +2443841|AAAAAAAABEKEFCAA|1978-11-28|946|4118|316|1978|2|11|28|4|1978|316|4118|Tuesday|1978Q4|N|N|N|2443814|2444117|2443476|2443749|N|N|N|N|N| +2443842|AAAAAAAACEKEFCAA|1978-11-29|946|4118|316|1978|3|11|29|4|1978|316|4118|Wednesday|1978Q4|N|N|N|2443814|2444117|2443477|2443750|N|N|N|N|N| +2443843|AAAAAAAADEKEFCAA|1978-11-30|946|4118|316|1978|4|11|30|4|1978|316|4118|Thursday|1978Q4|N|N|N|2443814|2444117|2443478|2443751|N|N|N|N|N| +2443844|AAAAAAAAEEKEFCAA|1978-12-01|947|4118|317|1978|5|12|1|4|1978|317|4118|Friday|1978Q4|N|Y|N|2443844|2444177|2443479|2443752|N|N|N|N|N| +2443845|AAAAAAAAFEKEFCAA|1978-12-02|947|4118|317|1978|6|12|2|4|1978|317|4118|Saturday|1978Q4|N|Y|N|2443844|2444177|2443480|2443753|N|N|N|N|N| +2443846|AAAAAAAAGEKEFCAA|1978-12-03|947|4118|317|1978|0|12|3|4|1978|317|4118|Sunday|1978Q4|N|N|N|2443844|2444177|2443481|2443754|N|N|N|N|N| +2443847|AAAAAAAAHEKEFCAA|1978-12-04|947|4118|317|1978|1|12|4|4|1978|317|4118|Monday|1978Q4|N|N|N|2443844|2444177|2443482|2443755|N|N|N|N|N| +2443848|AAAAAAAAIEKEFCAA|1978-12-05|947|4119|317|1978|2|12|5|4|1978|317|4119|Tuesday|1978Q4|N|N|N|2443844|2444177|2443483|2443756|N|N|N|N|N| +2443849|AAAAAAAAJEKEFCAA|1978-12-06|947|4119|317|1978|3|12|6|4|1978|317|4119|Wednesday|1978Q4|N|N|N|2443844|2444177|2443484|2443757|N|N|N|N|N| +2443850|AAAAAAAAKEKEFCAA|1978-12-07|947|4119|317|1978|4|12|7|4|1978|317|4119|Thursday|1978Q4|N|N|N|2443844|2444177|2443485|2443758|N|N|N|N|N| +2443851|AAAAAAAALEKEFCAA|1978-12-08|947|4119|317|1978|5|12|8|4|1978|317|4119|Friday|1978Q4|N|Y|N|2443844|2444177|2443486|2443759|N|N|N|N|N| +2443852|AAAAAAAAMEKEFCAA|1978-12-09|947|4119|317|1978|6|12|9|4|1978|317|4119|Saturday|1978Q4|N|Y|N|2443844|2444177|2443487|2443760|N|N|N|N|N| +2443853|AAAAAAAANEKEFCAA|1978-12-10|947|4119|317|1978|0|12|10|4|1978|317|4119|Sunday|1978Q4|N|N|N|2443844|2444177|2443488|2443761|N|N|N|N|N| +2443854|AAAAAAAAOEKEFCAA|1978-12-11|947|4119|317|1978|1|12|11|4|1978|317|4119|Monday|1978Q4|N|N|N|2443844|2444177|2443489|2443762|N|N|N|N|N| +2443855|AAAAAAAAPEKEFCAA|1978-12-12|947|4120|317|1978|2|12|12|4|1978|317|4120|Tuesday|1978Q4|N|N|N|2443844|2444177|2443490|2443763|N|N|N|N|N| +2443856|AAAAAAAAAFKEFCAA|1978-12-13|947|4120|317|1978|3|12|13|4|1978|317|4120|Wednesday|1978Q4|N|N|N|2443844|2444177|2443491|2443764|N|N|N|N|N| +2443857|AAAAAAAABFKEFCAA|1978-12-14|947|4120|317|1978|4|12|14|4|1978|317|4120|Thursday|1978Q4|N|N|N|2443844|2444177|2443492|2443765|N|N|N|N|N| +2443858|AAAAAAAACFKEFCAA|1978-12-15|947|4120|317|1978|5|12|15|4|1978|317|4120|Friday|1978Q4|N|Y|N|2443844|2444177|2443493|2443766|N|N|N|N|N| +2443859|AAAAAAAADFKEFCAA|1978-12-16|947|4120|317|1978|6|12|16|4|1978|317|4120|Saturday|1978Q4|N|Y|N|2443844|2444177|2443494|2443767|N|N|N|N|N| +2443860|AAAAAAAAEFKEFCAA|1978-12-17|947|4120|317|1978|0|12|17|4|1978|317|4120|Sunday|1978Q4|N|N|N|2443844|2444177|2443495|2443768|N|N|N|N|N| +2443861|AAAAAAAAFFKEFCAA|1978-12-18|947|4120|317|1978|1|12|18|4|1978|317|4120|Monday|1978Q4|N|N|N|2443844|2444177|2443496|2443769|N|N|N|N|N| +2443862|AAAAAAAAGFKEFCAA|1978-12-19|947|4121|317|1978|2|12|19|4|1978|317|4121|Tuesday|1978Q4|N|N|N|2443844|2444177|2443497|2443770|N|N|N|N|N| +2443863|AAAAAAAAHFKEFCAA|1978-12-20|947|4121|317|1978|3|12|20|4|1978|317|4121|Wednesday|1978Q4|N|N|N|2443844|2444177|2443498|2443771|N|N|N|N|N| +2443864|AAAAAAAAIFKEFCAA|1978-12-21|947|4121|317|1978|4|12|21|4|1978|317|4121|Thursday|1978Q4|N|N|N|2443844|2444177|2443499|2443772|N|N|N|N|N| +2443865|AAAAAAAAJFKEFCAA|1978-12-22|947|4121|317|1978|5|12|22|4|1978|317|4121|Friday|1978Q4|N|Y|N|2443844|2444177|2443500|2443773|N|N|N|N|N| +2443866|AAAAAAAAKFKEFCAA|1978-12-23|947|4121|317|1978|6|12|23|4|1978|317|4121|Saturday|1978Q4|N|Y|N|2443844|2444177|2443501|2443774|N|N|N|N|N| +2443867|AAAAAAAALFKEFCAA|1978-12-24|947|4121|317|1978|0|12|24|4|1978|317|4121|Sunday|1978Q4|N|N|N|2443844|2444177|2443502|2443775|N|N|N|N|N| +2443868|AAAAAAAAMFKEFCAA|1978-12-25|947|4121|317|1978|1|12|25|4|1978|317|4121|Monday|1978Q4|N|N|N|2443844|2444177|2443503|2443776|N|N|N|N|N| +2443869|AAAAAAAANFKEFCAA|1978-12-26|947|4122|317|1978|2|12|26|4|1978|317|4122|Tuesday|1978Q4|Y|N|N|2443844|2444177|2443504|2443777|N|N|N|N|N| +2443870|AAAAAAAAOFKEFCAA|1978-12-27|947|4122|317|1978|3|12|27|4|1978|317|4122|Wednesday|1978Q4|N|N|Y|2443844|2444177|2443505|2443778|N|N|N|N|N| +2443871|AAAAAAAAPFKEFCAA|1978-12-28|947|4122|317|1978|4|12|28|4|1978|317|4122|Thursday|1978Q4|N|N|N|2443844|2444177|2443506|2443779|N|N|N|N|N| +2443872|AAAAAAAAAGKEFCAA|1978-12-29|947|4122|317|1978|5|12|29|4|1978|317|4122|Friday|1978Q4|N|Y|N|2443844|2444177|2443507|2443780|N|N|N|N|N| +2443873|AAAAAAAABGKEFCAA|1978-12-30|947|4122|317|1978|6|12|30|4|1978|317|4122|Saturday|1978Q4|N|Y|N|2443844|2444177|2443508|2443781|N|N|N|N|N| +2443874|AAAAAAAACGKEFCAA|1978-12-31|947|4122|317|1978|0|12|31|4|1978|317|4122|Sunday|1978Q4|N|N|N|2443844|2444177|2443509|2443782|N|N|N|N|N| +2443875|AAAAAAAADGKEFCAA|1979-01-01|948|4122|317|1979|1|1|1|1|1979|317|4122|Monday|1979Q1|Y|N|N|2443875|2443874|2443510|2443783|N|N|N|N|N| +2443876|AAAAAAAAEGKEFCAA|1979-01-02|948|4123|317|1979|2|1|2|1|1979|317|4123|Tuesday|1979Q1|N|N|Y|2443875|2443874|2443511|2443784|N|N|N|N|N| +2443877|AAAAAAAAFGKEFCAA|1979-01-03|948|4123|317|1979|3|1|3|1|1979|317|4123|Wednesday|1979Q1|N|N|N|2443875|2443874|2443512|2443785|N|N|N|N|N| +2443878|AAAAAAAAGGKEFCAA|1979-01-04|948|4123|317|1979|4|1|4|1|1979|317|4123|Thursday|1979Q1|N|N|N|2443875|2443874|2443513|2443786|N|N|N|N|N| +2443879|AAAAAAAAHGKEFCAA|1979-01-05|948|4123|317|1979|5|1|5|1|1979|317|4123|Friday|1979Q1|N|Y|N|2443875|2443874|2443514|2443787|N|N|N|N|N| +2443880|AAAAAAAAIGKEFCAA|1979-01-06|948|4123|317|1979|6|1|6|1|1979|317|4123|Saturday|1979Q1|N|Y|N|2443875|2443874|2443515|2443788|N|N|N|N|N| +2443881|AAAAAAAAJGKEFCAA|1979-01-07|948|4123|317|1979|0|1|7|1|1979|317|4123|Sunday|1979Q1|N|N|N|2443875|2443874|2443516|2443789|N|N|N|N|N| +2443882|AAAAAAAAKGKEFCAA|1979-01-08|948|4123|317|1979|1|1|8|1|1979|317|4123|Monday|1979Q1|N|N|N|2443875|2443874|2443517|2443790|N|N|N|N|N| +2443883|AAAAAAAALGKEFCAA|1979-01-09|948|4124|317|1979|2|1|9|1|1979|317|4124|Tuesday|1979Q1|N|N|N|2443875|2443874|2443518|2443791|N|N|N|N|N| +2443884|AAAAAAAAMGKEFCAA|1979-01-10|948|4124|317|1979|3|1|10|1|1979|317|4124|Wednesday|1979Q1|N|N|N|2443875|2443874|2443519|2443792|N|N|N|N|N| +2443885|AAAAAAAANGKEFCAA|1979-01-11|948|4124|317|1979|4|1|11|1|1979|317|4124|Thursday|1979Q1|N|N|N|2443875|2443874|2443520|2443793|N|N|N|N|N| +2443886|AAAAAAAAOGKEFCAA|1979-01-12|948|4124|317|1979|5|1|12|1|1979|317|4124|Friday|1979Q1|N|Y|N|2443875|2443874|2443521|2443794|N|N|N|N|N| +2443887|AAAAAAAAPGKEFCAA|1979-01-13|948|4124|317|1979|6|1|13|1|1979|317|4124|Saturday|1979Q1|N|Y|N|2443875|2443874|2443522|2443795|N|N|N|N|N| +2443888|AAAAAAAAAHKEFCAA|1979-01-14|948|4124|317|1979|0|1|14|1|1979|317|4124|Sunday|1979Q1|N|N|N|2443875|2443874|2443523|2443796|N|N|N|N|N| +2443889|AAAAAAAABHKEFCAA|1979-01-15|948|4124|317|1979|1|1|15|1|1979|317|4124|Monday|1979Q1|N|N|N|2443875|2443874|2443524|2443797|N|N|N|N|N| +2443890|AAAAAAAACHKEFCAA|1979-01-16|948|4125|317|1979|2|1|16|1|1979|317|4125|Tuesday|1979Q1|N|N|N|2443875|2443874|2443525|2443798|N|N|N|N|N| +2443891|AAAAAAAADHKEFCAA|1979-01-17|948|4125|317|1979|3|1|17|1|1979|317|4125|Wednesday|1979Q1|N|N|N|2443875|2443874|2443526|2443799|N|N|N|N|N| +2443892|AAAAAAAAEHKEFCAA|1979-01-18|948|4125|317|1979|4|1|18|1|1979|317|4125|Thursday|1979Q1|N|N|N|2443875|2443874|2443527|2443800|N|N|N|N|N| +2443893|AAAAAAAAFHKEFCAA|1979-01-19|948|4125|317|1979|5|1|19|1|1979|317|4125|Friday|1979Q1|N|Y|N|2443875|2443874|2443528|2443801|N|N|N|N|N| +2443894|AAAAAAAAGHKEFCAA|1979-01-20|948|4125|317|1979|6|1|20|1|1979|317|4125|Saturday|1979Q1|N|Y|N|2443875|2443874|2443529|2443802|N|N|N|N|N| +2443895|AAAAAAAAHHKEFCAA|1979-01-21|948|4125|317|1979|0|1|21|1|1979|317|4125|Sunday|1979Q1|N|N|N|2443875|2443874|2443530|2443803|N|N|N|N|N| +2443896|AAAAAAAAIHKEFCAA|1979-01-22|948|4125|317|1979|1|1|22|1|1979|317|4125|Monday|1979Q1|N|N|N|2443875|2443874|2443531|2443804|N|N|N|N|N| +2443897|AAAAAAAAJHKEFCAA|1979-01-23|948|4126|317|1979|2|1|23|1|1979|317|4126|Tuesday|1979Q1|N|N|N|2443875|2443874|2443532|2443805|N|N|N|N|N| +2443898|AAAAAAAAKHKEFCAA|1979-01-24|948|4126|317|1979|3|1|24|1|1979|317|4126|Wednesday|1979Q1|N|N|N|2443875|2443874|2443533|2443806|N|N|N|N|N| +2443899|AAAAAAAALHKEFCAA|1979-01-25|948|4126|317|1979|4|1|25|1|1979|317|4126|Thursday|1979Q1|N|N|N|2443875|2443874|2443534|2443807|N|N|N|N|N| +2443900|AAAAAAAAMHKEFCAA|1979-01-26|948|4126|317|1979|5|1|26|1|1979|317|4126|Friday|1979Q1|N|Y|N|2443875|2443874|2443535|2443808|N|N|N|N|N| +2443901|AAAAAAAANHKEFCAA|1979-01-27|948|4126|317|1979|6|1|27|1|1979|317|4126|Saturday|1979Q1|N|Y|N|2443875|2443874|2443536|2443809|N|N|N|N|N| +2443902|AAAAAAAAOHKEFCAA|1979-01-28|948|4126|317|1979|0|1|28|1|1979|317|4126|Sunday|1979Q1|N|N|N|2443875|2443874|2443537|2443810|N|N|N|N|N| +2443903|AAAAAAAAPHKEFCAA|1979-01-29|948|4126|317|1979|1|1|29|1|1979|317|4126|Monday|1979Q1|N|N|N|2443875|2443874|2443538|2443811|N|N|N|N|N| +2443904|AAAAAAAAAIKEFCAA|1979-01-30|948|4127|317|1979|2|1|30|1|1979|317|4127|Tuesday|1979Q1|N|N|N|2443875|2443874|2443539|2443812|N|N|N|N|N| +2443905|AAAAAAAABIKEFCAA|1979-01-31|948|4127|317|1979|3|1|31|1|1979|317|4127|Wednesday|1979Q1|N|N|N|2443875|2443874|2443540|2443813|N|N|N|N|N| +2443906|AAAAAAAACIKEFCAA|1979-02-01|949|4127|317|1979|4|2|1|1|1979|317|4127|Thursday|1979Q1|N|N|N|2443906|2443936|2443541|2443814|N|N|N|N|N| +2443907|AAAAAAAADIKEFCAA|1979-02-02|949|4127|317|1979|5|2|2|1|1979|317|4127|Friday|1979Q1|N|Y|N|2443906|2443936|2443542|2443815|N|N|N|N|N| +2443908|AAAAAAAAEIKEFCAA|1979-02-03|949|4127|317|1979|6|2|3|1|1979|317|4127|Saturday|1979Q1|N|Y|N|2443906|2443936|2443543|2443816|N|N|N|N|N| +2443909|AAAAAAAAFIKEFCAA|1979-02-04|949|4127|317|1979|0|2|4|1|1979|317|4127|Sunday|1979Q1|N|N|N|2443906|2443936|2443544|2443817|N|N|N|N|N| +2443910|AAAAAAAAGIKEFCAA|1979-02-05|949|4127|317|1979|1|2|5|1|1979|317|4127|Monday|1979Q1|N|N|N|2443906|2443936|2443545|2443818|N|N|N|N|N| +2443911|AAAAAAAAHIKEFCAA|1979-02-06|949|4128|317|1979|2|2|6|1|1979|317|4128|Tuesday|1979Q1|N|N|N|2443906|2443936|2443546|2443819|N|N|N|N|N| +2443912|AAAAAAAAIIKEFCAA|1979-02-07|949|4128|317|1979|3|2|7|1|1979|317|4128|Wednesday|1979Q1|N|N|N|2443906|2443936|2443547|2443820|N|N|N|N|N| +2443913|AAAAAAAAJIKEFCAA|1979-02-08|949|4128|317|1979|4|2|8|1|1979|317|4128|Thursday|1979Q1|N|N|N|2443906|2443936|2443548|2443821|N|N|N|N|N| +2443914|AAAAAAAAKIKEFCAA|1979-02-09|949|4128|317|1979|5|2|9|1|1979|317|4128|Friday|1979Q1|N|Y|N|2443906|2443936|2443549|2443822|N|N|N|N|N| +2443915|AAAAAAAALIKEFCAA|1979-02-10|949|4128|317|1979|6|2|10|1|1979|317|4128|Saturday|1979Q1|N|Y|N|2443906|2443936|2443550|2443823|N|N|N|N|N| +2443916|AAAAAAAAMIKEFCAA|1979-02-11|949|4128|317|1979|0|2|11|1|1979|317|4128|Sunday|1979Q1|N|N|N|2443906|2443936|2443551|2443824|N|N|N|N|N| +2443917|AAAAAAAANIKEFCAA|1979-02-12|949|4128|317|1979|1|2|12|1|1979|317|4128|Monday|1979Q1|N|N|N|2443906|2443936|2443552|2443825|N|N|N|N|N| +2443918|AAAAAAAAOIKEFCAA|1979-02-13|949|4129|317|1979|2|2|13|1|1979|317|4129|Tuesday|1979Q1|N|N|N|2443906|2443936|2443553|2443826|N|N|N|N|N| +2443919|AAAAAAAAPIKEFCAA|1979-02-14|949|4129|317|1979|3|2|14|1|1979|317|4129|Wednesday|1979Q1|N|N|N|2443906|2443936|2443554|2443827|N|N|N|N|N| +2443920|AAAAAAAAAJKEFCAA|1979-02-15|949|4129|317|1979|4|2|15|1|1979|317|4129|Thursday|1979Q1|N|N|N|2443906|2443936|2443555|2443828|N|N|N|N|N| +2443921|AAAAAAAABJKEFCAA|1979-02-16|949|4129|317|1979|5|2|16|1|1979|317|4129|Friday|1979Q1|N|Y|N|2443906|2443936|2443556|2443829|N|N|N|N|N| +2443922|AAAAAAAACJKEFCAA|1979-02-17|949|4129|317|1979|6|2|17|1|1979|317|4129|Saturday|1979Q1|N|Y|N|2443906|2443936|2443557|2443830|N|N|N|N|N| +2443923|AAAAAAAADJKEFCAA|1979-02-18|949|4129|317|1979|0|2|18|1|1979|317|4129|Sunday|1979Q1|N|N|N|2443906|2443936|2443558|2443831|N|N|N|N|N| +2443924|AAAAAAAAEJKEFCAA|1979-02-19|949|4129|317|1979|1|2|19|1|1979|317|4129|Monday|1979Q1|N|N|N|2443906|2443936|2443559|2443832|N|N|N|N|N| +2443925|AAAAAAAAFJKEFCAA|1979-02-20|949|4130|317|1979|2|2|20|1|1979|317|4130|Tuesday|1979Q1|N|N|N|2443906|2443936|2443560|2443833|N|N|N|N|N| +2443926|AAAAAAAAGJKEFCAA|1979-02-21|949|4130|317|1979|3|2|21|1|1979|317|4130|Wednesday|1979Q1|N|N|N|2443906|2443936|2443561|2443834|N|N|N|N|N| +2443927|AAAAAAAAHJKEFCAA|1979-02-22|949|4130|317|1979|4|2|22|1|1979|317|4130|Thursday|1979Q1|N|N|N|2443906|2443936|2443562|2443835|N|N|N|N|N| +2443928|AAAAAAAAIJKEFCAA|1979-02-23|949|4130|317|1979|5|2|23|1|1979|317|4130|Friday|1979Q1|N|Y|N|2443906|2443936|2443563|2443836|N|N|N|N|N| +2443929|AAAAAAAAJJKEFCAA|1979-02-24|949|4130|317|1979|6|2|24|1|1979|317|4130|Saturday|1979Q1|N|Y|N|2443906|2443936|2443564|2443837|N|N|N|N|N| +2443930|AAAAAAAAKJKEFCAA|1979-02-25|949|4130|317|1979|0|2|25|1|1979|317|4130|Sunday|1979Q1|N|N|N|2443906|2443936|2443565|2443838|N|N|N|N|N| +2443931|AAAAAAAALJKEFCAA|1979-02-26|949|4130|317|1979|1|2|26|1|1979|317|4130|Monday|1979Q1|N|N|N|2443906|2443936|2443566|2443839|N|N|N|N|N| +2443932|AAAAAAAAMJKEFCAA|1979-02-27|949|4131|317|1979|2|2|27|1|1979|317|4131|Tuesday|1979Q1|N|N|N|2443906|2443936|2443567|2443840|N|N|N|N|N| +2443933|AAAAAAAANJKEFCAA|1979-02-28|949|4131|317|1979|3|2|28|1|1979|317|4131|Wednesday|1979Q1|N|N|N|2443906|2443936|2443568|2443841|N|N|N|N|N| +2443934|AAAAAAAAOJKEFCAA|1979-03-01|950|4131|318|1979|4|3|1|1|1979|318|4131|Thursday|1979Q1|N|N|N|2443934|2443992|2443569|2443842|N|N|N|N|N| +2443935|AAAAAAAAPJKEFCAA|1979-03-02|950|4131|318|1979|5|3|2|1|1979|318|4131|Friday|1979Q1|N|Y|N|2443934|2443992|2443570|2443843|N|N|N|N|N| +2443936|AAAAAAAAAKKEFCAA|1979-03-03|950|4131|318|1979|6|3|3|1|1979|318|4131|Saturday|1979Q1|N|Y|N|2443934|2443992|2443571|2443844|N|N|N|N|N| +2443937|AAAAAAAABKKEFCAA|1979-03-04|950|4131|318|1979|0|3|4|1|1979|318|4131|Sunday|1979Q1|N|N|N|2443934|2443992|2443572|2443845|N|N|N|N|N| +2443938|AAAAAAAACKKEFCAA|1979-03-05|950|4131|318|1979|1|3|5|1|1979|318|4131|Monday|1979Q1|N|N|N|2443934|2443992|2443573|2443846|N|N|N|N|N| +2443939|AAAAAAAADKKEFCAA|1979-03-06|950|4132|318|1979|2|3|6|1|1979|318|4132|Tuesday|1979Q1|N|N|N|2443934|2443992|2443574|2443847|N|N|N|N|N| +2443940|AAAAAAAAEKKEFCAA|1979-03-07|950|4132|318|1979|3|3|7|1|1979|318|4132|Wednesday|1979Q1|N|N|N|2443934|2443992|2443575|2443848|N|N|N|N|N| +2443941|AAAAAAAAFKKEFCAA|1979-03-08|950|4132|318|1979|4|3|8|1|1979|318|4132|Thursday|1979Q1|N|N|N|2443934|2443992|2443576|2443849|N|N|N|N|N| +2443942|AAAAAAAAGKKEFCAA|1979-03-09|950|4132|318|1979|5|3|9|1|1979|318|4132|Friday|1979Q1|N|Y|N|2443934|2443992|2443577|2443850|N|N|N|N|N| +2443943|AAAAAAAAHKKEFCAA|1979-03-10|950|4132|318|1979|6|3|10|1|1979|318|4132|Saturday|1979Q1|N|Y|N|2443934|2443992|2443578|2443851|N|N|N|N|N| +2443944|AAAAAAAAIKKEFCAA|1979-03-11|950|4132|318|1979|0|3|11|1|1979|318|4132|Sunday|1979Q1|N|N|N|2443934|2443992|2443579|2443852|N|N|N|N|N| +2443945|AAAAAAAAJKKEFCAA|1979-03-12|950|4132|318|1979|1|3|12|1|1979|318|4132|Monday|1979Q1|N|N|N|2443934|2443992|2443580|2443853|N|N|N|N|N| +2443946|AAAAAAAAKKKEFCAA|1979-03-13|950|4133|318|1979|2|3|13|1|1979|318|4133|Tuesday|1979Q1|N|N|N|2443934|2443992|2443581|2443854|N|N|N|N|N| +2443947|AAAAAAAALKKEFCAA|1979-03-14|950|4133|318|1979|3|3|14|1|1979|318|4133|Wednesday|1979Q1|N|N|N|2443934|2443992|2443582|2443855|N|N|N|N|N| +2443948|AAAAAAAAMKKEFCAA|1979-03-15|950|4133|318|1979|4|3|15|1|1979|318|4133|Thursday|1979Q1|N|N|N|2443934|2443992|2443583|2443856|N|N|N|N|N| +2443949|AAAAAAAANKKEFCAA|1979-03-16|950|4133|318|1979|5|3|16|1|1979|318|4133|Friday|1979Q1|N|Y|N|2443934|2443992|2443584|2443857|N|N|N|N|N| +2443950|AAAAAAAAOKKEFCAA|1979-03-17|950|4133|318|1979|6|3|17|1|1979|318|4133|Saturday|1979Q1|N|Y|N|2443934|2443992|2443585|2443858|N|N|N|N|N| +2443951|AAAAAAAAPKKEFCAA|1979-03-18|950|4133|318|1979|0|3|18|1|1979|318|4133|Sunday|1979Q1|N|N|N|2443934|2443992|2443586|2443859|N|N|N|N|N| +2443952|AAAAAAAAALKEFCAA|1979-03-19|950|4133|318|1979|1|3|19|1|1979|318|4133|Monday|1979Q1|N|N|N|2443934|2443992|2443587|2443860|N|N|N|N|N| +2443953|AAAAAAAABLKEFCAA|1979-03-20|950|4134|318|1979|2|3|20|1|1979|318|4134|Tuesday|1979Q1|N|N|N|2443934|2443992|2443588|2443861|N|N|N|N|N| +2443954|AAAAAAAACLKEFCAA|1979-03-21|950|4134|318|1979|3|3|21|1|1979|318|4134|Wednesday|1979Q1|N|N|N|2443934|2443992|2443589|2443862|N|N|N|N|N| +2443955|AAAAAAAADLKEFCAA|1979-03-22|950|4134|318|1979|4|3|22|1|1979|318|4134|Thursday|1979Q1|N|N|N|2443934|2443992|2443590|2443863|N|N|N|N|N| +2443956|AAAAAAAAELKEFCAA|1979-03-23|950|4134|318|1979|5|3|23|1|1979|318|4134|Friday|1979Q1|N|Y|N|2443934|2443992|2443591|2443864|N|N|N|N|N| +2443957|AAAAAAAAFLKEFCAA|1979-03-24|950|4134|318|1979|6|3|24|1|1979|318|4134|Saturday|1979Q1|N|Y|N|2443934|2443992|2443592|2443865|N|N|N|N|N| +2443958|AAAAAAAAGLKEFCAA|1979-03-25|950|4134|318|1979|0|3|25|1|1979|318|4134|Sunday|1979Q1|N|N|N|2443934|2443992|2443593|2443866|N|N|N|N|N| +2443959|AAAAAAAAHLKEFCAA|1979-03-26|950|4134|318|1979|1|3|26|1|1979|318|4134|Monday|1979Q1|N|N|N|2443934|2443992|2443594|2443867|N|N|N|N|N| +2443960|AAAAAAAAILKEFCAA|1979-03-27|950|4135|318|1979|2|3|27|1|1979|318|4135|Tuesday|1979Q1|N|N|N|2443934|2443992|2443595|2443868|N|N|N|N|N| +2443961|AAAAAAAAJLKEFCAA|1979-03-28|950|4135|318|1979|3|3|28|1|1979|318|4135|Wednesday|1979Q1|N|N|N|2443934|2443992|2443596|2443869|N|N|N|N|N| +2443962|AAAAAAAAKLKEFCAA|1979-03-29|950|4135|318|1979|4|3|29|1|1979|318|4135|Thursday|1979Q1|N|N|N|2443934|2443992|2443597|2443870|N|N|N|N|N| +2443963|AAAAAAAALLKEFCAA|1979-03-30|950|4135|318|1979|5|3|30|1|1979|318|4135|Friday|1979Q1|N|Y|N|2443934|2443992|2443598|2443871|N|N|N|N|N| +2443964|AAAAAAAAMLKEFCAA|1979-03-31|950|4135|318|1979|6|3|31|1|1979|318|4135|Saturday|1979Q1|N|Y|N|2443934|2443992|2443599|2443872|N|N|N|N|N| +2443965|AAAAAAAANLKEFCAA|1979-04-01|951|4135|318|1979|0|4|1|1|1979|318|4135|Sunday|1979Q1|N|N|N|2443965|2444054|2443600|2443875|N|N|N|N|N| +2443966|AAAAAAAAOLKEFCAA|1979-04-02|951|4135|318|1979|1|4|2|2|1979|318|4135|Monday|1979Q2|N|N|N|2443965|2444054|2443601|2443876|N|N|N|N|N| +2443967|AAAAAAAAPLKEFCAA|1979-04-03|951|4136|318|1979|2|4|3|2|1979|318|4136|Tuesday|1979Q2|N|N|N|2443965|2444054|2443602|2443877|N|N|N|N|N| +2443968|AAAAAAAAAMKEFCAA|1979-04-04|951|4136|318|1979|3|4|4|2|1979|318|4136|Wednesday|1979Q2|N|N|N|2443965|2444054|2443603|2443878|N|N|N|N|N| +2443969|AAAAAAAABMKEFCAA|1979-04-05|951|4136|318|1979|4|4|5|2|1979|318|4136|Thursday|1979Q2|N|N|N|2443965|2444054|2443604|2443879|N|N|N|N|N| +2443970|AAAAAAAACMKEFCAA|1979-04-06|951|4136|318|1979|5|4|6|2|1979|318|4136|Friday|1979Q2|N|Y|N|2443965|2444054|2443605|2443880|N|N|N|N|N| +2443971|AAAAAAAADMKEFCAA|1979-04-07|951|4136|318|1979|6|4|7|2|1979|318|4136|Saturday|1979Q2|N|Y|N|2443965|2444054|2443606|2443881|N|N|N|N|N| +2443972|AAAAAAAAEMKEFCAA|1979-04-08|951|4136|318|1979|0|4|8|2|1979|318|4136|Sunday|1979Q2|N|N|N|2443965|2444054|2443607|2443882|N|N|N|N|N| +2443973|AAAAAAAAFMKEFCAA|1979-04-09|951|4136|318|1979|1|4|9|2|1979|318|4136|Monday|1979Q2|N|N|N|2443965|2444054|2443608|2443883|N|N|N|N|N| +2443974|AAAAAAAAGMKEFCAA|1979-04-10|951|4137|318|1979|2|4|10|2|1979|318|4137|Tuesday|1979Q2|N|N|N|2443965|2444054|2443609|2443884|N|N|N|N|N| +2443975|AAAAAAAAHMKEFCAA|1979-04-11|951|4137|318|1979|3|4|11|2|1979|318|4137|Wednesday|1979Q2|N|N|N|2443965|2444054|2443610|2443885|N|N|N|N|N| +2443976|AAAAAAAAIMKEFCAA|1979-04-12|951|4137|318|1979|4|4|12|2|1979|318|4137|Thursday|1979Q2|N|N|N|2443965|2444054|2443611|2443886|N|N|N|N|N| +2443977|AAAAAAAAJMKEFCAA|1979-04-13|951|4137|318|1979|5|4|13|2|1979|318|4137|Friday|1979Q2|N|Y|N|2443965|2444054|2443612|2443887|N|N|N|N|N| +2443978|AAAAAAAAKMKEFCAA|1979-04-14|951|4137|318|1979|6|4|14|2|1979|318|4137|Saturday|1979Q2|N|Y|N|2443965|2444054|2443613|2443888|N|N|N|N|N| +2443979|AAAAAAAALMKEFCAA|1979-04-15|951|4137|318|1979|0|4|15|2|1979|318|4137|Sunday|1979Q2|N|N|N|2443965|2444054|2443614|2443889|N|N|N|N|N| +2443980|AAAAAAAAMMKEFCAA|1979-04-16|951|4137|318|1979|1|4|16|2|1979|318|4137|Monday|1979Q2|N|N|N|2443965|2444054|2443615|2443890|N|N|N|N|N| +2443981|AAAAAAAANMKEFCAA|1979-04-17|951|4138|318|1979|2|4|17|2|1979|318|4138|Tuesday|1979Q2|N|N|N|2443965|2444054|2443616|2443891|N|N|N|N|N| +2443982|AAAAAAAAOMKEFCAA|1979-04-18|951|4138|318|1979|3|4|18|2|1979|318|4138|Wednesday|1979Q2|N|N|N|2443965|2444054|2443617|2443892|N|N|N|N|N| +2443983|AAAAAAAAPMKEFCAA|1979-04-19|951|4138|318|1979|4|4|19|2|1979|318|4138|Thursday|1979Q2|N|N|N|2443965|2444054|2443618|2443893|N|N|N|N|N| +2443984|AAAAAAAAANKEFCAA|1979-04-20|951|4138|318|1979|5|4|20|2|1979|318|4138|Friday|1979Q2|N|Y|N|2443965|2444054|2443619|2443894|N|N|N|N|N| +2443985|AAAAAAAABNKEFCAA|1979-04-21|951|4138|318|1979|6|4|21|2|1979|318|4138|Saturday|1979Q2|N|Y|N|2443965|2444054|2443620|2443895|N|N|N|N|N| +2443986|AAAAAAAACNKEFCAA|1979-04-22|951|4138|318|1979|0|4|22|2|1979|318|4138|Sunday|1979Q2|N|N|N|2443965|2444054|2443621|2443896|N|N|N|N|N| +2443987|AAAAAAAADNKEFCAA|1979-04-23|951|4138|318|1979|1|4|23|2|1979|318|4138|Monday|1979Q2|N|N|N|2443965|2444054|2443622|2443897|N|N|N|N|N| +2443988|AAAAAAAAENKEFCAA|1979-04-24|951|4139|318|1979|2|4|24|2|1979|318|4139|Tuesday|1979Q2|N|N|N|2443965|2444054|2443623|2443898|N|N|N|N|N| +2443989|AAAAAAAAFNKEFCAA|1979-04-25|951|4139|318|1979|3|4|25|2|1979|318|4139|Wednesday|1979Q2|N|N|N|2443965|2444054|2443624|2443899|N|N|N|N|N| +2443990|AAAAAAAAGNKEFCAA|1979-04-26|951|4139|318|1979|4|4|26|2|1979|318|4139|Thursday|1979Q2|N|N|N|2443965|2444054|2443625|2443900|N|N|N|N|N| +2443991|AAAAAAAAHNKEFCAA|1979-04-27|951|4139|318|1979|5|4|27|2|1979|318|4139|Friday|1979Q2|N|Y|N|2443965|2444054|2443626|2443901|N|N|N|N|N| +2443992|AAAAAAAAINKEFCAA|1979-04-28|951|4139|318|1979|6|4|28|2|1979|318|4139|Saturday|1979Q2|N|Y|N|2443965|2444054|2443627|2443902|N|N|N|N|N| +2443993|AAAAAAAAJNKEFCAA|1979-04-29|951|4139|318|1979|0|4|29|2|1979|318|4139|Sunday|1979Q2|N|N|N|2443965|2444054|2443628|2443903|N|N|N|N|N| +2443994|AAAAAAAAKNKEFCAA|1979-04-30|951|4139|318|1979|1|4|30|2|1979|318|4139|Monday|1979Q2|N|N|N|2443965|2444054|2443629|2443904|N|N|N|N|N| +2443995|AAAAAAAALNKEFCAA|1979-05-01|952|4140|318|1979|2|5|1|2|1979|318|4140|Tuesday|1979Q2|N|N|N|2443995|2444114|2443630|2443905|N|N|N|N|N| +2443996|AAAAAAAAMNKEFCAA|1979-05-02|952|4140|318|1979|3|5|2|2|1979|318|4140|Wednesday|1979Q2|N|N|N|2443995|2444114|2443631|2443906|N|N|N|N|N| +2443997|AAAAAAAANNKEFCAA|1979-05-03|952|4140|318|1979|4|5|3|2|1979|318|4140|Thursday|1979Q2|N|N|N|2443995|2444114|2443632|2443907|N|N|N|N|N| +2443998|AAAAAAAAONKEFCAA|1979-05-04|952|4140|318|1979|5|5|4|2|1979|318|4140|Friday|1979Q2|N|Y|N|2443995|2444114|2443633|2443908|N|N|N|N|N| +2443999|AAAAAAAAPNKEFCAA|1979-05-05|952|4140|318|1979|6|5|5|2|1979|318|4140|Saturday|1979Q2|N|Y|N|2443995|2444114|2443634|2443909|N|N|N|N|N| +2444000|AAAAAAAAAOKEFCAA|1979-05-06|952|4140|318|1979|0|5|6|2|1979|318|4140|Sunday|1979Q2|N|N|N|2443995|2444114|2443635|2443910|N|N|N|N|N| +2444001|AAAAAAAABOKEFCAA|1979-05-07|952|4140|318|1979|1|5|7|2|1979|318|4140|Monday|1979Q2|N|N|N|2443995|2444114|2443636|2443911|N|N|N|N|N| +2444002|AAAAAAAACOKEFCAA|1979-05-08|952|4141|318|1979|2|5|8|2|1979|318|4141|Tuesday|1979Q2|N|N|N|2443995|2444114|2443637|2443912|N|N|N|N|N| +2444003|AAAAAAAADOKEFCAA|1979-05-09|952|4141|318|1979|3|5|9|2|1979|318|4141|Wednesday|1979Q2|N|N|N|2443995|2444114|2443638|2443913|N|N|N|N|N| +2444004|AAAAAAAAEOKEFCAA|1979-05-10|952|4141|318|1979|4|5|10|2|1979|318|4141|Thursday|1979Q2|N|N|N|2443995|2444114|2443639|2443914|N|N|N|N|N| +2444005|AAAAAAAAFOKEFCAA|1979-05-11|952|4141|318|1979|5|5|11|2|1979|318|4141|Friday|1979Q2|N|Y|N|2443995|2444114|2443640|2443915|N|N|N|N|N| +2444006|AAAAAAAAGOKEFCAA|1979-05-12|952|4141|318|1979|6|5|12|2|1979|318|4141|Saturday|1979Q2|N|Y|N|2443995|2444114|2443641|2443916|N|N|N|N|N| +2444007|AAAAAAAAHOKEFCAA|1979-05-13|952|4141|318|1979|0|5|13|2|1979|318|4141|Sunday|1979Q2|N|N|N|2443995|2444114|2443642|2443917|N|N|N|N|N| +2444008|AAAAAAAAIOKEFCAA|1979-05-14|952|4141|318|1979|1|5|14|2|1979|318|4141|Monday|1979Q2|N|N|N|2443995|2444114|2443643|2443918|N|N|N|N|N| +2444009|AAAAAAAAJOKEFCAA|1979-05-15|952|4142|318|1979|2|5|15|2|1979|318|4142|Tuesday|1979Q2|N|N|N|2443995|2444114|2443644|2443919|N|N|N|N|N| +2444010|AAAAAAAAKOKEFCAA|1979-05-16|952|4142|318|1979|3|5|16|2|1979|318|4142|Wednesday|1979Q2|N|N|N|2443995|2444114|2443645|2443920|N|N|N|N|N| +2444011|AAAAAAAALOKEFCAA|1979-05-17|952|4142|318|1979|4|5|17|2|1979|318|4142|Thursday|1979Q2|N|N|N|2443995|2444114|2443646|2443921|N|N|N|N|N| +2444012|AAAAAAAAMOKEFCAA|1979-05-18|952|4142|318|1979|5|5|18|2|1979|318|4142|Friday|1979Q2|N|Y|N|2443995|2444114|2443647|2443922|N|N|N|N|N| +2444013|AAAAAAAANOKEFCAA|1979-05-19|952|4142|318|1979|6|5|19|2|1979|318|4142|Saturday|1979Q2|N|Y|N|2443995|2444114|2443648|2443923|N|N|N|N|N| +2444014|AAAAAAAAOOKEFCAA|1979-05-20|952|4142|318|1979|0|5|20|2|1979|318|4142|Sunday|1979Q2|N|N|N|2443995|2444114|2443649|2443924|N|N|N|N|N| +2444015|AAAAAAAAPOKEFCAA|1979-05-21|952|4142|318|1979|1|5|21|2|1979|318|4142|Monday|1979Q2|N|N|N|2443995|2444114|2443650|2443925|N|N|N|N|N| +2444016|AAAAAAAAAPKEFCAA|1979-05-22|952|4143|318|1979|2|5|22|2|1979|318|4143|Tuesday|1979Q2|N|N|N|2443995|2444114|2443651|2443926|N|N|N|N|N| +2444017|AAAAAAAABPKEFCAA|1979-05-23|952|4143|318|1979|3|5|23|2|1979|318|4143|Wednesday|1979Q2|N|N|N|2443995|2444114|2443652|2443927|N|N|N|N|N| +2444018|AAAAAAAACPKEFCAA|1979-05-24|952|4143|318|1979|4|5|24|2|1979|318|4143|Thursday|1979Q2|N|N|N|2443995|2444114|2443653|2443928|N|N|N|N|N| +2444019|AAAAAAAADPKEFCAA|1979-05-25|952|4143|318|1979|5|5|25|2|1979|318|4143|Friday|1979Q2|N|Y|N|2443995|2444114|2443654|2443929|N|N|N|N|N| +2444020|AAAAAAAAEPKEFCAA|1979-05-26|952|4143|318|1979|6|5|26|2|1979|318|4143|Saturday|1979Q2|N|Y|N|2443995|2444114|2443655|2443930|N|N|N|N|N| +2444021|AAAAAAAAFPKEFCAA|1979-05-27|952|4143|318|1979|0|5|27|2|1979|318|4143|Sunday|1979Q2|N|N|N|2443995|2444114|2443656|2443931|N|N|N|N|N| +2444022|AAAAAAAAGPKEFCAA|1979-05-28|952|4143|318|1979|1|5|28|2|1979|318|4143|Monday|1979Q2|N|N|N|2443995|2444114|2443657|2443932|N|N|N|N|N| +2444023|AAAAAAAAHPKEFCAA|1979-05-29|952|4144|318|1979|2|5|29|2|1979|318|4144|Tuesday|1979Q2|N|N|N|2443995|2444114|2443658|2443933|N|N|N|N|N| +2444024|AAAAAAAAIPKEFCAA|1979-05-30|952|4144|318|1979|3|5|30|2|1979|318|4144|Wednesday|1979Q2|N|N|N|2443995|2444114|2443659|2443934|N|N|N|N|N| +2444025|AAAAAAAAJPKEFCAA|1979-05-31|952|4144|318|1979|4|5|31|2|1979|318|4144|Thursday|1979Q2|N|N|N|2443995|2444114|2443660|2443935|N|N|N|N|N| +2444026|AAAAAAAAKPKEFCAA|1979-06-01|953|4144|319|1979|5|6|1|2|1979|319|4144|Friday|1979Q2|N|Y|N|2444026|2444176|2443661|2443936|N|N|N|N|N| +2444027|AAAAAAAALPKEFCAA|1979-06-02|953|4144|319|1979|6|6|2|2|1979|319|4144|Saturday|1979Q2|N|Y|N|2444026|2444176|2443662|2443937|N|N|N|N|N| +2444028|AAAAAAAAMPKEFCAA|1979-06-03|953|4144|319|1979|0|6|3|2|1979|319|4144|Sunday|1979Q2|N|N|N|2444026|2444176|2443663|2443938|N|N|N|N|N| +2444029|AAAAAAAANPKEFCAA|1979-06-04|953|4144|319|1979|1|6|4|2|1979|319|4144|Monday|1979Q2|N|N|N|2444026|2444176|2443664|2443939|N|N|N|N|N| +2444030|AAAAAAAAOPKEFCAA|1979-06-05|953|4145|319|1979|2|6|5|2|1979|319|4145|Tuesday|1979Q2|N|N|N|2444026|2444176|2443665|2443940|N|N|N|N|N| +2444031|AAAAAAAAPPKEFCAA|1979-06-06|953|4145|319|1979|3|6|6|2|1979|319|4145|Wednesday|1979Q2|N|N|N|2444026|2444176|2443666|2443941|N|N|N|N|N| +2444032|AAAAAAAAAALEFCAA|1979-06-07|953|4145|319|1979|4|6|7|2|1979|319|4145|Thursday|1979Q2|N|N|N|2444026|2444176|2443667|2443942|N|N|N|N|N| +2444033|AAAAAAAABALEFCAA|1979-06-08|953|4145|319|1979|5|6|8|2|1979|319|4145|Friday|1979Q2|N|Y|N|2444026|2444176|2443668|2443943|N|N|N|N|N| +2444034|AAAAAAAACALEFCAA|1979-06-09|953|4145|319|1979|6|6|9|2|1979|319|4145|Saturday|1979Q2|N|Y|N|2444026|2444176|2443669|2443944|N|N|N|N|N| +2444035|AAAAAAAADALEFCAA|1979-06-10|953|4145|319|1979|0|6|10|2|1979|319|4145|Sunday|1979Q2|N|N|N|2444026|2444176|2443670|2443945|N|N|N|N|N| +2444036|AAAAAAAAEALEFCAA|1979-06-11|953|4145|319|1979|1|6|11|2|1979|319|4145|Monday|1979Q2|N|N|N|2444026|2444176|2443671|2443946|N|N|N|N|N| +2444037|AAAAAAAAFALEFCAA|1979-06-12|953|4146|319|1979|2|6|12|2|1979|319|4146|Tuesday|1979Q2|N|N|N|2444026|2444176|2443672|2443947|N|N|N|N|N| +2444038|AAAAAAAAGALEFCAA|1979-06-13|953|4146|319|1979|3|6|13|2|1979|319|4146|Wednesday|1979Q2|N|N|N|2444026|2444176|2443673|2443948|N|N|N|N|N| +2444039|AAAAAAAAHALEFCAA|1979-06-14|953|4146|319|1979|4|6|14|2|1979|319|4146|Thursday|1979Q2|N|N|N|2444026|2444176|2443674|2443949|N|N|N|N|N| +2444040|AAAAAAAAIALEFCAA|1979-06-15|953|4146|319|1979|5|6|15|2|1979|319|4146|Friday|1979Q2|N|Y|N|2444026|2444176|2443675|2443950|N|N|N|N|N| +2444041|AAAAAAAAJALEFCAA|1979-06-16|953|4146|319|1979|6|6|16|2|1979|319|4146|Saturday|1979Q2|N|Y|N|2444026|2444176|2443676|2443951|N|N|N|N|N| +2444042|AAAAAAAAKALEFCAA|1979-06-17|953|4146|319|1979|0|6|17|2|1979|319|4146|Sunday|1979Q2|N|N|N|2444026|2444176|2443677|2443952|N|N|N|N|N| +2444043|AAAAAAAALALEFCAA|1979-06-18|953|4146|319|1979|1|6|18|2|1979|319|4146|Monday|1979Q2|N|N|N|2444026|2444176|2443678|2443953|N|N|N|N|N| +2444044|AAAAAAAAMALEFCAA|1979-06-19|953|4147|319|1979|2|6|19|2|1979|319|4147|Tuesday|1979Q2|N|N|N|2444026|2444176|2443679|2443954|N|N|N|N|N| +2444045|AAAAAAAANALEFCAA|1979-06-20|953|4147|319|1979|3|6|20|2|1979|319|4147|Wednesday|1979Q2|N|N|N|2444026|2444176|2443680|2443955|N|N|N|N|N| +2444046|AAAAAAAAOALEFCAA|1979-06-21|953|4147|319|1979|4|6|21|2|1979|319|4147|Thursday|1979Q2|N|N|N|2444026|2444176|2443681|2443956|N|N|N|N|N| +2444047|AAAAAAAAPALEFCAA|1979-06-22|953|4147|319|1979|5|6|22|2|1979|319|4147|Friday|1979Q2|N|Y|N|2444026|2444176|2443682|2443957|N|N|N|N|N| +2444048|AAAAAAAAABLEFCAA|1979-06-23|953|4147|319|1979|6|6|23|2|1979|319|4147|Saturday|1979Q2|N|Y|N|2444026|2444176|2443683|2443958|N|N|N|N|N| +2444049|AAAAAAAABBLEFCAA|1979-06-24|953|4147|319|1979|0|6|24|2|1979|319|4147|Sunday|1979Q2|N|N|N|2444026|2444176|2443684|2443959|N|N|N|N|N| +2444050|AAAAAAAACBLEFCAA|1979-06-25|953|4147|319|1979|1|6|25|2|1979|319|4147|Monday|1979Q2|N|N|N|2444026|2444176|2443685|2443960|N|N|N|N|N| +2444051|AAAAAAAADBLEFCAA|1979-06-26|953|4148|319|1979|2|6|26|2|1979|319|4148|Tuesday|1979Q2|N|N|N|2444026|2444176|2443686|2443961|N|N|N|N|N| +2444052|AAAAAAAAEBLEFCAA|1979-06-27|953|4148|319|1979|3|6|27|2|1979|319|4148|Wednesday|1979Q2|N|N|N|2444026|2444176|2443687|2443962|N|N|N|N|N| +2444053|AAAAAAAAFBLEFCAA|1979-06-28|953|4148|319|1979|4|6|28|2|1979|319|4148|Thursday|1979Q2|N|N|N|2444026|2444176|2443688|2443963|N|N|N|N|N| +2444054|AAAAAAAAGBLEFCAA|1979-06-29|953|4148|319|1979|5|6|29|2|1979|319|4148|Friday|1979Q2|N|Y|N|2444026|2444176|2443689|2443964|N|N|N|N|N| +2444055|AAAAAAAAHBLEFCAA|1979-06-30|953|4148|319|1979|6|6|30|2|1979|319|4148|Saturday|1979Q2|N|Y|N|2444026|2444176|2443690|2443965|N|N|N|N|N| +2444056|AAAAAAAAIBLEFCAA|1979-07-01|954|4148|319|1979|0|7|1|2|1979|319|4148|Sunday|1979Q2|N|N|N|2444056|2444236|2443691|2443965|N|N|N|N|N| +2444057|AAAAAAAAJBLEFCAA|1979-07-02|954|4148|319|1979|1|7|2|3|1979|319|4148|Monday|1979Q3|N|N|N|2444056|2444236|2443692|2443966|N|N|N|N|N| +2444058|AAAAAAAAKBLEFCAA|1979-07-03|954|4149|319|1979|2|7|3|3|1979|319|4149|Tuesday|1979Q3|N|N|N|2444056|2444236|2443693|2443967|N|N|N|N|N| +2444059|AAAAAAAALBLEFCAA|1979-07-04|954|4149|319|1979|3|7|4|3|1979|319|4149|Wednesday|1979Q3|N|N|N|2444056|2444236|2443694|2443968|N|N|N|N|N| +2444060|AAAAAAAAMBLEFCAA|1979-07-05|954|4149|319|1979|4|7|5|3|1979|319|4149|Thursday|1979Q3|Y|N|N|2444056|2444236|2443695|2443969|N|N|N|N|N| +2444061|AAAAAAAANBLEFCAA|1979-07-06|954|4149|319|1979|5|7|6|3|1979|319|4149|Friday|1979Q3|N|Y|Y|2444056|2444236|2443696|2443970|N|N|N|N|N| +2444062|AAAAAAAAOBLEFCAA|1979-07-07|954|4149|319|1979|6|7|7|3|1979|319|4149|Saturday|1979Q3|N|Y|N|2444056|2444236|2443697|2443971|N|N|N|N|N| +2444063|AAAAAAAAPBLEFCAA|1979-07-08|954|4149|319|1979|0|7|8|3|1979|319|4149|Sunday|1979Q3|N|N|N|2444056|2444236|2443698|2443972|N|N|N|N|N| +2444064|AAAAAAAAACLEFCAA|1979-07-09|954|4149|319|1979|1|7|9|3|1979|319|4149|Monday|1979Q3|N|N|N|2444056|2444236|2443699|2443973|N|N|N|N|N| +2444065|AAAAAAAABCLEFCAA|1979-07-10|954|4150|319|1979|2|7|10|3|1979|319|4150|Tuesday|1979Q3|N|N|N|2444056|2444236|2443700|2443974|N|N|N|N|N| +2444066|AAAAAAAACCLEFCAA|1979-07-11|954|4150|319|1979|3|7|11|3|1979|319|4150|Wednesday|1979Q3|N|N|N|2444056|2444236|2443701|2443975|N|N|N|N|N| +2444067|AAAAAAAADCLEFCAA|1979-07-12|954|4150|319|1979|4|7|12|3|1979|319|4150|Thursday|1979Q3|N|N|N|2444056|2444236|2443702|2443976|N|N|N|N|N| +2444068|AAAAAAAAECLEFCAA|1979-07-13|954|4150|319|1979|5|7|13|3|1979|319|4150|Friday|1979Q3|N|Y|N|2444056|2444236|2443703|2443977|N|N|N|N|N| +2444069|AAAAAAAAFCLEFCAA|1979-07-14|954|4150|319|1979|6|7|14|3|1979|319|4150|Saturday|1979Q3|N|Y|N|2444056|2444236|2443704|2443978|N|N|N|N|N| +2444070|AAAAAAAAGCLEFCAA|1979-07-15|954|4150|319|1979|0|7|15|3|1979|319|4150|Sunday|1979Q3|N|N|N|2444056|2444236|2443705|2443979|N|N|N|N|N| +2444071|AAAAAAAAHCLEFCAA|1979-07-16|954|4150|319|1979|1|7|16|3|1979|319|4150|Monday|1979Q3|N|N|N|2444056|2444236|2443706|2443980|N|N|N|N|N| +2444072|AAAAAAAAICLEFCAA|1979-07-17|954|4151|319|1979|2|7|17|3|1979|319|4151|Tuesday|1979Q3|N|N|N|2444056|2444236|2443707|2443981|N|N|N|N|N| +2444073|AAAAAAAAJCLEFCAA|1979-07-18|954|4151|319|1979|3|7|18|3|1979|319|4151|Wednesday|1979Q3|N|N|N|2444056|2444236|2443708|2443982|N|N|N|N|N| +2444074|AAAAAAAAKCLEFCAA|1979-07-19|954|4151|319|1979|4|7|19|3|1979|319|4151|Thursday|1979Q3|N|N|N|2444056|2444236|2443709|2443983|N|N|N|N|N| +2444075|AAAAAAAALCLEFCAA|1979-07-20|954|4151|319|1979|5|7|20|3|1979|319|4151|Friday|1979Q3|N|Y|N|2444056|2444236|2443710|2443984|N|N|N|N|N| +2444076|AAAAAAAAMCLEFCAA|1979-07-21|954|4151|319|1979|6|7|21|3|1979|319|4151|Saturday|1979Q3|N|Y|N|2444056|2444236|2443711|2443985|N|N|N|N|N| +2444077|AAAAAAAANCLEFCAA|1979-07-22|954|4151|319|1979|0|7|22|3|1979|319|4151|Sunday|1979Q3|N|N|N|2444056|2444236|2443712|2443986|N|N|N|N|N| +2444078|AAAAAAAAOCLEFCAA|1979-07-23|954|4151|319|1979|1|7|23|3|1979|319|4151|Monday|1979Q3|N|N|N|2444056|2444236|2443713|2443987|N|N|N|N|N| +2444079|AAAAAAAAPCLEFCAA|1979-07-24|954|4152|319|1979|2|7|24|3|1979|319|4152|Tuesday|1979Q3|N|N|N|2444056|2444236|2443714|2443988|N|N|N|N|N| +2444080|AAAAAAAAADLEFCAA|1979-07-25|954|4152|319|1979|3|7|25|3|1979|319|4152|Wednesday|1979Q3|N|N|N|2444056|2444236|2443715|2443989|N|N|N|N|N| +2444081|AAAAAAAABDLEFCAA|1979-07-26|954|4152|319|1979|4|7|26|3|1979|319|4152|Thursday|1979Q3|N|N|N|2444056|2444236|2443716|2443990|N|N|N|N|N| +2444082|AAAAAAAACDLEFCAA|1979-07-27|954|4152|319|1979|5|7|27|3|1979|319|4152|Friday|1979Q3|N|Y|N|2444056|2444236|2443717|2443991|N|N|N|N|N| +2444083|AAAAAAAADDLEFCAA|1979-07-28|954|4152|319|1979|6|7|28|3|1979|319|4152|Saturday|1979Q3|N|Y|N|2444056|2444236|2443718|2443992|N|N|N|N|N| +2444084|AAAAAAAAEDLEFCAA|1979-07-29|954|4152|319|1979|0|7|29|3|1979|319|4152|Sunday|1979Q3|N|N|N|2444056|2444236|2443719|2443993|N|N|N|N|N| +2444085|AAAAAAAAFDLEFCAA|1979-07-30|954|4152|319|1979|1|7|30|3|1979|319|4152|Monday|1979Q3|N|N|N|2444056|2444236|2443720|2443994|N|N|N|N|N| +2444086|AAAAAAAAGDLEFCAA|1979-07-31|954|4153|319|1979|2|7|31|3|1979|319|4153|Tuesday|1979Q3|N|N|N|2444056|2444236|2443721|2443995|N|N|N|N|N| +2444087|AAAAAAAAHDLEFCAA|1979-08-01|955|4153|319|1979|3|8|1|3|1979|319|4153|Wednesday|1979Q3|N|N|N|2444087|2444298|2443722|2443996|N|N|N|N|N| +2444088|AAAAAAAAIDLEFCAA|1979-08-02|955|4153|319|1979|4|8|2|3|1979|319|4153|Thursday|1979Q3|N|N|N|2444087|2444298|2443723|2443997|N|N|N|N|N| +2444089|AAAAAAAAJDLEFCAA|1979-08-03|955|4153|319|1979|5|8|3|3|1979|319|4153|Friday|1979Q3|N|Y|N|2444087|2444298|2443724|2443998|N|N|N|N|N| +2444090|AAAAAAAAKDLEFCAA|1979-08-04|955|4153|319|1979|6|8|4|3|1979|319|4153|Saturday|1979Q3|N|Y|N|2444087|2444298|2443725|2443999|N|N|N|N|N| +2444091|AAAAAAAALDLEFCAA|1979-08-05|955|4153|319|1979|0|8|5|3|1979|319|4153|Sunday|1979Q3|N|N|N|2444087|2444298|2443726|2444000|N|N|N|N|N| +2444092|AAAAAAAAMDLEFCAA|1979-08-06|955|4153|319|1979|1|8|6|3|1979|319|4153|Monday|1979Q3|N|N|N|2444087|2444298|2443727|2444001|N|N|N|N|N| +2444093|AAAAAAAANDLEFCAA|1979-08-07|955|4154|319|1979|2|8|7|3|1979|319|4154|Tuesday|1979Q3|N|N|N|2444087|2444298|2443728|2444002|N|N|N|N|N| +2444094|AAAAAAAAODLEFCAA|1979-08-08|955|4154|319|1979|3|8|8|3|1979|319|4154|Wednesday|1979Q3|N|N|N|2444087|2444298|2443729|2444003|N|N|N|N|N| +2444095|AAAAAAAAPDLEFCAA|1979-08-09|955|4154|319|1979|4|8|9|3|1979|319|4154|Thursday|1979Q3|N|N|N|2444087|2444298|2443730|2444004|N|N|N|N|N| +2444096|AAAAAAAAAELEFCAA|1979-08-10|955|4154|319|1979|5|8|10|3|1979|319|4154|Friday|1979Q3|N|Y|N|2444087|2444298|2443731|2444005|N|N|N|N|N| +2444097|AAAAAAAABELEFCAA|1979-08-11|955|4154|319|1979|6|8|11|3|1979|319|4154|Saturday|1979Q3|N|Y|N|2444087|2444298|2443732|2444006|N|N|N|N|N| +2444098|AAAAAAAACELEFCAA|1979-08-12|955|4154|319|1979|0|8|12|3|1979|319|4154|Sunday|1979Q3|N|N|N|2444087|2444298|2443733|2444007|N|N|N|N|N| +2444099|AAAAAAAADELEFCAA|1979-08-13|955|4154|319|1979|1|8|13|3|1979|319|4154|Monday|1979Q3|N|N|N|2444087|2444298|2443734|2444008|N|N|N|N|N| +2444100|AAAAAAAAEELEFCAA|1979-08-14|955|4155|319|1979|2|8|14|3|1979|319|4155|Tuesday|1979Q3|N|N|N|2444087|2444298|2443735|2444009|N|N|N|N|N| +2444101|AAAAAAAAFELEFCAA|1979-08-15|955|4155|319|1979|3|8|15|3|1979|319|4155|Wednesday|1979Q3|N|N|N|2444087|2444298|2443736|2444010|N|N|N|N|N| +2444102|AAAAAAAAGELEFCAA|1979-08-16|955|4155|319|1979|4|8|16|3|1979|319|4155|Thursday|1979Q3|N|N|N|2444087|2444298|2443737|2444011|N|N|N|N|N| +2444103|AAAAAAAAHELEFCAA|1979-08-17|955|4155|319|1979|5|8|17|3|1979|319|4155|Friday|1979Q3|N|Y|N|2444087|2444298|2443738|2444012|N|N|N|N|N| +2444104|AAAAAAAAIELEFCAA|1979-08-18|955|4155|319|1979|6|8|18|3|1979|319|4155|Saturday|1979Q3|N|Y|N|2444087|2444298|2443739|2444013|N|N|N|N|N| +2444105|AAAAAAAAJELEFCAA|1979-08-19|955|4155|319|1979|0|8|19|3|1979|319|4155|Sunday|1979Q3|N|N|N|2444087|2444298|2443740|2444014|N|N|N|N|N| +2444106|AAAAAAAAKELEFCAA|1979-08-20|955|4155|319|1979|1|8|20|3|1979|319|4155|Monday|1979Q3|N|N|N|2444087|2444298|2443741|2444015|N|N|N|N|N| +2444107|AAAAAAAALELEFCAA|1979-08-21|955|4156|319|1979|2|8|21|3|1979|319|4156|Tuesday|1979Q3|N|N|N|2444087|2444298|2443742|2444016|N|N|N|N|N| +2444108|AAAAAAAAMELEFCAA|1979-08-22|955|4156|319|1979|3|8|22|3|1979|319|4156|Wednesday|1979Q3|N|N|N|2444087|2444298|2443743|2444017|N|N|N|N|N| +2444109|AAAAAAAANELEFCAA|1979-08-23|955|4156|319|1979|4|8|23|3|1979|319|4156|Thursday|1979Q3|N|N|N|2444087|2444298|2443744|2444018|N|N|N|N|N| +2444110|AAAAAAAAOELEFCAA|1979-08-24|955|4156|319|1979|5|8|24|3|1979|319|4156|Friday|1979Q3|N|Y|N|2444087|2444298|2443745|2444019|N|N|N|N|N| +2444111|AAAAAAAAPELEFCAA|1979-08-25|955|4156|319|1979|6|8|25|3|1979|319|4156|Saturday|1979Q3|N|Y|N|2444087|2444298|2443746|2444020|N|N|N|N|N| +2444112|AAAAAAAAAFLEFCAA|1979-08-26|955|4156|319|1979|0|8|26|3|1979|319|4156|Sunday|1979Q3|N|N|N|2444087|2444298|2443747|2444021|N|N|N|N|N| +2444113|AAAAAAAABFLEFCAA|1979-08-27|955|4156|319|1979|1|8|27|3|1979|319|4156|Monday|1979Q3|N|N|N|2444087|2444298|2443748|2444022|N|N|N|N|N| +2444114|AAAAAAAACFLEFCAA|1979-08-28|955|4157|319|1979|2|8|28|3|1979|319|4157|Tuesday|1979Q3|N|N|N|2444087|2444298|2443749|2444023|N|N|N|N|N| +2444115|AAAAAAAADFLEFCAA|1979-08-29|955|4157|319|1979|3|8|29|3|1979|319|4157|Wednesday|1979Q3|N|N|N|2444087|2444298|2443750|2444024|N|N|N|N|N| +2444116|AAAAAAAAEFLEFCAA|1979-08-30|955|4157|319|1979|4|8|30|3|1979|319|4157|Thursday|1979Q3|N|N|N|2444087|2444298|2443751|2444025|N|N|N|N|N| +2444117|AAAAAAAAFFLEFCAA|1979-08-31|955|4157|319|1979|5|8|31|3|1979|319|4157|Friday|1979Q3|N|Y|N|2444087|2444298|2443752|2444026|N|N|N|N|N| +2444118|AAAAAAAAGFLEFCAA|1979-09-01|956|4157|320|1979|6|9|1|3|1979|320|4157|Saturday|1979Q3|N|Y|N|2444118|2444360|2443753|2444027|N|N|N|N|N| +2444119|AAAAAAAAHFLEFCAA|1979-09-02|956|4157|320|1979|0|9|2|3|1979|320|4157|Sunday|1979Q3|N|N|N|2444118|2444360|2443754|2444028|N|N|N|N|N| +2444120|AAAAAAAAIFLEFCAA|1979-09-03|956|4157|320|1979|1|9|3|3|1979|320|4157|Monday|1979Q3|N|N|N|2444118|2444360|2443755|2444029|N|N|N|N|N| +2444121|AAAAAAAAJFLEFCAA|1979-09-04|956|4158|320|1979|2|9|4|3|1979|320|4158|Tuesday|1979Q3|N|N|N|2444118|2444360|2443756|2444030|N|N|N|N|N| +2444122|AAAAAAAAKFLEFCAA|1979-09-05|956|4158|320|1979|3|9|5|3|1979|320|4158|Wednesday|1979Q3|N|N|N|2444118|2444360|2443757|2444031|N|N|N|N|N| +2444123|AAAAAAAALFLEFCAA|1979-09-06|956|4158|320|1979|4|9|6|3|1979|320|4158|Thursday|1979Q3|N|N|N|2444118|2444360|2443758|2444032|N|N|N|N|N| +2444124|AAAAAAAAMFLEFCAA|1979-09-07|956|4158|320|1979|5|9|7|3|1979|320|4158|Friday|1979Q3|N|Y|N|2444118|2444360|2443759|2444033|N|N|N|N|N| +2444125|AAAAAAAANFLEFCAA|1979-09-08|956|4158|320|1979|6|9|8|3|1979|320|4158|Saturday|1979Q3|N|Y|N|2444118|2444360|2443760|2444034|N|N|N|N|N| +2444126|AAAAAAAAOFLEFCAA|1979-09-09|956|4158|320|1979|0|9|9|3|1979|320|4158|Sunday|1979Q3|N|N|N|2444118|2444360|2443761|2444035|N|N|N|N|N| +2444127|AAAAAAAAPFLEFCAA|1979-09-10|956|4158|320|1979|1|9|10|3|1979|320|4158|Monday|1979Q3|N|N|N|2444118|2444360|2443762|2444036|N|N|N|N|N| +2444128|AAAAAAAAAGLEFCAA|1979-09-11|956|4159|320|1979|2|9|11|3|1979|320|4159|Tuesday|1979Q3|N|N|N|2444118|2444360|2443763|2444037|N|N|N|N|N| +2444129|AAAAAAAABGLEFCAA|1979-09-12|956|4159|320|1979|3|9|12|3|1979|320|4159|Wednesday|1979Q3|N|N|N|2444118|2444360|2443764|2444038|N|N|N|N|N| +2444130|AAAAAAAACGLEFCAA|1979-09-13|956|4159|320|1979|4|9|13|3|1979|320|4159|Thursday|1979Q3|N|N|N|2444118|2444360|2443765|2444039|N|N|N|N|N| +2444131|AAAAAAAADGLEFCAA|1979-09-14|956|4159|320|1979|5|9|14|3|1979|320|4159|Friday|1979Q3|N|Y|N|2444118|2444360|2443766|2444040|N|N|N|N|N| +2444132|AAAAAAAAEGLEFCAA|1979-09-15|956|4159|320|1979|6|9|15|3|1979|320|4159|Saturday|1979Q3|N|Y|N|2444118|2444360|2443767|2444041|N|N|N|N|N| +2444133|AAAAAAAAFGLEFCAA|1979-09-16|956|4159|320|1979|0|9|16|3|1979|320|4159|Sunday|1979Q3|N|N|N|2444118|2444360|2443768|2444042|N|N|N|N|N| +2444134|AAAAAAAAGGLEFCAA|1979-09-17|956|4159|320|1979|1|9|17|3|1979|320|4159|Monday|1979Q3|N|N|N|2444118|2444360|2443769|2444043|N|N|N|N|N| +2444135|AAAAAAAAHGLEFCAA|1979-09-18|956|4160|320|1979|2|9|18|3|1979|320|4160|Tuesday|1979Q3|N|N|N|2444118|2444360|2443770|2444044|N|N|N|N|N| +2444136|AAAAAAAAIGLEFCAA|1979-09-19|956|4160|320|1979|3|9|19|3|1979|320|4160|Wednesday|1979Q3|N|N|N|2444118|2444360|2443771|2444045|N|N|N|N|N| +2444137|AAAAAAAAJGLEFCAA|1979-09-20|956|4160|320|1979|4|9|20|3|1979|320|4160|Thursday|1979Q3|N|N|N|2444118|2444360|2443772|2444046|N|N|N|N|N| +2444138|AAAAAAAAKGLEFCAA|1979-09-21|956|4160|320|1979|5|9|21|3|1979|320|4160|Friday|1979Q3|N|Y|N|2444118|2444360|2443773|2444047|N|N|N|N|N| +2444139|AAAAAAAALGLEFCAA|1979-09-22|956|4160|320|1979|6|9|22|3|1979|320|4160|Saturday|1979Q3|N|Y|N|2444118|2444360|2443774|2444048|N|N|N|N|N| +2444140|AAAAAAAAMGLEFCAA|1979-09-23|956|4160|320|1979|0|9|23|3|1979|320|4160|Sunday|1979Q3|N|N|N|2444118|2444360|2443775|2444049|N|N|N|N|N| +2444141|AAAAAAAANGLEFCAA|1979-09-24|956|4160|320|1979|1|9|24|3|1979|320|4160|Monday|1979Q3|N|N|N|2444118|2444360|2443776|2444050|N|N|N|N|N| +2444142|AAAAAAAAOGLEFCAA|1979-09-25|956|4161|320|1979|2|9|25|3|1979|320|4161|Tuesday|1979Q3|N|N|N|2444118|2444360|2443777|2444051|N|N|N|N|N| +2444143|AAAAAAAAPGLEFCAA|1979-09-26|956|4161|320|1979|3|9|26|3|1979|320|4161|Wednesday|1979Q3|N|N|N|2444118|2444360|2443778|2444052|N|N|N|N|N| +2444144|AAAAAAAAAHLEFCAA|1979-09-27|956|4161|320|1979|4|9|27|3|1979|320|4161|Thursday|1979Q3|N|N|N|2444118|2444360|2443779|2444053|N|N|N|N|N| +2444145|AAAAAAAABHLEFCAA|1979-09-28|956|4161|320|1979|5|9|28|3|1979|320|4161|Friday|1979Q3|N|Y|N|2444118|2444360|2443780|2444054|N|N|N|N|N| +2444146|AAAAAAAACHLEFCAA|1979-09-29|956|4161|320|1979|6|9|29|3|1979|320|4161|Saturday|1979Q3|N|Y|N|2444118|2444360|2443781|2444055|N|N|N|N|N| +2444147|AAAAAAAADHLEFCAA|1979-09-30|956|4161|320|1979|0|9|30|3|1979|320|4161|Sunday|1979Q3|N|N|N|2444118|2444360|2443782|2444056|N|N|N|N|N| +2444148|AAAAAAAAEHLEFCAA|1979-10-01|957|4161|320|1979|1|10|1|3|1979|320|4161|Monday|1979Q3|N|N|N|2444148|2444420|2443783|2444056|N|N|N|N|N| +2444149|AAAAAAAAFHLEFCAA|1979-10-02|957|4162|320|1979|2|10|2|4|1979|320|4162|Tuesday|1979Q4|N|N|N|2444148|2444420|2443784|2444057|N|N|N|N|N| +2444150|AAAAAAAAGHLEFCAA|1979-10-03|957|4162|320|1979|3|10|3|4|1979|320|4162|Wednesday|1979Q4|N|N|N|2444148|2444420|2443785|2444058|N|N|N|N|N| +2444151|AAAAAAAAHHLEFCAA|1979-10-04|957|4162|320|1979|4|10|4|4|1979|320|4162|Thursday|1979Q4|N|N|N|2444148|2444420|2443786|2444059|N|N|N|N|N| +2444152|AAAAAAAAIHLEFCAA|1979-10-05|957|4162|320|1979|5|10|5|4|1979|320|4162|Friday|1979Q4|N|Y|N|2444148|2444420|2443787|2444060|N|N|N|N|N| +2444153|AAAAAAAAJHLEFCAA|1979-10-06|957|4162|320|1979|6|10|6|4|1979|320|4162|Saturday|1979Q4|N|Y|N|2444148|2444420|2443788|2444061|N|N|N|N|N| +2444154|AAAAAAAAKHLEFCAA|1979-10-07|957|4162|320|1979|0|10|7|4|1979|320|4162|Sunday|1979Q4|N|N|N|2444148|2444420|2443789|2444062|N|N|N|N|N| +2444155|AAAAAAAALHLEFCAA|1979-10-08|957|4162|320|1979|1|10|8|4|1979|320|4162|Monday|1979Q4|N|N|N|2444148|2444420|2443790|2444063|N|N|N|N|N| +2444156|AAAAAAAAMHLEFCAA|1979-10-09|957|4163|320|1979|2|10|9|4|1979|320|4163|Tuesday|1979Q4|N|N|N|2444148|2444420|2443791|2444064|N|N|N|N|N| +2444157|AAAAAAAANHLEFCAA|1979-10-10|957|4163|320|1979|3|10|10|4|1979|320|4163|Wednesday|1979Q4|N|N|N|2444148|2444420|2443792|2444065|N|N|N|N|N| +2444158|AAAAAAAAOHLEFCAA|1979-10-11|957|4163|320|1979|4|10|11|4|1979|320|4163|Thursday|1979Q4|N|N|N|2444148|2444420|2443793|2444066|N|N|N|N|N| +2444159|AAAAAAAAPHLEFCAA|1979-10-12|957|4163|320|1979|5|10|12|4|1979|320|4163|Friday|1979Q4|N|Y|N|2444148|2444420|2443794|2444067|N|N|N|N|N| +2444160|AAAAAAAAAILEFCAA|1979-10-13|957|4163|320|1979|6|10|13|4|1979|320|4163|Saturday|1979Q4|N|Y|N|2444148|2444420|2443795|2444068|N|N|N|N|N| +2444161|AAAAAAAABILEFCAA|1979-10-14|957|4163|320|1979|0|10|14|4|1979|320|4163|Sunday|1979Q4|N|N|N|2444148|2444420|2443796|2444069|N|N|N|N|N| +2444162|AAAAAAAACILEFCAA|1979-10-15|957|4163|320|1979|1|10|15|4|1979|320|4163|Monday|1979Q4|N|N|N|2444148|2444420|2443797|2444070|N|N|N|N|N| +2444163|AAAAAAAADILEFCAA|1979-10-16|957|4164|320|1979|2|10|16|4|1979|320|4164|Tuesday|1979Q4|N|N|N|2444148|2444420|2443798|2444071|N|N|N|N|N| +2444164|AAAAAAAAEILEFCAA|1979-10-17|957|4164|320|1979|3|10|17|4|1979|320|4164|Wednesday|1979Q4|N|N|N|2444148|2444420|2443799|2444072|N|N|N|N|N| +2444165|AAAAAAAAFILEFCAA|1979-10-18|957|4164|320|1979|4|10|18|4|1979|320|4164|Thursday|1979Q4|N|N|N|2444148|2444420|2443800|2444073|N|N|N|N|N| +2444166|AAAAAAAAGILEFCAA|1979-10-19|957|4164|320|1979|5|10|19|4|1979|320|4164|Friday|1979Q4|N|Y|N|2444148|2444420|2443801|2444074|N|N|N|N|N| +2444167|AAAAAAAAHILEFCAA|1979-10-20|957|4164|320|1979|6|10|20|4|1979|320|4164|Saturday|1979Q4|N|Y|N|2444148|2444420|2443802|2444075|N|N|N|N|N| +2444168|AAAAAAAAIILEFCAA|1979-10-21|957|4164|320|1979|0|10|21|4|1979|320|4164|Sunday|1979Q4|N|N|N|2444148|2444420|2443803|2444076|N|N|N|N|N| +2444169|AAAAAAAAJILEFCAA|1979-10-22|957|4164|320|1979|1|10|22|4|1979|320|4164|Monday|1979Q4|N|N|N|2444148|2444420|2443804|2444077|N|N|N|N|N| +2444170|AAAAAAAAKILEFCAA|1979-10-23|957|4165|320|1979|2|10|23|4|1979|320|4165|Tuesday|1979Q4|N|N|N|2444148|2444420|2443805|2444078|N|N|N|N|N| +2444171|AAAAAAAALILEFCAA|1979-10-24|957|4165|320|1979|3|10|24|4|1979|320|4165|Wednesday|1979Q4|N|N|N|2444148|2444420|2443806|2444079|N|N|N|N|N| +2444172|AAAAAAAAMILEFCAA|1979-10-25|957|4165|320|1979|4|10|25|4|1979|320|4165|Thursday|1979Q4|N|N|N|2444148|2444420|2443807|2444080|N|N|N|N|N| +2444173|AAAAAAAANILEFCAA|1979-10-26|957|4165|320|1979|5|10|26|4|1979|320|4165|Friday|1979Q4|N|Y|N|2444148|2444420|2443808|2444081|N|N|N|N|N| +2444174|AAAAAAAAOILEFCAA|1979-10-27|957|4165|320|1979|6|10|27|4|1979|320|4165|Saturday|1979Q4|N|Y|N|2444148|2444420|2443809|2444082|N|N|N|N|N| +2444175|AAAAAAAAPILEFCAA|1979-10-28|957|4165|320|1979|0|10|28|4|1979|320|4165|Sunday|1979Q4|N|N|N|2444148|2444420|2443810|2444083|N|N|N|N|N| +2444176|AAAAAAAAAJLEFCAA|1979-10-29|957|4165|320|1979|1|10|29|4|1979|320|4165|Monday|1979Q4|N|N|N|2444148|2444420|2443811|2444084|N|N|N|N|N| +2444177|AAAAAAAABJLEFCAA|1979-10-30|957|4166|320|1979|2|10|30|4|1979|320|4166|Tuesday|1979Q4|N|N|N|2444148|2444420|2443812|2444085|N|N|N|N|N| +2444178|AAAAAAAACJLEFCAA|1979-10-31|957|4166|320|1979|3|10|31|4|1979|320|4166|Wednesday|1979Q4|N|N|N|2444148|2444420|2443813|2444086|N|N|N|N|N| +2444179|AAAAAAAADJLEFCAA|1979-11-01|958|4166|320|1979|4|11|1|4|1979|320|4166|Thursday|1979Q4|N|N|N|2444179|2444482|2443814|2444087|N|N|N|N|N| +2444180|AAAAAAAAEJLEFCAA|1979-11-02|958|4166|320|1979|5|11|2|4|1979|320|4166|Friday|1979Q4|N|Y|N|2444179|2444482|2443815|2444088|N|N|N|N|N| +2444181|AAAAAAAAFJLEFCAA|1979-11-03|958|4166|320|1979|6|11|3|4|1979|320|4166|Saturday|1979Q4|N|Y|N|2444179|2444482|2443816|2444089|N|N|N|N|N| +2444182|AAAAAAAAGJLEFCAA|1979-11-04|958|4166|320|1979|0|11|4|4|1979|320|4166|Sunday|1979Q4|N|N|N|2444179|2444482|2443817|2444090|N|N|N|N|N| +2444183|AAAAAAAAHJLEFCAA|1979-11-05|958|4166|320|1979|1|11|5|4|1979|320|4166|Monday|1979Q4|N|N|N|2444179|2444482|2443818|2444091|N|N|N|N|N| +2444184|AAAAAAAAIJLEFCAA|1979-11-06|958|4167|320|1979|2|11|6|4|1979|320|4167|Tuesday|1979Q4|N|N|N|2444179|2444482|2443819|2444092|N|N|N|N|N| +2444185|AAAAAAAAJJLEFCAA|1979-11-07|958|4167|320|1979|3|11|7|4|1979|320|4167|Wednesday|1979Q4|N|N|N|2444179|2444482|2443820|2444093|N|N|N|N|N| +2444186|AAAAAAAAKJLEFCAA|1979-11-08|958|4167|320|1979|4|11|8|4|1979|320|4167|Thursday|1979Q4|N|N|N|2444179|2444482|2443821|2444094|N|N|N|N|N| +2444187|AAAAAAAALJLEFCAA|1979-11-09|958|4167|320|1979|5|11|9|4|1979|320|4167|Friday|1979Q4|N|Y|N|2444179|2444482|2443822|2444095|N|N|N|N|N| +2444188|AAAAAAAAMJLEFCAA|1979-11-10|958|4167|320|1979|6|11|10|4|1979|320|4167|Saturday|1979Q4|N|Y|N|2444179|2444482|2443823|2444096|N|N|N|N|N| +2444189|AAAAAAAANJLEFCAA|1979-11-11|958|4167|320|1979|0|11|11|4|1979|320|4167|Sunday|1979Q4|N|N|N|2444179|2444482|2443824|2444097|N|N|N|N|N| +2444190|AAAAAAAAOJLEFCAA|1979-11-12|958|4167|320|1979|1|11|12|4|1979|320|4167|Monday|1979Q4|N|N|N|2444179|2444482|2443825|2444098|N|N|N|N|N| +2444191|AAAAAAAAPJLEFCAA|1979-11-13|958|4168|320|1979|2|11|13|4|1979|320|4168|Tuesday|1979Q4|N|N|N|2444179|2444482|2443826|2444099|N|N|N|N|N| +2444192|AAAAAAAAAKLEFCAA|1979-11-14|958|4168|320|1979|3|11|14|4|1979|320|4168|Wednesday|1979Q4|N|N|N|2444179|2444482|2443827|2444100|N|N|N|N|N| +2444193|AAAAAAAABKLEFCAA|1979-11-15|958|4168|320|1979|4|11|15|4|1979|320|4168|Thursday|1979Q4|N|N|N|2444179|2444482|2443828|2444101|N|N|N|N|N| +2444194|AAAAAAAACKLEFCAA|1979-11-16|958|4168|320|1979|5|11|16|4|1979|320|4168|Friday|1979Q4|N|Y|N|2444179|2444482|2443829|2444102|N|N|N|N|N| +2444195|AAAAAAAADKLEFCAA|1979-11-17|958|4168|320|1979|6|11|17|4|1979|320|4168|Saturday|1979Q4|N|Y|N|2444179|2444482|2443830|2444103|N|N|N|N|N| +2444196|AAAAAAAAEKLEFCAA|1979-11-18|958|4168|320|1979|0|11|18|4|1979|320|4168|Sunday|1979Q4|N|N|N|2444179|2444482|2443831|2444104|N|N|N|N|N| +2444197|AAAAAAAAFKLEFCAA|1979-11-19|958|4168|320|1979|1|11|19|4|1979|320|4168|Monday|1979Q4|N|N|N|2444179|2444482|2443832|2444105|N|N|N|N|N| +2444198|AAAAAAAAGKLEFCAA|1979-11-20|958|4169|320|1979|2|11|20|4|1979|320|4169|Tuesday|1979Q4|N|N|N|2444179|2444482|2443833|2444106|N|N|N|N|N| +2444199|AAAAAAAAHKLEFCAA|1979-11-21|958|4169|320|1979|3|11|21|4|1979|320|4169|Wednesday|1979Q4|N|N|N|2444179|2444482|2443834|2444107|N|N|N|N|N| +2444200|AAAAAAAAIKLEFCAA|1979-11-22|958|4169|320|1979|4|11|22|4|1979|320|4169|Thursday|1979Q4|N|N|N|2444179|2444482|2443835|2444108|N|N|N|N|N| +2444201|AAAAAAAAJKLEFCAA|1979-11-23|958|4169|320|1979|5|11|23|4|1979|320|4169|Friday|1979Q4|N|Y|N|2444179|2444482|2443836|2444109|N|N|N|N|N| +2444202|AAAAAAAAKKLEFCAA|1979-11-24|958|4169|320|1979|6|11|24|4|1979|320|4169|Saturday|1979Q4|N|Y|N|2444179|2444482|2443837|2444110|N|N|N|N|N| +2444203|AAAAAAAALKLEFCAA|1979-11-25|958|4169|320|1979|0|11|25|4|1979|320|4169|Sunday|1979Q4|N|N|N|2444179|2444482|2443838|2444111|N|N|N|N|N| +2444204|AAAAAAAAMKLEFCAA|1979-11-26|958|4169|320|1979|1|11|26|4|1979|320|4169|Monday|1979Q4|N|N|N|2444179|2444482|2443839|2444112|N|N|N|N|N| +2444205|AAAAAAAANKLEFCAA|1979-11-27|958|4170|320|1979|2|11|27|4|1979|320|4170|Tuesday|1979Q4|N|N|N|2444179|2444482|2443840|2444113|N|N|N|N|N| +2444206|AAAAAAAAOKLEFCAA|1979-11-28|958|4170|320|1979|3|11|28|4|1979|320|4170|Wednesday|1979Q4|N|N|N|2444179|2444482|2443841|2444114|N|N|N|N|N| +2444207|AAAAAAAAPKLEFCAA|1979-11-29|958|4170|320|1979|4|11|29|4|1979|320|4170|Thursday|1979Q4|N|N|N|2444179|2444482|2443842|2444115|N|N|N|N|N| +2444208|AAAAAAAAALLEFCAA|1979-11-30|958|4170|320|1979|5|11|30|4|1979|320|4170|Friday|1979Q4|N|Y|N|2444179|2444482|2443843|2444116|N|N|N|N|N| +2444209|AAAAAAAABLLEFCAA|1979-12-01|959|4170|321|1979|6|12|1|4|1979|321|4170|Saturday|1979Q4|N|Y|N|2444209|2444542|2443844|2444117|N|N|N|N|N| +2444210|AAAAAAAACLLEFCAA|1979-12-02|959|4170|321|1979|0|12|2|4|1979|321|4170|Sunday|1979Q4|N|N|N|2444209|2444542|2443845|2444118|N|N|N|N|N| +2444211|AAAAAAAADLLEFCAA|1979-12-03|959|4170|321|1979|1|12|3|4|1979|321|4170|Monday|1979Q4|N|N|N|2444209|2444542|2443846|2444119|N|N|N|N|N| +2444212|AAAAAAAAELLEFCAA|1979-12-04|959|4171|321|1979|2|12|4|4|1979|321|4171|Tuesday|1979Q4|N|N|N|2444209|2444542|2443847|2444120|N|N|N|N|N| +2444213|AAAAAAAAFLLEFCAA|1979-12-05|959|4171|321|1979|3|12|5|4|1979|321|4171|Wednesday|1979Q4|N|N|N|2444209|2444542|2443848|2444121|N|N|N|N|N| +2444214|AAAAAAAAGLLEFCAA|1979-12-06|959|4171|321|1979|4|12|6|4|1979|321|4171|Thursday|1979Q4|N|N|N|2444209|2444542|2443849|2444122|N|N|N|N|N| +2444215|AAAAAAAAHLLEFCAA|1979-12-07|959|4171|321|1979|5|12|7|4|1979|321|4171|Friday|1979Q4|N|Y|N|2444209|2444542|2443850|2444123|N|N|N|N|N| +2444216|AAAAAAAAILLEFCAA|1979-12-08|959|4171|321|1979|6|12|8|4|1979|321|4171|Saturday|1979Q4|N|Y|N|2444209|2444542|2443851|2444124|N|N|N|N|N| +2444217|AAAAAAAAJLLEFCAA|1979-12-09|959|4171|321|1979|0|12|9|4|1979|321|4171|Sunday|1979Q4|N|N|N|2444209|2444542|2443852|2444125|N|N|N|N|N| +2444218|AAAAAAAAKLLEFCAA|1979-12-10|959|4171|321|1979|1|12|10|4|1979|321|4171|Monday|1979Q4|N|N|N|2444209|2444542|2443853|2444126|N|N|N|N|N| +2444219|AAAAAAAALLLEFCAA|1979-12-11|959|4172|321|1979|2|12|11|4|1979|321|4172|Tuesday|1979Q4|N|N|N|2444209|2444542|2443854|2444127|N|N|N|N|N| +2444220|AAAAAAAAMLLEFCAA|1979-12-12|959|4172|321|1979|3|12|12|4|1979|321|4172|Wednesday|1979Q4|N|N|N|2444209|2444542|2443855|2444128|N|N|N|N|N| +2444221|AAAAAAAANLLEFCAA|1979-12-13|959|4172|321|1979|4|12|13|4|1979|321|4172|Thursday|1979Q4|N|N|N|2444209|2444542|2443856|2444129|N|N|N|N|N| +2444222|AAAAAAAAOLLEFCAA|1979-12-14|959|4172|321|1979|5|12|14|4|1979|321|4172|Friday|1979Q4|N|Y|N|2444209|2444542|2443857|2444130|N|N|N|N|N| +2444223|AAAAAAAAPLLEFCAA|1979-12-15|959|4172|321|1979|6|12|15|4|1979|321|4172|Saturday|1979Q4|N|Y|N|2444209|2444542|2443858|2444131|N|N|N|N|N| +2444224|AAAAAAAAAMLEFCAA|1979-12-16|959|4172|321|1979|0|12|16|4|1979|321|4172|Sunday|1979Q4|N|N|N|2444209|2444542|2443859|2444132|N|N|N|N|N| +2444225|AAAAAAAABMLEFCAA|1979-12-17|959|4172|321|1979|1|12|17|4|1979|321|4172|Monday|1979Q4|N|N|N|2444209|2444542|2443860|2444133|N|N|N|N|N| +2444226|AAAAAAAACMLEFCAA|1979-12-18|959|4173|321|1979|2|12|18|4|1979|321|4173|Tuesday|1979Q4|N|N|N|2444209|2444542|2443861|2444134|N|N|N|N|N| +2444227|AAAAAAAADMLEFCAA|1979-12-19|959|4173|321|1979|3|12|19|4|1979|321|4173|Wednesday|1979Q4|N|N|N|2444209|2444542|2443862|2444135|N|N|N|N|N| +2444228|AAAAAAAAEMLEFCAA|1979-12-20|959|4173|321|1979|4|12|20|4|1979|321|4173|Thursday|1979Q4|N|N|N|2444209|2444542|2443863|2444136|N|N|N|N|N| +2444229|AAAAAAAAFMLEFCAA|1979-12-21|959|4173|321|1979|5|12|21|4|1979|321|4173|Friday|1979Q4|N|Y|N|2444209|2444542|2443864|2444137|N|N|N|N|N| +2444230|AAAAAAAAGMLEFCAA|1979-12-22|959|4173|321|1979|6|12|22|4|1979|321|4173|Saturday|1979Q4|N|Y|N|2444209|2444542|2443865|2444138|N|N|N|N|N| +2444231|AAAAAAAAHMLEFCAA|1979-12-23|959|4173|321|1979|0|12|23|4|1979|321|4173|Sunday|1979Q4|N|N|N|2444209|2444542|2443866|2444139|N|N|N|N|N| +2444232|AAAAAAAAIMLEFCAA|1979-12-24|959|4173|321|1979|1|12|24|4|1979|321|4173|Monday|1979Q4|N|N|N|2444209|2444542|2443867|2444140|N|N|N|N|N| +2444233|AAAAAAAAJMLEFCAA|1979-12-25|959|4174|321|1979|2|12|25|4|1979|321|4174|Tuesday|1979Q4|N|N|N|2444209|2444542|2443868|2444141|N|N|N|N|N| +2444234|AAAAAAAAKMLEFCAA|1979-12-26|959|4174|321|1979|3|12|26|4|1979|321|4174|Wednesday|1979Q4|Y|N|N|2444209|2444542|2443869|2444142|N|N|N|N|N| +2444235|AAAAAAAALMLEFCAA|1979-12-27|959|4174|321|1979|4|12|27|4|1979|321|4174|Thursday|1979Q4|N|N|Y|2444209|2444542|2443870|2444143|N|N|N|N|N| +2444236|AAAAAAAAMMLEFCAA|1979-12-28|959|4174|321|1979|5|12|28|4|1979|321|4174|Friday|1979Q4|N|Y|N|2444209|2444542|2443871|2444144|N|N|N|N|N| +2444237|AAAAAAAANMLEFCAA|1979-12-29|959|4174|321|1979|6|12|29|4|1979|321|4174|Saturday|1979Q4|N|Y|N|2444209|2444542|2443872|2444145|N|N|N|N|N| +2444238|AAAAAAAAOMLEFCAA|1979-12-30|959|4174|321|1979|0|12|30|4|1979|321|4174|Sunday|1979Q4|N|N|N|2444209|2444542|2443873|2444146|N|N|N|N|N| +2444239|AAAAAAAAPMLEFCAA|1979-12-31|959|4174|321|1979|1|12|31|4|1979|321|4174|Monday|1979Q4|N|N|N|2444209|2444542|2443874|2444147|N|N|N|N|N| +2444240|AAAAAAAAANLEFCAA|1980-01-01|960|4175|321|1980|2|1|1|1|1980|321|4175|Tuesday|1980Q1|Y|N|N|2444240|2444239|2443875|2444148|N|N|N|N|N| +2444241|AAAAAAAABNLEFCAA|1980-01-02|960|4175|321|1980|3|1|2|1|1980|321|4175|Wednesday|1980Q1|N|N|Y|2444240|2444239|2443876|2444149|N|N|N|N|N| +2444242|AAAAAAAACNLEFCAA|1980-01-03|960|4175|321|1980|4|1|3|1|1980|321|4175|Thursday|1980Q1|N|N|N|2444240|2444239|2443877|2444150|N|N|N|N|N| +2444243|AAAAAAAADNLEFCAA|1980-01-04|960|4175|321|1980|5|1|4|1|1980|321|4175|Friday|1980Q1|N|Y|N|2444240|2444239|2443878|2444151|N|N|N|N|N| +2444244|AAAAAAAAENLEFCAA|1980-01-05|960|4175|321|1980|6|1|5|1|1980|321|4175|Saturday|1980Q1|N|Y|N|2444240|2444239|2443879|2444152|N|N|N|N|N| +2444245|AAAAAAAAFNLEFCAA|1980-01-06|960|4175|321|1980|0|1|6|1|1980|321|4175|Sunday|1980Q1|N|N|N|2444240|2444239|2443880|2444153|N|N|N|N|N| +2444246|AAAAAAAAGNLEFCAA|1980-01-07|960|4175|321|1980|1|1|7|1|1980|321|4175|Monday|1980Q1|N|N|N|2444240|2444239|2443881|2444154|N|N|N|N|N| +2444247|AAAAAAAAHNLEFCAA|1980-01-08|960|4176|321|1980|2|1|8|1|1980|321|4176|Tuesday|1980Q1|N|N|N|2444240|2444239|2443882|2444155|N|N|N|N|N| +2444248|AAAAAAAAINLEFCAA|1980-01-09|960|4176|321|1980|3|1|9|1|1980|321|4176|Wednesday|1980Q1|N|N|N|2444240|2444239|2443883|2444156|N|N|N|N|N| +2444249|AAAAAAAAJNLEFCAA|1980-01-10|960|4176|321|1980|4|1|10|1|1980|321|4176|Thursday|1980Q1|N|N|N|2444240|2444239|2443884|2444157|N|N|N|N|N| +2444250|AAAAAAAAKNLEFCAA|1980-01-11|960|4176|321|1980|5|1|11|1|1980|321|4176|Friday|1980Q1|N|Y|N|2444240|2444239|2443885|2444158|N|N|N|N|N| +2444251|AAAAAAAALNLEFCAA|1980-01-12|960|4176|321|1980|6|1|12|1|1980|321|4176|Saturday|1980Q1|N|Y|N|2444240|2444239|2443886|2444159|N|N|N|N|N| +2444252|AAAAAAAAMNLEFCAA|1980-01-13|960|4176|321|1980|0|1|13|1|1980|321|4176|Sunday|1980Q1|N|N|N|2444240|2444239|2443887|2444160|N|N|N|N|N| +2444253|AAAAAAAANNLEFCAA|1980-01-14|960|4176|321|1980|1|1|14|1|1980|321|4176|Monday|1980Q1|N|N|N|2444240|2444239|2443888|2444161|N|N|N|N|N| +2444254|AAAAAAAAONLEFCAA|1980-01-15|960|4177|321|1980|2|1|15|1|1980|321|4177|Tuesday|1980Q1|N|N|N|2444240|2444239|2443889|2444162|N|N|N|N|N| +2444255|AAAAAAAAPNLEFCAA|1980-01-16|960|4177|321|1980|3|1|16|1|1980|321|4177|Wednesday|1980Q1|N|N|N|2444240|2444239|2443890|2444163|N|N|N|N|N| +2444256|AAAAAAAAAOLEFCAA|1980-01-17|960|4177|321|1980|4|1|17|1|1980|321|4177|Thursday|1980Q1|N|N|N|2444240|2444239|2443891|2444164|N|N|N|N|N| +2444257|AAAAAAAABOLEFCAA|1980-01-18|960|4177|321|1980|5|1|18|1|1980|321|4177|Friday|1980Q1|N|Y|N|2444240|2444239|2443892|2444165|N|N|N|N|N| +2444258|AAAAAAAACOLEFCAA|1980-01-19|960|4177|321|1980|6|1|19|1|1980|321|4177|Saturday|1980Q1|N|Y|N|2444240|2444239|2443893|2444166|N|N|N|N|N| +2444259|AAAAAAAADOLEFCAA|1980-01-20|960|4177|321|1980|0|1|20|1|1980|321|4177|Sunday|1980Q1|N|N|N|2444240|2444239|2443894|2444167|N|N|N|N|N| +2444260|AAAAAAAAEOLEFCAA|1980-01-21|960|4177|321|1980|1|1|21|1|1980|321|4177|Monday|1980Q1|N|N|N|2444240|2444239|2443895|2444168|N|N|N|N|N| +2444261|AAAAAAAAFOLEFCAA|1980-01-22|960|4178|321|1980|2|1|22|1|1980|321|4178|Tuesday|1980Q1|N|N|N|2444240|2444239|2443896|2444169|N|N|N|N|N| +2444262|AAAAAAAAGOLEFCAA|1980-01-23|960|4178|321|1980|3|1|23|1|1980|321|4178|Wednesday|1980Q1|N|N|N|2444240|2444239|2443897|2444170|N|N|N|N|N| +2444263|AAAAAAAAHOLEFCAA|1980-01-24|960|4178|321|1980|4|1|24|1|1980|321|4178|Thursday|1980Q1|N|N|N|2444240|2444239|2443898|2444171|N|N|N|N|N| +2444264|AAAAAAAAIOLEFCAA|1980-01-25|960|4178|321|1980|5|1|25|1|1980|321|4178|Friday|1980Q1|N|Y|N|2444240|2444239|2443899|2444172|N|N|N|N|N| +2444265|AAAAAAAAJOLEFCAA|1980-01-26|960|4178|321|1980|6|1|26|1|1980|321|4178|Saturday|1980Q1|N|Y|N|2444240|2444239|2443900|2444173|N|N|N|N|N| +2444266|AAAAAAAAKOLEFCAA|1980-01-27|960|4178|321|1980|0|1|27|1|1980|321|4178|Sunday|1980Q1|N|N|N|2444240|2444239|2443901|2444174|N|N|N|N|N| +2444267|AAAAAAAALOLEFCAA|1980-01-28|960|4178|321|1980|1|1|28|1|1980|321|4178|Monday|1980Q1|N|N|N|2444240|2444239|2443902|2444175|N|N|N|N|N| +2444268|AAAAAAAAMOLEFCAA|1980-01-29|960|4179|321|1980|2|1|29|1|1980|321|4179|Tuesday|1980Q1|N|N|N|2444240|2444239|2443903|2444176|N|N|N|N|N| +2444269|AAAAAAAANOLEFCAA|1980-01-30|960|4179|321|1980|3|1|30|1|1980|321|4179|Wednesday|1980Q1|N|N|N|2444240|2444239|2443904|2444177|N|N|N|N|N| +2444270|AAAAAAAAOOLEFCAA|1980-01-31|960|4179|321|1980|4|1|31|1|1980|321|4179|Thursday|1980Q1|N|N|N|2444240|2444239|2443905|2444178|N|N|N|N|N| +2444271|AAAAAAAAPOLEFCAA|1980-02-01|961|4179|321|1980|5|2|1|1|1980|321|4179|Friday|1980Q1|N|Y|N|2444271|2444301|2443906|2444179|N|N|N|N|N| +2444272|AAAAAAAAAPLEFCAA|1980-02-02|961|4179|321|1980|6|2|2|1|1980|321|4179|Saturday|1980Q1|N|Y|N|2444271|2444301|2443907|2444180|N|N|N|N|N| +2444273|AAAAAAAABPLEFCAA|1980-02-03|961|4179|321|1980|0|2|3|1|1980|321|4179|Sunday|1980Q1|N|N|N|2444271|2444301|2443908|2444181|N|N|N|N|N| +2444274|AAAAAAAACPLEFCAA|1980-02-04|961|4179|321|1980|1|2|4|1|1980|321|4179|Monday|1980Q1|N|N|N|2444271|2444301|2443909|2444182|N|N|N|N|N| +2444275|AAAAAAAADPLEFCAA|1980-02-05|961|4180|321|1980|2|2|5|1|1980|321|4180|Tuesday|1980Q1|N|N|N|2444271|2444301|2443910|2444183|N|N|N|N|N| +2444276|AAAAAAAAEPLEFCAA|1980-02-06|961|4180|321|1980|3|2|6|1|1980|321|4180|Wednesday|1980Q1|N|N|N|2444271|2444301|2443911|2444184|N|N|N|N|N| +2444277|AAAAAAAAFPLEFCAA|1980-02-07|961|4180|321|1980|4|2|7|1|1980|321|4180|Thursday|1980Q1|N|N|N|2444271|2444301|2443912|2444185|N|N|N|N|N| +2444278|AAAAAAAAGPLEFCAA|1980-02-08|961|4180|321|1980|5|2|8|1|1980|321|4180|Friday|1980Q1|N|Y|N|2444271|2444301|2443913|2444186|N|N|N|N|N| +2444279|AAAAAAAAHPLEFCAA|1980-02-09|961|4180|321|1980|6|2|9|1|1980|321|4180|Saturday|1980Q1|N|Y|N|2444271|2444301|2443914|2444187|N|N|N|N|N| +2444280|AAAAAAAAIPLEFCAA|1980-02-10|961|4180|321|1980|0|2|10|1|1980|321|4180|Sunday|1980Q1|N|N|N|2444271|2444301|2443915|2444188|N|N|N|N|N| +2444281|AAAAAAAAJPLEFCAA|1980-02-11|961|4180|321|1980|1|2|11|1|1980|321|4180|Monday|1980Q1|N|N|N|2444271|2444301|2443916|2444189|N|N|N|N|N| +2444282|AAAAAAAAKPLEFCAA|1980-02-12|961|4181|321|1980|2|2|12|1|1980|321|4181|Tuesday|1980Q1|N|N|N|2444271|2444301|2443917|2444190|N|N|N|N|N| +2444283|AAAAAAAALPLEFCAA|1980-02-13|961|4181|321|1980|3|2|13|1|1980|321|4181|Wednesday|1980Q1|N|N|N|2444271|2444301|2443918|2444191|N|N|N|N|N| +2444284|AAAAAAAAMPLEFCAA|1980-02-14|961|4181|321|1980|4|2|14|1|1980|321|4181|Thursday|1980Q1|N|N|N|2444271|2444301|2443919|2444192|N|N|N|N|N| +2444285|AAAAAAAANPLEFCAA|1980-02-15|961|4181|321|1980|5|2|15|1|1980|321|4181|Friday|1980Q1|N|Y|N|2444271|2444301|2443920|2444193|N|N|N|N|N| +2444286|AAAAAAAAOPLEFCAA|1980-02-16|961|4181|321|1980|6|2|16|1|1980|321|4181|Saturday|1980Q1|N|Y|N|2444271|2444301|2443921|2444194|N|N|N|N|N| +2444287|AAAAAAAAPPLEFCAA|1980-02-17|961|4181|321|1980|0|2|17|1|1980|321|4181|Sunday|1980Q1|N|N|N|2444271|2444301|2443922|2444195|N|N|N|N|N| +2444288|AAAAAAAAAAMEFCAA|1980-02-18|961|4181|321|1980|1|2|18|1|1980|321|4181|Monday|1980Q1|N|N|N|2444271|2444301|2443923|2444196|N|N|N|N|N| +2444289|AAAAAAAABAMEFCAA|1980-02-19|961|4182|321|1980|2|2|19|1|1980|321|4182|Tuesday|1980Q1|N|N|N|2444271|2444301|2443924|2444197|N|N|N|N|N| +2444290|AAAAAAAACAMEFCAA|1980-02-20|961|4182|321|1980|3|2|20|1|1980|321|4182|Wednesday|1980Q1|N|N|N|2444271|2444301|2443925|2444198|N|N|N|N|N| +2444291|AAAAAAAADAMEFCAA|1980-02-21|961|4182|321|1980|4|2|21|1|1980|321|4182|Thursday|1980Q1|N|N|N|2444271|2444301|2443926|2444199|N|N|N|N|N| +2444292|AAAAAAAAEAMEFCAA|1980-02-22|961|4182|321|1980|5|2|22|1|1980|321|4182|Friday|1980Q1|N|Y|N|2444271|2444301|2443927|2444200|N|N|N|N|N| +2444293|AAAAAAAAFAMEFCAA|1980-02-23|961|4182|321|1980|6|2|23|1|1980|321|4182|Saturday|1980Q1|N|Y|N|2444271|2444301|2443928|2444201|N|N|N|N|N| +2444294|AAAAAAAAGAMEFCAA|1980-02-24|961|4182|321|1980|0|2|24|1|1980|321|4182|Sunday|1980Q1|N|N|N|2444271|2444301|2443929|2444202|N|N|N|N|N| +2444295|AAAAAAAAHAMEFCAA|1980-02-25|961|4182|321|1980|1|2|25|1|1980|321|4182|Monday|1980Q1|N|N|N|2444271|2444301|2443930|2444203|N|N|N|N|N| +2444296|AAAAAAAAIAMEFCAA|1980-02-26|961|4183|321|1980|2|2|26|1|1980|321|4183|Tuesday|1980Q1|N|N|N|2444271|2444301|2443931|2444204|N|N|N|N|N| +2444297|AAAAAAAAJAMEFCAA|1980-02-27|961|4183|321|1980|3|2|27|1|1980|321|4183|Wednesday|1980Q1|N|N|N|2444271|2444301|2443932|2444205|N|N|N|N|N| +2444298|AAAAAAAAKAMEFCAA|1980-02-28|961|4183|321|1980|4|2|28|1|1980|321|4183|Thursday|1980Q1|N|N|N|2444271|2444301|2443933|2444206|N|N|N|N|N| +2444299|AAAAAAAALAMEFCAA|1980-02-29|961|4183|321|1980|5|2|29|1|1980|321|4183|Friday|1980Q1|N|Y|N|2444271|2444301|2443933|2444207|N|N|N|N|N| +2444300|AAAAAAAAMAMEFCAA|1980-03-01|962|4183|322|1980|6|3|1|1|1980|322|4183|Saturday|1980Q1|N|Y|N|2444300|2444359|2443934|2444208|N|N|N|N|N| +2444301|AAAAAAAANAMEFCAA|1980-03-02|962|4183|322|1980|0|3|2|1|1980|322|4183|Sunday|1980Q1|N|N|N|2444300|2444359|2443935|2444209|N|N|N|N|N| +2444302|AAAAAAAAOAMEFCAA|1980-03-03|962|4183|322|1980|1|3|3|1|1980|322|4183|Monday|1980Q1|N|N|N|2444300|2444359|2443936|2444210|N|N|N|N|N| +2444303|AAAAAAAAPAMEFCAA|1980-03-04|962|4184|322|1980|2|3|4|1|1980|322|4184|Tuesday|1980Q1|N|N|N|2444300|2444359|2443937|2444211|N|N|N|N|N| +2444304|AAAAAAAAABMEFCAA|1980-03-05|962|4184|322|1980|3|3|5|1|1980|322|4184|Wednesday|1980Q1|N|N|N|2444300|2444359|2443938|2444212|N|N|N|N|N| +2444305|AAAAAAAABBMEFCAA|1980-03-06|962|4184|322|1980|4|3|6|1|1980|322|4184|Thursday|1980Q1|N|N|N|2444300|2444359|2443939|2444213|N|N|N|N|N| +2444306|AAAAAAAACBMEFCAA|1980-03-07|962|4184|322|1980|5|3|7|1|1980|322|4184|Friday|1980Q1|N|Y|N|2444300|2444359|2443940|2444214|N|N|N|N|N| +2444307|AAAAAAAADBMEFCAA|1980-03-08|962|4184|322|1980|6|3|8|1|1980|322|4184|Saturday|1980Q1|N|Y|N|2444300|2444359|2443941|2444215|N|N|N|N|N| +2444308|AAAAAAAAEBMEFCAA|1980-03-09|962|4184|322|1980|0|3|9|1|1980|322|4184|Sunday|1980Q1|N|N|N|2444300|2444359|2443942|2444216|N|N|N|N|N| +2444309|AAAAAAAAFBMEFCAA|1980-03-10|962|4184|322|1980|1|3|10|1|1980|322|4184|Monday|1980Q1|N|N|N|2444300|2444359|2443943|2444217|N|N|N|N|N| +2444310|AAAAAAAAGBMEFCAA|1980-03-11|962|4185|322|1980|2|3|11|1|1980|322|4185|Tuesday|1980Q1|N|N|N|2444300|2444359|2443944|2444218|N|N|N|N|N| +2444311|AAAAAAAAHBMEFCAA|1980-03-12|962|4185|322|1980|3|3|12|1|1980|322|4185|Wednesday|1980Q1|N|N|N|2444300|2444359|2443945|2444219|N|N|N|N|N| +2444312|AAAAAAAAIBMEFCAA|1980-03-13|962|4185|322|1980|4|3|13|1|1980|322|4185|Thursday|1980Q1|N|N|N|2444300|2444359|2443946|2444220|N|N|N|N|N| +2444313|AAAAAAAAJBMEFCAA|1980-03-14|962|4185|322|1980|5|3|14|1|1980|322|4185|Friday|1980Q1|N|Y|N|2444300|2444359|2443947|2444221|N|N|N|N|N| +2444314|AAAAAAAAKBMEFCAA|1980-03-15|962|4185|322|1980|6|3|15|1|1980|322|4185|Saturday|1980Q1|N|Y|N|2444300|2444359|2443948|2444222|N|N|N|N|N| +2444315|AAAAAAAALBMEFCAA|1980-03-16|962|4185|322|1980|0|3|16|1|1980|322|4185|Sunday|1980Q1|N|N|N|2444300|2444359|2443949|2444223|N|N|N|N|N| +2444316|AAAAAAAAMBMEFCAA|1980-03-17|962|4185|322|1980|1|3|17|1|1980|322|4185|Monday|1980Q1|N|N|N|2444300|2444359|2443950|2444224|N|N|N|N|N| +2444317|AAAAAAAANBMEFCAA|1980-03-18|962|4186|322|1980|2|3|18|1|1980|322|4186|Tuesday|1980Q1|N|N|N|2444300|2444359|2443951|2444225|N|N|N|N|N| +2444318|AAAAAAAAOBMEFCAA|1980-03-19|962|4186|322|1980|3|3|19|1|1980|322|4186|Wednesday|1980Q1|N|N|N|2444300|2444359|2443952|2444226|N|N|N|N|N| +2444319|AAAAAAAAPBMEFCAA|1980-03-20|962|4186|322|1980|4|3|20|1|1980|322|4186|Thursday|1980Q1|N|N|N|2444300|2444359|2443953|2444227|N|N|N|N|N| +2444320|AAAAAAAAACMEFCAA|1980-03-21|962|4186|322|1980|5|3|21|1|1980|322|4186|Friday|1980Q1|N|Y|N|2444300|2444359|2443954|2444228|N|N|N|N|N| +2444321|AAAAAAAABCMEFCAA|1980-03-22|962|4186|322|1980|6|3|22|1|1980|322|4186|Saturday|1980Q1|N|Y|N|2444300|2444359|2443955|2444229|N|N|N|N|N| +2444322|AAAAAAAACCMEFCAA|1980-03-23|962|4186|322|1980|0|3|23|1|1980|322|4186|Sunday|1980Q1|N|N|N|2444300|2444359|2443956|2444230|N|N|N|N|N| +2444323|AAAAAAAADCMEFCAA|1980-03-24|962|4186|322|1980|1|3|24|1|1980|322|4186|Monday|1980Q1|N|N|N|2444300|2444359|2443957|2444231|N|N|N|N|N| +2444324|AAAAAAAAECMEFCAA|1980-03-25|962|4187|322|1980|2|3|25|1|1980|322|4187|Tuesday|1980Q1|N|N|N|2444300|2444359|2443958|2444232|N|N|N|N|N| +2444325|AAAAAAAAFCMEFCAA|1980-03-26|962|4187|322|1980|3|3|26|1|1980|322|4187|Wednesday|1980Q1|N|N|N|2444300|2444359|2443959|2444233|N|N|N|N|N| +2444326|AAAAAAAAGCMEFCAA|1980-03-27|962|4187|322|1980|4|3|27|1|1980|322|4187|Thursday|1980Q1|N|N|N|2444300|2444359|2443960|2444234|N|N|N|N|N| +2444327|AAAAAAAAHCMEFCAA|1980-03-28|962|4187|322|1980|5|3|28|1|1980|322|4187|Friday|1980Q1|N|Y|N|2444300|2444359|2443961|2444235|N|N|N|N|N| +2444328|AAAAAAAAICMEFCAA|1980-03-29|962|4187|322|1980|6|3|29|1|1980|322|4187|Saturday|1980Q1|N|Y|N|2444300|2444359|2443962|2444236|N|N|N|N|N| +2444329|AAAAAAAAJCMEFCAA|1980-03-30|962|4187|322|1980|0|3|30|1|1980|322|4187|Sunday|1980Q1|N|N|N|2444300|2444359|2443963|2444237|N|N|N|N|N| +2444330|AAAAAAAAKCMEFCAA|1980-03-31|962|4187|322|1980|1|3|31|1|1980|322|4187|Monday|1980Q1|N|N|N|2444300|2444359|2443964|2444238|N|N|N|N|N| +2444331|AAAAAAAALCMEFCAA|1980-04-01|963|4188|322|1980|2|4|1|2|1980|322|4188|Tuesday|1980Q2|N|N|N|2444331|2444421|2443965|2444240|N|N|N|N|N| +2444332|AAAAAAAAMCMEFCAA|1980-04-02|963|4188|322|1980|3|4|2|2|1980|322|4188|Wednesday|1980Q2|N|N|N|2444331|2444421|2443966|2444241|N|N|N|N|N| +2444333|AAAAAAAANCMEFCAA|1980-04-03|963|4188|322|1980|4|4|3|2|1980|322|4188|Thursday|1980Q2|N|N|N|2444331|2444421|2443967|2444242|N|N|N|N|N| +2444334|AAAAAAAAOCMEFCAA|1980-04-04|963|4188|322|1980|5|4|4|2|1980|322|4188|Friday|1980Q2|N|Y|N|2444331|2444421|2443968|2444243|N|N|N|N|N| +2444335|AAAAAAAAPCMEFCAA|1980-04-05|963|4188|322|1980|6|4|5|2|1980|322|4188|Saturday|1980Q2|N|Y|N|2444331|2444421|2443969|2444244|N|N|N|N|N| +2444336|AAAAAAAAADMEFCAA|1980-04-06|963|4188|322|1980|0|4|6|2|1980|322|4188|Sunday|1980Q2|N|N|N|2444331|2444421|2443970|2444245|N|N|N|N|N| +2444337|AAAAAAAABDMEFCAA|1980-04-07|963|4188|322|1980|1|4|7|2|1980|322|4188|Monday|1980Q2|N|N|N|2444331|2444421|2443971|2444246|N|N|N|N|N| +2444338|AAAAAAAACDMEFCAA|1980-04-08|963|4189|322|1980|2|4|8|2|1980|322|4189|Tuesday|1980Q2|N|N|N|2444331|2444421|2443972|2444247|N|N|N|N|N| +2444339|AAAAAAAADDMEFCAA|1980-04-09|963|4189|322|1980|3|4|9|2|1980|322|4189|Wednesday|1980Q2|N|N|N|2444331|2444421|2443973|2444248|N|N|N|N|N| +2444340|AAAAAAAAEDMEFCAA|1980-04-10|963|4189|322|1980|4|4|10|2|1980|322|4189|Thursday|1980Q2|N|N|N|2444331|2444421|2443974|2444249|N|N|N|N|N| +2444341|AAAAAAAAFDMEFCAA|1980-04-11|963|4189|322|1980|5|4|11|2|1980|322|4189|Friday|1980Q2|N|Y|N|2444331|2444421|2443975|2444250|N|N|N|N|N| +2444342|AAAAAAAAGDMEFCAA|1980-04-12|963|4189|322|1980|6|4|12|2|1980|322|4189|Saturday|1980Q2|N|Y|N|2444331|2444421|2443976|2444251|N|N|N|N|N| +2444343|AAAAAAAAHDMEFCAA|1980-04-13|963|4189|322|1980|0|4|13|2|1980|322|4189|Sunday|1980Q2|N|N|N|2444331|2444421|2443977|2444252|N|N|N|N|N| +2444344|AAAAAAAAIDMEFCAA|1980-04-14|963|4189|322|1980|1|4|14|2|1980|322|4189|Monday|1980Q2|N|N|N|2444331|2444421|2443978|2444253|N|N|N|N|N| +2444345|AAAAAAAAJDMEFCAA|1980-04-15|963|4190|322|1980|2|4|15|2|1980|322|4190|Tuesday|1980Q2|N|N|N|2444331|2444421|2443979|2444254|N|N|N|N|N| +2444346|AAAAAAAAKDMEFCAA|1980-04-16|963|4190|322|1980|3|4|16|2|1980|322|4190|Wednesday|1980Q2|N|N|N|2444331|2444421|2443980|2444255|N|N|N|N|N| +2444347|AAAAAAAALDMEFCAA|1980-04-17|963|4190|322|1980|4|4|17|2|1980|322|4190|Thursday|1980Q2|N|N|N|2444331|2444421|2443981|2444256|N|N|N|N|N| +2444348|AAAAAAAAMDMEFCAA|1980-04-18|963|4190|322|1980|5|4|18|2|1980|322|4190|Friday|1980Q2|N|Y|N|2444331|2444421|2443982|2444257|N|N|N|N|N| +2444349|AAAAAAAANDMEFCAA|1980-04-19|963|4190|322|1980|6|4|19|2|1980|322|4190|Saturday|1980Q2|N|Y|N|2444331|2444421|2443983|2444258|N|N|N|N|N| +2444350|AAAAAAAAODMEFCAA|1980-04-20|963|4190|322|1980|0|4|20|2|1980|322|4190|Sunday|1980Q2|N|N|N|2444331|2444421|2443984|2444259|N|N|N|N|N| +2444351|AAAAAAAAPDMEFCAA|1980-04-21|963|4190|322|1980|1|4|21|2|1980|322|4190|Monday|1980Q2|N|N|N|2444331|2444421|2443985|2444260|N|N|N|N|N| +2444352|AAAAAAAAAEMEFCAA|1980-04-22|963|4191|322|1980|2|4|22|2|1980|322|4191|Tuesday|1980Q2|N|N|N|2444331|2444421|2443986|2444261|N|N|N|N|N| +2444353|AAAAAAAABEMEFCAA|1980-04-23|963|4191|322|1980|3|4|23|2|1980|322|4191|Wednesday|1980Q2|N|N|N|2444331|2444421|2443987|2444262|N|N|N|N|N| +2444354|AAAAAAAACEMEFCAA|1980-04-24|963|4191|322|1980|4|4|24|2|1980|322|4191|Thursday|1980Q2|N|N|N|2444331|2444421|2443988|2444263|N|N|N|N|N| +2444355|AAAAAAAADEMEFCAA|1980-04-25|963|4191|322|1980|5|4|25|2|1980|322|4191|Friday|1980Q2|N|Y|N|2444331|2444421|2443989|2444264|N|N|N|N|N| +2444356|AAAAAAAAEEMEFCAA|1980-04-26|963|4191|322|1980|6|4|26|2|1980|322|4191|Saturday|1980Q2|N|Y|N|2444331|2444421|2443990|2444265|N|N|N|N|N| +2444357|AAAAAAAAFEMEFCAA|1980-04-27|963|4191|322|1980|0|4|27|2|1980|322|4191|Sunday|1980Q2|N|N|N|2444331|2444421|2443991|2444266|N|N|N|N|N| +2444358|AAAAAAAAGEMEFCAA|1980-04-28|963|4191|322|1980|1|4|28|2|1980|322|4191|Monday|1980Q2|N|N|N|2444331|2444421|2443992|2444267|N|N|N|N|N| +2444359|AAAAAAAAHEMEFCAA|1980-04-29|963|4192|322|1980|2|4|29|2|1980|322|4192|Tuesday|1980Q2|N|N|N|2444331|2444421|2443993|2444268|N|N|N|N|N| +2444360|AAAAAAAAIEMEFCAA|1980-04-30|963|4192|322|1980|3|4|30|2|1980|322|4192|Wednesday|1980Q2|N|N|N|2444331|2444421|2443994|2444269|N|N|N|N|N| +2444361|AAAAAAAAJEMEFCAA|1980-05-01|964|4192|322|1980|4|5|1|2|1980|322|4192|Thursday|1980Q2|N|N|N|2444361|2444481|2443995|2444270|N|N|N|N|N| +2444362|AAAAAAAAKEMEFCAA|1980-05-02|964|4192|322|1980|5|5|2|2|1980|322|4192|Friday|1980Q2|N|Y|N|2444361|2444481|2443996|2444271|N|N|N|N|N| +2444363|AAAAAAAALEMEFCAA|1980-05-03|964|4192|322|1980|6|5|3|2|1980|322|4192|Saturday|1980Q2|N|Y|N|2444361|2444481|2443997|2444272|N|N|N|N|N| +2444364|AAAAAAAAMEMEFCAA|1980-05-04|964|4192|322|1980|0|5|4|2|1980|322|4192|Sunday|1980Q2|N|N|N|2444361|2444481|2443998|2444273|N|N|N|N|N| +2444365|AAAAAAAANEMEFCAA|1980-05-05|964|4192|322|1980|1|5|5|2|1980|322|4192|Monday|1980Q2|N|N|N|2444361|2444481|2443999|2444274|N|N|N|N|N| +2444366|AAAAAAAAOEMEFCAA|1980-05-06|964|4193|322|1980|2|5|6|2|1980|322|4193|Tuesday|1980Q2|N|N|N|2444361|2444481|2444000|2444275|N|N|N|N|N| +2444367|AAAAAAAAPEMEFCAA|1980-05-07|964|4193|322|1980|3|5|7|2|1980|322|4193|Wednesday|1980Q2|N|N|N|2444361|2444481|2444001|2444276|N|N|N|N|N| +2444368|AAAAAAAAAFMEFCAA|1980-05-08|964|4193|322|1980|4|5|8|2|1980|322|4193|Thursday|1980Q2|N|N|N|2444361|2444481|2444002|2444277|N|N|N|N|N| +2444369|AAAAAAAABFMEFCAA|1980-05-09|964|4193|322|1980|5|5|9|2|1980|322|4193|Friday|1980Q2|N|Y|N|2444361|2444481|2444003|2444278|N|N|N|N|N| +2444370|AAAAAAAACFMEFCAA|1980-05-10|964|4193|322|1980|6|5|10|2|1980|322|4193|Saturday|1980Q2|N|Y|N|2444361|2444481|2444004|2444279|N|N|N|N|N| +2444371|AAAAAAAADFMEFCAA|1980-05-11|964|4193|322|1980|0|5|11|2|1980|322|4193|Sunday|1980Q2|N|N|N|2444361|2444481|2444005|2444280|N|N|N|N|N| +2444372|AAAAAAAAEFMEFCAA|1980-05-12|964|4193|322|1980|1|5|12|2|1980|322|4193|Monday|1980Q2|N|N|N|2444361|2444481|2444006|2444281|N|N|N|N|N| +2444373|AAAAAAAAFFMEFCAA|1980-05-13|964|4194|322|1980|2|5|13|2|1980|322|4194|Tuesday|1980Q2|N|N|N|2444361|2444481|2444007|2444282|N|N|N|N|N| +2444374|AAAAAAAAGFMEFCAA|1980-05-14|964|4194|322|1980|3|5|14|2|1980|322|4194|Wednesday|1980Q2|N|N|N|2444361|2444481|2444008|2444283|N|N|N|N|N| +2444375|AAAAAAAAHFMEFCAA|1980-05-15|964|4194|322|1980|4|5|15|2|1980|322|4194|Thursday|1980Q2|N|N|N|2444361|2444481|2444009|2444284|N|N|N|N|N| +2444376|AAAAAAAAIFMEFCAA|1980-05-16|964|4194|322|1980|5|5|16|2|1980|322|4194|Friday|1980Q2|N|Y|N|2444361|2444481|2444010|2444285|N|N|N|N|N| +2444377|AAAAAAAAJFMEFCAA|1980-05-17|964|4194|322|1980|6|5|17|2|1980|322|4194|Saturday|1980Q2|N|Y|N|2444361|2444481|2444011|2444286|N|N|N|N|N| +2444378|AAAAAAAAKFMEFCAA|1980-05-18|964|4194|322|1980|0|5|18|2|1980|322|4194|Sunday|1980Q2|N|N|N|2444361|2444481|2444012|2444287|N|N|N|N|N| +2444379|AAAAAAAALFMEFCAA|1980-05-19|964|4194|322|1980|1|5|19|2|1980|322|4194|Monday|1980Q2|N|N|N|2444361|2444481|2444013|2444288|N|N|N|N|N| +2444380|AAAAAAAAMFMEFCAA|1980-05-20|964|4195|322|1980|2|5|20|2|1980|322|4195|Tuesday|1980Q2|N|N|N|2444361|2444481|2444014|2444289|N|N|N|N|N| +2444381|AAAAAAAANFMEFCAA|1980-05-21|964|4195|322|1980|3|5|21|2|1980|322|4195|Wednesday|1980Q2|N|N|N|2444361|2444481|2444015|2444290|N|N|N|N|N| +2444382|AAAAAAAAOFMEFCAA|1980-05-22|964|4195|322|1980|4|5|22|2|1980|322|4195|Thursday|1980Q2|N|N|N|2444361|2444481|2444016|2444291|N|N|N|N|N| +2444383|AAAAAAAAPFMEFCAA|1980-05-23|964|4195|322|1980|5|5|23|2|1980|322|4195|Friday|1980Q2|N|Y|N|2444361|2444481|2444017|2444292|N|N|N|N|N| +2444384|AAAAAAAAAGMEFCAA|1980-05-24|964|4195|322|1980|6|5|24|2|1980|322|4195|Saturday|1980Q2|N|Y|N|2444361|2444481|2444018|2444293|N|N|N|N|N| +2444385|AAAAAAAABGMEFCAA|1980-05-25|964|4195|322|1980|0|5|25|2|1980|322|4195|Sunday|1980Q2|N|N|N|2444361|2444481|2444019|2444294|N|N|N|N|N| +2444386|AAAAAAAACGMEFCAA|1980-05-26|964|4195|322|1980|1|5|26|2|1980|322|4195|Monday|1980Q2|N|N|N|2444361|2444481|2444020|2444295|N|N|N|N|N| +2444387|AAAAAAAADGMEFCAA|1980-05-27|964|4196|322|1980|2|5|27|2|1980|322|4196|Tuesday|1980Q2|N|N|N|2444361|2444481|2444021|2444296|N|N|N|N|N| +2444388|AAAAAAAAEGMEFCAA|1980-05-28|964|4196|322|1980|3|5|28|2|1980|322|4196|Wednesday|1980Q2|N|N|N|2444361|2444481|2444022|2444297|N|N|N|N|N| +2444389|AAAAAAAAFGMEFCAA|1980-05-29|964|4196|322|1980|4|5|29|2|1980|322|4196|Thursday|1980Q2|N|N|N|2444361|2444481|2444023|2444298|N|N|N|N|N| +2444390|AAAAAAAAGGMEFCAA|1980-05-30|964|4196|322|1980|5|5|30|2|1980|322|4196|Friday|1980Q2|N|Y|N|2444361|2444481|2444024|2444299|N|N|N|N|N| +2444391|AAAAAAAAHGMEFCAA|1980-05-31|964|4196|322|1980|6|5|31|2|1980|322|4196|Saturday|1980Q2|N|Y|N|2444361|2444481|2444025|2444300|N|N|N|N|N| +2444392|AAAAAAAAIGMEFCAA|1980-06-01|965|4196|323|1980|0|6|1|2|1980|323|4196|Sunday|1980Q2|N|N|N|2444392|2444543|2444026|2444301|N|N|N|N|N| +2444393|AAAAAAAAJGMEFCAA|1980-06-02|965|4196|323|1980|1|6|2|2|1980|323|4196|Monday|1980Q2|N|N|N|2444392|2444543|2444027|2444302|N|N|N|N|N| +2444394|AAAAAAAAKGMEFCAA|1980-06-03|965|4197|323|1980|2|6|3|2|1980|323|4197|Tuesday|1980Q2|N|N|N|2444392|2444543|2444028|2444303|N|N|N|N|N| +2444395|AAAAAAAALGMEFCAA|1980-06-04|965|4197|323|1980|3|6|4|2|1980|323|4197|Wednesday|1980Q2|N|N|N|2444392|2444543|2444029|2444304|N|N|N|N|N| +2444396|AAAAAAAAMGMEFCAA|1980-06-05|965|4197|323|1980|4|6|5|2|1980|323|4197|Thursday|1980Q2|N|N|N|2444392|2444543|2444030|2444305|N|N|N|N|N| +2444397|AAAAAAAANGMEFCAA|1980-06-06|965|4197|323|1980|5|6|6|2|1980|323|4197|Friday|1980Q2|N|Y|N|2444392|2444543|2444031|2444306|N|N|N|N|N| +2444398|AAAAAAAAOGMEFCAA|1980-06-07|965|4197|323|1980|6|6|7|2|1980|323|4197|Saturday|1980Q2|N|Y|N|2444392|2444543|2444032|2444307|N|N|N|N|N| +2444399|AAAAAAAAPGMEFCAA|1980-06-08|965|4197|323|1980|0|6|8|2|1980|323|4197|Sunday|1980Q2|N|N|N|2444392|2444543|2444033|2444308|N|N|N|N|N| +2444400|AAAAAAAAAHMEFCAA|1980-06-09|965|4197|323|1980|1|6|9|2|1980|323|4197|Monday|1980Q2|N|N|N|2444392|2444543|2444034|2444309|N|N|N|N|N| +2444401|AAAAAAAABHMEFCAA|1980-06-10|965|4198|323|1980|2|6|10|2|1980|323|4198|Tuesday|1980Q2|N|N|N|2444392|2444543|2444035|2444310|N|N|N|N|N| +2444402|AAAAAAAACHMEFCAA|1980-06-11|965|4198|323|1980|3|6|11|2|1980|323|4198|Wednesday|1980Q2|N|N|N|2444392|2444543|2444036|2444311|N|N|N|N|N| +2444403|AAAAAAAADHMEFCAA|1980-06-12|965|4198|323|1980|4|6|12|2|1980|323|4198|Thursday|1980Q2|N|N|N|2444392|2444543|2444037|2444312|N|N|N|N|N| +2444404|AAAAAAAAEHMEFCAA|1980-06-13|965|4198|323|1980|5|6|13|2|1980|323|4198|Friday|1980Q2|N|Y|N|2444392|2444543|2444038|2444313|N|N|N|N|N| +2444405|AAAAAAAAFHMEFCAA|1980-06-14|965|4198|323|1980|6|6|14|2|1980|323|4198|Saturday|1980Q2|N|Y|N|2444392|2444543|2444039|2444314|N|N|N|N|N| +2444406|AAAAAAAAGHMEFCAA|1980-06-15|965|4198|323|1980|0|6|15|2|1980|323|4198|Sunday|1980Q2|N|N|N|2444392|2444543|2444040|2444315|N|N|N|N|N| +2444407|AAAAAAAAHHMEFCAA|1980-06-16|965|4198|323|1980|1|6|16|2|1980|323|4198|Monday|1980Q2|N|N|N|2444392|2444543|2444041|2444316|N|N|N|N|N| +2444408|AAAAAAAAIHMEFCAA|1980-06-17|965|4199|323|1980|2|6|17|2|1980|323|4199|Tuesday|1980Q2|N|N|N|2444392|2444543|2444042|2444317|N|N|N|N|N| +2444409|AAAAAAAAJHMEFCAA|1980-06-18|965|4199|323|1980|3|6|18|2|1980|323|4199|Wednesday|1980Q2|N|N|N|2444392|2444543|2444043|2444318|N|N|N|N|N| +2444410|AAAAAAAAKHMEFCAA|1980-06-19|965|4199|323|1980|4|6|19|2|1980|323|4199|Thursday|1980Q2|N|N|N|2444392|2444543|2444044|2444319|N|N|N|N|N| +2444411|AAAAAAAALHMEFCAA|1980-06-20|965|4199|323|1980|5|6|20|2|1980|323|4199|Friday|1980Q2|N|Y|N|2444392|2444543|2444045|2444320|N|N|N|N|N| +2444412|AAAAAAAAMHMEFCAA|1980-06-21|965|4199|323|1980|6|6|21|2|1980|323|4199|Saturday|1980Q2|N|Y|N|2444392|2444543|2444046|2444321|N|N|N|N|N| +2444413|AAAAAAAANHMEFCAA|1980-06-22|965|4199|323|1980|0|6|22|2|1980|323|4199|Sunday|1980Q2|N|N|N|2444392|2444543|2444047|2444322|N|N|N|N|N| +2444414|AAAAAAAAOHMEFCAA|1980-06-23|965|4199|323|1980|1|6|23|2|1980|323|4199|Monday|1980Q2|N|N|N|2444392|2444543|2444048|2444323|N|N|N|N|N| +2444415|AAAAAAAAPHMEFCAA|1980-06-24|965|4200|323|1980|2|6|24|2|1980|323|4200|Tuesday|1980Q2|N|N|N|2444392|2444543|2444049|2444324|N|N|N|N|N| +2444416|AAAAAAAAAIMEFCAA|1980-06-25|965|4200|323|1980|3|6|25|2|1980|323|4200|Wednesday|1980Q2|N|N|N|2444392|2444543|2444050|2444325|N|N|N|N|N| +2444417|AAAAAAAABIMEFCAA|1980-06-26|965|4200|323|1980|4|6|26|2|1980|323|4200|Thursday|1980Q2|N|N|N|2444392|2444543|2444051|2444326|N|N|N|N|N| +2444418|AAAAAAAACIMEFCAA|1980-06-27|965|4200|323|1980|5|6|27|2|1980|323|4200|Friday|1980Q2|N|Y|N|2444392|2444543|2444052|2444327|N|N|N|N|N| +2444419|AAAAAAAADIMEFCAA|1980-06-28|965|4200|323|1980|6|6|28|2|1980|323|4200|Saturday|1980Q2|N|Y|N|2444392|2444543|2444053|2444328|N|N|N|N|N| +2444420|AAAAAAAAEIMEFCAA|1980-06-29|965|4200|323|1980|0|6|29|2|1980|323|4200|Sunday|1980Q2|N|N|N|2444392|2444543|2444054|2444329|N|N|N|N|N| +2444421|AAAAAAAAFIMEFCAA|1980-06-30|965|4200|323|1980|1|6|30|2|1980|323|4200|Monday|1980Q2|N|N|N|2444392|2444543|2444055|2444330|N|N|N|N|N| +2444422|AAAAAAAAGIMEFCAA|1980-07-01|966|4201|323|1980|2|7|1|3|1980|323|4201|Tuesday|1980Q3|N|N|N|2444422|2444603|2444056|2444331|N|N|N|N|N| +2444423|AAAAAAAAHIMEFCAA|1980-07-02|966|4201|323|1980|3|7|2|3|1980|323|4201|Wednesday|1980Q3|N|N|N|2444422|2444603|2444057|2444332|N|N|N|N|N| +2444424|AAAAAAAAIIMEFCAA|1980-07-03|966|4201|323|1980|4|7|3|3|1980|323|4201|Thursday|1980Q3|N|N|N|2444422|2444603|2444058|2444333|N|N|N|N|N| +2444425|AAAAAAAAJIMEFCAA|1980-07-04|966|4201|323|1980|5|7|4|3|1980|323|4201|Friday|1980Q3|Y|Y|N|2444422|2444603|2444059|2444334|N|N|N|N|N| +2444426|AAAAAAAAKIMEFCAA|1980-07-05|966|4201|323|1980|6|7|5|3|1980|323|4201|Saturday|1980Q3|N|Y|Y|2444422|2444603|2444060|2444335|N|N|N|N|N| +2444427|AAAAAAAALIMEFCAA|1980-07-06|966|4201|323|1980|0|7|6|3|1980|323|4201|Sunday|1980Q3|N|N|N|2444422|2444603|2444061|2444336|N|N|N|N|N| +2444428|AAAAAAAAMIMEFCAA|1980-07-07|966|4201|323|1980|1|7|7|3|1980|323|4201|Monday|1980Q3|N|N|N|2444422|2444603|2444062|2444337|N|N|N|N|N| +2444429|AAAAAAAANIMEFCAA|1980-07-08|966|4202|323|1980|2|7|8|3|1980|323|4202|Tuesday|1980Q3|N|N|N|2444422|2444603|2444063|2444338|N|N|N|N|N| +2444430|AAAAAAAAOIMEFCAA|1980-07-09|966|4202|323|1980|3|7|9|3|1980|323|4202|Wednesday|1980Q3|N|N|N|2444422|2444603|2444064|2444339|N|N|N|N|N| +2444431|AAAAAAAAPIMEFCAA|1980-07-10|966|4202|323|1980|4|7|10|3|1980|323|4202|Thursday|1980Q3|N|N|N|2444422|2444603|2444065|2444340|N|N|N|N|N| +2444432|AAAAAAAAAJMEFCAA|1980-07-11|966|4202|323|1980|5|7|11|3|1980|323|4202|Friday|1980Q3|N|Y|N|2444422|2444603|2444066|2444341|N|N|N|N|N| +2444433|AAAAAAAABJMEFCAA|1980-07-12|966|4202|323|1980|6|7|12|3|1980|323|4202|Saturday|1980Q3|N|Y|N|2444422|2444603|2444067|2444342|N|N|N|N|N| +2444434|AAAAAAAACJMEFCAA|1980-07-13|966|4202|323|1980|0|7|13|3|1980|323|4202|Sunday|1980Q3|N|N|N|2444422|2444603|2444068|2444343|N|N|N|N|N| +2444435|AAAAAAAADJMEFCAA|1980-07-14|966|4202|323|1980|1|7|14|3|1980|323|4202|Monday|1980Q3|N|N|N|2444422|2444603|2444069|2444344|N|N|N|N|N| +2444436|AAAAAAAAEJMEFCAA|1980-07-15|966|4203|323|1980|2|7|15|3|1980|323|4203|Tuesday|1980Q3|N|N|N|2444422|2444603|2444070|2444345|N|N|N|N|N| +2444437|AAAAAAAAFJMEFCAA|1980-07-16|966|4203|323|1980|3|7|16|3|1980|323|4203|Wednesday|1980Q3|N|N|N|2444422|2444603|2444071|2444346|N|N|N|N|N| +2444438|AAAAAAAAGJMEFCAA|1980-07-17|966|4203|323|1980|4|7|17|3|1980|323|4203|Thursday|1980Q3|N|N|N|2444422|2444603|2444072|2444347|N|N|N|N|N| +2444439|AAAAAAAAHJMEFCAA|1980-07-18|966|4203|323|1980|5|7|18|3|1980|323|4203|Friday|1980Q3|N|Y|N|2444422|2444603|2444073|2444348|N|N|N|N|N| +2444440|AAAAAAAAIJMEFCAA|1980-07-19|966|4203|323|1980|6|7|19|3|1980|323|4203|Saturday|1980Q3|N|Y|N|2444422|2444603|2444074|2444349|N|N|N|N|N| +2444441|AAAAAAAAJJMEFCAA|1980-07-20|966|4203|323|1980|0|7|20|3|1980|323|4203|Sunday|1980Q3|N|N|N|2444422|2444603|2444075|2444350|N|N|N|N|N| +2444442|AAAAAAAAKJMEFCAA|1980-07-21|966|4203|323|1980|1|7|21|3|1980|323|4203|Monday|1980Q3|N|N|N|2444422|2444603|2444076|2444351|N|N|N|N|N| +2444443|AAAAAAAALJMEFCAA|1980-07-22|966|4204|323|1980|2|7|22|3|1980|323|4204|Tuesday|1980Q3|N|N|N|2444422|2444603|2444077|2444352|N|N|N|N|N| +2444444|AAAAAAAAMJMEFCAA|1980-07-23|966|4204|323|1980|3|7|23|3|1980|323|4204|Wednesday|1980Q3|N|N|N|2444422|2444603|2444078|2444353|N|N|N|N|N| +2444445|AAAAAAAANJMEFCAA|1980-07-24|966|4204|323|1980|4|7|24|3|1980|323|4204|Thursday|1980Q3|N|N|N|2444422|2444603|2444079|2444354|N|N|N|N|N| +2444446|AAAAAAAAOJMEFCAA|1980-07-25|966|4204|323|1980|5|7|25|3|1980|323|4204|Friday|1980Q3|N|Y|N|2444422|2444603|2444080|2444355|N|N|N|N|N| +2444447|AAAAAAAAPJMEFCAA|1980-07-26|966|4204|323|1980|6|7|26|3|1980|323|4204|Saturday|1980Q3|N|Y|N|2444422|2444603|2444081|2444356|N|N|N|N|N| +2444448|AAAAAAAAAKMEFCAA|1980-07-27|966|4204|323|1980|0|7|27|3|1980|323|4204|Sunday|1980Q3|N|N|N|2444422|2444603|2444082|2444357|N|N|N|N|N| +2444449|AAAAAAAABKMEFCAA|1980-07-28|966|4204|323|1980|1|7|28|3|1980|323|4204|Monday|1980Q3|N|N|N|2444422|2444603|2444083|2444358|N|N|N|N|N| +2444450|AAAAAAAACKMEFCAA|1980-07-29|966|4205|323|1980|2|7|29|3|1980|323|4205|Tuesday|1980Q3|N|N|N|2444422|2444603|2444084|2444359|N|N|N|N|N| +2444451|AAAAAAAADKMEFCAA|1980-07-30|966|4205|323|1980|3|7|30|3|1980|323|4205|Wednesday|1980Q3|N|N|N|2444422|2444603|2444085|2444360|N|N|N|N|N| +2444452|AAAAAAAAEKMEFCAA|1980-07-31|966|4205|323|1980|4|7|31|3|1980|323|4205|Thursday|1980Q3|N|N|N|2444422|2444603|2444086|2444361|N|N|N|N|N| +2444453|AAAAAAAAFKMEFCAA|1980-08-01|967|4205|323|1980|5|8|1|3|1980|323|4205|Friday|1980Q3|N|Y|N|2444453|2444665|2444087|2444362|N|N|N|N|N| +2444454|AAAAAAAAGKMEFCAA|1980-08-02|967|4205|323|1980|6|8|2|3|1980|323|4205|Saturday|1980Q3|N|Y|N|2444453|2444665|2444088|2444363|N|N|N|N|N| +2444455|AAAAAAAAHKMEFCAA|1980-08-03|967|4205|323|1980|0|8|3|3|1980|323|4205|Sunday|1980Q3|N|N|N|2444453|2444665|2444089|2444364|N|N|N|N|N| +2444456|AAAAAAAAIKMEFCAA|1980-08-04|967|4205|323|1980|1|8|4|3|1980|323|4205|Monday|1980Q3|N|N|N|2444453|2444665|2444090|2444365|N|N|N|N|N| +2444457|AAAAAAAAJKMEFCAA|1980-08-05|967|4206|323|1980|2|8|5|3|1980|323|4206|Tuesday|1980Q3|N|N|N|2444453|2444665|2444091|2444366|N|N|N|N|N| +2444458|AAAAAAAAKKMEFCAA|1980-08-06|967|4206|323|1980|3|8|6|3|1980|323|4206|Wednesday|1980Q3|N|N|N|2444453|2444665|2444092|2444367|N|N|N|N|N| +2444459|AAAAAAAALKMEFCAA|1980-08-07|967|4206|323|1980|4|8|7|3|1980|323|4206|Thursday|1980Q3|N|N|N|2444453|2444665|2444093|2444368|N|N|N|N|N| +2444460|AAAAAAAAMKMEFCAA|1980-08-08|967|4206|323|1980|5|8|8|3|1980|323|4206|Friday|1980Q3|N|Y|N|2444453|2444665|2444094|2444369|N|N|N|N|N| +2444461|AAAAAAAANKMEFCAA|1980-08-09|967|4206|323|1980|6|8|9|3|1980|323|4206|Saturday|1980Q3|N|Y|N|2444453|2444665|2444095|2444370|N|N|N|N|N| +2444462|AAAAAAAAOKMEFCAA|1980-08-10|967|4206|323|1980|0|8|10|3|1980|323|4206|Sunday|1980Q3|N|N|N|2444453|2444665|2444096|2444371|N|N|N|N|N| +2444463|AAAAAAAAPKMEFCAA|1980-08-11|967|4206|323|1980|1|8|11|3|1980|323|4206|Monday|1980Q3|N|N|N|2444453|2444665|2444097|2444372|N|N|N|N|N| +2444464|AAAAAAAAALMEFCAA|1980-08-12|967|4207|323|1980|2|8|12|3|1980|323|4207|Tuesday|1980Q3|N|N|N|2444453|2444665|2444098|2444373|N|N|N|N|N| +2444465|AAAAAAAABLMEFCAA|1980-08-13|967|4207|323|1980|3|8|13|3|1980|323|4207|Wednesday|1980Q3|N|N|N|2444453|2444665|2444099|2444374|N|N|N|N|N| +2444466|AAAAAAAACLMEFCAA|1980-08-14|967|4207|323|1980|4|8|14|3|1980|323|4207|Thursday|1980Q3|N|N|N|2444453|2444665|2444100|2444375|N|N|N|N|N| +2444467|AAAAAAAADLMEFCAA|1980-08-15|967|4207|323|1980|5|8|15|3|1980|323|4207|Friday|1980Q3|N|Y|N|2444453|2444665|2444101|2444376|N|N|N|N|N| +2444468|AAAAAAAAELMEFCAA|1980-08-16|967|4207|323|1980|6|8|16|3|1980|323|4207|Saturday|1980Q3|N|Y|N|2444453|2444665|2444102|2444377|N|N|N|N|N| +2444469|AAAAAAAAFLMEFCAA|1980-08-17|967|4207|323|1980|0|8|17|3|1980|323|4207|Sunday|1980Q3|N|N|N|2444453|2444665|2444103|2444378|N|N|N|N|N| +2444470|AAAAAAAAGLMEFCAA|1980-08-18|967|4207|323|1980|1|8|18|3|1980|323|4207|Monday|1980Q3|N|N|N|2444453|2444665|2444104|2444379|N|N|N|N|N| +2444471|AAAAAAAAHLMEFCAA|1980-08-19|967|4208|323|1980|2|8|19|3|1980|323|4208|Tuesday|1980Q3|N|N|N|2444453|2444665|2444105|2444380|N|N|N|N|N| +2444472|AAAAAAAAILMEFCAA|1980-08-20|967|4208|323|1980|3|8|20|3|1980|323|4208|Wednesday|1980Q3|N|N|N|2444453|2444665|2444106|2444381|N|N|N|N|N| +2444473|AAAAAAAAJLMEFCAA|1980-08-21|967|4208|323|1980|4|8|21|3|1980|323|4208|Thursday|1980Q3|N|N|N|2444453|2444665|2444107|2444382|N|N|N|N|N| +2444474|AAAAAAAAKLMEFCAA|1980-08-22|967|4208|323|1980|5|8|22|3|1980|323|4208|Friday|1980Q3|N|Y|N|2444453|2444665|2444108|2444383|N|N|N|N|N| +2444475|AAAAAAAALLMEFCAA|1980-08-23|967|4208|323|1980|6|8|23|3|1980|323|4208|Saturday|1980Q3|N|Y|N|2444453|2444665|2444109|2444384|N|N|N|N|N| +2444476|AAAAAAAAMLMEFCAA|1980-08-24|967|4208|323|1980|0|8|24|3|1980|323|4208|Sunday|1980Q3|N|N|N|2444453|2444665|2444110|2444385|N|N|N|N|N| +2444477|AAAAAAAANLMEFCAA|1980-08-25|967|4208|323|1980|1|8|25|3|1980|323|4208|Monday|1980Q3|N|N|N|2444453|2444665|2444111|2444386|N|N|N|N|N| +2444478|AAAAAAAAOLMEFCAA|1980-08-26|967|4209|323|1980|2|8|26|3|1980|323|4209|Tuesday|1980Q3|N|N|N|2444453|2444665|2444112|2444387|N|N|N|N|N| +2444479|AAAAAAAAPLMEFCAA|1980-08-27|967|4209|323|1980|3|8|27|3|1980|323|4209|Wednesday|1980Q3|N|N|N|2444453|2444665|2444113|2444388|N|N|N|N|N| +2444480|AAAAAAAAAMMEFCAA|1980-08-28|967|4209|323|1980|4|8|28|3|1980|323|4209|Thursday|1980Q3|N|N|N|2444453|2444665|2444114|2444389|N|N|N|N|N| +2444481|AAAAAAAABMMEFCAA|1980-08-29|967|4209|323|1980|5|8|29|3|1980|323|4209|Friday|1980Q3|N|Y|N|2444453|2444665|2444115|2444390|N|N|N|N|N| +2444482|AAAAAAAACMMEFCAA|1980-08-30|967|4209|323|1980|6|8|30|3|1980|323|4209|Saturday|1980Q3|N|Y|N|2444453|2444665|2444116|2444391|N|N|N|N|N| +2444483|AAAAAAAADMMEFCAA|1980-08-31|967|4209|323|1980|0|8|31|3|1980|323|4209|Sunday|1980Q3|N|N|N|2444453|2444665|2444117|2444392|N|N|N|N|N| +2444484|AAAAAAAAEMMEFCAA|1980-09-01|968|4209|324|1980|1|9|1|3|1980|324|4209|Monday|1980Q3|N|N|N|2444484|2444727|2444118|2444393|N|N|N|N|N| +2444485|AAAAAAAAFMMEFCAA|1980-09-02|968|4210|324|1980|2|9|2|3|1980|324|4210|Tuesday|1980Q3|N|N|N|2444484|2444727|2444119|2444394|N|N|N|N|N| +2444486|AAAAAAAAGMMEFCAA|1980-09-03|968|4210|324|1980|3|9|3|3|1980|324|4210|Wednesday|1980Q3|N|N|N|2444484|2444727|2444120|2444395|N|N|N|N|N| +2444487|AAAAAAAAHMMEFCAA|1980-09-04|968|4210|324|1980|4|9|4|3|1980|324|4210|Thursday|1980Q3|N|N|N|2444484|2444727|2444121|2444396|N|N|N|N|N| +2444488|AAAAAAAAIMMEFCAA|1980-09-05|968|4210|324|1980|5|9|5|3|1980|324|4210|Friday|1980Q3|N|Y|N|2444484|2444727|2444122|2444397|N|N|N|N|N| +2444489|AAAAAAAAJMMEFCAA|1980-09-06|968|4210|324|1980|6|9|6|3|1980|324|4210|Saturday|1980Q3|N|Y|N|2444484|2444727|2444123|2444398|N|N|N|N|N| +2444490|AAAAAAAAKMMEFCAA|1980-09-07|968|4210|324|1980|0|9|7|3|1980|324|4210|Sunday|1980Q3|N|N|N|2444484|2444727|2444124|2444399|N|N|N|N|N| +2444491|AAAAAAAALMMEFCAA|1980-09-08|968|4210|324|1980|1|9|8|3|1980|324|4210|Monday|1980Q3|N|N|N|2444484|2444727|2444125|2444400|N|N|N|N|N| +2444492|AAAAAAAAMMMEFCAA|1980-09-09|968|4211|324|1980|2|9|9|3|1980|324|4211|Tuesday|1980Q3|N|N|N|2444484|2444727|2444126|2444401|N|N|N|N|N| +2444493|AAAAAAAANMMEFCAA|1980-09-10|968|4211|324|1980|3|9|10|3|1980|324|4211|Wednesday|1980Q3|N|N|N|2444484|2444727|2444127|2444402|N|N|N|N|N| +2444494|AAAAAAAAOMMEFCAA|1980-09-11|968|4211|324|1980|4|9|11|3|1980|324|4211|Thursday|1980Q3|N|N|N|2444484|2444727|2444128|2444403|N|N|N|N|N| +2444495|AAAAAAAAPMMEFCAA|1980-09-12|968|4211|324|1980|5|9|12|3|1980|324|4211|Friday|1980Q3|N|Y|N|2444484|2444727|2444129|2444404|N|N|N|N|N| +2444496|AAAAAAAAANMEFCAA|1980-09-13|968|4211|324|1980|6|9|13|3|1980|324|4211|Saturday|1980Q3|N|Y|N|2444484|2444727|2444130|2444405|N|N|N|N|N| +2444497|AAAAAAAABNMEFCAA|1980-09-14|968|4211|324|1980|0|9|14|3|1980|324|4211|Sunday|1980Q3|N|N|N|2444484|2444727|2444131|2444406|N|N|N|N|N| +2444498|AAAAAAAACNMEFCAA|1980-09-15|968|4211|324|1980|1|9|15|3|1980|324|4211|Monday|1980Q3|N|N|N|2444484|2444727|2444132|2444407|N|N|N|N|N| +2444499|AAAAAAAADNMEFCAA|1980-09-16|968|4212|324|1980|2|9|16|3|1980|324|4212|Tuesday|1980Q3|N|N|N|2444484|2444727|2444133|2444408|N|N|N|N|N| +2444500|AAAAAAAAENMEFCAA|1980-09-17|968|4212|324|1980|3|9|17|3|1980|324|4212|Wednesday|1980Q3|N|N|N|2444484|2444727|2444134|2444409|N|N|N|N|N| +2444501|AAAAAAAAFNMEFCAA|1980-09-18|968|4212|324|1980|4|9|18|3|1980|324|4212|Thursday|1980Q3|N|N|N|2444484|2444727|2444135|2444410|N|N|N|N|N| +2444502|AAAAAAAAGNMEFCAA|1980-09-19|968|4212|324|1980|5|9|19|3|1980|324|4212|Friday|1980Q3|N|Y|N|2444484|2444727|2444136|2444411|N|N|N|N|N| +2444503|AAAAAAAAHNMEFCAA|1980-09-20|968|4212|324|1980|6|9|20|3|1980|324|4212|Saturday|1980Q3|N|Y|N|2444484|2444727|2444137|2444412|N|N|N|N|N| +2444504|AAAAAAAAINMEFCAA|1980-09-21|968|4212|324|1980|0|9|21|3|1980|324|4212|Sunday|1980Q3|N|N|N|2444484|2444727|2444138|2444413|N|N|N|N|N| +2444505|AAAAAAAAJNMEFCAA|1980-09-22|968|4212|324|1980|1|9|22|3|1980|324|4212|Monday|1980Q3|N|N|N|2444484|2444727|2444139|2444414|N|N|N|N|N| +2444506|AAAAAAAAKNMEFCAA|1980-09-23|968|4213|324|1980|2|9|23|3|1980|324|4213|Tuesday|1980Q3|N|N|N|2444484|2444727|2444140|2444415|N|N|N|N|N| +2444507|AAAAAAAALNMEFCAA|1980-09-24|968|4213|324|1980|3|9|24|3|1980|324|4213|Wednesday|1980Q3|N|N|N|2444484|2444727|2444141|2444416|N|N|N|N|N| +2444508|AAAAAAAAMNMEFCAA|1980-09-25|968|4213|324|1980|4|9|25|3|1980|324|4213|Thursday|1980Q3|N|N|N|2444484|2444727|2444142|2444417|N|N|N|N|N| +2444509|AAAAAAAANNMEFCAA|1980-09-26|968|4213|324|1980|5|9|26|3|1980|324|4213|Friday|1980Q3|N|Y|N|2444484|2444727|2444143|2444418|N|N|N|N|N| +2444510|AAAAAAAAONMEFCAA|1980-09-27|968|4213|324|1980|6|9|27|3|1980|324|4213|Saturday|1980Q3|N|Y|N|2444484|2444727|2444144|2444419|N|N|N|N|N| +2444511|AAAAAAAAPNMEFCAA|1980-09-28|968|4213|324|1980|0|9|28|3|1980|324|4213|Sunday|1980Q3|N|N|N|2444484|2444727|2444145|2444420|N|N|N|N|N| +2444512|AAAAAAAAAOMEFCAA|1980-09-29|968|4213|324|1980|1|9|29|3|1980|324|4213|Monday|1980Q3|N|N|N|2444484|2444727|2444146|2444421|N|N|N|N|N| +2444513|AAAAAAAABOMEFCAA|1980-09-30|968|4214|324|1980|2|9|30|3|1980|324|4214|Tuesday|1980Q3|N|N|N|2444484|2444727|2444147|2444422|N|N|N|N|N| +2444514|AAAAAAAACOMEFCAA|1980-10-01|969|4214|324|1980|3|10|1|4|1980|324|4214|Wednesday|1980Q4|N|N|N|2444514|2444787|2444148|2444422|N|N|N|N|N| +2444515|AAAAAAAADOMEFCAA|1980-10-02|969|4214|324|1980|4|10|2|4|1980|324|4214|Thursday|1980Q4|N|N|N|2444514|2444787|2444149|2444423|N|N|N|N|N| +2444516|AAAAAAAAEOMEFCAA|1980-10-03|969|4214|324|1980|5|10|3|4|1980|324|4214|Friday|1980Q4|N|Y|N|2444514|2444787|2444150|2444424|N|N|N|N|N| +2444517|AAAAAAAAFOMEFCAA|1980-10-04|969|4214|324|1980|6|10|4|4|1980|324|4214|Saturday|1980Q4|N|Y|N|2444514|2444787|2444151|2444425|N|N|N|N|N| +2444518|AAAAAAAAGOMEFCAA|1980-10-05|969|4214|324|1980|0|10|5|4|1980|324|4214|Sunday|1980Q4|N|N|N|2444514|2444787|2444152|2444426|N|N|N|N|N| +2444519|AAAAAAAAHOMEFCAA|1980-10-06|969|4214|324|1980|1|10|6|4|1980|324|4214|Monday|1980Q4|N|N|N|2444514|2444787|2444153|2444427|N|N|N|N|N| +2444520|AAAAAAAAIOMEFCAA|1980-10-07|969|4215|324|1980|2|10|7|4|1980|324|4215|Tuesday|1980Q4|N|N|N|2444514|2444787|2444154|2444428|N|N|N|N|N| +2444521|AAAAAAAAJOMEFCAA|1980-10-08|969|4215|324|1980|3|10|8|4|1980|324|4215|Wednesday|1980Q4|N|N|N|2444514|2444787|2444155|2444429|N|N|N|N|N| +2444522|AAAAAAAAKOMEFCAA|1980-10-09|969|4215|324|1980|4|10|9|4|1980|324|4215|Thursday|1980Q4|N|N|N|2444514|2444787|2444156|2444430|N|N|N|N|N| +2444523|AAAAAAAALOMEFCAA|1980-10-10|969|4215|324|1980|5|10|10|4|1980|324|4215|Friday|1980Q4|N|Y|N|2444514|2444787|2444157|2444431|N|N|N|N|N| +2444524|AAAAAAAAMOMEFCAA|1980-10-11|969|4215|324|1980|6|10|11|4|1980|324|4215|Saturday|1980Q4|N|Y|N|2444514|2444787|2444158|2444432|N|N|N|N|N| +2444525|AAAAAAAANOMEFCAA|1980-10-12|969|4215|324|1980|0|10|12|4|1980|324|4215|Sunday|1980Q4|N|N|N|2444514|2444787|2444159|2444433|N|N|N|N|N| +2444526|AAAAAAAAOOMEFCAA|1980-10-13|969|4215|324|1980|1|10|13|4|1980|324|4215|Monday|1980Q4|N|N|N|2444514|2444787|2444160|2444434|N|N|N|N|N| +2444527|AAAAAAAAPOMEFCAA|1980-10-14|969|4216|324|1980|2|10|14|4|1980|324|4216|Tuesday|1980Q4|N|N|N|2444514|2444787|2444161|2444435|N|N|N|N|N| +2444528|AAAAAAAAAPMEFCAA|1980-10-15|969|4216|324|1980|3|10|15|4|1980|324|4216|Wednesday|1980Q4|N|N|N|2444514|2444787|2444162|2444436|N|N|N|N|N| +2444529|AAAAAAAABPMEFCAA|1980-10-16|969|4216|324|1980|4|10|16|4|1980|324|4216|Thursday|1980Q4|N|N|N|2444514|2444787|2444163|2444437|N|N|N|N|N| +2444530|AAAAAAAACPMEFCAA|1980-10-17|969|4216|324|1980|5|10|17|4|1980|324|4216|Friday|1980Q4|N|Y|N|2444514|2444787|2444164|2444438|N|N|N|N|N| +2444531|AAAAAAAADPMEFCAA|1980-10-18|969|4216|324|1980|6|10|18|4|1980|324|4216|Saturday|1980Q4|N|Y|N|2444514|2444787|2444165|2444439|N|N|N|N|N| +2444532|AAAAAAAAEPMEFCAA|1980-10-19|969|4216|324|1980|0|10|19|4|1980|324|4216|Sunday|1980Q4|N|N|N|2444514|2444787|2444166|2444440|N|N|N|N|N| +2444533|AAAAAAAAFPMEFCAA|1980-10-20|969|4216|324|1980|1|10|20|4|1980|324|4216|Monday|1980Q4|N|N|N|2444514|2444787|2444167|2444441|N|N|N|N|N| +2444534|AAAAAAAAGPMEFCAA|1980-10-21|969|4217|324|1980|2|10|21|4|1980|324|4217|Tuesday|1980Q4|N|N|N|2444514|2444787|2444168|2444442|N|N|N|N|N| +2444535|AAAAAAAAHPMEFCAA|1980-10-22|969|4217|324|1980|3|10|22|4|1980|324|4217|Wednesday|1980Q4|N|N|N|2444514|2444787|2444169|2444443|N|N|N|N|N| +2444536|AAAAAAAAIPMEFCAA|1980-10-23|969|4217|324|1980|4|10|23|4|1980|324|4217|Thursday|1980Q4|N|N|N|2444514|2444787|2444170|2444444|N|N|N|N|N| +2444537|AAAAAAAAJPMEFCAA|1980-10-24|969|4217|324|1980|5|10|24|4|1980|324|4217|Friday|1980Q4|N|Y|N|2444514|2444787|2444171|2444445|N|N|N|N|N| +2444538|AAAAAAAAKPMEFCAA|1980-10-25|969|4217|324|1980|6|10|25|4|1980|324|4217|Saturday|1980Q4|N|Y|N|2444514|2444787|2444172|2444446|N|N|N|N|N| +2444539|AAAAAAAALPMEFCAA|1980-10-26|969|4217|324|1980|0|10|26|4|1980|324|4217|Sunday|1980Q4|N|N|N|2444514|2444787|2444173|2444447|N|N|N|N|N| +2444540|AAAAAAAAMPMEFCAA|1980-10-27|969|4217|324|1980|1|10|27|4|1980|324|4217|Monday|1980Q4|N|N|N|2444514|2444787|2444174|2444448|N|N|N|N|N| +2444541|AAAAAAAANPMEFCAA|1980-10-28|969|4218|324|1980|2|10|28|4|1980|324|4218|Tuesday|1980Q4|N|N|N|2444514|2444787|2444175|2444449|N|N|N|N|N| +2444542|AAAAAAAAOPMEFCAA|1980-10-29|969|4218|324|1980|3|10|29|4|1980|324|4218|Wednesday|1980Q4|N|N|N|2444514|2444787|2444176|2444450|N|N|N|N|N| +2444543|AAAAAAAAPPMEFCAA|1980-10-30|969|4218|324|1980|4|10|30|4|1980|324|4218|Thursday|1980Q4|N|N|N|2444514|2444787|2444177|2444451|N|N|N|N|N| +2444544|AAAAAAAAAANEFCAA|1980-10-31|969|4218|324|1980|5|10|31|4|1980|324|4218|Friday|1980Q4|N|Y|N|2444514|2444787|2444178|2444452|N|N|N|N|N| +2444545|AAAAAAAABANEFCAA|1980-11-01|970|4218|324|1980|6|11|1|4|1980|324|4218|Saturday|1980Q4|N|Y|N|2444545|2444849|2444179|2444453|N|N|N|N|N| +2444546|AAAAAAAACANEFCAA|1980-11-02|970|4218|324|1980|0|11|2|4|1980|324|4218|Sunday|1980Q4|N|N|N|2444545|2444849|2444180|2444454|N|N|N|N|N| +2444547|AAAAAAAADANEFCAA|1980-11-03|970|4218|324|1980|1|11|3|4|1980|324|4218|Monday|1980Q4|N|N|N|2444545|2444849|2444181|2444455|N|N|N|N|N| +2444548|AAAAAAAAEANEFCAA|1980-11-04|970|4219|324|1980|2|11|4|4|1980|324|4219|Tuesday|1980Q4|N|N|N|2444545|2444849|2444182|2444456|N|N|N|N|N| +2444549|AAAAAAAAFANEFCAA|1980-11-05|970|4219|324|1980|3|11|5|4|1980|324|4219|Wednesday|1980Q4|N|N|N|2444545|2444849|2444183|2444457|N|N|N|N|N| +2444550|AAAAAAAAGANEFCAA|1980-11-06|970|4219|324|1980|4|11|6|4|1980|324|4219|Thursday|1980Q4|N|N|N|2444545|2444849|2444184|2444458|N|N|N|N|N| +2444551|AAAAAAAAHANEFCAA|1980-11-07|970|4219|324|1980|5|11|7|4|1980|324|4219|Friday|1980Q4|N|Y|N|2444545|2444849|2444185|2444459|N|N|N|N|N| +2444552|AAAAAAAAIANEFCAA|1980-11-08|970|4219|324|1980|6|11|8|4|1980|324|4219|Saturday|1980Q4|N|Y|N|2444545|2444849|2444186|2444460|N|N|N|N|N| +2444553|AAAAAAAAJANEFCAA|1980-11-09|970|4219|324|1980|0|11|9|4|1980|324|4219|Sunday|1980Q4|N|N|N|2444545|2444849|2444187|2444461|N|N|N|N|N| +2444554|AAAAAAAAKANEFCAA|1980-11-10|970|4219|324|1980|1|11|10|4|1980|324|4219|Monday|1980Q4|N|N|N|2444545|2444849|2444188|2444462|N|N|N|N|N| +2444555|AAAAAAAALANEFCAA|1980-11-11|970|4220|324|1980|2|11|11|4|1980|324|4220|Tuesday|1980Q4|N|N|N|2444545|2444849|2444189|2444463|N|N|N|N|N| +2444556|AAAAAAAAMANEFCAA|1980-11-12|970|4220|324|1980|3|11|12|4|1980|324|4220|Wednesday|1980Q4|N|N|N|2444545|2444849|2444190|2444464|N|N|N|N|N| +2444557|AAAAAAAANANEFCAA|1980-11-13|970|4220|324|1980|4|11|13|4|1980|324|4220|Thursday|1980Q4|N|N|N|2444545|2444849|2444191|2444465|N|N|N|N|N| +2444558|AAAAAAAAOANEFCAA|1980-11-14|970|4220|324|1980|5|11|14|4|1980|324|4220|Friday|1980Q4|N|Y|N|2444545|2444849|2444192|2444466|N|N|N|N|N| +2444559|AAAAAAAAPANEFCAA|1980-11-15|970|4220|324|1980|6|11|15|4|1980|324|4220|Saturday|1980Q4|N|Y|N|2444545|2444849|2444193|2444467|N|N|N|N|N| +2444560|AAAAAAAAABNEFCAA|1980-11-16|970|4220|324|1980|0|11|16|4|1980|324|4220|Sunday|1980Q4|N|N|N|2444545|2444849|2444194|2444468|N|N|N|N|N| +2444561|AAAAAAAABBNEFCAA|1980-11-17|970|4220|324|1980|1|11|17|4|1980|324|4220|Monday|1980Q4|N|N|N|2444545|2444849|2444195|2444469|N|N|N|N|N| +2444562|AAAAAAAACBNEFCAA|1980-11-18|970|4221|324|1980|2|11|18|4|1980|324|4221|Tuesday|1980Q4|N|N|N|2444545|2444849|2444196|2444470|N|N|N|N|N| +2444563|AAAAAAAADBNEFCAA|1980-11-19|970|4221|324|1980|3|11|19|4|1980|324|4221|Wednesday|1980Q4|N|N|N|2444545|2444849|2444197|2444471|N|N|N|N|N| +2444564|AAAAAAAAEBNEFCAA|1980-11-20|970|4221|324|1980|4|11|20|4|1980|324|4221|Thursday|1980Q4|N|N|N|2444545|2444849|2444198|2444472|N|N|N|N|N| +2444565|AAAAAAAAFBNEFCAA|1980-11-21|970|4221|324|1980|5|11|21|4|1980|324|4221|Friday|1980Q4|N|Y|N|2444545|2444849|2444199|2444473|N|N|N|N|N| +2444566|AAAAAAAAGBNEFCAA|1980-11-22|970|4221|324|1980|6|11|22|4|1980|324|4221|Saturday|1980Q4|N|Y|N|2444545|2444849|2444200|2444474|N|N|N|N|N| +2444567|AAAAAAAAHBNEFCAA|1980-11-23|970|4221|324|1980|0|11|23|4|1980|324|4221|Sunday|1980Q4|N|N|N|2444545|2444849|2444201|2444475|N|N|N|N|N| +2444568|AAAAAAAAIBNEFCAA|1980-11-24|970|4221|324|1980|1|11|24|4|1980|324|4221|Monday|1980Q4|N|N|N|2444545|2444849|2444202|2444476|N|N|N|N|N| +2444569|AAAAAAAAJBNEFCAA|1980-11-25|970|4222|324|1980|2|11|25|4|1980|324|4222|Tuesday|1980Q4|N|N|N|2444545|2444849|2444203|2444477|N|N|N|N|N| +2444570|AAAAAAAAKBNEFCAA|1980-11-26|970|4222|324|1980|3|11|26|4|1980|324|4222|Wednesday|1980Q4|N|N|N|2444545|2444849|2444204|2444478|N|N|N|N|N| +2444571|AAAAAAAALBNEFCAA|1980-11-27|970|4222|324|1980|4|11|27|4|1980|324|4222|Thursday|1980Q4|N|N|N|2444545|2444849|2444205|2444479|N|N|N|N|N| +2444572|AAAAAAAAMBNEFCAA|1980-11-28|970|4222|324|1980|5|11|28|4|1980|324|4222|Friday|1980Q4|N|Y|N|2444545|2444849|2444206|2444480|N|N|N|N|N| +2444573|AAAAAAAANBNEFCAA|1980-11-29|970|4222|324|1980|6|11|29|4|1980|324|4222|Saturday|1980Q4|N|Y|N|2444545|2444849|2444207|2444481|N|N|N|N|N| +2444574|AAAAAAAAOBNEFCAA|1980-11-30|970|4222|324|1980|0|11|30|4|1980|324|4222|Sunday|1980Q4|N|N|N|2444545|2444849|2444208|2444482|N|N|N|N|N| +2444575|AAAAAAAAPBNEFCAA|1980-12-01|971|4222|325|1980|1|12|1|4|1980|325|4222|Monday|1980Q4|N|N|N|2444575|2444909|2444209|2444483|N|N|N|N|N| +2444576|AAAAAAAAACNEFCAA|1980-12-02|971|4223|325|1980|2|12|2|4|1980|325|4223|Tuesday|1980Q4|N|N|N|2444575|2444909|2444210|2444484|N|N|N|N|N| +2444577|AAAAAAAABCNEFCAA|1980-12-03|971|4223|325|1980|3|12|3|4|1980|325|4223|Wednesday|1980Q4|N|N|N|2444575|2444909|2444211|2444485|N|N|N|N|N| +2444578|AAAAAAAACCNEFCAA|1980-12-04|971|4223|325|1980|4|12|4|4|1980|325|4223|Thursday|1980Q4|N|N|N|2444575|2444909|2444212|2444486|N|N|N|N|N| +2444579|AAAAAAAADCNEFCAA|1980-12-05|971|4223|325|1980|5|12|5|4|1980|325|4223|Friday|1980Q4|N|Y|N|2444575|2444909|2444213|2444487|N|N|N|N|N| +2444580|AAAAAAAAECNEFCAA|1980-12-06|971|4223|325|1980|6|12|6|4|1980|325|4223|Saturday|1980Q4|N|Y|N|2444575|2444909|2444214|2444488|N|N|N|N|N| +2444581|AAAAAAAAFCNEFCAA|1980-12-07|971|4223|325|1980|0|12|7|4|1980|325|4223|Sunday|1980Q4|N|N|N|2444575|2444909|2444215|2444489|N|N|N|N|N| +2444582|AAAAAAAAGCNEFCAA|1980-12-08|971|4223|325|1980|1|12|8|4|1980|325|4223|Monday|1980Q4|N|N|N|2444575|2444909|2444216|2444490|N|N|N|N|N| +2444583|AAAAAAAAHCNEFCAA|1980-12-09|971|4224|325|1980|2|12|9|4|1980|325|4224|Tuesday|1980Q4|N|N|N|2444575|2444909|2444217|2444491|N|N|N|N|N| +2444584|AAAAAAAAICNEFCAA|1980-12-10|971|4224|325|1980|3|12|10|4|1980|325|4224|Wednesday|1980Q4|N|N|N|2444575|2444909|2444218|2444492|N|N|N|N|N| +2444585|AAAAAAAAJCNEFCAA|1980-12-11|971|4224|325|1980|4|12|11|4|1980|325|4224|Thursday|1980Q4|N|N|N|2444575|2444909|2444219|2444493|N|N|N|N|N| +2444586|AAAAAAAAKCNEFCAA|1980-12-12|971|4224|325|1980|5|12|12|4|1980|325|4224|Friday|1980Q4|N|Y|N|2444575|2444909|2444220|2444494|N|N|N|N|N| +2444587|AAAAAAAALCNEFCAA|1980-12-13|971|4224|325|1980|6|12|13|4|1980|325|4224|Saturday|1980Q4|N|Y|N|2444575|2444909|2444221|2444495|N|N|N|N|N| +2444588|AAAAAAAAMCNEFCAA|1980-12-14|971|4224|325|1980|0|12|14|4|1980|325|4224|Sunday|1980Q4|N|N|N|2444575|2444909|2444222|2444496|N|N|N|N|N| +2444589|AAAAAAAANCNEFCAA|1980-12-15|971|4224|325|1980|1|12|15|4|1980|325|4224|Monday|1980Q4|N|N|N|2444575|2444909|2444223|2444497|N|N|N|N|N| +2444590|AAAAAAAAOCNEFCAA|1980-12-16|971|4225|325|1980|2|12|16|4|1980|325|4225|Tuesday|1980Q4|N|N|N|2444575|2444909|2444224|2444498|N|N|N|N|N| +2444591|AAAAAAAAPCNEFCAA|1980-12-17|971|4225|325|1980|3|12|17|4|1980|325|4225|Wednesday|1980Q4|N|N|N|2444575|2444909|2444225|2444499|N|N|N|N|N| +2444592|AAAAAAAAADNEFCAA|1980-12-18|971|4225|325|1980|4|12|18|4|1980|325|4225|Thursday|1980Q4|N|N|N|2444575|2444909|2444226|2444500|N|N|N|N|N| +2444593|AAAAAAAABDNEFCAA|1980-12-19|971|4225|325|1980|5|12|19|4|1980|325|4225|Friday|1980Q4|N|Y|N|2444575|2444909|2444227|2444501|N|N|N|N|N| +2444594|AAAAAAAACDNEFCAA|1980-12-20|971|4225|325|1980|6|12|20|4|1980|325|4225|Saturday|1980Q4|N|Y|N|2444575|2444909|2444228|2444502|N|N|N|N|N| +2444595|AAAAAAAADDNEFCAA|1980-12-21|971|4225|325|1980|0|12|21|4|1980|325|4225|Sunday|1980Q4|N|N|N|2444575|2444909|2444229|2444503|N|N|N|N|N| +2444596|AAAAAAAAEDNEFCAA|1980-12-22|971|4225|325|1980|1|12|22|4|1980|325|4225|Monday|1980Q4|N|N|N|2444575|2444909|2444230|2444504|N|N|N|N|N| +2444597|AAAAAAAAFDNEFCAA|1980-12-23|971|4226|325|1980|2|12|23|4|1980|325|4226|Tuesday|1980Q4|N|N|N|2444575|2444909|2444231|2444505|N|N|N|N|N| +2444598|AAAAAAAAGDNEFCAA|1980-12-24|971|4226|325|1980|3|12|24|4|1980|325|4226|Wednesday|1980Q4|N|N|N|2444575|2444909|2444232|2444506|N|N|N|N|N| +2444599|AAAAAAAAHDNEFCAA|1980-12-25|971|4226|325|1980|4|12|25|4|1980|325|4226|Thursday|1980Q4|Y|N|N|2444575|2444909|2444233|2444507|N|N|N|N|N| +2444600|AAAAAAAAIDNEFCAA|1980-12-26|971|4226|325|1980|5|12|26|4|1980|325|4226|Friday|1980Q4|N|Y|Y|2444575|2444909|2444234|2444508|N|N|N|N|N| +2444601|AAAAAAAAJDNEFCAA|1980-12-27|971|4226|325|1980|6|12|27|4|1980|325|4226|Saturday|1980Q4|N|Y|N|2444575|2444909|2444235|2444509|N|N|N|N|N| +2444602|AAAAAAAAKDNEFCAA|1980-12-28|971|4226|325|1980|0|12|28|4|1980|325|4226|Sunday|1980Q4|N|N|N|2444575|2444909|2444236|2444510|N|N|N|N|N| +2444603|AAAAAAAALDNEFCAA|1980-12-29|971|4226|325|1980|1|12|29|4|1980|325|4226|Monday|1980Q4|N|N|N|2444575|2444909|2444237|2444511|N|N|N|N|N| +2444604|AAAAAAAAMDNEFCAA|1980-12-30|971|4227|325|1980|2|12|30|4|1980|325|4227|Tuesday|1980Q4|N|N|N|2444575|2444909|2444238|2444512|N|N|N|N|N| +2444605|AAAAAAAANDNEFCAA|1980-12-31|971|4227|325|1980|3|12|31|4|1980|325|4227|Wednesday|1980Q4|Y|N|N|2444575|2444909|2444239|2444513|N|N|N|N|N| +2444606|AAAAAAAAODNEFCAA|1981-01-01|972|4227|325|1981|4|1|1|1|1981|325|4227|Thursday|1981Q1|Y|N|Y|2444606|2444605|2444240|2444514|N|N|N|N|N| +2444607|AAAAAAAAPDNEFCAA|1981-01-02|972|4227|325|1981|5|1|2|1|1981|325|4227|Friday|1981Q1|N|Y|Y|2444606|2444605|2444241|2444515|N|N|N|N|N| +2444608|AAAAAAAAAENEFCAA|1981-01-03|972|4227|325|1981|6|1|3|1|1981|325|4227|Saturday|1981Q1|N|Y|N|2444606|2444605|2444242|2444516|N|N|N|N|N| +2444609|AAAAAAAABENEFCAA|1981-01-04|972|4227|325|1981|0|1|4|1|1981|325|4227|Sunday|1981Q1|N|N|N|2444606|2444605|2444243|2444517|N|N|N|N|N| +2444610|AAAAAAAACENEFCAA|1981-01-05|972|4227|325|1981|1|1|5|1|1981|325|4227|Monday|1981Q1|N|N|N|2444606|2444605|2444244|2444518|N|N|N|N|N| +2444611|AAAAAAAADENEFCAA|1981-01-06|972|4228|325|1981|2|1|6|1|1981|325|4228|Tuesday|1981Q1|N|N|N|2444606|2444605|2444245|2444519|N|N|N|N|N| +2444612|AAAAAAAAEENEFCAA|1981-01-07|972|4228|325|1981|3|1|7|1|1981|325|4228|Wednesday|1981Q1|N|N|N|2444606|2444605|2444246|2444520|N|N|N|N|N| +2444613|AAAAAAAAFENEFCAA|1981-01-08|972|4228|325|1981|4|1|8|1|1981|325|4228|Thursday|1981Q1|N|N|N|2444606|2444605|2444247|2444521|N|N|N|N|N| +2444614|AAAAAAAAGENEFCAA|1981-01-09|972|4228|325|1981|5|1|9|1|1981|325|4228|Friday|1981Q1|N|Y|N|2444606|2444605|2444248|2444522|N|N|N|N|N| +2444615|AAAAAAAAHENEFCAA|1981-01-10|972|4228|325|1981|6|1|10|1|1981|325|4228|Saturday|1981Q1|N|Y|N|2444606|2444605|2444249|2444523|N|N|N|N|N| +2444616|AAAAAAAAIENEFCAA|1981-01-11|972|4228|325|1981|0|1|11|1|1981|325|4228|Sunday|1981Q1|N|N|N|2444606|2444605|2444250|2444524|N|N|N|N|N| +2444617|AAAAAAAAJENEFCAA|1981-01-12|972|4228|325|1981|1|1|12|1|1981|325|4228|Monday|1981Q1|N|N|N|2444606|2444605|2444251|2444525|N|N|N|N|N| +2444618|AAAAAAAAKENEFCAA|1981-01-13|972|4229|325|1981|2|1|13|1|1981|325|4229|Tuesday|1981Q1|N|N|N|2444606|2444605|2444252|2444526|N|N|N|N|N| +2444619|AAAAAAAALENEFCAA|1981-01-14|972|4229|325|1981|3|1|14|1|1981|325|4229|Wednesday|1981Q1|N|N|N|2444606|2444605|2444253|2444527|N|N|N|N|N| +2444620|AAAAAAAAMENEFCAA|1981-01-15|972|4229|325|1981|4|1|15|1|1981|325|4229|Thursday|1981Q1|N|N|N|2444606|2444605|2444254|2444528|N|N|N|N|N| +2444621|AAAAAAAANENEFCAA|1981-01-16|972|4229|325|1981|5|1|16|1|1981|325|4229|Friday|1981Q1|N|Y|N|2444606|2444605|2444255|2444529|N|N|N|N|N| +2444622|AAAAAAAAOENEFCAA|1981-01-17|972|4229|325|1981|6|1|17|1|1981|325|4229|Saturday|1981Q1|N|Y|N|2444606|2444605|2444256|2444530|N|N|N|N|N| +2444623|AAAAAAAAPENEFCAA|1981-01-18|972|4229|325|1981|0|1|18|1|1981|325|4229|Sunday|1981Q1|N|N|N|2444606|2444605|2444257|2444531|N|N|N|N|N| +2444624|AAAAAAAAAFNEFCAA|1981-01-19|972|4229|325|1981|1|1|19|1|1981|325|4229|Monday|1981Q1|N|N|N|2444606|2444605|2444258|2444532|N|N|N|N|N| +2444625|AAAAAAAABFNEFCAA|1981-01-20|972|4230|325|1981|2|1|20|1|1981|325|4230|Tuesday|1981Q1|N|N|N|2444606|2444605|2444259|2444533|N|N|N|N|N| +2444626|AAAAAAAACFNEFCAA|1981-01-21|972|4230|325|1981|3|1|21|1|1981|325|4230|Wednesday|1981Q1|N|N|N|2444606|2444605|2444260|2444534|N|N|N|N|N| +2444627|AAAAAAAADFNEFCAA|1981-01-22|972|4230|325|1981|4|1|22|1|1981|325|4230|Thursday|1981Q1|N|N|N|2444606|2444605|2444261|2444535|N|N|N|N|N| +2444628|AAAAAAAAEFNEFCAA|1981-01-23|972|4230|325|1981|5|1|23|1|1981|325|4230|Friday|1981Q1|N|Y|N|2444606|2444605|2444262|2444536|N|N|N|N|N| +2444629|AAAAAAAAFFNEFCAA|1981-01-24|972|4230|325|1981|6|1|24|1|1981|325|4230|Saturday|1981Q1|N|Y|N|2444606|2444605|2444263|2444537|N|N|N|N|N| +2444630|AAAAAAAAGFNEFCAA|1981-01-25|972|4230|325|1981|0|1|25|1|1981|325|4230|Sunday|1981Q1|N|N|N|2444606|2444605|2444264|2444538|N|N|N|N|N| +2444631|AAAAAAAAHFNEFCAA|1981-01-26|972|4230|325|1981|1|1|26|1|1981|325|4230|Monday|1981Q1|N|N|N|2444606|2444605|2444265|2444539|N|N|N|N|N| +2444632|AAAAAAAAIFNEFCAA|1981-01-27|972|4231|325|1981|2|1|27|1|1981|325|4231|Tuesday|1981Q1|N|N|N|2444606|2444605|2444266|2444540|N|N|N|N|N| +2444633|AAAAAAAAJFNEFCAA|1981-01-28|972|4231|325|1981|3|1|28|1|1981|325|4231|Wednesday|1981Q1|N|N|N|2444606|2444605|2444267|2444541|N|N|N|N|N| +2444634|AAAAAAAAKFNEFCAA|1981-01-29|972|4231|325|1981|4|1|29|1|1981|325|4231|Thursday|1981Q1|N|N|N|2444606|2444605|2444268|2444542|N|N|N|N|N| +2444635|AAAAAAAALFNEFCAA|1981-01-30|972|4231|325|1981|5|1|30|1|1981|325|4231|Friday|1981Q1|N|Y|N|2444606|2444605|2444269|2444543|N|N|N|N|N| +2444636|AAAAAAAAMFNEFCAA|1981-01-31|972|4231|325|1981|6|1|31|1|1981|325|4231|Saturday|1981Q1|N|Y|N|2444606|2444605|2444270|2444544|N|N|N|N|N| +2444637|AAAAAAAANFNEFCAA|1981-02-01|973|4231|325|1981|0|2|1|1|1981|325|4231|Sunday|1981Q1|N|N|N|2444637|2444667|2444271|2444545|N|N|N|N|N| +2444638|AAAAAAAAOFNEFCAA|1981-02-02|973|4231|325|1981|1|2|2|1|1981|325|4231|Monday|1981Q1|N|N|N|2444637|2444667|2444272|2444546|N|N|N|N|N| +2444639|AAAAAAAAPFNEFCAA|1981-02-03|973|4232|325|1981|2|2|3|1|1981|325|4232|Tuesday|1981Q1|N|N|N|2444637|2444667|2444273|2444547|N|N|N|N|N| +2444640|AAAAAAAAAGNEFCAA|1981-02-04|973|4232|325|1981|3|2|4|1|1981|325|4232|Wednesday|1981Q1|N|N|N|2444637|2444667|2444274|2444548|N|N|N|N|N| +2444641|AAAAAAAABGNEFCAA|1981-02-05|973|4232|325|1981|4|2|5|1|1981|325|4232|Thursday|1981Q1|N|N|N|2444637|2444667|2444275|2444549|N|N|N|N|N| +2444642|AAAAAAAACGNEFCAA|1981-02-06|973|4232|325|1981|5|2|6|1|1981|325|4232|Friday|1981Q1|N|Y|N|2444637|2444667|2444276|2444550|N|N|N|N|N| +2444643|AAAAAAAADGNEFCAA|1981-02-07|973|4232|325|1981|6|2|7|1|1981|325|4232|Saturday|1981Q1|N|Y|N|2444637|2444667|2444277|2444551|N|N|N|N|N| +2444644|AAAAAAAAEGNEFCAA|1981-02-08|973|4232|325|1981|0|2|8|1|1981|325|4232|Sunday|1981Q1|N|N|N|2444637|2444667|2444278|2444552|N|N|N|N|N| +2444645|AAAAAAAAFGNEFCAA|1981-02-09|973|4232|325|1981|1|2|9|1|1981|325|4232|Monday|1981Q1|N|N|N|2444637|2444667|2444279|2444553|N|N|N|N|N| +2444646|AAAAAAAAGGNEFCAA|1981-02-10|973|4233|325|1981|2|2|10|1|1981|325|4233|Tuesday|1981Q1|N|N|N|2444637|2444667|2444280|2444554|N|N|N|N|N| +2444647|AAAAAAAAHGNEFCAA|1981-02-11|973|4233|325|1981|3|2|11|1|1981|325|4233|Wednesday|1981Q1|N|N|N|2444637|2444667|2444281|2444555|N|N|N|N|N| +2444648|AAAAAAAAIGNEFCAA|1981-02-12|973|4233|325|1981|4|2|12|1|1981|325|4233|Thursday|1981Q1|N|N|N|2444637|2444667|2444282|2444556|N|N|N|N|N| +2444649|AAAAAAAAJGNEFCAA|1981-02-13|973|4233|325|1981|5|2|13|1|1981|325|4233|Friday|1981Q1|N|Y|N|2444637|2444667|2444283|2444557|N|N|N|N|N| +2444650|AAAAAAAAKGNEFCAA|1981-02-14|973|4233|325|1981|6|2|14|1|1981|325|4233|Saturday|1981Q1|N|Y|N|2444637|2444667|2444284|2444558|N|N|N|N|N| +2444651|AAAAAAAALGNEFCAA|1981-02-15|973|4233|325|1981|0|2|15|1|1981|325|4233|Sunday|1981Q1|N|N|N|2444637|2444667|2444285|2444559|N|N|N|N|N| +2444652|AAAAAAAAMGNEFCAA|1981-02-16|973|4233|325|1981|1|2|16|1|1981|325|4233|Monday|1981Q1|N|N|N|2444637|2444667|2444286|2444560|N|N|N|N|N| +2444653|AAAAAAAANGNEFCAA|1981-02-17|973|4234|325|1981|2|2|17|1|1981|325|4234|Tuesday|1981Q1|N|N|N|2444637|2444667|2444287|2444561|N|N|N|N|N| +2444654|AAAAAAAAOGNEFCAA|1981-02-18|973|4234|325|1981|3|2|18|1|1981|325|4234|Wednesday|1981Q1|N|N|N|2444637|2444667|2444288|2444562|N|N|N|N|N| +2444655|AAAAAAAAPGNEFCAA|1981-02-19|973|4234|325|1981|4|2|19|1|1981|325|4234|Thursday|1981Q1|N|N|N|2444637|2444667|2444289|2444563|N|N|N|N|N| +2444656|AAAAAAAAAHNEFCAA|1981-02-20|973|4234|325|1981|5|2|20|1|1981|325|4234|Friday|1981Q1|N|Y|N|2444637|2444667|2444290|2444564|N|N|N|N|N| +2444657|AAAAAAAABHNEFCAA|1981-02-21|973|4234|325|1981|6|2|21|1|1981|325|4234|Saturday|1981Q1|N|Y|N|2444637|2444667|2444291|2444565|N|N|N|N|N| +2444658|AAAAAAAACHNEFCAA|1981-02-22|973|4234|325|1981|0|2|22|1|1981|325|4234|Sunday|1981Q1|N|N|N|2444637|2444667|2444292|2444566|N|N|N|N|N| +2444659|AAAAAAAADHNEFCAA|1981-02-23|973|4234|325|1981|1|2|23|1|1981|325|4234|Monday|1981Q1|N|N|N|2444637|2444667|2444293|2444567|N|N|N|N|N| +2444660|AAAAAAAAEHNEFCAA|1981-02-24|973|4235|325|1981|2|2|24|1|1981|325|4235|Tuesday|1981Q1|N|N|N|2444637|2444667|2444294|2444568|N|N|N|N|N| +2444661|AAAAAAAAFHNEFCAA|1981-02-25|973|4235|325|1981|3|2|25|1|1981|325|4235|Wednesday|1981Q1|N|N|N|2444637|2444667|2444295|2444569|N|N|N|N|N| +2444662|AAAAAAAAGHNEFCAA|1981-02-26|973|4235|325|1981|4|2|26|1|1981|325|4235|Thursday|1981Q1|N|N|N|2444637|2444667|2444296|2444570|N|N|N|N|N| +2444663|AAAAAAAAHHNEFCAA|1981-02-27|973|4235|325|1981|5|2|27|1|1981|325|4235|Friday|1981Q1|N|Y|N|2444637|2444667|2444297|2444571|N|N|N|N|N| +2444664|AAAAAAAAIHNEFCAA|1981-02-28|973|4235|325|1981|6|2|28|1|1981|325|4235|Saturday|1981Q1|N|Y|N|2444637|2444667|2444298|2444572|N|N|N|N|N| +2444665|AAAAAAAAJHNEFCAA|1981-03-01|974|4235|326|1981|0|3|1|1|1981|326|4235|Sunday|1981Q1|N|N|N|2444665|2444723|2444300|2444573|N|N|N|N|N| +2444666|AAAAAAAAKHNEFCAA|1981-03-02|974|4235|326|1981|1|3|2|1|1981|326|4235|Monday|1981Q1|N|N|N|2444665|2444723|2444301|2444574|N|N|N|N|N| +2444667|AAAAAAAALHNEFCAA|1981-03-03|974|4236|326|1981|2|3|3|1|1981|326|4236|Tuesday|1981Q1|N|N|N|2444665|2444723|2444302|2444575|N|N|N|N|N| +2444668|AAAAAAAAMHNEFCAA|1981-03-04|974|4236|326|1981|3|3|4|1|1981|326|4236|Wednesday|1981Q1|N|N|N|2444665|2444723|2444303|2444576|N|N|N|N|N| +2444669|AAAAAAAANHNEFCAA|1981-03-05|974|4236|326|1981|4|3|5|1|1981|326|4236|Thursday|1981Q1|N|N|N|2444665|2444723|2444304|2444577|N|N|N|N|N| +2444670|AAAAAAAAOHNEFCAA|1981-03-06|974|4236|326|1981|5|3|6|1|1981|326|4236|Friday|1981Q1|N|Y|N|2444665|2444723|2444305|2444578|N|N|N|N|N| +2444671|AAAAAAAAPHNEFCAA|1981-03-07|974|4236|326|1981|6|3|7|1|1981|326|4236|Saturday|1981Q1|N|Y|N|2444665|2444723|2444306|2444579|N|N|N|N|N| +2444672|AAAAAAAAAINEFCAA|1981-03-08|974|4236|326|1981|0|3|8|1|1981|326|4236|Sunday|1981Q1|N|N|N|2444665|2444723|2444307|2444580|N|N|N|N|N| +2444673|AAAAAAAABINEFCAA|1981-03-09|974|4236|326|1981|1|3|9|1|1981|326|4236|Monday|1981Q1|N|N|N|2444665|2444723|2444308|2444581|N|N|N|N|N| +2444674|AAAAAAAACINEFCAA|1981-03-10|974|4237|326|1981|2|3|10|1|1981|326|4237|Tuesday|1981Q1|N|N|N|2444665|2444723|2444309|2444582|N|N|N|N|N| +2444675|AAAAAAAADINEFCAA|1981-03-11|974|4237|326|1981|3|3|11|1|1981|326|4237|Wednesday|1981Q1|N|N|N|2444665|2444723|2444310|2444583|N|N|N|N|N| +2444676|AAAAAAAAEINEFCAA|1981-03-12|974|4237|326|1981|4|3|12|1|1981|326|4237|Thursday|1981Q1|N|N|N|2444665|2444723|2444311|2444584|N|N|N|N|N| +2444677|AAAAAAAAFINEFCAA|1981-03-13|974|4237|326|1981|5|3|13|1|1981|326|4237|Friday|1981Q1|N|Y|N|2444665|2444723|2444312|2444585|N|N|N|N|N| +2444678|AAAAAAAAGINEFCAA|1981-03-14|974|4237|326|1981|6|3|14|1|1981|326|4237|Saturday|1981Q1|N|Y|N|2444665|2444723|2444313|2444586|N|N|N|N|N| +2444679|AAAAAAAAHINEFCAA|1981-03-15|974|4237|326|1981|0|3|15|1|1981|326|4237|Sunday|1981Q1|N|N|N|2444665|2444723|2444314|2444587|N|N|N|N|N| +2444680|AAAAAAAAIINEFCAA|1981-03-16|974|4237|326|1981|1|3|16|1|1981|326|4237|Monday|1981Q1|N|N|N|2444665|2444723|2444315|2444588|N|N|N|N|N| +2444681|AAAAAAAAJINEFCAA|1981-03-17|974|4238|326|1981|2|3|17|1|1981|326|4238|Tuesday|1981Q1|N|N|N|2444665|2444723|2444316|2444589|N|N|N|N|N| +2444682|AAAAAAAAKINEFCAA|1981-03-18|974|4238|326|1981|3|3|18|1|1981|326|4238|Wednesday|1981Q1|N|N|N|2444665|2444723|2444317|2444590|N|N|N|N|N| +2444683|AAAAAAAALINEFCAA|1981-03-19|974|4238|326|1981|4|3|19|1|1981|326|4238|Thursday|1981Q1|N|N|N|2444665|2444723|2444318|2444591|N|N|N|N|N| +2444684|AAAAAAAAMINEFCAA|1981-03-20|974|4238|326|1981|5|3|20|1|1981|326|4238|Friday|1981Q1|N|Y|N|2444665|2444723|2444319|2444592|N|N|N|N|N| +2444685|AAAAAAAANINEFCAA|1981-03-21|974|4238|326|1981|6|3|21|1|1981|326|4238|Saturday|1981Q1|N|Y|N|2444665|2444723|2444320|2444593|N|N|N|N|N| +2444686|AAAAAAAAOINEFCAA|1981-03-22|974|4238|326|1981|0|3|22|1|1981|326|4238|Sunday|1981Q1|N|N|N|2444665|2444723|2444321|2444594|N|N|N|N|N| +2444687|AAAAAAAAPINEFCAA|1981-03-23|974|4238|326|1981|1|3|23|1|1981|326|4238|Monday|1981Q1|N|N|N|2444665|2444723|2444322|2444595|N|N|N|N|N| +2444688|AAAAAAAAAJNEFCAA|1981-03-24|974|4239|326|1981|2|3|24|1|1981|326|4239|Tuesday|1981Q1|N|N|N|2444665|2444723|2444323|2444596|N|N|N|N|N| +2444689|AAAAAAAABJNEFCAA|1981-03-25|974|4239|326|1981|3|3|25|1|1981|326|4239|Wednesday|1981Q1|N|N|N|2444665|2444723|2444324|2444597|N|N|N|N|N| +2444690|AAAAAAAACJNEFCAA|1981-03-26|974|4239|326|1981|4|3|26|1|1981|326|4239|Thursday|1981Q1|N|N|N|2444665|2444723|2444325|2444598|N|N|N|N|N| +2444691|AAAAAAAADJNEFCAA|1981-03-27|974|4239|326|1981|5|3|27|1|1981|326|4239|Friday|1981Q1|N|Y|N|2444665|2444723|2444326|2444599|N|N|N|N|N| +2444692|AAAAAAAAEJNEFCAA|1981-03-28|974|4239|326|1981|6|3|28|1|1981|326|4239|Saturday|1981Q1|N|Y|N|2444665|2444723|2444327|2444600|N|N|N|N|N| +2444693|AAAAAAAAFJNEFCAA|1981-03-29|974|4239|326|1981|0|3|29|1|1981|326|4239|Sunday|1981Q1|N|N|N|2444665|2444723|2444328|2444601|N|N|N|N|N| +2444694|AAAAAAAAGJNEFCAA|1981-03-30|974|4239|326|1981|1|3|30|1|1981|326|4239|Monday|1981Q1|N|N|N|2444665|2444723|2444329|2444602|N|N|N|N|N| +2444695|AAAAAAAAHJNEFCAA|1981-03-31|974|4240|326|1981|2|3|31|1|1981|326|4240|Tuesday|1981Q1|N|N|N|2444665|2444723|2444330|2444603|N|N|N|N|N| +2444696|AAAAAAAAIJNEFCAA|1981-04-01|975|4240|326|1981|3|4|1|1|1981|326|4240|Wednesday|1981Q1|N|N|N|2444696|2444785|2444331|2444606|N|N|N|N|N| +2444697|AAAAAAAAJJNEFCAA|1981-04-02|975|4240|326|1981|4|4|2|2|1981|326|4240|Thursday|1981Q2|N|N|N|2444696|2444785|2444332|2444607|N|N|N|N|N| +2444698|AAAAAAAAKJNEFCAA|1981-04-03|975|4240|326|1981|5|4|3|2|1981|326|4240|Friday|1981Q2|N|Y|N|2444696|2444785|2444333|2444608|N|N|N|N|N| +2444699|AAAAAAAALJNEFCAA|1981-04-04|975|4240|326|1981|6|4|4|2|1981|326|4240|Saturday|1981Q2|N|Y|N|2444696|2444785|2444334|2444609|N|N|N|N|N| +2444700|AAAAAAAAMJNEFCAA|1981-04-05|975|4240|326|1981|0|4|5|2|1981|326|4240|Sunday|1981Q2|N|N|N|2444696|2444785|2444335|2444610|N|N|N|N|N| +2444701|AAAAAAAANJNEFCAA|1981-04-06|975|4240|326|1981|1|4|6|2|1981|326|4240|Monday|1981Q2|N|N|N|2444696|2444785|2444336|2444611|N|N|N|N|N| +2444702|AAAAAAAAOJNEFCAA|1981-04-07|975|4241|326|1981|2|4|7|2|1981|326|4241|Tuesday|1981Q2|N|N|N|2444696|2444785|2444337|2444612|N|N|N|N|N| +2444703|AAAAAAAAPJNEFCAA|1981-04-08|975|4241|326|1981|3|4|8|2|1981|326|4241|Wednesday|1981Q2|N|N|N|2444696|2444785|2444338|2444613|N|N|N|N|N| +2444704|AAAAAAAAAKNEFCAA|1981-04-09|975|4241|326|1981|4|4|9|2|1981|326|4241|Thursday|1981Q2|N|N|N|2444696|2444785|2444339|2444614|N|N|N|N|N| +2444705|AAAAAAAABKNEFCAA|1981-04-10|975|4241|326|1981|5|4|10|2|1981|326|4241|Friday|1981Q2|N|Y|N|2444696|2444785|2444340|2444615|N|N|N|N|N| +2444706|AAAAAAAACKNEFCAA|1981-04-11|975|4241|326|1981|6|4|11|2|1981|326|4241|Saturday|1981Q2|N|Y|N|2444696|2444785|2444341|2444616|N|N|N|N|N| +2444707|AAAAAAAADKNEFCAA|1981-04-12|975|4241|326|1981|0|4|12|2|1981|326|4241|Sunday|1981Q2|N|N|N|2444696|2444785|2444342|2444617|N|N|N|N|N| +2444708|AAAAAAAAEKNEFCAA|1981-04-13|975|4241|326|1981|1|4|13|2|1981|326|4241|Monday|1981Q2|N|N|N|2444696|2444785|2444343|2444618|N|N|N|N|N| +2444709|AAAAAAAAFKNEFCAA|1981-04-14|975|4242|326|1981|2|4|14|2|1981|326|4242|Tuesday|1981Q2|N|N|N|2444696|2444785|2444344|2444619|N|N|N|N|N| +2444710|AAAAAAAAGKNEFCAA|1981-04-15|975|4242|326|1981|3|4|15|2|1981|326|4242|Wednesday|1981Q2|N|N|N|2444696|2444785|2444345|2444620|N|N|N|N|N| +2444711|AAAAAAAAHKNEFCAA|1981-04-16|975|4242|326|1981|4|4|16|2|1981|326|4242|Thursday|1981Q2|N|N|N|2444696|2444785|2444346|2444621|N|N|N|N|N| +2444712|AAAAAAAAIKNEFCAA|1981-04-17|975|4242|326|1981|5|4|17|2|1981|326|4242|Friday|1981Q2|N|Y|N|2444696|2444785|2444347|2444622|N|N|N|N|N| +2444713|AAAAAAAAJKNEFCAA|1981-04-18|975|4242|326|1981|6|4|18|2|1981|326|4242|Saturday|1981Q2|N|Y|N|2444696|2444785|2444348|2444623|N|N|N|N|N| +2444714|AAAAAAAAKKNEFCAA|1981-04-19|975|4242|326|1981|0|4|19|2|1981|326|4242|Sunday|1981Q2|N|N|N|2444696|2444785|2444349|2444624|N|N|N|N|N| +2444715|AAAAAAAALKNEFCAA|1981-04-20|975|4242|326|1981|1|4|20|2|1981|326|4242|Monday|1981Q2|N|N|N|2444696|2444785|2444350|2444625|N|N|N|N|N| +2444716|AAAAAAAAMKNEFCAA|1981-04-21|975|4243|326|1981|2|4|21|2|1981|326|4243|Tuesday|1981Q2|N|N|N|2444696|2444785|2444351|2444626|N|N|N|N|N| +2444717|AAAAAAAANKNEFCAA|1981-04-22|975|4243|326|1981|3|4|22|2|1981|326|4243|Wednesday|1981Q2|N|N|N|2444696|2444785|2444352|2444627|N|N|N|N|N| +2444718|AAAAAAAAOKNEFCAA|1981-04-23|975|4243|326|1981|4|4|23|2|1981|326|4243|Thursday|1981Q2|N|N|N|2444696|2444785|2444353|2444628|N|N|N|N|N| +2444719|AAAAAAAAPKNEFCAA|1981-04-24|975|4243|326|1981|5|4|24|2|1981|326|4243|Friday|1981Q2|N|Y|N|2444696|2444785|2444354|2444629|N|N|N|N|N| +2444720|AAAAAAAAALNEFCAA|1981-04-25|975|4243|326|1981|6|4|25|2|1981|326|4243|Saturday|1981Q2|N|Y|N|2444696|2444785|2444355|2444630|N|N|N|N|N| +2444721|AAAAAAAABLNEFCAA|1981-04-26|975|4243|326|1981|0|4|26|2|1981|326|4243|Sunday|1981Q2|N|N|N|2444696|2444785|2444356|2444631|N|N|N|N|N| +2444722|AAAAAAAACLNEFCAA|1981-04-27|975|4243|326|1981|1|4|27|2|1981|326|4243|Monday|1981Q2|N|N|N|2444696|2444785|2444357|2444632|N|N|N|N|N| +2444723|AAAAAAAADLNEFCAA|1981-04-28|975|4244|326|1981|2|4|28|2|1981|326|4244|Tuesday|1981Q2|N|N|N|2444696|2444785|2444358|2444633|N|N|N|N|N| +2444724|AAAAAAAAELNEFCAA|1981-04-29|975|4244|326|1981|3|4|29|2|1981|326|4244|Wednesday|1981Q2|N|N|N|2444696|2444785|2444359|2444634|N|N|N|N|N| +2444725|AAAAAAAAFLNEFCAA|1981-04-30|975|4244|326|1981|4|4|30|2|1981|326|4244|Thursday|1981Q2|N|N|N|2444696|2444785|2444360|2444635|N|N|N|N|N| +2444726|AAAAAAAAGLNEFCAA|1981-05-01|976|4244|326|1981|5|5|1|2|1981|326|4244|Friday|1981Q2|N|Y|N|2444726|2444845|2444361|2444636|N|N|N|N|N| +2444727|AAAAAAAAHLNEFCAA|1981-05-02|976|4244|326|1981|6|5|2|2|1981|326|4244|Saturday|1981Q2|N|Y|N|2444726|2444845|2444362|2444637|N|N|N|N|N| +2444728|AAAAAAAAILNEFCAA|1981-05-03|976|4244|326|1981|0|5|3|2|1981|326|4244|Sunday|1981Q2|N|N|N|2444726|2444845|2444363|2444638|N|N|N|N|N| +2444729|AAAAAAAAJLNEFCAA|1981-05-04|976|4244|326|1981|1|5|4|2|1981|326|4244|Monday|1981Q2|N|N|N|2444726|2444845|2444364|2444639|N|N|N|N|N| +2444730|AAAAAAAAKLNEFCAA|1981-05-05|976|4245|326|1981|2|5|5|2|1981|326|4245|Tuesday|1981Q2|N|N|N|2444726|2444845|2444365|2444640|N|N|N|N|N| +2444731|AAAAAAAALLNEFCAA|1981-05-06|976|4245|326|1981|3|5|6|2|1981|326|4245|Wednesday|1981Q2|N|N|N|2444726|2444845|2444366|2444641|N|N|N|N|N| +2444732|AAAAAAAAMLNEFCAA|1981-05-07|976|4245|326|1981|4|5|7|2|1981|326|4245|Thursday|1981Q2|N|N|N|2444726|2444845|2444367|2444642|N|N|N|N|N| +2444733|AAAAAAAANLNEFCAA|1981-05-08|976|4245|326|1981|5|5|8|2|1981|326|4245|Friday|1981Q2|N|Y|N|2444726|2444845|2444368|2444643|N|N|N|N|N| +2444734|AAAAAAAAOLNEFCAA|1981-05-09|976|4245|326|1981|6|5|9|2|1981|326|4245|Saturday|1981Q2|N|Y|N|2444726|2444845|2444369|2444644|N|N|N|N|N| +2444735|AAAAAAAAPLNEFCAA|1981-05-10|976|4245|326|1981|0|5|10|2|1981|326|4245|Sunday|1981Q2|N|N|N|2444726|2444845|2444370|2444645|N|N|N|N|N| +2444736|AAAAAAAAAMNEFCAA|1981-05-11|976|4245|326|1981|1|5|11|2|1981|326|4245|Monday|1981Q2|N|N|N|2444726|2444845|2444371|2444646|N|N|N|N|N| +2444737|AAAAAAAABMNEFCAA|1981-05-12|976|4246|326|1981|2|5|12|2|1981|326|4246|Tuesday|1981Q2|N|N|N|2444726|2444845|2444372|2444647|N|N|N|N|N| +2444738|AAAAAAAACMNEFCAA|1981-05-13|976|4246|326|1981|3|5|13|2|1981|326|4246|Wednesday|1981Q2|N|N|N|2444726|2444845|2444373|2444648|N|N|N|N|N| +2444739|AAAAAAAADMNEFCAA|1981-05-14|976|4246|326|1981|4|5|14|2|1981|326|4246|Thursday|1981Q2|N|N|N|2444726|2444845|2444374|2444649|N|N|N|N|N| +2444740|AAAAAAAAEMNEFCAA|1981-05-15|976|4246|326|1981|5|5|15|2|1981|326|4246|Friday|1981Q2|N|Y|N|2444726|2444845|2444375|2444650|N|N|N|N|N| +2444741|AAAAAAAAFMNEFCAA|1981-05-16|976|4246|326|1981|6|5|16|2|1981|326|4246|Saturday|1981Q2|N|Y|N|2444726|2444845|2444376|2444651|N|N|N|N|N| +2444742|AAAAAAAAGMNEFCAA|1981-05-17|976|4246|326|1981|0|5|17|2|1981|326|4246|Sunday|1981Q2|N|N|N|2444726|2444845|2444377|2444652|N|N|N|N|N| +2444743|AAAAAAAAHMNEFCAA|1981-05-18|976|4246|326|1981|1|5|18|2|1981|326|4246|Monday|1981Q2|N|N|N|2444726|2444845|2444378|2444653|N|N|N|N|N| +2444744|AAAAAAAAIMNEFCAA|1981-05-19|976|4247|326|1981|2|5|19|2|1981|326|4247|Tuesday|1981Q2|N|N|N|2444726|2444845|2444379|2444654|N|N|N|N|N| +2444745|AAAAAAAAJMNEFCAA|1981-05-20|976|4247|326|1981|3|5|20|2|1981|326|4247|Wednesday|1981Q2|N|N|N|2444726|2444845|2444380|2444655|N|N|N|N|N| +2444746|AAAAAAAAKMNEFCAA|1981-05-21|976|4247|326|1981|4|5|21|2|1981|326|4247|Thursday|1981Q2|N|N|N|2444726|2444845|2444381|2444656|N|N|N|N|N| +2444747|AAAAAAAALMNEFCAA|1981-05-22|976|4247|326|1981|5|5|22|2|1981|326|4247|Friday|1981Q2|N|Y|N|2444726|2444845|2444382|2444657|N|N|N|N|N| +2444748|AAAAAAAAMMNEFCAA|1981-05-23|976|4247|326|1981|6|5|23|2|1981|326|4247|Saturday|1981Q2|N|Y|N|2444726|2444845|2444383|2444658|N|N|N|N|N| +2444749|AAAAAAAANMNEFCAA|1981-05-24|976|4247|326|1981|0|5|24|2|1981|326|4247|Sunday|1981Q2|N|N|N|2444726|2444845|2444384|2444659|N|N|N|N|N| +2444750|AAAAAAAAOMNEFCAA|1981-05-25|976|4247|326|1981|1|5|25|2|1981|326|4247|Monday|1981Q2|N|N|N|2444726|2444845|2444385|2444660|N|N|N|N|N| +2444751|AAAAAAAAPMNEFCAA|1981-05-26|976|4248|326|1981|2|5|26|2|1981|326|4248|Tuesday|1981Q2|N|N|N|2444726|2444845|2444386|2444661|N|N|N|N|N| +2444752|AAAAAAAAANNEFCAA|1981-05-27|976|4248|326|1981|3|5|27|2|1981|326|4248|Wednesday|1981Q2|N|N|N|2444726|2444845|2444387|2444662|N|N|N|N|N| +2444753|AAAAAAAABNNEFCAA|1981-05-28|976|4248|326|1981|4|5|28|2|1981|326|4248|Thursday|1981Q2|N|N|N|2444726|2444845|2444388|2444663|N|N|N|N|N| +2444754|AAAAAAAACNNEFCAA|1981-05-29|976|4248|326|1981|5|5|29|2|1981|326|4248|Friday|1981Q2|N|Y|N|2444726|2444845|2444389|2444664|N|N|N|N|N| +2444755|AAAAAAAADNNEFCAA|1981-05-30|976|4248|326|1981|6|5|30|2|1981|326|4248|Saturday|1981Q2|N|Y|N|2444726|2444845|2444390|2444665|N|N|N|N|N| +2444756|AAAAAAAAENNEFCAA|1981-05-31|976|4248|326|1981|0|5|31|2|1981|326|4248|Sunday|1981Q2|N|N|N|2444726|2444845|2444391|2444666|N|N|N|N|N| +2444757|AAAAAAAAFNNEFCAA|1981-06-01|977|4248|327|1981|1|6|1|2|1981|327|4248|Monday|1981Q2|N|N|N|2444757|2444907|2444392|2444667|N|N|N|N|N| +2444758|AAAAAAAAGNNEFCAA|1981-06-02|977|4249|327|1981|2|6|2|2|1981|327|4249|Tuesday|1981Q2|N|N|N|2444757|2444907|2444393|2444668|N|N|N|N|N| +2444759|AAAAAAAAHNNEFCAA|1981-06-03|977|4249|327|1981|3|6|3|2|1981|327|4249|Wednesday|1981Q2|N|N|N|2444757|2444907|2444394|2444669|N|N|N|N|N| +2444760|AAAAAAAAINNEFCAA|1981-06-04|977|4249|327|1981|4|6|4|2|1981|327|4249|Thursday|1981Q2|N|N|N|2444757|2444907|2444395|2444670|N|N|N|N|N| +2444761|AAAAAAAAJNNEFCAA|1981-06-05|977|4249|327|1981|5|6|5|2|1981|327|4249|Friday|1981Q2|N|Y|N|2444757|2444907|2444396|2444671|N|N|N|N|N| +2444762|AAAAAAAAKNNEFCAA|1981-06-06|977|4249|327|1981|6|6|6|2|1981|327|4249|Saturday|1981Q2|N|Y|N|2444757|2444907|2444397|2444672|N|N|N|N|N| +2444763|AAAAAAAALNNEFCAA|1981-06-07|977|4249|327|1981|0|6|7|2|1981|327|4249|Sunday|1981Q2|N|N|N|2444757|2444907|2444398|2444673|N|N|N|N|N| +2444764|AAAAAAAAMNNEFCAA|1981-06-08|977|4249|327|1981|1|6|8|2|1981|327|4249|Monday|1981Q2|N|N|N|2444757|2444907|2444399|2444674|N|N|N|N|N| +2444765|AAAAAAAANNNEFCAA|1981-06-09|977|4250|327|1981|2|6|9|2|1981|327|4250|Tuesday|1981Q2|N|N|N|2444757|2444907|2444400|2444675|N|N|N|N|N| +2444766|AAAAAAAAONNEFCAA|1981-06-10|977|4250|327|1981|3|6|10|2|1981|327|4250|Wednesday|1981Q2|N|N|N|2444757|2444907|2444401|2444676|N|N|N|N|N| +2444767|AAAAAAAAPNNEFCAA|1981-06-11|977|4250|327|1981|4|6|11|2|1981|327|4250|Thursday|1981Q2|N|N|N|2444757|2444907|2444402|2444677|N|N|N|N|N| +2444768|AAAAAAAAAONEFCAA|1981-06-12|977|4250|327|1981|5|6|12|2|1981|327|4250|Friday|1981Q2|N|Y|N|2444757|2444907|2444403|2444678|N|N|N|N|N| +2444769|AAAAAAAABONEFCAA|1981-06-13|977|4250|327|1981|6|6|13|2|1981|327|4250|Saturday|1981Q2|N|Y|N|2444757|2444907|2444404|2444679|N|N|N|N|N| +2444770|AAAAAAAACONEFCAA|1981-06-14|977|4250|327|1981|0|6|14|2|1981|327|4250|Sunday|1981Q2|N|N|N|2444757|2444907|2444405|2444680|N|N|N|N|N| +2444771|AAAAAAAADONEFCAA|1981-06-15|977|4250|327|1981|1|6|15|2|1981|327|4250|Monday|1981Q2|N|N|N|2444757|2444907|2444406|2444681|N|N|N|N|N| +2444772|AAAAAAAAEONEFCAA|1981-06-16|977|4251|327|1981|2|6|16|2|1981|327|4251|Tuesday|1981Q2|N|N|N|2444757|2444907|2444407|2444682|N|N|N|N|N| +2444773|AAAAAAAAFONEFCAA|1981-06-17|977|4251|327|1981|3|6|17|2|1981|327|4251|Wednesday|1981Q2|N|N|N|2444757|2444907|2444408|2444683|N|N|N|N|N| +2444774|AAAAAAAAGONEFCAA|1981-06-18|977|4251|327|1981|4|6|18|2|1981|327|4251|Thursday|1981Q2|N|N|N|2444757|2444907|2444409|2444684|N|N|N|N|N| +2444775|AAAAAAAAHONEFCAA|1981-06-19|977|4251|327|1981|5|6|19|2|1981|327|4251|Friday|1981Q2|N|Y|N|2444757|2444907|2444410|2444685|N|N|N|N|N| +2444776|AAAAAAAAIONEFCAA|1981-06-20|977|4251|327|1981|6|6|20|2|1981|327|4251|Saturday|1981Q2|N|Y|N|2444757|2444907|2444411|2444686|N|N|N|N|N| +2444777|AAAAAAAAJONEFCAA|1981-06-21|977|4251|327|1981|0|6|21|2|1981|327|4251|Sunday|1981Q2|N|N|N|2444757|2444907|2444412|2444687|N|N|N|N|N| +2444778|AAAAAAAAKONEFCAA|1981-06-22|977|4251|327|1981|1|6|22|2|1981|327|4251|Monday|1981Q2|N|N|N|2444757|2444907|2444413|2444688|N|N|N|N|N| +2444779|AAAAAAAALONEFCAA|1981-06-23|977|4252|327|1981|2|6|23|2|1981|327|4252|Tuesday|1981Q2|N|N|N|2444757|2444907|2444414|2444689|N|N|N|N|N| +2444780|AAAAAAAAMONEFCAA|1981-06-24|977|4252|327|1981|3|6|24|2|1981|327|4252|Wednesday|1981Q2|N|N|N|2444757|2444907|2444415|2444690|N|N|N|N|N| +2444781|AAAAAAAANONEFCAA|1981-06-25|977|4252|327|1981|4|6|25|2|1981|327|4252|Thursday|1981Q2|N|N|N|2444757|2444907|2444416|2444691|N|N|N|N|N| +2444782|AAAAAAAAOONEFCAA|1981-06-26|977|4252|327|1981|5|6|26|2|1981|327|4252|Friday|1981Q2|N|Y|N|2444757|2444907|2444417|2444692|N|N|N|N|N| +2444783|AAAAAAAAPONEFCAA|1981-06-27|977|4252|327|1981|6|6|27|2|1981|327|4252|Saturday|1981Q2|N|Y|N|2444757|2444907|2444418|2444693|N|N|N|N|N| +2444784|AAAAAAAAAPNEFCAA|1981-06-28|977|4252|327|1981|0|6|28|2|1981|327|4252|Sunday|1981Q2|N|N|N|2444757|2444907|2444419|2444694|N|N|N|N|N| +2444785|AAAAAAAABPNEFCAA|1981-06-29|977|4252|327|1981|1|6|29|2|1981|327|4252|Monday|1981Q2|N|N|N|2444757|2444907|2444420|2444695|N|N|N|N|N| +2444786|AAAAAAAACPNEFCAA|1981-06-30|977|4253|327|1981|2|6|30|2|1981|327|4253|Tuesday|1981Q2|N|N|N|2444757|2444907|2444421|2444696|N|N|N|N|N| +2444787|AAAAAAAADPNEFCAA|1981-07-01|978|4253|327|1981|3|7|1|2|1981|327|4253|Wednesday|1981Q2|N|N|N|2444787|2444967|2444422|2444696|N|N|N|N|N| +2444788|AAAAAAAAEPNEFCAA|1981-07-02|978|4253|327|1981|4|7|2|3|1981|327|4253|Thursday|1981Q3|N|N|N|2444787|2444967|2444423|2444697|N|N|N|N|N| +2444789|AAAAAAAAFPNEFCAA|1981-07-03|978|4253|327|1981|5|7|3|3|1981|327|4253|Friday|1981Q3|N|Y|N|2444787|2444967|2444424|2444698|N|N|N|N|N| +2444790|AAAAAAAAGPNEFCAA|1981-07-04|978|4253|327|1981|6|7|4|3|1981|327|4253|Saturday|1981Q3|N|Y|N|2444787|2444967|2444425|2444699|N|N|N|N|N| +2444791|AAAAAAAAHPNEFCAA|1981-07-05|978|4253|327|1981|0|7|5|3|1981|327|4253|Sunday|1981Q3|Y|N|N|2444787|2444967|2444426|2444700|N|N|N|N|N| +2444792|AAAAAAAAIPNEFCAA|1981-07-06|978|4253|327|1981|1|7|6|3|1981|327|4253|Monday|1981Q3|N|N|Y|2444787|2444967|2444427|2444701|N|N|N|N|N| +2444793|AAAAAAAAJPNEFCAA|1981-07-07|978|4254|327|1981|2|7|7|3|1981|327|4254|Tuesday|1981Q3|N|N|N|2444787|2444967|2444428|2444702|N|N|N|N|N| +2444794|AAAAAAAAKPNEFCAA|1981-07-08|978|4254|327|1981|3|7|8|3|1981|327|4254|Wednesday|1981Q3|N|N|N|2444787|2444967|2444429|2444703|N|N|N|N|N| +2444795|AAAAAAAALPNEFCAA|1981-07-09|978|4254|327|1981|4|7|9|3|1981|327|4254|Thursday|1981Q3|N|N|N|2444787|2444967|2444430|2444704|N|N|N|N|N| +2444796|AAAAAAAAMPNEFCAA|1981-07-10|978|4254|327|1981|5|7|10|3|1981|327|4254|Friday|1981Q3|N|Y|N|2444787|2444967|2444431|2444705|N|N|N|N|N| +2444797|AAAAAAAANPNEFCAA|1981-07-11|978|4254|327|1981|6|7|11|3|1981|327|4254|Saturday|1981Q3|N|Y|N|2444787|2444967|2444432|2444706|N|N|N|N|N| +2444798|AAAAAAAAOPNEFCAA|1981-07-12|978|4254|327|1981|0|7|12|3|1981|327|4254|Sunday|1981Q3|N|N|N|2444787|2444967|2444433|2444707|N|N|N|N|N| +2444799|AAAAAAAAPPNEFCAA|1981-07-13|978|4254|327|1981|1|7|13|3|1981|327|4254|Monday|1981Q3|N|N|N|2444787|2444967|2444434|2444708|N|N|N|N|N| +2444800|AAAAAAAAAAOEFCAA|1981-07-14|978|4255|327|1981|2|7|14|3|1981|327|4255|Tuesday|1981Q3|N|N|N|2444787|2444967|2444435|2444709|N|N|N|N|N| +2444801|AAAAAAAABAOEFCAA|1981-07-15|978|4255|327|1981|3|7|15|3|1981|327|4255|Wednesday|1981Q3|N|N|N|2444787|2444967|2444436|2444710|N|N|N|N|N| +2444802|AAAAAAAACAOEFCAA|1981-07-16|978|4255|327|1981|4|7|16|3|1981|327|4255|Thursday|1981Q3|N|N|N|2444787|2444967|2444437|2444711|N|N|N|N|N| +2444803|AAAAAAAADAOEFCAA|1981-07-17|978|4255|327|1981|5|7|17|3|1981|327|4255|Friday|1981Q3|N|Y|N|2444787|2444967|2444438|2444712|N|N|N|N|N| +2444804|AAAAAAAAEAOEFCAA|1981-07-18|978|4255|327|1981|6|7|18|3|1981|327|4255|Saturday|1981Q3|N|Y|N|2444787|2444967|2444439|2444713|N|N|N|N|N| +2444805|AAAAAAAAFAOEFCAA|1981-07-19|978|4255|327|1981|0|7|19|3|1981|327|4255|Sunday|1981Q3|N|N|N|2444787|2444967|2444440|2444714|N|N|N|N|N| +2444806|AAAAAAAAGAOEFCAA|1981-07-20|978|4255|327|1981|1|7|20|3|1981|327|4255|Monday|1981Q3|N|N|N|2444787|2444967|2444441|2444715|N|N|N|N|N| +2444807|AAAAAAAAHAOEFCAA|1981-07-21|978|4256|327|1981|2|7|21|3|1981|327|4256|Tuesday|1981Q3|N|N|N|2444787|2444967|2444442|2444716|N|N|N|N|N| +2444808|AAAAAAAAIAOEFCAA|1981-07-22|978|4256|327|1981|3|7|22|3|1981|327|4256|Wednesday|1981Q3|N|N|N|2444787|2444967|2444443|2444717|N|N|N|N|N| +2444809|AAAAAAAAJAOEFCAA|1981-07-23|978|4256|327|1981|4|7|23|3|1981|327|4256|Thursday|1981Q3|N|N|N|2444787|2444967|2444444|2444718|N|N|N|N|N| +2444810|AAAAAAAAKAOEFCAA|1981-07-24|978|4256|327|1981|5|7|24|3|1981|327|4256|Friday|1981Q3|N|Y|N|2444787|2444967|2444445|2444719|N|N|N|N|N| +2444811|AAAAAAAALAOEFCAA|1981-07-25|978|4256|327|1981|6|7|25|3|1981|327|4256|Saturday|1981Q3|N|Y|N|2444787|2444967|2444446|2444720|N|N|N|N|N| +2444812|AAAAAAAAMAOEFCAA|1981-07-26|978|4256|327|1981|0|7|26|3|1981|327|4256|Sunday|1981Q3|N|N|N|2444787|2444967|2444447|2444721|N|N|N|N|N| +2444813|AAAAAAAANAOEFCAA|1981-07-27|978|4256|327|1981|1|7|27|3|1981|327|4256|Monday|1981Q3|N|N|N|2444787|2444967|2444448|2444722|N|N|N|N|N| +2444814|AAAAAAAAOAOEFCAA|1981-07-28|978|4257|327|1981|2|7|28|3|1981|327|4257|Tuesday|1981Q3|N|N|N|2444787|2444967|2444449|2444723|N|N|N|N|N| +2444815|AAAAAAAAPAOEFCAA|1981-07-29|978|4257|327|1981|3|7|29|3|1981|327|4257|Wednesday|1981Q3|N|N|N|2444787|2444967|2444450|2444724|N|N|N|N|N| +2444816|AAAAAAAAABOEFCAA|1981-07-30|978|4257|327|1981|4|7|30|3|1981|327|4257|Thursday|1981Q3|N|N|N|2444787|2444967|2444451|2444725|N|N|N|N|N| +2444817|AAAAAAAABBOEFCAA|1981-07-31|978|4257|327|1981|5|7|31|3|1981|327|4257|Friday|1981Q3|N|Y|N|2444787|2444967|2444452|2444726|N|N|N|N|N| +2444818|AAAAAAAACBOEFCAA|1981-08-01|979|4257|327|1981|6|8|1|3|1981|327|4257|Saturday|1981Q3|N|Y|N|2444818|2445029|2444453|2444727|N|N|N|N|N| +2444819|AAAAAAAADBOEFCAA|1981-08-02|979|4257|327|1981|0|8|2|3|1981|327|4257|Sunday|1981Q3|N|N|N|2444818|2445029|2444454|2444728|N|N|N|N|N| +2444820|AAAAAAAAEBOEFCAA|1981-08-03|979|4257|327|1981|1|8|3|3|1981|327|4257|Monday|1981Q3|N|N|N|2444818|2445029|2444455|2444729|N|N|N|N|N| +2444821|AAAAAAAAFBOEFCAA|1981-08-04|979|4258|327|1981|2|8|4|3|1981|327|4258|Tuesday|1981Q3|N|N|N|2444818|2445029|2444456|2444730|N|N|N|N|N| +2444822|AAAAAAAAGBOEFCAA|1981-08-05|979|4258|327|1981|3|8|5|3|1981|327|4258|Wednesday|1981Q3|N|N|N|2444818|2445029|2444457|2444731|N|N|N|N|N| +2444823|AAAAAAAAHBOEFCAA|1981-08-06|979|4258|327|1981|4|8|6|3|1981|327|4258|Thursday|1981Q3|N|N|N|2444818|2445029|2444458|2444732|N|N|N|N|N| +2444824|AAAAAAAAIBOEFCAA|1981-08-07|979|4258|327|1981|5|8|7|3|1981|327|4258|Friday|1981Q3|N|Y|N|2444818|2445029|2444459|2444733|N|N|N|N|N| +2444825|AAAAAAAAJBOEFCAA|1981-08-08|979|4258|327|1981|6|8|8|3|1981|327|4258|Saturday|1981Q3|N|Y|N|2444818|2445029|2444460|2444734|N|N|N|N|N| +2444826|AAAAAAAAKBOEFCAA|1981-08-09|979|4258|327|1981|0|8|9|3|1981|327|4258|Sunday|1981Q3|N|N|N|2444818|2445029|2444461|2444735|N|N|N|N|N| +2444827|AAAAAAAALBOEFCAA|1981-08-10|979|4258|327|1981|1|8|10|3|1981|327|4258|Monday|1981Q3|N|N|N|2444818|2445029|2444462|2444736|N|N|N|N|N| +2444828|AAAAAAAAMBOEFCAA|1981-08-11|979|4259|327|1981|2|8|11|3|1981|327|4259|Tuesday|1981Q3|N|N|N|2444818|2445029|2444463|2444737|N|N|N|N|N| +2444829|AAAAAAAANBOEFCAA|1981-08-12|979|4259|327|1981|3|8|12|3|1981|327|4259|Wednesday|1981Q3|N|N|N|2444818|2445029|2444464|2444738|N|N|N|N|N| +2444830|AAAAAAAAOBOEFCAA|1981-08-13|979|4259|327|1981|4|8|13|3|1981|327|4259|Thursday|1981Q3|N|N|N|2444818|2445029|2444465|2444739|N|N|N|N|N| +2444831|AAAAAAAAPBOEFCAA|1981-08-14|979|4259|327|1981|5|8|14|3|1981|327|4259|Friday|1981Q3|N|Y|N|2444818|2445029|2444466|2444740|N|N|N|N|N| +2444832|AAAAAAAAACOEFCAA|1981-08-15|979|4259|327|1981|6|8|15|3|1981|327|4259|Saturday|1981Q3|N|Y|N|2444818|2445029|2444467|2444741|N|N|N|N|N| +2444833|AAAAAAAABCOEFCAA|1981-08-16|979|4259|327|1981|0|8|16|3|1981|327|4259|Sunday|1981Q3|N|N|N|2444818|2445029|2444468|2444742|N|N|N|N|N| +2444834|AAAAAAAACCOEFCAA|1981-08-17|979|4259|327|1981|1|8|17|3|1981|327|4259|Monday|1981Q3|N|N|N|2444818|2445029|2444469|2444743|N|N|N|N|N| +2444835|AAAAAAAADCOEFCAA|1981-08-18|979|4260|327|1981|2|8|18|3|1981|327|4260|Tuesday|1981Q3|N|N|N|2444818|2445029|2444470|2444744|N|N|N|N|N| +2444836|AAAAAAAAECOEFCAA|1981-08-19|979|4260|327|1981|3|8|19|3|1981|327|4260|Wednesday|1981Q3|N|N|N|2444818|2445029|2444471|2444745|N|N|N|N|N| +2444837|AAAAAAAAFCOEFCAA|1981-08-20|979|4260|327|1981|4|8|20|3|1981|327|4260|Thursday|1981Q3|N|N|N|2444818|2445029|2444472|2444746|N|N|N|N|N| +2444838|AAAAAAAAGCOEFCAA|1981-08-21|979|4260|327|1981|5|8|21|3|1981|327|4260|Friday|1981Q3|N|Y|N|2444818|2445029|2444473|2444747|N|N|N|N|N| +2444839|AAAAAAAAHCOEFCAA|1981-08-22|979|4260|327|1981|6|8|22|3|1981|327|4260|Saturday|1981Q3|N|Y|N|2444818|2445029|2444474|2444748|N|N|N|N|N| +2444840|AAAAAAAAICOEFCAA|1981-08-23|979|4260|327|1981|0|8|23|3|1981|327|4260|Sunday|1981Q3|N|N|N|2444818|2445029|2444475|2444749|N|N|N|N|N| +2444841|AAAAAAAAJCOEFCAA|1981-08-24|979|4260|327|1981|1|8|24|3|1981|327|4260|Monday|1981Q3|N|N|N|2444818|2445029|2444476|2444750|N|N|N|N|N| +2444842|AAAAAAAAKCOEFCAA|1981-08-25|979|4261|327|1981|2|8|25|3|1981|327|4261|Tuesday|1981Q3|N|N|N|2444818|2445029|2444477|2444751|N|N|N|N|N| +2444843|AAAAAAAALCOEFCAA|1981-08-26|979|4261|327|1981|3|8|26|3|1981|327|4261|Wednesday|1981Q3|N|N|N|2444818|2445029|2444478|2444752|N|N|N|N|N| +2444844|AAAAAAAAMCOEFCAA|1981-08-27|979|4261|327|1981|4|8|27|3|1981|327|4261|Thursday|1981Q3|N|N|N|2444818|2445029|2444479|2444753|N|N|N|N|N| +2444845|AAAAAAAANCOEFCAA|1981-08-28|979|4261|327|1981|5|8|28|3|1981|327|4261|Friday|1981Q3|N|Y|N|2444818|2445029|2444480|2444754|N|N|N|N|N| +2444846|AAAAAAAAOCOEFCAA|1981-08-29|979|4261|327|1981|6|8|29|3|1981|327|4261|Saturday|1981Q3|N|Y|N|2444818|2445029|2444481|2444755|N|N|N|N|N| +2444847|AAAAAAAAPCOEFCAA|1981-08-30|979|4261|327|1981|0|8|30|3|1981|327|4261|Sunday|1981Q3|N|N|N|2444818|2445029|2444482|2444756|N|N|N|N|N| +2444848|AAAAAAAAADOEFCAA|1981-08-31|979|4261|327|1981|1|8|31|3|1981|327|4261|Monday|1981Q3|N|N|N|2444818|2445029|2444483|2444757|N|N|N|N|N| +2444849|AAAAAAAABDOEFCAA|1981-09-01|980|4262|328|1981|2|9|1|3|1981|328|4262|Tuesday|1981Q3|N|N|N|2444849|2445091|2444484|2444758|N|N|N|N|N| +2444850|AAAAAAAACDOEFCAA|1981-09-02|980|4262|328|1981|3|9|2|3|1981|328|4262|Wednesday|1981Q3|N|N|N|2444849|2445091|2444485|2444759|N|N|N|N|N| +2444851|AAAAAAAADDOEFCAA|1981-09-03|980|4262|328|1981|4|9|3|3|1981|328|4262|Thursday|1981Q3|N|N|N|2444849|2445091|2444486|2444760|N|N|N|N|N| +2444852|AAAAAAAAEDOEFCAA|1981-09-04|980|4262|328|1981|5|9|4|3|1981|328|4262|Friday|1981Q3|N|Y|N|2444849|2445091|2444487|2444761|N|N|N|N|N| +2444853|AAAAAAAAFDOEFCAA|1981-09-05|980|4262|328|1981|6|9|5|3|1981|328|4262|Saturday|1981Q3|N|Y|N|2444849|2445091|2444488|2444762|N|N|N|N|N| +2444854|AAAAAAAAGDOEFCAA|1981-09-06|980|4262|328|1981|0|9|6|3|1981|328|4262|Sunday|1981Q3|N|N|N|2444849|2445091|2444489|2444763|N|N|N|N|N| +2444855|AAAAAAAAHDOEFCAA|1981-09-07|980|4262|328|1981|1|9|7|3|1981|328|4262|Monday|1981Q3|N|N|N|2444849|2445091|2444490|2444764|N|N|N|N|N| +2444856|AAAAAAAAIDOEFCAA|1981-09-08|980|4263|328|1981|2|9|8|3|1981|328|4263|Tuesday|1981Q3|N|N|N|2444849|2445091|2444491|2444765|N|N|N|N|N| +2444857|AAAAAAAAJDOEFCAA|1981-09-09|980|4263|328|1981|3|9|9|3|1981|328|4263|Wednesday|1981Q3|N|N|N|2444849|2445091|2444492|2444766|N|N|N|N|N| +2444858|AAAAAAAAKDOEFCAA|1981-09-10|980|4263|328|1981|4|9|10|3|1981|328|4263|Thursday|1981Q3|N|N|N|2444849|2445091|2444493|2444767|N|N|N|N|N| +2444859|AAAAAAAALDOEFCAA|1981-09-11|980|4263|328|1981|5|9|11|3|1981|328|4263|Friday|1981Q3|N|Y|N|2444849|2445091|2444494|2444768|N|N|N|N|N| +2444860|AAAAAAAAMDOEFCAA|1981-09-12|980|4263|328|1981|6|9|12|3|1981|328|4263|Saturday|1981Q3|N|Y|N|2444849|2445091|2444495|2444769|N|N|N|N|N| +2444861|AAAAAAAANDOEFCAA|1981-09-13|980|4263|328|1981|0|9|13|3|1981|328|4263|Sunday|1981Q3|N|N|N|2444849|2445091|2444496|2444770|N|N|N|N|N| +2444862|AAAAAAAAODOEFCAA|1981-09-14|980|4263|328|1981|1|9|14|3|1981|328|4263|Monday|1981Q3|N|N|N|2444849|2445091|2444497|2444771|N|N|N|N|N| +2444863|AAAAAAAAPDOEFCAA|1981-09-15|980|4264|328|1981|2|9|15|3|1981|328|4264|Tuesday|1981Q3|N|N|N|2444849|2445091|2444498|2444772|N|N|N|N|N| +2444864|AAAAAAAAAEOEFCAA|1981-09-16|980|4264|328|1981|3|9|16|3|1981|328|4264|Wednesday|1981Q3|N|N|N|2444849|2445091|2444499|2444773|N|N|N|N|N| +2444865|AAAAAAAABEOEFCAA|1981-09-17|980|4264|328|1981|4|9|17|3|1981|328|4264|Thursday|1981Q3|N|N|N|2444849|2445091|2444500|2444774|N|N|N|N|N| +2444866|AAAAAAAACEOEFCAA|1981-09-18|980|4264|328|1981|5|9|18|3|1981|328|4264|Friday|1981Q3|N|Y|N|2444849|2445091|2444501|2444775|N|N|N|N|N| +2444867|AAAAAAAADEOEFCAA|1981-09-19|980|4264|328|1981|6|9|19|3|1981|328|4264|Saturday|1981Q3|N|Y|N|2444849|2445091|2444502|2444776|N|N|N|N|N| +2444868|AAAAAAAAEEOEFCAA|1981-09-20|980|4264|328|1981|0|9|20|3|1981|328|4264|Sunday|1981Q3|N|N|N|2444849|2445091|2444503|2444777|N|N|N|N|N| +2444869|AAAAAAAAFEOEFCAA|1981-09-21|980|4264|328|1981|1|9|21|3|1981|328|4264|Monday|1981Q3|N|N|N|2444849|2445091|2444504|2444778|N|N|N|N|N| +2444870|AAAAAAAAGEOEFCAA|1981-09-22|980|4265|328|1981|2|9|22|3|1981|328|4265|Tuesday|1981Q3|N|N|N|2444849|2445091|2444505|2444779|N|N|N|N|N| +2444871|AAAAAAAAHEOEFCAA|1981-09-23|980|4265|328|1981|3|9|23|3|1981|328|4265|Wednesday|1981Q3|N|N|N|2444849|2445091|2444506|2444780|N|N|N|N|N| +2444872|AAAAAAAAIEOEFCAA|1981-09-24|980|4265|328|1981|4|9|24|3|1981|328|4265|Thursday|1981Q3|N|N|N|2444849|2445091|2444507|2444781|N|N|N|N|N| +2444873|AAAAAAAAJEOEFCAA|1981-09-25|980|4265|328|1981|5|9|25|3|1981|328|4265|Friday|1981Q3|N|Y|N|2444849|2445091|2444508|2444782|N|N|N|N|N| +2444874|AAAAAAAAKEOEFCAA|1981-09-26|980|4265|328|1981|6|9|26|3|1981|328|4265|Saturday|1981Q3|N|Y|N|2444849|2445091|2444509|2444783|N|N|N|N|N| +2444875|AAAAAAAALEOEFCAA|1981-09-27|980|4265|328|1981|0|9|27|3|1981|328|4265|Sunday|1981Q3|N|N|N|2444849|2445091|2444510|2444784|N|N|N|N|N| +2444876|AAAAAAAAMEOEFCAA|1981-09-28|980|4265|328|1981|1|9|28|3|1981|328|4265|Monday|1981Q3|N|N|N|2444849|2445091|2444511|2444785|N|N|N|N|N| +2444877|AAAAAAAANEOEFCAA|1981-09-29|980|4266|328|1981|2|9|29|3|1981|328|4266|Tuesday|1981Q3|N|N|N|2444849|2445091|2444512|2444786|N|N|N|N|N| +2444878|AAAAAAAAOEOEFCAA|1981-09-30|980|4266|328|1981|3|9|30|3|1981|328|4266|Wednesday|1981Q3|N|N|N|2444849|2445091|2444513|2444787|N|N|N|N|N| +2444879|AAAAAAAAPEOEFCAA|1981-10-01|981|4266|328|1981|4|10|1|3|1981|328|4266|Thursday|1981Q3|N|N|N|2444879|2445151|2444514|2444787|N|N|N|N|N| +2444880|AAAAAAAAAFOEFCAA|1981-10-02|981|4266|328|1981|5|10|2|4|1981|328|4266|Friday|1981Q4|N|Y|N|2444879|2445151|2444515|2444788|N|N|N|N|N| +2444881|AAAAAAAABFOEFCAA|1981-10-03|981|4266|328|1981|6|10|3|4|1981|328|4266|Saturday|1981Q4|N|Y|N|2444879|2445151|2444516|2444789|N|N|N|N|N| +2444882|AAAAAAAACFOEFCAA|1981-10-04|981|4266|328|1981|0|10|4|4|1981|328|4266|Sunday|1981Q4|N|N|N|2444879|2445151|2444517|2444790|N|N|N|N|N| +2444883|AAAAAAAADFOEFCAA|1981-10-05|981|4266|328|1981|1|10|5|4|1981|328|4266|Monday|1981Q4|N|N|N|2444879|2445151|2444518|2444791|N|N|N|N|N| +2444884|AAAAAAAAEFOEFCAA|1981-10-06|981|4267|328|1981|2|10|6|4|1981|328|4267|Tuesday|1981Q4|N|N|N|2444879|2445151|2444519|2444792|N|N|N|N|N| +2444885|AAAAAAAAFFOEFCAA|1981-10-07|981|4267|328|1981|3|10|7|4|1981|328|4267|Wednesday|1981Q4|N|N|N|2444879|2445151|2444520|2444793|N|N|N|N|N| +2444886|AAAAAAAAGFOEFCAA|1981-10-08|981|4267|328|1981|4|10|8|4|1981|328|4267|Thursday|1981Q4|N|N|N|2444879|2445151|2444521|2444794|N|N|N|N|N| +2444887|AAAAAAAAHFOEFCAA|1981-10-09|981|4267|328|1981|5|10|9|4|1981|328|4267|Friday|1981Q4|N|Y|N|2444879|2445151|2444522|2444795|N|N|N|N|N| +2444888|AAAAAAAAIFOEFCAA|1981-10-10|981|4267|328|1981|6|10|10|4|1981|328|4267|Saturday|1981Q4|N|Y|N|2444879|2445151|2444523|2444796|N|N|N|N|N| +2444889|AAAAAAAAJFOEFCAA|1981-10-11|981|4267|328|1981|0|10|11|4|1981|328|4267|Sunday|1981Q4|N|N|N|2444879|2445151|2444524|2444797|N|N|N|N|N| +2444890|AAAAAAAAKFOEFCAA|1981-10-12|981|4267|328|1981|1|10|12|4|1981|328|4267|Monday|1981Q4|N|N|N|2444879|2445151|2444525|2444798|N|N|N|N|N| +2444891|AAAAAAAALFOEFCAA|1981-10-13|981|4268|328|1981|2|10|13|4|1981|328|4268|Tuesday|1981Q4|N|N|N|2444879|2445151|2444526|2444799|N|N|N|N|N| +2444892|AAAAAAAAMFOEFCAA|1981-10-14|981|4268|328|1981|3|10|14|4|1981|328|4268|Wednesday|1981Q4|N|N|N|2444879|2445151|2444527|2444800|N|N|N|N|N| +2444893|AAAAAAAANFOEFCAA|1981-10-15|981|4268|328|1981|4|10|15|4|1981|328|4268|Thursday|1981Q4|N|N|N|2444879|2445151|2444528|2444801|N|N|N|N|N| +2444894|AAAAAAAAOFOEFCAA|1981-10-16|981|4268|328|1981|5|10|16|4|1981|328|4268|Friday|1981Q4|N|Y|N|2444879|2445151|2444529|2444802|N|N|N|N|N| +2444895|AAAAAAAAPFOEFCAA|1981-10-17|981|4268|328|1981|6|10|17|4|1981|328|4268|Saturday|1981Q4|N|Y|N|2444879|2445151|2444530|2444803|N|N|N|N|N| +2444896|AAAAAAAAAGOEFCAA|1981-10-18|981|4268|328|1981|0|10|18|4|1981|328|4268|Sunday|1981Q4|N|N|N|2444879|2445151|2444531|2444804|N|N|N|N|N| +2444897|AAAAAAAABGOEFCAA|1981-10-19|981|4268|328|1981|1|10|19|4|1981|328|4268|Monday|1981Q4|N|N|N|2444879|2445151|2444532|2444805|N|N|N|N|N| +2444898|AAAAAAAACGOEFCAA|1981-10-20|981|4269|328|1981|2|10|20|4|1981|328|4269|Tuesday|1981Q4|N|N|N|2444879|2445151|2444533|2444806|N|N|N|N|N| +2444899|AAAAAAAADGOEFCAA|1981-10-21|981|4269|328|1981|3|10|21|4|1981|328|4269|Wednesday|1981Q4|N|N|N|2444879|2445151|2444534|2444807|N|N|N|N|N| +2444900|AAAAAAAAEGOEFCAA|1981-10-22|981|4269|328|1981|4|10|22|4|1981|328|4269|Thursday|1981Q4|N|N|N|2444879|2445151|2444535|2444808|N|N|N|N|N| +2444901|AAAAAAAAFGOEFCAA|1981-10-23|981|4269|328|1981|5|10|23|4|1981|328|4269|Friday|1981Q4|N|Y|N|2444879|2445151|2444536|2444809|N|N|N|N|N| +2444902|AAAAAAAAGGOEFCAA|1981-10-24|981|4269|328|1981|6|10|24|4|1981|328|4269|Saturday|1981Q4|N|Y|N|2444879|2445151|2444537|2444810|N|N|N|N|N| +2444903|AAAAAAAAHGOEFCAA|1981-10-25|981|4269|328|1981|0|10|25|4|1981|328|4269|Sunday|1981Q4|N|N|N|2444879|2445151|2444538|2444811|N|N|N|N|N| +2444904|AAAAAAAAIGOEFCAA|1981-10-26|981|4269|328|1981|1|10|26|4|1981|328|4269|Monday|1981Q4|N|N|N|2444879|2445151|2444539|2444812|N|N|N|N|N| +2444905|AAAAAAAAJGOEFCAA|1981-10-27|981|4270|328|1981|2|10|27|4|1981|328|4270|Tuesday|1981Q4|N|N|N|2444879|2445151|2444540|2444813|N|N|N|N|N| +2444906|AAAAAAAAKGOEFCAA|1981-10-28|981|4270|328|1981|3|10|28|4|1981|328|4270|Wednesday|1981Q4|N|N|N|2444879|2445151|2444541|2444814|N|N|N|N|N| +2444907|AAAAAAAALGOEFCAA|1981-10-29|981|4270|328|1981|4|10|29|4|1981|328|4270|Thursday|1981Q4|N|N|N|2444879|2445151|2444542|2444815|N|N|N|N|N| +2444908|AAAAAAAAMGOEFCAA|1981-10-30|981|4270|328|1981|5|10|30|4|1981|328|4270|Friday|1981Q4|N|Y|N|2444879|2445151|2444543|2444816|N|N|N|N|N| +2444909|AAAAAAAANGOEFCAA|1981-10-31|981|4270|328|1981|6|10|31|4|1981|328|4270|Saturday|1981Q4|N|Y|N|2444879|2445151|2444544|2444817|N|N|N|N|N| +2444910|AAAAAAAAOGOEFCAA|1981-11-01|982|4270|328|1981|0|11|1|4|1981|328|4270|Sunday|1981Q4|N|N|N|2444910|2445213|2444545|2444818|N|N|N|N|N| +2444911|AAAAAAAAPGOEFCAA|1981-11-02|982|4270|328|1981|1|11|2|4|1981|328|4270|Monday|1981Q4|N|N|N|2444910|2445213|2444546|2444819|N|N|N|N|N| +2444912|AAAAAAAAAHOEFCAA|1981-11-03|982|4271|328|1981|2|11|3|4|1981|328|4271|Tuesday|1981Q4|N|N|N|2444910|2445213|2444547|2444820|N|N|N|N|N| +2444913|AAAAAAAABHOEFCAA|1981-11-04|982|4271|328|1981|3|11|4|4|1981|328|4271|Wednesday|1981Q4|N|N|N|2444910|2445213|2444548|2444821|N|N|N|N|N| +2444914|AAAAAAAACHOEFCAA|1981-11-05|982|4271|328|1981|4|11|5|4|1981|328|4271|Thursday|1981Q4|N|N|N|2444910|2445213|2444549|2444822|N|N|N|N|N| +2444915|AAAAAAAADHOEFCAA|1981-11-06|982|4271|328|1981|5|11|6|4|1981|328|4271|Friday|1981Q4|N|Y|N|2444910|2445213|2444550|2444823|N|N|N|N|N| +2444916|AAAAAAAAEHOEFCAA|1981-11-07|982|4271|328|1981|6|11|7|4|1981|328|4271|Saturday|1981Q4|N|Y|N|2444910|2445213|2444551|2444824|N|N|N|N|N| +2444917|AAAAAAAAFHOEFCAA|1981-11-08|982|4271|328|1981|0|11|8|4|1981|328|4271|Sunday|1981Q4|N|N|N|2444910|2445213|2444552|2444825|N|N|N|N|N| +2444918|AAAAAAAAGHOEFCAA|1981-11-09|982|4271|328|1981|1|11|9|4|1981|328|4271|Monday|1981Q4|N|N|N|2444910|2445213|2444553|2444826|N|N|N|N|N| +2444919|AAAAAAAAHHOEFCAA|1981-11-10|982|4272|328|1981|2|11|10|4|1981|328|4272|Tuesday|1981Q4|N|N|N|2444910|2445213|2444554|2444827|N|N|N|N|N| +2444920|AAAAAAAAIHOEFCAA|1981-11-11|982|4272|328|1981|3|11|11|4|1981|328|4272|Wednesday|1981Q4|N|N|N|2444910|2445213|2444555|2444828|N|N|N|N|N| +2444921|AAAAAAAAJHOEFCAA|1981-11-12|982|4272|328|1981|4|11|12|4|1981|328|4272|Thursday|1981Q4|N|N|N|2444910|2445213|2444556|2444829|N|N|N|N|N| +2444922|AAAAAAAAKHOEFCAA|1981-11-13|982|4272|328|1981|5|11|13|4|1981|328|4272|Friday|1981Q4|N|Y|N|2444910|2445213|2444557|2444830|N|N|N|N|N| +2444923|AAAAAAAALHOEFCAA|1981-11-14|982|4272|328|1981|6|11|14|4|1981|328|4272|Saturday|1981Q4|N|Y|N|2444910|2445213|2444558|2444831|N|N|N|N|N| +2444924|AAAAAAAAMHOEFCAA|1981-11-15|982|4272|328|1981|0|11|15|4|1981|328|4272|Sunday|1981Q4|N|N|N|2444910|2445213|2444559|2444832|N|N|N|N|N| +2444925|AAAAAAAANHOEFCAA|1981-11-16|982|4272|328|1981|1|11|16|4|1981|328|4272|Monday|1981Q4|N|N|N|2444910|2445213|2444560|2444833|N|N|N|N|N| +2444926|AAAAAAAAOHOEFCAA|1981-11-17|982|4273|328|1981|2|11|17|4|1981|328|4273|Tuesday|1981Q4|N|N|N|2444910|2445213|2444561|2444834|N|N|N|N|N| +2444927|AAAAAAAAPHOEFCAA|1981-11-18|982|4273|328|1981|3|11|18|4|1981|328|4273|Wednesday|1981Q4|N|N|N|2444910|2445213|2444562|2444835|N|N|N|N|N| +2444928|AAAAAAAAAIOEFCAA|1981-11-19|982|4273|328|1981|4|11|19|4|1981|328|4273|Thursday|1981Q4|N|N|N|2444910|2445213|2444563|2444836|N|N|N|N|N| +2444929|AAAAAAAABIOEFCAA|1981-11-20|982|4273|328|1981|5|11|20|4|1981|328|4273|Friday|1981Q4|N|Y|N|2444910|2445213|2444564|2444837|N|N|N|N|N| +2444930|AAAAAAAACIOEFCAA|1981-11-21|982|4273|328|1981|6|11|21|4|1981|328|4273|Saturday|1981Q4|N|Y|N|2444910|2445213|2444565|2444838|N|N|N|N|N| +2444931|AAAAAAAADIOEFCAA|1981-11-22|982|4273|328|1981|0|11|22|4|1981|328|4273|Sunday|1981Q4|N|N|N|2444910|2445213|2444566|2444839|N|N|N|N|N| +2444932|AAAAAAAAEIOEFCAA|1981-11-23|982|4273|328|1981|1|11|23|4|1981|328|4273|Monday|1981Q4|N|N|N|2444910|2445213|2444567|2444840|N|N|N|N|N| +2444933|AAAAAAAAFIOEFCAA|1981-11-24|982|4274|328|1981|2|11|24|4|1981|328|4274|Tuesday|1981Q4|N|N|N|2444910|2445213|2444568|2444841|N|N|N|N|N| +2444934|AAAAAAAAGIOEFCAA|1981-11-25|982|4274|328|1981|3|11|25|4|1981|328|4274|Wednesday|1981Q4|N|N|N|2444910|2445213|2444569|2444842|N|N|N|N|N| +2444935|AAAAAAAAHIOEFCAA|1981-11-26|982|4274|328|1981|4|11|26|4|1981|328|4274|Thursday|1981Q4|N|N|N|2444910|2445213|2444570|2444843|N|N|N|N|N| +2444936|AAAAAAAAIIOEFCAA|1981-11-27|982|4274|328|1981|5|11|27|4|1981|328|4274|Friday|1981Q4|N|Y|N|2444910|2445213|2444571|2444844|N|N|N|N|N| +2444937|AAAAAAAAJIOEFCAA|1981-11-28|982|4274|328|1981|6|11|28|4|1981|328|4274|Saturday|1981Q4|N|Y|N|2444910|2445213|2444572|2444845|N|N|N|N|N| +2444938|AAAAAAAAKIOEFCAA|1981-11-29|982|4274|328|1981|0|11|29|4|1981|328|4274|Sunday|1981Q4|N|N|N|2444910|2445213|2444573|2444846|N|N|N|N|N| +2444939|AAAAAAAALIOEFCAA|1981-11-30|982|4274|328|1981|1|11|30|4|1981|328|4274|Monday|1981Q4|N|N|N|2444910|2445213|2444574|2444847|N|N|N|N|N| +2444940|AAAAAAAAMIOEFCAA|1981-12-01|983|4275|329|1981|2|12|1|4|1981|329|4275|Tuesday|1981Q4|N|N|N|2444940|2445273|2444575|2444848|N|N|N|N|N| +2444941|AAAAAAAANIOEFCAA|1981-12-02|983|4275|329|1981|3|12|2|4|1981|329|4275|Wednesday|1981Q4|N|N|N|2444940|2445273|2444576|2444849|N|N|N|N|N| +2444942|AAAAAAAAOIOEFCAA|1981-12-03|983|4275|329|1981|4|12|3|4|1981|329|4275|Thursday|1981Q4|N|N|N|2444940|2445273|2444577|2444850|N|N|N|N|N| +2444943|AAAAAAAAPIOEFCAA|1981-12-04|983|4275|329|1981|5|12|4|4|1981|329|4275|Friday|1981Q4|N|Y|N|2444940|2445273|2444578|2444851|N|N|N|N|N| +2444944|AAAAAAAAAJOEFCAA|1981-12-05|983|4275|329|1981|6|12|5|4|1981|329|4275|Saturday|1981Q4|N|Y|N|2444940|2445273|2444579|2444852|N|N|N|N|N| +2444945|AAAAAAAABJOEFCAA|1981-12-06|983|4275|329|1981|0|12|6|4|1981|329|4275|Sunday|1981Q4|N|N|N|2444940|2445273|2444580|2444853|N|N|N|N|N| +2444946|AAAAAAAACJOEFCAA|1981-12-07|983|4275|329|1981|1|12|7|4|1981|329|4275|Monday|1981Q4|N|N|N|2444940|2445273|2444581|2444854|N|N|N|N|N| +2444947|AAAAAAAADJOEFCAA|1981-12-08|983|4276|329|1981|2|12|8|4|1981|329|4276|Tuesday|1981Q4|N|N|N|2444940|2445273|2444582|2444855|N|N|N|N|N| +2444948|AAAAAAAAEJOEFCAA|1981-12-09|983|4276|329|1981|3|12|9|4|1981|329|4276|Wednesday|1981Q4|N|N|N|2444940|2445273|2444583|2444856|N|N|N|N|N| +2444949|AAAAAAAAFJOEFCAA|1981-12-10|983|4276|329|1981|4|12|10|4|1981|329|4276|Thursday|1981Q4|N|N|N|2444940|2445273|2444584|2444857|N|N|N|N|N| +2444950|AAAAAAAAGJOEFCAA|1981-12-11|983|4276|329|1981|5|12|11|4|1981|329|4276|Friday|1981Q4|N|Y|N|2444940|2445273|2444585|2444858|N|N|N|N|N| +2444951|AAAAAAAAHJOEFCAA|1981-12-12|983|4276|329|1981|6|12|12|4|1981|329|4276|Saturday|1981Q4|N|Y|N|2444940|2445273|2444586|2444859|N|N|N|N|N| +2444952|AAAAAAAAIJOEFCAA|1981-12-13|983|4276|329|1981|0|12|13|4|1981|329|4276|Sunday|1981Q4|N|N|N|2444940|2445273|2444587|2444860|N|N|N|N|N| +2444953|AAAAAAAAJJOEFCAA|1981-12-14|983|4276|329|1981|1|12|14|4|1981|329|4276|Monday|1981Q4|N|N|N|2444940|2445273|2444588|2444861|N|N|N|N|N| +2444954|AAAAAAAAKJOEFCAA|1981-12-15|983|4277|329|1981|2|12|15|4|1981|329|4277|Tuesday|1981Q4|N|N|N|2444940|2445273|2444589|2444862|N|N|N|N|N| +2444955|AAAAAAAALJOEFCAA|1981-12-16|983|4277|329|1981|3|12|16|4|1981|329|4277|Wednesday|1981Q4|N|N|N|2444940|2445273|2444590|2444863|N|N|N|N|N| +2444956|AAAAAAAAMJOEFCAA|1981-12-17|983|4277|329|1981|4|12|17|4|1981|329|4277|Thursday|1981Q4|N|N|N|2444940|2445273|2444591|2444864|N|N|N|N|N| +2444957|AAAAAAAANJOEFCAA|1981-12-18|983|4277|329|1981|5|12|18|4|1981|329|4277|Friday|1981Q4|N|Y|N|2444940|2445273|2444592|2444865|N|N|N|N|N| +2444958|AAAAAAAAOJOEFCAA|1981-12-19|983|4277|329|1981|6|12|19|4|1981|329|4277|Saturday|1981Q4|N|Y|N|2444940|2445273|2444593|2444866|N|N|N|N|N| +2444959|AAAAAAAAPJOEFCAA|1981-12-20|983|4277|329|1981|0|12|20|4|1981|329|4277|Sunday|1981Q4|N|N|N|2444940|2445273|2444594|2444867|N|N|N|N|N| +2444960|AAAAAAAAAKOEFCAA|1981-12-21|983|4277|329|1981|1|12|21|4|1981|329|4277|Monday|1981Q4|N|N|N|2444940|2445273|2444595|2444868|N|N|N|N|N| +2444961|AAAAAAAABKOEFCAA|1981-12-22|983|4278|329|1981|2|12|22|4|1981|329|4278|Tuesday|1981Q4|N|N|N|2444940|2445273|2444596|2444869|N|N|N|N|N| +2444962|AAAAAAAACKOEFCAA|1981-12-23|983|4278|329|1981|3|12|23|4|1981|329|4278|Wednesday|1981Q4|N|N|N|2444940|2445273|2444597|2444870|N|N|N|N|N| +2444963|AAAAAAAADKOEFCAA|1981-12-24|983|4278|329|1981|4|12|24|4|1981|329|4278|Thursday|1981Q4|N|N|N|2444940|2445273|2444598|2444871|N|N|N|N|N| +2444964|AAAAAAAAEKOEFCAA|1981-12-25|983|4278|329|1981|5|12|25|4|1981|329|4278|Friday|1981Q4|N|Y|N|2444940|2445273|2444599|2444872|N|N|N|N|N| +2444965|AAAAAAAAFKOEFCAA|1981-12-26|983|4278|329|1981|6|12|26|4|1981|329|4278|Saturday|1981Q4|Y|Y|N|2444940|2445273|2444600|2444873|N|N|N|N|N| +2444966|AAAAAAAAGKOEFCAA|1981-12-27|983|4278|329|1981|0|12|27|4|1981|329|4278|Sunday|1981Q4|N|N|Y|2444940|2445273|2444601|2444874|N|N|N|N|N| +2444967|AAAAAAAAHKOEFCAA|1981-12-28|983|4278|329|1981|1|12|28|4|1981|329|4278|Monday|1981Q4|N|N|N|2444940|2445273|2444602|2444875|N|N|N|N|N| +2444968|AAAAAAAAIKOEFCAA|1981-12-29|983|4279|329|1981|2|12|29|4|1981|329|4279|Tuesday|1981Q4|N|N|N|2444940|2445273|2444603|2444876|N|N|N|N|N| +2444969|AAAAAAAAJKOEFCAA|1981-12-30|983|4279|329|1981|3|12|30|4|1981|329|4279|Wednesday|1981Q4|N|N|N|2444940|2445273|2444604|2444877|N|N|N|N|N| +2444970|AAAAAAAAKKOEFCAA|1981-12-31|983|4279|329|1981|4|12|31|4|1981|329|4279|Thursday|1981Q4|N|N|N|2444940|2445273|2444605|2444878|N|N|N|N|N| +2444971|AAAAAAAALKOEFCAA|1982-01-01|984|4279|329|1982|5|1|1|1|1982|329|4279|Friday|1982Q1|Y|Y|N|2444971|2444970|2444606|2444879|N|N|N|N|N| +2444972|AAAAAAAAMKOEFCAA|1982-01-02|984|4279|329|1982|6|1|2|1|1982|329|4279|Saturday|1982Q1|N|Y|Y|2444971|2444970|2444607|2444880|N|N|N|N|N| +2444973|AAAAAAAANKOEFCAA|1982-01-03|984|4279|329|1982|0|1|3|1|1982|329|4279|Sunday|1982Q1|N|N|N|2444971|2444970|2444608|2444881|N|N|N|N|N| +2444974|AAAAAAAAOKOEFCAA|1982-01-04|984|4279|329|1982|1|1|4|1|1982|329|4279|Monday|1982Q1|N|N|N|2444971|2444970|2444609|2444882|N|N|N|N|N| +2444975|AAAAAAAAPKOEFCAA|1982-01-05|984|4280|329|1982|2|1|5|1|1982|329|4280|Tuesday|1982Q1|N|N|N|2444971|2444970|2444610|2444883|N|N|N|N|N| +2444976|AAAAAAAAALOEFCAA|1982-01-06|984|4280|329|1982|3|1|6|1|1982|329|4280|Wednesday|1982Q1|N|N|N|2444971|2444970|2444611|2444884|N|N|N|N|N| +2444977|AAAAAAAABLOEFCAA|1982-01-07|984|4280|329|1982|4|1|7|1|1982|329|4280|Thursday|1982Q1|N|N|N|2444971|2444970|2444612|2444885|N|N|N|N|N| +2444978|AAAAAAAACLOEFCAA|1982-01-08|984|4280|329|1982|5|1|8|1|1982|329|4280|Friday|1982Q1|N|Y|N|2444971|2444970|2444613|2444886|N|N|N|N|N| +2444979|AAAAAAAADLOEFCAA|1982-01-09|984|4280|329|1982|6|1|9|1|1982|329|4280|Saturday|1982Q1|N|Y|N|2444971|2444970|2444614|2444887|N|N|N|N|N| +2444980|AAAAAAAAELOEFCAA|1982-01-10|984|4280|329|1982|0|1|10|1|1982|329|4280|Sunday|1982Q1|N|N|N|2444971|2444970|2444615|2444888|N|N|N|N|N| +2444981|AAAAAAAAFLOEFCAA|1982-01-11|984|4280|329|1982|1|1|11|1|1982|329|4280|Monday|1982Q1|N|N|N|2444971|2444970|2444616|2444889|N|N|N|N|N| +2444982|AAAAAAAAGLOEFCAA|1982-01-12|984|4281|329|1982|2|1|12|1|1982|329|4281|Tuesday|1982Q1|N|N|N|2444971|2444970|2444617|2444890|N|N|N|N|N| +2444983|AAAAAAAAHLOEFCAA|1982-01-13|984|4281|329|1982|3|1|13|1|1982|329|4281|Wednesday|1982Q1|N|N|N|2444971|2444970|2444618|2444891|N|N|N|N|N| +2444984|AAAAAAAAILOEFCAA|1982-01-14|984|4281|329|1982|4|1|14|1|1982|329|4281|Thursday|1982Q1|N|N|N|2444971|2444970|2444619|2444892|N|N|N|N|N| +2444985|AAAAAAAAJLOEFCAA|1982-01-15|984|4281|329|1982|5|1|15|1|1982|329|4281|Friday|1982Q1|N|Y|N|2444971|2444970|2444620|2444893|N|N|N|N|N| +2444986|AAAAAAAAKLOEFCAA|1982-01-16|984|4281|329|1982|6|1|16|1|1982|329|4281|Saturday|1982Q1|N|Y|N|2444971|2444970|2444621|2444894|N|N|N|N|N| +2444987|AAAAAAAALLOEFCAA|1982-01-17|984|4281|329|1982|0|1|17|1|1982|329|4281|Sunday|1982Q1|N|N|N|2444971|2444970|2444622|2444895|N|N|N|N|N| +2444988|AAAAAAAAMLOEFCAA|1982-01-18|984|4281|329|1982|1|1|18|1|1982|329|4281|Monday|1982Q1|N|N|N|2444971|2444970|2444623|2444896|N|N|N|N|N| +2444989|AAAAAAAANLOEFCAA|1982-01-19|984|4282|329|1982|2|1|19|1|1982|329|4282|Tuesday|1982Q1|N|N|N|2444971|2444970|2444624|2444897|N|N|N|N|N| +2444990|AAAAAAAAOLOEFCAA|1982-01-20|984|4282|329|1982|3|1|20|1|1982|329|4282|Wednesday|1982Q1|N|N|N|2444971|2444970|2444625|2444898|N|N|N|N|N| +2444991|AAAAAAAAPLOEFCAA|1982-01-21|984|4282|329|1982|4|1|21|1|1982|329|4282|Thursday|1982Q1|N|N|N|2444971|2444970|2444626|2444899|N|N|N|N|N| +2444992|AAAAAAAAAMOEFCAA|1982-01-22|984|4282|329|1982|5|1|22|1|1982|329|4282|Friday|1982Q1|N|Y|N|2444971|2444970|2444627|2444900|N|N|N|N|N| +2444993|AAAAAAAABMOEFCAA|1982-01-23|984|4282|329|1982|6|1|23|1|1982|329|4282|Saturday|1982Q1|N|Y|N|2444971|2444970|2444628|2444901|N|N|N|N|N| +2444994|AAAAAAAACMOEFCAA|1982-01-24|984|4282|329|1982|0|1|24|1|1982|329|4282|Sunday|1982Q1|N|N|N|2444971|2444970|2444629|2444902|N|N|N|N|N| +2444995|AAAAAAAADMOEFCAA|1982-01-25|984|4282|329|1982|1|1|25|1|1982|329|4282|Monday|1982Q1|N|N|N|2444971|2444970|2444630|2444903|N|N|N|N|N| +2444996|AAAAAAAAEMOEFCAA|1982-01-26|984|4283|329|1982|2|1|26|1|1982|329|4283|Tuesday|1982Q1|N|N|N|2444971|2444970|2444631|2444904|N|N|N|N|N| +2444997|AAAAAAAAFMOEFCAA|1982-01-27|984|4283|329|1982|3|1|27|1|1982|329|4283|Wednesday|1982Q1|N|N|N|2444971|2444970|2444632|2444905|N|N|N|N|N| +2444998|AAAAAAAAGMOEFCAA|1982-01-28|984|4283|329|1982|4|1|28|1|1982|329|4283|Thursday|1982Q1|N|N|N|2444971|2444970|2444633|2444906|N|N|N|N|N| +2444999|AAAAAAAAHMOEFCAA|1982-01-29|984|4283|329|1982|5|1|29|1|1982|329|4283|Friday|1982Q1|N|Y|N|2444971|2444970|2444634|2444907|N|N|N|N|N| +2445000|AAAAAAAAIMOEFCAA|1982-01-30|984|4283|329|1982|6|1|30|1|1982|329|4283|Saturday|1982Q1|N|Y|N|2444971|2444970|2444635|2444908|N|N|N|N|N| +2445001|AAAAAAAAJMOEFCAA|1982-01-31|984|4283|329|1982|0|1|31|1|1982|329|4283|Sunday|1982Q1|N|N|N|2444971|2444970|2444636|2444909|N|N|N|N|N| +2445002|AAAAAAAAKMOEFCAA|1982-02-01|985|4283|329|1982|1|2|1|1|1982|329|4283|Monday|1982Q1|N|N|N|2445002|2445032|2444637|2444910|N|N|N|N|N| +2445003|AAAAAAAALMOEFCAA|1982-02-02|985|4284|329|1982|2|2|2|1|1982|329|4284|Tuesday|1982Q1|N|N|N|2445002|2445032|2444638|2444911|N|N|N|N|N| +2445004|AAAAAAAAMMOEFCAA|1982-02-03|985|4284|329|1982|3|2|3|1|1982|329|4284|Wednesday|1982Q1|N|N|N|2445002|2445032|2444639|2444912|N|N|N|N|N| +2445005|AAAAAAAANMOEFCAA|1982-02-04|985|4284|329|1982|4|2|4|1|1982|329|4284|Thursday|1982Q1|N|N|N|2445002|2445032|2444640|2444913|N|N|N|N|N| +2445006|AAAAAAAAOMOEFCAA|1982-02-05|985|4284|329|1982|5|2|5|1|1982|329|4284|Friday|1982Q1|N|Y|N|2445002|2445032|2444641|2444914|N|N|N|N|N| +2445007|AAAAAAAAPMOEFCAA|1982-02-06|985|4284|329|1982|6|2|6|1|1982|329|4284|Saturday|1982Q1|N|Y|N|2445002|2445032|2444642|2444915|N|N|N|N|N| +2445008|AAAAAAAAANOEFCAA|1982-02-07|985|4284|329|1982|0|2|7|1|1982|329|4284|Sunday|1982Q1|N|N|N|2445002|2445032|2444643|2444916|N|N|N|N|N| +2445009|AAAAAAAABNOEFCAA|1982-02-08|985|4284|329|1982|1|2|8|1|1982|329|4284|Monday|1982Q1|N|N|N|2445002|2445032|2444644|2444917|N|N|N|N|N| +2445010|AAAAAAAACNOEFCAA|1982-02-09|985|4285|329|1982|2|2|9|1|1982|329|4285|Tuesday|1982Q1|N|N|N|2445002|2445032|2444645|2444918|N|N|N|N|N| +2445011|AAAAAAAADNOEFCAA|1982-02-10|985|4285|329|1982|3|2|10|1|1982|329|4285|Wednesday|1982Q1|N|N|N|2445002|2445032|2444646|2444919|N|N|N|N|N| +2445012|AAAAAAAAENOEFCAA|1982-02-11|985|4285|329|1982|4|2|11|1|1982|329|4285|Thursday|1982Q1|N|N|N|2445002|2445032|2444647|2444920|N|N|N|N|N| +2445013|AAAAAAAAFNOEFCAA|1982-02-12|985|4285|329|1982|5|2|12|1|1982|329|4285|Friday|1982Q1|N|Y|N|2445002|2445032|2444648|2444921|N|N|N|N|N| +2445014|AAAAAAAAGNOEFCAA|1982-02-13|985|4285|329|1982|6|2|13|1|1982|329|4285|Saturday|1982Q1|N|Y|N|2445002|2445032|2444649|2444922|N|N|N|N|N| +2445015|AAAAAAAAHNOEFCAA|1982-02-14|985|4285|329|1982|0|2|14|1|1982|329|4285|Sunday|1982Q1|N|N|N|2445002|2445032|2444650|2444923|N|N|N|N|N| +2445016|AAAAAAAAINOEFCAA|1982-02-15|985|4285|329|1982|1|2|15|1|1982|329|4285|Monday|1982Q1|N|N|N|2445002|2445032|2444651|2444924|N|N|N|N|N| +2445017|AAAAAAAAJNOEFCAA|1982-02-16|985|4286|329|1982|2|2|16|1|1982|329|4286|Tuesday|1982Q1|N|N|N|2445002|2445032|2444652|2444925|N|N|N|N|N| +2445018|AAAAAAAAKNOEFCAA|1982-02-17|985|4286|329|1982|3|2|17|1|1982|329|4286|Wednesday|1982Q1|N|N|N|2445002|2445032|2444653|2444926|N|N|N|N|N| +2445019|AAAAAAAALNOEFCAA|1982-02-18|985|4286|329|1982|4|2|18|1|1982|329|4286|Thursday|1982Q1|N|N|N|2445002|2445032|2444654|2444927|N|N|N|N|N| +2445020|AAAAAAAAMNOEFCAA|1982-02-19|985|4286|329|1982|5|2|19|1|1982|329|4286|Friday|1982Q1|N|Y|N|2445002|2445032|2444655|2444928|N|N|N|N|N| +2445021|AAAAAAAANNOEFCAA|1982-02-20|985|4286|329|1982|6|2|20|1|1982|329|4286|Saturday|1982Q1|N|Y|N|2445002|2445032|2444656|2444929|N|N|N|N|N| +2445022|AAAAAAAAONOEFCAA|1982-02-21|985|4286|329|1982|0|2|21|1|1982|329|4286|Sunday|1982Q1|N|N|N|2445002|2445032|2444657|2444930|N|N|N|N|N| +2445023|AAAAAAAAPNOEFCAA|1982-02-22|985|4286|329|1982|1|2|22|1|1982|329|4286|Monday|1982Q1|N|N|N|2445002|2445032|2444658|2444931|N|N|N|N|N| +2445024|AAAAAAAAAOOEFCAA|1982-02-23|985|4287|329|1982|2|2|23|1|1982|329|4287|Tuesday|1982Q1|N|N|N|2445002|2445032|2444659|2444932|N|N|N|N|N| +2445025|AAAAAAAABOOEFCAA|1982-02-24|985|4287|329|1982|3|2|24|1|1982|329|4287|Wednesday|1982Q1|N|N|N|2445002|2445032|2444660|2444933|N|N|N|N|N| +2445026|AAAAAAAACOOEFCAA|1982-02-25|985|4287|329|1982|4|2|25|1|1982|329|4287|Thursday|1982Q1|N|N|N|2445002|2445032|2444661|2444934|N|N|N|N|N| +2445027|AAAAAAAADOOEFCAA|1982-02-26|985|4287|329|1982|5|2|26|1|1982|329|4287|Friday|1982Q1|N|Y|N|2445002|2445032|2444662|2444935|N|N|N|N|N| +2445028|AAAAAAAAEOOEFCAA|1982-02-27|985|4287|329|1982|6|2|27|1|1982|329|4287|Saturday|1982Q1|N|Y|N|2445002|2445032|2444663|2444936|N|N|N|N|N| +2445029|AAAAAAAAFOOEFCAA|1982-02-28|985|4287|329|1982|0|2|28|1|1982|329|4287|Sunday|1982Q1|N|N|N|2445002|2445032|2444664|2444937|N|N|N|N|N| +2445030|AAAAAAAAGOOEFCAA|1982-03-01|986|4287|330|1982|1|3|1|1|1982|330|4287|Monday|1982Q1|N|N|N|2445030|2445088|2444665|2444938|N|N|N|N|N| +2445031|AAAAAAAAHOOEFCAA|1982-03-02|986|4288|330|1982|2|3|2|1|1982|330|4288|Tuesday|1982Q1|N|N|N|2445030|2445088|2444666|2444939|N|N|N|N|N| +2445032|AAAAAAAAIOOEFCAA|1982-03-03|986|4288|330|1982|3|3|3|1|1982|330|4288|Wednesday|1982Q1|N|N|N|2445030|2445088|2444667|2444940|N|N|N|N|N| +2445033|AAAAAAAAJOOEFCAA|1982-03-04|986|4288|330|1982|4|3|4|1|1982|330|4288|Thursday|1982Q1|N|N|N|2445030|2445088|2444668|2444941|N|N|N|N|N| +2445034|AAAAAAAAKOOEFCAA|1982-03-05|986|4288|330|1982|5|3|5|1|1982|330|4288|Friday|1982Q1|N|Y|N|2445030|2445088|2444669|2444942|N|N|N|N|N| +2445035|AAAAAAAALOOEFCAA|1982-03-06|986|4288|330|1982|6|3|6|1|1982|330|4288|Saturday|1982Q1|N|Y|N|2445030|2445088|2444670|2444943|N|N|N|N|N| +2445036|AAAAAAAAMOOEFCAA|1982-03-07|986|4288|330|1982|0|3|7|1|1982|330|4288|Sunday|1982Q1|N|N|N|2445030|2445088|2444671|2444944|N|N|N|N|N| +2445037|AAAAAAAANOOEFCAA|1982-03-08|986|4288|330|1982|1|3|8|1|1982|330|4288|Monday|1982Q1|N|N|N|2445030|2445088|2444672|2444945|N|N|N|N|N| +2445038|AAAAAAAAOOOEFCAA|1982-03-09|986|4289|330|1982|2|3|9|1|1982|330|4289|Tuesday|1982Q1|N|N|N|2445030|2445088|2444673|2444946|N|N|N|N|N| +2445039|AAAAAAAAPOOEFCAA|1982-03-10|986|4289|330|1982|3|3|10|1|1982|330|4289|Wednesday|1982Q1|N|N|N|2445030|2445088|2444674|2444947|N|N|N|N|N| +2445040|AAAAAAAAAPOEFCAA|1982-03-11|986|4289|330|1982|4|3|11|1|1982|330|4289|Thursday|1982Q1|N|N|N|2445030|2445088|2444675|2444948|N|N|N|N|N| +2445041|AAAAAAAABPOEFCAA|1982-03-12|986|4289|330|1982|5|3|12|1|1982|330|4289|Friday|1982Q1|N|Y|N|2445030|2445088|2444676|2444949|N|N|N|N|N| +2445042|AAAAAAAACPOEFCAA|1982-03-13|986|4289|330|1982|6|3|13|1|1982|330|4289|Saturday|1982Q1|N|Y|N|2445030|2445088|2444677|2444950|N|N|N|N|N| +2445043|AAAAAAAADPOEFCAA|1982-03-14|986|4289|330|1982|0|3|14|1|1982|330|4289|Sunday|1982Q1|N|N|N|2445030|2445088|2444678|2444951|N|N|N|N|N| +2445044|AAAAAAAAEPOEFCAA|1982-03-15|986|4289|330|1982|1|3|15|1|1982|330|4289|Monday|1982Q1|N|N|N|2445030|2445088|2444679|2444952|N|N|N|N|N| +2445045|AAAAAAAAFPOEFCAA|1982-03-16|986|4290|330|1982|2|3|16|1|1982|330|4290|Tuesday|1982Q1|N|N|N|2445030|2445088|2444680|2444953|N|N|N|N|N| +2445046|AAAAAAAAGPOEFCAA|1982-03-17|986|4290|330|1982|3|3|17|1|1982|330|4290|Wednesday|1982Q1|N|N|N|2445030|2445088|2444681|2444954|N|N|N|N|N| +2445047|AAAAAAAAHPOEFCAA|1982-03-18|986|4290|330|1982|4|3|18|1|1982|330|4290|Thursday|1982Q1|N|N|N|2445030|2445088|2444682|2444955|N|N|N|N|N| +2445048|AAAAAAAAIPOEFCAA|1982-03-19|986|4290|330|1982|5|3|19|1|1982|330|4290|Friday|1982Q1|N|Y|N|2445030|2445088|2444683|2444956|N|N|N|N|N| +2445049|AAAAAAAAJPOEFCAA|1982-03-20|986|4290|330|1982|6|3|20|1|1982|330|4290|Saturday|1982Q1|N|Y|N|2445030|2445088|2444684|2444957|N|N|N|N|N| +2445050|AAAAAAAAKPOEFCAA|1982-03-21|986|4290|330|1982|0|3|21|1|1982|330|4290|Sunday|1982Q1|N|N|N|2445030|2445088|2444685|2444958|N|N|N|N|N| +2445051|AAAAAAAALPOEFCAA|1982-03-22|986|4290|330|1982|1|3|22|1|1982|330|4290|Monday|1982Q1|N|N|N|2445030|2445088|2444686|2444959|N|N|N|N|N| +2445052|AAAAAAAAMPOEFCAA|1982-03-23|986|4291|330|1982|2|3|23|1|1982|330|4291|Tuesday|1982Q1|N|N|N|2445030|2445088|2444687|2444960|N|N|N|N|N| +2445053|AAAAAAAANPOEFCAA|1982-03-24|986|4291|330|1982|3|3|24|1|1982|330|4291|Wednesday|1982Q1|N|N|N|2445030|2445088|2444688|2444961|N|N|N|N|N| +2445054|AAAAAAAAOPOEFCAA|1982-03-25|986|4291|330|1982|4|3|25|1|1982|330|4291|Thursday|1982Q1|N|N|N|2445030|2445088|2444689|2444962|N|N|N|N|N| +2445055|AAAAAAAAPPOEFCAA|1982-03-26|986|4291|330|1982|5|3|26|1|1982|330|4291|Friday|1982Q1|N|Y|N|2445030|2445088|2444690|2444963|N|N|N|N|N| +2445056|AAAAAAAAAAPEFCAA|1982-03-27|986|4291|330|1982|6|3|27|1|1982|330|4291|Saturday|1982Q1|N|Y|N|2445030|2445088|2444691|2444964|N|N|N|N|N| +2445057|AAAAAAAABAPEFCAA|1982-03-28|986|4291|330|1982|0|3|28|1|1982|330|4291|Sunday|1982Q1|N|N|N|2445030|2445088|2444692|2444965|N|N|N|N|N| +2445058|AAAAAAAACAPEFCAA|1982-03-29|986|4291|330|1982|1|3|29|1|1982|330|4291|Monday|1982Q1|N|N|N|2445030|2445088|2444693|2444966|N|N|N|N|N| +2445059|AAAAAAAADAPEFCAA|1982-03-30|986|4292|330|1982|2|3|30|1|1982|330|4292|Tuesday|1982Q1|N|N|N|2445030|2445088|2444694|2444967|N|N|N|N|N| +2445060|AAAAAAAAEAPEFCAA|1982-03-31|986|4292|330|1982|3|3|31|1|1982|330|4292|Wednesday|1982Q1|N|N|N|2445030|2445088|2444695|2444968|N|N|N|N|N| +2445061|AAAAAAAAFAPEFCAA|1982-04-01|987|4292|330|1982|4|4|1|1|1982|330|4292|Thursday|1982Q1|N|N|N|2445061|2445150|2444696|2444971|N|N|N|N|N| +2445062|AAAAAAAAGAPEFCAA|1982-04-02|987|4292|330|1982|5|4|2|2|1982|330|4292|Friday|1982Q2|N|Y|N|2445061|2445150|2444697|2444972|N|N|N|N|N| +2445063|AAAAAAAAHAPEFCAA|1982-04-03|987|4292|330|1982|6|4|3|2|1982|330|4292|Saturday|1982Q2|N|Y|N|2445061|2445150|2444698|2444973|N|N|N|N|N| +2445064|AAAAAAAAIAPEFCAA|1982-04-04|987|4292|330|1982|0|4|4|2|1982|330|4292|Sunday|1982Q2|N|N|N|2445061|2445150|2444699|2444974|N|N|N|N|N| +2445065|AAAAAAAAJAPEFCAA|1982-04-05|987|4292|330|1982|1|4|5|2|1982|330|4292|Monday|1982Q2|N|N|N|2445061|2445150|2444700|2444975|N|N|N|N|N| +2445066|AAAAAAAAKAPEFCAA|1982-04-06|987|4293|330|1982|2|4|6|2|1982|330|4293|Tuesday|1982Q2|N|N|N|2445061|2445150|2444701|2444976|N|N|N|N|N| +2445067|AAAAAAAALAPEFCAA|1982-04-07|987|4293|330|1982|3|4|7|2|1982|330|4293|Wednesday|1982Q2|N|N|N|2445061|2445150|2444702|2444977|N|N|N|N|N| +2445068|AAAAAAAAMAPEFCAA|1982-04-08|987|4293|330|1982|4|4|8|2|1982|330|4293|Thursday|1982Q2|N|N|N|2445061|2445150|2444703|2444978|N|N|N|N|N| +2445069|AAAAAAAANAPEFCAA|1982-04-09|987|4293|330|1982|5|4|9|2|1982|330|4293|Friday|1982Q2|N|Y|N|2445061|2445150|2444704|2444979|N|N|N|N|N| +2445070|AAAAAAAAOAPEFCAA|1982-04-10|987|4293|330|1982|6|4|10|2|1982|330|4293|Saturday|1982Q2|N|Y|N|2445061|2445150|2444705|2444980|N|N|N|N|N| +2445071|AAAAAAAAPAPEFCAA|1982-04-11|987|4293|330|1982|0|4|11|2|1982|330|4293|Sunday|1982Q2|N|N|N|2445061|2445150|2444706|2444981|N|N|N|N|N| +2445072|AAAAAAAAABPEFCAA|1982-04-12|987|4293|330|1982|1|4|12|2|1982|330|4293|Monday|1982Q2|N|N|N|2445061|2445150|2444707|2444982|N|N|N|N|N| +2445073|AAAAAAAABBPEFCAA|1982-04-13|987|4294|330|1982|2|4|13|2|1982|330|4294|Tuesday|1982Q2|N|N|N|2445061|2445150|2444708|2444983|N|N|N|N|N| +2445074|AAAAAAAACBPEFCAA|1982-04-14|987|4294|330|1982|3|4|14|2|1982|330|4294|Wednesday|1982Q2|N|N|N|2445061|2445150|2444709|2444984|N|N|N|N|N| +2445075|AAAAAAAADBPEFCAA|1982-04-15|987|4294|330|1982|4|4|15|2|1982|330|4294|Thursday|1982Q2|N|N|N|2445061|2445150|2444710|2444985|N|N|N|N|N| +2445076|AAAAAAAAEBPEFCAA|1982-04-16|987|4294|330|1982|5|4|16|2|1982|330|4294|Friday|1982Q2|N|Y|N|2445061|2445150|2444711|2444986|N|N|N|N|N| +2445077|AAAAAAAAFBPEFCAA|1982-04-17|987|4294|330|1982|6|4|17|2|1982|330|4294|Saturday|1982Q2|N|Y|N|2445061|2445150|2444712|2444987|N|N|N|N|N| +2445078|AAAAAAAAGBPEFCAA|1982-04-18|987|4294|330|1982|0|4|18|2|1982|330|4294|Sunday|1982Q2|N|N|N|2445061|2445150|2444713|2444988|N|N|N|N|N| +2445079|AAAAAAAAHBPEFCAA|1982-04-19|987|4294|330|1982|1|4|19|2|1982|330|4294|Monday|1982Q2|N|N|N|2445061|2445150|2444714|2444989|N|N|N|N|N| +2445080|AAAAAAAAIBPEFCAA|1982-04-20|987|4295|330|1982|2|4|20|2|1982|330|4295|Tuesday|1982Q2|N|N|N|2445061|2445150|2444715|2444990|N|N|N|N|N| +2445081|AAAAAAAAJBPEFCAA|1982-04-21|987|4295|330|1982|3|4|21|2|1982|330|4295|Wednesday|1982Q2|N|N|N|2445061|2445150|2444716|2444991|N|N|N|N|N| +2445082|AAAAAAAAKBPEFCAA|1982-04-22|987|4295|330|1982|4|4|22|2|1982|330|4295|Thursday|1982Q2|N|N|N|2445061|2445150|2444717|2444992|N|N|N|N|N| +2445083|AAAAAAAALBPEFCAA|1982-04-23|987|4295|330|1982|5|4|23|2|1982|330|4295|Friday|1982Q2|N|Y|N|2445061|2445150|2444718|2444993|N|N|N|N|N| +2445084|AAAAAAAAMBPEFCAA|1982-04-24|987|4295|330|1982|6|4|24|2|1982|330|4295|Saturday|1982Q2|N|Y|N|2445061|2445150|2444719|2444994|N|N|N|N|N| +2445085|AAAAAAAANBPEFCAA|1982-04-25|987|4295|330|1982|0|4|25|2|1982|330|4295|Sunday|1982Q2|N|N|N|2445061|2445150|2444720|2444995|N|N|N|N|N| +2445086|AAAAAAAAOBPEFCAA|1982-04-26|987|4295|330|1982|1|4|26|2|1982|330|4295|Monday|1982Q2|N|N|N|2445061|2445150|2444721|2444996|N|N|N|N|N| +2445087|AAAAAAAAPBPEFCAA|1982-04-27|987|4296|330|1982|2|4|27|2|1982|330|4296|Tuesday|1982Q2|N|N|N|2445061|2445150|2444722|2444997|N|N|N|N|N| +2445088|AAAAAAAAACPEFCAA|1982-04-28|987|4296|330|1982|3|4|28|2|1982|330|4296|Wednesday|1982Q2|N|N|N|2445061|2445150|2444723|2444998|N|N|N|N|N| +2445089|AAAAAAAABCPEFCAA|1982-04-29|987|4296|330|1982|4|4|29|2|1982|330|4296|Thursday|1982Q2|N|N|N|2445061|2445150|2444724|2444999|N|N|N|N|N| +2445090|AAAAAAAACCPEFCAA|1982-04-30|987|4296|330|1982|5|4|30|2|1982|330|4296|Friday|1982Q2|N|Y|N|2445061|2445150|2444725|2445000|N|N|N|N|N| +2445091|AAAAAAAADCPEFCAA|1982-05-01|988|4296|330|1982|6|5|1|2|1982|330|4296|Saturday|1982Q2|N|Y|N|2445091|2445210|2444726|2445001|N|N|N|N|N| +2445092|AAAAAAAAECPEFCAA|1982-05-02|988|4296|330|1982|0|5|2|2|1982|330|4296|Sunday|1982Q2|N|N|N|2445091|2445210|2444727|2445002|N|N|N|N|N| +2445093|AAAAAAAAFCPEFCAA|1982-05-03|988|4296|330|1982|1|5|3|2|1982|330|4296|Monday|1982Q2|N|N|N|2445091|2445210|2444728|2445003|N|N|N|N|N| +2445094|AAAAAAAAGCPEFCAA|1982-05-04|988|4297|330|1982|2|5|4|2|1982|330|4297|Tuesday|1982Q2|N|N|N|2445091|2445210|2444729|2445004|N|N|N|N|N| +2445095|AAAAAAAAHCPEFCAA|1982-05-05|988|4297|330|1982|3|5|5|2|1982|330|4297|Wednesday|1982Q2|N|N|N|2445091|2445210|2444730|2445005|N|N|N|N|N| +2445096|AAAAAAAAICPEFCAA|1982-05-06|988|4297|330|1982|4|5|6|2|1982|330|4297|Thursday|1982Q2|N|N|N|2445091|2445210|2444731|2445006|N|N|N|N|N| +2445097|AAAAAAAAJCPEFCAA|1982-05-07|988|4297|330|1982|5|5|7|2|1982|330|4297|Friday|1982Q2|N|Y|N|2445091|2445210|2444732|2445007|N|N|N|N|N| +2445098|AAAAAAAAKCPEFCAA|1982-05-08|988|4297|330|1982|6|5|8|2|1982|330|4297|Saturday|1982Q2|N|Y|N|2445091|2445210|2444733|2445008|N|N|N|N|N| +2445099|AAAAAAAALCPEFCAA|1982-05-09|988|4297|330|1982|0|5|9|2|1982|330|4297|Sunday|1982Q2|N|N|N|2445091|2445210|2444734|2445009|N|N|N|N|N| +2445100|AAAAAAAAMCPEFCAA|1982-05-10|988|4297|330|1982|1|5|10|2|1982|330|4297|Monday|1982Q2|N|N|N|2445091|2445210|2444735|2445010|N|N|N|N|N| +2445101|AAAAAAAANCPEFCAA|1982-05-11|988|4298|330|1982|2|5|11|2|1982|330|4298|Tuesday|1982Q2|N|N|N|2445091|2445210|2444736|2445011|N|N|N|N|N| +2445102|AAAAAAAAOCPEFCAA|1982-05-12|988|4298|330|1982|3|5|12|2|1982|330|4298|Wednesday|1982Q2|N|N|N|2445091|2445210|2444737|2445012|N|N|N|N|N| +2445103|AAAAAAAAPCPEFCAA|1982-05-13|988|4298|330|1982|4|5|13|2|1982|330|4298|Thursday|1982Q2|N|N|N|2445091|2445210|2444738|2445013|N|N|N|N|N| +2445104|AAAAAAAAADPEFCAA|1982-05-14|988|4298|330|1982|5|5|14|2|1982|330|4298|Friday|1982Q2|N|Y|N|2445091|2445210|2444739|2445014|N|N|N|N|N| +2445105|AAAAAAAABDPEFCAA|1982-05-15|988|4298|330|1982|6|5|15|2|1982|330|4298|Saturday|1982Q2|N|Y|N|2445091|2445210|2444740|2445015|N|N|N|N|N| +2445106|AAAAAAAACDPEFCAA|1982-05-16|988|4298|330|1982|0|5|16|2|1982|330|4298|Sunday|1982Q2|N|N|N|2445091|2445210|2444741|2445016|N|N|N|N|N| +2445107|AAAAAAAADDPEFCAA|1982-05-17|988|4298|330|1982|1|5|17|2|1982|330|4298|Monday|1982Q2|N|N|N|2445091|2445210|2444742|2445017|N|N|N|N|N| +2445108|AAAAAAAAEDPEFCAA|1982-05-18|988|4299|330|1982|2|5|18|2|1982|330|4299|Tuesday|1982Q2|N|N|N|2445091|2445210|2444743|2445018|N|N|N|N|N| +2445109|AAAAAAAAFDPEFCAA|1982-05-19|988|4299|330|1982|3|5|19|2|1982|330|4299|Wednesday|1982Q2|N|N|N|2445091|2445210|2444744|2445019|N|N|N|N|N| +2445110|AAAAAAAAGDPEFCAA|1982-05-20|988|4299|330|1982|4|5|20|2|1982|330|4299|Thursday|1982Q2|N|N|N|2445091|2445210|2444745|2445020|N|N|N|N|N| +2445111|AAAAAAAAHDPEFCAA|1982-05-21|988|4299|330|1982|5|5|21|2|1982|330|4299|Friday|1982Q2|N|Y|N|2445091|2445210|2444746|2445021|N|N|N|N|N| +2445112|AAAAAAAAIDPEFCAA|1982-05-22|988|4299|330|1982|6|5|22|2|1982|330|4299|Saturday|1982Q2|N|Y|N|2445091|2445210|2444747|2445022|N|N|N|N|N| +2445113|AAAAAAAAJDPEFCAA|1982-05-23|988|4299|330|1982|0|5|23|2|1982|330|4299|Sunday|1982Q2|N|N|N|2445091|2445210|2444748|2445023|N|N|N|N|N| +2445114|AAAAAAAAKDPEFCAA|1982-05-24|988|4299|330|1982|1|5|24|2|1982|330|4299|Monday|1982Q2|N|N|N|2445091|2445210|2444749|2445024|N|N|N|N|N| +2445115|AAAAAAAALDPEFCAA|1982-05-25|988|4300|330|1982|2|5|25|2|1982|330|4300|Tuesday|1982Q2|N|N|N|2445091|2445210|2444750|2445025|N|N|N|N|N| +2445116|AAAAAAAAMDPEFCAA|1982-05-26|988|4300|330|1982|3|5|26|2|1982|330|4300|Wednesday|1982Q2|N|N|N|2445091|2445210|2444751|2445026|N|N|N|N|N| +2445117|AAAAAAAANDPEFCAA|1982-05-27|988|4300|330|1982|4|5|27|2|1982|330|4300|Thursday|1982Q2|N|N|N|2445091|2445210|2444752|2445027|N|N|N|N|N| +2445118|AAAAAAAAODPEFCAA|1982-05-28|988|4300|330|1982|5|5|28|2|1982|330|4300|Friday|1982Q2|N|Y|N|2445091|2445210|2444753|2445028|N|N|N|N|N| +2445119|AAAAAAAAPDPEFCAA|1982-05-29|988|4300|330|1982|6|5|29|2|1982|330|4300|Saturday|1982Q2|N|Y|N|2445091|2445210|2444754|2445029|N|N|N|N|N| +2445120|AAAAAAAAAEPEFCAA|1982-05-30|988|4300|330|1982|0|5|30|2|1982|330|4300|Sunday|1982Q2|N|N|N|2445091|2445210|2444755|2445030|N|N|N|N|N| +2445121|AAAAAAAABEPEFCAA|1982-05-31|988|4300|330|1982|1|5|31|2|1982|330|4300|Monday|1982Q2|N|N|N|2445091|2445210|2444756|2445031|N|N|N|N|N| +2445122|AAAAAAAACEPEFCAA|1982-06-01|989|4301|331|1982|2|6|1|2|1982|331|4301|Tuesday|1982Q2|N|N|N|2445122|2445272|2444757|2445032|N|N|N|N|N| +2445123|AAAAAAAADEPEFCAA|1982-06-02|989|4301|331|1982|3|6|2|2|1982|331|4301|Wednesday|1982Q2|N|N|N|2445122|2445272|2444758|2445033|N|N|N|N|N| +2445124|AAAAAAAAEEPEFCAA|1982-06-03|989|4301|331|1982|4|6|3|2|1982|331|4301|Thursday|1982Q2|N|N|N|2445122|2445272|2444759|2445034|N|N|N|N|N| +2445125|AAAAAAAAFEPEFCAA|1982-06-04|989|4301|331|1982|5|6|4|2|1982|331|4301|Friday|1982Q2|N|Y|N|2445122|2445272|2444760|2445035|N|N|N|N|N| +2445126|AAAAAAAAGEPEFCAA|1982-06-05|989|4301|331|1982|6|6|5|2|1982|331|4301|Saturday|1982Q2|N|Y|N|2445122|2445272|2444761|2445036|N|N|N|N|N| +2445127|AAAAAAAAHEPEFCAA|1982-06-06|989|4301|331|1982|0|6|6|2|1982|331|4301|Sunday|1982Q2|N|N|N|2445122|2445272|2444762|2445037|N|N|N|N|N| +2445128|AAAAAAAAIEPEFCAA|1982-06-07|989|4301|331|1982|1|6|7|2|1982|331|4301|Monday|1982Q2|N|N|N|2445122|2445272|2444763|2445038|N|N|N|N|N| +2445129|AAAAAAAAJEPEFCAA|1982-06-08|989|4302|331|1982|2|6|8|2|1982|331|4302|Tuesday|1982Q2|N|N|N|2445122|2445272|2444764|2445039|N|N|N|N|N| +2445130|AAAAAAAAKEPEFCAA|1982-06-09|989|4302|331|1982|3|6|9|2|1982|331|4302|Wednesday|1982Q2|N|N|N|2445122|2445272|2444765|2445040|N|N|N|N|N| +2445131|AAAAAAAALEPEFCAA|1982-06-10|989|4302|331|1982|4|6|10|2|1982|331|4302|Thursday|1982Q2|N|N|N|2445122|2445272|2444766|2445041|N|N|N|N|N| +2445132|AAAAAAAAMEPEFCAA|1982-06-11|989|4302|331|1982|5|6|11|2|1982|331|4302|Friday|1982Q2|N|Y|N|2445122|2445272|2444767|2445042|N|N|N|N|N| +2445133|AAAAAAAANEPEFCAA|1982-06-12|989|4302|331|1982|6|6|12|2|1982|331|4302|Saturday|1982Q2|N|Y|N|2445122|2445272|2444768|2445043|N|N|N|N|N| +2445134|AAAAAAAAOEPEFCAA|1982-06-13|989|4302|331|1982|0|6|13|2|1982|331|4302|Sunday|1982Q2|N|N|N|2445122|2445272|2444769|2445044|N|N|N|N|N| +2445135|AAAAAAAAPEPEFCAA|1982-06-14|989|4302|331|1982|1|6|14|2|1982|331|4302|Monday|1982Q2|N|N|N|2445122|2445272|2444770|2445045|N|N|N|N|N| +2445136|AAAAAAAAAFPEFCAA|1982-06-15|989|4303|331|1982|2|6|15|2|1982|331|4303|Tuesday|1982Q2|N|N|N|2445122|2445272|2444771|2445046|N|N|N|N|N| +2445137|AAAAAAAABFPEFCAA|1982-06-16|989|4303|331|1982|3|6|16|2|1982|331|4303|Wednesday|1982Q2|N|N|N|2445122|2445272|2444772|2445047|N|N|N|N|N| +2445138|AAAAAAAACFPEFCAA|1982-06-17|989|4303|331|1982|4|6|17|2|1982|331|4303|Thursday|1982Q2|N|N|N|2445122|2445272|2444773|2445048|N|N|N|N|N| +2445139|AAAAAAAADFPEFCAA|1982-06-18|989|4303|331|1982|5|6|18|2|1982|331|4303|Friday|1982Q2|N|Y|N|2445122|2445272|2444774|2445049|N|N|N|N|N| +2445140|AAAAAAAAEFPEFCAA|1982-06-19|989|4303|331|1982|6|6|19|2|1982|331|4303|Saturday|1982Q2|N|Y|N|2445122|2445272|2444775|2445050|N|N|N|N|N| +2445141|AAAAAAAAFFPEFCAA|1982-06-20|989|4303|331|1982|0|6|20|2|1982|331|4303|Sunday|1982Q2|N|N|N|2445122|2445272|2444776|2445051|N|N|N|N|N| +2445142|AAAAAAAAGFPEFCAA|1982-06-21|989|4303|331|1982|1|6|21|2|1982|331|4303|Monday|1982Q2|N|N|N|2445122|2445272|2444777|2445052|N|N|N|N|N| +2445143|AAAAAAAAHFPEFCAA|1982-06-22|989|4304|331|1982|2|6|22|2|1982|331|4304|Tuesday|1982Q2|N|N|N|2445122|2445272|2444778|2445053|N|N|N|N|N| +2445144|AAAAAAAAIFPEFCAA|1982-06-23|989|4304|331|1982|3|6|23|2|1982|331|4304|Wednesday|1982Q2|N|N|N|2445122|2445272|2444779|2445054|N|N|N|N|N| +2445145|AAAAAAAAJFPEFCAA|1982-06-24|989|4304|331|1982|4|6|24|2|1982|331|4304|Thursday|1982Q2|N|N|N|2445122|2445272|2444780|2445055|N|N|N|N|N| +2445146|AAAAAAAAKFPEFCAA|1982-06-25|989|4304|331|1982|5|6|25|2|1982|331|4304|Friday|1982Q2|N|Y|N|2445122|2445272|2444781|2445056|N|N|N|N|N| +2445147|AAAAAAAALFPEFCAA|1982-06-26|989|4304|331|1982|6|6|26|2|1982|331|4304|Saturday|1982Q2|N|Y|N|2445122|2445272|2444782|2445057|N|N|N|N|N| +2445148|AAAAAAAAMFPEFCAA|1982-06-27|989|4304|331|1982|0|6|27|2|1982|331|4304|Sunday|1982Q2|N|N|N|2445122|2445272|2444783|2445058|N|N|N|N|N| +2445149|AAAAAAAANFPEFCAA|1982-06-28|989|4304|331|1982|1|6|28|2|1982|331|4304|Monday|1982Q2|N|N|N|2445122|2445272|2444784|2445059|N|N|N|N|N| +2445150|AAAAAAAAOFPEFCAA|1982-06-29|989|4305|331|1982|2|6|29|2|1982|331|4305|Tuesday|1982Q2|N|N|N|2445122|2445272|2444785|2445060|N|N|N|N|N| +2445151|AAAAAAAAPFPEFCAA|1982-06-30|989|4305|331|1982|3|6|30|2|1982|331|4305|Wednesday|1982Q2|N|N|N|2445122|2445272|2444786|2445061|N|N|N|N|N| +2445152|AAAAAAAAAGPEFCAA|1982-07-01|990|4305|331|1982|4|7|1|2|1982|331|4305|Thursday|1982Q2|N|N|N|2445152|2445332|2444787|2445061|N|N|N|N|N| +2445153|AAAAAAAABGPEFCAA|1982-07-02|990|4305|331|1982|5|7|2|3|1982|331|4305|Friday|1982Q3|N|Y|N|2445152|2445332|2444788|2445062|N|N|N|N|N| +2445154|AAAAAAAACGPEFCAA|1982-07-03|990|4305|331|1982|6|7|3|3|1982|331|4305|Saturday|1982Q3|N|Y|N|2445152|2445332|2444789|2445063|N|N|N|N|N| +2445155|AAAAAAAADGPEFCAA|1982-07-04|990|4305|331|1982|0|7|4|3|1982|331|4305|Sunday|1982Q3|N|N|N|2445152|2445332|2444790|2445064|N|N|N|N|N| +2445156|AAAAAAAAEGPEFCAA|1982-07-05|990|4305|331|1982|1|7|5|3|1982|331|4305|Monday|1982Q3|Y|N|N|2445152|2445332|2444791|2445065|N|N|N|N|N| +2445157|AAAAAAAAFGPEFCAA|1982-07-06|990|4306|331|1982|2|7|6|3|1982|331|4306|Tuesday|1982Q3|N|N|Y|2445152|2445332|2444792|2445066|N|N|N|N|N| +2445158|AAAAAAAAGGPEFCAA|1982-07-07|990|4306|331|1982|3|7|7|3|1982|331|4306|Wednesday|1982Q3|N|N|N|2445152|2445332|2444793|2445067|N|N|N|N|N| +2445159|AAAAAAAAHGPEFCAA|1982-07-08|990|4306|331|1982|4|7|8|3|1982|331|4306|Thursday|1982Q3|N|N|N|2445152|2445332|2444794|2445068|N|N|N|N|N| +2445160|AAAAAAAAIGPEFCAA|1982-07-09|990|4306|331|1982|5|7|9|3|1982|331|4306|Friday|1982Q3|N|Y|N|2445152|2445332|2444795|2445069|N|N|N|N|N| +2445161|AAAAAAAAJGPEFCAA|1982-07-10|990|4306|331|1982|6|7|10|3|1982|331|4306|Saturday|1982Q3|N|Y|N|2445152|2445332|2444796|2445070|N|N|N|N|N| +2445162|AAAAAAAAKGPEFCAA|1982-07-11|990|4306|331|1982|0|7|11|3|1982|331|4306|Sunday|1982Q3|N|N|N|2445152|2445332|2444797|2445071|N|N|N|N|N| +2445163|AAAAAAAALGPEFCAA|1982-07-12|990|4306|331|1982|1|7|12|3|1982|331|4306|Monday|1982Q3|N|N|N|2445152|2445332|2444798|2445072|N|N|N|N|N| +2445164|AAAAAAAAMGPEFCAA|1982-07-13|990|4307|331|1982|2|7|13|3|1982|331|4307|Tuesday|1982Q3|N|N|N|2445152|2445332|2444799|2445073|N|N|N|N|N| +2445165|AAAAAAAANGPEFCAA|1982-07-14|990|4307|331|1982|3|7|14|3|1982|331|4307|Wednesday|1982Q3|N|N|N|2445152|2445332|2444800|2445074|N|N|N|N|N| +2445166|AAAAAAAAOGPEFCAA|1982-07-15|990|4307|331|1982|4|7|15|3|1982|331|4307|Thursday|1982Q3|N|N|N|2445152|2445332|2444801|2445075|N|N|N|N|N| +2445167|AAAAAAAAPGPEFCAA|1982-07-16|990|4307|331|1982|5|7|16|3|1982|331|4307|Friday|1982Q3|N|Y|N|2445152|2445332|2444802|2445076|N|N|N|N|N| +2445168|AAAAAAAAAHPEFCAA|1982-07-17|990|4307|331|1982|6|7|17|3|1982|331|4307|Saturday|1982Q3|N|Y|N|2445152|2445332|2444803|2445077|N|N|N|N|N| +2445169|AAAAAAAABHPEFCAA|1982-07-18|990|4307|331|1982|0|7|18|3|1982|331|4307|Sunday|1982Q3|N|N|N|2445152|2445332|2444804|2445078|N|N|N|N|N| +2445170|AAAAAAAACHPEFCAA|1982-07-19|990|4307|331|1982|1|7|19|3|1982|331|4307|Monday|1982Q3|N|N|N|2445152|2445332|2444805|2445079|N|N|N|N|N| +2445171|AAAAAAAADHPEFCAA|1982-07-20|990|4308|331|1982|2|7|20|3|1982|331|4308|Tuesday|1982Q3|N|N|N|2445152|2445332|2444806|2445080|N|N|N|N|N| +2445172|AAAAAAAAEHPEFCAA|1982-07-21|990|4308|331|1982|3|7|21|3|1982|331|4308|Wednesday|1982Q3|N|N|N|2445152|2445332|2444807|2445081|N|N|N|N|N| +2445173|AAAAAAAAFHPEFCAA|1982-07-22|990|4308|331|1982|4|7|22|3|1982|331|4308|Thursday|1982Q3|N|N|N|2445152|2445332|2444808|2445082|N|N|N|N|N| +2445174|AAAAAAAAGHPEFCAA|1982-07-23|990|4308|331|1982|5|7|23|3|1982|331|4308|Friday|1982Q3|N|Y|N|2445152|2445332|2444809|2445083|N|N|N|N|N| +2445175|AAAAAAAAHHPEFCAA|1982-07-24|990|4308|331|1982|6|7|24|3|1982|331|4308|Saturday|1982Q3|N|Y|N|2445152|2445332|2444810|2445084|N|N|N|N|N| +2445176|AAAAAAAAIHPEFCAA|1982-07-25|990|4308|331|1982|0|7|25|3|1982|331|4308|Sunday|1982Q3|N|N|N|2445152|2445332|2444811|2445085|N|N|N|N|N| +2445177|AAAAAAAAJHPEFCAA|1982-07-26|990|4308|331|1982|1|7|26|3|1982|331|4308|Monday|1982Q3|N|N|N|2445152|2445332|2444812|2445086|N|N|N|N|N| +2445178|AAAAAAAAKHPEFCAA|1982-07-27|990|4309|331|1982|2|7|27|3|1982|331|4309|Tuesday|1982Q3|N|N|N|2445152|2445332|2444813|2445087|N|N|N|N|N| +2445179|AAAAAAAALHPEFCAA|1982-07-28|990|4309|331|1982|3|7|28|3|1982|331|4309|Wednesday|1982Q3|N|N|N|2445152|2445332|2444814|2445088|N|N|N|N|N| +2445180|AAAAAAAAMHPEFCAA|1982-07-29|990|4309|331|1982|4|7|29|3|1982|331|4309|Thursday|1982Q3|N|N|N|2445152|2445332|2444815|2445089|N|N|N|N|N| +2445181|AAAAAAAANHPEFCAA|1982-07-30|990|4309|331|1982|5|7|30|3|1982|331|4309|Friday|1982Q3|N|Y|N|2445152|2445332|2444816|2445090|N|N|N|N|N| +2445182|AAAAAAAAOHPEFCAA|1982-07-31|990|4309|331|1982|6|7|31|3|1982|331|4309|Saturday|1982Q3|N|Y|N|2445152|2445332|2444817|2445091|N|N|N|N|N| +2445183|AAAAAAAAPHPEFCAA|1982-08-01|991|4309|331|1982|0|8|1|3|1982|331|4309|Sunday|1982Q3|N|N|N|2445183|2445394|2444818|2445092|N|N|N|N|N| +2445184|AAAAAAAAAIPEFCAA|1982-08-02|991|4309|331|1982|1|8|2|3|1982|331|4309|Monday|1982Q3|N|N|N|2445183|2445394|2444819|2445093|N|N|N|N|N| +2445185|AAAAAAAABIPEFCAA|1982-08-03|991|4310|331|1982|2|8|3|3|1982|331|4310|Tuesday|1982Q3|N|N|N|2445183|2445394|2444820|2445094|N|N|N|N|N| +2445186|AAAAAAAACIPEFCAA|1982-08-04|991|4310|331|1982|3|8|4|3|1982|331|4310|Wednesday|1982Q3|N|N|N|2445183|2445394|2444821|2445095|N|N|N|N|N| +2445187|AAAAAAAADIPEFCAA|1982-08-05|991|4310|331|1982|4|8|5|3|1982|331|4310|Thursday|1982Q3|N|N|N|2445183|2445394|2444822|2445096|N|N|N|N|N| +2445188|AAAAAAAAEIPEFCAA|1982-08-06|991|4310|331|1982|5|8|6|3|1982|331|4310|Friday|1982Q3|N|Y|N|2445183|2445394|2444823|2445097|N|N|N|N|N| +2445189|AAAAAAAAFIPEFCAA|1982-08-07|991|4310|331|1982|6|8|7|3|1982|331|4310|Saturday|1982Q3|N|Y|N|2445183|2445394|2444824|2445098|N|N|N|N|N| +2445190|AAAAAAAAGIPEFCAA|1982-08-08|991|4310|331|1982|0|8|8|3|1982|331|4310|Sunday|1982Q3|N|N|N|2445183|2445394|2444825|2445099|N|N|N|N|N| +2445191|AAAAAAAAHIPEFCAA|1982-08-09|991|4310|331|1982|1|8|9|3|1982|331|4310|Monday|1982Q3|N|N|N|2445183|2445394|2444826|2445100|N|N|N|N|N| +2445192|AAAAAAAAIIPEFCAA|1982-08-10|991|4311|331|1982|2|8|10|3|1982|331|4311|Tuesday|1982Q3|N|N|N|2445183|2445394|2444827|2445101|N|N|N|N|N| +2445193|AAAAAAAAJIPEFCAA|1982-08-11|991|4311|331|1982|3|8|11|3|1982|331|4311|Wednesday|1982Q3|N|N|N|2445183|2445394|2444828|2445102|N|N|N|N|N| +2445194|AAAAAAAAKIPEFCAA|1982-08-12|991|4311|331|1982|4|8|12|3|1982|331|4311|Thursday|1982Q3|N|N|N|2445183|2445394|2444829|2445103|N|N|N|N|N| +2445195|AAAAAAAALIPEFCAA|1982-08-13|991|4311|331|1982|5|8|13|3|1982|331|4311|Friday|1982Q3|N|Y|N|2445183|2445394|2444830|2445104|N|N|N|N|N| +2445196|AAAAAAAAMIPEFCAA|1982-08-14|991|4311|331|1982|6|8|14|3|1982|331|4311|Saturday|1982Q3|N|Y|N|2445183|2445394|2444831|2445105|N|N|N|N|N| +2445197|AAAAAAAANIPEFCAA|1982-08-15|991|4311|331|1982|0|8|15|3|1982|331|4311|Sunday|1982Q3|N|N|N|2445183|2445394|2444832|2445106|N|N|N|N|N| +2445198|AAAAAAAAOIPEFCAA|1982-08-16|991|4311|331|1982|1|8|16|3|1982|331|4311|Monday|1982Q3|N|N|N|2445183|2445394|2444833|2445107|N|N|N|N|N| +2445199|AAAAAAAAPIPEFCAA|1982-08-17|991|4312|331|1982|2|8|17|3|1982|331|4312|Tuesday|1982Q3|N|N|N|2445183|2445394|2444834|2445108|N|N|N|N|N| +2445200|AAAAAAAAAJPEFCAA|1982-08-18|991|4312|331|1982|3|8|18|3|1982|331|4312|Wednesday|1982Q3|N|N|N|2445183|2445394|2444835|2445109|N|N|N|N|N| +2445201|AAAAAAAABJPEFCAA|1982-08-19|991|4312|331|1982|4|8|19|3|1982|331|4312|Thursday|1982Q3|N|N|N|2445183|2445394|2444836|2445110|N|N|N|N|N| +2445202|AAAAAAAACJPEFCAA|1982-08-20|991|4312|331|1982|5|8|20|3|1982|331|4312|Friday|1982Q3|N|Y|N|2445183|2445394|2444837|2445111|N|N|N|N|N| +2445203|AAAAAAAADJPEFCAA|1982-08-21|991|4312|331|1982|6|8|21|3|1982|331|4312|Saturday|1982Q3|N|Y|N|2445183|2445394|2444838|2445112|N|N|N|N|N| +2445204|AAAAAAAAEJPEFCAA|1982-08-22|991|4312|331|1982|0|8|22|3|1982|331|4312|Sunday|1982Q3|N|N|N|2445183|2445394|2444839|2445113|N|N|N|N|N| +2445205|AAAAAAAAFJPEFCAA|1982-08-23|991|4312|331|1982|1|8|23|3|1982|331|4312|Monday|1982Q3|N|N|N|2445183|2445394|2444840|2445114|N|N|N|N|N| +2445206|AAAAAAAAGJPEFCAA|1982-08-24|991|4313|331|1982|2|8|24|3|1982|331|4313|Tuesday|1982Q3|N|N|N|2445183|2445394|2444841|2445115|N|N|N|N|N| +2445207|AAAAAAAAHJPEFCAA|1982-08-25|991|4313|331|1982|3|8|25|3|1982|331|4313|Wednesday|1982Q3|N|N|N|2445183|2445394|2444842|2445116|N|N|N|N|N| +2445208|AAAAAAAAIJPEFCAA|1982-08-26|991|4313|331|1982|4|8|26|3|1982|331|4313|Thursday|1982Q3|N|N|N|2445183|2445394|2444843|2445117|N|N|N|N|N| +2445209|AAAAAAAAJJPEFCAA|1982-08-27|991|4313|331|1982|5|8|27|3|1982|331|4313|Friday|1982Q3|N|Y|N|2445183|2445394|2444844|2445118|N|N|N|N|N| +2445210|AAAAAAAAKJPEFCAA|1982-08-28|991|4313|331|1982|6|8|28|3|1982|331|4313|Saturday|1982Q3|N|Y|N|2445183|2445394|2444845|2445119|N|N|N|N|N| +2445211|AAAAAAAALJPEFCAA|1982-08-29|991|4313|331|1982|0|8|29|3|1982|331|4313|Sunday|1982Q3|N|N|N|2445183|2445394|2444846|2445120|N|N|N|N|N| +2445212|AAAAAAAAMJPEFCAA|1982-08-30|991|4313|331|1982|1|8|30|3|1982|331|4313|Monday|1982Q3|N|N|N|2445183|2445394|2444847|2445121|N|N|N|N|N| +2445213|AAAAAAAANJPEFCAA|1982-08-31|991|4314|331|1982|2|8|31|3|1982|331|4314|Tuesday|1982Q3|N|N|N|2445183|2445394|2444848|2445122|N|N|N|N|N| +2445214|AAAAAAAAOJPEFCAA|1982-09-01|992|4314|332|1982|3|9|1|3|1982|332|4314|Wednesday|1982Q3|N|N|N|2445214|2445456|2444849|2445123|N|N|N|N|N| +2445215|AAAAAAAAPJPEFCAA|1982-09-02|992|4314|332|1982|4|9|2|3|1982|332|4314|Thursday|1982Q3|N|N|N|2445214|2445456|2444850|2445124|N|N|N|N|N| +2445216|AAAAAAAAAKPEFCAA|1982-09-03|992|4314|332|1982|5|9|3|3|1982|332|4314|Friday|1982Q3|N|Y|N|2445214|2445456|2444851|2445125|N|N|N|N|N| +2445217|AAAAAAAABKPEFCAA|1982-09-04|992|4314|332|1982|6|9|4|3|1982|332|4314|Saturday|1982Q3|N|Y|N|2445214|2445456|2444852|2445126|N|N|N|N|N| +2445218|AAAAAAAACKPEFCAA|1982-09-05|992|4314|332|1982|0|9|5|3|1982|332|4314|Sunday|1982Q3|N|N|N|2445214|2445456|2444853|2445127|N|N|N|N|N| +2445219|AAAAAAAADKPEFCAA|1982-09-06|992|4314|332|1982|1|9|6|3|1982|332|4314|Monday|1982Q3|N|N|N|2445214|2445456|2444854|2445128|N|N|N|N|N| +2445220|AAAAAAAAEKPEFCAA|1982-09-07|992|4315|332|1982|2|9|7|3|1982|332|4315|Tuesday|1982Q3|N|N|N|2445214|2445456|2444855|2445129|N|N|N|N|N| +2445221|AAAAAAAAFKPEFCAA|1982-09-08|992|4315|332|1982|3|9|8|3|1982|332|4315|Wednesday|1982Q3|N|N|N|2445214|2445456|2444856|2445130|N|N|N|N|N| +2445222|AAAAAAAAGKPEFCAA|1982-09-09|992|4315|332|1982|4|9|9|3|1982|332|4315|Thursday|1982Q3|N|N|N|2445214|2445456|2444857|2445131|N|N|N|N|N| +2445223|AAAAAAAAHKPEFCAA|1982-09-10|992|4315|332|1982|5|9|10|3|1982|332|4315|Friday|1982Q3|N|Y|N|2445214|2445456|2444858|2445132|N|N|N|N|N| +2445224|AAAAAAAAIKPEFCAA|1982-09-11|992|4315|332|1982|6|9|11|3|1982|332|4315|Saturday|1982Q3|N|Y|N|2445214|2445456|2444859|2445133|N|N|N|N|N| +2445225|AAAAAAAAJKPEFCAA|1982-09-12|992|4315|332|1982|0|9|12|3|1982|332|4315|Sunday|1982Q3|N|N|N|2445214|2445456|2444860|2445134|N|N|N|N|N| +2445226|AAAAAAAAKKPEFCAA|1982-09-13|992|4315|332|1982|1|9|13|3|1982|332|4315|Monday|1982Q3|N|N|N|2445214|2445456|2444861|2445135|N|N|N|N|N| +2445227|AAAAAAAALKPEFCAA|1982-09-14|992|4316|332|1982|2|9|14|3|1982|332|4316|Tuesday|1982Q3|N|N|N|2445214|2445456|2444862|2445136|N|N|N|N|N| +2445228|AAAAAAAAMKPEFCAA|1982-09-15|992|4316|332|1982|3|9|15|3|1982|332|4316|Wednesday|1982Q3|N|N|N|2445214|2445456|2444863|2445137|N|N|N|N|N| +2445229|AAAAAAAANKPEFCAA|1982-09-16|992|4316|332|1982|4|9|16|3|1982|332|4316|Thursday|1982Q3|N|N|N|2445214|2445456|2444864|2445138|N|N|N|N|N| +2445230|AAAAAAAAOKPEFCAA|1982-09-17|992|4316|332|1982|5|9|17|3|1982|332|4316|Friday|1982Q3|N|Y|N|2445214|2445456|2444865|2445139|N|N|N|N|N| +2445231|AAAAAAAAPKPEFCAA|1982-09-18|992|4316|332|1982|6|9|18|3|1982|332|4316|Saturday|1982Q3|N|Y|N|2445214|2445456|2444866|2445140|N|N|N|N|N| +2445232|AAAAAAAAALPEFCAA|1982-09-19|992|4316|332|1982|0|9|19|3|1982|332|4316|Sunday|1982Q3|N|N|N|2445214|2445456|2444867|2445141|N|N|N|N|N| +2445233|AAAAAAAABLPEFCAA|1982-09-20|992|4316|332|1982|1|9|20|3|1982|332|4316|Monday|1982Q3|N|N|N|2445214|2445456|2444868|2445142|N|N|N|N|N| +2445234|AAAAAAAACLPEFCAA|1982-09-21|992|4317|332|1982|2|9|21|3|1982|332|4317|Tuesday|1982Q3|N|N|N|2445214|2445456|2444869|2445143|N|N|N|N|N| +2445235|AAAAAAAADLPEFCAA|1982-09-22|992|4317|332|1982|3|9|22|3|1982|332|4317|Wednesday|1982Q3|N|N|N|2445214|2445456|2444870|2445144|N|N|N|N|N| +2445236|AAAAAAAAELPEFCAA|1982-09-23|992|4317|332|1982|4|9|23|3|1982|332|4317|Thursday|1982Q3|N|N|N|2445214|2445456|2444871|2445145|N|N|N|N|N| +2445237|AAAAAAAAFLPEFCAA|1982-09-24|992|4317|332|1982|5|9|24|3|1982|332|4317|Friday|1982Q3|N|Y|N|2445214|2445456|2444872|2445146|N|N|N|N|N| +2445238|AAAAAAAAGLPEFCAA|1982-09-25|992|4317|332|1982|6|9|25|3|1982|332|4317|Saturday|1982Q3|N|Y|N|2445214|2445456|2444873|2445147|N|N|N|N|N| +2445239|AAAAAAAAHLPEFCAA|1982-09-26|992|4317|332|1982|0|9|26|3|1982|332|4317|Sunday|1982Q3|N|N|N|2445214|2445456|2444874|2445148|N|N|N|N|N| +2445240|AAAAAAAAILPEFCAA|1982-09-27|992|4317|332|1982|1|9|27|3|1982|332|4317|Monday|1982Q3|N|N|N|2445214|2445456|2444875|2445149|N|N|N|N|N| +2445241|AAAAAAAAJLPEFCAA|1982-09-28|992|4318|332|1982|2|9|28|3|1982|332|4318|Tuesday|1982Q3|N|N|N|2445214|2445456|2444876|2445150|N|N|N|N|N| +2445242|AAAAAAAAKLPEFCAA|1982-09-29|992|4318|332|1982|3|9|29|3|1982|332|4318|Wednesday|1982Q3|N|N|N|2445214|2445456|2444877|2445151|N|N|N|N|N| +2445243|AAAAAAAALLPEFCAA|1982-09-30|992|4318|332|1982|4|9|30|3|1982|332|4318|Thursday|1982Q3|N|N|N|2445214|2445456|2444878|2445152|N|N|N|N|N| +2445244|AAAAAAAAMLPEFCAA|1982-10-01|993|4318|332|1982|5|10|1|3|1982|332|4318|Friday|1982Q3|N|Y|N|2445244|2445516|2444879|2445152|N|N|N|N|N| +2445245|AAAAAAAANLPEFCAA|1982-10-02|993|4318|332|1982|6|10|2|4|1982|332|4318|Saturday|1982Q4|N|Y|N|2445244|2445516|2444880|2445153|N|N|N|N|N| +2445246|AAAAAAAAOLPEFCAA|1982-10-03|993|4318|332|1982|0|10|3|4|1982|332|4318|Sunday|1982Q4|N|N|N|2445244|2445516|2444881|2445154|N|N|N|N|N| +2445247|AAAAAAAAPLPEFCAA|1982-10-04|993|4318|332|1982|1|10|4|4|1982|332|4318|Monday|1982Q4|N|N|N|2445244|2445516|2444882|2445155|N|N|N|N|N| +2445248|AAAAAAAAAMPEFCAA|1982-10-05|993|4319|332|1982|2|10|5|4|1982|332|4319|Tuesday|1982Q4|N|N|N|2445244|2445516|2444883|2445156|N|N|N|N|N| +2445249|AAAAAAAABMPEFCAA|1982-10-06|993|4319|332|1982|3|10|6|4|1982|332|4319|Wednesday|1982Q4|N|N|N|2445244|2445516|2444884|2445157|N|N|N|N|N| +2445250|AAAAAAAACMPEFCAA|1982-10-07|993|4319|332|1982|4|10|7|4|1982|332|4319|Thursday|1982Q4|N|N|N|2445244|2445516|2444885|2445158|N|N|N|N|N| +2445251|AAAAAAAADMPEFCAA|1982-10-08|993|4319|332|1982|5|10|8|4|1982|332|4319|Friday|1982Q4|N|Y|N|2445244|2445516|2444886|2445159|N|N|N|N|N| +2445252|AAAAAAAAEMPEFCAA|1982-10-09|993|4319|332|1982|6|10|9|4|1982|332|4319|Saturday|1982Q4|N|Y|N|2445244|2445516|2444887|2445160|N|N|N|N|N| +2445253|AAAAAAAAFMPEFCAA|1982-10-10|993|4319|332|1982|0|10|10|4|1982|332|4319|Sunday|1982Q4|N|N|N|2445244|2445516|2444888|2445161|N|N|N|N|N| +2445254|AAAAAAAAGMPEFCAA|1982-10-11|993|4319|332|1982|1|10|11|4|1982|332|4319|Monday|1982Q4|N|N|N|2445244|2445516|2444889|2445162|N|N|N|N|N| +2445255|AAAAAAAAHMPEFCAA|1982-10-12|993|4320|332|1982|2|10|12|4|1982|332|4320|Tuesday|1982Q4|N|N|N|2445244|2445516|2444890|2445163|N|N|N|N|N| +2445256|AAAAAAAAIMPEFCAA|1982-10-13|993|4320|332|1982|3|10|13|4|1982|332|4320|Wednesday|1982Q4|N|N|N|2445244|2445516|2444891|2445164|N|N|N|N|N| +2445257|AAAAAAAAJMPEFCAA|1982-10-14|993|4320|332|1982|4|10|14|4|1982|332|4320|Thursday|1982Q4|N|N|N|2445244|2445516|2444892|2445165|N|N|N|N|N| +2445258|AAAAAAAAKMPEFCAA|1982-10-15|993|4320|332|1982|5|10|15|4|1982|332|4320|Friday|1982Q4|N|Y|N|2445244|2445516|2444893|2445166|N|N|N|N|N| +2445259|AAAAAAAALMPEFCAA|1982-10-16|993|4320|332|1982|6|10|16|4|1982|332|4320|Saturday|1982Q4|N|Y|N|2445244|2445516|2444894|2445167|N|N|N|N|N| +2445260|AAAAAAAAMMPEFCAA|1982-10-17|993|4320|332|1982|0|10|17|4|1982|332|4320|Sunday|1982Q4|N|N|N|2445244|2445516|2444895|2445168|N|N|N|N|N| +2445261|AAAAAAAANMPEFCAA|1982-10-18|993|4320|332|1982|1|10|18|4|1982|332|4320|Monday|1982Q4|N|N|N|2445244|2445516|2444896|2445169|N|N|N|N|N| +2445262|AAAAAAAAOMPEFCAA|1982-10-19|993|4321|332|1982|2|10|19|4|1982|332|4321|Tuesday|1982Q4|N|N|N|2445244|2445516|2444897|2445170|N|N|N|N|N| +2445263|AAAAAAAAPMPEFCAA|1982-10-20|993|4321|332|1982|3|10|20|4|1982|332|4321|Wednesday|1982Q4|N|N|N|2445244|2445516|2444898|2445171|N|N|N|N|N| +2445264|AAAAAAAAANPEFCAA|1982-10-21|993|4321|332|1982|4|10|21|4|1982|332|4321|Thursday|1982Q4|N|N|N|2445244|2445516|2444899|2445172|N|N|N|N|N| +2445265|AAAAAAAABNPEFCAA|1982-10-22|993|4321|332|1982|5|10|22|4|1982|332|4321|Friday|1982Q4|N|Y|N|2445244|2445516|2444900|2445173|N|N|N|N|N| +2445266|AAAAAAAACNPEFCAA|1982-10-23|993|4321|332|1982|6|10|23|4|1982|332|4321|Saturday|1982Q4|N|Y|N|2445244|2445516|2444901|2445174|N|N|N|N|N| +2445267|AAAAAAAADNPEFCAA|1982-10-24|993|4321|332|1982|0|10|24|4|1982|332|4321|Sunday|1982Q4|N|N|N|2445244|2445516|2444902|2445175|N|N|N|N|N| +2445268|AAAAAAAAENPEFCAA|1982-10-25|993|4321|332|1982|1|10|25|4|1982|332|4321|Monday|1982Q4|N|N|N|2445244|2445516|2444903|2445176|N|N|N|N|N| +2445269|AAAAAAAAFNPEFCAA|1982-10-26|993|4322|332|1982|2|10|26|4|1982|332|4322|Tuesday|1982Q4|N|N|N|2445244|2445516|2444904|2445177|N|N|N|N|N| +2445270|AAAAAAAAGNPEFCAA|1982-10-27|993|4322|332|1982|3|10|27|4|1982|332|4322|Wednesday|1982Q4|N|N|N|2445244|2445516|2444905|2445178|N|N|N|N|N| +2445271|AAAAAAAAHNPEFCAA|1982-10-28|993|4322|332|1982|4|10|28|4|1982|332|4322|Thursday|1982Q4|N|N|N|2445244|2445516|2444906|2445179|N|N|N|N|N| +2445272|AAAAAAAAINPEFCAA|1982-10-29|993|4322|332|1982|5|10|29|4|1982|332|4322|Friday|1982Q4|N|Y|N|2445244|2445516|2444907|2445180|N|N|N|N|N| +2445273|AAAAAAAAJNPEFCAA|1982-10-30|993|4322|332|1982|6|10|30|4|1982|332|4322|Saturday|1982Q4|N|Y|N|2445244|2445516|2444908|2445181|N|N|N|N|N| +2445274|AAAAAAAAKNPEFCAA|1982-10-31|993|4322|332|1982|0|10|31|4|1982|332|4322|Sunday|1982Q4|N|N|N|2445244|2445516|2444909|2445182|N|N|N|N|N| +2445275|AAAAAAAALNPEFCAA|1982-11-01|994|4322|332|1982|1|11|1|4|1982|332|4322|Monday|1982Q4|N|N|N|2445275|2445578|2444910|2445183|N|N|N|N|N| +2445276|AAAAAAAAMNPEFCAA|1982-11-02|994|4323|332|1982|2|11|2|4|1982|332|4323|Tuesday|1982Q4|N|N|N|2445275|2445578|2444911|2445184|N|N|N|N|N| +2445277|AAAAAAAANNPEFCAA|1982-11-03|994|4323|332|1982|3|11|3|4|1982|332|4323|Wednesday|1982Q4|N|N|N|2445275|2445578|2444912|2445185|N|N|N|N|N| +2445278|AAAAAAAAONPEFCAA|1982-11-04|994|4323|332|1982|4|11|4|4|1982|332|4323|Thursday|1982Q4|N|N|N|2445275|2445578|2444913|2445186|N|N|N|N|N| +2445279|AAAAAAAAPNPEFCAA|1982-11-05|994|4323|332|1982|5|11|5|4|1982|332|4323|Friday|1982Q4|N|Y|N|2445275|2445578|2444914|2445187|N|N|N|N|N| +2445280|AAAAAAAAAOPEFCAA|1982-11-06|994|4323|332|1982|6|11|6|4|1982|332|4323|Saturday|1982Q4|N|Y|N|2445275|2445578|2444915|2445188|N|N|N|N|N| +2445281|AAAAAAAABOPEFCAA|1982-11-07|994|4323|332|1982|0|11|7|4|1982|332|4323|Sunday|1982Q4|N|N|N|2445275|2445578|2444916|2445189|N|N|N|N|N| +2445282|AAAAAAAACOPEFCAA|1982-11-08|994|4323|332|1982|1|11|8|4|1982|332|4323|Monday|1982Q4|N|N|N|2445275|2445578|2444917|2445190|N|N|N|N|N| +2445283|AAAAAAAADOPEFCAA|1982-11-09|994|4324|332|1982|2|11|9|4|1982|332|4324|Tuesday|1982Q4|N|N|N|2445275|2445578|2444918|2445191|N|N|N|N|N| +2445284|AAAAAAAAEOPEFCAA|1982-11-10|994|4324|332|1982|3|11|10|4|1982|332|4324|Wednesday|1982Q4|N|N|N|2445275|2445578|2444919|2445192|N|N|N|N|N| +2445285|AAAAAAAAFOPEFCAA|1982-11-11|994|4324|332|1982|4|11|11|4|1982|332|4324|Thursday|1982Q4|N|N|N|2445275|2445578|2444920|2445193|N|N|N|N|N| +2445286|AAAAAAAAGOPEFCAA|1982-11-12|994|4324|332|1982|5|11|12|4|1982|332|4324|Friday|1982Q4|N|Y|N|2445275|2445578|2444921|2445194|N|N|N|N|N| +2445287|AAAAAAAAHOPEFCAA|1982-11-13|994|4324|332|1982|6|11|13|4|1982|332|4324|Saturday|1982Q4|N|Y|N|2445275|2445578|2444922|2445195|N|N|N|N|N| +2445288|AAAAAAAAIOPEFCAA|1982-11-14|994|4324|332|1982|0|11|14|4|1982|332|4324|Sunday|1982Q4|N|N|N|2445275|2445578|2444923|2445196|N|N|N|N|N| +2445289|AAAAAAAAJOPEFCAA|1982-11-15|994|4324|332|1982|1|11|15|4|1982|332|4324|Monday|1982Q4|N|N|N|2445275|2445578|2444924|2445197|N|N|N|N|N| +2445290|AAAAAAAAKOPEFCAA|1982-11-16|994|4325|332|1982|2|11|16|4|1982|332|4325|Tuesday|1982Q4|N|N|N|2445275|2445578|2444925|2445198|N|N|N|N|N| +2445291|AAAAAAAALOPEFCAA|1982-11-17|994|4325|332|1982|3|11|17|4|1982|332|4325|Wednesday|1982Q4|N|N|N|2445275|2445578|2444926|2445199|N|N|N|N|N| +2445292|AAAAAAAAMOPEFCAA|1982-11-18|994|4325|332|1982|4|11|18|4|1982|332|4325|Thursday|1982Q4|N|N|N|2445275|2445578|2444927|2445200|N|N|N|N|N| +2445293|AAAAAAAANOPEFCAA|1982-11-19|994|4325|332|1982|5|11|19|4|1982|332|4325|Friday|1982Q4|N|Y|N|2445275|2445578|2444928|2445201|N|N|N|N|N| +2445294|AAAAAAAAOOPEFCAA|1982-11-20|994|4325|332|1982|6|11|20|4|1982|332|4325|Saturday|1982Q4|N|Y|N|2445275|2445578|2444929|2445202|N|N|N|N|N| +2445295|AAAAAAAAPOPEFCAA|1982-11-21|994|4325|332|1982|0|11|21|4|1982|332|4325|Sunday|1982Q4|N|N|N|2445275|2445578|2444930|2445203|N|N|N|N|N| +2445296|AAAAAAAAAPPEFCAA|1982-11-22|994|4325|332|1982|1|11|22|4|1982|332|4325|Monday|1982Q4|N|N|N|2445275|2445578|2444931|2445204|N|N|N|N|N| +2445297|AAAAAAAABPPEFCAA|1982-11-23|994|4326|332|1982|2|11|23|4|1982|332|4326|Tuesday|1982Q4|N|N|N|2445275|2445578|2444932|2445205|N|N|N|N|N| +2445298|AAAAAAAACPPEFCAA|1982-11-24|994|4326|332|1982|3|11|24|4|1982|332|4326|Wednesday|1982Q4|N|N|N|2445275|2445578|2444933|2445206|N|N|N|N|N| +2445299|AAAAAAAADPPEFCAA|1982-11-25|994|4326|332|1982|4|11|25|4|1982|332|4326|Thursday|1982Q4|N|N|N|2445275|2445578|2444934|2445207|N|N|N|N|N| +2445300|AAAAAAAAEPPEFCAA|1982-11-26|994|4326|332|1982|5|11|26|4|1982|332|4326|Friday|1982Q4|N|Y|N|2445275|2445578|2444935|2445208|N|N|N|N|N| +2445301|AAAAAAAAFPPEFCAA|1982-11-27|994|4326|332|1982|6|11|27|4|1982|332|4326|Saturday|1982Q4|N|Y|N|2445275|2445578|2444936|2445209|N|N|N|N|N| +2445302|AAAAAAAAGPPEFCAA|1982-11-28|994|4326|332|1982|0|11|28|4|1982|332|4326|Sunday|1982Q4|N|N|N|2445275|2445578|2444937|2445210|N|N|N|N|N| +2445303|AAAAAAAAHPPEFCAA|1982-11-29|994|4326|332|1982|1|11|29|4|1982|332|4326|Monday|1982Q4|N|N|N|2445275|2445578|2444938|2445211|N|N|N|N|N| +2445304|AAAAAAAAIPPEFCAA|1982-11-30|994|4327|332|1982|2|11|30|4|1982|332|4327|Tuesday|1982Q4|N|N|N|2445275|2445578|2444939|2445212|N|N|N|N|N| +2445305|AAAAAAAAJPPEFCAA|1982-12-01|995|4327|333|1982|3|12|1|4|1982|333|4327|Wednesday|1982Q4|N|N|N|2445305|2445638|2444940|2445213|N|N|N|N|N| +2445306|AAAAAAAAKPPEFCAA|1982-12-02|995|4327|333|1982|4|12|2|4|1982|333|4327|Thursday|1982Q4|N|N|N|2445305|2445638|2444941|2445214|N|N|N|N|N| +2445307|AAAAAAAALPPEFCAA|1982-12-03|995|4327|333|1982|5|12|3|4|1982|333|4327|Friday|1982Q4|N|Y|N|2445305|2445638|2444942|2445215|N|N|N|N|N| +2445308|AAAAAAAAMPPEFCAA|1982-12-04|995|4327|333|1982|6|12|4|4|1982|333|4327|Saturday|1982Q4|N|Y|N|2445305|2445638|2444943|2445216|N|N|N|N|N| +2445309|AAAAAAAANPPEFCAA|1982-12-05|995|4327|333|1982|0|12|5|4|1982|333|4327|Sunday|1982Q4|N|N|N|2445305|2445638|2444944|2445217|N|N|N|N|N| +2445310|AAAAAAAAOPPEFCAA|1982-12-06|995|4327|333|1982|1|12|6|4|1982|333|4327|Monday|1982Q4|N|N|N|2445305|2445638|2444945|2445218|N|N|N|N|N| +2445311|AAAAAAAAPPPEFCAA|1982-12-07|995|4328|333|1982|2|12|7|4|1982|333|4328|Tuesday|1982Q4|N|N|N|2445305|2445638|2444946|2445219|N|N|N|N|N| +2445312|AAAAAAAAAAAFFCAA|1982-12-08|995|4328|333|1982|3|12|8|4|1982|333|4328|Wednesday|1982Q4|N|N|N|2445305|2445638|2444947|2445220|N|N|N|N|N| +2445313|AAAAAAAABAAFFCAA|1982-12-09|995|4328|333|1982|4|12|9|4|1982|333|4328|Thursday|1982Q4|N|N|N|2445305|2445638|2444948|2445221|N|N|N|N|N| +2445314|AAAAAAAACAAFFCAA|1982-12-10|995|4328|333|1982|5|12|10|4|1982|333|4328|Friday|1982Q4|N|Y|N|2445305|2445638|2444949|2445222|N|N|N|N|N| +2445315|AAAAAAAADAAFFCAA|1982-12-11|995|4328|333|1982|6|12|11|4|1982|333|4328|Saturday|1982Q4|N|Y|N|2445305|2445638|2444950|2445223|N|N|N|N|N| +2445316|AAAAAAAAEAAFFCAA|1982-12-12|995|4328|333|1982|0|12|12|4|1982|333|4328|Sunday|1982Q4|N|N|N|2445305|2445638|2444951|2445224|N|N|N|N|N| +2445317|AAAAAAAAFAAFFCAA|1982-12-13|995|4328|333|1982|1|12|13|4|1982|333|4328|Monday|1982Q4|N|N|N|2445305|2445638|2444952|2445225|N|N|N|N|N| +2445318|AAAAAAAAGAAFFCAA|1982-12-14|995|4329|333|1982|2|12|14|4|1982|333|4329|Tuesday|1982Q4|N|N|N|2445305|2445638|2444953|2445226|N|N|N|N|N| +2445319|AAAAAAAAHAAFFCAA|1982-12-15|995|4329|333|1982|3|12|15|4|1982|333|4329|Wednesday|1982Q4|N|N|N|2445305|2445638|2444954|2445227|N|N|N|N|N| +2445320|AAAAAAAAIAAFFCAA|1982-12-16|995|4329|333|1982|4|12|16|4|1982|333|4329|Thursday|1982Q4|N|N|N|2445305|2445638|2444955|2445228|N|N|N|N|N| +2445321|AAAAAAAAJAAFFCAA|1982-12-17|995|4329|333|1982|5|12|17|4|1982|333|4329|Friday|1982Q4|N|Y|N|2445305|2445638|2444956|2445229|N|N|N|N|N| +2445322|AAAAAAAAKAAFFCAA|1982-12-18|995|4329|333|1982|6|12|18|4|1982|333|4329|Saturday|1982Q4|N|Y|N|2445305|2445638|2444957|2445230|N|N|N|N|N| +2445323|AAAAAAAALAAFFCAA|1982-12-19|995|4329|333|1982|0|12|19|4|1982|333|4329|Sunday|1982Q4|N|N|N|2445305|2445638|2444958|2445231|N|N|N|N|N| +2445324|AAAAAAAAMAAFFCAA|1982-12-20|995|4329|333|1982|1|12|20|4|1982|333|4329|Monday|1982Q4|N|N|N|2445305|2445638|2444959|2445232|N|N|N|N|N| +2445325|AAAAAAAANAAFFCAA|1982-12-21|995|4330|333|1982|2|12|21|4|1982|333|4330|Tuesday|1982Q4|N|N|N|2445305|2445638|2444960|2445233|N|N|N|N|N| +2445326|AAAAAAAAOAAFFCAA|1982-12-22|995|4330|333|1982|3|12|22|4|1982|333|4330|Wednesday|1982Q4|N|N|N|2445305|2445638|2444961|2445234|N|N|N|N|N| +2445327|AAAAAAAAPAAFFCAA|1982-12-23|995|4330|333|1982|4|12|23|4|1982|333|4330|Thursday|1982Q4|N|N|N|2445305|2445638|2444962|2445235|N|N|N|N|N| +2445328|AAAAAAAAABAFFCAA|1982-12-24|995|4330|333|1982|5|12|24|4|1982|333|4330|Friday|1982Q4|N|Y|N|2445305|2445638|2444963|2445236|N|N|N|N|N| +2445329|AAAAAAAABBAFFCAA|1982-12-25|995|4330|333|1982|6|12|25|4|1982|333|4330|Saturday|1982Q4|N|Y|N|2445305|2445638|2444964|2445237|N|N|N|N|N| +2445330|AAAAAAAACBAFFCAA|1982-12-26|995|4330|333|1982|0|12|26|4|1982|333|4330|Sunday|1982Q4|Y|N|N|2445305|2445638|2444965|2445238|N|N|N|N|N| +2445331|AAAAAAAADBAFFCAA|1982-12-27|995|4330|333|1982|1|12|27|4|1982|333|4330|Monday|1982Q4|N|N|Y|2445305|2445638|2444966|2445239|N|N|N|N|N| +2445332|AAAAAAAAEBAFFCAA|1982-12-28|995|4331|333|1982|2|12|28|4|1982|333|4331|Tuesday|1982Q4|N|N|N|2445305|2445638|2444967|2445240|N|N|N|N|N| +2445333|AAAAAAAAFBAFFCAA|1982-12-29|995|4331|333|1982|3|12|29|4|1982|333|4331|Wednesday|1982Q4|N|N|N|2445305|2445638|2444968|2445241|N|N|N|N|N| +2445334|AAAAAAAAGBAFFCAA|1982-12-30|995|4331|333|1982|4|12|30|4|1982|333|4331|Thursday|1982Q4|N|N|N|2445305|2445638|2444969|2445242|N|N|N|N|N| +2445335|AAAAAAAAHBAFFCAA|1982-12-31|995|4331|333|1982|5|12|31|4|1982|333|4331|Friday|1982Q4|N|Y|N|2445305|2445638|2444970|2445243|N|N|N|N|N| +2445336|AAAAAAAAIBAFFCAA|1983-01-01|996|4331|333|1983|6|1|1|1|1983|333|4331|Saturday|1983Q1|Y|Y|N|2445336|2445335|2444971|2445244|N|N|N|N|N| +2445337|AAAAAAAAJBAFFCAA|1983-01-02|996|4331|333|1983|0|1|2|1|1983|333|4331|Sunday|1983Q1|N|N|Y|2445336|2445335|2444972|2445245|N|N|N|N|N| +2445338|AAAAAAAAKBAFFCAA|1983-01-03|996|4331|333|1983|1|1|3|1|1983|333|4331|Monday|1983Q1|N|N|N|2445336|2445335|2444973|2445246|N|N|N|N|N| +2445339|AAAAAAAALBAFFCAA|1983-01-04|996|4332|333|1983|2|1|4|1|1983|333|4332|Tuesday|1983Q1|N|N|N|2445336|2445335|2444974|2445247|N|N|N|N|N| +2445340|AAAAAAAAMBAFFCAA|1983-01-05|996|4332|333|1983|3|1|5|1|1983|333|4332|Wednesday|1983Q1|N|N|N|2445336|2445335|2444975|2445248|N|N|N|N|N| +2445341|AAAAAAAANBAFFCAA|1983-01-06|996|4332|333|1983|4|1|6|1|1983|333|4332|Thursday|1983Q1|N|N|N|2445336|2445335|2444976|2445249|N|N|N|N|N| +2445342|AAAAAAAAOBAFFCAA|1983-01-07|996|4332|333|1983|5|1|7|1|1983|333|4332|Friday|1983Q1|N|Y|N|2445336|2445335|2444977|2445250|N|N|N|N|N| +2445343|AAAAAAAAPBAFFCAA|1983-01-08|996|4332|333|1983|6|1|8|1|1983|333|4332|Saturday|1983Q1|N|Y|N|2445336|2445335|2444978|2445251|N|N|N|N|N| +2445344|AAAAAAAAACAFFCAA|1983-01-09|996|4332|333|1983|0|1|9|1|1983|333|4332|Sunday|1983Q1|N|N|N|2445336|2445335|2444979|2445252|N|N|N|N|N| +2445345|AAAAAAAABCAFFCAA|1983-01-10|996|4332|333|1983|1|1|10|1|1983|333|4332|Monday|1983Q1|N|N|N|2445336|2445335|2444980|2445253|N|N|N|N|N| +2445346|AAAAAAAACCAFFCAA|1983-01-11|996|4333|333|1983|2|1|11|1|1983|333|4333|Tuesday|1983Q1|N|N|N|2445336|2445335|2444981|2445254|N|N|N|N|N| +2445347|AAAAAAAADCAFFCAA|1983-01-12|996|4333|333|1983|3|1|12|1|1983|333|4333|Wednesday|1983Q1|N|N|N|2445336|2445335|2444982|2445255|N|N|N|N|N| +2445348|AAAAAAAAECAFFCAA|1983-01-13|996|4333|333|1983|4|1|13|1|1983|333|4333|Thursday|1983Q1|N|N|N|2445336|2445335|2444983|2445256|N|N|N|N|N| +2445349|AAAAAAAAFCAFFCAA|1983-01-14|996|4333|333|1983|5|1|14|1|1983|333|4333|Friday|1983Q1|N|Y|N|2445336|2445335|2444984|2445257|N|N|N|N|N| +2445350|AAAAAAAAGCAFFCAA|1983-01-15|996|4333|333|1983|6|1|15|1|1983|333|4333|Saturday|1983Q1|N|Y|N|2445336|2445335|2444985|2445258|N|N|N|N|N| +2445351|AAAAAAAAHCAFFCAA|1983-01-16|996|4333|333|1983|0|1|16|1|1983|333|4333|Sunday|1983Q1|N|N|N|2445336|2445335|2444986|2445259|N|N|N|N|N| +2445352|AAAAAAAAICAFFCAA|1983-01-17|996|4333|333|1983|1|1|17|1|1983|333|4333|Monday|1983Q1|N|N|N|2445336|2445335|2444987|2445260|N|N|N|N|N| +2445353|AAAAAAAAJCAFFCAA|1983-01-18|996|4334|333|1983|2|1|18|1|1983|333|4334|Tuesday|1983Q1|N|N|N|2445336|2445335|2444988|2445261|N|N|N|N|N| +2445354|AAAAAAAAKCAFFCAA|1983-01-19|996|4334|333|1983|3|1|19|1|1983|333|4334|Wednesday|1983Q1|N|N|N|2445336|2445335|2444989|2445262|N|N|N|N|N| +2445355|AAAAAAAALCAFFCAA|1983-01-20|996|4334|333|1983|4|1|20|1|1983|333|4334|Thursday|1983Q1|N|N|N|2445336|2445335|2444990|2445263|N|N|N|N|N| +2445356|AAAAAAAAMCAFFCAA|1983-01-21|996|4334|333|1983|5|1|21|1|1983|333|4334|Friday|1983Q1|N|Y|N|2445336|2445335|2444991|2445264|N|N|N|N|N| +2445357|AAAAAAAANCAFFCAA|1983-01-22|996|4334|333|1983|6|1|22|1|1983|333|4334|Saturday|1983Q1|N|Y|N|2445336|2445335|2444992|2445265|N|N|N|N|N| +2445358|AAAAAAAAOCAFFCAA|1983-01-23|996|4334|333|1983|0|1|23|1|1983|333|4334|Sunday|1983Q1|N|N|N|2445336|2445335|2444993|2445266|N|N|N|N|N| +2445359|AAAAAAAAPCAFFCAA|1983-01-24|996|4334|333|1983|1|1|24|1|1983|333|4334|Monday|1983Q1|N|N|N|2445336|2445335|2444994|2445267|N|N|N|N|N| +2445360|AAAAAAAAADAFFCAA|1983-01-25|996|4335|333|1983|2|1|25|1|1983|333|4335|Tuesday|1983Q1|N|N|N|2445336|2445335|2444995|2445268|N|N|N|N|N| +2445361|AAAAAAAABDAFFCAA|1983-01-26|996|4335|333|1983|3|1|26|1|1983|333|4335|Wednesday|1983Q1|N|N|N|2445336|2445335|2444996|2445269|N|N|N|N|N| +2445362|AAAAAAAACDAFFCAA|1983-01-27|996|4335|333|1983|4|1|27|1|1983|333|4335|Thursday|1983Q1|N|N|N|2445336|2445335|2444997|2445270|N|N|N|N|N| +2445363|AAAAAAAADDAFFCAA|1983-01-28|996|4335|333|1983|5|1|28|1|1983|333|4335|Friday|1983Q1|N|Y|N|2445336|2445335|2444998|2445271|N|N|N|N|N| +2445364|AAAAAAAAEDAFFCAA|1983-01-29|996|4335|333|1983|6|1|29|1|1983|333|4335|Saturday|1983Q1|N|Y|N|2445336|2445335|2444999|2445272|N|N|N|N|N| +2445365|AAAAAAAAFDAFFCAA|1983-01-30|996|4335|333|1983|0|1|30|1|1983|333|4335|Sunday|1983Q1|N|N|N|2445336|2445335|2445000|2445273|N|N|N|N|N| +2445366|AAAAAAAAGDAFFCAA|1983-01-31|996|4335|333|1983|1|1|31|1|1983|333|4335|Monday|1983Q1|N|N|N|2445336|2445335|2445001|2445274|N|N|N|N|N| +2445367|AAAAAAAAHDAFFCAA|1983-02-01|997|4336|333|1983|2|2|1|1|1983|333|4336|Tuesday|1983Q1|N|N|N|2445367|2445397|2445002|2445275|N|N|N|N|N| +2445368|AAAAAAAAIDAFFCAA|1983-02-02|997|4336|333|1983|3|2|2|1|1983|333|4336|Wednesday|1983Q1|N|N|N|2445367|2445397|2445003|2445276|N|N|N|N|N| +2445369|AAAAAAAAJDAFFCAA|1983-02-03|997|4336|333|1983|4|2|3|1|1983|333|4336|Thursday|1983Q1|N|N|N|2445367|2445397|2445004|2445277|N|N|N|N|N| +2445370|AAAAAAAAKDAFFCAA|1983-02-04|997|4336|333|1983|5|2|4|1|1983|333|4336|Friday|1983Q1|N|Y|N|2445367|2445397|2445005|2445278|N|N|N|N|N| +2445371|AAAAAAAALDAFFCAA|1983-02-05|997|4336|333|1983|6|2|5|1|1983|333|4336|Saturday|1983Q1|N|Y|N|2445367|2445397|2445006|2445279|N|N|N|N|N| +2445372|AAAAAAAAMDAFFCAA|1983-02-06|997|4336|333|1983|0|2|6|1|1983|333|4336|Sunday|1983Q1|N|N|N|2445367|2445397|2445007|2445280|N|N|N|N|N| +2445373|AAAAAAAANDAFFCAA|1983-02-07|997|4336|333|1983|1|2|7|1|1983|333|4336|Monday|1983Q1|N|N|N|2445367|2445397|2445008|2445281|N|N|N|N|N| +2445374|AAAAAAAAODAFFCAA|1983-02-08|997|4337|333|1983|2|2|8|1|1983|333|4337|Tuesday|1983Q1|N|N|N|2445367|2445397|2445009|2445282|N|N|N|N|N| +2445375|AAAAAAAAPDAFFCAA|1983-02-09|997|4337|333|1983|3|2|9|1|1983|333|4337|Wednesday|1983Q1|N|N|N|2445367|2445397|2445010|2445283|N|N|N|N|N| +2445376|AAAAAAAAAEAFFCAA|1983-02-10|997|4337|333|1983|4|2|10|1|1983|333|4337|Thursday|1983Q1|N|N|N|2445367|2445397|2445011|2445284|N|N|N|N|N| +2445377|AAAAAAAABEAFFCAA|1983-02-11|997|4337|333|1983|5|2|11|1|1983|333|4337|Friday|1983Q1|N|Y|N|2445367|2445397|2445012|2445285|N|N|N|N|N| +2445378|AAAAAAAACEAFFCAA|1983-02-12|997|4337|333|1983|6|2|12|1|1983|333|4337|Saturday|1983Q1|N|Y|N|2445367|2445397|2445013|2445286|N|N|N|N|N| +2445379|AAAAAAAADEAFFCAA|1983-02-13|997|4337|333|1983|0|2|13|1|1983|333|4337|Sunday|1983Q1|N|N|N|2445367|2445397|2445014|2445287|N|N|N|N|N| +2445380|AAAAAAAAEEAFFCAA|1983-02-14|997|4337|333|1983|1|2|14|1|1983|333|4337|Monday|1983Q1|N|N|N|2445367|2445397|2445015|2445288|N|N|N|N|N| +2445381|AAAAAAAAFEAFFCAA|1983-02-15|997|4338|333|1983|2|2|15|1|1983|333|4338|Tuesday|1983Q1|N|N|N|2445367|2445397|2445016|2445289|N|N|N|N|N| +2445382|AAAAAAAAGEAFFCAA|1983-02-16|997|4338|333|1983|3|2|16|1|1983|333|4338|Wednesday|1983Q1|N|N|N|2445367|2445397|2445017|2445290|N|N|N|N|N| +2445383|AAAAAAAAHEAFFCAA|1983-02-17|997|4338|333|1983|4|2|17|1|1983|333|4338|Thursday|1983Q1|N|N|N|2445367|2445397|2445018|2445291|N|N|N|N|N| +2445384|AAAAAAAAIEAFFCAA|1983-02-18|997|4338|333|1983|5|2|18|1|1983|333|4338|Friday|1983Q1|N|Y|N|2445367|2445397|2445019|2445292|N|N|N|N|N| +2445385|AAAAAAAAJEAFFCAA|1983-02-19|997|4338|333|1983|6|2|19|1|1983|333|4338|Saturday|1983Q1|N|Y|N|2445367|2445397|2445020|2445293|N|N|N|N|N| +2445386|AAAAAAAAKEAFFCAA|1983-02-20|997|4338|333|1983|0|2|20|1|1983|333|4338|Sunday|1983Q1|N|N|N|2445367|2445397|2445021|2445294|N|N|N|N|N| +2445387|AAAAAAAALEAFFCAA|1983-02-21|997|4338|333|1983|1|2|21|1|1983|333|4338|Monday|1983Q1|N|N|N|2445367|2445397|2445022|2445295|N|N|N|N|N| +2445388|AAAAAAAAMEAFFCAA|1983-02-22|997|4339|333|1983|2|2|22|1|1983|333|4339|Tuesday|1983Q1|N|N|N|2445367|2445397|2445023|2445296|N|N|N|N|N| +2445389|AAAAAAAANEAFFCAA|1983-02-23|997|4339|333|1983|3|2|23|1|1983|333|4339|Wednesday|1983Q1|N|N|N|2445367|2445397|2445024|2445297|N|N|N|N|N| +2445390|AAAAAAAAOEAFFCAA|1983-02-24|997|4339|333|1983|4|2|24|1|1983|333|4339|Thursday|1983Q1|N|N|N|2445367|2445397|2445025|2445298|N|N|N|N|N| +2445391|AAAAAAAAPEAFFCAA|1983-02-25|997|4339|333|1983|5|2|25|1|1983|333|4339|Friday|1983Q1|N|Y|N|2445367|2445397|2445026|2445299|N|N|N|N|N| +2445392|AAAAAAAAAFAFFCAA|1983-02-26|997|4339|333|1983|6|2|26|1|1983|333|4339|Saturday|1983Q1|N|Y|N|2445367|2445397|2445027|2445300|N|N|N|N|N| +2445393|AAAAAAAABFAFFCAA|1983-02-27|997|4339|333|1983|0|2|27|1|1983|333|4339|Sunday|1983Q1|N|N|N|2445367|2445397|2445028|2445301|N|N|N|N|N| +2445394|AAAAAAAACFAFFCAA|1983-02-28|997|4339|333|1983|1|2|28|1|1983|333|4339|Monday|1983Q1|N|N|N|2445367|2445397|2445029|2445302|N|N|N|N|N| +2445395|AAAAAAAADFAFFCAA|1983-03-01|998|4340|334|1983|2|3|1|1|1983|334|4340|Tuesday|1983Q1|N|N|N|2445395|2445453|2445030|2445303|N|N|N|N|N| +2445396|AAAAAAAAEFAFFCAA|1983-03-02|998|4340|334|1983|3|3|2|1|1983|334|4340|Wednesday|1983Q1|N|N|N|2445395|2445453|2445031|2445304|N|N|N|N|N| +2445397|AAAAAAAAFFAFFCAA|1983-03-03|998|4340|334|1983|4|3|3|1|1983|334|4340|Thursday|1983Q1|N|N|N|2445395|2445453|2445032|2445305|N|N|N|N|N| +2445398|AAAAAAAAGFAFFCAA|1983-03-04|998|4340|334|1983|5|3|4|1|1983|334|4340|Friday|1983Q1|N|Y|N|2445395|2445453|2445033|2445306|N|N|N|N|N| +2445399|AAAAAAAAHFAFFCAA|1983-03-05|998|4340|334|1983|6|3|5|1|1983|334|4340|Saturday|1983Q1|N|Y|N|2445395|2445453|2445034|2445307|N|N|N|N|N| +2445400|AAAAAAAAIFAFFCAA|1983-03-06|998|4340|334|1983|0|3|6|1|1983|334|4340|Sunday|1983Q1|N|N|N|2445395|2445453|2445035|2445308|N|N|N|N|N| +2445401|AAAAAAAAJFAFFCAA|1983-03-07|998|4340|334|1983|1|3|7|1|1983|334|4340|Monday|1983Q1|N|N|N|2445395|2445453|2445036|2445309|N|N|N|N|N| +2445402|AAAAAAAAKFAFFCAA|1983-03-08|998|4341|334|1983|2|3|8|1|1983|334|4341|Tuesday|1983Q1|N|N|N|2445395|2445453|2445037|2445310|N|N|N|N|N| +2445403|AAAAAAAALFAFFCAA|1983-03-09|998|4341|334|1983|3|3|9|1|1983|334|4341|Wednesday|1983Q1|N|N|N|2445395|2445453|2445038|2445311|N|N|N|N|N| +2445404|AAAAAAAAMFAFFCAA|1983-03-10|998|4341|334|1983|4|3|10|1|1983|334|4341|Thursday|1983Q1|N|N|N|2445395|2445453|2445039|2445312|N|N|N|N|N| +2445405|AAAAAAAANFAFFCAA|1983-03-11|998|4341|334|1983|5|3|11|1|1983|334|4341|Friday|1983Q1|N|Y|N|2445395|2445453|2445040|2445313|N|N|N|N|N| +2445406|AAAAAAAAOFAFFCAA|1983-03-12|998|4341|334|1983|6|3|12|1|1983|334|4341|Saturday|1983Q1|N|Y|N|2445395|2445453|2445041|2445314|N|N|N|N|N| +2445407|AAAAAAAAPFAFFCAA|1983-03-13|998|4341|334|1983|0|3|13|1|1983|334|4341|Sunday|1983Q1|N|N|N|2445395|2445453|2445042|2445315|N|N|N|N|N| +2445408|AAAAAAAAAGAFFCAA|1983-03-14|998|4341|334|1983|1|3|14|1|1983|334|4341|Monday|1983Q1|N|N|N|2445395|2445453|2445043|2445316|N|N|N|N|N| +2445409|AAAAAAAABGAFFCAA|1983-03-15|998|4342|334|1983|2|3|15|1|1983|334|4342|Tuesday|1983Q1|N|N|N|2445395|2445453|2445044|2445317|N|N|N|N|N| +2445410|AAAAAAAACGAFFCAA|1983-03-16|998|4342|334|1983|3|3|16|1|1983|334|4342|Wednesday|1983Q1|N|N|N|2445395|2445453|2445045|2445318|N|N|N|N|N| +2445411|AAAAAAAADGAFFCAA|1983-03-17|998|4342|334|1983|4|3|17|1|1983|334|4342|Thursday|1983Q1|N|N|N|2445395|2445453|2445046|2445319|N|N|N|N|N| +2445412|AAAAAAAAEGAFFCAA|1983-03-18|998|4342|334|1983|5|3|18|1|1983|334|4342|Friday|1983Q1|N|Y|N|2445395|2445453|2445047|2445320|N|N|N|N|N| +2445413|AAAAAAAAFGAFFCAA|1983-03-19|998|4342|334|1983|6|3|19|1|1983|334|4342|Saturday|1983Q1|N|Y|N|2445395|2445453|2445048|2445321|N|N|N|N|N| +2445414|AAAAAAAAGGAFFCAA|1983-03-20|998|4342|334|1983|0|3|20|1|1983|334|4342|Sunday|1983Q1|N|N|N|2445395|2445453|2445049|2445322|N|N|N|N|N| +2445415|AAAAAAAAHGAFFCAA|1983-03-21|998|4342|334|1983|1|3|21|1|1983|334|4342|Monday|1983Q1|N|N|N|2445395|2445453|2445050|2445323|N|N|N|N|N| +2445416|AAAAAAAAIGAFFCAA|1983-03-22|998|4343|334|1983|2|3|22|1|1983|334|4343|Tuesday|1983Q1|N|N|N|2445395|2445453|2445051|2445324|N|N|N|N|N| +2445417|AAAAAAAAJGAFFCAA|1983-03-23|998|4343|334|1983|3|3|23|1|1983|334|4343|Wednesday|1983Q1|N|N|N|2445395|2445453|2445052|2445325|N|N|N|N|N| +2445418|AAAAAAAAKGAFFCAA|1983-03-24|998|4343|334|1983|4|3|24|1|1983|334|4343|Thursday|1983Q1|N|N|N|2445395|2445453|2445053|2445326|N|N|N|N|N| +2445419|AAAAAAAALGAFFCAA|1983-03-25|998|4343|334|1983|5|3|25|1|1983|334|4343|Friday|1983Q1|N|Y|N|2445395|2445453|2445054|2445327|N|N|N|N|N| +2445420|AAAAAAAAMGAFFCAA|1983-03-26|998|4343|334|1983|6|3|26|1|1983|334|4343|Saturday|1983Q1|N|Y|N|2445395|2445453|2445055|2445328|N|N|N|N|N| +2445421|AAAAAAAANGAFFCAA|1983-03-27|998|4343|334|1983|0|3|27|1|1983|334|4343|Sunday|1983Q1|N|N|N|2445395|2445453|2445056|2445329|N|N|N|N|N| +2445422|AAAAAAAAOGAFFCAA|1983-03-28|998|4343|334|1983|1|3|28|1|1983|334|4343|Monday|1983Q1|N|N|N|2445395|2445453|2445057|2445330|N|N|N|N|N| +2445423|AAAAAAAAPGAFFCAA|1983-03-29|998|4344|334|1983|2|3|29|1|1983|334|4344|Tuesday|1983Q1|N|N|N|2445395|2445453|2445058|2445331|N|N|N|N|N| +2445424|AAAAAAAAAHAFFCAA|1983-03-30|998|4344|334|1983|3|3|30|1|1983|334|4344|Wednesday|1983Q1|N|N|N|2445395|2445453|2445059|2445332|N|N|N|N|N| +2445425|AAAAAAAABHAFFCAA|1983-03-31|998|4344|334|1983|4|3|31|1|1983|334|4344|Thursday|1983Q1|N|N|N|2445395|2445453|2445060|2445333|N|N|N|N|N| +2445426|AAAAAAAACHAFFCAA|1983-04-01|999|4344|334|1983|5|4|1|1|1983|334|4344|Friday|1983Q1|N|Y|N|2445426|2445515|2445061|2445336|N|N|N|N|N| +2445427|AAAAAAAADHAFFCAA|1983-04-02|999|4344|334|1983|6|4|2|2|1983|334|4344|Saturday|1983Q2|N|Y|N|2445426|2445515|2445062|2445337|N|N|N|N|N| +2445428|AAAAAAAAEHAFFCAA|1983-04-03|999|4344|334|1983|0|4|3|2|1983|334|4344|Sunday|1983Q2|N|N|N|2445426|2445515|2445063|2445338|N|N|N|N|N| +2445429|AAAAAAAAFHAFFCAA|1983-04-04|999|4344|334|1983|1|4|4|2|1983|334|4344|Monday|1983Q2|N|N|N|2445426|2445515|2445064|2445339|N|N|N|N|N| +2445430|AAAAAAAAGHAFFCAA|1983-04-05|999|4345|334|1983|2|4|5|2|1983|334|4345|Tuesday|1983Q2|N|N|N|2445426|2445515|2445065|2445340|N|N|N|N|N| +2445431|AAAAAAAAHHAFFCAA|1983-04-06|999|4345|334|1983|3|4|6|2|1983|334|4345|Wednesday|1983Q2|N|N|N|2445426|2445515|2445066|2445341|N|N|N|N|N| +2445432|AAAAAAAAIHAFFCAA|1983-04-07|999|4345|334|1983|4|4|7|2|1983|334|4345|Thursday|1983Q2|N|N|N|2445426|2445515|2445067|2445342|N|N|N|N|N| +2445433|AAAAAAAAJHAFFCAA|1983-04-08|999|4345|334|1983|5|4|8|2|1983|334|4345|Friday|1983Q2|N|Y|N|2445426|2445515|2445068|2445343|N|N|N|N|N| +2445434|AAAAAAAAKHAFFCAA|1983-04-09|999|4345|334|1983|6|4|9|2|1983|334|4345|Saturday|1983Q2|N|Y|N|2445426|2445515|2445069|2445344|N|N|N|N|N| +2445435|AAAAAAAALHAFFCAA|1983-04-10|999|4345|334|1983|0|4|10|2|1983|334|4345|Sunday|1983Q2|N|N|N|2445426|2445515|2445070|2445345|N|N|N|N|N| +2445436|AAAAAAAAMHAFFCAA|1983-04-11|999|4345|334|1983|1|4|11|2|1983|334|4345|Monday|1983Q2|N|N|N|2445426|2445515|2445071|2445346|N|N|N|N|N| +2445437|AAAAAAAANHAFFCAA|1983-04-12|999|4346|334|1983|2|4|12|2|1983|334|4346|Tuesday|1983Q2|N|N|N|2445426|2445515|2445072|2445347|N|N|N|N|N| +2445438|AAAAAAAAOHAFFCAA|1983-04-13|999|4346|334|1983|3|4|13|2|1983|334|4346|Wednesday|1983Q2|N|N|N|2445426|2445515|2445073|2445348|N|N|N|N|N| +2445439|AAAAAAAAPHAFFCAA|1983-04-14|999|4346|334|1983|4|4|14|2|1983|334|4346|Thursday|1983Q2|N|N|N|2445426|2445515|2445074|2445349|N|N|N|N|N| +2445440|AAAAAAAAAIAFFCAA|1983-04-15|999|4346|334|1983|5|4|15|2|1983|334|4346|Friday|1983Q2|N|Y|N|2445426|2445515|2445075|2445350|N|N|N|N|N| +2445441|AAAAAAAABIAFFCAA|1983-04-16|999|4346|334|1983|6|4|16|2|1983|334|4346|Saturday|1983Q2|N|Y|N|2445426|2445515|2445076|2445351|N|N|N|N|N| +2445442|AAAAAAAACIAFFCAA|1983-04-17|999|4346|334|1983|0|4|17|2|1983|334|4346|Sunday|1983Q2|N|N|N|2445426|2445515|2445077|2445352|N|N|N|N|N| +2445443|AAAAAAAADIAFFCAA|1983-04-18|999|4346|334|1983|1|4|18|2|1983|334|4346|Monday|1983Q2|N|N|N|2445426|2445515|2445078|2445353|N|N|N|N|N| +2445444|AAAAAAAAEIAFFCAA|1983-04-19|999|4347|334|1983|2|4|19|2|1983|334|4347|Tuesday|1983Q2|N|N|N|2445426|2445515|2445079|2445354|N|N|N|N|N| +2445445|AAAAAAAAFIAFFCAA|1983-04-20|999|4347|334|1983|3|4|20|2|1983|334|4347|Wednesday|1983Q2|N|N|N|2445426|2445515|2445080|2445355|N|N|N|N|N| +2445446|AAAAAAAAGIAFFCAA|1983-04-21|999|4347|334|1983|4|4|21|2|1983|334|4347|Thursday|1983Q2|N|N|N|2445426|2445515|2445081|2445356|N|N|N|N|N| +2445447|AAAAAAAAHIAFFCAA|1983-04-22|999|4347|334|1983|5|4|22|2|1983|334|4347|Friday|1983Q2|N|Y|N|2445426|2445515|2445082|2445357|N|N|N|N|N| +2445448|AAAAAAAAIIAFFCAA|1983-04-23|999|4347|334|1983|6|4|23|2|1983|334|4347|Saturday|1983Q2|N|Y|N|2445426|2445515|2445083|2445358|N|N|N|N|N| +2445449|AAAAAAAAJIAFFCAA|1983-04-24|999|4347|334|1983|0|4|24|2|1983|334|4347|Sunday|1983Q2|N|N|N|2445426|2445515|2445084|2445359|N|N|N|N|N| +2445450|AAAAAAAAKIAFFCAA|1983-04-25|999|4347|334|1983|1|4|25|2|1983|334|4347|Monday|1983Q2|N|N|N|2445426|2445515|2445085|2445360|N|N|N|N|N| +2445451|AAAAAAAALIAFFCAA|1983-04-26|999|4348|334|1983|2|4|26|2|1983|334|4348|Tuesday|1983Q2|N|N|N|2445426|2445515|2445086|2445361|N|N|N|N|N| +2445452|AAAAAAAAMIAFFCAA|1983-04-27|999|4348|334|1983|3|4|27|2|1983|334|4348|Wednesday|1983Q2|N|N|N|2445426|2445515|2445087|2445362|N|N|N|N|N| +2445453|AAAAAAAANIAFFCAA|1983-04-28|999|4348|334|1983|4|4|28|2|1983|334|4348|Thursday|1983Q2|N|N|N|2445426|2445515|2445088|2445363|N|N|N|N|N| +2445454|AAAAAAAAOIAFFCAA|1983-04-29|999|4348|334|1983|5|4|29|2|1983|334|4348|Friday|1983Q2|N|Y|N|2445426|2445515|2445089|2445364|N|N|N|N|N| +2445455|AAAAAAAAPIAFFCAA|1983-04-30|999|4348|334|1983|6|4|30|2|1983|334|4348|Saturday|1983Q2|N|Y|N|2445426|2445515|2445090|2445365|N|N|N|N|N| +2445456|AAAAAAAAAJAFFCAA|1983-05-01|1000|4348|334|1983|0|5|1|2|1983|334|4348|Sunday|1983Q2|N|N|N|2445456|2445575|2445091|2445366|N|N|N|N|N| +2445457|AAAAAAAABJAFFCAA|1983-05-02|1000|4348|334|1983|1|5|2|2|1983|334|4348|Monday|1983Q2|N|N|N|2445456|2445575|2445092|2445367|N|N|N|N|N| +2445458|AAAAAAAACJAFFCAA|1983-05-03|1000|4349|334|1983|2|5|3|2|1983|334|4349|Tuesday|1983Q2|N|N|N|2445456|2445575|2445093|2445368|N|N|N|N|N| +2445459|AAAAAAAADJAFFCAA|1983-05-04|1000|4349|334|1983|3|5|4|2|1983|334|4349|Wednesday|1983Q2|N|N|N|2445456|2445575|2445094|2445369|N|N|N|N|N| +2445460|AAAAAAAAEJAFFCAA|1983-05-05|1000|4349|334|1983|4|5|5|2|1983|334|4349|Thursday|1983Q2|N|N|N|2445456|2445575|2445095|2445370|N|N|N|N|N| +2445461|AAAAAAAAFJAFFCAA|1983-05-06|1000|4349|334|1983|5|5|6|2|1983|334|4349|Friday|1983Q2|N|Y|N|2445456|2445575|2445096|2445371|N|N|N|N|N| +2445462|AAAAAAAAGJAFFCAA|1983-05-07|1000|4349|334|1983|6|5|7|2|1983|334|4349|Saturday|1983Q2|N|Y|N|2445456|2445575|2445097|2445372|N|N|N|N|N| +2445463|AAAAAAAAHJAFFCAA|1983-05-08|1000|4349|334|1983|0|5|8|2|1983|334|4349|Sunday|1983Q2|N|N|N|2445456|2445575|2445098|2445373|N|N|N|N|N| +2445464|AAAAAAAAIJAFFCAA|1983-05-09|1000|4349|334|1983|1|5|9|2|1983|334|4349|Monday|1983Q2|N|N|N|2445456|2445575|2445099|2445374|N|N|N|N|N| +2445465|AAAAAAAAJJAFFCAA|1983-05-10|1000|4350|334|1983|2|5|10|2|1983|334|4350|Tuesday|1983Q2|N|N|N|2445456|2445575|2445100|2445375|N|N|N|N|N| +2445466|AAAAAAAAKJAFFCAA|1983-05-11|1000|4350|334|1983|3|5|11|2|1983|334|4350|Wednesday|1983Q2|N|N|N|2445456|2445575|2445101|2445376|N|N|N|N|N| +2445467|AAAAAAAALJAFFCAA|1983-05-12|1000|4350|334|1983|4|5|12|2|1983|334|4350|Thursday|1983Q2|N|N|N|2445456|2445575|2445102|2445377|N|N|N|N|N| +2445468|AAAAAAAAMJAFFCAA|1983-05-13|1000|4350|334|1983|5|5|13|2|1983|334|4350|Friday|1983Q2|N|Y|N|2445456|2445575|2445103|2445378|N|N|N|N|N| +2445469|AAAAAAAANJAFFCAA|1983-05-14|1000|4350|334|1983|6|5|14|2|1983|334|4350|Saturday|1983Q2|N|Y|N|2445456|2445575|2445104|2445379|N|N|N|N|N| +2445470|AAAAAAAAOJAFFCAA|1983-05-15|1000|4350|334|1983|0|5|15|2|1983|334|4350|Sunday|1983Q2|N|N|N|2445456|2445575|2445105|2445380|N|N|N|N|N| +2445471|AAAAAAAAPJAFFCAA|1983-05-16|1000|4350|334|1983|1|5|16|2|1983|334|4350|Monday|1983Q2|N|N|N|2445456|2445575|2445106|2445381|N|N|N|N|N| +2445472|AAAAAAAAAKAFFCAA|1983-05-17|1000|4351|334|1983|2|5|17|2|1983|334|4351|Tuesday|1983Q2|N|N|N|2445456|2445575|2445107|2445382|N|N|N|N|N| +2445473|AAAAAAAABKAFFCAA|1983-05-18|1000|4351|334|1983|3|5|18|2|1983|334|4351|Wednesday|1983Q2|N|N|N|2445456|2445575|2445108|2445383|N|N|N|N|N| +2445474|AAAAAAAACKAFFCAA|1983-05-19|1000|4351|334|1983|4|5|19|2|1983|334|4351|Thursday|1983Q2|N|N|N|2445456|2445575|2445109|2445384|N|N|N|N|N| +2445475|AAAAAAAADKAFFCAA|1983-05-20|1000|4351|334|1983|5|5|20|2|1983|334|4351|Friday|1983Q2|N|Y|N|2445456|2445575|2445110|2445385|N|N|N|N|N| +2445476|AAAAAAAAEKAFFCAA|1983-05-21|1000|4351|334|1983|6|5|21|2|1983|334|4351|Saturday|1983Q2|N|Y|N|2445456|2445575|2445111|2445386|N|N|N|N|N| +2445477|AAAAAAAAFKAFFCAA|1983-05-22|1000|4351|334|1983|0|5|22|2|1983|334|4351|Sunday|1983Q2|N|N|N|2445456|2445575|2445112|2445387|N|N|N|N|N| +2445478|AAAAAAAAGKAFFCAA|1983-05-23|1000|4351|334|1983|1|5|23|2|1983|334|4351|Monday|1983Q2|N|N|N|2445456|2445575|2445113|2445388|N|N|N|N|N| +2445479|AAAAAAAAHKAFFCAA|1983-05-24|1000|4352|334|1983|2|5|24|2|1983|334|4352|Tuesday|1983Q2|N|N|N|2445456|2445575|2445114|2445389|N|N|N|N|N| +2445480|AAAAAAAAIKAFFCAA|1983-05-25|1000|4352|334|1983|3|5|25|2|1983|334|4352|Wednesday|1983Q2|N|N|N|2445456|2445575|2445115|2445390|N|N|N|N|N| +2445481|AAAAAAAAJKAFFCAA|1983-05-26|1000|4352|334|1983|4|5|26|2|1983|334|4352|Thursday|1983Q2|N|N|N|2445456|2445575|2445116|2445391|N|N|N|N|N| +2445482|AAAAAAAAKKAFFCAA|1983-05-27|1000|4352|334|1983|5|5|27|2|1983|334|4352|Friday|1983Q2|N|Y|N|2445456|2445575|2445117|2445392|N|N|N|N|N| +2445483|AAAAAAAALKAFFCAA|1983-05-28|1000|4352|334|1983|6|5|28|2|1983|334|4352|Saturday|1983Q2|N|Y|N|2445456|2445575|2445118|2445393|N|N|N|N|N| +2445484|AAAAAAAAMKAFFCAA|1983-05-29|1000|4352|334|1983|0|5|29|2|1983|334|4352|Sunday|1983Q2|N|N|N|2445456|2445575|2445119|2445394|N|N|N|N|N| +2445485|AAAAAAAANKAFFCAA|1983-05-30|1000|4352|334|1983|1|5|30|2|1983|334|4352|Monday|1983Q2|N|N|N|2445456|2445575|2445120|2445395|N|N|N|N|N| +2445486|AAAAAAAAOKAFFCAA|1983-05-31|1000|4353|334|1983|2|5|31|2|1983|334|4353|Tuesday|1983Q2|N|N|N|2445456|2445575|2445121|2445396|N|N|N|N|N| +2445487|AAAAAAAAPKAFFCAA|1983-06-01|1001|4353|335|1983|3|6|1|2|1983|335|4353|Wednesday|1983Q2|N|N|N|2445487|2445637|2445122|2445397|N|N|N|N|N| +2445488|AAAAAAAAALAFFCAA|1983-06-02|1001|4353|335|1983|4|6|2|2|1983|335|4353|Thursday|1983Q2|N|N|N|2445487|2445637|2445123|2445398|N|N|N|N|N| +2445489|AAAAAAAABLAFFCAA|1983-06-03|1001|4353|335|1983|5|6|3|2|1983|335|4353|Friday|1983Q2|N|Y|N|2445487|2445637|2445124|2445399|N|N|N|N|N| +2445490|AAAAAAAACLAFFCAA|1983-06-04|1001|4353|335|1983|6|6|4|2|1983|335|4353|Saturday|1983Q2|N|Y|N|2445487|2445637|2445125|2445400|N|N|N|N|N| +2445491|AAAAAAAADLAFFCAA|1983-06-05|1001|4353|335|1983|0|6|5|2|1983|335|4353|Sunday|1983Q2|N|N|N|2445487|2445637|2445126|2445401|N|N|N|N|N| +2445492|AAAAAAAAELAFFCAA|1983-06-06|1001|4353|335|1983|1|6|6|2|1983|335|4353|Monday|1983Q2|N|N|N|2445487|2445637|2445127|2445402|N|N|N|N|N| +2445493|AAAAAAAAFLAFFCAA|1983-06-07|1001|4354|335|1983|2|6|7|2|1983|335|4354|Tuesday|1983Q2|N|N|N|2445487|2445637|2445128|2445403|N|N|N|N|N| +2445494|AAAAAAAAGLAFFCAA|1983-06-08|1001|4354|335|1983|3|6|8|2|1983|335|4354|Wednesday|1983Q2|N|N|N|2445487|2445637|2445129|2445404|N|N|N|N|N| +2445495|AAAAAAAAHLAFFCAA|1983-06-09|1001|4354|335|1983|4|6|9|2|1983|335|4354|Thursday|1983Q2|N|N|N|2445487|2445637|2445130|2445405|N|N|N|N|N| +2445496|AAAAAAAAILAFFCAA|1983-06-10|1001|4354|335|1983|5|6|10|2|1983|335|4354|Friday|1983Q2|N|Y|N|2445487|2445637|2445131|2445406|N|N|N|N|N| +2445497|AAAAAAAAJLAFFCAA|1983-06-11|1001|4354|335|1983|6|6|11|2|1983|335|4354|Saturday|1983Q2|N|Y|N|2445487|2445637|2445132|2445407|N|N|N|N|N| +2445498|AAAAAAAAKLAFFCAA|1983-06-12|1001|4354|335|1983|0|6|12|2|1983|335|4354|Sunday|1983Q2|N|N|N|2445487|2445637|2445133|2445408|N|N|N|N|N| +2445499|AAAAAAAALLAFFCAA|1983-06-13|1001|4354|335|1983|1|6|13|2|1983|335|4354|Monday|1983Q2|N|N|N|2445487|2445637|2445134|2445409|N|N|N|N|N| +2445500|AAAAAAAAMLAFFCAA|1983-06-14|1001|4355|335|1983|2|6|14|2|1983|335|4355|Tuesday|1983Q2|N|N|N|2445487|2445637|2445135|2445410|N|N|N|N|N| +2445501|AAAAAAAANLAFFCAA|1983-06-15|1001|4355|335|1983|3|6|15|2|1983|335|4355|Wednesday|1983Q2|N|N|N|2445487|2445637|2445136|2445411|N|N|N|N|N| +2445502|AAAAAAAAOLAFFCAA|1983-06-16|1001|4355|335|1983|4|6|16|2|1983|335|4355|Thursday|1983Q2|N|N|N|2445487|2445637|2445137|2445412|N|N|N|N|N| +2445503|AAAAAAAAPLAFFCAA|1983-06-17|1001|4355|335|1983|5|6|17|2|1983|335|4355|Friday|1983Q2|N|Y|N|2445487|2445637|2445138|2445413|N|N|N|N|N| +2445504|AAAAAAAAAMAFFCAA|1983-06-18|1001|4355|335|1983|6|6|18|2|1983|335|4355|Saturday|1983Q2|N|Y|N|2445487|2445637|2445139|2445414|N|N|N|N|N| +2445505|AAAAAAAABMAFFCAA|1983-06-19|1001|4355|335|1983|0|6|19|2|1983|335|4355|Sunday|1983Q2|N|N|N|2445487|2445637|2445140|2445415|N|N|N|N|N| +2445506|AAAAAAAACMAFFCAA|1983-06-20|1001|4355|335|1983|1|6|20|2|1983|335|4355|Monday|1983Q2|N|N|N|2445487|2445637|2445141|2445416|N|N|N|N|N| +2445507|AAAAAAAADMAFFCAA|1983-06-21|1001|4356|335|1983|2|6|21|2|1983|335|4356|Tuesday|1983Q2|N|N|N|2445487|2445637|2445142|2445417|N|N|N|N|N| +2445508|AAAAAAAAEMAFFCAA|1983-06-22|1001|4356|335|1983|3|6|22|2|1983|335|4356|Wednesday|1983Q2|N|N|N|2445487|2445637|2445143|2445418|N|N|N|N|N| +2445509|AAAAAAAAFMAFFCAA|1983-06-23|1001|4356|335|1983|4|6|23|2|1983|335|4356|Thursday|1983Q2|N|N|N|2445487|2445637|2445144|2445419|N|N|N|N|N| +2445510|AAAAAAAAGMAFFCAA|1983-06-24|1001|4356|335|1983|5|6|24|2|1983|335|4356|Friday|1983Q2|N|Y|N|2445487|2445637|2445145|2445420|N|N|N|N|N| +2445511|AAAAAAAAHMAFFCAA|1983-06-25|1001|4356|335|1983|6|6|25|2|1983|335|4356|Saturday|1983Q2|N|Y|N|2445487|2445637|2445146|2445421|N|N|N|N|N| +2445512|AAAAAAAAIMAFFCAA|1983-06-26|1001|4356|335|1983|0|6|26|2|1983|335|4356|Sunday|1983Q2|N|N|N|2445487|2445637|2445147|2445422|N|N|N|N|N| +2445513|AAAAAAAAJMAFFCAA|1983-06-27|1001|4356|335|1983|1|6|27|2|1983|335|4356|Monday|1983Q2|N|N|N|2445487|2445637|2445148|2445423|N|N|N|N|N| +2445514|AAAAAAAAKMAFFCAA|1983-06-28|1001|4357|335|1983|2|6|28|2|1983|335|4357|Tuesday|1983Q2|N|N|N|2445487|2445637|2445149|2445424|N|N|N|N|N| +2445515|AAAAAAAALMAFFCAA|1983-06-29|1001|4357|335|1983|3|6|29|2|1983|335|4357|Wednesday|1983Q2|N|N|N|2445487|2445637|2445150|2445425|N|N|N|N|N| +2445516|AAAAAAAAMMAFFCAA|1983-06-30|1001|4357|335|1983|4|6|30|2|1983|335|4357|Thursday|1983Q2|N|N|N|2445487|2445637|2445151|2445426|N|N|N|N|N| +2445517|AAAAAAAANMAFFCAA|1983-07-01|1002|4357|335|1983|5|7|1|2|1983|335|4357|Friday|1983Q2|N|Y|N|2445517|2445697|2445152|2445426|N|N|N|N|N| +2445518|AAAAAAAAOMAFFCAA|1983-07-02|1002|4357|335|1983|6|7|2|3|1983|335|4357|Saturday|1983Q3|N|Y|N|2445517|2445697|2445153|2445427|N|N|N|N|N| +2445519|AAAAAAAAPMAFFCAA|1983-07-03|1002|4357|335|1983|0|7|3|3|1983|335|4357|Sunday|1983Q3|N|N|N|2445517|2445697|2445154|2445428|N|N|N|N|N| +2445520|AAAAAAAAANAFFCAA|1983-07-04|1002|4357|335|1983|1|7|4|3|1983|335|4357|Monday|1983Q3|N|N|N|2445517|2445697|2445155|2445429|N|N|N|N|N| +2445521|AAAAAAAABNAFFCAA|1983-07-05|1002|4358|335|1983|2|7|5|3|1983|335|4358|Tuesday|1983Q3|Y|N|N|2445517|2445697|2445156|2445430|N|N|N|N|N| +2445522|AAAAAAAACNAFFCAA|1983-07-06|1002|4358|335|1983|3|7|6|3|1983|335|4358|Wednesday|1983Q3|N|N|Y|2445517|2445697|2445157|2445431|N|N|N|N|N| +2445523|AAAAAAAADNAFFCAA|1983-07-07|1002|4358|335|1983|4|7|7|3|1983|335|4358|Thursday|1983Q3|N|N|N|2445517|2445697|2445158|2445432|N|N|N|N|N| +2445524|AAAAAAAAENAFFCAA|1983-07-08|1002|4358|335|1983|5|7|8|3|1983|335|4358|Friday|1983Q3|N|Y|N|2445517|2445697|2445159|2445433|N|N|N|N|N| +2445525|AAAAAAAAFNAFFCAA|1983-07-09|1002|4358|335|1983|6|7|9|3|1983|335|4358|Saturday|1983Q3|N|Y|N|2445517|2445697|2445160|2445434|N|N|N|N|N| +2445526|AAAAAAAAGNAFFCAA|1983-07-10|1002|4358|335|1983|0|7|10|3|1983|335|4358|Sunday|1983Q3|N|N|N|2445517|2445697|2445161|2445435|N|N|N|N|N| +2445527|AAAAAAAAHNAFFCAA|1983-07-11|1002|4358|335|1983|1|7|11|3|1983|335|4358|Monday|1983Q3|N|N|N|2445517|2445697|2445162|2445436|N|N|N|N|N| +2445528|AAAAAAAAINAFFCAA|1983-07-12|1002|4359|335|1983|2|7|12|3|1983|335|4359|Tuesday|1983Q3|N|N|N|2445517|2445697|2445163|2445437|N|N|N|N|N| +2445529|AAAAAAAAJNAFFCAA|1983-07-13|1002|4359|335|1983|3|7|13|3|1983|335|4359|Wednesday|1983Q3|N|N|N|2445517|2445697|2445164|2445438|N|N|N|N|N| +2445530|AAAAAAAAKNAFFCAA|1983-07-14|1002|4359|335|1983|4|7|14|3|1983|335|4359|Thursday|1983Q3|N|N|N|2445517|2445697|2445165|2445439|N|N|N|N|N| +2445531|AAAAAAAALNAFFCAA|1983-07-15|1002|4359|335|1983|5|7|15|3|1983|335|4359|Friday|1983Q3|N|Y|N|2445517|2445697|2445166|2445440|N|N|N|N|N| +2445532|AAAAAAAAMNAFFCAA|1983-07-16|1002|4359|335|1983|6|7|16|3|1983|335|4359|Saturday|1983Q3|N|Y|N|2445517|2445697|2445167|2445441|N|N|N|N|N| +2445533|AAAAAAAANNAFFCAA|1983-07-17|1002|4359|335|1983|0|7|17|3|1983|335|4359|Sunday|1983Q3|N|N|N|2445517|2445697|2445168|2445442|N|N|N|N|N| +2445534|AAAAAAAAONAFFCAA|1983-07-18|1002|4359|335|1983|1|7|18|3|1983|335|4359|Monday|1983Q3|N|N|N|2445517|2445697|2445169|2445443|N|N|N|N|N| +2445535|AAAAAAAAPNAFFCAA|1983-07-19|1002|4360|335|1983|2|7|19|3|1983|335|4360|Tuesday|1983Q3|N|N|N|2445517|2445697|2445170|2445444|N|N|N|N|N| +2445536|AAAAAAAAAOAFFCAA|1983-07-20|1002|4360|335|1983|3|7|20|3|1983|335|4360|Wednesday|1983Q3|N|N|N|2445517|2445697|2445171|2445445|N|N|N|N|N| +2445537|AAAAAAAABOAFFCAA|1983-07-21|1002|4360|335|1983|4|7|21|3|1983|335|4360|Thursday|1983Q3|N|N|N|2445517|2445697|2445172|2445446|N|N|N|N|N| +2445538|AAAAAAAACOAFFCAA|1983-07-22|1002|4360|335|1983|5|7|22|3|1983|335|4360|Friday|1983Q3|N|Y|N|2445517|2445697|2445173|2445447|N|N|N|N|N| +2445539|AAAAAAAADOAFFCAA|1983-07-23|1002|4360|335|1983|6|7|23|3|1983|335|4360|Saturday|1983Q3|N|Y|N|2445517|2445697|2445174|2445448|N|N|N|N|N| +2445540|AAAAAAAAEOAFFCAA|1983-07-24|1002|4360|335|1983|0|7|24|3|1983|335|4360|Sunday|1983Q3|N|N|N|2445517|2445697|2445175|2445449|N|N|N|N|N| +2445541|AAAAAAAAFOAFFCAA|1983-07-25|1002|4360|335|1983|1|7|25|3|1983|335|4360|Monday|1983Q3|N|N|N|2445517|2445697|2445176|2445450|N|N|N|N|N| +2445542|AAAAAAAAGOAFFCAA|1983-07-26|1002|4361|335|1983|2|7|26|3|1983|335|4361|Tuesday|1983Q3|N|N|N|2445517|2445697|2445177|2445451|N|N|N|N|N| +2445543|AAAAAAAAHOAFFCAA|1983-07-27|1002|4361|335|1983|3|7|27|3|1983|335|4361|Wednesday|1983Q3|N|N|N|2445517|2445697|2445178|2445452|N|N|N|N|N| +2445544|AAAAAAAAIOAFFCAA|1983-07-28|1002|4361|335|1983|4|7|28|3|1983|335|4361|Thursday|1983Q3|N|N|N|2445517|2445697|2445179|2445453|N|N|N|N|N| +2445545|AAAAAAAAJOAFFCAA|1983-07-29|1002|4361|335|1983|5|7|29|3|1983|335|4361|Friday|1983Q3|N|Y|N|2445517|2445697|2445180|2445454|N|N|N|N|N| +2445546|AAAAAAAAKOAFFCAA|1983-07-30|1002|4361|335|1983|6|7|30|3|1983|335|4361|Saturday|1983Q3|N|Y|N|2445517|2445697|2445181|2445455|N|N|N|N|N| +2445547|AAAAAAAALOAFFCAA|1983-07-31|1002|4361|335|1983|0|7|31|3|1983|335|4361|Sunday|1983Q3|N|N|N|2445517|2445697|2445182|2445456|N|N|N|N|N| +2445548|AAAAAAAAMOAFFCAA|1983-08-01|1003|4361|335|1983|1|8|1|3|1983|335|4361|Monday|1983Q3|N|N|N|2445548|2445759|2445183|2445457|N|N|N|N|N| +2445549|AAAAAAAANOAFFCAA|1983-08-02|1003|4362|335|1983|2|8|2|3|1983|335|4362|Tuesday|1983Q3|N|N|N|2445548|2445759|2445184|2445458|N|N|N|N|N| +2445550|AAAAAAAAOOAFFCAA|1983-08-03|1003|4362|335|1983|3|8|3|3|1983|335|4362|Wednesday|1983Q3|N|N|N|2445548|2445759|2445185|2445459|N|N|N|N|N| +2445551|AAAAAAAAPOAFFCAA|1983-08-04|1003|4362|335|1983|4|8|4|3|1983|335|4362|Thursday|1983Q3|N|N|N|2445548|2445759|2445186|2445460|N|N|N|N|N| +2445552|AAAAAAAAAPAFFCAA|1983-08-05|1003|4362|335|1983|5|8|5|3|1983|335|4362|Friday|1983Q3|N|Y|N|2445548|2445759|2445187|2445461|N|N|N|N|N| +2445553|AAAAAAAABPAFFCAA|1983-08-06|1003|4362|335|1983|6|8|6|3|1983|335|4362|Saturday|1983Q3|N|Y|N|2445548|2445759|2445188|2445462|N|N|N|N|N| +2445554|AAAAAAAACPAFFCAA|1983-08-07|1003|4362|335|1983|0|8|7|3|1983|335|4362|Sunday|1983Q3|N|N|N|2445548|2445759|2445189|2445463|N|N|N|N|N| +2445555|AAAAAAAADPAFFCAA|1983-08-08|1003|4362|335|1983|1|8|8|3|1983|335|4362|Monday|1983Q3|N|N|N|2445548|2445759|2445190|2445464|N|N|N|N|N| +2445556|AAAAAAAAEPAFFCAA|1983-08-09|1003|4363|335|1983|2|8|9|3|1983|335|4363|Tuesday|1983Q3|N|N|N|2445548|2445759|2445191|2445465|N|N|N|N|N| +2445557|AAAAAAAAFPAFFCAA|1983-08-10|1003|4363|335|1983|3|8|10|3|1983|335|4363|Wednesday|1983Q3|N|N|N|2445548|2445759|2445192|2445466|N|N|N|N|N| +2445558|AAAAAAAAGPAFFCAA|1983-08-11|1003|4363|335|1983|4|8|11|3|1983|335|4363|Thursday|1983Q3|N|N|N|2445548|2445759|2445193|2445467|N|N|N|N|N| +2445559|AAAAAAAAHPAFFCAA|1983-08-12|1003|4363|335|1983|5|8|12|3|1983|335|4363|Friday|1983Q3|N|Y|N|2445548|2445759|2445194|2445468|N|N|N|N|N| +2445560|AAAAAAAAIPAFFCAA|1983-08-13|1003|4363|335|1983|6|8|13|3|1983|335|4363|Saturday|1983Q3|N|Y|N|2445548|2445759|2445195|2445469|N|N|N|N|N| +2445561|AAAAAAAAJPAFFCAA|1983-08-14|1003|4363|335|1983|0|8|14|3|1983|335|4363|Sunday|1983Q3|N|N|N|2445548|2445759|2445196|2445470|N|N|N|N|N| +2445562|AAAAAAAAKPAFFCAA|1983-08-15|1003|4363|335|1983|1|8|15|3|1983|335|4363|Monday|1983Q3|N|N|N|2445548|2445759|2445197|2445471|N|N|N|N|N| +2445563|AAAAAAAALPAFFCAA|1983-08-16|1003|4364|335|1983|2|8|16|3|1983|335|4364|Tuesday|1983Q3|N|N|N|2445548|2445759|2445198|2445472|N|N|N|N|N| +2445564|AAAAAAAAMPAFFCAA|1983-08-17|1003|4364|335|1983|3|8|17|3|1983|335|4364|Wednesday|1983Q3|N|N|N|2445548|2445759|2445199|2445473|N|N|N|N|N| +2445565|AAAAAAAANPAFFCAA|1983-08-18|1003|4364|335|1983|4|8|18|3|1983|335|4364|Thursday|1983Q3|N|N|N|2445548|2445759|2445200|2445474|N|N|N|N|N| +2445566|AAAAAAAAOPAFFCAA|1983-08-19|1003|4364|335|1983|5|8|19|3|1983|335|4364|Friday|1983Q3|N|Y|N|2445548|2445759|2445201|2445475|N|N|N|N|N| +2445567|AAAAAAAAPPAFFCAA|1983-08-20|1003|4364|335|1983|6|8|20|3|1983|335|4364|Saturday|1983Q3|N|Y|N|2445548|2445759|2445202|2445476|N|N|N|N|N| +2445568|AAAAAAAAAABFFCAA|1983-08-21|1003|4364|335|1983|0|8|21|3|1983|335|4364|Sunday|1983Q3|N|N|N|2445548|2445759|2445203|2445477|N|N|N|N|N| +2445569|AAAAAAAABABFFCAA|1983-08-22|1003|4364|335|1983|1|8|22|3|1983|335|4364|Monday|1983Q3|N|N|N|2445548|2445759|2445204|2445478|N|N|N|N|N| +2445570|AAAAAAAACABFFCAA|1983-08-23|1003|4365|335|1983|2|8|23|3|1983|335|4365|Tuesday|1983Q3|N|N|N|2445548|2445759|2445205|2445479|N|N|N|N|N| +2445571|AAAAAAAADABFFCAA|1983-08-24|1003|4365|335|1983|3|8|24|3|1983|335|4365|Wednesday|1983Q3|N|N|N|2445548|2445759|2445206|2445480|N|N|N|N|N| +2445572|AAAAAAAAEABFFCAA|1983-08-25|1003|4365|335|1983|4|8|25|3|1983|335|4365|Thursday|1983Q3|N|N|N|2445548|2445759|2445207|2445481|N|N|N|N|N| +2445573|AAAAAAAAFABFFCAA|1983-08-26|1003|4365|335|1983|5|8|26|3|1983|335|4365|Friday|1983Q3|N|Y|N|2445548|2445759|2445208|2445482|N|N|N|N|N| +2445574|AAAAAAAAGABFFCAA|1983-08-27|1003|4365|335|1983|6|8|27|3|1983|335|4365|Saturday|1983Q3|N|Y|N|2445548|2445759|2445209|2445483|N|N|N|N|N| +2445575|AAAAAAAAHABFFCAA|1983-08-28|1003|4365|335|1983|0|8|28|3|1983|335|4365|Sunday|1983Q3|N|N|N|2445548|2445759|2445210|2445484|N|N|N|N|N| +2445576|AAAAAAAAIABFFCAA|1983-08-29|1003|4365|335|1983|1|8|29|3|1983|335|4365|Monday|1983Q3|N|N|N|2445548|2445759|2445211|2445485|N|N|N|N|N| +2445577|AAAAAAAAJABFFCAA|1983-08-30|1003|4366|335|1983|2|8|30|3|1983|335|4366|Tuesday|1983Q3|N|N|N|2445548|2445759|2445212|2445486|N|N|N|N|N| +2445578|AAAAAAAAKABFFCAA|1983-08-31|1003|4366|335|1983|3|8|31|3|1983|335|4366|Wednesday|1983Q3|N|N|N|2445548|2445759|2445213|2445487|N|N|N|N|N| +2445579|AAAAAAAALABFFCAA|1983-09-01|1004|4366|336|1983|4|9|1|3|1983|336|4366|Thursday|1983Q3|N|N|N|2445579|2445821|2445214|2445488|N|N|N|N|N| +2445580|AAAAAAAAMABFFCAA|1983-09-02|1004|4366|336|1983|5|9|2|3|1983|336|4366|Friday|1983Q3|N|Y|N|2445579|2445821|2445215|2445489|N|N|N|N|N| +2445581|AAAAAAAANABFFCAA|1983-09-03|1004|4366|336|1983|6|9|3|3|1983|336|4366|Saturday|1983Q3|N|Y|N|2445579|2445821|2445216|2445490|N|N|N|N|N| +2445582|AAAAAAAAOABFFCAA|1983-09-04|1004|4366|336|1983|0|9|4|3|1983|336|4366|Sunday|1983Q3|N|N|N|2445579|2445821|2445217|2445491|N|N|N|N|N| +2445583|AAAAAAAAPABFFCAA|1983-09-05|1004|4366|336|1983|1|9|5|3|1983|336|4366|Monday|1983Q3|N|N|N|2445579|2445821|2445218|2445492|N|N|N|N|N| +2445584|AAAAAAAAABBFFCAA|1983-09-06|1004|4367|336|1983|2|9|6|3|1983|336|4367|Tuesday|1983Q3|N|N|N|2445579|2445821|2445219|2445493|N|N|N|N|N| +2445585|AAAAAAAABBBFFCAA|1983-09-07|1004|4367|336|1983|3|9|7|3|1983|336|4367|Wednesday|1983Q3|N|N|N|2445579|2445821|2445220|2445494|N|N|N|N|N| +2445586|AAAAAAAACBBFFCAA|1983-09-08|1004|4367|336|1983|4|9|8|3|1983|336|4367|Thursday|1983Q3|N|N|N|2445579|2445821|2445221|2445495|N|N|N|N|N| +2445587|AAAAAAAADBBFFCAA|1983-09-09|1004|4367|336|1983|5|9|9|3|1983|336|4367|Friday|1983Q3|N|Y|N|2445579|2445821|2445222|2445496|N|N|N|N|N| +2445588|AAAAAAAAEBBFFCAA|1983-09-10|1004|4367|336|1983|6|9|10|3|1983|336|4367|Saturday|1983Q3|N|Y|N|2445579|2445821|2445223|2445497|N|N|N|N|N| +2445589|AAAAAAAAFBBFFCAA|1983-09-11|1004|4367|336|1983|0|9|11|3|1983|336|4367|Sunday|1983Q3|N|N|N|2445579|2445821|2445224|2445498|N|N|N|N|N| +2445590|AAAAAAAAGBBFFCAA|1983-09-12|1004|4367|336|1983|1|9|12|3|1983|336|4367|Monday|1983Q3|N|N|N|2445579|2445821|2445225|2445499|N|N|N|N|N| +2445591|AAAAAAAAHBBFFCAA|1983-09-13|1004|4368|336|1983|2|9|13|3|1983|336|4368|Tuesday|1983Q3|N|N|N|2445579|2445821|2445226|2445500|N|N|N|N|N| +2445592|AAAAAAAAIBBFFCAA|1983-09-14|1004|4368|336|1983|3|9|14|3|1983|336|4368|Wednesday|1983Q3|N|N|N|2445579|2445821|2445227|2445501|N|N|N|N|N| +2445593|AAAAAAAAJBBFFCAA|1983-09-15|1004|4368|336|1983|4|9|15|3|1983|336|4368|Thursday|1983Q3|N|N|N|2445579|2445821|2445228|2445502|N|N|N|N|N| +2445594|AAAAAAAAKBBFFCAA|1983-09-16|1004|4368|336|1983|5|9|16|3|1983|336|4368|Friday|1983Q3|N|Y|N|2445579|2445821|2445229|2445503|N|N|N|N|N| +2445595|AAAAAAAALBBFFCAA|1983-09-17|1004|4368|336|1983|6|9|17|3|1983|336|4368|Saturday|1983Q3|N|Y|N|2445579|2445821|2445230|2445504|N|N|N|N|N| +2445596|AAAAAAAAMBBFFCAA|1983-09-18|1004|4368|336|1983|0|9|18|3|1983|336|4368|Sunday|1983Q3|N|N|N|2445579|2445821|2445231|2445505|N|N|N|N|N| +2445597|AAAAAAAANBBFFCAA|1983-09-19|1004|4368|336|1983|1|9|19|3|1983|336|4368|Monday|1983Q3|N|N|N|2445579|2445821|2445232|2445506|N|N|N|N|N| +2445598|AAAAAAAAOBBFFCAA|1983-09-20|1004|4369|336|1983|2|9|20|3|1983|336|4369|Tuesday|1983Q3|N|N|N|2445579|2445821|2445233|2445507|N|N|N|N|N| +2445599|AAAAAAAAPBBFFCAA|1983-09-21|1004|4369|336|1983|3|9|21|3|1983|336|4369|Wednesday|1983Q3|N|N|N|2445579|2445821|2445234|2445508|N|N|N|N|N| +2445600|AAAAAAAAACBFFCAA|1983-09-22|1004|4369|336|1983|4|9|22|3|1983|336|4369|Thursday|1983Q3|N|N|N|2445579|2445821|2445235|2445509|N|N|N|N|N| +2445601|AAAAAAAABCBFFCAA|1983-09-23|1004|4369|336|1983|5|9|23|3|1983|336|4369|Friday|1983Q3|N|Y|N|2445579|2445821|2445236|2445510|N|N|N|N|N| +2445602|AAAAAAAACCBFFCAA|1983-09-24|1004|4369|336|1983|6|9|24|3|1983|336|4369|Saturday|1983Q3|N|Y|N|2445579|2445821|2445237|2445511|N|N|N|N|N| +2445603|AAAAAAAADCBFFCAA|1983-09-25|1004|4369|336|1983|0|9|25|3|1983|336|4369|Sunday|1983Q3|N|N|N|2445579|2445821|2445238|2445512|N|N|N|N|N| +2445604|AAAAAAAAECBFFCAA|1983-09-26|1004|4369|336|1983|1|9|26|3|1983|336|4369|Monday|1983Q3|N|N|N|2445579|2445821|2445239|2445513|N|N|N|N|N| +2445605|AAAAAAAAFCBFFCAA|1983-09-27|1004|4370|336|1983|2|9|27|3|1983|336|4370|Tuesday|1983Q3|N|N|N|2445579|2445821|2445240|2445514|N|N|N|N|N| +2445606|AAAAAAAAGCBFFCAA|1983-09-28|1004|4370|336|1983|3|9|28|3|1983|336|4370|Wednesday|1983Q3|N|N|N|2445579|2445821|2445241|2445515|N|N|N|N|N| +2445607|AAAAAAAAHCBFFCAA|1983-09-29|1004|4370|336|1983|4|9|29|3|1983|336|4370|Thursday|1983Q3|N|N|N|2445579|2445821|2445242|2445516|N|N|N|N|N| +2445608|AAAAAAAAICBFFCAA|1983-09-30|1004|4370|336|1983|5|9|30|3|1983|336|4370|Friday|1983Q3|N|Y|N|2445579|2445821|2445243|2445517|N|N|N|N|N| +2445609|AAAAAAAAJCBFFCAA|1983-10-01|1005|4370|336|1983|6|10|1|3|1983|336|4370|Saturday|1983Q3|N|Y|N|2445609|2445881|2445244|2445517|N|N|N|N|N| +2445610|AAAAAAAAKCBFFCAA|1983-10-02|1005|4370|336|1983|0|10|2|4|1983|336|4370|Sunday|1983Q4|N|N|N|2445609|2445881|2445245|2445518|N|N|N|N|N| +2445611|AAAAAAAALCBFFCAA|1983-10-03|1005|4370|336|1983|1|10|3|4|1983|336|4370|Monday|1983Q4|N|N|N|2445609|2445881|2445246|2445519|N|N|N|N|N| +2445612|AAAAAAAAMCBFFCAA|1983-10-04|1005|4371|336|1983|2|10|4|4|1983|336|4371|Tuesday|1983Q4|N|N|N|2445609|2445881|2445247|2445520|N|N|N|N|N| +2445613|AAAAAAAANCBFFCAA|1983-10-05|1005|4371|336|1983|3|10|5|4|1983|336|4371|Wednesday|1983Q4|N|N|N|2445609|2445881|2445248|2445521|N|N|N|N|N| +2445614|AAAAAAAAOCBFFCAA|1983-10-06|1005|4371|336|1983|4|10|6|4|1983|336|4371|Thursday|1983Q4|N|N|N|2445609|2445881|2445249|2445522|N|N|N|N|N| +2445615|AAAAAAAAPCBFFCAA|1983-10-07|1005|4371|336|1983|5|10|7|4|1983|336|4371|Friday|1983Q4|N|Y|N|2445609|2445881|2445250|2445523|N|N|N|N|N| +2445616|AAAAAAAAADBFFCAA|1983-10-08|1005|4371|336|1983|6|10|8|4|1983|336|4371|Saturday|1983Q4|N|Y|N|2445609|2445881|2445251|2445524|N|N|N|N|N| +2445617|AAAAAAAABDBFFCAA|1983-10-09|1005|4371|336|1983|0|10|9|4|1983|336|4371|Sunday|1983Q4|N|N|N|2445609|2445881|2445252|2445525|N|N|N|N|N| +2445618|AAAAAAAACDBFFCAA|1983-10-10|1005|4371|336|1983|1|10|10|4|1983|336|4371|Monday|1983Q4|N|N|N|2445609|2445881|2445253|2445526|N|N|N|N|N| +2445619|AAAAAAAADDBFFCAA|1983-10-11|1005|4372|336|1983|2|10|11|4|1983|336|4372|Tuesday|1983Q4|N|N|N|2445609|2445881|2445254|2445527|N|N|N|N|N| +2445620|AAAAAAAAEDBFFCAA|1983-10-12|1005|4372|336|1983|3|10|12|4|1983|336|4372|Wednesday|1983Q4|N|N|N|2445609|2445881|2445255|2445528|N|N|N|N|N| +2445621|AAAAAAAAFDBFFCAA|1983-10-13|1005|4372|336|1983|4|10|13|4|1983|336|4372|Thursday|1983Q4|N|N|N|2445609|2445881|2445256|2445529|N|N|N|N|N| +2445622|AAAAAAAAGDBFFCAA|1983-10-14|1005|4372|336|1983|5|10|14|4|1983|336|4372|Friday|1983Q4|N|Y|N|2445609|2445881|2445257|2445530|N|N|N|N|N| +2445623|AAAAAAAAHDBFFCAA|1983-10-15|1005|4372|336|1983|6|10|15|4|1983|336|4372|Saturday|1983Q4|N|Y|N|2445609|2445881|2445258|2445531|N|N|N|N|N| +2445624|AAAAAAAAIDBFFCAA|1983-10-16|1005|4372|336|1983|0|10|16|4|1983|336|4372|Sunday|1983Q4|N|N|N|2445609|2445881|2445259|2445532|N|N|N|N|N| +2445625|AAAAAAAAJDBFFCAA|1983-10-17|1005|4372|336|1983|1|10|17|4|1983|336|4372|Monday|1983Q4|N|N|N|2445609|2445881|2445260|2445533|N|N|N|N|N| +2445626|AAAAAAAAKDBFFCAA|1983-10-18|1005|4373|336|1983|2|10|18|4|1983|336|4373|Tuesday|1983Q4|N|N|N|2445609|2445881|2445261|2445534|N|N|N|N|N| +2445627|AAAAAAAALDBFFCAA|1983-10-19|1005|4373|336|1983|3|10|19|4|1983|336|4373|Wednesday|1983Q4|N|N|N|2445609|2445881|2445262|2445535|N|N|N|N|N| +2445628|AAAAAAAAMDBFFCAA|1983-10-20|1005|4373|336|1983|4|10|20|4|1983|336|4373|Thursday|1983Q4|N|N|N|2445609|2445881|2445263|2445536|N|N|N|N|N| +2445629|AAAAAAAANDBFFCAA|1983-10-21|1005|4373|336|1983|5|10|21|4|1983|336|4373|Friday|1983Q4|N|Y|N|2445609|2445881|2445264|2445537|N|N|N|N|N| +2445630|AAAAAAAAODBFFCAA|1983-10-22|1005|4373|336|1983|6|10|22|4|1983|336|4373|Saturday|1983Q4|N|Y|N|2445609|2445881|2445265|2445538|N|N|N|N|N| +2445631|AAAAAAAAPDBFFCAA|1983-10-23|1005|4373|336|1983|0|10|23|4|1983|336|4373|Sunday|1983Q4|N|N|N|2445609|2445881|2445266|2445539|N|N|N|N|N| +2445632|AAAAAAAAAEBFFCAA|1983-10-24|1005|4373|336|1983|1|10|24|4|1983|336|4373|Monday|1983Q4|N|N|N|2445609|2445881|2445267|2445540|N|N|N|N|N| +2445633|AAAAAAAABEBFFCAA|1983-10-25|1005|4374|336|1983|2|10|25|4|1983|336|4374|Tuesday|1983Q4|N|N|N|2445609|2445881|2445268|2445541|N|N|N|N|N| +2445634|AAAAAAAACEBFFCAA|1983-10-26|1005|4374|336|1983|3|10|26|4|1983|336|4374|Wednesday|1983Q4|N|N|N|2445609|2445881|2445269|2445542|N|N|N|N|N| +2445635|AAAAAAAADEBFFCAA|1983-10-27|1005|4374|336|1983|4|10|27|4|1983|336|4374|Thursday|1983Q4|N|N|N|2445609|2445881|2445270|2445543|N|N|N|N|N| +2445636|AAAAAAAAEEBFFCAA|1983-10-28|1005|4374|336|1983|5|10|28|4|1983|336|4374|Friday|1983Q4|N|Y|N|2445609|2445881|2445271|2445544|N|N|N|N|N| +2445637|AAAAAAAAFEBFFCAA|1983-10-29|1005|4374|336|1983|6|10|29|4|1983|336|4374|Saturday|1983Q4|N|Y|N|2445609|2445881|2445272|2445545|N|N|N|N|N| +2445638|AAAAAAAAGEBFFCAA|1983-10-30|1005|4374|336|1983|0|10|30|4|1983|336|4374|Sunday|1983Q4|N|N|N|2445609|2445881|2445273|2445546|N|N|N|N|N| +2445639|AAAAAAAAHEBFFCAA|1983-10-31|1005|4374|336|1983|1|10|31|4|1983|336|4374|Monday|1983Q4|N|N|N|2445609|2445881|2445274|2445547|N|N|N|N|N| +2445640|AAAAAAAAIEBFFCAA|1983-11-01|1006|4375|336|1983|2|11|1|4|1983|336|4375|Tuesday|1983Q4|N|N|N|2445640|2445943|2445275|2445548|N|N|N|N|N| +2445641|AAAAAAAAJEBFFCAA|1983-11-02|1006|4375|336|1983|3|11|2|4|1983|336|4375|Wednesday|1983Q4|N|N|N|2445640|2445943|2445276|2445549|N|N|N|N|N| +2445642|AAAAAAAAKEBFFCAA|1983-11-03|1006|4375|336|1983|4|11|3|4|1983|336|4375|Thursday|1983Q4|N|N|N|2445640|2445943|2445277|2445550|N|N|N|N|N| +2445643|AAAAAAAALEBFFCAA|1983-11-04|1006|4375|336|1983|5|11|4|4|1983|336|4375|Friday|1983Q4|N|Y|N|2445640|2445943|2445278|2445551|N|N|N|N|N| +2445644|AAAAAAAAMEBFFCAA|1983-11-05|1006|4375|336|1983|6|11|5|4|1983|336|4375|Saturday|1983Q4|N|Y|N|2445640|2445943|2445279|2445552|N|N|N|N|N| +2445645|AAAAAAAANEBFFCAA|1983-11-06|1006|4375|336|1983|0|11|6|4|1983|336|4375|Sunday|1983Q4|N|N|N|2445640|2445943|2445280|2445553|N|N|N|N|N| +2445646|AAAAAAAAOEBFFCAA|1983-11-07|1006|4375|336|1983|1|11|7|4|1983|336|4375|Monday|1983Q4|N|N|N|2445640|2445943|2445281|2445554|N|N|N|N|N| +2445647|AAAAAAAAPEBFFCAA|1983-11-08|1006|4376|336|1983|2|11|8|4|1983|336|4376|Tuesday|1983Q4|N|N|N|2445640|2445943|2445282|2445555|N|N|N|N|N| +2445648|AAAAAAAAAFBFFCAA|1983-11-09|1006|4376|336|1983|3|11|9|4|1983|336|4376|Wednesday|1983Q4|N|N|N|2445640|2445943|2445283|2445556|N|N|N|N|N| +2445649|AAAAAAAABFBFFCAA|1983-11-10|1006|4376|336|1983|4|11|10|4|1983|336|4376|Thursday|1983Q4|N|N|N|2445640|2445943|2445284|2445557|N|N|N|N|N| +2445650|AAAAAAAACFBFFCAA|1983-11-11|1006|4376|336|1983|5|11|11|4|1983|336|4376|Friday|1983Q4|N|Y|N|2445640|2445943|2445285|2445558|N|N|N|N|N| +2445651|AAAAAAAADFBFFCAA|1983-11-12|1006|4376|336|1983|6|11|12|4|1983|336|4376|Saturday|1983Q4|N|Y|N|2445640|2445943|2445286|2445559|N|N|N|N|N| +2445652|AAAAAAAAEFBFFCAA|1983-11-13|1006|4376|336|1983|0|11|13|4|1983|336|4376|Sunday|1983Q4|N|N|N|2445640|2445943|2445287|2445560|N|N|N|N|N| +2445653|AAAAAAAAFFBFFCAA|1983-11-14|1006|4376|336|1983|1|11|14|4|1983|336|4376|Monday|1983Q4|N|N|N|2445640|2445943|2445288|2445561|N|N|N|N|N| +2445654|AAAAAAAAGFBFFCAA|1983-11-15|1006|4377|336|1983|2|11|15|4|1983|336|4377|Tuesday|1983Q4|N|N|N|2445640|2445943|2445289|2445562|N|N|N|N|N| +2445655|AAAAAAAAHFBFFCAA|1983-11-16|1006|4377|336|1983|3|11|16|4|1983|336|4377|Wednesday|1983Q4|N|N|N|2445640|2445943|2445290|2445563|N|N|N|N|N| +2445656|AAAAAAAAIFBFFCAA|1983-11-17|1006|4377|336|1983|4|11|17|4|1983|336|4377|Thursday|1983Q4|N|N|N|2445640|2445943|2445291|2445564|N|N|N|N|N| +2445657|AAAAAAAAJFBFFCAA|1983-11-18|1006|4377|336|1983|5|11|18|4|1983|336|4377|Friday|1983Q4|N|Y|N|2445640|2445943|2445292|2445565|N|N|N|N|N| +2445658|AAAAAAAAKFBFFCAA|1983-11-19|1006|4377|336|1983|6|11|19|4|1983|336|4377|Saturday|1983Q4|N|Y|N|2445640|2445943|2445293|2445566|N|N|N|N|N| +2445659|AAAAAAAALFBFFCAA|1983-11-20|1006|4377|336|1983|0|11|20|4|1983|336|4377|Sunday|1983Q4|N|N|N|2445640|2445943|2445294|2445567|N|N|N|N|N| +2445660|AAAAAAAAMFBFFCAA|1983-11-21|1006|4377|336|1983|1|11|21|4|1983|336|4377|Monday|1983Q4|N|N|N|2445640|2445943|2445295|2445568|N|N|N|N|N| +2445661|AAAAAAAANFBFFCAA|1983-11-22|1006|4378|336|1983|2|11|22|4|1983|336|4378|Tuesday|1983Q4|N|N|N|2445640|2445943|2445296|2445569|N|N|N|N|N| +2445662|AAAAAAAAOFBFFCAA|1983-11-23|1006|4378|336|1983|3|11|23|4|1983|336|4378|Wednesday|1983Q4|N|N|N|2445640|2445943|2445297|2445570|N|N|N|N|N| +2445663|AAAAAAAAPFBFFCAA|1983-11-24|1006|4378|336|1983|4|11|24|4|1983|336|4378|Thursday|1983Q4|N|N|N|2445640|2445943|2445298|2445571|N|N|N|N|N| +2445664|AAAAAAAAAGBFFCAA|1983-11-25|1006|4378|336|1983|5|11|25|4|1983|336|4378|Friday|1983Q4|N|Y|N|2445640|2445943|2445299|2445572|N|N|N|N|N| +2445665|AAAAAAAABGBFFCAA|1983-11-26|1006|4378|336|1983|6|11|26|4|1983|336|4378|Saturday|1983Q4|N|Y|N|2445640|2445943|2445300|2445573|N|N|N|N|N| +2445666|AAAAAAAACGBFFCAA|1983-11-27|1006|4378|336|1983|0|11|27|4|1983|336|4378|Sunday|1983Q4|N|N|N|2445640|2445943|2445301|2445574|N|N|N|N|N| +2445667|AAAAAAAADGBFFCAA|1983-11-28|1006|4378|336|1983|1|11|28|4|1983|336|4378|Monday|1983Q4|N|N|N|2445640|2445943|2445302|2445575|N|N|N|N|N| +2445668|AAAAAAAAEGBFFCAA|1983-11-29|1006|4379|336|1983|2|11|29|4|1983|336|4379|Tuesday|1983Q4|N|N|N|2445640|2445943|2445303|2445576|N|N|N|N|N| +2445669|AAAAAAAAFGBFFCAA|1983-11-30|1006|4379|336|1983|3|11|30|4|1983|336|4379|Wednesday|1983Q4|N|N|N|2445640|2445943|2445304|2445577|N|N|N|N|N| +2445670|AAAAAAAAGGBFFCAA|1983-12-01|1007|4379|337|1983|4|12|1|4|1983|337|4379|Thursday|1983Q4|N|N|N|2445670|2446003|2445305|2445578|N|N|N|N|N| +2445671|AAAAAAAAHGBFFCAA|1983-12-02|1007|4379|337|1983|5|12|2|4|1983|337|4379|Friday|1983Q4|N|Y|N|2445670|2446003|2445306|2445579|N|N|N|N|N| +2445672|AAAAAAAAIGBFFCAA|1983-12-03|1007|4379|337|1983|6|12|3|4|1983|337|4379|Saturday|1983Q4|N|Y|N|2445670|2446003|2445307|2445580|N|N|N|N|N| +2445673|AAAAAAAAJGBFFCAA|1983-12-04|1007|4379|337|1983|0|12|4|4|1983|337|4379|Sunday|1983Q4|N|N|N|2445670|2446003|2445308|2445581|N|N|N|N|N| +2445674|AAAAAAAAKGBFFCAA|1983-12-05|1007|4379|337|1983|1|12|5|4|1983|337|4379|Monday|1983Q4|N|N|N|2445670|2446003|2445309|2445582|N|N|N|N|N| +2445675|AAAAAAAALGBFFCAA|1983-12-06|1007|4380|337|1983|2|12|6|4|1983|337|4380|Tuesday|1983Q4|N|N|N|2445670|2446003|2445310|2445583|N|N|N|N|N| +2445676|AAAAAAAAMGBFFCAA|1983-12-07|1007|4380|337|1983|3|12|7|4|1983|337|4380|Wednesday|1983Q4|N|N|N|2445670|2446003|2445311|2445584|N|N|N|N|N| +2445677|AAAAAAAANGBFFCAA|1983-12-08|1007|4380|337|1983|4|12|8|4|1983|337|4380|Thursday|1983Q4|N|N|N|2445670|2446003|2445312|2445585|N|N|N|N|N| +2445678|AAAAAAAAOGBFFCAA|1983-12-09|1007|4380|337|1983|5|12|9|4|1983|337|4380|Friday|1983Q4|N|Y|N|2445670|2446003|2445313|2445586|N|N|N|N|N| +2445679|AAAAAAAAPGBFFCAA|1983-12-10|1007|4380|337|1983|6|12|10|4|1983|337|4380|Saturday|1983Q4|N|Y|N|2445670|2446003|2445314|2445587|N|N|N|N|N| +2445680|AAAAAAAAAHBFFCAA|1983-12-11|1007|4380|337|1983|0|12|11|4|1983|337|4380|Sunday|1983Q4|N|N|N|2445670|2446003|2445315|2445588|N|N|N|N|N| +2445681|AAAAAAAABHBFFCAA|1983-12-12|1007|4380|337|1983|1|12|12|4|1983|337|4380|Monday|1983Q4|N|N|N|2445670|2446003|2445316|2445589|N|N|N|N|N| +2445682|AAAAAAAACHBFFCAA|1983-12-13|1007|4381|337|1983|2|12|13|4|1983|337|4381|Tuesday|1983Q4|N|N|N|2445670|2446003|2445317|2445590|N|N|N|N|N| +2445683|AAAAAAAADHBFFCAA|1983-12-14|1007|4381|337|1983|3|12|14|4|1983|337|4381|Wednesday|1983Q4|N|N|N|2445670|2446003|2445318|2445591|N|N|N|N|N| +2445684|AAAAAAAAEHBFFCAA|1983-12-15|1007|4381|337|1983|4|12|15|4|1983|337|4381|Thursday|1983Q4|N|N|N|2445670|2446003|2445319|2445592|N|N|N|N|N| +2445685|AAAAAAAAFHBFFCAA|1983-12-16|1007|4381|337|1983|5|12|16|4|1983|337|4381|Friday|1983Q4|N|Y|N|2445670|2446003|2445320|2445593|N|N|N|N|N| +2445686|AAAAAAAAGHBFFCAA|1983-12-17|1007|4381|337|1983|6|12|17|4|1983|337|4381|Saturday|1983Q4|N|Y|N|2445670|2446003|2445321|2445594|N|N|N|N|N| +2445687|AAAAAAAAHHBFFCAA|1983-12-18|1007|4381|337|1983|0|12|18|4|1983|337|4381|Sunday|1983Q4|N|N|N|2445670|2446003|2445322|2445595|N|N|N|N|N| +2445688|AAAAAAAAIHBFFCAA|1983-12-19|1007|4381|337|1983|1|12|19|4|1983|337|4381|Monday|1983Q4|N|N|N|2445670|2446003|2445323|2445596|N|N|N|N|N| +2445689|AAAAAAAAJHBFFCAA|1983-12-20|1007|4382|337|1983|2|12|20|4|1983|337|4382|Tuesday|1983Q4|N|N|N|2445670|2446003|2445324|2445597|N|N|N|N|N| +2445690|AAAAAAAAKHBFFCAA|1983-12-21|1007|4382|337|1983|3|12|21|4|1983|337|4382|Wednesday|1983Q4|N|N|N|2445670|2446003|2445325|2445598|N|N|N|N|N| +2445691|AAAAAAAALHBFFCAA|1983-12-22|1007|4382|337|1983|4|12|22|4|1983|337|4382|Thursday|1983Q4|N|N|N|2445670|2446003|2445326|2445599|N|N|N|N|N| +2445692|AAAAAAAAMHBFFCAA|1983-12-23|1007|4382|337|1983|5|12|23|4|1983|337|4382|Friday|1983Q4|N|Y|N|2445670|2446003|2445327|2445600|N|N|N|N|N| +2445693|AAAAAAAANHBFFCAA|1983-12-24|1007|4382|337|1983|6|12|24|4|1983|337|4382|Saturday|1983Q4|N|Y|N|2445670|2446003|2445328|2445601|N|N|N|N|N| +2445694|AAAAAAAAOHBFFCAA|1983-12-25|1007|4382|337|1983|0|12|25|4|1983|337|4382|Sunday|1983Q4|N|N|N|2445670|2446003|2445329|2445602|N|N|N|N|N| +2445695|AAAAAAAAPHBFFCAA|1983-12-26|1007|4382|337|1983|1|12|26|4|1983|337|4382|Monday|1983Q4|Y|N|N|2445670|2446003|2445330|2445603|N|N|N|N|N| +2445696|AAAAAAAAAIBFFCAA|1983-12-27|1007|4383|337|1983|2|12|27|4|1983|337|4383|Tuesday|1983Q4|N|N|Y|2445670|2446003|2445331|2445604|N|N|N|N|N| +2445697|AAAAAAAABIBFFCAA|1983-12-28|1007|4383|337|1983|3|12|28|4|1983|337|4383|Wednesday|1983Q4|N|N|N|2445670|2446003|2445332|2445605|N|N|N|N|N| +2445698|AAAAAAAACIBFFCAA|1983-12-29|1007|4383|337|1983|4|12|29|4|1983|337|4383|Thursday|1983Q4|N|N|N|2445670|2446003|2445333|2445606|N|N|N|N|N| +2445699|AAAAAAAADIBFFCAA|1983-12-30|1007|4383|337|1983|5|12|30|4|1983|337|4383|Friday|1983Q4|N|Y|N|2445670|2446003|2445334|2445607|N|N|N|N|N| +2445700|AAAAAAAAEIBFFCAA|1983-12-31|1007|4383|337|1983|6|12|31|4|1983|337|4383|Saturday|1983Q4|N|Y|N|2445670|2446003|2445335|2445608|N|N|N|N|N| +2445701|AAAAAAAAFIBFFCAA|1984-01-01|1008|4383|337|1984|0|1|1|1|1984|337|4383|Sunday|1984Q1|Y|N|N|2445701|2445700|2445336|2445609|N|N|N|N|N| +2445702|AAAAAAAAGIBFFCAA|1984-01-02|1008|4383|337|1984|1|1|2|1|1984|337|4383|Monday|1984Q1|N|N|Y|2445701|2445700|2445337|2445610|N|N|N|N|N| +2445703|AAAAAAAAHIBFFCAA|1984-01-03|1008|4384|337|1984|2|1|3|1|1984|337|4384|Tuesday|1984Q1|N|N|N|2445701|2445700|2445338|2445611|N|N|N|N|N| +2445704|AAAAAAAAIIBFFCAA|1984-01-04|1008|4384|337|1984|3|1|4|1|1984|337|4384|Wednesday|1984Q1|N|N|N|2445701|2445700|2445339|2445612|N|N|N|N|N| +2445705|AAAAAAAAJIBFFCAA|1984-01-05|1008|4384|337|1984|4|1|5|1|1984|337|4384|Thursday|1984Q1|N|N|N|2445701|2445700|2445340|2445613|N|N|N|N|N| +2445706|AAAAAAAAKIBFFCAA|1984-01-06|1008|4384|337|1984|5|1|6|1|1984|337|4384|Friday|1984Q1|N|Y|N|2445701|2445700|2445341|2445614|N|N|N|N|N| +2445707|AAAAAAAALIBFFCAA|1984-01-07|1008|4384|337|1984|6|1|7|1|1984|337|4384|Saturday|1984Q1|N|Y|N|2445701|2445700|2445342|2445615|N|N|N|N|N| +2445708|AAAAAAAAMIBFFCAA|1984-01-08|1008|4384|337|1984|0|1|8|1|1984|337|4384|Sunday|1984Q1|N|N|N|2445701|2445700|2445343|2445616|N|N|N|N|N| +2445709|AAAAAAAANIBFFCAA|1984-01-09|1008|4384|337|1984|1|1|9|1|1984|337|4384|Monday|1984Q1|N|N|N|2445701|2445700|2445344|2445617|N|N|N|N|N| +2445710|AAAAAAAAOIBFFCAA|1984-01-10|1008|4385|337|1984|2|1|10|1|1984|337|4385|Tuesday|1984Q1|N|N|N|2445701|2445700|2445345|2445618|N|N|N|N|N| +2445711|AAAAAAAAPIBFFCAA|1984-01-11|1008|4385|337|1984|3|1|11|1|1984|337|4385|Wednesday|1984Q1|N|N|N|2445701|2445700|2445346|2445619|N|N|N|N|N| +2445712|AAAAAAAAAJBFFCAA|1984-01-12|1008|4385|337|1984|4|1|12|1|1984|337|4385|Thursday|1984Q1|N|N|N|2445701|2445700|2445347|2445620|N|N|N|N|N| +2445713|AAAAAAAABJBFFCAA|1984-01-13|1008|4385|337|1984|5|1|13|1|1984|337|4385|Friday|1984Q1|N|Y|N|2445701|2445700|2445348|2445621|N|N|N|N|N| +2445714|AAAAAAAACJBFFCAA|1984-01-14|1008|4385|337|1984|6|1|14|1|1984|337|4385|Saturday|1984Q1|N|Y|N|2445701|2445700|2445349|2445622|N|N|N|N|N| +2445715|AAAAAAAADJBFFCAA|1984-01-15|1008|4385|337|1984|0|1|15|1|1984|337|4385|Sunday|1984Q1|N|N|N|2445701|2445700|2445350|2445623|N|N|N|N|N| +2445716|AAAAAAAAEJBFFCAA|1984-01-16|1008|4385|337|1984|1|1|16|1|1984|337|4385|Monday|1984Q1|N|N|N|2445701|2445700|2445351|2445624|N|N|N|N|N| +2445717|AAAAAAAAFJBFFCAA|1984-01-17|1008|4386|337|1984|2|1|17|1|1984|337|4386|Tuesday|1984Q1|N|N|N|2445701|2445700|2445352|2445625|N|N|N|N|N| +2445718|AAAAAAAAGJBFFCAA|1984-01-18|1008|4386|337|1984|3|1|18|1|1984|337|4386|Wednesday|1984Q1|N|N|N|2445701|2445700|2445353|2445626|N|N|N|N|N| +2445719|AAAAAAAAHJBFFCAA|1984-01-19|1008|4386|337|1984|4|1|19|1|1984|337|4386|Thursday|1984Q1|N|N|N|2445701|2445700|2445354|2445627|N|N|N|N|N| +2445720|AAAAAAAAIJBFFCAA|1984-01-20|1008|4386|337|1984|5|1|20|1|1984|337|4386|Friday|1984Q1|N|Y|N|2445701|2445700|2445355|2445628|N|N|N|N|N| +2445721|AAAAAAAAJJBFFCAA|1984-01-21|1008|4386|337|1984|6|1|21|1|1984|337|4386|Saturday|1984Q1|N|Y|N|2445701|2445700|2445356|2445629|N|N|N|N|N| +2445722|AAAAAAAAKJBFFCAA|1984-01-22|1008|4386|337|1984|0|1|22|1|1984|337|4386|Sunday|1984Q1|N|N|N|2445701|2445700|2445357|2445630|N|N|N|N|N| +2445723|AAAAAAAALJBFFCAA|1984-01-23|1008|4386|337|1984|1|1|23|1|1984|337|4386|Monday|1984Q1|N|N|N|2445701|2445700|2445358|2445631|N|N|N|N|N| +2445724|AAAAAAAAMJBFFCAA|1984-01-24|1008|4387|337|1984|2|1|24|1|1984|337|4387|Tuesday|1984Q1|N|N|N|2445701|2445700|2445359|2445632|N|N|N|N|N| +2445725|AAAAAAAANJBFFCAA|1984-01-25|1008|4387|337|1984|3|1|25|1|1984|337|4387|Wednesday|1984Q1|N|N|N|2445701|2445700|2445360|2445633|N|N|N|N|N| +2445726|AAAAAAAAOJBFFCAA|1984-01-26|1008|4387|337|1984|4|1|26|1|1984|337|4387|Thursday|1984Q1|N|N|N|2445701|2445700|2445361|2445634|N|N|N|N|N| +2445727|AAAAAAAAPJBFFCAA|1984-01-27|1008|4387|337|1984|5|1|27|1|1984|337|4387|Friday|1984Q1|N|Y|N|2445701|2445700|2445362|2445635|N|N|N|N|N| +2445728|AAAAAAAAAKBFFCAA|1984-01-28|1008|4387|337|1984|6|1|28|1|1984|337|4387|Saturday|1984Q1|N|Y|N|2445701|2445700|2445363|2445636|N|N|N|N|N| +2445729|AAAAAAAABKBFFCAA|1984-01-29|1008|4387|337|1984|0|1|29|1|1984|337|4387|Sunday|1984Q1|N|N|N|2445701|2445700|2445364|2445637|N|N|N|N|N| +2445730|AAAAAAAACKBFFCAA|1984-01-30|1008|4387|337|1984|1|1|30|1|1984|337|4387|Monday|1984Q1|N|N|N|2445701|2445700|2445365|2445638|N|N|N|N|N| +2445731|AAAAAAAADKBFFCAA|1984-01-31|1008|4388|337|1984|2|1|31|1|1984|337|4388|Tuesday|1984Q1|N|N|N|2445701|2445700|2445366|2445639|N|N|N|N|N| +2445732|AAAAAAAAEKBFFCAA|1984-02-01|1009|4388|337|1984|3|2|1|1|1984|337|4388|Wednesday|1984Q1|N|N|N|2445732|2445762|2445367|2445640|N|N|N|N|N| +2445733|AAAAAAAAFKBFFCAA|1984-02-02|1009|4388|337|1984|4|2|2|1|1984|337|4388|Thursday|1984Q1|N|N|N|2445732|2445762|2445368|2445641|N|N|N|N|N| +2445734|AAAAAAAAGKBFFCAA|1984-02-03|1009|4388|337|1984|5|2|3|1|1984|337|4388|Friday|1984Q1|N|Y|N|2445732|2445762|2445369|2445642|N|N|N|N|N| +2445735|AAAAAAAAHKBFFCAA|1984-02-04|1009|4388|337|1984|6|2|4|1|1984|337|4388|Saturday|1984Q1|N|Y|N|2445732|2445762|2445370|2445643|N|N|N|N|N| +2445736|AAAAAAAAIKBFFCAA|1984-02-05|1009|4388|337|1984|0|2|5|1|1984|337|4388|Sunday|1984Q1|N|N|N|2445732|2445762|2445371|2445644|N|N|N|N|N| +2445737|AAAAAAAAJKBFFCAA|1984-02-06|1009|4388|337|1984|1|2|6|1|1984|337|4388|Monday|1984Q1|N|N|N|2445732|2445762|2445372|2445645|N|N|N|N|N| +2445738|AAAAAAAAKKBFFCAA|1984-02-07|1009|4389|337|1984|2|2|7|1|1984|337|4389|Tuesday|1984Q1|N|N|N|2445732|2445762|2445373|2445646|N|N|N|N|N| +2445739|AAAAAAAALKBFFCAA|1984-02-08|1009|4389|337|1984|3|2|8|1|1984|337|4389|Wednesday|1984Q1|N|N|N|2445732|2445762|2445374|2445647|N|N|N|N|N| +2445740|AAAAAAAAMKBFFCAA|1984-02-09|1009|4389|337|1984|4|2|9|1|1984|337|4389|Thursday|1984Q1|N|N|N|2445732|2445762|2445375|2445648|N|N|N|N|N| +2445741|AAAAAAAANKBFFCAA|1984-02-10|1009|4389|337|1984|5|2|10|1|1984|337|4389|Friday|1984Q1|N|Y|N|2445732|2445762|2445376|2445649|N|N|N|N|N| +2445742|AAAAAAAAOKBFFCAA|1984-02-11|1009|4389|337|1984|6|2|11|1|1984|337|4389|Saturday|1984Q1|N|Y|N|2445732|2445762|2445377|2445650|N|N|N|N|N| +2445743|AAAAAAAAPKBFFCAA|1984-02-12|1009|4389|337|1984|0|2|12|1|1984|337|4389|Sunday|1984Q1|N|N|N|2445732|2445762|2445378|2445651|N|N|N|N|N| +2445744|AAAAAAAAALBFFCAA|1984-02-13|1009|4389|337|1984|1|2|13|1|1984|337|4389|Monday|1984Q1|N|N|N|2445732|2445762|2445379|2445652|N|N|N|N|N| +2445745|AAAAAAAABLBFFCAA|1984-02-14|1009|4390|337|1984|2|2|14|1|1984|337|4390|Tuesday|1984Q1|N|N|N|2445732|2445762|2445380|2445653|N|N|N|N|N| +2445746|AAAAAAAACLBFFCAA|1984-02-15|1009|4390|337|1984|3|2|15|1|1984|337|4390|Wednesday|1984Q1|N|N|N|2445732|2445762|2445381|2445654|N|N|N|N|N| +2445747|AAAAAAAADLBFFCAA|1984-02-16|1009|4390|337|1984|4|2|16|1|1984|337|4390|Thursday|1984Q1|N|N|N|2445732|2445762|2445382|2445655|N|N|N|N|N| +2445748|AAAAAAAAELBFFCAA|1984-02-17|1009|4390|337|1984|5|2|17|1|1984|337|4390|Friday|1984Q1|N|Y|N|2445732|2445762|2445383|2445656|N|N|N|N|N| +2445749|AAAAAAAAFLBFFCAA|1984-02-18|1009|4390|337|1984|6|2|18|1|1984|337|4390|Saturday|1984Q1|N|Y|N|2445732|2445762|2445384|2445657|N|N|N|N|N| +2445750|AAAAAAAAGLBFFCAA|1984-02-19|1009|4390|337|1984|0|2|19|1|1984|337|4390|Sunday|1984Q1|N|N|N|2445732|2445762|2445385|2445658|N|N|N|N|N| +2445751|AAAAAAAAHLBFFCAA|1984-02-20|1009|4390|337|1984|1|2|20|1|1984|337|4390|Monday|1984Q1|N|N|N|2445732|2445762|2445386|2445659|N|N|N|N|N| +2445752|AAAAAAAAILBFFCAA|1984-02-21|1009|4391|337|1984|2|2|21|1|1984|337|4391|Tuesday|1984Q1|N|N|N|2445732|2445762|2445387|2445660|N|N|N|N|N| +2445753|AAAAAAAAJLBFFCAA|1984-02-22|1009|4391|337|1984|3|2|22|1|1984|337|4391|Wednesday|1984Q1|N|N|N|2445732|2445762|2445388|2445661|N|N|N|N|N| +2445754|AAAAAAAAKLBFFCAA|1984-02-23|1009|4391|337|1984|4|2|23|1|1984|337|4391|Thursday|1984Q1|N|N|N|2445732|2445762|2445389|2445662|N|N|N|N|N| +2445755|AAAAAAAALLBFFCAA|1984-02-24|1009|4391|337|1984|5|2|24|1|1984|337|4391|Friday|1984Q1|N|Y|N|2445732|2445762|2445390|2445663|N|N|N|N|N| +2445756|AAAAAAAAMLBFFCAA|1984-02-25|1009|4391|337|1984|6|2|25|1|1984|337|4391|Saturday|1984Q1|N|Y|N|2445732|2445762|2445391|2445664|N|N|N|N|N| +2445757|AAAAAAAANLBFFCAA|1984-02-26|1009|4391|337|1984|0|2|26|1|1984|337|4391|Sunday|1984Q1|N|N|N|2445732|2445762|2445392|2445665|N|N|N|N|N| +2445758|AAAAAAAAOLBFFCAA|1984-02-27|1009|4391|337|1984|1|2|27|1|1984|337|4391|Monday|1984Q1|N|N|N|2445732|2445762|2445393|2445666|N|N|N|N|N| +2445759|AAAAAAAAPLBFFCAA|1984-02-28|1009|4392|337|1984|2|2|28|1|1984|337|4392|Tuesday|1984Q1|N|N|N|2445732|2445762|2445394|2445667|N|N|N|N|N| +2445760|AAAAAAAAAMBFFCAA|1984-02-29|1009|4392|337|1984|3|2|29|1|1984|337|4392|Wednesday|1984Q1|N|N|N|2445732|2445762|2445394|2445668|N|N|N|N|N| +2445761|AAAAAAAABMBFFCAA|1984-03-01|1010|4392|338|1984|4|3|1|1|1984|338|4392|Thursday|1984Q1|N|N|N|2445761|2445820|2445395|2445669|N|N|N|N|N| +2445762|AAAAAAAACMBFFCAA|1984-03-02|1010|4392|338|1984|5|3|2|1|1984|338|4392|Friday|1984Q1|N|Y|N|2445761|2445820|2445396|2445670|N|N|N|N|N| +2445763|AAAAAAAADMBFFCAA|1984-03-03|1010|4392|338|1984|6|3|3|1|1984|338|4392|Saturday|1984Q1|N|Y|N|2445761|2445820|2445397|2445671|N|N|N|N|N| +2445764|AAAAAAAAEMBFFCAA|1984-03-04|1010|4392|338|1984|0|3|4|1|1984|338|4392|Sunday|1984Q1|N|N|N|2445761|2445820|2445398|2445672|N|N|N|N|N| +2445765|AAAAAAAAFMBFFCAA|1984-03-05|1010|4392|338|1984|1|3|5|1|1984|338|4392|Monday|1984Q1|N|N|N|2445761|2445820|2445399|2445673|N|N|N|N|N| +2445766|AAAAAAAAGMBFFCAA|1984-03-06|1010|4393|338|1984|2|3|6|1|1984|338|4393|Tuesday|1984Q1|N|N|N|2445761|2445820|2445400|2445674|N|N|N|N|N| +2445767|AAAAAAAAHMBFFCAA|1984-03-07|1010|4393|338|1984|3|3|7|1|1984|338|4393|Wednesday|1984Q1|N|N|N|2445761|2445820|2445401|2445675|N|N|N|N|N| +2445768|AAAAAAAAIMBFFCAA|1984-03-08|1010|4393|338|1984|4|3|8|1|1984|338|4393|Thursday|1984Q1|N|N|N|2445761|2445820|2445402|2445676|N|N|N|N|N| +2445769|AAAAAAAAJMBFFCAA|1984-03-09|1010|4393|338|1984|5|3|9|1|1984|338|4393|Friday|1984Q1|N|Y|N|2445761|2445820|2445403|2445677|N|N|N|N|N| +2445770|AAAAAAAAKMBFFCAA|1984-03-10|1010|4393|338|1984|6|3|10|1|1984|338|4393|Saturday|1984Q1|N|Y|N|2445761|2445820|2445404|2445678|N|N|N|N|N| +2445771|AAAAAAAALMBFFCAA|1984-03-11|1010|4393|338|1984|0|3|11|1|1984|338|4393|Sunday|1984Q1|N|N|N|2445761|2445820|2445405|2445679|N|N|N|N|N| +2445772|AAAAAAAAMMBFFCAA|1984-03-12|1010|4393|338|1984|1|3|12|1|1984|338|4393|Monday|1984Q1|N|N|N|2445761|2445820|2445406|2445680|N|N|N|N|N| +2445773|AAAAAAAANMBFFCAA|1984-03-13|1010|4394|338|1984|2|3|13|1|1984|338|4394|Tuesday|1984Q1|N|N|N|2445761|2445820|2445407|2445681|N|N|N|N|N| +2445774|AAAAAAAAOMBFFCAA|1984-03-14|1010|4394|338|1984|3|3|14|1|1984|338|4394|Wednesday|1984Q1|N|N|N|2445761|2445820|2445408|2445682|N|N|N|N|N| +2445775|AAAAAAAAPMBFFCAA|1984-03-15|1010|4394|338|1984|4|3|15|1|1984|338|4394|Thursday|1984Q1|N|N|N|2445761|2445820|2445409|2445683|N|N|N|N|N| +2445776|AAAAAAAAANBFFCAA|1984-03-16|1010|4394|338|1984|5|3|16|1|1984|338|4394|Friday|1984Q1|N|Y|N|2445761|2445820|2445410|2445684|N|N|N|N|N| +2445777|AAAAAAAABNBFFCAA|1984-03-17|1010|4394|338|1984|6|3|17|1|1984|338|4394|Saturday|1984Q1|N|Y|N|2445761|2445820|2445411|2445685|N|N|N|N|N| +2445778|AAAAAAAACNBFFCAA|1984-03-18|1010|4394|338|1984|0|3|18|1|1984|338|4394|Sunday|1984Q1|N|N|N|2445761|2445820|2445412|2445686|N|N|N|N|N| +2445779|AAAAAAAADNBFFCAA|1984-03-19|1010|4394|338|1984|1|3|19|1|1984|338|4394|Monday|1984Q1|N|N|N|2445761|2445820|2445413|2445687|N|N|N|N|N| +2445780|AAAAAAAAENBFFCAA|1984-03-20|1010|4395|338|1984|2|3|20|1|1984|338|4395|Tuesday|1984Q1|N|N|N|2445761|2445820|2445414|2445688|N|N|N|N|N| +2445781|AAAAAAAAFNBFFCAA|1984-03-21|1010|4395|338|1984|3|3|21|1|1984|338|4395|Wednesday|1984Q1|N|N|N|2445761|2445820|2445415|2445689|N|N|N|N|N| +2445782|AAAAAAAAGNBFFCAA|1984-03-22|1010|4395|338|1984|4|3|22|1|1984|338|4395|Thursday|1984Q1|N|N|N|2445761|2445820|2445416|2445690|N|N|N|N|N| +2445783|AAAAAAAAHNBFFCAA|1984-03-23|1010|4395|338|1984|5|3|23|1|1984|338|4395|Friday|1984Q1|N|Y|N|2445761|2445820|2445417|2445691|N|N|N|N|N| +2445784|AAAAAAAAINBFFCAA|1984-03-24|1010|4395|338|1984|6|3|24|1|1984|338|4395|Saturday|1984Q1|N|Y|N|2445761|2445820|2445418|2445692|N|N|N|N|N| +2445785|AAAAAAAAJNBFFCAA|1984-03-25|1010|4395|338|1984|0|3|25|1|1984|338|4395|Sunday|1984Q1|N|N|N|2445761|2445820|2445419|2445693|N|N|N|N|N| +2445786|AAAAAAAAKNBFFCAA|1984-03-26|1010|4395|338|1984|1|3|26|1|1984|338|4395|Monday|1984Q1|N|N|N|2445761|2445820|2445420|2445694|N|N|N|N|N| +2445787|AAAAAAAALNBFFCAA|1984-03-27|1010|4396|338|1984|2|3|27|1|1984|338|4396|Tuesday|1984Q1|N|N|N|2445761|2445820|2445421|2445695|N|N|N|N|N| +2445788|AAAAAAAAMNBFFCAA|1984-03-28|1010|4396|338|1984|3|3|28|1|1984|338|4396|Wednesday|1984Q1|N|N|N|2445761|2445820|2445422|2445696|N|N|N|N|N| +2445789|AAAAAAAANNBFFCAA|1984-03-29|1010|4396|338|1984|4|3|29|1|1984|338|4396|Thursday|1984Q1|N|N|N|2445761|2445820|2445423|2445697|N|N|N|N|N| +2445790|AAAAAAAAONBFFCAA|1984-03-30|1010|4396|338|1984|5|3|30|1|1984|338|4396|Friday|1984Q1|N|Y|N|2445761|2445820|2445424|2445698|N|N|N|N|N| +2445791|AAAAAAAAPNBFFCAA|1984-03-31|1010|4396|338|1984|6|3|31|1|1984|338|4396|Saturday|1984Q1|N|Y|N|2445761|2445820|2445425|2445699|N|N|N|N|N| +2445792|AAAAAAAAAOBFFCAA|1984-04-01|1011|4396|338|1984|0|4|1|2|1984|338|4396|Sunday|1984Q2|N|N|N|2445792|2445882|2445426|2445701|N|N|N|N|N| +2445793|AAAAAAAABOBFFCAA|1984-04-02|1011|4396|338|1984|1|4|2|2|1984|338|4396|Monday|1984Q2|N|N|N|2445792|2445882|2445427|2445702|N|N|N|N|N| +2445794|AAAAAAAACOBFFCAA|1984-04-03|1011|4397|338|1984|2|4|3|2|1984|338|4397|Tuesday|1984Q2|N|N|N|2445792|2445882|2445428|2445703|N|N|N|N|N| +2445795|AAAAAAAADOBFFCAA|1984-04-04|1011|4397|338|1984|3|4|4|2|1984|338|4397|Wednesday|1984Q2|N|N|N|2445792|2445882|2445429|2445704|N|N|N|N|N| +2445796|AAAAAAAAEOBFFCAA|1984-04-05|1011|4397|338|1984|4|4|5|2|1984|338|4397|Thursday|1984Q2|N|N|N|2445792|2445882|2445430|2445705|N|N|N|N|N| +2445797|AAAAAAAAFOBFFCAA|1984-04-06|1011|4397|338|1984|5|4|6|2|1984|338|4397|Friday|1984Q2|N|Y|N|2445792|2445882|2445431|2445706|N|N|N|N|N| +2445798|AAAAAAAAGOBFFCAA|1984-04-07|1011|4397|338|1984|6|4|7|2|1984|338|4397|Saturday|1984Q2|N|Y|N|2445792|2445882|2445432|2445707|N|N|N|N|N| +2445799|AAAAAAAAHOBFFCAA|1984-04-08|1011|4397|338|1984|0|4|8|2|1984|338|4397|Sunday|1984Q2|N|N|N|2445792|2445882|2445433|2445708|N|N|N|N|N| +2445800|AAAAAAAAIOBFFCAA|1984-04-09|1011|4397|338|1984|1|4|9|2|1984|338|4397|Monday|1984Q2|N|N|N|2445792|2445882|2445434|2445709|N|N|N|N|N| +2445801|AAAAAAAAJOBFFCAA|1984-04-10|1011|4398|338|1984|2|4|10|2|1984|338|4398|Tuesday|1984Q2|N|N|N|2445792|2445882|2445435|2445710|N|N|N|N|N| +2445802|AAAAAAAAKOBFFCAA|1984-04-11|1011|4398|338|1984|3|4|11|2|1984|338|4398|Wednesday|1984Q2|N|N|N|2445792|2445882|2445436|2445711|N|N|N|N|N| +2445803|AAAAAAAALOBFFCAA|1984-04-12|1011|4398|338|1984|4|4|12|2|1984|338|4398|Thursday|1984Q2|N|N|N|2445792|2445882|2445437|2445712|N|N|N|N|N| +2445804|AAAAAAAAMOBFFCAA|1984-04-13|1011|4398|338|1984|5|4|13|2|1984|338|4398|Friday|1984Q2|N|Y|N|2445792|2445882|2445438|2445713|N|N|N|N|N| +2445805|AAAAAAAANOBFFCAA|1984-04-14|1011|4398|338|1984|6|4|14|2|1984|338|4398|Saturday|1984Q2|N|Y|N|2445792|2445882|2445439|2445714|N|N|N|N|N| +2445806|AAAAAAAAOOBFFCAA|1984-04-15|1011|4398|338|1984|0|4|15|2|1984|338|4398|Sunday|1984Q2|N|N|N|2445792|2445882|2445440|2445715|N|N|N|N|N| +2445807|AAAAAAAAPOBFFCAA|1984-04-16|1011|4398|338|1984|1|4|16|2|1984|338|4398|Monday|1984Q2|N|N|N|2445792|2445882|2445441|2445716|N|N|N|N|N| +2445808|AAAAAAAAAPBFFCAA|1984-04-17|1011|4399|338|1984|2|4|17|2|1984|338|4399|Tuesday|1984Q2|N|N|N|2445792|2445882|2445442|2445717|N|N|N|N|N| +2445809|AAAAAAAABPBFFCAA|1984-04-18|1011|4399|338|1984|3|4|18|2|1984|338|4399|Wednesday|1984Q2|N|N|N|2445792|2445882|2445443|2445718|N|N|N|N|N| +2445810|AAAAAAAACPBFFCAA|1984-04-19|1011|4399|338|1984|4|4|19|2|1984|338|4399|Thursday|1984Q2|N|N|N|2445792|2445882|2445444|2445719|N|N|N|N|N| +2445811|AAAAAAAADPBFFCAA|1984-04-20|1011|4399|338|1984|5|4|20|2|1984|338|4399|Friday|1984Q2|N|Y|N|2445792|2445882|2445445|2445720|N|N|N|N|N| +2445812|AAAAAAAAEPBFFCAA|1984-04-21|1011|4399|338|1984|6|4|21|2|1984|338|4399|Saturday|1984Q2|N|Y|N|2445792|2445882|2445446|2445721|N|N|N|N|N| +2445813|AAAAAAAAFPBFFCAA|1984-04-22|1011|4399|338|1984|0|4|22|2|1984|338|4399|Sunday|1984Q2|N|N|N|2445792|2445882|2445447|2445722|N|N|N|N|N| +2445814|AAAAAAAAGPBFFCAA|1984-04-23|1011|4399|338|1984|1|4|23|2|1984|338|4399|Monday|1984Q2|N|N|N|2445792|2445882|2445448|2445723|N|N|N|N|N| +2445815|AAAAAAAAHPBFFCAA|1984-04-24|1011|4400|338|1984|2|4|24|2|1984|338|4400|Tuesday|1984Q2|N|N|N|2445792|2445882|2445449|2445724|N|N|N|N|N| +2445816|AAAAAAAAIPBFFCAA|1984-04-25|1011|4400|338|1984|3|4|25|2|1984|338|4400|Wednesday|1984Q2|N|N|N|2445792|2445882|2445450|2445725|N|N|N|N|N| +2445817|AAAAAAAAJPBFFCAA|1984-04-26|1011|4400|338|1984|4|4|26|2|1984|338|4400|Thursday|1984Q2|N|N|N|2445792|2445882|2445451|2445726|N|N|N|N|N| +2445818|AAAAAAAAKPBFFCAA|1984-04-27|1011|4400|338|1984|5|4|27|2|1984|338|4400|Friday|1984Q2|N|Y|N|2445792|2445882|2445452|2445727|N|N|N|N|N| +2445819|AAAAAAAALPBFFCAA|1984-04-28|1011|4400|338|1984|6|4|28|2|1984|338|4400|Saturday|1984Q2|N|Y|N|2445792|2445882|2445453|2445728|N|N|N|N|N| +2445820|AAAAAAAAMPBFFCAA|1984-04-29|1011|4400|338|1984|0|4|29|2|1984|338|4400|Sunday|1984Q2|N|N|N|2445792|2445882|2445454|2445729|N|N|N|N|N| +2445821|AAAAAAAANPBFFCAA|1984-04-30|1011|4400|338|1984|1|4|30|2|1984|338|4400|Monday|1984Q2|N|N|N|2445792|2445882|2445455|2445730|N|N|N|N|N| +2445822|AAAAAAAAOPBFFCAA|1984-05-01|1012|4401|338|1984|2|5|1|2|1984|338|4401|Tuesday|1984Q2|N|N|N|2445822|2445942|2445456|2445731|N|N|N|N|N| +2445823|AAAAAAAAPPBFFCAA|1984-05-02|1012|4401|338|1984|3|5|2|2|1984|338|4401|Wednesday|1984Q2|N|N|N|2445822|2445942|2445457|2445732|N|N|N|N|N| +2445824|AAAAAAAAAACFFCAA|1984-05-03|1012|4401|338|1984|4|5|3|2|1984|338|4401|Thursday|1984Q2|N|N|N|2445822|2445942|2445458|2445733|N|N|N|N|N| +2445825|AAAAAAAABACFFCAA|1984-05-04|1012|4401|338|1984|5|5|4|2|1984|338|4401|Friday|1984Q2|N|Y|N|2445822|2445942|2445459|2445734|N|N|N|N|N| +2445826|AAAAAAAACACFFCAA|1984-05-05|1012|4401|338|1984|6|5|5|2|1984|338|4401|Saturday|1984Q2|N|Y|N|2445822|2445942|2445460|2445735|N|N|N|N|N| +2445827|AAAAAAAADACFFCAA|1984-05-06|1012|4401|338|1984|0|5|6|2|1984|338|4401|Sunday|1984Q2|N|N|N|2445822|2445942|2445461|2445736|N|N|N|N|N| +2445828|AAAAAAAAEACFFCAA|1984-05-07|1012|4401|338|1984|1|5|7|2|1984|338|4401|Monday|1984Q2|N|N|N|2445822|2445942|2445462|2445737|N|N|N|N|N| +2445829|AAAAAAAAFACFFCAA|1984-05-08|1012|4402|338|1984|2|5|8|2|1984|338|4402|Tuesday|1984Q2|N|N|N|2445822|2445942|2445463|2445738|N|N|N|N|N| +2445830|AAAAAAAAGACFFCAA|1984-05-09|1012|4402|338|1984|3|5|9|2|1984|338|4402|Wednesday|1984Q2|N|N|N|2445822|2445942|2445464|2445739|N|N|N|N|N| +2445831|AAAAAAAAHACFFCAA|1984-05-10|1012|4402|338|1984|4|5|10|2|1984|338|4402|Thursday|1984Q2|N|N|N|2445822|2445942|2445465|2445740|N|N|N|N|N| +2445832|AAAAAAAAIACFFCAA|1984-05-11|1012|4402|338|1984|5|5|11|2|1984|338|4402|Friday|1984Q2|N|Y|N|2445822|2445942|2445466|2445741|N|N|N|N|N| +2445833|AAAAAAAAJACFFCAA|1984-05-12|1012|4402|338|1984|6|5|12|2|1984|338|4402|Saturday|1984Q2|N|Y|N|2445822|2445942|2445467|2445742|N|N|N|N|N| +2445834|AAAAAAAAKACFFCAA|1984-05-13|1012|4402|338|1984|0|5|13|2|1984|338|4402|Sunday|1984Q2|N|N|N|2445822|2445942|2445468|2445743|N|N|N|N|N| +2445835|AAAAAAAALACFFCAA|1984-05-14|1012|4402|338|1984|1|5|14|2|1984|338|4402|Monday|1984Q2|N|N|N|2445822|2445942|2445469|2445744|N|N|N|N|N| +2445836|AAAAAAAAMACFFCAA|1984-05-15|1012|4403|338|1984|2|5|15|2|1984|338|4403|Tuesday|1984Q2|N|N|N|2445822|2445942|2445470|2445745|N|N|N|N|N| +2445837|AAAAAAAANACFFCAA|1984-05-16|1012|4403|338|1984|3|5|16|2|1984|338|4403|Wednesday|1984Q2|N|N|N|2445822|2445942|2445471|2445746|N|N|N|N|N| +2445838|AAAAAAAAOACFFCAA|1984-05-17|1012|4403|338|1984|4|5|17|2|1984|338|4403|Thursday|1984Q2|N|N|N|2445822|2445942|2445472|2445747|N|N|N|N|N| +2445839|AAAAAAAAPACFFCAA|1984-05-18|1012|4403|338|1984|5|5|18|2|1984|338|4403|Friday|1984Q2|N|Y|N|2445822|2445942|2445473|2445748|N|N|N|N|N| +2445840|AAAAAAAAABCFFCAA|1984-05-19|1012|4403|338|1984|6|5|19|2|1984|338|4403|Saturday|1984Q2|N|Y|N|2445822|2445942|2445474|2445749|N|N|N|N|N| +2445841|AAAAAAAABBCFFCAA|1984-05-20|1012|4403|338|1984|0|5|20|2|1984|338|4403|Sunday|1984Q2|N|N|N|2445822|2445942|2445475|2445750|N|N|N|N|N| +2445842|AAAAAAAACBCFFCAA|1984-05-21|1012|4403|338|1984|1|5|21|2|1984|338|4403|Monday|1984Q2|N|N|N|2445822|2445942|2445476|2445751|N|N|N|N|N| +2445843|AAAAAAAADBCFFCAA|1984-05-22|1012|4404|338|1984|2|5|22|2|1984|338|4404|Tuesday|1984Q2|N|N|N|2445822|2445942|2445477|2445752|N|N|N|N|N| +2445844|AAAAAAAAEBCFFCAA|1984-05-23|1012|4404|338|1984|3|5|23|2|1984|338|4404|Wednesday|1984Q2|N|N|N|2445822|2445942|2445478|2445753|N|N|N|N|N| +2445845|AAAAAAAAFBCFFCAA|1984-05-24|1012|4404|338|1984|4|5|24|2|1984|338|4404|Thursday|1984Q2|N|N|N|2445822|2445942|2445479|2445754|N|N|N|N|N| +2445846|AAAAAAAAGBCFFCAA|1984-05-25|1012|4404|338|1984|5|5|25|2|1984|338|4404|Friday|1984Q2|N|Y|N|2445822|2445942|2445480|2445755|N|N|N|N|N| +2445847|AAAAAAAAHBCFFCAA|1984-05-26|1012|4404|338|1984|6|5|26|2|1984|338|4404|Saturday|1984Q2|N|Y|N|2445822|2445942|2445481|2445756|N|N|N|N|N| +2445848|AAAAAAAAIBCFFCAA|1984-05-27|1012|4404|338|1984|0|5|27|2|1984|338|4404|Sunday|1984Q2|N|N|N|2445822|2445942|2445482|2445757|N|N|N|N|N| +2445849|AAAAAAAAJBCFFCAA|1984-05-28|1012|4404|338|1984|1|5|28|2|1984|338|4404|Monday|1984Q2|N|N|N|2445822|2445942|2445483|2445758|N|N|N|N|N| +2445850|AAAAAAAAKBCFFCAA|1984-05-29|1012|4405|338|1984|2|5|29|2|1984|338|4405|Tuesday|1984Q2|N|N|N|2445822|2445942|2445484|2445759|N|N|N|N|N| +2445851|AAAAAAAALBCFFCAA|1984-05-30|1012|4405|338|1984|3|5|30|2|1984|338|4405|Wednesday|1984Q2|N|N|N|2445822|2445942|2445485|2445760|N|N|N|N|N| +2445852|AAAAAAAAMBCFFCAA|1984-05-31|1012|4405|338|1984|4|5|31|2|1984|338|4405|Thursday|1984Q2|N|N|N|2445822|2445942|2445486|2445761|N|N|N|N|N| +2445853|AAAAAAAANBCFFCAA|1984-06-01|1013|4405|339|1984|5|6|1|2|1984|339|4405|Friday|1984Q2|N|Y|N|2445853|2446004|2445487|2445762|N|N|N|N|N| +2445854|AAAAAAAAOBCFFCAA|1984-06-02|1013|4405|339|1984|6|6|2|2|1984|339|4405|Saturday|1984Q2|N|Y|N|2445853|2446004|2445488|2445763|N|N|N|N|N| +2445855|AAAAAAAAPBCFFCAA|1984-06-03|1013|4405|339|1984|0|6|3|2|1984|339|4405|Sunday|1984Q2|N|N|N|2445853|2446004|2445489|2445764|N|N|N|N|N| +2445856|AAAAAAAAACCFFCAA|1984-06-04|1013|4405|339|1984|1|6|4|2|1984|339|4405|Monday|1984Q2|N|N|N|2445853|2446004|2445490|2445765|N|N|N|N|N| +2445857|AAAAAAAABCCFFCAA|1984-06-05|1013|4406|339|1984|2|6|5|2|1984|339|4406|Tuesday|1984Q2|N|N|N|2445853|2446004|2445491|2445766|N|N|N|N|N| +2445858|AAAAAAAACCCFFCAA|1984-06-06|1013|4406|339|1984|3|6|6|2|1984|339|4406|Wednesday|1984Q2|N|N|N|2445853|2446004|2445492|2445767|N|N|N|N|N| +2445859|AAAAAAAADCCFFCAA|1984-06-07|1013|4406|339|1984|4|6|7|2|1984|339|4406|Thursday|1984Q2|N|N|N|2445853|2446004|2445493|2445768|N|N|N|N|N| +2445860|AAAAAAAAECCFFCAA|1984-06-08|1013|4406|339|1984|5|6|8|2|1984|339|4406|Friday|1984Q2|N|Y|N|2445853|2446004|2445494|2445769|N|N|N|N|N| +2445861|AAAAAAAAFCCFFCAA|1984-06-09|1013|4406|339|1984|6|6|9|2|1984|339|4406|Saturday|1984Q2|N|Y|N|2445853|2446004|2445495|2445770|N|N|N|N|N| +2445862|AAAAAAAAGCCFFCAA|1984-06-10|1013|4406|339|1984|0|6|10|2|1984|339|4406|Sunday|1984Q2|N|N|N|2445853|2446004|2445496|2445771|N|N|N|N|N| +2445863|AAAAAAAAHCCFFCAA|1984-06-11|1013|4406|339|1984|1|6|11|2|1984|339|4406|Monday|1984Q2|N|N|N|2445853|2446004|2445497|2445772|N|N|N|N|N| +2445864|AAAAAAAAICCFFCAA|1984-06-12|1013|4407|339|1984|2|6|12|2|1984|339|4407|Tuesday|1984Q2|N|N|N|2445853|2446004|2445498|2445773|N|N|N|N|N| +2445865|AAAAAAAAJCCFFCAA|1984-06-13|1013|4407|339|1984|3|6|13|2|1984|339|4407|Wednesday|1984Q2|N|N|N|2445853|2446004|2445499|2445774|N|N|N|N|N| +2445866|AAAAAAAAKCCFFCAA|1984-06-14|1013|4407|339|1984|4|6|14|2|1984|339|4407|Thursday|1984Q2|N|N|N|2445853|2446004|2445500|2445775|N|N|N|N|N| +2445867|AAAAAAAALCCFFCAA|1984-06-15|1013|4407|339|1984|5|6|15|2|1984|339|4407|Friday|1984Q2|N|Y|N|2445853|2446004|2445501|2445776|N|N|N|N|N| +2445868|AAAAAAAAMCCFFCAA|1984-06-16|1013|4407|339|1984|6|6|16|2|1984|339|4407|Saturday|1984Q2|N|Y|N|2445853|2446004|2445502|2445777|N|N|N|N|N| +2445869|AAAAAAAANCCFFCAA|1984-06-17|1013|4407|339|1984|0|6|17|2|1984|339|4407|Sunday|1984Q2|N|N|N|2445853|2446004|2445503|2445778|N|N|N|N|N| +2445870|AAAAAAAAOCCFFCAA|1984-06-18|1013|4407|339|1984|1|6|18|2|1984|339|4407|Monday|1984Q2|N|N|N|2445853|2446004|2445504|2445779|N|N|N|N|N| +2445871|AAAAAAAAPCCFFCAA|1984-06-19|1013|4408|339|1984|2|6|19|2|1984|339|4408|Tuesday|1984Q2|N|N|N|2445853|2446004|2445505|2445780|N|N|N|N|N| +2445872|AAAAAAAAADCFFCAA|1984-06-20|1013|4408|339|1984|3|6|20|2|1984|339|4408|Wednesday|1984Q2|N|N|N|2445853|2446004|2445506|2445781|N|N|N|N|N| +2445873|AAAAAAAABDCFFCAA|1984-06-21|1013|4408|339|1984|4|6|21|2|1984|339|4408|Thursday|1984Q2|N|N|N|2445853|2446004|2445507|2445782|N|N|N|N|N| +2445874|AAAAAAAACDCFFCAA|1984-06-22|1013|4408|339|1984|5|6|22|2|1984|339|4408|Friday|1984Q2|N|Y|N|2445853|2446004|2445508|2445783|N|N|N|N|N| +2445875|AAAAAAAADDCFFCAA|1984-06-23|1013|4408|339|1984|6|6|23|2|1984|339|4408|Saturday|1984Q2|N|Y|N|2445853|2446004|2445509|2445784|N|N|N|N|N| +2445876|AAAAAAAAEDCFFCAA|1984-06-24|1013|4408|339|1984|0|6|24|2|1984|339|4408|Sunday|1984Q2|N|N|N|2445853|2446004|2445510|2445785|N|N|N|N|N| +2445877|AAAAAAAAFDCFFCAA|1984-06-25|1013|4408|339|1984|1|6|25|2|1984|339|4408|Monday|1984Q2|N|N|N|2445853|2446004|2445511|2445786|N|N|N|N|N| +2445878|AAAAAAAAGDCFFCAA|1984-06-26|1013|4409|339|1984|2|6|26|2|1984|339|4409|Tuesday|1984Q2|N|N|N|2445853|2446004|2445512|2445787|N|N|N|N|N| +2445879|AAAAAAAAHDCFFCAA|1984-06-27|1013|4409|339|1984|3|6|27|2|1984|339|4409|Wednesday|1984Q2|N|N|N|2445853|2446004|2445513|2445788|N|N|N|N|N| +2445880|AAAAAAAAIDCFFCAA|1984-06-28|1013|4409|339|1984|4|6|28|2|1984|339|4409|Thursday|1984Q2|N|N|N|2445853|2446004|2445514|2445789|N|N|N|N|N| +2445881|AAAAAAAAJDCFFCAA|1984-06-29|1013|4409|339|1984|5|6|29|2|1984|339|4409|Friday|1984Q2|N|Y|N|2445853|2446004|2445515|2445790|N|N|N|N|N| +2445882|AAAAAAAAKDCFFCAA|1984-06-30|1013|4409|339|1984|6|6|30|2|1984|339|4409|Saturday|1984Q2|N|Y|N|2445853|2446004|2445516|2445791|N|N|N|N|N| +2445883|AAAAAAAALDCFFCAA|1984-07-01|1014|4409|339|1984|0|7|1|3|1984|339|4409|Sunday|1984Q3|N|N|N|2445883|2446064|2445517|2445792|N|N|N|N|N| +2445884|AAAAAAAAMDCFFCAA|1984-07-02|1014|4409|339|1984|1|7|2|3|1984|339|4409|Monday|1984Q3|N|N|N|2445883|2446064|2445518|2445793|N|N|N|N|N| +2445885|AAAAAAAANDCFFCAA|1984-07-03|1014|4410|339|1984|2|7|3|3|1984|339|4410|Tuesday|1984Q3|N|N|N|2445883|2446064|2445519|2445794|N|N|N|N|N| +2445886|AAAAAAAAODCFFCAA|1984-07-04|1014|4410|339|1984|3|7|4|3|1984|339|4410|Wednesday|1984Q3|Y|N|N|2445883|2446064|2445520|2445795|N|N|N|N|N| +2445887|AAAAAAAAPDCFFCAA|1984-07-05|1014|4410|339|1984|4|7|5|3|1984|339|4410|Thursday|1984Q3|N|N|Y|2445883|2446064|2445521|2445796|N|N|N|N|N| +2445888|AAAAAAAAAECFFCAA|1984-07-06|1014|4410|339|1984|5|7|6|3|1984|339|4410|Friday|1984Q3|N|Y|N|2445883|2446064|2445522|2445797|N|N|N|N|N| +2445889|AAAAAAAABECFFCAA|1984-07-07|1014|4410|339|1984|6|7|7|3|1984|339|4410|Saturday|1984Q3|N|Y|N|2445883|2446064|2445523|2445798|N|N|N|N|N| +2445890|AAAAAAAACECFFCAA|1984-07-08|1014|4410|339|1984|0|7|8|3|1984|339|4410|Sunday|1984Q3|N|N|N|2445883|2446064|2445524|2445799|N|N|N|N|N| +2445891|AAAAAAAADECFFCAA|1984-07-09|1014|4410|339|1984|1|7|9|3|1984|339|4410|Monday|1984Q3|N|N|N|2445883|2446064|2445525|2445800|N|N|N|N|N| +2445892|AAAAAAAAEECFFCAA|1984-07-10|1014|4411|339|1984|2|7|10|3|1984|339|4411|Tuesday|1984Q3|N|N|N|2445883|2446064|2445526|2445801|N|N|N|N|N| +2445893|AAAAAAAAFECFFCAA|1984-07-11|1014|4411|339|1984|3|7|11|3|1984|339|4411|Wednesday|1984Q3|N|N|N|2445883|2446064|2445527|2445802|N|N|N|N|N| +2445894|AAAAAAAAGECFFCAA|1984-07-12|1014|4411|339|1984|4|7|12|3|1984|339|4411|Thursday|1984Q3|N|N|N|2445883|2446064|2445528|2445803|N|N|N|N|N| +2445895|AAAAAAAAHECFFCAA|1984-07-13|1014|4411|339|1984|5|7|13|3|1984|339|4411|Friday|1984Q3|N|Y|N|2445883|2446064|2445529|2445804|N|N|N|N|N| +2445896|AAAAAAAAIECFFCAA|1984-07-14|1014|4411|339|1984|6|7|14|3|1984|339|4411|Saturday|1984Q3|N|Y|N|2445883|2446064|2445530|2445805|N|N|N|N|N| +2445897|AAAAAAAAJECFFCAA|1984-07-15|1014|4411|339|1984|0|7|15|3|1984|339|4411|Sunday|1984Q3|N|N|N|2445883|2446064|2445531|2445806|N|N|N|N|N| +2445898|AAAAAAAAKECFFCAA|1984-07-16|1014|4411|339|1984|1|7|16|3|1984|339|4411|Monday|1984Q3|N|N|N|2445883|2446064|2445532|2445807|N|N|N|N|N| +2445899|AAAAAAAALECFFCAA|1984-07-17|1014|4412|339|1984|2|7|17|3|1984|339|4412|Tuesday|1984Q3|N|N|N|2445883|2446064|2445533|2445808|N|N|N|N|N| +2445900|AAAAAAAAMECFFCAA|1984-07-18|1014|4412|339|1984|3|7|18|3|1984|339|4412|Wednesday|1984Q3|N|N|N|2445883|2446064|2445534|2445809|N|N|N|N|N| +2445901|AAAAAAAANECFFCAA|1984-07-19|1014|4412|339|1984|4|7|19|3|1984|339|4412|Thursday|1984Q3|N|N|N|2445883|2446064|2445535|2445810|N|N|N|N|N| +2445902|AAAAAAAAOECFFCAA|1984-07-20|1014|4412|339|1984|5|7|20|3|1984|339|4412|Friday|1984Q3|N|Y|N|2445883|2446064|2445536|2445811|N|N|N|N|N| +2445903|AAAAAAAAPECFFCAA|1984-07-21|1014|4412|339|1984|6|7|21|3|1984|339|4412|Saturday|1984Q3|N|Y|N|2445883|2446064|2445537|2445812|N|N|N|N|N| +2445904|AAAAAAAAAFCFFCAA|1984-07-22|1014|4412|339|1984|0|7|22|3|1984|339|4412|Sunday|1984Q3|N|N|N|2445883|2446064|2445538|2445813|N|N|N|N|N| +2445905|AAAAAAAABFCFFCAA|1984-07-23|1014|4412|339|1984|1|7|23|3|1984|339|4412|Monday|1984Q3|N|N|N|2445883|2446064|2445539|2445814|N|N|N|N|N| +2445906|AAAAAAAACFCFFCAA|1984-07-24|1014|4413|339|1984|2|7|24|3|1984|339|4413|Tuesday|1984Q3|N|N|N|2445883|2446064|2445540|2445815|N|N|N|N|N| +2445907|AAAAAAAADFCFFCAA|1984-07-25|1014|4413|339|1984|3|7|25|3|1984|339|4413|Wednesday|1984Q3|N|N|N|2445883|2446064|2445541|2445816|N|N|N|N|N| +2445908|AAAAAAAAEFCFFCAA|1984-07-26|1014|4413|339|1984|4|7|26|3|1984|339|4413|Thursday|1984Q3|N|N|N|2445883|2446064|2445542|2445817|N|N|N|N|N| +2445909|AAAAAAAAFFCFFCAA|1984-07-27|1014|4413|339|1984|5|7|27|3|1984|339|4413|Friday|1984Q3|N|Y|N|2445883|2446064|2445543|2445818|N|N|N|N|N| +2445910|AAAAAAAAGFCFFCAA|1984-07-28|1014|4413|339|1984|6|7|28|3|1984|339|4413|Saturday|1984Q3|N|Y|N|2445883|2446064|2445544|2445819|N|N|N|N|N| +2445911|AAAAAAAAHFCFFCAA|1984-07-29|1014|4413|339|1984|0|7|29|3|1984|339|4413|Sunday|1984Q3|N|N|N|2445883|2446064|2445545|2445820|N|N|N|N|N| +2445912|AAAAAAAAIFCFFCAA|1984-07-30|1014|4413|339|1984|1|7|30|3|1984|339|4413|Monday|1984Q3|N|N|N|2445883|2446064|2445546|2445821|N|N|N|N|N| +2445913|AAAAAAAAJFCFFCAA|1984-07-31|1014|4414|339|1984|2|7|31|3|1984|339|4414|Tuesday|1984Q3|N|N|N|2445883|2446064|2445547|2445822|N|N|N|N|N| +2445914|AAAAAAAAKFCFFCAA|1984-08-01|1015|4414|339|1984|3|8|1|3|1984|339|4414|Wednesday|1984Q3|N|N|N|2445914|2446126|2445548|2445823|N|N|N|N|N| +2445915|AAAAAAAALFCFFCAA|1984-08-02|1015|4414|339|1984|4|8|2|3|1984|339|4414|Thursday|1984Q3|N|N|N|2445914|2446126|2445549|2445824|N|N|N|N|N| +2445916|AAAAAAAAMFCFFCAA|1984-08-03|1015|4414|339|1984|5|8|3|3|1984|339|4414|Friday|1984Q3|N|Y|N|2445914|2446126|2445550|2445825|N|N|N|N|N| +2445917|AAAAAAAANFCFFCAA|1984-08-04|1015|4414|339|1984|6|8|4|3|1984|339|4414|Saturday|1984Q3|N|Y|N|2445914|2446126|2445551|2445826|N|N|N|N|N| +2445918|AAAAAAAAOFCFFCAA|1984-08-05|1015|4414|339|1984|0|8|5|3|1984|339|4414|Sunday|1984Q3|N|N|N|2445914|2446126|2445552|2445827|N|N|N|N|N| +2445919|AAAAAAAAPFCFFCAA|1984-08-06|1015|4414|339|1984|1|8|6|3|1984|339|4414|Monday|1984Q3|N|N|N|2445914|2446126|2445553|2445828|N|N|N|N|N| +2445920|AAAAAAAAAGCFFCAA|1984-08-07|1015|4415|339|1984|2|8|7|3|1984|339|4415|Tuesday|1984Q3|N|N|N|2445914|2446126|2445554|2445829|N|N|N|N|N| +2445921|AAAAAAAABGCFFCAA|1984-08-08|1015|4415|339|1984|3|8|8|3|1984|339|4415|Wednesday|1984Q3|N|N|N|2445914|2446126|2445555|2445830|N|N|N|N|N| +2445922|AAAAAAAACGCFFCAA|1984-08-09|1015|4415|339|1984|4|8|9|3|1984|339|4415|Thursday|1984Q3|N|N|N|2445914|2446126|2445556|2445831|N|N|N|N|N| +2445923|AAAAAAAADGCFFCAA|1984-08-10|1015|4415|339|1984|5|8|10|3|1984|339|4415|Friday|1984Q3|N|Y|N|2445914|2446126|2445557|2445832|N|N|N|N|N| +2445924|AAAAAAAAEGCFFCAA|1984-08-11|1015|4415|339|1984|6|8|11|3|1984|339|4415|Saturday|1984Q3|N|Y|N|2445914|2446126|2445558|2445833|N|N|N|N|N| +2445925|AAAAAAAAFGCFFCAA|1984-08-12|1015|4415|339|1984|0|8|12|3|1984|339|4415|Sunday|1984Q3|N|N|N|2445914|2446126|2445559|2445834|N|N|N|N|N| +2445926|AAAAAAAAGGCFFCAA|1984-08-13|1015|4415|339|1984|1|8|13|3|1984|339|4415|Monday|1984Q3|N|N|N|2445914|2446126|2445560|2445835|N|N|N|N|N| +2445927|AAAAAAAAHGCFFCAA|1984-08-14|1015|4416|339|1984|2|8|14|3|1984|339|4416|Tuesday|1984Q3|N|N|N|2445914|2446126|2445561|2445836|N|N|N|N|N| +2445928|AAAAAAAAIGCFFCAA|1984-08-15|1015|4416|339|1984|3|8|15|3|1984|339|4416|Wednesday|1984Q3|N|N|N|2445914|2446126|2445562|2445837|N|N|N|N|N| +2445929|AAAAAAAAJGCFFCAA|1984-08-16|1015|4416|339|1984|4|8|16|3|1984|339|4416|Thursday|1984Q3|N|N|N|2445914|2446126|2445563|2445838|N|N|N|N|N| +2445930|AAAAAAAAKGCFFCAA|1984-08-17|1015|4416|339|1984|5|8|17|3|1984|339|4416|Friday|1984Q3|N|Y|N|2445914|2446126|2445564|2445839|N|N|N|N|N| +2445931|AAAAAAAALGCFFCAA|1984-08-18|1015|4416|339|1984|6|8|18|3|1984|339|4416|Saturday|1984Q3|N|Y|N|2445914|2446126|2445565|2445840|N|N|N|N|N| +2445932|AAAAAAAAMGCFFCAA|1984-08-19|1015|4416|339|1984|0|8|19|3|1984|339|4416|Sunday|1984Q3|N|N|N|2445914|2446126|2445566|2445841|N|N|N|N|N| +2445933|AAAAAAAANGCFFCAA|1984-08-20|1015|4416|339|1984|1|8|20|3|1984|339|4416|Monday|1984Q3|N|N|N|2445914|2446126|2445567|2445842|N|N|N|N|N| +2445934|AAAAAAAAOGCFFCAA|1984-08-21|1015|4417|339|1984|2|8|21|3|1984|339|4417|Tuesday|1984Q3|N|N|N|2445914|2446126|2445568|2445843|N|N|N|N|N| +2445935|AAAAAAAAPGCFFCAA|1984-08-22|1015|4417|339|1984|3|8|22|3|1984|339|4417|Wednesday|1984Q3|N|N|N|2445914|2446126|2445569|2445844|N|N|N|N|N| +2445936|AAAAAAAAAHCFFCAA|1984-08-23|1015|4417|339|1984|4|8|23|3|1984|339|4417|Thursday|1984Q3|N|N|N|2445914|2446126|2445570|2445845|N|N|N|N|N| +2445937|AAAAAAAABHCFFCAA|1984-08-24|1015|4417|339|1984|5|8|24|3|1984|339|4417|Friday|1984Q3|N|Y|N|2445914|2446126|2445571|2445846|N|N|N|N|N| +2445938|AAAAAAAACHCFFCAA|1984-08-25|1015|4417|339|1984|6|8|25|3|1984|339|4417|Saturday|1984Q3|N|Y|N|2445914|2446126|2445572|2445847|N|N|N|N|N| +2445939|AAAAAAAADHCFFCAA|1984-08-26|1015|4417|339|1984|0|8|26|3|1984|339|4417|Sunday|1984Q3|N|N|N|2445914|2446126|2445573|2445848|N|N|N|N|N| +2445940|AAAAAAAAEHCFFCAA|1984-08-27|1015|4417|339|1984|1|8|27|3|1984|339|4417|Monday|1984Q3|N|N|N|2445914|2446126|2445574|2445849|N|N|N|N|N| +2445941|AAAAAAAAFHCFFCAA|1984-08-28|1015|4418|339|1984|2|8|28|3|1984|339|4418|Tuesday|1984Q3|N|N|N|2445914|2446126|2445575|2445850|N|N|N|N|N| +2445942|AAAAAAAAGHCFFCAA|1984-08-29|1015|4418|339|1984|3|8|29|3|1984|339|4418|Wednesday|1984Q3|N|N|N|2445914|2446126|2445576|2445851|N|N|N|N|N| +2445943|AAAAAAAAHHCFFCAA|1984-08-30|1015|4418|339|1984|4|8|30|3|1984|339|4418|Thursday|1984Q3|N|N|N|2445914|2446126|2445577|2445852|N|N|N|N|N| +2445944|AAAAAAAAIHCFFCAA|1984-08-31|1015|4418|339|1984|5|8|31|3|1984|339|4418|Friday|1984Q3|N|Y|N|2445914|2446126|2445578|2445853|N|N|N|N|N| +2445945|AAAAAAAAJHCFFCAA|1984-09-01|1016|4418|340|1984|6|9|1|3|1984|340|4418|Saturday|1984Q3|N|Y|N|2445945|2446188|2445579|2445854|N|N|N|N|N| +2445946|AAAAAAAAKHCFFCAA|1984-09-02|1016|4418|340|1984|0|9|2|3|1984|340|4418|Sunday|1984Q3|N|N|N|2445945|2446188|2445580|2445855|N|N|N|N|N| +2445947|AAAAAAAALHCFFCAA|1984-09-03|1016|4418|340|1984|1|9|3|3|1984|340|4418|Monday|1984Q3|N|N|N|2445945|2446188|2445581|2445856|N|N|N|N|N| +2445948|AAAAAAAAMHCFFCAA|1984-09-04|1016|4419|340|1984|2|9|4|3|1984|340|4419|Tuesday|1984Q3|N|N|N|2445945|2446188|2445582|2445857|N|N|N|N|N| +2445949|AAAAAAAANHCFFCAA|1984-09-05|1016|4419|340|1984|3|9|5|3|1984|340|4419|Wednesday|1984Q3|N|N|N|2445945|2446188|2445583|2445858|N|N|N|N|N| +2445950|AAAAAAAAOHCFFCAA|1984-09-06|1016|4419|340|1984|4|9|6|3|1984|340|4419|Thursday|1984Q3|N|N|N|2445945|2446188|2445584|2445859|N|N|N|N|N| +2445951|AAAAAAAAPHCFFCAA|1984-09-07|1016|4419|340|1984|5|9|7|3|1984|340|4419|Friday|1984Q3|N|Y|N|2445945|2446188|2445585|2445860|N|N|N|N|N| +2445952|AAAAAAAAAICFFCAA|1984-09-08|1016|4419|340|1984|6|9|8|3|1984|340|4419|Saturday|1984Q3|N|Y|N|2445945|2446188|2445586|2445861|N|N|N|N|N| +2445953|AAAAAAAABICFFCAA|1984-09-09|1016|4419|340|1984|0|9|9|3|1984|340|4419|Sunday|1984Q3|N|N|N|2445945|2446188|2445587|2445862|N|N|N|N|N| +2445954|AAAAAAAACICFFCAA|1984-09-10|1016|4419|340|1984|1|9|10|3|1984|340|4419|Monday|1984Q3|N|N|N|2445945|2446188|2445588|2445863|N|N|N|N|N| +2445955|AAAAAAAADICFFCAA|1984-09-11|1016|4420|340|1984|2|9|11|3|1984|340|4420|Tuesday|1984Q3|N|N|N|2445945|2446188|2445589|2445864|N|N|N|N|N| +2445956|AAAAAAAAEICFFCAA|1984-09-12|1016|4420|340|1984|3|9|12|3|1984|340|4420|Wednesday|1984Q3|N|N|N|2445945|2446188|2445590|2445865|N|N|N|N|N| +2445957|AAAAAAAAFICFFCAA|1984-09-13|1016|4420|340|1984|4|9|13|3|1984|340|4420|Thursday|1984Q3|N|N|N|2445945|2446188|2445591|2445866|N|N|N|N|N| +2445958|AAAAAAAAGICFFCAA|1984-09-14|1016|4420|340|1984|5|9|14|3|1984|340|4420|Friday|1984Q3|N|Y|N|2445945|2446188|2445592|2445867|N|N|N|N|N| +2445959|AAAAAAAAHICFFCAA|1984-09-15|1016|4420|340|1984|6|9|15|3|1984|340|4420|Saturday|1984Q3|N|Y|N|2445945|2446188|2445593|2445868|N|N|N|N|N| +2445960|AAAAAAAAIICFFCAA|1984-09-16|1016|4420|340|1984|0|9|16|3|1984|340|4420|Sunday|1984Q3|N|N|N|2445945|2446188|2445594|2445869|N|N|N|N|N| +2445961|AAAAAAAAJICFFCAA|1984-09-17|1016|4420|340|1984|1|9|17|3|1984|340|4420|Monday|1984Q3|N|N|N|2445945|2446188|2445595|2445870|N|N|N|N|N| +2445962|AAAAAAAAKICFFCAA|1984-09-18|1016|4421|340|1984|2|9|18|3|1984|340|4421|Tuesday|1984Q3|N|N|N|2445945|2446188|2445596|2445871|N|N|N|N|N| +2445963|AAAAAAAALICFFCAA|1984-09-19|1016|4421|340|1984|3|9|19|3|1984|340|4421|Wednesday|1984Q3|N|N|N|2445945|2446188|2445597|2445872|N|N|N|N|N| +2445964|AAAAAAAAMICFFCAA|1984-09-20|1016|4421|340|1984|4|9|20|3|1984|340|4421|Thursday|1984Q3|N|N|N|2445945|2446188|2445598|2445873|N|N|N|N|N| +2445965|AAAAAAAANICFFCAA|1984-09-21|1016|4421|340|1984|5|9|21|3|1984|340|4421|Friday|1984Q3|N|Y|N|2445945|2446188|2445599|2445874|N|N|N|N|N| +2445966|AAAAAAAAOICFFCAA|1984-09-22|1016|4421|340|1984|6|9|22|3|1984|340|4421|Saturday|1984Q3|N|Y|N|2445945|2446188|2445600|2445875|N|N|N|N|N| +2445967|AAAAAAAAPICFFCAA|1984-09-23|1016|4421|340|1984|0|9|23|3|1984|340|4421|Sunday|1984Q3|N|N|N|2445945|2446188|2445601|2445876|N|N|N|N|N| +2445968|AAAAAAAAAJCFFCAA|1984-09-24|1016|4421|340|1984|1|9|24|3|1984|340|4421|Monday|1984Q3|N|N|N|2445945|2446188|2445602|2445877|N|N|N|N|N| +2445969|AAAAAAAABJCFFCAA|1984-09-25|1016|4422|340|1984|2|9|25|3|1984|340|4422|Tuesday|1984Q3|N|N|N|2445945|2446188|2445603|2445878|N|N|N|N|N| +2445970|AAAAAAAACJCFFCAA|1984-09-26|1016|4422|340|1984|3|9|26|3|1984|340|4422|Wednesday|1984Q3|N|N|N|2445945|2446188|2445604|2445879|N|N|N|N|N| +2445971|AAAAAAAADJCFFCAA|1984-09-27|1016|4422|340|1984|4|9|27|3|1984|340|4422|Thursday|1984Q3|N|N|N|2445945|2446188|2445605|2445880|N|N|N|N|N| +2445972|AAAAAAAAEJCFFCAA|1984-09-28|1016|4422|340|1984|5|9|28|3|1984|340|4422|Friday|1984Q3|N|Y|N|2445945|2446188|2445606|2445881|N|N|N|N|N| +2445973|AAAAAAAAFJCFFCAA|1984-09-29|1016|4422|340|1984|6|9|29|3|1984|340|4422|Saturday|1984Q3|N|Y|N|2445945|2446188|2445607|2445882|N|N|N|N|N| +2445974|AAAAAAAAGJCFFCAA|1984-09-30|1016|4422|340|1984|0|9|30|3|1984|340|4422|Sunday|1984Q3|N|N|N|2445945|2446188|2445608|2445883|N|N|N|N|N| +2445975|AAAAAAAAHJCFFCAA|1984-10-01|1017|4422|340|1984|1|10|1|4|1984|340|4422|Monday|1984Q4|N|N|N|2445975|2446248|2445609|2445883|N|N|N|N|N| +2445976|AAAAAAAAIJCFFCAA|1984-10-02|1017|4423|340|1984|2|10|2|4|1984|340|4423|Tuesday|1984Q4|N|N|N|2445975|2446248|2445610|2445884|N|N|N|N|N| +2445977|AAAAAAAAJJCFFCAA|1984-10-03|1017|4423|340|1984|3|10|3|4|1984|340|4423|Wednesday|1984Q4|N|N|N|2445975|2446248|2445611|2445885|N|N|N|N|N| +2445978|AAAAAAAAKJCFFCAA|1984-10-04|1017|4423|340|1984|4|10|4|4|1984|340|4423|Thursday|1984Q4|N|N|N|2445975|2446248|2445612|2445886|N|N|N|N|N| +2445979|AAAAAAAALJCFFCAA|1984-10-05|1017|4423|340|1984|5|10|5|4|1984|340|4423|Friday|1984Q4|N|Y|N|2445975|2446248|2445613|2445887|N|N|N|N|N| +2445980|AAAAAAAAMJCFFCAA|1984-10-06|1017|4423|340|1984|6|10|6|4|1984|340|4423|Saturday|1984Q4|N|Y|N|2445975|2446248|2445614|2445888|N|N|N|N|N| +2445981|AAAAAAAANJCFFCAA|1984-10-07|1017|4423|340|1984|0|10|7|4|1984|340|4423|Sunday|1984Q4|N|N|N|2445975|2446248|2445615|2445889|N|N|N|N|N| +2445982|AAAAAAAAOJCFFCAA|1984-10-08|1017|4423|340|1984|1|10|8|4|1984|340|4423|Monday|1984Q4|N|N|N|2445975|2446248|2445616|2445890|N|N|N|N|N| +2445983|AAAAAAAAPJCFFCAA|1984-10-09|1017|4424|340|1984|2|10|9|4|1984|340|4424|Tuesday|1984Q4|N|N|N|2445975|2446248|2445617|2445891|N|N|N|N|N| +2445984|AAAAAAAAAKCFFCAA|1984-10-10|1017|4424|340|1984|3|10|10|4|1984|340|4424|Wednesday|1984Q4|N|N|N|2445975|2446248|2445618|2445892|N|N|N|N|N| +2445985|AAAAAAAABKCFFCAA|1984-10-11|1017|4424|340|1984|4|10|11|4|1984|340|4424|Thursday|1984Q4|N|N|N|2445975|2446248|2445619|2445893|N|N|N|N|N| +2445986|AAAAAAAACKCFFCAA|1984-10-12|1017|4424|340|1984|5|10|12|4|1984|340|4424|Friday|1984Q4|N|Y|N|2445975|2446248|2445620|2445894|N|N|N|N|N| +2445987|AAAAAAAADKCFFCAA|1984-10-13|1017|4424|340|1984|6|10|13|4|1984|340|4424|Saturday|1984Q4|N|Y|N|2445975|2446248|2445621|2445895|N|N|N|N|N| +2445988|AAAAAAAAEKCFFCAA|1984-10-14|1017|4424|340|1984|0|10|14|4|1984|340|4424|Sunday|1984Q4|N|N|N|2445975|2446248|2445622|2445896|N|N|N|N|N| +2445989|AAAAAAAAFKCFFCAA|1984-10-15|1017|4424|340|1984|1|10|15|4|1984|340|4424|Monday|1984Q4|N|N|N|2445975|2446248|2445623|2445897|N|N|N|N|N| +2445990|AAAAAAAAGKCFFCAA|1984-10-16|1017|4425|340|1984|2|10|16|4|1984|340|4425|Tuesday|1984Q4|N|N|N|2445975|2446248|2445624|2445898|N|N|N|N|N| +2445991|AAAAAAAAHKCFFCAA|1984-10-17|1017|4425|340|1984|3|10|17|4|1984|340|4425|Wednesday|1984Q4|N|N|N|2445975|2446248|2445625|2445899|N|N|N|N|N| +2445992|AAAAAAAAIKCFFCAA|1984-10-18|1017|4425|340|1984|4|10|18|4|1984|340|4425|Thursday|1984Q4|N|N|N|2445975|2446248|2445626|2445900|N|N|N|N|N| +2445993|AAAAAAAAJKCFFCAA|1984-10-19|1017|4425|340|1984|5|10|19|4|1984|340|4425|Friday|1984Q4|N|Y|N|2445975|2446248|2445627|2445901|N|N|N|N|N| +2445994|AAAAAAAAKKCFFCAA|1984-10-20|1017|4425|340|1984|6|10|20|4|1984|340|4425|Saturday|1984Q4|N|Y|N|2445975|2446248|2445628|2445902|N|N|N|N|N| +2445995|AAAAAAAALKCFFCAA|1984-10-21|1017|4425|340|1984|0|10|21|4|1984|340|4425|Sunday|1984Q4|N|N|N|2445975|2446248|2445629|2445903|N|N|N|N|N| +2445996|AAAAAAAAMKCFFCAA|1984-10-22|1017|4425|340|1984|1|10|22|4|1984|340|4425|Monday|1984Q4|N|N|N|2445975|2446248|2445630|2445904|N|N|N|N|N| +2445997|AAAAAAAANKCFFCAA|1984-10-23|1017|4426|340|1984|2|10|23|4|1984|340|4426|Tuesday|1984Q4|N|N|N|2445975|2446248|2445631|2445905|N|N|N|N|N| +2445998|AAAAAAAAOKCFFCAA|1984-10-24|1017|4426|340|1984|3|10|24|4|1984|340|4426|Wednesday|1984Q4|N|N|N|2445975|2446248|2445632|2445906|N|N|N|N|N| +2445999|AAAAAAAAPKCFFCAA|1984-10-25|1017|4426|340|1984|4|10|25|4|1984|340|4426|Thursday|1984Q4|N|N|N|2445975|2446248|2445633|2445907|N|N|N|N|N| +2446000|AAAAAAAAALCFFCAA|1984-10-26|1017|4426|340|1984|5|10|26|4|1984|340|4426|Friday|1984Q4|N|Y|N|2445975|2446248|2445634|2445908|N|N|N|N|N| +2446001|AAAAAAAABLCFFCAA|1984-10-27|1017|4426|340|1984|6|10|27|4|1984|340|4426|Saturday|1984Q4|N|Y|N|2445975|2446248|2445635|2445909|N|N|N|N|N| +2446002|AAAAAAAACLCFFCAA|1984-10-28|1017|4426|340|1984|0|10|28|4|1984|340|4426|Sunday|1984Q4|N|N|N|2445975|2446248|2445636|2445910|N|N|N|N|N| +2446003|AAAAAAAADLCFFCAA|1984-10-29|1017|4426|340|1984|1|10|29|4|1984|340|4426|Monday|1984Q4|N|N|N|2445975|2446248|2445637|2445911|N|N|N|N|N| +2446004|AAAAAAAAELCFFCAA|1984-10-30|1017|4427|340|1984|2|10|30|4|1984|340|4427|Tuesday|1984Q4|N|N|N|2445975|2446248|2445638|2445912|N|N|N|N|N| +2446005|AAAAAAAAFLCFFCAA|1984-10-31|1017|4427|340|1984|3|10|31|4|1984|340|4427|Wednesday|1984Q4|N|N|N|2445975|2446248|2445639|2445913|N|N|N|N|N| +2446006|AAAAAAAAGLCFFCAA|1984-11-01|1018|4427|340|1984|4|11|1|4|1984|340|4427|Thursday|1984Q4|N|N|N|2446006|2446310|2445640|2445914|N|N|N|N|N| +2446007|AAAAAAAAHLCFFCAA|1984-11-02|1018|4427|340|1984|5|11|2|4|1984|340|4427|Friday|1984Q4|N|Y|N|2446006|2446310|2445641|2445915|N|N|N|N|N| +2446008|AAAAAAAAILCFFCAA|1984-11-03|1018|4427|340|1984|6|11|3|4|1984|340|4427|Saturday|1984Q4|N|Y|N|2446006|2446310|2445642|2445916|N|N|N|N|N| +2446009|AAAAAAAAJLCFFCAA|1984-11-04|1018|4427|340|1984|0|11|4|4|1984|340|4427|Sunday|1984Q4|N|N|N|2446006|2446310|2445643|2445917|N|N|N|N|N| +2446010|AAAAAAAAKLCFFCAA|1984-11-05|1018|4427|340|1984|1|11|5|4|1984|340|4427|Monday|1984Q4|N|N|N|2446006|2446310|2445644|2445918|N|N|N|N|N| +2446011|AAAAAAAALLCFFCAA|1984-11-06|1018|4428|340|1984|2|11|6|4|1984|340|4428|Tuesday|1984Q4|N|N|N|2446006|2446310|2445645|2445919|N|N|N|N|N| +2446012|AAAAAAAAMLCFFCAA|1984-11-07|1018|4428|340|1984|3|11|7|4|1984|340|4428|Wednesday|1984Q4|N|N|N|2446006|2446310|2445646|2445920|N|N|N|N|N| +2446013|AAAAAAAANLCFFCAA|1984-11-08|1018|4428|340|1984|4|11|8|4|1984|340|4428|Thursday|1984Q4|N|N|N|2446006|2446310|2445647|2445921|N|N|N|N|N| +2446014|AAAAAAAAOLCFFCAA|1984-11-09|1018|4428|340|1984|5|11|9|4|1984|340|4428|Friday|1984Q4|N|Y|N|2446006|2446310|2445648|2445922|N|N|N|N|N| +2446015|AAAAAAAAPLCFFCAA|1984-11-10|1018|4428|340|1984|6|11|10|4|1984|340|4428|Saturday|1984Q4|N|Y|N|2446006|2446310|2445649|2445923|N|N|N|N|N| +2446016|AAAAAAAAAMCFFCAA|1984-11-11|1018|4428|340|1984|0|11|11|4|1984|340|4428|Sunday|1984Q4|N|N|N|2446006|2446310|2445650|2445924|N|N|N|N|N| +2446017|AAAAAAAABMCFFCAA|1984-11-12|1018|4428|340|1984|1|11|12|4|1984|340|4428|Monday|1984Q4|N|N|N|2446006|2446310|2445651|2445925|N|N|N|N|N| +2446018|AAAAAAAACMCFFCAA|1984-11-13|1018|4429|340|1984|2|11|13|4|1984|340|4429|Tuesday|1984Q4|N|N|N|2446006|2446310|2445652|2445926|N|N|N|N|N| +2446019|AAAAAAAADMCFFCAA|1984-11-14|1018|4429|340|1984|3|11|14|4|1984|340|4429|Wednesday|1984Q4|N|N|N|2446006|2446310|2445653|2445927|N|N|N|N|N| +2446020|AAAAAAAAEMCFFCAA|1984-11-15|1018|4429|340|1984|4|11|15|4|1984|340|4429|Thursday|1984Q4|N|N|N|2446006|2446310|2445654|2445928|N|N|N|N|N| +2446021|AAAAAAAAFMCFFCAA|1984-11-16|1018|4429|340|1984|5|11|16|4|1984|340|4429|Friday|1984Q4|N|Y|N|2446006|2446310|2445655|2445929|N|N|N|N|N| +2446022|AAAAAAAAGMCFFCAA|1984-11-17|1018|4429|340|1984|6|11|17|4|1984|340|4429|Saturday|1984Q4|N|Y|N|2446006|2446310|2445656|2445930|N|N|N|N|N| +2446023|AAAAAAAAHMCFFCAA|1984-11-18|1018|4429|340|1984|0|11|18|4|1984|340|4429|Sunday|1984Q4|N|N|N|2446006|2446310|2445657|2445931|N|N|N|N|N| +2446024|AAAAAAAAIMCFFCAA|1984-11-19|1018|4429|340|1984|1|11|19|4|1984|340|4429|Monday|1984Q4|N|N|N|2446006|2446310|2445658|2445932|N|N|N|N|N| +2446025|AAAAAAAAJMCFFCAA|1984-11-20|1018|4430|340|1984|2|11|20|4|1984|340|4430|Tuesday|1984Q4|N|N|N|2446006|2446310|2445659|2445933|N|N|N|N|N| +2446026|AAAAAAAAKMCFFCAA|1984-11-21|1018|4430|340|1984|3|11|21|4|1984|340|4430|Wednesday|1984Q4|N|N|N|2446006|2446310|2445660|2445934|N|N|N|N|N| +2446027|AAAAAAAALMCFFCAA|1984-11-22|1018|4430|340|1984|4|11|22|4|1984|340|4430|Thursday|1984Q4|N|N|N|2446006|2446310|2445661|2445935|N|N|N|N|N| +2446028|AAAAAAAAMMCFFCAA|1984-11-23|1018|4430|340|1984|5|11|23|4|1984|340|4430|Friday|1984Q4|N|Y|N|2446006|2446310|2445662|2445936|N|N|N|N|N| +2446029|AAAAAAAANMCFFCAA|1984-11-24|1018|4430|340|1984|6|11|24|4|1984|340|4430|Saturday|1984Q4|N|Y|N|2446006|2446310|2445663|2445937|N|N|N|N|N| +2446030|AAAAAAAAOMCFFCAA|1984-11-25|1018|4430|340|1984|0|11|25|4|1984|340|4430|Sunday|1984Q4|N|N|N|2446006|2446310|2445664|2445938|N|N|N|N|N| +2446031|AAAAAAAAPMCFFCAA|1984-11-26|1018|4430|340|1984|1|11|26|4|1984|340|4430|Monday|1984Q4|N|N|N|2446006|2446310|2445665|2445939|N|N|N|N|N| +2446032|AAAAAAAAANCFFCAA|1984-11-27|1018|4431|340|1984|2|11|27|4|1984|340|4431|Tuesday|1984Q4|N|N|N|2446006|2446310|2445666|2445940|N|N|N|N|N| +2446033|AAAAAAAABNCFFCAA|1984-11-28|1018|4431|340|1984|3|11|28|4|1984|340|4431|Wednesday|1984Q4|N|N|N|2446006|2446310|2445667|2445941|N|N|N|N|N| +2446034|AAAAAAAACNCFFCAA|1984-11-29|1018|4431|340|1984|4|11|29|4|1984|340|4431|Thursday|1984Q4|N|N|N|2446006|2446310|2445668|2445942|N|N|N|N|N| +2446035|AAAAAAAADNCFFCAA|1984-11-30|1018|4431|340|1984|5|11|30|4|1984|340|4431|Friday|1984Q4|N|Y|N|2446006|2446310|2445669|2445943|N|N|N|N|N| +2446036|AAAAAAAAENCFFCAA|1984-12-01|1019|4431|341|1984|6|12|1|4|1984|341|4431|Saturday|1984Q4|N|Y|N|2446036|2446370|2445670|2445944|N|N|N|N|N| +2446037|AAAAAAAAFNCFFCAA|1984-12-02|1019|4431|341|1984|0|12|2|4|1984|341|4431|Sunday|1984Q4|N|N|N|2446036|2446370|2445671|2445945|N|N|N|N|N| +2446038|AAAAAAAAGNCFFCAA|1984-12-03|1019|4431|341|1984|1|12|3|4|1984|341|4431|Monday|1984Q4|N|N|N|2446036|2446370|2445672|2445946|N|N|N|N|N| +2446039|AAAAAAAAHNCFFCAA|1984-12-04|1019|4432|341|1984|2|12|4|4|1984|341|4432|Tuesday|1984Q4|N|N|N|2446036|2446370|2445673|2445947|N|N|N|N|N| +2446040|AAAAAAAAINCFFCAA|1984-12-05|1019|4432|341|1984|3|12|5|4|1984|341|4432|Wednesday|1984Q4|N|N|N|2446036|2446370|2445674|2445948|N|N|N|N|N| +2446041|AAAAAAAAJNCFFCAA|1984-12-06|1019|4432|341|1984|4|12|6|4|1984|341|4432|Thursday|1984Q4|N|N|N|2446036|2446370|2445675|2445949|N|N|N|N|N| +2446042|AAAAAAAAKNCFFCAA|1984-12-07|1019|4432|341|1984|5|12|7|4|1984|341|4432|Friday|1984Q4|N|Y|N|2446036|2446370|2445676|2445950|N|N|N|N|N| +2446043|AAAAAAAALNCFFCAA|1984-12-08|1019|4432|341|1984|6|12|8|4|1984|341|4432|Saturday|1984Q4|N|Y|N|2446036|2446370|2445677|2445951|N|N|N|N|N| +2446044|AAAAAAAAMNCFFCAA|1984-12-09|1019|4432|341|1984|0|12|9|4|1984|341|4432|Sunday|1984Q4|N|N|N|2446036|2446370|2445678|2445952|N|N|N|N|N| +2446045|AAAAAAAANNCFFCAA|1984-12-10|1019|4432|341|1984|1|12|10|4|1984|341|4432|Monday|1984Q4|N|N|N|2446036|2446370|2445679|2445953|N|N|N|N|N| +2446046|AAAAAAAAONCFFCAA|1984-12-11|1019|4433|341|1984|2|12|11|4|1984|341|4433|Tuesday|1984Q4|N|N|N|2446036|2446370|2445680|2445954|N|N|N|N|N| +2446047|AAAAAAAAPNCFFCAA|1984-12-12|1019|4433|341|1984|3|12|12|4|1984|341|4433|Wednesday|1984Q4|N|N|N|2446036|2446370|2445681|2445955|N|N|N|N|N| +2446048|AAAAAAAAAOCFFCAA|1984-12-13|1019|4433|341|1984|4|12|13|4|1984|341|4433|Thursday|1984Q4|N|N|N|2446036|2446370|2445682|2445956|N|N|N|N|N| +2446049|AAAAAAAABOCFFCAA|1984-12-14|1019|4433|341|1984|5|12|14|4|1984|341|4433|Friday|1984Q4|N|Y|N|2446036|2446370|2445683|2445957|N|N|N|N|N| +2446050|AAAAAAAACOCFFCAA|1984-12-15|1019|4433|341|1984|6|12|15|4|1984|341|4433|Saturday|1984Q4|N|Y|N|2446036|2446370|2445684|2445958|N|N|N|N|N| +2446051|AAAAAAAADOCFFCAA|1984-12-16|1019|4433|341|1984|0|12|16|4|1984|341|4433|Sunday|1984Q4|N|N|N|2446036|2446370|2445685|2445959|N|N|N|N|N| +2446052|AAAAAAAAEOCFFCAA|1984-12-17|1019|4433|341|1984|1|12|17|4|1984|341|4433|Monday|1984Q4|N|N|N|2446036|2446370|2445686|2445960|N|N|N|N|N| +2446053|AAAAAAAAFOCFFCAA|1984-12-18|1019|4434|341|1984|2|12|18|4|1984|341|4434|Tuesday|1984Q4|N|N|N|2446036|2446370|2445687|2445961|N|N|N|N|N| +2446054|AAAAAAAAGOCFFCAA|1984-12-19|1019|4434|341|1984|3|12|19|4|1984|341|4434|Wednesday|1984Q4|N|N|N|2446036|2446370|2445688|2445962|N|N|N|N|N| +2446055|AAAAAAAAHOCFFCAA|1984-12-20|1019|4434|341|1984|4|12|20|4|1984|341|4434|Thursday|1984Q4|N|N|N|2446036|2446370|2445689|2445963|N|N|N|N|N| +2446056|AAAAAAAAIOCFFCAA|1984-12-21|1019|4434|341|1984|5|12|21|4|1984|341|4434|Friday|1984Q4|N|Y|N|2446036|2446370|2445690|2445964|N|N|N|N|N| +2446057|AAAAAAAAJOCFFCAA|1984-12-22|1019|4434|341|1984|6|12|22|4|1984|341|4434|Saturday|1984Q4|N|Y|N|2446036|2446370|2445691|2445965|N|N|N|N|N| +2446058|AAAAAAAAKOCFFCAA|1984-12-23|1019|4434|341|1984|0|12|23|4|1984|341|4434|Sunday|1984Q4|N|N|N|2446036|2446370|2445692|2445966|N|N|N|N|N| +2446059|AAAAAAAALOCFFCAA|1984-12-24|1019|4434|341|1984|1|12|24|4|1984|341|4434|Monday|1984Q4|N|N|N|2446036|2446370|2445693|2445967|N|N|N|N|N| +2446060|AAAAAAAAMOCFFCAA|1984-12-25|1019|4435|341|1984|2|12|25|4|1984|341|4435|Tuesday|1984Q4|Y|N|N|2446036|2446370|2445694|2445968|N|N|N|N|N| +2446061|AAAAAAAANOCFFCAA|1984-12-26|1019|4435|341|1984|3|12|26|4|1984|341|4435|Wednesday|1984Q4|N|N|Y|2446036|2446370|2445695|2445969|N|N|N|N|N| +2446062|AAAAAAAAOOCFFCAA|1984-12-27|1019|4435|341|1984|4|12|27|4|1984|341|4435|Thursday|1984Q4|N|N|N|2446036|2446370|2445696|2445970|N|N|N|N|N| +2446063|AAAAAAAAPOCFFCAA|1984-12-28|1019|4435|341|1984|5|12|28|4|1984|341|4435|Friday|1984Q4|N|Y|N|2446036|2446370|2445697|2445971|N|N|N|N|N| +2446064|AAAAAAAAAPCFFCAA|1984-12-29|1019|4435|341|1984|6|12|29|4|1984|341|4435|Saturday|1984Q4|N|Y|N|2446036|2446370|2445698|2445972|N|N|N|N|N| +2446065|AAAAAAAABPCFFCAA|1984-12-30|1019|4435|341|1984|0|12|30|4|1984|341|4435|Sunday|1984Q4|N|N|N|2446036|2446370|2445699|2445973|N|N|N|N|N| +2446066|AAAAAAAACPCFFCAA|1984-12-31|1019|4435|341|1984|1|12|31|4|1984|341|4435|Monday|1984Q4|Y|N|N|2446036|2446370|2445700|2445974|N|N|N|N|N| +2446067|AAAAAAAADPCFFCAA|1985-01-01|1020|4436|341|1985|2|1|1|1|1985|341|4436|Tuesday|1985Q1|Y|N|Y|2446067|2446066|2445701|2445975|N|N|N|N|N| +2446068|AAAAAAAAEPCFFCAA|1985-01-02|1020|4436|341|1985|3|1|2|1|1985|341|4436|Wednesday|1985Q1|N|N|Y|2446067|2446066|2445702|2445976|N|N|N|N|N| +2446069|AAAAAAAAFPCFFCAA|1985-01-03|1020|4436|341|1985|4|1|3|1|1985|341|4436|Thursday|1985Q1|N|N|N|2446067|2446066|2445703|2445977|N|N|N|N|N| +2446070|AAAAAAAAGPCFFCAA|1985-01-04|1020|4436|341|1985|5|1|4|1|1985|341|4436|Friday|1985Q1|N|Y|N|2446067|2446066|2445704|2445978|N|N|N|N|N| +2446071|AAAAAAAAHPCFFCAA|1985-01-05|1020|4436|341|1985|6|1|5|1|1985|341|4436|Saturday|1985Q1|N|Y|N|2446067|2446066|2445705|2445979|N|N|N|N|N| +2446072|AAAAAAAAIPCFFCAA|1985-01-06|1020|4436|341|1985|0|1|6|1|1985|341|4436|Sunday|1985Q1|N|N|N|2446067|2446066|2445706|2445980|N|N|N|N|N| +2446073|AAAAAAAAJPCFFCAA|1985-01-07|1020|4436|341|1985|1|1|7|1|1985|341|4436|Monday|1985Q1|N|N|N|2446067|2446066|2445707|2445981|N|N|N|N|N| +2446074|AAAAAAAAKPCFFCAA|1985-01-08|1020|4437|341|1985|2|1|8|1|1985|341|4437|Tuesday|1985Q1|N|N|N|2446067|2446066|2445708|2445982|N|N|N|N|N| +2446075|AAAAAAAALPCFFCAA|1985-01-09|1020|4437|341|1985|3|1|9|1|1985|341|4437|Wednesday|1985Q1|N|N|N|2446067|2446066|2445709|2445983|N|N|N|N|N| +2446076|AAAAAAAAMPCFFCAA|1985-01-10|1020|4437|341|1985|4|1|10|1|1985|341|4437|Thursday|1985Q1|N|N|N|2446067|2446066|2445710|2445984|N|N|N|N|N| +2446077|AAAAAAAANPCFFCAA|1985-01-11|1020|4437|341|1985|5|1|11|1|1985|341|4437|Friday|1985Q1|N|Y|N|2446067|2446066|2445711|2445985|N|N|N|N|N| +2446078|AAAAAAAAOPCFFCAA|1985-01-12|1020|4437|341|1985|6|1|12|1|1985|341|4437|Saturday|1985Q1|N|Y|N|2446067|2446066|2445712|2445986|N|N|N|N|N| +2446079|AAAAAAAAPPCFFCAA|1985-01-13|1020|4437|341|1985|0|1|13|1|1985|341|4437|Sunday|1985Q1|N|N|N|2446067|2446066|2445713|2445987|N|N|N|N|N| +2446080|AAAAAAAAAADFFCAA|1985-01-14|1020|4437|341|1985|1|1|14|1|1985|341|4437|Monday|1985Q1|N|N|N|2446067|2446066|2445714|2445988|N|N|N|N|N| +2446081|AAAAAAAABADFFCAA|1985-01-15|1020|4438|341|1985|2|1|15|1|1985|341|4438|Tuesday|1985Q1|N|N|N|2446067|2446066|2445715|2445989|N|N|N|N|N| +2446082|AAAAAAAACADFFCAA|1985-01-16|1020|4438|341|1985|3|1|16|1|1985|341|4438|Wednesday|1985Q1|N|N|N|2446067|2446066|2445716|2445990|N|N|N|N|N| +2446083|AAAAAAAADADFFCAA|1985-01-17|1020|4438|341|1985|4|1|17|1|1985|341|4438|Thursday|1985Q1|N|N|N|2446067|2446066|2445717|2445991|N|N|N|N|N| +2446084|AAAAAAAAEADFFCAA|1985-01-18|1020|4438|341|1985|5|1|18|1|1985|341|4438|Friday|1985Q1|N|Y|N|2446067|2446066|2445718|2445992|N|N|N|N|N| +2446085|AAAAAAAAFADFFCAA|1985-01-19|1020|4438|341|1985|6|1|19|1|1985|341|4438|Saturday|1985Q1|N|Y|N|2446067|2446066|2445719|2445993|N|N|N|N|N| +2446086|AAAAAAAAGADFFCAA|1985-01-20|1020|4438|341|1985|0|1|20|1|1985|341|4438|Sunday|1985Q1|N|N|N|2446067|2446066|2445720|2445994|N|N|N|N|N| +2446087|AAAAAAAAHADFFCAA|1985-01-21|1020|4438|341|1985|1|1|21|1|1985|341|4438|Monday|1985Q1|N|N|N|2446067|2446066|2445721|2445995|N|N|N|N|N| +2446088|AAAAAAAAIADFFCAA|1985-01-22|1020|4439|341|1985|2|1|22|1|1985|341|4439|Tuesday|1985Q1|N|N|N|2446067|2446066|2445722|2445996|N|N|N|N|N| +2446089|AAAAAAAAJADFFCAA|1985-01-23|1020|4439|341|1985|3|1|23|1|1985|341|4439|Wednesday|1985Q1|N|N|N|2446067|2446066|2445723|2445997|N|N|N|N|N| +2446090|AAAAAAAAKADFFCAA|1985-01-24|1020|4439|341|1985|4|1|24|1|1985|341|4439|Thursday|1985Q1|N|N|N|2446067|2446066|2445724|2445998|N|N|N|N|N| +2446091|AAAAAAAALADFFCAA|1985-01-25|1020|4439|341|1985|5|1|25|1|1985|341|4439|Friday|1985Q1|N|Y|N|2446067|2446066|2445725|2445999|N|N|N|N|N| +2446092|AAAAAAAAMADFFCAA|1985-01-26|1020|4439|341|1985|6|1|26|1|1985|341|4439|Saturday|1985Q1|N|Y|N|2446067|2446066|2445726|2446000|N|N|N|N|N| +2446093|AAAAAAAANADFFCAA|1985-01-27|1020|4439|341|1985|0|1|27|1|1985|341|4439|Sunday|1985Q1|N|N|N|2446067|2446066|2445727|2446001|N|N|N|N|N| +2446094|AAAAAAAAOADFFCAA|1985-01-28|1020|4439|341|1985|1|1|28|1|1985|341|4439|Monday|1985Q1|N|N|N|2446067|2446066|2445728|2446002|N|N|N|N|N| +2446095|AAAAAAAAPADFFCAA|1985-01-29|1020|4440|341|1985|2|1|29|1|1985|341|4440|Tuesday|1985Q1|N|N|N|2446067|2446066|2445729|2446003|N|N|N|N|N| +2446096|AAAAAAAAABDFFCAA|1985-01-30|1020|4440|341|1985|3|1|30|1|1985|341|4440|Wednesday|1985Q1|N|N|N|2446067|2446066|2445730|2446004|N|N|N|N|N| +2446097|AAAAAAAABBDFFCAA|1985-01-31|1020|4440|341|1985|4|1|31|1|1985|341|4440|Thursday|1985Q1|N|N|N|2446067|2446066|2445731|2446005|N|N|N|N|N| +2446098|AAAAAAAACBDFFCAA|1985-02-01|1021|4440|341|1985|5|2|1|1|1985|341|4440|Friday|1985Q1|N|Y|N|2446098|2446128|2445732|2446006|N|N|N|N|N| +2446099|AAAAAAAADBDFFCAA|1985-02-02|1021|4440|341|1985|6|2|2|1|1985|341|4440|Saturday|1985Q1|N|Y|N|2446098|2446128|2445733|2446007|N|N|N|N|N| +2446100|AAAAAAAAEBDFFCAA|1985-02-03|1021|4440|341|1985|0|2|3|1|1985|341|4440|Sunday|1985Q1|N|N|N|2446098|2446128|2445734|2446008|N|N|N|N|N| +2446101|AAAAAAAAFBDFFCAA|1985-02-04|1021|4440|341|1985|1|2|4|1|1985|341|4440|Monday|1985Q1|N|N|N|2446098|2446128|2445735|2446009|N|N|N|N|N| +2446102|AAAAAAAAGBDFFCAA|1985-02-05|1021|4441|341|1985|2|2|5|1|1985|341|4441|Tuesday|1985Q1|N|N|N|2446098|2446128|2445736|2446010|N|N|N|N|N| +2446103|AAAAAAAAHBDFFCAA|1985-02-06|1021|4441|341|1985|3|2|6|1|1985|341|4441|Wednesday|1985Q1|N|N|N|2446098|2446128|2445737|2446011|N|N|N|N|N| +2446104|AAAAAAAAIBDFFCAA|1985-02-07|1021|4441|341|1985|4|2|7|1|1985|341|4441|Thursday|1985Q1|N|N|N|2446098|2446128|2445738|2446012|N|N|N|N|N| +2446105|AAAAAAAAJBDFFCAA|1985-02-08|1021|4441|341|1985|5|2|8|1|1985|341|4441|Friday|1985Q1|N|Y|N|2446098|2446128|2445739|2446013|N|N|N|N|N| +2446106|AAAAAAAAKBDFFCAA|1985-02-09|1021|4441|341|1985|6|2|9|1|1985|341|4441|Saturday|1985Q1|N|Y|N|2446098|2446128|2445740|2446014|N|N|N|N|N| +2446107|AAAAAAAALBDFFCAA|1985-02-10|1021|4441|341|1985|0|2|10|1|1985|341|4441|Sunday|1985Q1|N|N|N|2446098|2446128|2445741|2446015|N|N|N|N|N| +2446108|AAAAAAAAMBDFFCAA|1985-02-11|1021|4441|341|1985|1|2|11|1|1985|341|4441|Monday|1985Q1|N|N|N|2446098|2446128|2445742|2446016|N|N|N|N|N| +2446109|AAAAAAAANBDFFCAA|1985-02-12|1021|4442|341|1985|2|2|12|1|1985|341|4442|Tuesday|1985Q1|N|N|N|2446098|2446128|2445743|2446017|N|N|N|N|N| +2446110|AAAAAAAAOBDFFCAA|1985-02-13|1021|4442|341|1985|3|2|13|1|1985|341|4442|Wednesday|1985Q1|N|N|N|2446098|2446128|2445744|2446018|N|N|N|N|N| +2446111|AAAAAAAAPBDFFCAA|1985-02-14|1021|4442|341|1985|4|2|14|1|1985|341|4442|Thursday|1985Q1|N|N|N|2446098|2446128|2445745|2446019|N|N|N|N|N| +2446112|AAAAAAAAACDFFCAA|1985-02-15|1021|4442|341|1985|5|2|15|1|1985|341|4442|Friday|1985Q1|N|Y|N|2446098|2446128|2445746|2446020|N|N|N|N|N| +2446113|AAAAAAAABCDFFCAA|1985-02-16|1021|4442|341|1985|6|2|16|1|1985|341|4442|Saturday|1985Q1|N|Y|N|2446098|2446128|2445747|2446021|N|N|N|N|N| +2446114|AAAAAAAACCDFFCAA|1985-02-17|1021|4442|341|1985|0|2|17|1|1985|341|4442|Sunday|1985Q1|N|N|N|2446098|2446128|2445748|2446022|N|N|N|N|N| +2446115|AAAAAAAADCDFFCAA|1985-02-18|1021|4442|341|1985|1|2|18|1|1985|341|4442|Monday|1985Q1|N|N|N|2446098|2446128|2445749|2446023|N|N|N|N|N| +2446116|AAAAAAAAECDFFCAA|1985-02-19|1021|4443|341|1985|2|2|19|1|1985|341|4443|Tuesday|1985Q1|N|N|N|2446098|2446128|2445750|2446024|N|N|N|N|N| +2446117|AAAAAAAAFCDFFCAA|1985-02-20|1021|4443|341|1985|3|2|20|1|1985|341|4443|Wednesday|1985Q1|N|N|N|2446098|2446128|2445751|2446025|N|N|N|N|N| +2446118|AAAAAAAAGCDFFCAA|1985-02-21|1021|4443|341|1985|4|2|21|1|1985|341|4443|Thursday|1985Q1|N|N|N|2446098|2446128|2445752|2446026|N|N|N|N|N| +2446119|AAAAAAAAHCDFFCAA|1985-02-22|1021|4443|341|1985|5|2|22|1|1985|341|4443|Friday|1985Q1|N|Y|N|2446098|2446128|2445753|2446027|N|N|N|N|N| +2446120|AAAAAAAAICDFFCAA|1985-02-23|1021|4443|341|1985|6|2|23|1|1985|341|4443|Saturday|1985Q1|N|Y|N|2446098|2446128|2445754|2446028|N|N|N|N|N| +2446121|AAAAAAAAJCDFFCAA|1985-02-24|1021|4443|341|1985|0|2|24|1|1985|341|4443|Sunday|1985Q1|N|N|N|2446098|2446128|2445755|2446029|N|N|N|N|N| +2446122|AAAAAAAAKCDFFCAA|1985-02-25|1021|4443|341|1985|1|2|25|1|1985|341|4443|Monday|1985Q1|N|N|N|2446098|2446128|2445756|2446030|N|N|N|N|N| +2446123|AAAAAAAALCDFFCAA|1985-02-26|1021|4444|341|1985|2|2|26|1|1985|341|4444|Tuesday|1985Q1|N|N|N|2446098|2446128|2445757|2446031|N|N|N|N|N| +2446124|AAAAAAAAMCDFFCAA|1985-02-27|1021|4444|341|1985|3|2|27|1|1985|341|4444|Wednesday|1985Q1|N|N|N|2446098|2446128|2445758|2446032|N|N|N|N|N| +2446125|AAAAAAAANCDFFCAA|1985-02-28|1021|4444|341|1985|4|2|28|1|1985|341|4444|Thursday|1985Q1|N|N|N|2446098|2446128|2445759|2446033|N|N|N|N|N| +2446126|AAAAAAAAOCDFFCAA|1985-03-01|1022|4444|342|1985|5|3|1|1|1985|342|4444|Friday|1985Q1|N|Y|N|2446126|2446184|2445761|2446034|N|N|N|N|N| +2446127|AAAAAAAAPCDFFCAA|1985-03-02|1022|4444|342|1985|6|3|2|1|1985|342|4444|Saturday|1985Q1|N|Y|N|2446126|2446184|2445762|2446035|N|N|N|N|N| +2446128|AAAAAAAAADDFFCAA|1985-03-03|1022|4444|342|1985|0|3|3|1|1985|342|4444|Sunday|1985Q1|N|N|N|2446126|2446184|2445763|2446036|N|N|N|N|N| +2446129|AAAAAAAABDDFFCAA|1985-03-04|1022|4444|342|1985|1|3|4|1|1985|342|4444|Monday|1985Q1|N|N|N|2446126|2446184|2445764|2446037|N|N|N|N|N| +2446130|AAAAAAAACDDFFCAA|1985-03-05|1022|4445|342|1985|2|3|5|1|1985|342|4445|Tuesday|1985Q1|N|N|N|2446126|2446184|2445765|2446038|N|N|N|N|N| +2446131|AAAAAAAADDDFFCAA|1985-03-06|1022|4445|342|1985|3|3|6|1|1985|342|4445|Wednesday|1985Q1|N|N|N|2446126|2446184|2445766|2446039|N|N|N|N|N| +2446132|AAAAAAAAEDDFFCAA|1985-03-07|1022|4445|342|1985|4|3|7|1|1985|342|4445|Thursday|1985Q1|N|N|N|2446126|2446184|2445767|2446040|N|N|N|N|N| +2446133|AAAAAAAAFDDFFCAA|1985-03-08|1022|4445|342|1985|5|3|8|1|1985|342|4445|Friday|1985Q1|N|Y|N|2446126|2446184|2445768|2446041|N|N|N|N|N| +2446134|AAAAAAAAGDDFFCAA|1985-03-09|1022|4445|342|1985|6|3|9|1|1985|342|4445|Saturday|1985Q1|N|Y|N|2446126|2446184|2445769|2446042|N|N|N|N|N| +2446135|AAAAAAAAHDDFFCAA|1985-03-10|1022|4445|342|1985|0|3|10|1|1985|342|4445|Sunday|1985Q1|N|N|N|2446126|2446184|2445770|2446043|N|N|N|N|N| +2446136|AAAAAAAAIDDFFCAA|1985-03-11|1022|4445|342|1985|1|3|11|1|1985|342|4445|Monday|1985Q1|N|N|N|2446126|2446184|2445771|2446044|N|N|N|N|N| +2446137|AAAAAAAAJDDFFCAA|1985-03-12|1022|4446|342|1985|2|3|12|1|1985|342|4446|Tuesday|1985Q1|N|N|N|2446126|2446184|2445772|2446045|N|N|N|N|N| +2446138|AAAAAAAAKDDFFCAA|1985-03-13|1022|4446|342|1985|3|3|13|1|1985|342|4446|Wednesday|1985Q1|N|N|N|2446126|2446184|2445773|2446046|N|N|N|N|N| +2446139|AAAAAAAALDDFFCAA|1985-03-14|1022|4446|342|1985|4|3|14|1|1985|342|4446|Thursday|1985Q1|N|N|N|2446126|2446184|2445774|2446047|N|N|N|N|N| +2446140|AAAAAAAAMDDFFCAA|1985-03-15|1022|4446|342|1985|5|3|15|1|1985|342|4446|Friday|1985Q1|N|Y|N|2446126|2446184|2445775|2446048|N|N|N|N|N| +2446141|AAAAAAAANDDFFCAA|1985-03-16|1022|4446|342|1985|6|3|16|1|1985|342|4446|Saturday|1985Q1|N|Y|N|2446126|2446184|2445776|2446049|N|N|N|N|N| +2446142|AAAAAAAAODDFFCAA|1985-03-17|1022|4446|342|1985|0|3|17|1|1985|342|4446|Sunday|1985Q1|N|N|N|2446126|2446184|2445777|2446050|N|N|N|N|N| +2446143|AAAAAAAAPDDFFCAA|1985-03-18|1022|4446|342|1985|1|3|18|1|1985|342|4446|Monday|1985Q1|N|N|N|2446126|2446184|2445778|2446051|N|N|N|N|N| +2446144|AAAAAAAAAEDFFCAA|1985-03-19|1022|4447|342|1985|2|3|19|1|1985|342|4447|Tuesday|1985Q1|N|N|N|2446126|2446184|2445779|2446052|N|N|N|N|N| +2446145|AAAAAAAABEDFFCAA|1985-03-20|1022|4447|342|1985|3|3|20|1|1985|342|4447|Wednesday|1985Q1|N|N|N|2446126|2446184|2445780|2446053|N|N|N|N|N| +2446146|AAAAAAAACEDFFCAA|1985-03-21|1022|4447|342|1985|4|3|21|1|1985|342|4447|Thursday|1985Q1|N|N|N|2446126|2446184|2445781|2446054|N|N|N|N|N| +2446147|AAAAAAAADEDFFCAA|1985-03-22|1022|4447|342|1985|5|3|22|1|1985|342|4447|Friday|1985Q1|N|Y|N|2446126|2446184|2445782|2446055|N|N|N|N|N| +2446148|AAAAAAAAEEDFFCAA|1985-03-23|1022|4447|342|1985|6|3|23|1|1985|342|4447|Saturday|1985Q1|N|Y|N|2446126|2446184|2445783|2446056|N|N|N|N|N| +2446149|AAAAAAAAFEDFFCAA|1985-03-24|1022|4447|342|1985|0|3|24|1|1985|342|4447|Sunday|1985Q1|N|N|N|2446126|2446184|2445784|2446057|N|N|N|N|N| +2446150|AAAAAAAAGEDFFCAA|1985-03-25|1022|4447|342|1985|1|3|25|1|1985|342|4447|Monday|1985Q1|N|N|N|2446126|2446184|2445785|2446058|N|N|N|N|N| +2446151|AAAAAAAAHEDFFCAA|1985-03-26|1022|4448|342|1985|2|3|26|1|1985|342|4448|Tuesday|1985Q1|N|N|N|2446126|2446184|2445786|2446059|N|N|N|N|N| +2446152|AAAAAAAAIEDFFCAA|1985-03-27|1022|4448|342|1985|3|3|27|1|1985|342|4448|Wednesday|1985Q1|N|N|N|2446126|2446184|2445787|2446060|N|N|N|N|N| +2446153|AAAAAAAAJEDFFCAA|1985-03-28|1022|4448|342|1985|4|3|28|1|1985|342|4448|Thursday|1985Q1|N|N|N|2446126|2446184|2445788|2446061|N|N|N|N|N| +2446154|AAAAAAAAKEDFFCAA|1985-03-29|1022|4448|342|1985|5|3|29|1|1985|342|4448|Friday|1985Q1|N|Y|N|2446126|2446184|2445789|2446062|N|N|N|N|N| +2446155|AAAAAAAALEDFFCAA|1985-03-30|1022|4448|342|1985|6|3|30|1|1985|342|4448|Saturday|1985Q1|N|Y|N|2446126|2446184|2445790|2446063|N|N|N|N|N| +2446156|AAAAAAAAMEDFFCAA|1985-03-31|1022|4448|342|1985|0|3|31|1|1985|342|4448|Sunday|1985Q1|N|N|N|2446126|2446184|2445791|2446064|N|N|N|N|N| +2446157|AAAAAAAANEDFFCAA|1985-04-01|1023|4448|342|1985|1|4|1|1|1985|342|4448|Monday|1985Q1|N|N|N|2446157|2446246|2445792|2446067|N|N|N|N|N| +2446158|AAAAAAAAOEDFFCAA|1985-04-02|1023|4449|342|1985|2|4|2|2|1985|342|4449|Tuesday|1985Q2|N|N|N|2446157|2446246|2445793|2446068|N|N|N|N|N| +2446159|AAAAAAAAPEDFFCAA|1985-04-03|1023|4449|342|1985|3|4|3|2|1985|342|4449|Wednesday|1985Q2|N|N|N|2446157|2446246|2445794|2446069|N|N|N|N|N| +2446160|AAAAAAAAAFDFFCAA|1985-04-04|1023|4449|342|1985|4|4|4|2|1985|342|4449|Thursday|1985Q2|N|N|N|2446157|2446246|2445795|2446070|N|N|N|N|N| +2446161|AAAAAAAABFDFFCAA|1985-04-05|1023|4449|342|1985|5|4|5|2|1985|342|4449|Friday|1985Q2|N|Y|N|2446157|2446246|2445796|2446071|N|N|N|N|N| +2446162|AAAAAAAACFDFFCAA|1985-04-06|1023|4449|342|1985|6|4|6|2|1985|342|4449|Saturday|1985Q2|N|Y|N|2446157|2446246|2445797|2446072|N|N|N|N|N| +2446163|AAAAAAAADFDFFCAA|1985-04-07|1023|4449|342|1985|0|4|7|2|1985|342|4449|Sunday|1985Q2|N|N|N|2446157|2446246|2445798|2446073|N|N|N|N|N| +2446164|AAAAAAAAEFDFFCAA|1985-04-08|1023|4449|342|1985|1|4|8|2|1985|342|4449|Monday|1985Q2|N|N|N|2446157|2446246|2445799|2446074|N|N|N|N|N| +2446165|AAAAAAAAFFDFFCAA|1985-04-09|1023|4450|342|1985|2|4|9|2|1985|342|4450|Tuesday|1985Q2|N|N|N|2446157|2446246|2445800|2446075|N|N|N|N|N| +2446166|AAAAAAAAGFDFFCAA|1985-04-10|1023|4450|342|1985|3|4|10|2|1985|342|4450|Wednesday|1985Q2|N|N|N|2446157|2446246|2445801|2446076|N|N|N|N|N| +2446167|AAAAAAAAHFDFFCAA|1985-04-11|1023|4450|342|1985|4|4|11|2|1985|342|4450|Thursday|1985Q2|N|N|N|2446157|2446246|2445802|2446077|N|N|N|N|N| +2446168|AAAAAAAAIFDFFCAA|1985-04-12|1023|4450|342|1985|5|4|12|2|1985|342|4450|Friday|1985Q2|N|Y|N|2446157|2446246|2445803|2446078|N|N|N|N|N| +2446169|AAAAAAAAJFDFFCAA|1985-04-13|1023|4450|342|1985|6|4|13|2|1985|342|4450|Saturday|1985Q2|N|Y|N|2446157|2446246|2445804|2446079|N|N|N|N|N| +2446170|AAAAAAAAKFDFFCAA|1985-04-14|1023|4450|342|1985|0|4|14|2|1985|342|4450|Sunday|1985Q2|N|N|N|2446157|2446246|2445805|2446080|N|N|N|N|N| +2446171|AAAAAAAALFDFFCAA|1985-04-15|1023|4450|342|1985|1|4|15|2|1985|342|4450|Monday|1985Q2|N|N|N|2446157|2446246|2445806|2446081|N|N|N|N|N| +2446172|AAAAAAAAMFDFFCAA|1985-04-16|1023|4451|342|1985|2|4|16|2|1985|342|4451|Tuesday|1985Q2|N|N|N|2446157|2446246|2445807|2446082|N|N|N|N|N| +2446173|AAAAAAAANFDFFCAA|1985-04-17|1023|4451|342|1985|3|4|17|2|1985|342|4451|Wednesday|1985Q2|N|N|N|2446157|2446246|2445808|2446083|N|N|N|N|N| +2446174|AAAAAAAAOFDFFCAA|1985-04-18|1023|4451|342|1985|4|4|18|2|1985|342|4451|Thursday|1985Q2|N|N|N|2446157|2446246|2445809|2446084|N|N|N|N|N| +2446175|AAAAAAAAPFDFFCAA|1985-04-19|1023|4451|342|1985|5|4|19|2|1985|342|4451|Friday|1985Q2|N|Y|N|2446157|2446246|2445810|2446085|N|N|N|N|N| +2446176|AAAAAAAAAGDFFCAA|1985-04-20|1023|4451|342|1985|6|4|20|2|1985|342|4451|Saturday|1985Q2|N|Y|N|2446157|2446246|2445811|2446086|N|N|N|N|N| +2446177|AAAAAAAABGDFFCAA|1985-04-21|1023|4451|342|1985|0|4|21|2|1985|342|4451|Sunday|1985Q2|N|N|N|2446157|2446246|2445812|2446087|N|N|N|N|N| +2446178|AAAAAAAACGDFFCAA|1985-04-22|1023|4451|342|1985|1|4|22|2|1985|342|4451|Monday|1985Q2|N|N|N|2446157|2446246|2445813|2446088|N|N|N|N|N| +2446179|AAAAAAAADGDFFCAA|1985-04-23|1023|4452|342|1985|2|4|23|2|1985|342|4452|Tuesday|1985Q2|N|N|N|2446157|2446246|2445814|2446089|N|N|N|N|N| +2446180|AAAAAAAAEGDFFCAA|1985-04-24|1023|4452|342|1985|3|4|24|2|1985|342|4452|Wednesday|1985Q2|N|N|N|2446157|2446246|2445815|2446090|N|N|N|N|N| +2446181|AAAAAAAAFGDFFCAA|1985-04-25|1023|4452|342|1985|4|4|25|2|1985|342|4452|Thursday|1985Q2|N|N|N|2446157|2446246|2445816|2446091|N|N|N|N|N| +2446182|AAAAAAAAGGDFFCAA|1985-04-26|1023|4452|342|1985|5|4|26|2|1985|342|4452|Friday|1985Q2|N|Y|N|2446157|2446246|2445817|2446092|N|N|N|N|N| +2446183|AAAAAAAAHGDFFCAA|1985-04-27|1023|4452|342|1985|6|4|27|2|1985|342|4452|Saturday|1985Q2|N|Y|N|2446157|2446246|2445818|2446093|N|N|N|N|N| +2446184|AAAAAAAAIGDFFCAA|1985-04-28|1023|4452|342|1985|0|4|28|2|1985|342|4452|Sunday|1985Q2|N|N|N|2446157|2446246|2445819|2446094|N|N|N|N|N| +2446185|AAAAAAAAJGDFFCAA|1985-04-29|1023|4452|342|1985|1|4|29|2|1985|342|4452|Monday|1985Q2|N|N|N|2446157|2446246|2445820|2446095|N|N|N|N|N| +2446186|AAAAAAAAKGDFFCAA|1985-04-30|1023|4453|342|1985|2|4|30|2|1985|342|4453|Tuesday|1985Q2|N|N|N|2446157|2446246|2445821|2446096|N|N|N|N|N| +2446187|AAAAAAAALGDFFCAA|1985-05-01|1024|4453|342|1985|3|5|1|2|1985|342|4453|Wednesday|1985Q2|N|N|N|2446187|2446306|2445822|2446097|N|N|N|N|N| +2446188|AAAAAAAAMGDFFCAA|1985-05-02|1024|4453|342|1985|4|5|2|2|1985|342|4453|Thursday|1985Q2|N|N|N|2446187|2446306|2445823|2446098|N|N|N|N|N| +2446189|AAAAAAAANGDFFCAA|1985-05-03|1024|4453|342|1985|5|5|3|2|1985|342|4453|Friday|1985Q2|N|Y|N|2446187|2446306|2445824|2446099|N|N|N|N|N| +2446190|AAAAAAAAOGDFFCAA|1985-05-04|1024|4453|342|1985|6|5|4|2|1985|342|4453|Saturday|1985Q2|N|Y|N|2446187|2446306|2445825|2446100|N|N|N|N|N| +2446191|AAAAAAAAPGDFFCAA|1985-05-05|1024|4453|342|1985|0|5|5|2|1985|342|4453|Sunday|1985Q2|N|N|N|2446187|2446306|2445826|2446101|N|N|N|N|N| +2446192|AAAAAAAAAHDFFCAA|1985-05-06|1024|4453|342|1985|1|5|6|2|1985|342|4453|Monday|1985Q2|N|N|N|2446187|2446306|2445827|2446102|N|N|N|N|N| +2446193|AAAAAAAABHDFFCAA|1985-05-07|1024|4454|342|1985|2|5|7|2|1985|342|4454|Tuesday|1985Q2|N|N|N|2446187|2446306|2445828|2446103|N|N|N|N|N| +2446194|AAAAAAAACHDFFCAA|1985-05-08|1024|4454|342|1985|3|5|8|2|1985|342|4454|Wednesday|1985Q2|N|N|N|2446187|2446306|2445829|2446104|N|N|N|N|N| +2446195|AAAAAAAADHDFFCAA|1985-05-09|1024|4454|342|1985|4|5|9|2|1985|342|4454|Thursday|1985Q2|N|N|N|2446187|2446306|2445830|2446105|N|N|N|N|N| +2446196|AAAAAAAAEHDFFCAA|1985-05-10|1024|4454|342|1985|5|5|10|2|1985|342|4454|Friday|1985Q2|N|Y|N|2446187|2446306|2445831|2446106|N|N|N|N|N| +2446197|AAAAAAAAFHDFFCAA|1985-05-11|1024|4454|342|1985|6|5|11|2|1985|342|4454|Saturday|1985Q2|N|Y|N|2446187|2446306|2445832|2446107|N|N|N|N|N| +2446198|AAAAAAAAGHDFFCAA|1985-05-12|1024|4454|342|1985|0|5|12|2|1985|342|4454|Sunday|1985Q2|N|N|N|2446187|2446306|2445833|2446108|N|N|N|N|N| +2446199|AAAAAAAAHHDFFCAA|1985-05-13|1024|4454|342|1985|1|5|13|2|1985|342|4454|Monday|1985Q2|N|N|N|2446187|2446306|2445834|2446109|N|N|N|N|N| +2446200|AAAAAAAAIHDFFCAA|1985-05-14|1024|4455|342|1985|2|5|14|2|1985|342|4455|Tuesday|1985Q2|N|N|N|2446187|2446306|2445835|2446110|N|N|N|N|N| +2446201|AAAAAAAAJHDFFCAA|1985-05-15|1024|4455|342|1985|3|5|15|2|1985|342|4455|Wednesday|1985Q2|N|N|N|2446187|2446306|2445836|2446111|N|N|N|N|N| +2446202|AAAAAAAAKHDFFCAA|1985-05-16|1024|4455|342|1985|4|5|16|2|1985|342|4455|Thursday|1985Q2|N|N|N|2446187|2446306|2445837|2446112|N|N|N|N|N| +2446203|AAAAAAAALHDFFCAA|1985-05-17|1024|4455|342|1985|5|5|17|2|1985|342|4455|Friday|1985Q2|N|Y|N|2446187|2446306|2445838|2446113|N|N|N|N|N| +2446204|AAAAAAAAMHDFFCAA|1985-05-18|1024|4455|342|1985|6|5|18|2|1985|342|4455|Saturday|1985Q2|N|Y|N|2446187|2446306|2445839|2446114|N|N|N|N|N| +2446205|AAAAAAAANHDFFCAA|1985-05-19|1024|4455|342|1985|0|5|19|2|1985|342|4455|Sunday|1985Q2|N|N|N|2446187|2446306|2445840|2446115|N|N|N|N|N| +2446206|AAAAAAAAOHDFFCAA|1985-05-20|1024|4455|342|1985|1|5|20|2|1985|342|4455|Monday|1985Q2|N|N|N|2446187|2446306|2445841|2446116|N|N|N|N|N| +2446207|AAAAAAAAPHDFFCAA|1985-05-21|1024|4456|342|1985|2|5|21|2|1985|342|4456|Tuesday|1985Q2|N|N|N|2446187|2446306|2445842|2446117|N|N|N|N|N| +2446208|AAAAAAAAAIDFFCAA|1985-05-22|1024|4456|342|1985|3|5|22|2|1985|342|4456|Wednesday|1985Q2|N|N|N|2446187|2446306|2445843|2446118|N|N|N|N|N| +2446209|AAAAAAAABIDFFCAA|1985-05-23|1024|4456|342|1985|4|5|23|2|1985|342|4456|Thursday|1985Q2|N|N|N|2446187|2446306|2445844|2446119|N|N|N|N|N| +2446210|AAAAAAAACIDFFCAA|1985-05-24|1024|4456|342|1985|5|5|24|2|1985|342|4456|Friday|1985Q2|N|Y|N|2446187|2446306|2445845|2446120|N|N|N|N|N| +2446211|AAAAAAAADIDFFCAA|1985-05-25|1024|4456|342|1985|6|5|25|2|1985|342|4456|Saturday|1985Q2|N|Y|N|2446187|2446306|2445846|2446121|N|N|N|N|N| +2446212|AAAAAAAAEIDFFCAA|1985-05-26|1024|4456|342|1985|0|5|26|2|1985|342|4456|Sunday|1985Q2|N|N|N|2446187|2446306|2445847|2446122|N|N|N|N|N| +2446213|AAAAAAAAFIDFFCAA|1985-05-27|1024|4456|342|1985|1|5|27|2|1985|342|4456|Monday|1985Q2|N|N|N|2446187|2446306|2445848|2446123|N|N|N|N|N| +2446214|AAAAAAAAGIDFFCAA|1985-05-28|1024|4457|342|1985|2|5|28|2|1985|342|4457|Tuesday|1985Q2|N|N|N|2446187|2446306|2445849|2446124|N|N|N|N|N| +2446215|AAAAAAAAHIDFFCAA|1985-05-29|1024|4457|342|1985|3|5|29|2|1985|342|4457|Wednesday|1985Q2|N|N|N|2446187|2446306|2445850|2446125|N|N|N|N|N| +2446216|AAAAAAAAIIDFFCAA|1985-05-30|1024|4457|342|1985|4|5|30|2|1985|342|4457|Thursday|1985Q2|N|N|N|2446187|2446306|2445851|2446126|N|N|N|N|N| +2446217|AAAAAAAAJIDFFCAA|1985-05-31|1024|4457|342|1985|5|5|31|2|1985|342|4457|Friday|1985Q2|N|Y|N|2446187|2446306|2445852|2446127|N|N|N|N|N| +2446218|AAAAAAAAKIDFFCAA|1985-06-01|1025|4457|343|1985|6|6|1|2|1985|343|4457|Saturday|1985Q2|N|Y|N|2446218|2446368|2445853|2446128|N|N|N|N|N| +2446219|AAAAAAAALIDFFCAA|1985-06-02|1025|4457|343|1985|0|6|2|2|1985|343|4457|Sunday|1985Q2|N|N|N|2446218|2446368|2445854|2446129|N|N|N|N|N| +2446220|AAAAAAAAMIDFFCAA|1985-06-03|1025|4457|343|1985|1|6|3|2|1985|343|4457|Monday|1985Q2|N|N|N|2446218|2446368|2445855|2446130|N|N|N|N|N| +2446221|AAAAAAAANIDFFCAA|1985-06-04|1025|4458|343|1985|2|6|4|2|1985|343|4458|Tuesday|1985Q2|N|N|N|2446218|2446368|2445856|2446131|N|N|N|N|N| +2446222|AAAAAAAAOIDFFCAA|1985-06-05|1025|4458|343|1985|3|6|5|2|1985|343|4458|Wednesday|1985Q2|N|N|N|2446218|2446368|2445857|2446132|N|N|N|N|N| +2446223|AAAAAAAAPIDFFCAA|1985-06-06|1025|4458|343|1985|4|6|6|2|1985|343|4458|Thursday|1985Q2|N|N|N|2446218|2446368|2445858|2446133|N|N|N|N|N| +2446224|AAAAAAAAAJDFFCAA|1985-06-07|1025|4458|343|1985|5|6|7|2|1985|343|4458|Friday|1985Q2|N|Y|N|2446218|2446368|2445859|2446134|N|N|N|N|N| +2446225|AAAAAAAABJDFFCAA|1985-06-08|1025|4458|343|1985|6|6|8|2|1985|343|4458|Saturday|1985Q2|N|Y|N|2446218|2446368|2445860|2446135|N|N|N|N|N| +2446226|AAAAAAAACJDFFCAA|1985-06-09|1025|4458|343|1985|0|6|9|2|1985|343|4458|Sunday|1985Q2|N|N|N|2446218|2446368|2445861|2446136|N|N|N|N|N| +2446227|AAAAAAAADJDFFCAA|1985-06-10|1025|4458|343|1985|1|6|10|2|1985|343|4458|Monday|1985Q2|N|N|N|2446218|2446368|2445862|2446137|N|N|N|N|N| +2446228|AAAAAAAAEJDFFCAA|1985-06-11|1025|4459|343|1985|2|6|11|2|1985|343|4459|Tuesday|1985Q2|N|N|N|2446218|2446368|2445863|2446138|N|N|N|N|N| +2446229|AAAAAAAAFJDFFCAA|1985-06-12|1025|4459|343|1985|3|6|12|2|1985|343|4459|Wednesday|1985Q2|N|N|N|2446218|2446368|2445864|2446139|N|N|N|N|N| +2446230|AAAAAAAAGJDFFCAA|1985-06-13|1025|4459|343|1985|4|6|13|2|1985|343|4459|Thursday|1985Q2|N|N|N|2446218|2446368|2445865|2446140|N|N|N|N|N| +2446231|AAAAAAAAHJDFFCAA|1985-06-14|1025|4459|343|1985|5|6|14|2|1985|343|4459|Friday|1985Q2|N|Y|N|2446218|2446368|2445866|2446141|N|N|N|N|N| +2446232|AAAAAAAAIJDFFCAA|1985-06-15|1025|4459|343|1985|6|6|15|2|1985|343|4459|Saturday|1985Q2|N|Y|N|2446218|2446368|2445867|2446142|N|N|N|N|N| +2446233|AAAAAAAAJJDFFCAA|1985-06-16|1025|4459|343|1985|0|6|16|2|1985|343|4459|Sunday|1985Q2|N|N|N|2446218|2446368|2445868|2446143|N|N|N|N|N| +2446234|AAAAAAAAKJDFFCAA|1985-06-17|1025|4459|343|1985|1|6|17|2|1985|343|4459|Monday|1985Q2|N|N|N|2446218|2446368|2445869|2446144|N|N|N|N|N| +2446235|AAAAAAAALJDFFCAA|1985-06-18|1025|4460|343|1985|2|6|18|2|1985|343|4460|Tuesday|1985Q2|N|N|N|2446218|2446368|2445870|2446145|N|N|N|N|N| +2446236|AAAAAAAAMJDFFCAA|1985-06-19|1025|4460|343|1985|3|6|19|2|1985|343|4460|Wednesday|1985Q2|N|N|N|2446218|2446368|2445871|2446146|N|N|N|N|N| +2446237|AAAAAAAANJDFFCAA|1985-06-20|1025|4460|343|1985|4|6|20|2|1985|343|4460|Thursday|1985Q2|N|N|N|2446218|2446368|2445872|2446147|N|N|N|N|N| +2446238|AAAAAAAAOJDFFCAA|1985-06-21|1025|4460|343|1985|5|6|21|2|1985|343|4460|Friday|1985Q2|N|Y|N|2446218|2446368|2445873|2446148|N|N|N|N|N| +2446239|AAAAAAAAPJDFFCAA|1985-06-22|1025|4460|343|1985|6|6|22|2|1985|343|4460|Saturday|1985Q2|N|Y|N|2446218|2446368|2445874|2446149|N|N|N|N|N| +2446240|AAAAAAAAAKDFFCAA|1985-06-23|1025|4460|343|1985|0|6|23|2|1985|343|4460|Sunday|1985Q2|N|N|N|2446218|2446368|2445875|2446150|N|N|N|N|N| +2446241|AAAAAAAABKDFFCAA|1985-06-24|1025|4460|343|1985|1|6|24|2|1985|343|4460|Monday|1985Q2|N|N|N|2446218|2446368|2445876|2446151|N|N|N|N|N| +2446242|AAAAAAAACKDFFCAA|1985-06-25|1025|4461|343|1985|2|6|25|2|1985|343|4461|Tuesday|1985Q2|N|N|N|2446218|2446368|2445877|2446152|N|N|N|N|N| +2446243|AAAAAAAADKDFFCAA|1985-06-26|1025|4461|343|1985|3|6|26|2|1985|343|4461|Wednesday|1985Q2|N|N|N|2446218|2446368|2445878|2446153|N|N|N|N|N| +2446244|AAAAAAAAEKDFFCAA|1985-06-27|1025|4461|343|1985|4|6|27|2|1985|343|4461|Thursday|1985Q2|N|N|N|2446218|2446368|2445879|2446154|N|N|N|N|N| +2446245|AAAAAAAAFKDFFCAA|1985-06-28|1025|4461|343|1985|5|6|28|2|1985|343|4461|Friday|1985Q2|N|Y|N|2446218|2446368|2445880|2446155|N|N|N|N|N| +2446246|AAAAAAAAGKDFFCAA|1985-06-29|1025|4461|343|1985|6|6|29|2|1985|343|4461|Saturday|1985Q2|N|Y|N|2446218|2446368|2445881|2446156|N|N|N|N|N| +2446247|AAAAAAAAHKDFFCAA|1985-06-30|1025|4461|343|1985|0|6|30|2|1985|343|4461|Sunday|1985Q2|N|N|N|2446218|2446368|2445882|2446157|N|N|N|N|N| +2446248|AAAAAAAAIKDFFCAA|1985-07-01|1026|4461|343|1985|1|7|1|2|1985|343|4461|Monday|1985Q2|N|N|N|2446248|2446428|2445883|2446157|N|N|N|N|N| +2446249|AAAAAAAAJKDFFCAA|1985-07-02|1026|4462|343|1985|2|7|2|3|1985|343|4462|Tuesday|1985Q3|N|N|N|2446248|2446428|2445884|2446158|N|N|N|N|N| +2446250|AAAAAAAAKKDFFCAA|1985-07-03|1026|4462|343|1985|3|7|3|3|1985|343|4462|Wednesday|1985Q3|N|N|N|2446248|2446428|2445885|2446159|N|N|N|N|N| +2446251|AAAAAAAALKDFFCAA|1985-07-04|1026|4462|343|1985|4|7|4|3|1985|343|4462|Thursday|1985Q3|N|N|N|2446248|2446428|2445886|2446160|N|N|N|N|N| +2446252|AAAAAAAAMKDFFCAA|1985-07-05|1026|4462|343|1985|5|7|5|3|1985|343|4462|Friday|1985Q3|Y|Y|N|2446248|2446428|2445887|2446161|N|N|N|N|N| +2446253|AAAAAAAANKDFFCAA|1985-07-06|1026|4462|343|1985|6|7|6|3|1985|343|4462|Saturday|1985Q3|N|Y|Y|2446248|2446428|2445888|2446162|N|N|N|N|N| +2446254|AAAAAAAAOKDFFCAA|1985-07-07|1026|4462|343|1985|0|7|7|3|1985|343|4462|Sunday|1985Q3|N|N|N|2446248|2446428|2445889|2446163|N|N|N|N|N| +2446255|AAAAAAAAPKDFFCAA|1985-07-08|1026|4462|343|1985|1|7|8|3|1985|343|4462|Monday|1985Q3|N|N|N|2446248|2446428|2445890|2446164|N|N|N|N|N| +2446256|AAAAAAAAALDFFCAA|1985-07-09|1026|4463|343|1985|2|7|9|3|1985|343|4463|Tuesday|1985Q3|N|N|N|2446248|2446428|2445891|2446165|N|N|N|N|N| +2446257|AAAAAAAABLDFFCAA|1985-07-10|1026|4463|343|1985|3|7|10|3|1985|343|4463|Wednesday|1985Q3|N|N|N|2446248|2446428|2445892|2446166|N|N|N|N|N| +2446258|AAAAAAAACLDFFCAA|1985-07-11|1026|4463|343|1985|4|7|11|3|1985|343|4463|Thursday|1985Q3|N|N|N|2446248|2446428|2445893|2446167|N|N|N|N|N| +2446259|AAAAAAAADLDFFCAA|1985-07-12|1026|4463|343|1985|5|7|12|3|1985|343|4463|Friday|1985Q3|N|Y|N|2446248|2446428|2445894|2446168|N|N|N|N|N| +2446260|AAAAAAAAELDFFCAA|1985-07-13|1026|4463|343|1985|6|7|13|3|1985|343|4463|Saturday|1985Q3|N|Y|N|2446248|2446428|2445895|2446169|N|N|N|N|N| +2446261|AAAAAAAAFLDFFCAA|1985-07-14|1026|4463|343|1985|0|7|14|3|1985|343|4463|Sunday|1985Q3|N|N|N|2446248|2446428|2445896|2446170|N|N|N|N|N| +2446262|AAAAAAAAGLDFFCAA|1985-07-15|1026|4463|343|1985|1|7|15|3|1985|343|4463|Monday|1985Q3|N|N|N|2446248|2446428|2445897|2446171|N|N|N|N|N| +2446263|AAAAAAAAHLDFFCAA|1985-07-16|1026|4464|343|1985|2|7|16|3|1985|343|4464|Tuesday|1985Q3|N|N|N|2446248|2446428|2445898|2446172|N|N|N|N|N| +2446264|AAAAAAAAILDFFCAA|1985-07-17|1026|4464|343|1985|3|7|17|3|1985|343|4464|Wednesday|1985Q3|N|N|N|2446248|2446428|2445899|2446173|N|N|N|N|N| +2446265|AAAAAAAAJLDFFCAA|1985-07-18|1026|4464|343|1985|4|7|18|3|1985|343|4464|Thursday|1985Q3|N|N|N|2446248|2446428|2445900|2446174|N|N|N|N|N| +2446266|AAAAAAAAKLDFFCAA|1985-07-19|1026|4464|343|1985|5|7|19|3|1985|343|4464|Friday|1985Q3|N|Y|N|2446248|2446428|2445901|2446175|N|N|N|N|N| +2446267|AAAAAAAALLDFFCAA|1985-07-20|1026|4464|343|1985|6|7|20|3|1985|343|4464|Saturday|1985Q3|N|Y|N|2446248|2446428|2445902|2446176|N|N|N|N|N| +2446268|AAAAAAAAMLDFFCAA|1985-07-21|1026|4464|343|1985|0|7|21|3|1985|343|4464|Sunday|1985Q3|N|N|N|2446248|2446428|2445903|2446177|N|N|N|N|N| +2446269|AAAAAAAANLDFFCAA|1985-07-22|1026|4464|343|1985|1|7|22|3|1985|343|4464|Monday|1985Q3|N|N|N|2446248|2446428|2445904|2446178|N|N|N|N|N| +2446270|AAAAAAAAOLDFFCAA|1985-07-23|1026|4465|343|1985|2|7|23|3|1985|343|4465|Tuesday|1985Q3|N|N|N|2446248|2446428|2445905|2446179|N|N|N|N|N| +2446271|AAAAAAAAPLDFFCAA|1985-07-24|1026|4465|343|1985|3|7|24|3|1985|343|4465|Wednesday|1985Q3|N|N|N|2446248|2446428|2445906|2446180|N|N|N|N|N| +2446272|AAAAAAAAAMDFFCAA|1985-07-25|1026|4465|343|1985|4|7|25|3|1985|343|4465|Thursday|1985Q3|N|N|N|2446248|2446428|2445907|2446181|N|N|N|N|N| +2446273|AAAAAAAABMDFFCAA|1985-07-26|1026|4465|343|1985|5|7|26|3|1985|343|4465|Friday|1985Q3|N|Y|N|2446248|2446428|2445908|2446182|N|N|N|N|N| +2446274|AAAAAAAACMDFFCAA|1985-07-27|1026|4465|343|1985|6|7|27|3|1985|343|4465|Saturday|1985Q3|N|Y|N|2446248|2446428|2445909|2446183|N|N|N|N|N| +2446275|AAAAAAAADMDFFCAA|1985-07-28|1026|4465|343|1985|0|7|28|3|1985|343|4465|Sunday|1985Q3|N|N|N|2446248|2446428|2445910|2446184|N|N|N|N|N| +2446276|AAAAAAAAEMDFFCAA|1985-07-29|1026|4465|343|1985|1|7|29|3|1985|343|4465|Monday|1985Q3|N|N|N|2446248|2446428|2445911|2446185|N|N|N|N|N| +2446277|AAAAAAAAFMDFFCAA|1985-07-30|1026|4466|343|1985|2|7|30|3|1985|343|4466|Tuesday|1985Q3|N|N|N|2446248|2446428|2445912|2446186|N|N|N|N|N| +2446278|AAAAAAAAGMDFFCAA|1985-07-31|1026|4466|343|1985|3|7|31|3|1985|343|4466|Wednesday|1985Q3|N|N|N|2446248|2446428|2445913|2446187|N|N|N|N|N| +2446279|AAAAAAAAHMDFFCAA|1985-08-01|1027|4466|343|1985|4|8|1|3|1985|343|4466|Thursday|1985Q3|N|N|N|2446279|2446490|2445914|2446188|N|N|N|N|N| +2446280|AAAAAAAAIMDFFCAA|1985-08-02|1027|4466|343|1985|5|8|2|3|1985|343|4466|Friday|1985Q3|N|Y|N|2446279|2446490|2445915|2446189|N|N|N|N|N| +2446281|AAAAAAAAJMDFFCAA|1985-08-03|1027|4466|343|1985|6|8|3|3|1985|343|4466|Saturday|1985Q3|N|Y|N|2446279|2446490|2445916|2446190|N|N|N|N|N| +2446282|AAAAAAAAKMDFFCAA|1985-08-04|1027|4466|343|1985|0|8|4|3|1985|343|4466|Sunday|1985Q3|N|N|N|2446279|2446490|2445917|2446191|N|N|N|N|N| +2446283|AAAAAAAALMDFFCAA|1985-08-05|1027|4466|343|1985|1|8|5|3|1985|343|4466|Monday|1985Q3|N|N|N|2446279|2446490|2445918|2446192|N|N|N|N|N| +2446284|AAAAAAAAMMDFFCAA|1985-08-06|1027|4467|343|1985|2|8|6|3|1985|343|4467|Tuesday|1985Q3|N|N|N|2446279|2446490|2445919|2446193|N|N|N|N|N| +2446285|AAAAAAAANMDFFCAA|1985-08-07|1027|4467|343|1985|3|8|7|3|1985|343|4467|Wednesday|1985Q3|N|N|N|2446279|2446490|2445920|2446194|N|N|N|N|N| +2446286|AAAAAAAAOMDFFCAA|1985-08-08|1027|4467|343|1985|4|8|8|3|1985|343|4467|Thursday|1985Q3|N|N|N|2446279|2446490|2445921|2446195|N|N|N|N|N| +2446287|AAAAAAAAPMDFFCAA|1985-08-09|1027|4467|343|1985|5|8|9|3|1985|343|4467|Friday|1985Q3|N|Y|N|2446279|2446490|2445922|2446196|N|N|N|N|N| +2446288|AAAAAAAAANDFFCAA|1985-08-10|1027|4467|343|1985|6|8|10|3|1985|343|4467|Saturday|1985Q3|N|Y|N|2446279|2446490|2445923|2446197|N|N|N|N|N| +2446289|AAAAAAAABNDFFCAA|1985-08-11|1027|4467|343|1985|0|8|11|3|1985|343|4467|Sunday|1985Q3|N|N|N|2446279|2446490|2445924|2446198|N|N|N|N|N| +2446290|AAAAAAAACNDFFCAA|1985-08-12|1027|4467|343|1985|1|8|12|3|1985|343|4467|Monday|1985Q3|N|N|N|2446279|2446490|2445925|2446199|N|N|N|N|N| +2446291|AAAAAAAADNDFFCAA|1985-08-13|1027|4468|343|1985|2|8|13|3|1985|343|4468|Tuesday|1985Q3|N|N|N|2446279|2446490|2445926|2446200|N|N|N|N|N| +2446292|AAAAAAAAENDFFCAA|1985-08-14|1027|4468|343|1985|3|8|14|3|1985|343|4468|Wednesday|1985Q3|N|N|N|2446279|2446490|2445927|2446201|N|N|N|N|N| +2446293|AAAAAAAAFNDFFCAA|1985-08-15|1027|4468|343|1985|4|8|15|3|1985|343|4468|Thursday|1985Q3|N|N|N|2446279|2446490|2445928|2446202|N|N|N|N|N| +2446294|AAAAAAAAGNDFFCAA|1985-08-16|1027|4468|343|1985|5|8|16|3|1985|343|4468|Friday|1985Q3|N|Y|N|2446279|2446490|2445929|2446203|N|N|N|N|N| +2446295|AAAAAAAAHNDFFCAA|1985-08-17|1027|4468|343|1985|6|8|17|3|1985|343|4468|Saturday|1985Q3|N|Y|N|2446279|2446490|2445930|2446204|N|N|N|N|N| +2446296|AAAAAAAAINDFFCAA|1985-08-18|1027|4468|343|1985|0|8|18|3|1985|343|4468|Sunday|1985Q3|N|N|N|2446279|2446490|2445931|2446205|N|N|N|N|N| +2446297|AAAAAAAAJNDFFCAA|1985-08-19|1027|4468|343|1985|1|8|19|3|1985|343|4468|Monday|1985Q3|N|N|N|2446279|2446490|2445932|2446206|N|N|N|N|N| +2446298|AAAAAAAAKNDFFCAA|1985-08-20|1027|4469|343|1985|2|8|20|3|1985|343|4469|Tuesday|1985Q3|N|N|N|2446279|2446490|2445933|2446207|N|N|N|N|N| +2446299|AAAAAAAALNDFFCAA|1985-08-21|1027|4469|343|1985|3|8|21|3|1985|343|4469|Wednesday|1985Q3|N|N|N|2446279|2446490|2445934|2446208|N|N|N|N|N| +2446300|AAAAAAAAMNDFFCAA|1985-08-22|1027|4469|343|1985|4|8|22|3|1985|343|4469|Thursday|1985Q3|N|N|N|2446279|2446490|2445935|2446209|N|N|N|N|N| +2446301|AAAAAAAANNDFFCAA|1985-08-23|1027|4469|343|1985|5|8|23|3|1985|343|4469|Friday|1985Q3|N|Y|N|2446279|2446490|2445936|2446210|N|N|N|N|N| +2446302|AAAAAAAAONDFFCAA|1985-08-24|1027|4469|343|1985|6|8|24|3|1985|343|4469|Saturday|1985Q3|N|Y|N|2446279|2446490|2445937|2446211|N|N|N|N|N| +2446303|AAAAAAAAPNDFFCAA|1985-08-25|1027|4469|343|1985|0|8|25|3|1985|343|4469|Sunday|1985Q3|N|N|N|2446279|2446490|2445938|2446212|N|N|N|N|N| +2446304|AAAAAAAAAODFFCAA|1985-08-26|1027|4469|343|1985|1|8|26|3|1985|343|4469|Monday|1985Q3|N|N|N|2446279|2446490|2445939|2446213|N|N|N|N|N| +2446305|AAAAAAAABODFFCAA|1985-08-27|1027|4470|343|1985|2|8|27|3|1985|343|4470|Tuesday|1985Q3|N|N|N|2446279|2446490|2445940|2446214|N|N|N|N|N| +2446306|AAAAAAAACODFFCAA|1985-08-28|1027|4470|343|1985|3|8|28|3|1985|343|4470|Wednesday|1985Q3|N|N|N|2446279|2446490|2445941|2446215|N|N|N|N|N| +2446307|AAAAAAAADODFFCAA|1985-08-29|1027|4470|343|1985|4|8|29|3|1985|343|4470|Thursday|1985Q3|N|N|N|2446279|2446490|2445942|2446216|N|N|N|N|N| +2446308|AAAAAAAAEODFFCAA|1985-08-30|1027|4470|343|1985|5|8|30|3|1985|343|4470|Friday|1985Q3|N|Y|N|2446279|2446490|2445943|2446217|N|N|N|N|N| +2446309|AAAAAAAAFODFFCAA|1985-08-31|1027|4470|343|1985|6|8|31|3|1985|343|4470|Saturday|1985Q3|N|Y|N|2446279|2446490|2445944|2446218|N|N|N|N|N| +2446310|AAAAAAAAGODFFCAA|1985-09-01|1028|4470|344|1985|0|9|1|3|1985|344|4470|Sunday|1985Q3|N|N|N|2446310|2446552|2445945|2446219|N|N|N|N|N| +2446311|AAAAAAAAHODFFCAA|1985-09-02|1028|4470|344|1985|1|9|2|3|1985|344|4470|Monday|1985Q3|N|N|N|2446310|2446552|2445946|2446220|N|N|N|N|N| +2446312|AAAAAAAAIODFFCAA|1985-09-03|1028|4471|344|1985|2|9|3|3|1985|344|4471|Tuesday|1985Q3|N|N|N|2446310|2446552|2445947|2446221|N|N|N|N|N| +2446313|AAAAAAAAJODFFCAA|1985-09-04|1028|4471|344|1985|3|9|4|3|1985|344|4471|Wednesday|1985Q3|N|N|N|2446310|2446552|2445948|2446222|N|N|N|N|N| +2446314|AAAAAAAAKODFFCAA|1985-09-05|1028|4471|344|1985|4|9|5|3|1985|344|4471|Thursday|1985Q3|N|N|N|2446310|2446552|2445949|2446223|N|N|N|N|N| +2446315|AAAAAAAALODFFCAA|1985-09-06|1028|4471|344|1985|5|9|6|3|1985|344|4471|Friday|1985Q3|N|Y|N|2446310|2446552|2445950|2446224|N|N|N|N|N| +2446316|AAAAAAAAMODFFCAA|1985-09-07|1028|4471|344|1985|6|9|7|3|1985|344|4471|Saturday|1985Q3|N|Y|N|2446310|2446552|2445951|2446225|N|N|N|N|N| +2446317|AAAAAAAANODFFCAA|1985-09-08|1028|4471|344|1985|0|9|8|3|1985|344|4471|Sunday|1985Q3|N|N|N|2446310|2446552|2445952|2446226|N|N|N|N|N| +2446318|AAAAAAAAOODFFCAA|1985-09-09|1028|4471|344|1985|1|9|9|3|1985|344|4471|Monday|1985Q3|N|N|N|2446310|2446552|2445953|2446227|N|N|N|N|N| +2446319|AAAAAAAAPODFFCAA|1985-09-10|1028|4472|344|1985|2|9|10|3|1985|344|4472|Tuesday|1985Q3|N|N|N|2446310|2446552|2445954|2446228|N|N|N|N|N| +2446320|AAAAAAAAAPDFFCAA|1985-09-11|1028|4472|344|1985|3|9|11|3|1985|344|4472|Wednesday|1985Q3|N|N|N|2446310|2446552|2445955|2446229|N|N|N|N|N| +2446321|AAAAAAAABPDFFCAA|1985-09-12|1028|4472|344|1985|4|9|12|3|1985|344|4472|Thursday|1985Q3|N|N|N|2446310|2446552|2445956|2446230|N|N|N|N|N| +2446322|AAAAAAAACPDFFCAA|1985-09-13|1028|4472|344|1985|5|9|13|3|1985|344|4472|Friday|1985Q3|N|Y|N|2446310|2446552|2445957|2446231|N|N|N|N|N| +2446323|AAAAAAAADPDFFCAA|1985-09-14|1028|4472|344|1985|6|9|14|3|1985|344|4472|Saturday|1985Q3|N|Y|N|2446310|2446552|2445958|2446232|N|N|N|N|N| +2446324|AAAAAAAAEPDFFCAA|1985-09-15|1028|4472|344|1985|0|9|15|3|1985|344|4472|Sunday|1985Q3|N|N|N|2446310|2446552|2445959|2446233|N|N|N|N|N| +2446325|AAAAAAAAFPDFFCAA|1985-09-16|1028|4472|344|1985|1|9|16|3|1985|344|4472|Monday|1985Q3|N|N|N|2446310|2446552|2445960|2446234|N|N|N|N|N| +2446326|AAAAAAAAGPDFFCAA|1985-09-17|1028|4473|344|1985|2|9|17|3|1985|344|4473|Tuesday|1985Q3|N|N|N|2446310|2446552|2445961|2446235|N|N|N|N|N| +2446327|AAAAAAAAHPDFFCAA|1985-09-18|1028|4473|344|1985|3|9|18|3|1985|344|4473|Wednesday|1985Q3|N|N|N|2446310|2446552|2445962|2446236|N|N|N|N|N| +2446328|AAAAAAAAIPDFFCAA|1985-09-19|1028|4473|344|1985|4|9|19|3|1985|344|4473|Thursday|1985Q3|N|N|N|2446310|2446552|2445963|2446237|N|N|N|N|N| +2446329|AAAAAAAAJPDFFCAA|1985-09-20|1028|4473|344|1985|5|9|20|3|1985|344|4473|Friday|1985Q3|N|Y|N|2446310|2446552|2445964|2446238|N|N|N|N|N| +2446330|AAAAAAAAKPDFFCAA|1985-09-21|1028|4473|344|1985|6|9|21|3|1985|344|4473|Saturday|1985Q3|N|Y|N|2446310|2446552|2445965|2446239|N|N|N|N|N| +2446331|AAAAAAAALPDFFCAA|1985-09-22|1028|4473|344|1985|0|9|22|3|1985|344|4473|Sunday|1985Q3|N|N|N|2446310|2446552|2445966|2446240|N|N|N|N|N| +2446332|AAAAAAAAMPDFFCAA|1985-09-23|1028|4473|344|1985|1|9|23|3|1985|344|4473|Monday|1985Q3|N|N|N|2446310|2446552|2445967|2446241|N|N|N|N|N| +2446333|AAAAAAAANPDFFCAA|1985-09-24|1028|4474|344|1985|2|9|24|3|1985|344|4474|Tuesday|1985Q3|N|N|N|2446310|2446552|2445968|2446242|N|N|N|N|N| +2446334|AAAAAAAAOPDFFCAA|1985-09-25|1028|4474|344|1985|3|9|25|3|1985|344|4474|Wednesday|1985Q3|N|N|N|2446310|2446552|2445969|2446243|N|N|N|N|N| +2446335|AAAAAAAAPPDFFCAA|1985-09-26|1028|4474|344|1985|4|9|26|3|1985|344|4474|Thursday|1985Q3|N|N|N|2446310|2446552|2445970|2446244|N|N|N|N|N| +2446336|AAAAAAAAAAEFFCAA|1985-09-27|1028|4474|344|1985|5|9|27|3|1985|344|4474|Friday|1985Q3|N|Y|N|2446310|2446552|2445971|2446245|N|N|N|N|N| +2446337|AAAAAAAABAEFFCAA|1985-09-28|1028|4474|344|1985|6|9|28|3|1985|344|4474|Saturday|1985Q3|N|Y|N|2446310|2446552|2445972|2446246|N|N|N|N|N| +2446338|AAAAAAAACAEFFCAA|1985-09-29|1028|4474|344|1985|0|9|29|3|1985|344|4474|Sunday|1985Q3|N|N|N|2446310|2446552|2445973|2446247|N|N|N|N|N| +2446339|AAAAAAAADAEFFCAA|1985-09-30|1028|4474|344|1985|1|9|30|3|1985|344|4474|Monday|1985Q3|N|N|N|2446310|2446552|2445974|2446248|N|N|N|N|N| +2446340|AAAAAAAAEAEFFCAA|1985-10-01|1029|4475|344|1985|2|10|1|3|1985|344|4475|Tuesday|1985Q3|N|N|N|2446340|2446612|2445975|2446248|N|N|N|N|N| +2446341|AAAAAAAAFAEFFCAA|1985-10-02|1029|4475|344|1985|3|10|2|4|1985|344|4475|Wednesday|1985Q4|N|N|N|2446340|2446612|2445976|2446249|N|N|N|N|N| +2446342|AAAAAAAAGAEFFCAA|1985-10-03|1029|4475|344|1985|4|10|3|4|1985|344|4475|Thursday|1985Q4|N|N|N|2446340|2446612|2445977|2446250|N|N|N|N|N| +2446343|AAAAAAAAHAEFFCAA|1985-10-04|1029|4475|344|1985|5|10|4|4|1985|344|4475|Friday|1985Q4|N|Y|N|2446340|2446612|2445978|2446251|N|N|N|N|N| +2446344|AAAAAAAAIAEFFCAA|1985-10-05|1029|4475|344|1985|6|10|5|4|1985|344|4475|Saturday|1985Q4|N|Y|N|2446340|2446612|2445979|2446252|N|N|N|N|N| +2446345|AAAAAAAAJAEFFCAA|1985-10-06|1029|4475|344|1985|0|10|6|4|1985|344|4475|Sunday|1985Q4|N|N|N|2446340|2446612|2445980|2446253|N|N|N|N|N| +2446346|AAAAAAAAKAEFFCAA|1985-10-07|1029|4475|344|1985|1|10|7|4|1985|344|4475|Monday|1985Q4|N|N|N|2446340|2446612|2445981|2446254|N|N|N|N|N| +2446347|AAAAAAAALAEFFCAA|1985-10-08|1029|4476|344|1985|2|10|8|4|1985|344|4476|Tuesday|1985Q4|N|N|N|2446340|2446612|2445982|2446255|N|N|N|N|N| +2446348|AAAAAAAAMAEFFCAA|1985-10-09|1029|4476|344|1985|3|10|9|4|1985|344|4476|Wednesday|1985Q4|N|N|N|2446340|2446612|2445983|2446256|N|N|N|N|N| +2446349|AAAAAAAANAEFFCAA|1985-10-10|1029|4476|344|1985|4|10|10|4|1985|344|4476|Thursday|1985Q4|N|N|N|2446340|2446612|2445984|2446257|N|N|N|N|N| +2446350|AAAAAAAAOAEFFCAA|1985-10-11|1029|4476|344|1985|5|10|11|4|1985|344|4476|Friday|1985Q4|N|Y|N|2446340|2446612|2445985|2446258|N|N|N|N|N| +2446351|AAAAAAAAPAEFFCAA|1985-10-12|1029|4476|344|1985|6|10|12|4|1985|344|4476|Saturday|1985Q4|N|Y|N|2446340|2446612|2445986|2446259|N|N|N|N|N| +2446352|AAAAAAAAABEFFCAA|1985-10-13|1029|4476|344|1985|0|10|13|4|1985|344|4476|Sunday|1985Q4|N|N|N|2446340|2446612|2445987|2446260|N|N|N|N|N| +2446353|AAAAAAAABBEFFCAA|1985-10-14|1029|4476|344|1985|1|10|14|4|1985|344|4476|Monday|1985Q4|N|N|N|2446340|2446612|2445988|2446261|N|N|N|N|N| +2446354|AAAAAAAACBEFFCAA|1985-10-15|1029|4477|344|1985|2|10|15|4|1985|344|4477|Tuesday|1985Q4|N|N|N|2446340|2446612|2445989|2446262|N|N|N|N|N| +2446355|AAAAAAAADBEFFCAA|1985-10-16|1029|4477|344|1985|3|10|16|4|1985|344|4477|Wednesday|1985Q4|N|N|N|2446340|2446612|2445990|2446263|N|N|N|N|N| +2446356|AAAAAAAAEBEFFCAA|1985-10-17|1029|4477|344|1985|4|10|17|4|1985|344|4477|Thursday|1985Q4|N|N|N|2446340|2446612|2445991|2446264|N|N|N|N|N| +2446357|AAAAAAAAFBEFFCAA|1985-10-18|1029|4477|344|1985|5|10|18|4|1985|344|4477|Friday|1985Q4|N|Y|N|2446340|2446612|2445992|2446265|N|N|N|N|N| +2446358|AAAAAAAAGBEFFCAA|1985-10-19|1029|4477|344|1985|6|10|19|4|1985|344|4477|Saturday|1985Q4|N|Y|N|2446340|2446612|2445993|2446266|N|N|N|N|N| +2446359|AAAAAAAAHBEFFCAA|1985-10-20|1029|4477|344|1985|0|10|20|4|1985|344|4477|Sunday|1985Q4|N|N|N|2446340|2446612|2445994|2446267|N|N|N|N|N| +2446360|AAAAAAAAIBEFFCAA|1985-10-21|1029|4477|344|1985|1|10|21|4|1985|344|4477|Monday|1985Q4|N|N|N|2446340|2446612|2445995|2446268|N|N|N|N|N| +2446361|AAAAAAAAJBEFFCAA|1985-10-22|1029|4478|344|1985|2|10|22|4|1985|344|4478|Tuesday|1985Q4|N|N|N|2446340|2446612|2445996|2446269|N|N|N|N|N| +2446362|AAAAAAAAKBEFFCAA|1985-10-23|1029|4478|344|1985|3|10|23|4|1985|344|4478|Wednesday|1985Q4|N|N|N|2446340|2446612|2445997|2446270|N|N|N|N|N| +2446363|AAAAAAAALBEFFCAA|1985-10-24|1029|4478|344|1985|4|10|24|4|1985|344|4478|Thursday|1985Q4|N|N|N|2446340|2446612|2445998|2446271|N|N|N|N|N| +2446364|AAAAAAAAMBEFFCAA|1985-10-25|1029|4478|344|1985|5|10|25|4|1985|344|4478|Friday|1985Q4|N|Y|N|2446340|2446612|2445999|2446272|N|N|N|N|N| +2446365|AAAAAAAANBEFFCAA|1985-10-26|1029|4478|344|1985|6|10|26|4|1985|344|4478|Saturday|1985Q4|N|Y|N|2446340|2446612|2446000|2446273|N|N|N|N|N| +2446366|AAAAAAAAOBEFFCAA|1985-10-27|1029|4478|344|1985|0|10|27|4|1985|344|4478|Sunday|1985Q4|N|N|N|2446340|2446612|2446001|2446274|N|N|N|N|N| +2446367|AAAAAAAAPBEFFCAA|1985-10-28|1029|4478|344|1985|1|10|28|4|1985|344|4478|Monday|1985Q4|N|N|N|2446340|2446612|2446002|2446275|N|N|N|N|N| +2446368|AAAAAAAAACEFFCAA|1985-10-29|1029|4479|344|1985|2|10|29|4|1985|344|4479|Tuesday|1985Q4|N|N|N|2446340|2446612|2446003|2446276|N|N|N|N|N| +2446369|AAAAAAAABCEFFCAA|1985-10-30|1029|4479|344|1985|3|10|30|4|1985|344|4479|Wednesday|1985Q4|N|N|N|2446340|2446612|2446004|2446277|N|N|N|N|N| +2446370|AAAAAAAACCEFFCAA|1985-10-31|1029|4479|344|1985|4|10|31|4|1985|344|4479|Thursday|1985Q4|N|N|N|2446340|2446612|2446005|2446278|N|N|N|N|N| +2446371|AAAAAAAADCEFFCAA|1985-11-01|1030|4479|344|1985|5|11|1|4|1985|344|4479|Friday|1985Q4|N|Y|N|2446371|2446674|2446006|2446279|N|N|N|N|N| +2446372|AAAAAAAAECEFFCAA|1985-11-02|1030|4479|344|1985|6|11|2|4|1985|344|4479|Saturday|1985Q4|N|Y|N|2446371|2446674|2446007|2446280|N|N|N|N|N| +2446373|AAAAAAAAFCEFFCAA|1985-11-03|1030|4479|344|1985|0|11|3|4|1985|344|4479|Sunday|1985Q4|N|N|N|2446371|2446674|2446008|2446281|N|N|N|N|N| +2446374|AAAAAAAAGCEFFCAA|1985-11-04|1030|4479|344|1985|1|11|4|4|1985|344|4479|Monday|1985Q4|N|N|N|2446371|2446674|2446009|2446282|N|N|N|N|N| +2446375|AAAAAAAAHCEFFCAA|1985-11-05|1030|4480|344|1985|2|11|5|4|1985|344|4480|Tuesday|1985Q4|N|N|N|2446371|2446674|2446010|2446283|N|N|N|N|N| +2446376|AAAAAAAAICEFFCAA|1985-11-06|1030|4480|344|1985|3|11|6|4|1985|344|4480|Wednesday|1985Q4|N|N|N|2446371|2446674|2446011|2446284|N|N|N|N|N| +2446377|AAAAAAAAJCEFFCAA|1985-11-07|1030|4480|344|1985|4|11|7|4|1985|344|4480|Thursday|1985Q4|N|N|N|2446371|2446674|2446012|2446285|N|N|N|N|N| +2446378|AAAAAAAAKCEFFCAA|1985-11-08|1030|4480|344|1985|5|11|8|4|1985|344|4480|Friday|1985Q4|N|Y|N|2446371|2446674|2446013|2446286|N|N|N|N|N| +2446379|AAAAAAAALCEFFCAA|1985-11-09|1030|4480|344|1985|6|11|9|4|1985|344|4480|Saturday|1985Q4|N|Y|N|2446371|2446674|2446014|2446287|N|N|N|N|N| +2446380|AAAAAAAAMCEFFCAA|1985-11-10|1030|4480|344|1985|0|11|10|4|1985|344|4480|Sunday|1985Q4|N|N|N|2446371|2446674|2446015|2446288|N|N|N|N|N| +2446381|AAAAAAAANCEFFCAA|1985-11-11|1030|4480|344|1985|1|11|11|4|1985|344|4480|Monday|1985Q4|N|N|N|2446371|2446674|2446016|2446289|N|N|N|N|N| +2446382|AAAAAAAAOCEFFCAA|1985-11-12|1030|4481|344|1985|2|11|12|4|1985|344|4481|Tuesday|1985Q4|N|N|N|2446371|2446674|2446017|2446290|N|N|N|N|N| +2446383|AAAAAAAAPCEFFCAA|1985-11-13|1030|4481|344|1985|3|11|13|4|1985|344|4481|Wednesday|1985Q4|N|N|N|2446371|2446674|2446018|2446291|N|N|N|N|N| +2446384|AAAAAAAAADEFFCAA|1985-11-14|1030|4481|344|1985|4|11|14|4|1985|344|4481|Thursday|1985Q4|N|N|N|2446371|2446674|2446019|2446292|N|N|N|N|N| +2446385|AAAAAAAABDEFFCAA|1985-11-15|1030|4481|344|1985|5|11|15|4|1985|344|4481|Friday|1985Q4|N|Y|N|2446371|2446674|2446020|2446293|N|N|N|N|N| +2446386|AAAAAAAACDEFFCAA|1985-11-16|1030|4481|344|1985|6|11|16|4|1985|344|4481|Saturday|1985Q4|N|Y|N|2446371|2446674|2446021|2446294|N|N|N|N|N| +2446387|AAAAAAAADDEFFCAA|1985-11-17|1030|4481|344|1985|0|11|17|4|1985|344|4481|Sunday|1985Q4|N|N|N|2446371|2446674|2446022|2446295|N|N|N|N|N| +2446388|AAAAAAAAEDEFFCAA|1985-11-18|1030|4481|344|1985|1|11|18|4|1985|344|4481|Monday|1985Q4|N|N|N|2446371|2446674|2446023|2446296|N|N|N|N|N| +2446389|AAAAAAAAFDEFFCAA|1985-11-19|1030|4482|344|1985|2|11|19|4|1985|344|4482|Tuesday|1985Q4|N|N|N|2446371|2446674|2446024|2446297|N|N|N|N|N| +2446390|AAAAAAAAGDEFFCAA|1985-11-20|1030|4482|344|1985|3|11|20|4|1985|344|4482|Wednesday|1985Q4|N|N|N|2446371|2446674|2446025|2446298|N|N|N|N|N| +2446391|AAAAAAAAHDEFFCAA|1985-11-21|1030|4482|344|1985|4|11|21|4|1985|344|4482|Thursday|1985Q4|N|N|N|2446371|2446674|2446026|2446299|N|N|N|N|N| +2446392|AAAAAAAAIDEFFCAA|1985-11-22|1030|4482|344|1985|5|11|22|4|1985|344|4482|Friday|1985Q4|N|Y|N|2446371|2446674|2446027|2446300|N|N|N|N|N| +2446393|AAAAAAAAJDEFFCAA|1985-11-23|1030|4482|344|1985|6|11|23|4|1985|344|4482|Saturday|1985Q4|N|Y|N|2446371|2446674|2446028|2446301|N|N|N|N|N| +2446394|AAAAAAAAKDEFFCAA|1985-11-24|1030|4482|344|1985|0|11|24|4|1985|344|4482|Sunday|1985Q4|N|N|N|2446371|2446674|2446029|2446302|N|N|N|N|N| +2446395|AAAAAAAALDEFFCAA|1985-11-25|1030|4482|344|1985|1|11|25|4|1985|344|4482|Monday|1985Q4|N|N|N|2446371|2446674|2446030|2446303|N|N|N|N|N| +2446396|AAAAAAAAMDEFFCAA|1985-11-26|1030|4483|344|1985|2|11|26|4|1985|344|4483|Tuesday|1985Q4|N|N|N|2446371|2446674|2446031|2446304|N|N|N|N|N| +2446397|AAAAAAAANDEFFCAA|1985-11-27|1030|4483|344|1985|3|11|27|4|1985|344|4483|Wednesday|1985Q4|N|N|N|2446371|2446674|2446032|2446305|N|N|N|N|N| +2446398|AAAAAAAAODEFFCAA|1985-11-28|1030|4483|344|1985|4|11|28|4|1985|344|4483|Thursday|1985Q4|N|N|N|2446371|2446674|2446033|2446306|N|N|N|N|N| +2446399|AAAAAAAAPDEFFCAA|1985-11-29|1030|4483|344|1985|5|11|29|4|1985|344|4483|Friday|1985Q4|N|Y|N|2446371|2446674|2446034|2446307|N|N|N|N|N| +2446400|AAAAAAAAAEEFFCAA|1985-11-30|1030|4483|344|1985|6|11|30|4|1985|344|4483|Saturday|1985Q4|N|Y|N|2446371|2446674|2446035|2446308|N|N|N|N|N| +2446401|AAAAAAAABEEFFCAA|1985-12-01|1031|4483|345|1985|0|12|1|4|1985|345|4483|Sunday|1985Q4|N|N|N|2446401|2446734|2446036|2446309|N|N|N|N|N| +2446402|AAAAAAAACEEFFCAA|1985-12-02|1031|4483|345|1985|1|12|2|4|1985|345|4483|Monday|1985Q4|N|N|N|2446401|2446734|2446037|2446310|N|N|N|N|N| +2446403|AAAAAAAADEEFFCAA|1985-12-03|1031|4484|345|1985|2|12|3|4|1985|345|4484|Tuesday|1985Q4|N|N|N|2446401|2446734|2446038|2446311|N|N|N|N|N| +2446404|AAAAAAAAEEEFFCAA|1985-12-04|1031|4484|345|1985|3|12|4|4|1985|345|4484|Wednesday|1985Q4|N|N|N|2446401|2446734|2446039|2446312|N|N|N|N|N| +2446405|AAAAAAAAFEEFFCAA|1985-12-05|1031|4484|345|1985|4|12|5|4|1985|345|4484|Thursday|1985Q4|N|N|N|2446401|2446734|2446040|2446313|N|N|N|N|N| +2446406|AAAAAAAAGEEFFCAA|1985-12-06|1031|4484|345|1985|5|12|6|4|1985|345|4484|Friday|1985Q4|N|Y|N|2446401|2446734|2446041|2446314|N|N|N|N|N| +2446407|AAAAAAAAHEEFFCAA|1985-12-07|1031|4484|345|1985|6|12|7|4|1985|345|4484|Saturday|1985Q4|N|Y|N|2446401|2446734|2446042|2446315|N|N|N|N|N| +2446408|AAAAAAAAIEEFFCAA|1985-12-08|1031|4484|345|1985|0|12|8|4|1985|345|4484|Sunday|1985Q4|N|N|N|2446401|2446734|2446043|2446316|N|N|N|N|N| +2446409|AAAAAAAAJEEFFCAA|1985-12-09|1031|4484|345|1985|1|12|9|4|1985|345|4484|Monday|1985Q4|N|N|N|2446401|2446734|2446044|2446317|N|N|N|N|N| +2446410|AAAAAAAAKEEFFCAA|1985-12-10|1031|4485|345|1985|2|12|10|4|1985|345|4485|Tuesday|1985Q4|N|N|N|2446401|2446734|2446045|2446318|N|N|N|N|N| +2446411|AAAAAAAALEEFFCAA|1985-12-11|1031|4485|345|1985|3|12|11|4|1985|345|4485|Wednesday|1985Q4|N|N|N|2446401|2446734|2446046|2446319|N|N|N|N|N| +2446412|AAAAAAAAMEEFFCAA|1985-12-12|1031|4485|345|1985|4|12|12|4|1985|345|4485|Thursday|1985Q4|N|N|N|2446401|2446734|2446047|2446320|N|N|N|N|N| +2446413|AAAAAAAANEEFFCAA|1985-12-13|1031|4485|345|1985|5|12|13|4|1985|345|4485|Friday|1985Q4|N|Y|N|2446401|2446734|2446048|2446321|N|N|N|N|N| +2446414|AAAAAAAAOEEFFCAA|1985-12-14|1031|4485|345|1985|6|12|14|4|1985|345|4485|Saturday|1985Q4|N|Y|N|2446401|2446734|2446049|2446322|N|N|N|N|N| +2446415|AAAAAAAAPEEFFCAA|1985-12-15|1031|4485|345|1985|0|12|15|4|1985|345|4485|Sunday|1985Q4|N|N|N|2446401|2446734|2446050|2446323|N|N|N|N|N| +2446416|AAAAAAAAAFEFFCAA|1985-12-16|1031|4485|345|1985|1|12|16|4|1985|345|4485|Monday|1985Q4|N|N|N|2446401|2446734|2446051|2446324|N|N|N|N|N| +2446417|AAAAAAAABFEFFCAA|1985-12-17|1031|4486|345|1985|2|12|17|4|1985|345|4486|Tuesday|1985Q4|N|N|N|2446401|2446734|2446052|2446325|N|N|N|N|N| +2446418|AAAAAAAACFEFFCAA|1985-12-18|1031|4486|345|1985|3|12|18|4|1985|345|4486|Wednesday|1985Q4|N|N|N|2446401|2446734|2446053|2446326|N|N|N|N|N| +2446419|AAAAAAAADFEFFCAA|1985-12-19|1031|4486|345|1985|4|12|19|4|1985|345|4486|Thursday|1985Q4|N|N|N|2446401|2446734|2446054|2446327|N|N|N|N|N| +2446420|AAAAAAAAEFEFFCAA|1985-12-20|1031|4486|345|1985|5|12|20|4|1985|345|4486|Friday|1985Q4|N|Y|N|2446401|2446734|2446055|2446328|N|N|N|N|N| +2446421|AAAAAAAAFFEFFCAA|1985-12-21|1031|4486|345|1985|6|12|21|4|1985|345|4486|Saturday|1985Q4|N|Y|N|2446401|2446734|2446056|2446329|N|N|N|N|N| +2446422|AAAAAAAAGFEFFCAA|1985-12-22|1031|4486|345|1985|0|12|22|4|1985|345|4486|Sunday|1985Q4|N|N|N|2446401|2446734|2446057|2446330|N|N|N|N|N| +2446423|AAAAAAAAHFEFFCAA|1985-12-23|1031|4486|345|1985|1|12|23|4|1985|345|4486|Monday|1985Q4|N|N|N|2446401|2446734|2446058|2446331|N|N|N|N|N| +2446424|AAAAAAAAIFEFFCAA|1985-12-24|1031|4487|345|1985|2|12|24|4|1985|345|4487|Tuesday|1985Q4|N|N|N|2446401|2446734|2446059|2446332|N|N|N|N|N| +2446425|AAAAAAAAJFEFFCAA|1985-12-25|1031|4487|345|1985|3|12|25|4|1985|345|4487|Wednesday|1985Q4|N|N|N|2446401|2446734|2446060|2446333|N|N|N|N|N| +2446426|AAAAAAAAKFEFFCAA|1985-12-26|1031|4487|345|1985|4|12|26|4|1985|345|4487|Thursday|1985Q4|Y|N|N|2446401|2446734|2446061|2446334|N|N|N|N|N| +2446427|AAAAAAAALFEFFCAA|1985-12-27|1031|4487|345|1985|5|12|27|4|1985|345|4487|Friday|1985Q4|N|Y|Y|2446401|2446734|2446062|2446335|N|N|N|N|N| +2446428|AAAAAAAAMFEFFCAA|1985-12-28|1031|4487|345|1985|6|12|28|4|1985|345|4487|Saturday|1985Q4|N|Y|N|2446401|2446734|2446063|2446336|N|N|N|N|N| +2446429|AAAAAAAANFEFFCAA|1985-12-29|1031|4487|345|1985|0|12|29|4|1985|345|4487|Sunday|1985Q4|N|N|N|2446401|2446734|2446064|2446337|N|N|N|N|N| +2446430|AAAAAAAAOFEFFCAA|1985-12-30|1031|4487|345|1985|1|12|30|4|1985|345|4487|Monday|1985Q4|N|N|N|2446401|2446734|2446065|2446338|N|N|N|N|N| +2446431|AAAAAAAAPFEFFCAA|1985-12-31|1031|4488|345|1985|2|12|31|4|1985|345|4488|Tuesday|1985Q4|N|N|N|2446401|2446734|2446066|2446339|N|N|N|N|N| +2446432|AAAAAAAAAGEFFCAA|1986-01-01|1032|4488|345|1986|3|1|1|1|1986|345|4488|Wednesday|1986Q1|Y|N|N|2446432|2446431|2446067|2446340|N|N|N|N|N| +2446433|AAAAAAAABGEFFCAA|1986-01-02|1032|4488|345|1986|4|1|2|1|1986|345|4488|Thursday|1986Q1|N|N|Y|2446432|2446431|2446068|2446341|N|N|N|N|N| +2446434|AAAAAAAACGEFFCAA|1986-01-03|1032|4488|345|1986|5|1|3|1|1986|345|4488|Friday|1986Q1|N|Y|N|2446432|2446431|2446069|2446342|N|N|N|N|N| +2446435|AAAAAAAADGEFFCAA|1986-01-04|1032|4488|345|1986|6|1|4|1|1986|345|4488|Saturday|1986Q1|N|Y|N|2446432|2446431|2446070|2446343|N|N|N|N|N| +2446436|AAAAAAAAEGEFFCAA|1986-01-05|1032|4488|345|1986|0|1|5|1|1986|345|4488|Sunday|1986Q1|N|N|N|2446432|2446431|2446071|2446344|N|N|N|N|N| +2446437|AAAAAAAAFGEFFCAA|1986-01-06|1032|4488|345|1986|1|1|6|1|1986|345|4488|Monday|1986Q1|N|N|N|2446432|2446431|2446072|2446345|N|N|N|N|N| +2446438|AAAAAAAAGGEFFCAA|1986-01-07|1032|4489|345|1986|2|1|7|1|1986|345|4489|Tuesday|1986Q1|N|N|N|2446432|2446431|2446073|2446346|N|N|N|N|N| +2446439|AAAAAAAAHGEFFCAA|1986-01-08|1032|4489|345|1986|3|1|8|1|1986|345|4489|Wednesday|1986Q1|N|N|N|2446432|2446431|2446074|2446347|N|N|N|N|N| +2446440|AAAAAAAAIGEFFCAA|1986-01-09|1032|4489|345|1986|4|1|9|1|1986|345|4489|Thursday|1986Q1|N|N|N|2446432|2446431|2446075|2446348|N|N|N|N|N| +2446441|AAAAAAAAJGEFFCAA|1986-01-10|1032|4489|345|1986|5|1|10|1|1986|345|4489|Friday|1986Q1|N|Y|N|2446432|2446431|2446076|2446349|N|N|N|N|N| +2446442|AAAAAAAAKGEFFCAA|1986-01-11|1032|4489|345|1986|6|1|11|1|1986|345|4489|Saturday|1986Q1|N|Y|N|2446432|2446431|2446077|2446350|N|N|N|N|N| +2446443|AAAAAAAALGEFFCAA|1986-01-12|1032|4489|345|1986|0|1|12|1|1986|345|4489|Sunday|1986Q1|N|N|N|2446432|2446431|2446078|2446351|N|N|N|N|N| +2446444|AAAAAAAAMGEFFCAA|1986-01-13|1032|4489|345|1986|1|1|13|1|1986|345|4489|Monday|1986Q1|N|N|N|2446432|2446431|2446079|2446352|N|N|N|N|N| +2446445|AAAAAAAANGEFFCAA|1986-01-14|1032|4490|345|1986|2|1|14|1|1986|345|4490|Tuesday|1986Q1|N|N|N|2446432|2446431|2446080|2446353|N|N|N|N|N| +2446446|AAAAAAAAOGEFFCAA|1986-01-15|1032|4490|345|1986|3|1|15|1|1986|345|4490|Wednesday|1986Q1|N|N|N|2446432|2446431|2446081|2446354|N|N|N|N|N| +2446447|AAAAAAAAPGEFFCAA|1986-01-16|1032|4490|345|1986|4|1|16|1|1986|345|4490|Thursday|1986Q1|N|N|N|2446432|2446431|2446082|2446355|N|N|N|N|N| +2446448|AAAAAAAAAHEFFCAA|1986-01-17|1032|4490|345|1986|5|1|17|1|1986|345|4490|Friday|1986Q1|N|Y|N|2446432|2446431|2446083|2446356|N|N|N|N|N| +2446449|AAAAAAAABHEFFCAA|1986-01-18|1032|4490|345|1986|6|1|18|1|1986|345|4490|Saturday|1986Q1|N|Y|N|2446432|2446431|2446084|2446357|N|N|N|N|N| +2446450|AAAAAAAACHEFFCAA|1986-01-19|1032|4490|345|1986|0|1|19|1|1986|345|4490|Sunday|1986Q1|N|N|N|2446432|2446431|2446085|2446358|N|N|N|N|N| +2446451|AAAAAAAADHEFFCAA|1986-01-20|1032|4490|345|1986|1|1|20|1|1986|345|4490|Monday|1986Q1|N|N|N|2446432|2446431|2446086|2446359|N|N|N|N|N| +2446452|AAAAAAAAEHEFFCAA|1986-01-21|1032|4491|345|1986|2|1|21|1|1986|345|4491|Tuesday|1986Q1|N|N|N|2446432|2446431|2446087|2446360|N|N|N|N|N| +2446453|AAAAAAAAFHEFFCAA|1986-01-22|1032|4491|345|1986|3|1|22|1|1986|345|4491|Wednesday|1986Q1|N|N|N|2446432|2446431|2446088|2446361|N|N|N|N|N| +2446454|AAAAAAAAGHEFFCAA|1986-01-23|1032|4491|345|1986|4|1|23|1|1986|345|4491|Thursday|1986Q1|N|N|N|2446432|2446431|2446089|2446362|N|N|N|N|N| +2446455|AAAAAAAAHHEFFCAA|1986-01-24|1032|4491|345|1986|5|1|24|1|1986|345|4491|Friday|1986Q1|N|Y|N|2446432|2446431|2446090|2446363|N|N|N|N|N| +2446456|AAAAAAAAIHEFFCAA|1986-01-25|1032|4491|345|1986|6|1|25|1|1986|345|4491|Saturday|1986Q1|N|Y|N|2446432|2446431|2446091|2446364|N|N|N|N|N| +2446457|AAAAAAAAJHEFFCAA|1986-01-26|1032|4491|345|1986|0|1|26|1|1986|345|4491|Sunday|1986Q1|N|N|N|2446432|2446431|2446092|2446365|N|N|N|N|N| +2446458|AAAAAAAAKHEFFCAA|1986-01-27|1032|4491|345|1986|1|1|27|1|1986|345|4491|Monday|1986Q1|N|N|N|2446432|2446431|2446093|2446366|N|N|N|N|N| +2446459|AAAAAAAALHEFFCAA|1986-01-28|1032|4492|345|1986|2|1|28|1|1986|345|4492|Tuesday|1986Q1|N|N|N|2446432|2446431|2446094|2446367|N|N|N|N|N| +2446460|AAAAAAAAMHEFFCAA|1986-01-29|1032|4492|345|1986|3|1|29|1|1986|345|4492|Wednesday|1986Q1|N|N|N|2446432|2446431|2446095|2446368|N|N|N|N|N| +2446461|AAAAAAAANHEFFCAA|1986-01-30|1032|4492|345|1986|4|1|30|1|1986|345|4492|Thursday|1986Q1|N|N|N|2446432|2446431|2446096|2446369|N|N|N|N|N| +2446462|AAAAAAAAOHEFFCAA|1986-01-31|1032|4492|345|1986|5|1|31|1|1986|345|4492|Friday|1986Q1|N|Y|N|2446432|2446431|2446097|2446370|N|N|N|N|N| +2446463|AAAAAAAAPHEFFCAA|1986-02-01|1033|4492|345|1986|6|2|1|1|1986|345|4492|Saturday|1986Q1|N|Y|N|2446463|2446493|2446098|2446371|N|N|N|N|N| +2446464|AAAAAAAAAIEFFCAA|1986-02-02|1033|4492|345|1986|0|2|2|1|1986|345|4492|Sunday|1986Q1|N|N|N|2446463|2446493|2446099|2446372|N|N|N|N|N| +2446465|AAAAAAAABIEFFCAA|1986-02-03|1033|4492|345|1986|1|2|3|1|1986|345|4492|Monday|1986Q1|N|N|N|2446463|2446493|2446100|2446373|N|N|N|N|N| +2446466|AAAAAAAACIEFFCAA|1986-02-04|1033|4493|345|1986|2|2|4|1|1986|345|4493|Tuesday|1986Q1|N|N|N|2446463|2446493|2446101|2446374|N|N|N|N|N| +2446467|AAAAAAAADIEFFCAA|1986-02-05|1033|4493|345|1986|3|2|5|1|1986|345|4493|Wednesday|1986Q1|N|N|N|2446463|2446493|2446102|2446375|N|N|N|N|N| +2446468|AAAAAAAAEIEFFCAA|1986-02-06|1033|4493|345|1986|4|2|6|1|1986|345|4493|Thursday|1986Q1|N|N|N|2446463|2446493|2446103|2446376|N|N|N|N|N| +2446469|AAAAAAAAFIEFFCAA|1986-02-07|1033|4493|345|1986|5|2|7|1|1986|345|4493|Friday|1986Q1|N|Y|N|2446463|2446493|2446104|2446377|N|N|N|N|N| +2446470|AAAAAAAAGIEFFCAA|1986-02-08|1033|4493|345|1986|6|2|8|1|1986|345|4493|Saturday|1986Q1|N|Y|N|2446463|2446493|2446105|2446378|N|N|N|N|N| +2446471|AAAAAAAAHIEFFCAA|1986-02-09|1033|4493|345|1986|0|2|9|1|1986|345|4493|Sunday|1986Q1|N|N|N|2446463|2446493|2446106|2446379|N|N|N|N|N| +2446472|AAAAAAAAIIEFFCAA|1986-02-10|1033|4493|345|1986|1|2|10|1|1986|345|4493|Monday|1986Q1|N|N|N|2446463|2446493|2446107|2446380|N|N|N|N|N| +2446473|AAAAAAAAJIEFFCAA|1986-02-11|1033|4494|345|1986|2|2|11|1|1986|345|4494|Tuesday|1986Q1|N|N|N|2446463|2446493|2446108|2446381|N|N|N|N|N| +2446474|AAAAAAAAKIEFFCAA|1986-02-12|1033|4494|345|1986|3|2|12|1|1986|345|4494|Wednesday|1986Q1|N|N|N|2446463|2446493|2446109|2446382|N|N|N|N|N| +2446475|AAAAAAAALIEFFCAA|1986-02-13|1033|4494|345|1986|4|2|13|1|1986|345|4494|Thursday|1986Q1|N|N|N|2446463|2446493|2446110|2446383|N|N|N|N|N| +2446476|AAAAAAAAMIEFFCAA|1986-02-14|1033|4494|345|1986|5|2|14|1|1986|345|4494|Friday|1986Q1|N|Y|N|2446463|2446493|2446111|2446384|N|N|N|N|N| +2446477|AAAAAAAANIEFFCAA|1986-02-15|1033|4494|345|1986|6|2|15|1|1986|345|4494|Saturday|1986Q1|N|Y|N|2446463|2446493|2446112|2446385|N|N|N|N|N| +2446478|AAAAAAAAOIEFFCAA|1986-02-16|1033|4494|345|1986|0|2|16|1|1986|345|4494|Sunday|1986Q1|N|N|N|2446463|2446493|2446113|2446386|N|N|N|N|N| +2446479|AAAAAAAAPIEFFCAA|1986-02-17|1033|4494|345|1986|1|2|17|1|1986|345|4494|Monday|1986Q1|N|N|N|2446463|2446493|2446114|2446387|N|N|N|N|N| +2446480|AAAAAAAAAJEFFCAA|1986-02-18|1033|4495|345|1986|2|2|18|1|1986|345|4495|Tuesday|1986Q1|N|N|N|2446463|2446493|2446115|2446388|N|N|N|N|N| +2446481|AAAAAAAABJEFFCAA|1986-02-19|1033|4495|345|1986|3|2|19|1|1986|345|4495|Wednesday|1986Q1|N|N|N|2446463|2446493|2446116|2446389|N|N|N|N|N| +2446482|AAAAAAAACJEFFCAA|1986-02-20|1033|4495|345|1986|4|2|20|1|1986|345|4495|Thursday|1986Q1|N|N|N|2446463|2446493|2446117|2446390|N|N|N|N|N| +2446483|AAAAAAAADJEFFCAA|1986-02-21|1033|4495|345|1986|5|2|21|1|1986|345|4495|Friday|1986Q1|N|Y|N|2446463|2446493|2446118|2446391|N|N|N|N|N| +2446484|AAAAAAAAEJEFFCAA|1986-02-22|1033|4495|345|1986|6|2|22|1|1986|345|4495|Saturday|1986Q1|N|Y|N|2446463|2446493|2446119|2446392|N|N|N|N|N| +2446485|AAAAAAAAFJEFFCAA|1986-02-23|1033|4495|345|1986|0|2|23|1|1986|345|4495|Sunday|1986Q1|N|N|N|2446463|2446493|2446120|2446393|N|N|N|N|N| +2446486|AAAAAAAAGJEFFCAA|1986-02-24|1033|4495|345|1986|1|2|24|1|1986|345|4495|Monday|1986Q1|N|N|N|2446463|2446493|2446121|2446394|N|N|N|N|N| +2446487|AAAAAAAAHJEFFCAA|1986-02-25|1033|4496|345|1986|2|2|25|1|1986|345|4496|Tuesday|1986Q1|N|N|N|2446463|2446493|2446122|2446395|N|N|N|N|N| +2446488|AAAAAAAAIJEFFCAA|1986-02-26|1033|4496|345|1986|3|2|26|1|1986|345|4496|Wednesday|1986Q1|N|N|N|2446463|2446493|2446123|2446396|N|N|N|N|N| +2446489|AAAAAAAAJJEFFCAA|1986-02-27|1033|4496|345|1986|4|2|27|1|1986|345|4496|Thursday|1986Q1|N|N|N|2446463|2446493|2446124|2446397|N|N|N|N|N| +2446490|AAAAAAAAKJEFFCAA|1986-02-28|1033|4496|345|1986|5|2|28|1|1986|345|4496|Friday|1986Q1|N|Y|N|2446463|2446493|2446125|2446398|N|N|N|N|N| +2446491|AAAAAAAALJEFFCAA|1986-03-01|1034|4496|346|1986|6|3|1|1|1986|346|4496|Saturday|1986Q1|N|Y|N|2446491|2446549|2446126|2446399|N|N|N|N|N| +2446492|AAAAAAAAMJEFFCAA|1986-03-02|1034|4496|346|1986|0|3|2|1|1986|346|4496|Sunday|1986Q1|N|N|N|2446491|2446549|2446127|2446400|N|N|N|N|N| +2446493|AAAAAAAANJEFFCAA|1986-03-03|1034|4496|346|1986|1|3|3|1|1986|346|4496|Monday|1986Q1|N|N|N|2446491|2446549|2446128|2446401|N|N|N|N|N| +2446494|AAAAAAAAOJEFFCAA|1986-03-04|1034|4497|346|1986|2|3|4|1|1986|346|4497|Tuesday|1986Q1|N|N|N|2446491|2446549|2446129|2446402|N|N|N|N|N| +2446495|AAAAAAAAPJEFFCAA|1986-03-05|1034|4497|346|1986|3|3|5|1|1986|346|4497|Wednesday|1986Q1|N|N|N|2446491|2446549|2446130|2446403|N|N|N|N|N| +2446496|AAAAAAAAAKEFFCAA|1986-03-06|1034|4497|346|1986|4|3|6|1|1986|346|4497|Thursday|1986Q1|N|N|N|2446491|2446549|2446131|2446404|N|N|N|N|N| +2446497|AAAAAAAABKEFFCAA|1986-03-07|1034|4497|346|1986|5|3|7|1|1986|346|4497|Friday|1986Q1|N|Y|N|2446491|2446549|2446132|2446405|N|N|N|N|N| +2446498|AAAAAAAACKEFFCAA|1986-03-08|1034|4497|346|1986|6|3|8|1|1986|346|4497|Saturday|1986Q1|N|Y|N|2446491|2446549|2446133|2446406|N|N|N|N|N| +2446499|AAAAAAAADKEFFCAA|1986-03-09|1034|4497|346|1986|0|3|9|1|1986|346|4497|Sunday|1986Q1|N|N|N|2446491|2446549|2446134|2446407|N|N|N|N|N| +2446500|AAAAAAAAEKEFFCAA|1986-03-10|1034|4497|346|1986|1|3|10|1|1986|346|4497|Monday|1986Q1|N|N|N|2446491|2446549|2446135|2446408|N|N|N|N|N| +2446501|AAAAAAAAFKEFFCAA|1986-03-11|1034|4498|346|1986|2|3|11|1|1986|346|4498|Tuesday|1986Q1|N|N|N|2446491|2446549|2446136|2446409|N|N|N|N|N| +2446502|AAAAAAAAGKEFFCAA|1986-03-12|1034|4498|346|1986|3|3|12|1|1986|346|4498|Wednesday|1986Q1|N|N|N|2446491|2446549|2446137|2446410|N|N|N|N|N| +2446503|AAAAAAAAHKEFFCAA|1986-03-13|1034|4498|346|1986|4|3|13|1|1986|346|4498|Thursday|1986Q1|N|N|N|2446491|2446549|2446138|2446411|N|N|N|N|N| +2446504|AAAAAAAAIKEFFCAA|1986-03-14|1034|4498|346|1986|5|3|14|1|1986|346|4498|Friday|1986Q1|N|Y|N|2446491|2446549|2446139|2446412|N|N|N|N|N| +2446505|AAAAAAAAJKEFFCAA|1986-03-15|1034|4498|346|1986|6|3|15|1|1986|346|4498|Saturday|1986Q1|N|Y|N|2446491|2446549|2446140|2446413|N|N|N|N|N| +2446506|AAAAAAAAKKEFFCAA|1986-03-16|1034|4498|346|1986|0|3|16|1|1986|346|4498|Sunday|1986Q1|N|N|N|2446491|2446549|2446141|2446414|N|N|N|N|N| +2446507|AAAAAAAALKEFFCAA|1986-03-17|1034|4498|346|1986|1|3|17|1|1986|346|4498|Monday|1986Q1|N|N|N|2446491|2446549|2446142|2446415|N|N|N|N|N| +2446508|AAAAAAAAMKEFFCAA|1986-03-18|1034|4499|346|1986|2|3|18|1|1986|346|4499|Tuesday|1986Q1|N|N|N|2446491|2446549|2446143|2446416|N|N|N|N|N| +2446509|AAAAAAAANKEFFCAA|1986-03-19|1034|4499|346|1986|3|3|19|1|1986|346|4499|Wednesday|1986Q1|N|N|N|2446491|2446549|2446144|2446417|N|N|N|N|N| +2446510|AAAAAAAAOKEFFCAA|1986-03-20|1034|4499|346|1986|4|3|20|1|1986|346|4499|Thursday|1986Q1|N|N|N|2446491|2446549|2446145|2446418|N|N|N|N|N| +2446511|AAAAAAAAPKEFFCAA|1986-03-21|1034|4499|346|1986|5|3|21|1|1986|346|4499|Friday|1986Q1|N|Y|N|2446491|2446549|2446146|2446419|N|N|N|N|N| +2446512|AAAAAAAAALEFFCAA|1986-03-22|1034|4499|346|1986|6|3|22|1|1986|346|4499|Saturday|1986Q1|N|Y|N|2446491|2446549|2446147|2446420|N|N|N|N|N| +2446513|AAAAAAAABLEFFCAA|1986-03-23|1034|4499|346|1986|0|3|23|1|1986|346|4499|Sunday|1986Q1|N|N|N|2446491|2446549|2446148|2446421|N|N|N|N|N| +2446514|AAAAAAAACLEFFCAA|1986-03-24|1034|4499|346|1986|1|3|24|1|1986|346|4499|Monday|1986Q1|N|N|N|2446491|2446549|2446149|2446422|N|N|N|N|N| +2446515|AAAAAAAADLEFFCAA|1986-03-25|1034|4500|346|1986|2|3|25|1|1986|346|4500|Tuesday|1986Q1|N|N|N|2446491|2446549|2446150|2446423|N|N|N|N|N| +2446516|AAAAAAAAELEFFCAA|1986-03-26|1034|4500|346|1986|3|3|26|1|1986|346|4500|Wednesday|1986Q1|N|N|N|2446491|2446549|2446151|2446424|N|N|N|N|N| +2446517|AAAAAAAAFLEFFCAA|1986-03-27|1034|4500|346|1986|4|3|27|1|1986|346|4500|Thursday|1986Q1|N|N|N|2446491|2446549|2446152|2446425|N|N|N|N|N| +2446518|AAAAAAAAGLEFFCAA|1986-03-28|1034|4500|346|1986|5|3|28|1|1986|346|4500|Friday|1986Q1|N|Y|N|2446491|2446549|2446153|2446426|N|N|N|N|N| +2446519|AAAAAAAAHLEFFCAA|1986-03-29|1034|4500|346|1986|6|3|29|1|1986|346|4500|Saturday|1986Q1|N|Y|N|2446491|2446549|2446154|2446427|N|N|N|N|N| +2446520|AAAAAAAAILEFFCAA|1986-03-30|1034|4500|346|1986|0|3|30|1|1986|346|4500|Sunday|1986Q1|N|N|N|2446491|2446549|2446155|2446428|N|N|N|N|N| +2446521|AAAAAAAAJLEFFCAA|1986-03-31|1034|4500|346|1986|1|3|31|1|1986|346|4500|Monday|1986Q1|N|N|N|2446491|2446549|2446156|2446429|N|N|N|N|N| +2446522|AAAAAAAAKLEFFCAA|1986-04-01|1035|4501|346|1986|2|4|1|1|1986|346|4501|Tuesday|1986Q1|N|N|N|2446522|2446611|2446157|2446432|N|N|N|N|N| +2446523|AAAAAAAALLEFFCAA|1986-04-02|1035|4501|346|1986|3|4|2|2|1986|346|4501|Wednesday|1986Q2|N|N|N|2446522|2446611|2446158|2446433|N|N|N|N|N| +2446524|AAAAAAAAMLEFFCAA|1986-04-03|1035|4501|346|1986|4|4|3|2|1986|346|4501|Thursday|1986Q2|N|N|N|2446522|2446611|2446159|2446434|N|N|N|N|N| +2446525|AAAAAAAANLEFFCAA|1986-04-04|1035|4501|346|1986|5|4|4|2|1986|346|4501|Friday|1986Q2|N|Y|N|2446522|2446611|2446160|2446435|N|N|N|N|N| +2446526|AAAAAAAAOLEFFCAA|1986-04-05|1035|4501|346|1986|6|4|5|2|1986|346|4501|Saturday|1986Q2|N|Y|N|2446522|2446611|2446161|2446436|N|N|N|N|N| +2446527|AAAAAAAAPLEFFCAA|1986-04-06|1035|4501|346|1986|0|4|6|2|1986|346|4501|Sunday|1986Q2|N|N|N|2446522|2446611|2446162|2446437|N|N|N|N|N| +2446528|AAAAAAAAAMEFFCAA|1986-04-07|1035|4501|346|1986|1|4|7|2|1986|346|4501|Monday|1986Q2|N|N|N|2446522|2446611|2446163|2446438|N|N|N|N|N| +2446529|AAAAAAAABMEFFCAA|1986-04-08|1035|4502|346|1986|2|4|8|2|1986|346|4502|Tuesday|1986Q2|N|N|N|2446522|2446611|2446164|2446439|N|N|N|N|N| +2446530|AAAAAAAACMEFFCAA|1986-04-09|1035|4502|346|1986|3|4|9|2|1986|346|4502|Wednesday|1986Q2|N|N|N|2446522|2446611|2446165|2446440|N|N|N|N|N| +2446531|AAAAAAAADMEFFCAA|1986-04-10|1035|4502|346|1986|4|4|10|2|1986|346|4502|Thursday|1986Q2|N|N|N|2446522|2446611|2446166|2446441|N|N|N|N|N| +2446532|AAAAAAAAEMEFFCAA|1986-04-11|1035|4502|346|1986|5|4|11|2|1986|346|4502|Friday|1986Q2|N|Y|N|2446522|2446611|2446167|2446442|N|N|N|N|N| +2446533|AAAAAAAAFMEFFCAA|1986-04-12|1035|4502|346|1986|6|4|12|2|1986|346|4502|Saturday|1986Q2|N|Y|N|2446522|2446611|2446168|2446443|N|N|N|N|N| +2446534|AAAAAAAAGMEFFCAA|1986-04-13|1035|4502|346|1986|0|4|13|2|1986|346|4502|Sunday|1986Q2|N|N|N|2446522|2446611|2446169|2446444|N|N|N|N|N| +2446535|AAAAAAAAHMEFFCAA|1986-04-14|1035|4502|346|1986|1|4|14|2|1986|346|4502|Monday|1986Q2|N|N|N|2446522|2446611|2446170|2446445|N|N|N|N|N| +2446536|AAAAAAAAIMEFFCAA|1986-04-15|1035|4503|346|1986|2|4|15|2|1986|346|4503|Tuesday|1986Q2|N|N|N|2446522|2446611|2446171|2446446|N|N|N|N|N| +2446537|AAAAAAAAJMEFFCAA|1986-04-16|1035|4503|346|1986|3|4|16|2|1986|346|4503|Wednesday|1986Q2|N|N|N|2446522|2446611|2446172|2446447|N|N|N|N|N| +2446538|AAAAAAAAKMEFFCAA|1986-04-17|1035|4503|346|1986|4|4|17|2|1986|346|4503|Thursday|1986Q2|N|N|N|2446522|2446611|2446173|2446448|N|N|N|N|N| +2446539|AAAAAAAALMEFFCAA|1986-04-18|1035|4503|346|1986|5|4|18|2|1986|346|4503|Friday|1986Q2|N|Y|N|2446522|2446611|2446174|2446449|N|N|N|N|N| +2446540|AAAAAAAAMMEFFCAA|1986-04-19|1035|4503|346|1986|6|4|19|2|1986|346|4503|Saturday|1986Q2|N|Y|N|2446522|2446611|2446175|2446450|N|N|N|N|N| +2446541|AAAAAAAANMEFFCAA|1986-04-20|1035|4503|346|1986|0|4|20|2|1986|346|4503|Sunday|1986Q2|N|N|N|2446522|2446611|2446176|2446451|N|N|N|N|N| +2446542|AAAAAAAAOMEFFCAA|1986-04-21|1035|4503|346|1986|1|4|21|2|1986|346|4503|Monday|1986Q2|N|N|N|2446522|2446611|2446177|2446452|N|N|N|N|N| +2446543|AAAAAAAAPMEFFCAA|1986-04-22|1035|4504|346|1986|2|4|22|2|1986|346|4504|Tuesday|1986Q2|N|N|N|2446522|2446611|2446178|2446453|N|N|N|N|N| +2446544|AAAAAAAAANEFFCAA|1986-04-23|1035|4504|346|1986|3|4|23|2|1986|346|4504|Wednesday|1986Q2|N|N|N|2446522|2446611|2446179|2446454|N|N|N|N|N| +2446545|AAAAAAAABNEFFCAA|1986-04-24|1035|4504|346|1986|4|4|24|2|1986|346|4504|Thursday|1986Q2|N|N|N|2446522|2446611|2446180|2446455|N|N|N|N|N| +2446546|AAAAAAAACNEFFCAA|1986-04-25|1035|4504|346|1986|5|4|25|2|1986|346|4504|Friday|1986Q2|N|Y|N|2446522|2446611|2446181|2446456|N|N|N|N|N| +2446547|AAAAAAAADNEFFCAA|1986-04-26|1035|4504|346|1986|6|4|26|2|1986|346|4504|Saturday|1986Q2|N|Y|N|2446522|2446611|2446182|2446457|N|N|N|N|N| +2446548|AAAAAAAAENEFFCAA|1986-04-27|1035|4504|346|1986|0|4|27|2|1986|346|4504|Sunday|1986Q2|N|N|N|2446522|2446611|2446183|2446458|N|N|N|N|N| +2446549|AAAAAAAAFNEFFCAA|1986-04-28|1035|4504|346|1986|1|4|28|2|1986|346|4504|Monday|1986Q2|N|N|N|2446522|2446611|2446184|2446459|N|N|N|N|N| +2446550|AAAAAAAAGNEFFCAA|1986-04-29|1035|4505|346|1986|2|4|29|2|1986|346|4505|Tuesday|1986Q2|N|N|N|2446522|2446611|2446185|2446460|N|N|N|N|N| +2446551|AAAAAAAAHNEFFCAA|1986-04-30|1035|4505|346|1986|3|4|30|2|1986|346|4505|Wednesday|1986Q2|N|N|N|2446522|2446611|2446186|2446461|N|N|N|N|N| +2446552|AAAAAAAAINEFFCAA|1986-05-01|1036|4505|346|1986|4|5|1|2|1986|346|4505|Thursday|1986Q2|N|N|N|2446552|2446671|2446187|2446462|N|N|N|N|N| +2446553|AAAAAAAAJNEFFCAA|1986-05-02|1036|4505|346|1986|5|5|2|2|1986|346|4505|Friday|1986Q2|N|Y|N|2446552|2446671|2446188|2446463|N|N|N|N|N| +2446554|AAAAAAAAKNEFFCAA|1986-05-03|1036|4505|346|1986|6|5|3|2|1986|346|4505|Saturday|1986Q2|N|Y|N|2446552|2446671|2446189|2446464|N|N|N|N|N| +2446555|AAAAAAAALNEFFCAA|1986-05-04|1036|4505|346|1986|0|5|4|2|1986|346|4505|Sunday|1986Q2|N|N|N|2446552|2446671|2446190|2446465|N|N|N|N|N| +2446556|AAAAAAAAMNEFFCAA|1986-05-05|1036|4505|346|1986|1|5|5|2|1986|346|4505|Monday|1986Q2|N|N|N|2446552|2446671|2446191|2446466|N|N|N|N|N| +2446557|AAAAAAAANNEFFCAA|1986-05-06|1036|4506|346|1986|2|5|6|2|1986|346|4506|Tuesday|1986Q2|N|N|N|2446552|2446671|2446192|2446467|N|N|N|N|N| +2446558|AAAAAAAAONEFFCAA|1986-05-07|1036|4506|346|1986|3|5|7|2|1986|346|4506|Wednesday|1986Q2|N|N|N|2446552|2446671|2446193|2446468|N|N|N|N|N| +2446559|AAAAAAAAPNEFFCAA|1986-05-08|1036|4506|346|1986|4|5|8|2|1986|346|4506|Thursday|1986Q2|N|N|N|2446552|2446671|2446194|2446469|N|N|N|N|N| +2446560|AAAAAAAAAOEFFCAA|1986-05-09|1036|4506|346|1986|5|5|9|2|1986|346|4506|Friday|1986Q2|N|Y|N|2446552|2446671|2446195|2446470|N|N|N|N|N| +2446561|AAAAAAAABOEFFCAA|1986-05-10|1036|4506|346|1986|6|5|10|2|1986|346|4506|Saturday|1986Q2|N|Y|N|2446552|2446671|2446196|2446471|N|N|N|N|N| +2446562|AAAAAAAACOEFFCAA|1986-05-11|1036|4506|346|1986|0|5|11|2|1986|346|4506|Sunday|1986Q2|N|N|N|2446552|2446671|2446197|2446472|N|N|N|N|N| +2446563|AAAAAAAADOEFFCAA|1986-05-12|1036|4506|346|1986|1|5|12|2|1986|346|4506|Monday|1986Q2|N|N|N|2446552|2446671|2446198|2446473|N|N|N|N|N| +2446564|AAAAAAAAEOEFFCAA|1986-05-13|1036|4507|346|1986|2|5|13|2|1986|346|4507|Tuesday|1986Q2|N|N|N|2446552|2446671|2446199|2446474|N|N|N|N|N| +2446565|AAAAAAAAFOEFFCAA|1986-05-14|1036|4507|346|1986|3|5|14|2|1986|346|4507|Wednesday|1986Q2|N|N|N|2446552|2446671|2446200|2446475|N|N|N|N|N| +2446566|AAAAAAAAGOEFFCAA|1986-05-15|1036|4507|346|1986|4|5|15|2|1986|346|4507|Thursday|1986Q2|N|N|N|2446552|2446671|2446201|2446476|N|N|N|N|N| +2446567|AAAAAAAAHOEFFCAA|1986-05-16|1036|4507|346|1986|5|5|16|2|1986|346|4507|Friday|1986Q2|N|Y|N|2446552|2446671|2446202|2446477|N|N|N|N|N| +2446568|AAAAAAAAIOEFFCAA|1986-05-17|1036|4507|346|1986|6|5|17|2|1986|346|4507|Saturday|1986Q2|N|Y|N|2446552|2446671|2446203|2446478|N|N|N|N|N| +2446569|AAAAAAAAJOEFFCAA|1986-05-18|1036|4507|346|1986|0|5|18|2|1986|346|4507|Sunday|1986Q2|N|N|N|2446552|2446671|2446204|2446479|N|N|N|N|N| +2446570|AAAAAAAAKOEFFCAA|1986-05-19|1036|4507|346|1986|1|5|19|2|1986|346|4507|Monday|1986Q2|N|N|N|2446552|2446671|2446205|2446480|N|N|N|N|N| +2446571|AAAAAAAALOEFFCAA|1986-05-20|1036|4508|346|1986|2|5|20|2|1986|346|4508|Tuesday|1986Q2|N|N|N|2446552|2446671|2446206|2446481|N|N|N|N|N| +2446572|AAAAAAAAMOEFFCAA|1986-05-21|1036|4508|346|1986|3|5|21|2|1986|346|4508|Wednesday|1986Q2|N|N|N|2446552|2446671|2446207|2446482|N|N|N|N|N| +2446573|AAAAAAAANOEFFCAA|1986-05-22|1036|4508|346|1986|4|5|22|2|1986|346|4508|Thursday|1986Q2|N|N|N|2446552|2446671|2446208|2446483|N|N|N|N|N| +2446574|AAAAAAAAOOEFFCAA|1986-05-23|1036|4508|346|1986|5|5|23|2|1986|346|4508|Friday|1986Q2|N|Y|N|2446552|2446671|2446209|2446484|N|N|N|N|N| +2446575|AAAAAAAAPOEFFCAA|1986-05-24|1036|4508|346|1986|6|5|24|2|1986|346|4508|Saturday|1986Q2|N|Y|N|2446552|2446671|2446210|2446485|N|N|N|N|N| +2446576|AAAAAAAAAPEFFCAA|1986-05-25|1036|4508|346|1986|0|5|25|2|1986|346|4508|Sunday|1986Q2|N|N|N|2446552|2446671|2446211|2446486|N|N|N|N|N| +2446577|AAAAAAAABPEFFCAA|1986-05-26|1036|4508|346|1986|1|5|26|2|1986|346|4508|Monday|1986Q2|N|N|N|2446552|2446671|2446212|2446487|N|N|N|N|N| +2446578|AAAAAAAACPEFFCAA|1986-05-27|1036|4509|346|1986|2|5|27|2|1986|346|4509|Tuesday|1986Q2|N|N|N|2446552|2446671|2446213|2446488|N|N|N|N|N| +2446579|AAAAAAAADPEFFCAA|1986-05-28|1036|4509|346|1986|3|5|28|2|1986|346|4509|Wednesday|1986Q2|N|N|N|2446552|2446671|2446214|2446489|N|N|N|N|N| +2446580|AAAAAAAAEPEFFCAA|1986-05-29|1036|4509|346|1986|4|5|29|2|1986|346|4509|Thursday|1986Q2|N|N|N|2446552|2446671|2446215|2446490|N|N|N|N|N| +2446581|AAAAAAAAFPEFFCAA|1986-05-30|1036|4509|346|1986|5|5|30|2|1986|346|4509|Friday|1986Q2|N|Y|N|2446552|2446671|2446216|2446491|N|N|N|N|N| +2446582|AAAAAAAAGPEFFCAA|1986-05-31|1036|4509|346|1986|6|5|31|2|1986|346|4509|Saturday|1986Q2|N|Y|N|2446552|2446671|2446217|2446492|N|N|N|N|N| +2446583|AAAAAAAAHPEFFCAA|1986-06-01|1037|4509|347|1986|0|6|1|2|1986|347|4509|Sunday|1986Q2|N|N|N|2446583|2446733|2446218|2446493|N|N|N|N|N| +2446584|AAAAAAAAIPEFFCAA|1986-06-02|1037|4509|347|1986|1|6|2|2|1986|347|4509|Monday|1986Q2|N|N|N|2446583|2446733|2446219|2446494|N|N|N|N|N| +2446585|AAAAAAAAJPEFFCAA|1986-06-03|1037|4510|347|1986|2|6|3|2|1986|347|4510|Tuesday|1986Q2|N|N|N|2446583|2446733|2446220|2446495|N|N|N|N|N| +2446586|AAAAAAAAKPEFFCAA|1986-06-04|1037|4510|347|1986|3|6|4|2|1986|347|4510|Wednesday|1986Q2|N|N|N|2446583|2446733|2446221|2446496|N|N|N|N|N| +2446587|AAAAAAAALPEFFCAA|1986-06-05|1037|4510|347|1986|4|6|5|2|1986|347|4510|Thursday|1986Q2|N|N|N|2446583|2446733|2446222|2446497|N|N|N|N|N| +2446588|AAAAAAAAMPEFFCAA|1986-06-06|1037|4510|347|1986|5|6|6|2|1986|347|4510|Friday|1986Q2|N|Y|N|2446583|2446733|2446223|2446498|N|N|N|N|N| +2446589|AAAAAAAANPEFFCAA|1986-06-07|1037|4510|347|1986|6|6|7|2|1986|347|4510|Saturday|1986Q2|N|Y|N|2446583|2446733|2446224|2446499|N|N|N|N|N| +2446590|AAAAAAAAOPEFFCAA|1986-06-08|1037|4510|347|1986|0|6|8|2|1986|347|4510|Sunday|1986Q2|N|N|N|2446583|2446733|2446225|2446500|N|N|N|N|N| +2446591|AAAAAAAAPPEFFCAA|1986-06-09|1037|4510|347|1986|1|6|9|2|1986|347|4510|Monday|1986Q2|N|N|N|2446583|2446733|2446226|2446501|N|N|N|N|N| +2446592|AAAAAAAAAAFFFCAA|1986-06-10|1037|4511|347|1986|2|6|10|2|1986|347|4511|Tuesday|1986Q2|N|N|N|2446583|2446733|2446227|2446502|N|N|N|N|N| +2446593|AAAAAAAABAFFFCAA|1986-06-11|1037|4511|347|1986|3|6|11|2|1986|347|4511|Wednesday|1986Q2|N|N|N|2446583|2446733|2446228|2446503|N|N|N|N|N| +2446594|AAAAAAAACAFFFCAA|1986-06-12|1037|4511|347|1986|4|6|12|2|1986|347|4511|Thursday|1986Q2|N|N|N|2446583|2446733|2446229|2446504|N|N|N|N|N| +2446595|AAAAAAAADAFFFCAA|1986-06-13|1037|4511|347|1986|5|6|13|2|1986|347|4511|Friday|1986Q2|N|Y|N|2446583|2446733|2446230|2446505|N|N|N|N|N| +2446596|AAAAAAAAEAFFFCAA|1986-06-14|1037|4511|347|1986|6|6|14|2|1986|347|4511|Saturday|1986Q2|N|Y|N|2446583|2446733|2446231|2446506|N|N|N|N|N| +2446597|AAAAAAAAFAFFFCAA|1986-06-15|1037|4511|347|1986|0|6|15|2|1986|347|4511|Sunday|1986Q2|N|N|N|2446583|2446733|2446232|2446507|N|N|N|N|N| +2446598|AAAAAAAAGAFFFCAA|1986-06-16|1037|4511|347|1986|1|6|16|2|1986|347|4511|Monday|1986Q2|N|N|N|2446583|2446733|2446233|2446508|N|N|N|N|N| +2446599|AAAAAAAAHAFFFCAA|1986-06-17|1037|4512|347|1986|2|6|17|2|1986|347|4512|Tuesday|1986Q2|N|N|N|2446583|2446733|2446234|2446509|N|N|N|N|N| +2446600|AAAAAAAAIAFFFCAA|1986-06-18|1037|4512|347|1986|3|6|18|2|1986|347|4512|Wednesday|1986Q2|N|N|N|2446583|2446733|2446235|2446510|N|N|N|N|N| +2446601|AAAAAAAAJAFFFCAA|1986-06-19|1037|4512|347|1986|4|6|19|2|1986|347|4512|Thursday|1986Q2|N|N|N|2446583|2446733|2446236|2446511|N|N|N|N|N| +2446602|AAAAAAAAKAFFFCAA|1986-06-20|1037|4512|347|1986|5|6|20|2|1986|347|4512|Friday|1986Q2|N|Y|N|2446583|2446733|2446237|2446512|N|N|N|N|N| +2446603|AAAAAAAALAFFFCAA|1986-06-21|1037|4512|347|1986|6|6|21|2|1986|347|4512|Saturday|1986Q2|N|Y|N|2446583|2446733|2446238|2446513|N|N|N|N|N| +2446604|AAAAAAAAMAFFFCAA|1986-06-22|1037|4512|347|1986|0|6|22|2|1986|347|4512|Sunday|1986Q2|N|N|N|2446583|2446733|2446239|2446514|N|N|N|N|N| +2446605|AAAAAAAANAFFFCAA|1986-06-23|1037|4512|347|1986|1|6|23|2|1986|347|4512|Monday|1986Q2|N|N|N|2446583|2446733|2446240|2446515|N|N|N|N|N| +2446606|AAAAAAAAOAFFFCAA|1986-06-24|1037|4513|347|1986|2|6|24|2|1986|347|4513|Tuesday|1986Q2|N|N|N|2446583|2446733|2446241|2446516|N|N|N|N|N| +2446607|AAAAAAAAPAFFFCAA|1986-06-25|1037|4513|347|1986|3|6|25|2|1986|347|4513|Wednesday|1986Q2|N|N|N|2446583|2446733|2446242|2446517|N|N|N|N|N| +2446608|AAAAAAAAABFFFCAA|1986-06-26|1037|4513|347|1986|4|6|26|2|1986|347|4513|Thursday|1986Q2|N|N|N|2446583|2446733|2446243|2446518|N|N|N|N|N| +2446609|AAAAAAAABBFFFCAA|1986-06-27|1037|4513|347|1986|5|6|27|2|1986|347|4513|Friday|1986Q2|N|Y|N|2446583|2446733|2446244|2446519|N|N|N|N|N| +2446610|AAAAAAAACBFFFCAA|1986-06-28|1037|4513|347|1986|6|6|28|2|1986|347|4513|Saturday|1986Q2|N|Y|N|2446583|2446733|2446245|2446520|N|N|N|N|N| +2446611|AAAAAAAADBFFFCAA|1986-06-29|1037|4513|347|1986|0|6|29|2|1986|347|4513|Sunday|1986Q2|N|N|N|2446583|2446733|2446246|2446521|N|N|N|N|N| +2446612|AAAAAAAAEBFFFCAA|1986-06-30|1037|4513|347|1986|1|6|30|2|1986|347|4513|Monday|1986Q2|N|N|N|2446583|2446733|2446247|2446522|N|N|N|N|N| +2446613|AAAAAAAAFBFFFCAA|1986-07-01|1038|4514|347|1986|2|7|1|2|1986|347|4514|Tuesday|1986Q2|N|N|N|2446613|2446793|2446248|2446522|N|N|N|N|N| +2446614|AAAAAAAAGBFFFCAA|1986-07-02|1038|4514|347|1986|3|7|2|3|1986|347|4514|Wednesday|1986Q3|N|N|N|2446613|2446793|2446249|2446523|N|N|N|N|N| +2446615|AAAAAAAAHBFFFCAA|1986-07-03|1038|4514|347|1986|4|7|3|3|1986|347|4514|Thursday|1986Q3|N|N|N|2446613|2446793|2446250|2446524|N|N|N|N|N| +2446616|AAAAAAAAIBFFFCAA|1986-07-04|1038|4514|347|1986|5|7|4|3|1986|347|4514|Friday|1986Q3|N|Y|N|2446613|2446793|2446251|2446525|N|N|N|N|N| +2446617|AAAAAAAAJBFFFCAA|1986-07-05|1038|4514|347|1986|6|7|5|3|1986|347|4514|Saturday|1986Q3|Y|Y|N|2446613|2446793|2446252|2446526|N|N|N|N|N| +2446618|AAAAAAAAKBFFFCAA|1986-07-06|1038|4514|347|1986|0|7|6|3|1986|347|4514|Sunday|1986Q3|N|N|Y|2446613|2446793|2446253|2446527|N|N|N|N|N| +2446619|AAAAAAAALBFFFCAA|1986-07-07|1038|4514|347|1986|1|7|7|3|1986|347|4514|Monday|1986Q3|N|N|N|2446613|2446793|2446254|2446528|N|N|N|N|N| +2446620|AAAAAAAAMBFFFCAA|1986-07-08|1038|4515|347|1986|2|7|8|3|1986|347|4515|Tuesday|1986Q3|N|N|N|2446613|2446793|2446255|2446529|N|N|N|N|N| +2446621|AAAAAAAANBFFFCAA|1986-07-09|1038|4515|347|1986|3|7|9|3|1986|347|4515|Wednesday|1986Q3|N|N|N|2446613|2446793|2446256|2446530|N|N|N|N|N| +2446622|AAAAAAAAOBFFFCAA|1986-07-10|1038|4515|347|1986|4|7|10|3|1986|347|4515|Thursday|1986Q3|N|N|N|2446613|2446793|2446257|2446531|N|N|N|N|N| +2446623|AAAAAAAAPBFFFCAA|1986-07-11|1038|4515|347|1986|5|7|11|3|1986|347|4515|Friday|1986Q3|N|Y|N|2446613|2446793|2446258|2446532|N|N|N|N|N| +2446624|AAAAAAAAACFFFCAA|1986-07-12|1038|4515|347|1986|6|7|12|3|1986|347|4515|Saturday|1986Q3|N|Y|N|2446613|2446793|2446259|2446533|N|N|N|N|N| +2446625|AAAAAAAABCFFFCAA|1986-07-13|1038|4515|347|1986|0|7|13|3|1986|347|4515|Sunday|1986Q3|N|N|N|2446613|2446793|2446260|2446534|N|N|N|N|N| +2446626|AAAAAAAACCFFFCAA|1986-07-14|1038|4515|347|1986|1|7|14|3|1986|347|4515|Monday|1986Q3|N|N|N|2446613|2446793|2446261|2446535|N|N|N|N|N| +2446627|AAAAAAAADCFFFCAA|1986-07-15|1038|4516|347|1986|2|7|15|3|1986|347|4516|Tuesday|1986Q3|N|N|N|2446613|2446793|2446262|2446536|N|N|N|N|N| +2446628|AAAAAAAAECFFFCAA|1986-07-16|1038|4516|347|1986|3|7|16|3|1986|347|4516|Wednesday|1986Q3|N|N|N|2446613|2446793|2446263|2446537|N|N|N|N|N| +2446629|AAAAAAAAFCFFFCAA|1986-07-17|1038|4516|347|1986|4|7|17|3|1986|347|4516|Thursday|1986Q3|N|N|N|2446613|2446793|2446264|2446538|N|N|N|N|N| +2446630|AAAAAAAAGCFFFCAA|1986-07-18|1038|4516|347|1986|5|7|18|3|1986|347|4516|Friday|1986Q3|N|Y|N|2446613|2446793|2446265|2446539|N|N|N|N|N| +2446631|AAAAAAAAHCFFFCAA|1986-07-19|1038|4516|347|1986|6|7|19|3|1986|347|4516|Saturday|1986Q3|N|Y|N|2446613|2446793|2446266|2446540|N|N|N|N|N| +2446632|AAAAAAAAICFFFCAA|1986-07-20|1038|4516|347|1986|0|7|20|3|1986|347|4516|Sunday|1986Q3|N|N|N|2446613|2446793|2446267|2446541|N|N|N|N|N| +2446633|AAAAAAAAJCFFFCAA|1986-07-21|1038|4516|347|1986|1|7|21|3|1986|347|4516|Monday|1986Q3|N|N|N|2446613|2446793|2446268|2446542|N|N|N|N|N| +2446634|AAAAAAAAKCFFFCAA|1986-07-22|1038|4517|347|1986|2|7|22|3|1986|347|4517|Tuesday|1986Q3|N|N|N|2446613|2446793|2446269|2446543|N|N|N|N|N| +2446635|AAAAAAAALCFFFCAA|1986-07-23|1038|4517|347|1986|3|7|23|3|1986|347|4517|Wednesday|1986Q3|N|N|N|2446613|2446793|2446270|2446544|N|N|N|N|N| +2446636|AAAAAAAAMCFFFCAA|1986-07-24|1038|4517|347|1986|4|7|24|3|1986|347|4517|Thursday|1986Q3|N|N|N|2446613|2446793|2446271|2446545|N|N|N|N|N| +2446637|AAAAAAAANCFFFCAA|1986-07-25|1038|4517|347|1986|5|7|25|3|1986|347|4517|Friday|1986Q3|N|Y|N|2446613|2446793|2446272|2446546|N|N|N|N|N| +2446638|AAAAAAAAOCFFFCAA|1986-07-26|1038|4517|347|1986|6|7|26|3|1986|347|4517|Saturday|1986Q3|N|Y|N|2446613|2446793|2446273|2446547|N|N|N|N|N| +2446639|AAAAAAAAPCFFFCAA|1986-07-27|1038|4517|347|1986|0|7|27|3|1986|347|4517|Sunday|1986Q3|N|N|N|2446613|2446793|2446274|2446548|N|N|N|N|N| +2446640|AAAAAAAAADFFFCAA|1986-07-28|1038|4517|347|1986|1|7|28|3|1986|347|4517|Monday|1986Q3|N|N|N|2446613|2446793|2446275|2446549|N|N|N|N|N| +2446641|AAAAAAAABDFFFCAA|1986-07-29|1038|4518|347|1986|2|7|29|3|1986|347|4518|Tuesday|1986Q3|N|N|N|2446613|2446793|2446276|2446550|N|N|N|N|N| +2446642|AAAAAAAACDFFFCAA|1986-07-30|1038|4518|347|1986|3|7|30|3|1986|347|4518|Wednesday|1986Q3|N|N|N|2446613|2446793|2446277|2446551|N|N|N|N|N| +2446643|AAAAAAAADDFFFCAA|1986-07-31|1038|4518|347|1986|4|7|31|3|1986|347|4518|Thursday|1986Q3|N|N|N|2446613|2446793|2446278|2446552|N|N|N|N|N| +2446644|AAAAAAAAEDFFFCAA|1986-08-01|1039|4518|347|1986|5|8|1|3|1986|347|4518|Friday|1986Q3|N|Y|N|2446644|2446855|2446279|2446553|N|N|N|N|N| +2446645|AAAAAAAAFDFFFCAA|1986-08-02|1039|4518|347|1986|6|8|2|3|1986|347|4518|Saturday|1986Q3|N|Y|N|2446644|2446855|2446280|2446554|N|N|N|N|N| +2446646|AAAAAAAAGDFFFCAA|1986-08-03|1039|4518|347|1986|0|8|3|3|1986|347|4518|Sunday|1986Q3|N|N|N|2446644|2446855|2446281|2446555|N|N|N|N|N| +2446647|AAAAAAAAHDFFFCAA|1986-08-04|1039|4518|347|1986|1|8|4|3|1986|347|4518|Monday|1986Q3|N|N|N|2446644|2446855|2446282|2446556|N|N|N|N|N| +2446648|AAAAAAAAIDFFFCAA|1986-08-05|1039|4519|347|1986|2|8|5|3|1986|347|4519|Tuesday|1986Q3|N|N|N|2446644|2446855|2446283|2446557|N|N|N|N|N| +2446649|AAAAAAAAJDFFFCAA|1986-08-06|1039|4519|347|1986|3|8|6|3|1986|347|4519|Wednesday|1986Q3|N|N|N|2446644|2446855|2446284|2446558|N|N|N|N|N| +2446650|AAAAAAAAKDFFFCAA|1986-08-07|1039|4519|347|1986|4|8|7|3|1986|347|4519|Thursday|1986Q3|N|N|N|2446644|2446855|2446285|2446559|N|N|N|N|N| +2446651|AAAAAAAALDFFFCAA|1986-08-08|1039|4519|347|1986|5|8|8|3|1986|347|4519|Friday|1986Q3|N|Y|N|2446644|2446855|2446286|2446560|N|N|N|N|N| +2446652|AAAAAAAAMDFFFCAA|1986-08-09|1039|4519|347|1986|6|8|9|3|1986|347|4519|Saturday|1986Q3|N|Y|N|2446644|2446855|2446287|2446561|N|N|N|N|N| +2446653|AAAAAAAANDFFFCAA|1986-08-10|1039|4519|347|1986|0|8|10|3|1986|347|4519|Sunday|1986Q3|N|N|N|2446644|2446855|2446288|2446562|N|N|N|N|N| +2446654|AAAAAAAAODFFFCAA|1986-08-11|1039|4519|347|1986|1|8|11|3|1986|347|4519|Monday|1986Q3|N|N|N|2446644|2446855|2446289|2446563|N|N|N|N|N| +2446655|AAAAAAAAPDFFFCAA|1986-08-12|1039|4520|347|1986|2|8|12|3|1986|347|4520|Tuesday|1986Q3|N|N|N|2446644|2446855|2446290|2446564|N|N|N|N|N| +2446656|AAAAAAAAAEFFFCAA|1986-08-13|1039|4520|347|1986|3|8|13|3|1986|347|4520|Wednesday|1986Q3|N|N|N|2446644|2446855|2446291|2446565|N|N|N|N|N| +2446657|AAAAAAAABEFFFCAA|1986-08-14|1039|4520|347|1986|4|8|14|3|1986|347|4520|Thursday|1986Q3|N|N|N|2446644|2446855|2446292|2446566|N|N|N|N|N| +2446658|AAAAAAAACEFFFCAA|1986-08-15|1039|4520|347|1986|5|8|15|3|1986|347|4520|Friday|1986Q3|N|Y|N|2446644|2446855|2446293|2446567|N|N|N|N|N| +2446659|AAAAAAAADEFFFCAA|1986-08-16|1039|4520|347|1986|6|8|16|3|1986|347|4520|Saturday|1986Q3|N|Y|N|2446644|2446855|2446294|2446568|N|N|N|N|N| +2446660|AAAAAAAAEEFFFCAA|1986-08-17|1039|4520|347|1986|0|8|17|3|1986|347|4520|Sunday|1986Q3|N|N|N|2446644|2446855|2446295|2446569|N|N|N|N|N| +2446661|AAAAAAAAFEFFFCAA|1986-08-18|1039|4520|347|1986|1|8|18|3|1986|347|4520|Monday|1986Q3|N|N|N|2446644|2446855|2446296|2446570|N|N|N|N|N| +2446662|AAAAAAAAGEFFFCAA|1986-08-19|1039|4521|347|1986|2|8|19|3|1986|347|4521|Tuesday|1986Q3|N|N|N|2446644|2446855|2446297|2446571|N|N|N|N|N| +2446663|AAAAAAAAHEFFFCAA|1986-08-20|1039|4521|347|1986|3|8|20|3|1986|347|4521|Wednesday|1986Q3|N|N|N|2446644|2446855|2446298|2446572|N|N|N|N|N| +2446664|AAAAAAAAIEFFFCAA|1986-08-21|1039|4521|347|1986|4|8|21|3|1986|347|4521|Thursday|1986Q3|N|N|N|2446644|2446855|2446299|2446573|N|N|N|N|N| +2446665|AAAAAAAAJEFFFCAA|1986-08-22|1039|4521|347|1986|5|8|22|3|1986|347|4521|Friday|1986Q3|N|Y|N|2446644|2446855|2446300|2446574|N|N|N|N|N| +2446666|AAAAAAAAKEFFFCAA|1986-08-23|1039|4521|347|1986|6|8|23|3|1986|347|4521|Saturday|1986Q3|N|Y|N|2446644|2446855|2446301|2446575|N|N|N|N|N| +2446667|AAAAAAAALEFFFCAA|1986-08-24|1039|4521|347|1986|0|8|24|3|1986|347|4521|Sunday|1986Q3|N|N|N|2446644|2446855|2446302|2446576|N|N|N|N|N| +2446668|AAAAAAAAMEFFFCAA|1986-08-25|1039|4521|347|1986|1|8|25|3|1986|347|4521|Monday|1986Q3|N|N|N|2446644|2446855|2446303|2446577|N|N|N|N|N| +2446669|AAAAAAAANEFFFCAA|1986-08-26|1039|4522|347|1986|2|8|26|3|1986|347|4522|Tuesday|1986Q3|N|N|N|2446644|2446855|2446304|2446578|N|N|N|N|N| +2446670|AAAAAAAAOEFFFCAA|1986-08-27|1039|4522|347|1986|3|8|27|3|1986|347|4522|Wednesday|1986Q3|N|N|N|2446644|2446855|2446305|2446579|N|N|N|N|N| +2446671|AAAAAAAAPEFFFCAA|1986-08-28|1039|4522|347|1986|4|8|28|3|1986|347|4522|Thursday|1986Q3|N|N|N|2446644|2446855|2446306|2446580|N|N|N|N|N| +2446672|AAAAAAAAAFFFFCAA|1986-08-29|1039|4522|347|1986|5|8|29|3|1986|347|4522|Friday|1986Q3|N|Y|N|2446644|2446855|2446307|2446581|N|N|N|N|N| +2446673|AAAAAAAABFFFFCAA|1986-08-30|1039|4522|347|1986|6|8|30|3|1986|347|4522|Saturday|1986Q3|N|Y|N|2446644|2446855|2446308|2446582|N|N|N|N|N| +2446674|AAAAAAAACFFFFCAA|1986-08-31|1039|4522|347|1986|0|8|31|3|1986|347|4522|Sunday|1986Q3|N|N|N|2446644|2446855|2446309|2446583|N|N|N|N|N| +2446675|AAAAAAAADFFFFCAA|1986-09-01|1040|4522|348|1986|1|9|1|3|1986|348|4522|Monday|1986Q3|N|N|N|2446675|2446917|2446310|2446584|N|N|N|N|N| +2446676|AAAAAAAAEFFFFCAA|1986-09-02|1040|4523|348|1986|2|9|2|3|1986|348|4523|Tuesday|1986Q3|N|N|N|2446675|2446917|2446311|2446585|N|N|N|N|N| +2446677|AAAAAAAAFFFFFCAA|1986-09-03|1040|4523|348|1986|3|9|3|3|1986|348|4523|Wednesday|1986Q3|N|N|N|2446675|2446917|2446312|2446586|N|N|N|N|N| +2446678|AAAAAAAAGFFFFCAA|1986-09-04|1040|4523|348|1986|4|9|4|3|1986|348|4523|Thursday|1986Q3|N|N|N|2446675|2446917|2446313|2446587|N|N|N|N|N| +2446679|AAAAAAAAHFFFFCAA|1986-09-05|1040|4523|348|1986|5|9|5|3|1986|348|4523|Friday|1986Q3|N|Y|N|2446675|2446917|2446314|2446588|N|N|N|N|N| +2446680|AAAAAAAAIFFFFCAA|1986-09-06|1040|4523|348|1986|6|9|6|3|1986|348|4523|Saturday|1986Q3|N|Y|N|2446675|2446917|2446315|2446589|N|N|N|N|N| +2446681|AAAAAAAAJFFFFCAA|1986-09-07|1040|4523|348|1986|0|9|7|3|1986|348|4523|Sunday|1986Q3|N|N|N|2446675|2446917|2446316|2446590|N|N|N|N|N| +2446682|AAAAAAAAKFFFFCAA|1986-09-08|1040|4523|348|1986|1|9|8|3|1986|348|4523|Monday|1986Q3|N|N|N|2446675|2446917|2446317|2446591|N|N|N|N|N| +2446683|AAAAAAAALFFFFCAA|1986-09-09|1040|4524|348|1986|2|9|9|3|1986|348|4524|Tuesday|1986Q3|N|N|N|2446675|2446917|2446318|2446592|N|N|N|N|N| +2446684|AAAAAAAAMFFFFCAA|1986-09-10|1040|4524|348|1986|3|9|10|3|1986|348|4524|Wednesday|1986Q3|N|N|N|2446675|2446917|2446319|2446593|N|N|N|N|N| +2446685|AAAAAAAANFFFFCAA|1986-09-11|1040|4524|348|1986|4|9|11|3|1986|348|4524|Thursday|1986Q3|N|N|N|2446675|2446917|2446320|2446594|N|N|N|N|N| +2446686|AAAAAAAAOFFFFCAA|1986-09-12|1040|4524|348|1986|5|9|12|3|1986|348|4524|Friday|1986Q3|N|Y|N|2446675|2446917|2446321|2446595|N|N|N|N|N| +2446687|AAAAAAAAPFFFFCAA|1986-09-13|1040|4524|348|1986|6|9|13|3|1986|348|4524|Saturday|1986Q3|N|Y|N|2446675|2446917|2446322|2446596|N|N|N|N|N| +2446688|AAAAAAAAAGFFFCAA|1986-09-14|1040|4524|348|1986|0|9|14|3|1986|348|4524|Sunday|1986Q3|N|N|N|2446675|2446917|2446323|2446597|N|N|N|N|N| +2446689|AAAAAAAABGFFFCAA|1986-09-15|1040|4524|348|1986|1|9|15|3|1986|348|4524|Monday|1986Q3|N|N|N|2446675|2446917|2446324|2446598|N|N|N|N|N| +2446690|AAAAAAAACGFFFCAA|1986-09-16|1040|4525|348|1986|2|9|16|3|1986|348|4525|Tuesday|1986Q3|N|N|N|2446675|2446917|2446325|2446599|N|N|N|N|N| +2446691|AAAAAAAADGFFFCAA|1986-09-17|1040|4525|348|1986|3|9|17|3|1986|348|4525|Wednesday|1986Q3|N|N|N|2446675|2446917|2446326|2446600|N|N|N|N|N| +2446692|AAAAAAAAEGFFFCAA|1986-09-18|1040|4525|348|1986|4|9|18|3|1986|348|4525|Thursday|1986Q3|N|N|N|2446675|2446917|2446327|2446601|N|N|N|N|N| +2446693|AAAAAAAAFGFFFCAA|1986-09-19|1040|4525|348|1986|5|9|19|3|1986|348|4525|Friday|1986Q3|N|Y|N|2446675|2446917|2446328|2446602|N|N|N|N|N| +2446694|AAAAAAAAGGFFFCAA|1986-09-20|1040|4525|348|1986|6|9|20|3|1986|348|4525|Saturday|1986Q3|N|Y|N|2446675|2446917|2446329|2446603|N|N|N|N|N| +2446695|AAAAAAAAHGFFFCAA|1986-09-21|1040|4525|348|1986|0|9|21|3|1986|348|4525|Sunday|1986Q3|N|N|N|2446675|2446917|2446330|2446604|N|N|N|N|N| +2446696|AAAAAAAAIGFFFCAA|1986-09-22|1040|4525|348|1986|1|9|22|3|1986|348|4525|Monday|1986Q3|N|N|N|2446675|2446917|2446331|2446605|N|N|N|N|N| +2446697|AAAAAAAAJGFFFCAA|1986-09-23|1040|4526|348|1986|2|9|23|3|1986|348|4526|Tuesday|1986Q3|N|N|N|2446675|2446917|2446332|2446606|N|N|N|N|N| +2446698|AAAAAAAAKGFFFCAA|1986-09-24|1040|4526|348|1986|3|9|24|3|1986|348|4526|Wednesday|1986Q3|N|N|N|2446675|2446917|2446333|2446607|N|N|N|N|N| +2446699|AAAAAAAALGFFFCAA|1986-09-25|1040|4526|348|1986|4|9|25|3|1986|348|4526|Thursday|1986Q3|N|N|N|2446675|2446917|2446334|2446608|N|N|N|N|N| +2446700|AAAAAAAAMGFFFCAA|1986-09-26|1040|4526|348|1986|5|9|26|3|1986|348|4526|Friday|1986Q3|N|Y|N|2446675|2446917|2446335|2446609|N|N|N|N|N| +2446701|AAAAAAAANGFFFCAA|1986-09-27|1040|4526|348|1986|6|9|27|3|1986|348|4526|Saturday|1986Q3|N|Y|N|2446675|2446917|2446336|2446610|N|N|N|N|N| +2446702|AAAAAAAAOGFFFCAA|1986-09-28|1040|4526|348|1986|0|9|28|3|1986|348|4526|Sunday|1986Q3|N|N|N|2446675|2446917|2446337|2446611|N|N|N|N|N| +2446703|AAAAAAAAPGFFFCAA|1986-09-29|1040|4526|348|1986|1|9|29|3|1986|348|4526|Monday|1986Q3|N|N|N|2446675|2446917|2446338|2446612|N|N|N|N|N| +2446704|AAAAAAAAAHFFFCAA|1986-09-30|1040|4527|348|1986|2|9|30|3|1986|348|4527|Tuesday|1986Q3|N|N|N|2446675|2446917|2446339|2446613|N|N|N|N|N| +2446705|AAAAAAAABHFFFCAA|1986-10-01|1041|4527|348|1986|3|10|1|3|1986|348|4527|Wednesday|1986Q3|N|N|N|2446705|2446977|2446340|2446613|N|N|N|N|N| +2446706|AAAAAAAACHFFFCAA|1986-10-02|1041|4527|348|1986|4|10|2|4|1986|348|4527|Thursday|1986Q4|N|N|N|2446705|2446977|2446341|2446614|N|N|N|N|N| +2446707|AAAAAAAADHFFFCAA|1986-10-03|1041|4527|348|1986|5|10|3|4|1986|348|4527|Friday|1986Q4|N|Y|N|2446705|2446977|2446342|2446615|N|N|N|N|N| +2446708|AAAAAAAAEHFFFCAA|1986-10-04|1041|4527|348|1986|6|10|4|4|1986|348|4527|Saturday|1986Q4|N|Y|N|2446705|2446977|2446343|2446616|N|N|N|N|N| +2446709|AAAAAAAAFHFFFCAA|1986-10-05|1041|4527|348|1986|0|10|5|4|1986|348|4527|Sunday|1986Q4|N|N|N|2446705|2446977|2446344|2446617|N|N|N|N|N| +2446710|AAAAAAAAGHFFFCAA|1986-10-06|1041|4527|348|1986|1|10|6|4|1986|348|4527|Monday|1986Q4|N|N|N|2446705|2446977|2446345|2446618|N|N|N|N|N| +2446711|AAAAAAAAHHFFFCAA|1986-10-07|1041|4528|348|1986|2|10|7|4|1986|348|4528|Tuesday|1986Q4|N|N|N|2446705|2446977|2446346|2446619|N|N|N|N|N| +2446712|AAAAAAAAIHFFFCAA|1986-10-08|1041|4528|348|1986|3|10|8|4|1986|348|4528|Wednesday|1986Q4|N|N|N|2446705|2446977|2446347|2446620|N|N|N|N|N| +2446713|AAAAAAAAJHFFFCAA|1986-10-09|1041|4528|348|1986|4|10|9|4|1986|348|4528|Thursday|1986Q4|N|N|N|2446705|2446977|2446348|2446621|N|N|N|N|N| +2446714|AAAAAAAAKHFFFCAA|1986-10-10|1041|4528|348|1986|5|10|10|4|1986|348|4528|Friday|1986Q4|N|Y|N|2446705|2446977|2446349|2446622|N|N|N|N|N| +2446715|AAAAAAAALHFFFCAA|1986-10-11|1041|4528|348|1986|6|10|11|4|1986|348|4528|Saturday|1986Q4|N|Y|N|2446705|2446977|2446350|2446623|N|N|N|N|N| +2446716|AAAAAAAAMHFFFCAA|1986-10-12|1041|4528|348|1986|0|10|12|4|1986|348|4528|Sunday|1986Q4|N|N|N|2446705|2446977|2446351|2446624|N|N|N|N|N| +2446717|AAAAAAAANHFFFCAA|1986-10-13|1041|4528|348|1986|1|10|13|4|1986|348|4528|Monday|1986Q4|N|N|N|2446705|2446977|2446352|2446625|N|N|N|N|N| +2446718|AAAAAAAAOHFFFCAA|1986-10-14|1041|4529|348|1986|2|10|14|4|1986|348|4529|Tuesday|1986Q4|N|N|N|2446705|2446977|2446353|2446626|N|N|N|N|N| +2446719|AAAAAAAAPHFFFCAA|1986-10-15|1041|4529|348|1986|3|10|15|4|1986|348|4529|Wednesday|1986Q4|N|N|N|2446705|2446977|2446354|2446627|N|N|N|N|N| +2446720|AAAAAAAAAIFFFCAA|1986-10-16|1041|4529|348|1986|4|10|16|4|1986|348|4529|Thursday|1986Q4|N|N|N|2446705|2446977|2446355|2446628|N|N|N|N|N| +2446721|AAAAAAAABIFFFCAA|1986-10-17|1041|4529|348|1986|5|10|17|4|1986|348|4529|Friday|1986Q4|N|Y|N|2446705|2446977|2446356|2446629|N|N|N|N|N| +2446722|AAAAAAAACIFFFCAA|1986-10-18|1041|4529|348|1986|6|10|18|4|1986|348|4529|Saturday|1986Q4|N|Y|N|2446705|2446977|2446357|2446630|N|N|N|N|N| +2446723|AAAAAAAADIFFFCAA|1986-10-19|1041|4529|348|1986|0|10|19|4|1986|348|4529|Sunday|1986Q4|N|N|N|2446705|2446977|2446358|2446631|N|N|N|N|N| +2446724|AAAAAAAAEIFFFCAA|1986-10-20|1041|4529|348|1986|1|10|20|4|1986|348|4529|Monday|1986Q4|N|N|N|2446705|2446977|2446359|2446632|N|N|N|N|N| +2446725|AAAAAAAAFIFFFCAA|1986-10-21|1041|4530|348|1986|2|10|21|4|1986|348|4530|Tuesday|1986Q4|N|N|N|2446705|2446977|2446360|2446633|N|N|N|N|N| +2446726|AAAAAAAAGIFFFCAA|1986-10-22|1041|4530|348|1986|3|10|22|4|1986|348|4530|Wednesday|1986Q4|N|N|N|2446705|2446977|2446361|2446634|N|N|N|N|N| +2446727|AAAAAAAAHIFFFCAA|1986-10-23|1041|4530|348|1986|4|10|23|4|1986|348|4530|Thursday|1986Q4|N|N|N|2446705|2446977|2446362|2446635|N|N|N|N|N| +2446728|AAAAAAAAIIFFFCAA|1986-10-24|1041|4530|348|1986|5|10|24|4|1986|348|4530|Friday|1986Q4|N|Y|N|2446705|2446977|2446363|2446636|N|N|N|N|N| +2446729|AAAAAAAAJIFFFCAA|1986-10-25|1041|4530|348|1986|6|10|25|4|1986|348|4530|Saturday|1986Q4|N|Y|N|2446705|2446977|2446364|2446637|N|N|N|N|N| +2446730|AAAAAAAAKIFFFCAA|1986-10-26|1041|4530|348|1986|0|10|26|4|1986|348|4530|Sunday|1986Q4|N|N|N|2446705|2446977|2446365|2446638|N|N|N|N|N| +2446731|AAAAAAAALIFFFCAA|1986-10-27|1041|4530|348|1986|1|10|27|4|1986|348|4530|Monday|1986Q4|N|N|N|2446705|2446977|2446366|2446639|N|N|N|N|N| +2446732|AAAAAAAAMIFFFCAA|1986-10-28|1041|4531|348|1986|2|10|28|4|1986|348|4531|Tuesday|1986Q4|N|N|N|2446705|2446977|2446367|2446640|N|N|N|N|N| +2446733|AAAAAAAANIFFFCAA|1986-10-29|1041|4531|348|1986|3|10|29|4|1986|348|4531|Wednesday|1986Q4|N|N|N|2446705|2446977|2446368|2446641|N|N|N|N|N| +2446734|AAAAAAAAOIFFFCAA|1986-10-30|1041|4531|348|1986|4|10|30|4|1986|348|4531|Thursday|1986Q4|N|N|N|2446705|2446977|2446369|2446642|N|N|N|N|N| +2446735|AAAAAAAAPIFFFCAA|1986-10-31|1041|4531|348|1986|5|10|31|4|1986|348|4531|Friday|1986Q4|N|Y|N|2446705|2446977|2446370|2446643|N|N|N|N|N| +2446736|AAAAAAAAAJFFFCAA|1986-11-01|1042|4531|348|1986|6|11|1|4|1986|348|4531|Saturday|1986Q4|N|Y|N|2446736|2447039|2446371|2446644|N|N|N|N|N| +2446737|AAAAAAAABJFFFCAA|1986-11-02|1042|4531|348|1986|0|11|2|4|1986|348|4531|Sunday|1986Q4|N|N|N|2446736|2447039|2446372|2446645|N|N|N|N|N| +2446738|AAAAAAAACJFFFCAA|1986-11-03|1042|4531|348|1986|1|11|3|4|1986|348|4531|Monday|1986Q4|N|N|N|2446736|2447039|2446373|2446646|N|N|N|N|N| +2446739|AAAAAAAADJFFFCAA|1986-11-04|1042|4532|348|1986|2|11|4|4|1986|348|4532|Tuesday|1986Q4|N|N|N|2446736|2447039|2446374|2446647|N|N|N|N|N| +2446740|AAAAAAAAEJFFFCAA|1986-11-05|1042|4532|348|1986|3|11|5|4|1986|348|4532|Wednesday|1986Q4|N|N|N|2446736|2447039|2446375|2446648|N|N|N|N|N| +2446741|AAAAAAAAFJFFFCAA|1986-11-06|1042|4532|348|1986|4|11|6|4|1986|348|4532|Thursday|1986Q4|N|N|N|2446736|2447039|2446376|2446649|N|N|N|N|N| +2446742|AAAAAAAAGJFFFCAA|1986-11-07|1042|4532|348|1986|5|11|7|4|1986|348|4532|Friday|1986Q4|N|Y|N|2446736|2447039|2446377|2446650|N|N|N|N|N| +2446743|AAAAAAAAHJFFFCAA|1986-11-08|1042|4532|348|1986|6|11|8|4|1986|348|4532|Saturday|1986Q4|N|Y|N|2446736|2447039|2446378|2446651|N|N|N|N|N| +2446744|AAAAAAAAIJFFFCAA|1986-11-09|1042|4532|348|1986|0|11|9|4|1986|348|4532|Sunday|1986Q4|N|N|N|2446736|2447039|2446379|2446652|N|N|N|N|N| +2446745|AAAAAAAAJJFFFCAA|1986-11-10|1042|4532|348|1986|1|11|10|4|1986|348|4532|Monday|1986Q4|N|N|N|2446736|2447039|2446380|2446653|N|N|N|N|N| +2446746|AAAAAAAAKJFFFCAA|1986-11-11|1042|4533|348|1986|2|11|11|4|1986|348|4533|Tuesday|1986Q4|N|N|N|2446736|2447039|2446381|2446654|N|N|N|N|N| +2446747|AAAAAAAALJFFFCAA|1986-11-12|1042|4533|348|1986|3|11|12|4|1986|348|4533|Wednesday|1986Q4|N|N|N|2446736|2447039|2446382|2446655|N|N|N|N|N| +2446748|AAAAAAAAMJFFFCAA|1986-11-13|1042|4533|348|1986|4|11|13|4|1986|348|4533|Thursday|1986Q4|N|N|N|2446736|2447039|2446383|2446656|N|N|N|N|N| +2446749|AAAAAAAANJFFFCAA|1986-11-14|1042|4533|348|1986|5|11|14|4|1986|348|4533|Friday|1986Q4|N|Y|N|2446736|2447039|2446384|2446657|N|N|N|N|N| +2446750|AAAAAAAAOJFFFCAA|1986-11-15|1042|4533|348|1986|6|11|15|4|1986|348|4533|Saturday|1986Q4|N|Y|N|2446736|2447039|2446385|2446658|N|N|N|N|N| +2446751|AAAAAAAAPJFFFCAA|1986-11-16|1042|4533|348|1986|0|11|16|4|1986|348|4533|Sunday|1986Q4|N|N|N|2446736|2447039|2446386|2446659|N|N|N|N|N| +2446752|AAAAAAAAAKFFFCAA|1986-11-17|1042|4533|348|1986|1|11|17|4|1986|348|4533|Monday|1986Q4|N|N|N|2446736|2447039|2446387|2446660|N|N|N|N|N| +2446753|AAAAAAAABKFFFCAA|1986-11-18|1042|4534|348|1986|2|11|18|4|1986|348|4534|Tuesday|1986Q4|N|N|N|2446736|2447039|2446388|2446661|N|N|N|N|N| +2446754|AAAAAAAACKFFFCAA|1986-11-19|1042|4534|348|1986|3|11|19|4|1986|348|4534|Wednesday|1986Q4|N|N|N|2446736|2447039|2446389|2446662|N|N|N|N|N| +2446755|AAAAAAAADKFFFCAA|1986-11-20|1042|4534|348|1986|4|11|20|4|1986|348|4534|Thursday|1986Q4|N|N|N|2446736|2447039|2446390|2446663|N|N|N|N|N| +2446756|AAAAAAAAEKFFFCAA|1986-11-21|1042|4534|348|1986|5|11|21|4|1986|348|4534|Friday|1986Q4|N|Y|N|2446736|2447039|2446391|2446664|N|N|N|N|N| +2446757|AAAAAAAAFKFFFCAA|1986-11-22|1042|4534|348|1986|6|11|22|4|1986|348|4534|Saturday|1986Q4|N|Y|N|2446736|2447039|2446392|2446665|N|N|N|N|N| +2446758|AAAAAAAAGKFFFCAA|1986-11-23|1042|4534|348|1986|0|11|23|4|1986|348|4534|Sunday|1986Q4|N|N|N|2446736|2447039|2446393|2446666|N|N|N|N|N| +2446759|AAAAAAAAHKFFFCAA|1986-11-24|1042|4534|348|1986|1|11|24|4|1986|348|4534|Monday|1986Q4|N|N|N|2446736|2447039|2446394|2446667|N|N|N|N|N| +2446760|AAAAAAAAIKFFFCAA|1986-11-25|1042|4535|348|1986|2|11|25|4|1986|348|4535|Tuesday|1986Q4|N|N|N|2446736|2447039|2446395|2446668|N|N|N|N|N| +2446761|AAAAAAAAJKFFFCAA|1986-11-26|1042|4535|348|1986|3|11|26|4|1986|348|4535|Wednesday|1986Q4|N|N|N|2446736|2447039|2446396|2446669|N|N|N|N|N| +2446762|AAAAAAAAKKFFFCAA|1986-11-27|1042|4535|348|1986|4|11|27|4|1986|348|4535|Thursday|1986Q4|N|N|N|2446736|2447039|2446397|2446670|N|N|N|N|N| +2446763|AAAAAAAALKFFFCAA|1986-11-28|1042|4535|348|1986|5|11|28|4|1986|348|4535|Friday|1986Q4|N|Y|N|2446736|2447039|2446398|2446671|N|N|N|N|N| +2446764|AAAAAAAAMKFFFCAA|1986-11-29|1042|4535|348|1986|6|11|29|4|1986|348|4535|Saturday|1986Q4|N|Y|N|2446736|2447039|2446399|2446672|N|N|N|N|N| +2446765|AAAAAAAANKFFFCAA|1986-11-30|1042|4535|348|1986|0|11|30|4|1986|348|4535|Sunday|1986Q4|N|N|N|2446736|2447039|2446400|2446673|N|N|N|N|N| +2446766|AAAAAAAAOKFFFCAA|1986-12-01|1043|4535|349|1986|1|12|1|4|1986|349|4535|Monday|1986Q4|N|N|N|2446766|2447099|2446401|2446674|N|N|N|N|N| +2446767|AAAAAAAAPKFFFCAA|1986-12-02|1043|4536|349|1986|2|12|2|4|1986|349|4536|Tuesday|1986Q4|N|N|N|2446766|2447099|2446402|2446675|N|N|N|N|N| +2446768|AAAAAAAAALFFFCAA|1986-12-03|1043|4536|349|1986|3|12|3|4|1986|349|4536|Wednesday|1986Q4|N|N|N|2446766|2447099|2446403|2446676|N|N|N|N|N| +2446769|AAAAAAAABLFFFCAA|1986-12-04|1043|4536|349|1986|4|12|4|4|1986|349|4536|Thursday|1986Q4|N|N|N|2446766|2447099|2446404|2446677|N|N|N|N|N| +2446770|AAAAAAAACLFFFCAA|1986-12-05|1043|4536|349|1986|5|12|5|4|1986|349|4536|Friday|1986Q4|N|Y|N|2446766|2447099|2446405|2446678|N|N|N|N|N| +2446771|AAAAAAAADLFFFCAA|1986-12-06|1043|4536|349|1986|6|12|6|4|1986|349|4536|Saturday|1986Q4|N|Y|N|2446766|2447099|2446406|2446679|N|N|N|N|N| +2446772|AAAAAAAAELFFFCAA|1986-12-07|1043|4536|349|1986|0|12|7|4|1986|349|4536|Sunday|1986Q4|N|N|N|2446766|2447099|2446407|2446680|N|N|N|N|N| +2446773|AAAAAAAAFLFFFCAA|1986-12-08|1043|4536|349|1986|1|12|8|4|1986|349|4536|Monday|1986Q4|N|N|N|2446766|2447099|2446408|2446681|N|N|N|N|N| +2446774|AAAAAAAAGLFFFCAA|1986-12-09|1043|4537|349|1986|2|12|9|4|1986|349|4537|Tuesday|1986Q4|N|N|N|2446766|2447099|2446409|2446682|N|N|N|N|N| +2446775|AAAAAAAAHLFFFCAA|1986-12-10|1043|4537|349|1986|3|12|10|4|1986|349|4537|Wednesday|1986Q4|N|N|N|2446766|2447099|2446410|2446683|N|N|N|N|N| +2446776|AAAAAAAAILFFFCAA|1986-12-11|1043|4537|349|1986|4|12|11|4|1986|349|4537|Thursday|1986Q4|N|N|N|2446766|2447099|2446411|2446684|N|N|N|N|N| +2446777|AAAAAAAAJLFFFCAA|1986-12-12|1043|4537|349|1986|5|12|12|4|1986|349|4537|Friday|1986Q4|N|Y|N|2446766|2447099|2446412|2446685|N|N|N|N|N| +2446778|AAAAAAAAKLFFFCAA|1986-12-13|1043|4537|349|1986|6|12|13|4|1986|349|4537|Saturday|1986Q4|N|Y|N|2446766|2447099|2446413|2446686|N|N|N|N|N| +2446779|AAAAAAAALLFFFCAA|1986-12-14|1043|4537|349|1986|0|12|14|4|1986|349|4537|Sunday|1986Q4|N|N|N|2446766|2447099|2446414|2446687|N|N|N|N|N| +2446780|AAAAAAAAMLFFFCAA|1986-12-15|1043|4537|349|1986|1|12|15|4|1986|349|4537|Monday|1986Q4|N|N|N|2446766|2447099|2446415|2446688|N|N|N|N|N| +2446781|AAAAAAAANLFFFCAA|1986-12-16|1043|4538|349|1986|2|12|16|4|1986|349|4538|Tuesday|1986Q4|N|N|N|2446766|2447099|2446416|2446689|N|N|N|N|N| +2446782|AAAAAAAAOLFFFCAA|1986-12-17|1043|4538|349|1986|3|12|17|4|1986|349|4538|Wednesday|1986Q4|N|N|N|2446766|2447099|2446417|2446690|N|N|N|N|N| +2446783|AAAAAAAAPLFFFCAA|1986-12-18|1043|4538|349|1986|4|12|18|4|1986|349|4538|Thursday|1986Q4|N|N|N|2446766|2447099|2446418|2446691|N|N|N|N|N| +2446784|AAAAAAAAAMFFFCAA|1986-12-19|1043|4538|349|1986|5|12|19|4|1986|349|4538|Friday|1986Q4|N|Y|N|2446766|2447099|2446419|2446692|N|N|N|N|N| +2446785|AAAAAAAABMFFFCAA|1986-12-20|1043|4538|349|1986|6|12|20|4|1986|349|4538|Saturday|1986Q4|N|Y|N|2446766|2447099|2446420|2446693|N|N|N|N|N| +2446786|AAAAAAAACMFFFCAA|1986-12-21|1043|4538|349|1986|0|12|21|4|1986|349|4538|Sunday|1986Q4|N|N|N|2446766|2447099|2446421|2446694|N|N|N|N|N| +2446787|AAAAAAAADMFFFCAA|1986-12-22|1043|4538|349|1986|1|12|22|4|1986|349|4538|Monday|1986Q4|N|N|N|2446766|2447099|2446422|2446695|N|N|N|N|N| +2446788|AAAAAAAAEMFFFCAA|1986-12-23|1043|4539|349|1986|2|12|23|4|1986|349|4539|Tuesday|1986Q4|N|N|N|2446766|2447099|2446423|2446696|N|N|N|N|N| +2446789|AAAAAAAAFMFFFCAA|1986-12-24|1043|4539|349|1986|3|12|24|4|1986|349|4539|Wednesday|1986Q4|N|N|N|2446766|2447099|2446424|2446697|N|N|N|N|N| +2446790|AAAAAAAAGMFFFCAA|1986-12-25|1043|4539|349|1986|4|12|25|4|1986|349|4539|Thursday|1986Q4|N|N|N|2446766|2447099|2446425|2446698|N|N|N|N|N| +2446791|AAAAAAAAHMFFFCAA|1986-12-26|1043|4539|349|1986|5|12|26|4|1986|349|4539|Friday|1986Q4|Y|Y|N|2446766|2447099|2446426|2446699|N|N|N|N|N| +2446792|AAAAAAAAIMFFFCAA|1986-12-27|1043|4539|349|1986|6|12|27|4|1986|349|4539|Saturday|1986Q4|N|Y|Y|2446766|2447099|2446427|2446700|N|N|N|N|N| +2446793|AAAAAAAAJMFFFCAA|1986-12-28|1043|4539|349|1986|0|12|28|4|1986|349|4539|Sunday|1986Q4|N|N|N|2446766|2447099|2446428|2446701|N|N|N|N|N| +2446794|AAAAAAAAKMFFFCAA|1986-12-29|1043|4539|349|1986|1|12|29|4|1986|349|4539|Monday|1986Q4|N|N|N|2446766|2447099|2446429|2446702|N|N|N|N|N| +2446795|AAAAAAAALMFFFCAA|1986-12-30|1043|4540|349|1986|2|12|30|4|1986|349|4540|Tuesday|1986Q4|N|N|N|2446766|2447099|2446430|2446703|N|N|N|N|N| +2446796|AAAAAAAAMMFFFCAA|1986-12-31|1043|4540|349|1986|3|12|31|4|1986|349|4540|Wednesday|1986Q4|N|N|N|2446766|2447099|2446431|2446704|N|N|N|N|N| +2446797|AAAAAAAANMFFFCAA|1987-01-01|1044|4540|349|1987|4|1|1|1|1987|349|4540|Thursday|1987Q1|Y|N|N|2446797|2446796|2446432|2446705|N|N|N|N|N| +2446798|AAAAAAAAOMFFFCAA|1987-01-02|1044|4540|349|1987|5|1|2|1|1987|349|4540|Friday|1987Q1|N|Y|Y|2446797|2446796|2446433|2446706|N|N|N|N|N| +2446799|AAAAAAAAPMFFFCAA|1987-01-03|1044|4540|349|1987|6|1|3|1|1987|349|4540|Saturday|1987Q1|N|Y|N|2446797|2446796|2446434|2446707|N|N|N|N|N| +2446800|AAAAAAAAANFFFCAA|1987-01-04|1044|4540|349|1987|0|1|4|1|1987|349|4540|Sunday|1987Q1|N|N|N|2446797|2446796|2446435|2446708|N|N|N|N|N| +2446801|AAAAAAAABNFFFCAA|1987-01-05|1044|4540|349|1987|1|1|5|1|1987|349|4540|Monday|1987Q1|N|N|N|2446797|2446796|2446436|2446709|N|N|N|N|N| +2446802|AAAAAAAACNFFFCAA|1987-01-06|1044|4541|349|1987|2|1|6|1|1987|349|4541|Tuesday|1987Q1|N|N|N|2446797|2446796|2446437|2446710|N|N|N|N|N| +2446803|AAAAAAAADNFFFCAA|1987-01-07|1044|4541|349|1987|3|1|7|1|1987|349|4541|Wednesday|1987Q1|N|N|N|2446797|2446796|2446438|2446711|N|N|N|N|N| +2446804|AAAAAAAAENFFFCAA|1987-01-08|1044|4541|349|1987|4|1|8|1|1987|349|4541|Thursday|1987Q1|N|N|N|2446797|2446796|2446439|2446712|N|N|N|N|N| +2446805|AAAAAAAAFNFFFCAA|1987-01-09|1044|4541|349|1987|5|1|9|1|1987|349|4541|Friday|1987Q1|N|Y|N|2446797|2446796|2446440|2446713|N|N|N|N|N| +2446806|AAAAAAAAGNFFFCAA|1987-01-10|1044|4541|349|1987|6|1|10|1|1987|349|4541|Saturday|1987Q1|N|Y|N|2446797|2446796|2446441|2446714|N|N|N|N|N| +2446807|AAAAAAAAHNFFFCAA|1987-01-11|1044|4541|349|1987|0|1|11|1|1987|349|4541|Sunday|1987Q1|N|N|N|2446797|2446796|2446442|2446715|N|N|N|N|N| +2446808|AAAAAAAAINFFFCAA|1987-01-12|1044|4541|349|1987|1|1|12|1|1987|349|4541|Monday|1987Q1|N|N|N|2446797|2446796|2446443|2446716|N|N|N|N|N| +2446809|AAAAAAAAJNFFFCAA|1987-01-13|1044|4542|349|1987|2|1|13|1|1987|349|4542|Tuesday|1987Q1|N|N|N|2446797|2446796|2446444|2446717|N|N|N|N|N| +2446810|AAAAAAAAKNFFFCAA|1987-01-14|1044|4542|349|1987|3|1|14|1|1987|349|4542|Wednesday|1987Q1|N|N|N|2446797|2446796|2446445|2446718|N|N|N|N|N| +2446811|AAAAAAAALNFFFCAA|1987-01-15|1044|4542|349|1987|4|1|15|1|1987|349|4542|Thursday|1987Q1|N|N|N|2446797|2446796|2446446|2446719|N|N|N|N|N| +2446812|AAAAAAAAMNFFFCAA|1987-01-16|1044|4542|349|1987|5|1|16|1|1987|349|4542|Friday|1987Q1|N|Y|N|2446797|2446796|2446447|2446720|N|N|N|N|N| +2446813|AAAAAAAANNFFFCAA|1987-01-17|1044|4542|349|1987|6|1|17|1|1987|349|4542|Saturday|1987Q1|N|Y|N|2446797|2446796|2446448|2446721|N|N|N|N|N| +2446814|AAAAAAAAONFFFCAA|1987-01-18|1044|4542|349|1987|0|1|18|1|1987|349|4542|Sunday|1987Q1|N|N|N|2446797|2446796|2446449|2446722|N|N|N|N|N| +2446815|AAAAAAAAPNFFFCAA|1987-01-19|1044|4542|349|1987|1|1|19|1|1987|349|4542|Monday|1987Q1|N|N|N|2446797|2446796|2446450|2446723|N|N|N|N|N| +2446816|AAAAAAAAAOFFFCAA|1987-01-20|1044|4543|349|1987|2|1|20|1|1987|349|4543|Tuesday|1987Q1|N|N|N|2446797|2446796|2446451|2446724|N|N|N|N|N| +2446817|AAAAAAAABOFFFCAA|1987-01-21|1044|4543|349|1987|3|1|21|1|1987|349|4543|Wednesday|1987Q1|N|N|N|2446797|2446796|2446452|2446725|N|N|N|N|N| +2446818|AAAAAAAACOFFFCAA|1987-01-22|1044|4543|349|1987|4|1|22|1|1987|349|4543|Thursday|1987Q1|N|N|N|2446797|2446796|2446453|2446726|N|N|N|N|N| +2446819|AAAAAAAADOFFFCAA|1987-01-23|1044|4543|349|1987|5|1|23|1|1987|349|4543|Friday|1987Q1|N|Y|N|2446797|2446796|2446454|2446727|N|N|N|N|N| +2446820|AAAAAAAAEOFFFCAA|1987-01-24|1044|4543|349|1987|6|1|24|1|1987|349|4543|Saturday|1987Q1|N|Y|N|2446797|2446796|2446455|2446728|N|N|N|N|N| +2446821|AAAAAAAAFOFFFCAA|1987-01-25|1044|4543|349|1987|0|1|25|1|1987|349|4543|Sunday|1987Q1|N|N|N|2446797|2446796|2446456|2446729|N|N|N|N|N| +2446822|AAAAAAAAGOFFFCAA|1987-01-26|1044|4543|349|1987|1|1|26|1|1987|349|4543|Monday|1987Q1|N|N|N|2446797|2446796|2446457|2446730|N|N|N|N|N| +2446823|AAAAAAAAHOFFFCAA|1987-01-27|1044|4544|349|1987|2|1|27|1|1987|349|4544|Tuesday|1987Q1|N|N|N|2446797|2446796|2446458|2446731|N|N|N|N|N| +2446824|AAAAAAAAIOFFFCAA|1987-01-28|1044|4544|349|1987|3|1|28|1|1987|349|4544|Wednesday|1987Q1|N|N|N|2446797|2446796|2446459|2446732|N|N|N|N|N| +2446825|AAAAAAAAJOFFFCAA|1987-01-29|1044|4544|349|1987|4|1|29|1|1987|349|4544|Thursday|1987Q1|N|N|N|2446797|2446796|2446460|2446733|N|N|N|N|N| +2446826|AAAAAAAAKOFFFCAA|1987-01-30|1044|4544|349|1987|5|1|30|1|1987|349|4544|Friday|1987Q1|N|Y|N|2446797|2446796|2446461|2446734|N|N|N|N|N| +2446827|AAAAAAAALOFFFCAA|1987-01-31|1044|4544|349|1987|6|1|31|1|1987|349|4544|Saturday|1987Q1|N|Y|N|2446797|2446796|2446462|2446735|N|N|N|N|N| +2446828|AAAAAAAAMOFFFCAA|1987-02-01|1045|4544|349|1987|0|2|1|1|1987|349|4544|Sunday|1987Q1|N|N|N|2446828|2446858|2446463|2446736|N|N|N|N|N| +2446829|AAAAAAAANOFFFCAA|1987-02-02|1045|4544|349|1987|1|2|2|1|1987|349|4544|Monday|1987Q1|N|N|N|2446828|2446858|2446464|2446737|N|N|N|N|N| +2446830|AAAAAAAAOOFFFCAA|1987-02-03|1045|4545|349|1987|2|2|3|1|1987|349|4545|Tuesday|1987Q1|N|N|N|2446828|2446858|2446465|2446738|N|N|N|N|N| +2446831|AAAAAAAAPOFFFCAA|1987-02-04|1045|4545|349|1987|3|2|4|1|1987|349|4545|Wednesday|1987Q1|N|N|N|2446828|2446858|2446466|2446739|N|N|N|N|N| +2446832|AAAAAAAAAPFFFCAA|1987-02-05|1045|4545|349|1987|4|2|5|1|1987|349|4545|Thursday|1987Q1|N|N|N|2446828|2446858|2446467|2446740|N|N|N|N|N| +2446833|AAAAAAAABPFFFCAA|1987-02-06|1045|4545|349|1987|5|2|6|1|1987|349|4545|Friday|1987Q1|N|Y|N|2446828|2446858|2446468|2446741|N|N|N|N|N| +2446834|AAAAAAAACPFFFCAA|1987-02-07|1045|4545|349|1987|6|2|7|1|1987|349|4545|Saturday|1987Q1|N|Y|N|2446828|2446858|2446469|2446742|N|N|N|N|N| +2446835|AAAAAAAADPFFFCAA|1987-02-08|1045|4545|349|1987|0|2|8|1|1987|349|4545|Sunday|1987Q1|N|N|N|2446828|2446858|2446470|2446743|N|N|N|N|N| +2446836|AAAAAAAAEPFFFCAA|1987-02-09|1045|4545|349|1987|1|2|9|1|1987|349|4545|Monday|1987Q1|N|N|N|2446828|2446858|2446471|2446744|N|N|N|N|N| +2446837|AAAAAAAAFPFFFCAA|1987-02-10|1045|4546|349|1987|2|2|10|1|1987|349|4546|Tuesday|1987Q1|N|N|N|2446828|2446858|2446472|2446745|N|N|N|N|N| +2446838|AAAAAAAAGPFFFCAA|1987-02-11|1045|4546|349|1987|3|2|11|1|1987|349|4546|Wednesday|1987Q1|N|N|N|2446828|2446858|2446473|2446746|N|N|N|N|N| +2446839|AAAAAAAAHPFFFCAA|1987-02-12|1045|4546|349|1987|4|2|12|1|1987|349|4546|Thursday|1987Q1|N|N|N|2446828|2446858|2446474|2446747|N|N|N|N|N| +2446840|AAAAAAAAIPFFFCAA|1987-02-13|1045|4546|349|1987|5|2|13|1|1987|349|4546|Friday|1987Q1|N|Y|N|2446828|2446858|2446475|2446748|N|N|N|N|N| +2446841|AAAAAAAAJPFFFCAA|1987-02-14|1045|4546|349|1987|6|2|14|1|1987|349|4546|Saturday|1987Q1|N|Y|N|2446828|2446858|2446476|2446749|N|N|N|N|N| +2446842|AAAAAAAAKPFFFCAA|1987-02-15|1045|4546|349|1987|0|2|15|1|1987|349|4546|Sunday|1987Q1|N|N|N|2446828|2446858|2446477|2446750|N|N|N|N|N| +2446843|AAAAAAAALPFFFCAA|1987-02-16|1045|4546|349|1987|1|2|16|1|1987|349|4546|Monday|1987Q1|N|N|N|2446828|2446858|2446478|2446751|N|N|N|N|N| +2446844|AAAAAAAAMPFFFCAA|1987-02-17|1045|4547|349|1987|2|2|17|1|1987|349|4547|Tuesday|1987Q1|N|N|N|2446828|2446858|2446479|2446752|N|N|N|N|N| +2446845|AAAAAAAANPFFFCAA|1987-02-18|1045|4547|349|1987|3|2|18|1|1987|349|4547|Wednesday|1987Q1|N|N|N|2446828|2446858|2446480|2446753|N|N|N|N|N| +2446846|AAAAAAAAOPFFFCAA|1987-02-19|1045|4547|349|1987|4|2|19|1|1987|349|4547|Thursday|1987Q1|N|N|N|2446828|2446858|2446481|2446754|N|N|N|N|N| +2446847|AAAAAAAAPPFFFCAA|1987-02-20|1045|4547|349|1987|5|2|20|1|1987|349|4547|Friday|1987Q1|N|Y|N|2446828|2446858|2446482|2446755|N|N|N|N|N| +2446848|AAAAAAAAAAGFFCAA|1987-02-21|1045|4547|349|1987|6|2|21|1|1987|349|4547|Saturday|1987Q1|N|Y|N|2446828|2446858|2446483|2446756|N|N|N|N|N| +2446849|AAAAAAAABAGFFCAA|1987-02-22|1045|4547|349|1987|0|2|22|1|1987|349|4547|Sunday|1987Q1|N|N|N|2446828|2446858|2446484|2446757|N|N|N|N|N| +2446850|AAAAAAAACAGFFCAA|1987-02-23|1045|4547|349|1987|1|2|23|1|1987|349|4547|Monday|1987Q1|N|N|N|2446828|2446858|2446485|2446758|N|N|N|N|N| +2446851|AAAAAAAADAGFFCAA|1987-02-24|1045|4548|349|1987|2|2|24|1|1987|349|4548|Tuesday|1987Q1|N|N|N|2446828|2446858|2446486|2446759|N|N|N|N|N| +2446852|AAAAAAAAEAGFFCAA|1987-02-25|1045|4548|349|1987|3|2|25|1|1987|349|4548|Wednesday|1987Q1|N|N|N|2446828|2446858|2446487|2446760|N|N|N|N|N| +2446853|AAAAAAAAFAGFFCAA|1987-02-26|1045|4548|349|1987|4|2|26|1|1987|349|4548|Thursday|1987Q1|N|N|N|2446828|2446858|2446488|2446761|N|N|N|N|N| +2446854|AAAAAAAAGAGFFCAA|1987-02-27|1045|4548|349|1987|5|2|27|1|1987|349|4548|Friday|1987Q1|N|Y|N|2446828|2446858|2446489|2446762|N|N|N|N|N| +2446855|AAAAAAAAHAGFFCAA|1987-02-28|1045|4548|349|1987|6|2|28|1|1987|349|4548|Saturday|1987Q1|N|Y|N|2446828|2446858|2446490|2446763|N|N|N|N|N| +2446856|AAAAAAAAIAGFFCAA|1987-03-01|1046|4548|350|1987|0|3|1|1|1987|350|4548|Sunday|1987Q1|N|N|N|2446856|2446914|2446491|2446764|N|N|N|N|N| +2446857|AAAAAAAAJAGFFCAA|1987-03-02|1046|4548|350|1987|1|3|2|1|1987|350|4548|Monday|1987Q1|N|N|N|2446856|2446914|2446492|2446765|N|N|N|N|N| +2446858|AAAAAAAAKAGFFCAA|1987-03-03|1046|4549|350|1987|2|3|3|1|1987|350|4549|Tuesday|1987Q1|N|N|N|2446856|2446914|2446493|2446766|N|N|N|N|N| +2446859|AAAAAAAALAGFFCAA|1987-03-04|1046|4549|350|1987|3|3|4|1|1987|350|4549|Wednesday|1987Q1|N|N|N|2446856|2446914|2446494|2446767|N|N|N|N|N| +2446860|AAAAAAAAMAGFFCAA|1987-03-05|1046|4549|350|1987|4|3|5|1|1987|350|4549|Thursday|1987Q1|N|N|N|2446856|2446914|2446495|2446768|N|N|N|N|N| +2446861|AAAAAAAANAGFFCAA|1987-03-06|1046|4549|350|1987|5|3|6|1|1987|350|4549|Friday|1987Q1|N|Y|N|2446856|2446914|2446496|2446769|N|N|N|N|N| +2446862|AAAAAAAAOAGFFCAA|1987-03-07|1046|4549|350|1987|6|3|7|1|1987|350|4549|Saturday|1987Q1|N|Y|N|2446856|2446914|2446497|2446770|N|N|N|N|N| +2446863|AAAAAAAAPAGFFCAA|1987-03-08|1046|4549|350|1987|0|3|8|1|1987|350|4549|Sunday|1987Q1|N|N|N|2446856|2446914|2446498|2446771|N|N|N|N|N| +2446864|AAAAAAAAABGFFCAA|1987-03-09|1046|4549|350|1987|1|3|9|1|1987|350|4549|Monday|1987Q1|N|N|N|2446856|2446914|2446499|2446772|N|N|N|N|N| +2446865|AAAAAAAABBGFFCAA|1987-03-10|1046|4550|350|1987|2|3|10|1|1987|350|4550|Tuesday|1987Q1|N|N|N|2446856|2446914|2446500|2446773|N|N|N|N|N| +2446866|AAAAAAAACBGFFCAA|1987-03-11|1046|4550|350|1987|3|3|11|1|1987|350|4550|Wednesday|1987Q1|N|N|N|2446856|2446914|2446501|2446774|N|N|N|N|N| +2446867|AAAAAAAADBGFFCAA|1987-03-12|1046|4550|350|1987|4|3|12|1|1987|350|4550|Thursday|1987Q1|N|N|N|2446856|2446914|2446502|2446775|N|N|N|N|N| +2446868|AAAAAAAAEBGFFCAA|1987-03-13|1046|4550|350|1987|5|3|13|1|1987|350|4550|Friday|1987Q1|N|Y|N|2446856|2446914|2446503|2446776|N|N|N|N|N| +2446869|AAAAAAAAFBGFFCAA|1987-03-14|1046|4550|350|1987|6|3|14|1|1987|350|4550|Saturday|1987Q1|N|Y|N|2446856|2446914|2446504|2446777|N|N|N|N|N| +2446870|AAAAAAAAGBGFFCAA|1987-03-15|1046|4550|350|1987|0|3|15|1|1987|350|4550|Sunday|1987Q1|N|N|N|2446856|2446914|2446505|2446778|N|N|N|N|N| +2446871|AAAAAAAAHBGFFCAA|1987-03-16|1046|4550|350|1987|1|3|16|1|1987|350|4550|Monday|1987Q1|N|N|N|2446856|2446914|2446506|2446779|N|N|N|N|N| +2446872|AAAAAAAAIBGFFCAA|1987-03-17|1046|4551|350|1987|2|3|17|1|1987|350|4551|Tuesday|1987Q1|N|N|N|2446856|2446914|2446507|2446780|N|N|N|N|N| +2446873|AAAAAAAAJBGFFCAA|1987-03-18|1046|4551|350|1987|3|3|18|1|1987|350|4551|Wednesday|1987Q1|N|N|N|2446856|2446914|2446508|2446781|N|N|N|N|N| +2446874|AAAAAAAAKBGFFCAA|1987-03-19|1046|4551|350|1987|4|3|19|1|1987|350|4551|Thursday|1987Q1|N|N|N|2446856|2446914|2446509|2446782|N|N|N|N|N| +2446875|AAAAAAAALBGFFCAA|1987-03-20|1046|4551|350|1987|5|3|20|1|1987|350|4551|Friday|1987Q1|N|Y|N|2446856|2446914|2446510|2446783|N|N|N|N|N| +2446876|AAAAAAAAMBGFFCAA|1987-03-21|1046|4551|350|1987|6|3|21|1|1987|350|4551|Saturday|1987Q1|N|Y|N|2446856|2446914|2446511|2446784|N|N|N|N|N| +2446877|AAAAAAAANBGFFCAA|1987-03-22|1046|4551|350|1987|0|3|22|1|1987|350|4551|Sunday|1987Q1|N|N|N|2446856|2446914|2446512|2446785|N|N|N|N|N| +2446878|AAAAAAAAOBGFFCAA|1987-03-23|1046|4551|350|1987|1|3|23|1|1987|350|4551|Monday|1987Q1|N|N|N|2446856|2446914|2446513|2446786|N|N|N|N|N| +2446879|AAAAAAAAPBGFFCAA|1987-03-24|1046|4552|350|1987|2|3|24|1|1987|350|4552|Tuesday|1987Q1|N|N|N|2446856|2446914|2446514|2446787|N|N|N|N|N| +2446880|AAAAAAAAACGFFCAA|1987-03-25|1046|4552|350|1987|3|3|25|1|1987|350|4552|Wednesday|1987Q1|N|N|N|2446856|2446914|2446515|2446788|N|N|N|N|N| +2446881|AAAAAAAABCGFFCAA|1987-03-26|1046|4552|350|1987|4|3|26|1|1987|350|4552|Thursday|1987Q1|N|N|N|2446856|2446914|2446516|2446789|N|N|N|N|N| +2446882|AAAAAAAACCGFFCAA|1987-03-27|1046|4552|350|1987|5|3|27|1|1987|350|4552|Friday|1987Q1|N|Y|N|2446856|2446914|2446517|2446790|N|N|N|N|N| +2446883|AAAAAAAADCGFFCAA|1987-03-28|1046|4552|350|1987|6|3|28|1|1987|350|4552|Saturday|1987Q1|N|Y|N|2446856|2446914|2446518|2446791|N|N|N|N|N| +2446884|AAAAAAAAECGFFCAA|1987-03-29|1046|4552|350|1987|0|3|29|1|1987|350|4552|Sunday|1987Q1|N|N|N|2446856|2446914|2446519|2446792|N|N|N|N|N| +2446885|AAAAAAAAFCGFFCAA|1987-03-30|1046|4552|350|1987|1|3|30|1|1987|350|4552|Monday|1987Q1|N|N|N|2446856|2446914|2446520|2446793|N|N|N|N|N| +2446886|AAAAAAAAGCGFFCAA|1987-03-31|1046|4553|350|1987|2|3|31|1|1987|350|4553|Tuesday|1987Q1|N|N|N|2446856|2446914|2446521|2446794|N|N|N|N|N| +2446887|AAAAAAAAHCGFFCAA|1987-04-01|1047|4553|350|1987|3|4|1|1|1987|350|4553|Wednesday|1987Q1|N|N|N|2446887|2446976|2446522|2446797|N|N|N|N|N| +2446888|AAAAAAAAICGFFCAA|1987-04-02|1047|4553|350|1987|4|4|2|2|1987|350|4553|Thursday|1987Q2|N|N|N|2446887|2446976|2446523|2446798|N|N|N|N|N| +2446889|AAAAAAAAJCGFFCAA|1987-04-03|1047|4553|350|1987|5|4|3|2|1987|350|4553|Friday|1987Q2|N|Y|N|2446887|2446976|2446524|2446799|N|N|N|N|N| +2446890|AAAAAAAAKCGFFCAA|1987-04-04|1047|4553|350|1987|6|4|4|2|1987|350|4553|Saturday|1987Q2|N|Y|N|2446887|2446976|2446525|2446800|N|N|N|N|N| +2446891|AAAAAAAALCGFFCAA|1987-04-05|1047|4553|350|1987|0|4|5|2|1987|350|4553|Sunday|1987Q2|N|N|N|2446887|2446976|2446526|2446801|N|N|N|N|N| +2446892|AAAAAAAAMCGFFCAA|1987-04-06|1047|4553|350|1987|1|4|6|2|1987|350|4553|Monday|1987Q2|N|N|N|2446887|2446976|2446527|2446802|N|N|N|N|N| +2446893|AAAAAAAANCGFFCAA|1987-04-07|1047|4554|350|1987|2|4|7|2|1987|350|4554|Tuesday|1987Q2|N|N|N|2446887|2446976|2446528|2446803|N|N|N|N|N| +2446894|AAAAAAAAOCGFFCAA|1987-04-08|1047|4554|350|1987|3|4|8|2|1987|350|4554|Wednesday|1987Q2|N|N|N|2446887|2446976|2446529|2446804|N|N|N|N|N| +2446895|AAAAAAAAPCGFFCAA|1987-04-09|1047|4554|350|1987|4|4|9|2|1987|350|4554|Thursday|1987Q2|N|N|N|2446887|2446976|2446530|2446805|N|N|N|N|N| +2446896|AAAAAAAAADGFFCAA|1987-04-10|1047|4554|350|1987|5|4|10|2|1987|350|4554|Friday|1987Q2|N|Y|N|2446887|2446976|2446531|2446806|N|N|N|N|N| +2446897|AAAAAAAABDGFFCAA|1987-04-11|1047|4554|350|1987|6|4|11|2|1987|350|4554|Saturday|1987Q2|N|Y|N|2446887|2446976|2446532|2446807|N|N|N|N|N| +2446898|AAAAAAAACDGFFCAA|1987-04-12|1047|4554|350|1987|0|4|12|2|1987|350|4554|Sunday|1987Q2|N|N|N|2446887|2446976|2446533|2446808|N|N|N|N|N| +2446899|AAAAAAAADDGFFCAA|1987-04-13|1047|4554|350|1987|1|4|13|2|1987|350|4554|Monday|1987Q2|N|N|N|2446887|2446976|2446534|2446809|N|N|N|N|N| +2446900|AAAAAAAAEDGFFCAA|1987-04-14|1047|4555|350|1987|2|4|14|2|1987|350|4555|Tuesday|1987Q2|N|N|N|2446887|2446976|2446535|2446810|N|N|N|N|N| +2446901|AAAAAAAAFDGFFCAA|1987-04-15|1047|4555|350|1987|3|4|15|2|1987|350|4555|Wednesday|1987Q2|N|N|N|2446887|2446976|2446536|2446811|N|N|N|N|N| +2446902|AAAAAAAAGDGFFCAA|1987-04-16|1047|4555|350|1987|4|4|16|2|1987|350|4555|Thursday|1987Q2|N|N|N|2446887|2446976|2446537|2446812|N|N|N|N|N| +2446903|AAAAAAAAHDGFFCAA|1987-04-17|1047|4555|350|1987|5|4|17|2|1987|350|4555|Friday|1987Q2|N|Y|N|2446887|2446976|2446538|2446813|N|N|N|N|N| +2446904|AAAAAAAAIDGFFCAA|1987-04-18|1047|4555|350|1987|6|4|18|2|1987|350|4555|Saturday|1987Q2|N|Y|N|2446887|2446976|2446539|2446814|N|N|N|N|N| +2446905|AAAAAAAAJDGFFCAA|1987-04-19|1047|4555|350|1987|0|4|19|2|1987|350|4555|Sunday|1987Q2|N|N|N|2446887|2446976|2446540|2446815|N|N|N|N|N| +2446906|AAAAAAAAKDGFFCAA|1987-04-20|1047|4555|350|1987|1|4|20|2|1987|350|4555|Monday|1987Q2|N|N|N|2446887|2446976|2446541|2446816|N|N|N|N|N| +2446907|AAAAAAAALDGFFCAA|1987-04-21|1047|4556|350|1987|2|4|21|2|1987|350|4556|Tuesday|1987Q2|N|N|N|2446887|2446976|2446542|2446817|N|N|N|N|N| +2446908|AAAAAAAAMDGFFCAA|1987-04-22|1047|4556|350|1987|3|4|22|2|1987|350|4556|Wednesday|1987Q2|N|N|N|2446887|2446976|2446543|2446818|N|N|N|N|N| +2446909|AAAAAAAANDGFFCAA|1987-04-23|1047|4556|350|1987|4|4|23|2|1987|350|4556|Thursday|1987Q2|N|N|N|2446887|2446976|2446544|2446819|N|N|N|N|N| +2446910|AAAAAAAAODGFFCAA|1987-04-24|1047|4556|350|1987|5|4|24|2|1987|350|4556|Friday|1987Q2|N|Y|N|2446887|2446976|2446545|2446820|N|N|N|N|N| +2446911|AAAAAAAAPDGFFCAA|1987-04-25|1047|4556|350|1987|6|4|25|2|1987|350|4556|Saturday|1987Q2|N|Y|N|2446887|2446976|2446546|2446821|N|N|N|N|N| +2446912|AAAAAAAAAEGFFCAA|1987-04-26|1047|4556|350|1987|0|4|26|2|1987|350|4556|Sunday|1987Q2|N|N|N|2446887|2446976|2446547|2446822|N|N|N|N|N| +2446913|AAAAAAAABEGFFCAA|1987-04-27|1047|4556|350|1987|1|4|27|2|1987|350|4556|Monday|1987Q2|N|N|N|2446887|2446976|2446548|2446823|N|N|N|N|N| +2446914|AAAAAAAACEGFFCAA|1987-04-28|1047|4557|350|1987|2|4|28|2|1987|350|4557|Tuesday|1987Q2|N|N|N|2446887|2446976|2446549|2446824|N|N|N|N|N| +2446915|AAAAAAAADEGFFCAA|1987-04-29|1047|4557|350|1987|3|4|29|2|1987|350|4557|Wednesday|1987Q2|N|N|N|2446887|2446976|2446550|2446825|N|N|N|N|N| +2446916|AAAAAAAAEEGFFCAA|1987-04-30|1047|4557|350|1987|4|4|30|2|1987|350|4557|Thursday|1987Q2|N|N|N|2446887|2446976|2446551|2446826|N|N|N|N|N| +2446917|AAAAAAAAFEGFFCAA|1987-05-01|1048|4557|350|1987|5|5|1|2|1987|350|4557|Friday|1987Q2|N|Y|N|2446917|2447036|2446552|2446827|N|N|N|N|N| +2446918|AAAAAAAAGEGFFCAA|1987-05-02|1048|4557|350|1987|6|5|2|2|1987|350|4557|Saturday|1987Q2|N|Y|N|2446917|2447036|2446553|2446828|N|N|N|N|N| +2446919|AAAAAAAAHEGFFCAA|1987-05-03|1048|4557|350|1987|0|5|3|2|1987|350|4557|Sunday|1987Q2|N|N|N|2446917|2447036|2446554|2446829|N|N|N|N|N| +2446920|AAAAAAAAIEGFFCAA|1987-05-04|1048|4557|350|1987|1|5|4|2|1987|350|4557|Monday|1987Q2|N|N|N|2446917|2447036|2446555|2446830|N|N|N|N|N| +2446921|AAAAAAAAJEGFFCAA|1987-05-05|1048|4558|350|1987|2|5|5|2|1987|350|4558|Tuesday|1987Q2|N|N|N|2446917|2447036|2446556|2446831|N|N|N|N|N| +2446922|AAAAAAAAKEGFFCAA|1987-05-06|1048|4558|350|1987|3|5|6|2|1987|350|4558|Wednesday|1987Q2|N|N|N|2446917|2447036|2446557|2446832|N|N|N|N|N| +2446923|AAAAAAAALEGFFCAA|1987-05-07|1048|4558|350|1987|4|5|7|2|1987|350|4558|Thursday|1987Q2|N|N|N|2446917|2447036|2446558|2446833|N|N|N|N|N| +2446924|AAAAAAAAMEGFFCAA|1987-05-08|1048|4558|350|1987|5|5|8|2|1987|350|4558|Friday|1987Q2|N|Y|N|2446917|2447036|2446559|2446834|N|N|N|N|N| +2446925|AAAAAAAANEGFFCAA|1987-05-09|1048|4558|350|1987|6|5|9|2|1987|350|4558|Saturday|1987Q2|N|Y|N|2446917|2447036|2446560|2446835|N|N|N|N|N| +2446926|AAAAAAAAOEGFFCAA|1987-05-10|1048|4558|350|1987|0|5|10|2|1987|350|4558|Sunday|1987Q2|N|N|N|2446917|2447036|2446561|2446836|N|N|N|N|N| +2446927|AAAAAAAAPEGFFCAA|1987-05-11|1048|4558|350|1987|1|5|11|2|1987|350|4558|Monday|1987Q2|N|N|N|2446917|2447036|2446562|2446837|N|N|N|N|N| +2446928|AAAAAAAAAFGFFCAA|1987-05-12|1048|4559|350|1987|2|5|12|2|1987|350|4559|Tuesday|1987Q2|N|N|N|2446917|2447036|2446563|2446838|N|N|N|N|N| +2446929|AAAAAAAABFGFFCAA|1987-05-13|1048|4559|350|1987|3|5|13|2|1987|350|4559|Wednesday|1987Q2|N|N|N|2446917|2447036|2446564|2446839|N|N|N|N|N| +2446930|AAAAAAAACFGFFCAA|1987-05-14|1048|4559|350|1987|4|5|14|2|1987|350|4559|Thursday|1987Q2|N|N|N|2446917|2447036|2446565|2446840|N|N|N|N|N| +2446931|AAAAAAAADFGFFCAA|1987-05-15|1048|4559|350|1987|5|5|15|2|1987|350|4559|Friday|1987Q2|N|Y|N|2446917|2447036|2446566|2446841|N|N|N|N|N| +2446932|AAAAAAAAEFGFFCAA|1987-05-16|1048|4559|350|1987|6|5|16|2|1987|350|4559|Saturday|1987Q2|N|Y|N|2446917|2447036|2446567|2446842|N|N|N|N|N| +2446933|AAAAAAAAFFGFFCAA|1987-05-17|1048|4559|350|1987|0|5|17|2|1987|350|4559|Sunday|1987Q2|N|N|N|2446917|2447036|2446568|2446843|N|N|N|N|N| +2446934|AAAAAAAAGFGFFCAA|1987-05-18|1048|4559|350|1987|1|5|18|2|1987|350|4559|Monday|1987Q2|N|N|N|2446917|2447036|2446569|2446844|N|N|N|N|N| +2446935|AAAAAAAAHFGFFCAA|1987-05-19|1048|4560|350|1987|2|5|19|2|1987|350|4560|Tuesday|1987Q2|N|N|N|2446917|2447036|2446570|2446845|N|N|N|N|N| +2446936|AAAAAAAAIFGFFCAA|1987-05-20|1048|4560|350|1987|3|5|20|2|1987|350|4560|Wednesday|1987Q2|N|N|N|2446917|2447036|2446571|2446846|N|N|N|N|N| +2446937|AAAAAAAAJFGFFCAA|1987-05-21|1048|4560|350|1987|4|5|21|2|1987|350|4560|Thursday|1987Q2|N|N|N|2446917|2447036|2446572|2446847|N|N|N|N|N| +2446938|AAAAAAAAKFGFFCAA|1987-05-22|1048|4560|350|1987|5|5|22|2|1987|350|4560|Friday|1987Q2|N|Y|N|2446917|2447036|2446573|2446848|N|N|N|N|N| +2446939|AAAAAAAALFGFFCAA|1987-05-23|1048|4560|350|1987|6|5|23|2|1987|350|4560|Saturday|1987Q2|N|Y|N|2446917|2447036|2446574|2446849|N|N|N|N|N| +2446940|AAAAAAAAMFGFFCAA|1987-05-24|1048|4560|350|1987|0|5|24|2|1987|350|4560|Sunday|1987Q2|N|N|N|2446917|2447036|2446575|2446850|N|N|N|N|N| +2446941|AAAAAAAANFGFFCAA|1987-05-25|1048|4560|350|1987|1|5|25|2|1987|350|4560|Monday|1987Q2|N|N|N|2446917|2447036|2446576|2446851|N|N|N|N|N| +2446942|AAAAAAAAOFGFFCAA|1987-05-26|1048|4561|350|1987|2|5|26|2|1987|350|4561|Tuesday|1987Q2|N|N|N|2446917|2447036|2446577|2446852|N|N|N|N|N| +2446943|AAAAAAAAPFGFFCAA|1987-05-27|1048|4561|350|1987|3|5|27|2|1987|350|4561|Wednesday|1987Q2|N|N|N|2446917|2447036|2446578|2446853|N|N|N|N|N| +2446944|AAAAAAAAAGGFFCAA|1987-05-28|1048|4561|350|1987|4|5|28|2|1987|350|4561|Thursday|1987Q2|N|N|N|2446917|2447036|2446579|2446854|N|N|N|N|N| +2446945|AAAAAAAABGGFFCAA|1987-05-29|1048|4561|350|1987|5|5|29|2|1987|350|4561|Friday|1987Q2|N|Y|N|2446917|2447036|2446580|2446855|N|N|N|N|N| +2446946|AAAAAAAACGGFFCAA|1987-05-30|1048|4561|350|1987|6|5|30|2|1987|350|4561|Saturday|1987Q2|N|Y|N|2446917|2447036|2446581|2446856|N|N|N|N|N| +2446947|AAAAAAAADGGFFCAA|1987-05-31|1048|4561|350|1987|0|5|31|2|1987|350|4561|Sunday|1987Q2|N|N|N|2446917|2447036|2446582|2446857|N|N|N|N|N| +2446948|AAAAAAAAEGGFFCAA|1987-06-01|1049|4561|351|1987|1|6|1|2|1987|351|4561|Monday|1987Q2|N|N|N|2446948|2447098|2446583|2446858|N|N|N|N|N| +2446949|AAAAAAAAFGGFFCAA|1987-06-02|1049|4562|351|1987|2|6|2|2|1987|351|4562|Tuesday|1987Q2|N|N|N|2446948|2447098|2446584|2446859|N|N|N|N|N| +2446950|AAAAAAAAGGGFFCAA|1987-06-03|1049|4562|351|1987|3|6|3|2|1987|351|4562|Wednesday|1987Q2|N|N|N|2446948|2447098|2446585|2446860|N|N|N|N|N| +2446951|AAAAAAAAHGGFFCAA|1987-06-04|1049|4562|351|1987|4|6|4|2|1987|351|4562|Thursday|1987Q2|N|N|N|2446948|2447098|2446586|2446861|N|N|N|N|N| +2446952|AAAAAAAAIGGFFCAA|1987-06-05|1049|4562|351|1987|5|6|5|2|1987|351|4562|Friday|1987Q2|N|Y|N|2446948|2447098|2446587|2446862|N|N|N|N|N| +2446953|AAAAAAAAJGGFFCAA|1987-06-06|1049|4562|351|1987|6|6|6|2|1987|351|4562|Saturday|1987Q2|N|Y|N|2446948|2447098|2446588|2446863|N|N|N|N|N| +2446954|AAAAAAAAKGGFFCAA|1987-06-07|1049|4562|351|1987|0|6|7|2|1987|351|4562|Sunday|1987Q2|N|N|N|2446948|2447098|2446589|2446864|N|N|N|N|N| +2446955|AAAAAAAALGGFFCAA|1987-06-08|1049|4562|351|1987|1|6|8|2|1987|351|4562|Monday|1987Q2|N|N|N|2446948|2447098|2446590|2446865|N|N|N|N|N| +2446956|AAAAAAAAMGGFFCAA|1987-06-09|1049|4563|351|1987|2|6|9|2|1987|351|4563|Tuesday|1987Q2|N|N|N|2446948|2447098|2446591|2446866|N|N|N|N|N| +2446957|AAAAAAAANGGFFCAA|1987-06-10|1049|4563|351|1987|3|6|10|2|1987|351|4563|Wednesday|1987Q2|N|N|N|2446948|2447098|2446592|2446867|N|N|N|N|N| +2446958|AAAAAAAAOGGFFCAA|1987-06-11|1049|4563|351|1987|4|6|11|2|1987|351|4563|Thursday|1987Q2|N|N|N|2446948|2447098|2446593|2446868|N|N|N|N|N| +2446959|AAAAAAAAPGGFFCAA|1987-06-12|1049|4563|351|1987|5|6|12|2|1987|351|4563|Friday|1987Q2|N|Y|N|2446948|2447098|2446594|2446869|N|N|N|N|N| +2446960|AAAAAAAAAHGFFCAA|1987-06-13|1049|4563|351|1987|6|6|13|2|1987|351|4563|Saturday|1987Q2|N|Y|N|2446948|2447098|2446595|2446870|N|N|N|N|N| +2446961|AAAAAAAABHGFFCAA|1987-06-14|1049|4563|351|1987|0|6|14|2|1987|351|4563|Sunday|1987Q2|N|N|N|2446948|2447098|2446596|2446871|N|N|N|N|N| +2446962|AAAAAAAACHGFFCAA|1987-06-15|1049|4563|351|1987|1|6|15|2|1987|351|4563|Monday|1987Q2|N|N|N|2446948|2447098|2446597|2446872|N|N|N|N|N| +2446963|AAAAAAAADHGFFCAA|1987-06-16|1049|4564|351|1987|2|6|16|2|1987|351|4564|Tuesday|1987Q2|N|N|N|2446948|2447098|2446598|2446873|N|N|N|N|N| +2446964|AAAAAAAAEHGFFCAA|1987-06-17|1049|4564|351|1987|3|6|17|2|1987|351|4564|Wednesday|1987Q2|N|N|N|2446948|2447098|2446599|2446874|N|N|N|N|N| +2446965|AAAAAAAAFHGFFCAA|1987-06-18|1049|4564|351|1987|4|6|18|2|1987|351|4564|Thursday|1987Q2|N|N|N|2446948|2447098|2446600|2446875|N|N|N|N|N| +2446966|AAAAAAAAGHGFFCAA|1987-06-19|1049|4564|351|1987|5|6|19|2|1987|351|4564|Friday|1987Q2|N|Y|N|2446948|2447098|2446601|2446876|N|N|N|N|N| +2446967|AAAAAAAAHHGFFCAA|1987-06-20|1049|4564|351|1987|6|6|20|2|1987|351|4564|Saturday|1987Q2|N|Y|N|2446948|2447098|2446602|2446877|N|N|N|N|N| +2446968|AAAAAAAAIHGFFCAA|1987-06-21|1049|4564|351|1987|0|6|21|2|1987|351|4564|Sunday|1987Q2|N|N|N|2446948|2447098|2446603|2446878|N|N|N|N|N| +2446969|AAAAAAAAJHGFFCAA|1987-06-22|1049|4564|351|1987|1|6|22|2|1987|351|4564|Monday|1987Q2|N|N|N|2446948|2447098|2446604|2446879|N|N|N|N|N| +2446970|AAAAAAAAKHGFFCAA|1987-06-23|1049|4565|351|1987|2|6|23|2|1987|351|4565|Tuesday|1987Q2|N|N|N|2446948|2447098|2446605|2446880|N|N|N|N|N| +2446971|AAAAAAAALHGFFCAA|1987-06-24|1049|4565|351|1987|3|6|24|2|1987|351|4565|Wednesday|1987Q2|N|N|N|2446948|2447098|2446606|2446881|N|N|N|N|N| +2446972|AAAAAAAAMHGFFCAA|1987-06-25|1049|4565|351|1987|4|6|25|2|1987|351|4565|Thursday|1987Q2|N|N|N|2446948|2447098|2446607|2446882|N|N|N|N|N| +2446973|AAAAAAAANHGFFCAA|1987-06-26|1049|4565|351|1987|5|6|26|2|1987|351|4565|Friday|1987Q2|N|Y|N|2446948|2447098|2446608|2446883|N|N|N|N|N| +2446974|AAAAAAAAOHGFFCAA|1987-06-27|1049|4565|351|1987|6|6|27|2|1987|351|4565|Saturday|1987Q2|N|Y|N|2446948|2447098|2446609|2446884|N|N|N|N|N| +2446975|AAAAAAAAPHGFFCAA|1987-06-28|1049|4565|351|1987|0|6|28|2|1987|351|4565|Sunday|1987Q2|N|N|N|2446948|2447098|2446610|2446885|N|N|N|N|N| +2446976|AAAAAAAAAIGFFCAA|1987-06-29|1049|4565|351|1987|1|6|29|2|1987|351|4565|Monday|1987Q2|N|N|N|2446948|2447098|2446611|2446886|N|N|N|N|N| +2446977|AAAAAAAABIGFFCAA|1987-06-30|1049|4566|351|1987|2|6|30|2|1987|351|4566|Tuesday|1987Q2|N|N|N|2446948|2447098|2446612|2446887|N|N|N|N|N| +2446978|AAAAAAAACIGFFCAA|1987-07-01|1050|4566|351|1987|3|7|1|2|1987|351|4566|Wednesday|1987Q2|N|N|N|2446978|2447158|2446613|2446887|N|N|N|N|N| +2446979|AAAAAAAADIGFFCAA|1987-07-02|1050|4566|351|1987|4|7|2|3|1987|351|4566|Thursday|1987Q3|N|N|N|2446978|2447158|2446614|2446888|N|N|N|N|N| +2446980|AAAAAAAAEIGFFCAA|1987-07-03|1050|4566|351|1987|5|7|3|3|1987|351|4566|Friday|1987Q3|N|Y|N|2446978|2447158|2446615|2446889|N|N|N|N|N| +2446981|AAAAAAAAFIGFFCAA|1987-07-04|1050|4566|351|1987|6|7|4|3|1987|351|4566|Saturday|1987Q3|N|Y|N|2446978|2447158|2446616|2446890|N|N|N|N|N| +2446982|AAAAAAAAGIGFFCAA|1987-07-05|1050|4566|351|1987|0|7|5|3|1987|351|4566|Sunday|1987Q3|Y|N|N|2446978|2447158|2446617|2446891|N|N|N|N|N| +2446983|AAAAAAAAHIGFFCAA|1987-07-06|1050|4566|351|1987|1|7|6|3|1987|351|4566|Monday|1987Q3|N|N|Y|2446978|2447158|2446618|2446892|N|N|N|N|N| +2446984|AAAAAAAAIIGFFCAA|1987-07-07|1050|4567|351|1987|2|7|7|3|1987|351|4567|Tuesday|1987Q3|N|N|N|2446978|2447158|2446619|2446893|N|N|N|N|N| +2446985|AAAAAAAAJIGFFCAA|1987-07-08|1050|4567|351|1987|3|7|8|3|1987|351|4567|Wednesday|1987Q3|N|N|N|2446978|2447158|2446620|2446894|N|N|N|N|N| +2446986|AAAAAAAAKIGFFCAA|1987-07-09|1050|4567|351|1987|4|7|9|3|1987|351|4567|Thursday|1987Q3|N|N|N|2446978|2447158|2446621|2446895|N|N|N|N|N| +2446987|AAAAAAAALIGFFCAA|1987-07-10|1050|4567|351|1987|5|7|10|3|1987|351|4567|Friday|1987Q3|N|Y|N|2446978|2447158|2446622|2446896|N|N|N|N|N| +2446988|AAAAAAAAMIGFFCAA|1987-07-11|1050|4567|351|1987|6|7|11|3|1987|351|4567|Saturday|1987Q3|N|Y|N|2446978|2447158|2446623|2446897|N|N|N|N|N| +2446989|AAAAAAAANIGFFCAA|1987-07-12|1050|4567|351|1987|0|7|12|3|1987|351|4567|Sunday|1987Q3|N|N|N|2446978|2447158|2446624|2446898|N|N|N|N|N| +2446990|AAAAAAAAOIGFFCAA|1987-07-13|1050|4567|351|1987|1|7|13|3|1987|351|4567|Monday|1987Q3|N|N|N|2446978|2447158|2446625|2446899|N|N|N|N|N| +2446991|AAAAAAAAPIGFFCAA|1987-07-14|1050|4568|351|1987|2|7|14|3|1987|351|4568|Tuesday|1987Q3|N|N|N|2446978|2447158|2446626|2446900|N|N|N|N|N| +2446992|AAAAAAAAAJGFFCAA|1987-07-15|1050|4568|351|1987|3|7|15|3|1987|351|4568|Wednesday|1987Q3|N|N|N|2446978|2447158|2446627|2446901|N|N|N|N|N| +2446993|AAAAAAAABJGFFCAA|1987-07-16|1050|4568|351|1987|4|7|16|3|1987|351|4568|Thursday|1987Q3|N|N|N|2446978|2447158|2446628|2446902|N|N|N|N|N| +2446994|AAAAAAAACJGFFCAA|1987-07-17|1050|4568|351|1987|5|7|17|3|1987|351|4568|Friday|1987Q3|N|Y|N|2446978|2447158|2446629|2446903|N|N|N|N|N| +2446995|AAAAAAAADJGFFCAA|1987-07-18|1050|4568|351|1987|6|7|18|3|1987|351|4568|Saturday|1987Q3|N|Y|N|2446978|2447158|2446630|2446904|N|N|N|N|N| +2446996|AAAAAAAAEJGFFCAA|1987-07-19|1050|4568|351|1987|0|7|19|3|1987|351|4568|Sunday|1987Q3|N|N|N|2446978|2447158|2446631|2446905|N|N|N|N|N| +2446997|AAAAAAAAFJGFFCAA|1987-07-20|1050|4568|351|1987|1|7|20|3|1987|351|4568|Monday|1987Q3|N|N|N|2446978|2447158|2446632|2446906|N|N|N|N|N| +2446998|AAAAAAAAGJGFFCAA|1987-07-21|1050|4569|351|1987|2|7|21|3|1987|351|4569|Tuesday|1987Q3|N|N|N|2446978|2447158|2446633|2446907|N|N|N|N|N| +2446999|AAAAAAAAHJGFFCAA|1987-07-22|1050|4569|351|1987|3|7|22|3|1987|351|4569|Wednesday|1987Q3|N|N|N|2446978|2447158|2446634|2446908|N|N|N|N|N| +2447000|AAAAAAAAIJGFFCAA|1987-07-23|1050|4569|351|1987|4|7|23|3|1987|351|4569|Thursday|1987Q3|N|N|N|2446978|2447158|2446635|2446909|N|N|N|N|N| +2447001|AAAAAAAAJJGFFCAA|1987-07-24|1050|4569|351|1987|5|7|24|3|1987|351|4569|Friday|1987Q3|N|Y|N|2446978|2447158|2446636|2446910|N|N|N|N|N| +2447002|AAAAAAAAKJGFFCAA|1987-07-25|1050|4569|351|1987|6|7|25|3|1987|351|4569|Saturday|1987Q3|N|Y|N|2446978|2447158|2446637|2446911|N|N|N|N|N| +2447003|AAAAAAAALJGFFCAA|1987-07-26|1050|4569|351|1987|0|7|26|3|1987|351|4569|Sunday|1987Q3|N|N|N|2446978|2447158|2446638|2446912|N|N|N|N|N| +2447004|AAAAAAAAMJGFFCAA|1987-07-27|1050|4569|351|1987|1|7|27|3|1987|351|4569|Monday|1987Q3|N|N|N|2446978|2447158|2446639|2446913|N|N|N|N|N| +2447005|AAAAAAAANJGFFCAA|1987-07-28|1050|4570|351|1987|2|7|28|3|1987|351|4570|Tuesday|1987Q3|N|N|N|2446978|2447158|2446640|2446914|N|N|N|N|N| +2447006|AAAAAAAAOJGFFCAA|1987-07-29|1050|4570|351|1987|3|7|29|3|1987|351|4570|Wednesday|1987Q3|N|N|N|2446978|2447158|2446641|2446915|N|N|N|N|N| +2447007|AAAAAAAAPJGFFCAA|1987-07-30|1050|4570|351|1987|4|7|30|3|1987|351|4570|Thursday|1987Q3|N|N|N|2446978|2447158|2446642|2446916|N|N|N|N|N| +2447008|AAAAAAAAAKGFFCAA|1987-07-31|1050|4570|351|1987|5|7|31|3|1987|351|4570|Friday|1987Q3|N|Y|N|2446978|2447158|2446643|2446917|N|N|N|N|N| +2447009|AAAAAAAABKGFFCAA|1987-08-01|1051|4570|351|1987|6|8|1|3|1987|351|4570|Saturday|1987Q3|N|Y|N|2447009|2447220|2446644|2446918|N|N|N|N|N| +2447010|AAAAAAAACKGFFCAA|1987-08-02|1051|4570|351|1987|0|8|2|3|1987|351|4570|Sunday|1987Q3|N|N|N|2447009|2447220|2446645|2446919|N|N|N|N|N| +2447011|AAAAAAAADKGFFCAA|1987-08-03|1051|4570|351|1987|1|8|3|3|1987|351|4570|Monday|1987Q3|N|N|N|2447009|2447220|2446646|2446920|N|N|N|N|N| +2447012|AAAAAAAAEKGFFCAA|1987-08-04|1051|4571|351|1987|2|8|4|3|1987|351|4571|Tuesday|1987Q3|N|N|N|2447009|2447220|2446647|2446921|N|N|N|N|N| +2447013|AAAAAAAAFKGFFCAA|1987-08-05|1051|4571|351|1987|3|8|5|3|1987|351|4571|Wednesday|1987Q3|N|N|N|2447009|2447220|2446648|2446922|N|N|N|N|N| +2447014|AAAAAAAAGKGFFCAA|1987-08-06|1051|4571|351|1987|4|8|6|3|1987|351|4571|Thursday|1987Q3|N|N|N|2447009|2447220|2446649|2446923|N|N|N|N|N| +2447015|AAAAAAAAHKGFFCAA|1987-08-07|1051|4571|351|1987|5|8|7|3|1987|351|4571|Friday|1987Q3|N|Y|N|2447009|2447220|2446650|2446924|N|N|N|N|N| +2447016|AAAAAAAAIKGFFCAA|1987-08-08|1051|4571|351|1987|6|8|8|3|1987|351|4571|Saturday|1987Q3|N|Y|N|2447009|2447220|2446651|2446925|N|N|N|N|N| +2447017|AAAAAAAAJKGFFCAA|1987-08-09|1051|4571|351|1987|0|8|9|3|1987|351|4571|Sunday|1987Q3|N|N|N|2447009|2447220|2446652|2446926|N|N|N|N|N| +2447018|AAAAAAAAKKGFFCAA|1987-08-10|1051|4571|351|1987|1|8|10|3|1987|351|4571|Monday|1987Q3|N|N|N|2447009|2447220|2446653|2446927|N|N|N|N|N| +2447019|AAAAAAAALKGFFCAA|1987-08-11|1051|4572|351|1987|2|8|11|3|1987|351|4572|Tuesday|1987Q3|N|N|N|2447009|2447220|2446654|2446928|N|N|N|N|N| +2447020|AAAAAAAAMKGFFCAA|1987-08-12|1051|4572|351|1987|3|8|12|3|1987|351|4572|Wednesday|1987Q3|N|N|N|2447009|2447220|2446655|2446929|N|N|N|N|N| +2447021|AAAAAAAANKGFFCAA|1987-08-13|1051|4572|351|1987|4|8|13|3|1987|351|4572|Thursday|1987Q3|N|N|N|2447009|2447220|2446656|2446930|N|N|N|N|N| +2447022|AAAAAAAAOKGFFCAA|1987-08-14|1051|4572|351|1987|5|8|14|3|1987|351|4572|Friday|1987Q3|N|Y|N|2447009|2447220|2446657|2446931|N|N|N|N|N| +2447023|AAAAAAAAPKGFFCAA|1987-08-15|1051|4572|351|1987|6|8|15|3|1987|351|4572|Saturday|1987Q3|N|Y|N|2447009|2447220|2446658|2446932|N|N|N|N|N| +2447024|AAAAAAAAALGFFCAA|1987-08-16|1051|4572|351|1987|0|8|16|3|1987|351|4572|Sunday|1987Q3|N|N|N|2447009|2447220|2446659|2446933|N|N|N|N|N| +2447025|AAAAAAAABLGFFCAA|1987-08-17|1051|4572|351|1987|1|8|17|3|1987|351|4572|Monday|1987Q3|N|N|N|2447009|2447220|2446660|2446934|N|N|N|N|N| +2447026|AAAAAAAACLGFFCAA|1987-08-18|1051|4573|351|1987|2|8|18|3|1987|351|4573|Tuesday|1987Q3|N|N|N|2447009|2447220|2446661|2446935|N|N|N|N|N| +2447027|AAAAAAAADLGFFCAA|1987-08-19|1051|4573|351|1987|3|8|19|3|1987|351|4573|Wednesday|1987Q3|N|N|N|2447009|2447220|2446662|2446936|N|N|N|N|N| +2447028|AAAAAAAAELGFFCAA|1987-08-20|1051|4573|351|1987|4|8|20|3|1987|351|4573|Thursday|1987Q3|N|N|N|2447009|2447220|2446663|2446937|N|N|N|N|N| +2447029|AAAAAAAAFLGFFCAA|1987-08-21|1051|4573|351|1987|5|8|21|3|1987|351|4573|Friday|1987Q3|N|Y|N|2447009|2447220|2446664|2446938|N|N|N|N|N| +2447030|AAAAAAAAGLGFFCAA|1987-08-22|1051|4573|351|1987|6|8|22|3|1987|351|4573|Saturday|1987Q3|N|Y|N|2447009|2447220|2446665|2446939|N|N|N|N|N| +2447031|AAAAAAAAHLGFFCAA|1987-08-23|1051|4573|351|1987|0|8|23|3|1987|351|4573|Sunday|1987Q3|N|N|N|2447009|2447220|2446666|2446940|N|N|N|N|N| +2447032|AAAAAAAAILGFFCAA|1987-08-24|1051|4573|351|1987|1|8|24|3|1987|351|4573|Monday|1987Q3|N|N|N|2447009|2447220|2446667|2446941|N|N|N|N|N| +2447033|AAAAAAAAJLGFFCAA|1987-08-25|1051|4574|351|1987|2|8|25|3|1987|351|4574|Tuesday|1987Q3|N|N|N|2447009|2447220|2446668|2446942|N|N|N|N|N| +2447034|AAAAAAAAKLGFFCAA|1987-08-26|1051|4574|351|1987|3|8|26|3|1987|351|4574|Wednesday|1987Q3|N|N|N|2447009|2447220|2446669|2446943|N|N|N|N|N| +2447035|AAAAAAAALLGFFCAA|1987-08-27|1051|4574|351|1987|4|8|27|3|1987|351|4574|Thursday|1987Q3|N|N|N|2447009|2447220|2446670|2446944|N|N|N|N|N| +2447036|AAAAAAAAMLGFFCAA|1987-08-28|1051|4574|351|1987|5|8|28|3|1987|351|4574|Friday|1987Q3|N|Y|N|2447009|2447220|2446671|2446945|N|N|N|N|N| +2447037|AAAAAAAANLGFFCAA|1987-08-29|1051|4574|351|1987|6|8|29|3|1987|351|4574|Saturday|1987Q3|N|Y|N|2447009|2447220|2446672|2446946|N|N|N|N|N| +2447038|AAAAAAAAOLGFFCAA|1987-08-30|1051|4574|351|1987|0|8|30|3|1987|351|4574|Sunday|1987Q3|N|N|N|2447009|2447220|2446673|2446947|N|N|N|N|N| +2447039|AAAAAAAAPLGFFCAA|1987-08-31|1051|4574|351|1987|1|8|31|3|1987|351|4574|Monday|1987Q3|N|N|N|2447009|2447220|2446674|2446948|N|N|N|N|N| +2447040|AAAAAAAAAMGFFCAA|1987-09-01|1052|4575|352|1987|2|9|1|3|1987|352|4575|Tuesday|1987Q3|N|N|N|2447040|2447282|2446675|2446949|N|N|N|N|N| +2447041|AAAAAAAABMGFFCAA|1987-09-02|1052|4575|352|1987|3|9|2|3|1987|352|4575|Wednesday|1987Q3|N|N|N|2447040|2447282|2446676|2446950|N|N|N|N|N| +2447042|AAAAAAAACMGFFCAA|1987-09-03|1052|4575|352|1987|4|9|3|3|1987|352|4575|Thursday|1987Q3|N|N|N|2447040|2447282|2446677|2446951|N|N|N|N|N| +2447043|AAAAAAAADMGFFCAA|1987-09-04|1052|4575|352|1987|5|9|4|3|1987|352|4575|Friday|1987Q3|N|Y|N|2447040|2447282|2446678|2446952|N|N|N|N|N| +2447044|AAAAAAAAEMGFFCAA|1987-09-05|1052|4575|352|1987|6|9|5|3|1987|352|4575|Saturday|1987Q3|N|Y|N|2447040|2447282|2446679|2446953|N|N|N|N|N| +2447045|AAAAAAAAFMGFFCAA|1987-09-06|1052|4575|352|1987|0|9|6|3|1987|352|4575|Sunday|1987Q3|N|N|N|2447040|2447282|2446680|2446954|N|N|N|N|N| +2447046|AAAAAAAAGMGFFCAA|1987-09-07|1052|4575|352|1987|1|9|7|3|1987|352|4575|Monday|1987Q3|N|N|N|2447040|2447282|2446681|2446955|N|N|N|N|N| +2447047|AAAAAAAAHMGFFCAA|1987-09-08|1052|4576|352|1987|2|9|8|3|1987|352|4576|Tuesday|1987Q3|N|N|N|2447040|2447282|2446682|2446956|N|N|N|N|N| +2447048|AAAAAAAAIMGFFCAA|1987-09-09|1052|4576|352|1987|3|9|9|3|1987|352|4576|Wednesday|1987Q3|N|N|N|2447040|2447282|2446683|2446957|N|N|N|N|N| +2447049|AAAAAAAAJMGFFCAA|1987-09-10|1052|4576|352|1987|4|9|10|3|1987|352|4576|Thursday|1987Q3|N|N|N|2447040|2447282|2446684|2446958|N|N|N|N|N| +2447050|AAAAAAAAKMGFFCAA|1987-09-11|1052|4576|352|1987|5|9|11|3|1987|352|4576|Friday|1987Q3|N|Y|N|2447040|2447282|2446685|2446959|N|N|N|N|N| +2447051|AAAAAAAALMGFFCAA|1987-09-12|1052|4576|352|1987|6|9|12|3|1987|352|4576|Saturday|1987Q3|N|Y|N|2447040|2447282|2446686|2446960|N|N|N|N|N| +2447052|AAAAAAAAMMGFFCAA|1987-09-13|1052|4576|352|1987|0|9|13|3|1987|352|4576|Sunday|1987Q3|N|N|N|2447040|2447282|2446687|2446961|N|N|N|N|N| +2447053|AAAAAAAANMGFFCAA|1987-09-14|1052|4576|352|1987|1|9|14|3|1987|352|4576|Monday|1987Q3|N|N|N|2447040|2447282|2446688|2446962|N|N|N|N|N| +2447054|AAAAAAAAOMGFFCAA|1987-09-15|1052|4577|352|1987|2|9|15|3|1987|352|4577|Tuesday|1987Q3|N|N|N|2447040|2447282|2446689|2446963|N|N|N|N|N| +2447055|AAAAAAAAPMGFFCAA|1987-09-16|1052|4577|352|1987|3|9|16|3|1987|352|4577|Wednesday|1987Q3|N|N|N|2447040|2447282|2446690|2446964|N|N|N|N|N| +2447056|AAAAAAAAANGFFCAA|1987-09-17|1052|4577|352|1987|4|9|17|3|1987|352|4577|Thursday|1987Q3|N|N|N|2447040|2447282|2446691|2446965|N|N|N|N|N| +2447057|AAAAAAAABNGFFCAA|1987-09-18|1052|4577|352|1987|5|9|18|3|1987|352|4577|Friday|1987Q3|N|Y|N|2447040|2447282|2446692|2446966|N|N|N|N|N| +2447058|AAAAAAAACNGFFCAA|1987-09-19|1052|4577|352|1987|6|9|19|3|1987|352|4577|Saturday|1987Q3|N|Y|N|2447040|2447282|2446693|2446967|N|N|N|N|N| +2447059|AAAAAAAADNGFFCAA|1987-09-20|1052|4577|352|1987|0|9|20|3|1987|352|4577|Sunday|1987Q3|N|N|N|2447040|2447282|2446694|2446968|N|N|N|N|N| +2447060|AAAAAAAAENGFFCAA|1987-09-21|1052|4577|352|1987|1|9|21|3|1987|352|4577|Monday|1987Q3|N|N|N|2447040|2447282|2446695|2446969|N|N|N|N|N| +2447061|AAAAAAAAFNGFFCAA|1987-09-22|1052|4578|352|1987|2|9|22|3|1987|352|4578|Tuesday|1987Q3|N|N|N|2447040|2447282|2446696|2446970|N|N|N|N|N| +2447062|AAAAAAAAGNGFFCAA|1987-09-23|1052|4578|352|1987|3|9|23|3|1987|352|4578|Wednesday|1987Q3|N|N|N|2447040|2447282|2446697|2446971|N|N|N|N|N| +2447063|AAAAAAAAHNGFFCAA|1987-09-24|1052|4578|352|1987|4|9|24|3|1987|352|4578|Thursday|1987Q3|N|N|N|2447040|2447282|2446698|2446972|N|N|N|N|N| +2447064|AAAAAAAAINGFFCAA|1987-09-25|1052|4578|352|1987|5|9|25|3|1987|352|4578|Friday|1987Q3|N|Y|N|2447040|2447282|2446699|2446973|N|N|N|N|N| +2447065|AAAAAAAAJNGFFCAA|1987-09-26|1052|4578|352|1987|6|9|26|3|1987|352|4578|Saturday|1987Q3|N|Y|N|2447040|2447282|2446700|2446974|N|N|N|N|N| +2447066|AAAAAAAAKNGFFCAA|1987-09-27|1052|4578|352|1987|0|9|27|3|1987|352|4578|Sunday|1987Q3|N|N|N|2447040|2447282|2446701|2446975|N|N|N|N|N| +2447067|AAAAAAAALNGFFCAA|1987-09-28|1052|4578|352|1987|1|9|28|3|1987|352|4578|Monday|1987Q3|N|N|N|2447040|2447282|2446702|2446976|N|N|N|N|N| +2447068|AAAAAAAAMNGFFCAA|1987-09-29|1052|4579|352|1987|2|9|29|3|1987|352|4579|Tuesday|1987Q3|N|N|N|2447040|2447282|2446703|2446977|N|N|N|N|N| +2447069|AAAAAAAANNGFFCAA|1987-09-30|1052|4579|352|1987|3|9|30|3|1987|352|4579|Wednesday|1987Q3|N|N|N|2447040|2447282|2446704|2446978|N|N|N|N|N| +2447070|AAAAAAAAONGFFCAA|1987-10-01|1053|4579|352|1987|4|10|1|3|1987|352|4579|Thursday|1987Q3|N|N|N|2447070|2447342|2446705|2446978|N|N|N|N|N| +2447071|AAAAAAAAPNGFFCAA|1987-10-02|1053|4579|352|1987|5|10|2|4|1987|352|4579|Friday|1987Q4|N|Y|N|2447070|2447342|2446706|2446979|N|N|N|N|N| +2447072|AAAAAAAAAOGFFCAA|1987-10-03|1053|4579|352|1987|6|10|3|4|1987|352|4579|Saturday|1987Q4|N|Y|N|2447070|2447342|2446707|2446980|N|N|N|N|N| +2447073|AAAAAAAABOGFFCAA|1987-10-04|1053|4579|352|1987|0|10|4|4|1987|352|4579|Sunday|1987Q4|N|N|N|2447070|2447342|2446708|2446981|N|N|N|N|N| +2447074|AAAAAAAACOGFFCAA|1987-10-05|1053|4579|352|1987|1|10|5|4|1987|352|4579|Monday|1987Q4|N|N|N|2447070|2447342|2446709|2446982|N|N|N|N|N| +2447075|AAAAAAAADOGFFCAA|1987-10-06|1053|4580|352|1987|2|10|6|4|1987|352|4580|Tuesday|1987Q4|N|N|N|2447070|2447342|2446710|2446983|N|N|N|N|N| +2447076|AAAAAAAAEOGFFCAA|1987-10-07|1053|4580|352|1987|3|10|7|4|1987|352|4580|Wednesday|1987Q4|N|N|N|2447070|2447342|2446711|2446984|N|N|N|N|N| +2447077|AAAAAAAAFOGFFCAA|1987-10-08|1053|4580|352|1987|4|10|8|4|1987|352|4580|Thursday|1987Q4|N|N|N|2447070|2447342|2446712|2446985|N|N|N|N|N| +2447078|AAAAAAAAGOGFFCAA|1987-10-09|1053|4580|352|1987|5|10|9|4|1987|352|4580|Friday|1987Q4|N|Y|N|2447070|2447342|2446713|2446986|N|N|N|N|N| +2447079|AAAAAAAAHOGFFCAA|1987-10-10|1053|4580|352|1987|6|10|10|4|1987|352|4580|Saturday|1987Q4|N|Y|N|2447070|2447342|2446714|2446987|N|N|N|N|N| +2447080|AAAAAAAAIOGFFCAA|1987-10-11|1053|4580|352|1987|0|10|11|4|1987|352|4580|Sunday|1987Q4|N|N|N|2447070|2447342|2446715|2446988|N|N|N|N|N| +2447081|AAAAAAAAJOGFFCAA|1987-10-12|1053|4580|352|1987|1|10|12|4|1987|352|4580|Monday|1987Q4|N|N|N|2447070|2447342|2446716|2446989|N|N|N|N|N| +2447082|AAAAAAAAKOGFFCAA|1987-10-13|1053|4581|352|1987|2|10|13|4|1987|352|4581|Tuesday|1987Q4|N|N|N|2447070|2447342|2446717|2446990|N|N|N|N|N| +2447083|AAAAAAAALOGFFCAA|1987-10-14|1053|4581|352|1987|3|10|14|4|1987|352|4581|Wednesday|1987Q4|N|N|N|2447070|2447342|2446718|2446991|N|N|N|N|N| +2447084|AAAAAAAAMOGFFCAA|1987-10-15|1053|4581|352|1987|4|10|15|4|1987|352|4581|Thursday|1987Q4|N|N|N|2447070|2447342|2446719|2446992|N|N|N|N|N| +2447085|AAAAAAAANOGFFCAA|1987-10-16|1053|4581|352|1987|5|10|16|4|1987|352|4581|Friday|1987Q4|N|Y|N|2447070|2447342|2446720|2446993|N|N|N|N|N| +2447086|AAAAAAAAOOGFFCAA|1987-10-17|1053|4581|352|1987|6|10|17|4|1987|352|4581|Saturday|1987Q4|N|Y|N|2447070|2447342|2446721|2446994|N|N|N|N|N| +2447087|AAAAAAAAPOGFFCAA|1987-10-18|1053|4581|352|1987|0|10|18|4|1987|352|4581|Sunday|1987Q4|N|N|N|2447070|2447342|2446722|2446995|N|N|N|N|N| +2447088|AAAAAAAAAPGFFCAA|1987-10-19|1053|4581|352|1987|1|10|19|4|1987|352|4581|Monday|1987Q4|N|N|N|2447070|2447342|2446723|2446996|N|N|N|N|N| +2447089|AAAAAAAABPGFFCAA|1987-10-20|1053|4582|352|1987|2|10|20|4|1987|352|4582|Tuesday|1987Q4|N|N|N|2447070|2447342|2446724|2446997|N|N|N|N|N| +2447090|AAAAAAAACPGFFCAA|1987-10-21|1053|4582|352|1987|3|10|21|4|1987|352|4582|Wednesday|1987Q4|N|N|N|2447070|2447342|2446725|2446998|N|N|N|N|N| +2447091|AAAAAAAADPGFFCAA|1987-10-22|1053|4582|352|1987|4|10|22|4|1987|352|4582|Thursday|1987Q4|N|N|N|2447070|2447342|2446726|2446999|N|N|N|N|N| +2447092|AAAAAAAAEPGFFCAA|1987-10-23|1053|4582|352|1987|5|10|23|4|1987|352|4582|Friday|1987Q4|N|Y|N|2447070|2447342|2446727|2447000|N|N|N|N|N| +2447093|AAAAAAAAFPGFFCAA|1987-10-24|1053|4582|352|1987|6|10|24|4|1987|352|4582|Saturday|1987Q4|N|Y|N|2447070|2447342|2446728|2447001|N|N|N|N|N| +2447094|AAAAAAAAGPGFFCAA|1987-10-25|1053|4582|352|1987|0|10|25|4|1987|352|4582|Sunday|1987Q4|N|N|N|2447070|2447342|2446729|2447002|N|N|N|N|N| +2447095|AAAAAAAAHPGFFCAA|1987-10-26|1053|4582|352|1987|1|10|26|4|1987|352|4582|Monday|1987Q4|N|N|N|2447070|2447342|2446730|2447003|N|N|N|N|N| +2447096|AAAAAAAAIPGFFCAA|1987-10-27|1053|4583|352|1987|2|10|27|4|1987|352|4583|Tuesday|1987Q4|N|N|N|2447070|2447342|2446731|2447004|N|N|N|N|N| +2447097|AAAAAAAAJPGFFCAA|1987-10-28|1053|4583|352|1987|3|10|28|4|1987|352|4583|Wednesday|1987Q4|N|N|N|2447070|2447342|2446732|2447005|N|N|N|N|N| +2447098|AAAAAAAAKPGFFCAA|1987-10-29|1053|4583|352|1987|4|10|29|4|1987|352|4583|Thursday|1987Q4|N|N|N|2447070|2447342|2446733|2447006|N|N|N|N|N| +2447099|AAAAAAAALPGFFCAA|1987-10-30|1053|4583|352|1987|5|10|30|4|1987|352|4583|Friday|1987Q4|N|Y|N|2447070|2447342|2446734|2447007|N|N|N|N|N| +2447100|AAAAAAAAMPGFFCAA|1987-10-31|1053|4583|352|1987|6|10|31|4|1987|352|4583|Saturday|1987Q4|N|Y|N|2447070|2447342|2446735|2447008|N|N|N|N|N| +2447101|AAAAAAAANPGFFCAA|1987-11-01|1054|4583|352|1987|0|11|1|4|1987|352|4583|Sunday|1987Q4|N|N|N|2447101|2447404|2446736|2447009|N|N|N|N|N| +2447102|AAAAAAAAOPGFFCAA|1987-11-02|1054|4583|352|1987|1|11|2|4|1987|352|4583|Monday|1987Q4|N|N|N|2447101|2447404|2446737|2447010|N|N|N|N|N| +2447103|AAAAAAAAPPGFFCAA|1987-11-03|1054|4584|352|1987|2|11|3|4|1987|352|4584|Tuesday|1987Q4|N|N|N|2447101|2447404|2446738|2447011|N|N|N|N|N| +2447104|AAAAAAAAAAHFFCAA|1987-11-04|1054|4584|352|1987|3|11|4|4|1987|352|4584|Wednesday|1987Q4|N|N|N|2447101|2447404|2446739|2447012|N|N|N|N|N| +2447105|AAAAAAAABAHFFCAA|1987-11-05|1054|4584|352|1987|4|11|5|4|1987|352|4584|Thursday|1987Q4|N|N|N|2447101|2447404|2446740|2447013|N|N|N|N|N| +2447106|AAAAAAAACAHFFCAA|1987-11-06|1054|4584|352|1987|5|11|6|4|1987|352|4584|Friday|1987Q4|N|Y|N|2447101|2447404|2446741|2447014|N|N|N|N|N| +2447107|AAAAAAAADAHFFCAA|1987-11-07|1054|4584|352|1987|6|11|7|4|1987|352|4584|Saturday|1987Q4|N|Y|N|2447101|2447404|2446742|2447015|N|N|N|N|N| +2447108|AAAAAAAAEAHFFCAA|1987-11-08|1054|4584|352|1987|0|11|8|4|1987|352|4584|Sunday|1987Q4|N|N|N|2447101|2447404|2446743|2447016|N|N|N|N|N| +2447109|AAAAAAAAFAHFFCAA|1987-11-09|1054|4584|352|1987|1|11|9|4|1987|352|4584|Monday|1987Q4|N|N|N|2447101|2447404|2446744|2447017|N|N|N|N|N| +2447110|AAAAAAAAGAHFFCAA|1987-11-10|1054|4585|352|1987|2|11|10|4|1987|352|4585|Tuesday|1987Q4|N|N|N|2447101|2447404|2446745|2447018|N|N|N|N|N| +2447111|AAAAAAAAHAHFFCAA|1987-11-11|1054|4585|352|1987|3|11|11|4|1987|352|4585|Wednesday|1987Q4|N|N|N|2447101|2447404|2446746|2447019|N|N|N|N|N| +2447112|AAAAAAAAIAHFFCAA|1987-11-12|1054|4585|352|1987|4|11|12|4|1987|352|4585|Thursday|1987Q4|N|N|N|2447101|2447404|2446747|2447020|N|N|N|N|N| +2447113|AAAAAAAAJAHFFCAA|1987-11-13|1054|4585|352|1987|5|11|13|4|1987|352|4585|Friday|1987Q4|N|Y|N|2447101|2447404|2446748|2447021|N|N|N|N|N| +2447114|AAAAAAAAKAHFFCAA|1987-11-14|1054|4585|352|1987|6|11|14|4|1987|352|4585|Saturday|1987Q4|N|Y|N|2447101|2447404|2446749|2447022|N|N|N|N|N| +2447115|AAAAAAAALAHFFCAA|1987-11-15|1054|4585|352|1987|0|11|15|4|1987|352|4585|Sunday|1987Q4|N|N|N|2447101|2447404|2446750|2447023|N|N|N|N|N| +2447116|AAAAAAAAMAHFFCAA|1987-11-16|1054|4585|352|1987|1|11|16|4|1987|352|4585|Monday|1987Q4|N|N|N|2447101|2447404|2446751|2447024|N|N|N|N|N| +2447117|AAAAAAAANAHFFCAA|1987-11-17|1054|4586|352|1987|2|11|17|4|1987|352|4586|Tuesday|1987Q4|N|N|N|2447101|2447404|2446752|2447025|N|N|N|N|N| +2447118|AAAAAAAAOAHFFCAA|1987-11-18|1054|4586|352|1987|3|11|18|4|1987|352|4586|Wednesday|1987Q4|N|N|N|2447101|2447404|2446753|2447026|N|N|N|N|N| +2447119|AAAAAAAAPAHFFCAA|1987-11-19|1054|4586|352|1987|4|11|19|4|1987|352|4586|Thursday|1987Q4|N|N|N|2447101|2447404|2446754|2447027|N|N|N|N|N| +2447120|AAAAAAAAABHFFCAA|1987-11-20|1054|4586|352|1987|5|11|20|4|1987|352|4586|Friday|1987Q4|N|Y|N|2447101|2447404|2446755|2447028|N|N|N|N|N| +2447121|AAAAAAAABBHFFCAA|1987-11-21|1054|4586|352|1987|6|11|21|4|1987|352|4586|Saturday|1987Q4|N|Y|N|2447101|2447404|2446756|2447029|N|N|N|N|N| +2447122|AAAAAAAACBHFFCAA|1987-11-22|1054|4586|352|1987|0|11|22|4|1987|352|4586|Sunday|1987Q4|N|N|N|2447101|2447404|2446757|2447030|N|N|N|N|N| +2447123|AAAAAAAADBHFFCAA|1987-11-23|1054|4586|352|1987|1|11|23|4|1987|352|4586|Monday|1987Q4|N|N|N|2447101|2447404|2446758|2447031|N|N|N|N|N| +2447124|AAAAAAAAEBHFFCAA|1987-11-24|1054|4587|352|1987|2|11|24|4|1987|352|4587|Tuesday|1987Q4|N|N|N|2447101|2447404|2446759|2447032|N|N|N|N|N| +2447125|AAAAAAAAFBHFFCAA|1987-11-25|1054|4587|352|1987|3|11|25|4|1987|352|4587|Wednesday|1987Q4|N|N|N|2447101|2447404|2446760|2447033|N|N|N|N|N| +2447126|AAAAAAAAGBHFFCAA|1987-11-26|1054|4587|352|1987|4|11|26|4|1987|352|4587|Thursday|1987Q4|N|N|N|2447101|2447404|2446761|2447034|N|N|N|N|N| +2447127|AAAAAAAAHBHFFCAA|1987-11-27|1054|4587|352|1987|5|11|27|4|1987|352|4587|Friday|1987Q4|N|Y|N|2447101|2447404|2446762|2447035|N|N|N|N|N| +2447128|AAAAAAAAIBHFFCAA|1987-11-28|1054|4587|352|1987|6|11|28|4|1987|352|4587|Saturday|1987Q4|N|Y|N|2447101|2447404|2446763|2447036|N|N|N|N|N| +2447129|AAAAAAAAJBHFFCAA|1987-11-29|1054|4587|352|1987|0|11|29|4|1987|352|4587|Sunday|1987Q4|N|N|N|2447101|2447404|2446764|2447037|N|N|N|N|N| +2447130|AAAAAAAAKBHFFCAA|1987-11-30|1054|4587|352|1987|1|11|30|4|1987|352|4587|Monday|1987Q4|N|N|N|2447101|2447404|2446765|2447038|N|N|N|N|N| +2447131|AAAAAAAALBHFFCAA|1987-12-01|1055|4588|353|1987|2|12|1|4|1987|353|4588|Tuesday|1987Q4|N|N|N|2447131|2447464|2446766|2447039|N|N|N|N|N| +2447132|AAAAAAAAMBHFFCAA|1987-12-02|1055|4588|353|1987|3|12|2|4|1987|353|4588|Wednesday|1987Q4|N|N|N|2447131|2447464|2446767|2447040|N|N|N|N|N| +2447133|AAAAAAAANBHFFCAA|1987-12-03|1055|4588|353|1987|4|12|3|4|1987|353|4588|Thursday|1987Q4|N|N|N|2447131|2447464|2446768|2447041|N|N|N|N|N| +2447134|AAAAAAAAOBHFFCAA|1987-12-04|1055|4588|353|1987|5|12|4|4|1987|353|4588|Friday|1987Q4|N|Y|N|2447131|2447464|2446769|2447042|N|N|N|N|N| +2447135|AAAAAAAAPBHFFCAA|1987-12-05|1055|4588|353|1987|6|12|5|4|1987|353|4588|Saturday|1987Q4|N|Y|N|2447131|2447464|2446770|2447043|N|N|N|N|N| +2447136|AAAAAAAAACHFFCAA|1987-12-06|1055|4588|353|1987|0|12|6|4|1987|353|4588|Sunday|1987Q4|N|N|N|2447131|2447464|2446771|2447044|N|N|N|N|N| +2447137|AAAAAAAABCHFFCAA|1987-12-07|1055|4588|353|1987|1|12|7|4|1987|353|4588|Monday|1987Q4|N|N|N|2447131|2447464|2446772|2447045|N|N|N|N|N| +2447138|AAAAAAAACCHFFCAA|1987-12-08|1055|4589|353|1987|2|12|8|4|1987|353|4589|Tuesday|1987Q4|N|N|N|2447131|2447464|2446773|2447046|N|N|N|N|N| +2447139|AAAAAAAADCHFFCAA|1987-12-09|1055|4589|353|1987|3|12|9|4|1987|353|4589|Wednesday|1987Q4|N|N|N|2447131|2447464|2446774|2447047|N|N|N|N|N| +2447140|AAAAAAAAECHFFCAA|1987-12-10|1055|4589|353|1987|4|12|10|4|1987|353|4589|Thursday|1987Q4|N|N|N|2447131|2447464|2446775|2447048|N|N|N|N|N| +2447141|AAAAAAAAFCHFFCAA|1987-12-11|1055|4589|353|1987|5|12|11|4|1987|353|4589|Friday|1987Q4|N|Y|N|2447131|2447464|2446776|2447049|N|N|N|N|N| +2447142|AAAAAAAAGCHFFCAA|1987-12-12|1055|4589|353|1987|6|12|12|4|1987|353|4589|Saturday|1987Q4|N|Y|N|2447131|2447464|2446777|2447050|N|N|N|N|N| +2447143|AAAAAAAAHCHFFCAA|1987-12-13|1055|4589|353|1987|0|12|13|4|1987|353|4589|Sunday|1987Q4|N|N|N|2447131|2447464|2446778|2447051|N|N|N|N|N| +2447144|AAAAAAAAICHFFCAA|1987-12-14|1055|4589|353|1987|1|12|14|4|1987|353|4589|Monday|1987Q4|N|N|N|2447131|2447464|2446779|2447052|N|N|N|N|N| +2447145|AAAAAAAAJCHFFCAA|1987-12-15|1055|4590|353|1987|2|12|15|4|1987|353|4590|Tuesday|1987Q4|N|N|N|2447131|2447464|2446780|2447053|N|N|N|N|N| +2447146|AAAAAAAAKCHFFCAA|1987-12-16|1055|4590|353|1987|3|12|16|4|1987|353|4590|Wednesday|1987Q4|N|N|N|2447131|2447464|2446781|2447054|N|N|N|N|N| +2447147|AAAAAAAALCHFFCAA|1987-12-17|1055|4590|353|1987|4|12|17|4|1987|353|4590|Thursday|1987Q4|N|N|N|2447131|2447464|2446782|2447055|N|N|N|N|N| +2447148|AAAAAAAAMCHFFCAA|1987-12-18|1055|4590|353|1987|5|12|18|4|1987|353|4590|Friday|1987Q4|N|Y|N|2447131|2447464|2446783|2447056|N|N|N|N|N| +2447149|AAAAAAAANCHFFCAA|1987-12-19|1055|4590|353|1987|6|12|19|4|1987|353|4590|Saturday|1987Q4|N|Y|N|2447131|2447464|2446784|2447057|N|N|N|N|N| +2447150|AAAAAAAAOCHFFCAA|1987-12-20|1055|4590|353|1987|0|12|20|4|1987|353|4590|Sunday|1987Q4|N|N|N|2447131|2447464|2446785|2447058|N|N|N|N|N| +2447151|AAAAAAAAPCHFFCAA|1987-12-21|1055|4590|353|1987|1|12|21|4|1987|353|4590|Monday|1987Q4|N|N|N|2447131|2447464|2446786|2447059|N|N|N|N|N| +2447152|AAAAAAAAADHFFCAA|1987-12-22|1055|4591|353|1987|2|12|22|4|1987|353|4591|Tuesday|1987Q4|N|N|N|2447131|2447464|2446787|2447060|N|N|N|N|N| +2447153|AAAAAAAABDHFFCAA|1987-12-23|1055|4591|353|1987|3|12|23|4|1987|353|4591|Wednesday|1987Q4|N|N|N|2447131|2447464|2446788|2447061|N|N|N|N|N| +2447154|AAAAAAAACDHFFCAA|1987-12-24|1055|4591|353|1987|4|12|24|4|1987|353|4591|Thursday|1987Q4|N|N|N|2447131|2447464|2446789|2447062|N|N|N|N|N| +2447155|AAAAAAAADDHFFCAA|1987-12-25|1055|4591|353|1987|5|12|25|4|1987|353|4591|Friday|1987Q4|N|Y|N|2447131|2447464|2446790|2447063|N|N|N|N|N| +2447156|AAAAAAAAEDHFFCAA|1987-12-26|1055|4591|353|1987|6|12|26|4|1987|353|4591|Saturday|1987Q4|Y|Y|N|2447131|2447464|2446791|2447064|N|N|N|N|N| +2447157|AAAAAAAAFDHFFCAA|1987-12-27|1055|4591|353|1987|0|12|27|4|1987|353|4591|Sunday|1987Q4|N|N|Y|2447131|2447464|2446792|2447065|N|N|N|N|N| +2447158|AAAAAAAAGDHFFCAA|1987-12-28|1055|4591|353|1987|1|12|28|4|1987|353|4591|Monday|1987Q4|N|N|N|2447131|2447464|2446793|2447066|N|N|N|N|N| +2447159|AAAAAAAAHDHFFCAA|1987-12-29|1055|4592|353|1987|2|12|29|4|1987|353|4592|Tuesday|1987Q4|N|N|N|2447131|2447464|2446794|2447067|N|N|N|N|N| +2447160|AAAAAAAAIDHFFCAA|1987-12-30|1055|4592|353|1987|3|12|30|4|1987|353|4592|Wednesday|1987Q4|N|N|N|2447131|2447464|2446795|2447068|N|N|N|N|N| +2447161|AAAAAAAAJDHFFCAA|1987-12-31|1055|4592|353|1987|4|12|31|4|1987|353|4592|Thursday|1987Q4|N|N|N|2447131|2447464|2446796|2447069|N|N|N|N|N| +2447162|AAAAAAAAKDHFFCAA|1988-01-01|1056|4592|353|1988|5|1|1|1|1988|353|4592|Friday|1988Q1|Y|Y|N|2447162|2447161|2446797|2447070|N|N|N|N|N| +2447163|AAAAAAAALDHFFCAA|1988-01-02|1056|4592|353|1988|6|1|2|1|1988|353|4592|Saturday|1988Q1|N|Y|Y|2447162|2447161|2446798|2447071|N|N|N|N|N| +2447164|AAAAAAAAMDHFFCAA|1988-01-03|1056|4592|353|1988|0|1|3|1|1988|353|4592|Sunday|1988Q1|N|N|N|2447162|2447161|2446799|2447072|N|N|N|N|N| +2447165|AAAAAAAANDHFFCAA|1988-01-04|1056|4592|353|1988|1|1|4|1|1988|353|4592|Monday|1988Q1|N|N|N|2447162|2447161|2446800|2447073|N|N|N|N|N| +2447166|AAAAAAAAODHFFCAA|1988-01-05|1056|4593|353|1988|2|1|5|1|1988|353|4593|Tuesday|1988Q1|N|N|N|2447162|2447161|2446801|2447074|N|N|N|N|N| +2447167|AAAAAAAAPDHFFCAA|1988-01-06|1056|4593|353|1988|3|1|6|1|1988|353|4593|Wednesday|1988Q1|N|N|N|2447162|2447161|2446802|2447075|N|N|N|N|N| +2447168|AAAAAAAAAEHFFCAA|1988-01-07|1056|4593|353|1988|4|1|7|1|1988|353|4593|Thursday|1988Q1|N|N|N|2447162|2447161|2446803|2447076|N|N|N|N|N| +2447169|AAAAAAAABEHFFCAA|1988-01-08|1056|4593|353|1988|5|1|8|1|1988|353|4593|Friday|1988Q1|N|Y|N|2447162|2447161|2446804|2447077|N|N|N|N|N| +2447170|AAAAAAAACEHFFCAA|1988-01-09|1056|4593|353|1988|6|1|9|1|1988|353|4593|Saturday|1988Q1|N|Y|N|2447162|2447161|2446805|2447078|N|N|N|N|N| +2447171|AAAAAAAADEHFFCAA|1988-01-10|1056|4593|353|1988|0|1|10|1|1988|353|4593|Sunday|1988Q1|N|N|N|2447162|2447161|2446806|2447079|N|N|N|N|N| +2447172|AAAAAAAAEEHFFCAA|1988-01-11|1056|4593|353|1988|1|1|11|1|1988|353|4593|Monday|1988Q1|N|N|N|2447162|2447161|2446807|2447080|N|N|N|N|N| +2447173|AAAAAAAAFEHFFCAA|1988-01-12|1056|4594|353|1988|2|1|12|1|1988|353|4594|Tuesday|1988Q1|N|N|N|2447162|2447161|2446808|2447081|N|N|N|N|N| +2447174|AAAAAAAAGEHFFCAA|1988-01-13|1056|4594|353|1988|3|1|13|1|1988|353|4594|Wednesday|1988Q1|N|N|N|2447162|2447161|2446809|2447082|N|N|N|N|N| +2447175|AAAAAAAAHEHFFCAA|1988-01-14|1056|4594|353|1988|4|1|14|1|1988|353|4594|Thursday|1988Q1|N|N|N|2447162|2447161|2446810|2447083|N|N|N|N|N| +2447176|AAAAAAAAIEHFFCAA|1988-01-15|1056|4594|353|1988|5|1|15|1|1988|353|4594|Friday|1988Q1|N|Y|N|2447162|2447161|2446811|2447084|N|N|N|N|N| +2447177|AAAAAAAAJEHFFCAA|1988-01-16|1056|4594|353|1988|6|1|16|1|1988|353|4594|Saturday|1988Q1|N|Y|N|2447162|2447161|2446812|2447085|N|N|N|N|N| +2447178|AAAAAAAAKEHFFCAA|1988-01-17|1056|4594|353|1988|0|1|17|1|1988|353|4594|Sunday|1988Q1|N|N|N|2447162|2447161|2446813|2447086|N|N|N|N|N| +2447179|AAAAAAAALEHFFCAA|1988-01-18|1056|4594|353|1988|1|1|18|1|1988|353|4594|Monday|1988Q1|N|N|N|2447162|2447161|2446814|2447087|N|N|N|N|N| +2447180|AAAAAAAAMEHFFCAA|1988-01-19|1056|4595|353|1988|2|1|19|1|1988|353|4595|Tuesday|1988Q1|N|N|N|2447162|2447161|2446815|2447088|N|N|N|N|N| +2447181|AAAAAAAANEHFFCAA|1988-01-20|1056|4595|353|1988|3|1|20|1|1988|353|4595|Wednesday|1988Q1|N|N|N|2447162|2447161|2446816|2447089|N|N|N|N|N| +2447182|AAAAAAAAOEHFFCAA|1988-01-21|1056|4595|353|1988|4|1|21|1|1988|353|4595|Thursday|1988Q1|N|N|N|2447162|2447161|2446817|2447090|N|N|N|N|N| +2447183|AAAAAAAAPEHFFCAA|1988-01-22|1056|4595|353|1988|5|1|22|1|1988|353|4595|Friday|1988Q1|N|Y|N|2447162|2447161|2446818|2447091|N|N|N|N|N| +2447184|AAAAAAAAAFHFFCAA|1988-01-23|1056|4595|353|1988|6|1|23|1|1988|353|4595|Saturday|1988Q1|N|Y|N|2447162|2447161|2446819|2447092|N|N|N|N|N| +2447185|AAAAAAAABFHFFCAA|1988-01-24|1056|4595|353|1988|0|1|24|1|1988|353|4595|Sunday|1988Q1|N|N|N|2447162|2447161|2446820|2447093|N|N|N|N|N| +2447186|AAAAAAAACFHFFCAA|1988-01-25|1056|4595|353|1988|1|1|25|1|1988|353|4595|Monday|1988Q1|N|N|N|2447162|2447161|2446821|2447094|N|N|N|N|N| +2447187|AAAAAAAADFHFFCAA|1988-01-26|1056|4596|353|1988|2|1|26|1|1988|353|4596|Tuesday|1988Q1|N|N|N|2447162|2447161|2446822|2447095|N|N|N|N|N| +2447188|AAAAAAAAEFHFFCAA|1988-01-27|1056|4596|353|1988|3|1|27|1|1988|353|4596|Wednesday|1988Q1|N|N|N|2447162|2447161|2446823|2447096|N|N|N|N|N| +2447189|AAAAAAAAFFHFFCAA|1988-01-28|1056|4596|353|1988|4|1|28|1|1988|353|4596|Thursday|1988Q1|N|N|N|2447162|2447161|2446824|2447097|N|N|N|N|N| +2447190|AAAAAAAAGFHFFCAA|1988-01-29|1056|4596|353|1988|5|1|29|1|1988|353|4596|Friday|1988Q1|N|Y|N|2447162|2447161|2446825|2447098|N|N|N|N|N| +2447191|AAAAAAAAHFHFFCAA|1988-01-30|1056|4596|353|1988|6|1|30|1|1988|353|4596|Saturday|1988Q1|N|Y|N|2447162|2447161|2446826|2447099|N|N|N|N|N| +2447192|AAAAAAAAIFHFFCAA|1988-01-31|1056|4596|353|1988|0|1|31|1|1988|353|4596|Sunday|1988Q1|N|N|N|2447162|2447161|2446827|2447100|N|N|N|N|N| +2447193|AAAAAAAAJFHFFCAA|1988-02-01|1057|4596|353|1988|1|2|1|1|1988|353|4596|Monday|1988Q1|N|N|N|2447193|2447223|2446828|2447101|N|N|N|N|N| +2447194|AAAAAAAAKFHFFCAA|1988-02-02|1057|4597|353|1988|2|2|2|1|1988|353|4597|Tuesday|1988Q1|N|N|N|2447193|2447223|2446829|2447102|N|N|N|N|N| +2447195|AAAAAAAALFHFFCAA|1988-02-03|1057|4597|353|1988|3|2|3|1|1988|353|4597|Wednesday|1988Q1|N|N|N|2447193|2447223|2446830|2447103|N|N|N|N|N| +2447196|AAAAAAAAMFHFFCAA|1988-02-04|1057|4597|353|1988|4|2|4|1|1988|353|4597|Thursday|1988Q1|N|N|N|2447193|2447223|2446831|2447104|N|N|N|N|N| +2447197|AAAAAAAANFHFFCAA|1988-02-05|1057|4597|353|1988|5|2|5|1|1988|353|4597|Friday|1988Q1|N|Y|N|2447193|2447223|2446832|2447105|N|N|N|N|N| +2447198|AAAAAAAAOFHFFCAA|1988-02-06|1057|4597|353|1988|6|2|6|1|1988|353|4597|Saturday|1988Q1|N|Y|N|2447193|2447223|2446833|2447106|N|N|N|N|N| +2447199|AAAAAAAAPFHFFCAA|1988-02-07|1057|4597|353|1988|0|2|7|1|1988|353|4597|Sunday|1988Q1|N|N|N|2447193|2447223|2446834|2447107|N|N|N|N|N| +2447200|AAAAAAAAAGHFFCAA|1988-02-08|1057|4597|353|1988|1|2|8|1|1988|353|4597|Monday|1988Q1|N|N|N|2447193|2447223|2446835|2447108|N|N|N|N|N| +2447201|AAAAAAAABGHFFCAA|1988-02-09|1057|4598|353|1988|2|2|9|1|1988|353|4598|Tuesday|1988Q1|N|N|N|2447193|2447223|2446836|2447109|N|N|N|N|N| +2447202|AAAAAAAACGHFFCAA|1988-02-10|1057|4598|353|1988|3|2|10|1|1988|353|4598|Wednesday|1988Q1|N|N|N|2447193|2447223|2446837|2447110|N|N|N|N|N| +2447203|AAAAAAAADGHFFCAA|1988-02-11|1057|4598|353|1988|4|2|11|1|1988|353|4598|Thursday|1988Q1|N|N|N|2447193|2447223|2446838|2447111|N|N|N|N|N| +2447204|AAAAAAAAEGHFFCAA|1988-02-12|1057|4598|353|1988|5|2|12|1|1988|353|4598|Friday|1988Q1|N|Y|N|2447193|2447223|2446839|2447112|N|N|N|N|N| +2447205|AAAAAAAAFGHFFCAA|1988-02-13|1057|4598|353|1988|6|2|13|1|1988|353|4598|Saturday|1988Q1|N|Y|N|2447193|2447223|2446840|2447113|N|N|N|N|N| +2447206|AAAAAAAAGGHFFCAA|1988-02-14|1057|4598|353|1988|0|2|14|1|1988|353|4598|Sunday|1988Q1|N|N|N|2447193|2447223|2446841|2447114|N|N|N|N|N| +2447207|AAAAAAAAHGHFFCAA|1988-02-15|1057|4598|353|1988|1|2|15|1|1988|353|4598|Monday|1988Q1|N|N|N|2447193|2447223|2446842|2447115|N|N|N|N|N| +2447208|AAAAAAAAIGHFFCAA|1988-02-16|1057|4599|353|1988|2|2|16|1|1988|353|4599|Tuesday|1988Q1|N|N|N|2447193|2447223|2446843|2447116|N|N|N|N|N| +2447209|AAAAAAAAJGHFFCAA|1988-02-17|1057|4599|353|1988|3|2|17|1|1988|353|4599|Wednesday|1988Q1|N|N|N|2447193|2447223|2446844|2447117|N|N|N|N|N| +2447210|AAAAAAAAKGHFFCAA|1988-02-18|1057|4599|353|1988|4|2|18|1|1988|353|4599|Thursday|1988Q1|N|N|N|2447193|2447223|2446845|2447118|N|N|N|N|N| +2447211|AAAAAAAALGHFFCAA|1988-02-19|1057|4599|353|1988|5|2|19|1|1988|353|4599|Friday|1988Q1|N|Y|N|2447193|2447223|2446846|2447119|N|N|N|N|N| +2447212|AAAAAAAAMGHFFCAA|1988-02-20|1057|4599|353|1988|6|2|20|1|1988|353|4599|Saturday|1988Q1|N|Y|N|2447193|2447223|2446847|2447120|N|N|N|N|N| +2447213|AAAAAAAANGHFFCAA|1988-02-21|1057|4599|353|1988|0|2|21|1|1988|353|4599|Sunday|1988Q1|N|N|N|2447193|2447223|2446848|2447121|N|N|N|N|N| +2447214|AAAAAAAAOGHFFCAA|1988-02-22|1057|4599|353|1988|1|2|22|1|1988|353|4599|Monday|1988Q1|N|N|N|2447193|2447223|2446849|2447122|N|N|N|N|N| +2447215|AAAAAAAAPGHFFCAA|1988-02-23|1057|4600|353|1988|2|2|23|1|1988|353|4600|Tuesday|1988Q1|N|N|N|2447193|2447223|2446850|2447123|N|N|N|N|N| +2447216|AAAAAAAAAHHFFCAA|1988-02-24|1057|4600|353|1988|3|2|24|1|1988|353|4600|Wednesday|1988Q1|N|N|N|2447193|2447223|2446851|2447124|N|N|N|N|N| +2447217|AAAAAAAABHHFFCAA|1988-02-25|1057|4600|353|1988|4|2|25|1|1988|353|4600|Thursday|1988Q1|N|N|N|2447193|2447223|2446852|2447125|N|N|N|N|N| +2447218|AAAAAAAACHHFFCAA|1988-02-26|1057|4600|353|1988|5|2|26|1|1988|353|4600|Friday|1988Q1|N|Y|N|2447193|2447223|2446853|2447126|N|N|N|N|N| +2447219|AAAAAAAADHHFFCAA|1988-02-27|1057|4600|353|1988|6|2|27|1|1988|353|4600|Saturday|1988Q1|N|Y|N|2447193|2447223|2446854|2447127|N|N|N|N|N| +2447220|AAAAAAAAEHHFFCAA|1988-02-28|1057|4600|353|1988|0|2|28|1|1988|353|4600|Sunday|1988Q1|N|N|N|2447193|2447223|2446855|2447128|N|N|N|N|N| +2447221|AAAAAAAAFHHFFCAA|1988-02-29|1057|4600|353|1988|1|2|29|1|1988|353|4600|Monday|1988Q1|N|N|N|2447193|2447223|2446855|2447129|N|N|N|N|N| +2447222|AAAAAAAAGHHFFCAA|1988-03-01|1058|4601|354|1988|2|3|1|1|1988|354|4601|Tuesday|1988Q1|N|N|N|2447222|2447281|2446856|2447130|N|N|N|N|N| +2447223|AAAAAAAAHHHFFCAA|1988-03-02|1058|4601|354|1988|3|3|2|1|1988|354|4601|Wednesday|1988Q1|N|N|N|2447222|2447281|2446857|2447131|N|N|N|N|N| +2447224|AAAAAAAAIHHFFCAA|1988-03-03|1058|4601|354|1988|4|3|3|1|1988|354|4601|Thursday|1988Q1|N|N|N|2447222|2447281|2446858|2447132|N|N|N|N|N| +2447225|AAAAAAAAJHHFFCAA|1988-03-04|1058|4601|354|1988|5|3|4|1|1988|354|4601|Friday|1988Q1|N|Y|N|2447222|2447281|2446859|2447133|N|N|N|N|N| +2447226|AAAAAAAAKHHFFCAA|1988-03-05|1058|4601|354|1988|6|3|5|1|1988|354|4601|Saturday|1988Q1|N|Y|N|2447222|2447281|2446860|2447134|N|N|N|N|N| +2447227|AAAAAAAALHHFFCAA|1988-03-06|1058|4601|354|1988|0|3|6|1|1988|354|4601|Sunday|1988Q1|N|N|N|2447222|2447281|2446861|2447135|N|N|N|N|N| +2447228|AAAAAAAAMHHFFCAA|1988-03-07|1058|4601|354|1988|1|3|7|1|1988|354|4601|Monday|1988Q1|N|N|N|2447222|2447281|2446862|2447136|N|N|N|N|N| +2447229|AAAAAAAANHHFFCAA|1988-03-08|1058|4602|354|1988|2|3|8|1|1988|354|4602|Tuesday|1988Q1|N|N|N|2447222|2447281|2446863|2447137|N|N|N|N|N| +2447230|AAAAAAAAOHHFFCAA|1988-03-09|1058|4602|354|1988|3|3|9|1|1988|354|4602|Wednesday|1988Q1|N|N|N|2447222|2447281|2446864|2447138|N|N|N|N|N| +2447231|AAAAAAAAPHHFFCAA|1988-03-10|1058|4602|354|1988|4|3|10|1|1988|354|4602|Thursday|1988Q1|N|N|N|2447222|2447281|2446865|2447139|N|N|N|N|N| +2447232|AAAAAAAAAIHFFCAA|1988-03-11|1058|4602|354|1988|5|3|11|1|1988|354|4602|Friday|1988Q1|N|Y|N|2447222|2447281|2446866|2447140|N|N|N|N|N| +2447233|AAAAAAAABIHFFCAA|1988-03-12|1058|4602|354|1988|6|3|12|1|1988|354|4602|Saturday|1988Q1|N|Y|N|2447222|2447281|2446867|2447141|N|N|N|N|N| +2447234|AAAAAAAACIHFFCAA|1988-03-13|1058|4602|354|1988|0|3|13|1|1988|354|4602|Sunday|1988Q1|N|N|N|2447222|2447281|2446868|2447142|N|N|N|N|N| +2447235|AAAAAAAADIHFFCAA|1988-03-14|1058|4602|354|1988|1|3|14|1|1988|354|4602|Monday|1988Q1|N|N|N|2447222|2447281|2446869|2447143|N|N|N|N|N| +2447236|AAAAAAAAEIHFFCAA|1988-03-15|1058|4603|354|1988|2|3|15|1|1988|354|4603|Tuesday|1988Q1|N|N|N|2447222|2447281|2446870|2447144|N|N|N|N|N| +2447237|AAAAAAAAFIHFFCAA|1988-03-16|1058|4603|354|1988|3|3|16|1|1988|354|4603|Wednesday|1988Q1|N|N|N|2447222|2447281|2446871|2447145|N|N|N|N|N| +2447238|AAAAAAAAGIHFFCAA|1988-03-17|1058|4603|354|1988|4|3|17|1|1988|354|4603|Thursday|1988Q1|N|N|N|2447222|2447281|2446872|2447146|N|N|N|N|N| +2447239|AAAAAAAAHIHFFCAA|1988-03-18|1058|4603|354|1988|5|3|18|1|1988|354|4603|Friday|1988Q1|N|Y|N|2447222|2447281|2446873|2447147|N|N|N|N|N| +2447240|AAAAAAAAIIHFFCAA|1988-03-19|1058|4603|354|1988|6|3|19|1|1988|354|4603|Saturday|1988Q1|N|Y|N|2447222|2447281|2446874|2447148|N|N|N|N|N| +2447241|AAAAAAAAJIHFFCAA|1988-03-20|1058|4603|354|1988|0|3|20|1|1988|354|4603|Sunday|1988Q1|N|N|N|2447222|2447281|2446875|2447149|N|N|N|N|N| +2447242|AAAAAAAAKIHFFCAA|1988-03-21|1058|4603|354|1988|1|3|21|1|1988|354|4603|Monday|1988Q1|N|N|N|2447222|2447281|2446876|2447150|N|N|N|N|N| +2447243|AAAAAAAALIHFFCAA|1988-03-22|1058|4604|354|1988|2|3|22|1|1988|354|4604|Tuesday|1988Q1|N|N|N|2447222|2447281|2446877|2447151|N|N|N|N|N| +2447244|AAAAAAAAMIHFFCAA|1988-03-23|1058|4604|354|1988|3|3|23|1|1988|354|4604|Wednesday|1988Q1|N|N|N|2447222|2447281|2446878|2447152|N|N|N|N|N| +2447245|AAAAAAAANIHFFCAA|1988-03-24|1058|4604|354|1988|4|3|24|1|1988|354|4604|Thursday|1988Q1|N|N|N|2447222|2447281|2446879|2447153|N|N|N|N|N| +2447246|AAAAAAAAOIHFFCAA|1988-03-25|1058|4604|354|1988|5|3|25|1|1988|354|4604|Friday|1988Q1|N|Y|N|2447222|2447281|2446880|2447154|N|N|N|N|N| +2447247|AAAAAAAAPIHFFCAA|1988-03-26|1058|4604|354|1988|6|3|26|1|1988|354|4604|Saturday|1988Q1|N|Y|N|2447222|2447281|2446881|2447155|N|N|N|N|N| +2447248|AAAAAAAAAJHFFCAA|1988-03-27|1058|4604|354|1988|0|3|27|1|1988|354|4604|Sunday|1988Q1|N|N|N|2447222|2447281|2446882|2447156|N|N|N|N|N| +2447249|AAAAAAAABJHFFCAA|1988-03-28|1058|4604|354|1988|1|3|28|1|1988|354|4604|Monday|1988Q1|N|N|N|2447222|2447281|2446883|2447157|N|N|N|N|N| +2447250|AAAAAAAACJHFFCAA|1988-03-29|1058|4605|354|1988|2|3|29|1|1988|354|4605|Tuesday|1988Q1|N|N|N|2447222|2447281|2446884|2447158|N|N|N|N|N| +2447251|AAAAAAAADJHFFCAA|1988-03-30|1058|4605|354|1988|3|3|30|1|1988|354|4605|Wednesday|1988Q1|N|N|N|2447222|2447281|2446885|2447159|N|N|N|N|N| +2447252|AAAAAAAAEJHFFCAA|1988-03-31|1058|4605|354|1988|4|3|31|1|1988|354|4605|Thursday|1988Q1|N|N|N|2447222|2447281|2446886|2447160|N|N|N|N|N| +2447253|AAAAAAAAFJHFFCAA|1988-04-01|1059|4605|354|1988|5|4|1|2|1988|354|4605|Friday|1988Q2|N|Y|N|2447253|2447343|2446887|2447162|N|N|N|N|N| +2447254|AAAAAAAAGJHFFCAA|1988-04-02|1059|4605|354|1988|6|4|2|2|1988|354|4605|Saturday|1988Q2|N|Y|N|2447253|2447343|2446888|2447163|N|N|N|N|N| +2447255|AAAAAAAAHJHFFCAA|1988-04-03|1059|4605|354|1988|0|4|3|2|1988|354|4605|Sunday|1988Q2|N|N|N|2447253|2447343|2446889|2447164|N|N|N|N|N| +2447256|AAAAAAAAIJHFFCAA|1988-04-04|1059|4605|354|1988|1|4|4|2|1988|354|4605|Monday|1988Q2|N|N|N|2447253|2447343|2446890|2447165|N|N|N|N|N| +2447257|AAAAAAAAJJHFFCAA|1988-04-05|1059|4606|354|1988|2|4|5|2|1988|354|4606|Tuesday|1988Q2|N|N|N|2447253|2447343|2446891|2447166|N|N|N|N|N| +2447258|AAAAAAAAKJHFFCAA|1988-04-06|1059|4606|354|1988|3|4|6|2|1988|354|4606|Wednesday|1988Q2|N|N|N|2447253|2447343|2446892|2447167|N|N|N|N|N| +2447259|AAAAAAAALJHFFCAA|1988-04-07|1059|4606|354|1988|4|4|7|2|1988|354|4606|Thursday|1988Q2|N|N|N|2447253|2447343|2446893|2447168|N|N|N|N|N| +2447260|AAAAAAAAMJHFFCAA|1988-04-08|1059|4606|354|1988|5|4|8|2|1988|354|4606|Friday|1988Q2|N|Y|N|2447253|2447343|2446894|2447169|N|N|N|N|N| +2447261|AAAAAAAANJHFFCAA|1988-04-09|1059|4606|354|1988|6|4|9|2|1988|354|4606|Saturday|1988Q2|N|Y|N|2447253|2447343|2446895|2447170|N|N|N|N|N| +2447262|AAAAAAAAOJHFFCAA|1988-04-10|1059|4606|354|1988|0|4|10|2|1988|354|4606|Sunday|1988Q2|N|N|N|2447253|2447343|2446896|2447171|N|N|N|N|N| +2447263|AAAAAAAAPJHFFCAA|1988-04-11|1059|4606|354|1988|1|4|11|2|1988|354|4606|Monday|1988Q2|N|N|N|2447253|2447343|2446897|2447172|N|N|N|N|N| +2447264|AAAAAAAAAKHFFCAA|1988-04-12|1059|4607|354|1988|2|4|12|2|1988|354|4607|Tuesday|1988Q2|N|N|N|2447253|2447343|2446898|2447173|N|N|N|N|N| +2447265|AAAAAAAABKHFFCAA|1988-04-13|1059|4607|354|1988|3|4|13|2|1988|354|4607|Wednesday|1988Q2|N|N|N|2447253|2447343|2446899|2447174|N|N|N|N|N| +2447266|AAAAAAAACKHFFCAA|1988-04-14|1059|4607|354|1988|4|4|14|2|1988|354|4607|Thursday|1988Q2|N|N|N|2447253|2447343|2446900|2447175|N|N|N|N|N| +2447267|AAAAAAAADKHFFCAA|1988-04-15|1059|4607|354|1988|5|4|15|2|1988|354|4607|Friday|1988Q2|N|Y|N|2447253|2447343|2446901|2447176|N|N|N|N|N| +2447268|AAAAAAAAEKHFFCAA|1988-04-16|1059|4607|354|1988|6|4|16|2|1988|354|4607|Saturday|1988Q2|N|Y|N|2447253|2447343|2446902|2447177|N|N|N|N|N| +2447269|AAAAAAAAFKHFFCAA|1988-04-17|1059|4607|354|1988|0|4|17|2|1988|354|4607|Sunday|1988Q2|N|N|N|2447253|2447343|2446903|2447178|N|N|N|N|N| +2447270|AAAAAAAAGKHFFCAA|1988-04-18|1059|4607|354|1988|1|4|18|2|1988|354|4607|Monday|1988Q2|N|N|N|2447253|2447343|2446904|2447179|N|N|N|N|N| +2447271|AAAAAAAAHKHFFCAA|1988-04-19|1059|4608|354|1988|2|4|19|2|1988|354|4608|Tuesday|1988Q2|N|N|N|2447253|2447343|2446905|2447180|N|N|N|N|N| +2447272|AAAAAAAAIKHFFCAA|1988-04-20|1059|4608|354|1988|3|4|20|2|1988|354|4608|Wednesday|1988Q2|N|N|N|2447253|2447343|2446906|2447181|N|N|N|N|N| +2447273|AAAAAAAAJKHFFCAA|1988-04-21|1059|4608|354|1988|4|4|21|2|1988|354|4608|Thursday|1988Q2|N|N|N|2447253|2447343|2446907|2447182|N|N|N|N|N| +2447274|AAAAAAAAKKHFFCAA|1988-04-22|1059|4608|354|1988|5|4|22|2|1988|354|4608|Friday|1988Q2|N|Y|N|2447253|2447343|2446908|2447183|N|N|N|N|N| +2447275|AAAAAAAALKHFFCAA|1988-04-23|1059|4608|354|1988|6|4|23|2|1988|354|4608|Saturday|1988Q2|N|Y|N|2447253|2447343|2446909|2447184|N|N|N|N|N| +2447276|AAAAAAAAMKHFFCAA|1988-04-24|1059|4608|354|1988|0|4|24|2|1988|354|4608|Sunday|1988Q2|N|N|N|2447253|2447343|2446910|2447185|N|N|N|N|N| +2447277|AAAAAAAANKHFFCAA|1988-04-25|1059|4608|354|1988|1|4|25|2|1988|354|4608|Monday|1988Q2|N|N|N|2447253|2447343|2446911|2447186|N|N|N|N|N| +2447278|AAAAAAAAOKHFFCAA|1988-04-26|1059|4609|354|1988|2|4|26|2|1988|354|4609|Tuesday|1988Q2|N|N|N|2447253|2447343|2446912|2447187|N|N|N|N|N| +2447279|AAAAAAAAPKHFFCAA|1988-04-27|1059|4609|354|1988|3|4|27|2|1988|354|4609|Wednesday|1988Q2|N|N|N|2447253|2447343|2446913|2447188|N|N|N|N|N| +2447280|AAAAAAAAALHFFCAA|1988-04-28|1059|4609|354|1988|4|4|28|2|1988|354|4609|Thursday|1988Q2|N|N|N|2447253|2447343|2446914|2447189|N|N|N|N|N| +2447281|AAAAAAAABLHFFCAA|1988-04-29|1059|4609|354|1988|5|4|29|2|1988|354|4609|Friday|1988Q2|N|Y|N|2447253|2447343|2446915|2447190|N|N|N|N|N| +2447282|AAAAAAAACLHFFCAA|1988-04-30|1059|4609|354|1988|6|4|30|2|1988|354|4609|Saturday|1988Q2|N|Y|N|2447253|2447343|2446916|2447191|N|N|N|N|N| +2447283|AAAAAAAADLHFFCAA|1988-05-01|1060|4609|354|1988|0|5|1|2|1988|354|4609|Sunday|1988Q2|N|N|N|2447283|2447403|2446917|2447192|N|N|N|N|N| +2447284|AAAAAAAAELHFFCAA|1988-05-02|1060|4609|354|1988|1|5|2|2|1988|354|4609|Monday|1988Q2|N|N|N|2447283|2447403|2446918|2447193|N|N|N|N|N| +2447285|AAAAAAAAFLHFFCAA|1988-05-03|1060|4610|354|1988|2|5|3|2|1988|354|4610|Tuesday|1988Q2|N|N|N|2447283|2447403|2446919|2447194|N|N|N|N|N| +2447286|AAAAAAAAGLHFFCAA|1988-05-04|1060|4610|354|1988|3|5|4|2|1988|354|4610|Wednesday|1988Q2|N|N|N|2447283|2447403|2446920|2447195|N|N|N|N|N| +2447287|AAAAAAAAHLHFFCAA|1988-05-05|1060|4610|354|1988|4|5|5|2|1988|354|4610|Thursday|1988Q2|N|N|N|2447283|2447403|2446921|2447196|N|N|N|N|N| +2447288|AAAAAAAAILHFFCAA|1988-05-06|1060|4610|354|1988|5|5|6|2|1988|354|4610|Friday|1988Q2|N|Y|N|2447283|2447403|2446922|2447197|N|N|N|N|N| +2447289|AAAAAAAAJLHFFCAA|1988-05-07|1060|4610|354|1988|6|5|7|2|1988|354|4610|Saturday|1988Q2|N|Y|N|2447283|2447403|2446923|2447198|N|N|N|N|N| +2447290|AAAAAAAAKLHFFCAA|1988-05-08|1060|4610|354|1988|0|5|8|2|1988|354|4610|Sunday|1988Q2|N|N|N|2447283|2447403|2446924|2447199|N|N|N|N|N| +2447291|AAAAAAAALLHFFCAA|1988-05-09|1060|4610|354|1988|1|5|9|2|1988|354|4610|Monday|1988Q2|N|N|N|2447283|2447403|2446925|2447200|N|N|N|N|N| +2447292|AAAAAAAAMLHFFCAA|1988-05-10|1060|4611|354|1988|2|5|10|2|1988|354|4611|Tuesday|1988Q2|N|N|N|2447283|2447403|2446926|2447201|N|N|N|N|N| +2447293|AAAAAAAANLHFFCAA|1988-05-11|1060|4611|354|1988|3|5|11|2|1988|354|4611|Wednesday|1988Q2|N|N|N|2447283|2447403|2446927|2447202|N|N|N|N|N| +2447294|AAAAAAAAOLHFFCAA|1988-05-12|1060|4611|354|1988|4|5|12|2|1988|354|4611|Thursday|1988Q2|N|N|N|2447283|2447403|2446928|2447203|N|N|N|N|N| +2447295|AAAAAAAAPLHFFCAA|1988-05-13|1060|4611|354|1988|5|5|13|2|1988|354|4611|Friday|1988Q2|N|Y|N|2447283|2447403|2446929|2447204|N|N|N|N|N| +2447296|AAAAAAAAAMHFFCAA|1988-05-14|1060|4611|354|1988|6|5|14|2|1988|354|4611|Saturday|1988Q2|N|Y|N|2447283|2447403|2446930|2447205|N|N|N|N|N| +2447297|AAAAAAAABMHFFCAA|1988-05-15|1060|4611|354|1988|0|5|15|2|1988|354|4611|Sunday|1988Q2|N|N|N|2447283|2447403|2446931|2447206|N|N|N|N|N| +2447298|AAAAAAAACMHFFCAA|1988-05-16|1060|4611|354|1988|1|5|16|2|1988|354|4611|Monday|1988Q2|N|N|N|2447283|2447403|2446932|2447207|N|N|N|N|N| +2447299|AAAAAAAADMHFFCAA|1988-05-17|1060|4612|354|1988|2|5|17|2|1988|354|4612|Tuesday|1988Q2|N|N|N|2447283|2447403|2446933|2447208|N|N|N|N|N| +2447300|AAAAAAAAEMHFFCAA|1988-05-18|1060|4612|354|1988|3|5|18|2|1988|354|4612|Wednesday|1988Q2|N|N|N|2447283|2447403|2446934|2447209|N|N|N|N|N| +2447301|AAAAAAAAFMHFFCAA|1988-05-19|1060|4612|354|1988|4|5|19|2|1988|354|4612|Thursday|1988Q2|N|N|N|2447283|2447403|2446935|2447210|N|N|N|N|N| +2447302|AAAAAAAAGMHFFCAA|1988-05-20|1060|4612|354|1988|5|5|20|2|1988|354|4612|Friday|1988Q2|N|Y|N|2447283|2447403|2446936|2447211|N|N|N|N|N| +2447303|AAAAAAAAHMHFFCAA|1988-05-21|1060|4612|354|1988|6|5|21|2|1988|354|4612|Saturday|1988Q2|N|Y|N|2447283|2447403|2446937|2447212|N|N|N|N|N| +2447304|AAAAAAAAIMHFFCAA|1988-05-22|1060|4612|354|1988|0|5|22|2|1988|354|4612|Sunday|1988Q2|N|N|N|2447283|2447403|2446938|2447213|N|N|N|N|N| +2447305|AAAAAAAAJMHFFCAA|1988-05-23|1060|4612|354|1988|1|5|23|2|1988|354|4612|Monday|1988Q2|N|N|N|2447283|2447403|2446939|2447214|N|N|N|N|N| +2447306|AAAAAAAAKMHFFCAA|1988-05-24|1060|4613|354|1988|2|5|24|2|1988|354|4613|Tuesday|1988Q2|N|N|N|2447283|2447403|2446940|2447215|N|N|N|N|N| +2447307|AAAAAAAALMHFFCAA|1988-05-25|1060|4613|354|1988|3|5|25|2|1988|354|4613|Wednesday|1988Q2|N|N|N|2447283|2447403|2446941|2447216|N|N|N|N|N| +2447308|AAAAAAAAMMHFFCAA|1988-05-26|1060|4613|354|1988|4|5|26|2|1988|354|4613|Thursday|1988Q2|N|N|N|2447283|2447403|2446942|2447217|N|N|N|N|N| +2447309|AAAAAAAANMHFFCAA|1988-05-27|1060|4613|354|1988|5|5|27|2|1988|354|4613|Friday|1988Q2|N|Y|N|2447283|2447403|2446943|2447218|N|N|N|N|N| +2447310|AAAAAAAAOMHFFCAA|1988-05-28|1060|4613|354|1988|6|5|28|2|1988|354|4613|Saturday|1988Q2|N|Y|N|2447283|2447403|2446944|2447219|N|N|N|N|N| +2447311|AAAAAAAAPMHFFCAA|1988-05-29|1060|4613|354|1988|0|5|29|2|1988|354|4613|Sunday|1988Q2|N|N|N|2447283|2447403|2446945|2447220|N|N|N|N|N| +2447312|AAAAAAAAANHFFCAA|1988-05-30|1060|4613|354|1988|1|5|30|2|1988|354|4613|Monday|1988Q2|N|N|N|2447283|2447403|2446946|2447221|N|N|N|N|N| +2447313|AAAAAAAABNHFFCAA|1988-05-31|1060|4614|354|1988|2|5|31|2|1988|354|4614|Tuesday|1988Q2|N|N|N|2447283|2447403|2446947|2447222|N|N|N|N|N| +2447314|AAAAAAAACNHFFCAA|1988-06-01|1061|4614|355|1988|3|6|1|2|1988|355|4614|Wednesday|1988Q2|N|N|N|2447314|2447465|2446948|2447223|N|N|N|N|N| +2447315|AAAAAAAADNHFFCAA|1988-06-02|1061|4614|355|1988|4|6|2|2|1988|355|4614|Thursday|1988Q2|N|N|N|2447314|2447465|2446949|2447224|N|N|N|N|N| +2447316|AAAAAAAAENHFFCAA|1988-06-03|1061|4614|355|1988|5|6|3|2|1988|355|4614|Friday|1988Q2|N|Y|N|2447314|2447465|2446950|2447225|N|N|N|N|N| +2447317|AAAAAAAAFNHFFCAA|1988-06-04|1061|4614|355|1988|6|6|4|2|1988|355|4614|Saturday|1988Q2|N|Y|N|2447314|2447465|2446951|2447226|N|N|N|N|N| +2447318|AAAAAAAAGNHFFCAA|1988-06-05|1061|4614|355|1988|0|6|5|2|1988|355|4614|Sunday|1988Q2|N|N|N|2447314|2447465|2446952|2447227|N|N|N|N|N| +2447319|AAAAAAAAHNHFFCAA|1988-06-06|1061|4614|355|1988|1|6|6|2|1988|355|4614|Monday|1988Q2|N|N|N|2447314|2447465|2446953|2447228|N|N|N|N|N| +2447320|AAAAAAAAINHFFCAA|1988-06-07|1061|4615|355|1988|2|6|7|2|1988|355|4615|Tuesday|1988Q2|N|N|N|2447314|2447465|2446954|2447229|N|N|N|N|N| +2447321|AAAAAAAAJNHFFCAA|1988-06-08|1061|4615|355|1988|3|6|8|2|1988|355|4615|Wednesday|1988Q2|N|N|N|2447314|2447465|2446955|2447230|N|N|N|N|N| +2447322|AAAAAAAAKNHFFCAA|1988-06-09|1061|4615|355|1988|4|6|9|2|1988|355|4615|Thursday|1988Q2|N|N|N|2447314|2447465|2446956|2447231|N|N|N|N|N| +2447323|AAAAAAAALNHFFCAA|1988-06-10|1061|4615|355|1988|5|6|10|2|1988|355|4615|Friday|1988Q2|N|Y|N|2447314|2447465|2446957|2447232|N|N|N|N|N| +2447324|AAAAAAAAMNHFFCAA|1988-06-11|1061|4615|355|1988|6|6|11|2|1988|355|4615|Saturday|1988Q2|N|Y|N|2447314|2447465|2446958|2447233|N|N|N|N|N| +2447325|AAAAAAAANNHFFCAA|1988-06-12|1061|4615|355|1988|0|6|12|2|1988|355|4615|Sunday|1988Q2|N|N|N|2447314|2447465|2446959|2447234|N|N|N|N|N| +2447326|AAAAAAAAONHFFCAA|1988-06-13|1061|4615|355|1988|1|6|13|2|1988|355|4615|Monday|1988Q2|N|N|N|2447314|2447465|2446960|2447235|N|N|N|N|N| +2447327|AAAAAAAAPNHFFCAA|1988-06-14|1061|4616|355|1988|2|6|14|2|1988|355|4616|Tuesday|1988Q2|N|N|N|2447314|2447465|2446961|2447236|N|N|N|N|N| +2447328|AAAAAAAAAOHFFCAA|1988-06-15|1061|4616|355|1988|3|6|15|2|1988|355|4616|Wednesday|1988Q2|N|N|N|2447314|2447465|2446962|2447237|N|N|N|N|N| +2447329|AAAAAAAABOHFFCAA|1988-06-16|1061|4616|355|1988|4|6|16|2|1988|355|4616|Thursday|1988Q2|N|N|N|2447314|2447465|2446963|2447238|N|N|N|N|N| +2447330|AAAAAAAACOHFFCAA|1988-06-17|1061|4616|355|1988|5|6|17|2|1988|355|4616|Friday|1988Q2|N|Y|N|2447314|2447465|2446964|2447239|N|N|N|N|N| +2447331|AAAAAAAADOHFFCAA|1988-06-18|1061|4616|355|1988|6|6|18|2|1988|355|4616|Saturday|1988Q2|N|Y|N|2447314|2447465|2446965|2447240|N|N|N|N|N| +2447332|AAAAAAAAEOHFFCAA|1988-06-19|1061|4616|355|1988|0|6|19|2|1988|355|4616|Sunday|1988Q2|N|N|N|2447314|2447465|2446966|2447241|N|N|N|N|N| +2447333|AAAAAAAAFOHFFCAA|1988-06-20|1061|4616|355|1988|1|6|20|2|1988|355|4616|Monday|1988Q2|N|N|N|2447314|2447465|2446967|2447242|N|N|N|N|N| +2447334|AAAAAAAAGOHFFCAA|1988-06-21|1061|4617|355|1988|2|6|21|2|1988|355|4617|Tuesday|1988Q2|N|N|N|2447314|2447465|2446968|2447243|N|N|N|N|N| +2447335|AAAAAAAAHOHFFCAA|1988-06-22|1061|4617|355|1988|3|6|22|2|1988|355|4617|Wednesday|1988Q2|N|N|N|2447314|2447465|2446969|2447244|N|N|N|N|N| +2447336|AAAAAAAAIOHFFCAA|1988-06-23|1061|4617|355|1988|4|6|23|2|1988|355|4617|Thursday|1988Q2|N|N|N|2447314|2447465|2446970|2447245|N|N|N|N|N| +2447337|AAAAAAAAJOHFFCAA|1988-06-24|1061|4617|355|1988|5|6|24|2|1988|355|4617|Friday|1988Q2|N|Y|N|2447314|2447465|2446971|2447246|N|N|N|N|N| +2447338|AAAAAAAAKOHFFCAA|1988-06-25|1061|4617|355|1988|6|6|25|2|1988|355|4617|Saturday|1988Q2|N|Y|N|2447314|2447465|2446972|2447247|N|N|N|N|N| +2447339|AAAAAAAALOHFFCAA|1988-06-26|1061|4617|355|1988|0|6|26|2|1988|355|4617|Sunday|1988Q2|N|N|N|2447314|2447465|2446973|2447248|N|N|N|N|N| +2447340|AAAAAAAAMOHFFCAA|1988-06-27|1061|4617|355|1988|1|6|27|2|1988|355|4617|Monday|1988Q2|N|N|N|2447314|2447465|2446974|2447249|N|N|N|N|N| +2447341|AAAAAAAANOHFFCAA|1988-06-28|1061|4618|355|1988|2|6|28|2|1988|355|4618|Tuesday|1988Q2|N|N|N|2447314|2447465|2446975|2447250|N|N|N|N|N| +2447342|AAAAAAAAOOHFFCAA|1988-06-29|1061|4618|355|1988|3|6|29|2|1988|355|4618|Wednesday|1988Q2|N|N|N|2447314|2447465|2446976|2447251|N|N|N|N|N| +2447343|AAAAAAAAPOHFFCAA|1988-06-30|1061|4618|355|1988|4|6|30|2|1988|355|4618|Thursday|1988Q2|N|N|N|2447314|2447465|2446977|2447252|N|N|N|N|N| +2447344|AAAAAAAAAPHFFCAA|1988-07-01|1062|4618|355|1988|5|7|1|3|1988|355|4618|Friday|1988Q3|N|Y|N|2447344|2447525|2446978|2447253|N|N|N|N|N| +2447345|AAAAAAAABPHFFCAA|1988-07-02|1062|4618|355|1988|6|7|2|3|1988|355|4618|Saturday|1988Q3|N|Y|N|2447344|2447525|2446979|2447254|N|N|N|N|N| +2447346|AAAAAAAACPHFFCAA|1988-07-03|1062|4618|355|1988|0|7|3|3|1988|355|4618|Sunday|1988Q3|N|N|N|2447344|2447525|2446980|2447255|N|N|N|N|N| +2447347|AAAAAAAADPHFFCAA|1988-07-04|1062|4618|355|1988|1|7|4|3|1988|355|4618|Monday|1988Q3|Y|N|N|2447344|2447525|2446981|2447256|N|N|N|N|N| +2447348|AAAAAAAAEPHFFCAA|1988-07-05|1062|4619|355|1988|2|7|5|3|1988|355|4619|Tuesday|1988Q3|N|N|Y|2447344|2447525|2446982|2447257|N|N|N|N|N| +2447349|AAAAAAAAFPHFFCAA|1988-07-06|1062|4619|355|1988|3|7|6|3|1988|355|4619|Wednesday|1988Q3|N|N|N|2447344|2447525|2446983|2447258|N|N|N|N|N| +2447350|AAAAAAAAGPHFFCAA|1988-07-07|1062|4619|355|1988|4|7|7|3|1988|355|4619|Thursday|1988Q3|N|N|N|2447344|2447525|2446984|2447259|N|N|N|N|N| +2447351|AAAAAAAAHPHFFCAA|1988-07-08|1062|4619|355|1988|5|7|8|3|1988|355|4619|Friday|1988Q3|N|Y|N|2447344|2447525|2446985|2447260|N|N|N|N|N| +2447352|AAAAAAAAIPHFFCAA|1988-07-09|1062|4619|355|1988|6|7|9|3|1988|355|4619|Saturday|1988Q3|N|Y|N|2447344|2447525|2446986|2447261|N|N|N|N|N| +2447353|AAAAAAAAJPHFFCAA|1988-07-10|1062|4619|355|1988|0|7|10|3|1988|355|4619|Sunday|1988Q3|N|N|N|2447344|2447525|2446987|2447262|N|N|N|N|N| +2447354|AAAAAAAAKPHFFCAA|1988-07-11|1062|4619|355|1988|1|7|11|3|1988|355|4619|Monday|1988Q3|N|N|N|2447344|2447525|2446988|2447263|N|N|N|N|N| +2447355|AAAAAAAALPHFFCAA|1988-07-12|1062|4620|355|1988|2|7|12|3|1988|355|4620|Tuesday|1988Q3|N|N|N|2447344|2447525|2446989|2447264|N|N|N|N|N| +2447356|AAAAAAAAMPHFFCAA|1988-07-13|1062|4620|355|1988|3|7|13|3|1988|355|4620|Wednesday|1988Q3|N|N|N|2447344|2447525|2446990|2447265|N|N|N|N|N| +2447357|AAAAAAAANPHFFCAA|1988-07-14|1062|4620|355|1988|4|7|14|3|1988|355|4620|Thursday|1988Q3|N|N|N|2447344|2447525|2446991|2447266|N|N|N|N|N| +2447358|AAAAAAAAOPHFFCAA|1988-07-15|1062|4620|355|1988|5|7|15|3|1988|355|4620|Friday|1988Q3|N|Y|N|2447344|2447525|2446992|2447267|N|N|N|N|N| +2447359|AAAAAAAAPPHFFCAA|1988-07-16|1062|4620|355|1988|6|7|16|3|1988|355|4620|Saturday|1988Q3|N|Y|N|2447344|2447525|2446993|2447268|N|N|N|N|N| +2447360|AAAAAAAAAAIFFCAA|1988-07-17|1062|4620|355|1988|0|7|17|3|1988|355|4620|Sunday|1988Q3|N|N|N|2447344|2447525|2446994|2447269|N|N|N|N|N| +2447361|AAAAAAAABAIFFCAA|1988-07-18|1062|4620|355|1988|1|7|18|3|1988|355|4620|Monday|1988Q3|N|N|N|2447344|2447525|2446995|2447270|N|N|N|N|N| +2447362|AAAAAAAACAIFFCAA|1988-07-19|1062|4621|355|1988|2|7|19|3|1988|355|4621|Tuesday|1988Q3|N|N|N|2447344|2447525|2446996|2447271|N|N|N|N|N| +2447363|AAAAAAAADAIFFCAA|1988-07-20|1062|4621|355|1988|3|7|20|3|1988|355|4621|Wednesday|1988Q3|N|N|N|2447344|2447525|2446997|2447272|N|N|N|N|N| +2447364|AAAAAAAAEAIFFCAA|1988-07-21|1062|4621|355|1988|4|7|21|3|1988|355|4621|Thursday|1988Q3|N|N|N|2447344|2447525|2446998|2447273|N|N|N|N|N| +2447365|AAAAAAAAFAIFFCAA|1988-07-22|1062|4621|355|1988|5|7|22|3|1988|355|4621|Friday|1988Q3|N|Y|N|2447344|2447525|2446999|2447274|N|N|N|N|N| +2447366|AAAAAAAAGAIFFCAA|1988-07-23|1062|4621|355|1988|6|7|23|3|1988|355|4621|Saturday|1988Q3|N|Y|N|2447344|2447525|2447000|2447275|N|N|N|N|N| +2447367|AAAAAAAAHAIFFCAA|1988-07-24|1062|4621|355|1988|0|7|24|3|1988|355|4621|Sunday|1988Q3|N|N|N|2447344|2447525|2447001|2447276|N|N|N|N|N| +2447368|AAAAAAAAIAIFFCAA|1988-07-25|1062|4621|355|1988|1|7|25|3|1988|355|4621|Monday|1988Q3|N|N|N|2447344|2447525|2447002|2447277|N|N|N|N|N| +2447369|AAAAAAAAJAIFFCAA|1988-07-26|1062|4622|355|1988|2|7|26|3|1988|355|4622|Tuesday|1988Q3|N|N|N|2447344|2447525|2447003|2447278|N|N|N|N|N| +2447370|AAAAAAAAKAIFFCAA|1988-07-27|1062|4622|355|1988|3|7|27|3|1988|355|4622|Wednesday|1988Q3|N|N|N|2447344|2447525|2447004|2447279|N|N|N|N|N| +2447371|AAAAAAAALAIFFCAA|1988-07-28|1062|4622|355|1988|4|7|28|3|1988|355|4622|Thursday|1988Q3|N|N|N|2447344|2447525|2447005|2447280|N|N|N|N|N| +2447372|AAAAAAAAMAIFFCAA|1988-07-29|1062|4622|355|1988|5|7|29|3|1988|355|4622|Friday|1988Q3|N|Y|N|2447344|2447525|2447006|2447281|N|N|N|N|N| +2447373|AAAAAAAANAIFFCAA|1988-07-30|1062|4622|355|1988|6|7|30|3|1988|355|4622|Saturday|1988Q3|N|Y|N|2447344|2447525|2447007|2447282|N|N|N|N|N| +2447374|AAAAAAAAOAIFFCAA|1988-07-31|1062|4622|355|1988|0|7|31|3|1988|355|4622|Sunday|1988Q3|N|N|N|2447344|2447525|2447008|2447283|N|N|N|N|N| +2447375|AAAAAAAAPAIFFCAA|1988-08-01|1063|4622|355|1988|1|8|1|3|1988|355|4622|Monday|1988Q3|N|N|N|2447375|2447587|2447009|2447284|N|N|N|N|N| +2447376|AAAAAAAAABIFFCAA|1988-08-02|1063|4623|355|1988|2|8|2|3|1988|355|4623|Tuesday|1988Q3|N|N|N|2447375|2447587|2447010|2447285|N|N|N|N|N| +2447377|AAAAAAAABBIFFCAA|1988-08-03|1063|4623|355|1988|3|8|3|3|1988|355|4623|Wednesday|1988Q3|N|N|N|2447375|2447587|2447011|2447286|N|N|N|N|N| +2447378|AAAAAAAACBIFFCAA|1988-08-04|1063|4623|355|1988|4|8|4|3|1988|355|4623|Thursday|1988Q3|N|N|N|2447375|2447587|2447012|2447287|N|N|N|N|N| +2447379|AAAAAAAADBIFFCAA|1988-08-05|1063|4623|355|1988|5|8|5|3|1988|355|4623|Friday|1988Q3|N|Y|N|2447375|2447587|2447013|2447288|N|N|N|N|N| +2447380|AAAAAAAAEBIFFCAA|1988-08-06|1063|4623|355|1988|6|8|6|3|1988|355|4623|Saturday|1988Q3|N|Y|N|2447375|2447587|2447014|2447289|N|N|N|N|N| +2447381|AAAAAAAAFBIFFCAA|1988-08-07|1063|4623|355|1988|0|8|7|3|1988|355|4623|Sunday|1988Q3|N|N|N|2447375|2447587|2447015|2447290|N|N|N|N|N| +2447382|AAAAAAAAGBIFFCAA|1988-08-08|1063|4623|355|1988|1|8|8|3|1988|355|4623|Monday|1988Q3|N|N|N|2447375|2447587|2447016|2447291|N|N|N|N|N| +2447383|AAAAAAAAHBIFFCAA|1988-08-09|1063|4624|355|1988|2|8|9|3|1988|355|4624|Tuesday|1988Q3|N|N|N|2447375|2447587|2447017|2447292|N|N|N|N|N| +2447384|AAAAAAAAIBIFFCAA|1988-08-10|1063|4624|355|1988|3|8|10|3|1988|355|4624|Wednesday|1988Q3|N|N|N|2447375|2447587|2447018|2447293|N|N|N|N|N| +2447385|AAAAAAAAJBIFFCAA|1988-08-11|1063|4624|355|1988|4|8|11|3|1988|355|4624|Thursday|1988Q3|N|N|N|2447375|2447587|2447019|2447294|N|N|N|N|N| +2447386|AAAAAAAAKBIFFCAA|1988-08-12|1063|4624|355|1988|5|8|12|3|1988|355|4624|Friday|1988Q3|N|Y|N|2447375|2447587|2447020|2447295|N|N|N|N|N| +2447387|AAAAAAAALBIFFCAA|1988-08-13|1063|4624|355|1988|6|8|13|3|1988|355|4624|Saturday|1988Q3|N|Y|N|2447375|2447587|2447021|2447296|N|N|N|N|N| +2447388|AAAAAAAAMBIFFCAA|1988-08-14|1063|4624|355|1988|0|8|14|3|1988|355|4624|Sunday|1988Q3|N|N|N|2447375|2447587|2447022|2447297|N|N|N|N|N| +2447389|AAAAAAAANBIFFCAA|1988-08-15|1063|4624|355|1988|1|8|15|3|1988|355|4624|Monday|1988Q3|N|N|N|2447375|2447587|2447023|2447298|N|N|N|N|N| +2447390|AAAAAAAAOBIFFCAA|1988-08-16|1063|4625|355|1988|2|8|16|3|1988|355|4625|Tuesday|1988Q3|N|N|N|2447375|2447587|2447024|2447299|N|N|N|N|N| +2447391|AAAAAAAAPBIFFCAA|1988-08-17|1063|4625|355|1988|3|8|17|3|1988|355|4625|Wednesday|1988Q3|N|N|N|2447375|2447587|2447025|2447300|N|N|N|N|N| +2447392|AAAAAAAAACIFFCAA|1988-08-18|1063|4625|355|1988|4|8|18|3|1988|355|4625|Thursday|1988Q3|N|N|N|2447375|2447587|2447026|2447301|N|N|N|N|N| +2447393|AAAAAAAABCIFFCAA|1988-08-19|1063|4625|355|1988|5|8|19|3|1988|355|4625|Friday|1988Q3|N|Y|N|2447375|2447587|2447027|2447302|N|N|N|N|N| +2447394|AAAAAAAACCIFFCAA|1988-08-20|1063|4625|355|1988|6|8|20|3|1988|355|4625|Saturday|1988Q3|N|Y|N|2447375|2447587|2447028|2447303|N|N|N|N|N| +2447395|AAAAAAAADCIFFCAA|1988-08-21|1063|4625|355|1988|0|8|21|3|1988|355|4625|Sunday|1988Q3|N|N|N|2447375|2447587|2447029|2447304|N|N|N|N|N| +2447396|AAAAAAAAECIFFCAA|1988-08-22|1063|4625|355|1988|1|8|22|3|1988|355|4625|Monday|1988Q3|N|N|N|2447375|2447587|2447030|2447305|N|N|N|N|N| +2447397|AAAAAAAAFCIFFCAA|1988-08-23|1063|4626|355|1988|2|8|23|3|1988|355|4626|Tuesday|1988Q3|N|N|N|2447375|2447587|2447031|2447306|N|N|N|N|N| +2447398|AAAAAAAAGCIFFCAA|1988-08-24|1063|4626|355|1988|3|8|24|3|1988|355|4626|Wednesday|1988Q3|N|N|N|2447375|2447587|2447032|2447307|N|N|N|N|N| +2447399|AAAAAAAAHCIFFCAA|1988-08-25|1063|4626|355|1988|4|8|25|3|1988|355|4626|Thursday|1988Q3|N|N|N|2447375|2447587|2447033|2447308|N|N|N|N|N| +2447400|AAAAAAAAICIFFCAA|1988-08-26|1063|4626|355|1988|5|8|26|3|1988|355|4626|Friday|1988Q3|N|Y|N|2447375|2447587|2447034|2447309|N|N|N|N|N| +2447401|AAAAAAAAJCIFFCAA|1988-08-27|1063|4626|355|1988|6|8|27|3|1988|355|4626|Saturday|1988Q3|N|Y|N|2447375|2447587|2447035|2447310|N|N|N|N|N| +2447402|AAAAAAAAKCIFFCAA|1988-08-28|1063|4626|355|1988|0|8|28|3|1988|355|4626|Sunday|1988Q3|N|N|N|2447375|2447587|2447036|2447311|N|N|N|N|N| +2447403|AAAAAAAALCIFFCAA|1988-08-29|1063|4626|355|1988|1|8|29|3|1988|355|4626|Monday|1988Q3|N|N|N|2447375|2447587|2447037|2447312|N|N|N|N|N| +2447404|AAAAAAAAMCIFFCAA|1988-08-30|1063|4627|355|1988|2|8|30|3|1988|355|4627|Tuesday|1988Q3|N|N|N|2447375|2447587|2447038|2447313|N|N|N|N|N| +2447405|AAAAAAAANCIFFCAA|1988-08-31|1063|4627|355|1988|3|8|31|3|1988|355|4627|Wednesday|1988Q3|N|N|N|2447375|2447587|2447039|2447314|N|N|N|N|N| +2447406|AAAAAAAAOCIFFCAA|1988-09-01|1064|4627|356|1988|4|9|1|3|1988|356|4627|Thursday|1988Q3|N|N|N|2447406|2447649|2447040|2447315|N|N|N|N|N| +2447407|AAAAAAAAPCIFFCAA|1988-09-02|1064|4627|356|1988|5|9|2|3|1988|356|4627|Friday|1988Q3|N|Y|N|2447406|2447649|2447041|2447316|N|N|N|N|N| +2447408|AAAAAAAAADIFFCAA|1988-09-03|1064|4627|356|1988|6|9|3|3|1988|356|4627|Saturday|1988Q3|N|Y|N|2447406|2447649|2447042|2447317|N|N|N|N|N| +2447409|AAAAAAAABDIFFCAA|1988-09-04|1064|4627|356|1988|0|9|4|3|1988|356|4627|Sunday|1988Q3|N|N|N|2447406|2447649|2447043|2447318|N|N|N|N|N| +2447410|AAAAAAAACDIFFCAA|1988-09-05|1064|4627|356|1988|1|9|5|3|1988|356|4627|Monday|1988Q3|N|N|N|2447406|2447649|2447044|2447319|N|N|N|N|N| +2447411|AAAAAAAADDIFFCAA|1988-09-06|1064|4628|356|1988|2|9|6|3|1988|356|4628|Tuesday|1988Q3|N|N|N|2447406|2447649|2447045|2447320|N|N|N|N|N| +2447412|AAAAAAAAEDIFFCAA|1988-09-07|1064|4628|356|1988|3|9|7|3|1988|356|4628|Wednesday|1988Q3|N|N|N|2447406|2447649|2447046|2447321|N|N|N|N|N| +2447413|AAAAAAAAFDIFFCAA|1988-09-08|1064|4628|356|1988|4|9|8|3|1988|356|4628|Thursday|1988Q3|N|N|N|2447406|2447649|2447047|2447322|N|N|N|N|N| +2447414|AAAAAAAAGDIFFCAA|1988-09-09|1064|4628|356|1988|5|9|9|3|1988|356|4628|Friday|1988Q3|N|Y|N|2447406|2447649|2447048|2447323|N|N|N|N|N| +2447415|AAAAAAAAHDIFFCAA|1988-09-10|1064|4628|356|1988|6|9|10|3|1988|356|4628|Saturday|1988Q3|N|Y|N|2447406|2447649|2447049|2447324|N|N|N|N|N| +2447416|AAAAAAAAIDIFFCAA|1988-09-11|1064|4628|356|1988|0|9|11|3|1988|356|4628|Sunday|1988Q3|N|N|N|2447406|2447649|2447050|2447325|N|N|N|N|N| +2447417|AAAAAAAAJDIFFCAA|1988-09-12|1064|4628|356|1988|1|9|12|3|1988|356|4628|Monday|1988Q3|N|N|N|2447406|2447649|2447051|2447326|N|N|N|N|N| +2447418|AAAAAAAAKDIFFCAA|1988-09-13|1064|4629|356|1988|2|9|13|3|1988|356|4629|Tuesday|1988Q3|N|N|N|2447406|2447649|2447052|2447327|N|N|N|N|N| +2447419|AAAAAAAALDIFFCAA|1988-09-14|1064|4629|356|1988|3|9|14|3|1988|356|4629|Wednesday|1988Q3|N|N|N|2447406|2447649|2447053|2447328|N|N|N|N|N| +2447420|AAAAAAAAMDIFFCAA|1988-09-15|1064|4629|356|1988|4|9|15|3|1988|356|4629|Thursday|1988Q3|N|N|N|2447406|2447649|2447054|2447329|N|N|N|N|N| +2447421|AAAAAAAANDIFFCAA|1988-09-16|1064|4629|356|1988|5|9|16|3|1988|356|4629|Friday|1988Q3|N|Y|N|2447406|2447649|2447055|2447330|N|N|N|N|N| +2447422|AAAAAAAAODIFFCAA|1988-09-17|1064|4629|356|1988|6|9|17|3|1988|356|4629|Saturday|1988Q3|N|Y|N|2447406|2447649|2447056|2447331|N|N|N|N|N| +2447423|AAAAAAAAPDIFFCAA|1988-09-18|1064|4629|356|1988|0|9|18|3|1988|356|4629|Sunday|1988Q3|N|N|N|2447406|2447649|2447057|2447332|N|N|N|N|N| +2447424|AAAAAAAAAEIFFCAA|1988-09-19|1064|4629|356|1988|1|9|19|3|1988|356|4629|Monday|1988Q3|N|N|N|2447406|2447649|2447058|2447333|N|N|N|N|N| +2447425|AAAAAAAABEIFFCAA|1988-09-20|1064|4630|356|1988|2|9|20|3|1988|356|4630|Tuesday|1988Q3|N|N|N|2447406|2447649|2447059|2447334|N|N|N|N|N| +2447426|AAAAAAAACEIFFCAA|1988-09-21|1064|4630|356|1988|3|9|21|3|1988|356|4630|Wednesday|1988Q3|N|N|N|2447406|2447649|2447060|2447335|N|N|N|N|N| +2447427|AAAAAAAADEIFFCAA|1988-09-22|1064|4630|356|1988|4|9|22|3|1988|356|4630|Thursday|1988Q3|N|N|N|2447406|2447649|2447061|2447336|N|N|N|N|N| +2447428|AAAAAAAAEEIFFCAA|1988-09-23|1064|4630|356|1988|5|9|23|3|1988|356|4630|Friday|1988Q3|N|Y|N|2447406|2447649|2447062|2447337|N|N|N|N|N| +2447429|AAAAAAAAFEIFFCAA|1988-09-24|1064|4630|356|1988|6|9|24|3|1988|356|4630|Saturday|1988Q3|N|Y|N|2447406|2447649|2447063|2447338|N|N|N|N|N| +2447430|AAAAAAAAGEIFFCAA|1988-09-25|1064|4630|356|1988|0|9|25|3|1988|356|4630|Sunday|1988Q3|N|N|N|2447406|2447649|2447064|2447339|N|N|N|N|N| +2447431|AAAAAAAAHEIFFCAA|1988-09-26|1064|4630|356|1988|1|9|26|3|1988|356|4630|Monday|1988Q3|N|N|N|2447406|2447649|2447065|2447340|N|N|N|N|N| +2447432|AAAAAAAAIEIFFCAA|1988-09-27|1064|4631|356|1988|2|9|27|3|1988|356|4631|Tuesday|1988Q3|N|N|N|2447406|2447649|2447066|2447341|N|N|N|N|N| +2447433|AAAAAAAAJEIFFCAA|1988-09-28|1064|4631|356|1988|3|9|28|3|1988|356|4631|Wednesday|1988Q3|N|N|N|2447406|2447649|2447067|2447342|N|N|N|N|N| +2447434|AAAAAAAAKEIFFCAA|1988-09-29|1064|4631|356|1988|4|9|29|3|1988|356|4631|Thursday|1988Q3|N|N|N|2447406|2447649|2447068|2447343|N|N|N|N|N| +2447435|AAAAAAAALEIFFCAA|1988-09-30|1064|4631|356|1988|5|9|30|3|1988|356|4631|Friday|1988Q3|N|Y|N|2447406|2447649|2447069|2447344|N|N|N|N|N| +2447436|AAAAAAAAMEIFFCAA|1988-10-01|1065|4631|356|1988|6|10|1|4|1988|356|4631|Saturday|1988Q4|N|Y|N|2447436|2447709|2447070|2447344|N|N|N|N|N| +2447437|AAAAAAAANEIFFCAA|1988-10-02|1065|4631|356|1988|0|10|2|4|1988|356|4631|Sunday|1988Q4|N|N|N|2447436|2447709|2447071|2447345|N|N|N|N|N| +2447438|AAAAAAAAOEIFFCAA|1988-10-03|1065|4631|356|1988|1|10|3|4|1988|356|4631|Monday|1988Q4|N|N|N|2447436|2447709|2447072|2447346|N|N|N|N|N| +2447439|AAAAAAAAPEIFFCAA|1988-10-04|1065|4632|356|1988|2|10|4|4|1988|356|4632|Tuesday|1988Q4|N|N|N|2447436|2447709|2447073|2447347|N|N|N|N|N| +2447440|AAAAAAAAAFIFFCAA|1988-10-05|1065|4632|356|1988|3|10|5|4|1988|356|4632|Wednesday|1988Q4|N|N|N|2447436|2447709|2447074|2447348|N|N|N|N|N| +2447441|AAAAAAAABFIFFCAA|1988-10-06|1065|4632|356|1988|4|10|6|4|1988|356|4632|Thursday|1988Q4|N|N|N|2447436|2447709|2447075|2447349|N|N|N|N|N| +2447442|AAAAAAAACFIFFCAA|1988-10-07|1065|4632|356|1988|5|10|7|4|1988|356|4632|Friday|1988Q4|N|Y|N|2447436|2447709|2447076|2447350|N|N|N|N|N| +2447443|AAAAAAAADFIFFCAA|1988-10-08|1065|4632|356|1988|6|10|8|4|1988|356|4632|Saturday|1988Q4|N|Y|N|2447436|2447709|2447077|2447351|N|N|N|N|N| +2447444|AAAAAAAAEFIFFCAA|1988-10-09|1065|4632|356|1988|0|10|9|4|1988|356|4632|Sunday|1988Q4|N|N|N|2447436|2447709|2447078|2447352|N|N|N|N|N| +2447445|AAAAAAAAFFIFFCAA|1988-10-10|1065|4632|356|1988|1|10|10|4|1988|356|4632|Monday|1988Q4|N|N|N|2447436|2447709|2447079|2447353|N|N|N|N|N| +2447446|AAAAAAAAGFIFFCAA|1988-10-11|1065|4633|356|1988|2|10|11|4|1988|356|4633|Tuesday|1988Q4|N|N|N|2447436|2447709|2447080|2447354|N|N|N|N|N| +2447447|AAAAAAAAHFIFFCAA|1988-10-12|1065|4633|356|1988|3|10|12|4|1988|356|4633|Wednesday|1988Q4|N|N|N|2447436|2447709|2447081|2447355|N|N|N|N|N| +2447448|AAAAAAAAIFIFFCAA|1988-10-13|1065|4633|356|1988|4|10|13|4|1988|356|4633|Thursday|1988Q4|N|N|N|2447436|2447709|2447082|2447356|N|N|N|N|N| +2447449|AAAAAAAAJFIFFCAA|1988-10-14|1065|4633|356|1988|5|10|14|4|1988|356|4633|Friday|1988Q4|N|Y|N|2447436|2447709|2447083|2447357|N|N|N|N|N| +2447450|AAAAAAAAKFIFFCAA|1988-10-15|1065|4633|356|1988|6|10|15|4|1988|356|4633|Saturday|1988Q4|N|Y|N|2447436|2447709|2447084|2447358|N|N|N|N|N| +2447451|AAAAAAAALFIFFCAA|1988-10-16|1065|4633|356|1988|0|10|16|4|1988|356|4633|Sunday|1988Q4|N|N|N|2447436|2447709|2447085|2447359|N|N|N|N|N| +2447452|AAAAAAAAMFIFFCAA|1988-10-17|1065|4633|356|1988|1|10|17|4|1988|356|4633|Monday|1988Q4|N|N|N|2447436|2447709|2447086|2447360|N|N|N|N|N| +2447453|AAAAAAAANFIFFCAA|1988-10-18|1065|4634|356|1988|2|10|18|4|1988|356|4634|Tuesday|1988Q4|N|N|N|2447436|2447709|2447087|2447361|N|N|N|N|N| +2447454|AAAAAAAAOFIFFCAA|1988-10-19|1065|4634|356|1988|3|10|19|4|1988|356|4634|Wednesday|1988Q4|N|N|N|2447436|2447709|2447088|2447362|N|N|N|N|N| +2447455|AAAAAAAAPFIFFCAA|1988-10-20|1065|4634|356|1988|4|10|20|4|1988|356|4634|Thursday|1988Q4|N|N|N|2447436|2447709|2447089|2447363|N|N|N|N|N| +2447456|AAAAAAAAAGIFFCAA|1988-10-21|1065|4634|356|1988|5|10|21|4|1988|356|4634|Friday|1988Q4|N|Y|N|2447436|2447709|2447090|2447364|N|N|N|N|N| +2447457|AAAAAAAABGIFFCAA|1988-10-22|1065|4634|356|1988|6|10|22|4|1988|356|4634|Saturday|1988Q4|N|Y|N|2447436|2447709|2447091|2447365|N|N|N|N|N| +2447458|AAAAAAAACGIFFCAA|1988-10-23|1065|4634|356|1988|0|10|23|4|1988|356|4634|Sunday|1988Q4|N|N|N|2447436|2447709|2447092|2447366|N|N|N|N|N| +2447459|AAAAAAAADGIFFCAA|1988-10-24|1065|4634|356|1988|1|10|24|4|1988|356|4634|Monday|1988Q4|N|N|N|2447436|2447709|2447093|2447367|N|N|N|N|N| +2447460|AAAAAAAAEGIFFCAA|1988-10-25|1065|4635|356|1988|2|10|25|4|1988|356|4635|Tuesday|1988Q4|N|N|N|2447436|2447709|2447094|2447368|N|N|N|N|N| +2447461|AAAAAAAAFGIFFCAA|1988-10-26|1065|4635|356|1988|3|10|26|4|1988|356|4635|Wednesday|1988Q4|N|N|N|2447436|2447709|2447095|2447369|N|N|N|N|N| +2447462|AAAAAAAAGGIFFCAA|1988-10-27|1065|4635|356|1988|4|10|27|4|1988|356|4635|Thursday|1988Q4|N|N|N|2447436|2447709|2447096|2447370|N|N|N|N|N| +2447463|AAAAAAAAHGIFFCAA|1988-10-28|1065|4635|356|1988|5|10|28|4|1988|356|4635|Friday|1988Q4|N|Y|N|2447436|2447709|2447097|2447371|N|N|N|N|N| +2447464|AAAAAAAAIGIFFCAA|1988-10-29|1065|4635|356|1988|6|10|29|4|1988|356|4635|Saturday|1988Q4|N|Y|N|2447436|2447709|2447098|2447372|N|N|N|N|N| +2447465|AAAAAAAAJGIFFCAA|1988-10-30|1065|4635|356|1988|0|10|30|4|1988|356|4635|Sunday|1988Q4|N|N|N|2447436|2447709|2447099|2447373|N|N|N|N|N| +2447466|AAAAAAAAKGIFFCAA|1988-10-31|1065|4635|356|1988|1|10|31|4|1988|356|4635|Monday|1988Q4|N|N|N|2447436|2447709|2447100|2447374|N|N|N|N|N| +2447467|AAAAAAAALGIFFCAA|1988-11-01|1066|4636|356|1988|2|11|1|4|1988|356|4636|Tuesday|1988Q4|N|N|N|2447467|2447771|2447101|2447375|N|N|N|N|N| +2447468|AAAAAAAAMGIFFCAA|1988-11-02|1066|4636|356|1988|3|11|2|4|1988|356|4636|Wednesday|1988Q4|N|N|N|2447467|2447771|2447102|2447376|N|N|N|N|N| +2447469|AAAAAAAANGIFFCAA|1988-11-03|1066|4636|356|1988|4|11|3|4|1988|356|4636|Thursday|1988Q4|N|N|N|2447467|2447771|2447103|2447377|N|N|N|N|N| +2447470|AAAAAAAAOGIFFCAA|1988-11-04|1066|4636|356|1988|5|11|4|4|1988|356|4636|Friday|1988Q4|N|Y|N|2447467|2447771|2447104|2447378|N|N|N|N|N| +2447471|AAAAAAAAPGIFFCAA|1988-11-05|1066|4636|356|1988|6|11|5|4|1988|356|4636|Saturday|1988Q4|N|Y|N|2447467|2447771|2447105|2447379|N|N|N|N|N| +2447472|AAAAAAAAAHIFFCAA|1988-11-06|1066|4636|356|1988|0|11|6|4|1988|356|4636|Sunday|1988Q4|N|N|N|2447467|2447771|2447106|2447380|N|N|N|N|N| +2447473|AAAAAAAABHIFFCAA|1988-11-07|1066|4636|356|1988|1|11|7|4|1988|356|4636|Monday|1988Q4|N|N|N|2447467|2447771|2447107|2447381|N|N|N|N|N| +2447474|AAAAAAAACHIFFCAA|1988-11-08|1066|4637|356|1988|2|11|8|4|1988|356|4637|Tuesday|1988Q4|N|N|N|2447467|2447771|2447108|2447382|N|N|N|N|N| +2447475|AAAAAAAADHIFFCAA|1988-11-09|1066|4637|356|1988|3|11|9|4|1988|356|4637|Wednesday|1988Q4|N|N|N|2447467|2447771|2447109|2447383|N|N|N|N|N| +2447476|AAAAAAAAEHIFFCAA|1988-11-10|1066|4637|356|1988|4|11|10|4|1988|356|4637|Thursday|1988Q4|N|N|N|2447467|2447771|2447110|2447384|N|N|N|N|N| +2447477|AAAAAAAAFHIFFCAA|1988-11-11|1066|4637|356|1988|5|11|11|4|1988|356|4637|Friday|1988Q4|N|Y|N|2447467|2447771|2447111|2447385|N|N|N|N|N| +2447478|AAAAAAAAGHIFFCAA|1988-11-12|1066|4637|356|1988|6|11|12|4|1988|356|4637|Saturday|1988Q4|N|Y|N|2447467|2447771|2447112|2447386|N|N|N|N|N| +2447479|AAAAAAAAHHIFFCAA|1988-11-13|1066|4637|356|1988|0|11|13|4|1988|356|4637|Sunday|1988Q4|N|N|N|2447467|2447771|2447113|2447387|N|N|N|N|N| +2447480|AAAAAAAAIHIFFCAA|1988-11-14|1066|4637|356|1988|1|11|14|4|1988|356|4637|Monday|1988Q4|N|N|N|2447467|2447771|2447114|2447388|N|N|N|N|N| +2447481|AAAAAAAAJHIFFCAA|1988-11-15|1066|4638|356|1988|2|11|15|4|1988|356|4638|Tuesday|1988Q4|N|N|N|2447467|2447771|2447115|2447389|N|N|N|N|N| +2447482|AAAAAAAAKHIFFCAA|1988-11-16|1066|4638|356|1988|3|11|16|4|1988|356|4638|Wednesday|1988Q4|N|N|N|2447467|2447771|2447116|2447390|N|N|N|N|N| +2447483|AAAAAAAALHIFFCAA|1988-11-17|1066|4638|356|1988|4|11|17|4|1988|356|4638|Thursday|1988Q4|N|N|N|2447467|2447771|2447117|2447391|N|N|N|N|N| +2447484|AAAAAAAAMHIFFCAA|1988-11-18|1066|4638|356|1988|5|11|18|4|1988|356|4638|Friday|1988Q4|N|Y|N|2447467|2447771|2447118|2447392|N|N|N|N|N| +2447485|AAAAAAAANHIFFCAA|1988-11-19|1066|4638|356|1988|6|11|19|4|1988|356|4638|Saturday|1988Q4|N|Y|N|2447467|2447771|2447119|2447393|N|N|N|N|N| +2447486|AAAAAAAAOHIFFCAA|1988-11-20|1066|4638|356|1988|0|11|20|4|1988|356|4638|Sunday|1988Q4|N|N|N|2447467|2447771|2447120|2447394|N|N|N|N|N| +2447487|AAAAAAAAPHIFFCAA|1988-11-21|1066|4638|356|1988|1|11|21|4|1988|356|4638|Monday|1988Q4|N|N|N|2447467|2447771|2447121|2447395|N|N|N|N|N| +2447488|AAAAAAAAAIIFFCAA|1988-11-22|1066|4639|356|1988|2|11|22|4|1988|356|4639|Tuesday|1988Q4|N|N|N|2447467|2447771|2447122|2447396|N|N|N|N|N| +2447489|AAAAAAAABIIFFCAA|1988-11-23|1066|4639|356|1988|3|11|23|4|1988|356|4639|Wednesday|1988Q4|N|N|N|2447467|2447771|2447123|2447397|N|N|N|N|N| +2447490|AAAAAAAACIIFFCAA|1988-11-24|1066|4639|356|1988|4|11|24|4|1988|356|4639|Thursday|1988Q4|N|N|N|2447467|2447771|2447124|2447398|N|N|N|N|N| +2447491|AAAAAAAADIIFFCAA|1988-11-25|1066|4639|356|1988|5|11|25|4|1988|356|4639|Friday|1988Q4|N|Y|N|2447467|2447771|2447125|2447399|N|N|N|N|N| +2447492|AAAAAAAAEIIFFCAA|1988-11-26|1066|4639|356|1988|6|11|26|4|1988|356|4639|Saturday|1988Q4|N|Y|N|2447467|2447771|2447126|2447400|N|N|N|N|N| +2447493|AAAAAAAAFIIFFCAA|1988-11-27|1066|4639|356|1988|0|11|27|4|1988|356|4639|Sunday|1988Q4|N|N|N|2447467|2447771|2447127|2447401|N|N|N|N|N| +2447494|AAAAAAAAGIIFFCAA|1988-11-28|1066|4639|356|1988|1|11|28|4|1988|356|4639|Monday|1988Q4|N|N|N|2447467|2447771|2447128|2447402|N|N|N|N|N| +2447495|AAAAAAAAHIIFFCAA|1988-11-29|1066|4640|356|1988|2|11|29|4|1988|356|4640|Tuesday|1988Q4|N|N|N|2447467|2447771|2447129|2447403|N|N|N|N|N| +2447496|AAAAAAAAIIIFFCAA|1988-11-30|1066|4640|356|1988|3|11|30|4|1988|356|4640|Wednesday|1988Q4|N|N|N|2447467|2447771|2447130|2447404|N|N|N|N|N| +2447497|AAAAAAAAJIIFFCAA|1988-12-01|1067|4640|357|1988|4|12|1|4|1988|357|4640|Thursday|1988Q4|N|N|N|2447497|2447831|2447131|2447405|N|N|N|N|N| +2447498|AAAAAAAAKIIFFCAA|1988-12-02|1067|4640|357|1988|5|12|2|4|1988|357|4640|Friday|1988Q4|N|Y|N|2447497|2447831|2447132|2447406|N|N|N|N|N| +2447499|AAAAAAAALIIFFCAA|1988-12-03|1067|4640|357|1988|6|12|3|4|1988|357|4640|Saturday|1988Q4|N|Y|N|2447497|2447831|2447133|2447407|N|N|N|N|N| +2447500|AAAAAAAAMIIFFCAA|1988-12-04|1067|4640|357|1988|0|12|4|4|1988|357|4640|Sunday|1988Q4|N|N|N|2447497|2447831|2447134|2447408|N|N|N|N|N| +2447501|AAAAAAAANIIFFCAA|1988-12-05|1067|4640|357|1988|1|12|5|4|1988|357|4640|Monday|1988Q4|N|N|N|2447497|2447831|2447135|2447409|N|N|N|N|N| +2447502|AAAAAAAAOIIFFCAA|1988-12-06|1067|4641|357|1988|2|12|6|4|1988|357|4641|Tuesday|1988Q4|N|N|N|2447497|2447831|2447136|2447410|N|N|N|N|N| +2447503|AAAAAAAAPIIFFCAA|1988-12-07|1067|4641|357|1988|3|12|7|4|1988|357|4641|Wednesday|1988Q4|N|N|N|2447497|2447831|2447137|2447411|N|N|N|N|N| +2447504|AAAAAAAAAJIFFCAA|1988-12-08|1067|4641|357|1988|4|12|8|4|1988|357|4641|Thursday|1988Q4|N|N|N|2447497|2447831|2447138|2447412|N|N|N|N|N| +2447505|AAAAAAAABJIFFCAA|1988-12-09|1067|4641|357|1988|5|12|9|4|1988|357|4641|Friday|1988Q4|N|Y|N|2447497|2447831|2447139|2447413|N|N|N|N|N| +2447506|AAAAAAAACJIFFCAA|1988-12-10|1067|4641|357|1988|6|12|10|4|1988|357|4641|Saturday|1988Q4|N|Y|N|2447497|2447831|2447140|2447414|N|N|N|N|N| +2447507|AAAAAAAADJIFFCAA|1988-12-11|1067|4641|357|1988|0|12|11|4|1988|357|4641|Sunday|1988Q4|N|N|N|2447497|2447831|2447141|2447415|N|N|N|N|N| +2447508|AAAAAAAAEJIFFCAA|1988-12-12|1067|4641|357|1988|1|12|12|4|1988|357|4641|Monday|1988Q4|N|N|N|2447497|2447831|2447142|2447416|N|N|N|N|N| +2447509|AAAAAAAAFJIFFCAA|1988-12-13|1067|4642|357|1988|2|12|13|4|1988|357|4642|Tuesday|1988Q4|N|N|N|2447497|2447831|2447143|2447417|N|N|N|N|N| +2447510|AAAAAAAAGJIFFCAA|1988-12-14|1067|4642|357|1988|3|12|14|4|1988|357|4642|Wednesday|1988Q4|N|N|N|2447497|2447831|2447144|2447418|N|N|N|N|N| +2447511|AAAAAAAAHJIFFCAA|1988-12-15|1067|4642|357|1988|4|12|15|4|1988|357|4642|Thursday|1988Q4|N|N|N|2447497|2447831|2447145|2447419|N|N|N|N|N| +2447512|AAAAAAAAIJIFFCAA|1988-12-16|1067|4642|357|1988|5|12|16|4|1988|357|4642|Friday|1988Q4|N|Y|N|2447497|2447831|2447146|2447420|N|N|N|N|N| +2447513|AAAAAAAAJJIFFCAA|1988-12-17|1067|4642|357|1988|6|12|17|4|1988|357|4642|Saturday|1988Q4|N|Y|N|2447497|2447831|2447147|2447421|N|N|N|N|N| +2447514|AAAAAAAAKJIFFCAA|1988-12-18|1067|4642|357|1988|0|12|18|4|1988|357|4642|Sunday|1988Q4|N|N|N|2447497|2447831|2447148|2447422|N|N|N|N|N| +2447515|AAAAAAAALJIFFCAA|1988-12-19|1067|4642|357|1988|1|12|19|4|1988|357|4642|Monday|1988Q4|N|N|N|2447497|2447831|2447149|2447423|N|N|N|N|N| +2447516|AAAAAAAAMJIFFCAA|1988-12-20|1067|4643|357|1988|2|12|20|4|1988|357|4643|Tuesday|1988Q4|N|N|N|2447497|2447831|2447150|2447424|N|N|N|N|N| +2447517|AAAAAAAANJIFFCAA|1988-12-21|1067|4643|357|1988|3|12|21|4|1988|357|4643|Wednesday|1988Q4|N|N|N|2447497|2447831|2447151|2447425|N|N|N|N|N| +2447518|AAAAAAAAOJIFFCAA|1988-12-22|1067|4643|357|1988|4|12|22|4|1988|357|4643|Thursday|1988Q4|N|N|N|2447497|2447831|2447152|2447426|N|N|N|N|N| +2447519|AAAAAAAAPJIFFCAA|1988-12-23|1067|4643|357|1988|5|12|23|4|1988|357|4643|Friday|1988Q4|N|Y|N|2447497|2447831|2447153|2447427|N|N|N|N|N| +2447520|AAAAAAAAAKIFFCAA|1988-12-24|1067|4643|357|1988|6|12|24|4|1988|357|4643|Saturday|1988Q4|N|Y|N|2447497|2447831|2447154|2447428|N|N|N|N|N| +2447521|AAAAAAAABKIFFCAA|1988-12-25|1067|4643|357|1988|0|12|25|4|1988|357|4643|Sunday|1988Q4|Y|N|N|2447497|2447831|2447155|2447429|N|N|N|N|N| +2447522|AAAAAAAACKIFFCAA|1988-12-26|1067|4643|357|1988|1|12|26|4|1988|357|4643|Monday|1988Q4|N|N|Y|2447497|2447831|2447156|2447430|N|N|N|N|N| +2447523|AAAAAAAADKIFFCAA|1988-12-27|1067|4644|357|1988|2|12|27|4|1988|357|4644|Tuesday|1988Q4|N|N|N|2447497|2447831|2447157|2447431|N|N|N|N|N| +2447524|AAAAAAAAEKIFFCAA|1988-12-28|1067|4644|357|1988|3|12|28|4|1988|357|4644|Wednesday|1988Q4|N|N|N|2447497|2447831|2447158|2447432|N|N|N|N|N| +2447525|AAAAAAAAFKIFFCAA|1988-12-29|1067|4644|357|1988|4|12|29|4|1988|357|4644|Thursday|1988Q4|N|N|N|2447497|2447831|2447159|2447433|N|N|N|N|N| +2447526|AAAAAAAAGKIFFCAA|1988-12-30|1067|4644|357|1988|5|12|30|4|1988|357|4644|Friday|1988Q4|N|Y|N|2447497|2447831|2447160|2447434|N|N|N|N|N| +2447527|AAAAAAAAHKIFFCAA|1988-12-31|1067|4644|357|1988|6|12|31|4|1988|357|4644|Saturday|1988Q4|Y|Y|N|2447497|2447831|2447161|2447435|N|N|N|N|N| +2447528|AAAAAAAAIKIFFCAA|1989-01-01|1068|4644|357|1989|0|1|1|1|1989|357|4644|Sunday|1989Q1|Y|N|Y|2447528|2447527|2447162|2447436|N|N|N|N|N| +2447529|AAAAAAAAJKIFFCAA|1989-01-02|1068|4644|357|1989|1|1|2|1|1989|357|4644|Monday|1989Q1|N|N|Y|2447528|2447527|2447163|2447437|N|N|N|N|N| +2447530|AAAAAAAAKKIFFCAA|1989-01-03|1068|4645|357|1989|2|1|3|1|1989|357|4645|Tuesday|1989Q1|N|N|N|2447528|2447527|2447164|2447438|N|N|N|N|N| +2447531|AAAAAAAALKIFFCAA|1989-01-04|1068|4645|357|1989|3|1|4|1|1989|357|4645|Wednesday|1989Q1|N|N|N|2447528|2447527|2447165|2447439|N|N|N|N|N| +2447532|AAAAAAAAMKIFFCAA|1989-01-05|1068|4645|357|1989|4|1|5|1|1989|357|4645|Thursday|1989Q1|N|N|N|2447528|2447527|2447166|2447440|N|N|N|N|N| +2447533|AAAAAAAANKIFFCAA|1989-01-06|1068|4645|357|1989|5|1|6|1|1989|357|4645|Friday|1989Q1|N|Y|N|2447528|2447527|2447167|2447441|N|N|N|N|N| +2447534|AAAAAAAAOKIFFCAA|1989-01-07|1068|4645|357|1989|6|1|7|1|1989|357|4645|Saturday|1989Q1|N|Y|N|2447528|2447527|2447168|2447442|N|N|N|N|N| +2447535|AAAAAAAAPKIFFCAA|1989-01-08|1068|4645|357|1989|0|1|8|1|1989|357|4645|Sunday|1989Q1|N|N|N|2447528|2447527|2447169|2447443|N|N|N|N|N| +2447536|AAAAAAAAALIFFCAA|1989-01-09|1068|4645|357|1989|1|1|9|1|1989|357|4645|Monday|1989Q1|N|N|N|2447528|2447527|2447170|2447444|N|N|N|N|N| +2447537|AAAAAAAABLIFFCAA|1989-01-10|1068|4646|357|1989|2|1|10|1|1989|357|4646|Tuesday|1989Q1|N|N|N|2447528|2447527|2447171|2447445|N|N|N|N|N| +2447538|AAAAAAAACLIFFCAA|1989-01-11|1068|4646|357|1989|3|1|11|1|1989|357|4646|Wednesday|1989Q1|N|N|N|2447528|2447527|2447172|2447446|N|N|N|N|N| +2447539|AAAAAAAADLIFFCAA|1989-01-12|1068|4646|357|1989|4|1|12|1|1989|357|4646|Thursday|1989Q1|N|N|N|2447528|2447527|2447173|2447447|N|N|N|N|N| +2447540|AAAAAAAAELIFFCAA|1989-01-13|1068|4646|357|1989|5|1|13|1|1989|357|4646|Friday|1989Q1|N|Y|N|2447528|2447527|2447174|2447448|N|N|N|N|N| +2447541|AAAAAAAAFLIFFCAA|1989-01-14|1068|4646|357|1989|6|1|14|1|1989|357|4646|Saturday|1989Q1|N|Y|N|2447528|2447527|2447175|2447449|N|N|N|N|N| +2447542|AAAAAAAAGLIFFCAA|1989-01-15|1068|4646|357|1989|0|1|15|1|1989|357|4646|Sunday|1989Q1|N|N|N|2447528|2447527|2447176|2447450|N|N|N|N|N| +2447543|AAAAAAAAHLIFFCAA|1989-01-16|1068|4646|357|1989|1|1|16|1|1989|357|4646|Monday|1989Q1|N|N|N|2447528|2447527|2447177|2447451|N|N|N|N|N| +2447544|AAAAAAAAILIFFCAA|1989-01-17|1068|4647|357|1989|2|1|17|1|1989|357|4647|Tuesday|1989Q1|N|N|N|2447528|2447527|2447178|2447452|N|N|N|N|N| +2447545|AAAAAAAAJLIFFCAA|1989-01-18|1068|4647|357|1989|3|1|18|1|1989|357|4647|Wednesday|1989Q1|N|N|N|2447528|2447527|2447179|2447453|N|N|N|N|N| +2447546|AAAAAAAAKLIFFCAA|1989-01-19|1068|4647|357|1989|4|1|19|1|1989|357|4647|Thursday|1989Q1|N|N|N|2447528|2447527|2447180|2447454|N|N|N|N|N| +2447547|AAAAAAAALLIFFCAA|1989-01-20|1068|4647|357|1989|5|1|20|1|1989|357|4647|Friday|1989Q1|N|Y|N|2447528|2447527|2447181|2447455|N|N|N|N|N| +2447548|AAAAAAAAMLIFFCAA|1989-01-21|1068|4647|357|1989|6|1|21|1|1989|357|4647|Saturday|1989Q1|N|Y|N|2447528|2447527|2447182|2447456|N|N|N|N|N| +2447549|AAAAAAAANLIFFCAA|1989-01-22|1068|4647|357|1989|0|1|22|1|1989|357|4647|Sunday|1989Q1|N|N|N|2447528|2447527|2447183|2447457|N|N|N|N|N| +2447550|AAAAAAAAOLIFFCAA|1989-01-23|1068|4647|357|1989|1|1|23|1|1989|357|4647|Monday|1989Q1|N|N|N|2447528|2447527|2447184|2447458|N|N|N|N|N| +2447551|AAAAAAAAPLIFFCAA|1989-01-24|1068|4648|357|1989|2|1|24|1|1989|357|4648|Tuesday|1989Q1|N|N|N|2447528|2447527|2447185|2447459|N|N|N|N|N| +2447552|AAAAAAAAAMIFFCAA|1989-01-25|1068|4648|357|1989|3|1|25|1|1989|357|4648|Wednesday|1989Q1|N|N|N|2447528|2447527|2447186|2447460|N|N|N|N|N| +2447553|AAAAAAAABMIFFCAA|1989-01-26|1068|4648|357|1989|4|1|26|1|1989|357|4648|Thursday|1989Q1|N|N|N|2447528|2447527|2447187|2447461|N|N|N|N|N| +2447554|AAAAAAAACMIFFCAA|1989-01-27|1068|4648|357|1989|5|1|27|1|1989|357|4648|Friday|1989Q1|N|Y|N|2447528|2447527|2447188|2447462|N|N|N|N|N| +2447555|AAAAAAAADMIFFCAA|1989-01-28|1068|4648|357|1989|6|1|28|1|1989|357|4648|Saturday|1989Q1|N|Y|N|2447528|2447527|2447189|2447463|N|N|N|N|N| +2447556|AAAAAAAAEMIFFCAA|1989-01-29|1068|4648|357|1989|0|1|29|1|1989|357|4648|Sunday|1989Q1|N|N|N|2447528|2447527|2447190|2447464|N|N|N|N|N| +2447557|AAAAAAAAFMIFFCAA|1989-01-30|1068|4648|357|1989|1|1|30|1|1989|357|4648|Monday|1989Q1|N|N|N|2447528|2447527|2447191|2447465|N|N|N|N|N| +2447558|AAAAAAAAGMIFFCAA|1989-01-31|1068|4649|357|1989|2|1|31|1|1989|357|4649|Tuesday|1989Q1|N|N|N|2447528|2447527|2447192|2447466|N|N|N|N|N| +2447559|AAAAAAAAHMIFFCAA|1989-02-01|1069|4649|357|1989|3|2|1|1|1989|357|4649|Wednesday|1989Q1|N|N|N|2447559|2447589|2447193|2447467|N|N|N|N|N| +2447560|AAAAAAAAIMIFFCAA|1989-02-02|1069|4649|357|1989|4|2|2|1|1989|357|4649|Thursday|1989Q1|N|N|N|2447559|2447589|2447194|2447468|N|N|N|N|N| +2447561|AAAAAAAAJMIFFCAA|1989-02-03|1069|4649|357|1989|5|2|3|1|1989|357|4649|Friday|1989Q1|N|Y|N|2447559|2447589|2447195|2447469|N|N|N|N|N| +2447562|AAAAAAAAKMIFFCAA|1989-02-04|1069|4649|357|1989|6|2|4|1|1989|357|4649|Saturday|1989Q1|N|Y|N|2447559|2447589|2447196|2447470|N|N|N|N|N| +2447563|AAAAAAAALMIFFCAA|1989-02-05|1069|4649|357|1989|0|2|5|1|1989|357|4649|Sunday|1989Q1|N|N|N|2447559|2447589|2447197|2447471|N|N|N|N|N| +2447564|AAAAAAAAMMIFFCAA|1989-02-06|1069|4649|357|1989|1|2|6|1|1989|357|4649|Monday|1989Q1|N|N|N|2447559|2447589|2447198|2447472|N|N|N|N|N| +2447565|AAAAAAAANMIFFCAA|1989-02-07|1069|4650|357|1989|2|2|7|1|1989|357|4650|Tuesday|1989Q1|N|N|N|2447559|2447589|2447199|2447473|N|N|N|N|N| +2447566|AAAAAAAAOMIFFCAA|1989-02-08|1069|4650|357|1989|3|2|8|1|1989|357|4650|Wednesday|1989Q1|N|N|N|2447559|2447589|2447200|2447474|N|N|N|N|N| +2447567|AAAAAAAAPMIFFCAA|1989-02-09|1069|4650|357|1989|4|2|9|1|1989|357|4650|Thursday|1989Q1|N|N|N|2447559|2447589|2447201|2447475|N|N|N|N|N| +2447568|AAAAAAAAANIFFCAA|1989-02-10|1069|4650|357|1989|5|2|10|1|1989|357|4650|Friday|1989Q1|N|Y|N|2447559|2447589|2447202|2447476|N|N|N|N|N| +2447569|AAAAAAAABNIFFCAA|1989-02-11|1069|4650|357|1989|6|2|11|1|1989|357|4650|Saturday|1989Q1|N|Y|N|2447559|2447589|2447203|2447477|N|N|N|N|N| +2447570|AAAAAAAACNIFFCAA|1989-02-12|1069|4650|357|1989|0|2|12|1|1989|357|4650|Sunday|1989Q1|N|N|N|2447559|2447589|2447204|2447478|N|N|N|N|N| +2447571|AAAAAAAADNIFFCAA|1989-02-13|1069|4650|357|1989|1|2|13|1|1989|357|4650|Monday|1989Q1|N|N|N|2447559|2447589|2447205|2447479|N|N|N|N|N| +2447572|AAAAAAAAENIFFCAA|1989-02-14|1069|4651|357|1989|2|2|14|1|1989|357|4651|Tuesday|1989Q1|N|N|N|2447559|2447589|2447206|2447480|N|N|N|N|N| +2447573|AAAAAAAAFNIFFCAA|1989-02-15|1069|4651|357|1989|3|2|15|1|1989|357|4651|Wednesday|1989Q1|N|N|N|2447559|2447589|2447207|2447481|N|N|N|N|N| +2447574|AAAAAAAAGNIFFCAA|1989-02-16|1069|4651|357|1989|4|2|16|1|1989|357|4651|Thursday|1989Q1|N|N|N|2447559|2447589|2447208|2447482|N|N|N|N|N| +2447575|AAAAAAAAHNIFFCAA|1989-02-17|1069|4651|357|1989|5|2|17|1|1989|357|4651|Friday|1989Q1|N|Y|N|2447559|2447589|2447209|2447483|N|N|N|N|N| +2447576|AAAAAAAAINIFFCAA|1989-02-18|1069|4651|357|1989|6|2|18|1|1989|357|4651|Saturday|1989Q1|N|Y|N|2447559|2447589|2447210|2447484|N|N|N|N|N| +2447577|AAAAAAAAJNIFFCAA|1989-02-19|1069|4651|357|1989|0|2|19|1|1989|357|4651|Sunday|1989Q1|N|N|N|2447559|2447589|2447211|2447485|N|N|N|N|N| +2447578|AAAAAAAAKNIFFCAA|1989-02-20|1069|4651|357|1989|1|2|20|1|1989|357|4651|Monday|1989Q1|N|N|N|2447559|2447589|2447212|2447486|N|N|N|N|N| +2447579|AAAAAAAALNIFFCAA|1989-02-21|1069|4652|357|1989|2|2|21|1|1989|357|4652|Tuesday|1989Q1|N|N|N|2447559|2447589|2447213|2447487|N|N|N|N|N| +2447580|AAAAAAAAMNIFFCAA|1989-02-22|1069|4652|357|1989|3|2|22|1|1989|357|4652|Wednesday|1989Q1|N|N|N|2447559|2447589|2447214|2447488|N|N|N|N|N| +2447581|AAAAAAAANNIFFCAA|1989-02-23|1069|4652|357|1989|4|2|23|1|1989|357|4652|Thursday|1989Q1|N|N|N|2447559|2447589|2447215|2447489|N|N|N|N|N| +2447582|AAAAAAAAONIFFCAA|1989-02-24|1069|4652|357|1989|5|2|24|1|1989|357|4652|Friday|1989Q1|N|Y|N|2447559|2447589|2447216|2447490|N|N|N|N|N| +2447583|AAAAAAAAPNIFFCAA|1989-02-25|1069|4652|357|1989|6|2|25|1|1989|357|4652|Saturday|1989Q1|N|Y|N|2447559|2447589|2447217|2447491|N|N|N|N|N| +2447584|AAAAAAAAAOIFFCAA|1989-02-26|1069|4652|357|1989|0|2|26|1|1989|357|4652|Sunday|1989Q1|N|N|N|2447559|2447589|2447218|2447492|N|N|N|N|N| +2447585|AAAAAAAABOIFFCAA|1989-02-27|1069|4652|357|1989|1|2|27|1|1989|357|4652|Monday|1989Q1|N|N|N|2447559|2447589|2447219|2447493|N|N|N|N|N| +2447586|AAAAAAAACOIFFCAA|1989-02-28|1069|4653|357|1989|2|2|28|1|1989|357|4653|Tuesday|1989Q1|N|N|N|2447559|2447589|2447220|2447494|N|N|N|N|N| +2447587|AAAAAAAADOIFFCAA|1989-03-01|1070|4653|358|1989|3|3|1|1|1989|358|4653|Wednesday|1989Q1|N|N|N|2447587|2447645|2447222|2447495|N|N|N|N|N| +2447588|AAAAAAAAEOIFFCAA|1989-03-02|1070|4653|358|1989|4|3|2|1|1989|358|4653|Thursday|1989Q1|N|N|N|2447587|2447645|2447223|2447496|N|N|N|N|N| +2447589|AAAAAAAAFOIFFCAA|1989-03-03|1070|4653|358|1989|5|3|3|1|1989|358|4653|Friday|1989Q1|N|Y|N|2447587|2447645|2447224|2447497|N|N|N|N|N| +2447590|AAAAAAAAGOIFFCAA|1989-03-04|1070|4653|358|1989|6|3|4|1|1989|358|4653|Saturday|1989Q1|N|Y|N|2447587|2447645|2447225|2447498|N|N|N|N|N| +2447591|AAAAAAAAHOIFFCAA|1989-03-05|1070|4653|358|1989|0|3|5|1|1989|358|4653|Sunday|1989Q1|N|N|N|2447587|2447645|2447226|2447499|N|N|N|N|N| +2447592|AAAAAAAAIOIFFCAA|1989-03-06|1070|4653|358|1989|1|3|6|1|1989|358|4653|Monday|1989Q1|N|N|N|2447587|2447645|2447227|2447500|N|N|N|N|N| +2447593|AAAAAAAAJOIFFCAA|1989-03-07|1070|4654|358|1989|2|3|7|1|1989|358|4654|Tuesday|1989Q1|N|N|N|2447587|2447645|2447228|2447501|N|N|N|N|N| +2447594|AAAAAAAAKOIFFCAA|1989-03-08|1070|4654|358|1989|3|3|8|1|1989|358|4654|Wednesday|1989Q1|N|N|N|2447587|2447645|2447229|2447502|N|N|N|N|N| +2447595|AAAAAAAALOIFFCAA|1989-03-09|1070|4654|358|1989|4|3|9|1|1989|358|4654|Thursday|1989Q1|N|N|N|2447587|2447645|2447230|2447503|N|N|N|N|N| +2447596|AAAAAAAAMOIFFCAA|1989-03-10|1070|4654|358|1989|5|3|10|1|1989|358|4654|Friday|1989Q1|N|Y|N|2447587|2447645|2447231|2447504|N|N|N|N|N| +2447597|AAAAAAAANOIFFCAA|1989-03-11|1070|4654|358|1989|6|3|11|1|1989|358|4654|Saturday|1989Q1|N|Y|N|2447587|2447645|2447232|2447505|N|N|N|N|N| +2447598|AAAAAAAAOOIFFCAA|1989-03-12|1070|4654|358|1989|0|3|12|1|1989|358|4654|Sunday|1989Q1|N|N|N|2447587|2447645|2447233|2447506|N|N|N|N|N| +2447599|AAAAAAAAPOIFFCAA|1989-03-13|1070|4654|358|1989|1|3|13|1|1989|358|4654|Monday|1989Q1|N|N|N|2447587|2447645|2447234|2447507|N|N|N|N|N| +2447600|AAAAAAAAAPIFFCAA|1989-03-14|1070|4655|358|1989|2|3|14|1|1989|358|4655|Tuesday|1989Q1|N|N|N|2447587|2447645|2447235|2447508|N|N|N|N|N| +2447601|AAAAAAAABPIFFCAA|1989-03-15|1070|4655|358|1989|3|3|15|1|1989|358|4655|Wednesday|1989Q1|N|N|N|2447587|2447645|2447236|2447509|N|N|N|N|N| +2447602|AAAAAAAACPIFFCAA|1989-03-16|1070|4655|358|1989|4|3|16|1|1989|358|4655|Thursday|1989Q1|N|N|N|2447587|2447645|2447237|2447510|N|N|N|N|N| +2447603|AAAAAAAADPIFFCAA|1989-03-17|1070|4655|358|1989|5|3|17|1|1989|358|4655|Friday|1989Q1|N|Y|N|2447587|2447645|2447238|2447511|N|N|N|N|N| +2447604|AAAAAAAAEPIFFCAA|1989-03-18|1070|4655|358|1989|6|3|18|1|1989|358|4655|Saturday|1989Q1|N|Y|N|2447587|2447645|2447239|2447512|N|N|N|N|N| +2447605|AAAAAAAAFPIFFCAA|1989-03-19|1070|4655|358|1989|0|3|19|1|1989|358|4655|Sunday|1989Q1|N|N|N|2447587|2447645|2447240|2447513|N|N|N|N|N| +2447606|AAAAAAAAGPIFFCAA|1989-03-20|1070|4655|358|1989|1|3|20|1|1989|358|4655|Monday|1989Q1|N|N|N|2447587|2447645|2447241|2447514|N|N|N|N|N| +2447607|AAAAAAAAHPIFFCAA|1989-03-21|1070|4656|358|1989|2|3|21|1|1989|358|4656|Tuesday|1989Q1|N|N|N|2447587|2447645|2447242|2447515|N|N|N|N|N| +2447608|AAAAAAAAIPIFFCAA|1989-03-22|1070|4656|358|1989|3|3|22|1|1989|358|4656|Wednesday|1989Q1|N|N|N|2447587|2447645|2447243|2447516|N|N|N|N|N| +2447609|AAAAAAAAJPIFFCAA|1989-03-23|1070|4656|358|1989|4|3|23|1|1989|358|4656|Thursday|1989Q1|N|N|N|2447587|2447645|2447244|2447517|N|N|N|N|N| +2447610|AAAAAAAAKPIFFCAA|1989-03-24|1070|4656|358|1989|5|3|24|1|1989|358|4656|Friday|1989Q1|N|Y|N|2447587|2447645|2447245|2447518|N|N|N|N|N| +2447611|AAAAAAAALPIFFCAA|1989-03-25|1070|4656|358|1989|6|3|25|1|1989|358|4656|Saturday|1989Q1|N|Y|N|2447587|2447645|2447246|2447519|N|N|N|N|N| +2447612|AAAAAAAAMPIFFCAA|1989-03-26|1070|4656|358|1989|0|3|26|1|1989|358|4656|Sunday|1989Q1|N|N|N|2447587|2447645|2447247|2447520|N|N|N|N|N| +2447613|AAAAAAAANPIFFCAA|1989-03-27|1070|4656|358|1989|1|3|27|1|1989|358|4656|Monday|1989Q1|N|N|N|2447587|2447645|2447248|2447521|N|N|N|N|N| +2447614|AAAAAAAAOPIFFCAA|1989-03-28|1070|4657|358|1989|2|3|28|1|1989|358|4657|Tuesday|1989Q1|N|N|N|2447587|2447645|2447249|2447522|N|N|N|N|N| +2447615|AAAAAAAAPPIFFCAA|1989-03-29|1070|4657|358|1989|3|3|29|1|1989|358|4657|Wednesday|1989Q1|N|N|N|2447587|2447645|2447250|2447523|N|N|N|N|N| +2447616|AAAAAAAAAAJFFCAA|1989-03-30|1070|4657|358|1989|4|3|30|1|1989|358|4657|Thursday|1989Q1|N|N|N|2447587|2447645|2447251|2447524|N|N|N|N|N| +2447617|AAAAAAAABAJFFCAA|1989-03-31|1070|4657|358|1989|5|3|31|1|1989|358|4657|Friday|1989Q1|N|Y|N|2447587|2447645|2447252|2447525|N|N|N|N|N| +2447618|AAAAAAAACAJFFCAA|1989-04-01|1071|4657|358|1989|6|4|1|1|1989|358|4657|Saturday|1989Q1|N|Y|N|2447618|2447707|2447253|2447528|N|N|N|N|N| +2447619|AAAAAAAADAJFFCAA|1989-04-02|1071|4657|358|1989|0|4|2|2|1989|358|4657|Sunday|1989Q2|N|N|N|2447618|2447707|2447254|2447529|N|N|N|N|N| +2447620|AAAAAAAAEAJFFCAA|1989-04-03|1071|4657|358|1989|1|4|3|2|1989|358|4657|Monday|1989Q2|N|N|N|2447618|2447707|2447255|2447530|N|N|N|N|N| +2447621|AAAAAAAAFAJFFCAA|1989-04-04|1071|4658|358|1989|2|4|4|2|1989|358|4658|Tuesday|1989Q2|N|N|N|2447618|2447707|2447256|2447531|N|N|N|N|N| +2447622|AAAAAAAAGAJFFCAA|1989-04-05|1071|4658|358|1989|3|4|5|2|1989|358|4658|Wednesday|1989Q2|N|N|N|2447618|2447707|2447257|2447532|N|N|N|N|N| +2447623|AAAAAAAAHAJFFCAA|1989-04-06|1071|4658|358|1989|4|4|6|2|1989|358|4658|Thursday|1989Q2|N|N|N|2447618|2447707|2447258|2447533|N|N|N|N|N| +2447624|AAAAAAAAIAJFFCAA|1989-04-07|1071|4658|358|1989|5|4|7|2|1989|358|4658|Friday|1989Q2|N|Y|N|2447618|2447707|2447259|2447534|N|N|N|N|N| +2447625|AAAAAAAAJAJFFCAA|1989-04-08|1071|4658|358|1989|6|4|8|2|1989|358|4658|Saturday|1989Q2|N|Y|N|2447618|2447707|2447260|2447535|N|N|N|N|N| +2447626|AAAAAAAAKAJFFCAA|1989-04-09|1071|4658|358|1989|0|4|9|2|1989|358|4658|Sunday|1989Q2|N|N|N|2447618|2447707|2447261|2447536|N|N|N|N|N| +2447627|AAAAAAAALAJFFCAA|1989-04-10|1071|4658|358|1989|1|4|10|2|1989|358|4658|Monday|1989Q2|N|N|N|2447618|2447707|2447262|2447537|N|N|N|N|N| +2447628|AAAAAAAAMAJFFCAA|1989-04-11|1071|4659|358|1989|2|4|11|2|1989|358|4659|Tuesday|1989Q2|N|N|N|2447618|2447707|2447263|2447538|N|N|N|N|N| +2447629|AAAAAAAANAJFFCAA|1989-04-12|1071|4659|358|1989|3|4|12|2|1989|358|4659|Wednesday|1989Q2|N|N|N|2447618|2447707|2447264|2447539|N|N|N|N|N| +2447630|AAAAAAAAOAJFFCAA|1989-04-13|1071|4659|358|1989|4|4|13|2|1989|358|4659|Thursday|1989Q2|N|N|N|2447618|2447707|2447265|2447540|N|N|N|N|N| +2447631|AAAAAAAAPAJFFCAA|1989-04-14|1071|4659|358|1989|5|4|14|2|1989|358|4659|Friday|1989Q2|N|Y|N|2447618|2447707|2447266|2447541|N|N|N|N|N| +2447632|AAAAAAAAABJFFCAA|1989-04-15|1071|4659|358|1989|6|4|15|2|1989|358|4659|Saturday|1989Q2|N|Y|N|2447618|2447707|2447267|2447542|N|N|N|N|N| +2447633|AAAAAAAABBJFFCAA|1989-04-16|1071|4659|358|1989|0|4|16|2|1989|358|4659|Sunday|1989Q2|N|N|N|2447618|2447707|2447268|2447543|N|N|N|N|N| +2447634|AAAAAAAACBJFFCAA|1989-04-17|1071|4659|358|1989|1|4|17|2|1989|358|4659|Monday|1989Q2|N|N|N|2447618|2447707|2447269|2447544|N|N|N|N|N| +2447635|AAAAAAAADBJFFCAA|1989-04-18|1071|4660|358|1989|2|4|18|2|1989|358|4660|Tuesday|1989Q2|N|N|N|2447618|2447707|2447270|2447545|N|N|N|N|N| +2447636|AAAAAAAAEBJFFCAA|1989-04-19|1071|4660|358|1989|3|4|19|2|1989|358|4660|Wednesday|1989Q2|N|N|N|2447618|2447707|2447271|2447546|N|N|N|N|N| +2447637|AAAAAAAAFBJFFCAA|1989-04-20|1071|4660|358|1989|4|4|20|2|1989|358|4660|Thursday|1989Q2|N|N|N|2447618|2447707|2447272|2447547|N|N|N|N|N| +2447638|AAAAAAAAGBJFFCAA|1989-04-21|1071|4660|358|1989|5|4|21|2|1989|358|4660|Friday|1989Q2|N|Y|N|2447618|2447707|2447273|2447548|N|N|N|N|N| +2447639|AAAAAAAAHBJFFCAA|1989-04-22|1071|4660|358|1989|6|4|22|2|1989|358|4660|Saturday|1989Q2|N|Y|N|2447618|2447707|2447274|2447549|N|N|N|N|N| +2447640|AAAAAAAAIBJFFCAA|1989-04-23|1071|4660|358|1989|0|4|23|2|1989|358|4660|Sunday|1989Q2|N|N|N|2447618|2447707|2447275|2447550|N|N|N|N|N| +2447641|AAAAAAAAJBJFFCAA|1989-04-24|1071|4660|358|1989|1|4|24|2|1989|358|4660|Monday|1989Q2|N|N|N|2447618|2447707|2447276|2447551|N|N|N|N|N| +2447642|AAAAAAAAKBJFFCAA|1989-04-25|1071|4661|358|1989|2|4|25|2|1989|358|4661|Tuesday|1989Q2|N|N|N|2447618|2447707|2447277|2447552|N|N|N|N|N| +2447643|AAAAAAAALBJFFCAA|1989-04-26|1071|4661|358|1989|3|4|26|2|1989|358|4661|Wednesday|1989Q2|N|N|N|2447618|2447707|2447278|2447553|N|N|N|N|N| +2447644|AAAAAAAAMBJFFCAA|1989-04-27|1071|4661|358|1989|4|4|27|2|1989|358|4661|Thursday|1989Q2|N|N|N|2447618|2447707|2447279|2447554|N|N|N|N|N| +2447645|AAAAAAAANBJFFCAA|1989-04-28|1071|4661|358|1989|5|4|28|2|1989|358|4661|Friday|1989Q2|N|Y|N|2447618|2447707|2447280|2447555|N|N|N|N|N| +2447646|AAAAAAAAOBJFFCAA|1989-04-29|1071|4661|358|1989|6|4|29|2|1989|358|4661|Saturday|1989Q2|N|Y|N|2447618|2447707|2447281|2447556|N|N|N|N|N| +2447647|AAAAAAAAPBJFFCAA|1989-04-30|1071|4661|358|1989|0|4|30|2|1989|358|4661|Sunday|1989Q2|N|N|N|2447618|2447707|2447282|2447557|N|N|N|N|N| +2447648|AAAAAAAAACJFFCAA|1989-05-01|1072|4661|358|1989|1|5|1|2|1989|358|4661|Monday|1989Q2|N|N|N|2447648|2447767|2447283|2447558|N|N|N|N|N| +2447649|AAAAAAAABCJFFCAA|1989-05-02|1072|4662|358|1989|2|5|2|2|1989|358|4662|Tuesday|1989Q2|N|N|N|2447648|2447767|2447284|2447559|N|N|N|N|N| +2447650|AAAAAAAACCJFFCAA|1989-05-03|1072|4662|358|1989|3|5|3|2|1989|358|4662|Wednesday|1989Q2|N|N|N|2447648|2447767|2447285|2447560|N|N|N|N|N| +2447651|AAAAAAAADCJFFCAA|1989-05-04|1072|4662|358|1989|4|5|4|2|1989|358|4662|Thursday|1989Q2|N|N|N|2447648|2447767|2447286|2447561|N|N|N|N|N| +2447652|AAAAAAAAECJFFCAA|1989-05-05|1072|4662|358|1989|5|5|5|2|1989|358|4662|Friday|1989Q2|N|Y|N|2447648|2447767|2447287|2447562|N|N|N|N|N| +2447653|AAAAAAAAFCJFFCAA|1989-05-06|1072|4662|358|1989|6|5|6|2|1989|358|4662|Saturday|1989Q2|N|Y|N|2447648|2447767|2447288|2447563|N|N|N|N|N| +2447654|AAAAAAAAGCJFFCAA|1989-05-07|1072|4662|358|1989|0|5|7|2|1989|358|4662|Sunday|1989Q2|N|N|N|2447648|2447767|2447289|2447564|N|N|N|N|N| +2447655|AAAAAAAAHCJFFCAA|1989-05-08|1072|4662|358|1989|1|5|8|2|1989|358|4662|Monday|1989Q2|N|N|N|2447648|2447767|2447290|2447565|N|N|N|N|N| +2447656|AAAAAAAAICJFFCAA|1989-05-09|1072|4663|358|1989|2|5|9|2|1989|358|4663|Tuesday|1989Q2|N|N|N|2447648|2447767|2447291|2447566|N|N|N|N|N| +2447657|AAAAAAAAJCJFFCAA|1989-05-10|1072|4663|358|1989|3|5|10|2|1989|358|4663|Wednesday|1989Q2|N|N|N|2447648|2447767|2447292|2447567|N|N|N|N|N| +2447658|AAAAAAAAKCJFFCAA|1989-05-11|1072|4663|358|1989|4|5|11|2|1989|358|4663|Thursday|1989Q2|N|N|N|2447648|2447767|2447293|2447568|N|N|N|N|N| +2447659|AAAAAAAALCJFFCAA|1989-05-12|1072|4663|358|1989|5|5|12|2|1989|358|4663|Friday|1989Q2|N|Y|N|2447648|2447767|2447294|2447569|N|N|N|N|N| +2447660|AAAAAAAAMCJFFCAA|1989-05-13|1072|4663|358|1989|6|5|13|2|1989|358|4663|Saturday|1989Q2|N|Y|N|2447648|2447767|2447295|2447570|N|N|N|N|N| +2447661|AAAAAAAANCJFFCAA|1989-05-14|1072|4663|358|1989|0|5|14|2|1989|358|4663|Sunday|1989Q2|N|N|N|2447648|2447767|2447296|2447571|N|N|N|N|N| +2447662|AAAAAAAAOCJFFCAA|1989-05-15|1072|4663|358|1989|1|5|15|2|1989|358|4663|Monday|1989Q2|N|N|N|2447648|2447767|2447297|2447572|N|N|N|N|N| +2447663|AAAAAAAAPCJFFCAA|1989-05-16|1072|4664|358|1989|2|5|16|2|1989|358|4664|Tuesday|1989Q2|N|N|N|2447648|2447767|2447298|2447573|N|N|N|N|N| +2447664|AAAAAAAAADJFFCAA|1989-05-17|1072|4664|358|1989|3|5|17|2|1989|358|4664|Wednesday|1989Q2|N|N|N|2447648|2447767|2447299|2447574|N|N|N|N|N| +2447665|AAAAAAAABDJFFCAA|1989-05-18|1072|4664|358|1989|4|5|18|2|1989|358|4664|Thursday|1989Q2|N|N|N|2447648|2447767|2447300|2447575|N|N|N|N|N| +2447666|AAAAAAAACDJFFCAA|1989-05-19|1072|4664|358|1989|5|5|19|2|1989|358|4664|Friday|1989Q2|N|Y|N|2447648|2447767|2447301|2447576|N|N|N|N|N| +2447667|AAAAAAAADDJFFCAA|1989-05-20|1072|4664|358|1989|6|5|20|2|1989|358|4664|Saturday|1989Q2|N|Y|N|2447648|2447767|2447302|2447577|N|N|N|N|N| +2447668|AAAAAAAAEDJFFCAA|1989-05-21|1072|4664|358|1989|0|5|21|2|1989|358|4664|Sunday|1989Q2|N|N|N|2447648|2447767|2447303|2447578|N|N|N|N|N| +2447669|AAAAAAAAFDJFFCAA|1989-05-22|1072|4664|358|1989|1|5|22|2|1989|358|4664|Monday|1989Q2|N|N|N|2447648|2447767|2447304|2447579|N|N|N|N|N| +2447670|AAAAAAAAGDJFFCAA|1989-05-23|1072|4665|358|1989|2|5|23|2|1989|358|4665|Tuesday|1989Q2|N|N|N|2447648|2447767|2447305|2447580|N|N|N|N|N| +2447671|AAAAAAAAHDJFFCAA|1989-05-24|1072|4665|358|1989|3|5|24|2|1989|358|4665|Wednesday|1989Q2|N|N|N|2447648|2447767|2447306|2447581|N|N|N|N|N| +2447672|AAAAAAAAIDJFFCAA|1989-05-25|1072|4665|358|1989|4|5|25|2|1989|358|4665|Thursday|1989Q2|N|N|N|2447648|2447767|2447307|2447582|N|N|N|N|N| +2447673|AAAAAAAAJDJFFCAA|1989-05-26|1072|4665|358|1989|5|5|26|2|1989|358|4665|Friday|1989Q2|N|Y|N|2447648|2447767|2447308|2447583|N|N|N|N|N| +2447674|AAAAAAAAKDJFFCAA|1989-05-27|1072|4665|358|1989|6|5|27|2|1989|358|4665|Saturday|1989Q2|N|Y|N|2447648|2447767|2447309|2447584|N|N|N|N|N| +2447675|AAAAAAAALDJFFCAA|1989-05-28|1072|4665|358|1989|0|5|28|2|1989|358|4665|Sunday|1989Q2|N|N|N|2447648|2447767|2447310|2447585|N|N|N|N|N| +2447676|AAAAAAAAMDJFFCAA|1989-05-29|1072|4665|358|1989|1|5|29|2|1989|358|4665|Monday|1989Q2|N|N|N|2447648|2447767|2447311|2447586|N|N|N|N|N| +2447677|AAAAAAAANDJFFCAA|1989-05-30|1072|4666|358|1989|2|5|30|2|1989|358|4666|Tuesday|1989Q2|N|N|N|2447648|2447767|2447312|2447587|N|N|N|N|N| +2447678|AAAAAAAAODJFFCAA|1989-05-31|1072|4666|358|1989|3|5|31|2|1989|358|4666|Wednesday|1989Q2|N|N|N|2447648|2447767|2447313|2447588|N|N|N|N|N| +2447679|AAAAAAAAPDJFFCAA|1989-06-01|1073|4666|359|1989|4|6|1|2|1989|359|4666|Thursday|1989Q2|N|N|N|2447679|2447829|2447314|2447589|N|N|N|N|N| +2447680|AAAAAAAAAEJFFCAA|1989-06-02|1073|4666|359|1989|5|6|2|2|1989|359|4666|Friday|1989Q2|N|Y|N|2447679|2447829|2447315|2447590|N|N|N|N|N| +2447681|AAAAAAAABEJFFCAA|1989-06-03|1073|4666|359|1989|6|6|3|2|1989|359|4666|Saturday|1989Q2|N|Y|N|2447679|2447829|2447316|2447591|N|N|N|N|N| +2447682|AAAAAAAACEJFFCAA|1989-06-04|1073|4666|359|1989|0|6|4|2|1989|359|4666|Sunday|1989Q2|N|N|N|2447679|2447829|2447317|2447592|N|N|N|N|N| +2447683|AAAAAAAADEJFFCAA|1989-06-05|1073|4666|359|1989|1|6|5|2|1989|359|4666|Monday|1989Q2|N|N|N|2447679|2447829|2447318|2447593|N|N|N|N|N| +2447684|AAAAAAAAEEJFFCAA|1989-06-06|1073|4667|359|1989|2|6|6|2|1989|359|4667|Tuesday|1989Q2|N|N|N|2447679|2447829|2447319|2447594|N|N|N|N|N| +2447685|AAAAAAAAFEJFFCAA|1989-06-07|1073|4667|359|1989|3|6|7|2|1989|359|4667|Wednesday|1989Q2|N|N|N|2447679|2447829|2447320|2447595|N|N|N|N|N| +2447686|AAAAAAAAGEJFFCAA|1989-06-08|1073|4667|359|1989|4|6|8|2|1989|359|4667|Thursday|1989Q2|N|N|N|2447679|2447829|2447321|2447596|N|N|N|N|N| +2447687|AAAAAAAAHEJFFCAA|1989-06-09|1073|4667|359|1989|5|6|9|2|1989|359|4667|Friday|1989Q2|N|Y|N|2447679|2447829|2447322|2447597|N|N|N|N|N| +2447688|AAAAAAAAIEJFFCAA|1989-06-10|1073|4667|359|1989|6|6|10|2|1989|359|4667|Saturday|1989Q2|N|Y|N|2447679|2447829|2447323|2447598|N|N|N|N|N| +2447689|AAAAAAAAJEJFFCAA|1989-06-11|1073|4667|359|1989|0|6|11|2|1989|359|4667|Sunday|1989Q2|N|N|N|2447679|2447829|2447324|2447599|N|N|N|N|N| +2447690|AAAAAAAAKEJFFCAA|1989-06-12|1073|4667|359|1989|1|6|12|2|1989|359|4667|Monday|1989Q2|N|N|N|2447679|2447829|2447325|2447600|N|N|N|N|N| +2447691|AAAAAAAALEJFFCAA|1989-06-13|1073|4668|359|1989|2|6|13|2|1989|359|4668|Tuesday|1989Q2|N|N|N|2447679|2447829|2447326|2447601|N|N|N|N|N| +2447692|AAAAAAAAMEJFFCAA|1989-06-14|1073|4668|359|1989|3|6|14|2|1989|359|4668|Wednesday|1989Q2|N|N|N|2447679|2447829|2447327|2447602|N|N|N|N|N| +2447693|AAAAAAAANEJFFCAA|1989-06-15|1073|4668|359|1989|4|6|15|2|1989|359|4668|Thursday|1989Q2|N|N|N|2447679|2447829|2447328|2447603|N|N|N|N|N| +2447694|AAAAAAAAOEJFFCAA|1989-06-16|1073|4668|359|1989|5|6|16|2|1989|359|4668|Friday|1989Q2|N|Y|N|2447679|2447829|2447329|2447604|N|N|N|N|N| +2447695|AAAAAAAAPEJFFCAA|1989-06-17|1073|4668|359|1989|6|6|17|2|1989|359|4668|Saturday|1989Q2|N|Y|N|2447679|2447829|2447330|2447605|N|N|N|N|N| +2447696|AAAAAAAAAFJFFCAA|1989-06-18|1073|4668|359|1989|0|6|18|2|1989|359|4668|Sunday|1989Q2|N|N|N|2447679|2447829|2447331|2447606|N|N|N|N|N| +2447697|AAAAAAAABFJFFCAA|1989-06-19|1073|4668|359|1989|1|6|19|2|1989|359|4668|Monday|1989Q2|N|N|N|2447679|2447829|2447332|2447607|N|N|N|N|N| +2447698|AAAAAAAACFJFFCAA|1989-06-20|1073|4669|359|1989|2|6|20|2|1989|359|4669|Tuesday|1989Q2|N|N|N|2447679|2447829|2447333|2447608|N|N|N|N|N| +2447699|AAAAAAAADFJFFCAA|1989-06-21|1073|4669|359|1989|3|6|21|2|1989|359|4669|Wednesday|1989Q2|N|N|N|2447679|2447829|2447334|2447609|N|N|N|N|N| +2447700|AAAAAAAAEFJFFCAA|1989-06-22|1073|4669|359|1989|4|6|22|2|1989|359|4669|Thursday|1989Q2|N|N|N|2447679|2447829|2447335|2447610|N|N|N|N|N| +2447701|AAAAAAAAFFJFFCAA|1989-06-23|1073|4669|359|1989|5|6|23|2|1989|359|4669|Friday|1989Q2|N|Y|N|2447679|2447829|2447336|2447611|N|N|N|N|N| +2447702|AAAAAAAAGFJFFCAA|1989-06-24|1073|4669|359|1989|6|6|24|2|1989|359|4669|Saturday|1989Q2|N|Y|N|2447679|2447829|2447337|2447612|N|N|N|N|N| +2447703|AAAAAAAAHFJFFCAA|1989-06-25|1073|4669|359|1989|0|6|25|2|1989|359|4669|Sunday|1989Q2|N|N|N|2447679|2447829|2447338|2447613|N|N|N|N|N| +2447704|AAAAAAAAIFJFFCAA|1989-06-26|1073|4669|359|1989|1|6|26|2|1989|359|4669|Monday|1989Q2|N|N|N|2447679|2447829|2447339|2447614|N|N|N|N|N| +2447705|AAAAAAAAJFJFFCAA|1989-06-27|1073|4670|359|1989|2|6|27|2|1989|359|4670|Tuesday|1989Q2|N|N|N|2447679|2447829|2447340|2447615|N|N|N|N|N| +2447706|AAAAAAAAKFJFFCAA|1989-06-28|1073|4670|359|1989|3|6|28|2|1989|359|4670|Wednesday|1989Q2|N|N|N|2447679|2447829|2447341|2447616|N|N|N|N|N| +2447707|AAAAAAAALFJFFCAA|1989-06-29|1073|4670|359|1989|4|6|29|2|1989|359|4670|Thursday|1989Q2|N|N|N|2447679|2447829|2447342|2447617|N|N|N|N|N| +2447708|AAAAAAAAMFJFFCAA|1989-06-30|1073|4670|359|1989|5|6|30|2|1989|359|4670|Friday|1989Q2|N|Y|N|2447679|2447829|2447343|2447618|N|N|N|N|N| +2447709|AAAAAAAANFJFFCAA|1989-07-01|1074|4670|359|1989|6|7|1|2|1989|359|4670|Saturday|1989Q2|N|Y|N|2447709|2447889|2447344|2447618|N|N|N|N|N| +2447710|AAAAAAAAOFJFFCAA|1989-07-02|1074|4670|359|1989|0|7|2|3|1989|359|4670|Sunday|1989Q3|N|N|N|2447709|2447889|2447345|2447619|N|N|N|N|N| +2447711|AAAAAAAAPFJFFCAA|1989-07-03|1074|4670|359|1989|1|7|3|3|1989|359|4670|Monday|1989Q3|N|N|N|2447709|2447889|2447346|2447620|N|N|N|N|N| +2447712|AAAAAAAAAGJFFCAA|1989-07-04|1074|4671|359|1989|2|7|4|3|1989|359|4671|Tuesday|1989Q3|N|N|N|2447709|2447889|2447347|2447621|N|N|N|N|N| +2447713|AAAAAAAABGJFFCAA|1989-07-05|1074|4671|359|1989|3|7|5|3|1989|359|4671|Wednesday|1989Q3|Y|N|N|2447709|2447889|2447348|2447622|N|N|N|N|N| +2447714|AAAAAAAACGJFFCAA|1989-07-06|1074|4671|359|1989|4|7|6|3|1989|359|4671|Thursday|1989Q3|N|N|Y|2447709|2447889|2447349|2447623|N|N|N|N|N| +2447715|AAAAAAAADGJFFCAA|1989-07-07|1074|4671|359|1989|5|7|7|3|1989|359|4671|Friday|1989Q3|N|Y|N|2447709|2447889|2447350|2447624|N|N|N|N|N| +2447716|AAAAAAAAEGJFFCAA|1989-07-08|1074|4671|359|1989|6|7|8|3|1989|359|4671|Saturday|1989Q3|N|Y|N|2447709|2447889|2447351|2447625|N|N|N|N|N| +2447717|AAAAAAAAFGJFFCAA|1989-07-09|1074|4671|359|1989|0|7|9|3|1989|359|4671|Sunday|1989Q3|N|N|N|2447709|2447889|2447352|2447626|N|N|N|N|N| +2447718|AAAAAAAAGGJFFCAA|1989-07-10|1074|4671|359|1989|1|7|10|3|1989|359|4671|Monday|1989Q3|N|N|N|2447709|2447889|2447353|2447627|N|N|N|N|N| +2447719|AAAAAAAAHGJFFCAA|1989-07-11|1074|4672|359|1989|2|7|11|3|1989|359|4672|Tuesday|1989Q3|N|N|N|2447709|2447889|2447354|2447628|N|N|N|N|N| +2447720|AAAAAAAAIGJFFCAA|1989-07-12|1074|4672|359|1989|3|7|12|3|1989|359|4672|Wednesday|1989Q3|N|N|N|2447709|2447889|2447355|2447629|N|N|N|N|N| +2447721|AAAAAAAAJGJFFCAA|1989-07-13|1074|4672|359|1989|4|7|13|3|1989|359|4672|Thursday|1989Q3|N|N|N|2447709|2447889|2447356|2447630|N|N|N|N|N| +2447722|AAAAAAAAKGJFFCAA|1989-07-14|1074|4672|359|1989|5|7|14|3|1989|359|4672|Friday|1989Q3|N|Y|N|2447709|2447889|2447357|2447631|N|N|N|N|N| +2447723|AAAAAAAALGJFFCAA|1989-07-15|1074|4672|359|1989|6|7|15|3|1989|359|4672|Saturday|1989Q3|N|Y|N|2447709|2447889|2447358|2447632|N|N|N|N|N| +2447724|AAAAAAAAMGJFFCAA|1989-07-16|1074|4672|359|1989|0|7|16|3|1989|359|4672|Sunday|1989Q3|N|N|N|2447709|2447889|2447359|2447633|N|N|N|N|N| +2447725|AAAAAAAANGJFFCAA|1989-07-17|1074|4672|359|1989|1|7|17|3|1989|359|4672|Monday|1989Q3|N|N|N|2447709|2447889|2447360|2447634|N|N|N|N|N| +2447726|AAAAAAAAOGJFFCAA|1989-07-18|1074|4673|359|1989|2|7|18|3|1989|359|4673|Tuesday|1989Q3|N|N|N|2447709|2447889|2447361|2447635|N|N|N|N|N| +2447727|AAAAAAAAPGJFFCAA|1989-07-19|1074|4673|359|1989|3|7|19|3|1989|359|4673|Wednesday|1989Q3|N|N|N|2447709|2447889|2447362|2447636|N|N|N|N|N| +2447728|AAAAAAAAAHJFFCAA|1989-07-20|1074|4673|359|1989|4|7|20|3|1989|359|4673|Thursday|1989Q3|N|N|N|2447709|2447889|2447363|2447637|N|N|N|N|N| +2447729|AAAAAAAABHJFFCAA|1989-07-21|1074|4673|359|1989|5|7|21|3|1989|359|4673|Friday|1989Q3|N|Y|N|2447709|2447889|2447364|2447638|N|N|N|N|N| +2447730|AAAAAAAACHJFFCAA|1989-07-22|1074|4673|359|1989|6|7|22|3|1989|359|4673|Saturday|1989Q3|N|Y|N|2447709|2447889|2447365|2447639|N|N|N|N|N| +2447731|AAAAAAAADHJFFCAA|1989-07-23|1074|4673|359|1989|0|7|23|3|1989|359|4673|Sunday|1989Q3|N|N|N|2447709|2447889|2447366|2447640|N|N|N|N|N| +2447732|AAAAAAAAEHJFFCAA|1989-07-24|1074|4673|359|1989|1|7|24|3|1989|359|4673|Monday|1989Q3|N|N|N|2447709|2447889|2447367|2447641|N|N|N|N|N| +2447733|AAAAAAAAFHJFFCAA|1989-07-25|1074|4674|359|1989|2|7|25|3|1989|359|4674|Tuesday|1989Q3|N|N|N|2447709|2447889|2447368|2447642|N|N|N|N|N| +2447734|AAAAAAAAGHJFFCAA|1989-07-26|1074|4674|359|1989|3|7|26|3|1989|359|4674|Wednesday|1989Q3|N|N|N|2447709|2447889|2447369|2447643|N|N|N|N|N| +2447735|AAAAAAAAHHJFFCAA|1989-07-27|1074|4674|359|1989|4|7|27|3|1989|359|4674|Thursday|1989Q3|N|N|N|2447709|2447889|2447370|2447644|N|N|N|N|N| +2447736|AAAAAAAAIHJFFCAA|1989-07-28|1074|4674|359|1989|5|7|28|3|1989|359|4674|Friday|1989Q3|N|Y|N|2447709|2447889|2447371|2447645|N|N|N|N|N| +2447737|AAAAAAAAJHJFFCAA|1989-07-29|1074|4674|359|1989|6|7|29|3|1989|359|4674|Saturday|1989Q3|N|Y|N|2447709|2447889|2447372|2447646|N|N|N|N|N| +2447738|AAAAAAAAKHJFFCAA|1989-07-30|1074|4674|359|1989|0|7|30|3|1989|359|4674|Sunday|1989Q3|N|N|N|2447709|2447889|2447373|2447647|N|N|N|N|N| +2447739|AAAAAAAALHJFFCAA|1989-07-31|1074|4674|359|1989|1|7|31|3|1989|359|4674|Monday|1989Q3|N|N|N|2447709|2447889|2447374|2447648|N|N|N|N|N| +2447740|AAAAAAAAMHJFFCAA|1989-08-01|1075|4675|359|1989|2|8|1|3|1989|359|4675|Tuesday|1989Q3|N|N|N|2447740|2447951|2447375|2447649|N|N|N|N|N| +2447741|AAAAAAAANHJFFCAA|1989-08-02|1075|4675|359|1989|3|8|2|3|1989|359|4675|Wednesday|1989Q3|N|N|N|2447740|2447951|2447376|2447650|N|N|N|N|N| +2447742|AAAAAAAAOHJFFCAA|1989-08-03|1075|4675|359|1989|4|8|3|3|1989|359|4675|Thursday|1989Q3|N|N|N|2447740|2447951|2447377|2447651|N|N|N|N|N| +2447743|AAAAAAAAPHJFFCAA|1989-08-04|1075|4675|359|1989|5|8|4|3|1989|359|4675|Friday|1989Q3|N|Y|N|2447740|2447951|2447378|2447652|N|N|N|N|N| +2447744|AAAAAAAAAIJFFCAA|1989-08-05|1075|4675|359|1989|6|8|5|3|1989|359|4675|Saturday|1989Q3|N|Y|N|2447740|2447951|2447379|2447653|N|N|N|N|N| +2447745|AAAAAAAABIJFFCAA|1989-08-06|1075|4675|359|1989|0|8|6|3|1989|359|4675|Sunday|1989Q3|N|N|N|2447740|2447951|2447380|2447654|N|N|N|N|N| +2447746|AAAAAAAACIJFFCAA|1989-08-07|1075|4675|359|1989|1|8|7|3|1989|359|4675|Monday|1989Q3|N|N|N|2447740|2447951|2447381|2447655|N|N|N|N|N| +2447747|AAAAAAAADIJFFCAA|1989-08-08|1075|4676|359|1989|2|8|8|3|1989|359|4676|Tuesday|1989Q3|N|N|N|2447740|2447951|2447382|2447656|N|N|N|N|N| +2447748|AAAAAAAAEIJFFCAA|1989-08-09|1075|4676|359|1989|3|8|9|3|1989|359|4676|Wednesday|1989Q3|N|N|N|2447740|2447951|2447383|2447657|N|N|N|N|N| +2447749|AAAAAAAAFIJFFCAA|1989-08-10|1075|4676|359|1989|4|8|10|3|1989|359|4676|Thursday|1989Q3|N|N|N|2447740|2447951|2447384|2447658|N|N|N|N|N| +2447750|AAAAAAAAGIJFFCAA|1989-08-11|1075|4676|359|1989|5|8|11|3|1989|359|4676|Friday|1989Q3|N|Y|N|2447740|2447951|2447385|2447659|N|N|N|N|N| +2447751|AAAAAAAAHIJFFCAA|1989-08-12|1075|4676|359|1989|6|8|12|3|1989|359|4676|Saturday|1989Q3|N|Y|N|2447740|2447951|2447386|2447660|N|N|N|N|N| +2447752|AAAAAAAAIIJFFCAA|1989-08-13|1075|4676|359|1989|0|8|13|3|1989|359|4676|Sunday|1989Q3|N|N|N|2447740|2447951|2447387|2447661|N|N|N|N|N| +2447753|AAAAAAAAJIJFFCAA|1989-08-14|1075|4676|359|1989|1|8|14|3|1989|359|4676|Monday|1989Q3|N|N|N|2447740|2447951|2447388|2447662|N|N|N|N|N| +2447754|AAAAAAAAKIJFFCAA|1989-08-15|1075|4677|359|1989|2|8|15|3|1989|359|4677|Tuesday|1989Q3|N|N|N|2447740|2447951|2447389|2447663|N|N|N|N|N| +2447755|AAAAAAAALIJFFCAA|1989-08-16|1075|4677|359|1989|3|8|16|3|1989|359|4677|Wednesday|1989Q3|N|N|N|2447740|2447951|2447390|2447664|N|N|N|N|N| +2447756|AAAAAAAAMIJFFCAA|1989-08-17|1075|4677|359|1989|4|8|17|3|1989|359|4677|Thursday|1989Q3|N|N|N|2447740|2447951|2447391|2447665|N|N|N|N|N| +2447757|AAAAAAAANIJFFCAA|1989-08-18|1075|4677|359|1989|5|8|18|3|1989|359|4677|Friday|1989Q3|N|Y|N|2447740|2447951|2447392|2447666|N|N|N|N|N| +2447758|AAAAAAAAOIJFFCAA|1989-08-19|1075|4677|359|1989|6|8|19|3|1989|359|4677|Saturday|1989Q3|N|Y|N|2447740|2447951|2447393|2447667|N|N|N|N|N| +2447759|AAAAAAAAPIJFFCAA|1989-08-20|1075|4677|359|1989|0|8|20|3|1989|359|4677|Sunday|1989Q3|N|N|N|2447740|2447951|2447394|2447668|N|N|N|N|N| +2447760|AAAAAAAAAJJFFCAA|1989-08-21|1075|4677|359|1989|1|8|21|3|1989|359|4677|Monday|1989Q3|N|N|N|2447740|2447951|2447395|2447669|N|N|N|N|N| +2447761|AAAAAAAABJJFFCAA|1989-08-22|1075|4678|359|1989|2|8|22|3|1989|359|4678|Tuesday|1989Q3|N|N|N|2447740|2447951|2447396|2447670|N|N|N|N|N| +2447762|AAAAAAAACJJFFCAA|1989-08-23|1075|4678|359|1989|3|8|23|3|1989|359|4678|Wednesday|1989Q3|N|N|N|2447740|2447951|2447397|2447671|N|N|N|N|N| +2447763|AAAAAAAADJJFFCAA|1989-08-24|1075|4678|359|1989|4|8|24|3|1989|359|4678|Thursday|1989Q3|N|N|N|2447740|2447951|2447398|2447672|N|N|N|N|N| +2447764|AAAAAAAAEJJFFCAA|1989-08-25|1075|4678|359|1989|5|8|25|3|1989|359|4678|Friday|1989Q3|N|Y|N|2447740|2447951|2447399|2447673|N|N|N|N|N| +2447765|AAAAAAAAFJJFFCAA|1989-08-26|1075|4678|359|1989|6|8|26|3|1989|359|4678|Saturday|1989Q3|N|Y|N|2447740|2447951|2447400|2447674|N|N|N|N|N| +2447766|AAAAAAAAGJJFFCAA|1989-08-27|1075|4678|359|1989|0|8|27|3|1989|359|4678|Sunday|1989Q3|N|N|N|2447740|2447951|2447401|2447675|N|N|N|N|N| +2447767|AAAAAAAAHJJFFCAA|1989-08-28|1075|4678|359|1989|1|8|28|3|1989|359|4678|Monday|1989Q3|N|N|N|2447740|2447951|2447402|2447676|N|N|N|N|N| +2447768|AAAAAAAAIJJFFCAA|1989-08-29|1075|4679|359|1989|2|8|29|3|1989|359|4679|Tuesday|1989Q3|N|N|N|2447740|2447951|2447403|2447677|N|N|N|N|N| +2447769|AAAAAAAAJJJFFCAA|1989-08-30|1075|4679|359|1989|3|8|30|3|1989|359|4679|Wednesday|1989Q3|N|N|N|2447740|2447951|2447404|2447678|N|N|N|N|N| +2447770|AAAAAAAAKJJFFCAA|1989-08-31|1075|4679|359|1989|4|8|31|3|1989|359|4679|Thursday|1989Q3|N|N|N|2447740|2447951|2447405|2447679|N|N|N|N|N| +2447771|AAAAAAAALJJFFCAA|1989-09-01|1076|4679|360|1989|5|9|1|3|1989|360|4679|Friday|1989Q3|N|Y|N|2447771|2448013|2447406|2447680|N|N|N|N|N| +2447772|AAAAAAAAMJJFFCAA|1989-09-02|1076|4679|360|1989|6|9|2|3|1989|360|4679|Saturday|1989Q3|N|Y|N|2447771|2448013|2447407|2447681|N|N|N|N|N| +2447773|AAAAAAAANJJFFCAA|1989-09-03|1076|4679|360|1989|0|9|3|3|1989|360|4679|Sunday|1989Q3|N|N|N|2447771|2448013|2447408|2447682|N|N|N|N|N| +2447774|AAAAAAAAOJJFFCAA|1989-09-04|1076|4679|360|1989|1|9|4|3|1989|360|4679|Monday|1989Q3|N|N|N|2447771|2448013|2447409|2447683|N|N|N|N|N| +2447775|AAAAAAAAPJJFFCAA|1989-09-05|1076|4680|360|1989|2|9|5|3|1989|360|4680|Tuesday|1989Q3|N|N|N|2447771|2448013|2447410|2447684|N|N|N|N|N| +2447776|AAAAAAAAAKJFFCAA|1989-09-06|1076|4680|360|1989|3|9|6|3|1989|360|4680|Wednesday|1989Q3|N|N|N|2447771|2448013|2447411|2447685|N|N|N|N|N| +2447777|AAAAAAAABKJFFCAA|1989-09-07|1076|4680|360|1989|4|9|7|3|1989|360|4680|Thursday|1989Q3|N|N|N|2447771|2448013|2447412|2447686|N|N|N|N|N| +2447778|AAAAAAAACKJFFCAA|1989-09-08|1076|4680|360|1989|5|9|8|3|1989|360|4680|Friday|1989Q3|N|Y|N|2447771|2448013|2447413|2447687|N|N|N|N|N| +2447779|AAAAAAAADKJFFCAA|1989-09-09|1076|4680|360|1989|6|9|9|3|1989|360|4680|Saturday|1989Q3|N|Y|N|2447771|2448013|2447414|2447688|N|N|N|N|N| +2447780|AAAAAAAAEKJFFCAA|1989-09-10|1076|4680|360|1989|0|9|10|3|1989|360|4680|Sunday|1989Q3|N|N|N|2447771|2448013|2447415|2447689|N|N|N|N|N| +2447781|AAAAAAAAFKJFFCAA|1989-09-11|1076|4680|360|1989|1|9|11|3|1989|360|4680|Monday|1989Q3|N|N|N|2447771|2448013|2447416|2447690|N|N|N|N|N| +2447782|AAAAAAAAGKJFFCAA|1989-09-12|1076|4681|360|1989|2|9|12|3|1989|360|4681|Tuesday|1989Q3|N|N|N|2447771|2448013|2447417|2447691|N|N|N|N|N| +2447783|AAAAAAAAHKJFFCAA|1989-09-13|1076|4681|360|1989|3|9|13|3|1989|360|4681|Wednesday|1989Q3|N|N|N|2447771|2448013|2447418|2447692|N|N|N|N|N| +2447784|AAAAAAAAIKJFFCAA|1989-09-14|1076|4681|360|1989|4|9|14|3|1989|360|4681|Thursday|1989Q3|N|N|N|2447771|2448013|2447419|2447693|N|N|N|N|N| +2447785|AAAAAAAAJKJFFCAA|1989-09-15|1076|4681|360|1989|5|9|15|3|1989|360|4681|Friday|1989Q3|N|Y|N|2447771|2448013|2447420|2447694|N|N|N|N|N| +2447786|AAAAAAAAKKJFFCAA|1989-09-16|1076|4681|360|1989|6|9|16|3|1989|360|4681|Saturday|1989Q3|N|Y|N|2447771|2448013|2447421|2447695|N|N|N|N|N| +2447787|AAAAAAAALKJFFCAA|1989-09-17|1076|4681|360|1989|0|9|17|3|1989|360|4681|Sunday|1989Q3|N|N|N|2447771|2448013|2447422|2447696|N|N|N|N|N| +2447788|AAAAAAAAMKJFFCAA|1989-09-18|1076|4681|360|1989|1|9|18|3|1989|360|4681|Monday|1989Q3|N|N|N|2447771|2448013|2447423|2447697|N|N|N|N|N| +2447789|AAAAAAAANKJFFCAA|1989-09-19|1076|4682|360|1989|2|9|19|3|1989|360|4682|Tuesday|1989Q3|N|N|N|2447771|2448013|2447424|2447698|N|N|N|N|N| +2447790|AAAAAAAAOKJFFCAA|1989-09-20|1076|4682|360|1989|3|9|20|3|1989|360|4682|Wednesday|1989Q3|N|N|N|2447771|2448013|2447425|2447699|N|N|N|N|N| +2447791|AAAAAAAAPKJFFCAA|1989-09-21|1076|4682|360|1989|4|9|21|3|1989|360|4682|Thursday|1989Q3|N|N|N|2447771|2448013|2447426|2447700|N|N|N|N|N| +2447792|AAAAAAAAALJFFCAA|1989-09-22|1076|4682|360|1989|5|9|22|3|1989|360|4682|Friday|1989Q3|N|Y|N|2447771|2448013|2447427|2447701|N|N|N|N|N| +2447793|AAAAAAAABLJFFCAA|1989-09-23|1076|4682|360|1989|6|9|23|3|1989|360|4682|Saturday|1989Q3|N|Y|N|2447771|2448013|2447428|2447702|N|N|N|N|N| +2447794|AAAAAAAACLJFFCAA|1989-09-24|1076|4682|360|1989|0|9|24|3|1989|360|4682|Sunday|1989Q3|N|N|N|2447771|2448013|2447429|2447703|N|N|N|N|N| +2447795|AAAAAAAADLJFFCAA|1989-09-25|1076|4682|360|1989|1|9|25|3|1989|360|4682|Monday|1989Q3|N|N|N|2447771|2448013|2447430|2447704|N|N|N|N|N| +2447796|AAAAAAAAELJFFCAA|1989-09-26|1076|4683|360|1989|2|9|26|3|1989|360|4683|Tuesday|1989Q3|N|N|N|2447771|2448013|2447431|2447705|N|N|N|N|N| +2447797|AAAAAAAAFLJFFCAA|1989-09-27|1076|4683|360|1989|3|9|27|3|1989|360|4683|Wednesday|1989Q3|N|N|N|2447771|2448013|2447432|2447706|N|N|N|N|N| +2447798|AAAAAAAAGLJFFCAA|1989-09-28|1076|4683|360|1989|4|9|28|3|1989|360|4683|Thursday|1989Q3|N|N|N|2447771|2448013|2447433|2447707|N|N|N|N|N| +2447799|AAAAAAAAHLJFFCAA|1989-09-29|1076|4683|360|1989|5|9|29|3|1989|360|4683|Friday|1989Q3|N|Y|N|2447771|2448013|2447434|2447708|N|N|N|N|N| +2447800|AAAAAAAAILJFFCAA|1989-09-30|1076|4683|360|1989|6|9|30|3|1989|360|4683|Saturday|1989Q3|N|Y|N|2447771|2448013|2447435|2447709|N|N|N|N|N| +2447801|AAAAAAAAJLJFFCAA|1989-10-01|1077|4683|360|1989|0|10|1|3|1989|360|4683|Sunday|1989Q3|N|N|N|2447801|2448073|2447436|2447709|N|N|N|N|N| +2447802|AAAAAAAAKLJFFCAA|1989-10-02|1077|4683|360|1989|1|10|2|4|1989|360|4683|Monday|1989Q4|N|N|N|2447801|2448073|2447437|2447710|N|N|N|N|N| +2447803|AAAAAAAALLJFFCAA|1989-10-03|1077|4684|360|1989|2|10|3|4|1989|360|4684|Tuesday|1989Q4|N|N|N|2447801|2448073|2447438|2447711|N|N|N|N|N| +2447804|AAAAAAAAMLJFFCAA|1989-10-04|1077|4684|360|1989|3|10|4|4|1989|360|4684|Wednesday|1989Q4|N|N|N|2447801|2448073|2447439|2447712|N|N|N|N|N| +2447805|AAAAAAAANLJFFCAA|1989-10-05|1077|4684|360|1989|4|10|5|4|1989|360|4684|Thursday|1989Q4|N|N|N|2447801|2448073|2447440|2447713|N|N|N|N|N| +2447806|AAAAAAAAOLJFFCAA|1989-10-06|1077|4684|360|1989|5|10|6|4|1989|360|4684|Friday|1989Q4|N|Y|N|2447801|2448073|2447441|2447714|N|N|N|N|N| +2447807|AAAAAAAAPLJFFCAA|1989-10-07|1077|4684|360|1989|6|10|7|4|1989|360|4684|Saturday|1989Q4|N|Y|N|2447801|2448073|2447442|2447715|N|N|N|N|N| +2447808|AAAAAAAAAMJFFCAA|1989-10-08|1077|4684|360|1989|0|10|8|4|1989|360|4684|Sunday|1989Q4|N|N|N|2447801|2448073|2447443|2447716|N|N|N|N|N| +2447809|AAAAAAAABMJFFCAA|1989-10-09|1077|4684|360|1989|1|10|9|4|1989|360|4684|Monday|1989Q4|N|N|N|2447801|2448073|2447444|2447717|N|N|N|N|N| +2447810|AAAAAAAACMJFFCAA|1989-10-10|1077|4685|360|1989|2|10|10|4|1989|360|4685|Tuesday|1989Q4|N|N|N|2447801|2448073|2447445|2447718|N|N|N|N|N| +2447811|AAAAAAAADMJFFCAA|1989-10-11|1077|4685|360|1989|3|10|11|4|1989|360|4685|Wednesday|1989Q4|N|N|N|2447801|2448073|2447446|2447719|N|N|N|N|N| +2447812|AAAAAAAAEMJFFCAA|1989-10-12|1077|4685|360|1989|4|10|12|4|1989|360|4685|Thursday|1989Q4|N|N|N|2447801|2448073|2447447|2447720|N|N|N|N|N| +2447813|AAAAAAAAFMJFFCAA|1989-10-13|1077|4685|360|1989|5|10|13|4|1989|360|4685|Friday|1989Q4|N|Y|N|2447801|2448073|2447448|2447721|N|N|N|N|N| +2447814|AAAAAAAAGMJFFCAA|1989-10-14|1077|4685|360|1989|6|10|14|4|1989|360|4685|Saturday|1989Q4|N|Y|N|2447801|2448073|2447449|2447722|N|N|N|N|N| +2447815|AAAAAAAAHMJFFCAA|1989-10-15|1077|4685|360|1989|0|10|15|4|1989|360|4685|Sunday|1989Q4|N|N|N|2447801|2448073|2447450|2447723|N|N|N|N|N| +2447816|AAAAAAAAIMJFFCAA|1989-10-16|1077|4685|360|1989|1|10|16|4|1989|360|4685|Monday|1989Q4|N|N|N|2447801|2448073|2447451|2447724|N|N|N|N|N| +2447817|AAAAAAAAJMJFFCAA|1989-10-17|1077|4686|360|1989|2|10|17|4|1989|360|4686|Tuesday|1989Q4|N|N|N|2447801|2448073|2447452|2447725|N|N|N|N|N| +2447818|AAAAAAAAKMJFFCAA|1989-10-18|1077|4686|360|1989|3|10|18|4|1989|360|4686|Wednesday|1989Q4|N|N|N|2447801|2448073|2447453|2447726|N|N|N|N|N| +2447819|AAAAAAAALMJFFCAA|1989-10-19|1077|4686|360|1989|4|10|19|4|1989|360|4686|Thursday|1989Q4|N|N|N|2447801|2448073|2447454|2447727|N|N|N|N|N| +2447820|AAAAAAAAMMJFFCAA|1989-10-20|1077|4686|360|1989|5|10|20|4|1989|360|4686|Friday|1989Q4|N|Y|N|2447801|2448073|2447455|2447728|N|N|N|N|N| +2447821|AAAAAAAANMJFFCAA|1989-10-21|1077|4686|360|1989|6|10|21|4|1989|360|4686|Saturday|1989Q4|N|Y|N|2447801|2448073|2447456|2447729|N|N|N|N|N| +2447822|AAAAAAAAOMJFFCAA|1989-10-22|1077|4686|360|1989|0|10|22|4|1989|360|4686|Sunday|1989Q4|N|N|N|2447801|2448073|2447457|2447730|N|N|N|N|N| +2447823|AAAAAAAAPMJFFCAA|1989-10-23|1077|4686|360|1989|1|10|23|4|1989|360|4686|Monday|1989Q4|N|N|N|2447801|2448073|2447458|2447731|N|N|N|N|N| +2447824|AAAAAAAAANJFFCAA|1989-10-24|1077|4687|360|1989|2|10|24|4|1989|360|4687|Tuesday|1989Q4|N|N|N|2447801|2448073|2447459|2447732|N|N|N|N|N| +2447825|AAAAAAAABNJFFCAA|1989-10-25|1077|4687|360|1989|3|10|25|4|1989|360|4687|Wednesday|1989Q4|N|N|N|2447801|2448073|2447460|2447733|N|N|N|N|N| +2447826|AAAAAAAACNJFFCAA|1989-10-26|1077|4687|360|1989|4|10|26|4|1989|360|4687|Thursday|1989Q4|N|N|N|2447801|2448073|2447461|2447734|N|N|N|N|N| +2447827|AAAAAAAADNJFFCAA|1989-10-27|1077|4687|360|1989|5|10|27|4|1989|360|4687|Friday|1989Q4|N|Y|N|2447801|2448073|2447462|2447735|N|N|N|N|N| +2447828|AAAAAAAAENJFFCAA|1989-10-28|1077|4687|360|1989|6|10|28|4|1989|360|4687|Saturday|1989Q4|N|Y|N|2447801|2448073|2447463|2447736|N|N|N|N|N| +2447829|AAAAAAAAFNJFFCAA|1989-10-29|1077|4687|360|1989|0|10|29|4|1989|360|4687|Sunday|1989Q4|N|N|N|2447801|2448073|2447464|2447737|N|N|N|N|N| +2447830|AAAAAAAAGNJFFCAA|1989-10-30|1077|4687|360|1989|1|10|30|4|1989|360|4687|Monday|1989Q4|N|N|N|2447801|2448073|2447465|2447738|N|N|N|N|N| +2447831|AAAAAAAAHNJFFCAA|1989-10-31|1077|4688|360|1989|2|10|31|4|1989|360|4688|Tuesday|1989Q4|N|N|N|2447801|2448073|2447466|2447739|N|N|N|N|N| +2447832|AAAAAAAAINJFFCAA|1989-11-01|1078|4688|360|1989|3|11|1|4|1989|360|4688|Wednesday|1989Q4|N|N|N|2447832|2448135|2447467|2447740|N|N|N|N|N| +2447833|AAAAAAAAJNJFFCAA|1989-11-02|1078|4688|360|1989|4|11|2|4|1989|360|4688|Thursday|1989Q4|N|N|N|2447832|2448135|2447468|2447741|N|N|N|N|N| +2447834|AAAAAAAAKNJFFCAA|1989-11-03|1078|4688|360|1989|5|11|3|4|1989|360|4688|Friday|1989Q4|N|Y|N|2447832|2448135|2447469|2447742|N|N|N|N|N| +2447835|AAAAAAAALNJFFCAA|1989-11-04|1078|4688|360|1989|6|11|4|4|1989|360|4688|Saturday|1989Q4|N|Y|N|2447832|2448135|2447470|2447743|N|N|N|N|N| +2447836|AAAAAAAAMNJFFCAA|1989-11-05|1078|4688|360|1989|0|11|5|4|1989|360|4688|Sunday|1989Q4|N|N|N|2447832|2448135|2447471|2447744|N|N|N|N|N| +2447837|AAAAAAAANNJFFCAA|1989-11-06|1078|4688|360|1989|1|11|6|4|1989|360|4688|Monday|1989Q4|N|N|N|2447832|2448135|2447472|2447745|N|N|N|N|N| +2447838|AAAAAAAAONJFFCAA|1989-11-07|1078|4689|360|1989|2|11|7|4|1989|360|4689|Tuesday|1989Q4|N|N|N|2447832|2448135|2447473|2447746|N|N|N|N|N| +2447839|AAAAAAAAPNJFFCAA|1989-11-08|1078|4689|360|1989|3|11|8|4|1989|360|4689|Wednesday|1989Q4|N|N|N|2447832|2448135|2447474|2447747|N|N|N|N|N| +2447840|AAAAAAAAAOJFFCAA|1989-11-09|1078|4689|360|1989|4|11|9|4|1989|360|4689|Thursday|1989Q4|N|N|N|2447832|2448135|2447475|2447748|N|N|N|N|N| +2447841|AAAAAAAABOJFFCAA|1989-11-10|1078|4689|360|1989|5|11|10|4|1989|360|4689|Friday|1989Q4|N|Y|N|2447832|2448135|2447476|2447749|N|N|N|N|N| +2447842|AAAAAAAACOJFFCAA|1989-11-11|1078|4689|360|1989|6|11|11|4|1989|360|4689|Saturday|1989Q4|N|Y|N|2447832|2448135|2447477|2447750|N|N|N|N|N| +2447843|AAAAAAAADOJFFCAA|1989-11-12|1078|4689|360|1989|0|11|12|4|1989|360|4689|Sunday|1989Q4|N|N|N|2447832|2448135|2447478|2447751|N|N|N|N|N| +2447844|AAAAAAAAEOJFFCAA|1989-11-13|1078|4689|360|1989|1|11|13|4|1989|360|4689|Monday|1989Q4|N|N|N|2447832|2448135|2447479|2447752|N|N|N|N|N| +2447845|AAAAAAAAFOJFFCAA|1989-11-14|1078|4690|360|1989|2|11|14|4|1989|360|4690|Tuesday|1989Q4|N|N|N|2447832|2448135|2447480|2447753|N|N|N|N|N| +2447846|AAAAAAAAGOJFFCAA|1989-11-15|1078|4690|360|1989|3|11|15|4|1989|360|4690|Wednesday|1989Q4|N|N|N|2447832|2448135|2447481|2447754|N|N|N|N|N| +2447847|AAAAAAAAHOJFFCAA|1989-11-16|1078|4690|360|1989|4|11|16|4|1989|360|4690|Thursday|1989Q4|N|N|N|2447832|2448135|2447482|2447755|N|N|N|N|N| +2447848|AAAAAAAAIOJFFCAA|1989-11-17|1078|4690|360|1989|5|11|17|4|1989|360|4690|Friday|1989Q4|N|Y|N|2447832|2448135|2447483|2447756|N|N|N|N|N| +2447849|AAAAAAAAJOJFFCAA|1989-11-18|1078|4690|360|1989|6|11|18|4|1989|360|4690|Saturday|1989Q4|N|Y|N|2447832|2448135|2447484|2447757|N|N|N|N|N| +2447850|AAAAAAAAKOJFFCAA|1989-11-19|1078|4690|360|1989|0|11|19|4|1989|360|4690|Sunday|1989Q4|N|N|N|2447832|2448135|2447485|2447758|N|N|N|N|N| +2447851|AAAAAAAALOJFFCAA|1989-11-20|1078|4690|360|1989|1|11|20|4|1989|360|4690|Monday|1989Q4|N|N|N|2447832|2448135|2447486|2447759|N|N|N|N|N| +2447852|AAAAAAAAMOJFFCAA|1989-11-21|1078|4691|360|1989|2|11|21|4|1989|360|4691|Tuesday|1989Q4|N|N|N|2447832|2448135|2447487|2447760|N|N|N|N|N| +2447853|AAAAAAAANOJFFCAA|1989-11-22|1078|4691|360|1989|3|11|22|4|1989|360|4691|Wednesday|1989Q4|N|N|N|2447832|2448135|2447488|2447761|N|N|N|N|N| +2447854|AAAAAAAAOOJFFCAA|1989-11-23|1078|4691|360|1989|4|11|23|4|1989|360|4691|Thursday|1989Q4|N|N|N|2447832|2448135|2447489|2447762|N|N|N|N|N| +2447855|AAAAAAAAPOJFFCAA|1989-11-24|1078|4691|360|1989|5|11|24|4|1989|360|4691|Friday|1989Q4|N|Y|N|2447832|2448135|2447490|2447763|N|N|N|N|N| +2447856|AAAAAAAAAPJFFCAA|1989-11-25|1078|4691|360|1989|6|11|25|4|1989|360|4691|Saturday|1989Q4|N|Y|N|2447832|2448135|2447491|2447764|N|N|N|N|N| +2447857|AAAAAAAABPJFFCAA|1989-11-26|1078|4691|360|1989|0|11|26|4|1989|360|4691|Sunday|1989Q4|N|N|N|2447832|2448135|2447492|2447765|N|N|N|N|N| +2447858|AAAAAAAACPJFFCAA|1989-11-27|1078|4691|360|1989|1|11|27|4|1989|360|4691|Monday|1989Q4|N|N|N|2447832|2448135|2447493|2447766|N|N|N|N|N| +2447859|AAAAAAAADPJFFCAA|1989-11-28|1078|4692|360|1989|2|11|28|4|1989|360|4692|Tuesday|1989Q4|N|N|N|2447832|2448135|2447494|2447767|N|N|N|N|N| +2447860|AAAAAAAAEPJFFCAA|1989-11-29|1078|4692|360|1989|3|11|29|4|1989|360|4692|Wednesday|1989Q4|N|N|N|2447832|2448135|2447495|2447768|N|N|N|N|N| +2447861|AAAAAAAAFPJFFCAA|1989-11-30|1078|4692|360|1989|4|11|30|4|1989|360|4692|Thursday|1989Q4|N|N|N|2447832|2448135|2447496|2447769|N|N|N|N|N| +2447862|AAAAAAAAGPJFFCAA|1989-12-01|1079|4692|361|1989|5|12|1|4|1989|361|4692|Friday|1989Q4|N|Y|N|2447862|2448195|2447497|2447770|N|N|N|N|N| +2447863|AAAAAAAAHPJFFCAA|1989-12-02|1079|4692|361|1989|6|12|2|4|1989|361|4692|Saturday|1989Q4|N|Y|N|2447862|2448195|2447498|2447771|N|N|N|N|N| +2447864|AAAAAAAAIPJFFCAA|1989-12-03|1079|4692|361|1989|0|12|3|4|1989|361|4692|Sunday|1989Q4|N|N|N|2447862|2448195|2447499|2447772|N|N|N|N|N| +2447865|AAAAAAAAJPJFFCAA|1989-12-04|1079|4692|361|1989|1|12|4|4|1989|361|4692|Monday|1989Q4|N|N|N|2447862|2448195|2447500|2447773|N|N|N|N|N| +2447866|AAAAAAAAKPJFFCAA|1989-12-05|1079|4693|361|1989|2|12|5|4|1989|361|4693|Tuesday|1989Q4|N|N|N|2447862|2448195|2447501|2447774|N|N|N|N|N| +2447867|AAAAAAAALPJFFCAA|1989-12-06|1079|4693|361|1989|3|12|6|4|1989|361|4693|Wednesday|1989Q4|N|N|N|2447862|2448195|2447502|2447775|N|N|N|N|N| +2447868|AAAAAAAAMPJFFCAA|1989-12-07|1079|4693|361|1989|4|12|7|4|1989|361|4693|Thursday|1989Q4|N|N|N|2447862|2448195|2447503|2447776|N|N|N|N|N| +2447869|AAAAAAAANPJFFCAA|1989-12-08|1079|4693|361|1989|5|12|8|4|1989|361|4693|Friday|1989Q4|N|Y|N|2447862|2448195|2447504|2447777|N|N|N|N|N| +2447870|AAAAAAAAOPJFFCAA|1989-12-09|1079|4693|361|1989|6|12|9|4|1989|361|4693|Saturday|1989Q4|N|Y|N|2447862|2448195|2447505|2447778|N|N|N|N|N| +2447871|AAAAAAAAPPJFFCAA|1989-12-10|1079|4693|361|1989|0|12|10|4|1989|361|4693|Sunday|1989Q4|N|N|N|2447862|2448195|2447506|2447779|N|N|N|N|N| +2447872|AAAAAAAAAAKFFCAA|1989-12-11|1079|4693|361|1989|1|12|11|4|1989|361|4693|Monday|1989Q4|N|N|N|2447862|2448195|2447507|2447780|N|N|N|N|N| +2447873|AAAAAAAABAKFFCAA|1989-12-12|1079|4694|361|1989|2|12|12|4|1989|361|4694|Tuesday|1989Q4|N|N|N|2447862|2448195|2447508|2447781|N|N|N|N|N| +2447874|AAAAAAAACAKFFCAA|1989-12-13|1079|4694|361|1989|3|12|13|4|1989|361|4694|Wednesday|1989Q4|N|N|N|2447862|2448195|2447509|2447782|N|N|N|N|N| +2447875|AAAAAAAADAKFFCAA|1989-12-14|1079|4694|361|1989|4|12|14|4|1989|361|4694|Thursday|1989Q4|N|N|N|2447862|2448195|2447510|2447783|N|N|N|N|N| +2447876|AAAAAAAAEAKFFCAA|1989-12-15|1079|4694|361|1989|5|12|15|4|1989|361|4694|Friday|1989Q4|N|Y|N|2447862|2448195|2447511|2447784|N|N|N|N|N| +2447877|AAAAAAAAFAKFFCAA|1989-12-16|1079|4694|361|1989|6|12|16|4|1989|361|4694|Saturday|1989Q4|N|Y|N|2447862|2448195|2447512|2447785|N|N|N|N|N| +2447878|AAAAAAAAGAKFFCAA|1989-12-17|1079|4694|361|1989|0|12|17|4|1989|361|4694|Sunday|1989Q4|N|N|N|2447862|2448195|2447513|2447786|N|N|N|N|N| +2447879|AAAAAAAAHAKFFCAA|1989-12-18|1079|4694|361|1989|1|12|18|4|1989|361|4694|Monday|1989Q4|N|N|N|2447862|2448195|2447514|2447787|N|N|N|N|N| +2447880|AAAAAAAAIAKFFCAA|1989-12-19|1079|4695|361|1989|2|12|19|4|1989|361|4695|Tuesday|1989Q4|N|N|N|2447862|2448195|2447515|2447788|N|N|N|N|N| +2447881|AAAAAAAAJAKFFCAA|1989-12-20|1079|4695|361|1989|3|12|20|4|1989|361|4695|Wednesday|1989Q4|N|N|N|2447862|2448195|2447516|2447789|N|N|N|N|N| +2447882|AAAAAAAAKAKFFCAA|1989-12-21|1079|4695|361|1989|4|12|21|4|1989|361|4695|Thursday|1989Q4|N|N|N|2447862|2448195|2447517|2447790|N|N|N|N|N| +2447883|AAAAAAAALAKFFCAA|1989-12-22|1079|4695|361|1989|5|12|22|4|1989|361|4695|Friday|1989Q4|N|Y|N|2447862|2448195|2447518|2447791|N|N|N|N|N| +2447884|AAAAAAAAMAKFFCAA|1989-12-23|1079|4695|361|1989|6|12|23|4|1989|361|4695|Saturday|1989Q4|N|Y|N|2447862|2448195|2447519|2447792|N|N|N|N|N| +2447885|AAAAAAAANAKFFCAA|1989-12-24|1079|4695|361|1989|0|12|24|4|1989|361|4695|Sunday|1989Q4|N|N|N|2447862|2448195|2447520|2447793|N|N|N|N|N| +2447886|AAAAAAAAOAKFFCAA|1989-12-25|1079|4695|361|1989|1|12|25|4|1989|361|4695|Monday|1989Q4|N|N|N|2447862|2448195|2447521|2447794|N|N|N|N|N| +2447887|AAAAAAAAPAKFFCAA|1989-12-26|1079|4696|361|1989|2|12|26|4|1989|361|4696|Tuesday|1989Q4|Y|N|N|2447862|2448195|2447522|2447795|N|N|N|N|N| +2447888|AAAAAAAAABKFFCAA|1989-12-27|1079|4696|361|1989|3|12|27|4|1989|361|4696|Wednesday|1989Q4|N|N|Y|2447862|2448195|2447523|2447796|N|N|N|N|N| +2447889|AAAAAAAABBKFFCAA|1989-12-28|1079|4696|361|1989|4|12|28|4|1989|361|4696|Thursday|1989Q4|N|N|N|2447862|2448195|2447524|2447797|N|N|N|N|N| +2447890|AAAAAAAACBKFFCAA|1989-12-29|1079|4696|361|1989|5|12|29|4|1989|361|4696|Friday|1989Q4|N|Y|N|2447862|2448195|2447525|2447798|N|N|N|N|N| +2447891|AAAAAAAADBKFFCAA|1989-12-30|1079|4696|361|1989|6|12|30|4|1989|361|4696|Saturday|1989Q4|N|Y|N|2447862|2448195|2447526|2447799|N|N|N|N|N| +2447892|AAAAAAAAEBKFFCAA|1989-12-31|1079|4696|361|1989|0|12|31|4|1989|361|4696|Sunday|1989Q4|N|N|N|2447862|2448195|2447527|2447800|N|N|N|N|N| +2447893|AAAAAAAAFBKFFCAA|1990-01-01|1080|4696|361|1990|1|1|1|1|1990|361|4696|Monday|1990Q1|Y|N|N|2447893|2447892|2447528|2447801|N|N|N|N|N| +2447894|AAAAAAAAGBKFFCAA|1990-01-02|1080|4697|361|1990|2|1|2|1|1990|361|4697|Tuesday|1990Q1|N|N|Y|2447893|2447892|2447529|2447802|N|N|N|N|N| +2447895|AAAAAAAAHBKFFCAA|1990-01-03|1080|4697|361|1990|3|1|3|1|1990|361|4697|Wednesday|1990Q1|N|N|N|2447893|2447892|2447530|2447803|N|N|N|N|N| +2447896|AAAAAAAAIBKFFCAA|1990-01-04|1080|4697|361|1990|4|1|4|1|1990|361|4697|Thursday|1990Q1|N|N|N|2447893|2447892|2447531|2447804|N|N|N|N|N| +2447897|AAAAAAAAJBKFFCAA|1990-01-05|1080|4697|361|1990|5|1|5|1|1990|361|4697|Friday|1990Q1|N|Y|N|2447893|2447892|2447532|2447805|N|N|N|N|N| +2447898|AAAAAAAAKBKFFCAA|1990-01-06|1080|4697|361|1990|6|1|6|1|1990|361|4697|Saturday|1990Q1|N|Y|N|2447893|2447892|2447533|2447806|N|N|N|N|N| +2447899|AAAAAAAALBKFFCAA|1990-01-07|1080|4697|361|1990|0|1|7|1|1990|361|4697|Sunday|1990Q1|N|N|N|2447893|2447892|2447534|2447807|N|N|N|N|N| +2447900|AAAAAAAAMBKFFCAA|1990-01-08|1080|4697|361|1990|1|1|8|1|1990|361|4697|Monday|1990Q1|N|N|N|2447893|2447892|2447535|2447808|N|N|N|N|N| +2447901|AAAAAAAANBKFFCAA|1990-01-09|1080|4698|361|1990|2|1|9|1|1990|361|4698|Tuesday|1990Q1|N|N|N|2447893|2447892|2447536|2447809|N|N|N|N|N| +2447902|AAAAAAAAOBKFFCAA|1990-01-10|1080|4698|361|1990|3|1|10|1|1990|361|4698|Wednesday|1990Q1|N|N|N|2447893|2447892|2447537|2447810|N|N|N|N|N| +2447903|AAAAAAAAPBKFFCAA|1990-01-11|1080|4698|361|1990|4|1|11|1|1990|361|4698|Thursday|1990Q1|N|N|N|2447893|2447892|2447538|2447811|N|N|N|N|N| +2447904|AAAAAAAAACKFFCAA|1990-01-12|1080|4698|361|1990|5|1|12|1|1990|361|4698|Friday|1990Q1|N|Y|N|2447893|2447892|2447539|2447812|N|N|N|N|N| +2447905|AAAAAAAABCKFFCAA|1990-01-13|1080|4698|361|1990|6|1|13|1|1990|361|4698|Saturday|1990Q1|N|Y|N|2447893|2447892|2447540|2447813|N|N|N|N|N| +2447906|AAAAAAAACCKFFCAA|1990-01-14|1080|4698|361|1990|0|1|14|1|1990|361|4698|Sunday|1990Q1|N|N|N|2447893|2447892|2447541|2447814|N|N|N|N|N| +2447907|AAAAAAAADCKFFCAA|1990-01-15|1080|4698|361|1990|1|1|15|1|1990|361|4698|Monday|1990Q1|N|N|N|2447893|2447892|2447542|2447815|N|N|N|N|N| +2447908|AAAAAAAAECKFFCAA|1990-01-16|1080|4699|361|1990|2|1|16|1|1990|361|4699|Tuesday|1990Q1|N|N|N|2447893|2447892|2447543|2447816|N|N|N|N|N| +2447909|AAAAAAAAFCKFFCAA|1990-01-17|1080|4699|361|1990|3|1|17|1|1990|361|4699|Wednesday|1990Q1|N|N|N|2447893|2447892|2447544|2447817|N|N|N|N|N| +2447910|AAAAAAAAGCKFFCAA|1990-01-18|1080|4699|361|1990|4|1|18|1|1990|361|4699|Thursday|1990Q1|N|N|N|2447893|2447892|2447545|2447818|N|N|N|N|N| +2447911|AAAAAAAAHCKFFCAA|1990-01-19|1080|4699|361|1990|5|1|19|1|1990|361|4699|Friday|1990Q1|N|Y|N|2447893|2447892|2447546|2447819|N|N|N|N|N| +2447912|AAAAAAAAICKFFCAA|1990-01-20|1080|4699|361|1990|6|1|20|1|1990|361|4699|Saturday|1990Q1|N|Y|N|2447893|2447892|2447547|2447820|N|N|N|N|N| +2447913|AAAAAAAAJCKFFCAA|1990-01-21|1080|4699|361|1990|0|1|21|1|1990|361|4699|Sunday|1990Q1|N|N|N|2447893|2447892|2447548|2447821|N|N|N|N|N| +2447914|AAAAAAAAKCKFFCAA|1990-01-22|1080|4699|361|1990|1|1|22|1|1990|361|4699|Monday|1990Q1|N|N|N|2447893|2447892|2447549|2447822|N|N|N|N|N| +2447915|AAAAAAAALCKFFCAA|1990-01-23|1080|4700|361|1990|2|1|23|1|1990|361|4700|Tuesday|1990Q1|N|N|N|2447893|2447892|2447550|2447823|N|N|N|N|N| +2447916|AAAAAAAAMCKFFCAA|1990-01-24|1080|4700|361|1990|3|1|24|1|1990|361|4700|Wednesday|1990Q1|N|N|N|2447893|2447892|2447551|2447824|N|N|N|N|N| +2447917|AAAAAAAANCKFFCAA|1990-01-25|1080|4700|361|1990|4|1|25|1|1990|361|4700|Thursday|1990Q1|N|N|N|2447893|2447892|2447552|2447825|N|N|N|N|N| +2447918|AAAAAAAAOCKFFCAA|1990-01-26|1080|4700|361|1990|5|1|26|1|1990|361|4700|Friday|1990Q1|N|Y|N|2447893|2447892|2447553|2447826|N|N|N|N|N| +2447919|AAAAAAAAPCKFFCAA|1990-01-27|1080|4700|361|1990|6|1|27|1|1990|361|4700|Saturday|1990Q1|N|Y|N|2447893|2447892|2447554|2447827|N|N|N|N|N| +2447920|AAAAAAAAADKFFCAA|1990-01-28|1080|4700|361|1990|0|1|28|1|1990|361|4700|Sunday|1990Q1|N|N|N|2447893|2447892|2447555|2447828|N|N|N|N|N| +2447921|AAAAAAAABDKFFCAA|1990-01-29|1080|4700|361|1990|1|1|29|1|1990|361|4700|Monday|1990Q1|N|N|N|2447893|2447892|2447556|2447829|N|N|N|N|N| +2447922|AAAAAAAACDKFFCAA|1990-01-30|1080|4701|361|1990|2|1|30|1|1990|361|4701|Tuesday|1990Q1|N|N|N|2447893|2447892|2447557|2447830|N|N|N|N|N| +2447923|AAAAAAAADDKFFCAA|1990-01-31|1080|4701|361|1990|3|1|31|1|1990|361|4701|Wednesday|1990Q1|N|N|N|2447893|2447892|2447558|2447831|N|N|N|N|N| +2447924|AAAAAAAAEDKFFCAA|1990-02-01|1081|4701|361|1990|4|2|1|1|1990|361|4701|Thursday|1990Q1|N|N|N|2447924|2447954|2447559|2447832|N|N|N|N|N| +2447925|AAAAAAAAFDKFFCAA|1990-02-02|1081|4701|361|1990|5|2|2|1|1990|361|4701|Friday|1990Q1|N|Y|N|2447924|2447954|2447560|2447833|N|N|N|N|N| +2447926|AAAAAAAAGDKFFCAA|1990-02-03|1081|4701|361|1990|6|2|3|1|1990|361|4701|Saturday|1990Q1|N|Y|N|2447924|2447954|2447561|2447834|N|N|N|N|N| +2447927|AAAAAAAAHDKFFCAA|1990-02-04|1081|4701|361|1990|0|2|4|1|1990|361|4701|Sunday|1990Q1|N|N|N|2447924|2447954|2447562|2447835|N|N|N|N|N| +2447928|AAAAAAAAIDKFFCAA|1990-02-05|1081|4701|361|1990|1|2|5|1|1990|361|4701|Monday|1990Q1|N|N|N|2447924|2447954|2447563|2447836|N|N|N|N|N| +2447929|AAAAAAAAJDKFFCAA|1990-02-06|1081|4702|361|1990|2|2|6|1|1990|361|4702|Tuesday|1990Q1|N|N|N|2447924|2447954|2447564|2447837|N|N|N|N|N| +2447930|AAAAAAAAKDKFFCAA|1990-02-07|1081|4702|361|1990|3|2|7|1|1990|361|4702|Wednesday|1990Q1|N|N|N|2447924|2447954|2447565|2447838|N|N|N|N|N| +2447931|AAAAAAAALDKFFCAA|1990-02-08|1081|4702|361|1990|4|2|8|1|1990|361|4702|Thursday|1990Q1|N|N|N|2447924|2447954|2447566|2447839|N|N|N|N|N| +2447932|AAAAAAAAMDKFFCAA|1990-02-09|1081|4702|361|1990|5|2|9|1|1990|361|4702|Friday|1990Q1|N|Y|N|2447924|2447954|2447567|2447840|N|N|N|N|N| +2447933|AAAAAAAANDKFFCAA|1990-02-10|1081|4702|361|1990|6|2|10|1|1990|361|4702|Saturday|1990Q1|N|Y|N|2447924|2447954|2447568|2447841|N|N|N|N|N| +2447934|AAAAAAAAODKFFCAA|1990-02-11|1081|4702|361|1990|0|2|11|1|1990|361|4702|Sunday|1990Q1|N|N|N|2447924|2447954|2447569|2447842|N|N|N|N|N| +2447935|AAAAAAAAPDKFFCAA|1990-02-12|1081|4702|361|1990|1|2|12|1|1990|361|4702|Monday|1990Q1|N|N|N|2447924|2447954|2447570|2447843|N|N|N|N|N| +2447936|AAAAAAAAAEKFFCAA|1990-02-13|1081|4703|361|1990|2|2|13|1|1990|361|4703|Tuesday|1990Q1|N|N|N|2447924|2447954|2447571|2447844|N|N|N|N|N| +2447937|AAAAAAAABEKFFCAA|1990-02-14|1081|4703|361|1990|3|2|14|1|1990|361|4703|Wednesday|1990Q1|N|N|N|2447924|2447954|2447572|2447845|N|N|N|N|N| +2447938|AAAAAAAACEKFFCAA|1990-02-15|1081|4703|361|1990|4|2|15|1|1990|361|4703|Thursday|1990Q1|N|N|N|2447924|2447954|2447573|2447846|N|N|N|N|N| +2447939|AAAAAAAADEKFFCAA|1990-02-16|1081|4703|361|1990|5|2|16|1|1990|361|4703|Friday|1990Q1|N|Y|N|2447924|2447954|2447574|2447847|N|N|N|N|N| +2447940|AAAAAAAAEEKFFCAA|1990-02-17|1081|4703|361|1990|6|2|17|1|1990|361|4703|Saturday|1990Q1|N|Y|N|2447924|2447954|2447575|2447848|N|N|N|N|N| +2447941|AAAAAAAAFEKFFCAA|1990-02-18|1081|4703|361|1990|0|2|18|1|1990|361|4703|Sunday|1990Q1|N|N|N|2447924|2447954|2447576|2447849|N|N|N|N|N| +2447942|AAAAAAAAGEKFFCAA|1990-02-19|1081|4703|361|1990|1|2|19|1|1990|361|4703|Monday|1990Q1|N|N|N|2447924|2447954|2447577|2447850|N|N|N|N|N| +2447943|AAAAAAAAHEKFFCAA|1990-02-20|1081|4704|361|1990|2|2|20|1|1990|361|4704|Tuesday|1990Q1|N|N|N|2447924|2447954|2447578|2447851|N|N|N|N|N| +2447944|AAAAAAAAIEKFFCAA|1990-02-21|1081|4704|361|1990|3|2|21|1|1990|361|4704|Wednesday|1990Q1|N|N|N|2447924|2447954|2447579|2447852|N|N|N|N|N| +2447945|AAAAAAAAJEKFFCAA|1990-02-22|1081|4704|361|1990|4|2|22|1|1990|361|4704|Thursday|1990Q1|N|N|N|2447924|2447954|2447580|2447853|N|N|N|N|N| +2447946|AAAAAAAAKEKFFCAA|1990-02-23|1081|4704|361|1990|5|2|23|1|1990|361|4704|Friday|1990Q1|N|Y|N|2447924|2447954|2447581|2447854|N|N|N|N|N| +2447947|AAAAAAAALEKFFCAA|1990-02-24|1081|4704|361|1990|6|2|24|1|1990|361|4704|Saturday|1990Q1|N|Y|N|2447924|2447954|2447582|2447855|N|N|N|N|N| +2447948|AAAAAAAAMEKFFCAA|1990-02-25|1081|4704|361|1990|0|2|25|1|1990|361|4704|Sunday|1990Q1|N|N|N|2447924|2447954|2447583|2447856|N|N|N|N|N| +2447949|AAAAAAAANEKFFCAA|1990-02-26|1081|4704|361|1990|1|2|26|1|1990|361|4704|Monday|1990Q1|N|N|N|2447924|2447954|2447584|2447857|N|N|N|N|N| +2447950|AAAAAAAAOEKFFCAA|1990-02-27|1081|4705|361|1990|2|2|27|1|1990|361|4705|Tuesday|1990Q1|N|N|N|2447924|2447954|2447585|2447858|N|N|N|N|N| +2447951|AAAAAAAAPEKFFCAA|1990-02-28|1081|4705|361|1990|3|2|28|1|1990|361|4705|Wednesday|1990Q1|N|N|N|2447924|2447954|2447586|2447859|N|N|N|N|N| +2447952|AAAAAAAAAFKFFCAA|1990-03-01|1082|4705|362|1990|4|3|1|1|1990|362|4705|Thursday|1990Q1|N|N|N|2447952|2448010|2447587|2447860|N|N|N|N|N| +2447953|AAAAAAAABFKFFCAA|1990-03-02|1082|4705|362|1990|5|3|2|1|1990|362|4705|Friday|1990Q1|N|Y|N|2447952|2448010|2447588|2447861|N|N|N|N|N| +2447954|AAAAAAAACFKFFCAA|1990-03-03|1082|4705|362|1990|6|3|3|1|1990|362|4705|Saturday|1990Q1|N|Y|N|2447952|2448010|2447589|2447862|N|N|N|N|N| +2447955|AAAAAAAADFKFFCAA|1990-03-04|1082|4705|362|1990|0|3|4|1|1990|362|4705|Sunday|1990Q1|N|N|N|2447952|2448010|2447590|2447863|N|N|N|N|N| +2447956|AAAAAAAAEFKFFCAA|1990-03-05|1082|4705|362|1990|1|3|5|1|1990|362|4705|Monday|1990Q1|N|N|N|2447952|2448010|2447591|2447864|N|N|N|N|N| +2447957|AAAAAAAAFFKFFCAA|1990-03-06|1082|4706|362|1990|2|3|6|1|1990|362|4706|Tuesday|1990Q1|N|N|N|2447952|2448010|2447592|2447865|N|N|N|N|N| +2447958|AAAAAAAAGFKFFCAA|1990-03-07|1082|4706|362|1990|3|3|7|1|1990|362|4706|Wednesday|1990Q1|N|N|N|2447952|2448010|2447593|2447866|N|N|N|N|N| +2447959|AAAAAAAAHFKFFCAA|1990-03-08|1082|4706|362|1990|4|3|8|1|1990|362|4706|Thursday|1990Q1|N|N|N|2447952|2448010|2447594|2447867|N|N|N|N|N| +2447960|AAAAAAAAIFKFFCAA|1990-03-09|1082|4706|362|1990|5|3|9|1|1990|362|4706|Friday|1990Q1|N|Y|N|2447952|2448010|2447595|2447868|N|N|N|N|N| +2447961|AAAAAAAAJFKFFCAA|1990-03-10|1082|4706|362|1990|6|3|10|1|1990|362|4706|Saturday|1990Q1|N|Y|N|2447952|2448010|2447596|2447869|N|N|N|N|N| +2447962|AAAAAAAAKFKFFCAA|1990-03-11|1082|4706|362|1990|0|3|11|1|1990|362|4706|Sunday|1990Q1|N|N|N|2447952|2448010|2447597|2447870|N|N|N|N|N| +2447963|AAAAAAAALFKFFCAA|1990-03-12|1082|4706|362|1990|1|3|12|1|1990|362|4706|Monday|1990Q1|N|N|N|2447952|2448010|2447598|2447871|N|N|N|N|N| +2447964|AAAAAAAAMFKFFCAA|1990-03-13|1082|4707|362|1990|2|3|13|1|1990|362|4707|Tuesday|1990Q1|N|N|N|2447952|2448010|2447599|2447872|N|N|N|N|N| +2447965|AAAAAAAANFKFFCAA|1990-03-14|1082|4707|362|1990|3|3|14|1|1990|362|4707|Wednesday|1990Q1|N|N|N|2447952|2448010|2447600|2447873|N|N|N|N|N| +2447966|AAAAAAAAOFKFFCAA|1990-03-15|1082|4707|362|1990|4|3|15|1|1990|362|4707|Thursday|1990Q1|N|N|N|2447952|2448010|2447601|2447874|N|N|N|N|N| +2447967|AAAAAAAAPFKFFCAA|1990-03-16|1082|4707|362|1990|5|3|16|1|1990|362|4707|Friday|1990Q1|N|Y|N|2447952|2448010|2447602|2447875|N|N|N|N|N| +2447968|AAAAAAAAAGKFFCAA|1990-03-17|1082|4707|362|1990|6|3|17|1|1990|362|4707|Saturday|1990Q1|N|Y|N|2447952|2448010|2447603|2447876|N|N|N|N|N| +2447969|AAAAAAAABGKFFCAA|1990-03-18|1082|4707|362|1990|0|3|18|1|1990|362|4707|Sunday|1990Q1|N|N|N|2447952|2448010|2447604|2447877|N|N|N|N|N| +2447970|AAAAAAAACGKFFCAA|1990-03-19|1082|4707|362|1990|1|3|19|1|1990|362|4707|Monday|1990Q1|N|N|N|2447952|2448010|2447605|2447878|N|N|N|N|N| +2447971|AAAAAAAADGKFFCAA|1990-03-20|1082|4708|362|1990|2|3|20|1|1990|362|4708|Tuesday|1990Q1|N|N|N|2447952|2448010|2447606|2447879|N|N|N|N|N| +2447972|AAAAAAAAEGKFFCAA|1990-03-21|1082|4708|362|1990|3|3|21|1|1990|362|4708|Wednesday|1990Q1|N|N|N|2447952|2448010|2447607|2447880|N|N|N|N|N| +2447973|AAAAAAAAFGKFFCAA|1990-03-22|1082|4708|362|1990|4|3|22|1|1990|362|4708|Thursday|1990Q1|N|N|N|2447952|2448010|2447608|2447881|N|N|N|N|N| +2447974|AAAAAAAAGGKFFCAA|1990-03-23|1082|4708|362|1990|5|3|23|1|1990|362|4708|Friday|1990Q1|N|Y|N|2447952|2448010|2447609|2447882|N|N|N|N|N| +2447975|AAAAAAAAHGKFFCAA|1990-03-24|1082|4708|362|1990|6|3|24|1|1990|362|4708|Saturday|1990Q1|N|Y|N|2447952|2448010|2447610|2447883|N|N|N|N|N| +2447976|AAAAAAAAIGKFFCAA|1990-03-25|1082|4708|362|1990|0|3|25|1|1990|362|4708|Sunday|1990Q1|N|N|N|2447952|2448010|2447611|2447884|N|N|N|N|N| +2447977|AAAAAAAAJGKFFCAA|1990-03-26|1082|4708|362|1990|1|3|26|1|1990|362|4708|Monday|1990Q1|N|N|N|2447952|2448010|2447612|2447885|N|N|N|N|N| +2447978|AAAAAAAAKGKFFCAA|1990-03-27|1082|4709|362|1990|2|3|27|1|1990|362|4709|Tuesday|1990Q1|N|N|N|2447952|2448010|2447613|2447886|N|N|N|N|N| +2447979|AAAAAAAALGKFFCAA|1990-03-28|1082|4709|362|1990|3|3|28|1|1990|362|4709|Wednesday|1990Q1|N|N|N|2447952|2448010|2447614|2447887|N|N|N|N|N| +2447980|AAAAAAAAMGKFFCAA|1990-03-29|1082|4709|362|1990|4|3|29|1|1990|362|4709|Thursday|1990Q1|N|N|N|2447952|2448010|2447615|2447888|N|N|N|N|N| +2447981|AAAAAAAANGKFFCAA|1990-03-30|1082|4709|362|1990|5|3|30|1|1990|362|4709|Friday|1990Q1|N|Y|N|2447952|2448010|2447616|2447889|N|N|N|N|N| +2447982|AAAAAAAAOGKFFCAA|1990-03-31|1082|4709|362|1990|6|3|31|1|1990|362|4709|Saturday|1990Q1|N|Y|N|2447952|2448010|2447617|2447890|N|N|N|N|N| +2447983|AAAAAAAAPGKFFCAA|1990-04-01|1083|4709|362|1990|0|4|1|1|1990|362|4709|Sunday|1990Q1|N|N|N|2447983|2448072|2447618|2447893|N|N|N|N|N| +2447984|AAAAAAAAAHKFFCAA|1990-04-02|1083|4709|362|1990|1|4|2|2|1990|362|4709|Monday|1990Q2|N|N|N|2447983|2448072|2447619|2447894|N|N|N|N|N| +2447985|AAAAAAAABHKFFCAA|1990-04-03|1083|4710|362|1990|2|4|3|2|1990|362|4710|Tuesday|1990Q2|N|N|N|2447983|2448072|2447620|2447895|N|N|N|N|N| +2447986|AAAAAAAACHKFFCAA|1990-04-04|1083|4710|362|1990|3|4|4|2|1990|362|4710|Wednesday|1990Q2|N|N|N|2447983|2448072|2447621|2447896|N|N|N|N|N| +2447987|AAAAAAAADHKFFCAA|1990-04-05|1083|4710|362|1990|4|4|5|2|1990|362|4710|Thursday|1990Q2|N|N|N|2447983|2448072|2447622|2447897|N|N|N|N|N| +2447988|AAAAAAAAEHKFFCAA|1990-04-06|1083|4710|362|1990|5|4|6|2|1990|362|4710|Friday|1990Q2|N|Y|N|2447983|2448072|2447623|2447898|N|N|N|N|N| +2447989|AAAAAAAAFHKFFCAA|1990-04-07|1083|4710|362|1990|6|4|7|2|1990|362|4710|Saturday|1990Q2|N|Y|N|2447983|2448072|2447624|2447899|N|N|N|N|N| +2447990|AAAAAAAAGHKFFCAA|1990-04-08|1083|4710|362|1990|0|4|8|2|1990|362|4710|Sunday|1990Q2|N|N|N|2447983|2448072|2447625|2447900|N|N|N|N|N| +2447991|AAAAAAAAHHKFFCAA|1990-04-09|1083|4710|362|1990|1|4|9|2|1990|362|4710|Monday|1990Q2|N|N|N|2447983|2448072|2447626|2447901|N|N|N|N|N| +2447992|AAAAAAAAIHKFFCAA|1990-04-10|1083|4711|362|1990|2|4|10|2|1990|362|4711|Tuesday|1990Q2|N|N|N|2447983|2448072|2447627|2447902|N|N|N|N|N| +2447993|AAAAAAAAJHKFFCAA|1990-04-11|1083|4711|362|1990|3|4|11|2|1990|362|4711|Wednesday|1990Q2|N|N|N|2447983|2448072|2447628|2447903|N|N|N|N|N| +2447994|AAAAAAAAKHKFFCAA|1990-04-12|1083|4711|362|1990|4|4|12|2|1990|362|4711|Thursday|1990Q2|N|N|N|2447983|2448072|2447629|2447904|N|N|N|N|N| +2447995|AAAAAAAALHKFFCAA|1990-04-13|1083|4711|362|1990|5|4|13|2|1990|362|4711|Friday|1990Q2|N|Y|N|2447983|2448072|2447630|2447905|N|N|N|N|N| +2447996|AAAAAAAAMHKFFCAA|1990-04-14|1083|4711|362|1990|6|4|14|2|1990|362|4711|Saturday|1990Q2|N|Y|N|2447983|2448072|2447631|2447906|N|N|N|N|N| +2447997|AAAAAAAANHKFFCAA|1990-04-15|1083|4711|362|1990|0|4|15|2|1990|362|4711|Sunday|1990Q2|N|N|N|2447983|2448072|2447632|2447907|N|N|N|N|N| +2447998|AAAAAAAAOHKFFCAA|1990-04-16|1083|4711|362|1990|1|4|16|2|1990|362|4711|Monday|1990Q2|N|N|N|2447983|2448072|2447633|2447908|N|N|N|N|N| +2447999|AAAAAAAAPHKFFCAA|1990-04-17|1083|4712|362|1990|2|4|17|2|1990|362|4712|Tuesday|1990Q2|N|N|N|2447983|2448072|2447634|2447909|N|N|N|N|N| +2448000|AAAAAAAAAIKFFCAA|1990-04-18|1083|4712|362|1990|3|4|18|2|1990|362|4712|Wednesday|1990Q2|N|N|N|2447983|2448072|2447635|2447910|N|N|N|N|N| +2448001|AAAAAAAABIKFFCAA|1990-04-19|1083|4712|362|1990|4|4|19|2|1990|362|4712|Thursday|1990Q2|N|N|N|2447983|2448072|2447636|2447911|N|N|N|N|N| +2448002|AAAAAAAACIKFFCAA|1990-04-20|1083|4712|362|1990|5|4|20|2|1990|362|4712|Friday|1990Q2|N|Y|N|2447983|2448072|2447637|2447912|N|N|N|N|N| +2448003|AAAAAAAADIKFFCAA|1990-04-21|1083|4712|362|1990|6|4|21|2|1990|362|4712|Saturday|1990Q2|N|Y|N|2447983|2448072|2447638|2447913|N|N|N|N|N| +2448004|AAAAAAAAEIKFFCAA|1990-04-22|1083|4712|362|1990|0|4|22|2|1990|362|4712|Sunday|1990Q2|N|N|N|2447983|2448072|2447639|2447914|N|N|N|N|N| +2448005|AAAAAAAAFIKFFCAA|1990-04-23|1083|4712|362|1990|1|4|23|2|1990|362|4712|Monday|1990Q2|N|N|N|2447983|2448072|2447640|2447915|N|N|N|N|N| +2448006|AAAAAAAAGIKFFCAA|1990-04-24|1083|4713|362|1990|2|4|24|2|1990|362|4713|Tuesday|1990Q2|N|N|N|2447983|2448072|2447641|2447916|N|N|N|N|N| +2448007|AAAAAAAAHIKFFCAA|1990-04-25|1083|4713|362|1990|3|4|25|2|1990|362|4713|Wednesday|1990Q2|N|N|N|2447983|2448072|2447642|2447917|N|N|N|N|N| +2448008|AAAAAAAAIIKFFCAA|1990-04-26|1083|4713|362|1990|4|4|26|2|1990|362|4713|Thursday|1990Q2|N|N|N|2447983|2448072|2447643|2447918|N|N|N|N|N| +2448009|AAAAAAAAJIKFFCAA|1990-04-27|1083|4713|362|1990|5|4|27|2|1990|362|4713|Friday|1990Q2|N|Y|N|2447983|2448072|2447644|2447919|N|N|N|N|N| +2448010|AAAAAAAAKIKFFCAA|1990-04-28|1083|4713|362|1990|6|4|28|2|1990|362|4713|Saturday|1990Q2|N|Y|N|2447983|2448072|2447645|2447920|N|N|N|N|N| +2448011|AAAAAAAALIKFFCAA|1990-04-29|1083|4713|362|1990|0|4|29|2|1990|362|4713|Sunday|1990Q2|N|N|N|2447983|2448072|2447646|2447921|N|N|N|N|N| +2448012|AAAAAAAAMIKFFCAA|1990-04-30|1083|4713|362|1990|1|4|30|2|1990|362|4713|Monday|1990Q2|N|N|N|2447983|2448072|2447647|2447922|N|N|N|N|N| +2448013|AAAAAAAANIKFFCAA|1990-05-01|1084|4714|362|1990|2|5|1|2|1990|362|4714|Tuesday|1990Q2|N|N|N|2448013|2448132|2447648|2447923|N|N|N|N|N| +2448014|AAAAAAAAOIKFFCAA|1990-05-02|1084|4714|362|1990|3|5|2|2|1990|362|4714|Wednesday|1990Q2|N|N|N|2448013|2448132|2447649|2447924|N|N|N|N|N| +2448015|AAAAAAAAPIKFFCAA|1990-05-03|1084|4714|362|1990|4|5|3|2|1990|362|4714|Thursday|1990Q2|N|N|N|2448013|2448132|2447650|2447925|N|N|N|N|N| +2448016|AAAAAAAAAJKFFCAA|1990-05-04|1084|4714|362|1990|5|5|4|2|1990|362|4714|Friday|1990Q2|N|Y|N|2448013|2448132|2447651|2447926|N|N|N|N|N| +2448017|AAAAAAAABJKFFCAA|1990-05-05|1084|4714|362|1990|6|5|5|2|1990|362|4714|Saturday|1990Q2|N|Y|N|2448013|2448132|2447652|2447927|N|N|N|N|N| +2448018|AAAAAAAACJKFFCAA|1990-05-06|1084|4714|362|1990|0|5|6|2|1990|362|4714|Sunday|1990Q2|N|N|N|2448013|2448132|2447653|2447928|N|N|N|N|N| +2448019|AAAAAAAADJKFFCAA|1990-05-07|1084|4714|362|1990|1|5|7|2|1990|362|4714|Monday|1990Q2|N|N|N|2448013|2448132|2447654|2447929|N|N|N|N|N| +2448020|AAAAAAAAEJKFFCAA|1990-05-08|1084|4715|362|1990|2|5|8|2|1990|362|4715|Tuesday|1990Q2|N|N|N|2448013|2448132|2447655|2447930|N|N|N|N|N| +2448021|AAAAAAAAFJKFFCAA|1990-05-09|1084|4715|362|1990|3|5|9|2|1990|362|4715|Wednesday|1990Q2|N|N|N|2448013|2448132|2447656|2447931|N|N|N|N|N| +2448022|AAAAAAAAGJKFFCAA|1990-05-10|1084|4715|362|1990|4|5|10|2|1990|362|4715|Thursday|1990Q2|N|N|N|2448013|2448132|2447657|2447932|N|N|N|N|N| +2448023|AAAAAAAAHJKFFCAA|1990-05-11|1084|4715|362|1990|5|5|11|2|1990|362|4715|Friday|1990Q2|N|Y|N|2448013|2448132|2447658|2447933|N|N|N|N|N| +2448024|AAAAAAAAIJKFFCAA|1990-05-12|1084|4715|362|1990|6|5|12|2|1990|362|4715|Saturday|1990Q2|N|Y|N|2448013|2448132|2447659|2447934|N|N|N|N|N| +2448025|AAAAAAAAJJKFFCAA|1990-05-13|1084|4715|362|1990|0|5|13|2|1990|362|4715|Sunday|1990Q2|N|N|N|2448013|2448132|2447660|2447935|N|N|N|N|N| +2448026|AAAAAAAAKJKFFCAA|1990-05-14|1084|4715|362|1990|1|5|14|2|1990|362|4715|Monday|1990Q2|N|N|N|2448013|2448132|2447661|2447936|N|N|N|N|N| +2448027|AAAAAAAALJKFFCAA|1990-05-15|1084|4716|362|1990|2|5|15|2|1990|362|4716|Tuesday|1990Q2|N|N|N|2448013|2448132|2447662|2447937|N|N|N|N|N| +2448028|AAAAAAAAMJKFFCAA|1990-05-16|1084|4716|362|1990|3|5|16|2|1990|362|4716|Wednesday|1990Q2|N|N|N|2448013|2448132|2447663|2447938|N|N|N|N|N| +2448029|AAAAAAAANJKFFCAA|1990-05-17|1084|4716|362|1990|4|5|17|2|1990|362|4716|Thursday|1990Q2|N|N|N|2448013|2448132|2447664|2447939|N|N|N|N|N| +2448030|AAAAAAAAOJKFFCAA|1990-05-18|1084|4716|362|1990|5|5|18|2|1990|362|4716|Friday|1990Q2|N|Y|N|2448013|2448132|2447665|2447940|N|N|N|N|N| +2448031|AAAAAAAAPJKFFCAA|1990-05-19|1084|4716|362|1990|6|5|19|2|1990|362|4716|Saturday|1990Q2|N|Y|N|2448013|2448132|2447666|2447941|N|N|N|N|N| +2448032|AAAAAAAAAKKFFCAA|1990-05-20|1084|4716|362|1990|0|5|20|2|1990|362|4716|Sunday|1990Q2|N|N|N|2448013|2448132|2447667|2447942|N|N|N|N|N| +2448033|AAAAAAAABKKFFCAA|1990-05-21|1084|4716|362|1990|1|5|21|2|1990|362|4716|Monday|1990Q2|N|N|N|2448013|2448132|2447668|2447943|N|N|N|N|N| +2448034|AAAAAAAACKKFFCAA|1990-05-22|1084|4717|362|1990|2|5|22|2|1990|362|4717|Tuesday|1990Q2|N|N|N|2448013|2448132|2447669|2447944|N|N|N|N|N| +2448035|AAAAAAAADKKFFCAA|1990-05-23|1084|4717|362|1990|3|5|23|2|1990|362|4717|Wednesday|1990Q2|N|N|N|2448013|2448132|2447670|2447945|N|N|N|N|N| +2448036|AAAAAAAAEKKFFCAA|1990-05-24|1084|4717|362|1990|4|5|24|2|1990|362|4717|Thursday|1990Q2|N|N|N|2448013|2448132|2447671|2447946|N|N|N|N|N| +2448037|AAAAAAAAFKKFFCAA|1990-05-25|1084|4717|362|1990|5|5|25|2|1990|362|4717|Friday|1990Q2|N|Y|N|2448013|2448132|2447672|2447947|N|N|N|N|N| +2448038|AAAAAAAAGKKFFCAA|1990-05-26|1084|4717|362|1990|6|5|26|2|1990|362|4717|Saturday|1990Q2|N|Y|N|2448013|2448132|2447673|2447948|N|N|N|N|N| +2448039|AAAAAAAAHKKFFCAA|1990-05-27|1084|4717|362|1990|0|5|27|2|1990|362|4717|Sunday|1990Q2|N|N|N|2448013|2448132|2447674|2447949|N|N|N|N|N| +2448040|AAAAAAAAIKKFFCAA|1990-05-28|1084|4717|362|1990|1|5|28|2|1990|362|4717|Monday|1990Q2|N|N|N|2448013|2448132|2447675|2447950|N|N|N|N|N| +2448041|AAAAAAAAJKKFFCAA|1990-05-29|1084|4718|362|1990|2|5|29|2|1990|362|4718|Tuesday|1990Q2|N|N|N|2448013|2448132|2447676|2447951|N|N|N|N|N| +2448042|AAAAAAAAKKKFFCAA|1990-05-30|1084|4718|362|1990|3|5|30|2|1990|362|4718|Wednesday|1990Q2|N|N|N|2448013|2448132|2447677|2447952|N|N|N|N|N| +2448043|AAAAAAAALKKFFCAA|1990-05-31|1084|4718|362|1990|4|5|31|2|1990|362|4718|Thursday|1990Q2|N|N|N|2448013|2448132|2447678|2447953|N|N|N|N|N| +2448044|AAAAAAAAMKKFFCAA|1990-06-01|1085|4718|363|1990|5|6|1|2|1990|363|4718|Friday|1990Q2|N|Y|N|2448044|2448194|2447679|2447954|N|N|N|N|N| +2448045|AAAAAAAANKKFFCAA|1990-06-02|1085|4718|363|1990|6|6|2|2|1990|363|4718|Saturday|1990Q2|N|Y|N|2448044|2448194|2447680|2447955|N|N|N|N|N| +2448046|AAAAAAAAOKKFFCAA|1990-06-03|1085|4718|363|1990|0|6|3|2|1990|363|4718|Sunday|1990Q2|N|N|N|2448044|2448194|2447681|2447956|N|N|N|N|N| +2448047|AAAAAAAAPKKFFCAA|1990-06-04|1085|4718|363|1990|1|6|4|2|1990|363|4718|Monday|1990Q2|N|N|N|2448044|2448194|2447682|2447957|N|N|N|N|N| +2448048|AAAAAAAAALKFFCAA|1990-06-05|1085|4719|363|1990|2|6|5|2|1990|363|4719|Tuesday|1990Q2|N|N|N|2448044|2448194|2447683|2447958|N|N|N|N|N| +2448049|AAAAAAAABLKFFCAA|1990-06-06|1085|4719|363|1990|3|6|6|2|1990|363|4719|Wednesday|1990Q2|N|N|N|2448044|2448194|2447684|2447959|N|N|N|N|N| +2448050|AAAAAAAACLKFFCAA|1990-06-07|1085|4719|363|1990|4|6|7|2|1990|363|4719|Thursday|1990Q2|N|N|N|2448044|2448194|2447685|2447960|N|N|N|N|N| +2448051|AAAAAAAADLKFFCAA|1990-06-08|1085|4719|363|1990|5|6|8|2|1990|363|4719|Friday|1990Q2|N|Y|N|2448044|2448194|2447686|2447961|N|N|N|N|N| +2448052|AAAAAAAAELKFFCAA|1990-06-09|1085|4719|363|1990|6|6|9|2|1990|363|4719|Saturday|1990Q2|N|Y|N|2448044|2448194|2447687|2447962|N|N|N|N|N| +2448053|AAAAAAAAFLKFFCAA|1990-06-10|1085|4719|363|1990|0|6|10|2|1990|363|4719|Sunday|1990Q2|N|N|N|2448044|2448194|2447688|2447963|N|N|N|N|N| +2448054|AAAAAAAAGLKFFCAA|1990-06-11|1085|4719|363|1990|1|6|11|2|1990|363|4719|Monday|1990Q2|N|N|N|2448044|2448194|2447689|2447964|N|N|N|N|N| +2448055|AAAAAAAAHLKFFCAA|1990-06-12|1085|4720|363|1990|2|6|12|2|1990|363|4720|Tuesday|1990Q2|N|N|N|2448044|2448194|2447690|2447965|N|N|N|N|N| +2448056|AAAAAAAAILKFFCAA|1990-06-13|1085|4720|363|1990|3|6|13|2|1990|363|4720|Wednesday|1990Q2|N|N|N|2448044|2448194|2447691|2447966|N|N|N|N|N| +2448057|AAAAAAAAJLKFFCAA|1990-06-14|1085|4720|363|1990|4|6|14|2|1990|363|4720|Thursday|1990Q2|N|N|N|2448044|2448194|2447692|2447967|N|N|N|N|N| +2448058|AAAAAAAAKLKFFCAA|1990-06-15|1085|4720|363|1990|5|6|15|2|1990|363|4720|Friday|1990Q2|N|Y|N|2448044|2448194|2447693|2447968|N|N|N|N|N| +2448059|AAAAAAAALLKFFCAA|1990-06-16|1085|4720|363|1990|6|6|16|2|1990|363|4720|Saturday|1990Q2|N|Y|N|2448044|2448194|2447694|2447969|N|N|N|N|N| +2448060|AAAAAAAAMLKFFCAA|1990-06-17|1085|4720|363|1990|0|6|17|2|1990|363|4720|Sunday|1990Q2|N|N|N|2448044|2448194|2447695|2447970|N|N|N|N|N| +2448061|AAAAAAAANLKFFCAA|1990-06-18|1085|4720|363|1990|1|6|18|2|1990|363|4720|Monday|1990Q2|N|N|N|2448044|2448194|2447696|2447971|N|N|N|N|N| +2448062|AAAAAAAAOLKFFCAA|1990-06-19|1085|4721|363|1990|2|6|19|2|1990|363|4721|Tuesday|1990Q2|N|N|N|2448044|2448194|2447697|2447972|N|N|N|N|N| +2448063|AAAAAAAAPLKFFCAA|1990-06-20|1085|4721|363|1990|3|6|20|2|1990|363|4721|Wednesday|1990Q2|N|N|N|2448044|2448194|2447698|2447973|N|N|N|N|N| +2448064|AAAAAAAAAMKFFCAA|1990-06-21|1085|4721|363|1990|4|6|21|2|1990|363|4721|Thursday|1990Q2|N|N|N|2448044|2448194|2447699|2447974|N|N|N|N|N| +2448065|AAAAAAAABMKFFCAA|1990-06-22|1085|4721|363|1990|5|6|22|2|1990|363|4721|Friday|1990Q2|N|Y|N|2448044|2448194|2447700|2447975|N|N|N|N|N| +2448066|AAAAAAAACMKFFCAA|1990-06-23|1085|4721|363|1990|6|6|23|2|1990|363|4721|Saturday|1990Q2|N|Y|N|2448044|2448194|2447701|2447976|N|N|N|N|N| +2448067|AAAAAAAADMKFFCAA|1990-06-24|1085|4721|363|1990|0|6|24|2|1990|363|4721|Sunday|1990Q2|N|N|N|2448044|2448194|2447702|2447977|N|N|N|N|N| +2448068|AAAAAAAAEMKFFCAA|1990-06-25|1085|4721|363|1990|1|6|25|2|1990|363|4721|Monday|1990Q2|N|N|N|2448044|2448194|2447703|2447978|N|N|N|N|N| +2448069|AAAAAAAAFMKFFCAA|1990-06-26|1085|4722|363|1990|2|6|26|2|1990|363|4722|Tuesday|1990Q2|N|N|N|2448044|2448194|2447704|2447979|N|N|N|N|N| +2448070|AAAAAAAAGMKFFCAA|1990-06-27|1085|4722|363|1990|3|6|27|2|1990|363|4722|Wednesday|1990Q2|N|N|N|2448044|2448194|2447705|2447980|N|N|N|N|N| +2448071|AAAAAAAAHMKFFCAA|1990-06-28|1085|4722|363|1990|4|6|28|2|1990|363|4722|Thursday|1990Q2|N|N|N|2448044|2448194|2447706|2447981|N|N|N|N|N| +2448072|AAAAAAAAIMKFFCAA|1990-06-29|1085|4722|363|1990|5|6|29|2|1990|363|4722|Friday|1990Q2|N|Y|N|2448044|2448194|2447707|2447982|N|N|N|N|N| +2448073|AAAAAAAAJMKFFCAA|1990-06-30|1085|4722|363|1990|6|6|30|2|1990|363|4722|Saturday|1990Q2|N|Y|N|2448044|2448194|2447708|2447983|N|N|N|N|N| +2448074|AAAAAAAAKMKFFCAA|1990-07-01|1086|4722|363|1990|0|7|1|2|1990|363|4722|Sunday|1990Q2|N|N|N|2448074|2448254|2447709|2447983|N|N|N|N|N| +2448075|AAAAAAAALMKFFCAA|1990-07-02|1086|4722|363|1990|1|7|2|3|1990|363|4722|Monday|1990Q3|N|N|N|2448074|2448254|2447710|2447984|N|N|N|N|N| +2448076|AAAAAAAAMMKFFCAA|1990-07-03|1086|4723|363|1990|2|7|3|3|1990|363|4723|Tuesday|1990Q3|N|N|N|2448074|2448254|2447711|2447985|N|N|N|N|N| +2448077|AAAAAAAANMKFFCAA|1990-07-04|1086|4723|363|1990|3|7|4|3|1990|363|4723|Wednesday|1990Q3|N|N|N|2448074|2448254|2447712|2447986|N|N|N|N|N| +2448078|AAAAAAAAOMKFFCAA|1990-07-05|1086|4723|363|1990|4|7|5|3|1990|363|4723|Thursday|1990Q3|Y|N|N|2448074|2448254|2447713|2447987|N|N|N|N|N| +2448079|AAAAAAAAPMKFFCAA|1990-07-06|1086|4723|363|1990|5|7|6|3|1990|363|4723|Friday|1990Q3|N|Y|Y|2448074|2448254|2447714|2447988|N|N|N|N|N| +2448080|AAAAAAAAANKFFCAA|1990-07-07|1086|4723|363|1990|6|7|7|3|1990|363|4723|Saturday|1990Q3|N|Y|N|2448074|2448254|2447715|2447989|N|N|N|N|N| +2448081|AAAAAAAABNKFFCAA|1990-07-08|1086|4723|363|1990|0|7|8|3|1990|363|4723|Sunday|1990Q3|N|N|N|2448074|2448254|2447716|2447990|N|N|N|N|N| +2448082|AAAAAAAACNKFFCAA|1990-07-09|1086|4723|363|1990|1|7|9|3|1990|363|4723|Monday|1990Q3|N|N|N|2448074|2448254|2447717|2447991|N|N|N|N|N| +2448083|AAAAAAAADNKFFCAA|1990-07-10|1086|4724|363|1990|2|7|10|3|1990|363|4724|Tuesday|1990Q3|N|N|N|2448074|2448254|2447718|2447992|N|N|N|N|N| +2448084|AAAAAAAAENKFFCAA|1990-07-11|1086|4724|363|1990|3|7|11|3|1990|363|4724|Wednesday|1990Q3|N|N|N|2448074|2448254|2447719|2447993|N|N|N|N|N| +2448085|AAAAAAAAFNKFFCAA|1990-07-12|1086|4724|363|1990|4|7|12|3|1990|363|4724|Thursday|1990Q3|N|N|N|2448074|2448254|2447720|2447994|N|N|N|N|N| +2448086|AAAAAAAAGNKFFCAA|1990-07-13|1086|4724|363|1990|5|7|13|3|1990|363|4724|Friday|1990Q3|N|Y|N|2448074|2448254|2447721|2447995|N|N|N|N|N| +2448087|AAAAAAAAHNKFFCAA|1990-07-14|1086|4724|363|1990|6|7|14|3|1990|363|4724|Saturday|1990Q3|N|Y|N|2448074|2448254|2447722|2447996|N|N|N|N|N| +2448088|AAAAAAAAINKFFCAA|1990-07-15|1086|4724|363|1990|0|7|15|3|1990|363|4724|Sunday|1990Q3|N|N|N|2448074|2448254|2447723|2447997|N|N|N|N|N| +2448089|AAAAAAAAJNKFFCAA|1990-07-16|1086|4724|363|1990|1|7|16|3|1990|363|4724|Monday|1990Q3|N|N|N|2448074|2448254|2447724|2447998|N|N|N|N|N| +2448090|AAAAAAAAKNKFFCAA|1990-07-17|1086|4725|363|1990|2|7|17|3|1990|363|4725|Tuesday|1990Q3|N|N|N|2448074|2448254|2447725|2447999|N|N|N|N|N| +2448091|AAAAAAAALNKFFCAA|1990-07-18|1086|4725|363|1990|3|7|18|3|1990|363|4725|Wednesday|1990Q3|N|N|N|2448074|2448254|2447726|2448000|N|N|N|N|N| +2448092|AAAAAAAAMNKFFCAA|1990-07-19|1086|4725|363|1990|4|7|19|3|1990|363|4725|Thursday|1990Q3|N|N|N|2448074|2448254|2447727|2448001|N|N|N|N|N| +2448093|AAAAAAAANNKFFCAA|1990-07-20|1086|4725|363|1990|5|7|20|3|1990|363|4725|Friday|1990Q3|N|Y|N|2448074|2448254|2447728|2448002|N|N|N|N|N| +2448094|AAAAAAAAONKFFCAA|1990-07-21|1086|4725|363|1990|6|7|21|3|1990|363|4725|Saturday|1990Q3|N|Y|N|2448074|2448254|2447729|2448003|N|N|N|N|N| +2448095|AAAAAAAAPNKFFCAA|1990-07-22|1086|4725|363|1990|0|7|22|3|1990|363|4725|Sunday|1990Q3|N|N|N|2448074|2448254|2447730|2448004|N|N|N|N|N| +2448096|AAAAAAAAAOKFFCAA|1990-07-23|1086|4725|363|1990|1|7|23|3|1990|363|4725|Monday|1990Q3|N|N|N|2448074|2448254|2447731|2448005|N|N|N|N|N| +2448097|AAAAAAAABOKFFCAA|1990-07-24|1086|4726|363|1990|2|7|24|3|1990|363|4726|Tuesday|1990Q3|N|N|N|2448074|2448254|2447732|2448006|N|N|N|N|N| +2448098|AAAAAAAACOKFFCAA|1990-07-25|1086|4726|363|1990|3|7|25|3|1990|363|4726|Wednesday|1990Q3|N|N|N|2448074|2448254|2447733|2448007|N|N|N|N|N| +2448099|AAAAAAAADOKFFCAA|1990-07-26|1086|4726|363|1990|4|7|26|3|1990|363|4726|Thursday|1990Q3|N|N|N|2448074|2448254|2447734|2448008|N|N|N|N|N| +2448100|AAAAAAAAEOKFFCAA|1990-07-27|1086|4726|363|1990|5|7|27|3|1990|363|4726|Friday|1990Q3|N|Y|N|2448074|2448254|2447735|2448009|N|N|N|N|N| +2448101|AAAAAAAAFOKFFCAA|1990-07-28|1086|4726|363|1990|6|7|28|3|1990|363|4726|Saturday|1990Q3|N|Y|N|2448074|2448254|2447736|2448010|N|N|N|N|N| +2448102|AAAAAAAAGOKFFCAA|1990-07-29|1086|4726|363|1990|0|7|29|3|1990|363|4726|Sunday|1990Q3|N|N|N|2448074|2448254|2447737|2448011|N|N|N|N|N| +2448103|AAAAAAAAHOKFFCAA|1990-07-30|1086|4726|363|1990|1|7|30|3|1990|363|4726|Monday|1990Q3|N|N|N|2448074|2448254|2447738|2448012|N|N|N|N|N| +2448104|AAAAAAAAIOKFFCAA|1990-07-31|1086|4727|363|1990|2|7|31|3|1990|363|4727|Tuesday|1990Q3|N|N|N|2448074|2448254|2447739|2448013|N|N|N|N|N| +2448105|AAAAAAAAJOKFFCAA|1990-08-01|1087|4727|363|1990|3|8|1|3|1990|363|4727|Wednesday|1990Q3|N|N|N|2448105|2448316|2447740|2448014|N|N|N|N|N| +2448106|AAAAAAAAKOKFFCAA|1990-08-02|1087|4727|363|1990|4|8|2|3|1990|363|4727|Thursday|1990Q3|N|N|N|2448105|2448316|2447741|2448015|N|N|N|N|N| +2448107|AAAAAAAALOKFFCAA|1990-08-03|1087|4727|363|1990|5|8|3|3|1990|363|4727|Friday|1990Q3|N|Y|N|2448105|2448316|2447742|2448016|N|N|N|N|N| +2448108|AAAAAAAAMOKFFCAA|1990-08-04|1087|4727|363|1990|6|8|4|3|1990|363|4727|Saturday|1990Q3|N|Y|N|2448105|2448316|2447743|2448017|N|N|N|N|N| +2448109|AAAAAAAANOKFFCAA|1990-08-05|1087|4727|363|1990|0|8|5|3|1990|363|4727|Sunday|1990Q3|N|N|N|2448105|2448316|2447744|2448018|N|N|N|N|N| +2448110|AAAAAAAAOOKFFCAA|1990-08-06|1087|4727|363|1990|1|8|6|3|1990|363|4727|Monday|1990Q3|N|N|N|2448105|2448316|2447745|2448019|N|N|N|N|N| +2448111|AAAAAAAAPOKFFCAA|1990-08-07|1087|4728|363|1990|2|8|7|3|1990|363|4728|Tuesday|1990Q3|N|N|N|2448105|2448316|2447746|2448020|N|N|N|N|N| +2448112|AAAAAAAAAPKFFCAA|1990-08-08|1087|4728|363|1990|3|8|8|3|1990|363|4728|Wednesday|1990Q3|N|N|N|2448105|2448316|2447747|2448021|N|N|N|N|N| +2448113|AAAAAAAABPKFFCAA|1990-08-09|1087|4728|363|1990|4|8|9|3|1990|363|4728|Thursday|1990Q3|N|N|N|2448105|2448316|2447748|2448022|N|N|N|N|N| +2448114|AAAAAAAACPKFFCAA|1990-08-10|1087|4728|363|1990|5|8|10|3|1990|363|4728|Friday|1990Q3|N|Y|N|2448105|2448316|2447749|2448023|N|N|N|N|N| +2448115|AAAAAAAADPKFFCAA|1990-08-11|1087|4728|363|1990|6|8|11|3|1990|363|4728|Saturday|1990Q3|N|Y|N|2448105|2448316|2447750|2448024|N|N|N|N|N| +2448116|AAAAAAAAEPKFFCAA|1990-08-12|1087|4728|363|1990|0|8|12|3|1990|363|4728|Sunday|1990Q3|N|N|N|2448105|2448316|2447751|2448025|N|N|N|N|N| +2448117|AAAAAAAAFPKFFCAA|1990-08-13|1087|4728|363|1990|1|8|13|3|1990|363|4728|Monday|1990Q3|N|N|N|2448105|2448316|2447752|2448026|N|N|N|N|N| +2448118|AAAAAAAAGPKFFCAA|1990-08-14|1087|4729|363|1990|2|8|14|3|1990|363|4729|Tuesday|1990Q3|N|N|N|2448105|2448316|2447753|2448027|N|N|N|N|N| +2448119|AAAAAAAAHPKFFCAA|1990-08-15|1087|4729|363|1990|3|8|15|3|1990|363|4729|Wednesday|1990Q3|N|N|N|2448105|2448316|2447754|2448028|N|N|N|N|N| +2448120|AAAAAAAAIPKFFCAA|1990-08-16|1087|4729|363|1990|4|8|16|3|1990|363|4729|Thursday|1990Q3|N|N|N|2448105|2448316|2447755|2448029|N|N|N|N|N| +2448121|AAAAAAAAJPKFFCAA|1990-08-17|1087|4729|363|1990|5|8|17|3|1990|363|4729|Friday|1990Q3|N|Y|N|2448105|2448316|2447756|2448030|N|N|N|N|N| +2448122|AAAAAAAAKPKFFCAA|1990-08-18|1087|4729|363|1990|6|8|18|3|1990|363|4729|Saturday|1990Q3|N|Y|N|2448105|2448316|2447757|2448031|N|N|N|N|N| +2448123|AAAAAAAALPKFFCAA|1990-08-19|1087|4729|363|1990|0|8|19|3|1990|363|4729|Sunday|1990Q3|N|N|N|2448105|2448316|2447758|2448032|N|N|N|N|N| +2448124|AAAAAAAAMPKFFCAA|1990-08-20|1087|4729|363|1990|1|8|20|3|1990|363|4729|Monday|1990Q3|N|N|N|2448105|2448316|2447759|2448033|N|N|N|N|N| +2448125|AAAAAAAANPKFFCAA|1990-08-21|1087|4730|363|1990|2|8|21|3|1990|363|4730|Tuesday|1990Q3|N|N|N|2448105|2448316|2447760|2448034|N|N|N|N|N| +2448126|AAAAAAAAOPKFFCAA|1990-08-22|1087|4730|363|1990|3|8|22|3|1990|363|4730|Wednesday|1990Q3|N|N|N|2448105|2448316|2447761|2448035|N|N|N|N|N| +2448127|AAAAAAAAPPKFFCAA|1990-08-23|1087|4730|363|1990|4|8|23|3|1990|363|4730|Thursday|1990Q3|N|N|N|2448105|2448316|2447762|2448036|N|N|N|N|N| +2448128|AAAAAAAAAALFFCAA|1990-08-24|1087|4730|363|1990|5|8|24|3|1990|363|4730|Friday|1990Q3|N|Y|N|2448105|2448316|2447763|2448037|N|N|N|N|N| +2448129|AAAAAAAABALFFCAA|1990-08-25|1087|4730|363|1990|6|8|25|3|1990|363|4730|Saturday|1990Q3|N|Y|N|2448105|2448316|2447764|2448038|N|N|N|N|N| +2448130|AAAAAAAACALFFCAA|1990-08-26|1087|4730|363|1990|0|8|26|3|1990|363|4730|Sunday|1990Q3|N|N|N|2448105|2448316|2447765|2448039|N|N|N|N|N| +2448131|AAAAAAAADALFFCAA|1990-08-27|1087|4730|363|1990|1|8|27|3|1990|363|4730|Monday|1990Q3|N|N|N|2448105|2448316|2447766|2448040|N|N|N|N|N| +2448132|AAAAAAAAEALFFCAA|1990-08-28|1087|4731|363|1990|2|8|28|3|1990|363|4731|Tuesday|1990Q3|N|N|N|2448105|2448316|2447767|2448041|N|N|N|N|N| +2448133|AAAAAAAAFALFFCAA|1990-08-29|1087|4731|363|1990|3|8|29|3|1990|363|4731|Wednesday|1990Q3|N|N|N|2448105|2448316|2447768|2448042|N|N|N|N|N| +2448134|AAAAAAAAGALFFCAA|1990-08-30|1087|4731|363|1990|4|8|30|3|1990|363|4731|Thursday|1990Q3|N|N|N|2448105|2448316|2447769|2448043|N|N|N|N|N| +2448135|AAAAAAAAHALFFCAA|1990-08-31|1087|4731|363|1990|5|8|31|3|1990|363|4731|Friday|1990Q3|N|Y|N|2448105|2448316|2447770|2448044|N|N|N|N|N| +2448136|AAAAAAAAIALFFCAA|1990-09-01|1088|4731|364|1990|6|9|1|3|1990|364|4731|Saturday|1990Q3|N|Y|N|2448136|2448378|2447771|2448045|N|N|N|N|N| +2448137|AAAAAAAAJALFFCAA|1990-09-02|1088|4731|364|1990|0|9|2|3|1990|364|4731|Sunday|1990Q3|N|N|N|2448136|2448378|2447772|2448046|N|N|N|N|N| +2448138|AAAAAAAAKALFFCAA|1990-09-03|1088|4731|364|1990|1|9|3|3|1990|364|4731|Monday|1990Q3|N|N|N|2448136|2448378|2447773|2448047|N|N|N|N|N| +2448139|AAAAAAAALALFFCAA|1990-09-04|1088|4732|364|1990|2|9|4|3|1990|364|4732|Tuesday|1990Q3|N|N|N|2448136|2448378|2447774|2448048|N|N|N|N|N| +2448140|AAAAAAAAMALFFCAA|1990-09-05|1088|4732|364|1990|3|9|5|3|1990|364|4732|Wednesday|1990Q3|N|N|N|2448136|2448378|2447775|2448049|N|N|N|N|N| +2448141|AAAAAAAANALFFCAA|1990-09-06|1088|4732|364|1990|4|9|6|3|1990|364|4732|Thursday|1990Q3|N|N|N|2448136|2448378|2447776|2448050|N|N|N|N|N| +2448142|AAAAAAAAOALFFCAA|1990-09-07|1088|4732|364|1990|5|9|7|3|1990|364|4732|Friday|1990Q3|N|Y|N|2448136|2448378|2447777|2448051|N|N|N|N|N| +2448143|AAAAAAAAPALFFCAA|1990-09-08|1088|4732|364|1990|6|9|8|3|1990|364|4732|Saturday|1990Q3|N|Y|N|2448136|2448378|2447778|2448052|N|N|N|N|N| +2448144|AAAAAAAAABLFFCAA|1990-09-09|1088|4732|364|1990|0|9|9|3|1990|364|4732|Sunday|1990Q3|N|N|N|2448136|2448378|2447779|2448053|N|N|N|N|N| +2448145|AAAAAAAABBLFFCAA|1990-09-10|1088|4732|364|1990|1|9|10|3|1990|364|4732|Monday|1990Q3|N|N|N|2448136|2448378|2447780|2448054|N|N|N|N|N| +2448146|AAAAAAAACBLFFCAA|1990-09-11|1088|4733|364|1990|2|9|11|3|1990|364|4733|Tuesday|1990Q3|N|N|N|2448136|2448378|2447781|2448055|N|N|N|N|N| +2448147|AAAAAAAADBLFFCAA|1990-09-12|1088|4733|364|1990|3|9|12|3|1990|364|4733|Wednesday|1990Q3|N|N|N|2448136|2448378|2447782|2448056|N|N|N|N|N| +2448148|AAAAAAAAEBLFFCAA|1990-09-13|1088|4733|364|1990|4|9|13|3|1990|364|4733|Thursday|1990Q3|N|N|N|2448136|2448378|2447783|2448057|N|N|N|N|N| +2448149|AAAAAAAAFBLFFCAA|1990-09-14|1088|4733|364|1990|5|9|14|3|1990|364|4733|Friday|1990Q3|N|Y|N|2448136|2448378|2447784|2448058|N|N|N|N|N| +2448150|AAAAAAAAGBLFFCAA|1990-09-15|1088|4733|364|1990|6|9|15|3|1990|364|4733|Saturday|1990Q3|N|Y|N|2448136|2448378|2447785|2448059|N|N|N|N|N| +2448151|AAAAAAAAHBLFFCAA|1990-09-16|1088|4733|364|1990|0|9|16|3|1990|364|4733|Sunday|1990Q3|N|N|N|2448136|2448378|2447786|2448060|N|N|N|N|N| +2448152|AAAAAAAAIBLFFCAA|1990-09-17|1088|4733|364|1990|1|9|17|3|1990|364|4733|Monday|1990Q3|N|N|N|2448136|2448378|2447787|2448061|N|N|N|N|N| +2448153|AAAAAAAAJBLFFCAA|1990-09-18|1088|4734|364|1990|2|9|18|3|1990|364|4734|Tuesday|1990Q3|N|N|N|2448136|2448378|2447788|2448062|N|N|N|N|N| +2448154|AAAAAAAAKBLFFCAA|1990-09-19|1088|4734|364|1990|3|9|19|3|1990|364|4734|Wednesday|1990Q3|N|N|N|2448136|2448378|2447789|2448063|N|N|N|N|N| +2448155|AAAAAAAALBLFFCAA|1990-09-20|1088|4734|364|1990|4|9|20|3|1990|364|4734|Thursday|1990Q3|N|N|N|2448136|2448378|2447790|2448064|N|N|N|N|N| +2448156|AAAAAAAAMBLFFCAA|1990-09-21|1088|4734|364|1990|5|9|21|3|1990|364|4734|Friday|1990Q3|N|Y|N|2448136|2448378|2447791|2448065|N|N|N|N|N| +2448157|AAAAAAAANBLFFCAA|1990-09-22|1088|4734|364|1990|6|9|22|3|1990|364|4734|Saturday|1990Q3|N|Y|N|2448136|2448378|2447792|2448066|N|N|N|N|N| +2448158|AAAAAAAAOBLFFCAA|1990-09-23|1088|4734|364|1990|0|9|23|3|1990|364|4734|Sunday|1990Q3|N|N|N|2448136|2448378|2447793|2448067|N|N|N|N|N| +2448159|AAAAAAAAPBLFFCAA|1990-09-24|1088|4734|364|1990|1|9|24|3|1990|364|4734|Monday|1990Q3|N|N|N|2448136|2448378|2447794|2448068|N|N|N|N|N| +2448160|AAAAAAAAACLFFCAA|1990-09-25|1088|4735|364|1990|2|9|25|3|1990|364|4735|Tuesday|1990Q3|N|N|N|2448136|2448378|2447795|2448069|N|N|N|N|N| +2448161|AAAAAAAABCLFFCAA|1990-09-26|1088|4735|364|1990|3|9|26|3|1990|364|4735|Wednesday|1990Q3|N|N|N|2448136|2448378|2447796|2448070|N|N|N|N|N| +2448162|AAAAAAAACCLFFCAA|1990-09-27|1088|4735|364|1990|4|9|27|3|1990|364|4735|Thursday|1990Q3|N|N|N|2448136|2448378|2447797|2448071|N|N|N|N|N| +2448163|AAAAAAAADCLFFCAA|1990-09-28|1088|4735|364|1990|5|9|28|3|1990|364|4735|Friday|1990Q3|N|Y|N|2448136|2448378|2447798|2448072|N|N|N|N|N| +2448164|AAAAAAAAECLFFCAA|1990-09-29|1088|4735|364|1990|6|9|29|3|1990|364|4735|Saturday|1990Q3|N|Y|N|2448136|2448378|2447799|2448073|N|N|N|N|N| +2448165|AAAAAAAAFCLFFCAA|1990-09-30|1088|4735|364|1990|0|9|30|3|1990|364|4735|Sunday|1990Q3|N|N|N|2448136|2448378|2447800|2448074|N|N|N|N|N| +2448166|AAAAAAAAGCLFFCAA|1990-10-01|1089|4735|364|1990|1|10|1|3|1990|364|4735|Monday|1990Q3|N|N|N|2448166|2448438|2447801|2448074|N|N|N|N|N| +2448167|AAAAAAAAHCLFFCAA|1990-10-02|1089|4736|364|1990|2|10|2|4|1990|364|4736|Tuesday|1990Q4|N|N|N|2448166|2448438|2447802|2448075|N|N|N|N|N| +2448168|AAAAAAAAICLFFCAA|1990-10-03|1089|4736|364|1990|3|10|3|4|1990|364|4736|Wednesday|1990Q4|N|N|N|2448166|2448438|2447803|2448076|N|N|N|N|N| +2448169|AAAAAAAAJCLFFCAA|1990-10-04|1089|4736|364|1990|4|10|4|4|1990|364|4736|Thursday|1990Q4|N|N|N|2448166|2448438|2447804|2448077|N|N|N|N|N| +2448170|AAAAAAAAKCLFFCAA|1990-10-05|1089|4736|364|1990|5|10|5|4|1990|364|4736|Friday|1990Q4|N|Y|N|2448166|2448438|2447805|2448078|N|N|N|N|N| +2448171|AAAAAAAALCLFFCAA|1990-10-06|1089|4736|364|1990|6|10|6|4|1990|364|4736|Saturday|1990Q4|N|Y|N|2448166|2448438|2447806|2448079|N|N|N|N|N| +2448172|AAAAAAAAMCLFFCAA|1990-10-07|1089|4736|364|1990|0|10|7|4|1990|364|4736|Sunday|1990Q4|N|N|N|2448166|2448438|2447807|2448080|N|N|N|N|N| +2448173|AAAAAAAANCLFFCAA|1990-10-08|1089|4736|364|1990|1|10|8|4|1990|364|4736|Monday|1990Q4|N|N|N|2448166|2448438|2447808|2448081|N|N|N|N|N| +2448174|AAAAAAAAOCLFFCAA|1990-10-09|1089|4737|364|1990|2|10|9|4|1990|364|4737|Tuesday|1990Q4|N|N|N|2448166|2448438|2447809|2448082|N|N|N|N|N| +2448175|AAAAAAAAPCLFFCAA|1990-10-10|1089|4737|364|1990|3|10|10|4|1990|364|4737|Wednesday|1990Q4|N|N|N|2448166|2448438|2447810|2448083|N|N|N|N|N| +2448176|AAAAAAAAADLFFCAA|1990-10-11|1089|4737|364|1990|4|10|11|4|1990|364|4737|Thursday|1990Q4|N|N|N|2448166|2448438|2447811|2448084|N|N|N|N|N| +2448177|AAAAAAAABDLFFCAA|1990-10-12|1089|4737|364|1990|5|10|12|4|1990|364|4737|Friday|1990Q4|N|Y|N|2448166|2448438|2447812|2448085|N|N|N|N|N| +2448178|AAAAAAAACDLFFCAA|1990-10-13|1089|4737|364|1990|6|10|13|4|1990|364|4737|Saturday|1990Q4|N|Y|N|2448166|2448438|2447813|2448086|N|N|N|N|N| +2448179|AAAAAAAADDLFFCAA|1990-10-14|1089|4737|364|1990|0|10|14|4|1990|364|4737|Sunday|1990Q4|N|N|N|2448166|2448438|2447814|2448087|N|N|N|N|N| +2448180|AAAAAAAAEDLFFCAA|1990-10-15|1089|4737|364|1990|1|10|15|4|1990|364|4737|Monday|1990Q4|N|N|N|2448166|2448438|2447815|2448088|N|N|N|N|N| +2448181|AAAAAAAAFDLFFCAA|1990-10-16|1089|4738|364|1990|2|10|16|4|1990|364|4738|Tuesday|1990Q4|N|N|N|2448166|2448438|2447816|2448089|N|N|N|N|N| +2448182|AAAAAAAAGDLFFCAA|1990-10-17|1089|4738|364|1990|3|10|17|4|1990|364|4738|Wednesday|1990Q4|N|N|N|2448166|2448438|2447817|2448090|N|N|N|N|N| +2448183|AAAAAAAAHDLFFCAA|1990-10-18|1089|4738|364|1990|4|10|18|4|1990|364|4738|Thursday|1990Q4|N|N|N|2448166|2448438|2447818|2448091|N|N|N|N|N| +2448184|AAAAAAAAIDLFFCAA|1990-10-19|1089|4738|364|1990|5|10|19|4|1990|364|4738|Friday|1990Q4|N|Y|N|2448166|2448438|2447819|2448092|N|N|N|N|N| +2448185|AAAAAAAAJDLFFCAA|1990-10-20|1089|4738|364|1990|6|10|20|4|1990|364|4738|Saturday|1990Q4|N|Y|N|2448166|2448438|2447820|2448093|N|N|N|N|N| +2448186|AAAAAAAAKDLFFCAA|1990-10-21|1089|4738|364|1990|0|10|21|4|1990|364|4738|Sunday|1990Q4|N|N|N|2448166|2448438|2447821|2448094|N|N|N|N|N| +2448187|AAAAAAAALDLFFCAA|1990-10-22|1089|4738|364|1990|1|10|22|4|1990|364|4738|Monday|1990Q4|N|N|N|2448166|2448438|2447822|2448095|N|N|N|N|N| +2448188|AAAAAAAAMDLFFCAA|1990-10-23|1089|4739|364|1990|2|10|23|4|1990|364|4739|Tuesday|1990Q4|N|N|N|2448166|2448438|2447823|2448096|N|N|N|N|N| +2448189|AAAAAAAANDLFFCAA|1990-10-24|1089|4739|364|1990|3|10|24|4|1990|364|4739|Wednesday|1990Q4|N|N|N|2448166|2448438|2447824|2448097|N|N|N|N|N| +2448190|AAAAAAAAODLFFCAA|1990-10-25|1089|4739|364|1990|4|10|25|4|1990|364|4739|Thursday|1990Q4|N|N|N|2448166|2448438|2447825|2448098|N|N|N|N|N| +2448191|AAAAAAAAPDLFFCAA|1990-10-26|1089|4739|364|1990|5|10|26|4|1990|364|4739|Friday|1990Q4|N|Y|N|2448166|2448438|2447826|2448099|N|N|N|N|N| +2448192|AAAAAAAAAELFFCAA|1990-10-27|1089|4739|364|1990|6|10|27|4|1990|364|4739|Saturday|1990Q4|N|Y|N|2448166|2448438|2447827|2448100|N|N|N|N|N| +2448193|AAAAAAAABELFFCAA|1990-10-28|1089|4739|364|1990|0|10|28|4|1990|364|4739|Sunday|1990Q4|N|N|N|2448166|2448438|2447828|2448101|N|N|N|N|N| +2448194|AAAAAAAACELFFCAA|1990-10-29|1089|4739|364|1990|1|10|29|4|1990|364|4739|Monday|1990Q4|N|N|N|2448166|2448438|2447829|2448102|N|N|N|N|N| +2448195|AAAAAAAADELFFCAA|1990-10-30|1089|4740|364|1990|2|10|30|4|1990|364|4740|Tuesday|1990Q4|N|N|N|2448166|2448438|2447830|2448103|N|N|N|N|N| +2448196|AAAAAAAAEELFFCAA|1990-10-31|1089|4740|364|1990|3|10|31|4|1990|364|4740|Wednesday|1990Q4|N|N|N|2448166|2448438|2447831|2448104|N|N|N|N|N| +2448197|AAAAAAAAFELFFCAA|1990-11-01|1090|4740|364|1990|4|11|1|4|1990|364|4740|Thursday|1990Q4|N|N|N|2448197|2448500|2447832|2448105|N|N|N|N|N| +2448198|AAAAAAAAGELFFCAA|1990-11-02|1090|4740|364|1990|5|11|2|4|1990|364|4740|Friday|1990Q4|N|Y|N|2448197|2448500|2447833|2448106|N|N|N|N|N| +2448199|AAAAAAAAHELFFCAA|1990-11-03|1090|4740|364|1990|6|11|3|4|1990|364|4740|Saturday|1990Q4|N|Y|N|2448197|2448500|2447834|2448107|N|N|N|N|N| +2448200|AAAAAAAAIELFFCAA|1990-11-04|1090|4740|364|1990|0|11|4|4|1990|364|4740|Sunday|1990Q4|N|N|N|2448197|2448500|2447835|2448108|N|N|N|N|N| +2448201|AAAAAAAAJELFFCAA|1990-11-05|1090|4740|364|1990|1|11|5|4|1990|364|4740|Monday|1990Q4|N|N|N|2448197|2448500|2447836|2448109|N|N|N|N|N| +2448202|AAAAAAAAKELFFCAA|1990-11-06|1090|4741|364|1990|2|11|6|4|1990|364|4741|Tuesday|1990Q4|N|N|N|2448197|2448500|2447837|2448110|N|N|N|N|N| +2448203|AAAAAAAALELFFCAA|1990-11-07|1090|4741|364|1990|3|11|7|4|1990|364|4741|Wednesday|1990Q4|N|N|N|2448197|2448500|2447838|2448111|N|N|N|N|N| +2448204|AAAAAAAAMELFFCAA|1990-11-08|1090|4741|364|1990|4|11|8|4|1990|364|4741|Thursday|1990Q4|N|N|N|2448197|2448500|2447839|2448112|N|N|N|N|N| +2448205|AAAAAAAANELFFCAA|1990-11-09|1090|4741|364|1990|5|11|9|4|1990|364|4741|Friday|1990Q4|N|Y|N|2448197|2448500|2447840|2448113|N|N|N|N|N| +2448206|AAAAAAAAOELFFCAA|1990-11-10|1090|4741|364|1990|6|11|10|4|1990|364|4741|Saturday|1990Q4|N|Y|N|2448197|2448500|2447841|2448114|N|N|N|N|N| +2448207|AAAAAAAAPELFFCAA|1990-11-11|1090|4741|364|1990|0|11|11|4|1990|364|4741|Sunday|1990Q4|N|N|N|2448197|2448500|2447842|2448115|N|N|N|N|N| +2448208|AAAAAAAAAFLFFCAA|1990-11-12|1090|4741|364|1990|1|11|12|4|1990|364|4741|Monday|1990Q4|N|N|N|2448197|2448500|2447843|2448116|N|N|N|N|N| +2448209|AAAAAAAABFLFFCAA|1990-11-13|1090|4742|364|1990|2|11|13|4|1990|364|4742|Tuesday|1990Q4|N|N|N|2448197|2448500|2447844|2448117|N|N|N|N|N| +2448210|AAAAAAAACFLFFCAA|1990-11-14|1090|4742|364|1990|3|11|14|4|1990|364|4742|Wednesday|1990Q4|N|N|N|2448197|2448500|2447845|2448118|N|N|N|N|N| +2448211|AAAAAAAADFLFFCAA|1990-11-15|1090|4742|364|1990|4|11|15|4|1990|364|4742|Thursday|1990Q4|N|N|N|2448197|2448500|2447846|2448119|N|N|N|N|N| +2448212|AAAAAAAAEFLFFCAA|1990-11-16|1090|4742|364|1990|5|11|16|4|1990|364|4742|Friday|1990Q4|N|Y|N|2448197|2448500|2447847|2448120|N|N|N|N|N| +2448213|AAAAAAAAFFLFFCAA|1990-11-17|1090|4742|364|1990|6|11|17|4|1990|364|4742|Saturday|1990Q4|N|Y|N|2448197|2448500|2447848|2448121|N|N|N|N|N| +2448214|AAAAAAAAGFLFFCAA|1990-11-18|1090|4742|364|1990|0|11|18|4|1990|364|4742|Sunday|1990Q4|N|N|N|2448197|2448500|2447849|2448122|N|N|N|N|N| +2448215|AAAAAAAAHFLFFCAA|1990-11-19|1090|4742|364|1990|1|11|19|4|1990|364|4742|Monday|1990Q4|N|N|N|2448197|2448500|2447850|2448123|N|N|N|N|N| +2448216|AAAAAAAAIFLFFCAA|1990-11-20|1090|4743|364|1990|2|11|20|4|1990|364|4743|Tuesday|1990Q4|N|N|N|2448197|2448500|2447851|2448124|N|N|N|N|N| +2448217|AAAAAAAAJFLFFCAA|1990-11-21|1090|4743|364|1990|3|11|21|4|1990|364|4743|Wednesday|1990Q4|N|N|N|2448197|2448500|2447852|2448125|N|N|N|N|N| +2448218|AAAAAAAAKFLFFCAA|1990-11-22|1090|4743|364|1990|4|11|22|4|1990|364|4743|Thursday|1990Q4|N|N|N|2448197|2448500|2447853|2448126|N|N|N|N|N| +2448219|AAAAAAAALFLFFCAA|1990-11-23|1090|4743|364|1990|5|11|23|4|1990|364|4743|Friday|1990Q4|N|Y|N|2448197|2448500|2447854|2448127|N|N|N|N|N| +2448220|AAAAAAAAMFLFFCAA|1990-11-24|1090|4743|364|1990|6|11|24|4|1990|364|4743|Saturday|1990Q4|N|Y|N|2448197|2448500|2447855|2448128|N|N|N|N|N| +2448221|AAAAAAAANFLFFCAA|1990-11-25|1090|4743|364|1990|0|11|25|4|1990|364|4743|Sunday|1990Q4|N|N|N|2448197|2448500|2447856|2448129|N|N|N|N|N| +2448222|AAAAAAAAOFLFFCAA|1990-11-26|1090|4743|364|1990|1|11|26|4|1990|364|4743|Monday|1990Q4|N|N|N|2448197|2448500|2447857|2448130|N|N|N|N|N| +2448223|AAAAAAAAPFLFFCAA|1990-11-27|1090|4744|364|1990|2|11|27|4|1990|364|4744|Tuesday|1990Q4|N|N|N|2448197|2448500|2447858|2448131|N|N|N|N|N| +2448224|AAAAAAAAAGLFFCAA|1990-11-28|1090|4744|364|1990|3|11|28|4|1990|364|4744|Wednesday|1990Q4|N|N|N|2448197|2448500|2447859|2448132|N|N|N|N|N| +2448225|AAAAAAAABGLFFCAA|1990-11-29|1090|4744|364|1990|4|11|29|4|1990|364|4744|Thursday|1990Q4|N|N|N|2448197|2448500|2447860|2448133|N|N|N|N|N| +2448226|AAAAAAAACGLFFCAA|1990-11-30|1090|4744|364|1990|5|11|30|4|1990|364|4744|Friday|1990Q4|N|Y|N|2448197|2448500|2447861|2448134|N|N|N|N|N| +2448227|AAAAAAAADGLFFCAA|1990-12-01|1091|4744|365|1990|6|12|1|4|1990|365|4744|Saturday|1990Q4|N|Y|N|2448227|2448560|2447862|2448135|N|N|N|N|N| +2448228|AAAAAAAAEGLFFCAA|1990-12-02|1091|4744|365|1990|0|12|2|4|1990|365|4744|Sunday|1990Q4|N|N|N|2448227|2448560|2447863|2448136|N|N|N|N|N| +2448229|AAAAAAAAFGLFFCAA|1990-12-03|1091|4744|365|1990|1|12|3|4|1990|365|4744|Monday|1990Q4|N|N|N|2448227|2448560|2447864|2448137|N|N|N|N|N| +2448230|AAAAAAAAGGLFFCAA|1990-12-04|1091|4745|365|1990|2|12|4|4|1990|365|4745|Tuesday|1990Q4|N|N|N|2448227|2448560|2447865|2448138|N|N|N|N|N| +2448231|AAAAAAAAHGLFFCAA|1990-12-05|1091|4745|365|1990|3|12|5|4|1990|365|4745|Wednesday|1990Q4|N|N|N|2448227|2448560|2447866|2448139|N|N|N|N|N| +2448232|AAAAAAAAIGLFFCAA|1990-12-06|1091|4745|365|1990|4|12|6|4|1990|365|4745|Thursday|1990Q4|N|N|N|2448227|2448560|2447867|2448140|N|N|N|N|N| +2448233|AAAAAAAAJGLFFCAA|1990-12-07|1091|4745|365|1990|5|12|7|4|1990|365|4745|Friday|1990Q4|N|Y|N|2448227|2448560|2447868|2448141|N|N|N|N|N| +2448234|AAAAAAAAKGLFFCAA|1990-12-08|1091|4745|365|1990|6|12|8|4|1990|365|4745|Saturday|1990Q4|N|Y|N|2448227|2448560|2447869|2448142|N|N|N|N|N| +2448235|AAAAAAAALGLFFCAA|1990-12-09|1091|4745|365|1990|0|12|9|4|1990|365|4745|Sunday|1990Q4|N|N|N|2448227|2448560|2447870|2448143|N|N|N|N|N| +2448236|AAAAAAAAMGLFFCAA|1990-12-10|1091|4745|365|1990|1|12|10|4|1990|365|4745|Monday|1990Q4|N|N|N|2448227|2448560|2447871|2448144|N|N|N|N|N| +2448237|AAAAAAAANGLFFCAA|1990-12-11|1091|4746|365|1990|2|12|11|4|1990|365|4746|Tuesday|1990Q4|N|N|N|2448227|2448560|2447872|2448145|N|N|N|N|N| +2448238|AAAAAAAAOGLFFCAA|1990-12-12|1091|4746|365|1990|3|12|12|4|1990|365|4746|Wednesday|1990Q4|N|N|N|2448227|2448560|2447873|2448146|N|N|N|N|N| +2448239|AAAAAAAAPGLFFCAA|1990-12-13|1091|4746|365|1990|4|12|13|4|1990|365|4746|Thursday|1990Q4|N|N|N|2448227|2448560|2447874|2448147|N|N|N|N|N| +2448240|AAAAAAAAAHLFFCAA|1990-12-14|1091|4746|365|1990|5|12|14|4|1990|365|4746|Friday|1990Q4|N|Y|N|2448227|2448560|2447875|2448148|N|N|N|N|N| +2448241|AAAAAAAABHLFFCAA|1990-12-15|1091|4746|365|1990|6|12|15|4|1990|365|4746|Saturday|1990Q4|N|Y|N|2448227|2448560|2447876|2448149|N|N|N|N|N| +2448242|AAAAAAAACHLFFCAA|1990-12-16|1091|4746|365|1990|0|12|16|4|1990|365|4746|Sunday|1990Q4|N|N|N|2448227|2448560|2447877|2448150|N|N|N|N|N| +2448243|AAAAAAAADHLFFCAA|1990-12-17|1091|4746|365|1990|1|12|17|4|1990|365|4746|Monday|1990Q4|N|N|N|2448227|2448560|2447878|2448151|N|N|N|N|N| +2448244|AAAAAAAAEHLFFCAA|1990-12-18|1091|4747|365|1990|2|12|18|4|1990|365|4747|Tuesday|1990Q4|N|N|N|2448227|2448560|2447879|2448152|N|N|N|N|N| +2448245|AAAAAAAAFHLFFCAA|1990-12-19|1091|4747|365|1990|3|12|19|4|1990|365|4747|Wednesday|1990Q4|N|N|N|2448227|2448560|2447880|2448153|N|N|N|N|N| +2448246|AAAAAAAAGHLFFCAA|1990-12-20|1091|4747|365|1990|4|12|20|4|1990|365|4747|Thursday|1990Q4|N|N|N|2448227|2448560|2447881|2448154|N|N|N|N|N| +2448247|AAAAAAAAHHLFFCAA|1990-12-21|1091|4747|365|1990|5|12|21|4|1990|365|4747|Friday|1990Q4|N|Y|N|2448227|2448560|2447882|2448155|N|N|N|N|N| +2448248|AAAAAAAAIHLFFCAA|1990-12-22|1091|4747|365|1990|6|12|22|4|1990|365|4747|Saturday|1990Q4|N|Y|N|2448227|2448560|2447883|2448156|N|N|N|N|N| +2448249|AAAAAAAAJHLFFCAA|1990-12-23|1091|4747|365|1990|0|12|23|4|1990|365|4747|Sunday|1990Q4|N|N|N|2448227|2448560|2447884|2448157|N|N|N|N|N| +2448250|AAAAAAAAKHLFFCAA|1990-12-24|1091|4747|365|1990|1|12|24|4|1990|365|4747|Monday|1990Q4|N|N|N|2448227|2448560|2447885|2448158|N|N|N|N|N| +2448251|AAAAAAAALHLFFCAA|1990-12-25|1091|4748|365|1990|2|12|25|4|1990|365|4748|Tuesday|1990Q4|N|N|N|2448227|2448560|2447886|2448159|N|N|N|N|N| +2448252|AAAAAAAAMHLFFCAA|1990-12-26|1091|4748|365|1990|3|12|26|4|1990|365|4748|Wednesday|1990Q4|Y|N|N|2448227|2448560|2447887|2448160|N|N|N|N|N| +2448253|AAAAAAAANHLFFCAA|1990-12-27|1091|4748|365|1990|4|12|27|4|1990|365|4748|Thursday|1990Q4|N|N|Y|2448227|2448560|2447888|2448161|N|N|N|N|N| +2448254|AAAAAAAAOHLFFCAA|1990-12-28|1091|4748|365|1990|5|12|28|4|1990|365|4748|Friday|1990Q4|N|Y|N|2448227|2448560|2447889|2448162|N|N|N|N|N| +2448255|AAAAAAAAPHLFFCAA|1990-12-29|1091|4748|365|1990|6|12|29|4|1990|365|4748|Saturday|1990Q4|N|Y|N|2448227|2448560|2447890|2448163|N|N|N|N|N| +2448256|AAAAAAAAAILFFCAA|1990-12-30|1091|4748|365|1990|0|12|30|4|1990|365|4748|Sunday|1990Q4|N|N|N|2448227|2448560|2447891|2448164|N|N|N|N|N| +2448257|AAAAAAAABILFFCAA|1990-12-31|1091|4748|365|1990|1|12|31|4|1990|365|4748|Monday|1990Q4|N|N|N|2448227|2448560|2447892|2448165|N|N|N|N|N| +2448258|AAAAAAAACILFFCAA|1991-01-01|1092|4749|365|1991|2|1|1|1|1991|365|4749|Tuesday|1991Q1|Y|N|N|2448258|2448257|2447893|2448166|N|N|N|N|N| +2448259|AAAAAAAADILFFCAA|1991-01-02|1092|4749|365|1991|3|1|2|1|1991|365|4749|Wednesday|1991Q1|N|N|Y|2448258|2448257|2447894|2448167|N|N|N|N|N| +2448260|AAAAAAAAEILFFCAA|1991-01-03|1092|4749|365|1991|4|1|3|1|1991|365|4749|Thursday|1991Q1|N|N|N|2448258|2448257|2447895|2448168|N|N|N|N|N| +2448261|AAAAAAAAFILFFCAA|1991-01-04|1092|4749|365|1991|5|1|4|1|1991|365|4749|Friday|1991Q1|N|Y|N|2448258|2448257|2447896|2448169|N|N|N|N|N| +2448262|AAAAAAAAGILFFCAA|1991-01-05|1092|4749|365|1991|6|1|5|1|1991|365|4749|Saturday|1991Q1|N|Y|N|2448258|2448257|2447897|2448170|N|N|N|N|N| +2448263|AAAAAAAAHILFFCAA|1991-01-06|1092|4749|365|1991|0|1|6|1|1991|365|4749|Sunday|1991Q1|N|N|N|2448258|2448257|2447898|2448171|N|N|N|N|N| +2448264|AAAAAAAAIILFFCAA|1991-01-07|1092|4749|365|1991|1|1|7|1|1991|365|4749|Monday|1991Q1|N|N|N|2448258|2448257|2447899|2448172|N|N|N|N|N| +2448265|AAAAAAAAJILFFCAA|1991-01-08|1092|4750|365|1991|2|1|8|1|1991|365|4750|Tuesday|1991Q1|N|N|N|2448258|2448257|2447900|2448173|N|N|N|N|N| +2448266|AAAAAAAAKILFFCAA|1991-01-09|1092|4750|365|1991|3|1|9|1|1991|365|4750|Wednesday|1991Q1|N|N|N|2448258|2448257|2447901|2448174|N|N|N|N|N| +2448267|AAAAAAAALILFFCAA|1991-01-10|1092|4750|365|1991|4|1|10|1|1991|365|4750|Thursday|1991Q1|N|N|N|2448258|2448257|2447902|2448175|N|N|N|N|N| +2448268|AAAAAAAAMILFFCAA|1991-01-11|1092|4750|365|1991|5|1|11|1|1991|365|4750|Friday|1991Q1|N|Y|N|2448258|2448257|2447903|2448176|N|N|N|N|N| +2448269|AAAAAAAANILFFCAA|1991-01-12|1092|4750|365|1991|6|1|12|1|1991|365|4750|Saturday|1991Q1|N|Y|N|2448258|2448257|2447904|2448177|N|N|N|N|N| +2448270|AAAAAAAAOILFFCAA|1991-01-13|1092|4750|365|1991|0|1|13|1|1991|365|4750|Sunday|1991Q1|N|N|N|2448258|2448257|2447905|2448178|N|N|N|N|N| +2448271|AAAAAAAAPILFFCAA|1991-01-14|1092|4750|365|1991|1|1|14|1|1991|365|4750|Monday|1991Q1|N|N|N|2448258|2448257|2447906|2448179|N|N|N|N|N| +2448272|AAAAAAAAAJLFFCAA|1991-01-15|1092|4751|365|1991|2|1|15|1|1991|365|4751|Tuesday|1991Q1|N|N|N|2448258|2448257|2447907|2448180|N|N|N|N|N| +2448273|AAAAAAAABJLFFCAA|1991-01-16|1092|4751|365|1991|3|1|16|1|1991|365|4751|Wednesday|1991Q1|N|N|N|2448258|2448257|2447908|2448181|N|N|N|N|N| +2448274|AAAAAAAACJLFFCAA|1991-01-17|1092|4751|365|1991|4|1|17|1|1991|365|4751|Thursday|1991Q1|N|N|N|2448258|2448257|2447909|2448182|N|N|N|N|N| +2448275|AAAAAAAADJLFFCAA|1991-01-18|1092|4751|365|1991|5|1|18|1|1991|365|4751|Friday|1991Q1|N|Y|N|2448258|2448257|2447910|2448183|N|N|N|N|N| +2448276|AAAAAAAAEJLFFCAA|1991-01-19|1092|4751|365|1991|6|1|19|1|1991|365|4751|Saturday|1991Q1|N|Y|N|2448258|2448257|2447911|2448184|N|N|N|N|N| +2448277|AAAAAAAAFJLFFCAA|1991-01-20|1092|4751|365|1991|0|1|20|1|1991|365|4751|Sunday|1991Q1|N|N|N|2448258|2448257|2447912|2448185|N|N|N|N|N| +2448278|AAAAAAAAGJLFFCAA|1991-01-21|1092|4751|365|1991|1|1|21|1|1991|365|4751|Monday|1991Q1|N|N|N|2448258|2448257|2447913|2448186|N|N|N|N|N| +2448279|AAAAAAAAHJLFFCAA|1991-01-22|1092|4752|365|1991|2|1|22|1|1991|365|4752|Tuesday|1991Q1|N|N|N|2448258|2448257|2447914|2448187|N|N|N|N|N| +2448280|AAAAAAAAIJLFFCAA|1991-01-23|1092|4752|365|1991|3|1|23|1|1991|365|4752|Wednesday|1991Q1|N|N|N|2448258|2448257|2447915|2448188|N|N|N|N|N| +2448281|AAAAAAAAJJLFFCAA|1991-01-24|1092|4752|365|1991|4|1|24|1|1991|365|4752|Thursday|1991Q1|N|N|N|2448258|2448257|2447916|2448189|N|N|N|N|N| +2448282|AAAAAAAAKJLFFCAA|1991-01-25|1092|4752|365|1991|5|1|25|1|1991|365|4752|Friday|1991Q1|N|Y|N|2448258|2448257|2447917|2448190|N|N|N|N|N| +2448283|AAAAAAAALJLFFCAA|1991-01-26|1092|4752|365|1991|6|1|26|1|1991|365|4752|Saturday|1991Q1|N|Y|N|2448258|2448257|2447918|2448191|N|N|N|N|N| +2448284|AAAAAAAAMJLFFCAA|1991-01-27|1092|4752|365|1991|0|1|27|1|1991|365|4752|Sunday|1991Q1|N|N|N|2448258|2448257|2447919|2448192|N|N|N|N|N| +2448285|AAAAAAAANJLFFCAA|1991-01-28|1092|4752|365|1991|1|1|28|1|1991|365|4752|Monday|1991Q1|N|N|N|2448258|2448257|2447920|2448193|N|N|N|N|N| +2448286|AAAAAAAAOJLFFCAA|1991-01-29|1092|4753|365|1991|2|1|29|1|1991|365|4753|Tuesday|1991Q1|N|N|N|2448258|2448257|2447921|2448194|N|N|N|N|N| +2448287|AAAAAAAAPJLFFCAA|1991-01-30|1092|4753|365|1991|3|1|30|1|1991|365|4753|Wednesday|1991Q1|N|N|N|2448258|2448257|2447922|2448195|N|N|N|N|N| +2448288|AAAAAAAAAKLFFCAA|1991-01-31|1092|4753|365|1991|4|1|31|1|1991|365|4753|Thursday|1991Q1|N|N|N|2448258|2448257|2447923|2448196|N|N|N|N|N| +2448289|AAAAAAAABKLFFCAA|1991-02-01|1093|4753|365|1991|5|2|1|1|1991|365|4753|Friday|1991Q1|N|Y|N|2448289|2448319|2447924|2448197|N|N|N|N|N| +2448290|AAAAAAAACKLFFCAA|1991-02-02|1093|4753|365|1991|6|2|2|1|1991|365|4753|Saturday|1991Q1|N|Y|N|2448289|2448319|2447925|2448198|N|N|N|N|N| +2448291|AAAAAAAADKLFFCAA|1991-02-03|1093|4753|365|1991|0|2|3|1|1991|365|4753|Sunday|1991Q1|N|N|N|2448289|2448319|2447926|2448199|N|N|N|N|N| +2448292|AAAAAAAAEKLFFCAA|1991-02-04|1093|4753|365|1991|1|2|4|1|1991|365|4753|Monday|1991Q1|N|N|N|2448289|2448319|2447927|2448200|N|N|N|N|N| +2448293|AAAAAAAAFKLFFCAA|1991-02-05|1093|4754|365|1991|2|2|5|1|1991|365|4754|Tuesday|1991Q1|N|N|N|2448289|2448319|2447928|2448201|N|N|N|N|N| +2448294|AAAAAAAAGKLFFCAA|1991-02-06|1093|4754|365|1991|3|2|6|1|1991|365|4754|Wednesday|1991Q1|N|N|N|2448289|2448319|2447929|2448202|N|N|N|N|N| +2448295|AAAAAAAAHKLFFCAA|1991-02-07|1093|4754|365|1991|4|2|7|1|1991|365|4754|Thursday|1991Q1|N|N|N|2448289|2448319|2447930|2448203|N|N|N|N|N| +2448296|AAAAAAAAIKLFFCAA|1991-02-08|1093|4754|365|1991|5|2|8|1|1991|365|4754|Friday|1991Q1|N|Y|N|2448289|2448319|2447931|2448204|N|N|N|N|N| +2448297|AAAAAAAAJKLFFCAA|1991-02-09|1093|4754|365|1991|6|2|9|1|1991|365|4754|Saturday|1991Q1|N|Y|N|2448289|2448319|2447932|2448205|N|N|N|N|N| +2448298|AAAAAAAAKKLFFCAA|1991-02-10|1093|4754|365|1991|0|2|10|1|1991|365|4754|Sunday|1991Q1|N|N|N|2448289|2448319|2447933|2448206|N|N|N|N|N| +2448299|AAAAAAAALKLFFCAA|1991-02-11|1093|4754|365|1991|1|2|11|1|1991|365|4754|Monday|1991Q1|N|N|N|2448289|2448319|2447934|2448207|N|N|N|N|N| +2448300|AAAAAAAAMKLFFCAA|1991-02-12|1093|4755|365|1991|2|2|12|1|1991|365|4755|Tuesday|1991Q1|N|N|N|2448289|2448319|2447935|2448208|N|N|N|N|N| +2448301|AAAAAAAANKLFFCAA|1991-02-13|1093|4755|365|1991|3|2|13|1|1991|365|4755|Wednesday|1991Q1|N|N|N|2448289|2448319|2447936|2448209|N|N|N|N|N| +2448302|AAAAAAAAOKLFFCAA|1991-02-14|1093|4755|365|1991|4|2|14|1|1991|365|4755|Thursday|1991Q1|N|N|N|2448289|2448319|2447937|2448210|N|N|N|N|N| +2448303|AAAAAAAAPKLFFCAA|1991-02-15|1093|4755|365|1991|5|2|15|1|1991|365|4755|Friday|1991Q1|N|Y|N|2448289|2448319|2447938|2448211|N|N|N|N|N| +2448304|AAAAAAAAALLFFCAA|1991-02-16|1093|4755|365|1991|6|2|16|1|1991|365|4755|Saturday|1991Q1|N|Y|N|2448289|2448319|2447939|2448212|N|N|N|N|N| +2448305|AAAAAAAABLLFFCAA|1991-02-17|1093|4755|365|1991|0|2|17|1|1991|365|4755|Sunday|1991Q1|N|N|N|2448289|2448319|2447940|2448213|N|N|N|N|N| +2448306|AAAAAAAACLLFFCAA|1991-02-18|1093|4755|365|1991|1|2|18|1|1991|365|4755|Monday|1991Q1|N|N|N|2448289|2448319|2447941|2448214|N|N|N|N|N| +2448307|AAAAAAAADLLFFCAA|1991-02-19|1093|4756|365|1991|2|2|19|1|1991|365|4756|Tuesday|1991Q1|N|N|N|2448289|2448319|2447942|2448215|N|N|N|N|N| +2448308|AAAAAAAAELLFFCAA|1991-02-20|1093|4756|365|1991|3|2|20|1|1991|365|4756|Wednesday|1991Q1|N|N|N|2448289|2448319|2447943|2448216|N|N|N|N|N| +2448309|AAAAAAAAFLLFFCAA|1991-02-21|1093|4756|365|1991|4|2|21|1|1991|365|4756|Thursday|1991Q1|N|N|N|2448289|2448319|2447944|2448217|N|N|N|N|N| +2448310|AAAAAAAAGLLFFCAA|1991-02-22|1093|4756|365|1991|5|2|22|1|1991|365|4756|Friday|1991Q1|N|Y|N|2448289|2448319|2447945|2448218|N|N|N|N|N| +2448311|AAAAAAAAHLLFFCAA|1991-02-23|1093|4756|365|1991|6|2|23|1|1991|365|4756|Saturday|1991Q1|N|Y|N|2448289|2448319|2447946|2448219|N|N|N|N|N| +2448312|AAAAAAAAILLFFCAA|1991-02-24|1093|4756|365|1991|0|2|24|1|1991|365|4756|Sunday|1991Q1|N|N|N|2448289|2448319|2447947|2448220|N|N|N|N|N| +2448313|AAAAAAAAJLLFFCAA|1991-02-25|1093|4756|365|1991|1|2|25|1|1991|365|4756|Monday|1991Q1|N|N|N|2448289|2448319|2447948|2448221|N|N|N|N|N| +2448314|AAAAAAAAKLLFFCAA|1991-02-26|1093|4757|365|1991|2|2|26|1|1991|365|4757|Tuesday|1991Q1|N|N|N|2448289|2448319|2447949|2448222|N|N|N|N|N| +2448315|AAAAAAAALLLFFCAA|1991-02-27|1093|4757|365|1991|3|2|27|1|1991|365|4757|Wednesday|1991Q1|N|N|N|2448289|2448319|2447950|2448223|N|N|N|N|N| +2448316|AAAAAAAAMLLFFCAA|1991-02-28|1093|4757|365|1991|4|2|28|1|1991|365|4757|Thursday|1991Q1|N|N|N|2448289|2448319|2447951|2448224|N|N|N|N|N| +2448317|AAAAAAAANLLFFCAA|1991-03-01|1094|4757|366|1991|5|3|1|1|1991|366|4757|Friday|1991Q1|N|Y|N|2448317|2448375|2447952|2448225|N|N|N|N|N| +2448318|AAAAAAAAOLLFFCAA|1991-03-02|1094|4757|366|1991|6|3|2|1|1991|366|4757|Saturday|1991Q1|N|Y|N|2448317|2448375|2447953|2448226|N|N|N|N|N| +2448319|AAAAAAAAPLLFFCAA|1991-03-03|1094|4757|366|1991|0|3|3|1|1991|366|4757|Sunday|1991Q1|N|N|N|2448317|2448375|2447954|2448227|N|N|N|N|N| +2448320|AAAAAAAAAMLFFCAA|1991-03-04|1094|4757|366|1991|1|3|4|1|1991|366|4757|Monday|1991Q1|N|N|N|2448317|2448375|2447955|2448228|N|N|N|N|N| +2448321|AAAAAAAABMLFFCAA|1991-03-05|1094|4758|366|1991|2|3|5|1|1991|366|4758|Tuesday|1991Q1|N|N|N|2448317|2448375|2447956|2448229|N|N|N|N|N| +2448322|AAAAAAAACMLFFCAA|1991-03-06|1094|4758|366|1991|3|3|6|1|1991|366|4758|Wednesday|1991Q1|N|N|N|2448317|2448375|2447957|2448230|N|N|N|N|N| +2448323|AAAAAAAADMLFFCAA|1991-03-07|1094|4758|366|1991|4|3|7|1|1991|366|4758|Thursday|1991Q1|N|N|N|2448317|2448375|2447958|2448231|N|N|N|N|N| +2448324|AAAAAAAAEMLFFCAA|1991-03-08|1094|4758|366|1991|5|3|8|1|1991|366|4758|Friday|1991Q1|N|Y|N|2448317|2448375|2447959|2448232|N|N|N|N|N| +2448325|AAAAAAAAFMLFFCAA|1991-03-09|1094|4758|366|1991|6|3|9|1|1991|366|4758|Saturday|1991Q1|N|Y|N|2448317|2448375|2447960|2448233|N|N|N|N|N| +2448326|AAAAAAAAGMLFFCAA|1991-03-10|1094|4758|366|1991|0|3|10|1|1991|366|4758|Sunday|1991Q1|N|N|N|2448317|2448375|2447961|2448234|N|N|N|N|N| +2448327|AAAAAAAAHMLFFCAA|1991-03-11|1094|4758|366|1991|1|3|11|1|1991|366|4758|Monday|1991Q1|N|N|N|2448317|2448375|2447962|2448235|N|N|N|N|N| +2448328|AAAAAAAAIMLFFCAA|1991-03-12|1094|4759|366|1991|2|3|12|1|1991|366|4759|Tuesday|1991Q1|N|N|N|2448317|2448375|2447963|2448236|N|N|N|N|N| +2448329|AAAAAAAAJMLFFCAA|1991-03-13|1094|4759|366|1991|3|3|13|1|1991|366|4759|Wednesday|1991Q1|N|N|N|2448317|2448375|2447964|2448237|N|N|N|N|N| +2448330|AAAAAAAAKMLFFCAA|1991-03-14|1094|4759|366|1991|4|3|14|1|1991|366|4759|Thursday|1991Q1|N|N|N|2448317|2448375|2447965|2448238|N|N|N|N|N| +2448331|AAAAAAAALMLFFCAA|1991-03-15|1094|4759|366|1991|5|3|15|1|1991|366|4759|Friday|1991Q1|N|Y|N|2448317|2448375|2447966|2448239|N|N|N|N|N| +2448332|AAAAAAAAMMLFFCAA|1991-03-16|1094|4759|366|1991|6|3|16|1|1991|366|4759|Saturday|1991Q1|N|Y|N|2448317|2448375|2447967|2448240|N|N|N|N|N| +2448333|AAAAAAAANMLFFCAA|1991-03-17|1094|4759|366|1991|0|3|17|1|1991|366|4759|Sunday|1991Q1|N|N|N|2448317|2448375|2447968|2448241|N|N|N|N|N| +2448334|AAAAAAAAOMLFFCAA|1991-03-18|1094|4759|366|1991|1|3|18|1|1991|366|4759|Monday|1991Q1|N|N|N|2448317|2448375|2447969|2448242|N|N|N|N|N| +2448335|AAAAAAAAPMLFFCAA|1991-03-19|1094|4760|366|1991|2|3|19|1|1991|366|4760|Tuesday|1991Q1|N|N|N|2448317|2448375|2447970|2448243|N|N|N|N|N| +2448336|AAAAAAAAANLFFCAA|1991-03-20|1094|4760|366|1991|3|3|20|1|1991|366|4760|Wednesday|1991Q1|N|N|N|2448317|2448375|2447971|2448244|N|N|N|N|N| +2448337|AAAAAAAABNLFFCAA|1991-03-21|1094|4760|366|1991|4|3|21|1|1991|366|4760|Thursday|1991Q1|N|N|N|2448317|2448375|2447972|2448245|N|N|N|N|N| +2448338|AAAAAAAACNLFFCAA|1991-03-22|1094|4760|366|1991|5|3|22|1|1991|366|4760|Friday|1991Q1|N|Y|N|2448317|2448375|2447973|2448246|N|N|N|N|N| +2448339|AAAAAAAADNLFFCAA|1991-03-23|1094|4760|366|1991|6|3|23|1|1991|366|4760|Saturday|1991Q1|N|Y|N|2448317|2448375|2447974|2448247|N|N|N|N|N| +2448340|AAAAAAAAENLFFCAA|1991-03-24|1094|4760|366|1991|0|3|24|1|1991|366|4760|Sunday|1991Q1|N|N|N|2448317|2448375|2447975|2448248|N|N|N|N|N| +2448341|AAAAAAAAFNLFFCAA|1991-03-25|1094|4760|366|1991|1|3|25|1|1991|366|4760|Monday|1991Q1|N|N|N|2448317|2448375|2447976|2448249|N|N|N|N|N| +2448342|AAAAAAAAGNLFFCAA|1991-03-26|1094|4761|366|1991|2|3|26|1|1991|366|4761|Tuesday|1991Q1|N|N|N|2448317|2448375|2447977|2448250|N|N|N|N|N| +2448343|AAAAAAAAHNLFFCAA|1991-03-27|1094|4761|366|1991|3|3|27|1|1991|366|4761|Wednesday|1991Q1|N|N|N|2448317|2448375|2447978|2448251|N|N|N|N|N| +2448344|AAAAAAAAINLFFCAA|1991-03-28|1094|4761|366|1991|4|3|28|1|1991|366|4761|Thursday|1991Q1|N|N|N|2448317|2448375|2447979|2448252|N|N|N|N|N| +2448345|AAAAAAAAJNLFFCAA|1991-03-29|1094|4761|366|1991|5|3|29|1|1991|366|4761|Friday|1991Q1|N|Y|N|2448317|2448375|2447980|2448253|N|N|N|N|N| +2448346|AAAAAAAAKNLFFCAA|1991-03-30|1094|4761|366|1991|6|3|30|1|1991|366|4761|Saturday|1991Q1|N|Y|N|2448317|2448375|2447981|2448254|N|N|N|N|N| +2448347|AAAAAAAALNLFFCAA|1991-03-31|1094|4761|366|1991|0|3|31|1|1991|366|4761|Sunday|1991Q1|N|N|N|2448317|2448375|2447982|2448255|N|N|N|N|N| +2448348|AAAAAAAAMNLFFCAA|1991-04-01|1095|4761|366|1991|1|4|1|1|1991|366|4761|Monday|1991Q1|N|N|N|2448348|2448437|2447983|2448258|N|N|N|N|N| +2448349|AAAAAAAANNLFFCAA|1991-04-02|1095|4762|366|1991|2|4|2|2|1991|366|4762|Tuesday|1991Q2|N|N|N|2448348|2448437|2447984|2448259|N|N|N|N|N| +2448350|AAAAAAAAONLFFCAA|1991-04-03|1095|4762|366|1991|3|4|3|2|1991|366|4762|Wednesday|1991Q2|N|N|N|2448348|2448437|2447985|2448260|N|N|N|N|N| +2448351|AAAAAAAAPNLFFCAA|1991-04-04|1095|4762|366|1991|4|4|4|2|1991|366|4762|Thursday|1991Q2|N|N|N|2448348|2448437|2447986|2448261|N|N|N|N|N| +2448352|AAAAAAAAAOLFFCAA|1991-04-05|1095|4762|366|1991|5|4|5|2|1991|366|4762|Friday|1991Q2|N|Y|N|2448348|2448437|2447987|2448262|N|N|N|N|N| +2448353|AAAAAAAABOLFFCAA|1991-04-06|1095|4762|366|1991|6|4|6|2|1991|366|4762|Saturday|1991Q2|N|Y|N|2448348|2448437|2447988|2448263|N|N|N|N|N| +2448354|AAAAAAAACOLFFCAA|1991-04-07|1095|4762|366|1991|0|4|7|2|1991|366|4762|Sunday|1991Q2|N|N|N|2448348|2448437|2447989|2448264|N|N|N|N|N| +2448355|AAAAAAAADOLFFCAA|1991-04-08|1095|4762|366|1991|1|4|8|2|1991|366|4762|Monday|1991Q2|N|N|N|2448348|2448437|2447990|2448265|N|N|N|N|N| +2448356|AAAAAAAAEOLFFCAA|1991-04-09|1095|4763|366|1991|2|4|9|2|1991|366|4763|Tuesday|1991Q2|N|N|N|2448348|2448437|2447991|2448266|N|N|N|N|N| +2448357|AAAAAAAAFOLFFCAA|1991-04-10|1095|4763|366|1991|3|4|10|2|1991|366|4763|Wednesday|1991Q2|N|N|N|2448348|2448437|2447992|2448267|N|N|N|N|N| +2448358|AAAAAAAAGOLFFCAA|1991-04-11|1095|4763|366|1991|4|4|11|2|1991|366|4763|Thursday|1991Q2|N|N|N|2448348|2448437|2447993|2448268|N|N|N|N|N| +2448359|AAAAAAAAHOLFFCAA|1991-04-12|1095|4763|366|1991|5|4|12|2|1991|366|4763|Friday|1991Q2|N|Y|N|2448348|2448437|2447994|2448269|N|N|N|N|N| +2448360|AAAAAAAAIOLFFCAA|1991-04-13|1095|4763|366|1991|6|4|13|2|1991|366|4763|Saturday|1991Q2|N|Y|N|2448348|2448437|2447995|2448270|N|N|N|N|N| +2448361|AAAAAAAAJOLFFCAA|1991-04-14|1095|4763|366|1991|0|4|14|2|1991|366|4763|Sunday|1991Q2|N|N|N|2448348|2448437|2447996|2448271|N|N|N|N|N| +2448362|AAAAAAAAKOLFFCAA|1991-04-15|1095|4763|366|1991|1|4|15|2|1991|366|4763|Monday|1991Q2|N|N|N|2448348|2448437|2447997|2448272|N|N|N|N|N| +2448363|AAAAAAAALOLFFCAA|1991-04-16|1095|4764|366|1991|2|4|16|2|1991|366|4764|Tuesday|1991Q2|N|N|N|2448348|2448437|2447998|2448273|N|N|N|N|N| +2448364|AAAAAAAAMOLFFCAA|1991-04-17|1095|4764|366|1991|3|4|17|2|1991|366|4764|Wednesday|1991Q2|N|N|N|2448348|2448437|2447999|2448274|N|N|N|N|N| +2448365|AAAAAAAANOLFFCAA|1991-04-18|1095|4764|366|1991|4|4|18|2|1991|366|4764|Thursday|1991Q2|N|N|N|2448348|2448437|2448000|2448275|N|N|N|N|N| +2448366|AAAAAAAAOOLFFCAA|1991-04-19|1095|4764|366|1991|5|4|19|2|1991|366|4764|Friday|1991Q2|N|Y|N|2448348|2448437|2448001|2448276|N|N|N|N|N| +2448367|AAAAAAAAPOLFFCAA|1991-04-20|1095|4764|366|1991|6|4|20|2|1991|366|4764|Saturday|1991Q2|N|Y|N|2448348|2448437|2448002|2448277|N|N|N|N|N| +2448368|AAAAAAAAAPLFFCAA|1991-04-21|1095|4764|366|1991|0|4|21|2|1991|366|4764|Sunday|1991Q2|N|N|N|2448348|2448437|2448003|2448278|N|N|N|N|N| +2448369|AAAAAAAABPLFFCAA|1991-04-22|1095|4764|366|1991|1|4|22|2|1991|366|4764|Monday|1991Q2|N|N|N|2448348|2448437|2448004|2448279|N|N|N|N|N| +2448370|AAAAAAAACPLFFCAA|1991-04-23|1095|4765|366|1991|2|4|23|2|1991|366|4765|Tuesday|1991Q2|N|N|N|2448348|2448437|2448005|2448280|N|N|N|N|N| +2448371|AAAAAAAADPLFFCAA|1991-04-24|1095|4765|366|1991|3|4|24|2|1991|366|4765|Wednesday|1991Q2|N|N|N|2448348|2448437|2448006|2448281|N|N|N|N|N| +2448372|AAAAAAAAEPLFFCAA|1991-04-25|1095|4765|366|1991|4|4|25|2|1991|366|4765|Thursday|1991Q2|N|N|N|2448348|2448437|2448007|2448282|N|N|N|N|N| +2448373|AAAAAAAAFPLFFCAA|1991-04-26|1095|4765|366|1991|5|4|26|2|1991|366|4765|Friday|1991Q2|N|Y|N|2448348|2448437|2448008|2448283|N|N|N|N|N| +2448374|AAAAAAAAGPLFFCAA|1991-04-27|1095|4765|366|1991|6|4|27|2|1991|366|4765|Saturday|1991Q2|N|Y|N|2448348|2448437|2448009|2448284|N|N|N|N|N| +2448375|AAAAAAAAHPLFFCAA|1991-04-28|1095|4765|366|1991|0|4|28|2|1991|366|4765|Sunday|1991Q2|N|N|N|2448348|2448437|2448010|2448285|N|N|N|N|N| +2448376|AAAAAAAAIPLFFCAA|1991-04-29|1095|4765|366|1991|1|4|29|2|1991|366|4765|Monday|1991Q2|N|N|N|2448348|2448437|2448011|2448286|N|N|N|N|N| +2448377|AAAAAAAAJPLFFCAA|1991-04-30|1095|4766|366|1991|2|4|30|2|1991|366|4766|Tuesday|1991Q2|N|N|N|2448348|2448437|2448012|2448287|N|N|N|N|N| +2448378|AAAAAAAAKPLFFCAA|1991-05-01|1096|4766|366|1991|3|5|1|2|1991|366|4766|Wednesday|1991Q2|N|N|N|2448378|2448497|2448013|2448288|N|N|N|N|N| +2448379|AAAAAAAALPLFFCAA|1991-05-02|1096|4766|366|1991|4|5|2|2|1991|366|4766|Thursday|1991Q2|N|N|N|2448378|2448497|2448014|2448289|N|N|N|N|N| +2448380|AAAAAAAAMPLFFCAA|1991-05-03|1096|4766|366|1991|5|5|3|2|1991|366|4766|Friday|1991Q2|N|Y|N|2448378|2448497|2448015|2448290|N|N|N|N|N| +2448381|AAAAAAAANPLFFCAA|1991-05-04|1096|4766|366|1991|6|5|4|2|1991|366|4766|Saturday|1991Q2|N|Y|N|2448378|2448497|2448016|2448291|N|N|N|N|N| +2448382|AAAAAAAAOPLFFCAA|1991-05-05|1096|4766|366|1991|0|5|5|2|1991|366|4766|Sunday|1991Q2|N|N|N|2448378|2448497|2448017|2448292|N|N|N|N|N| +2448383|AAAAAAAAPPLFFCAA|1991-05-06|1096|4766|366|1991|1|5|6|2|1991|366|4766|Monday|1991Q2|N|N|N|2448378|2448497|2448018|2448293|N|N|N|N|N| +2448384|AAAAAAAAAAMFFCAA|1991-05-07|1096|4767|366|1991|2|5|7|2|1991|366|4767|Tuesday|1991Q2|N|N|N|2448378|2448497|2448019|2448294|N|N|N|N|N| +2448385|AAAAAAAABAMFFCAA|1991-05-08|1096|4767|366|1991|3|5|8|2|1991|366|4767|Wednesday|1991Q2|N|N|N|2448378|2448497|2448020|2448295|N|N|N|N|N| +2448386|AAAAAAAACAMFFCAA|1991-05-09|1096|4767|366|1991|4|5|9|2|1991|366|4767|Thursday|1991Q2|N|N|N|2448378|2448497|2448021|2448296|N|N|N|N|N| +2448387|AAAAAAAADAMFFCAA|1991-05-10|1096|4767|366|1991|5|5|10|2|1991|366|4767|Friday|1991Q2|N|Y|N|2448378|2448497|2448022|2448297|N|N|N|N|N| +2448388|AAAAAAAAEAMFFCAA|1991-05-11|1096|4767|366|1991|6|5|11|2|1991|366|4767|Saturday|1991Q2|N|Y|N|2448378|2448497|2448023|2448298|N|N|N|N|N| +2448389|AAAAAAAAFAMFFCAA|1991-05-12|1096|4767|366|1991|0|5|12|2|1991|366|4767|Sunday|1991Q2|N|N|N|2448378|2448497|2448024|2448299|N|N|N|N|N| +2448390|AAAAAAAAGAMFFCAA|1991-05-13|1096|4767|366|1991|1|5|13|2|1991|366|4767|Monday|1991Q2|N|N|N|2448378|2448497|2448025|2448300|N|N|N|N|N| +2448391|AAAAAAAAHAMFFCAA|1991-05-14|1096|4768|366|1991|2|5|14|2|1991|366|4768|Tuesday|1991Q2|N|N|N|2448378|2448497|2448026|2448301|N|N|N|N|N| +2448392|AAAAAAAAIAMFFCAA|1991-05-15|1096|4768|366|1991|3|5|15|2|1991|366|4768|Wednesday|1991Q2|N|N|N|2448378|2448497|2448027|2448302|N|N|N|N|N| +2448393|AAAAAAAAJAMFFCAA|1991-05-16|1096|4768|366|1991|4|5|16|2|1991|366|4768|Thursday|1991Q2|N|N|N|2448378|2448497|2448028|2448303|N|N|N|N|N| +2448394|AAAAAAAAKAMFFCAA|1991-05-17|1096|4768|366|1991|5|5|17|2|1991|366|4768|Friday|1991Q2|N|Y|N|2448378|2448497|2448029|2448304|N|N|N|N|N| +2448395|AAAAAAAALAMFFCAA|1991-05-18|1096|4768|366|1991|6|5|18|2|1991|366|4768|Saturday|1991Q2|N|Y|N|2448378|2448497|2448030|2448305|N|N|N|N|N| +2448396|AAAAAAAAMAMFFCAA|1991-05-19|1096|4768|366|1991|0|5|19|2|1991|366|4768|Sunday|1991Q2|N|N|N|2448378|2448497|2448031|2448306|N|N|N|N|N| +2448397|AAAAAAAANAMFFCAA|1991-05-20|1096|4768|366|1991|1|5|20|2|1991|366|4768|Monday|1991Q2|N|N|N|2448378|2448497|2448032|2448307|N|N|N|N|N| +2448398|AAAAAAAAOAMFFCAA|1991-05-21|1096|4769|366|1991|2|5|21|2|1991|366|4769|Tuesday|1991Q2|N|N|N|2448378|2448497|2448033|2448308|N|N|N|N|N| +2448399|AAAAAAAAPAMFFCAA|1991-05-22|1096|4769|366|1991|3|5|22|2|1991|366|4769|Wednesday|1991Q2|N|N|N|2448378|2448497|2448034|2448309|N|N|N|N|N| +2448400|AAAAAAAAABMFFCAA|1991-05-23|1096|4769|366|1991|4|5|23|2|1991|366|4769|Thursday|1991Q2|N|N|N|2448378|2448497|2448035|2448310|N|N|N|N|N| +2448401|AAAAAAAABBMFFCAA|1991-05-24|1096|4769|366|1991|5|5|24|2|1991|366|4769|Friday|1991Q2|N|Y|N|2448378|2448497|2448036|2448311|N|N|N|N|N| +2448402|AAAAAAAACBMFFCAA|1991-05-25|1096|4769|366|1991|6|5|25|2|1991|366|4769|Saturday|1991Q2|N|Y|N|2448378|2448497|2448037|2448312|N|N|N|N|N| +2448403|AAAAAAAADBMFFCAA|1991-05-26|1096|4769|366|1991|0|5|26|2|1991|366|4769|Sunday|1991Q2|N|N|N|2448378|2448497|2448038|2448313|N|N|N|N|N| +2448404|AAAAAAAAEBMFFCAA|1991-05-27|1096|4769|366|1991|1|5|27|2|1991|366|4769|Monday|1991Q2|N|N|N|2448378|2448497|2448039|2448314|N|N|N|N|N| +2448405|AAAAAAAAFBMFFCAA|1991-05-28|1096|4770|366|1991|2|5|28|2|1991|366|4770|Tuesday|1991Q2|N|N|N|2448378|2448497|2448040|2448315|N|N|N|N|N| +2448406|AAAAAAAAGBMFFCAA|1991-05-29|1096|4770|366|1991|3|5|29|2|1991|366|4770|Wednesday|1991Q2|N|N|N|2448378|2448497|2448041|2448316|N|N|N|N|N| +2448407|AAAAAAAAHBMFFCAA|1991-05-30|1096|4770|366|1991|4|5|30|2|1991|366|4770|Thursday|1991Q2|N|N|N|2448378|2448497|2448042|2448317|N|N|N|N|N| +2448408|AAAAAAAAIBMFFCAA|1991-05-31|1096|4770|366|1991|5|5|31|2|1991|366|4770|Friday|1991Q2|N|Y|N|2448378|2448497|2448043|2448318|N|N|N|N|N| +2448409|AAAAAAAAJBMFFCAA|1991-06-01|1097|4770|367|1991|6|6|1|2|1991|367|4770|Saturday|1991Q2|N|Y|N|2448409|2448559|2448044|2448319|N|N|N|N|N| +2448410|AAAAAAAAKBMFFCAA|1991-06-02|1097|4770|367|1991|0|6|2|2|1991|367|4770|Sunday|1991Q2|N|N|N|2448409|2448559|2448045|2448320|N|N|N|N|N| +2448411|AAAAAAAALBMFFCAA|1991-06-03|1097|4770|367|1991|1|6|3|2|1991|367|4770|Monday|1991Q2|N|N|N|2448409|2448559|2448046|2448321|N|N|N|N|N| +2448412|AAAAAAAAMBMFFCAA|1991-06-04|1097|4771|367|1991|2|6|4|2|1991|367|4771|Tuesday|1991Q2|N|N|N|2448409|2448559|2448047|2448322|N|N|N|N|N| +2448413|AAAAAAAANBMFFCAA|1991-06-05|1097|4771|367|1991|3|6|5|2|1991|367|4771|Wednesday|1991Q2|N|N|N|2448409|2448559|2448048|2448323|N|N|N|N|N| +2448414|AAAAAAAAOBMFFCAA|1991-06-06|1097|4771|367|1991|4|6|6|2|1991|367|4771|Thursday|1991Q2|N|N|N|2448409|2448559|2448049|2448324|N|N|N|N|N| +2448415|AAAAAAAAPBMFFCAA|1991-06-07|1097|4771|367|1991|5|6|7|2|1991|367|4771|Friday|1991Q2|N|Y|N|2448409|2448559|2448050|2448325|N|N|N|N|N| +2448416|AAAAAAAAACMFFCAA|1991-06-08|1097|4771|367|1991|6|6|8|2|1991|367|4771|Saturday|1991Q2|N|Y|N|2448409|2448559|2448051|2448326|N|N|N|N|N| +2448417|AAAAAAAABCMFFCAA|1991-06-09|1097|4771|367|1991|0|6|9|2|1991|367|4771|Sunday|1991Q2|N|N|N|2448409|2448559|2448052|2448327|N|N|N|N|N| +2448418|AAAAAAAACCMFFCAA|1991-06-10|1097|4771|367|1991|1|6|10|2|1991|367|4771|Monday|1991Q2|N|N|N|2448409|2448559|2448053|2448328|N|N|N|N|N| +2448419|AAAAAAAADCMFFCAA|1991-06-11|1097|4772|367|1991|2|6|11|2|1991|367|4772|Tuesday|1991Q2|N|N|N|2448409|2448559|2448054|2448329|N|N|N|N|N| +2448420|AAAAAAAAECMFFCAA|1991-06-12|1097|4772|367|1991|3|6|12|2|1991|367|4772|Wednesday|1991Q2|N|N|N|2448409|2448559|2448055|2448330|N|N|N|N|N| +2448421|AAAAAAAAFCMFFCAA|1991-06-13|1097|4772|367|1991|4|6|13|2|1991|367|4772|Thursday|1991Q2|N|N|N|2448409|2448559|2448056|2448331|N|N|N|N|N| +2448422|AAAAAAAAGCMFFCAA|1991-06-14|1097|4772|367|1991|5|6|14|2|1991|367|4772|Friday|1991Q2|N|Y|N|2448409|2448559|2448057|2448332|N|N|N|N|N| +2448423|AAAAAAAAHCMFFCAA|1991-06-15|1097|4772|367|1991|6|6|15|2|1991|367|4772|Saturday|1991Q2|N|Y|N|2448409|2448559|2448058|2448333|N|N|N|N|N| +2448424|AAAAAAAAICMFFCAA|1991-06-16|1097|4772|367|1991|0|6|16|2|1991|367|4772|Sunday|1991Q2|N|N|N|2448409|2448559|2448059|2448334|N|N|N|N|N| +2448425|AAAAAAAAJCMFFCAA|1991-06-17|1097|4772|367|1991|1|6|17|2|1991|367|4772|Monday|1991Q2|N|N|N|2448409|2448559|2448060|2448335|N|N|N|N|N| +2448426|AAAAAAAAKCMFFCAA|1991-06-18|1097|4773|367|1991|2|6|18|2|1991|367|4773|Tuesday|1991Q2|N|N|N|2448409|2448559|2448061|2448336|N|N|N|N|N| +2448427|AAAAAAAALCMFFCAA|1991-06-19|1097|4773|367|1991|3|6|19|2|1991|367|4773|Wednesday|1991Q2|N|N|N|2448409|2448559|2448062|2448337|N|N|N|N|N| +2448428|AAAAAAAAMCMFFCAA|1991-06-20|1097|4773|367|1991|4|6|20|2|1991|367|4773|Thursday|1991Q2|N|N|N|2448409|2448559|2448063|2448338|N|N|N|N|N| +2448429|AAAAAAAANCMFFCAA|1991-06-21|1097|4773|367|1991|5|6|21|2|1991|367|4773|Friday|1991Q2|N|Y|N|2448409|2448559|2448064|2448339|N|N|N|N|N| +2448430|AAAAAAAAOCMFFCAA|1991-06-22|1097|4773|367|1991|6|6|22|2|1991|367|4773|Saturday|1991Q2|N|Y|N|2448409|2448559|2448065|2448340|N|N|N|N|N| +2448431|AAAAAAAAPCMFFCAA|1991-06-23|1097|4773|367|1991|0|6|23|2|1991|367|4773|Sunday|1991Q2|N|N|N|2448409|2448559|2448066|2448341|N|N|N|N|N| +2448432|AAAAAAAAADMFFCAA|1991-06-24|1097|4773|367|1991|1|6|24|2|1991|367|4773|Monday|1991Q2|N|N|N|2448409|2448559|2448067|2448342|N|N|N|N|N| +2448433|AAAAAAAABDMFFCAA|1991-06-25|1097|4774|367|1991|2|6|25|2|1991|367|4774|Tuesday|1991Q2|N|N|N|2448409|2448559|2448068|2448343|N|N|N|N|N| +2448434|AAAAAAAACDMFFCAA|1991-06-26|1097|4774|367|1991|3|6|26|2|1991|367|4774|Wednesday|1991Q2|N|N|N|2448409|2448559|2448069|2448344|N|N|N|N|N| +2448435|AAAAAAAADDMFFCAA|1991-06-27|1097|4774|367|1991|4|6|27|2|1991|367|4774|Thursday|1991Q2|N|N|N|2448409|2448559|2448070|2448345|N|N|N|N|N| +2448436|AAAAAAAAEDMFFCAA|1991-06-28|1097|4774|367|1991|5|6|28|2|1991|367|4774|Friday|1991Q2|N|Y|N|2448409|2448559|2448071|2448346|N|N|N|N|N| +2448437|AAAAAAAAFDMFFCAA|1991-06-29|1097|4774|367|1991|6|6|29|2|1991|367|4774|Saturday|1991Q2|N|Y|N|2448409|2448559|2448072|2448347|N|N|N|N|N| +2448438|AAAAAAAAGDMFFCAA|1991-06-30|1097|4774|367|1991|0|6|30|2|1991|367|4774|Sunday|1991Q2|N|N|N|2448409|2448559|2448073|2448348|N|N|N|N|N| +2448439|AAAAAAAAHDMFFCAA|1991-07-01|1098|4774|367|1991|1|7|1|2|1991|367|4774|Monday|1991Q2|N|N|N|2448439|2448619|2448074|2448348|N|N|N|N|N| +2448440|AAAAAAAAIDMFFCAA|1991-07-02|1098|4775|367|1991|2|7|2|3|1991|367|4775|Tuesday|1991Q3|N|N|N|2448439|2448619|2448075|2448349|N|N|N|N|N| +2448441|AAAAAAAAJDMFFCAA|1991-07-03|1098|4775|367|1991|3|7|3|3|1991|367|4775|Wednesday|1991Q3|N|N|N|2448439|2448619|2448076|2448350|N|N|N|N|N| +2448442|AAAAAAAAKDMFFCAA|1991-07-04|1098|4775|367|1991|4|7|4|3|1991|367|4775|Thursday|1991Q3|N|N|N|2448439|2448619|2448077|2448351|N|N|N|N|N| +2448443|AAAAAAAALDMFFCAA|1991-07-05|1098|4775|367|1991|5|7|5|3|1991|367|4775|Friday|1991Q3|Y|Y|N|2448439|2448619|2448078|2448352|N|N|N|N|N| +2448444|AAAAAAAAMDMFFCAA|1991-07-06|1098|4775|367|1991|6|7|6|3|1991|367|4775|Saturday|1991Q3|N|Y|Y|2448439|2448619|2448079|2448353|N|N|N|N|N| +2448445|AAAAAAAANDMFFCAA|1991-07-07|1098|4775|367|1991|0|7|7|3|1991|367|4775|Sunday|1991Q3|N|N|N|2448439|2448619|2448080|2448354|N|N|N|N|N| +2448446|AAAAAAAAODMFFCAA|1991-07-08|1098|4775|367|1991|1|7|8|3|1991|367|4775|Monday|1991Q3|N|N|N|2448439|2448619|2448081|2448355|N|N|N|N|N| +2448447|AAAAAAAAPDMFFCAA|1991-07-09|1098|4776|367|1991|2|7|9|3|1991|367|4776|Tuesday|1991Q3|N|N|N|2448439|2448619|2448082|2448356|N|N|N|N|N| +2448448|AAAAAAAAAEMFFCAA|1991-07-10|1098|4776|367|1991|3|7|10|3|1991|367|4776|Wednesday|1991Q3|N|N|N|2448439|2448619|2448083|2448357|N|N|N|N|N| +2448449|AAAAAAAABEMFFCAA|1991-07-11|1098|4776|367|1991|4|7|11|3|1991|367|4776|Thursday|1991Q3|N|N|N|2448439|2448619|2448084|2448358|N|N|N|N|N| +2448450|AAAAAAAACEMFFCAA|1991-07-12|1098|4776|367|1991|5|7|12|3|1991|367|4776|Friday|1991Q3|N|Y|N|2448439|2448619|2448085|2448359|N|N|N|N|N| +2448451|AAAAAAAADEMFFCAA|1991-07-13|1098|4776|367|1991|6|7|13|3|1991|367|4776|Saturday|1991Q3|N|Y|N|2448439|2448619|2448086|2448360|N|N|N|N|N| +2448452|AAAAAAAAEEMFFCAA|1991-07-14|1098|4776|367|1991|0|7|14|3|1991|367|4776|Sunday|1991Q3|N|N|N|2448439|2448619|2448087|2448361|N|N|N|N|N| +2448453|AAAAAAAAFEMFFCAA|1991-07-15|1098|4776|367|1991|1|7|15|3|1991|367|4776|Monday|1991Q3|N|N|N|2448439|2448619|2448088|2448362|N|N|N|N|N| +2448454|AAAAAAAAGEMFFCAA|1991-07-16|1098|4777|367|1991|2|7|16|3|1991|367|4777|Tuesday|1991Q3|N|N|N|2448439|2448619|2448089|2448363|N|N|N|N|N| +2448455|AAAAAAAAHEMFFCAA|1991-07-17|1098|4777|367|1991|3|7|17|3|1991|367|4777|Wednesday|1991Q3|N|N|N|2448439|2448619|2448090|2448364|N|N|N|N|N| +2448456|AAAAAAAAIEMFFCAA|1991-07-18|1098|4777|367|1991|4|7|18|3|1991|367|4777|Thursday|1991Q3|N|N|N|2448439|2448619|2448091|2448365|N|N|N|N|N| +2448457|AAAAAAAAJEMFFCAA|1991-07-19|1098|4777|367|1991|5|7|19|3|1991|367|4777|Friday|1991Q3|N|Y|N|2448439|2448619|2448092|2448366|N|N|N|N|N| +2448458|AAAAAAAAKEMFFCAA|1991-07-20|1098|4777|367|1991|6|7|20|3|1991|367|4777|Saturday|1991Q3|N|Y|N|2448439|2448619|2448093|2448367|N|N|N|N|N| +2448459|AAAAAAAALEMFFCAA|1991-07-21|1098|4777|367|1991|0|7|21|3|1991|367|4777|Sunday|1991Q3|N|N|N|2448439|2448619|2448094|2448368|N|N|N|N|N| +2448460|AAAAAAAAMEMFFCAA|1991-07-22|1098|4777|367|1991|1|7|22|3|1991|367|4777|Monday|1991Q3|N|N|N|2448439|2448619|2448095|2448369|N|N|N|N|N| +2448461|AAAAAAAANEMFFCAA|1991-07-23|1098|4778|367|1991|2|7|23|3|1991|367|4778|Tuesday|1991Q3|N|N|N|2448439|2448619|2448096|2448370|N|N|N|N|N| +2448462|AAAAAAAAOEMFFCAA|1991-07-24|1098|4778|367|1991|3|7|24|3|1991|367|4778|Wednesday|1991Q3|N|N|N|2448439|2448619|2448097|2448371|N|N|N|N|N| +2448463|AAAAAAAAPEMFFCAA|1991-07-25|1098|4778|367|1991|4|7|25|3|1991|367|4778|Thursday|1991Q3|N|N|N|2448439|2448619|2448098|2448372|N|N|N|N|N| +2448464|AAAAAAAAAFMFFCAA|1991-07-26|1098|4778|367|1991|5|7|26|3|1991|367|4778|Friday|1991Q3|N|Y|N|2448439|2448619|2448099|2448373|N|N|N|N|N| +2448465|AAAAAAAABFMFFCAA|1991-07-27|1098|4778|367|1991|6|7|27|3|1991|367|4778|Saturday|1991Q3|N|Y|N|2448439|2448619|2448100|2448374|N|N|N|N|N| +2448466|AAAAAAAACFMFFCAA|1991-07-28|1098|4778|367|1991|0|7|28|3|1991|367|4778|Sunday|1991Q3|N|N|N|2448439|2448619|2448101|2448375|N|N|N|N|N| +2448467|AAAAAAAADFMFFCAA|1991-07-29|1098|4778|367|1991|1|7|29|3|1991|367|4778|Monday|1991Q3|N|N|N|2448439|2448619|2448102|2448376|N|N|N|N|N| +2448468|AAAAAAAAEFMFFCAA|1991-07-30|1098|4779|367|1991|2|7|30|3|1991|367|4779|Tuesday|1991Q3|N|N|N|2448439|2448619|2448103|2448377|N|N|N|N|N| +2448469|AAAAAAAAFFMFFCAA|1991-07-31|1098|4779|367|1991|3|7|31|3|1991|367|4779|Wednesday|1991Q3|N|N|N|2448439|2448619|2448104|2448378|N|N|N|N|N| +2448470|AAAAAAAAGFMFFCAA|1991-08-01|1099|4779|367|1991|4|8|1|3|1991|367|4779|Thursday|1991Q3|N|N|N|2448470|2448681|2448105|2448379|N|N|N|N|N| +2448471|AAAAAAAAHFMFFCAA|1991-08-02|1099|4779|367|1991|5|8|2|3|1991|367|4779|Friday|1991Q3|N|Y|N|2448470|2448681|2448106|2448380|N|N|N|N|N| +2448472|AAAAAAAAIFMFFCAA|1991-08-03|1099|4779|367|1991|6|8|3|3|1991|367|4779|Saturday|1991Q3|N|Y|N|2448470|2448681|2448107|2448381|N|N|N|N|N| +2448473|AAAAAAAAJFMFFCAA|1991-08-04|1099|4779|367|1991|0|8|4|3|1991|367|4779|Sunday|1991Q3|N|N|N|2448470|2448681|2448108|2448382|N|N|N|N|N| +2448474|AAAAAAAAKFMFFCAA|1991-08-05|1099|4779|367|1991|1|8|5|3|1991|367|4779|Monday|1991Q3|N|N|N|2448470|2448681|2448109|2448383|N|N|N|N|N| +2448475|AAAAAAAALFMFFCAA|1991-08-06|1099|4780|367|1991|2|8|6|3|1991|367|4780|Tuesday|1991Q3|N|N|N|2448470|2448681|2448110|2448384|N|N|N|N|N| +2448476|AAAAAAAAMFMFFCAA|1991-08-07|1099|4780|367|1991|3|8|7|3|1991|367|4780|Wednesday|1991Q3|N|N|N|2448470|2448681|2448111|2448385|N|N|N|N|N| +2448477|AAAAAAAANFMFFCAA|1991-08-08|1099|4780|367|1991|4|8|8|3|1991|367|4780|Thursday|1991Q3|N|N|N|2448470|2448681|2448112|2448386|N|N|N|N|N| +2448478|AAAAAAAAOFMFFCAA|1991-08-09|1099|4780|367|1991|5|8|9|3|1991|367|4780|Friday|1991Q3|N|Y|N|2448470|2448681|2448113|2448387|N|N|N|N|N| +2448479|AAAAAAAAPFMFFCAA|1991-08-10|1099|4780|367|1991|6|8|10|3|1991|367|4780|Saturday|1991Q3|N|Y|N|2448470|2448681|2448114|2448388|N|N|N|N|N| +2448480|AAAAAAAAAGMFFCAA|1991-08-11|1099|4780|367|1991|0|8|11|3|1991|367|4780|Sunday|1991Q3|N|N|N|2448470|2448681|2448115|2448389|N|N|N|N|N| +2448481|AAAAAAAABGMFFCAA|1991-08-12|1099|4780|367|1991|1|8|12|3|1991|367|4780|Monday|1991Q3|N|N|N|2448470|2448681|2448116|2448390|N|N|N|N|N| +2448482|AAAAAAAACGMFFCAA|1991-08-13|1099|4781|367|1991|2|8|13|3|1991|367|4781|Tuesday|1991Q3|N|N|N|2448470|2448681|2448117|2448391|N|N|N|N|N| +2448483|AAAAAAAADGMFFCAA|1991-08-14|1099|4781|367|1991|3|8|14|3|1991|367|4781|Wednesday|1991Q3|N|N|N|2448470|2448681|2448118|2448392|N|N|N|N|N| +2448484|AAAAAAAAEGMFFCAA|1991-08-15|1099|4781|367|1991|4|8|15|3|1991|367|4781|Thursday|1991Q3|N|N|N|2448470|2448681|2448119|2448393|N|N|N|N|N| +2448485|AAAAAAAAFGMFFCAA|1991-08-16|1099|4781|367|1991|5|8|16|3|1991|367|4781|Friday|1991Q3|N|Y|N|2448470|2448681|2448120|2448394|N|N|N|N|N| +2448486|AAAAAAAAGGMFFCAA|1991-08-17|1099|4781|367|1991|6|8|17|3|1991|367|4781|Saturday|1991Q3|N|Y|N|2448470|2448681|2448121|2448395|N|N|N|N|N| +2448487|AAAAAAAAHGMFFCAA|1991-08-18|1099|4781|367|1991|0|8|18|3|1991|367|4781|Sunday|1991Q3|N|N|N|2448470|2448681|2448122|2448396|N|N|N|N|N| +2448488|AAAAAAAAIGMFFCAA|1991-08-19|1099|4781|367|1991|1|8|19|3|1991|367|4781|Monday|1991Q3|N|N|N|2448470|2448681|2448123|2448397|N|N|N|N|N| +2448489|AAAAAAAAJGMFFCAA|1991-08-20|1099|4782|367|1991|2|8|20|3|1991|367|4782|Tuesday|1991Q3|N|N|N|2448470|2448681|2448124|2448398|N|N|N|N|N| +2448490|AAAAAAAAKGMFFCAA|1991-08-21|1099|4782|367|1991|3|8|21|3|1991|367|4782|Wednesday|1991Q3|N|N|N|2448470|2448681|2448125|2448399|N|N|N|N|N| +2448491|AAAAAAAALGMFFCAA|1991-08-22|1099|4782|367|1991|4|8|22|3|1991|367|4782|Thursday|1991Q3|N|N|N|2448470|2448681|2448126|2448400|N|N|N|N|N| +2448492|AAAAAAAAMGMFFCAA|1991-08-23|1099|4782|367|1991|5|8|23|3|1991|367|4782|Friday|1991Q3|N|Y|N|2448470|2448681|2448127|2448401|N|N|N|N|N| +2448493|AAAAAAAANGMFFCAA|1991-08-24|1099|4782|367|1991|6|8|24|3|1991|367|4782|Saturday|1991Q3|N|Y|N|2448470|2448681|2448128|2448402|N|N|N|N|N| +2448494|AAAAAAAAOGMFFCAA|1991-08-25|1099|4782|367|1991|0|8|25|3|1991|367|4782|Sunday|1991Q3|N|N|N|2448470|2448681|2448129|2448403|N|N|N|N|N| +2448495|AAAAAAAAPGMFFCAA|1991-08-26|1099|4782|367|1991|1|8|26|3|1991|367|4782|Monday|1991Q3|N|N|N|2448470|2448681|2448130|2448404|N|N|N|N|N| +2448496|AAAAAAAAAHMFFCAA|1991-08-27|1099|4783|367|1991|2|8|27|3|1991|367|4783|Tuesday|1991Q3|N|N|N|2448470|2448681|2448131|2448405|N|N|N|N|N| +2448497|AAAAAAAABHMFFCAA|1991-08-28|1099|4783|367|1991|3|8|28|3|1991|367|4783|Wednesday|1991Q3|N|N|N|2448470|2448681|2448132|2448406|N|N|N|N|N| +2448498|AAAAAAAACHMFFCAA|1991-08-29|1099|4783|367|1991|4|8|29|3|1991|367|4783|Thursday|1991Q3|N|N|N|2448470|2448681|2448133|2448407|N|N|N|N|N| +2448499|AAAAAAAADHMFFCAA|1991-08-30|1099|4783|367|1991|5|8|30|3|1991|367|4783|Friday|1991Q3|N|Y|N|2448470|2448681|2448134|2448408|N|N|N|N|N| +2448500|AAAAAAAAEHMFFCAA|1991-08-31|1099|4783|367|1991|6|8|31|3|1991|367|4783|Saturday|1991Q3|N|Y|N|2448470|2448681|2448135|2448409|N|N|N|N|N| +2448501|AAAAAAAAFHMFFCAA|1991-09-01|1100|4783|368|1991|0|9|1|3|1991|368|4783|Sunday|1991Q3|N|N|N|2448501|2448743|2448136|2448410|N|N|N|N|N| +2448502|AAAAAAAAGHMFFCAA|1991-09-02|1100|4783|368|1991|1|9|2|3|1991|368|4783|Monday|1991Q3|N|N|N|2448501|2448743|2448137|2448411|N|N|N|N|N| +2448503|AAAAAAAAHHMFFCAA|1991-09-03|1100|4784|368|1991|2|9|3|3|1991|368|4784|Tuesday|1991Q3|N|N|N|2448501|2448743|2448138|2448412|N|N|N|N|N| +2448504|AAAAAAAAIHMFFCAA|1991-09-04|1100|4784|368|1991|3|9|4|3|1991|368|4784|Wednesday|1991Q3|N|N|N|2448501|2448743|2448139|2448413|N|N|N|N|N| +2448505|AAAAAAAAJHMFFCAA|1991-09-05|1100|4784|368|1991|4|9|5|3|1991|368|4784|Thursday|1991Q3|N|N|N|2448501|2448743|2448140|2448414|N|N|N|N|N| +2448506|AAAAAAAAKHMFFCAA|1991-09-06|1100|4784|368|1991|5|9|6|3|1991|368|4784|Friday|1991Q3|N|Y|N|2448501|2448743|2448141|2448415|N|N|N|N|N| +2448507|AAAAAAAALHMFFCAA|1991-09-07|1100|4784|368|1991|6|9|7|3|1991|368|4784|Saturday|1991Q3|N|Y|N|2448501|2448743|2448142|2448416|N|N|N|N|N| +2448508|AAAAAAAAMHMFFCAA|1991-09-08|1100|4784|368|1991|0|9|8|3|1991|368|4784|Sunday|1991Q3|N|N|N|2448501|2448743|2448143|2448417|N|N|N|N|N| +2448509|AAAAAAAANHMFFCAA|1991-09-09|1100|4784|368|1991|1|9|9|3|1991|368|4784|Monday|1991Q3|N|N|N|2448501|2448743|2448144|2448418|N|N|N|N|N| +2448510|AAAAAAAAOHMFFCAA|1991-09-10|1100|4785|368|1991|2|9|10|3|1991|368|4785|Tuesday|1991Q3|N|N|N|2448501|2448743|2448145|2448419|N|N|N|N|N| +2448511|AAAAAAAAPHMFFCAA|1991-09-11|1100|4785|368|1991|3|9|11|3|1991|368|4785|Wednesday|1991Q3|N|N|N|2448501|2448743|2448146|2448420|N|N|N|N|N| +2448512|AAAAAAAAAIMFFCAA|1991-09-12|1100|4785|368|1991|4|9|12|3|1991|368|4785|Thursday|1991Q3|N|N|N|2448501|2448743|2448147|2448421|N|N|N|N|N| +2448513|AAAAAAAABIMFFCAA|1991-09-13|1100|4785|368|1991|5|9|13|3|1991|368|4785|Friday|1991Q3|N|Y|N|2448501|2448743|2448148|2448422|N|N|N|N|N| +2448514|AAAAAAAACIMFFCAA|1991-09-14|1100|4785|368|1991|6|9|14|3|1991|368|4785|Saturday|1991Q3|N|Y|N|2448501|2448743|2448149|2448423|N|N|N|N|N| +2448515|AAAAAAAADIMFFCAA|1991-09-15|1100|4785|368|1991|0|9|15|3|1991|368|4785|Sunday|1991Q3|N|N|N|2448501|2448743|2448150|2448424|N|N|N|N|N| +2448516|AAAAAAAAEIMFFCAA|1991-09-16|1100|4785|368|1991|1|9|16|3|1991|368|4785|Monday|1991Q3|N|N|N|2448501|2448743|2448151|2448425|N|N|N|N|N| +2448517|AAAAAAAAFIMFFCAA|1991-09-17|1100|4786|368|1991|2|9|17|3|1991|368|4786|Tuesday|1991Q3|N|N|N|2448501|2448743|2448152|2448426|N|N|N|N|N| +2448518|AAAAAAAAGIMFFCAA|1991-09-18|1100|4786|368|1991|3|9|18|3|1991|368|4786|Wednesday|1991Q3|N|N|N|2448501|2448743|2448153|2448427|N|N|N|N|N| +2448519|AAAAAAAAHIMFFCAA|1991-09-19|1100|4786|368|1991|4|9|19|3|1991|368|4786|Thursday|1991Q3|N|N|N|2448501|2448743|2448154|2448428|N|N|N|N|N| +2448520|AAAAAAAAIIMFFCAA|1991-09-20|1100|4786|368|1991|5|9|20|3|1991|368|4786|Friday|1991Q3|N|Y|N|2448501|2448743|2448155|2448429|N|N|N|N|N| +2448521|AAAAAAAAJIMFFCAA|1991-09-21|1100|4786|368|1991|6|9|21|3|1991|368|4786|Saturday|1991Q3|N|Y|N|2448501|2448743|2448156|2448430|N|N|N|N|N| +2448522|AAAAAAAAKIMFFCAA|1991-09-22|1100|4786|368|1991|0|9|22|3|1991|368|4786|Sunday|1991Q3|N|N|N|2448501|2448743|2448157|2448431|N|N|N|N|N| +2448523|AAAAAAAALIMFFCAA|1991-09-23|1100|4786|368|1991|1|9|23|3|1991|368|4786|Monday|1991Q3|N|N|N|2448501|2448743|2448158|2448432|N|N|N|N|N| +2448524|AAAAAAAAMIMFFCAA|1991-09-24|1100|4787|368|1991|2|9|24|3|1991|368|4787|Tuesday|1991Q3|N|N|N|2448501|2448743|2448159|2448433|N|N|N|N|N| +2448525|AAAAAAAANIMFFCAA|1991-09-25|1100|4787|368|1991|3|9|25|3|1991|368|4787|Wednesday|1991Q3|N|N|N|2448501|2448743|2448160|2448434|N|N|N|N|N| +2448526|AAAAAAAAOIMFFCAA|1991-09-26|1100|4787|368|1991|4|9|26|3|1991|368|4787|Thursday|1991Q3|N|N|N|2448501|2448743|2448161|2448435|N|N|N|N|N| +2448527|AAAAAAAAPIMFFCAA|1991-09-27|1100|4787|368|1991|5|9|27|3|1991|368|4787|Friday|1991Q3|N|Y|N|2448501|2448743|2448162|2448436|N|N|N|N|N| +2448528|AAAAAAAAAJMFFCAA|1991-09-28|1100|4787|368|1991|6|9|28|3|1991|368|4787|Saturday|1991Q3|N|Y|N|2448501|2448743|2448163|2448437|N|N|N|N|N| +2448529|AAAAAAAABJMFFCAA|1991-09-29|1100|4787|368|1991|0|9|29|3|1991|368|4787|Sunday|1991Q3|N|N|N|2448501|2448743|2448164|2448438|N|N|N|N|N| +2448530|AAAAAAAACJMFFCAA|1991-09-30|1100|4787|368|1991|1|9|30|3|1991|368|4787|Monday|1991Q3|N|N|N|2448501|2448743|2448165|2448439|N|N|N|N|N| +2448531|AAAAAAAADJMFFCAA|1991-10-01|1101|4788|368|1991|2|10|1|3|1991|368|4788|Tuesday|1991Q3|N|N|N|2448531|2448803|2448166|2448439|N|N|N|N|N| +2448532|AAAAAAAAEJMFFCAA|1991-10-02|1101|4788|368|1991|3|10|2|4|1991|368|4788|Wednesday|1991Q4|N|N|N|2448531|2448803|2448167|2448440|N|N|N|N|N| +2448533|AAAAAAAAFJMFFCAA|1991-10-03|1101|4788|368|1991|4|10|3|4|1991|368|4788|Thursday|1991Q4|N|N|N|2448531|2448803|2448168|2448441|N|N|N|N|N| +2448534|AAAAAAAAGJMFFCAA|1991-10-04|1101|4788|368|1991|5|10|4|4|1991|368|4788|Friday|1991Q4|N|Y|N|2448531|2448803|2448169|2448442|N|N|N|N|N| +2448535|AAAAAAAAHJMFFCAA|1991-10-05|1101|4788|368|1991|6|10|5|4|1991|368|4788|Saturday|1991Q4|N|Y|N|2448531|2448803|2448170|2448443|N|N|N|N|N| +2448536|AAAAAAAAIJMFFCAA|1991-10-06|1101|4788|368|1991|0|10|6|4|1991|368|4788|Sunday|1991Q4|N|N|N|2448531|2448803|2448171|2448444|N|N|N|N|N| +2448537|AAAAAAAAJJMFFCAA|1991-10-07|1101|4788|368|1991|1|10|7|4|1991|368|4788|Monday|1991Q4|N|N|N|2448531|2448803|2448172|2448445|N|N|N|N|N| +2448538|AAAAAAAAKJMFFCAA|1991-10-08|1101|4789|368|1991|2|10|8|4|1991|368|4789|Tuesday|1991Q4|N|N|N|2448531|2448803|2448173|2448446|N|N|N|N|N| +2448539|AAAAAAAALJMFFCAA|1991-10-09|1101|4789|368|1991|3|10|9|4|1991|368|4789|Wednesday|1991Q4|N|N|N|2448531|2448803|2448174|2448447|N|N|N|N|N| +2448540|AAAAAAAAMJMFFCAA|1991-10-10|1101|4789|368|1991|4|10|10|4|1991|368|4789|Thursday|1991Q4|N|N|N|2448531|2448803|2448175|2448448|N|N|N|N|N| +2448541|AAAAAAAANJMFFCAA|1991-10-11|1101|4789|368|1991|5|10|11|4|1991|368|4789|Friday|1991Q4|N|Y|N|2448531|2448803|2448176|2448449|N|N|N|N|N| +2448542|AAAAAAAAOJMFFCAA|1991-10-12|1101|4789|368|1991|6|10|12|4|1991|368|4789|Saturday|1991Q4|N|Y|N|2448531|2448803|2448177|2448450|N|N|N|N|N| +2448543|AAAAAAAAPJMFFCAA|1991-10-13|1101|4789|368|1991|0|10|13|4|1991|368|4789|Sunday|1991Q4|N|N|N|2448531|2448803|2448178|2448451|N|N|N|N|N| +2448544|AAAAAAAAAKMFFCAA|1991-10-14|1101|4789|368|1991|1|10|14|4|1991|368|4789|Monday|1991Q4|N|N|N|2448531|2448803|2448179|2448452|N|N|N|N|N| +2448545|AAAAAAAABKMFFCAA|1991-10-15|1101|4790|368|1991|2|10|15|4|1991|368|4790|Tuesday|1991Q4|N|N|N|2448531|2448803|2448180|2448453|N|N|N|N|N| +2448546|AAAAAAAACKMFFCAA|1991-10-16|1101|4790|368|1991|3|10|16|4|1991|368|4790|Wednesday|1991Q4|N|N|N|2448531|2448803|2448181|2448454|N|N|N|N|N| +2448547|AAAAAAAADKMFFCAA|1991-10-17|1101|4790|368|1991|4|10|17|4|1991|368|4790|Thursday|1991Q4|N|N|N|2448531|2448803|2448182|2448455|N|N|N|N|N| +2448548|AAAAAAAAEKMFFCAA|1991-10-18|1101|4790|368|1991|5|10|18|4|1991|368|4790|Friday|1991Q4|N|Y|N|2448531|2448803|2448183|2448456|N|N|N|N|N| +2448549|AAAAAAAAFKMFFCAA|1991-10-19|1101|4790|368|1991|6|10|19|4|1991|368|4790|Saturday|1991Q4|N|Y|N|2448531|2448803|2448184|2448457|N|N|N|N|N| +2448550|AAAAAAAAGKMFFCAA|1991-10-20|1101|4790|368|1991|0|10|20|4|1991|368|4790|Sunday|1991Q4|N|N|N|2448531|2448803|2448185|2448458|N|N|N|N|N| +2448551|AAAAAAAAHKMFFCAA|1991-10-21|1101|4790|368|1991|1|10|21|4|1991|368|4790|Monday|1991Q4|N|N|N|2448531|2448803|2448186|2448459|N|N|N|N|N| +2448552|AAAAAAAAIKMFFCAA|1991-10-22|1101|4791|368|1991|2|10|22|4|1991|368|4791|Tuesday|1991Q4|N|N|N|2448531|2448803|2448187|2448460|N|N|N|N|N| +2448553|AAAAAAAAJKMFFCAA|1991-10-23|1101|4791|368|1991|3|10|23|4|1991|368|4791|Wednesday|1991Q4|N|N|N|2448531|2448803|2448188|2448461|N|N|N|N|N| +2448554|AAAAAAAAKKMFFCAA|1991-10-24|1101|4791|368|1991|4|10|24|4|1991|368|4791|Thursday|1991Q4|N|N|N|2448531|2448803|2448189|2448462|N|N|N|N|N| +2448555|AAAAAAAALKMFFCAA|1991-10-25|1101|4791|368|1991|5|10|25|4|1991|368|4791|Friday|1991Q4|N|Y|N|2448531|2448803|2448190|2448463|N|N|N|N|N| +2448556|AAAAAAAAMKMFFCAA|1991-10-26|1101|4791|368|1991|6|10|26|4|1991|368|4791|Saturday|1991Q4|N|Y|N|2448531|2448803|2448191|2448464|N|N|N|N|N| +2448557|AAAAAAAANKMFFCAA|1991-10-27|1101|4791|368|1991|0|10|27|4|1991|368|4791|Sunday|1991Q4|N|N|N|2448531|2448803|2448192|2448465|N|N|N|N|N| +2448558|AAAAAAAAOKMFFCAA|1991-10-28|1101|4791|368|1991|1|10|28|4|1991|368|4791|Monday|1991Q4|N|N|N|2448531|2448803|2448193|2448466|N|N|N|N|N| +2448559|AAAAAAAAPKMFFCAA|1991-10-29|1101|4792|368|1991|2|10|29|4|1991|368|4792|Tuesday|1991Q4|N|N|N|2448531|2448803|2448194|2448467|N|N|N|N|N| +2448560|AAAAAAAAALMFFCAA|1991-10-30|1101|4792|368|1991|3|10|30|4|1991|368|4792|Wednesday|1991Q4|N|N|N|2448531|2448803|2448195|2448468|N|N|N|N|N| +2448561|AAAAAAAABLMFFCAA|1991-10-31|1101|4792|368|1991|4|10|31|4|1991|368|4792|Thursday|1991Q4|N|N|N|2448531|2448803|2448196|2448469|N|N|N|N|N| +2448562|AAAAAAAACLMFFCAA|1991-11-01|1102|4792|368|1991|5|11|1|4|1991|368|4792|Friday|1991Q4|N|Y|N|2448562|2448865|2448197|2448470|N|N|N|N|N| +2448563|AAAAAAAADLMFFCAA|1991-11-02|1102|4792|368|1991|6|11|2|4|1991|368|4792|Saturday|1991Q4|N|Y|N|2448562|2448865|2448198|2448471|N|N|N|N|N| +2448564|AAAAAAAAELMFFCAA|1991-11-03|1102|4792|368|1991|0|11|3|4|1991|368|4792|Sunday|1991Q4|N|N|N|2448562|2448865|2448199|2448472|N|N|N|N|N| +2448565|AAAAAAAAFLMFFCAA|1991-11-04|1102|4792|368|1991|1|11|4|4|1991|368|4792|Monday|1991Q4|N|N|N|2448562|2448865|2448200|2448473|N|N|N|N|N| +2448566|AAAAAAAAGLMFFCAA|1991-11-05|1102|4793|368|1991|2|11|5|4|1991|368|4793|Tuesday|1991Q4|N|N|N|2448562|2448865|2448201|2448474|N|N|N|N|N| +2448567|AAAAAAAAHLMFFCAA|1991-11-06|1102|4793|368|1991|3|11|6|4|1991|368|4793|Wednesday|1991Q4|N|N|N|2448562|2448865|2448202|2448475|N|N|N|N|N| +2448568|AAAAAAAAILMFFCAA|1991-11-07|1102|4793|368|1991|4|11|7|4|1991|368|4793|Thursday|1991Q4|N|N|N|2448562|2448865|2448203|2448476|N|N|N|N|N| +2448569|AAAAAAAAJLMFFCAA|1991-11-08|1102|4793|368|1991|5|11|8|4|1991|368|4793|Friday|1991Q4|N|Y|N|2448562|2448865|2448204|2448477|N|N|N|N|N| +2448570|AAAAAAAAKLMFFCAA|1991-11-09|1102|4793|368|1991|6|11|9|4|1991|368|4793|Saturday|1991Q4|N|Y|N|2448562|2448865|2448205|2448478|N|N|N|N|N| +2448571|AAAAAAAALLMFFCAA|1991-11-10|1102|4793|368|1991|0|11|10|4|1991|368|4793|Sunday|1991Q4|N|N|N|2448562|2448865|2448206|2448479|N|N|N|N|N| +2448572|AAAAAAAAMLMFFCAA|1991-11-11|1102|4793|368|1991|1|11|11|4|1991|368|4793|Monday|1991Q4|N|N|N|2448562|2448865|2448207|2448480|N|N|N|N|N| +2448573|AAAAAAAANLMFFCAA|1991-11-12|1102|4794|368|1991|2|11|12|4|1991|368|4794|Tuesday|1991Q4|N|N|N|2448562|2448865|2448208|2448481|N|N|N|N|N| +2448574|AAAAAAAAOLMFFCAA|1991-11-13|1102|4794|368|1991|3|11|13|4|1991|368|4794|Wednesday|1991Q4|N|N|N|2448562|2448865|2448209|2448482|N|N|N|N|N| +2448575|AAAAAAAAPLMFFCAA|1991-11-14|1102|4794|368|1991|4|11|14|4|1991|368|4794|Thursday|1991Q4|N|N|N|2448562|2448865|2448210|2448483|N|N|N|N|N| +2448576|AAAAAAAAAMMFFCAA|1991-11-15|1102|4794|368|1991|5|11|15|4|1991|368|4794|Friday|1991Q4|N|Y|N|2448562|2448865|2448211|2448484|N|N|N|N|N| +2448577|AAAAAAAABMMFFCAA|1991-11-16|1102|4794|368|1991|6|11|16|4|1991|368|4794|Saturday|1991Q4|N|Y|N|2448562|2448865|2448212|2448485|N|N|N|N|N| +2448578|AAAAAAAACMMFFCAA|1991-11-17|1102|4794|368|1991|0|11|17|4|1991|368|4794|Sunday|1991Q4|N|N|N|2448562|2448865|2448213|2448486|N|N|N|N|N| +2448579|AAAAAAAADMMFFCAA|1991-11-18|1102|4794|368|1991|1|11|18|4|1991|368|4794|Monday|1991Q4|N|N|N|2448562|2448865|2448214|2448487|N|N|N|N|N| +2448580|AAAAAAAAEMMFFCAA|1991-11-19|1102|4795|368|1991|2|11|19|4|1991|368|4795|Tuesday|1991Q4|N|N|N|2448562|2448865|2448215|2448488|N|N|N|N|N| +2448581|AAAAAAAAFMMFFCAA|1991-11-20|1102|4795|368|1991|3|11|20|4|1991|368|4795|Wednesday|1991Q4|N|N|N|2448562|2448865|2448216|2448489|N|N|N|N|N| +2448582|AAAAAAAAGMMFFCAA|1991-11-21|1102|4795|368|1991|4|11|21|4|1991|368|4795|Thursday|1991Q4|N|N|N|2448562|2448865|2448217|2448490|N|N|N|N|N| +2448583|AAAAAAAAHMMFFCAA|1991-11-22|1102|4795|368|1991|5|11|22|4|1991|368|4795|Friday|1991Q4|N|Y|N|2448562|2448865|2448218|2448491|N|N|N|N|N| +2448584|AAAAAAAAIMMFFCAA|1991-11-23|1102|4795|368|1991|6|11|23|4|1991|368|4795|Saturday|1991Q4|N|Y|N|2448562|2448865|2448219|2448492|N|N|N|N|N| +2448585|AAAAAAAAJMMFFCAA|1991-11-24|1102|4795|368|1991|0|11|24|4|1991|368|4795|Sunday|1991Q4|N|N|N|2448562|2448865|2448220|2448493|N|N|N|N|N| +2448586|AAAAAAAAKMMFFCAA|1991-11-25|1102|4795|368|1991|1|11|25|4|1991|368|4795|Monday|1991Q4|N|N|N|2448562|2448865|2448221|2448494|N|N|N|N|N| +2448587|AAAAAAAALMMFFCAA|1991-11-26|1102|4796|368|1991|2|11|26|4|1991|368|4796|Tuesday|1991Q4|N|N|N|2448562|2448865|2448222|2448495|N|N|N|N|N| +2448588|AAAAAAAAMMMFFCAA|1991-11-27|1102|4796|368|1991|3|11|27|4|1991|368|4796|Wednesday|1991Q4|N|N|N|2448562|2448865|2448223|2448496|N|N|N|N|N| +2448589|AAAAAAAANMMFFCAA|1991-11-28|1102|4796|368|1991|4|11|28|4|1991|368|4796|Thursday|1991Q4|N|N|N|2448562|2448865|2448224|2448497|N|N|N|N|N| +2448590|AAAAAAAAOMMFFCAA|1991-11-29|1102|4796|368|1991|5|11|29|4|1991|368|4796|Friday|1991Q4|N|Y|N|2448562|2448865|2448225|2448498|N|N|N|N|N| +2448591|AAAAAAAAPMMFFCAA|1991-11-30|1102|4796|368|1991|6|11|30|4|1991|368|4796|Saturday|1991Q4|N|Y|N|2448562|2448865|2448226|2448499|N|N|N|N|N| +2448592|AAAAAAAAANMFFCAA|1991-12-01|1103|4796|369|1991|0|12|1|4|1991|369|4796|Sunday|1991Q4|N|N|N|2448592|2448925|2448227|2448500|N|N|N|N|N| +2448593|AAAAAAAABNMFFCAA|1991-12-02|1103|4796|369|1991|1|12|2|4|1991|369|4796|Monday|1991Q4|N|N|N|2448592|2448925|2448228|2448501|N|N|N|N|N| +2448594|AAAAAAAACNMFFCAA|1991-12-03|1103|4797|369|1991|2|12|3|4|1991|369|4797|Tuesday|1991Q4|N|N|N|2448592|2448925|2448229|2448502|N|N|N|N|N| +2448595|AAAAAAAADNMFFCAA|1991-12-04|1103|4797|369|1991|3|12|4|4|1991|369|4797|Wednesday|1991Q4|N|N|N|2448592|2448925|2448230|2448503|N|N|N|N|N| +2448596|AAAAAAAAENMFFCAA|1991-12-05|1103|4797|369|1991|4|12|5|4|1991|369|4797|Thursday|1991Q4|N|N|N|2448592|2448925|2448231|2448504|N|N|N|N|N| +2448597|AAAAAAAAFNMFFCAA|1991-12-06|1103|4797|369|1991|5|12|6|4|1991|369|4797|Friday|1991Q4|N|Y|N|2448592|2448925|2448232|2448505|N|N|N|N|N| +2448598|AAAAAAAAGNMFFCAA|1991-12-07|1103|4797|369|1991|6|12|7|4|1991|369|4797|Saturday|1991Q4|N|Y|N|2448592|2448925|2448233|2448506|N|N|N|N|N| +2448599|AAAAAAAAHNMFFCAA|1991-12-08|1103|4797|369|1991|0|12|8|4|1991|369|4797|Sunday|1991Q4|N|N|N|2448592|2448925|2448234|2448507|N|N|N|N|N| +2448600|AAAAAAAAINMFFCAA|1991-12-09|1103|4797|369|1991|1|12|9|4|1991|369|4797|Monday|1991Q4|N|N|N|2448592|2448925|2448235|2448508|N|N|N|N|N| +2448601|AAAAAAAAJNMFFCAA|1991-12-10|1103|4798|369|1991|2|12|10|4|1991|369|4798|Tuesday|1991Q4|N|N|N|2448592|2448925|2448236|2448509|N|N|N|N|N| +2448602|AAAAAAAAKNMFFCAA|1991-12-11|1103|4798|369|1991|3|12|11|4|1991|369|4798|Wednesday|1991Q4|N|N|N|2448592|2448925|2448237|2448510|N|N|N|N|N| +2448603|AAAAAAAALNMFFCAA|1991-12-12|1103|4798|369|1991|4|12|12|4|1991|369|4798|Thursday|1991Q4|N|N|N|2448592|2448925|2448238|2448511|N|N|N|N|N| +2448604|AAAAAAAAMNMFFCAA|1991-12-13|1103|4798|369|1991|5|12|13|4|1991|369|4798|Friday|1991Q4|N|Y|N|2448592|2448925|2448239|2448512|N|N|N|N|N| +2448605|AAAAAAAANNMFFCAA|1991-12-14|1103|4798|369|1991|6|12|14|4|1991|369|4798|Saturday|1991Q4|N|Y|N|2448592|2448925|2448240|2448513|N|N|N|N|N| +2448606|AAAAAAAAONMFFCAA|1991-12-15|1103|4798|369|1991|0|12|15|4|1991|369|4798|Sunday|1991Q4|N|N|N|2448592|2448925|2448241|2448514|N|N|N|N|N| +2448607|AAAAAAAAPNMFFCAA|1991-12-16|1103|4798|369|1991|1|12|16|4|1991|369|4798|Monday|1991Q4|N|N|N|2448592|2448925|2448242|2448515|N|N|N|N|N| +2448608|AAAAAAAAAOMFFCAA|1991-12-17|1103|4799|369|1991|2|12|17|4|1991|369|4799|Tuesday|1991Q4|N|N|N|2448592|2448925|2448243|2448516|N|N|N|N|N| +2448609|AAAAAAAABOMFFCAA|1991-12-18|1103|4799|369|1991|3|12|18|4|1991|369|4799|Wednesday|1991Q4|N|N|N|2448592|2448925|2448244|2448517|N|N|N|N|N| +2448610|AAAAAAAACOMFFCAA|1991-12-19|1103|4799|369|1991|4|12|19|4|1991|369|4799|Thursday|1991Q4|N|N|N|2448592|2448925|2448245|2448518|N|N|N|N|N| +2448611|AAAAAAAADOMFFCAA|1991-12-20|1103|4799|369|1991|5|12|20|4|1991|369|4799|Friday|1991Q4|N|Y|N|2448592|2448925|2448246|2448519|N|N|N|N|N| +2448612|AAAAAAAAEOMFFCAA|1991-12-21|1103|4799|369|1991|6|12|21|4|1991|369|4799|Saturday|1991Q4|N|Y|N|2448592|2448925|2448247|2448520|N|N|N|N|N| +2448613|AAAAAAAAFOMFFCAA|1991-12-22|1103|4799|369|1991|0|12|22|4|1991|369|4799|Sunday|1991Q4|N|N|N|2448592|2448925|2448248|2448521|N|N|N|N|N| +2448614|AAAAAAAAGOMFFCAA|1991-12-23|1103|4799|369|1991|1|12|23|4|1991|369|4799|Monday|1991Q4|N|N|N|2448592|2448925|2448249|2448522|N|N|N|N|N| +2448615|AAAAAAAAHOMFFCAA|1991-12-24|1103|4800|369|1991|2|12|24|4|1991|369|4800|Tuesday|1991Q4|N|N|N|2448592|2448925|2448250|2448523|N|N|N|N|N| +2448616|AAAAAAAAIOMFFCAA|1991-12-25|1103|4800|369|1991|3|12|25|4|1991|369|4800|Wednesday|1991Q4|N|N|N|2448592|2448925|2448251|2448524|N|N|N|N|N| +2448617|AAAAAAAAJOMFFCAA|1991-12-26|1103|4800|369|1991|4|12|26|4|1991|369|4800|Thursday|1991Q4|Y|N|N|2448592|2448925|2448252|2448525|N|N|N|N|N| +2448618|AAAAAAAAKOMFFCAA|1991-12-27|1103|4800|369|1991|5|12|27|4|1991|369|4800|Friday|1991Q4|N|Y|Y|2448592|2448925|2448253|2448526|N|N|N|N|N| +2448619|AAAAAAAALOMFFCAA|1991-12-28|1103|4800|369|1991|6|12|28|4|1991|369|4800|Saturday|1991Q4|N|Y|N|2448592|2448925|2448254|2448527|N|N|N|N|N| +2448620|AAAAAAAAMOMFFCAA|1991-12-29|1103|4800|369|1991|0|12|29|4|1991|369|4800|Sunday|1991Q4|N|N|N|2448592|2448925|2448255|2448528|N|N|N|N|N| +2448621|AAAAAAAANOMFFCAA|1991-12-30|1103|4800|369|1991|1|12|30|4|1991|369|4800|Monday|1991Q4|N|N|N|2448592|2448925|2448256|2448529|N|N|N|N|N| +2448622|AAAAAAAAOOMFFCAA|1991-12-31|1103|4801|369|1991|2|12|31|4|1991|369|4801|Tuesday|1991Q4|N|N|N|2448592|2448925|2448257|2448530|N|N|N|N|N| +2448623|AAAAAAAAPOMFFCAA|1992-01-01|1104|4801|369|1992|3|1|1|1|1992|369|4801|Wednesday|1992Q1|Y|N|N|2448623|2448622|2448258|2448531|N|N|N|N|N| +2448624|AAAAAAAAAPMFFCAA|1992-01-02|1104|4801|369|1992|4|1|2|1|1992|369|4801|Thursday|1992Q1|N|N|Y|2448623|2448622|2448259|2448532|N|N|N|N|N| +2448625|AAAAAAAABPMFFCAA|1992-01-03|1104|4801|369|1992|5|1|3|1|1992|369|4801|Friday|1992Q1|N|Y|N|2448623|2448622|2448260|2448533|N|N|N|N|N| +2448626|AAAAAAAACPMFFCAA|1992-01-04|1104|4801|369|1992|6|1|4|1|1992|369|4801|Saturday|1992Q1|N|Y|N|2448623|2448622|2448261|2448534|N|N|N|N|N| +2448627|AAAAAAAADPMFFCAA|1992-01-05|1104|4801|369|1992|0|1|5|1|1992|369|4801|Sunday|1992Q1|N|N|N|2448623|2448622|2448262|2448535|N|N|N|N|N| +2448628|AAAAAAAAEPMFFCAA|1992-01-06|1104|4801|369|1992|1|1|6|1|1992|369|4801|Monday|1992Q1|N|N|N|2448623|2448622|2448263|2448536|N|N|N|N|N| +2448629|AAAAAAAAFPMFFCAA|1992-01-07|1104|4802|369|1992|2|1|7|1|1992|369|4802|Tuesday|1992Q1|N|N|N|2448623|2448622|2448264|2448537|N|N|N|N|N| +2448630|AAAAAAAAGPMFFCAA|1992-01-08|1104|4802|369|1992|3|1|8|1|1992|369|4802|Wednesday|1992Q1|N|N|N|2448623|2448622|2448265|2448538|N|N|N|N|N| +2448631|AAAAAAAAHPMFFCAA|1992-01-09|1104|4802|369|1992|4|1|9|1|1992|369|4802|Thursday|1992Q1|N|N|N|2448623|2448622|2448266|2448539|N|N|N|N|N| +2448632|AAAAAAAAIPMFFCAA|1992-01-10|1104|4802|369|1992|5|1|10|1|1992|369|4802|Friday|1992Q1|N|Y|N|2448623|2448622|2448267|2448540|N|N|N|N|N| +2448633|AAAAAAAAJPMFFCAA|1992-01-11|1104|4802|369|1992|6|1|11|1|1992|369|4802|Saturday|1992Q1|N|Y|N|2448623|2448622|2448268|2448541|N|N|N|N|N| +2448634|AAAAAAAAKPMFFCAA|1992-01-12|1104|4802|369|1992|0|1|12|1|1992|369|4802|Sunday|1992Q1|N|N|N|2448623|2448622|2448269|2448542|N|N|N|N|N| +2448635|AAAAAAAALPMFFCAA|1992-01-13|1104|4802|369|1992|1|1|13|1|1992|369|4802|Monday|1992Q1|N|N|N|2448623|2448622|2448270|2448543|N|N|N|N|N| +2448636|AAAAAAAAMPMFFCAA|1992-01-14|1104|4803|369|1992|2|1|14|1|1992|369|4803|Tuesday|1992Q1|N|N|N|2448623|2448622|2448271|2448544|N|N|N|N|N| +2448637|AAAAAAAANPMFFCAA|1992-01-15|1104|4803|369|1992|3|1|15|1|1992|369|4803|Wednesday|1992Q1|N|N|N|2448623|2448622|2448272|2448545|N|N|N|N|N| +2448638|AAAAAAAAOPMFFCAA|1992-01-16|1104|4803|369|1992|4|1|16|1|1992|369|4803|Thursday|1992Q1|N|N|N|2448623|2448622|2448273|2448546|N|N|N|N|N| +2448639|AAAAAAAAPPMFFCAA|1992-01-17|1104|4803|369|1992|5|1|17|1|1992|369|4803|Friday|1992Q1|N|Y|N|2448623|2448622|2448274|2448547|N|N|N|N|N| +2448640|AAAAAAAAAANFFCAA|1992-01-18|1104|4803|369|1992|6|1|18|1|1992|369|4803|Saturday|1992Q1|N|Y|N|2448623|2448622|2448275|2448548|N|N|N|N|N| +2448641|AAAAAAAABANFFCAA|1992-01-19|1104|4803|369|1992|0|1|19|1|1992|369|4803|Sunday|1992Q1|N|N|N|2448623|2448622|2448276|2448549|N|N|N|N|N| +2448642|AAAAAAAACANFFCAA|1992-01-20|1104|4803|369|1992|1|1|20|1|1992|369|4803|Monday|1992Q1|N|N|N|2448623|2448622|2448277|2448550|N|N|N|N|N| +2448643|AAAAAAAADANFFCAA|1992-01-21|1104|4804|369|1992|2|1|21|1|1992|369|4804|Tuesday|1992Q1|N|N|N|2448623|2448622|2448278|2448551|N|N|N|N|N| +2448644|AAAAAAAAEANFFCAA|1992-01-22|1104|4804|369|1992|3|1|22|1|1992|369|4804|Wednesday|1992Q1|N|N|N|2448623|2448622|2448279|2448552|N|N|N|N|N| +2448645|AAAAAAAAFANFFCAA|1992-01-23|1104|4804|369|1992|4|1|23|1|1992|369|4804|Thursday|1992Q1|N|N|N|2448623|2448622|2448280|2448553|N|N|N|N|N| +2448646|AAAAAAAAGANFFCAA|1992-01-24|1104|4804|369|1992|5|1|24|1|1992|369|4804|Friday|1992Q1|N|Y|N|2448623|2448622|2448281|2448554|N|N|N|N|N| +2448647|AAAAAAAAHANFFCAA|1992-01-25|1104|4804|369|1992|6|1|25|1|1992|369|4804|Saturday|1992Q1|N|Y|N|2448623|2448622|2448282|2448555|N|N|N|N|N| +2448648|AAAAAAAAIANFFCAA|1992-01-26|1104|4804|369|1992|0|1|26|1|1992|369|4804|Sunday|1992Q1|N|N|N|2448623|2448622|2448283|2448556|N|N|N|N|N| +2448649|AAAAAAAAJANFFCAA|1992-01-27|1104|4804|369|1992|1|1|27|1|1992|369|4804|Monday|1992Q1|N|N|N|2448623|2448622|2448284|2448557|N|N|N|N|N| +2448650|AAAAAAAAKANFFCAA|1992-01-28|1104|4805|369|1992|2|1|28|1|1992|369|4805|Tuesday|1992Q1|N|N|N|2448623|2448622|2448285|2448558|N|N|N|N|N| +2448651|AAAAAAAALANFFCAA|1992-01-29|1104|4805|369|1992|3|1|29|1|1992|369|4805|Wednesday|1992Q1|N|N|N|2448623|2448622|2448286|2448559|N|N|N|N|N| +2448652|AAAAAAAAMANFFCAA|1992-01-30|1104|4805|369|1992|4|1|30|1|1992|369|4805|Thursday|1992Q1|N|N|N|2448623|2448622|2448287|2448560|N|N|N|N|N| +2448653|AAAAAAAANANFFCAA|1992-01-31|1104|4805|369|1992|5|1|31|1|1992|369|4805|Friday|1992Q1|N|Y|N|2448623|2448622|2448288|2448561|N|N|N|N|N| +2448654|AAAAAAAAOANFFCAA|1992-02-01|1105|4805|369|1992|6|2|1|1|1992|369|4805|Saturday|1992Q1|N|Y|N|2448654|2448684|2448289|2448562|N|N|N|N|N| +2448655|AAAAAAAAPANFFCAA|1992-02-02|1105|4805|369|1992|0|2|2|1|1992|369|4805|Sunday|1992Q1|N|N|N|2448654|2448684|2448290|2448563|N|N|N|N|N| +2448656|AAAAAAAAABNFFCAA|1992-02-03|1105|4805|369|1992|1|2|3|1|1992|369|4805|Monday|1992Q1|N|N|N|2448654|2448684|2448291|2448564|N|N|N|N|N| +2448657|AAAAAAAABBNFFCAA|1992-02-04|1105|4806|369|1992|2|2|4|1|1992|369|4806|Tuesday|1992Q1|N|N|N|2448654|2448684|2448292|2448565|N|N|N|N|N| +2448658|AAAAAAAACBNFFCAA|1992-02-05|1105|4806|369|1992|3|2|5|1|1992|369|4806|Wednesday|1992Q1|N|N|N|2448654|2448684|2448293|2448566|N|N|N|N|N| +2448659|AAAAAAAADBNFFCAA|1992-02-06|1105|4806|369|1992|4|2|6|1|1992|369|4806|Thursday|1992Q1|N|N|N|2448654|2448684|2448294|2448567|N|N|N|N|N| +2448660|AAAAAAAAEBNFFCAA|1992-02-07|1105|4806|369|1992|5|2|7|1|1992|369|4806|Friday|1992Q1|N|Y|N|2448654|2448684|2448295|2448568|N|N|N|N|N| +2448661|AAAAAAAAFBNFFCAA|1992-02-08|1105|4806|369|1992|6|2|8|1|1992|369|4806|Saturday|1992Q1|N|Y|N|2448654|2448684|2448296|2448569|N|N|N|N|N| +2448662|AAAAAAAAGBNFFCAA|1992-02-09|1105|4806|369|1992|0|2|9|1|1992|369|4806|Sunday|1992Q1|N|N|N|2448654|2448684|2448297|2448570|N|N|N|N|N| +2448663|AAAAAAAAHBNFFCAA|1992-02-10|1105|4806|369|1992|1|2|10|1|1992|369|4806|Monday|1992Q1|N|N|N|2448654|2448684|2448298|2448571|N|N|N|N|N| +2448664|AAAAAAAAIBNFFCAA|1992-02-11|1105|4807|369|1992|2|2|11|1|1992|369|4807|Tuesday|1992Q1|N|N|N|2448654|2448684|2448299|2448572|N|N|N|N|N| +2448665|AAAAAAAAJBNFFCAA|1992-02-12|1105|4807|369|1992|3|2|12|1|1992|369|4807|Wednesday|1992Q1|N|N|N|2448654|2448684|2448300|2448573|N|N|N|N|N| +2448666|AAAAAAAAKBNFFCAA|1992-02-13|1105|4807|369|1992|4|2|13|1|1992|369|4807|Thursday|1992Q1|N|N|N|2448654|2448684|2448301|2448574|N|N|N|N|N| +2448667|AAAAAAAALBNFFCAA|1992-02-14|1105|4807|369|1992|5|2|14|1|1992|369|4807|Friday|1992Q1|N|Y|N|2448654|2448684|2448302|2448575|N|N|N|N|N| +2448668|AAAAAAAAMBNFFCAA|1992-02-15|1105|4807|369|1992|6|2|15|1|1992|369|4807|Saturday|1992Q1|N|Y|N|2448654|2448684|2448303|2448576|N|N|N|N|N| +2448669|AAAAAAAANBNFFCAA|1992-02-16|1105|4807|369|1992|0|2|16|1|1992|369|4807|Sunday|1992Q1|N|N|N|2448654|2448684|2448304|2448577|N|N|N|N|N| +2448670|AAAAAAAAOBNFFCAA|1992-02-17|1105|4807|369|1992|1|2|17|1|1992|369|4807|Monday|1992Q1|N|N|N|2448654|2448684|2448305|2448578|N|N|N|N|N| +2448671|AAAAAAAAPBNFFCAA|1992-02-18|1105|4808|369|1992|2|2|18|1|1992|369|4808|Tuesday|1992Q1|N|N|N|2448654|2448684|2448306|2448579|N|N|N|N|N| +2448672|AAAAAAAAACNFFCAA|1992-02-19|1105|4808|369|1992|3|2|19|1|1992|369|4808|Wednesday|1992Q1|N|N|N|2448654|2448684|2448307|2448580|N|N|N|N|N| +2448673|AAAAAAAABCNFFCAA|1992-02-20|1105|4808|369|1992|4|2|20|1|1992|369|4808|Thursday|1992Q1|N|N|N|2448654|2448684|2448308|2448581|N|N|N|N|N| +2448674|AAAAAAAACCNFFCAA|1992-02-21|1105|4808|369|1992|5|2|21|1|1992|369|4808|Friday|1992Q1|N|Y|N|2448654|2448684|2448309|2448582|N|N|N|N|N| +2448675|AAAAAAAADCNFFCAA|1992-02-22|1105|4808|369|1992|6|2|22|1|1992|369|4808|Saturday|1992Q1|N|Y|N|2448654|2448684|2448310|2448583|N|N|N|N|N| +2448676|AAAAAAAAECNFFCAA|1992-02-23|1105|4808|369|1992|0|2|23|1|1992|369|4808|Sunday|1992Q1|N|N|N|2448654|2448684|2448311|2448584|N|N|N|N|N| +2448677|AAAAAAAAFCNFFCAA|1992-02-24|1105|4808|369|1992|1|2|24|1|1992|369|4808|Monday|1992Q1|N|N|N|2448654|2448684|2448312|2448585|N|N|N|N|N| +2448678|AAAAAAAAGCNFFCAA|1992-02-25|1105|4809|369|1992|2|2|25|1|1992|369|4809|Tuesday|1992Q1|N|N|N|2448654|2448684|2448313|2448586|N|N|N|N|N| +2448679|AAAAAAAAHCNFFCAA|1992-02-26|1105|4809|369|1992|3|2|26|1|1992|369|4809|Wednesday|1992Q1|N|N|N|2448654|2448684|2448314|2448587|N|N|N|N|N| +2448680|AAAAAAAAICNFFCAA|1992-02-27|1105|4809|369|1992|4|2|27|1|1992|369|4809|Thursday|1992Q1|N|N|N|2448654|2448684|2448315|2448588|N|N|N|N|N| +2448681|AAAAAAAAJCNFFCAA|1992-02-28|1105|4809|369|1992|5|2|28|1|1992|369|4809|Friday|1992Q1|N|Y|N|2448654|2448684|2448316|2448589|N|N|N|N|N| +2448682|AAAAAAAAKCNFFCAA|1992-02-29|1105|4809|369|1992|6|2|29|1|1992|369|4809|Saturday|1992Q1|N|Y|N|2448654|2448684|2448316|2448590|N|N|N|N|N| +2448683|AAAAAAAALCNFFCAA|1992-03-01|1106|4809|370|1992|0|3|1|1|1992|370|4809|Sunday|1992Q1|N|N|N|2448683|2448742|2448317|2448591|N|N|N|N|N| +2448684|AAAAAAAAMCNFFCAA|1992-03-02|1106|4809|370|1992|1|3|2|1|1992|370|4809|Monday|1992Q1|N|N|N|2448683|2448742|2448318|2448592|N|N|N|N|N| +2448685|AAAAAAAANCNFFCAA|1992-03-03|1106|4810|370|1992|2|3|3|1|1992|370|4810|Tuesday|1992Q1|N|N|N|2448683|2448742|2448319|2448593|N|N|N|N|N| +2448686|AAAAAAAAOCNFFCAA|1992-03-04|1106|4810|370|1992|3|3|4|1|1992|370|4810|Wednesday|1992Q1|N|N|N|2448683|2448742|2448320|2448594|N|N|N|N|N| +2448687|AAAAAAAAPCNFFCAA|1992-03-05|1106|4810|370|1992|4|3|5|1|1992|370|4810|Thursday|1992Q1|N|N|N|2448683|2448742|2448321|2448595|N|N|N|N|N| +2448688|AAAAAAAAADNFFCAA|1992-03-06|1106|4810|370|1992|5|3|6|1|1992|370|4810|Friday|1992Q1|N|Y|N|2448683|2448742|2448322|2448596|N|N|N|N|N| +2448689|AAAAAAAABDNFFCAA|1992-03-07|1106|4810|370|1992|6|3|7|1|1992|370|4810|Saturday|1992Q1|N|Y|N|2448683|2448742|2448323|2448597|N|N|N|N|N| +2448690|AAAAAAAACDNFFCAA|1992-03-08|1106|4810|370|1992|0|3|8|1|1992|370|4810|Sunday|1992Q1|N|N|N|2448683|2448742|2448324|2448598|N|N|N|N|N| +2448691|AAAAAAAADDNFFCAA|1992-03-09|1106|4810|370|1992|1|3|9|1|1992|370|4810|Monday|1992Q1|N|N|N|2448683|2448742|2448325|2448599|N|N|N|N|N| +2448692|AAAAAAAAEDNFFCAA|1992-03-10|1106|4811|370|1992|2|3|10|1|1992|370|4811|Tuesday|1992Q1|N|N|N|2448683|2448742|2448326|2448600|N|N|N|N|N| +2448693|AAAAAAAAFDNFFCAA|1992-03-11|1106|4811|370|1992|3|3|11|1|1992|370|4811|Wednesday|1992Q1|N|N|N|2448683|2448742|2448327|2448601|N|N|N|N|N| +2448694|AAAAAAAAGDNFFCAA|1992-03-12|1106|4811|370|1992|4|3|12|1|1992|370|4811|Thursday|1992Q1|N|N|N|2448683|2448742|2448328|2448602|N|N|N|N|N| +2448695|AAAAAAAAHDNFFCAA|1992-03-13|1106|4811|370|1992|5|3|13|1|1992|370|4811|Friday|1992Q1|N|Y|N|2448683|2448742|2448329|2448603|N|N|N|N|N| +2448696|AAAAAAAAIDNFFCAA|1992-03-14|1106|4811|370|1992|6|3|14|1|1992|370|4811|Saturday|1992Q1|N|Y|N|2448683|2448742|2448330|2448604|N|N|N|N|N| +2448697|AAAAAAAAJDNFFCAA|1992-03-15|1106|4811|370|1992|0|3|15|1|1992|370|4811|Sunday|1992Q1|N|N|N|2448683|2448742|2448331|2448605|N|N|N|N|N| +2448698|AAAAAAAAKDNFFCAA|1992-03-16|1106|4811|370|1992|1|3|16|1|1992|370|4811|Monday|1992Q1|N|N|N|2448683|2448742|2448332|2448606|N|N|N|N|N| +2448699|AAAAAAAALDNFFCAA|1992-03-17|1106|4812|370|1992|2|3|17|1|1992|370|4812|Tuesday|1992Q1|N|N|N|2448683|2448742|2448333|2448607|N|N|N|N|N| +2448700|AAAAAAAAMDNFFCAA|1992-03-18|1106|4812|370|1992|3|3|18|1|1992|370|4812|Wednesday|1992Q1|N|N|N|2448683|2448742|2448334|2448608|N|N|N|N|N| +2448701|AAAAAAAANDNFFCAA|1992-03-19|1106|4812|370|1992|4|3|19|1|1992|370|4812|Thursday|1992Q1|N|N|N|2448683|2448742|2448335|2448609|N|N|N|N|N| +2448702|AAAAAAAAODNFFCAA|1992-03-20|1106|4812|370|1992|5|3|20|1|1992|370|4812|Friday|1992Q1|N|Y|N|2448683|2448742|2448336|2448610|N|N|N|N|N| +2448703|AAAAAAAAPDNFFCAA|1992-03-21|1106|4812|370|1992|6|3|21|1|1992|370|4812|Saturday|1992Q1|N|Y|N|2448683|2448742|2448337|2448611|N|N|N|N|N| +2448704|AAAAAAAAAENFFCAA|1992-03-22|1106|4812|370|1992|0|3|22|1|1992|370|4812|Sunday|1992Q1|N|N|N|2448683|2448742|2448338|2448612|N|N|N|N|N| +2448705|AAAAAAAABENFFCAA|1992-03-23|1106|4812|370|1992|1|3|23|1|1992|370|4812|Monday|1992Q1|N|N|N|2448683|2448742|2448339|2448613|N|N|N|N|N| +2448706|AAAAAAAACENFFCAA|1992-03-24|1106|4813|370|1992|2|3|24|1|1992|370|4813|Tuesday|1992Q1|N|N|N|2448683|2448742|2448340|2448614|N|N|N|N|N| +2448707|AAAAAAAADENFFCAA|1992-03-25|1106|4813|370|1992|3|3|25|1|1992|370|4813|Wednesday|1992Q1|N|N|N|2448683|2448742|2448341|2448615|N|N|N|N|N| +2448708|AAAAAAAAEENFFCAA|1992-03-26|1106|4813|370|1992|4|3|26|1|1992|370|4813|Thursday|1992Q1|N|N|N|2448683|2448742|2448342|2448616|N|N|N|N|N| +2448709|AAAAAAAAFENFFCAA|1992-03-27|1106|4813|370|1992|5|3|27|1|1992|370|4813|Friday|1992Q1|N|Y|N|2448683|2448742|2448343|2448617|N|N|N|N|N| +2448710|AAAAAAAAGENFFCAA|1992-03-28|1106|4813|370|1992|6|3|28|1|1992|370|4813|Saturday|1992Q1|N|Y|N|2448683|2448742|2448344|2448618|N|N|N|N|N| +2448711|AAAAAAAAHENFFCAA|1992-03-29|1106|4813|370|1992|0|3|29|1|1992|370|4813|Sunday|1992Q1|N|N|N|2448683|2448742|2448345|2448619|N|N|N|N|N| +2448712|AAAAAAAAIENFFCAA|1992-03-30|1106|4813|370|1992|1|3|30|1|1992|370|4813|Monday|1992Q1|N|N|N|2448683|2448742|2448346|2448620|N|N|N|N|N| +2448713|AAAAAAAAJENFFCAA|1992-03-31|1106|4814|370|1992|2|3|31|1|1992|370|4814|Tuesday|1992Q1|N|N|N|2448683|2448742|2448347|2448621|N|N|N|N|N| +2448714|AAAAAAAAKENFFCAA|1992-04-01|1107|4814|370|1992|3|4|1|2|1992|370|4814|Wednesday|1992Q2|N|N|N|2448714|2448804|2448348|2448623|N|N|N|N|N| +2448715|AAAAAAAALENFFCAA|1992-04-02|1107|4814|370|1992|4|4|2|2|1992|370|4814|Thursday|1992Q2|N|N|N|2448714|2448804|2448349|2448624|N|N|N|N|N| +2448716|AAAAAAAAMENFFCAA|1992-04-03|1107|4814|370|1992|5|4|3|2|1992|370|4814|Friday|1992Q2|N|Y|N|2448714|2448804|2448350|2448625|N|N|N|N|N| +2448717|AAAAAAAANENFFCAA|1992-04-04|1107|4814|370|1992|6|4|4|2|1992|370|4814|Saturday|1992Q2|N|Y|N|2448714|2448804|2448351|2448626|N|N|N|N|N| +2448718|AAAAAAAAOENFFCAA|1992-04-05|1107|4814|370|1992|0|4|5|2|1992|370|4814|Sunday|1992Q2|N|N|N|2448714|2448804|2448352|2448627|N|N|N|N|N| +2448719|AAAAAAAAPENFFCAA|1992-04-06|1107|4814|370|1992|1|4|6|2|1992|370|4814|Monday|1992Q2|N|N|N|2448714|2448804|2448353|2448628|N|N|N|N|N| +2448720|AAAAAAAAAFNFFCAA|1992-04-07|1107|4815|370|1992|2|4|7|2|1992|370|4815|Tuesday|1992Q2|N|N|N|2448714|2448804|2448354|2448629|N|N|N|N|N| +2448721|AAAAAAAABFNFFCAA|1992-04-08|1107|4815|370|1992|3|4|8|2|1992|370|4815|Wednesday|1992Q2|N|N|N|2448714|2448804|2448355|2448630|N|N|N|N|N| +2448722|AAAAAAAACFNFFCAA|1992-04-09|1107|4815|370|1992|4|4|9|2|1992|370|4815|Thursday|1992Q2|N|N|N|2448714|2448804|2448356|2448631|N|N|N|N|N| +2448723|AAAAAAAADFNFFCAA|1992-04-10|1107|4815|370|1992|5|4|10|2|1992|370|4815|Friday|1992Q2|N|Y|N|2448714|2448804|2448357|2448632|N|N|N|N|N| +2448724|AAAAAAAAEFNFFCAA|1992-04-11|1107|4815|370|1992|6|4|11|2|1992|370|4815|Saturday|1992Q2|N|Y|N|2448714|2448804|2448358|2448633|N|N|N|N|N| +2448725|AAAAAAAAFFNFFCAA|1992-04-12|1107|4815|370|1992|0|4|12|2|1992|370|4815|Sunday|1992Q2|N|N|N|2448714|2448804|2448359|2448634|N|N|N|N|N| +2448726|AAAAAAAAGFNFFCAA|1992-04-13|1107|4815|370|1992|1|4|13|2|1992|370|4815|Monday|1992Q2|N|N|N|2448714|2448804|2448360|2448635|N|N|N|N|N| +2448727|AAAAAAAAHFNFFCAA|1992-04-14|1107|4816|370|1992|2|4|14|2|1992|370|4816|Tuesday|1992Q2|N|N|N|2448714|2448804|2448361|2448636|N|N|N|N|N| +2448728|AAAAAAAAIFNFFCAA|1992-04-15|1107|4816|370|1992|3|4|15|2|1992|370|4816|Wednesday|1992Q2|N|N|N|2448714|2448804|2448362|2448637|N|N|N|N|N| +2448729|AAAAAAAAJFNFFCAA|1992-04-16|1107|4816|370|1992|4|4|16|2|1992|370|4816|Thursday|1992Q2|N|N|N|2448714|2448804|2448363|2448638|N|N|N|N|N| +2448730|AAAAAAAAKFNFFCAA|1992-04-17|1107|4816|370|1992|5|4|17|2|1992|370|4816|Friday|1992Q2|N|Y|N|2448714|2448804|2448364|2448639|N|N|N|N|N| +2448731|AAAAAAAALFNFFCAA|1992-04-18|1107|4816|370|1992|6|4|18|2|1992|370|4816|Saturday|1992Q2|N|Y|N|2448714|2448804|2448365|2448640|N|N|N|N|N| +2448732|AAAAAAAAMFNFFCAA|1992-04-19|1107|4816|370|1992|0|4|19|2|1992|370|4816|Sunday|1992Q2|N|N|N|2448714|2448804|2448366|2448641|N|N|N|N|N| +2448733|AAAAAAAANFNFFCAA|1992-04-20|1107|4816|370|1992|1|4|20|2|1992|370|4816|Monday|1992Q2|N|N|N|2448714|2448804|2448367|2448642|N|N|N|N|N| +2448734|AAAAAAAAOFNFFCAA|1992-04-21|1107|4817|370|1992|2|4|21|2|1992|370|4817|Tuesday|1992Q2|N|N|N|2448714|2448804|2448368|2448643|N|N|N|N|N| +2448735|AAAAAAAAPFNFFCAA|1992-04-22|1107|4817|370|1992|3|4|22|2|1992|370|4817|Wednesday|1992Q2|N|N|N|2448714|2448804|2448369|2448644|N|N|N|N|N| +2448736|AAAAAAAAAGNFFCAA|1992-04-23|1107|4817|370|1992|4|4|23|2|1992|370|4817|Thursday|1992Q2|N|N|N|2448714|2448804|2448370|2448645|N|N|N|N|N| +2448737|AAAAAAAABGNFFCAA|1992-04-24|1107|4817|370|1992|5|4|24|2|1992|370|4817|Friday|1992Q2|N|Y|N|2448714|2448804|2448371|2448646|N|N|N|N|N| +2448738|AAAAAAAACGNFFCAA|1992-04-25|1107|4817|370|1992|6|4|25|2|1992|370|4817|Saturday|1992Q2|N|Y|N|2448714|2448804|2448372|2448647|N|N|N|N|N| +2448739|AAAAAAAADGNFFCAA|1992-04-26|1107|4817|370|1992|0|4|26|2|1992|370|4817|Sunday|1992Q2|N|N|N|2448714|2448804|2448373|2448648|N|N|N|N|N| +2448740|AAAAAAAAEGNFFCAA|1992-04-27|1107|4817|370|1992|1|4|27|2|1992|370|4817|Monday|1992Q2|N|N|N|2448714|2448804|2448374|2448649|N|N|N|N|N| +2448741|AAAAAAAAFGNFFCAA|1992-04-28|1107|4818|370|1992|2|4|28|2|1992|370|4818|Tuesday|1992Q2|N|N|N|2448714|2448804|2448375|2448650|N|N|N|N|N| +2448742|AAAAAAAAGGNFFCAA|1992-04-29|1107|4818|370|1992|3|4|29|2|1992|370|4818|Wednesday|1992Q2|N|N|N|2448714|2448804|2448376|2448651|N|N|N|N|N| +2448743|AAAAAAAAHGNFFCAA|1992-04-30|1107|4818|370|1992|4|4|30|2|1992|370|4818|Thursday|1992Q2|N|N|N|2448714|2448804|2448377|2448652|N|N|N|N|N| +2448744|AAAAAAAAIGNFFCAA|1992-05-01|1108|4818|370|1992|5|5|1|2|1992|370|4818|Friday|1992Q2|N|Y|N|2448744|2448864|2448378|2448653|N|N|N|N|N| +2448745|AAAAAAAAJGNFFCAA|1992-05-02|1108|4818|370|1992|6|5|2|2|1992|370|4818|Saturday|1992Q2|N|Y|N|2448744|2448864|2448379|2448654|N|N|N|N|N| +2448746|AAAAAAAAKGNFFCAA|1992-05-03|1108|4818|370|1992|0|5|3|2|1992|370|4818|Sunday|1992Q2|N|N|N|2448744|2448864|2448380|2448655|N|N|N|N|N| +2448747|AAAAAAAALGNFFCAA|1992-05-04|1108|4818|370|1992|1|5|4|2|1992|370|4818|Monday|1992Q2|N|N|N|2448744|2448864|2448381|2448656|N|N|N|N|N| +2448748|AAAAAAAAMGNFFCAA|1992-05-05|1108|4819|370|1992|2|5|5|2|1992|370|4819|Tuesday|1992Q2|N|N|N|2448744|2448864|2448382|2448657|N|N|N|N|N| +2448749|AAAAAAAANGNFFCAA|1992-05-06|1108|4819|370|1992|3|5|6|2|1992|370|4819|Wednesday|1992Q2|N|N|N|2448744|2448864|2448383|2448658|N|N|N|N|N| +2448750|AAAAAAAAOGNFFCAA|1992-05-07|1108|4819|370|1992|4|5|7|2|1992|370|4819|Thursday|1992Q2|N|N|N|2448744|2448864|2448384|2448659|N|N|N|N|N| +2448751|AAAAAAAAPGNFFCAA|1992-05-08|1108|4819|370|1992|5|5|8|2|1992|370|4819|Friday|1992Q2|N|Y|N|2448744|2448864|2448385|2448660|N|N|N|N|N| +2448752|AAAAAAAAAHNFFCAA|1992-05-09|1108|4819|370|1992|6|5|9|2|1992|370|4819|Saturday|1992Q2|N|Y|N|2448744|2448864|2448386|2448661|N|N|N|N|N| +2448753|AAAAAAAABHNFFCAA|1992-05-10|1108|4819|370|1992|0|5|10|2|1992|370|4819|Sunday|1992Q2|N|N|N|2448744|2448864|2448387|2448662|N|N|N|N|N| +2448754|AAAAAAAACHNFFCAA|1992-05-11|1108|4819|370|1992|1|5|11|2|1992|370|4819|Monday|1992Q2|N|N|N|2448744|2448864|2448388|2448663|N|N|N|N|N| +2448755|AAAAAAAADHNFFCAA|1992-05-12|1108|4820|370|1992|2|5|12|2|1992|370|4820|Tuesday|1992Q2|N|N|N|2448744|2448864|2448389|2448664|N|N|N|N|N| +2448756|AAAAAAAAEHNFFCAA|1992-05-13|1108|4820|370|1992|3|5|13|2|1992|370|4820|Wednesday|1992Q2|N|N|N|2448744|2448864|2448390|2448665|N|N|N|N|N| +2448757|AAAAAAAAFHNFFCAA|1992-05-14|1108|4820|370|1992|4|5|14|2|1992|370|4820|Thursday|1992Q2|N|N|N|2448744|2448864|2448391|2448666|N|N|N|N|N| +2448758|AAAAAAAAGHNFFCAA|1992-05-15|1108|4820|370|1992|5|5|15|2|1992|370|4820|Friday|1992Q2|N|Y|N|2448744|2448864|2448392|2448667|N|N|N|N|N| +2448759|AAAAAAAAHHNFFCAA|1992-05-16|1108|4820|370|1992|6|5|16|2|1992|370|4820|Saturday|1992Q2|N|Y|N|2448744|2448864|2448393|2448668|N|N|N|N|N| +2448760|AAAAAAAAIHNFFCAA|1992-05-17|1108|4820|370|1992|0|5|17|2|1992|370|4820|Sunday|1992Q2|N|N|N|2448744|2448864|2448394|2448669|N|N|N|N|N| +2448761|AAAAAAAAJHNFFCAA|1992-05-18|1108|4820|370|1992|1|5|18|2|1992|370|4820|Monday|1992Q2|N|N|N|2448744|2448864|2448395|2448670|N|N|N|N|N| +2448762|AAAAAAAAKHNFFCAA|1992-05-19|1108|4821|370|1992|2|5|19|2|1992|370|4821|Tuesday|1992Q2|N|N|N|2448744|2448864|2448396|2448671|N|N|N|N|N| +2448763|AAAAAAAALHNFFCAA|1992-05-20|1108|4821|370|1992|3|5|20|2|1992|370|4821|Wednesday|1992Q2|N|N|N|2448744|2448864|2448397|2448672|N|N|N|N|N| +2448764|AAAAAAAAMHNFFCAA|1992-05-21|1108|4821|370|1992|4|5|21|2|1992|370|4821|Thursday|1992Q2|N|N|N|2448744|2448864|2448398|2448673|N|N|N|N|N| +2448765|AAAAAAAANHNFFCAA|1992-05-22|1108|4821|370|1992|5|5|22|2|1992|370|4821|Friday|1992Q2|N|Y|N|2448744|2448864|2448399|2448674|N|N|N|N|N| +2448766|AAAAAAAAOHNFFCAA|1992-05-23|1108|4821|370|1992|6|5|23|2|1992|370|4821|Saturday|1992Q2|N|Y|N|2448744|2448864|2448400|2448675|N|N|N|N|N| +2448767|AAAAAAAAPHNFFCAA|1992-05-24|1108|4821|370|1992|0|5|24|2|1992|370|4821|Sunday|1992Q2|N|N|N|2448744|2448864|2448401|2448676|N|N|N|N|N| +2448768|AAAAAAAAAINFFCAA|1992-05-25|1108|4821|370|1992|1|5|25|2|1992|370|4821|Monday|1992Q2|N|N|N|2448744|2448864|2448402|2448677|N|N|N|N|N| +2448769|AAAAAAAABINFFCAA|1992-05-26|1108|4822|370|1992|2|5|26|2|1992|370|4822|Tuesday|1992Q2|N|N|N|2448744|2448864|2448403|2448678|N|N|N|N|N| +2448770|AAAAAAAACINFFCAA|1992-05-27|1108|4822|370|1992|3|5|27|2|1992|370|4822|Wednesday|1992Q2|N|N|N|2448744|2448864|2448404|2448679|N|N|N|N|N| +2448771|AAAAAAAADINFFCAA|1992-05-28|1108|4822|370|1992|4|5|28|2|1992|370|4822|Thursday|1992Q2|N|N|N|2448744|2448864|2448405|2448680|N|N|N|N|N| +2448772|AAAAAAAAEINFFCAA|1992-05-29|1108|4822|370|1992|5|5|29|2|1992|370|4822|Friday|1992Q2|N|Y|N|2448744|2448864|2448406|2448681|N|N|N|N|N| +2448773|AAAAAAAAFINFFCAA|1992-05-30|1108|4822|370|1992|6|5|30|2|1992|370|4822|Saturday|1992Q2|N|Y|N|2448744|2448864|2448407|2448682|N|N|N|N|N| +2448774|AAAAAAAAGINFFCAA|1992-05-31|1108|4822|370|1992|0|5|31|2|1992|370|4822|Sunday|1992Q2|N|N|N|2448744|2448864|2448408|2448683|N|N|N|N|N| +2448775|AAAAAAAAHINFFCAA|1992-06-01|1109|4822|371|1992|1|6|1|2|1992|371|4822|Monday|1992Q2|N|N|N|2448775|2448926|2448409|2448684|N|N|N|N|N| +2448776|AAAAAAAAIINFFCAA|1992-06-02|1109|4823|371|1992|2|6|2|2|1992|371|4823|Tuesday|1992Q2|N|N|N|2448775|2448926|2448410|2448685|N|N|N|N|N| +2448777|AAAAAAAAJINFFCAA|1992-06-03|1109|4823|371|1992|3|6|3|2|1992|371|4823|Wednesday|1992Q2|N|N|N|2448775|2448926|2448411|2448686|N|N|N|N|N| +2448778|AAAAAAAAKINFFCAA|1992-06-04|1109|4823|371|1992|4|6|4|2|1992|371|4823|Thursday|1992Q2|N|N|N|2448775|2448926|2448412|2448687|N|N|N|N|N| +2448779|AAAAAAAALINFFCAA|1992-06-05|1109|4823|371|1992|5|6|5|2|1992|371|4823|Friday|1992Q2|N|Y|N|2448775|2448926|2448413|2448688|N|N|N|N|N| +2448780|AAAAAAAAMINFFCAA|1992-06-06|1109|4823|371|1992|6|6|6|2|1992|371|4823|Saturday|1992Q2|N|Y|N|2448775|2448926|2448414|2448689|N|N|N|N|N| +2448781|AAAAAAAANINFFCAA|1992-06-07|1109|4823|371|1992|0|6|7|2|1992|371|4823|Sunday|1992Q2|N|N|N|2448775|2448926|2448415|2448690|N|N|N|N|N| +2448782|AAAAAAAAOINFFCAA|1992-06-08|1109|4823|371|1992|1|6|8|2|1992|371|4823|Monday|1992Q2|N|N|N|2448775|2448926|2448416|2448691|N|N|N|N|N| +2448783|AAAAAAAAPINFFCAA|1992-06-09|1109|4824|371|1992|2|6|9|2|1992|371|4824|Tuesday|1992Q2|N|N|N|2448775|2448926|2448417|2448692|N|N|N|N|N| +2448784|AAAAAAAAAJNFFCAA|1992-06-10|1109|4824|371|1992|3|6|10|2|1992|371|4824|Wednesday|1992Q2|N|N|N|2448775|2448926|2448418|2448693|N|N|N|N|N| +2448785|AAAAAAAABJNFFCAA|1992-06-11|1109|4824|371|1992|4|6|11|2|1992|371|4824|Thursday|1992Q2|N|N|N|2448775|2448926|2448419|2448694|N|N|N|N|N| +2448786|AAAAAAAACJNFFCAA|1992-06-12|1109|4824|371|1992|5|6|12|2|1992|371|4824|Friday|1992Q2|N|Y|N|2448775|2448926|2448420|2448695|N|N|N|N|N| +2448787|AAAAAAAADJNFFCAA|1992-06-13|1109|4824|371|1992|6|6|13|2|1992|371|4824|Saturday|1992Q2|N|Y|N|2448775|2448926|2448421|2448696|N|N|N|N|N| +2448788|AAAAAAAAEJNFFCAA|1992-06-14|1109|4824|371|1992|0|6|14|2|1992|371|4824|Sunday|1992Q2|N|N|N|2448775|2448926|2448422|2448697|N|N|N|N|N| +2448789|AAAAAAAAFJNFFCAA|1992-06-15|1109|4824|371|1992|1|6|15|2|1992|371|4824|Monday|1992Q2|N|N|N|2448775|2448926|2448423|2448698|N|N|N|N|N| +2448790|AAAAAAAAGJNFFCAA|1992-06-16|1109|4825|371|1992|2|6|16|2|1992|371|4825|Tuesday|1992Q2|N|N|N|2448775|2448926|2448424|2448699|N|N|N|N|N| +2448791|AAAAAAAAHJNFFCAA|1992-06-17|1109|4825|371|1992|3|6|17|2|1992|371|4825|Wednesday|1992Q2|N|N|N|2448775|2448926|2448425|2448700|N|N|N|N|N| +2448792|AAAAAAAAIJNFFCAA|1992-06-18|1109|4825|371|1992|4|6|18|2|1992|371|4825|Thursday|1992Q2|N|N|N|2448775|2448926|2448426|2448701|N|N|N|N|N| +2448793|AAAAAAAAJJNFFCAA|1992-06-19|1109|4825|371|1992|5|6|19|2|1992|371|4825|Friday|1992Q2|N|Y|N|2448775|2448926|2448427|2448702|N|N|N|N|N| +2448794|AAAAAAAAKJNFFCAA|1992-06-20|1109|4825|371|1992|6|6|20|2|1992|371|4825|Saturday|1992Q2|N|Y|N|2448775|2448926|2448428|2448703|N|N|N|N|N| +2448795|AAAAAAAALJNFFCAA|1992-06-21|1109|4825|371|1992|0|6|21|2|1992|371|4825|Sunday|1992Q2|N|N|N|2448775|2448926|2448429|2448704|N|N|N|N|N| +2448796|AAAAAAAAMJNFFCAA|1992-06-22|1109|4825|371|1992|1|6|22|2|1992|371|4825|Monday|1992Q2|N|N|N|2448775|2448926|2448430|2448705|N|N|N|N|N| +2448797|AAAAAAAANJNFFCAA|1992-06-23|1109|4826|371|1992|2|6|23|2|1992|371|4826|Tuesday|1992Q2|N|N|N|2448775|2448926|2448431|2448706|N|N|N|N|N| +2448798|AAAAAAAAOJNFFCAA|1992-06-24|1109|4826|371|1992|3|6|24|2|1992|371|4826|Wednesday|1992Q2|N|N|N|2448775|2448926|2448432|2448707|N|N|N|N|N| +2448799|AAAAAAAAPJNFFCAA|1992-06-25|1109|4826|371|1992|4|6|25|2|1992|371|4826|Thursday|1992Q2|N|N|N|2448775|2448926|2448433|2448708|N|N|N|N|N| +2448800|AAAAAAAAAKNFFCAA|1992-06-26|1109|4826|371|1992|5|6|26|2|1992|371|4826|Friday|1992Q2|N|Y|N|2448775|2448926|2448434|2448709|N|N|N|N|N| +2448801|AAAAAAAABKNFFCAA|1992-06-27|1109|4826|371|1992|6|6|27|2|1992|371|4826|Saturday|1992Q2|N|Y|N|2448775|2448926|2448435|2448710|N|N|N|N|N| +2448802|AAAAAAAACKNFFCAA|1992-06-28|1109|4826|371|1992|0|6|28|2|1992|371|4826|Sunday|1992Q2|N|N|N|2448775|2448926|2448436|2448711|N|N|N|N|N| +2448803|AAAAAAAADKNFFCAA|1992-06-29|1109|4826|371|1992|1|6|29|2|1992|371|4826|Monday|1992Q2|N|N|N|2448775|2448926|2448437|2448712|N|N|N|N|N| +2448804|AAAAAAAAEKNFFCAA|1992-06-30|1109|4827|371|1992|2|6|30|2|1992|371|4827|Tuesday|1992Q2|N|N|N|2448775|2448926|2448438|2448713|N|N|N|N|N| +2448805|AAAAAAAAFKNFFCAA|1992-07-01|1110|4827|371|1992|3|7|1|3|1992|371|4827|Wednesday|1992Q3|N|N|N|2448805|2448986|2448439|2448714|N|N|N|N|N| +2448806|AAAAAAAAGKNFFCAA|1992-07-02|1110|4827|371|1992|4|7|2|3|1992|371|4827|Thursday|1992Q3|N|N|N|2448805|2448986|2448440|2448715|N|N|N|N|N| +2448807|AAAAAAAAHKNFFCAA|1992-07-03|1110|4827|371|1992|5|7|3|3|1992|371|4827|Friday|1992Q3|N|Y|N|2448805|2448986|2448441|2448716|N|N|N|N|N| +2448808|AAAAAAAAIKNFFCAA|1992-07-04|1110|4827|371|1992|6|7|4|3|1992|371|4827|Saturday|1992Q3|Y|Y|N|2448805|2448986|2448442|2448717|N|N|N|N|N| +2448809|AAAAAAAAJKNFFCAA|1992-07-05|1110|4827|371|1992|0|7|5|3|1992|371|4827|Sunday|1992Q3|N|N|Y|2448805|2448986|2448443|2448718|N|N|N|N|N| +2448810|AAAAAAAAKKNFFCAA|1992-07-06|1110|4827|371|1992|1|7|6|3|1992|371|4827|Monday|1992Q3|N|N|N|2448805|2448986|2448444|2448719|N|N|N|N|N| +2448811|AAAAAAAALKNFFCAA|1992-07-07|1110|4828|371|1992|2|7|7|3|1992|371|4828|Tuesday|1992Q3|N|N|N|2448805|2448986|2448445|2448720|N|N|N|N|N| +2448812|AAAAAAAAMKNFFCAA|1992-07-08|1110|4828|371|1992|3|7|8|3|1992|371|4828|Wednesday|1992Q3|N|N|N|2448805|2448986|2448446|2448721|N|N|N|N|N| +2448813|AAAAAAAANKNFFCAA|1992-07-09|1110|4828|371|1992|4|7|9|3|1992|371|4828|Thursday|1992Q3|N|N|N|2448805|2448986|2448447|2448722|N|N|N|N|N| +2448814|AAAAAAAAOKNFFCAA|1992-07-10|1110|4828|371|1992|5|7|10|3|1992|371|4828|Friday|1992Q3|N|Y|N|2448805|2448986|2448448|2448723|N|N|N|N|N| +2448815|AAAAAAAAPKNFFCAA|1992-07-11|1110|4828|371|1992|6|7|11|3|1992|371|4828|Saturday|1992Q3|N|Y|N|2448805|2448986|2448449|2448724|N|N|N|N|N| +2448816|AAAAAAAAALNFFCAA|1992-07-12|1110|4828|371|1992|0|7|12|3|1992|371|4828|Sunday|1992Q3|N|N|N|2448805|2448986|2448450|2448725|N|N|N|N|N| +2448817|AAAAAAAABLNFFCAA|1992-07-13|1110|4828|371|1992|1|7|13|3|1992|371|4828|Monday|1992Q3|N|N|N|2448805|2448986|2448451|2448726|N|N|N|N|N| +2448818|AAAAAAAACLNFFCAA|1992-07-14|1110|4829|371|1992|2|7|14|3|1992|371|4829|Tuesday|1992Q3|N|N|N|2448805|2448986|2448452|2448727|N|N|N|N|N| +2448819|AAAAAAAADLNFFCAA|1992-07-15|1110|4829|371|1992|3|7|15|3|1992|371|4829|Wednesday|1992Q3|N|N|N|2448805|2448986|2448453|2448728|N|N|N|N|N| +2448820|AAAAAAAAELNFFCAA|1992-07-16|1110|4829|371|1992|4|7|16|3|1992|371|4829|Thursday|1992Q3|N|N|N|2448805|2448986|2448454|2448729|N|N|N|N|N| +2448821|AAAAAAAAFLNFFCAA|1992-07-17|1110|4829|371|1992|5|7|17|3|1992|371|4829|Friday|1992Q3|N|Y|N|2448805|2448986|2448455|2448730|N|N|N|N|N| +2448822|AAAAAAAAGLNFFCAA|1992-07-18|1110|4829|371|1992|6|7|18|3|1992|371|4829|Saturday|1992Q3|N|Y|N|2448805|2448986|2448456|2448731|N|N|N|N|N| +2448823|AAAAAAAAHLNFFCAA|1992-07-19|1110|4829|371|1992|0|7|19|3|1992|371|4829|Sunday|1992Q3|N|N|N|2448805|2448986|2448457|2448732|N|N|N|N|N| +2448824|AAAAAAAAILNFFCAA|1992-07-20|1110|4829|371|1992|1|7|20|3|1992|371|4829|Monday|1992Q3|N|N|N|2448805|2448986|2448458|2448733|N|N|N|N|N| +2448825|AAAAAAAAJLNFFCAA|1992-07-21|1110|4830|371|1992|2|7|21|3|1992|371|4830|Tuesday|1992Q3|N|N|N|2448805|2448986|2448459|2448734|N|N|N|N|N| +2448826|AAAAAAAAKLNFFCAA|1992-07-22|1110|4830|371|1992|3|7|22|3|1992|371|4830|Wednesday|1992Q3|N|N|N|2448805|2448986|2448460|2448735|N|N|N|N|N| +2448827|AAAAAAAALLNFFCAA|1992-07-23|1110|4830|371|1992|4|7|23|3|1992|371|4830|Thursday|1992Q3|N|N|N|2448805|2448986|2448461|2448736|N|N|N|N|N| +2448828|AAAAAAAAMLNFFCAA|1992-07-24|1110|4830|371|1992|5|7|24|3|1992|371|4830|Friday|1992Q3|N|Y|N|2448805|2448986|2448462|2448737|N|N|N|N|N| +2448829|AAAAAAAANLNFFCAA|1992-07-25|1110|4830|371|1992|6|7|25|3|1992|371|4830|Saturday|1992Q3|N|Y|N|2448805|2448986|2448463|2448738|N|N|N|N|N| +2448830|AAAAAAAAOLNFFCAA|1992-07-26|1110|4830|371|1992|0|7|26|3|1992|371|4830|Sunday|1992Q3|N|N|N|2448805|2448986|2448464|2448739|N|N|N|N|N| +2448831|AAAAAAAAPLNFFCAA|1992-07-27|1110|4830|371|1992|1|7|27|3|1992|371|4830|Monday|1992Q3|N|N|N|2448805|2448986|2448465|2448740|N|N|N|N|N| +2448832|AAAAAAAAAMNFFCAA|1992-07-28|1110|4831|371|1992|2|7|28|3|1992|371|4831|Tuesday|1992Q3|N|N|N|2448805|2448986|2448466|2448741|N|N|N|N|N| +2448833|AAAAAAAABMNFFCAA|1992-07-29|1110|4831|371|1992|3|7|29|3|1992|371|4831|Wednesday|1992Q3|N|N|N|2448805|2448986|2448467|2448742|N|N|N|N|N| +2448834|AAAAAAAACMNFFCAA|1992-07-30|1110|4831|371|1992|4|7|30|3|1992|371|4831|Thursday|1992Q3|N|N|N|2448805|2448986|2448468|2448743|N|N|N|N|N| +2448835|AAAAAAAADMNFFCAA|1992-07-31|1110|4831|371|1992|5|7|31|3|1992|371|4831|Friday|1992Q3|N|Y|N|2448805|2448986|2448469|2448744|N|N|N|N|N| +2448836|AAAAAAAAEMNFFCAA|1992-08-01|1111|4831|371|1992|6|8|1|3|1992|371|4831|Saturday|1992Q3|N|Y|N|2448836|2449048|2448470|2448745|N|N|N|N|N| +2448837|AAAAAAAAFMNFFCAA|1992-08-02|1111|4831|371|1992|0|8|2|3|1992|371|4831|Sunday|1992Q3|N|N|N|2448836|2449048|2448471|2448746|N|N|N|N|N| +2448838|AAAAAAAAGMNFFCAA|1992-08-03|1111|4831|371|1992|1|8|3|3|1992|371|4831|Monday|1992Q3|N|N|N|2448836|2449048|2448472|2448747|N|N|N|N|N| +2448839|AAAAAAAAHMNFFCAA|1992-08-04|1111|4832|371|1992|2|8|4|3|1992|371|4832|Tuesday|1992Q3|N|N|N|2448836|2449048|2448473|2448748|N|N|N|N|N| +2448840|AAAAAAAAIMNFFCAA|1992-08-05|1111|4832|371|1992|3|8|5|3|1992|371|4832|Wednesday|1992Q3|N|N|N|2448836|2449048|2448474|2448749|N|N|N|N|N| +2448841|AAAAAAAAJMNFFCAA|1992-08-06|1111|4832|371|1992|4|8|6|3|1992|371|4832|Thursday|1992Q3|N|N|N|2448836|2449048|2448475|2448750|N|N|N|N|N| +2448842|AAAAAAAAKMNFFCAA|1992-08-07|1111|4832|371|1992|5|8|7|3|1992|371|4832|Friday|1992Q3|N|Y|N|2448836|2449048|2448476|2448751|N|N|N|N|N| +2448843|AAAAAAAALMNFFCAA|1992-08-08|1111|4832|371|1992|6|8|8|3|1992|371|4832|Saturday|1992Q3|N|Y|N|2448836|2449048|2448477|2448752|N|N|N|N|N| +2448844|AAAAAAAAMMNFFCAA|1992-08-09|1111|4832|371|1992|0|8|9|3|1992|371|4832|Sunday|1992Q3|N|N|N|2448836|2449048|2448478|2448753|N|N|N|N|N| +2448845|AAAAAAAANMNFFCAA|1992-08-10|1111|4832|371|1992|1|8|10|3|1992|371|4832|Monday|1992Q3|N|N|N|2448836|2449048|2448479|2448754|N|N|N|N|N| +2448846|AAAAAAAAOMNFFCAA|1992-08-11|1111|4833|371|1992|2|8|11|3|1992|371|4833|Tuesday|1992Q3|N|N|N|2448836|2449048|2448480|2448755|N|N|N|N|N| +2448847|AAAAAAAAPMNFFCAA|1992-08-12|1111|4833|371|1992|3|8|12|3|1992|371|4833|Wednesday|1992Q3|N|N|N|2448836|2449048|2448481|2448756|N|N|N|N|N| +2448848|AAAAAAAAANNFFCAA|1992-08-13|1111|4833|371|1992|4|8|13|3|1992|371|4833|Thursday|1992Q3|N|N|N|2448836|2449048|2448482|2448757|N|N|N|N|N| +2448849|AAAAAAAABNNFFCAA|1992-08-14|1111|4833|371|1992|5|8|14|3|1992|371|4833|Friday|1992Q3|N|Y|N|2448836|2449048|2448483|2448758|N|N|N|N|N| +2448850|AAAAAAAACNNFFCAA|1992-08-15|1111|4833|371|1992|6|8|15|3|1992|371|4833|Saturday|1992Q3|N|Y|N|2448836|2449048|2448484|2448759|N|N|N|N|N| +2448851|AAAAAAAADNNFFCAA|1992-08-16|1111|4833|371|1992|0|8|16|3|1992|371|4833|Sunday|1992Q3|N|N|N|2448836|2449048|2448485|2448760|N|N|N|N|N| +2448852|AAAAAAAAENNFFCAA|1992-08-17|1111|4833|371|1992|1|8|17|3|1992|371|4833|Monday|1992Q3|N|N|N|2448836|2449048|2448486|2448761|N|N|N|N|N| +2448853|AAAAAAAAFNNFFCAA|1992-08-18|1111|4834|371|1992|2|8|18|3|1992|371|4834|Tuesday|1992Q3|N|N|N|2448836|2449048|2448487|2448762|N|N|N|N|N| +2448854|AAAAAAAAGNNFFCAA|1992-08-19|1111|4834|371|1992|3|8|19|3|1992|371|4834|Wednesday|1992Q3|N|N|N|2448836|2449048|2448488|2448763|N|N|N|N|N| +2448855|AAAAAAAAHNNFFCAA|1992-08-20|1111|4834|371|1992|4|8|20|3|1992|371|4834|Thursday|1992Q3|N|N|N|2448836|2449048|2448489|2448764|N|N|N|N|N| +2448856|AAAAAAAAINNFFCAA|1992-08-21|1111|4834|371|1992|5|8|21|3|1992|371|4834|Friday|1992Q3|N|Y|N|2448836|2449048|2448490|2448765|N|N|N|N|N| +2448857|AAAAAAAAJNNFFCAA|1992-08-22|1111|4834|371|1992|6|8|22|3|1992|371|4834|Saturday|1992Q3|N|Y|N|2448836|2449048|2448491|2448766|N|N|N|N|N| +2448858|AAAAAAAAKNNFFCAA|1992-08-23|1111|4834|371|1992|0|8|23|3|1992|371|4834|Sunday|1992Q3|N|N|N|2448836|2449048|2448492|2448767|N|N|N|N|N| +2448859|AAAAAAAALNNFFCAA|1992-08-24|1111|4834|371|1992|1|8|24|3|1992|371|4834|Monday|1992Q3|N|N|N|2448836|2449048|2448493|2448768|N|N|N|N|N| +2448860|AAAAAAAAMNNFFCAA|1992-08-25|1111|4835|371|1992|2|8|25|3|1992|371|4835|Tuesday|1992Q3|N|N|N|2448836|2449048|2448494|2448769|N|N|N|N|N| +2448861|AAAAAAAANNNFFCAA|1992-08-26|1111|4835|371|1992|3|8|26|3|1992|371|4835|Wednesday|1992Q3|N|N|N|2448836|2449048|2448495|2448770|N|N|N|N|N| +2448862|AAAAAAAAONNFFCAA|1992-08-27|1111|4835|371|1992|4|8|27|3|1992|371|4835|Thursday|1992Q3|N|N|N|2448836|2449048|2448496|2448771|N|N|N|N|N| +2448863|AAAAAAAAPNNFFCAA|1992-08-28|1111|4835|371|1992|5|8|28|3|1992|371|4835|Friday|1992Q3|N|Y|N|2448836|2449048|2448497|2448772|N|N|N|N|N| +2448864|AAAAAAAAAONFFCAA|1992-08-29|1111|4835|371|1992|6|8|29|3|1992|371|4835|Saturday|1992Q3|N|Y|N|2448836|2449048|2448498|2448773|N|N|N|N|N| +2448865|AAAAAAAABONFFCAA|1992-08-30|1111|4835|371|1992|0|8|30|3|1992|371|4835|Sunday|1992Q3|N|N|N|2448836|2449048|2448499|2448774|N|N|N|N|N| +2448866|AAAAAAAACONFFCAA|1992-08-31|1111|4835|371|1992|1|8|31|3|1992|371|4835|Monday|1992Q3|N|N|N|2448836|2449048|2448500|2448775|N|N|N|N|N| +2448867|AAAAAAAADONFFCAA|1992-09-01|1112|4836|372|1992|2|9|1|3|1992|372|4836|Tuesday|1992Q3|N|N|N|2448867|2449110|2448501|2448776|N|N|N|N|N| +2448868|AAAAAAAAEONFFCAA|1992-09-02|1112|4836|372|1992|3|9|2|3|1992|372|4836|Wednesday|1992Q3|N|N|N|2448867|2449110|2448502|2448777|N|N|N|N|N| +2448869|AAAAAAAAFONFFCAA|1992-09-03|1112|4836|372|1992|4|9|3|3|1992|372|4836|Thursday|1992Q3|N|N|N|2448867|2449110|2448503|2448778|N|N|N|N|N| +2448870|AAAAAAAAGONFFCAA|1992-09-04|1112|4836|372|1992|5|9|4|3|1992|372|4836|Friday|1992Q3|N|Y|N|2448867|2449110|2448504|2448779|N|N|N|N|N| +2448871|AAAAAAAAHONFFCAA|1992-09-05|1112|4836|372|1992|6|9|5|3|1992|372|4836|Saturday|1992Q3|N|Y|N|2448867|2449110|2448505|2448780|N|N|N|N|N| +2448872|AAAAAAAAIONFFCAA|1992-09-06|1112|4836|372|1992|0|9|6|3|1992|372|4836|Sunday|1992Q3|N|N|N|2448867|2449110|2448506|2448781|N|N|N|N|N| +2448873|AAAAAAAAJONFFCAA|1992-09-07|1112|4836|372|1992|1|9|7|3|1992|372|4836|Monday|1992Q3|N|N|N|2448867|2449110|2448507|2448782|N|N|N|N|N| +2448874|AAAAAAAAKONFFCAA|1992-09-08|1112|4837|372|1992|2|9|8|3|1992|372|4837|Tuesday|1992Q3|N|N|N|2448867|2449110|2448508|2448783|N|N|N|N|N| +2448875|AAAAAAAALONFFCAA|1992-09-09|1112|4837|372|1992|3|9|9|3|1992|372|4837|Wednesday|1992Q3|N|N|N|2448867|2449110|2448509|2448784|N|N|N|N|N| +2448876|AAAAAAAAMONFFCAA|1992-09-10|1112|4837|372|1992|4|9|10|3|1992|372|4837|Thursday|1992Q3|N|N|N|2448867|2449110|2448510|2448785|N|N|N|N|N| +2448877|AAAAAAAANONFFCAA|1992-09-11|1112|4837|372|1992|5|9|11|3|1992|372|4837|Friday|1992Q3|N|Y|N|2448867|2449110|2448511|2448786|N|N|N|N|N| +2448878|AAAAAAAAOONFFCAA|1992-09-12|1112|4837|372|1992|6|9|12|3|1992|372|4837|Saturday|1992Q3|N|Y|N|2448867|2449110|2448512|2448787|N|N|N|N|N| +2448879|AAAAAAAAPONFFCAA|1992-09-13|1112|4837|372|1992|0|9|13|3|1992|372|4837|Sunday|1992Q3|N|N|N|2448867|2449110|2448513|2448788|N|N|N|N|N| +2448880|AAAAAAAAAPNFFCAA|1992-09-14|1112|4837|372|1992|1|9|14|3|1992|372|4837|Monday|1992Q3|N|N|N|2448867|2449110|2448514|2448789|N|N|N|N|N| +2448881|AAAAAAAABPNFFCAA|1992-09-15|1112|4838|372|1992|2|9|15|3|1992|372|4838|Tuesday|1992Q3|N|N|N|2448867|2449110|2448515|2448790|N|N|N|N|N| +2448882|AAAAAAAACPNFFCAA|1992-09-16|1112|4838|372|1992|3|9|16|3|1992|372|4838|Wednesday|1992Q3|N|N|N|2448867|2449110|2448516|2448791|N|N|N|N|N| +2448883|AAAAAAAADPNFFCAA|1992-09-17|1112|4838|372|1992|4|9|17|3|1992|372|4838|Thursday|1992Q3|N|N|N|2448867|2449110|2448517|2448792|N|N|N|N|N| +2448884|AAAAAAAAEPNFFCAA|1992-09-18|1112|4838|372|1992|5|9|18|3|1992|372|4838|Friday|1992Q3|N|Y|N|2448867|2449110|2448518|2448793|N|N|N|N|N| +2448885|AAAAAAAAFPNFFCAA|1992-09-19|1112|4838|372|1992|6|9|19|3|1992|372|4838|Saturday|1992Q3|N|Y|N|2448867|2449110|2448519|2448794|N|N|N|N|N| +2448886|AAAAAAAAGPNFFCAA|1992-09-20|1112|4838|372|1992|0|9|20|3|1992|372|4838|Sunday|1992Q3|N|N|N|2448867|2449110|2448520|2448795|N|N|N|N|N| +2448887|AAAAAAAAHPNFFCAA|1992-09-21|1112|4838|372|1992|1|9|21|3|1992|372|4838|Monday|1992Q3|N|N|N|2448867|2449110|2448521|2448796|N|N|N|N|N| +2448888|AAAAAAAAIPNFFCAA|1992-09-22|1112|4839|372|1992|2|9|22|3|1992|372|4839|Tuesday|1992Q3|N|N|N|2448867|2449110|2448522|2448797|N|N|N|N|N| +2448889|AAAAAAAAJPNFFCAA|1992-09-23|1112|4839|372|1992|3|9|23|3|1992|372|4839|Wednesday|1992Q3|N|N|N|2448867|2449110|2448523|2448798|N|N|N|N|N| +2448890|AAAAAAAAKPNFFCAA|1992-09-24|1112|4839|372|1992|4|9|24|3|1992|372|4839|Thursday|1992Q3|N|N|N|2448867|2449110|2448524|2448799|N|N|N|N|N| +2448891|AAAAAAAALPNFFCAA|1992-09-25|1112|4839|372|1992|5|9|25|3|1992|372|4839|Friday|1992Q3|N|Y|N|2448867|2449110|2448525|2448800|N|N|N|N|N| +2448892|AAAAAAAAMPNFFCAA|1992-09-26|1112|4839|372|1992|6|9|26|3|1992|372|4839|Saturday|1992Q3|N|Y|N|2448867|2449110|2448526|2448801|N|N|N|N|N| +2448893|AAAAAAAANPNFFCAA|1992-09-27|1112|4839|372|1992|0|9|27|3|1992|372|4839|Sunday|1992Q3|N|N|N|2448867|2449110|2448527|2448802|N|N|N|N|N| +2448894|AAAAAAAAOPNFFCAA|1992-09-28|1112|4839|372|1992|1|9|28|3|1992|372|4839|Monday|1992Q3|N|N|N|2448867|2449110|2448528|2448803|N|N|N|N|N| +2448895|AAAAAAAAPPNFFCAA|1992-09-29|1112|4840|372|1992|2|9|29|3|1992|372|4840|Tuesday|1992Q3|N|N|N|2448867|2449110|2448529|2448804|N|N|N|N|N| +2448896|AAAAAAAAAAOFFCAA|1992-09-30|1112|4840|372|1992|3|9|30|3|1992|372|4840|Wednesday|1992Q3|N|N|N|2448867|2449110|2448530|2448805|N|N|N|N|N| +2448897|AAAAAAAABAOFFCAA|1992-10-01|1113|4840|372|1992|4|10|1|4|1992|372|4840|Thursday|1992Q4|N|N|N|2448897|2449170|2448531|2448805|N|N|N|N|N| +2448898|AAAAAAAACAOFFCAA|1992-10-02|1113|4840|372|1992|5|10|2|4|1992|372|4840|Friday|1992Q4|N|Y|N|2448897|2449170|2448532|2448806|N|N|N|N|N| +2448899|AAAAAAAADAOFFCAA|1992-10-03|1113|4840|372|1992|6|10|3|4|1992|372|4840|Saturday|1992Q4|N|Y|N|2448897|2449170|2448533|2448807|N|N|N|N|N| +2448900|AAAAAAAAEAOFFCAA|1992-10-04|1113|4840|372|1992|0|10|4|4|1992|372|4840|Sunday|1992Q4|N|N|N|2448897|2449170|2448534|2448808|N|N|N|N|N| +2448901|AAAAAAAAFAOFFCAA|1992-10-05|1113|4840|372|1992|1|10|5|4|1992|372|4840|Monday|1992Q4|N|N|N|2448897|2449170|2448535|2448809|N|N|N|N|N| +2448902|AAAAAAAAGAOFFCAA|1992-10-06|1113|4841|372|1992|2|10|6|4|1992|372|4841|Tuesday|1992Q4|N|N|N|2448897|2449170|2448536|2448810|N|N|N|N|N| +2448903|AAAAAAAAHAOFFCAA|1992-10-07|1113|4841|372|1992|3|10|7|4|1992|372|4841|Wednesday|1992Q4|N|N|N|2448897|2449170|2448537|2448811|N|N|N|N|N| +2448904|AAAAAAAAIAOFFCAA|1992-10-08|1113|4841|372|1992|4|10|8|4|1992|372|4841|Thursday|1992Q4|N|N|N|2448897|2449170|2448538|2448812|N|N|N|N|N| +2448905|AAAAAAAAJAOFFCAA|1992-10-09|1113|4841|372|1992|5|10|9|4|1992|372|4841|Friday|1992Q4|N|Y|N|2448897|2449170|2448539|2448813|N|N|N|N|N| +2448906|AAAAAAAAKAOFFCAA|1992-10-10|1113|4841|372|1992|6|10|10|4|1992|372|4841|Saturday|1992Q4|N|Y|N|2448897|2449170|2448540|2448814|N|N|N|N|N| +2448907|AAAAAAAALAOFFCAA|1992-10-11|1113|4841|372|1992|0|10|11|4|1992|372|4841|Sunday|1992Q4|N|N|N|2448897|2449170|2448541|2448815|N|N|N|N|N| +2448908|AAAAAAAAMAOFFCAA|1992-10-12|1113|4841|372|1992|1|10|12|4|1992|372|4841|Monday|1992Q4|N|N|N|2448897|2449170|2448542|2448816|N|N|N|N|N| +2448909|AAAAAAAANAOFFCAA|1992-10-13|1113|4842|372|1992|2|10|13|4|1992|372|4842|Tuesday|1992Q4|N|N|N|2448897|2449170|2448543|2448817|N|N|N|N|N| +2448910|AAAAAAAAOAOFFCAA|1992-10-14|1113|4842|372|1992|3|10|14|4|1992|372|4842|Wednesday|1992Q4|N|N|N|2448897|2449170|2448544|2448818|N|N|N|N|N| +2448911|AAAAAAAAPAOFFCAA|1992-10-15|1113|4842|372|1992|4|10|15|4|1992|372|4842|Thursday|1992Q4|N|N|N|2448897|2449170|2448545|2448819|N|N|N|N|N| +2448912|AAAAAAAAABOFFCAA|1992-10-16|1113|4842|372|1992|5|10|16|4|1992|372|4842|Friday|1992Q4|N|Y|N|2448897|2449170|2448546|2448820|N|N|N|N|N| +2448913|AAAAAAAABBOFFCAA|1992-10-17|1113|4842|372|1992|6|10|17|4|1992|372|4842|Saturday|1992Q4|N|Y|N|2448897|2449170|2448547|2448821|N|N|N|N|N| +2448914|AAAAAAAACBOFFCAA|1992-10-18|1113|4842|372|1992|0|10|18|4|1992|372|4842|Sunday|1992Q4|N|N|N|2448897|2449170|2448548|2448822|N|N|N|N|N| +2448915|AAAAAAAADBOFFCAA|1992-10-19|1113|4842|372|1992|1|10|19|4|1992|372|4842|Monday|1992Q4|N|N|N|2448897|2449170|2448549|2448823|N|N|N|N|N| +2448916|AAAAAAAAEBOFFCAA|1992-10-20|1113|4843|372|1992|2|10|20|4|1992|372|4843|Tuesday|1992Q4|N|N|N|2448897|2449170|2448550|2448824|N|N|N|N|N| +2448917|AAAAAAAAFBOFFCAA|1992-10-21|1113|4843|372|1992|3|10|21|4|1992|372|4843|Wednesday|1992Q4|N|N|N|2448897|2449170|2448551|2448825|N|N|N|N|N| +2448918|AAAAAAAAGBOFFCAA|1992-10-22|1113|4843|372|1992|4|10|22|4|1992|372|4843|Thursday|1992Q4|N|N|N|2448897|2449170|2448552|2448826|N|N|N|N|N| +2448919|AAAAAAAAHBOFFCAA|1992-10-23|1113|4843|372|1992|5|10|23|4|1992|372|4843|Friday|1992Q4|N|Y|N|2448897|2449170|2448553|2448827|N|N|N|N|N| +2448920|AAAAAAAAIBOFFCAA|1992-10-24|1113|4843|372|1992|6|10|24|4|1992|372|4843|Saturday|1992Q4|N|Y|N|2448897|2449170|2448554|2448828|N|N|N|N|N| +2448921|AAAAAAAAJBOFFCAA|1992-10-25|1113|4843|372|1992|0|10|25|4|1992|372|4843|Sunday|1992Q4|N|N|N|2448897|2449170|2448555|2448829|N|N|N|N|N| +2448922|AAAAAAAAKBOFFCAA|1992-10-26|1113|4843|372|1992|1|10|26|4|1992|372|4843|Monday|1992Q4|N|N|N|2448897|2449170|2448556|2448830|N|N|N|N|N| +2448923|AAAAAAAALBOFFCAA|1992-10-27|1113|4844|372|1992|2|10|27|4|1992|372|4844|Tuesday|1992Q4|N|N|N|2448897|2449170|2448557|2448831|N|N|N|N|N| +2448924|AAAAAAAAMBOFFCAA|1992-10-28|1113|4844|372|1992|3|10|28|4|1992|372|4844|Wednesday|1992Q4|N|N|N|2448897|2449170|2448558|2448832|N|N|N|N|N| +2448925|AAAAAAAANBOFFCAA|1992-10-29|1113|4844|372|1992|4|10|29|4|1992|372|4844|Thursday|1992Q4|N|N|N|2448897|2449170|2448559|2448833|N|N|N|N|N| +2448926|AAAAAAAAOBOFFCAA|1992-10-30|1113|4844|372|1992|5|10|30|4|1992|372|4844|Friday|1992Q4|N|Y|N|2448897|2449170|2448560|2448834|N|N|N|N|N| +2448927|AAAAAAAAPBOFFCAA|1992-10-31|1113|4844|372|1992|6|10|31|4|1992|372|4844|Saturday|1992Q4|N|Y|N|2448897|2449170|2448561|2448835|N|N|N|N|N| +2448928|AAAAAAAAACOFFCAA|1992-11-01|1114|4844|372|1992|0|11|1|4|1992|372|4844|Sunday|1992Q4|N|N|N|2448928|2449232|2448562|2448836|N|N|N|N|N| +2448929|AAAAAAAABCOFFCAA|1992-11-02|1114|4844|372|1992|1|11|2|4|1992|372|4844|Monday|1992Q4|N|N|N|2448928|2449232|2448563|2448837|N|N|N|N|N| +2448930|AAAAAAAACCOFFCAA|1992-11-03|1114|4845|372|1992|2|11|3|4|1992|372|4845|Tuesday|1992Q4|N|N|N|2448928|2449232|2448564|2448838|N|N|N|N|N| +2448931|AAAAAAAADCOFFCAA|1992-11-04|1114|4845|372|1992|3|11|4|4|1992|372|4845|Wednesday|1992Q4|N|N|N|2448928|2449232|2448565|2448839|N|N|N|N|N| +2448932|AAAAAAAAECOFFCAA|1992-11-05|1114|4845|372|1992|4|11|5|4|1992|372|4845|Thursday|1992Q4|N|N|N|2448928|2449232|2448566|2448840|N|N|N|N|N| +2448933|AAAAAAAAFCOFFCAA|1992-11-06|1114|4845|372|1992|5|11|6|4|1992|372|4845|Friday|1992Q4|N|Y|N|2448928|2449232|2448567|2448841|N|N|N|N|N| +2448934|AAAAAAAAGCOFFCAA|1992-11-07|1114|4845|372|1992|6|11|7|4|1992|372|4845|Saturday|1992Q4|N|Y|N|2448928|2449232|2448568|2448842|N|N|N|N|N| +2448935|AAAAAAAAHCOFFCAA|1992-11-08|1114|4845|372|1992|0|11|8|4|1992|372|4845|Sunday|1992Q4|N|N|N|2448928|2449232|2448569|2448843|N|N|N|N|N| +2448936|AAAAAAAAICOFFCAA|1992-11-09|1114|4845|372|1992|1|11|9|4|1992|372|4845|Monday|1992Q4|N|N|N|2448928|2449232|2448570|2448844|N|N|N|N|N| +2448937|AAAAAAAAJCOFFCAA|1992-11-10|1114|4846|372|1992|2|11|10|4|1992|372|4846|Tuesday|1992Q4|N|N|N|2448928|2449232|2448571|2448845|N|N|N|N|N| +2448938|AAAAAAAAKCOFFCAA|1992-11-11|1114|4846|372|1992|3|11|11|4|1992|372|4846|Wednesday|1992Q4|N|N|N|2448928|2449232|2448572|2448846|N|N|N|N|N| +2448939|AAAAAAAALCOFFCAA|1992-11-12|1114|4846|372|1992|4|11|12|4|1992|372|4846|Thursday|1992Q4|N|N|N|2448928|2449232|2448573|2448847|N|N|N|N|N| +2448940|AAAAAAAAMCOFFCAA|1992-11-13|1114|4846|372|1992|5|11|13|4|1992|372|4846|Friday|1992Q4|N|Y|N|2448928|2449232|2448574|2448848|N|N|N|N|N| +2448941|AAAAAAAANCOFFCAA|1992-11-14|1114|4846|372|1992|6|11|14|4|1992|372|4846|Saturday|1992Q4|N|Y|N|2448928|2449232|2448575|2448849|N|N|N|N|N| +2448942|AAAAAAAAOCOFFCAA|1992-11-15|1114|4846|372|1992|0|11|15|4|1992|372|4846|Sunday|1992Q4|N|N|N|2448928|2449232|2448576|2448850|N|N|N|N|N| +2448943|AAAAAAAAPCOFFCAA|1992-11-16|1114|4846|372|1992|1|11|16|4|1992|372|4846|Monday|1992Q4|N|N|N|2448928|2449232|2448577|2448851|N|N|N|N|N| +2448944|AAAAAAAAADOFFCAA|1992-11-17|1114|4847|372|1992|2|11|17|4|1992|372|4847|Tuesday|1992Q4|N|N|N|2448928|2449232|2448578|2448852|N|N|N|N|N| +2448945|AAAAAAAABDOFFCAA|1992-11-18|1114|4847|372|1992|3|11|18|4|1992|372|4847|Wednesday|1992Q4|N|N|N|2448928|2449232|2448579|2448853|N|N|N|N|N| +2448946|AAAAAAAACDOFFCAA|1992-11-19|1114|4847|372|1992|4|11|19|4|1992|372|4847|Thursday|1992Q4|N|N|N|2448928|2449232|2448580|2448854|N|N|N|N|N| +2448947|AAAAAAAADDOFFCAA|1992-11-20|1114|4847|372|1992|5|11|20|4|1992|372|4847|Friday|1992Q4|N|Y|N|2448928|2449232|2448581|2448855|N|N|N|N|N| +2448948|AAAAAAAAEDOFFCAA|1992-11-21|1114|4847|372|1992|6|11|21|4|1992|372|4847|Saturday|1992Q4|N|Y|N|2448928|2449232|2448582|2448856|N|N|N|N|N| +2448949|AAAAAAAAFDOFFCAA|1992-11-22|1114|4847|372|1992|0|11|22|4|1992|372|4847|Sunday|1992Q4|N|N|N|2448928|2449232|2448583|2448857|N|N|N|N|N| +2448950|AAAAAAAAGDOFFCAA|1992-11-23|1114|4847|372|1992|1|11|23|4|1992|372|4847|Monday|1992Q4|N|N|N|2448928|2449232|2448584|2448858|N|N|N|N|N| +2448951|AAAAAAAAHDOFFCAA|1992-11-24|1114|4848|372|1992|2|11|24|4|1992|372|4848|Tuesday|1992Q4|N|N|N|2448928|2449232|2448585|2448859|N|N|N|N|N| +2448952|AAAAAAAAIDOFFCAA|1992-11-25|1114|4848|372|1992|3|11|25|4|1992|372|4848|Wednesday|1992Q4|N|N|N|2448928|2449232|2448586|2448860|N|N|N|N|N| +2448953|AAAAAAAAJDOFFCAA|1992-11-26|1114|4848|372|1992|4|11|26|4|1992|372|4848|Thursday|1992Q4|N|N|N|2448928|2449232|2448587|2448861|N|N|N|N|N| +2448954|AAAAAAAAKDOFFCAA|1992-11-27|1114|4848|372|1992|5|11|27|4|1992|372|4848|Friday|1992Q4|N|Y|N|2448928|2449232|2448588|2448862|N|N|N|N|N| +2448955|AAAAAAAALDOFFCAA|1992-11-28|1114|4848|372|1992|6|11|28|4|1992|372|4848|Saturday|1992Q4|N|Y|N|2448928|2449232|2448589|2448863|N|N|N|N|N| +2448956|AAAAAAAAMDOFFCAA|1992-11-29|1114|4848|372|1992|0|11|29|4|1992|372|4848|Sunday|1992Q4|N|N|N|2448928|2449232|2448590|2448864|N|N|N|N|N| +2448957|AAAAAAAANDOFFCAA|1992-11-30|1114|4848|372|1992|1|11|30|4|1992|372|4848|Monday|1992Q4|N|N|N|2448928|2449232|2448591|2448865|N|N|N|N|N| +2448958|AAAAAAAAODOFFCAA|1992-12-01|1115|4849|373|1992|2|12|1|4|1992|373|4849|Tuesday|1992Q4|N|N|N|2448958|2449292|2448592|2448866|N|N|N|N|N| +2448959|AAAAAAAAPDOFFCAA|1992-12-02|1115|4849|373|1992|3|12|2|4|1992|373|4849|Wednesday|1992Q4|N|N|N|2448958|2449292|2448593|2448867|N|N|N|N|N| +2448960|AAAAAAAAAEOFFCAA|1992-12-03|1115|4849|373|1992|4|12|3|4|1992|373|4849|Thursday|1992Q4|N|N|N|2448958|2449292|2448594|2448868|N|N|N|N|N| +2448961|AAAAAAAABEOFFCAA|1992-12-04|1115|4849|373|1992|5|12|4|4|1992|373|4849|Friday|1992Q4|N|Y|N|2448958|2449292|2448595|2448869|N|N|N|N|N| +2448962|AAAAAAAACEOFFCAA|1992-12-05|1115|4849|373|1992|6|12|5|4|1992|373|4849|Saturday|1992Q4|N|Y|N|2448958|2449292|2448596|2448870|N|N|N|N|N| +2448963|AAAAAAAADEOFFCAA|1992-12-06|1115|4849|373|1992|0|12|6|4|1992|373|4849|Sunday|1992Q4|N|N|N|2448958|2449292|2448597|2448871|N|N|N|N|N| +2448964|AAAAAAAAEEOFFCAA|1992-12-07|1115|4849|373|1992|1|12|7|4|1992|373|4849|Monday|1992Q4|N|N|N|2448958|2449292|2448598|2448872|N|N|N|N|N| +2448965|AAAAAAAAFEOFFCAA|1992-12-08|1115|4850|373|1992|2|12|8|4|1992|373|4850|Tuesday|1992Q4|N|N|N|2448958|2449292|2448599|2448873|N|N|N|N|N| +2448966|AAAAAAAAGEOFFCAA|1992-12-09|1115|4850|373|1992|3|12|9|4|1992|373|4850|Wednesday|1992Q4|N|N|N|2448958|2449292|2448600|2448874|N|N|N|N|N| +2448967|AAAAAAAAHEOFFCAA|1992-12-10|1115|4850|373|1992|4|12|10|4|1992|373|4850|Thursday|1992Q4|N|N|N|2448958|2449292|2448601|2448875|N|N|N|N|N| +2448968|AAAAAAAAIEOFFCAA|1992-12-11|1115|4850|373|1992|5|12|11|4|1992|373|4850|Friday|1992Q4|N|Y|N|2448958|2449292|2448602|2448876|N|N|N|N|N| +2448969|AAAAAAAAJEOFFCAA|1992-12-12|1115|4850|373|1992|6|12|12|4|1992|373|4850|Saturday|1992Q4|N|Y|N|2448958|2449292|2448603|2448877|N|N|N|N|N| +2448970|AAAAAAAAKEOFFCAA|1992-12-13|1115|4850|373|1992|0|12|13|4|1992|373|4850|Sunday|1992Q4|N|N|N|2448958|2449292|2448604|2448878|N|N|N|N|N| +2448971|AAAAAAAALEOFFCAA|1992-12-14|1115|4850|373|1992|1|12|14|4|1992|373|4850|Monday|1992Q4|N|N|N|2448958|2449292|2448605|2448879|N|N|N|N|N| +2448972|AAAAAAAAMEOFFCAA|1992-12-15|1115|4851|373|1992|2|12|15|4|1992|373|4851|Tuesday|1992Q4|N|N|N|2448958|2449292|2448606|2448880|N|N|N|N|N| +2448973|AAAAAAAANEOFFCAA|1992-12-16|1115|4851|373|1992|3|12|16|4|1992|373|4851|Wednesday|1992Q4|N|N|N|2448958|2449292|2448607|2448881|N|N|N|N|N| +2448974|AAAAAAAAOEOFFCAA|1992-12-17|1115|4851|373|1992|4|12|17|4|1992|373|4851|Thursday|1992Q4|N|N|N|2448958|2449292|2448608|2448882|N|N|N|N|N| +2448975|AAAAAAAAPEOFFCAA|1992-12-18|1115|4851|373|1992|5|12|18|4|1992|373|4851|Friday|1992Q4|N|Y|N|2448958|2449292|2448609|2448883|N|N|N|N|N| +2448976|AAAAAAAAAFOFFCAA|1992-12-19|1115|4851|373|1992|6|12|19|4|1992|373|4851|Saturday|1992Q4|N|Y|N|2448958|2449292|2448610|2448884|N|N|N|N|N| +2448977|AAAAAAAABFOFFCAA|1992-12-20|1115|4851|373|1992|0|12|20|4|1992|373|4851|Sunday|1992Q4|N|N|N|2448958|2449292|2448611|2448885|N|N|N|N|N| +2448978|AAAAAAAACFOFFCAA|1992-12-21|1115|4851|373|1992|1|12|21|4|1992|373|4851|Monday|1992Q4|N|N|N|2448958|2449292|2448612|2448886|N|N|N|N|N| +2448979|AAAAAAAADFOFFCAA|1992-12-22|1115|4852|373|1992|2|12|22|4|1992|373|4852|Tuesday|1992Q4|N|N|N|2448958|2449292|2448613|2448887|N|N|N|N|N| +2448980|AAAAAAAAEFOFFCAA|1992-12-23|1115|4852|373|1992|3|12|23|4|1992|373|4852|Wednesday|1992Q4|N|N|N|2448958|2449292|2448614|2448888|N|N|N|N|N| +2448981|AAAAAAAAFFOFFCAA|1992-12-24|1115|4852|373|1992|4|12|24|4|1992|373|4852|Thursday|1992Q4|N|N|N|2448958|2449292|2448615|2448889|N|N|N|N|N| +2448982|AAAAAAAAGFOFFCAA|1992-12-25|1115|4852|373|1992|5|12|25|4|1992|373|4852|Friday|1992Q4|Y|Y|N|2448958|2449292|2448616|2448890|N|N|N|N|N| +2448983|AAAAAAAAHFOFFCAA|1992-12-26|1115|4852|373|1992|6|12|26|4|1992|373|4852|Saturday|1992Q4|N|Y|Y|2448958|2449292|2448617|2448891|N|N|N|N|N| +2448984|AAAAAAAAIFOFFCAA|1992-12-27|1115|4852|373|1992|0|12|27|4|1992|373|4852|Sunday|1992Q4|N|N|N|2448958|2449292|2448618|2448892|N|N|N|N|N| +2448985|AAAAAAAAJFOFFCAA|1992-12-28|1115|4852|373|1992|1|12|28|4|1992|373|4852|Monday|1992Q4|N|N|N|2448958|2449292|2448619|2448893|N|N|N|N|N| +2448986|AAAAAAAAKFOFFCAA|1992-12-29|1115|4853|373|1992|2|12|29|4|1992|373|4853|Tuesday|1992Q4|N|N|N|2448958|2449292|2448620|2448894|N|N|N|N|N| +2448987|AAAAAAAALFOFFCAA|1992-12-30|1115|4853|373|1992|3|12|30|4|1992|373|4853|Wednesday|1992Q4|N|N|N|2448958|2449292|2448621|2448895|N|N|N|N|N| +2448988|AAAAAAAAMFOFFCAA|1992-12-31|1115|4853|373|1992|4|12|31|4|1992|373|4853|Thursday|1992Q4|Y|N|N|2448958|2449292|2448622|2448896|N|N|N|N|N| +2448989|AAAAAAAANFOFFCAA|1993-01-01|1116|4853|373|1993|5|1|1|1|1993|373|4853|Friday|1993Q1|Y|Y|Y|2448989|2448988|2448623|2448897|N|N|N|N|N| +2448990|AAAAAAAAOFOFFCAA|1993-01-02|1116|4853|373|1993|6|1|2|1|1993|373|4853|Saturday|1993Q1|N|Y|Y|2448989|2448988|2448624|2448898|N|N|N|N|N| +2448991|AAAAAAAAPFOFFCAA|1993-01-03|1116|4853|373|1993|0|1|3|1|1993|373|4853|Sunday|1993Q1|N|N|N|2448989|2448988|2448625|2448899|N|N|N|N|N| +2448992|AAAAAAAAAGOFFCAA|1993-01-04|1116|4853|373|1993|1|1|4|1|1993|373|4853|Monday|1993Q1|N|N|N|2448989|2448988|2448626|2448900|N|N|N|N|N| +2448993|AAAAAAAABGOFFCAA|1993-01-05|1116|4854|373|1993|2|1|5|1|1993|373|4854|Tuesday|1993Q1|N|N|N|2448989|2448988|2448627|2448901|N|N|N|N|N| +2448994|AAAAAAAACGOFFCAA|1993-01-06|1116|4854|373|1993|3|1|6|1|1993|373|4854|Wednesday|1993Q1|N|N|N|2448989|2448988|2448628|2448902|N|N|N|N|N| +2448995|AAAAAAAADGOFFCAA|1993-01-07|1116|4854|373|1993|4|1|7|1|1993|373|4854|Thursday|1993Q1|N|N|N|2448989|2448988|2448629|2448903|N|N|N|N|N| +2448996|AAAAAAAAEGOFFCAA|1993-01-08|1116|4854|373|1993|5|1|8|1|1993|373|4854|Friday|1993Q1|N|Y|N|2448989|2448988|2448630|2448904|N|N|N|N|N| +2448997|AAAAAAAAFGOFFCAA|1993-01-09|1116|4854|373|1993|6|1|9|1|1993|373|4854|Saturday|1993Q1|N|Y|N|2448989|2448988|2448631|2448905|N|N|N|N|N| +2448998|AAAAAAAAGGOFFCAA|1993-01-10|1116|4854|373|1993|0|1|10|1|1993|373|4854|Sunday|1993Q1|N|N|N|2448989|2448988|2448632|2448906|N|N|N|N|N| +2448999|AAAAAAAAHGOFFCAA|1993-01-11|1116|4854|373|1993|1|1|11|1|1993|373|4854|Monday|1993Q1|N|N|N|2448989|2448988|2448633|2448907|N|N|N|N|N| +2449000|AAAAAAAAIGOFFCAA|1993-01-12|1116|4855|373|1993|2|1|12|1|1993|373|4855|Tuesday|1993Q1|N|N|N|2448989|2448988|2448634|2448908|N|N|N|N|N| +2449001|AAAAAAAAJGOFFCAA|1993-01-13|1116|4855|373|1993|3|1|13|1|1993|373|4855|Wednesday|1993Q1|N|N|N|2448989|2448988|2448635|2448909|N|N|N|N|N| +2449002|AAAAAAAAKGOFFCAA|1993-01-14|1116|4855|373|1993|4|1|14|1|1993|373|4855|Thursday|1993Q1|N|N|N|2448989|2448988|2448636|2448910|N|N|N|N|N| +2449003|AAAAAAAALGOFFCAA|1993-01-15|1116|4855|373|1993|5|1|15|1|1993|373|4855|Friday|1993Q1|N|Y|N|2448989|2448988|2448637|2448911|N|N|N|N|N| +2449004|AAAAAAAAMGOFFCAA|1993-01-16|1116|4855|373|1993|6|1|16|1|1993|373|4855|Saturday|1993Q1|N|Y|N|2448989|2448988|2448638|2448912|N|N|N|N|N| +2449005|AAAAAAAANGOFFCAA|1993-01-17|1116|4855|373|1993|0|1|17|1|1993|373|4855|Sunday|1993Q1|N|N|N|2448989|2448988|2448639|2448913|N|N|N|N|N| +2449006|AAAAAAAAOGOFFCAA|1993-01-18|1116|4855|373|1993|1|1|18|1|1993|373|4855|Monday|1993Q1|N|N|N|2448989|2448988|2448640|2448914|N|N|N|N|N| +2449007|AAAAAAAAPGOFFCAA|1993-01-19|1116|4856|373|1993|2|1|19|1|1993|373|4856|Tuesday|1993Q1|N|N|N|2448989|2448988|2448641|2448915|N|N|N|N|N| +2449008|AAAAAAAAAHOFFCAA|1993-01-20|1116|4856|373|1993|3|1|20|1|1993|373|4856|Wednesday|1993Q1|N|N|N|2448989|2448988|2448642|2448916|N|N|N|N|N| +2449009|AAAAAAAABHOFFCAA|1993-01-21|1116|4856|373|1993|4|1|21|1|1993|373|4856|Thursday|1993Q1|N|N|N|2448989|2448988|2448643|2448917|N|N|N|N|N| +2449010|AAAAAAAACHOFFCAA|1993-01-22|1116|4856|373|1993|5|1|22|1|1993|373|4856|Friday|1993Q1|N|Y|N|2448989|2448988|2448644|2448918|N|N|N|N|N| +2449011|AAAAAAAADHOFFCAA|1993-01-23|1116|4856|373|1993|6|1|23|1|1993|373|4856|Saturday|1993Q1|N|Y|N|2448989|2448988|2448645|2448919|N|N|N|N|N| +2449012|AAAAAAAAEHOFFCAA|1993-01-24|1116|4856|373|1993|0|1|24|1|1993|373|4856|Sunday|1993Q1|N|N|N|2448989|2448988|2448646|2448920|N|N|N|N|N| +2449013|AAAAAAAAFHOFFCAA|1993-01-25|1116|4856|373|1993|1|1|25|1|1993|373|4856|Monday|1993Q1|N|N|N|2448989|2448988|2448647|2448921|N|N|N|N|N| +2449014|AAAAAAAAGHOFFCAA|1993-01-26|1116|4857|373|1993|2|1|26|1|1993|373|4857|Tuesday|1993Q1|N|N|N|2448989|2448988|2448648|2448922|N|N|N|N|N| +2449015|AAAAAAAAHHOFFCAA|1993-01-27|1116|4857|373|1993|3|1|27|1|1993|373|4857|Wednesday|1993Q1|N|N|N|2448989|2448988|2448649|2448923|N|N|N|N|N| +2449016|AAAAAAAAIHOFFCAA|1993-01-28|1116|4857|373|1993|4|1|28|1|1993|373|4857|Thursday|1993Q1|N|N|N|2448989|2448988|2448650|2448924|N|N|N|N|N| +2449017|AAAAAAAAJHOFFCAA|1993-01-29|1116|4857|373|1993|5|1|29|1|1993|373|4857|Friday|1993Q1|N|Y|N|2448989|2448988|2448651|2448925|N|N|N|N|N| +2449018|AAAAAAAAKHOFFCAA|1993-01-30|1116|4857|373|1993|6|1|30|1|1993|373|4857|Saturday|1993Q1|N|Y|N|2448989|2448988|2448652|2448926|N|N|N|N|N| +2449019|AAAAAAAALHOFFCAA|1993-01-31|1116|4857|373|1993|0|1|31|1|1993|373|4857|Sunday|1993Q1|N|N|N|2448989|2448988|2448653|2448927|N|N|N|N|N| +2449020|AAAAAAAAMHOFFCAA|1993-02-01|1117|4857|373|1993|1|2|1|1|1993|373|4857|Monday|1993Q1|N|N|N|2449020|2449050|2448654|2448928|N|N|N|N|N| +2449021|AAAAAAAANHOFFCAA|1993-02-02|1117|4858|373|1993|2|2|2|1|1993|373|4858|Tuesday|1993Q1|N|N|N|2449020|2449050|2448655|2448929|N|N|N|N|N| +2449022|AAAAAAAAOHOFFCAA|1993-02-03|1117|4858|373|1993|3|2|3|1|1993|373|4858|Wednesday|1993Q1|N|N|N|2449020|2449050|2448656|2448930|N|N|N|N|N| +2449023|AAAAAAAAPHOFFCAA|1993-02-04|1117|4858|373|1993|4|2|4|1|1993|373|4858|Thursday|1993Q1|N|N|N|2449020|2449050|2448657|2448931|N|N|N|N|N| +2449024|AAAAAAAAAIOFFCAA|1993-02-05|1117|4858|373|1993|5|2|5|1|1993|373|4858|Friday|1993Q1|N|Y|N|2449020|2449050|2448658|2448932|N|N|N|N|N| +2449025|AAAAAAAABIOFFCAA|1993-02-06|1117|4858|373|1993|6|2|6|1|1993|373|4858|Saturday|1993Q1|N|Y|N|2449020|2449050|2448659|2448933|N|N|N|N|N| +2449026|AAAAAAAACIOFFCAA|1993-02-07|1117|4858|373|1993|0|2|7|1|1993|373|4858|Sunday|1993Q1|N|N|N|2449020|2449050|2448660|2448934|N|N|N|N|N| +2449027|AAAAAAAADIOFFCAA|1993-02-08|1117|4858|373|1993|1|2|8|1|1993|373|4858|Monday|1993Q1|N|N|N|2449020|2449050|2448661|2448935|N|N|N|N|N| +2449028|AAAAAAAAEIOFFCAA|1993-02-09|1117|4859|373|1993|2|2|9|1|1993|373|4859|Tuesday|1993Q1|N|N|N|2449020|2449050|2448662|2448936|N|N|N|N|N| +2449029|AAAAAAAAFIOFFCAA|1993-02-10|1117|4859|373|1993|3|2|10|1|1993|373|4859|Wednesday|1993Q1|N|N|N|2449020|2449050|2448663|2448937|N|N|N|N|N| +2449030|AAAAAAAAGIOFFCAA|1993-02-11|1117|4859|373|1993|4|2|11|1|1993|373|4859|Thursday|1993Q1|N|N|N|2449020|2449050|2448664|2448938|N|N|N|N|N| +2449031|AAAAAAAAHIOFFCAA|1993-02-12|1117|4859|373|1993|5|2|12|1|1993|373|4859|Friday|1993Q1|N|Y|N|2449020|2449050|2448665|2448939|N|N|N|N|N| +2449032|AAAAAAAAIIOFFCAA|1993-02-13|1117|4859|373|1993|6|2|13|1|1993|373|4859|Saturday|1993Q1|N|Y|N|2449020|2449050|2448666|2448940|N|N|N|N|N| +2449033|AAAAAAAAJIOFFCAA|1993-02-14|1117|4859|373|1993|0|2|14|1|1993|373|4859|Sunday|1993Q1|N|N|N|2449020|2449050|2448667|2448941|N|N|N|N|N| +2449034|AAAAAAAAKIOFFCAA|1993-02-15|1117|4859|373|1993|1|2|15|1|1993|373|4859|Monday|1993Q1|N|N|N|2449020|2449050|2448668|2448942|N|N|N|N|N| +2449035|AAAAAAAALIOFFCAA|1993-02-16|1117|4860|373|1993|2|2|16|1|1993|373|4860|Tuesday|1993Q1|N|N|N|2449020|2449050|2448669|2448943|N|N|N|N|N| +2449036|AAAAAAAAMIOFFCAA|1993-02-17|1117|4860|373|1993|3|2|17|1|1993|373|4860|Wednesday|1993Q1|N|N|N|2449020|2449050|2448670|2448944|N|N|N|N|N| +2449037|AAAAAAAANIOFFCAA|1993-02-18|1117|4860|373|1993|4|2|18|1|1993|373|4860|Thursday|1993Q1|N|N|N|2449020|2449050|2448671|2448945|N|N|N|N|N| +2449038|AAAAAAAAOIOFFCAA|1993-02-19|1117|4860|373|1993|5|2|19|1|1993|373|4860|Friday|1993Q1|N|Y|N|2449020|2449050|2448672|2448946|N|N|N|N|N| +2449039|AAAAAAAAPIOFFCAA|1993-02-20|1117|4860|373|1993|6|2|20|1|1993|373|4860|Saturday|1993Q1|N|Y|N|2449020|2449050|2448673|2448947|N|N|N|N|N| +2449040|AAAAAAAAAJOFFCAA|1993-02-21|1117|4860|373|1993|0|2|21|1|1993|373|4860|Sunday|1993Q1|N|N|N|2449020|2449050|2448674|2448948|N|N|N|N|N| +2449041|AAAAAAAABJOFFCAA|1993-02-22|1117|4860|373|1993|1|2|22|1|1993|373|4860|Monday|1993Q1|N|N|N|2449020|2449050|2448675|2448949|N|N|N|N|N| +2449042|AAAAAAAACJOFFCAA|1993-02-23|1117|4861|373|1993|2|2|23|1|1993|373|4861|Tuesday|1993Q1|N|N|N|2449020|2449050|2448676|2448950|N|N|N|N|N| +2449043|AAAAAAAADJOFFCAA|1993-02-24|1117|4861|373|1993|3|2|24|1|1993|373|4861|Wednesday|1993Q1|N|N|N|2449020|2449050|2448677|2448951|N|N|N|N|N| +2449044|AAAAAAAAEJOFFCAA|1993-02-25|1117|4861|373|1993|4|2|25|1|1993|373|4861|Thursday|1993Q1|N|N|N|2449020|2449050|2448678|2448952|N|N|N|N|N| +2449045|AAAAAAAAFJOFFCAA|1993-02-26|1117|4861|373|1993|5|2|26|1|1993|373|4861|Friday|1993Q1|N|Y|N|2449020|2449050|2448679|2448953|N|N|N|N|N| +2449046|AAAAAAAAGJOFFCAA|1993-02-27|1117|4861|373|1993|6|2|27|1|1993|373|4861|Saturday|1993Q1|N|Y|N|2449020|2449050|2448680|2448954|N|N|N|N|N| +2449047|AAAAAAAAHJOFFCAA|1993-02-28|1117|4861|373|1993|0|2|28|1|1993|373|4861|Sunday|1993Q1|N|N|N|2449020|2449050|2448681|2448955|N|N|N|N|N| +2449048|AAAAAAAAIJOFFCAA|1993-03-01|1118|4861|374|1993|1|3|1|1|1993|374|4861|Monday|1993Q1|N|N|N|2449048|2449106|2448683|2448956|N|N|N|N|N| +2449049|AAAAAAAAJJOFFCAA|1993-03-02|1118|4862|374|1993|2|3|2|1|1993|374|4862|Tuesday|1993Q1|N|N|N|2449048|2449106|2448684|2448957|N|N|N|N|N| +2449050|AAAAAAAAKJOFFCAA|1993-03-03|1118|4862|374|1993|3|3|3|1|1993|374|4862|Wednesday|1993Q1|N|N|N|2449048|2449106|2448685|2448958|N|N|N|N|N| +2449051|AAAAAAAALJOFFCAA|1993-03-04|1118|4862|374|1993|4|3|4|1|1993|374|4862|Thursday|1993Q1|N|N|N|2449048|2449106|2448686|2448959|N|N|N|N|N| +2449052|AAAAAAAAMJOFFCAA|1993-03-05|1118|4862|374|1993|5|3|5|1|1993|374|4862|Friday|1993Q1|N|Y|N|2449048|2449106|2448687|2448960|N|N|N|N|N| +2449053|AAAAAAAANJOFFCAA|1993-03-06|1118|4862|374|1993|6|3|6|1|1993|374|4862|Saturday|1993Q1|N|Y|N|2449048|2449106|2448688|2448961|N|N|N|N|N| +2449054|AAAAAAAAOJOFFCAA|1993-03-07|1118|4862|374|1993|0|3|7|1|1993|374|4862|Sunday|1993Q1|N|N|N|2449048|2449106|2448689|2448962|N|N|N|N|N| +2449055|AAAAAAAAPJOFFCAA|1993-03-08|1118|4862|374|1993|1|3|8|1|1993|374|4862|Monday|1993Q1|N|N|N|2449048|2449106|2448690|2448963|N|N|N|N|N| +2449056|AAAAAAAAAKOFFCAA|1993-03-09|1118|4863|374|1993|2|3|9|1|1993|374|4863|Tuesday|1993Q1|N|N|N|2449048|2449106|2448691|2448964|N|N|N|N|N| +2449057|AAAAAAAABKOFFCAA|1993-03-10|1118|4863|374|1993|3|3|10|1|1993|374|4863|Wednesday|1993Q1|N|N|N|2449048|2449106|2448692|2448965|N|N|N|N|N| +2449058|AAAAAAAACKOFFCAA|1993-03-11|1118|4863|374|1993|4|3|11|1|1993|374|4863|Thursday|1993Q1|N|N|N|2449048|2449106|2448693|2448966|N|N|N|N|N| +2449059|AAAAAAAADKOFFCAA|1993-03-12|1118|4863|374|1993|5|3|12|1|1993|374|4863|Friday|1993Q1|N|Y|N|2449048|2449106|2448694|2448967|N|N|N|N|N| +2449060|AAAAAAAAEKOFFCAA|1993-03-13|1118|4863|374|1993|6|3|13|1|1993|374|4863|Saturday|1993Q1|N|Y|N|2449048|2449106|2448695|2448968|N|N|N|N|N| +2449061|AAAAAAAAFKOFFCAA|1993-03-14|1118|4863|374|1993|0|3|14|1|1993|374|4863|Sunday|1993Q1|N|N|N|2449048|2449106|2448696|2448969|N|N|N|N|N| +2449062|AAAAAAAAGKOFFCAA|1993-03-15|1118|4863|374|1993|1|3|15|1|1993|374|4863|Monday|1993Q1|N|N|N|2449048|2449106|2448697|2448970|N|N|N|N|N| +2449063|AAAAAAAAHKOFFCAA|1993-03-16|1118|4864|374|1993|2|3|16|1|1993|374|4864|Tuesday|1993Q1|N|N|N|2449048|2449106|2448698|2448971|N|N|N|N|N| +2449064|AAAAAAAAIKOFFCAA|1993-03-17|1118|4864|374|1993|3|3|17|1|1993|374|4864|Wednesday|1993Q1|N|N|N|2449048|2449106|2448699|2448972|N|N|N|N|N| +2449065|AAAAAAAAJKOFFCAA|1993-03-18|1118|4864|374|1993|4|3|18|1|1993|374|4864|Thursday|1993Q1|N|N|N|2449048|2449106|2448700|2448973|N|N|N|N|N| +2449066|AAAAAAAAKKOFFCAA|1993-03-19|1118|4864|374|1993|5|3|19|1|1993|374|4864|Friday|1993Q1|N|Y|N|2449048|2449106|2448701|2448974|N|N|N|N|N| +2449067|AAAAAAAALKOFFCAA|1993-03-20|1118|4864|374|1993|6|3|20|1|1993|374|4864|Saturday|1993Q1|N|Y|N|2449048|2449106|2448702|2448975|N|N|N|N|N| +2449068|AAAAAAAAMKOFFCAA|1993-03-21|1118|4864|374|1993|0|3|21|1|1993|374|4864|Sunday|1993Q1|N|N|N|2449048|2449106|2448703|2448976|N|N|N|N|N| +2449069|AAAAAAAANKOFFCAA|1993-03-22|1118|4864|374|1993|1|3|22|1|1993|374|4864|Monday|1993Q1|N|N|N|2449048|2449106|2448704|2448977|N|N|N|N|N| +2449070|AAAAAAAAOKOFFCAA|1993-03-23|1118|4865|374|1993|2|3|23|1|1993|374|4865|Tuesday|1993Q1|N|N|N|2449048|2449106|2448705|2448978|N|N|N|N|N| +2449071|AAAAAAAAPKOFFCAA|1993-03-24|1118|4865|374|1993|3|3|24|1|1993|374|4865|Wednesday|1993Q1|N|N|N|2449048|2449106|2448706|2448979|N|N|N|N|N| +2449072|AAAAAAAAALOFFCAA|1993-03-25|1118|4865|374|1993|4|3|25|1|1993|374|4865|Thursday|1993Q1|N|N|N|2449048|2449106|2448707|2448980|N|N|N|N|N| +2449073|AAAAAAAABLOFFCAA|1993-03-26|1118|4865|374|1993|5|3|26|1|1993|374|4865|Friday|1993Q1|N|Y|N|2449048|2449106|2448708|2448981|N|N|N|N|N| +2449074|AAAAAAAACLOFFCAA|1993-03-27|1118|4865|374|1993|6|3|27|1|1993|374|4865|Saturday|1993Q1|N|Y|N|2449048|2449106|2448709|2448982|N|N|N|N|N| +2449075|AAAAAAAADLOFFCAA|1993-03-28|1118|4865|374|1993|0|3|28|1|1993|374|4865|Sunday|1993Q1|N|N|N|2449048|2449106|2448710|2448983|N|N|N|N|N| +2449076|AAAAAAAAELOFFCAA|1993-03-29|1118|4865|374|1993|1|3|29|1|1993|374|4865|Monday|1993Q1|N|N|N|2449048|2449106|2448711|2448984|N|N|N|N|N| +2449077|AAAAAAAAFLOFFCAA|1993-03-30|1118|4866|374|1993|2|3|30|1|1993|374|4866|Tuesday|1993Q1|N|N|N|2449048|2449106|2448712|2448985|N|N|N|N|N| +2449078|AAAAAAAAGLOFFCAA|1993-03-31|1118|4866|374|1993|3|3|31|1|1993|374|4866|Wednesday|1993Q1|N|N|N|2449048|2449106|2448713|2448986|N|N|N|N|N| +2449079|AAAAAAAAHLOFFCAA|1993-04-01|1119|4866|374|1993|4|4|1|1|1993|374|4866|Thursday|1993Q1|N|N|N|2449079|2449168|2448714|2448989|N|N|N|N|N| +2449080|AAAAAAAAILOFFCAA|1993-04-02|1119|4866|374|1993|5|4|2|2|1993|374|4866|Friday|1993Q2|N|Y|N|2449079|2449168|2448715|2448990|N|N|N|N|N| +2449081|AAAAAAAAJLOFFCAA|1993-04-03|1119|4866|374|1993|6|4|3|2|1993|374|4866|Saturday|1993Q2|N|Y|N|2449079|2449168|2448716|2448991|N|N|N|N|N| +2449082|AAAAAAAAKLOFFCAA|1993-04-04|1119|4866|374|1993|0|4|4|2|1993|374|4866|Sunday|1993Q2|N|N|N|2449079|2449168|2448717|2448992|N|N|N|N|N| +2449083|AAAAAAAALLOFFCAA|1993-04-05|1119|4866|374|1993|1|4|5|2|1993|374|4866|Monday|1993Q2|N|N|N|2449079|2449168|2448718|2448993|N|N|N|N|N| +2449084|AAAAAAAAMLOFFCAA|1993-04-06|1119|4867|374|1993|2|4|6|2|1993|374|4867|Tuesday|1993Q2|N|N|N|2449079|2449168|2448719|2448994|N|N|N|N|N| +2449085|AAAAAAAANLOFFCAA|1993-04-07|1119|4867|374|1993|3|4|7|2|1993|374|4867|Wednesday|1993Q2|N|N|N|2449079|2449168|2448720|2448995|N|N|N|N|N| +2449086|AAAAAAAAOLOFFCAA|1993-04-08|1119|4867|374|1993|4|4|8|2|1993|374|4867|Thursday|1993Q2|N|N|N|2449079|2449168|2448721|2448996|N|N|N|N|N| +2449087|AAAAAAAAPLOFFCAA|1993-04-09|1119|4867|374|1993|5|4|9|2|1993|374|4867|Friday|1993Q2|N|Y|N|2449079|2449168|2448722|2448997|N|N|N|N|N| +2449088|AAAAAAAAAMOFFCAA|1993-04-10|1119|4867|374|1993|6|4|10|2|1993|374|4867|Saturday|1993Q2|N|Y|N|2449079|2449168|2448723|2448998|N|N|N|N|N| +2449089|AAAAAAAABMOFFCAA|1993-04-11|1119|4867|374|1993|0|4|11|2|1993|374|4867|Sunday|1993Q2|N|N|N|2449079|2449168|2448724|2448999|N|N|N|N|N| +2449090|AAAAAAAACMOFFCAA|1993-04-12|1119|4867|374|1993|1|4|12|2|1993|374|4867|Monday|1993Q2|N|N|N|2449079|2449168|2448725|2449000|N|N|N|N|N| +2449091|AAAAAAAADMOFFCAA|1993-04-13|1119|4868|374|1993|2|4|13|2|1993|374|4868|Tuesday|1993Q2|N|N|N|2449079|2449168|2448726|2449001|N|N|N|N|N| +2449092|AAAAAAAAEMOFFCAA|1993-04-14|1119|4868|374|1993|3|4|14|2|1993|374|4868|Wednesday|1993Q2|N|N|N|2449079|2449168|2448727|2449002|N|N|N|N|N| +2449093|AAAAAAAAFMOFFCAA|1993-04-15|1119|4868|374|1993|4|4|15|2|1993|374|4868|Thursday|1993Q2|N|N|N|2449079|2449168|2448728|2449003|N|N|N|N|N| +2449094|AAAAAAAAGMOFFCAA|1993-04-16|1119|4868|374|1993|5|4|16|2|1993|374|4868|Friday|1993Q2|N|Y|N|2449079|2449168|2448729|2449004|N|N|N|N|N| +2449095|AAAAAAAAHMOFFCAA|1993-04-17|1119|4868|374|1993|6|4|17|2|1993|374|4868|Saturday|1993Q2|N|Y|N|2449079|2449168|2448730|2449005|N|N|N|N|N| +2449096|AAAAAAAAIMOFFCAA|1993-04-18|1119|4868|374|1993|0|4|18|2|1993|374|4868|Sunday|1993Q2|N|N|N|2449079|2449168|2448731|2449006|N|N|N|N|N| +2449097|AAAAAAAAJMOFFCAA|1993-04-19|1119|4868|374|1993|1|4|19|2|1993|374|4868|Monday|1993Q2|N|N|N|2449079|2449168|2448732|2449007|N|N|N|N|N| +2449098|AAAAAAAAKMOFFCAA|1993-04-20|1119|4869|374|1993|2|4|20|2|1993|374|4869|Tuesday|1993Q2|N|N|N|2449079|2449168|2448733|2449008|N|N|N|N|N| +2449099|AAAAAAAALMOFFCAA|1993-04-21|1119|4869|374|1993|3|4|21|2|1993|374|4869|Wednesday|1993Q2|N|N|N|2449079|2449168|2448734|2449009|N|N|N|N|N| +2449100|AAAAAAAAMMOFFCAA|1993-04-22|1119|4869|374|1993|4|4|22|2|1993|374|4869|Thursday|1993Q2|N|N|N|2449079|2449168|2448735|2449010|N|N|N|N|N| +2449101|AAAAAAAANMOFFCAA|1993-04-23|1119|4869|374|1993|5|4|23|2|1993|374|4869|Friday|1993Q2|N|Y|N|2449079|2449168|2448736|2449011|N|N|N|N|N| +2449102|AAAAAAAAOMOFFCAA|1993-04-24|1119|4869|374|1993|6|4|24|2|1993|374|4869|Saturday|1993Q2|N|Y|N|2449079|2449168|2448737|2449012|N|N|N|N|N| +2449103|AAAAAAAAPMOFFCAA|1993-04-25|1119|4869|374|1993|0|4|25|2|1993|374|4869|Sunday|1993Q2|N|N|N|2449079|2449168|2448738|2449013|N|N|N|N|N| +2449104|AAAAAAAAANOFFCAA|1993-04-26|1119|4869|374|1993|1|4|26|2|1993|374|4869|Monday|1993Q2|N|N|N|2449079|2449168|2448739|2449014|N|N|N|N|N| +2449105|AAAAAAAABNOFFCAA|1993-04-27|1119|4870|374|1993|2|4|27|2|1993|374|4870|Tuesday|1993Q2|N|N|N|2449079|2449168|2448740|2449015|N|N|N|N|N| +2449106|AAAAAAAACNOFFCAA|1993-04-28|1119|4870|374|1993|3|4|28|2|1993|374|4870|Wednesday|1993Q2|N|N|N|2449079|2449168|2448741|2449016|N|N|N|N|N| +2449107|AAAAAAAADNOFFCAA|1993-04-29|1119|4870|374|1993|4|4|29|2|1993|374|4870|Thursday|1993Q2|N|N|N|2449079|2449168|2448742|2449017|N|N|N|N|N| +2449108|AAAAAAAAENOFFCAA|1993-04-30|1119|4870|374|1993|5|4|30|2|1993|374|4870|Friday|1993Q2|N|Y|N|2449079|2449168|2448743|2449018|N|N|N|N|N| +2449109|AAAAAAAAFNOFFCAA|1993-05-01|1120|4870|374|1993|6|5|1|2|1993|374|4870|Saturday|1993Q2|N|Y|N|2449109|2449228|2448744|2449019|N|N|N|N|N| +2449110|AAAAAAAAGNOFFCAA|1993-05-02|1120|4870|374|1993|0|5|2|2|1993|374|4870|Sunday|1993Q2|N|N|N|2449109|2449228|2448745|2449020|N|N|N|N|N| +2449111|AAAAAAAAHNOFFCAA|1993-05-03|1120|4870|374|1993|1|5|3|2|1993|374|4870|Monday|1993Q2|N|N|N|2449109|2449228|2448746|2449021|N|N|N|N|N| +2449112|AAAAAAAAINOFFCAA|1993-05-04|1120|4871|374|1993|2|5|4|2|1993|374|4871|Tuesday|1993Q2|N|N|N|2449109|2449228|2448747|2449022|N|N|N|N|N| +2449113|AAAAAAAAJNOFFCAA|1993-05-05|1120|4871|374|1993|3|5|5|2|1993|374|4871|Wednesday|1993Q2|N|N|N|2449109|2449228|2448748|2449023|N|N|N|N|N| +2449114|AAAAAAAAKNOFFCAA|1993-05-06|1120|4871|374|1993|4|5|6|2|1993|374|4871|Thursday|1993Q2|N|N|N|2449109|2449228|2448749|2449024|N|N|N|N|N| +2449115|AAAAAAAALNOFFCAA|1993-05-07|1120|4871|374|1993|5|5|7|2|1993|374|4871|Friday|1993Q2|N|Y|N|2449109|2449228|2448750|2449025|N|N|N|N|N| +2449116|AAAAAAAAMNOFFCAA|1993-05-08|1120|4871|374|1993|6|5|8|2|1993|374|4871|Saturday|1993Q2|N|Y|N|2449109|2449228|2448751|2449026|N|N|N|N|N| +2449117|AAAAAAAANNOFFCAA|1993-05-09|1120|4871|374|1993|0|5|9|2|1993|374|4871|Sunday|1993Q2|N|N|N|2449109|2449228|2448752|2449027|N|N|N|N|N| +2449118|AAAAAAAAONOFFCAA|1993-05-10|1120|4871|374|1993|1|5|10|2|1993|374|4871|Monday|1993Q2|N|N|N|2449109|2449228|2448753|2449028|N|N|N|N|N| +2449119|AAAAAAAAPNOFFCAA|1993-05-11|1120|4872|374|1993|2|5|11|2|1993|374|4872|Tuesday|1993Q2|N|N|N|2449109|2449228|2448754|2449029|N|N|N|N|N| +2449120|AAAAAAAAAOOFFCAA|1993-05-12|1120|4872|374|1993|3|5|12|2|1993|374|4872|Wednesday|1993Q2|N|N|N|2449109|2449228|2448755|2449030|N|N|N|N|N| +2449121|AAAAAAAABOOFFCAA|1993-05-13|1120|4872|374|1993|4|5|13|2|1993|374|4872|Thursday|1993Q2|N|N|N|2449109|2449228|2448756|2449031|N|N|N|N|N| +2449122|AAAAAAAACOOFFCAA|1993-05-14|1120|4872|374|1993|5|5|14|2|1993|374|4872|Friday|1993Q2|N|Y|N|2449109|2449228|2448757|2449032|N|N|N|N|N| +2449123|AAAAAAAADOOFFCAA|1993-05-15|1120|4872|374|1993|6|5|15|2|1993|374|4872|Saturday|1993Q2|N|Y|N|2449109|2449228|2448758|2449033|N|N|N|N|N| +2449124|AAAAAAAAEOOFFCAA|1993-05-16|1120|4872|374|1993|0|5|16|2|1993|374|4872|Sunday|1993Q2|N|N|N|2449109|2449228|2448759|2449034|N|N|N|N|N| +2449125|AAAAAAAAFOOFFCAA|1993-05-17|1120|4872|374|1993|1|5|17|2|1993|374|4872|Monday|1993Q2|N|N|N|2449109|2449228|2448760|2449035|N|N|N|N|N| +2449126|AAAAAAAAGOOFFCAA|1993-05-18|1120|4873|374|1993|2|5|18|2|1993|374|4873|Tuesday|1993Q2|N|N|N|2449109|2449228|2448761|2449036|N|N|N|N|N| +2449127|AAAAAAAAHOOFFCAA|1993-05-19|1120|4873|374|1993|3|5|19|2|1993|374|4873|Wednesday|1993Q2|N|N|N|2449109|2449228|2448762|2449037|N|N|N|N|N| +2449128|AAAAAAAAIOOFFCAA|1993-05-20|1120|4873|374|1993|4|5|20|2|1993|374|4873|Thursday|1993Q2|N|N|N|2449109|2449228|2448763|2449038|N|N|N|N|N| +2449129|AAAAAAAAJOOFFCAA|1993-05-21|1120|4873|374|1993|5|5|21|2|1993|374|4873|Friday|1993Q2|N|Y|N|2449109|2449228|2448764|2449039|N|N|N|N|N| +2449130|AAAAAAAAKOOFFCAA|1993-05-22|1120|4873|374|1993|6|5|22|2|1993|374|4873|Saturday|1993Q2|N|Y|N|2449109|2449228|2448765|2449040|N|N|N|N|N| +2449131|AAAAAAAALOOFFCAA|1993-05-23|1120|4873|374|1993|0|5|23|2|1993|374|4873|Sunday|1993Q2|N|N|N|2449109|2449228|2448766|2449041|N|N|N|N|N| +2449132|AAAAAAAAMOOFFCAA|1993-05-24|1120|4873|374|1993|1|5|24|2|1993|374|4873|Monday|1993Q2|N|N|N|2449109|2449228|2448767|2449042|N|N|N|N|N| +2449133|AAAAAAAANOOFFCAA|1993-05-25|1120|4874|374|1993|2|5|25|2|1993|374|4874|Tuesday|1993Q2|N|N|N|2449109|2449228|2448768|2449043|N|N|N|N|N| +2449134|AAAAAAAAOOOFFCAA|1993-05-26|1120|4874|374|1993|3|5|26|2|1993|374|4874|Wednesday|1993Q2|N|N|N|2449109|2449228|2448769|2449044|N|N|N|N|N| +2449135|AAAAAAAAPOOFFCAA|1993-05-27|1120|4874|374|1993|4|5|27|2|1993|374|4874|Thursday|1993Q2|N|N|N|2449109|2449228|2448770|2449045|N|N|N|N|N| +2449136|AAAAAAAAAPOFFCAA|1993-05-28|1120|4874|374|1993|5|5|28|2|1993|374|4874|Friday|1993Q2|N|Y|N|2449109|2449228|2448771|2449046|N|N|N|N|N| +2449137|AAAAAAAABPOFFCAA|1993-05-29|1120|4874|374|1993|6|5|29|2|1993|374|4874|Saturday|1993Q2|N|Y|N|2449109|2449228|2448772|2449047|N|N|N|N|N| +2449138|AAAAAAAACPOFFCAA|1993-05-30|1120|4874|374|1993|0|5|30|2|1993|374|4874|Sunday|1993Q2|N|N|N|2449109|2449228|2448773|2449048|N|N|N|N|N| +2449139|AAAAAAAADPOFFCAA|1993-05-31|1120|4874|374|1993|1|5|31|2|1993|374|4874|Monday|1993Q2|N|N|N|2449109|2449228|2448774|2449049|N|N|N|N|N| +2449140|AAAAAAAAEPOFFCAA|1993-06-01|1121|4875|375|1993|2|6|1|2|1993|375|4875|Tuesday|1993Q2|N|N|N|2449140|2449290|2448775|2449050|N|N|N|N|N| +2449141|AAAAAAAAFPOFFCAA|1993-06-02|1121|4875|375|1993|3|6|2|2|1993|375|4875|Wednesday|1993Q2|N|N|N|2449140|2449290|2448776|2449051|N|N|N|N|N| +2449142|AAAAAAAAGPOFFCAA|1993-06-03|1121|4875|375|1993|4|6|3|2|1993|375|4875|Thursday|1993Q2|N|N|N|2449140|2449290|2448777|2449052|N|N|N|N|N| +2449143|AAAAAAAAHPOFFCAA|1993-06-04|1121|4875|375|1993|5|6|4|2|1993|375|4875|Friday|1993Q2|N|Y|N|2449140|2449290|2448778|2449053|N|N|N|N|N| +2449144|AAAAAAAAIPOFFCAA|1993-06-05|1121|4875|375|1993|6|6|5|2|1993|375|4875|Saturday|1993Q2|N|Y|N|2449140|2449290|2448779|2449054|N|N|N|N|N| +2449145|AAAAAAAAJPOFFCAA|1993-06-06|1121|4875|375|1993|0|6|6|2|1993|375|4875|Sunday|1993Q2|N|N|N|2449140|2449290|2448780|2449055|N|N|N|N|N| +2449146|AAAAAAAAKPOFFCAA|1993-06-07|1121|4875|375|1993|1|6|7|2|1993|375|4875|Monday|1993Q2|N|N|N|2449140|2449290|2448781|2449056|N|N|N|N|N| +2449147|AAAAAAAALPOFFCAA|1993-06-08|1121|4876|375|1993|2|6|8|2|1993|375|4876|Tuesday|1993Q2|N|N|N|2449140|2449290|2448782|2449057|N|N|N|N|N| +2449148|AAAAAAAAMPOFFCAA|1993-06-09|1121|4876|375|1993|3|6|9|2|1993|375|4876|Wednesday|1993Q2|N|N|N|2449140|2449290|2448783|2449058|N|N|N|N|N| +2449149|AAAAAAAANPOFFCAA|1993-06-10|1121|4876|375|1993|4|6|10|2|1993|375|4876|Thursday|1993Q2|N|N|N|2449140|2449290|2448784|2449059|N|N|N|N|N| +2449150|AAAAAAAAOPOFFCAA|1993-06-11|1121|4876|375|1993|5|6|11|2|1993|375|4876|Friday|1993Q2|N|Y|N|2449140|2449290|2448785|2449060|N|N|N|N|N| +2449151|AAAAAAAAPPOFFCAA|1993-06-12|1121|4876|375|1993|6|6|12|2|1993|375|4876|Saturday|1993Q2|N|Y|N|2449140|2449290|2448786|2449061|N|N|N|N|N| +2449152|AAAAAAAAAAPFFCAA|1993-06-13|1121|4876|375|1993|0|6|13|2|1993|375|4876|Sunday|1993Q2|N|N|N|2449140|2449290|2448787|2449062|N|N|N|N|N| +2449153|AAAAAAAABAPFFCAA|1993-06-14|1121|4876|375|1993|1|6|14|2|1993|375|4876|Monday|1993Q2|N|N|N|2449140|2449290|2448788|2449063|N|N|N|N|N| +2449154|AAAAAAAACAPFFCAA|1993-06-15|1121|4877|375|1993|2|6|15|2|1993|375|4877|Tuesday|1993Q2|N|N|N|2449140|2449290|2448789|2449064|N|N|N|N|N| +2449155|AAAAAAAADAPFFCAA|1993-06-16|1121|4877|375|1993|3|6|16|2|1993|375|4877|Wednesday|1993Q2|N|N|N|2449140|2449290|2448790|2449065|N|N|N|N|N| +2449156|AAAAAAAAEAPFFCAA|1993-06-17|1121|4877|375|1993|4|6|17|2|1993|375|4877|Thursday|1993Q2|N|N|N|2449140|2449290|2448791|2449066|N|N|N|N|N| +2449157|AAAAAAAAFAPFFCAA|1993-06-18|1121|4877|375|1993|5|6|18|2|1993|375|4877|Friday|1993Q2|N|Y|N|2449140|2449290|2448792|2449067|N|N|N|N|N| +2449158|AAAAAAAAGAPFFCAA|1993-06-19|1121|4877|375|1993|6|6|19|2|1993|375|4877|Saturday|1993Q2|N|Y|N|2449140|2449290|2448793|2449068|N|N|N|N|N| +2449159|AAAAAAAAHAPFFCAA|1993-06-20|1121|4877|375|1993|0|6|20|2|1993|375|4877|Sunday|1993Q2|N|N|N|2449140|2449290|2448794|2449069|N|N|N|N|N| +2449160|AAAAAAAAIAPFFCAA|1993-06-21|1121|4877|375|1993|1|6|21|2|1993|375|4877|Monday|1993Q2|N|N|N|2449140|2449290|2448795|2449070|N|N|N|N|N| +2449161|AAAAAAAAJAPFFCAA|1993-06-22|1121|4878|375|1993|2|6|22|2|1993|375|4878|Tuesday|1993Q2|N|N|N|2449140|2449290|2448796|2449071|N|N|N|N|N| +2449162|AAAAAAAAKAPFFCAA|1993-06-23|1121|4878|375|1993|3|6|23|2|1993|375|4878|Wednesday|1993Q2|N|N|N|2449140|2449290|2448797|2449072|N|N|N|N|N| +2449163|AAAAAAAALAPFFCAA|1993-06-24|1121|4878|375|1993|4|6|24|2|1993|375|4878|Thursday|1993Q2|N|N|N|2449140|2449290|2448798|2449073|N|N|N|N|N| +2449164|AAAAAAAAMAPFFCAA|1993-06-25|1121|4878|375|1993|5|6|25|2|1993|375|4878|Friday|1993Q2|N|Y|N|2449140|2449290|2448799|2449074|N|N|N|N|N| +2449165|AAAAAAAANAPFFCAA|1993-06-26|1121|4878|375|1993|6|6|26|2|1993|375|4878|Saturday|1993Q2|N|Y|N|2449140|2449290|2448800|2449075|N|N|N|N|N| +2449166|AAAAAAAAOAPFFCAA|1993-06-27|1121|4878|375|1993|0|6|27|2|1993|375|4878|Sunday|1993Q2|N|N|N|2449140|2449290|2448801|2449076|N|N|N|N|N| +2449167|AAAAAAAAPAPFFCAA|1993-06-28|1121|4878|375|1993|1|6|28|2|1993|375|4878|Monday|1993Q2|N|N|N|2449140|2449290|2448802|2449077|N|N|N|N|N| +2449168|AAAAAAAAABPFFCAA|1993-06-29|1121|4879|375|1993|2|6|29|2|1993|375|4879|Tuesday|1993Q2|N|N|N|2449140|2449290|2448803|2449078|N|N|N|N|N| +2449169|AAAAAAAABBPFFCAA|1993-06-30|1121|4879|375|1993|3|6|30|2|1993|375|4879|Wednesday|1993Q2|N|N|N|2449140|2449290|2448804|2449079|N|N|N|N|N| +2449170|AAAAAAAACBPFFCAA|1993-07-01|1122|4879|375|1993|4|7|1|2|1993|375|4879|Thursday|1993Q2|N|N|N|2449170|2449350|2448805|2449079|N|N|N|N|N| +2449171|AAAAAAAADBPFFCAA|1993-07-02|1122|4879|375|1993|5|7|2|3|1993|375|4879|Friday|1993Q3|N|Y|N|2449170|2449350|2448806|2449080|N|N|N|N|N| +2449172|AAAAAAAAEBPFFCAA|1993-07-03|1122|4879|375|1993|6|7|3|3|1993|375|4879|Saturday|1993Q3|N|Y|N|2449170|2449350|2448807|2449081|N|N|N|N|N| +2449173|AAAAAAAAFBPFFCAA|1993-07-04|1122|4879|375|1993|0|7|4|3|1993|375|4879|Sunday|1993Q3|N|N|N|2449170|2449350|2448808|2449082|N|N|N|N|N| +2449174|AAAAAAAAGBPFFCAA|1993-07-05|1122|4879|375|1993|1|7|5|3|1993|375|4879|Monday|1993Q3|Y|N|N|2449170|2449350|2448809|2449083|N|N|N|N|N| +2449175|AAAAAAAAHBPFFCAA|1993-07-06|1122|4880|375|1993|2|7|6|3|1993|375|4880|Tuesday|1993Q3|N|N|Y|2449170|2449350|2448810|2449084|N|N|N|N|N| +2449176|AAAAAAAAIBPFFCAA|1993-07-07|1122|4880|375|1993|3|7|7|3|1993|375|4880|Wednesday|1993Q3|N|N|N|2449170|2449350|2448811|2449085|N|N|N|N|N| +2449177|AAAAAAAAJBPFFCAA|1993-07-08|1122|4880|375|1993|4|7|8|3|1993|375|4880|Thursday|1993Q3|N|N|N|2449170|2449350|2448812|2449086|N|N|N|N|N| +2449178|AAAAAAAAKBPFFCAA|1993-07-09|1122|4880|375|1993|5|7|9|3|1993|375|4880|Friday|1993Q3|N|Y|N|2449170|2449350|2448813|2449087|N|N|N|N|N| +2449179|AAAAAAAALBPFFCAA|1993-07-10|1122|4880|375|1993|6|7|10|3|1993|375|4880|Saturday|1993Q3|N|Y|N|2449170|2449350|2448814|2449088|N|N|N|N|N| +2449180|AAAAAAAAMBPFFCAA|1993-07-11|1122|4880|375|1993|0|7|11|3|1993|375|4880|Sunday|1993Q3|N|N|N|2449170|2449350|2448815|2449089|N|N|N|N|N| +2449181|AAAAAAAANBPFFCAA|1993-07-12|1122|4880|375|1993|1|7|12|3|1993|375|4880|Monday|1993Q3|N|N|N|2449170|2449350|2448816|2449090|N|N|N|N|N| +2449182|AAAAAAAAOBPFFCAA|1993-07-13|1122|4881|375|1993|2|7|13|3|1993|375|4881|Tuesday|1993Q3|N|N|N|2449170|2449350|2448817|2449091|N|N|N|N|N| +2449183|AAAAAAAAPBPFFCAA|1993-07-14|1122|4881|375|1993|3|7|14|3|1993|375|4881|Wednesday|1993Q3|N|N|N|2449170|2449350|2448818|2449092|N|N|N|N|N| +2449184|AAAAAAAAACPFFCAA|1993-07-15|1122|4881|375|1993|4|7|15|3|1993|375|4881|Thursday|1993Q3|N|N|N|2449170|2449350|2448819|2449093|N|N|N|N|N| +2449185|AAAAAAAABCPFFCAA|1993-07-16|1122|4881|375|1993|5|7|16|3|1993|375|4881|Friday|1993Q3|N|Y|N|2449170|2449350|2448820|2449094|N|N|N|N|N| +2449186|AAAAAAAACCPFFCAA|1993-07-17|1122|4881|375|1993|6|7|17|3|1993|375|4881|Saturday|1993Q3|N|Y|N|2449170|2449350|2448821|2449095|N|N|N|N|N| +2449187|AAAAAAAADCPFFCAA|1993-07-18|1122|4881|375|1993|0|7|18|3|1993|375|4881|Sunday|1993Q3|N|N|N|2449170|2449350|2448822|2449096|N|N|N|N|N| +2449188|AAAAAAAAECPFFCAA|1993-07-19|1122|4881|375|1993|1|7|19|3|1993|375|4881|Monday|1993Q3|N|N|N|2449170|2449350|2448823|2449097|N|N|N|N|N| +2449189|AAAAAAAAFCPFFCAA|1993-07-20|1122|4882|375|1993|2|7|20|3|1993|375|4882|Tuesday|1993Q3|N|N|N|2449170|2449350|2448824|2449098|N|N|N|N|N| +2449190|AAAAAAAAGCPFFCAA|1993-07-21|1122|4882|375|1993|3|7|21|3|1993|375|4882|Wednesday|1993Q3|N|N|N|2449170|2449350|2448825|2449099|N|N|N|N|N| +2449191|AAAAAAAAHCPFFCAA|1993-07-22|1122|4882|375|1993|4|7|22|3|1993|375|4882|Thursday|1993Q3|N|N|N|2449170|2449350|2448826|2449100|N|N|N|N|N| +2449192|AAAAAAAAICPFFCAA|1993-07-23|1122|4882|375|1993|5|7|23|3|1993|375|4882|Friday|1993Q3|N|Y|N|2449170|2449350|2448827|2449101|N|N|N|N|N| +2449193|AAAAAAAAJCPFFCAA|1993-07-24|1122|4882|375|1993|6|7|24|3|1993|375|4882|Saturday|1993Q3|N|Y|N|2449170|2449350|2448828|2449102|N|N|N|N|N| +2449194|AAAAAAAAKCPFFCAA|1993-07-25|1122|4882|375|1993|0|7|25|3|1993|375|4882|Sunday|1993Q3|N|N|N|2449170|2449350|2448829|2449103|N|N|N|N|N| +2449195|AAAAAAAALCPFFCAA|1993-07-26|1122|4882|375|1993|1|7|26|3|1993|375|4882|Monday|1993Q3|N|N|N|2449170|2449350|2448830|2449104|N|N|N|N|N| +2449196|AAAAAAAAMCPFFCAA|1993-07-27|1122|4883|375|1993|2|7|27|3|1993|375|4883|Tuesday|1993Q3|N|N|N|2449170|2449350|2448831|2449105|N|N|N|N|N| +2449197|AAAAAAAANCPFFCAA|1993-07-28|1122|4883|375|1993|3|7|28|3|1993|375|4883|Wednesday|1993Q3|N|N|N|2449170|2449350|2448832|2449106|N|N|N|N|N| +2449198|AAAAAAAAOCPFFCAA|1993-07-29|1122|4883|375|1993|4|7|29|3|1993|375|4883|Thursday|1993Q3|N|N|N|2449170|2449350|2448833|2449107|N|N|N|N|N| +2449199|AAAAAAAAPCPFFCAA|1993-07-30|1122|4883|375|1993|5|7|30|3|1993|375|4883|Friday|1993Q3|N|Y|N|2449170|2449350|2448834|2449108|N|N|N|N|N| +2449200|AAAAAAAAADPFFCAA|1993-07-31|1122|4883|375|1993|6|7|31|3|1993|375|4883|Saturday|1993Q3|N|Y|N|2449170|2449350|2448835|2449109|N|N|N|N|N| +2449201|AAAAAAAABDPFFCAA|1993-08-01|1123|4883|375|1993|0|8|1|3|1993|375|4883|Sunday|1993Q3|N|N|N|2449201|2449412|2448836|2449110|N|N|N|N|N| +2449202|AAAAAAAACDPFFCAA|1993-08-02|1123|4883|375|1993|1|8|2|3|1993|375|4883|Monday|1993Q3|N|N|N|2449201|2449412|2448837|2449111|N|N|N|N|N| +2449203|AAAAAAAADDPFFCAA|1993-08-03|1123|4884|375|1993|2|8|3|3|1993|375|4884|Tuesday|1993Q3|N|N|N|2449201|2449412|2448838|2449112|N|N|N|N|N| +2449204|AAAAAAAAEDPFFCAA|1993-08-04|1123|4884|375|1993|3|8|4|3|1993|375|4884|Wednesday|1993Q3|N|N|N|2449201|2449412|2448839|2449113|N|N|N|N|N| +2449205|AAAAAAAAFDPFFCAA|1993-08-05|1123|4884|375|1993|4|8|5|3|1993|375|4884|Thursday|1993Q3|N|N|N|2449201|2449412|2448840|2449114|N|N|N|N|N| +2449206|AAAAAAAAGDPFFCAA|1993-08-06|1123|4884|375|1993|5|8|6|3|1993|375|4884|Friday|1993Q3|N|Y|N|2449201|2449412|2448841|2449115|N|N|N|N|N| +2449207|AAAAAAAAHDPFFCAA|1993-08-07|1123|4884|375|1993|6|8|7|3|1993|375|4884|Saturday|1993Q3|N|Y|N|2449201|2449412|2448842|2449116|N|N|N|N|N| +2449208|AAAAAAAAIDPFFCAA|1993-08-08|1123|4884|375|1993|0|8|8|3|1993|375|4884|Sunday|1993Q3|N|N|N|2449201|2449412|2448843|2449117|N|N|N|N|N| +2449209|AAAAAAAAJDPFFCAA|1993-08-09|1123|4884|375|1993|1|8|9|3|1993|375|4884|Monday|1993Q3|N|N|N|2449201|2449412|2448844|2449118|N|N|N|N|N| +2449210|AAAAAAAAKDPFFCAA|1993-08-10|1123|4885|375|1993|2|8|10|3|1993|375|4885|Tuesday|1993Q3|N|N|N|2449201|2449412|2448845|2449119|N|N|N|N|N| +2449211|AAAAAAAALDPFFCAA|1993-08-11|1123|4885|375|1993|3|8|11|3|1993|375|4885|Wednesday|1993Q3|N|N|N|2449201|2449412|2448846|2449120|N|N|N|N|N| +2449212|AAAAAAAAMDPFFCAA|1993-08-12|1123|4885|375|1993|4|8|12|3|1993|375|4885|Thursday|1993Q3|N|N|N|2449201|2449412|2448847|2449121|N|N|N|N|N| +2449213|AAAAAAAANDPFFCAA|1993-08-13|1123|4885|375|1993|5|8|13|3|1993|375|4885|Friday|1993Q3|N|Y|N|2449201|2449412|2448848|2449122|N|N|N|N|N| +2449214|AAAAAAAAODPFFCAA|1993-08-14|1123|4885|375|1993|6|8|14|3|1993|375|4885|Saturday|1993Q3|N|Y|N|2449201|2449412|2448849|2449123|N|N|N|N|N| +2449215|AAAAAAAAPDPFFCAA|1993-08-15|1123|4885|375|1993|0|8|15|3|1993|375|4885|Sunday|1993Q3|N|N|N|2449201|2449412|2448850|2449124|N|N|N|N|N| +2449216|AAAAAAAAAEPFFCAA|1993-08-16|1123|4885|375|1993|1|8|16|3|1993|375|4885|Monday|1993Q3|N|N|N|2449201|2449412|2448851|2449125|N|N|N|N|N| +2449217|AAAAAAAABEPFFCAA|1993-08-17|1123|4886|375|1993|2|8|17|3|1993|375|4886|Tuesday|1993Q3|N|N|N|2449201|2449412|2448852|2449126|N|N|N|N|N| +2449218|AAAAAAAACEPFFCAA|1993-08-18|1123|4886|375|1993|3|8|18|3|1993|375|4886|Wednesday|1993Q3|N|N|N|2449201|2449412|2448853|2449127|N|N|N|N|N| +2449219|AAAAAAAADEPFFCAA|1993-08-19|1123|4886|375|1993|4|8|19|3|1993|375|4886|Thursday|1993Q3|N|N|N|2449201|2449412|2448854|2449128|N|N|N|N|N| +2449220|AAAAAAAAEEPFFCAA|1993-08-20|1123|4886|375|1993|5|8|20|3|1993|375|4886|Friday|1993Q3|N|Y|N|2449201|2449412|2448855|2449129|N|N|N|N|N| +2449221|AAAAAAAAFEPFFCAA|1993-08-21|1123|4886|375|1993|6|8|21|3|1993|375|4886|Saturday|1993Q3|N|Y|N|2449201|2449412|2448856|2449130|N|N|N|N|N| +2449222|AAAAAAAAGEPFFCAA|1993-08-22|1123|4886|375|1993|0|8|22|3|1993|375|4886|Sunday|1993Q3|N|N|N|2449201|2449412|2448857|2449131|N|N|N|N|N| +2449223|AAAAAAAAHEPFFCAA|1993-08-23|1123|4886|375|1993|1|8|23|3|1993|375|4886|Monday|1993Q3|N|N|N|2449201|2449412|2448858|2449132|N|N|N|N|N| +2449224|AAAAAAAAIEPFFCAA|1993-08-24|1123|4887|375|1993|2|8|24|3|1993|375|4887|Tuesday|1993Q3|N|N|N|2449201|2449412|2448859|2449133|N|N|N|N|N| +2449225|AAAAAAAAJEPFFCAA|1993-08-25|1123|4887|375|1993|3|8|25|3|1993|375|4887|Wednesday|1993Q3|N|N|N|2449201|2449412|2448860|2449134|N|N|N|N|N| +2449226|AAAAAAAAKEPFFCAA|1993-08-26|1123|4887|375|1993|4|8|26|3|1993|375|4887|Thursday|1993Q3|N|N|N|2449201|2449412|2448861|2449135|N|N|N|N|N| +2449227|AAAAAAAALEPFFCAA|1993-08-27|1123|4887|375|1993|5|8|27|3|1993|375|4887|Friday|1993Q3|N|Y|N|2449201|2449412|2448862|2449136|N|N|N|N|N| +2449228|AAAAAAAAMEPFFCAA|1993-08-28|1123|4887|375|1993|6|8|28|3|1993|375|4887|Saturday|1993Q3|N|Y|N|2449201|2449412|2448863|2449137|N|N|N|N|N| +2449229|AAAAAAAANEPFFCAA|1993-08-29|1123|4887|375|1993|0|8|29|3|1993|375|4887|Sunday|1993Q3|N|N|N|2449201|2449412|2448864|2449138|N|N|N|N|N| +2449230|AAAAAAAAOEPFFCAA|1993-08-30|1123|4887|375|1993|1|8|30|3|1993|375|4887|Monday|1993Q3|N|N|N|2449201|2449412|2448865|2449139|N|N|N|N|N| +2449231|AAAAAAAAPEPFFCAA|1993-08-31|1123|4888|375|1993|2|8|31|3|1993|375|4888|Tuesday|1993Q3|N|N|N|2449201|2449412|2448866|2449140|N|N|N|N|N| +2449232|AAAAAAAAAFPFFCAA|1993-09-01|1124|4888|376|1993|3|9|1|3|1993|376|4888|Wednesday|1993Q3|N|N|N|2449232|2449474|2448867|2449141|N|N|N|N|N| +2449233|AAAAAAAABFPFFCAA|1993-09-02|1124|4888|376|1993|4|9|2|3|1993|376|4888|Thursday|1993Q3|N|N|N|2449232|2449474|2448868|2449142|N|N|N|N|N| +2449234|AAAAAAAACFPFFCAA|1993-09-03|1124|4888|376|1993|5|9|3|3|1993|376|4888|Friday|1993Q3|N|Y|N|2449232|2449474|2448869|2449143|N|N|N|N|N| +2449235|AAAAAAAADFPFFCAA|1993-09-04|1124|4888|376|1993|6|9|4|3|1993|376|4888|Saturday|1993Q3|N|Y|N|2449232|2449474|2448870|2449144|N|N|N|N|N| +2449236|AAAAAAAAEFPFFCAA|1993-09-05|1124|4888|376|1993|0|9|5|3|1993|376|4888|Sunday|1993Q3|N|N|N|2449232|2449474|2448871|2449145|N|N|N|N|N| +2449237|AAAAAAAAFFPFFCAA|1993-09-06|1124|4888|376|1993|1|9|6|3|1993|376|4888|Monday|1993Q3|N|N|N|2449232|2449474|2448872|2449146|N|N|N|N|N| +2449238|AAAAAAAAGFPFFCAA|1993-09-07|1124|4889|376|1993|2|9|7|3|1993|376|4889|Tuesday|1993Q3|N|N|N|2449232|2449474|2448873|2449147|N|N|N|N|N| +2449239|AAAAAAAAHFPFFCAA|1993-09-08|1124|4889|376|1993|3|9|8|3|1993|376|4889|Wednesday|1993Q3|N|N|N|2449232|2449474|2448874|2449148|N|N|N|N|N| +2449240|AAAAAAAAIFPFFCAA|1993-09-09|1124|4889|376|1993|4|9|9|3|1993|376|4889|Thursday|1993Q3|N|N|N|2449232|2449474|2448875|2449149|N|N|N|N|N| +2449241|AAAAAAAAJFPFFCAA|1993-09-10|1124|4889|376|1993|5|9|10|3|1993|376|4889|Friday|1993Q3|N|Y|N|2449232|2449474|2448876|2449150|N|N|N|N|N| +2449242|AAAAAAAAKFPFFCAA|1993-09-11|1124|4889|376|1993|6|9|11|3|1993|376|4889|Saturday|1993Q3|N|Y|N|2449232|2449474|2448877|2449151|N|N|N|N|N| +2449243|AAAAAAAALFPFFCAA|1993-09-12|1124|4889|376|1993|0|9|12|3|1993|376|4889|Sunday|1993Q3|N|N|N|2449232|2449474|2448878|2449152|N|N|N|N|N| +2449244|AAAAAAAAMFPFFCAA|1993-09-13|1124|4889|376|1993|1|9|13|3|1993|376|4889|Monday|1993Q3|N|N|N|2449232|2449474|2448879|2449153|N|N|N|N|N| +2449245|AAAAAAAANFPFFCAA|1993-09-14|1124|4890|376|1993|2|9|14|3|1993|376|4890|Tuesday|1993Q3|N|N|N|2449232|2449474|2448880|2449154|N|N|N|N|N| +2449246|AAAAAAAAOFPFFCAA|1993-09-15|1124|4890|376|1993|3|9|15|3|1993|376|4890|Wednesday|1993Q3|N|N|N|2449232|2449474|2448881|2449155|N|N|N|N|N| +2449247|AAAAAAAAPFPFFCAA|1993-09-16|1124|4890|376|1993|4|9|16|3|1993|376|4890|Thursday|1993Q3|N|N|N|2449232|2449474|2448882|2449156|N|N|N|N|N| +2449248|AAAAAAAAAGPFFCAA|1993-09-17|1124|4890|376|1993|5|9|17|3|1993|376|4890|Friday|1993Q3|N|Y|N|2449232|2449474|2448883|2449157|N|N|N|N|N| +2449249|AAAAAAAABGPFFCAA|1993-09-18|1124|4890|376|1993|6|9|18|3|1993|376|4890|Saturday|1993Q3|N|Y|N|2449232|2449474|2448884|2449158|N|N|N|N|N| +2449250|AAAAAAAACGPFFCAA|1993-09-19|1124|4890|376|1993|0|9|19|3|1993|376|4890|Sunday|1993Q3|N|N|N|2449232|2449474|2448885|2449159|N|N|N|N|N| +2449251|AAAAAAAADGPFFCAA|1993-09-20|1124|4890|376|1993|1|9|20|3|1993|376|4890|Monday|1993Q3|N|N|N|2449232|2449474|2448886|2449160|N|N|N|N|N| +2449252|AAAAAAAAEGPFFCAA|1993-09-21|1124|4891|376|1993|2|9|21|3|1993|376|4891|Tuesday|1993Q3|N|N|N|2449232|2449474|2448887|2449161|N|N|N|N|N| +2449253|AAAAAAAAFGPFFCAA|1993-09-22|1124|4891|376|1993|3|9|22|3|1993|376|4891|Wednesday|1993Q3|N|N|N|2449232|2449474|2448888|2449162|N|N|N|N|N| +2449254|AAAAAAAAGGPFFCAA|1993-09-23|1124|4891|376|1993|4|9|23|3|1993|376|4891|Thursday|1993Q3|N|N|N|2449232|2449474|2448889|2449163|N|N|N|N|N| +2449255|AAAAAAAAHGPFFCAA|1993-09-24|1124|4891|376|1993|5|9|24|3|1993|376|4891|Friday|1993Q3|N|Y|N|2449232|2449474|2448890|2449164|N|N|N|N|N| +2449256|AAAAAAAAIGPFFCAA|1993-09-25|1124|4891|376|1993|6|9|25|3|1993|376|4891|Saturday|1993Q3|N|Y|N|2449232|2449474|2448891|2449165|N|N|N|N|N| +2449257|AAAAAAAAJGPFFCAA|1993-09-26|1124|4891|376|1993|0|9|26|3|1993|376|4891|Sunday|1993Q3|N|N|N|2449232|2449474|2448892|2449166|N|N|N|N|N| +2449258|AAAAAAAAKGPFFCAA|1993-09-27|1124|4891|376|1993|1|9|27|3|1993|376|4891|Monday|1993Q3|N|N|N|2449232|2449474|2448893|2449167|N|N|N|N|N| +2449259|AAAAAAAALGPFFCAA|1993-09-28|1124|4892|376|1993|2|9|28|3|1993|376|4892|Tuesday|1993Q3|N|N|N|2449232|2449474|2448894|2449168|N|N|N|N|N| +2449260|AAAAAAAAMGPFFCAA|1993-09-29|1124|4892|376|1993|3|9|29|3|1993|376|4892|Wednesday|1993Q3|N|N|N|2449232|2449474|2448895|2449169|N|N|N|N|N| +2449261|AAAAAAAANGPFFCAA|1993-09-30|1124|4892|376|1993|4|9|30|3|1993|376|4892|Thursday|1993Q3|N|N|N|2449232|2449474|2448896|2449170|N|N|N|N|N| +2449262|AAAAAAAAOGPFFCAA|1993-10-01|1125|4892|376|1993|5|10|1|3|1993|376|4892|Friday|1993Q3|N|Y|N|2449262|2449534|2448897|2449170|N|N|N|N|N| +2449263|AAAAAAAAPGPFFCAA|1993-10-02|1125|4892|376|1993|6|10|2|4|1993|376|4892|Saturday|1993Q4|N|Y|N|2449262|2449534|2448898|2449171|N|N|N|N|N| +2449264|AAAAAAAAAHPFFCAA|1993-10-03|1125|4892|376|1993|0|10|3|4|1993|376|4892|Sunday|1993Q4|N|N|N|2449262|2449534|2448899|2449172|N|N|N|N|N| +2449265|AAAAAAAABHPFFCAA|1993-10-04|1125|4892|376|1993|1|10|4|4|1993|376|4892|Monday|1993Q4|N|N|N|2449262|2449534|2448900|2449173|N|N|N|N|N| +2449266|AAAAAAAACHPFFCAA|1993-10-05|1125|4893|376|1993|2|10|5|4|1993|376|4893|Tuesday|1993Q4|N|N|N|2449262|2449534|2448901|2449174|N|N|N|N|N| +2449267|AAAAAAAADHPFFCAA|1993-10-06|1125|4893|376|1993|3|10|6|4|1993|376|4893|Wednesday|1993Q4|N|N|N|2449262|2449534|2448902|2449175|N|N|N|N|N| +2449268|AAAAAAAAEHPFFCAA|1993-10-07|1125|4893|376|1993|4|10|7|4|1993|376|4893|Thursday|1993Q4|N|N|N|2449262|2449534|2448903|2449176|N|N|N|N|N| +2449269|AAAAAAAAFHPFFCAA|1993-10-08|1125|4893|376|1993|5|10|8|4|1993|376|4893|Friday|1993Q4|N|Y|N|2449262|2449534|2448904|2449177|N|N|N|N|N| +2449270|AAAAAAAAGHPFFCAA|1993-10-09|1125|4893|376|1993|6|10|9|4|1993|376|4893|Saturday|1993Q4|N|Y|N|2449262|2449534|2448905|2449178|N|N|N|N|N| +2449271|AAAAAAAAHHPFFCAA|1993-10-10|1125|4893|376|1993|0|10|10|4|1993|376|4893|Sunday|1993Q4|N|N|N|2449262|2449534|2448906|2449179|N|N|N|N|N| +2449272|AAAAAAAAIHPFFCAA|1993-10-11|1125|4893|376|1993|1|10|11|4|1993|376|4893|Monday|1993Q4|N|N|N|2449262|2449534|2448907|2449180|N|N|N|N|N| +2449273|AAAAAAAAJHPFFCAA|1993-10-12|1125|4894|376|1993|2|10|12|4|1993|376|4894|Tuesday|1993Q4|N|N|N|2449262|2449534|2448908|2449181|N|N|N|N|N| +2449274|AAAAAAAAKHPFFCAA|1993-10-13|1125|4894|376|1993|3|10|13|4|1993|376|4894|Wednesday|1993Q4|N|N|N|2449262|2449534|2448909|2449182|N|N|N|N|N| +2449275|AAAAAAAALHPFFCAA|1993-10-14|1125|4894|376|1993|4|10|14|4|1993|376|4894|Thursday|1993Q4|N|N|N|2449262|2449534|2448910|2449183|N|N|N|N|N| +2449276|AAAAAAAAMHPFFCAA|1993-10-15|1125|4894|376|1993|5|10|15|4|1993|376|4894|Friday|1993Q4|N|Y|N|2449262|2449534|2448911|2449184|N|N|N|N|N| +2449277|AAAAAAAANHPFFCAA|1993-10-16|1125|4894|376|1993|6|10|16|4|1993|376|4894|Saturday|1993Q4|N|Y|N|2449262|2449534|2448912|2449185|N|N|N|N|N| +2449278|AAAAAAAAOHPFFCAA|1993-10-17|1125|4894|376|1993|0|10|17|4|1993|376|4894|Sunday|1993Q4|N|N|N|2449262|2449534|2448913|2449186|N|N|N|N|N| +2449279|AAAAAAAAPHPFFCAA|1993-10-18|1125|4894|376|1993|1|10|18|4|1993|376|4894|Monday|1993Q4|N|N|N|2449262|2449534|2448914|2449187|N|N|N|N|N| +2449280|AAAAAAAAAIPFFCAA|1993-10-19|1125|4895|376|1993|2|10|19|4|1993|376|4895|Tuesday|1993Q4|N|N|N|2449262|2449534|2448915|2449188|N|N|N|N|N| +2449281|AAAAAAAABIPFFCAA|1993-10-20|1125|4895|376|1993|3|10|20|4|1993|376|4895|Wednesday|1993Q4|N|N|N|2449262|2449534|2448916|2449189|N|N|N|N|N| +2449282|AAAAAAAACIPFFCAA|1993-10-21|1125|4895|376|1993|4|10|21|4|1993|376|4895|Thursday|1993Q4|N|N|N|2449262|2449534|2448917|2449190|N|N|N|N|N| +2449283|AAAAAAAADIPFFCAA|1993-10-22|1125|4895|376|1993|5|10|22|4|1993|376|4895|Friday|1993Q4|N|Y|N|2449262|2449534|2448918|2449191|N|N|N|N|N| +2449284|AAAAAAAAEIPFFCAA|1993-10-23|1125|4895|376|1993|6|10|23|4|1993|376|4895|Saturday|1993Q4|N|Y|N|2449262|2449534|2448919|2449192|N|N|N|N|N| +2449285|AAAAAAAAFIPFFCAA|1993-10-24|1125|4895|376|1993|0|10|24|4|1993|376|4895|Sunday|1993Q4|N|N|N|2449262|2449534|2448920|2449193|N|N|N|N|N| +2449286|AAAAAAAAGIPFFCAA|1993-10-25|1125|4895|376|1993|1|10|25|4|1993|376|4895|Monday|1993Q4|N|N|N|2449262|2449534|2448921|2449194|N|N|N|N|N| +2449287|AAAAAAAAHIPFFCAA|1993-10-26|1125|4896|376|1993|2|10|26|4|1993|376|4896|Tuesday|1993Q4|N|N|N|2449262|2449534|2448922|2449195|N|N|N|N|N| +2449288|AAAAAAAAIIPFFCAA|1993-10-27|1125|4896|376|1993|3|10|27|4|1993|376|4896|Wednesday|1993Q4|N|N|N|2449262|2449534|2448923|2449196|N|N|N|N|N| +2449289|AAAAAAAAJIPFFCAA|1993-10-28|1125|4896|376|1993|4|10|28|4|1993|376|4896|Thursday|1993Q4|N|N|N|2449262|2449534|2448924|2449197|N|N|N|N|N| +2449290|AAAAAAAAKIPFFCAA|1993-10-29|1125|4896|376|1993|5|10|29|4|1993|376|4896|Friday|1993Q4|N|Y|N|2449262|2449534|2448925|2449198|N|N|N|N|N| +2449291|AAAAAAAALIPFFCAA|1993-10-30|1125|4896|376|1993|6|10|30|4|1993|376|4896|Saturday|1993Q4|N|Y|N|2449262|2449534|2448926|2449199|N|N|N|N|N| +2449292|AAAAAAAAMIPFFCAA|1993-10-31|1125|4896|376|1993|0|10|31|4|1993|376|4896|Sunday|1993Q4|N|N|N|2449262|2449534|2448927|2449200|N|N|N|N|N| +2449293|AAAAAAAANIPFFCAA|1993-11-01|1126|4896|376|1993|1|11|1|4|1993|376|4896|Monday|1993Q4|N|N|N|2449293|2449596|2448928|2449201|N|N|N|N|N| +2449294|AAAAAAAAOIPFFCAA|1993-11-02|1126|4897|376|1993|2|11|2|4|1993|376|4897|Tuesday|1993Q4|N|N|N|2449293|2449596|2448929|2449202|N|N|N|N|N| +2449295|AAAAAAAAPIPFFCAA|1993-11-03|1126|4897|376|1993|3|11|3|4|1993|376|4897|Wednesday|1993Q4|N|N|N|2449293|2449596|2448930|2449203|N|N|N|N|N| +2449296|AAAAAAAAAJPFFCAA|1993-11-04|1126|4897|376|1993|4|11|4|4|1993|376|4897|Thursday|1993Q4|N|N|N|2449293|2449596|2448931|2449204|N|N|N|N|N| +2449297|AAAAAAAABJPFFCAA|1993-11-05|1126|4897|376|1993|5|11|5|4|1993|376|4897|Friday|1993Q4|N|Y|N|2449293|2449596|2448932|2449205|N|N|N|N|N| +2449298|AAAAAAAACJPFFCAA|1993-11-06|1126|4897|376|1993|6|11|6|4|1993|376|4897|Saturday|1993Q4|N|Y|N|2449293|2449596|2448933|2449206|N|N|N|N|N| +2449299|AAAAAAAADJPFFCAA|1993-11-07|1126|4897|376|1993|0|11|7|4|1993|376|4897|Sunday|1993Q4|N|N|N|2449293|2449596|2448934|2449207|N|N|N|N|N| +2449300|AAAAAAAAEJPFFCAA|1993-11-08|1126|4897|376|1993|1|11|8|4|1993|376|4897|Monday|1993Q4|N|N|N|2449293|2449596|2448935|2449208|N|N|N|N|N| +2449301|AAAAAAAAFJPFFCAA|1993-11-09|1126|4898|376|1993|2|11|9|4|1993|376|4898|Tuesday|1993Q4|N|N|N|2449293|2449596|2448936|2449209|N|N|N|N|N| +2449302|AAAAAAAAGJPFFCAA|1993-11-10|1126|4898|376|1993|3|11|10|4|1993|376|4898|Wednesday|1993Q4|N|N|N|2449293|2449596|2448937|2449210|N|N|N|N|N| +2449303|AAAAAAAAHJPFFCAA|1993-11-11|1126|4898|376|1993|4|11|11|4|1993|376|4898|Thursday|1993Q4|N|N|N|2449293|2449596|2448938|2449211|N|N|N|N|N| +2449304|AAAAAAAAIJPFFCAA|1993-11-12|1126|4898|376|1993|5|11|12|4|1993|376|4898|Friday|1993Q4|N|Y|N|2449293|2449596|2448939|2449212|N|N|N|N|N| +2449305|AAAAAAAAJJPFFCAA|1993-11-13|1126|4898|376|1993|6|11|13|4|1993|376|4898|Saturday|1993Q4|N|Y|N|2449293|2449596|2448940|2449213|N|N|N|N|N| +2449306|AAAAAAAAKJPFFCAA|1993-11-14|1126|4898|376|1993|0|11|14|4|1993|376|4898|Sunday|1993Q4|N|N|N|2449293|2449596|2448941|2449214|N|N|N|N|N| +2449307|AAAAAAAALJPFFCAA|1993-11-15|1126|4898|376|1993|1|11|15|4|1993|376|4898|Monday|1993Q4|N|N|N|2449293|2449596|2448942|2449215|N|N|N|N|N| +2449308|AAAAAAAAMJPFFCAA|1993-11-16|1126|4899|376|1993|2|11|16|4|1993|376|4899|Tuesday|1993Q4|N|N|N|2449293|2449596|2448943|2449216|N|N|N|N|N| +2449309|AAAAAAAANJPFFCAA|1993-11-17|1126|4899|376|1993|3|11|17|4|1993|376|4899|Wednesday|1993Q4|N|N|N|2449293|2449596|2448944|2449217|N|N|N|N|N| +2449310|AAAAAAAAOJPFFCAA|1993-11-18|1126|4899|376|1993|4|11|18|4|1993|376|4899|Thursday|1993Q4|N|N|N|2449293|2449596|2448945|2449218|N|N|N|N|N| +2449311|AAAAAAAAPJPFFCAA|1993-11-19|1126|4899|376|1993|5|11|19|4|1993|376|4899|Friday|1993Q4|N|Y|N|2449293|2449596|2448946|2449219|N|N|N|N|N| +2449312|AAAAAAAAAKPFFCAA|1993-11-20|1126|4899|376|1993|6|11|20|4|1993|376|4899|Saturday|1993Q4|N|Y|N|2449293|2449596|2448947|2449220|N|N|N|N|N| +2449313|AAAAAAAABKPFFCAA|1993-11-21|1126|4899|376|1993|0|11|21|4|1993|376|4899|Sunday|1993Q4|N|N|N|2449293|2449596|2448948|2449221|N|N|N|N|N| +2449314|AAAAAAAACKPFFCAA|1993-11-22|1126|4899|376|1993|1|11|22|4|1993|376|4899|Monday|1993Q4|N|N|N|2449293|2449596|2448949|2449222|N|N|N|N|N| +2449315|AAAAAAAADKPFFCAA|1993-11-23|1126|4900|376|1993|2|11|23|4|1993|376|4900|Tuesday|1993Q4|N|N|N|2449293|2449596|2448950|2449223|N|N|N|N|N| +2449316|AAAAAAAAEKPFFCAA|1993-11-24|1126|4900|376|1993|3|11|24|4|1993|376|4900|Wednesday|1993Q4|N|N|N|2449293|2449596|2448951|2449224|N|N|N|N|N| +2449317|AAAAAAAAFKPFFCAA|1993-11-25|1126|4900|376|1993|4|11|25|4|1993|376|4900|Thursday|1993Q4|N|N|N|2449293|2449596|2448952|2449225|N|N|N|N|N| +2449318|AAAAAAAAGKPFFCAA|1993-11-26|1126|4900|376|1993|5|11|26|4|1993|376|4900|Friday|1993Q4|N|Y|N|2449293|2449596|2448953|2449226|N|N|N|N|N| +2449319|AAAAAAAAHKPFFCAA|1993-11-27|1126|4900|376|1993|6|11|27|4|1993|376|4900|Saturday|1993Q4|N|Y|N|2449293|2449596|2448954|2449227|N|N|N|N|N| +2449320|AAAAAAAAIKPFFCAA|1993-11-28|1126|4900|376|1993|0|11|28|4|1993|376|4900|Sunday|1993Q4|N|N|N|2449293|2449596|2448955|2449228|N|N|N|N|N| +2449321|AAAAAAAAJKPFFCAA|1993-11-29|1126|4900|376|1993|1|11|29|4|1993|376|4900|Monday|1993Q4|N|N|N|2449293|2449596|2448956|2449229|N|N|N|N|N| +2449322|AAAAAAAAKKPFFCAA|1993-11-30|1126|4901|376|1993|2|11|30|4|1993|376|4901|Tuesday|1993Q4|N|N|N|2449293|2449596|2448957|2449230|N|N|N|N|N| +2449323|AAAAAAAALKPFFCAA|1993-12-01|1127|4901|377|1993|3|12|1|4|1993|377|4901|Wednesday|1993Q4|N|N|N|2449323|2449656|2448958|2449231|N|N|N|N|N| +2449324|AAAAAAAAMKPFFCAA|1993-12-02|1127|4901|377|1993|4|12|2|4|1993|377|4901|Thursday|1993Q4|N|N|N|2449323|2449656|2448959|2449232|N|N|N|N|N| +2449325|AAAAAAAANKPFFCAA|1993-12-03|1127|4901|377|1993|5|12|3|4|1993|377|4901|Friday|1993Q4|N|Y|N|2449323|2449656|2448960|2449233|N|N|N|N|N| +2449326|AAAAAAAAOKPFFCAA|1993-12-04|1127|4901|377|1993|6|12|4|4|1993|377|4901|Saturday|1993Q4|N|Y|N|2449323|2449656|2448961|2449234|N|N|N|N|N| +2449327|AAAAAAAAPKPFFCAA|1993-12-05|1127|4901|377|1993|0|12|5|4|1993|377|4901|Sunday|1993Q4|N|N|N|2449323|2449656|2448962|2449235|N|N|N|N|N| +2449328|AAAAAAAAALPFFCAA|1993-12-06|1127|4901|377|1993|1|12|6|4|1993|377|4901|Monday|1993Q4|N|N|N|2449323|2449656|2448963|2449236|N|N|N|N|N| +2449329|AAAAAAAABLPFFCAA|1993-12-07|1127|4902|377|1993|2|12|7|4|1993|377|4902|Tuesday|1993Q4|N|N|N|2449323|2449656|2448964|2449237|N|N|N|N|N| +2449330|AAAAAAAACLPFFCAA|1993-12-08|1127|4902|377|1993|3|12|8|4|1993|377|4902|Wednesday|1993Q4|N|N|N|2449323|2449656|2448965|2449238|N|N|N|N|N| +2449331|AAAAAAAADLPFFCAA|1993-12-09|1127|4902|377|1993|4|12|9|4|1993|377|4902|Thursday|1993Q4|N|N|N|2449323|2449656|2448966|2449239|N|N|N|N|N| +2449332|AAAAAAAAELPFFCAA|1993-12-10|1127|4902|377|1993|5|12|10|4|1993|377|4902|Friday|1993Q4|N|Y|N|2449323|2449656|2448967|2449240|N|N|N|N|N| +2449333|AAAAAAAAFLPFFCAA|1993-12-11|1127|4902|377|1993|6|12|11|4|1993|377|4902|Saturday|1993Q4|N|Y|N|2449323|2449656|2448968|2449241|N|N|N|N|N| +2449334|AAAAAAAAGLPFFCAA|1993-12-12|1127|4902|377|1993|0|12|12|4|1993|377|4902|Sunday|1993Q4|N|N|N|2449323|2449656|2448969|2449242|N|N|N|N|N| +2449335|AAAAAAAAHLPFFCAA|1993-12-13|1127|4902|377|1993|1|12|13|4|1993|377|4902|Monday|1993Q4|N|N|N|2449323|2449656|2448970|2449243|N|N|N|N|N| +2449336|AAAAAAAAILPFFCAA|1993-12-14|1127|4903|377|1993|2|12|14|4|1993|377|4903|Tuesday|1993Q4|N|N|N|2449323|2449656|2448971|2449244|N|N|N|N|N| +2449337|AAAAAAAAJLPFFCAA|1993-12-15|1127|4903|377|1993|3|12|15|4|1993|377|4903|Wednesday|1993Q4|N|N|N|2449323|2449656|2448972|2449245|N|N|N|N|N| +2449338|AAAAAAAAKLPFFCAA|1993-12-16|1127|4903|377|1993|4|12|16|4|1993|377|4903|Thursday|1993Q4|N|N|N|2449323|2449656|2448973|2449246|N|N|N|N|N| +2449339|AAAAAAAALLPFFCAA|1993-12-17|1127|4903|377|1993|5|12|17|4|1993|377|4903|Friday|1993Q4|N|Y|N|2449323|2449656|2448974|2449247|N|N|N|N|N| +2449340|AAAAAAAAMLPFFCAA|1993-12-18|1127|4903|377|1993|6|12|18|4|1993|377|4903|Saturday|1993Q4|N|Y|N|2449323|2449656|2448975|2449248|N|N|N|N|N| +2449341|AAAAAAAANLPFFCAA|1993-12-19|1127|4903|377|1993|0|12|19|4|1993|377|4903|Sunday|1993Q4|N|N|N|2449323|2449656|2448976|2449249|N|N|N|N|N| +2449342|AAAAAAAAOLPFFCAA|1993-12-20|1127|4903|377|1993|1|12|20|4|1993|377|4903|Monday|1993Q4|N|N|N|2449323|2449656|2448977|2449250|N|N|N|N|N| +2449343|AAAAAAAAPLPFFCAA|1993-12-21|1127|4904|377|1993|2|12|21|4|1993|377|4904|Tuesday|1993Q4|N|N|N|2449323|2449656|2448978|2449251|N|N|N|N|N| +2449344|AAAAAAAAAMPFFCAA|1993-12-22|1127|4904|377|1993|3|12|22|4|1993|377|4904|Wednesday|1993Q4|N|N|N|2449323|2449656|2448979|2449252|N|N|N|N|N| +2449345|AAAAAAAABMPFFCAA|1993-12-23|1127|4904|377|1993|4|12|23|4|1993|377|4904|Thursday|1993Q4|N|N|N|2449323|2449656|2448980|2449253|N|N|N|N|N| +2449346|AAAAAAAACMPFFCAA|1993-12-24|1127|4904|377|1993|5|12|24|4|1993|377|4904|Friday|1993Q4|N|Y|N|2449323|2449656|2448981|2449254|N|N|N|N|N| +2449347|AAAAAAAADMPFFCAA|1993-12-25|1127|4904|377|1993|6|12|25|4|1993|377|4904|Saturday|1993Q4|N|Y|N|2449323|2449656|2448982|2449255|N|N|N|N|N| +2449348|AAAAAAAAEMPFFCAA|1993-12-26|1127|4904|377|1993|0|12|26|4|1993|377|4904|Sunday|1993Q4|Y|N|N|2449323|2449656|2448983|2449256|N|N|N|N|N| +2449349|AAAAAAAAFMPFFCAA|1993-12-27|1127|4904|377|1993|1|12|27|4|1993|377|4904|Monday|1993Q4|N|N|Y|2449323|2449656|2448984|2449257|N|N|N|N|N| +2449350|AAAAAAAAGMPFFCAA|1993-12-28|1127|4905|377|1993|2|12|28|4|1993|377|4905|Tuesday|1993Q4|N|N|N|2449323|2449656|2448985|2449258|N|N|N|N|N| +2449351|AAAAAAAAHMPFFCAA|1993-12-29|1127|4905|377|1993|3|12|29|4|1993|377|4905|Wednesday|1993Q4|N|N|N|2449323|2449656|2448986|2449259|N|N|N|N|N| +2449352|AAAAAAAAIMPFFCAA|1993-12-30|1127|4905|377|1993|4|12|30|4|1993|377|4905|Thursday|1993Q4|N|N|N|2449323|2449656|2448987|2449260|N|N|N|N|N| +2449353|AAAAAAAAJMPFFCAA|1993-12-31|1127|4905|377|1993|5|12|31|4|1993|377|4905|Friday|1993Q4|N|Y|N|2449323|2449656|2448988|2449261|N|N|N|N|N| +2449354|AAAAAAAAKMPFFCAA|1994-01-01|1128|4905|377|1994|6|1|1|1|1994|377|4905|Saturday|1994Q1|Y|Y|N|2449354|2449353|2448989|2449262|N|N|N|N|N| +2449355|AAAAAAAALMPFFCAA|1994-01-02|1128|4905|377|1994|0|1|2|1|1994|377|4905|Sunday|1994Q1|N|N|Y|2449354|2449353|2448990|2449263|N|N|N|N|N| +2449356|AAAAAAAAMMPFFCAA|1994-01-03|1128|4905|377|1994|1|1|3|1|1994|377|4905|Monday|1994Q1|N|N|N|2449354|2449353|2448991|2449264|N|N|N|N|N| +2449357|AAAAAAAANMPFFCAA|1994-01-04|1128|4906|377|1994|2|1|4|1|1994|377|4906|Tuesday|1994Q1|N|N|N|2449354|2449353|2448992|2449265|N|N|N|N|N| +2449358|AAAAAAAAOMPFFCAA|1994-01-05|1128|4906|377|1994|3|1|5|1|1994|377|4906|Wednesday|1994Q1|N|N|N|2449354|2449353|2448993|2449266|N|N|N|N|N| +2449359|AAAAAAAAPMPFFCAA|1994-01-06|1128|4906|377|1994|4|1|6|1|1994|377|4906|Thursday|1994Q1|N|N|N|2449354|2449353|2448994|2449267|N|N|N|N|N| +2449360|AAAAAAAAANPFFCAA|1994-01-07|1128|4906|377|1994|5|1|7|1|1994|377|4906|Friday|1994Q1|N|Y|N|2449354|2449353|2448995|2449268|N|N|N|N|N| +2449361|AAAAAAAABNPFFCAA|1994-01-08|1128|4906|377|1994|6|1|8|1|1994|377|4906|Saturday|1994Q1|N|Y|N|2449354|2449353|2448996|2449269|N|N|N|N|N| +2449362|AAAAAAAACNPFFCAA|1994-01-09|1128|4906|377|1994|0|1|9|1|1994|377|4906|Sunday|1994Q1|N|N|N|2449354|2449353|2448997|2449270|N|N|N|N|N| +2449363|AAAAAAAADNPFFCAA|1994-01-10|1128|4906|377|1994|1|1|10|1|1994|377|4906|Monday|1994Q1|N|N|N|2449354|2449353|2448998|2449271|N|N|N|N|N| +2449364|AAAAAAAAENPFFCAA|1994-01-11|1128|4907|377|1994|2|1|11|1|1994|377|4907|Tuesday|1994Q1|N|N|N|2449354|2449353|2448999|2449272|N|N|N|N|N| +2449365|AAAAAAAAFNPFFCAA|1994-01-12|1128|4907|377|1994|3|1|12|1|1994|377|4907|Wednesday|1994Q1|N|N|N|2449354|2449353|2449000|2449273|N|N|N|N|N| +2449366|AAAAAAAAGNPFFCAA|1994-01-13|1128|4907|377|1994|4|1|13|1|1994|377|4907|Thursday|1994Q1|N|N|N|2449354|2449353|2449001|2449274|N|N|N|N|N| +2449367|AAAAAAAAHNPFFCAA|1994-01-14|1128|4907|377|1994|5|1|14|1|1994|377|4907|Friday|1994Q1|N|Y|N|2449354|2449353|2449002|2449275|N|N|N|N|N| +2449368|AAAAAAAAINPFFCAA|1994-01-15|1128|4907|377|1994|6|1|15|1|1994|377|4907|Saturday|1994Q1|N|Y|N|2449354|2449353|2449003|2449276|N|N|N|N|N| +2449369|AAAAAAAAJNPFFCAA|1994-01-16|1128|4907|377|1994|0|1|16|1|1994|377|4907|Sunday|1994Q1|N|N|N|2449354|2449353|2449004|2449277|N|N|N|N|N| +2449370|AAAAAAAAKNPFFCAA|1994-01-17|1128|4907|377|1994|1|1|17|1|1994|377|4907|Monday|1994Q1|N|N|N|2449354|2449353|2449005|2449278|N|N|N|N|N| +2449371|AAAAAAAALNPFFCAA|1994-01-18|1128|4908|377|1994|2|1|18|1|1994|377|4908|Tuesday|1994Q1|N|N|N|2449354|2449353|2449006|2449279|N|N|N|N|N| +2449372|AAAAAAAAMNPFFCAA|1994-01-19|1128|4908|377|1994|3|1|19|1|1994|377|4908|Wednesday|1994Q1|N|N|N|2449354|2449353|2449007|2449280|N|N|N|N|N| +2449373|AAAAAAAANNPFFCAA|1994-01-20|1128|4908|377|1994|4|1|20|1|1994|377|4908|Thursday|1994Q1|N|N|N|2449354|2449353|2449008|2449281|N|N|N|N|N| +2449374|AAAAAAAAONPFFCAA|1994-01-21|1128|4908|377|1994|5|1|21|1|1994|377|4908|Friday|1994Q1|N|Y|N|2449354|2449353|2449009|2449282|N|N|N|N|N| +2449375|AAAAAAAAPNPFFCAA|1994-01-22|1128|4908|377|1994|6|1|22|1|1994|377|4908|Saturday|1994Q1|N|Y|N|2449354|2449353|2449010|2449283|N|N|N|N|N| +2449376|AAAAAAAAAOPFFCAA|1994-01-23|1128|4908|377|1994|0|1|23|1|1994|377|4908|Sunday|1994Q1|N|N|N|2449354|2449353|2449011|2449284|N|N|N|N|N| +2449377|AAAAAAAABOPFFCAA|1994-01-24|1128|4908|377|1994|1|1|24|1|1994|377|4908|Monday|1994Q1|N|N|N|2449354|2449353|2449012|2449285|N|N|N|N|N| +2449378|AAAAAAAACOPFFCAA|1994-01-25|1128|4909|377|1994|2|1|25|1|1994|377|4909|Tuesday|1994Q1|N|N|N|2449354|2449353|2449013|2449286|N|N|N|N|N| +2449379|AAAAAAAADOPFFCAA|1994-01-26|1128|4909|377|1994|3|1|26|1|1994|377|4909|Wednesday|1994Q1|N|N|N|2449354|2449353|2449014|2449287|N|N|N|N|N| +2449380|AAAAAAAAEOPFFCAA|1994-01-27|1128|4909|377|1994|4|1|27|1|1994|377|4909|Thursday|1994Q1|N|N|N|2449354|2449353|2449015|2449288|N|N|N|N|N| +2449381|AAAAAAAAFOPFFCAA|1994-01-28|1128|4909|377|1994|5|1|28|1|1994|377|4909|Friday|1994Q1|N|Y|N|2449354|2449353|2449016|2449289|N|N|N|N|N| +2449382|AAAAAAAAGOPFFCAA|1994-01-29|1128|4909|377|1994|6|1|29|1|1994|377|4909|Saturday|1994Q1|N|Y|N|2449354|2449353|2449017|2449290|N|N|N|N|N| +2449383|AAAAAAAAHOPFFCAA|1994-01-30|1128|4909|377|1994|0|1|30|1|1994|377|4909|Sunday|1994Q1|N|N|N|2449354|2449353|2449018|2449291|N|N|N|N|N| +2449384|AAAAAAAAIOPFFCAA|1994-01-31|1128|4909|377|1994|1|1|31|1|1994|377|4909|Monday|1994Q1|N|N|N|2449354|2449353|2449019|2449292|N|N|N|N|N| +2449385|AAAAAAAAJOPFFCAA|1994-02-01|1129|4910|377|1994|2|2|1|1|1994|377|4910|Tuesday|1994Q1|N|N|N|2449385|2449415|2449020|2449293|N|N|N|N|N| +2449386|AAAAAAAAKOPFFCAA|1994-02-02|1129|4910|377|1994|3|2|2|1|1994|377|4910|Wednesday|1994Q1|N|N|N|2449385|2449415|2449021|2449294|N|N|N|N|N| +2449387|AAAAAAAALOPFFCAA|1994-02-03|1129|4910|377|1994|4|2|3|1|1994|377|4910|Thursday|1994Q1|N|N|N|2449385|2449415|2449022|2449295|N|N|N|N|N| +2449388|AAAAAAAAMOPFFCAA|1994-02-04|1129|4910|377|1994|5|2|4|1|1994|377|4910|Friday|1994Q1|N|Y|N|2449385|2449415|2449023|2449296|N|N|N|N|N| +2449389|AAAAAAAANOPFFCAA|1994-02-05|1129|4910|377|1994|6|2|5|1|1994|377|4910|Saturday|1994Q1|N|Y|N|2449385|2449415|2449024|2449297|N|N|N|N|N| +2449390|AAAAAAAAOOPFFCAA|1994-02-06|1129|4910|377|1994|0|2|6|1|1994|377|4910|Sunday|1994Q1|N|N|N|2449385|2449415|2449025|2449298|N|N|N|N|N| +2449391|AAAAAAAAPOPFFCAA|1994-02-07|1129|4910|377|1994|1|2|7|1|1994|377|4910|Monday|1994Q1|N|N|N|2449385|2449415|2449026|2449299|N|N|N|N|N| +2449392|AAAAAAAAAPPFFCAA|1994-02-08|1129|4911|377|1994|2|2|8|1|1994|377|4911|Tuesday|1994Q1|N|N|N|2449385|2449415|2449027|2449300|N|N|N|N|N| +2449393|AAAAAAAABPPFFCAA|1994-02-09|1129|4911|377|1994|3|2|9|1|1994|377|4911|Wednesday|1994Q1|N|N|N|2449385|2449415|2449028|2449301|N|N|N|N|N| +2449394|AAAAAAAACPPFFCAA|1994-02-10|1129|4911|377|1994|4|2|10|1|1994|377|4911|Thursday|1994Q1|N|N|N|2449385|2449415|2449029|2449302|N|N|N|N|N| +2449395|AAAAAAAADPPFFCAA|1994-02-11|1129|4911|377|1994|5|2|11|1|1994|377|4911|Friday|1994Q1|N|Y|N|2449385|2449415|2449030|2449303|N|N|N|N|N| +2449396|AAAAAAAAEPPFFCAA|1994-02-12|1129|4911|377|1994|6|2|12|1|1994|377|4911|Saturday|1994Q1|N|Y|N|2449385|2449415|2449031|2449304|N|N|N|N|N| +2449397|AAAAAAAAFPPFFCAA|1994-02-13|1129|4911|377|1994|0|2|13|1|1994|377|4911|Sunday|1994Q1|N|N|N|2449385|2449415|2449032|2449305|N|N|N|N|N| +2449398|AAAAAAAAGPPFFCAA|1994-02-14|1129|4911|377|1994|1|2|14|1|1994|377|4911|Monday|1994Q1|N|N|N|2449385|2449415|2449033|2449306|N|N|N|N|N| +2449399|AAAAAAAAHPPFFCAA|1994-02-15|1129|4912|377|1994|2|2|15|1|1994|377|4912|Tuesday|1994Q1|N|N|N|2449385|2449415|2449034|2449307|N|N|N|N|N| +2449400|AAAAAAAAIPPFFCAA|1994-02-16|1129|4912|377|1994|3|2|16|1|1994|377|4912|Wednesday|1994Q1|N|N|N|2449385|2449415|2449035|2449308|N|N|N|N|N| +2449401|AAAAAAAAJPPFFCAA|1994-02-17|1129|4912|377|1994|4|2|17|1|1994|377|4912|Thursday|1994Q1|N|N|N|2449385|2449415|2449036|2449309|N|N|N|N|N| +2449402|AAAAAAAAKPPFFCAA|1994-02-18|1129|4912|377|1994|5|2|18|1|1994|377|4912|Friday|1994Q1|N|Y|N|2449385|2449415|2449037|2449310|N|N|N|N|N| +2449403|AAAAAAAALPPFFCAA|1994-02-19|1129|4912|377|1994|6|2|19|1|1994|377|4912|Saturday|1994Q1|N|Y|N|2449385|2449415|2449038|2449311|N|N|N|N|N| +2449404|AAAAAAAAMPPFFCAA|1994-02-20|1129|4912|377|1994|0|2|20|1|1994|377|4912|Sunday|1994Q1|N|N|N|2449385|2449415|2449039|2449312|N|N|N|N|N| +2449405|AAAAAAAANPPFFCAA|1994-02-21|1129|4912|377|1994|1|2|21|1|1994|377|4912|Monday|1994Q1|N|N|N|2449385|2449415|2449040|2449313|N|N|N|N|N| +2449406|AAAAAAAAOPPFFCAA|1994-02-22|1129|4913|377|1994|2|2|22|1|1994|377|4913|Tuesday|1994Q1|N|N|N|2449385|2449415|2449041|2449314|N|N|N|N|N| +2449407|AAAAAAAAPPPFFCAA|1994-02-23|1129|4913|377|1994|3|2|23|1|1994|377|4913|Wednesday|1994Q1|N|N|N|2449385|2449415|2449042|2449315|N|N|N|N|N| +2449408|AAAAAAAAAAAGFCAA|1994-02-24|1129|4913|377|1994|4|2|24|1|1994|377|4913|Thursday|1994Q1|N|N|N|2449385|2449415|2449043|2449316|N|N|N|N|N| +2449409|AAAAAAAABAAGFCAA|1994-02-25|1129|4913|377|1994|5|2|25|1|1994|377|4913|Friday|1994Q1|N|Y|N|2449385|2449415|2449044|2449317|N|N|N|N|N| +2449410|AAAAAAAACAAGFCAA|1994-02-26|1129|4913|377|1994|6|2|26|1|1994|377|4913|Saturday|1994Q1|N|Y|N|2449385|2449415|2449045|2449318|N|N|N|N|N| +2449411|AAAAAAAADAAGFCAA|1994-02-27|1129|4913|377|1994|0|2|27|1|1994|377|4913|Sunday|1994Q1|N|N|N|2449385|2449415|2449046|2449319|N|N|N|N|N| +2449412|AAAAAAAAEAAGFCAA|1994-02-28|1129|4913|377|1994|1|2|28|1|1994|377|4913|Monday|1994Q1|N|N|N|2449385|2449415|2449047|2449320|N|N|N|N|N| +2449413|AAAAAAAAFAAGFCAA|1994-03-01|1130|4914|378|1994|2|3|1|1|1994|378|4914|Tuesday|1994Q1|N|N|N|2449413|2449471|2449048|2449321|N|N|N|N|N| +2449414|AAAAAAAAGAAGFCAA|1994-03-02|1130|4914|378|1994|3|3|2|1|1994|378|4914|Wednesday|1994Q1|N|N|N|2449413|2449471|2449049|2449322|N|N|N|N|N| +2449415|AAAAAAAAHAAGFCAA|1994-03-03|1130|4914|378|1994|4|3|3|1|1994|378|4914|Thursday|1994Q1|N|N|N|2449413|2449471|2449050|2449323|N|N|N|N|N| +2449416|AAAAAAAAIAAGFCAA|1994-03-04|1130|4914|378|1994|5|3|4|1|1994|378|4914|Friday|1994Q1|N|Y|N|2449413|2449471|2449051|2449324|N|N|N|N|N| +2449417|AAAAAAAAJAAGFCAA|1994-03-05|1130|4914|378|1994|6|3|5|1|1994|378|4914|Saturday|1994Q1|N|Y|N|2449413|2449471|2449052|2449325|N|N|N|N|N| +2449418|AAAAAAAAKAAGFCAA|1994-03-06|1130|4914|378|1994|0|3|6|1|1994|378|4914|Sunday|1994Q1|N|N|N|2449413|2449471|2449053|2449326|N|N|N|N|N| +2449419|AAAAAAAALAAGFCAA|1994-03-07|1130|4914|378|1994|1|3|7|1|1994|378|4914|Monday|1994Q1|N|N|N|2449413|2449471|2449054|2449327|N|N|N|N|N| +2449420|AAAAAAAAMAAGFCAA|1994-03-08|1130|4915|378|1994|2|3|8|1|1994|378|4915|Tuesday|1994Q1|N|N|N|2449413|2449471|2449055|2449328|N|N|N|N|N| +2449421|AAAAAAAANAAGFCAA|1994-03-09|1130|4915|378|1994|3|3|9|1|1994|378|4915|Wednesday|1994Q1|N|N|N|2449413|2449471|2449056|2449329|N|N|N|N|N| +2449422|AAAAAAAAOAAGFCAA|1994-03-10|1130|4915|378|1994|4|3|10|1|1994|378|4915|Thursday|1994Q1|N|N|N|2449413|2449471|2449057|2449330|N|N|N|N|N| +2449423|AAAAAAAAPAAGFCAA|1994-03-11|1130|4915|378|1994|5|3|11|1|1994|378|4915|Friday|1994Q1|N|Y|N|2449413|2449471|2449058|2449331|N|N|N|N|N| +2449424|AAAAAAAAABAGFCAA|1994-03-12|1130|4915|378|1994|6|3|12|1|1994|378|4915|Saturday|1994Q1|N|Y|N|2449413|2449471|2449059|2449332|N|N|N|N|N| +2449425|AAAAAAAABBAGFCAA|1994-03-13|1130|4915|378|1994|0|3|13|1|1994|378|4915|Sunday|1994Q1|N|N|N|2449413|2449471|2449060|2449333|N|N|N|N|N| +2449426|AAAAAAAACBAGFCAA|1994-03-14|1130|4915|378|1994|1|3|14|1|1994|378|4915|Monday|1994Q1|N|N|N|2449413|2449471|2449061|2449334|N|N|N|N|N| +2449427|AAAAAAAADBAGFCAA|1994-03-15|1130|4916|378|1994|2|3|15|1|1994|378|4916|Tuesday|1994Q1|N|N|N|2449413|2449471|2449062|2449335|N|N|N|N|N| +2449428|AAAAAAAAEBAGFCAA|1994-03-16|1130|4916|378|1994|3|3|16|1|1994|378|4916|Wednesday|1994Q1|N|N|N|2449413|2449471|2449063|2449336|N|N|N|N|N| +2449429|AAAAAAAAFBAGFCAA|1994-03-17|1130|4916|378|1994|4|3|17|1|1994|378|4916|Thursday|1994Q1|N|N|N|2449413|2449471|2449064|2449337|N|N|N|N|N| +2449430|AAAAAAAAGBAGFCAA|1994-03-18|1130|4916|378|1994|5|3|18|1|1994|378|4916|Friday|1994Q1|N|Y|N|2449413|2449471|2449065|2449338|N|N|N|N|N| +2449431|AAAAAAAAHBAGFCAA|1994-03-19|1130|4916|378|1994|6|3|19|1|1994|378|4916|Saturday|1994Q1|N|Y|N|2449413|2449471|2449066|2449339|N|N|N|N|N| +2449432|AAAAAAAAIBAGFCAA|1994-03-20|1130|4916|378|1994|0|3|20|1|1994|378|4916|Sunday|1994Q1|N|N|N|2449413|2449471|2449067|2449340|N|N|N|N|N| +2449433|AAAAAAAAJBAGFCAA|1994-03-21|1130|4916|378|1994|1|3|21|1|1994|378|4916|Monday|1994Q1|N|N|N|2449413|2449471|2449068|2449341|N|N|N|N|N| +2449434|AAAAAAAAKBAGFCAA|1994-03-22|1130|4917|378|1994|2|3|22|1|1994|378|4917|Tuesday|1994Q1|N|N|N|2449413|2449471|2449069|2449342|N|N|N|N|N| +2449435|AAAAAAAALBAGFCAA|1994-03-23|1130|4917|378|1994|3|3|23|1|1994|378|4917|Wednesday|1994Q1|N|N|N|2449413|2449471|2449070|2449343|N|N|N|N|N| +2449436|AAAAAAAAMBAGFCAA|1994-03-24|1130|4917|378|1994|4|3|24|1|1994|378|4917|Thursday|1994Q1|N|N|N|2449413|2449471|2449071|2449344|N|N|N|N|N| +2449437|AAAAAAAANBAGFCAA|1994-03-25|1130|4917|378|1994|5|3|25|1|1994|378|4917|Friday|1994Q1|N|Y|N|2449413|2449471|2449072|2449345|N|N|N|N|N| +2449438|AAAAAAAAOBAGFCAA|1994-03-26|1130|4917|378|1994|6|3|26|1|1994|378|4917|Saturday|1994Q1|N|Y|N|2449413|2449471|2449073|2449346|N|N|N|N|N| +2449439|AAAAAAAAPBAGFCAA|1994-03-27|1130|4917|378|1994|0|3|27|1|1994|378|4917|Sunday|1994Q1|N|N|N|2449413|2449471|2449074|2449347|N|N|N|N|N| +2449440|AAAAAAAAACAGFCAA|1994-03-28|1130|4917|378|1994|1|3|28|1|1994|378|4917|Monday|1994Q1|N|N|N|2449413|2449471|2449075|2449348|N|N|N|N|N| +2449441|AAAAAAAABCAGFCAA|1994-03-29|1130|4918|378|1994|2|3|29|1|1994|378|4918|Tuesday|1994Q1|N|N|N|2449413|2449471|2449076|2449349|N|N|N|N|N| +2449442|AAAAAAAACCAGFCAA|1994-03-30|1130|4918|378|1994|3|3|30|1|1994|378|4918|Wednesday|1994Q1|N|N|N|2449413|2449471|2449077|2449350|N|N|N|N|N| +2449443|AAAAAAAADCAGFCAA|1994-03-31|1130|4918|378|1994|4|3|31|1|1994|378|4918|Thursday|1994Q1|N|N|N|2449413|2449471|2449078|2449351|N|N|N|N|N| +2449444|AAAAAAAAECAGFCAA|1994-04-01|1131|4918|378|1994|5|4|1|1|1994|378|4918|Friday|1994Q1|N|Y|N|2449444|2449533|2449079|2449354|N|N|N|N|N| +2449445|AAAAAAAAFCAGFCAA|1994-04-02|1131|4918|378|1994|6|4|2|2|1994|378|4918|Saturday|1994Q2|N|Y|N|2449444|2449533|2449080|2449355|N|N|N|N|N| +2449446|AAAAAAAAGCAGFCAA|1994-04-03|1131|4918|378|1994|0|4|3|2|1994|378|4918|Sunday|1994Q2|N|N|N|2449444|2449533|2449081|2449356|N|N|N|N|N| +2449447|AAAAAAAAHCAGFCAA|1994-04-04|1131|4918|378|1994|1|4|4|2|1994|378|4918|Monday|1994Q2|N|N|N|2449444|2449533|2449082|2449357|N|N|N|N|N| +2449448|AAAAAAAAICAGFCAA|1994-04-05|1131|4919|378|1994|2|4|5|2|1994|378|4919|Tuesday|1994Q2|N|N|N|2449444|2449533|2449083|2449358|N|N|N|N|N| +2449449|AAAAAAAAJCAGFCAA|1994-04-06|1131|4919|378|1994|3|4|6|2|1994|378|4919|Wednesday|1994Q2|N|N|N|2449444|2449533|2449084|2449359|N|N|N|N|N| +2449450|AAAAAAAAKCAGFCAA|1994-04-07|1131|4919|378|1994|4|4|7|2|1994|378|4919|Thursday|1994Q2|N|N|N|2449444|2449533|2449085|2449360|N|N|N|N|N| +2449451|AAAAAAAALCAGFCAA|1994-04-08|1131|4919|378|1994|5|4|8|2|1994|378|4919|Friday|1994Q2|N|Y|N|2449444|2449533|2449086|2449361|N|N|N|N|N| +2449452|AAAAAAAAMCAGFCAA|1994-04-09|1131|4919|378|1994|6|4|9|2|1994|378|4919|Saturday|1994Q2|N|Y|N|2449444|2449533|2449087|2449362|N|N|N|N|N| +2449453|AAAAAAAANCAGFCAA|1994-04-10|1131|4919|378|1994|0|4|10|2|1994|378|4919|Sunday|1994Q2|N|N|N|2449444|2449533|2449088|2449363|N|N|N|N|N| +2449454|AAAAAAAAOCAGFCAA|1994-04-11|1131|4919|378|1994|1|4|11|2|1994|378|4919|Monday|1994Q2|N|N|N|2449444|2449533|2449089|2449364|N|N|N|N|N| +2449455|AAAAAAAAPCAGFCAA|1994-04-12|1131|4920|378|1994|2|4|12|2|1994|378|4920|Tuesday|1994Q2|N|N|N|2449444|2449533|2449090|2449365|N|N|N|N|N| +2449456|AAAAAAAAADAGFCAA|1994-04-13|1131|4920|378|1994|3|4|13|2|1994|378|4920|Wednesday|1994Q2|N|N|N|2449444|2449533|2449091|2449366|N|N|N|N|N| +2449457|AAAAAAAABDAGFCAA|1994-04-14|1131|4920|378|1994|4|4|14|2|1994|378|4920|Thursday|1994Q2|N|N|N|2449444|2449533|2449092|2449367|N|N|N|N|N| +2449458|AAAAAAAACDAGFCAA|1994-04-15|1131|4920|378|1994|5|4|15|2|1994|378|4920|Friday|1994Q2|N|Y|N|2449444|2449533|2449093|2449368|N|N|N|N|N| +2449459|AAAAAAAADDAGFCAA|1994-04-16|1131|4920|378|1994|6|4|16|2|1994|378|4920|Saturday|1994Q2|N|Y|N|2449444|2449533|2449094|2449369|N|N|N|N|N| +2449460|AAAAAAAAEDAGFCAA|1994-04-17|1131|4920|378|1994|0|4|17|2|1994|378|4920|Sunday|1994Q2|N|N|N|2449444|2449533|2449095|2449370|N|N|N|N|N| +2449461|AAAAAAAAFDAGFCAA|1994-04-18|1131|4920|378|1994|1|4|18|2|1994|378|4920|Monday|1994Q2|N|N|N|2449444|2449533|2449096|2449371|N|N|N|N|N| +2449462|AAAAAAAAGDAGFCAA|1994-04-19|1131|4921|378|1994|2|4|19|2|1994|378|4921|Tuesday|1994Q2|N|N|N|2449444|2449533|2449097|2449372|N|N|N|N|N| +2449463|AAAAAAAAHDAGFCAA|1994-04-20|1131|4921|378|1994|3|4|20|2|1994|378|4921|Wednesday|1994Q2|N|N|N|2449444|2449533|2449098|2449373|N|N|N|N|N| +2449464|AAAAAAAAIDAGFCAA|1994-04-21|1131|4921|378|1994|4|4|21|2|1994|378|4921|Thursday|1994Q2|N|N|N|2449444|2449533|2449099|2449374|N|N|N|N|N| +2449465|AAAAAAAAJDAGFCAA|1994-04-22|1131|4921|378|1994|5|4|22|2|1994|378|4921|Friday|1994Q2|N|Y|N|2449444|2449533|2449100|2449375|N|N|N|N|N| +2449466|AAAAAAAAKDAGFCAA|1994-04-23|1131|4921|378|1994|6|4|23|2|1994|378|4921|Saturday|1994Q2|N|Y|N|2449444|2449533|2449101|2449376|N|N|N|N|N| +2449467|AAAAAAAALDAGFCAA|1994-04-24|1131|4921|378|1994|0|4|24|2|1994|378|4921|Sunday|1994Q2|N|N|N|2449444|2449533|2449102|2449377|N|N|N|N|N| +2449468|AAAAAAAAMDAGFCAA|1994-04-25|1131|4921|378|1994|1|4|25|2|1994|378|4921|Monday|1994Q2|N|N|N|2449444|2449533|2449103|2449378|N|N|N|N|N| +2449469|AAAAAAAANDAGFCAA|1994-04-26|1131|4922|378|1994|2|4|26|2|1994|378|4922|Tuesday|1994Q2|N|N|N|2449444|2449533|2449104|2449379|N|N|N|N|N| +2449470|AAAAAAAAODAGFCAA|1994-04-27|1131|4922|378|1994|3|4|27|2|1994|378|4922|Wednesday|1994Q2|N|N|N|2449444|2449533|2449105|2449380|N|N|N|N|N| +2449471|AAAAAAAAPDAGFCAA|1994-04-28|1131|4922|378|1994|4|4|28|2|1994|378|4922|Thursday|1994Q2|N|N|N|2449444|2449533|2449106|2449381|N|N|N|N|N| +2449472|AAAAAAAAAEAGFCAA|1994-04-29|1131|4922|378|1994|5|4|29|2|1994|378|4922|Friday|1994Q2|N|Y|N|2449444|2449533|2449107|2449382|N|N|N|N|N| +2449473|AAAAAAAABEAGFCAA|1994-04-30|1131|4922|378|1994|6|4|30|2|1994|378|4922|Saturday|1994Q2|N|Y|N|2449444|2449533|2449108|2449383|N|N|N|N|N| +2449474|AAAAAAAACEAGFCAA|1994-05-01|1132|4922|378|1994|0|5|1|2|1994|378|4922|Sunday|1994Q2|N|N|N|2449474|2449593|2449109|2449384|N|N|N|N|N| +2449475|AAAAAAAADEAGFCAA|1994-05-02|1132|4922|378|1994|1|5|2|2|1994|378|4922|Monday|1994Q2|N|N|N|2449474|2449593|2449110|2449385|N|N|N|N|N| +2449476|AAAAAAAAEEAGFCAA|1994-05-03|1132|4923|378|1994|2|5|3|2|1994|378|4923|Tuesday|1994Q2|N|N|N|2449474|2449593|2449111|2449386|N|N|N|N|N| +2449477|AAAAAAAAFEAGFCAA|1994-05-04|1132|4923|378|1994|3|5|4|2|1994|378|4923|Wednesday|1994Q2|N|N|N|2449474|2449593|2449112|2449387|N|N|N|N|N| +2449478|AAAAAAAAGEAGFCAA|1994-05-05|1132|4923|378|1994|4|5|5|2|1994|378|4923|Thursday|1994Q2|N|N|N|2449474|2449593|2449113|2449388|N|N|N|N|N| +2449479|AAAAAAAAHEAGFCAA|1994-05-06|1132|4923|378|1994|5|5|6|2|1994|378|4923|Friday|1994Q2|N|Y|N|2449474|2449593|2449114|2449389|N|N|N|N|N| +2449480|AAAAAAAAIEAGFCAA|1994-05-07|1132|4923|378|1994|6|5|7|2|1994|378|4923|Saturday|1994Q2|N|Y|N|2449474|2449593|2449115|2449390|N|N|N|N|N| +2449481|AAAAAAAAJEAGFCAA|1994-05-08|1132|4923|378|1994|0|5|8|2|1994|378|4923|Sunday|1994Q2|N|N|N|2449474|2449593|2449116|2449391|N|N|N|N|N| +2449482|AAAAAAAAKEAGFCAA|1994-05-09|1132|4923|378|1994|1|5|9|2|1994|378|4923|Monday|1994Q2|N|N|N|2449474|2449593|2449117|2449392|N|N|N|N|N| +2449483|AAAAAAAALEAGFCAA|1994-05-10|1132|4924|378|1994|2|5|10|2|1994|378|4924|Tuesday|1994Q2|N|N|N|2449474|2449593|2449118|2449393|N|N|N|N|N| +2449484|AAAAAAAAMEAGFCAA|1994-05-11|1132|4924|378|1994|3|5|11|2|1994|378|4924|Wednesday|1994Q2|N|N|N|2449474|2449593|2449119|2449394|N|N|N|N|N| +2449485|AAAAAAAANEAGFCAA|1994-05-12|1132|4924|378|1994|4|5|12|2|1994|378|4924|Thursday|1994Q2|N|N|N|2449474|2449593|2449120|2449395|N|N|N|N|N| +2449486|AAAAAAAAOEAGFCAA|1994-05-13|1132|4924|378|1994|5|5|13|2|1994|378|4924|Friday|1994Q2|N|Y|N|2449474|2449593|2449121|2449396|N|N|N|N|N| +2449487|AAAAAAAAPEAGFCAA|1994-05-14|1132|4924|378|1994|6|5|14|2|1994|378|4924|Saturday|1994Q2|N|Y|N|2449474|2449593|2449122|2449397|N|N|N|N|N| +2449488|AAAAAAAAAFAGFCAA|1994-05-15|1132|4924|378|1994|0|5|15|2|1994|378|4924|Sunday|1994Q2|N|N|N|2449474|2449593|2449123|2449398|N|N|N|N|N| +2449489|AAAAAAAABFAGFCAA|1994-05-16|1132|4924|378|1994|1|5|16|2|1994|378|4924|Monday|1994Q2|N|N|N|2449474|2449593|2449124|2449399|N|N|N|N|N| +2449490|AAAAAAAACFAGFCAA|1994-05-17|1132|4925|378|1994|2|5|17|2|1994|378|4925|Tuesday|1994Q2|N|N|N|2449474|2449593|2449125|2449400|N|N|N|N|N| +2449491|AAAAAAAADFAGFCAA|1994-05-18|1132|4925|378|1994|3|5|18|2|1994|378|4925|Wednesday|1994Q2|N|N|N|2449474|2449593|2449126|2449401|N|N|N|N|N| +2449492|AAAAAAAAEFAGFCAA|1994-05-19|1132|4925|378|1994|4|5|19|2|1994|378|4925|Thursday|1994Q2|N|N|N|2449474|2449593|2449127|2449402|N|N|N|N|N| +2449493|AAAAAAAAFFAGFCAA|1994-05-20|1132|4925|378|1994|5|5|20|2|1994|378|4925|Friday|1994Q2|N|Y|N|2449474|2449593|2449128|2449403|N|N|N|N|N| +2449494|AAAAAAAAGFAGFCAA|1994-05-21|1132|4925|378|1994|6|5|21|2|1994|378|4925|Saturday|1994Q2|N|Y|N|2449474|2449593|2449129|2449404|N|N|N|N|N| +2449495|AAAAAAAAHFAGFCAA|1994-05-22|1132|4925|378|1994|0|5|22|2|1994|378|4925|Sunday|1994Q2|N|N|N|2449474|2449593|2449130|2449405|N|N|N|N|N| +2449496|AAAAAAAAIFAGFCAA|1994-05-23|1132|4925|378|1994|1|5|23|2|1994|378|4925|Monday|1994Q2|N|N|N|2449474|2449593|2449131|2449406|N|N|N|N|N| +2449497|AAAAAAAAJFAGFCAA|1994-05-24|1132|4926|378|1994|2|5|24|2|1994|378|4926|Tuesday|1994Q2|N|N|N|2449474|2449593|2449132|2449407|N|N|N|N|N| +2449498|AAAAAAAAKFAGFCAA|1994-05-25|1132|4926|378|1994|3|5|25|2|1994|378|4926|Wednesday|1994Q2|N|N|N|2449474|2449593|2449133|2449408|N|N|N|N|N| +2449499|AAAAAAAALFAGFCAA|1994-05-26|1132|4926|378|1994|4|5|26|2|1994|378|4926|Thursday|1994Q2|N|N|N|2449474|2449593|2449134|2449409|N|N|N|N|N| +2449500|AAAAAAAAMFAGFCAA|1994-05-27|1132|4926|378|1994|5|5|27|2|1994|378|4926|Friday|1994Q2|N|Y|N|2449474|2449593|2449135|2449410|N|N|N|N|N| +2449501|AAAAAAAANFAGFCAA|1994-05-28|1132|4926|378|1994|6|5|28|2|1994|378|4926|Saturday|1994Q2|N|Y|N|2449474|2449593|2449136|2449411|N|N|N|N|N| +2449502|AAAAAAAAOFAGFCAA|1994-05-29|1132|4926|378|1994|0|5|29|2|1994|378|4926|Sunday|1994Q2|N|N|N|2449474|2449593|2449137|2449412|N|N|N|N|N| +2449503|AAAAAAAAPFAGFCAA|1994-05-30|1132|4926|378|1994|1|5|30|2|1994|378|4926|Monday|1994Q2|N|N|N|2449474|2449593|2449138|2449413|N|N|N|N|N| +2449504|AAAAAAAAAGAGFCAA|1994-05-31|1132|4927|378|1994|2|5|31|2|1994|378|4927|Tuesday|1994Q2|N|N|N|2449474|2449593|2449139|2449414|N|N|N|N|N| +2449505|AAAAAAAABGAGFCAA|1994-06-01|1133|4927|379|1994|3|6|1|2|1994|379|4927|Wednesday|1994Q2|N|N|N|2449505|2449655|2449140|2449415|N|N|N|N|N| +2449506|AAAAAAAACGAGFCAA|1994-06-02|1133|4927|379|1994|4|6|2|2|1994|379|4927|Thursday|1994Q2|N|N|N|2449505|2449655|2449141|2449416|N|N|N|N|N| +2449507|AAAAAAAADGAGFCAA|1994-06-03|1133|4927|379|1994|5|6|3|2|1994|379|4927|Friday|1994Q2|N|Y|N|2449505|2449655|2449142|2449417|N|N|N|N|N| +2449508|AAAAAAAAEGAGFCAA|1994-06-04|1133|4927|379|1994|6|6|4|2|1994|379|4927|Saturday|1994Q2|N|Y|N|2449505|2449655|2449143|2449418|N|N|N|N|N| +2449509|AAAAAAAAFGAGFCAA|1994-06-05|1133|4927|379|1994|0|6|5|2|1994|379|4927|Sunday|1994Q2|N|N|N|2449505|2449655|2449144|2449419|N|N|N|N|N| +2449510|AAAAAAAAGGAGFCAA|1994-06-06|1133|4927|379|1994|1|6|6|2|1994|379|4927|Monday|1994Q2|N|N|N|2449505|2449655|2449145|2449420|N|N|N|N|N| +2449511|AAAAAAAAHGAGFCAA|1994-06-07|1133|4928|379|1994|2|6|7|2|1994|379|4928|Tuesday|1994Q2|N|N|N|2449505|2449655|2449146|2449421|N|N|N|N|N| +2449512|AAAAAAAAIGAGFCAA|1994-06-08|1133|4928|379|1994|3|6|8|2|1994|379|4928|Wednesday|1994Q2|N|N|N|2449505|2449655|2449147|2449422|N|N|N|N|N| +2449513|AAAAAAAAJGAGFCAA|1994-06-09|1133|4928|379|1994|4|6|9|2|1994|379|4928|Thursday|1994Q2|N|N|N|2449505|2449655|2449148|2449423|N|N|N|N|N| +2449514|AAAAAAAAKGAGFCAA|1994-06-10|1133|4928|379|1994|5|6|10|2|1994|379|4928|Friday|1994Q2|N|Y|N|2449505|2449655|2449149|2449424|N|N|N|N|N| +2449515|AAAAAAAALGAGFCAA|1994-06-11|1133|4928|379|1994|6|6|11|2|1994|379|4928|Saturday|1994Q2|N|Y|N|2449505|2449655|2449150|2449425|N|N|N|N|N| +2449516|AAAAAAAAMGAGFCAA|1994-06-12|1133|4928|379|1994|0|6|12|2|1994|379|4928|Sunday|1994Q2|N|N|N|2449505|2449655|2449151|2449426|N|N|N|N|N| +2449517|AAAAAAAANGAGFCAA|1994-06-13|1133|4928|379|1994|1|6|13|2|1994|379|4928|Monday|1994Q2|N|N|N|2449505|2449655|2449152|2449427|N|N|N|N|N| +2449518|AAAAAAAAOGAGFCAA|1994-06-14|1133|4929|379|1994|2|6|14|2|1994|379|4929|Tuesday|1994Q2|N|N|N|2449505|2449655|2449153|2449428|N|N|N|N|N| +2449519|AAAAAAAAPGAGFCAA|1994-06-15|1133|4929|379|1994|3|6|15|2|1994|379|4929|Wednesday|1994Q2|N|N|N|2449505|2449655|2449154|2449429|N|N|N|N|N| +2449520|AAAAAAAAAHAGFCAA|1994-06-16|1133|4929|379|1994|4|6|16|2|1994|379|4929|Thursday|1994Q2|N|N|N|2449505|2449655|2449155|2449430|N|N|N|N|N| +2449521|AAAAAAAABHAGFCAA|1994-06-17|1133|4929|379|1994|5|6|17|2|1994|379|4929|Friday|1994Q2|N|Y|N|2449505|2449655|2449156|2449431|N|N|N|N|N| +2449522|AAAAAAAACHAGFCAA|1994-06-18|1133|4929|379|1994|6|6|18|2|1994|379|4929|Saturday|1994Q2|N|Y|N|2449505|2449655|2449157|2449432|N|N|N|N|N| +2449523|AAAAAAAADHAGFCAA|1994-06-19|1133|4929|379|1994|0|6|19|2|1994|379|4929|Sunday|1994Q2|N|N|N|2449505|2449655|2449158|2449433|N|N|N|N|N| +2449524|AAAAAAAAEHAGFCAA|1994-06-20|1133|4929|379|1994|1|6|20|2|1994|379|4929|Monday|1994Q2|N|N|N|2449505|2449655|2449159|2449434|N|N|N|N|N| +2449525|AAAAAAAAFHAGFCAA|1994-06-21|1133|4930|379|1994|2|6|21|2|1994|379|4930|Tuesday|1994Q2|N|N|N|2449505|2449655|2449160|2449435|N|N|N|N|N| +2449526|AAAAAAAAGHAGFCAA|1994-06-22|1133|4930|379|1994|3|6|22|2|1994|379|4930|Wednesday|1994Q2|N|N|N|2449505|2449655|2449161|2449436|N|N|N|N|N| +2449527|AAAAAAAAHHAGFCAA|1994-06-23|1133|4930|379|1994|4|6|23|2|1994|379|4930|Thursday|1994Q2|N|N|N|2449505|2449655|2449162|2449437|N|N|N|N|N| +2449528|AAAAAAAAIHAGFCAA|1994-06-24|1133|4930|379|1994|5|6|24|2|1994|379|4930|Friday|1994Q2|N|Y|N|2449505|2449655|2449163|2449438|N|N|N|N|N| +2449529|AAAAAAAAJHAGFCAA|1994-06-25|1133|4930|379|1994|6|6|25|2|1994|379|4930|Saturday|1994Q2|N|Y|N|2449505|2449655|2449164|2449439|N|N|N|N|N| +2449530|AAAAAAAAKHAGFCAA|1994-06-26|1133|4930|379|1994|0|6|26|2|1994|379|4930|Sunday|1994Q2|N|N|N|2449505|2449655|2449165|2449440|N|N|N|N|N| +2449531|AAAAAAAALHAGFCAA|1994-06-27|1133|4930|379|1994|1|6|27|2|1994|379|4930|Monday|1994Q2|N|N|N|2449505|2449655|2449166|2449441|N|N|N|N|N| +2449532|AAAAAAAAMHAGFCAA|1994-06-28|1133|4931|379|1994|2|6|28|2|1994|379|4931|Tuesday|1994Q2|N|N|N|2449505|2449655|2449167|2449442|N|N|N|N|N| +2449533|AAAAAAAANHAGFCAA|1994-06-29|1133|4931|379|1994|3|6|29|2|1994|379|4931|Wednesday|1994Q2|N|N|N|2449505|2449655|2449168|2449443|N|N|N|N|N| +2449534|AAAAAAAAOHAGFCAA|1994-06-30|1133|4931|379|1994|4|6|30|2|1994|379|4931|Thursday|1994Q2|N|N|N|2449505|2449655|2449169|2449444|N|N|N|N|N| +2449535|AAAAAAAAPHAGFCAA|1994-07-01|1134|4931|379|1994|5|7|1|2|1994|379|4931|Friday|1994Q2|N|Y|N|2449535|2449715|2449170|2449444|N|N|N|N|N| +2449536|AAAAAAAAAIAGFCAA|1994-07-02|1134|4931|379|1994|6|7|2|3|1994|379|4931|Saturday|1994Q3|N|Y|N|2449535|2449715|2449171|2449445|N|N|N|N|N| +2449537|AAAAAAAABIAGFCAA|1994-07-03|1134|4931|379|1994|0|7|3|3|1994|379|4931|Sunday|1994Q3|N|N|N|2449535|2449715|2449172|2449446|N|N|N|N|N| +2449538|AAAAAAAACIAGFCAA|1994-07-04|1134|4931|379|1994|1|7|4|3|1994|379|4931|Monday|1994Q3|N|N|N|2449535|2449715|2449173|2449447|N|N|N|N|N| +2449539|AAAAAAAADIAGFCAA|1994-07-05|1134|4932|379|1994|2|7|5|3|1994|379|4932|Tuesday|1994Q3|Y|N|N|2449535|2449715|2449174|2449448|N|N|N|N|N| +2449540|AAAAAAAAEIAGFCAA|1994-07-06|1134|4932|379|1994|3|7|6|3|1994|379|4932|Wednesday|1994Q3|N|N|Y|2449535|2449715|2449175|2449449|N|N|N|N|N| +2449541|AAAAAAAAFIAGFCAA|1994-07-07|1134|4932|379|1994|4|7|7|3|1994|379|4932|Thursday|1994Q3|N|N|N|2449535|2449715|2449176|2449450|N|N|N|N|N| +2449542|AAAAAAAAGIAGFCAA|1994-07-08|1134|4932|379|1994|5|7|8|3|1994|379|4932|Friday|1994Q3|N|Y|N|2449535|2449715|2449177|2449451|N|N|N|N|N| +2449543|AAAAAAAAHIAGFCAA|1994-07-09|1134|4932|379|1994|6|7|9|3|1994|379|4932|Saturday|1994Q3|N|Y|N|2449535|2449715|2449178|2449452|N|N|N|N|N| +2449544|AAAAAAAAIIAGFCAA|1994-07-10|1134|4932|379|1994|0|7|10|3|1994|379|4932|Sunday|1994Q3|N|N|N|2449535|2449715|2449179|2449453|N|N|N|N|N| +2449545|AAAAAAAAJIAGFCAA|1994-07-11|1134|4932|379|1994|1|7|11|3|1994|379|4932|Monday|1994Q3|N|N|N|2449535|2449715|2449180|2449454|N|N|N|N|N| +2449546|AAAAAAAAKIAGFCAA|1994-07-12|1134|4933|379|1994|2|7|12|3|1994|379|4933|Tuesday|1994Q3|N|N|N|2449535|2449715|2449181|2449455|N|N|N|N|N| +2449547|AAAAAAAALIAGFCAA|1994-07-13|1134|4933|379|1994|3|7|13|3|1994|379|4933|Wednesday|1994Q3|N|N|N|2449535|2449715|2449182|2449456|N|N|N|N|N| +2449548|AAAAAAAAMIAGFCAA|1994-07-14|1134|4933|379|1994|4|7|14|3|1994|379|4933|Thursday|1994Q3|N|N|N|2449535|2449715|2449183|2449457|N|N|N|N|N| +2449549|AAAAAAAANIAGFCAA|1994-07-15|1134|4933|379|1994|5|7|15|3|1994|379|4933|Friday|1994Q3|N|Y|N|2449535|2449715|2449184|2449458|N|N|N|N|N| +2449550|AAAAAAAAOIAGFCAA|1994-07-16|1134|4933|379|1994|6|7|16|3|1994|379|4933|Saturday|1994Q3|N|Y|N|2449535|2449715|2449185|2449459|N|N|N|N|N| +2449551|AAAAAAAAPIAGFCAA|1994-07-17|1134|4933|379|1994|0|7|17|3|1994|379|4933|Sunday|1994Q3|N|N|N|2449535|2449715|2449186|2449460|N|N|N|N|N| +2449552|AAAAAAAAAJAGFCAA|1994-07-18|1134|4933|379|1994|1|7|18|3|1994|379|4933|Monday|1994Q3|N|N|N|2449535|2449715|2449187|2449461|N|N|N|N|N| +2449553|AAAAAAAABJAGFCAA|1994-07-19|1134|4934|379|1994|2|7|19|3|1994|379|4934|Tuesday|1994Q3|N|N|N|2449535|2449715|2449188|2449462|N|N|N|N|N| +2449554|AAAAAAAACJAGFCAA|1994-07-20|1134|4934|379|1994|3|7|20|3|1994|379|4934|Wednesday|1994Q3|N|N|N|2449535|2449715|2449189|2449463|N|N|N|N|N| +2449555|AAAAAAAADJAGFCAA|1994-07-21|1134|4934|379|1994|4|7|21|3|1994|379|4934|Thursday|1994Q3|N|N|N|2449535|2449715|2449190|2449464|N|N|N|N|N| +2449556|AAAAAAAAEJAGFCAA|1994-07-22|1134|4934|379|1994|5|7|22|3|1994|379|4934|Friday|1994Q3|N|Y|N|2449535|2449715|2449191|2449465|N|N|N|N|N| +2449557|AAAAAAAAFJAGFCAA|1994-07-23|1134|4934|379|1994|6|7|23|3|1994|379|4934|Saturday|1994Q3|N|Y|N|2449535|2449715|2449192|2449466|N|N|N|N|N| +2449558|AAAAAAAAGJAGFCAA|1994-07-24|1134|4934|379|1994|0|7|24|3|1994|379|4934|Sunday|1994Q3|N|N|N|2449535|2449715|2449193|2449467|N|N|N|N|N| +2449559|AAAAAAAAHJAGFCAA|1994-07-25|1134|4934|379|1994|1|7|25|3|1994|379|4934|Monday|1994Q3|N|N|N|2449535|2449715|2449194|2449468|N|N|N|N|N| +2449560|AAAAAAAAIJAGFCAA|1994-07-26|1134|4935|379|1994|2|7|26|3|1994|379|4935|Tuesday|1994Q3|N|N|N|2449535|2449715|2449195|2449469|N|N|N|N|N| +2449561|AAAAAAAAJJAGFCAA|1994-07-27|1134|4935|379|1994|3|7|27|3|1994|379|4935|Wednesday|1994Q3|N|N|N|2449535|2449715|2449196|2449470|N|N|N|N|N| +2449562|AAAAAAAAKJAGFCAA|1994-07-28|1134|4935|379|1994|4|7|28|3|1994|379|4935|Thursday|1994Q3|N|N|N|2449535|2449715|2449197|2449471|N|N|N|N|N| +2449563|AAAAAAAALJAGFCAA|1994-07-29|1134|4935|379|1994|5|7|29|3|1994|379|4935|Friday|1994Q3|N|Y|N|2449535|2449715|2449198|2449472|N|N|N|N|N| +2449564|AAAAAAAAMJAGFCAA|1994-07-30|1134|4935|379|1994|6|7|30|3|1994|379|4935|Saturday|1994Q3|N|Y|N|2449535|2449715|2449199|2449473|N|N|N|N|N| +2449565|AAAAAAAANJAGFCAA|1994-07-31|1134|4935|379|1994|0|7|31|3|1994|379|4935|Sunday|1994Q3|N|N|N|2449535|2449715|2449200|2449474|N|N|N|N|N| +2449566|AAAAAAAAOJAGFCAA|1994-08-01|1135|4935|379|1994|1|8|1|3|1994|379|4935|Monday|1994Q3|N|N|N|2449566|2449777|2449201|2449475|N|N|N|N|N| +2449567|AAAAAAAAPJAGFCAA|1994-08-02|1135|4936|379|1994|2|8|2|3|1994|379|4936|Tuesday|1994Q3|N|N|N|2449566|2449777|2449202|2449476|N|N|N|N|N| +2449568|AAAAAAAAAKAGFCAA|1994-08-03|1135|4936|379|1994|3|8|3|3|1994|379|4936|Wednesday|1994Q3|N|N|N|2449566|2449777|2449203|2449477|N|N|N|N|N| +2449569|AAAAAAAABKAGFCAA|1994-08-04|1135|4936|379|1994|4|8|4|3|1994|379|4936|Thursday|1994Q3|N|N|N|2449566|2449777|2449204|2449478|N|N|N|N|N| +2449570|AAAAAAAACKAGFCAA|1994-08-05|1135|4936|379|1994|5|8|5|3|1994|379|4936|Friday|1994Q3|N|Y|N|2449566|2449777|2449205|2449479|N|N|N|N|N| +2449571|AAAAAAAADKAGFCAA|1994-08-06|1135|4936|379|1994|6|8|6|3|1994|379|4936|Saturday|1994Q3|N|Y|N|2449566|2449777|2449206|2449480|N|N|N|N|N| +2449572|AAAAAAAAEKAGFCAA|1994-08-07|1135|4936|379|1994|0|8|7|3|1994|379|4936|Sunday|1994Q3|N|N|N|2449566|2449777|2449207|2449481|N|N|N|N|N| +2449573|AAAAAAAAFKAGFCAA|1994-08-08|1135|4936|379|1994|1|8|8|3|1994|379|4936|Monday|1994Q3|N|N|N|2449566|2449777|2449208|2449482|N|N|N|N|N| +2449574|AAAAAAAAGKAGFCAA|1994-08-09|1135|4937|379|1994|2|8|9|3|1994|379|4937|Tuesday|1994Q3|N|N|N|2449566|2449777|2449209|2449483|N|N|N|N|N| +2449575|AAAAAAAAHKAGFCAA|1994-08-10|1135|4937|379|1994|3|8|10|3|1994|379|4937|Wednesday|1994Q3|N|N|N|2449566|2449777|2449210|2449484|N|N|N|N|N| +2449576|AAAAAAAAIKAGFCAA|1994-08-11|1135|4937|379|1994|4|8|11|3|1994|379|4937|Thursday|1994Q3|N|N|N|2449566|2449777|2449211|2449485|N|N|N|N|N| +2449577|AAAAAAAAJKAGFCAA|1994-08-12|1135|4937|379|1994|5|8|12|3|1994|379|4937|Friday|1994Q3|N|Y|N|2449566|2449777|2449212|2449486|N|N|N|N|N| +2449578|AAAAAAAAKKAGFCAA|1994-08-13|1135|4937|379|1994|6|8|13|3|1994|379|4937|Saturday|1994Q3|N|Y|N|2449566|2449777|2449213|2449487|N|N|N|N|N| +2449579|AAAAAAAALKAGFCAA|1994-08-14|1135|4937|379|1994|0|8|14|3|1994|379|4937|Sunday|1994Q3|N|N|N|2449566|2449777|2449214|2449488|N|N|N|N|N| +2449580|AAAAAAAAMKAGFCAA|1994-08-15|1135|4937|379|1994|1|8|15|3|1994|379|4937|Monday|1994Q3|N|N|N|2449566|2449777|2449215|2449489|N|N|N|N|N| +2449581|AAAAAAAANKAGFCAA|1994-08-16|1135|4938|379|1994|2|8|16|3|1994|379|4938|Tuesday|1994Q3|N|N|N|2449566|2449777|2449216|2449490|N|N|N|N|N| +2449582|AAAAAAAAOKAGFCAA|1994-08-17|1135|4938|379|1994|3|8|17|3|1994|379|4938|Wednesday|1994Q3|N|N|N|2449566|2449777|2449217|2449491|N|N|N|N|N| +2449583|AAAAAAAAPKAGFCAA|1994-08-18|1135|4938|379|1994|4|8|18|3|1994|379|4938|Thursday|1994Q3|N|N|N|2449566|2449777|2449218|2449492|N|N|N|N|N| +2449584|AAAAAAAAALAGFCAA|1994-08-19|1135|4938|379|1994|5|8|19|3|1994|379|4938|Friday|1994Q3|N|Y|N|2449566|2449777|2449219|2449493|N|N|N|N|N| +2449585|AAAAAAAABLAGFCAA|1994-08-20|1135|4938|379|1994|6|8|20|3|1994|379|4938|Saturday|1994Q3|N|Y|N|2449566|2449777|2449220|2449494|N|N|N|N|N| +2449586|AAAAAAAACLAGFCAA|1994-08-21|1135|4938|379|1994|0|8|21|3|1994|379|4938|Sunday|1994Q3|N|N|N|2449566|2449777|2449221|2449495|N|N|N|N|N| +2449587|AAAAAAAADLAGFCAA|1994-08-22|1135|4938|379|1994|1|8|22|3|1994|379|4938|Monday|1994Q3|N|N|N|2449566|2449777|2449222|2449496|N|N|N|N|N| +2449588|AAAAAAAAELAGFCAA|1994-08-23|1135|4939|379|1994|2|8|23|3|1994|379|4939|Tuesday|1994Q3|N|N|N|2449566|2449777|2449223|2449497|N|N|N|N|N| +2449589|AAAAAAAAFLAGFCAA|1994-08-24|1135|4939|379|1994|3|8|24|3|1994|379|4939|Wednesday|1994Q3|N|N|N|2449566|2449777|2449224|2449498|N|N|N|N|N| +2449590|AAAAAAAAGLAGFCAA|1994-08-25|1135|4939|379|1994|4|8|25|3|1994|379|4939|Thursday|1994Q3|N|N|N|2449566|2449777|2449225|2449499|N|N|N|N|N| +2449591|AAAAAAAAHLAGFCAA|1994-08-26|1135|4939|379|1994|5|8|26|3|1994|379|4939|Friday|1994Q3|N|Y|N|2449566|2449777|2449226|2449500|N|N|N|N|N| +2449592|AAAAAAAAILAGFCAA|1994-08-27|1135|4939|379|1994|6|8|27|3|1994|379|4939|Saturday|1994Q3|N|Y|N|2449566|2449777|2449227|2449501|N|N|N|N|N| +2449593|AAAAAAAAJLAGFCAA|1994-08-28|1135|4939|379|1994|0|8|28|3|1994|379|4939|Sunday|1994Q3|N|N|N|2449566|2449777|2449228|2449502|N|N|N|N|N| +2449594|AAAAAAAAKLAGFCAA|1994-08-29|1135|4939|379|1994|1|8|29|3|1994|379|4939|Monday|1994Q3|N|N|N|2449566|2449777|2449229|2449503|N|N|N|N|N| +2449595|AAAAAAAALLAGFCAA|1994-08-30|1135|4940|379|1994|2|8|30|3|1994|379|4940|Tuesday|1994Q3|N|N|N|2449566|2449777|2449230|2449504|N|N|N|N|N| +2449596|AAAAAAAAMLAGFCAA|1994-08-31|1135|4940|379|1994|3|8|31|3|1994|379|4940|Wednesday|1994Q3|N|N|N|2449566|2449777|2449231|2449505|N|N|N|N|N| +2449597|AAAAAAAANLAGFCAA|1994-09-01|1136|4940|380|1994|4|9|1|3|1994|380|4940|Thursday|1994Q3|N|N|N|2449597|2449839|2449232|2449506|N|N|N|N|N| +2449598|AAAAAAAAOLAGFCAA|1994-09-02|1136|4940|380|1994|5|9|2|3|1994|380|4940|Friday|1994Q3|N|Y|N|2449597|2449839|2449233|2449507|N|N|N|N|N| +2449599|AAAAAAAAPLAGFCAA|1994-09-03|1136|4940|380|1994|6|9|3|3|1994|380|4940|Saturday|1994Q3|N|Y|N|2449597|2449839|2449234|2449508|N|N|N|N|N| +2449600|AAAAAAAAAMAGFCAA|1994-09-04|1136|4940|380|1994|0|9|4|3|1994|380|4940|Sunday|1994Q3|N|N|N|2449597|2449839|2449235|2449509|N|N|N|N|N| +2449601|AAAAAAAABMAGFCAA|1994-09-05|1136|4940|380|1994|1|9|5|3|1994|380|4940|Monday|1994Q3|N|N|N|2449597|2449839|2449236|2449510|N|N|N|N|N| +2449602|AAAAAAAACMAGFCAA|1994-09-06|1136|4941|380|1994|2|9|6|3|1994|380|4941|Tuesday|1994Q3|N|N|N|2449597|2449839|2449237|2449511|N|N|N|N|N| +2449603|AAAAAAAADMAGFCAA|1994-09-07|1136|4941|380|1994|3|9|7|3|1994|380|4941|Wednesday|1994Q3|N|N|N|2449597|2449839|2449238|2449512|N|N|N|N|N| +2449604|AAAAAAAAEMAGFCAA|1994-09-08|1136|4941|380|1994|4|9|8|3|1994|380|4941|Thursday|1994Q3|N|N|N|2449597|2449839|2449239|2449513|N|N|N|N|N| +2449605|AAAAAAAAFMAGFCAA|1994-09-09|1136|4941|380|1994|5|9|9|3|1994|380|4941|Friday|1994Q3|N|Y|N|2449597|2449839|2449240|2449514|N|N|N|N|N| +2449606|AAAAAAAAGMAGFCAA|1994-09-10|1136|4941|380|1994|6|9|10|3|1994|380|4941|Saturday|1994Q3|N|Y|N|2449597|2449839|2449241|2449515|N|N|N|N|N| +2449607|AAAAAAAAHMAGFCAA|1994-09-11|1136|4941|380|1994|0|9|11|3|1994|380|4941|Sunday|1994Q3|N|N|N|2449597|2449839|2449242|2449516|N|N|N|N|N| +2449608|AAAAAAAAIMAGFCAA|1994-09-12|1136|4941|380|1994|1|9|12|3|1994|380|4941|Monday|1994Q3|N|N|N|2449597|2449839|2449243|2449517|N|N|N|N|N| +2449609|AAAAAAAAJMAGFCAA|1994-09-13|1136|4942|380|1994|2|9|13|3|1994|380|4942|Tuesday|1994Q3|N|N|N|2449597|2449839|2449244|2449518|N|N|N|N|N| +2449610|AAAAAAAAKMAGFCAA|1994-09-14|1136|4942|380|1994|3|9|14|3|1994|380|4942|Wednesday|1994Q3|N|N|N|2449597|2449839|2449245|2449519|N|N|N|N|N| +2449611|AAAAAAAALMAGFCAA|1994-09-15|1136|4942|380|1994|4|9|15|3|1994|380|4942|Thursday|1994Q3|N|N|N|2449597|2449839|2449246|2449520|N|N|N|N|N| +2449612|AAAAAAAAMMAGFCAA|1994-09-16|1136|4942|380|1994|5|9|16|3|1994|380|4942|Friday|1994Q3|N|Y|N|2449597|2449839|2449247|2449521|N|N|N|N|N| +2449613|AAAAAAAANMAGFCAA|1994-09-17|1136|4942|380|1994|6|9|17|3|1994|380|4942|Saturday|1994Q3|N|Y|N|2449597|2449839|2449248|2449522|N|N|N|N|N| +2449614|AAAAAAAAOMAGFCAA|1994-09-18|1136|4942|380|1994|0|9|18|3|1994|380|4942|Sunday|1994Q3|N|N|N|2449597|2449839|2449249|2449523|N|N|N|N|N| +2449615|AAAAAAAAPMAGFCAA|1994-09-19|1136|4942|380|1994|1|9|19|3|1994|380|4942|Monday|1994Q3|N|N|N|2449597|2449839|2449250|2449524|N|N|N|N|N| +2449616|AAAAAAAAANAGFCAA|1994-09-20|1136|4943|380|1994|2|9|20|3|1994|380|4943|Tuesday|1994Q3|N|N|N|2449597|2449839|2449251|2449525|N|N|N|N|N| +2449617|AAAAAAAABNAGFCAA|1994-09-21|1136|4943|380|1994|3|9|21|3|1994|380|4943|Wednesday|1994Q3|N|N|N|2449597|2449839|2449252|2449526|N|N|N|N|N| +2449618|AAAAAAAACNAGFCAA|1994-09-22|1136|4943|380|1994|4|9|22|3|1994|380|4943|Thursday|1994Q3|N|N|N|2449597|2449839|2449253|2449527|N|N|N|N|N| +2449619|AAAAAAAADNAGFCAA|1994-09-23|1136|4943|380|1994|5|9|23|3|1994|380|4943|Friday|1994Q3|N|Y|N|2449597|2449839|2449254|2449528|N|N|N|N|N| +2449620|AAAAAAAAENAGFCAA|1994-09-24|1136|4943|380|1994|6|9|24|3|1994|380|4943|Saturday|1994Q3|N|Y|N|2449597|2449839|2449255|2449529|N|N|N|N|N| +2449621|AAAAAAAAFNAGFCAA|1994-09-25|1136|4943|380|1994|0|9|25|3|1994|380|4943|Sunday|1994Q3|N|N|N|2449597|2449839|2449256|2449530|N|N|N|N|N| +2449622|AAAAAAAAGNAGFCAA|1994-09-26|1136|4943|380|1994|1|9|26|3|1994|380|4943|Monday|1994Q3|N|N|N|2449597|2449839|2449257|2449531|N|N|N|N|N| +2449623|AAAAAAAAHNAGFCAA|1994-09-27|1136|4944|380|1994|2|9|27|3|1994|380|4944|Tuesday|1994Q3|N|N|N|2449597|2449839|2449258|2449532|N|N|N|N|N| +2449624|AAAAAAAAINAGFCAA|1994-09-28|1136|4944|380|1994|3|9|28|3|1994|380|4944|Wednesday|1994Q3|N|N|N|2449597|2449839|2449259|2449533|N|N|N|N|N| +2449625|AAAAAAAAJNAGFCAA|1994-09-29|1136|4944|380|1994|4|9|29|3|1994|380|4944|Thursday|1994Q3|N|N|N|2449597|2449839|2449260|2449534|N|N|N|N|N| +2449626|AAAAAAAAKNAGFCAA|1994-09-30|1136|4944|380|1994|5|9|30|3|1994|380|4944|Friday|1994Q3|N|Y|N|2449597|2449839|2449261|2449535|N|N|N|N|N| +2449627|AAAAAAAALNAGFCAA|1994-10-01|1137|4944|380|1994|6|10|1|3|1994|380|4944|Saturday|1994Q3|N|Y|N|2449627|2449899|2449262|2449535|N|N|N|N|N| +2449628|AAAAAAAAMNAGFCAA|1994-10-02|1137|4944|380|1994|0|10|2|4|1994|380|4944|Sunday|1994Q4|N|N|N|2449627|2449899|2449263|2449536|N|N|N|N|N| +2449629|AAAAAAAANNAGFCAA|1994-10-03|1137|4944|380|1994|1|10|3|4|1994|380|4944|Monday|1994Q4|N|N|N|2449627|2449899|2449264|2449537|N|N|N|N|N| +2449630|AAAAAAAAONAGFCAA|1994-10-04|1137|4945|380|1994|2|10|4|4|1994|380|4945|Tuesday|1994Q4|N|N|N|2449627|2449899|2449265|2449538|N|N|N|N|N| +2449631|AAAAAAAAPNAGFCAA|1994-10-05|1137|4945|380|1994|3|10|5|4|1994|380|4945|Wednesday|1994Q4|N|N|N|2449627|2449899|2449266|2449539|N|N|N|N|N| +2449632|AAAAAAAAAOAGFCAA|1994-10-06|1137|4945|380|1994|4|10|6|4|1994|380|4945|Thursday|1994Q4|N|N|N|2449627|2449899|2449267|2449540|N|N|N|N|N| +2449633|AAAAAAAABOAGFCAA|1994-10-07|1137|4945|380|1994|5|10|7|4|1994|380|4945|Friday|1994Q4|N|Y|N|2449627|2449899|2449268|2449541|N|N|N|N|N| +2449634|AAAAAAAACOAGFCAA|1994-10-08|1137|4945|380|1994|6|10|8|4|1994|380|4945|Saturday|1994Q4|N|Y|N|2449627|2449899|2449269|2449542|N|N|N|N|N| +2449635|AAAAAAAADOAGFCAA|1994-10-09|1137|4945|380|1994|0|10|9|4|1994|380|4945|Sunday|1994Q4|N|N|N|2449627|2449899|2449270|2449543|N|N|N|N|N| +2449636|AAAAAAAAEOAGFCAA|1994-10-10|1137|4945|380|1994|1|10|10|4|1994|380|4945|Monday|1994Q4|N|N|N|2449627|2449899|2449271|2449544|N|N|N|N|N| +2449637|AAAAAAAAFOAGFCAA|1994-10-11|1137|4946|380|1994|2|10|11|4|1994|380|4946|Tuesday|1994Q4|N|N|N|2449627|2449899|2449272|2449545|N|N|N|N|N| +2449638|AAAAAAAAGOAGFCAA|1994-10-12|1137|4946|380|1994|3|10|12|4|1994|380|4946|Wednesday|1994Q4|N|N|N|2449627|2449899|2449273|2449546|N|N|N|N|N| +2449639|AAAAAAAAHOAGFCAA|1994-10-13|1137|4946|380|1994|4|10|13|4|1994|380|4946|Thursday|1994Q4|N|N|N|2449627|2449899|2449274|2449547|N|N|N|N|N| +2449640|AAAAAAAAIOAGFCAA|1994-10-14|1137|4946|380|1994|5|10|14|4|1994|380|4946|Friday|1994Q4|N|Y|N|2449627|2449899|2449275|2449548|N|N|N|N|N| +2449641|AAAAAAAAJOAGFCAA|1994-10-15|1137|4946|380|1994|6|10|15|4|1994|380|4946|Saturday|1994Q4|N|Y|N|2449627|2449899|2449276|2449549|N|N|N|N|N| +2449642|AAAAAAAAKOAGFCAA|1994-10-16|1137|4946|380|1994|0|10|16|4|1994|380|4946|Sunday|1994Q4|N|N|N|2449627|2449899|2449277|2449550|N|N|N|N|N| +2449643|AAAAAAAALOAGFCAA|1994-10-17|1137|4946|380|1994|1|10|17|4|1994|380|4946|Monday|1994Q4|N|N|N|2449627|2449899|2449278|2449551|N|N|N|N|N| +2449644|AAAAAAAAMOAGFCAA|1994-10-18|1137|4947|380|1994|2|10|18|4|1994|380|4947|Tuesday|1994Q4|N|N|N|2449627|2449899|2449279|2449552|N|N|N|N|N| +2449645|AAAAAAAANOAGFCAA|1994-10-19|1137|4947|380|1994|3|10|19|4|1994|380|4947|Wednesday|1994Q4|N|N|N|2449627|2449899|2449280|2449553|N|N|N|N|N| +2449646|AAAAAAAAOOAGFCAA|1994-10-20|1137|4947|380|1994|4|10|20|4|1994|380|4947|Thursday|1994Q4|N|N|N|2449627|2449899|2449281|2449554|N|N|N|N|N| +2449647|AAAAAAAAPOAGFCAA|1994-10-21|1137|4947|380|1994|5|10|21|4|1994|380|4947|Friday|1994Q4|N|Y|N|2449627|2449899|2449282|2449555|N|N|N|N|N| +2449648|AAAAAAAAAPAGFCAA|1994-10-22|1137|4947|380|1994|6|10|22|4|1994|380|4947|Saturday|1994Q4|N|Y|N|2449627|2449899|2449283|2449556|N|N|N|N|N| +2449649|AAAAAAAABPAGFCAA|1994-10-23|1137|4947|380|1994|0|10|23|4|1994|380|4947|Sunday|1994Q4|N|N|N|2449627|2449899|2449284|2449557|N|N|N|N|N| +2449650|AAAAAAAACPAGFCAA|1994-10-24|1137|4947|380|1994|1|10|24|4|1994|380|4947|Monday|1994Q4|N|N|N|2449627|2449899|2449285|2449558|N|N|N|N|N| +2449651|AAAAAAAADPAGFCAA|1994-10-25|1137|4948|380|1994|2|10|25|4|1994|380|4948|Tuesday|1994Q4|N|N|N|2449627|2449899|2449286|2449559|N|N|N|N|N| +2449652|AAAAAAAAEPAGFCAA|1994-10-26|1137|4948|380|1994|3|10|26|4|1994|380|4948|Wednesday|1994Q4|N|N|N|2449627|2449899|2449287|2449560|N|N|N|N|N| +2449653|AAAAAAAAFPAGFCAA|1994-10-27|1137|4948|380|1994|4|10|27|4|1994|380|4948|Thursday|1994Q4|N|N|N|2449627|2449899|2449288|2449561|N|N|N|N|N| +2449654|AAAAAAAAGPAGFCAA|1994-10-28|1137|4948|380|1994|5|10|28|4|1994|380|4948|Friday|1994Q4|N|Y|N|2449627|2449899|2449289|2449562|N|N|N|N|N| +2449655|AAAAAAAAHPAGFCAA|1994-10-29|1137|4948|380|1994|6|10|29|4|1994|380|4948|Saturday|1994Q4|N|Y|N|2449627|2449899|2449290|2449563|N|N|N|N|N| +2449656|AAAAAAAAIPAGFCAA|1994-10-30|1137|4948|380|1994|0|10|30|4|1994|380|4948|Sunday|1994Q4|N|N|N|2449627|2449899|2449291|2449564|N|N|N|N|N| +2449657|AAAAAAAAJPAGFCAA|1994-10-31|1137|4948|380|1994|1|10|31|4|1994|380|4948|Monday|1994Q4|N|N|N|2449627|2449899|2449292|2449565|N|N|N|N|N| +2449658|AAAAAAAAKPAGFCAA|1994-11-01|1138|4949|380|1994|2|11|1|4|1994|380|4949|Tuesday|1994Q4|N|N|N|2449658|2449961|2449293|2449566|N|N|N|N|N| +2449659|AAAAAAAALPAGFCAA|1994-11-02|1138|4949|380|1994|3|11|2|4|1994|380|4949|Wednesday|1994Q4|N|N|N|2449658|2449961|2449294|2449567|N|N|N|N|N| +2449660|AAAAAAAAMPAGFCAA|1994-11-03|1138|4949|380|1994|4|11|3|4|1994|380|4949|Thursday|1994Q4|N|N|N|2449658|2449961|2449295|2449568|N|N|N|N|N| +2449661|AAAAAAAANPAGFCAA|1994-11-04|1138|4949|380|1994|5|11|4|4|1994|380|4949|Friday|1994Q4|N|Y|N|2449658|2449961|2449296|2449569|N|N|N|N|N| +2449662|AAAAAAAAOPAGFCAA|1994-11-05|1138|4949|380|1994|6|11|5|4|1994|380|4949|Saturday|1994Q4|N|Y|N|2449658|2449961|2449297|2449570|N|N|N|N|N| +2449663|AAAAAAAAPPAGFCAA|1994-11-06|1138|4949|380|1994|0|11|6|4|1994|380|4949|Sunday|1994Q4|N|N|N|2449658|2449961|2449298|2449571|N|N|N|N|N| +2449664|AAAAAAAAAABGFCAA|1994-11-07|1138|4949|380|1994|1|11|7|4|1994|380|4949|Monday|1994Q4|N|N|N|2449658|2449961|2449299|2449572|N|N|N|N|N| +2449665|AAAAAAAABABGFCAA|1994-11-08|1138|4950|380|1994|2|11|8|4|1994|380|4950|Tuesday|1994Q4|N|N|N|2449658|2449961|2449300|2449573|N|N|N|N|N| +2449666|AAAAAAAACABGFCAA|1994-11-09|1138|4950|380|1994|3|11|9|4|1994|380|4950|Wednesday|1994Q4|N|N|N|2449658|2449961|2449301|2449574|N|N|N|N|N| +2449667|AAAAAAAADABGFCAA|1994-11-10|1138|4950|380|1994|4|11|10|4|1994|380|4950|Thursday|1994Q4|N|N|N|2449658|2449961|2449302|2449575|N|N|N|N|N| +2449668|AAAAAAAAEABGFCAA|1994-11-11|1138|4950|380|1994|5|11|11|4|1994|380|4950|Friday|1994Q4|N|Y|N|2449658|2449961|2449303|2449576|N|N|N|N|N| +2449669|AAAAAAAAFABGFCAA|1994-11-12|1138|4950|380|1994|6|11|12|4|1994|380|4950|Saturday|1994Q4|N|Y|N|2449658|2449961|2449304|2449577|N|N|N|N|N| +2449670|AAAAAAAAGABGFCAA|1994-11-13|1138|4950|380|1994|0|11|13|4|1994|380|4950|Sunday|1994Q4|N|N|N|2449658|2449961|2449305|2449578|N|N|N|N|N| +2449671|AAAAAAAAHABGFCAA|1994-11-14|1138|4950|380|1994|1|11|14|4|1994|380|4950|Monday|1994Q4|N|N|N|2449658|2449961|2449306|2449579|N|N|N|N|N| +2449672|AAAAAAAAIABGFCAA|1994-11-15|1138|4951|380|1994|2|11|15|4|1994|380|4951|Tuesday|1994Q4|N|N|N|2449658|2449961|2449307|2449580|N|N|N|N|N| +2449673|AAAAAAAAJABGFCAA|1994-11-16|1138|4951|380|1994|3|11|16|4|1994|380|4951|Wednesday|1994Q4|N|N|N|2449658|2449961|2449308|2449581|N|N|N|N|N| +2449674|AAAAAAAAKABGFCAA|1994-11-17|1138|4951|380|1994|4|11|17|4|1994|380|4951|Thursday|1994Q4|N|N|N|2449658|2449961|2449309|2449582|N|N|N|N|N| +2449675|AAAAAAAALABGFCAA|1994-11-18|1138|4951|380|1994|5|11|18|4|1994|380|4951|Friday|1994Q4|N|Y|N|2449658|2449961|2449310|2449583|N|N|N|N|N| +2449676|AAAAAAAAMABGFCAA|1994-11-19|1138|4951|380|1994|6|11|19|4|1994|380|4951|Saturday|1994Q4|N|Y|N|2449658|2449961|2449311|2449584|N|N|N|N|N| +2449677|AAAAAAAANABGFCAA|1994-11-20|1138|4951|380|1994|0|11|20|4|1994|380|4951|Sunday|1994Q4|N|N|N|2449658|2449961|2449312|2449585|N|N|N|N|N| +2449678|AAAAAAAAOABGFCAA|1994-11-21|1138|4951|380|1994|1|11|21|4|1994|380|4951|Monday|1994Q4|N|N|N|2449658|2449961|2449313|2449586|N|N|N|N|N| +2449679|AAAAAAAAPABGFCAA|1994-11-22|1138|4952|380|1994|2|11|22|4|1994|380|4952|Tuesday|1994Q4|N|N|N|2449658|2449961|2449314|2449587|N|N|N|N|N| +2449680|AAAAAAAAABBGFCAA|1994-11-23|1138|4952|380|1994|3|11|23|4|1994|380|4952|Wednesday|1994Q4|N|N|N|2449658|2449961|2449315|2449588|N|N|N|N|N| +2449681|AAAAAAAABBBGFCAA|1994-11-24|1138|4952|380|1994|4|11|24|4|1994|380|4952|Thursday|1994Q4|N|N|N|2449658|2449961|2449316|2449589|N|N|N|N|N| +2449682|AAAAAAAACBBGFCAA|1994-11-25|1138|4952|380|1994|5|11|25|4|1994|380|4952|Friday|1994Q4|N|Y|N|2449658|2449961|2449317|2449590|N|N|N|N|N| +2449683|AAAAAAAADBBGFCAA|1994-11-26|1138|4952|380|1994|6|11|26|4|1994|380|4952|Saturday|1994Q4|N|Y|N|2449658|2449961|2449318|2449591|N|N|N|N|N| +2449684|AAAAAAAAEBBGFCAA|1994-11-27|1138|4952|380|1994|0|11|27|4|1994|380|4952|Sunday|1994Q4|N|N|N|2449658|2449961|2449319|2449592|N|N|N|N|N| +2449685|AAAAAAAAFBBGFCAA|1994-11-28|1138|4952|380|1994|1|11|28|4|1994|380|4952|Monday|1994Q4|N|N|N|2449658|2449961|2449320|2449593|N|N|N|N|N| +2449686|AAAAAAAAGBBGFCAA|1994-11-29|1138|4953|380|1994|2|11|29|4|1994|380|4953|Tuesday|1994Q4|N|N|N|2449658|2449961|2449321|2449594|N|N|N|N|N| +2449687|AAAAAAAAHBBGFCAA|1994-11-30|1138|4953|380|1994|3|11|30|4|1994|380|4953|Wednesday|1994Q4|N|N|N|2449658|2449961|2449322|2449595|N|N|N|N|N| +2449688|AAAAAAAAIBBGFCAA|1994-12-01|1139|4953|381|1994|4|12|1|4|1994|381|4953|Thursday|1994Q4|N|N|N|2449688|2450021|2449323|2449596|N|N|N|N|N| +2449689|AAAAAAAAJBBGFCAA|1994-12-02|1139|4953|381|1994|5|12|2|4|1994|381|4953|Friday|1994Q4|N|Y|N|2449688|2450021|2449324|2449597|N|N|N|N|N| +2449690|AAAAAAAAKBBGFCAA|1994-12-03|1139|4953|381|1994|6|12|3|4|1994|381|4953|Saturday|1994Q4|N|Y|N|2449688|2450021|2449325|2449598|N|N|N|N|N| +2449691|AAAAAAAALBBGFCAA|1994-12-04|1139|4953|381|1994|0|12|4|4|1994|381|4953|Sunday|1994Q4|N|N|N|2449688|2450021|2449326|2449599|N|N|N|N|N| +2449692|AAAAAAAAMBBGFCAA|1994-12-05|1139|4953|381|1994|1|12|5|4|1994|381|4953|Monday|1994Q4|N|N|N|2449688|2450021|2449327|2449600|N|N|N|N|N| +2449693|AAAAAAAANBBGFCAA|1994-12-06|1139|4954|381|1994|2|12|6|4|1994|381|4954|Tuesday|1994Q4|N|N|N|2449688|2450021|2449328|2449601|N|N|N|N|N| +2449694|AAAAAAAAOBBGFCAA|1994-12-07|1139|4954|381|1994|3|12|7|4|1994|381|4954|Wednesday|1994Q4|N|N|N|2449688|2450021|2449329|2449602|N|N|N|N|N| +2449695|AAAAAAAAPBBGFCAA|1994-12-08|1139|4954|381|1994|4|12|8|4|1994|381|4954|Thursday|1994Q4|N|N|N|2449688|2450021|2449330|2449603|N|N|N|N|N| +2449696|AAAAAAAAACBGFCAA|1994-12-09|1139|4954|381|1994|5|12|9|4|1994|381|4954|Friday|1994Q4|N|Y|N|2449688|2450021|2449331|2449604|N|N|N|N|N| +2449697|AAAAAAAABCBGFCAA|1994-12-10|1139|4954|381|1994|6|12|10|4|1994|381|4954|Saturday|1994Q4|N|Y|N|2449688|2450021|2449332|2449605|N|N|N|N|N| +2449698|AAAAAAAACCBGFCAA|1994-12-11|1139|4954|381|1994|0|12|11|4|1994|381|4954|Sunday|1994Q4|N|N|N|2449688|2450021|2449333|2449606|N|N|N|N|N| +2449699|AAAAAAAADCBGFCAA|1994-12-12|1139|4954|381|1994|1|12|12|4|1994|381|4954|Monday|1994Q4|N|N|N|2449688|2450021|2449334|2449607|N|N|N|N|N| +2449700|AAAAAAAAECBGFCAA|1994-12-13|1139|4955|381|1994|2|12|13|4|1994|381|4955|Tuesday|1994Q4|N|N|N|2449688|2450021|2449335|2449608|N|N|N|N|N| +2449701|AAAAAAAAFCBGFCAA|1994-12-14|1139|4955|381|1994|3|12|14|4|1994|381|4955|Wednesday|1994Q4|N|N|N|2449688|2450021|2449336|2449609|N|N|N|N|N| +2449702|AAAAAAAAGCBGFCAA|1994-12-15|1139|4955|381|1994|4|12|15|4|1994|381|4955|Thursday|1994Q4|N|N|N|2449688|2450021|2449337|2449610|N|N|N|N|N| +2449703|AAAAAAAAHCBGFCAA|1994-12-16|1139|4955|381|1994|5|12|16|4|1994|381|4955|Friday|1994Q4|N|Y|N|2449688|2450021|2449338|2449611|N|N|N|N|N| +2449704|AAAAAAAAICBGFCAA|1994-12-17|1139|4955|381|1994|6|12|17|4|1994|381|4955|Saturday|1994Q4|N|Y|N|2449688|2450021|2449339|2449612|N|N|N|N|N| +2449705|AAAAAAAAJCBGFCAA|1994-12-18|1139|4955|381|1994|0|12|18|4|1994|381|4955|Sunday|1994Q4|N|N|N|2449688|2450021|2449340|2449613|N|N|N|N|N| +2449706|AAAAAAAAKCBGFCAA|1994-12-19|1139|4955|381|1994|1|12|19|4|1994|381|4955|Monday|1994Q4|N|N|N|2449688|2450021|2449341|2449614|N|N|N|N|N| +2449707|AAAAAAAALCBGFCAA|1994-12-20|1139|4956|381|1994|2|12|20|4|1994|381|4956|Tuesday|1994Q4|N|N|N|2449688|2450021|2449342|2449615|N|N|N|N|N| +2449708|AAAAAAAAMCBGFCAA|1994-12-21|1139|4956|381|1994|3|12|21|4|1994|381|4956|Wednesday|1994Q4|N|N|N|2449688|2450021|2449343|2449616|N|N|N|N|N| +2449709|AAAAAAAANCBGFCAA|1994-12-22|1139|4956|381|1994|4|12|22|4|1994|381|4956|Thursday|1994Q4|N|N|N|2449688|2450021|2449344|2449617|N|N|N|N|N| +2449710|AAAAAAAAOCBGFCAA|1994-12-23|1139|4956|381|1994|5|12|23|4|1994|381|4956|Friday|1994Q4|N|Y|N|2449688|2450021|2449345|2449618|N|N|N|N|N| +2449711|AAAAAAAAPCBGFCAA|1994-12-24|1139|4956|381|1994|6|12|24|4|1994|381|4956|Saturday|1994Q4|N|Y|N|2449688|2450021|2449346|2449619|N|N|N|N|N| +2449712|AAAAAAAAADBGFCAA|1994-12-25|1139|4956|381|1994|0|12|25|4|1994|381|4956|Sunday|1994Q4|N|N|N|2449688|2450021|2449347|2449620|N|N|N|N|N| +2449713|AAAAAAAABDBGFCAA|1994-12-26|1139|4956|381|1994|1|12|26|4|1994|381|4956|Monday|1994Q4|Y|N|N|2449688|2450021|2449348|2449621|N|N|N|N|N| +2449714|AAAAAAAACDBGFCAA|1994-12-27|1139|4957|381|1994|2|12|27|4|1994|381|4957|Tuesday|1994Q4|N|N|Y|2449688|2450021|2449349|2449622|N|N|N|N|N| +2449715|AAAAAAAADDBGFCAA|1994-12-28|1139|4957|381|1994|3|12|28|4|1994|381|4957|Wednesday|1994Q4|N|N|N|2449688|2450021|2449350|2449623|N|N|N|N|N| +2449716|AAAAAAAAEDBGFCAA|1994-12-29|1139|4957|381|1994|4|12|29|4|1994|381|4957|Thursday|1994Q4|N|N|N|2449688|2450021|2449351|2449624|N|N|N|N|N| +2449717|AAAAAAAAFDBGFCAA|1994-12-30|1139|4957|381|1994|5|12|30|4|1994|381|4957|Friday|1994Q4|N|Y|N|2449688|2450021|2449352|2449625|N|N|N|N|N| +2449718|AAAAAAAAGDBGFCAA|1994-12-31|1139|4957|381|1994|6|12|31|4|1994|381|4957|Saturday|1994Q4|N|Y|N|2449688|2450021|2449353|2449626|N|N|N|N|N| +2449719|AAAAAAAAHDBGFCAA|1995-01-01|1140|4957|381|1995|0|1|1|1|1995|381|4957|Sunday|1995Q1|Y|N|N|2449719|2449718|2449354|2449627|N|N|N|N|N| +2449720|AAAAAAAAIDBGFCAA|1995-01-02|1140|4957|381|1995|1|1|2|1|1995|381|4957|Monday|1995Q1|N|N|Y|2449719|2449718|2449355|2449628|N|N|N|N|N| +2449721|AAAAAAAAJDBGFCAA|1995-01-03|1140|4958|381|1995|2|1|3|1|1995|381|4958|Tuesday|1995Q1|N|N|N|2449719|2449718|2449356|2449629|N|N|N|N|N| +2449722|AAAAAAAAKDBGFCAA|1995-01-04|1140|4958|381|1995|3|1|4|1|1995|381|4958|Wednesday|1995Q1|N|N|N|2449719|2449718|2449357|2449630|N|N|N|N|N| +2449723|AAAAAAAALDBGFCAA|1995-01-05|1140|4958|381|1995|4|1|5|1|1995|381|4958|Thursday|1995Q1|N|N|N|2449719|2449718|2449358|2449631|N|N|N|N|N| +2449724|AAAAAAAAMDBGFCAA|1995-01-06|1140|4958|381|1995|5|1|6|1|1995|381|4958|Friday|1995Q1|N|Y|N|2449719|2449718|2449359|2449632|N|N|N|N|N| +2449725|AAAAAAAANDBGFCAA|1995-01-07|1140|4958|381|1995|6|1|7|1|1995|381|4958|Saturday|1995Q1|N|Y|N|2449719|2449718|2449360|2449633|N|N|N|N|N| +2449726|AAAAAAAAODBGFCAA|1995-01-08|1140|4958|381|1995|0|1|8|1|1995|381|4958|Sunday|1995Q1|N|N|N|2449719|2449718|2449361|2449634|N|N|N|N|N| +2449727|AAAAAAAAPDBGFCAA|1995-01-09|1140|4958|381|1995|1|1|9|1|1995|381|4958|Monday|1995Q1|N|N|N|2449719|2449718|2449362|2449635|N|N|N|N|N| +2449728|AAAAAAAAAEBGFCAA|1995-01-10|1140|4959|381|1995|2|1|10|1|1995|381|4959|Tuesday|1995Q1|N|N|N|2449719|2449718|2449363|2449636|N|N|N|N|N| +2449729|AAAAAAAABEBGFCAA|1995-01-11|1140|4959|381|1995|3|1|11|1|1995|381|4959|Wednesday|1995Q1|N|N|N|2449719|2449718|2449364|2449637|N|N|N|N|N| +2449730|AAAAAAAACEBGFCAA|1995-01-12|1140|4959|381|1995|4|1|12|1|1995|381|4959|Thursday|1995Q1|N|N|N|2449719|2449718|2449365|2449638|N|N|N|N|N| +2449731|AAAAAAAADEBGFCAA|1995-01-13|1140|4959|381|1995|5|1|13|1|1995|381|4959|Friday|1995Q1|N|Y|N|2449719|2449718|2449366|2449639|N|N|N|N|N| +2449732|AAAAAAAAEEBGFCAA|1995-01-14|1140|4959|381|1995|6|1|14|1|1995|381|4959|Saturday|1995Q1|N|Y|N|2449719|2449718|2449367|2449640|N|N|N|N|N| +2449733|AAAAAAAAFEBGFCAA|1995-01-15|1140|4959|381|1995|0|1|15|1|1995|381|4959|Sunday|1995Q1|N|N|N|2449719|2449718|2449368|2449641|N|N|N|N|N| +2449734|AAAAAAAAGEBGFCAA|1995-01-16|1140|4959|381|1995|1|1|16|1|1995|381|4959|Monday|1995Q1|N|N|N|2449719|2449718|2449369|2449642|N|N|N|N|N| +2449735|AAAAAAAAHEBGFCAA|1995-01-17|1140|4960|381|1995|2|1|17|1|1995|381|4960|Tuesday|1995Q1|N|N|N|2449719|2449718|2449370|2449643|N|N|N|N|N| +2449736|AAAAAAAAIEBGFCAA|1995-01-18|1140|4960|381|1995|3|1|18|1|1995|381|4960|Wednesday|1995Q1|N|N|N|2449719|2449718|2449371|2449644|N|N|N|N|N| +2449737|AAAAAAAAJEBGFCAA|1995-01-19|1140|4960|381|1995|4|1|19|1|1995|381|4960|Thursday|1995Q1|N|N|N|2449719|2449718|2449372|2449645|N|N|N|N|N| +2449738|AAAAAAAAKEBGFCAA|1995-01-20|1140|4960|381|1995|5|1|20|1|1995|381|4960|Friday|1995Q1|N|Y|N|2449719|2449718|2449373|2449646|N|N|N|N|N| +2449739|AAAAAAAALEBGFCAA|1995-01-21|1140|4960|381|1995|6|1|21|1|1995|381|4960|Saturday|1995Q1|N|Y|N|2449719|2449718|2449374|2449647|N|N|N|N|N| +2449740|AAAAAAAAMEBGFCAA|1995-01-22|1140|4960|381|1995|0|1|22|1|1995|381|4960|Sunday|1995Q1|N|N|N|2449719|2449718|2449375|2449648|N|N|N|N|N| +2449741|AAAAAAAANEBGFCAA|1995-01-23|1140|4960|381|1995|1|1|23|1|1995|381|4960|Monday|1995Q1|N|N|N|2449719|2449718|2449376|2449649|N|N|N|N|N| +2449742|AAAAAAAAOEBGFCAA|1995-01-24|1140|4961|381|1995|2|1|24|1|1995|381|4961|Tuesday|1995Q1|N|N|N|2449719|2449718|2449377|2449650|N|N|N|N|N| +2449743|AAAAAAAAPEBGFCAA|1995-01-25|1140|4961|381|1995|3|1|25|1|1995|381|4961|Wednesday|1995Q1|N|N|N|2449719|2449718|2449378|2449651|N|N|N|N|N| +2449744|AAAAAAAAAFBGFCAA|1995-01-26|1140|4961|381|1995|4|1|26|1|1995|381|4961|Thursday|1995Q1|N|N|N|2449719|2449718|2449379|2449652|N|N|N|N|N| +2449745|AAAAAAAABFBGFCAA|1995-01-27|1140|4961|381|1995|5|1|27|1|1995|381|4961|Friday|1995Q1|N|Y|N|2449719|2449718|2449380|2449653|N|N|N|N|N| +2449746|AAAAAAAACFBGFCAA|1995-01-28|1140|4961|381|1995|6|1|28|1|1995|381|4961|Saturday|1995Q1|N|Y|N|2449719|2449718|2449381|2449654|N|N|N|N|N| +2449747|AAAAAAAADFBGFCAA|1995-01-29|1140|4961|381|1995|0|1|29|1|1995|381|4961|Sunday|1995Q1|N|N|N|2449719|2449718|2449382|2449655|N|N|N|N|N| +2449748|AAAAAAAAEFBGFCAA|1995-01-30|1140|4961|381|1995|1|1|30|1|1995|381|4961|Monday|1995Q1|N|N|N|2449719|2449718|2449383|2449656|N|N|N|N|N| +2449749|AAAAAAAAFFBGFCAA|1995-01-31|1140|4962|381|1995|2|1|31|1|1995|381|4962|Tuesday|1995Q1|N|N|N|2449719|2449718|2449384|2449657|N|N|N|N|N| +2449750|AAAAAAAAGFBGFCAA|1995-02-01|1141|4962|381|1995|3|2|1|1|1995|381|4962|Wednesday|1995Q1|N|N|N|2449750|2449780|2449385|2449658|N|N|N|N|N| +2449751|AAAAAAAAHFBGFCAA|1995-02-02|1141|4962|381|1995|4|2|2|1|1995|381|4962|Thursday|1995Q1|N|N|N|2449750|2449780|2449386|2449659|N|N|N|N|N| +2449752|AAAAAAAAIFBGFCAA|1995-02-03|1141|4962|381|1995|5|2|3|1|1995|381|4962|Friday|1995Q1|N|Y|N|2449750|2449780|2449387|2449660|N|N|N|N|N| +2449753|AAAAAAAAJFBGFCAA|1995-02-04|1141|4962|381|1995|6|2|4|1|1995|381|4962|Saturday|1995Q1|N|Y|N|2449750|2449780|2449388|2449661|N|N|N|N|N| +2449754|AAAAAAAAKFBGFCAA|1995-02-05|1141|4962|381|1995|0|2|5|1|1995|381|4962|Sunday|1995Q1|N|N|N|2449750|2449780|2449389|2449662|N|N|N|N|N| +2449755|AAAAAAAALFBGFCAA|1995-02-06|1141|4962|381|1995|1|2|6|1|1995|381|4962|Monday|1995Q1|N|N|N|2449750|2449780|2449390|2449663|N|N|N|N|N| +2449756|AAAAAAAAMFBGFCAA|1995-02-07|1141|4963|381|1995|2|2|7|1|1995|381|4963|Tuesday|1995Q1|N|N|N|2449750|2449780|2449391|2449664|N|N|N|N|N| +2449757|AAAAAAAANFBGFCAA|1995-02-08|1141|4963|381|1995|3|2|8|1|1995|381|4963|Wednesday|1995Q1|N|N|N|2449750|2449780|2449392|2449665|N|N|N|N|N| +2449758|AAAAAAAAOFBGFCAA|1995-02-09|1141|4963|381|1995|4|2|9|1|1995|381|4963|Thursday|1995Q1|N|N|N|2449750|2449780|2449393|2449666|N|N|N|N|N| +2449759|AAAAAAAAPFBGFCAA|1995-02-10|1141|4963|381|1995|5|2|10|1|1995|381|4963|Friday|1995Q1|N|Y|N|2449750|2449780|2449394|2449667|N|N|N|N|N| +2449760|AAAAAAAAAGBGFCAA|1995-02-11|1141|4963|381|1995|6|2|11|1|1995|381|4963|Saturday|1995Q1|N|Y|N|2449750|2449780|2449395|2449668|N|N|N|N|N| +2449761|AAAAAAAABGBGFCAA|1995-02-12|1141|4963|381|1995|0|2|12|1|1995|381|4963|Sunday|1995Q1|N|N|N|2449750|2449780|2449396|2449669|N|N|N|N|N| +2449762|AAAAAAAACGBGFCAA|1995-02-13|1141|4963|381|1995|1|2|13|1|1995|381|4963|Monday|1995Q1|N|N|N|2449750|2449780|2449397|2449670|N|N|N|N|N| +2449763|AAAAAAAADGBGFCAA|1995-02-14|1141|4964|381|1995|2|2|14|1|1995|381|4964|Tuesday|1995Q1|N|N|N|2449750|2449780|2449398|2449671|N|N|N|N|N| +2449764|AAAAAAAAEGBGFCAA|1995-02-15|1141|4964|381|1995|3|2|15|1|1995|381|4964|Wednesday|1995Q1|N|N|N|2449750|2449780|2449399|2449672|N|N|N|N|N| +2449765|AAAAAAAAFGBGFCAA|1995-02-16|1141|4964|381|1995|4|2|16|1|1995|381|4964|Thursday|1995Q1|N|N|N|2449750|2449780|2449400|2449673|N|N|N|N|N| +2449766|AAAAAAAAGGBGFCAA|1995-02-17|1141|4964|381|1995|5|2|17|1|1995|381|4964|Friday|1995Q1|N|Y|N|2449750|2449780|2449401|2449674|N|N|N|N|N| +2449767|AAAAAAAAHGBGFCAA|1995-02-18|1141|4964|381|1995|6|2|18|1|1995|381|4964|Saturday|1995Q1|N|Y|N|2449750|2449780|2449402|2449675|N|N|N|N|N| +2449768|AAAAAAAAIGBGFCAA|1995-02-19|1141|4964|381|1995|0|2|19|1|1995|381|4964|Sunday|1995Q1|N|N|N|2449750|2449780|2449403|2449676|N|N|N|N|N| +2449769|AAAAAAAAJGBGFCAA|1995-02-20|1141|4964|381|1995|1|2|20|1|1995|381|4964|Monday|1995Q1|N|N|N|2449750|2449780|2449404|2449677|N|N|N|N|N| +2449770|AAAAAAAAKGBGFCAA|1995-02-21|1141|4965|381|1995|2|2|21|1|1995|381|4965|Tuesday|1995Q1|N|N|N|2449750|2449780|2449405|2449678|N|N|N|N|N| +2449771|AAAAAAAALGBGFCAA|1995-02-22|1141|4965|381|1995|3|2|22|1|1995|381|4965|Wednesday|1995Q1|N|N|N|2449750|2449780|2449406|2449679|N|N|N|N|N| +2449772|AAAAAAAAMGBGFCAA|1995-02-23|1141|4965|381|1995|4|2|23|1|1995|381|4965|Thursday|1995Q1|N|N|N|2449750|2449780|2449407|2449680|N|N|N|N|N| +2449773|AAAAAAAANGBGFCAA|1995-02-24|1141|4965|381|1995|5|2|24|1|1995|381|4965|Friday|1995Q1|N|Y|N|2449750|2449780|2449408|2449681|N|N|N|N|N| +2449774|AAAAAAAAOGBGFCAA|1995-02-25|1141|4965|381|1995|6|2|25|1|1995|381|4965|Saturday|1995Q1|N|Y|N|2449750|2449780|2449409|2449682|N|N|N|N|N| +2449775|AAAAAAAAPGBGFCAA|1995-02-26|1141|4965|381|1995|0|2|26|1|1995|381|4965|Sunday|1995Q1|N|N|N|2449750|2449780|2449410|2449683|N|N|N|N|N| +2449776|AAAAAAAAAHBGFCAA|1995-02-27|1141|4965|381|1995|1|2|27|1|1995|381|4965|Monday|1995Q1|N|N|N|2449750|2449780|2449411|2449684|N|N|N|N|N| +2449777|AAAAAAAABHBGFCAA|1995-02-28|1141|4966|381|1995|2|2|28|1|1995|381|4966|Tuesday|1995Q1|N|N|N|2449750|2449780|2449412|2449685|N|N|N|N|N| +2449778|AAAAAAAACHBGFCAA|1995-03-01|1142|4966|382|1995|3|3|1|1|1995|382|4966|Wednesday|1995Q1|N|N|N|2449778|2449836|2449413|2449686|N|N|N|N|N| +2449779|AAAAAAAADHBGFCAA|1995-03-02|1142|4966|382|1995|4|3|2|1|1995|382|4966|Thursday|1995Q1|N|N|N|2449778|2449836|2449414|2449687|N|N|N|N|N| +2449780|AAAAAAAAEHBGFCAA|1995-03-03|1142|4966|382|1995|5|3|3|1|1995|382|4966|Friday|1995Q1|N|Y|N|2449778|2449836|2449415|2449688|N|N|N|N|N| +2449781|AAAAAAAAFHBGFCAA|1995-03-04|1142|4966|382|1995|6|3|4|1|1995|382|4966|Saturday|1995Q1|N|Y|N|2449778|2449836|2449416|2449689|N|N|N|N|N| +2449782|AAAAAAAAGHBGFCAA|1995-03-05|1142|4966|382|1995|0|3|5|1|1995|382|4966|Sunday|1995Q1|N|N|N|2449778|2449836|2449417|2449690|N|N|N|N|N| +2449783|AAAAAAAAHHBGFCAA|1995-03-06|1142|4966|382|1995|1|3|6|1|1995|382|4966|Monday|1995Q1|N|N|N|2449778|2449836|2449418|2449691|N|N|N|N|N| +2449784|AAAAAAAAIHBGFCAA|1995-03-07|1142|4967|382|1995|2|3|7|1|1995|382|4967|Tuesday|1995Q1|N|N|N|2449778|2449836|2449419|2449692|N|N|N|N|N| +2449785|AAAAAAAAJHBGFCAA|1995-03-08|1142|4967|382|1995|3|3|8|1|1995|382|4967|Wednesday|1995Q1|N|N|N|2449778|2449836|2449420|2449693|N|N|N|N|N| +2449786|AAAAAAAAKHBGFCAA|1995-03-09|1142|4967|382|1995|4|3|9|1|1995|382|4967|Thursday|1995Q1|N|N|N|2449778|2449836|2449421|2449694|N|N|N|N|N| +2449787|AAAAAAAALHBGFCAA|1995-03-10|1142|4967|382|1995|5|3|10|1|1995|382|4967|Friday|1995Q1|N|Y|N|2449778|2449836|2449422|2449695|N|N|N|N|N| +2449788|AAAAAAAAMHBGFCAA|1995-03-11|1142|4967|382|1995|6|3|11|1|1995|382|4967|Saturday|1995Q1|N|Y|N|2449778|2449836|2449423|2449696|N|N|N|N|N| +2449789|AAAAAAAANHBGFCAA|1995-03-12|1142|4967|382|1995|0|3|12|1|1995|382|4967|Sunday|1995Q1|N|N|N|2449778|2449836|2449424|2449697|N|N|N|N|N| +2449790|AAAAAAAAOHBGFCAA|1995-03-13|1142|4967|382|1995|1|3|13|1|1995|382|4967|Monday|1995Q1|N|N|N|2449778|2449836|2449425|2449698|N|N|N|N|N| +2449791|AAAAAAAAPHBGFCAA|1995-03-14|1142|4968|382|1995|2|3|14|1|1995|382|4968|Tuesday|1995Q1|N|N|N|2449778|2449836|2449426|2449699|N|N|N|N|N| +2449792|AAAAAAAAAIBGFCAA|1995-03-15|1142|4968|382|1995|3|3|15|1|1995|382|4968|Wednesday|1995Q1|N|N|N|2449778|2449836|2449427|2449700|N|N|N|N|N| +2449793|AAAAAAAABIBGFCAA|1995-03-16|1142|4968|382|1995|4|3|16|1|1995|382|4968|Thursday|1995Q1|N|N|N|2449778|2449836|2449428|2449701|N|N|N|N|N| +2449794|AAAAAAAACIBGFCAA|1995-03-17|1142|4968|382|1995|5|3|17|1|1995|382|4968|Friday|1995Q1|N|Y|N|2449778|2449836|2449429|2449702|N|N|N|N|N| +2449795|AAAAAAAADIBGFCAA|1995-03-18|1142|4968|382|1995|6|3|18|1|1995|382|4968|Saturday|1995Q1|N|Y|N|2449778|2449836|2449430|2449703|N|N|N|N|N| +2449796|AAAAAAAAEIBGFCAA|1995-03-19|1142|4968|382|1995|0|3|19|1|1995|382|4968|Sunday|1995Q1|N|N|N|2449778|2449836|2449431|2449704|N|N|N|N|N| +2449797|AAAAAAAAFIBGFCAA|1995-03-20|1142|4968|382|1995|1|3|20|1|1995|382|4968|Monday|1995Q1|N|N|N|2449778|2449836|2449432|2449705|N|N|N|N|N| +2449798|AAAAAAAAGIBGFCAA|1995-03-21|1142|4969|382|1995|2|3|21|1|1995|382|4969|Tuesday|1995Q1|N|N|N|2449778|2449836|2449433|2449706|N|N|N|N|N| +2449799|AAAAAAAAHIBGFCAA|1995-03-22|1142|4969|382|1995|3|3|22|1|1995|382|4969|Wednesday|1995Q1|N|N|N|2449778|2449836|2449434|2449707|N|N|N|N|N| +2449800|AAAAAAAAIIBGFCAA|1995-03-23|1142|4969|382|1995|4|3|23|1|1995|382|4969|Thursday|1995Q1|N|N|N|2449778|2449836|2449435|2449708|N|N|N|N|N| +2449801|AAAAAAAAJIBGFCAA|1995-03-24|1142|4969|382|1995|5|3|24|1|1995|382|4969|Friday|1995Q1|N|Y|N|2449778|2449836|2449436|2449709|N|N|N|N|N| +2449802|AAAAAAAAKIBGFCAA|1995-03-25|1142|4969|382|1995|6|3|25|1|1995|382|4969|Saturday|1995Q1|N|Y|N|2449778|2449836|2449437|2449710|N|N|N|N|N| +2449803|AAAAAAAALIBGFCAA|1995-03-26|1142|4969|382|1995|0|3|26|1|1995|382|4969|Sunday|1995Q1|N|N|N|2449778|2449836|2449438|2449711|N|N|N|N|N| +2449804|AAAAAAAAMIBGFCAA|1995-03-27|1142|4969|382|1995|1|3|27|1|1995|382|4969|Monday|1995Q1|N|N|N|2449778|2449836|2449439|2449712|N|N|N|N|N| +2449805|AAAAAAAANIBGFCAA|1995-03-28|1142|4970|382|1995|2|3|28|1|1995|382|4970|Tuesday|1995Q1|N|N|N|2449778|2449836|2449440|2449713|N|N|N|N|N| +2449806|AAAAAAAAOIBGFCAA|1995-03-29|1142|4970|382|1995|3|3|29|1|1995|382|4970|Wednesday|1995Q1|N|N|N|2449778|2449836|2449441|2449714|N|N|N|N|N| +2449807|AAAAAAAAPIBGFCAA|1995-03-30|1142|4970|382|1995|4|3|30|1|1995|382|4970|Thursday|1995Q1|N|N|N|2449778|2449836|2449442|2449715|N|N|N|N|N| +2449808|AAAAAAAAAJBGFCAA|1995-03-31|1142|4970|382|1995|5|3|31|1|1995|382|4970|Friday|1995Q1|N|Y|N|2449778|2449836|2449443|2449716|N|N|N|N|N| +2449809|AAAAAAAABJBGFCAA|1995-04-01|1143|4970|382|1995|6|4|1|1|1995|382|4970|Saturday|1995Q1|N|Y|N|2449809|2449898|2449444|2449719|N|N|N|N|N| +2449810|AAAAAAAACJBGFCAA|1995-04-02|1143|4970|382|1995|0|4|2|2|1995|382|4970|Sunday|1995Q2|N|N|N|2449809|2449898|2449445|2449720|N|N|N|N|N| +2449811|AAAAAAAADJBGFCAA|1995-04-03|1143|4970|382|1995|1|4|3|2|1995|382|4970|Monday|1995Q2|N|N|N|2449809|2449898|2449446|2449721|N|N|N|N|N| +2449812|AAAAAAAAEJBGFCAA|1995-04-04|1143|4971|382|1995|2|4|4|2|1995|382|4971|Tuesday|1995Q2|N|N|N|2449809|2449898|2449447|2449722|N|N|N|N|N| +2449813|AAAAAAAAFJBGFCAA|1995-04-05|1143|4971|382|1995|3|4|5|2|1995|382|4971|Wednesday|1995Q2|N|N|N|2449809|2449898|2449448|2449723|N|N|N|N|N| +2449814|AAAAAAAAGJBGFCAA|1995-04-06|1143|4971|382|1995|4|4|6|2|1995|382|4971|Thursday|1995Q2|N|N|N|2449809|2449898|2449449|2449724|N|N|N|N|N| +2449815|AAAAAAAAHJBGFCAA|1995-04-07|1143|4971|382|1995|5|4|7|2|1995|382|4971|Friday|1995Q2|N|Y|N|2449809|2449898|2449450|2449725|N|N|N|N|N| +2449816|AAAAAAAAIJBGFCAA|1995-04-08|1143|4971|382|1995|6|4|8|2|1995|382|4971|Saturday|1995Q2|N|Y|N|2449809|2449898|2449451|2449726|N|N|N|N|N| +2449817|AAAAAAAAJJBGFCAA|1995-04-09|1143|4971|382|1995|0|4|9|2|1995|382|4971|Sunday|1995Q2|N|N|N|2449809|2449898|2449452|2449727|N|N|N|N|N| +2449818|AAAAAAAAKJBGFCAA|1995-04-10|1143|4971|382|1995|1|4|10|2|1995|382|4971|Monday|1995Q2|N|N|N|2449809|2449898|2449453|2449728|N|N|N|N|N| +2449819|AAAAAAAALJBGFCAA|1995-04-11|1143|4972|382|1995|2|4|11|2|1995|382|4972|Tuesday|1995Q2|N|N|N|2449809|2449898|2449454|2449729|N|N|N|N|N| +2449820|AAAAAAAAMJBGFCAA|1995-04-12|1143|4972|382|1995|3|4|12|2|1995|382|4972|Wednesday|1995Q2|N|N|N|2449809|2449898|2449455|2449730|N|N|N|N|N| +2449821|AAAAAAAANJBGFCAA|1995-04-13|1143|4972|382|1995|4|4|13|2|1995|382|4972|Thursday|1995Q2|N|N|N|2449809|2449898|2449456|2449731|N|N|N|N|N| +2449822|AAAAAAAAOJBGFCAA|1995-04-14|1143|4972|382|1995|5|4|14|2|1995|382|4972|Friday|1995Q2|N|Y|N|2449809|2449898|2449457|2449732|N|N|N|N|N| +2449823|AAAAAAAAPJBGFCAA|1995-04-15|1143|4972|382|1995|6|4|15|2|1995|382|4972|Saturday|1995Q2|N|Y|N|2449809|2449898|2449458|2449733|N|N|N|N|N| +2449824|AAAAAAAAAKBGFCAA|1995-04-16|1143|4972|382|1995|0|4|16|2|1995|382|4972|Sunday|1995Q2|N|N|N|2449809|2449898|2449459|2449734|N|N|N|N|N| +2449825|AAAAAAAABKBGFCAA|1995-04-17|1143|4972|382|1995|1|4|17|2|1995|382|4972|Monday|1995Q2|N|N|N|2449809|2449898|2449460|2449735|N|N|N|N|N| +2449826|AAAAAAAACKBGFCAA|1995-04-18|1143|4973|382|1995|2|4|18|2|1995|382|4973|Tuesday|1995Q2|N|N|N|2449809|2449898|2449461|2449736|N|N|N|N|N| +2449827|AAAAAAAADKBGFCAA|1995-04-19|1143|4973|382|1995|3|4|19|2|1995|382|4973|Wednesday|1995Q2|N|N|N|2449809|2449898|2449462|2449737|N|N|N|N|N| +2449828|AAAAAAAAEKBGFCAA|1995-04-20|1143|4973|382|1995|4|4|20|2|1995|382|4973|Thursday|1995Q2|N|N|N|2449809|2449898|2449463|2449738|N|N|N|N|N| +2449829|AAAAAAAAFKBGFCAA|1995-04-21|1143|4973|382|1995|5|4|21|2|1995|382|4973|Friday|1995Q2|N|Y|N|2449809|2449898|2449464|2449739|N|N|N|N|N| +2449830|AAAAAAAAGKBGFCAA|1995-04-22|1143|4973|382|1995|6|4|22|2|1995|382|4973|Saturday|1995Q2|N|Y|N|2449809|2449898|2449465|2449740|N|N|N|N|N| +2449831|AAAAAAAAHKBGFCAA|1995-04-23|1143|4973|382|1995|0|4|23|2|1995|382|4973|Sunday|1995Q2|N|N|N|2449809|2449898|2449466|2449741|N|N|N|N|N| +2449832|AAAAAAAAIKBGFCAA|1995-04-24|1143|4973|382|1995|1|4|24|2|1995|382|4973|Monday|1995Q2|N|N|N|2449809|2449898|2449467|2449742|N|N|N|N|N| +2449833|AAAAAAAAJKBGFCAA|1995-04-25|1143|4974|382|1995|2|4|25|2|1995|382|4974|Tuesday|1995Q2|N|N|N|2449809|2449898|2449468|2449743|N|N|N|N|N| +2449834|AAAAAAAAKKBGFCAA|1995-04-26|1143|4974|382|1995|3|4|26|2|1995|382|4974|Wednesday|1995Q2|N|N|N|2449809|2449898|2449469|2449744|N|N|N|N|N| +2449835|AAAAAAAALKBGFCAA|1995-04-27|1143|4974|382|1995|4|4|27|2|1995|382|4974|Thursday|1995Q2|N|N|N|2449809|2449898|2449470|2449745|N|N|N|N|N| +2449836|AAAAAAAAMKBGFCAA|1995-04-28|1143|4974|382|1995|5|4|28|2|1995|382|4974|Friday|1995Q2|N|Y|N|2449809|2449898|2449471|2449746|N|N|N|N|N| +2449837|AAAAAAAANKBGFCAA|1995-04-29|1143|4974|382|1995|6|4|29|2|1995|382|4974|Saturday|1995Q2|N|Y|N|2449809|2449898|2449472|2449747|N|N|N|N|N| +2449838|AAAAAAAAOKBGFCAA|1995-04-30|1143|4974|382|1995|0|4|30|2|1995|382|4974|Sunday|1995Q2|N|N|N|2449809|2449898|2449473|2449748|N|N|N|N|N| +2449839|AAAAAAAAPKBGFCAA|1995-05-01|1144|4974|382|1995|1|5|1|2|1995|382|4974|Monday|1995Q2|N|N|N|2449839|2449958|2449474|2449749|N|N|N|N|N| +2449840|AAAAAAAAALBGFCAA|1995-05-02|1144|4975|382|1995|2|5|2|2|1995|382|4975|Tuesday|1995Q2|N|N|N|2449839|2449958|2449475|2449750|N|N|N|N|N| +2449841|AAAAAAAABLBGFCAA|1995-05-03|1144|4975|382|1995|3|5|3|2|1995|382|4975|Wednesday|1995Q2|N|N|N|2449839|2449958|2449476|2449751|N|N|N|N|N| +2449842|AAAAAAAACLBGFCAA|1995-05-04|1144|4975|382|1995|4|5|4|2|1995|382|4975|Thursday|1995Q2|N|N|N|2449839|2449958|2449477|2449752|N|N|N|N|N| +2449843|AAAAAAAADLBGFCAA|1995-05-05|1144|4975|382|1995|5|5|5|2|1995|382|4975|Friday|1995Q2|N|Y|N|2449839|2449958|2449478|2449753|N|N|N|N|N| +2449844|AAAAAAAAELBGFCAA|1995-05-06|1144|4975|382|1995|6|5|6|2|1995|382|4975|Saturday|1995Q2|N|Y|N|2449839|2449958|2449479|2449754|N|N|N|N|N| +2449845|AAAAAAAAFLBGFCAA|1995-05-07|1144|4975|382|1995|0|5|7|2|1995|382|4975|Sunday|1995Q2|N|N|N|2449839|2449958|2449480|2449755|N|N|N|N|N| +2449846|AAAAAAAAGLBGFCAA|1995-05-08|1144|4975|382|1995|1|5|8|2|1995|382|4975|Monday|1995Q2|N|N|N|2449839|2449958|2449481|2449756|N|N|N|N|N| +2449847|AAAAAAAAHLBGFCAA|1995-05-09|1144|4976|382|1995|2|5|9|2|1995|382|4976|Tuesday|1995Q2|N|N|N|2449839|2449958|2449482|2449757|N|N|N|N|N| +2449848|AAAAAAAAILBGFCAA|1995-05-10|1144|4976|382|1995|3|5|10|2|1995|382|4976|Wednesday|1995Q2|N|N|N|2449839|2449958|2449483|2449758|N|N|N|N|N| +2449849|AAAAAAAAJLBGFCAA|1995-05-11|1144|4976|382|1995|4|5|11|2|1995|382|4976|Thursday|1995Q2|N|N|N|2449839|2449958|2449484|2449759|N|N|N|N|N| +2449850|AAAAAAAAKLBGFCAA|1995-05-12|1144|4976|382|1995|5|5|12|2|1995|382|4976|Friday|1995Q2|N|Y|N|2449839|2449958|2449485|2449760|N|N|N|N|N| +2449851|AAAAAAAALLBGFCAA|1995-05-13|1144|4976|382|1995|6|5|13|2|1995|382|4976|Saturday|1995Q2|N|Y|N|2449839|2449958|2449486|2449761|N|N|N|N|N| +2449852|AAAAAAAAMLBGFCAA|1995-05-14|1144|4976|382|1995|0|5|14|2|1995|382|4976|Sunday|1995Q2|N|N|N|2449839|2449958|2449487|2449762|N|N|N|N|N| +2449853|AAAAAAAANLBGFCAA|1995-05-15|1144|4976|382|1995|1|5|15|2|1995|382|4976|Monday|1995Q2|N|N|N|2449839|2449958|2449488|2449763|N|N|N|N|N| +2449854|AAAAAAAAOLBGFCAA|1995-05-16|1144|4977|382|1995|2|5|16|2|1995|382|4977|Tuesday|1995Q2|N|N|N|2449839|2449958|2449489|2449764|N|N|N|N|N| +2449855|AAAAAAAAPLBGFCAA|1995-05-17|1144|4977|382|1995|3|5|17|2|1995|382|4977|Wednesday|1995Q2|N|N|N|2449839|2449958|2449490|2449765|N|N|N|N|N| +2449856|AAAAAAAAAMBGFCAA|1995-05-18|1144|4977|382|1995|4|5|18|2|1995|382|4977|Thursday|1995Q2|N|N|N|2449839|2449958|2449491|2449766|N|N|N|N|N| +2449857|AAAAAAAABMBGFCAA|1995-05-19|1144|4977|382|1995|5|5|19|2|1995|382|4977|Friday|1995Q2|N|Y|N|2449839|2449958|2449492|2449767|N|N|N|N|N| +2449858|AAAAAAAACMBGFCAA|1995-05-20|1144|4977|382|1995|6|5|20|2|1995|382|4977|Saturday|1995Q2|N|Y|N|2449839|2449958|2449493|2449768|N|N|N|N|N| +2449859|AAAAAAAADMBGFCAA|1995-05-21|1144|4977|382|1995|0|5|21|2|1995|382|4977|Sunday|1995Q2|N|N|N|2449839|2449958|2449494|2449769|N|N|N|N|N| +2449860|AAAAAAAAEMBGFCAA|1995-05-22|1144|4977|382|1995|1|5|22|2|1995|382|4977|Monday|1995Q2|N|N|N|2449839|2449958|2449495|2449770|N|N|N|N|N| +2449861|AAAAAAAAFMBGFCAA|1995-05-23|1144|4978|382|1995|2|5|23|2|1995|382|4978|Tuesday|1995Q2|N|N|N|2449839|2449958|2449496|2449771|N|N|N|N|N| +2449862|AAAAAAAAGMBGFCAA|1995-05-24|1144|4978|382|1995|3|5|24|2|1995|382|4978|Wednesday|1995Q2|N|N|N|2449839|2449958|2449497|2449772|N|N|N|N|N| +2449863|AAAAAAAAHMBGFCAA|1995-05-25|1144|4978|382|1995|4|5|25|2|1995|382|4978|Thursday|1995Q2|N|N|N|2449839|2449958|2449498|2449773|N|N|N|N|N| +2449864|AAAAAAAAIMBGFCAA|1995-05-26|1144|4978|382|1995|5|5|26|2|1995|382|4978|Friday|1995Q2|N|Y|N|2449839|2449958|2449499|2449774|N|N|N|N|N| +2449865|AAAAAAAAJMBGFCAA|1995-05-27|1144|4978|382|1995|6|5|27|2|1995|382|4978|Saturday|1995Q2|N|Y|N|2449839|2449958|2449500|2449775|N|N|N|N|N| +2449866|AAAAAAAAKMBGFCAA|1995-05-28|1144|4978|382|1995|0|5|28|2|1995|382|4978|Sunday|1995Q2|N|N|N|2449839|2449958|2449501|2449776|N|N|N|N|N| +2449867|AAAAAAAALMBGFCAA|1995-05-29|1144|4978|382|1995|1|5|29|2|1995|382|4978|Monday|1995Q2|N|N|N|2449839|2449958|2449502|2449777|N|N|N|N|N| +2449868|AAAAAAAAMMBGFCAA|1995-05-30|1144|4979|382|1995|2|5|30|2|1995|382|4979|Tuesday|1995Q2|N|N|N|2449839|2449958|2449503|2449778|N|N|N|N|N| +2449869|AAAAAAAANMBGFCAA|1995-05-31|1144|4979|382|1995|3|5|31|2|1995|382|4979|Wednesday|1995Q2|N|N|N|2449839|2449958|2449504|2449779|N|N|N|N|N| +2449870|AAAAAAAAOMBGFCAA|1995-06-01|1145|4979|383|1995|4|6|1|2|1995|383|4979|Thursday|1995Q2|N|N|N|2449870|2450020|2449505|2449780|N|N|N|N|N| +2449871|AAAAAAAAPMBGFCAA|1995-06-02|1145|4979|383|1995|5|6|2|2|1995|383|4979|Friday|1995Q2|N|Y|N|2449870|2450020|2449506|2449781|N|N|N|N|N| +2449872|AAAAAAAAANBGFCAA|1995-06-03|1145|4979|383|1995|6|6|3|2|1995|383|4979|Saturday|1995Q2|N|Y|N|2449870|2450020|2449507|2449782|N|N|N|N|N| +2449873|AAAAAAAABNBGFCAA|1995-06-04|1145|4979|383|1995|0|6|4|2|1995|383|4979|Sunday|1995Q2|N|N|N|2449870|2450020|2449508|2449783|N|N|N|N|N| +2449874|AAAAAAAACNBGFCAA|1995-06-05|1145|4979|383|1995|1|6|5|2|1995|383|4979|Monday|1995Q2|N|N|N|2449870|2450020|2449509|2449784|N|N|N|N|N| +2449875|AAAAAAAADNBGFCAA|1995-06-06|1145|4980|383|1995|2|6|6|2|1995|383|4980|Tuesday|1995Q2|N|N|N|2449870|2450020|2449510|2449785|N|N|N|N|N| +2449876|AAAAAAAAENBGFCAA|1995-06-07|1145|4980|383|1995|3|6|7|2|1995|383|4980|Wednesday|1995Q2|N|N|N|2449870|2450020|2449511|2449786|N|N|N|N|N| +2449877|AAAAAAAAFNBGFCAA|1995-06-08|1145|4980|383|1995|4|6|8|2|1995|383|4980|Thursday|1995Q2|N|N|N|2449870|2450020|2449512|2449787|N|N|N|N|N| +2449878|AAAAAAAAGNBGFCAA|1995-06-09|1145|4980|383|1995|5|6|9|2|1995|383|4980|Friday|1995Q2|N|Y|N|2449870|2450020|2449513|2449788|N|N|N|N|N| +2449879|AAAAAAAAHNBGFCAA|1995-06-10|1145|4980|383|1995|6|6|10|2|1995|383|4980|Saturday|1995Q2|N|Y|N|2449870|2450020|2449514|2449789|N|N|N|N|N| +2449880|AAAAAAAAINBGFCAA|1995-06-11|1145|4980|383|1995|0|6|11|2|1995|383|4980|Sunday|1995Q2|N|N|N|2449870|2450020|2449515|2449790|N|N|N|N|N| +2449881|AAAAAAAAJNBGFCAA|1995-06-12|1145|4980|383|1995|1|6|12|2|1995|383|4980|Monday|1995Q2|N|N|N|2449870|2450020|2449516|2449791|N|N|N|N|N| +2449882|AAAAAAAAKNBGFCAA|1995-06-13|1145|4981|383|1995|2|6|13|2|1995|383|4981|Tuesday|1995Q2|N|N|N|2449870|2450020|2449517|2449792|N|N|N|N|N| +2449883|AAAAAAAALNBGFCAA|1995-06-14|1145|4981|383|1995|3|6|14|2|1995|383|4981|Wednesday|1995Q2|N|N|N|2449870|2450020|2449518|2449793|N|N|N|N|N| +2449884|AAAAAAAAMNBGFCAA|1995-06-15|1145|4981|383|1995|4|6|15|2|1995|383|4981|Thursday|1995Q2|N|N|N|2449870|2450020|2449519|2449794|N|N|N|N|N| +2449885|AAAAAAAANNBGFCAA|1995-06-16|1145|4981|383|1995|5|6|16|2|1995|383|4981|Friday|1995Q2|N|Y|N|2449870|2450020|2449520|2449795|N|N|N|N|N| +2449886|AAAAAAAAONBGFCAA|1995-06-17|1145|4981|383|1995|6|6|17|2|1995|383|4981|Saturday|1995Q2|N|Y|N|2449870|2450020|2449521|2449796|N|N|N|N|N| +2449887|AAAAAAAAPNBGFCAA|1995-06-18|1145|4981|383|1995|0|6|18|2|1995|383|4981|Sunday|1995Q2|N|N|N|2449870|2450020|2449522|2449797|N|N|N|N|N| +2449888|AAAAAAAAAOBGFCAA|1995-06-19|1145|4981|383|1995|1|6|19|2|1995|383|4981|Monday|1995Q2|N|N|N|2449870|2450020|2449523|2449798|N|N|N|N|N| +2449889|AAAAAAAABOBGFCAA|1995-06-20|1145|4982|383|1995|2|6|20|2|1995|383|4982|Tuesday|1995Q2|N|N|N|2449870|2450020|2449524|2449799|N|N|N|N|N| +2449890|AAAAAAAACOBGFCAA|1995-06-21|1145|4982|383|1995|3|6|21|2|1995|383|4982|Wednesday|1995Q2|N|N|N|2449870|2450020|2449525|2449800|N|N|N|N|N| +2449891|AAAAAAAADOBGFCAA|1995-06-22|1145|4982|383|1995|4|6|22|2|1995|383|4982|Thursday|1995Q2|N|N|N|2449870|2450020|2449526|2449801|N|N|N|N|N| +2449892|AAAAAAAAEOBGFCAA|1995-06-23|1145|4982|383|1995|5|6|23|2|1995|383|4982|Friday|1995Q2|N|Y|N|2449870|2450020|2449527|2449802|N|N|N|N|N| +2449893|AAAAAAAAFOBGFCAA|1995-06-24|1145|4982|383|1995|6|6|24|2|1995|383|4982|Saturday|1995Q2|N|Y|N|2449870|2450020|2449528|2449803|N|N|N|N|N| +2449894|AAAAAAAAGOBGFCAA|1995-06-25|1145|4982|383|1995|0|6|25|2|1995|383|4982|Sunday|1995Q2|N|N|N|2449870|2450020|2449529|2449804|N|N|N|N|N| +2449895|AAAAAAAAHOBGFCAA|1995-06-26|1145|4982|383|1995|1|6|26|2|1995|383|4982|Monday|1995Q2|N|N|N|2449870|2450020|2449530|2449805|N|N|N|N|N| +2449896|AAAAAAAAIOBGFCAA|1995-06-27|1145|4983|383|1995|2|6|27|2|1995|383|4983|Tuesday|1995Q2|N|N|N|2449870|2450020|2449531|2449806|N|N|N|N|N| +2449897|AAAAAAAAJOBGFCAA|1995-06-28|1145|4983|383|1995|3|6|28|2|1995|383|4983|Wednesday|1995Q2|N|N|N|2449870|2450020|2449532|2449807|N|N|N|N|N| +2449898|AAAAAAAAKOBGFCAA|1995-06-29|1145|4983|383|1995|4|6|29|2|1995|383|4983|Thursday|1995Q2|N|N|N|2449870|2450020|2449533|2449808|N|N|N|N|N| +2449899|AAAAAAAALOBGFCAA|1995-06-30|1145|4983|383|1995|5|6|30|2|1995|383|4983|Friday|1995Q2|N|Y|N|2449870|2450020|2449534|2449809|N|N|N|N|N| +2449900|AAAAAAAAMOBGFCAA|1995-07-01|1146|4983|383|1995|6|7|1|2|1995|383|4983|Saturday|1995Q2|N|Y|N|2449900|2450080|2449535|2449809|N|N|N|N|N| +2449901|AAAAAAAANOBGFCAA|1995-07-02|1146|4983|383|1995|0|7|2|3|1995|383|4983|Sunday|1995Q3|N|N|N|2449900|2450080|2449536|2449810|N|N|N|N|N| +2449902|AAAAAAAAOOBGFCAA|1995-07-03|1146|4983|383|1995|1|7|3|3|1995|383|4983|Monday|1995Q3|N|N|N|2449900|2450080|2449537|2449811|N|N|N|N|N| +2449903|AAAAAAAAPOBGFCAA|1995-07-04|1146|4984|383|1995|2|7|4|3|1995|383|4984|Tuesday|1995Q3|N|N|N|2449900|2450080|2449538|2449812|N|N|N|N|N| +2449904|AAAAAAAAAPBGFCAA|1995-07-05|1146|4984|383|1995|3|7|5|3|1995|383|4984|Wednesday|1995Q3|Y|N|N|2449900|2450080|2449539|2449813|N|N|N|N|N| +2449905|AAAAAAAABPBGFCAA|1995-07-06|1146|4984|383|1995|4|7|6|3|1995|383|4984|Thursday|1995Q3|N|N|Y|2449900|2450080|2449540|2449814|N|N|N|N|N| +2449906|AAAAAAAACPBGFCAA|1995-07-07|1146|4984|383|1995|5|7|7|3|1995|383|4984|Friday|1995Q3|N|Y|N|2449900|2450080|2449541|2449815|N|N|N|N|N| +2449907|AAAAAAAADPBGFCAA|1995-07-08|1146|4984|383|1995|6|7|8|3|1995|383|4984|Saturday|1995Q3|N|Y|N|2449900|2450080|2449542|2449816|N|N|N|N|N| +2449908|AAAAAAAAEPBGFCAA|1995-07-09|1146|4984|383|1995|0|7|9|3|1995|383|4984|Sunday|1995Q3|N|N|N|2449900|2450080|2449543|2449817|N|N|N|N|N| +2449909|AAAAAAAAFPBGFCAA|1995-07-10|1146|4984|383|1995|1|7|10|3|1995|383|4984|Monday|1995Q3|N|N|N|2449900|2450080|2449544|2449818|N|N|N|N|N| +2449910|AAAAAAAAGPBGFCAA|1995-07-11|1146|4985|383|1995|2|7|11|3|1995|383|4985|Tuesday|1995Q3|N|N|N|2449900|2450080|2449545|2449819|N|N|N|N|N| +2449911|AAAAAAAAHPBGFCAA|1995-07-12|1146|4985|383|1995|3|7|12|3|1995|383|4985|Wednesday|1995Q3|N|N|N|2449900|2450080|2449546|2449820|N|N|N|N|N| +2449912|AAAAAAAAIPBGFCAA|1995-07-13|1146|4985|383|1995|4|7|13|3|1995|383|4985|Thursday|1995Q3|N|N|N|2449900|2450080|2449547|2449821|N|N|N|N|N| +2449913|AAAAAAAAJPBGFCAA|1995-07-14|1146|4985|383|1995|5|7|14|3|1995|383|4985|Friday|1995Q3|N|Y|N|2449900|2450080|2449548|2449822|N|N|N|N|N| +2449914|AAAAAAAAKPBGFCAA|1995-07-15|1146|4985|383|1995|6|7|15|3|1995|383|4985|Saturday|1995Q3|N|Y|N|2449900|2450080|2449549|2449823|N|N|N|N|N| +2449915|AAAAAAAALPBGFCAA|1995-07-16|1146|4985|383|1995|0|7|16|3|1995|383|4985|Sunday|1995Q3|N|N|N|2449900|2450080|2449550|2449824|N|N|N|N|N| +2449916|AAAAAAAAMPBGFCAA|1995-07-17|1146|4985|383|1995|1|7|17|3|1995|383|4985|Monday|1995Q3|N|N|N|2449900|2450080|2449551|2449825|N|N|N|N|N| +2449917|AAAAAAAANPBGFCAA|1995-07-18|1146|4986|383|1995|2|7|18|3|1995|383|4986|Tuesday|1995Q3|N|N|N|2449900|2450080|2449552|2449826|N|N|N|N|N| +2449918|AAAAAAAAOPBGFCAA|1995-07-19|1146|4986|383|1995|3|7|19|3|1995|383|4986|Wednesday|1995Q3|N|N|N|2449900|2450080|2449553|2449827|N|N|N|N|N| +2449919|AAAAAAAAPPBGFCAA|1995-07-20|1146|4986|383|1995|4|7|20|3|1995|383|4986|Thursday|1995Q3|N|N|N|2449900|2450080|2449554|2449828|N|N|N|N|N| +2449920|AAAAAAAAAACGFCAA|1995-07-21|1146|4986|383|1995|5|7|21|3|1995|383|4986|Friday|1995Q3|N|Y|N|2449900|2450080|2449555|2449829|N|N|N|N|N| +2449921|AAAAAAAABACGFCAA|1995-07-22|1146|4986|383|1995|6|7|22|3|1995|383|4986|Saturday|1995Q3|N|Y|N|2449900|2450080|2449556|2449830|N|N|N|N|N| +2449922|AAAAAAAACACGFCAA|1995-07-23|1146|4986|383|1995|0|7|23|3|1995|383|4986|Sunday|1995Q3|N|N|N|2449900|2450080|2449557|2449831|N|N|N|N|N| +2449923|AAAAAAAADACGFCAA|1995-07-24|1146|4986|383|1995|1|7|24|3|1995|383|4986|Monday|1995Q3|N|N|N|2449900|2450080|2449558|2449832|N|N|N|N|N| +2449924|AAAAAAAAEACGFCAA|1995-07-25|1146|4987|383|1995|2|7|25|3|1995|383|4987|Tuesday|1995Q3|N|N|N|2449900|2450080|2449559|2449833|N|N|N|N|N| +2449925|AAAAAAAAFACGFCAA|1995-07-26|1146|4987|383|1995|3|7|26|3|1995|383|4987|Wednesday|1995Q3|N|N|N|2449900|2450080|2449560|2449834|N|N|N|N|N| +2449926|AAAAAAAAGACGFCAA|1995-07-27|1146|4987|383|1995|4|7|27|3|1995|383|4987|Thursday|1995Q3|N|N|N|2449900|2450080|2449561|2449835|N|N|N|N|N| +2449927|AAAAAAAAHACGFCAA|1995-07-28|1146|4987|383|1995|5|7|28|3|1995|383|4987|Friday|1995Q3|N|Y|N|2449900|2450080|2449562|2449836|N|N|N|N|N| +2449928|AAAAAAAAIACGFCAA|1995-07-29|1146|4987|383|1995|6|7|29|3|1995|383|4987|Saturday|1995Q3|N|Y|N|2449900|2450080|2449563|2449837|N|N|N|N|N| +2449929|AAAAAAAAJACGFCAA|1995-07-30|1146|4987|383|1995|0|7|30|3|1995|383|4987|Sunday|1995Q3|N|N|N|2449900|2450080|2449564|2449838|N|N|N|N|N| +2449930|AAAAAAAAKACGFCAA|1995-07-31|1146|4987|383|1995|1|7|31|3|1995|383|4987|Monday|1995Q3|N|N|N|2449900|2450080|2449565|2449839|N|N|N|N|N| +2449931|AAAAAAAALACGFCAA|1995-08-01|1147|4988|383|1995|2|8|1|3|1995|383|4988|Tuesday|1995Q3|N|N|N|2449931|2450142|2449566|2449840|N|N|N|N|N| +2449932|AAAAAAAAMACGFCAA|1995-08-02|1147|4988|383|1995|3|8|2|3|1995|383|4988|Wednesday|1995Q3|N|N|N|2449931|2450142|2449567|2449841|N|N|N|N|N| +2449933|AAAAAAAANACGFCAA|1995-08-03|1147|4988|383|1995|4|8|3|3|1995|383|4988|Thursday|1995Q3|N|N|N|2449931|2450142|2449568|2449842|N|N|N|N|N| +2449934|AAAAAAAAOACGFCAA|1995-08-04|1147|4988|383|1995|5|8|4|3|1995|383|4988|Friday|1995Q3|N|Y|N|2449931|2450142|2449569|2449843|N|N|N|N|N| +2449935|AAAAAAAAPACGFCAA|1995-08-05|1147|4988|383|1995|6|8|5|3|1995|383|4988|Saturday|1995Q3|N|Y|N|2449931|2450142|2449570|2449844|N|N|N|N|N| +2449936|AAAAAAAAABCGFCAA|1995-08-06|1147|4988|383|1995|0|8|6|3|1995|383|4988|Sunday|1995Q3|N|N|N|2449931|2450142|2449571|2449845|N|N|N|N|N| +2449937|AAAAAAAABBCGFCAA|1995-08-07|1147|4988|383|1995|1|8|7|3|1995|383|4988|Monday|1995Q3|N|N|N|2449931|2450142|2449572|2449846|N|N|N|N|N| +2449938|AAAAAAAACBCGFCAA|1995-08-08|1147|4989|383|1995|2|8|8|3|1995|383|4989|Tuesday|1995Q3|N|N|N|2449931|2450142|2449573|2449847|N|N|N|N|N| +2449939|AAAAAAAADBCGFCAA|1995-08-09|1147|4989|383|1995|3|8|9|3|1995|383|4989|Wednesday|1995Q3|N|N|N|2449931|2450142|2449574|2449848|N|N|N|N|N| +2449940|AAAAAAAAEBCGFCAA|1995-08-10|1147|4989|383|1995|4|8|10|3|1995|383|4989|Thursday|1995Q3|N|N|N|2449931|2450142|2449575|2449849|N|N|N|N|N| +2449941|AAAAAAAAFBCGFCAA|1995-08-11|1147|4989|383|1995|5|8|11|3|1995|383|4989|Friday|1995Q3|N|Y|N|2449931|2450142|2449576|2449850|N|N|N|N|N| +2449942|AAAAAAAAGBCGFCAA|1995-08-12|1147|4989|383|1995|6|8|12|3|1995|383|4989|Saturday|1995Q3|N|Y|N|2449931|2450142|2449577|2449851|N|N|N|N|N| +2449943|AAAAAAAAHBCGFCAA|1995-08-13|1147|4989|383|1995|0|8|13|3|1995|383|4989|Sunday|1995Q3|N|N|N|2449931|2450142|2449578|2449852|N|N|N|N|N| +2449944|AAAAAAAAIBCGFCAA|1995-08-14|1147|4989|383|1995|1|8|14|3|1995|383|4989|Monday|1995Q3|N|N|N|2449931|2450142|2449579|2449853|N|N|N|N|N| +2449945|AAAAAAAAJBCGFCAA|1995-08-15|1147|4990|383|1995|2|8|15|3|1995|383|4990|Tuesday|1995Q3|N|N|N|2449931|2450142|2449580|2449854|N|N|N|N|N| +2449946|AAAAAAAAKBCGFCAA|1995-08-16|1147|4990|383|1995|3|8|16|3|1995|383|4990|Wednesday|1995Q3|N|N|N|2449931|2450142|2449581|2449855|N|N|N|N|N| +2449947|AAAAAAAALBCGFCAA|1995-08-17|1147|4990|383|1995|4|8|17|3|1995|383|4990|Thursday|1995Q3|N|N|N|2449931|2450142|2449582|2449856|N|N|N|N|N| +2449948|AAAAAAAAMBCGFCAA|1995-08-18|1147|4990|383|1995|5|8|18|3|1995|383|4990|Friday|1995Q3|N|Y|N|2449931|2450142|2449583|2449857|N|N|N|N|N| +2449949|AAAAAAAANBCGFCAA|1995-08-19|1147|4990|383|1995|6|8|19|3|1995|383|4990|Saturday|1995Q3|N|Y|N|2449931|2450142|2449584|2449858|N|N|N|N|N| +2449950|AAAAAAAAOBCGFCAA|1995-08-20|1147|4990|383|1995|0|8|20|3|1995|383|4990|Sunday|1995Q3|N|N|N|2449931|2450142|2449585|2449859|N|N|N|N|N| +2449951|AAAAAAAAPBCGFCAA|1995-08-21|1147|4990|383|1995|1|8|21|3|1995|383|4990|Monday|1995Q3|N|N|N|2449931|2450142|2449586|2449860|N|N|N|N|N| +2449952|AAAAAAAAACCGFCAA|1995-08-22|1147|4991|383|1995|2|8|22|3|1995|383|4991|Tuesday|1995Q3|N|N|N|2449931|2450142|2449587|2449861|N|N|N|N|N| +2449953|AAAAAAAABCCGFCAA|1995-08-23|1147|4991|383|1995|3|8|23|3|1995|383|4991|Wednesday|1995Q3|N|N|N|2449931|2450142|2449588|2449862|N|N|N|N|N| +2449954|AAAAAAAACCCGFCAA|1995-08-24|1147|4991|383|1995|4|8|24|3|1995|383|4991|Thursday|1995Q3|N|N|N|2449931|2450142|2449589|2449863|N|N|N|N|N| +2449955|AAAAAAAADCCGFCAA|1995-08-25|1147|4991|383|1995|5|8|25|3|1995|383|4991|Friday|1995Q3|N|Y|N|2449931|2450142|2449590|2449864|N|N|N|N|N| +2449956|AAAAAAAAECCGFCAA|1995-08-26|1147|4991|383|1995|6|8|26|3|1995|383|4991|Saturday|1995Q3|N|Y|N|2449931|2450142|2449591|2449865|N|N|N|N|N| +2449957|AAAAAAAAFCCGFCAA|1995-08-27|1147|4991|383|1995|0|8|27|3|1995|383|4991|Sunday|1995Q3|N|N|N|2449931|2450142|2449592|2449866|N|N|N|N|N| +2449958|AAAAAAAAGCCGFCAA|1995-08-28|1147|4991|383|1995|1|8|28|3|1995|383|4991|Monday|1995Q3|N|N|N|2449931|2450142|2449593|2449867|N|N|N|N|N| +2449959|AAAAAAAAHCCGFCAA|1995-08-29|1147|4992|383|1995|2|8|29|3|1995|383|4992|Tuesday|1995Q3|N|N|N|2449931|2450142|2449594|2449868|N|N|N|N|N| +2449960|AAAAAAAAICCGFCAA|1995-08-30|1147|4992|383|1995|3|8|30|3|1995|383|4992|Wednesday|1995Q3|N|N|N|2449931|2450142|2449595|2449869|N|N|N|N|N| +2449961|AAAAAAAAJCCGFCAA|1995-08-31|1147|4992|383|1995|4|8|31|3|1995|383|4992|Thursday|1995Q3|N|N|N|2449931|2450142|2449596|2449870|N|N|N|N|N| +2449962|AAAAAAAAKCCGFCAA|1995-09-01|1148|4992|384|1995|5|9|1|3|1995|384|4992|Friday|1995Q3|N|Y|N|2449962|2450204|2449597|2449871|N|N|N|N|N| +2449963|AAAAAAAALCCGFCAA|1995-09-02|1148|4992|384|1995|6|9|2|3|1995|384|4992|Saturday|1995Q3|N|Y|N|2449962|2450204|2449598|2449872|N|N|N|N|N| +2449964|AAAAAAAAMCCGFCAA|1995-09-03|1148|4992|384|1995|0|9|3|3|1995|384|4992|Sunday|1995Q3|N|N|N|2449962|2450204|2449599|2449873|N|N|N|N|N| +2449965|AAAAAAAANCCGFCAA|1995-09-04|1148|4992|384|1995|1|9|4|3|1995|384|4992|Monday|1995Q3|N|N|N|2449962|2450204|2449600|2449874|N|N|N|N|N| +2449966|AAAAAAAAOCCGFCAA|1995-09-05|1148|4993|384|1995|2|9|5|3|1995|384|4993|Tuesday|1995Q3|N|N|N|2449962|2450204|2449601|2449875|N|N|N|N|N| +2449967|AAAAAAAAPCCGFCAA|1995-09-06|1148|4993|384|1995|3|9|6|3|1995|384|4993|Wednesday|1995Q3|N|N|N|2449962|2450204|2449602|2449876|N|N|N|N|N| +2449968|AAAAAAAAADCGFCAA|1995-09-07|1148|4993|384|1995|4|9|7|3|1995|384|4993|Thursday|1995Q3|N|N|N|2449962|2450204|2449603|2449877|N|N|N|N|N| +2449969|AAAAAAAABDCGFCAA|1995-09-08|1148|4993|384|1995|5|9|8|3|1995|384|4993|Friday|1995Q3|N|Y|N|2449962|2450204|2449604|2449878|N|N|N|N|N| +2449970|AAAAAAAACDCGFCAA|1995-09-09|1148|4993|384|1995|6|9|9|3|1995|384|4993|Saturday|1995Q3|N|Y|N|2449962|2450204|2449605|2449879|N|N|N|N|N| +2449971|AAAAAAAADDCGFCAA|1995-09-10|1148|4993|384|1995|0|9|10|3|1995|384|4993|Sunday|1995Q3|N|N|N|2449962|2450204|2449606|2449880|N|N|N|N|N| +2449972|AAAAAAAAEDCGFCAA|1995-09-11|1148|4993|384|1995|1|9|11|3|1995|384|4993|Monday|1995Q3|N|N|N|2449962|2450204|2449607|2449881|N|N|N|N|N| +2449973|AAAAAAAAFDCGFCAA|1995-09-12|1148|4994|384|1995|2|9|12|3|1995|384|4994|Tuesday|1995Q3|N|N|N|2449962|2450204|2449608|2449882|N|N|N|N|N| +2449974|AAAAAAAAGDCGFCAA|1995-09-13|1148|4994|384|1995|3|9|13|3|1995|384|4994|Wednesday|1995Q3|N|N|N|2449962|2450204|2449609|2449883|N|N|N|N|N| +2449975|AAAAAAAAHDCGFCAA|1995-09-14|1148|4994|384|1995|4|9|14|3|1995|384|4994|Thursday|1995Q3|N|N|N|2449962|2450204|2449610|2449884|N|N|N|N|N| +2449976|AAAAAAAAIDCGFCAA|1995-09-15|1148|4994|384|1995|5|9|15|3|1995|384|4994|Friday|1995Q3|N|Y|N|2449962|2450204|2449611|2449885|N|N|N|N|N| +2449977|AAAAAAAAJDCGFCAA|1995-09-16|1148|4994|384|1995|6|9|16|3|1995|384|4994|Saturday|1995Q3|N|Y|N|2449962|2450204|2449612|2449886|N|N|N|N|N| +2449978|AAAAAAAAKDCGFCAA|1995-09-17|1148|4994|384|1995|0|9|17|3|1995|384|4994|Sunday|1995Q3|N|N|N|2449962|2450204|2449613|2449887|N|N|N|N|N| +2449979|AAAAAAAALDCGFCAA|1995-09-18|1148|4994|384|1995|1|9|18|3|1995|384|4994|Monday|1995Q3|N|N|N|2449962|2450204|2449614|2449888|N|N|N|N|N| +2449980|AAAAAAAAMDCGFCAA|1995-09-19|1148|4995|384|1995|2|9|19|3|1995|384|4995|Tuesday|1995Q3|N|N|N|2449962|2450204|2449615|2449889|N|N|N|N|N| +2449981|AAAAAAAANDCGFCAA|1995-09-20|1148|4995|384|1995|3|9|20|3|1995|384|4995|Wednesday|1995Q3|N|N|N|2449962|2450204|2449616|2449890|N|N|N|N|N| +2449982|AAAAAAAAODCGFCAA|1995-09-21|1148|4995|384|1995|4|9|21|3|1995|384|4995|Thursday|1995Q3|N|N|N|2449962|2450204|2449617|2449891|N|N|N|N|N| +2449983|AAAAAAAAPDCGFCAA|1995-09-22|1148|4995|384|1995|5|9|22|3|1995|384|4995|Friday|1995Q3|N|Y|N|2449962|2450204|2449618|2449892|N|N|N|N|N| +2449984|AAAAAAAAAECGFCAA|1995-09-23|1148|4995|384|1995|6|9|23|3|1995|384|4995|Saturday|1995Q3|N|Y|N|2449962|2450204|2449619|2449893|N|N|N|N|N| +2449985|AAAAAAAABECGFCAA|1995-09-24|1148|4995|384|1995|0|9|24|3|1995|384|4995|Sunday|1995Q3|N|N|N|2449962|2450204|2449620|2449894|N|N|N|N|N| +2449986|AAAAAAAACECGFCAA|1995-09-25|1148|4995|384|1995|1|9|25|3|1995|384|4995|Monday|1995Q3|N|N|N|2449962|2450204|2449621|2449895|N|N|N|N|N| +2449987|AAAAAAAADECGFCAA|1995-09-26|1148|4996|384|1995|2|9|26|3|1995|384|4996|Tuesday|1995Q3|N|N|N|2449962|2450204|2449622|2449896|N|N|N|N|N| +2449988|AAAAAAAAEECGFCAA|1995-09-27|1148|4996|384|1995|3|9|27|3|1995|384|4996|Wednesday|1995Q3|N|N|N|2449962|2450204|2449623|2449897|N|N|N|N|N| +2449989|AAAAAAAAFECGFCAA|1995-09-28|1148|4996|384|1995|4|9|28|3|1995|384|4996|Thursday|1995Q3|N|N|N|2449962|2450204|2449624|2449898|N|N|N|N|N| +2449990|AAAAAAAAGECGFCAA|1995-09-29|1148|4996|384|1995|5|9|29|3|1995|384|4996|Friday|1995Q3|N|Y|N|2449962|2450204|2449625|2449899|N|N|N|N|N| +2449991|AAAAAAAAHECGFCAA|1995-09-30|1148|4996|384|1995|6|9|30|3|1995|384|4996|Saturday|1995Q3|N|Y|N|2449962|2450204|2449626|2449900|N|N|N|N|N| +2449992|AAAAAAAAIECGFCAA|1995-10-01|1149|4996|384|1995|0|10|1|3|1995|384|4996|Sunday|1995Q3|N|N|N|2449992|2450264|2449627|2449900|N|N|N|N|N| +2449993|AAAAAAAAJECGFCAA|1995-10-02|1149|4996|384|1995|1|10|2|4|1995|384|4996|Monday|1995Q4|N|N|N|2449992|2450264|2449628|2449901|N|N|N|N|N| +2449994|AAAAAAAAKECGFCAA|1995-10-03|1149|4997|384|1995|2|10|3|4|1995|384|4997|Tuesday|1995Q4|N|N|N|2449992|2450264|2449629|2449902|N|N|N|N|N| +2449995|AAAAAAAALECGFCAA|1995-10-04|1149|4997|384|1995|3|10|4|4|1995|384|4997|Wednesday|1995Q4|N|N|N|2449992|2450264|2449630|2449903|N|N|N|N|N| +2449996|AAAAAAAAMECGFCAA|1995-10-05|1149|4997|384|1995|4|10|5|4|1995|384|4997|Thursday|1995Q4|N|N|N|2449992|2450264|2449631|2449904|N|N|N|N|N| +2449997|AAAAAAAANECGFCAA|1995-10-06|1149|4997|384|1995|5|10|6|4|1995|384|4997|Friday|1995Q4|N|Y|N|2449992|2450264|2449632|2449905|N|N|N|N|N| +2449998|AAAAAAAAOECGFCAA|1995-10-07|1149|4997|384|1995|6|10|7|4|1995|384|4997|Saturday|1995Q4|N|Y|N|2449992|2450264|2449633|2449906|N|N|N|N|N| +2449999|AAAAAAAAPECGFCAA|1995-10-08|1149|4997|384|1995|0|10|8|4|1995|384|4997|Sunday|1995Q4|N|N|N|2449992|2450264|2449634|2449907|N|N|N|N|N| +2450000|AAAAAAAAAFCGFCAA|1995-10-09|1149|4997|384|1995|1|10|9|4|1995|384|4997|Monday|1995Q4|N|N|N|2449992|2450264|2449635|2449908|N|N|N|N|N| +2450001|AAAAAAAABFCGFCAA|1995-10-10|1149|4998|384|1995|2|10|10|4|1995|384|4998|Tuesday|1995Q4|N|N|N|2449992|2450264|2449636|2449909|N|N|N|N|N| +2450002|AAAAAAAACFCGFCAA|1995-10-11|1149|4998|384|1995|3|10|11|4|1995|384|4998|Wednesday|1995Q4|N|N|N|2449992|2450264|2449637|2449910|N|N|N|N|N| +2450003|AAAAAAAADFCGFCAA|1995-10-12|1149|4998|384|1995|4|10|12|4|1995|384|4998|Thursday|1995Q4|N|N|N|2449992|2450264|2449638|2449911|N|N|N|N|N| +2450004|AAAAAAAAEFCGFCAA|1995-10-13|1149|4998|384|1995|5|10|13|4|1995|384|4998|Friday|1995Q4|N|Y|N|2449992|2450264|2449639|2449912|N|N|N|N|N| +2450005|AAAAAAAAFFCGFCAA|1995-10-14|1149|4998|384|1995|6|10|14|4|1995|384|4998|Saturday|1995Q4|N|Y|N|2449992|2450264|2449640|2449913|N|N|N|N|N| +2450006|AAAAAAAAGFCGFCAA|1995-10-15|1149|4998|384|1995|0|10|15|4|1995|384|4998|Sunday|1995Q4|N|N|N|2449992|2450264|2449641|2449914|N|N|N|N|N| +2450007|AAAAAAAAHFCGFCAA|1995-10-16|1149|4998|384|1995|1|10|16|4|1995|384|4998|Monday|1995Q4|N|N|N|2449992|2450264|2449642|2449915|N|N|N|N|N| +2450008|AAAAAAAAIFCGFCAA|1995-10-17|1149|4999|384|1995|2|10|17|4|1995|384|4999|Tuesday|1995Q4|N|N|N|2449992|2450264|2449643|2449916|N|N|N|N|N| +2450009|AAAAAAAAJFCGFCAA|1995-10-18|1149|4999|384|1995|3|10|18|4|1995|384|4999|Wednesday|1995Q4|N|N|N|2449992|2450264|2449644|2449917|N|N|N|N|N| +2450010|AAAAAAAAKFCGFCAA|1995-10-19|1149|4999|384|1995|4|10|19|4|1995|384|4999|Thursday|1995Q4|N|N|N|2449992|2450264|2449645|2449918|N|N|N|N|N| +2450011|AAAAAAAALFCGFCAA|1995-10-20|1149|4999|384|1995|5|10|20|4|1995|384|4999|Friday|1995Q4|N|Y|N|2449992|2450264|2449646|2449919|N|N|N|N|N| +2450012|AAAAAAAAMFCGFCAA|1995-10-21|1149|4999|384|1995|6|10|21|4|1995|384|4999|Saturday|1995Q4|N|Y|N|2449992|2450264|2449647|2449920|N|N|N|N|N| +2450013|AAAAAAAANFCGFCAA|1995-10-22|1149|4999|384|1995|0|10|22|4|1995|384|4999|Sunday|1995Q4|N|N|N|2449992|2450264|2449648|2449921|N|N|N|N|N| +2450014|AAAAAAAAOFCGFCAA|1995-10-23|1149|4999|384|1995|1|10|23|4|1995|384|4999|Monday|1995Q4|N|N|N|2449992|2450264|2449649|2449922|N|N|N|N|N| +2450015|AAAAAAAAPFCGFCAA|1995-10-24|1149|5000|384|1995|2|10|24|4|1995|384|5000|Tuesday|1995Q4|N|N|N|2449992|2450264|2449650|2449923|N|N|N|N|N| +2450016|AAAAAAAAAGCGFCAA|1995-10-25|1149|5000|384|1995|3|10|25|4|1995|384|5000|Wednesday|1995Q4|N|N|N|2449992|2450264|2449651|2449924|N|N|N|N|N| +2450017|AAAAAAAABGCGFCAA|1995-10-26|1149|5000|384|1995|4|10|26|4|1995|384|5000|Thursday|1995Q4|N|N|N|2449992|2450264|2449652|2449925|N|N|N|N|N| +2450018|AAAAAAAACGCGFCAA|1995-10-27|1149|5000|384|1995|5|10|27|4|1995|384|5000|Friday|1995Q4|N|Y|N|2449992|2450264|2449653|2449926|N|N|N|N|N| +2450019|AAAAAAAADGCGFCAA|1995-10-28|1149|5000|384|1995|6|10|28|4|1995|384|5000|Saturday|1995Q4|N|Y|N|2449992|2450264|2449654|2449927|N|N|N|N|N| +2450020|AAAAAAAAEGCGFCAA|1995-10-29|1149|5000|384|1995|0|10|29|4|1995|384|5000|Sunday|1995Q4|N|N|N|2449992|2450264|2449655|2449928|N|N|N|N|N| +2450021|AAAAAAAAFGCGFCAA|1995-10-30|1149|5000|384|1995|1|10|30|4|1995|384|5000|Monday|1995Q4|N|N|N|2449992|2450264|2449656|2449929|N|N|N|N|N| +2450022|AAAAAAAAGGCGFCAA|1995-10-31|1149|5001|384|1995|2|10|31|4|1995|384|5001|Tuesday|1995Q4|N|N|N|2449992|2450264|2449657|2449930|N|N|N|N|N| +2450023|AAAAAAAAHGCGFCAA|1995-11-01|1150|5001|384|1995|3|11|1|4|1995|384|5001|Wednesday|1995Q4|N|N|N|2450023|2450326|2449658|2449931|N|N|N|N|N| +2450024|AAAAAAAAIGCGFCAA|1995-11-02|1150|5001|384|1995|4|11|2|4|1995|384|5001|Thursday|1995Q4|N|N|N|2450023|2450326|2449659|2449932|N|N|N|N|N| +2450025|AAAAAAAAJGCGFCAA|1995-11-03|1150|5001|384|1995|5|11|3|4|1995|384|5001|Friday|1995Q4|N|Y|N|2450023|2450326|2449660|2449933|N|N|N|N|N| +2450026|AAAAAAAAKGCGFCAA|1995-11-04|1150|5001|384|1995|6|11|4|4|1995|384|5001|Saturday|1995Q4|N|Y|N|2450023|2450326|2449661|2449934|N|N|N|N|N| +2450027|AAAAAAAALGCGFCAA|1995-11-05|1150|5001|384|1995|0|11|5|4|1995|384|5001|Sunday|1995Q4|N|N|N|2450023|2450326|2449662|2449935|N|N|N|N|N| +2450028|AAAAAAAAMGCGFCAA|1995-11-06|1150|5001|384|1995|1|11|6|4|1995|384|5001|Monday|1995Q4|N|N|N|2450023|2450326|2449663|2449936|N|N|N|N|N| +2450029|AAAAAAAANGCGFCAA|1995-11-07|1150|5002|384|1995|2|11|7|4|1995|384|5002|Tuesday|1995Q4|N|N|N|2450023|2450326|2449664|2449937|N|N|N|N|N| +2450030|AAAAAAAAOGCGFCAA|1995-11-08|1150|5002|384|1995|3|11|8|4|1995|384|5002|Wednesday|1995Q4|N|N|N|2450023|2450326|2449665|2449938|N|N|N|N|N| +2450031|AAAAAAAAPGCGFCAA|1995-11-09|1150|5002|384|1995|4|11|9|4|1995|384|5002|Thursday|1995Q4|N|N|N|2450023|2450326|2449666|2449939|N|N|N|N|N| +2450032|AAAAAAAAAHCGFCAA|1995-11-10|1150|5002|384|1995|5|11|10|4|1995|384|5002|Friday|1995Q4|N|Y|N|2450023|2450326|2449667|2449940|N|N|N|N|N| +2450033|AAAAAAAABHCGFCAA|1995-11-11|1150|5002|384|1995|6|11|11|4|1995|384|5002|Saturday|1995Q4|N|Y|N|2450023|2450326|2449668|2449941|N|N|N|N|N| +2450034|AAAAAAAACHCGFCAA|1995-11-12|1150|5002|384|1995|0|11|12|4|1995|384|5002|Sunday|1995Q4|N|N|N|2450023|2450326|2449669|2449942|N|N|N|N|N| +2450035|AAAAAAAADHCGFCAA|1995-11-13|1150|5002|384|1995|1|11|13|4|1995|384|5002|Monday|1995Q4|N|N|N|2450023|2450326|2449670|2449943|N|N|N|N|N| +2450036|AAAAAAAAEHCGFCAA|1995-11-14|1150|5003|384|1995|2|11|14|4|1995|384|5003|Tuesday|1995Q4|N|N|N|2450023|2450326|2449671|2449944|N|N|N|N|N| +2450037|AAAAAAAAFHCGFCAA|1995-11-15|1150|5003|384|1995|3|11|15|4|1995|384|5003|Wednesday|1995Q4|N|N|N|2450023|2450326|2449672|2449945|N|N|N|N|N| +2450038|AAAAAAAAGHCGFCAA|1995-11-16|1150|5003|384|1995|4|11|16|4|1995|384|5003|Thursday|1995Q4|N|N|N|2450023|2450326|2449673|2449946|N|N|N|N|N| +2450039|AAAAAAAAHHCGFCAA|1995-11-17|1150|5003|384|1995|5|11|17|4|1995|384|5003|Friday|1995Q4|N|Y|N|2450023|2450326|2449674|2449947|N|N|N|N|N| +2450040|AAAAAAAAIHCGFCAA|1995-11-18|1150|5003|384|1995|6|11|18|4|1995|384|5003|Saturday|1995Q4|N|Y|N|2450023|2450326|2449675|2449948|N|N|N|N|N| +2450041|AAAAAAAAJHCGFCAA|1995-11-19|1150|5003|384|1995|0|11|19|4|1995|384|5003|Sunday|1995Q4|N|N|N|2450023|2450326|2449676|2449949|N|N|N|N|N| +2450042|AAAAAAAAKHCGFCAA|1995-11-20|1150|5003|384|1995|1|11|20|4|1995|384|5003|Monday|1995Q4|N|N|N|2450023|2450326|2449677|2449950|N|N|N|N|N| +2450043|AAAAAAAALHCGFCAA|1995-11-21|1150|5004|384|1995|2|11|21|4|1995|384|5004|Tuesday|1995Q4|N|N|N|2450023|2450326|2449678|2449951|N|N|N|N|N| +2450044|AAAAAAAAMHCGFCAA|1995-11-22|1150|5004|384|1995|3|11|22|4|1995|384|5004|Wednesday|1995Q4|N|N|N|2450023|2450326|2449679|2449952|N|N|N|N|N| +2450045|AAAAAAAANHCGFCAA|1995-11-23|1150|5004|384|1995|4|11|23|4|1995|384|5004|Thursday|1995Q4|N|N|N|2450023|2450326|2449680|2449953|N|N|N|N|N| +2450046|AAAAAAAAOHCGFCAA|1995-11-24|1150|5004|384|1995|5|11|24|4|1995|384|5004|Friday|1995Q4|N|Y|N|2450023|2450326|2449681|2449954|N|N|N|N|N| +2450047|AAAAAAAAPHCGFCAA|1995-11-25|1150|5004|384|1995|6|11|25|4|1995|384|5004|Saturday|1995Q4|N|Y|N|2450023|2450326|2449682|2449955|N|N|N|N|N| +2450048|AAAAAAAAAICGFCAA|1995-11-26|1150|5004|384|1995|0|11|26|4|1995|384|5004|Sunday|1995Q4|N|N|N|2450023|2450326|2449683|2449956|N|N|N|N|N| +2450049|AAAAAAAABICGFCAA|1995-11-27|1150|5004|384|1995|1|11|27|4|1995|384|5004|Monday|1995Q4|N|N|N|2450023|2450326|2449684|2449957|N|N|N|N|N| +2450050|AAAAAAAACICGFCAA|1995-11-28|1150|5005|384|1995|2|11|28|4|1995|384|5005|Tuesday|1995Q4|N|N|N|2450023|2450326|2449685|2449958|N|N|N|N|N| +2450051|AAAAAAAADICGFCAA|1995-11-29|1150|5005|384|1995|3|11|29|4|1995|384|5005|Wednesday|1995Q4|N|N|N|2450023|2450326|2449686|2449959|N|N|N|N|N| +2450052|AAAAAAAAEICGFCAA|1995-11-30|1150|5005|384|1995|4|11|30|4|1995|384|5005|Thursday|1995Q4|N|N|N|2450023|2450326|2449687|2449960|N|N|N|N|N| +2450053|AAAAAAAAFICGFCAA|1995-12-01|1151|5005|385|1995|5|12|1|4|1995|385|5005|Friday|1995Q4|N|Y|N|2450053|2450386|2449688|2449961|N|N|N|N|N| +2450054|AAAAAAAAGICGFCAA|1995-12-02|1151|5005|385|1995|6|12|2|4|1995|385|5005|Saturday|1995Q4|N|Y|N|2450053|2450386|2449689|2449962|N|N|N|N|N| +2450055|AAAAAAAAHICGFCAA|1995-12-03|1151|5005|385|1995|0|12|3|4|1995|385|5005|Sunday|1995Q4|N|N|N|2450053|2450386|2449690|2449963|N|N|N|N|N| +2450056|AAAAAAAAIICGFCAA|1995-12-04|1151|5005|385|1995|1|12|4|4|1995|385|5005|Monday|1995Q4|N|N|N|2450053|2450386|2449691|2449964|N|N|N|N|N| +2450057|AAAAAAAAJICGFCAA|1995-12-05|1151|5006|385|1995|2|12|5|4|1995|385|5006|Tuesday|1995Q4|N|N|N|2450053|2450386|2449692|2449965|N|N|N|N|N| +2450058|AAAAAAAAKICGFCAA|1995-12-06|1151|5006|385|1995|3|12|6|4|1995|385|5006|Wednesday|1995Q4|N|N|N|2450053|2450386|2449693|2449966|N|N|N|N|N| +2450059|AAAAAAAALICGFCAA|1995-12-07|1151|5006|385|1995|4|12|7|4|1995|385|5006|Thursday|1995Q4|N|N|N|2450053|2450386|2449694|2449967|N|N|N|N|N| +2450060|AAAAAAAAMICGFCAA|1995-12-08|1151|5006|385|1995|5|12|8|4|1995|385|5006|Friday|1995Q4|N|Y|N|2450053|2450386|2449695|2449968|N|N|N|N|N| +2450061|AAAAAAAANICGFCAA|1995-12-09|1151|5006|385|1995|6|12|9|4|1995|385|5006|Saturday|1995Q4|N|Y|N|2450053|2450386|2449696|2449969|N|N|N|N|N| +2450062|AAAAAAAAOICGFCAA|1995-12-10|1151|5006|385|1995|0|12|10|4|1995|385|5006|Sunday|1995Q4|N|N|N|2450053|2450386|2449697|2449970|N|N|N|N|N| +2450063|AAAAAAAAPICGFCAA|1995-12-11|1151|5006|385|1995|1|12|11|4|1995|385|5006|Monday|1995Q4|N|N|N|2450053|2450386|2449698|2449971|N|N|N|N|N| +2450064|AAAAAAAAAJCGFCAA|1995-12-12|1151|5007|385|1995|2|12|12|4|1995|385|5007|Tuesday|1995Q4|N|N|N|2450053|2450386|2449699|2449972|N|N|N|N|N| +2450065|AAAAAAAABJCGFCAA|1995-12-13|1151|5007|385|1995|3|12|13|4|1995|385|5007|Wednesday|1995Q4|N|N|N|2450053|2450386|2449700|2449973|N|N|N|N|N| +2450066|AAAAAAAACJCGFCAA|1995-12-14|1151|5007|385|1995|4|12|14|4|1995|385|5007|Thursday|1995Q4|N|N|N|2450053|2450386|2449701|2449974|N|N|N|N|N| +2450067|AAAAAAAADJCGFCAA|1995-12-15|1151|5007|385|1995|5|12|15|4|1995|385|5007|Friday|1995Q4|N|Y|N|2450053|2450386|2449702|2449975|N|N|N|N|N| +2450068|AAAAAAAAEJCGFCAA|1995-12-16|1151|5007|385|1995|6|12|16|4|1995|385|5007|Saturday|1995Q4|N|Y|N|2450053|2450386|2449703|2449976|N|N|N|N|N| +2450069|AAAAAAAAFJCGFCAA|1995-12-17|1151|5007|385|1995|0|12|17|4|1995|385|5007|Sunday|1995Q4|N|N|N|2450053|2450386|2449704|2449977|N|N|N|N|N| +2450070|AAAAAAAAGJCGFCAA|1995-12-18|1151|5007|385|1995|1|12|18|4|1995|385|5007|Monday|1995Q4|N|N|N|2450053|2450386|2449705|2449978|N|N|N|N|N| +2450071|AAAAAAAAHJCGFCAA|1995-12-19|1151|5008|385|1995|2|12|19|4|1995|385|5008|Tuesday|1995Q4|N|N|N|2450053|2450386|2449706|2449979|N|N|N|N|N| +2450072|AAAAAAAAIJCGFCAA|1995-12-20|1151|5008|385|1995|3|12|20|4|1995|385|5008|Wednesday|1995Q4|N|N|N|2450053|2450386|2449707|2449980|N|N|N|N|N| +2450073|AAAAAAAAJJCGFCAA|1995-12-21|1151|5008|385|1995|4|12|21|4|1995|385|5008|Thursday|1995Q4|N|N|N|2450053|2450386|2449708|2449981|N|N|N|N|N| +2450074|AAAAAAAAKJCGFCAA|1995-12-22|1151|5008|385|1995|5|12|22|4|1995|385|5008|Friday|1995Q4|N|Y|N|2450053|2450386|2449709|2449982|N|N|N|N|N| +2450075|AAAAAAAALJCGFCAA|1995-12-23|1151|5008|385|1995|6|12|23|4|1995|385|5008|Saturday|1995Q4|N|Y|N|2450053|2450386|2449710|2449983|N|N|N|N|N| +2450076|AAAAAAAAMJCGFCAA|1995-12-24|1151|5008|385|1995|0|12|24|4|1995|385|5008|Sunday|1995Q4|N|N|N|2450053|2450386|2449711|2449984|N|N|N|N|N| +2450077|AAAAAAAANJCGFCAA|1995-12-25|1151|5008|385|1995|1|12|25|4|1995|385|5008|Monday|1995Q4|N|N|N|2450053|2450386|2449712|2449985|N|N|N|N|N| +2450078|AAAAAAAAOJCGFCAA|1995-12-26|1151|5009|385|1995|2|12|26|4|1995|385|5009|Tuesday|1995Q4|Y|N|N|2450053|2450386|2449713|2449986|N|N|N|N|N| +2450079|AAAAAAAAPJCGFCAA|1995-12-27|1151|5009|385|1995|3|12|27|4|1995|385|5009|Wednesday|1995Q4|N|N|Y|2450053|2450386|2449714|2449987|N|N|N|N|N| +2450080|AAAAAAAAAKCGFCAA|1995-12-28|1151|5009|385|1995|4|12|28|4|1995|385|5009|Thursday|1995Q4|N|N|N|2450053|2450386|2449715|2449988|N|N|N|N|N| +2450081|AAAAAAAABKCGFCAA|1995-12-29|1151|5009|385|1995|5|12|29|4|1995|385|5009|Friday|1995Q4|N|Y|N|2450053|2450386|2449716|2449989|N|N|N|N|N| +2450082|AAAAAAAACKCGFCAA|1995-12-30|1151|5009|385|1995|6|12|30|4|1995|385|5009|Saturday|1995Q4|N|Y|N|2450053|2450386|2449717|2449990|N|N|N|N|N| +2450083|AAAAAAAADKCGFCAA|1995-12-31|1151|5009|385|1995|0|12|31|4|1995|385|5009|Sunday|1995Q4|N|N|N|2450053|2450386|2449718|2449991|N|N|N|N|N| +2450084|AAAAAAAAEKCGFCAA|1996-01-01|1152|5009|385|1996|1|1|1|1|1996|385|5009|Monday|1996Q1|Y|N|N|2450084|2450083|2449719|2449992|N|N|N|N|N| +2450085|AAAAAAAAFKCGFCAA|1996-01-02|1152|5010|385|1996|2|1|2|1|1996|385|5010|Tuesday|1996Q1|N|N|Y|2450084|2450083|2449720|2449993|N|N|N|N|N| +2450086|AAAAAAAAGKCGFCAA|1996-01-03|1152|5010|385|1996|3|1|3|1|1996|385|5010|Wednesday|1996Q1|N|N|N|2450084|2450083|2449721|2449994|N|N|N|N|N| +2450087|AAAAAAAAHKCGFCAA|1996-01-04|1152|5010|385|1996|4|1|4|1|1996|385|5010|Thursday|1996Q1|N|N|N|2450084|2450083|2449722|2449995|N|N|N|N|N| +2450088|AAAAAAAAIKCGFCAA|1996-01-05|1152|5010|385|1996|5|1|5|1|1996|385|5010|Friday|1996Q1|N|Y|N|2450084|2450083|2449723|2449996|N|N|N|N|N| +2450089|AAAAAAAAJKCGFCAA|1996-01-06|1152|5010|385|1996|6|1|6|1|1996|385|5010|Saturday|1996Q1|N|Y|N|2450084|2450083|2449724|2449997|N|N|N|N|N| +2450090|AAAAAAAAKKCGFCAA|1996-01-07|1152|5010|385|1996|0|1|7|1|1996|385|5010|Sunday|1996Q1|N|N|N|2450084|2450083|2449725|2449998|N|N|N|N|N| +2450091|AAAAAAAALKCGFCAA|1996-01-08|1152|5010|385|1996|1|1|8|1|1996|385|5010|Monday|1996Q1|N|N|N|2450084|2450083|2449726|2449999|N|N|N|N|N| +2450092|AAAAAAAAMKCGFCAA|1996-01-09|1152|5011|385|1996|2|1|9|1|1996|385|5011|Tuesday|1996Q1|N|N|N|2450084|2450083|2449727|2450000|N|N|N|N|N| +2450093|AAAAAAAANKCGFCAA|1996-01-10|1152|5011|385|1996|3|1|10|1|1996|385|5011|Wednesday|1996Q1|N|N|N|2450084|2450083|2449728|2450001|N|N|N|N|N| +2450094|AAAAAAAAOKCGFCAA|1996-01-11|1152|5011|385|1996|4|1|11|1|1996|385|5011|Thursday|1996Q1|N|N|N|2450084|2450083|2449729|2450002|N|N|N|N|N| +2450095|AAAAAAAAPKCGFCAA|1996-01-12|1152|5011|385|1996|5|1|12|1|1996|385|5011|Friday|1996Q1|N|Y|N|2450084|2450083|2449730|2450003|N|N|N|N|N| +2450096|AAAAAAAAALCGFCAA|1996-01-13|1152|5011|385|1996|6|1|13|1|1996|385|5011|Saturday|1996Q1|N|Y|N|2450084|2450083|2449731|2450004|N|N|N|N|N| +2450097|AAAAAAAABLCGFCAA|1996-01-14|1152|5011|385|1996|0|1|14|1|1996|385|5011|Sunday|1996Q1|N|N|N|2450084|2450083|2449732|2450005|N|N|N|N|N| +2450098|AAAAAAAACLCGFCAA|1996-01-15|1152|5011|385|1996|1|1|15|1|1996|385|5011|Monday|1996Q1|N|N|N|2450084|2450083|2449733|2450006|N|N|N|N|N| +2450099|AAAAAAAADLCGFCAA|1996-01-16|1152|5012|385|1996|2|1|16|1|1996|385|5012|Tuesday|1996Q1|N|N|N|2450084|2450083|2449734|2450007|N|N|N|N|N| +2450100|AAAAAAAAELCGFCAA|1996-01-17|1152|5012|385|1996|3|1|17|1|1996|385|5012|Wednesday|1996Q1|N|N|N|2450084|2450083|2449735|2450008|N|N|N|N|N| +2450101|AAAAAAAAFLCGFCAA|1996-01-18|1152|5012|385|1996|4|1|18|1|1996|385|5012|Thursday|1996Q1|N|N|N|2450084|2450083|2449736|2450009|N|N|N|N|N| +2450102|AAAAAAAAGLCGFCAA|1996-01-19|1152|5012|385|1996|5|1|19|1|1996|385|5012|Friday|1996Q1|N|Y|N|2450084|2450083|2449737|2450010|N|N|N|N|N| +2450103|AAAAAAAAHLCGFCAA|1996-01-20|1152|5012|385|1996|6|1|20|1|1996|385|5012|Saturday|1996Q1|N|Y|N|2450084|2450083|2449738|2450011|N|N|N|N|N| +2450104|AAAAAAAAILCGFCAA|1996-01-21|1152|5012|385|1996|0|1|21|1|1996|385|5012|Sunday|1996Q1|N|N|N|2450084|2450083|2449739|2450012|N|N|N|N|N| +2450105|AAAAAAAAJLCGFCAA|1996-01-22|1152|5012|385|1996|1|1|22|1|1996|385|5012|Monday|1996Q1|N|N|N|2450084|2450083|2449740|2450013|N|N|N|N|N| +2450106|AAAAAAAAKLCGFCAA|1996-01-23|1152|5013|385|1996|2|1|23|1|1996|385|5013|Tuesday|1996Q1|N|N|N|2450084|2450083|2449741|2450014|N|N|N|N|N| +2450107|AAAAAAAALLCGFCAA|1996-01-24|1152|5013|385|1996|3|1|24|1|1996|385|5013|Wednesday|1996Q1|N|N|N|2450084|2450083|2449742|2450015|N|N|N|N|N| +2450108|AAAAAAAAMLCGFCAA|1996-01-25|1152|5013|385|1996|4|1|25|1|1996|385|5013|Thursday|1996Q1|N|N|N|2450084|2450083|2449743|2450016|N|N|N|N|N| +2450109|AAAAAAAANLCGFCAA|1996-01-26|1152|5013|385|1996|5|1|26|1|1996|385|5013|Friday|1996Q1|N|Y|N|2450084|2450083|2449744|2450017|N|N|N|N|N| +2450110|AAAAAAAAOLCGFCAA|1996-01-27|1152|5013|385|1996|6|1|27|1|1996|385|5013|Saturday|1996Q1|N|Y|N|2450084|2450083|2449745|2450018|N|N|N|N|N| +2450111|AAAAAAAAPLCGFCAA|1996-01-28|1152|5013|385|1996|0|1|28|1|1996|385|5013|Sunday|1996Q1|N|N|N|2450084|2450083|2449746|2450019|N|N|N|N|N| +2450112|AAAAAAAAAMCGFCAA|1996-01-29|1152|5013|385|1996|1|1|29|1|1996|385|5013|Monday|1996Q1|N|N|N|2450084|2450083|2449747|2450020|N|N|N|N|N| +2450113|AAAAAAAABMCGFCAA|1996-01-30|1152|5014|385|1996|2|1|30|1|1996|385|5014|Tuesday|1996Q1|N|N|N|2450084|2450083|2449748|2450021|N|N|N|N|N| +2450114|AAAAAAAACMCGFCAA|1996-01-31|1152|5014|385|1996|3|1|31|1|1996|385|5014|Wednesday|1996Q1|N|N|N|2450084|2450083|2449749|2450022|N|N|N|N|N| +2450115|AAAAAAAADMCGFCAA|1996-02-01|1153|5014|385|1996|4|2|1|1|1996|385|5014|Thursday|1996Q1|N|N|N|2450115|2450145|2449750|2450023|N|N|N|N|N| +2450116|AAAAAAAAEMCGFCAA|1996-02-02|1153|5014|385|1996|5|2|2|1|1996|385|5014|Friday|1996Q1|N|Y|N|2450115|2450145|2449751|2450024|N|N|N|N|N| +2450117|AAAAAAAAFMCGFCAA|1996-02-03|1153|5014|385|1996|6|2|3|1|1996|385|5014|Saturday|1996Q1|N|Y|N|2450115|2450145|2449752|2450025|N|N|N|N|N| +2450118|AAAAAAAAGMCGFCAA|1996-02-04|1153|5014|385|1996|0|2|4|1|1996|385|5014|Sunday|1996Q1|N|N|N|2450115|2450145|2449753|2450026|N|N|N|N|N| +2450119|AAAAAAAAHMCGFCAA|1996-02-05|1153|5014|385|1996|1|2|5|1|1996|385|5014|Monday|1996Q1|N|N|N|2450115|2450145|2449754|2450027|N|N|N|N|N| +2450120|AAAAAAAAIMCGFCAA|1996-02-06|1153|5015|385|1996|2|2|6|1|1996|385|5015|Tuesday|1996Q1|N|N|N|2450115|2450145|2449755|2450028|N|N|N|N|N| +2450121|AAAAAAAAJMCGFCAA|1996-02-07|1153|5015|385|1996|3|2|7|1|1996|385|5015|Wednesday|1996Q1|N|N|N|2450115|2450145|2449756|2450029|N|N|N|N|N| +2450122|AAAAAAAAKMCGFCAA|1996-02-08|1153|5015|385|1996|4|2|8|1|1996|385|5015|Thursday|1996Q1|N|N|N|2450115|2450145|2449757|2450030|N|N|N|N|N| +2450123|AAAAAAAALMCGFCAA|1996-02-09|1153|5015|385|1996|5|2|9|1|1996|385|5015|Friday|1996Q1|N|Y|N|2450115|2450145|2449758|2450031|N|N|N|N|N| +2450124|AAAAAAAAMMCGFCAA|1996-02-10|1153|5015|385|1996|6|2|10|1|1996|385|5015|Saturday|1996Q1|N|Y|N|2450115|2450145|2449759|2450032|N|N|N|N|N| +2450125|AAAAAAAANMCGFCAA|1996-02-11|1153|5015|385|1996|0|2|11|1|1996|385|5015|Sunday|1996Q1|N|N|N|2450115|2450145|2449760|2450033|N|N|N|N|N| +2450126|AAAAAAAAOMCGFCAA|1996-02-12|1153|5015|385|1996|1|2|12|1|1996|385|5015|Monday|1996Q1|N|N|N|2450115|2450145|2449761|2450034|N|N|N|N|N| +2450127|AAAAAAAAPMCGFCAA|1996-02-13|1153|5016|385|1996|2|2|13|1|1996|385|5016|Tuesday|1996Q1|N|N|N|2450115|2450145|2449762|2450035|N|N|N|N|N| +2450128|AAAAAAAAANCGFCAA|1996-02-14|1153|5016|385|1996|3|2|14|1|1996|385|5016|Wednesday|1996Q1|N|N|N|2450115|2450145|2449763|2450036|N|N|N|N|N| +2450129|AAAAAAAABNCGFCAA|1996-02-15|1153|5016|385|1996|4|2|15|1|1996|385|5016|Thursday|1996Q1|N|N|N|2450115|2450145|2449764|2450037|N|N|N|N|N| +2450130|AAAAAAAACNCGFCAA|1996-02-16|1153|5016|385|1996|5|2|16|1|1996|385|5016|Friday|1996Q1|N|Y|N|2450115|2450145|2449765|2450038|N|N|N|N|N| +2450131|AAAAAAAADNCGFCAA|1996-02-17|1153|5016|385|1996|6|2|17|1|1996|385|5016|Saturday|1996Q1|N|Y|N|2450115|2450145|2449766|2450039|N|N|N|N|N| +2450132|AAAAAAAAENCGFCAA|1996-02-18|1153|5016|385|1996|0|2|18|1|1996|385|5016|Sunday|1996Q1|N|N|N|2450115|2450145|2449767|2450040|N|N|N|N|N| +2450133|AAAAAAAAFNCGFCAA|1996-02-19|1153|5016|385|1996|1|2|19|1|1996|385|5016|Monday|1996Q1|N|N|N|2450115|2450145|2449768|2450041|N|N|N|N|N| +2450134|AAAAAAAAGNCGFCAA|1996-02-20|1153|5017|385|1996|2|2|20|1|1996|385|5017|Tuesday|1996Q1|N|N|N|2450115|2450145|2449769|2450042|N|N|N|N|N| +2450135|AAAAAAAAHNCGFCAA|1996-02-21|1153|5017|385|1996|3|2|21|1|1996|385|5017|Wednesday|1996Q1|N|N|N|2450115|2450145|2449770|2450043|N|N|N|N|N| +2450136|AAAAAAAAINCGFCAA|1996-02-22|1153|5017|385|1996|4|2|22|1|1996|385|5017|Thursday|1996Q1|N|N|N|2450115|2450145|2449771|2450044|N|N|N|N|N| +2450137|AAAAAAAAJNCGFCAA|1996-02-23|1153|5017|385|1996|5|2|23|1|1996|385|5017|Friday|1996Q1|N|Y|N|2450115|2450145|2449772|2450045|N|N|N|N|N| +2450138|AAAAAAAAKNCGFCAA|1996-02-24|1153|5017|385|1996|6|2|24|1|1996|385|5017|Saturday|1996Q1|N|Y|N|2450115|2450145|2449773|2450046|N|N|N|N|N| +2450139|AAAAAAAALNCGFCAA|1996-02-25|1153|5017|385|1996|0|2|25|1|1996|385|5017|Sunday|1996Q1|N|N|N|2450115|2450145|2449774|2450047|N|N|N|N|N| +2450140|AAAAAAAAMNCGFCAA|1996-02-26|1153|5017|385|1996|1|2|26|1|1996|385|5017|Monday|1996Q1|N|N|N|2450115|2450145|2449775|2450048|N|N|N|N|N| +2450141|AAAAAAAANNCGFCAA|1996-02-27|1153|5018|385|1996|2|2|27|1|1996|385|5018|Tuesday|1996Q1|N|N|N|2450115|2450145|2449776|2450049|N|N|N|N|N| +2450142|AAAAAAAAONCGFCAA|1996-02-28|1153|5018|385|1996|3|2|28|1|1996|385|5018|Wednesday|1996Q1|N|N|N|2450115|2450145|2449777|2450050|N|N|N|N|N| +2450143|AAAAAAAAPNCGFCAA|1996-02-29|1153|5018|385|1996|4|2|29|1|1996|385|5018|Thursday|1996Q1|N|N|N|2450115|2450145|2449777|2450051|N|N|N|N|N| +2450144|AAAAAAAAAOCGFCAA|1996-03-01|1154|5018|386|1996|5|3|1|1|1996|386|5018|Friday|1996Q1|N|Y|N|2450144|2450203|2449778|2450052|N|N|N|N|N| +2450145|AAAAAAAABOCGFCAA|1996-03-02|1154|5018|386|1996|6|3|2|1|1996|386|5018|Saturday|1996Q1|N|Y|N|2450144|2450203|2449779|2450053|N|N|N|N|N| +2450146|AAAAAAAACOCGFCAA|1996-03-03|1154|5018|386|1996|0|3|3|1|1996|386|5018|Sunday|1996Q1|N|N|N|2450144|2450203|2449780|2450054|N|N|N|N|N| +2450147|AAAAAAAADOCGFCAA|1996-03-04|1154|5018|386|1996|1|3|4|1|1996|386|5018|Monday|1996Q1|N|N|N|2450144|2450203|2449781|2450055|N|N|N|N|N| +2450148|AAAAAAAAEOCGFCAA|1996-03-05|1154|5019|386|1996|2|3|5|1|1996|386|5019|Tuesday|1996Q1|N|N|N|2450144|2450203|2449782|2450056|N|N|N|N|N| +2450149|AAAAAAAAFOCGFCAA|1996-03-06|1154|5019|386|1996|3|3|6|1|1996|386|5019|Wednesday|1996Q1|N|N|N|2450144|2450203|2449783|2450057|N|N|N|N|N| +2450150|AAAAAAAAGOCGFCAA|1996-03-07|1154|5019|386|1996|4|3|7|1|1996|386|5019|Thursday|1996Q1|N|N|N|2450144|2450203|2449784|2450058|N|N|N|N|N| +2450151|AAAAAAAAHOCGFCAA|1996-03-08|1154|5019|386|1996|5|3|8|1|1996|386|5019|Friday|1996Q1|N|Y|N|2450144|2450203|2449785|2450059|N|N|N|N|N| +2450152|AAAAAAAAIOCGFCAA|1996-03-09|1154|5019|386|1996|6|3|9|1|1996|386|5019|Saturday|1996Q1|N|Y|N|2450144|2450203|2449786|2450060|N|N|N|N|N| +2450153|AAAAAAAAJOCGFCAA|1996-03-10|1154|5019|386|1996|0|3|10|1|1996|386|5019|Sunday|1996Q1|N|N|N|2450144|2450203|2449787|2450061|N|N|N|N|N| +2450154|AAAAAAAAKOCGFCAA|1996-03-11|1154|5019|386|1996|1|3|11|1|1996|386|5019|Monday|1996Q1|N|N|N|2450144|2450203|2449788|2450062|N|N|N|N|N| +2450155|AAAAAAAALOCGFCAA|1996-03-12|1154|5020|386|1996|2|3|12|1|1996|386|5020|Tuesday|1996Q1|N|N|N|2450144|2450203|2449789|2450063|N|N|N|N|N| +2450156|AAAAAAAAMOCGFCAA|1996-03-13|1154|5020|386|1996|3|3|13|1|1996|386|5020|Wednesday|1996Q1|N|N|N|2450144|2450203|2449790|2450064|N|N|N|N|N| +2450157|AAAAAAAANOCGFCAA|1996-03-14|1154|5020|386|1996|4|3|14|1|1996|386|5020|Thursday|1996Q1|N|N|N|2450144|2450203|2449791|2450065|N|N|N|N|N| +2450158|AAAAAAAAOOCGFCAA|1996-03-15|1154|5020|386|1996|5|3|15|1|1996|386|5020|Friday|1996Q1|N|Y|N|2450144|2450203|2449792|2450066|N|N|N|N|N| +2450159|AAAAAAAAPOCGFCAA|1996-03-16|1154|5020|386|1996|6|3|16|1|1996|386|5020|Saturday|1996Q1|N|Y|N|2450144|2450203|2449793|2450067|N|N|N|N|N| +2450160|AAAAAAAAAPCGFCAA|1996-03-17|1154|5020|386|1996|0|3|17|1|1996|386|5020|Sunday|1996Q1|N|N|N|2450144|2450203|2449794|2450068|N|N|N|N|N| +2450161|AAAAAAAABPCGFCAA|1996-03-18|1154|5020|386|1996|1|3|18|1|1996|386|5020|Monday|1996Q1|N|N|N|2450144|2450203|2449795|2450069|N|N|N|N|N| +2450162|AAAAAAAACPCGFCAA|1996-03-19|1154|5021|386|1996|2|3|19|1|1996|386|5021|Tuesday|1996Q1|N|N|N|2450144|2450203|2449796|2450070|N|N|N|N|N| +2450163|AAAAAAAADPCGFCAA|1996-03-20|1154|5021|386|1996|3|3|20|1|1996|386|5021|Wednesday|1996Q1|N|N|N|2450144|2450203|2449797|2450071|N|N|N|N|N| +2450164|AAAAAAAAEPCGFCAA|1996-03-21|1154|5021|386|1996|4|3|21|1|1996|386|5021|Thursday|1996Q1|N|N|N|2450144|2450203|2449798|2450072|N|N|N|N|N| +2450165|AAAAAAAAFPCGFCAA|1996-03-22|1154|5021|386|1996|5|3|22|1|1996|386|5021|Friday|1996Q1|N|Y|N|2450144|2450203|2449799|2450073|N|N|N|N|N| +2450166|AAAAAAAAGPCGFCAA|1996-03-23|1154|5021|386|1996|6|3|23|1|1996|386|5021|Saturday|1996Q1|N|Y|N|2450144|2450203|2449800|2450074|N|N|N|N|N| +2450167|AAAAAAAAHPCGFCAA|1996-03-24|1154|5021|386|1996|0|3|24|1|1996|386|5021|Sunday|1996Q1|N|N|N|2450144|2450203|2449801|2450075|N|N|N|N|N| +2450168|AAAAAAAAIPCGFCAA|1996-03-25|1154|5021|386|1996|1|3|25|1|1996|386|5021|Monday|1996Q1|N|N|N|2450144|2450203|2449802|2450076|N|N|N|N|N| +2450169|AAAAAAAAJPCGFCAA|1996-03-26|1154|5022|386|1996|2|3|26|1|1996|386|5022|Tuesday|1996Q1|N|N|N|2450144|2450203|2449803|2450077|N|N|N|N|N| +2450170|AAAAAAAAKPCGFCAA|1996-03-27|1154|5022|386|1996|3|3|27|1|1996|386|5022|Wednesday|1996Q1|N|N|N|2450144|2450203|2449804|2450078|N|N|N|N|N| +2450171|AAAAAAAALPCGFCAA|1996-03-28|1154|5022|386|1996|4|3|28|1|1996|386|5022|Thursday|1996Q1|N|N|N|2450144|2450203|2449805|2450079|N|N|N|N|N| +2450172|AAAAAAAAMPCGFCAA|1996-03-29|1154|5022|386|1996|5|3|29|1|1996|386|5022|Friday|1996Q1|N|Y|N|2450144|2450203|2449806|2450080|N|N|N|N|N| +2450173|AAAAAAAANPCGFCAA|1996-03-30|1154|5022|386|1996|6|3|30|1|1996|386|5022|Saturday|1996Q1|N|Y|N|2450144|2450203|2449807|2450081|N|N|N|N|N| +2450174|AAAAAAAAOPCGFCAA|1996-03-31|1154|5022|386|1996|0|3|31|1|1996|386|5022|Sunday|1996Q1|N|N|N|2450144|2450203|2449808|2450082|N|N|N|N|N| +2450175|AAAAAAAAPPCGFCAA|1996-04-01|1155|5022|386|1996|1|4|1|2|1996|386|5022|Monday|1996Q2|N|N|N|2450175|2450265|2449809|2450084|N|N|N|N|N| +2450176|AAAAAAAAAADGFCAA|1996-04-02|1155|5023|386|1996|2|4|2|2|1996|386|5023|Tuesday|1996Q2|N|N|N|2450175|2450265|2449810|2450085|N|N|N|N|N| +2450177|AAAAAAAABADGFCAA|1996-04-03|1155|5023|386|1996|3|4|3|2|1996|386|5023|Wednesday|1996Q2|N|N|N|2450175|2450265|2449811|2450086|N|N|N|N|N| +2450178|AAAAAAAACADGFCAA|1996-04-04|1155|5023|386|1996|4|4|4|2|1996|386|5023|Thursday|1996Q2|N|N|N|2450175|2450265|2449812|2450087|N|N|N|N|N| +2450179|AAAAAAAADADGFCAA|1996-04-05|1155|5023|386|1996|5|4|5|2|1996|386|5023|Friday|1996Q2|N|Y|N|2450175|2450265|2449813|2450088|N|N|N|N|N| +2450180|AAAAAAAAEADGFCAA|1996-04-06|1155|5023|386|1996|6|4|6|2|1996|386|5023|Saturday|1996Q2|N|Y|N|2450175|2450265|2449814|2450089|N|N|N|N|N| +2450181|AAAAAAAAFADGFCAA|1996-04-07|1155|5023|386|1996|0|4|7|2|1996|386|5023|Sunday|1996Q2|N|N|N|2450175|2450265|2449815|2450090|N|N|N|N|N| +2450182|AAAAAAAAGADGFCAA|1996-04-08|1155|5023|386|1996|1|4|8|2|1996|386|5023|Monday|1996Q2|N|N|N|2450175|2450265|2449816|2450091|N|N|N|N|N| +2450183|AAAAAAAAHADGFCAA|1996-04-09|1155|5024|386|1996|2|4|9|2|1996|386|5024|Tuesday|1996Q2|N|N|N|2450175|2450265|2449817|2450092|N|N|N|N|N| +2450184|AAAAAAAAIADGFCAA|1996-04-10|1155|5024|386|1996|3|4|10|2|1996|386|5024|Wednesday|1996Q2|N|N|N|2450175|2450265|2449818|2450093|N|N|N|N|N| +2450185|AAAAAAAAJADGFCAA|1996-04-11|1155|5024|386|1996|4|4|11|2|1996|386|5024|Thursday|1996Q2|N|N|N|2450175|2450265|2449819|2450094|N|N|N|N|N| +2450186|AAAAAAAAKADGFCAA|1996-04-12|1155|5024|386|1996|5|4|12|2|1996|386|5024|Friday|1996Q2|N|Y|N|2450175|2450265|2449820|2450095|N|N|N|N|N| +2450187|AAAAAAAALADGFCAA|1996-04-13|1155|5024|386|1996|6|4|13|2|1996|386|5024|Saturday|1996Q2|N|Y|N|2450175|2450265|2449821|2450096|N|N|N|N|N| +2450188|AAAAAAAAMADGFCAA|1996-04-14|1155|5024|386|1996|0|4|14|2|1996|386|5024|Sunday|1996Q2|N|N|N|2450175|2450265|2449822|2450097|N|N|N|N|N| +2450189|AAAAAAAANADGFCAA|1996-04-15|1155|5024|386|1996|1|4|15|2|1996|386|5024|Monday|1996Q2|N|N|N|2450175|2450265|2449823|2450098|N|N|N|N|N| +2450190|AAAAAAAAOADGFCAA|1996-04-16|1155|5025|386|1996|2|4|16|2|1996|386|5025|Tuesday|1996Q2|N|N|N|2450175|2450265|2449824|2450099|N|N|N|N|N| +2450191|AAAAAAAAPADGFCAA|1996-04-17|1155|5025|386|1996|3|4|17|2|1996|386|5025|Wednesday|1996Q2|N|N|N|2450175|2450265|2449825|2450100|N|N|N|N|N| +2450192|AAAAAAAAABDGFCAA|1996-04-18|1155|5025|386|1996|4|4|18|2|1996|386|5025|Thursday|1996Q2|N|N|N|2450175|2450265|2449826|2450101|N|N|N|N|N| +2450193|AAAAAAAABBDGFCAA|1996-04-19|1155|5025|386|1996|5|4|19|2|1996|386|5025|Friday|1996Q2|N|Y|N|2450175|2450265|2449827|2450102|N|N|N|N|N| +2450194|AAAAAAAACBDGFCAA|1996-04-20|1155|5025|386|1996|6|4|20|2|1996|386|5025|Saturday|1996Q2|N|Y|N|2450175|2450265|2449828|2450103|N|N|N|N|N| +2450195|AAAAAAAADBDGFCAA|1996-04-21|1155|5025|386|1996|0|4|21|2|1996|386|5025|Sunday|1996Q2|N|N|N|2450175|2450265|2449829|2450104|N|N|N|N|N| +2450196|AAAAAAAAEBDGFCAA|1996-04-22|1155|5025|386|1996|1|4|22|2|1996|386|5025|Monday|1996Q2|N|N|N|2450175|2450265|2449830|2450105|N|N|N|N|N| +2450197|AAAAAAAAFBDGFCAA|1996-04-23|1155|5026|386|1996|2|4|23|2|1996|386|5026|Tuesday|1996Q2|N|N|N|2450175|2450265|2449831|2450106|N|N|N|N|N| +2450198|AAAAAAAAGBDGFCAA|1996-04-24|1155|5026|386|1996|3|4|24|2|1996|386|5026|Wednesday|1996Q2|N|N|N|2450175|2450265|2449832|2450107|N|N|N|N|N| +2450199|AAAAAAAAHBDGFCAA|1996-04-25|1155|5026|386|1996|4|4|25|2|1996|386|5026|Thursday|1996Q2|N|N|N|2450175|2450265|2449833|2450108|N|N|N|N|N| +2450200|AAAAAAAAIBDGFCAA|1996-04-26|1155|5026|386|1996|5|4|26|2|1996|386|5026|Friday|1996Q2|N|Y|N|2450175|2450265|2449834|2450109|N|N|N|N|N| +2450201|AAAAAAAAJBDGFCAA|1996-04-27|1155|5026|386|1996|6|4|27|2|1996|386|5026|Saturday|1996Q2|N|Y|N|2450175|2450265|2449835|2450110|N|N|N|N|N| +2450202|AAAAAAAAKBDGFCAA|1996-04-28|1155|5026|386|1996|0|4|28|2|1996|386|5026|Sunday|1996Q2|N|N|N|2450175|2450265|2449836|2450111|N|N|N|N|N| +2450203|AAAAAAAALBDGFCAA|1996-04-29|1155|5026|386|1996|1|4|29|2|1996|386|5026|Monday|1996Q2|N|N|N|2450175|2450265|2449837|2450112|N|N|N|N|N| +2450204|AAAAAAAAMBDGFCAA|1996-04-30|1155|5027|386|1996|2|4|30|2|1996|386|5027|Tuesday|1996Q2|N|N|N|2450175|2450265|2449838|2450113|N|N|N|N|N| +2450205|AAAAAAAANBDGFCAA|1996-05-01|1156|5027|386|1996|3|5|1|2|1996|386|5027|Wednesday|1996Q2|N|N|N|2450205|2450325|2449839|2450114|N|N|N|N|N| +2450206|AAAAAAAAOBDGFCAA|1996-05-02|1156|5027|386|1996|4|5|2|2|1996|386|5027|Thursday|1996Q2|N|N|N|2450205|2450325|2449840|2450115|N|N|N|N|N| +2450207|AAAAAAAAPBDGFCAA|1996-05-03|1156|5027|386|1996|5|5|3|2|1996|386|5027|Friday|1996Q2|N|Y|N|2450205|2450325|2449841|2450116|N|N|N|N|N| +2450208|AAAAAAAAACDGFCAA|1996-05-04|1156|5027|386|1996|6|5|4|2|1996|386|5027|Saturday|1996Q2|N|Y|N|2450205|2450325|2449842|2450117|N|N|N|N|N| +2450209|AAAAAAAABCDGFCAA|1996-05-05|1156|5027|386|1996|0|5|5|2|1996|386|5027|Sunday|1996Q2|N|N|N|2450205|2450325|2449843|2450118|N|N|N|N|N| +2450210|AAAAAAAACCDGFCAA|1996-05-06|1156|5027|386|1996|1|5|6|2|1996|386|5027|Monday|1996Q2|N|N|N|2450205|2450325|2449844|2450119|N|N|N|N|N| +2450211|AAAAAAAADCDGFCAA|1996-05-07|1156|5028|386|1996|2|5|7|2|1996|386|5028|Tuesday|1996Q2|N|N|N|2450205|2450325|2449845|2450120|N|N|N|N|N| +2450212|AAAAAAAAECDGFCAA|1996-05-08|1156|5028|386|1996|3|5|8|2|1996|386|5028|Wednesday|1996Q2|N|N|N|2450205|2450325|2449846|2450121|N|N|N|N|N| +2450213|AAAAAAAAFCDGFCAA|1996-05-09|1156|5028|386|1996|4|5|9|2|1996|386|5028|Thursday|1996Q2|N|N|N|2450205|2450325|2449847|2450122|N|N|N|N|N| +2450214|AAAAAAAAGCDGFCAA|1996-05-10|1156|5028|386|1996|5|5|10|2|1996|386|5028|Friday|1996Q2|N|Y|N|2450205|2450325|2449848|2450123|N|N|N|N|N| +2450215|AAAAAAAAHCDGFCAA|1996-05-11|1156|5028|386|1996|6|5|11|2|1996|386|5028|Saturday|1996Q2|N|Y|N|2450205|2450325|2449849|2450124|N|N|N|N|N| +2450216|AAAAAAAAICDGFCAA|1996-05-12|1156|5028|386|1996|0|5|12|2|1996|386|5028|Sunday|1996Q2|N|N|N|2450205|2450325|2449850|2450125|N|N|N|N|N| +2450217|AAAAAAAAJCDGFCAA|1996-05-13|1156|5028|386|1996|1|5|13|2|1996|386|5028|Monday|1996Q2|N|N|N|2450205|2450325|2449851|2450126|N|N|N|N|N| +2450218|AAAAAAAAKCDGFCAA|1996-05-14|1156|5029|386|1996|2|5|14|2|1996|386|5029|Tuesday|1996Q2|N|N|N|2450205|2450325|2449852|2450127|N|N|N|N|N| +2450219|AAAAAAAALCDGFCAA|1996-05-15|1156|5029|386|1996|3|5|15|2|1996|386|5029|Wednesday|1996Q2|N|N|N|2450205|2450325|2449853|2450128|N|N|N|N|N| +2450220|AAAAAAAAMCDGFCAA|1996-05-16|1156|5029|386|1996|4|5|16|2|1996|386|5029|Thursday|1996Q2|N|N|N|2450205|2450325|2449854|2450129|N|N|N|N|N| +2450221|AAAAAAAANCDGFCAA|1996-05-17|1156|5029|386|1996|5|5|17|2|1996|386|5029|Friday|1996Q2|N|Y|N|2450205|2450325|2449855|2450130|N|N|N|N|N| +2450222|AAAAAAAAOCDGFCAA|1996-05-18|1156|5029|386|1996|6|5|18|2|1996|386|5029|Saturday|1996Q2|N|Y|N|2450205|2450325|2449856|2450131|N|N|N|N|N| +2450223|AAAAAAAAPCDGFCAA|1996-05-19|1156|5029|386|1996|0|5|19|2|1996|386|5029|Sunday|1996Q2|N|N|N|2450205|2450325|2449857|2450132|N|N|N|N|N| +2450224|AAAAAAAAADDGFCAA|1996-05-20|1156|5029|386|1996|1|5|20|2|1996|386|5029|Monday|1996Q2|N|N|N|2450205|2450325|2449858|2450133|N|N|N|N|N| +2450225|AAAAAAAABDDGFCAA|1996-05-21|1156|5030|386|1996|2|5|21|2|1996|386|5030|Tuesday|1996Q2|N|N|N|2450205|2450325|2449859|2450134|N|N|N|N|N| +2450226|AAAAAAAACDDGFCAA|1996-05-22|1156|5030|386|1996|3|5|22|2|1996|386|5030|Wednesday|1996Q2|N|N|N|2450205|2450325|2449860|2450135|N|N|N|N|N| +2450227|AAAAAAAADDDGFCAA|1996-05-23|1156|5030|386|1996|4|5|23|2|1996|386|5030|Thursday|1996Q2|N|N|N|2450205|2450325|2449861|2450136|N|N|N|N|N| +2450228|AAAAAAAAEDDGFCAA|1996-05-24|1156|5030|386|1996|5|5|24|2|1996|386|5030|Friday|1996Q2|N|Y|N|2450205|2450325|2449862|2450137|N|N|N|N|N| +2450229|AAAAAAAAFDDGFCAA|1996-05-25|1156|5030|386|1996|6|5|25|2|1996|386|5030|Saturday|1996Q2|N|Y|N|2450205|2450325|2449863|2450138|N|N|N|N|N| +2450230|AAAAAAAAGDDGFCAA|1996-05-26|1156|5030|386|1996|0|5|26|2|1996|386|5030|Sunday|1996Q2|N|N|N|2450205|2450325|2449864|2450139|N|N|N|N|N| +2450231|AAAAAAAAHDDGFCAA|1996-05-27|1156|5030|386|1996|1|5|27|2|1996|386|5030|Monday|1996Q2|N|N|N|2450205|2450325|2449865|2450140|N|N|N|N|N| +2450232|AAAAAAAAIDDGFCAA|1996-05-28|1156|5031|386|1996|2|5|28|2|1996|386|5031|Tuesday|1996Q2|N|N|N|2450205|2450325|2449866|2450141|N|N|N|N|N| +2450233|AAAAAAAAJDDGFCAA|1996-05-29|1156|5031|386|1996|3|5|29|2|1996|386|5031|Wednesday|1996Q2|N|N|N|2450205|2450325|2449867|2450142|N|N|N|N|N| +2450234|AAAAAAAAKDDGFCAA|1996-05-30|1156|5031|386|1996|4|5|30|2|1996|386|5031|Thursday|1996Q2|N|N|N|2450205|2450325|2449868|2450143|N|N|N|N|N| +2450235|AAAAAAAALDDGFCAA|1996-05-31|1156|5031|386|1996|5|5|31|2|1996|386|5031|Friday|1996Q2|N|Y|N|2450205|2450325|2449869|2450144|N|N|N|N|N| +2450236|AAAAAAAAMDDGFCAA|1996-06-01|1157|5031|387|1996|6|6|1|2|1996|387|5031|Saturday|1996Q2|N|Y|N|2450236|2450387|2449870|2450145|N|N|N|N|N| +2450237|AAAAAAAANDDGFCAA|1996-06-02|1157|5031|387|1996|0|6|2|2|1996|387|5031|Sunday|1996Q2|N|N|N|2450236|2450387|2449871|2450146|N|N|N|N|N| +2450238|AAAAAAAAODDGFCAA|1996-06-03|1157|5031|387|1996|1|6|3|2|1996|387|5031|Monday|1996Q2|N|N|N|2450236|2450387|2449872|2450147|N|N|N|N|N| +2450239|AAAAAAAAPDDGFCAA|1996-06-04|1157|5032|387|1996|2|6|4|2|1996|387|5032|Tuesday|1996Q2|N|N|N|2450236|2450387|2449873|2450148|N|N|N|N|N| +2450240|AAAAAAAAAEDGFCAA|1996-06-05|1157|5032|387|1996|3|6|5|2|1996|387|5032|Wednesday|1996Q2|N|N|N|2450236|2450387|2449874|2450149|N|N|N|N|N| +2450241|AAAAAAAABEDGFCAA|1996-06-06|1157|5032|387|1996|4|6|6|2|1996|387|5032|Thursday|1996Q2|N|N|N|2450236|2450387|2449875|2450150|N|N|N|N|N| +2450242|AAAAAAAACEDGFCAA|1996-06-07|1157|5032|387|1996|5|6|7|2|1996|387|5032|Friday|1996Q2|N|Y|N|2450236|2450387|2449876|2450151|N|N|N|N|N| +2450243|AAAAAAAADEDGFCAA|1996-06-08|1157|5032|387|1996|6|6|8|2|1996|387|5032|Saturday|1996Q2|N|Y|N|2450236|2450387|2449877|2450152|N|N|N|N|N| +2450244|AAAAAAAAEEDGFCAA|1996-06-09|1157|5032|387|1996|0|6|9|2|1996|387|5032|Sunday|1996Q2|N|N|N|2450236|2450387|2449878|2450153|N|N|N|N|N| +2450245|AAAAAAAAFEDGFCAA|1996-06-10|1157|5032|387|1996|1|6|10|2|1996|387|5032|Monday|1996Q2|N|N|N|2450236|2450387|2449879|2450154|N|N|N|N|N| +2450246|AAAAAAAAGEDGFCAA|1996-06-11|1157|5033|387|1996|2|6|11|2|1996|387|5033|Tuesday|1996Q2|N|N|N|2450236|2450387|2449880|2450155|N|N|N|N|N| +2450247|AAAAAAAAHEDGFCAA|1996-06-12|1157|5033|387|1996|3|6|12|2|1996|387|5033|Wednesday|1996Q2|N|N|N|2450236|2450387|2449881|2450156|N|N|N|N|N| +2450248|AAAAAAAAIEDGFCAA|1996-06-13|1157|5033|387|1996|4|6|13|2|1996|387|5033|Thursday|1996Q2|N|N|N|2450236|2450387|2449882|2450157|N|N|N|N|N| +2450249|AAAAAAAAJEDGFCAA|1996-06-14|1157|5033|387|1996|5|6|14|2|1996|387|5033|Friday|1996Q2|N|Y|N|2450236|2450387|2449883|2450158|N|N|N|N|N| +2450250|AAAAAAAAKEDGFCAA|1996-06-15|1157|5033|387|1996|6|6|15|2|1996|387|5033|Saturday|1996Q2|N|Y|N|2450236|2450387|2449884|2450159|N|N|N|N|N| +2450251|AAAAAAAALEDGFCAA|1996-06-16|1157|5033|387|1996|0|6|16|2|1996|387|5033|Sunday|1996Q2|N|N|N|2450236|2450387|2449885|2450160|N|N|N|N|N| +2450252|AAAAAAAAMEDGFCAA|1996-06-17|1157|5033|387|1996|1|6|17|2|1996|387|5033|Monday|1996Q2|N|N|N|2450236|2450387|2449886|2450161|N|N|N|N|N| +2450253|AAAAAAAANEDGFCAA|1996-06-18|1157|5034|387|1996|2|6|18|2|1996|387|5034|Tuesday|1996Q2|N|N|N|2450236|2450387|2449887|2450162|N|N|N|N|N| +2450254|AAAAAAAAOEDGFCAA|1996-06-19|1157|5034|387|1996|3|6|19|2|1996|387|5034|Wednesday|1996Q2|N|N|N|2450236|2450387|2449888|2450163|N|N|N|N|N| +2450255|AAAAAAAAPEDGFCAA|1996-06-20|1157|5034|387|1996|4|6|20|2|1996|387|5034|Thursday|1996Q2|N|N|N|2450236|2450387|2449889|2450164|N|N|N|N|N| +2450256|AAAAAAAAAFDGFCAA|1996-06-21|1157|5034|387|1996|5|6|21|2|1996|387|5034|Friday|1996Q2|N|Y|N|2450236|2450387|2449890|2450165|N|N|N|N|N| +2450257|AAAAAAAABFDGFCAA|1996-06-22|1157|5034|387|1996|6|6|22|2|1996|387|5034|Saturday|1996Q2|N|Y|N|2450236|2450387|2449891|2450166|N|N|N|N|N| +2450258|AAAAAAAACFDGFCAA|1996-06-23|1157|5034|387|1996|0|6|23|2|1996|387|5034|Sunday|1996Q2|N|N|N|2450236|2450387|2449892|2450167|N|N|N|N|N| +2450259|AAAAAAAADFDGFCAA|1996-06-24|1157|5034|387|1996|1|6|24|2|1996|387|5034|Monday|1996Q2|N|N|N|2450236|2450387|2449893|2450168|N|N|N|N|N| +2450260|AAAAAAAAEFDGFCAA|1996-06-25|1157|5035|387|1996|2|6|25|2|1996|387|5035|Tuesday|1996Q2|N|N|N|2450236|2450387|2449894|2450169|N|N|N|N|N| +2450261|AAAAAAAAFFDGFCAA|1996-06-26|1157|5035|387|1996|3|6|26|2|1996|387|5035|Wednesday|1996Q2|N|N|N|2450236|2450387|2449895|2450170|N|N|N|N|N| +2450262|AAAAAAAAGFDGFCAA|1996-06-27|1157|5035|387|1996|4|6|27|2|1996|387|5035|Thursday|1996Q2|N|N|N|2450236|2450387|2449896|2450171|N|N|N|N|N| +2450263|AAAAAAAAHFDGFCAA|1996-06-28|1157|5035|387|1996|5|6|28|2|1996|387|5035|Friday|1996Q2|N|Y|N|2450236|2450387|2449897|2450172|N|N|N|N|N| +2450264|AAAAAAAAIFDGFCAA|1996-06-29|1157|5035|387|1996|6|6|29|2|1996|387|5035|Saturday|1996Q2|N|Y|N|2450236|2450387|2449898|2450173|N|N|N|N|N| +2450265|AAAAAAAAJFDGFCAA|1996-06-30|1157|5035|387|1996|0|6|30|2|1996|387|5035|Sunday|1996Q2|N|N|N|2450236|2450387|2449899|2450174|N|N|N|N|N| +2450266|AAAAAAAAKFDGFCAA|1996-07-01|1158|5035|387|1996|1|7|1|3|1996|387|5035|Monday|1996Q3|N|N|N|2450266|2450447|2449900|2450175|N|N|N|N|N| +2450267|AAAAAAAALFDGFCAA|1996-07-02|1158|5036|387|1996|2|7|2|3|1996|387|5036|Tuesday|1996Q3|N|N|N|2450266|2450447|2449901|2450176|N|N|N|N|N| +2450268|AAAAAAAAMFDGFCAA|1996-07-03|1158|5036|387|1996|3|7|3|3|1996|387|5036|Wednesday|1996Q3|N|N|N|2450266|2450447|2449902|2450177|N|N|N|N|N| +2450269|AAAAAAAANFDGFCAA|1996-07-04|1158|5036|387|1996|4|7|4|3|1996|387|5036|Thursday|1996Q3|Y|N|N|2450266|2450447|2449903|2450178|N|N|N|N|N| +2450270|AAAAAAAAOFDGFCAA|1996-07-05|1158|5036|387|1996|5|7|5|3|1996|387|5036|Friday|1996Q3|N|Y|Y|2450266|2450447|2449904|2450179|N|N|N|N|N| +2450271|AAAAAAAAPFDGFCAA|1996-07-06|1158|5036|387|1996|6|7|6|3|1996|387|5036|Saturday|1996Q3|N|Y|N|2450266|2450447|2449905|2450180|N|N|N|N|N| +2450272|AAAAAAAAAGDGFCAA|1996-07-07|1158|5036|387|1996|0|7|7|3|1996|387|5036|Sunday|1996Q3|N|N|N|2450266|2450447|2449906|2450181|N|N|N|N|N| +2450273|AAAAAAAABGDGFCAA|1996-07-08|1158|5036|387|1996|1|7|8|3|1996|387|5036|Monday|1996Q3|N|N|N|2450266|2450447|2449907|2450182|N|N|N|N|N| +2450274|AAAAAAAACGDGFCAA|1996-07-09|1158|5037|387|1996|2|7|9|3|1996|387|5037|Tuesday|1996Q3|N|N|N|2450266|2450447|2449908|2450183|N|N|N|N|N| +2450275|AAAAAAAADGDGFCAA|1996-07-10|1158|5037|387|1996|3|7|10|3|1996|387|5037|Wednesday|1996Q3|N|N|N|2450266|2450447|2449909|2450184|N|N|N|N|N| +2450276|AAAAAAAAEGDGFCAA|1996-07-11|1158|5037|387|1996|4|7|11|3|1996|387|5037|Thursday|1996Q3|N|N|N|2450266|2450447|2449910|2450185|N|N|N|N|N| +2450277|AAAAAAAAFGDGFCAA|1996-07-12|1158|5037|387|1996|5|7|12|3|1996|387|5037|Friday|1996Q3|N|Y|N|2450266|2450447|2449911|2450186|N|N|N|N|N| +2450278|AAAAAAAAGGDGFCAA|1996-07-13|1158|5037|387|1996|6|7|13|3|1996|387|5037|Saturday|1996Q3|N|Y|N|2450266|2450447|2449912|2450187|N|N|N|N|N| +2450279|AAAAAAAAHGDGFCAA|1996-07-14|1158|5037|387|1996|0|7|14|3|1996|387|5037|Sunday|1996Q3|N|N|N|2450266|2450447|2449913|2450188|N|N|N|N|N| +2450280|AAAAAAAAIGDGFCAA|1996-07-15|1158|5037|387|1996|1|7|15|3|1996|387|5037|Monday|1996Q3|N|N|N|2450266|2450447|2449914|2450189|N|N|N|N|N| +2450281|AAAAAAAAJGDGFCAA|1996-07-16|1158|5038|387|1996|2|7|16|3|1996|387|5038|Tuesday|1996Q3|N|N|N|2450266|2450447|2449915|2450190|N|N|N|N|N| +2450282|AAAAAAAAKGDGFCAA|1996-07-17|1158|5038|387|1996|3|7|17|3|1996|387|5038|Wednesday|1996Q3|N|N|N|2450266|2450447|2449916|2450191|N|N|N|N|N| +2450283|AAAAAAAALGDGFCAA|1996-07-18|1158|5038|387|1996|4|7|18|3|1996|387|5038|Thursday|1996Q3|N|N|N|2450266|2450447|2449917|2450192|N|N|N|N|N| +2450284|AAAAAAAAMGDGFCAA|1996-07-19|1158|5038|387|1996|5|7|19|3|1996|387|5038|Friday|1996Q3|N|Y|N|2450266|2450447|2449918|2450193|N|N|N|N|N| +2450285|AAAAAAAANGDGFCAA|1996-07-20|1158|5038|387|1996|6|7|20|3|1996|387|5038|Saturday|1996Q3|N|Y|N|2450266|2450447|2449919|2450194|N|N|N|N|N| +2450286|AAAAAAAAOGDGFCAA|1996-07-21|1158|5038|387|1996|0|7|21|3|1996|387|5038|Sunday|1996Q3|N|N|N|2450266|2450447|2449920|2450195|N|N|N|N|N| +2450287|AAAAAAAAPGDGFCAA|1996-07-22|1158|5038|387|1996|1|7|22|3|1996|387|5038|Monday|1996Q3|N|N|N|2450266|2450447|2449921|2450196|N|N|N|N|N| +2450288|AAAAAAAAAHDGFCAA|1996-07-23|1158|5039|387|1996|2|7|23|3|1996|387|5039|Tuesday|1996Q3|N|N|N|2450266|2450447|2449922|2450197|N|N|N|N|N| +2450289|AAAAAAAABHDGFCAA|1996-07-24|1158|5039|387|1996|3|7|24|3|1996|387|5039|Wednesday|1996Q3|N|N|N|2450266|2450447|2449923|2450198|N|N|N|N|N| +2450290|AAAAAAAACHDGFCAA|1996-07-25|1158|5039|387|1996|4|7|25|3|1996|387|5039|Thursday|1996Q3|N|N|N|2450266|2450447|2449924|2450199|N|N|N|N|N| +2450291|AAAAAAAADHDGFCAA|1996-07-26|1158|5039|387|1996|5|7|26|3|1996|387|5039|Friday|1996Q3|N|Y|N|2450266|2450447|2449925|2450200|N|N|N|N|N| +2450292|AAAAAAAAEHDGFCAA|1996-07-27|1158|5039|387|1996|6|7|27|3|1996|387|5039|Saturday|1996Q3|N|Y|N|2450266|2450447|2449926|2450201|N|N|N|N|N| +2450293|AAAAAAAAFHDGFCAA|1996-07-28|1158|5039|387|1996|0|7|28|3|1996|387|5039|Sunday|1996Q3|N|N|N|2450266|2450447|2449927|2450202|N|N|N|N|N| +2450294|AAAAAAAAGHDGFCAA|1996-07-29|1158|5039|387|1996|1|7|29|3|1996|387|5039|Monday|1996Q3|N|N|N|2450266|2450447|2449928|2450203|N|N|N|N|N| +2450295|AAAAAAAAHHDGFCAA|1996-07-30|1158|5040|387|1996|2|7|30|3|1996|387|5040|Tuesday|1996Q3|N|N|N|2450266|2450447|2449929|2450204|N|N|N|N|N| +2450296|AAAAAAAAIHDGFCAA|1996-07-31|1158|5040|387|1996|3|7|31|3|1996|387|5040|Wednesday|1996Q3|N|N|N|2450266|2450447|2449930|2450205|N|N|N|N|N| +2450297|AAAAAAAAJHDGFCAA|1996-08-01|1159|5040|387|1996|4|8|1|3|1996|387|5040|Thursday|1996Q3|N|N|N|2450297|2450509|2449931|2450206|N|N|N|N|N| +2450298|AAAAAAAAKHDGFCAA|1996-08-02|1159|5040|387|1996|5|8|2|3|1996|387|5040|Friday|1996Q3|N|Y|N|2450297|2450509|2449932|2450207|N|N|N|N|N| +2450299|AAAAAAAALHDGFCAA|1996-08-03|1159|5040|387|1996|6|8|3|3|1996|387|5040|Saturday|1996Q3|N|Y|N|2450297|2450509|2449933|2450208|N|N|N|N|N| +2450300|AAAAAAAAMHDGFCAA|1996-08-04|1159|5040|387|1996|0|8|4|3|1996|387|5040|Sunday|1996Q3|N|N|N|2450297|2450509|2449934|2450209|N|N|N|N|N| +2450301|AAAAAAAANHDGFCAA|1996-08-05|1159|5040|387|1996|1|8|5|3|1996|387|5040|Monday|1996Q3|N|N|N|2450297|2450509|2449935|2450210|N|N|N|N|N| +2450302|AAAAAAAAOHDGFCAA|1996-08-06|1159|5041|387|1996|2|8|6|3|1996|387|5041|Tuesday|1996Q3|N|N|N|2450297|2450509|2449936|2450211|N|N|N|N|N| +2450303|AAAAAAAAPHDGFCAA|1996-08-07|1159|5041|387|1996|3|8|7|3|1996|387|5041|Wednesday|1996Q3|N|N|N|2450297|2450509|2449937|2450212|N|N|N|N|N| +2450304|AAAAAAAAAIDGFCAA|1996-08-08|1159|5041|387|1996|4|8|8|3|1996|387|5041|Thursday|1996Q3|N|N|N|2450297|2450509|2449938|2450213|N|N|N|N|N| +2450305|AAAAAAAABIDGFCAA|1996-08-09|1159|5041|387|1996|5|8|9|3|1996|387|5041|Friday|1996Q3|N|Y|N|2450297|2450509|2449939|2450214|N|N|N|N|N| +2450306|AAAAAAAACIDGFCAA|1996-08-10|1159|5041|387|1996|6|8|10|3|1996|387|5041|Saturday|1996Q3|N|Y|N|2450297|2450509|2449940|2450215|N|N|N|N|N| +2450307|AAAAAAAADIDGFCAA|1996-08-11|1159|5041|387|1996|0|8|11|3|1996|387|5041|Sunday|1996Q3|N|N|N|2450297|2450509|2449941|2450216|N|N|N|N|N| +2450308|AAAAAAAAEIDGFCAA|1996-08-12|1159|5041|387|1996|1|8|12|3|1996|387|5041|Monday|1996Q3|N|N|N|2450297|2450509|2449942|2450217|N|N|N|N|N| +2450309|AAAAAAAAFIDGFCAA|1996-08-13|1159|5042|387|1996|2|8|13|3|1996|387|5042|Tuesday|1996Q3|N|N|N|2450297|2450509|2449943|2450218|N|N|N|N|N| +2450310|AAAAAAAAGIDGFCAA|1996-08-14|1159|5042|387|1996|3|8|14|3|1996|387|5042|Wednesday|1996Q3|N|N|N|2450297|2450509|2449944|2450219|N|N|N|N|N| +2450311|AAAAAAAAHIDGFCAA|1996-08-15|1159|5042|387|1996|4|8|15|3|1996|387|5042|Thursday|1996Q3|N|N|N|2450297|2450509|2449945|2450220|N|N|N|N|N| +2450312|AAAAAAAAIIDGFCAA|1996-08-16|1159|5042|387|1996|5|8|16|3|1996|387|5042|Friday|1996Q3|N|Y|N|2450297|2450509|2449946|2450221|N|N|N|N|N| +2450313|AAAAAAAAJIDGFCAA|1996-08-17|1159|5042|387|1996|6|8|17|3|1996|387|5042|Saturday|1996Q3|N|Y|N|2450297|2450509|2449947|2450222|N|N|N|N|N| +2450314|AAAAAAAAKIDGFCAA|1996-08-18|1159|5042|387|1996|0|8|18|3|1996|387|5042|Sunday|1996Q3|N|N|N|2450297|2450509|2449948|2450223|N|N|N|N|N| +2450315|AAAAAAAALIDGFCAA|1996-08-19|1159|5042|387|1996|1|8|19|3|1996|387|5042|Monday|1996Q3|N|N|N|2450297|2450509|2449949|2450224|N|N|N|N|N| +2450316|AAAAAAAAMIDGFCAA|1996-08-20|1159|5043|387|1996|2|8|20|3|1996|387|5043|Tuesday|1996Q3|N|N|N|2450297|2450509|2449950|2450225|N|N|N|N|N| +2450317|AAAAAAAANIDGFCAA|1996-08-21|1159|5043|387|1996|3|8|21|3|1996|387|5043|Wednesday|1996Q3|N|N|N|2450297|2450509|2449951|2450226|N|N|N|N|N| +2450318|AAAAAAAAOIDGFCAA|1996-08-22|1159|5043|387|1996|4|8|22|3|1996|387|5043|Thursday|1996Q3|N|N|N|2450297|2450509|2449952|2450227|N|N|N|N|N| +2450319|AAAAAAAAPIDGFCAA|1996-08-23|1159|5043|387|1996|5|8|23|3|1996|387|5043|Friday|1996Q3|N|Y|N|2450297|2450509|2449953|2450228|N|N|N|N|N| +2450320|AAAAAAAAAJDGFCAA|1996-08-24|1159|5043|387|1996|6|8|24|3|1996|387|5043|Saturday|1996Q3|N|Y|N|2450297|2450509|2449954|2450229|N|N|N|N|N| +2450321|AAAAAAAABJDGFCAA|1996-08-25|1159|5043|387|1996|0|8|25|3|1996|387|5043|Sunday|1996Q3|N|N|N|2450297|2450509|2449955|2450230|N|N|N|N|N| +2450322|AAAAAAAACJDGFCAA|1996-08-26|1159|5043|387|1996|1|8|26|3|1996|387|5043|Monday|1996Q3|N|N|N|2450297|2450509|2449956|2450231|N|N|N|N|N| +2450323|AAAAAAAADJDGFCAA|1996-08-27|1159|5044|387|1996|2|8|27|3|1996|387|5044|Tuesday|1996Q3|N|N|N|2450297|2450509|2449957|2450232|N|N|N|N|N| +2450324|AAAAAAAAEJDGFCAA|1996-08-28|1159|5044|387|1996|3|8|28|3|1996|387|5044|Wednesday|1996Q3|N|N|N|2450297|2450509|2449958|2450233|N|N|N|N|N| +2450325|AAAAAAAAFJDGFCAA|1996-08-29|1159|5044|387|1996|4|8|29|3|1996|387|5044|Thursday|1996Q3|N|N|N|2450297|2450509|2449959|2450234|N|N|N|N|N| +2450326|AAAAAAAAGJDGFCAA|1996-08-30|1159|5044|387|1996|5|8|30|3|1996|387|5044|Friday|1996Q3|N|Y|N|2450297|2450509|2449960|2450235|N|N|N|N|N| +2450327|AAAAAAAAHJDGFCAA|1996-08-31|1159|5044|387|1996|6|8|31|3|1996|387|5044|Saturday|1996Q3|N|Y|N|2450297|2450509|2449961|2450236|N|N|N|N|N| +2450328|AAAAAAAAIJDGFCAA|1996-09-01|1160|5044|388|1996|0|9|1|3|1996|388|5044|Sunday|1996Q3|N|N|N|2450328|2450571|2449962|2450237|N|N|N|N|N| +2450329|AAAAAAAAJJDGFCAA|1996-09-02|1160|5044|388|1996|1|9|2|3|1996|388|5044|Monday|1996Q3|N|N|N|2450328|2450571|2449963|2450238|N|N|N|N|N| +2450330|AAAAAAAAKJDGFCAA|1996-09-03|1160|5045|388|1996|2|9|3|3|1996|388|5045|Tuesday|1996Q3|N|N|N|2450328|2450571|2449964|2450239|N|N|N|N|N| +2450331|AAAAAAAALJDGFCAA|1996-09-04|1160|5045|388|1996|3|9|4|3|1996|388|5045|Wednesday|1996Q3|N|N|N|2450328|2450571|2449965|2450240|N|N|N|N|N| +2450332|AAAAAAAAMJDGFCAA|1996-09-05|1160|5045|388|1996|4|9|5|3|1996|388|5045|Thursday|1996Q3|N|N|N|2450328|2450571|2449966|2450241|N|N|N|N|N| +2450333|AAAAAAAANJDGFCAA|1996-09-06|1160|5045|388|1996|5|9|6|3|1996|388|5045|Friday|1996Q3|N|Y|N|2450328|2450571|2449967|2450242|N|N|N|N|N| +2450334|AAAAAAAAOJDGFCAA|1996-09-07|1160|5045|388|1996|6|9|7|3|1996|388|5045|Saturday|1996Q3|N|Y|N|2450328|2450571|2449968|2450243|N|N|N|N|N| +2450335|AAAAAAAAPJDGFCAA|1996-09-08|1160|5045|388|1996|0|9|8|3|1996|388|5045|Sunday|1996Q3|N|N|N|2450328|2450571|2449969|2450244|N|N|N|N|N| +2450336|AAAAAAAAAKDGFCAA|1996-09-09|1160|5045|388|1996|1|9|9|3|1996|388|5045|Monday|1996Q3|N|N|N|2450328|2450571|2449970|2450245|N|N|N|N|N| +2450337|AAAAAAAABKDGFCAA|1996-09-10|1160|5046|388|1996|2|9|10|3|1996|388|5046|Tuesday|1996Q3|N|N|N|2450328|2450571|2449971|2450246|N|N|N|N|N| +2450338|AAAAAAAACKDGFCAA|1996-09-11|1160|5046|388|1996|3|9|11|3|1996|388|5046|Wednesday|1996Q3|N|N|N|2450328|2450571|2449972|2450247|N|N|N|N|N| +2450339|AAAAAAAADKDGFCAA|1996-09-12|1160|5046|388|1996|4|9|12|3|1996|388|5046|Thursday|1996Q3|N|N|N|2450328|2450571|2449973|2450248|N|N|N|N|N| +2450340|AAAAAAAAEKDGFCAA|1996-09-13|1160|5046|388|1996|5|9|13|3|1996|388|5046|Friday|1996Q3|N|Y|N|2450328|2450571|2449974|2450249|N|N|N|N|N| +2450341|AAAAAAAAFKDGFCAA|1996-09-14|1160|5046|388|1996|6|9|14|3|1996|388|5046|Saturday|1996Q3|N|Y|N|2450328|2450571|2449975|2450250|N|N|N|N|N| +2450342|AAAAAAAAGKDGFCAA|1996-09-15|1160|5046|388|1996|0|9|15|3|1996|388|5046|Sunday|1996Q3|N|N|N|2450328|2450571|2449976|2450251|N|N|N|N|N| +2450343|AAAAAAAAHKDGFCAA|1996-09-16|1160|5046|388|1996|1|9|16|3|1996|388|5046|Monday|1996Q3|N|N|N|2450328|2450571|2449977|2450252|N|N|N|N|N| +2450344|AAAAAAAAIKDGFCAA|1996-09-17|1160|5047|388|1996|2|9|17|3|1996|388|5047|Tuesday|1996Q3|N|N|N|2450328|2450571|2449978|2450253|N|N|N|N|N| +2450345|AAAAAAAAJKDGFCAA|1996-09-18|1160|5047|388|1996|3|9|18|3|1996|388|5047|Wednesday|1996Q3|N|N|N|2450328|2450571|2449979|2450254|N|N|N|N|N| +2450346|AAAAAAAAKKDGFCAA|1996-09-19|1160|5047|388|1996|4|9|19|3|1996|388|5047|Thursday|1996Q3|N|N|N|2450328|2450571|2449980|2450255|N|N|N|N|N| +2450347|AAAAAAAALKDGFCAA|1996-09-20|1160|5047|388|1996|5|9|20|3|1996|388|5047|Friday|1996Q3|N|Y|N|2450328|2450571|2449981|2450256|N|N|N|N|N| +2450348|AAAAAAAAMKDGFCAA|1996-09-21|1160|5047|388|1996|6|9|21|3|1996|388|5047|Saturday|1996Q3|N|Y|N|2450328|2450571|2449982|2450257|N|N|N|N|N| +2450349|AAAAAAAANKDGFCAA|1996-09-22|1160|5047|388|1996|0|9|22|3|1996|388|5047|Sunday|1996Q3|N|N|N|2450328|2450571|2449983|2450258|N|N|N|N|N| +2450350|AAAAAAAAOKDGFCAA|1996-09-23|1160|5047|388|1996|1|9|23|3|1996|388|5047|Monday|1996Q3|N|N|N|2450328|2450571|2449984|2450259|N|N|N|N|N| +2450351|AAAAAAAAPKDGFCAA|1996-09-24|1160|5048|388|1996|2|9|24|3|1996|388|5048|Tuesday|1996Q3|N|N|N|2450328|2450571|2449985|2450260|N|N|N|N|N| +2450352|AAAAAAAAALDGFCAA|1996-09-25|1160|5048|388|1996|3|9|25|3|1996|388|5048|Wednesday|1996Q3|N|N|N|2450328|2450571|2449986|2450261|N|N|N|N|N| +2450353|AAAAAAAABLDGFCAA|1996-09-26|1160|5048|388|1996|4|9|26|3|1996|388|5048|Thursday|1996Q3|N|N|N|2450328|2450571|2449987|2450262|N|N|N|N|N| +2450354|AAAAAAAACLDGFCAA|1996-09-27|1160|5048|388|1996|5|9|27|3|1996|388|5048|Friday|1996Q3|N|Y|N|2450328|2450571|2449988|2450263|N|N|N|N|N| +2450355|AAAAAAAADLDGFCAA|1996-09-28|1160|5048|388|1996|6|9|28|3|1996|388|5048|Saturday|1996Q3|N|Y|N|2450328|2450571|2449989|2450264|N|N|N|N|N| +2450356|AAAAAAAAELDGFCAA|1996-09-29|1160|5048|388|1996|0|9|29|3|1996|388|5048|Sunday|1996Q3|N|N|N|2450328|2450571|2449990|2450265|N|N|N|N|N| +2450357|AAAAAAAAFLDGFCAA|1996-09-30|1160|5048|388|1996|1|9|30|3|1996|388|5048|Monday|1996Q3|N|N|N|2450328|2450571|2449991|2450266|N|N|N|N|N| +2450358|AAAAAAAAGLDGFCAA|1996-10-01|1161|5049|388|1996|2|10|1|4|1996|388|5049|Tuesday|1996Q4|N|N|N|2450358|2450631|2449992|2450266|N|N|N|N|N| +2450359|AAAAAAAAHLDGFCAA|1996-10-02|1161|5049|388|1996|3|10|2|4|1996|388|5049|Wednesday|1996Q4|N|N|N|2450358|2450631|2449993|2450267|N|N|N|N|N| +2450360|AAAAAAAAILDGFCAA|1996-10-03|1161|5049|388|1996|4|10|3|4|1996|388|5049|Thursday|1996Q4|N|N|N|2450358|2450631|2449994|2450268|N|N|N|N|N| +2450361|AAAAAAAAJLDGFCAA|1996-10-04|1161|5049|388|1996|5|10|4|4|1996|388|5049|Friday|1996Q4|N|Y|N|2450358|2450631|2449995|2450269|N|N|N|N|N| +2450362|AAAAAAAAKLDGFCAA|1996-10-05|1161|5049|388|1996|6|10|5|4|1996|388|5049|Saturday|1996Q4|N|Y|N|2450358|2450631|2449996|2450270|N|N|N|N|N| +2450363|AAAAAAAALLDGFCAA|1996-10-06|1161|5049|388|1996|0|10|6|4|1996|388|5049|Sunday|1996Q4|N|N|N|2450358|2450631|2449997|2450271|N|N|N|N|N| +2450364|AAAAAAAAMLDGFCAA|1996-10-07|1161|5049|388|1996|1|10|7|4|1996|388|5049|Monday|1996Q4|N|N|N|2450358|2450631|2449998|2450272|N|N|N|N|N| +2450365|AAAAAAAANLDGFCAA|1996-10-08|1161|5050|388|1996|2|10|8|4|1996|388|5050|Tuesday|1996Q4|N|N|N|2450358|2450631|2449999|2450273|N|N|N|N|N| +2450366|AAAAAAAAOLDGFCAA|1996-10-09|1161|5050|388|1996|3|10|9|4|1996|388|5050|Wednesday|1996Q4|N|N|N|2450358|2450631|2450000|2450274|N|N|N|N|N| +2450367|AAAAAAAAPLDGFCAA|1996-10-10|1161|5050|388|1996|4|10|10|4|1996|388|5050|Thursday|1996Q4|N|N|N|2450358|2450631|2450001|2450275|N|N|N|N|N| +2450368|AAAAAAAAAMDGFCAA|1996-10-11|1161|5050|388|1996|5|10|11|4|1996|388|5050|Friday|1996Q4|N|Y|N|2450358|2450631|2450002|2450276|N|N|N|N|N| +2450369|AAAAAAAABMDGFCAA|1996-10-12|1161|5050|388|1996|6|10|12|4|1996|388|5050|Saturday|1996Q4|N|Y|N|2450358|2450631|2450003|2450277|N|N|N|N|N| +2450370|AAAAAAAACMDGFCAA|1996-10-13|1161|5050|388|1996|0|10|13|4|1996|388|5050|Sunday|1996Q4|N|N|N|2450358|2450631|2450004|2450278|N|N|N|N|N| +2450371|AAAAAAAADMDGFCAA|1996-10-14|1161|5050|388|1996|1|10|14|4|1996|388|5050|Monday|1996Q4|N|N|N|2450358|2450631|2450005|2450279|N|N|N|N|N| +2450372|AAAAAAAAEMDGFCAA|1996-10-15|1161|5051|388|1996|2|10|15|4|1996|388|5051|Tuesday|1996Q4|N|N|N|2450358|2450631|2450006|2450280|N|N|N|N|N| +2450373|AAAAAAAAFMDGFCAA|1996-10-16|1161|5051|388|1996|3|10|16|4|1996|388|5051|Wednesday|1996Q4|N|N|N|2450358|2450631|2450007|2450281|N|N|N|N|N| +2450374|AAAAAAAAGMDGFCAA|1996-10-17|1161|5051|388|1996|4|10|17|4|1996|388|5051|Thursday|1996Q4|N|N|N|2450358|2450631|2450008|2450282|N|N|N|N|N| +2450375|AAAAAAAAHMDGFCAA|1996-10-18|1161|5051|388|1996|5|10|18|4|1996|388|5051|Friday|1996Q4|N|Y|N|2450358|2450631|2450009|2450283|N|N|N|N|N| +2450376|AAAAAAAAIMDGFCAA|1996-10-19|1161|5051|388|1996|6|10|19|4|1996|388|5051|Saturday|1996Q4|N|Y|N|2450358|2450631|2450010|2450284|N|N|N|N|N| +2450377|AAAAAAAAJMDGFCAA|1996-10-20|1161|5051|388|1996|0|10|20|4|1996|388|5051|Sunday|1996Q4|N|N|N|2450358|2450631|2450011|2450285|N|N|N|N|N| +2450378|AAAAAAAAKMDGFCAA|1996-10-21|1161|5051|388|1996|1|10|21|4|1996|388|5051|Monday|1996Q4|N|N|N|2450358|2450631|2450012|2450286|N|N|N|N|N| +2450379|AAAAAAAALMDGFCAA|1996-10-22|1161|5052|388|1996|2|10|22|4|1996|388|5052|Tuesday|1996Q4|N|N|N|2450358|2450631|2450013|2450287|N|N|N|N|N| +2450380|AAAAAAAAMMDGFCAA|1996-10-23|1161|5052|388|1996|3|10|23|4|1996|388|5052|Wednesday|1996Q4|N|N|N|2450358|2450631|2450014|2450288|N|N|N|N|N| +2450381|AAAAAAAANMDGFCAA|1996-10-24|1161|5052|388|1996|4|10|24|4|1996|388|5052|Thursday|1996Q4|N|N|N|2450358|2450631|2450015|2450289|N|N|N|N|N| +2450382|AAAAAAAAOMDGFCAA|1996-10-25|1161|5052|388|1996|5|10|25|4|1996|388|5052|Friday|1996Q4|N|Y|N|2450358|2450631|2450016|2450290|N|N|N|N|N| +2450383|AAAAAAAAPMDGFCAA|1996-10-26|1161|5052|388|1996|6|10|26|4|1996|388|5052|Saturday|1996Q4|N|Y|N|2450358|2450631|2450017|2450291|N|N|N|N|N| +2450384|AAAAAAAAANDGFCAA|1996-10-27|1161|5052|388|1996|0|10|27|4|1996|388|5052|Sunday|1996Q4|N|N|N|2450358|2450631|2450018|2450292|N|N|N|N|N| +2450385|AAAAAAAABNDGFCAA|1996-10-28|1161|5052|388|1996|1|10|28|4|1996|388|5052|Monday|1996Q4|N|N|N|2450358|2450631|2450019|2450293|N|N|N|N|N| +2450386|AAAAAAAACNDGFCAA|1996-10-29|1161|5053|388|1996|2|10|29|4|1996|388|5053|Tuesday|1996Q4|N|N|N|2450358|2450631|2450020|2450294|N|N|N|N|N| +2450387|AAAAAAAADNDGFCAA|1996-10-30|1161|5053|388|1996|3|10|30|4|1996|388|5053|Wednesday|1996Q4|N|N|N|2450358|2450631|2450021|2450295|N|N|N|N|N| +2450388|AAAAAAAAENDGFCAA|1996-10-31|1161|5053|388|1996|4|10|31|4|1996|388|5053|Thursday|1996Q4|N|N|N|2450358|2450631|2450022|2450296|N|N|N|N|N| +2450389|AAAAAAAAFNDGFCAA|1996-11-01|1162|5053|388|1996|5|11|1|4|1996|388|5053|Friday|1996Q4|N|Y|N|2450389|2450693|2450023|2450297|N|N|N|N|N| +2450390|AAAAAAAAGNDGFCAA|1996-11-02|1162|5053|388|1996|6|11|2|4|1996|388|5053|Saturday|1996Q4|N|Y|N|2450389|2450693|2450024|2450298|N|N|N|N|N| +2450391|AAAAAAAAHNDGFCAA|1996-11-03|1162|5053|388|1996|0|11|3|4|1996|388|5053|Sunday|1996Q4|N|N|N|2450389|2450693|2450025|2450299|N|N|N|N|N| +2450392|AAAAAAAAINDGFCAA|1996-11-04|1162|5053|388|1996|1|11|4|4|1996|388|5053|Monday|1996Q4|N|N|N|2450389|2450693|2450026|2450300|N|N|N|N|N| +2450393|AAAAAAAAJNDGFCAA|1996-11-05|1162|5054|388|1996|2|11|5|4|1996|388|5054|Tuesday|1996Q4|N|N|N|2450389|2450693|2450027|2450301|N|N|N|N|N| +2450394|AAAAAAAAKNDGFCAA|1996-11-06|1162|5054|388|1996|3|11|6|4|1996|388|5054|Wednesday|1996Q4|N|N|N|2450389|2450693|2450028|2450302|N|N|N|N|N| +2450395|AAAAAAAALNDGFCAA|1996-11-07|1162|5054|388|1996|4|11|7|4|1996|388|5054|Thursday|1996Q4|N|N|N|2450389|2450693|2450029|2450303|N|N|N|N|N| +2450396|AAAAAAAAMNDGFCAA|1996-11-08|1162|5054|388|1996|5|11|8|4|1996|388|5054|Friday|1996Q4|N|Y|N|2450389|2450693|2450030|2450304|N|N|N|N|N| +2450397|AAAAAAAANNDGFCAA|1996-11-09|1162|5054|388|1996|6|11|9|4|1996|388|5054|Saturday|1996Q4|N|Y|N|2450389|2450693|2450031|2450305|N|N|N|N|N| +2450398|AAAAAAAAONDGFCAA|1996-11-10|1162|5054|388|1996|0|11|10|4|1996|388|5054|Sunday|1996Q4|N|N|N|2450389|2450693|2450032|2450306|N|N|N|N|N| +2450399|AAAAAAAAPNDGFCAA|1996-11-11|1162|5054|388|1996|1|11|11|4|1996|388|5054|Monday|1996Q4|N|N|N|2450389|2450693|2450033|2450307|N|N|N|N|N| +2450400|AAAAAAAAAODGFCAA|1996-11-12|1162|5055|388|1996|2|11|12|4|1996|388|5055|Tuesday|1996Q4|N|N|N|2450389|2450693|2450034|2450308|N|N|N|N|N| +2450401|AAAAAAAABODGFCAA|1996-11-13|1162|5055|388|1996|3|11|13|4|1996|388|5055|Wednesday|1996Q4|N|N|N|2450389|2450693|2450035|2450309|N|N|N|N|N| +2450402|AAAAAAAACODGFCAA|1996-11-14|1162|5055|388|1996|4|11|14|4|1996|388|5055|Thursday|1996Q4|N|N|N|2450389|2450693|2450036|2450310|N|N|N|N|N| +2450403|AAAAAAAADODGFCAA|1996-11-15|1162|5055|388|1996|5|11|15|4|1996|388|5055|Friday|1996Q4|N|Y|N|2450389|2450693|2450037|2450311|N|N|N|N|N| +2450404|AAAAAAAAEODGFCAA|1996-11-16|1162|5055|388|1996|6|11|16|4|1996|388|5055|Saturday|1996Q4|N|Y|N|2450389|2450693|2450038|2450312|N|N|N|N|N| +2450405|AAAAAAAAFODGFCAA|1996-11-17|1162|5055|388|1996|0|11|17|4|1996|388|5055|Sunday|1996Q4|N|N|N|2450389|2450693|2450039|2450313|N|N|N|N|N| +2450406|AAAAAAAAGODGFCAA|1996-11-18|1162|5055|388|1996|1|11|18|4|1996|388|5055|Monday|1996Q4|N|N|N|2450389|2450693|2450040|2450314|N|N|N|N|N| +2450407|AAAAAAAAHODGFCAA|1996-11-19|1162|5056|388|1996|2|11|19|4|1996|388|5056|Tuesday|1996Q4|N|N|N|2450389|2450693|2450041|2450315|N|N|N|N|N| +2450408|AAAAAAAAIODGFCAA|1996-11-20|1162|5056|388|1996|3|11|20|4|1996|388|5056|Wednesday|1996Q4|N|N|N|2450389|2450693|2450042|2450316|N|N|N|N|N| +2450409|AAAAAAAAJODGFCAA|1996-11-21|1162|5056|388|1996|4|11|21|4|1996|388|5056|Thursday|1996Q4|N|N|N|2450389|2450693|2450043|2450317|N|N|N|N|N| +2450410|AAAAAAAAKODGFCAA|1996-11-22|1162|5056|388|1996|5|11|22|4|1996|388|5056|Friday|1996Q4|N|Y|N|2450389|2450693|2450044|2450318|N|N|N|N|N| +2450411|AAAAAAAALODGFCAA|1996-11-23|1162|5056|388|1996|6|11|23|4|1996|388|5056|Saturday|1996Q4|N|Y|N|2450389|2450693|2450045|2450319|N|N|N|N|N| +2450412|AAAAAAAAMODGFCAA|1996-11-24|1162|5056|388|1996|0|11|24|4|1996|388|5056|Sunday|1996Q4|N|N|N|2450389|2450693|2450046|2450320|N|N|N|N|N| +2450413|AAAAAAAANODGFCAA|1996-11-25|1162|5056|388|1996|1|11|25|4|1996|388|5056|Monday|1996Q4|N|N|N|2450389|2450693|2450047|2450321|N|N|N|N|N| +2450414|AAAAAAAAOODGFCAA|1996-11-26|1162|5057|388|1996|2|11|26|4|1996|388|5057|Tuesday|1996Q4|N|N|N|2450389|2450693|2450048|2450322|N|N|N|N|N| +2450415|AAAAAAAAPODGFCAA|1996-11-27|1162|5057|388|1996|3|11|27|4|1996|388|5057|Wednesday|1996Q4|N|N|N|2450389|2450693|2450049|2450323|N|N|N|N|N| +2450416|AAAAAAAAAPDGFCAA|1996-11-28|1162|5057|388|1996|4|11|28|4|1996|388|5057|Thursday|1996Q4|N|N|N|2450389|2450693|2450050|2450324|N|N|N|N|N| +2450417|AAAAAAAABPDGFCAA|1996-11-29|1162|5057|388|1996|5|11|29|4|1996|388|5057|Friday|1996Q4|N|Y|N|2450389|2450693|2450051|2450325|N|N|N|N|N| +2450418|AAAAAAAACPDGFCAA|1996-11-30|1162|5057|388|1996|6|11|30|4|1996|388|5057|Saturday|1996Q4|N|Y|N|2450389|2450693|2450052|2450326|N|N|N|N|N| +2450419|AAAAAAAADPDGFCAA|1996-12-01|1163|5057|389|1996|0|12|1|4|1996|389|5057|Sunday|1996Q4|N|N|N|2450419|2450753|2450053|2450327|N|N|N|N|N| +2450420|AAAAAAAAEPDGFCAA|1996-12-02|1163|5057|389|1996|1|12|2|4|1996|389|5057|Monday|1996Q4|N|N|N|2450419|2450753|2450054|2450328|N|N|N|N|N| +2450421|AAAAAAAAFPDGFCAA|1996-12-03|1163|5058|389|1996|2|12|3|4|1996|389|5058|Tuesday|1996Q4|N|N|N|2450419|2450753|2450055|2450329|N|N|N|N|N| +2450422|AAAAAAAAGPDGFCAA|1996-12-04|1163|5058|389|1996|3|12|4|4|1996|389|5058|Wednesday|1996Q4|N|N|N|2450419|2450753|2450056|2450330|N|N|N|N|N| +2450423|AAAAAAAAHPDGFCAA|1996-12-05|1163|5058|389|1996|4|12|5|4|1996|389|5058|Thursday|1996Q4|N|N|N|2450419|2450753|2450057|2450331|N|N|N|N|N| +2450424|AAAAAAAAIPDGFCAA|1996-12-06|1163|5058|389|1996|5|12|6|4|1996|389|5058|Friday|1996Q4|N|Y|N|2450419|2450753|2450058|2450332|N|N|N|N|N| +2450425|AAAAAAAAJPDGFCAA|1996-12-07|1163|5058|389|1996|6|12|7|4|1996|389|5058|Saturday|1996Q4|N|Y|N|2450419|2450753|2450059|2450333|N|N|N|N|N| +2450426|AAAAAAAAKPDGFCAA|1996-12-08|1163|5058|389|1996|0|12|8|4|1996|389|5058|Sunday|1996Q4|N|N|N|2450419|2450753|2450060|2450334|N|N|N|N|N| +2450427|AAAAAAAALPDGFCAA|1996-12-09|1163|5058|389|1996|1|12|9|4|1996|389|5058|Monday|1996Q4|N|N|N|2450419|2450753|2450061|2450335|N|N|N|N|N| +2450428|AAAAAAAAMPDGFCAA|1996-12-10|1163|5059|389|1996|2|12|10|4|1996|389|5059|Tuesday|1996Q4|N|N|N|2450419|2450753|2450062|2450336|N|N|N|N|N| +2450429|AAAAAAAANPDGFCAA|1996-12-11|1163|5059|389|1996|3|12|11|4|1996|389|5059|Wednesday|1996Q4|N|N|N|2450419|2450753|2450063|2450337|N|N|N|N|N| +2450430|AAAAAAAAOPDGFCAA|1996-12-12|1163|5059|389|1996|4|12|12|4|1996|389|5059|Thursday|1996Q4|N|N|N|2450419|2450753|2450064|2450338|N|N|N|N|N| +2450431|AAAAAAAAPPDGFCAA|1996-12-13|1163|5059|389|1996|5|12|13|4|1996|389|5059|Friday|1996Q4|N|Y|N|2450419|2450753|2450065|2450339|N|N|N|N|N| +2450432|AAAAAAAAAAEGFCAA|1996-12-14|1163|5059|389|1996|6|12|14|4|1996|389|5059|Saturday|1996Q4|N|Y|N|2450419|2450753|2450066|2450340|N|N|N|N|N| +2450433|AAAAAAAABAEGFCAA|1996-12-15|1163|5059|389|1996|0|12|15|4|1996|389|5059|Sunday|1996Q4|N|N|N|2450419|2450753|2450067|2450341|N|N|N|N|N| +2450434|AAAAAAAACAEGFCAA|1996-12-16|1163|5059|389|1996|1|12|16|4|1996|389|5059|Monday|1996Q4|N|N|N|2450419|2450753|2450068|2450342|N|N|N|N|N| +2450435|AAAAAAAADAEGFCAA|1996-12-17|1163|5060|389|1996|2|12|17|4|1996|389|5060|Tuesday|1996Q4|N|N|N|2450419|2450753|2450069|2450343|N|N|N|N|N| +2450436|AAAAAAAAEAEGFCAA|1996-12-18|1163|5060|389|1996|3|12|18|4|1996|389|5060|Wednesday|1996Q4|N|N|N|2450419|2450753|2450070|2450344|N|N|N|N|N| +2450437|AAAAAAAAFAEGFCAA|1996-12-19|1163|5060|389|1996|4|12|19|4|1996|389|5060|Thursday|1996Q4|N|N|N|2450419|2450753|2450071|2450345|N|N|N|N|N| +2450438|AAAAAAAAGAEGFCAA|1996-12-20|1163|5060|389|1996|5|12|20|4|1996|389|5060|Friday|1996Q4|N|Y|N|2450419|2450753|2450072|2450346|N|N|N|N|N| +2450439|AAAAAAAAHAEGFCAA|1996-12-21|1163|5060|389|1996|6|12|21|4|1996|389|5060|Saturday|1996Q4|N|Y|N|2450419|2450753|2450073|2450347|N|N|N|N|N| +2450440|AAAAAAAAIAEGFCAA|1996-12-22|1163|5060|389|1996|0|12|22|4|1996|389|5060|Sunday|1996Q4|N|N|N|2450419|2450753|2450074|2450348|N|N|N|N|N| +2450441|AAAAAAAAJAEGFCAA|1996-12-23|1163|5060|389|1996|1|12|23|4|1996|389|5060|Monday|1996Q4|N|N|N|2450419|2450753|2450075|2450349|N|N|N|N|N| +2450442|AAAAAAAAKAEGFCAA|1996-12-24|1163|5061|389|1996|2|12|24|4|1996|389|5061|Tuesday|1996Q4|N|N|N|2450419|2450753|2450076|2450350|N|N|N|N|N| +2450443|AAAAAAAALAEGFCAA|1996-12-25|1163|5061|389|1996|3|12|25|4|1996|389|5061|Wednesday|1996Q4|Y|N|N|2450419|2450753|2450077|2450351|N|N|N|N|N| +2450444|AAAAAAAAMAEGFCAA|1996-12-26|1163|5061|389|1996|4|12|26|4|1996|389|5061|Thursday|1996Q4|N|N|Y|2450419|2450753|2450078|2450352|N|N|N|N|N| +2450445|AAAAAAAANAEGFCAA|1996-12-27|1163|5061|389|1996|5|12|27|4|1996|389|5061|Friday|1996Q4|N|Y|N|2450419|2450753|2450079|2450353|N|N|N|N|N| +2450446|AAAAAAAAOAEGFCAA|1996-12-28|1163|5061|389|1996|6|12|28|4|1996|389|5061|Saturday|1996Q4|N|Y|N|2450419|2450753|2450080|2450354|N|N|N|N|N| +2450447|AAAAAAAAPAEGFCAA|1996-12-29|1163|5061|389|1996|0|12|29|4|1996|389|5061|Sunday|1996Q4|N|N|N|2450419|2450753|2450081|2450355|N|N|N|N|N| +2450448|AAAAAAAAABEGFCAA|1996-12-30|1163|5061|389|1996|1|12|30|4|1996|389|5061|Monday|1996Q4|N|N|N|2450419|2450753|2450082|2450356|N|N|N|N|N| +2450449|AAAAAAAABBEGFCAA|1996-12-31|1163|5062|389|1996|2|12|31|4|1996|389|5062|Tuesday|1996Q4|Y|N|N|2450419|2450753|2450083|2450357|N|N|N|N|N| +2450450|AAAAAAAACBEGFCAA|1997-01-01|1164|5062|389|1997|3|1|1|1|1997|389|5062|Wednesday|1997Q1|Y|N|Y|2450450|2450449|2450084|2450358|N|N|N|N|N| +2450451|AAAAAAAADBEGFCAA|1997-01-02|1164|5062|389|1997|4|1|2|1|1997|389|5062|Thursday|1997Q1|N|N|Y|2450450|2450449|2450085|2450359|N|N|N|N|N| +2450452|AAAAAAAAEBEGFCAA|1997-01-03|1164|5062|389|1997|5|1|3|1|1997|389|5062|Friday|1997Q1|N|Y|N|2450450|2450449|2450086|2450360|N|N|N|N|N| +2450453|AAAAAAAAFBEGFCAA|1997-01-04|1164|5062|389|1997|6|1|4|1|1997|389|5062|Saturday|1997Q1|N|Y|N|2450450|2450449|2450087|2450361|N|N|N|N|N| +2450454|AAAAAAAAGBEGFCAA|1997-01-05|1164|5062|389|1997|0|1|5|1|1997|389|5062|Sunday|1997Q1|N|N|N|2450450|2450449|2450088|2450362|N|N|N|N|N| +2450455|AAAAAAAAHBEGFCAA|1997-01-06|1164|5062|389|1997|1|1|6|1|1997|389|5062|Monday|1997Q1|N|N|N|2450450|2450449|2450089|2450363|N|N|N|N|N| +2450456|AAAAAAAAIBEGFCAA|1997-01-07|1164|5063|389|1997|2|1|7|1|1997|389|5063|Tuesday|1997Q1|N|N|N|2450450|2450449|2450090|2450364|N|N|N|N|N| +2450457|AAAAAAAAJBEGFCAA|1997-01-08|1164|5063|389|1997|3|1|8|1|1997|389|5063|Wednesday|1997Q1|N|N|N|2450450|2450449|2450091|2450365|N|N|N|N|N| +2450458|AAAAAAAAKBEGFCAA|1997-01-09|1164|5063|389|1997|4|1|9|1|1997|389|5063|Thursday|1997Q1|N|N|N|2450450|2450449|2450092|2450366|N|N|N|N|N| +2450459|AAAAAAAALBEGFCAA|1997-01-10|1164|5063|389|1997|5|1|10|1|1997|389|5063|Friday|1997Q1|N|Y|N|2450450|2450449|2450093|2450367|N|N|N|N|N| +2450460|AAAAAAAAMBEGFCAA|1997-01-11|1164|5063|389|1997|6|1|11|1|1997|389|5063|Saturday|1997Q1|N|Y|N|2450450|2450449|2450094|2450368|N|N|N|N|N| +2450461|AAAAAAAANBEGFCAA|1997-01-12|1164|5063|389|1997|0|1|12|1|1997|389|5063|Sunday|1997Q1|N|N|N|2450450|2450449|2450095|2450369|N|N|N|N|N| +2450462|AAAAAAAAOBEGFCAA|1997-01-13|1164|5063|389|1997|1|1|13|1|1997|389|5063|Monday|1997Q1|N|N|N|2450450|2450449|2450096|2450370|N|N|N|N|N| +2450463|AAAAAAAAPBEGFCAA|1997-01-14|1164|5064|389|1997|2|1|14|1|1997|389|5064|Tuesday|1997Q1|N|N|N|2450450|2450449|2450097|2450371|N|N|N|N|N| +2450464|AAAAAAAAACEGFCAA|1997-01-15|1164|5064|389|1997|3|1|15|1|1997|389|5064|Wednesday|1997Q1|N|N|N|2450450|2450449|2450098|2450372|N|N|N|N|N| +2450465|AAAAAAAABCEGFCAA|1997-01-16|1164|5064|389|1997|4|1|16|1|1997|389|5064|Thursday|1997Q1|N|N|N|2450450|2450449|2450099|2450373|N|N|N|N|N| +2450466|AAAAAAAACCEGFCAA|1997-01-17|1164|5064|389|1997|5|1|17|1|1997|389|5064|Friday|1997Q1|N|Y|N|2450450|2450449|2450100|2450374|N|N|N|N|N| +2450467|AAAAAAAADCEGFCAA|1997-01-18|1164|5064|389|1997|6|1|18|1|1997|389|5064|Saturday|1997Q1|N|Y|N|2450450|2450449|2450101|2450375|N|N|N|N|N| +2450468|AAAAAAAAECEGFCAA|1997-01-19|1164|5064|389|1997|0|1|19|1|1997|389|5064|Sunday|1997Q1|N|N|N|2450450|2450449|2450102|2450376|N|N|N|N|N| +2450469|AAAAAAAAFCEGFCAA|1997-01-20|1164|5064|389|1997|1|1|20|1|1997|389|5064|Monday|1997Q1|N|N|N|2450450|2450449|2450103|2450377|N|N|N|N|N| +2450470|AAAAAAAAGCEGFCAA|1997-01-21|1164|5065|389|1997|2|1|21|1|1997|389|5065|Tuesday|1997Q1|N|N|N|2450450|2450449|2450104|2450378|N|N|N|N|N| +2450471|AAAAAAAAHCEGFCAA|1997-01-22|1164|5065|389|1997|3|1|22|1|1997|389|5065|Wednesday|1997Q1|N|N|N|2450450|2450449|2450105|2450379|N|N|N|N|N| +2450472|AAAAAAAAICEGFCAA|1997-01-23|1164|5065|389|1997|4|1|23|1|1997|389|5065|Thursday|1997Q1|N|N|N|2450450|2450449|2450106|2450380|N|N|N|N|N| +2450473|AAAAAAAAJCEGFCAA|1997-01-24|1164|5065|389|1997|5|1|24|1|1997|389|5065|Friday|1997Q1|N|Y|N|2450450|2450449|2450107|2450381|N|N|N|N|N| +2450474|AAAAAAAAKCEGFCAA|1997-01-25|1164|5065|389|1997|6|1|25|1|1997|389|5065|Saturday|1997Q1|N|Y|N|2450450|2450449|2450108|2450382|N|N|N|N|N| +2450475|AAAAAAAALCEGFCAA|1997-01-26|1164|5065|389|1997|0|1|26|1|1997|389|5065|Sunday|1997Q1|N|N|N|2450450|2450449|2450109|2450383|N|N|N|N|N| +2450476|AAAAAAAAMCEGFCAA|1997-01-27|1164|5065|389|1997|1|1|27|1|1997|389|5065|Monday|1997Q1|N|N|N|2450450|2450449|2450110|2450384|N|N|N|N|N| +2450477|AAAAAAAANCEGFCAA|1997-01-28|1164|5066|389|1997|2|1|28|1|1997|389|5066|Tuesday|1997Q1|N|N|N|2450450|2450449|2450111|2450385|N|N|N|N|N| +2450478|AAAAAAAAOCEGFCAA|1997-01-29|1164|5066|389|1997|3|1|29|1|1997|389|5066|Wednesday|1997Q1|N|N|N|2450450|2450449|2450112|2450386|N|N|N|N|N| +2450479|AAAAAAAAPCEGFCAA|1997-01-30|1164|5066|389|1997|4|1|30|1|1997|389|5066|Thursday|1997Q1|N|N|N|2450450|2450449|2450113|2450387|N|N|N|N|N| +2450480|AAAAAAAAADEGFCAA|1997-01-31|1164|5066|389|1997|5|1|31|1|1997|389|5066|Friday|1997Q1|N|Y|N|2450450|2450449|2450114|2450388|N|N|N|N|N| +2450481|AAAAAAAABDEGFCAA|1997-02-01|1165|5066|389|1997|6|2|1|1|1997|389|5066|Saturday|1997Q1|N|Y|N|2450481|2450511|2450115|2450389|N|N|N|N|N| +2450482|AAAAAAAACDEGFCAA|1997-02-02|1165|5066|389|1997|0|2|2|1|1997|389|5066|Sunday|1997Q1|N|N|N|2450481|2450511|2450116|2450390|N|N|N|N|N| +2450483|AAAAAAAADDEGFCAA|1997-02-03|1165|5066|389|1997|1|2|3|1|1997|389|5066|Monday|1997Q1|N|N|N|2450481|2450511|2450117|2450391|N|N|N|N|N| +2450484|AAAAAAAAEDEGFCAA|1997-02-04|1165|5067|389|1997|2|2|4|1|1997|389|5067|Tuesday|1997Q1|N|N|N|2450481|2450511|2450118|2450392|N|N|N|N|N| +2450485|AAAAAAAAFDEGFCAA|1997-02-05|1165|5067|389|1997|3|2|5|1|1997|389|5067|Wednesday|1997Q1|N|N|N|2450481|2450511|2450119|2450393|N|N|N|N|N| +2450486|AAAAAAAAGDEGFCAA|1997-02-06|1165|5067|389|1997|4|2|6|1|1997|389|5067|Thursday|1997Q1|N|N|N|2450481|2450511|2450120|2450394|N|N|N|N|N| +2450487|AAAAAAAAHDEGFCAA|1997-02-07|1165|5067|389|1997|5|2|7|1|1997|389|5067|Friday|1997Q1|N|Y|N|2450481|2450511|2450121|2450395|N|N|N|N|N| +2450488|AAAAAAAAIDEGFCAA|1997-02-08|1165|5067|389|1997|6|2|8|1|1997|389|5067|Saturday|1997Q1|N|Y|N|2450481|2450511|2450122|2450396|N|N|N|N|N| +2450489|AAAAAAAAJDEGFCAA|1997-02-09|1165|5067|389|1997|0|2|9|1|1997|389|5067|Sunday|1997Q1|N|N|N|2450481|2450511|2450123|2450397|N|N|N|N|N| +2450490|AAAAAAAAKDEGFCAA|1997-02-10|1165|5067|389|1997|1|2|10|1|1997|389|5067|Monday|1997Q1|N|N|N|2450481|2450511|2450124|2450398|N|N|N|N|N| +2450491|AAAAAAAALDEGFCAA|1997-02-11|1165|5068|389|1997|2|2|11|1|1997|389|5068|Tuesday|1997Q1|N|N|N|2450481|2450511|2450125|2450399|N|N|N|N|N| +2450492|AAAAAAAAMDEGFCAA|1997-02-12|1165|5068|389|1997|3|2|12|1|1997|389|5068|Wednesday|1997Q1|N|N|N|2450481|2450511|2450126|2450400|N|N|N|N|N| +2450493|AAAAAAAANDEGFCAA|1997-02-13|1165|5068|389|1997|4|2|13|1|1997|389|5068|Thursday|1997Q1|N|N|N|2450481|2450511|2450127|2450401|N|N|N|N|N| +2450494|AAAAAAAAODEGFCAA|1997-02-14|1165|5068|389|1997|5|2|14|1|1997|389|5068|Friday|1997Q1|N|Y|N|2450481|2450511|2450128|2450402|N|N|N|N|N| +2450495|AAAAAAAAPDEGFCAA|1997-02-15|1165|5068|389|1997|6|2|15|1|1997|389|5068|Saturday|1997Q1|N|Y|N|2450481|2450511|2450129|2450403|N|N|N|N|N| +2450496|AAAAAAAAAEEGFCAA|1997-02-16|1165|5068|389|1997|0|2|16|1|1997|389|5068|Sunday|1997Q1|N|N|N|2450481|2450511|2450130|2450404|N|N|N|N|N| +2450497|AAAAAAAABEEGFCAA|1997-02-17|1165|5068|389|1997|1|2|17|1|1997|389|5068|Monday|1997Q1|N|N|N|2450481|2450511|2450131|2450405|N|N|N|N|N| +2450498|AAAAAAAACEEGFCAA|1997-02-18|1165|5069|389|1997|2|2|18|1|1997|389|5069|Tuesday|1997Q1|N|N|N|2450481|2450511|2450132|2450406|N|N|N|N|N| +2450499|AAAAAAAADEEGFCAA|1997-02-19|1165|5069|389|1997|3|2|19|1|1997|389|5069|Wednesday|1997Q1|N|N|N|2450481|2450511|2450133|2450407|N|N|N|N|N| +2450500|AAAAAAAAEEEGFCAA|1997-02-20|1165|5069|389|1997|4|2|20|1|1997|389|5069|Thursday|1997Q1|N|N|N|2450481|2450511|2450134|2450408|N|N|N|N|N| +2450501|AAAAAAAAFEEGFCAA|1997-02-21|1165|5069|389|1997|5|2|21|1|1997|389|5069|Friday|1997Q1|N|Y|N|2450481|2450511|2450135|2450409|N|N|N|N|N| +2450502|AAAAAAAAGEEGFCAA|1997-02-22|1165|5069|389|1997|6|2|22|1|1997|389|5069|Saturday|1997Q1|N|Y|N|2450481|2450511|2450136|2450410|N|N|N|N|N| +2450503|AAAAAAAAHEEGFCAA|1997-02-23|1165|5069|389|1997|0|2|23|1|1997|389|5069|Sunday|1997Q1|N|N|N|2450481|2450511|2450137|2450411|N|N|N|N|N| +2450504|AAAAAAAAIEEGFCAA|1997-02-24|1165|5069|389|1997|1|2|24|1|1997|389|5069|Monday|1997Q1|N|N|N|2450481|2450511|2450138|2450412|N|N|N|N|N| +2450505|AAAAAAAAJEEGFCAA|1997-02-25|1165|5070|389|1997|2|2|25|1|1997|389|5070|Tuesday|1997Q1|N|N|N|2450481|2450511|2450139|2450413|N|N|N|N|N| +2450506|AAAAAAAAKEEGFCAA|1997-02-26|1165|5070|389|1997|3|2|26|1|1997|389|5070|Wednesday|1997Q1|N|N|N|2450481|2450511|2450140|2450414|N|N|N|N|N| +2450507|AAAAAAAALEEGFCAA|1997-02-27|1165|5070|389|1997|4|2|27|1|1997|389|5070|Thursday|1997Q1|N|N|N|2450481|2450511|2450141|2450415|N|N|N|N|N| +2450508|AAAAAAAAMEEGFCAA|1997-02-28|1165|5070|389|1997|5|2|28|1|1997|389|5070|Friday|1997Q1|N|Y|N|2450481|2450511|2450142|2450416|N|N|N|N|N| +2450509|AAAAAAAANEEGFCAA|1997-03-01|1166|5070|390|1997|6|3|1|1|1997|390|5070|Saturday|1997Q1|N|Y|N|2450509|2450567|2450144|2450417|N|N|N|N|N| +2450510|AAAAAAAAOEEGFCAA|1997-03-02|1166|5070|390|1997|0|3|2|1|1997|390|5070|Sunday|1997Q1|N|N|N|2450509|2450567|2450145|2450418|N|N|N|N|N| +2450511|AAAAAAAAPEEGFCAA|1997-03-03|1166|5070|390|1997|1|3|3|1|1997|390|5070|Monday|1997Q1|N|N|N|2450509|2450567|2450146|2450419|N|N|N|N|N| +2450512|AAAAAAAAAFEGFCAA|1997-03-04|1166|5071|390|1997|2|3|4|1|1997|390|5071|Tuesday|1997Q1|N|N|N|2450509|2450567|2450147|2450420|N|N|N|N|N| +2450513|AAAAAAAABFEGFCAA|1997-03-05|1166|5071|390|1997|3|3|5|1|1997|390|5071|Wednesday|1997Q1|N|N|N|2450509|2450567|2450148|2450421|N|N|N|N|N| +2450514|AAAAAAAACFEGFCAA|1997-03-06|1166|5071|390|1997|4|3|6|1|1997|390|5071|Thursday|1997Q1|N|N|N|2450509|2450567|2450149|2450422|N|N|N|N|N| +2450515|AAAAAAAADFEGFCAA|1997-03-07|1166|5071|390|1997|5|3|7|1|1997|390|5071|Friday|1997Q1|N|Y|N|2450509|2450567|2450150|2450423|N|N|N|N|N| +2450516|AAAAAAAAEFEGFCAA|1997-03-08|1166|5071|390|1997|6|3|8|1|1997|390|5071|Saturday|1997Q1|N|Y|N|2450509|2450567|2450151|2450424|N|N|N|N|N| +2450517|AAAAAAAAFFEGFCAA|1997-03-09|1166|5071|390|1997|0|3|9|1|1997|390|5071|Sunday|1997Q1|N|N|N|2450509|2450567|2450152|2450425|N|N|N|N|N| +2450518|AAAAAAAAGFEGFCAA|1997-03-10|1166|5071|390|1997|1|3|10|1|1997|390|5071|Monday|1997Q1|N|N|N|2450509|2450567|2450153|2450426|N|N|N|N|N| +2450519|AAAAAAAAHFEGFCAA|1997-03-11|1166|5072|390|1997|2|3|11|1|1997|390|5072|Tuesday|1997Q1|N|N|N|2450509|2450567|2450154|2450427|N|N|N|N|N| +2450520|AAAAAAAAIFEGFCAA|1997-03-12|1166|5072|390|1997|3|3|12|1|1997|390|5072|Wednesday|1997Q1|N|N|N|2450509|2450567|2450155|2450428|N|N|N|N|N| +2450521|AAAAAAAAJFEGFCAA|1997-03-13|1166|5072|390|1997|4|3|13|1|1997|390|5072|Thursday|1997Q1|N|N|N|2450509|2450567|2450156|2450429|N|N|N|N|N| +2450522|AAAAAAAAKFEGFCAA|1997-03-14|1166|5072|390|1997|5|3|14|1|1997|390|5072|Friday|1997Q1|N|Y|N|2450509|2450567|2450157|2450430|N|N|N|N|N| +2450523|AAAAAAAALFEGFCAA|1997-03-15|1166|5072|390|1997|6|3|15|1|1997|390|5072|Saturday|1997Q1|N|Y|N|2450509|2450567|2450158|2450431|N|N|N|N|N| +2450524|AAAAAAAAMFEGFCAA|1997-03-16|1166|5072|390|1997|0|3|16|1|1997|390|5072|Sunday|1997Q1|N|N|N|2450509|2450567|2450159|2450432|N|N|N|N|N| +2450525|AAAAAAAANFEGFCAA|1997-03-17|1166|5072|390|1997|1|3|17|1|1997|390|5072|Monday|1997Q1|N|N|N|2450509|2450567|2450160|2450433|N|N|N|N|N| +2450526|AAAAAAAAOFEGFCAA|1997-03-18|1166|5073|390|1997|2|3|18|1|1997|390|5073|Tuesday|1997Q1|N|N|N|2450509|2450567|2450161|2450434|N|N|N|N|N| +2450527|AAAAAAAAPFEGFCAA|1997-03-19|1166|5073|390|1997|3|3|19|1|1997|390|5073|Wednesday|1997Q1|N|N|N|2450509|2450567|2450162|2450435|N|N|N|N|N| +2450528|AAAAAAAAAGEGFCAA|1997-03-20|1166|5073|390|1997|4|3|20|1|1997|390|5073|Thursday|1997Q1|N|N|N|2450509|2450567|2450163|2450436|N|N|N|N|N| +2450529|AAAAAAAABGEGFCAA|1997-03-21|1166|5073|390|1997|5|3|21|1|1997|390|5073|Friday|1997Q1|N|Y|N|2450509|2450567|2450164|2450437|N|N|N|N|N| +2450530|AAAAAAAACGEGFCAA|1997-03-22|1166|5073|390|1997|6|3|22|1|1997|390|5073|Saturday|1997Q1|N|Y|N|2450509|2450567|2450165|2450438|N|N|N|N|N| +2450531|AAAAAAAADGEGFCAA|1997-03-23|1166|5073|390|1997|0|3|23|1|1997|390|5073|Sunday|1997Q1|N|N|N|2450509|2450567|2450166|2450439|N|N|N|N|N| +2450532|AAAAAAAAEGEGFCAA|1997-03-24|1166|5073|390|1997|1|3|24|1|1997|390|5073|Monday|1997Q1|N|N|N|2450509|2450567|2450167|2450440|N|N|N|N|N| +2450533|AAAAAAAAFGEGFCAA|1997-03-25|1166|5074|390|1997|2|3|25|1|1997|390|5074|Tuesday|1997Q1|N|N|N|2450509|2450567|2450168|2450441|N|N|N|N|N| +2450534|AAAAAAAAGGEGFCAA|1997-03-26|1166|5074|390|1997|3|3|26|1|1997|390|5074|Wednesday|1997Q1|N|N|N|2450509|2450567|2450169|2450442|N|N|N|N|N| +2450535|AAAAAAAAHGEGFCAA|1997-03-27|1166|5074|390|1997|4|3|27|1|1997|390|5074|Thursday|1997Q1|N|N|N|2450509|2450567|2450170|2450443|N|N|N|N|N| +2450536|AAAAAAAAIGEGFCAA|1997-03-28|1166|5074|390|1997|5|3|28|1|1997|390|5074|Friday|1997Q1|N|Y|N|2450509|2450567|2450171|2450444|N|N|N|N|N| +2450537|AAAAAAAAJGEGFCAA|1997-03-29|1166|5074|390|1997|6|3|29|1|1997|390|5074|Saturday|1997Q1|N|Y|N|2450509|2450567|2450172|2450445|N|N|N|N|N| +2450538|AAAAAAAAKGEGFCAA|1997-03-30|1166|5074|390|1997|0|3|30|1|1997|390|5074|Sunday|1997Q1|N|N|N|2450509|2450567|2450173|2450446|N|N|N|N|N| +2450539|AAAAAAAALGEGFCAA|1997-03-31|1166|5074|390|1997|1|3|31|1|1997|390|5074|Monday|1997Q1|N|N|N|2450509|2450567|2450174|2450447|N|N|N|N|N| +2450540|AAAAAAAAMGEGFCAA|1997-04-01|1167|5075|390|1997|2|4|1|1|1997|390|5075|Tuesday|1997Q1|N|N|N|2450540|2450629|2450175|2450450|N|N|N|N|N| +2450541|AAAAAAAANGEGFCAA|1997-04-02|1167|5075|390|1997|3|4|2|2|1997|390|5075|Wednesday|1997Q2|N|N|N|2450540|2450629|2450176|2450451|N|N|N|N|N| +2450542|AAAAAAAAOGEGFCAA|1997-04-03|1167|5075|390|1997|4|4|3|2|1997|390|5075|Thursday|1997Q2|N|N|N|2450540|2450629|2450177|2450452|N|N|N|N|N| +2450543|AAAAAAAAPGEGFCAA|1997-04-04|1167|5075|390|1997|5|4|4|2|1997|390|5075|Friday|1997Q2|N|Y|N|2450540|2450629|2450178|2450453|N|N|N|N|N| +2450544|AAAAAAAAAHEGFCAA|1997-04-05|1167|5075|390|1997|6|4|5|2|1997|390|5075|Saturday|1997Q2|N|Y|N|2450540|2450629|2450179|2450454|N|N|N|N|N| +2450545|AAAAAAAABHEGFCAA|1997-04-06|1167|5075|390|1997|0|4|6|2|1997|390|5075|Sunday|1997Q2|N|N|N|2450540|2450629|2450180|2450455|N|N|N|N|N| +2450546|AAAAAAAACHEGFCAA|1997-04-07|1167|5075|390|1997|1|4|7|2|1997|390|5075|Monday|1997Q2|N|N|N|2450540|2450629|2450181|2450456|N|N|N|N|N| +2450547|AAAAAAAADHEGFCAA|1997-04-08|1167|5076|390|1997|2|4|8|2|1997|390|5076|Tuesday|1997Q2|N|N|N|2450540|2450629|2450182|2450457|N|N|N|N|N| +2450548|AAAAAAAAEHEGFCAA|1997-04-09|1167|5076|390|1997|3|4|9|2|1997|390|5076|Wednesday|1997Q2|N|N|N|2450540|2450629|2450183|2450458|N|N|N|N|N| +2450549|AAAAAAAAFHEGFCAA|1997-04-10|1167|5076|390|1997|4|4|10|2|1997|390|5076|Thursday|1997Q2|N|N|N|2450540|2450629|2450184|2450459|N|N|N|N|N| +2450550|AAAAAAAAGHEGFCAA|1997-04-11|1167|5076|390|1997|5|4|11|2|1997|390|5076|Friday|1997Q2|N|Y|N|2450540|2450629|2450185|2450460|N|N|N|N|N| +2450551|AAAAAAAAHHEGFCAA|1997-04-12|1167|5076|390|1997|6|4|12|2|1997|390|5076|Saturday|1997Q2|N|Y|N|2450540|2450629|2450186|2450461|N|N|N|N|N| +2450552|AAAAAAAAIHEGFCAA|1997-04-13|1167|5076|390|1997|0|4|13|2|1997|390|5076|Sunday|1997Q2|N|N|N|2450540|2450629|2450187|2450462|N|N|N|N|N| +2450553|AAAAAAAAJHEGFCAA|1997-04-14|1167|5076|390|1997|1|4|14|2|1997|390|5076|Monday|1997Q2|N|N|N|2450540|2450629|2450188|2450463|N|N|N|N|N| +2450554|AAAAAAAAKHEGFCAA|1997-04-15|1167|5077|390|1997|2|4|15|2|1997|390|5077|Tuesday|1997Q2|N|N|N|2450540|2450629|2450189|2450464|N|N|N|N|N| +2450555|AAAAAAAALHEGFCAA|1997-04-16|1167|5077|390|1997|3|4|16|2|1997|390|5077|Wednesday|1997Q2|N|N|N|2450540|2450629|2450190|2450465|N|N|N|N|N| +2450556|AAAAAAAAMHEGFCAA|1997-04-17|1167|5077|390|1997|4|4|17|2|1997|390|5077|Thursday|1997Q2|N|N|N|2450540|2450629|2450191|2450466|N|N|N|N|N| +2450557|AAAAAAAANHEGFCAA|1997-04-18|1167|5077|390|1997|5|4|18|2|1997|390|5077|Friday|1997Q2|N|Y|N|2450540|2450629|2450192|2450467|N|N|N|N|N| +2450558|AAAAAAAAOHEGFCAA|1997-04-19|1167|5077|390|1997|6|4|19|2|1997|390|5077|Saturday|1997Q2|N|Y|N|2450540|2450629|2450193|2450468|N|N|N|N|N| +2450559|AAAAAAAAPHEGFCAA|1997-04-20|1167|5077|390|1997|0|4|20|2|1997|390|5077|Sunday|1997Q2|N|N|N|2450540|2450629|2450194|2450469|N|N|N|N|N| +2450560|AAAAAAAAAIEGFCAA|1997-04-21|1167|5077|390|1997|1|4|21|2|1997|390|5077|Monday|1997Q2|N|N|N|2450540|2450629|2450195|2450470|N|N|N|N|N| +2450561|AAAAAAAABIEGFCAA|1997-04-22|1167|5078|390|1997|2|4|22|2|1997|390|5078|Tuesday|1997Q2|N|N|N|2450540|2450629|2450196|2450471|N|N|N|N|N| +2450562|AAAAAAAACIEGFCAA|1997-04-23|1167|5078|390|1997|3|4|23|2|1997|390|5078|Wednesday|1997Q2|N|N|N|2450540|2450629|2450197|2450472|N|N|N|N|N| +2450563|AAAAAAAADIEGFCAA|1997-04-24|1167|5078|390|1997|4|4|24|2|1997|390|5078|Thursday|1997Q2|N|N|N|2450540|2450629|2450198|2450473|N|N|N|N|N| +2450564|AAAAAAAAEIEGFCAA|1997-04-25|1167|5078|390|1997|5|4|25|2|1997|390|5078|Friday|1997Q2|N|Y|N|2450540|2450629|2450199|2450474|N|N|N|N|N| +2450565|AAAAAAAAFIEGFCAA|1997-04-26|1167|5078|390|1997|6|4|26|2|1997|390|5078|Saturday|1997Q2|N|Y|N|2450540|2450629|2450200|2450475|N|N|N|N|N| +2450566|AAAAAAAAGIEGFCAA|1997-04-27|1167|5078|390|1997|0|4|27|2|1997|390|5078|Sunday|1997Q2|N|N|N|2450540|2450629|2450201|2450476|N|N|N|N|N| +2450567|AAAAAAAAHIEGFCAA|1997-04-28|1167|5078|390|1997|1|4|28|2|1997|390|5078|Monday|1997Q2|N|N|N|2450540|2450629|2450202|2450477|N|N|N|N|N| +2450568|AAAAAAAAIIEGFCAA|1997-04-29|1167|5079|390|1997|2|4|29|2|1997|390|5079|Tuesday|1997Q2|N|N|N|2450540|2450629|2450203|2450478|N|N|N|N|N| +2450569|AAAAAAAAJIEGFCAA|1997-04-30|1167|5079|390|1997|3|4|30|2|1997|390|5079|Wednesday|1997Q2|N|N|N|2450540|2450629|2450204|2450479|N|N|N|N|N| +2450570|AAAAAAAAKIEGFCAA|1997-05-01|1168|5079|390|1997|4|5|1|2|1997|390|5079|Thursday|1997Q2|N|N|N|2450570|2450689|2450205|2450480|N|N|N|N|N| +2450571|AAAAAAAALIEGFCAA|1997-05-02|1168|5079|390|1997|5|5|2|2|1997|390|5079|Friday|1997Q2|N|Y|N|2450570|2450689|2450206|2450481|N|N|N|N|N| +2450572|AAAAAAAAMIEGFCAA|1997-05-03|1168|5079|390|1997|6|5|3|2|1997|390|5079|Saturday|1997Q2|N|Y|N|2450570|2450689|2450207|2450482|N|N|N|N|N| +2450573|AAAAAAAANIEGFCAA|1997-05-04|1168|5079|390|1997|0|5|4|2|1997|390|5079|Sunday|1997Q2|N|N|N|2450570|2450689|2450208|2450483|N|N|N|N|N| +2450574|AAAAAAAAOIEGFCAA|1997-05-05|1168|5079|390|1997|1|5|5|2|1997|390|5079|Monday|1997Q2|N|N|N|2450570|2450689|2450209|2450484|N|N|N|N|N| +2450575|AAAAAAAAPIEGFCAA|1997-05-06|1168|5080|390|1997|2|5|6|2|1997|390|5080|Tuesday|1997Q2|N|N|N|2450570|2450689|2450210|2450485|N|N|N|N|N| +2450576|AAAAAAAAAJEGFCAA|1997-05-07|1168|5080|390|1997|3|5|7|2|1997|390|5080|Wednesday|1997Q2|N|N|N|2450570|2450689|2450211|2450486|N|N|N|N|N| +2450577|AAAAAAAABJEGFCAA|1997-05-08|1168|5080|390|1997|4|5|8|2|1997|390|5080|Thursday|1997Q2|N|N|N|2450570|2450689|2450212|2450487|N|N|N|N|N| +2450578|AAAAAAAACJEGFCAA|1997-05-09|1168|5080|390|1997|5|5|9|2|1997|390|5080|Friday|1997Q2|N|Y|N|2450570|2450689|2450213|2450488|N|N|N|N|N| +2450579|AAAAAAAADJEGFCAA|1997-05-10|1168|5080|390|1997|6|5|10|2|1997|390|5080|Saturday|1997Q2|N|Y|N|2450570|2450689|2450214|2450489|N|N|N|N|N| +2450580|AAAAAAAAEJEGFCAA|1997-05-11|1168|5080|390|1997|0|5|11|2|1997|390|5080|Sunday|1997Q2|N|N|N|2450570|2450689|2450215|2450490|N|N|N|N|N| +2450581|AAAAAAAAFJEGFCAA|1997-05-12|1168|5080|390|1997|1|5|12|2|1997|390|5080|Monday|1997Q2|N|N|N|2450570|2450689|2450216|2450491|N|N|N|N|N| +2450582|AAAAAAAAGJEGFCAA|1997-05-13|1168|5081|390|1997|2|5|13|2|1997|390|5081|Tuesday|1997Q2|N|N|N|2450570|2450689|2450217|2450492|N|N|N|N|N| +2450583|AAAAAAAAHJEGFCAA|1997-05-14|1168|5081|390|1997|3|5|14|2|1997|390|5081|Wednesday|1997Q2|N|N|N|2450570|2450689|2450218|2450493|N|N|N|N|N| +2450584|AAAAAAAAIJEGFCAA|1997-05-15|1168|5081|390|1997|4|5|15|2|1997|390|5081|Thursday|1997Q2|N|N|N|2450570|2450689|2450219|2450494|N|N|N|N|N| +2450585|AAAAAAAAJJEGFCAA|1997-05-16|1168|5081|390|1997|5|5|16|2|1997|390|5081|Friday|1997Q2|N|Y|N|2450570|2450689|2450220|2450495|N|N|N|N|N| +2450586|AAAAAAAAKJEGFCAA|1997-05-17|1168|5081|390|1997|6|5|17|2|1997|390|5081|Saturday|1997Q2|N|Y|N|2450570|2450689|2450221|2450496|N|N|N|N|N| +2450587|AAAAAAAALJEGFCAA|1997-05-18|1168|5081|390|1997|0|5|18|2|1997|390|5081|Sunday|1997Q2|N|N|N|2450570|2450689|2450222|2450497|N|N|N|N|N| +2450588|AAAAAAAAMJEGFCAA|1997-05-19|1168|5081|390|1997|1|5|19|2|1997|390|5081|Monday|1997Q2|N|N|N|2450570|2450689|2450223|2450498|N|N|N|N|N| +2450589|AAAAAAAANJEGFCAA|1997-05-20|1168|5082|390|1997|2|5|20|2|1997|390|5082|Tuesday|1997Q2|N|N|N|2450570|2450689|2450224|2450499|N|N|N|N|N| +2450590|AAAAAAAAOJEGFCAA|1997-05-21|1168|5082|390|1997|3|5|21|2|1997|390|5082|Wednesday|1997Q2|N|N|N|2450570|2450689|2450225|2450500|N|N|N|N|N| +2450591|AAAAAAAAPJEGFCAA|1997-05-22|1168|5082|390|1997|4|5|22|2|1997|390|5082|Thursday|1997Q2|N|N|N|2450570|2450689|2450226|2450501|N|N|N|N|N| +2450592|AAAAAAAAAKEGFCAA|1997-05-23|1168|5082|390|1997|5|5|23|2|1997|390|5082|Friday|1997Q2|N|Y|N|2450570|2450689|2450227|2450502|N|N|N|N|N| +2450593|AAAAAAAABKEGFCAA|1997-05-24|1168|5082|390|1997|6|5|24|2|1997|390|5082|Saturday|1997Q2|N|Y|N|2450570|2450689|2450228|2450503|N|N|N|N|N| +2450594|AAAAAAAACKEGFCAA|1997-05-25|1168|5082|390|1997|0|5|25|2|1997|390|5082|Sunday|1997Q2|N|N|N|2450570|2450689|2450229|2450504|N|N|N|N|N| +2450595|AAAAAAAADKEGFCAA|1997-05-26|1168|5082|390|1997|1|5|26|2|1997|390|5082|Monday|1997Q2|N|N|N|2450570|2450689|2450230|2450505|N|N|N|N|N| +2450596|AAAAAAAAEKEGFCAA|1997-05-27|1168|5083|390|1997|2|5|27|2|1997|390|5083|Tuesday|1997Q2|N|N|N|2450570|2450689|2450231|2450506|N|N|N|N|N| +2450597|AAAAAAAAFKEGFCAA|1997-05-28|1168|5083|390|1997|3|5|28|2|1997|390|5083|Wednesday|1997Q2|N|N|N|2450570|2450689|2450232|2450507|N|N|N|N|N| +2450598|AAAAAAAAGKEGFCAA|1997-05-29|1168|5083|390|1997|4|5|29|2|1997|390|5083|Thursday|1997Q2|N|N|N|2450570|2450689|2450233|2450508|N|N|N|N|N| +2450599|AAAAAAAAHKEGFCAA|1997-05-30|1168|5083|390|1997|5|5|30|2|1997|390|5083|Friday|1997Q2|N|Y|N|2450570|2450689|2450234|2450509|N|N|N|N|N| +2450600|AAAAAAAAIKEGFCAA|1997-05-31|1168|5083|390|1997|6|5|31|2|1997|390|5083|Saturday|1997Q2|N|Y|N|2450570|2450689|2450235|2450510|N|N|N|N|N| +2450601|AAAAAAAAJKEGFCAA|1997-06-01|1169|5083|391|1997|0|6|1|2|1997|391|5083|Sunday|1997Q2|N|N|N|2450601|2450751|2450236|2450511|N|N|N|N|N| +2450602|AAAAAAAAKKEGFCAA|1997-06-02|1169|5083|391|1997|1|6|2|2|1997|391|5083|Monday|1997Q2|N|N|N|2450601|2450751|2450237|2450512|N|N|N|N|N| +2450603|AAAAAAAALKEGFCAA|1997-06-03|1169|5084|391|1997|2|6|3|2|1997|391|5084|Tuesday|1997Q2|N|N|N|2450601|2450751|2450238|2450513|N|N|N|N|N| +2450604|AAAAAAAAMKEGFCAA|1997-06-04|1169|5084|391|1997|3|6|4|2|1997|391|5084|Wednesday|1997Q2|N|N|N|2450601|2450751|2450239|2450514|N|N|N|N|N| +2450605|AAAAAAAANKEGFCAA|1997-06-05|1169|5084|391|1997|4|6|5|2|1997|391|5084|Thursday|1997Q2|N|N|N|2450601|2450751|2450240|2450515|N|N|N|N|N| +2450606|AAAAAAAAOKEGFCAA|1997-06-06|1169|5084|391|1997|5|6|6|2|1997|391|5084|Friday|1997Q2|N|Y|N|2450601|2450751|2450241|2450516|N|N|N|N|N| +2450607|AAAAAAAAPKEGFCAA|1997-06-07|1169|5084|391|1997|6|6|7|2|1997|391|5084|Saturday|1997Q2|N|Y|N|2450601|2450751|2450242|2450517|N|N|N|N|N| +2450608|AAAAAAAAALEGFCAA|1997-06-08|1169|5084|391|1997|0|6|8|2|1997|391|5084|Sunday|1997Q2|N|N|N|2450601|2450751|2450243|2450518|N|N|N|N|N| +2450609|AAAAAAAABLEGFCAA|1997-06-09|1169|5084|391|1997|1|6|9|2|1997|391|5084|Monday|1997Q2|N|N|N|2450601|2450751|2450244|2450519|N|N|N|N|N| +2450610|AAAAAAAACLEGFCAA|1997-06-10|1169|5085|391|1997|2|6|10|2|1997|391|5085|Tuesday|1997Q2|N|N|N|2450601|2450751|2450245|2450520|N|N|N|N|N| +2450611|AAAAAAAADLEGFCAA|1997-06-11|1169|5085|391|1997|3|6|11|2|1997|391|5085|Wednesday|1997Q2|N|N|N|2450601|2450751|2450246|2450521|N|N|N|N|N| +2450612|AAAAAAAAELEGFCAA|1997-06-12|1169|5085|391|1997|4|6|12|2|1997|391|5085|Thursday|1997Q2|N|N|N|2450601|2450751|2450247|2450522|N|N|N|N|N| +2450613|AAAAAAAAFLEGFCAA|1997-06-13|1169|5085|391|1997|5|6|13|2|1997|391|5085|Friday|1997Q2|N|Y|N|2450601|2450751|2450248|2450523|N|N|N|N|N| +2450614|AAAAAAAAGLEGFCAA|1997-06-14|1169|5085|391|1997|6|6|14|2|1997|391|5085|Saturday|1997Q2|N|Y|N|2450601|2450751|2450249|2450524|N|N|N|N|N| +2450615|AAAAAAAAHLEGFCAA|1997-06-15|1169|5085|391|1997|0|6|15|2|1997|391|5085|Sunday|1997Q2|N|N|N|2450601|2450751|2450250|2450525|N|N|N|N|N| +2450616|AAAAAAAAILEGFCAA|1997-06-16|1169|5085|391|1997|1|6|16|2|1997|391|5085|Monday|1997Q2|N|N|N|2450601|2450751|2450251|2450526|N|N|N|N|N| +2450617|AAAAAAAAJLEGFCAA|1997-06-17|1169|5086|391|1997|2|6|17|2|1997|391|5086|Tuesday|1997Q2|N|N|N|2450601|2450751|2450252|2450527|N|N|N|N|N| +2450618|AAAAAAAAKLEGFCAA|1997-06-18|1169|5086|391|1997|3|6|18|2|1997|391|5086|Wednesday|1997Q2|N|N|N|2450601|2450751|2450253|2450528|N|N|N|N|N| +2450619|AAAAAAAALLEGFCAA|1997-06-19|1169|5086|391|1997|4|6|19|2|1997|391|5086|Thursday|1997Q2|N|N|N|2450601|2450751|2450254|2450529|N|N|N|N|N| +2450620|AAAAAAAAMLEGFCAA|1997-06-20|1169|5086|391|1997|5|6|20|2|1997|391|5086|Friday|1997Q2|N|Y|N|2450601|2450751|2450255|2450530|N|N|N|N|N| +2450621|AAAAAAAANLEGFCAA|1997-06-21|1169|5086|391|1997|6|6|21|2|1997|391|5086|Saturday|1997Q2|N|Y|N|2450601|2450751|2450256|2450531|N|N|N|N|N| +2450622|AAAAAAAAOLEGFCAA|1997-06-22|1169|5086|391|1997|0|6|22|2|1997|391|5086|Sunday|1997Q2|N|N|N|2450601|2450751|2450257|2450532|N|N|N|N|N| +2450623|AAAAAAAAPLEGFCAA|1997-06-23|1169|5086|391|1997|1|6|23|2|1997|391|5086|Monday|1997Q2|N|N|N|2450601|2450751|2450258|2450533|N|N|N|N|N| +2450624|AAAAAAAAAMEGFCAA|1997-06-24|1169|5087|391|1997|2|6|24|2|1997|391|5087|Tuesday|1997Q2|N|N|N|2450601|2450751|2450259|2450534|N|N|N|N|N| +2450625|AAAAAAAABMEGFCAA|1997-06-25|1169|5087|391|1997|3|6|25|2|1997|391|5087|Wednesday|1997Q2|N|N|N|2450601|2450751|2450260|2450535|N|N|N|N|N| +2450626|AAAAAAAACMEGFCAA|1997-06-26|1169|5087|391|1997|4|6|26|2|1997|391|5087|Thursday|1997Q2|N|N|N|2450601|2450751|2450261|2450536|N|N|N|N|N| +2450627|AAAAAAAADMEGFCAA|1997-06-27|1169|5087|391|1997|5|6|27|2|1997|391|5087|Friday|1997Q2|N|Y|N|2450601|2450751|2450262|2450537|N|N|N|N|N| +2450628|AAAAAAAAEMEGFCAA|1997-06-28|1169|5087|391|1997|6|6|28|2|1997|391|5087|Saturday|1997Q2|N|Y|N|2450601|2450751|2450263|2450538|N|N|N|N|N| +2450629|AAAAAAAAFMEGFCAA|1997-06-29|1169|5087|391|1997|0|6|29|2|1997|391|5087|Sunday|1997Q2|N|N|N|2450601|2450751|2450264|2450539|N|N|N|N|N| +2450630|AAAAAAAAGMEGFCAA|1997-06-30|1169|5087|391|1997|1|6|30|2|1997|391|5087|Monday|1997Q2|N|N|N|2450601|2450751|2450265|2450540|N|N|N|N|N| +2450631|AAAAAAAAHMEGFCAA|1997-07-01|1170|5088|391|1997|2|7|1|2|1997|391|5088|Tuesday|1997Q2|N|N|N|2450631|2450811|2450266|2450540|N|N|N|N|N| +2450632|AAAAAAAAIMEGFCAA|1997-07-02|1170|5088|391|1997|3|7|2|3|1997|391|5088|Wednesday|1997Q3|N|N|N|2450631|2450811|2450267|2450541|N|N|N|N|N| +2450633|AAAAAAAAJMEGFCAA|1997-07-03|1170|5088|391|1997|4|7|3|3|1997|391|5088|Thursday|1997Q3|N|N|N|2450631|2450811|2450268|2450542|N|N|N|N|N| +2450634|AAAAAAAAKMEGFCAA|1997-07-04|1170|5088|391|1997|5|7|4|3|1997|391|5088|Friday|1997Q3|N|Y|N|2450631|2450811|2450269|2450543|N|N|N|N|N| +2450635|AAAAAAAALMEGFCAA|1997-07-05|1170|5088|391|1997|6|7|5|3|1997|391|5088|Saturday|1997Q3|Y|Y|N|2450631|2450811|2450270|2450544|N|N|N|N|N| +2450636|AAAAAAAAMMEGFCAA|1997-07-06|1170|5088|391|1997|0|7|6|3|1997|391|5088|Sunday|1997Q3|N|N|Y|2450631|2450811|2450271|2450545|N|N|N|N|N| +2450637|AAAAAAAANMEGFCAA|1997-07-07|1170|5088|391|1997|1|7|7|3|1997|391|5088|Monday|1997Q3|N|N|N|2450631|2450811|2450272|2450546|N|N|N|N|N| +2450638|AAAAAAAAOMEGFCAA|1997-07-08|1170|5089|391|1997|2|7|8|3|1997|391|5089|Tuesday|1997Q3|N|N|N|2450631|2450811|2450273|2450547|N|N|N|N|N| +2450639|AAAAAAAAPMEGFCAA|1997-07-09|1170|5089|391|1997|3|7|9|3|1997|391|5089|Wednesday|1997Q3|N|N|N|2450631|2450811|2450274|2450548|N|N|N|N|N| +2450640|AAAAAAAAANEGFCAA|1997-07-10|1170|5089|391|1997|4|7|10|3|1997|391|5089|Thursday|1997Q3|N|N|N|2450631|2450811|2450275|2450549|N|N|N|N|N| +2450641|AAAAAAAABNEGFCAA|1997-07-11|1170|5089|391|1997|5|7|11|3|1997|391|5089|Friday|1997Q3|N|Y|N|2450631|2450811|2450276|2450550|N|N|N|N|N| +2450642|AAAAAAAACNEGFCAA|1997-07-12|1170|5089|391|1997|6|7|12|3|1997|391|5089|Saturday|1997Q3|N|Y|N|2450631|2450811|2450277|2450551|N|N|N|N|N| +2450643|AAAAAAAADNEGFCAA|1997-07-13|1170|5089|391|1997|0|7|13|3|1997|391|5089|Sunday|1997Q3|N|N|N|2450631|2450811|2450278|2450552|N|N|N|N|N| +2450644|AAAAAAAAENEGFCAA|1997-07-14|1170|5089|391|1997|1|7|14|3|1997|391|5089|Monday|1997Q3|N|N|N|2450631|2450811|2450279|2450553|N|N|N|N|N| +2450645|AAAAAAAAFNEGFCAA|1997-07-15|1170|5090|391|1997|2|7|15|3|1997|391|5090|Tuesday|1997Q3|N|N|N|2450631|2450811|2450280|2450554|N|N|N|N|N| +2450646|AAAAAAAAGNEGFCAA|1997-07-16|1170|5090|391|1997|3|7|16|3|1997|391|5090|Wednesday|1997Q3|N|N|N|2450631|2450811|2450281|2450555|N|N|N|N|N| +2450647|AAAAAAAAHNEGFCAA|1997-07-17|1170|5090|391|1997|4|7|17|3|1997|391|5090|Thursday|1997Q3|N|N|N|2450631|2450811|2450282|2450556|N|N|N|N|N| +2450648|AAAAAAAAINEGFCAA|1997-07-18|1170|5090|391|1997|5|7|18|3|1997|391|5090|Friday|1997Q3|N|Y|N|2450631|2450811|2450283|2450557|N|N|N|N|N| +2450649|AAAAAAAAJNEGFCAA|1997-07-19|1170|5090|391|1997|6|7|19|3|1997|391|5090|Saturday|1997Q3|N|Y|N|2450631|2450811|2450284|2450558|N|N|N|N|N| +2450650|AAAAAAAAKNEGFCAA|1997-07-20|1170|5090|391|1997|0|7|20|3|1997|391|5090|Sunday|1997Q3|N|N|N|2450631|2450811|2450285|2450559|N|N|N|N|N| +2450651|AAAAAAAALNEGFCAA|1997-07-21|1170|5090|391|1997|1|7|21|3|1997|391|5090|Monday|1997Q3|N|N|N|2450631|2450811|2450286|2450560|N|N|N|N|N| +2450652|AAAAAAAAMNEGFCAA|1997-07-22|1170|5091|391|1997|2|7|22|3|1997|391|5091|Tuesday|1997Q3|N|N|N|2450631|2450811|2450287|2450561|N|N|N|N|N| +2450653|AAAAAAAANNEGFCAA|1997-07-23|1170|5091|391|1997|3|7|23|3|1997|391|5091|Wednesday|1997Q3|N|N|N|2450631|2450811|2450288|2450562|N|N|N|N|N| +2450654|AAAAAAAAONEGFCAA|1997-07-24|1170|5091|391|1997|4|7|24|3|1997|391|5091|Thursday|1997Q3|N|N|N|2450631|2450811|2450289|2450563|N|N|N|N|N| +2450655|AAAAAAAAPNEGFCAA|1997-07-25|1170|5091|391|1997|5|7|25|3|1997|391|5091|Friday|1997Q3|N|Y|N|2450631|2450811|2450290|2450564|N|N|N|N|N| +2450656|AAAAAAAAAOEGFCAA|1997-07-26|1170|5091|391|1997|6|7|26|3|1997|391|5091|Saturday|1997Q3|N|Y|N|2450631|2450811|2450291|2450565|N|N|N|N|N| +2450657|AAAAAAAABOEGFCAA|1997-07-27|1170|5091|391|1997|0|7|27|3|1997|391|5091|Sunday|1997Q3|N|N|N|2450631|2450811|2450292|2450566|N|N|N|N|N| +2450658|AAAAAAAACOEGFCAA|1997-07-28|1170|5091|391|1997|1|7|28|3|1997|391|5091|Monday|1997Q3|N|N|N|2450631|2450811|2450293|2450567|N|N|N|N|N| +2450659|AAAAAAAADOEGFCAA|1997-07-29|1170|5092|391|1997|2|7|29|3|1997|391|5092|Tuesday|1997Q3|N|N|N|2450631|2450811|2450294|2450568|N|N|N|N|N| +2450660|AAAAAAAAEOEGFCAA|1997-07-30|1170|5092|391|1997|3|7|30|3|1997|391|5092|Wednesday|1997Q3|N|N|N|2450631|2450811|2450295|2450569|N|N|N|N|N| +2450661|AAAAAAAAFOEGFCAA|1997-07-31|1170|5092|391|1997|4|7|31|3|1997|391|5092|Thursday|1997Q3|N|N|N|2450631|2450811|2450296|2450570|N|N|N|N|N| +2450662|AAAAAAAAGOEGFCAA|1997-08-01|1171|5092|391|1997|5|8|1|3|1997|391|5092|Friday|1997Q3|N|Y|N|2450662|2450873|2450297|2450571|N|N|N|N|N| +2450663|AAAAAAAAHOEGFCAA|1997-08-02|1171|5092|391|1997|6|8|2|3|1997|391|5092|Saturday|1997Q3|N|Y|N|2450662|2450873|2450298|2450572|N|N|N|N|N| +2450664|AAAAAAAAIOEGFCAA|1997-08-03|1171|5092|391|1997|0|8|3|3|1997|391|5092|Sunday|1997Q3|N|N|N|2450662|2450873|2450299|2450573|N|N|N|N|N| +2450665|AAAAAAAAJOEGFCAA|1997-08-04|1171|5092|391|1997|1|8|4|3|1997|391|5092|Monday|1997Q3|N|N|N|2450662|2450873|2450300|2450574|N|N|N|N|N| +2450666|AAAAAAAAKOEGFCAA|1997-08-05|1171|5093|391|1997|2|8|5|3|1997|391|5093|Tuesday|1997Q3|N|N|N|2450662|2450873|2450301|2450575|N|N|N|N|N| +2450667|AAAAAAAALOEGFCAA|1997-08-06|1171|5093|391|1997|3|8|6|3|1997|391|5093|Wednesday|1997Q3|N|N|N|2450662|2450873|2450302|2450576|N|N|N|N|N| +2450668|AAAAAAAAMOEGFCAA|1997-08-07|1171|5093|391|1997|4|8|7|3|1997|391|5093|Thursday|1997Q3|N|N|N|2450662|2450873|2450303|2450577|N|N|N|N|N| +2450669|AAAAAAAANOEGFCAA|1997-08-08|1171|5093|391|1997|5|8|8|3|1997|391|5093|Friday|1997Q3|N|Y|N|2450662|2450873|2450304|2450578|N|N|N|N|N| +2450670|AAAAAAAAOOEGFCAA|1997-08-09|1171|5093|391|1997|6|8|9|3|1997|391|5093|Saturday|1997Q3|N|Y|N|2450662|2450873|2450305|2450579|N|N|N|N|N| +2450671|AAAAAAAAPOEGFCAA|1997-08-10|1171|5093|391|1997|0|8|10|3|1997|391|5093|Sunday|1997Q3|N|N|N|2450662|2450873|2450306|2450580|N|N|N|N|N| +2450672|AAAAAAAAAPEGFCAA|1997-08-11|1171|5093|391|1997|1|8|11|3|1997|391|5093|Monday|1997Q3|N|N|N|2450662|2450873|2450307|2450581|N|N|N|N|N| +2450673|AAAAAAAABPEGFCAA|1997-08-12|1171|5094|391|1997|2|8|12|3|1997|391|5094|Tuesday|1997Q3|N|N|N|2450662|2450873|2450308|2450582|N|N|N|N|N| +2450674|AAAAAAAACPEGFCAA|1997-08-13|1171|5094|391|1997|3|8|13|3|1997|391|5094|Wednesday|1997Q3|N|N|N|2450662|2450873|2450309|2450583|N|N|N|N|N| +2450675|AAAAAAAADPEGFCAA|1997-08-14|1171|5094|391|1997|4|8|14|3|1997|391|5094|Thursday|1997Q3|N|N|N|2450662|2450873|2450310|2450584|N|N|N|N|N| +2450676|AAAAAAAAEPEGFCAA|1997-08-15|1171|5094|391|1997|5|8|15|3|1997|391|5094|Friday|1997Q3|N|Y|N|2450662|2450873|2450311|2450585|N|N|N|N|N| +2450677|AAAAAAAAFPEGFCAA|1997-08-16|1171|5094|391|1997|6|8|16|3|1997|391|5094|Saturday|1997Q3|N|Y|N|2450662|2450873|2450312|2450586|N|N|N|N|N| +2450678|AAAAAAAAGPEGFCAA|1997-08-17|1171|5094|391|1997|0|8|17|3|1997|391|5094|Sunday|1997Q3|N|N|N|2450662|2450873|2450313|2450587|N|N|N|N|N| +2450679|AAAAAAAAHPEGFCAA|1997-08-18|1171|5094|391|1997|1|8|18|3|1997|391|5094|Monday|1997Q3|N|N|N|2450662|2450873|2450314|2450588|N|N|N|N|N| +2450680|AAAAAAAAIPEGFCAA|1997-08-19|1171|5095|391|1997|2|8|19|3|1997|391|5095|Tuesday|1997Q3|N|N|N|2450662|2450873|2450315|2450589|N|N|N|N|N| +2450681|AAAAAAAAJPEGFCAA|1997-08-20|1171|5095|391|1997|3|8|20|3|1997|391|5095|Wednesday|1997Q3|N|N|N|2450662|2450873|2450316|2450590|N|N|N|N|N| +2450682|AAAAAAAAKPEGFCAA|1997-08-21|1171|5095|391|1997|4|8|21|3|1997|391|5095|Thursday|1997Q3|N|N|N|2450662|2450873|2450317|2450591|N|N|N|N|N| +2450683|AAAAAAAALPEGFCAA|1997-08-22|1171|5095|391|1997|5|8|22|3|1997|391|5095|Friday|1997Q3|N|Y|N|2450662|2450873|2450318|2450592|N|N|N|N|N| +2450684|AAAAAAAAMPEGFCAA|1997-08-23|1171|5095|391|1997|6|8|23|3|1997|391|5095|Saturday|1997Q3|N|Y|N|2450662|2450873|2450319|2450593|N|N|N|N|N| +2450685|AAAAAAAANPEGFCAA|1997-08-24|1171|5095|391|1997|0|8|24|3|1997|391|5095|Sunday|1997Q3|N|N|N|2450662|2450873|2450320|2450594|N|N|N|N|N| +2450686|AAAAAAAAOPEGFCAA|1997-08-25|1171|5095|391|1997|1|8|25|3|1997|391|5095|Monday|1997Q3|N|N|N|2450662|2450873|2450321|2450595|N|N|N|N|N| +2450687|AAAAAAAAPPEGFCAA|1997-08-26|1171|5096|391|1997|2|8|26|3|1997|391|5096|Tuesday|1997Q3|N|N|N|2450662|2450873|2450322|2450596|N|N|N|N|N| +2450688|AAAAAAAAAAFGFCAA|1997-08-27|1171|5096|391|1997|3|8|27|3|1997|391|5096|Wednesday|1997Q3|N|N|N|2450662|2450873|2450323|2450597|N|N|N|N|N| +2450689|AAAAAAAABAFGFCAA|1997-08-28|1171|5096|391|1997|4|8|28|3|1997|391|5096|Thursday|1997Q3|N|N|N|2450662|2450873|2450324|2450598|N|N|N|N|N| +2450690|AAAAAAAACAFGFCAA|1997-08-29|1171|5096|391|1997|5|8|29|3|1997|391|5096|Friday|1997Q3|N|Y|N|2450662|2450873|2450325|2450599|N|N|N|N|N| +2450691|AAAAAAAADAFGFCAA|1997-08-30|1171|5096|391|1997|6|8|30|3|1997|391|5096|Saturday|1997Q3|N|Y|N|2450662|2450873|2450326|2450600|N|N|N|N|N| +2450692|AAAAAAAAEAFGFCAA|1997-08-31|1171|5096|391|1997|0|8|31|3|1997|391|5096|Sunday|1997Q3|N|N|N|2450662|2450873|2450327|2450601|N|N|N|N|N| +2450693|AAAAAAAAFAFGFCAA|1997-09-01|1172|5096|392|1997|1|9|1|3|1997|392|5096|Monday|1997Q3|N|N|N|2450693|2450935|2450328|2450602|N|N|N|N|N| +2450694|AAAAAAAAGAFGFCAA|1997-09-02|1172|5097|392|1997|2|9|2|3|1997|392|5097|Tuesday|1997Q3|N|N|N|2450693|2450935|2450329|2450603|N|N|N|N|N| +2450695|AAAAAAAAHAFGFCAA|1997-09-03|1172|5097|392|1997|3|9|3|3|1997|392|5097|Wednesday|1997Q3|N|N|N|2450693|2450935|2450330|2450604|N|N|N|N|N| +2450696|AAAAAAAAIAFGFCAA|1997-09-04|1172|5097|392|1997|4|9|4|3|1997|392|5097|Thursday|1997Q3|N|N|N|2450693|2450935|2450331|2450605|N|N|N|N|N| +2450697|AAAAAAAAJAFGFCAA|1997-09-05|1172|5097|392|1997|5|9|5|3|1997|392|5097|Friday|1997Q3|N|Y|N|2450693|2450935|2450332|2450606|N|N|N|N|N| +2450698|AAAAAAAAKAFGFCAA|1997-09-06|1172|5097|392|1997|6|9|6|3|1997|392|5097|Saturday|1997Q3|N|Y|N|2450693|2450935|2450333|2450607|N|N|N|N|N| +2450699|AAAAAAAALAFGFCAA|1997-09-07|1172|5097|392|1997|0|9|7|3|1997|392|5097|Sunday|1997Q3|N|N|N|2450693|2450935|2450334|2450608|N|N|N|N|N| +2450700|AAAAAAAAMAFGFCAA|1997-09-08|1172|5097|392|1997|1|9|8|3|1997|392|5097|Monday|1997Q3|N|N|N|2450693|2450935|2450335|2450609|N|N|N|N|N| +2450701|AAAAAAAANAFGFCAA|1997-09-09|1172|5098|392|1997|2|9|9|3|1997|392|5098|Tuesday|1997Q3|N|N|N|2450693|2450935|2450336|2450610|N|N|N|N|N| +2450702|AAAAAAAAOAFGFCAA|1997-09-10|1172|5098|392|1997|3|9|10|3|1997|392|5098|Wednesday|1997Q3|N|N|N|2450693|2450935|2450337|2450611|N|N|N|N|N| +2450703|AAAAAAAAPAFGFCAA|1997-09-11|1172|5098|392|1997|4|9|11|3|1997|392|5098|Thursday|1997Q3|N|N|N|2450693|2450935|2450338|2450612|N|N|N|N|N| +2450704|AAAAAAAAABFGFCAA|1997-09-12|1172|5098|392|1997|5|9|12|3|1997|392|5098|Friday|1997Q3|N|Y|N|2450693|2450935|2450339|2450613|N|N|N|N|N| +2450705|AAAAAAAABBFGFCAA|1997-09-13|1172|5098|392|1997|6|9|13|3|1997|392|5098|Saturday|1997Q3|N|Y|N|2450693|2450935|2450340|2450614|N|N|N|N|N| +2450706|AAAAAAAACBFGFCAA|1997-09-14|1172|5098|392|1997|0|9|14|3|1997|392|5098|Sunday|1997Q3|N|N|N|2450693|2450935|2450341|2450615|N|N|N|N|N| +2450707|AAAAAAAADBFGFCAA|1997-09-15|1172|5098|392|1997|1|9|15|3|1997|392|5098|Monday|1997Q3|N|N|N|2450693|2450935|2450342|2450616|N|N|N|N|N| +2450708|AAAAAAAAEBFGFCAA|1997-09-16|1172|5099|392|1997|2|9|16|3|1997|392|5099|Tuesday|1997Q3|N|N|N|2450693|2450935|2450343|2450617|N|N|N|N|N| +2450709|AAAAAAAAFBFGFCAA|1997-09-17|1172|5099|392|1997|3|9|17|3|1997|392|5099|Wednesday|1997Q3|N|N|N|2450693|2450935|2450344|2450618|N|N|N|N|N| +2450710|AAAAAAAAGBFGFCAA|1997-09-18|1172|5099|392|1997|4|9|18|3|1997|392|5099|Thursday|1997Q3|N|N|N|2450693|2450935|2450345|2450619|N|N|N|N|N| +2450711|AAAAAAAAHBFGFCAA|1997-09-19|1172|5099|392|1997|5|9|19|3|1997|392|5099|Friday|1997Q3|N|Y|N|2450693|2450935|2450346|2450620|N|N|N|N|N| +2450712|AAAAAAAAIBFGFCAA|1997-09-20|1172|5099|392|1997|6|9|20|3|1997|392|5099|Saturday|1997Q3|N|Y|N|2450693|2450935|2450347|2450621|N|N|N|N|N| +2450713|AAAAAAAAJBFGFCAA|1997-09-21|1172|5099|392|1997|0|9|21|3|1997|392|5099|Sunday|1997Q3|N|N|N|2450693|2450935|2450348|2450622|N|N|N|N|N| +2450714|AAAAAAAAKBFGFCAA|1997-09-22|1172|5099|392|1997|1|9|22|3|1997|392|5099|Monday|1997Q3|N|N|N|2450693|2450935|2450349|2450623|N|N|N|N|N| +2450715|AAAAAAAALBFGFCAA|1997-09-23|1172|5100|392|1997|2|9|23|3|1997|392|5100|Tuesday|1997Q3|N|N|N|2450693|2450935|2450350|2450624|N|N|N|N|N| +2450716|AAAAAAAAMBFGFCAA|1997-09-24|1172|5100|392|1997|3|9|24|3|1997|392|5100|Wednesday|1997Q3|N|N|N|2450693|2450935|2450351|2450625|N|N|N|N|N| +2450717|AAAAAAAANBFGFCAA|1997-09-25|1172|5100|392|1997|4|9|25|3|1997|392|5100|Thursday|1997Q3|N|N|N|2450693|2450935|2450352|2450626|N|N|N|N|N| +2450718|AAAAAAAAOBFGFCAA|1997-09-26|1172|5100|392|1997|5|9|26|3|1997|392|5100|Friday|1997Q3|N|Y|N|2450693|2450935|2450353|2450627|N|N|N|N|N| +2450719|AAAAAAAAPBFGFCAA|1997-09-27|1172|5100|392|1997|6|9|27|3|1997|392|5100|Saturday|1997Q3|N|Y|N|2450693|2450935|2450354|2450628|N|N|N|N|N| +2450720|AAAAAAAAACFGFCAA|1997-09-28|1172|5100|392|1997|0|9|28|3|1997|392|5100|Sunday|1997Q3|N|N|N|2450693|2450935|2450355|2450629|N|N|N|N|N| +2450721|AAAAAAAABCFGFCAA|1997-09-29|1172|5100|392|1997|1|9|29|3|1997|392|5100|Monday|1997Q3|N|N|N|2450693|2450935|2450356|2450630|N|N|N|N|N| +2450722|AAAAAAAACCFGFCAA|1997-09-30|1172|5101|392|1997|2|9|30|3|1997|392|5101|Tuesday|1997Q3|N|N|N|2450693|2450935|2450357|2450631|N|N|N|N|N| +2450723|AAAAAAAADCFGFCAA|1997-10-01|1173|5101|392|1997|3|10|1|3|1997|392|5101|Wednesday|1997Q3|N|N|N|2450723|2450995|2450358|2450631|N|N|N|N|N| +2450724|AAAAAAAAECFGFCAA|1997-10-02|1173|5101|392|1997|4|10|2|4|1997|392|5101|Thursday|1997Q4|N|N|N|2450723|2450995|2450359|2450632|N|N|N|N|N| +2450725|AAAAAAAAFCFGFCAA|1997-10-03|1173|5101|392|1997|5|10|3|4|1997|392|5101|Friday|1997Q4|N|Y|N|2450723|2450995|2450360|2450633|N|N|N|N|N| +2450726|AAAAAAAAGCFGFCAA|1997-10-04|1173|5101|392|1997|6|10|4|4|1997|392|5101|Saturday|1997Q4|N|Y|N|2450723|2450995|2450361|2450634|N|N|N|N|N| +2450727|AAAAAAAAHCFGFCAA|1997-10-05|1173|5101|392|1997|0|10|5|4|1997|392|5101|Sunday|1997Q4|N|N|N|2450723|2450995|2450362|2450635|N|N|N|N|N| +2450728|AAAAAAAAICFGFCAA|1997-10-06|1173|5101|392|1997|1|10|6|4|1997|392|5101|Monday|1997Q4|N|N|N|2450723|2450995|2450363|2450636|N|N|N|N|N| +2450729|AAAAAAAAJCFGFCAA|1997-10-07|1173|5102|392|1997|2|10|7|4|1997|392|5102|Tuesday|1997Q4|N|N|N|2450723|2450995|2450364|2450637|N|N|N|N|N| +2450730|AAAAAAAAKCFGFCAA|1997-10-08|1173|5102|392|1997|3|10|8|4|1997|392|5102|Wednesday|1997Q4|N|N|N|2450723|2450995|2450365|2450638|N|N|N|N|N| +2450731|AAAAAAAALCFGFCAA|1997-10-09|1173|5102|392|1997|4|10|9|4|1997|392|5102|Thursday|1997Q4|N|N|N|2450723|2450995|2450366|2450639|N|N|N|N|N| +2450732|AAAAAAAAMCFGFCAA|1997-10-10|1173|5102|392|1997|5|10|10|4|1997|392|5102|Friday|1997Q4|N|Y|N|2450723|2450995|2450367|2450640|N|N|N|N|N| +2450733|AAAAAAAANCFGFCAA|1997-10-11|1173|5102|392|1997|6|10|11|4|1997|392|5102|Saturday|1997Q4|N|Y|N|2450723|2450995|2450368|2450641|N|N|N|N|N| +2450734|AAAAAAAAOCFGFCAA|1997-10-12|1173|5102|392|1997|0|10|12|4|1997|392|5102|Sunday|1997Q4|N|N|N|2450723|2450995|2450369|2450642|N|N|N|N|N| +2450735|AAAAAAAAPCFGFCAA|1997-10-13|1173|5102|392|1997|1|10|13|4|1997|392|5102|Monday|1997Q4|N|N|N|2450723|2450995|2450370|2450643|N|N|N|N|N| +2450736|AAAAAAAAADFGFCAA|1997-10-14|1173|5103|392|1997|2|10|14|4|1997|392|5103|Tuesday|1997Q4|N|N|N|2450723|2450995|2450371|2450644|N|N|N|N|N| +2450737|AAAAAAAABDFGFCAA|1997-10-15|1173|5103|392|1997|3|10|15|4|1997|392|5103|Wednesday|1997Q4|N|N|N|2450723|2450995|2450372|2450645|N|N|N|N|N| +2450738|AAAAAAAACDFGFCAA|1997-10-16|1173|5103|392|1997|4|10|16|4|1997|392|5103|Thursday|1997Q4|N|N|N|2450723|2450995|2450373|2450646|N|N|N|N|N| +2450739|AAAAAAAADDFGFCAA|1997-10-17|1173|5103|392|1997|5|10|17|4|1997|392|5103|Friday|1997Q4|N|Y|N|2450723|2450995|2450374|2450647|N|N|N|N|N| +2450740|AAAAAAAAEDFGFCAA|1997-10-18|1173|5103|392|1997|6|10|18|4|1997|392|5103|Saturday|1997Q4|N|Y|N|2450723|2450995|2450375|2450648|N|N|N|N|N| +2450741|AAAAAAAAFDFGFCAA|1997-10-19|1173|5103|392|1997|0|10|19|4|1997|392|5103|Sunday|1997Q4|N|N|N|2450723|2450995|2450376|2450649|N|N|N|N|N| +2450742|AAAAAAAAGDFGFCAA|1997-10-20|1173|5103|392|1997|1|10|20|4|1997|392|5103|Monday|1997Q4|N|N|N|2450723|2450995|2450377|2450650|N|N|N|N|N| +2450743|AAAAAAAAHDFGFCAA|1997-10-21|1173|5104|392|1997|2|10|21|4|1997|392|5104|Tuesday|1997Q4|N|N|N|2450723|2450995|2450378|2450651|N|N|N|N|N| +2450744|AAAAAAAAIDFGFCAA|1997-10-22|1173|5104|392|1997|3|10|22|4|1997|392|5104|Wednesday|1997Q4|N|N|N|2450723|2450995|2450379|2450652|N|N|N|N|N| +2450745|AAAAAAAAJDFGFCAA|1997-10-23|1173|5104|392|1997|4|10|23|4|1997|392|5104|Thursday|1997Q4|N|N|N|2450723|2450995|2450380|2450653|N|N|N|N|N| +2450746|AAAAAAAAKDFGFCAA|1997-10-24|1173|5104|392|1997|5|10|24|4|1997|392|5104|Friday|1997Q4|N|Y|N|2450723|2450995|2450381|2450654|N|N|N|N|N| +2450747|AAAAAAAALDFGFCAA|1997-10-25|1173|5104|392|1997|6|10|25|4|1997|392|5104|Saturday|1997Q4|N|Y|N|2450723|2450995|2450382|2450655|N|N|N|N|N| +2450748|AAAAAAAAMDFGFCAA|1997-10-26|1173|5104|392|1997|0|10|26|4|1997|392|5104|Sunday|1997Q4|N|N|N|2450723|2450995|2450383|2450656|N|N|N|N|N| +2450749|AAAAAAAANDFGFCAA|1997-10-27|1173|5104|392|1997|1|10|27|4|1997|392|5104|Monday|1997Q4|N|N|N|2450723|2450995|2450384|2450657|N|N|N|N|N| +2450750|AAAAAAAAODFGFCAA|1997-10-28|1173|5105|392|1997|2|10|28|4|1997|392|5105|Tuesday|1997Q4|N|N|N|2450723|2450995|2450385|2450658|N|N|N|N|N| +2450751|AAAAAAAAPDFGFCAA|1997-10-29|1173|5105|392|1997|3|10|29|4|1997|392|5105|Wednesday|1997Q4|N|N|N|2450723|2450995|2450386|2450659|N|N|N|N|N| +2450752|AAAAAAAAAEFGFCAA|1997-10-30|1173|5105|392|1997|4|10|30|4|1997|392|5105|Thursday|1997Q4|N|N|N|2450723|2450995|2450387|2450660|N|N|N|N|N| +2450753|AAAAAAAABEFGFCAA|1997-10-31|1173|5105|392|1997|5|10|31|4|1997|392|5105|Friday|1997Q4|N|Y|N|2450723|2450995|2450388|2450661|N|N|N|N|N| +2450754|AAAAAAAACEFGFCAA|1997-11-01|1174|5105|392|1997|6|11|1|4|1997|392|5105|Saturday|1997Q4|N|Y|N|2450754|2451057|2450389|2450662|N|N|N|N|N| +2450755|AAAAAAAADEFGFCAA|1997-11-02|1174|5105|392|1997|0|11|2|4|1997|392|5105|Sunday|1997Q4|N|N|N|2450754|2451057|2450390|2450663|N|N|N|N|N| +2450756|AAAAAAAAEEFGFCAA|1997-11-03|1174|5105|392|1997|1|11|3|4|1997|392|5105|Monday|1997Q4|N|N|N|2450754|2451057|2450391|2450664|N|N|N|N|N| +2450757|AAAAAAAAFEFGFCAA|1997-11-04|1174|5106|392|1997|2|11|4|4|1997|392|5106|Tuesday|1997Q4|N|N|N|2450754|2451057|2450392|2450665|N|N|N|N|N| +2450758|AAAAAAAAGEFGFCAA|1997-11-05|1174|5106|392|1997|3|11|5|4|1997|392|5106|Wednesday|1997Q4|N|N|N|2450754|2451057|2450393|2450666|N|N|N|N|N| +2450759|AAAAAAAAHEFGFCAA|1997-11-06|1174|5106|392|1997|4|11|6|4|1997|392|5106|Thursday|1997Q4|N|N|N|2450754|2451057|2450394|2450667|N|N|N|N|N| +2450760|AAAAAAAAIEFGFCAA|1997-11-07|1174|5106|392|1997|5|11|7|4|1997|392|5106|Friday|1997Q4|N|Y|N|2450754|2451057|2450395|2450668|N|N|N|N|N| +2450761|AAAAAAAAJEFGFCAA|1997-11-08|1174|5106|392|1997|6|11|8|4|1997|392|5106|Saturday|1997Q4|N|Y|N|2450754|2451057|2450396|2450669|N|N|N|N|N| +2450762|AAAAAAAAKEFGFCAA|1997-11-09|1174|5106|392|1997|0|11|9|4|1997|392|5106|Sunday|1997Q4|N|N|N|2450754|2451057|2450397|2450670|N|N|N|N|N| +2450763|AAAAAAAALEFGFCAA|1997-11-10|1174|5106|392|1997|1|11|10|4|1997|392|5106|Monday|1997Q4|N|N|N|2450754|2451057|2450398|2450671|N|N|N|N|N| +2450764|AAAAAAAAMEFGFCAA|1997-11-11|1174|5107|392|1997|2|11|11|4|1997|392|5107|Tuesday|1997Q4|N|N|N|2450754|2451057|2450399|2450672|N|N|N|N|N| +2450765|AAAAAAAANEFGFCAA|1997-11-12|1174|5107|392|1997|3|11|12|4|1997|392|5107|Wednesday|1997Q4|N|N|N|2450754|2451057|2450400|2450673|N|N|N|N|N| +2450766|AAAAAAAAOEFGFCAA|1997-11-13|1174|5107|392|1997|4|11|13|4|1997|392|5107|Thursday|1997Q4|N|N|N|2450754|2451057|2450401|2450674|N|N|N|N|N| +2450767|AAAAAAAAPEFGFCAA|1997-11-14|1174|5107|392|1997|5|11|14|4|1997|392|5107|Friday|1997Q4|N|Y|N|2450754|2451057|2450402|2450675|N|N|N|N|N| +2450768|AAAAAAAAAFFGFCAA|1997-11-15|1174|5107|392|1997|6|11|15|4|1997|392|5107|Saturday|1997Q4|N|Y|N|2450754|2451057|2450403|2450676|N|N|N|N|N| +2450769|AAAAAAAABFFGFCAA|1997-11-16|1174|5107|392|1997|0|11|16|4|1997|392|5107|Sunday|1997Q4|N|N|N|2450754|2451057|2450404|2450677|N|N|N|N|N| +2450770|AAAAAAAACFFGFCAA|1997-11-17|1174|5107|392|1997|1|11|17|4|1997|392|5107|Monday|1997Q4|N|N|N|2450754|2451057|2450405|2450678|N|N|N|N|N| +2450771|AAAAAAAADFFGFCAA|1997-11-18|1174|5108|392|1997|2|11|18|4|1997|392|5108|Tuesday|1997Q4|N|N|N|2450754|2451057|2450406|2450679|N|N|N|N|N| +2450772|AAAAAAAAEFFGFCAA|1997-11-19|1174|5108|392|1997|3|11|19|4|1997|392|5108|Wednesday|1997Q4|N|N|N|2450754|2451057|2450407|2450680|N|N|N|N|N| +2450773|AAAAAAAAFFFGFCAA|1997-11-20|1174|5108|392|1997|4|11|20|4|1997|392|5108|Thursday|1997Q4|N|N|N|2450754|2451057|2450408|2450681|N|N|N|N|N| +2450774|AAAAAAAAGFFGFCAA|1997-11-21|1174|5108|392|1997|5|11|21|4|1997|392|5108|Friday|1997Q4|N|Y|N|2450754|2451057|2450409|2450682|N|N|N|N|N| +2450775|AAAAAAAAHFFGFCAA|1997-11-22|1174|5108|392|1997|6|11|22|4|1997|392|5108|Saturday|1997Q4|N|Y|N|2450754|2451057|2450410|2450683|N|N|N|N|N| +2450776|AAAAAAAAIFFGFCAA|1997-11-23|1174|5108|392|1997|0|11|23|4|1997|392|5108|Sunday|1997Q4|N|N|N|2450754|2451057|2450411|2450684|N|N|N|N|N| +2450777|AAAAAAAAJFFGFCAA|1997-11-24|1174|5108|392|1997|1|11|24|4|1997|392|5108|Monday|1997Q4|N|N|N|2450754|2451057|2450412|2450685|N|N|N|N|N| +2450778|AAAAAAAAKFFGFCAA|1997-11-25|1174|5109|392|1997|2|11|25|4|1997|392|5109|Tuesday|1997Q4|N|N|N|2450754|2451057|2450413|2450686|N|N|N|N|N| +2450779|AAAAAAAALFFGFCAA|1997-11-26|1174|5109|392|1997|3|11|26|4|1997|392|5109|Wednesday|1997Q4|N|N|N|2450754|2451057|2450414|2450687|N|N|N|N|N| +2450780|AAAAAAAAMFFGFCAA|1997-11-27|1174|5109|392|1997|4|11|27|4|1997|392|5109|Thursday|1997Q4|N|N|N|2450754|2451057|2450415|2450688|N|N|N|N|N| +2450781|AAAAAAAANFFGFCAA|1997-11-28|1174|5109|392|1997|5|11|28|4|1997|392|5109|Friday|1997Q4|N|Y|N|2450754|2451057|2450416|2450689|N|N|N|N|N| +2450782|AAAAAAAAOFFGFCAA|1997-11-29|1174|5109|392|1997|6|11|29|4|1997|392|5109|Saturday|1997Q4|N|Y|N|2450754|2451057|2450417|2450690|N|N|N|N|N| +2450783|AAAAAAAAPFFGFCAA|1997-11-30|1174|5109|392|1997|0|11|30|4|1997|392|5109|Sunday|1997Q4|N|N|N|2450754|2451057|2450418|2450691|N|N|N|N|N| +2450784|AAAAAAAAAGFGFCAA|1997-12-01|1175|5109|393|1997|1|12|1|4|1997|393|5109|Monday|1997Q4|N|N|N|2450784|2451117|2450419|2450692|N|N|N|N|N| +2450785|AAAAAAAABGFGFCAA|1997-12-02|1175|5110|393|1997|2|12|2|4|1997|393|5110|Tuesday|1997Q4|N|N|N|2450784|2451117|2450420|2450693|N|N|N|N|N| +2450786|AAAAAAAACGFGFCAA|1997-12-03|1175|5110|393|1997|3|12|3|4|1997|393|5110|Wednesday|1997Q4|N|N|N|2450784|2451117|2450421|2450694|N|N|N|N|N| +2450787|AAAAAAAADGFGFCAA|1997-12-04|1175|5110|393|1997|4|12|4|4|1997|393|5110|Thursday|1997Q4|N|N|N|2450784|2451117|2450422|2450695|N|N|N|N|N| +2450788|AAAAAAAAEGFGFCAA|1997-12-05|1175|5110|393|1997|5|12|5|4|1997|393|5110|Friday|1997Q4|N|Y|N|2450784|2451117|2450423|2450696|N|N|N|N|N| +2450789|AAAAAAAAFGFGFCAA|1997-12-06|1175|5110|393|1997|6|12|6|4|1997|393|5110|Saturday|1997Q4|N|Y|N|2450784|2451117|2450424|2450697|N|N|N|N|N| +2450790|AAAAAAAAGGFGFCAA|1997-12-07|1175|5110|393|1997|0|12|7|4|1997|393|5110|Sunday|1997Q4|N|N|N|2450784|2451117|2450425|2450698|N|N|N|N|N| +2450791|AAAAAAAAHGFGFCAA|1997-12-08|1175|5110|393|1997|1|12|8|4|1997|393|5110|Monday|1997Q4|N|N|N|2450784|2451117|2450426|2450699|N|N|N|N|N| +2450792|AAAAAAAAIGFGFCAA|1997-12-09|1175|5111|393|1997|2|12|9|4|1997|393|5111|Tuesday|1997Q4|N|N|N|2450784|2451117|2450427|2450700|N|N|N|N|N| +2450793|AAAAAAAAJGFGFCAA|1997-12-10|1175|5111|393|1997|3|12|10|4|1997|393|5111|Wednesday|1997Q4|N|N|N|2450784|2451117|2450428|2450701|N|N|N|N|N| +2450794|AAAAAAAAKGFGFCAA|1997-12-11|1175|5111|393|1997|4|12|11|4|1997|393|5111|Thursday|1997Q4|N|N|N|2450784|2451117|2450429|2450702|N|N|N|N|N| +2450795|AAAAAAAALGFGFCAA|1997-12-12|1175|5111|393|1997|5|12|12|4|1997|393|5111|Friday|1997Q4|N|Y|N|2450784|2451117|2450430|2450703|N|N|N|N|N| +2450796|AAAAAAAAMGFGFCAA|1997-12-13|1175|5111|393|1997|6|12|13|4|1997|393|5111|Saturday|1997Q4|N|Y|N|2450784|2451117|2450431|2450704|N|N|N|N|N| +2450797|AAAAAAAANGFGFCAA|1997-12-14|1175|5111|393|1997|0|12|14|4|1997|393|5111|Sunday|1997Q4|N|N|N|2450784|2451117|2450432|2450705|N|N|N|N|N| +2450798|AAAAAAAAOGFGFCAA|1997-12-15|1175|5111|393|1997|1|12|15|4|1997|393|5111|Monday|1997Q4|N|N|N|2450784|2451117|2450433|2450706|N|N|N|N|N| +2450799|AAAAAAAAPGFGFCAA|1997-12-16|1175|5112|393|1997|2|12|16|4|1997|393|5112|Tuesday|1997Q4|N|N|N|2450784|2451117|2450434|2450707|N|N|N|N|N| +2450800|AAAAAAAAAHFGFCAA|1997-12-17|1175|5112|393|1997|3|12|17|4|1997|393|5112|Wednesday|1997Q4|N|N|N|2450784|2451117|2450435|2450708|N|N|N|N|N| +2450801|AAAAAAAABHFGFCAA|1997-12-18|1175|5112|393|1997|4|12|18|4|1997|393|5112|Thursday|1997Q4|N|N|N|2450784|2451117|2450436|2450709|N|N|N|N|N| +2450802|AAAAAAAACHFGFCAA|1997-12-19|1175|5112|393|1997|5|12|19|4|1997|393|5112|Friday|1997Q4|N|Y|N|2450784|2451117|2450437|2450710|N|N|N|N|N| +2450803|AAAAAAAADHFGFCAA|1997-12-20|1175|5112|393|1997|6|12|20|4|1997|393|5112|Saturday|1997Q4|N|Y|N|2450784|2451117|2450438|2450711|N|N|N|N|N| +2450804|AAAAAAAAEHFGFCAA|1997-12-21|1175|5112|393|1997|0|12|21|4|1997|393|5112|Sunday|1997Q4|N|N|N|2450784|2451117|2450439|2450712|N|N|N|N|N| +2450805|AAAAAAAAFHFGFCAA|1997-12-22|1175|5112|393|1997|1|12|22|4|1997|393|5112|Monday|1997Q4|N|N|N|2450784|2451117|2450440|2450713|N|N|N|N|N| +2450806|AAAAAAAAGHFGFCAA|1997-12-23|1175|5113|393|1997|2|12|23|4|1997|393|5113|Tuesday|1997Q4|N|N|N|2450784|2451117|2450441|2450714|N|N|N|N|N| +2450807|AAAAAAAAHHFGFCAA|1997-12-24|1175|5113|393|1997|3|12|24|4|1997|393|5113|Wednesday|1997Q4|N|N|N|2450784|2451117|2450442|2450715|N|N|N|N|N| +2450808|AAAAAAAAIHFGFCAA|1997-12-25|1175|5113|393|1997|4|12|25|4|1997|393|5113|Thursday|1997Q4|N|N|N|2450784|2451117|2450443|2450716|N|N|N|N|N| +2450809|AAAAAAAAJHFGFCAA|1997-12-26|1175|5113|393|1997|5|12|26|4|1997|393|5113|Friday|1997Q4|Y|Y|N|2450784|2451117|2450444|2450717|N|N|N|N|N| +2450810|AAAAAAAAKHFGFCAA|1997-12-27|1175|5113|393|1997|6|12|27|4|1997|393|5113|Saturday|1997Q4|N|Y|Y|2450784|2451117|2450445|2450718|N|N|N|N|N| +2450811|AAAAAAAALHFGFCAA|1997-12-28|1175|5113|393|1997|0|12|28|4|1997|393|5113|Sunday|1997Q4|N|N|N|2450784|2451117|2450446|2450719|N|N|N|N|N| +2450812|AAAAAAAAMHFGFCAA|1997-12-29|1175|5113|393|1997|1|12|29|4|1997|393|5113|Monday|1997Q4|N|N|N|2450784|2451117|2450447|2450720|N|N|N|N|N| +2450813|AAAAAAAANHFGFCAA|1997-12-30|1175|5114|393|1997|2|12|30|4|1997|393|5114|Tuesday|1997Q4|N|N|N|2450784|2451117|2450448|2450721|N|N|N|N|N| +2450814|AAAAAAAAOHFGFCAA|1997-12-31|1175|5114|393|1997|3|12|31|4|1997|393|5114|Wednesday|1997Q4|N|N|N|2450784|2451117|2450449|2450722|N|N|N|N|N| +2450815|AAAAAAAAPHFGFCAA|1998-01-01|1176|5114|393|1998|4|1|1|1|1998|393|5114|Thursday|1998Q1|Y|N|N|2450815|2450814|2450450|2450723|N|N|N|N|N| +2450816|AAAAAAAAAIFGFCAA|1998-01-02|1176|5114|393|1998|5|1|2|1|1998|393|5114|Friday|1998Q1|N|Y|Y|2450815|2450814|2450451|2450724|N|N|N|N|N| +2450817|AAAAAAAABIFGFCAA|1998-01-03|1176|5114|393|1998|6|1|3|1|1998|393|5114|Saturday|1998Q1|N|Y|N|2450815|2450814|2450452|2450725|N|N|N|N|N| +2450818|AAAAAAAACIFGFCAA|1998-01-04|1176|5114|393|1998|0|1|4|1|1998|393|5114|Sunday|1998Q1|N|N|N|2450815|2450814|2450453|2450726|N|N|N|N|N| +2450819|AAAAAAAADIFGFCAA|1998-01-05|1176|5114|393|1998|1|1|5|1|1998|393|5114|Monday|1998Q1|N|N|N|2450815|2450814|2450454|2450727|N|N|N|N|N| +2450820|AAAAAAAAEIFGFCAA|1998-01-06|1176|5115|393|1998|2|1|6|1|1998|393|5115|Tuesday|1998Q1|N|N|N|2450815|2450814|2450455|2450728|N|N|N|N|N| +2450821|AAAAAAAAFIFGFCAA|1998-01-07|1176|5115|393|1998|3|1|7|1|1998|393|5115|Wednesday|1998Q1|N|N|N|2450815|2450814|2450456|2450729|N|N|N|N|N| +2450822|AAAAAAAAGIFGFCAA|1998-01-08|1176|5115|393|1998|4|1|8|1|1998|393|5115|Thursday|1998Q1|N|N|N|2450815|2450814|2450457|2450730|N|N|N|N|N| +2450823|AAAAAAAAHIFGFCAA|1998-01-09|1176|5115|393|1998|5|1|9|1|1998|393|5115|Friday|1998Q1|N|Y|N|2450815|2450814|2450458|2450731|N|N|N|N|N| +2450824|AAAAAAAAIIFGFCAA|1998-01-10|1176|5115|393|1998|6|1|10|1|1998|393|5115|Saturday|1998Q1|N|Y|N|2450815|2450814|2450459|2450732|N|N|N|N|N| +2450825|AAAAAAAAJIFGFCAA|1998-01-11|1176|5115|393|1998|0|1|11|1|1998|393|5115|Sunday|1998Q1|N|N|N|2450815|2450814|2450460|2450733|N|N|N|N|N| +2450826|AAAAAAAAKIFGFCAA|1998-01-12|1176|5115|393|1998|1|1|12|1|1998|393|5115|Monday|1998Q1|N|N|N|2450815|2450814|2450461|2450734|N|N|N|N|N| +2450827|AAAAAAAALIFGFCAA|1998-01-13|1176|5116|393|1998|2|1|13|1|1998|393|5116|Tuesday|1998Q1|N|N|N|2450815|2450814|2450462|2450735|N|N|N|N|N| +2450828|AAAAAAAAMIFGFCAA|1998-01-14|1176|5116|393|1998|3|1|14|1|1998|393|5116|Wednesday|1998Q1|N|N|N|2450815|2450814|2450463|2450736|N|N|N|N|N| +2450829|AAAAAAAANIFGFCAA|1998-01-15|1176|5116|393|1998|4|1|15|1|1998|393|5116|Thursday|1998Q1|N|N|N|2450815|2450814|2450464|2450737|N|N|N|N|N| +2450830|AAAAAAAAOIFGFCAA|1998-01-16|1176|5116|393|1998|5|1|16|1|1998|393|5116|Friday|1998Q1|N|Y|N|2450815|2450814|2450465|2450738|N|N|N|N|N| +2450831|AAAAAAAAPIFGFCAA|1998-01-17|1176|5116|393|1998|6|1|17|1|1998|393|5116|Saturday|1998Q1|N|Y|N|2450815|2450814|2450466|2450739|N|N|N|N|N| +2450832|AAAAAAAAAJFGFCAA|1998-01-18|1176|5116|393|1998|0|1|18|1|1998|393|5116|Sunday|1998Q1|N|N|N|2450815|2450814|2450467|2450740|N|N|N|N|N| +2450833|AAAAAAAABJFGFCAA|1998-01-19|1176|5116|393|1998|1|1|19|1|1998|393|5116|Monday|1998Q1|N|N|N|2450815|2450814|2450468|2450741|N|N|N|N|N| +2450834|AAAAAAAACJFGFCAA|1998-01-20|1176|5117|393|1998|2|1|20|1|1998|393|5117|Tuesday|1998Q1|N|N|N|2450815|2450814|2450469|2450742|N|N|N|N|N| +2450835|AAAAAAAADJFGFCAA|1998-01-21|1176|5117|393|1998|3|1|21|1|1998|393|5117|Wednesday|1998Q1|N|N|N|2450815|2450814|2450470|2450743|N|N|N|N|N| +2450836|AAAAAAAAEJFGFCAA|1998-01-22|1176|5117|393|1998|4|1|22|1|1998|393|5117|Thursday|1998Q1|N|N|N|2450815|2450814|2450471|2450744|N|N|N|N|N| +2450837|AAAAAAAAFJFGFCAA|1998-01-23|1176|5117|393|1998|5|1|23|1|1998|393|5117|Friday|1998Q1|N|Y|N|2450815|2450814|2450472|2450745|N|N|N|N|N| +2450838|AAAAAAAAGJFGFCAA|1998-01-24|1176|5117|393|1998|6|1|24|1|1998|393|5117|Saturday|1998Q1|N|Y|N|2450815|2450814|2450473|2450746|N|N|N|N|N| +2450839|AAAAAAAAHJFGFCAA|1998-01-25|1176|5117|393|1998|0|1|25|1|1998|393|5117|Sunday|1998Q1|N|N|N|2450815|2450814|2450474|2450747|N|N|N|N|N| +2450840|AAAAAAAAIJFGFCAA|1998-01-26|1176|5117|393|1998|1|1|26|1|1998|393|5117|Monday|1998Q1|N|N|N|2450815|2450814|2450475|2450748|N|N|N|N|N| +2450841|AAAAAAAAJJFGFCAA|1998-01-27|1176|5118|393|1998|2|1|27|1|1998|393|5118|Tuesday|1998Q1|N|N|N|2450815|2450814|2450476|2450749|N|N|N|N|N| +2450842|AAAAAAAAKJFGFCAA|1998-01-28|1176|5118|393|1998|3|1|28|1|1998|393|5118|Wednesday|1998Q1|N|N|N|2450815|2450814|2450477|2450750|N|N|N|N|N| +2450843|AAAAAAAALJFGFCAA|1998-01-29|1176|5118|393|1998|4|1|29|1|1998|393|5118|Thursday|1998Q1|N|N|N|2450815|2450814|2450478|2450751|N|N|N|N|N| +2450844|AAAAAAAAMJFGFCAA|1998-01-30|1176|5118|393|1998|5|1|30|1|1998|393|5118|Friday|1998Q1|N|Y|N|2450815|2450814|2450479|2450752|N|N|N|N|N| +2450845|AAAAAAAANJFGFCAA|1998-01-31|1176|5118|393|1998|6|1|31|1|1998|393|5118|Saturday|1998Q1|N|Y|N|2450815|2450814|2450480|2450753|N|N|N|N|N| +2450846|AAAAAAAAOJFGFCAA|1998-02-01|1177|5118|393|1998|0|2|1|1|1998|393|5118|Sunday|1998Q1|N|N|N|2450846|2450876|2450481|2450754|N|N|N|N|N| +2450847|AAAAAAAAPJFGFCAA|1998-02-02|1177|5118|393|1998|1|2|2|1|1998|393|5118|Monday|1998Q1|N|N|N|2450846|2450876|2450482|2450755|N|N|N|N|N| +2450848|AAAAAAAAAKFGFCAA|1998-02-03|1177|5119|393|1998|2|2|3|1|1998|393|5119|Tuesday|1998Q1|N|N|N|2450846|2450876|2450483|2450756|N|N|N|N|N| +2450849|AAAAAAAABKFGFCAA|1998-02-04|1177|5119|393|1998|3|2|4|1|1998|393|5119|Wednesday|1998Q1|N|N|N|2450846|2450876|2450484|2450757|N|N|N|N|N| +2450850|AAAAAAAACKFGFCAA|1998-02-05|1177|5119|393|1998|4|2|5|1|1998|393|5119|Thursday|1998Q1|N|N|N|2450846|2450876|2450485|2450758|N|N|N|N|N| +2450851|AAAAAAAADKFGFCAA|1998-02-06|1177|5119|393|1998|5|2|6|1|1998|393|5119|Friday|1998Q1|N|Y|N|2450846|2450876|2450486|2450759|N|N|N|N|N| +2450852|AAAAAAAAEKFGFCAA|1998-02-07|1177|5119|393|1998|6|2|7|1|1998|393|5119|Saturday|1998Q1|N|Y|N|2450846|2450876|2450487|2450760|N|N|N|N|N| +2450853|AAAAAAAAFKFGFCAA|1998-02-08|1177|5119|393|1998|0|2|8|1|1998|393|5119|Sunday|1998Q1|N|N|N|2450846|2450876|2450488|2450761|N|N|N|N|N| +2450854|AAAAAAAAGKFGFCAA|1998-02-09|1177|5119|393|1998|1|2|9|1|1998|393|5119|Monday|1998Q1|N|N|N|2450846|2450876|2450489|2450762|N|N|N|N|N| +2450855|AAAAAAAAHKFGFCAA|1998-02-10|1177|5120|393|1998|2|2|10|1|1998|393|5120|Tuesday|1998Q1|N|N|N|2450846|2450876|2450490|2450763|N|N|N|N|N| +2450856|AAAAAAAAIKFGFCAA|1998-02-11|1177|5120|393|1998|3|2|11|1|1998|393|5120|Wednesday|1998Q1|N|N|N|2450846|2450876|2450491|2450764|N|N|N|N|N| +2450857|AAAAAAAAJKFGFCAA|1998-02-12|1177|5120|393|1998|4|2|12|1|1998|393|5120|Thursday|1998Q1|N|N|N|2450846|2450876|2450492|2450765|N|N|N|N|N| +2450858|AAAAAAAAKKFGFCAA|1998-02-13|1177|5120|393|1998|5|2|13|1|1998|393|5120|Friday|1998Q1|N|Y|N|2450846|2450876|2450493|2450766|N|N|N|N|N| +2450859|AAAAAAAALKFGFCAA|1998-02-14|1177|5120|393|1998|6|2|14|1|1998|393|5120|Saturday|1998Q1|N|Y|N|2450846|2450876|2450494|2450767|N|N|N|N|N| +2450860|AAAAAAAAMKFGFCAA|1998-02-15|1177|5120|393|1998|0|2|15|1|1998|393|5120|Sunday|1998Q1|N|N|N|2450846|2450876|2450495|2450768|N|N|N|N|N| +2450861|AAAAAAAANKFGFCAA|1998-02-16|1177|5120|393|1998|1|2|16|1|1998|393|5120|Monday|1998Q1|N|N|N|2450846|2450876|2450496|2450769|N|N|N|N|N| +2450862|AAAAAAAAOKFGFCAA|1998-02-17|1177|5121|393|1998|2|2|17|1|1998|393|5121|Tuesday|1998Q1|N|N|N|2450846|2450876|2450497|2450770|N|N|N|N|N| +2450863|AAAAAAAAPKFGFCAA|1998-02-18|1177|5121|393|1998|3|2|18|1|1998|393|5121|Wednesday|1998Q1|N|N|N|2450846|2450876|2450498|2450771|N|N|N|N|N| +2450864|AAAAAAAAALFGFCAA|1998-02-19|1177|5121|393|1998|4|2|19|1|1998|393|5121|Thursday|1998Q1|N|N|N|2450846|2450876|2450499|2450772|N|N|N|N|N| +2450865|AAAAAAAABLFGFCAA|1998-02-20|1177|5121|393|1998|5|2|20|1|1998|393|5121|Friday|1998Q1|N|Y|N|2450846|2450876|2450500|2450773|N|N|N|N|N| +2450866|AAAAAAAACLFGFCAA|1998-02-21|1177|5121|393|1998|6|2|21|1|1998|393|5121|Saturday|1998Q1|N|Y|N|2450846|2450876|2450501|2450774|N|N|N|N|N| +2450867|AAAAAAAADLFGFCAA|1998-02-22|1177|5121|393|1998|0|2|22|1|1998|393|5121|Sunday|1998Q1|N|N|N|2450846|2450876|2450502|2450775|N|N|N|N|N| +2450868|AAAAAAAAELFGFCAA|1998-02-23|1177|5121|393|1998|1|2|23|1|1998|393|5121|Monday|1998Q1|N|N|N|2450846|2450876|2450503|2450776|N|N|N|N|N| +2450869|AAAAAAAAFLFGFCAA|1998-02-24|1177|5122|393|1998|2|2|24|1|1998|393|5122|Tuesday|1998Q1|N|N|N|2450846|2450876|2450504|2450777|N|N|N|N|N| +2450870|AAAAAAAAGLFGFCAA|1998-02-25|1177|5122|393|1998|3|2|25|1|1998|393|5122|Wednesday|1998Q1|N|N|N|2450846|2450876|2450505|2450778|N|N|N|N|N| +2450871|AAAAAAAAHLFGFCAA|1998-02-26|1177|5122|393|1998|4|2|26|1|1998|393|5122|Thursday|1998Q1|N|N|N|2450846|2450876|2450506|2450779|N|N|N|N|N| +2450872|AAAAAAAAILFGFCAA|1998-02-27|1177|5122|393|1998|5|2|27|1|1998|393|5122|Friday|1998Q1|N|Y|N|2450846|2450876|2450507|2450780|N|N|N|N|N| +2450873|AAAAAAAAJLFGFCAA|1998-02-28|1177|5122|393|1998|6|2|28|1|1998|393|5122|Saturday|1998Q1|N|Y|N|2450846|2450876|2450508|2450781|N|N|N|N|N| +2450874|AAAAAAAAKLFGFCAA|1998-03-01|1178|5122|394|1998|0|3|1|1|1998|394|5122|Sunday|1998Q1|N|N|N|2450874|2450932|2450509|2450782|N|N|N|N|N| +2450875|AAAAAAAALLFGFCAA|1998-03-02|1178|5122|394|1998|1|3|2|1|1998|394|5122|Monday|1998Q1|N|N|N|2450874|2450932|2450510|2450783|N|N|N|N|N| +2450876|AAAAAAAAMLFGFCAA|1998-03-03|1178|5123|394|1998|2|3|3|1|1998|394|5123|Tuesday|1998Q1|N|N|N|2450874|2450932|2450511|2450784|N|N|N|N|N| +2450877|AAAAAAAANLFGFCAA|1998-03-04|1178|5123|394|1998|3|3|4|1|1998|394|5123|Wednesday|1998Q1|N|N|N|2450874|2450932|2450512|2450785|N|N|N|N|N| +2450878|AAAAAAAAOLFGFCAA|1998-03-05|1178|5123|394|1998|4|3|5|1|1998|394|5123|Thursday|1998Q1|N|N|N|2450874|2450932|2450513|2450786|N|N|N|N|N| +2450879|AAAAAAAAPLFGFCAA|1998-03-06|1178|5123|394|1998|5|3|6|1|1998|394|5123|Friday|1998Q1|N|Y|N|2450874|2450932|2450514|2450787|N|N|N|N|N| +2450880|AAAAAAAAAMFGFCAA|1998-03-07|1178|5123|394|1998|6|3|7|1|1998|394|5123|Saturday|1998Q1|N|Y|N|2450874|2450932|2450515|2450788|N|N|N|N|N| +2450881|AAAAAAAABMFGFCAA|1998-03-08|1178|5123|394|1998|0|3|8|1|1998|394|5123|Sunday|1998Q1|N|N|N|2450874|2450932|2450516|2450789|N|N|N|N|N| +2450882|AAAAAAAACMFGFCAA|1998-03-09|1178|5123|394|1998|1|3|9|1|1998|394|5123|Monday|1998Q1|N|N|N|2450874|2450932|2450517|2450790|N|N|N|N|N| +2450883|AAAAAAAADMFGFCAA|1998-03-10|1178|5124|394|1998|2|3|10|1|1998|394|5124|Tuesday|1998Q1|N|N|N|2450874|2450932|2450518|2450791|N|N|N|N|N| +2450884|AAAAAAAAEMFGFCAA|1998-03-11|1178|5124|394|1998|3|3|11|1|1998|394|5124|Wednesday|1998Q1|N|N|N|2450874|2450932|2450519|2450792|N|N|N|N|N| +2450885|AAAAAAAAFMFGFCAA|1998-03-12|1178|5124|394|1998|4|3|12|1|1998|394|5124|Thursday|1998Q1|N|N|N|2450874|2450932|2450520|2450793|N|N|N|N|N| +2450886|AAAAAAAAGMFGFCAA|1998-03-13|1178|5124|394|1998|5|3|13|1|1998|394|5124|Friday|1998Q1|N|Y|N|2450874|2450932|2450521|2450794|N|N|N|N|N| +2450887|AAAAAAAAHMFGFCAA|1998-03-14|1178|5124|394|1998|6|3|14|1|1998|394|5124|Saturday|1998Q1|N|Y|N|2450874|2450932|2450522|2450795|N|N|N|N|N| +2450888|AAAAAAAAIMFGFCAA|1998-03-15|1178|5124|394|1998|0|3|15|1|1998|394|5124|Sunday|1998Q1|N|N|N|2450874|2450932|2450523|2450796|N|N|N|N|N| +2450889|AAAAAAAAJMFGFCAA|1998-03-16|1178|5124|394|1998|1|3|16|1|1998|394|5124|Monday|1998Q1|N|N|N|2450874|2450932|2450524|2450797|N|N|N|N|N| +2450890|AAAAAAAAKMFGFCAA|1998-03-17|1178|5125|394|1998|2|3|17|1|1998|394|5125|Tuesday|1998Q1|N|N|N|2450874|2450932|2450525|2450798|N|N|N|N|N| +2450891|AAAAAAAALMFGFCAA|1998-03-18|1178|5125|394|1998|3|3|18|1|1998|394|5125|Wednesday|1998Q1|N|N|N|2450874|2450932|2450526|2450799|N|N|N|N|N| +2450892|AAAAAAAAMMFGFCAA|1998-03-19|1178|5125|394|1998|4|3|19|1|1998|394|5125|Thursday|1998Q1|N|N|N|2450874|2450932|2450527|2450800|N|N|N|N|N| +2450893|AAAAAAAANMFGFCAA|1998-03-20|1178|5125|394|1998|5|3|20|1|1998|394|5125|Friday|1998Q1|N|Y|N|2450874|2450932|2450528|2450801|N|N|N|N|N| +2450894|AAAAAAAAOMFGFCAA|1998-03-21|1178|5125|394|1998|6|3|21|1|1998|394|5125|Saturday|1998Q1|N|Y|N|2450874|2450932|2450529|2450802|N|N|N|N|N| +2450895|AAAAAAAAPMFGFCAA|1998-03-22|1178|5125|394|1998|0|3|22|1|1998|394|5125|Sunday|1998Q1|N|N|N|2450874|2450932|2450530|2450803|N|N|N|N|N| +2450896|AAAAAAAAANFGFCAA|1998-03-23|1178|5125|394|1998|1|3|23|1|1998|394|5125|Monday|1998Q1|N|N|N|2450874|2450932|2450531|2450804|N|N|N|N|N| +2450897|AAAAAAAABNFGFCAA|1998-03-24|1178|5126|394|1998|2|3|24|1|1998|394|5126|Tuesday|1998Q1|N|N|N|2450874|2450932|2450532|2450805|N|N|N|N|N| +2450898|AAAAAAAACNFGFCAA|1998-03-25|1178|5126|394|1998|3|3|25|1|1998|394|5126|Wednesday|1998Q1|N|N|N|2450874|2450932|2450533|2450806|N|N|N|N|N| +2450899|AAAAAAAADNFGFCAA|1998-03-26|1178|5126|394|1998|4|3|26|1|1998|394|5126|Thursday|1998Q1|N|N|N|2450874|2450932|2450534|2450807|N|N|N|N|N| +2450900|AAAAAAAAENFGFCAA|1998-03-27|1178|5126|394|1998|5|3|27|1|1998|394|5126|Friday|1998Q1|N|Y|N|2450874|2450932|2450535|2450808|N|N|N|N|N| +2450901|AAAAAAAAFNFGFCAA|1998-03-28|1178|5126|394|1998|6|3|28|1|1998|394|5126|Saturday|1998Q1|N|Y|N|2450874|2450932|2450536|2450809|N|N|N|N|N| +2450902|AAAAAAAAGNFGFCAA|1998-03-29|1178|5126|394|1998|0|3|29|1|1998|394|5126|Sunday|1998Q1|N|N|N|2450874|2450932|2450537|2450810|N|N|N|N|N| +2450903|AAAAAAAAHNFGFCAA|1998-03-30|1178|5126|394|1998|1|3|30|1|1998|394|5126|Monday|1998Q1|N|N|N|2450874|2450932|2450538|2450811|N|N|N|N|N| +2450904|AAAAAAAAINFGFCAA|1998-03-31|1178|5127|394|1998|2|3|31|1|1998|394|5127|Tuesday|1998Q1|N|N|N|2450874|2450932|2450539|2450812|N|N|N|N|N| +2450905|AAAAAAAAJNFGFCAA|1998-04-01|1179|5127|394|1998|3|4|1|1|1998|394|5127|Wednesday|1998Q1|N|N|N|2450905|2450994|2450540|2450815|N|N|N|N|N| +2450906|AAAAAAAAKNFGFCAA|1998-04-02|1179|5127|394|1998|4|4|2|2|1998|394|5127|Thursday|1998Q2|N|N|N|2450905|2450994|2450541|2450816|N|N|N|N|N| +2450907|AAAAAAAALNFGFCAA|1998-04-03|1179|5127|394|1998|5|4|3|2|1998|394|5127|Friday|1998Q2|N|Y|N|2450905|2450994|2450542|2450817|N|N|N|N|N| +2450908|AAAAAAAAMNFGFCAA|1998-04-04|1179|5127|394|1998|6|4|4|2|1998|394|5127|Saturday|1998Q2|N|Y|N|2450905|2450994|2450543|2450818|N|N|N|N|N| +2450909|AAAAAAAANNFGFCAA|1998-04-05|1179|5127|394|1998|0|4|5|2|1998|394|5127|Sunday|1998Q2|N|N|N|2450905|2450994|2450544|2450819|N|N|N|N|N| +2450910|AAAAAAAAONFGFCAA|1998-04-06|1179|5127|394|1998|1|4|6|2|1998|394|5127|Monday|1998Q2|N|N|N|2450905|2450994|2450545|2450820|N|N|N|N|N| +2450911|AAAAAAAAPNFGFCAA|1998-04-07|1179|5128|394|1998|2|4|7|2|1998|394|5128|Tuesday|1998Q2|N|N|N|2450905|2450994|2450546|2450821|N|N|N|N|N| +2450912|AAAAAAAAAOFGFCAA|1998-04-08|1179|5128|394|1998|3|4|8|2|1998|394|5128|Wednesday|1998Q2|N|N|N|2450905|2450994|2450547|2450822|N|N|N|N|N| +2450913|AAAAAAAABOFGFCAA|1998-04-09|1179|5128|394|1998|4|4|9|2|1998|394|5128|Thursday|1998Q2|N|N|N|2450905|2450994|2450548|2450823|N|N|N|N|N| +2450914|AAAAAAAACOFGFCAA|1998-04-10|1179|5128|394|1998|5|4|10|2|1998|394|5128|Friday|1998Q2|N|Y|N|2450905|2450994|2450549|2450824|N|N|N|N|N| +2450915|AAAAAAAADOFGFCAA|1998-04-11|1179|5128|394|1998|6|4|11|2|1998|394|5128|Saturday|1998Q2|N|Y|N|2450905|2450994|2450550|2450825|N|N|N|N|N| +2450916|AAAAAAAAEOFGFCAA|1998-04-12|1179|5128|394|1998|0|4|12|2|1998|394|5128|Sunday|1998Q2|N|N|N|2450905|2450994|2450551|2450826|N|N|N|N|N| +2450917|AAAAAAAAFOFGFCAA|1998-04-13|1179|5128|394|1998|1|4|13|2|1998|394|5128|Monday|1998Q2|N|N|N|2450905|2450994|2450552|2450827|N|N|N|N|N| +2450918|AAAAAAAAGOFGFCAA|1998-04-14|1179|5129|394|1998|2|4|14|2|1998|394|5129|Tuesday|1998Q2|N|N|N|2450905|2450994|2450553|2450828|N|N|N|N|N| +2450919|AAAAAAAAHOFGFCAA|1998-04-15|1179|5129|394|1998|3|4|15|2|1998|394|5129|Wednesday|1998Q2|N|N|N|2450905|2450994|2450554|2450829|N|N|N|N|N| +2450920|AAAAAAAAIOFGFCAA|1998-04-16|1179|5129|394|1998|4|4|16|2|1998|394|5129|Thursday|1998Q2|N|N|N|2450905|2450994|2450555|2450830|N|N|N|N|N| +2450921|AAAAAAAAJOFGFCAA|1998-04-17|1179|5129|394|1998|5|4|17|2|1998|394|5129|Friday|1998Q2|N|Y|N|2450905|2450994|2450556|2450831|N|N|N|N|N| +2450922|AAAAAAAAKOFGFCAA|1998-04-18|1179|5129|394|1998|6|4|18|2|1998|394|5129|Saturday|1998Q2|N|Y|N|2450905|2450994|2450557|2450832|N|N|N|N|N| +2450923|AAAAAAAALOFGFCAA|1998-04-19|1179|5129|394|1998|0|4|19|2|1998|394|5129|Sunday|1998Q2|N|N|N|2450905|2450994|2450558|2450833|N|N|N|N|N| +2450924|AAAAAAAAMOFGFCAA|1998-04-20|1179|5129|394|1998|1|4|20|2|1998|394|5129|Monday|1998Q2|N|N|N|2450905|2450994|2450559|2450834|N|N|N|N|N| +2450925|AAAAAAAANOFGFCAA|1998-04-21|1179|5130|394|1998|2|4|21|2|1998|394|5130|Tuesday|1998Q2|N|N|N|2450905|2450994|2450560|2450835|N|N|N|N|N| +2450926|AAAAAAAAOOFGFCAA|1998-04-22|1179|5130|394|1998|3|4|22|2|1998|394|5130|Wednesday|1998Q2|N|N|N|2450905|2450994|2450561|2450836|N|N|N|N|N| +2450927|AAAAAAAAPOFGFCAA|1998-04-23|1179|5130|394|1998|4|4|23|2|1998|394|5130|Thursday|1998Q2|N|N|N|2450905|2450994|2450562|2450837|N|N|N|N|N| +2450928|AAAAAAAAAPFGFCAA|1998-04-24|1179|5130|394|1998|5|4|24|2|1998|394|5130|Friday|1998Q2|N|Y|N|2450905|2450994|2450563|2450838|N|N|N|N|N| +2450929|AAAAAAAABPFGFCAA|1998-04-25|1179|5130|394|1998|6|4|25|2|1998|394|5130|Saturday|1998Q2|N|Y|N|2450905|2450994|2450564|2450839|N|N|N|N|N| +2450930|AAAAAAAACPFGFCAA|1998-04-26|1179|5130|394|1998|0|4|26|2|1998|394|5130|Sunday|1998Q2|N|N|N|2450905|2450994|2450565|2450840|N|N|N|N|N| +2450931|AAAAAAAADPFGFCAA|1998-04-27|1179|5130|394|1998|1|4|27|2|1998|394|5130|Monday|1998Q2|N|N|N|2450905|2450994|2450566|2450841|N|N|N|N|N| +2450932|AAAAAAAAEPFGFCAA|1998-04-28|1179|5131|394|1998|2|4|28|2|1998|394|5131|Tuesday|1998Q2|N|N|N|2450905|2450994|2450567|2450842|N|N|N|N|N| +2450933|AAAAAAAAFPFGFCAA|1998-04-29|1179|5131|394|1998|3|4|29|2|1998|394|5131|Wednesday|1998Q2|N|N|N|2450905|2450994|2450568|2450843|N|N|N|N|N| +2450934|AAAAAAAAGPFGFCAA|1998-04-30|1179|5131|394|1998|4|4|30|2|1998|394|5131|Thursday|1998Q2|N|N|N|2450905|2450994|2450569|2450844|N|N|N|N|N| +2450935|AAAAAAAAHPFGFCAA|1998-05-01|1180|5131|394|1998|5|5|1|2|1998|394|5131|Friday|1998Q2|N|Y|N|2450935|2451054|2450570|2450845|N|N|N|N|N| +2450936|AAAAAAAAIPFGFCAA|1998-05-02|1180|5131|394|1998|6|5|2|2|1998|394|5131|Saturday|1998Q2|N|Y|N|2450935|2451054|2450571|2450846|N|N|N|N|N| +2450937|AAAAAAAAJPFGFCAA|1998-05-03|1180|5131|394|1998|0|5|3|2|1998|394|5131|Sunday|1998Q2|N|N|N|2450935|2451054|2450572|2450847|N|N|N|N|N| +2450938|AAAAAAAAKPFGFCAA|1998-05-04|1180|5131|394|1998|1|5|4|2|1998|394|5131|Monday|1998Q2|N|N|N|2450935|2451054|2450573|2450848|N|N|N|N|N| +2450939|AAAAAAAALPFGFCAA|1998-05-05|1180|5132|394|1998|2|5|5|2|1998|394|5132|Tuesday|1998Q2|N|N|N|2450935|2451054|2450574|2450849|N|N|N|N|N| +2450940|AAAAAAAAMPFGFCAA|1998-05-06|1180|5132|394|1998|3|5|6|2|1998|394|5132|Wednesday|1998Q2|N|N|N|2450935|2451054|2450575|2450850|N|N|N|N|N| +2450941|AAAAAAAANPFGFCAA|1998-05-07|1180|5132|394|1998|4|5|7|2|1998|394|5132|Thursday|1998Q2|N|N|N|2450935|2451054|2450576|2450851|N|N|N|N|N| +2450942|AAAAAAAAOPFGFCAA|1998-05-08|1180|5132|394|1998|5|5|8|2|1998|394|5132|Friday|1998Q2|N|Y|N|2450935|2451054|2450577|2450852|N|N|N|N|N| +2450943|AAAAAAAAPPFGFCAA|1998-05-09|1180|5132|394|1998|6|5|9|2|1998|394|5132|Saturday|1998Q2|N|Y|N|2450935|2451054|2450578|2450853|N|N|N|N|N| +2450944|AAAAAAAAAAGGFCAA|1998-05-10|1180|5132|394|1998|0|5|10|2|1998|394|5132|Sunday|1998Q2|N|N|N|2450935|2451054|2450579|2450854|N|N|N|N|N| +2450945|AAAAAAAABAGGFCAA|1998-05-11|1180|5132|394|1998|1|5|11|2|1998|394|5132|Monday|1998Q2|N|N|N|2450935|2451054|2450580|2450855|N|N|N|N|N| +2450946|AAAAAAAACAGGFCAA|1998-05-12|1180|5133|394|1998|2|5|12|2|1998|394|5133|Tuesday|1998Q2|N|N|N|2450935|2451054|2450581|2450856|N|N|N|N|N| +2450947|AAAAAAAADAGGFCAA|1998-05-13|1180|5133|394|1998|3|5|13|2|1998|394|5133|Wednesday|1998Q2|N|N|N|2450935|2451054|2450582|2450857|N|N|N|N|N| +2450948|AAAAAAAAEAGGFCAA|1998-05-14|1180|5133|394|1998|4|5|14|2|1998|394|5133|Thursday|1998Q2|N|N|N|2450935|2451054|2450583|2450858|N|N|N|N|N| +2450949|AAAAAAAAFAGGFCAA|1998-05-15|1180|5133|394|1998|5|5|15|2|1998|394|5133|Friday|1998Q2|N|Y|N|2450935|2451054|2450584|2450859|N|N|N|N|N| +2450950|AAAAAAAAGAGGFCAA|1998-05-16|1180|5133|394|1998|6|5|16|2|1998|394|5133|Saturday|1998Q2|N|Y|N|2450935|2451054|2450585|2450860|N|N|N|N|N| +2450951|AAAAAAAAHAGGFCAA|1998-05-17|1180|5133|394|1998|0|5|17|2|1998|394|5133|Sunday|1998Q2|N|N|N|2450935|2451054|2450586|2450861|N|N|N|N|N| +2450952|AAAAAAAAIAGGFCAA|1998-05-18|1180|5133|394|1998|1|5|18|2|1998|394|5133|Monday|1998Q2|N|N|N|2450935|2451054|2450587|2450862|N|N|N|N|N| +2450953|AAAAAAAAJAGGFCAA|1998-05-19|1180|5134|394|1998|2|5|19|2|1998|394|5134|Tuesday|1998Q2|N|N|N|2450935|2451054|2450588|2450863|N|N|N|N|N| +2450954|AAAAAAAAKAGGFCAA|1998-05-20|1180|5134|394|1998|3|5|20|2|1998|394|5134|Wednesday|1998Q2|N|N|N|2450935|2451054|2450589|2450864|N|N|N|N|N| +2450955|AAAAAAAALAGGFCAA|1998-05-21|1180|5134|394|1998|4|5|21|2|1998|394|5134|Thursday|1998Q2|N|N|N|2450935|2451054|2450590|2450865|N|N|N|N|N| +2450956|AAAAAAAAMAGGFCAA|1998-05-22|1180|5134|394|1998|5|5|22|2|1998|394|5134|Friday|1998Q2|N|Y|N|2450935|2451054|2450591|2450866|N|N|N|N|N| +2450957|AAAAAAAANAGGFCAA|1998-05-23|1180|5134|394|1998|6|5|23|2|1998|394|5134|Saturday|1998Q2|N|Y|N|2450935|2451054|2450592|2450867|N|N|N|N|N| +2450958|AAAAAAAAOAGGFCAA|1998-05-24|1180|5134|394|1998|0|5|24|2|1998|394|5134|Sunday|1998Q2|N|N|N|2450935|2451054|2450593|2450868|N|N|N|N|N| +2450959|AAAAAAAAPAGGFCAA|1998-05-25|1180|5134|394|1998|1|5|25|2|1998|394|5134|Monday|1998Q2|N|N|N|2450935|2451054|2450594|2450869|N|N|N|N|N| +2450960|AAAAAAAAABGGFCAA|1998-05-26|1180|5135|394|1998|2|5|26|2|1998|394|5135|Tuesday|1998Q2|N|N|N|2450935|2451054|2450595|2450870|N|N|N|N|N| +2450961|AAAAAAAABBGGFCAA|1998-05-27|1180|5135|394|1998|3|5|27|2|1998|394|5135|Wednesday|1998Q2|N|N|N|2450935|2451054|2450596|2450871|N|N|N|N|N| +2450962|AAAAAAAACBGGFCAA|1998-05-28|1180|5135|394|1998|4|5|28|2|1998|394|5135|Thursday|1998Q2|N|N|N|2450935|2451054|2450597|2450872|N|N|N|N|N| +2450963|AAAAAAAADBGGFCAA|1998-05-29|1180|5135|394|1998|5|5|29|2|1998|394|5135|Friday|1998Q2|N|Y|N|2450935|2451054|2450598|2450873|N|N|N|N|N| +2450964|AAAAAAAAEBGGFCAA|1998-05-30|1180|5135|394|1998|6|5|30|2|1998|394|5135|Saturday|1998Q2|N|Y|N|2450935|2451054|2450599|2450874|N|N|N|N|N| +2450965|AAAAAAAAFBGGFCAA|1998-05-31|1180|5135|394|1998|0|5|31|2|1998|394|5135|Sunday|1998Q2|N|N|N|2450935|2451054|2450600|2450875|N|N|N|N|N| +2450966|AAAAAAAAGBGGFCAA|1998-06-01|1181|5135|395|1998|1|6|1|2|1998|395|5135|Monday|1998Q2|N|N|N|2450966|2451116|2450601|2450876|N|N|N|N|N| +2450967|AAAAAAAAHBGGFCAA|1998-06-02|1181|5136|395|1998|2|6|2|2|1998|395|5136|Tuesday|1998Q2|N|N|N|2450966|2451116|2450602|2450877|N|N|N|N|N| +2450968|AAAAAAAAIBGGFCAA|1998-06-03|1181|5136|395|1998|3|6|3|2|1998|395|5136|Wednesday|1998Q2|N|N|N|2450966|2451116|2450603|2450878|N|N|N|N|N| +2450969|AAAAAAAAJBGGFCAA|1998-06-04|1181|5136|395|1998|4|6|4|2|1998|395|5136|Thursday|1998Q2|N|N|N|2450966|2451116|2450604|2450879|N|N|N|N|N| +2450970|AAAAAAAAKBGGFCAA|1998-06-05|1181|5136|395|1998|5|6|5|2|1998|395|5136|Friday|1998Q2|N|Y|N|2450966|2451116|2450605|2450880|N|N|N|N|N| +2450971|AAAAAAAALBGGFCAA|1998-06-06|1181|5136|395|1998|6|6|6|2|1998|395|5136|Saturday|1998Q2|N|Y|N|2450966|2451116|2450606|2450881|N|N|N|N|N| +2450972|AAAAAAAAMBGGFCAA|1998-06-07|1181|5136|395|1998|0|6|7|2|1998|395|5136|Sunday|1998Q2|N|N|N|2450966|2451116|2450607|2450882|N|N|N|N|N| +2450973|AAAAAAAANBGGFCAA|1998-06-08|1181|5136|395|1998|1|6|8|2|1998|395|5136|Monday|1998Q2|N|N|N|2450966|2451116|2450608|2450883|N|N|N|N|N| +2450974|AAAAAAAAOBGGFCAA|1998-06-09|1181|5137|395|1998|2|6|9|2|1998|395|5137|Tuesday|1998Q2|N|N|N|2450966|2451116|2450609|2450884|N|N|N|N|N| +2450975|AAAAAAAAPBGGFCAA|1998-06-10|1181|5137|395|1998|3|6|10|2|1998|395|5137|Wednesday|1998Q2|N|N|N|2450966|2451116|2450610|2450885|N|N|N|N|N| +2450976|AAAAAAAAACGGFCAA|1998-06-11|1181|5137|395|1998|4|6|11|2|1998|395|5137|Thursday|1998Q2|N|N|N|2450966|2451116|2450611|2450886|N|N|N|N|N| +2450977|AAAAAAAABCGGFCAA|1998-06-12|1181|5137|395|1998|5|6|12|2|1998|395|5137|Friday|1998Q2|N|Y|N|2450966|2451116|2450612|2450887|N|N|N|N|N| +2450978|AAAAAAAACCGGFCAA|1998-06-13|1181|5137|395|1998|6|6|13|2|1998|395|5137|Saturday|1998Q2|N|Y|N|2450966|2451116|2450613|2450888|N|N|N|N|N| +2450979|AAAAAAAADCGGFCAA|1998-06-14|1181|5137|395|1998|0|6|14|2|1998|395|5137|Sunday|1998Q2|N|N|N|2450966|2451116|2450614|2450889|N|N|N|N|N| +2450980|AAAAAAAAECGGFCAA|1998-06-15|1181|5137|395|1998|1|6|15|2|1998|395|5137|Monday|1998Q2|N|N|N|2450966|2451116|2450615|2450890|N|N|N|N|N| +2450981|AAAAAAAAFCGGFCAA|1998-06-16|1181|5138|395|1998|2|6|16|2|1998|395|5138|Tuesday|1998Q2|N|N|N|2450966|2451116|2450616|2450891|N|N|N|N|N| +2450982|AAAAAAAAGCGGFCAA|1998-06-17|1181|5138|395|1998|3|6|17|2|1998|395|5138|Wednesday|1998Q2|N|N|N|2450966|2451116|2450617|2450892|N|N|N|N|N| +2450983|AAAAAAAAHCGGFCAA|1998-06-18|1181|5138|395|1998|4|6|18|2|1998|395|5138|Thursday|1998Q2|N|N|N|2450966|2451116|2450618|2450893|N|N|N|N|N| +2450984|AAAAAAAAICGGFCAA|1998-06-19|1181|5138|395|1998|5|6|19|2|1998|395|5138|Friday|1998Q2|N|Y|N|2450966|2451116|2450619|2450894|N|N|N|N|N| +2450985|AAAAAAAAJCGGFCAA|1998-06-20|1181|5138|395|1998|6|6|20|2|1998|395|5138|Saturday|1998Q2|N|Y|N|2450966|2451116|2450620|2450895|N|N|N|N|N| +2450986|AAAAAAAAKCGGFCAA|1998-06-21|1181|5138|395|1998|0|6|21|2|1998|395|5138|Sunday|1998Q2|N|N|N|2450966|2451116|2450621|2450896|N|N|N|N|N| +2450987|AAAAAAAALCGGFCAA|1998-06-22|1181|5138|395|1998|1|6|22|2|1998|395|5138|Monday|1998Q2|N|N|N|2450966|2451116|2450622|2450897|N|N|N|N|N| +2450988|AAAAAAAAMCGGFCAA|1998-06-23|1181|5139|395|1998|2|6|23|2|1998|395|5139|Tuesday|1998Q2|N|N|N|2450966|2451116|2450623|2450898|N|N|N|N|N| +2450989|AAAAAAAANCGGFCAA|1998-06-24|1181|5139|395|1998|3|6|24|2|1998|395|5139|Wednesday|1998Q2|N|N|N|2450966|2451116|2450624|2450899|N|N|N|N|N| +2450990|AAAAAAAAOCGGFCAA|1998-06-25|1181|5139|395|1998|4|6|25|2|1998|395|5139|Thursday|1998Q2|N|N|N|2450966|2451116|2450625|2450900|N|N|N|N|N| +2450991|AAAAAAAAPCGGFCAA|1998-06-26|1181|5139|395|1998|5|6|26|2|1998|395|5139|Friday|1998Q2|N|Y|N|2450966|2451116|2450626|2450901|N|N|N|N|N| +2450992|AAAAAAAAADGGFCAA|1998-06-27|1181|5139|395|1998|6|6|27|2|1998|395|5139|Saturday|1998Q2|N|Y|N|2450966|2451116|2450627|2450902|N|N|N|N|N| +2450993|AAAAAAAABDGGFCAA|1998-06-28|1181|5139|395|1998|0|6|28|2|1998|395|5139|Sunday|1998Q2|N|N|N|2450966|2451116|2450628|2450903|N|N|N|N|N| +2450994|AAAAAAAACDGGFCAA|1998-06-29|1181|5139|395|1998|1|6|29|2|1998|395|5139|Monday|1998Q2|N|N|N|2450966|2451116|2450629|2450904|N|N|N|N|N| +2450995|AAAAAAAADDGGFCAA|1998-06-30|1181|5140|395|1998|2|6|30|2|1998|395|5140|Tuesday|1998Q2|N|N|N|2450966|2451116|2450630|2450905|N|N|N|N|N| +2450996|AAAAAAAAEDGGFCAA|1998-07-01|1182|5140|395|1998|3|7|1|2|1998|395|5140|Wednesday|1998Q2|N|N|N|2450996|2451176|2450631|2450905|N|N|N|N|N| +2450997|AAAAAAAAFDGGFCAA|1998-07-02|1182|5140|395|1998|4|7|2|3|1998|395|5140|Thursday|1998Q3|N|N|N|2450996|2451176|2450632|2450906|N|N|N|N|N| +2450998|AAAAAAAAGDGGFCAA|1998-07-03|1182|5140|395|1998|5|7|3|3|1998|395|5140|Friday|1998Q3|N|Y|N|2450996|2451176|2450633|2450907|N|N|N|N|N| +2450999|AAAAAAAAHDGGFCAA|1998-07-04|1182|5140|395|1998|6|7|4|3|1998|395|5140|Saturday|1998Q3|N|Y|N|2450996|2451176|2450634|2450908|N|N|N|N|N| +2451000|AAAAAAAAIDGGFCAA|1998-07-05|1182|5140|395|1998|0|7|5|3|1998|395|5140|Sunday|1998Q3|Y|N|N|2450996|2451176|2450635|2450909|N|N|N|N|N| +2451001|AAAAAAAAJDGGFCAA|1998-07-06|1182|5140|395|1998|1|7|6|3|1998|395|5140|Monday|1998Q3|N|N|Y|2450996|2451176|2450636|2450910|N|N|N|N|N| +2451002|AAAAAAAAKDGGFCAA|1998-07-07|1182|5141|395|1998|2|7|7|3|1998|395|5141|Tuesday|1998Q3|N|N|N|2450996|2451176|2450637|2450911|N|N|N|N|N| +2451003|AAAAAAAALDGGFCAA|1998-07-08|1182|5141|395|1998|3|7|8|3|1998|395|5141|Wednesday|1998Q3|N|N|N|2450996|2451176|2450638|2450912|N|N|N|N|N| +2451004|AAAAAAAAMDGGFCAA|1998-07-09|1182|5141|395|1998|4|7|9|3|1998|395|5141|Thursday|1998Q3|N|N|N|2450996|2451176|2450639|2450913|N|N|N|N|N| +2451005|AAAAAAAANDGGFCAA|1998-07-10|1182|5141|395|1998|5|7|10|3|1998|395|5141|Friday|1998Q3|N|Y|N|2450996|2451176|2450640|2450914|N|N|N|N|N| +2451006|AAAAAAAAODGGFCAA|1998-07-11|1182|5141|395|1998|6|7|11|3|1998|395|5141|Saturday|1998Q3|N|Y|N|2450996|2451176|2450641|2450915|N|N|N|N|N| +2451007|AAAAAAAAPDGGFCAA|1998-07-12|1182|5141|395|1998|0|7|12|3|1998|395|5141|Sunday|1998Q3|N|N|N|2450996|2451176|2450642|2450916|N|N|N|N|N| +2451008|AAAAAAAAAEGGFCAA|1998-07-13|1182|5141|395|1998|1|7|13|3|1998|395|5141|Monday|1998Q3|N|N|N|2450996|2451176|2450643|2450917|N|N|N|N|N| +2451009|AAAAAAAABEGGFCAA|1998-07-14|1182|5142|395|1998|2|7|14|3|1998|395|5142|Tuesday|1998Q3|N|N|N|2450996|2451176|2450644|2450918|N|N|N|N|N| +2451010|AAAAAAAACEGGFCAA|1998-07-15|1182|5142|395|1998|3|7|15|3|1998|395|5142|Wednesday|1998Q3|N|N|N|2450996|2451176|2450645|2450919|N|N|N|N|N| +2451011|AAAAAAAADEGGFCAA|1998-07-16|1182|5142|395|1998|4|7|16|3|1998|395|5142|Thursday|1998Q3|N|N|N|2450996|2451176|2450646|2450920|N|N|N|N|N| +2451012|AAAAAAAAEEGGFCAA|1998-07-17|1182|5142|395|1998|5|7|17|3|1998|395|5142|Friday|1998Q3|N|Y|N|2450996|2451176|2450647|2450921|N|N|N|N|N| +2451013|AAAAAAAAFEGGFCAA|1998-07-18|1182|5142|395|1998|6|7|18|3|1998|395|5142|Saturday|1998Q3|N|Y|N|2450996|2451176|2450648|2450922|N|N|N|N|N| +2451014|AAAAAAAAGEGGFCAA|1998-07-19|1182|5142|395|1998|0|7|19|3|1998|395|5142|Sunday|1998Q3|N|N|N|2450996|2451176|2450649|2450923|N|N|N|N|N| +2451015|AAAAAAAAHEGGFCAA|1998-07-20|1182|5142|395|1998|1|7|20|3|1998|395|5142|Monday|1998Q3|N|N|N|2450996|2451176|2450650|2450924|N|N|N|N|N| +2451016|AAAAAAAAIEGGFCAA|1998-07-21|1182|5143|395|1998|2|7|21|3|1998|395|5143|Tuesday|1998Q3|N|N|N|2450996|2451176|2450651|2450925|N|N|N|N|N| +2451017|AAAAAAAAJEGGFCAA|1998-07-22|1182|5143|395|1998|3|7|22|3|1998|395|5143|Wednesday|1998Q3|N|N|N|2450996|2451176|2450652|2450926|N|N|N|N|N| +2451018|AAAAAAAAKEGGFCAA|1998-07-23|1182|5143|395|1998|4|7|23|3|1998|395|5143|Thursday|1998Q3|N|N|N|2450996|2451176|2450653|2450927|N|N|N|N|N| +2451019|AAAAAAAALEGGFCAA|1998-07-24|1182|5143|395|1998|5|7|24|3|1998|395|5143|Friday|1998Q3|N|Y|N|2450996|2451176|2450654|2450928|N|N|N|N|N| +2451020|AAAAAAAAMEGGFCAA|1998-07-25|1182|5143|395|1998|6|7|25|3|1998|395|5143|Saturday|1998Q3|N|Y|N|2450996|2451176|2450655|2450929|N|N|N|N|N| +2451021|AAAAAAAANEGGFCAA|1998-07-26|1182|5143|395|1998|0|7|26|3|1998|395|5143|Sunday|1998Q3|N|N|N|2450996|2451176|2450656|2450930|N|N|N|N|N| +2451022|AAAAAAAAOEGGFCAA|1998-07-27|1182|5143|395|1998|1|7|27|3|1998|395|5143|Monday|1998Q3|N|N|N|2450996|2451176|2450657|2450931|N|N|N|N|N| +2451023|AAAAAAAAPEGGFCAA|1998-07-28|1182|5144|395|1998|2|7|28|3|1998|395|5144|Tuesday|1998Q3|N|N|N|2450996|2451176|2450658|2450932|N|N|N|N|N| +2451024|AAAAAAAAAFGGFCAA|1998-07-29|1182|5144|395|1998|3|7|29|3|1998|395|5144|Wednesday|1998Q3|N|N|N|2450996|2451176|2450659|2450933|N|N|N|N|N| +2451025|AAAAAAAABFGGFCAA|1998-07-30|1182|5144|395|1998|4|7|30|3|1998|395|5144|Thursday|1998Q3|N|N|N|2450996|2451176|2450660|2450934|N|N|N|N|N| +2451026|AAAAAAAACFGGFCAA|1998-07-31|1182|5144|395|1998|5|7|31|3|1998|395|5144|Friday|1998Q3|N|Y|N|2450996|2451176|2450661|2450935|N|N|N|N|N| +2451027|AAAAAAAADFGGFCAA|1998-08-01|1183|5144|395|1998|6|8|1|3|1998|395|5144|Saturday|1998Q3|N|Y|N|2451027|2451238|2450662|2450936|N|N|N|N|N| +2451028|AAAAAAAAEFGGFCAA|1998-08-02|1183|5144|395|1998|0|8|2|3|1998|395|5144|Sunday|1998Q3|N|N|N|2451027|2451238|2450663|2450937|N|N|N|N|N| +2451029|AAAAAAAAFFGGFCAA|1998-08-03|1183|5144|395|1998|1|8|3|3|1998|395|5144|Monday|1998Q3|N|N|N|2451027|2451238|2450664|2450938|N|N|N|N|N| +2451030|AAAAAAAAGFGGFCAA|1998-08-04|1183|5145|395|1998|2|8|4|3|1998|395|5145|Tuesday|1998Q3|N|N|N|2451027|2451238|2450665|2450939|N|N|N|N|N| +2451031|AAAAAAAAHFGGFCAA|1998-08-05|1183|5145|395|1998|3|8|5|3|1998|395|5145|Wednesday|1998Q3|N|N|N|2451027|2451238|2450666|2450940|N|N|N|N|N| +2451032|AAAAAAAAIFGGFCAA|1998-08-06|1183|5145|395|1998|4|8|6|3|1998|395|5145|Thursday|1998Q3|N|N|N|2451027|2451238|2450667|2450941|N|N|N|N|N| +2451033|AAAAAAAAJFGGFCAA|1998-08-07|1183|5145|395|1998|5|8|7|3|1998|395|5145|Friday|1998Q3|N|Y|N|2451027|2451238|2450668|2450942|N|N|N|N|N| +2451034|AAAAAAAAKFGGFCAA|1998-08-08|1183|5145|395|1998|6|8|8|3|1998|395|5145|Saturday|1998Q3|N|Y|N|2451027|2451238|2450669|2450943|N|N|N|N|N| +2451035|AAAAAAAALFGGFCAA|1998-08-09|1183|5145|395|1998|0|8|9|3|1998|395|5145|Sunday|1998Q3|N|N|N|2451027|2451238|2450670|2450944|N|N|N|N|N| +2451036|AAAAAAAAMFGGFCAA|1998-08-10|1183|5145|395|1998|1|8|10|3|1998|395|5145|Monday|1998Q3|N|N|N|2451027|2451238|2450671|2450945|N|N|N|N|N| +2451037|AAAAAAAANFGGFCAA|1998-08-11|1183|5146|395|1998|2|8|11|3|1998|395|5146|Tuesday|1998Q3|N|N|N|2451027|2451238|2450672|2450946|N|N|N|N|N| +2451038|AAAAAAAAOFGGFCAA|1998-08-12|1183|5146|395|1998|3|8|12|3|1998|395|5146|Wednesday|1998Q3|N|N|N|2451027|2451238|2450673|2450947|N|N|N|N|N| +2451039|AAAAAAAAPFGGFCAA|1998-08-13|1183|5146|395|1998|4|8|13|3|1998|395|5146|Thursday|1998Q3|N|N|N|2451027|2451238|2450674|2450948|N|N|N|N|N| +2451040|AAAAAAAAAGGGFCAA|1998-08-14|1183|5146|395|1998|5|8|14|3|1998|395|5146|Friday|1998Q3|N|Y|N|2451027|2451238|2450675|2450949|N|N|N|N|N| +2451041|AAAAAAAABGGGFCAA|1998-08-15|1183|5146|395|1998|6|8|15|3|1998|395|5146|Saturday|1998Q3|N|Y|N|2451027|2451238|2450676|2450950|N|N|N|N|N| +2451042|AAAAAAAACGGGFCAA|1998-08-16|1183|5146|395|1998|0|8|16|3|1998|395|5146|Sunday|1998Q3|N|N|N|2451027|2451238|2450677|2450951|N|N|N|N|N| +2451043|AAAAAAAADGGGFCAA|1998-08-17|1183|5146|395|1998|1|8|17|3|1998|395|5146|Monday|1998Q3|N|N|N|2451027|2451238|2450678|2450952|N|N|N|N|N| +2451044|AAAAAAAAEGGGFCAA|1998-08-18|1183|5147|395|1998|2|8|18|3|1998|395|5147|Tuesday|1998Q3|N|N|N|2451027|2451238|2450679|2450953|N|N|N|N|N| +2451045|AAAAAAAAFGGGFCAA|1998-08-19|1183|5147|395|1998|3|8|19|3|1998|395|5147|Wednesday|1998Q3|N|N|N|2451027|2451238|2450680|2450954|N|N|N|N|N| +2451046|AAAAAAAAGGGGFCAA|1998-08-20|1183|5147|395|1998|4|8|20|3|1998|395|5147|Thursday|1998Q3|N|N|N|2451027|2451238|2450681|2450955|N|N|N|N|N| +2451047|AAAAAAAAHGGGFCAA|1998-08-21|1183|5147|395|1998|5|8|21|3|1998|395|5147|Friday|1998Q3|N|Y|N|2451027|2451238|2450682|2450956|N|N|N|N|N| +2451048|AAAAAAAAIGGGFCAA|1998-08-22|1183|5147|395|1998|6|8|22|3|1998|395|5147|Saturday|1998Q3|N|Y|N|2451027|2451238|2450683|2450957|N|N|N|N|N| +2451049|AAAAAAAAJGGGFCAA|1998-08-23|1183|5147|395|1998|0|8|23|3|1998|395|5147|Sunday|1998Q3|N|N|N|2451027|2451238|2450684|2450958|N|N|N|N|N| +2451050|AAAAAAAAKGGGFCAA|1998-08-24|1183|5147|395|1998|1|8|24|3|1998|395|5147|Monday|1998Q3|N|N|N|2451027|2451238|2450685|2450959|N|N|N|N|N| +2451051|AAAAAAAALGGGFCAA|1998-08-25|1183|5148|395|1998|2|8|25|3|1998|395|5148|Tuesday|1998Q3|N|N|N|2451027|2451238|2450686|2450960|N|N|N|N|N| +2451052|AAAAAAAAMGGGFCAA|1998-08-26|1183|5148|395|1998|3|8|26|3|1998|395|5148|Wednesday|1998Q3|N|N|N|2451027|2451238|2450687|2450961|N|N|N|N|N| +2451053|AAAAAAAANGGGFCAA|1998-08-27|1183|5148|395|1998|4|8|27|3|1998|395|5148|Thursday|1998Q3|N|N|N|2451027|2451238|2450688|2450962|N|N|N|N|N| +2451054|AAAAAAAAOGGGFCAA|1998-08-28|1183|5148|395|1998|5|8|28|3|1998|395|5148|Friday|1998Q3|N|Y|N|2451027|2451238|2450689|2450963|N|N|N|N|N| +2451055|AAAAAAAAPGGGFCAA|1998-08-29|1183|5148|395|1998|6|8|29|3|1998|395|5148|Saturday|1998Q3|N|Y|N|2451027|2451238|2450690|2450964|N|N|N|N|N| +2451056|AAAAAAAAAHGGFCAA|1998-08-30|1183|5148|395|1998|0|8|30|3|1998|395|5148|Sunday|1998Q3|N|N|N|2451027|2451238|2450691|2450965|N|N|N|N|N| +2451057|AAAAAAAABHGGFCAA|1998-08-31|1183|5148|395|1998|1|8|31|3|1998|395|5148|Monday|1998Q3|N|N|N|2451027|2451238|2450692|2450966|N|N|N|N|N| +2451058|AAAAAAAACHGGFCAA|1998-09-01|1184|5149|396|1998|2|9|1|3|1998|396|5149|Tuesday|1998Q3|N|N|N|2451058|2451300|2450693|2450967|N|N|N|N|N| +2451059|AAAAAAAADHGGFCAA|1998-09-02|1184|5149|396|1998|3|9|2|3|1998|396|5149|Wednesday|1998Q3|N|N|N|2451058|2451300|2450694|2450968|N|N|N|N|N| +2451060|AAAAAAAAEHGGFCAA|1998-09-03|1184|5149|396|1998|4|9|3|3|1998|396|5149|Thursday|1998Q3|N|N|N|2451058|2451300|2450695|2450969|N|N|N|N|N| +2451061|AAAAAAAAFHGGFCAA|1998-09-04|1184|5149|396|1998|5|9|4|3|1998|396|5149|Friday|1998Q3|N|Y|N|2451058|2451300|2450696|2450970|N|N|N|N|N| +2451062|AAAAAAAAGHGGFCAA|1998-09-05|1184|5149|396|1998|6|9|5|3|1998|396|5149|Saturday|1998Q3|N|Y|N|2451058|2451300|2450697|2450971|N|N|N|N|N| +2451063|AAAAAAAAHHGGFCAA|1998-09-06|1184|5149|396|1998|0|9|6|3|1998|396|5149|Sunday|1998Q3|N|N|N|2451058|2451300|2450698|2450972|N|N|N|N|N| +2451064|AAAAAAAAIHGGFCAA|1998-09-07|1184|5149|396|1998|1|9|7|3|1998|396|5149|Monday|1998Q3|N|N|N|2451058|2451300|2450699|2450973|N|N|N|N|N| +2451065|AAAAAAAAJHGGFCAA|1998-09-08|1184|5150|396|1998|2|9|8|3|1998|396|5150|Tuesday|1998Q3|N|N|N|2451058|2451300|2450700|2450974|N|N|N|N|N| +2451066|AAAAAAAAKHGGFCAA|1998-09-09|1184|5150|396|1998|3|9|9|3|1998|396|5150|Wednesday|1998Q3|N|N|N|2451058|2451300|2450701|2450975|N|N|N|N|N| +2451067|AAAAAAAALHGGFCAA|1998-09-10|1184|5150|396|1998|4|9|10|3|1998|396|5150|Thursday|1998Q3|N|N|N|2451058|2451300|2450702|2450976|N|N|N|N|N| +2451068|AAAAAAAAMHGGFCAA|1998-09-11|1184|5150|396|1998|5|9|11|3|1998|396|5150|Friday|1998Q3|N|Y|N|2451058|2451300|2450703|2450977|N|N|N|N|N| +2451069|AAAAAAAANHGGFCAA|1998-09-12|1184|5150|396|1998|6|9|12|3|1998|396|5150|Saturday|1998Q3|N|Y|N|2451058|2451300|2450704|2450978|N|N|N|N|N| +2451070|AAAAAAAAOHGGFCAA|1998-09-13|1184|5150|396|1998|0|9|13|3|1998|396|5150|Sunday|1998Q3|N|N|N|2451058|2451300|2450705|2450979|N|N|N|N|N| +2451071|AAAAAAAAPHGGFCAA|1998-09-14|1184|5150|396|1998|1|9|14|3|1998|396|5150|Monday|1998Q3|N|N|N|2451058|2451300|2450706|2450980|N|N|N|N|N| +2451072|AAAAAAAAAIGGFCAA|1998-09-15|1184|5151|396|1998|2|9|15|3|1998|396|5151|Tuesday|1998Q3|N|N|N|2451058|2451300|2450707|2450981|N|N|N|N|N| +2451073|AAAAAAAABIGGFCAA|1998-09-16|1184|5151|396|1998|3|9|16|3|1998|396|5151|Wednesday|1998Q3|N|N|N|2451058|2451300|2450708|2450982|N|N|N|N|N| +2451074|AAAAAAAACIGGFCAA|1998-09-17|1184|5151|396|1998|4|9|17|3|1998|396|5151|Thursday|1998Q3|N|N|N|2451058|2451300|2450709|2450983|N|N|N|N|N| +2451075|AAAAAAAADIGGFCAA|1998-09-18|1184|5151|396|1998|5|9|18|3|1998|396|5151|Friday|1998Q3|N|Y|N|2451058|2451300|2450710|2450984|N|N|N|N|N| +2451076|AAAAAAAAEIGGFCAA|1998-09-19|1184|5151|396|1998|6|9|19|3|1998|396|5151|Saturday|1998Q3|N|Y|N|2451058|2451300|2450711|2450985|N|N|N|N|N| +2451077|AAAAAAAAFIGGFCAA|1998-09-20|1184|5151|396|1998|0|9|20|3|1998|396|5151|Sunday|1998Q3|N|N|N|2451058|2451300|2450712|2450986|N|N|N|N|N| +2451078|AAAAAAAAGIGGFCAA|1998-09-21|1184|5151|396|1998|1|9|21|3|1998|396|5151|Monday|1998Q3|N|N|N|2451058|2451300|2450713|2450987|N|N|N|N|N| +2451079|AAAAAAAAHIGGFCAA|1998-09-22|1184|5152|396|1998|2|9|22|3|1998|396|5152|Tuesday|1998Q3|N|N|N|2451058|2451300|2450714|2450988|N|N|N|N|N| +2451080|AAAAAAAAIIGGFCAA|1998-09-23|1184|5152|396|1998|3|9|23|3|1998|396|5152|Wednesday|1998Q3|N|N|N|2451058|2451300|2450715|2450989|N|N|N|N|N| +2451081|AAAAAAAAJIGGFCAA|1998-09-24|1184|5152|396|1998|4|9|24|3|1998|396|5152|Thursday|1998Q3|N|N|N|2451058|2451300|2450716|2450990|N|N|N|N|N| +2451082|AAAAAAAAKIGGFCAA|1998-09-25|1184|5152|396|1998|5|9|25|3|1998|396|5152|Friday|1998Q3|N|Y|N|2451058|2451300|2450717|2450991|N|N|N|N|N| +2451083|AAAAAAAALIGGFCAA|1998-09-26|1184|5152|396|1998|6|9|26|3|1998|396|5152|Saturday|1998Q3|N|Y|N|2451058|2451300|2450718|2450992|N|N|N|N|N| +2451084|AAAAAAAAMIGGFCAA|1998-09-27|1184|5152|396|1998|0|9|27|3|1998|396|5152|Sunday|1998Q3|N|N|N|2451058|2451300|2450719|2450993|N|N|N|N|N| +2451085|AAAAAAAANIGGFCAA|1998-09-28|1184|5152|396|1998|1|9|28|3|1998|396|5152|Monday|1998Q3|N|N|N|2451058|2451300|2450720|2450994|N|N|N|N|N| +2451086|AAAAAAAAOIGGFCAA|1998-09-29|1184|5153|396|1998|2|9|29|3|1998|396|5153|Tuesday|1998Q3|N|N|N|2451058|2451300|2450721|2450995|N|N|N|N|N| +2451087|AAAAAAAAPIGGFCAA|1998-09-30|1184|5153|396|1998|3|9|30|3|1998|396|5153|Wednesday|1998Q3|N|N|N|2451058|2451300|2450722|2450996|N|N|N|N|N| +2451088|AAAAAAAAAJGGFCAA|1998-10-01|1185|5153|396|1998|4|10|1|3|1998|396|5153|Thursday|1998Q3|N|N|N|2451088|2451360|2450723|2450996|N|N|N|N|N| +2451089|AAAAAAAABJGGFCAA|1998-10-02|1185|5153|396|1998|5|10|2|4|1998|396|5153|Friday|1998Q4|N|Y|N|2451088|2451360|2450724|2450997|N|N|N|N|N| +2451090|AAAAAAAACJGGFCAA|1998-10-03|1185|5153|396|1998|6|10|3|4|1998|396|5153|Saturday|1998Q4|N|Y|N|2451088|2451360|2450725|2450998|N|N|N|N|N| +2451091|AAAAAAAADJGGFCAA|1998-10-04|1185|5153|396|1998|0|10|4|4|1998|396|5153|Sunday|1998Q4|N|N|N|2451088|2451360|2450726|2450999|N|N|N|N|N| +2451092|AAAAAAAAEJGGFCAA|1998-10-05|1185|5153|396|1998|1|10|5|4|1998|396|5153|Monday|1998Q4|N|N|N|2451088|2451360|2450727|2451000|N|N|N|N|N| +2451093|AAAAAAAAFJGGFCAA|1998-10-06|1185|5154|396|1998|2|10|6|4|1998|396|5154|Tuesday|1998Q4|N|N|N|2451088|2451360|2450728|2451001|N|N|N|N|N| +2451094|AAAAAAAAGJGGFCAA|1998-10-07|1185|5154|396|1998|3|10|7|4|1998|396|5154|Wednesday|1998Q4|N|N|N|2451088|2451360|2450729|2451002|N|N|N|N|N| +2451095|AAAAAAAAHJGGFCAA|1998-10-08|1185|5154|396|1998|4|10|8|4|1998|396|5154|Thursday|1998Q4|N|N|N|2451088|2451360|2450730|2451003|N|N|N|N|N| +2451096|AAAAAAAAIJGGFCAA|1998-10-09|1185|5154|396|1998|5|10|9|4|1998|396|5154|Friday|1998Q4|N|Y|N|2451088|2451360|2450731|2451004|N|N|N|N|N| +2451097|AAAAAAAAJJGGFCAA|1998-10-10|1185|5154|396|1998|6|10|10|4|1998|396|5154|Saturday|1998Q4|N|Y|N|2451088|2451360|2450732|2451005|N|N|N|N|N| +2451098|AAAAAAAAKJGGFCAA|1998-10-11|1185|5154|396|1998|0|10|11|4|1998|396|5154|Sunday|1998Q4|N|N|N|2451088|2451360|2450733|2451006|N|N|N|N|N| +2451099|AAAAAAAALJGGFCAA|1998-10-12|1185|5154|396|1998|1|10|12|4|1998|396|5154|Monday|1998Q4|N|N|N|2451088|2451360|2450734|2451007|N|N|N|N|N| +2451100|AAAAAAAAMJGGFCAA|1998-10-13|1185|5155|396|1998|2|10|13|4|1998|396|5155|Tuesday|1998Q4|N|N|N|2451088|2451360|2450735|2451008|N|N|N|N|N| +2451101|AAAAAAAANJGGFCAA|1998-10-14|1185|5155|396|1998|3|10|14|4|1998|396|5155|Wednesday|1998Q4|N|N|N|2451088|2451360|2450736|2451009|N|N|N|N|N| +2451102|AAAAAAAAOJGGFCAA|1998-10-15|1185|5155|396|1998|4|10|15|4|1998|396|5155|Thursday|1998Q4|N|N|N|2451088|2451360|2450737|2451010|N|N|N|N|N| +2451103|AAAAAAAAPJGGFCAA|1998-10-16|1185|5155|396|1998|5|10|16|4|1998|396|5155|Friday|1998Q4|N|Y|N|2451088|2451360|2450738|2451011|N|N|N|N|N| +2451104|AAAAAAAAAKGGFCAA|1998-10-17|1185|5155|396|1998|6|10|17|4|1998|396|5155|Saturday|1998Q4|N|Y|N|2451088|2451360|2450739|2451012|N|N|N|N|N| +2451105|AAAAAAAABKGGFCAA|1998-10-18|1185|5155|396|1998|0|10|18|4|1998|396|5155|Sunday|1998Q4|N|N|N|2451088|2451360|2450740|2451013|N|N|N|N|N| +2451106|AAAAAAAACKGGFCAA|1998-10-19|1185|5155|396|1998|1|10|19|4|1998|396|5155|Monday|1998Q4|N|N|N|2451088|2451360|2450741|2451014|N|N|N|N|N| +2451107|AAAAAAAADKGGFCAA|1998-10-20|1185|5156|396|1998|2|10|20|4|1998|396|5156|Tuesday|1998Q4|N|N|N|2451088|2451360|2450742|2451015|N|N|N|N|N| +2451108|AAAAAAAAEKGGFCAA|1998-10-21|1185|5156|396|1998|3|10|21|4|1998|396|5156|Wednesday|1998Q4|N|N|N|2451088|2451360|2450743|2451016|N|N|N|N|N| +2451109|AAAAAAAAFKGGFCAA|1998-10-22|1185|5156|396|1998|4|10|22|4|1998|396|5156|Thursday|1998Q4|N|N|N|2451088|2451360|2450744|2451017|N|N|N|N|N| +2451110|AAAAAAAAGKGGFCAA|1998-10-23|1185|5156|396|1998|5|10|23|4|1998|396|5156|Friday|1998Q4|N|Y|N|2451088|2451360|2450745|2451018|N|N|N|N|N| +2451111|AAAAAAAAHKGGFCAA|1998-10-24|1185|5156|396|1998|6|10|24|4|1998|396|5156|Saturday|1998Q4|N|Y|N|2451088|2451360|2450746|2451019|N|N|N|N|N| +2451112|AAAAAAAAIKGGFCAA|1998-10-25|1185|5156|396|1998|0|10|25|4|1998|396|5156|Sunday|1998Q4|N|N|N|2451088|2451360|2450747|2451020|N|N|N|N|N| +2451113|AAAAAAAAJKGGFCAA|1998-10-26|1185|5156|396|1998|1|10|26|4|1998|396|5156|Monday|1998Q4|N|N|N|2451088|2451360|2450748|2451021|N|N|N|N|N| +2451114|AAAAAAAAKKGGFCAA|1998-10-27|1185|5157|396|1998|2|10|27|4|1998|396|5157|Tuesday|1998Q4|N|N|N|2451088|2451360|2450749|2451022|N|N|N|N|N| +2451115|AAAAAAAALKGGFCAA|1998-10-28|1185|5157|396|1998|3|10|28|4|1998|396|5157|Wednesday|1998Q4|N|N|N|2451088|2451360|2450750|2451023|N|N|N|N|N| +2451116|AAAAAAAAMKGGFCAA|1998-10-29|1185|5157|396|1998|4|10|29|4|1998|396|5157|Thursday|1998Q4|N|N|N|2451088|2451360|2450751|2451024|N|N|N|N|N| +2451117|AAAAAAAANKGGFCAA|1998-10-30|1185|5157|396|1998|5|10|30|4|1998|396|5157|Friday|1998Q4|N|Y|N|2451088|2451360|2450752|2451025|N|N|N|N|N| +2451118|AAAAAAAAOKGGFCAA|1998-10-31|1185|5157|396|1998|6|10|31|4|1998|396|5157|Saturday|1998Q4|N|Y|N|2451088|2451360|2450753|2451026|N|N|N|N|N| +2451119|AAAAAAAAPKGGFCAA|1998-11-01|1186|5157|396|1998|0|11|1|4|1998|396|5157|Sunday|1998Q4|N|N|N|2451119|2451422|2450754|2451027|N|N|N|N|N| +2451120|AAAAAAAAALGGFCAA|1998-11-02|1186|5157|396|1998|1|11|2|4|1998|396|5157|Monday|1998Q4|N|N|N|2451119|2451422|2450755|2451028|N|N|N|N|N| +2451121|AAAAAAAABLGGFCAA|1998-11-03|1186|5158|396|1998|2|11|3|4|1998|396|5158|Tuesday|1998Q4|N|N|N|2451119|2451422|2450756|2451029|N|N|N|N|N| +2451122|AAAAAAAACLGGFCAA|1998-11-04|1186|5158|396|1998|3|11|4|4|1998|396|5158|Wednesday|1998Q4|N|N|N|2451119|2451422|2450757|2451030|N|N|N|N|N| +2451123|AAAAAAAADLGGFCAA|1998-11-05|1186|5158|396|1998|4|11|5|4|1998|396|5158|Thursday|1998Q4|N|N|N|2451119|2451422|2450758|2451031|N|N|N|N|N| +2451124|AAAAAAAAELGGFCAA|1998-11-06|1186|5158|396|1998|5|11|6|4|1998|396|5158|Friday|1998Q4|N|Y|N|2451119|2451422|2450759|2451032|N|N|N|N|N| +2451125|AAAAAAAAFLGGFCAA|1998-11-07|1186|5158|396|1998|6|11|7|4|1998|396|5158|Saturday|1998Q4|N|Y|N|2451119|2451422|2450760|2451033|N|N|N|N|N| +2451126|AAAAAAAAGLGGFCAA|1998-11-08|1186|5158|396|1998|0|11|8|4|1998|396|5158|Sunday|1998Q4|N|N|N|2451119|2451422|2450761|2451034|N|N|N|N|N| +2451127|AAAAAAAAHLGGFCAA|1998-11-09|1186|5158|396|1998|1|11|9|4|1998|396|5158|Monday|1998Q4|N|N|N|2451119|2451422|2450762|2451035|N|N|N|N|N| +2451128|AAAAAAAAILGGFCAA|1998-11-10|1186|5159|396|1998|2|11|10|4|1998|396|5159|Tuesday|1998Q4|N|N|N|2451119|2451422|2450763|2451036|N|N|N|N|N| +2451129|AAAAAAAAJLGGFCAA|1998-11-11|1186|5159|396|1998|3|11|11|4|1998|396|5159|Wednesday|1998Q4|N|N|N|2451119|2451422|2450764|2451037|N|N|N|N|N| +2451130|AAAAAAAAKLGGFCAA|1998-11-12|1186|5159|396|1998|4|11|12|4|1998|396|5159|Thursday|1998Q4|N|N|N|2451119|2451422|2450765|2451038|N|N|N|N|N| +2451131|AAAAAAAALLGGFCAA|1998-11-13|1186|5159|396|1998|5|11|13|4|1998|396|5159|Friday|1998Q4|N|Y|N|2451119|2451422|2450766|2451039|N|N|N|N|N| +2451132|AAAAAAAAMLGGFCAA|1998-11-14|1186|5159|396|1998|6|11|14|4|1998|396|5159|Saturday|1998Q4|N|Y|N|2451119|2451422|2450767|2451040|N|N|N|N|N| +2451133|AAAAAAAANLGGFCAA|1998-11-15|1186|5159|396|1998|0|11|15|4|1998|396|5159|Sunday|1998Q4|N|N|N|2451119|2451422|2450768|2451041|N|N|N|N|N| +2451134|AAAAAAAAOLGGFCAA|1998-11-16|1186|5159|396|1998|1|11|16|4|1998|396|5159|Monday|1998Q4|N|N|N|2451119|2451422|2450769|2451042|N|N|N|N|N| +2451135|AAAAAAAAPLGGFCAA|1998-11-17|1186|5160|396|1998|2|11|17|4|1998|396|5160|Tuesday|1998Q4|N|N|N|2451119|2451422|2450770|2451043|N|N|N|N|N| +2451136|AAAAAAAAAMGGFCAA|1998-11-18|1186|5160|396|1998|3|11|18|4|1998|396|5160|Wednesday|1998Q4|N|N|N|2451119|2451422|2450771|2451044|N|N|N|N|N| +2451137|AAAAAAAABMGGFCAA|1998-11-19|1186|5160|396|1998|4|11|19|4|1998|396|5160|Thursday|1998Q4|N|N|N|2451119|2451422|2450772|2451045|N|N|N|N|N| +2451138|AAAAAAAACMGGFCAA|1998-11-20|1186|5160|396|1998|5|11|20|4|1998|396|5160|Friday|1998Q4|N|Y|N|2451119|2451422|2450773|2451046|N|N|N|N|N| +2451139|AAAAAAAADMGGFCAA|1998-11-21|1186|5160|396|1998|6|11|21|4|1998|396|5160|Saturday|1998Q4|N|Y|N|2451119|2451422|2450774|2451047|N|N|N|N|N| +2451140|AAAAAAAAEMGGFCAA|1998-11-22|1186|5160|396|1998|0|11|22|4|1998|396|5160|Sunday|1998Q4|N|N|N|2451119|2451422|2450775|2451048|N|N|N|N|N| +2451141|AAAAAAAAFMGGFCAA|1998-11-23|1186|5160|396|1998|1|11|23|4|1998|396|5160|Monday|1998Q4|N|N|N|2451119|2451422|2450776|2451049|N|N|N|N|N| +2451142|AAAAAAAAGMGGFCAA|1998-11-24|1186|5161|396|1998|2|11|24|4|1998|396|5161|Tuesday|1998Q4|N|N|N|2451119|2451422|2450777|2451050|N|N|N|N|N| +2451143|AAAAAAAAHMGGFCAA|1998-11-25|1186|5161|396|1998|3|11|25|4|1998|396|5161|Wednesday|1998Q4|N|N|N|2451119|2451422|2450778|2451051|N|N|N|N|N| +2451144|AAAAAAAAIMGGFCAA|1998-11-26|1186|5161|396|1998|4|11|26|4|1998|396|5161|Thursday|1998Q4|N|N|N|2451119|2451422|2450779|2451052|N|N|N|N|N| +2451145|AAAAAAAAJMGGFCAA|1998-11-27|1186|5161|396|1998|5|11|27|4|1998|396|5161|Friday|1998Q4|N|Y|N|2451119|2451422|2450780|2451053|N|N|N|N|N| +2451146|AAAAAAAAKMGGFCAA|1998-11-28|1186|5161|396|1998|6|11|28|4|1998|396|5161|Saturday|1998Q4|N|Y|N|2451119|2451422|2450781|2451054|N|N|N|N|N| +2451147|AAAAAAAALMGGFCAA|1998-11-29|1186|5161|396|1998|0|11|29|4|1998|396|5161|Sunday|1998Q4|N|N|N|2451119|2451422|2450782|2451055|N|N|N|N|N| +2451148|AAAAAAAAMMGGFCAA|1998-11-30|1186|5161|396|1998|1|11|30|4|1998|396|5161|Monday|1998Q4|N|N|N|2451119|2451422|2450783|2451056|N|N|N|N|N| +2451149|AAAAAAAANMGGFCAA|1998-12-01|1187|5162|397|1998|2|12|1|4|1998|397|5162|Tuesday|1998Q4|N|N|N|2451149|2451482|2450784|2451057|N|N|N|N|N| +2451150|AAAAAAAAOMGGFCAA|1998-12-02|1187|5162|397|1998|3|12|2|4|1998|397|5162|Wednesday|1998Q4|N|N|N|2451149|2451482|2450785|2451058|N|N|N|N|N| +2451151|AAAAAAAAPMGGFCAA|1998-12-03|1187|5162|397|1998|4|12|3|4|1998|397|5162|Thursday|1998Q4|N|N|N|2451149|2451482|2450786|2451059|N|N|N|N|N| +2451152|AAAAAAAAANGGFCAA|1998-12-04|1187|5162|397|1998|5|12|4|4|1998|397|5162|Friday|1998Q4|N|Y|N|2451149|2451482|2450787|2451060|N|N|N|N|N| +2451153|AAAAAAAABNGGFCAA|1998-12-05|1187|5162|397|1998|6|12|5|4|1998|397|5162|Saturday|1998Q4|N|Y|N|2451149|2451482|2450788|2451061|N|N|N|N|N| +2451154|AAAAAAAACNGGFCAA|1998-12-06|1187|5162|397|1998|0|12|6|4|1998|397|5162|Sunday|1998Q4|N|N|N|2451149|2451482|2450789|2451062|N|N|N|N|N| +2451155|AAAAAAAADNGGFCAA|1998-12-07|1187|5162|397|1998|1|12|7|4|1998|397|5162|Monday|1998Q4|N|N|N|2451149|2451482|2450790|2451063|N|N|N|N|N| +2451156|AAAAAAAAENGGFCAA|1998-12-08|1187|5163|397|1998|2|12|8|4|1998|397|5163|Tuesday|1998Q4|N|N|N|2451149|2451482|2450791|2451064|N|N|N|N|N| +2451157|AAAAAAAAFNGGFCAA|1998-12-09|1187|5163|397|1998|3|12|9|4|1998|397|5163|Wednesday|1998Q4|N|N|N|2451149|2451482|2450792|2451065|N|N|N|N|N| +2451158|AAAAAAAAGNGGFCAA|1998-12-10|1187|5163|397|1998|4|12|10|4|1998|397|5163|Thursday|1998Q4|N|N|N|2451149|2451482|2450793|2451066|N|N|N|N|N| +2451159|AAAAAAAAHNGGFCAA|1998-12-11|1187|5163|397|1998|5|12|11|4|1998|397|5163|Friday|1998Q4|N|Y|N|2451149|2451482|2450794|2451067|N|N|N|N|N| +2451160|AAAAAAAAINGGFCAA|1998-12-12|1187|5163|397|1998|6|12|12|4|1998|397|5163|Saturday|1998Q4|N|Y|N|2451149|2451482|2450795|2451068|N|N|N|N|N| +2451161|AAAAAAAAJNGGFCAA|1998-12-13|1187|5163|397|1998|0|12|13|4|1998|397|5163|Sunday|1998Q4|N|N|N|2451149|2451482|2450796|2451069|N|N|N|N|N| +2451162|AAAAAAAAKNGGFCAA|1998-12-14|1187|5163|397|1998|1|12|14|4|1998|397|5163|Monday|1998Q4|N|N|N|2451149|2451482|2450797|2451070|N|N|N|N|N| +2451163|AAAAAAAALNGGFCAA|1998-12-15|1187|5164|397|1998|2|12|15|4|1998|397|5164|Tuesday|1998Q4|N|N|N|2451149|2451482|2450798|2451071|N|N|N|N|N| +2451164|AAAAAAAAMNGGFCAA|1998-12-16|1187|5164|397|1998|3|12|16|4|1998|397|5164|Wednesday|1998Q4|N|N|N|2451149|2451482|2450799|2451072|N|N|N|N|N| +2451165|AAAAAAAANNGGFCAA|1998-12-17|1187|5164|397|1998|4|12|17|4|1998|397|5164|Thursday|1998Q4|N|N|N|2451149|2451482|2450800|2451073|N|N|N|N|N| +2451166|AAAAAAAAONGGFCAA|1998-12-18|1187|5164|397|1998|5|12|18|4|1998|397|5164|Friday|1998Q4|N|Y|N|2451149|2451482|2450801|2451074|N|N|N|N|N| +2451167|AAAAAAAAPNGGFCAA|1998-12-19|1187|5164|397|1998|6|12|19|4|1998|397|5164|Saturday|1998Q4|N|Y|N|2451149|2451482|2450802|2451075|N|N|N|N|N| +2451168|AAAAAAAAAOGGFCAA|1998-12-20|1187|5164|397|1998|0|12|20|4|1998|397|5164|Sunday|1998Q4|N|N|N|2451149|2451482|2450803|2451076|N|N|N|N|N| +2451169|AAAAAAAABOGGFCAA|1998-12-21|1187|5164|397|1998|1|12|21|4|1998|397|5164|Monday|1998Q4|N|N|N|2451149|2451482|2450804|2451077|N|N|N|N|N| +2451170|AAAAAAAACOGGFCAA|1998-12-22|1187|5165|397|1998|2|12|22|4|1998|397|5165|Tuesday|1998Q4|N|N|N|2451149|2451482|2450805|2451078|N|N|N|N|N| +2451171|AAAAAAAADOGGFCAA|1998-12-23|1187|5165|397|1998|3|12|23|4|1998|397|5165|Wednesday|1998Q4|N|N|N|2451149|2451482|2450806|2451079|N|N|N|N|N| +2451172|AAAAAAAAEOGGFCAA|1998-12-24|1187|5165|397|1998|4|12|24|4|1998|397|5165|Thursday|1998Q4|N|N|N|2451149|2451482|2450807|2451080|N|N|N|N|N| +2451173|AAAAAAAAFOGGFCAA|1998-12-25|1187|5165|397|1998|5|12|25|4|1998|397|5165|Friday|1998Q4|N|Y|N|2451149|2451482|2450808|2451081|N|N|N|N|N| +2451174|AAAAAAAAGOGGFCAA|1998-12-26|1187|5165|397|1998|6|12|26|4|1998|397|5165|Saturday|1998Q4|Y|Y|N|2451149|2451482|2450809|2451082|N|N|N|N|N| +2451175|AAAAAAAAHOGGFCAA|1998-12-27|1187|5165|397|1998|0|12|27|4|1998|397|5165|Sunday|1998Q4|N|N|Y|2451149|2451482|2450810|2451083|N|N|N|N|N| +2451176|AAAAAAAAIOGGFCAA|1998-12-28|1187|5165|397|1998|1|12|28|4|1998|397|5165|Monday|1998Q4|N|N|N|2451149|2451482|2450811|2451084|N|N|N|N|N| +2451177|AAAAAAAAJOGGFCAA|1998-12-29|1187|5166|397|1998|2|12|29|4|1998|397|5166|Tuesday|1998Q4|N|N|N|2451149|2451482|2450812|2451085|N|N|N|N|N| +2451178|AAAAAAAAKOGGFCAA|1998-12-30|1187|5166|397|1998|3|12|30|4|1998|397|5166|Wednesday|1998Q4|N|N|N|2451149|2451482|2450813|2451086|N|N|N|N|N| +2451179|AAAAAAAALOGGFCAA|1998-12-31|1187|5166|397|1998|4|12|31|4|1998|397|5166|Thursday|1998Q4|N|N|N|2451149|2451482|2450814|2451087|N|N|N|N|N| +2451180|AAAAAAAAMOGGFCAA|1999-01-01|1188|5166|397|1999|5|1|1|1|1999|397|5166|Friday|1999Q1|Y|Y|N|2451180|2451179|2450815|2451088|N|N|N|N|N| +2451181|AAAAAAAANOGGFCAA|1999-01-02|1188|5166|397|1999|6|1|2|1|1999|397|5166|Saturday|1999Q1|N|Y|Y|2451180|2451179|2450816|2451089|N|N|N|N|N| +2451182|AAAAAAAAOOGGFCAA|1999-01-03|1188|5166|397|1999|0|1|3|1|1999|397|5166|Sunday|1999Q1|N|N|N|2451180|2451179|2450817|2451090|N|N|N|N|N| +2451183|AAAAAAAAPOGGFCAA|1999-01-04|1188|5166|397|1999|1|1|4|1|1999|397|5166|Monday|1999Q1|N|N|N|2451180|2451179|2450818|2451091|N|N|N|N|N| +2451184|AAAAAAAAAPGGFCAA|1999-01-05|1188|5167|397|1999|2|1|5|1|1999|397|5167|Tuesday|1999Q1|N|N|N|2451180|2451179|2450819|2451092|N|N|N|N|N| +2451185|AAAAAAAABPGGFCAA|1999-01-06|1188|5167|397|1999|3|1|6|1|1999|397|5167|Wednesday|1999Q1|N|N|N|2451180|2451179|2450820|2451093|N|N|N|N|N| +2451186|AAAAAAAACPGGFCAA|1999-01-07|1188|5167|397|1999|4|1|7|1|1999|397|5167|Thursday|1999Q1|N|N|N|2451180|2451179|2450821|2451094|N|N|N|N|N| +2451187|AAAAAAAADPGGFCAA|1999-01-08|1188|5167|397|1999|5|1|8|1|1999|397|5167|Friday|1999Q1|N|Y|N|2451180|2451179|2450822|2451095|N|N|N|N|N| +2451188|AAAAAAAAEPGGFCAA|1999-01-09|1188|5167|397|1999|6|1|9|1|1999|397|5167|Saturday|1999Q1|N|Y|N|2451180|2451179|2450823|2451096|N|N|N|N|N| +2451189|AAAAAAAAFPGGFCAA|1999-01-10|1188|5167|397|1999|0|1|10|1|1999|397|5167|Sunday|1999Q1|N|N|N|2451180|2451179|2450824|2451097|N|N|N|N|N| +2451190|AAAAAAAAGPGGFCAA|1999-01-11|1188|5167|397|1999|1|1|11|1|1999|397|5167|Monday|1999Q1|N|N|N|2451180|2451179|2450825|2451098|N|N|N|N|N| +2451191|AAAAAAAAHPGGFCAA|1999-01-12|1188|5168|397|1999|2|1|12|1|1999|397|5168|Tuesday|1999Q1|N|N|N|2451180|2451179|2450826|2451099|N|N|N|N|N| +2451192|AAAAAAAAIPGGFCAA|1999-01-13|1188|5168|397|1999|3|1|13|1|1999|397|5168|Wednesday|1999Q1|N|N|N|2451180|2451179|2450827|2451100|N|N|N|N|N| +2451193|AAAAAAAAJPGGFCAA|1999-01-14|1188|5168|397|1999|4|1|14|1|1999|397|5168|Thursday|1999Q1|N|N|N|2451180|2451179|2450828|2451101|N|N|N|N|N| +2451194|AAAAAAAAKPGGFCAA|1999-01-15|1188|5168|397|1999|5|1|15|1|1999|397|5168|Friday|1999Q1|N|Y|N|2451180|2451179|2450829|2451102|N|N|N|N|N| +2451195|AAAAAAAALPGGFCAA|1999-01-16|1188|5168|397|1999|6|1|16|1|1999|397|5168|Saturday|1999Q1|N|Y|N|2451180|2451179|2450830|2451103|N|N|N|N|N| +2451196|AAAAAAAAMPGGFCAA|1999-01-17|1188|5168|397|1999|0|1|17|1|1999|397|5168|Sunday|1999Q1|N|N|N|2451180|2451179|2450831|2451104|N|N|N|N|N| +2451197|AAAAAAAANPGGFCAA|1999-01-18|1188|5168|397|1999|1|1|18|1|1999|397|5168|Monday|1999Q1|N|N|N|2451180|2451179|2450832|2451105|N|N|N|N|N| +2451198|AAAAAAAAOPGGFCAA|1999-01-19|1188|5169|397|1999|2|1|19|1|1999|397|5169|Tuesday|1999Q1|N|N|N|2451180|2451179|2450833|2451106|N|N|N|N|N| +2451199|AAAAAAAAPPGGFCAA|1999-01-20|1188|5169|397|1999|3|1|20|1|1999|397|5169|Wednesday|1999Q1|N|N|N|2451180|2451179|2450834|2451107|N|N|N|N|N| +2451200|AAAAAAAAAAHGFCAA|1999-01-21|1188|5169|397|1999|4|1|21|1|1999|397|5169|Thursday|1999Q1|N|N|N|2451180|2451179|2450835|2451108|N|N|N|N|N| +2451201|AAAAAAAABAHGFCAA|1999-01-22|1188|5169|397|1999|5|1|22|1|1999|397|5169|Friday|1999Q1|N|Y|N|2451180|2451179|2450836|2451109|N|N|N|N|N| +2451202|AAAAAAAACAHGFCAA|1999-01-23|1188|5169|397|1999|6|1|23|1|1999|397|5169|Saturday|1999Q1|N|Y|N|2451180|2451179|2450837|2451110|N|N|N|N|N| +2451203|AAAAAAAADAHGFCAA|1999-01-24|1188|5169|397|1999|0|1|24|1|1999|397|5169|Sunday|1999Q1|N|N|N|2451180|2451179|2450838|2451111|N|N|N|N|N| +2451204|AAAAAAAAEAHGFCAA|1999-01-25|1188|5169|397|1999|1|1|25|1|1999|397|5169|Monday|1999Q1|N|N|N|2451180|2451179|2450839|2451112|N|N|N|N|N| +2451205|AAAAAAAAFAHGFCAA|1999-01-26|1188|5170|397|1999|2|1|26|1|1999|397|5170|Tuesday|1999Q1|N|N|N|2451180|2451179|2450840|2451113|N|N|N|N|N| +2451206|AAAAAAAAGAHGFCAA|1999-01-27|1188|5170|397|1999|3|1|27|1|1999|397|5170|Wednesday|1999Q1|N|N|N|2451180|2451179|2450841|2451114|N|N|N|N|N| +2451207|AAAAAAAAHAHGFCAA|1999-01-28|1188|5170|397|1999|4|1|28|1|1999|397|5170|Thursday|1999Q1|N|N|N|2451180|2451179|2450842|2451115|N|N|N|N|N| +2451208|AAAAAAAAIAHGFCAA|1999-01-29|1188|5170|397|1999|5|1|29|1|1999|397|5170|Friday|1999Q1|N|Y|N|2451180|2451179|2450843|2451116|N|N|N|N|N| +2451209|AAAAAAAAJAHGFCAA|1999-01-30|1188|5170|397|1999|6|1|30|1|1999|397|5170|Saturday|1999Q1|N|Y|N|2451180|2451179|2450844|2451117|N|N|N|N|N| +2451210|AAAAAAAAKAHGFCAA|1999-01-31|1188|5170|397|1999|0|1|31|1|1999|397|5170|Sunday|1999Q1|N|N|N|2451180|2451179|2450845|2451118|N|N|N|N|N| +2451211|AAAAAAAALAHGFCAA|1999-02-01|1189|5170|397|1999|1|2|1|1|1999|397|5170|Monday|1999Q1|N|N|N|2451211|2451241|2450846|2451119|N|N|N|N|N| +2451212|AAAAAAAAMAHGFCAA|1999-02-02|1189|5171|397|1999|2|2|2|1|1999|397|5171|Tuesday|1999Q1|N|N|N|2451211|2451241|2450847|2451120|N|N|N|N|N| +2451213|AAAAAAAANAHGFCAA|1999-02-03|1189|5171|397|1999|3|2|3|1|1999|397|5171|Wednesday|1999Q1|N|N|N|2451211|2451241|2450848|2451121|N|N|N|N|N| +2451214|AAAAAAAAOAHGFCAA|1999-02-04|1189|5171|397|1999|4|2|4|1|1999|397|5171|Thursday|1999Q1|N|N|N|2451211|2451241|2450849|2451122|N|N|N|N|N| +2451215|AAAAAAAAPAHGFCAA|1999-02-05|1189|5171|397|1999|5|2|5|1|1999|397|5171|Friday|1999Q1|N|Y|N|2451211|2451241|2450850|2451123|N|N|N|N|N| +2451216|AAAAAAAAABHGFCAA|1999-02-06|1189|5171|397|1999|6|2|6|1|1999|397|5171|Saturday|1999Q1|N|Y|N|2451211|2451241|2450851|2451124|N|N|N|N|N| +2451217|AAAAAAAABBHGFCAA|1999-02-07|1189|5171|397|1999|0|2|7|1|1999|397|5171|Sunday|1999Q1|N|N|N|2451211|2451241|2450852|2451125|N|N|N|N|N| +2451218|AAAAAAAACBHGFCAA|1999-02-08|1189|5171|397|1999|1|2|8|1|1999|397|5171|Monday|1999Q1|N|N|N|2451211|2451241|2450853|2451126|N|N|N|N|N| +2451219|AAAAAAAADBHGFCAA|1999-02-09|1189|5172|397|1999|2|2|9|1|1999|397|5172|Tuesday|1999Q1|N|N|N|2451211|2451241|2450854|2451127|N|N|N|N|N| +2451220|AAAAAAAAEBHGFCAA|1999-02-10|1189|5172|397|1999|3|2|10|1|1999|397|5172|Wednesday|1999Q1|N|N|N|2451211|2451241|2450855|2451128|N|N|N|N|N| +2451221|AAAAAAAAFBHGFCAA|1999-02-11|1189|5172|397|1999|4|2|11|1|1999|397|5172|Thursday|1999Q1|N|N|N|2451211|2451241|2450856|2451129|N|N|N|N|N| +2451222|AAAAAAAAGBHGFCAA|1999-02-12|1189|5172|397|1999|5|2|12|1|1999|397|5172|Friday|1999Q1|N|Y|N|2451211|2451241|2450857|2451130|N|N|N|N|N| +2451223|AAAAAAAAHBHGFCAA|1999-02-13|1189|5172|397|1999|6|2|13|1|1999|397|5172|Saturday|1999Q1|N|Y|N|2451211|2451241|2450858|2451131|N|N|N|N|N| +2451224|AAAAAAAAIBHGFCAA|1999-02-14|1189|5172|397|1999|0|2|14|1|1999|397|5172|Sunday|1999Q1|N|N|N|2451211|2451241|2450859|2451132|N|N|N|N|N| +2451225|AAAAAAAAJBHGFCAA|1999-02-15|1189|5172|397|1999|1|2|15|1|1999|397|5172|Monday|1999Q1|N|N|N|2451211|2451241|2450860|2451133|N|N|N|N|N| +2451226|AAAAAAAAKBHGFCAA|1999-02-16|1189|5173|397|1999|2|2|16|1|1999|397|5173|Tuesday|1999Q1|N|N|N|2451211|2451241|2450861|2451134|N|N|N|N|N| +2451227|AAAAAAAALBHGFCAA|1999-02-17|1189|5173|397|1999|3|2|17|1|1999|397|5173|Wednesday|1999Q1|N|N|N|2451211|2451241|2450862|2451135|N|N|N|N|N| +2451228|AAAAAAAAMBHGFCAA|1999-02-18|1189|5173|397|1999|4|2|18|1|1999|397|5173|Thursday|1999Q1|N|N|N|2451211|2451241|2450863|2451136|N|N|N|N|N| +2451229|AAAAAAAANBHGFCAA|1999-02-19|1189|5173|397|1999|5|2|19|1|1999|397|5173|Friday|1999Q1|N|Y|N|2451211|2451241|2450864|2451137|N|N|N|N|N| +2451230|AAAAAAAAOBHGFCAA|1999-02-20|1189|5173|397|1999|6|2|20|1|1999|397|5173|Saturday|1999Q1|N|Y|N|2451211|2451241|2450865|2451138|N|N|N|N|N| +2451231|AAAAAAAAPBHGFCAA|1999-02-21|1189|5173|397|1999|0|2|21|1|1999|397|5173|Sunday|1999Q1|N|N|N|2451211|2451241|2450866|2451139|N|N|N|N|N| +2451232|AAAAAAAAACHGFCAA|1999-02-22|1189|5173|397|1999|1|2|22|1|1999|397|5173|Monday|1999Q1|N|N|N|2451211|2451241|2450867|2451140|N|N|N|N|N| +2451233|AAAAAAAABCHGFCAA|1999-02-23|1189|5174|397|1999|2|2|23|1|1999|397|5174|Tuesday|1999Q1|N|N|N|2451211|2451241|2450868|2451141|N|N|N|N|N| +2451234|AAAAAAAACCHGFCAA|1999-02-24|1189|5174|397|1999|3|2|24|1|1999|397|5174|Wednesday|1999Q1|N|N|N|2451211|2451241|2450869|2451142|N|N|N|N|N| +2451235|AAAAAAAADCHGFCAA|1999-02-25|1189|5174|397|1999|4|2|25|1|1999|397|5174|Thursday|1999Q1|N|N|N|2451211|2451241|2450870|2451143|N|N|N|N|N| +2451236|AAAAAAAAECHGFCAA|1999-02-26|1189|5174|397|1999|5|2|26|1|1999|397|5174|Friday|1999Q1|N|Y|N|2451211|2451241|2450871|2451144|N|N|N|N|N| +2451237|AAAAAAAAFCHGFCAA|1999-02-27|1189|5174|397|1999|6|2|27|1|1999|397|5174|Saturday|1999Q1|N|Y|N|2451211|2451241|2450872|2451145|N|N|N|N|N| +2451238|AAAAAAAAGCHGFCAA|1999-02-28|1189|5174|397|1999|0|2|28|1|1999|397|5174|Sunday|1999Q1|N|N|N|2451211|2451241|2450873|2451146|N|N|N|N|N| +2451239|AAAAAAAAHCHGFCAA|1999-03-01|1190|5174|398|1999|1|3|1|1|1999|398|5174|Monday|1999Q1|N|N|N|2451239|2451297|2450874|2451147|N|N|N|N|N| +2451240|AAAAAAAAICHGFCAA|1999-03-02|1190|5175|398|1999|2|3|2|1|1999|398|5175|Tuesday|1999Q1|N|N|N|2451239|2451297|2450875|2451148|N|N|N|N|N| +2451241|AAAAAAAAJCHGFCAA|1999-03-03|1190|5175|398|1999|3|3|3|1|1999|398|5175|Wednesday|1999Q1|N|N|N|2451239|2451297|2450876|2451149|N|N|N|N|N| +2451242|AAAAAAAAKCHGFCAA|1999-03-04|1190|5175|398|1999|4|3|4|1|1999|398|5175|Thursday|1999Q1|N|N|N|2451239|2451297|2450877|2451150|N|N|N|N|N| +2451243|AAAAAAAALCHGFCAA|1999-03-05|1190|5175|398|1999|5|3|5|1|1999|398|5175|Friday|1999Q1|N|Y|N|2451239|2451297|2450878|2451151|N|N|N|N|N| +2451244|AAAAAAAAMCHGFCAA|1999-03-06|1190|5175|398|1999|6|3|6|1|1999|398|5175|Saturday|1999Q1|N|Y|N|2451239|2451297|2450879|2451152|N|N|N|N|N| +2451245|AAAAAAAANCHGFCAA|1999-03-07|1190|5175|398|1999|0|3|7|1|1999|398|5175|Sunday|1999Q1|N|N|N|2451239|2451297|2450880|2451153|N|N|N|N|N| +2451246|AAAAAAAAOCHGFCAA|1999-03-08|1190|5175|398|1999|1|3|8|1|1999|398|5175|Monday|1999Q1|N|N|N|2451239|2451297|2450881|2451154|N|N|N|N|N| +2451247|AAAAAAAAPCHGFCAA|1999-03-09|1190|5176|398|1999|2|3|9|1|1999|398|5176|Tuesday|1999Q1|N|N|N|2451239|2451297|2450882|2451155|N|N|N|N|N| +2451248|AAAAAAAAADHGFCAA|1999-03-10|1190|5176|398|1999|3|3|10|1|1999|398|5176|Wednesday|1999Q1|N|N|N|2451239|2451297|2450883|2451156|N|N|N|N|N| +2451249|AAAAAAAABDHGFCAA|1999-03-11|1190|5176|398|1999|4|3|11|1|1999|398|5176|Thursday|1999Q1|N|N|N|2451239|2451297|2450884|2451157|N|N|N|N|N| +2451250|AAAAAAAACDHGFCAA|1999-03-12|1190|5176|398|1999|5|3|12|1|1999|398|5176|Friday|1999Q1|N|Y|N|2451239|2451297|2450885|2451158|N|N|N|N|N| +2451251|AAAAAAAADDHGFCAA|1999-03-13|1190|5176|398|1999|6|3|13|1|1999|398|5176|Saturday|1999Q1|N|Y|N|2451239|2451297|2450886|2451159|N|N|N|N|N| +2451252|AAAAAAAAEDHGFCAA|1999-03-14|1190|5176|398|1999|0|3|14|1|1999|398|5176|Sunday|1999Q1|N|N|N|2451239|2451297|2450887|2451160|N|N|N|N|N| +2451253|AAAAAAAAFDHGFCAA|1999-03-15|1190|5176|398|1999|1|3|15|1|1999|398|5176|Monday|1999Q1|N|N|N|2451239|2451297|2450888|2451161|N|N|N|N|N| +2451254|AAAAAAAAGDHGFCAA|1999-03-16|1190|5177|398|1999|2|3|16|1|1999|398|5177|Tuesday|1999Q1|N|N|N|2451239|2451297|2450889|2451162|N|N|N|N|N| +2451255|AAAAAAAAHDHGFCAA|1999-03-17|1190|5177|398|1999|3|3|17|1|1999|398|5177|Wednesday|1999Q1|N|N|N|2451239|2451297|2450890|2451163|N|N|N|N|N| +2451256|AAAAAAAAIDHGFCAA|1999-03-18|1190|5177|398|1999|4|3|18|1|1999|398|5177|Thursday|1999Q1|N|N|N|2451239|2451297|2450891|2451164|N|N|N|N|N| +2451257|AAAAAAAAJDHGFCAA|1999-03-19|1190|5177|398|1999|5|3|19|1|1999|398|5177|Friday|1999Q1|N|Y|N|2451239|2451297|2450892|2451165|N|N|N|N|N| +2451258|AAAAAAAAKDHGFCAA|1999-03-20|1190|5177|398|1999|6|3|20|1|1999|398|5177|Saturday|1999Q1|N|Y|N|2451239|2451297|2450893|2451166|N|N|N|N|N| +2451259|AAAAAAAALDHGFCAA|1999-03-21|1190|5177|398|1999|0|3|21|1|1999|398|5177|Sunday|1999Q1|N|N|N|2451239|2451297|2450894|2451167|N|N|N|N|N| +2451260|AAAAAAAAMDHGFCAA|1999-03-22|1190|5177|398|1999|1|3|22|1|1999|398|5177|Monday|1999Q1|N|N|N|2451239|2451297|2450895|2451168|N|N|N|N|N| +2451261|AAAAAAAANDHGFCAA|1999-03-23|1190|5178|398|1999|2|3|23|1|1999|398|5178|Tuesday|1999Q1|N|N|N|2451239|2451297|2450896|2451169|N|N|N|N|N| +2451262|AAAAAAAAODHGFCAA|1999-03-24|1190|5178|398|1999|3|3|24|1|1999|398|5178|Wednesday|1999Q1|N|N|N|2451239|2451297|2450897|2451170|N|N|N|N|N| +2451263|AAAAAAAAPDHGFCAA|1999-03-25|1190|5178|398|1999|4|3|25|1|1999|398|5178|Thursday|1999Q1|N|N|N|2451239|2451297|2450898|2451171|N|N|N|N|N| +2451264|AAAAAAAAAEHGFCAA|1999-03-26|1190|5178|398|1999|5|3|26|1|1999|398|5178|Friday|1999Q1|N|Y|N|2451239|2451297|2450899|2451172|N|N|N|N|N| +2451265|AAAAAAAABEHGFCAA|1999-03-27|1190|5178|398|1999|6|3|27|1|1999|398|5178|Saturday|1999Q1|N|Y|N|2451239|2451297|2450900|2451173|N|N|N|N|N| +2451266|AAAAAAAACEHGFCAA|1999-03-28|1190|5178|398|1999|0|3|28|1|1999|398|5178|Sunday|1999Q1|N|N|N|2451239|2451297|2450901|2451174|N|N|N|N|N| +2451267|AAAAAAAADEHGFCAA|1999-03-29|1190|5178|398|1999|1|3|29|1|1999|398|5178|Monday|1999Q1|N|N|N|2451239|2451297|2450902|2451175|N|N|N|N|N| +2451268|AAAAAAAAEEHGFCAA|1999-03-30|1190|5179|398|1999|2|3|30|1|1999|398|5179|Tuesday|1999Q1|N|N|N|2451239|2451297|2450903|2451176|N|N|N|N|N| +2451269|AAAAAAAAFEHGFCAA|1999-03-31|1190|5179|398|1999|3|3|31|1|1999|398|5179|Wednesday|1999Q1|N|N|N|2451239|2451297|2450904|2451177|N|N|N|N|N| +2451270|AAAAAAAAGEHGFCAA|1999-04-01|1191|5179|398|1999|4|4|1|1|1999|398|5179|Thursday|1999Q1|N|N|N|2451270|2451359|2450905|2451180|N|N|N|N|N| +2451271|AAAAAAAAHEHGFCAA|1999-04-02|1191|5179|398|1999|5|4|2|2|1999|398|5179|Friday|1999Q2|N|Y|N|2451270|2451359|2450906|2451181|N|N|N|N|N| +2451272|AAAAAAAAIEHGFCAA|1999-04-03|1191|5179|398|1999|6|4|3|2|1999|398|5179|Saturday|1999Q2|N|Y|N|2451270|2451359|2450907|2451182|N|N|N|N|N| +2451273|AAAAAAAAJEHGFCAA|1999-04-04|1191|5179|398|1999|0|4|4|2|1999|398|5179|Sunday|1999Q2|N|N|N|2451270|2451359|2450908|2451183|N|N|N|N|N| +2451274|AAAAAAAAKEHGFCAA|1999-04-05|1191|5179|398|1999|1|4|5|2|1999|398|5179|Monday|1999Q2|N|N|N|2451270|2451359|2450909|2451184|N|N|N|N|N| +2451275|AAAAAAAALEHGFCAA|1999-04-06|1191|5180|398|1999|2|4|6|2|1999|398|5180|Tuesday|1999Q2|N|N|N|2451270|2451359|2450910|2451185|N|N|N|N|N| +2451276|AAAAAAAAMEHGFCAA|1999-04-07|1191|5180|398|1999|3|4|7|2|1999|398|5180|Wednesday|1999Q2|N|N|N|2451270|2451359|2450911|2451186|N|N|N|N|N| +2451277|AAAAAAAANEHGFCAA|1999-04-08|1191|5180|398|1999|4|4|8|2|1999|398|5180|Thursday|1999Q2|N|N|N|2451270|2451359|2450912|2451187|N|N|N|N|N| +2451278|AAAAAAAAOEHGFCAA|1999-04-09|1191|5180|398|1999|5|4|9|2|1999|398|5180|Friday|1999Q2|N|Y|N|2451270|2451359|2450913|2451188|N|N|N|N|N| +2451279|AAAAAAAAPEHGFCAA|1999-04-10|1191|5180|398|1999|6|4|10|2|1999|398|5180|Saturday|1999Q2|N|Y|N|2451270|2451359|2450914|2451189|N|N|N|N|N| +2451280|AAAAAAAAAFHGFCAA|1999-04-11|1191|5180|398|1999|0|4|11|2|1999|398|5180|Sunday|1999Q2|N|N|N|2451270|2451359|2450915|2451190|N|N|N|N|N| +2451281|AAAAAAAABFHGFCAA|1999-04-12|1191|5180|398|1999|1|4|12|2|1999|398|5180|Monday|1999Q2|N|N|N|2451270|2451359|2450916|2451191|N|N|N|N|N| +2451282|AAAAAAAACFHGFCAA|1999-04-13|1191|5181|398|1999|2|4|13|2|1999|398|5181|Tuesday|1999Q2|N|N|N|2451270|2451359|2450917|2451192|N|N|N|N|N| +2451283|AAAAAAAADFHGFCAA|1999-04-14|1191|5181|398|1999|3|4|14|2|1999|398|5181|Wednesday|1999Q2|N|N|N|2451270|2451359|2450918|2451193|N|N|N|N|N| +2451284|AAAAAAAAEFHGFCAA|1999-04-15|1191|5181|398|1999|4|4|15|2|1999|398|5181|Thursday|1999Q2|N|N|N|2451270|2451359|2450919|2451194|N|N|N|N|N| +2451285|AAAAAAAAFFHGFCAA|1999-04-16|1191|5181|398|1999|5|4|16|2|1999|398|5181|Friday|1999Q2|N|Y|N|2451270|2451359|2450920|2451195|N|N|N|N|N| +2451286|AAAAAAAAGFHGFCAA|1999-04-17|1191|5181|398|1999|6|4|17|2|1999|398|5181|Saturday|1999Q2|N|Y|N|2451270|2451359|2450921|2451196|N|N|N|N|N| +2451287|AAAAAAAAHFHGFCAA|1999-04-18|1191|5181|398|1999|0|4|18|2|1999|398|5181|Sunday|1999Q2|N|N|N|2451270|2451359|2450922|2451197|N|N|N|N|N| +2451288|AAAAAAAAIFHGFCAA|1999-04-19|1191|5181|398|1999|1|4|19|2|1999|398|5181|Monday|1999Q2|N|N|N|2451270|2451359|2450923|2451198|N|N|N|N|N| +2451289|AAAAAAAAJFHGFCAA|1999-04-20|1191|5182|398|1999|2|4|20|2|1999|398|5182|Tuesday|1999Q2|N|N|N|2451270|2451359|2450924|2451199|N|N|N|N|N| +2451290|AAAAAAAAKFHGFCAA|1999-04-21|1191|5182|398|1999|3|4|21|2|1999|398|5182|Wednesday|1999Q2|N|N|N|2451270|2451359|2450925|2451200|N|N|N|N|N| +2451291|AAAAAAAALFHGFCAA|1999-04-22|1191|5182|398|1999|4|4|22|2|1999|398|5182|Thursday|1999Q2|N|N|N|2451270|2451359|2450926|2451201|N|N|N|N|N| +2451292|AAAAAAAAMFHGFCAA|1999-04-23|1191|5182|398|1999|5|4|23|2|1999|398|5182|Friday|1999Q2|N|Y|N|2451270|2451359|2450927|2451202|N|N|N|N|N| +2451293|AAAAAAAANFHGFCAA|1999-04-24|1191|5182|398|1999|6|4|24|2|1999|398|5182|Saturday|1999Q2|N|Y|N|2451270|2451359|2450928|2451203|N|N|N|N|N| +2451294|AAAAAAAAOFHGFCAA|1999-04-25|1191|5182|398|1999|0|4|25|2|1999|398|5182|Sunday|1999Q2|N|N|N|2451270|2451359|2450929|2451204|N|N|N|N|N| +2451295|AAAAAAAAPFHGFCAA|1999-04-26|1191|5182|398|1999|1|4|26|2|1999|398|5182|Monday|1999Q2|N|N|N|2451270|2451359|2450930|2451205|N|N|N|N|N| +2451296|AAAAAAAAAGHGFCAA|1999-04-27|1191|5183|398|1999|2|4|27|2|1999|398|5183|Tuesday|1999Q2|N|N|N|2451270|2451359|2450931|2451206|N|N|N|N|N| +2451297|AAAAAAAABGHGFCAA|1999-04-28|1191|5183|398|1999|3|4|28|2|1999|398|5183|Wednesday|1999Q2|N|N|N|2451270|2451359|2450932|2451207|N|N|N|N|N| +2451298|AAAAAAAACGHGFCAA|1999-04-29|1191|5183|398|1999|4|4|29|2|1999|398|5183|Thursday|1999Q2|N|N|N|2451270|2451359|2450933|2451208|N|N|N|N|N| +2451299|AAAAAAAADGHGFCAA|1999-04-30|1191|5183|398|1999|5|4|30|2|1999|398|5183|Friday|1999Q2|N|Y|N|2451270|2451359|2450934|2451209|N|N|N|N|N| +2451300|AAAAAAAAEGHGFCAA|1999-05-01|1192|5183|398|1999|6|5|1|2|1999|398|5183|Saturday|1999Q2|N|Y|N|2451300|2451419|2450935|2451210|N|N|N|N|N| +2451301|AAAAAAAAFGHGFCAA|1999-05-02|1192|5183|398|1999|0|5|2|2|1999|398|5183|Sunday|1999Q2|N|N|N|2451300|2451419|2450936|2451211|N|N|N|N|N| +2451302|AAAAAAAAGGHGFCAA|1999-05-03|1192|5183|398|1999|1|5|3|2|1999|398|5183|Monday|1999Q2|N|N|N|2451300|2451419|2450937|2451212|N|N|N|N|N| +2451303|AAAAAAAAHGHGFCAA|1999-05-04|1192|5184|398|1999|2|5|4|2|1999|398|5184|Tuesday|1999Q2|N|N|N|2451300|2451419|2450938|2451213|N|N|N|N|N| +2451304|AAAAAAAAIGHGFCAA|1999-05-05|1192|5184|398|1999|3|5|5|2|1999|398|5184|Wednesday|1999Q2|N|N|N|2451300|2451419|2450939|2451214|N|N|N|N|N| +2451305|AAAAAAAAJGHGFCAA|1999-05-06|1192|5184|398|1999|4|5|6|2|1999|398|5184|Thursday|1999Q2|N|N|N|2451300|2451419|2450940|2451215|N|N|N|N|N| +2451306|AAAAAAAAKGHGFCAA|1999-05-07|1192|5184|398|1999|5|5|7|2|1999|398|5184|Friday|1999Q2|N|Y|N|2451300|2451419|2450941|2451216|N|N|N|N|N| +2451307|AAAAAAAALGHGFCAA|1999-05-08|1192|5184|398|1999|6|5|8|2|1999|398|5184|Saturday|1999Q2|N|Y|N|2451300|2451419|2450942|2451217|N|N|N|N|N| +2451308|AAAAAAAAMGHGFCAA|1999-05-09|1192|5184|398|1999|0|5|9|2|1999|398|5184|Sunday|1999Q2|N|N|N|2451300|2451419|2450943|2451218|N|N|N|N|N| +2451309|AAAAAAAANGHGFCAA|1999-05-10|1192|5184|398|1999|1|5|10|2|1999|398|5184|Monday|1999Q2|N|N|N|2451300|2451419|2450944|2451219|N|N|N|N|N| +2451310|AAAAAAAAOGHGFCAA|1999-05-11|1192|5185|398|1999|2|5|11|2|1999|398|5185|Tuesday|1999Q2|N|N|N|2451300|2451419|2450945|2451220|N|N|N|N|N| +2451311|AAAAAAAAPGHGFCAA|1999-05-12|1192|5185|398|1999|3|5|12|2|1999|398|5185|Wednesday|1999Q2|N|N|N|2451300|2451419|2450946|2451221|N|N|N|N|N| +2451312|AAAAAAAAAHHGFCAA|1999-05-13|1192|5185|398|1999|4|5|13|2|1999|398|5185|Thursday|1999Q2|N|N|N|2451300|2451419|2450947|2451222|N|N|N|N|N| +2451313|AAAAAAAABHHGFCAA|1999-05-14|1192|5185|398|1999|5|5|14|2|1999|398|5185|Friday|1999Q2|N|Y|N|2451300|2451419|2450948|2451223|N|N|N|N|N| +2451314|AAAAAAAACHHGFCAA|1999-05-15|1192|5185|398|1999|6|5|15|2|1999|398|5185|Saturday|1999Q2|N|Y|N|2451300|2451419|2450949|2451224|N|N|N|N|N| +2451315|AAAAAAAADHHGFCAA|1999-05-16|1192|5185|398|1999|0|5|16|2|1999|398|5185|Sunday|1999Q2|N|N|N|2451300|2451419|2450950|2451225|N|N|N|N|N| +2451316|AAAAAAAAEHHGFCAA|1999-05-17|1192|5185|398|1999|1|5|17|2|1999|398|5185|Monday|1999Q2|N|N|N|2451300|2451419|2450951|2451226|N|N|N|N|N| +2451317|AAAAAAAAFHHGFCAA|1999-05-18|1192|5186|398|1999|2|5|18|2|1999|398|5186|Tuesday|1999Q2|N|N|N|2451300|2451419|2450952|2451227|N|N|N|N|N| +2451318|AAAAAAAAGHHGFCAA|1999-05-19|1192|5186|398|1999|3|5|19|2|1999|398|5186|Wednesday|1999Q2|N|N|N|2451300|2451419|2450953|2451228|N|N|N|N|N| +2451319|AAAAAAAAHHHGFCAA|1999-05-20|1192|5186|398|1999|4|5|20|2|1999|398|5186|Thursday|1999Q2|N|N|N|2451300|2451419|2450954|2451229|N|N|N|N|N| +2451320|AAAAAAAAIHHGFCAA|1999-05-21|1192|5186|398|1999|5|5|21|2|1999|398|5186|Friday|1999Q2|N|Y|N|2451300|2451419|2450955|2451230|N|N|N|N|N| +2451321|AAAAAAAAJHHGFCAA|1999-05-22|1192|5186|398|1999|6|5|22|2|1999|398|5186|Saturday|1999Q2|N|Y|N|2451300|2451419|2450956|2451231|N|N|N|N|N| +2451322|AAAAAAAAKHHGFCAA|1999-05-23|1192|5186|398|1999|0|5|23|2|1999|398|5186|Sunday|1999Q2|N|N|N|2451300|2451419|2450957|2451232|N|N|N|N|N| +2451323|AAAAAAAALHHGFCAA|1999-05-24|1192|5186|398|1999|1|5|24|2|1999|398|5186|Monday|1999Q2|N|N|N|2451300|2451419|2450958|2451233|N|N|N|N|N| +2451324|AAAAAAAAMHHGFCAA|1999-05-25|1192|5187|398|1999|2|5|25|2|1999|398|5187|Tuesday|1999Q2|N|N|N|2451300|2451419|2450959|2451234|N|N|N|N|N| +2451325|AAAAAAAANHHGFCAA|1999-05-26|1192|5187|398|1999|3|5|26|2|1999|398|5187|Wednesday|1999Q2|N|N|N|2451300|2451419|2450960|2451235|N|N|N|N|N| +2451326|AAAAAAAAOHHGFCAA|1999-05-27|1192|5187|398|1999|4|5|27|2|1999|398|5187|Thursday|1999Q2|N|N|N|2451300|2451419|2450961|2451236|N|N|N|N|N| +2451327|AAAAAAAAPHHGFCAA|1999-05-28|1192|5187|398|1999|5|5|28|2|1999|398|5187|Friday|1999Q2|N|Y|N|2451300|2451419|2450962|2451237|N|N|N|N|N| +2451328|AAAAAAAAAIHGFCAA|1999-05-29|1192|5187|398|1999|6|5|29|2|1999|398|5187|Saturday|1999Q2|N|Y|N|2451300|2451419|2450963|2451238|N|N|N|N|N| +2451329|AAAAAAAABIHGFCAA|1999-05-30|1192|5187|398|1999|0|5|30|2|1999|398|5187|Sunday|1999Q2|N|N|N|2451300|2451419|2450964|2451239|N|N|N|N|N| +2451330|AAAAAAAACIHGFCAA|1999-05-31|1192|5187|398|1999|1|5|31|2|1999|398|5187|Monday|1999Q2|N|N|N|2451300|2451419|2450965|2451240|N|N|N|N|N| +2451331|AAAAAAAADIHGFCAA|1999-06-01|1193|5188|399|1999|2|6|1|2|1999|399|5188|Tuesday|1999Q2|N|N|N|2451331|2451481|2450966|2451241|N|N|N|N|N| +2451332|AAAAAAAAEIHGFCAA|1999-06-02|1193|5188|399|1999|3|6|2|2|1999|399|5188|Wednesday|1999Q2|N|N|N|2451331|2451481|2450967|2451242|N|N|N|N|N| +2451333|AAAAAAAAFIHGFCAA|1999-06-03|1193|5188|399|1999|4|6|3|2|1999|399|5188|Thursday|1999Q2|N|N|N|2451331|2451481|2450968|2451243|N|N|N|N|N| +2451334|AAAAAAAAGIHGFCAA|1999-06-04|1193|5188|399|1999|5|6|4|2|1999|399|5188|Friday|1999Q2|N|Y|N|2451331|2451481|2450969|2451244|N|N|N|N|N| +2451335|AAAAAAAAHIHGFCAA|1999-06-05|1193|5188|399|1999|6|6|5|2|1999|399|5188|Saturday|1999Q2|N|Y|N|2451331|2451481|2450970|2451245|N|N|N|N|N| +2451336|AAAAAAAAIIHGFCAA|1999-06-06|1193|5188|399|1999|0|6|6|2|1999|399|5188|Sunday|1999Q2|N|N|N|2451331|2451481|2450971|2451246|N|N|N|N|N| +2451337|AAAAAAAAJIHGFCAA|1999-06-07|1193|5188|399|1999|1|6|7|2|1999|399|5188|Monday|1999Q2|N|N|N|2451331|2451481|2450972|2451247|N|N|N|N|N| +2451338|AAAAAAAAKIHGFCAA|1999-06-08|1193|5189|399|1999|2|6|8|2|1999|399|5189|Tuesday|1999Q2|N|N|N|2451331|2451481|2450973|2451248|N|N|N|N|N| +2451339|AAAAAAAALIHGFCAA|1999-06-09|1193|5189|399|1999|3|6|9|2|1999|399|5189|Wednesday|1999Q2|N|N|N|2451331|2451481|2450974|2451249|N|N|N|N|N| +2451340|AAAAAAAAMIHGFCAA|1999-06-10|1193|5189|399|1999|4|6|10|2|1999|399|5189|Thursday|1999Q2|N|N|N|2451331|2451481|2450975|2451250|N|N|N|N|N| +2451341|AAAAAAAANIHGFCAA|1999-06-11|1193|5189|399|1999|5|6|11|2|1999|399|5189|Friday|1999Q2|N|Y|N|2451331|2451481|2450976|2451251|N|N|N|N|N| +2451342|AAAAAAAAOIHGFCAA|1999-06-12|1193|5189|399|1999|6|6|12|2|1999|399|5189|Saturday|1999Q2|N|Y|N|2451331|2451481|2450977|2451252|N|N|N|N|N| +2451343|AAAAAAAAPIHGFCAA|1999-06-13|1193|5189|399|1999|0|6|13|2|1999|399|5189|Sunday|1999Q2|N|N|N|2451331|2451481|2450978|2451253|N|N|N|N|N| +2451344|AAAAAAAAAJHGFCAA|1999-06-14|1193|5189|399|1999|1|6|14|2|1999|399|5189|Monday|1999Q2|N|N|N|2451331|2451481|2450979|2451254|N|N|N|N|N| +2451345|AAAAAAAABJHGFCAA|1999-06-15|1193|5190|399|1999|2|6|15|2|1999|399|5190|Tuesday|1999Q2|N|N|N|2451331|2451481|2450980|2451255|N|N|N|N|N| +2451346|AAAAAAAACJHGFCAA|1999-06-16|1193|5190|399|1999|3|6|16|2|1999|399|5190|Wednesday|1999Q2|N|N|N|2451331|2451481|2450981|2451256|N|N|N|N|N| +2451347|AAAAAAAADJHGFCAA|1999-06-17|1193|5190|399|1999|4|6|17|2|1999|399|5190|Thursday|1999Q2|N|N|N|2451331|2451481|2450982|2451257|N|N|N|N|N| +2451348|AAAAAAAAEJHGFCAA|1999-06-18|1193|5190|399|1999|5|6|18|2|1999|399|5190|Friday|1999Q2|N|Y|N|2451331|2451481|2450983|2451258|N|N|N|N|N| +2451349|AAAAAAAAFJHGFCAA|1999-06-19|1193|5190|399|1999|6|6|19|2|1999|399|5190|Saturday|1999Q2|N|Y|N|2451331|2451481|2450984|2451259|N|N|N|N|N| +2451350|AAAAAAAAGJHGFCAA|1999-06-20|1193|5190|399|1999|0|6|20|2|1999|399|5190|Sunday|1999Q2|N|N|N|2451331|2451481|2450985|2451260|N|N|N|N|N| +2451351|AAAAAAAAHJHGFCAA|1999-06-21|1193|5190|399|1999|1|6|21|2|1999|399|5190|Monday|1999Q2|N|N|N|2451331|2451481|2450986|2451261|N|N|N|N|N| +2451352|AAAAAAAAIJHGFCAA|1999-06-22|1193|5191|399|1999|2|6|22|2|1999|399|5191|Tuesday|1999Q2|N|N|N|2451331|2451481|2450987|2451262|N|N|N|N|N| +2451353|AAAAAAAAJJHGFCAA|1999-06-23|1193|5191|399|1999|3|6|23|2|1999|399|5191|Wednesday|1999Q2|N|N|N|2451331|2451481|2450988|2451263|N|N|N|N|N| +2451354|AAAAAAAAKJHGFCAA|1999-06-24|1193|5191|399|1999|4|6|24|2|1999|399|5191|Thursday|1999Q2|N|N|N|2451331|2451481|2450989|2451264|N|N|N|N|N| +2451355|AAAAAAAALJHGFCAA|1999-06-25|1193|5191|399|1999|5|6|25|2|1999|399|5191|Friday|1999Q2|N|Y|N|2451331|2451481|2450990|2451265|N|N|N|N|N| +2451356|AAAAAAAAMJHGFCAA|1999-06-26|1193|5191|399|1999|6|6|26|2|1999|399|5191|Saturday|1999Q2|N|Y|N|2451331|2451481|2450991|2451266|N|N|N|N|N| +2451357|AAAAAAAANJHGFCAA|1999-06-27|1193|5191|399|1999|0|6|27|2|1999|399|5191|Sunday|1999Q2|N|N|N|2451331|2451481|2450992|2451267|N|N|N|N|N| +2451358|AAAAAAAAOJHGFCAA|1999-06-28|1193|5191|399|1999|1|6|28|2|1999|399|5191|Monday|1999Q2|N|N|N|2451331|2451481|2450993|2451268|N|N|N|N|N| +2451359|AAAAAAAAPJHGFCAA|1999-06-29|1193|5192|399|1999|2|6|29|2|1999|399|5192|Tuesday|1999Q2|N|N|N|2451331|2451481|2450994|2451269|N|N|N|N|N| +2451360|AAAAAAAAAKHGFCAA|1999-06-30|1193|5192|399|1999|3|6|30|2|1999|399|5192|Wednesday|1999Q2|N|N|N|2451331|2451481|2450995|2451270|N|N|N|N|N| +2451361|AAAAAAAABKHGFCAA|1999-07-01|1194|5192|399|1999|4|7|1|2|1999|399|5192|Thursday|1999Q2|N|N|N|2451361|2451541|2450996|2451270|N|N|N|N|N| +2451362|AAAAAAAACKHGFCAA|1999-07-02|1194|5192|399|1999|5|7|2|3|1999|399|5192|Friday|1999Q3|N|Y|N|2451361|2451541|2450997|2451271|N|N|N|N|N| +2451363|AAAAAAAADKHGFCAA|1999-07-03|1194|5192|399|1999|6|7|3|3|1999|399|5192|Saturday|1999Q3|N|Y|N|2451361|2451541|2450998|2451272|N|N|N|N|N| +2451364|AAAAAAAAEKHGFCAA|1999-07-04|1194|5192|399|1999|0|7|4|3|1999|399|5192|Sunday|1999Q3|N|N|N|2451361|2451541|2450999|2451273|N|N|N|N|N| +2451365|AAAAAAAAFKHGFCAA|1999-07-05|1194|5192|399|1999|1|7|5|3|1999|399|5192|Monday|1999Q3|Y|N|N|2451361|2451541|2451000|2451274|N|N|N|N|N| +2451366|AAAAAAAAGKHGFCAA|1999-07-06|1194|5193|399|1999|2|7|6|3|1999|399|5193|Tuesday|1999Q3|N|N|Y|2451361|2451541|2451001|2451275|N|N|N|N|N| +2451367|AAAAAAAAHKHGFCAA|1999-07-07|1194|5193|399|1999|3|7|7|3|1999|399|5193|Wednesday|1999Q3|N|N|N|2451361|2451541|2451002|2451276|N|N|N|N|N| +2451368|AAAAAAAAIKHGFCAA|1999-07-08|1194|5193|399|1999|4|7|8|3|1999|399|5193|Thursday|1999Q3|N|N|N|2451361|2451541|2451003|2451277|N|N|N|N|N| +2451369|AAAAAAAAJKHGFCAA|1999-07-09|1194|5193|399|1999|5|7|9|3|1999|399|5193|Friday|1999Q3|N|Y|N|2451361|2451541|2451004|2451278|N|N|N|N|N| +2451370|AAAAAAAAKKHGFCAA|1999-07-10|1194|5193|399|1999|6|7|10|3|1999|399|5193|Saturday|1999Q3|N|Y|N|2451361|2451541|2451005|2451279|N|N|N|N|N| +2451371|AAAAAAAALKHGFCAA|1999-07-11|1194|5193|399|1999|0|7|11|3|1999|399|5193|Sunday|1999Q3|N|N|N|2451361|2451541|2451006|2451280|N|N|N|N|N| +2451372|AAAAAAAAMKHGFCAA|1999-07-12|1194|5193|399|1999|1|7|12|3|1999|399|5193|Monday|1999Q3|N|N|N|2451361|2451541|2451007|2451281|N|N|N|N|N| +2451373|AAAAAAAANKHGFCAA|1999-07-13|1194|5194|399|1999|2|7|13|3|1999|399|5194|Tuesday|1999Q3|N|N|N|2451361|2451541|2451008|2451282|N|N|N|N|N| +2451374|AAAAAAAAOKHGFCAA|1999-07-14|1194|5194|399|1999|3|7|14|3|1999|399|5194|Wednesday|1999Q3|N|N|N|2451361|2451541|2451009|2451283|N|N|N|N|N| +2451375|AAAAAAAAPKHGFCAA|1999-07-15|1194|5194|399|1999|4|7|15|3|1999|399|5194|Thursday|1999Q3|N|N|N|2451361|2451541|2451010|2451284|N|N|N|N|N| +2451376|AAAAAAAAALHGFCAA|1999-07-16|1194|5194|399|1999|5|7|16|3|1999|399|5194|Friday|1999Q3|N|Y|N|2451361|2451541|2451011|2451285|N|N|N|N|N| +2451377|AAAAAAAABLHGFCAA|1999-07-17|1194|5194|399|1999|6|7|17|3|1999|399|5194|Saturday|1999Q3|N|Y|N|2451361|2451541|2451012|2451286|N|N|N|N|N| +2451378|AAAAAAAACLHGFCAA|1999-07-18|1194|5194|399|1999|0|7|18|3|1999|399|5194|Sunday|1999Q3|N|N|N|2451361|2451541|2451013|2451287|N|N|N|N|N| +2451379|AAAAAAAADLHGFCAA|1999-07-19|1194|5194|399|1999|1|7|19|3|1999|399|5194|Monday|1999Q3|N|N|N|2451361|2451541|2451014|2451288|N|N|N|N|N| +2451380|AAAAAAAAELHGFCAA|1999-07-20|1194|5195|399|1999|2|7|20|3|1999|399|5195|Tuesday|1999Q3|N|N|N|2451361|2451541|2451015|2451289|N|N|N|N|N| +2451381|AAAAAAAAFLHGFCAA|1999-07-21|1194|5195|399|1999|3|7|21|3|1999|399|5195|Wednesday|1999Q3|N|N|N|2451361|2451541|2451016|2451290|N|N|N|N|N| +2451382|AAAAAAAAGLHGFCAA|1999-07-22|1194|5195|399|1999|4|7|22|3|1999|399|5195|Thursday|1999Q3|N|N|N|2451361|2451541|2451017|2451291|N|N|N|N|N| +2451383|AAAAAAAAHLHGFCAA|1999-07-23|1194|5195|399|1999|5|7|23|3|1999|399|5195|Friday|1999Q3|N|Y|N|2451361|2451541|2451018|2451292|N|N|N|N|N| +2451384|AAAAAAAAILHGFCAA|1999-07-24|1194|5195|399|1999|6|7|24|3|1999|399|5195|Saturday|1999Q3|N|Y|N|2451361|2451541|2451019|2451293|N|N|N|N|N| +2451385|AAAAAAAAJLHGFCAA|1999-07-25|1194|5195|399|1999|0|7|25|3|1999|399|5195|Sunday|1999Q3|N|N|N|2451361|2451541|2451020|2451294|N|N|N|N|N| +2451386|AAAAAAAAKLHGFCAA|1999-07-26|1194|5195|399|1999|1|7|26|3|1999|399|5195|Monday|1999Q3|N|N|N|2451361|2451541|2451021|2451295|N|N|N|N|N| +2451387|AAAAAAAALLHGFCAA|1999-07-27|1194|5196|399|1999|2|7|27|3|1999|399|5196|Tuesday|1999Q3|N|N|N|2451361|2451541|2451022|2451296|N|N|N|N|N| +2451388|AAAAAAAAMLHGFCAA|1999-07-28|1194|5196|399|1999|3|7|28|3|1999|399|5196|Wednesday|1999Q3|N|N|N|2451361|2451541|2451023|2451297|N|N|N|N|N| +2451389|AAAAAAAANLHGFCAA|1999-07-29|1194|5196|399|1999|4|7|29|3|1999|399|5196|Thursday|1999Q3|N|N|N|2451361|2451541|2451024|2451298|N|N|N|N|N| +2451390|AAAAAAAAOLHGFCAA|1999-07-30|1194|5196|399|1999|5|7|30|3|1999|399|5196|Friday|1999Q3|N|Y|N|2451361|2451541|2451025|2451299|N|N|N|N|N| +2451391|AAAAAAAAPLHGFCAA|1999-07-31|1194|5196|399|1999|6|7|31|3|1999|399|5196|Saturday|1999Q3|N|Y|N|2451361|2451541|2451026|2451300|N|N|N|N|N| +2451392|AAAAAAAAAMHGFCAA|1999-08-01|1195|5196|399|1999|0|8|1|3|1999|399|5196|Sunday|1999Q3|N|N|N|2451392|2451603|2451027|2451301|N|N|N|N|N| +2451393|AAAAAAAABMHGFCAA|1999-08-02|1195|5196|399|1999|1|8|2|3|1999|399|5196|Monday|1999Q3|N|N|N|2451392|2451603|2451028|2451302|N|N|N|N|N| +2451394|AAAAAAAACMHGFCAA|1999-08-03|1195|5197|399|1999|2|8|3|3|1999|399|5197|Tuesday|1999Q3|N|N|N|2451392|2451603|2451029|2451303|N|N|N|N|N| +2451395|AAAAAAAADMHGFCAA|1999-08-04|1195|5197|399|1999|3|8|4|3|1999|399|5197|Wednesday|1999Q3|N|N|N|2451392|2451603|2451030|2451304|N|N|N|N|N| +2451396|AAAAAAAAEMHGFCAA|1999-08-05|1195|5197|399|1999|4|8|5|3|1999|399|5197|Thursday|1999Q3|N|N|N|2451392|2451603|2451031|2451305|N|N|N|N|N| +2451397|AAAAAAAAFMHGFCAA|1999-08-06|1195|5197|399|1999|5|8|6|3|1999|399|5197|Friday|1999Q3|N|Y|N|2451392|2451603|2451032|2451306|N|N|N|N|N| +2451398|AAAAAAAAGMHGFCAA|1999-08-07|1195|5197|399|1999|6|8|7|3|1999|399|5197|Saturday|1999Q3|N|Y|N|2451392|2451603|2451033|2451307|N|N|N|N|N| +2451399|AAAAAAAAHMHGFCAA|1999-08-08|1195|5197|399|1999|0|8|8|3|1999|399|5197|Sunday|1999Q3|N|N|N|2451392|2451603|2451034|2451308|N|N|N|N|N| +2451400|AAAAAAAAIMHGFCAA|1999-08-09|1195|5197|399|1999|1|8|9|3|1999|399|5197|Monday|1999Q3|N|N|N|2451392|2451603|2451035|2451309|N|N|N|N|N| +2451401|AAAAAAAAJMHGFCAA|1999-08-10|1195|5198|399|1999|2|8|10|3|1999|399|5198|Tuesday|1999Q3|N|N|N|2451392|2451603|2451036|2451310|N|N|N|N|N| +2451402|AAAAAAAAKMHGFCAA|1999-08-11|1195|5198|399|1999|3|8|11|3|1999|399|5198|Wednesday|1999Q3|N|N|N|2451392|2451603|2451037|2451311|N|N|N|N|N| +2451403|AAAAAAAALMHGFCAA|1999-08-12|1195|5198|399|1999|4|8|12|3|1999|399|5198|Thursday|1999Q3|N|N|N|2451392|2451603|2451038|2451312|N|N|N|N|N| +2451404|AAAAAAAAMMHGFCAA|1999-08-13|1195|5198|399|1999|5|8|13|3|1999|399|5198|Friday|1999Q3|N|Y|N|2451392|2451603|2451039|2451313|N|N|N|N|N| +2451405|AAAAAAAANMHGFCAA|1999-08-14|1195|5198|399|1999|6|8|14|3|1999|399|5198|Saturday|1999Q3|N|Y|N|2451392|2451603|2451040|2451314|N|N|N|N|N| +2451406|AAAAAAAAOMHGFCAA|1999-08-15|1195|5198|399|1999|0|8|15|3|1999|399|5198|Sunday|1999Q3|N|N|N|2451392|2451603|2451041|2451315|N|N|N|N|N| +2451407|AAAAAAAAPMHGFCAA|1999-08-16|1195|5198|399|1999|1|8|16|3|1999|399|5198|Monday|1999Q3|N|N|N|2451392|2451603|2451042|2451316|N|N|N|N|N| +2451408|AAAAAAAAANHGFCAA|1999-08-17|1195|5199|399|1999|2|8|17|3|1999|399|5199|Tuesday|1999Q3|N|N|N|2451392|2451603|2451043|2451317|N|N|N|N|N| +2451409|AAAAAAAABNHGFCAA|1999-08-18|1195|5199|399|1999|3|8|18|3|1999|399|5199|Wednesday|1999Q3|N|N|N|2451392|2451603|2451044|2451318|N|N|N|N|N| +2451410|AAAAAAAACNHGFCAA|1999-08-19|1195|5199|399|1999|4|8|19|3|1999|399|5199|Thursday|1999Q3|N|N|N|2451392|2451603|2451045|2451319|N|N|N|N|N| +2451411|AAAAAAAADNHGFCAA|1999-08-20|1195|5199|399|1999|5|8|20|3|1999|399|5199|Friday|1999Q3|N|Y|N|2451392|2451603|2451046|2451320|N|N|N|N|N| +2451412|AAAAAAAAENHGFCAA|1999-08-21|1195|5199|399|1999|6|8|21|3|1999|399|5199|Saturday|1999Q3|N|Y|N|2451392|2451603|2451047|2451321|N|N|N|N|N| +2451413|AAAAAAAAFNHGFCAA|1999-08-22|1195|5199|399|1999|0|8|22|3|1999|399|5199|Sunday|1999Q3|N|N|N|2451392|2451603|2451048|2451322|N|N|N|N|N| +2451414|AAAAAAAAGNHGFCAA|1999-08-23|1195|5199|399|1999|1|8|23|3|1999|399|5199|Monday|1999Q3|N|N|N|2451392|2451603|2451049|2451323|N|N|N|N|N| +2451415|AAAAAAAAHNHGFCAA|1999-08-24|1195|5200|399|1999|2|8|24|3|1999|399|5200|Tuesday|1999Q3|N|N|N|2451392|2451603|2451050|2451324|N|N|N|N|N| +2451416|AAAAAAAAINHGFCAA|1999-08-25|1195|5200|399|1999|3|8|25|3|1999|399|5200|Wednesday|1999Q3|N|N|N|2451392|2451603|2451051|2451325|N|N|N|N|N| +2451417|AAAAAAAAJNHGFCAA|1999-08-26|1195|5200|399|1999|4|8|26|3|1999|399|5200|Thursday|1999Q3|N|N|N|2451392|2451603|2451052|2451326|N|N|N|N|N| +2451418|AAAAAAAAKNHGFCAA|1999-08-27|1195|5200|399|1999|5|8|27|3|1999|399|5200|Friday|1999Q3|N|Y|N|2451392|2451603|2451053|2451327|N|N|N|N|N| +2451419|AAAAAAAALNHGFCAA|1999-08-28|1195|5200|399|1999|6|8|28|3|1999|399|5200|Saturday|1999Q3|N|Y|N|2451392|2451603|2451054|2451328|N|N|N|N|N| +2451420|AAAAAAAAMNHGFCAA|1999-08-29|1195|5200|399|1999|0|8|29|3|1999|399|5200|Sunday|1999Q3|N|N|N|2451392|2451603|2451055|2451329|N|N|N|N|N| +2451421|AAAAAAAANNHGFCAA|1999-08-30|1195|5200|399|1999|1|8|30|3|1999|399|5200|Monday|1999Q3|N|N|N|2451392|2451603|2451056|2451330|N|N|N|N|N| +2451422|AAAAAAAAONHGFCAA|1999-08-31|1195|5201|399|1999|2|8|31|3|1999|399|5201|Tuesday|1999Q3|N|N|N|2451392|2451603|2451057|2451331|N|N|N|N|N| +2451423|AAAAAAAAPNHGFCAA|1999-09-01|1196|5201|400|1999|3|9|1|3|1999|400|5201|Wednesday|1999Q3|N|N|N|2451423|2451665|2451058|2451332|N|N|N|N|N| +2451424|AAAAAAAAAOHGFCAA|1999-09-02|1196|5201|400|1999|4|9|2|3|1999|400|5201|Thursday|1999Q3|N|N|N|2451423|2451665|2451059|2451333|N|N|N|N|N| +2451425|AAAAAAAABOHGFCAA|1999-09-03|1196|5201|400|1999|5|9|3|3|1999|400|5201|Friday|1999Q3|N|Y|N|2451423|2451665|2451060|2451334|N|N|N|N|N| +2451426|AAAAAAAACOHGFCAA|1999-09-04|1196|5201|400|1999|6|9|4|3|1999|400|5201|Saturday|1999Q3|N|Y|N|2451423|2451665|2451061|2451335|N|N|N|N|N| +2451427|AAAAAAAADOHGFCAA|1999-09-05|1196|5201|400|1999|0|9|5|3|1999|400|5201|Sunday|1999Q3|N|N|N|2451423|2451665|2451062|2451336|N|N|N|N|N| +2451428|AAAAAAAAEOHGFCAA|1999-09-06|1196|5201|400|1999|1|9|6|3|1999|400|5201|Monday|1999Q3|N|N|N|2451423|2451665|2451063|2451337|N|N|N|N|N| +2451429|AAAAAAAAFOHGFCAA|1999-09-07|1196|5202|400|1999|2|9|7|3|1999|400|5202|Tuesday|1999Q3|N|N|N|2451423|2451665|2451064|2451338|N|N|N|N|N| +2451430|AAAAAAAAGOHGFCAA|1999-09-08|1196|5202|400|1999|3|9|8|3|1999|400|5202|Wednesday|1999Q3|N|N|N|2451423|2451665|2451065|2451339|N|N|N|N|N| +2451431|AAAAAAAAHOHGFCAA|1999-09-09|1196|5202|400|1999|4|9|9|3|1999|400|5202|Thursday|1999Q3|N|N|N|2451423|2451665|2451066|2451340|N|N|N|N|N| +2451432|AAAAAAAAIOHGFCAA|1999-09-10|1196|5202|400|1999|5|9|10|3|1999|400|5202|Friday|1999Q3|N|Y|N|2451423|2451665|2451067|2451341|N|N|N|N|N| +2451433|AAAAAAAAJOHGFCAA|1999-09-11|1196|5202|400|1999|6|9|11|3|1999|400|5202|Saturday|1999Q3|N|Y|N|2451423|2451665|2451068|2451342|N|N|N|N|N| +2451434|AAAAAAAAKOHGFCAA|1999-09-12|1196|5202|400|1999|0|9|12|3|1999|400|5202|Sunday|1999Q3|N|N|N|2451423|2451665|2451069|2451343|N|N|N|N|N| +2451435|AAAAAAAALOHGFCAA|1999-09-13|1196|5202|400|1999|1|9|13|3|1999|400|5202|Monday|1999Q3|N|N|N|2451423|2451665|2451070|2451344|N|N|N|N|N| +2451436|AAAAAAAAMOHGFCAA|1999-09-14|1196|5203|400|1999|2|9|14|3|1999|400|5203|Tuesday|1999Q3|N|N|N|2451423|2451665|2451071|2451345|N|N|N|N|N| +2451437|AAAAAAAANOHGFCAA|1999-09-15|1196|5203|400|1999|3|9|15|3|1999|400|5203|Wednesday|1999Q3|N|N|N|2451423|2451665|2451072|2451346|N|N|N|N|N| +2451438|AAAAAAAAOOHGFCAA|1999-09-16|1196|5203|400|1999|4|9|16|3|1999|400|5203|Thursday|1999Q3|N|N|N|2451423|2451665|2451073|2451347|N|N|N|N|N| +2451439|AAAAAAAAPOHGFCAA|1999-09-17|1196|5203|400|1999|5|9|17|3|1999|400|5203|Friday|1999Q3|N|Y|N|2451423|2451665|2451074|2451348|N|N|N|N|N| +2451440|AAAAAAAAAPHGFCAA|1999-09-18|1196|5203|400|1999|6|9|18|3|1999|400|5203|Saturday|1999Q3|N|Y|N|2451423|2451665|2451075|2451349|N|N|N|N|N| +2451441|AAAAAAAABPHGFCAA|1999-09-19|1196|5203|400|1999|0|9|19|3|1999|400|5203|Sunday|1999Q3|N|N|N|2451423|2451665|2451076|2451350|N|N|N|N|N| +2451442|AAAAAAAACPHGFCAA|1999-09-20|1196|5203|400|1999|1|9|20|3|1999|400|5203|Monday|1999Q3|N|N|N|2451423|2451665|2451077|2451351|N|N|N|N|N| +2451443|AAAAAAAADPHGFCAA|1999-09-21|1196|5204|400|1999|2|9|21|3|1999|400|5204|Tuesday|1999Q3|N|N|N|2451423|2451665|2451078|2451352|N|N|N|N|N| +2451444|AAAAAAAAEPHGFCAA|1999-09-22|1196|5204|400|1999|3|9|22|3|1999|400|5204|Wednesday|1999Q3|N|N|N|2451423|2451665|2451079|2451353|N|N|N|N|N| +2451445|AAAAAAAAFPHGFCAA|1999-09-23|1196|5204|400|1999|4|9|23|3|1999|400|5204|Thursday|1999Q3|N|N|N|2451423|2451665|2451080|2451354|N|N|N|N|N| +2451446|AAAAAAAAGPHGFCAA|1999-09-24|1196|5204|400|1999|5|9|24|3|1999|400|5204|Friday|1999Q3|N|Y|N|2451423|2451665|2451081|2451355|N|N|N|N|N| +2451447|AAAAAAAAHPHGFCAA|1999-09-25|1196|5204|400|1999|6|9|25|3|1999|400|5204|Saturday|1999Q3|N|Y|N|2451423|2451665|2451082|2451356|N|N|N|N|N| +2451448|AAAAAAAAIPHGFCAA|1999-09-26|1196|5204|400|1999|0|9|26|3|1999|400|5204|Sunday|1999Q3|N|N|N|2451423|2451665|2451083|2451357|N|N|N|N|N| +2451449|AAAAAAAAJPHGFCAA|1999-09-27|1196|5204|400|1999|1|9|27|3|1999|400|5204|Monday|1999Q3|N|N|N|2451423|2451665|2451084|2451358|N|N|N|N|N| +2451450|AAAAAAAAKPHGFCAA|1999-09-28|1196|5205|400|1999|2|9|28|3|1999|400|5205|Tuesday|1999Q3|N|N|N|2451423|2451665|2451085|2451359|N|N|N|N|N| +2451451|AAAAAAAALPHGFCAA|1999-09-29|1196|5205|400|1999|3|9|29|3|1999|400|5205|Wednesday|1999Q3|N|N|N|2451423|2451665|2451086|2451360|N|N|N|N|N| +2451452|AAAAAAAAMPHGFCAA|1999-09-30|1196|5205|400|1999|4|9|30|3|1999|400|5205|Thursday|1999Q3|N|N|N|2451423|2451665|2451087|2451361|N|N|N|N|N| +2451453|AAAAAAAANPHGFCAA|1999-10-01|1197|5205|400|1999|5|10|1|3|1999|400|5205|Friday|1999Q3|N|Y|N|2451453|2451725|2451088|2451361|N|N|N|N|N| +2451454|AAAAAAAAOPHGFCAA|1999-10-02|1197|5205|400|1999|6|10|2|4|1999|400|5205|Saturday|1999Q4|N|Y|N|2451453|2451725|2451089|2451362|N|N|N|N|N| +2451455|AAAAAAAAPPHGFCAA|1999-10-03|1197|5205|400|1999|0|10|3|4|1999|400|5205|Sunday|1999Q4|N|N|N|2451453|2451725|2451090|2451363|N|N|N|N|N| +2451456|AAAAAAAAAAIGFCAA|1999-10-04|1197|5205|400|1999|1|10|4|4|1999|400|5205|Monday|1999Q4|N|N|N|2451453|2451725|2451091|2451364|N|N|N|N|N| +2451457|AAAAAAAABAIGFCAA|1999-10-05|1197|5206|400|1999|2|10|5|4|1999|400|5206|Tuesday|1999Q4|N|N|N|2451453|2451725|2451092|2451365|N|N|N|N|N| +2451458|AAAAAAAACAIGFCAA|1999-10-06|1197|5206|400|1999|3|10|6|4|1999|400|5206|Wednesday|1999Q4|N|N|N|2451453|2451725|2451093|2451366|N|N|N|N|N| +2451459|AAAAAAAADAIGFCAA|1999-10-07|1197|5206|400|1999|4|10|7|4|1999|400|5206|Thursday|1999Q4|N|N|N|2451453|2451725|2451094|2451367|N|N|N|N|N| +2451460|AAAAAAAAEAIGFCAA|1999-10-08|1197|5206|400|1999|5|10|8|4|1999|400|5206|Friday|1999Q4|N|Y|N|2451453|2451725|2451095|2451368|N|N|N|N|N| +2451461|AAAAAAAAFAIGFCAA|1999-10-09|1197|5206|400|1999|6|10|9|4|1999|400|5206|Saturday|1999Q4|N|Y|N|2451453|2451725|2451096|2451369|N|N|N|N|N| +2451462|AAAAAAAAGAIGFCAA|1999-10-10|1197|5206|400|1999|0|10|10|4|1999|400|5206|Sunday|1999Q4|N|N|N|2451453|2451725|2451097|2451370|N|N|N|N|N| +2451463|AAAAAAAAHAIGFCAA|1999-10-11|1197|5206|400|1999|1|10|11|4|1999|400|5206|Monday|1999Q4|N|N|N|2451453|2451725|2451098|2451371|N|N|N|N|N| +2451464|AAAAAAAAIAIGFCAA|1999-10-12|1197|5207|400|1999|2|10|12|4|1999|400|5207|Tuesday|1999Q4|N|N|N|2451453|2451725|2451099|2451372|N|N|N|N|N| +2451465|AAAAAAAAJAIGFCAA|1999-10-13|1197|5207|400|1999|3|10|13|4|1999|400|5207|Wednesday|1999Q4|N|N|N|2451453|2451725|2451100|2451373|N|N|N|N|N| +2451466|AAAAAAAAKAIGFCAA|1999-10-14|1197|5207|400|1999|4|10|14|4|1999|400|5207|Thursday|1999Q4|N|N|N|2451453|2451725|2451101|2451374|N|N|N|N|N| +2451467|AAAAAAAALAIGFCAA|1999-10-15|1197|5207|400|1999|5|10|15|4|1999|400|5207|Friday|1999Q4|N|Y|N|2451453|2451725|2451102|2451375|N|N|N|N|N| +2451468|AAAAAAAAMAIGFCAA|1999-10-16|1197|5207|400|1999|6|10|16|4|1999|400|5207|Saturday|1999Q4|N|Y|N|2451453|2451725|2451103|2451376|N|N|N|N|N| +2451469|AAAAAAAANAIGFCAA|1999-10-17|1197|5207|400|1999|0|10|17|4|1999|400|5207|Sunday|1999Q4|N|N|N|2451453|2451725|2451104|2451377|N|N|N|N|N| +2451470|AAAAAAAAOAIGFCAA|1999-10-18|1197|5207|400|1999|1|10|18|4|1999|400|5207|Monday|1999Q4|N|N|N|2451453|2451725|2451105|2451378|N|N|N|N|N| +2451471|AAAAAAAAPAIGFCAA|1999-10-19|1197|5208|400|1999|2|10|19|4|1999|400|5208|Tuesday|1999Q4|N|N|N|2451453|2451725|2451106|2451379|N|N|N|N|N| +2451472|AAAAAAAAABIGFCAA|1999-10-20|1197|5208|400|1999|3|10|20|4|1999|400|5208|Wednesday|1999Q4|N|N|N|2451453|2451725|2451107|2451380|N|N|N|N|N| +2451473|AAAAAAAABBIGFCAA|1999-10-21|1197|5208|400|1999|4|10|21|4|1999|400|5208|Thursday|1999Q4|N|N|N|2451453|2451725|2451108|2451381|N|N|N|N|N| +2451474|AAAAAAAACBIGFCAA|1999-10-22|1197|5208|400|1999|5|10|22|4|1999|400|5208|Friday|1999Q4|N|Y|N|2451453|2451725|2451109|2451382|N|N|N|N|N| +2451475|AAAAAAAADBIGFCAA|1999-10-23|1197|5208|400|1999|6|10|23|4|1999|400|5208|Saturday|1999Q4|N|Y|N|2451453|2451725|2451110|2451383|N|N|N|N|N| +2451476|AAAAAAAAEBIGFCAA|1999-10-24|1197|5208|400|1999|0|10|24|4|1999|400|5208|Sunday|1999Q4|N|N|N|2451453|2451725|2451111|2451384|N|N|N|N|N| +2451477|AAAAAAAAFBIGFCAA|1999-10-25|1197|5208|400|1999|1|10|25|4|1999|400|5208|Monday|1999Q4|N|N|N|2451453|2451725|2451112|2451385|N|N|N|N|N| +2451478|AAAAAAAAGBIGFCAA|1999-10-26|1197|5209|400|1999|2|10|26|4|1999|400|5209|Tuesday|1999Q4|N|N|N|2451453|2451725|2451113|2451386|N|N|N|N|N| +2451479|AAAAAAAAHBIGFCAA|1999-10-27|1197|5209|400|1999|3|10|27|4|1999|400|5209|Wednesday|1999Q4|N|N|N|2451453|2451725|2451114|2451387|N|N|N|N|N| +2451480|AAAAAAAAIBIGFCAA|1999-10-28|1197|5209|400|1999|4|10|28|4|1999|400|5209|Thursday|1999Q4|N|N|N|2451453|2451725|2451115|2451388|N|N|N|N|N| +2451481|AAAAAAAAJBIGFCAA|1999-10-29|1197|5209|400|1999|5|10|29|4|1999|400|5209|Friday|1999Q4|N|Y|N|2451453|2451725|2451116|2451389|N|N|N|N|N| +2451482|AAAAAAAAKBIGFCAA|1999-10-30|1197|5209|400|1999|6|10|30|4|1999|400|5209|Saturday|1999Q4|N|Y|N|2451453|2451725|2451117|2451390|N|N|N|N|N| +2451483|AAAAAAAALBIGFCAA|1999-10-31|1197|5209|400|1999|0|10|31|4|1999|400|5209|Sunday|1999Q4|N|N|N|2451453|2451725|2451118|2451391|N|N|N|N|N| +2451484|AAAAAAAAMBIGFCAA|1999-11-01|1198|5209|400|1999|1|11|1|4|1999|400|5209|Monday|1999Q4|N|N|N|2451484|2451787|2451119|2451392|N|N|N|N|N| +2451485|AAAAAAAANBIGFCAA|1999-11-02|1198|5210|400|1999|2|11|2|4|1999|400|5210|Tuesday|1999Q4|N|N|N|2451484|2451787|2451120|2451393|N|N|N|N|N| +2451486|AAAAAAAAOBIGFCAA|1999-11-03|1198|5210|400|1999|3|11|3|4|1999|400|5210|Wednesday|1999Q4|N|N|N|2451484|2451787|2451121|2451394|N|N|N|N|N| +2451487|AAAAAAAAPBIGFCAA|1999-11-04|1198|5210|400|1999|4|11|4|4|1999|400|5210|Thursday|1999Q4|N|N|N|2451484|2451787|2451122|2451395|N|N|N|N|N| +2451488|AAAAAAAAACIGFCAA|1999-11-05|1198|5210|400|1999|5|11|5|4|1999|400|5210|Friday|1999Q4|N|Y|N|2451484|2451787|2451123|2451396|N|N|N|N|N| +2451489|AAAAAAAABCIGFCAA|1999-11-06|1198|5210|400|1999|6|11|6|4|1999|400|5210|Saturday|1999Q4|N|Y|N|2451484|2451787|2451124|2451397|N|N|N|N|N| +2451490|AAAAAAAACCIGFCAA|1999-11-07|1198|5210|400|1999|0|11|7|4|1999|400|5210|Sunday|1999Q4|N|N|N|2451484|2451787|2451125|2451398|N|N|N|N|N| +2451491|AAAAAAAADCIGFCAA|1999-11-08|1198|5210|400|1999|1|11|8|4|1999|400|5210|Monday|1999Q4|N|N|N|2451484|2451787|2451126|2451399|N|N|N|N|N| +2451492|AAAAAAAAECIGFCAA|1999-11-09|1198|5211|400|1999|2|11|9|4|1999|400|5211|Tuesday|1999Q4|N|N|N|2451484|2451787|2451127|2451400|N|N|N|N|N| +2451493|AAAAAAAAFCIGFCAA|1999-11-10|1198|5211|400|1999|3|11|10|4|1999|400|5211|Wednesday|1999Q4|N|N|N|2451484|2451787|2451128|2451401|N|N|N|N|N| +2451494|AAAAAAAAGCIGFCAA|1999-11-11|1198|5211|400|1999|4|11|11|4|1999|400|5211|Thursday|1999Q4|N|N|N|2451484|2451787|2451129|2451402|N|N|N|N|N| +2451495|AAAAAAAAHCIGFCAA|1999-11-12|1198|5211|400|1999|5|11|12|4|1999|400|5211|Friday|1999Q4|N|Y|N|2451484|2451787|2451130|2451403|N|N|N|N|N| +2451496|AAAAAAAAICIGFCAA|1999-11-13|1198|5211|400|1999|6|11|13|4|1999|400|5211|Saturday|1999Q4|N|Y|N|2451484|2451787|2451131|2451404|N|N|N|N|N| +2451497|AAAAAAAAJCIGFCAA|1999-11-14|1198|5211|400|1999|0|11|14|4|1999|400|5211|Sunday|1999Q4|N|N|N|2451484|2451787|2451132|2451405|N|N|N|N|N| +2451498|AAAAAAAAKCIGFCAA|1999-11-15|1198|5211|400|1999|1|11|15|4|1999|400|5211|Monday|1999Q4|N|N|N|2451484|2451787|2451133|2451406|N|N|N|N|N| +2451499|AAAAAAAALCIGFCAA|1999-11-16|1198|5212|400|1999|2|11|16|4|1999|400|5212|Tuesday|1999Q4|N|N|N|2451484|2451787|2451134|2451407|N|N|N|N|N| +2451500|AAAAAAAAMCIGFCAA|1999-11-17|1198|5212|400|1999|3|11|17|4|1999|400|5212|Wednesday|1999Q4|N|N|N|2451484|2451787|2451135|2451408|N|N|N|N|N| +2451501|AAAAAAAANCIGFCAA|1999-11-18|1198|5212|400|1999|4|11|18|4|1999|400|5212|Thursday|1999Q4|N|N|N|2451484|2451787|2451136|2451409|N|N|N|N|N| +2451502|AAAAAAAAOCIGFCAA|1999-11-19|1198|5212|400|1999|5|11|19|4|1999|400|5212|Friday|1999Q4|N|Y|N|2451484|2451787|2451137|2451410|N|N|N|N|N| +2451503|AAAAAAAAPCIGFCAA|1999-11-20|1198|5212|400|1999|6|11|20|4|1999|400|5212|Saturday|1999Q4|N|Y|N|2451484|2451787|2451138|2451411|N|N|N|N|N| +2451504|AAAAAAAAADIGFCAA|1999-11-21|1198|5212|400|1999|0|11|21|4|1999|400|5212|Sunday|1999Q4|N|N|N|2451484|2451787|2451139|2451412|N|N|N|N|N| +2451505|AAAAAAAABDIGFCAA|1999-11-22|1198|5212|400|1999|1|11|22|4|1999|400|5212|Monday|1999Q4|N|N|N|2451484|2451787|2451140|2451413|N|N|N|N|N| +2451506|AAAAAAAACDIGFCAA|1999-11-23|1198|5213|400|1999|2|11|23|4|1999|400|5213|Tuesday|1999Q4|N|N|N|2451484|2451787|2451141|2451414|N|N|N|N|N| +2451507|AAAAAAAADDIGFCAA|1999-11-24|1198|5213|400|1999|3|11|24|4|1999|400|5213|Wednesday|1999Q4|N|N|N|2451484|2451787|2451142|2451415|N|N|N|N|N| +2451508|AAAAAAAAEDIGFCAA|1999-11-25|1198|5213|400|1999|4|11|25|4|1999|400|5213|Thursday|1999Q4|N|N|N|2451484|2451787|2451143|2451416|N|N|N|N|N| +2451509|AAAAAAAAFDIGFCAA|1999-11-26|1198|5213|400|1999|5|11|26|4|1999|400|5213|Friday|1999Q4|N|Y|N|2451484|2451787|2451144|2451417|N|N|N|N|N| +2451510|AAAAAAAAGDIGFCAA|1999-11-27|1198|5213|400|1999|6|11|27|4|1999|400|5213|Saturday|1999Q4|N|Y|N|2451484|2451787|2451145|2451418|N|N|N|N|N| +2451511|AAAAAAAAHDIGFCAA|1999-11-28|1198|5213|400|1999|0|11|28|4|1999|400|5213|Sunday|1999Q4|N|N|N|2451484|2451787|2451146|2451419|N|N|N|N|N| +2451512|AAAAAAAAIDIGFCAA|1999-11-29|1198|5213|400|1999|1|11|29|4|1999|400|5213|Monday|1999Q4|N|N|N|2451484|2451787|2451147|2451420|N|N|N|N|N| +2451513|AAAAAAAAJDIGFCAA|1999-11-30|1198|5214|400|1999|2|11|30|4|1999|400|5214|Tuesday|1999Q4|N|N|N|2451484|2451787|2451148|2451421|N|N|N|N|N| +2451514|AAAAAAAAKDIGFCAA|1999-12-01|1199|5214|401|1999|3|12|1|4|1999|401|5214|Wednesday|1999Q4|N|N|N|2451514|2451847|2451149|2451422|N|N|N|N|N| +2451515|AAAAAAAALDIGFCAA|1999-12-02|1199|5214|401|1999|4|12|2|4|1999|401|5214|Thursday|1999Q4|N|N|N|2451514|2451847|2451150|2451423|N|N|N|N|N| +2451516|AAAAAAAAMDIGFCAA|1999-12-03|1199|5214|401|1999|5|12|3|4|1999|401|5214|Friday|1999Q4|N|Y|N|2451514|2451847|2451151|2451424|N|N|N|N|N| +2451517|AAAAAAAANDIGFCAA|1999-12-04|1199|5214|401|1999|6|12|4|4|1999|401|5214|Saturday|1999Q4|N|Y|N|2451514|2451847|2451152|2451425|N|N|N|N|N| +2451518|AAAAAAAAODIGFCAA|1999-12-05|1199|5214|401|1999|0|12|5|4|1999|401|5214|Sunday|1999Q4|N|N|N|2451514|2451847|2451153|2451426|N|N|N|N|N| +2451519|AAAAAAAAPDIGFCAA|1999-12-06|1199|5214|401|1999|1|12|6|4|1999|401|5214|Monday|1999Q4|N|N|N|2451514|2451847|2451154|2451427|N|N|N|N|N| +2451520|AAAAAAAAAEIGFCAA|1999-12-07|1199|5215|401|1999|2|12|7|4|1999|401|5215|Tuesday|1999Q4|N|N|N|2451514|2451847|2451155|2451428|N|N|N|N|N| +2451521|AAAAAAAABEIGFCAA|1999-12-08|1199|5215|401|1999|3|12|8|4|1999|401|5215|Wednesday|1999Q4|N|N|N|2451514|2451847|2451156|2451429|N|N|N|N|N| +2451522|AAAAAAAACEIGFCAA|1999-12-09|1199|5215|401|1999|4|12|9|4|1999|401|5215|Thursday|1999Q4|N|N|N|2451514|2451847|2451157|2451430|N|N|N|N|N| +2451523|AAAAAAAADEIGFCAA|1999-12-10|1199|5215|401|1999|5|12|10|4|1999|401|5215|Friday|1999Q4|N|Y|N|2451514|2451847|2451158|2451431|N|N|N|N|N| +2451524|AAAAAAAAEEIGFCAA|1999-12-11|1199|5215|401|1999|6|12|11|4|1999|401|5215|Saturday|1999Q4|N|Y|N|2451514|2451847|2451159|2451432|N|N|N|N|N| +2451525|AAAAAAAAFEIGFCAA|1999-12-12|1199|5215|401|1999|0|12|12|4|1999|401|5215|Sunday|1999Q4|N|N|N|2451514|2451847|2451160|2451433|N|N|N|N|N| +2451526|AAAAAAAAGEIGFCAA|1999-12-13|1199|5215|401|1999|1|12|13|4|1999|401|5215|Monday|1999Q4|N|N|N|2451514|2451847|2451161|2451434|N|N|N|N|N| +2451527|AAAAAAAAHEIGFCAA|1999-12-14|1199|5216|401|1999|2|12|14|4|1999|401|5216|Tuesday|1999Q4|N|N|N|2451514|2451847|2451162|2451435|N|N|N|N|N| +2451528|AAAAAAAAIEIGFCAA|1999-12-15|1199|5216|401|1999|3|12|15|4|1999|401|5216|Wednesday|1999Q4|N|N|N|2451514|2451847|2451163|2451436|N|N|N|N|N| +2451529|AAAAAAAAJEIGFCAA|1999-12-16|1199|5216|401|1999|4|12|16|4|1999|401|5216|Thursday|1999Q4|N|N|N|2451514|2451847|2451164|2451437|N|N|N|N|N| +2451530|AAAAAAAAKEIGFCAA|1999-12-17|1199|5216|401|1999|5|12|17|4|1999|401|5216|Friday|1999Q4|N|Y|N|2451514|2451847|2451165|2451438|N|N|N|N|N| +2451531|AAAAAAAALEIGFCAA|1999-12-18|1199|5216|401|1999|6|12|18|4|1999|401|5216|Saturday|1999Q4|N|Y|N|2451514|2451847|2451166|2451439|N|N|N|N|N| +2451532|AAAAAAAAMEIGFCAA|1999-12-19|1199|5216|401|1999|0|12|19|4|1999|401|5216|Sunday|1999Q4|N|N|N|2451514|2451847|2451167|2451440|N|N|N|N|N| +2451533|AAAAAAAANEIGFCAA|1999-12-20|1199|5216|401|1999|1|12|20|4|1999|401|5216|Monday|1999Q4|N|N|N|2451514|2451847|2451168|2451441|N|N|N|N|N| +2451534|AAAAAAAAOEIGFCAA|1999-12-21|1199|5217|401|1999|2|12|21|4|1999|401|5217|Tuesday|1999Q4|N|N|N|2451514|2451847|2451169|2451442|N|N|N|N|N| +2451535|AAAAAAAAPEIGFCAA|1999-12-22|1199|5217|401|1999|3|12|22|4|1999|401|5217|Wednesday|1999Q4|N|N|N|2451514|2451847|2451170|2451443|N|N|N|N|N| +2451536|AAAAAAAAAFIGFCAA|1999-12-23|1199|5217|401|1999|4|12|23|4|1999|401|5217|Thursday|1999Q4|N|N|N|2451514|2451847|2451171|2451444|N|N|N|N|N| +2451537|AAAAAAAABFIGFCAA|1999-12-24|1199|5217|401|1999|5|12|24|4|1999|401|5217|Friday|1999Q4|N|Y|N|2451514|2451847|2451172|2451445|N|N|N|N|N| +2451538|AAAAAAAACFIGFCAA|1999-12-25|1199|5217|401|1999|6|12|25|4|1999|401|5217|Saturday|1999Q4|N|Y|N|2451514|2451847|2451173|2451446|N|N|N|N|N| +2451539|AAAAAAAADFIGFCAA|1999-12-26|1199|5217|401|1999|0|12|26|4|1999|401|5217|Sunday|1999Q4|Y|N|N|2451514|2451847|2451174|2451447|N|N|N|N|N| +2451540|AAAAAAAAEFIGFCAA|1999-12-27|1199|5217|401|1999|1|12|27|4|1999|401|5217|Monday|1999Q4|N|N|Y|2451514|2451847|2451175|2451448|N|N|N|N|N| +2451541|AAAAAAAAFFIGFCAA|1999-12-28|1199|5218|401|1999|2|12|28|4|1999|401|5218|Tuesday|1999Q4|N|N|N|2451514|2451847|2451176|2451449|N|N|N|N|N| +2451542|AAAAAAAAGFIGFCAA|1999-12-29|1199|5218|401|1999|3|12|29|4|1999|401|5218|Wednesday|1999Q4|N|N|N|2451514|2451847|2451177|2451450|N|N|N|N|N| +2451543|AAAAAAAAHFIGFCAA|1999-12-30|1199|5218|401|1999|4|12|30|4|1999|401|5218|Thursday|1999Q4|N|N|N|2451514|2451847|2451178|2451451|N|N|N|N|N| +2451544|AAAAAAAAIFIGFCAA|1999-12-31|1199|5218|401|1999|5|12|31|4|1999|401|5218|Friday|1999Q4|N|Y|N|2451514|2451847|2451179|2451452|N|N|N|N|N| +2451545|AAAAAAAAJFIGFCAA|2000-01-01|1200|5218|401|2000|6|1|1|1|2000|401|5218|Saturday|2000Q1|Y|Y|N|2451545|2451544|2451180|2451453|N|N|N|N|N| +2451546|AAAAAAAAKFIGFCAA|2000-01-02|1200|5218|401|2000|0|1|2|1|2000|401|5218|Sunday|2000Q1|N|N|Y|2451545|2451544|2451181|2451454|N|N|N|N|N| +2451547|AAAAAAAALFIGFCAA|2000-01-03|1200|5218|401|2000|1|1|3|1|2000|401|5218|Monday|2000Q1|N|N|N|2451545|2451544|2451182|2451455|N|N|N|N|N| +2451548|AAAAAAAAMFIGFCAA|2000-01-04|1200|5219|401|2000|2|1|4|1|2000|401|5219|Tuesday|2000Q1|N|N|N|2451545|2451544|2451183|2451456|N|N|N|N|N| +2451549|AAAAAAAANFIGFCAA|2000-01-05|1200|5219|401|2000|3|1|5|1|2000|401|5219|Wednesday|2000Q1|N|N|N|2451545|2451544|2451184|2451457|N|N|N|N|N| +2451550|AAAAAAAAOFIGFCAA|2000-01-06|1200|5219|401|2000|4|1|6|1|2000|401|5219|Thursday|2000Q1|N|N|N|2451545|2451544|2451185|2451458|N|N|N|N|N| +2451551|AAAAAAAAPFIGFCAA|2000-01-07|1200|5219|401|2000|5|1|7|1|2000|401|5219|Friday|2000Q1|N|Y|N|2451545|2451544|2451186|2451459|N|N|N|N|N| +2451552|AAAAAAAAAGIGFCAA|2000-01-08|1200|5219|401|2000|6|1|8|1|2000|401|5219|Saturday|2000Q1|N|Y|N|2451545|2451544|2451187|2451460|N|N|N|N|N| +2451553|AAAAAAAABGIGFCAA|2000-01-09|1200|5219|401|2000|0|1|9|1|2000|401|5219|Sunday|2000Q1|N|N|N|2451545|2451544|2451188|2451461|N|N|N|N|N| +2451554|AAAAAAAACGIGFCAA|2000-01-10|1200|5219|401|2000|1|1|10|1|2000|401|5219|Monday|2000Q1|N|N|N|2451545|2451544|2451189|2451462|N|N|N|N|N| +2451555|AAAAAAAADGIGFCAA|2000-01-11|1200|5220|401|2000|2|1|11|1|2000|401|5220|Tuesday|2000Q1|N|N|N|2451545|2451544|2451190|2451463|N|N|N|N|N| +2451556|AAAAAAAAEGIGFCAA|2000-01-12|1200|5220|401|2000|3|1|12|1|2000|401|5220|Wednesday|2000Q1|N|N|N|2451545|2451544|2451191|2451464|N|N|N|N|N| +2451557|AAAAAAAAFGIGFCAA|2000-01-13|1200|5220|401|2000|4|1|13|1|2000|401|5220|Thursday|2000Q1|N|N|N|2451545|2451544|2451192|2451465|N|N|N|N|N| +2451558|AAAAAAAAGGIGFCAA|2000-01-14|1200|5220|401|2000|5|1|14|1|2000|401|5220|Friday|2000Q1|N|Y|N|2451545|2451544|2451193|2451466|N|N|N|N|N| +2451559|AAAAAAAAHGIGFCAA|2000-01-15|1200|5220|401|2000|6|1|15|1|2000|401|5220|Saturday|2000Q1|N|Y|N|2451545|2451544|2451194|2451467|N|N|N|N|N| +2451560|AAAAAAAAIGIGFCAA|2000-01-16|1200|5220|401|2000|0|1|16|1|2000|401|5220|Sunday|2000Q1|N|N|N|2451545|2451544|2451195|2451468|N|N|N|N|N| +2451561|AAAAAAAAJGIGFCAA|2000-01-17|1200|5220|401|2000|1|1|17|1|2000|401|5220|Monday|2000Q1|N|N|N|2451545|2451544|2451196|2451469|N|N|N|N|N| +2451562|AAAAAAAAKGIGFCAA|2000-01-18|1200|5221|401|2000|2|1|18|1|2000|401|5221|Tuesday|2000Q1|N|N|N|2451545|2451544|2451197|2451470|N|N|N|N|N| +2451563|AAAAAAAALGIGFCAA|2000-01-19|1200|5221|401|2000|3|1|19|1|2000|401|5221|Wednesday|2000Q1|N|N|N|2451545|2451544|2451198|2451471|N|N|N|N|N| +2451564|AAAAAAAAMGIGFCAA|2000-01-20|1200|5221|401|2000|4|1|20|1|2000|401|5221|Thursday|2000Q1|N|N|N|2451545|2451544|2451199|2451472|N|N|N|N|N| +2451565|AAAAAAAANGIGFCAA|2000-01-21|1200|5221|401|2000|5|1|21|1|2000|401|5221|Friday|2000Q1|N|Y|N|2451545|2451544|2451200|2451473|N|N|N|N|N| +2451566|AAAAAAAAOGIGFCAA|2000-01-22|1200|5221|401|2000|6|1|22|1|2000|401|5221|Saturday|2000Q1|N|Y|N|2451545|2451544|2451201|2451474|N|N|N|N|N| +2451567|AAAAAAAAPGIGFCAA|2000-01-23|1200|5221|401|2000|0|1|23|1|2000|401|5221|Sunday|2000Q1|N|N|N|2451545|2451544|2451202|2451475|N|N|N|N|N| +2451568|AAAAAAAAAHIGFCAA|2000-01-24|1200|5221|401|2000|1|1|24|1|2000|401|5221|Monday|2000Q1|N|N|N|2451545|2451544|2451203|2451476|N|N|N|N|N| +2451569|AAAAAAAABHIGFCAA|2000-01-25|1200|5222|401|2000|2|1|25|1|2000|401|5222|Tuesday|2000Q1|N|N|N|2451545|2451544|2451204|2451477|N|N|N|N|N| +2451570|AAAAAAAACHIGFCAA|2000-01-26|1200|5222|401|2000|3|1|26|1|2000|401|5222|Wednesday|2000Q1|N|N|N|2451545|2451544|2451205|2451478|N|N|N|N|N| +2451571|AAAAAAAADHIGFCAA|2000-01-27|1200|5222|401|2000|4|1|27|1|2000|401|5222|Thursday|2000Q1|N|N|N|2451545|2451544|2451206|2451479|N|N|N|N|N| +2451572|AAAAAAAAEHIGFCAA|2000-01-28|1200|5222|401|2000|5|1|28|1|2000|401|5222|Friday|2000Q1|N|Y|N|2451545|2451544|2451207|2451480|N|N|N|N|N| +2451573|AAAAAAAAFHIGFCAA|2000-01-29|1200|5222|401|2000|6|1|29|1|2000|401|5222|Saturday|2000Q1|N|Y|N|2451545|2451544|2451208|2451481|N|N|N|N|N| +2451574|AAAAAAAAGHIGFCAA|2000-01-30|1200|5222|401|2000|0|1|30|1|2000|401|5222|Sunday|2000Q1|N|N|N|2451545|2451544|2451209|2451482|N|N|N|N|N| +2451575|AAAAAAAAHHIGFCAA|2000-01-31|1200|5222|401|2000|1|1|31|1|2000|401|5222|Monday|2000Q1|N|N|N|2451545|2451544|2451210|2451483|N|N|N|N|N| +2451576|AAAAAAAAIHIGFCAA|2000-02-01|1201|5223|401|2000|2|2|1|1|2000|401|5223|Tuesday|2000Q1|N|N|N|2451576|2451606|2451211|2451484|N|N|N|N|N| +2451577|AAAAAAAAJHIGFCAA|2000-02-02|1201|5223|401|2000|3|2|2|1|2000|401|5223|Wednesday|2000Q1|N|N|N|2451576|2451606|2451212|2451485|N|N|N|N|N| +2451578|AAAAAAAAKHIGFCAA|2000-02-03|1201|5223|401|2000|4|2|3|1|2000|401|5223|Thursday|2000Q1|N|N|N|2451576|2451606|2451213|2451486|N|N|N|N|N| +2451579|AAAAAAAALHIGFCAA|2000-02-04|1201|5223|401|2000|5|2|4|1|2000|401|5223|Friday|2000Q1|N|Y|N|2451576|2451606|2451214|2451487|N|N|N|N|N| +2451580|AAAAAAAAMHIGFCAA|2000-02-05|1201|5223|401|2000|6|2|5|1|2000|401|5223|Saturday|2000Q1|N|Y|N|2451576|2451606|2451215|2451488|N|N|N|N|N| +2451581|AAAAAAAANHIGFCAA|2000-02-06|1201|5223|401|2000|0|2|6|1|2000|401|5223|Sunday|2000Q1|N|N|N|2451576|2451606|2451216|2451489|N|N|N|N|N| +2451582|AAAAAAAAOHIGFCAA|2000-02-07|1201|5223|401|2000|1|2|7|1|2000|401|5223|Monday|2000Q1|N|N|N|2451576|2451606|2451217|2451490|N|N|N|N|N| +2451583|AAAAAAAAPHIGFCAA|2000-02-08|1201|5224|401|2000|2|2|8|1|2000|401|5224|Tuesday|2000Q1|N|N|N|2451576|2451606|2451218|2451491|N|N|N|N|N| +2451584|AAAAAAAAAIIGFCAA|2000-02-09|1201|5224|401|2000|3|2|9|1|2000|401|5224|Wednesday|2000Q1|N|N|N|2451576|2451606|2451219|2451492|N|N|N|N|N| +2451585|AAAAAAAABIIGFCAA|2000-02-10|1201|5224|401|2000|4|2|10|1|2000|401|5224|Thursday|2000Q1|N|N|N|2451576|2451606|2451220|2451493|N|N|N|N|N| +2451586|AAAAAAAACIIGFCAA|2000-02-11|1201|5224|401|2000|5|2|11|1|2000|401|5224|Friday|2000Q1|N|Y|N|2451576|2451606|2451221|2451494|N|N|N|N|N| +2451587|AAAAAAAADIIGFCAA|2000-02-12|1201|5224|401|2000|6|2|12|1|2000|401|5224|Saturday|2000Q1|N|Y|N|2451576|2451606|2451222|2451495|N|N|N|N|N| +2451588|AAAAAAAAEIIGFCAA|2000-02-13|1201|5224|401|2000|0|2|13|1|2000|401|5224|Sunday|2000Q1|N|N|N|2451576|2451606|2451223|2451496|N|N|N|N|N| +2451589|AAAAAAAAFIIGFCAA|2000-02-14|1201|5224|401|2000|1|2|14|1|2000|401|5224|Monday|2000Q1|N|N|N|2451576|2451606|2451224|2451497|N|N|N|N|N| +2451590|AAAAAAAAGIIGFCAA|2000-02-15|1201|5225|401|2000|2|2|15|1|2000|401|5225|Tuesday|2000Q1|N|N|N|2451576|2451606|2451225|2451498|N|N|N|N|N| +2451591|AAAAAAAAHIIGFCAA|2000-02-16|1201|5225|401|2000|3|2|16|1|2000|401|5225|Wednesday|2000Q1|N|N|N|2451576|2451606|2451226|2451499|N|N|N|N|N| +2451592|AAAAAAAAIIIGFCAA|2000-02-17|1201|5225|401|2000|4|2|17|1|2000|401|5225|Thursday|2000Q1|N|N|N|2451576|2451606|2451227|2451500|N|N|N|N|N| +2451593|AAAAAAAAJIIGFCAA|2000-02-18|1201|5225|401|2000|5|2|18|1|2000|401|5225|Friday|2000Q1|N|Y|N|2451576|2451606|2451228|2451501|N|N|N|N|N| +2451594|AAAAAAAAKIIGFCAA|2000-02-19|1201|5225|401|2000|6|2|19|1|2000|401|5225|Saturday|2000Q1|N|Y|N|2451576|2451606|2451229|2451502|N|N|N|N|N| +2451595|AAAAAAAALIIGFCAA|2000-02-20|1201|5225|401|2000|0|2|20|1|2000|401|5225|Sunday|2000Q1|N|N|N|2451576|2451606|2451230|2451503|N|N|N|N|N| +2451596|AAAAAAAAMIIGFCAA|2000-02-21|1201|5225|401|2000|1|2|21|1|2000|401|5225|Monday|2000Q1|N|N|N|2451576|2451606|2451231|2451504|N|N|N|N|N| +2451597|AAAAAAAANIIGFCAA|2000-02-22|1201|5226|401|2000|2|2|22|1|2000|401|5226|Tuesday|2000Q1|N|N|N|2451576|2451606|2451232|2451505|N|N|N|N|N| +2451598|AAAAAAAAOIIGFCAA|2000-02-23|1201|5226|401|2000|3|2|23|1|2000|401|5226|Wednesday|2000Q1|N|N|N|2451576|2451606|2451233|2451506|N|N|N|N|N| +2451599|AAAAAAAAPIIGFCAA|2000-02-24|1201|5226|401|2000|4|2|24|1|2000|401|5226|Thursday|2000Q1|N|N|N|2451576|2451606|2451234|2451507|N|N|N|N|N| +2451600|AAAAAAAAAJIGFCAA|2000-02-25|1201|5226|401|2000|5|2|25|1|2000|401|5226|Friday|2000Q1|N|Y|N|2451576|2451606|2451235|2451508|N|N|N|N|N| +2451601|AAAAAAAABJIGFCAA|2000-02-26|1201|5226|401|2000|6|2|26|1|2000|401|5226|Saturday|2000Q1|N|Y|N|2451576|2451606|2451236|2451509|N|N|N|N|N| +2451602|AAAAAAAACJIGFCAA|2000-02-27|1201|5226|401|2000|0|2|27|1|2000|401|5226|Sunday|2000Q1|N|N|N|2451576|2451606|2451237|2451510|N|N|N|N|N| +2451603|AAAAAAAADJIGFCAA|2000-02-28|1201|5226|401|2000|1|2|28|1|2000|401|5226|Monday|2000Q1|N|N|N|2451576|2451606|2451238|2451511|N|N|N|N|N| +2451604|AAAAAAAAEJIGFCAA|2000-02-29|1201|5227|401|2000|2|2|29|1|2000|401|5227|Tuesday|2000Q1|N|N|N|2451576|2451606|2451238|2451512|N|N|N|N|N| +2451605|AAAAAAAAFJIGFCAA|2000-03-01|1202|5227|402|2000|3|3|1|1|2000|402|5227|Wednesday|2000Q1|N|N|N|2451605|2451664|2451239|2451513|N|N|N|N|N| +2451606|AAAAAAAAGJIGFCAA|2000-03-02|1202|5227|402|2000|4|3|2|1|2000|402|5227|Thursday|2000Q1|N|N|N|2451605|2451664|2451240|2451514|N|N|N|N|N| +2451607|AAAAAAAAHJIGFCAA|2000-03-03|1202|5227|402|2000|5|3|3|1|2000|402|5227|Friday|2000Q1|N|Y|N|2451605|2451664|2451241|2451515|N|N|N|N|N| +2451608|AAAAAAAAIJIGFCAA|2000-03-04|1202|5227|402|2000|6|3|4|1|2000|402|5227|Saturday|2000Q1|N|Y|N|2451605|2451664|2451242|2451516|N|N|N|N|N| +2451609|AAAAAAAAJJIGFCAA|2000-03-05|1202|5227|402|2000|0|3|5|1|2000|402|5227|Sunday|2000Q1|N|N|N|2451605|2451664|2451243|2451517|N|N|N|N|N| +2451610|AAAAAAAAKJIGFCAA|2000-03-06|1202|5227|402|2000|1|3|6|1|2000|402|5227|Monday|2000Q1|N|N|N|2451605|2451664|2451244|2451518|N|N|N|N|N| +2451611|AAAAAAAALJIGFCAA|2000-03-07|1202|5228|402|2000|2|3|7|1|2000|402|5228|Tuesday|2000Q1|N|N|N|2451605|2451664|2451245|2451519|N|N|N|N|N| +2451612|AAAAAAAAMJIGFCAA|2000-03-08|1202|5228|402|2000|3|3|8|1|2000|402|5228|Wednesday|2000Q1|N|N|N|2451605|2451664|2451246|2451520|N|N|N|N|N| +2451613|AAAAAAAANJIGFCAA|2000-03-09|1202|5228|402|2000|4|3|9|1|2000|402|5228|Thursday|2000Q1|N|N|N|2451605|2451664|2451247|2451521|N|N|N|N|N| +2451614|AAAAAAAAOJIGFCAA|2000-03-10|1202|5228|402|2000|5|3|10|1|2000|402|5228|Friday|2000Q1|N|Y|N|2451605|2451664|2451248|2451522|N|N|N|N|N| +2451615|AAAAAAAAPJIGFCAA|2000-03-11|1202|5228|402|2000|6|3|11|1|2000|402|5228|Saturday|2000Q1|N|Y|N|2451605|2451664|2451249|2451523|N|N|N|N|N| +2451616|AAAAAAAAAKIGFCAA|2000-03-12|1202|5228|402|2000|0|3|12|1|2000|402|5228|Sunday|2000Q1|N|N|N|2451605|2451664|2451250|2451524|N|N|N|N|N| +2451617|AAAAAAAABKIGFCAA|2000-03-13|1202|5228|402|2000|1|3|13|1|2000|402|5228|Monday|2000Q1|N|N|N|2451605|2451664|2451251|2451525|N|N|N|N|N| +2451618|AAAAAAAACKIGFCAA|2000-03-14|1202|5229|402|2000|2|3|14|1|2000|402|5229|Tuesday|2000Q1|N|N|N|2451605|2451664|2451252|2451526|N|N|N|N|N| +2451619|AAAAAAAADKIGFCAA|2000-03-15|1202|5229|402|2000|3|3|15|1|2000|402|5229|Wednesday|2000Q1|N|N|N|2451605|2451664|2451253|2451527|N|N|N|N|N| +2451620|AAAAAAAAEKIGFCAA|2000-03-16|1202|5229|402|2000|4|3|16|1|2000|402|5229|Thursday|2000Q1|N|N|N|2451605|2451664|2451254|2451528|N|N|N|N|N| +2451621|AAAAAAAAFKIGFCAA|2000-03-17|1202|5229|402|2000|5|3|17|1|2000|402|5229|Friday|2000Q1|N|Y|N|2451605|2451664|2451255|2451529|N|N|N|N|N| +2451622|AAAAAAAAGKIGFCAA|2000-03-18|1202|5229|402|2000|6|3|18|1|2000|402|5229|Saturday|2000Q1|N|Y|N|2451605|2451664|2451256|2451530|N|N|N|N|N| +2451623|AAAAAAAAHKIGFCAA|2000-03-19|1202|5229|402|2000|0|3|19|1|2000|402|5229|Sunday|2000Q1|N|N|N|2451605|2451664|2451257|2451531|N|N|N|N|N| +2451624|AAAAAAAAIKIGFCAA|2000-03-20|1202|5229|402|2000|1|3|20|1|2000|402|5229|Monday|2000Q1|N|N|N|2451605|2451664|2451258|2451532|N|N|N|N|N| +2451625|AAAAAAAAJKIGFCAA|2000-03-21|1202|5230|402|2000|2|3|21|1|2000|402|5230|Tuesday|2000Q1|N|N|N|2451605|2451664|2451259|2451533|N|N|N|N|N| +2451626|AAAAAAAAKKIGFCAA|2000-03-22|1202|5230|402|2000|3|3|22|1|2000|402|5230|Wednesday|2000Q1|N|N|N|2451605|2451664|2451260|2451534|N|N|N|N|N| +2451627|AAAAAAAALKIGFCAA|2000-03-23|1202|5230|402|2000|4|3|23|1|2000|402|5230|Thursday|2000Q1|N|N|N|2451605|2451664|2451261|2451535|N|N|N|N|N| +2451628|AAAAAAAAMKIGFCAA|2000-03-24|1202|5230|402|2000|5|3|24|1|2000|402|5230|Friday|2000Q1|N|Y|N|2451605|2451664|2451262|2451536|N|N|N|N|N| +2451629|AAAAAAAANKIGFCAA|2000-03-25|1202|5230|402|2000|6|3|25|1|2000|402|5230|Saturday|2000Q1|N|Y|N|2451605|2451664|2451263|2451537|N|N|N|N|N| +2451630|AAAAAAAAOKIGFCAA|2000-03-26|1202|5230|402|2000|0|3|26|1|2000|402|5230|Sunday|2000Q1|N|N|N|2451605|2451664|2451264|2451538|N|N|N|N|N| +2451631|AAAAAAAAPKIGFCAA|2000-03-27|1202|5230|402|2000|1|3|27|1|2000|402|5230|Monday|2000Q1|N|N|N|2451605|2451664|2451265|2451539|N|N|N|N|N| +2451632|AAAAAAAAALIGFCAA|2000-03-28|1202|5231|402|2000|2|3|28|1|2000|402|5231|Tuesday|2000Q1|N|N|N|2451605|2451664|2451266|2451540|N|N|N|N|N| +2451633|AAAAAAAABLIGFCAA|2000-03-29|1202|5231|402|2000|3|3|29|1|2000|402|5231|Wednesday|2000Q1|N|N|N|2451605|2451664|2451267|2451541|N|N|N|N|N| +2451634|AAAAAAAACLIGFCAA|2000-03-30|1202|5231|402|2000|4|3|30|1|2000|402|5231|Thursday|2000Q1|N|N|N|2451605|2451664|2451268|2451542|N|N|N|N|N| +2451635|AAAAAAAADLIGFCAA|2000-03-31|1202|5231|402|2000|5|3|31|1|2000|402|5231|Friday|2000Q1|N|Y|N|2451605|2451664|2451269|2451543|N|N|N|N|N| +2451636|AAAAAAAAELIGFCAA|2000-04-01|1203|5231|402|2000|6|4|1|2|2000|402|5231|Saturday|2000Q2|N|Y|N|2451636|2451726|2451270|2451545|N|N|N|N|N| +2451637|AAAAAAAAFLIGFCAA|2000-04-02|1203|5231|402|2000|0|4|2|2|2000|402|5231|Sunday|2000Q2|N|N|N|2451636|2451726|2451271|2451546|N|N|N|N|N| +2451638|AAAAAAAAGLIGFCAA|2000-04-03|1203|5231|402|2000|1|4|3|2|2000|402|5231|Monday|2000Q2|N|N|N|2451636|2451726|2451272|2451547|N|N|N|N|N| +2451639|AAAAAAAAHLIGFCAA|2000-04-04|1203|5232|402|2000|2|4|4|2|2000|402|5232|Tuesday|2000Q2|N|N|N|2451636|2451726|2451273|2451548|N|N|N|N|N| +2451640|AAAAAAAAILIGFCAA|2000-04-05|1203|5232|402|2000|3|4|5|2|2000|402|5232|Wednesday|2000Q2|N|N|N|2451636|2451726|2451274|2451549|N|N|N|N|N| +2451641|AAAAAAAAJLIGFCAA|2000-04-06|1203|5232|402|2000|4|4|6|2|2000|402|5232|Thursday|2000Q2|N|N|N|2451636|2451726|2451275|2451550|N|N|N|N|N| +2451642|AAAAAAAAKLIGFCAA|2000-04-07|1203|5232|402|2000|5|4|7|2|2000|402|5232|Friday|2000Q2|N|Y|N|2451636|2451726|2451276|2451551|N|N|N|N|N| +2451643|AAAAAAAALLIGFCAA|2000-04-08|1203|5232|402|2000|6|4|8|2|2000|402|5232|Saturday|2000Q2|N|Y|N|2451636|2451726|2451277|2451552|N|N|N|N|N| +2451644|AAAAAAAAMLIGFCAA|2000-04-09|1203|5232|402|2000|0|4|9|2|2000|402|5232|Sunday|2000Q2|N|N|N|2451636|2451726|2451278|2451553|N|N|N|N|N| +2451645|AAAAAAAANLIGFCAA|2000-04-10|1203|5232|402|2000|1|4|10|2|2000|402|5232|Monday|2000Q2|N|N|N|2451636|2451726|2451279|2451554|N|N|N|N|N| +2451646|AAAAAAAAOLIGFCAA|2000-04-11|1203|5233|402|2000|2|4|11|2|2000|402|5233|Tuesday|2000Q2|N|N|N|2451636|2451726|2451280|2451555|N|N|N|N|N| +2451647|AAAAAAAAPLIGFCAA|2000-04-12|1203|5233|402|2000|3|4|12|2|2000|402|5233|Wednesday|2000Q2|N|N|N|2451636|2451726|2451281|2451556|N|N|N|N|N| +2451648|AAAAAAAAAMIGFCAA|2000-04-13|1203|5233|402|2000|4|4|13|2|2000|402|5233|Thursday|2000Q2|N|N|N|2451636|2451726|2451282|2451557|N|N|N|N|N| +2451649|AAAAAAAABMIGFCAA|2000-04-14|1203|5233|402|2000|5|4|14|2|2000|402|5233|Friday|2000Q2|N|Y|N|2451636|2451726|2451283|2451558|N|N|N|N|N| +2451650|AAAAAAAACMIGFCAA|2000-04-15|1203|5233|402|2000|6|4|15|2|2000|402|5233|Saturday|2000Q2|N|Y|N|2451636|2451726|2451284|2451559|N|N|N|N|N| +2451651|AAAAAAAADMIGFCAA|2000-04-16|1203|5233|402|2000|0|4|16|2|2000|402|5233|Sunday|2000Q2|N|N|N|2451636|2451726|2451285|2451560|N|N|N|N|N| +2451652|AAAAAAAAEMIGFCAA|2000-04-17|1203|5233|402|2000|1|4|17|2|2000|402|5233|Monday|2000Q2|N|N|N|2451636|2451726|2451286|2451561|N|N|N|N|N| +2451653|AAAAAAAAFMIGFCAA|2000-04-18|1203|5234|402|2000|2|4|18|2|2000|402|5234|Tuesday|2000Q2|N|N|N|2451636|2451726|2451287|2451562|N|N|N|N|N| +2451654|AAAAAAAAGMIGFCAA|2000-04-19|1203|5234|402|2000|3|4|19|2|2000|402|5234|Wednesday|2000Q2|N|N|N|2451636|2451726|2451288|2451563|N|N|N|N|N| +2451655|AAAAAAAAHMIGFCAA|2000-04-20|1203|5234|402|2000|4|4|20|2|2000|402|5234|Thursday|2000Q2|N|N|N|2451636|2451726|2451289|2451564|N|N|N|N|N| +2451656|AAAAAAAAIMIGFCAA|2000-04-21|1203|5234|402|2000|5|4|21|2|2000|402|5234|Friday|2000Q2|N|Y|N|2451636|2451726|2451290|2451565|N|N|N|N|N| +2451657|AAAAAAAAJMIGFCAA|2000-04-22|1203|5234|402|2000|6|4|22|2|2000|402|5234|Saturday|2000Q2|N|Y|N|2451636|2451726|2451291|2451566|N|N|N|N|N| +2451658|AAAAAAAAKMIGFCAA|2000-04-23|1203|5234|402|2000|0|4|23|2|2000|402|5234|Sunday|2000Q2|N|N|N|2451636|2451726|2451292|2451567|N|N|N|N|N| +2451659|AAAAAAAALMIGFCAA|2000-04-24|1203|5234|402|2000|1|4|24|2|2000|402|5234|Monday|2000Q2|N|N|N|2451636|2451726|2451293|2451568|N|N|N|N|N| +2451660|AAAAAAAAMMIGFCAA|2000-04-25|1203|5235|402|2000|2|4|25|2|2000|402|5235|Tuesday|2000Q2|N|N|N|2451636|2451726|2451294|2451569|N|N|N|N|N| +2451661|AAAAAAAANMIGFCAA|2000-04-26|1203|5235|402|2000|3|4|26|2|2000|402|5235|Wednesday|2000Q2|N|N|N|2451636|2451726|2451295|2451570|N|N|N|N|N| +2451662|AAAAAAAAOMIGFCAA|2000-04-27|1203|5235|402|2000|4|4|27|2|2000|402|5235|Thursday|2000Q2|N|N|N|2451636|2451726|2451296|2451571|N|N|N|N|N| +2451663|AAAAAAAAPMIGFCAA|2000-04-28|1203|5235|402|2000|5|4|28|2|2000|402|5235|Friday|2000Q2|N|Y|N|2451636|2451726|2451297|2451572|N|N|N|N|N| +2451664|AAAAAAAAANIGFCAA|2000-04-29|1203|5235|402|2000|6|4|29|2|2000|402|5235|Saturday|2000Q2|N|Y|N|2451636|2451726|2451298|2451573|N|N|N|N|N| +2451665|AAAAAAAABNIGFCAA|2000-04-30|1203|5235|402|2000|0|4|30|2|2000|402|5235|Sunday|2000Q2|N|N|N|2451636|2451726|2451299|2451574|N|N|N|N|N| +2451666|AAAAAAAACNIGFCAA|2000-05-01|1204|5235|402|2000|1|5|1|2|2000|402|5235|Monday|2000Q2|N|N|N|2451666|2451786|2451300|2451575|N|N|N|N|N| +2451667|AAAAAAAADNIGFCAA|2000-05-02|1204|5236|402|2000|2|5|2|2|2000|402|5236|Tuesday|2000Q2|N|N|N|2451666|2451786|2451301|2451576|N|N|N|N|N| +2451668|AAAAAAAAENIGFCAA|2000-05-03|1204|5236|402|2000|3|5|3|2|2000|402|5236|Wednesday|2000Q2|N|N|N|2451666|2451786|2451302|2451577|N|N|N|N|N| +2451669|AAAAAAAAFNIGFCAA|2000-05-04|1204|5236|402|2000|4|5|4|2|2000|402|5236|Thursday|2000Q2|N|N|N|2451666|2451786|2451303|2451578|N|N|N|N|N| +2451670|AAAAAAAAGNIGFCAA|2000-05-05|1204|5236|402|2000|5|5|5|2|2000|402|5236|Friday|2000Q2|N|Y|N|2451666|2451786|2451304|2451579|N|N|N|N|N| +2451671|AAAAAAAAHNIGFCAA|2000-05-06|1204|5236|402|2000|6|5|6|2|2000|402|5236|Saturday|2000Q2|N|Y|N|2451666|2451786|2451305|2451580|N|N|N|N|N| +2451672|AAAAAAAAINIGFCAA|2000-05-07|1204|5236|402|2000|0|5|7|2|2000|402|5236|Sunday|2000Q2|N|N|N|2451666|2451786|2451306|2451581|N|N|N|N|N| +2451673|AAAAAAAAJNIGFCAA|2000-05-08|1204|5236|402|2000|1|5|8|2|2000|402|5236|Monday|2000Q2|N|N|N|2451666|2451786|2451307|2451582|N|N|N|N|N| +2451674|AAAAAAAAKNIGFCAA|2000-05-09|1204|5237|402|2000|2|5|9|2|2000|402|5237|Tuesday|2000Q2|N|N|N|2451666|2451786|2451308|2451583|N|N|N|N|N| +2451675|AAAAAAAALNIGFCAA|2000-05-10|1204|5237|402|2000|3|5|10|2|2000|402|5237|Wednesday|2000Q2|N|N|N|2451666|2451786|2451309|2451584|N|N|N|N|N| +2451676|AAAAAAAAMNIGFCAA|2000-05-11|1204|5237|402|2000|4|5|11|2|2000|402|5237|Thursday|2000Q2|N|N|N|2451666|2451786|2451310|2451585|N|N|N|N|N| +2451677|AAAAAAAANNIGFCAA|2000-05-12|1204|5237|402|2000|5|5|12|2|2000|402|5237|Friday|2000Q2|N|Y|N|2451666|2451786|2451311|2451586|N|N|N|N|N| +2451678|AAAAAAAAONIGFCAA|2000-05-13|1204|5237|402|2000|6|5|13|2|2000|402|5237|Saturday|2000Q2|N|Y|N|2451666|2451786|2451312|2451587|N|N|N|N|N| +2451679|AAAAAAAAPNIGFCAA|2000-05-14|1204|5237|402|2000|0|5|14|2|2000|402|5237|Sunday|2000Q2|N|N|N|2451666|2451786|2451313|2451588|N|N|N|N|N| +2451680|AAAAAAAAAOIGFCAA|2000-05-15|1204|5237|402|2000|1|5|15|2|2000|402|5237|Monday|2000Q2|N|N|N|2451666|2451786|2451314|2451589|N|N|N|N|N| +2451681|AAAAAAAABOIGFCAA|2000-05-16|1204|5238|402|2000|2|5|16|2|2000|402|5238|Tuesday|2000Q2|N|N|N|2451666|2451786|2451315|2451590|N|N|N|N|N| +2451682|AAAAAAAACOIGFCAA|2000-05-17|1204|5238|402|2000|3|5|17|2|2000|402|5238|Wednesday|2000Q2|N|N|N|2451666|2451786|2451316|2451591|N|N|N|N|N| +2451683|AAAAAAAADOIGFCAA|2000-05-18|1204|5238|402|2000|4|5|18|2|2000|402|5238|Thursday|2000Q2|N|N|N|2451666|2451786|2451317|2451592|N|N|N|N|N| +2451684|AAAAAAAAEOIGFCAA|2000-05-19|1204|5238|402|2000|5|5|19|2|2000|402|5238|Friday|2000Q2|N|Y|N|2451666|2451786|2451318|2451593|N|N|N|N|N| +2451685|AAAAAAAAFOIGFCAA|2000-05-20|1204|5238|402|2000|6|5|20|2|2000|402|5238|Saturday|2000Q2|N|Y|N|2451666|2451786|2451319|2451594|N|N|N|N|N| +2451686|AAAAAAAAGOIGFCAA|2000-05-21|1204|5238|402|2000|0|5|21|2|2000|402|5238|Sunday|2000Q2|N|N|N|2451666|2451786|2451320|2451595|N|N|N|N|N| +2451687|AAAAAAAAHOIGFCAA|2000-05-22|1204|5238|402|2000|1|5|22|2|2000|402|5238|Monday|2000Q2|N|N|N|2451666|2451786|2451321|2451596|N|N|N|N|N| +2451688|AAAAAAAAIOIGFCAA|2000-05-23|1204|5239|402|2000|2|5|23|2|2000|402|5239|Tuesday|2000Q2|N|N|N|2451666|2451786|2451322|2451597|N|N|N|N|N| +2451689|AAAAAAAAJOIGFCAA|2000-05-24|1204|5239|402|2000|3|5|24|2|2000|402|5239|Wednesday|2000Q2|N|N|N|2451666|2451786|2451323|2451598|N|N|N|N|N| +2451690|AAAAAAAAKOIGFCAA|2000-05-25|1204|5239|402|2000|4|5|25|2|2000|402|5239|Thursday|2000Q2|N|N|N|2451666|2451786|2451324|2451599|N|N|N|N|N| +2451691|AAAAAAAALOIGFCAA|2000-05-26|1204|5239|402|2000|5|5|26|2|2000|402|5239|Friday|2000Q2|N|Y|N|2451666|2451786|2451325|2451600|N|N|N|N|N| +2451692|AAAAAAAAMOIGFCAA|2000-05-27|1204|5239|402|2000|6|5|27|2|2000|402|5239|Saturday|2000Q2|N|Y|N|2451666|2451786|2451326|2451601|N|N|N|N|N| +2451693|AAAAAAAANOIGFCAA|2000-05-28|1204|5239|402|2000|0|5|28|2|2000|402|5239|Sunday|2000Q2|N|N|N|2451666|2451786|2451327|2451602|N|N|N|N|N| +2451694|AAAAAAAAOOIGFCAA|2000-05-29|1204|5239|402|2000|1|5|29|2|2000|402|5239|Monday|2000Q2|N|N|N|2451666|2451786|2451328|2451603|N|N|N|N|N| +2451695|AAAAAAAAPOIGFCAA|2000-05-30|1204|5240|402|2000|2|5|30|2|2000|402|5240|Tuesday|2000Q2|N|N|N|2451666|2451786|2451329|2451604|N|N|N|N|N| +2451696|AAAAAAAAAPIGFCAA|2000-05-31|1204|5240|402|2000|3|5|31|2|2000|402|5240|Wednesday|2000Q2|N|N|N|2451666|2451786|2451330|2451605|N|N|N|N|N| +2451697|AAAAAAAABPIGFCAA|2000-06-01|1205|5240|403|2000|4|6|1|2|2000|403|5240|Thursday|2000Q2|N|N|N|2451697|2451848|2451331|2451606|N|N|N|N|N| +2451698|AAAAAAAACPIGFCAA|2000-06-02|1205|5240|403|2000|5|6|2|2|2000|403|5240|Friday|2000Q2|N|Y|N|2451697|2451848|2451332|2451607|N|N|N|N|N| +2451699|AAAAAAAADPIGFCAA|2000-06-03|1205|5240|403|2000|6|6|3|2|2000|403|5240|Saturday|2000Q2|N|Y|N|2451697|2451848|2451333|2451608|N|N|N|N|N| +2451700|AAAAAAAAEPIGFCAA|2000-06-04|1205|5240|403|2000|0|6|4|2|2000|403|5240|Sunday|2000Q2|N|N|N|2451697|2451848|2451334|2451609|N|N|N|N|N| +2451701|AAAAAAAAFPIGFCAA|2000-06-05|1205|5240|403|2000|1|6|5|2|2000|403|5240|Monday|2000Q2|N|N|N|2451697|2451848|2451335|2451610|N|N|N|N|N| +2451702|AAAAAAAAGPIGFCAA|2000-06-06|1205|5241|403|2000|2|6|6|2|2000|403|5241|Tuesday|2000Q2|N|N|N|2451697|2451848|2451336|2451611|N|N|N|N|N| +2451703|AAAAAAAAHPIGFCAA|2000-06-07|1205|5241|403|2000|3|6|7|2|2000|403|5241|Wednesday|2000Q2|N|N|N|2451697|2451848|2451337|2451612|N|N|N|N|N| +2451704|AAAAAAAAIPIGFCAA|2000-06-08|1205|5241|403|2000|4|6|8|2|2000|403|5241|Thursday|2000Q2|N|N|N|2451697|2451848|2451338|2451613|N|N|N|N|N| +2451705|AAAAAAAAJPIGFCAA|2000-06-09|1205|5241|403|2000|5|6|9|2|2000|403|5241|Friday|2000Q2|N|Y|N|2451697|2451848|2451339|2451614|N|N|N|N|N| +2451706|AAAAAAAAKPIGFCAA|2000-06-10|1205|5241|403|2000|6|6|10|2|2000|403|5241|Saturday|2000Q2|N|Y|N|2451697|2451848|2451340|2451615|N|N|N|N|N| +2451707|AAAAAAAALPIGFCAA|2000-06-11|1205|5241|403|2000|0|6|11|2|2000|403|5241|Sunday|2000Q2|N|N|N|2451697|2451848|2451341|2451616|N|N|N|N|N| +2451708|AAAAAAAAMPIGFCAA|2000-06-12|1205|5241|403|2000|1|6|12|2|2000|403|5241|Monday|2000Q2|N|N|N|2451697|2451848|2451342|2451617|N|N|N|N|N| +2451709|AAAAAAAANPIGFCAA|2000-06-13|1205|5242|403|2000|2|6|13|2|2000|403|5242|Tuesday|2000Q2|N|N|N|2451697|2451848|2451343|2451618|N|N|N|N|N| +2451710|AAAAAAAAOPIGFCAA|2000-06-14|1205|5242|403|2000|3|6|14|2|2000|403|5242|Wednesday|2000Q2|N|N|N|2451697|2451848|2451344|2451619|N|N|N|N|N| +2451711|AAAAAAAAPPIGFCAA|2000-06-15|1205|5242|403|2000|4|6|15|2|2000|403|5242|Thursday|2000Q2|N|N|N|2451697|2451848|2451345|2451620|N|N|N|N|N| +2451712|AAAAAAAAAAJGFCAA|2000-06-16|1205|5242|403|2000|5|6|16|2|2000|403|5242|Friday|2000Q2|N|Y|N|2451697|2451848|2451346|2451621|N|N|N|N|N| +2451713|AAAAAAAABAJGFCAA|2000-06-17|1205|5242|403|2000|6|6|17|2|2000|403|5242|Saturday|2000Q2|N|Y|N|2451697|2451848|2451347|2451622|N|N|N|N|N| +2451714|AAAAAAAACAJGFCAA|2000-06-18|1205|5242|403|2000|0|6|18|2|2000|403|5242|Sunday|2000Q2|N|N|N|2451697|2451848|2451348|2451623|N|N|N|N|N| +2451715|AAAAAAAADAJGFCAA|2000-06-19|1205|5242|403|2000|1|6|19|2|2000|403|5242|Monday|2000Q2|N|N|N|2451697|2451848|2451349|2451624|N|N|N|N|N| +2451716|AAAAAAAAEAJGFCAA|2000-06-20|1205|5243|403|2000|2|6|20|2|2000|403|5243|Tuesday|2000Q2|N|N|N|2451697|2451848|2451350|2451625|N|N|N|N|N| +2451717|AAAAAAAAFAJGFCAA|2000-06-21|1205|5243|403|2000|3|6|21|2|2000|403|5243|Wednesday|2000Q2|N|N|N|2451697|2451848|2451351|2451626|N|N|N|N|N| +2451718|AAAAAAAAGAJGFCAA|2000-06-22|1205|5243|403|2000|4|6|22|2|2000|403|5243|Thursday|2000Q2|N|N|N|2451697|2451848|2451352|2451627|N|N|N|N|N| +2451719|AAAAAAAAHAJGFCAA|2000-06-23|1205|5243|403|2000|5|6|23|2|2000|403|5243|Friday|2000Q2|N|Y|N|2451697|2451848|2451353|2451628|N|N|N|N|N| +2451720|AAAAAAAAIAJGFCAA|2000-06-24|1205|5243|403|2000|6|6|24|2|2000|403|5243|Saturday|2000Q2|N|Y|N|2451697|2451848|2451354|2451629|N|N|N|N|N| +2451721|AAAAAAAAJAJGFCAA|2000-06-25|1205|5243|403|2000|0|6|25|2|2000|403|5243|Sunday|2000Q2|N|N|N|2451697|2451848|2451355|2451630|N|N|N|N|N| +2451722|AAAAAAAAKAJGFCAA|2000-06-26|1205|5243|403|2000|1|6|26|2|2000|403|5243|Monday|2000Q2|N|N|N|2451697|2451848|2451356|2451631|N|N|N|N|N| +2451723|AAAAAAAALAJGFCAA|2000-06-27|1205|5244|403|2000|2|6|27|2|2000|403|5244|Tuesday|2000Q2|N|N|N|2451697|2451848|2451357|2451632|N|N|N|N|N| +2451724|AAAAAAAAMAJGFCAA|2000-06-28|1205|5244|403|2000|3|6|28|2|2000|403|5244|Wednesday|2000Q2|N|N|N|2451697|2451848|2451358|2451633|N|N|N|N|N| +2451725|AAAAAAAANAJGFCAA|2000-06-29|1205|5244|403|2000|4|6|29|2|2000|403|5244|Thursday|2000Q2|N|N|N|2451697|2451848|2451359|2451634|N|N|N|N|N| +2451726|AAAAAAAAOAJGFCAA|2000-06-30|1205|5244|403|2000|5|6|30|2|2000|403|5244|Friday|2000Q2|N|Y|N|2451697|2451848|2451360|2451635|N|N|N|N|N| +2451727|AAAAAAAAPAJGFCAA|2000-07-01|1206|5244|403|2000|6|7|1|3|2000|403|5244|Saturday|2000Q3|N|Y|N|2451727|2451908|2451361|2451636|N|N|N|N|N| +2451728|AAAAAAAAABJGFCAA|2000-07-02|1206|5244|403|2000|0|7|2|3|2000|403|5244|Sunday|2000Q3|N|N|N|2451727|2451908|2451362|2451637|N|N|N|N|N| +2451729|AAAAAAAABBJGFCAA|2000-07-03|1206|5244|403|2000|1|7|3|3|2000|403|5244|Monday|2000Q3|N|N|N|2451727|2451908|2451363|2451638|N|N|N|N|N| +2451730|AAAAAAAACBJGFCAA|2000-07-04|1206|5245|403|2000|2|7|4|3|2000|403|5245|Tuesday|2000Q3|Y|N|N|2451727|2451908|2451364|2451639|N|N|N|N|N| +2451731|AAAAAAAADBJGFCAA|2000-07-05|1206|5245|403|2000|3|7|5|3|2000|403|5245|Wednesday|2000Q3|N|N|Y|2451727|2451908|2451365|2451640|N|N|N|N|N| +2451732|AAAAAAAAEBJGFCAA|2000-07-06|1206|5245|403|2000|4|7|6|3|2000|403|5245|Thursday|2000Q3|N|N|N|2451727|2451908|2451366|2451641|N|N|N|N|N| +2451733|AAAAAAAAFBJGFCAA|2000-07-07|1206|5245|403|2000|5|7|7|3|2000|403|5245|Friday|2000Q3|N|Y|N|2451727|2451908|2451367|2451642|N|N|N|N|N| +2451734|AAAAAAAAGBJGFCAA|2000-07-08|1206|5245|403|2000|6|7|8|3|2000|403|5245|Saturday|2000Q3|N|Y|N|2451727|2451908|2451368|2451643|N|N|N|N|N| +2451735|AAAAAAAAHBJGFCAA|2000-07-09|1206|5245|403|2000|0|7|9|3|2000|403|5245|Sunday|2000Q3|N|N|N|2451727|2451908|2451369|2451644|N|N|N|N|N| +2451736|AAAAAAAAIBJGFCAA|2000-07-10|1206|5245|403|2000|1|7|10|3|2000|403|5245|Monday|2000Q3|N|N|N|2451727|2451908|2451370|2451645|N|N|N|N|N| +2451737|AAAAAAAAJBJGFCAA|2000-07-11|1206|5246|403|2000|2|7|11|3|2000|403|5246|Tuesday|2000Q3|N|N|N|2451727|2451908|2451371|2451646|N|N|N|N|N| +2451738|AAAAAAAAKBJGFCAA|2000-07-12|1206|5246|403|2000|3|7|12|3|2000|403|5246|Wednesday|2000Q3|N|N|N|2451727|2451908|2451372|2451647|N|N|N|N|N| +2451739|AAAAAAAALBJGFCAA|2000-07-13|1206|5246|403|2000|4|7|13|3|2000|403|5246|Thursday|2000Q3|N|N|N|2451727|2451908|2451373|2451648|N|N|N|N|N| +2451740|AAAAAAAAMBJGFCAA|2000-07-14|1206|5246|403|2000|5|7|14|3|2000|403|5246|Friday|2000Q3|N|Y|N|2451727|2451908|2451374|2451649|N|N|N|N|N| +2451741|AAAAAAAANBJGFCAA|2000-07-15|1206|5246|403|2000|6|7|15|3|2000|403|5246|Saturday|2000Q3|N|Y|N|2451727|2451908|2451375|2451650|N|N|N|N|N| +2451742|AAAAAAAAOBJGFCAA|2000-07-16|1206|5246|403|2000|0|7|16|3|2000|403|5246|Sunday|2000Q3|N|N|N|2451727|2451908|2451376|2451651|N|N|N|N|N| +2451743|AAAAAAAAPBJGFCAA|2000-07-17|1206|5246|403|2000|1|7|17|3|2000|403|5246|Monday|2000Q3|N|N|N|2451727|2451908|2451377|2451652|N|N|N|N|N| +2451744|AAAAAAAAACJGFCAA|2000-07-18|1206|5247|403|2000|2|7|18|3|2000|403|5247|Tuesday|2000Q3|N|N|N|2451727|2451908|2451378|2451653|N|N|N|N|N| +2451745|AAAAAAAABCJGFCAA|2000-07-19|1206|5247|403|2000|3|7|19|3|2000|403|5247|Wednesday|2000Q3|N|N|N|2451727|2451908|2451379|2451654|N|N|N|N|N| +2451746|AAAAAAAACCJGFCAA|2000-07-20|1206|5247|403|2000|4|7|20|3|2000|403|5247|Thursday|2000Q3|N|N|N|2451727|2451908|2451380|2451655|N|N|N|N|N| +2451747|AAAAAAAADCJGFCAA|2000-07-21|1206|5247|403|2000|5|7|21|3|2000|403|5247|Friday|2000Q3|N|Y|N|2451727|2451908|2451381|2451656|N|N|N|N|N| +2451748|AAAAAAAAECJGFCAA|2000-07-22|1206|5247|403|2000|6|7|22|3|2000|403|5247|Saturday|2000Q3|N|Y|N|2451727|2451908|2451382|2451657|N|N|N|N|N| +2451749|AAAAAAAAFCJGFCAA|2000-07-23|1206|5247|403|2000|0|7|23|3|2000|403|5247|Sunday|2000Q3|N|N|N|2451727|2451908|2451383|2451658|N|N|N|N|N| +2451750|AAAAAAAAGCJGFCAA|2000-07-24|1206|5247|403|2000|1|7|24|3|2000|403|5247|Monday|2000Q3|N|N|N|2451727|2451908|2451384|2451659|N|N|N|N|N| +2451751|AAAAAAAAHCJGFCAA|2000-07-25|1206|5248|403|2000|2|7|25|3|2000|403|5248|Tuesday|2000Q3|N|N|N|2451727|2451908|2451385|2451660|N|N|N|N|N| +2451752|AAAAAAAAICJGFCAA|2000-07-26|1206|5248|403|2000|3|7|26|3|2000|403|5248|Wednesday|2000Q3|N|N|N|2451727|2451908|2451386|2451661|N|N|N|N|N| +2451753|AAAAAAAAJCJGFCAA|2000-07-27|1206|5248|403|2000|4|7|27|3|2000|403|5248|Thursday|2000Q3|N|N|N|2451727|2451908|2451387|2451662|N|N|N|N|N| +2451754|AAAAAAAAKCJGFCAA|2000-07-28|1206|5248|403|2000|5|7|28|3|2000|403|5248|Friday|2000Q3|N|Y|N|2451727|2451908|2451388|2451663|N|N|N|N|N| +2451755|AAAAAAAALCJGFCAA|2000-07-29|1206|5248|403|2000|6|7|29|3|2000|403|5248|Saturday|2000Q3|N|Y|N|2451727|2451908|2451389|2451664|N|N|N|N|N| +2451756|AAAAAAAAMCJGFCAA|2000-07-30|1206|5248|403|2000|0|7|30|3|2000|403|5248|Sunday|2000Q3|N|N|N|2451727|2451908|2451390|2451665|N|N|N|N|N| +2451757|AAAAAAAANCJGFCAA|2000-07-31|1206|5248|403|2000|1|7|31|3|2000|403|5248|Monday|2000Q3|N|N|N|2451727|2451908|2451391|2451666|N|N|N|N|N| +2451758|AAAAAAAAOCJGFCAA|2000-08-01|1207|5249|403|2000|2|8|1|3|2000|403|5249|Tuesday|2000Q3|N|N|N|2451758|2451970|2451392|2451667|N|N|N|N|N| +2451759|AAAAAAAAPCJGFCAA|2000-08-02|1207|5249|403|2000|3|8|2|3|2000|403|5249|Wednesday|2000Q3|N|N|N|2451758|2451970|2451393|2451668|N|N|N|N|N| +2451760|AAAAAAAAADJGFCAA|2000-08-03|1207|5249|403|2000|4|8|3|3|2000|403|5249|Thursday|2000Q3|N|N|N|2451758|2451970|2451394|2451669|N|N|N|N|N| +2451761|AAAAAAAABDJGFCAA|2000-08-04|1207|5249|403|2000|5|8|4|3|2000|403|5249|Friday|2000Q3|N|Y|N|2451758|2451970|2451395|2451670|N|N|N|N|N| +2451762|AAAAAAAACDJGFCAA|2000-08-05|1207|5249|403|2000|6|8|5|3|2000|403|5249|Saturday|2000Q3|N|Y|N|2451758|2451970|2451396|2451671|N|N|N|N|N| +2451763|AAAAAAAADDJGFCAA|2000-08-06|1207|5249|403|2000|0|8|6|3|2000|403|5249|Sunday|2000Q3|N|N|N|2451758|2451970|2451397|2451672|N|N|N|N|N| +2451764|AAAAAAAAEDJGFCAA|2000-08-07|1207|5249|403|2000|1|8|7|3|2000|403|5249|Monday|2000Q3|N|N|N|2451758|2451970|2451398|2451673|N|N|N|N|N| +2451765|AAAAAAAAFDJGFCAA|2000-08-08|1207|5250|403|2000|2|8|8|3|2000|403|5250|Tuesday|2000Q3|N|N|N|2451758|2451970|2451399|2451674|N|N|N|N|N| +2451766|AAAAAAAAGDJGFCAA|2000-08-09|1207|5250|403|2000|3|8|9|3|2000|403|5250|Wednesday|2000Q3|N|N|N|2451758|2451970|2451400|2451675|N|N|N|N|N| +2451767|AAAAAAAAHDJGFCAA|2000-08-10|1207|5250|403|2000|4|8|10|3|2000|403|5250|Thursday|2000Q3|N|N|N|2451758|2451970|2451401|2451676|N|N|N|N|N| +2451768|AAAAAAAAIDJGFCAA|2000-08-11|1207|5250|403|2000|5|8|11|3|2000|403|5250|Friday|2000Q3|N|Y|N|2451758|2451970|2451402|2451677|N|N|N|N|N| +2451769|AAAAAAAAJDJGFCAA|2000-08-12|1207|5250|403|2000|6|8|12|3|2000|403|5250|Saturday|2000Q3|N|Y|N|2451758|2451970|2451403|2451678|N|N|N|N|N| +2451770|AAAAAAAAKDJGFCAA|2000-08-13|1207|5250|403|2000|0|8|13|3|2000|403|5250|Sunday|2000Q3|N|N|N|2451758|2451970|2451404|2451679|N|N|N|N|N| +2451771|AAAAAAAALDJGFCAA|2000-08-14|1207|5250|403|2000|1|8|14|3|2000|403|5250|Monday|2000Q3|N|N|N|2451758|2451970|2451405|2451680|N|N|N|N|N| +2451772|AAAAAAAAMDJGFCAA|2000-08-15|1207|5251|403|2000|2|8|15|3|2000|403|5251|Tuesday|2000Q3|N|N|N|2451758|2451970|2451406|2451681|N|N|N|N|N| +2451773|AAAAAAAANDJGFCAA|2000-08-16|1207|5251|403|2000|3|8|16|3|2000|403|5251|Wednesday|2000Q3|N|N|N|2451758|2451970|2451407|2451682|N|N|N|N|N| +2451774|AAAAAAAAODJGFCAA|2000-08-17|1207|5251|403|2000|4|8|17|3|2000|403|5251|Thursday|2000Q3|N|N|N|2451758|2451970|2451408|2451683|N|N|N|N|N| +2451775|AAAAAAAAPDJGFCAA|2000-08-18|1207|5251|403|2000|5|8|18|3|2000|403|5251|Friday|2000Q3|N|Y|N|2451758|2451970|2451409|2451684|N|N|N|N|N| +2451776|AAAAAAAAAEJGFCAA|2000-08-19|1207|5251|403|2000|6|8|19|3|2000|403|5251|Saturday|2000Q3|N|Y|N|2451758|2451970|2451410|2451685|N|N|N|N|N| +2451777|AAAAAAAABEJGFCAA|2000-08-20|1207|5251|403|2000|0|8|20|3|2000|403|5251|Sunday|2000Q3|N|N|N|2451758|2451970|2451411|2451686|N|N|N|N|N| +2451778|AAAAAAAACEJGFCAA|2000-08-21|1207|5251|403|2000|1|8|21|3|2000|403|5251|Monday|2000Q3|N|N|N|2451758|2451970|2451412|2451687|N|N|N|N|N| +2451779|AAAAAAAADEJGFCAA|2000-08-22|1207|5252|403|2000|2|8|22|3|2000|403|5252|Tuesday|2000Q3|N|N|N|2451758|2451970|2451413|2451688|N|N|N|N|N| +2451780|AAAAAAAAEEJGFCAA|2000-08-23|1207|5252|403|2000|3|8|23|3|2000|403|5252|Wednesday|2000Q3|N|N|N|2451758|2451970|2451414|2451689|N|N|N|N|N| +2451781|AAAAAAAAFEJGFCAA|2000-08-24|1207|5252|403|2000|4|8|24|3|2000|403|5252|Thursday|2000Q3|N|N|N|2451758|2451970|2451415|2451690|N|N|N|N|N| +2451782|AAAAAAAAGEJGFCAA|2000-08-25|1207|5252|403|2000|5|8|25|3|2000|403|5252|Friday|2000Q3|N|Y|N|2451758|2451970|2451416|2451691|N|N|N|N|N| +2451783|AAAAAAAAHEJGFCAA|2000-08-26|1207|5252|403|2000|6|8|26|3|2000|403|5252|Saturday|2000Q3|N|Y|N|2451758|2451970|2451417|2451692|N|N|N|N|N| +2451784|AAAAAAAAIEJGFCAA|2000-08-27|1207|5252|403|2000|0|8|27|3|2000|403|5252|Sunday|2000Q3|N|N|N|2451758|2451970|2451418|2451693|N|N|N|N|N| +2451785|AAAAAAAAJEJGFCAA|2000-08-28|1207|5252|403|2000|1|8|28|3|2000|403|5252|Monday|2000Q3|N|N|N|2451758|2451970|2451419|2451694|N|N|N|N|N| +2451786|AAAAAAAAKEJGFCAA|2000-08-29|1207|5253|403|2000|2|8|29|3|2000|403|5253|Tuesday|2000Q3|N|N|N|2451758|2451970|2451420|2451695|N|N|N|N|N| +2451787|AAAAAAAALEJGFCAA|2000-08-30|1207|5253|403|2000|3|8|30|3|2000|403|5253|Wednesday|2000Q3|N|N|N|2451758|2451970|2451421|2451696|N|N|N|N|N| +2451788|AAAAAAAAMEJGFCAA|2000-08-31|1207|5253|403|2000|4|8|31|3|2000|403|5253|Thursday|2000Q3|N|N|N|2451758|2451970|2451422|2451697|N|N|N|N|N| +2451789|AAAAAAAANEJGFCAA|2000-09-01|1208|5253|404|2000|5|9|1|3|2000|404|5253|Friday|2000Q3|N|Y|N|2451789|2452032|2451423|2451698|N|N|N|N|N| +2451790|AAAAAAAAOEJGFCAA|2000-09-02|1208|5253|404|2000|6|9|2|3|2000|404|5253|Saturday|2000Q3|N|Y|N|2451789|2452032|2451424|2451699|N|N|N|N|N| +2451791|AAAAAAAAPEJGFCAA|2000-09-03|1208|5253|404|2000|0|9|3|3|2000|404|5253|Sunday|2000Q3|N|N|N|2451789|2452032|2451425|2451700|N|N|N|N|N| +2451792|AAAAAAAAAFJGFCAA|2000-09-04|1208|5253|404|2000|1|9|4|3|2000|404|5253|Monday|2000Q3|N|N|N|2451789|2452032|2451426|2451701|N|N|N|N|N| +2451793|AAAAAAAABFJGFCAA|2000-09-05|1208|5254|404|2000|2|9|5|3|2000|404|5254|Tuesday|2000Q3|N|N|N|2451789|2452032|2451427|2451702|N|N|N|N|N| +2451794|AAAAAAAACFJGFCAA|2000-09-06|1208|5254|404|2000|3|9|6|3|2000|404|5254|Wednesday|2000Q3|N|N|N|2451789|2452032|2451428|2451703|N|N|N|N|N| +2451795|AAAAAAAADFJGFCAA|2000-09-07|1208|5254|404|2000|4|9|7|3|2000|404|5254|Thursday|2000Q3|N|N|N|2451789|2452032|2451429|2451704|N|N|N|N|N| +2451796|AAAAAAAAEFJGFCAA|2000-09-08|1208|5254|404|2000|5|9|8|3|2000|404|5254|Friday|2000Q3|N|Y|N|2451789|2452032|2451430|2451705|N|N|N|N|N| +2451797|AAAAAAAAFFJGFCAA|2000-09-09|1208|5254|404|2000|6|9|9|3|2000|404|5254|Saturday|2000Q3|N|Y|N|2451789|2452032|2451431|2451706|N|N|N|N|N| +2451798|AAAAAAAAGFJGFCAA|2000-09-10|1208|5254|404|2000|0|9|10|3|2000|404|5254|Sunday|2000Q3|N|N|N|2451789|2452032|2451432|2451707|N|N|N|N|N| +2451799|AAAAAAAAHFJGFCAA|2000-09-11|1208|5254|404|2000|1|9|11|3|2000|404|5254|Monday|2000Q3|N|N|N|2451789|2452032|2451433|2451708|N|N|N|N|N| +2451800|AAAAAAAAIFJGFCAA|2000-09-12|1208|5255|404|2000|2|9|12|3|2000|404|5255|Tuesday|2000Q3|N|N|N|2451789|2452032|2451434|2451709|N|N|N|N|N| +2451801|AAAAAAAAJFJGFCAA|2000-09-13|1208|5255|404|2000|3|9|13|3|2000|404|5255|Wednesday|2000Q3|N|N|N|2451789|2452032|2451435|2451710|N|N|N|N|N| +2451802|AAAAAAAAKFJGFCAA|2000-09-14|1208|5255|404|2000|4|9|14|3|2000|404|5255|Thursday|2000Q3|N|N|N|2451789|2452032|2451436|2451711|N|N|N|N|N| +2451803|AAAAAAAALFJGFCAA|2000-09-15|1208|5255|404|2000|5|9|15|3|2000|404|5255|Friday|2000Q3|N|Y|N|2451789|2452032|2451437|2451712|N|N|N|N|N| +2451804|AAAAAAAAMFJGFCAA|2000-09-16|1208|5255|404|2000|6|9|16|3|2000|404|5255|Saturday|2000Q3|N|Y|N|2451789|2452032|2451438|2451713|N|N|N|N|N| +2451805|AAAAAAAANFJGFCAA|2000-09-17|1208|5255|404|2000|0|9|17|3|2000|404|5255|Sunday|2000Q3|N|N|N|2451789|2452032|2451439|2451714|N|N|N|N|N| +2451806|AAAAAAAAOFJGFCAA|2000-09-18|1208|5255|404|2000|1|9|18|3|2000|404|5255|Monday|2000Q3|N|N|N|2451789|2452032|2451440|2451715|N|N|N|N|N| +2451807|AAAAAAAAPFJGFCAA|2000-09-19|1208|5256|404|2000|2|9|19|3|2000|404|5256|Tuesday|2000Q3|N|N|N|2451789|2452032|2451441|2451716|N|N|N|N|N| +2451808|AAAAAAAAAGJGFCAA|2000-09-20|1208|5256|404|2000|3|9|20|3|2000|404|5256|Wednesday|2000Q3|N|N|N|2451789|2452032|2451442|2451717|N|N|N|N|N| +2451809|AAAAAAAABGJGFCAA|2000-09-21|1208|5256|404|2000|4|9|21|3|2000|404|5256|Thursday|2000Q3|N|N|N|2451789|2452032|2451443|2451718|N|N|N|N|N| +2451810|AAAAAAAACGJGFCAA|2000-09-22|1208|5256|404|2000|5|9|22|3|2000|404|5256|Friday|2000Q3|N|Y|N|2451789|2452032|2451444|2451719|N|N|N|N|N| +2451811|AAAAAAAADGJGFCAA|2000-09-23|1208|5256|404|2000|6|9|23|3|2000|404|5256|Saturday|2000Q3|N|Y|N|2451789|2452032|2451445|2451720|N|N|N|N|N| +2451812|AAAAAAAAEGJGFCAA|2000-09-24|1208|5256|404|2000|0|9|24|3|2000|404|5256|Sunday|2000Q3|N|N|N|2451789|2452032|2451446|2451721|N|N|N|N|N| +2451813|AAAAAAAAFGJGFCAA|2000-09-25|1208|5256|404|2000|1|9|25|3|2000|404|5256|Monday|2000Q3|N|N|N|2451789|2452032|2451447|2451722|N|N|N|N|N| +2451814|AAAAAAAAGGJGFCAA|2000-09-26|1208|5257|404|2000|2|9|26|3|2000|404|5257|Tuesday|2000Q3|N|N|N|2451789|2452032|2451448|2451723|N|N|N|N|N| +2451815|AAAAAAAAHGJGFCAA|2000-09-27|1208|5257|404|2000|3|9|27|3|2000|404|5257|Wednesday|2000Q3|N|N|N|2451789|2452032|2451449|2451724|N|N|N|N|N| +2451816|AAAAAAAAIGJGFCAA|2000-09-28|1208|5257|404|2000|4|9|28|3|2000|404|5257|Thursday|2000Q3|N|N|N|2451789|2452032|2451450|2451725|N|N|N|N|N| +2451817|AAAAAAAAJGJGFCAA|2000-09-29|1208|5257|404|2000|5|9|29|3|2000|404|5257|Friday|2000Q3|N|Y|N|2451789|2452032|2451451|2451726|N|N|N|N|N| +2451818|AAAAAAAAKGJGFCAA|2000-09-30|1208|5257|404|2000|6|9|30|3|2000|404|5257|Saturday|2000Q3|N|Y|N|2451789|2452032|2451452|2451727|N|N|N|N|N| +2451819|AAAAAAAALGJGFCAA|2000-10-01|1209|5257|404|2000|0|10|1|4|2000|404|5257|Sunday|2000Q4|N|N|N|2451819|2452092|2451453|2451727|N|N|N|N|N| +2451820|AAAAAAAAMGJGFCAA|2000-10-02|1209|5257|404|2000|1|10|2|4|2000|404|5257|Monday|2000Q4|N|N|N|2451819|2452092|2451454|2451728|N|N|N|N|N| +2451821|AAAAAAAANGJGFCAA|2000-10-03|1209|5258|404|2000|2|10|3|4|2000|404|5258|Tuesday|2000Q4|N|N|N|2451819|2452092|2451455|2451729|N|N|N|N|N| +2451822|AAAAAAAAOGJGFCAA|2000-10-04|1209|5258|404|2000|3|10|4|4|2000|404|5258|Wednesday|2000Q4|N|N|N|2451819|2452092|2451456|2451730|N|N|N|N|N| +2451823|AAAAAAAAPGJGFCAA|2000-10-05|1209|5258|404|2000|4|10|5|4|2000|404|5258|Thursday|2000Q4|N|N|N|2451819|2452092|2451457|2451731|N|N|N|N|N| +2451824|AAAAAAAAAHJGFCAA|2000-10-06|1209|5258|404|2000|5|10|6|4|2000|404|5258|Friday|2000Q4|N|Y|N|2451819|2452092|2451458|2451732|N|N|N|N|N| +2451825|AAAAAAAABHJGFCAA|2000-10-07|1209|5258|404|2000|6|10|7|4|2000|404|5258|Saturday|2000Q4|N|Y|N|2451819|2452092|2451459|2451733|N|N|N|N|N| +2451826|AAAAAAAACHJGFCAA|2000-10-08|1209|5258|404|2000|0|10|8|4|2000|404|5258|Sunday|2000Q4|N|N|N|2451819|2452092|2451460|2451734|N|N|N|N|N| +2451827|AAAAAAAADHJGFCAA|2000-10-09|1209|5258|404|2000|1|10|9|4|2000|404|5258|Monday|2000Q4|N|N|N|2451819|2452092|2451461|2451735|N|N|N|N|N| +2451828|AAAAAAAAEHJGFCAA|2000-10-10|1209|5259|404|2000|2|10|10|4|2000|404|5259|Tuesday|2000Q4|N|N|N|2451819|2452092|2451462|2451736|N|N|N|N|N| +2451829|AAAAAAAAFHJGFCAA|2000-10-11|1209|5259|404|2000|3|10|11|4|2000|404|5259|Wednesday|2000Q4|N|N|N|2451819|2452092|2451463|2451737|N|N|N|N|N| +2451830|AAAAAAAAGHJGFCAA|2000-10-12|1209|5259|404|2000|4|10|12|4|2000|404|5259|Thursday|2000Q4|N|N|N|2451819|2452092|2451464|2451738|N|N|N|N|N| +2451831|AAAAAAAAHHJGFCAA|2000-10-13|1209|5259|404|2000|5|10|13|4|2000|404|5259|Friday|2000Q4|N|Y|N|2451819|2452092|2451465|2451739|N|N|N|N|N| +2451832|AAAAAAAAIHJGFCAA|2000-10-14|1209|5259|404|2000|6|10|14|4|2000|404|5259|Saturday|2000Q4|N|Y|N|2451819|2452092|2451466|2451740|N|N|N|N|N| +2451833|AAAAAAAAJHJGFCAA|2000-10-15|1209|5259|404|2000|0|10|15|4|2000|404|5259|Sunday|2000Q4|N|N|N|2451819|2452092|2451467|2451741|N|N|N|N|N| +2451834|AAAAAAAAKHJGFCAA|2000-10-16|1209|5259|404|2000|1|10|16|4|2000|404|5259|Monday|2000Q4|N|N|N|2451819|2452092|2451468|2451742|N|N|N|N|N| +2451835|AAAAAAAALHJGFCAA|2000-10-17|1209|5260|404|2000|2|10|17|4|2000|404|5260|Tuesday|2000Q4|N|N|N|2451819|2452092|2451469|2451743|N|N|N|N|N| +2451836|AAAAAAAAMHJGFCAA|2000-10-18|1209|5260|404|2000|3|10|18|4|2000|404|5260|Wednesday|2000Q4|N|N|N|2451819|2452092|2451470|2451744|N|N|N|N|N| +2451837|AAAAAAAANHJGFCAA|2000-10-19|1209|5260|404|2000|4|10|19|4|2000|404|5260|Thursday|2000Q4|N|N|N|2451819|2452092|2451471|2451745|N|N|N|N|N| +2451838|AAAAAAAAOHJGFCAA|2000-10-20|1209|5260|404|2000|5|10|20|4|2000|404|5260|Friday|2000Q4|N|Y|N|2451819|2452092|2451472|2451746|N|N|N|N|N| +2451839|AAAAAAAAPHJGFCAA|2000-10-21|1209|5260|404|2000|6|10|21|4|2000|404|5260|Saturday|2000Q4|N|Y|N|2451819|2452092|2451473|2451747|N|N|N|N|N| +2451840|AAAAAAAAAIJGFCAA|2000-10-22|1209|5260|404|2000|0|10|22|4|2000|404|5260|Sunday|2000Q4|N|N|N|2451819|2452092|2451474|2451748|N|N|N|N|N| +2451841|AAAAAAAABIJGFCAA|2000-10-23|1209|5260|404|2000|1|10|23|4|2000|404|5260|Monday|2000Q4|N|N|N|2451819|2452092|2451475|2451749|N|N|N|N|N| +2451842|AAAAAAAACIJGFCAA|2000-10-24|1209|5261|404|2000|2|10|24|4|2000|404|5261|Tuesday|2000Q4|N|N|N|2451819|2452092|2451476|2451750|N|N|N|N|N| +2451843|AAAAAAAADIJGFCAA|2000-10-25|1209|5261|404|2000|3|10|25|4|2000|404|5261|Wednesday|2000Q4|N|N|N|2451819|2452092|2451477|2451751|N|N|N|N|N| +2451844|AAAAAAAAEIJGFCAA|2000-10-26|1209|5261|404|2000|4|10|26|4|2000|404|5261|Thursday|2000Q4|N|N|N|2451819|2452092|2451478|2451752|N|N|N|N|N| +2451845|AAAAAAAAFIJGFCAA|2000-10-27|1209|5261|404|2000|5|10|27|4|2000|404|5261|Friday|2000Q4|N|Y|N|2451819|2452092|2451479|2451753|N|N|N|N|N| +2451846|AAAAAAAAGIJGFCAA|2000-10-28|1209|5261|404|2000|6|10|28|4|2000|404|5261|Saturday|2000Q4|N|Y|N|2451819|2452092|2451480|2451754|N|N|N|N|N| +2451847|AAAAAAAAHIJGFCAA|2000-10-29|1209|5261|404|2000|0|10|29|4|2000|404|5261|Sunday|2000Q4|N|N|N|2451819|2452092|2451481|2451755|N|N|N|N|N| +2451848|AAAAAAAAIIJGFCAA|2000-10-30|1209|5261|404|2000|1|10|30|4|2000|404|5261|Monday|2000Q4|N|N|N|2451819|2452092|2451482|2451756|N|N|N|N|N| +2451849|AAAAAAAAJIJGFCAA|2000-10-31|1209|5262|404|2000|2|10|31|4|2000|404|5262|Tuesday|2000Q4|N|N|N|2451819|2452092|2451483|2451757|N|N|N|N|N| +2451850|AAAAAAAAKIJGFCAA|2000-11-01|1210|5262|404|2000|3|11|1|4|2000|404|5262|Wednesday|2000Q4|N|N|N|2451850|2452154|2451484|2451758|N|N|N|N|N| +2451851|AAAAAAAALIJGFCAA|2000-11-02|1210|5262|404|2000|4|11|2|4|2000|404|5262|Thursday|2000Q4|N|N|N|2451850|2452154|2451485|2451759|N|N|N|N|N| +2451852|AAAAAAAAMIJGFCAA|2000-11-03|1210|5262|404|2000|5|11|3|4|2000|404|5262|Friday|2000Q4|N|Y|N|2451850|2452154|2451486|2451760|N|N|N|N|N| +2451853|AAAAAAAANIJGFCAA|2000-11-04|1210|5262|404|2000|6|11|4|4|2000|404|5262|Saturday|2000Q4|N|Y|N|2451850|2452154|2451487|2451761|N|N|N|N|N| +2451854|AAAAAAAAOIJGFCAA|2000-11-05|1210|5262|404|2000|0|11|5|4|2000|404|5262|Sunday|2000Q4|N|N|N|2451850|2452154|2451488|2451762|N|N|N|N|N| +2451855|AAAAAAAAPIJGFCAA|2000-11-06|1210|5262|404|2000|1|11|6|4|2000|404|5262|Monday|2000Q4|N|N|N|2451850|2452154|2451489|2451763|N|N|N|N|N| +2451856|AAAAAAAAAJJGFCAA|2000-11-07|1210|5263|404|2000|2|11|7|4|2000|404|5263|Tuesday|2000Q4|N|N|N|2451850|2452154|2451490|2451764|N|N|N|N|N| +2451857|AAAAAAAABJJGFCAA|2000-11-08|1210|5263|404|2000|3|11|8|4|2000|404|5263|Wednesday|2000Q4|N|N|N|2451850|2452154|2451491|2451765|N|N|N|N|N| +2451858|AAAAAAAACJJGFCAA|2000-11-09|1210|5263|404|2000|4|11|9|4|2000|404|5263|Thursday|2000Q4|N|N|N|2451850|2452154|2451492|2451766|N|N|N|N|N| +2451859|AAAAAAAADJJGFCAA|2000-11-10|1210|5263|404|2000|5|11|10|4|2000|404|5263|Friday|2000Q4|N|Y|N|2451850|2452154|2451493|2451767|N|N|N|N|N| +2451860|AAAAAAAAEJJGFCAA|2000-11-11|1210|5263|404|2000|6|11|11|4|2000|404|5263|Saturday|2000Q4|N|Y|N|2451850|2452154|2451494|2451768|N|N|N|N|N| +2451861|AAAAAAAAFJJGFCAA|2000-11-12|1210|5263|404|2000|0|11|12|4|2000|404|5263|Sunday|2000Q4|N|N|N|2451850|2452154|2451495|2451769|N|N|N|N|N| +2451862|AAAAAAAAGJJGFCAA|2000-11-13|1210|5263|404|2000|1|11|13|4|2000|404|5263|Monday|2000Q4|N|N|N|2451850|2452154|2451496|2451770|N|N|N|N|N| +2451863|AAAAAAAAHJJGFCAA|2000-11-14|1210|5264|404|2000|2|11|14|4|2000|404|5264|Tuesday|2000Q4|N|N|N|2451850|2452154|2451497|2451771|N|N|N|N|N| +2451864|AAAAAAAAIJJGFCAA|2000-11-15|1210|5264|404|2000|3|11|15|4|2000|404|5264|Wednesday|2000Q4|N|N|N|2451850|2452154|2451498|2451772|N|N|N|N|N| +2451865|AAAAAAAAJJJGFCAA|2000-11-16|1210|5264|404|2000|4|11|16|4|2000|404|5264|Thursday|2000Q4|N|N|N|2451850|2452154|2451499|2451773|N|N|N|N|N| +2451866|AAAAAAAAKJJGFCAA|2000-11-17|1210|5264|404|2000|5|11|17|4|2000|404|5264|Friday|2000Q4|N|Y|N|2451850|2452154|2451500|2451774|N|N|N|N|N| +2451867|AAAAAAAALJJGFCAA|2000-11-18|1210|5264|404|2000|6|11|18|4|2000|404|5264|Saturday|2000Q4|N|Y|N|2451850|2452154|2451501|2451775|N|N|N|N|N| +2451868|AAAAAAAAMJJGFCAA|2000-11-19|1210|5264|404|2000|0|11|19|4|2000|404|5264|Sunday|2000Q4|N|N|N|2451850|2452154|2451502|2451776|N|N|N|N|N| +2451869|AAAAAAAANJJGFCAA|2000-11-20|1210|5264|404|2000|1|11|20|4|2000|404|5264|Monday|2000Q4|N|N|N|2451850|2452154|2451503|2451777|N|N|N|N|N| +2451870|AAAAAAAAOJJGFCAA|2000-11-21|1210|5265|404|2000|2|11|21|4|2000|404|5265|Tuesday|2000Q4|N|N|N|2451850|2452154|2451504|2451778|N|N|N|N|N| +2451871|AAAAAAAAPJJGFCAA|2000-11-22|1210|5265|404|2000|3|11|22|4|2000|404|5265|Wednesday|2000Q4|N|N|N|2451850|2452154|2451505|2451779|N|N|N|N|N| +2451872|AAAAAAAAAKJGFCAA|2000-11-23|1210|5265|404|2000|4|11|23|4|2000|404|5265|Thursday|2000Q4|N|N|N|2451850|2452154|2451506|2451780|N|N|N|N|N| +2451873|AAAAAAAABKJGFCAA|2000-11-24|1210|5265|404|2000|5|11|24|4|2000|404|5265|Friday|2000Q4|N|Y|N|2451850|2452154|2451507|2451781|N|N|N|N|N| +2451874|AAAAAAAACKJGFCAA|2000-11-25|1210|5265|404|2000|6|11|25|4|2000|404|5265|Saturday|2000Q4|N|Y|N|2451850|2452154|2451508|2451782|N|N|N|N|N| +2451875|AAAAAAAADKJGFCAA|2000-11-26|1210|5265|404|2000|0|11|26|4|2000|404|5265|Sunday|2000Q4|N|N|N|2451850|2452154|2451509|2451783|N|N|N|N|N| +2451876|AAAAAAAAEKJGFCAA|2000-11-27|1210|5265|404|2000|1|11|27|4|2000|404|5265|Monday|2000Q4|N|N|N|2451850|2452154|2451510|2451784|N|N|N|N|N| +2451877|AAAAAAAAFKJGFCAA|2000-11-28|1210|5266|404|2000|2|11|28|4|2000|404|5266|Tuesday|2000Q4|N|N|N|2451850|2452154|2451511|2451785|N|N|N|N|N| +2451878|AAAAAAAAGKJGFCAA|2000-11-29|1210|5266|404|2000|3|11|29|4|2000|404|5266|Wednesday|2000Q4|N|N|N|2451850|2452154|2451512|2451786|N|N|N|N|N| +2451879|AAAAAAAAHKJGFCAA|2000-11-30|1210|5266|404|2000|4|11|30|4|2000|404|5266|Thursday|2000Q4|N|N|N|2451850|2452154|2451513|2451787|N|N|N|N|N| +2451880|AAAAAAAAIKJGFCAA|2000-12-01|1211|5266|405|2000|5|12|1|4|2000|405|5266|Friday|2000Q4|N|Y|N|2451880|2452214|2451514|2451788|N|N|N|N|N| +2451881|AAAAAAAAJKJGFCAA|2000-12-02|1211|5266|405|2000|6|12|2|4|2000|405|5266|Saturday|2000Q4|N|Y|N|2451880|2452214|2451515|2451789|N|N|N|N|N| +2451882|AAAAAAAAKKJGFCAA|2000-12-03|1211|5266|405|2000|0|12|3|4|2000|405|5266|Sunday|2000Q4|N|N|N|2451880|2452214|2451516|2451790|N|N|N|N|N| +2451883|AAAAAAAALKJGFCAA|2000-12-04|1211|5266|405|2000|1|12|4|4|2000|405|5266|Monday|2000Q4|N|N|N|2451880|2452214|2451517|2451791|N|N|N|N|N| +2451884|AAAAAAAAMKJGFCAA|2000-12-05|1211|5267|405|2000|2|12|5|4|2000|405|5267|Tuesday|2000Q4|N|N|N|2451880|2452214|2451518|2451792|N|N|N|N|N| +2451885|AAAAAAAANKJGFCAA|2000-12-06|1211|5267|405|2000|3|12|6|4|2000|405|5267|Wednesday|2000Q4|N|N|N|2451880|2452214|2451519|2451793|N|N|N|N|N| +2451886|AAAAAAAAOKJGFCAA|2000-12-07|1211|5267|405|2000|4|12|7|4|2000|405|5267|Thursday|2000Q4|N|N|N|2451880|2452214|2451520|2451794|N|N|N|N|N| +2451887|AAAAAAAAPKJGFCAA|2000-12-08|1211|5267|405|2000|5|12|8|4|2000|405|5267|Friday|2000Q4|N|Y|N|2451880|2452214|2451521|2451795|N|N|N|N|N| +2451888|AAAAAAAAALJGFCAA|2000-12-09|1211|5267|405|2000|6|12|9|4|2000|405|5267|Saturday|2000Q4|N|Y|N|2451880|2452214|2451522|2451796|N|N|N|N|N| +2451889|AAAAAAAABLJGFCAA|2000-12-10|1211|5267|405|2000|0|12|10|4|2000|405|5267|Sunday|2000Q4|N|N|N|2451880|2452214|2451523|2451797|N|N|N|N|N| +2451890|AAAAAAAACLJGFCAA|2000-12-11|1211|5267|405|2000|1|12|11|4|2000|405|5267|Monday|2000Q4|N|N|N|2451880|2452214|2451524|2451798|N|N|N|N|N| +2451891|AAAAAAAADLJGFCAA|2000-12-12|1211|5268|405|2000|2|12|12|4|2000|405|5268|Tuesday|2000Q4|N|N|N|2451880|2452214|2451525|2451799|N|N|N|N|N| +2451892|AAAAAAAAELJGFCAA|2000-12-13|1211|5268|405|2000|3|12|13|4|2000|405|5268|Wednesday|2000Q4|N|N|N|2451880|2452214|2451526|2451800|N|N|N|N|N| +2451893|AAAAAAAAFLJGFCAA|2000-12-14|1211|5268|405|2000|4|12|14|4|2000|405|5268|Thursday|2000Q4|N|N|N|2451880|2452214|2451527|2451801|N|N|N|N|N| +2451894|AAAAAAAAGLJGFCAA|2000-12-15|1211|5268|405|2000|5|12|15|4|2000|405|5268|Friday|2000Q4|N|Y|N|2451880|2452214|2451528|2451802|N|N|N|N|N| +2451895|AAAAAAAAHLJGFCAA|2000-12-16|1211|5268|405|2000|6|12|16|4|2000|405|5268|Saturday|2000Q4|N|Y|N|2451880|2452214|2451529|2451803|N|N|N|N|N| +2451896|AAAAAAAAILJGFCAA|2000-12-17|1211|5268|405|2000|0|12|17|4|2000|405|5268|Sunday|2000Q4|N|N|N|2451880|2452214|2451530|2451804|N|N|N|N|N| +2451897|AAAAAAAAJLJGFCAA|2000-12-18|1211|5268|405|2000|1|12|18|4|2000|405|5268|Monday|2000Q4|N|N|N|2451880|2452214|2451531|2451805|N|N|N|N|N| +2451898|AAAAAAAAKLJGFCAA|2000-12-19|1211|5269|405|2000|2|12|19|4|2000|405|5269|Tuesday|2000Q4|N|N|N|2451880|2452214|2451532|2451806|N|N|N|N|N| +2451899|AAAAAAAALLJGFCAA|2000-12-20|1211|5269|405|2000|3|12|20|4|2000|405|5269|Wednesday|2000Q4|N|N|N|2451880|2452214|2451533|2451807|N|N|N|N|N| +2451900|AAAAAAAAMLJGFCAA|2000-12-21|1211|5269|405|2000|4|12|21|4|2000|405|5269|Thursday|2000Q4|N|N|N|2451880|2452214|2451534|2451808|N|N|N|N|N| +2451901|AAAAAAAANLJGFCAA|2000-12-22|1211|5269|405|2000|5|12|22|4|2000|405|5269|Friday|2000Q4|N|Y|N|2451880|2452214|2451535|2451809|N|N|N|N|N| +2451902|AAAAAAAAOLJGFCAA|2000-12-23|1211|5269|405|2000|6|12|23|4|2000|405|5269|Saturday|2000Q4|N|Y|N|2451880|2452214|2451536|2451810|N|N|N|N|N| +2451903|AAAAAAAAPLJGFCAA|2000-12-24|1211|5269|405|2000|0|12|24|4|2000|405|5269|Sunday|2000Q4|N|N|N|2451880|2452214|2451537|2451811|N|N|N|N|N| +2451904|AAAAAAAAAMJGFCAA|2000-12-25|1211|5269|405|2000|1|12|25|4|2000|405|5269|Monday|2000Q4|Y|N|N|2451880|2452214|2451538|2451812|N|N|N|N|N| +2451905|AAAAAAAABMJGFCAA|2000-12-26|1211|5270|405|2000|2|12|26|4|2000|405|5270|Tuesday|2000Q4|N|N|Y|2451880|2452214|2451539|2451813|N|N|N|N|N| +2451906|AAAAAAAACMJGFCAA|2000-12-27|1211|5270|405|2000|3|12|27|4|2000|405|5270|Wednesday|2000Q4|N|N|N|2451880|2452214|2451540|2451814|N|N|N|N|N| +2451907|AAAAAAAADMJGFCAA|2000-12-28|1211|5270|405|2000|4|12|28|4|2000|405|5270|Thursday|2000Q4|N|N|N|2451880|2452214|2451541|2451815|N|N|N|N|N| +2451908|AAAAAAAAEMJGFCAA|2000-12-29|1211|5270|405|2000|5|12|29|4|2000|405|5270|Friday|2000Q4|N|Y|N|2451880|2452214|2451542|2451816|N|N|N|N|N| +2451909|AAAAAAAAFMJGFCAA|2000-12-30|1211|5270|405|2000|6|12|30|4|2000|405|5270|Saturday|2000Q4|N|Y|N|2451880|2452214|2451543|2451817|N|N|N|N|N| +2451910|AAAAAAAAGMJGFCAA|2000-12-31|1211|5270|405|2000|0|12|31|4|2000|405|5270|Sunday|2000Q4|Y|N|N|2451880|2452214|2451544|2451818|N|N|N|N|N| +2451911|AAAAAAAAHMJGFCAA|2001-01-01|1212|5270|405|2001|1|1|1|1|2001|405|5270|Monday|2001Q1|Y|N|Y|2451911|2451910|2451545|2451819|N|N|N|N|N| +2451912|AAAAAAAAIMJGFCAA|2001-01-02|1212|5271|405|2001|2|1|2|1|2001|405|5271|Tuesday|2001Q1|N|N|Y|2451911|2451910|2451546|2451820|N|N|N|N|N| +2451913|AAAAAAAAJMJGFCAA|2001-01-03|1212|5271|405|2001|3|1|3|1|2001|405|5271|Wednesday|2001Q1|N|N|N|2451911|2451910|2451547|2451821|N|N|N|N|N| +2451914|AAAAAAAAKMJGFCAA|2001-01-04|1212|5271|405|2001|4|1|4|1|2001|405|5271|Thursday|2001Q1|N|N|N|2451911|2451910|2451548|2451822|N|N|N|N|N| +2451915|AAAAAAAALMJGFCAA|2001-01-05|1212|5271|405|2001|5|1|5|1|2001|405|5271|Friday|2001Q1|N|Y|N|2451911|2451910|2451549|2451823|N|N|N|N|N| +2451916|AAAAAAAAMMJGFCAA|2001-01-06|1212|5271|405|2001|6|1|6|1|2001|405|5271|Saturday|2001Q1|N|Y|N|2451911|2451910|2451550|2451824|N|N|N|N|N| +2451917|AAAAAAAANMJGFCAA|2001-01-07|1212|5271|405|2001|0|1|7|1|2001|405|5271|Sunday|2001Q1|N|N|N|2451911|2451910|2451551|2451825|N|N|N|N|N| +2451918|AAAAAAAAOMJGFCAA|2001-01-08|1212|5271|405|2001|1|1|8|1|2001|405|5271|Monday|2001Q1|N|N|N|2451911|2451910|2451552|2451826|N|N|N|N|N| +2451919|AAAAAAAAPMJGFCAA|2001-01-09|1212|5272|405|2001|2|1|9|1|2001|405|5272|Tuesday|2001Q1|N|N|N|2451911|2451910|2451553|2451827|N|N|N|N|N| +2451920|AAAAAAAAANJGFCAA|2001-01-10|1212|5272|405|2001|3|1|10|1|2001|405|5272|Wednesday|2001Q1|N|N|N|2451911|2451910|2451554|2451828|N|N|N|N|N| +2451921|AAAAAAAABNJGFCAA|2001-01-11|1212|5272|405|2001|4|1|11|1|2001|405|5272|Thursday|2001Q1|N|N|N|2451911|2451910|2451555|2451829|N|N|N|N|N| +2451922|AAAAAAAACNJGFCAA|2001-01-12|1212|5272|405|2001|5|1|12|1|2001|405|5272|Friday|2001Q1|N|Y|N|2451911|2451910|2451556|2451830|N|N|N|N|N| +2451923|AAAAAAAADNJGFCAA|2001-01-13|1212|5272|405|2001|6|1|13|1|2001|405|5272|Saturday|2001Q1|N|Y|N|2451911|2451910|2451557|2451831|N|N|N|N|N| +2451924|AAAAAAAAENJGFCAA|2001-01-14|1212|5272|405|2001|0|1|14|1|2001|405|5272|Sunday|2001Q1|N|N|N|2451911|2451910|2451558|2451832|N|N|N|N|N| +2451925|AAAAAAAAFNJGFCAA|2001-01-15|1212|5272|405|2001|1|1|15|1|2001|405|5272|Monday|2001Q1|N|N|N|2451911|2451910|2451559|2451833|N|N|N|N|N| +2451926|AAAAAAAAGNJGFCAA|2001-01-16|1212|5273|405|2001|2|1|16|1|2001|405|5273|Tuesday|2001Q1|N|N|N|2451911|2451910|2451560|2451834|N|N|N|N|N| +2451927|AAAAAAAAHNJGFCAA|2001-01-17|1212|5273|405|2001|3|1|17|1|2001|405|5273|Wednesday|2001Q1|N|N|N|2451911|2451910|2451561|2451835|N|N|N|N|N| +2451928|AAAAAAAAINJGFCAA|2001-01-18|1212|5273|405|2001|4|1|18|1|2001|405|5273|Thursday|2001Q1|N|N|N|2451911|2451910|2451562|2451836|N|N|N|N|N| +2451929|AAAAAAAAJNJGFCAA|2001-01-19|1212|5273|405|2001|5|1|19|1|2001|405|5273|Friday|2001Q1|N|Y|N|2451911|2451910|2451563|2451837|N|N|N|N|N| +2451930|AAAAAAAAKNJGFCAA|2001-01-20|1212|5273|405|2001|6|1|20|1|2001|405|5273|Saturday|2001Q1|N|Y|N|2451911|2451910|2451564|2451838|N|N|N|N|N| +2451931|AAAAAAAALNJGFCAA|2001-01-21|1212|5273|405|2001|0|1|21|1|2001|405|5273|Sunday|2001Q1|N|N|N|2451911|2451910|2451565|2451839|N|N|N|N|N| +2451932|AAAAAAAAMNJGFCAA|2001-01-22|1212|5273|405|2001|1|1|22|1|2001|405|5273|Monday|2001Q1|N|N|N|2451911|2451910|2451566|2451840|N|N|N|N|N| +2451933|AAAAAAAANNJGFCAA|2001-01-23|1212|5274|405|2001|2|1|23|1|2001|405|5274|Tuesday|2001Q1|N|N|N|2451911|2451910|2451567|2451841|N|N|N|N|N| +2451934|AAAAAAAAONJGFCAA|2001-01-24|1212|5274|405|2001|3|1|24|1|2001|405|5274|Wednesday|2001Q1|N|N|N|2451911|2451910|2451568|2451842|N|N|N|N|N| +2451935|AAAAAAAAPNJGFCAA|2001-01-25|1212|5274|405|2001|4|1|25|1|2001|405|5274|Thursday|2001Q1|N|N|N|2451911|2451910|2451569|2451843|N|N|N|N|N| +2451936|AAAAAAAAAOJGFCAA|2001-01-26|1212|5274|405|2001|5|1|26|1|2001|405|5274|Friday|2001Q1|N|Y|N|2451911|2451910|2451570|2451844|N|N|N|N|N| +2451937|AAAAAAAABOJGFCAA|2001-01-27|1212|5274|405|2001|6|1|27|1|2001|405|5274|Saturday|2001Q1|N|Y|N|2451911|2451910|2451571|2451845|N|N|N|N|N| +2451938|AAAAAAAACOJGFCAA|2001-01-28|1212|5274|405|2001|0|1|28|1|2001|405|5274|Sunday|2001Q1|N|N|N|2451911|2451910|2451572|2451846|N|N|N|N|N| +2451939|AAAAAAAADOJGFCAA|2001-01-29|1212|5274|405|2001|1|1|29|1|2001|405|5274|Monday|2001Q1|N|N|N|2451911|2451910|2451573|2451847|N|N|N|N|N| +2451940|AAAAAAAAEOJGFCAA|2001-01-30|1212|5275|405|2001|2|1|30|1|2001|405|5275|Tuesday|2001Q1|N|N|N|2451911|2451910|2451574|2451848|N|N|N|N|N| +2451941|AAAAAAAAFOJGFCAA|2001-01-31|1212|5275|405|2001|3|1|31|1|2001|405|5275|Wednesday|2001Q1|N|N|N|2451911|2451910|2451575|2451849|N|N|N|N|N| +2451942|AAAAAAAAGOJGFCAA|2001-02-01|1213|5275|405|2001|4|2|1|1|2001|405|5275|Thursday|2001Q1|N|N|N|2451942|2451972|2451576|2451850|N|N|N|N|N| +2451943|AAAAAAAAHOJGFCAA|2001-02-02|1213|5275|405|2001|5|2|2|1|2001|405|5275|Friday|2001Q1|N|Y|N|2451942|2451972|2451577|2451851|N|N|N|N|N| +2451944|AAAAAAAAIOJGFCAA|2001-02-03|1213|5275|405|2001|6|2|3|1|2001|405|5275|Saturday|2001Q1|N|Y|N|2451942|2451972|2451578|2451852|N|N|N|N|N| +2451945|AAAAAAAAJOJGFCAA|2001-02-04|1213|5275|405|2001|0|2|4|1|2001|405|5275|Sunday|2001Q1|N|N|N|2451942|2451972|2451579|2451853|N|N|N|N|N| +2451946|AAAAAAAAKOJGFCAA|2001-02-05|1213|5275|405|2001|1|2|5|1|2001|405|5275|Monday|2001Q1|N|N|N|2451942|2451972|2451580|2451854|N|N|N|N|N| +2451947|AAAAAAAALOJGFCAA|2001-02-06|1213|5276|405|2001|2|2|6|1|2001|405|5276|Tuesday|2001Q1|N|N|N|2451942|2451972|2451581|2451855|N|N|N|N|N| +2451948|AAAAAAAAMOJGFCAA|2001-02-07|1213|5276|405|2001|3|2|7|1|2001|405|5276|Wednesday|2001Q1|N|N|N|2451942|2451972|2451582|2451856|N|N|N|N|N| +2451949|AAAAAAAANOJGFCAA|2001-02-08|1213|5276|405|2001|4|2|8|1|2001|405|5276|Thursday|2001Q1|N|N|N|2451942|2451972|2451583|2451857|N|N|N|N|N| +2451950|AAAAAAAAOOJGFCAA|2001-02-09|1213|5276|405|2001|5|2|9|1|2001|405|5276|Friday|2001Q1|N|Y|N|2451942|2451972|2451584|2451858|N|N|N|N|N| +2451951|AAAAAAAAPOJGFCAA|2001-02-10|1213|5276|405|2001|6|2|10|1|2001|405|5276|Saturday|2001Q1|N|Y|N|2451942|2451972|2451585|2451859|N|N|N|N|N| +2451952|AAAAAAAAAPJGFCAA|2001-02-11|1213|5276|405|2001|0|2|11|1|2001|405|5276|Sunday|2001Q1|N|N|N|2451942|2451972|2451586|2451860|N|N|N|N|N| +2451953|AAAAAAAABPJGFCAA|2001-02-12|1213|5276|405|2001|1|2|12|1|2001|405|5276|Monday|2001Q1|N|N|N|2451942|2451972|2451587|2451861|N|N|N|N|N| +2451954|AAAAAAAACPJGFCAA|2001-02-13|1213|5277|405|2001|2|2|13|1|2001|405|5277|Tuesday|2001Q1|N|N|N|2451942|2451972|2451588|2451862|N|N|N|N|N| +2451955|AAAAAAAADPJGFCAA|2001-02-14|1213|5277|405|2001|3|2|14|1|2001|405|5277|Wednesday|2001Q1|N|N|N|2451942|2451972|2451589|2451863|N|N|N|N|N| +2451956|AAAAAAAAEPJGFCAA|2001-02-15|1213|5277|405|2001|4|2|15|1|2001|405|5277|Thursday|2001Q1|N|N|N|2451942|2451972|2451590|2451864|N|N|N|N|N| +2451957|AAAAAAAAFPJGFCAA|2001-02-16|1213|5277|405|2001|5|2|16|1|2001|405|5277|Friday|2001Q1|N|Y|N|2451942|2451972|2451591|2451865|N|N|N|N|N| +2451958|AAAAAAAAGPJGFCAA|2001-02-17|1213|5277|405|2001|6|2|17|1|2001|405|5277|Saturday|2001Q1|N|Y|N|2451942|2451972|2451592|2451866|N|N|N|N|N| +2451959|AAAAAAAAHPJGFCAA|2001-02-18|1213|5277|405|2001|0|2|18|1|2001|405|5277|Sunday|2001Q1|N|N|N|2451942|2451972|2451593|2451867|N|N|N|N|N| +2451960|AAAAAAAAIPJGFCAA|2001-02-19|1213|5277|405|2001|1|2|19|1|2001|405|5277|Monday|2001Q1|N|N|N|2451942|2451972|2451594|2451868|N|N|N|N|N| +2451961|AAAAAAAAJPJGFCAA|2001-02-20|1213|5278|405|2001|2|2|20|1|2001|405|5278|Tuesday|2001Q1|N|N|N|2451942|2451972|2451595|2451869|N|N|N|N|N| +2451962|AAAAAAAAKPJGFCAA|2001-02-21|1213|5278|405|2001|3|2|21|1|2001|405|5278|Wednesday|2001Q1|N|N|N|2451942|2451972|2451596|2451870|N|N|N|N|N| +2451963|AAAAAAAALPJGFCAA|2001-02-22|1213|5278|405|2001|4|2|22|1|2001|405|5278|Thursday|2001Q1|N|N|N|2451942|2451972|2451597|2451871|N|N|N|N|N| +2451964|AAAAAAAAMPJGFCAA|2001-02-23|1213|5278|405|2001|5|2|23|1|2001|405|5278|Friday|2001Q1|N|Y|N|2451942|2451972|2451598|2451872|N|N|N|N|N| +2451965|AAAAAAAANPJGFCAA|2001-02-24|1213|5278|405|2001|6|2|24|1|2001|405|5278|Saturday|2001Q1|N|Y|N|2451942|2451972|2451599|2451873|N|N|N|N|N| +2451966|AAAAAAAAOPJGFCAA|2001-02-25|1213|5278|405|2001|0|2|25|1|2001|405|5278|Sunday|2001Q1|N|N|N|2451942|2451972|2451600|2451874|N|N|N|N|N| +2451967|AAAAAAAAPPJGFCAA|2001-02-26|1213|5278|405|2001|1|2|26|1|2001|405|5278|Monday|2001Q1|N|N|N|2451942|2451972|2451601|2451875|N|N|N|N|N| +2451968|AAAAAAAAAAKGFCAA|2001-02-27|1213|5279|405|2001|2|2|27|1|2001|405|5279|Tuesday|2001Q1|N|N|N|2451942|2451972|2451602|2451876|N|N|N|N|N| +2451969|AAAAAAAABAKGFCAA|2001-02-28|1213|5279|405|2001|3|2|28|1|2001|405|5279|Wednesday|2001Q1|N|N|N|2451942|2451972|2451603|2451877|N|N|N|N|N| +2451970|AAAAAAAACAKGFCAA|2001-03-01|1214|5279|406|2001|4|3|1|1|2001|406|5279|Thursday|2001Q1|N|N|N|2451970|2452028|2451605|2451878|N|N|N|N|N| +2451971|AAAAAAAADAKGFCAA|2001-03-02|1214|5279|406|2001|5|3|2|1|2001|406|5279|Friday|2001Q1|N|Y|N|2451970|2452028|2451606|2451879|N|N|N|N|N| +2451972|AAAAAAAAEAKGFCAA|2001-03-03|1214|5279|406|2001|6|3|3|1|2001|406|5279|Saturday|2001Q1|N|Y|N|2451970|2452028|2451607|2451880|N|N|N|N|N| +2451973|AAAAAAAAFAKGFCAA|2001-03-04|1214|5279|406|2001|0|3|4|1|2001|406|5279|Sunday|2001Q1|N|N|N|2451970|2452028|2451608|2451881|N|N|N|N|N| +2451974|AAAAAAAAGAKGFCAA|2001-03-05|1214|5279|406|2001|1|3|5|1|2001|406|5279|Monday|2001Q1|N|N|N|2451970|2452028|2451609|2451882|N|N|N|N|N| +2451975|AAAAAAAAHAKGFCAA|2001-03-06|1214|5280|406|2001|2|3|6|1|2001|406|5280|Tuesday|2001Q1|N|N|N|2451970|2452028|2451610|2451883|N|N|N|N|N| +2451976|AAAAAAAAIAKGFCAA|2001-03-07|1214|5280|406|2001|3|3|7|1|2001|406|5280|Wednesday|2001Q1|N|N|N|2451970|2452028|2451611|2451884|N|N|N|N|N| +2451977|AAAAAAAAJAKGFCAA|2001-03-08|1214|5280|406|2001|4|3|8|1|2001|406|5280|Thursday|2001Q1|N|N|N|2451970|2452028|2451612|2451885|N|N|N|N|N| +2451978|AAAAAAAAKAKGFCAA|2001-03-09|1214|5280|406|2001|5|3|9|1|2001|406|5280|Friday|2001Q1|N|Y|N|2451970|2452028|2451613|2451886|N|N|N|N|N| +2451979|AAAAAAAALAKGFCAA|2001-03-10|1214|5280|406|2001|6|3|10|1|2001|406|5280|Saturday|2001Q1|N|Y|N|2451970|2452028|2451614|2451887|N|N|N|N|N| +2451980|AAAAAAAAMAKGFCAA|2001-03-11|1214|5280|406|2001|0|3|11|1|2001|406|5280|Sunday|2001Q1|N|N|N|2451970|2452028|2451615|2451888|N|N|N|N|N| +2451981|AAAAAAAANAKGFCAA|2001-03-12|1214|5280|406|2001|1|3|12|1|2001|406|5280|Monday|2001Q1|N|N|N|2451970|2452028|2451616|2451889|N|N|N|N|N| +2451982|AAAAAAAAOAKGFCAA|2001-03-13|1214|5281|406|2001|2|3|13|1|2001|406|5281|Tuesday|2001Q1|N|N|N|2451970|2452028|2451617|2451890|N|N|N|N|N| +2451983|AAAAAAAAPAKGFCAA|2001-03-14|1214|5281|406|2001|3|3|14|1|2001|406|5281|Wednesday|2001Q1|N|N|N|2451970|2452028|2451618|2451891|N|N|N|N|N| +2451984|AAAAAAAAABKGFCAA|2001-03-15|1214|5281|406|2001|4|3|15|1|2001|406|5281|Thursday|2001Q1|N|N|N|2451970|2452028|2451619|2451892|N|N|N|N|N| +2451985|AAAAAAAABBKGFCAA|2001-03-16|1214|5281|406|2001|5|3|16|1|2001|406|5281|Friday|2001Q1|N|Y|N|2451970|2452028|2451620|2451893|N|N|N|N|N| +2451986|AAAAAAAACBKGFCAA|2001-03-17|1214|5281|406|2001|6|3|17|1|2001|406|5281|Saturday|2001Q1|N|Y|N|2451970|2452028|2451621|2451894|N|N|N|N|N| +2451987|AAAAAAAADBKGFCAA|2001-03-18|1214|5281|406|2001|0|3|18|1|2001|406|5281|Sunday|2001Q1|N|N|N|2451970|2452028|2451622|2451895|N|N|N|N|N| +2451988|AAAAAAAAEBKGFCAA|2001-03-19|1214|5281|406|2001|1|3|19|1|2001|406|5281|Monday|2001Q1|N|N|N|2451970|2452028|2451623|2451896|N|N|N|N|N| +2451989|AAAAAAAAFBKGFCAA|2001-03-20|1214|5282|406|2001|2|3|20|1|2001|406|5282|Tuesday|2001Q1|N|N|N|2451970|2452028|2451624|2451897|N|N|N|N|N| +2451990|AAAAAAAAGBKGFCAA|2001-03-21|1214|5282|406|2001|3|3|21|1|2001|406|5282|Wednesday|2001Q1|N|N|N|2451970|2452028|2451625|2451898|N|N|N|N|N| +2451991|AAAAAAAAHBKGFCAA|2001-03-22|1214|5282|406|2001|4|3|22|1|2001|406|5282|Thursday|2001Q1|N|N|N|2451970|2452028|2451626|2451899|N|N|N|N|N| +2451992|AAAAAAAAIBKGFCAA|2001-03-23|1214|5282|406|2001|5|3|23|1|2001|406|5282|Friday|2001Q1|N|Y|N|2451970|2452028|2451627|2451900|N|N|N|N|N| +2451993|AAAAAAAAJBKGFCAA|2001-03-24|1214|5282|406|2001|6|3|24|1|2001|406|5282|Saturday|2001Q1|N|Y|N|2451970|2452028|2451628|2451901|N|N|N|N|N| +2451994|AAAAAAAAKBKGFCAA|2001-03-25|1214|5282|406|2001|0|3|25|1|2001|406|5282|Sunday|2001Q1|N|N|N|2451970|2452028|2451629|2451902|N|N|N|N|N| +2451995|AAAAAAAALBKGFCAA|2001-03-26|1214|5282|406|2001|1|3|26|1|2001|406|5282|Monday|2001Q1|N|N|N|2451970|2452028|2451630|2451903|N|N|N|N|N| +2451996|AAAAAAAAMBKGFCAA|2001-03-27|1214|5283|406|2001|2|3|27|1|2001|406|5283|Tuesday|2001Q1|N|N|N|2451970|2452028|2451631|2451904|N|N|N|N|N| +2451997|AAAAAAAANBKGFCAA|2001-03-28|1214|5283|406|2001|3|3|28|1|2001|406|5283|Wednesday|2001Q1|N|N|N|2451970|2452028|2451632|2451905|N|N|N|N|N| +2451998|AAAAAAAAOBKGFCAA|2001-03-29|1214|5283|406|2001|4|3|29|1|2001|406|5283|Thursday|2001Q1|N|N|N|2451970|2452028|2451633|2451906|N|N|N|N|N| +2451999|AAAAAAAAPBKGFCAA|2001-03-30|1214|5283|406|2001|5|3|30|1|2001|406|5283|Friday|2001Q1|N|Y|N|2451970|2452028|2451634|2451907|N|N|N|N|N| +2452000|AAAAAAAAACKGFCAA|2001-03-31|1214|5283|406|2001|6|3|31|1|2001|406|5283|Saturday|2001Q1|N|Y|N|2451970|2452028|2451635|2451908|N|N|N|N|N| +2452001|AAAAAAAABCKGFCAA|2001-04-01|1215|5283|406|2001|0|4|1|1|2001|406|5283|Sunday|2001Q1|N|N|N|2452001|2452090|2451636|2451911|N|N|N|N|N| +2452002|AAAAAAAACCKGFCAA|2001-04-02|1215|5283|406|2001|1|4|2|2|2001|406|5283|Monday|2001Q2|N|N|N|2452001|2452090|2451637|2451912|N|N|N|N|N| +2452003|AAAAAAAADCKGFCAA|2001-04-03|1215|5284|406|2001|2|4|3|2|2001|406|5284|Tuesday|2001Q2|N|N|N|2452001|2452090|2451638|2451913|N|N|N|N|N| +2452004|AAAAAAAAECKGFCAA|2001-04-04|1215|5284|406|2001|3|4|4|2|2001|406|5284|Wednesday|2001Q2|N|N|N|2452001|2452090|2451639|2451914|N|N|N|N|N| +2452005|AAAAAAAAFCKGFCAA|2001-04-05|1215|5284|406|2001|4|4|5|2|2001|406|5284|Thursday|2001Q2|N|N|N|2452001|2452090|2451640|2451915|N|N|N|N|N| +2452006|AAAAAAAAGCKGFCAA|2001-04-06|1215|5284|406|2001|5|4|6|2|2001|406|5284|Friday|2001Q2|N|Y|N|2452001|2452090|2451641|2451916|N|N|N|N|N| +2452007|AAAAAAAAHCKGFCAA|2001-04-07|1215|5284|406|2001|6|4|7|2|2001|406|5284|Saturday|2001Q2|N|Y|N|2452001|2452090|2451642|2451917|N|N|N|N|N| +2452008|AAAAAAAAICKGFCAA|2001-04-08|1215|5284|406|2001|0|4|8|2|2001|406|5284|Sunday|2001Q2|N|N|N|2452001|2452090|2451643|2451918|N|N|N|N|N| +2452009|AAAAAAAAJCKGFCAA|2001-04-09|1215|5284|406|2001|1|4|9|2|2001|406|5284|Monday|2001Q2|N|N|N|2452001|2452090|2451644|2451919|N|N|N|N|N| +2452010|AAAAAAAAKCKGFCAA|2001-04-10|1215|5285|406|2001|2|4|10|2|2001|406|5285|Tuesday|2001Q2|N|N|N|2452001|2452090|2451645|2451920|N|N|N|N|N| +2452011|AAAAAAAALCKGFCAA|2001-04-11|1215|5285|406|2001|3|4|11|2|2001|406|5285|Wednesday|2001Q2|N|N|N|2452001|2452090|2451646|2451921|N|N|N|N|N| +2452012|AAAAAAAAMCKGFCAA|2001-04-12|1215|5285|406|2001|4|4|12|2|2001|406|5285|Thursday|2001Q2|N|N|N|2452001|2452090|2451647|2451922|N|N|N|N|N| +2452013|AAAAAAAANCKGFCAA|2001-04-13|1215|5285|406|2001|5|4|13|2|2001|406|5285|Friday|2001Q2|N|Y|N|2452001|2452090|2451648|2451923|N|N|N|N|N| +2452014|AAAAAAAAOCKGFCAA|2001-04-14|1215|5285|406|2001|6|4|14|2|2001|406|5285|Saturday|2001Q2|N|Y|N|2452001|2452090|2451649|2451924|N|N|N|N|N| +2452015|AAAAAAAAPCKGFCAA|2001-04-15|1215|5285|406|2001|0|4|15|2|2001|406|5285|Sunday|2001Q2|N|N|N|2452001|2452090|2451650|2451925|N|N|N|N|N| +2452016|AAAAAAAAADKGFCAA|2001-04-16|1215|5285|406|2001|1|4|16|2|2001|406|5285|Monday|2001Q2|N|N|N|2452001|2452090|2451651|2451926|N|N|N|N|N| +2452017|AAAAAAAABDKGFCAA|2001-04-17|1215|5286|406|2001|2|4|17|2|2001|406|5286|Tuesday|2001Q2|N|N|N|2452001|2452090|2451652|2451927|N|N|N|N|N| +2452018|AAAAAAAACDKGFCAA|2001-04-18|1215|5286|406|2001|3|4|18|2|2001|406|5286|Wednesday|2001Q2|N|N|N|2452001|2452090|2451653|2451928|N|N|N|N|N| +2452019|AAAAAAAADDKGFCAA|2001-04-19|1215|5286|406|2001|4|4|19|2|2001|406|5286|Thursday|2001Q2|N|N|N|2452001|2452090|2451654|2451929|N|N|N|N|N| +2452020|AAAAAAAAEDKGFCAA|2001-04-20|1215|5286|406|2001|5|4|20|2|2001|406|5286|Friday|2001Q2|N|Y|N|2452001|2452090|2451655|2451930|N|N|N|N|N| +2452021|AAAAAAAAFDKGFCAA|2001-04-21|1215|5286|406|2001|6|4|21|2|2001|406|5286|Saturday|2001Q2|N|Y|N|2452001|2452090|2451656|2451931|N|N|N|N|N| +2452022|AAAAAAAAGDKGFCAA|2001-04-22|1215|5286|406|2001|0|4|22|2|2001|406|5286|Sunday|2001Q2|N|N|N|2452001|2452090|2451657|2451932|N|N|N|N|N| +2452023|AAAAAAAAHDKGFCAA|2001-04-23|1215|5286|406|2001|1|4|23|2|2001|406|5286|Monday|2001Q2|N|N|N|2452001|2452090|2451658|2451933|N|N|N|N|N| +2452024|AAAAAAAAIDKGFCAA|2001-04-24|1215|5287|406|2001|2|4|24|2|2001|406|5287|Tuesday|2001Q2|N|N|N|2452001|2452090|2451659|2451934|N|N|N|N|N| +2452025|AAAAAAAAJDKGFCAA|2001-04-25|1215|5287|406|2001|3|4|25|2|2001|406|5287|Wednesday|2001Q2|N|N|N|2452001|2452090|2451660|2451935|N|N|N|N|N| +2452026|AAAAAAAAKDKGFCAA|2001-04-26|1215|5287|406|2001|4|4|26|2|2001|406|5287|Thursday|2001Q2|N|N|N|2452001|2452090|2451661|2451936|N|N|N|N|N| +2452027|AAAAAAAALDKGFCAA|2001-04-27|1215|5287|406|2001|5|4|27|2|2001|406|5287|Friday|2001Q2|N|Y|N|2452001|2452090|2451662|2451937|N|N|N|N|N| +2452028|AAAAAAAAMDKGFCAA|2001-04-28|1215|5287|406|2001|6|4|28|2|2001|406|5287|Saturday|2001Q2|N|Y|N|2452001|2452090|2451663|2451938|N|N|N|N|N| +2452029|AAAAAAAANDKGFCAA|2001-04-29|1215|5287|406|2001|0|4|29|2|2001|406|5287|Sunday|2001Q2|N|N|N|2452001|2452090|2451664|2451939|N|N|N|N|N| +2452030|AAAAAAAAODKGFCAA|2001-04-30|1215|5287|406|2001|1|4|30|2|2001|406|5287|Monday|2001Q2|N|N|N|2452001|2452090|2451665|2451940|N|N|N|N|N| +2452031|AAAAAAAAPDKGFCAA|2001-05-01|1216|5288|406|2001|2|5|1|2|2001|406|5288|Tuesday|2001Q2|N|N|N|2452031|2452150|2451666|2451941|N|N|N|N|N| +2452032|AAAAAAAAAEKGFCAA|2001-05-02|1216|5288|406|2001|3|5|2|2|2001|406|5288|Wednesday|2001Q2|N|N|N|2452031|2452150|2451667|2451942|N|N|N|N|N| +2452033|AAAAAAAABEKGFCAA|2001-05-03|1216|5288|406|2001|4|5|3|2|2001|406|5288|Thursday|2001Q2|N|N|N|2452031|2452150|2451668|2451943|N|N|N|N|N| +2452034|AAAAAAAACEKGFCAA|2001-05-04|1216|5288|406|2001|5|5|4|2|2001|406|5288|Friday|2001Q2|N|Y|N|2452031|2452150|2451669|2451944|N|N|N|N|N| +2452035|AAAAAAAADEKGFCAA|2001-05-05|1216|5288|406|2001|6|5|5|2|2001|406|5288|Saturday|2001Q2|N|Y|N|2452031|2452150|2451670|2451945|N|N|N|N|N| +2452036|AAAAAAAAEEKGFCAA|2001-05-06|1216|5288|406|2001|0|5|6|2|2001|406|5288|Sunday|2001Q2|N|N|N|2452031|2452150|2451671|2451946|N|N|N|N|N| +2452037|AAAAAAAAFEKGFCAA|2001-05-07|1216|5288|406|2001|1|5|7|2|2001|406|5288|Monday|2001Q2|N|N|N|2452031|2452150|2451672|2451947|N|N|N|N|N| +2452038|AAAAAAAAGEKGFCAA|2001-05-08|1216|5289|406|2001|2|5|8|2|2001|406|5289|Tuesday|2001Q2|N|N|N|2452031|2452150|2451673|2451948|N|N|N|N|N| +2452039|AAAAAAAAHEKGFCAA|2001-05-09|1216|5289|406|2001|3|5|9|2|2001|406|5289|Wednesday|2001Q2|N|N|N|2452031|2452150|2451674|2451949|N|N|N|N|N| +2452040|AAAAAAAAIEKGFCAA|2001-05-10|1216|5289|406|2001|4|5|10|2|2001|406|5289|Thursday|2001Q2|N|N|N|2452031|2452150|2451675|2451950|N|N|N|N|N| +2452041|AAAAAAAAJEKGFCAA|2001-05-11|1216|5289|406|2001|5|5|11|2|2001|406|5289|Friday|2001Q2|N|Y|N|2452031|2452150|2451676|2451951|N|N|N|N|N| +2452042|AAAAAAAAKEKGFCAA|2001-05-12|1216|5289|406|2001|6|5|12|2|2001|406|5289|Saturday|2001Q2|N|Y|N|2452031|2452150|2451677|2451952|N|N|N|N|N| +2452043|AAAAAAAALEKGFCAA|2001-05-13|1216|5289|406|2001|0|5|13|2|2001|406|5289|Sunday|2001Q2|N|N|N|2452031|2452150|2451678|2451953|N|N|N|N|N| +2452044|AAAAAAAAMEKGFCAA|2001-05-14|1216|5289|406|2001|1|5|14|2|2001|406|5289|Monday|2001Q2|N|N|N|2452031|2452150|2451679|2451954|N|N|N|N|N| +2452045|AAAAAAAANEKGFCAA|2001-05-15|1216|5290|406|2001|2|5|15|2|2001|406|5290|Tuesday|2001Q2|N|N|N|2452031|2452150|2451680|2451955|N|N|N|N|N| +2452046|AAAAAAAAOEKGFCAA|2001-05-16|1216|5290|406|2001|3|5|16|2|2001|406|5290|Wednesday|2001Q2|N|N|N|2452031|2452150|2451681|2451956|N|N|N|N|N| +2452047|AAAAAAAAPEKGFCAA|2001-05-17|1216|5290|406|2001|4|5|17|2|2001|406|5290|Thursday|2001Q2|N|N|N|2452031|2452150|2451682|2451957|N|N|N|N|N| +2452048|AAAAAAAAAFKGFCAA|2001-05-18|1216|5290|406|2001|5|5|18|2|2001|406|5290|Friday|2001Q2|N|Y|N|2452031|2452150|2451683|2451958|N|N|N|N|N| +2452049|AAAAAAAABFKGFCAA|2001-05-19|1216|5290|406|2001|6|5|19|2|2001|406|5290|Saturday|2001Q2|N|Y|N|2452031|2452150|2451684|2451959|N|N|N|N|N| +2452050|AAAAAAAACFKGFCAA|2001-05-20|1216|5290|406|2001|0|5|20|2|2001|406|5290|Sunday|2001Q2|N|N|N|2452031|2452150|2451685|2451960|N|N|N|N|N| +2452051|AAAAAAAADFKGFCAA|2001-05-21|1216|5290|406|2001|1|5|21|2|2001|406|5290|Monday|2001Q2|N|N|N|2452031|2452150|2451686|2451961|N|N|N|N|N| +2452052|AAAAAAAAEFKGFCAA|2001-05-22|1216|5291|406|2001|2|5|22|2|2001|406|5291|Tuesday|2001Q2|N|N|N|2452031|2452150|2451687|2451962|N|N|N|N|N| +2452053|AAAAAAAAFFKGFCAA|2001-05-23|1216|5291|406|2001|3|5|23|2|2001|406|5291|Wednesday|2001Q2|N|N|N|2452031|2452150|2451688|2451963|N|N|N|N|N| +2452054|AAAAAAAAGFKGFCAA|2001-05-24|1216|5291|406|2001|4|5|24|2|2001|406|5291|Thursday|2001Q2|N|N|N|2452031|2452150|2451689|2451964|N|N|N|N|N| +2452055|AAAAAAAAHFKGFCAA|2001-05-25|1216|5291|406|2001|5|5|25|2|2001|406|5291|Friday|2001Q2|N|Y|N|2452031|2452150|2451690|2451965|N|N|N|N|N| +2452056|AAAAAAAAIFKGFCAA|2001-05-26|1216|5291|406|2001|6|5|26|2|2001|406|5291|Saturday|2001Q2|N|Y|N|2452031|2452150|2451691|2451966|N|N|N|N|N| +2452057|AAAAAAAAJFKGFCAA|2001-05-27|1216|5291|406|2001|0|5|27|2|2001|406|5291|Sunday|2001Q2|N|N|N|2452031|2452150|2451692|2451967|N|N|N|N|N| +2452058|AAAAAAAAKFKGFCAA|2001-05-28|1216|5291|406|2001|1|5|28|2|2001|406|5291|Monday|2001Q2|N|N|N|2452031|2452150|2451693|2451968|N|N|N|N|N| +2452059|AAAAAAAALFKGFCAA|2001-05-29|1216|5292|406|2001|2|5|29|2|2001|406|5292|Tuesday|2001Q2|N|N|N|2452031|2452150|2451694|2451969|N|N|N|N|N| +2452060|AAAAAAAAMFKGFCAA|2001-05-30|1216|5292|406|2001|3|5|30|2|2001|406|5292|Wednesday|2001Q2|N|N|N|2452031|2452150|2451695|2451970|N|N|N|N|N| +2452061|AAAAAAAANFKGFCAA|2001-05-31|1216|5292|406|2001|4|5|31|2|2001|406|5292|Thursday|2001Q2|N|N|N|2452031|2452150|2451696|2451971|N|N|N|N|N| +2452062|AAAAAAAAOFKGFCAA|2001-06-01|1217|5292|407|2001|5|6|1|2|2001|407|5292|Friday|2001Q2|N|Y|N|2452062|2452212|2451697|2451972|N|N|N|N|N| +2452063|AAAAAAAAPFKGFCAA|2001-06-02|1217|5292|407|2001|6|6|2|2|2001|407|5292|Saturday|2001Q2|N|Y|N|2452062|2452212|2451698|2451973|N|N|N|N|N| +2452064|AAAAAAAAAGKGFCAA|2001-06-03|1217|5292|407|2001|0|6|3|2|2001|407|5292|Sunday|2001Q2|N|N|N|2452062|2452212|2451699|2451974|N|N|N|N|N| +2452065|AAAAAAAABGKGFCAA|2001-06-04|1217|5292|407|2001|1|6|4|2|2001|407|5292|Monday|2001Q2|N|N|N|2452062|2452212|2451700|2451975|N|N|N|N|N| +2452066|AAAAAAAACGKGFCAA|2001-06-05|1217|5293|407|2001|2|6|5|2|2001|407|5293|Tuesday|2001Q2|N|N|N|2452062|2452212|2451701|2451976|N|N|N|N|N| +2452067|AAAAAAAADGKGFCAA|2001-06-06|1217|5293|407|2001|3|6|6|2|2001|407|5293|Wednesday|2001Q2|N|N|N|2452062|2452212|2451702|2451977|N|N|N|N|N| +2452068|AAAAAAAAEGKGFCAA|2001-06-07|1217|5293|407|2001|4|6|7|2|2001|407|5293|Thursday|2001Q2|N|N|N|2452062|2452212|2451703|2451978|N|N|N|N|N| +2452069|AAAAAAAAFGKGFCAA|2001-06-08|1217|5293|407|2001|5|6|8|2|2001|407|5293|Friday|2001Q2|N|Y|N|2452062|2452212|2451704|2451979|N|N|N|N|N| +2452070|AAAAAAAAGGKGFCAA|2001-06-09|1217|5293|407|2001|6|6|9|2|2001|407|5293|Saturday|2001Q2|N|Y|N|2452062|2452212|2451705|2451980|N|N|N|N|N| +2452071|AAAAAAAAHGKGFCAA|2001-06-10|1217|5293|407|2001|0|6|10|2|2001|407|5293|Sunday|2001Q2|N|N|N|2452062|2452212|2451706|2451981|N|N|N|N|N| +2452072|AAAAAAAAIGKGFCAA|2001-06-11|1217|5293|407|2001|1|6|11|2|2001|407|5293|Monday|2001Q2|N|N|N|2452062|2452212|2451707|2451982|N|N|N|N|N| +2452073|AAAAAAAAJGKGFCAA|2001-06-12|1217|5294|407|2001|2|6|12|2|2001|407|5294|Tuesday|2001Q2|N|N|N|2452062|2452212|2451708|2451983|N|N|N|N|N| +2452074|AAAAAAAAKGKGFCAA|2001-06-13|1217|5294|407|2001|3|6|13|2|2001|407|5294|Wednesday|2001Q2|N|N|N|2452062|2452212|2451709|2451984|N|N|N|N|N| +2452075|AAAAAAAALGKGFCAA|2001-06-14|1217|5294|407|2001|4|6|14|2|2001|407|5294|Thursday|2001Q2|N|N|N|2452062|2452212|2451710|2451985|N|N|N|N|N| +2452076|AAAAAAAAMGKGFCAA|2001-06-15|1217|5294|407|2001|5|6|15|2|2001|407|5294|Friday|2001Q2|N|Y|N|2452062|2452212|2451711|2451986|N|N|N|N|N| +2452077|AAAAAAAANGKGFCAA|2001-06-16|1217|5294|407|2001|6|6|16|2|2001|407|5294|Saturday|2001Q2|N|Y|N|2452062|2452212|2451712|2451987|N|N|N|N|N| +2452078|AAAAAAAAOGKGFCAA|2001-06-17|1217|5294|407|2001|0|6|17|2|2001|407|5294|Sunday|2001Q2|N|N|N|2452062|2452212|2451713|2451988|N|N|N|N|N| +2452079|AAAAAAAAPGKGFCAA|2001-06-18|1217|5294|407|2001|1|6|18|2|2001|407|5294|Monday|2001Q2|N|N|N|2452062|2452212|2451714|2451989|N|N|N|N|N| +2452080|AAAAAAAAAHKGFCAA|2001-06-19|1217|5295|407|2001|2|6|19|2|2001|407|5295|Tuesday|2001Q2|N|N|N|2452062|2452212|2451715|2451990|N|N|N|N|N| +2452081|AAAAAAAABHKGFCAA|2001-06-20|1217|5295|407|2001|3|6|20|2|2001|407|5295|Wednesday|2001Q2|N|N|N|2452062|2452212|2451716|2451991|N|N|N|N|N| +2452082|AAAAAAAACHKGFCAA|2001-06-21|1217|5295|407|2001|4|6|21|2|2001|407|5295|Thursday|2001Q2|N|N|N|2452062|2452212|2451717|2451992|N|N|N|N|N| +2452083|AAAAAAAADHKGFCAA|2001-06-22|1217|5295|407|2001|5|6|22|2|2001|407|5295|Friday|2001Q2|N|Y|N|2452062|2452212|2451718|2451993|N|N|N|N|N| +2452084|AAAAAAAAEHKGFCAA|2001-06-23|1217|5295|407|2001|6|6|23|2|2001|407|5295|Saturday|2001Q2|N|Y|N|2452062|2452212|2451719|2451994|N|N|N|N|N| +2452085|AAAAAAAAFHKGFCAA|2001-06-24|1217|5295|407|2001|0|6|24|2|2001|407|5295|Sunday|2001Q2|N|N|N|2452062|2452212|2451720|2451995|N|N|N|N|N| +2452086|AAAAAAAAGHKGFCAA|2001-06-25|1217|5295|407|2001|1|6|25|2|2001|407|5295|Monday|2001Q2|N|N|N|2452062|2452212|2451721|2451996|N|N|N|N|N| +2452087|AAAAAAAAHHKGFCAA|2001-06-26|1217|5296|407|2001|2|6|26|2|2001|407|5296|Tuesday|2001Q2|N|N|N|2452062|2452212|2451722|2451997|N|N|N|N|N| +2452088|AAAAAAAAIHKGFCAA|2001-06-27|1217|5296|407|2001|3|6|27|2|2001|407|5296|Wednesday|2001Q2|N|N|N|2452062|2452212|2451723|2451998|N|N|N|N|N| +2452089|AAAAAAAAJHKGFCAA|2001-06-28|1217|5296|407|2001|4|6|28|2|2001|407|5296|Thursday|2001Q2|N|N|N|2452062|2452212|2451724|2451999|N|N|N|N|N| +2452090|AAAAAAAAKHKGFCAA|2001-06-29|1217|5296|407|2001|5|6|29|2|2001|407|5296|Friday|2001Q2|N|Y|N|2452062|2452212|2451725|2452000|N|N|N|N|N| +2452091|AAAAAAAALHKGFCAA|2001-06-30|1217|5296|407|2001|6|6|30|2|2001|407|5296|Saturday|2001Q2|N|Y|N|2452062|2452212|2451726|2452001|N|N|N|N|N| +2452092|AAAAAAAAMHKGFCAA|2001-07-01|1218|5296|407|2001|0|7|1|2|2001|407|5296|Sunday|2001Q2|N|N|N|2452092|2452272|2451727|2452001|N|N|N|N|N| +2452093|AAAAAAAANHKGFCAA|2001-07-02|1218|5296|407|2001|1|7|2|3|2001|407|5296|Monday|2001Q3|N|N|N|2452092|2452272|2451728|2452002|N|N|N|N|N| +2452094|AAAAAAAAOHKGFCAA|2001-07-03|1218|5297|407|2001|2|7|3|3|2001|407|5297|Tuesday|2001Q3|N|N|N|2452092|2452272|2451729|2452003|N|N|N|N|N| +2452095|AAAAAAAAPHKGFCAA|2001-07-04|1218|5297|407|2001|3|7|4|3|2001|407|5297|Wednesday|2001Q3|N|N|N|2452092|2452272|2451730|2452004|N|N|N|N|N| +2452096|AAAAAAAAAIKGFCAA|2001-07-05|1218|5297|407|2001|4|7|5|3|2001|407|5297|Thursday|2001Q3|Y|N|N|2452092|2452272|2451731|2452005|N|N|N|N|N| +2452097|AAAAAAAABIKGFCAA|2001-07-06|1218|5297|407|2001|5|7|6|3|2001|407|5297|Friday|2001Q3|N|Y|Y|2452092|2452272|2451732|2452006|N|N|N|N|N| +2452098|AAAAAAAACIKGFCAA|2001-07-07|1218|5297|407|2001|6|7|7|3|2001|407|5297|Saturday|2001Q3|N|Y|N|2452092|2452272|2451733|2452007|N|N|N|N|N| +2452099|AAAAAAAADIKGFCAA|2001-07-08|1218|5297|407|2001|0|7|8|3|2001|407|5297|Sunday|2001Q3|N|N|N|2452092|2452272|2451734|2452008|N|N|N|N|N| +2452100|AAAAAAAAEIKGFCAA|2001-07-09|1218|5297|407|2001|1|7|9|3|2001|407|5297|Monday|2001Q3|N|N|N|2452092|2452272|2451735|2452009|N|N|N|N|N| +2452101|AAAAAAAAFIKGFCAA|2001-07-10|1218|5298|407|2001|2|7|10|3|2001|407|5298|Tuesday|2001Q3|N|N|N|2452092|2452272|2451736|2452010|N|N|N|N|N| +2452102|AAAAAAAAGIKGFCAA|2001-07-11|1218|5298|407|2001|3|7|11|3|2001|407|5298|Wednesday|2001Q3|N|N|N|2452092|2452272|2451737|2452011|N|N|N|N|N| +2452103|AAAAAAAAHIKGFCAA|2001-07-12|1218|5298|407|2001|4|7|12|3|2001|407|5298|Thursday|2001Q3|N|N|N|2452092|2452272|2451738|2452012|N|N|N|N|N| +2452104|AAAAAAAAIIKGFCAA|2001-07-13|1218|5298|407|2001|5|7|13|3|2001|407|5298|Friday|2001Q3|N|Y|N|2452092|2452272|2451739|2452013|N|N|N|N|N| +2452105|AAAAAAAAJIKGFCAA|2001-07-14|1218|5298|407|2001|6|7|14|3|2001|407|5298|Saturday|2001Q3|N|Y|N|2452092|2452272|2451740|2452014|N|N|N|N|N| +2452106|AAAAAAAAKIKGFCAA|2001-07-15|1218|5298|407|2001|0|7|15|3|2001|407|5298|Sunday|2001Q3|N|N|N|2452092|2452272|2451741|2452015|N|N|N|N|N| +2452107|AAAAAAAALIKGFCAA|2001-07-16|1218|5298|407|2001|1|7|16|3|2001|407|5298|Monday|2001Q3|N|N|N|2452092|2452272|2451742|2452016|N|N|N|N|N| +2452108|AAAAAAAAMIKGFCAA|2001-07-17|1218|5299|407|2001|2|7|17|3|2001|407|5299|Tuesday|2001Q3|N|N|N|2452092|2452272|2451743|2452017|N|N|N|N|N| +2452109|AAAAAAAANIKGFCAA|2001-07-18|1218|5299|407|2001|3|7|18|3|2001|407|5299|Wednesday|2001Q3|N|N|N|2452092|2452272|2451744|2452018|N|N|N|N|N| +2452110|AAAAAAAAOIKGFCAA|2001-07-19|1218|5299|407|2001|4|7|19|3|2001|407|5299|Thursday|2001Q3|N|N|N|2452092|2452272|2451745|2452019|N|N|N|N|N| +2452111|AAAAAAAAPIKGFCAA|2001-07-20|1218|5299|407|2001|5|7|20|3|2001|407|5299|Friday|2001Q3|N|Y|N|2452092|2452272|2451746|2452020|N|N|N|N|N| +2452112|AAAAAAAAAJKGFCAA|2001-07-21|1218|5299|407|2001|6|7|21|3|2001|407|5299|Saturday|2001Q3|N|Y|N|2452092|2452272|2451747|2452021|N|N|N|N|N| +2452113|AAAAAAAABJKGFCAA|2001-07-22|1218|5299|407|2001|0|7|22|3|2001|407|5299|Sunday|2001Q3|N|N|N|2452092|2452272|2451748|2452022|N|N|N|N|N| +2452114|AAAAAAAACJKGFCAA|2001-07-23|1218|5299|407|2001|1|7|23|3|2001|407|5299|Monday|2001Q3|N|N|N|2452092|2452272|2451749|2452023|N|N|N|N|N| +2452115|AAAAAAAADJKGFCAA|2001-07-24|1218|5300|407|2001|2|7|24|3|2001|407|5300|Tuesday|2001Q3|N|N|N|2452092|2452272|2451750|2452024|N|N|N|N|N| +2452116|AAAAAAAAEJKGFCAA|2001-07-25|1218|5300|407|2001|3|7|25|3|2001|407|5300|Wednesday|2001Q3|N|N|N|2452092|2452272|2451751|2452025|N|N|N|N|N| +2452117|AAAAAAAAFJKGFCAA|2001-07-26|1218|5300|407|2001|4|7|26|3|2001|407|5300|Thursday|2001Q3|N|N|N|2452092|2452272|2451752|2452026|N|N|N|N|N| +2452118|AAAAAAAAGJKGFCAA|2001-07-27|1218|5300|407|2001|5|7|27|3|2001|407|5300|Friday|2001Q3|N|Y|N|2452092|2452272|2451753|2452027|N|N|N|N|N| +2452119|AAAAAAAAHJKGFCAA|2001-07-28|1218|5300|407|2001|6|7|28|3|2001|407|5300|Saturday|2001Q3|N|Y|N|2452092|2452272|2451754|2452028|N|N|N|N|N| +2452120|AAAAAAAAIJKGFCAA|2001-07-29|1218|5300|407|2001|0|7|29|3|2001|407|5300|Sunday|2001Q3|N|N|N|2452092|2452272|2451755|2452029|N|N|N|N|N| +2452121|AAAAAAAAJJKGFCAA|2001-07-30|1218|5300|407|2001|1|7|30|3|2001|407|5300|Monday|2001Q3|N|N|N|2452092|2452272|2451756|2452030|N|N|N|N|N| +2452122|AAAAAAAAKJKGFCAA|2001-07-31|1218|5301|407|2001|2|7|31|3|2001|407|5301|Tuesday|2001Q3|N|N|N|2452092|2452272|2451757|2452031|N|N|N|N|N| +2452123|AAAAAAAALJKGFCAA|2001-08-01|1219|5301|407|2001|3|8|1|3|2001|407|5301|Wednesday|2001Q3|N|N|N|2452123|2452334|2451758|2452032|N|N|N|N|N| +2452124|AAAAAAAAMJKGFCAA|2001-08-02|1219|5301|407|2001|4|8|2|3|2001|407|5301|Thursday|2001Q3|N|N|N|2452123|2452334|2451759|2452033|N|N|N|N|N| +2452125|AAAAAAAANJKGFCAA|2001-08-03|1219|5301|407|2001|5|8|3|3|2001|407|5301|Friday|2001Q3|N|Y|N|2452123|2452334|2451760|2452034|N|N|N|N|N| +2452126|AAAAAAAAOJKGFCAA|2001-08-04|1219|5301|407|2001|6|8|4|3|2001|407|5301|Saturday|2001Q3|N|Y|N|2452123|2452334|2451761|2452035|N|N|N|N|N| +2452127|AAAAAAAAPJKGFCAA|2001-08-05|1219|5301|407|2001|0|8|5|3|2001|407|5301|Sunday|2001Q3|N|N|N|2452123|2452334|2451762|2452036|N|N|N|N|N| +2452128|AAAAAAAAAKKGFCAA|2001-08-06|1219|5301|407|2001|1|8|6|3|2001|407|5301|Monday|2001Q3|N|N|N|2452123|2452334|2451763|2452037|N|N|N|N|N| +2452129|AAAAAAAABKKGFCAA|2001-08-07|1219|5302|407|2001|2|8|7|3|2001|407|5302|Tuesday|2001Q3|N|N|N|2452123|2452334|2451764|2452038|N|N|N|N|N| +2452130|AAAAAAAACKKGFCAA|2001-08-08|1219|5302|407|2001|3|8|8|3|2001|407|5302|Wednesday|2001Q3|N|N|N|2452123|2452334|2451765|2452039|N|N|N|N|N| +2452131|AAAAAAAADKKGFCAA|2001-08-09|1219|5302|407|2001|4|8|9|3|2001|407|5302|Thursday|2001Q3|N|N|N|2452123|2452334|2451766|2452040|N|N|N|N|N| +2452132|AAAAAAAAEKKGFCAA|2001-08-10|1219|5302|407|2001|5|8|10|3|2001|407|5302|Friday|2001Q3|N|Y|N|2452123|2452334|2451767|2452041|N|N|N|N|N| +2452133|AAAAAAAAFKKGFCAA|2001-08-11|1219|5302|407|2001|6|8|11|3|2001|407|5302|Saturday|2001Q3|N|Y|N|2452123|2452334|2451768|2452042|N|N|N|N|N| +2452134|AAAAAAAAGKKGFCAA|2001-08-12|1219|5302|407|2001|0|8|12|3|2001|407|5302|Sunday|2001Q3|N|N|N|2452123|2452334|2451769|2452043|N|N|N|N|N| +2452135|AAAAAAAAHKKGFCAA|2001-08-13|1219|5302|407|2001|1|8|13|3|2001|407|5302|Monday|2001Q3|N|N|N|2452123|2452334|2451770|2452044|N|N|N|N|N| +2452136|AAAAAAAAIKKGFCAA|2001-08-14|1219|5303|407|2001|2|8|14|3|2001|407|5303|Tuesday|2001Q3|N|N|N|2452123|2452334|2451771|2452045|N|N|N|N|N| +2452137|AAAAAAAAJKKGFCAA|2001-08-15|1219|5303|407|2001|3|8|15|3|2001|407|5303|Wednesday|2001Q3|N|N|N|2452123|2452334|2451772|2452046|N|N|N|N|N| +2452138|AAAAAAAAKKKGFCAA|2001-08-16|1219|5303|407|2001|4|8|16|3|2001|407|5303|Thursday|2001Q3|N|N|N|2452123|2452334|2451773|2452047|N|N|N|N|N| +2452139|AAAAAAAALKKGFCAA|2001-08-17|1219|5303|407|2001|5|8|17|3|2001|407|5303|Friday|2001Q3|N|Y|N|2452123|2452334|2451774|2452048|N|N|N|N|N| +2452140|AAAAAAAAMKKGFCAA|2001-08-18|1219|5303|407|2001|6|8|18|3|2001|407|5303|Saturday|2001Q3|N|Y|N|2452123|2452334|2451775|2452049|N|N|N|N|N| +2452141|AAAAAAAANKKGFCAA|2001-08-19|1219|5303|407|2001|0|8|19|3|2001|407|5303|Sunday|2001Q3|N|N|N|2452123|2452334|2451776|2452050|N|N|N|N|N| +2452142|AAAAAAAAOKKGFCAA|2001-08-20|1219|5303|407|2001|1|8|20|3|2001|407|5303|Monday|2001Q3|N|N|N|2452123|2452334|2451777|2452051|N|N|N|N|N| +2452143|AAAAAAAAPKKGFCAA|2001-08-21|1219|5304|407|2001|2|8|21|3|2001|407|5304|Tuesday|2001Q3|N|N|N|2452123|2452334|2451778|2452052|N|N|N|N|N| +2452144|AAAAAAAAALKGFCAA|2001-08-22|1219|5304|407|2001|3|8|22|3|2001|407|5304|Wednesday|2001Q3|N|N|N|2452123|2452334|2451779|2452053|N|N|N|N|N| +2452145|AAAAAAAABLKGFCAA|2001-08-23|1219|5304|407|2001|4|8|23|3|2001|407|5304|Thursday|2001Q3|N|N|N|2452123|2452334|2451780|2452054|N|N|N|N|N| +2452146|AAAAAAAACLKGFCAA|2001-08-24|1219|5304|407|2001|5|8|24|3|2001|407|5304|Friday|2001Q3|N|Y|N|2452123|2452334|2451781|2452055|N|N|N|N|N| +2452147|AAAAAAAADLKGFCAA|2001-08-25|1219|5304|407|2001|6|8|25|3|2001|407|5304|Saturday|2001Q3|N|Y|N|2452123|2452334|2451782|2452056|N|N|N|N|N| +2452148|AAAAAAAAELKGFCAA|2001-08-26|1219|5304|407|2001|0|8|26|3|2001|407|5304|Sunday|2001Q3|N|N|N|2452123|2452334|2451783|2452057|N|N|N|N|N| +2452149|AAAAAAAAFLKGFCAA|2001-08-27|1219|5304|407|2001|1|8|27|3|2001|407|5304|Monday|2001Q3|N|N|N|2452123|2452334|2451784|2452058|N|N|N|N|N| +2452150|AAAAAAAAGLKGFCAA|2001-08-28|1219|5305|407|2001|2|8|28|3|2001|407|5305|Tuesday|2001Q3|N|N|N|2452123|2452334|2451785|2452059|N|N|N|N|N| +2452151|AAAAAAAAHLKGFCAA|2001-08-29|1219|5305|407|2001|3|8|29|3|2001|407|5305|Wednesday|2001Q3|N|N|N|2452123|2452334|2451786|2452060|N|N|N|N|N| +2452152|AAAAAAAAILKGFCAA|2001-08-30|1219|5305|407|2001|4|8|30|3|2001|407|5305|Thursday|2001Q3|N|N|N|2452123|2452334|2451787|2452061|N|N|N|N|N| +2452153|AAAAAAAAJLKGFCAA|2001-08-31|1219|5305|407|2001|5|8|31|3|2001|407|5305|Friday|2001Q3|N|Y|N|2452123|2452334|2451788|2452062|N|N|N|N|N| +2452154|AAAAAAAAKLKGFCAA|2001-09-01|1220|5305|408|2001|6|9|1|3|2001|408|5305|Saturday|2001Q3|N|Y|N|2452154|2452396|2451789|2452063|N|N|N|N|N| +2452155|AAAAAAAALLKGFCAA|2001-09-02|1220|5305|408|2001|0|9|2|3|2001|408|5305|Sunday|2001Q3|N|N|N|2452154|2452396|2451790|2452064|N|N|N|N|N| +2452156|AAAAAAAAMLKGFCAA|2001-09-03|1220|5305|408|2001|1|9|3|3|2001|408|5305|Monday|2001Q3|N|N|N|2452154|2452396|2451791|2452065|N|N|N|N|N| +2452157|AAAAAAAANLKGFCAA|2001-09-04|1220|5306|408|2001|2|9|4|3|2001|408|5306|Tuesday|2001Q3|N|N|N|2452154|2452396|2451792|2452066|N|N|N|N|N| +2452158|AAAAAAAAOLKGFCAA|2001-09-05|1220|5306|408|2001|3|9|5|3|2001|408|5306|Wednesday|2001Q3|N|N|N|2452154|2452396|2451793|2452067|N|N|N|N|N| +2452159|AAAAAAAAPLKGFCAA|2001-09-06|1220|5306|408|2001|4|9|6|3|2001|408|5306|Thursday|2001Q3|N|N|N|2452154|2452396|2451794|2452068|N|N|N|N|N| +2452160|AAAAAAAAAMKGFCAA|2001-09-07|1220|5306|408|2001|5|9|7|3|2001|408|5306|Friday|2001Q3|N|Y|N|2452154|2452396|2451795|2452069|N|N|N|N|N| +2452161|AAAAAAAABMKGFCAA|2001-09-08|1220|5306|408|2001|6|9|8|3|2001|408|5306|Saturday|2001Q3|N|Y|N|2452154|2452396|2451796|2452070|N|N|N|N|N| +2452162|AAAAAAAACMKGFCAA|2001-09-09|1220|5306|408|2001|0|9|9|3|2001|408|5306|Sunday|2001Q3|N|N|N|2452154|2452396|2451797|2452071|N|N|N|N|N| +2452163|AAAAAAAADMKGFCAA|2001-09-10|1220|5306|408|2001|1|9|10|3|2001|408|5306|Monday|2001Q3|N|N|N|2452154|2452396|2451798|2452072|N|N|N|N|N| +2452164|AAAAAAAAEMKGFCAA|2001-09-11|1220|5307|408|2001|2|9|11|3|2001|408|5307|Tuesday|2001Q3|N|N|N|2452154|2452396|2451799|2452073|N|N|N|N|N| +2452165|AAAAAAAAFMKGFCAA|2001-09-12|1220|5307|408|2001|3|9|12|3|2001|408|5307|Wednesday|2001Q3|N|N|N|2452154|2452396|2451800|2452074|N|N|N|N|N| +2452166|AAAAAAAAGMKGFCAA|2001-09-13|1220|5307|408|2001|4|9|13|3|2001|408|5307|Thursday|2001Q3|N|N|N|2452154|2452396|2451801|2452075|N|N|N|N|N| +2452167|AAAAAAAAHMKGFCAA|2001-09-14|1220|5307|408|2001|5|9|14|3|2001|408|5307|Friday|2001Q3|N|Y|N|2452154|2452396|2451802|2452076|N|N|N|N|N| +2452168|AAAAAAAAIMKGFCAA|2001-09-15|1220|5307|408|2001|6|9|15|3|2001|408|5307|Saturday|2001Q3|N|Y|N|2452154|2452396|2451803|2452077|N|N|N|N|N| +2452169|AAAAAAAAJMKGFCAA|2001-09-16|1220|5307|408|2001|0|9|16|3|2001|408|5307|Sunday|2001Q3|N|N|N|2452154|2452396|2451804|2452078|N|N|N|N|N| +2452170|AAAAAAAAKMKGFCAA|2001-09-17|1220|5307|408|2001|1|9|17|3|2001|408|5307|Monday|2001Q3|N|N|N|2452154|2452396|2451805|2452079|N|N|N|N|N| +2452171|AAAAAAAALMKGFCAA|2001-09-18|1220|5308|408|2001|2|9|18|3|2001|408|5308|Tuesday|2001Q3|N|N|N|2452154|2452396|2451806|2452080|N|N|N|N|N| +2452172|AAAAAAAAMMKGFCAA|2001-09-19|1220|5308|408|2001|3|9|19|3|2001|408|5308|Wednesday|2001Q3|N|N|N|2452154|2452396|2451807|2452081|N|N|N|N|N| +2452173|AAAAAAAANMKGFCAA|2001-09-20|1220|5308|408|2001|4|9|20|3|2001|408|5308|Thursday|2001Q3|N|N|N|2452154|2452396|2451808|2452082|N|N|N|N|N| +2452174|AAAAAAAAOMKGFCAA|2001-09-21|1220|5308|408|2001|5|9|21|3|2001|408|5308|Friday|2001Q3|N|Y|N|2452154|2452396|2451809|2452083|N|N|N|N|N| +2452175|AAAAAAAAPMKGFCAA|2001-09-22|1220|5308|408|2001|6|9|22|3|2001|408|5308|Saturday|2001Q3|N|Y|N|2452154|2452396|2451810|2452084|N|N|N|N|N| +2452176|AAAAAAAAANKGFCAA|2001-09-23|1220|5308|408|2001|0|9|23|3|2001|408|5308|Sunday|2001Q3|N|N|N|2452154|2452396|2451811|2452085|N|N|N|N|N| +2452177|AAAAAAAABNKGFCAA|2001-09-24|1220|5308|408|2001|1|9|24|3|2001|408|5308|Monday|2001Q3|N|N|N|2452154|2452396|2451812|2452086|N|N|N|N|N| +2452178|AAAAAAAACNKGFCAA|2001-09-25|1220|5309|408|2001|2|9|25|3|2001|408|5309|Tuesday|2001Q3|N|N|N|2452154|2452396|2451813|2452087|N|N|N|N|N| +2452179|AAAAAAAADNKGFCAA|2001-09-26|1220|5309|408|2001|3|9|26|3|2001|408|5309|Wednesday|2001Q3|N|N|N|2452154|2452396|2451814|2452088|N|N|N|N|N| +2452180|AAAAAAAAENKGFCAA|2001-09-27|1220|5309|408|2001|4|9|27|3|2001|408|5309|Thursday|2001Q3|N|N|N|2452154|2452396|2451815|2452089|N|N|N|N|N| +2452181|AAAAAAAAFNKGFCAA|2001-09-28|1220|5309|408|2001|5|9|28|3|2001|408|5309|Friday|2001Q3|N|Y|N|2452154|2452396|2451816|2452090|N|N|N|N|N| +2452182|AAAAAAAAGNKGFCAA|2001-09-29|1220|5309|408|2001|6|9|29|3|2001|408|5309|Saturday|2001Q3|N|Y|N|2452154|2452396|2451817|2452091|N|N|N|N|N| +2452183|AAAAAAAAHNKGFCAA|2001-09-30|1220|5309|408|2001|0|9|30|3|2001|408|5309|Sunday|2001Q3|N|N|N|2452154|2452396|2451818|2452092|N|N|N|N|N| +2452184|AAAAAAAAINKGFCAA|2001-10-01|1221|5309|408|2001|1|10|1|3|2001|408|5309|Monday|2001Q3|N|N|N|2452184|2452456|2451819|2452092|N|N|N|N|N| +2452185|AAAAAAAAJNKGFCAA|2001-10-02|1221|5310|408|2001|2|10|2|4|2001|408|5310|Tuesday|2001Q4|N|N|N|2452184|2452456|2451820|2452093|N|N|N|N|N| +2452186|AAAAAAAAKNKGFCAA|2001-10-03|1221|5310|408|2001|3|10|3|4|2001|408|5310|Wednesday|2001Q4|N|N|N|2452184|2452456|2451821|2452094|N|N|N|N|N| +2452187|AAAAAAAALNKGFCAA|2001-10-04|1221|5310|408|2001|4|10|4|4|2001|408|5310|Thursday|2001Q4|N|N|N|2452184|2452456|2451822|2452095|N|N|N|N|N| +2452188|AAAAAAAAMNKGFCAA|2001-10-05|1221|5310|408|2001|5|10|5|4|2001|408|5310|Friday|2001Q4|N|Y|N|2452184|2452456|2451823|2452096|N|N|N|N|N| +2452189|AAAAAAAANNKGFCAA|2001-10-06|1221|5310|408|2001|6|10|6|4|2001|408|5310|Saturday|2001Q4|N|Y|N|2452184|2452456|2451824|2452097|N|N|N|N|N| +2452190|AAAAAAAAONKGFCAA|2001-10-07|1221|5310|408|2001|0|10|7|4|2001|408|5310|Sunday|2001Q4|N|N|N|2452184|2452456|2451825|2452098|N|N|N|N|N| +2452191|AAAAAAAAPNKGFCAA|2001-10-08|1221|5310|408|2001|1|10|8|4|2001|408|5310|Monday|2001Q4|N|N|N|2452184|2452456|2451826|2452099|N|N|N|N|N| +2452192|AAAAAAAAAOKGFCAA|2001-10-09|1221|5311|408|2001|2|10|9|4|2001|408|5311|Tuesday|2001Q4|N|N|N|2452184|2452456|2451827|2452100|N|N|N|N|N| +2452193|AAAAAAAABOKGFCAA|2001-10-10|1221|5311|408|2001|3|10|10|4|2001|408|5311|Wednesday|2001Q4|N|N|N|2452184|2452456|2451828|2452101|N|N|N|N|N| +2452194|AAAAAAAACOKGFCAA|2001-10-11|1221|5311|408|2001|4|10|11|4|2001|408|5311|Thursday|2001Q4|N|N|N|2452184|2452456|2451829|2452102|N|N|N|N|N| +2452195|AAAAAAAADOKGFCAA|2001-10-12|1221|5311|408|2001|5|10|12|4|2001|408|5311|Friday|2001Q4|N|Y|N|2452184|2452456|2451830|2452103|N|N|N|N|N| +2452196|AAAAAAAAEOKGFCAA|2001-10-13|1221|5311|408|2001|6|10|13|4|2001|408|5311|Saturday|2001Q4|N|Y|N|2452184|2452456|2451831|2452104|N|N|N|N|N| +2452197|AAAAAAAAFOKGFCAA|2001-10-14|1221|5311|408|2001|0|10|14|4|2001|408|5311|Sunday|2001Q4|N|N|N|2452184|2452456|2451832|2452105|N|N|N|N|N| +2452198|AAAAAAAAGOKGFCAA|2001-10-15|1221|5311|408|2001|1|10|15|4|2001|408|5311|Monday|2001Q4|N|N|N|2452184|2452456|2451833|2452106|N|N|N|N|N| +2452199|AAAAAAAAHOKGFCAA|2001-10-16|1221|5312|408|2001|2|10|16|4|2001|408|5312|Tuesday|2001Q4|N|N|N|2452184|2452456|2451834|2452107|N|N|N|N|N| +2452200|AAAAAAAAIOKGFCAA|2001-10-17|1221|5312|408|2001|3|10|17|4|2001|408|5312|Wednesday|2001Q4|N|N|N|2452184|2452456|2451835|2452108|N|N|N|N|N| +2452201|AAAAAAAAJOKGFCAA|2001-10-18|1221|5312|408|2001|4|10|18|4|2001|408|5312|Thursday|2001Q4|N|N|N|2452184|2452456|2451836|2452109|N|N|N|N|N| +2452202|AAAAAAAAKOKGFCAA|2001-10-19|1221|5312|408|2001|5|10|19|4|2001|408|5312|Friday|2001Q4|N|Y|N|2452184|2452456|2451837|2452110|N|N|N|N|N| +2452203|AAAAAAAALOKGFCAA|2001-10-20|1221|5312|408|2001|6|10|20|4|2001|408|5312|Saturday|2001Q4|N|Y|N|2452184|2452456|2451838|2452111|N|N|N|N|N| +2452204|AAAAAAAAMOKGFCAA|2001-10-21|1221|5312|408|2001|0|10|21|4|2001|408|5312|Sunday|2001Q4|N|N|N|2452184|2452456|2451839|2452112|N|N|N|N|N| +2452205|AAAAAAAANOKGFCAA|2001-10-22|1221|5312|408|2001|1|10|22|4|2001|408|5312|Monday|2001Q4|N|N|N|2452184|2452456|2451840|2452113|N|N|N|N|N| +2452206|AAAAAAAAOOKGFCAA|2001-10-23|1221|5313|408|2001|2|10|23|4|2001|408|5313|Tuesday|2001Q4|N|N|N|2452184|2452456|2451841|2452114|N|N|N|N|N| +2452207|AAAAAAAAPOKGFCAA|2001-10-24|1221|5313|408|2001|3|10|24|4|2001|408|5313|Wednesday|2001Q4|N|N|N|2452184|2452456|2451842|2452115|N|N|N|N|N| +2452208|AAAAAAAAAPKGFCAA|2001-10-25|1221|5313|408|2001|4|10|25|4|2001|408|5313|Thursday|2001Q4|N|N|N|2452184|2452456|2451843|2452116|N|N|N|N|N| +2452209|AAAAAAAABPKGFCAA|2001-10-26|1221|5313|408|2001|5|10|26|4|2001|408|5313|Friday|2001Q4|N|Y|N|2452184|2452456|2451844|2452117|N|N|N|N|N| +2452210|AAAAAAAACPKGFCAA|2001-10-27|1221|5313|408|2001|6|10|27|4|2001|408|5313|Saturday|2001Q4|N|Y|N|2452184|2452456|2451845|2452118|N|N|N|N|N| +2452211|AAAAAAAADPKGFCAA|2001-10-28|1221|5313|408|2001|0|10|28|4|2001|408|5313|Sunday|2001Q4|N|N|N|2452184|2452456|2451846|2452119|N|N|N|N|N| +2452212|AAAAAAAAEPKGFCAA|2001-10-29|1221|5313|408|2001|1|10|29|4|2001|408|5313|Monday|2001Q4|N|N|N|2452184|2452456|2451847|2452120|N|N|N|N|N| +2452213|AAAAAAAAFPKGFCAA|2001-10-30|1221|5314|408|2001|2|10|30|4|2001|408|5314|Tuesday|2001Q4|N|N|N|2452184|2452456|2451848|2452121|N|N|N|N|N| +2452214|AAAAAAAAGPKGFCAA|2001-10-31|1221|5314|408|2001|3|10|31|4|2001|408|5314|Wednesday|2001Q4|N|N|N|2452184|2452456|2451849|2452122|N|N|N|N|N| +2452215|AAAAAAAAHPKGFCAA|2001-11-01|1222|5314|408|2001|4|11|1|4|2001|408|5314|Thursday|2001Q4|N|N|N|2452215|2452518|2451850|2452123|N|N|N|N|N| +2452216|AAAAAAAAIPKGFCAA|2001-11-02|1222|5314|408|2001|5|11|2|4|2001|408|5314|Friday|2001Q4|N|Y|N|2452215|2452518|2451851|2452124|N|N|N|N|N| +2452217|AAAAAAAAJPKGFCAA|2001-11-03|1222|5314|408|2001|6|11|3|4|2001|408|5314|Saturday|2001Q4|N|Y|N|2452215|2452518|2451852|2452125|N|N|N|N|N| +2452218|AAAAAAAAKPKGFCAA|2001-11-04|1222|5314|408|2001|0|11|4|4|2001|408|5314|Sunday|2001Q4|N|N|N|2452215|2452518|2451853|2452126|N|N|N|N|N| +2452219|AAAAAAAALPKGFCAA|2001-11-05|1222|5314|408|2001|1|11|5|4|2001|408|5314|Monday|2001Q4|N|N|N|2452215|2452518|2451854|2452127|N|N|N|N|N| +2452220|AAAAAAAAMPKGFCAA|2001-11-06|1222|5315|408|2001|2|11|6|4|2001|408|5315|Tuesday|2001Q4|N|N|N|2452215|2452518|2451855|2452128|N|N|N|N|N| +2452221|AAAAAAAANPKGFCAA|2001-11-07|1222|5315|408|2001|3|11|7|4|2001|408|5315|Wednesday|2001Q4|N|N|N|2452215|2452518|2451856|2452129|N|N|N|N|N| +2452222|AAAAAAAAOPKGFCAA|2001-11-08|1222|5315|408|2001|4|11|8|4|2001|408|5315|Thursday|2001Q4|N|N|N|2452215|2452518|2451857|2452130|N|N|N|N|N| +2452223|AAAAAAAAPPKGFCAA|2001-11-09|1222|5315|408|2001|5|11|9|4|2001|408|5315|Friday|2001Q4|N|Y|N|2452215|2452518|2451858|2452131|N|N|N|N|N| +2452224|AAAAAAAAAALGFCAA|2001-11-10|1222|5315|408|2001|6|11|10|4|2001|408|5315|Saturday|2001Q4|N|Y|N|2452215|2452518|2451859|2452132|N|N|N|N|N| +2452225|AAAAAAAABALGFCAA|2001-11-11|1222|5315|408|2001|0|11|11|4|2001|408|5315|Sunday|2001Q4|N|N|N|2452215|2452518|2451860|2452133|N|N|N|N|N| +2452226|AAAAAAAACALGFCAA|2001-11-12|1222|5315|408|2001|1|11|12|4|2001|408|5315|Monday|2001Q4|N|N|N|2452215|2452518|2451861|2452134|N|N|N|N|N| +2452227|AAAAAAAADALGFCAA|2001-11-13|1222|5316|408|2001|2|11|13|4|2001|408|5316|Tuesday|2001Q4|N|N|N|2452215|2452518|2451862|2452135|N|N|N|N|N| +2452228|AAAAAAAAEALGFCAA|2001-11-14|1222|5316|408|2001|3|11|14|4|2001|408|5316|Wednesday|2001Q4|N|N|N|2452215|2452518|2451863|2452136|N|N|N|N|N| +2452229|AAAAAAAAFALGFCAA|2001-11-15|1222|5316|408|2001|4|11|15|4|2001|408|5316|Thursday|2001Q4|N|N|N|2452215|2452518|2451864|2452137|N|N|N|N|N| +2452230|AAAAAAAAGALGFCAA|2001-11-16|1222|5316|408|2001|5|11|16|4|2001|408|5316|Friday|2001Q4|N|Y|N|2452215|2452518|2451865|2452138|N|N|N|N|N| +2452231|AAAAAAAAHALGFCAA|2001-11-17|1222|5316|408|2001|6|11|17|4|2001|408|5316|Saturday|2001Q4|N|Y|N|2452215|2452518|2451866|2452139|N|N|N|N|N| +2452232|AAAAAAAAIALGFCAA|2001-11-18|1222|5316|408|2001|0|11|18|4|2001|408|5316|Sunday|2001Q4|N|N|N|2452215|2452518|2451867|2452140|N|N|N|N|N| +2452233|AAAAAAAAJALGFCAA|2001-11-19|1222|5316|408|2001|1|11|19|4|2001|408|5316|Monday|2001Q4|N|N|N|2452215|2452518|2451868|2452141|N|N|N|N|N| +2452234|AAAAAAAAKALGFCAA|2001-11-20|1222|5317|408|2001|2|11|20|4|2001|408|5317|Tuesday|2001Q4|N|N|N|2452215|2452518|2451869|2452142|N|N|N|N|N| +2452235|AAAAAAAALALGFCAA|2001-11-21|1222|5317|408|2001|3|11|21|4|2001|408|5317|Wednesday|2001Q4|N|N|N|2452215|2452518|2451870|2452143|N|N|N|N|N| +2452236|AAAAAAAAMALGFCAA|2001-11-22|1222|5317|408|2001|4|11|22|4|2001|408|5317|Thursday|2001Q4|N|N|N|2452215|2452518|2451871|2452144|N|N|N|N|N| +2452237|AAAAAAAANALGFCAA|2001-11-23|1222|5317|408|2001|5|11|23|4|2001|408|5317|Friday|2001Q4|N|Y|N|2452215|2452518|2451872|2452145|N|N|N|N|N| +2452238|AAAAAAAAOALGFCAA|2001-11-24|1222|5317|408|2001|6|11|24|4|2001|408|5317|Saturday|2001Q4|N|Y|N|2452215|2452518|2451873|2452146|N|N|N|N|N| +2452239|AAAAAAAAPALGFCAA|2001-11-25|1222|5317|408|2001|0|11|25|4|2001|408|5317|Sunday|2001Q4|N|N|N|2452215|2452518|2451874|2452147|N|N|N|N|N| +2452240|AAAAAAAAABLGFCAA|2001-11-26|1222|5317|408|2001|1|11|26|4|2001|408|5317|Monday|2001Q4|N|N|N|2452215|2452518|2451875|2452148|N|N|N|N|N| +2452241|AAAAAAAABBLGFCAA|2001-11-27|1222|5318|408|2001|2|11|27|4|2001|408|5318|Tuesday|2001Q4|N|N|N|2452215|2452518|2451876|2452149|N|N|N|N|N| +2452242|AAAAAAAACBLGFCAA|2001-11-28|1222|5318|408|2001|3|11|28|4|2001|408|5318|Wednesday|2001Q4|N|N|N|2452215|2452518|2451877|2452150|N|N|N|N|N| +2452243|AAAAAAAADBLGFCAA|2001-11-29|1222|5318|408|2001|4|11|29|4|2001|408|5318|Thursday|2001Q4|N|N|N|2452215|2452518|2451878|2452151|N|N|N|N|N| +2452244|AAAAAAAAEBLGFCAA|2001-11-30|1222|5318|408|2001|5|11|30|4|2001|408|5318|Friday|2001Q4|N|Y|N|2452215|2452518|2451879|2452152|N|N|N|N|N| +2452245|AAAAAAAAFBLGFCAA|2001-12-01|1223|5318|409|2001|6|12|1|4|2001|409|5318|Saturday|2001Q4|N|Y|N|2452245|2452578|2451880|2452153|N|N|N|N|N| +2452246|AAAAAAAAGBLGFCAA|2001-12-02|1223|5318|409|2001|0|12|2|4|2001|409|5318|Sunday|2001Q4|N|N|N|2452245|2452578|2451881|2452154|N|N|N|N|N| +2452247|AAAAAAAAHBLGFCAA|2001-12-03|1223|5318|409|2001|1|12|3|4|2001|409|5318|Monday|2001Q4|N|N|N|2452245|2452578|2451882|2452155|N|N|N|N|N| +2452248|AAAAAAAAIBLGFCAA|2001-12-04|1223|5319|409|2001|2|12|4|4|2001|409|5319|Tuesday|2001Q4|N|N|N|2452245|2452578|2451883|2452156|N|N|N|N|N| +2452249|AAAAAAAAJBLGFCAA|2001-12-05|1223|5319|409|2001|3|12|5|4|2001|409|5319|Wednesday|2001Q4|N|N|N|2452245|2452578|2451884|2452157|N|N|N|N|N| +2452250|AAAAAAAAKBLGFCAA|2001-12-06|1223|5319|409|2001|4|12|6|4|2001|409|5319|Thursday|2001Q4|N|N|N|2452245|2452578|2451885|2452158|N|N|N|N|N| +2452251|AAAAAAAALBLGFCAA|2001-12-07|1223|5319|409|2001|5|12|7|4|2001|409|5319|Friday|2001Q4|N|Y|N|2452245|2452578|2451886|2452159|N|N|N|N|N| +2452252|AAAAAAAAMBLGFCAA|2001-12-08|1223|5319|409|2001|6|12|8|4|2001|409|5319|Saturday|2001Q4|N|Y|N|2452245|2452578|2451887|2452160|N|N|N|N|N| +2452253|AAAAAAAANBLGFCAA|2001-12-09|1223|5319|409|2001|0|12|9|4|2001|409|5319|Sunday|2001Q4|N|N|N|2452245|2452578|2451888|2452161|N|N|N|N|N| +2452254|AAAAAAAAOBLGFCAA|2001-12-10|1223|5319|409|2001|1|12|10|4|2001|409|5319|Monday|2001Q4|N|N|N|2452245|2452578|2451889|2452162|N|N|N|N|N| +2452255|AAAAAAAAPBLGFCAA|2001-12-11|1223|5320|409|2001|2|12|11|4|2001|409|5320|Tuesday|2001Q4|N|N|N|2452245|2452578|2451890|2452163|N|N|N|N|N| +2452256|AAAAAAAAACLGFCAA|2001-12-12|1223|5320|409|2001|3|12|12|4|2001|409|5320|Wednesday|2001Q4|N|N|N|2452245|2452578|2451891|2452164|N|N|N|N|N| +2452257|AAAAAAAABCLGFCAA|2001-12-13|1223|5320|409|2001|4|12|13|4|2001|409|5320|Thursday|2001Q4|N|N|N|2452245|2452578|2451892|2452165|N|N|N|N|N| +2452258|AAAAAAAACCLGFCAA|2001-12-14|1223|5320|409|2001|5|12|14|4|2001|409|5320|Friday|2001Q4|N|Y|N|2452245|2452578|2451893|2452166|N|N|N|N|N| +2452259|AAAAAAAADCLGFCAA|2001-12-15|1223|5320|409|2001|6|12|15|4|2001|409|5320|Saturday|2001Q4|N|Y|N|2452245|2452578|2451894|2452167|N|N|N|N|N| +2452260|AAAAAAAAECLGFCAA|2001-12-16|1223|5320|409|2001|0|12|16|4|2001|409|5320|Sunday|2001Q4|N|N|N|2452245|2452578|2451895|2452168|N|N|N|N|N| +2452261|AAAAAAAAFCLGFCAA|2001-12-17|1223|5320|409|2001|1|12|17|4|2001|409|5320|Monday|2001Q4|N|N|N|2452245|2452578|2451896|2452169|N|N|N|N|N| +2452262|AAAAAAAAGCLGFCAA|2001-12-18|1223|5321|409|2001|2|12|18|4|2001|409|5321|Tuesday|2001Q4|N|N|N|2452245|2452578|2451897|2452170|N|N|N|N|N| +2452263|AAAAAAAAHCLGFCAA|2001-12-19|1223|5321|409|2001|3|12|19|4|2001|409|5321|Wednesday|2001Q4|N|N|N|2452245|2452578|2451898|2452171|N|N|N|N|N| +2452264|AAAAAAAAICLGFCAA|2001-12-20|1223|5321|409|2001|4|12|20|4|2001|409|5321|Thursday|2001Q4|N|N|N|2452245|2452578|2451899|2452172|N|N|N|N|N| +2452265|AAAAAAAAJCLGFCAA|2001-12-21|1223|5321|409|2001|5|12|21|4|2001|409|5321|Friday|2001Q4|N|Y|N|2452245|2452578|2451900|2452173|N|N|N|N|N| +2452266|AAAAAAAAKCLGFCAA|2001-12-22|1223|5321|409|2001|6|12|22|4|2001|409|5321|Saturday|2001Q4|N|Y|N|2452245|2452578|2451901|2452174|N|N|N|N|N| +2452267|AAAAAAAALCLGFCAA|2001-12-23|1223|5321|409|2001|0|12|23|4|2001|409|5321|Sunday|2001Q4|N|N|N|2452245|2452578|2451902|2452175|N|N|N|N|N| +2452268|AAAAAAAAMCLGFCAA|2001-12-24|1223|5321|409|2001|1|12|24|4|2001|409|5321|Monday|2001Q4|N|N|N|2452245|2452578|2451903|2452176|N|N|N|N|N| +2452269|AAAAAAAANCLGFCAA|2001-12-25|1223|5322|409|2001|2|12|25|4|2001|409|5322|Tuesday|2001Q4|N|N|N|2452245|2452578|2451904|2452177|N|N|N|N|N| +2452270|AAAAAAAAOCLGFCAA|2001-12-26|1223|5322|409|2001|3|12|26|4|2001|409|5322|Wednesday|2001Q4|Y|N|N|2452245|2452578|2451905|2452178|N|N|N|N|N| +2452271|AAAAAAAAPCLGFCAA|2001-12-27|1223|5322|409|2001|4|12|27|4|2001|409|5322|Thursday|2001Q4|N|N|Y|2452245|2452578|2451906|2452179|N|N|N|N|N| +2452272|AAAAAAAAADLGFCAA|2001-12-28|1223|5322|409|2001|5|12|28|4|2001|409|5322|Friday|2001Q4|N|Y|N|2452245|2452578|2451907|2452180|N|N|N|N|N| +2452273|AAAAAAAABDLGFCAA|2001-12-29|1223|5322|409|2001|6|12|29|4|2001|409|5322|Saturday|2001Q4|N|Y|N|2452245|2452578|2451908|2452181|N|N|N|N|N| +2452274|AAAAAAAACDLGFCAA|2001-12-30|1223|5322|409|2001|0|12|30|4|2001|409|5322|Sunday|2001Q4|N|N|N|2452245|2452578|2451909|2452182|N|N|N|N|N| +2452275|AAAAAAAADDLGFCAA|2001-12-31|1223|5322|409|2001|1|12|31|4|2001|409|5322|Monday|2001Q4|N|N|N|2452245|2452578|2451910|2452183|N|N|N|N|N| +2452276|AAAAAAAAEDLGFCAA|2002-01-01|1224|5323|409|2002|2|1|1|1|2002|409|5323|Tuesday|2002Q1|Y|N|N|2452276|2452275|2451911|2452184|N|N|N|N|N| +2452277|AAAAAAAAFDLGFCAA|2002-01-02|1224|5323|409|2002|3|1|2|1|2002|409|5323|Wednesday|2002Q1|N|N|Y|2452276|2452275|2451912|2452185|N|N|N|N|N| +2452278|AAAAAAAAGDLGFCAA|2002-01-03|1224|5323|409|2002|4|1|3|1|2002|409|5323|Thursday|2002Q1|N|N|N|2452276|2452275|2451913|2452186|N|N|N|N|N| +2452279|AAAAAAAAHDLGFCAA|2002-01-04|1224|5323|409|2002|5|1|4|1|2002|409|5323|Friday|2002Q1|N|Y|N|2452276|2452275|2451914|2452187|N|N|N|N|N| +2452280|AAAAAAAAIDLGFCAA|2002-01-05|1224|5323|409|2002|6|1|5|1|2002|409|5323|Saturday|2002Q1|N|Y|N|2452276|2452275|2451915|2452188|N|N|N|N|N| +2452281|AAAAAAAAJDLGFCAA|2002-01-06|1224|5323|409|2002|0|1|6|1|2002|409|5323|Sunday|2002Q1|N|N|N|2452276|2452275|2451916|2452189|N|N|N|N|N| +2452282|AAAAAAAAKDLGFCAA|2002-01-07|1224|5323|409|2002|1|1|7|1|2002|409|5323|Monday|2002Q1|N|N|N|2452276|2452275|2451917|2452190|N|N|N|N|N| +2452283|AAAAAAAALDLGFCAA|2002-01-08|1224|5324|409|2002|2|1|8|1|2002|409|5324|Tuesday|2002Q1|N|N|N|2452276|2452275|2451918|2452191|N|N|N|N|N| +2452284|AAAAAAAAMDLGFCAA|2002-01-09|1224|5324|409|2002|3|1|9|1|2002|409|5324|Wednesday|2002Q1|N|N|N|2452276|2452275|2451919|2452192|N|N|N|N|N| +2452285|AAAAAAAANDLGFCAA|2002-01-10|1224|5324|409|2002|4|1|10|1|2002|409|5324|Thursday|2002Q1|N|N|N|2452276|2452275|2451920|2452193|N|N|N|N|N| +2452286|AAAAAAAAODLGFCAA|2002-01-11|1224|5324|409|2002|5|1|11|1|2002|409|5324|Friday|2002Q1|N|Y|N|2452276|2452275|2451921|2452194|N|N|N|N|N| +2452287|AAAAAAAAPDLGFCAA|2002-01-12|1224|5324|409|2002|6|1|12|1|2002|409|5324|Saturday|2002Q1|N|Y|N|2452276|2452275|2451922|2452195|N|N|N|N|N| +2452288|AAAAAAAAAELGFCAA|2002-01-13|1224|5324|409|2002|0|1|13|1|2002|409|5324|Sunday|2002Q1|N|N|N|2452276|2452275|2451923|2452196|N|N|N|N|N| +2452289|AAAAAAAABELGFCAA|2002-01-14|1224|5324|409|2002|1|1|14|1|2002|409|5324|Monday|2002Q1|N|N|N|2452276|2452275|2451924|2452197|N|N|N|N|N| +2452290|AAAAAAAACELGFCAA|2002-01-15|1224|5325|409|2002|2|1|15|1|2002|409|5325|Tuesday|2002Q1|N|N|N|2452276|2452275|2451925|2452198|N|N|N|N|N| +2452291|AAAAAAAADELGFCAA|2002-01-16|1224|5325|409|2002|3|1|16|1|2002|409|5325|Wednesday|2002Q1|N|N|N|2452276|2452275|2451926|2452199|N|N|N|N|N| +2452292|AAAAAAAAEELGFCAA|2002-01-17|1224|5325|409|2002|4|1|17|1|2002|409|5325|Thursday|2002Q1|N|N|N|2452276|2452275|2451927|2452200|N|N|N|N|N| +2452293|AAAAAAAAFELGFCAA|2002-01-18|1224|5325|409|2002|5|1|18|1|2002|409|5325|Friday|2002Q1|N|Y|N|2452276|2452275|2451928|2452201|N|N|N|N|N| +2452294|AAAAAAAAGELGFCAA|2002-01-19|1224|5325|409|2002|6|1|19|1|2002|409|5325|Saturday|2002Q1|N|Y|N|2452276|2452275|2451929|2452202|N|N|N|N|N| +2452295|AAAAAAAAHELGFCAA|2002-01-20|1224|5325|409|2002|0|1|20|1|2002|409|5325|Sunday|2002Q1|N|N|N|2452276|2452275|2451930|2452203|N|N|N|N|N| +2452296|AAAAAAAAIELGFCAA|2002-01-21|1224|5325|409|2002|1|1|21|1|2002|409|5325|Monday|2002Q1|N|N|N|2452276|2452275|2451931|2452204|N|N|N|N|N| +2452297|AAAAAAAAJELGFCAA|2002-01-22|1224|5326|409|2002|2|1|22|1|2002|409|5326|Tuesday|2002Q1|N|N|N|2452276|2452275|2451932|2452205|N|N|N|N|N| +2452298|AAAAAAAAKELGFCAA|2002-01-23|1224|5326|409|2002|3|1|23|1|2002|409|5326|Wednesday|2002Q1|N|N|N|2452276|2452275|2451933|2452206|N|N|N|N|N| +2452299|AAAAAAAALELGFCAA|2002-01-24|1224|5326|409|2002|4|1|24|1|2002|409|5326|Thursday|2002Q1|N|N|N|2452276|2452275|2451934|2452207|N|N|N|N|N| +2452300|AAAAAAAAMELGFCAA|2002-01-25|1224|5326|409|2002|5|1|25|1|2002|409|5326|Friday|2002Q1|N|Y|N|2452276|2452275|2451935|2452208|N|N|N|N|N| +2452301|AAAAAAAANELGFCAA|2002-01-26|1224|5326|409|2002|6|1|26|1|2002|409|5326|Saturday|2002Q1|N|Y|N|2452276|2452275|2451936|2452209|N|N|N|N|N| +2452302|AAAAAAAAOELGFCAA|2002-01-27|1224|5326|409|2002|0|1|27|1|2002|409|5326|Sunday|2002Q1|N|N|N|2452276|2452275|2451937|2452210|N|N|N|N|N| +2452303|AAAAAAAAPELGFCAA|2002-01-28|1224|5326|409|2002|1|1|28|1|2002|409|5326|Monday|2002Q1|N|N|N|2452276|2452275|2451938|2452211|N|N|N|N|N| +2452304|AAAAAAAAAFLGFCAA|2002-01-29|1224|5327|409|2002|2|1|29|1|2002|409|5327|Tuesday|2002Q1|N|N|N|2452276|2452275|2451939|2452212|N|N|N|N|N| +2452305|AAAAAAAABFLGFCAA|2002-01-30|1224|5327|409|2002|3|1|30|1|2002|409|5327|Wednesday|2002Q1|N|N|N|2452276|2452275|2451940|2452213|N|N|N|N|N| +2452306|AAAAAAAACFLGFCAA|2002-01-31|1224|5327|409|2002|4|1|31|1|2002|409|5327|Thursday|2002Q1|N|N|N|2452276|2452275|2451941|2452214|N|N|N|N|N| +2452307|AAAAAAAADFLGFCAA|2002-02-01|1225|5327|409|2002|5|2|1|1|2002|409|5327|Friday|2002Q1|N|Y|N|2452307|2452337|2451942|2452215|N|N|N|N|N| +2452308|AAAAAAAAEFLGFCAA|2002-02-02|1225|5327|409|2002|6|2|2|1|2002|409|5327|Saturday|2002Q1|N|Y|N|2452307|2452337|2451943|2452216|N|N|N|N|N| +2452309|AAAAAAAAFFLGFCAA|2002-02-03|1225|5327|409|2002|0|2|3|1|2002|409|5327|Sunday|2002Q1|N|N|N|2452307|2452337|2451944|2452217|N|N|N|N|N| +2452310|AAAAAAAAGFLGFCAA|2002-02-04|1225|5327|409|2002|1|2|4|1|2002|409|5327|Monday|2002Q1|N|N|N|2452307|2452337|2451945|2452218|N|N|N|N|N| +2452311|AAAAAAAAHFLGFCAA|2002-02-05|1225|5328|409|2002|2|2|5|1|2002|409|5328|Tuesday|2002Q1|N|N|N|2452307|2452337|2451946|2452219|N|N|N|N|N| +2452312|AAAAAAAAIFLGFCAA|2002-02-06|1225|5328|409|2002|3|2|6|1|2002|409|5328|Wednesday|2002Q1|N|N|N|2452307|2452337|2451947|2452220|N|N|N|N|N| +2452313|AAAAAAAAJFLGFCAA|2002-02-07|1225|5328|409|2002|4|2|7|1|2002|409|5328|Thursday|2002Q1|N|N|N|2452307|2452337|2451948|2452221|N|N|N|N|N| +2452314|AAAAAAAAKFLGFCAA|2002-02-08|1225|5328|409|2002|5|2|8|1|2002|409|5328|Friday|2002Q1|N|Y|N|2452307|2452337|2451949|2452222|N|N|N|N|N| +2452315|AAAAAAAALFLGFCAA|2002-02-09|1225|5328|409|2002|6|2|9|1|2002|409|5328|Saturday|2002Q1|N|Y|N|2452307|2452337|2451950|2452223|N|N|N|N|N| +2452316|AAAAAAAAMFLGFCAA|2002-02-10|1225|5328|409|2002|0|2|10|1|2002|409|5328|Sunday|2002Q1|N|N|N|2452307|2452337|2451951|2452224|N|N|N|N|N| +2452317|AAAAAAAANFLGFCAA|2002-02-11|1225|5328|409|2002|1|2|11|1|2002|409|5328|Monday|2002Q1|N|N|N|2452307|2452337|2451952|2452225|N|N|N|N|N| +2452318|AAAAAAAAOFLGFCAA|2002-02-12|1225|5329|409|2002|2|2|12|1|2002|409|5329|Tuesday|2002Q1|N|N|N|2452307|2452337|2451953|2452226|N|N|N|N|N| +2452319|AAAAAAAAPFLGFCAA|2002-02-13|1225|5329|409|2002|3|2|13|1|2002|409|5329|Wednesday|2002Q1|N|N|N|2452307|2452337|2451954|2452227|N|N|N|N|N| +2452320|AAAAAAAAAGLGFCAA|2002-02-14|1225|5329|409|2002|4|2|14|1|2002|409|5329|Thursday|2002Q1|N|N|N|2452307|2452337|2451955|2452228|N|N|N|N|N| +2452321|AAAAAAAABGLGFCAA|2002-02-15|1225|5329|409|2002|5|2|15|1|2002|409|5329|Friday|2002Q1|N|Y|N|2452307|2452337|2451956|2452229|N|N|N|N|N| +2452322|AAAAAAAACGLGFCAA|2002-02-16|1225|5329|409|2002|6|2|16|1|2002|409|5329|Saturday|2002Q1|N|Y|N|2452307|2452337|2451957|2452230|N|N|N|N|N| +2452323|AAAAAAAADGLGFCAA|2002-02-17|1225|5329|409|2002|0|2|17|1|2002|409|5329|Sunday|2002Q1|N|N|N|2452307|2452337|2451958|2452231|N|N|N|N|N| +2452324|AAAAAAAAEGLGFCAA|2002-02-18|1225|5329|409|2002|1|2|18|1|2002|409|5329|Monday|2002Q1|N|N|N|2452307|2452337|2451959|2452232|N|N|N|N|N| +2452325|AAAAAAAAFGLGFCAA|2002-02-19|1225|5330|409|2002|2|2|19|1|2002|409|5330|Tuesday|2002Q1|N|N|N|2452307|2452337|2451960|2452233|N|N|N|N|N| +2452326|AAAAAAAAGGLGFCAA|2002-02-20|1225|5330|409|2002|3|2|20|1|2002|409|5330|Wednesday|2002Q1|N|N|N|2452307|2452337|2451961|2452234|N|N|N|N|N| +2452327|AAAAAAAAHGLGFCAA|2002-02-21|1225|5330|409|2002|4|2|21|1|2002|409|5330|Thursday|2002Q1|N|N|N|2452307|2452337|2451962|2452235|N|N|N|N|N| +2452328|AAAAAAAAIGLGFCAA|2002-02-22|1225|5330|409|2002|5|2|22|1|2002|409|5330|Friday|2002Q1|N|Y|N|2452307|2452337|2451963|2452236|N|N|N|N|N| +2452329|AAAAAAAAJGLGFCAA|2002-02-23|1225|5330|409|2002|6|2|23|1|2002|409|5330|Saturday|2002Q1|N|Y|N|2452307|2452337|2451964|2452237|N|N|N|N|N| +2452330|AAAAAAAAKGLGFCAA|2002-02-24|1225|5330|409|2002|0|2|24|1|2002|409|5330|Sunday|2002Q1|N|N|N|2452307|2452337|2451965|2452238|N|N|N|N|N| +2452331|AAAAAAAALGLGFCAA|2002-02-25|1225|5330|409|2002|1|2|25|1|2002|409|5330|Monday|2002Q1|N|N|N|2452307|2452337|2451966|2452239|N|N|N|N|N| +2452332|AAAAAAAAMGLGFCAA|2002-02-26|1225|5331|409|2002|2|2|26|1|2002|409|5331|Tuesday|2002Q1|N|N|N|2452307|2452337|2451967|2452240|N|N|N|N|N| +2452333|AAAAAAAANGLGFCAA|2002-02-27|1225|5331|409|2002|3|2|27|1|2002|409|5331|Wednesday|2002Q1|N|N|N|2452307|2452337|2451968|2452241|N|N|N|N|N| +2452334|AAAAAAAAOGLGFCAA|2002-02-28|1225|5331|409|2002|4|2|28|1|2002|409|5331|Thursday|2002Q1|N|N|N|2452307|2452337|2451969|2452242|N|N|N|N|N| +2452335|AAAAAAAAPGLGFCAA|2002-03-01|1226|5331|410|2002|5|3|1|1|2002|410|5331|Friday|2002Q1|N|Y|N|2452335|2452393|2451970|2452243|N|N|N|N|N| +2452336|AAAAAAAAAHLGFCAA|2002-03-02|1226|5331|410|2002|6|3|2|1|2002|410|5331|Saturday|2002Q1|N|Y|N|2452335|2452393|2451971|2452244|N|N|N|N|N| +2452337|AAAAAAAABHLGFCAA|2002-03-03|1226|5331|410|2002|0|3|3|1|2002|410|5331|Sunday|2002Q1|N|N|N|2452335|2452393|2451972|2452245|N|N|N|N|N| +2452338|AAAAAAAACHLGFCAA|2002-03-04|1226|5331|410|2002|1|3|4|1|2002|410|5331|Monday|2002Q1|N|N|N|2452335|2452393|2451973|2452246|N|N|N|N|N| +2452339|AAAAAAAADHLGFCAA|2002-03-05|1226|5332|410|2002|2|3|5|1|2002|410|5332|Tuesday|2002Q1|N|N|N|2452335|2452393|2451974|2452247|N|N|N|N|N| +2452340|AAAAAAAAEHLGFCAA|2002-03-06|1226|5332|410|2002|3|3|6|1|2002|410|5332|Wednesday|2002Q1|N|N|N|2452335|2452393|2451975|2452248|N|N|N|N|N| +2452341|AAAAAAAAFHLGFCAA|2002-03-07|1226|5332|410|2002|4|3|7|1|2002|410|5332|Thursday|2002Q1|N|N|N|2452335|2452393|2451976|2452249|N|N|N|N|N| +2452342|AAAAAAAAGHLGFCAA|2002-03-08|1226|5332|410|2002|5|3|8|1|2002|410|5332|Friday|2002Q1|N|Y|N|2452335|2452393|2451977|2452250|N|N|N|N|N| +2452343|AAAAAAAAHHLGFCAA|2002-03-09|1226|5332|410|2002|6|3|9|1|2002|410|5332|Saturday|2002Q1|N|Y|N|2452335|2452393|2451978|2452251|N|N|N|N|N| +2452344|AAAAAAAAIHLGFCAA|2002-03-10|1226|5332|410|2002|0|3|10|1|2002|410|5332|Sunday|2002Q1|N|N|N|2452335|2452393|2451979|2452252|N|N|N|N|N| +2452345|AAAAAAAAJHLGFCAA|2002-03-11|1226|5332|410|2002|1|3|11|1|2002|410|5332|Monday|2002Q1|N|N|N|2452335|2452393|2451980|2452253|N|N|N|N|N| +2452346|AAAAAAAAKHLGFCAA|2002-03-12|1226|5333|410|2002|2|3|12|1|2002|410|5333|Tuesday|2002Q1|N|N|N|2452335|2452393|2451981|2452254|N|N|N|N|N| +2452347|AAAAAAAALHLGFCAA|2002-03-13|1226|5333|410|2002|3|3|13|1|2002|410|5333|Wednesday|2002Q1|N|N|N|2452335|2452393|2451982|2452255|N|N|N|N|N| +2452348|AAAAAAAAMHLGFCAA|2002-03-14|1226|5333|410|2002|4|3|14|1|2002|410|5333|Thursday|2002Q1|N|N|N|2452335|2452393|2451983|2452256|N|N|N|N|N| +2452349|AAAAAAAANHLGFCAA|2002-03-15|1226|5333|410|2002|5|3|15|1|2002|410|5333|Friday|2002Q1|N|Y|N|2452335|2452393|2451984|2452257|N|N|N|N|N| +2452350|AAAAAAAAOHLGFCAA|2002-03-16|1226|5333|410|2002|6|3|16|1|2002|410|5333|Saturday|2002Q1|N|Y|N|2452335|2452393|2451985|2452258|N|N|N|N|N| +2452351|AAAAAAAAPHLGFCAA|2002-03-17|1226|5333|410|2002|0|3|17|1|2002|410|5333|Sunday|2002Q1|N|N|N|2452335|2452393|2451986|2452259|N|N|N|N|N| +2452352|AAAAAAAAAILGFCAA|2002-03-18|1226|5333|410|2002|1|3|18|1|2002|410|5333|Monday|2002Q1|N|N|N|2452335|2452393|2451987|2452260|N|N|N|N|N| +2452353|AAAAAAAABILGFCAA|2002-03-19|1226|5334|410|2002|2|3|19|1|2002|410|5334|Tuesday|2002Q1|N|N|N|2452335|2452393|2451988|2452261|N|N|N|N|N| +2452354|AAAAAAAACILGFCAA|2002-03-20|1226|5334|410|2002|3|3|20|1|2002|410|5334|Wednesday|2002Q1|N|N|N|2452335|2452393|2451989|2452262|N|N|N|N|N| +2452355|AAAAAAAADILGFCAA|2002-03-21|1226|5334|410|2002|4|3|21|1|2002|410|5334|Thursday|2002Q1|N|N|N|2452335|2452393|2451990|2452263|N|N|N|N|N| +2452356|AAAAAAAAEILGFCAA|2002-03-22|1226|5334|410|2002|5|3|22|1|2002|410|5334|Friday|2002Q1|N|Y|N|2452335|2452393|2451991|2452264|N|N|N|N|N| +2452357|AAAAAAAAFILGFCAA|2002-03-23|1226|5334|410|2002|6|3|23|1|2002|410|5334|Saturday|2002Q1|N|Y|N|2452335|2452393|2451992|2452265|N|N|N|N|N| +2452358|AAAAAAAAGILGFCAA|2002-03-24|1226|5334|410|2002|0|3|24|1|2002|410|5334|Sunday|2002Q1|N|N|N|2452335|2452393|2451993|2452266|N|N|N|N|N| +2452359|AAAAAAAAHILGFCAA|2002-03-25|1226|5334|410|2002|1|3|25|1|2002|410|5334|Monday|2002Q1|N|N|N|2452335|2452393|2451994|2452267|N|N|N|N|N| +2452360|AAAAAAAAIILGFCAA|2002-03-26|1226|5335|410|2002|2|3|26|1|2002|410|5335|Tuesday|2002Q1|N|N|N|2452335|2452393|2451995|2452268|N|N|N|N|N| +2452361|AAAAAAAAJILGFCAA|2002-03-27|1226|5335|410|2002|3|3|27|1|2002|410|5335|Wednesday|2002Q1|N|N|N|2452335|2452393|2451996|2452269|N|N|N|N|N| +2452362|AAAAAAAAKILGFCAA|2002-03-28|1226|5335|410|2002|4|3|28|1|2002|410|5335|Thursday|2002Q1|N|N|N|2452335|2452393|2451997|2452270|N|N|N|N|N| +2452363|AAAAAAAALILGFCAA|2002-03-29|1226|5335|410|2002|5|3|29|1|2002|410|5335|Friday|2002Q1|N|Y|N|2452335|2452393|2451998|2452271|N|N|N|N|N| +2452364|AAAAAAAAMILGFCAA|2002-03-30|1226|5335|410|2002|6|3|30|1|2002|410|5335|Saturday|2002Q1|N|Y|N|2452335|2452393|2451999|2452272|N|N|N|N|N| +2452365|AAAAAAAANILGFCAA|2002-03-31|1226|5335|410|2002|0|3|31|1|2002|410|5335|Sunday|2002Q1|N|N|N|2452335|2452393|2452000|2452273|N|N|N|N|N| +2452366|AAAAAAAAOILGFCAA|2002-04-01|1227|5335|410|2002|1|4|1|1|2002|410|5335|Monday|2002Q1|N|N|N|2452366|2452455|2452001|2452276|N|N|N|N|N| +2452367|AAAAAAAAPILGFCAA|2002-04-02|1227|5336|410|2002|2|4|2|2|2002|410|5336|Tuesday|2002Q2|N|N|N|2452366|2452455|2452002|2452277|N|N|N|N|N| +2452368|AAAAAAAAAJLGFCAA|2002-04-03|1227|5336|410|2002|3|4|3|2|2002|410|5336|Wednesday|2002Q2|N|N|N|2452366|2452455|2452003|2452278|N|N|N|N|N| +2452369|AAAAAAAABJLGFCAA|2002-04-04|1227|5336|410|2002|4|4|4|2|2002|410|5336|Thursday|2002Q2|N|N|N|2452366|2452455|2452004|2452279|N|N|N|N|N| +2452370|AAAAAAAACJLGFCAA|2002-04-05|1227|5336|410|2002|5|4|5|2|2002|410|5336|Friday|2002Q2|N|Y|N|2452366|2452455|2452005|2452280|N|N|N|N|N| +2452371|AAAAAAAADJLGFCAA|2002-04-06|1227|5336|410|2002|6|4|6|2|2002|410|5336|Saturday|2002Q2|N|Y|N|2452366|2452455|2452006|2452281|N|N|N|N|N| +2452372|AAAAAAAAEJLGFCAA|2002-04-07|1227|5336|410|2002|0|4|7|2|2002|410|5336|Sunday|2002Q2|N|N|N|2452366|2452455|2452007|2452282|N|N|N|N|N| +2452373|AAAAAAAAFJLGFCAA|2002-04-08|1227|5336|410|2002|1|4|8|2|2002|410|5336|Monday|2002Q2|N|N|N|2452366|2452455|2452008|2452283|N|N|N|N|N| +2452374|AAAAAAAAGJLGFCAA|2002-04-09|1227|5337|410|2002|2|4|9|2|2002|410|5337|Tuesday|2002Q2|N|N|N|2452366|2452455|2452009|2452284|N|N|N|N|N| +2452375|AAAAAAAAHJLGFCAA|2002-04-10|1227|5337|410|2002|3|4|10|2|2002|410|5337|Wednesday|2002Q2|N|N|N|2452366|2452455|2452010|2452285|N|N|N|N|N| +2452376|AAAAAAAAIJLGFCAA|2002-04-11|1227|5337|410|2002|4|4|11|2|2002|410|5337|Thursday|2002Q2|N|N|N|2452366|2452455|2452011|2452286|N|N|N|N|N| +2452377|AAAAAAAAJJLGFCAA|2002-04-12|1227|5337|410|2002|5|4|12|2|2002|410|5337|Friday|2002Q2|N|Y|N|2452366|2452455|2452012|2452287|N|N|N|N|N| +2452378|AAAAAAAAKJLGFCAA|2002-04-13|1227|5337|410|2002|6|4|13|2|2002|410|5337|Saturday|2002Q2|N|Y|N|2452366|2452455|2452013|2452288|N|N|N|N|N| +2452379|AAAAAAAALJLGFCAA|2002-04-14|1227|5337|410|2002|0|4|14|2|2002|410|5337|Sunday|2002Q2|N|N|N|2452366|2452455|2452014|2452289|N|N|N|N|N| +2452380|AAAAAAAAMJLGFCAA|2002-04-15|1227|5337|410|2002|1|4|15|2|2002|410|5337|Monday|2002Q2|N|N|N|2452366|2452455|2452015|2452290|N|N|N|N|N| +2452381|AAAAAAAANJLGFCAA|2002-04-16|1227|5338|410|2002|2|4|16|2|2002|410|5338|Tuesday|2002Q2|N|N|N|2452366|2452455|2452016|2452291|N|N|N|N|N| +2452382|AAAAAAAAOJLGFCAA|2002-04-17|1227|5338|410|2002|3|4|17|2|2002|410|5338|Wednesday|2002Q2|N|N|N|2452366|2452455|2452017|2452292|N|N|N|N|N| +2452383|AAAAAAAAPJLGFCAA|2002-04-18|1227|5338|410|2002|4|4|18|2|2002|410|5338|Thursday|2002Q2|N|N|N|2452366|2452455|2452018|2452293|N|N|N|N|N| +2452384|AAAAAAAAAKLGFCAA|2002-04-19|1227|5338|410|2002|5|4|19|2|2002|410|5338|Friday|2002Q2|N|Y|N|2452366|2452455|2452019|2452294|N|N|N|N|N| +2452385|AAAAAAAABKLGFCAA|2002-04-20|1227|5338|410|2002|6|4|20|2|2002|410|5338|Saturday|2002Q2|N|Y|N|2452366|2452455|2452020|2452295|N|N|N|N|N| +2452386|AAAAAAAACKLGFCAA|2002-04-21|1227|5338|410|2002|0|4|21|2|2002|410|5338|Sunday|2002Q2|N|N|N|2452366|2452455|2452021|2452296|N|N|N|N|N| +2452387|AAAAAAAADKLGFCAA|2002-04-22|1227|5338|410|2002|1|4|22|2|2002|410|5338|Monday|2002Q2|N|N|N|2452366|2452455|2452022|2452297|N|N|N|N|N| +2452388|AAAAAAAAEKLGFCAA|2002-04-23|1227|5339|410|2002|2|4|23|2|2002|410|5339|Tuesday|2002Q2|N|N|N|2452366|2452455|2452023|2452298|N|N|N|N|N| +2452389|AAAAAAAAFKLGFCAA|2002-04-24|1227|5339|410|2002|3|4|24|2|2002|410|5339|Wednesday|2002Q2|N|N|N|2452366|2452455|2452024|2452299|N|N|N|N|N| +2452390|AAAAAAAAGKLGFCAA|2002-04-25|1227|5339|410|2002|4|4|25|2|2002|410|5339|Thursday|2002Q2|N|N|N|2452366|2452455|2452025|2452300|N|N|N|N|N| +2452391|AAAAAAAAHKLGFCAA|2002-04-26|1227|5339|410|2002|5|4|26|2|2002|410|5339|Friday|2002Q2|N|Y|N|2452366|2452455|2452026|2452301|N|N|N|N|N| +2452392|AAAAAAAAIKLGFCAA|2002-04-27|1227|5339|410|2002|6|4|27|2|2002|410|5339|Saturday|2002Q2|N|Y|N|2452366|2452455|2452027|2452302|N|N|N|N|N| +2452393|AAAAAAAAJKLGFCAA|2002-04-28|1227|5339|410|2002|0|4|28|2|2002|410|5339|Sunday|2002Q2|N|N|N|2452366|2452455|2452028|2452303|N|N|N|N|N| +2452394|AAAAAAAAKKLGFCAA|2002-04-29|1227|5339|410|2002|1|4|29|2|2002|410|5339|Monday|2002Q2|N|N|N|2452366|2452455|2452029|2452304|N|N|N|N|N| +2452395|AAAAAAAALKLGFCAA|2002-04-30|1227|5340|410|2002|2|4|30|2|2002|410|5340|Tuesday|2002Q2|N|N|N|2452366|2452455|2452030|2452305|N|N|N|N|N| +2452396|AAAAAAAAMKLGFCAA|2002-05-01|1228|5340|410|2002|3|5|1|2|2002|410|5340|Wednesday|2002Q2|N|N|N|2452396|2452515|2452031|2452306|N|N|N|N|N| +2452397|AAAAAAAANKLGFCAA|2002-05-02|1228|5340|410|2002|4|5|2|2|2002|410|5340|Thursday|2002Q2|N|N|N|2452396|2452515|2452032|2452307|N|N|N|N|N| +2452398|AAAAAAAAOKLGFCAA|2002-05-03|1228|5340|410|2002|5|5|3|2|2002|410|5340|Friday|2002Q2|N|Y|N|2452396|2452515|2452033|2452308|N|N|N|N|N| +2452399|AAAAAAAAPKLGFCAA|2002-05-04|1228|5340|410|2002|6|5|4|2|2002|410|5340|Saturday|2002Q2|N|Y|N|2452396|2452515|2452034|2452309|N|N|N|N|N| +2452400|AAAAAAAAALLGFCAA|2002-05-05|1228|5340|410|2002|0|5|5|2|2002|410|5340|Sunday|2002Q2|N|N|N|2452396|2452515|2452035|2452310|N|N|N|N|N| +2452401|AAAAAAAABLLGFCAA|2002-05-06|1228|5340|410|2002|1|5|6|2|2002|410|5340|Monday|2002Q2|N|N|N|2452396|2452515|2452036|2452311|N|N|N|N|N| +2452402|AAAAAAAACLLGFCAA|2002-05-07|1228|5341|410|2002|2|5|7|2|2002|410|5341|Tuesday|2002Q2|N|N|N|2452396|2452515|2452037|2452312|N|N|N|N|N| +2452403|AAAAAAAADLLGFCAA|2002-05-08|1228|5341|410|2002|3|5|8|2|2002|410|5341|Wednesday|2002Q2|N|N|N|2452396|2452515|2452038|2452313|N|N|N|N|N| +2452404|AAAAAAAAELLGFCAA|2002-05-09|1228|5341|410|2002|4|5|9|2|2002|410|5341|Thursday|2002Q2|N|N|N|2452396|2452515|2452039|2452314|N|N|N|N|N| +2452405|AAAAAAAAFLLGFCAA|2002-05-10|1228|5341|410|2002|5|5|10|2|2002|410|5341|Friday|2002Q2|N|Y|N|2452396|2452515|2452040|2452315|N|N|N|N|N| +2452406|AAAAAAAAGLLGFCAA|2002-05-11|1228|5341|410|2002|6|5|11|2|2002|410|5341|Saturday|2002Q2|N|Y|N|2452396|2452515|2452041|2452316|N|N|N|N|N| +2452407|AAAAAAAAHLLGFCAA|2002-05-12|1228|5341|410|2002|0|5|12|2|2002|410|5341|Sunday|2002Q2|N|N|N|2452396|2452515|2452042|2452317|N|N|N|N|N| +2452408|AAAAAAAAILLGFCAA|2002-05-13|1228|5341|410|2002|1|5|13|2|2002|410|5341|Monday|2002Q2|N|N|N|2452396|2452515|2452043|2452318|N|N|N|N|N| +2452409|AAAAAAAAJLLGFCAA|2002-05-14|1228|5342|410|2002|2|5|14|2|2002|410|5342|Tuesday|2002Q2|N|N|N|2452396|2452515|2452044|2452319|N|N|N|N|N| +2452410|AAAAAAAAKLLGFCAA|2002-05-15|1228|5342|410|2002|3|5|15|2|2002|410|5342|Wednesday|2002Q2|N|N|N|2452396|2452515|2452045|2452320|N|N|N|N|N| +2452411|AAAAAAAALLLGFCAA|2002-05-16|1228|5342|410|2002|4|5|16|2|2002|410|5342|Thursday|2002Q2|N|N|N|2452396|2452515|2452046|2452321|N|N|N|N|N| +2452412|AAAAAAAAMLLGFCAA|2002-05-17|1228|5342|410|2002|5|5|17|2|2002|410|5342|Friday|2002Q2|N|Y|N|2452396|2452515|2452047|2452322|N|N|N|N|N| +2452413|AAAAAAAANLLGFCAA|2002-05-18|1228|5342|410|2002|6|5|18|2|2002|410|5342|Saturday|2002Q2|N|Y|N|2452396|2452515|2452048|2452323|N|N|N|N|N| +2452414|AAAAAAAAOLLGFCAA|2002-05-19|1228|5342|410|2002|0|5|19|2|2002|410|5342|Sunday|2002Q2|N|N|N|2452396|2452515|2452049|2452324|N|N|N|N|N| +2452415|AAAAAAAAPLLGFCAA|2002-05-20|1228|5342|410|2002|1|5|20|2|2002|410|5342|Monday|2002Q2|N|N|N|2452396|2452515|2452050|2452325|N|N|N|N|N| +2452416|AAAAAAAAAMLGFCAA|2002-05-21|1228|5343|410|2002|2|5|21|2|2002|410|5343|Tuesday|2002Q2|N|N|N|2452396|2452515|2452051|2452326|N|N|N|N|N| +2452417|AAAAAAAABMLGFCAA|2002-05-22|1228|5343|410|2002|3|5|22|2|2002|410|5343|Wednesday|2002Q2|N|N|N|2452396|2452515|2452052|2452327|N|N|N|N|N| +2452418|AAAAAAAACMLGFCAA|2002-05-23|1228|5343|410|2002|4|5|23|2|2002|410|5343|Thursday|2002Q2|N|N|N|2452396|2452515|2452053|2452328|N|N|N|N|N| +2452419|AAAAAAAADMLGFCAA|2002-05-24|1228|5343|410|2002|5|5|24|2|2002|410|5343|Friday|2002Q2|N|Y|N|2452396|2452515|2452054|2452329|N|N|N|N|N| +2452420|AAAAAAAAEMLGFCAA|2002-05-25|1228|5343|410|2002|6|5|25|2|2002|410|5343|Saturday|2002Q2|N|Y|N|2452396|2452515|2452055|2452330|N|N|N|N|N| +2452421|AAAAAAAAFMLGFCAA|2002-05-26|1228|5343|410|2002|0|5|26|2|2002|410|5343|Sunday|2002Q2|N|N|N|2452396|2452515|2452056|2452331|N|N|N|N|N| +2452422|AAAAAAAAGMLGFCAA|2002-05-27|1228|5343|410|2002|1|5|27|2|2002|410|5343|Monday|2002Q2|N|N|N|2452396|2452515|2452057|2452332|N|N|N|N|N| +2452423|AAAAAAAAHMLGFCAA|2002-05-28|1228|5344|410|2002|2|5|28|2|2002|410|5344|Tuesday|2002Q2|N|N|N|2452396|2452515|2452058|2452333|N|N|N|N|N| +2452424|AAAAAAAAIMLGFCAA|2002-05-29|1228|5344|410|2002|3|5|29|2|2002|410|5344|Wednesday|2002Q2|N|N|N|2452396|2452515|2452059|2452334|N|N|N|N|N| +2452425|AAAAAAAAJMLGFCAA|2002-05-30|1228|5344|410|2002|4|5|30|2|2002|410|5344|Thursday|2002Q2|N|N|N|2452396|2452515|2452060|2452335|N|N|N|N|N| +2452426|AAAAAAAAKMLGFCAA|2002-05-31|1228|5344|410|2002|5|5|31|2|2002|410|5344|Friday|2002Q2|N|Y|N|2452396|2452515|2452061|2452336|N|N|N|N|N| +2452427|AAAAAAAALMLGFCAA|2002-06-01|1229|5344|411|2002|6|6|1|2|2002|411|5344|Saturday|2002Q2|N|Y|N|2452427|2452577|2452062|2452337|N|N|N|N|N| +2452428|AAAAAAAAMMLGFCAA|2002-06-02|1229|5344|411|2002|0|6|2|2|2002|411|5344|Sunday|2002Q2|N|N|N|2452427|2452577|2452063|2452338|N|N|N|N|N| +2452429|AAAAAAAANMLGFCAA|2002-06-03|1229|5344|411|2002|1|6|3|2|2002|411|5344|Monday|2002Q2|N|N|N|2452427|2452577|2452064|2452339|N|N|N|N|N| +2452430|AAAAAAAAOMLGFCAA|2002-06-04|1229|5345|411|2002|2|6|4|2|2002|411|5345|Tuesday|2002Q2|N|N|N|2452427|2452577|2452065|2452340|N|N|N|N|N| +2452431|AAAAAAAAPMLGFCAA|2002-06-05|1229|5345|411|2002|3|6|5|2|2002|411|5345|Wednesday|2002Q2|N|N|N|2452427|2452577|2452066|2452341|N|N|N|N|N| +2452432|AAAAAAAAANLGFCAA|2002-06-06|1229|5345|411|2002|4|6|6|2|2002|411|5345|Thursday|2002Q2|N|N|N|2452427|2452577|2452067|2452342|N|N|N|N|N| +2452433|AAAAAAAABNLGFCAA|2002-06-07|1229|5345|411|2002|5|6|7|2|2002|411|5345|Friday|2002Q2|N|Y|N|2452427|2452577|2452068|2452343|N|N|N|N|N| +2452434|AAAAAAAACNLGFCAA|2002-06-08|1229|5345|411|2002|6|6|8|2|2002|411|5345|Saturday|2002Q2|N|Y|N|2452427|2452577|2452069|2452344|N|N|N|N|N| +2452435|AAAAAAAADNLGFCAA|2002-06-09|1229|5345|411|2002|0|6|9|2|2002|411|5345|Sunday|2002Q2|N|N|N|2452427|2452577|2452070|2452345|N|N|N|N|N| +2452436|AAAAAAAAENLGFCAA|2002-06-10|1229|5345|411|2002|1|6|10|2|2002|411|5345|Monday|2002Q2|N|N|N|2452427|2452577|2452071|2452346|N|N|N|N|N| +2452437|AAAAAAAAFNLGFCAA|2002-06-11|1229|5346|411|2002|2|6|11|2|2002|411|5346|Tuesday|2002Q2|N|N|N|2452427|2452577|2452072|2452347|N|N|N|N|N| +2452438|AAAAAAAAGNLGFCAA|2002-06-12|1229|5346|411|2002|3|6|12|2|2002|411|5346|Wednesday|2002Q2|N|N|N|2452427|2452577|2452073|2452348|N|N|N|N|N| +2452439|AAAAAAAAHNLGFCAA|2002-06-13|1229|5346|411|2002|4|6|13|2|2002|411|5346|Thursday|2002Q2|N|N|N|2452427|2452577|2452074|2452349|N|N|N|N|N| +2452440|AAAAAAAAINLGFCAA|2002-06-14|1229|5346|411|2002|5|6|14|2|2002|411|5346|Friday|2002Q2|N|Y|N|2452427|2452577|2452075|2452350|N|N|N|N|N| +2452441|AAAAAAAAJNLGFCAA|2002-06-15|1229|5346|411|2002|6|6|15|2|2002|411|5346|Saturday|2002Q2|N|Y|N|2452427|2452577|2452076|2452351|N|N|N|N|N| +2452442|AAAAAAAAKNLGFCAA|2002-06-16|1229|5346|411|2002|0|6|16|2|2002|411|5346|Sunday|2002Q2|N|N|N|2452427|2452577|2452077|2452352|N|N|N|N|N| +2452443|AAAAAAAALNLGFCAA|2002-06-17|1229|5346|411|2002|1|6|17|2|2002|411|5346|Monday|2002Q2|N|N|N|2452427|2452577|2452078|2452353|N|N|N|N|N| +2452444|AAAAAAAAMNLGFCAA|2002-06-18|1229|5347|411|2002|2|6|18|2|2002|411|5347|Tuesday|2002Q2|N|N|N|2452427|2452577|2452079|2452354|N|N|N|N|N| +2452445|AAAAAAAANNLGFCAA|2002-06-19|1229|5347|411|2002|3|6|19|2|2002|411|5347|Wednesday|2002Q2|N|N|N|2452427|2452577|2452080|2452355|N|N|N|N|N| +2452446|AAAAAAAAONLGFCAA|2002-06-20|1229|5347|411|2002|4|6|20|2|2002|411|5347|Thursday|2002Q2|N|N|N|2452427|2452577|2452081|2452356|N|N|N|N|N| +2452447|AAAAAAAAPNLGFCAA|2002-06-21|1229|5347|411|2002|5|6|21|2|2002|411|5347|Friday|2002Q2|N|Y|N|2452427|2452577|2452082|2452357|N|N|N|N|N| +2452448|AAAAAAAAAOLGFCAA|2002-06-22|1229|5347|411|2002|6|6|22|2|2002|411|5347|Saturday|2002Q2|N|Y|N|2452427|2452577|2452083|2452358|N|N|N|N|N| +2452449|AAAAAAAABOLGFCAA|2002-06-23|1229|5347|411|2002|0|6|23|2|2002|411|5347|Sunday|2002Q2|N|N|N|2452427|2452577|2452084|2452359|N|N|N|N|N| +2452450|AAAAAAAACOLGFCAA|2002-06-24|1229|5347|411|2002|1|6|24|2|2002|411|5347|Monday|2002Q2|N|N|N|2452427|2452577|2452085|2452360|N|N|N|N|N| +2452451|AAAAAAAADOLGFCAA|2002-06-25|1229|5348|411|2002|2|6|25|2|2002|411|5348|Tuesday|2002Q2|N|N|N|2452427|2452577|2452086|2452361|N|N|N|N|N| +2452452|AAAAAAAAEOLGFCAA|2002-06-26|1229|5348|411|2002|3|6|26|2|2002|411|5348|Wednesday|2002Q2|N|N|N|2452427|2452577|2452087|2452362|N|N|N|N|N| +2452453|AAAAAAAAFOLGFCAA|2002-06-27|1229|5348|411|2002|4|6|27|2|2002|411|5348|Thursday|2002Q2|N|N|N|2452427|2452577|2452088|2452363|N|N|N|N|N| +2452454|AAAAAAAAGOLGFCAA|2002-06-28|1229|5348|411|2002|5|6|28|2|2002|411|5348|Friday|2002Q2|N|Y|N|2452427|2452577|2452089|2452364|N|N|N|N|N| +2452455|AAAAAAAAHOLGFCAA|2002-06-29|1229|5348|411|2002|6|6|29|2|2002|411|5348|Saturday|2002Q2|N|Y|N|2452427|2452577|2452090|2452365|N|N|N|N|N| +2452456|AAAAAAAAIOLGFCAA|2002-06-30|1229|5348|411|2002|0|6|30|2|2002|411|5348|Sunday|2002Q2|N|N|N|2452427|2452577|2452091|2452366|N|N|N|N|N| +2452457|AAAAAAAAJOLGFCAA|2002-07-01|1230|5348|411|2002|1|7|1|2|2002|411|5348|Monday|2002Q2|N|N|N|2452457|2452637|2452092|2452366|N|N|N|N|N| +2452458|AAAAAAAAKOLGFCAA|2002-07-02|1230|5349|411|2002|2|7|2|3|2002|411|5349|Tuesday|2002Q3|N|N|N|2452457|2452637|2452093|2452367|N|N|N|N|N| +2452459|AAAAAAAALOLGFCAA|2002-07-03|1230|5349|411|2002|3|7|3|3|2002|411|5349|Wednesday|2002Q3|N|N|N|2452457|2452637|2452094|2452368|N|N|N|N|N| +2452460|AAAAAAAAMOLGFCAA|2002-07-04|1230|5349|411|2002|4|7|4|3|2002|411|5349|Thursday|2002Q3|N|N|N|2452457|2452637|2452095|2452369|N|N|N|N|N| +2452461|AAAAAAAANOLGFCAA|2002-07-05|1230|5349|411|2002|5|7|5|3|2002|411|5349|Friday|2002Q3|Y|Y|N|2452457|2452637|2452096|2452370|N|N|N|N|N| +2452462|AAAAAAAAOOLGFCAA|2002-07-06|1230|5349|411|2002|6|7|6|3|2002|411|5349|Saturday|2002Q3|N|Y|Y|2452457|2452637|2452097|2452371|N|N|N|N|N| +2452463|AAAAAAAAPOLGFCAA|2002-07-07|1230|5349|411|2002|0|7|7|3|2002|411|5349|Sunday|2002Q3|N|N|N|2452457|2452637|2452098|2452372|N|N|N|N|N| +2452464|AAAAAAAAAPLGFCAA|2002-07-08|1230|5349|411|2002|1|7|8|3|2002|411|5349|Monday|2002Q3|N|N|N|2452457|2452637|2452099|2452373|N|N|N|N|N| +2452465|AAAAAAAABPLGFCAA|2002-07-09|1230|5350|411|2002|2|7|9|3|2002|411|5350|Tuesday|2002Q3|N|N|N|2452457|2452637|2452100|2452374|N|N|N|N|N| +2452466|AAAAAAAACPLGFCAA|2002-07-10|1230|5350|411|2002|3|7|10|3|2002|411|5350|Wednesday|2002Q3|N|N|N|2452457|2452637|2452101|2452375|N|N|N|N|N| +2452467|AAAAAAAADPLGFCAA|2002-07-11|1230|5350|411|2002|4|7|11|3|2002|411|5350|Thursday|2002Q3|N|N|N|2452457|2452637|2452102|2452376|N|N|N|N|N| +2452468|AAAAAAAAEPLGFCAA|2002-07-12|1230|5350|411|2002|5|7|12|3|2002|411|5350|Friday|2002Q3|N|Y|N|2452457|2452637|2452103|2452377|N|N|N|N|N| +2452469|AAAAAAAAFPLGFCAA|2002-07-13|1230|5350|411|2002|6|7|13|3|2002|411|5350|Saturday|2002Q3|N|Y|N|2452457|2452637|2452104|2452378|N|N|N|N|N| +2452470|AAAAAAAAGPLGFCAA|2002-07-14|1230|5350|411|2002|0|7|14|3|2002|411|5350|Sunday|2002Q3|N|N|N|2452457|2452637|2452105|2452379|N|N|N|N|N| +2452471|AAAAAAAAHPLGFCAA|2002-07-15|1230|5350|411|2002|1|7|15|3|2002|411|5350|Monday|2002Q3|N|N|N|2452457|2452637|2452106|2452380|N|N|N|N|N| +2452472|AAAAAAAAIPLGFCAA|2002-07-16|1230|5351|411|2002|2|7|16|3|2002|411|5351|Tuesday|2002Q3|N|N|N|2452457|2452637|2452107|2452381|N|N|N|N|N| +2452473|AAAAAAAAJPLGFCAA|2002-07-17|1230|5351|411|2002|3|7|17|3|2002|411|5351|Wednesday|2002Q3|N|N|N|2452457|2452637|2452108|2452382|N|N|N|N|N| +2452474|AAAAAAAAKPLGFCAA|2002-07-18|1230|5351|411|2002|4|7|18|3|2002|411|5351|Thursday|2002Q3|N|N|N|2452457|2452637|2452109|2452383|N|N|N|N|N| +2452475|AAAAAAAALPLGFCAA|2002-07-19|1230|5351|411|2002|5|7|19|3|2002|411|5351|Friday|2002Q3|N|Y|N|2452457|2452637|2452110|2452384|N|N|N|N|N| +2452476|AAAAAAAAMPLGFCAA|2002-07-20|1230|5351|411|2002|6|7|20|3|2002|411|5351|Saturday|2002Q3|N|Y|N|2452457|2452637|2452111|2452385|N|N|N|N|N| +2452477|AAAAAAAANPLGFCAA|2002-07-21|1230|5351|411|2002|0|7|21|3|2002|411|5351|Sunday|2002Q3|N|N|N|2452457|2452637|2452112|2452386|N|N|N|N|N| +2452478|AAAAAAAAOPLGFCAA|2002-07-22|1230|5351|411|2002|1|7|22|3|2002|411|5351|Monday|2002Q3|N|N|N|2452457|2452637|2452113|2452387|N|N|N|N|N| +2452479|AAAAAAAAPPLGFCAA|2002-07-23|1230|5352|411|2002|2|7|23|3|2002|411|5352|Tuesday|2002Q3|N|N|N|2452457|2452637|2452114|2452388|N|N|N|N|N| +2452480|AAAAAAAAAAMGFCAA|2002-07-24|1230|5352|411|2002|3|7|24|3|2002|411|5352|Wednesday|2002Q3|N|N|N|2452457|2452637|2452115|2452389|N|N|N|N|N| +2452481|AAAAAAAABAMGFCAA|2002-07-25|1230|5352|411|2002|4|7|25|3|2002|411|5352|Thursday|2002Q3|N|N|N|2452457|2452637|2452116|2452390|N|N|N|N|N| +2452482|AAAAAAAACAMGFCAA|2002-07-26|1230|5352|411|2002|5|7|26|3|2002|411|5352|Friday|2002Q3|N|Y|N|2452457|2452637|2452117|2452391|N|N|N|N|N| +2452483|AAAAAAAADAMGFCAA|2002-07-27|1230|5352|411|2002|6|7|27|3|2002|411|5352|Saturday|2002Q3|N|Y|N|2452457|2452637|2452118|2452392|N|N|N|N|N| +2452484|AAAAAAAAEAMGFCAA|2002-07-28|1230|5352|411|2002|0|7|28|3|2002|411|5352|Sunday|2002Q3|N|N|N|2452457|2452637|2452119|2452393|N|N|N|N|N| +2452485|AAAAAAAAFAMGFCAA|2002-07-29|1230|5352|411|2002|1|7|29|3|2002|411|5352|Monday|2002Q3|N|N|N|2452457|2452637|2452120|2452394|N|N|N|N|N| +2452486|AAAAAAAAGAMGFCAA|2002-07-30|1230|5353|411|2002|2|7|30|3|2002|411|5353|Tuesday|2002Q3|N|N|N|2452457|2452637|2452121|2452395|N|N|N|N|N| +2452487|AAAAAAAAHAMGFCAA|2002-07-31|1230|5353|411|2002|3|7|31|3|2002|411|5353|Wednesday|2002Q3|N|N|N|2452457|2452637|2452122|2452396|N|N|N|N|N| +2452488|AAAAAAAAIAMGFCAA|2002-08-01|1231|5353|411|2002|4|8|1|3|2002|411|5353|Thursday|2002Q3|N|N|N|2452488|2452699|2452123|2452397|N|N|N|N|N| +2452489|AAAAAAAAJAMGFCAA|2002-08-02|1231|5353|411|2002|5|8|2|3|2002|411|5353|Friday|2002Q3|N|Y|N|2452488|2452699|2452124|2452398|N|N|N|N|N| +2452490|AAAAAAAAKAMGFCAA|2002-08-03|1231|5353|411|2002|6|8|3|3|2002|411|5353|Saturday|2002Q3|N|Y|N|2452488|2452699|2452125|2452399|N|N|N|N|N| +2452491|AAAAAAAALAMGFCAA|2002-08-04|1231|5353|411|2002|0|8|4|3|2002|411|5353|Sunday|2002Q3|N|N|N|2452488|2452699|2452126|2452400|N|N|N|N|N| +2452492|AAAAAAAAMAMGFCAA|2002-08-05|1231|5353|411|2002|1|8|5|3|2002|411|5353|Monday|2002Q3|N|N|N|2452488|2452699|2452127|2452401|N|N|N|N|N| +2452493|AAAAAAAANAMGFCAA|2002-08-06|1231|5354|411|2002|2|8|6|3|2002|411|5354|Tuesday|2002Q3|N|N|N|2452488|2452699|2452128|2452402|N|N|N|N|N| +2452494|AAAAAAAAOAMGFCAA|2002-08-07|1231|5354|411|2002|3|8|7|3|2002|411|5354|Wednesday|2002Q3|N|N|N|2452488|2452699|2452129|2452403|N|N|N|N|N| +2452495|AAAAAAAAPAMGFCAA|2002-08-08|1231|5354|411|2002|4|8|8|3|2002|411|5354|Thursday|2002Q3|N|N|N|2452488|2452699|2452130|2452404|N|N|N|N|N| +2452496|AAAAAAAAABMGFCAA|2002-08-09|1231|5354|411|2002|5|8|9|3|2002|411|5354|Friday|2002Q3|N|Y|N|2452488|2452699|2452131|2452405|N|N|N|N|N| +2452497|AAAAAAAABBMGFCAA|2002-08-10|1231|5354|411|2002|6|8|10|3|2002|411|5354|Saturday|2002Q3|N|Y|N|2452488|2452699|2452132|2452406|N|N|N|N|N| +2452498|AAAAAAAACBMGFCAA|2002-08-11|1231|5354|411|2002|0|8|11|3|2002|411|5354|Sunday|2002Q3|N|N|N|2452488|2452699|2452133|2452407|N|N|N|N|N| +2452499|AAAAAAAADBMGFCAA|2002-08-12|1231|5354|411|2002|1|8|12|3|2002|411|5354|Monday|2002Q3|N|N|N|2452488|2452699|2452134|2452408|N|N|N|N|N| +2452500|AAAAAAAAEBMGFCAA|2002-08-13|1231|5355|411|2002|2|8|13|3|2002|411|5355|Tuesday|2002Q3|N|N|N|2452488|2452699|2452135|2452409|N|N|N|N|N| +2452501|AAAAAAAAFBMGFCAA|2002-08-14|1231|5355|411|2002|3|8|14|3|2002|411|5355|Wednesday|2002Q3|N|N|N|2452488|2452699|2452136|2452410|N|N|N|N|N| +2452502|AAAAAAAAGBMGFCAA|2002-08-15|1231|5355|411|2002|4|8|15|3|2002|411|5355|Thursday|2002Q3|N|N|N|2452488|2452699|2452137|2452411|N|N|N|N|N| +2452503|AAAAAAAAHBMGFCAA|2002-08-16|1231|5355|411|2002|5|8|16|3|2002|411|5355|Friday|2002Q3|N|Y|N|2452488|2452699|2452138|2452412|N|N|N|N|N| +2452504|AAAAAAAAIBMGFCAA|2002-08-17|1231|5355|411|2002|6|8|17|3|2002|411|5355|Saturday|2002Q3|N|Y|N|2452488|2452699|2452139|2452413|N|N|N|N|N| +2452505|AAAAAAAAJBMGFCAA|2002-08-18|1231|5355|411|2002|0|8|18|3|2002|411|5355|Sunday|2002Q3|N|N|N|2452488|2452699|2452140|2452414|N|N|N|N|N| +2452506|AAAAAAAAKBMGFCAA|2002-08-19|1231|5355|411|2002|1|8|19|3|2002|411|5355|Monday|2002Q3|N|N|N|2452488|2452699|2452141|2452415|N|N|N|N|N| +2452507|AAAAAAAALBMGFCAA|2002-08-20|1231|5356|411|2002|2|8|20|3|2002|411|5356|Tuesday|2002Q3|N|N|N|2452488|2452699|2452142|2452416|N|N|N|N|N| +2452508|AAAAAAAAMBMGFCAA|2002-08-21|1231|5356|411|2002|3|8|21|3|2002|411|5356|Wednesday|2002Q3|N|N|N|2452488|2452699|2452143|2452417|N|N|N|N|N| +2452509|AAAAAAAANBMGFCAA|2002-08-22|1231|5356|411|2002|4|8|22|3|2002|411|5356|Thursday|2002Q3|N|N|N|2452488|2452699|2452144|2452418|N|N|N|N|N| +2452510|AAAAAAAAOBMGFCAA|2002-08-23|1231|5356|411|2002|5|8|23|3|2002|411|5356|Friday|2002Q3|N|Y|N|2452488|2452699|2452145|2452419|N|N|N|N|N| +2452511|AAAAAAAAPBMGFCAA|2002-08-24|1231|5356|411|2002|6|8|24|3|2002|411|5356|Saturday|2002Q3|N|Y|N|2452488|2452699|2452146|2452420|N|N|N|N|N| +2452512|AAAAAAAAACMGFCAA|2002-08-25|1231|5356|411|2002|0|8|25|3|2002|411|5356|Sunday|2002Q3|N|N|N|2452488|2452699|2452147|2452421|N|N|N|N|N| +2452513|AAAAAAAABCMGFCAA|2002-08-26|1231|5356|411|2002|1|8|26|3|2002|411|5356|Monday|2002Q3|N|N|N|2452488|2452699|2452148|2452422|N|N|N|N|N| +2452514|AAAAAAAACCMGFCAA|2002-08-27|1231|5357|411|2002|2|8|27|3|2002|411|5357|Tuesday|2002Q3|N|N|N|2452488|2452699|2452149|2452423|N|N|N|N|N| +2452515|AAAAAAAADCMGFCAA|2002-08-28|1231|5357|411|2002|3|8|28|3|2002|411|5357|Wednesday|2002Q3|N|N|N|2452488|2452699|2452150|2452424|N|N|N|N|N| +2452516|AAAAAAAAECMGFCAA|2002-08-29|1231|5357|411|2002|4|8|29|3|2002|411|5357|Thursday|2002Q3|N|N|N|2452488|2452699|2452151|2452425|N|N|N|N|N| +2452517|AAAAAAAAFCMGFCAA|2002-08-30|1231|5357|411|2002|5|8|30|3|2002|411|5357|Friday|2002Q3|N|Y|N|2452488|2452699|2452152|2452426|N|N|N|N|N| +2452518|AAAAAAAAGCMGFCAA|2002-08-31|1231|5357|411|2002|6|8|31|3|2002|411|5357|Saturday|2002Q3|N|Y|N|2452488|2452699|2452153|2452427|N|N|N|N|N| +2452519|AAAAAAAAHCMGFCAA|2002-09-01|1232|5357|412|2002|0|9|1|3|2002|412|5357|Sunday|2002Q3|N|N|N|2452519|2452761|2452154|2452428|N|N|N|N|N| +2452520|AAAAAAAAICMGFCAA|2002-09-02|1232|5357|412|2002|1|9|2|3|2002|412|5357|Monday|2002Q3|N|N|N|2452519|2452761|2452155|2452429|N|N|N|N|N| +2452521|AAAAAAAAJCMGFCAA|2002-09-03|1232|5358|412|2002|2|9|3|3|2002|412|5358|Tuesday|2002Q3|N|N|N|2452519|2452761|2452156|2452430|N|N|N|N|N| +2452522|AAAAAAAAKCMGFCAA|2002-09-04|1232|5358|412|2002|3|9|4|3|2002|412|5358|Wednesday|2002Q3|N|N|N|2452519|2452761|2452157|2452431|N|N|N|N|N| +2452523|AAAAAAAALCMGFCAA|2002-09-05|1232|5358|412|2002|4|9|5|3|2002|412|5358|Thursday|2002Q3|N|N|N|2452519|2452761|2452158|2452432|N|N|N|N|N| +2452524|AAAAAAAAMCMGFCAA|2002-09-06|1232|5358|412|2002|5|9|6|3|2002|412|5358|Friday|2002Q3|N|Y|N|2452519|2452761|2452159|2452433|N|N|N|N|N| +2452525|AAAAAAAANCMGFCAA|2002-09-07|1232|5358|412|2002|6|9|7|3|2002|412|5358|Saturday|2002Q3|N|Y|N|2452519|2452761|2452160|2452434|N|N|N|N|N| +2452526|AAAAAAAAOCMGFCAA|2002-09-08|1232|5358|412|2002|0|9|8|3|2002|412|5358|Sunday|2002Q3|N|N|N|2452519|2452761|2452161|2452435|N|N|N|N|N| +2452527|AAAAAAAAPCMGFCAA|2002-09-09|1232|5358|412|2002|1|9|9|3|2002|412|5358|Monday|2002Q3|N|N|N|2452519|2452761|2452162|2452436|N|N|N|N|N| +2452528|AAAAAAAAADMGFCAA|2002-09-10|1232|5359|412|2002|2|9|10|3|2002|412|5359|Tuesday|2002Q3|N|N|N|2452519|2452761|2452163|2452437|N|N|N|N|N| +2452529|AAAAAAAABDMGFCAA|2002-09-11|1232|5359|412|2002|3|9|11|3|2002|412|5359|Wednesday|2002Q3|N|N|N|2452519|2452761|2452164|2452438|N|N|N|N|N| +2452530|AAAAAAAACDMGFCAA|2002-09-12|1232|5359|412|2002|4|9|12|3|2002|412|5359|Thursday|2002Q3|N|N|N|2452519|2452761|2452165|2452439|N|N|N|N|N| +2452531|AAAAAAAADDMGFCAA|2002-09-13|1232|5359|412|2002|5|9|13|3|2002|412|5359|Friday|2002Q3|N|Y|N|2452519|2452761|2452166|2452440|N|N|N|N|N| +2452532|AAAAAAAAEDMGFCAA|2002-09-14|1232|5359|412|2002|6|9|14|3|2002|412|5359|Saturday|2002Q3|N|Y|N|2452519|2452761|2452167|2452441|N|N|N|N|N| +2452533|AAAAAAAAFDMGFCAA|2002-09-15|1232|5359|412|2002|0|9|15|3|2002|412|5359|Sunday|2002Q3|N|N|N|2452519|2452761|2452168|2452442|N|N|N|N|N| +2452534|AAAAAAAAGDMGFCAA|2002-09-16|1232|5359|412|2002|1|9|16|3|2002|412|5359|Monday|2002Q3|N|N|N|2452519|2452761|2452169|2452443|N|N|N|N|N| +2452535|AAAAAAAAHDMGFCAA|2002-09-17|1232|5360|412|2002|2|9|17|3|2002|412|5360|Tuesday|2002Q3|N|N|N|2452519|2452761|2452170|2452444|N|N|N|N|N| +2452536|AAAAAAAAIDMGFCAA|2002-09-18|1232|5360|412|2002|3|9|18|3|2002|412|5360|Wednesday|2002Q3|N|N|N|2452519|2452761|2452171|2452445|N|N|N|N|N| +2452537|AAAAAAAAJDMGFCAA|2002-09-19|1232|5360|412|2002|4|9|19|3|2002|412|5360|Thursday|2002Q3|N|N|N|2452519|2452761|2452172|2452446|N|N|N|N|N| +2452538|AAAAAAAAKDMGFCAA|2002-09-20|1232|5360|412|2002|5|9|20|3|2002|412|5360|Friday|2002Q3|N|Y|N|2452519|2452761|2452173|2452447|N|N|N|N|N| +2452539|AAAAAAAALDMGFCAA|2002-09-21|1232|5360|412|2002|6|9|21|3|2002|412|5360|Saturday|2002Q3|N|Y|N|2452519|2452761|2452174|2452448|N|N|N|N|N| +2452540|AAAAAAAAMDMGFCAA|2002-09-22|1232|5360|412|2002|0|9|22|3|2002|412|5360|Sunday|2002Q3|N|N|N|2452519|2452761|2452175|2452449|N|N|N|N|N| +2452541|AAAAAAAANDMGFCAA|2002-09-23|1232|5360|412|2002|1|9|23|3|2002|412|5360|Monday|2002Q3|N|N|N|2452519|2452761|2452176|2452450|N|N|N|N|N| +2452542|AAAAAAAAODMGFCAA|2002-09-24|1232|5361|412|2002|2|9|24|3|2002|412|5361|Tuesday|2002Q3|N|N|N|2452519|2452761|2452177|2452451|N|N|N|N|N| +2452543|AAAAAAAAPDMGFCAA|2002-09-25|1232|5361|412|2002|3|9|25|3|2002|412|5361|Wednesday|2002Q3|N|N|N|2452519|2452761|2452178|2452452|N|N|N|N|N| +2452544|AAAAAAAAAEMGFCAA|2002-09-26|1232|5361|412|2002|4|9|26|3|2002|412|5361|Thursday|2002Q3|N|N|N|2452519|2452761|2452179|2452453|N|N|N|N|N| +2452545|AAAAAAAABEMGFCAA|2002-09-27|1232|5361|412|2002|5|9|27|3|2002|412|5361|Friday|2002Q3|N|Y|N|2452519|2452761|2452180|2452454|N|N|N|N|N| +2452546|AAAAAAAACEMGFCAA|2002-09-28|1232|5361|412|2002|6|9|28|3|2002|412|5361|Saturday|2002Q3|N|Y|N|2452519|2452761|2452181|2452455|N|N|N|N|N| +2452547|AAAAAAAADEMGFCAA|2002-09-29|1232|5361|412|2002|0|9|29|3|2002|412|5361|Sunday|2002Q3|N|N|N|2452519|2452761|2452182|2452456|N|N|N|N|N| +2452548|AAAAAAAAEEMGFCAA|2002-09-30|1232|5361|412|2002|1|9|30|3|2002|412|5361|Monday|2002Q3|N|N|N|2452519|2452761|2452183|2452457|N|N|N|N|N| +2452549|AAAAAAAAFEMGFCAA|2002-10-01|1233|5362|412|2002|2|10|1|3|2002|412|5362|Tuesday|2002Q3|N|N|N|2452549|2452821|2452184|2452457|N|N|N|N|N| +2452550|AAAAAAAAGEMGFCAA|2002-10-02|1233|5362|412|2002|3|10|2|4|2002|412|5362|Wednesday|2002Q4|N|N|N|2452549|2452821|2452185|2452458|N|N|N|N|N| +2452551|AAAAAAAAHEMGFCAA|2002-10-03|1233|5362|412|2002|4|10|3|4|2002|412|5362|Thursday|2002Q4|N|N|N|2452549|2452821|2452186|2452459|N|N|N|N|N| +2452552|AAAAAAAAIEMGFCAA|2002-10-04|1233|5362|412|2002|5|10|4|4|2002|412|5362|Friday|2002Q4|N|Y|N|2452549|2452821|2452187|2452460|N|N|N|N|N| +2452553|AAAAAAAAJEMGFCAA|2002-10-05|1233|5362|412|2002|6|10|5|4|2002|412|5362|Saturday|2002Q4|N|Y|N|2452549|2452821|2452188|2452461|N|N|N|N|N| +2452554|AAAAAAAAKEMGFCAA|2002-10-06|1233|5362|412|2002|0|10|6|4|2002|412|5362|Sunday|2002Q4|N|N|N|2452549|2452821|2452189|2452462|N|N|N|N|N| +2452555|AAAAAAAALEMGFCAA|2002-10-07|1233|5362|412|2002|1|10|7|4|2002|412|5362|Monday|2002Q4|N|N|N|2452549|2452821|2452190|2452463|N|N|N|N|N| +2452556|AAAAAAAAMEMGFCAA|2002-10-08|1233|5363|412|2002|2|10|8|4|2002|412|5363|Tuesday|2002Q4|N|N|N|2452549|2452821|2452191|2452464|N|N|N|N|N| +2452557|AAAAAAAANEMGFCAA|2002-10-09|1233|5363|412|2002|3|10|9|4|2002|412|5363|Wednesday|2002Q4|N|N|N|2452549|2452821|2452192|2452465|N|N|N|N|N| +2452558|AAAAAAAAOEMGFCAA|2002-10-10|1233|5363|412|2002|4|10|10|4|2002|412|5363|Thursday|2002Q4|N|N|N|2452549|2452821|2452193|2452466|N|N|N|N|N| +2452559|AAAAAAAAPEMGFCAA|2002-10-11|1233|5363|412|2002|5|10|11|4|2002|412|5363|Friday|2002Q4|N|Y|N|2452549|2452821|2452194|2452467|N|N|N|N|N| +2452560|AAAAAAAAAFMGFCAA|2002-10-12|1233|5363|412|2002|6|10|12|4|2002|412|5363|Saturday|2002Q4|N|Y|N|2452549|2452821|2452195|2452468|N|N|N|N|N| +2452561|AAAAAAAABFMGFCAA|2002-10-13|1233|5363|412|2002|0|10|13|4|2002|412|5363|Sunday|2002Q4|N|N|N|2452549|2452821|2452196|2452469|N|N|N|N|N| +2452562|AAAAAAAACFMGFCAA|2002-10-14|1233|5363|412|2002|1|10|14|4|2002|412|5363|Monday|2002Q4|N|N|N|2452549|2452821|2452197|2452470|N|N|N|N|N| +2452563|AAAAAAAADFMGFCAA|2002-10-15|1233|5364|412|2002|2|10|15|4|2002|412|5364|Tuesday|2002Q4|N|N|N|2452549|2452821|2452198|2452471|N|N|N|N|N| +2452564|AAAAAAAAEFMGFCAA|2002-10-16|1233|5364|412|2002|3|10|16|4|2002|412|5364|Wednesday|2002Q4|N|N|N|2452549|2452821|2452199|2452472|N|N|N|N|N| +2452565|AAAAAAAAFFMGFCAA|2002-10-17|1233|5364|412|2002|4|10|17|4|2002|412|5364|Thursday|2002Q4|N|N|N|2452549|2452821|2452200|2452473|N|N|N|N|N| +2452566|AAAAAAAAGFMGFCAA|2002-10-18|1233|5364|412|2002|5|10|18|4|2002|412|5364|Friday|2002Q4|N|Y|N|2452549|2452821|2452201|2452474|N|N|N|N|N| +2452567|AAAAAAAAHFMGFCAA|2002-10-19|1233|5364|412|2002|6|10|19|4|2002|412|5364|Saturday|2002Q4|N|Y|N|2452549|2452821|2452202|2452475|N|N|N|N|N| +2452568|AAAAAAAAIFMGFCAA|2002-10-20|1233|5364|412|2002|0|10|20|4|2002|412|5364|Sunday|2002Q4|N|N|N|2452549|2452821|2452203|2452476|N|N|N|N|N| +2452569|AAAAAAAAJFMGFCAA|2002-10-21|1233|5364|412|2002|1|10|21|4|2002|412|5364|Monday|2002Q4|N|N|N|2452549|2452821|2452204|2452477|N|N|N|N|N| +2452570|AAAAAAAAKFMGFCAA|2002-10-22|1233|5365|412|2002|2|10|22|4|2002|412|5365|Tuesday|2002Q4|N|N|N|2452549|2452821|2452205|2452478|N|N|N|N|N| +2452571|AAAAAAAALFMGFCAA|2002-10-23|1233|5365|412|2002|3|10|23|4|2002|412|5365|Wednesday|2002Q4|N|N|N|2452549|2452821|2452206|2452479|N|N|N|N|N| +2452572|AAAAAAAAMFMGFCAA|2002-10-24|1233|5365|412|2002|4|10|24|4|2002|412|5365|Thursday|2002Q4|N|N|N|2452549|2452821|2452207|2452480|N|N|N|N|N| +2452573|AAAAAAAANFMGFCAA|2002-10-25|1233|5365|412|2002|5|10|25|4|2002|412|5365|Friday|2002Q4|N|Y|N|2452549|2452821|2452208|2452481|N|N|N|N|N| +2452574|AAAAAAAAOFMGFCAA|2002-10-26|1233|5365|412|2002|6|10|26|4|2002|412|5365|Saturday|2002Q4|N|Y|N|2452549|2452821|2452209|2452482|N|N|N|N|N| +2452575|AAAAAAAAPFMGFCAA|2002-10-27|1233|5365|412|2002|0|10|27|4|2002|412|5365|Sunday|2002Q4|N|N|N|2452549|2452821|2452210|2452483|N|N|N|N|N| +2452576|AAAAAAAAAGMGFCAA|2002-10-28|1233|5365|412|2002|1|10|28|4|2002|412|5365|Monday|2002Q4|N|N|N|2452549|2452821|2452211|2452484|N|N|N|N|N| +2452577|AAAAAAAABGMGFCAA|2002-10-29|1233|5366|412|2002|2|10|29|4|2002|412|5366|Tuesday|2002Q4|N|N|N|2452549|2452821|2452212|2452485|N|N|N|N|N| +2452578|AAAAAAAACGMGFCAA|2002-10-30|1233|5366|412|2002|3|10|30|4|2002|412|5366|Wednesday|2002Q4|N|N|N|2452549|2452821|2452213|2452486|N|N|N|N|N| +2452579|AAAAAAAADGMGFCAA|2002-10-31|1233|5366|412|2002|4|10|31|4|2002|412|5366|Thursday|2002Q4|N|N|N|2452549|2452821|2452214|2452487|N|N|N|N|N| +2452580|AAAAAAAAEGMGFCAA|2002-11-01|1234|5366|412|2002|5|11|1|4|2002|412|5366|Friday|2002Q4|N|Y|N|2452580|2452883|2452215|2452488|N|N|N|N|N| +2452581|AAAAAAAAFGMGFCAA|2002-11-02|1234|5366|412|2002|6|11|2|4|2002|412|5366|Saturday|2002Q4|N|Y|N|2452580|2452883|2452216|2452489|N|N|N|N|N| +2452582|AAAAAAAAGGMGFCAA|2002-11-03|1234|5366|412|2002|0|11|3|4|2002|412|5366|Sunday|2002Q4|N|N|N|2452580|2452883|2452217|2452490|N|N|N|N|N| +2452583|AAAAAAAAHGMGFCAA|2002-11-04|1234|5366|412|2002|1|11|4|4|2002|412|5366|Monday|2002Q4|N|N|N|2452580|2452883|2452218|2452491|N|N|N|N|N| +2452584|AAAAAAAAIGMGFCAA|2002-11-05|1234|5367|412|2002|2|11|5|4|2002|412|5367|Tuesday|2002Q4|N|N|N|2452580|2452883|2452219|2452492|N|N|N|N|N| +2452585|AAAAAAAAJGMGFCAA|2002-11-06|1234|5367|412|2002|3|11|6|4|2002|412|5367|Wednesday|2002Q4|N|N|N|2452580|2452883|2452220|2452493|N|N|N|N|N| +2452586|AAAAAAAAKGMGFCAA|2002-11-07|1234|5367|412|2002|4|11|7|4|2002|412|5367|Thursday|2002Q4|N|N|N|2452580|2452883|2452221|2452494|N|N|N|N|N| +2452587|AAAAAAAALGMGFCAA|2002-11-08|1234|5367|412|2002|5|11|8|4|2002|412|5367|Friday|2002Q4|N|Y|N|2452580|2452883|2452222|2452495|N|N|N|N|N| +2452588|AAAAAAAAMGMGFCAA|2002-11-09|1234|5367|412|2002|6|11|9|4|2002|412|5367|Saturday|2002Q4|N|Y|N|2452580|2452883|2452223|2452496|N|N|N|N|N| +2452589|AAAAAAAANGMGFCAA|2002-11-10|1234|5367|412|2002|0|11|10|4|2002|412|5367|Sunday|2002Q4|N|N|N|2452580|2452883|2452224|2452497|N|N|N|N|N| +2452590|AAAAAAAAOGMGFCAA|2002-11-11|1234|5367|412|2002|1|11|11|4|2002|412|5367|Monday|2002Q4|N|N|N|2452580|2452883|2452225|2452498|N|N|N|N|N| +2452591|AAAAAAAAPGMGFCAA|2002-11-12|1234|5368|412|2002|2|11|12|4|2002|412|5368|Tuesday|2002Q4|N|N|N|2452580|2452883|2452226|2452499|N|N|N|N|N| +2452592|AAAAAAAAAHMGFCAA|2002-11-13|1234|5368|412|2002|3|11|13|4|2002|412|5368|Wednesday|2002Q4|N|N|N|2452580|2452883|2452227|2452500|N|N|N|N|N| +2452593|AAAAAAAABHMGFCAA|2002-11-14|1234|5368|412|2002|4|11|14|4|2002|412|5368|Thursday|2002Q4|N|N|N|2452580|2452883|2452228|2452501|N|N|N|N|N| +2452594|AAAAAAAACHMGFCAA|2002-11-15|1234|5368|412|2002|5|11|15|4|2002|412|5368|Friday|2002Q4|N|Y|N|2452580|2452883|2452229|2452502|N|N|N|N|N| +2452595|AAAAAAAADHMGFCAA|2002-11-16|1234|5368|412|2002|6|11|16|4|2002|412|5368|Saturday|2002Q4|N|Y|N|2452580|2452883|2452230|2452503|N|N|N|N|N| +2452596|AAAAAAAAEHMGFCAA|2002-11-17|1234|5368|412|2002|0|11|17|4|2002|412|5368|Sunday|2002Q4|N|N|N|2452580|2452883|2452231|2452504|N|N|N|N|N| +2452597|AAAAAAAAFHMGFCAA|2002-11-18|1234|5368|412|2002|1|11|18|4|2002|412|5368|Monday|2002Q4|N|N|N|2452580|2452883|2452232|2452505|N|N|N|N|N| +2452598|AAAAAAAAGHMGFCAA|2002-11-19|1234|5369|412|2002|2|11|19|4|2002|412|5369|Tuesday|2002Q4|N|N|N|2452580|2452883|2452233|2452506|N|N|N|N|N| +2452599|AAAAAAAAHHMGFCAA|2002-11-20|1234|5369|412|2002|3|11|20|4|2002|412|5369|Wednesday|2002Q4|N|N|N|2452580|2452883|2452234|2452507|N|N|N|N|N| +2452600|AAAAAAAAIHMGFCAA|2002-11-21|1234|5369|412|2002|4|11|21|4|2002|412|5369|Thursday|2002Q4|N|N|N|2452580|2452883|2452235|2452508|N|N|N|N|N| +2452601|AAAAAAAAJHMGFCAA|2002-11-22|1234|5369|412|2002|5|11|22|4|2002|412|5369|Friday|2002Q4|N|Y|N|2452580|2452883|2452236|2452509|N|N|N|N|N| +2452602|AAAAAAAAKHMGFCAA|2002-11-23|1234|5369|412|2002|6|11|23|4|2002|412|5369|Saturday|2002Q4|N|Y|N|2452580|2452883|2452237|2452510|N|N|N|N|N| +2452603|AAAAAAAALHMGFCAA|2002-11-24|1234|5369|412|2002|0|11|24|4|2002|412|5369|Sunday|2002Q4|N|N|N|2452580|2452883|2452238|2452511|N|N|N|N|N| +2452604|AAAAAAAAMHMGFCAA|2002-11-25|1234|5369|412|2002|1|11|25|4|2002|412|5369|Monday|2002Q4|N|N|N|2452580|2452883|2452239|2452512|N|N|N|N|N| +2452605|AAAAAAAANHMGFCAA|2002-11-26|1234|5370|412|2002|2|11|26|4|2002|412|5370|Tuesday|2002Q4|N|N|N|2452580|2452883|2452240|2452513|N|N|N|N|N| +2452606|AAAAAAAAOHMGFCAA|2002-11-27|1234|5370|412|2002|3|11|27|4|2002|412|5370|Wednesday|2002Q4|N|N|N|2452580|2452883|2452241|2452514|N|N|N|N|N| +2452607|AAAAAAAAPHMGFCAA|2002-11-28|1234|5370|412|2002|4|11|28|4|2002|412|5370|Thursday|2002Q4|N|N|N|2452580|2452883|2452242|2452515|N|N|N|N|N| +2452608|AAAAAAAAAIMGFCAA|2002-11-29|1234|5370|412|2002|5|11|29|4|2002|412|5370|Friday|2002Q4|N|Y|N|2452580|2452883|2452243|2452516|N|N|N|N|N| +2452609|AAAAAAAABIMGFCAA|2002-11-30|1234|5370|412|2002|6|11|30|4|2002|412|5370|Saturday|2002Q4|N|Y|N|2452580|2452883|2452244|2452517|N|N|N|N|N| +2452610|AAAAAAAACIMGFCAA|2002-12-01|1235|5370|413|2002|0|12|1|4|2002|413|5370|Sunday|2002Q4|N|N|N|2452610|2452943|2452245|2452518|N|N|N|N|N| +2452611|AAAAAAAADIMGFCAA|2002-12-02|1235|5370|413|2002|1|12|2|4|2002|413|5370|Monday|2002Q4|N|N|N|2452610|2452943|2452246|2452519|N|N|N|N|N| +2452612|AAAAAAAAEIMGFCAA|2002-12-03|1235|5371|413|2002|2|12|3|4|2002|413|5371|Tuesday|2002Q4|N|N|N|2452610|2452943|2452247|2452520|N|N|N|N|N| +2452613|AAAAAAAAFIMGFCAA|2002-12-04|1235|5371|413|2002|3|12|4|4|2002|413|5371|Wednesday|2002Q4|N|N|N|2452610|2452943|2452248|2452521|N|N|N|N|N| +2452614|AAAAAAAAGIMGFCAA|2002-12-05|1235|5371|413|2002|4|12|5|4|2002|413|5371|Thursday|2002Q4|N|N|N|2452610|2452943|2452249|2452522|N|N|N|N|N| +2452615|AAAAAAAAHIMGFCAA|2002-12-06|1235|5371|413|2002|5|12|6|4|2002|413|5371|Friday|2002Q4|N|Y|N|2452610|2452943|2452250|2452523|N|N|N|N|N| +2452616|AAAAAAAAIIMGFCAA|2002-12-07|1235|5371|413|2002|6|12|7|4|2002|413|5371|Saturday|2002Q4|N|Y|N|2452610|2452943|2452251|2452524|N|N|N|N|N| +2452617|AAAAAAAAJIMGFCAA|2002-12-08|1235|5371|413|2002|0|12|8|4|2002|413|5371|Sunday|2002Q4|N|N|N|2452610|2452943|2452252|2452525|N|N|N|N|N| +2452618|AAAAAAAAKIMGFCAA|2002-12-09|1235|5371|413|2002|1|12|9|4|2002|413|5371|Monday|2002Q4|N|N|N|2452610|2452943|2452253|2452526|N|N|N|N|N| +2452619|AAAAAAAALIMGFCAA|2002-12-10|1235|5372|413|2002|2|12|10|4|2002|413|5372|Tuesday|2002Q4|N|N|N|2452610|2452943|2452254|2452527|N|N|N|N|N| +2452620|AAAAAAAAMIMGFCAA|2002-12-11|1235|5372|413|2002|3|12|11|4|2002|413|5372|Wednesday|2002Q4|N|N|N|2452610|2452943|2452255|2452528|N|N|N|N|N| +2452621|AAAAAAAANIMGFCAA|2002-12-12|1235|5372|413|2002|4|12|12|4|2002|413|5372|Thursday|2002Q4|N|N|N|2452610|2452943|2452256|2452529|N|N|N|N|N| +2452622|AAAAAAAAOIMGFCAA|2002-12-13|1235|5372|413|2002|5|12|13|4|2002|413|5372|Friday|2002Q4|N|Y|N|2452610|2452943|2452257|2452530|N|N|N|N|N| +2452623|AAAAAAAAPIMGFCAA|2002-12-14|1235|5372|413|2002|6|12|14|4|2002|413|5372|Saturday|2002Q4|N|Y|N|2452610|2452943|2452258|2452531|N|N|N|N|N| +2452624|AAAAAAAAAJMGFCAA|2002-12-15|1235|5372|413|2002|0|12|15|4|2002|413|5372|Sunday|2002Q4|N|N|N|2452610|2452943|2452259|2452532|N|N|N|N|N| +2452625|AAAAAAAABJMGFCAA|2002-12-16|1235|5372|413|2002|1|12|16|4|2002|413|5372|Monday|2002Q4|N|N|N|2452610|2452943|2452260|2452533|N|N|N|N|N| +2452626|AAAAAAAACJMGFCAA|2002-12-17|1235|5373|413|2002|2|12|17|4|2002|413|5373|Tuesday|2002Q4|N|N|N|2452610|2452943|2452261|2452534|N|N|N|N|N| +2452627|AAAAAAAADJMGFCAA|2002-12-18|1235|5373|413|2002|3|12|18|4|2002|413|5373|Wednesday|2002Q4|N|N|N|2452610|2452943|2452262|2452535|N|N|N|N|N| +2452628|AAAAAAAAEJMGFCAA|2002-12-19|1235|5373|413|2002|4|12|19|4|2002|413|5373|Thursday|2002Q4|N|N|N|2452610|2452943|2452263|2452536|N|N|N|N|N| +2452629|AAAAAAAAFJMGFCAA|2002-12-20|1235|5373|413|2002|5|12|20|4|2002|413|5373|Friday|2002Q4|N|Y|N|2452610|2452943|2452264|2452537|N|N|N|N|N| +2452630|AAAAAAAAGJMGFCAA|2002-12-21|1235|5373|413|2002|6|12|21|4|2002|413|5373|Saturday|2002Q4|N|Y|N|2452610|2452943|2452265|2452538|N|N|N|N|N| +2452631|AAAAAAAAHJMGFCAA|2002-12-22|1235|5373|413|2002|0|12|22|4|2002|413|5373|Sunday|2002Q4|N|N|N|2452610|2452943|2452266|2452539|N|N|N|N|N| +2452632|AAAAAAAAIJMGFCAA|2002-12-23|1235|5373|413|2002|1|12|23|4|2002|413|5373|Monday|2002Q4|N|N|N|2452610|2452943|2452267|2452540|N|N|N|N|N| +2452633|AAAAAAAAJJMGFCAA|2002-12-24|1235|5374|413|2002|2|12|24|4|2002|413|5374|Tuesday|2002Q4|N|N|N|2452610|2452943|2452268|2452541|N|N|N|N|N| +2452634|AAAAAAAAKJMGFCAA|2002-12-25|1235|5374|413|2002|3|12|25|4|2002|413|5374|Wednesday|2002Q4|N|N|N|2452610|2452943|2452269|2452542|N|N|N|N|N| +2452635|AAAAAAAALJMGFCAA|2002-12-26|1235|5374|413|2002|4|12|26|4|2002|413|5374|Thursday|2002Q4|Y|N|N|2452610|2452943|2452270|2452543|N|N|N|N|N| +2452636|AAAAAAAAMJMGFCAA|2002-12-27|1235|5374|413|2002|5|12|27|4|2002|413|5374|Friday|2002Q4|N|Y|Y|2452610|2452943|2452271|2452544|N|N|N|N|N| +2452637|AAAAAAAANJMGFCAA|2002-12-28|1235|5374|413|2002|6|12|28|4|2002|413|5374|Saturday|2002Q4|N|Y|N|2452610|2452943|2452272|2452545|N|N|N|N|N| +2452638|AAAAAAAAOJMGFCAA|2002-12-29|1235|5374|413|2002|0|12|29|4|2002|413|5374|Sunday|2002Q4|N|N|N|2452610|2452943|2452273|2452546|N|N|N|N|N| +2452639|AAAAAAAAPJMGFCAA|2002-12-30|1235|5374|413|2002|1|12|30|4|2002|413|5374|Monday|2002Q4|N|N|N|2452610|2452943|2452274|2452547|N|N|N|N|N| +2452640|AAAAAAAAAKMGFCAA|2002-12-31|1235|5375|413|2002|2|12|31|4|2002|413|5375|Tuesday|2002Q4|N|N|N|2452610|2452943|2452275|2452548|N|N|N|N|N| +2452641|AAAAAAAABKMGFCAA|2003-01-01|1236|5375|413|2003|3|1|1|1|2003|413|5375|Wednesday|2003Q1|Y|N|N|2452641|2452640|2452276|2452549|N|N|Y|Y|Y| +2452642|AAAAAAAACKMGFCAA|2003-01-02|1236|5375|413|2003|4|1|2|1|2003|413|5375|Thursday|2003Q1|N|N|Y|2452641|2452640|2452277|2452550|N|N|Y|Y|Y| +2452643|AAAAAAAADKMGFCAA|2003-01-03|1236|5375|413|2003|5|1|3|1|2003|413|5375|Friday|2003Q1|N|Y|N|2452641|2452640|2452278|2452551|N|N|Y|Y|Y| +2452644|AAAAAAAAEKMGFCAA|2003-01-04|1236|5375|413|2003|6|1|4|1|2003|413|5375|Saturday|2003Q1|N|Y|N|2452641|2452640|2452279|2452552|N|N|Y|Y|Y| +2452645|AAAAAAAAFKMGFCAA|2003-01-05|1236|5375|413|2003|0|1|5|1|2003|413|5375|Sunday|2003Q1|N|N|N|2452641|2452640|2452280|2452553|N|N|Y|Y|Y| +2452646|AAAAAAAAGKMGFCAA|2003-01-06|1236|5375|413|2003|1|1|6|1|2003|413|5375|Monday|2003Q1|N|N|N|2452641|2452640|2452281|2452554|N|N|Y|Y|Y| +2452647|AAAAAAAAHKMGFCAA|2003-01-07|1236|5376|413|2003|2|1|7|1|2003|413|5376|Tuesday|2003Q1|N|N|N|2452641|2452640|2452282|2452555|N|N|Y|Y|Y| +2452648|AAAAAAAAIKMGFCAA|2003-01-08|1236|5376|413|2003|3|1|8|1|2003|413|5376|Wednesday|2003Q1|N|N|N|2452641|2452640|2452283|2452556|N|N|Y|Y|Y| +2452649|AAAAAAAAJKMGFCAA|2003-01-09|1236|5376|413|2003|4|1|9|1|2003|413|5376|Thursday|2003Q1|N|N|N|2452641|2452640|2452284|2452557|N|N|Y|Y|Y| +2452650|AAAAAAAAKKMGFCAA|2003-01-10|1236|5376|413|2003|5|1|10|1|2003|413|5376|Friday|2003Q1|N|Y|N|2452641|2452640|2452285|2452558|N|N|Y|Y|Y| +2452651|AAAAAAAALKMGFCAA|2003-01-11|1236|5376|413|2003|6|1|11|1|2003|413|5376|Saturday|2003Q1|N|Y|N|2452641|2452640|2452286|2452559|N|N|Y|Y|Y| +2452652|AAAAAAAAMKMGFCAA|2003-01-12|1236|5376|413|2003|0|1|12|1|2003|413|5376|Sunday|2003Q1|N|N|N|2452641|2452640|2452287|2452560|N|N|Y|Y|Y| +2452653|AAAAAAAANKMGFCAA|2003-01-13|1236|5376|413|2003|1|1|13|1|2003|413|5376|Monday|2003Q1|N|N|N|2452641|2452640|2452288|2452561|N|N|Y|Y|Y| +2452654|AAAAAAAAOKMGFCAA|2003-01-14|1236|5377|413|2003|2|1|14|1|2003|413|5377|Tuesday|2003Q1|N|N|N|2452641|2452640|2452289|2452562|N|N|Y|Y|Y| +2452655|AAAAAAAAPKMGFCAA|2003-01-15|1236|5377|413|2003|3|1|15|1|2003|413|5377|Wednesday|2003Q1|N|N|N|2452641|2452640|2452290|2452563|N|N|Y|Y|Y| +2452656|AAAAAAAAALMGFCAA|2003-01-16|1236|5377|413|2003|4|1|16|1|2003|413|5377|Thursday|2003Q1|N|N|N|2452641|2452640|2452291|2452564|N|N|Y|Y|Y| +2452657|AAAAAAAABLMGFCAA|2003-01-17|1236|5377|413|2003|5|1|17|1|2003|413|5377|Friday|2003Q1|N|Y|N|2452641|2452640|2452292|2452565|N|N|Y|Y|Y| +2452658|AAAAAAAACLMGFCAA|2003-01-18|1236|5377|413|2003|6|1|18|1|2003|413|5377|Saturday|2003Q1|N|Y|N|2452641|2452640|2452293|2452566|N|N|Y|Y|Y| +2452659|AAAAAAAADLMGFCAA|2003-01-19|1236|5377|413|2003|0|1|19|1|2003|413|5377|Sunday|2003Q1|N|N|N|2452641|2452640|2452294|2452567|N|N|Y|Y|Y| +2452660|AAAAAAAAELMGFCAA|2003-01-20|1236|5377|413|2003|1|1|20|1|2003|413|5377|Monday|2003Q1|N|N|N|2452641|2452640|2452295|2452568|N|N|Y|Y|Y| +2452661|AAAAAAAAFLMGFCAA|2003-01-21|1236|5378|413|2003|2|1|21|1|2003|413|5378|Tuesday|2003Q1|N|N|N|2452641|2452640|2452296|2452569|N|N|Y|Y|Y| +2452662|AAAAAAAAGLMGFCAA|2003-01-22|1236|5378|413|2003|3|1|22|1|2003|413|5378|Wednesday|2003Q1|N|N|N|2452641|2452640|2452297|2452570|N|N|Y|Y|Y| +2452663|AAAAAAAAHLMGFCAA|2003-01-23|1236|5378|413|2003|4|1|23|1|2003|413|5378|Thursday|2003Q1|N|N|N|2452641|2452640|2452298|2452571|N|N|Y|Y|Y| +2452664|AAAAAAAAILMGFCAA|2003-01-24|1236|5378|413|2003|5|1|24|1|2003|413|5378|Friday|2003Q1|N|Y|N|2452641|2452640|2452299|2452572|N|N|Y|Y|Y| +2452665|AAAAAAAAJLMGFCAA|2003-01-25|1236|5378|413|2003|6|1|25|1|2003|413|5378|Saturday|2003Q1|N|Y|N|2452641|2452640|2452300|2452573|N|N|Y|Y|Y| +2452666|AAAAAAAAKLMGFCAA|2003-01-26|1236|5378|413|2003|0|1|26|1|2003|413|5378|Sunday|2003Q1|N|N|N|2452641|2452640|2452301|2452574|N|N|Y|Y|Y| +2452667|AAAAAAAALLMGFCAA|2003-01-27|1236|5378|413|2003|1|1|27|1|2003|413|5378|Monday|2003Q1|N|N|N|2452641|2452640|2452302|2452575|N|N|Y|Y|Y| +2452668|AAAAAAAAMLMGFCAA|2003-01-28|1236|5379|413|2003|2|1|28|1|2003|413|5379|Tuesday|2003Q1|N|N|N|2452641|2452640|2452303|2452576|N|N|Y|Y|Y| +2452669|AAAAAAAANLMGFCAA|2003-01-29|1236|5379|413|2003|3|1|29|1|2003|413|5379|Wednesday|2003Q1|N|N|N|2452641|2452640|2452304|2452577|N|N|Y|Y|Y| +2452670|AAAAAAAAOLMGFCAA|2003-01-30|1236|5379|413|2003|4|1|30|1|2003|413|5379|Thursday|2003Q1|N|N|N|2452641|2452640|2452305|2452578|N|N|Y|Y|Y| +2452671|AAAAAAAAPLMGFCAA|2003-01-31|1236|5379|413|2003|5|1|31|1|2003|413|5379|Friday|2003Q1|N|Y|N|2452641|2452640|2452306|2452579|N|N|Y|Y|Y| +2452672|AAAAAAAAAMMGFCAA|2003-02-01|1237|5379|413|2003|6|2|1|1|2003|413|5379|Saturday|2003Q1|N|Y|N|2452672|2452702|2452307|2452580|N|N|N|Y|Y| +2452673|AAAAAAAABMMGFCAA|2003-02-02|1237|5379|413|2003|0|2|2|1|2003|413|5379|Sunday|2003Q1|N|N|N|2452672|2452702|2452308|2452581|N|N|N|Y|Y| +2452674|AAAAAAAACMMGFCAA|2003-02-03|1237|5379|413|2003|1|2|3|1|2003|413|5379|Monday|2003Q1|N|N|N|2452672|2452702|2452309|2452582|N|N|N|Y|Y| +2452675|AAAAAAAADMMGFCAA|2003-02-04|1237|5380|413|2003|2|2|4|1|2003|413|5380|Tuesday|2003Q1|N|N|N|2452672|2452702|2452310|2452583|N|N|N|Y|Y| +2452676|AAAAAAAAEMMGFCAA|2003-02-05|1237|5380|413|2003|3|2|5|1|2003|413|5380|Wednesday|2003Q1|N|N|N|2452672|2452702|2452311|2452584|N|N|N|Y|Y| +2452677|AAAAAAAAFMMGFCAA|2003-02-06|1237|5380|413|2003|4|2|6|1|2003|413|5380|Thursday|2003Q1|N|N|N|2452672|2452702|2452312|2452585|N|N|N|Y|Y| +2452678|AAAAAAAAGMMGFCAA|2003-02-07|1237|5380|413|2003|5|2|7|1|2003|413|5380|Friday|2003Q1|N|Y|N|2452672|2452702|2452313|2452586|N|N|N|Y|Y| +2452679|AAAAAAAAHMMGFCAA|2003-02-08|1237|5380|413|2003|6|2|8|1|2003|413|5380|Saturday|2003Q1|N|Y|N|2452672|2452702|2452314|2452587|N|N|N|Y|Y| +2452680|AAAAAAAAIMMGFCAA|2003-02-09|1237|5380|413|2003|0|2|9|1|2003|413|5380|Sunday|2003Q1|N|N|N|2452672|2452702|2452315|2452588|N|N|N|Y|Y| +2452681|AAAAAAAAJMMGFCAA|2003-02-10|1237|5380|413|2003|1|2|10|1|2003|413|5380|Monday|2003Q1|N|N|N|2452672|2452702|2452316|2452589|N|N|N|Y|Y| +2452682|AAAAAAAAKMMGFCAA|2003-02-11|1237|5381|413|2003|2|2|11|1|2003|413|5381|Tuesday|2003Q1|N|N|N|2452672|2452702|2452317|2452590|N|N|N|Y|Y| +2452683|AAAAAAAALMMGFCAA|2003-02-12|1237|5381|413|2003|3|2|12|1|2003|413|5381|Wednesday|2003Q1|N|N|N|2452672|2452702|2452318|2452591|N|N|N|Y|Y| +2452684|AAAAAAAAMMMGFCAA|2003-02-13|1237|5381|413|2003|4|2|13|1|2003|413|5381|Thursday|2003Q1|N|N|N|2452672|2452702|2452319|2452592|N|N|N|Y|Y| +2452685|AAAAAAAANMMGFCAA|2003-02-14|1237|5381|413|2003|5|2|14|1|2003|413|5381|Friday|2003Q1|N|Y|N|2452672|2452702|2452320|2452593|N|N|N|Y|Y| +2452686|AAAAAAAAOMMGFCAA|2003-02-15|1237|5381|413|2003|6|2|15|1|2003|413|5381|Saturday|2003Q1|N|Y|N|2452672|2452702|2452321|2452594|N|N|N|Y|Y| +2452687|AAAAAAAAPMMGFCAA|2003-02-16|1237|5381|413|2003|0|2|16|1|2003|413|5381|Sunday|2003Q1|N|N|N|2452672|2452702|2452322|2452595|N|N|N|Y|Y| +2452688|AAAAAAAAANMGFCAA|2003-02-17|1237|5381|413|2003|1|2|17|1|2003|413|5381|Monday|2003Q1|N|N|N|2452672|2452702|2452323|2452596|N|N|N|Y|Y| +2452689|AAAAAAAABNMGFCAA|2003-02-18|1237|5382|413|2003|2|2|18|1|2003|413|5382|Tuesday|2003Q1|N|N|N|2452672|2452702|2452324|2452597|N|N|N|Y|Y| +2452690|AAAAAAAACNMGFCAA|2003-02-19|1237|5382|413|2003|3|2|19|1|2003|413|5382|Wednesday|2003Q1|N|N|N|2452672|2452702|2452325|2452598|N|N|N|Y|Y| +2452691|AAAAAAAADNMGFCAA|2003-02-20|1237|5382|413|2003|4|2|20|1|2003|413|5382|Thursday|2003Q1|N|N|N|2452672|2452702|2452326|2452599|N|N|N|Y|Y| +2452692|AAAAAAAAENMGFCAA|2003-02-21|1237|5382|413|2003|5|2|21|1|2003|413|5382|Friday|2003Q1|N|Y|N|2452672|2452702|2452327|2452600|N|N|N|Y|Y| +2452693|AAAAAAAAFNMGFCAA|2003-02-22|1237|5382|413|2003|6|2|22|1|2003|413|5382|Saturday|2003Q1|N|Y|N|2452672|2452702|2452328|2452601|N|N|N|Y|Y| +2452694|AAAAAAAAGNMGFCAA|2003-02-23|1237|5382|413|2003|0|2|23|1|2003|413|5382|Sunday|2003Q1|N|N|N|2452672|2452702|2452329|2452602|N|N|N|Y|Y| +2452695|AAAAAAAAHNMGFCAA|2003-02-24|1237|5382|413|2003|1|2|24|1|2003|413|5382|Monday|2003Q1|N|N|N|2452672|2452702|2452330|2452603|N|N|N|Y|Y| +2452696|AAAAAAAAINMGFCAA|2003-02-25|1237|5383|413|2003|2|2|25|1|2003|413|5383|Tuesday|2003Q1|N|N|N|2452672|2452702|2452331|2452604|N|N|N|Y|Y| +2452697|AAAAAAAAJNMGFCAA|2003-02-26|1237|5383|413|2003|3|2|26|1|2003|413|5383|Wednesday|2003Q1|N|N|N|2452672|2452702|2452332|2452605|N|N|N|Y|Y| +2452698|AAAAAAAAKNMGFCAA|2003-02-27|1237|5383|413|2003|4|2|27|1|2003|413|5383|Thursday|2003Q1|N|N|N|2452672|2452702|2452333|2452606|N|N|N|Y|Y| +2452699|AAAAAAAALNMGFCAA|2003-02-28|1237|5383|413|2003|5|2|28|1|2003|413|5383|Friday|2003Q1|N|Y|N|2452672|2452702|2452334|2452607|N|N|N|Y|Y| +2452700|AAAAAAAAMNMGFCAA|2003-03-01|1238|5383|414|2003|6|3|1|1|2003|414|5383|Saturday|2003Q1|N|Y|N|2452700|2452758|2452335|2452608|N|N|N|Y|Y| +2452701|AAAAAAAANNMGFCAA|2003-03-02|1238|5383|414|2003|0|3|2|1|2003|414|5383|Sunday|2003Q1|N|N|N|2452700|2452758|2452336|2452609|N|N|N|Y|Y| +2452702|AAAAAAAAONMGFCAA|2003-03-03|1238|5383|414|2003|1|3|3|1|2003|414|5383|Monday|2003Q1|N|N|N|2452700|2452758|2452337|2452610|N|N|N|Y|Y| +2452703|AAAAAAAAPNMGFCAA|2003-03-04|1238|5384|414|2003|2|3|4|1|2003|414|5384|Tuesday|2003Q1|N|N|N|2452700|2452758|2452338|2452611|N|N|N|Y|Y| +2452704|AAAAAAAAAOMGFCAA|2003-03-05|1238|5384|414|2003|3|3|5|1|2003|414|5384|Wednesday|2003Q1|N|N|N|2452700|2452758|2452339|2452612|N|N|N|Y|Y| +2452705|AAAAAAAABOMGFCAA|2003-03-06|1238|5384|414|2003|4|3|6|1|2003|414|5384|Thursday|2003Q1|N|N|N|2452700|2452758|2452340|2452613|N|N|N|Y|Y| +2452706|AAAAAAAACOMGFCAA|2003-03-07|1238|5384|414|2003|5|3|7|1|2003|414|5384|Friday|2003Q1|N|Y|N|2452700|2452758|2452341|2452614|N|N|N|Y|Y| +2452707|AAAAAAAADOMGFCAA|2003-03-08|1238|5384|414|2003|6|3|8|1|2003|414|5384|Saturday|2003Q1|N|Y|N|2452700|2452758|2452342|2452615|N|N|N|Y|Y| +2452708|AAAAAAAAEOMGFCAA|2003-03-09|1238|5384|414|2003|0|3|9|1|2003|414|5384|Sunday|2003Q1|N|N|N|2452700|2452758|2452343|2452616|N|N|N|Y|Y| +2452709|AAAAAAAAFOMGFCAA|2003-03-10|1238|5384|414|2003|1|3|10|1|2003|414|5384|Monday|2003Q1|N|N|N|2452700|2452758|2452344|2452617|N|N|N|Y|Y| +2452710|AAAAAAAAGOMGFCAA|2003-03-11|1238|5385|414|2003|2|3|11|1|2003|414|5385|Tuesday|2003Q1|N|N|N|2452700|2452758|2452345|2452618|N|N|N|Y|Y| +2452711|AAAAAAAAHOMGFCAA|2003-03-12|1238|5385|414|2003|3|3|12|1|2003|414|5385|Wednesday|2003Q1|N|N|N|2452700|2452758|2452346|2452619|N|N|N|Y|Y| +2452712|AAAAAAAAIOMGFCAA|2003-03-13|1238|5385|414|2003|4|3|13|1|2003|414|5385|Thursday|2003Q1|N|N|N|2452700|2452758|2452347|2452620|N|N|N|Y|Y| +2452713|AAAAAAAAJOMGFCAA|2003-03-14|1238|5385|414|2003|5|3|14|1|2003|414|5385|Friday|2003Q1|N|Y|N|2452700|2452758|2452348|2452621|N|N|N|Y|Y| +2452714|AAAAAAAAKOMGFCAA|2003-03-15|1238|5385|414|2003|6|3|15|1|2003|414|5385|Saturday|2003Q1|N|Y|N|2452700|2452758|2452349|2452622|N|N|N|Y|Y| +2452715|AAAAAAAALOMGFCAA|2003-03-16|1238|5385|414|2003|0|3|16|1|2003|414|5385|Sunday|2003Q1|N|N|N|2452700|2452758|2452350|2452623|N|N|N|Y|Y| +2452716|AAAAAAAAMOMGFCAA|2003-03-17|1238|5385|414|2003|1|3|17|1|2003|414|5385|Monday|2003Q1|N|N|N|2452700|2452758|2452351|2452624|N|N|N|Y|Y| +2452717|AAAAAAAANOMGFCAA|2003-03-18|1238|5386|414|2003|2|3|18|1|2003|414|5386|Tuesday|2003Q1|N|N|N|2452700|2452758|2452352|2452625|N|N|N|Y|Y| +2452718|AAAAAAAAOOMGFCAA|2003-03-19|1238|5386|414|2003|3|3|19|1|2003|414|5386|Wednesday|2003Q1|N|N|N|2452700|2452758|2452353|2452626|N|N|N|Y|Y| +2452719|AAAAAAAAPOMGFCAA|2003-03-20|1238|5386|414|2003|4|3|20|1|2003|414|5386|Thursday|2003Q1|N|N|N|2452700|2452758|2452354|2452627|N|N|N|Y|Y| +2452720|AAAAAAAAAPMGFCAA|2003-03-21|1238|5386|414|2003|5|3|21|1|2003|414|5386|Friday|2003Q1|N|Y|N|2452700|2452758|2452355|2452628|N|N|N|Y|Y| +2452721|AAAAAAAABPMGFCAA|2003-03-22|1238|5386|414|2003|6|3|22|1|2003|414|5386|Saturday|2003Q1|N|Y|N|2452700|2452758|2452356|2452629|N|N|N|Y|Y| +2452722|AAAAAAAACPMGFCAA|2003-03-23|1238|5386|414|2003|0|3|23|1|2003|414|5386|Sunday|2003Q1|N|N|N|2452700|2452758|2452357|2452630|N|N|N|Y|Y| +2452723|AAAAAAAADPMGFCAA|2003-03-24|1238|5386|414|2003|1|3|24|1|2003|414|5386|Monday|2003Q1|N|N|N|2452700|2452758|2452358|2452631|N|N|N|Y|Y| +2452724|AAAAAAAAEPMGFCAA|2003-03-25|1238|5387|414|2003|2|3|25|1|2003|414|5387|Tuesday|2003Q1|N|N|N|2452700|2452758|2452359|2452632|N|N|N|Y|Y| +2452725|AAAAAAAAFPMGFCAA|2003-03-26|1238|5387|414|2003|3|3|26|1|2003|414|5387|Wednesday|2003Q1|N|N|N|2452700|2452758|2452360|2452633|N|N|N|Y|Y| +2452726|AAAAAAAAGPMGFCAA|2003-03-27|1238|5387|414|2003|4|3|27|1|2003|414|5387|Thursday|2003Q1|N|N|N|2452700|2452758|2452361|2452634|N|N|N|Y|Y| +2452727|AAAAAAAAHPMGFCAA|2003-03-28|1238|5387|414|2003|5|3|28|1|2003|414|5387|Friday|2003Q1|N|Y|N|2452700|2452758|2452362|2452635|N|N|N|Y|Y| +2452728|AAAAAAAAIPMGFCAA|2003-03-29|1238|5387|414|2003|6|3|29|1|2003|414|5387|Saturday|2003Q1|N|Y|N|2452700|2452758|2452363|2452636|N|N|N|Y|Y| +2452729|AAAAAAAAJPMGFCAA|2003-03-30|1238|5387|414|2003|0|3|30|1|2003|414|5387|Sunday|2003Q1|N|N|N|2452700|2452758|2452364|2452637|N|N|N|Y|Y| +2452730|AAAAAAAAKPMGFCAA|2003-03-31|1238|5387|414|2003|1|3|31|1|2003|414|5387|Monday|2003Q1|N|N|N|2452700|2452758|2452365|2452638|N|N|N|Y|Y| +2452731|AAAAAAAALPMGFCAA|2003-04-01|1239|5388|414|2003|2|4|1|1|2003|414|5388|Tuesday|2003Q1|N|N|N|2452731|2452820|2452366|2452641|N|N|N|Y|Y| +2452732|AAAAAAAAMPMGFCAA|2003-04-02|1239|5388|414|2003|3|4|2|2|2003|414|5388|Wednesday|2003Q2|N|N|N|2452731|2452820|2452367|2452642|N|N|N|N|Y| +2452733|AAAAAAAANPMGFCAA|2003-04-03|1239|5388|414|2003|4|4|3|2|2003|414|5388|Thursday|2003Q2|N|N|N|2452731|2452820|2452368|2452643|N|N|N|N|Y| +2452734|AAAAAAAAOPMGFCAA|2003-04-04|1239|5388|414|2003|5|4|4|2|2003|414|5388|Friday|2003Q2|N|Y|N|2452731|2452820|2452369|2452644|N|N|N|N|Y| +2452735|AAAAAAAAPPMGFCAA|2003-04-05|1239|5388|414|2003|6|4|5|2|2003|414|5388|Saturday|2003Q2|N|Y|N|2452731|2452820|2452370|2452645|N|N|N|N|Y| +2452736|AAAAAAAAAANGFCAA|2003-04-06|1239|5388|414|2003|0|4|6|2|2003|414|5388|Sunday|2003Q2|N|N|N|2452731|2452820|2452371|2452646|N|N|N|N|Y| +2452737|AAAAAAAABANGFCAA|2003-04-07|1239|5388|414|2003|1|4|7|2|2003|414|5388|Monday|2003Q2|N|N|N|2452731|2452820|2452372|2452647|N|N|N|N|Y| +2452738|AAAAAAAACANGFCAA|2003-04-08|1239|5389|414|2003|2|4|8|2|2003|414|5389|Tuesday|2003Q2|N|N|N|2452731|2452820|2452373|2452648|N|N|N|N|Y| +2452739|AAAAAAAADANGFCAA|2003-04-09|1239|5389|414|2003|3|4|9|2|2003|414|5389|Wednesday|2003Q2|N|N|N|2452731|2452820|2452374|2452649|N|N|N|N|Y| +2452740|AAAAAAAAEANGFCAA|2003-04-10|1239|5389|414|2003|4|4|10|2|2003|414|5389|Thursday|2003Q2|N|N|N|2452731|2452820|2452375|2452650|N|N|N|N|Y| +2452741|AAAAAAAAFANGFCAA|2003-04-11|1239|5389|414|2003|5|4|11|2|2003|414|5389|Friday|2003Q2|N|Y|N|2452731|2452820|2452376|2452651|N|N|N|N|Y| +2452742|AAAAAAAAGANGFCAA|2003-04-12|1239|5389|414|2003|6|4|12|2|2003|414|5389|Saturday|2003Q2|N|Y|N|2452731|2452820|2452377|2452652|N|N|N|N|Y| +2452743|AAAAAAAAHANGFCAA|2003-04-13|1239|5389|414|2003|0|4|13|2|2003|414|5389|Sunday|2003Q2|N|N|N|2452731|2452820|2452378|2452653|N|N|N|N|Y| +2452744|AAAAAAAAIANGFCAA|2003-04-14|1239|5389|414|2003|1|4|14|2|2003|414|5389|Monday|2003Q2|N|N|N|2452731|2452820|2452379|2452654|N|N|N|N|Y| +2452745|AAAAAAAAJANGFCAA|2003-04-15|1239|5390|414|2003|2|4|15|2|2003|414|5390|Tuesday|2003Q2|N|N|N|2452731|2452820|2452380|2452655|N|N|N|N|Y| +2452746|AAAAAAAAKANGFCAA|2003-04-16|1239|5390|414|2003|3|4|16|2|2003|414|5390|Wednesday|2003Q2|N|N|N|2452731|2452820|2452381|2452656|N|N|N|N|Y| +2452747|AAAAAAAALANGFCAA|2003-04-17|1239|5390|414|2003|4|4|17|2|2003|414|5390|Thursday|2003Q2|N|N|N|2452731|2452820|2452382|2452657|N|N|N|N|Y| +2452748|AAAAAAAAMANGFCAA|2003-04-18|1239|5390|414|2003|5|4|18|2|2003|414|5390|Friday|2003Q2|N|Y|N|2452731|2452820|2452383|2452658|N|N|N|N|Y| +2452749|AAAAAAAANANGFCAA|2003-04-19|1239|5390|414|2003|6|4|19|2|2003|414|5390|Saturday|2003Q2|N|Y|N|2452731|2452820|2452384|2452659|N|N|N|N|Y| +2452750|AAAAAAAAOANGFCAA|2003-04-20|1239|5390|414|2003|0|4|20|2|2003|414|5390|Sunday|2003Q2|N|N|N|2452731|2452820|2452385|2452660|N|N|N|N|Y| +2452751|AAAAAAAAPANGFCAA|2003-04-21|1239|5390|414|2003|1|4|21|2|2003|414|5390|Monday|2003Q2|N|N|N|2452731|2452820|2452386|2452661|N|N|N|N|Y| +2452752|AAAAAAAAABNGFCAA|2003-04-22|1239|5391|414|2003|2|4|22|2|2003|414|5391|Tuesday|2003Q2|N|N|N|2452731|2452820|2452387|2452662|N|N|N|N|Y| +2452753|AAAAAAAABBNGFCAA|2003-04-23|1239|5391|414|2003|3|4|23|2|2003|414|5391|Wednesday|2003Q2|N|N|N|2452731|2452820|2452388|2452663|N|N|N|N|Y| +2452754|AAAAAAAACBNGFCAA|2003-04-24|1239|5391|414|2003|4|4|24|2|2003|414|5391|Thursday|2003Q2|N|N|N|2452731|2452820|2452389|2452664|N|N|N|N|Y| +2452755|AAAAAAAADBNGFCAA|2003-04-25|1239|5391|414|2003|5|4|25|2|2003|414|5391|Friday|2003Q2|N|Y|N|2452731|2452820|2452390|2452665|N|N|N|N|Y| +2452756|AAAAAAAAEBNGFCAA|2003-04-26|1239|5391|414|2003|6|4|26|2|2003|414|5391|Saturday|2003Q2|N|Y|N|2452731|2452820|2452391|2452666|N|N|N|N|Y| +2452757|AAAAAAAAFBNGFCAA|2003-04-27|1239|5391|414|2003|0|4|27|2|2003|414|5391|Sunday|2003Q2|N|N|N|2452731|2452820|2452392|2452667|N|N|N|N|Y| +2452758|AAAAAAAAGBNGFCAA|2003-04-28|1239|5391|414|2003|1|4|28|2|2003|414|5391|Monday|2003Q2|N|N|N|2452731|2452820|2452393|2452668|N|N|N|N|Y| +2452759|AAAAAAAAHBNGFCAA|2003-04-29|1239|5392|414|2003|2|4|29|2|2003|414|5392|Tuesday|2003Q2|N|N|N|2452731|2452820|2452394|2452669|N|N|N|N|Y| +2452760|AAAAAAAAIBNGFCAA|2003-04-30|1239|5392|414|2003|3|4|30|2|2003|414|5392|Wednesday|2003Q2|N|N|N|2452731|2452820|2452395|2452670|N|N|N|N|Y| +2452761|AAAAAAAAJBNGFCAA|2003-05-01|1240|5392|414|2003|4|5|1|2|2003|414|5392|Thursday|2003Q2|N|N|N|2452761|2452880|2452396|2452671|N|N|N|N|Y| +2452762|AAAAAAAAKBNGFCAA|2003-05-02|1240|5392|414|2003|5|5|2|2|2003|414|5392|Friday|2003Q2|N|Y|N|2452761|2452880|2452397|2452672|N|N|N|N|Y| +2452763|AAAAAAAALBNGFCAA|2003-05-03|1240|5392|414|2003|6|5|3|2|2003|414|5392|Saturday|2003Q2|N|Y|N|2452761|2452880|2452398|2452673|N|N|N|N|Y| +2452764|AAAAAAAAMBNGFCAA|2003-05-04|1240|5392|414|2003|0|5|4|2|2003|414|5392|Sunday|2003Q2|N|N|N|2452761|2452880|2452399|2452674|N|N|N|N|Y| +2452765|AAAAAAAANBNGFCAA|2003-05-05|1240|5392|414|2003|1|5|5|2|2003|414|5392|Monday|2003Q2|N|N|N|2452761|2452880|2452400|2452675|N|N|N|N|Y| +2452766|AAAAAAAAOBNGFCAA|2003-05-06|1240|5393|414|2003|2|5|6|2|2003|414|5393|Tuesday|2003Q2|N|N|N|2452761|2452880|2452401|2452676|N|N|N|N|Y| +2452767|AAAAAAAAPBNGFCAA|2003-05-07|1240|5393|414|2003|3|5|7|2|2003|414|5393|Wednesday|2003Q2|N|N|N|2452761|2452880|2452402|2452677|N|N|N|N|Y| +2452768|AAAAAAAAACNGFCAA|2003-05-08|1240|5393|414|2003|4|5|8|2|2003|414|5393|Thursday|2003Q2|N|N|N|2452761|2452880|2452403|2452678|N|N|N|N|Y| +2452769|AAAAAAAABCNGFCAA|2003-05-09|1240|5393|414|2003|5|5|9|2|2003|414|5393|Friday|2003Q2|N|Y|N|2452761|2452880|2452404|2452679|N|N|N|N|Y| +2452770|AAAAAAAACCNGFCAA|2003-05-10|1240|5393|414|2003|6|5|10|2|2003|414|5393|Saturday|2003Q2|N|Y|N|2452761|2452880|2452405|2452680|N|N|N|N|Y| +2452771|AAAAAAAADCNGFCAA|2003-05-11|1240|5393|414|2003|0|5|11|2|2003|414|5393|Sunday|2003Q2|N|N|N|2452761|2452880|2452406|2452681|N|N|N|N|Y| +2452772|AAAAAAAAECNGFCAA|2003-05-12|1240|5393|414|2003|1|5|12|2|2003|414|5393|Monday|2003Q2|N|N|N|2452761|2452880|2452407|2452682|N|N|N|N|Y| +2452773|AAAAAAAAFCNGFCAA|2003-05-13|1240|5394|414|2003|2|5|13|2|2003|414|5394|Tuesday|2003Q2|N|N|N|2452761|2452880|2452408|2452683|N|N|N|N|Y| +2452774|AAAAAAAAGCNGFCAA|2003-05-14|1240|5394|414|2003|3|5|14|2|2003|414|5394|Wednesday|2003Q2|N|N|N|2452761|2452880|2452409|2452684|N|N|N|N|Y| +2452775|AAAAAAAAHCNGFCAA|2003-05-15|1240|5394|414|2003|4|5|15|2|2003|414|5394|Thursday|2003Q2|N|N|N|2452761|2452880|2452410|2452685|N|N|N|N|Y| +2452776|AAAAAAAAICNGFCAA|2003-05-16|1240|5394|414|2003|5|5|16|2|2003|414|5394|Friday|2003Q2|N|Y|N|2452761|2452880|2452411|2452686|N|N|N|N|Y| +2452777|AAAAAAAAJCNGFCAA|2003-05-17|1240|5394|414|2003|6|5|17|2|2003|414|5394|Saturday|2003Q2|N|Y|N|2452761|2452880|2452412|2452687|N|N|N|N|Y| +2452778|AAAAAAAAKCNGFCAA|2003-05-18|1240|5394|414|2003|0|5|18|2|2003|414|5394|Sunday|2003Q2|N|N|N|2452761|2452880|2452413|2452688|N|N|N|N|Y| +2452779|AAAAAAAALCNGFCAA|2003-05-19|1240|5394|414|2003|1|5|19|2|2003|414|5394|Monday|2003Q2|N|N|N|2452761|2452880|2452414|2452689|N|N|N|N|Y| +2452780|AAAAAAAAMCNGFCAA|2003-05-20|1240|5395|414|2003|2|5|20|2|2003|414|5395|Tuesday|2003Q2|N|N|N|2452761|2452880|2452415|2452690|N|N|N|N|Y| +2452781|AAAAAAAANCNGFCAA|2003-05-21|1240|5395|414|2003|3|5|21|2|2003|414|5395|Wednesday|2003Q2|N|N|N|2452761|2452880|2452416|2452691|N|N|N|N|Y| +2452782|AAAAAAAAOCNGFCAA|2003-05-22|1240|5395|414|2003|4|5|22|2|2003|414|5395|Thursday|2003Q2|N|N|N|2452761|2452880|2452417|2452692|N|N|N|N|Y| +2452783|AAAAAAAAPCNGFCAA|2003-05-23|1240|5395|414|2003|5|5|23|2|2003|414|5395|Friday|2003Q2|N|Y|N|2452761|2452880|2452418|2452693|N|N|N|N|Y| +2452784|AAAAAAAAADNGFCAA|2003-05-24|1240|5395|414|2003|6|5|24|2|2003|414|5395|Saturday|2003Q2|N|Y|N|2452761|2452880|2452419|2452694|N|N|N|N|Y| +2452785|AAAAAAAABDNGFCAA|2003-05-25|1240|5395|414|2003|0|5|25|2|2003|414|5395|Sunday|2003Q2|N|N|N|2452761|2452880|2452420|2452695|N|N|N|N|Y| +2452786|AAAAAAAACDNGFCAA|2003-05-26|1240|5395|414|2003|1|5|26|2|2003|414|5395|Monday|2003Q2|N|N|N|2452761|2452880|2452421|2452696|N|N|N|N|Y| +2452787|AAAAAAAADDNGFCAA|2003-05-27|1240|5396|414|2003|2|5|27|2|2003|414|5396|Tuesday|2003Q2|N|N|N|2452761|2452880|2452422|2452697|N|N|N|N|Y| +2452788|AAAAAAAAEDNGFCAA|2003-05-28|1240|5396|414|2003|3|5|28|2|2003|414|5396|Wednesday|2003Q2|N|N|N|2452761|2452880|2452423|2452698|N|N|N|N|Y| +2452789|AAAAAAAAFDNGFCAA|2003-05-29|1240|5396|414|2003|4|5|29|2|2003|414|5396|Thursday|2003Q2|N|N|N|2452761|2452880|2452424|2452699|N|N|N|N|Y| +2452790|AAAAAAAAGDNGFCAA|2003-05-30|1240|5396|414|2003|5|5|30|2|2003|414|5396|Friday|2003Q2|N|Y|N|2452761|2452880|2452425|2452700|N|N|N|N|Y| +2452791|AAAAAAAAHDNGFCAA|2003-05-31|1240|5396|414|2003|6|5|31|2|2003|414|5396|Saturday|2003Q2|N|Y|N|2452761|2452880|2452426|2452701|N|N|N|N|Y| +2452792|AAAAAAAAIDNGFCAA|2003-06-01|1241|5396|415|2003|0|6|1|2|2003|415|5396|Sunday|2003Q2|N|N|N|2452792|2452942|2452427|2452702|N|N|N|N|Y| +2452793|AAAAAAAAJDNGFCAA|2003-06-02|1241|5396|415|2003|1|6|2|2|2003|415|5396|Monday|2003Q2|N|N|N|2452792|2452942|2452428|2452703|N|N|N|N|Y| +2452794|AAAAAAAAKDNGFCAA|2003-06-03|1241|5397|415|2003|2|6|3|2|2003|415|5397|Tuesday|2003Q2|N|N|N|2452792|2452942|2452429|2452704|N|N|N|N|Y| +2452795|AAAAAAAALDNGFCAA|2003-06-04|1241|5397|415|2003|3|6|4|2|2003|415|5397|Wednesday|2003Q2|N|N|N|2452792|2452942|2452430|2452705|N|N|N|N|Y| +2452796|AAAAAAAAMDNGFCAA|2003-06-05|1241|5397|415|2003|4|6|5|2|2003|415|5397|Thursday|2003Q2|N|N|N|2452792|2452942|2452431|2452706|N|N|N|N|Y| +2452797|AAAAAAAANDNGFCAA|2003-06-06|1241|5397|415|2003|5|6|6|2|2003|415|5397|Friday|2003Q2|N|Y|N|2452792|2452942|2452432|2452707|N|N|N|N|Y| +2452798|AAAAAAAAODNGFCAA|2003-06-07|1241|5397|415|2003|6|6|7|2|2003|415|5397|Saturday|2003Q2|N|Y|N|2452792|2452942|2452433|2452708|N|N|N|N|Y| +2452799|AAAAAAAAPDNGFCAA|2003-06-08|1241|5397|415|2003|0|6|8|2|2003|415|5397|Sunday|2003Q2|N|N|N|2452792|2452942|2452434|2452709|N|N|N|N|Y| +2452800|AAAAAAAAAENGFCAA|2003-06-09|1241|5397|415|2003|1|6|9|2|2003|415|5397|Monday|2003Q2|N|N|N|2452792|2452942|2452435|2452710|N|N|N|N|Y| +2452801|AAAAAAAABENGFCAA|2003-06-10|1241|5398|415|2003|2|6|10|2|2003|415|5398|Tuesday|2003Q2|N|N|N|2452792|2452942|2452436|2452711|N|N|N|N|Y| +2452802|AAAAAAAACENGFCAA|2003-06-11|1241|5398|415|2003|3|6|11|2|2003|415|5398|Wednesday|2003Q2|N|N|N|2452792|2452942|2452437|2452712|N|N|N|N|Y| +2452803|AAAAAAAADENGFCAA|2003-06-12|1241|5398|415|2003|4|6|12|2|2003|415|5398|Thursday|2003Q2|N|N|N|2452792|2452942|2452438|2452713|N|N|N|N|Y| +2452804|AAAAAAAAEENGFCAA|2003-06-13|1241|5398|415|2003|5|6|13|2|2003|415|5398|Friday|2003Q2|N|Y|N|2452792|2452942|2452439|2452714|N|N|N|N|Y| +2452805|AAAAAAAAFENGFCAA|2003-06-14|1241|5398|415|2003|6|6|14|2|2003|415|5398|Saturday|2003Q2|N|Y|N|2452792|2452942|2452440|2452715|N|N|N|N|Y| +2452806|AAAAAAAAGENGFCAA|2003-06-15|1241|5398|415|2003|0|6|15|2|2003|415|5398|Sunday|2003Q2|N|N|N|2452792|2452942|2452441|2452716|N|N|N|N|Y| +2452807|AAAAAAAAHENGFCAA|2003-06-16|1241|5398|415|2003|1|6|16|2|2003|415|5398|Monday|2003Q2|N|N|N|2452792|2452942|2452442|2452717|N|N|N|N|Y| +2452808|AAAAAAAAIENGFCAA|2003-06-17|1241|5399|415|2003|2|6|17|2|2003|415|5399|Tuesday|2003Q2|N|N|N|2452792|2452942|2452443|2452718|N|N|N|N|Y| +2452809|AAAAAAAAJENGFCAA|2003-06-18|1241|5399|415|2003|3|6|18|2|2003|415|5399|Wednesday|2003Q2|N|N|N|2452792|2452942|2452444|2452719|N|N|N|N|Y| +2452810|AAAAAAAAKENGFCAA|2003-06-19|1241|5399|415|2003|4|6|19|2|2003|415|5399|Thursday|2003Q2|N|N|N|2452792|2452942|2452445|2452720|N|N|N|N|Y| +2452811|AAAAAAAALENGFCAA|2003-06-20|1241|5399|415|2003|5|6|20|2|2003|415|5399|Friday|2003Q2|N|Y|N|2452792|2452942|2452446|2452721|N|N|N|N|Y| +2452812|AAAAAAAAMENGFCAA|2003-06-21|1241|5399|415|2003|6|6|21|2|2003|415|5399|Saturday|2003Q2|N|Y|N|2452792|2452942|2452447|2452722|N|N|N|N|Y| +2452813|AAAAAAAANENGFCAA|2003-06-22|1241|5399|415|2003|0|6|22|2|2003|415|5399|Sunday|2003Q2|N|N|N|2452792|2452942|2452448|2452723|N|N|N|N|Y| +2452814|AAAAAAAAOENGFCAA|2003-06-23|1241|5399|415|2003|1|6|23|2|2003|415|5399|Monday|2003Q2|N|N|N|2452792|2452942|2452449|2452724|N|N|N|N|Y| +2452815|AAAAAAAAPENGFCAA|2003-06-24|1241|5400|415|2003|2|6|24|2|2003|415|5400|Tuesday|2003Q2|N|N|N|2452792|2452942|2452450|2452725|N|N|N|N|Y| +2452816|AAAAAAAAAFNGFCAA|2003-06-25|1241|5400|415|2003|3|6|25|2|2003|415|5400|Wednesday|2003Q2|N|N|N|2452792|2452942|2452451|2452726|N|N|N|N|Y| +2452817|AAAAAAAABFNGFCAA|2003-06-26|1241|5400|415|2003|4|6|26|2|2003|415|5400|Thursday|2003Q2|N|N|N|2452792|2452942|2452452|2452727|N|N|N|N|Y| +2452818|AAAAAAAACFNGFCAA|2003-06-27|1241|5400|415|2003|5|6|27|2|2003|415|5400|Friday|2003Q2|N|Y|N|2452792|2452942|2452453|2452728|N|N|N|N|Y| +2452819|AAAAAAAADFNGFCAA|2003-06-28|1241|5400|415|2003|6|6|28|2|2003|415|5400|Saturday|2003Q2|N|Y|N|2452792|2452942|2452454|2452729|N|N|N|N|Y| +2452820|AAAAAAAAEFNGFCAA|2003-06-29|1241|5400|415|2003|0|6|29|2|2003|415|5400|Sunday|2003Q2|N|N|N|2452792|2452942|2452455|2452730|N|N|N|N|Y| +2452821|AAAAAAAAFFNGFCAA|2003-06-30|1241|5400|415|2003|1|6|30|2|2003|415|5400|Monday|2003Q2|N|N|N|2452792|2452942|2452456|2452731|N|N|N|N|Y| +2452822|AAAAAAAAGFNGFCAA|2003-07-01|1242|5401|415|2003|2|7|1|2|2003|415|5401|Tuesday|2003Q2|N|N|N|2452822|2453002|2452457|2452731|N|N|N|N|Y| +2452823|AAAAAAAAHFNGFCAA|2003-07-02|1242|5401|415|2003|3|7|2|3|2003|415|5401|Wednesday|2003Q3|N|N|N|2452822|2453002|2452458|2452732|N|N|N|N|Y| +2452824|AAAAAAAAIFNGFCAA|2003-07-03|1242|5401|415|2003|4|7|3|3|2003|415|5401|Thursday|2003Q3|N|N|N|2452822|2453002|2452459|2452733|N|N|N|N|Y| +2452825|AAAAAAAAJFNGFCAA|2003-07-04|1242|5401|415|2003|5|7|4|3|2003|415|5401|Friday|2003Q3|N|Y|N|2452822|2453002|2452460|2452734|N|N|N|N|Y| +2452826|AAAAAAAAKFNGFCAA|2003-07-05|1242|5401|415|2003|6|7|5|3|2003|415|5401|Saturday|2003Q3|Y|Y|N|2452822|2453002|2452461|2452735|N|N|N|N|Y| +2452827|AAAAAAAALFNGFCAA|2003-07-06|1242|5401|415|2003|0|7|6|3|2003|415|5401|Sunday|2003Q3|N|N|Y|2452822|2453002|2452462|2452736|N|N|N|N|Y| +2452828|AAAAAAAAMFNGFCAA|2003-07-07|1242|5401|415|2003|1|7|7|3|2003|415|5401|Monday|2003Q3|N|N|N|2452822|2453002|2452463|2452737|N|N|N|N|Y| +2452829|AAAAAAAANFNGFCAA|2003-07-08|1242|5402|415|2003|2|7|8|3|2003|415|5402|Tuesday|2003Q3|N|N|N|2452822|2453002|2452464|2452738|N|N|N|N|Y| +2452830|AAAAAAAAOFNGFCAA|2003-07-09|1242|5402|415|2003|3|7|9|3|2003|415|5402|Wednesday|2003Q3|N|N|N|2452822|2453002|2452465|2452739|N|N|N|N|Y| +2452831|AAAAAAAAPFNGFCAA|2003-07-10|1242|5402|415|2003|4|7|10|3|2003|415|5402|Thursday|2003Q3|N|N|N|2452822|2453002|2452466|2452740|N|N|N|N|Y| +2452832|AAAAAAAAAGNGFCAA|2003-07-11|1242|5402|415|2003|5|7|11|3|2003|415|5402|Friday|2003Q3|N|Y|N|2452822|2453002|2452467|2452741|N|N|N|N|Y| +2452833|AAAAAAAABGNGFCAA|2003-07-12|1242|5402|415|2003|6|7|12|3|2003|415|5402|Saturday|2003Q3|N|Y|N|2452822|2453002|2452468|2452742|N|N|N|N|Y| +2452834|AAAAAAAACGNGFCAA|2003-07-13|1242|5402|415|2003|0|7|13|3|2003|415|5402|Sunday|2003Q3|N|N|N|2452822|2453002|2452469|2452743|N|N|N|N|Y| +2452835|AAAAAAAADGNGFCAA|2003-07-14|1242|5402|415|2003|1|7|14|3|2003|415|5402|Monday|2003Q3|N|N|N|2452822|2453002|2452470|2452744|N|N|N|N|Y| +2452836|AAAAAAAAEGNGFCAA|2003-07-15|1242|5403|415|2003|2|7|15|3|2003|415|5403|Tuesday|2003Q3|N|N|N|2452822|2453002|2452471|2452745|N|N|N|N|Y| +2452837|AAAAAAAAFGNGFCAA|2003-07-16|1242|5403|415|2003|3|7|16|3|2003|415|5403|Wednesday|2003Q3|N|N|N|2452822|2453002|2452472|2452746|N|N|N|N|Y| +2452838|AAAAAAAAGGNGFCAA|2003-07-17|1242|5403|415|2003|4|7|17|3|2003|415|5403|Thursday|2003Q3|N|N|N|2452822|2453002|2452473|2452747|N|N|N|N|Y| +2452839|AAAAAAAAHGNGFCAA|2003-07-18|1242|5403|415|2003|5|7|18|3|2003|415|5403|Friday|2003Q3|N|Y|N|2452822|2453002|2452474|2452748|N|N|N|N|Y| +2452840|AAAAAAAAIGNGFCAA|2003-07-19|1242|5403|415|2003|6|7|19|3|2003|415|5403|Saturday|2003Q3|N|Y|N|2452822|2453002|2452475|2452749|N|N|N|N|Y| +2452841|AAAAAAAAJGNGFCAA|2003-07-20|1242|5403|415|2003|0|7|20|3|2003|415|5403|Sunday|2003Q3|N|N|N|2452822|2453002|2452476|2452750|N|N|N|N|Y| +2452842|AAAAAAAAKGNGFCAA|2003-07-21|1242|5403|415|2003|1|7|21|3|2003|415|5403|Monday|2003Q3|N|N|N|2452822|2453002|2452477|2452751|N|N|N|N|Y| +2452843|AAAAAAAALGNGFCAA|2003-07-22|1242|5404|415|2003|2|7|22|3|2003|415|5404|Tuesday|2003Q3|N|N|N|2452822|2453002|2452478|2452752|N|N|N|N|Y| +2452844|AAAAAAAAMGNGFCAA|2003-07-23|1242|5404|415|2003|3|7|23|3|2003|415|5404|Wednesday|2003Q3|N|N|N|2452822|2453002|2452479|2452753|N|N|N|N|Y| +2452845|AAAAAAAANGNGFCAA|2003-07-24|1242|5404|415|2003|4|7|24|3|2003|415|5404|Thursday|2003Q3|N|N|N|2452822|2453002|2452480|2452754|N|N|N|N|Y| +2452846|AAAAAAAAOGNGFCAA|2003-07-25|1242|5404|415|2003|5|7|25|3|2003|415|5404|Friday|2003Q3|N|Y|N|2452822|2453002|2452481|2452755|N|N|N|N|Y| +2452847|AAAAAAAAPGNGFCAA|2003-07-26|1242|5404|415|2003|6|7|26|3|2003|415|5404|Saturday|2003Q3|N|Y|N|2452822|2453002|2452482|2452756|N|N|N|N|Y| +2452848|AAAAAAAAAHNGFCAA|2003-07-27|1242|5404|415|2003|0|7|27|3|2003|415|5404|Sunday|2003Q3|N|N|N|2452822|2453002|2452483|2452757|N|N|N|N|Y| +2452849|AAAAAAAABHNGFCAA|2003-07-28|1242|5404|415|2003|1|7|28|3|2003|415|5404|Monday|2003Q3|N|N|N|2452822|2453002|2452484|2452758|N|N|N|N|Y| +2452850|AAAAAAAACHNGFCAA|2003-07-29|1242|5405|415|2003|2|7|29|3|2003|415|5405|Tuesday|2003Q3|N|N|N|2452822|2453002|2452485|2452759|N|N|N|N|Y| +2452851|AAAAAAAADHNGFCAA|2003-07-30|1242|5405|415|2003|3|7|30|3|2003|415|5405|Wednesday|2003Q3|N|N|N|2452822|2453002|2452486|2452760|N|N|N|N|Y| +2452852|AAAAAAAAEHNGFCAA|2003-07-31|1242|5405|415|2003|4|7|31|3|2003|415|5405|Thursday|2003Q3|N|N|N|2452822|2453002|2452487|2452761|N|N|N|N|Y| +2452853|AAAAAAAAFHNGFCAA|2003-08-01|1243|5405|415|2003|5|8|1|3|2003|415|5405|Friday|2003Q3|N|Y|N|2452853|2453064|2452488|2452762|N|N|N|N|Y| +2452854|AAAAAAAAGHNGFCAA|2003-08-02|1243|5405|415|2003|6|8|2|3|2003|415|5405|Saturday|2003Q3|N|Y|N|2452853|2453064|2452489|2452763|N|N|N|N|Y| +2452855|AAAAAAAAHHNGFCAA|2003-08-03|1243|5405|415|2003|0|8|3|3|2003|415|5405|Sunday|2003Q3|N|N|N|2452853|2453064|2452490|2452764|N|N|N|N|Y| +2452856|AAAAAAAAIHNGFCAA|2003-08-04|1243|5405|415|2003|1|8|4|3|2003|415|5405|Monday|2003Q3|N|N|N|2452853|2453064|2452491|2452765|N|N|N|N|Y| +2452857|AAAAAAAAJHNGFCAA|2003-08-05|1243|5406|415|2003|2|8|5|3|2003|415|5406|Tuesday|2003Q3|N|N|N|2452853|2453064|2452492|2452766|N|N|N|N|Y| +2452858|AAAAAAAAKHNGFCAA|2003-08-06|1243|5406|415|2003|3|8|6|3|2003|415|5406|Wednesday|2003Q3|N|N|N|2452853|2453064|2452493|2452767|N|N|N|N|Y| +2452859|AAAAAAAALHNGFCAA|2003-08-07|1243|5406|415|2003|4|8|7|3|2003|415|5406|Thursday|2003Q3|N|N|N|2452853|2453064|2452494|2452768|N|N|N|N|Y| +2452860|AAAAAAAAMHNGFCAA|2003-08-08|1243|5406|415|2003|5|8|8|3|2003|415|5406|Friday|2003Q3|N|Y|N|2452853|2453064|2452495|2452769|N|N|N|N|Y| +2452861|AAAAAAAANHNGFCAA|2003-08-09|1243|5406|415|2003|6|8|9|3|2003|415|5406|Saturday|2003Q3|N|Y|N|2452853|2453064|2452496|2452770|N|N|N|N|Y| +2452862|AAAAAAAAOHNGFCAA|2003-08-10|1243|5406|415|2003|0|8|10|3|2003|415|5406|Sunday|2003Q3|N|N|N|2452853|2453064|2452497|2452771|N|N|N|N|Y| +2452863|AAAAAAAAPHNGFCAA|2003-08-11|1243|5406|415|2003|1|8|11|3|2003|415|5406|Monday|2003Q3|N|N|N|2452853|2453064|2452498|2452772|N|N|N|N|Y| +2452864|AAAAAAAAAINGFCAA|2003-08-12|1243|5407|415|2003|2|8|12|3|2003|415|5407|Tuesday|2003Q3|N|N|N|2452853|2453064|2452499|2452773|N|N|N|N|Y| +2452865|AAAAAAAABINGFCAA|2003-08-13|1243|5407|415|2003|3|8|13|3|2003|415|5407|Wednesday|2003Q3|N|N|N|2452853|2453064|2452500|2452774|N|N|N|N|Y| +2452866|AAAAAAAACINGFCAA|2003-08-14|1243|5407|415|2003|4|8|14|3|2003|415|5407|Thursday|2003Q3|N|N|N|2452853|2453064|2452501|2452775|N|N|N|N|Y| +2452867|AAAAAAAADINGFCAA|2003-08-15|1243|5407|415|2003|5|8|15|3|2003|415|5407|Friday|2003Q3|N|Y|N|2452853|2453064|2452502|2452776|N|N|N|N|Y| +2452868|AAAAAAAAEINGFCAA|2003-08-16|1243|5407|415|2003|6|8|16|3|2003|415|5407|Saturday|2003Q3|N|Y|N|2452853|2453064|2452503|2452777|N|N|N|N|Y| +2452869|AAAAAAAAFINGFCAA|2003-08-17|1243|5407|415|2003|0|8|17|3|2003|415|5407|Sunday|2003Q3|N|N|N|2452853|2453064|2452504|2452778|N|N|N|N|Y| +2452870|AAAAAAAAGINGFCAA|2003-08-18|1243|5407|415|2003|1|8|18|3|2003|415|5407|Monday|2003Q3|N|N|N|2452853|2453064|2452505|2452779|N|N|N|N|Y| +2452871|AAAAAAAAHINGFCAA|2003-08-19|1243|5408|415|2003|2|8|19|3|2003|415|5408|Tuesday|2003Q3|N|N|N|2452853|2453064|2452506|2452780|N|N|N|N|Y| +2452872|AAAAAAAAIINGFCAA|2003-08-20|1243|5408|415|2003|3|8|20|3|2003|415|5408|Wednesday|2003Q3|N|N|N|2452853|2453064|2452507|2452781|N|N|N|N|Y| +2452873|AAAAAAAAJINGFCAA|2003-08-21|1243|5408|415|2003|4|8|21|3|2003|415|5408|Thursday|2003Q3|N|N|N|2452853|2453064|2452508|2452782|N|N|N|N|Y| +2452874|AAAAAAAAKINGFCAA|2003-08-22|1243|5408|415|2003|5|8|22|3|2003|415|5408|Friday|2003Q3|N|Y|N|2452853|2453064|2452509|2452783|N|N|N|N|Y| +2452875|AAAAAAAALINGFCAA|2003-08-23|1243|5408|415|2003|6|8|23|3|2003|415|5408|Saturday|2003Q3|N|Y|N|2452853|2453064|2452510|2452784|N|N|N|N|Y| +2452876|AAAAAAAAMINGFCAA|2003-08-24|1243|5408|415|2003|0|8|24|3|2003|415|5408|Sunday|2003Q3|N|N|N|2452853|2453064|2452511|2452785|N|N|N|N|Y| +2452877|AAAAAAAANINGFCAA|2003-08-25|1243|5408|415|2003|1|8|25|3|2003|415|5408|Monday|2003Q3|N|N|N|2452853|2453064|2452512|2452786|N|N|N|N|Y| +2452878|AAAAAAAAOINGFCAA|2003-08-26|1243|5409|415|2003|2|8|26|3|2003|415|5409|Tuesday|2003Q3|N|N|N|2452853|2453064|2452513|2452787|N|N|N|N|Y| +2452879|AAAAAAAAPINGFCAA|2003-08-27|1243|5409|415|2003|3|8|27|3|2003|415|5409|Wednesday|2003Q3|N|N|N|2452853|2453064|2452514|2452788|N|N|N|N|Y| +2452880|AAAAAAAAAJNGFCAA|2003-08-28|1243|5409|415|2003|4|8|28|3|2003|415|5409|Thursday|2003Q3|N|N|N|2452853|2453064|2452515|2452789|N|N|N|N|Y| +2452881|AAAAAAAABJNGFCAA|2003-08-29|1243|5409|415|2003|5|8|29|3|2003|415|5409|Friday|2003Q3|N|Y|N|2452853|2453064|2452516|2452790|N|N|N|N|Y| +2452882|AAAAAAAACJNGFCAA|2003-08-30|1243|5409|415|2003|6|8|30|3|2003|415|5409|Saturday|2003Q3|N|Y|N|2452853|2453064|2452517|2452791|N|N|N|N|Y| +2452883|AAAAAAAADJNGFCAA|2003-08-31|1243|5409|415|2003|0|8|31|3|2003|415|5409|Sunday|2003Q3|N|N|N|2452853|2453064|2452518|2452792|N|N|N|N|Y| +2452884|AAAAAAAAEJNGFCAA|2003-09-01|1244|5409|416|2003|1|9|1|3|2003|416|5409|Monday|2003Q3|N|N|N|2452884|2453126|2452519|2452793|N|N|N|N|Y| +2452885|AAAAAAAAFJNGFCAA|2003-09-02|1244|5410|416|2003|2|9|2|3|2003|416|5410|Tuesday|2003Q3|N|N|N|2452884|2453126|2452520|2452794|N|N|N|N|Y| +2452886|AAAAAAAAGJNGFCAA|2003-09-03|1244|5410|416|2003|3|9|3|3|2003|416|5410|Wednesday|2003Q3|N|N|N|2452884|2453126|2452521|2452795|N|N|N|N|Y| +2452887|AAAAAAAAHJNGFCAA|2003-09-04|1244|5410|416|2003|4|9|4|3|2003|416|5410|Thursday|2003Q3|N|N|N|2452884|2453126|2452522|2452796|N|N|N|N|Y| +2452888|AAAAAAAAIJNGFCAA|2003-09-05|1244|5410|416|2003|5|9|5|3|2003|416|5410|Friday|2003Q3|N|Y|N|2452884|2453126|2452523|2452797|N|N|N|N|Y| +2452889|AAAAAAAAJJNGFCAA|2003-09-06|1244|5410|416|2003|6|9|6|3|2003|416|5410|Saturday|2003Q3|N|Y|N|2452884|2453126|2452524|2452798|N|N|N|N|Y| +2452890|AAAAAAAAKJNGFCAA|2003-09-07|1244|5410|416|2003|0|9|7|3|2003|416|5410|Sunday|2003Q3|N|N|N|2452884|2453126|2452525|2452799|N|N|N|N|Y| +2452891|AAAAAAAALJNGFCAA|2003-09-08|1244|5410|416|2003|1|9|8|3|2003|416|5410|Monday|2003Q3|N|N|N|2452884|2453126|2452526|2452800|N|N|N|N|Y| +2452892|AAAAAAAAMJNGFCAA|2003-09-09|1244|5411|416|2003|2|9|9|3|2003|416|5411|Tuesday|2003Q3|N|N|N|2452884|2453126|2452527|2452801|N|N|N|N|Y| +2452893|AAAAAAAANJNGFCAA|2003-09-10|1244|5411|416|2003|3|9|10|3|2003|416|5411|Wednesday|2003Q3|N|N|N|2452884|2453126|2452528|2452802|N|N|N|N|Y| +2452894|AAAAAAAAOJNGFCAA|2003-09-11|1244|5411|416|2003|4|9|11|3|2003|416|5411|Thursday|2003Q3|N|N|N|2452884|2453126|2452529|2452803|N|N|N|N|Y| +2452895|AAAAAAAAPJNGFCAA|2003-09-12|1244|5411|416|2003|5|9|12|3|2003|416|5411|Friday|2003Q3|N|Y|N|2452884|2453126|2452530|2452804|N|N|N|N|Y| +2452896|AAAAAAAAAKNGFCAA|2003-09-13|1244|5411|416|2003|6|9|13|3|2003|416|5411|Saturday|2003Q3|N|Y|N|2452884|2453126|2452531|2452805|N|N|N|N|Y| +2452897|AAAAAAAABKNGFCAA|2003-09-14|1244|5411|416|2003|0|9|14|3|2003|416|5411|Sunday|2003Q3|N|N|N|2452884|2453126|2452532|2452806|N|N|N|N|Y| +2452898|AAAAAAAACKNGFCAA|2003-09-15|1244|5411|416|2003|1|9|15|3|2003|416|5411|Monday|2003Q3|N|N|N|2452884|2453126|2452533|2452807|N|N|N|N|Y| +2452899|AAAAAAAADKNGFCAA|2003-09-16|1244|5412|416|2003|2|9|16|3|2003|416|5412|Tuesday|2003Q3|N|N|N|2452884|2453126|2452534|2452808|N|N|N|N|Y| +2452900|AAAAAAAAEKNGFCAA|2003-09-17|1244|5412|416|2003|3|9|17|3|2003|416|5412|Wednesday|2003Q3|N|N|N|2452884|2453126|2452535|2452809|N|N|N|N|Y| +2452901|AAAAAAAAFKNGFCAA|2003-09-18|1244|5412|416|2003|4|9|18|3|2003|416|5412|Thursday|2003Q3|N|N|N|2452884|2453126|2452536|2452810|N|N|N|N|Y| +2452902|AAAAAAAAGKNGFCAA|2003-09-19|1244|5412|416|2003|5|9|19|3|2003|416|5412|Friday|2003Q3|N|Y|N|2452884|2453126|2452537|2452811|N|N|N|N|Y| +2452903|AAAAAAAAHKNGFCAA|2003-09-20|1244|5412|416|2003|6|9|20|3|2003|416|5412|Saturday|2003Q3|N|Y|N|2452884|2453126|2452538|2452812|N|N|N|N|Y| +2452904|AAAAAAAAIKNGFCAA|2003-09-21|1244|5412|416|2003|0|9|21|3|2003|416|5412|Sunday|2003Q3|N|N|N|2452884|2453126|2452539|2452813|N|N|N|N|Y| +2452905|AAAAAAAAJKNGFCAA|2003-09-22|1244|5412|416|2003|1|9|22|3|2003|416|5412|Monday|2003Q3|N|N|N|2452884|2453126|2452540|2452814|N|N|N|N|Y| +2452906|AAAAAAAAKKNGFCAA|2003-09-23|1244|5413|416|2003|2|9|23|3|2003|416|5413|Tuesday|2003Q3|N|N|N|2452884|2453126|2452541|2452815|N|N|N|N|Y| +2452907|AAAAAAAALKNGFCAA|2003-09-24|1244|5413|416|2003|3|9|24|3|2003|416|5413|Wednesday|2003Q3|N|N|N|2452884|2453126|2452542|2452816|N|N|N|N|Y| +2452908|AAAAAAAAMKNGFCAA|2003-09-25|1244|5413|416|2003|4|9|25|3|2003|416|5413|Thursday|2003Q3|N|N|N|2452884|2453126|2452543|2452817|N|N|N|N|Y| +2452909|AAAAAAAANKNGFCAA|2003-09-26|1244|5413|416|2003|5|9|26|3|2003|416|5413|Friday|2003Q3|N|Y|N|2452884|2453126|2452544|2452818|N|N|N|N|Y| +2452910|AAAAAAAAOKNGFCAA|2003-09-27|1244|5413|416|2003|6|9|27|3|2003|416|5413|Saturday|2003Q3|N|Y|N|2452884|2453126|2452545|2452819|N|N|N|N|Y| +2452911|AAAAAAAAPKNGFCAA|2003-09-28|1244|5413|416|2003|0|9|28|3|2003|416|5413|Sunday|2003Q3|N|N|N|2452884|2453126|2452546|2452820|N|N|N|N|Y| +2452912|AAAAAAAAALNGFCAA|2003-09-29|1244|5413|416|2003|1|9|29|3|2003|416|5413|Monday|2003Q3|N|N|N|2452884|2453126|2452547|2452821|N|N|N|N|Y| +2452913|AAAAAAAABLNGFCAA|2003-09-30|1244|5414|416|2003|2|9|30|3|2003|416|5414|Tuesday|2003Q3|N|N|N|2452884|2453126|2452548|2452822|N|N|N|N|Y| +2452914|AAAAAAAACLNGFCAA|2003-10-01|1245|5414|416|2003|3|10|1|3|2003|416|5414|Wednesday|2003Q3|N|N|N|2452914|2453186|2452549|2452822|N|N|N|N|Y| +2452915|AAAAAAAADLNGFCAA|2003-10-02|1245|5414|416|2003|4|10|2|4|2003|416|5414|Thursday|2003Q4|N|N|N|2452914|2453186|2452550|2452823|N|N|N|N|Y| +2452916|AAAAAAAAELNGFCAA|2003-10-03|1245|5414|416|2003|5|10|3|4|2003|416|5414|Friday|2003Q4|N|Y|N|2452914|2453186|2452551|2452824|N|N|N|N|Y| +2452917|AAAAAAAAFLNGFCAA|2003-10-04|1245|5414|416|2003|6|10|4|4|2003|416|5414|Saturday|2003Q4|N|Y|N|2452914|2453186|2452552|2452825|N|N|N|N|Y| +2452918|AAAAAAAAGLNGFCAA|2003-10-05|1245|5414|416|2003|0|10|5|4|2003|416|5414|Sunday|2003Q4|N|N|N|2452914|2453186|2452553|2452826|N|N|N|N|Y| +2452919|AAAAAAAAHLNGFCAA|2003-10-06|1245|5414|416|2003|1|10|6|4|2003|416|5414|Monday|2003Q4|N|N|N|2452914|2453186|2452554|2452827|N|N|N|N|Y| +2452920|AAAAAAAAILNGFCAA|2003-10-07|1245|5415|416|2003|2|10|7|4|2003|416|5415|Tuesday|2003Q4|N|N|N|2452914|2453186|2452555|2452828|N|N|N|N|Y| +2452921|AAAAAAAAJLNGFCAA|2003-10-08|1245|5415|416|2003|3|10|8|4|2003|416|5415|Wednesday|2003Q4|N|N|N|2452914|2453186|2452556|2452829|N|N|N|N|Y| +2452922|AAAAAAAAKLNGFCAA|2003-10-09|1245|5415|416|2003|4|10|9|4|2003|416|5415|Thursday|2003Q4|N|N|N|2452914|2453186|2452557|2452830|N|N|N|N|Y| +2452923|AAAAAAAALLNGFCAA|2003-10-10|1245|5415|416|2003|5|10|10|4|2003|416|5415|Friday|2003Q4|N|Y|N|2452914|2453186|2452558|2452831|N|N|N|N|Y| +2452924|AAAAAAAAMLNGFCAA|2003-10-11|1245|5415|416|2003|6|10|11|4|2003|416|5415|Saturday|2003Q4|N|Y|N|2452914|2453186|2452559|2452832|N|N|N|N|Y| +2452925|AAAAAAAANLNGFCAA|2003-10-12|1245|5415|416|2003|0|10|12|4|2003|416|5415|Sunday|2003Q4|N|N|N|2452914|2453186|2452560|2452833|N|N|N|N|Y| +2452926|AAAAAAAAOLNGFCAA|2003-10-13|1245|5415|416|2003|1|10|13|4|2003|416|5415|Monday|2003Q4|N|N|N|2452914|2453186|2452561|2452834|N|N|N|N|Y| +2452927|AAAAAAAAPLNGFCAA|2003-10-14|1245|5416|416|2003|2|10|14|4|2003|416|5416|Tuesday|2003Q4|N|N|N|2452914|2453186|2452562|2452835|N|N|N|N|Y| +2452928|AAAAAAAAAMNGFCAA|2003-10-15|1245|5416|416|2003|3|10|15|4|2003|416|5416|Wednesday|2003Q4|N|N|N|2452914|2453186|2452563|2452836|N|N|N|N|Y| +2452929|AAAAAAAABMNGFCAA|2003-10-16|1245|5416|416|2003|4|10|16|4|2003|416|5416|Thursday|2003Q4|N|N|N|2452914|2453186|2452564|2452837|N|N|N|N|Y| +2452930|AAAAAAAACMNGFCAA|2003-10-17|1245|5416|416|2003|5|10|17|4|2003|416|5416|Friday|2003Q4|N|Y|N|2452914|2453186|2452565|2452838|N|N|N|N|Y| +2452931|AAAAAAAADMNGFCAA|2003-10-18|1245|5416|416|2003|6|10|18|4|2003|416|5416|Saturday|2003Q4|N|Y|N|2452914|2453186|2452566|2452839|N|N|N|N|Y| +2452932|AAAAAAAAEMNGFCAA|2003-10-19|1245|5416|416|2003|0|10|19|4|2003|416|5416|Sunday|2003Q4|N|N|N|2452914|2453186|2452567|2452840|N|N|N|N|Y| +2452933|AAAAAAAAFMNGFCAA|2003-10-20|1245|5416|416|2003|1|10|20|4|2003|416|5416|Monday|2003Q4|N|N|N|2452914|2453186|2452568|2452841|N|N|N|N|Y| +2452934|AAAAAAAAGMNGFCAA|2003-10-21|1245|5417|416|2003|2|10|21|4|2003|416|5417|Tuesday|2003Q4|N|N|N|2452914|2453186|2452569|2452842|N|N|N|N|Y| +2452935|AAAAAAAAHMNGFCAA|2003-10-22|1245|5417|416|2003|3|10|22|4|2003|416|5417|Wednesday|2003Q4|N|N|N|2452914|2453186|2452570|2452843|N|N|N|N|Y| +2452936|AAAAAAAAIMNGFCAA|2003-10-23|1245|5417|416|2003|4|10|23|4|2003|416|5417|Thursday|2003Q4|N|N|N|2452914|2453186|2452571|2452844|N|N|N|N|Y| +2452937|AAAAAAAAJMNGFCAA|2003-10-24|1245|5417|416|2003|5|10|24|4|2003|416|5417|Friday|2003Q4|N|Y|N|2452914|2453186|2452572|2452845|N|N|N|N|Y| +2452938|AAAAAAAAKMNGFCAA|2003-10-25|1245|5417|416|2003|6|10|25|4|2003|416|5417|Saturday|2003Q4|N|Y|N|2452914|2453186|2452573|2452846|N|N|N|N|Y| +2452939|AAAAAAAALMNGFCAA|2003-10-26|1245|5417|416|2003|0|10|26|4|2003|416|5417|Sunday|2003Q4|N|N|N|2452914|2453186|2452574|2452847|N|N|N|N|Y| +2452940|AAAAAAAAMMNGFCAA|2003-10-27|1245|5417|416|2003|1|10|27|4|2003|416|5417|Monday|2003Q4|N|N|N|2452914|2453186|2452575|2452848|N|N|N|N|Y| +2452941|AAAAAAAANMNGFCAA|2003-10-28|1245|5418|416|2003|2|10|28|4|2003|416|5418|Tuesday|2003Q4|N|N|N|2452914|2453186|2452576|2452849|N|N|N|N|Y| +2452942|AAAAAAAAOMNGFCAA|2003-10-29|1245|5418|416|2003|3|10|29|4|2003|416|5418|Wednesday|2003Q4|N|N|N|2452914|2453186|2452577|2452850|N|N|N|N|Y| +2452943|AAAAAAAAPMNGFCAA|2003-10-30|1245|5418|416|2003|4|10|30|4|2003|416|5418|Thursday|2003Q4|N|N|N|2452914|2453186|2452578|2452851|N|N|N|N|Y| +2452944|AAAAAAAAANNGFCAA|2003-10-31|1245|5418|416|2003|5|10|31|4|2003|416|5418|Friday|2003Q4|N|Y|N|2452914|2453186|2452579|2452852|N|N|N|N|Y| +2452945|AAAAAAAABNNGFCAA|2003-11-01|1246|5418|416|2003|6|11|1|4|2003|416|5418|Saturday|2003Q4|N|Y|N|2452945|2453248|2452580|2452853|N|N|N|N|Y| +2452946|AAAAAAAACNNGFCAA|2003-11-02|1246|5418|416|2003|0|11|2|4|2003|416|5418|Sunday|2003Q4|N|N|N|2452945|2453248|2452581|2452854|N|N|N|N|Y| +2452947|AAAAAAAADNNGFCAA|2003-11-03|1246|5418|416|2003|1|11|3|4|2003|416|5418|Monday|2003Q4|N|N|N|2452945|2453248|2452582|2452855|N|N|N|N|Y| +2452948|AAAAAAAAENNGFCAA|2003-11-04|1246|5419|416|2003|2|11|4|4|2003|416|5419|Tuesday|2003Q4|N|N|N|2452945|2453248|2452583|2452856|N|N|N|N|Y| +2452949|AAAAAAAAFNNGFCAA|2003-11-05|1246|5419|416|2003|3|11|5|4|2003|416|5419|Wednesday|2003Q4|N|N|N|2452945|2453248|2452584|2452857|N|N|N|N|Y| +2452950|AAAAAAAAGNNGFCAA|2003-11-06|1246|5419|416|2003|4|11|6|4|2003|416|5419|Thursday|2003Q4|N|N|N|2452945|2453248|2452585|2452858|N|N|N|N|Y| +2452951|AAAAAAAAHNNGFCAA|2003-11-07|1246|5419|416|2003|5|11|7|4|2003|416|5419|Friday|2003Q4|N|Y|N|2452945|2453248|2452586|2452859|N|N|N|N|Y| +2452952|AAAAAAAAINNGFCAA|2003-11-08|1246|5419|416|2003|6|11|8|4|2003|416|5419|Saturday|2003Q4|N|Y|N|2452945|2453248|2452587|2452860|N|N|N|N|Y| +2452953|AAAAAAAAJNNGFCAA|2003-11-09|1246|5419|416|2003|0|11|9|4|2003|416|5419|Sunday|2003Q4|N|N|N|2452945|2453248|2452588|2452861|N|N|N|N|Y| +2452954|AAAAAAAAKNNGFCAA|2003-11-10|1246|5419|416|2003|1|11|10|4|2003|416|5419|Monday|2003Q4|N|N|N|2452945|2453248|2452589|2452862|N|N|N|N|Y| +2452955|AAAAAAAALNNGFCAA|2003-11-11|1246|5420|416|2003|2|11|11|4|2003|416|5420|Tuesday|2003Q4|N|N|N|2452945|2453248|2452590|2452863|N|N|N|N|Y| +2452956|AAAAAAAAMNNGFCAA|2003-11-12|1246|5420|416|2003|3|11|12|4|2003|416|5420|Wednesday|2003Q4|N|N|N|2452945|2453248|2452591|2452864|N|N|N|N|Y| +2452957|AAAAAAAANNNGFCAA|2003-11-13|1246|5420|416|2003|4|11|13|4|2003|416|5420|Thursday|2003Q4|N|N|N|2452945|2453248|2452592|2452865|N|N|N|N|Y| +2452958|AAAAAAAAONNGFCAA|2003-11-14|1246|5420|416|2003|5|11|14|4|2003|416|5420|Friday|2003Q4|N|Y|N|2452945|2453248|2452593|2452866|N|N|N|N|Y| +2452959|AAAAAAAAPNNGFCAA|2003-11-15|1246|5420|416|2003|6|11|15|4|2003|416|5420|Saturday|2003Q4|N|Y|N|2452945|2453248|2452594|2452867|N|N|N|N|Y| +2452960|AAAAAAAAAONGFCAA|2003-11-16|1246|5420|416|2003|0|11|16|4|2003|416|5420|Sunday|2003Q4|N|N|N|2452945|2453248|2452595|2452868|N|N|N|N|Y| +2452961|AAAAAAAABONGFCAA|2003-11-17|1246|5420|416|2003|1|11|17|4|2003|416|5420|Monday|2003Q4|N|N|N|2452945|2453248|2452596|2452869|N|N|N|N|Y| +2452962|AAAAAAAACONGFCAA|2003-11-18|1246|5421|416|2003|2|11|18|4|2003|416|5421|Tuesday|2003Q4|N|N|N|2452945|2453248|2452597|2452870|N|N|N|N|Y| +2452963|AAAAAAAADONGFCAA|2003-11-19|1246|5421|416|2003|3|11|19|4|2003|416|5421|Wednesday|2003Q4|N|N|N|2452945|2453248|2452598|2452871|N|N|N|N|Y| +2452964|AAAAAAAAEONGFCAA|2003-11-20|1246|5421|416|2003|4|11|20|4|2003|416|5421|Thursday|2003Q4|N|N|N|2452945|2453248|2452599|2452872|N|N|N|N|Y| +2452965|AAAAAAAAFONGFCAA|2003-11-21|1246|5421|416|2003|5|11|21|4|2003|416|5421|Friday|2003Q4|N|Y|N|2452945|2453248|2452600|2452873|N|N|N|N|Y| +2452966|AAAAAAAAGONGFCAA|2003-11-22|1246|5421|416|2003|6|11|22|4|2003|416|5421|Saturday|2003Q4|N|Y|N|2452945|2453248|2452601|2452874|N|N|N|N|Y| +2452967|AAAAAAAAHONGFCAA|2003-11-23|1246|5421|416|2003|0|11|23|4|2003|416|5421|Sunday|2003Q4|N|N|N|2452945|2453248|2452602|2452875|N|N|N|N|Y| +2452968|AAAAAAAAIONGFCAA|2003-11-24|1246|5421|416|2003|1|11|24|4|2003|416|5421|Monday|2003Q4|N|N|N|2452945|2453248|2452603|2452876|N|N|N|N|Y| +2452969|AAAAAAAAJONGFCAA|2003-11-25|1246|5422|416|2003|2|11|25|4|2003|416|5422|Tuesday|2003Q4|N|N|N|2452945|2453248|2452604|2452877|N|N|N|N|Y| +2452970|AAAAAAAAKONGFCAA|2003-11-26|1246|5422|416|2003|3|11|26|4|2003|416|5422|Wednesday|2003Q4|N|N|N|2452945|2453248|2452605|2452878|N|N|N|N|Y| +2452971|AAAAAAAALONGFCAA|2003-11-27|1246|5422|416|2003|4|11|27|4|2003|416|5422|Thursday|2003Q4|N|N|N|2452945|2453248|2452606|2452879|N|N|N|N|Y| +2452972|AAAAAAAAMONGFCAA|2003-11-28|1246|5422|416|2003|5|11|28|4|2003|416|5422|Friday|2003Q4|N|Y|N|2452945|2453248|2452607|2452880|N|N|N|N|Y| +2452973|AAAAAAAANONGFCAA|2003-11-29|1246|5422|416|2003|6|11|29|4|2003|416|5422|Saturday|2003Q4|N|Y|N|2452945|2453248|2452608|2452881|N|N|N|N|Y| +2452974|AAAAAAAAOONGFCAA|2003-11-30|1246|5422|416|2003|0|11|30|4|2003|416|5422|Sunday|2003Q4|N|N|N|2452945|2453248|2452609|2452882|N|N|N|N|Y| +2452975|AAAAAAAAPONGFCAA|2003-12-01|1247|5422|417|2003|1|12|1|4|2003|417|5422|Monday|2003Q4|N|N|N|2452975|2453308|2452610|2452883|N|N|N|N|Y| +2452976|AAAAAAAAAPNGFCAA|2003-12-02|1247|5423|417|2003|2|12|2|4|2003|417|5423|Tuesday|2003Q4|N|N|N|2452975|2453308|2452611|2452884|N|N|N|N|Y| +2452977|AAAAAAAABPNGFCAA|2003-12-03|1247|5423|417|2003|3|12|3|4|2003|417|5423|Wednesday|2003Q4|N|N|N|2452975|2453308|2452612|2452885|N|N|N|N|Y| +2452978|AAAAAAAACPNGFCAA|2003-12-04|1247|5423|417|2003|4|12|4|4|2003|417|5423|Thursday|2003Q4|N|N|N|2452975|2453308|2452613|2452886|N|N|N|N|Y| +2452979|AAAAAAAADPNGFCAA|2003-12-05|1247|5423|417|2003|5|12|5|4|2003|417|5423|Friday|2003Q4|N|Y|N|2452975|2453308|2452614|2452887|N|N|N|N|Y| +2452980|AAAAAAAAEPNGFCAA|2003-12-06|1247|5423|417|2003|6|12|6|4|2003|417|5423|Saturday|2003Q4|N|Y|N|2452975|2453308|2452615|2452888|N|N|N|N|Y| +2452981|AAAAAAAAFPNGFCAA|2003-12-07|1247|5423|417|2003|0|12|7|4|2003|417|5423|Sunday|2003Q4|N|N|N|2452975|2453308|2452616|2452889|N|N|N|N|Y| +2452982|AAAAAAAAGPNGFCAA|2003-12-08|1247|5423|417|2003|1|12|8|4|2003|417|5423|Monday|2003Q4|N|N|N|2452975|2453308|2452617|2452890|N|N|N|N|Y| +2452983|AAAAAAAAHPNGFCAA|2003-12-09|1247|5424|417|2003|2|12|9|4|2003|417|5424|Tuesday|2003Q4|N|N|N|2452975|2453308|2452618|2452891|N|N|N|N|Y| +2452984|AAAAAAAAIPNGFCAA|2003-12-10|1247|5424|417|2003|3|12|10|4|2003|417|5424|Wednesday|2003Q4|N|N|N|2452975|2453308|2452619|2452892|N|N|N|N|Y| +2452985|AAAAAAAAJPNGFCAA|2003-12-11|1247|5424|417|2003|4|12|11|4|2003|417|5424|Thursday|2003Q4|N|N|N|2452975|2453308|2452620|2452893|N|N|N|N|Y| +2452986|AAAAAAAAKPNGFCAA|2003-12-12|1247|5424|417|2003|5|12|12|4|2003|417|5424|Friday|2003Q4|N|Y|N|2452975|2453308|2452621|2452894|N|N|N|N|Y| +2452987|AAAAAAAALPNGFCAA|2003-12-13|1247|5424|417|2003|6|12|13|4|2003|417|5424|Saturday|2003Q4|N|Y|N|2452975|2453308|2452622|2452895|N|N|N|N|Y| +2452988|AAAAAAAAMPNGFCAA|2003-12-14|1247|5424|417|2003|0|12|14|4|2003|417|5424|Sunday|2003Q4|N|N|N|2452975|2453308|2452623|2452896|N|N|N|N|Y| +2452989|AAAAAAAANPNGFCAA|2003-12-15|1247|5424|417|2003|1|12|15|4|2003|417|5424|Monday|2003Q4|N|N|N|2452975|2453308|2452624|2452897|N|N|N|N|Y| +2452990|AAAAAAAAOPNGFCAA|2003-12-16|1247|5425|417|2003|2|12|16|4|2003|417|5425|Tuesday|2003Q4|N|N|N|2452975|2453308|2452625|2452898|N|N|N|N|Y| +2452991|AAAAAAAAPPNGFCAA|2003-12-17|1247|5425|417|2003|3|12|17|4|2003|417|5425|Wednesday|2003Q4|N|N|N|2452975|2453308|2452626|2452899|N|N|N|N|Y| +2452992|AAAAAAAAAAOGFCAA|2003-12-18|1247|5425|417|2003|4|12|18|4|2003|417|5425|Thursday|2003Q4|N|N|N|2452975|2453308|2452627|2452900|N|N|N|N|Y| +2452993|AAAAAAAABAOGFCAA|2003-12-19|1247|5425|417|2003|5|12|19|4|2003|417|5425|Friday|2003Q4|N|Y|N|2452975|2453308|2452628|2452901|N|N|N|N|Y| +2452994|AAAAAAAACAOGFCAA|2003-12-20|1247|5425|417|2003|6|12|20|4|2003|417|5425|Saturday|2003Q4|N|Y|N|2452975|2453308|2452629|2452902|N|N|N|N|Y| +2452995|AAAAAAAADAOGFCAA|2003-12-21|1247|5425|417|2003|0|12|21|4|2003|417|5425|Sunday|2003Q4|N|N|N|2452975|2453308|2452630|2452903|N|N|N|N|Y| +2452996|AAAAAAAAEAOGFCAA|2003-12-22|1247|5425|417|2003|1|12|22|4|2003|417|5425|Monday|2003Q4|N|N|N|2452975|2453308|2452631|2452904|N|N|N|N|Y| +2452997|AAAAAAAAFAOGFCAA|2003-12-23|1247|5426|417|2003|2|12|23|4|2003|417|5426|Tuesday|2003Q4|N|N|N|2452975|2453308|2452632|2452905|N|N|N|N|Y| +2452998|AAAAAAAAGAOGFCAA|2003-12-24|1247|5426|417|2003|3|12|24|4|2003|417|5426|Wednesday|2003Q4|N|N|N|2452975|2453308|2452633|2452906|N|N|N|N|Y| +2452999|AAAAAAAAHAOGFCAA|2003-12-25|1247|5426|417|2003|4|12|25|4|2003|417|5426|Thursday|2003Q4|N|N|N|2452975|2453308|2452634|2452907|N|N|N|N|Y| +2453000|AAAAAAAAIAOGFCAA|2003-12-26|1247|5426|417|2003|5|12|26|4|2003|417|5426|Friday|2003Q4|Y|Y|N|2452975|2453308|2452635|2452908|N|N|N|N|Y| +2453001|AAAAAAAAJAOGFCAA|2003-12-27|1247|5426|417|2003|6|12|27|4|2003|417|5426|Saturday|2003Q4|N|Y|Y|2452975|2453308|2452636|2452909|N|N|N|N|Y| +2453002|AAAAAAAAKAOGFCAA|2003-12-28|1247|5426|417|2003|0|12|28|4|2003|417|5426|Sunday|2003Q4|N|N|N|2452975|2453308|2452637|2452910|N|N|N|N|Y| +2453003|AAAAAAAALAOGFCAA|2003-12-29|1247|5426|417|2003|1|12|29|4|2003|417|5426|Monday|2003Q4|N|N|N|2452975|2453308|2452638|2452911|N|N|N|N|Y| +2453004|AAAAAAAAMAOGFCAA|2003-12-30|1247|5427|417|2003|2|12|30|4|2003|417|5427|Tuesday|2003Q4|N|N|N|2452975|2453308|2452639|2452912|N|N|N|N|Y| +2453005|AAAAAAAANAOGFCAA|2003-12-31|1247|5427|417|2003|3|12|31|4|2003|417|5427|Wednesday|2003Q4|N|N|N|2452975|2453308|2452640|2452913|N|N|N|N|Y| +2453006|AAAAAAAAOAOGFCAA|2004-01-01|1248|5427|417|2004|4|1|1|1|2004|417|5427|Thursday|2004Q1|Y|N|N|2453006|2453005|2452641|2452914|N|N|N|N|N| +2453007|AAAAAAAAPAOGFCAA|2004-01-02|1248|5427|417|2004|5|1|2|1|2004|417|5427|Friday|2004Q1|N|Y|Y|2453006|2453005|2452642|2452915|N|N|N|N|N| +2453008|AAAAAAAAABOGFCAA|2004-01-03|1248|5427|417|2004|6|1|3|1|2004|417|5427|Saturday|2004Q1|N|Y|N|2453006|2453005|2452643|2452916|N|N|N|N|N| +2453009|AAAAAAAABBOGFCAA|2004-01-04|1248|5427|417|2004|0|1|4|1|2004|417|5427|Sunday|2004Q1|N|N|N|2453006|2453005|2452644|2452917|N|N|N|N|N| +2453010|AAAAAAAACBOGFCAA|2004-01-05|1248|5427|417|2004|1|1|5|1|2004|417|5427|Monday|2004Q1|N|N|N|2453006|2453005|2452645|2452918|N|N|N|N|N| +2453011|AAAAAAAADBOGFCAA|2004-01-06|1248|5428|417|2004|2|1|6|1|2004|417|5428|Tuesday|2004Q1|N|N|N|2453006|2453005|2452646|2452919|N|N|N|N|N| +2453012|AAAAAAAAEBOGFCAA|2004-01-07|1248|5428|417|2004|3|1|7|1|2004|417|5428|Wednesday|2004Q1|N|N|N|2453006|2453005|2452647|2452920|N|N|N|N|N| +2453013|AAAAAAAAFBOGFCAA|2004-01-08|1248|5428|417|2004|4|1|8|1|2004|417|5428|Thursday|2004Q1|N|N|N|2453006|2453005|2452648|2452921|N|N|N|N|N| +2453014|AAAAAAAAGBOGFCAA|2004-01-09|1248|5428|417|2004|5|1|9|1|2004|417|5428|Friday|2004Q1|N|Y|N|2453006|2453005|2452649|2452922|N|N|N|N|N| +2453015|AAAAAAAAHBOGFCAA|2004-01-10|1248|5428|417|2004|6|1|10|1|2004|417|5428|Saturday|2004Q1|N|Y|N|2453006|2453005|2452650|2452923|N|N|N|N|N| +2453016|AAAAAAAAIBOGFCAA|2004-01-11|1248|5428|417|2004|0|1|11|1|2004|417|5428|Sunday|2004Q1|N|N|N|2453006|2453005|2452651|2452924|N|N|N|N|N| +2453017|AAAAAAAAJBOGFCAA|2004-01-12|1248|5428|417|2004|1|1|12|1|2004|417|5428|Monday|2004Q1|N|N|N|2453006|2453005|2452652|2452925|N|N|N|N|N| +2453018|AAAAAAAAKBOGFCAA|2004-01-13|1248|5429|417|2004|2|1|13|1|2004|417|5429|Tuesday|2004Q1|N|N|N|2453006|2453005|2452653|2452926|N|N|N|N|N| +2453019|AAAAAAAALBOGFCAA|2004-01-14|1248|5429|417|2004|3|1|14|1|2004|417|5429|Wednesday|2004Q1|N|N|N|2453006|2453005|2452654|2452927|N|N|N|N|N| +2453020|AAAAAAAAMBOGFCAA|2004-01-15|1248|5429|417|2004|4|1|15|1|2004|417|5429|Thursday|2004Q1|N|N|N|2453006|2453005|2452655|2452928|N|N|N|N|N| +2453021|AAAAAAAANBOGFCAA|2004-01-16|1248|5429|417|2004|5|1|16|1|2004|417|5429|Friday|2004Q1|N|Y|N|2453006|2453005|2452656|2452929|N|N|N|N|N| +2453022|AAAAAAAAOBOGFCAA|2004-01-17|1248|5429|417|2004|6|1|17|1|2004|417|5429|Saturday|2004Q1|N|Y|N|2453006|2453005|2452657|2452930|N|N|N|N|N| +2453023|AAAAAAAAPBOGFCAA|2004-01-18|1248|5429|417|2004|0|1|18|1|2004|417|5429|Sunday|2004Q1|N|N|N|2453006|2453005|2452658|2452931|N|N|N|N|N| +2453024|AAAAAAAAACOGFCAA|2004-01-19|1248|5429|417|2004|1|1|19|1|2004|417|5429|Monday|2004Q1|N|N|N|2453006|2453005|2452659|2452932|N|N|N|N|N| +2453025|AAAAAAAABCOGFCAA|2004-01-20|1248|5430|417|2004|2|1|20|1|2004|417|5430|Tuesday|2004Q1|N|N|N|2453006|2453005|2452660|2452933|N|N|N|N|N| +2453026|AAAAAAAACCOGFCAA|2004-01-21|1248|5430|417|2004|3|1|21|1|2004|417|5430|Wednesday|2004Q1|N|N|N|2453006|2453005|2452661|2452934|N|N|N|N|N| +2453027|AAAAAAAADCOGFCAA|2004-01-22|1248|5430|417|2004|4|1|22|1|2004|417|5430|Thursday|2004Q1|N|N|N|2453006|2453005|2452662|2452935|N|N|N|N|N| +2453028|AAAAAAAAECOGFCAA|2004-01-23|1248|5430|417|2004|5|1|23|1|2004|417|5430|Friday|2004Q1|N|Y|N|2453006|2453005|2452663|2452936|N|N|N|N|N| +2453029|AAAAAAAAFCOGFCAA|2004-01-24|1248|5430|417|2004|6|1|24|1|2004|417|5430|Saturday|2004Q1|N|Y|N|2453006|2453005|2452664|2452937|N|N|N|N|N| +2453030|AAAAAAAAGCOGFCAA|2004-01-25|1248|5430|417|2004|0|1|25|1|2004|417|5430|Sunday|2004Q1|N|N|N|2453006|2453005|2452665|2452938|N|N|N|N|N| +2453031|AAAAAAAAHCOGFCAA|2004-01-26|1248|5430|417|2004|1|1|26|1|2004|417|5430|Monday|2004Q1|N|N|N|2453006|2453005|2452666|2452939|N|N|N|N|N| +2453032|AAAAAAAAICOGFCAA|2004-01-27|1248|5431|417|2004|2|1|27|1|2004|417|5431|Tuesday|2004Q1|N|N|N|2453006|2453005|2452667|2452940|N|N|N|N|N| +2453033|AAAAAAAAJCOGFCAA|2004-01-28|1248|5431|417|2004|3|1|28|1|2004|417|5431|Wednesday|2004Q1|N|N|N|2453006|2453005|2452668|2452941|N|N|N|N|N| +2453034|AAAAAAAAKCOGFCAA|2004-01-29|1248|5431|417|2004|4|1|29|1|2004|417|5431|Thursday|2004Q1|N|N|N|2453006|2453005|2452669|2452942|N|N|N|N|N| +2453035|AAAAAAAALCOGFCAA|2004-01-30|1248|5431|417|2004|5|1|30|1|2004|417|5431|Friday|2004Q1|N|Y|N|2453006|2453005|2452670|2452943|N|N|N|N|N| +2453036|AAAAAAAAMCOGFCAA|2004-01-31|1248|5431|417|2004|6|1|31|1|2004|417|5431|Saturday|2004Q1|N|Y|N|2453006|2453005|2452671|2452944|N|N|N|N|N| +2453037|AAAAAAAANCOGFCAA|2004-02-01|1249|5431|417|2004|0|2|1|1|2004|417|5431|Sunday|2004Q1|N|N|N|2453037|2453067|2452672|2452945|N|N|N|N|N| +2453038|AAAAAAAAOCOGFCAA|2004-02-02|1249|5431|417|2004|1|2|2|1|2004|417|5431|Monday|2004Q1|N|N|N|2453037|2453067|2452673|2452946|N|N|N|N|N| +2453039|AAAAAAAAPCOGFCAA|2004-02-03|1249|5432|417|2004|2|2|3|1|2004|417|5432|Tuesday|2004Q1|N|N|N|2453037|2453067|2452674|2452947|N|N|N|N|N| +2453040|AAAAAAAAADOGFCAA|2004-02-04|1249|5432|417|2004|3|2|4|1|2004|417|5432|Wednesday|2004Q1|N|N|N|2453037|2453067|2452675|2452948|N|N|N|N|N| +2453041|AAAAAAAABDOGFCAA|2004-02-05|1249|5432|417|2004|4|2|5|1|2004|417|5432|Thursday|2004Q1|N|N|N|2453037|2453067|2452676|2452949|N|N|N|N|N| +2453042|AAAAAAAACDOGFCAA|2004-02-06|1249|5432|417|2004|5|2|6|1|2004|417|5432|Friday|2004Q1|N|Y|N|2453037|2453067|2452677|2452950|N|N|N|N|N| +2453043|AAAAAAAADDOGFCAA|2004-02-07|1249|5432|417|2004|6|2|7|1|2004|417|5432|Saturday|2004Q1|N|Y|N|2453037|2453067|2452678|2452951|N|N|N|N|N| +2453044|AAAAAAAAEDOGFCAA|2004-02-08|1249|5432|417|2004|0|2|8|1|2004|417|5432|Sunday|2004Q1|N|N|N|2453037|2453067|2452679|2452952|N|N|N|N|N| +2453045|AAAAAAAAFDOGFCAA|2004-02-09|1249|5432|417|2004|1|2|9|1|2004|417|5432|Monday|2004Q1|N|N|N|2453037|2453067|2452680|2452953|N|N|N|N|N| +2453046|AAAAAAAAGDOGFCAA|2004-02-10|1249|5433|417|2004|2|2|10|1|2004|417|5433|Tuesday|2004Q1|N|N|N|2453037|2453067|2452681|2452954|N|N|N|N|N| +2453047|AAAAAAAAHDOGFCAA|2004-02-11|1249|5433|417|2004|3|2|11|1|2004|417|5433|Wednesday|2004Q1|N|N|N|2453037|2453067|2452682|2452955|N|N|N|N|N| +2453048|AAAAAAAAIDOGFCAA|2004-02-12|1249|5433|417|2004|4|2|12|1|2004|417|5433|Thursday|2004Q1|N|N|N|2453037|2453067|2452683|2452956|N|N|N|N|N| +2453049|AAAAAAAAJDOGFCAA|2004-02-13|1249|5433|417|2004|5|2|13|1|2004|417|5433|Friday|2004Q1|N|Y|N|2453037|2453067|2452684|2452957|N|N|N|N|N| +2453050|AAAAAAAAKDOGFCAA|2004-02-14|1249|5433|417|2004|6|2|14|1|2004|417|5433|Saturday|2004Q1|N|Y|N|2453037|2453067|2452685|2452958|N|N|N|N|N| +2453051|AAAAAAAALDOGFCAA|2004-02-15|1249|5433|417|2004|0|2|15|1|2004|417|5433|Sunday|2004Q1|N|N|N|2453037|2453067|2452686|2452959|N|N|N|N|N| +2453052|AAAAAAAAMDOGFCAA|2004-02-16|1249|5433|417|2004|1|2|16|1|2004|417|5433|Monday|2004Q1|N|N|N|2453037|2453067|2452687|2452960|N|N|N|N|N| +2453053|AAAAAAAANDOGFCAA|2004-02-17|1249|5434|417|2004|2|2|17|1|2004|417|5434|Tuesday|2004Q1|N|N|N|2453037|2453067|2452688|2452961|N|N|N|N|N| +2453054|AAAAAAAAODOGFCAA|2004-02-18|1249|5434|417|2004|3|2|18|1|2004|417|5434|Wednesday|2004Q1|N|N|N|2453037|2453067|2452689|2452962|N|N|N|N|N| +2453055|AAAAAAAAPDOGFCAA|2004-02-19|1249|5434|417|2004|4|2|19|1|2004|417|5434|Thursday|2004Q1|N|N|N|2453037|2453067|2452690|2452963|N|N|N|N|N| +2453056|AAAAAAAAAEOGFCAA|2004-02-20|1249|5434|417|2004|5|2|20|1|2004|417|5434|Friday|2004Q1|N|Y|N|2453037|2453067|2452691|2452964|N|N|N|N|N| +2453057|AAAAAAAABEOGFCAA|2004-02-21|1249|5434|417|2004|6|2|21|1|2004|417|5434|Saturday|2004Q1|N|Y|N|2453037|2453067|2452692|2452965|N|N|N|N|N| +2453058|AAAAAAAACEOGFCAA|2004-02-22|1249|5434|417|2004|0|2|22|1|2004|417|5434|Sunday|2004Q1|N|N|N|2453037|2453067|2452693|2452966|N|N|N|N|N| +2453059|AAAAAAAADEOGFCAA|2004-02-23|1249|5434|417|2004|1|2|23|1|2004|417|5434|Monday|2004Q1|N|N|N|2453037|2453067|2452694|2452967|N|N|N|N|N| +2453060|AAAAAAAAEEOGFCAA|2004-02-24|1249|5435|417|2004|2|2|24|1|2004|417|5435|Tuesday|2004Q1|N|N|N|2453037|2453067|2452695|2452968|N|N|N|N|N| +2453061|AAAAAAAAFEOGFCAA|2004-02-25|1249|5435|417|2004|3|2|25|1|2004|417|5435|Wednesday|2004Q1|N|N|N|2453037|2453067|2452696|2452969|N|N|N|N|N| +2453062|AAAAAAAAGEOGFCAA|2004-02-26|1249|5435|417|2004|4|2|26|1|2004|417|5435|Thursday|2004Q1|N|N|N|2453037|2453067|2452697|2452970|N|N|N|N|N| +2453063|AAAAAAAAHEOGFCAA|2004-02-27|1249|5435|417|2004|5|2|27|1|2004|417|5435|Friday|2004Q1|N|Y|N|2453037|2453067|2452698|2452971|N|N|N|N|N| +2453064|AAAAAAAAIEOGFCAA|2004-02-28|1249|5435|417|2004|6|2|28|1|2004|417|5435|Saturday|2004Q1|N|Y|N|2453037|2453067|2452699|2452972|N|N|N|N|N| +2453065|AAAAAAAAJEOGFCAA|2004-02-29|1249|5435|417|2004|0|2|29|1|2004|417|5435|Sunday|2004Q1|N|N|N|2453037|2453067|2452699|2452973|N|N|N|N|N| +2453066|AAAAAAAAKEOGFCAA|2004-03-01|1250|5435|418|2004|1|3|1|1|2004|418|5435|Monday|2004Q1|N|N|N|2453066|2453125|2452700|2452974|N|N|N|N|N| +2453067|AAAAAAAALEOGFCAA|2004-03-02|1250|5436|418|2004|2|3|2|1|2004|418|5436|Tuesday|2004Q1|N|N|N|2453066|2453125|2452701|2452975|N|N|N|N|N| +2453068|AAAAAAAAMEOGFCAA|2004-03-03|1250|5436|418|2004|3|3|3|1|2004|418|5436|Wednesday|2004Q1|N|N|N|2453066|2453125|2452702|2452976|N|N|N|N|N| +2453069|AAAAAAAANEOGFCAA|2004-03-04|1250|5436|418|2004|4|3|4|1|2004|418|5436|Thursday|2004Q1|N|N|N|2453066|2453125|2452703|2452977|N|N|N|N|N| +2453070|AAAAAAAAOEOGFCAA|2004-03-05|1250|5436|418|2004|5|3|5|1|2004|418|5436|Friday|2004Q1|N|Y|N|2453066|2453125|2452704|2452978|N|N|N|N|N| +2453071|AAAAAAAAPEOGFCAA|2004-03-06|1250|5436|418|2004|6|3|6|1|2004|418|5436|Saturday|2004Q1|N|Y|N|2453066|2453125|2452705|2452979|N|N|N|N|N| +2453072|AAAAAAAAAFOGFCAA|2004-03-07|1250|5436|418|2004|0|3|7|1|2004|418|5436|Sunday|2004Q1|N|N|N|2453066|2453125|2452706|2452980|N|N|N|N|N| +2453073|AAAAAAAABFOGFCAA|2004-03-08|1250|5436|418|2004|1|3|8|1|2004|418|5436|Monday|2004Q1|N|N|N|2453066|2453125|2452707|2452981|N|N|N|N|N| +2453074|AAAAAAAACFOGFCAA|2004-03-09|1250|5437|418|2004|2|3|9|1|2004|418|5437|Tuesday|2004Q1|N|N|N|2453066|2453125|2452708|2452982|N|N|N|N|N| +2453075|AAAAAAAADFOGFCAA|2004-03-10|1250|5437|418|2004|3|3|10|1|2004|418|5437|Wednesday|2004Q1|N|N|N|2453066|2453125|2452709|2452983|N|N|N|N|N| +2453076|AAAAAAAAEFOGFCAA|2004-03-11|1250|5437|418|2004|4|3|11|1|2004|418|5437|Thursday|2004Q1|N|N|N|2453066|2453125|2452710|2452984|N|N|N|N|N| +2453077|AAAAAAAAFFOGFCAA|2004-03-12|1250|5437|418|2004|5|3|12|1|2004|418|5437|Friday|2004Q1|N|Y|N|2453066|2453125|2452711|2452985|N|N|N|N|N| +2453078|AAAAAAAAGFOGFCAA|2004-03-13|1250|5437|418|2004|6|3|13|1|2004|418|5437|Saturday|2004Q1|N|Y|N|2453066|2453125|2452712|2452986|N|N|N|N|N| +2453079|AAAAAAAAHFOGFCAA|2004-03-14|1250|5437|418|2004|0|3|14|1|2004|418|5437|Sunday|2004Q1|N|N|N|2453066|2453125|2452713|2452987|N|N|N|N|N| +2453080|AAAAAAAAIFOGFCAA|2004-03-15|1250|5437|418|2004|1|3|15|1|2004|418|5437|Monday|2004Q1|N|N|N|2453066|2453125|2452714|2452988|N|N|N|N|N| +2453081|AAAAAAAAJFOGFCAA|2004-03-16|1250|5438|418|2004|2|3|16|1|2004|418|5438|Tuesday|2004Q1|N|N|N|2453066|2453125|2452715|2452989|N|N|N|N|N| +2453082|AAAAAAAAKFOGFCAA|2004-03-17|1250|5438|418|2004|3|3|17|1|2004|418|5438|Wednesday|2004Q1|N|N|N|2453066|2453125|2452716|2452990|N|N|N|N|N| +2453083|AAAAAAAALFOGFCAA|2004-03-18|1250|5438|418|2004|4|3|18|1|2004|418|5438|Thursday|2004Q1|N|N|N|2453066|2453125|2452717|2452991|N|N|N|N|N| +2453084|AAAAAAAAMFOGFCAA|2004-03-19|1250|5438|418|2004|5|3|19|1|2004|418|5438|Friday|2004Q1|N|Y|N|2453066|2453125|2452718|2452992|N|N|N|N|N| +2453085|AAAAAAAANFOGFCAA|2004-03-20|1250|5438|418|2004|6|3|20|1|2004|418|5438|Saturday|2004Q1|N|Y|N|2453066|2453125|2452719|2452993|N|N|N|N|N| +2453086|AAAAAAAAOFOGFCAA|2004-03-21|1250|5438|418|2004|0|3|21|1|2004|418|5438|Sunday|2004Q1|N|N|N|2453066|2453125|2452720|2452994|N|N|N|N|N| +2453087|AAAAAAAAPFOGFCAA|2004-03-22|1250|5438|418|2004|1|3|22|1|2004|418|5438|Monday|2004Q1|N|N|N|2453066|2453125|2452721|2452995|N|N|N|N|N| +2453088|AAAAAAAAAGOGFCAA|2004-03-23|1250|5439|418|2004|2|3|23|1|2004|418|5439|Tuesday|2004Q1|N|N|N|2453066|2453125|2452722|2452996|N|N|N|N|N| +2453089|AAAAAAAABGOGFCAA|2004-03-24|1250|5439|418|2004|3|3|24|1|2004|418|5439|Wednesday|2004Q1|N|N|N|2453066|2453125|2452723|2452997|N|N|N|N|N| +2453090|AAAAAAAACGOGFCAA|2004-03-25|1250|5439|418|2004|4|3|25|1|2004|418|5439|Thursday|2004Q1|N|N|N|2453066|2453125|2452724|2452998|N|N|N|N|N| +2453091|AAAAAAAADGOGFCAA|2004-03-26|1250|5439|418|2004|5|3|26|1|2004|418|5439|Friday|2004Q1|N|Y|N|2453066|2453125|2452725|2452999|N|N|N|N|N| +2453092|AAAAAAAAEGOGFCAA|2004-03-27|1250|5439|418|2004|6|3|27|1|2004|418|5439|Saturday|2004Q1|N|Y|N|2453066|2453125|2452726|2453000|N|N|N|N|N| +2453093|AAAAAAAAFGOGFCAA|2004-03-28|1250|5439|418|2004|0|3|28|1|2004|418|5439|Sunday|2004Q1|N|N|N|2453066|2453125|2452727|2453001|N|N|N|N|N| +2453094|AAAAAAAAGGOGFCAA|2004-03-29|1250|5439|418|2004|1|3|29|1|2004|418|5439|Monday|2004Q1|N|N|N|2453066|2453125|2452728|2453002|N|N|N|N|N| +2453095|AAAAAAAAHGOGFCAA|2004-03-30|1250|5440|418|2004|2|3|30|1|2004|418|5440|Tuesday|2004Q1|N|N|N|2453066|2453125|2452729|2453003|N|N|N|N|N| +2453096|AAAAAAAAIGOGFCAA|2004-03-31|1250|5440|418|2004|3|3|31|1|2004|418|5440|Wednesday|2004Q1|N|N|N|2453066|2453125|2452730|2453004|N|N|N|N|N| +2453097|AAAAAAAAJGOGFCAA|2004-04-01|1251|5440|418|2004|4|4|1|2|2004|418|5440|Thursday|2004Q2|N|N|N|2453097|2453187|2452731|2453006|N|N|N|N|N| +2453098|AAAAAAAAKGOGFCAA|2004-04-02|1251|5440|418|2004|5|4|2|2|2004|418|5440|Friday|2004Q2|N|Y|N|2453097|2453187|2452732|2453007|N|N|N|N|N| +2453099|AAAAAAAALGOGFCAA|2004-04-03|1251|5440|418|2004|6|4|3|2|2004|418|5440|Saturday|2004Q2|N|Y|N|2453097|2453187|2452733|2453008|N|N|N|N|N| +2453100|AAAAAAAAMGOGFCAA|2004-04-04|1251|5440|418|2004|0|4|4|2|2004|418|5440|Sunday|2004Q2|N|N|N|2453097|2453187|2452734|2453009|N|N|N|N|N| +2453101|AAAAAAAANGOGFCAA|2004-04-05|1251|5440|418|2004|1|4|5|2|2004|418|5440|Monday|2004Q2|N|N|N|2453097|2453187|2452735|2453010|N|N|N|N|N| +2453102|AAAAAAAAOGOGFCAA|2004-04-06|1251|5441|418|2004|2|4|6|2|2004|418|5441|Tuesday|2004Q2|N|N|N|2453097|2453187|2452736|2453011|N|N|N|N|N| +2453103|AAAAAAAAPGOGFCAA|2004-04-07|1251|5441|418|2004|3|4|7|2|2004|418|5441|Wednesday|2004Q2|N|N|N|2453097|2453187|2452737|2453012|N|N|N|N|N| +2453104|AAAAAAAAAHOGFCAA|2004-04-08|1251|5441|418|2004|4|4|8|2|2004|418|5441|Thursday|2004Q2|N|N|N|2453097|2453187|2452738|2453013|N|N|N|N|N| +2453105|AAAAAAAABHOGFCAA|2004-04-09|1251|5441|418|2004|5|4|9|2|2004|418|5441|Friday|2004Q2|N|Y|N|2453097|2453187|2452739|2453014|N|N|N|N|N| +2453106|AAAAAAAACHOGFCAA|2004-04-10|1251|5441|418|2004|6|4|10|2|2004|418|5441|Saturday|2004Q2|N|Y|N|2453097|2453187|2452740|2453015|N|N|N|N|N| +2453107|AAAAAAAADHOGFCAA|2004-04-11|1251|5441|418|2004|0|4|11|2|2004|418|5441|Sunday|2004Q2|N|N|N|2453097|2453187|2452741|2453016|N|N|N|N|N| +2453108|AAAAAAAAEHOGFCAA|2004-04-12|1251|5441|418|2004|1|4|12|2|2004|418|5441|Monday|2004Q2|N|N|N|2453097|2453187|2452742|2453017|N|N|N|N|N| +2453109|AAAAAAAAFHOGFCAA|2004-04-13|1251|5442|418|2004|2|4|13|2|2004|418|5442|Tuesday|2004Q2|N|N|N|2453097|2453187|2452743|2453018|N|N|N|N|N| +2453110|AAAAAAAAGHOGFCAA|2004-04-14|1251|5442|418|2004|3|4|14|2|2004|418|5442|Wednesday|2004Q2|N|N|N|2453097|2453187|2452744|2453019|N|N|N|N|N| +2453111|AAAAAAAAHHOGFCAA|2004-04-15|1251|5442|418|2004|4|4|15|2|2004|418|5442|Thursday|2004Q2|N|N|N|2453097|2453187|2452745|2453020|N|N|N|N|N| +2453112|AAAAAAAAIHOGFCAA|2004-04-16|1251|5442|418|2004|5|4|16|2|2004|418|5442|Friday|2004Q2|N|Y|N|2453097|2453187|2452746|2453021|N|N|N|N|N| +2453113|AAAAAAAAJHOGFCAA|2004-04-17|1251|5442|418|2004|6|4|17|2|2004|418|5442|Saturday|2004Q2|N|Y|N|2453097|2453187|2452747|2453022|N|N|N|N|N| +2453114|AAAAAAAAKHOGFCAA|2004-04-18|1251|5442|418|2004|0|4|18|2|2004|418|5442|Sunday|2004Q2|N|N|N|2453097|2453187|2452748|2453023|N|N|N|N|N| +2453115|AAAAAAAALHOGFCAA|2004-04-19|1251|5442|418|2004|1|4|19|2|2004|418|5442|Monday|2004Q2|N|N|N|2453097|2453187|2452749|2453024|N|N|N|N|N| +2453116|AAAAAAAAMHOGFCAA|2004-04-20|1251|5443|418|2004|2|4|20|2|2004|418|5443|Tuesday|2004Q2|N|N|N|2453097|2453187|2452750|2453025|N|N|N|N|N| +2453117|AAAAAAAANHOGFCAA|2004-04-21|1251|5443|418|2004|3|4|21|2|2004|418|5443|Wednesday|2004Q2|N|N|N|2453097|2453187|2452751|2453026|N|N|N|N|N| +2453118|AAAAAAAAOHOGFCAA|2004-04-22|1251|5443|418|2004|4|4|22|2|2004|418|5443|Thursday|2004Q2|N|N|N|2453097|2453187|2452752|2453027|N|N|N|N|N| +2453119|AAAAAAAAPHOGFCAA|2004-04-23|1251|5443|418|2004|5|4|23|2|2004|418|5443|Friday|2004Q2|N|Y|N|2453097|2453187|2452753|2453028|N|N|N|N|N| +2453120|AAAAAAAAAIOGFCAA|2004-04-24|1251|5443|418|2004|6|4|24|2|2004|418|5443|Saturday|2004Q2|N|Y|N|2453097|2453187|2452754|2453029|N|N|N|N|N| +2453121|AAAAAAAABIOGFCAA|2004-04-25|1251|5443|418|2004|0|4|25|2|2004|418|5443|Sunday|2004Q2|N|N|N|2453097|2453187|2452755|2453030|N|N|N|N|N| +2453122|AAAAAAAACIOGFCAA|2004-04-26|1251|5443|418|2004|1|4|26|2|2004|418|5443|Monday|2004Q2|N|N|N|2453097|2453187|2452756|2453031|N|N|N|N|N| +2453123|AAAAAAAADIOGFCAA|2004-04-27|1251|5444|418|2004|2|4|27|2|2004|418|5444|Tuesday|2004Q2|N|N|N|2453097|2453187|2452757|2453032|N|N|N|N|N| +2453124|AAAAAAAAEIOGFCAA|2004-04-28|1251|5444|418|2004|3|4|28|2|2004|418|5444|Wednesday|2004Q2|N|N|N|2453097|2453187|2452758|2453033|N|N|N|N|N| +2453125|AAAAAAAAFIOGFCAA|2004-04-29|1251|5444|418|2004|4|4|29|2|2004|418|5444|Thursday|2004Q2|N|N|N|2453097|2453187|2452759|2453034|N|N|N|N|N| +2453126|AAAAAAAAGIOGFCAA|2004-04-30|1251|5444|418|2004|5|4|30|2|2004|418|5444|Friday|2004Q2|N|Y|N|2453097|2453187|2452760|2453035|N|N|N|N|N| +2453127|AAAAAAAAHIOGFCAA|2004-05-01|1252|5444|418|2004|6|5|1|2|2004|418|5444|Saturday|2004Q2|N|Y|N|2453127|2453247|2452761|2453036|N|N|N|N|N| +2453128|AAAAAAAAIIOGFCAA|2004-05-02|1252|5444|418|2004|0|5|2|2|2004|418|5444|Sunday|2004Q2|N|N|N|2453127|2453247|2452762|2453037|N|N|N|N|N| +2453129|AAAAAAAAJIOGFCAA|2004-05-03|1252|5444|418|2004|1|5|3|2|2004|418|5444|Monday|2004Q2|N|N|N|2453127|2453247|2452763|2453038|N|N|N|N|N| +2453130|AAAAAAAAKIOGFCAA|2004-05-04|1252|5445|418|2004|2|5|4|2|2004|418|5445|Tuesday|2004Q2|N|N|N|2453127|2453247|2452764|2453039|N|N|N|N|N| +2453131|AAAAAAAALIOGFCAA|2004-05-05|1252|5445|418|2004|3|5|5|2|2004|418|5445|Wednesday|2004Q2|N|N|N|2453127|2453247|2452765|2453040|N|N|N|N|N| +2453132|AAAAAAAAMIOGFCAA|2004-05-06|1252|5445|418|2004|4|5|6|2|2004|418|5445|Thursday|2004Q2|N|N|N|2453127|2453247|2452766|2453041|N|N|N|N|N| +2453133|AAAAAAAANIOGFCAA|2004-05-07|1252|5445|418|2004|5|5|7|2|2004|418|5445|Friday|2004Q2|N|Y|N|2453127|2453247|2452767|2453042|N|N|N|N|N| +2453134|AAAAAAAAOIOGFCAA|2004-05-08|1252|5445|418|2004|6|5|8|2|2004|418|5445|Saturday|2004Q2|N|Y|N|2453127|2453247|2452768|2453043|N|N|N|N|N| +2453135|AAAAAAAAPIOGFCAA|2004-05-09|1252|5445|418|2004|0|5|9|2|2004|418|5445|Sunday|2004Q2|N|N|N|2453127|2453247|2452769|2453044|N|N|N|N|N| +2453136|AAAAAAAAAJOGFCAA|2004-05-10|1252|5445|418|2004|1|5|10|2|2004|418|5445|Monday|2004Q2|N|N|N|2453127|2453247|2452770|2453045|N|N|N|N|N| +2453137|AAAAAAAABJOGFCAA|2004-05-11|1252|5446|418|2004|2|5|11|2|2004|418|5446|Tuesday|2004Q2|N|N|N|2453127|2453247|2452771|2453046|N|N|N|N|N| +2453138|AAAAAAAACJOGFCAA|2004-05-12|1252|5446|418|2004|3|5|12|2|2004|418|5446|Wednesday|2004Q2|N|N|N|2453127|2453247|2452772|2453047|N|N|N|N|N| +2453139|AAAAAAAADJOGFCAA|2004-05-13|1252|5446|418|2004|4|5|13|2|2004|418|5446|Thursday|2004Q2|N|N|N|2453127|2453247|2452773|2453048|N|N|N|N|N| +2453140|AAAAAAAAEJOGFCAA|2004-05-14|1252|5446|418|2004|5|5|14|2|2004|418|5446|Friday|2004Q2|N|Y|N|2453127|2453247|2452774|2453049|N|N|N|N|N| +2453141|AAAAAAAAFJOGFCAA|2004-05-15|1252|5446|418|2004|6|5|15|2|2004|418|5446|Saturday|2004Q2|N|Y|N|2453127|2453247|2452775|2453050|N|N|N|N|N| +2453142|AAAAAAAAGJOGFCAA|2004-05-16|1252|5446|418|2004|0|5|16|2|2004|418|5446|Sunday|2004Q2|N|N|N|2453127|2453247|2452776|2453051|N|N|N|N|N| +2453143|AAAAAAAAHJOGFCAA|2004-05-17|1252|5446|418|2004|1|5|17|2|2004|418|5446|Monday|2004Q2|N|N|N|2453127|2453247|2452777|2453052|N|N|N|N|N| +2453144|AAAAAAAAIJOGFCAA|2004-05-18|1252|5447|418|2004|2|5|18|2|2004|418|5447|Tuesday|2004Q2|N|N|N|2453127|2453247|2452778|2453053|N|N|N|N|N| +2453145|AAAAAAAAJJOGFCAA|2004-05-19|1252|5447|418|2004|3|5|19|2|2004|418|5447|Wednesday|2004Q2|N|N|N|2453127|2453247|2452779|2453054|N|N|N|N|N| +2453146|AAAAAAAAKJOGFCAA|2004-05-20|1252|5447|418|2004|4|5|20|2|2004|418|5447|Thursday|2004Q2|N|N|N|2453127|2453247|2452780|2453055|N|N|N|N|N| +2453147|AAAAAAAALJOGFCAA|2004-05-21|1252|5447|418|2004|5|5|21|2|2004|418|5447|Friday|2004Q2|N|Y|N|2453127|2453247|2452781|2453056|N|N|N|N|N| +2453148|AAAAAAAAMJOGFCAA|2004-05-22|1252|5447|418|2004|6|5|22|2|2004|418|5447|Saturday|2004Q2|N|Y|N|2453127|2453247|2452782|2453057|N|N|N|N|N| +2453149|AAAAAAAANJOGFCAA|2004-05-23|1252|5447|418|2004|0|5|23|2|2004|418|5447|Sunday|2004Q2|N|N|N|2453127|2453247|2452783|2453058|N|N|N|N|N| +2453150|AAAAAAAAOJOGFCAA|2004-05-24|1252|5447|418|2004|1|5|24|2|2004|418|5447|Monday|2004Q2|N|N|N|2453127|2453247|2452784|2453059|N|N|N|N|N| +2453151|AAAAAAAAPJOGFCAA|2004-05-25|1252|5448|418|2004|2|5|25|2|2004|418|5448|Tuesday|2004Q2|N|N|N|2453127|2453247|2452785|2453060|N|N|N|N|N| +2453152|AAAAAAAAAKOGFCAA|2004-05-26|1252|5448|418|2004|3|5|26|2|2004|418|5448|Wednesday|2004Q2|N|N|N|2453127|2453247|2452786|2453061|N|N|N|N|N| +2453153|AAAAAAAABKOGFCAA|2004-05-27|1252|5448|418|2004|4|5|27|2|2004|418|5448|Thursday|2004Q2|N|N|N|2453127|2453247|2452787|2453062|N|N|N|N|N| +2453154|AAAAAAAACKOGFCAA|2004-05-28|1252|5448|418|2004|5|5|28|2|2004|418|5448|Friday|2004Q2|N|Y|N|2453127|2453247|2452788|2453063|N|N|N|N|N| +2453155|AAAAAAAADKOGFCAA|2004-05-29|1252|5448|418|2004|6|5|29|2|2004|418|5448|Saturday|2004Q2|N|Y|N|2453127|2453247|2452789|2453064|N|N|N|N|N| +2453156|AAAAAAAAEKOGFCAA|2004-05-30|1252|5448|418|2004|0|5|30|2|2004|418|5448|Sunday|2004Q2|N|N|N|2453127|2453247|2452790|2453065|N|N|N|N|N| +2453157|AAAAAAAAFKOGFCAA|2004-05-31|1252|5448|418|2004|1|5|31|2|2004|418|5448|Monday|2004Q2|N|N|N|2453127|2453247|2452791|2453066|N|N|N|N|N| +2453158|AAAAAAAAGKOGFCAA|2004-06-01|1253|5449|419|2004|2|6|1|2|2004|419|5449|Tuesday|2004Q2|N|N|N|2453158|2453309|2452792|2453067|N|N|N|N|N| +2453159|AAAAAAAAHKOGFCAA|2004-06-02|1253|5449|419|2004|3|6|2|2|2004|419|5449|Wednesday|2004Q2|N|N|N|2453158|2453309|2452793|2453068|N|N|N|N|N| +2453160|AAAAAAAAIKOGFCAA|2004-06-03|1253|5449|419|2004|4|6|3|2|2004|419|5449|Thursday|2004Q2|N|N|N|2453158|2453309|2452794|2453069|N|N|N|N|N| +2453161|AAAAAAAAJKOGFCAA|2004-06-04|1253|5449|419|2004|5|6|4|2|2004|419|5449|Friday|2004Q2|N|Y|N|2453158|2453309|2452795|2453070|N|N|N|N|N| +2453162|AAAAAAAAKKOGFCAA|2004-06-05|1253|5449|419|2004|6|6|5|2|2004|419|5449|Saturday|2004Q2|N|Y|N|2453158|2453309|2452796|2453071|N|N|N|N|N| +2453163|AAAAAAAALKOGFCAA|2004-06-06|1253|5449|419|2004|0|6|6|2|2004|419|5449|Sunday|2004Q2|N|N|N|2453158|2453309|2452797|2453072|N|N|N|N|N| +2453164|AAAAAAAAMKOGFCAA|2004-06-07|1253|5449|419|2004|1|6|7|2|2004|419|5449|Monday|2004Q2|N|N|N|2453158|2453309|2452798|2453073|N|N|N|N|N| +2453165|AAAAAAAANKOGFCAA|2004-06-08|1253|5450|419|2004|2|6|8|2|2004|419|5450|Tuesday|2004Q2|N|N|N|2453158|2453309|2452799|2453074|N|N|N|N|N| +2453166|AAAAAAAAOKOGFCAA|2004-06-09|1253|5450|419|2004|3|6|9|2|2004|419|5450|Wednesday|2004Q2|N|N|N|2453158|2453309|2452800|2453075|N|N|N|N|N| +2453167|AAAAAAAAPKOGFCAA|2004-06-10|1253|5450|419|2004|4|6|10|2|2004|419|5450|Thursday|2004Q2|N|N|N|2453158|2453309|2452801|2453076|N|N|N|N|N| +2453168|AAAAAAAAALOGFCAA|2004-06-11|1253|5450|419|2004|5|6|11|2|2004|419|5450|Friday|2004Q2|N|Y|N|2453158|2453309|2452802|2453077|N|N|N|N|N| +2453169|AAAAAAAABLOGFCAA|2004-06-12|1253|5450|419|2004|6|6|12|2|2004|419|5450|Saturday|2004Q2|N|Y|N|2453158|2453309|2452803|2453078|N|N|N|N|N| +2453170|AAAAAAAACLOGFCAA|2004-06-13|1253|5450|419|2004|0|6|13|2|2004|419|5450|Sunday|2004Q2|N|N|N|2453158|2453309|2452804|2453079|N|N|N|N|N| +2453171|AAAAAAAADLOGFCAA|2004-06-14|1253|5450|419|2004|1|6|14|2|2004|419|5450|Monday|2004Q2|N|N|N|2453158|2453309|2452805|2453080|N|N|N|N|N| +2453172|AAAAAAAAELOGFCAA|2004-06-15|1253|5451|419|2004|2|6|15|2|2004|419|5451|Tuesday|2004Q2|N|N|N|2453158|2453309|2452806|2453081|N|N|N|N|N| +2453173|AAAAAAAAFLOGFCAA|2004-06-16|1253|5451|419|2004|3|6|16|2|2004|419|5451|Wednesday|2004Q2|N|N|N|2453158|2453309|2452807|2453082|N|N|N|N|N| +2453174|AAAAAAAAGLOGFCAA|2004-06-17|1253|5451|419|2004|4|6|17|2|2004|419|5451|Thursday|2004Q2|N|N|N|2453158|2453309|2452808|2453083|N|N|N|N|N| +2453175|AAAAAAAAHLOGFCAA|2004-06-18|1253|5451|419|2004|5|6|18|2|2004|419|5451|Friday|2004Q2|N|Y|N|2453158|2453309|2452809|2453084|N|N|N|N|N| +2453176|AAAAAAAAILOGFCAA|2004-06-19|1253|5451|419|2004|6|6|19|2|2004|419|5451|Saturday|2004Q2|N|Y|N|2453158|2453309|2452810|2453085|N|N|N|N|N| +2453177|AAAAAAAAJLOGFCAA|2004-06-20|1253|5451|419|2004|0|6|20|2|2004|419|5451|Sunday|2004Q2|N|N|N|2453158|2453309|2452811|2453086|N|N|N|N|N| +2453178|AAAAAAAAKLOGFCAA|2004-06-21|1253|5451|419|2004|1|6|21|2|2004|419|5451|Monday|2004Q2|N|N|N|2453158|2453309|2452812|2453087|N|N|N|N|N| +2453179|AAAAAAAALLOGFCAA|2004-06-22|1253|5452|419|2004|2|6|22|2|2004|419|5452|Tuesday|2004Q2|N|N|N|2453158|2453309|2452813|2453088|N|N|N|N|N| +2453180|AAAAAAAAMLOGFCAA|2004-06-23|1253|5452|419|2004|3|6|23|2|2004|419|5452|Wednesday|2004Q2|N|N|N|2453158|2453309|2452814|2453089|N|N|N|N|N| +2453181|AAAAAAAANLOGFCAA|2004-06-24|1253|5452|419|2004|4|6|24|2|2004|419|5452|Thursday|2004Q2|N|N|N|2453158|2453309|2452815|2453090|N|N|N|N|N| +2453182|AAAAAAAAOLOGFCAA|2004-06-25|1253|5452|419|2004|5|6|25|2|2004|419|5452|Friday|2004Q2|N|Y|N|2453158|2453309|2452816|2453091|N|N|N|N|N| +2453183|AAAAAAAAPLOGFCAA|2004-06-26|1253|5452|419|2004|6|6|26|2|2004|419|5452|Saturday|2004Q2|N|Y|N|2453158|2453309|2452817|2453092|N|N|N|N|N| +2453184|AAAAAAAAAMOGFCAA|2004-06-27|1253|5452|419|2004|0|6|27|2|2004|419|5452|Sunday|2004Q2|N|N|N|2453158|2453309|2452818|2453093|N|N|N|N|N| +2453185|AAAAAAAABMOGFCAA|2004-06-28|1253|5452|419|2004|1|6|28|2|2004|419|5452|Monday|2004Q2|N|N|N|2453158|2453309|2452819|2453094|N|N|N|N|N| +2453186|AAAAAAAACMOGFCAA|2004-06-29|1253|5453|419|2004|2|6|29|2|2004|419|5453|Tuesday|2004Q2|N|N|N|2453158|2453309|2452820|2453095|N|N|N|N|N| +2453187|AAAAAAAADMOGFCAA|2004-06-30|1253|5453|419|2004|3|6|30|2|2004|419|5453|Wednesday|2004Q2|N|N|N|2453158|2453309|2452821|2453096|N|N|N|N|N| +2453188|AAAAAAAAEMOGFCAA|2004-07-01|1254|5453|419|2004|4|7|1|3|2004|419|5453|Thursday|2004Q3|N|N|N|2453188|2453369|2452822|2453097|N|N|N|N|N| +2453189|AAAAAAAAFMOGFCAA|2004-07-02|1254|5453|419|2004|5|7|2|3|2004|419|5453|Friday|2004Q3|N|Y|N|2453188|2453369|2452823|2453098|N|N|N|N|N| +2453190|AAAAAAAAGMOGFCAA|2004-07-03|1254|5453|419|2004|6|7|3|3|2004|419|5453|Saturday|2004Q3|N|Y|N|2453188|2453369|2452824|2453099|N|N|N|N|N| +2453191|AAAAAAAAHMOGFCAA|2004-07-04|1254|5453|419|2004|0|7|4|3|2004|419|5453|Sunday|2004Q3|Y|N|N|2453188|2453369|2452825|2453100|N|N|N|N|N| +2453192|AAAAAAAAIMOGFCAA|2004-07-05|1254|5453|419|2004|1|7|5|3|2004|419|5453|Monday|2004Q3|N|N|Y|2453188|2453369|2452826|2453101|N|N|N|N|N| +2453193|AAAAAAAAJMOGFCAA|2004-07-06|1254|5454|419|2004|2|7|6|3|2004|419|5454|Tuesday|2004Q3|N|N|N|2453188|2453369|2452827|2453102|N|N|N|N|N| +2453194|AAAAAAAAKMOGFCAA|2004-07-07|1254|5454|419|2004|3|7|7|3|2004|419|5454|Wednesday|2004Q3|N|N|N|2453188|2453369|2452828|2453103|N|N|N|N|N| +2453195|AAAAAAAALMOGFCAA|2004-07-08|1254|5454|419|2004|4|7|8|3|2004|419|5454|Thursday|2004Q3|N|N|N|2453188|2453369|2452829|2453104|N|N|N|N|N| +2453196|AAAAAAAAMMOGFCAA|2004-07-09|1254|5454|419|2004|5|7|9|3|2004|419|5454|Friday|2004Q3|N|Y|N|2453188|2453369|2452830|2453105|N|N|N|N|N| +2453197|AAAAAAAANMOGFCAA|2004-07-10|1254|5454|419|2004|6|7|10|3|2004|419|5454|Saturday|2004Q3|N|Y|N|2453188|2453369|2452831|2453106|N|N|N|N|N| +2453198|AAAAAAAAOMOGFCAA|2004-07-11|1254|5454|419|2004|0|7|11|3|2004|419|5454|Sunday|2004Q3|N|N|N|2453188|2453369|2452832|2453107|N|N|N|N|N| +2453199|AAAAAAAAPMOGFCAA|2004-07-12|1254|5454|419|2004|1|7|12|3|2004|419|5454|Monday|2004Q3|N|N|N|2453188|2453369|2452833|2453108|N|N|N|N|N| +2453200|AAAAAAAAANOGFCAA|2004-07-13|1254|5455|419|2004|2|7|13|3|2004|419|5455|Tuesday|2004Q3|N|N|N|2453188|2453369|2452834|2453109|N|N|N|N|N| +2453201|AAAAAAAABNOGFCAA|2004-07-14|1254|5455|419|2004|3|7|14|3|2004|419|5455|Wednesday|2004Q3|N|N|N|2453188|2453369|2452835|2453110|N|N|N|N|N| +2453202|AAAAAAAACNOGFCAA|2004-07-15|1254|5455|419|2004|4|7|15|3|2004|419|5455|Thursday|2004Q3|N|N|N|2453188|2453369|2452836|2453111|N|N|N|N|N| +2453203|AAAAAAAADNOGFCAA|2004-07-16|1254|5455|419|2004|5|7|16|3|2004|419|5455|Friday|2004Q3|N|Y|N|2453188|2453369|2452837|2453112|N|N|N|N|N| +2453204|AAAAAAAAENOGFCAA|2004-07-17|1254|5455|419|2004|6|7|17|3|2004|419|5455|Saturday|2004Q3|N|Y|N|2453188|2453369|2452838|2453113|N|N|N|N|N| +2453205|AAAAAAAAFNOGFCAA|2004-07-18|1254|5455|419|2004|0|7|18|3|2004|419|5455|Sunday|2004Q3|N|N|N|2453188|2453369|2452839|2453114|N|N|N|N|N| +2453206|AAAAAAAAGNOGFCAA|2004-07-19|1254|5455|419|2004|1|7|19|3|2004|419|5455|Monday|2004Q3|N|N|N|2453188|2453369|2452840|2453115|N|N|N|N|N| +2453207|AAAAAAAAHNOGFCAA|2004-07-20|1254|5456|419|2004|2|7|20|3|2004|419|5456|Tuesday|2004Q3|N|N|N|2453188|2453369|2452841|2453116|N|N|N|N|N| +2453208|AAAAAAAAINOGFCAA|2004-07-21|1254|5456|419|2004|3|7|21|3|2004|419|5456|Wednesday|2004Q3|N|N|N|2453188|2453369|2452842|2453117|N|N|N|N|N| +2453209|AAAAAAAAJNOGFCAA|2004-07-22|1254|5456|419|2004|4|7|22|3|2004|419|5456|Thursday|2004Q3|N|N|N|2453188|2453369|2452843|2453118|N|N|N|N|N| +2453210|AAAAAAAAKNOGFCAA|2004-07-23|1254|5456|419|2004|5|7|23|3|2004|419|5456|Friday|2004Q3|N|Y|N|2453188|2453369|2452844|2453119|N|N|N|N|N| +2453211|AAAAAAAALNOGFCAA|2004-07-24|1254|5456|419|2004|6|7|24|3|2004|419|5456|Saturday|2004Q3|N|Y|N|2453188|2453369|2452845|2453120|N|N|N|N|N| +2453212|AAAAAAAAMNOGFCAA|2004-07-25|1254|5456|419|2004|0|7|25|3|2004|419|5456|Sunday|2004Q3|N|N|N|2453188|2453369|2452846|2453121|N|N|N|N|N| +2453213|AAAAAAAANNOGFCAA|2004-07-26|1254|5456|419|2004|1|7|26|3|2004|419|5456|Monday|2004Q3|N|N|N|2453188|2453369|2452847|2453122|N|N|N|N|N| +2453214|AAAAAAAAONOGFCAA|2004-07-27|1254|5457|419|2004|2|7|27|3|2004|419|5457|Tuesday|2004Q3|N|N|N|2453188|2453369|2452848|2453123|N|N|N|N|N| +2453215|AAAAAAAAPNOGFCAA|2004-07-28|1254|5457|419|2004|3|7|28|3|2004|419|5457|Wednesday|2004Q3|N|N|N|2453188|2453369|2452849|2453124|N|N|N|N|N| +2453216|AAAAAAAAAOOGFCAA|2004-07-29|1254|5457|419|2004|4|7|29|3|2004|419|5457|Thursday|2004Q3|N|N|N|2453188|2453369|2452850|2453125|N|N|N|N|N| +2453217|AAAAAAAABOOGFCAA|2004-07-30|1254|5457|419|2004|5|7|30|3|2004|419|5457|Friday|2004Q3|N|Y|N|2453188|2453369|2452851|2453126|N|N|N|N|N| +2453218|AAAAAAAACOOGFCAA|2004-07-31|1254|5457|419|2004|6|7|31|3|2004|419|5457|Saturday|2004Q3|N|Y|N|2453188|2453369|2452852|2453127|N|N|N|N|N| +2453219|AAAAAAAADOOGFCAA|2004-08-01|1255|5457|419|2004|0|8|1|3|2004|419|5457|Sunday|2004Q3|N|N|N|2453219|2453431|2452853|2453128|N|N|N|N|N| +2453220|AAAAAAAAEOOGFCAA|2004-08-02|1255|5457|419|2004|1|8|2|3|2004|419|5457|Monday|2004Q3|N|N|N|2453219|2453431|2452854|2453129|N|N|N|N|N| +2453221|AAAAAAAAFOOGFCAA|2004-08-03|1255|5458|419|2004|2|8|3|3|2004|419|5458|Tuesday|2004Q3|N|N|N|2453219|2453431|2452855|2453130|N|N|N|N|N| +2453222|AAAAAAAAGOOGFCAA|2004-08-04|1255|5458|419|2004|3|8|4|3|2004|419|5458|Wednesday|2004Q3|N|N|N|2453219|2453431|2452856|2453131|N|N|N|N|N| +2453223|AAAAAAAAHOOGFCAA|2004-08-05|1255|5458|419|2004|4|8|5|3|2004|419|5458|Thursday|2004Q3|N|N|N|2453219|2453431|2452857|2453132|N|N|N|N|N| +2453224|AAAAAAAAIOOGFCAA|2004-08-06|1255|5458|419|2004|5|8|6|3|2004|419|5458|Friday|2004Q3|N|Y|N|2453219|2453431|2452858|2453133|N|N|N|N|N| +2453225|AAAAAAAAJOOGFCAA|2004-08-07|1255|5458|419|2004|6|8|7|3|2004|419|5458|Saturday|2004Q3|N|Y|N|2453219|2453431|2452859|2453134|N|N|N|N|N| +2453226|AAAAAAAAKOOGFCAA|2004-08-08|1255|5458|419|2004|0|8|8|3|2004|419|5458|Sunday|2004Q3|N|N|N|2453219|2453431|2452860|2453135|N|N|N|N|N| +2453227|AAAAAAAALOOGFCAA|2004-08-09|1255|5458|419|2004|1|8|9|3|2004|419|5458|Monday|2004Q3|N|N|N|2453219|2453431|2452861|2453136|N|N|N|N|N| +2453228|AAAAAAAAMOOGFCAA|2004-08-10|1255|5459|419|2004|2|8|10|3|2004|419|5459|Tuesday|2004Q3|N|N|N|2453219|2453431|2452862|2453137|N|N|N|N|N| +2453229|AAAAAAAANOOGFCAA|2004-08-11|1255|5459|419|2004|3|8|11|3|2004|419|5459|Wednesday|2004Q3|N|N|N|2453219|2453431|2452863|2453138|N|N|N|N|N| +2453230|AAAAAAAAOOOGFCAA|2004-08-12|1255|5459|419|2004|4|8|12|3|2004|419|5459|Thursday|2004Q3|N|N|N|2453219|2453431|2452864|2453139|N|N|N|N|N| +2453231|AAAAAAAAPOOGFCAA|2004-08-13|1255|5459|419|2004|5|8|13|3|2004|419|5459|Friday|2004Q3|N|Y|N|2453219|2453431|2452865|2453140|N|N|N|N|N| +2453232|AAAAAAAAAPOGFCAA|2004-08-14|1255|5459|419|2004|6|8|14|3|2004|419|5459|Saturday|2004Q3|N|Y|N|2453219|2453431|2452866|2453141|N|N|N|N|N| +2453233|AAAAAAAABPOGFCAA|2004-08-15|1255|5459|419|2004|0|8|15|3|2004|419|5459|Sunday|2004Q3|N|N|N|2453219|2453431|2452867|2453142|N|N|N|N|N| +2453234|AAAAAAAACPOGFCAA|2004-08-16|1255|5459|419|2004|1|8|16|3|2004|419|5459|Monday|2004Q3|N|N|N|2453219|2453431|2452868|2453143|N|N|N|N|N| +2453235|AAAAAAAADPOGFCAA|2004-08-17|1255|5460|419|2004|2|8|17|3|2004|419|5460|Tuesday|2004Q3|N|N|N|2453219|2453431|2452869|2453144|N|N|N|N|N| +2453236|AAAAAAAAEPOGFCAA|2004-08-18|1255|5460|419|2004|3|8|18|3|2004|419|5460|Wednesday|2004Q3|N|N|N|2453219|2453431|2452870|2453145|N|N|N|N|N| +2453237|AAAAAAAAFPOGFCAA|2004-08-19|1255|5460|419|2004|4|8|19|3|2004|419|5460|Thursday|2004Q3|N|N|N|2453219|2453431|2452871|2453146|N|N|N|N|N| +2453238|AAAAAAAAGPOGFCAA|2004-08-20|1255|5460|419|2004|5|8|20|3|2004|419|5460|Friday|2004Q3|N|Y|N|2453219|2453431|2452872|2453147|N|N|N|N|N| +2453239|AAAAAAAAHPOGFCAA|2004-08-21|1255|5460|419|2004|6|8|21|3|2004|419|5460|Saturday|2004Q3|N|Y|N|2453219|2453431|2452873|2453148|N|N|N|N|N| +2453240|AAAAAAAAIPOGFCAA|2004-08-22|1255|5460|419|2004|0|8|22|3|2004|419|5460|Sunday|2004Q3|N|N|N|2453219|2453431|2452874|2453149|N|N|N|N|N| +2453241|AAAAAAAAJPOGFCAA|2004-08-23|1255|5460|419|2004|1|8|23|3|2004|419|5460|Monday|2004Q3|N|N|N|2453219|2453431|2452875|2453150|N|N|N|N|N| +2453242|AAAAAAAAKPOGFCAA|2004-08-24|1255|5461|419|2004|2|8|24|3|2004|419|5461|Tuesday|2004Q3|N|N|N|2453219|2453431|2452876|2453151|N|N|N|N|N| +2453243|AAAAAAAALPOGFCAA|2004-08-25|1255|5461|419|2004|3|8|25|3|2004|419|5461|Wednesday|2004Q3|N|N|N|2453219|2453431|2452877|2453152|N|N|N|N|N| +2453244|AAAAAAAAMPOGFCAA|2004-08-26|1255|5461|419|2004|4|8|26|3|2004|419|5461|Thursday|2004Q3|N|N|N|2453219|2453431|2452878|2453153|N|N|N|N|N| +2453245|AAAAAAAANPOGFCAA|2004-08-27|1255|5461|419|2004|5|8|27|3|2004|419|5461|Friday|2004Q3|N|Y|N|2453219|2453431|2452879|2453154|N|N|N|N|N| +2453246|AAAAAAAAOPOGFCAA|2004-08-28|1255|5461|419|2004|6|8|28|3|2004|419|5461|Saturday|2004Q3|N|Y|N|2453219|2453431|2452880|2453155|N|N|N|N|N| +2453247|AAAAAAAAPPOGFCAA|2004-08-29|1255|5461|419|2004|0|8|29|3|2004|419|5461|Sunday|2004Q3|N|N|N|2453219|2453431|2452881|2453156|N|N|N|N|N| +2453248|AAAAAAAAAAPGFCAA|2004-08-30|1255|5461|419|2004|1|8|30|3|2004|419|5461|Monday|2004Q3|N|N|N|2453219|2453431|2452882|2453157|N|N|N|N|N| +2453249|AAAAAAAABAPGFCAA|2004-08-31|1255|5462|419|2004|2|8|31|3|2004|419|5462|Tuesday|2004Q3|N|N|N|2453219|2453431|2452883|2453158|N|N|N|N|N| +2453250|AAAAAAAACAPGFCAA|2004-09-01|1256|5462|420|2004|3|9|1|3|2004|420|5462|Wednesday|2004Q3|N|N|N|2453250|2453493|2452884|2453159|N|N|N|N|N| +2453251|AAAAAAAADAPGFCAA|2004-09-02|1256|5462|420|2004|4|9|2|3|2004|420|5462|Thursday|2004Q3|N|N|N|2453250|2453493|2452885|2453160|N|N|N|N|N| +2453252|AAAAAAAAEAPGFCAA|2004-09-03|1256|5462|420|2004|5|9|3|3|2004|420|5462|Friday|2004Q3|N|Y|N|2453250|2453493|2452886|2453161|N|N|N|N|N| +2453253|AAAAAAAAFAPGFCAA|2004-09-04|1256|5462|420|2004|6|9|4|3|2004|420|5462|Saturday|2004Q3|N|Y|N|2453250|2453493|2452887|2453162|N|N|N|N|N| +2453254|AAAAAAAAGAPGFCAA|2004-09-05|1256|5462|420|2004|0|9|5|3|2004|420|5462|Sunday|2004Q3|N|N|N|2453250|2453493|2452888|2453163|N|N|N|N|N| +2453255|AAAAAAAAHAPGFCAA|2004-09-06|1256|5462|420|2004|1|9|6|3|2004|420|5462|Monday|2004Q3|N|N|N|2453250|2453493|2452889|2453164|N|N|N|N|N| +2453256|AAAAAAAAIAPGFCAA|2004-09-07|1256|5463|420|2004|2|9|7|3|2004|420|5463|Tuesday|2004Q3|N|N|N|2453250|2453493|2452890|2453165|N|N|N|N|N| +2453257|AAAAAAAAJAPGFCAA|2004-09-08|1256|5463|420|2004|3|9|8|3|2004|420|5463|Wednesday|2004Q3|N|N|N|2453250|2453493|2452891|2453166|N|N|N|N|N| +2453258|AAAAAAAAKAPGFCAA|2004-09-09|1256|5463|420|2004|4|9|9|3|2004|420|5463|Thursday|2004Q3|N|N|N|2453250|2453493|2452892|2453167|N|N|N|N|N| +2453259|AAAAAAAALAPGFCAA|2004-09-10|1256|5463|420|2004|5|9|10|3|2004|420|5463|Friday|2004Q3|N|Y|N|2453250|2453493|2452893|2453168|N|N|N|N|N| +2453260|AAAAAAAAMAPGFCAA|2004-09-11|1256|5463|420|2004|6|9|11|3|2004|420|5463|Saturday|2004Q3|N|Y|N|2453250|2453493|2452894|2453169|N|N|N|N|N| +2453261|AAAAAAAANAPGFCAA|2004-09-12|1256|5463|420|2004|0|9|12|3|2004|420|5463|Sunday|2004Q3|N|N|N|2453250|2453493|2452895|2453170|N|N|N|N|N| +2453262|AAAAAAAAOAPGFCAA|2004-09-13|1256|5463|420|2004|1|9|13|3|2004|420|5463|Monday|2004Q3|N|N|N|2453250|2453493|2452896|2453171|N|N|N|N|N| +2453263|AAAAAAAAPAPGFCAA|2004-09-14|1256|5464|420|2004|2|9|14|3|2004|420|5464|Tuesday|2004Q3|N|N|N|2453250|2453493|2452897|2453172|N|N|N|N|N| +2453264|AAAAAAAAABPGFCAA|2004-09-15|1256|5464|420|2004|3|9|15|3|2004|420|5464|Wednesday|2004Q3|N|N|N|2453250|2453493|2452898|2453173|N|N|N|N|N| +2453265|AAAAAAAABBPGFCAA|2004-09-16|1256|5464|420|2004|4|9|16|3|2004|420|5464|Thursday|2004Q3|N|N|N|2453250|2453493|2452899|2453174|N|N|N|N|N| +2453266|AAAAAAAACBPGFCAA|2004-09-17|1256|5464|420|2004|5|9|17|3|2004|420|5464|Friday|2004Q3|N|Y|N|2453250|2453493|2452900|2453175|N|N|N|N|N| +2453267|AAAAAAAADBPGFCAA|2004-09-18|1256|5464|420|2004|6|9|18|3|2004|420|5464|Saturday|2004Q3|N|Y|N|2453250|2453493|2452901|2453176|N|N|N|N|N| +2453268|AAAAAAAAEBPGFCAA|2004-09-19|1256|5464|420|2004|0|9|19|3|2004|420|5464|Sunday|2004Q3|N|N|N|2453250|2453493|2452902|2453177|N|N|N|N|N| +2453269|AAAAAAAAFBPGFCAA|2004-09-20|1256|5464|420|2004|1|9|20|3|2004|420|5464|Monday|2004Q3|N|N|N|2453250|2453493|2452903|2453178|N|N|N|N|N| +2453270|AAAAAAAAGBPGFCAA|2004-09-21|1256|5465|420|2004|2|9|21|3|2004|420|5465|Tuesday|2004Q3|N|N|N|2453250|2453493|2452904|2453179|N|N|N|N|N| +2453271|AAAAAAAAHBPGFCAA|2004-09-22|1256|5465|420|2004|3|9|22|3|2004|420|5465|Wednesday|2004Q3|N|N|N|2453250|2453493|2452905|2453180|N|N|N|N|N| +2453272|AAAAAAAAIBPGFCAA|2004-09-23|1256|5465|420|2004|4|9|23|3|2004|420|5465|Thursday|2004Q3|N|N|N|2453250|2453493|2452906|2453181|N|N|N|N|N| +2453273|AAAAAAAAJBPGFCAA|2004-09-24|1256|5465|420|2004|5|9|24|3|2004|420|5465|Friday|2004Q3|N|Y|N|2453250|2453493|2452907|2453182|N|N|N|N|N| +2453274|AAAAAAAAKBPGFCAA|2004-09-25|1256|5465|420|2004|6|9|25|3|2004|420|5465|Saturday|2004Q3|N|Y|N|2453250|2453493|2452908|2453183|N|N|N|N|N| +2453275|AAAAAAAALBPGFCAA|2004-09-26|1256|5465|420|2004|0|9|26|3|2004|420|5465|Sunday|2004Q3|N|N|N|2453250|2453493|2452909|2453184|N|N|N|N|N| +2453276|AAAAAAAAMBPGFCAA|2004-09-27|1256|5465|420|2004|1|9|27|3|2004|420|5465|Monday|2004Q3|N|N|N|2453250|2453493|2452910|2453185|N|N|N|N|N| +2453277|AAAAAAAANBPGFCAA|2004-09-28|1256|5466|420|2004|2|9|28|3|2004|420|5466|Tuesday|2004Q3|N|N|N|2453250|2453493|2452911|2453186|N|N|N|N|N| +2453278|AAAAAAAAOBPGFCAA|2004-09-29|1256|5466|420|2004|3|9|29|3|2004|420|5466|Wednesday|2004Q3|N|N|N|2453250|2453493|2452912|2453187|N|N|N|N|N| +2453279|AAAAAAAAPBPGFCAA|2004-09-30|1256|5466|420|2004|4|9|30|3|2004|420|5466|Thursday|2004Q3|N|N|N|2453250|2453493|2452913|2453188|N|N|N|N|N| +2453280|AAAAAAAAACPGFCAA|2004-10-01|1257|5466|420|2004|5|10|1|4|2004|420|5466|Friday|2004Q4|N|Y|N|2453280|2453553|2452914|2453188|N|N|N|N|N| +2453281|AAAAAAAABCPGFCAA|2004-10-02|1257|5466|420|2004|6|10|2|4|2004|420|5466|Saturday|2004Q4|N|Y|N|2453280|2453553|2452915|2453189|N|N|N|N|N| +2453282|AAAAAAAACCPGFCAA|2004-10-03|1257|5466|420|2004|0|10|3|4|2004|420|5466|Sunday|2004Q4|N|N|N|2453280|2453553|2452916|2453190|N|N|N|N|N| +2453283|AAAAAAAADCPGFCAA|2004-10-04|1257|5466|420|2004|1|10|4|4|2004|420|5466|Monday|2004Q4|N|N|N|2453280|2453553|2452917|2453191|N|N|N|N|N| +2453284|AAAAAAAAECPGFCAA|2004-10-05|1257|5467|420|2004|2|10|5|4|2004|420|5467|Tuesday|2004Q4|N|N|N|2453280|2453553|2452918|2453192|N|N|N|N|N| +2453285|AAAAAAAAFCPGFCAA|2004-10-06|1257|5467|420|2004|3|10|6|4|2004|420|5467|Wednesday|2004Q4|N|N|N|2453280|2453553|2452919|2453193|N|N|N|N|N| +2453286|AAAAAAAAGCPGFCAA|2004-10-07|1257|5467|420|2004|4|10|7|4|2004|420|5467|Thursday|2004Q4|N|N|N|2453280|2453553|2452920|2453194|N|N|N|N|N| +2453287|AAAAAAAAHCPGFCAA|2004-10-08|1257|5467|420|2004|5|10|8|4|2004|420|5467|Friday|2004Q4|N|Y|N|2453280|2453553|2452921|2453195|N|N|N|N|N| +2453288|AAAAAAAAICPGFCAA|2004-10-09|1257|5467|420|2004|6|10|9|4|2004|420|5467|Saturday|2004Q4|N|Y|N|2453280|2453553|2452922|2453196|N|N|N|N|N| +2453289|AAAAAAAAJCPGFCAA|2004-10-10|1257|5467|420|2004|0|10|10|4|2004|420|5467|Sunday|2004Q4|N|N|N|2453280|2453553|2452923|2453197|N|N|N|N|N| +2453290|AAAAAAAAKCPGFCAA|2004-10-11|1257|5467|420|2004|1|10|11|4|2004|420|5467|Monday|2004Q4|N|N|N|2453280|2453553|2452924|2453198|N|N|N|N|N| +2453291|AAAAAAAALCPGFCAA|2004-10-12|1257|5468|420|2004|2|10|12|4|2004|420|5468|Tuesday|2004Q4|N|N|N|2453280|2453553|2452925|2453199|N|N|N|N|N| +2453292|AAAAAAAAMCPGFCAA|2004-10-13|1257|5468|420|2004|3|10|13|4|2004|420|5468|Wednesday|2004Q4|N|N|N|2453280|2453553|2452926|2453200|N|N|N|N|N| +2453293|AAAAAAAANCPGFCAA|2004-10-14|1257|5468|420|2004|4|10|14|4|2004|420|5468|Thursday|2004Q4|N|N|N|2453280|2453553|2452927|2453201|N|N|N|N|N| +2453294|AAAAAAAAOCPGFCAA|2004-10-15|1257|5468|420|2004|5|10|15|4|2004|420|5468|Friday|2004Q4|N|Y|N|2453280|2453553|2452928|2453202|N|N|N|N|N| +2453295|AAAAAAAAPCPGFCAA|2004-10-16|1257|5468|420|2004|6|10|16|4|2004|420|5468|Saturday|2004Q4|N|Y|N|2453280|2453553|2452929|2453203|N|N|N|N|N| +2453296|AAAAAAAAADPGFCAA|2004-10-17|1257|5468|420|2004|0|10|17|4|2004|420|5468|Sunday|2004Q4|N|N|N|2453280|2453553|2452930|2453204|N|N|N|N|N| +2453297|AAAAAAAABDPGFCAA|2004-10-18|1257|5468|420|2004|1|10|18|4|2004|420|5468|Monday|2004Q4|N|N|N|2453280|2453553|2452931|2453205|N|N|N|N|N| +2453298|AAAAAAAACDPGFCAA|2004-10-19|1257|5469|420|2004|2|10|19|4|2004|420|5469|Tuesday|2004Q4|N|N|N|2453280|2453553|2452932|2453206|N|N|N|N|N| +2453299|AAAAAAAADDPGFCAA|2004-10-20|1257|5469|420|2004|3|10|20|4|2004|420|5469|Wednesday|2004Q4|N|N|N|2453280|2453553|2452933|2453207|N|N|N|N|N| +2453300|AAAAAAAAEDPGFCAA|2004-10-21|1257|5469|420|2004|4|10|21|4|2004|420|5469|Thursday|2004Q4|N|N|N|2453280|2453553|2452934|2453208|N|N|N|N|N| +2453301|AAAAAAAAFDPGFCAA|2004-10-22|1257|5469|420|2004|5|10|22|4|2004|420|5469|Friday|2004Q4|N|Y|N|2453280|2453553|2452935|2453209|N|N|N|N|N| +2453302|AAAAAAAAGDPGFCAA|2004-10-23|1257|5469|420|2004|6|10|23|4|2004|420|5469|Saturday|2004Q4|N|Y|N|2453280|2453553|2452936|2453210|N|N|N|N|N| +2453303|AAAAAAAAHDPGFCAA|2004-10-24|1257|5469|420|2004|0|10|24|4|2004|420|5469|Sunday|2004Q4|N|N|N|2453280|2453553|2452937|2453211|N|N|N|N|N| +2453304|AAAAAAAAIDPGFCAA|2004-10-25|1257|5469|420|2004|1|10|25|4|2004|420|5469|Monday|2004Q4|N|N|N|2453280|2453553|2452938|2453212|N|N|N|N|N| +2453305|AAAAAAAAJDPGFCAA|2004-10-26|1257|5470|420|2004|2|10|26|4|2004|420|5470|Tuesday|2004Q4|N|N|N|2453280|2453553|2452939|2453213|N|N|N|N|N| +2453306|AAAAAAAAKDPGFCAA|2004-10-27|1257|5470|420|2004|3|10|27|4|2004|420|5470|Wednesday|2004Q4|N|N|N|2453280|2453553|2452940|2453214|N|N|N|N|N| +2453307|AAAAAAAALDPGFCAA|2004-10-28|1257|5470|420|2004|4|10|28|4|2004|420|5470|Thursday|2004Q4|N|N|N|2453280|2453553|2452941|2453215|N|N|N|N|N| +2453308|AAAAAAAAMDPGFCAA|2004-10-29|1257|5470|420|2004|5|10|29|4|2004|420|5470|Friday|2004Q4|N|Y|N|2453280|2453553|2452942|2453216|N|N|N|N|N| +2453309|AAAAAAAANDPGFCAA|2004-10-30|1257|5470|420|2004|6|10|30|4|2004|420|5470|Saturday|2004Q4|N|Y|N|2453280|2453553|2452943|2453217|N|N|N|N|N| +2453310|AAAAAAAAODPGFCAA|2004-10-31|1257|5470|420|2004|0|10|31|4|2004|420|5470|Sunday|2004Q4|N|N|N|2453280|2453553|2452944|2453218|N|N|N|N|N| +2453311|AAAAAAAAPDPGFCAA|2004-11-01|1258|5470|420|2004|1|11|1|4|2004|420|5470|Monday|2004Q4|N|N|N|2453311|2453615|2452945|2453219|N|N|N|N|N| +2453312|AAAAAAAAAEPGFCAA|2004-11-02|1258|5471|420|2004|2|11|2|4|2004|420|5471|Tuesday|2004Q4|N|N|N|2453311|2453615|2452946|2453220|N|N|N|N|N| +2453313|AAAAAAAABEPGFCAA|2004-11-03|1258|5471|420|2004|3|11|3|4|2004|420|5471|Wednesday|2004Q4|N|N|N|2453311|2453615|2452947|2453221|N|N|N|N|N| +2453314|AAAAAAAACEPGFCAA|2004-11-04|1258|5471|420|2004|4|11|4|4|2004|420|5471|Thursday|2004Q4|N|N|N|2453311|2453615|2452948|2453222|N|N|N|N|N| +2453315|AAAAAAAADEPGFCAA|2004-11-05|1258|5471|420|2004|5|11|5|4|2004|420|5471|Friday|2004Q4|N|Y|N|2453311|2453615|2452949|2453223|N|N|N|N|N| +2453316|AAAAAAAAEEPGFCAA|2004-11-06|1258|5471|420|2004|6|11|6|4|2004|420|5471|Saturday|2004Q4|N|Y|N|2453311|2453615|2452950|2453224|N|N|N|N|N| +2453317|AAAAAAAAFEPGFCAA|2004-11-07|1258|5471|420|2004|0|11|7|4|2004|420|5471|Sunday|2004Q4|N|N|N|2453311|2453615|2452951|2453225|N|N|N|N|N| +2453318|AAAAAAAAGEPGFCAA|2004-11-08|1258|5471|420|2004|1|11|8|4|2004|420|5471|Monday|2004Q4|N|N|N|2453311|2453615|2452952|2453226|N|N|N|N|N| +2453319|AAAAAAAAHEPGFCAA|2004-11-09|1258|5472|420|2004|2|11|9|4|2004|420|5472|Tuesday|2004Q4|N|N|N|2453311|2453615|2452953|2453227|N|N|N|N|N| +2453320|AAAAAAAAIEPGFCAA|2004-11-10|1258|5472|420|2004|3|11|10|4|2004|420|5472|Wednesday|2004Q4|N|N|N|2453311|2453615|2452954|2453228|N|N|N|N|N| +2453321|AAAAAAAAJEPGFCAA|2004-11-11|1258|5472|420|2004|4|11|11|4|2004|420|5472|Thursday|2004Q4|N|N|N|2453311|2453615|2452955|2453229|N|N|N|N|N| +2453322|AAAAAAAAKEPGFCAA|2004-11-12|1258|5472|420|2004|5|11|12|4|2004|420|5472|Friday|2004Q4|N|Y|N|2453311|2453615|2452956|2453230|N|N|N|N|N| +2453323|AAAAAAAALEPGFCAA|2004-11-13|1258|5472|420|2004|6|11|13|4|2004|420|5472|Saturday|2004Q4|N|Y|N|2453311|2453615|2452957|2453231|N|N|N|N|N| +2453324|AAAAAAAAMEPGFCAA|2004-11-14|1258|5472|420|2004|0|11|14|4|2004|420|5472|Sunday|2004Q4|N|N|N|2453311|2453615|2452958|2453232|N|N|N|N|N| +2453325|AAAAAAAANEPGFCAA|2004-11-15|1258|5472|420|2004|1|11|15|4|2004|420|5472|Monday|2004Q4|N|N|N|2453311|2453615|2452959|2453233|N|N|N|N|N| +2453326|AAAAAAAAOEPGFCAA|2004-11-16|1258|5473|420|2004|2|11|16|4|2004|420|5473|Tuesday|2004Q4|N|N|N|2453311|2453615|2452960|2453234|N|N|N|N|N| +2453327|AAAAAAAAPEPGFCAA|2004-11-17|1258|5473|420|2004|3|11|17|4|2004|420|5473|Wednesday|2004Q4|N|N|N|2453311|2453615|2452961|2453235|N|N|N|N|N| +2453328|AAAAAAAAAFPGFCAA|2004-11-18|1258|5473|420|2004|4|11|18|4|2004|420|5473|Thursday|2004Q4|N|N|N|2453311|2453615|2452962|2453236|N|N|N|N|N| +2453329|AAAAAAAABFPGFCAA|2004-11-19|1258|5473|420|2004|5|11|19|4|2004|420|5473|Friday|2004Q4|N|Y|N|2453311|2453615|2452963|2453237|N|N|N|N|N| +2453330|AAAAAAAACFPGFCAA|2004-11-20|1258|5473|420|2004|6|11|20|4|2004|420|5473|Saturday|2004Q4|N|Y|N|2453311|2453615|2452964|2453238|N|N|N|N|N| +2453331|AAAAAAAADFPGFCAA|2004-11-21|1258|5473|420|2004|0|11|21|4|2004|420|5473|Sunday|2004Q4|N|N|N|2453311|2453615|2452965|2453239|N|N|N|N|N| +2453332|AAAAAAAAEFPGFCAA|2004-11-22|1258|5473|420|2004|1|11|22|4|2004|420|5473|Monday|2004Q4|N|N|N|2453311|2453615|2452966|2453240|N|N|N|N|N| +2453333|AAAAAAAAFFPGFCAA|2004-11-23|1258|5474|420|2004|2|11|23|4|2004|420|5474|Tuesday|2004Q4|N|N|N|2453311|2453615|2452967|2453241|N|N|N|N|N| +2453334|AAAAAAAAGFPGFCAA|2004-11-24|1258|5474|420|2004|3|11|24|4|2004|420|5474|Wednesday|2004Q4|N|N|N|2453311|2453615|2452968|2453242|N|N|N|N|N| +2453335|AAAAAAAAHFPGFCAA|2004-11-25|1258|5474|420|2004|4|11|25|4|2004|420|5474|Thursday|2004Q4|N|N|N|2453311|2453615|2452969|2453243|N|N|N|N|N| +2453336|AAAAAAAAIFPGFCAA|2004-11-26|1258|5474|420|2004|5|11|26|4|2004|420|5474|Friday|2004Q4|N|Y|N|2453311|2453615|2452970|2453244|N|N|N|N|N| +2453337|AAAAAAAAJFPGFCAA|2004-11-27|1258|5474|420|2004|6|11|27|4|2004|420|5474|Saturday|2004Q4|N|Y|N|2453311|2453615|2452971|2453245|N|N|N|N|N| +2453338|AAAAAAAAKFPGFCAA|2004-11-28|1258|5474|420|2004|0|11|28|4|2004|420|5474|Sunday|2004Q4|N|N|N|2453311|2453615|2452972|2453246|N|N|N|N|N| +2453339|AAAAAAAALFPGFCAA|2004-11-29|1258|5474|420|2004|1|11|29|4|2004|420|5474|Monday|2004Q4|N|N|N|2453311|2453615|2452973|2453247|N|N|N|N|N| +2453340|AAAAAAAAMFPGFCAA|2004-11-30|1258|5475|420|2004|2|11|30|4|2004|420|5475|Tuesday|2004Q4|N|N|N|2453311|2453615|2452974|2453248|N|N|N|N|N| +2453341|AAAAAAAANFPGFCAA|2004-12-01|1259|5475|421|2004|3|12|1|4|2004|421|5475|Wednesday|2004Q4|N|N|N|2453341|2453675|2452975|2453249|N|N|N|N|N| +2453342|AAAAAAAAOFPGFCAA|2004-12-02|1259|5475|421|2004|4|12|2|4|2004|421|5475|Thursday|2004Q4|N|N|N|2453341|2453675|2452976|2453250|N|N|N|N|N| +2453343|AAAAAAAAPFPGFCAA|2004-12-03|1259|5475|421|2004|5|12|3|4|2004|421|5475|Friday|2004Q4|N|Y|N|2453341|2453675|2452977|2453251|N|N|N|N|N| +2453344|AAAAAAAAAGPGFCAA|2004-12-04|1259|5475|421|2004|6|12|4|4|2004|421|5475|Saturday|2004Q4|N|Y|N|2453341|2453675|2452978|2453252|N|N|N|N|N| +2453345|AAAAAAAABGPGFCAA|2004-12-05|1259|5475|421|2004|0|12|5|4|2004|421|5475|Sunday|2004Q4|N|N|N|2453341|2453675|2452979|2453253|N|N|N|N|N| +2453346|AAAAAAAACGPGFCAA|2004-12-06|1259|5475|421|2004|1|12|6|4|2004|421|5475|Monday|2004Q4|N|N|N|2453341|2453675|2452980|2453254|N|N|N|N|N| +2453347|AAAAAAAADGPGFCAA|2004-12-07|1259|5476|421|2004|2|12|7|4|2004|421|5476|Tuesday|2004Q4|N|N|N|2453341|2453675|2452981|2453255|N|N|N|N|N| +2453348|AAAAAAAAEGPGFCAA|2004-12-08|1259|5476|421|2004|3|12|8|4|2004|421|5476|Wednesday|2004Q4|N|N|N|2453341|2453675|2452982|2453256|N|N|N|N|N| +2453349|AAAAAAAAFGPGFCAA|2004-12-09|1259|5476|421|2004|4|12|9|4|2004|421|5476|Thursday|2004Q4|N|N|N|2453341|2453675|2452983|2453257|N|N|N|N|N| +2453350|AAAAAAAAGGPGFCAA|2004-12-10|1259|5476|421|2004|5|12|10|4|2004|421|5476|Friday|2004Q4|N|Y|N|2453341|2453675|2452984|2453258|N|N|N|N|N| +2453351|AAAAAAAAHGPGFCAA|2004-12-11|1259|5476|421|2004|6|12|11|4|2004|421|5476|Saturday|2004Q4|N|Y|N|2453341|2453675|2452985|2453259|N|N|N|N|N| +2453352|AAAAAAAAIGPGFCAA|2004-12-12|1259|5476|421|2004|0|12|12|4|2004|421|5476|Sunday|2004Q4|N|N|N|2453341|2453675|2452986|2453260|N|N|N|N|N| +2453353|AAAAAAAAJGPGFCAA|2004-12-13|1259|5476|421|2004|1|12|13|4|2004|421|5476|Monday|2004Q4|N|N|N|2453341|2453675|2452987|2453261|N|N|N|N|N| +2453354|AAAAAAAAKGPGFCAA|2004-12-14|1259|5477|421|2004|2|12|14|4|2004|421|5477|Tuesday|2004Q4|N|N|N|2453341|2453675|2452988|2453262|N|N|N|N|N| +2453355|AAAAAAAALGPGFCAA|2004-12-15|1259|5477|421|2004|3|12|15|4|2004|421|5477|Wednesday|2004Q4|N|N|N|2453341|2453675|2452989|2453263|N|N|N|N|N| +2453356|AAAAAAAAMGPGFCAA|2004-12-16|1259|5477|421|2004|4|12|16|4|2004|421|5477|Thursday|2004Q4|N|N|N|2453341|2453675|2452990|2453264|N|N|N|N|N| +2453357|AAAAAAAANGPGFCAA|2004-12-17|1259|5477|421|2004|5|12|17|4|2004|421|5477|Friday|2004Q4|N|Y|N|2453341|2453675|2452991|2453265|N|N|N|N|N| +2453358|AAAAAAAAOGPGFCAA|2004-12-18|1259|5477|421|2004|6|12|18|4|2004|421|5477|Saturday|2004Q4|N|Y|N|2453341|2453675|2452992|2453266|N|N|N|N|N| +2453359|AAAAAAAAPGPGFCAA|2004-12-19|1259|5477|421|2004|0|12|19|4|2004|421|5477|Sunday|2004Q4|N|N|N|2453341|2453675|2452993|2453267|N|N|N|N|N| +2453360|AAAAAAAAAHPGFCAA|2004-12-20|1259|5477|421|2004|1|12|20|4|2004|421|5477|Monday|2004Q4|N|N|N|2453341|2453675|2452994|2453268|N|N|N|N|N| +2453361|AAAAAAAABHPGFCAA|2004-12-21|1259|5478|421|2004|2|12|21|4|2004|421|5478|Tuesday|2004Q4|N|N|N|2453341|2453675|2452995|2453269|N|N|N|N|N| +2453362|AAAAAAAACHPGFCAA|2004-12-22|1259|5478|421|2004|3|12|22|4|2004|421|5478|Wednesday|2004Q4|N|N|N|2453341|2453675|2452996|2453270|N|N|N|N|N| +2453363|AAAAAAAADHPGFCAA|2004-12-23|1259|5478|421|2004|4|12|23|4|2004|421|5478|Thursday|2004Q4|N|N|N|2453341|2453675|2452997|2453271|N|N|N|N|N| +2453364|AAAAAAAAEHPGFCAA|2004-12-24|1259|5478|421|2004|5|12|24|4|2004|421|5478|Friday|2004Q4|N|Y|N|2453341|2453675|2452998|2453272|N|N|N|N|N| +2453365|AAAAAAAAFHPGFCAA|2004-12-25|1259|5478|421|2004|6|12|25|4|2004|421|5478|Saturday|2004Q4|Y|Y|N|2453341|2453675|2452999|2453273|N|N|N|N|N| +2453366|AAAAAAAAGHPGFCAA|2004-12-26|1259|5478|421|2004|0|12|26|4|2004|421|5478|Sunday|2004Q4|N|N|Y|2453341|2453675|2453000|2453274|N|N|N|N|N| +2453367|AAAAAAAAHHPGFCAA|2004-12-27|1259|5478|421|2004|1|12|27|4|2004|421|5478|Monday|2004Q4|N|N|N|2453341|2453675|2453001|2453275|N|N|N|N|N| +2453368|AAAAAAAAIHPGFCAA|2004-12-28|1259|5479|421|2004|2|12|28|4|2004|421|5479|Tuesday|2004Q4|N|N|N|2453341|2453675|2453002|2453276|N|N|N|N|N| +2453369|AAAAAAAAJHPGFCAA|2004-12-29|1259|5479|421|2004|3|12|29|4|2004|421|5479|Wednesday|2004Q4|N|N|N|2453341|2453675|2453003|2453277|N|N|N|N|N| +2453370|AAAAAAAAKHPGFCAA|2004-12-30|1259|5479|421|2004|4|12|30|4|2004|421|5479|Thursday|2004Q4|N|N|N|2453341|2453675|2453004|2453278|N|N|N|N|N| +2453371|AAAAAAAALHPGFCAA|2004-12-31|1259|5479|421|2004|5|12|31|4|2004|421|5479|Friday|2004Q4|Y|Y|N|2453341|2453675|2453005|2453279|N|N|N|N|N| +2453372|AAAAAAAAMHPGFCAA|2005-01-01|1260|5479|421|2005|6|1|1|1|2005|421|5479|Saturday|2005Q1|Y|Y|Y|2453372|2453371|2453006|2453280|N|N|N|N|N| +2453373|AAAAAAAANHPGFCAA|2005-01-02|1260|5479|421|2005|0|1|2|1|2005|421|5479|Sunday|2005Q1|N|N|Y|2453372|2453371|2453007|2453281|N|N|N|N|N| +2453374|AAAAAAAAOHPGFCAA|2005-01-03|1260|5479|421|2005|1|1|3|1|2005|421|5479|Monday|2005Q1|N|N|N|2453372|2453371|2453008|2453282|N|N|N|N|N| +2453375|AAAAAAAAPHPGFCAA|2005-01-04|1260|5480|421|2005|2|1|4|1|2005|421|5480|Tuesday|2005Q1|N|N|N|2453372|2453371|2453009|2453283|N|N|N|N|N| +2453376|AAAAAAAAAIPGFCAA|2005-01-05|1260|5480|421|2005|3|1|5|1|2005|421|5480|Wednesday|2005Q1|N|N|N|2453372|2453371|2453010|2453284|N|N|N|N|N| +2453377|AAAAAAAABIPGFCAA|2005-01-06|1260|5480|421|2005|4|1|6|1|2005|421|5480|Thursday|2005Q1|N|N|N|2453372|2453371|2453011|2453285|N|N|N|N|N| +2453378|AAAAAAAACIPGFCAA|2005-01-07|1260|5480|421|2005|5|1|7|1|2005|421|5480|Friday|2005Q1|N|Y|N|2453372|2453371|2453012|2453286|N|N|N|N|N| +2453379|AAAAAAAADIPGFCAA|2005-01-08|1260|5480|421|2005|6|1|8|1|2005|421|5480|Saturday|2005Q1|N|Y|N|2453372|2453371|2453013|2453287|N|N|N|N|N| +2453380|AAAAAAAAEIPGFCAA|2005-01-09|1260|5480|421|2005|0|1|9|1|2005|421|5480|Sunday|2005Q1|N|N|N|2453372|2453371|2453014|2453288|N|N|N|N|N| +2453381|AAAAAAAAFIPGFCAA|2005-01-10|1260|5480|421|2005|1|1|10|1|2005|421|5480|Monday|2005Q1|N|N|N|2453372|2453371|2453015|2453289|N|N|N|N|N| +2453382|AAAAAAAAGIPGFCAA|2005-01-11|1260|5481|421|2005|2|1|11|1|2005|421|5481|Tuesday|2005Q1|N|N|N|2453372|2453371|2453016|2453290|N|N|N|N|N| +2453383|AAAAAAAAHIPGFCAA|2005-01-12|1260|5481|421|2005|3|1|12|1|2005|421|5481|Wednesday|2005Q1|N|N|N|2453372|2453371|2453017|2453291|N|N|N|N|N| +2453384|AAAAAAAAIIPGFCAA|2005-01-13|1260|5481|421|2005|4|1|13|1|2005|421|5481|Thursday|2005Q1|N|N|N|2453372|2453371|2453018|2453292|N|N|N|N|N| +2453385|AAAAAAAAJIPGFCAA|2005-01-14|1260|5481|421|2005|5|1|14|1|2005|421|5481|Friday|2005Q1|N|Y|N|2453372|2453371|2453019|2453293|N|N|N|N|N| +2453386|AAAAAAAAKIPGFCAA|2005-01-15|1260|5481|421|2005|6|1|15|1|2005|421|5481|Saturday|2005Q1|N|Y|N|2453372|2453371|2453020|2453294|N|N|N|N|N| +2453387|AAAAAAAALIPGFCAA|2005-01-16|1260|5481|421|2005|0|1|16|1|2005|421|5481|Sunday|2005Q1|N|N|N|2453372|2453371|2453021|2453295|N|N|N|N|N| +2453388|AAAAAAAAMIPGFCAA|2005-01-17|1260|5481|421|2005|1|1|17|1|2005|421|5481|Monday|2005Q1|N|N|N|2453372|2453371|2453022|2453296|N|N|N|N|N| +2453389|AAAAAAAANIPGFCAA|2005-01-18|1260|5482|421|2005|2|1|18|1|2005|421|5482|Tuesday|2005Q1|N|N|N|2453372|2453371|2453023|2453297|N|N|N|N|N| +2453390|AAAAAAAAOIPGFCAA|2005-01-19|1260|5482|421|2005|3|1|19|1|2005|421|5482|Wednesday|2005Q1|N|N|N|2453372|2453371|2453024|2453298|N|N|N|N|N| +2453391|AAAAAAAAPIPGFCAA|2005-01-20|1260|5482|421|2005|4|1|20|1|2005|421|5482|Thursday|2005Q1|N|N|N|2453372|2453371|2453025|2453299|N|N|N|N|N| +2453392|AAAAAAAAAJPGFCAA|2005-01-21|1260|5482|421|2005|5|1|21|1|2005|421|5482|Friday|2005Q1|N|Y|N|2453372|2453371|2453026|2453300|N|N|N|N|N| +2453393|AAAAAAAABJPGFCAA|2005-01-22|1260|5482|421|2005|6|1|22|1|2005|421|5482|Saturday|2005Q1|N|Y|N|2453372|2453371|2453027|2453301|N|N|N|N|N| +2453394|AAAAAAAACJPGFCAA|2005-01-23|1260|5482|421|2005|0|1|23|1|2005|421|5482|Sunday|2005Q1|N|N|N|2453372|2453371|2453028|2453302|N|N|N|N|N| +2453395|AAAAAAAADJPGFCAA|2005-01-24|1260|5482|421|2005|1|1|24|1|2005|421|5482|Monday|2005Q1|N|N|N|2453372|2453371|2453029|2453303|N|N|N|N|N| +2453396|AAAAAAAAEJPGFCAA|2005-01-25|1260|5483|421|2005|2|1|25|1|2005|421|5483|Tuesday|2005Q1|N|N|N|2453372|2453371|2453030|2453304|N|N|N|N|N| +2453397|AAAAAAAAFJPGFCAA|2005-01-26|1260|5483|421|2005|3|1|26|1|2005|421|5483|Wednesday|2005Q1|N|N|N|2453372|2453371|2453031|2453305|N|N|N|N|N| +2453398|AAAAAAAAGJPGFCAA|2005-01-27|1260|5483|421|2005|4|1|27|1|2005|421|5483|Thursday|2005Q1|N|N|N|2453372|2453371|2453032|2453306|N|N|N|N|N| +2453399|AAAAAAAAHJPGFCAA|2005-01-28|1260|5483|421|2005|5|1|28|1|2005|421|5483|Friday|2005Q1|N|Y|N|2453372|2453371|2453033|2453307|N|N|N|N|N| +2453400|AAAAAAAAIJPGFCAA|2005-01-29|1260|5483|421|2005|6|1|29|1|2005|421|5483|Saturday|2005Q1|N|Y|N|2453372|2453371|2453034|2453308|N|N|N|N|N| +2453401|AAAAAAAAJJPGFCAA|2005-01-30|1260|5483|421|2005|0|1|30|1|2005|421|5483|Sunday|2005Q1|N|N|N|2453372|2453371|2453035|2453309|N|N|N|N|N| +2453402|AAAAAAAAKJPGFCAA|2005-01-31|1260|5483|421|2005|1|1|31|1|2005|421|5483|Monday|2005Q1|N|N|N|2453372|2453371|2453036|2453310|N|N|N|N|N| +2453403|AAAAAAAALJPGFCAA|2005-02-01|1261|5484|421|2005|2|2|1|1|2005|421|5484|Tuesday|2005Q1|N|N|N|2453403|2453433|2453037|2453311|N|N|N|N|N| +2453404|AAAAAAAAMJPGFCAA|2005-02-02|1261|5484|421|2005|3|2|2|1|2005|421|5484|Wednesday|2005Q1|N|N|N|2453403|2453433|2453038|2453312|N|N|N|N|N| +2453405|AAAAAAAANJPGFCAA|2005-02-03|1261|5484|421|2005|4|2|3|1|2005|421|5484|Thursday|2005Q1|N|N|N|2453403|2453433|2453039|2453313|N|N|N|N|N| +2453406|AAAAAAAAOJPGFCAA|2005-02-04|1261|5484|421|2005|5|2|4|1|2005|421|5484|Friday|2005Q1|N|Y|N|2453403|2453433|2453040|2453314|N|N|N|N|N| +2453407|AAAAAAAAPJPGFCAA|2005-02-05|1261|5484|421|2005|6|2|5|1|2005|421|5484|Saturday|2005Q1|N|Y|N|2453403|2453433|2453041|2453315|N|N|N|N|N| +2453408|AAAAAAAAAKPGFCAA|2005-02-06|1261|5484|421|2005|0|2|6|1|2005|421|5484|Sunday|2005Q1|N|N|N|2453403|2453433|2453042|2453316|N|N|N|N|N| +2453409|AAAAAAAABKPGFCAA|2005-02-07|1261|5484|421|2005|1|2|7|1|2005|421|5484|Monday|2005Q1|N|N|N|2453403|2453433|2453043|2453317|N|N|N|N|N| +2453410|AAAAAAAACKPGFCAA|2005-02-08|1261|5485|421|2005|2|2|8|1|2005|421|5485|Tuesday|2005Q1|N|N|N|2453403|2453433|2453044|2453318|N|N|N|N|N| +2453411|AAAAAAAADKPGFCAA|2005-02-09|1261|5485|421|2005|3|2|9|1|2005|421|5485|Wednesday|2005Q1|N|N|N|2453403|2453433|2453045|2453319|N|N|N|N|N| +2453412|AAAAAAAAEKPGFCAA|2005-02-10|1261|5485|421|2005|4|2|10|1|2005|421|5485|Thursday|2005Q1|N|N|N|2453403|2453433|2453046|2453320|N|N|N|N|N| +2453413|AAAAAAAAFKPGFCAA|2005-02-11|1261|5485|421|2005|5|2|11|1|2005|421|5485|Friday|2005Q1|N|Y|N|2453403|2453433|2453047|2453321|N|N|N|N|N| +2453414|AAAAAAAAGKPGFCAA|2005-02-12|1261|5485|421|2005|6|2|12|1|2005|421|5485|Saturday|2005Q1|N|Y|N|2453403|2453433|2453048|2453322|N|N|N|N|N| +2453415|AAAAAAAAHKPGFCAA|2005-02-13|1261|5485|421|2005|0|2|13|1|2005|421|5485|Sunday|2005Q1|N|N|N|2453403|2453433|2453049|2453323|N|N|N|N|N| +2453416|AAAAAAAAIKPGFCAA|2005-02-14|1261|5485|421|2005|1|2|14|1|2005|421|5485|Monday|2005Q1|N|N|N|2453403|2453433|2453050|2453324|N|N|N|N|N| +2453417|AAAAAAAAJKPGFCAA|2005-02-15|1261|5486|421|2005|2|2|15|1|2005|421|5486|Tuesday|2005Q1|N|N|N|2453403|2453433|2453051|2453325|N|N|N|N|N| +2453418|AAAAAAAAKKPGFCAA|2005-02-16|1261|5486|421|2005|3|2|16|1|2005|421|5486|Wednesday|2005Q1|N|N|N|2453403|2453433|2453052|2453326|N|N|N|N|N| +2453419|AAAAAAAALKPGFCAA|2005-02-17|1261|5486|421|2005|4|2|17|1|2005|421|5486|Thursday|2005Q1|N|N|N|2453403|2453433|2453053|2453327|N|N|N|N|N| +2453420|AAAAAAAAMKPGFCAA|2005-02-18|1261|5486|421|2005|5|2|18|1|2005|421|5486|Friday|2005Q1|N|Y|N|2453403|2453433|2453054|2453328|N|N|N|N|N| +2453421|AAAAAAAANKPGFCAA|2005-02-19|1261|5486|421|2005|6|2|19|1|2005|421|5486|Saturday|2005Q1|N|Y|N|2453403|2453433|2453055|2453329|N|N|N|N|N| +2453422|AAAAAAAAOKPGFCAA|2005-02-20|1261|5486|421|2005|0|2|20|1|2005|421|5486|Sunday|2005Q1|N|N|N|2453403|2453433|2453056|2453330|N|N|N|N|N| +2453423|AAAAAAAAPKPGFCAA|2005-02-21|1261|5486|421|2005|1|2|21|1|2005|421|5486|Monday|2005Q1|N|N|N|2453403|2453433|2453057|2453331|N|N|N|N|N| +2453424|AAAAAAAAALPGFCAA|2005-02-22|1261|5487|421|2005|2|2|22|1|2005|421|5487|Tuesday|2005Q1|N|N|N|2453403|2453433|2453058|2453332|N|N|N|N|N| +2453425|AAAAAAAABLPGFCAA|2005-02-23|1261|5487|421|2005|3|2|23|1|2005|421|5487|Wednesday|2005Q1|N|N|N|2453403|2453433|2453059|2453333|N|N|N|N|N| +2453426|AAAAAAAACLPGFCAA|2005-02-24|1261|5487|421|2005|4|2|24|1|2005|421|5487|Thursday|2005Q1|N|N|N|2453403|2453433|2453060|2453334|N|N|N|N|N| +2453427|AAAAAAAADLPGFCAA|2005-02-25|1261|5487|421|2005|5|2|25|1|2005|421|5487|Friday|2005Q1|N|Y|N|2453403|2453433|2453061|2453335|N|N|N|N|N| +2453428|AAAAAAAAELPGFCAA|2005-02-26|1261|5487|421|2005|6|2|26|1|2005|421|5487|Saturday|2005Q1|N|Y|N|2453403|2453433|2453062|2453336|N|N|N|N|N| +2453429|AAAAAAAAFLPGFCAA|2005-02-27|1261|5487|421|2005|0|2|27|1|2005|421|5487|Sunday|2005Q1|N|N|N|2453403|2453433|2453063|2453337|N|N|N|N|N| +2453430|AAAAAAAAGLPGFCAA|2005-02-28|1261|5487|421|2005|1|2|28|1|2005|421|5487|Monday|2005Q1|N|N|N|2453403|2453433|2453064|2453338|N|N|N|N|N| +2453431|AAAAAAAAHLPGFCAA|2005-03-01|1262|5488|422|2005|2|3|1|1|2005|422|5488|Tuesday|2005Q1|N|N|N|2453431|2453489|2453066|2453339|N|N|N|N|N| +2453432|AAAAAAAAILPGFCAA|2005-03-02|1262|5488|422|2005|3|3|2|1|2005|422|5488|Wednesday|2005Q1|N|N|N|2453431|2453489|2453067|2453340|N|N|N|N|N| +2453433|AAAAAAAAJLPGFCAA|2005-03-03|1262|5488|422|2005|4|3|3|1|2005|422|5488|Thursday|2005Q1|N|N|N|2453431|2453489|2453068|2453341|N|N|N|N|N| +2453434|AAAAAAAAKLPGFCAA|2005-03-04|1262|5488|422|2005|5|3|4|1|2005|422|5488|Friday|2005Q1|N|Y|N|2453431|2453489|2453069|2453342|N|N|N|N|N| +2453435|AAAAAAAALLPGFCAA|2005-03-05|1262|5488|422|2005|6|3|5|1|2005|422|5488|Saturday|2005Q1|N|Y|N|2453431|2453489|2453070|2453343|N|N|N|N|N| +2453436|AAAAAAAAMLPGFCAA|2005-03-06|1262|5488|422|2005|0|3|6|1|2005|422|5488|Sunday|2005Q1|N|N|N|2453431|2453489|2453071|2453344|N|N|N|N|N| +2453437|AAAAAAAANLPGFCAA|2005-03-07|1262|5488|422|2005|1|3|7|1|2005|422|5488|Monday|2005Q1|N|N|N|2453431|2453489|2453072|2453345|N|N|N|N|N| +2453438|AAAAAAAAOLPGFCAA|2005-03-08|1262|5489|422|2005|2|3|8|1|2005|422|5489|Tuesday|2005Q1|N|N|N|2453431|2453489|2453073|2453346|N|N|N|N|N| +2453439|AAAAAAAAPLPGFCAA|2005-03-09|1262|5489|422|2005|3|3|9|1|2005|422|5489|Wednesday|2005Q1|N|N|N|2453431|2453489|2453074|2453347|N|N|N|N|N| +2453440|AAAAAAAAAMPGFCAA|2005-03-10|1262|5489|422|2005|4|3|10|1|2005|422|5489|Thursday|2005Q1|N|N|N|2453431|2453489|2453075|2453348|N|N|N|N|N| +2453441|AAAAAAAABMPGFCAA|2005-03-11|1262|5489|422|2005|5|3|11|1|2005|422|5489|Friday|2005Q1|N|Y|N|2453431|2453489|2453076|2453349|N|N|N|N|N| +2453442|AAAAAAAACMPGFCAA|2005-03-12|1262|5489|422|2005|6|3|12|1|2005|422|5489|Saturday|2005Q1|N|Y|N|2453431|2453489|2453077|2453350|N|N|N|N|N| +2453443|AAAAAAAADMPGFCAA|2005-03-13|1262|5489|422|2005|0|3|13|1|2005|422|5489|Sunday|2005Q1|N|N|N|2453431|2453489|2453078|2453351|N|N|N|N|N| +2453444|AAAAAAAAEMPGFCAA|2005-03-14|1262|5489|422|2005|1|3|14|1|2005|422|5489|Monday|2005Q1|N|N|N|2453431|2453489|2453079|2453352|N|N|N|N|N| +2453445|AAAAAAAAFMPGFCAA|2005-03-15|1262|5490|422|2005|2|3|15|1|2005|422|5490|Tuesday|2005Q1|N|N|N|2453431|2453489|2453080|2453353|N|N|N|N|N| +2453446|AAAAAAAAGMPGFCAA|2005-03-16|1262|5490|422|2005|3|3|16|1|2005|422|5490|Wednesday|2005Q1|N|N|N|2453431|2453489|2453081|2453354|N|N|N|N|N| +2453447|AAAAAAAAHMPGFCAA|2005-03-17|1262|5490|422|2005|4|3|17|1|2005|422|5490|Thursday|2005Q1|N|N|N|2453431|2453489|2453082|2453355|N|N|N|N|N| +2453448|AAAAAAAAIMPGFCAA|2005-03-18|1262|5490|422|2005|5|3|18|1|2005|422|5490|Friday|2005Q1|N|Y|N|2453431|2453489|2453083|2453356|N|N|N|N|N| +2453449|AAAAAAAAJMPGFCAA|2005-03-19|1262|5490|422|2005|6|3|19|1|2005|422|5490|Saturday|2005Q1|N|Y|N|2453431|2453489|2453084|2453357|N|N|N|N|N| +2453450|AAAAAAAAKMPGFCAA|2005-03-20|1262|5490|422|2005|0|3|20|1|2005|422|5490|Sunday|2005Q1|N|N|N|2453431|2453489|2453085|2453358|N|N|N|N|N| +2453451|AAAAAAAALMPGFCAA|2005-03-21|1262|5490|422|2005|1|3|21|1|2005|422|5490|Monday|2005Q1|N|N|N|2453431|2453489|2453086|2453359|N|N|N|N|N| +2453452|AAAAAAAAMMPGFCAA|2005-03-22|1262|5491|422|2005|2|3|22|1|2005|422|5491|Tuesday|2005Q1|N|N|N|2453431|2453489|2453087|2453360|N|N|N|N|N| +2453453|AAAAAAAANMPGFCAA|2005-03-23|1262|5491|422|2005|3|3|23|1|2005|422|5491|Wednesday|2005Q1|N|N|N|2453431|2453489|2453088|2453361|N|N|N|N|N| +2453454|AAAAAAAAOMPGFCAA|2005-03-24|1262|5491|422|2005|4|3|24|1|2005|422|5491|Thursday|2005Q1|N|N|N|2453431|2453489|2453089|2453362|N|N|N|N|N| +2453455|AAAAAAAAPMPGFCAA|2005-03-25|1262|5491|422|2005|5|3|25|1|2005|422|5491|Friday|2005Q1|N|Y|N|2453431|2453489|2453090|2453363|N|N|N|N|N| +2453456|AAAAAAAAANPGFCAA|2005-03-26|1262|5491|422|2005|6|3|26|1|2005|422|5491|Saturday|2005Q1|N|Y|N|2453431|2453489|2453091|2453364|N|N|N|N|N| +2453457|AAAAAAAABNPGFCAA|2005-03-27|1262|5491|422|2005|0|3|27|1|2005|422|5491|Sunday|2005Q1|N|N|N|2453431|2453489|2453092|2453365|N|N|N|N|N| +2453458|AAAAAAAACNPGFCAA|2005-03-28|1262|5491|422|2005|1|3|28|1|2005|422|5491|Monday|2005Q1|N|N|N|2453431|2453489|2453093|2453366|N|N|N|N|N| +2453459|AAAAAAAADNPGFCAA|2005-03-29|1262|5492|422|2005|2|3|29|1|2005|422|5492|Tuesday|2005Q1|N|N|N|2453431|2453489|2453094|2453367|N|N|N|N|N| +2453460|AAAAAAAAENPGFCAA|2005-03-30|1262|5492|422|2005|3|3|30|1|2005|422|5492|Wednesday|2005Q1|N|N|N|2453431|2453489|2453095|2453368|N|N|N|N|N| +2453461|AAAAAAAAFNPGFCAA|2005-03-31|1262|5492|422|2005|4|3|31|1|2005|422|5492|Thursday|2005Q1|N|N|N|2453431|2453489|2453096|2453369|N|N|N|N|N| +2453462|AAAAAAAAGNPGFCAA|2005-04-01|1263|5492|422|2005|5|4|1|1|2005|422|5492|Friday|2005Q1|N|Y|N|2453462|2453551|2453097|2453372|N|N|N|N|N| +2453463|AAAAAAAAHNPGFCAA|2005-04-02|1263|5492|422|2005|6|4|2|2|2005|422|5492|Saturday|2005Q2|N|Y|N|2453462|2453551|2453098|2453373|N|N|N|N|N| +2453464|AAAAAAAAINPGFCAA|2005-04-03|1263|5492|422|2005|0|4|3|2|2005|422|5492|Sunday|2005Q2|N|N|N|2453462|2453551|2453099|2453374|N|N|N|N|N| +2453465|AAAAAAAAJNPGFCAA|2005-04-04|1263|5492|422|2005|1|4|4|2|2005|422|5492|Monday|2005Q2|N|N|N|2453462|2453551|2453100|2453375|N|N|N|N|N| +2453466|AAAAAAAAKNPGFCAA|2005-04-05|1263|5493|422|2005|2|4|5|2|2005|422|5493|Tuesday|2005Q2|N|N|N|2453462|2453551|2453101|2453376|N|N|N|N|N| +2453467|AAAAAAAALNPGFCAA|2005-04-06|1263|5493|422|2005|3|4|6|2|2005|422|5493|Wednesday|2005Q2|N|N|N|2453462|2453551|2453102|2453377|N|N|N|N|N| +2453468|AAAAAAAAMNPGFCAA|2005-04-07|1263|5493|422|2005|4|4|7|2|2005|422|5493|Thursday|2005Q2|N|N|N|2453462|2453551|2453103|2453378|N|N|N|N|N| +2453469|AAAAAAAANNPGFCAA|2005-04-08|1263|5493|422|2005|5|4|8|2|2005|422|5493|Friday|2005Q2|N|Y|N|2453462|2453551|2453104|2453379|N|N|N|N|N| +2453470|AAAAAAAAONPGFCAA|2005-04-09|1263|5493|422|2005|6|4|9|2|2005|422|5493|Saturday|2005Q2|N|Y|N|2453462|2453551|2453105|2453380|N|N|N|N|N| +2453471|AAAAAAAAPNPGFCAA|2005-04-10|1263|5493|422|2005|0|4|10|2|2005|422|5493|Sunday|2005Q2|N|N|N|2453462|2453551|2453106|2453381|N|N|N|N|N| +2453472|AAAAAAAAAOPGFCAA|2005-04-11|1263|5493|422|2005|1|4|11|2|2005|422|5493|Monday|2005Q2|N|N|N|2453462|2453551|2453107|2453382|N|N|N|N|N| +2453473|AAAAAAAABOPGFCAA|2005-04-12|1263|5494|422|2005|2|4|12|2|2005|422|5494|Tuesday|2005Q2|N|N|N|2453462|2453551|2453108|2453383|N|N|N|N|N| +2453474|AAAAAAAACOPGFCAA|2005-04-13|1263|5494|422|2005|3|4|13|2|2005|422|5494|Wednesday|2005Q2|N|N|N|2453462|2453551|2453109|2453384|N|N|N|N|N| +2453475|AAAAAAAADOPGFCAA|2005-04-14|1263|5494|422|2005|4|4|14|2|2005|422|5494|Thursday|2005Q2|N|N|N|2453462|2453551|2453110|2453385|N|N|N|N|N| +2453476|AAAAAAAAEOPGFCAA|2005-04-15|1263|5494|422|2005|5|4|15|2|2005|422|5494|Friday|2005Q2|N|Y|N|2453462|2453551|2453111|2453386|N|N|N|N|N| +2453477|AAAAAAAAFOPGFCAA|2005-04-16|1263|5494|422|2005|6|4|16|2|2005|422|5494|Saturday|2005Q2|N|Y|N|2453462|2453551|2453112|2453387|N|N|N|N|N| +2453478|AAAAAAAAGOPGFCAA|2005-04-17|1263|5494|422|2005|0|4|17|2|2005|422|5494|Sunday|2005Q2|N|N|N|2453462|2453551|2453113|2453388|N|N|N|N|N| +2453479|AAAAAAAAHOPGFCAA|2005-04-18|1263|5494|422|2005|1|4|18|2|2005|422|5494|Monday|2005Q2|N|N|N|2453462|2453551|2453114|2453389|N|N|N|N|N| +2453480|AAAAAAAAIOPGFCAA|2005-04-19|1263|5495|422|2005|2|4|19|2|2005|422|5495|Tuesday|2005Q2|N|N|N|2453462|2453551|2453115|2453390|N|N|N|N|N| +2453481|AAAAAAAAJOPGFCAA|2005-04-20|1263|5495|422|2005|3|4|20|2|2005|422|5495|Wednesday|2005Q2|N|N|N|2453462|2453551|2453116|2453391|N|N|N|N|N| +2453482|AAAAAAAAKOPGFCAA|2005-04-21|1263|5495|422|2005|4|4|21|2|2005|422|5495|Thursday|2005Q2|N|N|N|2453462|2453551|2453117|2453392|N|N|N|N|N| +2453483|AAAAAAAALOPGFCAA|2005-04-22|1263|5495|422|2005|5|4|22|2|2005|422|5495|Friday|2005Q2|N|Y|N|2453462|2453551|2453118|2453393|N|N|N|N|N| +2453484|AAAAAAAAMOPGFCAA|2005-04-23|1263|5495|422|2005|6|4|23|2|2005|422|5495|Saturday|2005Q2|N|Y|N|2453462|2453551|2453119|2453394|N|N|N|N|N| +2453485|AAAAAAAANOPGFCAA|2005-04-24|1263|5495|422|2005|0|4|24|2|2005|422|5495|Sunday|2005Q2|N|N|N|2453462|2453551|2453120|2453395|N|N|N|N|N| +2453486|AAAAAAAAOOPGFCAA|2005-04-25|1263|5495|422|2005|1|4|25|2|2005|422|5495|Monday|2005Q2|N|N|N|2453462|2453551|2453121|2453396|N|N|N|N|N| +2453487|AAAAAAAAPOPGFCAA|2005-04-26|1263|5496|422|2005|2|4|26|2|2005|422|5496|Tuesday|2005Q2|N|N|N|2453462|2453551|2453122|2453397|N|N|N|N|N| +2453488|AAAAAAAAAPPGFCAA|2005-04-27|1263|5496|422|2005|3|4|27|2|2005|422|5496|Wednesday|2005Q2|N|N|N|2453462|2453551|2453123|2453398|N|N|N|N|N| +2453489|AAAAAAAABPPGFCAA|2005-04-28|1263|5496|422|2005|4|4|28|2|2005|422|5496|Thursday|2005Q2|N|N|N|2453462|2453551|2453124|2453399|N|N|N|N|N| +2453490|AAAAAAAACPPGFCAA|2005-04-29|1263|5496|422|2005|5|4|29|2|2005|422|5496|Friday|2005Q2|N|Y|N|2453462|2453551|2453125|2453400|N|N|N|N|N| +2453491|AAAAAAAADPPGFCAA|2005-04-30|1263|5496|422|2005|6|4|30|2|2005|422|5496|Saturday|2005Q2|N|Y|N|2453462|2453551|2453126|2453401|N|N|N|N|N| +2453492|AAAAAAAAEPPGFCAA|2005-05-01|1264|5496|422|2005|0|5|1|2|2005|422|5496|Sunday|2005Q2|N|N|N|2453492|2453611|2453127|2453402|N|N|N|N|N| +2453493|AAAAAAAAFPPGFCAA|2005-05-02|1264|5496|422|2005|1|5|2|2|2005|422|5496|Monday|2005Q2|N|N|N|2453492|2453611|2453128|2453403|N|N|N|N|N| +2453494|AAAAAAAAGPPGFCAA|2005-05-03|1264|5497|422|2005|2|5|3|2|2005|422|5497|Tuesday|2005Q2|N|N|N|2453492|2453611|2453129|2453404|N|N|N|N|N| +2453495|AAAAAAAAHPPGFCAA|2005-05-04|1264|5497|422|2005|3|5|4|2|2005|422|5497|Wednesday|2005Q2|N|N|N|2453492|2453611|2453130|2453405|N|N|N|N|N| +2453496|AAAAAAAAIPPGFCAA|2005-05-05|1264|5497|422|2005|4|5|5|2|2005|422|5497|Thursday|2005Q2|N|N|N|2453492|2453611|2453131|2453406|N|N|N|N|N| +2453497|AAAAAAAAJPPGFCAA|2005-05-06|1264|5497|422|2005|5|5|6|2|2005|422|5497|Friday|2005Q2|N|Y|N|2453492|2453611|2453132|2453407|N|N|N|N|N| +2453498|AAAAAAAAKPPGFCAA|2005-05-07|1264|5497|422|2005|6|5|7|2|2005|422|5497|Saturday|2005Q2|N|Y|N|2453492|2453611|2453133|2453408|N|N|N|N|N| +2453499|AAAAAAAALPPGFCAA|2005-05-08|1264|5497|422|2005|0|5|8|2|2005|422|5497|Sunday|2005Q2|N|N|N|2453492|2453611|2453134|2453409|N|N|N|N|N| +2453500|AAAAAAAAMPPGFCAA|2005-05-09|1264|5497|422|2005|1|5|9|2|2005|422|5497|Monday|2005Q2|N|N|N|2453492|2453611|2453135|2453410|N|N|N|N|N| +2453501|AAAAAAAANPPGFCAA|2005-05-10|1264|5498|422|2005|2|5|10|2|2005|422|5498|Tuesday|2005Q2|N|N|N|2453492|2453611|2453136|2453411|N|N|N|N|N| +2453502|AAAAAAAAOPPGFCAA|2005-05-11|1264|5498|422|2005|3|5|11|2|2005|422|5498|Wednesday|2005Q2|N|N|N|2453492|2453611|2453137|2453412|N|N|N|N|N| +2453503|AAAAAAAAPPPGFCAA|2005-05-12|1264|5498|422|2005|4|5|12|2|2005|422|5498|Thursday|2005Q2|N|N|N|2453492|2453611|2453138|2453413|N|N|N|N|N| +2453504|AAAAAAAAAAAHFCAA|2005-05-13|1264|5498|422|2005|5|5|13|2|2005|422|5498|Friday|2005Q2|N|Y|N|2453492|2453611|2453139|2453414|N|N|N|N|N| +2453505|AAAAAAAABAAHFCAA|2005-05-14|1264|5498|422|2005|6|5|14|2|2005|422|5498|Saturday|2005Q2|N|Y|N|2453492|2453611|2453140|2453415|N|N|N|N|N| +2453506|AAAAAAAACAAHFCAA|2005-05-15|1264|5498|422|2005|0|5|15|2|2005|422|5498|Sunday|2005Q2|N|N|N|2453492|2453611|2453141|2453416|N|N|N|N|N| +2453507|AAAAAAAADAAHFCAA|2005-05-16|1264|5498|422|2005|1|5|16|2|2005|422|5498|Monday|2005Q2|N|N|N|2453492|2453611|2453142|2453417|N|N|N|N|N| +2453508|AAAAAAAAEAAHFCAA|2005-05-17|1264|5499|422|2005|2|5|17|2|2005|422|5499|Tuesday|2005Q2|N|N|N|2453492|2453611|2453143|2453418|N|N|N|N|N| +2453509|AAAAAAAAFAAHFCAA|2005-05-18|1264|5499|422|2005|3|5|18|2|2005|422|5499|Wednesday|2005Q2|N|N|N|2453492|2453611|2453144|2453419|N|N|N|N|N| +2453510|AAAAAAAAGAAHFCAA|2005-05-19|1264|5499|422|2005|4|5|19|2|2005|422|5499|Thursday|2005Q2|N|N|N|2453492|2453611|2453145|2453420|N|N|N|N|N| +2453511|AAAAAAAAHAAHFCAA|2005-05-20|1264|5499|422|2005|5|5|20|2|2005|422|5499|Friday|2005Q2|N|Y|N|2453492|2453611|2453146|2453421|N|N|N|N|N| +2453512|AAAAAAAAIAAHFCAA|2005-05-21|1264|5499|422|2005|6|5|21|2|2005|422|5499|Saturday|2005Q2|N|Y|N|2453492|2453611|2453147|2453422|N|N|N|N|N| +2453513|AAAAAAAAJAAHFCAA|2005-05-22|1264|5499|422|2005|0|5|22|2|2005|422|5499|Sunday|2005Q2|N|N|N|2453492|2453611|2453148|2453423|N|N|N|N|N| +2453514|AAAAAAAAKAAHFCAA|2005-05-23|1264|5499|422|2005|1|5|23|2|2005|422|5499|Monday|2005Q2|N|N|N|2453492|2453611|2453149|2453424|N|N|N|N|N| +2453515|AAAAAAAALAAHFCAA|2005-05-24|1264|5500|422|2005|2|5|24|2|2005|422|5500|Tuesday|2005Q2|N|N|N|2453492|2453611|2453150|2453425|N|N|N|N|N| +2453516|AAAAAAAAMAAHFCAA|2005-05-25|1264|5500|422|2005|3|5|25|2|2005|422|5500|Wednesday|2005Q2|N|N|N|2453492|2453611|2453151|2453426|N|N|N|N|N| +2453517|AAAAAAAANAAHFCAA|2005-05-26|1264|5500|422|2005|4|5|26|2|2005|422|5500|Thursday|2005Q2|N|N|N|2453492|2453611|2453152|2453427|N|N|N|N|N| +2453518|AAAAAAAAOAAHFCAA|2005-05-27|1264|5500|422|2005|5|5|27|2|2005|422|5500|Friday|2005Q2|N|Y|N|2453492|2453611|2453153|2453428|N|N|N|N|N| +2453519|AAAAAAAAPAAHFCAA|2005-05-28|1264|5500|422|2005|6|5|28|2|2005|422|5500|Saturday|2005Q2|N|Y|N|2453492|2453611|2453154|2453429|N|N|N|N|N| +2453520|AAAAAAAAABAHFCAA|2005-05-29|1264|5500|422|2005|0|5|29|2|2005|422|5500|Sunday|2005Q2|N|N|N|2453492|2453611|2453155|2453430|N|N|N|N|N| +2453521|AAAAAAAABBAHFCAA|2005-05-30|1264|5500|422|2005|1|5|30|2|2005|422|5500|Monday|2005Q2|N|N|N|2453492|2453611|2453156|2453431|N|N|N|N|N| +2453522|AAAAAAAACBAHFCAA|2005-05-31|1264|5501|422|2005|2|5|31|2|2005|422|5501|Tuesday|2005Q2|N|N|N|2453492|2453611|2453157|2453432|N|N|N|N|N| +2453523|AAAAAAAADBAHFCAA|2005-06-01|1265|5501|423|2005|3|6|1|2|2005|423|5501|Wednesday|2005Q2|N|N|N|2453523|2453673|2453158|2453433|N|N|N|N|N| +2453524|AAAAAAAAEBAHFCAA|2005-06-02|1265|5501|423|2005|4|6|2|2|2005|423|5501|Thursday|2005Q2|N|N|N|2453523|2453673|2453159|2453434|N|N|N|N|N| +2453525|AAAAAAAAFBAHFCAA|2005-06-03|1265|5501|423|2005|5|6|3|2|2005|423|5501|Friday|2005Q2|N|Y|N|2453523|2453673|2453160|2453435|N|N|N|N|N| +2453526|AAAAAAAAGBAHFCAA|2005-06-04|1265|5501|423|2005|6|6|4|2|2005|423|5501|Saturday|2005Q2|N|Y|N|2453523|2453673|2453161|2453436|N|N|N|N|N| +2453527|AAAAAAAAHBAHFCAA|2005-06-05|1265|5501|423|2005|0|6|5|2|2005|423|5501|Sunday|2005Q2|N|N|N|2453523|2453673|2453162|2453437|N|N|N|N|N| +2453528|AAAAAAAAIBAHFCAA|2005-06-06|1265|5501|423|2005|1|6|6|2|2005|423|5501|Monday|2005Q2|N|N|N|2453523|2453673|2453163|2453438|N|N|N|N|N| +2453529|AAAAAAAAJBAHFCAA|2005-06-07|1265|5502|423|2005|2|6|7|2|2005|423|5502|Tuesday|2005Q2|N|N|N|2453523|2453673|2453164|2453439|N|N|N|N|N| +2453530|AAAAAAAAKBAHFCAA|2005-06-08|1265|5502|423|2005|3|6|8|2|2005|423|5502|Wednesday|2005Q2|N|N|N|2453523|2453673|2453165|2453440|N|N|N|N|N| +2453531|AAAAAAAALBAHFCAA|2005-06-09|1265|5502|423|2005|4|6|9|2|2005|423|5502|Thursday|2005Q2|N|N|N|2453523|2453673|2453166|2453441|N|N|N|N|N| +2453532|AAAAAAAAMBAHFCAA|2005-06-10|1265|5502|423|2005|5|6|10|2|2005|423|5502|Friday|2005Q2|N|Y|N|2453523|2453673|2453167|2453442|N|N|N|N|N| +2453533|AAAAAAAANBAHFCAA|2005-06-11|1265|5502|423|2005|6|6|11|2|2005|423|5502|Saturday|2005Q2|N|Y|N|2453523|2453673|2453168|2453443|N|N|N|N|N| +2453534|AAAAAAAAOBAHFCAA|2005-06-12|1265|5502|423|2005|0|6|12|2|2005|423|5502|Sunday|2005Q2|N|N|N|2453523|2453673|2453169|2453444|N|N|N|N|N| +2453535|AAAAAAAAPBAHFCAA|2005-06-13|1265|5502|423|2005|1|6|13|2|2005|423|5502|Monday|2005Q2|N|N|N|2453523|2453673|2453170|2453445|N|N|N|N|N| +2453536|AAAAAAAAACAHFCAA|2005-06-14|1265|5503|423|2005|2|6|14|2|2005|423|5503|Tuesday|2005Q2|N|N|N|2453523|2453673|2453171|2453446|N|N|N|N|N| +2453537|AAAAAAAABCAHFCAA|2005-06-15|1265|5503|423|2005|3|6|15|2|2005|423|5503|Wednesday|2005Q2|N|N|N|2453523|2453673|2453172|2453447|N|N|N|N|N| +2453538|AAAAAAAACCAHFCAA|2005-06-16|1265|5503|423|2005|4|6|16|2|2005|423|5503|Thursday|2005Q2|N|N|N|2453523|2453673|2453173|2453448|N|N|N|N|N| +2453539|AAAAAAAADCAHFCAA|2005-06-17|1265|5503|423|2005|5|6|17|2|2005|423|5503|Friday|2005Q2|N|Y|N|2453523|2453673|2453174|2453449|N|N|N|N|N| +2453540|AAAAAAAAECAHFCAA|2005-06-18|1265|5503|423|2005|6|6|18|2|2005|423|5503|Saturday|2005Q2|N|Y|N|2453523|2453673|2453175|2453450|N|N|N|N|N| +2453541|AAAAAAAAFCAHFCAA|2005-06-19|1265|5503|423|2005|0|6|19|2|2005|423|5503|Sunday|2005Q2|N|N|N|2453523|2453673|2453176|2453451|N|N|N|N|N| +2453542|AAAAAAAAGCAHFCAA|2005-06-20|1265|5503|423|2005|1|6|20|2|2005|423|5503|Monday|2005Q2|N|N|N|2453523|2453673|2453177|2453452|N|N|N|N|N| +2453543|AAAAAAAAHCAHFCAA|2005-06-21|1265|5504|423|2005|2|6|21|2|2005|423|5504|Tuesday|2005Q2|N|N|N|2453523|2453673|2453178|2453453|N|N|N|N|N| +2453544|AAAAAAAAICAHFCAA|2005-06-22|1265|5504|423|2005|3|6|22|2|2005|423|5504|Wednesday|2005Q2|N|N|N|2453523|2453673|2453179|2453454|N|N|N|N|N| +2453545|AAAAAAAAJCAHFCAA|2005-06-23|1265|5504|423|2005|4|6|23|2|2005|423|5504|Thursday|2005Q2|N|N|N|2453523|2453673|2453180|2453455|N|N|N|N|N| +2453546|AAAAAAAAKCAHFCAA|2005-06-24|1265|5504|423|2005|5|6|24|2|2005|423|5504|Friday|2005Q2|N|Y|N|2453523|2453673|2453181|2453456|N|N|N|N|N| +2453547|AAAAAAAALCAHFCAA|2005-06-25|1265|5504|423|2005|6|6|25|2|2005|423|5504|Saturday|2005Q2|N|Y|N|2453523|2453673|2453182|2453457|N|N|N|N|N| +2453548|AAAAAAAAMCAHFCAA|2005-06-26|1265|5504|423|2005|0|6|26|2|2005|423|5504|Sunday|2005Q2|N|N|N|2453523|2453673|2453183|2453458|N|N|N|N|N| +2453549|AAAAAAAANCAHFCAA|2005-06-27|1265|5504|423|2005|1|6|27|2|2005|423|5504|Monday|2005Q2|N|N|N|2453523|2453673|2453184|2453459|N|N|N|N|N| +2453550|AAAAAAAAOCAHFCAA|2005-06-28|1265|5505|423|2005|2|6|28|2|2005|423|5505|Tuesday|2005Q2|N|N|N|2453523|2453673|2453185|2453460|N|N|N|N|N| +2453551|AAAAAAAAPCAHFCAA|2005-06-29|1265|5505|423|2005|3|6|29|2|2005|423|5505|Wednesday|2005Q2|N|N|N|2453523|2453673|2453186|2453461|N|N|N|N|N| +2453552|AAAAAAAAADAHFCAA|2005-06-30|1265|5505|423|2005|4|6|30|2|2005|423|5505|Thursday|2005Q2|N|N|N|2453523|2453673|2453187|2453462|N|N|N|N|N| +2453553|AAAAAAAABDAHFCAA|2005-07-01|1266|5505|423|2005|5|7|1|2|2005|423|5505|Friday|2005Q2|N|Y|N|2453553|2453733|2453188|2453462|N|N|N|N|N| +2453554|AAAAAAAACDAHFCAA|2005-07-02|1266|5505|423|2005|6|7|2|3|2005|423|5505|Saturday|2005Q3|N|Y|N|2453553|2453733|2453189|2453463|N|N|N|N|N| +2453555|AAAAAAAADDAHFCAA|2005-07-03|1266|5505|423|2005|0|7|3|3|2005|423|5505|Sunday|2005Q3|N|N|N|2453553|2453733|2453190|2453464|N|N|N|N|N| +2453556|AAAAAAAAEDAHFCAA|2005-07-04|1266|5505|423|2005|1|7|4|3|2005|423|5505|Monday|2005Q3|N|N|N|2453553|2453733|2453191|2453465|N|N|N|N|N| +2453557|AAAAAAAAFDAHFCAA|2005-07-05|1266|5506|423|2005|2|7|5|3|2005|423|5506|Tuesday|2005Q3|Y|N|N|2453553|2453733|2453192|2453466|N|N|N|N|N| +2453558|AAAAAAAAGDAHFCAA|2005-07-06|1266|5506|423|2005|3|7|6|3|2005|423|5506|Wednesday|2005Q3|N|N|Y|2453553|2453733|2453193|2453467|N|N|N|N|N| +2453559|AAAAAAAAHDAHFCAA|2005-07-07|1266|5506|423|2005|4|7|7|3|2005|423|5506|Thursday|2005Q3|N|N|N|2453553|2453733|2453194|2453468|N|N|N|N|N| +2453560|AAAAAAAAIDAHFCAA|2005-07-08|1266|5506|423|2005|5|7|8|3|2005|423|5506|Friday|2005Q3|N|Y|N|2453553|2453733|2453195|2453469|N|N|N|N|N| +2453561|AAAAAAAAJDAHFCAA|2005-07-09|1266|5506|423|2005|6|7|9|3|2005|423|5506|Saturday|2005Q3|N|Y|N|2453553|2453733|2453196|2453470|N|N|N|N|N| +2453562|AAAAAAAAKDAHFCAA|2005-07-10|1266|5506|423|2005|0|7|10|3|2005|423|5506|Sunday|2005Q3|N|N|N|2453553|2453733|2453197|2453471|N|N|N|N|N| +2453563|AAAAAAAALDAHFCAA|2005-07-11|1266|5506|423|2005|1|7|11|3|2005|423|5506|Monday|2005Q3|N|N|N|2453553|2453733|2453198|2453472|N|N|N|N|N| +2453564|AAAAAAAAMDAHFCAA|2005-07-12|1266|5507|423|2005|2|7|12|3|2005|423|5507|Tuesday|2005Q3|N|N|N|2453553|2453733|2453199|2453473|N|N|N|N|N| +2453565|AAAAAAAANDAHFCAA|2005-07-13|1266|5507|423|2005|3|7|13|3|2005|423|5507|Wednesday|2005Q3|N|N|N|2453553|2453733|2453200|2453474|N|N|N|N|N| +2453566|AAAAAAAAODAHFCAA|2005-07-14|1266|5507|423|2005|4|7|14|3|2005|423|5507|Thursday|2005Q3|N|N|N|2453553|2453733|2453201|2453475|N|N|N|N|N| +2453567|AAAAAAAAPDAHFCAA|2005-07-15|1266|5507|423|2005|5|7|15|3|2005|423|5507|Friday|2005Q3|N|Y|N|2453553|2453733|2453202|2453476|N|N|N|N|N| +2453568|AAAAAAAAAEAHFCAA|2005-07-16|1266|5507|423|2005|6|7|16|3|2005|423|5507|Saturday|2005Q3|N|Y|N|2453553|2453733|2453203|2453477|N|N|N|N|N| +2453569|AAAAAAAABEAHFCAA|2005-07-17|1266|5507|423|2005|0|7|17|3|2005|423|5507|Sunday|2005Q3|N|N|N|2453553|2453733|2453204|2453478|N|N|N|N|N| +2453570|AAAAAAAACEAHFCAA|2005-07-18|1266|5507|423|2005|1|7|18|3|2005|423|5507|Monday|2005Q3|N|N|N|2453553|2453733|2453205|2453479|N|N|N|N|N| +2453571|AAAAAAAADEAHFCAA|2005-07-19|1266|5508|423|2005|2|7|19|3|2005|423|5508|Tuesday|2005Q3|N|N|N|2453553|2453733|2453206|2453480|N|N|N|N|N| +2453572|AAAAAAAAEEAHFCAA|2005-07-20|1266|5508|423|2005|3|7|20|3|2005|423|5508|Wednesday|2005Q3|N|N|N|2453553|2453733|2453207|2453481|N|N|N|N|N| +2453573|AAAAAAAAFEAHFCAA|2005-07-21|1266|5508|423|2005|4|7|21|3|2005|423|5508|Thursday|2005Q3|N|N|N|2453553|2453733|2453208|2453482|N|N|N|N|N| +2453574|AAAAAAAAGEAHFCAA|2005-07-22|1266|5508|423|2005|5|7|22|3|2005|423|5508|Friday|2005Q3|N|Y|N|2453553|2453733|2453209|2453483|N|N|N|N|N| +2453575|AAAAAAAAHEAHFCAA|2005-07-23|1266|5508|423|2005|6|7|23|3|2005|423|5508|Saturday|2005Q3|N|Y|N|2453553|2453733|2453210|2453484|N|N|N|N|N| +2453576|AAAAAAAAIEAHFCAA|2005-07-24|1266|5508|423|2005|0|7|24|3|2005|423|5508|Sunday|2005Q3|N|N|N|2453553|2453733|2453211|2453485|N|N|N|N|N| +2453577|AAAAAAAAJEAHFCAA|2005-07-25|1266|5508|423|2005|1|7|25|3|2005|423|5508|Monday|2005Q3|N|N|N|2453553|2453733|2453212|2453486|N|N|N|N|N| +2453578|AAAAAAAAKEAHFCAA|2005-07-26|1266|5509|423|2005|2|7|26|3|2005|423|5509|Tuesday|2005Q3|N|N|N|2453553|2453733|2453213|2453487|N|N|N|N|N| +2453579|AAAAAAAALEAHFCAA|2005-07-27|1266|5509|423|2005|3|7|27|3|2005|423|5509|Wednesday|2005Q3|N|N|N|2453553|2453733|2453214|2453488|N|N|N|N|N| +2453580|AAAAAAAAMEAHFCAA|2005-07-28|1266|5509|423|2005|4|7|28|3|2005|423|5509|Thursday|2005Q3|N|N|N|2453553|2453733|2453215|2453489|N|N|N|N|N| +2453581|AAAAAAAANEAHFCAA|2005-07-29|1266|5509|423|2005|5|7|29|3|2005|423|5509|Friday|2005Q3|N|Y|N|2453553|2453733|2453216|2453490|N|N|N|N|N| +2453582|AAAAAAAAOEAHFCAA|2005-07-30|1266|5509|423|2005|6|7|30|3|2005|423|5509|Saturday|2005Q3|N|Y|N|2453553|2453733|2453217|2453491|N|N|N|N|N| +2453583|AAAAAAAAPEAHFCAA|2005-07-31|1266|5509|423|2005|0|7|31|3|2005|423|5509|Sunday|2005Q3|N|N|N|2453553|2453733|2453218|2453492|N|N|N|N|N| +2453584|AAAAAAAAAFAHFCAA|2005-08-01|1267|5509|423|2005|1|8|1|3|2005|423|5509|Monday|2005Q3|N|N|N|2453584|2453795|2453219|2453493|N|N|N|N|N| +2453585|AAAAAAAABFAHFCAA|2005-08-02|1267|5510|423|2005|2|8|2|3|2005|423|5510|Tuesday|2005Q3|N|N|N|2453584|2453795|2453220|2453494|N|N|N|N|N| +2453586|AAAAAAAACFAHFCAA|2005-08-03|1267|5510|423|2005|3|8|3|3|2005|423|5510|Wednesday|2005Q3|N|N|N|2453584|2453795|2453221|2453495|N|N|N|N|N| +2453587|AAAAAAAADFAHFCAA|2005-08-04|1267|5510|423|2005|4|8|4|3|2005|423|5510|Thursday|2005Q3|N|N|N|2453584|2453795|2453222|2453496|N|N|N|N|N| +2453588|AAAAAAAAEFAHFCAA|2005-08-05|1267|5510|423|2005|5|8|5|3|2005|423|5510|Friday|2005Q3|N|Y|N|2453584|2453795|2453223|2453497|N|N|N|N|N| +2453589|AAAAAAAAFFAHFCAA|2005-08-06|1267|5510|423|2005|6|8|6|3|2005|423|5510|Saturday|2005Q3|N|Y|N|2453584|2453795|2453224|2453498|N|N|N|N|N| +2453590|AAAAAAAAGFAHFCAA|2005-08-07|1267|5510|423|2005|0|8|7|3|2005|423|5510|Sunday|2005Q3|N|N|N|2453584|2453795|2453225|2453499|N|N|N|N|N| +2453591|AAAAAAAAHFAHFCAA|2005-08-08|1267|5510|423|2005|1|8|8|3|2005|423|5510|Monday|2005Q3|N|N|N|2453584|2453795|2453226|2453500|N|N|N|N|N| +2453592|AAAAAAAAIFAHFCAA|2005-08-09|1267|5511|423|2005|2|8|9|3|2005|423|5511|Tuesday|2005Q3|N|N|N|2453584|2453795|2453227|2453501|N|N|N|N|N| +2453593|AAAAAAAAJFAHFCAA|2005-08-10|1267|5511|423|2005|3|8|10|3|2005|423|5511|Wednesday|2005Q3|N|N|N|2453584|2453795|2453228|2453502|N|N|N|N|N| +2453594|AAAAAAAAKFAHFCAA|2005-08-11|1267|5511|423|2005|4|8|11|3|2005|423|5511|Thursday|2005Q3|N|N|N|2453584|2453795|2453229|2453503|N|N|N|N|N| +2453595|AAAAAAAALFAHFCAA|2005-08-12|1267|5511|423|2005|5|8|12|3|2005|423|5511|Friday|2005Q3|N|Y|N|2453584|2453795|2453230|2453504|N|N|N|N|N| +2453596|AAAAAAAAMFAHFCAA|2005-08-13|1267|5511|423|2005|6|8|13|3|2005|423|5511|Saturday|2005Q3|N|Y|N|2453584|2453795|2453231|2453505|N|N|N|N|N| +2453597|AAAAAAAANFAHFCAA|2005-08-14|1267|5511|423|2005|0|8|14|3|2005|423|5511|Sunday|2005Q3|N|N|N|2453584|2453795|2453232|2453506|N|N|N|N|N| +2453598|AAAAAAAAOFAHFCAA|2005-08-15|1267|5511|423|2005|1|8|15|3|2005|423|5511|Monday|2005Q3|N|N|N|2453584|2453795|2453233|2453507|N|N|N|N|N| +2453599|AAAAAAAAPFAHFCAA|2005-08-16|1267|5512|423|2005|2|8|16|3|2005|423|5512|Tuesday|2005Q3|N|N|N|2453584|2453795|2453234|2453508|N|N|N|N|N| +2453600|AAAAAAAAAGAHFCAA|2005-08-17|1267|5512|423|2005|3|8|17|3|2005|423|5512|Wednesday|2005Q3|N|N|N|2453584|2453795|2453235|2453509|N|N|N|N|N| +2453601|AAAAAAAABGAHFCAA|2005-08-18|1267|5512|423|2005|4|8|18|3|2005|423|5512|Thursday|2005Q3|N|N|N|2453584|2453795|2453236|2453510|N|N|N|N|N| +2453602|AAAAAAAACGAHFCAA|2005-08-19|1267|5512|423|2005|5|8|19|3|2005|423|5512|Friday|2005Q3|N|Y|N|2453584|2453795|2453237|2453511|N|N|N|N|N| +2453603|AAAAAAAADGAHFCAA|2005-08-20|1267|5512|423|2005|6|8|20|3|2005|423|5512|Saturday|2005Q3|N|Y|N|2453584|2453795|2453238|2453512|N|N|N|N|N| +2453604|AAAAAAAAEGAHFCAA|2005-08-21|1267|5512|423|2005|0|8|21|3|2005|423|5512|Sunday|2005Q3|N|N|N|2453584|2453795|2453239|2453513|N|N|N|N|N| +2453605|AAAAAAAAFGAHFCAA|2005-08-22|1267|5512|423|2005|1|8|22|3|2005|423|5512|Monday|2005Q3|N|N|N|2453584|2453795|2453240|2453514|N|N|N|N|N| +2453606|AAAAAAAAGGAHFCAA|2005-08-23|1267|5513|423|2005|2|8|23|3|2005|423|5513|Tuesday|2005Q3|N|N|N|2453584|2453795|2453241|2453515|N|N|N|N|N| +2453607|AAAAAAAAHGAHFCAA|2005-08-24|1267|5513|423|2005|3|8|24|3|2005|423|5513|Wednesday|2005Q3|N|N|N|2453584|2453795|2453242|2453516|N|N|N|N|N| +2453608|AAAAAAAAIGAHFCAA|2005-08-25|1267|5513|423|2005|4|8|25|3|2005|423|5513|Thursday|2005Q3|N|N|N|2453584|2453795|2453243|2453517|N|N|N|N|N| +2453609|AAAAAAAAJGAHFCAA|2005-08-26|1267|5513|423|2005|5|8|26|3|2005|423|5513|Friday|2005Q3|N|Y|N|2453584|2453795|2453244|2453518|N|N|N|N|N| +2453610|AAAAAAAAKGAHFCAA|2005-08-27|1267|5513|423|2005|6|8|27|3|2005|423|5513|Saturday|2005Q3|N|Y|N|2453584|2453795|2453245|2453519|N|N|N|N|N| +2453611|AAAAAAAALGAHFCAA|2005-08-28|1267|5513|423|2005|0|8|28|3|2005|423|5513|Sunday|2005Q3|N|N|N|2453584|2453795|2453246|2453520|N|N|N|N|N| +2453612|AAAAAAAAMGAHFCAA|2005-08-29|1267|5513|423|2005|1|8|29|3|2005|423|5513|Monday|2005Q3|N|N|N|2453584|2453795|2453247|2453521|N|N|N|N|N| +2453613|AAAAAAAANGAHFCAA|2005-08-30|1267|5514|423|2005|2|8|30|3|2005|423|5514|Tuesday|2005Q3|N|N|N|2453584|2453795|2453248|2453522|N|N|N|N|N| +2453614|AAAAAAAAOGAHFCAA|2005-08-31|1267|5514|423|2005|3|8|31|3|2005|423|5514|Wednesday|2005Q3|N|N|N|2453584|2453795|2453249|2453523|N|N|N|N|N| +2453615|AAAAAAAAPGAHFCAA|2005-09-01|1268|5514|424|2005|4|9|1|3|2005|424|5514|Thursday|2005Q3|N|N|N|2453615|2453857|2453250|2453524|N|N|N|N|N| +2453616|AAAAAAAAAHAHFCAA|2005-09-02|1268|5514|424|2005|5|9|2|3|2005|424|5514|Friday|2005Q3|N|Y|N|2453615|2453857|2453251|2453525|N|N|N|N|N| +2453617|AAAAAAAABHAHFCAA|2005-09-03|1268|5514|424|2005|6|9|3|3|2005|424|5514|Saturday|2005Q3|N|Y|N|2453615|2453857|2453252|2453526|N|N|N|N|N| +2453618|AAAAAAAACHAHFCAA|2005-09-04|1268|5514|424|2005|0|9|4|3|2005|424|5514|Sunday|2005Q3|N|N|N|2453615|2453857|2453253|2453527|N|N|N|N|N| +2453619|AAAAAAAADHAHFCAA|2005-09-05|1268|5514|424|2005|1|9|5|3|2005|424|5514|Monday|2005Q3|N|N|N|2453615|2453857|2453254|2453528|N|N|N|N|N| +2453620|AAAAAAAAEHAHFCAA|2005-09-06|1268|5515|424|2005|2|9|6|3|2005|424|5515|Tuesday|2005Q3|N|N|N|2453615|2453857|2453255|2453529|N|N|N|N|N| +2453621|AAAAAAAAFHAHFCAA|2005-09-07|1268|5515|424|2005|3|9|7|3|2005|424|5515|Wednesday|2005Q3|N|N|N|2453615|2453857|2453256|2453530|N|N|N|N|N| +2453622|AAAAAAAAGHAHFCAA|2005-09-08|1268|5515|424|2005|4|9|8|3|2005|424|5515|Thursday|2005Q3|N|N|N|2453615|2453857|2453257|2453531|N|N|N|N|N| +2453623|AAAAAAAAHHAHFCAA|2005-09-09|1268|5515|424|2005|5|9|9|3|2005|424|5515|Friday|2005Q3|N|Y|N|2453615|2453857|2453258|2453532|N|N|N|N|N| +2453624|AAAAAAAAIHAHFCAA|2005-09-10|1268|5515|424|2005|6|9|10|3|2005|424|5515|Saturday|2005Q3|N|Y|N|2453615|2453857|2453259|2453533|N|N|N|N|N| +2453625|AAAAAAAAJHAHFCAA|2005-09-11|1268|5515|424|2005|0|9|11|3|2005|424|5515|Sunday|2005Q3|N|N|N|2453615|2453857|2453260|2453534|N|N|N|N|N| +2453626|AAAAAAAAKHAHFCAA|2005-09-12|1268|5515|424|2005|1|9|12|3|2005|424|5515|Monday|2005Q3|N|N|N|2453615|2453857|2453261|2453535|N|N|N|N|N| +2453627|AAAAAAAALHAHFCAA|2005-09-13|1268|5516|424|2005|2|9|13|3|2005|424|5516|Tuesday|2005Q3|N|N|N|2453615|2453857|2453262|2453536|N|N|N|N|N| +2453628|AAAAAAAAMHAHFCAA|2005-09-14|1268|5516|424|2005|3|9|14|3|2005|424|5516|Wednesday|2005Q3|N|N|N|2453615|2453857|2453263|2453537|N|N|N|N|N| +2453629|AAAAAAAANHAHFCAA|2005-09-15|1268|5516|424|2005|4|9|15|3|2005|424|5516|Thursday|2005Q3|N|N|N|2453615|2453857|2453264|2453538|N|N|N|N|N| +2453630|AAAAAAAAOHAHFCAA|2005-09-16|1268|5516|424|2005|5|9|16|3|2005|424|5516|Friday|2005Q3|N|Y|N|2453615|2453857|2453265|2453539|N|N|N|N|N| +2453631|AAAAAAAAPHAHFCAA|2005-09-17|1268|5516|424|2005|6|9|17|3|2005|424|5516|Saturday|2005Q3|N|Y|N|2453615|2453857|2453266|2453540|N|N|N|N|N| +2453632|AAAAAAAAAIAHFCAA|2005-09-18|1268|5516|424|2005|0|9|18|3|2005|424|5516|Sunday|2005Q3|N|N|N|2453615|2453857|2453267|2453541|N|N|N|N|N| +2453633|AAAAAAAABIAHFCAA|2005-09-19|1268|5516|424|2005|1|9|19|3|2005|424|5516|Monday|2005Q3|N|N|N|2453615|2453857|2453268|2453542|N|N|N|N|N| +2453634|AAAAAAAACIAHFCAA|2005-09-20|1268|5517|424|2005|2|9|20|3|2005|424|5517|Tuesday|2005Q3|N|N|N|2453615|2453857|2453269|2453543|N|N|N|N|N| +2453635|AAAAAAAADIAHFCAA|2005-09-21|1268|5517|424|2005|3|9|21|3|2005|424|5517|Wednesday|2005Q3|N|N|N|2453615|2453857|2453270|2453544|N|N|N|N|N| +2453636|AAAAAAAAEIAHFCAA|2005-09-22|1268|5517|424|2005|4|9|22|3|2005|424|5517|Thursday|2005Q3|N|N|N|2453615|2453857|2453271|2453545|N|N|N|N|N| +2453637|AAAAAAAAFIAHFCAA|2005-09-23|1268|5517|424|2005|5|9|23|3|2005|424|5517|Friday|2005Q3|N|Y|N|2453615|2453857|2453272|2453546|N|N|N|N|N| +2453638|AAAAAAAAGIAHFCAA|2005-09-24|1268|5517|424|2005|6|9|24|3|2005|424|5517|Saturday|2005Q3|N|Y|N|2453615|2453857|2453273|2453547|N|N|N|N|N| +2453639|AAAAAAAAHIAHFCAA|2005-09-25|1268|5517|424|2005|0|9|25|3|2005|424|5517|Sunday|2005Q3|N|N|N|2453615|2453857|2453274|2453548|N|N|N|N|N| +2453640|AAAAAAAAIIAHFCAA|2005-09-26|1268|5517|424|2005|1|9|26|3|2005|424|5517|Monday|2005Q3|N|N|N|2453615|2453857|2453275|2453549|N|N|N|N|N| +2453641|AAAAAAAAJIAHFCAA|2005-09-27|1268|5518|424|2005|2|9|27|3|2005|424|5518|Tuesday|2005Q3|N|N|N|2453615|2453857|2453276|2453550|N|N|N|N|N| +2453642|AAAAAAAAKIAHFCAA|2005-09-28|1268|5518|424|2005|3|9|28|3|2005|424|5518|Wednesday|2005Q3|N|N|N|2453615|2453857|2453277|2453551|N|N|N|N|N| +2453643|AAAAAAAALIAHFCAA|2005-09-29|1268|5518|424|2005|4|9|29|3|2005|424|5518|Thursday|2005Q3|N|N|N|2453615|2453857|2453278|2453552|N|N|N|N|N| +2453644|AAAAAAAAMIAHFCAA|2005-09-30|1268|5518|424|2005|5|9|30|3|2005|424|5518|Friday|2005Q3|N|Y|N|2453615|2453857|2453279|2453553|N|N|N|N|N| +2453645|AAAAAAAANIAHFCAA|2005-10-01|1269|5518|424|2005|6|10|1|3|2005|424|5518|Saturday|2005Q3|N|Y|N|2453645|2453917|2453280|2453553|N|N|N|N|N| +2453646|AAAAAAAAOIAHFCAA|2005-10-02|1269|5518|424|2005|0|10|2|4|2005|424|5518|Sunday|2005Q4|N|N|N|2453645|2453917|2453281|2453554|N|N|N|N|N| +2453647|AAAAAAAAPIAHFCAA|2005-10-03|1269|5518|424|2005|1|10|3|4|2005|424|5518|Monday|2005Q4|N|N|N|2453645|2453917|2453282|2453555|N|N|N|N|N| +2453648|AAAAAAAAAJAHFCAA|2005-10-04|1269|5519|424|2005|2|10|4|4|2005|424|5519|Tuesday|2005Q4|N|N|N|2453645|2453917|2453283|2453556|N|N|N|N|N| +2453649|AAAAAAAABJAHFCAA|2005-10-05|1269|5519|424|2005|3|10|5|4|2005|424|5519|Wednesday|2005Q4|N|N|N|2453645|2453917|2453284|2453557|N|N|N|N|N| +2453650|AAAAAAAACJAHFCAA|2005-10-06|1269|5519|424|2005|4|10|6|4|2005|424|5519|Thursday|2005Q4|N|N|N|2453645|2453917|2453285|2453558|N|N|N|N|N| +2453651|AAAAAAAADJAHFCAA|2005-10-07|1269|5519|424|2005|5|10|7|4|2005|424|5519|Friday|2005Q4|N|Y|N|2453645|2453917|2453286|2453559|N|N|N|N|N| +2453652|AAAAAAAAEJAHFCAA|2005-10-08|1269|5519|424|2005|6|10|8|4|2005|424|5519|Saturday|2005Q4|N|Y|N|2453645|2453917|2453287|2453560|N|N|N|N|N| +2453653|AAAAAAAAFJAHFCAA|2005-10-09|1269|5519|424|2005|0|10|9|4|2005|424|5519|Sunday|2005Q4|N|N|N|2453645|2453917|2453288|2453561|N|N|N|N|N| +2453654|AAAAAAAAGJAHFCAA|2005-10-10|1269|5519|424|2005|1|10|10|4|2005|424|5519|Monday|2005Q4|N|N|N|2453645|2453917|2453289|2453562|N|N|N|N|N| +2453655|AAAAAAAAHJAHFCAA|2005-10-11|1269|5520|424|2005|2|10|11|4|2005|424|5520|Tuesday|2005Q4|N|N|N|2453645|2453917|2453290|2453563|N|N|N|N|N| +2453656|AAAAAAAAIJAHFCAA|2005-10-12|1269|5520|424|2005|3|10|12|4|2005|424|5520|Wednesday|2005Q4|N|N|N|2453645|2453917|2453291|2453564|N|N|N|N|N| +2453657|AAAAAAAAJJAHFCAA|2005-10-13|1269|5520|424|2005|4|10|13|4|2005|424|5520|Thursday|2005Q4|N|N|N|2453645|2453917|2453292|2453565|N|N|N|N|N| +2453658|AAAAAAAAKJAHFCAA|2005-10-14|1269|5520|424|2005|5|10|14|4|2005|424|5520|Friday|2005Q4|N|Y|N|2453645|2453917|2453293|2453566|N|N|N|N|N| +2453659|AAAAAAAALJAHFCAA|2005-10-15|1269|5520|424|2005|6|10|15|4|2005|424|5520|Saturday|2005Q4|N|Y|N|2453645|2453917|2453294|2453567|N|N|N|N|N| +2453660|AAAAAAAAMJAHFCAA|2005-10-16|1269|5520|424|2005|0|10|16|4|2005|424|5520|Sunday|2005Q4|N|N|N|2453645|2453917|2453295|2453568|N|N|N|N|N| +2453661|AAAAAAAANJAHFCAA|2005-10-17|1269|5520|424|2005|1|10|17|4|2005|424|5520|Monday|2005Q4|N|N|N|2453645|2453917|2453296|2453569|N|N|N|N|N| +2453662|AAAAAAAAOJAHFCAA|2005-10-18|1269|5521|424|2005|2|10|18|4|2005|424|5521|Tuesday|2005Q4|N|N|N|2453645|2453917|2453297|2453570|N|N|N|N|N| +2453663|AAAAAAAAPJAHFCAA|2005-10-19|1269|5521|424|2005|3|10|19|4|2005|424|5521|Wednesday|2005Q4|N|N|N|2453645|2453917|2453298|2453571|N|N|N|N|N| +2453664|AAAAAAAAAKAHFCAA|2005-10-20|1269|5521|424|2005|4|10|20|4|2005|424|5521|Thursday|2005Q4|N|N|N|2453645|2453917|2453299|2453572|N|N|N|N|N| +2453665|AAAAAAAABKAHFCAA|2005-10-21|1269|5521|424|2005|5|10|21|4|2005|424|5521|Friday|2005Q4|N|Y|N|2453645|2453917|2453300|2453573|N|N|N|N|N| +2453666|AAAAAAAACKAHFCAA|2005-10-22|1269|5521|424|2005|6|10|22|4|2005|424|5521|Saturday|2005Q4|N|Y|N|2453645|2453917|2453301|2453574|N|N|N|N|N| +2453667|AAAAAAAADKAHFCAA|2005-10-23|1269|5521|424|2005|0|10|23|4|2005|424|5521|Sunday|2005Q4|N|N|N|2453645|2453917|2453302|2453575|N|N|N|N|N| +2453668|AAAAAAAAEKAHFCAA|2005-10-24|1269|5521|424|2005|1|10|24|4|2005|424|5521|Monday|2005Q4|N|N|N|2453645|2453917|2453303|2453576|N|N|N|N|N| +2453669|AAAAAAAAFKAHFCAA|2005-10-25|1269|5522|424|2005|2|10|25|4|2005|424|5522|Tuesday|2005Q4|N|N|N|2453645|2453917|2453304|2453577|N|N|N|N|N| +2453670|AAAAAAAAGKAHFCAA|2005-10-26|1269|5522|424|2005|3|10|26|4|2005|424|5522|Wednesday|2005Q4|N|N|N|2453645|2453917|2453305|2453578|N|N|N|N|N| +2453671|AAAAAAAAHKAHFCAA|2005-10-27|1269|5522|424|2005|4|10|27|4|2005|424|5522|Thursday|2005Q4|N|N|N|2453645|2453917|2453306|2453579|N|N|N|N|N| +2453672|AAAAAAAAIKAHFCAA|2005-10-28|1269|5522|424|2005|5|10|28|4|2005|424|5522|Friday|2005Q4|N|Y|N|2453645|2453917|2453307|2453580|N|N|N|N|N| +2453673|AAAAAAAAJKAHFCAA|2005-10-29|1269|5522|424|2005|6|10|29|4|2005|424|5522|Saturday|2005Q4|N|Y|N|2453645|2453917|2453308|2453581|N|N|N|N|N| +2453674|AAAAAAAAKKAHFCAA|2005-10-30|1269|5522|424|2005|0|10|30|4|2005|424|5522|Sunday|2005Q4|N|N|N|2453645|2453917|2453309|2453582|N|N|N|N|N| +2453675|AAAAAAAALKAHFCAA|2005-10-31|1269|5522|424|2005|1|10|31|4|2005|424|5522|Monday|2005Q4|N|N|N|2453645|2453917|2453310|2453583|N|N|N|N|N| +2453676|AAAAAAAAMKAHFCAA|2005-11-01|1270|5523|424|2005|2|11|1|4|2005|424|5523|Tuesday|2005Q4|N|N|N|2453676|2453979|2453311|2453584|N|N|N|N|N| +2453677|AAAAAAAANKAHFCAA|2005-11-02|1270|5523|424|2005|3|11|2|4|2005|424|5523|Wednesday|2005Q4|N|N|N|2453676|2453979|2453312|2453585|N|N|N|N|N| +2453678|AAAAAAAAOKAHFCAA|2005-11-03|1270|5523|424|2005|4|11|3|4|2005|424|5523|Thursday|2005Q4|N|N|N|2453676|2453979|2453313|2453586|N|N|N|N|N| +2453679|AAAAAAAAPKAHFCAA|2005-11-04|1270|5523|424|2005|5|11|4|4|2005|424|5523|Friday|2005Q4|N|Y|N|2453676|2453979|2453314|2453587|N|N|N|N|N| +2453680|AAAAAAAAALAHFCAA|2005-11-05|1270|5523|424|2005|6|11|5|4|2005|424|5523|Saturday|2005Q4|N|Y|N|2453676|2453979|2453315|2453588|N|N|N|N|N| +2453681|AAAAAAAABLAHFCAA|2005-11-06|1270|5523|424|2005|0|11|6|4|2005|424|5523|Sunday|2005Q4|N|N|N|2453676|2453979|2453316|2453589|N|N|N|N|N| +2453682|AAAAAAAACLAHFCAA|2005-11-07|1270|5523|424|2005|1|11|7|4|2005|424|5523|Monday|2005Q4|N|N|N|2453676|2453979|2453317|2453590|N|N|N|N|N| +2453683|AAAAAAAADLAHFCAA|2005-11-08|1270|5524|424|2005|2|11|8|4|2005|424|5524|Tuesday|2005Q4|N|N|N|2453676|2453979|2453318|2453591|N|N|N|N|N| +2453684|AAAAAAAAELAHFCAA|2005-11-09|1270|5524|424|2005|3|11|9|4|2005|424|5524|Wednesday|2005Q4|N|N|N|2453676|2453979|2453319|2453592|N|N|N|N|N| +2453685|AAAAAAAAFLAHFCAA|2005-11-10|1270|5524|424|2005|4|11|10|4|2005|424|5524|Thursday|2005Q4|N|N|N|2453676|2453979|2453320|2453593|N|N|N|N|N| +2453686|AAAAAAAAGLAHFCAA|2005-11-11|1270|5524|424|2005|5|11|11|4|2005|424|5524|Friday|2005Q4|N|Y|N|2453676|2453979|2453321|2453594|N|N|N|N|N| +2453687|AAAAAAAAHLAHFCAA|2005-11-12|1270|5524|424|2005|6|11|12|4|2005|424|5524|Saturday|2005Q4|N|Y|N|2453676|2453979|2453322|2453595|N|N|N|N|N| +2453688|AAAAAAAAILAHFCAA|2005-11-13|1270|5524|424|2005|0|11|13|4|2005|424|5524|Sunday|2005Q4|N|N|N|2453676|2453979|2453323|2453596|N|N|N|N|N| +2453689|AAAAAAAAJLAHFCAA|2005-11-14|1270|5524|424|2005|1|11|14|4|2005|424|5524|Monday|2005Q4|N|N|N|2453676|2453979|2453324|2453597|N|N|N|N|N| +2453690|AAAAAAAAKLAHFCAA|2005-11-15|1270|5525|424|2005|2|11|15|4|2005|424|5525|Tuesday|2005Q4|N|N|N|2453676|2453979|2453325|2453598|N|N|N|N|N| +2453691|AAAAAAAALLAHFCAA|2005-11-16|1270|5525|424|2005|3|11|16|4|2005|424|5525|Wednesday|2005Q4|N|N|N|2453676|2453979|2453326|2453599|N|N|N|N|N| +2453692|AAAAAAAAMLAHFCAA|2005-11-17|1270|5525|424|2005|4|11|17|4|2005|424|5525|Thursday|2005Q4|N|N|N|2453676|2453979|2453327|2453600|N|N|N|N|N| +2453693|AAAAAAAANLAHFCAA|2005-11-18|1270|5525|424|2005|5|11|18|4|2005|424|5525|Friday|2005Q4|N|Y|N|2453676|2453979|2453328|2453601|N|N|N|N|N| +2453694|AAAAAAAAOLAHFCAA|2005-11-19|1270|5525|424|2005|6|11|19|4|2005|424|5525|Saturday|2005Q4|N|Y|N|2453676|2453979|2453329|2453602|N|N|N|N|N| +2453695|AAAAAAAAPLAHFCAA|2005-11-20|1270|5525|424|2005|0|11|20|4|2005|424|5525|Sunday|2005Q4|N|N|N|2453676|2453979|2453330|2453603|N|N|N|N|N| +2453696|AAAAAAAAAMAHFCAA|2005-11-21|1270|5525|424|2005|1|11|21|4|2005|424|5525|Monday|2005Q4|N|N|N|2453676|2453979|2453331|2453604|N|N|N|N|N| +2453697|AAAAAAAABMAHFCAA|2005-11-22|1270|5526|424|2005|2|11|22|4|2005|424|5526|Tuesday|2005Q4|N|N|N|2453676|2453979|2453332|2453605|N|N|N|N|N| +2453698|AAAAAAAACMAHFCAA|2005-11-23|1270|5526|424|2005|3|11|23|4|2005|424|5526|Wednesday|2005Q4|N|N|N|2453676|2453979|2453333|2453606|N|N|N|N|N| +2453699|AAAAAAAADMAHFCAA|2005-11-24|1270|5526|424|2005|4|11|24|4|2005|424|5526|Thursday|2005Q4|N|N|N|2453676|2453979|2453334|2453607|N|N|N|N|N| +2453700|AAAAAAAAEMAHFCAA|2005-11-25|1270|5526|424|2005|5|11|25|4|2005|424|5526|Friday|2005Q4|N|Y|N|2453676|2453979|2453335|2453608|N|N|N|N|N| +2453701|AAAAAAAAFMAHFCAA|2005-11-26|1270|5526|424|2005|6|11|26|4|2005|424|5526|Saturday|2005Q4|N|Y|N|2453676|2453979|2453336|2453609|N|N|N|N|N| +2453702|AAAAAAAAGMAHFCAA|2005-11-27|1270|5526|424|2005|0|11|27|4|2005|424|5526|Sunday|2005Q4|N|N|N|2453676|2453979|2453337|2453610|N|N|N|N|N| +2453703|AAAAAAAAHMAHFCAA|2005-11-28|1270|5526|424|2005|1|11|28|4|2005|424|5526|Monday|2005Q4|N|N|N|2453676|2453979|2453338|2453611|N|N|N|N|N| +2453704|AAAAAAAAIMAHFCAA|2005-11-29|1270|5527|424|2005|2|11|29|4|2005|424|5527|Tuesday|2005Q4|N|N|N|2453676|2453979|2453339|2453612|N|N|N|N|N| +2453705|AAAAAAAAJMAHFCAA|2005-11-30|1270|5527|424|2005|3|11|30|4|2005|424|5527|Wednesday|2005Q4|N|N|N|2453676|2453979|2453340|2453613|N|N|N|N|N| +2453706|AAAAAAAAKMAHFCAA|2005-12-01|1271|5527|425|2005|4|12|1|4|2005|425|5527|Thursday|2005Q4|N|N|N|2453706|2454039|2453341|2453614|N|N|N|N|N| +2453707|AAAAAAAALMAHFCAA|2005-12-02|1271|5527|425|2005|5|12|2|4|2005|425|5527|Friday|2005Q4|N|Y|N|2453706|2454039|2453342|2453615|N|N|N|N|N| +2453708|AAAAAAAAMMAHFCAA|2005-12-03|1271|5527|425|2005|6|12|3|4|2005|425|5527|Saturday|2005Q4|N|Y|N|2453706|2454039|2453343|2453616|N|N|N|N|N| +2453709|AAAAAAAANMAHFCAA|2005-12-04|1271|5527|425|2005|0|12|4|4|2005|425|5527|Sunday|2005Q4|N|N|N|2453706|2454039|2453344|2453617|N|N|N|N|N| +2453710|AAAAAAAAOMAHFCAA|2005-12-05|1271|5527|425|2005|1|12|5|4|2005|425|5527|Monday|2005Q4|N|N|N|2453706|2454039|2453345|2453618|N|N|N|N|N| +2453711|AAAAAAAAPMAHFCAA|2005-12-06|1271|5528|425|2005|2|12|6|4|2005|425|5528|Tuesday|2005Q4|N|N|N|2453706|2454039|2453346|2453619|N|N|N|N|N| +2453712|AAAAAAAAANAHFCAA|2005-12-07|1271|5528|425|2005|3|12|7|4|2005|425|5528|Wednesday|2005Q4|N|N|N|2453706|2454039|2453347|2453620|N|N|N|N|N| +2453713|AAAAAAAABNAHFCAA|2005-12-08|1271|5528|425|2005|4|12|8|4|2005|425|5528|Thursday|2005Q4|N|N|N|2453706|2454039|2453348|2453621|N|N|N|N|N| +2453714|AAAAAAAACNAHFCAA|2005-12-09|1271|5528|425|2005|5|12|9|4|2005|425|5528|Friday|2005Q4|N|Y|N|2453706|2454039|2453349|2453622|N|N|N|N|N| +2453715|AAAAAAAADNAHFCAA|2005-12-10|1271|5528|425|2005|6|12|10|4|2005|425|5528|Saturday|2005Q4|N|Y|N|2453706|2454039|2453350|2453623|N|N|N|N|N| +2453716|AAAAAAAAENAHFCAA|2005-12-11|1271|5528|425|2005|0|12|11|4|2005|425|5528|Sunday|2005Q4|N|N|N|2453706|2454039|2453351|2453624|N|N|N|N|N| +2453717|AAAAAAAAFNAHFCAA|2005-12-12|1271|5528|425|2005|1|12|12|4|2005|425|5528|Monday|2005Q4|N|N|N|2453706|2454039|2453352|2453625|N|N|N|N|N| +2453718|AAAAAAAAGNAHFCAA|2005-12-13|1271|5529|425|2005|2|12|13|4|2005|425|5529|Tuesday|2005Q4|N|N|N|2453706|2454039|2453353|2453626|N|N|N|N|N| +2453719|AAAAAAAAHNAHFCAA|2005-12-14|1271|5529|425|2005|3|12|14|4|2005|425|5529|Wednesday|2005Q4|N|N|N|2453706|2454039|2453354|2453627|N|N|N|N|N| +2453720|AAAAAAAAINAHFCAA|2005-12-15|1271|5529|425|2005|4|12|15|4|2005|425|5529|Thursday|2005Q4|N|N|N|2453706|2454039|2453355|2453628|N|N|N|N|N| +2453721|AAAAAAAAJNAHFCAA|2005-12-16|1271|5529|425|2005|5|12|16|4|2005|425|5529|Friday|2005Q4|N|Y|N|2453706|2454039|2453356|2453629|N|N|N|N|N| +2453722|AAAAAAAAKNAHFCAA|2005-12-17|1271|5529|425|2005|6|12|17|4|2005|425|5529|Saturday|2005Q4|N|Y|N|2453706|2454039|2453357|2453630|N|N|N|N|N| +2453723|AAAAAAAALNAHFCAA|2005-12-18|1271|5529|425|2005|0|12|18|4|2005|425|5529|Sunday|2005Q4|N|N|N|2453706|2454039|2453358|2453631|N|N|N|N|N| +2453724|AAAAAAAAMNAHFCAA|2005-12-19|1271|5529|425|2005|1|12|19|4|2005|425|5529|Monday|2005Q4|N|N|N|2453706|2454039|2453359|2453632|N|N|N|N|N| +2453725|AAAAAAAANNAHFCAA|2005-12-20|1271|5530|425|2005|2|12|20|4|2005|425|5530|Tuesday|2005Q4|N|N|N|2453706|2454039|2453360|2453633|N|N|N|N|N| +2453726|AAAAAAAAONAHFCAA|2005-12-21|1271|5530|425|2005|3|12|21|4|2005|425|5530|Wednesday|2005Q4|N|N|N|2453706|2454039|2453361|2453634|N|N|N|N|N| +2453727|AAAAAAAAPNAHFCAA|2005-12-22|1271|5530|425|2005|4|12|22|4|2005|425|5530|Thursday|2005Q4|N|N|N|2453706|2454039|2453362|2453635|N|N|N|N|N| +2453728|AAAAAAAAAOAHFCAA|2005-12-23|1271|5530|425|2005|5|12|23|4|2005|425|5530|Friday|2005Q4|N|Y|N|2453706|2454039|2453363|2453636|N|N|N|N|N| +2453729|AAAAAAAABOAHFCAA|2005-12-24|1271|5530|425|2005|6|12|24|4|2005|425|5530|Saturday|2005Q4|N|Y|N|2453706|2454039|2453364|2453637|N|N|N|N|N| +2453730|AAAAAAAACOAHFCAA|2005-12-25|1271|5530|425|2005|0|12|25|4|2005|425|5530|Sunday|2005Q4|N|N|N|2453706|2454039|2453365|2453638|N|N|N|N|N| +2453731|AAAAAAAADOAHFCAA|2005-12-26|1271|5530|425|2005|1|12|26|4|2005|425|5530|Monday|2005Q4|Y|N|N|2453706|2454039|2453366|2453639|N|N|N|N|N| +2453732|AAAAAAAAEOAHFCAA|2005-12-27|1271|5531|425|2005|2|12|27|4|2005|425|5531|Tuesday|2005Q4|N|N|Y|2453706|2454039|2453367|2453640|N|N|N|N|N| +2453733|AAAAAAAAFOAHFCAA|2005-12-28|1271|5531|425|2005|3|12|28|4|2005|425|5531|Wednesday|2005Q4|N|N|N|2453706|2454039|2453368|2453641|N|N|N|N|N| +2453734|AAAAAAAAGOAHFCAA|2005-12-29|1271|5531|425|2005|4|12|29|4|2005|425|5531|Thursday|2005Q4|N|N|N|2453706|2454039|2453369|2453642|N|N|N|N|N| +2453735|AAAAAAAAHOAHFCAA|2005-12-30|1271|5531|425|2005|5|12|30|4|2005|425|5531|Friday|2005Q4|N|Y|N|2453706|2454039|2453370|2453643|N|N|N|N|N| +2453736|AAAAAAAAIOAHFCAA|2005-12-31|1271|5531|425|2005|6|12|31|4|2005|425|5531|Saturday|2005Q4|N|Y|N|2453706|2454039|2453371|2453644|N|N|N|N|N| +2453737|AAAAAAAAJOAHFCAA|2006-01-01|1272|5531|425|2006|0|1|1|1|2006|425|5531|Sunday|2006Q1|Y|N|N|2453737|2453736|2453372|2453645|N|N|N|N|N| +2453738|AAAAAAAAKOAHFCAA|2006-01-02|1272|5531|425|2006|1|1|2|1|2006|425|5531|Monday|2006Q1|N|N|Y|2453737|2453736|2453373|2453646|N|N|N|N|N| +2453739|AAAAAAAALOAHFCAA|2006-01-03|1272|5532|425|2006|2|1|3|1|2006|425|5532|Tuesday|2006Q1|N|N|N|2453737|2453736|2453374|2453647|N|N|N|N|N| +2453740|AAAAAAAAMOAHFCAA|2006-01-04|1272|5532|425|2006|3|1|4|1|2006|425|5532|Wednesday|2006Q1|N|N|N|2453737|2453736|2453375|2453648|N|N|N|N|N| +2453741|AAAAAAAANOAHFCAA|2006-01-05|1272|5532|425|2006|4|1|5|1|2006|425|5532|Thursday|2006Q1|N|N|N|2453737|2453736|2453376|2453649|N|N|N|N|N| +2453742|AAAAAAAAOOAHFCAA|2006-01-06|1272|5532|425|2006|5|1|6|1|2006|425|5532|Friday|2006Q1|N|Y|N|2453737|2453736|2453377|2453650|N|N|N|N|N| +2453743|AAAAAAAAPOAHFCAA|2006-01-07|1272|5532|425|2006|6|1|7|1|2006|425|5532|Saturday|2006Q1|N|Y|N|2453737|2453736|2453378|2453651|N|N|N|N|N| +2453744|AAAAAAAAAPAHFCAA|2006-01-08|1272|5532|425|2006|0|1|8|1|2006|425|5532|Sunday|2006Q1|N|N|N|2453737|2453736|2453379|2453652|N|N|N|N|N| +2453745|AAAAAAAABPAHFCAA|2006-01-09|1272|5532|425|2006|1|1|9|1|2006|425|5532|Monday|2006Q1|N|N|N|2453737|2453736|2453380|2453653|N|N|N|N|N| +2453746|AAAAAAAACPAHFCAA|2006-01-10|1272|5533|425|2006|2|1|10|1|2006|425|5533|Tuesday|2006Q1|N|N|N|2453737|2453736|2453381|2453654|N|N|N|N|N| +2453747|AAAAAAAADPAHFCAA|2006-01-11|1272|5533|425|2006|3|1|11|1|2006|425|5533|Wednesday|2006Q1|N|N|N|2453737|2453736|2453382|2453655|N|N|N|N|N| +2453748|AAAAAAAAEPAHFCAA|2006-01-12|1272|5533|425|2006|4|1|12|1|2006|425|5533|Thursday|2006Q1|N|N|N|2453737|2453736|2453383|2453656|N|N|N|N|N| +2453749|AAAAAAAAFPAHFCAA|2006-01-13|1272|5533|425|2006|5|1|13|1|2006|425|5533|Friday|2006Q1|N|Y|N|2453737|2453736|2453384|2453657|N|N|N|N|N| +2453750|AAAAAAAAGPAHFCAA|2006-01-14|1272|5533|425|2006|6|1|14|1|2006|425|5533|Saturday|2006Q1|N|Y|N|2453737|2453736|2453385|2453658|N|N|N|N|N| +2453751|AAAAAAAAHPAHFCAA|2006-01-15|1272|5533|425|2006|0|1|15|1|2006|425|5533|Sunday|2006Q1|N|N|N|2453737|2453736|2453386|2453659|N|N|N|N|N| +2453752|AAAAAAAAIPAHFCAA|2006-01-16|1272|5533|425|2006|1|1|16|1|2006|425|5533|Monday|2006Q1|N|N|N|2453737|2453736|2453387|2453660|N|N|N|N|N| +2453753|AAAAAAAAJPAHFCAA|2006-01-17|1272|5534|425|2006|2|1|17|1|2006|425|5534|Tuesday|2006Q1|N|N|N|2453737|2453736|2453388|2453661|N|N|N|N|N| +2453754|AAAAAAAAKPAHFCAA|2006-01-18|1272|5534|425|2006|3|1|18|1|2006|425|5534|Wednesday|2006Q1|N|N|N|2453737|2453736|2453389|2453662|N|N|N|N|N| +2453755|AAAAAAAALPAHFCAA|2006-01-19|1272|5534|425|2006|4|1|19|1|2006|425|5534|Thursday|2006Q1|N|N|N|2453737|2453736|2453390|2453663|N|N|N|N|N| +2453756|AAAAAAAAMPAHFCAA|2006-01-20|1272|5534|425|2006|5|1|20|1|2006|425|5534|Friday|2006Q1|N|Y|N|2453737|2453736|2453391|2453664|N|N|N|N|N| +2453757|AAAAAAAANPAHFCAA|2006-01-21|1272|5534|425|2006|6|1|21|1|2006|425|5534|Saturday|2006Q1|N|Y|N|2453737|2453736|2453392|2453665|N|N|N|N|N| +2453758|AAAAAAAAOPAHFCAA|2006-01-22|1272|5534|425|2006|0|1|22|1|2006|425|5534|Sunday|2006Q1|N|N|N|2453737|2453736|2453393|2453666|N|N|N|N|N| +2453759|AAAAAAAAPPAHFCAA|2006-01-23|1272|5534|425|2006|1|1|23|1|2006|425|5534|Monday|2006Q1|N|N|N|2453737|2453736|2453394|2453667|N|N|N|N|N| +2453760|AAAAAAAAAABHFCAA|2006-01-24|1272|5535|425|2006|2|1|24|1|2006|425|5535|Tuesday|2006Q1|N|N|N|2453737|2453736|2453395|2453668|N|N|N|N|N| +2453761|AAAAAAAABABHFCAA|2006-01-25|1272|5535|425|2006|3|1|25|1|2006|425|5535|Wednesday|2006Q1|N|N|N|2453737|2453736|2453396|2453669|N|N|N|N|N| +2453762|AAAAAAAACABHFCAA|2006-01-26|1272|5535|425|2006|4|1|26|1|2006|425|5535|Thursday|2006Q1|N|N|N|2453737|2453736|2453397|2453670|N|N|N|N|N| +2453763|AAAAAAAADABHFCAA|2006-01-27|1272|5535|425|2006|5|1|27|1|2006|425|5535|Friday|2006Q1|N|Y|N|2453737|2453736|2453398|2453671|N|N|N|N|N| +2453764|AAAAAAAAEABHFCAA|2006-01-28|1272|5535|425|2006|6|1|28|1|2006|425|5535|Saturday|2006Q1|N|Y|N|2453737|2453736|2453399|2453672|N|N|N|N|N| +2453765|AAAAAAAAFABHFCAA|2006-01-29|1272|5535|425|2006|0|1|29|1|2006|425|5535|Sunday|2006Q1|N|N|N|2453737|2453736|2453400|2453673|N|N|N|N|N| +2453766|AAAAAAAAGABHFCAA|2006-01-30|1272|5535|425|2006|1|1|30|1|2006|425|5535|Monday|2006Q1|N|N|N|2453737|2453736|2453401|2453674|N|N|N|N|N| +2453767|AAAAAAAAHABHFCAA|2006-01-31|1272|5536|425|2006|2|1|31|1|2006|425|5536|Tuesday|2006Q1|N|N|N|2453737|2453736|2453402|2453675|N|N|N|N|N| +2453768|AAAAAAAAIABHFCAA|2006-02-01|1273|5536|425|2006|3|2|1|1|2006|425|5536|Wednesday|2006Q1|N|N|N|2453768|2453798|2453403|2453676|N|N|N|N|N| +2453769|AAAAAAAAJABHFCAA|2006-02-02|1273|5536|425|2006|4|2|2|1|2006|425|5536|Thursday|2006Q1|N|N|N|2453768|2453798|2453404|2453677|N|N|N|N|N| +2453770|AAAAAAAAKABHFCAA|2006-02-03|1273|5536|425|2006|5|2|3|1|2006|425|5536|Friday|2006Q1|N|Y|N|2453768|2453798|2453405|2453678|N|N|N|N|N| +2453771|AAAAAAAALABHFCAA|2006-02-04|1273|5536|425|2006|6|2|4|1|2006|425|5536|Saturday|2006Q1|N|Y|N|2453768|2453798|2453406|2453679|N|N|N|N|N| +2453772|AAAAAAAAMABHFCAA|2006-02-05|1273|5536|425|2006|0|2|5|1|2006|425|5536|Sunday|2006Q1|N|N|N|2453768|2453798|2453407|2453680|N|N|N|N|N| +2453773|AAAAAAAANABHFCAA|2006-02-06|1273|5536|425|2006|1|2|6|1|2006|425|5536|Monday|2006Q1|N|N|N|2453768|2453798|2453408|2453681|N|N|N|N|N| +2453774|AAAAAAAAOABHFCAA|2006-02-07|1273|5537|425|2006|2|2|7|1|2006|425|5537|Tuesday|2006Q1|N|N|N|2453768|2453798|2453409|2453682|N|N|N|N|N| +2453775|AAAAAAAAPABHFCAA|2006-02-08|1273|5537|425|2006|3|2|8|1|2006|425|5537|Wednesday|2006Q1|N|N|N|2453768|2453798|2453410|2453683|N|N|N|N|N| +2453776|AAAAAAAAABBHFCAA|2006-02-09|1273|5537|425|2006|4|2|9|1|2006|425|5537|Thursday|2006Q1|N|N|N|2453768|2453798|2453411|2453684|N|N|N|N|N| +2453777|AAAAAAAABBBHFCAA|2006-02-10|1273|5537|425|2006|5|2|10|1|2006|425|5537|Friday|2006Q1|N|Y|N|2453768|2453798|2453412|2453685|N|N|N|N|N| +2453778|AAAAAAAACBBHFCAA|2006-02-11|1273|5537|425|2006|6|2|11|1|2006|425|5537|Saturday|2006Q1|N|Y|N|2453768|2453798|2453413|2453686|N|N|N|N|N| +2453779|AAAAAAAADBBHFCAA|2006-02-12|1273|5537|425|2006|0|2|12|1|2006|425|5537|Sunday|2006Q1|N|N|N|2453768|2453798|2453414|2453687|N|N|N|N|N| +2453780|AAAAAAAAEBBHFCAA|2006-02-13|1273|5537|425|2006|1|2|13|1|2006|425|5537|Monday|2006Q1|N|N|N|2453768|2453798|2453415|2453688|N|N|N|N|N| +2453781|AAAAAAAAFBBHFCAA|2006-02-14|1273|5538|425|2006|2|2|14|1|2006|425|5538|Tuesday|2006Q1|N|N|N|2453768|2453798|2453416|2453689|N|N|N|N|N| +2453782|AAAAAAAAGBBHFCAA|2006-02-15|1273|5538|425|2006|3|2|15|1|2006|425|5538|Wednesday|2006Q1|N|N|N|2453768|2453798|2453417|2453690|N|N|N|N|N| +2453783|AAAAAAAAHBBHFCAA|2006-02-16|1273|5538|425|2006|4|2|16|1|2006|425|5538|Thursday|2006Q1|N|N|N|2453768|2453798|2453418|2453691|N|N|N|N|N| +2453784|AAAAAAAAIBBHFCAA|2006-02-17|1273|5538|425|2006|5|2|17|1|2006|425|5538|Friday|2006Q1|N|Y|N|2453768|2453798|2453419|2453692|N|N|N|N|N| +2453785|AAAAAAAAJBBHFCAA|2006-02-18|1273|5538|425|2006|6|2|18|1|2006|425|5538|Saturday|2006Q1|N|Y|N|2453768|2453798|2453420|2453693|N|N|N|N|N| +2453786|AAAAAAAAKBBHFCAA|2006-02-19|1273|5538|425|2006|0|2|19|1|2006|425|5538|Sunday|2006Q1|N|N|N|2453768|2453798|2453421|2453694|N|N|N|N|N| +2453787|AAAAAAAALBBHFCAA|2006-02-20|1273|5538|425|2006|1|2|20|1|2006|425|5538|Monday|2006Q1|N|N|N|2453768|2453798|2453422|2453695|N|N|N|N|N| +2453788|AAAAAAAAMBBHFCAA|2006-02-21|1273|5539|425|2006|2|2|21|1|2006|425|5539|Tuesday|2006Q1|N|N|N|2453768|2453798|2453423|2453696|N|N|N|N|N| +2453789|AAAAAAAANBBHFCAA|2006-02-22|1273|5539|425|2006|3|2|22|1|2006|425|5539|Wednesday|2006Q1|N|N|N|2453768|2453798|2453424|2453697|N|N|N|N|N| +2453790|AAAAAAAAOBBHFCAA|2006-02-23|1273|5539|425|2006|4|2|23|1|2006|425|5539|Thursday|2006Q1|N|N|N|2453768|2453798|2453425|2453698|N|N|N|N|N| +2453791|AAAAAAAAPBBHFCAA|2006-02-24|1273|5539|425|2006|5|2|24|1|2006|425|5539|Friday|2006Q1|N|Y|N|2453768|2453798|2453426|2453699|N|N|N|N|N| +2453792|AAAAAAAAACBHFCAA|2006-02-25|1273|5539|425|2006|6|2|25|1|2006|425|5539|Saturday|2006Q1|N|Y|N|2453768|2453798|2453427|2453700|N|N|N|N|N| +2453793|AAAAAAAABCBHFCAA|2006-02-26|1273|5539|425|2006|0|2|26|1|2006|425|5539|Sunday|2006Q1|N|N|N|2453768|2453798|2453428|2453701|N|N|N|N|N| +2453794|AAAAAAAACCBHFCAA|2006-02-27|1273|5539|425|2006|1|2|27|1|2006|425|5539|Monday|2006Q1|N|N|N|2453768|2453798|2453429|2453702|N|N|N|N|N| +2453795|AAAAAAAADCBHFCAA|2006-02-28|1273|5540|425|2006|2|2|28|1|2006|425|5540|Tuesday|2006Q1|N|N|N|2453768|2453798|2453430|2453703|N|N|N|N|N| +2453796|AAAAAAAAECBHFCAA|2006-03-01|1274|5540|426|2006|3|3|1|1|2006|426|5540|Wednesday|2006Q1|N|N|N|2453796|2453854|2453431|2453704|N|N|N|N|N| +2453797|AAAAAAAAFCBHFCAA|2006-03-02|1274|5540|426|2006|4|3|2|1|2006|426|5540|Thursday|2006Q1|N|N|N|2453796|2453854|2453432|2453705|N|N|N|N|N| +2453798|AAAAAAAAGCBHFCAA|2006-03-03|1274|5540|426|2006|5|3|3|1|2006|426|5540|Friday|2006Q1|N|Y|N|2453796|2453854|2453433|2453706|N|N|N|N|N| +2453799|AAAAAAAAHCBHFCAA|2006-03-04|1274|5540|426|2006|6|3|4|1|2006|426|5540|Saturday|2006Q1|N|Y|N|2453796|2453854|2453434|2453707|N|N|N|N|N| +2453800|AAAAAAAAICBHFCAA|2006-03-05|1274|5540|426|2006|0|3|5|1|2006|426|5540|Sunday|2006Q1|N|N|N|2453796|2453854|2453435|2453708|N|N|N|N|N| +2453801|AAAAAAAAJCBHFCAA|2006-03-06|1274|5540|426|2006|1|3|6|1|2006|426|5540|Monday|2006Q1|N|N|N|2453796|2453854|2453436|2453709|N|N|N|N|N| +2453802|AAAAAAAAKCBHFCAA|2006-03-07|1274|5541|426|2006|2|3|7|1|2006|426|5541|Tuesday|2006Q1|N|N|N|2453796|2453854|2453437|2453710|N|N|N|N|N| +2453803|AAAAAAAALCBHFCAA|2006-03-08|1274|5541|426|2006|3|3|8|1|2006|426|5541|Wednesday|2006Q1|N|N|N|2453796|2453854|2453438|2453711|N|N|N|N|N| +2453804|AAAAAAAAMCBHFCAA|2006-03-09|1274|5541|426|2006|4|3|9|1|2006|426|5541|Thursday|2006Q1|N|N|N|2453796|2453854|2453439|2453712|N|N|N|N|N| +2453805|AAAAAAAANCBHFCAA|2006-03-10|1274|5541|426|2006|5|3|10|1|2006|426|5541|Friday|2006Q1|N|Y|N|2453796|2453854|2453440|2453713|N|N|N|N|N| +2453806|AAAAAAAAOCBHFCAA|2006-03-11|1274|5541|426|2006|6|3|11|1|2006|426|5541|Saturday|2006Q1|N|Y|N|2453796|2453854|2453441|2453714|N|N|N|N|N| +2453807|AAAAAAAAPCBHFCAA|2006-03-12|1274|5541|426|2006|0|3|12|1|2006|426|5541|Sunday|2006Q1|N|N|N|2453796|2453854|2453442|2453715|N|N|N|N|N| +2453808|AAAAAAAAADBHFCAA|2006-03-13|1274|5541|426|2006|1|3|13|1|2006|426|5541|Monday|2006Q1|N|N|N|2453796|2453854|2453443|2453716|N|N|N|N|N| +2453809|AAAAAAAABDBHFCAA|2006-03-14|1274|5542|426|2006|2|3|14|1|2006|426|5542|Tuesday|2006Q1|N|N|N|2453796|2453854|2453444|2453717|N|N|N|N|N| +2453810|AAAAAAAACDBHFCAA|2006-03-15|1274|5542|426|2006|3|3|15|1|2006|426|5542|Wednesday|2006Q1|N|N|N|2453796|2453854|2453445|2453718|N|N|N|N|N| +2453811|AAAAAAAADDBHFCAA|2006-03-16|1274|5542|426|2006|4|3|16|1|2006|426|5542|Thursday|2006Q1|N|N|N|2453796|2453854|2453446|2453719|N|N|N|N|N| +2453812|AAAAAAAAEDBHFCAA|2006-03-17|1274|5542|426|2006|5|3|17|1|2006|426|5542|Friday|2006Q1|N|Y|N|2453796|2453854|2453447|2453720|N|N|N|N|N| +2453813|AAAAAAAAFDBHFCAA|2006-03-18|1274|5542|426|2006|6|3|18|1|2006|426|5542|Saturday|2006Q1|N|Y|N|2453796|2453854|2453448|2453721|N|N|N|N|N| +2453814|AAAAAAAAGDBHFCAA|2006-03-19|1274|5542|426|2006|0|3|19|1|2006|426|5542|Sunday|2006Q1|N|N|N|2453796|2453854|2453449|2453722|N|N|N|N|N| +2453815|AAAAAAAAHDBHFCAA|2006-03-20|1274|5542|426|2006|1|3|20|1|2006|426|5542|Monday|2006Q1|N|N|N|2453796|2453854|2453450|2453723|N|N|N|N|N| +2453816|AAAAAAAAIDBHFCAA|2006-03-21|1274|5543|426|2006|2|3|21|1|2006|426|5543|Tuesday|2006Q1|N|N|N|2453796|2453854|2453451|2453724|N|N|N|N|N| +2453817|AAAAAAAAJDBHFCAA|2006-03-22|1274|5543|426|2006|3|3|22|1|2006|426|5543|Wednesday|2006Q1|N|N|N|2453796|2453854|2453452|2453725|N|N|N|N|N| +2453818|AAAAAAAAKDBHFCAA|2006-03-23|1274|5543|426|2006|4|3|23|1|2006|426|5543|Thursday|2006Q1|N|N|N|2453796|2453854|2453453|2453726|N|N|N|N|N| +2453819|AAAAAAAALDBHFCAA|2006-03-24|1274|5543|426|2006|5|3|24|1|2006|426|5543|Friday|2006Q1|N|Y|N|2453796|2453854|2453454|2453727|N|N|N|N|N| +2453820|AAAAAAAAMDBHFCAA|2006-03-25|1274|5543|426|2006|6|3|25|1|2006|426|5543|Saturday|2006Q1|N|Y|N|2453796|2453854|2453455|2453728|N|N|N|N|N| +2453821|AAAAAAAANDBHFCAA|2006-03-26|1274|5543|426|2006|0|3|26|1|2006|426|5543|Sunday|2006Q1|N|N|N|2453796|2453854|2453456|2453729|N|N|N|N|N| +2453822|AAAAAAAAODBHFCAA|2006-03-27|1274|5543|426|2006|1|3|27|1|2006|426|5543|Monday|2006Q1|N|N|N|2453796|2453854|2453457|2453730|N|N|N|N|N| +2453823|AAAAAAAAPDBHFCAA|2006-03-28|1274|5544|426|2006|2|3|28|1|2006|426|5544|Tuesday|2006Q1|N|N|N|2453796|2453854|2453458|2453731|N|N|N|N|N| +2453824|AAAAAAAAAEBHFCAA|2006-03-29|1274|5544|426|2006|3|3|29|1|2006|426|5544|Wednesday|2006Q1|N|N|N|2453796|2453854|2453459|2453732|N|N|N|N|N| +2453825|AAAAAAAABEBHFCAA|2006-03-30|1274|5544|426|2006|4|3|30|1|2006|426|5544|Thursday|2006Q1|N|N|N|2453796|2453854|2453460|2453733|N|N|N|N|N| +2453826|AAAAAAAACEBHFCAA|2006-03-31|1274|5544|426|2006|5|3|31|1|2006|426|5544|Friday|2006Q1|N|Y|N|2453796|2453854|2453461|2453734|N|N|N|N|N| +2453827|AAAAAAAADEBHFCAA|2006-04-01|1275|5544|426|2006|6|4|1|1|2006|426|5544|Saturday|2006Q1|N|Y|N|2453827|2453916|2453462|2453737|N|N|N|N|N| +2453828|AAAAAAAAEEBHFCAA|2006-04-02|1275|5544|426|2006|0|4|2|2|2006|426|5544|Sunday|2006Q2|N|N|N|2453827|2453916|2453463|2453738|N|N|N|N|N| +2453829|AAAAAAAAFEBHFCAA|2006-04-03|1275|5544|426|2006|1|4|3|2|2006|426|5544|Monday|2006Q2|N|N|N|2453827|2453916|2453464|2453739|N|N|N|N|N| +2453830|AAAAAAAAGEBHFCAA|2006-04-04|1275|5545|426|2006|2|4|4|2|2006|426|5545|Tuesday|2006Q2|N|N|N|2453827|2453916|2453465|2453740|N|N|N|N|N| +2453831|AAAAAAAAHEBHFCAA|2006-04-05|1275|5545|426|2006|3|4|5|2|2006|426|5545|Wednesday|2006Q2|N|N|N|2453827|2453916|2453466|2453741|N|N|N|N|N| +2453832|AAAAAAAAIEBHFCAA|2006-04-06|1275|5545|426|2006|4|4|6|2|2006|426|5545|Thursday|2006Q2|N|N|N|2453827|2453916|2453467|2453742|N|N|N|N|N| +2453833|AAAAAAAAJEBHFCAA|2006-04-07|1275|5545|426|2006|5|4|7|2|2006|426|5545|Friday|2006Q2|N|Y|N|2453827|2453916|2453468|2453743|N|N|N|N|N| +2453834|AAAAAAAAKEBHFCAA|2006-04-08|1275|5545|426|2006|6|4|8|2|2006|426|5545|Saturday|2006Q2|N|Y|N|2453827|2453916|2453469|2453744|N|N|N|N|N| +2453835|AAAAAAAALEBHFCAA|2006-04-09|1275|5545|426|2006|0|4|9|2|2006|426|5545|Sunday|2006Q2|N|N|N|2453827|2453916|2453470|2453745|N|N|N|N|N| +2453836|AAAAAAAAMEBHFCAA|2006-04-10|1275|5545|426|2006|1|4|10|2|2006|426|5545|Monday|2006Q2|N|N|N|2453827|2453916|2453471|2453746|N|N|N|N|N| +2453837|AAAAAAAANEBHFCAA|2006-04-11|1275|5546|426|2006|2|4|11|2|2006|426|5546|Tuesday|2006Q2|N|N|N|2453827|2453916|2453472|2453747|N|N|N|N|N| +2453838|AAAAAAAAOEBHFCAA|2006-04-12|1275|5546|426|2006|3|4|12|2|2006|426|5546|Wednesday|2006Q2|N|N|N|2453827|2453916|2453473|2453748|N|N|N|N|N| +2453839|AAAAAAAAPEBHFCAA|2006-04-13|1275|5546|426|2006|4|4|13|2|2006|426|5546|Thursday|2006Q2|N|N|N|2453827|2453916|2453474|2453749|N|N|N|N|N| +2453840|AAAAAAAAAFBHFCAA|2006-04-14|1275|5546|426|2006|5|4|14|2|2006|426|5546|Friday|2006Q2|N|Y|N|2453827|2453916|2453475|2453750|N|N|N|N|N| +2453841|AAAAAAAABFBHFCAA|2006-04-15|1275|5546|426|2006|6|4|15|2|2006|426|5546|Saturday|2006Q2|N|Y|N|2453827|2453916|2453476|2453751|N|N|N|N|N| +2453842|AAAAAAAACFBHFCAA|2006-04-16|1275|5546|426|2006|0|4|16|2|2006|426|5546|Sunday|2006Q2|N|N|N|2453827|2453916|2453477|2453752|N|N|N|N|N| +2453843|AAAAAAAADFBHFCAA|2006-04-17|1275|5546|426|2006|1|4|17|2|2006|426|5546|Monday|2006Q2|N|N|N|2453827|2453916|2453478|2453753|N|N|N|N|N| +2453844|AAAAAAAAEFBHFCAA|2006-04-18|1275|5547|426|2006|2|4|18|2|2006|426|5547|Tuesday|2006Q2|N|N|N|2453827|2453916|2453479|2453754|N|N|N|N|N| +2453845|AAAAAAAAFFBHFCAA|2006-04-19|1275|5547|426|2006|3|4|19|2|2006|426|5547|Wednesday|2006Q2|N|N|N|2453827|2453916|2453480|2453755|N|N|N|N|N| +2453846|AAAAAAAAGFBHFCAA|2006-04-20|1275|5547|426|2006|4|4|20|2|2006|426|5547|Thursday|2006Q2|N|N|N|2453827|2453916|2453481|2453756|N|N|N|N|N| +2453847|AAAAAAAAHFBHFCAA|2006-04-21|1275|5547|426|2006|5|4|21|2|2006|426|5547|Friday|2006Q2|N|Y|N|2453827|2453916|2453482|2453757|N|N|N|N|N| +2453848|AAAAAAAAIFBHFCAA|2006-04-22|1275|5547|426|2006|6|4|22|2|2006|426|5547|Saturday|2006Q2|N|Y|N|2453827|2453916|2453483|2453758|N|N|N|N|N| +2453849|AAAAAAAAJFBHFCAA|2006-04-23|1275|5547|426|2006|0|4|23|2|2006|426|5547|Sunday|2006Q2|N|N|N|2453827|2453916|2453484|2453759|N|N|N|N|N| +2453850|AAAAAAAAKFBHFCAA|2006-04-24|1275|5547|426|2006|1|4|24|2|2006|426|5547|Monday|2006Q2|N|N|N|2453827|2453916|2453485|2453760|N|N|N|N|N| +2453851|AAAAAAAALFBHFCAA|2006-04-25|1275|5548|426|2006|2|4|25|2|2006|426|5548|Tuesday|2006Q2|N|N|N|2453827|2453916|2453486|2453761|N|N|N|N|N| +2453852|AAAAAAAAMFBHFCAA|2006-04-26|1275|5548|426|2006|3|4|26|2|2006|426|5548|Wednesday|2006Q2|N|N|N|2453827|2453916|2453487|2453762|N|N|N|N|N| +2453853|AAAAAAAANFBHFCAA|2006-04-27|1275|5548|426|2006|4|4|27|2|2006|426|5548|Thursday|2006Q2|N|N|N|2453827|2453916|2453488|2453763|N|N|N|N|N| +2453854|AAAAAAAAOFBHFCAA|2006-04-28|1275|5548|426|2006|5|4|28|2|2006|426|5548|Friday|2006Q2|N|Y|N|2453827|2453916|2453489|2453764|N|N|N|N|N| +2453855|AAAAAAAAPFBHFCAA|2006-04-29|1275|5548|426|2006|6|4|29|2|2006|426|5548|Saturday|2006Q2|N|Y|N|2453827|2453916|2453490|2453765|N|N|N|N|N| +2453856|AAAAAAAAAGBHFCAA|2006-04-30|1275|5548|426|2006|0|4|30|2|2006|426|5548|Sunday|2006Q2|N|N|N|2453827|2453916|2453491|2453766|N|N|N|N|N| +2453857|AAAAAAAABGBHFCAA|2006-05-01|1276|5548|426|2006|1|5|1|2|2006|426|5548|Monday|2006Q2|N|N|N|2453857|2453976|2453492|2453767|N|N|N|N|N| +2453858|AAAAAAAACGBHFCAA|2006-05-02|1276|5549|426|2006|2|5|2|2|2006|426|5549|Tuesday|2006Q2|N|N|N|2453857|2453976|2453493|2453768|N|N|N|N|N| +2453859|AAAAAAAADGBHFCAA|2006-05-03|1276|5549|426|2006|3|5|3|2|2006|426|5549|Wednesday|2006Q2|N|N|N|2453857|2453976|2453494|2453769|N|N|N|N|N| +2453860|AAAAAAAAEGBHFCAA|2006-05-04|1276|5549|426|2006|4|5|4|2|2006|426|5549|Thursday|2006Q2|N|N|N|2453857|2453976|2453495|2453770|N|N|N|N|N| +2453861|AAAAAAAAFGBHFCAA|2006-05-05|1276|5549|426|2006|5|5|5|2|2006|426|5549|Friday|2006Q2|N|Y|N|2453857|2453976|2453496|2453771|N|N|N|N|N| +2453862|AAAAAAAAGGBHFCAA|2006-05-06|1276|5549|426|2006|6|5|6|2|2006|426|5549|Saturday|2006Q2|N|Y|N|2453857|2453976|2453497|2453772|N|N|N|N|N| +2453863|AAAAAAAAHGBHFCAA|2006-05-07|1276|5549|426|2006|0|5|7|2|2006|426|5549|Sunday|2006Q2|N|N|N|2453857|2453976|2453498|2453773|N|N|N|N|N| +2453864|AAAAAAAAIGBHFCAA|2006-05-08|1276|5549|426|2006|1|5|8|2|2006|426|5549|Monday|2006Q2|N|N|N|2453857|2453976|2453499|2453774|N|N|N|N|N| +2453865|AAAAAAAAJGBHFCAA|2006-05-09|1276|5550|426|2006|2|5|9|2|2006|426|5550|Tuesday|2006Q2|N|N|N|2453857|2453976|2453500|2453775|N|N|N|N|N| +2453866|AAAAAAAAKGBHFCAA|2006-05-10|1276|5550|426|2006|3|5|10|2|2006|426|5550|Wednesday|2006Q2|N|N|N|2453857|2453976|2453501|2453776|N|N|N|N|N| +2453867|AAAAAAAALGBHFCAA|2006-05-11|1276|5550|426|2006|4|5|11|2|2006|426|5550|Thursday|2006Q2|N|N|N|2453857|2453976|2453502|2453777|N|N|N|N|N| +2453868|AAAAAAAAMGBHFCAA|2006-05-12|1276|5550|426|2006|5|5|12|2|2006|426|5550|Friday|2006Q2|N|Y|N|2453857|2453976|2453503|2453778|N|N|N|N|N| +2453869|AAAAAAAANGBHFCAA|2006-05-13|1276|5550|426|2006|6|5|13|2|2006|426|5550|Saturday|2006Q2|N|Y|N|2453857|2453976|2453504|2453779|N|N|N|N|N| +2453870|AAAAAAAAOGBHFCAA|2006-05-14|1276|5550|426|2006|0|5|14|2|2006|426|5550|Sunday|2006Q2|N|N|N|2453857|2453976|2453505|2453780|N|N|N|N|N| +2453871|AAAAAAAAPGBHFCAA|2006-05-15|1276|5550|426|2006|1|5|15|2|2006|426|5550|Monday|2006Q2|N|N|N|2453857|2453976|2453506|2453781|N|N|N|N|N| +2453872|AAAAAAAAAHBHFCAA|2006-05-16|1276|5551|426|2006|2|5|16|2|2006|426|5551|Tuesday|2006Q2|N|N|N|2453857|2453976|2453507|2453782|N|N|N|N|N| +2453873|AAAAAAAABHBHFCAA|2006-05-17|1276|5551|426|2006|3|5|17|2|2006|426|5551|Wednesday|2006Q2|N|N|N|2453857|2453976|2453508|2453783|N|N|N|N|N| +2453874|AAAAAAAACHBHFCAA|2006-05-18|1276|5551|426|2006|4|5|18|2|2006|426|5551|Thursday|2006Q2|N|N|N|2453857|2453976|2453509|2453784|N|N|N|N|N| +2453875|AAAAAAAADHBHFCAA|2006-05-19|1276|5551|426|2006|5|5|19|2|2006|426|5551|Friday|2006Q2|N|Y|N|2453857|2453976|2453510|2453785|N|N|N|N|N| +2453876|AAAAAAAAEHBHFCAA|2006-05-20|1276|5551|426|2006|6|5|20|2|2006|426|5551|Saturday|2006Q2|N|Y|N|2453857|2453976|2453511|2453786|N|N|N|N|N| +2453877|AAAAAAAAFHBHFCAA|2006-05-21|1276|5551|426|2006|0|5|21|2|2006|426|5551|Sunday|2006Q2|N|N|N|2453857|2453976|2453512|2453787|N|N|N|N|N| +2453878|AAAAAAAAGHBHFCAA|2006-05-22|1276|5551|426|2006|1|5|22|2|2006|426|5551|Monday|2006Q2|N|N|N|2453857|2453976|2453513|2453788|N|N|N|N|N| +2453879|AAAAAAAAHHBHFCAA|2006-05-23|1276|5552|426|2006|2|5|23|2|2006|426|5552|Tuesday|2006Q2|N|N|N|2453857|2453976|2453514|2453789|N|N|N|N|N| +2453880|AAAAAAAAIHBHFCAA|2006-05-24|1276|5552|426|2006|3|5|24|2|2006|426|5552|Wednesday|2006Q2|N|N|N|2453857|2453976|2453515|2453790|N|N|N|N|N| +2453881|AAAAAAAAJHBHFCAA|2006-05-25|1276|5552|426|2006|4|5|25|2|2006|426|5552|Thursday|2006Q2|N|N|N|2453857|2453976|2453516|2453791|N|N|N|N|N| +2453882|AAAAAAAAKHBHFCAA|2006-05-26|1276|5552|426|2006|5|5|26|2|2006|426|5552|Friday|2006Q2|N|Y|N|2453857|2453976|2453517|2453792|N|N|N|N|N| +2453883|AAAAAAAALHBHFCAA|2006-05-27|1276|5552|426|2006|6|5|27|2|2006|426|5552|Saturday|2006Q2|N|Y|N|2453857|2453976|2453518|2453793|N|N|N|N|N| +2453884|AAAAAAAAMHBHFCAA|2006-05-28|1276|5552|426|2006|0|5|28|2|2006|426|5552|Sunday|2006Q2|N|N|N|2453857|2453976|2453519|2453794|N|N|N|N|N| +2453885|AAAAAAAANHBHFCAA|2006-05-29|1276|5552|426|2006|1|5|29|2|2006|426|5552|Monday|2006Q2|N|N|N|2453857|2453976|2453520|2453795|N|N|N|N|N| +2453886|AAAAAAAAOHBHFCAA|2006-05-30|1276|5553|426|2006|2|5|30|2|2006|426|5553|Tuesday|2006Q2|N|N|N|2453857|2453976|2453521|2453796|N|N|N|N|N| +2453887|AAAAAAAAPHBHFCAA|2006-05-31|1276|5553|426|2006|3|5|31|2|2006|426|5553|Wednesday|2006Q2|N|N|N|2453857|2453976|2453522|2453797|N|N|N|N|N| +2453888|AAAAAAAAAIBHFCAA|2006-06-01|1277|5553|427|2006|4|6|1|2|2006|427|5553|Thursday|2006Q2|N|N|N|2453888|2454038|2453523|2453798|N|N|N|N|N| +2453889|AAAAAAAABIBHFCAA|2006-06-02|1277|5553|427|2006|5|6|2|2|2006|427|5553|Friday|2006Q2|N|Y|N|2453888|2454038|2453524|2453799|N|N|N|N|N| +2453890|AAAAAAAACIBHFCAA|2006-06-03|1277|5553|427|2006|6|6|3|2|2006|427|5553|Saturday|2006Q2|N|Y|N|2453888|2454038|2453525|2453800|N|N|N|N|N| +2453891|AAAAAAAADIBHFCAA|2006-06-04|1277|5553|427|2006|0|6|4|2|2006|427|5553|Sunday|2006Q2|N|N|N|2453888|2454038|2453526|2453801|N|N|N|N|N| +2453892|AAAAAAAAEIBHFCAA|2006-06-05|1277|5553|427|2006|1|6|5|2|2006|427|5553|Monday|2006Q2|N|N|N|2453888|2454038|2453527|2453802|N|N|N|N|N| +2453893|AAAAAAAAFIBHFCAA|2006-06-06|1277|5554|427|2006|2|6|6|2|2006|427|5554|Tuesday|2006Q2|N|N|N|2453888|2454038|2453528|2453803|N|N|N|N|N| +2453894|AAAAAAAAGIBHFCAA|2006-06-07|1277|5554|427|2006|3|6|7|2|2006|427|5554|Wednesday|2006Q2|N|N|N|2453888|2454038|2453529|2453804|N|N|N|N|N| +2453895|AAAAAAAAHIBHFCAA|2006-06-08|1277|5554|427|2006|4|6|8|2|2006|427|5554|Thursday|2006Q2|N|N|N|2453888|2454038|2453530|2453805|N|N|N|N|N| +2453896|AAAAAAAAIIBHFCAA|2006-06-09|1277|5554|427|2006|5|6|9|2|2006|427|5554|Friday|2006Q2|N|Y|N|2453888|2454038|2453531|2453806|N|N|N|N|N| +2453897|AAAAAAAAJIBHFCAA|2006-06-10|1277|5554|427|2006|6|6|10|2|2006|427|5554|Saturday|2006Q2|N|Y|N|2453888|2454038|2453532|2453807|N|N|N|N|N| +2453898|AAAAAAAAKIBHFCAA|2006-06-11|1277|5554|427|2006|0|6|11|2|2006|427|5554|Sunday|2006Q2|N|N|N|2453888|2454038|2453533|2453808|N|N|N|N|N| +2453899|AAAAAAAALIBHFCAA|2006-06-12|1277|5554|427|2006|1|6|12|2|2006|427|5554|Monday|2006Q2|N|N|N|2453888|2454038|2453534|2453809|N|N|N|N|N| +2453900|AAAAAAAAMIBHFCAA|2006-06-13|1277|5555|427|2006|2|6|13|2|2006|427|5555|Tuesday|2006Q2|N|N|N|2453888|2454038|2453535|2453810|N|N|N|N|N| +2453901|AAAAAAAANIBHFCAA|2006-06-14|1277|5555|427|2006|3|6|14|2|2006|427|5555|Wednesday|2006Q2|N|N|N|2453888|2454038|2453536|2453811|N|N|N|N|N| +2453902|AAAAAAAAOIBHFCAA|2006-06-15|1277|5555|427|2006|4|6|15|2|2006|427|5555|Thursday|2006Q2|N|N|N|2453888|2454038|2453537|2453812|N|N|N|N|N| +2453903|AAAAAAAAPIBHFCAA|2006-06-16|1277|5555|427|2006|5|6|16|2|2006|427|5555|Friday|2006Q2|N|Y|N|2453888|2454038|2453538|2453813|N|N|N|N|N| +2453904|AAAAAAAAAJBHFCAA|2006-06-17|1277|5555|427|2006|6|6|17|2|2006|427|5555|Saturday|2006Q2|N|Y|N|2453888|2454038|2453539|2453814|N|N|N|N|N| +2453905|AAAAAAAABJBHFCAA|2006-06-18|1277|5555|427|2006|0|6|18|2|2006|427|5555|Sunday|2006Q2|N|N|N|2453888|2454038|2453540|2453815|N|N|N|N|N| +2453906|AAAAAAAACJBHFCAA|2006-06-19|1277|5555|427|2006|1|6|19|2|2006|427|5555|Monday|2006Q2|N|N|N|2453888|2454038|2453541|2453816|N|N|N|N|N| +2453907|AAAAAAAADJBHFCAA|2006-06-20|1277|5556|427|2006|2|6|20|2|2006|427|5556|Tuesday|2006Q2|N|N|N|2453888|2454038|2453542|2453817|N|N|N|N|N| +2453908|AAAAAAAAEJBHFCAA|2006-06-21|1277|5556|427|2006|3|6|21|2|2006|427|5556|Wednesday|2006Q2|N|N|N|2453888|2454038|2453543|2453818|N|N|N|N|N| +2453909|AAAAAAAAFJBHFCAA|2006-06-22|1277|5556|427|2006|4|6|22|2|2006|427|5556|Thursday|2006Q2|N|N|N|2453888|2454038|2453544|2453819|N|N|N|N|N| +2453910|AAAAAAAAGJBHFCAA|2006-06-23|1277|5556|427|2006|5|6|23|2|2006|427|5556|Friday|2006Q2|N|Y|N|2453888|2454038|2453545|2453820|N|N|N|N|N| +2453911|AAAAAAAAHJBHFCAA|2006-06-24|1277|5556|427|2006|6|6|24|2|2006|427|5556|Saturday|2006Q2|N|Y|N|2453888|2454038|2453546|2453821|N|N|N|N|N| +2453912|AAAAAAAAIJBHFCAA|2006-06-25|1277|5556|427|2006|0|6|25|2|2006|427|5556|Sunday|2006Q2|N|N|N|2453888|2454038|2453547|2453822|N|N|N|N|N| +2453913|AAAAAAAAJJBHFCAA|2006-06-26|1277|5556|427|2006|1|6|26|2|2006|427|5556|Monday|2006Q2|N|N|N|2453888|2454038|2453548|2453823|N|N|N|N|N| +2453914|AAAAAAAAKJBHFCAA|2006-06-27|1277|5557|427|2006|2|6|27|2|2006|427|5557|Tuesday|2006Q2|N|N|N|2453888|2454038|2453549|2453824|N|N|N|N|N| +2453915|AAAAAAAALJBHFCAA|2006-06-28|1277|5557|427|2006|3|6|28|2|2006|427|5557|Wednesday|2006Q2|N|N|N|2453888|2454038|2453550|2453825|N|N|N|N|N| +2453916|AAAAAAAAMJBHFCAA|2006-06-29|1277|5557|427|2006|4|6|29|2|2006|427|5557|Thursday|2006Q2|N|N|N|2453888|2454038|2453551|2453826|N|N|N|N|N| +2453917|AAAAAAAANJBHFCAA|2006-06-30|1277|5557|427|2006|5|6|30|2|2006|427|5557|Friday|2006Q2|N|Y|N|2453888|2454038|2453552|2453827|N|N|N|N|N| +2453918|AAAAAAAAOJBHFCAA|2006-07-01|1278|5557|427|2006|6|7|1|2|2006|427|5557|Saturday|2006Q2|N|Y|N|2453918|2454098|2453553|2453827|N|N|N|N|N| +2453919|AAAAAAAAPJBHFCAA|2006-07-02|1278|5557|427|2006|0|7|2|3|2006|427|5557|Sunday|2006Q3|N|N|N|2453918|2454098|2453554|2453828|N|N|N|N|N| +2453920|AAAAAAAAAKBHFCAA|2006-07-03|1278|5557|427|2006|1|7|3|3|2006|427|5557|Monday|2006Q3|N|N|N|2453918|2454098|2453555|2453829|N|N|N|N|N| +2453921|AAAAAAAABKBHFCAA|2006-07-04|1278|5558|427|2006|2|7|4|3|2006|427|5558|Tuesday|2006Q3|N|N|N|2453918|2454098|2453556|2453830|N|N|N|N|N| +2453922|AAAAAAAACKBHFCAA|2006-07-05|1278|5558|427|2006|3|7|5|3|2006|427|5558|Wednesday|2006Q3|Y|N|N|2453918|2454098|2453557|2453831|N|N|N|N|N| +2453923|AAAAAAAADKBHFCAA|2006-07-06|1278|5558|427|2006|4|7|6|3|2006|427|5558|Thursday|2006Q3|N|N|Y|2453918|2454098|2453558|2453832|N|N|N|N|N| +2453924|AAAAAAAAEKBHFCAA|2006-07-07|1278|5558|427|2006|5|7|7|3|2006|427|5558|Friday|2006Q3|N|Y|N|2453918|2454098|2453559|2453833|N|N|N|N|N| +2453925|AAAAAAAAFKBHFCAA|2006-07-08|1278|5558|427|2006|6|7|8|3|2006|427|5558|Saturday|2006Q3|N|Y|N|2453918|2454098|2453560|2453834|N|N|N|N|N| +2453926|AAAAAAAAGKBHFCAA|2006-07-09|1278|5558|427|2006|0|7|9|3|2006|427|5558|Sunday|2006Q3|N|N|N|2453918|2454098|2453561|2453835|N|N|N|N|N| +2453927|AAAAAAAAHKBHFCAA|2006-07-10|1278|5558|427|2006|1|7|10|3|2006|427|5558|Monday|2006Q3|N|N|N|2453918|2454098|2453562|2453836|N|N|N|N|N| +2453928|AAAAAAAAIKBHFCAA|2006-07-11|1278|5559|427|2006|2|7|11|3|2006|427|5559|Tuesday|2006Q3|N|N|N|2453918|2454098|2453563|2453837|N|N|N|N|N| +2453929|AAAAAAAAJKBHFCAA|2006-07-12|1278|5559|427|2006|3|7|12|3|2006|427|5559|Wednesday|2006Q3|N|N|N|2453918|2454098|2453564|2453838|N|N|N|N|N| +2453930|AAAAAAAAKKBHFCAA|2006-07-13|1278|5559|427|2006|4|7|13|3|2006|427|5559|Thursday|2006Q3|N|N|N|2453918|2454098|2453565|2453839|N|N|N|N|N| +2453931|AAAAAAAALKBHFCAA|2006-07-14|1278|5559|427|2006|5|7|14|3|2006|427|5559|Friday|2006Q3|N|Y|N|2453918|2454098|2453566|2453840|N|N|N|N|N| +2453932|AAAAAAAAMKBHFCAA|2006-07-15|1278|5559|427|2006|6|7|15|3|2006|427|5559|Saturday|2006Q3|N|Y|N|2453918|2454098|2453567|2453841|N|N|N|N|N| +2453933|AAAAAAAANKBHFCAA|2006-07-16|1278|5559|427|2006|0|7|16|3|2006|427|5559|Sunday|2006Q3|N|N|N|2453918|2454098|2453568|2453842|N|N|N|N|N| +2453934|AAAAAAAAOKBHFCAA|2006-07-17|1278|5559|427|2006|1|7|17|3|2006|427|5559|Monday|2006Q3|N|N|N|2453918|2454098|2453569|2453843|N|N|N|N|N| +2453935|AAAAAAAAPKBHFCAA|2006-07-18|1278|5560|427|2006|2|7|18|3|2006|427|5560|Tuesday|2006Q3|N|N|N|2453918|2454098|2453570|2453844|N|N|N|N|N| +2453936|AAAAAAAAALBHFCAA|2006-07-19|1278|5560|427|2006|3|7|19|3|2006|427|5560|Wednesday|2006Q3|N|N|N|2453918|2454098|2453571|2453845|N|N|N|N|N| +2453937|AAAAAAAABLBHFCAA|2006-07-20|1278|5560|427|2006|4|7|20|3|2006|427|5560|Thursday|2006Q3|N|N|N|2453918|2454098|2453572|2453846|N|N|N|N|N| +2453938|AAAAAAAACLBHFCAA|2006-07-21|1278|5560|427|2006|5|7|21|3|2006|427|5560|Friday|2006Q3|N|Y|N|2453918|2454098|2453573|2453847|N|N|N|N|N| +2453939|AAAAAAAADLBHFCAA|2006-07-22|1278|5560|427|2006|6|7|22|3|2006|427|5560|Saturday|2006Q3|N|Y|N|2453918|2454098|2453574|2453848|N|N|N|N|N| +2453940|AAAAAAAAELBHFCAA|2006-07-23|1278|5560|427|2006|0|7|23|3|2006|427|5560|Sunday|2006Q3|N|N|N|2453918|2454098|2453575|2453849|N|N|N|N|N| +2453941|AAAAAAAAFLBHFCAA|2006-07-24|1278|5560|427|2006|1|7|24|3|2006|427|5560|Monday|2006Q3|N|N|N|2453918|2454098|2453576|2453850|N|N|N|N|N| +2453942|AAAAAAAAGLBHFCAA|2006-07-25|1278|5561|427|2006|2|7|25|3|2006|427|5561|Tuesday|2006Q3|N|N|N|2453918|2454098|2453577|2453851|N|N|N|N|N| +2453943|AAAAAAAAHLBHFCAA|2006-07-26|1278|5561|427|2006|3|7|26|3|2006|427|5561|Wednesday|2006Q3|N|N|N|2453918|2454098|2453578|2453852|N|N|N|N|N| +2453944|AAAAAAAAILBHFCAA|2006-07-27|1278|5561|427|2006|4|7|27|3|2006|427|5561|Thursday|2006Q3|N|N|N|2453918|2454098|2453579|2453853|N|N|N|N|N| +2453945|AAAAAAAAJLBHFCAA|2006-07-28|1278|5561|427|2006|5|7|28|3|2006|427|5561|Friday|2006Q3|N|Y|N|2453918|2454098|2453580|2453854|N|N|N|N|N| +2453946|AAAAAAAAKLBHFCAA|2006-07-29|1278|5561|427|2006|6|7|29|3|2006|427|5561|Saturday|2006Q3|N|Y|N|2453918|2454098|2453581|2453855|N|N|N|N|N| +2453947|AAAAAAAALLBHFCAA|2006-07-30|1278|5561|427|2006|0|7|30|3|2006|427|5561|Sunday|2006Q3|N|N|N|2453918|2454098|2453582|2453856|N|N|N|N|N| +2453948|AAAAAAAAMLBHFCAA|2006-07-31|1278|5561|427|2006|1|7|31|3|2006|427|5561|Monday|2006Q3|N|N|N|2453918|2454098|2453583|2453857|N|N|N|N|N| +2453949|AAAAAAAANLBHFCAA|2006-08-01|1279|5562|427|2006|2|8|1|3|2006|427|5562|Tuesday|2006Q3|N|N|N|2453949|2454160|2453584|2453858|N|N|N|N|N| +2453950|AAAAAAAAOLBHFCAA|2006-08-02|1279|5562|427|2006|3|8|2|3|2006|427|5562|Wednesday|2006Q3|N|N|N|2453949|2454160|2453585|2453859|N|N|N|N|N| +2453951|AAAAAAAAPLBHFCAA|2006-08-03|1279|5562|427|2006|4|8|3|3|2006|427|5562|Thursday|2006Q3|N|N|N|2453949|2454160|2453586|2453860|N|N|N|N|N| +2453952|AAAAAAAAAMBHFCAA|2006-08-04|1279|5562|427|2006|5|8|4|3|2006|427|5562|Friday|2006Q3|N|Y|N|2453949|2454160|2453587|2453861|N|N|N|N|N| +2453953|AAAAAAAABMBHFCAA|2006-08-05|1279|5562|427|2006|6|8|5|3|2006|427|5562|Saturday|2006Q3|N|Y|N|2453949|2454160|2453588|2453862|N|N|N|N|N| +2453954|AAAAAAAACMBHFCAA|2006-08-06|1279|5562|427|2006|0|8|6|3|2006|427|5562|Sunday|2006Q3|N|N|N|2453949|2454160|2453589|2453863|N|N|N|N|N| +2453955|AAAAAAAADMBHFCAA|2006-08-07|1279|5562|427|2006|1|8|7|3|2006|427|5562|Monday|2006Q3|N|N|N|2453949|2454160|2453590|2453864|N|N|N|N|N| +2453956|AAAAAAAAEMBHFCAA|2006-08-08|1279|5563|427|2006|2|8|8|3|2006|427|5563|Tuesday|2006Q3|N|N|N|2453949|2454160|2453591|2453865|N|N|N|N|N| +2453957|AAAAAAAAFMBHFCAA|2006-08-09|1279|5563|427|2006|3|8|9|3|2006|427|5563|Wednesday|2006Q3|N|N|N|2453949|2454160|2453592|2453866|N|N|N|N|N| +2453958|AAAAAAAAGMBHFCAA|2006-08-10|1279|5563|427|2006|4|8|10|3|2006|427|5563|Thursday|2006Q3|N|N|N|2453949|2454160|2453593|2453867|N|N|N|N|N| +2453959|AAAAAAAAHMBHFCAA|2006-08-11|1279|5563|427|2006|5|8|11|3|2006|427|5563|Friday|2006Q3|N|Y|N|2453949|2454160|2453594|2453868|N|N|N|N|N| +2453960|AAAAAAAAIMBHFCAA|2006-08-12|1279|5563|427|2006|6|8|12|3|2006|427|5563|Saturday|2006Q3|N|Y|N|2453949|2454160|2453595|2453869|N|N|N|N|N| +2453961|AAAAAAAAJMBHFCAA|2006-08-13|1279|5563|427|2006|0|8|13|3|2006|427|5563|Sunday|2006Q3|N|N|N|2453949|2454160|2453596|2453870|N|N|N|N|N| +2453962|AAAAAAAAKMBHFCAA|2006-08-14|1279|5563|427|2006|1|8|14|3|2006|427|5563|Monday|2006Q3|N|N|N|2453949|2454160|2453597|2453871|N|N|N|N|N| +2453963|AAAAAAAALMBHFCAA|2006-08-15|1279|5564|427|2006|2|8|15|3|2006|427|5564|Tuesday|2006Q3|N|N|N|2453949|2454160|2453598|2453872|N|N|N|N|N| +2453964|AAAAAAAAMMBHFCAA|2006-08-16|1279|5564|427|2006|3|8|16|3|2006|427|5564|Wednesday|2006Q3|N|N|N|2453949|2454160|2453599|2453873|N|N|N|N|N| +2453965|AAAAAAAANMBHFCAA|2006-08-17|1279|5564|427|2006|4|8|17|3|2006|427|5564|Thursday|2006Q3|N|N|N|2453949|2454160|2453600|2453874|N|N|N|N|N| +2453966|AAAAAAAAOMBHFCAA|2006-08-18|1279|5564|427|2006|5|8|18|3|2006|427|5564|Friday|2006Q3|N|Y|N|2453949|2454160|2453601|2453875|N|N|N|N|N| +2453967|AAAAAAAAPMBHFCAA|2006-08-19|1279|5564|427|2006|6|8|19|3|2006|427|5564|Saturday|2006Q3|N|Y|N|2453949|2454160|2453602|2453876|N|N|N|N|N| +2453968|AAAAAAAAANBHFCAA|2006-08-20|1279|5564|427|2006|0|8|20|3|2006|427|5564|Sunday|2006Q3|N|N|N|2453949|2454160|2453603|2453877|N|N|N|N|N| +2453969|AAAAAAAABNBHFCAA|2006-08-21|1279|5564|427|2006|1|8|21|3|2006|427|5564|Monday|2006Q3|N|N|N|2453949|2454160|2453604|2453878|N|N|N|N|N| +2453970|AAAAAAAACNBHFCAA|2006-08-22|1279|5565|427|2006|2|8|22|3|2006|427|5565|Tuesday|2006Q3|N|N|N|2453949|2454160|2453605|2453879|N|N|N|N|N| +2453971|AAAAAAAADNBHFCAA|2006-08-23|1279|5565|427|2006|3|8|23|3|2006|427|5565|Wednesday|2006Q3|N|N|N|2453949|2454160|2453606|2453880|N|N|N|N|N| +2453972|AAAAAAAAENBHFCAA|2006-08-24|1279|5565|427|2006|4|8|24|3|2006|427|5565|Thursday|2006Q3|N|N|N|2453949|2454160|2453607|2453881|N|N|N|N|N| +2453973|AAAAAAAAFNBHFCAA|2006-08-25|1279|5565|427|2006|5|8|25|3|2006|427|5565|Friday|2006Q3|N|Y|N|2453949|2454160|2453608|2453882|N|N|N|N|N| +2453974|AAAAAAAAGNBHFCAA|2006-08-26|1279|5565|427|2006|6|8|26|3|2006|427|5565|Saturday|2006Q3|N|Y|N|2453949|2454160|2453609|2453883|N|N|N|N|N| +2453975|AAAAAAAAHNBHFCAA|2006-08-27|1279|5565|427|2006|0|8|27|3|2006|427|5565|Sunday|2006Q3|N|N|N|2453949|2454160|2453610|2453884|N|N|N|N|N| +2453976|AAAAAAAAINBHFCAA|2006-08-28|1279|5565|427|2006|1|8|28|3|2006|427|5565|Monday|2006Q3|N|N|N|2453949|2454160|2453611|2453885|N|N|N|N|N| +2453977|AAAAAAAAJNBHFCAA|2006-08-29|1279|5566|427|2006|2|8|29|3|2006|427|5566|Tuesday|2006Q3|N|N|N|2453949|2454160|2453612|2453886|N|N|N|N|N| +2453978|AAAAAAAAKNBHFCAA|2006-08-30|1279|5566|427|2006|3|8|30|3|2006|427|5566|Wednesday|2006Q3|N|N|N|2453949|2454160|2453613|2453887|N|N|N|N|N| +2453979|AAAAAAAALNBHFCAA|2006-08-31|1279|5566|427|2006|4|8|31|3|2006|427|5566|Thursday|2006Q3|N|N|N|2453949|2454160|2453614|2453888|N|N|N|N|N| +2453980|AAAAAAAAMNBHFCAA|2006-09-01|1280|5566|428|2006|5|9|1|3|2006|428|5566|Friday|2006Q3|N|Y|N|2453980|2454222|2453615|2453889|N|N|N|N|N| +2453981|AAAAAAAANNBHFCAA|2006-09-02|1280|5566|428|2006|6|9|2|3|2006|428|5566|Saturday|2006Q3|N|Y|N|2453980|2454222|2453616|2453890|N|N|N|N|N| +2453982|AAAAAAAAONBHFCAA|2006-09-03|1280|5566|428|2006|0|9|3|3|2006|428|5566|Sunday|2006Q3|N|N|N|2453980|2454222|2453617|2453891|N|N|N|N|N| +2453983|AAAAAAAAPNBHFCAA|2006-09-04|1280|5566|428|2006|1|9|4|3|2006|428|5566|Monday|2006Q3|N|N|N|2453980|2454222|2453618|2453892|N|N|N|N|N| +2453984|AAAAAAAAAOBHFCAA|2006-09-05|1280|5567|428|2006|2|9|5|3|2006|428|5567|Tuesday|2006Q3|N|N|N|2453980|2454222|2453619|2453893|N|N|N|N|N| +2453985|AAAAAAAABOBHFCAA|2006-09-06|1280|5567|428|2006|3|9|6|3|2006|428|5567|Wednesday|2006Q3|N|N|N|2453980|2454222|2453620|2453894|N|N|N|N|N| +2453986|AAAAAAAACOBHFCAA|2006-09-07|1280|5567|428|2006|4|9|7|3|2006|428|5567|Thursday|2006Q3|N|N|N|2453980|2454222|2453621|2453895|N|N|N|N|N| +2453987|AAAAAAAADOBHFCAA|2006-09-08|1280|5567|428|2006|5|9|8|3|2006|428|5567|Friday|2006Q3|N|Y|N|2453980|2454222|2453622|2453896|N|N|N|N|N| +2453988|AAAAAAAAEOBHFCAA|2006-09-09|1280|5567|428|2006|6|9|9|3|2006|428|5567|Saturday|2006Q3|N|Y|N|2453980|2454222|2453623|2453897|N|N|N|N|N| +2453989|AAAAAAAAFOBHFCAA|2006-09-10|1280|5567|428|2006|0|9|10|3|2006|428|5567|Sunday|2006Q3|N|N|N|2453980|2454222|2453624|2453898|N|N|N|N|N| +2453990|AAAAAAAAGOBHFCAA|2006-09-11|1280|5567|428|2006|1|9|11|3|2006|428|5567|Monday|2006Q3|N|N|N|2453980|2454222|2453625|2453899|N|N|N|N|N| +2453991|AAAAAAAAHOBHFCAA|2006-09-12|1280|5568|428|2006|2|9|12|3|2006|428|5568|Tuesday|2006Q3|N|N|N|2453980|2454222|2453626|2453900|N|N|N|N|N| +2453992|AAAAAAAAIOBHFCAA|2006-09-13|1280|5568|428|2006|3|9|13|3|2006|428|5568|Wednesday|2006Q3|N|N|N|2453980|2454222|2453627|2453901|N|N|N|N|N| +2453993|AAAAAAAAJOBHFCAA|2006-09-14|1280|5568|428|2006|4|9|14|3|2006|428|5568|Thursday|2006Q3|N|N|N|2453980|2454222|2453628|2453902|N|N|N|N|N| +2453994|AAAAAAAAKOBHFCAA|2006-09-15|1280|5568|428|2006|5|9|15|3|2006|428|5568|Friday|2006Q3|N|Y|N|2453980|2454222|2453629|2453903|N|N|N|N|N| +2453995|AAAAAAAALOBHFCAA|2006-09-16|1280|5568|428|2006|6|9|16|3|2006|428|5568|Saturday|2006Q3|N|Y|N|2453980|2454222|2453630|2453904|N|N|N|N|N| +2453996|AAAAAAAAMOBHFCAA|2006-09-17|1280|5568|428|2006|0|9|17|3|2006|428|5568|Sunday|2006Q3|N|N|N|2453980|2454222|2453631|2453905|N|N|N|N|N| +2453997|AAAAAAAANOBHFCAA|2006-09-18|1280|5568|428|2006|1|9|18|3|2006|428|5568|Monday|2006Q3|N|N|N|2453980|2454222|2453632|2453906|N|N|N|N|N| +2453998|AAAAAAAAOOBHFCAA|2006-09-19|1280|5569|428|2006|2|9|19|3|2006|428|5569|Tuesday|2006Q3|N|N|N|2453980|2454222|2453633|2453907|N|N|N|N|N| +2453999|AAAAAAAAPOBHFCAA|2006-09-20|1280|5569|428|2006|3|9|20|3|2006|428|5569|Wednesday|2006Q3|N|N|N|2453980|2454222|2453634|2453908|N|N|N|N|N| +2454000|AAAAAAAAAPBHFCAA|2006-09-21|1280|5569|428|2006|4|9|21|3|2006|428|5569|Thursday|2006Q3|N|N|N|2453980|2454222|2453635|2453909|N|N|N|N|N| +2454001|AAAAAAAABPBHFCAA|2006-09-22|1280|5569|428|2006|5|9|22|3|2006|428|5569|Friday|2006Q3|N|Y|N|2453980|2454222|2453636|2453910|N|N|N|N|N| +2454002|AAAAAAAACPBHFCAA|2006-09-23|1280|5569|428|2006|6|9|23|3|2006|428|5569|Saturday|2006Q3|N|Y|N|2453980|2454222|2453637|2453911|N|N|N|N|N| +2454003|AAAAAAAADPBHFCAA|2006-09-24|1280|5569|428|2006|0|9|24|3|2006|428|5569|Sunday|2006Q3|N|N|N|2453980|2454222|2453638|2453912|N|N|N|N|N| +2454004|AAAAAAAAEPBHFCAA|2006-09-25|1280|5569|428|2006|1|9|25|3|2006|428|5569|Monday|2006Q3|N|N|N|2453980|2454222|2453639|2453913|N|N|N|N|N| +2454005|AAAAAAAAFPBHFCAA|2006-09-26|1280|5570|428|2006|2|9|26|3|2006|428|5570|Tuesday|2006Q3|N|N|N|2453980|2454222|2453640|2453914|N|N|N|N|N| +2454006|AAAAAAAAGPBHFCAA|2006-09-27|1280|5570|428|2006|3|9|27|3|2006|428|5570|Wednesday|2006Q3|N|N|N|2453980|2454222|2453641|2453915|N|N|N|N|N| +2454007|AAAAAAAAHPBHFCAA|2006-09-28|1280|5570|428|2006|4|9|28|3|2006|428|5570|Thursday|2006Q3|N|N|N|2453980|2454222|2453642|2453916|N|N|N|N|N| +2454008|AAAAAAAAIPBHFCAA|2006-09-29|1280|5570|428|2006|5|9|29|3|2006|428|5570|Friday|2006Q3|N|Y|N|2453980|2454222|2453643|2453917|N|N|N|N|N| +2454009|AAAAAAAAJPBHFCAA|2006-09-30|1280|5570|428|2006|6|9|30|3|2006|428|5570|Saturday|2006Q3|N|Y|N|2453980|2454222|2453644|2453918|N|N|N|N|N| +2454010|AAAAAAAAKPBHFCAA|2006-10-01|1281|5570|428|2006|0|10|1|3|2006|428|5570|Sunday|2006Q3|N|N|N|2454010|2454282|2453645|2453918|N|N|N|N|N| +2454011|AAAAAAAALPBHFCAA|2006-10-02|1281|5570|428|2006|1|10|2|4|2006|428|5570|Monday|2006Q4|N|N|N|2454010|2454282|2453646|2453919|N|N|N|N|N| +2454012|AAAAAAAAMPBHFCAA|2006-10-03|1281|5571|428|2006|2|10|3|4|2006|428|5571|Tuesday|2006Q4|N|N|N|2454010|2454282|2453647|2453920|N|N|N|N|N| +2454013|AAAAAAAANPBHFCAA|2006-10-04|1281|5571|428|2006|3|10|4|4|2006|428|5571|Wednesday|2006Q4|N|N|N|2454010|2454282|2453648|2453921|N|N|N|N|N| +2454014|AAAAAAAAOPBHFCAA|2006-10-05|1281|5571|428|2006|4|10|5|4|2006|428|5571|Thursday|2006Q4|N|N|N|2454010|2454282|2453649|2453922|N|N|N|N|N| +2454015|AAAAAAAAPPBHFCAA|2006-10-06|1281|5571|428|2006|5|10|6|4|2006|428|5571|Friday|2006Q4|N|Y|N|2454010|2454282|2453650|2453923|N|N|N|N|N| +2454016|AAAAAAAAAACHFCAA|2006-10-07|1281|5571|428|2006|6|10|7|4|2006|428|5571|Saturday|2006Q4|N|Y|N|2454010|2454282|2453651|2453924|N|N|N|N|N| +2454017|AAAAAAAABACHFCAA|2006-10-08|1281|5571|428|2006|0|10|8|4|2006|428|5571|Sunday|2006Q4|N|N|N|2454010|2454282|2453652|2453925|N|N|N|N|N| +2454018|AAAAAAAACACHFCAA|2006-10-09|1281|5571|428|2006|1|10|9|4|2006|428|5571|Monday|2006Q4|N|N|N|2454010|2454282|2453653|2453926|N|N|N|N|N| +2454019|AAAAAAAADACHFCAA|2006-10-10|1281|5572|428|2006|2|10|10|4|2006|428|5572|Tuesday|2006Q4|N|N|N|2454010|2454282|2453654|2453927|N|N|N|N|N| +2454020|AAAAAAAAEACHFCAA|2006-10-11|1281|5572|428|2006|3|10|11|4|2006|428|5572|Wednesday|2006Q4|N|N|N|2454010|2454282|2453655|2453928|N|N|N|N|N| +2454021|AAAAAAAAFACHFCAA|2006-10-12|1281|5572|428|2006|4|10|12|4|2006|428|5572|Thursday|2006Q4|N|N|N|2454010|2454282|2453656|2453929|N|N|N|N|N| +2454022|AAAAAAAAGACHFCAA|2006-10-13|1281|5572|428|2006|5|10|13|4|2006|428|5572|Friday|2006Q4|N|Y|N|2454010|2454282|2453657|2453930|N|N|N|N|N| +2454023|AAAAAAAAHACHFCAA|2006-10-14|1281|5572|428|2006|6|10|14|4|2006|428|5572|Saturday|2006Q4|N|Y|N|2454010|2454282|2453658|2453931|N|N|N|N|N| +2454024|AAAAAAAAIACHFCAA|2006-10-15|1281|5572|428|2006|0|10|15|4|2006|428|5572|Sunday|2006Q4|N|N|N|2454010|2454282|2453659|2453932|N|N|N|N|N| +2454025|AAAAAAAAJACHFCAA|2006-10-16|1281|5572|428|2006|1|10|16|4|2006|428|5572|Monday|2006Q4|N|N|N|2454010|2454282|2453660|2453933|N|N|N|N|N| +2454026|AAAAAAAAKACHFCAA|2006-10-17|1281|5573|428|2006|2|10|17|4|2006|428|5573|Tuesday|2006Q4|N|N|N|2454010|2454282|2453661|2453934|N|N|N|N|N| +2454027|AAAAAAAALACHFCAA|2006-10-18|1281|5573|428|2006|3|10|18|4|2006|428|5573|Wednesday|2006Q4|N|N|N|2454010|2454282|2453662|2453935|N|N|N|N|N| +2454028|AAAAAAAAMACHFCAA|2006-10-19|1281|5573|428|2006|4|10|19|4|2006|428|5573|Thursday|2006Q4|N|N|N|2454010|2454282|2453663|2453936|N|N|N|N|N| +2454029|AAAAAAAANACHFCAA|2006-10-20|1281|5573|428|2006|5|10|20|4|2006|428|5573|Friday|2006Q4|N|Y|N|2454010|2454282|2453664|2453937|N|N|N|N|N| +2454030|AAAAAAAAOACHFCAA|2006-10-21|1281|5573|428|2006|6|10|21|4|2006|428|5573|Saturday|2006Q4|N|Y|N|2454010|2454282|2453665|2453938|N|N|N|N|N| +2454031|AAAAAAAAPACHFCAA|2006-10-22|1281|5573|428|2006|0|10|22|4|2006|428|5573|Sunday|2006Q4|N|N|N|2454010|2454282|2453666|2453939|N|N|N|N|N| +2454032|AAAAAAAAABCHFCAA|2006-10-23|1281|5573|428|2006|1|10|23|4|2006|428|5573|Monday|2006Q4|N|N|N|2454010|2454282|2453667|2453940|N|N|N|N|N| +2454033|AAAAAAAABBCHFCAA|2006-10-24|1281|5574|428|2006|2|10|24|4|2006|428|5574|Tuesday|2006Q4|N|N|N|2454010|2454282|2453668|2453941|N|N|N|N|N| +2454034|AAAAAAAACBCHFCAA|2006-10-25|1281|5574|428|2006|3|10|25|4|2006|428|5574|Wednesday|2006Q4|N|N|N|2454010|2454282|2453669|2453942|N|N|N|N|N| +2454035|AAAAAAAADBCHFCAA|2006-10-26|1281|5574|428|2006|4|10|26|4|2006|428|5574|Thursday|2006Q4|N|N|N|2454010|2454282|2453670|2453943|N|N|N|N|N| +2454036|AAAAAAAAEBCHFCAA|2006-10-27|1281|5574|428|2006|5|10|27|4|2006|428|5574|Friday|2006Q4|N|Y|N|2454010|2454282|2453671|2453944|N|N|N|N|N| +2454037|AAAAAAAAFBCHFCAA|2006-10-28|1281|5574|428|2006|6|10|28|4|2006|428|5574|Saturday|2006Q4|N|Y|N|2454010|2454282|2453672|2453945|N|N|N|N|N| +2454038|AAAAAAAAGBCHFCAA|2006-10-29|1281|5574|428|2006|0|10|29|4|2006|428|5574|Sunday|2006Q4|N|N|N|2454010|2454282|2453673|2453946|N|N|N|N|N| +2454039|AAAAAAAAHBCHFCAA|2006-10-30|1281|5574|428|2006|1|10|30|4|2006|428|5574|Monday|2006Q4|N|N|N|2454010|2454282|2453674|2453947|N|N|N|N|N| +2454040|AAAAAAAAIBCHFCAA|2006-10-31|1281|5575|428|2006|2|10|31|4|2006|428|5575|Tuesday|2006Q4|N|N|N|2454010|2454282|2453675|2453948|N|N|N|N|N| +2454041|AAAAAAAAJBCHFCAA|2006-11-01|1282|5575|428|2006|3|11|1|4|2006|428|5575|Wednesday|2006Q4|N|N|N|2454041|2454344|2453676|2453949|N|N|N|N|N| +2454042|AAAAAAAAKBCHFCAA|2006-11-02|1282|5575|428|2006|4|11|2|4|2006|428|5575|Thursday|2006Q4|N|N|N|2454041|2454344|2453677|2453950|N|N|N|N|N| +2454043|AAAAAAAALBCHFCAA|2006-11-03|1282|5575|428|2006|5|11|3|4|2006|428|5575|Friday|2006Q4|N|Y|N|2454041|2454344|2453678|2453951|N|N|N|N|N| +2454044|AAAAAAAAMBCHFCAA|2006-11-04|1282|5575|428|2006|6|11|4|4|2006|428|5575|Saturday|2006Q4|N|Y|N|2454041|2454344|2453679|2453952|N|N|N|N|N| +2454045|AAAAAAAANBCHFCAA|2006-11-05|1282|5575|428|2006|0|11|5|4|2006|428|5575|Sunday|2006Q4|N|N|N|2454041|2454344|2453680|2453953|N|N|N|N|N| +2454046|AAAAAAAAOBCHFCAA|2006-11-06|1282|5575|428|2006|1|11|6|4|2006|428|5575|Monday|2006Q4|N|N|N|2454041|2454344|2453681|2453954|N|N|N|N|N| +2454047|AAAAAAAAPBCHFCAA|2006-11-07|1282|5576|428|2006|2|11|7|4|2006|428|5576|Tuesday|2006Q4|N|N|N|2454041|2454344|2453682|2453955|N|N|N|N|N| +2454048|AAAAAAAAACCHFCAA|2006-11-08|1282|5576|428|2006|3|11|8|4|2006|428|5576|Wednesday|2006Q4|N|N|N|2454041|2454344|2453683|2453956|N|N|N|N|N| +2454049|AAAAAAAABCCHFCAA|2006-11-09|1282|5576|428|2006|4|11|9|4|2006|428|5576|Thursday|2006Q4|N|N|N|2454041|2454344|2453684|2453957|N|N|N|N|N| +2454050|AAAAAAAACCCHFCAA|2006-11-10|1282|5576|428|2006|5|11|10|4|2006|428|5576|Friday|2006Q4|N|Y|N|2454041|2454344|2453685|2453958|N|N|N|N|N| +2454051|AAAAAAAADCCHFCAA|2006-11-11|1282|5576|428|2006|6|11|11|4|2006|428|5576|Saturday|2006Q4|N|Y|N|2454041|2454344|2453686|2453959|N|N|N|N|N| +2454052|AAAAAAAAECCHFCAA|2006-11-12|1282|5576|428|2006|0|11|12|4|2006|428|5576|Sunday|2006Q4|N|N|N|2454041|2454344|2453687|2453960|N|N|N|N|N| +2454053|AAAAAAAAFCCHFCAA|2006-11-13|1282|5576|428|2006|1|11|13|4|2006|428|5576|Monday|2006Q4|N|N|N|2454041|2454344|2453688|2453961|N|N|N|N|N| +2454054|AAAAAAAAGCCHFCAA|2006-11-14|1282|5577|428|2006|2|11|14|4|2006|428|5577|Tuesday|2006Q4|N|N|N|2454041|2454344|2453689|2453962|N|N|N|N|N| +2454055|AAAAAAAAHCCHFCAA|2006-11-15|1282|5577|428|2006|3|11|15|4|2006|428|5577|Wednesday|2006Q4|N|N|N|2454041|2454344|2453690|2453963|N|N|N|N|N| +2454056|AAAAAAAAICCHFCAA|2006-11-16|1282|5577|428|2006|4|11|16|4|2006|428|5577|Thursday|2006Q4|N|N|N|2454041|2454344|2453691|2453964|N|N|N|N|N| +2454057|AAAAAAAAJCCHFCAA|2006-11-17|1282|5577|428|2006|5|11|17|4|2006|428|5577|Friday|2006Q4|N|Y|N|2454041|2454344|2453692|2453965|N|N|N|N|N| +2454058|AAAAAAAAKCCHFCAA|2006-11-18|1282|5577|428|2006|6|11|18|4|2006|428|5577|Saturday|2006Q4|N|Y|N|2454041|2454344|2453693|2453966|N|N|N|N|N| +2454059|AAAAAAAALCCHFCAA|2006-11-19|1282|5577|428|2006|0|11|19|4|2006|428|5577|Sunday|2006Q4|N|N|N|2454041|2454344|2453694|2453967|N|N|N|N|N| +2454060|AAAAAAAAMCCHFCAA|2006-11-20|1282|5577|428|2006|1|11|20|4|2006|428|5577|Monday|2006Q4|N|N|N|2454041|2454344|2453695|2453968|N|N|N|N|N| +2454061|AAAAAAAANCCHFCAA|2006-11-21|1282|5578|428|2006|2|11|21|4|2006|428|5578|Tuesday|2006Q4|N|N|N|2454041|2454344|2453696|2453969|N|N|N|N|N| +2454062|AAAAAAAAOCCHFCAA|2006-11-22|1282|5578|428|2006|3|11|22|4|2006|428|5578|Wednesday|2006Q4|N|N|N|2454041|2454344|2453697|2453970|N|N|N|N|N| +2454063|AAAAAAAAPCCHFCAA|2006-11-23|1282|5578|428|2006|4|11|23|4|2006|428|5578|Thursday|2006Q4|N|N|N|2454041|2454344|2453698|2453971|N|N|N|N|N| +2454064|AAAAAAAAADCHFCAA|2006-11-24|1282|5578|428|2006|5|11|24|4|2006|428|5578|Friday|2006Q4|N|Y|N|2454041|2454344|2453699|2453972|N|N|N|N|N| +2454065|AAAAAAAABDCHFCAA|2006-11-25|1282|5578|428|2006|6|11|25|4|2006|428|5578|Saturday|2006Q4|N|Y|N|2454041|2454344|2453700|2453973|N|N|N|N|N| +2454066|AAAAAAAACDCHFCAA|2006-11-26|1282|5578|428|2006|0|11|26|4|2006|428|5578|Sunday|2006Q4|N|N|N|2454041|2454344|2453701|2453974|N|N|N|N|N| +2454067|AAAAAAAADDCHFCAA|2006-11-27|1282|5578|428|2006|1|11|27|4|2006|428|5578|Monday|2006Q4|N|N|N|2454041|2454344|2453702|2453975|N|N|N|N|N| +2454068|AAAAAAAAEDCHFCAA|2006-11-28|1282|5579|428|2006|2|11|28|4|2006|428|5579|Tuesday|2006Q4|N|N|N|2454041|2454344|2453703|2453976|N|N|N|N|N| +2454069|AAAAAAAAFDCHFCAA|2006-11-29|1282|5579|428|2006|3|11|29|4|2006|428|5579|Wednesday|2006Q4|N|N|N|2454041|2454344|2453704|2453977|N|N|N|N|N| +2454070|AAAAAAAAGDCHFCAA|2006-11-30|1282|5579|428|2006|4|11|30|4|2006|428|5579|Thursday|2006Q4|N|N|N|2454041|2454344|2453705|2453978|N|N|N|N|N| +2454071|AAAAAAAAHDCHFCAA|2006-12-01|1283|5579|429|2006|5|12|1|4|2006|429|5579|Friday|2006Q4|N|Y|N|2454071|2454404|2453706|2453979|N|N|N|N|N| +2454072|AAAAAAAAIDCHFCAA|2006-12-02|1283|5579|429|2006|6|12|2|4|2006|429|5579|Saturday|2006Q4|N|Y|N|2454071|2454404|2453707|2453980|N|N|N|N|N| +2454073|AAAAAAAAJDCHFCAA|2006-12-03|1283|5579|429|2006|0|12|3|4|2006|429|5579|Sunday|2006Q4|N|N|N|2454071|2454404|2453708|2453981|N|N|N|N|N| +2454074|AAAAAAAAKDCHFCAA|2006-12-04|1283|5579|429|2006|1|12|4|4|2006|429|5579|Monday|2006Q4|N|N|N|2454071|2454404|2453709|2453982|N|N|N|N|N| +2454075|AAAAAAAALDCHFCAA|2006-12-05|1283|5580|429|2006|2|12|5|4|2006|429|5580|Tuesday|2006Q4|N|N|N|2454071|2454404|2453710|2453983|N|N|N|N|N| +2454076|AAAAAAAAMDCHFCAA|2006-12-06|1283|5580|429|2006|3|12|6|4|2006|429|5580|Wednesday|2006Q4|N|N|N|2454071|2454404|2453711|2453984|N|N|N|N|N| +2454077|AAAAAAAANDCHFCAA|2006-12-07|1283|5580|429|2006|4|12|7|4|2006|429|5580|Thursday|2006Q4|N|N|N|2454071|2454404|2453712|2453985|N|N|N|N|N| +2454078|AAAAAAAAODCHFCAA|2006-12-08|1283|5580|429|2006|5|12|8|4|2006|429|5580|Friday|2006Q4|N|Y|N|2454071|2454404|2453713|2453986|N|N|N|N|N| +2454079|AAAAAAAAPDCHFCAA|2006-12-09|1283|5580|429|2006|6|12|9|4|2006|429|5580|Saturday|2006Q4|N|Y|N|2454071|2454404|2453714|2453987|N|N|N|N|N| +2454080|AAAAAAAAAECHFCAA|2006-12-10|1283|5580|429|2006|0|12|10|4|2006|429|5580|Sunday|2006Q4|N|N|N|2454071|2454404|2453715|2453988|N|N|N|N|N| +2454081|AAAAAAAABECHFCAA|2006-12-11|1283|5580|429|2006|1|12|11|4|2006|429|5580|Monday|2006Q4|N|N|N|2454071|2454404|2453716|2453989|N|N|N|N|N| +2454082|AAAAAAAACECHFCAA|2006-12-12|1283|5581|429|2006|2|12|12|4|2006|429|5581|Tuesday|2006Q4|N|N|N|2454071|2454404|2453717|2453990|N|N|N|N|N| +2454083|AAAAAAAADECHFCAA|2006-12-13|1283|5581|429|2006|3|12|13|4|2006|429|5581|Wednesday|2006Q4|N|N|N|2454071|2454404|2453718|2453991|N|N|N|N|N| +2454084|AAAAAAAAEECHFCAA|2006-12-14|1283|5581|429|2006|4|12|14|4|2006|429|5581|Thursday|2006Q4|N|N|N|2454071|2454404|2453719|2453992|N|N|N|N|N| +2454085|AAAAAAAAFECHFCAA|2006-12-15|1283|5581|429|2006|5|12|15|4|2006|429|5581|Friday|2006Q4|N|Y|N|2454071|2454404|2453720|2453993|N|N|N|N|N| +2454086|AAAAAAAAGECHFCAA|2006-12-16|1283|5581|429|2006|6|12|16|4|2006|429|5581|Saturday|2006Q4|N|Y|N|2454071|2454404|2453721|2453994|N|N|N|N|N| +2454087|AAAAAAAAHECHFCAA|2006-12-17|1283|5581|429|2006|0|12|17|4|2006|429|5581|Sunday|2006Q4|N|N|N|2454071|2454404|2453722|2453995|N|N|N|N|N| +2454088|AAAAAAAAIECHFCAA|2006-12-18|1283|5581|429|2006|1|12|18|4|2006|429|5581|Monday|2006Q4|N|N|N|2454071|2454404|2453723|2453996|N|N|N|N|N| +2454089|AAAAAAAAJECHFCAA|2006-12-19|1283|5582|429|2006|2|12|19|4|2006|429|5582|Tuesday|2006Q4|N|N|N|2454071|2454404|2453724|2453997|N|N|N|N|N| +2454090|AAAAAAAAKECHFCAA|2006-12-20|1283|5582|429|2006|3|12|20|4|2006|429|5582|Wednesday|2006Q4|N|N|N|2454071|2454404|2453725|2453998|N|N|N|N|N| +2454091|AAAAAAAALECHFCAA|2006-12-21|1283|5582|429|2006|4|12|21|4|2006|429|5582|Thursday|2006Q4|N|N|N|2454071|2454404|2453726|2453999|N|N|N|N|N| +2454092|AAAAAAAAMECHFCAA|2006-12-22|1283|5582|429|2006|5|12|22|4|2006|429|5582|Friday|2006Q4|N|Y|N|2454071|2454404|2453727|2454000|N|N|N|N|N| +2454093|AAAAAAAANECHFCAA|2006-12-23|1283|5582|429|2006|6|12|23|4|2006|429|5582|Saturday|2006Q4|N|Y|N|2454071|2454404|2453728|2454001|N|N|N|N|N| +2454094|AAAAAAAAOECHFCAA|2006-12-24|1283|5582|429|2006|0|12|24|4|2006|429|5582|Sunday|2006Q4|N|N|N|2454071|2454404|2453729|2454002|N|N|N|N|N| +2454095|AAAAAAAAPECHFCAA|2006-12-25|1283|5582|429|2006|1|12|25|4|2006|429|5582|Monday|2006Q4|N|N|N|2454071|2454404|2453730|2454003|N|N|N|N|N| +2454096|AAAAAAAAAFCHFCAA|2006-12-26|1283|5583|429|2006|2|12|26|4|2006|429|5583|Tuesday|2006Q4|Y|N|N|2454071|2454404|2453731|2454004|N|N|N|N|N| +2454097|AAAAAAAABFCHFCAA|2006-12-27|1283|5583|429|2006|3|12|27|4|2006|429|5583|Wednesday|2006Q4|N|N|Y|2454071|2454404|2453732|2454005|N|N|N|N|N| +2454098|AAAAAAAACFCHFCAA|2006-12-28|1283|5583|429|2006|4|12|28|4|2006|429|5583|Thursday|2006Q4|N|N|N|2454071|2454404|2453733|2454006|N|N|N|N|N| +2454099|AAAAAAAADFCHFCAA|2006-12-29|1283|5583|429|2006|5|12|29|4|2006|429|5583|Friday|2006Q4|N|Y|N|2454071|2454404|2453734|2454007|N|N|N|N|N| +2454100|AAAAAAAAEFCHFCAA|2006-12-30|1283|5583|429|2006|6|12|30|4|2006|429|5583|Saturday|2006Q4|N|Y|N|2454071|2454404|2453735|2454008|N|N|N|N|N| +2454101|AAAAAAAAFFCHFCAA|2006-12-31|1283|5583|429|2006|0|12|31|4|2006|429|5583|Sunday|2006Q4|N|N|N|2454071|2454404|2453736|2454009|N|N|N|N|N| +2454102|AAAAAAAAGFCHFCAA|2007-01-01|1284|5583|429|2007|1|1|1|1|2007|429|5583|Monday|2007Q1|Y|N|N|2454102|2454101|2453737|2454010|N|N|N|N|N| +2454103|AAAAAAAAHFCHFCAA|2007-01-02|1284|5584|429|2007|2|1|2|1|2007|429|5584|Tuesday|2007Q1|N|N|Y|2454102|2454101|2453738|2454011|N|N|N|N|N| +2454104|AAAAAAAAIFCHFCAA|2007-01-03|1284|5584|429|2007|3|1|3|1|2007|429|5584|Wednesday|2007Q1|N|N|N|2454102|2454101|2453739|2454012|N|N|N|N|N| +2454105|AAAAAAAAJFCHFCAA|2007-01-04|1284|5584|429|2007|4|1|4|1|2007|429|5584|Thursday|2007Q1|N|N|N|2454102|2454101|2453740|2454013|N|N|N|N|N| +2454106|AAAAAAAAKFCHFCAA|2007-01-05|1284|5584|429|2007|5|1|5|1|2007|429|5584|Friday|2007Q1|N|Y|N|2454102|2454101|2453741|2454014|N|N|N|N|N| +2454107|AAAAAAAALFCHFCAA|2007-01-06|1284|5584|429|2007|6|1|6|1|2007|429|5584|Saturday|2007Q1|N|Y|N|2454102|2454101|2453742|2454015|N|N|N|N|N| +2454108|AAAAAAAAMFCHFCAA|2007-01-07|1284|5584|429|2007|0|1|7|1|2007|429|5584|Sunday|2007Q1|N|N|N|2454102|2454101|2453743|2454016|N|N|N|N|N| +2454109|AAAAAAAANFCHFCAA|2007-01-08|1284|5584|429|2007|1|1|8|1|2007|429|5584|Monday|2007Q1|N|N|N|2454102|2454101|2453744|2454017|N|N|N|N|N| +2454110|AAAAAAAAOFCHFCAA|2007-01-09|1284|5585|429|2007|2|1|9|1|2007|429|5585|Tuesday|2007Q1|N|N|N|2454102|2454101|2453745|2454018|N|N|N|N|N| +2454111|AAAAAAAAPFCHFCAA|2007-01-10|1284|5585|429|2007|3|1|10|1|2007|429|5585|Wednesday|2007Q1|N|N|N|2454102|2454101|2453746|2454019|N|N|N|N|N| +2454112|AAAAAAAAAGCHFCAA|2007-01-11|1284|5585|429|2007|4|1|11|1|2007|429|5585|Thursday|2007Q1|N|N|N|2454102|2454101|2453747|2454020|N|N|N|N|N| +2454113|AAAAAAAABGCHFCAA|2007-01-12|1284|5585|429|2007|5|1|12|1|2007|429|5585|Friday|2007Q1|N|Y|N|2454102|2454101|2453748|2454021|N|N|N|N|N| +2454114|AAAAAAAACGCHFCAA|2007-01-13|1284|5585|429|2007|6|1|13|1|2007|429|5585|Saturday|2007Q1|N|Y|N|2454102|2454101|2453749|2454022|N|N|N|N|N| +2454115|AAAAAAAADGCHFCAA|2007-01-14|1284|5585|429|2007|0|1|14|1|2007|429|5585|Sunday|2007Q1|N|N|N|2454102|2454101|2453750|2454023|N|N|N|N|N| +2454116|AAAAAAAAEGCHFCAA|2007-01-15|1284|5585|429|2007|1|1|15|1|2007|429|5585|Monday|2007Q1|N|N|N|2454102|2454101|2453751|2454024|N|N|N|N|N| +2454117|AAAAAAAAFGCHFCAA|2007-01-16|1284|5586|429|2007|2|1|16|1|2007|429|5586|Tuesday|2007Q1|N|N|N|2454102|2454101|2453752|2454025|N|N|N|N|N| +2454118|AAAAAAAAGGCHFCAA|2007-01-17|1284|5586|429|2007|3|1|17|1|2007|429|5586|Wednesday|2007Q1|N|N|N|2454102|2454101|2453753|2454026|N|N|N|N|N| +2454119|AAAAAAAAHGCHFCAA|2007-01-18|1284|5586|429|2007|4|1|18|1|2007|429|5586|Thursday|2007Q1|N|N|N|2454102|2454101|2453754|2454027|N|N|N|N|N| +2454120|AAAAAAAAIGCHFCAA|2007-01-19|1284|5586|429|2007|5|1|19|1|2007|429|5586|Friday|2007Q1|N|Y|N|2454102|2454101|2453755|2454028|N|N|N|N|N| +2454121|AAAAAAAAJGCHFCAA|2007-01-20|1284|5586|429|2007|6|1|20|1|2007|429|5586|Saturday|2007Q1|N|Y|N|2454102|2454101|2453756|2454029|N|N|N|N|N| +2454122|AAAAAAAAKGCHFCAA|2007-01-21|1284|5586|429|2007|0|1|21|1|2007|429|5586|Sunday|2007Q1|N|N|N|2454102|2454101|2453757|2454030|N|N|N|N|N| +2454123|AAAAAAAALGCHFCAA|2007-01-22|1284|5586|429|2007|1|1|22|1|2007|429|5586|Monday|2007Q1|N|N|N|2454102|2454101|2453758|2454031|N|N|N|N|N| +2454124|AAAAAAAAMGCHFCAA|2007-01-23|1284|5587|429|2007|2|1|23|1|2007|429|5587|Tuesday|2007Q1|N|N|N|2454102|2454101|2453759|2454032|N|N|N|N|N| +2454125|AAAAAAAANGCHFCAA|2007-01-24|1284|5587|429|2007|3|1|24|1|2007|429|5587|Wednesday|2007Q1|N|N|N|2454102|2454101|2453760|2454033|N|N|N|N|N| +2454126|AAAAAAAAOGCHFCAA|2007-01-25|1284|5587|429|2007|4|1|25|1|2007|429|5587|Thursday|2007Q1|N|N|N|2454102|2454101|2453761|2454034|N|N|N|N|N| +2454127|AAAAAAAAPGCHFCAA|2007-01-26|1284|5587|429|2007|5|1|26|1|2007|429|5587|Friday|2007Q1|N|Y|N|2454102|2454101|2453762|2454035|N|N|N|N|N| +2454128|AAAAAAAAAHCHFCAA|2007-01-27|1284|5587|429|2007|6|1|27|1|2007|429|5587|Saturday|2007Q1|N|Y|N|2454102|2454101|2453763|2454036|N|N|N|N|N| +2454129|AAAAAAAABHCHFCAA|2007-01-28|1284|5587|429|2007|0|1|28|1|2007|429|5587|Sunday|2007Q1|N|N|N|2454102|2454101|2453764|2454037|N|N|N|N|N| +2454130|AAAAAAAACHCHFCAA|2007-01-29|1284|5587|429|2007|1|1|29|1|2007|429|5587|Monday|2007Q1|N|N|N|2454102|2454101|2453765|2454038|N|N|N|N|N| +2454131|AAAAAAAADHCHFCAA|2007-01-30|1284|5588|429|2007|2|1|30|1|2007|429|5588|Tuesday|2007Q1|N|N|N|2454102|2454101|2453766|2454039|N|N|N|N|N| +2454132|AAAAAAAAEHCHFCAA|2007-01-31|1284|5588|429|2007|3|1|31|1|2007|429|5588|Wednesday|2007Q1|N|N|N|2454102|2454101|2453767|2454040|N|N|N|N|N| +2454133|AAAAAAAAFHCHFCAA|2007-02-01|1285|5588|429|2007|4|2|1|1|2007|429|5588|Thursday|2007Q1|N|N|N|2454133|2454163|2453768|2454041|N|N|N|N|N| +2454134|AAAAAAAAGHCHFCAA|2007-02-02|1285|5588|429|2007|5|2|2|1|2007|429|5588|Friday|2007Q1|N|Y|N|2454133|2454163|2453769|2454042|N|N|N|N|N| +2454135|AAAAAAAAHHCHFCAA|2007-02-03|1285|5588|429|2007|6|2|3|1|2007|429|5588|Saturday|2007Q1|N|Y|N|2454133|2454163|2453770|2454043|N|N|N|N|N| +2454136|AAAAAAAAIHCHFCAA|2007-02-04|1285|5588|429|2007|0|2|4|1|2007|429|5588|Sunday|2007Q1|N|N|N|2454133|2454163|2453771|2454044|N|N|N|N|N| +2454137|AAAAAAAAJHCHFCAA|2007-02-05|1285|5588|429|2007|1|2|5|1|2007|429|5588|Monday|2007Q1|N|N|N|2454133|2454163|2453772|2454045|N|N|N|N|N| +2454138|AAAAAAAAKHCHFCAA|2007-02-06|1285|5589|429|2007|2|2|6|1|2007|429|5589|Tuesday|2007Q1|N|N|N|2454133|2454163|2453773|2454046|N|N|N|N|N| +2454139|AAAAAAAALHCHFCAA|2007-02-07|1285|5589|429|2007|3|2|7|1|2007|429|5589|Wednesday|2007Q1|N|N|N|2454133|2454163|2453774|2454047|N|N|N|N|N| +2454140|AAAAAAAAMHCHFCAA|2007-02-08|1285|5589|429|2007|4|2|8|1|2007|429|5589|Thursday|2007Q1|N|N|N|2454133|2454163|2453775|2454048|N|N|N|N|N| +2454141|AAAAAAAANHCHFCAA|2007-02-09|1285|5589|429|2007|5|2|9|1|2007|429|5589|Friday|2007Q1|N|Y|N|2454133|2454163|2453776|2454049|N|N|N|N|N| +2454142|AAAAAAAAOHCHFCAA|2007-02-10|1285|5589|429|2007|6|2|10|1|2007|429|5589|Saturday|2007Q1|N|Y|N|2454133|2454163|2453777|2454050|N|N|N|N|N| +2454143|AAAAAAAAPHCHFCAA|2007-02-11|1285|5589|429|2007|0|2|11|1|2007|429|5589|Sunday|2007Q1|N|N|N|2454133|2454163|2453778|2454051|N|N|N|N|N| +2454144|AAAAAAAAAICHFCAA|2007-02-12|1285|5589|429|2007|1|2|12|1|2007|429|5589|Monday|2007Q1|N|N|N|2454133|2454163|2453779|2454052|N|N|N|N|N| +2454145|AAAAAAAABICHFCAA|2007-02-13|1285|5590|429|2007|2|2|13|1|2007|429|5590|Tuesday|2007Q1|N|N|N|2454133|2454163|2453780|2454053|N|N|N|N|N| +2454146|AAAAAAAACICHFCAA|2007-02-14|1285|5590|429|2007|3|2|14|1|2007|429|5590|Wednesday|2007Q1|N|N|N|2454133|2454163|2453781|2454054|N|N|N|N|N| +2454147|AAAAAAAADICHFCAA|2007-02-15|1285|5590|429|2007|4|2|15|1|2007|429|5590|Thursday|2007Q1|N|N|N|2454133|2454163|2453782|2454055|N|N|N|N|N| +2454148|AAAAAAAAEICHFCAA|2007-02-16|1285|5590|429|2007|5|2|16|1|2007|429|5590|Friday|2007Q1|N|Y|N|2454133|2454163|2453783|2454056|N|N|N|N|N| +2454149|AAAAAAAAFICHFCAA|2007-02-17|1285|5590|429|2007|6|2|17|1|2007|429|5590|Saturday|2007Q1|N|Y|N|2454133|2454163|2453784|2454057|N|N|N|N|N| +2454150|AAAAAAAAGICHFCAA|2007-02-18|1285|5590|429|2007|0|2|18|1|2007|429|5590|Sunday|2007Q1|N|N|N|2454133|2454163|2453785|2454058|N|N|N|N|N| +2454151|AAAAAAAAHICHFCAA|2007-02-19|1285|5590|429|2007|1|2|19|1|2007|429|5590|Monday|2007Q1|N|N|N|2454133|2454163|2453786|2454059|N|N|N|N|N| +2454152|AAAAAAAAIICHFCAA|2007-02-20|1285|5591|429|2007|2|2|20|1|2007|429|5591|Tuesday|2007Q1|N|N|N|2454133|2454163|2453787|2454060|N|N|N|N|N| +2454153|AAAAAAAAJICHFCAA|2007-02-21|1285|5591|429|2007|3|2|21|1|2007|429|5591|Wednesday|2007Q1|N|N|N|2454133|2454163|2453788|2454061|N|N|N|N|N| +2454154|AAAAAAAAKICHFCAA|2007-02-22|1285|5591|429|2007|4|2|22|1|2007|429|5591|Thursday|2007Q1|N|N|N|2454133|2454163|2453789|2454062|N|N|N|N|N| +2454155|AAAAAAAALICHFCAA|2007-02-23|1285|5591|429|2007|5|2|23|1|2007|429|5591|Friday|2007Q1|N|Y|N|2454133|2454163|2453790|2454063|N|N|N|N|N| +2454156|AAAAAAAAMICHFCAA|2007-02-24|1285|5591|429|2007|6|2|24|1|2007|429|5591|Saturday|2007Q1|N|Y|N|2454133|2454163|2453791|2454064|N|N|N|N|N| +2454157|AAAAAAAANICHFCAA|2007-02-25|1285|5591|429|2007|0|2|25|1|2007|429|5591|Sunday|2007Q1|N|N|N|2454133|2454163|2453792|2454065|N|N|N|N|N| +2454158|AAAAAAAAOICHFCAA|2007-02-26|1285|5591|429|2007|1|2|26|1|2007|429|5591|Monday|2007Q1|N|N|N|2454133|2454163|2453793|2454066|N|N|N|N|N| +2454159|AAAAAAAAPICHFCAA|2007-02-27|1285|5592|429|2007|2|2|27|1|2007|429|5592|Tuesday|2007Q1|N|N|N|2454133|2454163|2453794|2454067|N|N|N|N|N| +2454160|AAAAAAAAAJCHFCAA|2007-02-28|1285|5592|429|2007|3|2|28|1|2007|429|5592|Wednesday|2007Q1|N|N|N|2454133|2454163|2453795|2454068|N|N|N|N|N| +2454161|AAAAAAAABJCHFCAA|2007-03-01|1286|5592|430|2007|4|3|1|1|2007|430|5592|Thursday|2007Q1|N|N|N|2454161|2454219|2453796|2454069|N|N|N|N|N| +2454162|AAAAAAAACJCHFCAA|2007-03-02|1286|5592|430|2007|5|3|2|1|2007|430|5592|Friday|2007Q1|N|Y|N|2454161|2454219|2453797|2454070|N|N|N|N|N| +2454163|AAAAAAAADJCHFCAA|2007-03-03|1286|5592|430|2007|6|3|3|1|2007|430|5592|Saturday|2007Q1|N|Y|N|2454161|2454219|2453798|2454071|N|N|N|N|N| +2454164|AAAAAAAAEJCHFCAA|2007-03-04|1286|5592|430|2007|0|3|4|1|2007|430|5592|Sunday|2007Q1|N|N|N|2454161|2454219|2453799|2454072|N|N|N|N|N| +2454165|AAAAAAAAFJCHFCAA|2007-03-05|1286|5592|430|2007|1|3|5|1|2007|430|5592|Monday|2007Q1|N|N|N|2454161|2454219|2453800|2454073|N|N|N|N|N| +2454166|AAAAAAAAGJCHFCAA|2007-03-06|1286|5593|430|2007|2|3|6|1|2007|430|5593|Tuesday|2007Q1|N|N|N|2454161|2454219|2453801|2454074|N|N|N|N|N| +2454167|AAAAAAAAHJCHFCAA|2007-03-07|1286|5593|430|2007|3|3|7|1|2007|430|5593|Wednesday|2007Q1|N|N|N|2454161|2454219|2453802|2454075|N|N|N|N|N| +2454168|AAAAAAAAIJCHFCAA|2007-03-08|1286|5593|430|2007|4|3|8|1|2007|430|5593|Thursday|2007Q1|N|N|N|2454161|2454219|2453803|2454076|N|N|N|N|N| +2454169|AAAAAAAAJJCHFCAA|2007-03-09|1286|5593|430|2007|5|3|9|1|2007|430|5593|Friday|2007Q1|N|Y|N|2454161|2454219|2453804|2454077|N|N|N|N|N| +2454170|AAAAAAAAKJCHFCAA|2007-03-10|1286|5593|430|2007|6|3|10|1|2007|430|5593|Saturday|2007Q1|N|Y|N|2454161|2454219|2453805|2454078|N|N|N|N|N| +2454171|AAAAAAAALJCHFCAA|2007-03-11|1286|5593|430|2007|0|3|11|1|2007|430|5593|Sunday|2007Q1|N|N|N|2454161|2454219|2453806|2454079|N|N|N|N|N| +2454172|AAAAAAAAMJCHFCAA|2007-03-12|1286|5593|430|2007|1|3|12|1|2007|430|5593|Monday|2007Q1|N|N|N|2454161|2454219|2453807|2454080|N|N|N|N|N| +2454173|AAAAAAAANJCHFCAA|2007-03-13|1286|5594|430|2007|2|3|13|1|2007|430|5594|Tuesday|2007Q1|N|N|N|2454161|2454219|2453808|2454081|N|N|N|N|N| +2454174|AAAAAAAAOJCHFCAA|2007-03-14|1286|5594|430|2007|3|3|14|1|2007|430|5594|Wednesday|2007Q1|N|N|N|2454161|2454219|2453809|2454082|N|N|N|N|N| +2454175|AAAAAAAAPJCHFCAA|2007-03-15|1286|5594|430|2007|4|3|15|1|2007|430|5594|Thursday|2007Q1|N|N|N|2454161|2454219|2453810|2454083|N|N|N|N|N| +2454176|AAAAAAAAAKCHFCAA|2007-03-16|1286|5594|430|2007|5|3|16|1|2007|430|5594|Friday|2007Q1|N|Y|N|2454161|2454219|2453811|2454084|N|N|N|N|N| +2454177|AAAAAAAABKCHFCAA|2007-03-17|1286|5594|430|2007|6|3|17|1|2007|430|5594|Saturday|2007Q1|N|Y|N|2454161|2454219|2453812|2454085|N|N|N|N|N| +2454178|AAAAAAAACKCHFCAA|2007-03-18|1286|5594|430|2007|0|3|18|1|2007|430|5594|Sunday|2007Q1|N|N|N|2454161|2454219|2453813|2454086|N|N|N|N|N| +2454179|AAAAAAAADKCHFCAA|2007-03-19|1286|5594|430|2007|1|3|19|1|2007|430|5594|Monday|2007Q1|N|N|N|2454161|2454219|2453814|2454087|N|N|N|N|N| +2454180|AAAAAAAAEKCHFCAA|2007-03-20|1286|5595|430|2007|2|3|20|1|2007|430|5595|Tuesday|2007Q1|N|N|N|2454161|2454219|2453815|2454088|N|N|N|N|N| +2454181|AAAAAAAAFKCHFCAA|2007-03-21|1286|5595|430|2007|3|3|21|1|2007|430|5595|Wednesday|2007Q1|N|N|N|2454161|2454219|2453816|2454089|N|N|N|N|N| +2454182|AAAAAAAAGKCHFCAA|2007-03-22|1286|5595|430|2007|4|3|22|1|2007|430|5595|Thursday|2007Q1|N|N|N|2454161|2454219|2453817|2454090|N|N|N|N|N| +2454183|AAAAAAAAHKCHFCAA|2007-03-23|1286|5595|430|2007|5|3|23|1|2007|430|5595|Friday|2007Q1|N|Y|N|2454161|2454219|2453818|2454091|N|N|N|N|N| +2454184|AAAAAAAAIKCHFCAA|2007-03-24|1286|5595|430|2007|6|3|24|1|2007|430|5595|Saturday|2007Q1|N|Y|N|2454161|2454219|2453819|2454092|N|N|N|N|N| +2454185|AAAAAAAAJKCHFCAA|2007-03-25|1286|5595|430|2007|0|3|25|1|2007|430|5595|Sunday|2007Q1|N|N|N|2454161|2454219|2453820|2454093|N|N|N|N|N| +2454186|AAAAAAAAKKCHFCAA|2007-03-26|1286|5595|430|2007|1|3|26|1|2007|430|5595|Monday|2007Q1|N|N|N|2454161|2454219|2453821|2454094|N|N|N|N|N| +2454187|AAAAAAAALKCHFCAA|2007-03-27|1286|5596|430|2007|2|3|27|1|2007|430|5596|Tuesday|2007Q1|N|N|N|2454161|2454219|2453822|2454095|N|N|N|N|N| +2454188|AAAAAAAAMKCHFCAA|2007-03-28|1286|5596|430|2007|3|3|28|1|2007|430|5596|Wednesday|2007Q1|N|N|N|2454161|2454219|2453823|2454096|N|N|N|N|N| +2454189|AAAAAAAANKCHFCAA|2007-03-29|1286|5596|430|2007|4|3|29|1|2007|430|5596|Thursday|2007Q1|N|N|N|2454161|2454219|2453824|2454097|N|N|N|N|N| +2454190|AAAAAAAAOKCHFCAA|2007-03-30|1286|5596|430|2007|5|3|30|1|2007|430|5596|Friday|2007Q1|N|Y|N|2454161|2454219|2453825|2454098|N|N|N|N|N| +2454191|AAAAAAAAPKCHFCAA|2007-03-31|1286|5596|430|2007|6|3|31|1|2007|430|5596|Saturday|2007Q1|N|Y|N|2454161|2454219|2453826|2454099|N|N|N|N|N| +2454192|AAAAAAAAALCHFCAA|2007-04-01|1287|5596|430|2007|0|4|1|1|2007|430|5596|Sunday|2007Q1|N|N|N|2454192|2454281|2453827|2454102|N|N|N|N|N| +2454193|AAAAAAAABLCHFCAA|2007-04-02|1287|5596|430|2007|1|4|2|2|2007|430|5596|Monday|2007Q2|N|N|N|2454192|2454281|2453828|2454103|N|N|N|N|N| +2454194|AAAAAAAACLCHFCAA|2007-04-03|1287|5597|430|2007|2|4|3|2|2007|430|5597|Tuesday|2007Q2|N|N|N|2454192|2454281|2453829|2454104|N|N|N|N|N| +2454195|AAAAAAAADLCHFCAA|2007-04-04|1287|5597|430|2007|3|4|4|2|2007|430|5597|Wednesday|2007Q2|N|N|N|2454192|2454281|2453830|2454105|N|N|N|N|N| +2454196|AAAAAAAAELCHFCAA|2007-04-05|1287|5597|430|2007|4|4|5|2|2007|430|5597|Thursday|2007Q2|N|N|N|2454192|2454281|2453831|2454106|N|N|N|N|N| +2454197|AAAAAAAAFLCHFCAA|2007-04-06|1287|5597|430|2007|5|4|6|2|2007|430|5597|Friday|2007Q2|N|Y|N|2454192|2454281|2453832|2454107|N|N|N|N|N| +2454198|AAAAAAAAGLCHFCAA|2007-04-07|1287|5597|430|2007|6|4|7|2|2007|430|5597|Saturday|2007Q2|N|Y|N|2454192|2454281|2453833|2454108|N|N|N|N|N| +2454199|AAAAAAAAHLCHFCAA|2007-04-08|1287|5597|430|2007|0|4|8|2|2007|430|5597|Sunday|2007Q2|N|N|N|2454192|2454281|2453834|2454109|N|N|N|N|N| +2454200|AAAAAAAAILCHFCAA|2007-04-09|1287|5597|430|2007|1|4|9|2|2007|430|5597|Monday|2007Q2|N|N|N|2454192|2454281|2453835|2454110|N|N|N|N|N| +2454201|AAAAAAAAJLCHFCAA|2007-04-10|1287|5598|430|2007|2|4|10|2|2007|430|5598|Tuesday|2007Q2|N|N|N|2454192|2454281|2453836|2454111|N|N|N|N|N| +2454202|AAAAAAAAKLCHFCAA|2007-04-11|1287|5598|430|2007|3|4|11|2|2007|430|5598|Wednesday|2007Q2|N|N|N|2454192|2454281|2453837|2454112|N|N|N|N|N| +2454203|AAAAAAAALLCHFCAA|2007-04-12|1287|5598|430|2007|4|4|12|2|2007|430|5598|Thursday|2007Q2|N|N|N|2454192|2454281|2453838|2454113|N|N|N|N|N| +2454204|AAAAAAAAMLCHFCAA|2007-04-13|1287|5598|430|2007|5|4|13|2|2007|430|5598|Friday|2007Q2|N|Y|N|2454192|2454281|2453839|2454114|N|N|N|N|N| +2454205|AAAAAAAANLCHFCAA|2007-04-14|1287|5598|430|2007|6|4|14|2|2007|430|5598|Saturday|2007Q2|N|Y|N|2454192|2454281|2453840|2454115|N|N|N|N|N| +2454206|AAAAAAAAOLCHFCAA|2007-04-15|1287|5598|430|2007|0|4|15|2|2007|430|5598|Sunday|2007Q2|N|N|N|2454192|2454281|2453841|2454116|N|N|N|N|N| +2454207|AAAAAAAAPLCHFCAA|2007-04-16|1287|5598|430|2007|1|4|16|2|2007|430|5598|Monday|2007Q2|N|N|N|2454192|2454281|2453842|2454117|N|N|N|N|N| +2454208|AAAAAAAAAMCHFCAA|2007-04-17|1287|5599|430|2007|2|4|17|2|2007|430|5599|Tuesday|2007Q2|N|N|N|2454192|2454281|2453843|2454118|N|N|N|N|N| +2454209|AAAAAAAABMCHFCAA|2007-04-18|1287|5599|430|2007|3|4|18|2|2007|430|5599|Wednesday|2007Q2|N|N|N|2454192|2454281|2453844|2454119|N|N|N|N|N| +2454210|AAAAAAAACMCHFCAA|2007-04-19|1287|5599|430|2007|4|4|19|2|2007|430|5599|Thursday|2007Q2|N|N|N|2454192|2454281|2453845|2454120|N|N|N|N|N| +2454211|AAAAAAAADMCHFCAA|2007-04-20|1287|5599|430|2007|5|4|20|2|2007|430|5599|Friday|2007Q2|N|Y|N|2454192|2454281|2453846|2454121|N|N|N|N|N| +2454212|AAAAAAAAEMCHFCAA|2007-04-21|1287|5599|430|2007|6|4|21|2|2007|430|5599|Saturday|2007Q2|N|Y|N|2454192|2454281|2453847|2454122|N|N|N|N|N| +2454213|AAAAAAAAFMCHFCAA|2007-04-22|1287|5599|430|2007|0|4|22|2|2007|430|5599|Sunday|2007Q2|N|N|N|2454192|2454281|2453848|2454123|N|N|N|N|N| +2454214|AAAAAAAAGMCHFCAA|2007-04-23|1287|5599|430|2007|1|4|23|2|2007|430|5599|Monday|2007Q2|N|N|N|2454192|2454281|2453849|2454124|N|N|N|N|N| +2454215|AAAAAAAAHMCHFCAA|2007-04-24|1287|5600|430|2007|2|4|24|2|2007|430|5600|Tuesday|2007Q2|N|N|N|2454192|2454281|2453850|2454125|N|N|N|N|N| +2454216|AAAAAAAAIMCHFCAA|2007-04-25|1287|5600|430|2007|3|4|25|2|2007|430|5600|Wednesday|2007Q2|N|N|N|2454192|2454281|2453851|2454126|N|N|N|N|N| +2454217|AAAAAAAAJMCHFCAA|2007-04-26|1287|5600|430|2007|4|4|26|2|2007|430|5600|Thursday|2007Q2|N|N|N|2454192|2454281|2453852|2454127|N|N|N|N|N| +2454218|AAAAAAAAKMCHFCAA|2007-04-27|1287|5600|430|2007|5|4|27|2|2007|430|5600|Friday|2007Q2|N|Y|N|2454192|2454281|2453853|2454128|N|N|N|N|N| +2454219|AAAAAAAALMCHFCAA|2007-04-28|1287|5600|430|2007|6|4|28|2|2007|430|5600|Saturday|2007Q2|N|Y|N|2454192|2454281|2453854|2454129|N|N|N|N|N| +2454220|AAAAAAAAMMCHFCAA|2007-04-29|1287|5600|430|2007|0|4|29|2|2007|430|5600|Sunday|2007Q2|N|N|N|2454192|2454281|2453855|2454130|N|N|N|N|N| +2454221|AAAAAAAANMCHFCAA|2007-04-30|1287|5600|430|2007|1|4|30|2|2007|430|5600|Monday|2007Q2|N|N|N|2454192|2454281|2453856|2454131|N|N|N|N|N| +2454222|AAAAAAAAOMCHFCAA|2007-05-01|1288|5601|430|2007|2|5|1|2|2007|430|5601|Tuesday|2007Q2|N|N|N|2454222|2454341|2453857|2454132|N|N|N|N|N| +2454223|AAAAAAAAPMCHFCAA|2007-05-02|1288|5601|430|2007|3|5|2|2|2007|430|5601|Wednesday|2007Q2|N|N|N|2454222|2454341|2453858|2454133|N|N|N|N|N| +2454224|AAAAAAAAANCHFCAA|2007-05-03|1288|5601|430|2007|4|5|3|2|2007|430|5601|Thursday|2007Q2|N|N|N|2454222|2454341|2453859|2454134|N|N|N|N|N| +2454225|AAAAAAAABNCHFCAA|2007-05-04|1288|5601|430|2007|5|5|4|2|2007|430|5601|Friday|2007Q2|N|Y|N|2454222|2454341|2453860|2454135|N|N|N|N|N| +2454226|AAAAAAAACNCHFCAA|2007-05-05|1288|5601|430|2007|6|5|5|2|2007|430|5601|Saturday|2007Q2|N|Y|N|2454222|2454341|2453861|2454136|N|N|N|N|N| +2454227|AAAAAAAADNCHFCAA|2007-05-06|1288|5601|430|2007|0|5|6|2|2007|430|5601|Sunday|2007Q2|N|N|N|2454222|2454341|2453862|2454137|N|N|N|N|N| +2454228|AAAAAAAAENCHFCAA|2007-05-07|1288|5601|430|2007|1|5|7|2|2007|430|5601|Monday|2007Q2|N|N|N|2454222|2454341|2453863|2454138|N|N|N|N|N| +2454229|AAAAAAAAFNCHFCAA|2007-05-08|1288|5602|430|2007|2|5|8|2|2007|430|5602|Tuesday|2007Q2|N|N|N|2454222|2454341|2453864|2454139|N|N|N|N|N| +2454230|AAAAAAAAGNCHFCAA|2007-05-09|1288|5602|430|2007|3|5|9|2|2007|430|5602|Wednesday|2007Q2|N|N|N|2454222|2454341|2453865|2454140|N|N|N|N|N| +2454231|AAAAAAAAHNCHFCAA|2007-05-10|1288|5602|430|2007|4|5|10|2|2007|430|5602|Thursday|2007Q2|N|N|N|2454222|2454341|2453866|2454141|N|N|N|N|N| +2454232|AAAAAAAAINCHFCAA|2007-05-11|1288|5602|430|2007|5|5|11|2|2007|430|5602|Friday|2007Q2|N|Y|N|2454222|2454341|2453867|2454142|N|N|N|N|N| +2454233|AAAAAAAAJNCHFCAA|2007-05-12|1288|5602|430|2007|6|5|12|2|2007|430|5602|Saturday|2007Q2|N|Y|N|2454222|2454341|2453868|2454143|N|N|N|N|N| +2454234|AAAAAAAAKNCHFCAA|2007-05-13|1288|5602|430|2007|0|5|13|2|2007|430|5602|Sunday|2007Q2|N|N|N|2454222|2454341|2453869|2454144|N|N|N|N|N| +2454235|AAAAAAAALNCHFCAA|2007-05-14|1288|5602|430|2007|1|5|14|2|2007|430|5602|Monday|2007Q2|N|N|N|2454222|2454341|2453870|2454145|N|N|N|N|N| +2454236|AAAAAAAAMNCHFCAA|2007-05-15|1288|5603|430|2007|2|5|15|2|2007|430|5603|Tuesday|2007Q2|N|N|N|2454222|2454341|2453871|2454146|N|N|N|N|N| +2454237|AAAAAAAANNCHFCAA|2007-05-16|1288|5603|430|2007|3|5|16|2|2007|430|5603|Wednesday|2007Q2|N|N|N|2454222|2454341|2453872|2454147|N|N|N|N|N| +2454238|AAAAAAAAONCHFCAA|2007-05-17|1288|5603|430|2007|4|5|17|2|2007|430|5603|Thursday|2007Q2|N|N|N|2454222|2454341|2453873|2454148|N|N|N|N|N| +2454239|AAAAAAAAPNCHFCAA|2007-05-18|1288|5603|430|2007|5|5|18|2|2007|430|5603|Friday|2007Q2|N|Y|N|2454222|2454341|2453874|2454149|N|N|N|N|N| +2454240|AAAAAAAAAOCHFCAA|2007-05-19|1288|5603|430|2007|6|5|19|2|2007|430|5603|Saturday|2007Q2|N|Y|N|2454222|2454341|2453875|2454150|N|N|N|N|N| +2454241|AAAAAAAABOCHFCAA|2007-05-20|1288|5603|430|2007|0|5|20|2|2007|430|5603|Sunday|2007Q2|N|N|N|2454222|2454341|2453876|2454151|N|N|N|N|N| +2454242|AAAAAAAACOCHFCAA|2007-05-21|1288|5603|430|2007|1|5|21|2|2007|430|5603|Monday|2007Q2|N|N|N|2454222|2454341|2453877|2454152|N|N|N|N|N| +2454243|AAAAAAAADOCHFCAA|2007-05-22|1288|5604|430|2007|2|5|22|2|2007|430|5604|Tuesday|2007Q2|N|N|N|2454222|2454341|2453878|2454153|N|N|N|N|N| +2454244|AAAAAAAAEOCHFCAA|2007-05-23|1288|5604|430|2007|3|5|23|2|2007|430|5604|Wednesday|2007Q2|N|N|N|2454222|2454341|2453879|2454154|N|N|N|N|N| +2454245|AAAAAAAAFOCHFCAA|2007-05-24|1288|5604|430|2007|4|5|24|2|2007|430|5604|Thursday|2007Q2|N|N|N|2454222|2454341|2453880|2454155|N|N|N|N|N| +2454246|AAAAAAAAGOCHFCAA|2007-05-25|1288|5604|430|2007|5|5|25|2|2007|430|5604|Friday|2007Q2|N|Y|N|2454222|2454341|2453881|2454156|N|N|N|N|N| +2454247|AAAAAAAAHOCHFCAA|2007-05-26|1288|5604|430|2007|6|5|26|2|2007|430|5604|Saturday|2007Q2|N|Y|N|2454222|2454341|2453882|2454157|N|N|N|N|N| +2454248|AAAAAAAAIOCHFCAA|2007-05-27|1288|5604|430|2007|0|5|27|2|2007|430|5604|Sunday|2007Q2|N|N|N|2454222|2454341|2453883|2454158|N|N|N|N|N| +2454249|AAAAAAAAJOCHFCAA|2007-05-28|1288|5604|430|2007|1|5|28|2|2007|430|5604|Monday|2007Q2|N|N|N|2454222|2454341|2453884|2454159|N|N|N|N|N| +2454250|AAAAAAAAKOCHFCAA|2007-05-29|1288|5605|430|2007|2|5|29|2|2007|430|5605|Tuesday|2007Q2|N|N|N|2454222|2454341|2453885|2454160|N|N|N|N|N| +2454251|AAAAAAAALOCHFCAA|2007-05-30|1288|5605|430|2007|3|5|30|2|2007|430|5605|Wednesday|2007Q2|N|N|N|2454222|2454341|2453886|2454161|N|N|N|N|N| +2454252|AAAAAAAAMOCHFCAA|2007-05-31|1288|5605|430|2007|4|5|31|2|2007|430|5605|Thursday|2007Q2|N|N|N|2454222|2454341|2453887|2454162|N|N|N|N|N| +2454253|AAAAAAAANOCHFCAA|2007-06-01|1289|5605|431|2007|5|6|1|2|2007|431|5605|Friday|2007Q2|N|Y|N|2454253|2454403|2453888|2454163|N|N|N|N|N| +2454254|AAAAAAAAOOCHFCAA|2007-06-02|1289|5605|431|2007|6|6|2|2|2007|431|5605|Saturday|2007Q2|N|Y|N|2454253|2454403|2453889|2454164|N|N|N|N|N| +2454255|AAAAAAAAPOCHFCAA|2007-06-03|1289|5605|431|2007|0|6|3|2|2007|431|5605|Sunday|2007Q2|N|N|N|2454253|2454403|2453890|2454165|N|N|N|N|N| +2454256|AAAAAAAAAPCHFCAA|2007-06-04|1289|5605|431|2007|1|6|4|2|2007|431|5605|Monday|2007Q2|N|N|N|2454253|2454403|2453891|2454166|N|N|N|N|N| +2454257|AAAAAAAABPCHFCAA|2007-06-05|1289|5606|431|2007|2|6|5|2|2007|431|5606|Tuesday|2007Q2|N|N|N|2454253|2454403|2453892|2454167|N|N|N|N|N| +2454258|AAAAAAAACPCHFCAA|2007-06-06|1289|5606|431|2007|3|6|6|2|2007|431|5606|Wednesday|2007Q2|N|N|N|2454253|2454403|2453893|2454168|N|N|N|N|N| +2454259|AAAAAAAADPCHFCAA|2007-06-07|1289|5606|431|2007|4|6|7|2|2007|431|5606|Thursday|2007Q2|N|N|N|2454253|2454403|2453894|2454169|N|N|N|N|N| +2454260|AAAAAAAAEPCHFCAA|2007-06-08|1289|5606|431|2007|5|6|8|2|2007|431|5606|Friday|2007Q2|N|Y|N|2454253|2454403|2453895|2454170|N|N|N|N|N| +2454261|AAAAAAAAFPCHFCAA|2007-06-09|1289|5606|431|2007|6|6|9|2|2007|431|5606|Saturday|2007Q2|N|Y|N|2454253|2454403|2453896|2454171|N|N|N|N|N| +2454262|AAAAAAAAGPCHFCAA|2007-06-10|1289|5606|431|2007|0|6|10|2|2007|431|5606|Sunday|2007Q2|N|N|N|2454253|2454403|2453897|2454172|N|N|N|N|N| +2454263|AAAAAAAAHPCHFCAA|2007-06-11|1289|5606|431|2007|1|6|11|2|2007|431|5606|Monday|2007Q2|N|N|N|2454253|2454403|2453898|2454173|N|N|N|N|N| +2454264|AAAAAAAAIPCHFCAA|2007-06-12|1289|5607|431|2007|2|6|12|2|2007|431|5607|Tuesday|2007Q2|N|N|N|2454253|2454403|2453899|2454174|N|N|N|N|N| +2454265|AAAAAAAAJPCHFCAA|2007-06-13|1289|5607|431|2007|3|6|13|2|2007|431|5607|Wednesday|2007Q2|N|N|N|2454253|2454403|2453900|2454175|N|N|N|N|N| +2454266|AAAAAAAAKPCHFCAA|2007-06-14|1289|5607|431|2007|4|6|14|2|2007|431|5607|Thursday|2007Q2|N|N|N|2454253|2454403|2453901|2454176|N|N|N|N|N| +2454267|AAAAAAAALPCHFCAA|2007-06-15|1289|5607|431|2007|5|6|15|2|2007|431|5607|Friday|2007Q2|N|Y|N|2454253|2454403|2453902|2454177|N|N|N|N|N| +2454268|AAAAAAAAMPCHFCAA|2007-06-16|1289|5607|431|2007|6|6|16|2|2007|431|5607|Saturday|2007Q2|N|Y|N|2454253|2454403|2453903|2454178|N|N|N|N|N| +2454269|AAAAAAAANPCHFCAA|2007-06-17|1289|5607|431|2007|0|6|17|2|2007|431|5607|Sunday|2007Q2|N|N|N|2454253|2454403|2453904|2454179|N|N|N|N|N| +2454270|AAAAAAAAOPCHFCAA|2007-06-18|1289|5607|431|2007|1|6|18|2|2007|431|5607|Monday|2007Q2|N|N|N|2454253|2454403|2453905|2454180|N|N|N|N|N| +2454271|AAAAAAAAPPCHFCAA|2007-06-19|1289|5608|431|2007|2|6|19|2|2007|431|5608|Tuesday|2007Q2|N|N|N|2454253|2454403|2453906|2454181|N|N|N|N|N| +2454272|AAAAAAAAAADHFCAA|2007-06-20|1289|5608|431|2007|3|6|20|2|2007|431|5608|Wednesday|2007Q2|N|N|N|2454253|2454403|2453907|2454182|N|N|N|N|N| +2454273|AAAAAAAABADHFCAA|2007-06-21|1289|5608|431|2007|4|6|21|2|2007|431|5608|Thursday|2007Q2|N|N|N|2454253|2454403|2453908|2454183|N|N|N|N|N| +2454274|AAAAAAAACADHFCAA|2007-06-22|1289|5608|431|2007|5|6|22|2|2007|431|5608|Friday|2007Q2|N|Y|N|2454253|2454403|2453909|2454184|N|N|N|N|N| +2454275|AAAAAAAADADHFCAA|2007-06-23|1289|5608|431|2007|6|6|23|2|2007|431|5608|Saturday|2007Q2|N|Y|N|2454253|2454403|2453910|2454185|N|N|N|N|N| +2454276|AAAAAAAAEADHFCAA|2007-06-24|1289|5608|431|2007|0|6|24|2|2007|431|5608|Sunday|2007Q2|N|N|N|2454253|2454403|2453911|2454186|N|N|N|N|N| +2454277|AAAAAAAAFADHFCAA|2007-06-25|1289|5608|431|2007|1|6|25|2|2007|431|5608|Monday|2007Q2|N|N|N|2454253|2454403|2453912|2454187|N|N|N|N|N| +2454278|AAAAAAAAGADHFCAA|2007-06-26|1289|5609|431|2007|2|6|26|2|2007|431|5609|Tuesday|2007Q2|N|N|N|2454253|2454403|2453913|2454188|N|N|N|N|N| +2454279|AAAAAAAAHADHFCAA|2007-06-27|1289|5609|431|2007|3|6|27|2|2007|431|5609|Wednesday|2007Q2|N|N|N|2454253|2454403|2453914|2454189|N|N|N|N|N| +2454280|AAAAAAAAIADHFCAA|2007-06-28|1289|5609|431|2007|4|6|28|2|2007|431|5609|Thursday|2007Q2|N|N|N|2454253|2454403|2453915|2454190|N|N|N|N|N| +2454281|AAAAAAAAJADHFCAA|2007-06-29|1289|5609|431|2007|5|6|29|2|2007|431|5609|Friday|2007Q2|N|Y|N|2454253|2454403|2453916|2454191|N|N|N|N|N| +2454282|AAAAAAAAKADHFCAA|2007-06-30|1289|5609|431|2007|6|6|30|2|2007|431|5609|Saturday|2007Q2|N|Y|N|2454253|2454403|2453917|2454192|N|N|N|N|N| +2454283|AAAAAAAALADHFCAA|2007-07-01|1290|5609|431|2007|0|7|1|2|2007|431|5609|Sunday|2007Q2|N|N|N|2454283|2454463|2453918|2454192|N|N|N|N|N| +2454284|AAAAAAAAMADHFCAA|2007-07-02|1290|5609|431|2007|1|7|2|3|2007|431|5609|Monday|2007Q3|N|N|N|2454283|2454463|2453919|2454193|N|N|N|N|N| +2454285|AAAAAAAANADHFCAA|2007-07-03|1290|5610|431|2007|2|7|3|3|2007|431|5610|Tuesday|2007Q3|N|N|N|2454283|2454463|2453920|2454194|N|N|N|N|N| +2454286|AAAAAAAAOADHFCAA|2007-07-04|1290|5610|431|2007|3|7|4|3|2007|431|5610|Wednesday|2007Q3|N|N|N|2454283|2454463|2453921|2454195|N|N|N|N|N| +2454287|AAAAAAAAPADHFCAA|2007-07-05|1290|5610|431|2007|4|7|5|3|2007|431|5610|Thursday|2007Q3|Y|N|N|2454283|2454463|2453922|2454196|N|N|N|N|N| +2454288|AAAAAAAAABDHFCAA|2007-07-06|1290|5610|431|2007|5|7|6|3|2007|431|5610|Friday|2007Q3|N|Y|Y|2454283|2454463|2453923|2454197|N|N|N|N|N| +2454289|AAAAAAAABBDHFCAA|2007-07-07|1290|5610|431|2007|6|7|7|3|2007|431|5610|Saturday|2007Q3|N|Y|N|2454283|2454463|2453924|2454198|N|N|N|N|N| +2454290|AAAAAAAACBDHFCAA|2007-07-08|1290|5610|431|2007|0|7|8|3|2007|431|5610|Sunday|2007Q3|N|N|N|2454283|2454463|2453925|2454199|N|N|N|N|N| +2454291|AAAAAAAADBDHFCAA|2007-07-09|1290|5610|431|2007|1|7|9|3|2007|431|5610|Monday|2007Q3|N|N|N|2454283|2454463|2453926|2454200|N|N|N|N|N| +2454292|AAAAAAAAEBDHFCAA|2007-07-10|1290|5611|431|2007|2|7|10|3|2007|431|5611|Tuesday|2007Q3|N|N|N|2454283|2454463|2453927|2454201|N|N|N|N|N| +2454293|AAAAAAAAFBDHFCAA|2007-07-11|1290|5611|431|2007|3|7|11|3|2007|431|5611|Wednesday|2007Q3|N|N|N|2454283|2454463|2453928|2454202|N|N|N|N|N| +2454294|AAAAAAAAGBDHFCAA|2007-07-12|1290|5611|431|2007|4|7|12|3|2007|431|5611|Thursday|2007Q3|N|N|N|2454283|2454463|2453929|2454203|N|N|N|N|N| +2454295|AAAAAAAAHBDHFCAA|2007-07-13|1290|5611|431|2007|5|7|13|3|2007|431|5611|Friday|2007Q3|N|Y|N|2454283|2454463|2453930|2454204|N|N|N|N|N| +2454296|AAAAAAAAIBDHFCAA|2007-07-14|1290|5611|431|2007|6|7|14|3|2007|431|5611|Saturday|2007Q3|N|Y|N|2454283|2454463|2453931|2454205|N|N|N|N|N| +2454297|AAAAAAAAJBDHFCAA|2007-07-15|1290|5611|431|2007|0|7|15|3|2007|431|5611|Sunday|2007Q3|N|N|N|2454283|2454463|2453932|2454206|N|N|N|N|N| +2454298|AAAAAAAAKBDHFCAA|2007-07-16|1290|5611|431|2007|1|7|16|3|2007|431|5611|Monday|2007Q3|N|N|N|2454283|2454463|2453933|2454207|N|N|N|N|N| +2454299|AAAAAAAALBDHFCAA|2007-07-17|1290|5612|431|2007|2|7|17|3|2007|431|5612|Tuesday|2007Q3|N|N|N|2454283|2454463|2453934|2454208|N|N|N|N|N| +2454300|AAAAAAAAMBDHFCAA|2007-07-18|1290|5612|431|2007|3|7|18|3|2007|431|5612|Wednesday|2007Q3|N|N|N|2454283|2454463|2453935|2454209|N|N|N|N|N| +2454301|AAAAAAAANBDHFCAA|2007-07-19|1290|5612|431|2007|4|7|19|3|2007|431|5612|Thursday|2007Q3|N|N|N|2454283|2454463|2453936|2454210|N|N|N|N|N| +2454302|AAAAAAAAOBDHFCAA|2007-07-20|1290|5612|431|2007|5|7|20|3|2007|431|5612|Friday|2007Q3|N|Y|N|2454283|2454463|2453937|2454211|N|N|N|N|N| +2454303|AAAAAAAAPBDHFCAA|2007-07-21|1290|5612|431|2007|6|7|21|3|2007|431|5612|Saturday|2007Q3|N|Y|N|2454283|2454463|2453938|2454212|N|N|N|N|N| +2454304|AAAAAAAAACDHFCAA|2007-07-22|1290|5612|431|2007|0|7|22|3|2007|431|5612|Sunday|2007Q3|N|N|N|2454283|2454463|2453939|2454213|N|N|N|N|N| +2454305|AAAAAAAABCDHFCAA|2007-07-23|1290|5612|431|2007|1|7|23|3|2007|431|5612|Monday|2007Q3|N|N|N|2454283|2454463|2453940|2454214|N|N|N|N|N| +2454306|AAAAAAAACCDHFCAA|2007-07-24|1290|5613|431|2007|2|7|24|3|2007|431|5613|Tuesday|2007Q3|N|N|N|2454283|2454463|2453941|2454215|N|N|N|N|N| +2454307|AAAAAAAADCDHFCAA|2007-07-25|1290|5613|431|2007|3|7|25|3|2007|431|5613|Wednesday|2007Q3|N|N|N|2454283|2454463|2453942|2454216|N|N|N|N|N| +2454308|AAAAAAAAECDHFCAA|2007-07-26|1290|5613|431|2007|4|7|26|3|2007|431|5613|Thursday|2007Q3|N|N|N|2454283|2454463|2453943|2454217|N|N|N|N|N| +2454309|AAAAAAAAFCDHFCAA|2007-07-27|1290|5613|431|2007|5|7|27|3|2007|431|5613|Friday|2007Q3|N|Y|N|2454283|2454463|2453944|2454218|N|N|N|N|N| +2454310|AAAAAAAAGCDHFCAA|2007-07-28|1290|5613|431|2007|6|7|28|3|2007|431|5613|Saturday|2007Q3|N|Y|N|2454283|2454463|2453945|2454219|N|N|N|N|N| +2454311|AAAAAAAAHCDHFCAA|2007-07-29|1290|5613|431|2007|0|7|29|3|2007|431|5613|Sunday|2007Q3|N|N|N|2454283|2454463|2453946|2454220|N|N|N|N|N| +2454312|AAAAAAAAICDHFCAA|2007-07-30|1290|5613|431|2007|1|7|30|3|2007|431|5613|Monday|2007Q3|N|N|N|2454283|2454463|2453947|2454221|N|N|N|N|N| +2454313|AAAAAAAAJCDHFCAA|2007-07-31|1290|5614|431|2007|2|7|31|3|2007|431|5614|Tuesday|2007Q3|N|N|N|2454283|2454463|2453948|2454222|N|N|N|N|N| +2454314|AAAAAAAAKCDHFCAA|2007-08-01|1291|5614|431|2007|3|8|1|3|2007|431|5614|Wednesday|2007Q3|N|N|N|2454314|2454525|2453949|2454223|N|N|N|N|N| +2454315|AAAAAAAALCDHFCAA|2007-08-02|1291|5614|431|2007|4|8|2|3|2007|431|5614|Thursday|2007Q3|N|N|N|2454314|2454525|2453950|2454224|N|N|N|N|N| +2454316|AAAAAAAAMCDHFCAA|2007-08-03|1291|5614|431|2007|5|8|3|3|2007|431|5614|Friday|2007Q3|N|Y|N|2454314|2454525|2453951|2454225|N|N|N|N|N| +2454317|AAAAAAAANCDHFCAA|2007-08-04|1291|5614|431|2007|6|8|4|3|2007|431|5614|Saturday|2007Q3|N|Y|N|2454314|2454525|2453952|2454226|N|N|N|N|N| +2454318|AAAAAAAAOCDHFCAA|2007-08-05|1291|5614|431|2007|0|8|5|3|2007|431|5614|Sunday|2007Q3|N|N|N|2454314|2454525|2453953|2454227|N|N|N|N|N| +2454319|AAAAAAAAPCDHFCAA|2007-08-06|1291|5614|431|2007|1|8|6|3|2007|431|5614|Monday|2007Q3|N|N|N|2454314|2454525|2453954|2454228|N|N|N|N|N| +2454320|AAAAAAAAADDHFCAA|2007-08-07|1291|5615|431|2007|2|8|7|3|2007|431|5615|Tuesday|2007Q3|N|N|N|2454314|2454525|2453955|2454229|N|N|N|N|N| +2454321|AAAAAAAABDDHFCAA|2007-08-08|1291|5615|431|2007|3|8|8|3|2007|431|5615|Wednesday|2007Q3|N|N|N|2454314|2454525|2453956|2454230|N|N|N|N|N| +2454322|AAAAAAAACDDHFCAA|2007-08-09|1291|5615|431|2007|4|8|9|3|2007|431|5615|Thursday|2007Q3|N|N|N|2454314|2454525|2453957|2454231|N|N|N|N|N| +2454323|AAAAAAAADDDHFCAA|2007-08-10|1291|5615|431|2007|5|8|10|3|2007|431|5615|Friday|2007Q3|N|Y|N|2454314|2454525|2453958|2454232|N|N|N|N|N| +2454324|AAAAAAAAEDDHFCAA|2007-08-11|1291|5615|431|2007|6|8|11|3|2007|431|5615|Saturday|2007Q3|N|Y|N|2454314|2454525|2453959|2454233|N|N|N|N|N| +2454325|AAAAAAAAFDDHFCAA|2007-08-12|1291|5615|431|2007|0|8|12|3|2007|431|5615|Sunday|2007Q3|N|N|N|2454314|2454525|2453960|2454234|N|N|N|N|N| +2454326|AAAAAAAAGDDHFCAA|2007-08-13|1291|5615|431|2007|1|8|13|3|2007|431|5615|Monday|2007Q3|N|N|N|2454314|2454525|2453961|2454235|N|N|N|N|N| +2454327|AAAAAAAAHDDHFCAA|2007-08-14|1291|5616|431|2007|2|8|14|3|2007|431|5616|Tuesday|2007Q3|N|N|N|2454314|2454525|2453962|2454236|N|N|N|N|N| +2454328|AAAAAAAAIDDHFCAA|2007-08-15|1291|5616|431|2007|3|8|15|3|2007|431|5616|Wednesday|2007Q3|N|N|N|2454314|2454525|2453963|2454237|N|N|N|N|N| +2454329|AAAAAAAAJDDHFCAA|2007-08-16|1291|5616|431|2007|4|8|16|3|2007|431|5616|Thursday|2007Q3|N|N|N|2454314|2454525|2453964|2454238|N|N|N|N|N| +2454330|AAAAAAAAKDDHFCAA|2007-08-17|1291|5616|431|2007|5|8|17|3|2007|431|5616|Friday|2007Q3|N|Y|N|2454314|2454525|2453965|2454239|N|N|N|N|N| +2454331|AAAAAAAALDDHFCAA|2007-08-18|1291|5616|431|2007|6|8|18|3|2007|431|5616|Saturday|2007Q3|N|Y|N|2454314|2454525|2453966|2454240|N|N|N|N|N| +2454332|AAAAAAAAMDDHFCAA|2007-08-19|1291|5616|431|2007|0|8|19|3|2007|431|5616|Sunday|2007Q3|N|N|N|2454314|2454525|2453967|2454241|N|N|N|N|N| +2454333|AAAAAAAANDDHFCAA|2007-08-20|1291|5616|431|2007|1|8|20|3|2007|431|5616|Monday|2007Q3|N|N|N|2454314|2454525|2453968|2454242|N|N|N|N|N| +2454334|AAAAAAAAODDHFCAA|2007-08-21|1291|5617|431|2007|2|8|21|3|2007|431|5617|Tuesday|2007Q3|N|N|N|2454314|2454525|2453969|2454243|N|N|N|N|N| +2454335|AAAAAAAAPDDHFCAA|2007-08-22|1291|5617|431|2007|3|8|22|3|2007|431|5617|Wednesday|2007Q3|N|N|N|2454314|2454525|2453970|2454244|N|N|N|N|N| +2454336|AAAAAAAAAEDHFCAA|2007-08-23|1291|5617|431|2007|4|8|23|3|2007|431|5617|Thursday|2007Q3|N|N|N|2454314|2454525|2453971|2454245|N|N|N|N|N| +2454337|AAAAAAAABEDHFCAA|2007-08-24|1291|5617|431|2007|5|8|24|3|2007|431|5617|Friday|2007Q3|N|Y|N|2454314|2454525|2453972|2454246|N|N|N|N|N| +2454338|AAAAAAAACEDHFCAA|2007-08-25|1291|5617|431|2007|6|8|25|3|2007|431|5617|Saturday|2007Q3|N|Y|N|2454314|2454525|2453973|2454247|N|N|N|N|N| +2454339|AAAAAAAADEDHFCAA|2007-08-26|1291|5617|431|2007|0|8|26|3|2007|431|5617|Sunday|2007Q3|N|N|N|2454314|2454525|2453974|2454248|N|N|N|N|N| +2454340|AAAAAAAAEEDHFCAA|2007-08-27|1291|5617|431|2007|1|8|27|3|2007|431|5617|Monday|2007Q3|N|N|N|2454314|2454525|2453975|2454249|N|N|N|N|N| +2454341|AAAAAAAAFEDHFCAA|2007-08-28|1291|5618|431|2007|2|8|28|3|2007|431|5618|Tuesday|2007Q3|N|N|N|2454314|2454525|2453976|2454250|N|N|N|N|N| +2454342|AAAAAAAAGEDHFCAA|2007-08-29|1291|5618|431|2007|3|8|29|3|2007|431|5618|Wednesday|2007Q3|N|N|N|2454314|2454525|2453977|2454251|N|N|N|N|N| +2454343|AAAAAAAAHEDHFCAA|2007-08-30|1291|5618|431|2007|4|8|30|3|2007|431|5618|Thursday|2007Q3|N|N|N|2454314|2454525|2453978|2454252|N|N|N|N|N| +2454344|AAAAAAAAIEDHFCAA|2007-08-31|1291|5618|431|2007|5|8|31|3|2007|431|5618|Friday|2007Q3|N|Y|N|2454314|2454525|2453979|2454253|N|N|N|N|N| +2454345|AAAAAAAAJEDHFCAA|2007-09-01|1292|5618|432|2007|6|9|1|3|2007|432|5618|Saturday|2007Q3|N|Y|N|2454345|2454587|2453980|2454254|N|N|N|N|N| +2454346|AAAAAAAAKEDHFCAA|2007-09-02|1292|5618|432|2007|0|9|2|3|2007|432|5618|Sunday|2007Q3|N|N|N|2454345|2454587|2453981|2454255|N|N|N|N|N| +2454347|AAAAAAAALEDHFCAA|2007-09-03|1292|5618|432|2007|1|9|3|3|2007|432|5618|Monday|2007Q3|N|N|N|2454345|2454587|2453982|2454256|N|N|N|N|N| +2454348|AAAAAAAAMEDHFCAA|2007-09-04|1292|5619|432|2007|2|9|4|3|2007|432|5619|Tuesday|2007Q3|N|N|N|2454345|2454587|2453983|2454257|N|N|N|N|N| +2454349|AAAAAAAANEDHFCAA|2007-09-05|1292|5619|432|2007|3|9|5|3|2007|432|5619|Wednesday|2007Q3|N|N|N|2454345|2454587|2453984|2454258|N|N|N|N|N| +2454350|AAAAAAAAOEDHFCAA|2007-09-06|1292|5619|432|2007|4|9|6|3|2007|432|5619|Thursday|2007Q3|N|N|N|2454345|2454587|2453985|2454259|N|N|N|N|N| +2454351|AAAAAAAAPEDHFCAA|2007-09-07|1292|5619|432|2007|5|9|7|3|2007|432|5619|Friday|2007Q3|N|Y|N|2454345|2454587|2453986|2454260|N|N|N|N|N| +2454352|AAAAAAAAAFDHFCAA|2007-09-08|1292|5619|432|2007|6|9|8|3|2007|432|5619|Saturday|2007Q3|N|Y|N|2454345|2454587|2453987|2454261|N|N|N|N|N| +2454353|AAAAAAAABFDHFCAA|2007-09-09|1292|5619|432|2007|0|9|9|3|2007|432|5619|Sunday|2007Q3|N|N|N|2454345|2454587|2453988|2454262|N|N|N|N|N| +2454354|AAAAAAAACFDHFCAA|2007-09-10|1292|5619|432|2007|1|9|10|3|2007|432|5619|Monday|2007Q3|N|N|N|2454345|2454587|2453989|2454263|N|N|N|N|N| +2454355|AAAAAAAADFDHFCAA|2007-09-11|1292|5620|432|2007|2|9|11|3|2007|432|5620|Tuesday|2007Q3|N|N|N|2454345|2454587|2453990|2454264|N|N|N|N|N| +2454356|AAAAAAAAEFDHFCAA|2007-09-12|1292|5620|432|2007|3|9|12|3|2007|432|5620|Wednesday|2007Q3|N|N|N|2454345|2454587|2453991|2454265|N|N|N|N|N| +2454357|AAAAAAAAFFDHFCAA|2007-09-13|1292|5620|432|2007|4|9|13|3|2007|432|5620|Thursday|2007Q3|N|N|N|2454345|2454587|2453992|2454266|N|N|N|N|N| +2454358|AAAAAAAAGFDHFCAA|2007-09-14|1292|5620|432|2007|5|9|14|3|2007|432|5620|Friday|2007Q3|N|Y|N|2454345|2454587|2453993|2454267|N|N|N|N|N| +2454359|AAAAAAAAHFDHFCAA|2007-09-15|1292|5620|432|2007|6|9|15|3|2007|432|5620|Saturday|2007Q3|N|Y|N|2454345|2454587|2453994|2454268|N|N|N|N|N| +2454360|AAAAAAAAIFDHFCAA|2007-09-16|1292|5620|432|2007|0|9|16|3|2007|432|5620|Sunday|2007Q3|N|N|N|2454345|2454587|2453995|2454269|N|N|N|N|N| +2454361|AAAAAAAAJFDHFCAA|2007-09-17|1292|5620|432|2007|1|9|17|3|2007|432|5620|Monday|2007Q3|N|N|N|2454345|2454587|2453996|2454270|N|N|N|N|N| +2454362|AAAAAAAAKFDHFCAA|2007-09-18|1292|5621|432|2007|2|9|18|3|2007|432|5621|Tuesday|2007Q3|N|N|N|2454345|2454587|2453997|2454271|N|N|N|N|N| +2454363|AAAAAAAALFDHFCAA|2007-09-19|1292|5621|432|2007|3|9|19|3|2007|432|5621|Wednesday|2007Q3|N|N|N|2454345|2454587|2453998|2454272|N|N|N|N|N| +2454364|AAAAAAAAMFDHFCAA|2007-09-20|1292|5621|432|2007|4|9|20|3|2007|432|5621|Thursday|2007Q3|N|N|N|2454345|2454587|2453999|2454273|N|N|N|N|N| +2454365|AAAAAAAANFDHFCAA|2007-09-21|1292|5621|432|2007|5|9|21|3|2007|432|5621|Friday|2007Q3|N|Y|N|2454345|2454587|2454000|2454274|N|N|N|N|N| +2454366|AAAAAAAAOFDHFCAA|2007-09-22|1292|5621|432|2007|6|9|22|3|2007|432|5621|Saturday|2007Q3|N|Y|N|2454345|2454587|2454001|2454275|N|N|N|N|N| +2454367|AAAAAAAAPFDHFCAA|2007-09-23|1292|5621|432|2007|0|9|23|3|2007|432|5621|Sunday|2007Q3|N|N|N|2454345|2454587|2454002|2454276|N|N|N|N|N| +2454368|AAAAAAAAAGDHFCAA|2007-09-24|1292|5621|432|2007|1|9|24|3|2007|432|5621|Monday|2007Q3|N|N|N|2454345|2454587|2454003|2454277|N|N|N|N|N| +2454369|AAAAAAAABGDHFCAA|2007-09-25|1292|5622|432|2007|2|9|25|3|2007|432|5622|Tuesday|2007Q3|N|N|N|2454345|2454587|2454004|2454278|N|N|N|N|N| +2454370|AAAAAAAACGDHFCAA|2007-09-26|1292|5622|432|2007|3|9|26|3|2007|432|5622|Wednesday|2007Q3|N|N|N|2454345|2454587|2454005|2454279|N|N|N|N|N| +2454371|AAAAAAAADGDHFCAA|2007-09-27|1292|5622|432|2007|4|9|27|3|2007|432|5622|Thursday|2007Q3|N|N|N|2454345|2454587|2454006|2454280|N|N|N|N|N| +2454372|AAAAAAAAEGDHFCAA|2007-09-28|1292|5622|432|2007|5|9|28|3|2007|432|5622|Friday|2007Q3|N|Y|N|2454345|2454587|2454007|2454281|N|N|N|N|N| +2454373|AAAAAAAAFGDHFCAA|2007-09-29|1292|5622|432|2007|6|9|29|3|2007|432|5622|Saturday|2007Q3|N|Y|N|2454345|2454587|2454008|2454282|N|N|N|N|N| +2454374|AAAAAAAAGGDHFCAA|2007-09-30|1292|5622|432|2007|0|9|30|3|2007|432|5622|Sunday|2007Q3|N|N|N|2454345|2454587|2454009|2454283|N|N|N|N|N| +2454375|AAAAAAAAHGDHFCAA|2007-10-01|1293|5622|432|2007|1|10|1|3|2007|432|5622|Monday|2007Q3|N|N|N|2454375|2454647|2454010|2454283|N|N|N|N|N| +2454376|AAAAAAAAIGDHFCAA|2007-10-02|1293|5623|432|2007|2|10|2|4|2007|432|5623|Tuesday|2007Q4|N|N|N|2454375|2454647|2454011|2454284|N|N|N|N|N| +2454377|AAAAAAAAJGDHFCAA|2007-10-03|1293|5623|432|2007|3|10|3|4|2007|432|5623|Wednesday|2007Q4|N|N|N|2454375|2454647|2454012|2454285|N|N|N|N|N| +2454378|AAAAAAAAKGDHFCAA|2007-10-04|1293|5623|432|2007|4|10|4|4|2007|432|5623|Thursday|2007Q4|N|N|N|2454375|2454647|2454013|2454286|N|N|N|N|N| +2454379|AAAAAAAALGDHFCAA|2007-10-05|1293|5623|432|2007|5|10|5|4|2007|432|5623|Friday|2007Q4|N|Y|N|2454375|2454647|2454014|2454287|N|N|N|N|N| +2454380|AAAAAAAAMGDHFCAA|2007-10-06|1293|5623|432|2007|6|10|6|4|2007|432|5623|Saturday|2007Q4|N|Y|N|2454375|2454647|2454015|2454288|N|N|N|N|N| +2454381|AAAAAAAANGDHFCAA|2007-10-07|1293|5623|432|2007|0|10|7|4|2007|432|5623|Sunday|2007Q4|N|N|N|2454375|2454647|2454016|2454289|N|N|N|N|N| +2454382|AAAAAAAAOGDHFCAA|2007-10-08|1293|5623|432|2007|1|10|8|4|2007|432|5623|Monday|2007Q4|N|N|N|2454375|2454647|2454017|2454290|N|N|N|N|N| +2454383|AAAAAAAAPGDHFCAA|2007-10-09|1293|5624|432|2007|2|10|9|4|2007|432|5624|Tuesday|2007Q4|N|N|N|2454375|2454647|2454018|2454291|N|N|N|N|N| +2454384|AAAAAAAAAHDHFCAA|2007-10-10|1293|5624|432|2007|3|10|10|4|2007|432|5624|Wednesday|2007Q4|N|N|N|2454375|2454647|2454019|2454292|N|N|N|N|N| +2454385|AAAAAAAABHDHFCAA|2007-10-11|1293|5624|432|2007|4|10|11|4|2007|432|5624|Thursday|2007Q4|N|N|N|2454375|2454647|2454020|2454293|N|N|N|N|N| +2454386|AAAAAAAACHDHFCAA|2007-10-12|1293|5624|432|2007|5|10|12|4|2007|432|5624|Friday|2007Q4|N|Y|N|2454375|2454647|2454021|2454294|N|N|N|N|N| +2454387|AAAAAAAADHDHFCAA|2007-10-13|1293|5624|432|2007|6|10|13|4|2007|432|5624|Saturday|2007Q4|N|Y|N|2454375|2454647|2454022|2454295|N|N|N|N|N| +2454388|AAAAAAAAEHDHFCAA|2007-10-14|1293|5624|432|2007|0|10|14|4|2007|432|5624|Sunday|2007Q4|N|N|N|2454375|2454647|2454023|2454296|N|N|N|N|N| +2454389|AAAAAAAAFHDHFCAA|2007-10-15|1293|5624|432|2007|1|10|15|4|2007|432|5624|Monday|2007Q4|N|N|N|2454375|2454647|2454024|2454297|N|N|N|N|N| +2454390|AAAAAAAAGHDHFCAA|2007-10-16|1293|5625|432|2007|2|10|16|4|2007|432|5625|Tuesday|2007Q4|N|N|N|2454375|2454647|2454025|2454298|N|N|N|N|N| +2454391|AAAAAAAAHHDHFCAA|2007-10-17|1293|5625|432|2007|3|10|17|4|2007|432|5625|Wednesday|2007Q4|N|N|N|2454375|2454647|2454026|2454299|N|N|N|N|N| +2454392|AAAAAAAAIHDHFCAA|2007-10-18|1293|5625|432|2007|4|10|18|4|2007|432|5625|Thursday|2007Q4|N|N|N|2454375|2454647|2454027|2454300|N|N|N|N|N| +2454393|AAAAAAAAJHDHFCAA|2007-10-19|1293|5625|432|2007|5|10|19|4|2007|432|5625|Friday|2007Q4|N|Y|N|2454375|2454647|2454028|2454301|N|N|N|N|N| +2454394|AAAAAAAAKHDHFCAA|2007-10-20|1293|5625|432|2007|6|10|20|4|2007|432|5625|Saturday|2007Q4|N|Y|N|2454375|2454647|2454029|2454302|N|N|N|N|N| +2454395|AAAAAAAALHDHFCAA|2007-10-21|1293|5625|432|2007|0|10|21|4|2007|432|5625|Sunday|2007Q4|N|N|N|2454375|2454647|2454030|2454303|N|N|N|N|N| +2454396|AAAAAAAAMHDHFCAA|2007-10-22|1293|5625|432|2007|1|10|22|4|2007|432|5625|Monday|2007Q4|N|N|N|2454375|2454647|2454031|2454304|N|N|N|N|N| +2454397|AAAAAAAANHDHFCAA|2007-10-23|1293|5626|432|2007|2|10|23|4|2007|432|5626|Tuesday|2007Q4|N|N|N|2454375|2454647|2454032|2454305|N|N|N|N|N| +2454398|AAAAAAAAOHDHFCAA|2007-10-24|1293|5626|432|2007|3|10|24|4|2007|432|5626|Wednesday|2007Q4|N|N|N|2454375|2454647|2454033|2454306|N|N|N|N|N| +2454399|AAAAAAAAPHDHFCAA|2007-10-25|1293|5626|432|2007|4|10|25|4|2007|432|5626|Thursday|2007Q4|N|N|N|2454375|2454647|2454034|2454307|N|N|N|N|N| +2454400|AAAAAAAAAIDHFCAA|2007-10-26|1293|5626|432|2007|5|10|26|4|2007|432|5626|Friday|2007Q4|N|Y|N|2454375|2454647|2454035|2454308|N|N|N|N|N| +2454401|AAAAAAAABIDHFCAA|2007-10-27|1293|5626|432|2007|6|10|27|4|2007|432|5626|Saturday|2007Q4|N|Y|N|2454375|2454647|2454036|2454309|N|N|N|N|N| +2454402|AAAAAAAACIDHFCAA|2007-10-28|1293|5626|432|2007|0|10|28|4|2007|432|5626|Sunday|2007Q4|N|N|N|2454375|2454647|2454037|2454310|N|N|N|N|N| +2454403|AAAAAAAADIDHFCAA|2007-10-29|1293|5626|432|2007|1|10|29|4|2007|432|5626|Monday|2007Q4|N|N|N|2454375|2454647|2454038|2454311|N|N|N|N|N| +2454404|AAAAAAAAEIDHFCAA|2007-10-30|1293|5627|432|2007|2|10|30|4|2007|432|5627|Tuesday|2007Q4|N|N|N|2454375|2454647|2454039|2454312|N|N|N|N|N| +2454405|AAAAAAAAFIDHFCAA|2007-10-31|1293|5627|432|2007|3|10|31|4|2007|432|5627|Wednesday|2007Q4|N|N|N|2454375|2454647|2454040|2454313|N|N|N|N|N| +2454406|AAAAAAAAGIDHFCAA|2007-11-01|1294|5627|432|2007|4|11|1|4|2007|432|5627|Thursday|2007Q4|N|N|N|2454406|2454709|2454041|2454314|N|N|N|N|N| +2454407|AAAAAAAAHIDHFCAA|2007-11-02|1294|5627|432|2007|5|11|2|4|2007|432|5627|Friday|2007Q4|N|Y|N|2454406|2454709|2454042|2454315|N|N|N|N|N| +2454408|AAAAAAAAIIDHFCAA|2007-11-03|1294|5627|432|2007|6|11|3|4|2007|432|5627|Saturday|2007Q4|N|Y|N|2454406|2454709|2454043|2454316|N|N|N|N|N| +2454409|AAAAAAAAJIDHFCAA|2007-11-04|1294|5627|432|2007|0|11|4|4|2007|432|5627|Sunday|2007Q4|N|N|N|2454406|2454709|2454044|2454317|N|N|N|N|N| +2454410|AAAAAAAAKIDHFCAA|2007-11-05|1294|5627|432|2007|1|11|5|4|2007|432|5627|Monday|2007Q4|N|N|N|2454406|2454709|2454045|2454318|N|N|N|N|N| +2454411|AAAAAAAALIDHFCAA|2007-11-06|1294|5628|432|2007|2|11|6|4|2007|432|5628|Tuesday|2007Q4|N|N|N|2454406|2454709|2454046|2454319|N|N|N|N|N| +2454412|AAAAAAAAMIDHFCAA|2007-11-07|1294|5628|432|2007|3|11|7|4|2007|432|5628|Wednesday|2007Q4|N|N|N|2454406|2454709|2454047|2454320|N|N|N|N|N| +2454413|AAAAAAAANIDHFCAA|2007-11-08|1294|5628|432|2007|4|11|8|4|2007|432|5628|Thursday|2007Q4|N|N|N|2454406|2454709|2454048|2454321|N|N|N|N|N| +2454414|AAAAAAAAOIDHFCAA|2007-11-09|1294|5628|432|2007|5|11|9|4|2007|432|5628|Friday|2007Q4|N|Y|N|2454406|2454709|2454049|2454322|N|N|N|N|N| +2454415|AAAAAAAAPIDHFCAA|2007-11-10|1294|5628|432|2007|6|11|10|4|2007|432|5628|Saturday|2007Q4|N|Y|N|2454406|2454709|2454050|2454323|N|N|N|N|N| +2454416|AAAAAAAAAJDHFCAA|2007-11-11|1294|5628|432|2007|0|11|11|4|2007|432|5628|Sunday|2007Q4|N|N|N|2454406|2454709|2454051|2454324|N|N|N|N|N| +2454417|AAAAAAAABJDHFCAA|2007-11-12|1294|5628|432|2007|1|11|12|4|2007|432|5628|Monday|2007Q4|N|N|N|2454406|2454709|2454052|2454325|N|N|N|N|N| +2454418|AAAAAAAACJDHFCAA|2007-11-13|1294|5629|432|2007|2|11|13|4|2007|432|5629|Tuesday|2007Q4|N|N|N|2454406|2454709|2454053|2454326|N|N|N|N|N| +2454419|AAAAAAAADJDHFCAA|2007-11-14|1294|5629|432|2007|3|11|14|4|2007|432|5629|Wednesday|2007Q4|N|N|N|2454406|2454709|2454054|2454327|N|N|N|N|N| +2454420|AAAAAAAAEJDHFCAA|2007-11-15|1294|5629|432|2007|4|11|15|4|2007|432|5629|Thursday|2007Q4|N|N|N|2454406|2454709|2454055|2454328|N|N|N|N|N| +2454421|AAAAAAAAFJDHFCAA|2007-11-16|1294|5629|432|2007|5|11|16|4|2007|432|5629|Friday|2007Q4|N|Y|N|2454406|2454709|2454056|2454329|N|N|N|N|N| +2454422|AAAAAAAAGJDHFCAA|2007-11-17|1294|5629|432|2007|6|11|17|4|2007|432|5629|Saturday|2007Q4|N|Y|N|2454406|2454709|2454057|2454330|N|N|N|N|N| +2454423|AAAAAAAAHJDHFCAA|2007-11-18|1294|5629|432|2007|0|11|18|4|2007|432|5629|Sunday|2007Q4|N|N|N|2454406|2454709|2454058|2454331|N|N|N|N|N| +2454424|AAAAAAAAIJDHFCAA|2007-11-19|1294|5629|432|2007|1|11|19|4|2007|432|5629|Monday|2007Q4|N|N|N|2454406|2454709|2454059|2454332|N|N|N|N|N| +2454425|AAAAAAAAJJDHFCAA|2007-11-20|1294|5630|432|2007|2|11|20|4|2007|432|5630|Tuesday|2007Q4|N|N|N|2454406|2454709|2454060|2454333|N|N|N|N|N| +2454426|AAAAAAAAKJDHFCAA|2007-11-21|1294|5630|432|2007|3|11|21|4|2007|432|5630|Wednesday|2007Q4|N|N|N|2454406|2454709|2454061|2454334|N|N|N|N|N| +2454427|AAAAAAAALJDHFCAA|2007-11-22|1294|5630|432|2007|4|11|22|4|2007|432|5630|Thursday|2007Q4|N|N|N|2454406|2454709|2454062|2454335|N|N|N|N|N| +2454428|AAAAAAAAMJDHFCAA|2007-11-23|1294|5630|432|2007|5|11|23|4|2007|432|5630|Friday|2007Q4|N|Y|N|2454406|2454709|2454063|2454336|N|N|N|N|N| +2454429|AAAAAAAANJDHFCAA|2007-11-24|1294|5630|432|2007|6|11|24|4|2007|432|5630|Saturday|2007Q4|N|Y|N|2454406|2454709|2454064|2454337|N|N|N|N|N| +2454430|AAAAAAAAOJDHFCAA|2007-11-25|1294|5630|432|2007|0|11|25|4|2007|432|5630|Sunday|2007Q4|N|N|N|2454406|2454709|2454065|2454338|N|N|N|N|N| +2454431|AAAAAAAAPJDHFCAA|2007-11-26|1294|5630|432|2007|1|11|26|4|2007|432|5630|Monday|2007Q4|N|N|N|2454406|2454709|2454066|2454339|N|N|N|N|N| +2454432|AAAAAAAAAKDHFCAA|2007-11-27|1294|5631|432|2007|2|11|27|4|2007|432|5631|Tuesday|2007Q4|N|N|N|2454406|2454709|2454067|2454340|N|N|N|N|N| +2454433|AAAAAAAABKDHFCAA|2007-11-28|1294|5631|432|2007|3|11|28|4|2007|432|5631|Wednesday|2007Q4|N|N|N|2454406|2454709|2454068|2454341|N|N|N|N|N| +2454434|AAAAAAAACKDHFCAA|2007-11-29|1294|5631|432|2007|4|11|29|4|2007|432|5631|Thursday|2007Q4|N|N|N|2454406|2454709|2454069|2454342|N|N|N|N|N| +2454435|AAAAAAAADKDHFCAA|2007-11-30|1294|5631|432|2007|5|11|30|4|2007|432|5631|Friday|2007Q4|N|Y|N|2454406|2454709|2454070|2454343|N|N|N|N|N| +2454436|AAAAAAAAEKDHFCAA|2007-12-01|1295|5631|433|2007|6|12|1|4|2007|433|5631|Saturday|2007Q4|N|Y|N|2454436|2454769|2454071|2454344|N|N|N|N|N| +2454437|AAAAAAAAFKDHFCAA|2007-12-02|1295|5631|433|2007|0|12|2|4|2007|433|5631|Sunday|2007Q4|N|N|N|2454436|2454769|2454072|2454345|N|N|N|N|N| +2454438|AAAAAAAAGKDHFCAA|2007-12-03|1295|5631|433|2007|1|12|3|4|2007|433|5631|Monday|2007Q4|N|N|N|2454436|2454769|2454073|2454346|N|N|N|N|N| +2454439|AAAAAAAAHKDHFCAA|2007-12-04|1295|5632|433|2007|2|12|4|4|2007|433|5632|Tuesday|2007Q4|N|N|N|2454436|2454769|2454074|2454347|N|N|N|N|N| +2454440|AAAAAAAAIKDHFCAA|2007-12-05|1295|5632|433|2007|3|12|5|4|2007|433|5632|Wednesday|2007Q4|N|N|N|2454436|2454769|2454075|2454348|N|N|N|N|N| +2454441|AAAAAAAAJKDHFCAA|2007-12-06|1295|5632|433|2007|4|12|6|4|2007|433|5632|Thursday|2007Q4|N|N|N|2454436|2454769|2454076|2454349|N|N|N|N|N| +2454442|AAAAAAAAKKDHFCAA|2007-12-07|1295|5632|433|2007|5|12|7|4|2007|433|5632|Friday|2007Q4|N|Y|N|2454436|2454769|2454077|2454350|N|N|N|N|N| +2454443|AAAAAAAALKDHFCAA|2007-12-08|1295|5632|433|2007|6|12|8|4|2007|433|5632|Saturday|2007Q4|N|Y|N|2454436|2454769|2454078|2454351|N|N|N|N|N| +2454444|AAAAAAAAMKDHFCAA|2007-12-09|1295|5632|433|2007|0|12|9|4|2007|433|5632|Sunday|2007Q4|N|N|N|2454436|2454769|2454079|2454352|N|N|N|N|N| +2454445|AAAAAAAANKDHFCAA|2007-12-10|1295|5632|433|2007|1|12|10|4|2007|433|5632|Monday|2007Q4|N|N|N|2454436|2454769|2454080|2454353|N|N|N|N|N| +2454446|AAAAAAAAOKDHFCAA|2007-12-11|1295|5633|433|2007|2|12|11|4|2007|433|5633|Tuesday|2007Q4|N|N|N|2454436|2454769|2454081|2454354|N|N|N|N|N| +2454447|AAAAAAAAPKDHFCAA|2007-12-12|1295|5633|433|2007|3|12|12|4|2007|433|5633|Wednesday|2007Q4|N|N|N|2454436|2454769|2454082|2454355|N|N|N|N|N| +2454448|AAAAAAAAALDHFCAA|2007-12-13|1295|5633|433|2007|4|12|13|4|2007|433|5633|Thursday|2007Q4|N|N|N|2454436|2454769|2454083|2454356|N|N|N|N|N| +2454449|AAAAAAAABLDHFCAA|2007-12-14|1295|5633|433|2007|5|12|14|4|2007|433|5633|Friday|2007Q4|N|Y|N|2454436|2454769|2454084|2454357|N|N|N|N|N| +2454450|AAAAAAAACLDHFCAA|2007-12-15|1295|5633|433|2007|6|12|15|4|2007|433|5633|Saturday|2007Q4|N|Y|N|2454436|2454769|2454085|2454358|N|N|N|N|N| +2454451|AAAAAAAADLDHFCAA|2007-12-16|1295|5633|433|2007|0|12|16|4|2007|433|5633|Sunday|2007Q4|N|N|N|2454436|2454769|2454086|2454359|N|N|N|N|N| +2454452|AAAAAAAAELDHFCAA|2007-12-17|1295|5633|433|2007|1|12|17|4|2007|433|5633|Monday|2007Q4|N|N|N|2454436|2454769|2454087|2454360|N|N|N|N|N| +2454453|AAAAAAAAFLDHFCAA|2007-12-18|1295|5634|433|2007|2|12|18|4|2007|433|5634|Tuesday|2007Q4|N|N|N|2454436|2454769|2454088|2454361|N|N|N|N|N| +2454454|AAAAAAAAGLDHFCAA|2007-12-19|1295|5634|433|2007|3|12|19|4|2007|433|5634|Wednesday|2007Q4|N|N|N|2454436|2454769|2454089|2454362|N|N|N|N|N| +2454455|AAAAAAAAHLDHFCAA|2007-12-20|1295|5634|433|2007|4|12|20|4|2007|433|5634|Thursday|2007Q4|N|N|N|2454436|2454769|2454090|2454363|N|N|N|N|N| +2454456|AAAAAAAAILDHFCAA|2007-12-21|1295|5634|433|2007|5|12|21|4|2007|433|5634|Friday|2007Q4|N|Y|N|2454436|2454769|2454091|2454364|N|N|N|N|N| +2454457|AAAAAAAAJLDHFCAA|2007-12-22|1295|5634|433|2007|6|12|22|4|2007|433|5634|Saturday|2007Q4|N|Y|N|2454436|2454769|2454092|2454365|N|N|N|N|N| +2454458|AAAAAAAAKLDHFCAA|2007-12-23|1295|5634|433|2007|0|12|23|4|2007|433|5634|Sunday|2007Q4|N|N|N|2454436|2454769|2454093|2454366|N|N|N|N|N| +2454459|AAAAAAAALLDHFCAA|2007-12-24|1295|5634|433|2007|1|12|24|4|2007|433|5634|Monday|2007Q4|N|N|N|2454436|2454769|2454094|2454367|N|N|N|N|N| +2454460|AAAAAAAAMLDHFCAA|2007-12-25|1295|5635|433|2007|2|12|25|4|2007|433|5635|Tuesday|2007Q4|N|N|N|2454436|2454769|2454095|2454368|N|N|N|N|N| +2454461|AAAAAAAANLDHFCAA|2007-12-26|1295|5635|433|2007|3|12|26|4|2007|433|5635|Wednesday|2007Q4|Y|N|N|2454436|2454769|2454096|2454369|N|N|N|N|N| +2454462|AAAAAAAAOLDHFCAA|2007-12-27|1295|5635|433|2007|4|12|27|4|2007|433|5635|Thursday|2007Q4|N|N|Y|2454436|2454769|2454097|2454370|N|N|N|N|N| +2454463|AAAAAAAAPLDHFCAA|2007-12-28|1295|5635|433|2007|5|12|28|4|2007|433|5635|Friday|2007Q4|N|Y|N|2454436|2454769|2454098|2454371|N|N|N|N|N| +2454464|AAAAAAAAAMDHFCAA|2007-12-29|1295|5635|433|2007|6|12|29|4|2007|433|5635|Saturday|2007Q4|N|Y|N|2454436|2454769|2454099|2454372|N|N|N|N|N| +2454465|AAAAAAAABMDHFCAA|2007-12-30|1295|5635|433|2007|0|12|30|4|2007|433|5635|Sunday|2007Q4|N|N|N|2454436|2454769|2454100|2454373|N|N|N|N|N| +2454466|AAAAAAAACMDHFCAA|2007-12-31|1295|5635|433|2007|1|12|31|4|2007|433|5635|Monday|2007Q4|N|N|N|2454436|2454769|2454101|2454374|N|N|N|N|N| +2454467|AAAAAAAADMDHFCAA|2008-01-01|1296|5636|433|2008|2|1|1|1|2008|433|5636|Tuesday|2008Q1|Y|N|N|2454467|2454466|2454102|2454375|N|N|N|N|N| +2454468|AAAAAAAAEMDHFCAA|2008-01-02|1296|5636|433|2008|3|1|2|1|2008|433|5636|Wednesday|2008Q1|N|N|Y|2454467|2454466|2454103|2454376|N|N|N|N|N| +2454469|AAAAAAAAFMDHFCAA|2008-01-03|1296|5636|433|2008|4|1|3|1|2008|433|5636|Thursday|2008Q1|N|N|N|2454467|2454466|2454104|2454377|N|N|N|N|N| +2454470|AAAAAAAAGMDHFCAA|2008-01-04|1296|5636|433|2008|5|1|4|1|2008|433|5636|Friday|2008Q1|N|Y|N|2454467|2454466|2454105|2454378|N|N|N|N|N| +2454471|AAAAAAAAHMDHFCAA|2008-01-05|1296|5636|433|2008|6|1|5|1|2008|433|5636|Saturday|2008Q1|N|Y|N|2454467|2454466|2454106|2454379|N|N|N|N|N| +2454472|AAAAAAAAIMDHFCAA|2008-01-06|1296|5636|433|2008|0|1|6|1|2008|433|5636|Sunday|2008Q1|N|N|N|2454467|2454466|2454107|2454380|N|N|N|N|N| +2454473|AAAAAAAAJMDHFCAA|2008-01-07|1296|5636|433|2008|1|1|7|1|2008|433|5636|Monday|2008Q1|N|N|N|2454467|2454466|2454108|2454381|N|N|N|N|N| +2454474|AAAAAAAAKMDHFCAA|2008-01-08|1296|5637|433|2008|2|1|8|1|2008|433|5637|Tuesday|2008Q1|N|N|N|2454467|2454466|2454109|2454382|N|N|N|N|N| +2454475|AAAAAAAALMDHFCAA|2008-01-09|1296|5637|433|2008|3|1|9|1|2008|433|5637|Wednesday|2008Q1|N|N|N|2454467|2454466|2454110|2454383|N|N|N|N|N| +2454476|AAAAAAAAMMDHFCAA|2008-01-10|1296|5637|433|2008|4|1|10|1|2008|433|5637|Thursday|2008Q1|N|N|N|2454467|2454466|2454111|2454384|N|N|N|N|N| +2454477|AAAAAAAANMDHFCAA|2008-01-11|1296|5637|433|2008|5|1|11|1|2008|433|5637|Friday|2008Q1|N|Y|N|2454467|2454466|2454112|2454385|N|N|N|N|N| +2454478|AAAAAAAAOMDHFCAA|2008-01-12|1296|5637|433|2008|6|1|12|1|2008|433|5637|Saturday|2008Q1|N|Y|N|2454467|2454466|2454113|2454386|N|N|N|N|N| +2454479|AAAAAAAAPMDHFCAA|2008-01-13|1296|5637|433|2008|0|1|13|1|2008|433|5637|Sunday|2008Q1|N|N|N|2454467|2454466|2454114|2454387|N|N|N|N|N| +2454480|AAAAAAAAANDHFCAA|2008-01-14|1296|5637|433|2008|1|1|14|1|2008|433|5637|Monday|2008Q1|N|N|N|2454467|2454466|2454115|2454388|N|N|N|N|N| +2454481|AAAAAAAABNDHFCAA|2008-01-15|1296|5638|433|2008|2|1|15|1|2008|433|5638|Tuesday|2008Q1|N|N|N|2454467|2454466|2454116|2454389|N|N|N|N|N| +2454482|AAAAAAAACNDHFCAA|2008-01-16|1296|5638|433|2008|3|1|16|1|2008|433|5638|Wednesday|2008Q1|N|N|N|2454467|2454466|2454117|2454390|N|N|N|N|N| +2454483|AAAAAAAADNDHFCAA|2008-01-17|1296|5638|433|2008|4|1|17|1|2008|433|5638|Thursday|2008Q1|N|N|N|2454467|2454466|2454118|2454391|N|N|N|N|N| +2454484|AAAAAAAAENDHFCAA|2008-01-18|1296|5638|433|2008|5|1|18|1|2008|433|5638|Friday|2008Q1|N|Y|N|2454467|2454466|2454119|2454392|N|N|N|N|N| +2454485|AAAAAAAAFNDHFCAA|2008-01-19|1296|5638|433|2008|6|1|19|1|2008|433|5638|Saturday|2008Q1|N|Y|N|2454467|2454466|2454120|2454393|N|N|N|N|N| +2454486|AAAAAAAAGNDHFCAA|2008-01-20|1296|5638|433|2008|0|1|20|1|2008|433|5638|Sunday|2008Q1|N|N|N|2454467|2454466|2454121|2454394|N|N|N|N|N| +2454487|AAAAAAAAHNDHFCAA|2008-01-21|1296|5638|433|2008|1|1|21|1|2008|433|5638|Monday|2008Q1|N|N|N|2454467|2454466|2454122|2454395|N|N|N|N|N| +2454488|AAAAAAAAINDHFCAA|2008-01-22|1296|5639|433|2008|2|1|22|1|2008|433|5639|Tuesday|2008Q1|N|N|N|2454467|2454466|2454123|2454396|N|N|N|N|N| +2454489|AAAAAAAAJNDHFCAA|2008-01-23|1296|5639|433|2008|3|1|23|1|2008|433|5639|Wednesday|2008Q1|N|N|N|2454467|2454466|2454124|2454397|N|N|N|N|N| +2454490|AAAAAAAAKNDHFCAA|2008-01-24|1296|5639|433|2008|4|1|24|1|2008|433|5639|Thursday|2008Q1|N|N|N|2454467|2454466|2454125|2454398|N|N|N|N|N| +2454491|AAAAAAAALNDHFCAA|2008-01-25|1296|5639|433|2008|5|1|25|1|2008|433|5639|Friday|2008Q1|N|Y|N|2454467|2454466|2454126|2454399|N|N|N|N|N| +2454492|AAAAAAAAMNDHFCAA|2008-01-26|1296|5639|433|2008|6|1|26|1|2008|433|5639|Saturday|2008Q1|N|Y|N|2454467|2454466|2454127|2454400|N|N|N|N|N| +2454493|AAAAAAAANNDHFCAA|2008-01-27|1296|5639|433|2008|0|1|27|1|2008|433|5639|Sunday|2008Q1|N|N|N|2454467|2454466|2454128|2454401|N|N|N|N|N| +2454494|AAAAAAAAONDHFCAA|2008-01-28|1296|5639|433|2008|1|1|28|1|2008|433|5639|Monday|2008Q1|N|N|N|2454467|2454466|2454129|2454402|N|N|N|N|N| +2454495|AAAAAAAAPNDHFCAA|2008-01-29|1296|5640|433|2008|2|1|29|1|2008|433|5640|Tuesday|2008Q1|N|N|N|2454467|2454466|2454130|2454403|N|N|N|N|N| +2454496|AAAAAAAAAODHFCAA|2008-01-30|1296|5640|433|2008|3|1|30|1|2008|433|5640|Wednesday|2008Q1|N|N|N|2454467|2454466|2454131|2454404|N|N|N|N|N| +2454497|AAAAAAAABODHFCAA|2008-01-31|1296|5640|433|2008|4|1|31|1|2008|433|5640|Thursday|2008Q1|N|N|N|2454467|2454466|2454132|2454405|N|N|N|N|N| +2454498|AAAAAAAACODHFCAA|2008-02-01|1297|5640|433|2008|5|2|1|1|2008|433|5640|Friday|2008Q1|N|Y|N|2454498|2454528|2454133|2454406|N|N|N|N|N| +2454499|AAAAAAAADODHFCAA|2008-02-02|1297|5640|433|2008|6|2|2|1|2008|433|5640|Saturday|2008Q1|N|Y|N|2454498|2454528|2454134|2454407|N|N|N|N|N| +2454500|AAAAAAAAEODHFCAA|2008-02-03|1297|5640|433|2008|0|2|3|1|2008|433|5640|Sunday|2008Q1|N|N|N|2454498|2454528|2454135|2454408|N|N|N|N|N| +2454501|AAAAAAAAFODHFCAA|2008-02-04|1297|5640|433|2008|1|2|4|1|2008|433|5640|Monday|2008Q1|N|N|N|2454498|2454528|2454136|2454409|N|N|N|N|N| +2454502|AAAAAAAAGODHFCAA|2008-02-05|1297|5641|433|2008|2|2|5|1|2008|433|5641|Tuesday|2008Q1|N|N|N|2454498|2454528|2454137|2454410|N|N|N|N|N| +2454503|AAAAAAAAHODHFCAA|2008-02-06|1297|5641|433|2008|3|2|6|1|2008|433|5641|Wednesday|2008Q1|N|N|N|2454498|2454528|2454138|2454411|N|N|N|N|N| +2454504|AAAAAAAAIODHFCAA|2008-02-07|1297|5641|433|2008|4|2|7|1|2008|433|5641|Thursday|2008Q1|N|N|N|2454498|2454528|2454139|2454412|N|N|N|N|N| +2454505|AAAAAAAAJODHFCAA|2008-02-08|1297|5641|433|2008|5|2|8|1|2008|433|5641|Friday|2008Q1|N|Y|N|2454498|2454528|2454140|2454413|N|N|N|N|N| +2454506|AAAAAAAAKODHFCAA|2008-02-09|1297|5641|433|2008|6|2|9|1|2008|433|5641|Saturday|2008Q1|N|Y|N|2454498|2454528|2454141|2454414|N|N|N|N|N| +2454507|AAAAAAAALODHFCAA|2008-02-10|1297|5641|433|2008|0|2|10|1|2008|433|5641|Sunday|2008Q1|N|N|N|2454498|2454528|2454142|2454415|N|N|N|N|N| +2454508|AAAAAAAAMODHFCAA|2008-02-11|1297|5641|433|2008|1|2|11|1|2008|433|5641|Monday|2008Q1|N|N|N|2454498|2454528|2454143|2454416|N|N|N|N|N| +2454509|AAAAAAAANODHFCAA|2008-02-12|1297|5642|433|2008|2|2|12|1|2008|433|5642|Tuesday|2008Q1|N|N|N|2454498|2454528|2454144|2454417|N|N|N|N|N| +2454510|AAAAAAAAOODHFCAA|2008-02-13|1297|5642|433|2008|3|2|13|1|2008|433|5642|Wednesday|2008Q1|N|N|N|2454498|2454528|2454145|2454418|N|N|N|N|N| +2454511|AAAAAAAAPODHFCAA|2008-02-14|1297|5642|433|2008|4|2|14|1|2008|433|5642|Thursday|2008Q1|N|N|N|2454498|2454528|2454146|2454419|N|N|N|N|N| +2454512|AAAAAAAAAPDHFCAA|2008-02-15|1297|5642|433|2008|5|2|15|1|2008|433|5642|Friday|2008Q1|N|Y|N|2454498|2454528|2454147|2454420|N|N|N|N|N| +2454513|AAAAAAAABPDHFCAA|2008-02-16|1297|5642|433|2008|6|2|16|1|2008|433|5642|Saturday|2008Q1|N|Y|N|2454498|2454528|2454148|2454421|N|N|N|N|N| +2454514|AAAAAAAACPDHFCAA|2008-02-17|1297|5642|433|2008|0|2|17|1|2008|433|5642|Sunday|2008Q1|N|N|N|2454498|2454528|2454149|2454422|N|N|N|N|N| +2454515|AAAAAAAADPDHFCAA|2008-02-18|1297|5642|433|2008|1|2|18|1|2008|433|5642|Monday|2008Q1|N|N|N|2454498|2454528|2454150|2454423|N|N|N|N|N| +2454516|AAAAAAAAEPDHFCAA|2008-02-19|1297|5643|433|2008|2|2|19|1|2008|433|5643|Tuesday|2008Q1|N|N|N|2454498|2454528|2454151|2454424|N|N|N|N|N| +2454517|AAAAAAAAFPDHFCAA|2008-02-20|1297|5643|433|2008|3|2|20|1|2008|433|5643|Wednesday|2008Q1|N|N|N|2454498|2454528|2454152|2454425|N|N|N|N|N| +2454518|AAAAAAAAGPDHFCAA|2008-02-21|1297|5643|433|2008|4|2|21|1|2008|433|5643|Thursday|2008Q1|N|N|N|2454498|2454528|2454153|2454426|N|N|N|N|N| +2454519|AAAAAAAAHPDHFCAA|2008-02-22|1297|5643|433|2008|5|2|22|1|2008|433|5643|Friday|2008Q1|N|Y|N|2454498|2454528|2454154|2454427|N|N|N|N|N| +2454520|AAAAAAAAIPDHFCAA|2008-02-23|1297|5643|433|2008|6|2|23|1|2008|433|5643|Saturday|2008Q1|N|Y|N|2454498|2454528|2454155|2454428|N|N|N|N|N| +2454521|AAAAAAAAJPDHFCAA|2008-02-24|1297|5643|433|2008|0|2|24|1|2008|433|5643|Sunday|2008Q1|N|N|N|2454498|2454528|2454156|2454429|N|N|N|N|N| +2454522|AAAAAAAAKPDHFCAA|2008-02-25|1297|5643|433|2008|1|2|25|1|2008|433|5643|Monday|2008Q1|N|N|N|2454498|2454528|2454157|2454430|N|N|N|N|N| +2454523|AAAAAAAALPDHFCAA|2008-02-26|1297|5644|433|2008|2|2|26|1|2008|433|5644|Tuesday|2008Q1|N|N|N|2454498|2454528|2454158|2454431|N|N|N|N|N| +2454524|AAAAAAAAMPDHFCAA|2008-02-27|1297|5644|433|2008|3|2|27|1|2008|433|5644|Wednesday|2008Q1|N|N|N|2454498|2454528|2454159|2454432|N|N|N|N|N| +2454525|AAAAAAAANPDHFCAA|2008-02-28|1297|5644|433|2008|4|2|28|1|2008|433|5644|Thursday|2008Q1|N|N|N|2454498|2454528|2454160|2454433|N|N|N|N|N| +2454526|AAAAAAAAOPDHFCAA|2008-02-29|1297|5644|433|2008|5|2|29|1|2008|433|5644|Friday|2008Q1|N|Y|N|2454498|2454528|2454160|2454434|N|N|N|N|N| +2454527|AAAAAAAAPPDHFCAA|2008-03-01|1298|5644|434|2008|6|3|1|1|2008|434|5644|Saturday|2008Q1|N|Y|N|2454527|2454586|2454161|2454435|N|N|N|N|N| +2454528|AAAAAAAAAAEHFCAA|2008-03-02|1298|5644|434|2008|0|3|2|1|2008|434|5644|Sunday|2008Q1|N|N|N|2454527|2454586|2454162|2454436|N|N|N|N|N| +2454529|AAAAAAAABAEHFCAA|2008-03-03|1298|5644|434|2008|1|3|3|1|2008|434|5644|Monday|2008Q1|N|N|N|2454527|2454586|2454163|2454437|N|N|N|N|N| +2454530|AAAAAAAACAEHFCAA|2008-03-04|1298|5645|434|2008|2|3|4|1|2008|434|5645|Tuesday|2008Q1|N|N|N|2454527|2454586|2454164|2454438|N|N|N|N|N| +2454531|AAAAAAAADAEHFCAA|2008-03-05|1298|5645|434|2008|3|3|5|1|2008|434|5645|Wednesday|2008Q1|N|N|N|2454527|2454586|2454165|2454439|N|N|N|N|N| +2454532|AAAAAAAAEAEHFCAA|2008-03-06|1298|5645|434|2008|4|3|6|1|2008|434|5645|Thursday|2008Q1|N|N|N|2454527|2454586|2454166|2454440|N|N|N|N|N| +2454533|AAAAAAAAFAEHFCAA|2008-03-07|1298|5645|434|2008|5|3|7|1|2008|434|5645|Friday|2008Q1|N|Y|N|2454527|2454586|2454167|2454441|N|N|N|N|N| +2454534|AAAAAAAAGAEHFCAA|2008-03-08|1298|5645|434|2008|6|3|8|1|2008|434|5645|Saturday|2008Q1|N|Y|N|2454527|2454586|2454168|2454442|N|N|N|N|N| +2454535|AAAAAAAAHAEHFCAA|2008-03-09|1298|5645|434|2008|0|3|9|1|2008|434|5645|Sunday|2008Q1|N|N|N|2454527|2454586|2454169|2454443|N|N|N|N|N| +2454536|AAAAAAAAIAEHFCAA|2008-03-10|1298|5645|434|2008|1|3|10|1|2008|434|5645|Monday|2008Q1|N|N|N|2454527|2454586|2454170|2454444|N|N|N|N|N| +2454537|AAAAAAAAJAEHFCAA|2008-03-11|1298|5646|434|2008|2|3|11|1|2008|434|5646|Tuesday|2008Q1|N|N|N|2454527|2454586|2454171|2454445|N|N|N|N|N| +2454538|AAAAAAAAKAEHFCAA|2008-03-12|1298|5646|434|2008|3|3|12|1|2008|434|5646|Wednesday|2008Q1|N|N|N|2454527|2454586|2454172|2454446|N|N|N|N|N| +2454539|AAAAAAAALAEHFCAA|2008-03-13|1298|5646|434|2008|4|3|13|1|2008|434|5646|Thursday|2008Q1|N|N|N|2454527|2454586|2454173|2454447|N|N|N|N|N| +2454540|AAAAAAAAMAEHFCAA|2008-03-14|1298|5646|434|2008|5|3|14|1|2008|434|5646|Friday|2008Q1|N|Y|N|2454527|2454586|2454174|2454448|N|N|N|N|N| +2454541|AAAAAAAANAEHFCAA|2008-03-15|1298|5646|434|2008|6|3|15|1|2008|434|5646|Saturday|2008Q1|N|Y|N|2454527|2454586|2454175|2454449|N|N|N|N|N| +2454542|AAAAAAAAOAEHFCAA|2008-03-16|1298|5646|434|2008|0|3|16|1|2008|434|5646|Sunday|2008Q1|N|N|N|2454527|2454586|2454176|2454450|N|N|N|N|N| +2454543|AAAAAAAAPAEHFCAA|2008-03-17|1298|5646|434|2008|1|3|17|1|2008|434|5646|Monday|2008Q1|N|N|N|2454527|2454586|2454177|2454451|N|N|N|N|N| +2454544|AAAAAAAAABEHFCAA|2008-03-18|1298|5647|434|2008|2|3|18|1|2008|434|5647|Tuesday|2008Q1|N|N|N|2454527|2454586|2454178|2454452|N|N|N|N|N| +2454545|AAAAAAAABBEHFCAA|2008-03-19|1298|5647|434|2008|3|3|19|1|2008|434|5647|Wednesday|2008Q1|N|N|N|2454527|2454586|2454179|2454453|N|N|N|N|N| +2454546|AAAAAAAACBEHFCAA|2008-03-20|1298|5647|434|2008|4|3|20|1|2008|434|5647|Thursday|2008Q1|N|N|N|2454527|2454586|2454180|2454454|N|N|N|N|N| +2454547|AAAAAAAADBEHFCAA|2008-03-21|1298|5647|434|2008|5|3|21|1|2008|434|5647|Friday|2008Q1|N|Y|N|2454527|2454586|2454181|2454455|N|N|N|N|N| +2454548|AAAAAAAAEBEHFCAA|2008-03-22|1298|5647|434|2008|6|3|22|1|2008|434|5647|Saturday|2008Q1|N|Y|N|2454527|2454586|2454182|2454456|N|N|N|N|N| +2454549|AAAAAAAAFBEHFCAA|2008-03-23|1298|5647|434|2008|0|3|23|1|2008|434|5647|Sunday|2008Q1|N|N|N|2454527|2454586|2454183|2454457|N|N|N|N|N| +2454550|AAAAAAAAGBEHFCAA|2008-03-24|1298|5647|434|2008|1|3|24|1|2008|434|5647|Monday|2008Q1|N|N|N|2454527|2454586|2454184|2454458|N|N|N|N|N| +2454551|AAAAAAAAHBEHFCAA|2008-03-25|1298|5648|434|2008|2|3|25|1|2008|434|5648|Tuesday|2008Q1|N|N|N|2454527|2454586|2454185|2454459|N|N|N|N|N| +2454552|AAAAAAAAIBEHFCAA|2008-03-26|1298|5648|434|2008|3|3|26|1|2008|434|5648|Wednesday|2008Q1|N|N|N|2454527|2454586|2454186|2454460|N|N|N|N|N| +2454553|AAAAAAAAJBEHFCAA|2008-03-27|1298|5648|434|2008|4|3|27|1|2008|434|5648|Thursday|2008Q1|N|N|N|2454527|2454586|2454187|2454461|N|N|N|N|N| +2454554|AAAAAAAAKBEHFCAA|2008-03-28|1298|5648|434|2008|5|3|28|1|2008|434|5648|Friday|2008Q1|N|Y|N|2454527|2454586|2454188|2454462|N|N|N|N|N| +2454555|AAAAAAAALBEHFCAA|2008-03-29|1298|5648|434|2008|6|3|29|1|2008|434|5648|Saturday|2008Q1|N|Y|N|2454527|2454586|2454189|2454463|N|N|N|N|N| +2454556|AAAAAAAAMBEHFCAA|2008-03-30|1298|5648|434|2008|0|3|30|1|2008|434|5648|Sunday|2008Q1|N|N|N|2454527|2454586|2454190|2454464|N|N|N|N|N| +2454557|AAAAAAAANBEHFCAA|2008-03-31|1298|5648|434|2008|1|3|31|1|2008|434|5648|Monday|2008Q1|N|N|N|2454527|2454586|2454191|2454465|N|N|N|N|N| +2454558|AAAAAAAAOBEHFCAA|2008-04-01|1299|5649|434|2008|2|4|1|2|2008|434|5649|Tuesday|2008Q2|N|N|N|2454558|2454648|2454192|2454467|N|N|N|N|N| +2454559|AAAAAAAAPBEHFCAA|2008-04-02|1299|5649|434|2008|3|4|2|2|2008|434|5649|Wednesday|2008Q2|N|N|N|2454558|2454648|2454193|2454468|N|N|N|N|N| +2454560|AAAAAAAAACEHFCAA|2008-04-03|1299|5649|434|2008|4|4|3|2|2008|434|5649|Thursday|2008Q2|N|N|N|2454558|2454648|2454194|2454469|N|N|N|N|N| +2454561|AAAAAAAABCEHFCAA|2008-04-04|1299|5649|434|2008|5|4|4|2|2008|434|5649|Friday|2008Q2|N|Y|N|2454558|2454648|2454195|2454470|N|N|N|N|N| +2454562|AAAAAAAACCEHFCAA|2008-04-05|1299|5649|434|2008|6|4|5|2|2008|434|5649|Saturday|2008Q2|N|Y|N|2454558|2454648|2454196|2454471|N|N|N|N|N| +2454563|AAAAAAAADCEHFCAA|2008-04-06|1299|5649|434|2008|0|4|6|2|2008|434|5649|Sunday|2008Q2|N|N|N|2454558|2454648|2454197|2454472|N|N|N|N|N| +2454564|AAAAAAAAECEHFCAA|2008-04-07|1299|5649|434|2008|1|4|7|2|2008|434|5649|Monday|2008Q2|N|N|N|2454558|2454648|2454198|2454473|N|N|N|N|N| +2454565|AAAAAAAAFCEHFCAA|2008-04-08|1299|5650|434|2008|2|4|8|2|2008|434|5650|Tuesday|2008Q2|N|N|N|2454558|2454648|2454199|2454474|N|N|N|N|N| +2454566|AAAAAAAAGCEHFCAA|2008-04-09|1299|5650|434|2008|3|4|9|2|2008|434|5650|Wednesday|2008Q2|N|N|N|2454558|2454648|2454200|2454475|N|N|N|N|N| +2454567|AAAAAAAAHCEHFCAA|2008-04-10|1299|5650|434|2008|4|4|10|2|2008|434|5650|Thursday|2008Q2|N|N|N|2454558|2454648|2454201|2454476|N|N|N|N|N| +2454568|AAAAAAAAICEHFCAA|2008-04-11|1299|5650|434|2008|5|4|11|2|2008|434|5650|Friday|2008Q2|N|Y|N|2454558|2454648|2454202|2454477|N|N|N|N|N| +2454569|AAAAAAAAJCEHFCAA|2008-04-12|1299|5650|434|2008|6|4|12|2|2008|434|5650|Saturday|2008Q2|N|Y|N|2454558|2454648|2454203|2454478|N|N|N|N|N| +2454570|AAAAAAAAKCEHFCAA|2008-04-13|1299|5650|434|2008|0|4|13|2|2008|434|5650|Sunday|2008Q2|N|N|N|2454558|2454648|2454204|2454479|N|N|N|N|N| +2454571|AAAAAAAALCEHFCAA|2008-04-14|1299|5650|434|2008|1|4|14|2|2008|434|5650|Monday|2008Q2|N|N|N|2454558|2454648|2454205|2454480|N|N|N|N|N| +2454572|AAAAAAAAMCEHFCAA|2008-04-15|1299|5651|434|2008|2|4|15|2|2008|434|5651|Tuesday|2008Q2|N|N|N|2454558|2454648|2454206|2454481|N|N|N|N|N| +2454573|AAAAAAAANCEHFCAA|2008-04-16|1299|5651|434|2008|3|4|16|2|2008|434|5651|Wednesday|2008Q2|N|N|N|2454558|2454648|2454207|2454482|N|N|N|N|N| +2454574|AAAAAAAAOCEHFCAA|2008-04-17|1299|5651|434|2008|4|4|17|2|2008|434|5651|Thursday|2008Q2|N|N|N|2454558|2454648|2454208|2454483|N|N|N|N|N| +2454575|AAAAAAAAPCEHFCAA|2008-04-18|1299|5651|434|2008|5|4|18|2|2008|434|5651|Friday|2008Q2|N|Y|N|2454558|2454648|2454209|2454484|N|N|N|N|N| +2454576|AAAAAAAAADEHFCAA|2008-04-19|1299|5651|434|2008|6|4|19|2|2008|434|5651|Saturday|2008Q2|N|Y|N|2454558|2454648|2454210|2454485|N|N|N|N|N| +2454577|AAAAAAAABDEHFCAA|2008-04-20|1299|5651|434|2008|0|4|20|2|2008|434|5651|Sunday|2008Q2|N|N|N|2454558|2454648|2454211|2454486|N|N|N|N|N| +2454578|AAAAAAAACDEHFCAA|2008-04-21|1299|5651|434|2008|1|4|21|2|2008|434|5651|Monday|2008Q2|N|N|N|2454558|2454648|2454212|2454487|N|N|N|N|N| +2454579|AAAAAAAADDEHFCAA|2008-04-22|1299|5652|434|2008|2|4|22|2|2008|434|5652|Tuesday|2008Q2|N|N|N|2454558|2454648|2454213|2454488|N|N|N|N|N| +2454580|AAAAAAAAEDEHFCAA|2008-04-23|1299|5652|434|2008|3|4|23|2|2008|434|5652|Wednesday|2008Q2|N|N|N|2454558|2454648|2454214|2454489|N|N|N|N|N| +2454581|AAAAAAAAFDEHFCAA|2008-04-24|1299|5652|434|2008|4|4|24|2|2008|434|5652|Thursday|2008Q2|N|N|N|2454558|2454648|2454215|2454490|N|N|N|N|N| +2454582|AAAAAAAAGDEHFCAA|2008-04-25|1299|5652|434|2008|5|4|25|2|2008|434|5652|Friday|2008Q2|N|Y|N|2454558|2454648|2454216|2454491|N|N|N|N|N| +2454583|AAAAAAAAHDEHFCAA|2008-04-26|1299|5652|434|2008|6|4|26|2|2008|434|5652|Saturday|2008Q2|N|Y|N|2454558|2454648|2454217|2454492|N|N|N|N|N| +2454584|AAAAAAAAIDEHFCAA|2008-04-27|1299|5652|434|2008|0|4|27|2|2008|434|5652|Sunday|2008Q2|N|N|N|2454558|2454648|2454218|2454493|N|N|N|N|N| +2454585|AAAAAAAAJDEHFCAA|2008-04-28|1299|5652|434|2008|1|4|28|2|2008|434|5652|Monday|2008Q2|N|N|N|2454558|2454648|2454219|2454494|N|N|N|N|N| +2454586|AAAAAAAAKDEHFCAA|2008-04-29|1299|5653|434|2008|2|4|29|2|2008|434|5653|Tuesday|2008Q2|N|N|N|2454558|2454648|2454220|2454495|N|N|N|N|N| +2454587|AAAAAAAALDEHFCAA|2008-04-30|1299|5653|434|2008|3|4|30|2|2008|434|5653|Wednesday|2008Q2|N|N|N|2454558|2454648|2454221|2454496|N|N|N|N|N| +2454588|AAAAAAAAMDEHFCAA|2008-05-01|1300|5653|434|2008|4|5|1|2|2008|434|5653|Thursday|2008Q2|N|N|N|2454588|2454708|2454222|2454497|N|N|N|N|N| +2454589|AAAAAAAANDEHFCAA|2008-05-02|1300|5653|434|2008|5|5|2|2|2008|434|5653|Friday|2008Q2|N|Y|N|2454588|2454708|2454223|2454498|N|N|N|N|N| +2454590|AAAAAAAAODEHFCAA|2008-05-03|1300|5653|434|2008|6|5|3|2|2008|434|5653|Saturday|2008Q2|N|Y|N|2454588|2454708|2454224|2454499|N|N|N|N|N| +2454591|AAAAAAAAPDEHFCAA|2008-05-04|1300|5653|434|2008|0|5|4|2|2008|434|5653|Sunday|2008Q2|N|N|N|2454588|2454708|2454225|2454500|N|N|N|N|N| +2454592|AAAAAAAAAEEHFCAA|2008-05-05|1300|5653|434|2008|1|5|5|2|2008|434|5653|Monday|2008Q2|N|N|N|2454588|2454708|2454226|2454501|N|N|N|N|N| +2454593|AAAAAAAABEEHFCAA|2008-05-06|1300|5654|434|2008|2|5|6|2|2008|434|5654|Tuesday|2008Q2|N|N|N|2454588|2454708|2454227|2454502|N|N|N|N|N| +2454594|AAAAAAAACEEHFCAA|2008-05-07|1300|5654|434|2008|3|5|7|2|2008|434|5654|Wednesday|2008Q2|N|N|N|2454588|2454708|2454228|2454503|N|N|N|N|N| +2454595|AAAAAAAADEEHFCAA|2008-05-08|1300|5654|434|2008|4|5|8|2|2008|434|5654|Thursday|2008Q2|N|N|N|2454588|2454708|2454229|2454504|N|N|N|N|N| +2454596|AAAAAAAAEEEHFCAA|2008-05-09|1300|5654|434|2008|5|5|9|2|2008|434|5654|Friday|2008Q2|N|Y|N|2454588|2454708|2454230|2454505|N|N|N|N|N| +2454597|AAAAAAAAFEEHFCAA|2008-05-10|1300|5654|434|2008|6|5|10|2|2008|434|5654|Saturday|2008Q2|N|Y|N|2454588|2454708|2454231|2454506|N|N|N|N|N| +2454598|AAAAAAAAGEEHFCAA|2008-05-11|1300|5654|434|2008|0|5|11|2|2008|434|5654|Sunday|2008Q2|N|N|N|2454588|2454708|2454232|2454507|N|N|N|N|N| +2454599|AAAAAAAAHEEHFCAA|2008-05-12|1300|5654|434|2008|1|5|12|2|2008|434|5654|Monday|2008Q2|N|N|N|2454588|2454708|2454233|2454508|N|N|N|N|N| +2454600|AAAAAAAAIEEHFCAA|2008-05-13|1300|5655|434|2008|2|5|13|2|2008|434|5655|Tuesday|2008Q2|N|N|N|2454588|2454708|2454234|2454509|N|N|N|N|N| +2454601|AAAAAAAAJEEHFCAA|2008-05-14|1300|5655|434|2008|3|5|14|2|2008|434|5655|Wednesday|2008Q2|N|N|N|2454588|2454708|2454235|2454510|N|N|N|N|N| +2454602|AAAAAAAAKEEHFCAA|2008-05-15|1300|5655|434|2008|4|5|15|2|2008|434|5655|Thursday|2008Q2|N|N|N|2454588|2454708|2454236|2454511|N|N|N|N|N| +2454603|AAAAAAAALEEHFCAA|2008-05-16|1300|5655|434|2008|5|5|16|2|2008|434|5655|Friday|2008Q2|N|Y|N|2454588|2454708|2454237|2454512|N|N|N|N|N| +2454604|AAAAAAAAMEEHFCAA|2008-05-17|1300|5655|434|2008|6|5|17|2|2008|434|5655|Saturday|2008Q2|N|Y|N|2454588|2454708|2454238|2454513|N|N|N|N|N| +2454605|AAAAAAAANEEHFCAA|2008-05-18|1300|5655|434|2008|0|5|18|2|2008|434|5655|Sunday|2008Q2|N|N|N|2454588|2454708|2454239|2454514|N|N|N|N|N| +2454606|AAAAAAAAOEEHFCAA|2008-05-19|1300|5655|434|2008|1|5|19|2|2008|434|5655|Monday|2008Q2|N|N|N|2454588|2454708|2454240|2454515|N|N|N|N|N| +2454607|AAAAAAAAPEEHFCAA|2008-05-20|1300|5656|434|2008|2|5|20|2|2008|434|5656|Tuesday|2008Q2|N|N|N|2454588|2454708|2454241|2454516|N|N|N|N|N| +2454608|AAAAAAAAAFEHFCAA|2008-05-21|1300|5656|434|2008|3|5|21|2|2008|434|5656|Wednesday|2008Q2|N|N|N|2454588|2454708|2454242|2454517|N|N|N|N|N| +2454609|AAAAAAAABFEHFCAA|2008-05-22|1300|5656|434|2008|4|5|22|2|2008|434|5656|Thursday|2008Q2|N|N|N|2454588|2454708|2454243|2454518|N|N|N|N|N| +2454610|AAAAAAAACFEHFCAA|2008-05-23|1300|5656|434|2008|5|5|23|2|2008|434|5656|Friday|2008Q2|N|Y|N|2454588|2454708|2454244|2454519|N|N|N|N|N| +2454611|AAAAAAAADFEHFCAA|2008-05-24|1300|5656|434|2008|6|5|24|2|2008|434|5656|Saturday|2008Q2|N|Y|N|2454588|2454708|2454245|2454520|N|N|N|N|N| +2454612|AAAAAAAAEFEHFCAA|2008-05-25|1300|5656|434|2008|0|5|25|2|2008|434|5656|Sunday|2008Q2|N|N|N|2454588|2454708|2454246|2454521|N|N|N|N|N| +2454613|AAAAAAAAFFEHFCAA|2008-05-26|1300|5656|434|2008|1|5|26|2|2008|434|5656|Monday|2008Q2|N|N|N|2454588|2454708|2454247|2454522|N|N|N|N|N| +2454614|AAAAAAAAGFEHFCAA|2008-05-27|1300|5657|434|2008|2|5|27|2|2008|434|5657|Tuesday|2008Q2|N|N|N|2454588|2454708|2454248|2454523|N|N|N|N|N| +2454615|AAAAAAAAHFEHFCAA|2008-05-28|1300|5657|434|2008|3|5|28|2|2008|434|5657|Wednesday|2008Q2|N|N|N|2454588|2454708|2454249|2454524|N|N|N|N|N| +2454616|AAAAAAAAIFEHFCAA|2008-05-29|1300|5657|434|2008|4|5|29|2|2008|434|5657|Thursday|2008Q2|N|N|N|2454588|2454708|2454250|2454525|N|N|N|N|N| +2454617|AAAAAAAAJFEHFCAA|2008-05-30|1300|5657|434|2008|5|5|30|2|2008|434|5657|Friday|2008Q2|N|Y|N|2454588|2454708|2454251|2454526|N|N|N|N|N| +2454618|AAAAAAAAKFEHFCAA|2008-05-31|1300|5657|434|2008|6|5|31|2|2008|434|5657|Saturday|2008Q2|N|Y|N|2454588|2454708|2454252|2454527|N|N|N|N|N| +2454619|AAAAAAAALFEHFCAA|2008-06-01|1301|5657|435|2008|0|6|1|2|2008|435|5657|Sunday|2008Q2|N|N|N|2454619|2454770|2454253|2454528|N|N|N|N|N| +2454620|AAAAAAAAMFEHFCAA|2008-06-02|1301|5657|435|2008|1|6|2|2|2008|435|5657|Monday|2008Q2|N|N|N|2454619|2454770|2454254|2454529|N|N|N|N|N| +2454621|AAAAAAAANFEHFCAA|2008-06-03|1301|5658|435|2008|2|6|3|2|2008|435|5658|Tuesday|2008Q2|N|N|N|2454619|2454770|2454255|2454530|N|N|N|N|N| +2454622|AAAAAAAAOFEHFCAA|2008-06-04|1301|5658|435|2008|3|6|4|2|2008|435|5658|Wednesday|2008Q2|N|N|N|2454619|2454770|2454256|2454531|N|N|N|N|N| +2454623|AAAAAAAAPFEHFCAA|2008-06-05|1301|5658|435|2008|4|6|5|2|2008|435|5658|Thursday|2008Q2|N|N|N|2454619|2454770|2454257|2454532|N|N|N|N|N| +2454624|AAAAAAAAAGEHFCAA|2008-06-06|1301|5658|435|2008|5|6|6|2|2008|435|5658|Friday|2008Q2|N|Y|N|2454619|2454770|2454258|2454533|N|N|N|N|N| +2454625|AAAAAAAABGEHFCAA|2008-06-07|1301|5658|435|2008|6|6|7|2|2008|435|5658|Saturday|2008Q2|N|Y|N|2454619|2454770|2454259|2454534|N|N|N|N|N| +2454626|AAAAAAAACGEHFCAA|2008-06-08|1301|5658|435|2008|0|6|8|2|2008|435|5658|Sunday|2008Q2|N|N|N|2454619|2454770|2454260|2454535|N|N|N|N|N| +2454627|AAAAAAAADGEHFCAA|2008-06-09|1301|5658|435|2008|1|6|9|2|2008|435|5658|Monday|2008Q2|N|N|N|2454619|2454770|2454261|2454536|N|N|N|N|N| +2454628|AAAAAAAAEGEHFCAA|2008-06-10|1301|5659|435|2008|2|6|10|2|2008|435|5659|Tuesday|2008Q2|N|N|N|2454619|2454770|2454262|2454537|N|N|N|N|N| +2454629|AAAAAAAAFGEHFCAA|2008-06-11|1301|5659|435|2008|3|6|11|2|2008|435|5659|Wednesday|2008Q2|N|N|N|2454619|2454770|2454263|2454538|N|N|N|N|N| +2454630|AAAAAAAAGGEHFCAA|2008-06-12|1301|5659|435|2008|4|6|12|2|2008|435|5659|Thursday|2008Q2|N|N|N|2454619|2454770|2454264|2454539|N|N|N|N|N| +2454631|AAAAAAAAHGEHFCAA|2008-06-13|1301|5659|435|2008|5|6|13|2|2008|435|5659|Friday|2008Q2|N|Y|N|2454619|2454770|2454265|2454540|N|N|N|N|N| +2454632|AAAAAAAAIGEHFCAA|2008-06-14|1301|5659|435|2008|6|6|14|2|2008|435|5659|Saturday|2008Q2|N|Y|N|2454619|2454770|2454266|2454541|N|N|N|N|N| +2454633|AAAAAAAAJGEHFCAA|2008-06-15|1301|5659|435|2008|0|6|15|2|2008|435|5659|Sunday|2008Q2|N|N|N|2454619|2454770|2454267|2454542|N|N|N|N|N| +2454634|AAAAAAAAKGEHFCAA|2008-06-16|1301|5659|435|2008|1|6|16|2|2008|435|5659|Monday|2008Q2|N|N|N|2454619|2454770|2454268|2454543|N|N|N|N|N| +2454635|AAAAAAAALGEHFCAA|2008-06-17|1301|5660|435|2008|2|6|17|2|2008|435|5660|Tuesday|2008Q2|N|N|N|2454619|2454770|2454269|2454544|N|N|N|N|N| +2454636|AAAAAAAAMGEHFCAA|2008-06-18|1301|5660|435|2008|3|6|18|2|2008|435|5660|Wednesday|2008Q2|N|N|N|2454619|2454770|2454270|2454545|N|N|N|N|N| +2454637|AAAAAAAANGEHFCAA|2008-06-19|1301|5660|435|2008|4|6|19|2|2008|435|5660|Thursday|2008Q2|N|N|N|2454619|2454770|2454271|2454546|N|N|N|N|N| +2454638|AAAAAAAAOGEHFCAA|2008-06-20|1301|5660|435|2008|5|6|20|2|2008|435|5660|Friday|2008Q2|N|Y|N|2454619|2454770|2454272|2454547|N|N|N|N|N| +2454639|AAAAAAAAPGEHFCAA|2008-06-21|1301|5660|435|2008|6|6|21|2|2008|435|5660|Saturday|2008Q2|N|Y|N|2454619|2454770|2454273|2454548|N|N|N|N|N| +2454640|AAAAAAAAAHEHFCAA|2008-06-22|1301|5660|435|2008|0|6|22|2|2008|435|5660|Sunday|2008Q2|N|N|N|2454619|2454770|2454274|2454549|N|N|N|N|N| +2454641|AAAAAAAABHEHFCAA|2008-06-23|1301|5660|435|2008|1|6|23|2|2008|435|5660|Monday|2008Q2|N|N|N|2454619|2454770|2454275|2454550|N|N|N|N|N| +2454642|AAAAAAAACHEHFCAA|2008-06-24|1301|5661|435|2008|2|6|24|2|2008|435|5661|Tuesday|2008Q2|N|N|N|2454619|2454770|2454276|2454551|N|N|N|N|N| +2454643|AAAAAAAADHEHFCAA|2008-06-25|1301|5661|435|2008|3|6|25|2|2008|435|5661|Wednesday|2008Q2|N|N|N|2454619|2454770|2454277|2454552|N|N|N|N|N| +2454644|AAAAAAAAEHEHFCAA|2008-06-26|1301|5661|435|2008|4|6|26|2|2008|435|5661|Thursday|2008Q2|N|N|N|2454619|2454770|2454278|2454553|N|N|N|N|N| +2454645|AAAAAAAAFHEHFCAA|2008-06-27|1301|5661|435|2008|5|6|27|2|2008|435|5661|Friday|2008Q2|N|Y|N|2454619|2454770|2454279|2454554|N|N|N|N|N| +2454646|AAAAAAAAGHEHFCAA|2008-06-28|1301|5661|435|2008|6|6|28|2|2008|435|5661|Saturday|2008Q2|N|Y|N|2454619|2454770|2454280|2454555|N|N|N|N|N| +2454647|AAAAAAAAHHEHFCAA|2008-06-29|1301|5661|435|2008|0|6|29|2|2008|435|5661|Sunday|2008Q2|N|N|N|2454619|2454770|2454281|2454556|N|N|N|N|N| +2454648|AAAAAAAAIHEHFCAA|2008-06-30|1301|5661|435|2008|1|6|30|2|2008|435|5661|Monday|2008Q2|N|N|N|2454619|2454770|2454282|2454557|N|N|N|N|N| +2454649|AAAAAAAAJHEHFCAA|2008-07-01|1302|5662|435|2008|2|7|1|3|2008|435|5662|Tuesday|2008Q3|N|N|N|2454649|2454830|2454283|2454558|N|N|N|N|N| +2454650|AAAAAAAAKHEHFCAA|2008-07-02|1302|5662|435|2008|3|7|2|3|2008|435|5662|Wednesday|2008Q3|N|N|N|2454649|2454830|2454284|2454559|N|N|N|N|N| +2454651|AAAAAAAALHEHFCAA|2008-07-03|1302|5662|435|2008|4|7|3|3|2008|435|5662|Thursday|2008Q3|N|N|N|2454649|2454830|2454285|2454560|N|N|N|N|N| +2454652|AAAAAAAAMHEHFCAA|2008-07-04|1302|5662|435|2008|5|7|4|3|2008|435|5662|Friday|2008Q3|Y|Y|N|2454649|2454830|2454286|2454561|N|N|N|N|N| +2454653|AAAAAAAANHEHFCAA|2008-07-05|1302|5662|435|2008|6|7|5|3|2008|435|5662|Saturday|2008Q3|N|Y|Y|2454649|2454830|2454287|2454562|N|N|N|N|N| +2454654|AAAAAAAAOHEHFCAA|2008-07-06|1302|5662|435|2008|0|7|6|3|2008|435|5662|Sunday|2008Q3|N|N|N|2454649|2454830|2454288|2454563|N|N|N|N|N| +2454655|AAAAAAAAPHEHFCAA|2008-07-07|1302|5662|435|2008|1|7|7|3|2008|435|5662|Monday|2008Q3|N|N|N|2454649|2454830|2454289|2454564|N|N|N|N|N| +2454656|AAAAAAAAAIEHFCAA|2008-07-08|1302|5663|435|2008|2|7|8|3|2008|435|5663|Tuesday|2008Q3|N|N|N|2454649|2454830|2454290|2454565|N|N|N|N|N| +2454657|AAAAAAAABIEHFCAA|2008-07-09|1302|5663|435|2008|3|7|9|3|2008|435|5663|Wednesday|2008Q3|N|N|N|2454649|2454830|2454291|2454566|N|N|N|N|N| +2454658|AAAAAAAACIEHFCAA|2008-07-10|1302|5663|435|2008|4|7|10|3|2008|435|5663|Thursday|2008Q3|N|N|N|2454649|2454830|2454292|2454567|N|N|N|N|N| +2454659|AAAAAAAADIEHFCAA|2008-07-11|1302|5663|435|2008|5|7|11|3|2008|435|5663|Friday|2008Q3|N|Y|N|2454649|2454830|2454293|2454568|N|N|N|N|N| +2454660|AAAAAAAAEIEHFCAA|2008-07-12|1302|5663|435|2008|6|7|12|3|2008|435|5663|Saturday|2008Q3|N|Y|N|2454649|2454830|2454294|2454569|N|N|N|N|N| +2454661|AAAAAAAAFIEHFCAA|2008-07-13|1302|5663|435|2008|0|7|13|3|2008|435|5663|Sunday|2008Q3|N|N|N|2454649|2454830|2454295|2454570|N|N|N|N|N| +2454662|AAAAAAAAGIEHFCAA|2008-07-14|1302|5663|435|2008|1|7|14|3|2008|435|5663|Monday|2008Q3|N|N|N|2454649|2454830|2454296|2454571|N|N|N|N|N| +2454663|AAAAAAAAHIEHFCAA|2008-07-15|1302|5664|435|2008|2|7|15|3|2008|435|5664|Tuesday|2008Q3|N|N|N|2454649|2454830|2454297|2454572|N|N|N|N|N| +2454664|AAAAAAAAIIEHFCAA|2008-07-16|1302|5664|435|2008|3|7|16|3|2008|435|5664|Wednesday|2008Q3|N|N|N|2454649|2454830|2454298|2454573|N|N|N|N|N| +2454665|AAAAAAAAJIEHFCAA|2008-07-17|1302|5664|435|2008|4|7|17|3|2008|435|5664|Thursday|2008Q3|N|N|N|2454649|2454830|2454299|2454574|N|N|N|N|N| +2454666|AAAAAAAAKIEHFCAA|2008-07-18|1302|5664|435|2008|5|7|18|3|2008|435|5664|Friday|2008Q3|N|Y|N|2454649|2454830|2454300|2454575|N|N|N|N|N| +2454667|AAAAAAAALIEHFCAA|2008-07-19|1302|5664|435|2008|6|7|19|3|2008|435|5664|Saturday|2008Q3|N|Y|N|2454649|2454830|2454301|2454576|N|N|N|N|N| +2454668|AAAAAAAAMIEHFCAA|2008-07-20|1302|5664|435|2008|0|7|20|3|2008|435|5664|Sunday|2008Q3|N|N|N|2454649|2454830|2454302|2454577|N|N|N|N|N| +2454669|AAAAAAAANIEHFCAA|2008-07-21|1302|5664|435|2008|1|7|21|3|2008|435|5664|Monday|2008Q3|N|N|N|2454649|2454830|2454303|2454578|N|N|N|N|N| +2454670|AAAAAAAAOIEHFCAA|2008-07-22|1302|5665|435|2008|2|7|22|3|2008|435|5665|Tuesday|2008Q3|N|N|N|2454649|2454830|2454304|2454579|N|N|N|N|N| +2454671|AAAAAAAAPIEHFCAA|2008-07-23|1302|5665|435|2008|3|7|23|3|2008|435|5665|Wednesday|2008Q3|N|N|N|2454649|2454830|2454305|2454580|N|N|N|N|N| +2454672|AAAAAAAAAJEHFCAA|2008-07-24|1302|5665|435|2008|4|7|24|3|2008|435|5665|Thursday|2008Q3|N|N|N|2454649|2454830|2454306|2454581|N|N|N|N|N| +2454673|AAAAAAAABJEHFCAA|2008-07-25|1302|5665|435|2008|5|7|25|3|2008|435|5665|Friday|2008Q3|N|Y|N|2454649|2454830|2454307|2454582|N|N|N|N|N| +2454674|AAAAAAAACJEHFCAA|2008-07-26|1302|5665|435|2008|6|7|26|3|2008|435|5665|Saturday|2008Q3|N|Y|N|2454649|2454830|2454308|2454583|N|N|N|N|N| +2454675|AAAAAAAADJEHFCAA|2008-07-27|1302|5665|435|2008|0|7|27|3|2008|435|5665|Sunday|2008Q3|N|N|N|2454649|2454830|2454309|2454584|N|N|N|N|N| +2454676|AAAAAAAAEJEHFCAA|2008-07-28|1302|5665|435|2008|1|7|28|3|2008|435|5665|Monday|2008Q3|N|N|N|2454649|2454830|2454310|2454585|N|N|N|N|N| +2454677|AAAAAAAAFJEHFCAA|2008-07-29|1302|5666|435|2008|2|7|29|3|2008|435|5666|Tuesday|2008Q3|N|N|N|2454649|2454830|2454311|2454586|N|N|N|N|N| +2454678|AAAAAAAAGJEHFCAA|2008-07-30|1302|5666|435|2008|3|7|30|3|2008|435|5666|Wednesday|2008Q3|N|N|N|2454649|2454830|2454312|2454587|N|N|N|N|N| +2454679|AAAAAAAAHJEHFCAA|2008-07-31|1302|5666|435|2008|4|7|31|3|2008|435|5666|Thursday|2008Q3|N|N|N|2454649|2454830|2454313|2454588|N|N|N|N|N| +2454680|AAAAAAAAIJEHFCAA|2008-08-01|1303|5666|435|2008|5|8|1|3|2008|435|5666|Friday|2008Q3|N|Y|N|2454680|2454892|2454314|2454589|N|N|N|N|N| +2454681|AAAAAAAAJJEHFCAA|2008-08-02|1303|5666|435|2008|6|8|2|3|2008|435|5666|Saturday|2008Q3|N|Y|N|2454680|2454892|2454315|2454590|N|N|N|N|N| +2454682|AAAAAAAAKJEHFCAA|2008-08-03|1303|5666|435|2008|0|8|3|3|2008|435|5666|Sunday|2008Q3|N|N|N|2454680|2454892|2454316|2454591|N|N|N|N|N| +2454683|AAAAAAAALJEHFCAA|2008-08-04|1303|5666|435|2008|1|8|4|3|2008|435|5666|Monday|2008Q3|N|N|N|2454680|2454892|2454317|2454592|N|N|N|N|N| +2454684|AAAAAAAAMJEHFCAA|2008-08-05|1303|5667|435|2008|2|8|5|3|2008|435|5667|Tuesday|2008Q3|N|N|N|2454680|2454892|2454318|2454593|N|N|N|N|N| +2454685|AAAAAAAANJEHFCAA|2008-08-06|1303|5667|435|2008|3|8|6|3|2008|435|5667|Wednesday|2008Q3|N|N|N|2454680|2454892|2454319|2454594|N|N|N|N|N| +2454686|AAAAAAAAOJEHFCAA|2008-08-07|1303|5667|435|2008|4|8|7|3|2008|435|5667|Thursday|2008Q3|N|N|N|2454680|2454892|2454320|2454595|N|N|N|N|N| +2454687|AAAAAAAAPJEHFCAA|2008-08-08|1303|5667|435|2008|5|8|8|3|2008|435|5667|Friday|2008Q3|N|Y|N|2454680|2454892|2454321|2454596|N|N|N|N|N| +2454688|AAAAAAAAAKEHFCAA|2008-08-09|1303|5667|435|2008|6|8|9|3|2008|435|5667|Saturday|2008Q3|N|Y|N|2454680|2454892|2454322|2454597|N|N|N|N|N| +2454689|AAAAAAAABKEHFCAA|2008-08-10|1303|5667|435|2008|0|8|10|3|2008|435|5667|Sunday|2008Q3|N|N|N|2454680|2454892|2454323|2454598|N|N|N|N|N| +2454690|AAAAAAAACKEHFCAA|2008-08-11|1303|5667|435|2008|1|8|11|3|2008|435|5667|Monday|2008Q3|N|N|N|2454680|2454892|2454324|2454599|N|N|N|N|N| +2454691|AAAAAAAADKEHFCAA|2008-08-12|1303|5668|435|2008|2|8|12|3|2008|435|5668|Tuesday|2008Q3|N|N|N|2454680|2454892|2454325|2454600|N|N|N|N|N| +2454692|AAAAAAAAEKEHFCAA|2008-08-13|1303|5668|435|2008|3|8|13|3|2008|435|5668|Wednesday|2008Q3|N|N|N|2454680|2454892|2454326|2454601|N|N|N|N|N| +2454693|AAAAAAAAFKEHFCAA|2008-08-14|1303|5668|435|2008|4|8|14|3|2008|435|5668|Thursday|2008Q3|N|N|N|2454680|2454892|2454327|2454602|N|N|N|N|N| +2454694|AAAAAAAAGKEHFCAA|2008-08-15|1303|5668|435|2008|5|8|15|3|2008|435|5668|Friday|2008Q3|N|Y|N|2454680|2454892|2454328|2454603|N|N|N|N|N| +2454695|AAAAAAAAHKEHFCAA|2008-08-16|1303|5668|435|2008|6|8|16|3|2008|435|5668|Saturday|2008Q3|N|Y|N|2454680|2454892|2454329|2454604|N|N|N|N|N| +2454696|AAAAAAAAIKEHFCAA|2008-08-17|1303|5668|435|2008|0|8|17|3|2008|435|5668|Sunday|2008Q3|N|N|N|2454680|2454892|2454330|2454605|N|N|N|N|N| +2454697|AAAAAAAAJKEHFCAA|2008-08-18|1303|5668|435|2008|1|8|18|3|2008|435|5668|Monday|2008Q3|N|N|N|2454680|2454892|2454331|2454606|N|N|N|N|N| +2454698|AAAAAAAAKKEHFCAA|2008-08-19|1303|5669|435|2008|2|8|19|3|2008|435|5669|Tuesday|2008Q3|N|N|N|2454680|2454892|2454332|2454607|N|N|N|N|N| +2454699|AAAAAAAALKEHFCAA|2008-08-20|1303|5669|435|2008|3|8|20|3|2008|435|5669|Wednesday|2008Q3|N|N|N|2454680|2454892|2454333|2454608|N|N|N|N|N| +2454700|AAAAAAAAMKEHFCAA|2008-08-21|1303|5669|435|2008|4|8|21|3|2008|435|5669|Thursday|2008Q3|N|N|N|2454680|2454892|2454334|2454609|N|N|N|N|N| +2454701|AAAAAAAANKEHFCAA|2008-08-22|1303|5669|435|2008|5|8|22|3|2008|435|5669|Friday|2008Q3|N|Y|N|2454680|2454892|2454335|2454610|N|N|N|N|N| +2454702|AAAAAAAAOKEHFCAA|2008-08-23|1303|5669|435|2008|6|8|23|3|2008|435|5669|Saturday|2008Q3|N|Y|N|2454680|2454892|2454336|2454611|N|N|N|N|N| +2454703|AAAAAAAAPKEHFCAA|2008-08-24|1303|5669|435|2008|0|8|24|3|2008|435|5669|Sunday|2008Q3|N|N|N|2454680|2454892|2454337|2454612|N|N|N|N|N| +2454704|AAAAAAAAALEHFCAA|2008-08-25|1303|5669|435|2008|1|8|25|3|2008|435|5669|Monday|2008Q3|N|N|N|2454680|2454892|2454338|2454613|N|N|N|N|N| +2454705|AAAAAAAABLEHFCAA|2008-08-26|1303|5670|435|2008|2|8|26|3|2008|435|5670|Tuesday|2008Q3|N|N|N|2454680|2454892|2454339|2454614|N|N|N|N|N| +2454706|AAAAAAAACLEHFCAA|2008-08-27|1303|5670|435|2008|3|8|27|3|2008|435|5670|Wednesday|2008Q3|N|N|N|2454680|2454892|2454340|2454615|N|N|N|N|N| +2454707|AAAAAAAADLEHFCAA|2008-08-28|1303|5670|435|2008|4|8|28|3|2008|435|5670|Thursday|2008Q3|N|N|N|2454680|2454892|2454341|2454616|N|N|N|N|N| +2454708|AAAAAAAAELEHFCAA|2008-08-29|1303|5670|435|2008|5|8|29|3|2008|435|5670|Friday|2008Q3|N|Y|N|2454680|2454892|2454342|2454617|N|N|N|N|N| +2454709|AAAAAAAAFLEHFCAA|2008-08-30|1303|5670|435|2008|6|8|30|3|2008|435|5670|Saturday|2008Q3|N|Y|N|2454680|2454892|2454343|2454618|N|N|N|N|N| +2454710|AAAAAAAAGLEHFCAA|2008-08-31|1303|5670|435|2008|0|8|31|3|2008|435|5670|Sunday|2008Q3|N|N|N|2454680|2454892|2454344|2454619|N|N|N|N|N| +2454711|AAAAAAAAHLEHFCAA|2008-09-01|1304|5670|436|2008|1|9|1|3|2008|436|5670|Monday|2008Q3|N|N|N|2454711|2454954|2454345|2454620|N|N|N|N|N| +2454712|AAAAAAAAILEHFCAA|2008-09-02|1304|5671|436|2008|2|9|2|3|2008|436|5671|Tuesday|2008Q3|N|N|N|2454711|2454954|2454346|2454621|N|N|N|N|N| +2454713|AAAAAAAAJLEHFCAA|2008-09-03|1304|5671|436|2008|3|9|3|3|2008|436|5671|Wednesday|2008Q3|N|N|N|2454711|2454954|2454347|2454622|N|N|N|N|N| +2454714|AAAAAAAAKLEHFCAA|2008-09-04|1304|5671|436|2008|4|9|4|3|2008|436|5671|Thursday|2008Q3|N|N|N|2454711|2454954|2454348|2454623|N|N|N|N|N| +2454715|AAAAAAAALLEHFCAA|2008-09-05|1304|5671|436|2008|5|9|5|3|2008|436|5671|Friday|2008Q3|N|Y|N|2454711|2454954|2454349|2454624|N|N|N|N|N| +2454716|AAAAAAAAMLEHFCAA|2008-09-06|1304|5671|436|2008|6|9|6|3|2008|436|5671|Saturday|2008Q3|N|Y|N|2454711|2454954|2454350|2454625|N|N|N|N|N| +2454717|AAAAAAAANLEHFCAA|2008-09-07|1304|5671|436|2008|0|9|7|3|2008|436|5671|Sunday|2008Q3|N|N|N|2454711|2454954|2454351|2454626|N|N|N|N|N| +2454718|AAAAAAAAOLEHFCAA|2008-09-08|1304|5671|436|2008|1|9|8|3|2008|436|5671|Monday|2008Q3|N|N|N|2454711|2454954|2454352|2454627|N|N|N|N|N| +2454719|AAAAAAAAPLEHFCAA|2008-09-09|1304|5672|436|2008|2|9|9|3|2008|436|5672|Tuesday|2008Q3|N|N|N|2454711|2454954|2454353|2454628|N|N|N|N|N| +2454720|AAAAAAAAAMEHFCAA|2008-09-10|1304|5672|436|2008|3|9|10|3|2008|436|5672|Wednesday|2008Q3|N|N|N|2454711|2454954|2454354|2454629|N|N|N|N|N| +2454721|AAAAAAAABMEHFCAA|2008-09-11|1304|5672|436|2008|4|9|11|3|2008|436|5672|Thursday|2008Q3|N|N|N|2454711|2454954|2454355|2454630|N|N|N|N|N| +2454722|AAAAAAAACMEHFCAA|2008-09-12|1304|5672|436|2008|5|9|12|3|2008|436|5672|Friday|2008Q3|N|Y|N|2454711|2454954|2454356|2454631|N|N|N|N|N| +2454723|AAAAAAAADMEHFCAA|2008-09-13|1304|5672|436|2008|6|9|13|3|2008|436|5672|Saturday|2008Q3|N|Y|N|2454711|2454954|2454357|2454632|N|N|N|N|N| +2454724|AAAAAAAAEMEHFCAA|2008-09-14|1304|5672|436|2008|0|9|14|3|2008|436|5672|Sunday|2008Q3|N|N|N|2454711|2454954|2454358|2454633|N|N|N|N|N| +2454725|AAAAAAAAFMEHFCAA|2008-09-15|1304|5672|436|2008|1|9|15|3|2008|436|5672|Monday|2008Q3|N|N|N|2454711|2454954|2454359|2454634|N|N|N|N|N| +2454726|AAAAAAAAGMEHFCAA|2008-09-16|1304|5673|436|2008|2|9|16|3|2008|436|5673|Tuesday|2008Q3|N|N|N|2454711|2454954|2454360|2454635|N|N|N|N|N| +2454727|AAAAAAAAHMEHFCAA|2008-09-17|1304|5673|436|2008|3|9|17|3|2008|436|5673|Wednesday|2008Q3|N|N|N|2454711|2454954|2454361|2454636|N|N|N|N|N| +2454728|AAAAAAAAIMEHFCAA|2008-09-18|1304|5673|436|2008|4|9|18|3|2008|436|5673|Thursday|2008Q3|N|N|N|2454711|2454954|2454362|2454637|N|N|N|N|N| +2454729|AAAAAAAAJMEHFCAA|2008-09-19|1304|5673|436|2008|5|9|19|3|2008|436|5673|Friday|2008Q3|N|Y|N|2454711|2454954|2454363|2454638|N|N|N|N|N| +2454730|AAAAAAAAKMEHFCAA|2008-09-20|1304|5673|436|2008|6|9|20|3|2008|436|5673|Saturday|2008Q3|N|Y|N|2454711|2454954|2454364|2454639|N|N|N|N|N| +2454731|AAAAAAAALMEHFCAA|2008-09-21|1304|5673|436|2008|0|9|21|3|2008|436|5673|Sunday|2008Q3|N|N|N|2454711|2454954|2454365|2454640|N|N|N|N|N| +2454732|AAAAAAAAMMEHFCAA|2008-09-22|1304|5673|436|2008|1|9|22|3|2008|436|5673|Monday|2008Q3|N|N|N|2454711|2454954|2454366|2454641|N|N|N|N|N| +2454733|AAAAAAAANMEHFCAA|2008-09-23|1304|5674|436|2008|2|9|23|3|2008|436|5674|Tuesday|2008Q3|N|N|N|2454711|2454954|2454367|2454642|N|N|N|N|N| +2454734|AAAAAAAAOMEHFCAA|2008-09-24|1304|5674|436|2008|3|9|24|3|2008|436|5674|Wednesday|2008Q3|N|N|N|2454711|2454954|2454368|2454643|N|N|N|N|N| +2454735|AAAAAAAAPMEHFCAA|2008-09-25|1304|5674|436|2008|4|9|25|3|2008|436|5674|Thursday|2008Q3|N|N|N|2454711|2454954|2454369|2454644|N|N|N|N|N| +2454736|AAAAAAAAANEHFCAA|2008-09-26|1304|5674|436|2008|5|9|26|3|2008|436|5674|Friday|2008Q3|N|Y|N|2454711|2454954|2454370|2454645|N|N|N|N|N| +2454737|AAAAAAAABNEHFCAA|2008-09-27|1304|5674|436|2008|6|9|27|3|2008|436|5674|Saturday|2008Q3|N|Y|N|2454711|2454954|2454371|2454646|N|N|N|N|N| +2454738|AAAAAAAACNEHFCAA|2008-09-28|1304|5674|436|2008|0|9|28|3|2008|436|5674|Sunday|2008Q3|N|N|N|2454711|2454954|2454372|2454647|N|N|N|N|N| +2454739|AAAAAAAADNEHFCAA|2008-09-29|1304|5674|436|2008|1|9|29|3|2008|436|5674|Monday|2008Q3|N|N|N|2454711|2454954|2454373|2454648|N|N|N|N|N| +2454740|AAAAAAAAENEHFCAA|2008-09-30|1304|5675|436|2008|2|9|30|3|2008|436|5675|Tuesday|2008Q3|N|N|N|2454711|2454954|2454374|2454649|N|N|N|N|N| +2454741|AAAAAAAAFNEHFCAA|2008-10-01|1305|5675|436|2008|3|10|1|4|2008|436|5675|Wednesday|2008Q4|N|N|N|2454741|2455014|2454375|2454649|N|N|N|N|N| +2454742|AAAAAAAAGNEHFCAA|2008-10-02|1305|5675|436|2008|4|10|2|4|2008|436|5675|Thursday|2008Q4|N|N|N|2454741|2455014|2454376|2454650|N|N|N|N|N| +2454743|AAAAAAAAHNEHFCAA|2008-10-03|1305|5675|436|2008|5|10|3|4|2008|436|5675|Friday|2008Q4|N|Y|N|2454741|2455014|2454377|2454651|N|N|N|N|N| +2454744|AAAAAAAAINEHFCAA|2008-10-04|1305|5675|436|2008|6|10|4|4|2008|436|5675|Saturday|2008Q4|N|Y|N|2454741|2455014|2454378|2454652|N|N|N|N|N| +2454745|AAAAAAAAJNEHFCAA|2008-10-05|1305|5675|436|2008|0|10|5|4|2008|436|5675|Sunday|2008Q4|N|N|N|2454741|2455014|2454379|2454653|N|N|N|N|N| +2454746|AAAAAAAAKNEHFCAA|2008-10-06|1305|5675|436|2008|1|10|6|4|2008|436|5675|Monday|2008Q4|N|N|N|2454741|2455014|2454380|2454654|N|N|N|N|N| +2454747|AAAAAAAALNEHFCAA|2008-10-07|1305|5676|436|2008|2|10|7|4|2008|436|5676|Tuesday|2008Q4|N|N|N|2454741|2455014|2454381|2454655|N|N|N|N|N| +2454748|AAAAAAAAMNEHFCAA|2008-10-08|1305|5676|436|2008|3|10|8|4|2008|436|5676|Wednesday|2008Q4|N|N|N|2454741|2455014|2454382|2454656|N|N|N|N|N| +2454749|AAAAAAAANNEHFCAA|2008-10-09|1305|5676|436|2008|4|10|9|4|2008|436|5676|Thursday|2008Q4|N|N|N|2454741|2455014|2454383|2454657|N|N|N|N|N| +2454750|AAAAAAAAONEHFCAA|2008-10-10|1305|5676|436|2008|5|10|10|4|2008|436|5676|Friday|2008Q4|N|Y|N|2454741|2455014|2454384|2454658|N|N|N|N|N| +2454751|AAAAAAAAPNEHFCAA|2008-10-11|1305|5676|436|2008|6|10|11|4|2008|436|5676|Saturday|2008Q4|N|Y|N|2454741|2455014|2454385|2454659|N|N|N|N|N| +2454752|AAAAAAAAAOEHFCAA|2008-10-12|1305|5676|436|2008|0|10|12|4|2008|436|5676|Sunday|2008Q4|N|N|N|2454741|2455014|2454386|2454660|N|N|N|N|N| +2454753|AAAAAAAABOEHFCAA|2008-10-13|1305|5676|436|2008|1|10|13|4|2008|436|5676|Monday|2008Q4|N|N|N|2454741|2455014|2454387|2454661|N|N|N|N|N| +2454754|AAAAAAAACOEHFCAA|2008-10-14|1305|5677|436|2008|2|10|14|4|2008|436|5677|Tuesday|2008Q4|N|N|N|2454741|2455014|2454388|2454662|N|N|N|N|N| +2454755|AAAAAAAADOEHFCAA|2008-10-15|1305|5677|436|2008|3|10|15|4|2008|436|5677|Wednesday|2008Q4|N|N|N|2454741|2455014|2454389|2454663|N|N|N|N|N| +2454756|AAAAAAAAEOEHFCAA|2008-10-16|1305|5677|436|2008|4|10|16|4|2008|436|5677|Thursday|2008Q4|N|N|N|2454741|2455014|2454390|2454664|N|N|N|N|N| +2454757|AAAAAAAAFOEHFCAA|2008-10-17|1305|5677|436|2008|5|10|17|4|2008|436|5677|Friday|2008Q4|N|Y|N|2454741|2455014|2454391|2454665|N|N|N|N|N| +2454758|AAAAAAAAGOEHFCAA|2008-10-18|1305|5677|436|2008|6|10|18|4|2008|436|5677|Saturday|2008Q4|N|Y|N|2454741|2455014|2454392|2454666|N|N|N|N|N| +2454759|AAAAAAAAHOEHFCAA|2008-10-19|1305|5677|436|2008|0|10|19|4|2008|436|5677|Sunday|2008Q4|N|N|N|2454741|2455014|2454393|2454667|N|N|N|N|N| +2454760|AAAAAAAAIOEHFCAA|2008-10-20|1305|5677|436|2008|1|10|20|4|2008|436|5677|Monday|2008Q4|N|N|N|2454741|2455014|2454394|2454668|N|N|N|N|N| +2454761|AAAAAAAAJOEHFCAA|2008-10-21|1305|5678|436|2008|2|10|21|4|2008|436|5678|Tuesday|2008Q4|N|N|N|2454741|2455014|2454395|2454669|N|N|N|N|N| +2454762|AAAAAAAAKOEHFCAA|2008-10-22|1305|5678|436|2008|3|10|22|4|2008|436|5678|Wednesday|2008Q4|N|N|N|2454741|2455014|2454396|2454670|N|N|N|N|N| +2454763|AAAAAAAALOEHFCAA|2008-10-23|1305|5678|436|2008|4|10|23|4|2008|436|5678|Thursday|2008Q4|N|N|N|2454741|2455014|2454397|2454671|N|N|N|N|N| +2454764|AAAAAAAAMOEHFCAA|2008-10-24|1305|5678|436|2008|5|10|24|4|2008|436|5678|Friday|2008Q4|N|Y|N|2454741|2455014|2454398|2454672|N|N|N|N|N| +2454765|AAAAAAAANOEHFCAA|2008-10-25|1305|5678|436|2008|6|10|25|4|2008|436|5678|Saturday|2008Q4|N|Y|N|2454741|2455014|2454399|2454673|N|N|N|N|N| +2454766|AAAAAAAAOOEHFCAA|2008-10-26|1305|5678|436|2008|0|10|26|4|2008|436|5678|Sunday|2008Q4|N|N|N|2454741|2455014|2454400|2454674|N|N|N|N|N| +2454767|AAAAAAAAPOEHFCAA|2008-10-27|1305|5678|436|2008|1|10|27|4|2008|436|5678|Monday|2008Q4|N|N|N|2454741|2455014|2454401|2454675|N|N|N|N|N| +2454768|AAAAAAAAAPEHFCAA|2008-10-28|1305|5679|436|2008|2|10|28|4|2008|436|5679|Tuesday|2008Q4|N|N|N|2454741|2455014|2454402|2454676|N|N|N|N|N| +2454769|AAAAAAAABPEHFCAA|2008-10-29|1305|5679|436|2008|3|10|29|4|2008|436|5679|Wednesday|2008Q4|N|N|N|2454741|2455014|2454403|2454677|N|N|N|N|N| +2454770|AAAAAAAACPEHFCAA|2008-10-30|1305|5679|436|2008|4|10|30|4|2008|436|5679|Thursday|2008Q4|N|N|N|2454741|2455014|2454404|2454678|N|N|N|N|N| +2454771|AAAAAAAADPEHFCAA|2008-10-31|1305|5679|436|2008|5|10|31|4|2008|436|5679|Friday|2008Q4|N|Y|N|2454741|2455014|2454405|2454679|N|N|N|N|N| +2454772|AAAAAAAAEPEHFCAA|2008-11-01|1306|5679|436|2008|6|11|1|4|2008|436|5679|Saturday|2008Q4|N|Y|N|2454772|2455076|2454406|2454680|N|N|N|N|N| +2454773|AAAAAAAAFPEHFCAA|2008-11-02|1306|5679|436|2008|0|11|2|4|2008|436|5679|Sunday|2008Q4|N|N|N|2454772|2455076|2454407|2454681|N|N|N|N|N| +2454774|AAAAAAAAGPEHFCAA|2008-11-03|1306|5679|436|2008|1|11|3|4|2008|436|5679|Monday|2008Q4|N|N|N|2454772|2455076|2454408|2454682|N|N|N|N|N| +2454775|AAAAAAAAHPEHFCAA|2008-11-04|1306|5680|436|2008|2|11|4|4|2008|436|5680|Tuesday|2008Q4|N|N|N|2454772|2455076|2454409|2454683|N|N|N|N|N| +2454776|AAAAAAAAIPEHFCAA|2008-11-05|1306|5680|436|2008|3|11|5|4|2008|436|5680|Wednesday|2008Q4|N|N|N|2454772|2455076|2454410|2454684|N|N|N|N|N| +2454777|AAAAAAAAJPEHFCAA|2008-11-06|1306|5680|436|2008|4|11|6|4|2008|436|5680|Thursday|2008Q4|N|N|N|2454772|2455076|2454411|2454685|N|N|N|N|N| +2454778|AAAAAAAAKPEHFCAA|2008-11-07|1306|5680|436|2008|5|11|7|4|2008|436|5680|Friday|2008Q4|N|Y|N|2454772|2455076|2454412|2454686|N|N|N|N|N| +2454779|AAAAAAAALPEHFCAA|2008-11-08|1306|5680|436|2008|6|11|8|4|2008|436|5680|Saturday|2008Q4|N|Y|N|2454772|2455076|2454413|2454687|N|N|N|N|N| +2454780|AAAAAAAAMPEHFCAA|2008-11-09|1306|5680|436|2008|0|11|9|4|2008|436|5680|Sunday|2008Q4|N|N|N|2454772|2455076|2454414|2454688|N|N|N|N|N| +2454781|AAAAAAAANPEHFCAA|2008-11-10|1306|5680|436|2008|1|11|10|4|2008|436|5680|Monday|2008Q4|N|N|N|2454772|2455076|2454415|2454689|N|N|N|N|N| +2454782|AAAAAAAAOPEHFCAA|2008-11-11|1306|5681|436|2008|2|11|11|4|2008|436|5681|Tuesday|2008Q4|N|N|N|2454772|2455076|2454416|2454690|N|N|N|N|N| +2454783|AAAAAAAAPPEHFCAA|2008-11-12|1306|5681|436|2008|3|11|12|4|2008|436|5681|Wednesday|2008Q4|N|N|N|2454772|2455076|2454417|2454691|N|N|N|N|N| +2454784|AAAAAAAAAAFHFCAA|2008-11-13|1306|5681|436|2008|4|11|13|4|2008|436|5681|Thursday|2008Q4|N|N|N|2454772|2455076|2454418|2454692|N|N|N|N|N| +2454785|AAAAAAAABAFHFCAA|2008-11-14|1306|5681|436|2008|5|11|14|4|2008|436|5681|Friday|2008Q4|N|Y|N|2454772|2455076|2454419|2454693|N|N|N|N|N| +2454786|AAAAAAAACAFHFCAA|2008-11-15|1306|5681|436|2008|6|11|15|4|2008|436|5681|Saturday|2008Q4|N|Y|N|2454772|2455076|2454420|2454694|N|N|N|N|N| +2454787|AAAAAAAADAFHFCAA|2008-11-16|1306|5681|436|2008|0|11|16|4|2008|436|5681|Sunday|2008Q4|N|N|N|2454772|2455076|2454421|2454695|N|N|N|N|N| +2454788|AAAAAAAAEAFHFCAA|2008-11-17|1306|5681|436|2008|1|11|17|4|2008|436|5681|Monday|2008Q4|N|N|N|2454772|2455076|2454422|2454696|N|N|N|N|N| +2454789|AAAAAAAAFAFHFCAA|2008-11-18|1306|5682|436|2008|2|11|18|4|2008|436|5682|Tuesday|2008Q4|N|N|N|2454772|2455076|2454423|2454697|N|N|N|N|N| +2454790|AAAAAAAAGAFHFCAA|2008-11-19|1306|5682|436|2008|3|11|19|4|2008|436|5682|Wednesday|2008Q4|N|N|N|2454772|2455076|2454424|2454698|N|N|N|N|N| +2454791|AAAAAAAAHAFHFCAA|2008-11-20|1306|5682|436|2008|4|11|20|4|2008|436|5682|Thursday|2008Q4|N|N|N|2454772|2455076|2454425|2454699|N|N|N|N|N| +2454792|AAAAAAAAIAFHFCAA|2008-11-21|1306|5682|436|2008|5|11|21|4|2008|436|5682|Friday|2008Q4|N|Y|N|2454772|2455076|2454426|2454700|N|N|N|N|N| +2454793|AAAAAAAAJAFHFCAA|2008-11-22|1306|5682|436|2008|6|11|22|4|2008|436|5682|Saturday|2008Q4|N|Y|N|2454772|2455076|2454427|2454701|N|N|N|N|N| +2454794|AAAAAAAAKAFHFCAA|2008-11-23|1306|5682|436|2008|0|11|23|4|2008|436|5682|Sunday|2008Q4|N|N|N|2454772|2455076|2454428|2454702|N|N|N|N|N| +2454795|AAAAAAAALAFHFCAA|2008-11-24|1306|5682|436|2008|1|11|24|4|2008|436|5682|Monday|2008Q4|N|N|N|2454772|2455076|2454429|2454703|N|N|N|N|N| +2454796|AAAAAAAAMAFHFCAA|2008-11-25|1306|5683|436|2008|2|11|25|4|2008|436|5683|Tuesday|2008Q4|N|N|N|2454772|2455076|2454430|2454704|N|N|N|N|N| +2454797|AAAAAAAANAFHFCAA|2008-11-26|1306|5683|436|2008|3|11|26|4|2008|436|5683|Wednesday|2008Q4|N|N|N|2454772|2455076|2454431|2454705|N|N|N|N|N| +2454798|AAAAAAAAOAFHFCAA|2008-11-27|1306|5683|436|2008|4|11|27|4|2008|436|5683|Thursday|2008Q4|N|N|N|2454772|2455076|2454432|2454706|N|N|N|N|N| +2454799|AAAAAAAAPAFHFCAA|2008-11-28|1306|5683|436|2008|5|11|28|4|2008|436|5683|Friday|2008Q4|N|Y|N|2454772|2455076|2454433|2454707|N|N|N|N|N| +2454800|AAAAAAAAABFHFCAA|2008-11-29|1306|5683|436|2008|6|11|29|4|2008|436|5683|Saturday|2008Q4|N|Y|N|2454772|2455076|2454434|2454708|N|N|N|N|N| +2454801|AAAAAAAABBFHFCAA|2008-11-30|1306|5683|436|2008|0|11|30|4|2008|436|5683|Sunday|2008Q4|N|N|N|2454772|2455076|2454435|2454709|N|N|N|N|N| +2454802|AAAAAAAACBFHFCAA|2008-12-01|1307|5683|437|2008|1|12|1|4|2008|437|5683|Monday|2008Q4|N|N|N|2454802|2455136|2454436|2454710|N|N|N|N|N| +2454803|AAAAAAAADBFHFCAA|2008-12-02|1307|5684|437|2008|2|12|2|4|2008|437|5684|Tuesday|2008Q4|N|N|N|2454802|2455136|2454437|2454711|N|N|N|N|N| +2454804|AAAAAAAAEBFHFCAA|2008-12-03|1307|5684|437|2008|3|12|3|4|2008|437|5684|Wednesday|2008Q4|N|N|N|2454802|2455136|2454438|2454712|N|N|N|N|N| +2454805|AAAAAAAAFBFHFCAA|2008-12-04|1307|5684|437|2008|4|12|4|4|2008|437|5684|Thursday|2008Q4|N|N|N|2454802|2455136|2454439|2454713|N|N|N|N|N| +2454806|AAAAAAAAGBFHFCAA|2008-12-05|1307|5684|437|2008|5|12|5|4|2008|437|5684|Friday|2008Q4|N|Y|N|2454802|2455136|2454440|2454714|N|N|N|N|N| +2454807|AAAAAAAAHBFHFCAA|2008-12-06|1307|5684|437|2008|6|12|6|4|2008|437|5684|Saturday|2008Q4|N|Y|N|2454802|2455136|2454441|2454715|N|N|N|N|N| +2454808|AAAAAAAAIBFHFCAA|2008-12-07|1307|5684|437|2008|0|12|7|4|2008|437|5684|Sunday|2008Q4|N|N|N|2454802|2455136|2454442|2454716|N|N|N|N|N| +2454809|AAAAAAAAJBFHFCAA|2008-12-08|1307|5684|437|2008|1|12|8|4|2008|437|5684|Monday|2008Q4|N|N|N|2454802|2455136|2454443|2454717|N|N|N|N|N| +2454810|AAAAAAAAKBFHFCAA|2008-12-09|1307|5685|437|2008|2|12|9|4|2008|437|5685|Tuesday|2008Q4|N|N|N|2454802|2455136|2454444|2454718|N|N|N|N|N| +2454811|AAAAAAAALBFHFCAA|2008-12-10|1307|5685|437|2008|3|12|10|4|2008|437|5685|Wednesday|2008Q4|N|N|N|2454802|2455136|2454445|2454719|N|N|N|N|N| +2454812|AAAAAAAAMBFHFCAA|2008-12-11|1307|5685|437|2008|4|12|11|4|2008|437|5685|Thursday|2008Q4|N|N|N|2454802|2455136|2454446|2454720|N|N|N|N|N| +2454813|AAAAAAAANBFHFCAA|2008-12-12|1307|5685|437|2008|5|12|12|4|2008|437|5685|Friday|2008Q4|N|Y|N|2454802|2455136|2454447|2454721|N|N|N|N|N| +2454814|AAAAAAAAOBFHFCAA|2008-12-13|1307|5685|437|2008|6|12|13|4|2008|437|5685|Saturday|2008Q4|N|Y|N|2454802|2455136|2454448|2454722|N|N|N|N|N| +2454815|AAAAAAAAPBFHFCAA|2008-12-14|1307|5685|437|2008|0|12|14|4|2008|437|5685|Sunday|2008Q4|N|N|N|2454802|2455136|2454449|2454723|N|N|N|N|N| +2454816|AAAAAAAAACFHFCAA|2008-12-15|1307|5685|437|2008|1|12|15|4|2008|437|5685|Monday|2008Q4|N|N|N|2454802|2455136|2454450|2454724|N|N|N|N|N| +2454817|AAAAAAAABCFHFCAA|2008-12-16|1307|5686|437|2008|2|12|16|4|2008|437|5686|Tuesday|2008Q4|N|N|N|2454802|2455136|2454451|2454725|N|N|N|N|N| +2454818|AAAAAAAACCFHFCAA|2008-12-17|1307|5686|437|2008|3|12|17|4|2008|437|5686|Wednesday|2008Q4|N|N|N|2454802|2455136|2454452|2454726|N|N|N|N|N| +2454819|AAAAAAAADCFHFCAA|2008-12-18|1307|5686|437|2008|4|12|18|4|2008|437|5686|Thursday|2008Q4|N|N|N|2454802|2455136|2454453|2454727|N|N|N|N|N| +2454820|AAAAAAAAECFHFCAA|2008-12-19|1307|5686|437|2008|5|12|19|4|2008|437|5686|Friday|2008Q4|N|Y|N|2454802|2455136|2454454|2454728|N|N|N|N|N| +2454821|AAAAAAAAFCFHFCAA|2008-12-20|1307|5686|437|2008|6|12|20|4|2008|437|5686|Saturday|2008Q4|N|Y|N|2454802|2455136|2454455|2454729|N|N|N|N|N| +2454822|AAAAAAAAGCFHFCAA|2008-12-21|1307|5686|437|2008|0|12|21|4|2008|437|5686|Sunday|2008Q4|N|N|N|2454802|2455136|2454456|2454730|N|N|N|N|N| +2454823|AAAAAAAAHCFHFCAA|2008-12-22|1307|5686|437|2008|1|12|22|4|2008|437|5686|Monday|2008Q4|N|N|N|2454802|2455136|2454457|2454731|N|N|N|N|N| +2454824|AAAAAAAAICFHFCAA|2008-12-23|1307|5687|437|2008|2|12|23|4|2008|437|5687|Tuesday|2008Q4|N|N|N|2454802|2455136|2454458|2454732|N|N|N|N|N| +2454825|AAAAAAAAJCFHFCAA|2008-12-24|1307|5687|437|2008|3|12|24|4|2008|437|5687|Wednesday|2008Q4|N|N|N|2454802|2455136|2454459|2454733|N|N|N|N|N| +2454826|AAAAAAAAKCFHFCAA|2008-12-25|1307|5687|437|2008|4|12|25|4|2008|437|5687|Thursday|2008Q4|Y|N|N|2454802|2455136|2454460|2454734|N|N|N|N|N| +2454827|AAAAAAAALCFHFCAA|2008-12-26|1307|5687|437|2008|5|12|26|4|2008|437|5687|Friday|2008Q4|N|Y|Y|2454802|2455136|2454461|2454735|N|N|N|N|N| +2454828|AAAAAAAAMCFHFCAA|2008-12-27|1307|5687|437|2008|6|12|27|4|2008|437|5687|Saturday|2008Q4|N|Y|N|2454802|2455136|2454462|2454736|N|N|N|N|N| +2454829|AAAAAAAANCFHFCAA|2008-12-28|1307|5687|437|2008|0|12|28|4|2008|437|5687|Sunday|2008Q4|N|N|N|2454802|2455136|2454463|2454737|N|N|N|N|N| +2454830|AAAAAAAAOCFHFCAA|2008-12-29|1307|5687|437|2008|1|12|29|4|2008|437|5687|Monday|2008Q4|N|N|N|2454802|2455136|2454464|2454738|N|N|N|N|N| +2454831|AAAAAAAAPCFHFCAA|2008-12-30|1307|5688|437|2008|2|12|30|4|2008|437|5688|Tuesday|2008Q4|N|N|N|2454802|2455136|2454465|2454739|N|N|N|N|N| +2454832|AAAAAAAAADFHFCAA|2008-12-31|1307|5688|437|2008|3|12|31|4|2008|437|5688|Wednesday|2008Q4|Y|N|N|2454802|2455136|2454466|2454740|N|N|N|N|N| +2454833|AAAAAAAABDFHFCAA|2009-01-01|1308|5688|437|2009|4|1|1|1|2009|437|5688|Thursday|2009Q1|Y|N|Y|2454833|2454832|2454467|2454741|N|N|N|N|N| +2454834|AAAAAAAACDFHFCAA|2009-01-02|1308|5688|437|2009|5|1|2|1|2009|437|5688|Friday|2009Q1|N|Y|Y|2454833|2454832|2454468|2454742|N|N|N|N|N| +2454835|AAAAAAAADDFHFCAA|2009-01-03|1308|5688|437|2009|6|1|3|1|2009|437|5688|Saturday|2009Q1|N|Y|N|2454833|2454832|2454469|2454743|N|N|N|N|N| +2454836|AAAAAAAAEDFHFCAA|2009-01-04|1308|5688|437|2009|0|1|4|1|2009|437|5688|Sunday|2009Q1|N|N|N|2454833|2454832|2454470|2454744|N|N|N|N|N| +2454837|AAAAAAAAFDFHFCAA|2009-01-05|1308|5688|437|2009|1|1|5|1|2009|437|5688|Monday|2009Q1|N|N|N|2454833|2454832|2454471|2454745|N|N|N|N|N| +2454838|AAAAAAAAGDFHFCAA|2009-01-06|1308|5689|437|2009|2|1|6|1|2009|437|5689|Tuesday|2009Q1|N|N|N|2454833|2454832|2454472|2454746|N|N|N|N|N| +2454839|AAAAAAAAHDFHFCAA|2009-01-07|1308|5689|437|2009|3|1|7|1|2009|437|5689|Wednesday|2009Q1|N|N|N|2454833|2454832|2454473|2454747|N|N|N|N|N| +2454840|AAAAAAAAIDFHFCAA|2009-01-08|1308|5689|437|2009|4|1|8|1|2009|437|5689|Thursday|2009Q1|N|N|N|2454833|2454832|2454474|2454748|N|N|N|N|N| +2454841|AAAAAAAAJDFHFCAA|2009-01-09|1308|5689|437|2009|5|1|9|1|2009|437|5689|Friday|2009Q1|N|Y|N|2454833|2454832|2454475|2454749|N|N|N|N|N| +2454842|AAAAAAAAKDFHFCAA|2009-01-10|1308|5689|437|2009|6|1|10|1|2009|437|5689|Saturday|2009Q1|N|Y|N|2454833|2454832|2454476|2454750|N|N|N|N|N| +2454843|AAAAAAAALDFHFCAA|2009-01-11|1308|5689|437|2009|0|1|11|1|2009|437|5689|Sunday|2009Q1|N|N|N|2454833|2454832|2454477|2454751|N|N|N|N|N| +2454844|AAAAAAAAMDFHFCAA|2009-01-12|1308|5689|437|2009|1|1|12|1|2009|437|5689|Monday|2009Q1|N|N|N|2454833|2454832|2454478|2454752|N|N|N|N|N| +2454845|AAAAAAAANDFHFCAA|2009-01-13|1308|5690|437|2009|2|1|13|1|2009|437|5690|Tuesday|2009Q1|N|N|N|2454833|2454832|2454479|2454753|N|N|N|N|N| +2454846|AAAAAAAAODFHFCAA|2009-01-14|1308|5690|437|2009|3|1|14|1|2009|437|5690|Wednesday|2009Q1|N|N|N|2454833|2454832|2454480|2454754|N|N|N|N|N| +2454847|AAAAAAAAPDFHFCAA|2009-01-15|1308|5690|437|2009|4|1|15|1|2009|437|5690|Thursday|2009Q1|N|N|N|2454833|2454832|2454481|2454755|N|N|N|N|N| +2454848|AAAAAAAAAEFHFCAA|2009-01-16|1308|5690|437|2009|5|1|16|1|2009|437|5690|Friday|2009Q1|N|Y|N|2454833|2454832|2454482|2454756|N|N|N|N|N| +2454849|AAAAAAAABEFHFCAA|2009-01-17|1308|5690|437|2009|6|1|17|1|2009|437|5690|Saturday|2009Q1|N|Y|N|2454833|2454832|2454483|2454757|N|N|N|N|N| +2454850|AAAAAAAACEFHFCAA|2009-01-18|1308|5690|437|2009|0|1|18|1|2009|437|5690|Sunday|2009Q1|N|N|N|2454833|2454832|2454484|2454758|N|N|N|N|N| +2454851|AAAAAAAADEFHFCAA|2009-01-19|1308|5690|437|2009|1|1|19|1|2009|437|5690|Monday|2009Q1|N|N|N|2454833|2454832|2454485|2454759|N|N|N|N|N| +2454852|AAAAAAAAEEFHFCAA|2009-01-20|1308|5691|437|2009|2|1|20|1|2009|437|5691|Tuesday|2009Q1|N|N|N|2454833|2454832|2454486|2454760|N|N|N|N|N| +2454853|AAAAAAAAFEFHFCAA|2009-01-21|1308|5691|437|2009|3|1|21|1|2009|437|5691|Wednesday|2009Q1|N|N|N|2454833|2454832|2454487|2454761|N|N|N|N|N| +2454854|AAAAAAAAGEFHFCAA|2009-01-22|1308|5691|437|2009|4|1|22|1|2009|437|5691|Thursday|2009Q1|N|N|N|2454833|2454832|2454488|2454762|N|N|N|N|N| +2454855|AAAAAAAAHEFHFCAA|2009-01-23|1308|5691|437|2009|5|1|23|1|2009|437|5691|Friday|2009Q1|N|Y|N|2454833|2454832|2454489|2454763|N|N|N|N|N| +2454856|AAAAAAAAIEFHFCAA|2009-01-24|1308|5691|437|2009|6|1|24|1|2009|437|5691|Saturday|2009Q1|N|Y|N|2454833|2454832|2454490|2454764|N|N|N|N|N| +2454857|AAAAAAAAJEFHFCAA|2009-01-25|1308|5691|437|2009|0|1|25|1|2009|437|5691|Sunday|2009Q1|N|N|N|2454833|2454832|2454491|2454765|N|N|N|N|N| +2454858|AAAAAAAAKEFHFCAA|2009-01-26|1308|5691|437|2009|1|1|26|1|2009|437|5691|Monday|2009Q1|N|N|N|2454833|2454832|2454492|2454766|N|N|N|N|N| +2454859|AAAAAAAALEFHFCAA|2009-01-27|1308|5692|437|2009|2|1|27|1|2009|437|5692|Tuesday|2009Q1|N|N|N|2454833|2454832|2454493|2454767|N|N|N|N|N| +2454860|AAAAAAAAMEFHFCAA|2009-01-28|1308|5692|437|2009|3|1|28|1|2009|437|5692|Wednesday|2009Q1|N|N|N|2454833|2454832|2454494|2454768|N|N|N|N|N| +2454861|AAAAAAAANEFHFCAA|2009-01-29|1308|5692|437|2009|4|1|29|1|2009|437|5692|Thursday|2009Q1|N|N|N|2454833|2454832|2454495|2454769|N|N|N|N|N| +2454862|AAAAAAAAOEFHFCAA|2009-01-30|1308|5692|437|2009|5|1|30|1|2009|437|5692|Friday|2009Q1|N|Y|N|2454833|2454832|2454496|2454770|N|N|N|N|N| +2454863|AAAAAAAAPEFHFCAA|2009-01-31|1308|5692|437|2009|6|1|31|1|2009|437|5692|Saturday|2009Q1|N|Y|N|2454833|2454832|2454497|2454771|N|N|N|N|N| +2454864|AAAAAAAAAFFHFCAA|2009-02-01|1309|5692|437|2009|0|2|1|1|2009|437|5692|Sunday|2009Q1|N|N|N|2454864|2454894|2454498|2454772|N|N|N|N|N| +2454865|AAAAAAAABFFHFCAA|2009-02-02|1309|5692|437|2009|1|2|2|1|2009|437|5692|Monday|2009Q1|N|N|N|2454864|2454894|2454499|2454773|N|N|N|N|N| +2454866|AAAAAAAACFFHFCAA|2009-02-03|1309|5693|437|2009|2|2|3|1|2009|437|5693|Tuesday|2009Q1|N|N|N|2454864|2454894|2454500|2454774|N|N|N|N|N| +2454867|AAAAAAAADFFHFCAA|2009-02-04|1309|5693|437|2009|3|2|4|1|2009|437|5693|Wednesday|2009Q1|N|N|N|2454864|2454894|2454501|2454775|N|N|N|N|N| +2454868|AAAAAAAAEFFHFCAA|2009-02-05|1309|5693|437|2009|4|2|5|1|2009|437|5693|Thursday|2009Q1|N|N|N|2454864|2454894|2454502|2454776|N|N|N|N|N| +2454869|AAAAAAAAFFFHFCAA|2009-02-06|1309|5693|437|2009|5|2|6|1|2009|437|5693|Friday|2009Q1|N|Y|N|2454864|2454894|2454503|2454777|N|N|N|N|N| +2454870|AAAAAAAAGFFHFCAA|2009-02-07|1309|5693|437|2009|6|2|7|1|2009|437|5693|Saturday|2009Q1|N|Y|N|2454864|2454894|2454504|2454778|N|N|N|N|N| +2454871|AAAAAAAAHFFHFCAA|2009-02-08|1309|5693|437|2009|0|2|8|1|2009|437|5693|Sunday|2009Q1|N|N|N|2454864|2454894|2454505|2454779|N|N|N|N|N| +2454872|AAAAAAAAIFFHFCAA|2009-02-09|1309|5693|437|2009|1|2|9|1|2009|437|5693|Monday|2009Q1|N|N|N|2454864|2454894|2454506|2454780|N|N|N|N|N| +2454873|AAAAAAAAJFFHFCAA|2009-02-10|1309|5694|437|2009|2|2|10|1|2009|437|5694|Tuesday|2009Q1|N|N|N|2454864|2454894|2454507|2454781|N|N|N|N|N| +2454874|AAAAAAAAKFFHFCAA|2009-02-11|1309|5694|437|2009|3|2|11|1|2009|437|5694|Wednesday|2009Q1|N|N|N|2454864|2454894|2454508|2454782|N|N|N|N|N| +2454875|AAAAAAAALFFHFCAA|2009-02-12|1309|5694|437|2009|4|2|12|1|2009|437|5694|Thursday|2009Q1|N|N|N|2454864|2454894|2454509|2454783|N|N|N|N|N| +2454876|AAAAAAAAMFFHFCAA|2009-02-13|1309|5694|437|2009|5|2|13|1|2009|437|5694|Friday|2009Q1|N|Y|N|2454864|2454894|2454510|2454784|N|N|N|N|N| +2454877|AAAAAAAANFFHFCAA|2009-02-14|1309|5694|437|2009|6|2|14|1|2009|437|5694|Saturday|2009Q1|N|Y|N|2454864|2454894|2454511|2454785|N|N|N|N|N| +2454878|AAAAAAAAOFFHFCAA|2009-02-15|1309|5694|437|2009|0|2|15|1|2009|437|5694|Sunday|2009Q1|N|N|N|2454864|2454894|2454512|2454786|N|N|N|N|N| +2454879|AAAAAAAAPFFHFCAA|2009-02-16|1309|5694|437|2009|1|2|16|1|2009|437|5694|Monday|2009Q1|N|N|N|2454864|2454894|2454513|2454787|N|N|N|N|N| +2454880|AAAAAAAAAGFHFCAA|2009-02-17|1309|5695|437|2009|2|2|17|1|2009|437|5695|Tuesday|2009Q1|N|N|N|2454864|2454894|2454514|2454788|N|N|N|N|N| +2454881|AAAAAAAABGFHFCAA|2009-02-18|1309|5695|437|2009|3|2|18|1|2009|437|5695|Wednesday|2009Q1|N|N|N|2454864|2454894|2454515|2454789|N|N|N|N|N| +2454882|AAAAAAAACGFHFCAA|2009-02-19|1309|5695|437|2009|4|2|19|1|2009|437|5695|Thursday|2009Q1|N|N|N|2454864|2454894|2454516|2454790|N|N|N|N|N| +2454883|AAAAAAAADGFHFCAA|2009-02-20|1309|5695|437|2009|5|2|20|1|2009|437|5695|Friday|2009Q1|N|Y|N|2454864|2454894|2454517|2454791|N|N|N|N|N| +2454884|AAAAAAAAEGFHFCAA|2009-02-21|1309|5695|437|2009|6|2|21|1|2009|437|5695|Saturday|2009Q1|N|Y|N|2454864|2454894|2454518|2454792|N|N|N|N|N| +2454885|AAAAAAAAFGFHFCAA|2009-02-22|1309|5695|437|2009|0|2|22|1|2009|437|5695|Sunday|2009Q1|N|N|N|2454864|2454894|2454519|2454793|N|N|N|N|N| +2454886|AAAAAAAAGGFHFCAA|2009-02-23|1309|5695|437|2009|1|2|23|1|2009|437|5695|Monday|2009Q1|N|N|N|2454864|2454894|2454520|2454794|N|N|N|N|N| +2454887|AAAAAAAAHGFHFCAA|2009-02-24|1309|5696|437|2009|2|2|24|1|2009|437|5696|Tuesday|2009Q1|N|N|N|2454864|2454894|2454521|2454795|N|N|N|N|N| +2454888|AAAAAAAAIGFHFCAA|2009-02-25|1309|5696|437|2009|3|2|25|1|2009|437|5696|Wednesday|2009Q1|N|N|N|2454864|2454894|2454522|2454796|N|N|N|N|N| +2454889|AAAAAAAAJGFHFCAA|2009-02-26|1309|5696|437|2009|4|2|26|1|2009|437|5696|Thursday|2009Q1|N|N|N|2454864|2454894|2454523|2454797|N|N|N|N|N| +2454890|AAAAAAAAKGFHFCAA|2009-02-27|1309|5696|437|2009|5|2|27|1|2009|437|5696|Friday|2009Q1|N|Y|N|2454864|2454894|2454524|2454798|N|N|N|N|N| +2454891|AAAAAAAALGFHFCAA|2009-02-28|1309|5696|437|2009|6|2|28|1|2009|437|5696|Saturday|2009Q1|N|Y|N|2454864|2454894|2454525|2454799|N|N|N|N|N| +2454892|AAAAAAAAMGFHFCAA|2009-03-01|1310|5696|438|2009|0|3|1|1|2009|438|5696|Sunday|2009Q1|N|N|N|2454892|2454950|2454527|2454800|N|N|N|N|N| +2454893|AAAAAAAANGFHFCAA|2009-03-02|1310|5696|438|2009|1|3|2|1|2009|438|5696|Monday|2009Q1|N|N|N|2454892|2454950|2454528|2454801|N|N|N|N|N| +2454894|AAAAAAAAOGFHFCAA|2009-03-03|1310|5697|438|2009|2|3|3|1|2009|438|5697|Tuesday|2009Q1|N|N|N|2454892|2454950|2454529|2454802|N|N|N|N|N| +2454895|AAAAAAAAPGFHFCAA|2009-03-04|1310|5697|438|2009|3|3|4|1|2009|438|5697|Wednesday|2009Q1|N|N|N|2454892|2454950|2454530|2454803|N|N|N|N|N| +2454896|AAAAAAAAAHFHFCAA|2009-03-05|1310|5697|438|2009|4|3|5|1|2009|438|5697|Thursday|2009Q1|N|N|N|2454892|2454950|2454531|2454804|N|N|N|N|N| +2454897|AAAAAAAABHFHFCAA|2009-03-06|1310|5697|438|2009|5|3|6|1|2009|438|5697|Friday|2009Q1|N|Y|N|2454892|2454950|2454532|2454805|N|N|N|N|N| +2454898|AAAAAAAACHFHFCAA|2009-03-07|1310|5697|438|2009|6|3|7|1|2009|438|5697|Saturday|2009Q1|N|Y|N|2454892|2454950|2454533|2454806|N|N|N|N|N| +2454899|AAAAAAAADHFHFCAA|2009-03-08|1310|5697|438|2009|0|3|8|1|2009|438|5697|Sunday|2009Q1|N|N|N|2454892|2454950|2454534|2454807|N|N|N|N|N| +2454900|AAAAAAAAEHFHFCAA|2009-03-09|1310|5697|438|2009|1|3|9|1|2009|438|5697|Monday|2009Q1|N|N|N|2454892|2454950|2454535|2454808|N|N|N|N|N| +2454901|AAAAAAAAFHFHFCAA|2009-03-10|1310|5698|438|2009|2|3|10|1|2009|438|5698|Tuesday|2009Q1|N|N|N|2454892|2454950|2454536|2454809|N|N|N|N|N| +2454902|AAAAAAAAGHFHFCAA|2009-03-11|1310|5698|438|2009|3|3|11|1|2009|438|5698|Wednesday|2009Q1|N|N|N|2454892|2454950|2454537|2454810|N|N|N|N|N| +2454903|AAAAAAAAHHFHFCAA|2009-03-12|1310|5698|438|2009|4|3|12|1|2009|438|5698|Thursday|2009Q1|N|N|N|2454892|2454950|2454538|2454811|N|N|N|N|N| +2454904|AAAAAAAAIHFHFCAA|2009-03-13|1310|5698|438|2009|5|3|13|1|2009|438|5698|Friday|2009Q1|N|Y|N|2454892|2454950|2454539|2454812|N|N|N|N|N| +2454905|AAAAAAAAJHFHFCAA|2009-03-14|1310|5698|438|2009|6|3|14|1|2009|438|5698|Saturday|2009Q1|N|Y|N|2454892|2454950|2454540|2454813|N|N|N|N|N| +2454906|AAAAAAAAKHFHFCAA|2009-03-15|1310|5698|438|2009|0|3|15|1|2009|438|5698|Sunday|2009Q1|N|N|N|2454892|2454950|2454541|2454814|N|N|N|N|N| +2454907|AAAAAAAALHFHFCAA|2009-03-16|1310|5698|438|2009|1|3|16|1|2009|438|5698|Monday|2009Q1|N|N|N|2454892|2454950|2454542|2454815|N|N|N|N|N| +2454908|AAAAAAAAMHFHFCAA|2009-03-17|1310|5699|438|2009|2|3|17|1|2009|438|5699|Tuesday|2009Q1|N|N|N|2454892|2454950|2454543|2454816|N|N|N|N|N| +2454909|AAAAAAAANHFHFCAA|2009-03-18|1310|5699|438|2009|3|3|18|1|2009|438|5699|Wednesday|2009Q1|N|N|N|2454892|2454950|2454544|2454817|N|N|N|N|N| +2454910|AAAAAAAAOHFHFCAA|2009-03-19|1310|5699|438|2009|4|3|19|1|2009|438|5699|Thursday|2009Q1|N|N|N|2454892|2454950|2454545|2454818|N|N|N|N|N| +2454911|AAAAAAAAPHFHFCAA|2009-03-20|1310|5699|438|2009|5|3|20|1|2009|438|5699|Friday|2009Q1|N|Y|N|2454892|2454950|2454546|2454819|N|N|N|N|N| +2454912|AAAAAAAAAIFHFCAA|2009-03-21|1310|5699|438|2009|6|3|21|1|2009|438|5699|Saturday|2009Q1|N|Y|N|2454892|2454950|2454547|2454820|N|N|N|N|N| +2454913|AAAAAAAABIFHFCAA|2009-03-22|1310|5699|438|2009|0|3|22|1|2009|438|5699|Sunday|2009Q1|N|N|N|2454892|2454950|2454548|2454821|N|N|N|N|N| +2454914|AAAAAAAACIFHFCAA|2009-03-23|1310|5699|438|2009|1|3|23|1|2009|438|5699|Monday|2009Q1|N|N|N|2454892|2454950|2454549|2454822|N|N|N|N|N| +2454915|AAAAAAAADIFHFCAA|2009-03-24|1310|5700|438|2009|2|3|24|1|2009|438|5700|Tuesday|2009Q1|N|N|N|2454892|2454950|2454550|2454823|N|N|N|N|N| +2454916|AAAAAAAAEIFHFCAA|2009-03-25|1310|5700|438|2009|3|3|25|1|2009|438|5700|Wednesday|2009Q1|N|N|N|2454892|2454950|2454551|2454824|N|N|N|N|N| +2454917|AAAAAAAAFIFHFCAA|2009-03-26|1310|5700|438|2009|4|3|26|1|2009|438|5700|Thursday|2009Q1|N|N|N|2454892|2454950|2454552|2454825|N|N|N|N|N| +2454918|AAAAAAAAGIFHFCAA|2009-03-27|1310|5700|438|2009|5|3|27|1|2009|438|5700|Friday|2009Q1|N|Y|N|2454892|2454950|2454553|2454826|N|N|N|N|N| +2454919|AAAAAAAAHIFHFCAA|2009-03-28|1310|5700|438|2009|6|3|28|1|2009|438|5700|Saturday|2009Q1|N|Y|N|2454892|2454950|2454554|2454827|N|N|N|N|N| +2454920|AAAAAAAAIIFHFCAA|2009-03-29|1310|5700|438|2009|0|3|29|1|2009|438|5700|Sunday|2009Q1|N|N|N|2454892|2454950|2454555|2454828|N|N|N|N|N| +2454921|AAAAAAAAJIFHFCAA|2009-03-30|1310|5700|438|2009|1|3|30|1|2009|438|5700|Monday|2009Q1|N|N|N|2454892|2454950|2454556|2454829|N|N|N|N|N| +2454922|AAAAAAAAKIFHFCAA|2009-03-31|1310|5701|438|2009|2|3|31|1|2009|438|5701|Tuesday|2009Q1|N|N|N|2454892|2454950|2454557|2454830|N|N|N|N|N| +2454923|AAAAAAAALIFHFCAA|2009-04-01|1311|5701|438|2009|3|4|1|1|2009|438|5701|Wednesday|2009Q1|N|N|N|2454923|2455012|2454558|2454833|N|N|N|N|N| +2454924|AAAAAAAAMIFHFCAA|2009-04-02|1311|5701|438|2009|4|4|2|2|2009|438|5701|Thursday|2009Q2|N|N|N|2454923|2455012|2454559|2454834|N|N|N|N|N| +2454925|AAAAAAAANIFHFCAA|2009-04-03|1311|5701|438|2009|5|4|3|2|2009|438|5701|Friday|2009Q2|N|Y|N|2454923|2455012|2454560|2454835|N|N|N|N|N| +2454926|AAAAAAAAOIFHFCAA|2009-04-04|1311|5701|438|2009|6|4|4|2|2009|438|5701|Saturday|2009Q2|N|Y|N|2454923|2455012|2454561|2454836|N|N|N|N|N| +2454927|AAAAAAAAPIFHFCAA|2009-04-05|1311|5701|438|2009|0|4|5|2|2009|438|5701|Sunday|2009Q2|N|N|N|2454923|2455012|2454562|2454837|N|N|N|N|N| +2454928|AAAAAAAAAJFHFCAA|2009-04-06|1311|5701|438|2009|1|4|6|2|2009|438|5701|Monday|2009Q2|N|N|N|2454923|2455012|2454563|2454838|N|N|N|N|N| +2454929|AAAAAAAABJFHFCAA|2009-04-07|1311|5702|438|2009|2|4|7|2|2009|438|5702|Tuesday|2009Q2|N|N|N|2454923|2455012|2454564|2454839|N|N|N|N|N| +2454930|AAAAAAAACJFHFCAA|2009-04-08|1311|5702|438|2009|3|4|8|2|2009|438|5702|Wednesday|2009Q2|N|N|N|2454923|2455012|2454565|2454840|N|N|N|N|N| +2454931|AAAAAAAADJFHFCAA|2009-04-09|1311|5702|438|2009|4|4|9|2|2009|438|5702|Thursday|2009Q2|N|N|N|2454923|2455012|2454566|2454841|N|N|N|N|N| +2454932|AAAAAAAAEJFHFCAA|2009-04-10|1311|5702|438|2009|5|4|10|2|2009|438|5702|Friday|2009Q2|N|Y|N|2454923|2455012|2454567|2454842|N|N|N|N|N| +2454933|AAAAAAAAFJFHFCAA|2009-04-11|1311|5702|438|2009|6|4|11|2|2009|438|5702|Saturday|2009Q2|N|Y|N|2454923|2455012|2454568|2454843|N|N|N|N|N| +2454934|AAAAAAAAGJFHFCAA|2009-04-12|1311|5702|438|2009|0|4|12|2|2009|438|5702|Sunday|2009Q2|N|N|N|2454923|2455012|2454569|2454844|N|N|N|N|N| +2454935|AAAAAAAAHJFHFCAA|2009-04-13|1311|5702|438|2009|1|4|13|2|2009|438|5702|Monday|2009Q2|N|N|N|2454923|2455012|2454570|2454845|N|N|N|N|N| +2454936|AAAAAAAAIJFHFCAA|2009-04-14|1311|5703|438|2009|2|4|14|2|2009|438|5703|Tuesday|2009Q2|N|N|N|2454923|2455012|2454571|2454846|N|N|N|N|N| +2454937|AAAAAAAAJJFHFCAA|2009-04-15|1311|5703|438|2009|3|4|15|2|2009|438|5703|Wednesday|2009Q2|N|N|N|2454923|2455012|2454572|2454847|N|N|N|N|N| +2454938|AAAAAAAAKJFHFCAA|2009-04-16|1311|5703|438|2009|4|4|16|2|2009|438|5703|Thursday|2009Q2|N|N|N|2454923|2455012|2454573|2454848|N|N|N|N|N| +2454939|AAAAAAAALJFHFCAA|2009-04-17|1311|5703|438|2009|5|4|17|2|2009|438|5703|Friday|2009Q2|N|Y|N|2454923|2455012|2454574|2454849|N|N|N|N|N| +2454940|AAAAAAAAMJFHFCAA|2009-04-18|1311|5703|438|2009|6|4|18|2|2009|438|5703|Saturday|2009Q2|N|Y|N|2454923|2455012|2454575|2454850|N|N|N|N|N| +2454941|AAAAAAAANJFHFCAA|2009-04-19|1311|5703|438|2009|0|4|19|2|2009|438|5703|Sunday|2009Q2|N|N|N|2454923|2455012|2454576|2454851|N|N|N|N|N| +2454942|AAAAAAAAOJFHFCAA|2009-04-20|1311|5703|438|2009|1|4|20|2|2009|438|5703|Monday|2009Q2|N|N|N|2454923|2455012|2454577|2454852|N|N|N|N|N| +2454943|AAAAAAAAPJFHFCAA|2009-04-21|1311|5704|438|2009|2|4|21|2|2009|438|5704|Tuesday|2009Q2|N|N|N|2454923|2455012|2454578|2454853|N|N|N|N|N| +2454944|AAAAAAAAAKFHFCAA|2009-04-22|1311|5704|438|2009|3|4|22|2|2009|438|5704|Wednesday|2009Q2|N|N|N|2454923|2455012|2454579|2454854|N|N|N|N|N| +2454945|AAAAAAAABKFHFCAA|2009-04-23|1311|5704|438|2009|4|4|23|2|2009|438|5704|Thursday|2009Q2|N|N|N|2454923|2455012|2454580|2454855|N|N|N|N|N| +2454946|AAAAAAAACKFHFCAA|2009-04-24|1311|5704|438|2009|5|4|24|2|2009|438|5704|Friday|2009Q2|N|Y|N|2454923|2455012|2454581|2454856|N|N|N|N|N| +2454947|AAAAAAAADKFHFCAA|2009-04-25|1311|5704|438|2009|6|4|25|2|2009|438|5704|Saturday|2009Q2|N|Y|N|2454923|2455012|2454582|2454857|N|N|N|N|N| +2454948|AAAAAAAAEKFHFCAA|2009-04-26|1311|5704|438|2009|0|4|26|2|2009|438|5704|Sunday|2009Q2|N|N|N|2454923|2455012|2454583|2454858|N|N|N|N|N| +2454949|AAAAAAAAFKFHFCAA|2009-04-27|1311|5704|438|2009|1|4|27|2|2009|438|5704|Monday|2009Q2|N|N|N|2454923|2455012|2454584|2454859|N|N|N|N|N| +2454950|AAAAAAAAGKFHFCAA|2009-04-28|1311|5705|438|2009|2|4|28|2|2009|438|5705|Tuesday|2009Q2|N|N|N|2454923|2455012|2454585|2454860|N|N|N|N|N| +2454951|AAAAAAAAHKFHFCAA|2009-04-29|1311|5705|438|2009|3|4|29|2|2009|438|5705|Wednesday|2009Q2|N|N|N|2454923|2455012|2454586|2454861|N|N|N|N|N| +2454952|AAAAAAAAIKFHFCAA|2009-04-30|1311|5705|438|2009|4|4|30|2|2009|438|5705|Thursday|2009Q2|N|N|N|2454923|2455012|2454587|2454862|N|N|N|N|N| +2454953|AAAAAAAAJKFHFCAA|2009-05-01|1312|5705|438|2009|5|5|1|2|2009|438|5705|Friday|2009Q2|N|Y|N|2454953|2455072|2454588|2454863|N|N|N|N|N| +2454954|AAAAAAAAKKFHFCAA|2009-05-02|1312|5705|438|2009|6|5|2|2|2009|438|5705|Saturday|2009Q2|N|Y|N|2454953|2455072|2454589|2454864|N|N|N|N|N| +2454955|AAAAAAAALKFHFCAA|2009-05-03|1312|5705|438|2009|0|5|3|2|2009|438|5705|Sunday|2009Q2|N|N|N|2454953|2455072|2454590|2454865|N|N|N|N|N| +2454956|AAAAAAAAMKFHFCAA|2009-05-04|1312|5705|438|2009|1|5|4|2|2009|438|5705|Monday|2009Q2|N|N|N|2454953|2455072|2454591|2454866|N|N|N|N|N| +2454957|AAAAAAAANKFHFCAA|2009-05-05|1312|5706|438|2009|2|5|5|2|2009|438|5706|Tuesday|2009Q2|N|N|N|2454953|2455072|2454592|2454867|N|N|N|N|N| +2454958|AAAAAAAAOKFHFCAA|2009-05-06|1312|5706|438|2009|3|5|6|2|2009|438|5706|Wednesday|2009Q2|N|N|N|2454953|2455072|2454593|2454868|N|N|N|N|N| +2454959|AAAAAAAAPKFHFCAA|2009-05-07|1312|5706|438|2009|4|5|7|2|2009|438|5706|Thursday|2009Q2|N|N|N|2454953|2455072|2454594|2454869|N|N|N|N|N| +2454960|AAAAAAAAALFHFCAA|2009-05-08|1312|5706|438|2009|5|5|8|2|2009|438|5706|Friday|2009Q2|N|Y|N|2454953|2455072|2454595|2454870|N|N|N|N|N| +2454961|AAAAAAAABLFHFCAA|2009-05-09|1312|5706|438|2009|6|5|9|2|2009|438|5706|Saturday|2009Q2|N|Y|N|2454953|2455072|2454596|2454871|N|N|N|N|N| +2454962|AAAAAAAACLFHFCAA|2009-05-10|1312|5706|438|2009|0|5|10|2|2009|438|5706|Sunday|2009Q2|N|N|N|2454953|2455072|2454597|2454872|N|N|N|N|N| +2454963|AAAAAAAADLFHFCAA|2009-05-11|1312|5706|438|2009|1|5|11|2|2009|438|5706|Monday|2009Q2|N|N|N|2454953|2455072|2454598|2454873|N|N|N|N|N| +2454964|AAAAAAAAELFHFCAA|2009-05-12|1312|5707|438|2009|2|5|12|2|2009|438|5707|Tuesday|2009Q2|N|N|N|2454953|2455072|2454599|2454874|N|N|N|N|N| +2454965|AAAAAAAAFLFHFCAA|2009-05-13|1312|5707|438|2009|3|5|13|2|2009|438|5707|Wednesday|2009Q2|N|N|N|2454953|2455072|2454600|2454875|N|N|N|N|N| +2454966|AAAAAAAAGLFHFCAA|2009-05-14|1312|5707|438|2009|4|5|14|2|2009|438|5707|Thursday|2009Q2|N|N|N|2454953|2455072|2454601|2454876|N|N|N|N|N| +2454967|AAAAAAAAHLFHFCAA|2009-05-15|1312|5707|438|2009|5|5|15|2|2009|438|5707|Friday|2009Q2|N|Y|N|2454953|2455072|2454602|2454877|N|N|N|N|N| +2454968|AAAAAAAAILFHFCAA|2009-05-16|1312|5707|438|2009|6|5|16|2|2009|438|5707|Saturday|2009Q2|N|Y|N|2454953|2455072|2454603|2454878|N|N|N|N|N| +2454969|AAAAAAAAJLFHFCAA|2009-05-17|1312|5707|438|2009|0|5|17|2|2009|438|5707|Sunday|2009Q2|N|N|N|2454953|2455072|2454604|2454879|N|N|N|N|N| +2454970|AAAAAAAAKLFHFCAA|2009-05-18|1312|5707|438|2009|1|5|18|2|2009|438|5707|Monday|2009Q2|N|N|N|2454953|2455072|2454605|2454880|N|N|N|N|N| +2454971|AAAAAAAALLFHFCAA|2009-05-19|1312|5708|438|2009|2|5|19|2|2009|438|5708|Tuesday|2009Q2|N|N|N|2454953|2455072|2454606|2454881|N|N|N|N|N| +2454972|AAAAAAAAMLFHFCAA|2009-05-20|1312|5708|438|2009|3|5|20|2|2009|438|5708|Wednesday|2009Q2|N|N|N|2454953|2455072|2454607|2454882|N|N|N|N|N| +2454973|AAAAAAAANLFHFCAA|2009-05-21|1312|5708|438|2009|4|5|21|2|2009|438|5708|Thursday|2009Q2|N|N|N|2454953|2455072|2454608|2454883|N|N|N|N|N| +2454974|AAAAAAAAOLFHFCAA|2009-05-22|1312|5708|438|2009|5|5|22|2|2009|438|5708|Friday|2009Q2|N|Y|N|2454953|2455072|2454609|2454884|N|N|N|N|N| +2454975|AAAAAAAAPLFHFCAA|2009-05-23|1312|5708|438|2009|6|5|23|2|2009|438|5708|Saturday|2009Q2|N|Y|N|2454953|2455072|2454610|2454885|N|N|N|N|N| +2454976|AAAAAAAAAMFHFCAA|2009-05-24|1312|5708|438|2009|0|5|24|2|2009|438|5708|Sunday|2009Q2|N|N|N|2454953|2455072|2454611|2454886|N|N|N|N|N| +2454977|AAAAAAAABMFHFCAA|2009-05-25|1312|5708|438|2009|1|5|25|2|2009|438|5708|Monday|2009Q2|N|N|N|2454953|2455072|2454612|2454887|N|N|N|N|N| +2454978|AAAAAAAACMFHFCAA|2009-05-26|1312|5709|438|2009|2|5|26|2|2009|438|5709|Tuesday|2009Q2|N|N|N|2454953|2455072|2454613|2454888|N|N|N|N|N| +2454979|AAAAAAAADMFHFCAA|2009-05-27|1312|5709|438|2009|3|5|27|2|2009|438|5709|Wednesday|2009Q2|N|N|N|2454953|2455072|2454614|2454889|N|N|N|N|N| +2454980|AAAAAAAAEMFHFCAA|2009-05-28|1312|5709|438|2009|4|5|28|2|2009|438|5709|Thursday|2009Q2|N|N|N|2454953|2455072|2454615|2454890|N|N|N|N|N| +2454981|AAAAAAAAFMFHFCAA|2009-05-29|1312|5709|438|2009|5|5|29|2|2009|438|5709|Friday|2009Q2|N|Y|N|2454953|2455072|2454616|2454891|N|N|N|N|N| +2454982|AAAAAAAAGMFHFCAA|2009-05-30|1312|5709|438|2009|6|5|30|2|2009|438|5709|Saturday|2009Q2|N|Y|N|2454953|2455072|2454617|2454892|N|N|N|N|N| +2454983|AAAAAAAAHMFHFCAA|2009-05-31|1312|5709|438|2009|0|5|31|2|2009|438|5709|Sunday|2009Q2|N|N|N|2454953|2455072|2454618|2454893|N|N|N|N|N| +2454984|AAAAAAAAIMFHFCAA|2009-06-01|1313|5709|439|2009|1|6|1|2|2009|439|5709|Monday|2009Q2|N|N|N|2454984|2455134|2454619|2454894|N|N|N|N|N| +2454985|AAAAAAAAJMFHFCAA|2009-06-02|1313|5710|439|2009|2|6|2|2|2009|439|5710|Tuesday|2009Q2|N|N|N|2454984|2455134|2454620|2454895|N|N|N|N|N| +2454986|AAAAAAAAKMFHFCAA|2009-06-03|1313|5710|439|2009|3|6|3|2|2009|439|5710|Wednesday|2009Q2|N|N|N|2454984|2455134|2454621|2454896|N|N|N|N|N| +2454987|AAAAAAAALMFHFCAA|2009-06-04|1313|5710|439|2009|4|6|4|2|2009|439|5710|Thursday|2009Q2|N|N|N|2454984|2455134|2454622|2454897|N|N|N|N|N| +2454988|AAAAAAAAMMFHFCAA|2009-06-05|1313|5710|439|2009|5|6|5|2|2009|439|5710|Friday|2009Q2|N|Y|N|2454984|2455134|2454623|2454898|N|N|N|N|N| +2454989|AAAAAAAANMFHFCAA|2009-06-06|1313|5710|439|2009|6|6|6|2|2009|439|5710|Saturday|2009Q2|N|Y|N|2454984|2455134|2454624|2454899|N|N|N|N|N| +2454990|AAAAAAAAOMFHFCAA|2009-06-07|1313|5710|439|2009|0|6|7|2|2009|439|5710|Sunday|2009Q2|N|N|N|2454984|2455134|2454625|2454900|N|N|N|N|N| +2454991|AAAAAAAAPMFHFCAA|2009-06-08|1313|5710|439|2009|1|6|8|2|2009|439|5710|Monday|2009Q2|N|N|N|2454984|2455134|2454626|2454901|N|N|N|N|N| +2454992|AAAAAAAAANFHFCAA|2009-06-09|1313|5711|439|2009|2|6|9|2|2009|439|5711|Tuesday|2009Q2|N|N|N|2454984|2455134|2454627|2454902|N|N|N|N|N| +2454993|AAAAAAAABNFHFCAA|2009-06-10|1313|5711|439|2009|3|6|10|2|2009|439|5711|Wednesday|2009Q2|N|N|N|2454984|2455134|2454628|2454903|N|N|N|N|N| +2454994|AAAAAAAACNFHFCAA|2009-06-11|1313|5711|439|2009|4|6|11|2|2009|439|5711|Thursday|2009Q2|N|N|N|2454984|2455134|2454629|2454904|N|N|N|N|N| +2454995|AAAAAAAADNFHFCAA|2009-06-12|1313|5711|439|2009|5|6|12|2|2009|439|5711|Friday|2009Q2|N|Y|N|2454984|2455134|2454630|2454905|N|N|N|N|N| +2454996|AAAAAAAAENFHFCAA|2009-06-13|1313|5711|439|2009|6|6|13|2|2009|439|5711|Saturday|2009Q2|N|Y|N|2454984|2455134|2454631|2454906|N|N|N|N|N| +2454997|AAAAAAAAFNFHFCAA|2009-06-14|1313|5711|439|2009|0|6|14|2|2009|439|5711|Sunday|2009Q2|N|N|N|2454984|2455134|2454632|2454907|N|N|N|N|N| +2454998|AAAAAAAAGNFHFCAA|2009-06-15|1313|5711|439|2009|1|6|15|2|2009|439|5711|Monday|2009Q2|N|N|N|2454984|2455134|2454633|2454908|N|N|N|N|N| +2454999|AAAAAAAAHNFHFCAA|2009-06-16|1313|5712|439|2009|2|6|16|2|2009|439|5712|Tuesday|2009Q2|N|N|N|2454984|2455134|2454634|2454909|N|N|N|N|N| +2455000|AAAAAAAAINFHFCAA|2009-06-17|1313|5712|439|2009|3|6|17|2|2009|439|5712|Wednesday|2009Q2|N|N|N|2454984|2455134|2454635|2454910|N|N|N|N|N| +2455001|AAAAAAAAJNFHFCAA|2009-06-18|1313|5712|439|2009|4|6|18|2|2009|439|5712|Thursday|2009Q2|N|N|N|2454984|2455134|2454636|2454911|N|N|N|N|N| +2455002|AAAAAAAAKNFHFCAA|2009-06-19|1313|5712|439|2009|5|6|19|2|2009|439|5712|Friday|2009Q2|N|Y|N|2454984|2455134|2454637|2454912|N|N|N|N|N| +2455003|AAAAAAAALNFHFCAA|2009-06-20|1313|5712|439|2009|6|6|20|2|2009|439|5712|Saturday|2009Q2|N|Y|N|2454984|2455134|2454638|2454913|N|N|N|N|N| +2455004|AAAAAAAAMNFHFCAA|2009-06-21|1313|5712|439|2009|0|6|21|2|2009|439|5712|Sunday|2009Q2|N|N|N|2454984|2455134|2454639|2454914|N|N|N|N|N| +2455005|AAAAAAAANNFHFCAA|2009-06-22|1313|5712|439|2009|1|6|22|2|2009|439|5712|Monday|2009Q2|N|N|N|2454984|2455134|2454640|2454915|N|N|N|N|N| +2455006|AAAAAAAAONFHFCAA|2009-06-23|1313|5713|439|2009|2|6|23|2|2009|439|5713|Tuesday|2009Q2|N|N|N|2454984|2455134|2454641|2454916|N|N|N|N|N| +2455007|AAAAAAAAPNFHFCAA|2009-06-24|1313|5713|439|2009|3|6|24|2|2009|439|5713|Wednesday|2009Q2|N|N|N|2454984|2455134|2454642|2454917|N|N|N|N|N| +2455008|AAAAAAAAAOFHFCAA|2009-06-25|1313|5713|439|2009|4|6|25|2|2009|439|5713|Thursday|2009Q2|N|N|N|2454984|2455134|2454643|2454918|N|N|N|N|N| +2455009|AAAAAAAABOFHFCAA|2009-06-26|1313|5713|439|2009|5|6|26|2|2009|439|5713|Friday|2009Q2|N|Y|N|2454984|2455134|2454644|2454919|N|N|N|N|N| +2455010|AAAAAAAACOFHFCAA|2009-06-27|1313|5713|439|2009|6|6|27|2|2009|439|5713|Saturday|2009Q2|N|Y|N|2454984|2455134|2454645|2454920|N|N|N|N|N| +2455011|AAAAAAAADOFHFCAA|2009-06-28|1313|5713|439|2009|0|6|28|2|2009|439|5713|Sunday|2009Q2|N|N|N|2454984|2455134|2454646|2454921|N|N|N|N|N| +2455012|AAAAAAAAEOFHFCAA|2009-06-29|1313|5713|439|2009|1|6|29|2|2009|439|5713|Monday|2009Q2|N|N|N|2454984|2455134|2454647|2454922|N|N|N|N|N| +2455013|AAAAAAAAFOFHFCAA|2009-06-30|1313|5714|439|2009|2|6|30|2|2009|439|5714|Tuesday|2009Q2|N|N|N|2454984|2455134|2454648|2454923|N|N|N|N|N| +2455014|AAAAAAAAGOFHFCAA|2009-07-01|1314|5714|439|2009|3|7|1|2|2009|439|5714|Wednesday|2009Q2|N|N|N|2455014|2455194|2454649|2454923|N|N|N|N|N| +2455015|AAAAAAAAHOFHFCAA|2009-07-02|1314|5714|439|2009|4|7|2|3|2009|439|5714|Thursday|2009Q3|N|N|N|2455014|2455194|2454650|2454924|N|N|N|N|N| +2455016|AAAAAAAAIOFHFCAA|2009-07-03|1314|5714|439|2009|5|7|3|3|2009|439|5714|Friday|2009Q3|N|Y|N|2455014|2455194|2454651|2454925|N|N|N|N|N| +2455017|AAAAAAAAJOFHFCAA|2009-07-04|1314|5714|439|2009|6|7|4|3|2009|439|5714|Saturday|2009Q3|N|Y|N|2455014|2455194|2454652|2454926|N|N|N|N|N| +2455018|AAAAAAAAKOFHFCAA|2009-07-05|1314|5714|439|2009|0|7|5|3|2009|439|5714|Sunday|2009Q3|Y|N|N|2455014|2455194|2454653|2454927|N|N|N|N|N| +2455019|AAAAAAAALOFHFCAA|2009-07-06|1314|5714|439|2009|1|7|6|3|2009|439|5714|Monday|2009Q3|N|N|Y|2455014|2455194|2454654|2454928|N|N|N|N|N| +2455020|AAAAAAAAMOFHFCAA|2009-07-07|1314|5715|439|2009|2|7|7|3|2009|439|5715|Tuesday|2009Q3|N|N|N|2455014|2455194|2454655|2454929|N|N|N|N|N| +2455021|AAAAAAAANOFHFCAA|2009-07-08|1314|5715|439|2009|3|7|8|3|2009|439|5715|Wednesday|2009Q3|N|N|N|2455014|2455194|2454656|2454930|N|N|N|N|N| +2455022|AAAAAAAAOOFHFCAA|2009-07-09|1314|5715|439|2009|4|7|9|3|2009|439|5715|Thursday|2009Q3|N|N|N|2455014|2455194|2454657|2454931|N|N|N|N|N| +2455023|AAAAAAAAPOFHFCAA|2009-07-10|1314|5715|439|2009|5|7|10|3|2009|439|5715|Friday|2009Q3|N|Y|N|2455014|2455194|2454658|2454932|N|N|N|N|N| +2455024|AAAAAAAAAPFHFCAA|2009-07-11|1314|5715|439|2009|6|7|11|3|2009|439|5715|Saturday|2009Q3|N|Y|N|2455014|2455194|2454659|2454933|N|N|N|N|N| +2455025|AAAAAAAABPFHFCAA|2009-07-12|1314|5715|439|2009|0|7|12|3|2009|439|5715|Sunday|2009Q3|N|N|N|2455014|2455194|2454660|2454934|N|N|N|N|N| +2455026|AAAAAAAACPFHFCAA|2009-07-13|1314|5715|439|2009|1|7|13|3|2009|439|5715|Monday|2009Q3|N|N|N|2455014|2455194|2454661|2454935|N|N|N|N|N| +2455027|AAAAAAAADPFHFCAA|2009-07-14|1314|5716|439|2009|2|7|14|3|2009|439|5716|Tuesday|2009Q3|N|N|N|2455014|2455194|2454662|2454936|N|N|N|N|N| +2455028|AAAAAAAAEPFHFCAA|2009-07-15|1314|5716|439|2009|3|7|15|3|2009|439|5716|Wednesday|2009Q3|N|N|N|2455014|2455194|2454663|2454937|N|N|N|N|N| +2455029|AAAAAAAAFPFHFCAA|2009-07-16|1314|5716|439|2009|4|7|16|3|2009|439|5716|Thursday|2009Q3|N|N|N|2455014|2455194|2454664|2454938|N|N|N|N|N| +2455030|AAAAAAAAGPFHFCAA|2009-07-17|1314|5716|439|2009|5|7|17|3|2009|439|5716|Friday|2009Q3|N|Y|N|2455014|2455194|2454665|2454939|N|N|N|N|N| +2455031|AAAAAAAAHPFHFCAA|2009-07-18|1314|5716|439|2009|6|7|18|3|2009|439|5716|Saturday|2009Q3|N|Y|N|2455014|2455194|2454666|2454940|N|N|N|N|N| +2455032|AAAAAAAAIPFHFCAA|2009-07-19|1314|5716|439|2009|0|7|19|3|2009|439|5716|Sunday|2009Q3|N|N|N|2455014|2455194|2454667|2454941|N|N|N|N|N| +2455033|AAAAAAAAJPFHFCAA|2009-07-20|1314|5716|439|2009|1|7|20|3|2009|439|5716|Monday|2009Q3|N|N|N|2455014|2455194|2454668|2454942|N|N|N|N|N| +2455034|AAAAAAAAKPFHFCAA|2009-07-21|1314|5717|439|2009|2|7|21|3|2009|439|5717|Tuesday|2009Q3|N|N|N|2455014|2455194|2454669|2454943|N|N|N|N|N| +2455035|AAAAAAAALPFHFCAA|2009-07-22|1314|5717|439|2009|3|7|22|3|2009|439|5717|Wednesday|2009Q3|N|N|N|2455014|2455194|2454670|2454944|N|N|N|N|N| +2455036|AAAAAAAAMPFHFCAA|2009-07-23|1314|5717|439|2009|4|7|23|3|2009|439|5717|Thursday|2009Q3|N|N|N|2455014|2455194|2454671|2454945|N|N|N|N|N| +2455037|AAAAAAAANPFHFCAA|2009-07-24|1314|5717|439|2009|5|7|24|3|2009|439|5717|Friday|2009Q3|N|Y|N|2455014|2455194|2454672|2454946|N|N|N|N|N| +2455038|AAAAAAAAOPFHFCAA|2009-07-25|1314|5717|439|2009|6|7|25|3|2009|439|5717|Saturday|2009Q3|N|Y|N|2455014|2455194|2454673|2454947|N|N|N|N|N| +2455039|AAAAAAAAPPFHFCAA|2009-07-26|1314|5717|439|2009|0|7|26|3|2009|439|5717|Sunday|2009Q3|N|N|N|2455014|2455194|2454674|2454948|N|N|N|N|N| +2455040|AAAAAAAAAAGHFCAA|2009-07-27|1314|5717|439|2009|1|7|27|3|2009|439|5717|Monday|2009Q3|N|N|N|2455014|2455194|2454675|2454949|N|N|N|N|N| +2455041|AAAAAAAABAGHFCAA|2009-07-28|1314|5718|439|2009|2|7|28|3|2009|439|5718|Tuesday|2009Q3|N|N|N|2455014|2455194|2454676|2454950|N|N|N|N|N| +2455042|AAAAAAAACAGHFCAA|2009-07-29|1314|5718|439|2009|3|7|29|3|2009|439|5718|Wednesday|2009Q3|N|N|N|2455014|2455194|2454677|2454951|N|N|N|N|N| +2455043|AAAAAAAADAGHFCAA|2009-07-30|1314|5718|439|2009|4|7|30|3|2009|439|5718|Thursday|2009Q3|N|N|N|2455014|2455194|2454678|2454952|N|N|N|N|N| +2455044|AAAAAAAAEAGHFCAA|2009-07-31|1314|5718|439|2009|5|7|31|3|2009|439|5718|Friday|2009Q3|N|Y|N|2455014|2455194|2454679|2454953|N|N|N|N|N| +2455045|AAAAAAAAFAGHFCAA|2009-08-01|1315|5718|439|2009|6|8|1|3|2009|439|5718|Saturday|2009Q3|N|Y|N|2455045|2455256|2454680|2454954|N|N|N|N|N| +2455046|AAAAAAAAGAGHFCAA|2009-08-02|1315|5718|439|2009|0|8|2|3|2009|439|5718|Sunday|2009Q3|N|N|N|2455045|2455256|2454681|2454955|N|N|N|N|N| +2455047|AAAAAAAAHAGHFCAA|2009-08-03|1315|5718|439|2009|1|8|3|3|2009|439|5718|Monday|2009Q3|N|N|N|2455045|2455256|2454682|2454956|N|N|N|N|N| +2455048|AAAAAAAAIAGHFCAA|2009-08-04|1315|5719|439|2009|2|8|4|3|2009|439|5719|Tuesday|2009Q3|N|N|N|2455045|2455256|2454683|2454957|N|N|N|N|N| +2455049|AAAAAAAAJAGHFCAA|2009-08-05|1315|5719|439|2009|3|8|5|3|2009|439|5719|Wednesday|2009Q3|N|N|N|2455045|2455256|2454684|2454958|N|N|N|N|N| +2455050|AAAAAAAAKAGHFCAA|2009-08-06|1315|5719|439|2009|4|8|6|3|2009|439|5719|Thursday|2009Q3|N|N|N|2455045|2455256|2454685|2454959|N|N|N|N|N| +2455051|AAAAAAAALAGHFCAA|2009-08-07|1315|5719|439|2009|5|8|7|3|2009|439|5719|Friday|2009Q3|N|Y|N|2455045|2455256|2454686|2454960|N|N|N|N|N| +2455052|AAAAAAAAMAGHFCAA|2009-08-08|1315|5719|439|2009|6|8|8|3|2009|439|5719|Saturday|2009Q3|N|Y|N|2455045|2455256|2454687|2454961|N|N|N|N|N| +2455053|AAAAAAAANAGHFCAA|2009-08-09|1315|5719|439|2009|0|8|9|3|2009|439|5719|Sunday|2009Q3|N|N|N|2455045|2455256|2454688|2454962|N|N|N|N|N| +2455054|AAAAAAAAOAGHFCAA|2009-08-10|1315|5719|439|2009|1|8|10|3|2009|439|5719|Monday|2009Q3|N|N|N|2455045|2455256|2454689|2454963|N|N|N|N|N| +2455055|AAAAAAAAPAGHFCAA|2009-08-11|1315|5720|439|2009|2|8|11|3|2009|439|5720|Tuesday|2009Q3|N|N|N|2455045|2455256|2454690|2454964|N|N|N|N|N| +2455056|AAAAAAAAABGHFCAA|2009-08-12|1315|5720|439|2009|3|8|12|3|2009|439|5720|Wednesday|2009Q3|N|N|N|2455045|2455256|2454691|2454965|N|N|N|N|N| +2455057|AAAAAAAABBGHFCAA|2009-08-13|1315|5720|439|2009|4|8|13|3|2009|439|5720|Thursday|2009Q3|N|N|N|2455045|2455256|2454692|2454966|N|N|N|N|N| +2455058|AAAAAAAACBGHFCAA|2009-08-14|1315|5720|439|2009|5|8|14|3|2009|439|5720|Friday|2009Q3|N|Y|N|2455045|2455256|2454693|2454967|N|N|N|N|N| +2455059|AAAAAAAADBGHFCAA|2009-08-15|1315|5720|439|2009|6|8|15|3|2009|439|5720|Saturday|2009Q3|N|Y|N|2455045|2455256|2454694|2454968|N|N|N|N|N| +2455060|AAAAAAAAEBGHFCAA|2009-08-16|1315|5720|439|2009|0|8|16|3|2009|439|5720|Sunday|2009Q3|N|N|N|2455045|2455256|2454695|2454969|N|N|N|N|N| +2455061|AAAAAAAAFBGHFCAA|2009-08-17|1315|5720|439|2009|1|8|17|3|2009|439|5720|Monday|2009Q3|N|N|N|2455045|2455256|2454696|2454970|N|N|N|N|N| +2455062|AAAAAAAAGBGHFCAA|2009-08-18|1315|5721|439|2009|2|8|18|3|2009|439|5721|Tuesday|2009Q3|N|N|N|2455045|2455256|2454697|2454971|N|N|N|N|N| +2455063|AAAAAAAAHBGHFCAA|2009-08-19|1315|5721|439|2009|3|8|19|3|2009|439|5721|Wednesday|2009Q3|N|N|N|2455045|2455256|2454698|2454972|N|N|N|N|N| +2455064|AAAAAAAAIBGHFCAA|2009-08-20|1315|5721|439|2009|4|8|20|3|2009|439|5721|Thursday|2009Q3|N|N|N|2455045|2455256|2454699|2454973|N|N|N|N|N| +2455065|AAAAAAAAJBGHFCAA|2009-08-21|1315|5721|439|2009|5|8|21|3|2009|439|5721|Friday|2009Q3|N|Y|N|2455045|2455256|2454700|2454974|N|N|N|N|N| +2455066|AAAAAAAAKBGHFCAA|2009-08-22|1315|5721|439|2009|6|8|22|3|2009|439|5721|Saturday|2009Q3|N|Y|N|2455045|2455256|2454701|2454975|N|N|N|N|N| +2455067|AAAAAAAALBGHFCAA|2009-08-23|1315|5721|439|2009|0|8|23|3|2009|439|5721|Sunday|2009Q3|N|N|N|2455045|2455256|2454702|2454976|N|N|N|N|N| +2455068|AAAAAAAAMBGHFCAA|2009-08-24|1315|5721|439|2009|1|8|24|3|2009|439|5721|Monday|2009Q3|N|N|N|2455045|2455256|2454703|2454977|N|N|N|N|N| +2455069|AAAAAAAANBGHFCAA|2009-08-25|1315|5722|439|2009|2|8|25|3|2009|439|5722|Tuesday|2009Q3|N|N|N|2455045|2455256|2454704|2454978|N|N|N|N|N| +2455070|AAAAAAAAOBGHFCAA|2009-08-26|1315|5722|439|2009|3|8|26|3|2009|439|5722|Wednesday|2009Q3|N|N|N|2455045|2455256|2454705|2454979|N|N|N|N|N| +2455071|AAAAAAAAPBGHFCAA|2009-08-27|1315|5722|439|2009|4|8|27|3|2009|439|5722|Thursday|2009Q3|N|N|N|2455045|2455256|2454706|2454980|N|N|N|N|N| +2455072|AAAAAAAAACGHFCAA|2009-08-28|1315|5722|439|2009|5|8|28|3|2009|439|5722|Friday|2009Q3|N|Y|N|2455045|2455256|2454707|2454981|N|N|N|N|N| +2455073|AAAAAAAABCGHFCAA|2009-08-29|1315|5722|439|2009|6|8|29|3|2009|439|5722|Saturday|2009Q3|N|Y|N|2455045|2455256|2454708|2454982|N|N|N|N|N| +2455074|AAAAAAAACCGHFCAA|2009-08-30|1315|5722|439|2009|0|8|30|3|2009|439|5722|Sunday|2009Q3|N|N|N|2455045|2455256|2454709|2454983|N|N|N|N|N| +2455075|AAAAAAAADCGHFCAA|2009-08-31|1315|5722|439|2009|1|8|31|3|2009|439|5722|Monday|2009Q3|N|N|N|2455045|2455256|2454710|2454984|N|N|N|N|N| +2455076|AAAAAAAAECGHFCAA|2009-09-01|1316|5723|440|2009|2|9|1|3|2009|440|5723|Tuesday|2009Q3|N|N|N|2455076|2455318|2454711|2454985|N|N|N|N|N| +2455077|AAAAAAAAFCGHFCAA|2009-09-02|1316|5723|440|2009|3|9|2|3|2009|440|5723|Wednesday|2009Q3|N|N|N|2455076|2455318|2454712|2454986|N|N|N|N|N| +2455078|AAAAAAAAGCGHFCAA|2009-09-03|1316|5723|440|2009|4|9|3|3|2009|440|5723|Thursday|2009Q3|N|N|N|2455076|2455318|2454713|2454987|N|N|N|N|N| +2455079|AAAAAAAAHCGHFCAA|2009-09-04|1316|5723|440|2009|5|9|4|3|2009|440|5723|Friday|2009Q3|N|Y|N|2455076|2455318|2454714|2454988|N|N|N|N|N| +2455080|AAAAAAAAICGHFCAA|2009-09-05|1316|5723|440|2009|6|9|5|3|2009|440|5723|Saturday|2009Q3|N|Y|N|2455076|2455318|2454715|2454989|N|N|N|N|N| +2455081|AAAAAAAAJCGHFCAA|2009-09-06|1316|5723|440|2009|0|9|6|3|2009|440|5723|Sunday|2009Q3|N|N|N|2455076|2455318|2454716|2454990|N|N|N|N|N| +2455082|AAAAAAAAKCGHFCAA|2009-09-07|1316|5723|440|2009|1|9|7|3|2009|440|5723|Monday|2009Q3|N|N|N|2455076|2455318|2454717|2454991|N|N|N|N|N| +2455083|AAAAAAAALCGHFCAA|2009-09-08|1316|5724|440|2009|2|9|8|3|2009|440|5724|Tuesday|2009Q3|N|N|N|2455076|2455318|2454718|2454992|N|N|N|N|N| +2455084|AAAAAAAAMCGHFCAA|2009-09-09|1316|5724|440|2009|3|9|9|3|2009|440|5724|Wednesday|2009Q3|N|N|N|2455076|2455318|2454719|2454993|N|N|N|N|N| +2455085|AAAAAAAANCGHFCAA|2009-09-10|1316|5724|440|2009|4|9|10|3|2009|440|5724|Thursday|2009Q3|N|N|N|2455076|2455318|2454720|2454994|N|N|N|N|N| +2455086|AAAAAAAAOCGHFCAA|2009-09-11|1316|5724|440|2009|5|9|11|3|2009|440|5724|Friday|2009Q3|N|Y|N|2455076|2455318|2454721|2454995|N|N|N|N|N| +2455087|AAAAAAAAPCGHFCAA|2009-09-12|1316|5724|440|2009|6|9|12|3|2009|440|5724|Saturday|2009Q3|N|Y|N|2455076|2455318|2454722|2454996|N|N|N|N|N| +2455088|AAAAAAAAADGHFCAA|2009-09-13|1316|5724|440|2009|0|9|13|3|2009|440|5724|Sunday|2009Q3|N|N|N|2455076|2455318|2454723|2454997|N|N|N|N|N| +2455089|AAAAAAAABDGHFCAA|2009-09-14|1316|5724|440|2009|1|9|14|3|2009|440|5724|Monday|2009Q3|N|N|N|2455076|2455318|2454724|2454998|N|N|N|N|N| +2455090|AAAAAAAACDGHFCAA|2009-09-15|1316|5725|440|2009|2|9|15|3|2009|440|5725|Tuesday|2009Q3|N|N|N|2455076|2455318|2454725|2454999|N|N|N|N|N| +2455091|AAAAAAAADDGHFCAA|2009-09-16|1316|5725|440|2009|3|9|16|3|2009|440|5725|Wednesday|2009Q3|N|N|N|2455076|2455318|2454726|2455000|N|N|N|N|N| +2455092|AAAAAAAAEDGHFCAA|2009-09-17|1316|5725|440|2009|4|9|17|3|2009|440|5725|Thursday|2009Q3|N|N|N|2455076|2455318|2454727|2455001|N|N|N|N|N| +2455093|AAAAAAAAFDGHFCAA|2009-09-18|1316|5725|440|2009|5|9|18|3|2009|440|5725|Friday|2009Q3|N|Y|N|2455076|2455318|2454728|2455002|N|N|N|N|N| +2455094|AAAAAAAAGDGHFCAA|2009-09-19|1316|5725|440|2009|6|9|19|3|2009|440|5725|Saturday|2009Q3|N|Y|N|2455076|2455318|2454729|2455003|N|N|N|N|N| +2455095|AAAAAAAAHDGHFCAA|2009-09-20|1316|5725|440|2009|0|9|20|3|2009|440|5725|Sunday|2009Q3|N|N|N|2455076|2455318|2454730|2455004|N|N|N|N|N| +2455096|AAAAAAAAIDGHFCAA|2009-09-21|1316|5725|440|2009|1|9|21|3|2009|440|5725|Monday|2009Q3|N|N|N|2455076|2455318|2454731|2455005|N|N|N|N|N| +2455097|AAAAAAAAJDGHFCAA|2009-09-22|1316|5726|440|2009|2|9|22|3|2009|440|5726|Tuesday|2009Q3|N|N|N|2455076|2455318|2454732|2455006|N|N|N|N|N| +2455098|AAAAAAAAKDGHFCAA|2009-09-23|1316|5726|440|2009|3|9|23|3|2009|440|5726|Wednesday|2009Q3|N|N|N|2455076|2455318|2454733|2455007|N|N|N|N|N| +2455099|AAAAAAAALDGHFCAA|2009-09-24|1316|5726|440|2009|4|9|24|3|2009|440|5726|Thursday|2009Q3|N|N|N|2455076|2455318|2454734|2455008|N|N|N|N|N| +2455100|AAAAAAAAMDGHFCAA|2009-09-25|1316|5726|440|2009|5|9|25|3|2009|440|5726|Friday|2009Q3|N|Y|N|2455076|2455318|2454735|2455009|N|N|N|N|N| +2455101|AAAAAAAANDGHFCAA|2009-09-26|1316|5726|440|2009|6|9|26|3|2009|440|5726|Saturday|2009Q3|N|Y|N|2455076|2455318|2454736|2455010|N|N|N|N|N| +2455102|AAAAAAAAODGHFCAA|2009-09-27|1316|5726|440|2009|0|9|27|3|2009|440|5726|Sunday|2009Q3|N|N|N|2455076|2455318|2454737|2455011|N|N|N|N|N| +2455103|AAAAAAAAPDGHFCAA|2009-09-28|1316|5726|440|2009|1|9|28|3|2009|440|5726|Monday|2009Q3|N|N|N|2455076|2455318|2454738|2455012|N|N|N|N|N| +2455104|AAAAAAAAAEGHFCAA|2009-09-29|1316|5727|440|2009|2|9|29|3|2009|440|5727|Tuesday|2009Q3|N|N|N|2455076|2455318|2454739|2455013|N|N|N|N|N| +2455105|AAAAAAAABEGHFCAA|2009-09-30|1316|5727|440|2009|3|9|30|3|2009|440|5727|Wednesday|2009Q3|N|N|N|2455076|2455318|2454740|2455014|N|N|N|N|N| +2455106|AAAAAAAACEGHFCAA|2009-10-01|1317|5727|440|2009|4|10|1|3|2009|440|5727|Thursday|2009Q3|N|N|N|2455106|2455378|2454741|2455014|N|N|N|N|N| +2455107|AAAAAAAADEGHFCAA|2009-10-02|1317|5727|440|2009|5|10|2|4|2009|440|5727|Friday|2009Q4|N|Y|N|2455106|2455378|2454742|2455015|N|N|N|N|N| +2455108|AAAAAAAAEEGHFCAA|2009-10-03|1317|5727|440|2009|6|10|3|4|2009|440|5727|Saturday|2009Q4|N|Y|N|2455106|2455378|2454743|2455016|N|N|N|N|N| +2455109|AAAAAAAAFEGHFCAA|2009-10-04|1317|5727|440|2009|0|10|4|4|2009|440|5727|Sunday|2009Q4|N|N|N|2455106|2455378|2454744|2455017|N|N|N|N|N| +2455110|AAAAAAAAGEGHFCAA|2009-10-05|1317|5727|440|2009|1|10|5|4|2009|440|5727|Monday|2009Q4|N|N|N|2455106|2455378|2454745|2455018|N|N|N|N|N| +2455111|AAAAAAAAHEGHFCAA|2009-10-06|1317|5728|440|2009|2|10|6|4|2009|440|5728|Tuesday|2009Q4|N|N|N|2455106|2455378|2454746|2455019|N|N|N|N|N| +2455112|AAAAAAAAIEGHFCAA|2009-10-07|1317|5728|440|2009|3|10|7|4|2009|440|5728|Wednesday|2009Q4|N|N|N|2455106|2455378|2454747|2455020|N|N|N|N|N| +2455113|AAAAAAAAJEGHFCAA|2009-10-08|1317|5728|440|2009|4|10|8|4|2009|440|5728|Thursday|2009Q4|N|N|N|2455106|2455378|2454748|2455021|N|N|N|N|N| +2455114|AAAAAAAAKEGHFCAA|2009-10-09|1317|5728|440|2009|5|10|9|4|2009|440|5728|Friday|2009Q4|N|Y|N|2455106|2455378|2454749|2455022|N|N|N|N|N| +2455115|AAAAAAAALEGHFCAA|2009-10-10|1317|5728|440|2009|6|10|10|4|2009|440|5728|Saturday|2009Q4|N|Y|N|2455106|2455378|2454750|2455023|N|N|N|N|N| +2455116|AAAAAAAAMEGHFCAA|2009-10-11|1317|5728|440|2009|0|10|11|4|2009|440|5728|Sunday|2009Q4|N|N|N|2455106|2455378|2454751|2455024|N|N|N|N|N| +2455117|AAAAAAAANEGHFCAA|2009-10-12|1317|5728|440|2009|1|10|12|4|2009|440|5728|Monday|2009Q4|N|N|N|2455106|2455378|2454752|2455025|N|N|N|N|N| +2455118|AAAAAAAAOEGHFCAA|2009-10-13|1317|5729|440|2009|2|10|13|4|2009|440|5729|Tuesday|2009Q4|N|N|N|2455106|2455378|2454753|2455026|N|N|N|N|N| +2455119|AAAAAAAAPEGHFCAA|2009-10-14|1317|5729|440|2009|3|10|14|4|2009|440|5729|Wednesday|2009Q4|N|N|N|2455106|2455378|2454754|2455027|N|N|N|N|N| +2455120|AAAAAAAAAFGHFCAA|2009-10-15|1317|5729|440|2009|4|10|15|4|2009|440|5729|Thursday|2009Q4|N|N|N|2455106|2455378|2454755|2455028|N|N|N|N|N| +2455121|AAAAAAAABFGHFCAA|2009-10-16|1317|5729|440|2009|5|10|16|4|2009|440|5729|Friday|2009Q4|N|Y|N|2455106|2455378|2454756|2455029|N|N|N|N|N| +2455122|AAAAAAAACFGHFCAA|2009-10-17|1317|5729|440|2009|6|10|17|4|2009|440|5729|Saturday|2009Q4|N|Y|N|2455106|2455378|2454757|2455030|N|N|N|N|N| +2455123|AAAAAAAADFGHFCAA|2009-10-18|1317|5729|440|2009|0|10|18|4|2009|440|5729|Sunday|2009Q4|N|N|N|2455106|2455378|2454758|2455031|N|N|N|N|N| +2455124|AAAAAAAAEFGHFCAA|2009-10-19|1317|5729|440|2009|1|10|19|4|2009|440|5729|Monday|2009Q4|N|N|N|2455106|2455378|2454759|2455032|N|N|N|N|N| +2455125|AAAAAAAAFFGHFCAA|2009-10-20|1317|5730|440|2009|2|10|20|4|2009|440|5730|Tuesday|2009Q4|N|N|N|2455106|2455378|2454760|2455033|N|N|N|N|N| +2455126|AAAAAAAAGFGHFCAA|2009-10-21|1317|5730|440|2009|3|10|21|4|2009|440|5730|Wednesday|2009Q4|N|N|N|2455106|2455378|2454761|2455034|N|N|N|N|N| +2455127|AAAAAAAAHFGHFCAA|2009-10-22|1317|5730|440|2009|4|10|22|4|2009|440|5730|Thursday|2009Q4|N|N|N|2455106|2455378|2454762|2455035|N|N|N|N|N| +2455128|AAAAAAAAIFGHFCAA|2009-10-23|1317|5730|440|2009|5|10|23|4|2009|440|5730|Friday|2009Q4|N|Y|N|2455106|2455378|2454763|2455036|N|N|N|N|N| +2455129|AAAAAAAAJFGHFCAA|2009-10-24|1317|5730|440|2009|6|10|24|4|2009|440|5730|Saturday|2009Q4|N|Y|N|2455106|2455378|2454764|2455037|N|N|N|N|N| +2455130|AAAAAAAAKFGHFCAA|2009-10-25|1317|5730|440|2009|0|10|25|4|2009|440|5730|Sunday|2009Q4|N|N|N|2455106|2455378|2454765|2455038|N|N|N|N|N| +2455131|AAAAAAAALFGHFCAA|2009-10-26|1317|5730|440|2009|1|10|26|4|2009|440|5730|Monday|2009Q4|N|N|N|2455106|2455378|2454766|2455039|N|N|N|N|N| +2455132|AAAAAAAAMFGHFCAA|2009-10-27|1317|5731|440|2009|2|10|27|4|2009|440|5731|Tuesday|2009Q4|N|N|N|2455106|2455378|2454767|2455040|N|N|N|N|N| +2455133|AAAAAAAANFGHFCAA|2009-10-28|1317|5731|440|2009|3|10|28|4|2009|440|5731|Wednesday|2009Q4|N|N|N|2455106|2455378|2454768|2455041|N|N|N|N|N| +2455134|AAAAAAAAOFGHFCAA|2009-10-29|1317|5731|440|2009|4|10|29|4|2009|440|5731|Thursday|2009Q4|N|N|N|2455106|2455378|2454769|2455042|N|N|N|N|N| +2455135|AAAAAAAAPFGHFCAA|2009-10-30|1317|5731|440|2009|5|10|30|4|2009|440|5731|Friday|2009Q4|N|Y|N|2455106|2455378|2454770|2455043|N|N|N|N|N| +2455136|AAAAAAAAAGGHFCAA|2009-10-31|1317|5731|440|2009|6|10|31|4|2009|440|5731|Saturday|2009Q4|N|Y|N|2455106|2455378|2454771|2455044|N|N|N|N|N| +2455137|AAAAAAAABGGHFCAA|2009-11-01|1318|5731|440|2009|0|11|1|4|2009|440|5731|Sunday|2009Q4|N|N|N|2455137|2455440|2454772|2455045|N|N|N|N|N| +2455138|AAAAAAAACGGHFCAA|2009-11-02|1318|5731|440|2009|1|11|2|4|2009|440|5731|Monday|2009Q4|N|N|N|2455137|2455440|2454773|2455046|N|N|N|N|N| +2455139|AAAAAAAADGGHFCAA|2009-11-03|1318|5732|440|2009|2|11|3|4|2009|440|5732|Tuesday|2009Q4|N|N|N|2455137|2455440|2454774|2455047|N|N|N|N|N| +2455140|AAAAAAAAEGGHFCAA|2009-11-04|1318|5732|440|2009|3|11|4|4|2009|440|5732|Wednesday|2009Q4|N|N|N|2455137|2455440|2454775|2455048|N|N|N|N|N| +2455141|AAAAAAAAFGGHFCAA|2009-11-05|1318|5732|440|2009|4|11|5|4|2009|440|5732|Thursday|2009Q4|N|N|N|2455137|2455440|2454776|2455049|N|N|N|N|N| +2455142|AAAAAAAAGGGHFCAA|2009-11-06|1318|5732|440|2009|5|11|6|4|2009|440|5732|Friday|2009Q4|N|Y|N|2455137|2455440|2454777|2455050|N|N|N|N|N| +2455143|AAAAAAAAHGGHFCAA|2009-11-07|1318|5732|440|2009|6|11|7|4|2009|440|5732|Saturday|2009Q4|N|Y|N|2455137|2455440|2454778|2455051|N|N|N|N|N| +2455144|AAAAAAAAIGGHFCAA|2009-11-08|1318|5732|440|2009|0|11|8|4|2009|440|5732|Sunday|2009Q4|N|N|N|2455137|2455440|2454779|2455052|N|N|N|N|N| +2455145|AAAAAAAAJGGHFCAA|2009-11-09|1318|5732|440|2009|1|11|9|4|2009|440|5732|Monday|2009Q4|N|N|N|2455137|2455440|2454780|2455053|N|N|N|N|N| +2455146|AAAAAAAAKGGHFCAA|2009-11-10|1318|5733|440|2009|2|11|10|4|2009|440|5733|Tuesday|2009Q4|N|N|N|2455137|2455440|2454781|2455054|N|N|N|N|N| +2455147|AAAAAAAALGGHFCAA|2009-11-11|1318|5733|440|2009|3|11|11|4|2009|440|5733|Wednesday|2009Q4|N|N|N|2455137|2455440|2454782|2455055|N|N|N|N|N| +2455148|AAAAAAAAMGGHFCAA|2009-11-12|1318|5733|440|2009|4|11|12|4|2009|440|5733|Thursday|2009Q4|N|N|N|2455137|2455440|2454783|2455056|N|N|N|N|N| +2455149|AAAAAAAANGGHFCAA|2009-11-13|1318|5733|440|2009|5|11|13|4|2009|440|5733|Friday|2009Q4|N|Y|N|2455137|2455440|2454784|2455057|N|N|N|N|N| +2455150|AAAAAAAAOGGHFCAA|2009-11-14|1318|5733|440|2009|6|11|14|4|2009|440|5733|Saturday|2009Q4|N|Y|N|2455137|2455440|2454785|2455058|N|N|N|N|N| +2455151|AAAAAAAAPGGHFCAA|2009-11-15|1318|5733|440|2009|0|11|15|4|2009|440|5733|Sunday|2009Q4|N|N|N|2455137|2455440|2454786|2455059|N|N|N|N|N| +2455152|AAAAAAAAAHGHFCAA|2009-11-16|1318|5733|440|2009|1|11|16|4|2009|440|5733|Monday|2009Q4|N|N|N|2455137|2455440|2454787|2455060|N|N|N|N|N| +2455153|AAAAAAAABHGHFCAA|2009-11-17|1318|5734|440|2009|2|11|17|4|2009|440|5734|Tuesday|2009Q4|N|N|N|2455137|2455440|2454788|2455061|N|N|N|N|N| +2455154|AAAAAAAACHGHFCAA|2009-11-18|1318|5734|440|2009|3|11|18|4|2009|440|5734|Wednesday|2009Q4|N|N|N|2455137|2455440|2454789|2455062|N|N|N|N|N| +2455155|AAAAAAAADHGHFCAA|2009-11-19|1318|5734|440|2009|4|11|19|4|2009|440|5734|Thursday|2009Q4|N|N|N|2455137|2455440|2454790|2455063|N|N|N|N|N| +2455156|AAAAAAAAEHGHFCAA|2009-11-20|1318|5734|440|2009|5|11|20|4|2009|440|5734|Friday|2009Q4|N|Y|N|2455137|2455440|2454791|2455064|N|N|N|N|N| +2455157|AAAAAAAAFHGHFCAA|2009-11-21|1318|5734|440|2009|6|11|21|4|2009|440|5734|Saturday|2009Q4|N|Y|N|2455137|2455440|2454792|2455065|N|N|N|N|N| +2455158|AAAAAAAAGHGHFCAA|2009-11-22|1318|5734|440|2009|0|11|22|4|2009|440|5734|Sunday|2009Q4|N|N|N|2455137|2455440|2454793|2455066|N|N|N|N|N| +2455159|AAAAAAAAHHGHFCAA|2009-11-23|1318|5734|440|2009|1|11|23|4|2009|440|5734|Monday|2009Q4|N|N|N|2455137|2455440|2454794|2455067|N|N|N|N|N| +2455160|AAAAAAAAIHGHFCAA|2009-11-24|1318|5735|440|2009|2|11|24|4|2009|440|5735|Tuesday|2009Q4|N|N|N|2455137|2455440|2454795|2455068|N|N|N|N|N| +2455161|AAAAAAAAJHGHFCAA|2009-11-25|1318|5735|440|2009|3|11|25|4|2009|440|5735|Wednesday|2009Q4|N|N|N|2455137|2455440|2454796|2455069|N|N|N|N|N| +2455162|AAAAAAAAKHGHFCAA|2009-11-26|1318|5735|440|2009|4|11|26|4|2009|440|5735|Thursday|2009Q4|N|N|N|2455137|2455440|2454797|2455070|N|N|N|N|N| +2455163|AAAAAAAALHGHFCAA|2009-11-27|1318|5735|440|2009|5|11|27|4|2009|440|5735|Friday|2009Q4|N|Y|N|2455137|2455440|2454798|2455071|N|N|N|N|N| +2455164|AAAAAAAAMHGHFCAA|2009-11-28|1318|5735|440|2009|6|11|28|4|2009|440|5735|Saturday|2009Q4|N|Y|N|2455137|2455440|2454799|2455072|N|N|N|N|N| +2455165|AAAAAAAANHGHFCAA|2009-11-29|1318|5735|440|2009|0|11|29|4|2009|440|5735|Sunday|2009Q4|N|N|N|2455137|2455440|2454800|2455073|N|N|N|N|N| +2455166|AAAAAAAAOHGHFCAA|2009-11-30|1318|5735|440|2009|1|11|30|4|2009|440|5735|Monday|2009Q4|N|N|N|2455137|2455440|2454801|2455074|N|N|N|N|N| +2455167|AAAAAAAAPHGHFCAA|2009-12-01|1319|5736|441|2009|2|12|1|4|2009|441|5736|Tuesday|2009Q4|N|N|N|2455167|2455500|2454802|2455075|N|N|N|N|N| +2455168|AAAAAAAAAIGHFCAA|2009-12-02|1319|5736|441|2009|3|12|2|4|2009|441|5736|Wednesday|2009Q4|N|N|N|2455167|2455500|2454803|2455076|N|N|N|N|N| +2455169|AAAAAAAABIGHFCAA|2009-12-03|1319|5736|441|2009|4|12|3|4|2009|441|5736|Thursday|2009Q4|N|N|N|2455167|2455500|2454804|2455077|N|N|N|N|N| +2455170|AAAAAAAACIGHFCAA|2009-12-04|1319|5736|441|2009|5|12|4|4|2009|441|5736|Friday|2009Q4|N|Y|N|2455167|2455500|2454805|2455078|N|N|N|N|N| +2455171|AAAAAAAADIGHFCAA|2009-12-05|1319|5736|441|2009|6|12|5|4|2009|441|5736|Saturday|2009Q4|N|Y|N|2455167|2455500|2454806|2455079|N|N|N|N|N| +2455172|AAAAAAAAEIGHFCAA|2009-12-06|1319|5736|441|2009|0|12|6|4|2009|441|5736|Sunday|2009Q4|N|N|N|2455167|2455500|2454807|2455080|N|N|N|N|N| +2455173|AAAAAAAAFIGHFCAA|2009-12-07|1319|5736|441|2009|1|12|7|4|2009|441|5736|Monday|2009Q4|N|N|N|2455167|2455500|2454808|2455081|N|N|N|N|N| +2455174|AAAAAAAAGIGHFCAA|2009-12-08|1319|5737|441|2009|2|12|8|4|2009|441|5737|Tuesday|2009Q4|N|N|N|2455167|2455500|2454809|2455082|N|N|N|N|N| +2455175|AAAAAAAAHIGHFCAA|2009-12-09|1319|5737|441|2009|3|12|9|4|2009|441|5737|Wednesday|2009Q4|N|N|N|2455167|2455500|2454810|2455083|N|N|N|N|N| +2455176|AAAAAAAAIIGHFCAA|2009-12-10|1319|5737|441|2009|4|12|10|4|2009|441|5737|Thursday|2009Q4|N|N|N|2455167|2455500|2454811|2455084|N|N|N|N|N| +2455177|AAAAAAAAJIGHFCAA|2009-12-11|1319|5737|441|2009|5|12|11|4|2009|441|5737|Friday|2009Q4|N|Y|N|2455167|2455500|2454812|2455085|N|N|N|N|N| +2455178|AAAAAAAAKIGHFCAA|2009-12-12|1319|5737|441|2009|6|12|12|4|2009|441|5737|Saturday|2009Q4|N|Y|N|2455167|2455500|2454813|2455086|N|N|N|N|N| +2455179|AAAAAAAALIGHFCAA|2009-12-13|1319|5737|441|2009|0|12|13|4|2009|441|5737|Sunday|2009Q4|N|N|N|2455167|2455500|2454814|2455087|N|N|N|N|N| +2455180|AAAAAAAAMIGHFCAA|2009-12-14|1319|5737|441|2009|1|12|14|4|2009|441|5737|Monday|2009Q4|N|N|N|2455167|2455500|2454815|2455088|N|N|N|N|N| +2455181|AAAAAAAANIGHFCAA|2009-12-15|1319|5738|441|2009|2|12|15|4|2009|441|5738|Tuesday|2009Q4|N|N|N|2455167|2455500|2454816|2455089|N|N|N|N|N| +2455182|AAAAAAAAOIGHFCAA|2009-12-16|1319|5738|441|2009|3|12|16|4|2009|441|5738|Wednesday|2009Q4|N|N|N|2455167|2455500|2454817|2455090|N|N|N|N|N| +2455183|AAAAAAAAPIGHFCAA|2009-12-17|1319|5738|441|2009|4|12|17|4|2009|441|5738|Thursday|2009Q4|N|N|N|2455167|2455500|2454818|2455091|N|N|N|N|N| +2455184|AAAAAAAAAJGHFCAA|2009-12-18|1319|5738|441|2009|5|12|18|4|2009|441|5738|Friday|2009Q4|N|Y|N|2455167|2455500|2454819|2455092|N|N|N|N|N| +2455185|AAAAAAAABJGHFCAA|2009-12-19|1319|5738|441|2009|6|12|19|4|2009|441|5738|Saturday|2009Q4|N|Y|N|2455167|2455500|2454820|2455093|N|N|N|N|N| +2455186|AAAAAAAACJGHFCAA|2009-12-20|1319|5738|441|2009|0|12|20|4|2009|441|5738|Sunday|2009Q4|N|N|N|2455167|2455500|2454821|2455094|N|N|N|N|N| +2455187|AAAAAAAADJGHFCAA|2009-12-21|1319|5738|441|2009|1|12|21|4|2009|441|5738|Monday|2009Q4|N|N|N|2455167|2455500|2454822|2455095|N|N|N|N|N| +2455188|AAAAAAAAEJGHFCAA|2009-12-22|1319|5739|441|2009|2|12|22|4|2009|441|5739|Tuesday|2009Q4|N|N|N|2455167|2455500|2454823|2455096|N|N|N|N|N| +2455189|AAAAAAAAFJGHFCAA|2009-12-23|1319|5739|441|2009|3|12|23|4|2009|441|5739|Wednesday|2009Q4|N|N|N|2455167|2455500|2454824|2455097|N|N|N|N|N| +2455190|AAAAAAAAGJGHFCAA|2009-12-24|1319|5739|441|2009|4|12|24|4|2009|441|5739|Thursday|2009Q4|N|N|N|2455167|2455500|2454825|2455098|N|N|N|N|N| +2455191|AAAAAAAAHJGHFCAA|2009-12-25|1319|5739|441|2009|5|12|25|4|2009|441|5739|Friday|2009Q4|N|Y|N|2455167|2455500|2454826|2455099|N|N|N|N|N| +2455192|AAAAAAAAIJGHFCAA|2009-12-26|1319|5739|441|2009|6|12|26|4|2009|441|5739|Saturday|2009Q4|Y|Y|N|2455167|2455500|2454827|2455100|N|N|N|N|N| +2455193|AAAAAAAAJJGHFCAA|2009-12-27|1319|5739|441|2009|0|12|27|4|2009|441|5739|Sunday|2009Q4|N|N|Y|2455167|2455500|2454828|2455101|N|N|N|N|N| +2455194|AAAAAAAAKJGHFCAA|2009-12-28|1319|5739|441|2009|1|12|28|4|2009|441|5739|Monday|2009Q4|N|N|N|2455167|2455500|2454829|2455102|N|N|N|N|N| +2455195|AAAAAAAALJGHFCAA|2009-12-29|1319|5740|441|2009|2|12|29|4|2009|441|5740|Tuesday|2009Q4|N|N|N|2455167|2455500|2454830|2455103|N|N|N|N|N| +2455196|AAAAAAAAMJGHFCAA|2009-12-30|1319|5740|441|2009|3|12|30|4|2009|441|5740|Wednesday|2009Q4|N|N|N|2455167|2455500|2454831|2455104|N|N|N|N|N| +2455197|AAAAAAAANJGHFCAA|2009-12-31|1319|5740|441|2009|4|12|31|4|2009|441|5740|Thursday|2009Q4|N|N|N|2455167|2455500|2454832|2455105|N|N|N|N|N| +2455198|AAAAAAAAOJGHFCAA|2010-01-01|1320|5740|441|2010|5|1|1|1|2010|441|5740|Friday|2010Q1|Y|Y|N|2455198|2455197|2454833|2455106|N|N|N|N|N| +2455199|AAAAAAAAPJGHFCAA|2010-01-02|1320|5740|441|2010|6|1|2|1|2010|441|5740|Saturday|2010Q1|N|Y|Y|2455198|2455197|2454834|2455107|N|N|N|N|N| +2455200|AAAAAAAAAKGHFCAA|2010-01-03|1320|5740|441|2010|0|1|3|1|2010|441|5740|Sunday|2010Q1|N|N|N|2455198|2455197|2454835|2455108|N|N|N|N|N| +2455201|AAAAAAAABKGHFCAA|2010-01-04|1320|5740|441|2010|1|1|4|1|2010|441|5740|Monday|2010Q1|N|N|N|2455198|2455197|2454836|2455109|N|N|N|N|N| +2455202|AAAAAAAACKGHFCAA|2010-01-05|1320|5741|441|2010|2|1|5|1|2010|441|5741|Tuesday|2010Q1|N|N|N|2455198|2455197|2454837|2455110|N|N|N|N|N| +2455203|AAAAAAAADKGHFCAA|2010-01-06|1320|5741|441|2010|3|1|6|1|2010|441|5741|Wednesday|2010Q1|N|N|N|2455198|2455197|2454838|2455111|N|N|N|N|N| +2455204|AAAAAAAAEKGHFCAA|2010-01-07|1320|5741|441|2010|4|1|7|1|2010|441|5741|Thursday|2010Q1|N|N|N|2455198|2455197|2454839|2455112|N|N|N|N|N| +2455205|AAAAAAAAFKGHFCAA|2010-01-08|1320|5741|441|2010|5|1|8|1|2010|441|5741|Friday|2010Q1|N|Y|N|2455198|2455197|2454840|2455113|N|N|N|N|N| +2455206|AAAAAAAAGKGHFCAA|2010-01-09|1320|5741|441|2010|6|1|9|1|2010|441|5741|Saturday|2010Q1|N|Y|N|2455198|2455197|2454841|2455114|N|N|N|N|N| +2455207|AAAAAAAAHKGHFCAA|2010-01-10|1320|5741|441|2010|0|1|10|1|2010|441|5741|Sunday|2010Q1|N|N|N|2455198|2455197|2454842|2455115|N|N|N|N|N| +2455208|AAAAAAAAIKGHFCAA|2010-01-11|1320|5741|441|2010|1|1|11|1|2010|441|5741|Monday|2010Q1|N|N|N|2455198|2455197|2454843|2455116|N|N|N|N|N| +2455209|AAAAAAAAJKGHFCAA|2010-01-12|1320|5742|441|2010|2|1|12|1|2010|441|5742|Tuesday|2010Q1|N|N|N|2455198|2455197|2454844|2455117|N|N|N|N|N| +2455210|AAAAAAAAKKGHFCAA|2010-01-13|1320|5742|441|2010|3|1|13|1|2010|441|5742|Wednesday|2010Q1|N|N|N|2455198|2455197|2454845|2455118|N|N|N|N|N| +2455211|AAAAAAAALKGHFCAA|2010-01-14|1320|5742|441|2010|4|1|14|1|2010|441|5742|Thursday|2010Q1|N|N|N|2455198|2455197|2454846|2455119|N|N|N|N|N| +2455212|AAAAAAAAMKGHFCAA|2010-01-15|1320|5742|441|2010|5|1|15|1|2010|441|5742|Friday|2010Q1|N|Y|N|2455198|2455197|2454847|2455120|N|N|N|N|N| +2455213|AAAAAAAANKGHFCAA|2010-01-16|1320|5742|441|2010|6|1|16|1|2010|441|5742|Saturday|2010Q1|N|Y|N|2455198|2455197|2454848|2455121|N|N|N|N|N| +2455214|AAAAAAAAOKGHFCAA|2010-01-17|1320|5742|441|2010|0|1|17|1|2010|441|5742|Sunday|2010Q1|N|N|N|2455198|2455197|2454849|2455122|N|N|N|N|N| +2455215|AAAAAAAAPKGHFCAA|2010-01-18|1320|5742|441|2010|1|1|18|1|2010|441|5742|Monday|2010Q1|N|N|N|2455198|2455197|2454850|2455123|N|N|N|N|N| +2455216|AAAAAAAAALGHFCAA|2010-01-19|1320|5743|441|2010|2|1|19|1|2010|441|5743|Tuesday|2010Q1|N|N|N|2455198|2455197|2454851|2455124|N|N|N|N|N| +2455217|AAAAAAAABLGHFCAA|2010-01-20|1320|5743|441|2010|3|1|20|1|2010|441|5743|Wednesday|2010Q1|N|N|N|2455198|2455197|2454852|2455125|N|N|N|N|N| +2455218|AAAAAAAACLGHFCAA|2010-01-21|1320|5743|441|2010|4|1|21|1|2010|441|5743|Thursday|2010Q1|N|N|N|2455198|2455197|2454853|2455126|N|N|N|N|N| +2455219|AAAAAAAADLGHFCAA|2010-01-22|1320|5743|441|2010|5|1|22|1|2010|441|5743|Friday|2010Q1|N|Y|N|2455198|2455197|2454854|2455127|N|N|N|N|N| +2455220|AAAAAAAAELGHFCAA|2010-01-23|1320|5743|441|2010|6|1|23|1|2010|441|5743|Saturday|2010Q1|N|Y|N|2455198|2455197|2454855|2455128|N|N|N|N|N| +2455221|AAAAAAAAFLGHFCAA|2010-01-24|1320|5743|441|2010|0|1|24|1|2010|441|5743|Sunday|2010Q1|N|N|N|2455198|2455197|2454856|2455129|N|N|N|N|N| +2455222|AAAAAAAAGLGHFCAA|2010-01-25|1320|5743|441|2010|1|1|25|1|2010|441|5743|Monday|2010Q1|N|N|N|2455198|2455197|2454857|2455130|N|N|N|N|N| +2455223|AAAAAAAAHLGHFCAA|2010-01-26|1320|5744|441|2010|2|1|26|1|2010|441|5744|Tuesday|2010Q1|N|N|N|2455198|2455197|2454858|2455131|N|N|N|N|N| +2455224|AAAAAAAAILGHFCAA|2010-01-27|1320|5744|441|2010|3|1|27|1|2010|441|5744|Wednesday|2010Q1|N|N|N|2455198|2455197|2454859|2455132|N|N|N|N|N| +2455225|AAAAAAAAJLGHFCAA|2010-01-28|1320|5744|441|2010|4|1|28|1|2010|441|5744|Thursday|2010Q1|N|N|N|2455198|2455197|2454860|2455133|N|N|N|N|N| +2455226|AAAAAAAAKLGHFCAA|2010-01-29|1320|5744|441|2010|5|1|29|1|2010|441|5744|Friday|2010Q1|N|Y|N|2455198|2455197|2454861|2455134|N|N|N|N|N| +2455227|AAAAAAAALLGHFCAA|2010-01-30|1320|5744|441|2010|6|1|30|1|2010|441|5744|Saturday|2010Q1|N|Y|N|2455198|2455197|2454862|2455135|N|N|N|N|N| +2455228|AAAAAAAAMLGHFCAA|2010-01-31|1320|5744|441|2010|0|1|31|1|2010|441|5744|Sunday|2010Q1|N|N|N|2455198|2455197|2454863|2455136|N|N|N|N|N| +2455229|AAAAAAAANLGHFCAA|2010-02-01|1321|5744|441|2010|1|2|1|1|2010|441|5744|Monday|2010Q1|N|N|N|2455229|2455259|2454864|2455137|N|N|N|N|N| +2455230|AAAAAAAAOLGHFCAA|2010-02-02|1321|5745|441|2010|2|2|2|1|2010|441|5745|Tuesday|2010Q1|N|N|N|2455229|2455259|2454865|2455138|N|N|N|N|N| +2455231|AAAAAAAAPLGHFCAA|2010-02-03|1321|5745|441|2010|3|2|3|1|2010|441|5745|Wednesday|2010Q1|N|N|N|2455229|2455259|2454866|2455139|N|N|N|N|N| +2455232|AAAAAAAAAMGHFCAA|2010-02-04|1321|5745|441|2010|4|2|4|1|2010|441|5745|Thursday|2010Q1|N|N|N|2455229|2455259|2454867|2455140|N|N|N|N|N| +2455233|AAAAAAAABMGHFCAA|2010-02-05|1321|5745|441|2010|5|2|5|1|2010|441|5745|Friday|2010Q1|N|Y|N|2455229|2455259|2454868|2455141|N|N|N|N|N| +2455234|AAAAAAAACMGHFCAA|2010-02-06|1321|5745|441|2010|6|2|6|1|2010|441|5745|Saturday|2010Q1|N|Y|N|2455229|2455259|2454869|2455142|N|N|N|N|N| +2455235|AAAAAAAADMGHFCAA|2010-02-07|1321|5745|441|2010|0|2|7|1|2010|441|5745|Sunday|2010Q1|N|N|N|2455229|2455259|2454870|2455143|N|N|N|N|N| +2455236|AAAAAAAAEMGHFCAA|2010-02-08|1321|5745|441|2010|1|2|8|1|2010|441|5745|Monday|2010Q1|N|N|N|2455229|2455259|2454871|2455144|N|N|N|N|N| +2455237|AAAAAAAAFMGHFCAA|2010-02-09|1321|5746|441|2010|2|2|9|1|2010|441|5746|Tuesday|2010Q1|N|N|N|2455229|2455259|2454872|2455145|N|N|N|N|N| +2455238|AAAAAAAAGMGHFCAA|2010-02-10|1321|5746|441|2010|3|2|10|1|2010|441|5746|Wednesday|2010Q1|N|N|N|2455229|2455259|2454873|2455146|N|N|N|N|N| +2455239|AAAAAAAAHMGHFCAA|2010-02-11|1321|5746|441|2010|4|2|11|1|2010|441|5746|Thursday|2010Q1|N|N|N|2455229|2455259|2454874|2455147|N|N|N|N|N| +2455240|AAAAAAAAIMGHFCAA|2010-02-12|1321|5746|441|2010|5|2|12|1|2010|441|5746|Friday|2010Q1|N|Y|N|2455229|2455259|2454875|2455148|N|N|N|N|N| +2455241|AAAAAAAAJMGHFCAA|2010-02-13|1321|5746|441|2010|6|2|13|1|2010|441|5746|Saturday|2010Q1|N|Y|N|2455229|2455259|2454876|2455149|N|N|N|N|N| +2455242|AAAAAAAAKMGHFCAA|2010-02-14|1321|5746|441|2010|0|2|14|1|2010|441|5746|Sunday|2010Q1|N|N|N|2455229|2455259|2454877|2455150|N|N|N|N|N| +2455243|AAAAAAAALMGHFCAA|2010-02-15|1321|5746|441|2010|1|2|15|1|2010|441|5746|Monday|2010Q1|N|N|N|2455229|2455259|2454878|2455151|N|N|N|N|N| +2455244|AAAAAAAAMMGHFCAA|2010-02-16|1321|5747|441|2010|2|2|16|1|2010|441|5747|Tuesday|2010Q1|N|N|N|2455229|2455259|2454879|2455152|N|N|N|N|N| +2455245|AAAAAAAANMGHFCAA|2010-02-17|1321|5747|441|2010|3|2|17|1|2010|441|5747|Wednesday|2010Q1|N|N|N|2455229|2455259|2454880|2455153|N|N|N|N|N| +2455246|AAAAAAAAOMGHFCAA|2010-02-18|1321|5747|441|2010|4|2|18|1|2010|441|5747|Thursday|2010Q1|N|N|N|2455229|2455259|2454881|2455154|N|N|N|N|N| +2455247|AAAAAAAAPMGHFCAA|2010-02-19|1321|5747|441|2010|5|2|19|1|2010|441|5747|Friday|2010Q1|N|Y|N|2455229|2455259|2454882|2455155|N|N|N|N|N| +2455248|AAAAAAAAANGHFCAA|2010-02-20|1321|5747|441|2010|6|2|20|1|2010|441|5747|Saturday|2010Q1|N|Y|N|2455229|2455259|2454883|2455156|N|N|N|N|N| +2455249|AAAAAAAABNGHFCAA|2010-02-21|1321|5747|441|2010|0|2|21|1|2010|441|5747|Sunday|2010Q1|N|N|N|2455229|2455259|2454884|2455157|N|N|N|N|N| +2455250|AAAAAAAACNGHFCAA|2010-02-22|1321|5747|441|2010|1|2|22|1|2010|441|5747|Monday|2010Q1|N|N|N|2455229|2455259|2454885|2455158|N|N|N|N|N| +2455251|AAAAAAAADNGHFCAA|2010-02-23|1321|5748|441|2010|2|2|23|1|2010|441|5748|Tuesday|2010Q1|N|N|N|2455229|2455259|2454886|2455159|N|N|N|N|N| +2455252|AAAAAAAAENGHFCAA|2010-02-24|1321|5748|441|2010|3|2|24|1|2010|441|5748|Wednesday|2010Q1|N|N|N|2455229|2455259|2454887|2455160|N|N|N|N|N| +2455253|AAAAAAAAFNGHFCAA|2010-02-25|1321|5748|441|2010|4|2|25|1|2010|441|5748|Thursday|2010Q1|N|N|N|2455229|2455259|2454888|2455161|N|N|N|N|N| +2455254|AAAAAAAAGNGHFCAA|2010-02-26|1321|5748|441|2010|5|2|26|1|2010|441|5748|Friday|2010Q1|N|Y|N|2455229|2455259|2454889|2455162|N|N|N|N|N| +2455255|AAAAAAAAHNGHFCAA|2010-02-27|1321|5748|441|2010|6|2|27|1|2010|441|5748|Saturday|2010Q1|N|Y|N|2455229|2455259|2454890|2455163|N|N|N|N|N| +2455256|AAAAAAAAINGHFCAA|2010-02-28|1321|5748|441|2010|0|2|28|1|2010|441|5748|Sunday|2010Q1|N|N|N|2455229|2455259|2454891|2455164|N|N|N|N|N| +2455257|AAAAAAAAJNGHFCAA|2010-03-01|1322|5748|442|2010|1|3|1|1|2010|442|5748|Monday|2010Q1|N|N|N|2455257|2455315|2454892|2455165|N|N|N|N|N| +2455258|AAAAAAAAKNGHFCAA|2010-03-02|1322|5749|442|2010|2|3|2|1|2010|442|5749|Tuesday|2010Q1|N|N|N|2455257|2455315|2454893|2455166|N|N|N|N|N| +2455259|AAAAAAAALNGHFCAA|2010-03-03|1322|5749|442|2010|3|3|3|1|2010|442|5749|Wednesday|2010Q1|N|N|N|2455257|2455315|2454894|2455167|N|N|N|N|N| +2455260|AAAAAAAAMNGHFCAA|2010-03-04|1322|5749|442|2010|4|3|4|1|2010|442|5749|Thursday|2010Q1|N|N|N|2455257|2455315|2454895|2455168|N|N|N|N|N| +2455261|AAAAAAAANNGHFCAA|2010-03-05|1322|5749|442|2010|5|3|5|1|2010|442|5749|Friday|2010Q1|N|Y|N|2455257|2455315|2454896|2455169|N|N|N|N|N| +2455262|AAAAAAAAONGHFCAA|2010-03-06|1322|5749|442|2010|6|3|6|1|2010|442|5749|Saturday|2010Q1|N|Y|N|2455257|2455315|2454897|2455170|N|N|N|N|N| +2455263|AAAAAAAAPNGHFCAA|2010-03-07|1322|5749|442|2010|0|3|7|1|2010|442|5749|Sunday|2010Q1|N|N|N|2455257|2455315|2454898|2455171|N|N|N|N|N| +2455264|AAAAAAAAAOGHFCAA|2010-03-08|1322|5749|442|2010|1|3|8|1|2010|442|5749|Monday|2010Q1|N|N|N|2455257|2455315|2454899|2455172|N|N|N|N|N| +2455265|AAAAAAAABOGHFCAA|2010-03-09|1322|5750|442|2010|2|3|9|1|2010|442|5750|Tuesday|2010Q1|N|N|N|2455257|2455315|2454900|2455173|N|N|N|N|N| +2455266|AAAAAAAACOGHFCAA|2010-03-10|1322|5750|442|2010|3|3|10|1|2010|442|5750|Wednesday|2010Q1|N|N|N|2455257|2455315|2454901|2455174|N|N|N|N|N| +2455267|AAAAAAAADOGHFCAA|2010-03-11|1322|5750|442|2010|4|3|11|1|2010|442|5750|Thursday|2010Q1|N|N|N|2455257|2455315|2454902|2455175|N|N|N|N|N| +2455268|AAAAAAAAEOGHFCAA|2010-03-12|1322|5750|442|2010|5|3|12|1|2010|442|5750|Friday|2010Q1|N|Y|N|2455257|2455315|2454903|2455176|N|N|N|N|N| +2455269|AAAAAAAAFOGHFCAA|2010-03-13|1322|5750|442|2010|6|3|13|1|2010|442|5750|Saturday|2010Q1|N|Y|N|2455257|2455315|2454904|2455177|N|N|N|N|N| +2455270|AAAAAAAAGOGHFCAA|2010-03-14|1322|5750|442|2010|0|3|14|1|2010|442|5750|Sunday|2010Q1|N|N|N|2455257|2455315|2454905|2455178|N|N|N|N|N| +2455271|AAAAAAAAHOGHFCAA|2010-03-15|1322|5750|442|2010|1|3|15|1|2010|442|5750|Monday|2010Q1|N|N|N|2455257|2455315|2454906|2455179|N|N|N|N|N| +2455272|AAAAAAAAIOGHFCAA|2010-03-16|1322|5751|442|2010|2|3|16|1|2010|442|5751|Tuesday|2010Q1|N|N|N|2455257|2455315|2454907|2455180|N|N|N|N|N| +2455273|AAAAAAAAJOGHFCAA|2010-03-17|1322|5751|442|2010|3|3|17|1|2010|442|5751|Wednesday|2010Q1|N|N|N|2455257|2455315|2454908|2455181|N|N|N|N|N| +2455274|AAAAAAAAKOGHFCAA|2010-03-18|1322|5751|442|2010|4|3|18|1|2010|442|5751|Thursday|2010Q1|N|N|N|2455257|2455315|2454909|2455182|N|N|N|N|N| +2455275|AAAAAAAALOGHFCAA|2010-03-19|1322|5751|442|2010|5|3|19|1|2010|442|5751|Friday|2010Q1|N|Y|N|2455257|2455315|2454910|2455183|N|N|N|N|N| +2455276|AAAAAAAAMOGHFCAA|2010-03-20|1322|5751|442|2010|6|3|20|1|2010|442|5751|Saturday|2010Q1|N|Y|N|2455257|2455315|2454911|2455184|N|N|N|N|N| +2455277|AAAAAAAANOGHFCAA|2010-03-21|1322|5751|442|2010|0|3|21|1|2010|442|5751|Sunday|2010Q1|N|N|N|2455257|2455315|2454912|2455185|N|N|N|N|N| +2455278|AAAAAAAAOOGHFCAA|2010-03-22|1322|5751|442|2010|1|3|22|1|2010|442|5751|Monday|2010Q1|N|N|N|2455257|2455315|2454913|2455186|N|N|N|N|N| +2455279|AAAAAAAAPOGHFCAA|2010-03-23|1322|5752|442|2010|2|3|23|1|2010|442|5752|Tuesday|2010Q1|N|N|N|2455257|2455315|2454914|2455187|N|N|N|N|N| +2455280|AAAAAAAAAPGHFCAA|2010-03-24|1322|5752|442|2010|3|3|24|1|2010|442|5752|Wednesday|2010Q1|N|N|N|2455257|2455315|2454915|2455188|N|N|N|N|N| +2455281|AAAAAAAABPGHFCAA|2010-03-25|1322|5752|442|2010|4|3|25|1|2010|442|5752|Thursday|2010Q1|N|N|N|2455257|2455315|2454916|2455189|N|N|N|N|N| +2455282|AAAAAAAACPGHFCAA|2010-03-26|1322|5752|442|2010|5|3|26|1|2010|442|5752|Friday|2010Q1|N|Y|N|2455257|2455315|2454917|2455190|N|N|N|N|N| +2455283|AAAAAAAADPGHFCAA|2010-03-27|1322|5752|442|2010|6|3|27|1|2010|442|5752|Saturday|2010Q1|N|Y|N|2455257|2455315|2454918|2455191|N|N|N|N|N| +2455284|AAAAAAAAEPGHFCAA|2010-03-28|1322|5752|442|2010|0|3|28|1|2010|442|5752|Sunday|2010Q1|N|N|N|2455257|2455315|2454919|2455192|N|N|N|N|N| +2455285|AAAAAAAAFPGHFCAA|2010-03-29|1322|5752|442|2010|1|3|29|1|2010|442|5752|Monday|2010Q1|N|N|N|2455257|2455315|2454920|2455193|N|N|N|N|N| +2455286|AAAAAAAAGPGHFCAA|2010-03-30|1322|5753|442|2010|2|3|30|1|2010|442|5753|Tuesday|2010Q1|N|N|N|2455257|2455315|2454921|2455194|N|N|N|N|N| +2455287|AAAAAAAAHPGHFCAA|2010-03-31|1322|5753|442|2010|3|3|31|1|2010|442|5753|Wednesday|2010Q1|N|N|N|2455257|2455315|2454922|2455195|N|N|N|N|N| +2455288|AAAAAAAAIPGHFCAA|2010-04-01|1323|5753|442|2010|4|4|1|1|2010|442|5753|Thursday|2010Q1|N|N|N|2455288|2455377|2454923|2455198|N|N|N|N|N| +2455289|AAAAAAAAJPGHFCAA|2010-04-02|1323|5753|442|2010|5|4|2|2|2010|442|5753|Friday|2010Q2|N|Y|N|2455288|2455377|2454924|2455199|N|N|N|N|N| +2455290|AAAAAAAAKPGHFCAA|2010-04-03|1323|5753|442|2010|6|4|3|2|2010|442|5753|Saturday|2010Q2|N|Y|N|2455288|2455377|2454925|2455200|N|N|N|N|N| +2455291|AAAAAAAALPGHFCAA|2010-04-04|1323|5753|442|2010|0|4|4|2|2010|442|5753|Sunday|2010Q2|N|N|N|2455288|2455377|2454926|2455201|N|N|N|N|N| +2455292|AAAAAAAAMPGHFCAA|2010-04-05|1323|5753|442|2010|1|4|5|2|2010|442|5753|Monday|2010Q2|N|N|N|2455288|2455377|2454927|2455202|N|N|N|N|N| +2455293|AAAAAAAANPGHFCAA|2010-04-06|1323|5754|442|2010|2|4|6|2|2010|442|5754|Tuesday|2010Q2|N|N|N|2455288|2455377|2454928|2455203|N|N|N|N|N| +2455294|AAAAAAAAOPGHFCAA|2010-04-07|1323|5754|442|2010|3|4|7|2|2010|442|5754|Wednesday|2010Q2|N|N|N|2455288|2455377|2454929|2455204|N|N|N|N|N| +2455295|AAAAAAAAPPGHFCAA|2010-04-08|1323|5754|442|2010|4|4|8|2|2010|442|5754|Thursday|2010Q2|N|N|N|2455288|2455377|2454930|2455205|N|N|N|N|N| +2455296|AAAAAAAAAAHHFCAA|2010-04-09|1323|5754|442|2010|5|4|9|2|2010|442|5754|Friday|2010Q2|N|Y|N|2455288|2455377|2454931|2455206|N|N|N|N|N| +2455297|AAAAAAAABAHHFCAA|2010-04-10|1323|5754|442|2010|6|4|10|2|2010|442|5754|Saturday|2010Q2|N|Y|N|2455288|2455377|2454932|2455207|N|N|N|N|N| +2455298|AAAAAAAACAHHFCAA|2010-04-11|1323|5754|442|2010|0|4|11|2|2010|442|5754|Sunday|2010Q2|N|N|N|2455288|2455377|2454933|2455208|N|N|N|N|N| +2455299|AAAAAAAADAHHFCAA|2010-04-12|1323|5754|442|2010|1|4|12|2|2010|442|5754|Monday|2010Q2|N|N|N|2455288|2455377|2454934|2455209|N|N|N|N|N| +2455300|AAAAAAAAEAHHFCAA|2010-04-13|1323|5755|442|2010|2|4|13|2|2010|442|5755|Tuesday|2010Q2|N|N|N|2455288|2455377|2454935|2455210|N|N|N|N|N| +2455301|AAAAAAAAFAHHFCAA|2010-04-14|1323|5755|442|2010|3|4|14|2|2010|442|5755|Wednesday|2010Q2|N|N|N|2455288|2455377|2454936|2455211|N|N|N|N|N| +2455302|AAAAAAAAGAHHFCAA|2010-04-15|1323|5755|442|2010|4|4|15|2|2010|442|5755|Thursday|2010Q2|N|N|N|2455288|2455377|2454937|2455212|N|N|N|N|N| +2455303|AAAAAAAAHAHHFCAA|2010-04-16|1323|5755|442|2010|5|4|16|2|2010|442|5755|Friday|2010Q2|N|Y|N|2455288|2455377|2454938|2455213|N|N|N|N|N| +2455304|AAAAAAAAIAHHFCAA|2010-04-17|1323|5755|442|2010|6|4|17|2|2010|442|5755|Saturday|2010Q2|N|Y|N|2455288|2455377|2454939|2455214|N|N|N|N|N| +2455305|AAAAAAAAJAHHFCAA|2010-04-18|1323|5755|442|2010|0|4|18|2|2010|442|5755|Sunday|2010Q2|N|N|N|2455288|2455377|2454940|2455215|N|N|N|N|N| +2455306|AAAAAAAAKAHHFCAA|2010-04-19|1323|5755|442|2010|1|4|19|2|2010|442|5755|Monday|2010Q2|N|N|N|2455288|2455377|2454941|2455216|N|N|N|N|N| +2455307|AAAAAAAALAHHFCAA|2010-04-20|1323|5756|442|2010|2|4|20|2|2010|442|5756|Tuesday|2010Q2|N|N|N|2455288|2455377|2454942|2455217|N|N|N|N|N| +2455308|AAAAAAAAMAHHFCAA|2010-04-21|1323|5756|442|2010|3|4|21|2|2010|442|5756|Wednesday|2010Q2|N|N|N|2455288|2455377|2454943|2455218|N|N|N|N|N| +2455309|AAAAAAAANAHHFCAA|2010-04-22|1323|5756|442|2010|4|4|22|2|2010|442|5756|Thursday|2010Q2|N|N|N|2455288|2455377|2454944|2455219|N|N|N|N|N| +2455310|AAAAAAAAOAHHFCAA|2010-04-23|1323|5756|442|2010|5|4|23|2|2010|442|5756|Friday|2010Q2|N|Y|N|2455288|2455377|2454945|2455220|N|N|N|N|N| +2455311|AAAAAAAAPAHHFCAA|2010-04-24|1323|5756|442|2010|6|4|24|2|2010|442|5756|Saturday|2010Q2|N|Y|N|2455288|2455377|2454946|2455221|N|N|N|N|N| +2455312|AAAAAAAAABHHFCAA|2010-04-25|1323|5756|442|2010|0|4|25|2|2010|442|5756|Sunday|2010Q2|N|N|N|2455288|2455377|2454947|2455222|N|N|N|N|N| +2455313|AAAAAAAABBHHFCAA|2010-04-26|1323|5756|442|2010|1|4|26|2|2010|442|5756|Monday|2010Q2|N|N|N|2455288|2455377|2454948|2455223|N|N|N|N|N| +2455314|AAAAAAAACBHHFCAA|2010-04-27|1323|5757|442|2010|2|4|27|2|2010|442|5757|Tuesday|2010Q2|N|N|N|2455288|2455377|2454949|2455224|N|N|N|N|N| +2455315|AAAAAAAADBHHFCAA|2010-04-28|1323|5757|442|2010|3|4|28|2|2010|442|5757|Wednesday|2010Q2|N|N|N|2455288|2455377|2454950|2455225|N|N|N|N|N| +2455316|AAAAAAAAEBHHFCAA|2010-04-29|1323|5757|442|2010|4|4|29|2|2010|442|5757|Thursday|2010Q2|N|N|N|2455288|2455377|2454951|2455226|N|N|N|N|N| +2455317|AAAAAAAAFBHHFCAA|2010-04-30|1323|5757|442|2010|5|4|30|2|2010|442|5757|Friday|2010Q2|N|Y|N|2455288|2455377|2454952|2455227|N|N|N|N|N| +2455318|AAAAAAAAGBHHFCAA|2010-05-01|1324|5757|442|2010|6|5|1|2|2010|442|5757|Saturday|2010Q2|N|Y|N|2455318|2455437|2454953|2455228|N|N|N|N|N| +2455319|AAAAAAAAHBHHFCAA|2010-05-02|1324|5757|442|2010|0|5|2|2|2010|442|5757|Sunday|2010Q2|N|N|N|2455318|2455437|2454954|2455229|N|N|N|N|N| +2455320|AAAAAAAAIBHHFCAA|2010-05-03|1324|5757|442|2010|1|5|3|2|2010|442|5757|Monday|2010Q2|N|N|N|2455318|2455437|2454955|2455230|N|N|N|N|N| +2455321|AAAAAAAAJBHHFCAA|2010-05-04|1324|5758|442|2010|2|5|4|2|2010|442|5758|Tuesday|2010Q2|N|N|N|2455318|2455437|2454956|2455231|N|N|N|N|N| +2455322|AAAAAAAAKBHHFCAA|2010-05-05|1324|5758|442|2010|3|5|5|2|2010|442|5758|Wednesday|2010Q2|N|N|N|2455318|2455437|2454957|2455232|N|N|N|N|N| +2455323|AAAAAAAALBHHFCAA|2010-05-06|1324|5758|442|2010|4|5|6|2|2010|442|5758|Thursday|2010Q2|N|N|N|2455318|2455437|2454958|2455233|N|N|N|N|N| +2455324|AAAAAAAAMBHHFCAA|2010-05-07|1324|5758|442|2010|5|5|7|2|2010|442|5758|Friday|2010Q2|N|Y|N|2455318|2455437|2454959|2455234|N|N|N|N|N| +2455325|AAAAAAAANBHHFCAA|2010-05-08|1324|5758|442|2010|6|5|8|2|2010|442|5758|Saturday|2010Q2|N|Y|N|2455318|2455437|2454960|2455235|N|N|N|N|N| +2455326|AAAAAAAAOBHHFCAA|2010-05-09|1324|5758|442|2010|0|5|9|2|2010|442|5758|Sunday|2010Q2|N|N|N|2455318|2455437|2454961|2455236|N|N|N|N|N| +2455327|AAAAAAAAPBHHFCAA|2010-05-10|1324|5758|442|2010|1|5|10|2|2010|442|5758|Monday|2010Q2|N|N|N|2455318|2455437|2454962|2455237|N|N|N|N|N| +2455328|AAAAAAAAACHHFCAA|2010-05-11|1324|5759|442|2010|2|5|11|2|2010|442|5759|Tuesday|2010Q2|N|N|N|2455318|2455437|2454963|2455238|N|N|N|N|N| +2455329|AAAAAAAABCHHFCAA|2010-05-12|1324|5759|442|2010|3|5|12|2|2010|442|5759|Wednesday|2010Q2|N|N|N|2455318|2455437|2454964|2455239|N|N|N|N|N| +2455330|AAAAAAAACCHHFCAA|2010-05-13|1324|5759|442|2010|4|5|13|2|2010|442|5759|Thursday|2010Q2|N|N|N|2455318|2455437|2454965|2455240|N|N|N|N|N| +2455331|AAAAAAAADCHHFCAA|2010-05-14|1324|5759|442|2010|5|5|14|2|2010|442|5759|Friday|2010Q2|N|Y|N|2455318|2455437|2454966|2455241|N|N|N|N|N| +2455332|AAAAAAAAECHHFCAA|2010-05-15|1324|5759|442|2010|6|5|15|2|2010|442|5759|Saturday|2010Q2|N|Y|N|2455318|2455437|2454967|2455242|N|N|N|N|N| +2455333|AAAAAAAAFCHHFCAA|2010-05-16|1324|5759|442|2010|0|5|16|2|2010|442|5759|Sunday|2010Q2|N|N|N|2455318|2455437|2454968|2455243|N|N|N|N|N| +2455334|AAAAAAAAGCHHFCAA|2010-05-17|1324|5759|442|2010|1|5|17|2|2010|442|5759|Monday|2010Q2|N|N|N|2455318|2455437|2454969|2455244|N|N|N|N|N| +2455335|AAAAAAAAHCHHFCAA|2010-05-18|1324|5760|442|2010|2|5|18|2|2010|442|5760|Tuesday|2010Q2|N|N|N|2455318|2455437|2454970|2455245|N|N|N|N|N| +2455336|AAAAAAAAICHHFCAA|2010-05-19|1324|5760|442|2010|3|5|19|2|2010|442|5760|Wednesday|2010Q2|N|N|N|2455318|2455437|2454971|2455246|N|N|N|N|N| +2455337|AAAAAAAAJCHHFCAA|2010-05-20|1324|5760|442|2010|4|5|20|2|2010|442|5760|Thursday|2010Q2|N|N|N|2455318|2455437|2454972|2455247|N|N|N|N|N| +2455338|AAAAAAAAKCHHFCAA|2010-05-21|1324|5760|442|2010|5|5|21|2|2010|442|5760|Friday|2010Q2|N|Y|N|2455318|2455437|2454973|2455248|N|N|N|N|N| +2455339|AAAAAAAALCHHFCAA|2010-05-22|1324|5760|442|2010|6|5|22|2|2010|442|5760|Saturday|2010Q2|N|Y|N|2455318|2455437|2454974|2455249|N|N|N|N|N| +2455340|AAAAAAAAMCHHFCAA|2010-05-23|1324|5760|442|2010|0|5|23|2|2010|442|5760|Sunday|2010Q2|N|N|N|2455318|2455437|2454975|2455250|N|N|N|N|N| +2455341|AAAAAAAANCHHFCAA|2010-05-24|1324|5760|442|2010|1|5|24|2|2010|442|5760|Monday|2010Q2|N|N|N|2455318|2455437|2454976|2455251|N|N|N|N|N| +2455342|AAAAAAAAOCHHFCAA|2010-05-25|1324|5761|442|2010|2|5|25|2|2010|442|5761|Tuesday|2010Q2|N|N|N|2455318|2455437|2454977|2455252|N|N|N|N|N| +2455343|AAAAAAAAPCHHFCAA|2010-05-26|1324|5761|442|2010|3|5|26|2|2010|442|5761|Wednesday|2010Q2|N|N|N|2455318|2455437|2454978|2455253|N|N|N|N|N| +2455344|AAAAAAAAADHHFCAA|2010-05-27|1324|5761|442|2010|4|5|27|2|2010|442|5761|Thursday|2010Q2|N|N|N|2455318|2455437|2454979|2455254|N|N|N|N|N| +2455345|AAAAAAAABDHHFCAA|2010-05-28|1324|5761|442|2010|5|5|28|2|2010|442|5761|Friday|2010Q2|N|Y|N|2455318|2455437|2454980|2455255|N|N|N|N|N| +2455346|AAAAAAAACDHHFCAA|2010-05-29|1324|5761|442|2010|6|5|29|2|2010|442|5761|Saturday|2010Q2|N|Y|N|2455318|2455437|2454981|2455256|N|N|N|N|N| +2455347|AAAAAAAADDHHFCAA|2010-05-30|1324|5761|442|2010|0|5|30|2|2010|442|5761|Sunday|2010Q2|N|N|N|2455318|2455437|2454982|2455257|N|N|N|N|N| +2455348|AAAAAAAAEDHHFCAA|2010-05-31|1324|5761|442|2010|1|5|31|2|2010|442|5761|Monday|2010Q2|N|N|N|2455318|2455437|2454983|2455258|N|N|N|N|N| +2455349|AAAAAAAAFDHHFCAA|2010-06-01|1325|5762|443|2010|2|6|1|2|2010|443|5762|Tuesday|2010Q2|N|N|N|2455349|2455499|2454984|2455259|N|N|N|N|N| +2455350|AAAAAAAAGDHHFCAA|2010-06-02|1325|5762|443|2010|3|6|2|2|2010|443|5762|Wednesday|2010Q2|N|N|N|2455349|2455499|2454985|2455260|N|N|N|N|N| +2455351|AAAAAAAAHDHHFCAA|2010-06-03|1325|5762|443|2010|4|6|3|2|2010|443|5762|Thursday|2010Q2|N|N|N|2455349|2455499|2454986|2455261|N|N|N|N|N| +2455352|AAAAAAAAIDHHFCAA|2010-06-04|1325|5762|443|2010|5|6|4|2|2010|443|5762|Friday|2010Q2|N|Y|N|2455349|2455499|2454987|2455262|N|N|N|N|N| +2455353|AAAAAAAAJDHHFCAA|2010-06-05|1325|5762|443|2010|6|6|5|2|2010|443|5762|Saturday|2010Q2|N|Y|N|2455349|2455499|2454988|2455263|N|N|N|N|N| +2455354|AAAAAAAAKDHHFCAA|2010-06-06|1325|5762|443|2010|0|6|6|2|2010|443|5762|Sunday|2010Q2|N|N|N|2455349|2455499|2454989|2455264|N|N|N|N|N| +2455355|AAAAAAAALDHHFCAA|2010-06-07|1325|5762|443|2010|1|6|7|2|2010|443|5762|Monday|2010Q2|N|N|N|2455349|2455499|2454990|2455265|N|N|N|N|N| +2455356|AAAAAAAAMDHHFCAA|2010-06-08|1325|5763|443|2010|2|6|8|2|2010|443|5763|Tuesday|2010Q2|N|N|N|2455349|2455499|2454991|2455266|N|N|N|N|N| +2455357|AAAAAAAANDHHFCAA|2010-06-09|1325|5763|443|2010|3|6|9|2|2010|443|5763|Wednesday|2010Q2|N|N|N|2455349|2455499|2454992|2455267|N|N|N|N|N| +2455358|AAAAAAAAODHHFCAA|2010-06-10|1325|5763|443|2010|4|6|10|2|2010|443|5763|Thursday|2010Q2|N|N|N|2455349|2455499|2454993|2455268|N|N|N|N|N| +2455359|AAAAAAAAPDHHFCAA|2010-06-11|1325|5763|443|2010|5|6|11|2|2010|443|5763|Friday|2010Q2|N|Y|N|2455349|2455499|2454994|2455269|N|N|N|N|N| +2455360|AAAAAAAAAEHHFCAA|2010-06-12|1325|5763|443|2010|6|6|12|2|2010|443|5763|Saturday|2010Q2|N|Y|N|2455349|2455499|2454995|2455270|N|N|N|N|N| +2455361|AAAAAAAABEHHFCAA|2010-06-13|1325|5763|443|2010|0|6|13|2|2010|443|5763|Sunday|2010Q2|N|N|N|2455349|2455499|2454996|2455271|N|N|N|N|N| +2455362|AAAAAAAACEHHFCAA|2010-06-14|1325|5763|443|2010|1|6|14|2|2010|443|5763|Monday|2010Q2|N|N|N|2455349|2455499|2454997|2455272|N|N|N|N|N| +2455363|AAAAAAAADEHHFCAA|2010-06-15|1325|5764|443|2010|2|6|15|2|2010|443|5764|Tuesday|2010Q2|N|N|N|2455349|2455499|2454998|2455273|N|N|N|N|N| +2455364|AAAAAAAAEEHHFCAA|2010-06-16|1325|5764|443|2010|3|6|16|2|2010|443|5764|Wednesday|2010Q2|N|N|N|2455349|2455499|2454999|2455274|N|N|N|N|N| +2455365|AAAAAAAAFEHHFCAA|2010-06-17|1325|5764|443|2010|4|6|17|2|2010|443|5764|Thursday|2010Q2|N|N|N|2455349|2455499|2455000|2455275|N|N|N|N|N| +2455366|AAAAAAAAGEHHFCAA|2010-06-18|1325|5764|443|2010|5|6|18|2|2010|443|5764|Friday|2010Q2|N|Y|N|2455349|2455499|2455001|2455276|N|N|N|N|N| +2455367|AAAAAAAAHEHHFCAA|2010-06-19|1325|5764|443|2010|6|6|19|2|2010|443|5764|Saturday|2010Q2|N|Y|N|2455349|2455499|2455002|2455277|N|N|N|N|N| +2455368|AAAAAAAAIEHHFCAA|2010-06-20|1325|5764|443|2010|0|6|20|2|2010|443|5764|Sunday|2010Q2|N|N|N|2455349|2455499|2455003|2455278|N|N|N|N|N| +2455369|AAAAAAAAJEHHFCAA|2010-06-21|1325|5764|443|2010|1|6|21|2|2010|443|5764|Monday|2010Q2|N|N|N|2455349|2455499|2455004|2455279|N|N|N|N|N| +2455370|AAAAAAAAKEHHFCAA|2010-06-22|1325|5765|443|2010|2|6|22|2|2010|443|5765|Tuesday|2010Q2|N|N|N|2455349|2455499|2455005|2455280|N|N|N|N|N| +2455371|AAAAAAAALEHHFCAA|2010-06-23|1325|5765|443|2010|3|6|23|2|2010|443|5765|Wednesday|2010Q2|N|N|N|2455349|2455499|2455006|2455281|N|N|N|N|N| +2455372|AAAAAAAAMEHHFCAA|2010-06-24|1325|5765|443|2010|4|6|24|2|2010|443|5765|Thursday|2010Q2|N|N|N|2455349|2455499|2455007|2455282|N|N|N|N|N| +2455373|AAAAAAAANEHHFCAA|2010-06-25|1325|5765|443|2010|5|6|25|2|2010|443|5765|Friday|2010Q2|N|Y|N|2455349|2455499|2455008|2455283|N|N|N|N|N| +2455374|AAAAAAAAOEHHFCAA|2010-06-26|1325|5765|443|2010|6|6|26|2|2010|443|5765|Saturday|2010Q2|N|Y|N|2455349|2455499|2455009|2455284|N|N|N|N|N| +2455375|AAAAAAAAPEHHFCAA|2010-06-27|1325|5765|443|2010|0|6|27|2|2010|443|5765|Sunday|2010Q2|N|N|N|2455349|2455499|2455010|2455285|N|N|N|N|N| +2455376|AAAAAAAAAFHHFCAA|2010-06-28|1325|5765|443|2010|1|6|28|2|2010|443|5765|Monday|2010Q2|N|N|N|2455349|2455499|2455011|2455286|N|N|N|N|N| +2455377|AAAAAAAABFHHFCAA|2010-06-29|1325|5766|443|2010|2|6|29|2|2010|443|5766|Tuesday|2010Q2|N|N|N|2455349|2455499|2455012|2455287|N|N|N|N|N| +2455378|AAAAAAAACFHHFCAA|2010-06-30|1325|5766|443|2010|3|6|30|2|2010|443|5766|Wednesday|2010Q2|N|N|N|2455349|2455499|2455013|2455288|N|N|N|N|N| +2455379|AAAAAAAADFHHFCAA|2010-07-01|1326|5766|443|2010|4|7|1|2|2010|443|5766|Thursday|2010Q2|N|N|N|2455379|2455559|2455014|2455288|N|N|N|N|N| +2455380|AAAAAAAAEFHHFCAA|2010-07-02|1326|5766|443|2010|5|7|2|3|2010|443|5766|Friday|2010Q3|N|Y|N|2455379|2455559|2455015|2455289|N|N|N|N|N| +2455381|AAAAAAAAFFHHFCAA|2010-07-03|1326|5766|443|2010|6|7|3|3|2010|443|5766|Saturday|2010Q3|N|Y|N|2455379|2455559|2455016|2455290|N|N|N|N|N| +2455382|AAAAAAAAGFHHFCAA|2010-07-04|1326|5766|443|2010|0|7|4|3|2010|443|5766|Sunday|2010Q3|N|N|N|2455379|2455559|2455017|2455291|N|N|N|N|N| +2455383|AAAAAAAAHFHHFCAA|2010-07-05|1326|5766|443|2010|1|7|5|3|2010|443|5766|Monday|2010Q3|Y|N|N|2455379|2455559|2455018|2455292|N|N|N|N|N| +2455384|AAAAAAAAIFHHFCAA|2010-07-06|1326|5767|443|2010|2|7|6|3|2010|443|5767|Tuesday|2010Q3|N|N|Y|2455379|2455559|2455019|2455293|N|N|N|N|N| +2455385|AAAAAAAAJFHHFCAA|2010-07-07|1326|5767|443|2010|3|7|7|3|2010|443|5767|Wednesday|2010Q3|N|N|N|2455379|2455559|2455020|2455294|N|N|N|N|N| +2455386|AAAAAAAAKFHHFCAA|2010-07-08|1326|5767|443|2010|4|7|8|3|2010|443|5767|Thursday|2010Q3|N|N|N|2455379|2455559|2455021|2455295|N|N|N|N|N| +2455387|AAAAAAAALFHHFCAA|2010-07-09|1326|5767|443|2010|5|7|9|3|2010|443|5767|Friday|2010Q3|N|Y|N|2455379|2455559|2455022|2455296|N|N|N|N|N| +2455388|AAAAAAAAMFHHFCAA|2010-07-10|1326|5767|443|2010|6|7|10|3|2010|443|5767|Saturday|2010Q3|N|Y|N|2455379|2455559|2455023|2455297|N|N|N|N|N| +2455389|AAAAAAAANFHHFCAA|2010-07-11|1326|5767|443|2010|0|7|11|3|2010|443|5767|Sunday|2010Q3|N|N|N|2455379|2455559|2455024|2455298|N|N|N|N|N| +2455390|AAAAAAAAOFHHFCAA|2010-07-12|1326|5767|443|2010|1|7|12|3|2010|443|5767|Monday|2010Q3|N|N|N|2455379|2455559|2455025|2455299|N|N|N|N|N| +2455391|AAAAAAAAPFHHFCAA|2010-07-13|1326|5768|443|2010|2|7|13|3|2010|443|5768|Tuesday|2010Q3|N|N|N|2455379|2455559|2455026|2455300|N|N|N|N|N| +2455392|AAAAAAAAAGHHFCAA|2010-07-14|1326|5768|443|2010|3|7|14|3|2010|443|5768|Wednesday|2010Q3|N|N|N|2455379|2455559|2455027|2455301|N|N|N|N|N| +2455393|AAAAAAAABGHHFCAA|2010-07-15|1326|5768|443|2010|4|7|15|3|2010|443|5768|Thursday|2010Q3|N|N|N|2455379|2455559|2455028|2455302|N|N|N|N|N| +2455394|AAAAAAAACGHHFCAA|2010-07-16|1326|5768|443|2010|5|7|16|3|2010|443|5768|Friday|2010Q3|N|Y|N|2455379|2455559|2455029|2455303|N|N|N|N|N| +2455395|AAAAAAAADGHHFCAA|2010-07-17|1326|5768|443|2010|6|7|17|3|2010|443|5768|Saturday|2010Q3|N|Y|N|2455379|2455559|2455030|2455304|N|N|N|N|N| +2455396|AAAAAAAAEGHHFCAA|2010-07-18|1326|5768|443|2010|0|7|18|3|2010|443|5768|Sunday|2010Q3|N|N|N|2455379|2455559|2455031|2455305|N|N|N|N|N| +2455397|AAAAAAAAFGHHFCAA|2010-07-19|1326|5768|443|2010|1|7|19|3|2010|443|5768|Monday|2010Q3|N|N|N|2455379|2455559|2455032|2455306|N|N|N|N|N| +2455398|AAAAAAAAGGHHFCAA|2010-07-20|1326|5769|443|2010|2|7|20|3|2010|443|5769|Tuesday|2010Q3|N|N|N|2455379|2455559|2455033|2455307|N|N|N|N|N| +2455399|AAAAAAAAHGHHFCAA|2010-07-21|1326|5769|443|2010|3|7|21|3|2010|443|5769|Wednesday|2010Q3|N|N|N|2455379|2455559|2455034|2455308|N|N|N|N|N| +2455400|AAAAAAAAIGHHFCAA|2010-07-22|1326|5769|443|2010|4|7|22|3|2010|443|5769|Thursday|2010Q3|N|N|N|2455379|2455559|2455035|2455309|N|N|N|N|N| +2455401|AAAAAAAAJGHHFCAA|2010-07-23|1326|5769|443|2010|5|7|23|3|2010|443|5769|Friday|2010Q3|N|Y|N|2455379|2455559|2455036|2455310|N|N|N|N|N| +2455402|AAAAAAAAKGHHFCAA|2010-07-24|1326|5769|443|2010|6|7|24|3|2010|443|5769|Saturday|2010Q3|N|Y|N|2455379|2455559|2455037|2455311|N|N|N|N|N| +2455403|AAAAAAAALGHHFCAA|2010-07-25|1326|5769|443|2010|0|7|25|3|2010|443|5769|Sunday|2010Q3|N|N|N|2455379|2455559|2455038|2455312|N|N|N|N|N| +2455404|AAAAAAAAMGHHFCAA|2010-07-26|1326|5769|443|2010|1|7|26|3|2010|443|5769|Monday|2010Q3|N|N|N|2455379|2455559|2455039|2455313|N|N|N|N|N| +2455405|AAAAAAAANGHHFCAA|2010-07-27|1326|5770|443|2010|2|7|27|3|2010|443|5770|Tuesday|2010Q3|N|N|N|2455379|2455559|2455040|2455314|N|N|N|N|N| +2455406|AAAAAAAAOGHHFCAA|2010-07-28|1326|5770|443|2010|3|7|28|3|2010|443|5770|Wednesday|2010Q3|N|N|N|2455379|2455559|2455041|2455315|N|N|N|N|N| +2455407|AAAAAAAAPGHHFCAA|2010-07-29|1326|5770|443|2010|4|7|29|3|2010|443|5770|Thursday|2010Q3|N|N|N|2455379|2455559|2455042|2455316|N|N|N|N|N| +2455408|AAAAAAAAAHHHFCAA|2010-07-30|1326|5770|443|2010|5|7|30|3|2010|443|5770|Friday|2010Q3|N|Y|N|2455379|2455559|2455043|2455317|N|N|N|N|N| +2455409|AAAAAAAABHHHFCAA|2010-07-31|1326|5770|443|2010|6|7|31|3|2010|443|5770|Saturday|2010Q3|N|Y|N|2455379|2455559|2455044|2455318|N|N|N|N|N| +2455410|AAAAAAAACHHHFCAA|2010-08-01|1327|5770|443|2010|0|8|1|3|2010|443|5770|Sunday|2010Q3|N|N|N|2455410|2455621|2455045|2455319|N|N|N|N|N| +2455411|AAAAAAAADHHHFCAA|2010-08-02|1327|5770|443|2010|1|8|2|3|2010|443|5770|Monday|2010Q3|N|N|N|2455410|2455621|2455046|2455320|N|N|N|N|N| +2455412|AAAAAAAAEHHHFCAA|2010-08-03|1327|5771|443|2010|2|8|3|3|2010|443|5771|Tuesday|2010Q3|N|N|N|2455410|2455621|2455047|2455321|N|N|N|N|N| +2455413|AAAAAAAAFHHHFCAA|2010-08-04|1327|5771|443|2010|3|8|4|3|2010|443|5771|Wednesday|2010Q3|N|N|N|2455410|2455621|2455048|2455322|N|N|N|N|N| +2455414|AAAAAAAAGHHHFCAA|2010-08-05|1327|5771|443|2010|4|8|5|3|2010|443|5771|Thursday|2010Q3|N|N|N|2455410|2455621|2455049|2455323|N|N|N|N|N| +2455415|AAAAAAAAHHHHFCAA|2010-08-06|1327|5771|443|2010|5|8|6|3|2010|443|5771|Friday|2010Q3|N|Y|N|2455410|2455621|2455050|2455324|N|N|N|N|N| +2455416|AAAAAAAAIHHHFCAA|2010-08-07|1327|5771|443|2010|6|8|7|3|2010|443|5771|Saturday|2010Q3|N|Y|N|2455410|2455621|2455051|2455325|N|N|N|N|N| +2455417|AAAAAAAAJHHHFCAA|2010-08-08|1327|5771|443|2010|0|8|8|3|2010|443|5771|Sunday|2010Q3|N|N|N|2455410|2455621|2455052|2455326|N|N|N|N|N| +2455418|AAAAAAAAKHHHFCAA|2010-08-09|1327|5771|443|2010|1|8|9|3|2010|443|5771|Monday|2010Q3|N|N|N|2455410|2455621|2455053|2455327|N|N|N|N|N| +2455419|AAAAAAAALHHHFCAA|2010-08-10|1327|5772|443|2010|2|8|10|3|2010|443|5772|Tuesday|2010Q3|N|N|N|2455410|2455621|2455054|2455328|N|N|N|N|N| +2455420|AAAAAAAAMHHHFCAA|2010-08-11|1327|5772|443|2010|3|8|11|3|2010|443|5772|Wednesday|2010Q3|N|N|N|2455410|2455621|2455055|2455329|N|N|N|N|N| +2455421|AAAAAAAANHHHFCAA|2010-08-12|1327|5772|443|2010|4|8|12|3|2010|443|5772|Thursday|2010Q3|N|N|N|2455410|2455621|2455056|2455330|N|N|N|N|N| +2455422|AAAAAAAAOHHHFCAA|2010-08-13|1327|5772|443|2010|5|8|13|3|2010|443|5772|Friday|2010Q3|N|Y|N|2455410|2455621|2455057|2455331|N|N|N|N|N| +2455423|AAAAAAAAPHHHFCAA|2010-08-14|1327|5772|443|2010|6|8|14|3|2010|443|5772|Saturday|2010Q3|N|Y|N|2455410|2455621|2455058|2455332|N|N|N|N|N| +2455424|AAAAAAAAAIHHFCAA|2010-08-15|1327|5772|443|2010|0|8|15|3|2010|443|5772|Sunday|2010Q3|N|N|N|2455410|2455621|2455059|2455333|N|N|N|N|N| +2455425|AAAAAAAABIHHFCAA|2010-08-16|1327|5772|443|2010|1|8|16|3|2010|443|5772|Monday|2010Q3|N|N|N|2455410|2455621|2455060|2455334|N|N|N|N|N| +2455426|AAAAAAAACIHHFCAA|2010-08-17|1327|5773|443|2010|2|8|17|3|2010|443|5773|Tuesday|2010Q3|N|N|N|2455410|2455621|2455061|2455335|N|N|N|N|N| +2455427|AAAAAAAADIHHFCAA|2010-08-18|1327|5773|443|2010|3|8|18|3|2010|443|5773|Wednesday|2010Q3|N|N|N|2455410|2455621|2455062|2455336|N|N|N|N|N| +2455428|AAAAAAAAEIHHFCAA|2010-08-19|1327|5773|443|2010|4|8|19|3|2010|443|5773|Thursday|2010Q3|N|N|N|2455410|2455621|2455063|2455337|N|N|N|N|N| +2455429|AAAAAAAAFIHHFCAA|2010-08-20|1327|5773|443|2010|5|8|20|3|2010|443|5773|Friday|2010Q3|N|Y|N|2455410|2455621|2455064|2455338|N|N|N|N|N| +2455430|AAAAAAAAGIHHFCAA|2010-08-21|1327|5773|443|2010|6|8|21|3|2010|443|5773|Saturday|2010Q3|N|Y|N|2455410|2455621|2455065|2455339|N|N|N|N|N| +2455431|AAAAAAAAHIHHFCAA|2010-08-22|1327|5773|443|2010|0|8|22|3|2010|443|5773|Sunday|2010Q3|N|N|N|2455410|2455621|2455066|2455340|N|N|N|N|N| +2455432|AAAAAAAAIIHHFCAA|2010-08-23|1327|5773|443|2010|1|8|23|3|2010|443|5773|Monday|2010Q3|N|N|N|2455410|2455621|2455067|2455341|N|N|N|N|N| +2455433|AAAAAAAAJIHHFCAA|2010-08-24|1327|5774|443|2010|2|8|24|3|2010|443|5774|Tuesday|2010Q3|N|N|N|2455410|2455621|2455068|2455342|N|N|N|N|N| +2455434|AAAAAAAAKIHHFCAA|2010-08-25|1327|5774|443|2010|3|8|25|3|2010|443|5774|Wednesday|2010Q3|N|N|N|2455410|2455621|2455069|2455343|N|N|N|N|N| +2455435|AAAAAAAALIHHFCAA|2010-08-26|1327|5774|443|2010|4|8|26|3|2010|443|5774|Thursday|2010Q3|N|N|N|2455410|2455621|2455070|2455344|N|N|N|N|N| +2455436|AAAAAAAAMIHHFCAA|2010-08-27|1327|5774|443|2010|5|8|27|3|2010|443|5774|Friday|2010Q3|N|Y|N|2455410|2455621|2455071|2455345|N|N|N|N|N| +2455437|AAAAAAAANIHHFCAA|2010-08-28|1327|5774|443|2010|6|8|28|3|2010|443|5774|Saturday|2010Q3|N|Y|N|2455410|2455621|2455072|2455346|N|N|N|N|N| +2455438|AAAAAAAAOIHHFCAA|2010-08-29|1327|5774|443|2010|0|8|29|3|2010|443|5774|Sunday|2010Q3|N|N|N|2455410|2455621|2455073|2455347|N|N|N|N|N| +2455439|AAAAAAAAPIHHFCAA|2010-08-30|1327|5774|443|2010|1|8|30|3|2010|443|5774|Monday|2010Q3|N|N|N|2455410|2455621|2455074|2455348|N|N|N|N|N| +2455440|AAAAAAAAAJHHFCAA|2010-08-31|1327|5775|443|2010|2|8|31|3|2010|443|5775|Tuesday|2010Q3|N|N|N|2455410|2455621|2455075|2455349|N|N|N|N|N| +2455441|AAAAAAAABJHHFCAA|2010-09-01|1328|5775|444|2010|3|9|1|3|2010|444|5775|Wednesday|2010Q3|N|N|N|2455441|2455683|2455076|2455350|N|N|N|N|N| +2455442|AAAAAAAACJHHFCAA|2010-09-02|1328|5775|444|2010|4|9|2|3|2010|444|5775|Thursday|2010Q3|N|N|N|2455441|2455683|2455077|2455351|N|N|N|N|N| +2455443|AAAAAAAADJHHFCAA|2010-09-03|1328|5775|444|2010|5|9|3|3|2010|444|5775|Friday|2010Q3|N|Y|N|2455441|2455683|2455078|2455352|N|N|N|N|N| +2455444|AAAAAAAAEJHHFCAA|2010-09-04|1328|5775|444|2010|6|9|4|3|2010|444|5775|Saturday|2010Q3|N|Y|N|2455441|2455683|2455079|2455353|N|N|N|N|N| +2455445|AAAAAAAAFJHHFCAA|2010-09-05|1328|5775|444|2010|0|9|5|3|2010|444|5775|Sunday|2010Q3|N|N|N|2455441|2455683|2455080|2455354|N|N|N|N|N| +2455446|AAAAAAAAGJHHFCAA|2010-09-06|1328|5775|444|2010|1|9|6|3|2010|444|5775|Monday|2010Q3|N|N|N|2455441|2455683|2455081|2455355|N|N|N|N|N| +2455447|AAAAAAAAHJHHFCAA|2010-09-07|1328|5776|444|2010|2|9|7|3|2010|444|5776|Tuesday|2010Q3|N|N|N|2455441|2455683|2455082|2455356|N|N|N|N|N| +2455448|AAAAAAAAIJHHFCAA|2010-09-08|1328|5776|444|2010|3|9|8|3|2010|444|5776|Wednesday|2010Q3|N|N|N|2455441|2455683|2455083|2455357|N|N|N|N|N| +2455449|AAAAAAAAJJHHFCAA|2010-09-09|1328|5776|444|2010|4|9|9|3|2010|444|5776|Thursday|2010Q3|N|N|N|2455441|2455683|2455084|2455358|N|N|N|N|N| +2455450|AAAAAAAAKJHHFCAA|2010-09-10|1328|5776|444|2010|5|9|10|3|2010|444|5776|Friday|2010Q3|N|Y|N|2455441|2455683|2455085|2455359|N|N|N|N|N| +2455451|AAAAAAAALJHHFCAA|2010-09-11|1328|5776|444|2010|6|9|11|3|2010|444|5776|Saturday|2010Q3|N|Y|N|2455441|2455683|2455086|2455360|N|N|N|N|N| +2455452|AAAAAAAAMJHHFCAA|2010-09-12|1328|5776|444|2010|0|9|12|3|2010|444|5776|Sunday|2010Q3|N|N|N|2455441|2455683|2455087|2455361|N|N|N|N|N| +2455453|AAAAAAAANJHHFCAA|2010-09-13|1328|5776|444|2010|1|9|13|3|2010|444|5776|Monday|2010Q3|N|N|N|2455441|2455683|2455088|2455362|N|N|N|N|N| +2455454|AAAAAAAAOJHHFCAA|2010-09-14|1328|5777|444|2010|2|9|14|3|2010|444|5777|Tuesday|2010Q3|N|N|N|2455441|2455683|2455089|2455363|N|N|N|N|N| +2455455|AAAAAAAAPJHHFCAA|2010-09-15|1328|5777|444|2010|3|9|15|3|2010|444|5777|Wednesday|2010Q3|N|N|N|2455441|2455683|2455090|2455364|N|N|N|N|N| +2455456|AAAAAAAAAKHHFCAA|2010-09-16|1328|5777|444|2010|4|9|16|3|2010|444|5777|Thursday|2010Q3|N|N|N|2455441|2455683|2455091|2455365|N|N|N|N|N| +2455457|AAAAAAAABKHHFCAA|2010-09-17|1328|5777|444|2010|5|9|17|3|2010|444|5777|Friday|2010Q3|N|Y|N|2455441|2455683|2455092|2455366|N|N|N|N|N| +2455458|AAAAAAAACKHHFCAA|2010-09-18|1328|5777|444|2010|6|9|18|3|2010|444|5777|Saturday|2010Q3|N|Y|N|2455441|2455683|2455093|2455367|N|N|N|N|N| +2455459|AAAAAAAADKHHFCAA|2010-09-19|1328|5777|444|2010|0|9|19|3|2010|444|5777|Sunday|2010Q3|N|N|N|2455441|2455683|2455094|2455368|N|N|N|N|N| +2455460|AAAAAAAAEKHHFCAA|2010-09-20|1328|5777|444|2010|1|9|20|3|2010|444|5777|Monday|2010Q3|N|N|N|2455441|2455683|2455095|2455369|N|N|N|N|N| +2455461|AAAAAAAAFKHHFCAA|2010-09-21|1328|5778|444|2010|2|9|21|3|2010|444|5778|Tuesday|2010Q3|N|N|N|2455441|2455683|2455096|2455370|N|N|N|N|N| +2455462|AAAAAAAAGKHHFCAA|2010-09-22|1328|5778|444|2010|3|9|22|3|2010|444|5778|Wednesday|2010Q3|N|N|N|2455441|2455683|2455097|2455371|N|N|N|N|N| +2455463|AAAAAAAAHKHHFCAA|2010-09-23|1328|5778|444|2010|4|9|23|3|2010|444|5778|Thursday|2010Q3|N|N|N|2455441|2455683|2455098|2455372|N|N|N|N|N| +2455464|AAAAAAAAIKHHFCAA|2010-09-24|1328|5778|444|2010|5|9|24|3|2010|444|5778|Friday|2010Q3|N|Y|N|2455441|2455683|2455099|2455373|N|N|N|N|N| +2455465|AAAAAAAAJKHHFCAA|2010-09-25|1328|5778|444|2010|6|9|25|3|2010|444|5778|Saturday|2010Q3|N|Y|N|2455441|2455683|2455100|2455374|N|N|N|N|N| +2455466|AAAAAAAAKKHHFCAA|2010-09-26|1328|5778|444|2010|0|9|26|3|2010|444|5778|Sunday|2010Q3|N|N|N|2455441|2455683|2455101|2455375|N|N|N|N|N| +2455467|AAAAAAAALKHHFCAA|2010-09-27|1328|5778|444|2010|1|9|27|3|2010|444|5778|Monday|2010Q3|N|N|N|2455441|2455683|2455102|2455376|N|N|N|N|N| +2455468|AAAAAAAAMKHHFCAA|2010-09-28|1328|5779|444|2010|2|9|28|3|2010|444|5779|Tuesday|2010Q3|N|N|N|2455441|2455683|2455103|2455377|N|N|N|N|N| +2455469|AAAAAAAANKHHFCAA|2010-09-29|1328|5779|444|2010|3|9|29|3|2010|444|5779|Wednesday|2010Q3|N|N|N|2455441|2455683|2455104|2455378|N|N|N|N|N| +2455470|AAAAAAAAOKHHFCAA|2010-09-30|1328|5779|444|2010|4|9|30|3|2010|444|5779|Thursday|2010Q3|N|N|N|2455441|2455683|2455105|2455379|N|N|N|N|N| +2455471|AAAAAAAAPKHHFCAA|2010-10-01|1329|5779|444|2010|5|10|1|3|2010|444|5779|Friday|2010Q3|N|Y|N|2455471|2455743|2455106|2455379|N|N|N|N|N| +2455472|AAAAAAAAALHHFCAA|2010-10-02|1329|5779|444|2010|6|10|2|4|2010|444|5779|Saturday|2010Q4|N|Y|N|2455471|2455743|2455107|2455380|N|N|N|N|N| +2455473|AAAAAAAABLHHFCAA|2010-10-03|1329|5779|444|2010|0|10|3|4|2010|444|5779|Sunday|2010Q4|N|N|N|2455471|2455743|2455108|2455381|N|N|N|N|N| +2455474|AAAAAAAACLHHFCAA|2010-10-04|1329|5779|444|2010|1|10|4|4|2010|444|5779|Monday|2010Q4|N|N|N|2455471|2455743|2455109|2455382|N|N|N|N|N| +2455475|AAAAAAAADLHHFCAA|2010-10-05|1329|5780|444|2010|2|10|5|4|2010|444|5780|Tuesday|2010Q4|N|N|N|2455471|2455743|2455110|2455383|N|N|N|N|N| +2455476|AAAAAAAAELHHFCAA|2010-10-06|1329|5780|444|2010|3|10|6|4|2010|444|5780|Wednesday|2010Q4|N|N|N|2455471|2455743|2455111|2455384|N|N|N|N|N| +2455477|AAAAAAAAFLHHFCAA|2010-10-07|1329|5780|444|2010|4|10|7|4|2010|444|5780|Thursday|2010Q4|N|N|N|2455471|2455743|2455112|2455385|N|N|N|N|N| +2455478|AAAAAAAAGLHHFCAA|2010-10-08|1329|5780|444|2010|5|10|8|4|2010|444|5780|Friday|2010Q4|N|Y|N|2455471|2455743|2455113|2455386|N|N|N|N|N| +2455479|AAAAAAAAHLHHFCAA|2010-10-09|1329|5780|444|2010|6|10|9|4|2010|444|5780|Saturday|2010Q4|N|Y|N|2455471|2455743|2455114|2455387|N|N|N|N|N| +2455480|AAAAAAAAILHHFCAA|2010-10-10|1329|5780|444|2010|0|10|10|4|2010|444|5780|Sunday|2010Q4|N|N|N|2455471|2455743|2455115|2455388|N|N|N|N|N| +2455481|AAAAAAAAJLHHFCAA|2010-10-11|1329|5780|444|2010|1|10|11|4|2010|444|5780|Monday|2010Q4|N|N|N|2455471|2455743|2455116|2455389|N|N|N|N|N| +2455482|AAAAAAAAKLHHFCAA|2010-10-12|1329|5781|444|2010|2|10|12|4|2010|444|5781|Tuesday|2010Q4|N|N|N|2455471|2455743|2455117|2455390|N|N|N|N|N| +2455483|AAAAAAAALLHHFCAA|2010-10-13|1329|5781|444|2010|3|10|13|4|2010|444|5781|Wednesday|2010Q4|N|N|N|2455471|2455743|2455118|2455391|N|N|N|N|N| +2455484|AAAAAAAAMLHHFCAA|2010-10-14|1329|5781|444|2010|4|10|14|4|2010|444|5781|Thursday|2010Q4|N|N|N|2455471|2455743|2455119|2455392|N|N|N|N|N| +2455485|AAAAAAAANLHHFCAA|2010-10-15|1329|5781|444|2010|5|10|15|4|2010|444|5781|Friday|2010Q4|N|Y|N|2455471|2455743|2455120|2455393|N|N|N|N|N| +2455486|AAAAAAAAOLHHFCAA|2010-10-16|1329|5781|444|2010|6|10|16|4|2010|444|5781|Saturday|2010Q4|N|Y|N|2455471|2455743|2455121|2455394|N|N|N|N|N| +2455487|AAAAAAAAPLHHFCAA|2010-10-17|1329|5781|444|2010|0|10|17|4|2010|444|5781|Sunday|2010Q4|N|N|N|2455471|2455743|2455122|2455395|N|N|N|N|N| +2455488|AAAAAAAAAMHHFCAA|2010-10-18|1329|5781|444|2010|1|10|18|4|2010|444|5781|Monday|2010Q4|N|N|N|2455471|2455743|2455123|2455396|N|N|N|N|N| +2455489|AAAAAAAABMHHFCAA|2010-10-19|1329|5782|444|2010|2|10|19|4|2010|444|5782|Tuesday|2010Q4|N|N|N|2455471|2455743|2455124|2455397|N|N|N|N|N| +2455490|AAAAAAAACMHHFCAA|2010-10-20|1329|5782|444|2010|3|10|20|4|2010|444|5782|Wednesday|2010Q4|N|N|N|2455471|2455743|2455125|2455398|N|N|N|N|N| +2455491|AAAAAAAADMHHFCAA|2010-10-21|1329|5782|444|2010|4|10|21|4|2010|444|5782|Thursday|2010Q4|N|N|N|2455471|2455743|2455126|2455399|N|N|N|N|N| +2455492|AAAAAAAAEMHHFCAA|2010-10-22|1329|5782|444|2010|5|10|22|4|2010|444|5782|Friday|2010Q4|N|Y|N|2455471|2455743|2455127|2455400|N|N|N|N|N| +2455493|AAAAAAAAFMHHFCAA|2010-10-23|1329|5782|444|2010|6|10|23|4|2010|444|5782|Saturday|2010Q4|N|Y|N|2455471|2455743|2455128|2455401|N|N|N|N|N| +2455494|AAAAAAAAGMHHFCAA|2010-10-24|1329|5782|444|2010|0|10|24|4|2010|444|5782|Sunday|2010Q4|N|N|N|2455471|2455743|2455129|2455402|N|N|N|N|N| +2455495|AAAAAAAAHMHHFCAA|2010-10-25|1329|5782|444|2010|1|10|25|4|2010|444|5782|Monday|2010Q4|N|N|N|2455471|2455743|2455130|2455403|N|N|N|N|N| +2455496|AAAAAAAAIMHHFCAA|2010-10-26|1329|5783|444|2010|2|10|26|4|2010|444|5783|Tuesday|2010Q4|N|N|N|2455471|2455743|2455131|2455404|N|N|N|N|N| +2455497|AAAAAAAAJMHHFCAA|2010-10-27|1329|5783|444|2010|3|10|27|4|2010|444|5783|Wednesday|2010Q4|N|N|N|2455471|2455743|2455132|2455405|N|N|N|N|N| +2455498|AAAAAAAAKMHHFCAA|2010-10-28|1329|5783|444|2010|4|10|28|4|2010|444|5783|Thursday|2010Q4|N|N|N|2455471|2455743|2455133|2455406|N|N|N|N|N| +2455499|AAAAAAAALMHHFCAA|2010-10-29|1329|5783|444|2010|5|10|29|4|2010|444|5783|Friday|2010Q4|N|Y|N|2455471|2455743|2455134|2455407|N|N|N|N|N| +2455500|AAAAAAAAMMHHFCAA|2010-10-30|1329|5783|444|2010|6|10|30|4|2010|444|5783|Saturday|2010Q4|N|Y|N|2455471|2455743|2455135|2455408|N|N|N|N|N| +2455501|AAAAAAAANMHHFCAA|2010-10-31|1329|5783|444|2010|0|10|31|4|2010|444|5783|Sunday|2010Q4|N|N|N|2455471|2455743|2455136|2455409|N|N|N|N|N| +2455502|AAAAAAAAOMHHFCAA|2010-11-01|1330|5783|444|2010|1|11|1|4|2010|444|5783|Monday|2010Q4|N|N|N|2455502|2455805|2455137|2455410|N|N|N|N|N| +2455503|AAAAAAAAPMHHFCAA|2010-11-02|1330|5784|444|2010|2|11|2|4|2010|444|5784|Tuesday|2010Q4|N|N|N|2455502|2455805|2455138|2455411|N|N|N|N|N| +2455504|AAAAAAAAANHHFCAA|2010-11-03|1330|5784|444|2010|3|11|3|4|2010|444|5784|Wednesday|2010Q4|N|N|N|2455502|2455805|2455139|2455412|N|N|N|N|N| +2455505|AAAAAAAABNHHFCAA|2010-11-04|1330|5784|444|2010|4|11|4|4|2010|444|5784|Thursday|2010Q4|N|N|N|2455502|2455805|2455140|2455413|N|N|N|N|N| +2455506|AAAAAAAACNHHFCAA|2010-11-05|1330|5784|444|2010|5|11|5|4|2010|444|5784|Friday|2010Q4|N|Y|N|2455502|2455805|2455141|2455414|N|N|N|N|N| +2455507|AAAAAAAADNHHFCAA|2010-11-06|1330|5784|444|2010|6|11|6|4|2010|444|5784|Saturday|2010Q4|N|Y|N|2455502|2455805|2455142|2455415|N|N|N|N|N| +2455508|AAAAAAAAENHHFCAA|2010-11-07|1330|5784|444|2010|0|11|7|4|2010|444|5784|Sunday|2010Q4|N|N|N|2455502|2455805|2455143|2455416|N|N|N|N|N| +2455509|AAAAAAAAFNHHFCAA|2010-11-08|1330|5784|444|2010|1|11|8|4|2010|444|5784|Monday|2010Q4|N|N|N|2455502|2455805|2455144|2455417|N|N|N|N|N| +2455510|AAAAAAAAGNHHFCAA|2010-11-09|1330|5785|444|2010|2|11|9|4|2010|444|5785|Tuesday|2010Q4|N|N|N|2455502|2455805|2455145|2455418|N|N|N|N|N| +2455511|AAAAAAAAHNHHFCAA|2010-11-10|1330|5785|444|2010|3|11|10|4|2010|444|5785|Wednesday|2010Q4|N|N|N|2455502|2455805|2455146|2455419|N|N|N|N|N| +2455512|AAAAAAAAINHHFCAA|2010-11-11|1330|5785|444|2010|4|11|11|4|2010|444|5785|Thursday|2010Q4|N|N|N|2455502|2455805|2455147|2455420|N|N|N|N|N| +2455513|AAAAAAAAJNHHFCAA|2010-11-12|1330|5785|444|2010|5|11|12|4|2010|444|5785|Friday|2010Q4|N|Y|N|2455502|2455805|2455148|2455421|N|N|N|N|N| +2455514|AAAAAAAAKNHHFCAA|2010-11-13|1330|5785|444|2010|6|11|13|4|2010|444|5785|Saturday|2010Q4|N|Y|N|2455502|2455805|2455149|2455422|N|N|N|N|N| +2455515|AAAAAAAALNHHFCAA|2010-11-14|1330|5785|444|2010|0|11|14|4|2010|444|5785|Sunday|2010Q4|N|N|N|2455502|2455805|2455150|2455423|N|N|N|N|N| +2455516|AAAAAAAAMNHHFCAA|2010-11-15|1330|5785|444|2010|1|11|15|4|2010|444|5785|Monday|2010Q4|N|N|N|2455502|2455805|2455151|2455424|N|N|N|N|N| +2455517|AAAAAAAANNHHFCAA|2010-11-16|1330|5786|444|2010|2|11|16|4|2010|444|5786|Tuesday|2010Q4|N|N|N|2455502|2455805|2455152|2455425|N|N|N|N|N| +2455518|AAAAAAAAONHHFCAA|2010-11-17|1330|5786|444|2010|3|11|17|4|2010|444|5786|Wednesday|2010Q4|N|N|N|2455502|2455805|2455153|2455426|N|N|N|N|N| +2455519|AAAAAAAAPNHHFCAA|2010-11-18|1330|5786|444|2010|4|11|18|4|2010|444|5786|Thursday|2010Q4|N|N|N|2455502|2455805|2455154|2455427|N|N|N|N|N| +2455520|AAAAAAAAAOHHFCAA|2010-11-19|1330|5786|444|2010|5|11|19|4|2010|444|5786|Friday|2010Q4|N|Y|N|2455502|2455805|2455155|2455428|N|N|N|N|N| +2455521|AAAAAAAABOHHFCAA|2010-11-20|1330|5786|444|2010|6|11|20|4|2010|444|5786|Saturday|2010Q4|N|Y|N|2455502|2455805|2455156|2455429|N|N|N|N|N| +2455522|AAAAAAAACOHHFCAA|2010-11-21|1330|5786|444|2010|0|11|21|4|2010|444|5786|Sunday|2010Q4|N|N|N|2455502|2455805|2455157|2455430|N|N|N|N|N| +2455523|AAAAAAAADOHHFCAA|2010-11-22|1330|5786|444|2010|1|11|22|4|2010|444|5786|Monday|2010Q4|N|N|N|2455502|2455805|2455158|2455431|N|N|N|N|N| +2455524|AAAAAAAAEOHHFCAA|2010-11-23|1330|5787|444|2010|2|11|23|4|2010|444|5787|Tuesday|2010Q4|N|N|N|2455502|2455805|2455159|2455432|N|N|N|N|N| +2455525|AAAAAAAAFOHHFCAA|2010-11-24|1330|5787|444|2010|3|11|24|4|2010|444|5787|Wednesday|2010Q4|N|N|N|2455502|2455805|2455160|2455433|N|N|N|N|N| +2455526|AAAAAAAAGOHHFCAA|2010-11-25|1330|5787|444|2010|4|11|25|4|2010|444|5787|Thursday|2010Q4|N|N|N|2455502|2455805|2455161|2455434|N|N|N|N|N| +2455527|AAAAAAAAHOHHFCAA|2010-11-26|1330|5787|444|2010|5|11|26|4|2010|444|5787|Friday|2010Q4|N|Y|N|2455502|2455805|2455162|2455435|N|N|N|N|N| +2455528|AAAAAAAAIOHHFCAA|2010-11-27|1330|5787|444|2010|6|11|27|4|2010|444|5787|Saturday|2010Q4|N|Y|N|2455502|2455805|2455163|2455436|N|N|N|N|N| +2455529|AAAAAAAAJOHHFCAA|2010-11-28|1330|5787|444|2010|0|11|28|4|2010|444|5787|Sunday|2010Q4|N|N|N|2455502|2455805|2455164|2455437|N|N|N|N|N| +2455530|AAAAAAAAKOHHFCAA|2010-11-29|1330|5787|444|2010|1|11|29|4|2010|444|5787|Monday|2010Q4|N|N|N|2455502|2455805|2455165|2455438|N|N|N|N|N| +2455531|AAAAAAAALOHHFCAA|2010-11-30|1330|5788|444|2010|2|11|30|4|2010|444|5788|Tuesday|2010Q4|N|N|N|2455502|2455805|2455166|2455439|N|N|N|N|N| +2455532|AAAAAAAAMOHHFCAA|2010-12-01|1331|5788|445|2010|3|12|1|4|2010|445|5788|Wednesday|2010Q4|N|N|N|2455532|2455865|2455167|2455440|N|N|N|N|N| +2455533|AAAAAAAANOHHFCAA|2010-12-02|1331|5788|445|2010|4|12|2|4|2010|445|5788|Thursday|2010Q4|N|N|N|2455532|2455865|2455168|2455441|N|N|N|N|N| +2455534|AAAAAAAAOOHHFCAA|2010-12-03|1331|5788|445|2010|5|12|3|4|2010|445|5788|Friday|2010Q4|N|Y|N|2455532|2455865|2455169|2455442|N|N|N|N|N| +2455535|AAAAAAAAPOHHFCAA|2010-12-04|1331|5788|445|2010|6|12|4|4|2010|445|5788|Saturday|2010Q4|N|Y|N|2455532|2455865|2455170|2455443|N|N|N|N|N| +2455536|AAAAAAAAAPHHFCAA|2010-12-05|1331|5788|445|2010|0|12|5|4|2010|445|5788|Sunday|2010Q4|N|N|N|2455532|2455865|2455171|2455444|N|N|N|N|N| +2455537|AAAAAAAABPHHFCAA|2010-12-06|1331|5788|445|2010|1|12|6|4|2010|445|5788|Monday|2010Q4|N|N|N|2455532|2455865|2455172|2455445|N|N|N|N|N| +2455538|AAAAAAAACPHHFCAA|2010-12-07|1331|5789|445|2010|2|12|7|4|2010|445|5789|Tuesday|2010Q4|N|N|N|2455532|2455865|2455173|2455446|N|N|N|N|N| +2455539|AAAAAAAADPHHFCAA|2010-12-08|1331|5789|445|2010|3|12|8|4|2010|445|5789|Wednesday|2010Q4|N|N|N|2455532|2455865|2455174|2455447|N|N|N|N|N| +2455540|AAAAAAAAEPHHFCAA|2010-12-09|1331|5789|445|2010|4|12|9|4|2010|445|5789|Thursday|2010Q4|N|N|N|2455532|2455865|2455175|2455448|N|N|N|N|N| +2455541|AAAAAAAAFPHHFCAA|2010-12-10|1331|5789|445|2010|5|12|10|4|2010|445|5789|Friday|2010Q4|N|Y|N|2455532|2455865|2455176|2455449|N|N|N|N|N| +2455542|AAAAAAAAGPHHFCAA|2010-12-11|1331|5789|445|2010|6|12|11|4|2010|445|5789|Saturday|2010Q4|N|Y|N|2455532|2455865|2455177|2455450|N|N|N|N|N| +2455543|AAAAAAAAHPHHFCAA|2010-12-12|1331|5789|445|2010|0|12|12|4|2010|445|5789|Sunday|2010Q4|N|N|N|2455532|2455865|2455178|2455451|N|N|N|N|N| +2455544|AAAAAAAAIPHHFCAA|2010-12-13|1331|5789|445|2010|1|12|13|4|2010|445|5789|Monday|2010Q4|N|N|N|2455532|2455865|2455179|2455452|N|N|N|N|N| +2455545|AAAAAAAAJPHHFCAA|2010-12-14|1331|5790|445|2010|2|12|14|4|2010|445|5790|Tuesday|2010Q4|N|N|N|2455532|2455865|2455180|2455453|N|N|N|N|N| +2455546|AAAAAAAAKPHHFCAA|2010-12-15|1331|5790|445|2010|3|12|15|4|2010|445|5790|Wednesday|2010Q4|N|N|N|2455532|2455865|2455181|2455454|N|N|N|N|N| +2455547|AAAAAAAALPHHFCAA|2010-12-16|1331|5790|445|2010|4|12|16|4|2010|445|5790|Thursday|2010Q4|N|N|N|2455532|2455865|2455182|2455455|N|N|N|N|N| +2455548|AAAAAAAAMPHHFCAA|2010-12-17|1331|5790|445|2010|5|12|17|4|2010|445|5790|Friday|2010Q4|N|Y|N|2455532|2455865|2455183|2455456|N|N|N|N|N| +2455549|AAAAAAAANPHHFCAA|2010-12-18|1331|5790|445|2010|6|12|18|4|2010|445|5790|Saturday|2010Q4|N|Y|N|2455532|2455865|2455184|2455457|N|N|N|N|N| +2455550|AAAAAAAAOPHHFCAA|2010-12-19|1331|5790|445|2010|0|12|19|4|2010|445|5790|Sunday|2010Q4|N|N|N|2455532|2455865|2455185|2455458|N|N|N|N|N| +2455551|AAAAAAAAPPHHFCAA|2010-12-20|1331|5790|445|2010|1|12|20|4|2010|445|5790|Monday|2010Q4|N|N|N|2455532|2455865|2455186|2455459|N|N|N|N|N| +2455552|AAAAAAAAAAIHFCAA|2010-12-21|1331|5791|445|2010|2|12|21|4|2010|445|5791|Tuesday|2010Q4|N|N|N|2455532|2455865|2455187|2455460|N|N|N|N|N| +2455553|AAAAAAAABAIHFCAA|2010-12-22|1331|5791|445|2010|3|12|22|4|2010|445|5791|Wednesday|2010Q4|N|N|N|2455532|2455865|2455188|2455461|N|N|N|N|N| +2455554|AAAAAAAACAIHFCAA|2010-12-23|1331|5791|445|2010|4|12|23|4|2010|445|5791|Thursday|2010Q4|N|N|N|2455532|2455865|2455189|2455462|N|N|N|N|N| +2455555|AAAAAAAADAIHFCAA|2010-12-24|1331|5791|445|2010|5|12|24|4|2010|445|5791|Friday|2010Q4|N|Y|N|2455532|2455865|2455190|2455463|N|N|N|N|N| +2455556|AAAAAAAAEAIHFCAA|2010-12-25|1331|5791|445|2010|6|12|25|4|2010|445|5791|Saturday|2010Q4|N|Y|N|2455532|2455865|2455191|2455464|N|N|N|N|N| +2455557|AAAAAAAAFAIHFCAA|2010-12-26|1331|5791|445|2010|0|12|26|4|2010|445|5791|Sunday|2010Q4|Y|N|N|2455532|2455865|2455192|2455465|N|N|N|N|N| +2455558|AAAAAAAAGAIHFCAA|2010-12-27|1331|5791|445|2010|1|12|27|4|2010|445|5791|Monday|2010Q4|N|N|Y|2455532|2455865|2455193|2455466|N|N|N|N|N| +2455559|AAAAAAAAHAIHFCAA|2010-12-28|1331|5792|445|2010|2|12|28|4|2010|445|5792|Tuesday|2010Q4|N|N|N|2455532|2455865|2455194|2455467|N|N|N|N|N| +2455560|AAAAAAAAIAIHFCAA|2010-12-29|1331|5792|445|2010|3|12|29|4|2010|445|5792|Wednesday|2010Q4|N|N|N|2455532|2455865|2455195|2455468|N|N|N|N|N| +2455561|AAAAAAAAJAIHFCAA|2010-12-30|1331|5792|445|2010|4|12|30|4|2010|445|5792|Thursday|2010Q4|N|N|N|2455532|2455865|2455196|2455469|N|N|N|N|N| +2455562|AAAAAAAAKAIHFCAA|2010-12-31|1331|5792|445|2010|5|12|31|4|2010|445|5792|Friday|2010Q4|N|Y|N|2455532|2455865|2455197|2455470|N|N|N|N|N| +2455563|AAAAAAAALAIHFCAA|2011-01-01|1332|5792|445|2011|6|1|1|1|2011|445|5792|Saturday|2011Q1|Y|Y|N|2455563|2455562|2455198|2455471|N|N|N|N|N| +2455564|AAAAAAAAMAIHFCAA|2011-01-02|1332|5792|445|2011|0|1|2|1|2011|445|5792|Sunday|2011Q1|N|N|Y|2455563|2455562|2455199|2455472|N|N|N|N|N| +2455565|AAAAAAAANAIHFCAA|2011-01-03|1332|5792|445|2011|1|1|3|1|2011|445|5792|Monday|2011Q1|N|N|N|2455563|2455562|2455200|2455473|N|N|N|N|N| +2455566|AAAAAAAAOAIHFCAA|2011-01-04|1332|5793|445|2011|2|1|4|1|2011|445|5793|Tuesday|2011Q1|N|N|N|2455563|2455562|2455201|2455474|N|N|N|N|N| +2455567|AAAAAAAAPAIHFCAA|2011-01-05|1332|5793|445|2011|3|1|5|1|2011|445|5793|Wednesday|2011Q1|N|N|N|2455563|2455562|2455202|2455475|N|N|N|N|N| +2455568|AAAAAAAAABIHFCAA|2011-01-06|1332|5793|445|2011|4|1|6|1|2011|445|5793|Thursday|2011Q1|N|N|N|2455563|2455562|2455203|2455476|N|N|N|N|N| +2455569|AAAAAAAABBIHFCAA|2011-01-07|1332|5793|445|2011|5|1|7|1|2011|445|5793|Friday|2011Q1|N|Y|N|2455563|2455562|2455204|2455477|N|N|N|N|N| +2455570|AAAAAAAACBIHFCAA|2011-01-08|1332|5793|445|2011|6|1|8|1|2011|445|5793|Saturday|2011Q1|N|Y|N|2455563|2455562|2455205|2455478|N|N|N|N|N| +2455571|AAAAAAAADBIHFCAA|2011-01-09|1332|5793|445|2011|0|1|9|1|2011|445|5793|Sunday|2011Q1|N|N|N|2455563|2455562|2455206|2455479|N|N|N|N|N| +2455572|AAAAAAAAEBIHFCAA|2011-01-10|1332|5793|445|2011|1|1|10|1|2011|445|5793|Monday|2011Q1|N|N|N|2455563|2455562|2455207|2455480|N|N|N|N|N| +2455573|AAAAAAAAFBIHFCAA|2011-01-11|1332|5794|445|2011|2|1|11|1|2011|445|5794|Tuesday|2011Q1|N|N|N|2455563|2455562|2455208|2455481|N|N|N|N|N| +2455574|AAAAAAAAGBIHFCAA|2011-01-12|1332|5794|445|2011|3|1|12|1|2011|445|5794|Wednesday|2011Q1|N|N|N|2455563|2455562|2455209|2455482|N|N|N|N|N| +2455575|AAAAAAAAHBIHFCAA|2011-01-13|1332|5794|445|2011|4|1|13|1|2011|445|5794|Thursday|2011Q1|N|N|N|2455563|2455562|2455210|2455483|N|N|N|N|N| +2455576|AAAAAAAAIBIHFCAA|2011-01-14|1332|5794|445|2011|5|1|14|1|2011|445|5794|Friday|2011Q1|N|Y|N|2455563|2455562|2455211|2455484|N|N|N|N|N| +2455577|AAAAAAAAJBIHFCAA|2011-01-15|1332|5794|445|2011|6|1|15|1|2011|445|5794|Saturday|2011Q1|N|Y|N|2455563|2455562|2455212|2455485|N|N|N|N|N| +2455578|AAAAAAAAKBIHFCAA|2011-01-16|1332|5794|445|2011|0|1|16|1|2011|445|5794|Sunday|2011Q1|N|N|N|2455563|2455562|2455213|2455486|N|N|N|N|N| +2455579|AAAAAAAALBIHFCAA|2011-01-17|1332|5794|445|2011|1|1|17|1|2011|445|5794|Monday|2011Q1|N|N|N|2455563|2455562|2455214|2455487|N|N|N|N|N| +2455580|AAAAAAAAMBIHFCAA|2011-01-18|1332|5795|445|2011|2|1|18|1|2011|445|5795|Tuesday|2011Q1|N|N|N|2455563|2455562|2455215|2455488|N|N|N|N|N| +2455581|AAAAAAAANBIHFCAA|2011-01-19|1332|5795|445|2011|3|1|19|1|2011|445|5795|Wednesday|2011Q1|N|N|N|2455563|2455562|2455216|2455489|N|N|N|N|N| +2455582|AAAAAAAAOBIHFCAA|2011-01-20|1332|5795|445|2011|4|1|20|1|2011|445|5795|Thursday|2011Q1|N|N|N|2455563|2455562|2455217|2455490|N|N|N|N|N| +2455583|AAAAAAAAPBIHFCAA|2011-01-21|1332|5795|445|2011|5|1|21|1|2011|445|5795|Friday|2011Q1|N|Y|N|2455563|2455562|2455218|2455491|N|N|N|N|N| +2455584|AAAAAAAAACIHFCAA|2011-01-22|1332|5795|445|2011|6|1|22|1|2011|445|5795|Saturday|2011Q1|N|Y|N|2455563|2455562|2455219|2455492|N|N|N|N|N| +2455585|AAAAAAAABCIHFCAA|2011-01-23|1332|5795|445|2011|0|1|23|1|2011|445|5795|Sunday|2011Q1|N|N|N|2455563|2455562|2455220|2455493|N|N|N|N|N| +2455586|AAAAAAAACCIHFCAA|2011-01-24|1332|5795|445|2011|1|1|24|1|2011|445|5795|Monday|2011Q1|N|N|N|2455563|2455562|2455221|2455494|N|N|N|N|N| +2455587|AAAAAAAADCIHFCAA|2011-01-25|1332|5796|445|2011|2|1|25|1|2011|445|5796|Tuesday|2011Q1|N|N|N|2455563|2455562|2455222|2455495|N|N|N|N|N| +2455588|AAAAAAAAECIHFCAA|2011-01-26|1332|5796|445|2011|3|1|26|1|2011|445|5796|Wednesday|2011Q1|N|N|N|2455563|2455562|2455223|2455496|N|N|N|N|N| +2455589|AAAAAAAAFCIHFCAA|2011-01-27|1332|5796|445|2011|4|1|27|1|2011|445|5796|Thursday|2011Q1|N|N|N|2455563|2455562|2455224|2455497|N|N|N|N|N| +2455590|AAAAAAAAGCIHFCAA|2011-01-28|1332|5796|445|2011|5|1|28|1|2011|445|5796|Friday|2011Q1|N|Y|N|2455563|2455562|2455225|2455498|N|N|N|N|N| +2455591|AAAAAAAAHCIHFCAA|2011-01-29|1332|5796|445|2011|6|1|29|1|2011|445|5796|Saturday|2011Q1|N|Y|N|2455563|2455562|2455226|2455499|N|N|N|N|N| +2455592|AAAAAAAAICIHFCAA|2011-01-30|1332|5796|445|2011|0|1|30|1|2011|445|5796|Sunday|2011Q1|N|N|N|2455563|2455562|2455227|2455500|N|N|N|N|N| +2455593|AAAAAAAAJCIHFCAA|2011-01-31|1332|5796|445|2011|1|1|31|1|2011|445|5796|Monday|2011Q1|N|N|N|2455563|2455562|2455228|2455501|N|N|N|N|N| +2455594|AAAAAAAAKCIHFCAA|2011-02-01|1333|5797|445|2011|2|2|1|1|2011|445|5797|Tuesday|2011Q1|N|N|N|2455594|2455624|2455229|2455502|N|N|N|N|N| +2455595|AAAAAAAALCIHFCAA|2011-02-02|1333|5797|445|2011|3|2|2|1|2011|445|5797|Wednesday|2011Q1|N|N|N|2455594|2455624|2455230|2455503|N|N|N|N|N| +2455596|AAAAAAAAMCIHFCAA|2011-02-03|1333|5797|445|2011|4|2|3|1|2011|445|5797|Thursday|2011Q1|N|N|N|2455594|2455624|2455231|2455504|N|N|N|N|N| +2455597|AAAAAAAANCIHFCAA|2011-02-04|1333|5797|445|2011|5|2|4|1|2011|445|5797|Friday|2011Q1|N|Y|N|2455594|2455624|2455232|2455505|N|N|N|N|N| +2455598|AAAAAAAAOCIHFCAA|2011-02-05|1333|5797|445|2011|6|2|5|1|2011|445|5797|Saturday|2011Q1|N|Y|N|2455594|2455624|2455233|2455506|N|N|N|N|N| +2455599|AAAAAAAAPCIHFCAA|2011-02-06|1333|5797|445|2011|0|2|6|1|2011|445|5797|Sunday|2011Q1|N|N|N|2455594|2455624|2455234|2455507|N|N|N|N|N| +2455600|AAAAAAAAADIHFCAA|2011-02-07|1333|5797|445|2011|1|2|7|1|2011|445|5797|Monday|2011Q1|N|N|N|2455594|2455624|2455235|2455508|N|N|N|N|N| +2455601|AAAAAAAABDIHFCAA|2011-02-08|1333|5798|445|2011|2|2|8|1|2011|445|5798|Tuesday|2011Q1|N|N|N|2455594|2455624|2455236|2455509|N|N|N|N|N| +2455602|AAAAAAAACDIHFCAA|2011-02-09|1333|5798|445|2011|3|2|9|1|2011|445|5798|Wednesday|2011Q1|N|N|N|2455594|2455624|2455237|2455510|N|N|N|N|N| +2455603|AAAAAAAADDIHFCAA|2011-02-10|1333|5798|445|2011|4|2|10|1|2011|445|5798|Thursday|2011Q1|N|N|N|2455594|2455624|2455238|2455511|N|N|N|N|N| +2455604|AAAAAAAAEDIHFCAA|2011-02-11|1333|5798|445|2011|5|2|11|1|2011|445|5798|Friday|2011Q1|N|Y|N|2455594|2455624|2455239|2455512|N|N|N|N|N| +2455605|AAAAAAAAFDIHFCAA|2011-02-12|1333|5798|445|2011|6|2|12|1|2011|445|5798|Saturday|2011Q1|N|Y|N|2455594|2455624|2455240|2455513|N|N|N|N|N| +2455606|AAAAAAAAGDIHFCAA|2011-02-13|1333|5798|445|2011|0|2|13|1|2011|445|5798|Sunday|2011Q1|N|N|N|2455594|2455624|2455241|2455514|N|N|N|N|N| +2455607|AAAAAAAAHDIHFCAA|2011-02-14|1333|5798|445|2011|1|2|14|1|2011|445|5798|Monday|2011Q1|N|N|N|2455594|2455624|2455242|2455515|N|N|N|N|N| +2455608|AAAAAAAAIDIHFCAA|2011-02-15|1333|5799|445|2011|2|2|15|1|2011|445|5799|Tuesday|2011Q1|N|N|N|2455594|2455624|2455243|2455516|N|N|N|N|N| +2455609|AAAAAAAAJDIHFCAA|2011-02-16|1333|5799|445|2011|3|2|16|1|2011|445|5799|Wednesday|2011Q1|N|N|N|2455594|2455624|2455244|2455517|N|N|N|N|N| +2455610|AAAAAAAAKDIHFCAA|2011-02-17|1333|5799|445|2011|4|2|17|1|2011|445|5799|Thursday|2011Q1|N|N|N|2455594|2455624|2455245|2455518|N|N|N|N|N| +2455611|AAAAAAAALDIHFCAA|2011-02-18|1333|5799|445|2011|5|2|18|1|2011|445|5799|Friday|2011Q1|N|Y|N|2455594|2455624|2455246|2455519|N|N|N|N|N| +2455612|AAAAAAAAMDIHFCAA|2011-02-19|1333|5799|445|2011|6|2|19|1|2011|445|5799|Saturday|2011Q1|N|Y|N|2455594|2455624|2455247|2455520|N|N|N|N|N| +2455613|AAAAAAAANDIHFCAA|2011-02-20|1333|5799|445|2011|0|2|20|1|2011|445|5799|Sunday|2011Q1|N|N|N|2455594|2455624|2455248|2455521|N|N|N|N|N| +2455614|AAAAAAAAODIHFCAA|2011-02-21|1333|5799|445|2011|1|2|21|1|2011|445|5799|Monday|2011Q1|N|N|N|2455594|2455624|2455249|2455522|N|N|N|N|N| +2455615|AAAAAAAAPDIHFCAA|2011-02-22|1333|5800|445|2011|2|2|22|1|2011|445|5800|Tuesday|2011Q1|N|N|N|2455594|2455624|2455250|2455523|N|N|N|N|N| +2455616|AAAAAAAAAEIHFCAA|2011-02-23|1333|5800|445|2011|3|2|23|1|2011|445|5800|Wednesday|2011Q1|N|N|N|2455594|2455624|2455251|2455524|N|N|N|N|N| +2455617|AAAAAAAABEIHFCAA|2011-02-24|1333|5800|445|2011|4|2|24|1|2011|445|5800|Thursday|2011Q1|N|N|N|2455594|2455624|2455252|2455525|N|N|N|N|N| +2455618|AAAAAAAACEIHFCAA|2011-02-25|1333|5800|445|2011|5|2|25|1|2011|445|5800|Friday|2011Q1|N|Y|N|2455594|2455624|2455253|2455526|N|N|N|N|N| +2455619|AAAAAAAADEIHFCAA|2011-02-26|1333|5800|445|2011|6|2|26|1|2011|445|5800|Saturday|2011Q1|N|Y|N|2455594|2455624|2455254|2455527|N|N|N|N|N| +2455620|AAAAAAAAEEIHFCAA|2011-02-27|1333|5800|445|2011|0|2|27|1|2011|445|5800|Sunday|2011Q1|N|N|N|2455594|2455624|2455255|2455528|N|N|N|N|N| +2455621|AAAAAAAAFEIHFCAA|2011-02-28|1333|5800|445|2011|1|2|28|1|2011|445|5800|Monday|2011Q1|N|N|N|2455594|2455624|2455256|2455529|N|N|N|N|N| +2455622|AAAAAAAAGEIHFCAA|2011-03-01|1334|5801|446|2011|2|3|1|1|2011|446|5801|Tuesday|2011Q1|N|N|N|2455622|2455680|2455257|2455530|N|N|N|N|N| +2455623|AAAAAAAAHEIHFCAA|2011-03-02|1334|5801|446|2011|3|3|2|1|2011|446|5801|Wednesday|2011Q1|N|N|N|2455622|2455680|2455258|2455531|N|N|N|N|N| +2455624|AAAAAAAAIEIHFCAA|2011-03-03|1334|5801|446|2011|4|3|3|1|2011|446|5801|Thursday|2011Q1|N|N|N|2455622|2455680|2455259|2455532|N|N|N|N|N| +2455625|AAAAAAAAJEIHFCAA|2011-03-04|1334|5801|446|2011|5|3|4|1|2011|446|5801|Friday|2011Q1|N|Y|N|2455622|2455680|2455260|2455533|N|N|N|N|N| +2455626|AAAAAAAAKEIHFCAA|2011-03-05|1334|5801|446|2011|6|3|5|1|2011|446|5801|Saturday|2011Q1|N|Y|N|2455622|2455680|2455261|2455534|N|N|N|N|N| +2455627|AAAAAAAALEIHFCAA|2011-03-06|1334|5801|446|2011|0|3|6|1|2011|446|5801|Sunday|2011Q1|N|N|N|2455622|2455680|2455262|2455535|N|N|N|N|N| +2455628|AAAAAAAAMEIHFCAA|2011-03-07|1334|5801|446|2011|1|3|7|1|2011|446|5801|Monday|2011Q1|N|N|N|2455622|2455680|2455263|2455536|N|N|N|N|N| +2455629|AAAAAAAANEIHFCAA|2011-03-08|1334|5802|446|2011|2|3|8|1|2011|446|5802|Tuesday|2011Q1|N|N|N|2455622|2455680|2455264|2455537|N|N|N|N|N| +2455630|AAAAAAAAOEIHFCAA|2011-03-09|1334|5802|446|2011|3|3|9|1|2011|446|5802|Wednesday|2011Q1|N|N|N|2455622|2455680|2455265|2455538|N|N|N|N|N| +2455631|AAAAAAAAPEIHFCAA|2011-03-10|1334|5802|446|2011|4|3|10|1|2011|446|5802|Thursday|2011Q1|N|N|N|2455622|2455680|2455266|2455539|N|N|N|N|N| +2455632|AAAAAAAAAFIHFCAA|2011-03-11|1334|5802|446|2011|5|3|11|1|2011|446|5802|Friday|2011Q1|N|Y|N|2455622|2455680|2455267|2455540|N|N|N|N|N| +2455633|AAAAAAAABFIHFCAA|2011-03-12|1334|5802|446|2011|6|3|12|1|2011|446|5802|Saturday|2011Q1|N|Y|N|2455622|2455680|2455268|2455541|N|N|N|N|N| +2455634|AAAAAAAACFIHFCAA|2011-03-13|1334|5802|446|2011|0|3|13|1|2011|446|5802|Sunday|2011Q1|N|N|N|2455622|2455680|2455269|2455542|N|N|N|N|N| +2455635|AAAAAAAADFIHFCAA|2011-03-14|1334|5802|446|2011|1|3|14|1|2011|446|5802|Monday|2011Q1|N|N|N|2455622|2455680|2455270|2455543|N|N|N|N|N| +2455636|AAAAAAAAEFIHFCAA|2011-03-15|1334|5803|446|2011|2|3|15|1|2011|446|5803|Tuesday|2011Q1|N|N|N|2455622|2455680|2455271|2455544|N|N|N|N|N| +2455637|AAAAAAAAFFIHFCAA|2011-03-16|1334|5803|446|2011|3|3|16|1|2011|446|5803|Wednesday|2011Q1|N|N|N|2455622|2455680|2455272|2455545|N|N|N|N|N| +2455638|AAAAAAAAGFIHFCAA|2011-03-17|1334|5803|446|2011|4|3|17|1|2011|446|5803|Thursday|2011Q1|N|N|N|2455622|2455680|2455273|2455546|N|N|N|N|N| +2455639|AAAAAAAAHFIHFCAA|2011-03-18|1334|5803|446|2011|5|3|18|1|2011|446|5803|Friday|2011Q1|N|Y|N|2455622|2455680|2455274|2455547|N|N|N|N|N| +2455640|AAAAAAAAIFIHFCAA|2011-03-19|1334|5803|446|2011|6|3|19|1|2011|446|5803|Saturday|2011Q1|N|Y|N|2455622|2455680|2455275|2455548|N|N|N|N|N| +2455641|AAAAAAAAJFIHFCAA|2011-03-20|1334|5803|446|2011|0|3|20|1|2011|446|5803|Sunday|2011Q1|N|N|N|2455622|2455680|2455276|2455549|N|N|N|N|N| +2455642|AAAAAAAAKFIHFCAA|2011-03-21|1334|5803|446|2011|1|3|21|1|2011|446|5803|Monday|2011Q1|N|N|N|2455622|2455680|2455277|2455550|N|N|N|N|N| +2455643|AAAAAAAALFIHFCAA|2011-03-22|1334|5804|446|2011|2|3|22|1|2011|446|5804|Tuesday|2011Q1|N|N|N|2455622|2455680|2455278|2455551|N|N|N|N|N| +2455644|AAAAAAAAMFIHFCAA|2011-03-23|1334|5804|446|2011|3|3|23|1|2011|446|5804|Wednesday|2011Q1|N|N|N|2455622|2455680|2455279|2455552|N|N|N|N|N| +2455645|AAAAAAAANFIHFCAA|2011-03-24|1334|5804|446|2011|4|3|24|1|2011|446|5804|Thursday|2011Q1|N|N|N|2455622|2455680|2455280|2455553|N|N|N|N|N| +2455646|AAAAAAAAOFIHFCAA|2011-03-25|1334|5804|446|2011|5|3|25|1|2011|446|5804|Friday|2011Q1|N|Y|N|2455622|2455680|2455281|2455554|N|N|N|N|N| +2455647|AAAAAAAAPFIHFCAA|2011-03-26|1334|5804|446|2011|6|3|26|1|2011|446|5804|Saturday|2011Q1|N|Y|N|2455622|2455680|2455282|2455555|N|N|N|N|N| +2455648|AAAAAAAAAGIHFCAA|2011-03-27|1334|5804|446|2011|0|3|27|1|2011|446|5804|Sunday|2011Q1|N|N|N|2455622|2455680|2455283|2455556|N|N|N|N|N| +2455649|AAAAAAAABGIHFCAA|2011-03-28|1334|5804|446|2011|1|3|28|1|2011|446|5804|Monday|2011Q1|N|N|N|2455622|2455680|2455284|2455557|N|N|N|N|N| +2455650|AAAAAAAACGIHFCAA|2011-03-29|1334|5805|446|2011|2|3|29|1|2011|446|5805|Tuesday|2011Q1|N|N|N|2455622|2455680|2455285|2455558|N|N|N|N|N| +2455651|AAAAAAAADGIHFCAA|2011-03-30|1334|5805|446|2011|3|3|30|1|2011|446|5805|Wednesday|2011Q1|N|N|N|2455622|2455680|2455286|2455559|N|N|N|N|N| +2455652|AAAAAAAAEGIHFCAA|2011-03-31|1334|5805|446|2011|4|3|31|1|2011|446|5805|Thursday|2011Q1|N|N|N|2455622|2455680|2455287|2455560|N|N|N|N|N| +2455653|AAAAAAAAFGIHFCAA|2011-04-01|1335|5805|446|2011|5|4|1|1|2011|446|5805|Friday|2011Q1|N|Y|N|2455653|2455742|2455288|2455563|N|N|N|N|N| +2455654|AAAAAAAAGGIHFCAA|2011-04-02|1335|5805|446|2011|6|4|2|2|2011|446|5805|Saturday|2011Q2|N|Y|N|2455653|2455742|2455289|2455564|N|N|N|N|N| +2455655|AAAAAAAAHGIHFCAA|2011-04-03|1335|5805|446|2011|0|4|3|2|2011|446|5805|Sunday|2011Q2|N|N|N|2455653|2455742|2455290|2455565|N|N|N|N|N| +2455656|AAAAAAAAIGIHFCAA|2011-04-04|1335|5805|446|2011|1|4|4|2|2011|446|5805|Monday|2011Q2|N|N|N|2455653|2455742|2455291|2455566|N|N|N|N|N| +2455657|AAAAAAAAJGIHFCAA|2011-04-05|1335|5806|446|2011|2|4|5|2|2011|446|5806|Tuesday|2011Q2|N|N|N|2455653|2455742|2455292|2455567|N|N|N|N|N| +2455658|AAAAAAAAKGIHFCAA|2011-04-06|1335|5806|446|2011|3|4|6|2|2011|446|5806|Wednesday|2011Q2|N|N|N|2455653|2455742|2455293|2455568|N|N|N|N|N| +2455659|AAAAAAAALGIHFCAA|2011-04-07|1335|5806|446|2011|4|4|7|2|2011|446|5806|Thursday|2011Q2|N|N|N|2455653|2455742|2455294|2455569|N|N|N|N|N| +2455660|AAAAAAAAMGIHFCAA|2011-04-08|1335|5806|446|2011|5|4|8|2|2011|446|5806|Friday|2011Q2|N|Y|N|2455653|2455742|2455295|2455570|N|N|N|N|N| +2455661|AAAAAAAANGIHFCAA|2011-04-09|1335|5806|446|2011|6|4|9|2|2011|446|5806|Saturday|2011Q2|N|Y|N|2455653|2455742|2455296|2455571|N|N|N|N|N| +2455662|AAAAAAAAOGIHFCAA|2011-04-10|1335|5806|446|2011|0|4|10|2|2011|446|5806|Sunday|2011Q2|N|N|N|2455653|2455742|2455297|2455572|N|N|N|N|N| +2455663|AAAAAAAAPGIHFCAA|2011-04-11|1335|5806|446|2011|1|4|11|2|2011|446|5806|Monday|2011Q2|N|N|N|2455653|2455742|2455298|2455573|N|N|N|N|N| +2455664|AAAAAAAAAHIHFCAA|2011-04-12|1335|5807|446|2011|2|4|12|2|2011|446|5807|Tuesday|2011Q2|N|N|N|2455653|2455742|2455299|2455574|N|N|N|N|N| +2455665|AAAAAAAABHIHFCAA|2011-04-13|1335|5807|446|2011|3|4|13|2|2011|446|5807|Wednesday|2011Q2|N|N|N|2455653|2455742|2455300|2455575|N|N|N|N|N| +2455666|AAAAAAAACHIHFCAA|2011-04-14|1335|5807|446|2011|4|4|14|2|2011|446|5807|Thursday|2011Q2|N|N|N|2455653|2455742|2455301|2455576|N|N|N|N|N| +2455667|AAAAAAAADHIHFCAA|2011-04-15|1335|5807|446|2011|5|4|15|2|2011|446|5807|Friday|2011Q2|N|Y|N|2455653|2455742|2455302|2455577|N|N|N|N|N| +2455668|AAAAAAAAEHIHFCAA|2011-04-16|1335|5807|446|2011|6|4|16|2|2011|446|5807|Saturday|2011Q2|N|Y|N|2455653|2455742|2455303|2455578|N|N|N|N|N| +2455669|AAAAAAAAFHIHFCAA|2011-04-17|1335|5807|446|2011|0|4|17|2|2011|446|5807|Sunday|2011Q2|N|N|N|2455653|2455742|2455304|2455579|N|N|N|N|N| +2455670|AAAAAAAAGHIHFCAA|2011-04-18|1335|5807|446|2011|1|4|18|2|2011|446|5807|Monday|2011Q2|N|N|N|2455653|2455742|2455305|2455580|N|N|N|N|N| +2455671|AAAAAAAAHHIHFCAA|2011-04-19|1335|5808|446|2011|2|4|19|2|2011|446|5808|Tuesday|2011Q2|N|N|N|2455653|2455742|2455306|2455581|N|N|N|N|N| +2455672|AAAAAAAAIHIHFCAA|2011-04-20|1335|5808|446|2011|3|4|20|2|2011|446|5808|Wednesday|2011Q2|N|N|N|2455653|2455742|2455307|2455582|N|N|N|N|N| +2455673|AAAAAAAAJHIHFCAA|2011-04-21|1335|5808|446|2011|4|4|21|2|2011|446|5808|Thursday|2011Q2|N|N|N|2455653|2455742|2455308|2455583|N|N|N|N|N| +2455674|AAAAAAAAKHIHFCAA|2011-04-22|1335|5808|446|2011|5|4|22|2|2011|446|5808|Friday|2011Q2|N|Y|N|2455653|2455742|2455309|2455584|N|N|N|N|N| +2455675|AAAAAAAALHIHFCAA|2011-04-23|1335|5808|446|2011|6|4|23|2|2011|446|5808|Saturday|2011Q2|N|Y|N|2455653|2455742|2455310|2455585|N|N|N|N|N| +2455676|AAAAAAAAMHIHFCAA|2011-04-24|1335|5808|446|2011|0|4|24|2|2011|446|5808|Sunday|2011Q2|N|N|N|2455653|2455742|2455311|2455586|N|N|N|N|N| +2455677|AAAAAAAANHIHFCAA|2011-04-25|1335|5808|446|2011|1|4|25|2|2011|446|5808|Monday|2011Q2|N|N|N|2455653|2455742|2455312|2455587|N|N|N|N|N| +2455678|AAAAAAAAOHIHFCAA|2011-04-26|1335|5809|446|2011|2|4|26|2|2011|446|5809|Tuesday|2011Q2|N|N|N|2455653|2455742|2455313|2455588|N|N|N|N|N| +2455679|AAAAAAAAPHIHFCAA|2011-04-27|1335|5809|446|2011|3|4|27|2|2011|446|5809|Wednesday|2011Q2|N|N|N|2455653|2455742|2455314|2455589|N|N|N|N|N| +2455680|AAAAAAAAAIIHFCAA|2011-04-28|1335|5809|446|2011|4|4|28|2|2011|446|5809|Thursday|2011Q2|N|N|N|2455653|2455742|2455315|2455590|N|N|N|N|N| +2455681|AAAAAAAABIIHFCAA|2011-04-29|1335|5809|446|2011|5|4|29|2|2011|446|5809|Friday|2011Q2|N|Y|N|2455653|2455742|2455316|2455591|N|N|N|N|N| +2455682|AAAAAAAACIIHFCAA|2011-04-30|1335|5809|446|2011|6|4|30|2|2011|446|5809|Saturday|2011Q2|N|Y|N|2455653|2455742|2455317|2455592|N|N|N|N|N| +2455683|AAAAAAAADIIHFCAA|2011-05-01|1336|5809|446|2011|0|5|1|2|2011|446|5809|Sunday|2011Q2|N|N|N|2455683|2455802|2455318|2455593|N|N|N|N|N| +2455684|AAAAAAAAEIIHFCAA|2011-05-02|1336|5809|446|2011|1|5|2|2|2011|446|5809|Monday|2011Q2|N|N|N|2455683|2455802|2455319|2455594|N|N|N|N|N| +2455685|AAAAAAAAFIIHFCAA|2011-05-03|1336|5810|446|2011|2|5|3|2|2011|446|5810|Tuesday|2011Q2|N|N|N|2455683|2455802|2455320|2455595|N|N|N|N|N| +2455686|AAAAAAAAGIIHFCAA|2011-05-04|1336|5810|446|2011|3|5|4|2|2011|446|5810|Wednesday|2011Q2|N|N|N|2455683|2455802|2455321|2455596|N|N|N|N|N| +2455687|AAAAAAAAHIIHFCAA|2011-05-05|1336|5810|446|2011|4|5|5|2|2011|446|5810|Thursday|2011Q2|N|N|N|2455683|2455802|2455322|2455597|N|N|N|N|N| +2455688|AAAAAAAAIIIHFCAA|2011-05-06|1336|5810|446|2011|5|5|6|2|2011|446|5810|Friday|2011Q2|N|Y|N|2455683|2455802|2455323|2455598|N|N|N|N|N| +2455689|AAAAAAAAJIIHFCAA|2011-05-07|1336|5810|446|2011|6|5|7|2|2011|446|5810|Saturday|2011Q2|N|Y|N|2455683|2455802|2455324|2455599|N|N|N|N|N| +2455690|AAAAAAAAKIIHFCAA|2011-05-08|1336|5810|446|2011|0|5|8|2|2011|446|5810|Sunday|2011Q2|N|N|N|2455683|2455802|2455325|2455600|N|N|N|N|N| +2455691|AAAAAAAALIIHFCAA|2011-05-09|1336|5810|446|2011|1|5|9|2|2011|446|5810|Monday|2011Q2|N|N|N|2455683|2455802|2455326|2455601|N|N|N|N|N| +2455692|AAAAAAAAMIIHFCAA|2011-05-10|1336|5811|446|2011|2|5|10|2|2011|446|5811|Tuesday|2011Q2|N|N|N|2455683|2455802|2455327|2455602|N|N|N|N|N| +2455693|AAAAAAAANIIHFCAA|2011-05-11|1336|5811|446|2011|3|5|11|2|2011|446|5811|Wednesday|2011Q2|N|N|N|2455683|2455802|2455328|2455603|N|N|N|N|N| +2455694|AAAAAAAAOIIHFCAA|2011-05-12|1336|5811|446|2011|4|5|12|2|2011|446|5811|Thursday|2011Q2|N|N|N|2455683|2455802|2455329|2455604|N|N|N|N|N| +2455695|AAAAAAAAPIIHFCAA|2011-05-13|1336|5811|446|2011|5|5|13|2|2011|446|5811|Friday|2011Q2|N|Y|N|2455683|2455802|2455330|2455605|N|N|N|N|N| +2455696|AAAAAAAAAJIHFCAA|2011-05-14|1336|5811|446|2011|6|5|14|2|2011|446|5811|Saturday|2011Q2|N|Y|N|2455683|2455802|2455331|2455606|N|N|N|N|N| +2455697|AAAAAAAABJIHFCAA|2011-05-15|1336|5811|446|2011|0|5|15|2|2011|446|5811|Sunday|2011Q2|N|N|N|2455683|2455802|2455332|2455607|N|N|N|N|N| +2455698|AAAAAAAACJIHFCAA|2011-05-16|1336|5811|446|2011|1|5|16|2|2011|446|5811|Monday|2011Q2|N|N|N|2455683|2455802|2455333|2455608|N|N|N|N|N| +2455699|AAAAAAAADJIHFCAA|2011-05-17|1336|5812|446|2011|2|5|17|2|2011|446|5812|Tuesday|2011Q2|N|N|N|2455683|2455802|2455334|2455609|N|N|N|N|N| +2455700|AAAAAAAAEJIHFCAA|2011-05-18|1336|5812|446|2011|3|5|18|2|2011|446|5812|Wednesday|2011Q2|N|N|N|2455683|2455802|2455335|2455610|N|N|N|N|N| +2455701|AAAAAAAAFJIHFCAA|2011-05-19|1336|5812|446|2011|4|5|19|2|2011|446|5812|Thursday|2011Q2|N|N|N|2455683|2455802|2455336|2455611|N|N|N|N|N| +2455702|AAAAAAAAGJIHFCAA|2011-05-20|1336|5812|446|2011|5|5|20|2|2011|446|5812|Friday|2011Q2|N|Y|N|2455683|2455802|2455337|2455612|N|N|N|N|N| +2455703|AAAAAAAAHJIHFCAA|2011-05-21|1336|5812|446|2011|6|5|21|2|2011|446|5812|Saturday|2011Q2|N|Y|N|2455683|2455802|2455338|2455613|N|N|N|N|N| +2455704|AAAAAAAAIJIHFCAA|2011-05-22|1336|5812|446|2011|0|5|22|2|2011|446|5812|Sunday|2011Q2|N|N|N|2455683|2455802|2455339|2455614|N|N|N|N|N| +2455705|AAAAAAAAJJIHFCAA|2011-05-23|1336|5812|446|2011|1|5|23|2|2011|446|5812|Monday|2011Q2|N|N|N|2455683|2455802|2455340|2455615|N|N|N|N|N| +2455706|AAAAAAAAKJIHFCAA|2011-05-24|1336|5813|446|2011|2|5|24|2|2011|446|5813|Tuesday|2011Q2|N|N|N|2455683|2455802|2455341|2455616|N|N|N|N|N| +2455707|AAAAAAAALJIHFCAA|2011-05-25|1336|5813|446|2011|3|5|25|2|2011|446|5813|Wednesday|2011Q2|N|N|N|2455683|2455802|2455342|2455617|N|N|N|N|N| +2455708|AAAAAAAAMJIHFCAA|2011-05-26|1336|5813|446|2011|4|5|26|2|2011|446|5813|Thursday|2011Q2|N|N|N|2455683|2455802|2455343|2455618|N|N|N|N|N| +2455709|AAAAAAAANJIHFCAA|2011-05-27|1336|5813|446|2011|5|5|27|2|2011|446|5813|Friday|2011Q2|N|Y|N|2455683|2455802|2455344|2455619|N|N|N|N|N| +2455710|AAAAAAAAOJIHFCAA|2011-05-28|1336|5813|446|2011|6|5|28|2|2011|446|5813|Saturday|2011Q2|N|Y|N|2455683|2455802|2455345|2455620|N|N|N|N|N| +2455711|AAAAAAAAPJIHFCAA|2011-05-29|1336|5813|446|2011|0|5|29|2|2011|446|5813|Sunday|2011Q2|N|N|N|2455683|2455802|2455346|2455621|N|N|N|N|N| +2455712|AAAAAAAAAKIHFCAA|2011-05-30|1336|5813|446|2011|1|5|30|2|2011|446|5813|Monday|2011Q2|N|N|N|2455683|2455802|2455347|2455622|N|N|N|N|N| +2455713|AAAAAAAABKIHFCAA|2011-05-31|1336|5814|446|2011|2|5|31|2|2011|446|5814|Tuesday|2011Q2|N|N|N|2455683|2455802|2455348|2455623|N|N|N|N|N| +2455714|AAAAAAAACKIHFCAA|2011-06-01|1337|5814|447|2011|3|6|1|2|2011|447|5814|Wednesday|2011Q2|N|N|N|2455714|2455864|2455349|2455624|N|N|N|N|N| +2455715|AAAAAAAADKIHFCAA|2011-06-02|1337|5814|447|2011|4|6|2|2|2011|447|5814|Thursday|2011Q2|N|N|N|2455714|2455864|2455350|2455625|N|N|N|N|N| +2455716|AAAAAAAAEKIHFCAA|2011-06-03|1337|5814|447|2011|5|6|3|2|2011|447|5814|Friday|2011Q2|N|Y|N|2455714|2455864|2455351|2455626|N|N|N|N|N| +2455717|AAAAAAAAFKIHFCAA|2011-06-04|1337|5814|447|2011|6|6|4|2|2011|447|5814|Saturday|2011Q2|N|Y|N|2455714|2455864|2455352|2455627|N|N|N|N|N| +2455718|AAAAAAAAGKIHFCAA|2011-06-05|1337|5814|447|2011|0|6|5|2|2011|447|5814|Sunday|2011Q2|N|N|N|2455714|2455864|2455353|2455628|N|N|N|N|N| +2455719|AAAAAAAAHKIHFCAA|2011-06-06|1337|5814|447|2011|1|6|6|2|2011|447|5814|Monday|2011Q2|N|N|N|2455714|2455864|2455354|2455629|N|N|N|N|N| +2455720|AAAAAAAAIKIHFCAA|2011-06-07|1337|5815|447|2011|2|6|7|2|2011|447|5815|Tuesday|2011Q2|N|N|N|2455714|2455864|2455355|2455630|N|N|N|N|N| +2455721|AAAAAAAAJKIHFCAA|2011-06-08|1337|5815|447|2011|3|6|8|2|2011|447|5815|Wednesday|2011Q2|N|N|N|2455714|2455864|2455356|2455631|N|N|N|N|N| +2455722|AAAAAAAAKKIHFCAA|2011-06-09|1337|5815|447|2011|4|6|9|2|2011|447|5815|Thursday|2011Q2|N|N|N|2455714|2455864|2455357|2455632|N|N|N|N|N| +2455723|AAAAAAAALKIHFCAA|2011-06-10|1337|5815|447|2011|5|6|10|2|2011|447|5815|Friday|2011Q2|N|Y|N|2455714|2455864|2455358|2455633|N|N|N|N|N| +2455724|AAAAAAAAMKIHFCAA|2011-06-11|1337|5815|447|2011|6|6|11|2|2011|447|5815|Saturday|2011Q2|N|Y|N|2455714|2455864|2455359|2455634|N|N|N|N|N| +2455725|AAAAAAAANKIHFCAA|2011-06-12|1337|5815|447|2011|0|6|12|2|2011|447|5815|Sunday|2011Q2|N|N|N|2455714|2455864|2455360|2455635|N|N|N|N|N| +2455726|AAAAAAAAOKIHFCAA|2011-06-13|1337|5815|447|2011|1|6|13|2|2011|447|5815|Monday|2011Q2|N|N|N|2455714|2455864|2455361|2455636|N|N|N|N|N| +2455727|AAAAAAAAPKIHFCAA|2011-06-14|1337|5816|447|2011|2|6|14|2|2011|447|5816|Tuesday|2011Q2|N|N|N|2455714|2455864|2455362|2455637|N|N|N|N|N| +2455728|AAAAAAAAALIHFCAA|2011-06-15|1337|5816|447|2011|3|6|15|2|2011|447|5816|Wednesday|2011Q2|N|N|N|2455714|2455864|2455363|2455638|N|N|N|N|N| +2455729|AAAAAAAABLIHFCAA|2011-06-16|1337|5816|447|2011|4|6|16|2|2011|447|5816|Thursday|2011Q2|N|N|N|2455714|2455864|2455364|2455639|N|N|N|N|N| +2455730|AAAAAAAACLIHFCAA|2011-06-17|1337|5816|447|2011|5|6|17|2|2011|447|5816|Friday|2011Q2|N|Y|N|2455714|2455864|2455365|2455640|N|N|N|N|N| +2455731|AAAAAAAADLIHFCAA|2011-06-18|1337|5816|447|2011|6|6|18|2|2011|447|5816|Saturday|2011Q2|N|Y|N|2455714|2455864|2455366|2455641|N|N|N|N|N| +2455732|AAAAAAAAELIHFCAA|2011-06-19|1337|5816|447|2011|0|6|19|2|2011|447|5816|Sunday|2011Q2|N|N|N|2455714|2455864|2455367|2455642|N|N|N|N|N| +2455733|AAAAAAAAFLIHFCAA|2011-06-20|1337|5816|447|2011|1|6|20|2|2011|447|5816|Monday|2011Q2|N|N|N|2455714|2455864|2455368|2455643|N|N|N|N|N| +2455734|AAAAAAAAGLIHFCAA|2011-06-21|1337|5817|447|2011|2|6|21|2|2011|447|5817|Tuesday|2011Q2|N|N|N|2455714|2455864|2455369|2455644|N|N|N|N|N| +2455735|AAAAAAAAHLIHFCAA|2011-06-22|1337|5817|447|2011|3|6|22|2|2011|447|5817|Wednesday|2011Q2|N|N|N|2455714|2455864|2455370|2455645|N|N|N|N|N| +2455736|AAAAAAAAILIHFCAA|2011-06-23|1337|5817|447|2011|4|6|23|2|2011|447|5817|Thursday|2011Q2|N|N|N|2455714|2455864|2455371|2455646|N|N|N|N|N| +2455737|AAAAAAAAJLIHFCAA|2011-06-24|1337|5817|447|2011|5|6|24|2|2011|447|5817|Friday|2011Q2|N|Y|N|2455714|2455864|2455372|2455647|N|N|N|N|N| +2455738|AAAAAAAAKLIHFCAA|2011-06-25|1337|5817|447|2011|6|6|25|2|2011|447|5817|Saturday|2011Q2|N|Y|N|2455714|2455864|2455373|2455648|N|N|N|N|N| +2455739|AAAAAAAALLIHFCAA|2011-06-26|1337|5817|447|2011|0|6|26|2|2011|447|5817|Sunday|2011Q2|N|N|N|2455714|2455864|2455374|2455649|N|N|N|N|N| +2455740|AAAAAAAAMLIHFCAA|2011-06-27|1337|5817|447|2011|1|6|27|2|2011|447|5817|Monday|2011Q2|N|N|N|2455714|2455864|2455375|2455650|N|N|N|N|N| +2455741|AAAAAAAANLIHFCAA|2011-06-28|1337|5818|447|2011|2|6|28|2|2011|447|5818|Tuesday|2011Q2|N|N|N|2455714|2455864|2455376|2455651|N|N|N|N|N| +2455742|AAAAAAAAOLIHFCAA|2011-06-29|1337|5818|447|2011|3|6|29|2|2011|447|5818|Wednesday|2011Q2|N|N|N|2455714|2455864|2455377|2455652|N|N|N|N|N| +2455743|AAAAAAAAPLIHFCAA|2011-06-30|1337|5818|447|2011|4|6|30|2|2011|447|5818|Thursday|2011Q2|N|N|N|2455714|2455864|2455378|2455653|N|N|N|N|N| +2455744|AAAAAAAAAMIHFCAA|2011-07-01|1338|5818|447|2011|5|7|1|2|2011|447|5818|Friday|2011Q2|N|Y|N|2455744|2455924|2455379|2455653|N|N|N|N|N| +2455745|AAAAAAAABMIHFCAA|2011-07-02|1338|5818|447|2011|6|7|2|3|2011|447|5818|Saturday|2011Q3|N|Y|N|2455744|2455924|2455380|2455654|N|N|N|N|N| +2455746|AAAAAAAACMIHFCAA|2011-07-03|1338|5818|447|2011|0|7|3|3|2011|447|5818|Sunday|2011Q3|N|N|N|2455744|2455924|2455381|2455655|N|N|N|N|N| +2455747|AAAAAAAADMIHFCAA|2011-07-04|1338|5818|447|2011|1|7|4|3|2011|447|5818|Monday|2011Q3|N|N|N|2455744|2455924|2455382|2455656|N|N|N|N|N| +2455748|AAAAAAAAEMIHFCAA|2011-07-05|1338|5819|447|2011|2|7|5|3|2011|447|5819|Tuesday|2011Q3|Y|N|N|2455744|2455924|2455383|2455657|N|N|N|N|N| +2455749|AAAAAAAAFMIHFCAA|2011-07-06|1338|5819|447|2011|3|7|6|3|2011|447|5819|Wednesday|2011Q3|N|N|Y|2455744|2455924|2455384|2455658|N|N|N|N|N| +2455750|AAAAAAAAGMIHFCAA|2011-07-07|1338|5819|447|2011|4|7|7|3|2011|447|5819|Thursday|2011Q3|N|N|N|2455744|2455924|2455385|2455659|N|N|N|N|N| +2455751|AAAAAAAAHMIHFCAA|2011-07-08|1338|5819|447|2011|5|7|8|3|2011|447|5819|Friday|2011Q3|N|Y|N|2455744|2455924|2455386|2455660|N|N|N|N|N| +2455752|AAAAAAAAIMIHFCAA|2011-07-09|1338|5819|447|2011|6|7|9|3|2011|447|5819|Saturday|2011Q3|N|Y|N|2455744|2455924|2455387|2455661|N|N|N|N|N| +2455753|AAAAAAAAJMIHFCAA|2011-07-10|1338|5819|447|2011|0|7|10|3|2011|447|5819|Sunday|2011Q3|N|N|N|2455744|2455924|2455388|2455662|N|N|N|N|N| +2455754|AAAAAAAAKMIHFCAA|2011-07-11|1338|5819|447|2011|1|7|11|3|2011|447|5819|Monday|2011Q3|N|N|N|2455744|2455924|2455389|2455663|N|N|N|N|N| +2455755|AAAAAAAALMIHFCAA|2011-07-12|1338|5820|447|2011|2|7|12|3|2011|447|5820|Tuesday|2011Q3|N|N|N|2455744|2455924|2455390|2455664|N|N|N|N|N| +2455756|AAAAAAAAMMIHFCAA|2011-07-13|1338|5820|447|2011|3|7|13|3|2011|447|5820|Wednesday|2011Q3|N|N|N|2455744|2455924|2455391|2455665|N|N|N|N|N| +2455757|AAAAAAAANMIHFCAA|2011-07-14|1338|5820|447|2011|4|7|14|3|2011|447|5820|Thursday|2011Q3|N|N|N|2455744|2455924|2455392|2455666|N|N|N|N|N| +2455758|AAAAAAAAOMIHFCAA|2011-07-15|1338|5820|447|2011|5|7|15|3|2011|447|5820|Friday|2011Q3|N|Y|N|2455744|2455924|2455393|2455667|N|N|N|N|N| +2455759|AAAAAAAAPMIHFCAA|2011-07-16|1338|5820|447|2011|6|7|16|3|2011|447|5820|Saturday|2011Q3|N|Y|N|2455744|2455924|2455394|2455668|N|N|N|N|N| +2455760|AAAAAAAAANIHFCAA|2011-07-17|1338|5820|447|2011|0|7|17|3|2011|447|5820|Sunday|2011Q3|N|N|N|2455744|2455924|2455395|2455669|N|N|N|N|N| +2455761|AAAAAAAABNIHFCAA|2011-07-18|1338|5820|447|2011|1|7|18|3|2011|447|5820|Monday|2011Q3|N|N|N|2455744|2455924|2455396|2455670|N|N|N|N|N| +2455762|AAAAAAAACNIHFCAA|2011-07-19|1338|5821|447|2011|2|7|19|3|2011|447|5821|Tuesday|2011Q3|N|N|N|2455744|2455924|2455397|2455671|N|N|N|N|N| +2455763|AAAAAAAADNIHFCAA|2011-07-20|1338|5821|447|2011|3|7|20|3|2011|447|5821|Wednesday|2011Q3|N|N|N|2455744|2455924|2455398|2455672|N|N|N|N|N| +2455764|AAAAAAAAENIHFCAA|2011-07-21|1338|5821|447|2011|4|7|21|3|2011|447|5821|Thursday|2011Q3|N|N|N|2455744|2455924|2455399|2455673|N|N|N|N|N| +2455765|AAAAAAAAFNIHFCAA|2011-07-22|1338|5821|447|2011|5|7|22|3|2011|447|5821|Friday|2011Q3|N|Y|N|2455744|2455924|2455400|2455674|N|N|N|N|N| +2455766|AAAAAAAAGNIHFCAA|2011-07-23|1338|5821|447|2011|6|7|23|3|2011|447|5821|Saturday|2011Q3|N|Y|N|2455744|2455924|2455401|2455675|N|N|N|N|N| +2455767|AAAAAAAAHNIHFCAA|2011-07-24|1338|5821|447|2011|0|7|24|3|2011|447|5821|Sunday|2011Q3|N|N|N|2455744|2455924|2455402|2455676|N|N|N|N|N| +2455768|AAAAAAAAINIHFCAA|2011-07-25|1338|5821|447|2011|1|7|25|3|2011|447|5821|Monday|2011Q3|N|N|N|2455744|2455924|2455403|2455677|N|N|N|N|N| +2455769|AAAAAAAAJNIHFCAA|2011-07-26|1338|5822|447|2011|2|7|26|3|2011|447|5822|Tuesday|2011Q3|N|N|N|2455744|2455924|2455404|2455678|N|N|N|N|N| +2455770|AAAAAAAAKNIHFCAA|2011-07-27|1338|5822|447|2011|3|7|27|3|2011|447|5822|Wednesday|2011Q3|N|N|N|2455744|2455924|2455405|2455679|N|N|N|N|N| +2455771|AAAAAAAALNIHFCAA|2011-07-28|1338|5822|447|2011|4|7|28|3|2011|447|5822|Thursday|2011Q3|N|N|N|2455744|2455924|2455406|2455680|N|N|N|N|N| +2455772|AAAAAAAAMNIHFCAA|2011-07-29|1338|5822|447|2011|5|7|29|3|2011|447|5822|Friday|2011Q3|N|Y|N|2455744|2455924|2455407|2455681|N|N|N|N|N| +2455773|AAAAAAAANNIHFCAA|2011-07-30|1338|5822|447|2011|6|7|30|3|2011|447|5822|Saturday|2011Q3|N|Y|N|2455744|2455924|2455408|2455682|N|N|N|N|N| +2455774|AAAAAAAAONIHFCAA|2011-07-31|1338|5822|447|2011|0|7|31|3|2011|447|5822|Sunday|2011Q3|N|N|N|2455744|2455924|2455409|2455683|N|N|N|N|N| +2455775|AAAAAAAAPNIHFCAA|2011-08-01|1339|5822|447|2011|1|8|1|3|2011|447|5822|Monday|2011Q3|N|N|N|2455775|2455986|2455410|2455684|N|N|N|N|N| +2455776|AAAAAAAAAOIHFCAA|2011-08-02|1339|5823|447|2011|2|8|2|3|2011|447|5823|Tuesday|2011Q3|N|N|N|2455775|2455986|2455411|2455685|N|N|N|N|N| +2455777|AAAAAAAABOIHFCAA|2011-08-03|1339|5823|447|2011|3|8|3|3|2011|447|5823|Wednesday|2011Q3|N|N|N|2455775|2455986|2455412|2455686|N|N|N|N|N| +2455778|AAAAAAAACOIHFCAA|2011-08-04|1339|5823|447|2011|4|8|4|3|2011|447|5823|Thursday|2011Q3|N|N|N|2455775|2455986|2455413|2455687|N|N|N|N|N| +2455779|AAAAAAAADOIHFCAA|2011-08-05|1339|5823|447|2011|5|8|5|3|2011|447|5823|Friday|2011Q3|N|Y|N|2455775|2455986|2455414|2455688|N|N|N|N|N| +2455780|AAAAAAAAEOIHFCAA|2011-08-06|1339|5823|447|2011|6|8|6|3|2011|447|5823|Saturday|2011Q3|N|Y|N|2455775|2455986|2455415|2455689|N|N|N|N|N| +2455781|AAAAAAAAFOIHFCAA|2011-08-07|1339|5823|447|2011|0|8|7|3|2011|447|5823|Sunday|2011Q3|N|N|N|2455775|2455986|2455416|2455690|N|N|N|N|N| +2455782|AAAAAAAAGOIHFCAA|2011-08-08|1339|5823|447|2011|1|8|8|3|2011|447|5823|Monday|2011Q3|N|N|N|2455775|2455986|2455417|2455691|N|N|N|N|N| +2455783|AAAAAAAAHOIHFCAA|2011-08-09|1339|5824|447|2011|2|8|9|3|2011|447|5824|Tuesday|2011Q3|N|N|N|2455775|2455986|2455418|2455692|N|N|N|N|N| +2455784|AAAAAAAAIOIHFCAA|2011-08-10|1339|5824|447|2011|3|8|10|3|2011|447|5824|Wednesday|2011Q3|N|N|N|2455775|2455986|2455419|2455693|N|N|N|N|N| +2455785|AAAAAAAAJOIHFCAA|2011-08-11|1339|5824|447|2011|4|8|11|3|2011|447|5824|Thursday|2011Q3|N|N|N|2455775|2455986|2455420|2455694|N|N|N|N|N| +2455786|AAAAAAAAKOIHFCAA|2011-08-12|1339|5824|447|2011|5|8|12|3|2011|447|5824|Friday|2011Q3|N|Y|N|2455775|2455986|2455421|2455695|N|N|N|N|N| +2455787|AAAAAAAALOIHFCAA|2011-08-13|1339|5824|447|2011|6|8|13|3|2011|447|5824|Saturday|2011Q3|N|Y|N|2455775|2455986|2455422|2455696|N|N|N|N|N| +2455788|AAAAAAAAMOIHFCAA|2011-08-14|1339|5824|447|2011|0|8|14|3|2011|447|5824|Sunday|2011Q3|N|N|N|2455775|2455986|2455423|2455697|N|N|N|N|N| +2455789|AAAAAAAANOIHFCAA|2011-08-15|1339|5824|447|2011|1|8|15|3|2011|447|5824|Monday|2011Q3|N|N|N|2455775|2455986|2455424|2455698|N|N|N|N|N| +2455790|AAAAAAAAOOIHFCAA|2011-08-16|1339|5825|447|2011|2|8|16|3|2011|447|5825|Tuesday|2011Q3|N|N|N|2455775|2455986|2455425|2455699|N|N|N|N|N| +2455791|AAAAAAAAPOIHFCAA|2011-08-17|1339|5825|447|2011|3|8|17|3|2011|447|5825|Wednesday|2011Q3|N|N|N|2455775|2455986|2455426|2455700|N|N|N|N|N| +2455792|AAAAAAAAAPIHFCAA|2011-08-18|1339|5825|447|2011|4|8|18|3|2011|447|5825|Thursday|2011Q3|N|N|N|2455775|2455986|2455427|2455701|N|N|N|N|N| +2455793|AAAAAAAABPIHFCAA|2011-08-19|1339|5825|447|2011|5|8|19|3|2011|447|5825|Friday|2011Q3|N|Y|N|2455775|2455986|2455428|2455702|N|N|N|N|N| +2455794|AAAAAAAACPIHFCAA|2011-08-20|1339|5825|447|2011|6|8|20|3|2011|447|5825|Saturday|2011Q3|N|Y|N|2455775|2455986|2455429|2455703|N|N|N|N|N| +2455795|AAAAAAAADPIHFCAA|2011-08-21|1339|5825|447|2011|0|8|21|3|2011|447|5825|Sunday|2011Q3|N|N|N|2455775|2455986|2455430|2455704|N|N|N|N|N| +2455796|AAAAAAAAEPIHFCAA|2011-08-22|1339|5825|447|2011|1|8|22|3|2011|447|5825|Monday|2011Q3|N|N|N|2455775|2455986|2455431|2455705|N|N|N|N|N| +2455797|AAAAAAAAFPIHFCAA|2011-08-23|1339|5826|447|2011|2|8|23|3|2011|447|5826|Tuesday|2011Q3|N|N|N|2455775|2455986|2455432|2455706|N|N|N|N|N| +2455798|AAAAAAAAGPIHFCAA|2011-08-24|1339|5826|447|2011|3|8|24|3|2011|447|5826|Wednesday|2011Q3|N|N|N|2455775|2455986|2455433|2455707|N|N|N|N|N| +2455799|AAAAAAAAHPIHFCAA|2011-08-25|1339|5826|447|2011|4|8|25|3|2011|447|5826|Thursday|2011Q3|N|N|N|2455775|2455986|2455434|2455708|N|N|N|N|N| +2455800|AAAAAAAAIPIHFCAA|2011-08-26|1339|5826|447|2011|5|8|26|3|2011|447|5826|Friday|2011Q3|N|Y|N|2455775|2455986|2455435|2455709|N|N|N|N|N| +2455801|AAAAAAAAJPIHFCAA|2011-08-27|1339|5826|447|2011|6|8|27|3|2011|447|5826|Saturday|2011Q3|N|Y|N|2455775|2455986|2455436|2455710|N|N|N|N|N| +2455802|AAAAAAAAKPIHFCAA|2011-08-28|1339|5826|447|2011|0|8|28|3|2011|447|5826|Sunday|2011Q3|N|N|N|2455775|2455986|2455437|2455711|N|N|N|N|N| +2455803|AAAAAAAALPIHFCAA|2011-08-29|1339|5826|447|2011|1|8|29|3|2011|447|5826|Monday|2011Q3|N|N|N|2455775|2455986|2455438|2455712|N|N|N|N|N| +2455804|AAAAAAAAMPIHFCAA|2011-08-30|1339|5827|447|2011|2|8|30|3|2011|447|5827|Tuesday|2011Q3|N|N|N|2455775|2455986|2455439|2455713|N|N|N|N|N| +2455805|AAAAAAAANPIHFCAA|2011-08-31|1339|5827|447|2011|3|8|31|3|2011|447|5827|Wednesday|2011Q3|N|N|N|2455775|2455986|2455440|2455714|N|N|N|N|N| +2455806|AAAAAAAAOPIHFCAA|2011-09-01|1340|5827|448|2011|4|9|1|3|2011|448|5827|Thursday|2011Q3|N|N|N|2455806|2456048|2455441|2455715|N|N|N|N|N| +2455807|AAAAAAAAPPIHFCAA|2011-09-02|1340|5827|448|2011|5|9|2|3|2011|448|5827|Friday|2011Q3|N|Y|N|2455806|2456048|2455442|2455716|N|N|N|N|N| +2455808|AAAAAAAAAAJHFCAA|2011-09-03|1340|5827|448|2011|6|9|3|3|2011|448|5827|Saturday|2011Q3|N|Y|N|2455806|2456048|2455443|2455717|N|N|N|N|N| +2455809|AAAAAAAABAJHFCAA|2011-09-04|1340|5827|448|2011|0|9|4|3|2011|448|5827|Sunday|2011Q3|N|N|N|2455806|2456048|2455444|2455718|N|N|N|N|N| +2455810|AAAAAAAACAJHFCAA|2011-09-05|1340|5827|448|2011|1|9|5|3|2011|448|5827|Monday|2011Q3|N|N|N|2455806|2456048|2455445|2455719|N|N|N|N|N| +2455811|AAAAAAAADAJHFCAA|2011-09-06|1340|5828|448|2011|2|9|6|3|2011|448|5828|Tuesday|2011Q3|N|N|N|2455806|2456048|2455446|2455720|N|N|N|N|N| +2455812|AAAAAAAAEAJHFCAA|2011-09-07|1340|5828|448|2011|3|9|7|3|2011|448|5828|Wednesday|2011Q3|N|N|N|2455806|2456048|2455447|2455721|N|N|N|N|N| +2455813|AAAAAAAAFAJHFCAA|2011-09-08|1340|5828|448|2011|4|9|8|3|2011|448|5828|Thursday|2011Q3|N|N|N|2455806|2456048|2455448|2455722|N|N|N|N|N| +2455814|AAAAAAAAGAJHFCAA|2011-09-09|1340|5828|448|2011|5|9|9|3|2011|448|5828|Friday|2011Q3|N|Y|N|2455806|2456048|2455449|2455723|N|N|N|N|N| +2455815|AAAAAAAAHAJHFCAA|2011-09-10|1340|5828|448|2011|6|9|10|3|2011|448|5828|Saturday|2011Q3|N|Y|N|2455806|2456048|2455450|2455724|N|N|N|N|N| +2455816|AAAAAAAAIAJHFCAA|2011-09-11|1340|5828|448|2011|0|9|11|3|2011|448|5828|Sunday|2011Q3|N|N|N|2455806|2456048|2455451|2455725|N|N|N|N|N| +2455817|AAAAAAAAJAJHFCAA|2011-09-12|1340|5828|448|2011|1|9|12|3|2011|448|5828|Monday|2011Q3|N|N|N|2455806|2456048|2455452|2455726|N|N|N|N|N| +2455818|AAAAAAAAKAJHFCAA|2011-09-13|1340|5829|448|2011|2|9|13|3|2011|448|5829|Tuesday|2011Q3|N|N|N|2455806|2456048|2455453|2455727|N|N|N|N|N| +2455819|AAAAAAAALAJHFCAA|2011-09-14|1340|5829|448|2011|3|9|14|3|2011|448|5829|Wednesday|2011Q3|N|N|N|2455806|2456048|2455454|2455728|N|N|N|N|N| +2455820|AAAAAAAAMAJHFCAA|2011-09-15|1340|5829|448|2011|4|9|15|3|2011|448|5829|Thursday|2011Q3|N|N|N|2455806|2456048|2455455|2455729|N|N|N|N|N| +2455821|AAAAAAAANAJHFCAA|2011-09-16|1340|5829|448|2011|5|9|16|3|2011|448|5829|Friday|2011Q3|N|Y|N|2455806|2456048|2455456|2455730|N|N|N|N|N| +2455822|AAAAAAAAOAJHFCAA|2011-09-17|1340|5829|448|2011|6|9|17|3|2011|448|5829|Saturday|2011Q3|N|Y|N|2455806|2456048|2455457|2455731|N|N|N|N|N| +2455823|AAAAAAAAPAJHFCAA|2011-09-18|1340|5829|448|2011|0|9|18|3|2011|448|5829|Sunday|2011Q3|N|N|N|2455806|2456048|2455458|2455732|N|N|N|N|N| +2455824|AAAAAAAAABJHFCAA|2011-09-19|1340|5829|448|2011|1|9|19|3|2011|448|5829|Monday|2011Q3|N|N|N|2455806|2456048|2455459|2455733|N|N|N|N|N| +2455825|AAAAAAAABBJHFCAA|2011-09-20|1340|5830|448|2011|2|9|20|3|2011|448|5830|Tuesday|2011Q3|N|N|N|2455806|2456048|2455460|2455734|N|N|N|N|N| +2455826|AAAAAAAACBJHFCAA|2011-09-21|1340|5830|448|2011|3|9|21|3|2011|448|5830|Wednesday|2011Q3|N|N|N|2455806|2456048|2455461|2455735|N|N|N|N|N| +2455827|AAAAAAAADBJHFCAA|2011-09-22|1340|5830|448|2011|4|9|22|3|2011|448|5830|Thursday|2011Q3|N|N|N|2455806|2456048|2455462|2455736|N|N|N|N|N| +2455828|AAAAAAAAEBJHFCAA|2011-09-23|1340|5830|448|2011|5|9|23|3|2011|448|5830|Friday|2011Q3|N|Y|N|2455806|2456048|2455463|2455737|N|N|N|N|N| +2455829|AAAAAAAAFBJHFCAA|2011-09-24|1340|5830|448|2011|6|9|24|3|2011|448|5830|Saturday|2011Q3|N|Y|N|2455806|2456048|2455464|2455738|N|N|N|N|N| +2455830|AAAAAAAAGBJHFCAA|2011-09-25|1340|5830|448|2011|0|9|25|3|2011|448|5830|Sunday|2011Q3|N|N|N|2455806|2456048|2455465|2455739|N|N|N|N|N| +2455831|AAAAAAAAHBJHFCAA|2011-09-26|1340|5830|448|2011|1|9|26|3|2011|448|5830|Monday|2011Q3|N|N|N|2455806|2456048|2455466|2455740|N|N|N|N|N| +2455832|AAAAAAAAIBJHFCAA|2011-09-27|1340|5831|448|2011|2|9|27|3|2011|448|5831|Tuesday|2011Q3|N|N|N|2455806|2456048|2455467|2455741|N|N|N|N|N| +2455833|AAAAAAAAJBJHFCAA|2011-09-28|1340|5831|448|2011|3|9|28|3|2011|448|5831|Wednesday|2011Q3|N|N|N|2455806|2456048|2455468|2455742|N|N|N|N|N| +2455834|AAAAAAAAKBJHFCAA|2011-09-29|1340|5831|448|2011|4|9|29|3|2011|448|5831|Thursday|2011Q3|N|N|N|2455806|2456048|2455469|2455743|N|N|N|N|N| +2455835|AAAAAAAALBJHFCAA|2011-09-30|1340|5831|448|2011|5|9|30|3|2011|448|5831|Friday|2011Q3|N|Y|N|2455806|2456048|2455470|2455744|N|N|N|N|N| +2455836|AAAAAAAAMBJHFCAA|2011-10-01|1341|5831|448|2011|6|10|1|3|2011|448|5831|Saturday|2011Q3|N|Y|N|2455836|2456108|2455471|2455744|N|N|N|N|N| +2455837|AAAAAAAANBJHFCAA|2011-10-02|1341|5831|448|2011|0|10|2|4|2011|448|5831|Sunday|2011Q4|N|N|N|2455836|2456108|2455472|2455745|N|N|N|N|N| +2455838|AAAAAAAAOBJHFCAA|2011-10-03|1341|5831|448|2011|1|10|3|4|2011|448|5831|Monday|2011Q4|N|N|N|2455836|2456108|2455473|2455746|N|N|N|N|N| +2455839|AAAAAAAAPBJHFCAA|2011-10-04|1341|5832|448|2011|2|10|4|4|2011|448|5832|Tuesday|2011Q4|N|N|N|2455836|2456108|2455474|2455747|N|N|N|N|N| +2455840|AAAAAAAAACJHFCAA|2011-10-05|1341|5832|448|2011|3|10|5|4|2011|448|5832|Wednesday|2011Q4|N|N|N|2455836|2456108|2455475|2455748|N|N|N|N|N| +2455841|AAAAAAAABCJHFCAA|2011-10-06|1341|5832|448|2011|4|10|6|4|2011|448|5832|Thursday|2011Q4|N|N|N|2455836|2456108|2455476|2455749|N|N|N|N|N| +2455842|AAAAAAAACCJHFCAA|2011-10-07|1341|5832|448|2011|5|10|7|4|2011|448|5832|Friday|2011Q4|N|Y|N|2455836|2456108|2455477|2455750|N|N|N|N|N| +2455843|AAAAAAAADCJHFCAA|2011-10-08|1341|5832|448|2011|6|10|8|4|2011|448|5832|Saturday|2011Q4|N|Y|N|2455836|2456108|2455478|2455751|N|N|N|N|N| +2455844|AAAAAAAAECJHFCAA|2011-10-09|1341|5832|448|2011|0|10|9|4|2011|448|5832|Sunday|2011Q4|N|N|N|2455836|2456108|2455479|2455752|N|N|N|N|N| +2455845|AAAAAAAAFCJHFCAA|2011-10-10|1341|5832|448|2011|1|10|10|4|2011|448|5832|Monday|2011Q4|N|N|N|2455836|2456108|2455480|2455753|N|N|N|N|N| +2455846|AAAAAAAAGCJHFCAA|2011-10-11|1341|5833|448|2011|2|10|11|4|2011|448|5833|Tuesday|2011Q4|N|N|N|2455836|2456108|2455481|2455754|N|N|N|N|N| +2455847|AAAAAAAAHCJHFCAA|2011-10-12|1341|5833|448|2011|3|10|12|4|2011|448|5833|Wednesday|2011Q4|N|N|N|2455836|2456108|2455482|2455755|N|N|N|N|N| +2455848|AAAAAAAAICJHFCAA|2011-10-13|1341|5833|448|2011|4|10|13|4|2011|448|5833|Thursday|2011Q4|N|N|N|2455836|2456108|2455483|2455756|N|N|N|N|N| +2455849|AAAAAAAAJCJHFCAA|2011-10-14|1341|5833|448|2011|5|10|14|4|2011|448|5833|Friday|2011Q4|N|Y|N|2455836|2456108|2455484|2455757|N|N|N|N|N| +2455850|AAAAAAAAKCJHFCAA|2011-10-15|1341|5833|448|2011|6|10|15|4|2011|448|5833|Saturday|2011Q4|N|Y|N|2455836|2456108|2455485|2455758|N|N|N|N|N| +2455851|AAAAAAAALCJHFCAA|2011-10-16|1341|5833|448|2011|0|10|16|4|2011|448|5833|Sunday|2011Q4|N|N|N|2455836|2456108|2455486|2455759|N|N|N|N|N| +2455852|AAAAAAAAMCJHFCAA|2011-10-17|1341|5833|448|2011|1|10|17|4|2011|448|5833|Monday|2011Q4|N|N|N|2455836|2456108|2455487|2455760|N|N|N|N|N| +2455853|AAAAAAAANCJHFCAA|2011-10-18|1341|5834|448|2011|2|10|18|4|2011|448|5834|Tuesday|2011Q4|N|N|N|2455836|2456108|2455488|2455761|N|N|N|N|N| +2455854|AAAAAAAAOCJHFCAA|2011-10-19|1341|5834|448|2011|3|10|19|4|2011|448|5834|Wednesday|2011Q4|N|N|N|2455836|2456108|2455489|2455762|N|N|N|N|N| +2455855|AAAAAAAAPCJHFCAA|2011-10-20|1341|5834|448|2011|4|10|20|4|2011|448|5834|Thursday|2011Q4|N|N|N|2455836|2456108|2455490|2455763|N|N|N|N|N| +2455856|AAAAAAAAADJHFCAA|2011-10-21|1341|5834|448|2011|5|10|21|4|2011|448|5834|Friday|2011Q4|N|Y|N|2455836|2456108|2455491|2455764|N|N|N|N|N| +2455857|AAAAAAAABDJHFCAA|2011-10-22|1341|5834|448|2011|6|10|22|4|2011|448|5834|Saturday|2011Q4|N|Y|N|2455836|2456108|2455492|2455765|N|N|N|N|N| +2455858|AAAAAAAACDJHFCAA|2011-10-23|1341|5834|448|2011|0|10|23|4|2011|448|5834|Sunday|2011Q4|N|N|N|2455836|2456108|2455493|2455766|N|N|N|N|N| +2455859|AAAAAAAADDJHFCAA|2011-10-24|1341|5834|448|2011|1|10|24|4|2011|448|5834|Monday|2011Q4|N|N|N|2455836|2456108|2455494|2455767|N|N|N|N|N| +2455860|AAAAAAAAEDJHFCAA|2011-10-25|1341|5835|448|2011|2|10|25|4|2011|448|5835|Tuesday|2011Q4|N|N|N|2455836|2456108|2455495|2455768|N|N|N|N|N| +2455861|AAAAAAAAFDJHFCAA|2011-10-26|1341|5835|448|2011|3|10|26|4|2011|448|5835|Wednesday|2011Q4|N|N|N|2455836|2456108|2455496|2455769|N|N|N|N|N| +2455862|AAAAAAAAGDJHFCAA|2011-10-27|1341|5835|448|2011|4|10|27|4|2011|448|5835|Thursday|2011Q4|N|N|N|2455836|2456108|2455497|2455770|N|N|N|N|N| +2455863|AAAAAAAAHDJHFCAA|2011-10-28|1341|5835|448|2011|5|10|28|4|2011|448|5835|Friday|2011Q4|N|Y|N|2455836|2456108|2455498|2455771|N|N|N|N|N| +2455864|AAAAAAAAIDJHFCAA|2011-10-29|1341|5835|448|2011|6|10|29|4|2011|448|5835|Saturday|2011Q4|N|Y|N|2455836|2456108|2455499|2455772|N|N|N|N|N| +2455865|AAAAAAAAJDJHFCAA|2011-10-30|1341|5835|448|2011|0|10|30|4|2011|448|5835|Sunday|2011Q4|N|N|N|2455836|2456108|2455500|2455773|N|N|N|N|N| +2455866|AAAAAAAAKDJHFCAA|2011-10-31|1341|5835|448|2011|1|10|31|4|2011|448|5835|Monday|2011Q4|N|N|N|2455836|2456108|2455501|2455774|N|N|N|N|N| +2455867|AAAAAAAALDJHFCAA|2011-11-01|1342|5836|448|2011|2|11|1|4|2011|448|5836|Tuesday|2011Q4|N|N|N|2455867|2456170|2455502|2455775|N|N|N|N|N| +2455868|AAAAAAAAMDJHFCAA|2011-11-02|1342|5836|448|2011|3|11|2|4|2011|448|5836|Wednesday|2011Q4|N|N|N|2455867|2456170|2455503|2455776|N|N|N|N|N| +2455869|AAAAAAAANDJHFCAA|2011-11-03|1342|5836|448|2011|4|11|3|4|2011|448|5836|Thursday|2011Q4|N|N|N|2455867|2456170|2455504|2455777|N|N|N|N|N| +2455870|AAAAAAAAODJHFCAA|2011-11-04|1342|5836|448|2011|5|11|4|4|2011|448|5836|Friday|2011Q4|N|Y|N|2455867|2456170|2455505|2455778|N|N|N|N|N| +2455871|AAAAAAAAPDJHFCAA|2011-11-05|1342|5836|448|2011|6|11|5|4|2011|448|5836|Saturday|2011Q4|N|Y|N|2455867|2456170|2455506|2455779|N|N|N|N|N| +2455872|AAAAAAAAAEJHFCAA|2011-11-06|1342|5836|448|2011|0|11|6|4|2011|448|5836|Sunday|2011Q4|N|N|N|2455867|2456170|2455507|2455780|N|N|N|N|N| +2455873|AAAAAAAABEJHFCAA|2011-11-07|1342|5836|448|2011|1|11|7|4|2011|448|5836|Monday|2011Q4|N|N|N|2455867|2456170|2455508|2455781|N|N|N|N|N| +2455874|AAAAAAAACEJHFCAA|2011-11-08|1342|5837|448|2011|2|11|8|4|2011|448|5837|Tuesday|2011Q4|N|N|N|2455867|2456170|2455509|2455782|N|N|N|N|N| +2455875|AAAAAAAADEJHFCAA|2011-11-09|1342|5837|448|2011|3|11|9|4|2011|448|5837|Wednesday|2011Q4|N|N|N|2455867|2456170|2455510|2455783|N|N|N|N|N| +2455876|AAAAAAAAEEJHFCAA|2011-11-10|1342|5837|448|2011|4|11|10|4|2011|448|5837|Thursday|2011Q4|N|N|N|2455867|2456170|2455511|2455784|N|N|N|N|N| +2455877|AAAAAAAAFEJHFCAA|2011-11-11|1342|5837|448|2011|5|11|11|4|2011|448|5837|Friday|2011Q4|N|Y|N|2455867|2456170|2455512|2455785|N|N|N|N|N| +2455878|AAAAAAAAGEJHFCAA|2011-11-12|1342|5837|448|2011|6|11|12|4|2011|448|5837|Saturday|2011Q4|N|Y|N|2455867|2456170|2455513|2455786|N|N|N|N|N| +2455879|AAAAAAAAHEJHFCAA|2011-11-13|1342|5837|448|2011|0|11|13|4|2011|448|5837|Sunday|2011Q4|N|N|N|2455867|2456170|2455514|2455787|N|N|N|N|N| +2455880|AAAAAAAAIEJHFCAA|2011-11-14|1342|5837|448|2011|1|11|14|4|2011|448|5837|Monday|2011Q4|N|N|N|2455867|2456170|2455515|2455788|N|N|N|N|N| +2455881|AAAAAAAAJEJHFCAA|2011-11-15|1342|5838|448|2011|2|11|15|4|2011|448|5838|Tuesday|2011Q4|N|N|N|2455867|2456170|2455516|2455789|N|N|N|N|N| +2455882|AAAAAAAAKEJHFCAA|2011-11-16|1342|5838|448|2011|3|11|16|4|2011|448|5838|Wednesday|2011Q4|N|N|N|2455867|2456170|2455517|2455790|N|N|N|N|N| +2455883|AAAAAAAALEJHFCAA|2011-11-17|1342|5838|448|2011|4|11|17|4|2011|448|5838|Thursday|2011Q4|N|N|N|2455867|2456170|2455518|2455791|N|N|N|N|N| +2455884|AAAAAAAAMEJHFCAA|2011-11-18|1342|5838|448|2011|5|11|18|4|2011|448|5838|Friday|2011Q4|N|Y|N|2455867|2456170|2455519|2455792|N|N|N|N|N| +2455885|AAAAAAAANEJHFCAA|2011-11-19|1342|5838|448|2011|6|11|19|4|2011|448|5838|Saturday|2011Q4|N|Y|N|2455867|2456170|2455520|2455793|N|N|N|N|N| +2455886|AAAAAAAAOEJHFCAA|2011-11-20|1342|5838|448|2011|0|11|20|4|2011|448|5838|Sunday|2011Q4|N|N|N|2455867|2456170|2455521|2455794|N|N|N|N|N| +2455887|AAAAAAAAPEJHFCAA|2011-11-21|1342|5838|448|2011|1|11|21|4|2011|448|5838|Monday|2011Q4|N|N|N|2455867|2456170|2455522|2455795|N|N|N|N|N| +2455888|AAAAAAAAAFJHFCAA|2011-11-22|1342|5839|448|2011|2|11|22|4|2011|448|5839|Tuesday|2011Q4|N|N|N|2455867|2456170|2455523|2455796|N|N|N|N|N| +2455889|AAAAAAAABFJHFCAA|2011-11-23|1342|5839|448|2011|3|11|23|4|2011|448|5839|Wednesday|2011Q4|N|N|N|2455867|2456170|2455524|2455797|N|N|N|N|N| +2455890|AAAAAAAACFJHFCAA|2011-11-24|1342|5839|448|2011|4|11|24|4|2011|448|5839|Thursday|2011Q4|N|N|N|2455867|2456170|2455525|2455798|N|N|N|N|N| +2455891|AAAAAAAADFJHFCAA|2011-11-25|1342|5839|448|2011|5|11|25|4|2011|448|5839|Friday|2011Q4|N|Y|N|2455867|2456170|2455526|2455799|N|N|N|N|N| +2455892|AAAAAAAAEFJHFCAA|2011-11-26|1342|5839|448|2011|6|11|26|4|2011|448|5839|Saturday|2011Q4|N|Y|N|2455867|2456170|2455527|2455800|N|N|N|N|N| +2455893|AAAAAAAAFFJHFCAA|2011-11-27|1342|5839|448|2011|0|11|27|4|2011|448|5839|Sunday|2011Q4|N|N|N|2455867|2456170|2455528|2455801|N|N|N|N|N| +2455894|AAAAAAAAGFJHFCAA|2011-11-28|1342|5839|448|2011|1|11|28|4|2011|448|5839|Monday|2011Q4|N|N|N|2455867|2456170|2455529|2455802|N|N|N|N|N| +2455895|AAAAAAAAHFJHFCAA|2011-11-29|1342|5840|448|2011|2|11|29|4|2011|448|5840|Tuesday|2011Q4|N|N|N|2455867|2456170|2455530|2455803|N|N|N|N|N| +2455896|AAAAAAAAIFJHFCAA|2011-11-30|1342|5840|448|2011|3|11|30|4|2011|448|5840|Wednesday|2011Q4|N|N|N|2455867|2456170|2455531|2455804|N|N|N|N|N| +2455897|AAAAAAAAJFJHFCAA|2011-12-01|1343|5840|449|2011|4|12|1|4|2011|449|5840|Thursday|2011Q4|N|N|N|2455897|2456230|2455532|2455805|N|N|N|N|N| +2455898|AAAAAAAAKFJHFCAA|2011-12-02|1343|5840|449|2011|5|12|2|4|2011|449|5840|Friday|2011Q4|N|Y|N|2455897|2456230|2455533|2455806|N|N|N|N|N| +2455899|AAAAAAAALFJHFCAA|2011-12-03|1343|5840|449|2011|6|12|3|4|2011|449|5840|Saturday|2011Q4|N|Y|N|2455897|2456230|2455534|2455807|N|N|N|N|N| +2455900|AAAAAAAAMFJHFCAA|2011-12-04|1343|5840|449|2011|0|12|4|4|2011|449|5840|Sunday|2011Q4|N|N|N|2455897|2456230|2455535|2455808|N|N|N|N|N| +2455901|AAAAAAAANFJHFCAA|2011-12-05|1343|5840|449|2011|1|12|5|4|2011|449|5840|Monday|2011Q4|N|N|N|2455897|2456230|2455536|2455809|N|N|N|N|N| +2455902|AAAAAAAAOFJHFCAA|2011-12-06|1343|5841|449|2011|2|12|6|4|2011|449|5841|Tuesday|2011Q4|N|N|N|2455897|2456230|2455537|2455810|N|N|N|N|N| +2455903|AAAAAAAAPFJHFCAA|2011-12-07|1343|5841|449|2011|3|12|7|4|2011|449|5841|Wednesday|2011Q4|N|N|N|2455897|2456230|2455538|2455811|N|N|N|N|N| +2455904|AAAAAAAAAGJHFCAA|2011-12-08|1343|5841|449|2011|4|12|8|4|2011|449|5841|Thursday|2011Q4|N|N|N|2455897|2456230|2455539|2455812|N|N|N|N|N| +2455905|AAAAAAAABGJHFCAA|2011-12-09|1343|5841|449|2011|5|12|9|4|2011|449|5841|Friday|2011Q4|N|Y|N|2455897|2456230|2455540|2455813|N|N|N|N|N| +2455906|AAAAAAAACGJHFCAA|2011-12-10|1343|5841|449|2011|6|12|10|4|2011|449|5841|Saturday|2011Q4|N|Y|N|2455897|2456230|2455541|2455814|N|N|N|N|N| +2455907|AAAAAAAADGJHFCAA|2011-12-11|1343|5841|449|2011|0|12|11|4|2011|449|5841|Sunday|2011Q4|N|N|N|2455897|2456230|2455542|2455815|N|N|N|N|N| +2455908|AAAAAAAAEGJHFCAA|2011-12-12|1343|5841|449|2011|1|12|12|4|2011|449|5841|Monday|2011Q4|N|N|N|2455897|2456230|2455543|2455816|N|N|N|N|N| +2455909|AAAAAAAAFGJHFCAA|2011-12-13|1343|5842|449|2011|2|12|13|4|2011|449|5842|Tuesday|2011Q4|N|N|N|2455897|2456230|2455544|2455817|N|N|N|N|N| +2455910|AAAAAAAAGGJHFCAA|2011-12-14|1343|5842|449|2011|3|12|14|4|2011|449|5842|Wednesday|2011Q4|N|N|N|2455897|2456230|2455545|2455818|N|N|N|N|N| +2455911|AAAAAAAAHGJHFCAA|2011-12-15|1343|5842|449|2011|4|12|15|4|2011|449|5842|Thursday|2011Q4|N|N|N|2455897|2456230|2455546|2455819|N|N|N|N|N| +2455912|AAAAAAAAIGJHFCAA|2011-12-16|1343|5842|449|2011|5|12|16|4|2011|449|5842|Friday|2011Q4|N|Y|N|2455897|2456230|2455547|2455820|N|N|N|N|N| +2455913|AAAAAAAAJGJHFCAA|2011-12-17|1343|5842|449|2011|6|12|17|4|2011|449|5842|Saturday|2011Q4|N|Y|N|2455897|2456230|2455548|2455821|N|N|N|N|N| +2455914|AAAAAAAAKGJHFCAA|2011-12-18|1343|5842|449|2011|0|12|18|4|2011|449|5842|Sunday|2011Q4|N|N|N|2455897|2456230|2455549|2455822|N|N|N|N|N| +2455915|AAAAAAAALGJHFCAA|2011-12-19|1343|5842|449|2011|1|12|19|4|2011|449|5842|Monday|2011Q4|N|N|N|2455897|2456230|2455550|2455823|N|N|N|N|N| +2455916|AAAAAAAAMGJHFCAA|2011-12-20|1343|5843|449|2011|2|12|20|4|2011|449|5843|Tuesday|2011Q4|N|N|N|2455897|2456230|2455551|2455824|N|N|N|N|N| +2455917|AAAAAAAANGJHFCAA|2011-12-21|1343|5843|449|2011|3|12|21|4|2011|449|5843|Wednesday|2011Q4|N|N|N|2455897|2456230|2455552|2455825|N|N|N|N|N| +2455918|AAAAAAAAOGJHFCAA|2011-12-22|1343|5843|449|2011|4|12|22|4|2011|449|5843|Thursday|2011Q4|N|N|N|2455897|2456230|2455553|2455826|N|N|N|N|N| +2455919|AAAAAAAAPGJHFCAA|2011-12-23|1343|5843|449|2011|5|12|23|4|2011|449|5843|Friday|2011Q4|N|Y|N|2455897|2456230|2455554|2455827|N|N|N|N|N| +2455920|AAAAAAAAAHJHFCAA|2011-12-24|1343|5843|449|2011|6|12|24|4|2011|449|5843|Saturday|2011Q4|N|Y|N|2455897|2456230|2455555|2455828|N|N|N|N|N| +2455921|AAAAAAAABHJHFCAA|2011-12-25|1343|5843|449|2011|0|12|25|4|2011|449|5843|Sunday|2011Q4|N|N|N|2455897|2456230|2455556|2455829|N|N|N|N|N| +2455922|AAAAAAAACHJHFCAA|2011-12-26|1343|5843|449|2011|1|12|26|4|2011|449|5843|Monday|2011Q4|Y|N|N|2455897|2456230|2455557|2455830|N|N|N|N|N| +2455923|AAAAAAAADHJHFCAA|2011-12-27|1343|5844|449|2011|2|12|27|4|2011|449|5844|Tuesday|2011Q4|N|N|Y|2455897|2456230|2455558|2455831|N|N|N|N|N| +2455924|AAAAAAAAEHJHFCAA|2011-12-28|1343|5844|449|2011|3|12|28|4|2011|449|5844|Wednesday|2011Q4|N|N|N|2455897|2456230|2455559|2455832|N|N|N|N|N| +2455925|AAAAAAAAFHJHFCAA|2011-12-29|1343|5844|449|2011|4|12|29|4|2011|449|5844|Thursday|2011Q4|N|N|N|2455897|2456230|2455560|2455833|N|N|N|N|N| +2455926|AAAAAAAAGHJHFCAA|2011-12-30|1343|5844|449|2011|5|12|30|4|2011|449|5844|Friday|2011Q4|N|Y|N|2455897|2456230|2455561|2455834|N|N|N|N|N| +2455927|AAAAAAAAHHJHFCAA|2011-12-31|1343|5844|449|2011|6|12|31|4|2011|449|5844|Saturday|2011Q4|N|Y|N|2455897|2456230|2455562|2455835|N|N|N|N|N| +2455928|AAAAAAAAIHJHFCAA|2012-01-01|1344|5844|449|2012|0|1|1|1|2012|449|5844|Sunday|2012Q1|Y|N|N|2455928|2455927|2455563|2455836|N|N|N|N|N| +2455929|AAAAAAAAJHJHFCAA|2012-01-02|1344|5844|449|2012|1|1|2|1|2012|449|5844|Monday|2012Q1|N|N|Y|2455928|2455927|2455564|2455837|N|N|N|N|N| +2455930|AAAAAAAAKHJHFCAA|2012-01-03|1344|5845|449|2012|2|1|3|1|2012|449|5845|Tuesday|2012Q1|N|N|N|2455928|2455927|2455565|2455838|N|N|N|N|N| +2455931|AAAAAAAALHJHFCAA|2012-01-04|1344|5845|449|2012|3|1|4|1|2012|449|5845|Wednesday|2012Q1|N|N|N|2455928|2455927|2455566|2455839|N|N|N|N|N| +2455932|AAAAAAAAMHJHFCAA|2012-01-05|1344|5845|449|2012|4|1|5|1|2012|449|5845|Thursday|2012Q1|N|N|N|2455928|2455927|2455567|2455840|N|N|N|N|N| +2455933|AAAAAAAANHJHFCAA|2012-01-06|1344|5845|449|2012|5|1|6|1|2012|449|5845|Friday|2012Q1|N|Y|N|2455928|2455927|2455568|2455841|N|N|N|N|N| +2455934|AAAAAAAAOHJHFCAA|2012-01-07|1344|5845|449|2012|6|1|7|1|2012|449|5845|Saturday|2012Q1|N|Y|N|2455928|2455927|2455569|2455842|N|N|N|N|N| +2455935|AAAAAAAAPHJHFCAA|2012-01-08|1344|5845|449|2012|0|1|8|1|2012|449|5845|Sunday|2012Q1|N|N|N|2455928|2455927|2455570|2455843|N|N|N|N|N| +2455936|AAAAAAAAAIJHFCAA|2012-01-09|1344|5845|449|2012|1|1|9|1|2012|449|5845|Monday|2012Q1|N|N|N|2455928|2455927|2455571|2455844|N|N|N|N|N| +2455937|AAAAAAAABIJHFCAA|2012-01-10|1344|5846|449|2012|2|1|10|1|2012|449|5846|Tuesday|2012Q1|N|N|N|2455928|2455927|2455572|2455845|N|N|N|N|N| +2455938|AAAAAAAACIJHFCAA|2012-01-11|1344|5846|449|2012|3|1|11|1|2012|449|5846|Wednesday|2012Q1|N|N|N|2455928|2455927|2455573|2455846|N|N|N|N|N| +2455939|AAAAAAAADIJHFCAA|2012-01-12|1344|5846|449|2012|4|1|12|1|2012|449|5846|Thursday|2012Q1|N|N|N|2455928|2455927|2455574|2455847|N|N|N|N|N| +2455940|AAAAAAAAEIJHFCAA|2012-01-13|1344|5846|449|2012|5|1|13|1|2012|449|5846|Friday|2012Q1|N|Y|N|2455928|2455927|2455575|2455848|N|N|N|N|N| +2455941|AAAAAAAAFIJHFCAA|2012-01-14|1344|5846|449|2012|6|1|14|1|2012|449|5846|Saturday|2012Q1|N|Y|N|2455928|2455927|2455576|2455849|N|N|N|N|N| +2455942|AAAAAAAAGIJHFCAA|2012-01-15|1344|5846|449|2012|0|1|15|1|2012|449|5846|Sunday|2012Q1|N|N|N|2455928|2455927|2455577|2455850|N|N|N|N|N| +2455943|AAAAAAAAHIJHFCAA|2012-01-16|1344|5846|449|2012|1|1|16|1|2012|449|5846|Monday|2012Q1|N|N|N|2455928|2455927|2455578|2455851|N|N|N|N|N| +2455944|AAAAAAAAIIJHFCAA|2012-01-17|1344|5847|449|2012|2|1|17|1|2012|449|5847|Tuesday|2012Q1|N|N|N|2455928|2455927|2455579|2455852|N|N|N|N|N| +2455945|AAAAAAAAJIJHFCAA|2012-01-18|1344|5847|449|2012|3|1|18|1|2012|449|5847|Wednesday|2012Q1|N|N|N|2455928|2455927|2455580|2455853|N|N|N|N|N| +2455946|AAAAAAAAKIJHFCAA|2012-01-19|1344|5847|449|2012|4|1|19|1|2012|449|5847|Thursday|2012Q1|N|N|N|2455928|2455927|2455581|2455854|N|N|N|N|N| +2455947|AAAAAAAALIJHFCAA|2012-01-20|1344|5847|449|2012|5|1|20|1|2012|449|5847|Friday|2012Q1|N|Y|N|2455928|2455927|2455582|2455855|N|N|N|N|N| +2455948|AAAAAAAAMIJHFCAA|2012-01-21|1344|5847|449|2012|6|1|21|1|2012|449|5847|Saturday|2012Q1|N|Y|N|2455928|2455927|2455583|2455856|N|N|N|N|N| +2455949|AAAAAAAANIJHFCAA|2012-01-22|1344|5847|449|2012|0|1|22|1|2012|449|5847|Sunday|2012Q1|N|N|N|2455928|2455927|2455584|2455857|N|N|N|N|N| +2455950|AAAAAAAAOIJHFCAA|2012-01-23|1344|5847|449|2012|1|1|23|1|2012|449|5847|Monday|2012Q1|N|N|N|2455928|2455927|2455585|2455858|N|N|N|N|N| +2455951|AAAAAAAAPIJHFCAA|2012-01-24|1344|5848|449|2012|2|1|24|1|2012|449|5848|Tuesday|2012Q1|N|N|N|2455928|2455927|2455586|2455859|N|N|N|N|N| +2455952|AAAAAAAAAJJHFCAA|2012-01-25|1344|5848|449|2012|3|1|25|1|2012|449|5848|Wednesday|2012Q1|N|N|N|2455928|2455927|2455587|2455860|N|N|N|N|N| +2455953|AAAAAAAABJJHFCAA|2012-01-26|1344|5848|449|2012|4|1|26|1|2012|449|5848|Thursday|2012Q1|N|N|N|2455928|2455927|2455588|2455861|N|N|N|N|N| +2455954|AAAAAAAACJJHFCAA|2012-01-27|1344|5848|449|2012|5|1|27|1|2012|449|5848|Friday|2012Q1|N|Y|N|2455928|2455927|2455589|2455862|N|N|N|N|N| +2455955|AAAAAAAADJJHFCAA|2012-01-28|1344|5848|449|2012|6|1|28|1|2012|449|5848|Saturday|2012Q1|N|Y|N|2455928|2455927|2455590|2455863|N|N|N|N|N| +2455956|AAAAAAAAEJJHFCAA|2012-01-29|1344|5848|449|2012|0|1|29|1|2012|449|5848|Sunday|2012Q1|N|N|N|2455928|2455927|2455591|2455864|N|N|N|N|N| +2455957|AAAAAAAAFJJHFCAA|2012-01-30|1344|5848|449|2012|1|1|30|1|2012|449|5848|Monday|2012Q1|N|N|N|2455928|2455927|2455592|2455865|N|N|N|N|N| +2455958|AAAAAAAAGJJHFCAA|2012-01-31|1344|5849|449|2012|2|1|31|1|2012|449|5849|Tuesday|2012Q1|N|N|N|2455928|2455927|2455593|2455866|N|N|N|N|N| +2455959|AAAAAAAAHJJHFCAA|2012-02-01|1345|5849|449|2012|3|2|1|1|2012|449|5849|Wednesday|2012Q1|N|N|N|2455959|2455989|2455594|2455867|N|N|N|N|N| +2455960|AAAAAAAAIJJHFCAA|2012-02-02|1345|5849|449|2012|4|2|2|1|2012|449|5849|Thursday|2012Q1|N|N|N|2455959|2455989|2455595|2455868|N|N|N|N|N| +2455961|AAAAAAAAJJJHFCAA|2012-02-03|1345|5849|449|2012|5|2|3|1|2012|449|5849|Friday|2012Q1|N|Y|N|2455959|2455989|2455596|2455869|N|N|N|N|N| +2455962|AAAAAAAAKJJHFCAA|2012-02-04|1345|5849|449|2012|6|2|4|1|2012|449|5849|Saturday|2012Q1|N|Y|N|2455959|2455989|2455597|2455870|N|N|N|N|N| +2455963|AAAAAAAALJJHFCAA|2012-02-05|1345|5849|449|2012|0|2|5|1|2012|449|5849|Sunday|2012Q1|N|N|N|2455959|2455989|2455598|2455871|N|N|N|N|N| +2455964|AAAAAAAAMJJHFCAA|2012-02-06|1345|5849|449|2012|1|2|6|1|2012|449|5849|Monday|2012Q1|N|N|N|2455959|2455989|2455599|2455872|N|N|N|N|N| +2455965|AAAAAAAANJJHFCAA|2012-02-07|1345|5850|449|2012|2|2|7|1|2012|449|5850|Tuesday|2012Q1|N|N|N|2455959|2455989|2455600|2455873|N|N|N|N|N| +2455966|AAAAAAAAOJJHFCAA|2012-02-08|1345|5850|449|2012|3|2|8|1|2012|449|5850|Wednesday|2012Q1|N|N|N|2455959|2455989|2455601|2455874|N|N|N|N|N| +2455967|AAAAAAAAPJJHFCAA|2012-02-09|1345|5850|449|2012|4|2|9|1|2012|449|5850|Thursday|2012Q1|N|N|N|2455959|2455989|2455602|2455875|N|N|N|N|N| +2455968|AAAAAAAAAKJHFCAA|2012-02-10|1345|5850|449|2012|5|2|10|1|2012|449|5850|Friday|2012Q1|N|Y|N|2455959|2455989|2455603|2455876|N|N|N|N|N| +2455969|AAAAAAAABKJHFCAA|2012-02-11|1345|5850|449|2012|6|2|11|1|2012|449|5850|Saturday|2012Q1|N|Y|N|2455959|2455989|2455604|2455877|N|N|N|N|N| +2455970|AAAAAAAACKJHFCAA|2012-02-12|1345|5850|449|2012|0|2|12|1|2012|449|5850|Sunday|2012Q1|N|N|N|2455959|2455989|2455605|2455878|N|N|N|N|N| +2455971|AAAAAAAADKJHFCAA|2012-02-13|1345|5850|449|2012|1|2|13|1|2012|449|5850|Monday|2012Q1|N|N|N|2455959|2455989|2455606|2455879|N|N|N|N|N| +2455972|AAAAAAAAEKJHFCAA|2012-02-14|1345|5851|449|2012|2|2|14|1|2012|449|5851|Tuesday|2012Q1|N|N|N|2455959|2455989|2455607|2455880|N|N|N|N|N| +2455973|AAAAAAAAFKJHFCAA|2012-02-15|1345|5851|449|2012|3|2|15|1|2012|449|5851|Wednesday|2012Q1|N|N|N|2455959|2455989|2455608|2455881|N|N|N|N|N| +2455974|AAAAAAAAGKJHFCAA|2012-02-16|1345|5851|449|2012|4|2|16|1|2012|449|5851|Thursday|2012Q1|N|N|N|2455959|2455989|2455609|2455882|N|N|N|N|N| +2455975|AAAAAAAAHKJHFCAA|2012-02-17|1345|5851|449|2012|5|2|17|1|2012|449|5851|Friday|2012Q1|N|Y|N|2455959|2455989|2455610|2455883|N|N|N|N|N| +2455976|AAAAAAAAIKJHFCAA|2012-02-18|1345|5851|449|2012|6|2|18|1|2012|449|5851|Saturday|2012Q1|N|Y|N|2455959|2455989|2455611|2455884|N|N|N|N|N| +2455977|AAAAAAAAJKJHFCAA|2012-02-19|1345|5851|449|2012|0|2|19|1|2012|449|5851|Sunday|2012Q1|N|N|N|2455959|2455989|2455612|2455885|N|N|N|N|N| +2455978|AAAAAAAAKKJHFCAA|2012-02-20|1345|5851|449|2012|1|2|20|1|2012|449|5851|Monday|2012Q1|N|N|N|2455959|2455989|2455613|2455886|N|N|N|N|N| +2455979|AAAAAAAALKJHFCAA|2012-02-21|1345|5852|449|2012|2|2|21|1|2012|449|5852|Tuesday|2012Q1|N|N|N|2455959|2455989|2455614|2455887|N|N|N|N|N| +2455980|AAAAAAAAMKJHFCAA|2012-02-22|1345|5852|449|2012|3|2|22|1|2012|449|5852|Wednesday|2012Q1|N|N|N|2455959|2455989|2455615|2455888|N|N|N|N|N| +2455981|AAAAAAAANKJHFCAA|2012-02-23|1345|5852|449|2012|4|2|23|1|2012|449|5852|Thursday|2012Q1|N|N|N|2455959|2455989|2455616|2455889|N|N|N|N|N| +2455982|AAAAAAAAOKJHFCAA|2012-02-24|1345|5852|449|2012|5|2|24|1|2012|449|5852|Friday|2012Q1|N|Y|N|2455959|2455989|2455617|2455890|N|N|N|N|N| +2455983|AAAAAAAAPKJHFCAA|2012-02-25|1345|5852|449|2012|6|2|25|1|2012|449|5852|Saturday|2012Q1|N|Y|N|2455959|2455989|2455618|2455891|N|N|N|N|N| +2455984|AAAAAAAAALJHFCAA|2012-02-26|1345|5852|449|2012|0|2|26|1|2012|449|5852|Sunday|2012Q1|N|N|N|2455959|2455989|2455619|2455892|N|N|N|N|N| +2455985|AAAAAAAABLJHFCAA|2012-02-27|1345|5852|449|2012|1|2|27|1|2012|449|5852|Monday|2012Q1|N|N|N|2455959|2455989|2455620|2455893|N|N|N|N|N| +2455986|AAAAAAAACLJHFCAA|2012-02-28|1345|5853|449|2012|2|2|28|1|2012|449|5853|Tuesday|2012Q1|N|N|N|2455959|2455989|2455621|2455894|N|N|N|N|N| +2455987|AAAAAAAADLJHFCAA|2012-02-29|1345|5853|449|2012|3|2|29|1|2012|449|5853|Wednesday|2012Q1|N|N|N|2455959|2455989|2455621|2455895|N|N|N|N|N| +2455988|AAAAAAAAELJHFCAA|2012-03-01|1346|5853|450|2012|4|3|1|1|2012|450|5853|Thursday|2012Q1|N|N|N|2455988|2456047|2455622|2455896|N|N|N|N|N| +2455989|AAAAAAAAFLJHFCAA|2012-03-02|1346|5853|450|2012|5|3|2|1|2012|450|5853|Friday|2012Q1|N|Y|N|2455988|2456047|2455623|2455897|N|N|N|N|N| +2455990|AAAAAAAAGLJHFCAA|2012-03-03|1346|5853|450|2012|6|3|3|1|2012|450|5853|Saturday|2012Q1|N|Y|N|2455988|2456047|2455624|2455898|N|N|N|N|N| +2455991|AAAAAAAAHLJHFCAA|2012-03-04|1346|5853|450|2012|0|3|4|1|2012|450|5853|Sunday|2012Q1|N|N|N|2455988|2456047|2455625|2455899|N|N|N|N|N| +2455992|AAAAAAAAILJHFCAA|2012-03-05|1346|5853|450|2012|1|3|5|1|2012|450|5853|Monday|2012Q1|N|N|N|2455988|2456047|2455626|2455900|N|N|N|N|N| +2455993|AAAAAAAAJLJHFCAA|2012-03-06|1346|5854|450|2012|2|3|6|1|2012|450|5854|Tuesday|2012Q1|N|N|N|2455988|2456047|2455627|2455901|N|N|N|N|N| +2455994|AAAAAAAAKLJHFCAA|2012-03-07|1346|5854|450|2012|3|3|7|1|2012|450|5854|Wednesday|2012Q1|N|N|N|2455988|2456047|2455628|2455902|N|N|N|N|N| +2455995|AAAAAAAALLJHFCAA|2012-03-08|1346|5854|450|2012|4|3|8|1|2012|450|5854|Thursday|2012Q1|N|N|N|2455988|2456047|2455629|2455903|N|N|N|N|N| +2455996|AAAAAAAAMLJHFCAA|2012-03-09|1346|5854|450|2012|5|3|9|1|2012|450|5854|Friday|2012Q1|N|Y|N|2455988|2456047|2455630|2455904|N|N|N|N|N| +2455997|AAAAAAAANLJHFCAA|2012-03-10|1346|5854|450|2012|6|3|10|1|2012|450|5854|Saturday|2012Q1|N|Y|N|2455988|2456047|2455631|2455905|N|N|N|N|N| +2455998|AAAAAAAAOLJHFCAA|2012-03-11|1346|5854|450|2012|0|3|11|1|2012|450|5854|Sunday|2012Q1|N|N|N|2455988|2456047|2455632|2455906|N|N|N|N|N| +2455999|AAAAAAAAPLJHFCAA|2012-03-12|1346|5854|450|2012|1|3|12|1|2012|450|5854|Monday|2012Q1|N|N|N|2455988|2456047|2455633|2455907|N|N|N|N|N| +2456000|AAAAAAAAAMJHFCAA|2012-03-13|1346|5855|450|2012|2|3|13|1|2012|450|5855|Tuesday|2012Q1|N|N|N|2455988|2456047|2455634|2455908|N|N|N|N|N| +2456001|AAAAAAAABMJHFCAA|2012-03-14|1346|5855|450|2012|3|3|14|1|2012|450|5855|Wednesday|2012Q1|N|N|N|2455988|2456047|2455635|2455909|N|N|N|N|N| +2456002|AAAAAAAACMJHFCAA|2012-03-15|1346|5855|450|2012|4|3|15|1|2012|450|5855|Thursday|2012Q1|N|N|N|2455988|2456047|2455636|2455910|N|N|N|N|N| +2456003|AAAAAAAADMJHFCAA|2012-03-16|1346|5855|450|2012|5|3|16|1|2012|450|5855|Friday|2012Q1|N|Y|N|2455988|2456047|2455637|2455911|N|N|N|N|N| +2456004|AAAAAAAAEMJHFCAA|2012-03-17|1346|5855|450|2012|6|3|17|1|2012|450|5855|Saturday|2012Q1|N|Y|N|2455988|2456047|2455638|2455912|N|N|N|N|N| +2456005|AAAAAAAAFMJHFCAA|2012-03-18|1346|5855|450|2012|0|3|18|1|2012|450|5855|Sunday|2012Q1|N|N|N|2455988|2456047|2455639|2455913|N|N|N|N|N| +2456006|AAAAAAAAGMJHFCAA|2012-03-19|1346|5855|450|2012|1|3|19|1|2012|450|5855|Monday|2012Q1|N|N|N|2455988|2456047|2455640|2455914|N|N|N|N|N| +2456007|AAAAAAAAHMJHFCAA|2012-03-20|1346|5856|450|2012|2|3|20|1|2012|450|5856|Tuesday|2012Q1|N|N|N|2455988|2456047|2455641|2455915|N|N|N|N|N| +2456008|AAAAAAAAIMJHFCAA|2012-03-21|1346|5856|450|2012|3|3|21|1|2012|450|5856|Wednesday|2012Q1|N|N|N|2455988|2456047|2455642|2455916|N|N|N|N|N| +2456009|AAAAAAAAJMJHFCAA|2012-03-22|1346|5856|450|2012|4|3|22|1|2012|450|5856|Thursday|2012Q1|N|N|N|2455988|2456047|2455643|2455917|N|N|N|N|N| +2456010|AAAAAAAAKMJHFCAA|2012-03-23|1346|5856|450|2012|5|3|23|1|2012|450|5856|Friday|2012Q1|N|Y|N|2455988|2456047|2455644|2455918|N|N|N|N|N| +2456011|AAAAAAAALMJHFCAA|2012-03-24|1346|5856|450|2012|6|3|24|1|2012|450|5856|Saturday|2012Q1|N|Y|N|2455988|2456047|2455645|2455919|N|N|N|N|N| +2456012|AAAAAAAAMMJHFCAA|2012-03-25|1346|5856|450|2012|0|3|25|1|2012|450|5856|Sunday|2012Q1|N|N|N|2455988|2456047|2455646|2455920|N|N|N|N|N| +2456013|AAAAAAAANMJHFCAA|2012-03-26|1346|5856|450|2012|1|3|26|1|2012|450|5856|Monday|2012Q1|N|N|N|2455988|2456047|2455647|2455921|N|N|N|N|N| +2456014|AAAAAAAAOMJHFCAA|2012-03-27|1346|5857|450|2012|2|3|27|1|2012|450|5857|Tuesday|2012Q1|N|N|N|2455988|2456047|2455648|2455922|N|N|N|N|N| +2456015|AAAAAAAAPMJHFCAA|2012-03-28|1346|5857|450|2012|3|3|28|1|2012|450|5857|Wednesday|2012Q1|N|N|N|2455988|2456047|2455649|2455923|N|N|N|N|N| +2456016|AAAAAAAAANJHFCAA|2012-03-29|1346|5857|450|2012|4|3|29|1|2012|450|5857|Thursday|2012Q1|N|N|N|2455988|2456047|2455650|2455924|N|N|N|N|N| +2456017|AAAAAAAABNJHFCAA|2012-03-30|1346|5857|450|2012|5|3|30|1|2012|450|5857|Friday|2012Q1|N|Y|N|2455988|2456047|2455651|2455925|N|N|N|N|N| +2456018|AAAAAAAACNJHFCAA|2012-03-31|1346|5857|450|2012|6|3|31|1|2012|450|5857|Saturday|2012Q1|N|Y|N|2455988|2456047|2455652|2455926|N|N|N|N|N| +2456019|AAAAAAAADNJHFCAA|2012-04-01|1347|5857|450|2012|0|4|1|2|2012|450|5857|Sunday|2012Q2|N|N|N|2456019|2456109|2455653|2455928|N|N|N|N|N| +2456020|AAAAAAAAENJHFCAA|2012-04-02|1347|5857|450|2012|1|4|2|2|2012|450|5857|Monday|2012Q2|N|N|N|2456019|2456109|2455654|2455929|N|N|N|N|N| +2456021|AAAAAAAAFNJHFCAA|2012-04-03|1347|5858|450|2012|2|4|3|2|2012|450|5858|Tuesday|2012Q2|N|N|N|2456019|2456109|2455655|2455930|N|N|N|N|N| +2456022|AAAAAAAAGNJHFCAA|2012-04-04|1347|5858|450|2012|3|4|4|2|2012|450|5858|Wednesday|2012Q2|N|N|N|2456019|2456109|2455656|2455931|N|N|N|N|N| +2456023|AAAAAAAAHNJHFCAA|2012-04-05|1347|5858|450|2012|4|4|5|2|2012|450|5858|Thursday|2012Q2|N|N|N|2456019|2456109|2455657|2455932|N|N|N|N|N| +2456024|AAAAAAAAINJHFCAA|2012-04-06|1347|5858|450|2012|5|4|6|2|2012|450|5858|Friday|2012Q2|N|Y|N|2456019|2456109|2455658|2455933|N|N|N|N|N| +2456025|AAAAAAAAJNJHFCAA|2012-04-07|1347|5858|450|2012|6|4|7|2|2012|450|5858|Saturday|2012Q2|N|Y|N|2456019|2456109|2455659|2455934|N|N|N|N|N| +2456026|AAAAAAAAKNJHFCAA|2012-04-08|1347|5858|450|2012|0|4|8|2|2012|450|5858|Sunday|2012Q2|N|N|N|2456019|2456109|2455660|2455935|N|N|N|N|N| +2456027|AAAAAAAALNJHFCAA|2012-04-09|1347|5858|450|2012|1|4|9|2|2012|450|5858|Monday|2012Q2|N|N|N|2456019|2456109|2455661|2455936|N|N|N|N|N| +2456028|AAAAAAAAMNJHFCAA|2012-04-10|1347|5859|450|2012|2|4|10|2|2012|450|5859|Tuesday|2012Q2|N|N|N|2456019|2456109|2455662|2455937|N|N|N|N|N| +2456029|AAAAAAAANNJHFCAA|2012-04-11|1347|5859|450|2012|3|4|11|2|2012|450|5859|Wednesday|2012Q2|N|N|N|2456019|2456109|2455663|2455938|N|N|N|N|N| +2456030|AAAAAAAAONJHFCAA|2012-04-12|1347|5859|450|2012|4|4|12|2|2012|450|5859|Thursday|2012Q2|N|N|N|2456019|2456109|2455664|2455939|N|N|N|N|N| +2456031|AAAAAAAAPNJHFCAA|2012-04-13|1347|5859|450|2012|5|4|13|2|2012|450|5859|Friday|2012Q2|N|Y|N|2456019|2456109|2455665|2455940|N|N|N|N|N| +2456032|AAAAAAAAAOJHFCAA|2012-04-14|1347|5859|450|2012|6|4|14|2|2012|450|5859|Saturday|2012Q2|N|Y|N|2456019|2456109|2455666|2455941|N|N|N|N|N| +2456033|AAAAAAAABOJHFCAA|2012-04-15|1347|5859|450|2012|0|4|15|2|2012|450|5859|Sunday|2012Q2|N|N|N|2456019|2456109|2455667|2455942|N|N|N|N|N| +2456034|AAAAAAAACOJHFCAA|2012-04-16|1347|5859|450|2012|1|4|16|2|2012|450|5859|Monday|2012Q2|N|N|N|2456019|2456109|2455668|2455943|N|N|N|N|N| +2456035|AAAAAAAADOJHFCAA|2012-04-17|1347|5860|450|2012|2|4|17|2|2012|450|5860|Tuesday|2012Q2|N|N|N|2456019|2456109|2455669|2455944|N|N|N|N|N| +2456036|AAAAAAAAEOJHFCAA|2012-04-18|1347|5860|450|2012|3|4|18|2|2012|450|5860|Wednesday|2012Q2|N|N|N|2456019|2456109|2455670|2455945|N|N|N|N|N| +2456037|AAAAAAAAFOJHFCAA|2012-04-19|1347|5860|450|2012|4|4|19|2|2012|450|5860|Thursday|2012Q2|N|N|N|2456019|2456109|2455671|2455946|N|N|N|N|N| +2456038|AAAAAAAAGOJHFCAA|2012-04-20|1347|5860|450|2012|5|4|20|2|2012|450|5860|Friday|2012Q2|N|Y|N|2456019|2456109|2455672|2455947|N|N|N|N|N| +2456039|AAAAAAAAHOJHFCAA|2012-04-21|1347|5860|450|2012|6|4|21|2|2012|450|5860|Saturday|2012Q2|N|Y|N|2456019|2456109|2455673|2455948|N|N|N|N|N| +2456040|AAAAAAAAIOJHFCAA|2012-04-22|1347|5860|450|2012|0|4|22|2|2012|450|5860|Sunday|2012Q2|N|N|N|2456019|2456109|2455674|2455949|N|N|N|N|N| +2456041|AAAAAAAAJOJHFCAA|2012-04-23|1347|5860|450|2012|1|4|23|2|2012|450|5860|Monday|2012Q2|N|N|N|2456019|2456109|2455675|2455950|N|N|N|N|N| +2456042|AAAAAAAAKOJHFCAA|2012-04-24|1347|5861|450|2012|2|4|24|2|2012|450|5861|Tuesday|2012Q2|N|N|N|2456019|2456109|2455676|2455951|N|N|N|N|N| +2456043|AAAAAAAALOJHFCAA|2012-04-25|1347|5861|450|2012|3|4|25|2|2012|450|5861|Wednesday|2012Q2|N|N|N|2456019|2456109|2455677|2455952|N|N|N|N|N| +2456044|AAAAAAAAMOJHFCAA|2012-04-26|1347|5861|450|2012|4|4|26|2|2012|450|5861|Thursday|2012Q2|N|N|N|2456019|2456109|2455678|2455953|N|N|N|N|N| +2456045|AAAAAAAANOJHFCAA|2012-04-27|1347|5861|450|2012|5|4|27|2|2012|450|5861|Friday|2012Q2|N|Y|N|2456019|2456109|2455679|2455954|N|N|N|N|N| +2456046|AAAAAAAAOOJHFCAA|2012-04-28|1347|5861|450|2012|6|4|28|2|2012|450|5861|Saturday|2012Q2|N|Y|N|2456019|2456109|2455680|2455955|N|N|N|N|N| +2456047|AAAAAAAAPOJHFCAA|2012-04-29|1347|5861|450|2012|0|4|29|2|2012|450|5861|Sunday|2012Q2|N|N|N|2456019|2456109|2455681|2455956|N|N|N|N|N| +2456048|AAAAAAAAAPJHFCAA|2012-04-30|1347|5861|450|2012|1|4|30|2|2012|450|5861|Monday|2012Q2|N|N|N|2456019|2456109|2455682|2455957|N|N|N|N|N| +2456049|AAAAAAAABPJHFCAA|2012-05-01|1348|5862|450|2012|2|5|1|2|2012|450|5862|Tuesday|2012Q2|N|N|N|2456049|2456169|2455683|2455958|N|N|N|N|N| +2456050|AAAAAAAACPJHFCAA|2012-05-02|1348|5862|450|2012|3|5|2|2|2012|450|5862|Wednesday|2012Q2|N|N|N|2456049|2456169|2455684|2455959|N|N|N|N|N| +2456051|AAAAAAAADPJHFCAA|2012-05-03|1348|5862|450|2012|4|5|3|2|2012|450|5862|Thursday|2012Q2|N|N|N|2456049|2456169|2455685|2455960|N|N|N|N|N| +2456052|AAAAAAAAEPJHFCAA|2012-05-04|1348|5862|450|2012|5|5|4|2|2012|450|5862|Friday|2012Q2|N|Y|N|2456049|2456169|2455686|2455961|N|N|N|N|N| +2456053|AAAAAAAAFPJHFCAA|2012-05-05|1348|5862|450|2012|6|5|5|2|2012|450|5862|Saturday|2012Q2|N|Y|N|2456049|2456169|2455687|2455962|N|N|N|N|N| +2456054|AAAAAAAAGPJHFCAA|2012-05-06|1348|5862|450|2012|0|5|6|2|2012|450|5862|Sunday|2012Q2|N|N|N|2456049|2456169|2455688|2455963|N|N|N|N|N| +2456055|AAAAAAAAHPJHFCAA|2012-05-07|1348|5862|450|2012|1|5|7|2|2012|450|5862|Monday|2012Q2|N|N|N|2456049|2456169|2455689|2455964|N|N|N|N|N| +2456056|AAAAAAAAIPJHFCAA|2012-05-08|1348|5863|450|2012|2|5|8|2|2012|450|5863|Tuesday|2012Q2|N|N|N|2456049|2456169|2455690|2455965|N|N|N|N|N| +2456057|AAAAAAAAJPJHFCAA|2012-05-09|1348|5863|450|2012|3|5|9|2|2012|450|5863|Wednesday|2012Q2|N|N|N|2456049|2456169|2455691|2455966|N|N|N|N|N| +2456058|AAAAAAAAKPJHFCAA|2012-05-10|1348|5863|450|2012|4|5|10|2|2012|450|5863|Thursday|2012Q2|N|N|N|2456049|2456169|2455692|2455967|N|N|N|N|N| +2456059|AAAAAAAALPJHFCAA|2012-05-11|1348|5863|450|2012|5|5|11|2|2012|450|5863|Friday|2012Q2|N|Y|N|2456049|2456169|2455693|2455968|N|N|N|N|N| +2456060|AAAAAAAAMPJHFCAA|2012-05-12|1348|5863|450|2012|6|5|12|2|2012|450|5863|Saturday|2012Q2|N|Y|N|2456049|2456169|2455694|2455969|N|N|N|N|N| +2456061|AAAAAAAANPJHFCAA|2012-05-13|1348|5863|450|2012|0|5|13|2|2012|450|5863|Sunday|2012Q2|N|N|N|2456049|2456169|2455695|2455970|N|N|N|N|N| +2456062|AAAAAAAAOPJHFCAA|2012-05-14|1348|5863|450|2012|1|5|14|2|2012|450|5863|Monday|2012Q2|N|N|N|2456049|2456169|2455696|2455971|N|N|N|N|N| +2456063|AAAAAAAAPPJHFCAA|2012-05-15|1348|5864|450|2012|2|5|15|2|2012|450|5864|Tuesday|2012Q2|N|N|N|2456049|2456169|2455697|2455972|N|N|N|N|N| +2456064|AAAAAAAAAAKHFCAA|2012-05-16|1348|5864|450|2012|3|5|16|2|2012|450|5864|Wednesday|2012Q2|N|N|N|2456049|2456169|2455698|2455973|N|N|N|N|N| +2456065|AAAAAAAABAKHFCAA|2012-05-17|1348|5864|450|2012|4|5|17|2|2012|450|5864|Thursday|2012Q2|N|N|N|2456049|2456169|2455699|2455974|N|N|N|N|N| +2456066|AAAAAAAACAKHFCAA|2012-05-18|1348|5864|450|2012|5|5|18|2|2012|450|5864|Friday|2012Q2|N|Y|N|2456049|2456169|2455700|2455975|N|N|N|N|N| +2456067|AAAAAAAADAKHFCAA|2012-05-19|1348|5864|450|2012|6|5|19|2|2012|450|5864|Saturday|2012Q2|N|Y|N|2456049|2456169|2455701|2455976|N|N|N|N|N| +2456068|AAAAAAAAEAKHFCAA|2012-05-20|1348|5864|450|2012|0|5|20|2|2012|450|5864|Sunday|2012Q2|N|N|N|2456049|2456169|2455702|2455977|N|N|N|N|N| +2456069|AAAAAAAAFAKHFCAA|2012-05-21|1348|5864|450|2012|1|5|21|2|2012|450|5864|Monday|2012Q2|N|N|N|2456049|2456169|2455703|2455978|N|N|N|N|N| +2456070|AAAAAAAAGAKHFCAA|2012-05-22|1348|5865|450|2012|2|5|22|2|2012|450|5865|Tuesday|2012Q2|N|N|N|2456049|2456169|2455704|2455979|N|N|N|N|N| +2456071|AAAAAAAAHAKHFCAA|2012-05-23|1348|5865|450|2012|3|5|23|2|2012|450|5865|Wednesday|2012Q2|N|N|N|2456049|2456169|2455705|2455980|N|N|N|N|N| +2456072|AAAAAAAAIAKHFCAA|2012-05-24|1348|5865|450|2012|4|5|24|2|2012|450|5865|Thursday|2012Q2|N|N|N|2456049|2456169|2455706|2455981|N|N|N|N|N| +2456073|AAAAAAAAJAKHFCAA|2012-05-25|1348|5865|450|2012|5|5|25|2|2012|450|5865|Friday|2012Q2|N|Y|N|2456049|2456169|2455707|2455982|N|N|N|N|N| +2456074|AAAAAAAAKAKHFCAA|2012-05-26|1348|5865|450|2012|6|5|26|2|2012|450|5865|Saturday|2012Q2|N|Y|N|2456049|2456169|2455708|2455983|N|N|N|N|N| +2456075|AAAAAAAALAKHFCAA|2012-05-27|1348|5865|450|2012|0|5|27|2|2012|450|5865|Sunday|2012Q2|N|N|N|2456049|2456169|2455709|2455984|N|N|N|N|N| +2456076|AAAAAAAAMAKHFCAA|2012-05-28|1348|5865|450|2012|1|5|28|2|2012|450|5865|Monday|2012Q2|N|N|N|2456049|2456169|2455710|2455985|N|N|N|N|N| +2456077|AAAAAAAANAKHFCAA|2012-05-29|1348|5866|450|2012|2|5|29|2|2012|450|5866|Tuesday|2012Q2|N|N|N|2456049|2456169|2455711|2455986|N|N|N|N|N| +2456078|AAAAAAAAOAKHFCAA|2012-05-30|1348|5866|450|2012|3|5|30|2|2012|450|5866|Wednesday|2012Q2|N|N|N|2456049|2456169|2455712|2455987|N|N|N|N|N| +2456079|AAAAAAAAPAKHFCAA|2012-05-31|1348|5866|450|2012|4|5|31|2|2012|450|5866|Thursday|2012Q2|N|N|N|2456049|2456169|2455713|2455988|N|N|N|N|N| +2456080|AAAAAAAAABKHFCAA|2012-06-01|1349|5866|451|2012|5|6|1|2|2012|451|5866|Friday|2012Q2|N|Y|N|2456080|2456231|2455714|2455989|N|N|N|N|N| +2456081|AAAAAAAABBKHFCAA|2012-06-02|1349|5866|451|2012|6|6|2|2|2012|451|5866|Saturday|2012Q2|N|Y|N|2456080|2456231|2455715|2455990|N|N|N|N|N| +2456082|AAAAAAAACBKHFCAA|2012-06-03|1349|5866|451|2012|0|6|3|2|2012|451|5866|Sunday|2012Q2|N|N|N|2456080|2456231|2455716|2455991|N|N|N|N|N| +2456083|AAAAAAAADBKHFCAA|2012-06-04|1349|5866|451|2012|1|6|4|2|2012|451|5866|Monday|2012Q2|N|N|N|2456080|2456231|2455717|2455992|N|N|N|N|N| +2456084|AAAAAAAAEBKHFCAA|2012-06-05|1349|5867|451|2012|2|6|5|2|2012|451|5867|Tuesday|2012Q2|N|N|N|2456080|2456231|2455718|2455993|N|N|N|N|N| +2456085|AAAAAAAAFBKHFCAA|2012-06-06|1349|5867|451|2012|3|6|6|2|2012|451|5867|Wednesday|2012Q2|N|N|N|2456080|2456231|2455719|2455994|N|N|N|N|N| +2456086|AAAAAAAAGBKHFCAA|2012-06-07|1349|5867|451|2012|4|6|7|2|2012|451|5867|Thursday|2012Q2|N|N|N|2456080|2456231|2455720|2455995|N|N|N|N|N| +2456087|AAAAAAAAHBKHFCAA|2012-06-08|1349|5867|451|2012|5|6|8|2|2012|451|5867|Friday|2012Q2|N|Y|N|2456080|2456231|2455721|2455996|N|N|N|N|N| +2456088|AAAAAAAAIBKHFCAA|2012-06-09|1349|5867|451|2012|6|6|9|2|2012|451|5867|Saturday|2012Q2|N|Y|N|2456080|2456231|2455722|2455997|N|N|N|N|N| +2456089|AAAAAAAAJBKHFCAA|2012-06-10|1349|5867|451|2012|0|6|10|2|2012|451|5867|Sunday|2012Q2|N|N|N|2456080|2456231|2455723|2455998|N|N|N|N|N| +2456090|AAAAAAAAKBKHFCAA|2012-06-11|1349|5867|451|2012|1|6|11|2|2012|451|5867|Monday|2012Q2|N|N|N|2456080|2456231|2455724|2455999|N|N|N|N|N| +2456091|AAAAAAAALBKHFCAA|2012-06-12|1349|5868|451|2012|2|6|12|2|2012|451|5868|Tuesday|2012Q2|N|N|N|2456080|2456231|2455725|2456000|N|N|N|N|N| +2456092|AAAAAAAAMBKHFCAA|2012-06-13|1349|5868|451|2012|3|6|13|2|2012|451|5868|Wednesday|2012Q2|N|N|N|2456080|2456231|2455726|2456001|N|N|N|N|N| +2456093|AAAAAAAANBKHFCAA|2012-06-14|1349|5868|451|2012|4|6|14|2|2012|451|5868|Thursday|2012Q2|N|N|N|2456080|2456231|2455727|2456002|N|N|N|N|N| +2456094|AAAAAAAAOBKHFCAA|2012-06-15|1349|5868|451|2012|5|6|15|2|2012|451|5868|Friday|2012Q2|N|Y|N|2456080|2456231|2455728|2456003|N|N|N|N|N| +2456095|AAAAAAAAPBKHFCAA|2012-06-16|1349|5868|451|2012|6|6|16|2|2012|451|5868|Saturday|2012Q2|N|Y|N|2456080|2456231|2455729|2456004|N|N|N|N|N| +2456096|AAAAAAAAACKHFCAA|2012-06-17|1349|5868|451|2012|0|6|17|2|2012|451|5868|Sunday|2012Q2|N|N|N|2456080|2456231|2455730|2456005|N|N|N|N|N| +2456097|AAAAAAAABCKHFCAA|2012-06-18|1349|5868|451|2012|1|6|18|2|2012|451|5868|Monday|2012Q2|N|N|N|2456080|2456231|2455731|2456006|N|N|N|N|N| +2456098|AAAAAAAACCKHFCAA|2012-06-19|1349|5869|451|2012|2|6|19|2|2012|451|5869|Tuesday|2012Q2|N|N|N|2456080|2456231|2455732|2456007|N|N|N|N|N| +2456099|AAAAAAAADCKHFCAA|2012-06-20|1349|5869|451|2012|3|6|20|2|2012|451|5869|Wednesday|2012Q2|N|N|N|2456080|2456231|2455733|2456008|N|N|N|N|N| +2456100|AAAAAAAAECKHFCAA|2012-06-21|1349|5869|451|2012|4|6|21|2|2012|451|5869|Thursday|2012Q2|N|N|N|2456080|2456231|2455734|2456009|N|N|N|N|N| +2456101|AAAAAAAAFCKHFCAA|2012-06-22|1349|5869|451|2012|5|6|22|2|2012|451|5869|Friday|2012Q2|N|Y|N|2456080|2456231|2455735|2456010|N|N|N|N|N| +2456102|AAAAAAAAGCKHFCAA|2012-06-23|1349|5869|451|2012|6|6|23|2|2012|451|5869|Saturday|2012Q2|N|Y|N|2456080|2456231|2455736|2456011|N|N|N|N|N| +2456103|AAAAAAAAHCKHFCAA|2012-06-24|1349|5869|451|2012|0|6|24|2|2012|451|5869|Sunday|2012Q2|N|N|N|2456080|2456231|2455737|2456012|N|N|N|N|N| +2456104|AAAAAAAAICKHFCAA|2012-06-25|1349|5869|451|2012|1|6|25|2|2012|451|5869|Monday|2012Q2|N|N|N|2456080|2456231|2455738|2456013|N|N|N|N|N| +2456105|AAAAAAAAJCKHFCAA|2012-06-26|1349|5870|451|2012|2|6|26|2|2012|451|5870|Tuesday|2012Q2|N|N|N|2456080|2456231|2455739|2456014|N|N|N|N|N| +2456106|AAAAAAAAKCKHFCAA|2012-06-27|1349|5870|451|2012|3|6|27|2|2012|451|5870|Wednesday|2012Q2|N|N|N|2456080|2456231|2455740|2456015|N|N|N|N|N| +2456107|AAAAAAAALCKHFCAA|2012-06-28|1349|5870|451|2012|4|6|28|2|2012|451|5870|Thursday|2012Q2|N|N|N|2456080|2456231|2455741|2456016|N|N|N|N|N| +2456108|AAAAAAAAMCKHFCAA|2012-06-29|1349|5870|451|2012|5|6|29|2|2012|451|5870|Friday|2012Q2|N|Y|N|2456080|2456231|2455742|2456017|N|N|N|N|N| +2456109|AAAAAAAANCKHFCAA|2012-06-30|1349|5870|451|2012|6|6|30|2|2012|451|5870|Saturday|2012Q2|N|Y|N|2456080|2456231|2455743|2456018|N|N|N|N|N| +2456110|AAAAAAAAOCKHFCAA|2012-07-01|1350|5870|451|2012|0|7|1|3|2012|451|5870|Sunday|2012Q3|N|N|N|2456110|2456291|2455744|2456019|N|N|N|N|N| +2456111|AAAAAAAAPCKHFCAA|2012-07-02|1350|5870|451|2012|1|7|2|3|2012|451|5870|Monday|2012Q3|N|N|N|2456110|2456291|2455745|2456020|N|N|N|N|N| +2456112|AAAAAAAAADKHFCAA|2012-07-03|1350|5871|451|2012|2|7|3|3|2012|451|5871|Tuesday|2012Q3|N|N|N|2456110|2456291|2455746|2456021|N|N|N|N|N| +2456113|AAAAAAAABDKHFCAA|2012-07-04|1350|5871|451|2012|3|7|4|3|2012|451|5871|Wednesday|2012Q3|Y|N|N|2456110|2456291|2455747|2456022|N|N|N|N|N| +2456114|AAAAAAAACDKHFCAA|2012-07-05|1350|5871|451|2012|4|7|5|3|2012|451|5871|Thursday|2012Q3|N|N|Y|2456110|2456291|2455748|2456023|N|N|N|N|N| +2456115|AAAAAAAADDKHFCAA|2012-07-06|1350|5871|451|2012|5|7|6|3|2012|451|5871|Friday|2012Q3|N|Y|N|2456110|2456291|2455749|2456024|N|N|N|N|N| +2456116|AAAAAAAAEDKHFCAA|2012-07-07|1350|5871|451|2012|6|7|7|3|2012|451|5871|Saturday|2012Q3|N|Y|N|2456110|2456291|2455750|2456025|N|N|N|N|N| +2456117|AAAAAAAAFDKHFCAA|2012-07-08|1350|5871|451|2012|0|7|8|3|2012|451|5871|Sunday|2012Q3|N|N|N|2456110|2456291|2455751|2456026|N|N|N|N|N| +2456118|AAAAAAAAGDKHFCAA|2012-07-09|1350|5871|451|2012|1|7|9|3|2012|451|5871|Monday|2012Q3|N|N|N|2456110|2456291|2455752|2456027|N|N|N|N|N| +2456119|AAAAAAAAHDKHFCAA|2012-07-10|1350|5872|451|2012|2|7|10|3|2012|451|5872|Tuesday|2012Q3|N|N|N|2456110|2456291|2455753|2456028|N|N|N|N|N| +2456120|AAAAAAAAIDKHFCAA|2012-07-11|1350|5872|451|2012|3|7|11|3|2012|451|5872|Wednesday|2012Q3|N|N|N|2456110|2456291|2455754|2456029|N|N|N|N|N| +2456121|AAAAAAAAJDKHFCAA|2012-07-12|1350|5872|451|2012|4|7|12|3|2012|451|5872|Thursday|2012Q3|N|N|N|2456110|2456291|2455755|2456030|N|N|N|N|N| +2456122|AAAAAAAAKDKHFCAA|2012-07-13|1350|5872|451|2012|5|7|13|3|2012|451|5872|Friday|2012Q3|N|Y|N|2456110|2456291|2455756|2456031|N|N|N|N|N| +2456123|AAAAAAAALDKHFCAA|2012-07-14|1350|5872|451|2012|6|7|14|3|2012|451|5872|Saturday|2012Q3|N|Y|N|2456110|2456291|2455757|2456032|N|N|N|N|N| +2456124|AAAAAAAAMDKHFCAA|2012-07-15|1350|5872|451|2012|0|7|15|3|2012|451|5872|Sunday|2012Q3|N|N|N|2456110|2456291|2455758|2456033|N|N|N|N|N| +2456125|AAAAAAAANDKHFCAA|2012-07-16|1350|5872|451|2012|1|7|16|3|2012|451|5872|Monday|2012Q3|N|N|N|2456110|2456291|2455759|2456034|N|N|N|N|N| +2456126|AAAAAAAAODKHFCAA|2012-07-17|1350|5873|451|2012|2|7|17|3|2012|451|5873|Tuesday|2012Q3|N|N|N|2456110|2456291|2455760|2456035|N|N|N|N|N| +2456127|AAAAAAAAPDKHFCAA|2012-07-18|1350|5873|451|2012|3|7|18|3|2012|451|5873|Wednesday|2012Q3|N|N|N|2456110|2456291|2455761|2456036|N|N|N|N|N| +2456128|AAAAAAAAAEKHFCAA|2012-07-19|1350|5873|451|2012|4|7|19|3|2012|451|5873|Thursday|2012Q3|N|N|N|2456110|2456291|2455762|2456037|N|N|N|N|N| +2456129|AAAAAAAABEKHFCAA|2012-07-20|1350|5873|451|2012|5|7|20|3|2012|451|5873|Friday|2012Q3|N|Y|N|2456110|2456291|2455763|2456038|N|N|N|N|N| +2456130|AAAAAAAACEKHFCAA|2012-07-21|1350|5873|451|2012|6|7|21|3|2012|451|5873|Saturday|2012Q3|N|Y|N|2456110|2456291|2455764|2456039|N|N|N|N|N| +2456131|AAAAAAAADEKHFCAA|2012-07-22|1350|5873|451|2012|0|7|22|3|2012|451|5873|Sunday|2012Q3|N|N|N|2456110|2456291|2455765|2456040|N|N|N|N|N| +2456132|AAAAAAAAEEKHFCAA|2012-07-23|1350|5873|451|2012|1|7|23|3|2012|451|5873|Monday|2012Q3|N|N|N|2456110|2456291|2455766|2456041|N|N|N|N|N| +2456133|AAAAAAAAFEKHFCAA|2012-07-24|1350|5874|451|2012|2|7|24|3|2012|451|5874|Tuesday|2012Q3|N|N|N|2456110|2456291|2455767|2456042|N|N|N|N|N| +2456134|AAAAAAAAGEKHFCAA|2012-07-25|1350|5874|451|2012|3|7|25|3|2012|451|5874|Wednesday|2012Q3|N|N|N|2456110|2456291|2455768|2456043|N|N|N|N|N| +2456135|AAAAAAAAHEKHFCAA|2012-07-26|1350|5874|451|2012|4|7|26|3|2012|451|5874|Thursday|2012Q3|N|N|N|2456110|2456291|2455769|2456044|N|N|N|N|N| +2456136|AAAAAAAAIEKHFCAA|2012-07-27|1350|5874|451|2012|5|7|27|3|2012|451|5874|Friday|2012Q3|N|Y|N|2456110|2456291|2455770|2456045|N|N|N|N|N| +2456137|AAAAAAAAJEKHFCAA|2012-07-28|1350|5874|451|2012|6|7|28|3|2012|451|5874|Saturday|2012Q3|N|Y|N|2456110|2456291|2455771|2456046|N|N|N|N|N| +2456138|AAAAAAAAKEKHFCAA|2012-07-29|1350|5874|451|2012|0|7|29|3|2012|451|5874|Sunday|2012Q3|N|N|N|2456110|2456291|2455772|2456047|N|N|N|N|N| +2456139|AAAAAAAALEKHFCAA|2012-07-30|1350|5874|451|2012|1|7|30|3|2012|451|5874|Monday|2012Q3|N|N|N|2456110|2456291|2455773|2456048|N|N|N|N|N| +2456140|AAAAAAAAMEKHFCAA|2012-07-31|1350|5875|451|2012|2|7|31|3|2012|451|5875|Tuesday|2012Q3|N|N|N|2456110|2456291|2455774|2456049|N|N|N|N|N| +2456141|AAAAAAAANEKHFCAA|2012-08-01|1351|5875|451|2012|3|8|1|3|2012|451|5875|Wednesday|2012Q3|N|N|N|2456141|2456353|2455775|2456050|N|N|N|N|N| +2456142|AAAAAAAAOEKHFCAA|2012-08-02|1351|5875|451|2012|4|8|2|3|2012|451|5875|Thursday|2012Q3|N|N|N|2456141|2456353|2455776|2456051|N|N|N|N|N| +2456143|AAAAAAAAPEKHFCAA|2012-08-03|1351|5875|451|2012|5|8|3|3|2012|451|5875|Friday|2012Q3|N|Y|N|2456141|2456353|2455777|2456052|N|N|N|N|N| +2456144|AAAAAAAAAFKHFCAA|2012-08-04|1351|5875|451|2012|6|8|4|3|2012|451|5875|Saturday|2012Q3|N|Y|N|2456141|2456353|2455778|2456053|N|N|N|N|N| +2456145|AAAAAAAABFKHFCAA|2012-08-05|1351|5875|451|2012|0|8|5|3|2012|451|5875|Sunday|2012Q3|N|N|N|2456141|2456353|2455779|2456054|N|N|N|N|N| +2456146|AAAAAAAACFKHFCAA|2012-08-06|1351|5875|451|2012|1|8|6|3|2012|451|5875|Monday|2012Q3|N|N|N|2456141|2456353|2455780|2456055|N|N|N|N|N| +2456147|AAAAAAAADFKHFCAA|2012-08-07|1351|5876|451|2012|2|8|7|3|2012|451|5876|Tuesday|2012Q3|N|N|N|2456141|2456353|2455781|2456056|N|N|N|N|N| +2456148|AAAAAAAAEFKHFCAA|2012-08-08|1351|5876|451|2012|3|8|8|3|2012|451|5876|Wednesday|2012Q3|N|N|N|2456141|2456353|2455782|2456057|N|N|N|N|N| +2456149|AAAAAAAAFFKHFCAA|2012-08-09|1351|5876|451|2012|4|8|9|3|2012|451|5876|Thursday|2012Q3|N|N|N|2456141|2456353|2455783|2456058|N|N|N|N|N| +2456150|AAAAAAAAGFKHFCAA|2012-08-10|1351|5876|451|2012|5|8|10|3|2012|451|5876|Friday|2012Q3|N|Y|N|2456141|2456353|2455784|2456059|N|N|N|N|N| +2456151|AAAAAAAAHFKHFCAA|2012-08-11|1351|5876|451|2012|6|8|11|3|2012|451|5876|Saturday|2012Q3|N|Y|N|2456141|2456353|2455785|2456060|N|N|N|N|N| +2456152|AAAAAAAAIFKHFCAA|2012-08-12|1351|5876|451|2012|0|8|12|3|2012|451|5876|Sunday|2012Q3|N|N|N|2456141|2456353|2455786|2456061|N|N|N|N|N| +2456153|AAAAAAAAJFKHFCAA|2012-08-13|1351|5876|451|2012|1|8|13|3|2012|451|5876|Monday|2012Q3|N|N|N|2456141|2456353|2455787|2456062|N|N|N|N|N| +2456154|AAAAAAAAKFKHFCAA|2012-08-14|1351|5877|451|2012|2|8|14|3|2012|451|5877|Tuesday|2012Q3|N|N|N|2456141|2456353|2455788|2456063|N|N|N|N|N| +2456155|AAAAAAAALFKHFCAA|2012-08-15|1351|5877|451|2012|3|8|15|3|2012|451|5877|Wednesday|2012Q3|N|N|N|2456141|2456353|2455789|2456064|N|N|N|N|N| +2456156|AAAAAAAAMFKHFCAA|2012-08-16|1351|5877|451|2012|4|8|16|3|2012|451|5877|Thursday|2012Q3|N|N|N|2456141|2456353|2455790|2456065|N|N|N|N|N| +2456157|AAAAAAAANFKHFCAA|2012-08-17|1351|5877|451|2012|5|8|17|3|2012|451|5877|Friday|2012Q3|N|Y|N|2456141|2456353|2455791|2456066|N|N|N|N|N| +2456158|AAAAAAAAOFKHFCAA|2012-08-18|1351|5877|451|2012|6|8|18|3|2012|451|5877|Saturday|2012Q3|N|Y|N|2456141|2456353|2455792|2456067|N|N|N|N|N| +2456159|AAAAAAAAPFKHFCAA|2012-08-19|1351|5877|451|2012|0|8|19|3|2012|451|5877|Sunday|2012Q3|N|N|N|2456141|2456353|2455793|2456068|N|N|N|N|N| +2456160|AAAAAAAAAGKHFCAA|2012-08-20|1351|5877|451|2012|1|8|20|3|2012|451|5877|Monday|2012Q3|N|N|N|2456141|2456353|2455794|2456069|N|N|N|N|N| +2456161|AAAAAAAABGKHFCAA|2012-08-21|1351|5878|451|2012|2|8|21|3|2012|451|5878|Tuesday|2012Q3|N|N|N|2456141|2456353|2455795|2456070|N|N|N|N|N| +2456162|AAAAAAAACGKHFCAA|2012-08-22|1351|5878|451|2012|3|8|22|3|2012|451|5878|Wednesday|2012Q3|N|N|N|2456141|2456353|2455796|2456071|N|N|N|N|N| +2456163|AAAAAAAADGKHFCAA|2012-08-23|1351|5878|451|2012|4|8|23|3|2012|451|5878|Thursday|2012Q3|N|N|N|2456141|2456353|2455797|2456072|N|N|N|N|N| +2456164|AAAAAAAAEGKHFCAA|2012-08-24|1351|5878|451|2012|5|8|24|3|2012|451|5878|Friday|2012Q3|N|Y|N|2456141|2456353|2455798|2456073|N|N|N|N|N| +2456165|AAAAAAAAFGKHFCAA|2012-08-25|1351|5878|451|2012|6|8|25|3|2012|451|5878|Saturday|2012Q3|N|Y|N|2456141|2456353|2455799|2456074|N|N|N|N|N| +2456166|AAAAAAAAGGKHFCAA|2012-08-26|1351|5878|451|2012|0|8|26|3|2012|451|5878|Sunday|2012Q3|N|N|N|2456141|2456353|2455800|2456075|N|N|N|N|N| +2456167|AAAAAAAAHGKHFCAA|2012-08-27|1351|5878|451|2012|1|8|27|3|2012|451|5878|Monday|2012Q3|N|N|N|2456141|2456353|2455801|2456076|N|N|N|N|N| +2456168|AAAAAAAAIGKHFCAA|2012-08-28|1351|5879|451|2012|2|8|28|3|2012|451|5879|Tuesday|2012Q3|N|N|N|2456141|2456353|2455802|2456077|N|N|N|N|N| +2456169|AAAAAAAAJGKHFCAA|2012-08-29|1351|5879|451|2012|3|8|29|3|2012|451|5879|Wednesday|2012Q3|N|N|N|2456141|2456353|2455803|2456078|N|N|N|N|N| +2456170|AAAAAAAAKGKHFCAA|2012-08-30|1351|5879|451|2012|4|8|30|3|2012|451|5879|Thursday|2012Q3|N|N|N|2456141|2456353|2455804|2456079|N|N|N|N|N| +2456171|AAAAAAAALGKHFCAA|2012-08-31|1351|5879|451|2012|5|8|31|3|2012|451|5879|Friday|2012Q3|N|Y|N|2456141|2456353|2455805|2456080|N|N|N|N|N| +2456172|AAAAAAAAMGKHFCAA|2012-09-01|1352|5879|452|2012|6|9|1|3|2012|452|5879|Saturday|2012Q3|N|Y|N|2456172|2456415|2455806|2456081|N|N|N|N|N| +2456173|AAAAAAAANGKHFCAA|2012-09-02|1352|5879|452|2012|0|9|2|3|2012|452|5879|Sunday|2012Q3|N|N|N|2456172|2456415|2455807|2456082|N|N|N|N|N| +2456174|AAAAAAAAOGKHFCAA|2012-09-03|1352|5879|452|2012|1|9|3|3|2012|452|5879|Monday|2012Q3|N|N|N|2456172|2456415|2455808|2456083|N|N|N|N|N| +2456175|AAAAAAAAPGKHFCAA|2012-09-04|1352|5880|452|2012|2|9|4|3|2012|452|5880|Tuesday|2012Q3|N|N|N|2456172|2456415|2455809|2456084|N|N|N|N|N| +2456176|AAAAAAAAAHKHFCAA|2012-09-05|1352|5880|452|2012|3|9|5|3|2012|452|5880|Wednesday|2012Q3|N|N|N|2456172|2456415|2455810|2456085|N|N|N|N|N| +2456177|AAAAAAAABHKHFCAA|2012-09-06|1352|5880|452|2012|4|9|6|3|2012|452|5880|Thursday|2012Q3|N|N|N|2456172|2456415|2455811|2456086|N|N|N|N|N| +2456178|AAAAAAAACHKHFCAA|2012-09-07|1352|5880|452|2012|5|9|7|3|2012|452|5880|Friday|2012Q3|N|Y|N|2456172|2456415|2455812|2456087|N|N|N|N|N| +2456179|AAAAAAAADHKHFCAA|2012-09-08|1352|5880|452|2012|6|9|8|3|2012|452|5880|Saturday|2012Q3|N|Y|N|2456172|2456415|2455813|2456088|N|N|N|N|N| +2456180|AAAAAAAAEHKHFCAA|2012-09-09|1352|5880|452|2012|0|9|9|3|2012|452|5880|Sunday|2012Q3|N|N|N|2456172|2456415|2455814|2456089|N|N|N|N|N| +2456181|AAAAAAAAFHKHFCAA|2012-09-10|1352|5880|452|2012|1|9|10|3|2012|452|5880|Monday|2012Q3|N|N|N|2456172|2456415|2455815|2456090|N|N|N|N|N| +2456182|AAAAAAAAGHKHFCAA|2012-09-11|1352|5881|452|2012|2|9|11|3|2012|452|5881|Tuesday|2012Q3|N|N|N|2456172|2456415|2455816|2456091|N|N|N|N|N| +2456183|AAAAAAAAHHKHFCAA|2012-09-12|1352|5881|452|2012|3|9|12|3|2012|452|5881|Wednesday|2012Q3|N|N|N|2456172|2456415|2455817|2456092|N|N|N|N|N| +2456184|AAAAAAAAIHKHFCAA|2012-09-13|1352|5881|452|2012|4|9|13|3|2012|452|5881|Thursday|2012Q3|N|N|N|2456172|2456415|2455818|2456093|N|N|N|N|N| +2456185|AAAAAAAAJHKHFCAA|2012-09-14|1352|5881|452|2012|5|9|14|3|2012|452|5881|Friday|2012Q3|N|Y|N|2456172|2456415|2455819|2456094|N|N|N|N|N| +2456186|AAAAAAAAKHKHFCAA|2012-09-15|1352|5881|452|2012|6|9|15|3|2012|452|5881|Saturday|2012Q3|N|Y|N|2456172|2456415|2455820|2456095|N|N|N|N|N| +2456187|AAAAAAAALHKHFCAA|2012-09-16|1352|5881|452|2012|0|9|16|3|2012|452|5881|Sunday|2012Q3|N|N|N|2456172|2456415|2455821|2456096|N|N|N|N|N| +2456188|AAAAAAAAMHKHFCAA|2012-09-17|1352|5881|452|2012|1|9|17|3|2012|452|5881|Monday|2012Q3|N|N|N|2456172|2456415|2455822|2456097|N|N|N|N|N| +2456189|AAAAAAAANHKHFCAA|2012-09-18|1352|5882|452|2012|2|9|18|3|2012|452|5882|Tuesday|2012Q3|N|N|N|2456172|2456415|2455823|2456098|N|N|N|N|N| +2456190|AAAAAAAAOHKHFCAA|2012-09-19|1352|5882|452|2012|3|9|19|3|2012|452|5882|Wednesday|2012Q3|N|N|N|2456172|2456415|2455824|2456099|N|N|N|N|N| +2456191|AAAAAAAAPHKHFCAA|2012-09-20|1352|5882|452|2012|4|9|20|3|2012|452|5882|Thursday|2012Q3|N|N|N|2456172|2456415|2455825|2456100|N|N|N|N|N| +2456192|AAAAAAAAAIKHFCAA|2012-09-21|1352|5882|452|2012|5|9|21|3|2012|452|5882|Friday|2012Q3|N|Y|N|2456172|2456415|2455826|2456101|N|N|N|N|N| +2456193|AAAAAAAABIKHFCAA|2012-09-22|1352|5882|452|2012|6|9|22|3|2012|452|5882|Saturday|2012Q3|N|Y|N|2456172|2456415|2455827|2456102|N|N|N|N|N| +2456194|AAAAAAAACIKHFCAA|2012-09-23|1352|5882|452|2012|0|9|23|3|2012|452|5882|Sunday|2012Q3|N|N|N|2456172|2456415|2455828|2456103|N|N|N|N|N| +2456195|AAAAAAAADIKHFCAA|2012-09-24|1352|5882|452|2012|1|9|24|3|2012|452|5882|Monday|2012Q3|N|N|N|2456172|2456415|2455829|2456104|N|N|N|N|N| +2456196|AAAAAAAAEIKHFCAA|2012-09-25|1352|5883|452|2012|2|9|25|3|2012|452|5883|Tuesday|2012Q3|N|N|N|2456172|2456415|2455830|2456105|N|N|N|N|N| +2456197|AAAAAAAAFIKHFCAA|2012-09-26|1352|5883|452|2012|3|9|26|3|2012|452|5883|Wednesday|2012Q3|N|N|N|2456172|2456415|2455831|2456106|N|N|N|N|N| +2456198|AAAAAAAAGIKHFCAA|2012-09-27|1352|5883|452|2012|4|9|27|3|2012|452|5883|Thursday|2012Q3|N|N|N|2456172|2456415|2455832|2456107|N|N|N|N|N| +2456199|AAAAAAAAHIKHFCAA|2012-09-28|1352|5883|452|2012|5|9|28|3|2012|452|5883|Friday|2012Q3|N|Y|N|2456172|2456415|2455833|2456108|N|N|N|N|N| +2456200|AAAAAAAAIIKHFCAA|2012-09-29|1352|5883|452|2012|6|9|29|3|2012|452|5883|Saturday|2012Q3|N|Y|N|2456172|2456415|2455834|2456109|N|N|N|N|N| +2456201|AAAAAAAAJIKHFCAA|2012-09-30|1352|5883|452|2012|0|9|30|3|2012|452|5883|Sunday|2012Q3|N|N|N|2456172|2456415|2455835|2456110|N|N|N|N|N| +2456202|AAAAAAAAKIKHFCAA|2012-10-01|1353|5883|452|2012|1|10|1|4|2012|452|5883|Monday|2012Q4|N|N|N|2456202|2456475|2455836|2456110|N|N|N|N|N| +2456203|AAAAAAAALIKHFCAA|2012-10-02|1353|5884|452|2012|2|10|2|4|2012|452|5884|Tuesday|2012Q4|N|N|N|2456202|2456475|2455837|2456111|N|N|N|N|N| +2456204|AAAAAAAAMIKHFCAA|2012-10-03|1353|5884|452|2012|3|10|3|4|2012|452|5884|Wednesday|2012Q4|N|N|N|2456202|2456475|2455838|2456112|N|N|N|N|N| +2456205|AAAAAAAANIKHFCAA|2012-10-04|1353|5884|452|2012|4|10|4|4|2012|452|5884|Thursday|2012Q4|N|N|N|2456202|2456475|2455839|2456113|N|N|N|N|N| +2456206|AAAAAAAAOIKHFCAA|2012-10-05|1353|5884|452|2012|5|10|5|4|2012|452|5884|Friday|2012Q4|N|Y|N|2456202|2456475|2455840|2456114|N|N|N|N|N| +2456207|AAAAAAAAPIKHFCAA|2012-10-06|1353|5884|452|2012|6|10|6|4|2012|452|5884|Saturday|2012Q4|N|Y|N|2456202|2456475|2455841|2456115|N|N|N|N|N| +2456208|AAAAAAAAAJKHFCAA|2012-10-07|1353|5884|452|2012|0|10|7|4|2012|452|5884|Sunday|2012Q4|N|N|N|2456202|2456475|2455842|2456116|N|N|N|N|N| +2456209|AAAAAAAABJKHFCAA|2012-10-08|1353|5884|452|2012|1|10|8|4|2012|452|5884|Monday|2012Q4|N|N|N|2456202|2456475|2455843|2456117|N|N|N|N|N| +2456210|AAAAAAAACJKHFCAA|2012-10-09|1353|5885|452|2012|2|10|9|4|2012|452|5885|Tuesday|2012Q4|N|N|N|2456202|2456475|2455844|2456118|N|N|N|N|N| +2456211|AAAAAAAADJKHFCAA|2012-10-10|1353|5885|452|2012|3|10|10|4|2012|452|5885|Wednesday|2012Q4|N|N|N|2456202|2456475|2455845|2456119|N|N|N|N|N| +2456212|AAAAAAAAEJKHFCAA|2012-10-11|1353|5885|452|2012|4|10|11|4|2012|452|5885|Thursday|2012Q4|N|N|N|2456202|2456475|2455846|2456120|N|N|N|N|N| +2456213|AAAAAAAAFJKHFCAA|2012-10-12|1353|5885|452|2012|5|10|12|4|2012|452|5885|Friday|2012Q4|N|Y|N|2456202|2456475|2455847|2456121|N|N|N|N|N| +2456214|AAAAAAAAGJKHFCAA|2012-10-13|1353|5885|452|2012|6|10|13|4|2012|452|5885|Saturday|2012Q4|N|Y|N|2456202|2456475|2455848|2456122|N|N|N|N|N| +2456215|AAAAAAAAHJKHFCAA|2012-10-14|1353|5885|452|2012|0|10|14|4|2012|452|5885|Sunday|2012Q4|N|N|N|2456202|2456475|2455849|2456123|N|N|N|N|N| +2456216|AAAAAAAAIJKHFCAA|2012-10-15|1353|5885|452|2012|1|10|15|4|2012|452|5885|Monday|2012Q4|N|N|N|2456202|2456475|2455850|2456124|N|N|N|N|N| +2456217|AAAAAAAAJJKHFCAA|2012-10-16|1353|5886|452|2012|2|10|16|4|2012|452|5886|Tuesday|2012Q4|N|N|N|2456202|2456475|2455851|2456125|N|N|N|N|N| +2456218|AAAAAAAAKJKHFCAA|2012-10-17|1353|5886|452|2012|3|10|17|4|2012|452|5886|Wednesday|2012Q4|N|N|N|2456202|2456475|2455852|2456126|N|N|N|N|N| +2456219|AAAAAAAALJKHFCAA|2012-10-18|1353|5886|452|2012|4|10|18|4|2012|452|5886|Thursday|2012Q4|N|N|N|2456202|2456475|2455853|2456127|N|N|N|N|N| +2456220|AAAAAAAAMJKHFCAA|2012-10-19|1353|5886|452|2012|5|10|19|4|2012|452|5886|Friday|2012Q4|N|Y|N|2456202|2456475|2455854|2456128|N|N|N|N|N| +2456221|AAAAAAAANJKHFCAA|2012-10-20|1353|5886|452|2012|6|10|20|4|2012|452|5886|Saturday|2012Q4|N|Y|N|2456202|2456475|2455855|2456129|N|N|N|N|N| +2456222|AAAAAAAAOJKHFCAA|2012-10-21|1353|5886|452|2012|0|10|21|4|2012|452|5886|Sunday|2012Q4|N|N|N|2456202|2456475|2455856|2456130|N|N|N|N|N| +2456223|AAAAAAAAPJKHFCAA|2012-10-22|1353|5886|452|2012|1|10|22|4|2012|452|5886|Monday|2012Q4|N|N|N|2456202|2456475|2455857|2456131|N|N|N|N|N| +2456224|AAAAAAAAAKKHFCAA|2012-10-23|1353|5887|452|2012|2|10|23|4|2012|452|5887|Tuesday|2012Q4|N|N|N|2456202|2456475|2455858|2456132|N|N|N|N|N| +2456225|AAAAAAAABKKHFCAA|2012-10-24|1353|5887|452|2012|3|10|24|4|2012|452|5887|Wednesday|2012Q4|N|N|N|2456202|2456475|2455859|2456133|N|N|N|N|N| +2456226|AAAAAAAACKKHFCAA|2012-10-25|1353|5887|452|2012|4|10|25|4|2012|452|5887|Thursday|2012Q4|N|N|N|2456202|2456475|2455860|2456134|N|N|N|N|N| +2456227|AAAAAAAADKKHFCAA|2012-10-26|1353|5887|452|2012|5|10|26|4|2012|452|5887|Friday|2012Q4|N|Y|N|2456202|2456475|2455861|2456135|N|N|N|N|N| +2456228|AAAAAAAAEKKHFCAA|2012-10-27|1353|5887|452|2012|6|10|27|4|2012|452|5887|Saturday|2012Q4|N|Y|N|2456202|2456475|2455862|2456136|N|N|N|N|N| +2456229|AAAAAAAAFKKHFCAA|2012-10-28|1353|5887|452|2012|0|10|28|4|2012|452|5887|Sunday|2012Q4|N|N|N|2456202|2456475|2455863|2456137|N|N|N|N|N| +2456230|AAAAAAAAGKKHFCAA|2012-10-29|1353|5887|452|2012|1|10|29|4|2012|452|5887|Monday|2012Q4|N|N|N|2456202|2456475|2455864|2456138|N|N|N|N|N| +2456231|AAAAAAAAHKKHFCAA|2012-10-30|1353|5888|452|2012|2|10|30|4|2012|452|5888|Tuesday|2012Q4|N|N|N|2456202|2456475|2455865|2456139|N|N|N|N|N| +2456232|AAAAAAAAIKKHFCAA|2012-10-31|1353|5888|452|2012|3|10|31|4|2012|452|5888|Wednesday|2012Q4|N|N|N|2456202|2456475|2455866|2456140|N|N|N|N|N| +2456233|AAAAAAAAJKKHFCAA|2012-11-01|1354|5888|452|2012|4|11|1|4|2012|452|5888|Thursday|2012Q4|N|N|N|2456233|2456537|2455867|2456141|N|N|N|N|N| +2456234|AAAAAAAAKKKHFCAA|2012-11-02|1354|5888|452|2012|5|11|2|4|2012|452|5888|Friday|2012Q4|N|Y|N|2456233|2456537|2455868|2456142|N|N|N|N|N| +2456235|AAAAAAAALKKHFCAA|2012-11-03|1354|5888|452|2012|6|11|3|4|2012|452|5888|Saturday|2012Q4|N|Y|N|2456233|2456537|2455869|2456143|N|N|N|N|N| +2456236|AAAAAAAAMKKHFCAA|2012-11-04|1354|5888|452|2012|0|11|4|4|2012|452|5888|Sunday|2012Q4|N|N|N|2456233|2456537|2455870|2456144|N|N|N|N|N| +2456237|AAAAAAAANKKHFCAA|2012-11-05|1354|5888|452|2012|1|11|5|4|2012|452|5888|Monday|2012Q4|N|N|N|2456233|2456537|2455871|2456145|N|N|N|N|N| +2456238|AAAAAAAAOKKHFCAA|2012-11-06|1354|5889|452|2012|2|11|6|4|2012|452|5889|Tuesday|2012Q4|N|N|N|2456233|2456537|2455872|2456146|N|N|N|N|N| +2456239|AAAAAAAAPKKHFCAA|2012-11-07|1354|5889|452|2012|3|11|7|4|2012|452|5889|Wednesday|2012Q4|N|N|N|2456233|2456537|2455873|2456147|N|N|N|N|N| +2456240|AAAAAAAAALKHFCAA|2012-11-08|1354|5889|452|2012|4|11|8|4|2012|452|5889|Thursday|2012Q4|N|N|N|2456233|2456537|2455874|2456148|N|N|N|N|N| +2456241|AAAAAAAABLKHFCAA|2012-11-09|1354|5889|452|2012|5|11|9|4|2012|452|5889|Friday|2012Q4|N|Y|N|2456233|2456537|2455875|2456149|N|N|N|N|N| +2456242|AAAAAAAACLKHFCAA|2012-11-10|1354|5889|452|2012|6|11|10|4|2012|452|5889|Saturday|2012Q4|N|Y|N|2456233|2456537|2455876|2456150|N|N|N|N|N| +2456243|AAAAAAAADLKHFCAA|2012-11-11|1354|5889|452|2012|0|11|11|4|2012|452|5889|Sunday|2012Q4|N|N|N|2456233|2456537|2455877|2456151|N|N|N|N|N| +2456244|AAAAAAAAELKHFCAA|2012-11-12|1354|5889|452|2012|1|11|12|4|2012|452|5889|Monday|2012Q4|N|N|N|2456233|2456537|2455878|2456152|N|N|N|N|N| +2456245|AAAAAAAAFLKHFCAA|2012-11-13|1354|5890|452|2012|2|11|13|4|2012|452|5890|Tuesday|2012Q4|N|N|N|2456233|2456537|2455879|2456153|N|N|N|N|N| +2456246|AAAAAAAAGLKHFCAA|2012-11-14|1354|5890|452|2012|3|11|14|4|2012|452|5890|Wednesday|2012Q4|N|N|N|2456233|2456537|2455880|2456154|N|N|N|N|N| +2456247|AAAAAAAAHLKHFCAA|2012-11-15|1354|5890|452|2012|4|11|15|4|2012|452|5890|Thursday|2012Q4|N|N|N|2456233|2456537|2455881|2456155|N|N|N|N|N| +2456248|AAAAAAAAILKHFCAA|2012-11-16|1354|5890|452|2012|5|11|16|4|2012|452|5890|Friday|2012Q4|N|Y|N|2456233|2456537|2455882|2456156|N|N|N|N|N| +2456249|AAAAAAAAJLKHFCAA|2012-11-17|1354|5890|452|2012|6|11|17|4|2012|452|5890|Saturday|2012Q4|N|Y|N|2456233|2456537|2455883|2456157|N|N|N|N|N| +2456250|AAAAAAAAKLKHFCAA|2012-11-18|1354|5890|452|2012|0|11|18|4|2012|452|5890|Sunday|2012Q4|N|N|N|2456233|2456537|2455884|2456158|N|N|N|N|N| +2456251|AAAAAAAALLKHFCAA|2012-11-19|1354|5890|452|2012|1|11|19|4|2012|452|5890|Monday|2012Q4|N|N|N|2456233|2456537|2455885|2456159|N|N|N|N|N| +2456252|AAAAAAAAMLKHFCAA|2012-11-20|1354|5891|452|2012|2|11|20|4|2012|452|5891|Tuesday|2012Q4|N|N|N|2456233|2456537|2455886|2456160|N|N|N|N|N| +2456253|AAAAAAAANLKHFCAA|2012-11-21|1354|5891|452|2012|3|11|21|4|2012|452|5891|Wednesday|2012Q4|N|N|N|2456233|2456537|2455887|2456161|N|N|N|N|N| +2456254|AAAAAAAAOLKHFCAA|2012-11-22|1354|5891|452|2012|4|11|22|4|2012|452|5891|Thursday|2012Q4|N|N|N|2456233|2456537|2455888|2456162|N|N|N|N|N| +2456255|AAAAAAAAPLKHFCAA|2012-11-23|1354|5891|452|2012|5|11|23|4|2012|452|5891|Friday|2012Q4|N|Y|N|2456233|2456537|2455889|2456163|N|N|N|N|N| +2456256|AAAAAAAAAMKHFCAA|2012-11-24|1354|5891|452|2012|6|11|24|4|2012|452|5891|Saturday|2012Q4|N|Y|N|2456233|2456537|2455890|2456164|N|N|N|N|N| +2456257|AAAAAAAABMKHFCAA|2012-11-25|1354|5891|452|2012|0|11|25|4|2012|452|5891|Sunday|2012Q4|N|N|N|2456233|2456537|2455891|2456165|N|N|N|N|N| +2456258|AAAAAAAACMKHFCAA|2012-11-26|1354|5891|452|2012|1|11|26|4|2012|452|5891|Monday|2012Q4|N|N|N|2456233|2456537|2455892|2456166|N|N|N|N|N| +2456259|AAAAAAAADMKHFCAA|2012-11-27|1354|5892|452|2012|2|11|27|4|2012|452|5892|Tuesday|2012Q4|N|N|N|2456233|2456537|2455893|2456167|N|N|N|N|N| +2456260|AAAAAAAAEMKHFCAA|2012-11-28|1354|5892|452|2012|3|11|28|4|2012|452|5892|Wednesday|2012Q4|N|N|N|2456233|2456537|2455894|2456168|N|N|N|N|N| +2456261|AAAAAAAAFMKHFCAA|2012-11-29|1354|5892|452|2012|4|11|29|4|2012|452|5892|Thursday|2012Q4|N|N|N|2456233|2456537|2455895|2456169|N|N|N|N|N| +2456262|AAAAAAAAGMKHFCAA|2012-11-30|1354|5892|452|2012|5|11|30|4|2012|452|5892|Friday|2012Q4|N|Y|N|2456233|2456537|2455896|2456170|N|N|N|N|N| +2456263|AAAAAAAAHMKHFCAA|2012-12-01|1355|5892|453|2012|6|12|1|4|2012|453|5892|Saturday|2012Q4|N|Y|N|2456263|2456597|2455897|2456171|N|N|N|N|N| +2456264|AAAAAAAAIMKHFCAA|2012-12-02|1355|5892|453|2012|0|12|2|4|2012|453|5892|Sunday|2012Q4|N|N|N|2456263|2456597|2455898|2456172|N|N|N|N|N| +2456265|AAAAAAAAJMKHFCAA|2012-12-03|1355|5892|453|2012|1|12|3|4|2012|453|5892|Monday|2012Q4|N|N|N|2456263|2456597|2455899|2456173|N|N|N|N|N| +2456266|AAAAAAAAKMKHFCAA|2012-12-04|1355|5893|453|2012|2|12|4|4|2012|453|5893|Tuesday|2012Q4|N|N|N|2456263|2456597|2455900|2456174|N|N|N|N|N| +2456267|AAAAAAAALMKHFCAA|2012-12-05|1355|5893|453|2012|3|12|5|4|2012|453|5893|Wednesday|2012Q4|N|N|N|2456263|2456597|2455901|2456175|N|N|N|N|N| +2456268|AAAAAAAAMMKHFCAA|2012-12-06|1355|5893|453|2012|4|12|6|4|2012|453|5893|Thursday|2012Q4|N|N|N|2456263|2456597|2455902|2456176|N|N|N|N|N| +2456269|AAAAAAAANMKHFCAA|2012-12-07|1355|5893|453|2012|5|12|7|4|2012|453|5893|Friday|2012Q4|N|Y|N|2456263|2456597|2455903|2456177|N|N|N|N|N| +2456270|AAAAAAAAOMKHFCAA|2012-12-08|1355|5893|453|2012|6|12|8|4|2012|453|5893|Saturday|2012Q4|N|Y|N|2456263|2456597|2455904|2456178|N|N|N|N|N| +2456271|AAAAAAAAPMKHFCAA|2012-12-09|1355|5893|453|2012|0|12|9|4|2012|453|5893|Sunday|2012Q4|N|N|N|2456263|2456597|2455905|2456179|N|N|N|N|N| +2456272|AAAAAAAAANKHFCAA|2012-12-10|1355|5893|453|2012|1|12|10|4|2012|453|5893|Monday|2012Q4|N|N|N|2456263|2456597|2455906|2456180|N|N|N|N|N| +2456273|AAAAAAAABNKHFCAA|2012-12-11|1355|5894|453|2012|2|12|11|4|2012|453|5894|Tuesday|2012Q4|N|N|N|2456263|2456597|2455907|2456181|N|N|N|N|N| +2456274|AAAAAAAACNKHFCAA|2012-12-12|1355|5894|453|2012|3|12|12|4|2012|453|5894|Wednesday|2012Q4|N|N|N|2456263|2456597|2455908|2456182|N|N|N|N|N| +2456275|AAAAAAAADNKHFCAA|2012-12-13|1355|5894|453|2012|4|12|13|4|2012|453|5894|Thursday|2012Q4|N|N|N|2456263|2456597|2455909|2456183|N|N|N|N|N| +2456276|AAAAAAAAENKHFCAA|2012-12-14|1355|5894|453|2012|5|12|14|4|2012|453|5894|Friday|2012Q4|N|Y|N|2456263|2456597|2455910|2456184|N|N|N|N|N| +2456277|AAAAAAAAFNKHFCAA|2012-12-15|1355|5894|453|2012|6|12|15|4|2012|453|5894|Saturday|2012Q4|N|Y|N|2456263|2456597|2455911|2456185|N|N|N|N|N| +2456278|AAAAAAAAGNKHFCAA|2012-12-16|1355|5894|453|2012|0|12|16|4|2012|453|5894|Sunday|2012Q4|N|N|N|2456263|2456597|2455912|2456186|N|N|N|N|N| +2456279|AAAAAAAAHNKHFCAA|2012-12-17|1355|5894|453|2012|1|12|17|4|2012|453|5894|Monday|2012Q4|N|N|N|2456263|2456597|2455913|2456187|N|N|N|N|N| +2456280|AAAAAAAAINKHFCAA|2012-12-18|1355|5895|453|2012|2|12|18|4|2012|453|5895|Tuesday|2012Q4|N|N|N|2456263|2456597|2455914|2456188|N|N|N|N|N| +2456281|AAAAAAAAJNKHFCAA|2012-12-19|1355|5895|453|2012|3|12|19|4|2012|453|5895|Wednesday|2012Q4|N|N|N|2456263|2456597|2455915|2456189|N|N|N|N|N| +2456282|AAAAAAAAKNKHFCAA|2012-12-20|1355|5895|453|2012|4|12|20|4|2012|453|5895|Thursday|2012Q4|N|N|N|2456263|2456597|2455916|2456190|N|N|N|N|N| +2456283|AAAAAAAALNKHFCAA|2012-12-21|1355|5895|453|2012|5|12|21|4|2012|453|5895|Friday|2012Q4|N|Y|N|2456263|2456597|2455917|2456191|N|N|N|N|N| +2456284|AAAAAAAAMNKHFCAA|2012-12-22|1355|5895|453|2012|6|12|22|4|2012|453|5895|Saturday|2012Q4|N|Y|N|2456263|2456597|2455918|2456192|N|N|N|N|N| +2456285|AAAAAAAANNKHFCAA|2012-12-23|1355|5895|453|2012|0|12|23|4|2012|453|5895|Sunday|2012Q4|N|N|N|2456263|2456597|2455919|2456193|N|N|N|N|N| +2456286|AAAAAAAAONKHFCAA|2012-12-24|1355|5895|453|2012|1|12|24|4|2012|453|5895|Monday|2012Q4|N|N|N|2456263|2456597|2455920|2456194|N|N|N|N|N| +2456287|AAAAAAAAPNKHFCAA|2012-12-25|1355|5896|453|2012|2|12|25|4|2012|453|5896|Tuesday|2012Q4|Y|N|N|2456263|2456597|2455921|2456195|N|N|N|N|N| +2456288|AAAAAAAAAOKHFCAA|2012-12-26|1355|5896|453|2012|3|12|26|4|2012|453|5896|Wednesday|2012Q4|N|N|Y|2456263|2456597|2455922|2456196|N|N|N|N|N| +2456289|AAAAAAAABOKHFCAA|2012-12-27|1355|5896|453|2012|4|12|27|4|2012|453|5896|Thursday|2012Q4|N|N|N|2456263|2456597|2455923|2456197|N|N|N|N|N| +2456290|AAAAAAAACOKHFCAA|2012-12-28|1355|5896|453|2012|5|12|28|4|2012|453|5896|Friday|2012Q4|N|Y|N|2456263|2456597|2455924|2456198|N|N|N|N|N| +2456291|AAAAAAAADOKHFCAA|2012-12-29|1355|5896|453|2012|6|12|29|4|2012|453|5896|Saturday|2012Q4|N|Y|N|2456263|2456597|2455925|2456199|N|N|N|N|N| +2456292|AAAAAAAAEOKHFCAA|2012-12-30|1355|5896|453|2012|0|12|30|4|2012|453|5896|Sunday|2012Q4|N|N|N|2456263|2456597|2455926|2456200|N|N|N|N|N| +2456293|AAAAAAAAFOKHFCAA|2012-12-31|1355|5896|453|2012|1|12|31|4|2012|453|5896|Monday|2012Q4|Y|N|N|2456263|2456597|2455927|2456201|N|N|N|N|N| +2456294|AAAAAAAAGOKHFCAA|2013-01-01|1356|5897|453|2013|2|1|1|1|2013|453|5897|Tuesday|2013Q1|Y|N|Y|2456294|2456293|2455928|2456202|N|N|N|N|N| +2456295|AAAAAAAAHOKHFCAA|2013-01-02|1356|5897|453|2013|3|1|2|1|2013|453|5897|Wednesday|2013Q1|N|N|Y|2456294|2456293|2455929|2456203|N|N|N|N|N| +2456296|AAAAAAAAIOKHFCAA|2013-01-03|1356|5897|453|2013|4|1|3|1|2013|453|5897|Thursday|2013Q1|N|N|N|2456294|2456293|2455930|2456204|N|N|N|N|N| +2456297|AAAAAAAAJOKHFCAA|2013-01-04|1356|5897|453|2013|5|1|4|1|2013|453|5897|Friday|2013Q1|N|Y|N|2456294|2456293|2455931|2456205|N|N|N|N|N| +2456298|AAAAAAAAKOKHFCAA|2013-01-05|1356|5897|453|2013|6|1|5|1|2013|453|5897|Saturday|2013Q1|N|Y|N|2456294|2456293|2455932|2456206|N|N|N|N|N| +2456299|AAAAAAAALOKHFCAA|2013-01-06|1356|5897|453|2013|0|1|6|1|2013|453|5897|Sunday|2013Q1|N|N|N|2456294|2456293|2455933|2456207|N|N|N|N|N| +2456300|AAAAAAAAMOKHFCAA|2013-01-07|1356|5897|453|2013|1|1|7|1|2013|453|5897|Monday|2013Q1|N|N|N|2456294|2456293|2455934|2456208|N|N|N|N|N| +2456301|AAAAAAAANOKHFCAA|2013-01-08|1356|5898|453|2013|2|1|8|1|2013|453|5898|Tuesday|2013Q1|N|N|N|2456294|2456293|2455935|2456209|N|N|N|N|N| +2456302|AAAAAAAAOOKHFCAA|2013-01-09|1356|5898|453|2013|3|1|9|1|2013|453|5898|Wednesday|2013Q1|N|N|N|2456294|2456293|2455936|2456210|N|N|N|N|N| +2456303|AAAAAAAAPOKHFCAA|2013-01-10|1356|5898|453|2013|4|1|10|1|2013|453|5898|Thursday|2013Q1|N|N|N|2456294|2456293|2455937|2456211|N|N|N|N|N| +2456304|AAAAAAAAAPKHFCAA|2013-01-11|1356|5898|453|2013|5|1|11|1|2013|453|5898|Friday|2013Q1|N|Y|N|2456294|2456293|2455938|2456212|N|N|N|N|N| +2456305|AAAAAAAABPKHFCAA|2013-01-12|1356|5898|453|2013|6|1|12|1|2013|453|5898|Saturday|2013Q1|N|Y|N|2456294|2456293|2455939|2456213|N|N|N|N|N| +2456306|AAAAAAAACPKHFCAA|2013-01-13|1356|5898|453|2013|0|1|13|1|2013|453|5898|Sunday|2013Q1|N|N|N|2456294|2456293|2455940|2456214|N|N|N|N|N| +2456307|AAAAAAAADPKHFCAA|2013-01-14|1356|5898|453|2013|1|1|14|1|2013|453|5898|Monday|2013Q1|N|N|N|2456294|2456293|2455941|2456215|N|N|N|N|N| +2456308|AAAAAAAAEPKHFCAA|2013-01-15|1356|5899|453|2013|2|1|15|1|2013|453|5899|Tuesday|2013Q1|N|N|N|2456294|2456293|2455942|2456216|N|N|N|N|N| +2456309|AAAAAAAAFPKHFCAA|2013-01-16|1356|5899|453|2013|3|1|16|1|2013|453|5899|Wednesday|2013Q1|N|N|N|2456294|2456293|2455943|2456217|N|N|N|N|N| +2456310|AAAAAAAAGPKHFCAA|2013-01-17|1356|5899|453|2013|4|1|17|1|2013|453|5899|Thursday|2013Q1|N|N|N|2456294|2456293|2455944|2456218|N|N|N|N|N| +2456311|AAAAAAAAHPKHFCAA|2013-01-18|1356|5899|453|2013|5|1|18|1|2013|453|5899|Friday|2013Q1|N|Y|N|2456294|2456293|2455945|2456219|N|N|N|N|N| +2456312|AAAAAAAAIPKHFCAA|2013-01-19|1356|5899|453|2013|6|1|19|1|2013|453|5899|Saturday|2013Q1|N|Y|N|2456294|2456293|2455946|2456220|N|N|N|N|N| +2456313|AAAAAAAAJPKHFCAA|2013-01-20|1356|5899|453|2013|0|1|20|1|2013|453|5899|Sunday|2013Q1|N|N|N|2456294|2456293|2455947|2456221|N|N|N|N|N| +2456314|AAAAAAAAKPKHFCAA|2013-01-21|1356|5899|453|2013|1|1|21|1|2013|453|5899|Monday|2013Q1|N|N|N|2456294|2456293|2455948|2456222|N|N|N|N|N| +2456315|AAAAAAAALPKHFCAA|2013-01-22|1356|5900|453|2013|2|1|22|1|2013|453|5900|Tuesday|2013Q1|N|N|N|2456294|2456293|2455949|2456223|N|N|N|N|N| +2456316|AAAAAAAAMPKHFCAA|2013-01-23|1356|5900|453|2013|3|1|23|1|2013|453|5900|Wednesday|2013Q1|N|N|N|2456294|2456293|2455950|2456224|N|N|N|N|N| +2456317|AAAAAAAANPKHFCAA|2013-01-24|1356|5900|453|2013|4|1|24|1|2013|453|5900|Thursday|2013Q1|N|N|N|2456294|2456293|2455951|2456225|N|N|N|N|N| +2456318|AAAAAAAAOPKHFCAA|2013-01-25|1356|5900|453|2013|5|1|25|1|2013|453|5900|Friday|2013Q1|N|Y|N|2456294|2456293|2455952|2456226|N|N|N|N|N| +2456319|AAAAAAAAPPKHFCAA|2013-01-26|1356|5900|453|2013|6|1|26|1|2013|453|5900|Saturday|2013Q1|N|Y|N|2456294|2456293|2455953|2456227|N|N|N|N|N| +2456320|AAAAAAAAAALHFCAA|2013-01-27|1356|5900|453|2013|0|1|27|1|2013|453|5900|Sunday|2013Q1|N|N|N|2456294|2456293|2455954|2456228|N|N|N|N|N| +2456321|AAAAAAAABALHFCAA|2013-01-28|1356|5900|453|2013|1|1|28|1|2013|453|5900|Monday|2013Q1|N|N|N|2456294|2456293|2455955|2456229|N|N|N|N|N| +2456322|AAAAAAAACALHFCAA|2013-01-29|1356|5901|453|2013|2|1|29|1|2013|453|5901|Tuesday|2013Q1|N|N|N|2456294|2456293|2455956|2456230|N|N|N|N|N| +2456323|AAAAAAAADALHFCAA|2013-01-30|1356|5901|453|2013|3|1|30|1|2013|453|5901|Wednesday|2013Q1|N|N|N|2456294|2456293|2455957|2456231|N|N|N|N|N| +2456324|AAAAAAAAEALHFCAA|2013-01-31|1356|5901|453|2013|4|1|31|1|2013|453|5901|Thursday|2013Q1|N|N|N|2456294|2456293|2455958|2456232|N|N|N|N|N| +2456325|AAAAAAAAFALHFCAA|2013-02-01|1357|5901|453|2013|5|2|1|1|2013|453|5901|Friday|2013Q1|N|Y|N|2456325|2456355|2455959|2456233|N|N|N|N|N| +2456326|AAAAAAAAGALHFCAA|2013-02-02|1357|5901|453|2013|6|2|2|1|2013|453|5901|Saturday|2013Q1|N|Y|N|2456325|2456355|2455960|2456234|N|N|N|N|N| +2456327|AAAAAAAAHALHFCAA|2013-02-03|1357|5901|453|2013|0|2|3|1|2013|453|5901|Sunday|2013Q1|N|N|N|2456325|2456355|2455961|2456235|N|N|N|N|N| +2456328|AAAAAAAAIALHFCAA|2013-02-04|1357|5901|453|2013|1|2|4|1|2013|453|5901|Monday|2013Q1|N|N|N|2456325|2456355|2455962|2456236|N|N|N|N|N| +2456329|AAAAAAAAJALHFCAA|2013-02-05|1357|5902|453|2013|2|2|5|1|2013|453|5902|Tuesday|2013Q1|N|N|N|2456325|2456355|2455963|2456237|N|N|N|N|N| +2456330|AAAAAAAAKALHFCAA|2013-02-06|1357|5902|453|2013|3|2|6|1|2013|453|5902|Wednesday|2013Q1|N|N|N|2456325|2456355|2455964|2456238|N|N|N|N|N| +2456331|AAAAAAAALALHFCAA|2013-02-07|1357|5902|453|2013|4|2|7|1|2013|453|5902|Thursday|2013Q1|N|N|N|2456325|2456355|2455965|2456239|N|N|N|N|N| +2456332|AAAAAAAAMALHFCAA|2013-02-08|1357|5902|453|2013|5|2|8|1|2013|453|5902|Friday|2013Q1|N|Y|N|2456325|2456355|2455966|2456240|N|N|N|N|N| +2456333|AAAAAAAANALHFCAA|2013-02-09|1357|5902|453|2013|6|2|9|1|2013|453|5902|Saturday|2013Q1|N|Y|N|2456325|2456355|2455967|2456241|N|N|N|N|N| +2456334|AAAAAAAAOALHFCAA|2013-02-10|1357|5902|453|2013|0|2|10|1|2013|453|5902|Sunday|2013Q1|N|N|N|2456325|2456355|2455968|2456242|N|N|N|N|N| +2456335|AAAAAAAAPALHFCAA|2013-02-11|1357|5902|453|2013|1|2|11|1|2013|453|5902|Monday|2013Q1|N|N|N|2456325|2456355|2455969|2456243|N|N|N|N|N| +2456336|AAAAAAAAABLHFCAA|2013-02-12|1357|5903|453|2013|2|2|12|1|2013|453|5903|Tuesday|2013Q1|N|N|N|2456325|2456355|2455970|2456244|N|N|N|N|N| +2456337|AAAAAAAABBLHFCAA|2013-02-13|1357|5903|453|2013|3|2|13|1|2013|453|5903|Wednesday|2013Q1|N|N|N|2456325|2456355|2455971|2456245|N|N|N|N|N| +2456338|AAAAAAAACBLHFCAA|2013-02-14|1357|5903|453|2013|4|2|14|1|2013|453|5903|Thursday|2013Q1|N|N|N|2456325|2456355|2455972|2456246|N|N|N|N|N| +2456339|AAAAAAAADBLHFCAA|2013-02-15|1357|5903|453|2013|5|2|15|1|2013|453|5903|Friday|2013Q1|N|Y|N|2456325|2456355|2455973|2456247|N|N|N|N|N| +2456340|AAAAAAAAEBLHFCAA|2013-02-16|1357|5903|453|2013|6|2|16|1|2013|453|5903|Saturday|2013Q1|N|Y|N|2456325|2456355|2455974|2456248|N|N|N|N|N| +2456341|AAAAAAAAFBLHFCAA|2013-02-17|1357|5903|453|2013|0|2|17|1|2013|453|5903|Sunday|2013Q1|N|N|N|2456325|2456355|2455975|2456249|N|N|N|N|N| +2456342|AAAAAAAAGBLHFCAA|2013-02-18|1357|5903|453|2013|1|2|18|1|2013|453|5903|Monday|2013Q1|N|N|N|2456325|2456355|2455976|2456250|N|N|N|N|N| +2456343|AAAAAAAAHBLHFCAA|2013-02-19|1357|5904|453|2013|2|2|19|1|2013|453|5904|Tuesday|2013Q1|N|N|N|2456325|2456355|2455977|2456251|N|N|N|N|N| +2456344|AAAAAAAAIBLHFCAA|2013-02-20|1357|5904|453|2013|3|2|20|1|2013|453|5904|Wednesday|2013Q1|N|N|N|2456325|2456355|2455978|2456252|N|N|N|N|N| +2456345|AAAAAAAAJBLHFCAA|2013-02-21|1357|5904|453|2013|4|2|21|1|2013|453|5904|Thursday|2013Q1|N|N|N|2456325|2456355|2455979|2456253|N|N|N|N|N| +2456346|AAAAAAAAKBLHFCAA|2013-02-22|1357|5904|453|2013|5|2|22|1|2013|453|5904|Friday|2013Q1|N|Y|N|2456325|2456355|2455980|2456254|N|N|N|N|N| +2456347|AAAAAAAALBLHFCAA|2013-02-23|1357|5904|453|2013|6|2|23|1|2013|453|5904|Saturday|2013Q1|N|Y|N|2456325|2456355|2455981|2456255|N|N|N|N|N| +2456348|AAAAAAAAMBLHFCAA|2013-02-24|1357|5904|453|2013|0|2|24|1|2013|453|5904|Sunday|2013Q1|N|N|N|2456325|2456355|2455982|2456256|N|N|N|N|N| +2456349|AAAAAAAANBLHFCAA|2013-02-25|1357|5904|453|2013|1|2|25|1|2013|453|5904|Monday|2013Q1|N|N|N|2456325|2456355|2455983|2456257|N|N|N|N|N| +2456350|AAAAAAAAOBLHFCAA|2013-02-26|1357|5905|453|2013|2|2|26|1|2013|453|5905|Tuesday|2013Q1|N|N|N|2456325|2456355|2455984|2456258|N|N|N|N|N| +2456351|AAAAAAAAPBLHFCAA|2013-02-27|1357|5905|453|2013|3|2|27|1|2013|453|5905|Wednesday|2013Q1|N|N|N|2456325|2456355|2455985|2456259|N|N|N|N|N| +2456352|AAAAAAAAACLHFCAA|2013-02-28|1357|5905|453|2013|4|2|28|1|2013|453|5905|Thursday|2013Q1|N|N|N|2456325|2456355|2455986|2456260|N|N|N|N|N| +2456353|AAAAAAAABCLHFCAA|2013-03-01|1358|5905|454|2013|5|3|1|1|2013|454|5905|Friday|2013Q1|N|Y|N|2456353|2456411|2455988|2456261|N|N|N|N|N| +2456354|AAAAAAAACCLHFCAA|2013-03-02|1358|5905|454|2013|6|3|2|1|2013|454|5905|Saturday|2013Q1|N|Y|N|2456353|2456411|2455989|2456262|N|N|N|N|N| +2456355|AAAAAAAADCLHFCAA|2013-03-03|1358|5905|454|2013|0|3|3|1|2013|454|5905|Sunday|2013Q1|N|N|N|2456353|2456411|2455990|2456263|N|N|N|N|N| +2456356|AAAAAAAAECLHFCAA|2013-03-04|1358|5905|454|2013|1|3|4|1|2013|454|5905|Monday|2013Q1|N|N|N|2456353|2456411|2455991|2456264|N|N|N|N|N| +2456357|AAAAAAAAFCLHFCAA|2013-03-05|1358|5906|454|2013|2|3|5|1|2013|454|5906|Tuesday|2013Q1|N|N|N|2456353|2456411|2455992|2456265|N|N|N|N|N| +2456358|AAAAAAAAGCLHFCAA|2013-03-06|1358|5906|454|2013|3|3|6|1|2013|454|5906|Wednesday|2013Q1|N|N|N|2456353|2456411|2455993|2456266|N|N|N|N|N| +2456359|AAAAAAAAHCLHFCAA|2013-03-07|1358|5906|454|2013|4|3|7|1|2013|454|5906|Thursday|2013Q1|N|N|N|2456353|2456411|2455994|2456267|N|N|N|N|N| +2456360|AAAAAAAAICLHFCAA|2013-03-08|1358|5906|454|2013|5|3|8|1|2013|454|5906|Friday|2013Q1|N|Y|N|2456353|2456411|2455995|2456268|N|N|N|N|N| +2456361|AAAAAAAAJCLHFCAA|2013-03-09|1358|5906|454|2013|6|3|9|1|2013|454|5906|Saturday|2013Q1|N|Y|N|2456353|2456411|2455996|2456269|N|N|N|N|N| +2456362|AAAAAAAAKCLHFCAA|2013-03-10|1358|5906|454|2013|0|3|10|1|2013|454|5906|Sunday|2013Q1|N|N|N|2456353|2456411|2455997|2456270|N|N|N|N|N| +2456363|AAAAAAAALCLHFCAA|2013-03-11|1358|5906|454|2013|1|3|11|1|2013|454|5906|Monday|2013Q1|N|N|N|2456353|2456411|2455998|2456271|N|N|N|N|N| +2456364|AAAAAAAAMCLHFCAA|2013-03-12|1358|5907|454|2013|2|3|12|1|2013|454|5907|Tuesday|2013Q1|N|N|N|2456353|2456411|2455999|2456272|N|N|N|N|N| +2456365|AAAAAAAANCLHFCAA|2013-03-13|1358|5907|454|2013|3|3|13|1|2013|454|5907|Wednesday|2013Q1|N|N|N|2456353|2456411|2456000|2456273|N|N|N|N|N| +2456366|AAAAAAAAOCLHFCAA|2013-03-14|1358|5907|454|2013|4|3|14|1|2013|454|5907|Thursday|2013Q1|N|N|N|2456353|2456411|2456001|2456274|N|N|N|N|N| +2456367|AAAAAAAAPCLHFCAA|2013-03-15|1358|5907|454|2013|5|3|15|1|2013|454|5907|Friday|2013Q1|N|Y|N|2456353|2456411|2456002|2456275|N|N|N|N|N| +2456368|AAAAAAAAADLHFCAA|2013-03-16|1358|5907|454|2013|6|3|16|1|2013|454|5907|Saturday|2013Q1|N|Y|N|2456353|2456411|2456003|2456276|N|N|N|N|N| +2456369|AAAAAAAABDLHFCAA|2013-03-17|1358|5907|454|2013|0|3|17|1|2013|454|5907|Sunday|2013Q1|N|N|N|2456353|2456411|2456004|2456277|N|N|N|N|N| +2456370|AAAAAAAACDLHFCAA|2013-03-18|1358|5907|454|2013|1|3|18|1|2013|454|5907|Monday|2013Q1|N|N|N|2456353|2456411|2456005|2456278|N|N|N|N|N| +2456371|AAAAAAAADDLHFCAA|2013-03-19|1358|5908|454|2013|2|3|19|1|2013|454|5908|Tuesday|2013Q1|N|N|N|2456353|2456411|2456006|2456279|N|N|N|N|N| +2456372|AAAAAAAAEDLHFCAA|2013-03-20|1358|5908|454|2013|3|3|20|1|2013|454|5908|Wednesday|2013Q1|N|N|N|2456353|2456411|2456007|2456280|N|N|N|N|N| +2456373|AAAAAAAAFDLHFCAA|2013-03-21|1358|5908|454|2013|4|3|21|1|2013|454|5908|Thursday|2013Q1|N|N|N|2456353|2456411|2456008|2456281|N|N|N|N|N| +2456374|AAAAAAAAGDLHFCAA|2013-03-22|1358|5908|454|2013|5|3|22|1|2013|454|5908|Friday|2013Q1|N|Y|N|2456353|2456411|2456009|2456282|N|N|N|N|N| +2456375|AAAAAAAAHDLHFCAA|2013-03-23|1358|5908|454|2013|6|3|23|1|2013|454|5908|Saturday|2013Q1|N|Y|N|2456353|2456411|2456010|2456283|N|N|N|N|N| +2456376|AAAAAAAAIDLHFCAA|2013-03-24|1358|5908|454|2013|0|3|24|1|2013|454|5908|Sunday|2013Q1|N|N|N|2456353|2456411|2456011|2456284|N|N|N|N|N| +2456377|AAAAAAAAJDLHFCAA|2013-03-25|1358|5908|454|2013|1|3|25|1|2013|454|5908|Monday|2013Q1|N|N|N|2456353|2456411|2456012|2456285|N|N|N|N|N| +2456378|AAAAAAAAKDLHFCAA|2013-03-26|1358|5909|454|2013|2|3|26|1|2013|454|5909|Tuesday|2013Q1|N|N|N|2456353|2456411|2456013|2456286|N|N|N|N|N| +2456379|AAAAAAAALDLHFCAA|2013-03-27|1358|5909|454|2013|3|3|27|1|2013|454|5909|Wednesday|2013Q1|N|N|N|2456353|2456411|2456014|2456287|N|N|N|N|N| +2456380|AAAAAAAAMDLHFCAA|2013-03-28|1358|5909|454|2013|4|3|28|1|2013|454|5909|Thursday|2013Q1|N|N|N|2456353|2456411|2456015|2456288|N|N|N|N|N| +2456381|AAAAAAAANDLHFCAA|2013-03-29|1358|5909|454|2013|5|3|29|1|2013|454|5909|Friday|2013Q1|N|Y|N|2456353|2456411|2456016|2456289|N|N|N|N|N| +2456382|AAAAAAAAODLHFCAA|2013-03-30|1358|5909|454|2013|6|3|30|1|2013|454|5909|Saturday|2013Q1|N|Y|N|2456353|2456411|2456017|2456290|N|N|N|N|N| +2456383|AAAAAAAAPDLHFCAA|2013-03-31|1358|5909|454|2013|0|3|31|1|2013|454|5909|Sunday|2013Q1|N|N|N|2456353|2456411|2456018|2456291|N|N|N|N|N| +2456384|AAAAAAAAAELHFCAA|2013-04-01|1359|5909|454|2013|1|4|1|1|2013|454|5909|Monday|2013Q1|N|N|N|2456384|2456473|2456019|2456294|N|N|N|N|N| +2456385|AAAAAAAABELHFCAA|2013-04-02|1359|5910|454|2013|2|4|2|2|2013|454|5910|Tuesday|2013Q2|N|N|N|2456384|2456473|2456020|2456295|N|N|N|N|N| +2456386|AAAAAAAACELHFCAA|2013-04-03|1359|5910|454|2013|3|4|3|2|2013|454|5910|Wednesday|2013Q2|N|N|N|2456384|2456473|2456021|2456296|N|N|N|N|N| +2456387|AAAAAAAADELHFCAA|2013-04-04|1359|5910|454|2013|4|4|4|2|2013|454|5910|Thursday|2013Q2|N|N|N|2456384|2456473|2456022|2456297|N|N|N|N|N| +2456388|AAAAAAAAEELHFCAA|2013-04-05|1359|5910|454|2013|5|4|5|2|2013|454|5910|Friday|2013Q2|N|Y|N|2456384|2456473|2456023|2456298|N|N|N|N|N| +2456389|AAAAAAAAFELHFCAA|2013-04-06|1359|5910|454|2013|6|4|6|2|2013|454|5910|Saturday|2013Q2|N|Y|N|2456384|2456473|2456024|2456299|N|N|N|N|N| +2456390|AAAAAAAAGELHFCAA|2013-04-07|1359|5910|454|2013|0|4|7|2|2013|454|5910|Sunday|2013Q2|N|N|N|2456384|2456473|2456025|2456300|N|N|N|N|N| +2456391|AAAAAAAAHELHFCAA|2013-04-08|1359|5910|454|2013|1|4|8|2|2013|454|5910|Monday|2013Q2|N|N|N|2456384|2456473|2456026|2456301|N|N|N|N|N| +2456392|AAAAAAAAIELHFCAA|2013-04-09|1359|5911|454|2013|2|4|9|2|2013|454|5911|Tuesday|2013Q2|N|N|N|2456384|2456473|2456027|2456302|N|N|N|N|N| +2456393|AAAAAAAAJELHFCAA|2013-04-10|1359|5911|454|2013|3|4|10|2|2013|454|5911|Wednesday|2013Q2|N|N|N|2456384|2456473|2456028|2456303|N|N|N|N|N| +2456394|AAAAAAAAKELHFCAA|2013-04-11|1359|5911|454|2013|4|4|11|2|2013|454|5911|Thursday|2013Q2|N|N|N|2456384|2456473|2456029|2456304|N|N|N|N|N| +2456395|AAAAAAAALELHFCAA|2013-04-12|1359|5911|454|2013|5|4|12|2|2013|454|5911|Friday|2013Q2|N|Y|N|2456384|2456473|2456030|2456305|N|N|N|N|N| +2456396|AAAAAAAAMELHFCAA|2013-04-13|1359|5911|454|2013|6|4|13|2|2013|454|5911|Saturday|2013Q2|N|Y|N|2456384|2456473|2456031|2456306|N|N|N|N|N| +2456397|AAAAAAAANELHFCAA|2013-04-14|1359|5911|454|2013|0|4|14|2|2013|454|5911|Sunday|2013Q2|N|N|N|2456384|2456473|2456032|2456307|N|N|N|N|N| +2456398|AAAAAAAAOELHFCAA|2013-04-15|1359|5911|454|2013|1|4|15|2|2013|454|5911|Monday|2013Q2|N|N|N|2456384|2456473|2456033|2456308|N|N|N|N|N| +2456399|AAAAAAAAPELHFCAA|2013-04-16|1359|5912|454|2013|2|4|16|2|2013|454|5912|Tuesday|2013Q2|N|N|N|2456384|2456473|2456034|2456309|N|N|N|N|N| +2456400|AAAAAAAAAFLHFCAA|2013-04-17|1359|5912|454|2013|3|4|17|2|2013|454|5912|Wednesday|2013Q2|N|N|N|2456384|2456473|2456035|2456310|N|N|N|N|N| +2456401|AAAAAAAABFLHFCAA|2013-04-18|1359|5912|454|2013|4|4|18|2|2013|454|5912|Thursday|2013Q2|N|N|N|2456384|2456473|2456036|2456311|N|N|N|N|N| +2456402|AAAAAAAACFLHFCAA|2013-04-19|1359|5912|454|2013|5|4|19|2|2013|454|5912|Friday|2013Q2|N|Y|N|2456384|2456473|2456037|2456312|N|N|N|N|N| +2456403|AAAAAAAADFLHFCAA|2013-04-20|1359|5912|454|2013|6|4|20|2|2013|454|5912|Saturday|2013Q2|N|Y|N|2456384|2456473|2456038|2456313|N|N|N|N|N| +2456404|AAAAAAAAEFLHFCAA|2013-04-21|1359|5912|454|2013|0|4|21|2|2013|454|5912|Sunday|2013Q2|N|N|N|2456384|2456473|2456039|2456314|N|N|N|N|N| +2456405|AAAAAAAAFFLHFCAA|2013-04-22|1359|5912|454|2013|1|4|22|2|2013|454|5912|Monday|2013Q2|N|N|N|2456384|2456473|2456040|2456315|N|N|N|N|N| +2456406|AAAAAAAAGFLHFCAA|2013-04-23|1359|5913|454|2013|2|4|23|2|2013|454|5913|Tuesday|2013Q2|N|N|N|2456384|2456473|2456041|2456316|N|N|N|N|N| +2456407|AAAAAAAAHFLHFCAA|2013-04-24|1359|5913|454|2013|3|4|24|2|2013|454|5913|Wednesday|2013Q2|N|N|N|2456384|2456473|2456042|2456317|N|N|N|N|N| +2456408|AAAAAAAAIFLHFCAA|2013-04-25|1359|5913|454|2013|4|4|25|2|2013|454|5913|Thursday|2013Q2|N|N|N|2456384|2456473|2456043|2456318|N|N|N|N|N| +2456409|AAAAAAAAJFLHFCAA|2013-04-26|1359|5913|454|2013|5|4|26|2|2013|454|5913|Friday|2013Q2|N|Y|N|2456384|2456473|2456044|2456319|N|N|N|N|N| +2456410|AAAAAAAAKFLHFCAA|2013-04-27|1359|5913|454|2013|6|4|27|2|2013|454|5913|Saturday|2013Q2|N|Y|N|2456384|2456473|2456045|2456320|N|N|N|N|N| +2456411|AAAAAAAALFLHFCAA|2013-04-28|1359|5913|454|2013|0|4|28|2|2013|454|5913|Sunday|2013Q2|N|N|N|2456384|2456473|2456046|2456321|N|N|N|N|N| +2456412|AAAAAAAAMFLHFCAA|2013-04-29|1359|5913|454|2013|1|4|29|2|2013|454|5913|Monday|2013Q2|N|N|N|2456384|2456473|2456047|2456322|N|N|N|N|N| +2456413|AAAAAAAANFLHFCAA|2013-04-30|1359|5914|454|2013|2|4|30|2|2013|454|5914|Tuesday|2013Q2|N|N|N|2456384|2456473|2456048|2456323|N|N|N|N|N| +2456414|AAAAAAAAOFLHFCAA|2013-05-01|1360|5914|454|2013|3|5|1|2|2013|454|5914|Wednesday|2013Q2|N|N|N|2456414|2456533|2456049|2456324|N|N|N|N|N| +2456415|AAAAAAAAPFLHFCAA|2013-05-02|1360|5914|454|2013|4|5|2|2|2013|454|5914|Thursday|2013Q2|N|N|N|2456414|2456533|2456050|2456325|N|N|N|N|N| +2456416|AAAAAAAAAGLHFCAA|2013-05-03|1360|5914|454|2013|5|5|3|2|2013|454|5914|Friday|2013Q2|N|Y|N|2456414|2456533|2456051|2456326|N|N|N|N|N| +2456417|AAAAAAAABGLHFCAA|2013-05-04|1360|5914|454|2013|6|5|4|2|2013|454|5914|Saturday|2013Q2|N|Y|N|2456414|2456533|2456052|2456327|N|N|N|N|N| +2456418|AAAAAAAACGLHFCAA|2013-05-05|1360|5914|454|2013|0|5|5|2|2013|454|5914|Sunday|2013Q2|N|N|N|2456414|2456533|2456053|2456328|N|N|N|N|N| +2456419|AAAAAAAADGLHFCAA|2013-05-06|1360|5914|454|2013|1|5|6|2|2013|454|5914|Monday|2013Q2|N|N|N|2456414|2456533|2456054|2456329|N|N|N|N|N| +2456420|AAAAAAAAEGLHFCAA|2013-05-07|1360|5915|454|2013|2|5|7|2|2013|454|5915|Tuesday|2013Q2|N|N|N|2456414|2456533|2456055|2456330|N|N|N|N|N| +2456421|AAAAAAAAFGLHFCAA|2013-05-08|1360|5915|454|2013|3|5|8|2|2013|454|5915|Wednesday|2013Q2|N|N|N|2456414|2456533|2456056|2456331|N|N|N|N|N| +2456422|AAAAAAAAGGLHFCAA|2013-05-09|1360|5915|454|2013|4|5|9|2|2013|454|5915|Thursday|2013Q2|N|N|N|2456414|2456533|2456057|2456332|N|N|N|N|N| +2456423|AAAAAAAAHGLHFCAA|2013-05-10|1360|5915|454|2013|5|5|10|2|2013|454|5915|Friday|2013Q2|N|Y|N|2456414|2456533|2456058|2456333|N|N|N|N|N| +2456424|AAAAAAAAIGLHFCAA|2013-05-11|1360|5915|454|2013|6|5|11|2|2013|454|5915|Saturday|2013Q2|N|Y|N|2456414|2456533|2456059|2456334|N|N|N|N|N| +2456425|AAAAAAAAJGLHFCAA|2013-05-12|1360|5915|454|2013|0|5|12|2|2013|454|5915|Sunday|2013Q2|N|N|N|2456414|2456533|2456060|2456335|N|N|N|N|N| +2456426|AAAAAAAAKGLHFCAA|2013-05-13|1360|5915|454|2013|1|5|13|2|2013|454|5915|Monday|2013Q2|N|N|N|2456414|2456533|2456061|2456336|N|N|N|N|N| +2456427|AAAAAAAALGLHFCAA|2013-05-14|1360|5916|454|2013|2|5|14|2|2013|454|5916|Tuesday|2013Q2|N|N|N|2456414|2456533|2456062|2456337|N|N|N|N|N| +2456428|AAAAAAAAMGLHFCAA|2013-05-15|1360|5916|454|2013|3|5|15|2|2013|454|5916|Wednesday|2013Q2|N|N|N|2456414|2456533|2456063|2456338|N|N|N|N|N| +2456429|AAAAAAAANGLHFCAA|2013-05-16|1360|5916|454|2013|4|5|16|2|2013|454|5916|Thursday|2013Q2|N|N|N|2456414|2456533|2456064|2456339|N|N|N|N|N| +2456430|AAAAAAAAOGLHFCAA|2013-05-17|1360|5916|454|2013|5|5|17|2|2013|454|5916|Friday|2013Q2|N|Y|N|2456414|2456533|2456065|2456340|N|N|N|N|N| +2456431|AAAAAAAAPGLHFCAA|2013-05-18|1360|5916|454|2013|6|5|18|2|2013|454|5916|Saturday|2013Q2|N|Y|N|2456414|2456533|2456066|2456341|N|N|N|N|N| +2456432|AAAAAAAAAHLHFCAA|2013-05-19|1360|5916|454|2013|0|5|19|2|2013|454|5916|Sunday|2013Q2|N|N|N|2456414|2456533|2456067|2456342|N|N|N|N|N| +2456433|AAAAAAAABHLHFCAA|2013-05-20|1360|5916|454|2013|1|5|20|2|2013|454|5916|Monday|2013Q2|N|N|N|2456414|2456533|2456068|2456343|N|N|N|N|N| +2456434|AAAAAAAACHLHFCAA|2013-05-21|1360|5917|454|2013|2|5|21|2|2013|454|5917|Tuesday|2013Q2|N|N|N|2456414|2456533|2456069|2456344|N|N|N|N|N| +2456435|AAAAAAAADHLHFCAA|2013-05-22|1360|5917|454|2013|3|5|22|2|2013|454|5917|Wednesday|2013Q2|N|N|N|2456414|2456533|2456070|2456345|N|N|N|N|N| +2456436|AAAAAAAAEHLHFCAA|2013-05-23|1360|5917|454|2013|4|5|23|2|2013|454|5917|Thursday|2013Q2|N|N|N|2456414|2456533|2456071|2456346|N|N|N|N|N| +2456437|AAAAAAAAFHLHFCAA|2013-05-24|1360|5917|454|2013|5|5|24|2|2013|454|5917|Friday|2013Q2|N|Y|N|2456414|2456533|2456072|2456347|N|N|N|N|N| +2456438|AAAAAAAAGHLHFCAA|2013-05-25|1360|5917|454|2013|6|5|25|2|2013|454|5917|Saturday|2013Q2|N|Y|N|2456414|2456533|2456073|2456348|N|N|N|N|N| +2456439|AAAAAAAAHHLHFCAA|2013-05-26|1360|5917|454|2013|0|5|26|2|2013|454|5917|Sunday|2013Q2|N|N|N|2456414|2456533|2456074|2456349|N|N|N|N|N| +2456440|AAAAAAAAIHLHFCAA|2013-05-27|1360|5917|454|2013|1|5|27|2|2013|454|5917|Monday|2013Q2|N|N|N|2456414|2456533|2456075|2456350|N|N|N|N|N| +2456441|AAAAAAAAJHLHFCAA|2013-05-28|1360|5918|454|2013|2|5|28|2|2013|454|5918|Tuesday|2013Q2|N|N|N|2456414|2456533|2456076|2456351|N|N|N|N|N| +2456442|AAAAAAAAKHLHFCAA|2013-05-29|1360|5918|454|2013|3|5|29|2|2013|454|5918|Wednesday|2013Q2|N|N|N|2456414|2456533|2456077|2456352|N|N|N|N|N| +2456443|AAAAAAAALHLHFCAA|2013-05-30|1360|5918|454|2013|4|5|30|2|2013|454|5918|Thursday|2013Q2|N|N|N|2456414|2456533|2456078|2456353|N|N|N|N|N| +2456444|AAAAAAAAMHLHFCAA|2013-05-31|1360|5918|454|2013|5|5|31|2|2013|454|5918|Friday|2013Q2|N|Y|N|2456414|2456533|2456079|2456354|N|N|N|N|N| +2456445|AAAAAAAANHLHFCAA|2013-06-01|1361|5918|455|2013|6|6|1|2|2013|455|5918|Saturday|2013Q2|N|Y|N|2456445|2456595|2456080|2456355|N|N|N|N|N| +2456446|AAAAAAAAOHLHFCAA|2013-06-02|1361|5918|455|2013|0|6|2|2|2013|455|5918|Sunday|2013Q2|N|N|N|2456445|2456595|2456081|2456356|N|N|N|N|N| +2456447|AAAAAAAAPHLHFCAA|2013-06-03|1361|5918|455|2013|1|6|3|2|2013|455|5918|Monday|2013Q2|N|N|N|2456445|2456595|2456082|2456357|N|N|N|N|N| +2456448|AAAAAAAAAILHFCAA|2013-06-04|1361|5919|455|2013|2|6|4|2|2013|455|5919|Tuesday|2013Q2|N|N|N|2456445|2456595|2456083|2456358|N|N|N|N|N| +2456449|AAAAAAAABILHFCAA|2013-06-05|1361|5919|455|2013|3|6|5|2|2013|455|5919|Wednesday|2013Q2|N|N|N|2456445|2456595|2456084|2456359|N|N|N|N|N| +2456450|AAAAAAAACILHFCAA|2013-06-06|1361|5919|455|2013|4|6|6|2|2013|455|5919|Thursday|2013Q2|N|N|N|2456445|2456595|2456085|2456360|N|N|N|N|N| +2456451|AAAAAAAADILHFCAA|2013-06-07|1361|5919|455|2013|5|6|7|2|2013|455|5919|Friday|2013Q2|N|Y|N|2456445|2456595|2456086|2456361|N|N|N|N|N| +2456452|AAAAAAAAEILHFCAA|2013-06-08|1361|5919|455|2013|6|6|8|2|2013|455|5919|Saturday|2013Q2|N|Y|N|2456445|2456595|2456087|2456362|N|N|N|N|N| +2456453|AAAAAAAAFILHFCAA|2013-06-09|1361|5919|455|2013|0|6|9|2|2013|455|5919|Sunday|2013Q2|N|N|N|2456445|2456595|2456088|2456363|N|N|N|N|N| +2456454|AAAAAAAAGILHFCAA|2013-06-10|1361|5919|455|2013|1|6|10|2|2013|455|5919|Monday|2013Q2|N|N|N|2456445|2456595|2456089|2456364|N|N|N|N|N| +2456455|AAAAAAAAHILHFCAA|2013-06-11|1361|5920|455|2013|2|6|11|2|2013|455|5920|Tuesday|2013Q2|N|N|N|2456445|2456595|2456090|2456365|N|N|N|N|N| +2456456|AAAAAAAAIILHFCAA|2013-06-12|1361|5920|455|2013|3|6|12|2|2013|455|5920|Wednesday|2013Q2|N|N|N|2456445|2456595|2456091|2456366|N|N|N|N|N| +2456457|AAAAAAAAJILHFCAA|2013-06-13|1361|5920|455|2013|4|6|13|2|2013|455|5920|Thursday|2013Q2|N|N|N|2456445|2456595|2456092|2456367|N|N|N|N|N| +2456458|AAAAAAAAKILHFCAA|2013-06-14|1361|5920|455|2013|5|6|14|2|2013|455|5920|Friday|2013Q2|N|Y|N|2456445|2456595|2456093|2456368|N|N|N|N|N| +2456459|AAAAAAAALILHFCAA|2013-06-15|1361|5920|455|2013|6|6|15|2|2013|455|5920|Saturday|2013Q2|N|Y|N|2456445|2456595|2456094|2456369|N|N|N|N|N| +2456460|AAAAAAAAMILHFCAA|2013-06-16|1361|5920|455|2013|0|6|16|2|2013|455|5920|Sunday|2013Q2|N|N|N|2456445|2456595|2456095|2456370|N|N|N|N|N| +2456461|AAAAAAAANILHFCAA|2013-06-17|1361|5920|455|2013|1|6|17|2|2013|455|5920|Monday|2013Q2|N|N|N|2456445|2456595|2456096|2456371|N|N|N|N|N| +2456462|AAAAAAAAOILHFCAA|2013-06-18|1361|5921|455|2013|2|6|18|2|2013|455|5921|Tuesday|2013Q2|N|N|N|2456445|2456595|2456097|2456372|N|N|N|N|N| +2456463|AAAAAAAAPILHFCAA|2013-06-19|1361|5921|455|2013|3|6|19|2|2013|455|5921|Wednesday|2013Q2|N|N|N|2456445|2456595|2456098|2456373|N|N|N|N|N| +2456464|AAAAAAAAAJLHFCAA|2013-06-20|1361|5921|455|2013|4|6|20|2|2013|455|5921|Thursday|2013Q2|N|N|N|2456445|2456595|2456099|2456374|N|N|N|N|N| +2456465|AAAAAAAABJLHFCAA|2013-06-21|1361|5921|455|2013|5|6|21|2|2013|455|5921|Friday|2013Q2|N|Y|N|2456445|2456595|2456100|2456375|N|N|N|N|N| +2456466|AAAAAAAACJLHFCAA|2013-06-22|1361|5921|455|2013|6|6|22|2|2013|455|5921|Saturday|2013Q2|N|Y|N|2456445|2456595|2456101|2456376|N|N|N|N|N| +2456467|AAAAAAAADJLHFCAA|2013-06-23|1361|5921|455|2013|0|6|23|2|2013|455|5921|Sunday|2013Q2|N|N|N|2456445|2456595|2456102|2456377|N|N|N|N|N| +2456468|AAAAAAAAEJLHFCAA|2013-06-24|1361|5921|455|2013|1|6|24|2|2013|455|5921|Monday|2013Q2|N|N|N|2456445|2456595|2456103|2456378|N|N|N|N|N| +2456469|AAAAAAAAFJLHFCAA|2013-06-25|1361|5922|455|2013|2|6|25|2|2013|455|5922|Tuesday|2013Q2|N|N|N|2456445|2456595|2456104|2456379|N|N|N|N|N| +2456470|AAAAAAAAGJLHFCAA|2013-06-26|1361|5922|455|2013|3|6|26|2|2013|455|5922|Wednesday|2013Q2|N|N|N|2456445|2456595|2456105|2456380|N|N|N|N|N| +2456471|AAAAAAAAHJLHFCAA|2013-06-27|1361|5922|455|2013|4|6|27|2|2013|455|5922|Thursday|2013Q2|N|N|N|2456445|2456595|2456106|2456381|N|N|N|N|N| +2456472|AAAAAAAAIJLHFCAA|2013-06-28|1361|5922|455|2013|5|6|28|2|2013|455|5922|Friday|2013Q2|N|Y|N|2456445|2456595|2456107|2456382|N|N|N|N|N| +2456473|AAAAAAAAJJLHFCAA|2013-06-29|1361|5922|455|2013|6|6|29|2|2013|455|5922|Saturday|2013Q2|N|Y|N|2456445|2456595|2456108|2456383|N|N|N|N|N| +2456474|AAAAAAAAKJLHFCAA|2013-06-30|1361|5922|455|2013|0|6|30|2|2013|455|5922|Sunday|2013Q2|N|N|N|2456445|2456595|2456109|2456384|N|N|N|N|N| +2456475|AAAAAAAALJLHFCAA|2013-07-01|1362|5922|455|2013|1|7|1|2|2013|455|5922|Monday|2013Q2|N|N|N|2456475|2456655|2456110|2456384|N|N|N|N|N| +2456476|AAAAAAAAMJLHFCAA|2013-07-02|1362|5923|455|2013|2|7|2|3|2013|455|5923|Tuesday|2013Q3|N|N|N|2456475|2456655|2456111|2456385|N|N|N|N|N| +2456477|AAAAAAAANJLHFCAA|2013-07-03|1362|5923|455|2013|3|7|3|3|2013|455|5923|Wednesday|2013Q3|N|N|N|2456475|2456655|2456112|2456386|N|N|N|N|N| +2456478|AAAAAAAAOJLHFCAA|2013-07-04|1362|5923|455|2013|4|7|4|3|2013|455|5923|Thursday|2013Q3|N|N|N|2456475|2456655|2456113|2456387|N|N|N|N|N| +2456479|AAAAAAAAPJLHFCAA|2013-07-05|1362|5923|455|2013|5|7|5|3|2013|455|5923|Friday|2013Q3|Y|Y|N|2456475|2456655|2456114|2456388|N|N|N|N|N| +2456480|AAAAAAAAAKLHFCAA|2013-07-06|1362|5923|455|2013|6|7|6|3|2013|455|5923|Saturday|2013Q3|N|Y|Y|2456475|2456655|2456115|2456389|N|N|N|N|N| +2456481|AAAAAAAABKLHFCAA|2013-07-07|1362|5923|455|2013|0|7|7|3|2013|455|5923|Sunday|2013Q3|N|N|N|2456475|2456655|2456116|2456390|N|N|N|N|N| +2456482|AAAAAAAACKLHFCAA|2013-07-08|1362|5923|455|2013|1|7|8|3|2013|455|5923|Monday|2013Q3|N|N|N|2456475|2456655|2456117|2456391|N|N|N|N|N| +2456483|AAAAAAAADKLHFCAA|2013-07-09|1362|5924|455|2013|2|7|9|3|2013|455|5924|Tuesday|2013Q3|N|N|N|2456475|2456655|2456118|2456392|N|N|N|N|N| +2456484|AAAAAAAAEKLHFCAA|2013-07-10|1362|5924|455|2013|3|7|10|3|2013|455|5924|Wednesday|2013Q3|N|N|N|2456475|2456655|2456119|2456393|N|N|N|N|N| +2456485|AAAAAAAAFKLHFCAA|2013-07-11|1362|5924|455|2013|4|7|11|3|2013|455|5924|Thursday|2013Q3|N|N|N|2456475|2456655|2456120|2456394|N|N|N|N|N| +2456486|AAAAAAAAGKLHFCAA|2013-07-12|1362|5924|455|2013|5|7|12|3|2013|455|5924|Friday|2013Q3|N|Y|N|2456475|2456655|2456121|2456395|N|N|N|N|N| +2456487|AAAAAAAAHKLHFCAA|2013-07-13|1362|5924|455|2013|6|7|13|3|2013|455|5924|Saturday|2013Q3|N|Y|N|2456475|2456655|2456122|2456396|N|N|N|N|N| +2456488|AAAAAAAAIKLHFCAA|2013-07-14|1362|5924|455|2013|0|7|14|3|2013|455|5924|Sunday|2013Q3|N|N|N|2456475|2456655|2456123|2456397|N|N|N|N|N| +2456489|AAAAAAAAJKLHFCAA|2013-07-15|1362|5924|455|2013|1|7|15|3|2013|455|5924|Monday|2013Q3|N|N|N|2456475|2456655|2456124|2456398|N|N|N|N|N| +2456490|AAAAAAAAKKLHFCAA|2013-07-16|1362|5925|455|2013|2|7|16|3|2013|455|5925|Tuesday|2013Q3|N|N|N|2456475|2456655|2456125|2456399|N|N|N|N|N| +2456491|AAAAAAAALKLHFCAA|2013-07-17|1362|5925|455|2013|3|7|17|3|2013|455|5925|Wednesday|2013Q3|N|N|N|2456475|2456655|2456126|2456400|N|N|N|N|N| +2456492|AAAAAAAAMKLHFCAA|2013-07-18|1362|5925|455|2013|4|7|18|3|2013|455|5925|Thursday|2013Q3|N|N|N|2456475|2456655|2456127|2456401|N|N|N|N|N| +2456493|AAAAAAAANKLHFCAA|2013-07-19|1362|5925|455|2013|5|7|19|3|2013|455|5925|Friday|2013Q3|N|Y|N|2456475|2456655|2456128|2456402|N|N|N|N|N| +2456494|AAAAAAAAOKLHFCAA|2013-07-20|1362|5925|455|2013|6|7|20|3|2013|455|5925|Saturday|2013Q3|N|Y|N|2456475|2456655|2456129|2456403|N|N|N|N|N| +2456495|AAAAAAAAPKLHFCAA|2013-07-21|1362|5925|455|2013|0|7|21|3|2013|455|5925|Sunday|2013Q3|N|N|N|2456475|2456655|2456130|2456404|N|N|N|N|N| +2456496|AAAAAAAAALLHFCAA|2013-07-22|1362|5925|455|2013|1|7|22|3|2013|455|5925|Monday|2013Q3|N|N|N|2456475|2456655|2456131|2456405|N|N|N|N|N| +2456497|AAAAAAAABLLHFCAA|2013-07-23|1362|5926|455|2013|2|7|23|3|2013|455|5926|Tuesday|2013Q3|N|N|N|2456475|2456655|2456132|2456406|N|N|N|N|N| +2456498|AAAAAAAACLLHFCAA|2013-07-24|1362|5926|455|2013|3|7|24|3|2013|455|5926|Wednesday|2013Q3|N|N|N|2456475|2456655|2456133|2456407|N|N|N|N|N| +2456499|AAAAAAAADLLHFCAA|2013-07-25|1362|5926|455|2013|4|7|25|3|2013|455|5926|Thursday|2013Q3|N|N|N|2456475|2456655|2456134|2456408|N|N|N|N|N| +2456500|AAAAAAAAELLHFCAA|2013-07-26|1362|5926|455|2013|5|7|26|3|2013|455|5926|Friday|2013Q3|N|Y|N|2456475|2456655|2456135|2456409|N|N|N|N|N| +2456501|AAAAAAAAFLLHFCAA|2013-07-27|1362|5926|455|2013|6|7|27|3|2013|455|5926|Saturday|2013Q3|N|Y|N|2456475|2456655|2456136|2456410|N|N|N|N|N| +2456502|AAAAAAAAGLLHFCAA|2013-07-28|1362|5926|455|2013|0|7|28|3|2013|455|5926|Sunday|2013Q3|N|N|N|2456475|2456655|2456137|2456411|N|N|N|N|N| +2456503|AAAAAAAAHLLHFCAA|2013-07-29|1362|5926|455|2013|1|7|29|3|2013|455|5926|Monday|2013Q3|N|N|N|2456475|2456655|2456138|2456412|N|N|N|N|N| +2456504|AAAAAAAAILLHFCAA|2013-07-30|1362|5927|455|2013|2|7|30|3|2013|455|5927|Tuesday|2013Q3|N|N|N|2456475|2456655|2456139|2456413|N|N|N|N|N| +2456505|AAAAAAAAJLLHFCAA|2013-07-31|1362|5927|455|2013|3|7|31|3|2013|455|5927|Wednesday|2013Q3|N|N|N|2456475|2456655|2456140|2456414|N|N|N|N|N| +2456506|AAAAAAAAKLLHFCAA|2013-08-01|1363|5927|455|2013|4|8|1|3|2013|455|5927|Thursday|2013Q3|N|N|N|2456506|2456717|2456141|2456415|N|N|N|N|N| +2456507|AAAAAAAALLLHFCAA|2013-08-02|1363|5927|455|2013|5|8|2|3|2013|455|5927|Friday|2013Q3|N|Y|N|2456506|2456717|2456142|2456416|N|N|N|N|N| +2456508|AAAAAAAAMLLHFCAA|2013-08-03|1363|5927|455|2013|6|8|3|3|2013|455|5927|Saturday|2013Q3|N|Y|N|2456506|2456717|2456143|2456417|N|N|N|N|N| +2456509|AAAAAAAANLLHFCAA|2013-08-04|1363|5927|455|2013|0|8|4|3|2013|455|5927|Sunday|2013Q3|N|N|N|2456506|2456717|2456144|2456418|N|N|N|N|N| +2456510|AAAAAAAAOLLHFCAA|2013-08-05|1363|5927|455|2013|1|8|5|3|2013|455|5927|Monday|2013Q3|N|N|N|2456506|2456717|2456145|2456419|N|N|N|N|N| +2456511|AAAAAAAAPLLHFCAA|2013-08-06|1363|5928|455|2013|2|8|6|3|2013|455|5928|Tuesday|2013Q3|N|N|N|2456506|2456717|2456146|2456420|N|N|N|N|N| +2456512|AAAAAAAAAMLHFCAA|2013-08-07|1363|5928|455|2013|3|8|7|3|2013|455|5928|Wednesday|2013Q3|N|N|N|2456506|2456717|2456147|2456421|N|N|N|N|N| +2456513|AAAAAAAABMLHFCAA|2013-08-08|1363|5928|455|2013|4|8|8|3|2013|455|5928|Thursday|2013Q3|N|N|N|2456506|2456717|2456148|2456422|N|N|N|N|N| +2456514|AAAAAAAACMLHFCAA|2013-08-09|1363|5928|455|2013|5|8|9|3|2013|455|5928|Friday|2013Q3|N|Y|N|2456506|2456717|2456149|2456423|N|N|N|N|N| +2456515|AAAAAAAADMLHFCAA|2013-08-10|1363|5928|455|2013|6|8|10|3|2013|455|5928|Saturday|2013Q3|N|Y|N|2456506|2456717|2456150|2456424|N|N|N|N|N| +2456516|AAAAAAAAEMLHFCAA|2013-08-11|1363|5928|455|2013|0|8|11|3|2013|455|5928|Sunday|2013Q3|N|N|N|2456506|2456717|2456151|2456425|N|N|N|N|N| +2456517|AAAAAAAAFMLHFCAA|2013-08-12|1363|5928|455|2013|1|8|12|3|2013|455|5928|Monday|2013Q3|N|N|N|2456506|2456717|2456152|2456426|N|N|N|N|N| +2456518|AAAAAAAAGMLHFCAA|2013-08-13|1363|5929|455|2013|2|8|13|3|2013|455|5929|Tuesday|2013Q3|N|N|N|2456506|2456717|2456153|2456427|N|N|N|N|N| +2456519|AAAAAAAAHMLHFCAA|2013-08-14|1363|5929|455|2013|3|8|14|3|2013|455|5929|Wednesday|2013Q3|N|N|N|2456506|2456717|2456154|2456428|N|N|N|N|N| +2456520|AAAAAAAAIMLHFCAA|2013-08-15|1363|5929|455|2013|4|8|15|3|2013|455|5929|Thursday|2013Q3|N|N|N|2456506|2456717|2456155|2456429|N|N|N|N|N| +2456521|AAAAAAAAJMLHFCAA|2013-08-16|1363|5929|455|2013|5|8|16|3|2013|455|5929|Friday|2013Q3|N|Y|N|2456506|2456717|2456156|2456430|N|N|N|N|N| +2456522|AAAAAAAAKMLHFCAA|2013-08-17|1363|5929|455|2013|6|8|17|3|2013|455|5929|Saturday|2013Q3|N|Y|N|2456506|2456717|2456157|2456431|N|N|N|N|N| +2456523|AAAAAAAALMLHFCAA|2013-08-18|1363|5929|455|2013|0|8|18|3|2013|455|5929|Sunday|2013Q3|N|N|N|2456506|2456717|2456158|2456432|N|N|N|N|N| +2456524|AAAAAAAAMMLHFCAA|2013-08-19|1363|5929|455|2013|1|8|19|3|2013|455|5929|Monday|2013Q3|N|N|N|2456506|2456717|2456159|2456433|N|N|N|N|N| +2456525|AAAAAAAANMLHFCAA|2013-08-20|1363|5930|455|2013|2|8|20|3|2013|455|5930|Tuesday|2013Q3|N|N|N|2456506|2456717|2456160|2456434|N|N|N|N|N| +2456526|AAAAAAAAOMLHFCAA|2013-08-21|1363|5930|455|2013|3|8|21|3|2013|455|5930|Wednesday|2013Q3|N|N|N|2456506|2456717|2456161|2456435|N|N|N|N|N| +2456527|AAAAAAAAPMLHFCAA|2013-08-22|1363|5930|455|2013|4|8|22|3|2013|455|5930|Thursday|2013Q3|N|N|N|2456506|2456717|2456162|2456436|N|N|N|N|N| +2456528|AAAAAAAAANLHFCAA|2013-08-23|1363|5930|455|2013|5|8|23|3|2013|455|5930|Friday|2013Q3|N|Y|N|2456506|2456717|2456163|2456437|N|N|N|N|N| +2456529|AAAAAAAABNLHFCAA|2013-08-24|1363|5930|455|2013|6|8|24|3|2013|455|5930|Saturday|2013Q3|N|Y|N|2456506|2456717|2456164|2456438|N|N|N|N|N| +2456530|AAAAAAAACNLHFCAA|2013-08-25|1363|5930|455|2013|0|8|25|3|2013|455|5930|Sunday|2013Q3|N|N|N|2456506|2456717|2456165|2456439|N|N|N|N|N| +2456531|AAAAAAAADNLHFCAA|2013-08-26|1363|5930|455|2013|1|8|26|3|2013|455|5930|Monday|2013Q3|N|N|N|2456506|2456717|2456166|2456440|N|N|N|N|N| +2456532|AAAAAAAAENLHFCAA|2013-08-27|1363|5931|455|2013|2|8|27|3|2013|455|5931|Tuesday|2013Q3|N|N|N|2456506|2456717|2456167|2456441|N|N|N|N|N| +2456533|AAAAAAAAFNLHFCAA|2013-08-28|1363|5931|455|2013|3|8|28|3|2013|455|5931|Wednesday|2013Q3|N|N|N|2456506|2456717|2456168|2456442|N|N|N|N|N| +2456534|AAAAAAAAGNLHFCAA|2013-08-29|1363|5931|455|2013|4|8|29|3|2013|455|5931|Thursday|2013Q3|N|N|N|2456506|2456717|2456169|2456443|N|N|N|N|N| +2456535|AAAAAAAAHNLHFCAA|2013-08-30|1363|5931|455|2013|5|8|30|3|2013|455|5931|Friday|2013Q3|N|Y|N|2456506|2456717|2456170|2456444|N|N|N|N|N| +2456536|AAAAAAAAINLHFCAA|2013-08-31|1363|5931|455|2013|6|8|31|3|2013|455|5931|Saturday|2013Q3|N|Y|N|2456506|2456717|2456171|2456445|N|N|N|N|N| +2456537|AAAAAAAAJNLHFCAA|2013-09-01|1364|5931|456|2013|0|9|1|3|2013|456|5931|Sunday|2013Q3|N|N|N|2456537|2456779|2456172|2456446|N|N|N|N|N| +2456538|AAAAAAAAKNLHFCAA|2013-09-02|1364|5931|456|2013|1|9|2|3|2013|456|5931|Monday|2013Q3|N|N|N|2456537|2456779|2456173|2456447|N|N|N|N|N| +2456539|AAAAAAAALNLHFCAA|2013-09-03|1364|5932|456|2013|2|9|3|3|2013|456|5932|Tuesday|2013Q3|N|N|N|2456537|2456779|2456174|2456448|N|N|N|N|N| +2456540|AAAAAAAAMNLHFCAA|2013-09-04|1364|5932|456|2013|3|9|4|3|2013|456|5932|Wednesday|2013Q3|N|N|N|2456537|2456779|2456175|2456449|N|N|N|N|N| +2456541|AAAAAAAANNLHFCAA|2013-09-05|1364|5932|456|2013|4|9|5|3|2013|456|5932|Thursday|2013Q3|N|N|N|2456537|2456779|2456176|2456450|N|N|N|N|N| +2456542|AAAAAAAAONLHFCAA|2013-09-06|1364|5932|456|2013|5|9|6|3|2013|456|5932|Friday|2013Q3|N|Y|N|2456537|2456779|2456177|2456451|N|N|N|N|N| +2456543|AAAAAAAAPNLHFCAA|2013-09-07|1364|5932|456|2013|6|9|7|3|2013|456|5932|Saturday|2013Q3|N|Y|N|2456537|2456779|2456178|2456452|N|N|N|N|N| +2456544|AAAAAAAAAOLHFCAA|2013-09-08|1364|5932|456|2013|0|9|8|3|2013|456|5932|Sunday|2013Q3|N|N|N|2456537|2456779|2456179|2456453|N|N|N|N|N| +2456545|AAAAAAAABOLHFCAA|2013-09-09|1364|5932|456|2013|1|9|9|3|2013|456|5932|Monday|2013Q3|N|N|N|2456537|2456779|2456180|2456454|N|N|N|N|N| +2456546|AAAAAAAACOLHFCAA|2013-09-10|1364|5933|456|2013|2|9|10|3|2013|456|5933|Tuesday|2013Q3|N|N|N|2456537|2456779|2456181|2456455|N|N|N|N|N| +2456547|AAAAAAAADOLHFCAA|2013-09-11|1364|5933|456|2013|3|9|11|3|2013|456|5933|Wednesday|2013Q3|N|N|N|2456537|2456779|2456182|2456456|N|N|N|N|N| +2456548|AAAAAAAAEOLHFCAA|2013-09-12|1364|5933|456|2013|4|9|12|3|2013|456|5933|Thursday|2013Q3|N|N|N|2456537|2456779|2456183|2456457|N|N|N|N|N| +2456549|AAAAAAAAFOLHFCAA|2013-09-13|1364|5933|456|2013|5|9|13|3|2013|456|5933|Friday|2013Q3|N|Y|N|2456537|2456779|2456184|2456458|N|N|N|N|N| +2456550|AAAAAAAAGOLHFCAA|2013-09-14|1364|5933|456|2013|6|9|14|3|2013|456|5933|Saturday|2013Q3|N|Y|N|2456537|2456779|2456185|2456459|N|N|N|N|N| +2456551|AAAAAAAAHOLHFCAA|2013-09-15|1364|5933|456|2013|0|9|15|3|2013|456|5933|Sunday|2013Q3|N|N|N|2456537|2456779|2456186|2456460|N|N|N|N|N| +2456552|AAAAAAAAIOLHFCAA|2013-09-16|1364|5933|456|2013|1|9|16|3|2013|456|5933|Monday|2013Q3|N|N|N|2456537|2456779|2456187|2456461|N|N|N|N|N| +2456553|AAAAAAAAJOLHFCAA|2013-09-17|1364|5934|456|2013|2|9|17|3|2013|456|5934|Tuesday|2013Q3|N|N|N|2456537|2456779|2456188|2456462|N|N|N|N|N| +2456554|AAAAAAAAKOLHFCAA|2013-09-18|1364|5934|456|2013|3|9|18|3|2013|456|5934|Wednesday|2013Q3|N|N|N|2456537|2456779|2456189|2456463|N|N|N|N|N| +2456555|AAAAAAAALOLHFCAA|2013-09-19|1364|5934|456|2013|4|9|19|3|2013|456|5934|Thursday|2013Q3|N|N|N|2456537|2456779|2456190|2456464|N|N|N|N|N| +2456556|AAAAAAAAMOLHFCAA|2013-09-20|1364|5934|456|2013|5|9|20|3|2013|456|5934|Friday|2013Q3|N|Y|N|2456537|2456779|2456191|2456465|N|N|N|N|N| +2456557|AAAAAAAANOLHFCAA|2013-09-21|1364|5934|456|2013|6|9|21|3|2013|456|5934|Saturday|2013Q3|N|Y|N|2456537|2456779|2456192|2456466|N|N|N|N|N| +2456558|AAAAAAAAOOLHFCAA|2013-09-22|1364|5934|456|2013|0|9|22|3|2013|456|5934|Sunday|2013Q3|N|N|N|2456537|2456779|2456193|2456467|N|N|N|N|N| +2456559|AAAAAAAAPOLHFCAA|2013-09-23|1364|5934|456|2013|1|9|23|3|2013|456|5934|Monday|2013Q3|N|N|N|2456537|2456779|2456194|2456468|N|N|N|N|N| +2456560|AAAAAAAAAPLHFCAA|2013-09-24|1364|5935|456|2013|2|9|24|3|2013|456|5935|Tuesday|2013Q3|N|N|N|2456537|2456779|2456195|2456469|N|N|N|N|N| +2456561|AAAAAAAABPLHFCAA|2013-09-25|1364|5935|456|2013|3|9|25|3|2013|456|5935|Wednesday|2013Q3|N|N|N|2456537|2456779|2456196|2456470|N|N|N|N|N| +2456562|AAAAAAAACPLHFCAA|2013-09-26|1364|5935|456|2013|4|9|26|3|2013|456|5935|Thursday|2013Q3|N|N|N|2456537|2456779|2456197|2456471|N|N|N|N|N| +2456563|AAAAAAAADPLHFCAA|2013-09-27|1364|5935|456|2013|5|9|27|3|2013|456|5935|Friday|2013Q3|N|Y|N|2456537|2456779|2456198|2456472|N|N|N|N|N| +2456564|AAAAAAAAEPLHFCAA|2013-09-28|1364|5935|456|2013|6|9|28|3|2013|456|5935|Saturday|2013Q3|N|Y|N|2456537|2456779|2456199|2456473|N|N|N|N|N| +2456565|AAAAAAAAFPLHFCAA|2013-09-29|1364|5935|456|2013|0|9|29|3|2013|456|5935|Sunday|2013Q3|N|N|N|2456537|2456779|2456200|2456474|N|N|N|N|N| +2456566|AAAAAAAAGPLHFCAA|2013-09-30|1364|5935|456|2013|1|9|30|3|2013|456|5935|Monday|2013Q3|N|N|N|2456537|2456779|2456201|2456475|N|N|N|N|N| +2456567|AAAAAAAAHPLHFCAA|2013-10-01|1365|5936|456|2013|2|10|1|3|2013|456|5936|Tuesday|2013Q3|N|N|N|2456567|2456839|2456202|2456475|N|N|N|N|N| +2456568|AAAAAAAAIPLHFCAA|2013-10-02|1365|5936|456|2013|3|10|2|4|2013|456|5936|Wednesday|2013Q4|N|N|N|2456567|2456839|2456203|2456476|N|N|N|N|N| +2456569|AAAAAAAAJPLHFCAA|2013-10-03|1365|5936|456|2013|4|10|3|4|2013|456|5936|Thursday|2013Q4|N|N|N|2456567|2456839|2456204|2456477|N|N|N|N|N| +2456570|AAAAAAAAKPLHFCAA|2013-10-04|1365|5936|456|2013|5|10|4|4|2013|456|5936|Friday|2013Q4|N|Y|N|2456567|2456839|2456205|2456478|N|N|N|N|N| +2456571|AAAAAAAALPLHFCAA|2013-10-05|1365|5936|456|2013|6|10|5|4|2013|456|5936|Saturday|2013Q4|N|Y|N|2456567|2456839|2456206|2456479|N|N|N|N|N| +2456572|AAAAAAAAMPLHFCAA|2013-10-06|1365|5936|456|2013|0|10|6|4|2013|456|5936|Sunday|2013Q4|N|N|N|2456567|2456839|2456207|2456480|N|N|N|N|N| +2456573|AAAAAAAANPLHFCAA|2013-10-07|1365|5936|456|2013|1|10|7|4|2013|456|5936|Monday|2013Q4|N|N|N|2456567|2456839|2456208|2456481|N|N|N|N|N| +2456574|AAAAAAAAOPLHFCAA|2013-10-08|1365|5937|456|2013|2|10|8|4|2013|456|5937|Tuesday|2013Q4|N|N|N|2456567|2456839|2456209|2456482|N|N|N|N|N| +2456575|AAAAAAAAPPLHFCAA|2013-10-09|1365|5937|456|2013|3|10|9|4|2013|456|5937|Wednesday|2013Q4|N|N|N|2456567|2456839|2456210|2456483|N|N|N|N|N| +2456576|AAAAAAAAAAMHFCAA|2013-10-10|1365|5937|456|2013|4|10|10|4|2013|456|5937|Thursday|2013Q4|N|N|N|2456567|2456839|2456211|2456484|N|N|N|N|N| +2456577|AAAAAAAABAMHFCAA|2013-10-11|1365|5937|456|2013|5|10|11|4|2013|456|5937|Friday|2013Q4|N|Y|N|2456567|2456839|2456212|2456485|N|N|N|N|N| +2456578|AAAAAAAACAMHFCAA|2013-10-12|1365|5937|456|2013|6|10|12|4|2013|456|5937|Saturday|2013Q4|N|Y|N|2456567|2456839|2456213|2456486|N|N|N|N|N| +2456579|AAAAAAAADAMHFCAA|2013-10-13|1365|5937|456|2013|0|10|13|4|2013|456|5937|Sunday|2013Q4|N|N|N|2456567|2456839|2456214|2456487|N|N|N|N|N| +2456580|AAAAAAAAEAMHFCAA|2013-10-14|1365|5937|456|2013|1|10|14|4|2013|456|5937|Monday|2013Q4|N|N|N|2456567|2456839|2456215|2456488|N|N|N|N|N| +2456581|AAAAAAAAFAMHFCAA|2013-10-15|1365|5938|456|2013|2|10|15|4|2013|456|5938|Tuesday|2013Q4|N|N|N|2456567|2456839|2456216|2456489|N|N|N|N|N| +2456582|AAAAAAAAGAMHFCAA|2013-10-16|1365|5938|456|2013|3|10|16|4|2013|456|5938|Wednesday|2013Q4|N|N|N|2456567|2456839|2456217|2456490|N|N|N|N|N| +2456583|AAAAAAAAHAMHFCAA|2013-10-17|1365|5938|456|2013|4|10|17|4|2013|456|5938|Thursday|2013Q4|N|N|N|2456567|2456839|2456218|2456491|N|N|N|N|N| +2456584|AAAAAAAAIAMHFCAA|2013-10-18|1365|5938|456|2013|5|10|18|4|2013|456|5938|Friday|2013Q4|N|Y|N|2456567|2456839|2456219|2456492|N|N|N|N|N| +2456585|AAAAAAAAJAMHFCAA|2013-10-19|1365|5938|456|2013|6|10|19|4|2013|456|5938|Saturday|2013Q4|N|Y|N|2456567|2456839|2456220|2456493|N|N|N|N|N| +2456586|AAAAAAAAKAMHFCAA|2013-10-20|1365|5938|456|2013|0|10|20|4|2013|456|5938|Sunday|2013Q4|N|N|N|2456567|2456839|2456221|2456494|N|N|N|N|N| +2456587|AAAAAAAALAMHFCAA|2013-10-21|1365|5938|456|2013|1|10|21|4|2013|456|5938|Monday|2013Q4|N|N|N|2456567|2456839|2456222|2456495|N|N|N|N|N| +2456588|AAAAAAAAMAMHFCAA|2013-10-22|1365|5939|456|2013|2|10|22|4|2013|456|5939|Tuesday|2013Q4|N|N|N|2456567|2456839|2456223|2456496|N|N|N|N|N| +2456589|AAAAAAAANAMHFCAA|2013-10-23|1365|5939|456|2013|3|10|23|4|2013|456|5939|Wednesday|2013Q4|N|N|N|2456567|2456839|2456224|2456497|N|N|N|N|N| +2456590|AAAAAAAAOAMHFCAA|2013-10-24|1365|5939|456|2013|4|10|24|4|2013|456|5939|Thursday|2013Q4|N|N|N|2456567|2456839|2456225|2456498|N|N|N|N|N| +2456591|AAAAAAAAPAMHFCAA|2013-10-25|1365|5939|456|2013|5|10|25|4|2013|456|5939|Friday|2013Q4|N|Y|N|2456567|2456839|2456226|2456499|N|N|N|N|N| +2456592|AAAAAAAAABMHFCAA|2013-10-26|1365|5939|456|2013|6|10|26|4|2013|456|5939|Saturday|2013Q4|N|Y|N|2456567|2456839|2456227|2456500|N|N|N|N|N| +2456593|AAAAAAAABBMHFCAA|2013-10-27|1365|5939|456|2013|0|10|27|4|2013|456|5939|Sunday|2013Q4|N|N|N|2456567|2456839|2456228|2456501|N|N|N|N|N| +2456594|AAAAAAAACBMHFCAA|2013-10-28|1365|5939|456|2013|1|10|28|4|2013|456|5939|Monday|2013Q4|N|N|N|2456567|2456839|2456229|2456502|N|N|N|N|N| +2456595|AAAAAAAADBMHFCAA|2013-10-29|1365|5940|456|2013|2|10|29|4|2013|456|5940|Tuesday|2013Q4|N|N|N|2456567|2456839|2456230|2456503|N|N|N|N|N| +2456596|AAAAAAAAEBMHFCAA|2013-10-30|1365|5940|456|2013|3|10|30|4|2013|456|5940|Wednesday|2013Q4|N|N|N|2456567|2456839|2456231|2456504|N|N|N|N|N| +2456597|AAAAAAAAFBMHFCAA|2013-10-31|1365|5940|456|2013|4|10|31|4|2013|456|5940|Thursday|2013Q4|N|N|N|2456567|2456839|2456232|2456505|N|N|N|N|N| +2456598|AAAAAAAAGBMHFCAA|2013-11-01|1366|5940|456|2013|5|11|1|4|2013|456|5940|Friday|2013Q4|N|Y|N|2456598|2456901|2456233|2456506|N|N|N|N|N| +2456599|AAAAAAAAHBMHFCAA|2013-11-02|1366|5940|456|2013|6|11|2|4|2013|456|5940|Saturday|2013Q4|N|Y|N|2456598|2456901|2456234|2456507|N|N|N|N|N| +2456600|AAAAAAAAIBMHFCAA|2013-11-03|1366|5940|456|2013|0|11|3|4|2013|456|5940|Sunday|2013Q4|N|N|N|2456598|2456901|2456235|2456508|N|N|N|N|N| +2456601|AAAAAAAAJBMHFCAA|2013-11-04|1366|5940|456|2013|1|11|4|4|2013|456|5940|Monday|2013Q4|N|N|N|2456598|2456901|2456236|2456509|N|N|N|N|N| +2456602|AAAAAAAAKBMHFCAA|2013-11-05|1366|5941|456|2013|2|11|5|4|2013|456|5941|Tuesday|2013Q4|N|N|N|2456598|2456901|2456237|2456510|N|N|N|N|N| +2456603|AAAAAAAALBMHFCAA|2013-11-06|1366|5941|456|2013|3|11|6|4|2013|456|5941|Wednesday|2013Q4|N|N|N|2456598|2456901|2456238|2456511|N|N|N|N|N| +2456604|AAAAAAAAMBMHFCAA|2013-11-07|1366|5941|456|2013|4|11|7|4|2013|456|5941|Thursday|2013Q4|N|N|N|2456598|2456901|2456239|2456512|N|N|N|N|N| +2456605|AAAAAAAANBMHFCAA|2013-11-08|1366|5941|456|2013|5|11|8|4|2013|456|5941|Friday|2013Q4|N|Y|N|2456598|2456901|2456240|2456513|N|N|N|N|N| +2456606|AAAAAAAAOBMHFCAA|2013-11-09|1366|5941|456|2013|6|11|9|4|2013|456|5941|Saturday|2013Q4|N|Y|N|2456598|2456901|2456241|2456514|N|N|N|N|N| +2456607|AAAAAAAAPBMHFCAA|2013-11-10|1366|5941|456|2013|0|11|10|4|2013|456|5941|Sunday|2013Q4|N|N|N|2456598|2456901|2456242|2456515|N|N|N|N|N| +2456608|AAAAAAAAACMHFCAA|2013-11-11|1366|5941|456|2013|1|11|11|4|2013|456|5941|Monday|2013Q4|N|N|N|2456598|2456901|2456243|2456516|N|N|N|N|N| +2456609|AAAAAAAABCMHFCAA|2013-11-12|1366|5942|456|2013|2|11|12|4|2013|456|5942|Tuesday|2013Q4|N|N|N|2456598|2456901|2456244|2456517|N|N|N|N|N| +2456610|AAAAAAAACCMHFCAA|2013-11-13|1366|5942|456|2013|3|11|13|4|2013|456|5942|Wednesday|2013Q4|N|N|N|2456598|2456901|2456245|2456518|N|N|N|N|N| +2456611|AAAAAAAADCMHFCAA|2013-11-14|1366|5942|456|2013|4|11|14|4|2013|456|5942|Thursday|2013Q4|N|N|N|2456598|2456901|2456246|2456519|N|N|N|N|N| +2456612|AAAAAAAAECMHFCAA|2013-11-15|1366|5942|456|2013|5|11|15|4|2013|456|5942|Friday|2013Q4|N|Y|N|2456598|2456901|2456247|2456520|N|N|N|N|N| +2456613|AAAAAAAAFCMHFCAA|2013-11-16|1366|5942|456|2013|6|11|16|4|2013|456|5942|Saturday|2013Q4|N|Y|N|2456598|2456901|2456248|2456521|N|N|N|N|N| +2456614|AAAAAAAAGCMHFCAA|2013-11-17|1366|5942|456|2013|0|11|17|4|2013|456|5942|Sunday|2013Q4|N|N|N|2456598|2456901|2456249|2456522|N|N|N|N|N| +2456615|AAAAAAAAHCMHFCAA|2013-11-18|1366|5942|456|2013|1|11|18|4|2013|456|5942|Monday|2013Q4|N|N|N|2456598|2456901|2456250|2456523|N|N|N|N|N| +2456616|AAAAAAAAICMHFCAA|2013-11-19|1366|5943|456|2013|2|11|19|4|2013|456|5943|Tuesday|2013Q4|N|N|N|2456598|2456901|2456251|2456524|N|N|N|N|N| +2456617|AAAAAAAAJCMHFCAA|2013-11-20|1366|5943|456|2013|3|11|20|4|2013|456|5943|Wednesday|2013Q4|N|N|N|2456598|2456901|2456252|2456525|N|N|N|N|N| +2456618|AAAAAAAAKCMHFCAA|2013-11-21|1366|5943|456|2013|4|11|21|4|2013|456|5943|Thursday|2013Q4|N|N|N|2456598|2456901|2456253|2456526|N|N|N|N|N| +2456619|AAAAAAAALCMHFCAA|2013-11-22|1366|5943|456|2013|5|11|22|4|2013|456|5943|Friday|2013Q4|N|Y|N|2456598|2456901|2456254|2456527|N|N|N|N|N| +2456620|AAAAAAAAMCMHFCAA|2013-11-23|1366|5943|456|2013|6|11|23|4|2013|456|5943|Saturday|2013Q4|N|Y|N|2456598|2456901|2456255|2456528|N|N|N|N|N| +2456621|AAAAAAAANCMHFCAA|2013-11-24|1366|5943|456|2013|0|11|24|4|2013|456|5943|Sunday|2013Q4|N|N|N|2456598|2456901|2456256|2456529|N|N|N|N|N| +2456622|AAAAAAAAOCMHFCAA|2013-11-25|1366|5943|456|2013|1|11|25|4|2013|456|5943|Monday|2013Q4|N|N|N|2456598|2456901|2456257|2456530|N|N|N|N|N| +2456623|AAAAAAAAPCMHFCAA|2013-11-26|1366|5944|456|2013|2|11|26|4|2013|456|5944|Tuesday|2013Q4|N|N|N|2456598|2456901|2456258|2456531|N|N|N|N|N| +2456624|AAAAAAAAADMHFCAA|2013-11-27|1366|5944|456|2013|3|11|27|4|2013|456|5944|Wednesday|2013Q4|N|N|N|2456598|2456901|2456259|2456532|N|N|N|N|N| +2456625|AAAAAAAABDMHFCAA|2013-11-28|1366|5944|456|2013|4|11|28|4|2013|456|5944|Thursday|2013Q4|N|N|N|2456598|2456901|2456260|2456533|N|N|N|N|N| +2456626|AAAAAAAACDMHFCAA|2013-11-29|1366|5944|456|2013|5|11|29|4|2013|456|5944|Friday|2013Q4|N|Y|N|2456598|2456901|2456261|2456534|N|N|N|N|N| +2456627|AAAAAAAADDMHFCAA|2013-11-30|1366|5944|456|2013|6|11|30|4|2013|456|5944|Saturday|2013Q4|N|Y|N|2456598|2456901|2456262|2456535|N|N|N|N|N| +2456628|AAAAAAAAEDMHFCAA|2013-12-01|1367|5944|457|2013|0|12|1|4|2013|457|5944|Sunday|2013Q4|N|N|N|2456628|2456961|2456263|2456536|N|N|N|N|N| +2456629|AAAAAAAAFDMHFCAA|2013-12-02|1367|5944|457|2013|1|12|2|4|2013|457|5944|Monday|2013Q4|N|N|N|2456628|2456961|2456264|2456537|N|N|N|N|N| +2456630|AAAAAAAAGDMHFCAA|2013-12-03|1367|5945|457|2013|2|12|3|4|2013|457|5945|Tuesday|2013Q4|N|N|N|2456628|2456961|2456265|2456538|N|N|N|N|N| +2456631|AAAAAAAAHDMHFCAA|2013-12-04|1367|5945|457|2013|3|12|4|4|2013|457|5945|Wednesday|2013Q4|N|N|N|2456628|2456961|2456266|2456539|N|N|N|N|N| +2456632|AAAAAAAAIDMHFCAA|2013-12-05|1367|5945|457|2013|4|12|5|4|2013|457|5945|Thursday|2013Q4|N|N|N|2456628|2456961|2456267|2456540|N|N|N|N|N| +2456633|AAAAAAAAJDMHFCAA|2013-12-06|1367|5945|457|2013|5|12|6|4|2013|457|5945|Friday|2013Q4|N|Y|N|2456628|2456961|2456268|2456541|N|N|N|N|N| +2456634|AAAAAAAAKDMHFCAA|2013-12-07|1367|5945|457|2013|6|12|7|4|2013|457|5945|Saturday|2013Q4|N|Y|N|2456628|2456961|2456269|2456542|N|N|N|N|N| +2456635|AAAAAAAALDMHFCAA|2013-12-08|1367|5945|457|2013|0|12|8|4|2013|457|5945|Sunday|2013Q4|N|N|N|2456628|2456961|2456270|2456543|N|N|N|N|N| +2456636|AAAAAAAAMDMHFCAA|2013-12-09|1367|5945|457|2013|1|12|9|4|2013|457|5945|Monday|2013Q4|N|N|N|2456628|2456961|2456271|2456544|N|N|N|N|N| +2456637|AAAAAAAANDMHFCAA|2013-12-10|1367|5946|457|2013|2|12|10|4|2013|457|5946|Tuesday|2013Q4|N|N|N|2456628|2456961|2456272|2456545|N|N|N|N|N| +2456638|AAAAAAAAODMHFCAA|2013-12-11|1367|5946|457|2013|3|12|11|4|2013|457|5946|Wednesday|2013Q4|N|N|N|2456628|2456961|2456273|2456546|N|N|N|N|N| +2456639|AAAAAAAAPDMHFCAA|2013-12-12|1367|5946|457|2013|4|12|12|4|2013|457|5946|Thursday|2013Q4|N|N|N|2456628|2456961|2456274|2456547|N|N|N|N|N| +2456640|AAAAAAAAAEMHFCAA|2013-12-13|1367|5946|457|2013|5|12|13|4|2013|457|5946|Friday|2013Q4|N|Y|N|2456628|2456961|2456275|2456548|N|N|N|N|N| +2456641|AAAAAAAABEMHFCAA|2013-12-14|1367|5946|457|2013|6|12|14|4|2013|457|5946|Saturday|2013Q4|N|Y|N|2456628|2456961|2456276|2456549|N|N|N|N|N| +2456642|AAAAAAAACEMHFCAA|2013-12-15|1367|5946|457|2013|0|12|15|4|2013|457|5946|Sunday|2013Q4|N|N|N|2456628|2456961|2456277|2456550|N|N|N|N|N| +2456643|AAAAAAAADEMHFCAA|2013-12-16|1367|5946|457|2013|1|12|16|4|2013|457|5946|Monday|2013Q4|N|N|N|2456628|2456961|2456278|2456551|N|N|N|N|N| +2456644|AAAAAAAAEEMHFCAA|2013-12-17|1367|5947|457|2013|2|12|17|4|2013|457|5947|Tuesday|2013Q4|N|N|N|2456628|2456961|2456279|2456552|N|N|N|N|N| +2456645|AAAAAAAAFEMHFCAA|2013-12-18|1367|5947|457|2013|3|12|18|4|2013|457|5947|Wednesday|2013Q4|N|N|N|2456628|2456961|2456280|2456553|N|N|N|N|N| +2456646|AAAAAAAAGEMHFCAA|2013-12-19|1367|5947|457|2013|4|12|19|4|2013|457|5947|Thursday|2013Q4|N|N|N|2456628|2456961|2456281|2456554|N|N|N|N|N| +2456647|AAAAAAAAHEMHFCAA|2013-12-20|1367|5947|457|2013|5|12|20|4|2013|457|5947|Friday|2013Q4|N|Y|N|2456628|2456961|2456282|2456555|N|N|N|N|N| +2456648|AAAAAAAAIEMHFCAA|2013-12-21|1367|5947|457|2013|6|12|21|4|2013|457|5947|Saturday|2013Q4|N|Y|N|2456628|2456961|2456283|2456556|N|N|N|N|N| +2456649|AAAAAAAAJEMHFCAA|2013-12-22|1367|5947|457|2013|0|12|22|4|2013|457|5947|Sunday|2013Q4|N|N|N|2456628|2456961|2456284|2456557|N|N|N|N|N| +2456650|AAAAAAAAKEMHFCAA|2013-12-23|1367|5947|457|2013|1|12|23|4|2013|457|5947|Monday|2013Q4|N|N|N|2456628|2456961|2456285|2456558|N|N|N|N|N| +2456651|AAAAAAAALEMHFCAA|2013-12-24|1367|5948|457|2013|2|12|24|4|2013|457|5948|Tuesday|2013Q4|N|N|N|2456628|2456961|2456286|2456559|N|N|N|N|N| +2456652|AAAAAAAAMEMHFCAA|2013-12-25|1367|5948|457|2013|3|12|25|4|2013|457|5948|Wednesday|2013Q4|N|N|N|2456628|2456961|2456287|2456560|N|N|N|N|N| +2456653|AAAAAAAANEMHFCAA|2013-12-26|1367|5948|457|2013|4|12|26|4|2013|457|5948|Thursday|2013Q4|Y|N|N|2456628|2456961|2456288|2456561|N|N|N|N|N| +2456654|AAAAAAAAOEMHFCAA|2013-12-27|1367|5948|457|2013|5|12|27|4|2013|457|5948|Friday|2013Q4|N|Y|Y|2456628|2456961|2456289|2456562|N|N|N|N|N| +2456655|AAAAAAAAPEMHFCAA|2013-12-28|1367|5948|457|2013|6|12|28|4|2013|457|5948|Saturday|2013Q4|N|Y|N|2456628|2456961|2456290|2456563|N|N|N|N|N| +2456656|AAAAAAAAAFMHFCAA|2013-12-29|1367|5948|457|2013|0|12|29|4|2013|457|5948|Sunday|2013Q4|N|N|N|2456628|2456961|2456291|2456564|N|N|N|N|N| +2456657|AAAAAAAABFMHFCAA|2013-12-30|1367|5948|457|2013|1|12|30|4|2013|457|5948|Monday|2013Q4|N|N|N|2456628|2456961|2456292|2456565|N|N|N|N|N| +2456658|AAAAAAAACFMHFCAA|2013-12-31|1367|5949|457|2013|2|12|31|4|2013|457|5949|Tuesday|2013Q4|N|N|N|2456628|2456961|2456293|2456566|N|N|N|N|N| +2456659|AAAAAAAADFMHFCAA|2014-01-01|1368|5949|457|2014|3|1|1|1|2014|457|5949|Wednesday|2014Q1|Y|N|N|2456659|2456658|2456294|2456567|N|N|N|N|N| +2456660|AAAAAAAAEFMHFCAA|2014-01-02|1368|5949|457|2014|4|1|2|1|2014|457|5949|Thursday|2014Q1|N|N|Y|2456659|2456658|2456295|2456568|N|N|N|N|N| +2456661|AAAAAAAAFFMHFCAA|2014-01-03|1368|5949|457|2014|5|1|3|1|2014|457|5949|Friday|2014Q1|N|Y|N|2456659|2456658|2456296|2456569|N|N|N|N|N| +2456662|AAAAAAAAGFMHFCAA|2014-01-04|1368|5949|457|2014|6|1|4|1|2014|457|5949|Saturday|2014Q1|N|Y|N|2456659|2456658|2456297|2456570|N|N|N|N|N| +2456663|AAAAAAAAHFMHFCAA|2014-01-05|1368|5949|457|2014|0|1|5|1|2014|457|5949|Sunday|2014Q1|N|N|N|2456659|2456658|2456298|2456571|N|N|N|N|N| +2456664|AAAAAAAAIFMHFCAA|2014-01-06|1368|5949|457|2014|1|1|6|1|2014|457|5949|Monday|2014Q1|N|N|N|2456659|2456658|2456299|2456572|N|N|N|N|N| +2456665|AAAAAAAAJFMHFCAA|2014-01-07|1368|5950|457|2014|2|1|7|1|2014|457|5950|Tuesday|2014Q1|N|N|N|2456659|2456658|2456300|2456573|N|N|N|N|N| +2456666|AAAAAAAAKFMHFCAA|2014-01-08|1368|5950|457|2014|3|1|8|1|2014|457|5950|Wednesday|2014Q1|N|N|N|2456659|2456658|2456301|2456574|N|N|N|N|N| +2456667|AAAAAAAALFMHFCAA|2014-01-09|1368|5950|457|2014|4|1|9|1|2014|457|5950|Thursday|2014Q1|N|N|N|2456659|2456658|2456302|2456575|N|N|N|N|N| +2456668|AAAAAAAAMFMHFCAA|2014-01-10|1368|5950|457|2014|5|1|10|1|2014|457|5950|Friday|2014Q1|N|Y|N|2456659|2456658|2456303|2456576|N|N|N|N|N| +2456669|AAAAAAAANFMHFCAA|2014-01-11|1368|5950|457|2014|6|1|11|1|2014|457|5950|Saturday|2014Q1|N|Y|N|2456659|2456658|2456304|2456577|N|N|N|N|N| +2456670|AAAAAAAAOFMHFCAA|2014-01-12|1368|5950|457|2014|0|1|12|1|2014|457|5950|Sunday|2014Q1|N|N|N|2456659|2456658|2456305|2456578|N|N|N|N|N| +2456671|AAAAAAAAPFMHFCAA|2014-01-13|1368|5950|457|2014|1|1|13|1|2014|457|5950|Monday|2014Q1|N|N|N|2456659|2456658|2456306|2456579|N|N|N|N|N| +2456672|AAAAAAAAAGMHFCAA|2014-01-14|1368|5951|457|2014|2|1|14|1|2014|457|5951|Tuesday|2014Q1|N|N|N|2456659|2456658|2456307|2456580|N|N|N|N|N| +2456673|AAAAAAAABGMHFCAA|2014-01-15|1368|5951|457|2014|3|1|15|1|2014|457|5951|Wednesday|2014Q1|N|N|N|2456659|2456658|2456308|2456581|N|N|N|N|N| +2456674|AAAAAAAACGMHFCAA|2014-01-16|1368|5951|457|2014|4|1|16|1|2014|457|5951|Thursday|2014Q1|N|N|N|2456659|2456658|2456309|2456582|N|N|N|N|N| +2456675|AAAAAAAADGMHFCAA|2014-01-17|1368|5951|457|2014|5|1|17|1|2014|457|5951|Friday|2014Q1|N|Y|N|2456659|2456658|2456310|2456583|N|N|N|N|N| +2456676|AAAAAAAAEGMHFCAA|2014-01-18|1368|5951|457|2014|6|1|18|1|2014|457|5951|Saturday|2014Q1|N|Y|N|2456659|2456658|2456311|2456584|N|N|N|N|N| +2456677|AAAAAAAAFGMHFCAA|2014-01-19|1368|5951|457|2014|0|1|19|1|2014|457|5951|Sunday|2014Q1|N|N|N|2456659|2456658|2456312|2456585|N|N|N|N|N| +2456678|AAAAAAAAGGMHFCAA|2014-01-20|1368|5951|457|2014|1|1|20|1|2014|457|5951|Monday|2014Q1|N|N|N|2456659|2456658|2456313|2456586|N|N|N|N|N| +2456679|AAAAAAAAHGMHFCAA|2014-01-21|1368|5952|457|2014|2|1|21|1|2014|457|5952|Tuesday|2014Q1|N|N|N|2456659|2456658|2456314|2456587|N|N|N|N|N| +2456680|AAAAAAAAIGMHFCAA|2014-01-22|1368|5952|457|2014|3|1|22|1|2014|457|5952|Wednesday|2014Q1|N|N|N|2456659|2456658|2456315|2456588|N|N|N|N|N| +2456681|AAAAAAAAJGMHFCAA|2014-01-23|1368|5952|457|2014|4|1|23|1|2014|457|5952|Thursday|2014Q1|N|N|N|2456659|2456658|2456316|2456589|N|N|N|N|N| +2456682|AAAAAAAAKGMHFCAA|2014-01-24|1368|5952|457|2014|5|1|24|1|2014|457|5952|Friday|2014Q1|N|Y|N|2456659|2456658|2456317|2456590|N|N|N|N|N| +2456683|AAAAAAAALGMHFCAA|2014-01-25|1368|5952|457|2014|6|1|25|1|2014|457|5952|Saturday|2014Q1|N|Y|N|2456659|2456658|2456318|2456591|N|N|N|N|N| +2456684|AAAAAAAAMGMHFCAA|2014-01-26|1368|5952|457|2014|0|1|26|1|2014|457|5952|Sunday|2014Q1|N|N|N|2456659|2456658|2456319|2456592|N|N|N|N|N| +2456685|AAAAAAAANGMHFCAA|2014-01-27|1368|5952|457|2014|1|1|27|1|2014|457|5952|Monday|2014Q1|N|N|N|2456659|2456658|2456320|2456593|N|N|N|N|N| +2456686|AAAAAAAAOGMHFCAA|2014-01-28|1368|5953|457|2014|2|1|28|1|2014|457|5953|Tuesday|2014Q1|N|N|N|2456659|2456658|2456321|2456594|N|N|N|N|N| +2456687|AAAAAAAAPGMHFCAA|2014-01-29|1368|5953|457|2014|3|1|29|1|2014|457|5953|Wednesday|2014Q1|N|N|N|2456659|2456658|2456322|2456595|N|N|N|N|N| +2456688|AAAAAAAAAHMHFCAA|2014-01-30|1368|5953|457|2014|4|1|30|1|2014|457|5953|Thursday|2014Q1|N|N|N|2456659|2456658|2456323|2456596|N|N|N|N|N| +2456689|AAAAAAAABHMHFCAA|2014-01-31|1368|5953|457|2014|5|1|31|1|2014|457|5953|Friday|2014Q1|N|Y|N|2456659|2456658|2456324|2456597|N|N|N|N|N| +2456690|AAAAAAAACHMHFCAA|2014-02-01|1369|5953|457|2014|6|2|1|1|2014|457|5953|Saturday|2014Q1|N|Y|N|2456690|2456720|2456325|2456598|N|N|N|N|N| +2456691|AAAAAAAADHMHFCAA|2014-02-02|1369|5953|457|2014|0|2|2|1|2014|457|5953|Sunday|2014Q1|N|N|N|2456690|2456720|2456326|2456599|N|N|N|N|N| +2456692|AAAAAAAAEHMHFCAA|2014-02-03|1369|5953|457|2014|1|2|3|1|2014|457|5953|Monday|2014Q1|N|N|N|2456690|2456720|2456327|2456600|N|N|N|N|N| +2456693|AAAAAAAAFHMHFCAA|2014-02-04|1369|5954|457|2014|2|2|4|1|2014|457|5954|Tuesday|2014Q1|N|N|N|2456690|2456720|2456328|2456601|N|N|N|N|N| +2456694|AAAAAAAAGHMHFCAA|2014-02-05|1369|5954|457|2014|3|2|5|1|2014|457|5954|Wednesday|2014Q1|N|N|N|2456690|2456720|2456329|2456602|N|N|N|N|N| +2456695|AAAAAAAAHHMHFCAA|2014-02-06|1369|5954|457|2014|4|2|6|1|2014|457|5954|Thursday|2014Q1|N|N|N|2456690|2456720|2456330|2456603|N|N|N|N|N| +2456696|AAAAAAAAIHMHFCAA|2014-02-07|1369|5954|457|2014|5|2|7|1|2014|457|5954|Friday|2014Q1|N|Y|N|2456690|2456720|2456331|2456604|N|N|N|N|N| +2456697|AAAAAAAAJHMHFCAA|2014-02-08|1369|5954|457|2014|6|2|8|1|2014|457|5954|Saturday|2014Q1|N|Y|N|2456690|2456720|2456332|2456605|N|N|N|N|N| +2456698|AAAAAAAAKHMHFCAA|2014-02-09|1369|5954|457|2014|0|2|9|1|2014|457|5954|Sunday|2014Q1|N|N|N|2456690|2456720|2456333|2456606|N|N|N|N|N| +2456699|AAAAAAAALHMHFCAA|2014-02-10|1369|5954|457|2014|1|2|10|1|2014|457|5954|Monday|2014Q1|N|N|N|2456690|2456720|2456334|2456607|N|N|N|N|N| +2456700|AAAAAAAAMHMHFCAA|2014-02-11|1369|5955|457|2014|2|2|11|1|2014|457|5955|Tuesday|2014Q1|N|N|N|2456690|2456720|2456335|2456608|N|N|N|N|N| +2456701|AAAAAAAANHMHFCAA|2014-02-12|1369|5955|457|2014|3|2|12|1|2014|457|5955|Wednesday|2014Q1|N|N|N|2456690|2456720|2456336|2456609|N|N|N|N|N| +2456702|AAAAAAAAOHMHFCAA|2014-02-13|1369|5955|457|2014|4|2|13|1|2014|457|5955|Thursday|2014Q1|N|N|N|2456690|2456720|2456337|2456610|N|N|N|N|N| +2456703|AAAAAAAAPHMHFCAA|2014-02-14|1369|5955|457|2014|5|2|14|1|2014|457|5955|Friday|2014Q1|N|Y|N|2456690|2456720|2456338|2456611|N|N|N|N|N| +2456704|AAAAAAAAAIMHFCAA|2014-02-15|1369|5955|457|2014|6|2|15|1|2014|457|5955|Saturday|2014Q1|N|Y|N|2456690|2456720|2456339|2456612|N|N|N|N|N| +2456705|AAAAAAAABIMHFCAA|2014-02-16|1369|5955|457|2014|0|2|16|1|2014|457|5955|Sunday|2014Q1|N|N|N|2456690|2456720|2456340|2456613|N|N|N|N|N| +2456706|AAAAAAAACIMHFCAA|2014-02-17|1369|5955|457|2014|1|2|17|1|2014|457|5955|Monday|2014Q1|N|N|N|2456690|2456720|2456341|2456614|N|N|N|N|N| +2456707|AAAAAAAADIMHFCAA|2014-02-18|1369|5956|457|2014|2|2|18|1|2014|457|5956|Tuesday|2014Q1|N|N|N|2456690|2456720|2456342|2456615|N|N|N|N|N| +2456708|AAAAAAAAEIMHFCAA|2014-02-19|1369|5956|457|2014|3|2|19|1|2014|457|5956|Wednesday|2014Q1|N|N|N|2456690|2456720|2456343|2456616|N|N|N|N|N| +2456709|AAAAAAAAFIMHFCAA|2014-02-20|1369|5956|457|2014|4|2|20|1|2014|457|5956|Thursday|2014Q1|N|N|N|2456690|2456720|2456344|2456617|N|N|N|N|N| +2456710|AAAAAAAAGIMHFCAA|2014-02-21|1369|5956|457|2014|5|2|21|1|2014|457|5956|Friday|2014Q1|N|Y|N|2456690|2456720|2456345|2456618|N|N|N|N|N| +2456711|AAAAAAAAHIMHFCAA|2014-02-22|1369|5956|457|2014|6|2|22|1|2014|457|5956|Saturday|2014Q1|N|Y|N|2456690|2456720|2456346|2456619|N|N|N|N|N| +2456712|AAAAAAAAIIMHFCAA|2014-02-23|1369|5956|457|2014|0|2|23|1|2014|457|5956|Sunday|2014Q1|N|N|N|2456690|2456720|2456347|2456620|N|N|N|N|N| +2456713|AAAAAAAAJIMHFCAA|2014-02-24|1369|5956|457|2014|1|2|24|1|2014|457|5956|Monday|2014Q1|N|N|N|2456690|2456720|2456348|2456621|N|N|N|N|N| +2456714|AAAAAAAAKIMHFCAA|2014-02-25|1369|5957|457|2014|2|2|25|1|2014|457|5957|Tuesday|2014Q1|N|N|N|2456690|2456720|2456349|2456622|N|N|N|N|N| +2456715|AAAAAAAALIMHFCAA|2014-02-26|1369|5957|457|2014|3|2|26|1|2014|457|5957|Wednesday|2014Q1|N|N|N|2456690|2456720|2456350|2456623|N|N|N|N|N| +2456716|AAAAAAAAMIMHFCAA|2014-02-27|1369|5957|457|2014|4|2|27|1|2014|457|5957|Thursday|2014Q1|N|N|N|2456690|2456720|2456351|2456624|N|N|N|N|N| +2456717|AAAAAAAANIMHFCAA|2014-02-28|1369|5957|457|2014|5|2|28|1|2014|457|5957|Friday|2014Q1|N|Y|N|2456690|2456720|2456352|2456625|N|N|N|N|N| +2456718|AAAAAAAAOIMHFCAA|2014-03-01|1370|5957|458|2014|6|3|1|1|2014|458|5957|Saturday|2014Q1|N|Y|N|2456718|2456776|2456353|2456626|N|N|N|N|N| +2456719|AAAAAAAAPIMHFCAA|2014-03-02|1370|5957|458|2014|0|3|2|1|2014|458|5957|Sunday|2014Q1|N|N|N|2456718|2456776|2456354|2456627|N|N|N|N|N| +2456720|AAAAAAAAAJMHFCAA|2014-03-03|1370|5957|458|2014|1|3|3|1|2014|458|5957|Monday|2014Q1|N|N|N|2456718|2456776|2456355|2456628|N|N|N|N|N| +2456721|AAAAAAAABJMHFCAA|2014-03-04|1370|5958|458|2014|2|3|4|1|2014|458|5958|Tuesday|2014Q1|N|N|N|2456718|2456776|2456356|2456629|N|N|N|N|N| +2456722|AAAAAAAACJMHFCAA|2014-03-05|1370|5958|458|2014|3|3|5|1|2014|458|5958|Wednesday|2014Q1|N|N|N|2456718|2456776|2456357|2456630|N|N|N|N|N| +2456723|AAAAAAAADJMHFCAA|2014-03-06|1370|5958|458|2014|4|3|6|1|2014|458|5958|Thursday|2014Q1|N|N|N|2456718|2456776|2456358|2456631|N|N|N|N|N| +2456724|AAAAAAAAEJMHFCAA|2014-03-07|1370|5958|458|2014|5|3|7|1|2014|458|5958|Friday|2014Q1|N|Y|N|2456718|2456776|2456359|2456632|N|N|N|N|N| +2456725|AAAAAAAAFJMHFCAA|2014-03-08|1370|5958|458|2014|6|3|8|1|2014|458|5958|Saturday|2014Q1|N|Y|N|2456718|2456776|2456360|2456633|N|N|N|N|N| +2456726|AAAAAAAAGJMHFCAA|2014-03-09|1370|5958|458|2014|0|3|9|1|2014|458|5958|Sunday|2014Q1|N|N|N|2456718|2456776|2456361|2456634|N|N|N|N|N| +2456727|AAAAAAAAHJMHFCAA|2014-03-10|1370|5958|458|2014|1|3|10|1|2014|458|5958|Monday|2014Q1|N|N|N|2456718|2456776|2456362|2456635|N|N|N|N|N| +2456728|AAAAAAAAIJMHFCAA|2014-03-11|1370|5959|458|2014|2|3|11|1|2014|458|5959|Tuesday|2014Q1|N|N|N|2456718|2456776|2456363|2456636|N|N|N|N|N| +2456729|AAAAAAAAJJMHFCAA|2014-03-12|1370|5959|458|2014|3|3|12|1|2014|458|5959|Wednesday|2014Q1|N|N|N|2456718|2456776|2456364|2456637|N|N|N|N|N| +2456730|AAAAAAAAKJMHFCAA|2014-03-13|1370|5959|458|2014|4|3|13|1|2014|458|5959|Thursday|2014Q1|N|N|N|2456718|2456776|2456365|2456638|N|N|N|N|N| +2456731|AAAAAAAALJMHFCAA|2014-03-14|1370|5959|458|2014|5|3|14|1|2014|458|5959|Friday|2014Q1|N|Y|N|2456718|2456776|2456366|2456639|N|N|N|N|N| +2456732|AAAAAAAAMJMHFCAA|2014-03-15|1370|5959|458|2014|6|3|15|1|2014|458|5959|Saturday|2014Q1|N|Y|N|2456718|2456776|2456367|2456640|N|N|N|N|N| +2456733|AAAAAAAANJMHFCAA|2014-03-16|1370|5959|458|2014|0|3|16|1|2014|458|5959|Sunday|2014Q1|N|N|N|2456718|2456776|2456368|2456641|N|N|N|N|N| +2456734|AAAAAAAAOJMHFCAA|2014-03-17|1370|5959|458|2014|1|3|17|1|2014|458|5959|Monday|2014Q1|N|N|N|2456718|2456776|2456369|2456642|N|N|N|N|N| +2456735|AAAAAAAAPJMHFCAA|2014-03-18|1370|5960|458|2014|2|3|18|1|2014|458|5960|Tuesday|2014Q1|N|N|N|2456718|2456776|2456370|2456643|N|N|N|N|N| +2456736|AAAAAAAAAKMHFCAA|2014-03-19|1370|5960|458|2014|3|3|19|1|2014|458|5960|Wednesday|2014Q1|N|N|N|2456718|2456776|2456371|2456644|N|N|N|N|N| +2456737|AAAAAAAABKMHFCAA|2014-03-20|1370|5960|458|2014|4|3|20|1|2014|458|5960|Thursday|2014Q1|N|N|N|2456718|2456776|2456372|2456645|N|N|N|N|N| +2456738|AAAAAAAACKMHFCAA|2014-03-21|1370|5960|458|2014|5|3|21|1|2014|458|5960|Friday|2014Q1|N|Y|N|2456718|2456776|2456373|2456646|N|N|N|N|N| +2456739|AAAAAAAADKMHFCAA|2014-03-22|1370|5960|458|2014|6|3|22|1|2014|458|5960|Saturday|2014Q1|N|Y|N|2456718|2456776|2456374|2456647|N|N|N|N|N| +2456740|AAAAAAAAEKMHFCAA|2014-03-23|1370|5960|458|2014|0|3|23|1|2014|458|5960|Sunday|2014Q1|N|N|N|2456718|2456776|2456375|2456648|N|N|N|N|N| +2456741|AAAAAAAAFKMHFCAA|2014-03-24|1370|5960|458|2014|1|3|24|1|2014|458|5960|Monday|2014Q1|N|N|N|2456718|2456776|2456376|2456649|N|N|N|N|N| +2456742|AAAAAAAAGKMHFCAA|2014-03-25|1370|5961|458|2014|2|3|25|1|2014|458|5961|Tuesday|2014Q1|N|N|N|2456718|2456776|2456377|2456650|N|N|N|N|N| +2456743|AAAAAAAAHKMHFCAA|2014-03-26|1370|5961|458|2014|3|3|26|1|2014|458|5961|Wednesday|2014Q1|N|N|N|2456718|2456776|2456378|2456651|N|N|N|N|N| +2456744|AAAAAAAAIKMHFCAA|2014-03-27|1370|5961|458|2014|4|3|27|1|2014|458|5961|Thursday|2014Q1|N|N|N|2456718|2456776|2456379|2456652|N|N|N|N|N| +2456745|AAAAAAAAJKMHFCAA|2014-03-28|1370|5961|458|2014|5|3|28|1|2014|458|5961|Friday|2014Q1|N|Y|N|2456718|2456776|2456380|2456653|N|N|N|N|N| +2456746|AAAAAAAAKKMHFCAA|2014-03-29|1370|5961|458|2014|6|3|29|1|2014|458|5961|Saturday|2014Q1|N|Y|N|2456718|2456776|2456381|2456654|N|N|N|N|N| +2456747|AAAAAAAALKMHFCAA|2014-03-30|1370|5961|458|2014|0|3|30|1|2014|458|5961|Sunday|2014Q1|N|N|N|2456718|2456776|2456382|2456655|N|N|N|N|N| +2456748|AAAAAAAAMKMHFCAA|2014-03-31|1370|5961|458|2014|1|3|31|1|2014|458|5961|Monday|2014Q1|N|N|N|2456718|2456776|2456383|2456656|N|N|N|N|N| +2456749|AAAAAAAANKMHFCAA|2014-04-01|1371|5962|458|2014|2|4|1|1|2014|458|5962|Tuesday|2014Q1|N|N|N|2456749|2456838|2456384|2456659|N|N|N|N|N| +2456750|AAAAAAAAOKMHFCAA|2014-04-02|1371|5962|458|2014|3|4|2|2|2014|458|5962|Wednesday|2014Q2|N|N|N|2456749|2456838|2456385|2456660|N|N|N|N|N| +2456751|AAAAAAAAPKMHFCAA|2014-04-03|1371|5962|458|2014|4|4|3|2|2014|458|5962|Thursday|2014Q2|N|N|N|2456749|2456838|2456386|2456661|N|N|N|N|N| +2456752|AAAAAAAAALMHFCAA|2014-04-04|1371|5962|458|2014|5|4|4|2|2014|458|5962|Friday|2014Q2|N|Y|N|2456749|2456838|2456387|2456662|N|N|N|N|N| +2456753|AAAAAAAABLMHFCAA|2014-04-05|1371|5962|458|2014|6|4|5|2|2014|458|5962|Saturday|2014Q2|N|Y|N|2456749|2456838|2456388|2456663|N|N|N|N|N| +2456754|AAAAAAAACLMHFCAA|2014-04-06|1371|5962|458|2014|0|4|6|2|2014|458|5962|Sunday|2014Q2|N|N|N|2456749|2456838|2456389|2456664|N|N|N|N|N| +2456755|AAAAAAAADLMHFCAA|2014-04-07|1371|5962|458|2014|1|4|7|2|2014|458|5962|Monday|2014Q2|N|N|N|2456749|2456838|2456390|2456665|N|N|N|N|N| +2456756|AAAAAAAAELMHFCAA|2014-04-08|1371|5963|458|2014|2|4|8|2|2014|458|5963|Tuesday|2014Q2|N|N|N|2456749|2456838|2456391|2456666|N|N|N|N|N| +2456757|AAAAAAAAFLMHFCAA|2014-04-09|1371|5963|458|2014|3|4|9|2|2014|458|5963|Wednesday|2014Q2|N|N|N|2456749|2456838|2456392|2456667|N|N|N|N|N| +2456758|AAAAAAAAGLMHFCAA|2014-04-10|1371|5963|458|2014|4|4|10|2|2014|458|5963|Thursday|2014Q2|N|N|N|2456749|2456838|2456393|2456668|N|N|N|N|N| +2456759|AAAAAAAAHLMHFCAA|2014-04-11|1371|5963|458|2014|5|4|11|2|2014|458|5963|Friday|2014Q2|N|Y|N|2456749|2456838|2456394|2456669|N|N|N|N|N| +2456760|AAAAAAAAILMHFCAA|2014-04-12|1371|5963|458|2014|6|4|12|2|2014|458|5963|Saturday|2014Q2|N|Y|N|2456749|2456838|2456395|2456670|N|N|N|N|N| +2456761|AAAAAAAAJLMHFCAA|2014-04-13|1371|5963|458|2014|0|4|13|2|2014|458|5963|Sunday|2014Q2|N|N|N|2456749|2456838|2456396|2456671|N|N|N|N|N| +2456762|AAAAAAAAKLMHFCAA|2014-04-14|1371|5963|458|2014|1|4|14|2|2014|458|5963|Monday|2014Q2|N|N|N|2456749|2456838|2456397|2456672|N|N|N|N|N| +2456763|AAAAAAAALLMHFCAA|2014-04-15|1371|5964|458|2014|2|4|15|2|2014|458|5964|Tuesday|2014Q2|N|N|N|2456749|2456838|2456398|2456673|N|N|N|N|N| +2456764|AAAAAAAAMLMHFCAA|2014-04-16|1371|5964|458|2014|3|4|16|2|2014|458|5964|Wednesday|2014Q2|N|N|N|2456749|2456838|2456399|2456674|N|N|N|N|N| +2456765|AAAAAAAANLMHFCAA|2014-04-17|1371|5964|458|2014|4|4|17|2|2014|458|5964|Thursday|2014Q2|N|N|N|2456749|2456838|2456400|2456675|N|N|N|N|N| +2456766|AAAAAAAAOLMHFCAA|2014-04-18|1371|5964|458|2014|5|4|18|2|2014|458|5964|Friday|2014Q2|N|Y|N|2456749|2456838|2456401|2456676|N|N|N|N|N| +2456767|AAAAAAAAPLMHFCAA|2014-04-19|1371|5964|458|2014|6|4|19|2|2014|458|5964|Saturday|2014Q2|N|Y|N|2456749|2456838|2456402|2456677|N|N|N|N|N| +2456768|AAAAAAAAAMMHFCAA|2014-04-20|1371|5964|458|2014|0|4|20|2|2014|458|5964|Sunday|2014Q2|N|N|N|2456749|2456838|2456403|2456678|N|N|N|N|N| +2456769|AAAAAAAABMMHFCAA|2014-04-21|1371|5964|458|2014|1|4|21|2|2014|458|5964|Monday|2014Q2|N|N|N|2456749|2456838|2456404|2456679|N|N|N|N|N| +2456770|AAAAAAAACMMHFCAA|2014-04-22|1371|5965|458|2014|2|4|22|2|2014|458|5965|Tuesday|2014Q2|N|N|N|2456749|2456838|2456405|2456680|N|N|N|N|N| +2456771|AAAAAAAADMMHFCAA|2014-04-23|1371|5965|458|2014|3|4|23|2|2014|458|5965|Wednesday|2014Q2|N|N|N|2456749|2456838|2456406|2456681|N|N|N|N|N| +2456772|AAAAAAAAEMMHFCAA|2014-04-24|1371|5965|458|2014|4|4|24|2|2014|458|5965|Thursday|2014Q2|N|N|N|2456749|2456838|2456407|2456682|N|N|N|N|N| +2456773|AAAAAAAAFMMHFCAA|2014-04-25|1371|5965|458|2014|5|4|25|2|2014|458|5965|Friday|2014Q2|N|Y|N|2456749|2456838|2456408|2456683|N|N|N|N|N| +2456774|AAAAAAAAGMMHFCAA|2014-04-26|1371|5965|458|2014|6|4|26|2|2014|458|5965|Saturday|2014Q2|N|Y|N|2456749|2456838|2456409|2456684|N|N|N|N|N| +2456775|AAAAAAAAHMMHFCAA|2014-04-27|1371|5965|458|2014|0|4|27|2|2014|458|5965|Sunday|2014Q2|N|N|N|2456749|2456838|2456410|2456685|N|N|N|N|N| +2456776|AAAAAAAAIMMHFCAA|2014-04-28|1371|5965|458|2014|1|4|28|2|2014|458|5965|Monday|2014Q2|N|N|N|2456749|2456838|2456411|2456686|N|N|N|N|N| +2456777|AAAAAAAAJMMHFCAA|2014-04-29|1371|5966|458|2014|2|4|29|2|2014|458|5966|Tuesday|2014Q2|N|N|N|2456749|2456838|2456412|2456687|N|N|N|N|N| +2456778|AAAAAAAAKMMHFCAA|2014-04-30|1371|5966|458|2014|3|4|30|2|2014|458|5966|Wednesday|2014Q2|N|N|N|2456749|2456838|2456413|2456688|N|N|N|N|N| +2456779|AAAAAAAALMMHFCAA|2014-05-01|1372|5966|458|2014|4|5|1|2|2014|458|5966|Thursday|2014Q2|N|N|N|2456779|2456898|2456414|2456689|N|N|N|N|N| +2456780|AAAAAAAAMMMHFCAA|2014-05-02|1372|5966|458|2014|5|5|2|2|2014|458|5966|Friday|2014Q2|N|Y|N|2456779|2456898|2456415|2456690|N|N|N|N|N| +2456781|AAAAAAAANMMHFCAA|2014-05-03|1372|5966|458|2014|6|5|3|2|2014|458|5966|Saturday|2014Q2|N|Y|N|2456779|2456898|2456416|2456691|N|N|N|N|N| +2456782|AAAAAAAAOMMHFCAA|2014-05-04|1372|5966|458|2014|0|5|4|2|2014|458|5966|Sunday|2014Q2|N|N|N|2456779|2456898|2456417|2456692|N|N|N|N|N| +2456783|AAAAAAAAPMMHFCAA|2014-05-05|1372|5966|458|2014|1|5|5|2|2014|458|5966|Monday|2014Q2|N|N|N|2456779|2456898|2456418|2456693|N|N|N|N|N| +2456784|AAAAAAAAANMHFCAA|2014-05-06|1372|5967|458|2014|2|5|6|2|2014|458|5967|Tuesday|2014Q2|N|N|N|2456779|2456898|2456419|2456694|N|N|N|N|N| +2456785|AAAAAAAABNMHFCAA|2014-05-07|1372|5967|458|2014|3|5|7|2|2014|458|5967|Wednesday|2014Q2|N|N|N|2456779|2456898|2456420|2456695|N|N|N|N|N| +2456786|AAAAAAAACNMHFCAA|2014-05-08|1372|5967|458|2014|4|5|8|2|2014|458|5967|Thursday|2014Q2|N|N|N|2456779|2456898|2456421|2456696|N|N|N|N|N| +2456787|AAAAAAAADNMHFCAA|2014-05-09|1372|5967|458|2014|5|5|9|2|2014|458|5967|Friday|2014Q2|N|Y|N|2456779|2456898|2456422|2456697|N|N|N|N|N| +2456788|AAAAAAAAENMHFCAA|2014-05-10|1372|5967|458|2014|6|5|10|2|2014|458|5967|Saturday|2014Q2|N|Y|N|2456779|2456898|2456423|2456698|N|N|N|N|N| +2456789|AAAAAAAAFNMHFCAA|2014-05-11|1372|5967|458|2014|0|5|11|2|2014|458|5967|Sunday|2014Q2|N|N|N|2456779|2456898|2456424|2456699|N|N|N|N|N| +2456790|AAAAAAAAGNMHFCAA|2014-05-12|1372|5967|458|2014|1|5|12|2|2014|458|5967|Monday|2014Q2|N|N|N|2456779|2456898|2456425|2456700|N|N|N|N|N| +2456791|AAAAAAAAHNMHFCAA|2014-05-13|1372|5968|458|2014|2|5|13|2|2014|458|5968|Tuesday|2014Q2|N|N|N|2456779|2456898|2456426|2456701|N|N|N|N|N| +2456792|AAAAAAAAINMHFCAA|2014-05-14|1372|5968|458|2014|3|5|14|2|2014|458|5968|Wednesday|2014Q2|N|N|N|2456779|2456898|2456427|2456702|N|N|N|N|N| +2456793|AAAAAAAAJNMHFCAA|2014-05-15|1372|5968|458|2014|4|5|15|2|2014|458|5968|Thursday|2014Q2|N|N|N|2456779|2456898|2456428|2456703|N|N|N|N|N| +2456794|AAAAAAAAKNMHFCAA|2014-05-16|1372|5968|458|2014|5|5|16|2|2014|458|5968|Friday|2014Q2|N|Y|N|2456779|2456898|2456429|2456704|N|N|N|N|N| +2456795|AAAAAAAALNMHFCAA|2014-05-17|1372|5968|458|2014|6|5|17|2|2014|458|5968|Saturday|2014Q2|N|Y|N|2456779|2456898|2456430|2456705|N|N|N|N|N| +2456796|AAAAAAAAMNMHFCAA|2014-05-18|1372|5968|458|2014|0|5|18|2|2014|458|5968|Sunday|2014Q2|N|N|N|2456779|2456898|2456431|2456706|N|N|N|N|N| +2456797|AAAAAAAANNMHFCAA|2014-05-19|1372|5968|458|2014|1|5|19|2|2014|458|5968|Monday|2014Q2|N|N|N|2456779|2456898|2456432|2456707|N|N|N|N|N| +2456798|AAAAAAAAONMHFCAA|2014-05-20|1372|5969|458|2014|2|5|20|2|2014|458|5969|Tuesday|2014Q2|N|N|N|2456779|2456898|2456433|2456708|N|N|N|N|N| +2456799|AAAAAAAAPNMHFCAA|2014-05-21|1372|5969|458|2014|3|5|21|2|2014|458|5969|Wednesday|2014Q2|N|N|N|2456779|2456898|2456434|2456709|N|N|N|N|N| +2456800|AAAAAAAAAOMHFCAA|2014-05-22|1372|5969|458|2014|4|5|22|2|2014|458|5969|Thursday|2014Q2|N|N|N|2456779|2456898|2456435|2456710|N|N|N|N|N| +2456801|AAAAAAAABOMHFCAA|2014-05-23|1372|5969|458|2014|5|5|23|2|2014|458|5969|Friday|2014Q2|N|Y|N|2456779|2456898|2456436|2456711|N|N|N|N|N| +2456802|AAAAAAAACOMHFCAA|2014-05-24|1372|5969|458|2014|6|5|24|2|2014|458|5969|Saturday|2014Q2|N|Y|N|2456779|2456898|2456437|2456712|N|N|N|N|N| +2456803|AAAAAAAADOMHFCAA|2014-05-25|1372|5969|458|2014|0|5|25|2|2014|458|5969|Sunday|2014Q2|N|N|N|2456779|2456898|2456438|2456713|N|N|N|N|N| +2456804|AAAAAAAAEOMHFCAA|2014-05-26|1372|5969|458|2014|1|5|26|2|2014|458|5969|Monday|2014Q2|N|N|N|2456779|2456898|2456439|2456714|N|N|N|N|N| +2456805|AAAAAAAAFOMHFCAA|2014-05-27|1372|5970|458|2014|2|5|27|2|2014|458|5970|Tuesday|2014Q2|N|N|N|2456779|2456898|2456440|2456715|N|N|N|N|N| +2456806|AAAAAAAAGOMHFCAA|2014-05-28|1372|5970|458|2014|3|5|28|2|2014|458|5970|Wednesday|2014Q2|N|N|N|2456779|2456898|2456441|2456716|N|N|N|N|N| +2456807|AAAAAAAAHOMHFCAA|2014-05-29|1372|5970|458|2014|4|5|29|2|2014|458|5970|Thursday|2014Q2|N|N|N|2456779|2456898|2456442|2456717|N|N|N|N|N| +2456808|AAAAAAAAIOMHFCAA|2014-05-30|1372|5970|458|2014|5|5|30|2|2014|458|5970|Friday|2014Q2|N|Y|N|2456779|2456898|2456443|2456718|N|N|N|N|N| +2456809|AAAAAAAAJOMHFCAA|2014-05-31|1372|5970|458|2014|6|5|31|2|2014|458|5970|Saturday|2014Q2|N|Y|N|2456779|2456898|2456444|2456719|N|N|N|N|N| +2456810|AAAAAAAAKOMHFCAA|2014-06-01|1373|5970|459|2014|0|6|1|2|2014|459|5970|Sunday|2014Q2|N|N|N|2456810|2456960|2456445|2456720|N|N|N|N|N| +2456811|AAAAAAAALOMHFCAA|2014-06-02|1373|5970|459|2014|1|6|2|2|2014|459|5970|Monday|2014Q2|N|N|N|2456810|2456960|2456446|2456721|N|N|N|N|N| +2456812|AAAAAAAAMOMHFCAA|2014-06-03|1373|5971|459|2014|2|6|3|2|2014|459|5971|Tuesday|2014Q2|N|N|N|2456810|2456960|2456447|2456722|N|N|N|N|N| +2456813|AAAAAAAANOMHFCAA|2014-06-04|1373|5971|459|2014|3|6|4|2|2014|459|5971|Wednesday|2014Q2|N|N|N|2456810|2456960|2456448|2456723|N|N|N|N|N| +2456814|AAAAAAAAOOMHFCAA|2014-06-05|1373|5971|459|2014|4|6|5|2|2014|459|5971|Thursday|2014Q2|N|N|N|2456810|2456960|2456449|2456724|N|N|N|N|N| +2456815|AAAAAAAAPOMHFCAA|2014-06-06|1373|5971|459|2014|5|6|6|2|2014|459|5971|Friday|2014Q2|N|Y|N|2456810|2456960|2456450|2456725|N|N|N|N|N| +2456816|AAAAAAAAAPMHFCAA|2014-06-07|1373|5971|459|2014|6|6|7|2|2014|459|5971|Saturday|2014Q2|N|Y|N|2456810|2456960|2456451|2456726|N|N|N|N|N| +2456817|AAAAAAAABPMHFCAA|2014-06-08|1373|5971|459|2014|0|6|8|2|2014|459|5971|Sunday|2014Q2|N|N|N|2456810|2456960|2456452|2456727|N|N|N|N|N| +2456818|AAAAAAAACPMHFCAA|2014-06-09|1373|5971|459|2014|1|6|9|2|2014|459|5971|Monday|2014Q2|N|N|N|2456810|2456960|2456453|2456728|N|N|N|N|N| +2456819|AAAAAAAADPMHFCAA|2014-06-10|1373|5972|459|2014|2|6|10|2|2014|459|5972|Tuesday|2014Q2|N|N|N|2456810|2456960|2456454|2456729|N|N|N|N|N| +2456820|AAAAAAAAEPMHFCAA|2014-06-11|1373|5972|459|2014|3|6|11|2|2014|459|5972|Wednesday|2014Q2|N|N|N|2456810|2456960|2456455|2456730|N|N|N|N|N| +2456821|AAAAAAAAFPMHFCAA|2014-06-12|1373|5972|459|2014|4|6|12|2|2014|459|5972|Thursday|2014Q2|N|N|N|2456810|2456960|2456456|2456731|N|N|N|N|N| +2456822|AAAAAAAAGPMHFCAA|2014-06-13|1373|5972|459|2014|5|6|13|2|2014|459|5972|Friday|2014Q2|N|Y|N|2456810|2456960|2456457|2456732|N|N|N|N|N| +2456823|AAAAAAAAHPMHFCAA|2014-06-14|1373|5972|459|2014|6|6|14|2|2014|459|5972|Saturday|2014Q2|N|Y|N|2456810|2456960|2456458|2456733|N|N|N|N|N| +2456824|AAAAAAAAIPMHFCAA|2014-06-15|1373|5972|459|2014|0|6|15|2|2014|459|5972|Sunday|2014Q2|N|N|N|2456810|2456960|2456459|2456734|N|N|N|N|N| +2456825|AAAAAAAAJPMHFCAA|2014-06-16|1373|5972|459|2014|1|6|16|2|2014|459|5972|Monday|2014Q2|N|N|N|2456810|2456960|2456460|2456735|N|N|N|N|N| +2456826|AAAAAAAAKPMHFCAA|2014-06-17|1373|5973|459|2014|2|6|17|2|2014|459|5973|Tuesday|2014Q2|N|N|N|2456810|2456960|2456461|2456736|N|N|N|N|N| +2456827|AAAAAAAALPMHFCAA|2014-06-18|1373|5973|459|2014|3|6|18|2|2014|459|5973|Wednesday|2014Q2|N|N|N|2456810|2456960|2456462|2456737|N|N|N|N|N| +2456828|AAAAAAAAMPMHFCAA|2014-06-19|1373|5973|459|2014|4|6|19|2|2014|459|5973|Thursday|2014Q2|N|N|N|2456810|2456960|2456463|2456738|N|N|N|N|N| +2456829|AAAAAAAANPMHFCAA|2014-06-20|1373|5973|459|2014|5|6|20|2|2014|459|5973|Friday|2014Q2|N|Y|N|2456810|2456960|2456464|2456739|N|N|N|N|N| +2456830|AAAAAAAAOPMHFCAA|2014-06-21|1373|5973|459|2014|6|6|21|2|2014|459|5973|Saturday|2014Q2|N|Y|N|2456810|2456960|2456465|2456740|N|N|N|N|N| +2456831|AAAAAAAAPPMHFCAA|2014-06-22|1373|5973|459|2014|0|6|22|2|2014|459|5973|Sunday|2014Q2|N|N|N|2456810|2456960|2456466|2456741|N|N|N|N|N| +2456832|AAAAAAAAAANHFCAA|2014-06-23|1373|5973|459|2014|1|6|23|2|2014|459|5973|Monday|2014Q2|N|N|N|2456810|2456960|2456467|2456742|N|N|N|N|N| +2456833|AAAAAAAABANHFCAA|2014-06-24|1373|5974|459|2014|2|6|24|2|2014|459|5974|Tuesday|2014Q2|N|N|N|2456810|2456960|2456468|2456743|N|N|N|N|N| +2456834|AAAAAAAACANHFCAA|2014-06-25|1373|5974|459|2014|3|6|25|2|2014|459|5974|Wednesday|2014Q2|N|N|N|2456810|2456960|2456469|2456744|N|N|N|N|N| +2456835|AAAAAAAADANHFCAA|2014-06-26|1373|5974|459|2014|4|6|26|2|2014|459|5974|Thursday|2014Q2|N|N|N|2456810|2456960|2456470|2456745|N|N|N|N|N| +2456836|AAAAAAAAEANHFCAA|2014-06-27|1373|5974|459|2014|5|6|27|2|2014|459|5974|Friday|2014Q2|N|Y|N|2456810|2456960|2456471|2456746|N|N|N|N|N| +2456837|AAAAAAAAFANHFCAA|2014-06-28|1373|5974|459|2014|6|6|28|2|2014|459|5974|Saturday|2014Q2|N|Y|N|2456810|2456960|2456472|2456747|N|N|N|N|N| +2456838|AAAAAAAAGANHFCAA|2014-06-29|1373|5974|459|2014|0|6|29|2|2014|459|5974|Sunday|2014Q2|N|N|N|2456810|2456960|2456473|2456748|N|N|N|N|N| +2456839|AAAAAAAAHANHFCAA|2014-06-30|1373|5974|459|2014|1|6|30|2|2014|459|5974|Monday|2014Q2|N|N|N|2456810|2456960|2456474|2456749|N|N|N|N|N| +2456840|AAAAAAAAIANHFCAA|2014-07-01|1374|5975|459|2014|2|7|1|2|2014|459|5975|Tuesday|2014Q2|N|N|N|2456840|2457020|2456475|2456749|N|N|N|N|N| +2456841|AAAAAAAAJANHFCAA|2014-07-02|1374|5975|459|2014|3|7|2|3|2014|459|5975|Wednesday|2014Q3|N|N|N|2456840|2457020|2456476|2456750|N|N|N|N|N| +2456842|AAAAAAAAKANHFCAA|2014-07-03|1374|5975|459|2014|4|7|3|3|2014|459|5975|Thursday|2014Q3|N|N|N|2456840|2457020|2456477|2456751|N|N|N|N|N| +2456843|AAAAAAAALANHFCAA|2014-07-04|1374|5975|459|2014|5|7|4|3|2014|459|5975|Friday|2014Q3|N|Y|N|2456840|2457020|2456478|2456752|N|N|N|N|N| +2456844|AAAAAAAAMANHFCAA|2014-07-05|1374|5975|459|2014|6|7|5|3|2014|459|5975|Saturday|2014Q3|Y|Y|N|2456840|2457020|2456479|2456753|N|N|N|N|N| +2456845|AAAAAAAANANHFCAA|2014-07-06|1374|5975|459|2014|0|7|6|3|2014|459|5975|Sunday|2014Q3|N|N|Y|2456840|2457020|2456480|2456754|N|N|N|N|N| +2456846|AAAAAAAAOANHFCAA|2014-07-07|1374|5975|459|2014|1|7|7|3|2014|459|5975|Monday|2014Q3|N|N|N|2456840|2457020|2456481|2456755|N|N|N|N|N| +2456847|AAAAAAAAPANHFCAA|2014-07-08|1374|5976|459|2014|2|7|8|3|2014|459|5976|Tuesday|2014Q3|N|N|N|2456840|2457020|2456482|2456756|N|N|N|N|N| +2456848|AAAAAAAAABNHFCAA|2014-07-09|1374|5976|459|2014|3|7|9|3|2014|459|5976|Wednesday|2014Q3|N|N|N|2456840|2457020|2456483|2456757|N|N|N|N|N| +2456849|AAAAAAAABBNHFCAA|2014-07-10|1374|5976|459|2014|4|7|10|3|2014|459|5976|Thursday|2014Q3|N|N|N|2456840|2457020|2456484|2456758|N|N|N|N|N| +2456850|AAAAAAAACBNHFCAA|2014-07-11|1374|5976|459|2014|5|7|11|3|2014|459|5976|Friday|2014Q3|N|Y|N|2456840|2457020|2456485|2456759|N|N|N|N|N| +2456851|AAAAAAAADBNHFCAA|2014-07-12|1374|5976|459|2014|6|7|12|3|2014|459|5976|Saturday|2014Q3|N|Y|N|2456840|2457020|2456486|2456760|N|N|N|N|N| +2456852|AAAAAAAAEBNHFCAA|2014-07-13|1374|5976|459|2014|0|7|13|3|2014|459|5976|Sunday|2014Q3|N|N|N|2456840|2457020|2456487|2456761|N|N|N|N|N| +2456853|AAAAAAAAFBNHFCAA|2014-07-14|1374|5976|459|2014|1|7|14|3|2014|459|5976|Monday|2014Q3|N|N|N|2456840|2457020|2456488|2456762|N|N|N|N|N| +2456854|AAAAAAAAGBNHFCAA|2014-07-15|1374|5977|459|2014|2|7|15|3|2014|459|5977|Tuesday|2014Q3|N|N|N|2456840|2457020|2456489|2456763|N|N|N|N|N| +2456855|AAAAAAAAHBNHFCAA|2014-07-16|1374|5977|459|2014|3|7|16|3|2014|459|5977|Wednesday|2014Q3|N|N|N|2456840|2457020|2456490|2456764|N|N|N|N|N| +2456856|AAAAAAAAIBNHFCAA|2014-07-17|1374|5977|459|2014|4|7|17|3|2014|459|5977|Thursday|2014Q3|N|N|N|2456840|2457020|2456491|2456765|N|N|N|N|N| +2456857|AAAAAAAAJBNHFCAA|2014-07-18|1374|5977|459|2014|5|7|18|3|2014|459|5977|Friday|2014Q3|N|Y|N|2456840|2457020|2456492|2456766|N|N|N|N|N| +2456858|AAAAAAAAKBNHFCAA|2014-07-19|1374|5977|459|2014|6|7|19|3|2014|459|5977|Saturday|2014Q3|N|Y|N|2456840|2457020|2456493|2456767|N|N|N|N|N| +2456859|AAAAAAAALBNHFCAA|2014-07-20|1374|5977|459|2014|0|7|20|3|2014|459|5977|Sunday|2014Q3|N|N|N|2456840|2457020|2456494|2456768|N|N|N|N|N| +2456860|AAAAAAAAMBNHFCAA|2014-07-21|1374|5977|459|2014|1|7|21|3|2014|459|5977|Monday|2014Q3|N|N|N|2456840|2457020|2456495|2456769|N|N|N|N|N| +2456861|AAAAAAAANBNHFCAA|2014-07-22|1374|5978|459|2014|2|7|22|3|2014|459|5978|Tuesday|2014Q3|N|N|N|2456840|2457020|2456496|2456770|N|N|N|N|N| +2456862|AAAAAAAAOBNHFCAA|2014-07-23|1374|5978|459|2014|3|7|23|3|2014|459|5978|Wednesday|2014Q3|N|N|N|2456840|2457020|2456497|2456771|N|N|N|N|N| +2456863|AAAAAAAAPBNHFCAA|2014-07-24|1374|5978|459|2014|4|7|24|3|2014|459|5978|Thursday|2014Q3|N|N|N|2456840|2457020|2456498|2456772|N|N|N|N|N| +2456864|AAAAAAAAACNHFCAA|2014-07-25|1374|5978|459|2014|5|7|25|3|2014|459|5978|Friday|2014Q3|N|Y|N|2456840|2457020|2456499|2456773|N|N|N|N|N| +2456865|AAAAAAAABCNHFCAA|2014-07-26|1374|5978|459|2014|6|7|26|3|2014|459|5978|Saturday|2014Q3|N|Y|N|2456840|2457020|2456500|2456774|N|N|N|N|N| +2456866|AAAAAAAACCNHFCAA|2014-07-27|1374|5978|459|2014|0|7|27|3|2014|459|5978|Sunday|2014Q3|N|N|N|2456840|2457020|2456501|2456775|N|N|N|N|N| +2456867|AAAAAAAADCNHFCAA|2014-07-28|1374|5978|459|2014|1|7|28|3|2014|459|5978|Monday|2014Q3|N|N|N|2456840|2457020|2456502|2456776|N|N|N|N|N| +2456868|AAAAAAAAECNHFCAA|2014-07-29|1374|5979|459|2014|2|7|29|3|2014|459|5979|Tuesday|2014Q3|N|N|N|2456840|2457020|2456503|2456777|N|N|N|N|N| +2456869|AAAAAAAAFCNHFCAA|2014-07-30|1374|5979|459|2014|3|7|30|3|2014|459|5979|Wednesday|2014Q3|N|N|N|2456840|2457020|2456504|2456778|N|N|N|N|N| +2456870|AAAAAAAAGCNHFCAA|2014-07-31|1374|5979|459|2014|4|7|31|3|2014|459|5979|Thursday|2014Q3|N|N|N|2456840|2457020|2456505|2456779|N|N|N|N|N| +2456871|AAAAAAAAHCNHFCAA|2014-08-01|1375|5979|459|2014|5|8|1|3|2014|459|5979|Friday|2014Q3|N|Y|N|2456871|2457082|2456506|2456780|N|N|N|N|N| +2456872|AAAAAAAAICNHFCAA|2014-08-02|1375|5979|459|2014|6|8|2|3|2014|459|5979|Saturday|2014Q3|N|Y|N|2456871|2457082|2456507|2456781|N|N|N|N|N| +2456873|AAAAAAAAJCNHFCAA|2014-08-03|1375|5979|459|2014|0|8|3|3|2014|459|5979|Sunday|2014Q3|N|N|N|2456871|2457082|2456508|2456782|N|N|N|N|N| +2456874|AAAAAAAAKCNHFCAA|2014-08-04|1375|5979|459|2014|1|8|4|3|2014|459|5979|Monday|2014Q3|N|N|N|2456871|2457082|2456509|2456783|N|N|N|N|N| +2456875|AAAAAAAALCNHFCAA|2014-08-05|1375|5980|459|2014|2|8|5|3|2014|459|5980|Tuesday|2014Q3|N|N|N|2456871|2457082|2456510|2456784|N|N|N|N|N| +2456876|AAAAAAAAMCNHFCAA|2014-08-06|1375|5980|459|2014|3|8|6|3|2014|459|5980|Wednesday|2014Q3|N|N|N|2456871|2457082|2456511|2456785|N|N|N|N|N| +2456877|AAAAAAAANCNHFCAA|2014-08-07|1375|5980|459|2014|4|8|7|3|2014|459|5980|Thursday|2014Q3|N|N|N|2456871|2457082|2456512|2456786|N|N|N|N|N| +2456878|AAAAAAAAOCNHFCAA|2014-08-08|1375|5980|459|2014|5|8|8|3|2014|459|5980|Friday|2014Q3|N|Y|N|2456871|2457082|2456513|2456787|N|N|N|N|N| +2456879|AAAAAAAAPCNHFCAA|2014-08-09|1375|5980|459|2014|6|8|9|3|2014|459|5980|Saturday|2014Q3|N|Y|N|2456871|2457082|2456514|2456788|N|N|N|N|N| +2456880|AAAAAAAAADNHFCAA|2014-08-10|1375|5980|459|2014|0|8|10|3|2014|459|5980|Sunday|2014Q3|N|N|N|2456871|2457082|2456515|2456789|N|N|N|N|N| +2456881|AAAAAAAABDNHFCAA|2014-08-11|1375|5980|459|2014|1|8|11|3|2014|459|5980|Monday|2014Q3|N|N|N|2456871|2457082|2456516|2456790|N|N|N|N|N| +2456882|AAAAAAAACDNHFCAA|2014-08-12|1375|5981|459|2014|2|8|12|3|2014|459|5981|Tuesday|2014Q3|N|N|N|2456871|2457082|2456517|2456791|N|N|N|N|N| +2456883|AAAAAAAADDNHFCAA|2014-08-13|1375|5981|459|2014|3|8|13|3|2014|459|5981|Wednesday|2014Q3|N|N|N|2456871|2457082|2456518|2456792|N|N|N|N|N| +2456884|AAAAAAAAEDNHFCAA|2014-08-14|1375|5981|459|2014|4|8|14|3|2014|459|5981|Thursday|2014Q3|N|N|N|2456871|2457082|2456519|2456793|N|N|N|N|N| +2456885|AAAAAAAAFDNHFCAA|2014-08-15|1375|5981|459|2014|5|8|15|3|2014|459|5981|Friday|2014Q3|N|Y|N|2456871|2457082|2456520|2456794|N|N|N|N|N| +2456886|AAAAAAAAGDNHFCAA|2014-08-16|1375|5981|459|2014|6|8|16|3|2014|459|5981|Saturday|2014Q3|N|Y|N|2456871|2457082|2456521|2456795|N|N|N|N|N| +2456887|AAAAAAAAHDNHFCAA|2014-08-17|1375|5981|459|2014|0|8|17|3|2014|459|5981|Sunday|2014Q3|N|N|N|2456871|2457082|2456522|2456796|N|N|N|N|N| +2456888|AAAAAAAAIDNHFCAA|2014-08-18|1375|5981|459|2014|1|8|18|3|2014|459|5981|Monday|2014Q3|N|N|N|2456871|2457082|2456523|2456797|N|N|N|N|N| +2456889|AAAAAAAAJDNHFCAA|2014-08-19|1375|5982|459|2014|2|8|19|3|2014|459|5982|Tuesday|2014Q3|N|N|N|2456871|2457082|2456524|2456798|N|N|N|N|N| +2456890|AAAAAAAAKDNHFCAA|2014-08-20|1375|5982|459|2014|3|8|20|3|2014|459|5982|Wednesday|2014Q3|N|N|N|2456871|2457082|2456525|2456799|N|N|N|N|N| +2456891|AAAAAAAALDNHFCAA|2014-08-21|1375|5982|459|2014|4|8|21|3|2014|459|5982|Thursday|2014Q3|N|N|N|2456871|2457082|2456526|2456800|N|N|N|N|N| +2456892|AAAAAAAAMDNHFCAA|2014-08-22|1375|5982|459|2014|5|8|22|3|2014|459|5982|Friday|2014Q3|N|Y|N|2456871|2457082|2456527|2456801|N|N|N|N|N| +2456893|AAAAAAAANDNHFCAA|2014-08-23|1375|5982|459|2014|6|8|23|3|2014|459|5982|Saturday|2014Q3|N|Y|N|2456871|2457082|2456528|2456802|N|N|N|N|N| +2456894|AAAAAAAAODNHFCAA|2014-08-24|1375|5982|459|2014|0|8|24|3|2014|459|5982|Sunday|2014Q3|N|N|N|2456871|2457082|2456529|2456803|N|N|N|N|N| +2456895|AAAAAAAAPDNHFCAA|2014-08-25|1375|5982|459|2014|1|8|25|3|2014|459|5982|Monday|2014Q3|N|N|N|2456871|2457082|2456530|2456804|N|N|N|N|N| +2456896|AAAAAAAAAENHFCAA|2014-08-26|1375|5983|459|2014|2|8|26|3|2014|459|5983|Tuesday|2014Q3|N|N|N|2456871|2457082|2456531|2456805|N|N|N|N|N| +2456897|AAAAAAAABENHFCAA|2014-08-27|1375|5983|459|2014|3|8|27|3|2014|459|5983|Wednesday|2014Q3|N|N|N|2456871|2457082|2456532|2456806|N|N|N|N|N| +2456898|AAAAAAAACENHFCAA|2014-08-28|1375|5983|459|2014|4|8|28|3|2014|459|5983|Thursday|2014Q3|N|N|N|2456871|2457082|2456533|2456807|N|N|N|N|N| +2456899|AAAAAAAADENHFCAA|2014-08-29|1375|5983|459|2014|5|8|29|3|2014|459|5983|Friday|2014Q3|N|Y|N|2456871|2457082|2456534|2456808|N|N|N|N|N| +2456900|AAAAAAAAEENHFCAA|2014-08-30|1375|5983|459|2014|6|8|30|3|2014|459|5983|Saturday|2014Q3|N|Y|N|2456871|2457082|2456535|2456809|N|N|N|N|N| +2456901|AAAAAAAAFENHFCAA|2014-08-31|1375|5983|459|2014|0|8|31|3|2014|459|5983|Sunday|2014Q3|N|N|N|2456871|2457082|2456536|2456810|N|N|N|N|N| +2456902|AAAAAAAAGENHFCAA|2014-09-01|1376|5983|460|2014|1|9|1|3|2014|460|5983|Monday|2014Q3|N|N|N|2456902|2457144|2456537|2456811|N|N|N|N|N| +2456903|AAAAAAAAHENHFCAA|2014-09-02|1376|5984|460|2014|2|9|2|3|2014|460|5984|Tuesday|2014Q3|N|N|N|2456902|2457144|2456538|2456812|N|N|N|N|N| +2456904|AAAAAAAAIENHFCAA|2014-09-03|1376|5984|460|2014|3|9|3|3|2014|460|5984|Wednesday|2014Q3|N|N|N|2456902|2457144|2456539|2456813|N|N|N|N|N| +2456905|AAAAAAAAJENHFCAA|2014-09-04|1376|5984|460|2014|4|9|4|3|2014|460|5984|Thursday|2014Q3|N|N|N|2456902|2457144|2456540|2456814|N|N|N|N|N| +2456906|AAAAAAAAKENHFCAA|2014-09-05|1376|5984|460|2014|5|9|5|3|2014|460|5984|Friday|2014Q3|N|Y|N|2456902|2457144|2456541|2456815|N|N|N|N|N| +2456907|AAAAAAAALENHFCAA|2014-09-06|1376|5984|460|2014|6|9|6|3|2014|460|5984|Saturday|2014Q3|N|Y|N|2456902|2457144|2456542|2456816|N|N|N|N|N| +2456908|AAAAAAAAMENHFCAA|2014-09-07|1376|5984|460|2014|0|9|7|3|2014|460|5984|Sunday|2014Q3|N|N|N|2456902|2457144|2456543|2456817|N|N|N|N|N| +2456909|AAAAAAAANENHFCAA|2014-09-08|1376|5984|460|2014|1|9|8|3|2014|460|5984|Monday|2014Q3|N|N|N|2456902|2457144|2456544|2456818|N|N|N|N|N| +2456910|AAAAAAAAOENHFCAA|2014-09-09|1376|5985|460|2014|2|9|9|3|2014|460|5985|Tuesday|2014Q3|N|N|N|2456902|2457144|2456545|2456819|N|N|N|N|N| +2456911|AAAAAAAAPENHFCAA|2014-09-10|1376|5985|460|2014|3|9|10|3|2014|460|5985|Wednesday|2014Q3|N|N|N|2456902|2457144|2456546|2456820|N|N|N|N|N| +2456912|AAAAAAAAAFNHFCAA|2014-09-11|1376|5985|460|2014|4|9|11|3|2014|460|5985|Thursday|2014Q3|N|N|N|2456902|2457144|2456547|2456821|N|N|N|N|N| +2456913|AAAAAAAABFNHFCAA|2014-09-12|1376|5985|460|2014|5|9|12|3|2014|460|5985|Friday|2014Q3|N|Y|N|2456902|2457144|2456548|2456822|N|N|N|N|N| +2456914|AAAAAAAACFNHFCAA|2014-09-13|1376|5985|460|2014|6|9|13|3|2014|460|5985|Saturday|2014Q3|N|Y|N|2456902|2457144|2456549|2456823|N|N|N|N|N| +2456915|AAAAAAAADFNHFCAA|2014-09-14|1376|5985|460|2014|0|9|14|3|2014|460|5985|Sunday|2014Q3|N|N|N|2456902|2457144|2456550|2456824|N|N|N|N|N| +2456916|AAAAAAAAEFNHFCAA|2014-09-15|1376|5985|460|2014|1|9|15|3|2014|460|5985|Monday|2014Q3|N|N|N|2456902|2457144|2456551|2456825|N|N|N|N|N| +2456917|AAAAAAAAFFNHFCAA|2014-09-16|1376|5986|460|2014|2|9|16|3|2014|460|5986|Tuesday|2014Q3|N|N|N|2456902|2457144|2456552|2456826|N|N|N|N|N| +2456918|AAAAAAAAGFNHFCAA|2014-09-17|1376|5986|460|2014|3|9|17|3|2014|460|5986|Wednesday|2014Q3|N|N|N|2456902|2457144|2456553|2456827|N|N|N|N|N| +2456919|AAAAAAAAHFNHFCAA|2014-09-18|1376|5986|460|2014|4|9|18|3|2014|460|5986|Thursday|2014Q3|N|N|N|2456902|2457144|2456554|2456828|N|N|N|N|N| +2456920|AAAAAAAAIFNHFCAA|2014-09-19|1376|5986|460|2014|5|9|19|3|2014|460|5986|Friday|2014Q3|N|Y|N|2456902|2457144|2456555|2456829|N|N|N|N|N| +2456921|AAAAAAAAJFNHFCAA|2014-09-20|1376|5986|460|2014|6|9|20|3|2014|460|5986|Saturday|2014Q3|N|Y|N|2456902|2457144|2456556|2456830|N|N|N|N|N| +2456922|AAAAAAAAKFNHFCAA|2014-09-21|1376|5986|460|2014|0|9|21|3|2014|460|5986|Sunday|2014Q3|N|N|N|2456902|2457144|2456557|2456831|N|N|N|N|N| +2456923|AAAAAAAALFNHFCAA|2014-09-22|1376|5986|460|2014|1|9|22|3|2014|460|5986|Monday|2014Q3|N|N|N|2456902|2457144|2456558|2456832|N|N|N|N|N| +2456924|AAAAAAAAMFNHFCAA|2014-09-23|1376|5987|460|2014|2|9|23|3|2014|460|5987|Tuesday|2014Q3|N|N|N|2456902|2457144|2456559|2456833|N|N|N|N|N| +2456925|AAAAAAAANFNHFCAA|2014-09-24|1376|5987|460|2014|3|9|24|3|2014|460|5987|Wednesday|2014Q3|N|N|N|2456902|2457144|2456560|2456834|N|N|N|N|N| +2456926|AAAAAAAAOFNHFCAA|2014-09-25|1376|5987|460|2014|4|9|25|3|2014|460|5987|Thursday|2014Q3|N|N|N|2456902|2457144|2456561|2456835|N|N|N|N|N| +2456927|AAAAAAAAPFNHFCAA|2014-09-26|1376|5987|460|2014|5|9|26|3|2014|460|5987|Friday|2014Q3|N|Y|N|2456902|2457144|2456562|2456836|N|N|N|N|N| +2456928|AAAAAAAAAGNHFCAA|2014-09-27|1376|5987|460|2014|6|9|27|3|2014|460|5987|Saturday|2014Q3|N|Y|N|2456902|2457144|2456563|2456837|N|N|N|N|N| +2456929|AAAAAAAABGNHFCAA|2014-09-28|1376|5987|460|2014|0|9|28|3|2014|460|5987|Sunday|2014Q3|N|N|N|2456902|2457144|2456564|2456838|N|N|N|N|N| +2456930|AAAAAAAACGNHFCAA|2014-09-29|1376|5987|460|2014|1|9|29|3|2014|460|5987|Monday|2014Q3|N|N|N|2456902|2457144|2456565|2456839|N|N|N|N|N| +2456931|AAAAAAAADGNHFCAA|2014-09-30|1376|5988|460|2014|2|9|30|3|2014|460|5988|Tuesday|2014Q3|N|N|N|2456902|2457144|2456566|2456840|N|N|N|N|N| +2456932|AAAAAAAAEGNHFCAA|2014-10-01|1377|5988|460|2014|3|10|1|3|2014|460|5988|Wednesday|2014Q3|N|N|N|2456932|2457204|2456567|2456840|N|N|N|N|N| +2456933|AAAAAAAAFGNHFCAA|2014-10-02|1377|5988|460|2014|4|10|2|4|2014|460|5988|Thursday|2014Q4|N|N|N|2456932|2457204|2456568|2456841|N|N|N|N|N| +2456934|AAAAAAAAGGNHFCAA|2014-10-03|1377|5988|460|2014|5|10|3|4|2014|460|5988|Friday|2014Q4|N|Y|N|2456932|2457204|2456569|2456842|N|N|N|N|N| +2456935|AAAAAAAAHGNHFCAA|2014-10-04|1377|5988|460|2014|6|10|4|4|2014|460|5988|Saturday|2014Q4|N|Y|N|2456932|2457204|2456570|2456843|N|N|N|N|N| +2456936|AAAAAAAAIGNHFCAA|2014-10-05|1377|5988|460|2014|0|10|5|4|2014|460|5988|Sunday|2014Q4|N|N|N|2456932|2457204|2456571|2456844|N|N|N|N|N| +2456937|AAAAAAAAJGNHFCAA|2014-10-06|1377|5988|460|2014|1|10|6|4|2014|460|5988|Monday|2014Q4|N|N|N|2456932|2457204|2456572|2456845|N|N|N|N|N| +2456938|AAAAAAAAKGNHFCAA|2014-10-07|1377|5989|460|2014|2|10|7|4|2014|460|5989|Tuesday|2014Q4|N|N|N|2456932|2457204|2456573|2456846|N|N|N|N|N| +2456939|AAAAAAAALGNHFCAA|2014-10-08|1377|5989|460|2014|3|10|8|4|2014|460|5989|Wednesday|2014Q4|N|N|N|2456932|2457204|2456574|2456847|N|N|N|N|N| +2456940|AAAAAAAAMGNHFCAA|2014-10-09|1377|5989|460|2014|4|10|9|4|2014|460|5989|Thursday|2014Q4|N|N|N|2456932|2457204|2456575|2456848|N|N|N|N|N| +2456941|AAAAAAAANGNHFCAA|2014-10-10|1377|5989|460|2014|5|10|10|4|2014|460|5989|Friday|2014Q4|N|Y|N|2456932|2457204|2456576|2456849|N|N|N|N|N| +2456942|AAAAAAAAOGNHFCAA|2014-10-11|1377|5989|460|2014|6|10|11|4|2014|460|5989|Saturday|2014Q4|N|Y|N|2456932|2457204|2456577|2456850|N|N|N|N|N| +2456943|AAAAAAAAPGNHFCAA|2014-10-12|1377|5989|460|2014|0|10|12|4|2014|460|5989|Sunday|2014Q4|N|N|N|2456932|2457204|2456578|2456851|N|N|N|N|N| +2456944|AAAAAAAAAHNHFCAA|2014-10-13|1377|5989|460|2014|1|10|13|4|2014|460|5989|Monday|2014Q4|N|N|N|2456932|2457204|2456579|2456852|N|N|N|N|N| +2456945|AAAAAAAABHNHFCAA|2014-10-14|1377|5990|460|2014|2|10|14|4|2014|460|5990|Tuesday|2014Q4|N|N|N|2456932|2457204|2456580|2456853|N|N|N|N|N| +2456946|AAAAAAAACHNHFCAA|2014-10-15|1377|5990|460|2014|3|10|15|4|2014|460|5990|Wednesday|2014Q4|N|N|N|2456932|2457204|2456581|2456854|N|N|N|N|N| +2456947|AAAAAAAADHNHFCAA|2014-10-16|1377|5990|460|2014|4|10|16|4|2014|460|5990|Thursday|2014Q4|N|N|N|2456932|2457204|2456582|2456855|N|N|N|N|N| +2456948|AAAAAAAAEHNHFCAA|2014-10-17|1377|5990|460|2014|5|10|17|4|2014|460|5990|Friday|2014Q4|N|Y|N|2456932|2457204|2456583|2456856|N|N|N|N|N| +2456949|AAAAAAAAFHNHFCAA|2014-10-18|1377|5990|460|2014|6|10|18|4|2014|460|5990|Saturday|2014Q4|N|Y|N|2456932|2457204|2456584|2456857|N|N|N|N|N| +2456950|AAAAAAAAGHNHFCAA|2014-10-19|1377|5990|460|2014|0|10|19|4|2014|460|5990|Sunday|2014Q4|N|N|N|2456932|2457204|2456585|2456858|N|N|N|N|N| +2456951|AAAAAAAAHHNHFCAA|2014-10-20|1377|5990|460|2014|1|10|20|4|2014|460|5990|Monday|2014Q4|N|N|N|2456932|2457204|2456586|2456859|N|N|N|N|N| +2456952|AAAAAAAAIHNHFCAA|2014-10-21|1377|5991|460|2014|2|10|21|4|2014|460|5991|Tuesday|2014Q4|N|N|N|2456932|2457204|2456587|2456860|N|N|N|N|N| +2456953|AAAAAAAAJHNHFCAA|2014-10-22|1377|5991|460|2014|3|10|22|4|2014|460|5991|Wednesday|2014Q4|N|N|N|2456932|2457204|2456588|2456861|N|N|N|N|N| +2456954|AAAAAAAAKHNHFCAA|2014-10-23|1377|5991|460|2014|4|10|23|4|2014|460|5991|Thursday|2014Q4|N|N|N|2456932|2457204|2456589|2456862|N|N|N|N|N| +2456955|AAAAAAAALHNHFCAA|2014-10-24|1377|5991|460|2014|5|10|24|4|2014|460|5991|Friday|2014Q4|N|Y|N|2456932|2457204|2456590|2456863|N|N|N|N|N| +2456956|AAAAAAAAMHNHFCAA|2014-10-25|1377|5991|460|2014|6|10|25|4|2014|460|5991|Saturday|2014Q4|N|Y|N|2456932|2457204|2456591|2456864|N|N|N|N|N| +2456957|AAAAAAAANHNHFCAA|2014-10-26|1377|5991|460|2014|0|10|26|4|2014|460|5991|Sunday|2014Q4|N|N|N|2456932|2457204|2456592|2456865|N|N|N|N|N| +2456958|AAAAAAAAOHNHFCAA|2014-10-27|1377|5991|460|2014|1|10|27|4|2014|460|5991|Monday|2014Q4|N|N|N|2456932|2457204|2456593|2456866|N|N|N|N|N| +2456959|AAAAAAAAPHNHFCAA|2014-10-28|1377|5992|460|2014|2|10|28|4|2014|460|5992|Tuesday|2014Q4|N|N|N|2456932|2457204|2456594|2456867|N|N|N|N|N| +2456960|AAAAAAAAAINHFCAA|2014-10-29|1377|5992|460|2014|3|10|29|4|2014|460|5992|Wednesday|2014Q4|N|N|N|2456932|2457204|2456595|2456868|N|N|N|N|N| +2456961|AAAAAAAABINHFCAA|2014-10-30|1377|5992|460|2014|4|10|30|4|2014|460|5992|Thursday|2014Q4|N|N|N|2456932|2457204|2456596|2456869|N|N|N|N|N| +2456962|AAAAAAAACINHFCAA|2014-10-31|1377|5992|460|2014|5|10|31|4|2014|460|5992|Friday|2014Q4|N|Y|N|2456932|2457204|2456597|2456870|N|N|N|N|N| +2456963|AAAAAAAADINHFCAA|2014-11-01|1378|5992|460|2014|6|11|1|4|2014|460|5992|Saturday|2014Q4|N|Y|N|2456963|2457266|2456598|2456871|N|N|N|N|N| +2456964|AAAAAAAAEINHFCAA|2014-11-02|1378|5992|460|2014|0|11|2|4|2014|460|5992|Sunday|2014Q4|N|N|N|2456963|2457266|2456599|2456872|N|N|N|N|N| +2456965|AAAAAAAAFINHFCAA|2014-11-03|1378|5992|460|2014|1|11|3|4|2014|460|5992|Monday|2014Q4|N|N|N|2456963|2457266|2456600|2456873|N|N|N|N|N| +2456966|AAAAAAAAGINHFCAA|2014-11-04|1378|5993|460|2014|2|11|4|4|2014|460|5993|Tuesday|2014Q4|N|N|N|2456963|2457266|2456601|2456874|N|N|N|N|N| +2456967|AAAAAAAAHINHFCAA|2014-11-05|1378|5993|460|2014|3|11|5|4|2014|460|5993|Wednesday|2014Q4|N|N|N|2456963|2457266|2456602|2456875|N|N|N|N|N| +2456968|AAAAAAAAIINHFCAA|2014-11-06|1378|5993|460|2014|4|11|6|4|2014|460|5993|Thursday|2014Q4|N|N|N|2456963|2457266|2456603|2456876|N|N|N|N|N| +2456969|AAAAAAAAJINHFCAA|2014-11-07|1378|5993|460|2014|5|11|7|4|2014|460|5993|Friday|2014Q4|N|Y|N|2456963|2457266|2456604|2456877|N|N|N|N|N| +2456970|AAAAAAAAKINHFCAA|2014-11-08|1378|5993|460|2014|6|11|8|4|2014|460|5993|Saturday|2014Q4|N|Y|N|2456963|2457266|2456605|2456878|N|N|N|N|N| +2456971|AAAAAAAALINHFCAA|2014-11-09|1378|5993|460|2014|0|11|9|4|2014|460|5993|Sunday|2014Q4|N|N|N|2456963|2457266|2456606|2456879|N|N|N|N|N| +2456972|AAAAAAAAMINHFCAA|2014-11-10|1378|5993|460|2014|1|11|10|4|2014|460|5993|Monday|2014Q4|N|N|N|2456963|2457266|2456607|2456880|N|N|N|N|N| +2456973|AAAAAAAANINHFCAA|2014-11-11|1378|5994|460|2014|2|11|11|4|2014|460|5994|Tuesday|2014Q4|N|N|N|2456963|2457266|2456608|2456881|N|N|N|N|N| +2456974|AAAAAAAAOINHFCAA|2014-11-12|1378|5994|460|2014|3|11|12|4|2014|460|5994|Wednesday|2014Q4|N|N|N|2456963|2457266|2456609|2456882|N|N|N|N|N| +2456975|AAAAAAAAPINHFCAA|2014-11-13|1378|5994|460|2014|4|11|13|4|2014|460|5994|Thursday|2014Q4|N|N|N|2456963|2457266|2456610|2456883|N|N|N|N|N| +2456976|AAAAAAAAAJNHFCAA|2014-11-14|1378|5994|460|2014|5|11|14|4|2014|460|5994|Friday|2014Q4|N|Y|N|2456963|2457266|2456611|2456884|N|N|N|N|N| +2456977|AAAAAAAABJNHFCAA|2014-11-15|1378|5994|460|2014|6|11|15|4|2014|460|5994|Saturday|2014Q4|N|Y|N|2456963|2457266|2456612|2456885|N|N|N|N|N| +2456978|AAAAAAAACJNHFCAA|2014-11-16|1378|5994|460|2014|0|11|16|4|2014|460|5994|Sunday|2014Q4|N|N|N|2456963|2457266|2456613|2456886|N|N|N|N|N| +2456979|AAAAAAAADJNHFCAA|2014-11-17|1378|5994|460|2014|1|11|17|4|2014|460|5994|Monday|2014Q4|N|N|N|2456963|2457266|2456614|2456887|N|N|N|N|N| +2456980|AAAAAAAAEJNHFCAA|2014-11-18|1378|5995|460|2014|2|11|18|4|2014|460|5995|Tuesday|2014Q4|N|N|N|2456963|2457266|2456615|2456888|N|N|N|N|N| +2456981|AAAAAAAAFJNHFCAA|2014-11-19|1378|5995|460|2014|3|11|19|4|2014|460|5995|Wednesday|2014Q4|N|N|N|2456963|2457266|2456616|2456889|N|N|N|N|N| +2456982|AAAAAAAAGJNHFCAA|2014-11-20|1378|5995|460|2014|4|11|20|4|2014|460|5995|Thursday|2014Q4|N|N|N|2456963|2457266|2456617|2456890|N|N|N|N|N| +2456983|AAAAAAAAHJNHFCAA|2014-11-21|1378|5995|460|2014|5|11|21|4|2014|460|5995|Friday|2014Q4|N|Y|N|2456963|2457266|2456618|2456891|N|N|N|N|N| +2456984|AAAAAAAAIJNHFCAA|2014-11-22|1378|5995|460|2014|6|11|22|4|2014|460|5995|Saturday|2014Q4|N|Y|N|2456963|2457266|2456619|2456892|N|N|N|N|N| +2456985|AAAAAAAAJJNHFCAA|2014-11-23|1378|5995|460|2014|0|11|23|4|2014|460|5995|Sunday|2014Q4|N|N|N|2456963|2457266|2456620|2456893|N|N|N|N|N| +2456986|AAAAAAAAKJNHFCAA|2014-11-24|1378|5995|460|2014|1|11|24|4|2014|460|5995|Monday|2014Q4|N|N|N|2456963|2457266|2456621|2456894|N|N|N|N|N| +2456987|AAAAAAAALJNHFCAA|2014-11-25|1378|5996|460|2014|2|11|25|4|2014|460|5996|Tuesday|2014Q4|N|N|N|2456963|2457266|2456622|2456895|N|N|N|N|N| +2456988|AAAAAAAAMJNHFCAA|2014-11-26|1378|5996|460|2014|3|11|26|4|2014|460|5996|Wednesday|2014Q4|N|N|N|2456963|2457266|2456623|2456896|N|N|N|N|N| +2456989|AAAAAAAANJNHFCAA|2014-11-27|1378|5996|460|2014|4|11|27|4|2014|460|5996|Thursday|2014Q4|N|N|N|2456963|2457266|2456624|2456897|N|N|N|N|N| +2456990|AAAAAAAAOJNHFCAA|2014-11-28|1378|5996|460|2014|5|11|28|4|2014|460|5996|Friday|2014Q4|N|Y|N|2456963|2457266|2456625|2456898|N|N|N|N|N| +2456991|AAAAAAAAPJNHFCAA|2014-11-29|1378|5996|460|2014|6|11|29|4|2014|460|5996|Saturday|2014Q4|N|Y|N|2456963|2457266|2456626|2456899|N|N|N|N|N| +2456992|AAAAAAAAAKNHFCAA|2014-11-30|1378|5996|460|2014|0|11|30|4|2014|460|5996|Sunday|2014Q4|N|N|N|2456963|2457266|2456627|2456900|N|N|N|N|N| +2456993|AAAAAAAABKNHFCAA|2014-12-01|1379|5996|461|2014|1|12|1|4|2014|461|5996|Monday|2014Q4|N|N|N|2456993|2457326|2456628|2456901|N|N|N|N|N| +2456994|AAAAAAAACKNHFCAA|2014-12-02|1379|5997|461|2014|2|12|2|4|2014|461|5997|Tuesday|2014Q4|N|N|N|2456993|2457326|2456629|2456902|N|N|N|N|N| +2456995|AAAAAAAADKNHFCAA|2014-12-03|1379|5997|461|2014|3|12|3|4|2014|461|5997|Wednesday|2014Q4|N|N|N|2456993|2457326|2456630|2456903|N|N|N|N|N| +2456996|AAAAAAAAEKNHFCAA|2014-12-04|1379|5997|461|2014|4|12|4|4|2014|461|5997|Thursday|2014Q4|N|N|N|2456993|2457326|2456631|2456904|N|N|N|N|N| +2456997|AAAAAAAAFKNHFCAA|2014-12-05|1379|5997|461|2014|5|12|5|4|2014|461|5997|Friday|2014Q4|N|Y|N|2456993|2457326|2456632|2456905|N|N|N|N|N| +2456998|AAAAAAAAGKNHFCAA|2014-12-06|1379|5997|461|2014|6|12|6|4|2014|461|5997|Saturday|2014Q4|N|Y|N|2456993|2457326|2456633|2456906|N|N|N|N|N| +2456999|AAAAAAAAHKNHFCAA|2014-12-07|1379|5997|461|2014|0|12|7|4|2014|461|5997|Sunday|2014Q4|N|N|N|2456993|2457326|2456634|2456907|N|N|N|N|N| +2457000|AAAAAAAAIKNHFCAA|2014-12-08|1379|5997|461|2014|1|12|8|4|2014|461|5997|Monday|2014Q4|N|N|N|2456993|2457326|2456635|2456908|N|N|N|N|N| +2457001|AAAAAAAAJKNHFCAA|2014-12-09|1379|5998|461|2014|2|12|9|4|2014|461|5998|Tuesday|2014Q4|N|N|N|2456993|2457326|2456636|2456909|N|N|N|N|N| +2457002|AAAAAAAAKKNHFCAA|2014-12-10|1379|5998|461|2014|3|12|10|4|2014|461|5998|Wednesday|2014Q4|N|N|N|2456993|2457326|2456637|2456910|N|N|N|N|N| +2457003|AAAAAAAALKNHFCAA|2014-12-11|1379|5998|461|2014|4|12|11|4|2014|461|5998|Thursday|2014Q4|N|N|N|2456993|2457326|2456638|2456911|N|N|N|N|N| +2457004|AAAAAAAAMKNHFCAA|2014-12-12|1379|5998|461|2014|5|12|12|4|2014|461|5998|Friday|2014Q4|N|Y|N|2456993|2457326|2456639|2456912|N|N|N|N|N| +2457005|AAAAAAAANKNHFCAA|2014-12-13|1379|5998|461|2014|6|12|13|4|2014|461|5998|Saturday|2014Q4|N|Y|N|2456993|2457326|2456640|2456913|N|N|N|N|N| +2457006|AAAAAAAAOKNHFCAA|2014-12-14|1379|5998|461|2014|0|12|14|4|2014|461|5998|Sunday|2014Q4|N|N|N|2456993|2457326|2456641|2456914|N|N|N|N|N| +2457007|AAAAAAAAPKNHFCAA|2014-12-15|1379|5998|461|2014|1|12|15|4|2014|461|5998|Monday|2014Q4|N|N|N|2456993|2457326|2456642|2456915|N|N|N|N|N| +2457008|AAAAAAAAALNHFCAA|2014-12-16|1379|5999|461|2014|2|12|16|4|2014|461|5999|Tuesday|2014Q4|N|N|N|2456993|2457326|2456643|2456916|N|N|N|N|N| +2457009|AAAAAAAABLNHFCAA|2014-12-17|1379|5999|461|2014|3|12|17|4|2014|461|5999|Wednesday|2014Q4|N|N|N|2456993|2457326|2456644|2456917|N|N|N|N|N| +2457010|AAAAAAAACLNHFCAA|2014-12-18|1379|5999|461|2014|4|12|18|4|2014|461|5999|Thursday|2014Q4|N|N|N|2456993|2457326|2456645|2456918|N|N|N|N|N| +2457011|AAAAAAAADLNHFCAA|2014-12-19|1379|5999|461|2014|5|12|19|4|2014|461|5999|Friday|2014Q4|N|Y|N|2456993|2457326|2456646|2456919|N|N|N|N|N| +2457012|AAAAAAAAELNHFCAA|2014-12-20|1379|5999|461|2014|6|12|20|4|2014|461|5999|Saturday|2014Q4|N|Y|N|2456993|2457326|2456647|2456920|N|N|N|N|N| +2457013|AAAAAAAAFLNHFCAA|2014-12-21|1379|5999|461|2014|0|12|21|4|2014|461|5999|Sunday|2014Q4|N|N|N|2456993|2457326|2456648|2456921|N|N|N|N|N| +2457014|AAAAAAAAGLNHFCAA|2014-12-22|1379|5999|461|2014|1|12|22|4|2014|461|5999|Monday|2014Q4|N|N|N|2456993|2457326|2456649|2456922|N|N|N|N|N| +2457015|AAAAAAAAHLNHFCAA|2014-12-23|1379|6000|461|2014|2|12|23|4|2014|461|6000|Tuesday|2014Q4|N|N|N|2456993|2457326|2456650|2456923|N|N|N|N|N| +2457016|AAAAAAAAILNHFCAA|2014-12-24|1379|6000|461|2014|3|12|24|4|2014|461|6000|Wednesday|2014Q4|N|N|N|2456993|2457326|2456651|2456924|N|N|N|N|N| +2457017|AAAAAAAAJLNHFCAA|2014-12-25|1379|6000|461|2014|4|12|25|4|2014|461|6000|Thursday|2014Q4|N|N|N|2456993|2457326|2456652|2456925|N|N|N|N|N| +2457018|AAAAAAAAKLNHFCAA|2014-12-26|1379|6000|461|2014|5|12|26|4|2014|461|6000|Friday|2014Q4|Y|Y|N|2456993|2457326|2456653|2456926|N|N|N|N|N| +2457019|AAAAAAAALLNHFCAA|2014-12-27|1379|6000|461|2014|6|12|27|4|2014|461|6000|Saturday|2014Q4|N|Y|Y|2456993|2457326|2456654|2456927|N|N|N|N|N| +2457020|AAAAAAAAMLNHFCAA|2014-12-28|1379|6000|461|2014|0|12|28|4|2014|461|6000|Sunday|2014Q4|N|N|N|2456993|2457326|2456655|2456928|N|N|N|N|N| +2457021|AAAAAAAANLNHFCAA|2014-12-29|1379|6000|461|2014|1|12|29|4|2014|461|6000|Monday|2014Q4|N|N|N|2456993|2457326|2456656|2456929|N|N|N|N|N| +2457022|AAAAAAAAOLNHFCAA|2014-12-30|1379|6001|461|2014|2|12|30|4|2014|461|6001|Tuesday|2014Q4|N|N|N|2456993|2457326|2456657|2456930|N|N|N|N|N| +2457023|AAAAAAAAPLNHFCAA|2014-12-31|1379|6001|461|2014|3|12|31|4|2014|461|6001|Wednesday|2014Q4|N|N|N|2456993|2457326|2456658|2456931|N|N|N|N|N| +2457024|AAAAAAAAAMNHFCAA|2015-01-01|1380|6001|461|2015|4|1|1|1|2015|461|6001|Thursday|2015Q1|Y|N|N|2457024|2457023|2456659|2456932|N|N|N|N|N| +2457025|AAAAAAAABMNHFCAA|2015-01-02|1380|6001|461|2015|5|1|2|1|2015|461|6001|Friday|2015Q1|N|Y|Y|2457024|2457023|2456660|2456933|N|N|N|N|N| +2457026|AAAAAAAACMNHFCAA|2015-01-03|1380|6001|461|2015|6|1|3|1|2015|461|6001|Saturday|2015Q1|N|Y|N|2457024|2457023|2456661|2456934|N|N|N|N|N| +2457027|AAAAAAAADMNHFCAA|2015-01-04|1380|6001|461|2015|0|1|4|1|2015|461|6001|Sunday|2015Q1|N|N|N|2457024|2457023|2456662|2456935|N|N|N|N|N| +2457028|AAAAAAAAEMNHFCAA|2015-01-05|1380|6001|461|2015|1|1|5|1|2015|461|6001|Monday|2015Q1|N|N|N|2457024|2457023|2456663|2456936|N|N|N|N|N| +2457029|AAAAAAAAFMNHFCAA|2015-01-06|1380|6002|461|2015|2|1|6|1|2015|461|6002|Tuesday|2015Q1|N|N|N|2457024|2457023|2456664|2456937|N|N|N|N|N| +2457030|AAAAAAAAGMNHFCAA|2015-01-07|1380|6002|461|2015|3|1|7|1|2015|461|6002|Wednesday|2015Q1|N|N|N|2457024|2457023|2456665|2456938|N|N|N|N|N| +2457031|AAAAAAAAHMNHFCAA|2015-01-08|1380|6002|461|2015|4|1|8|1|2015|461|6002|Thursday|2015Q1|N|N|N|2457024|2457023|2456666|2456939|N|N|N|N|N| +2457032|AAAAAAAAIMNHFCAA|2015-01-09|1380|6002|461|2015|5|1|9|1|2015|461|6002|Friday|2015Q1|N|Y|N|2457024|2457023|2456667|2456940|N|N|N|N|N| +2457033|AAAAAAAAJMNHFCAA|2015-01-10|1380|6002|461|2015|6|1|10|1|2015|461|6002|Saturday|2015Q1|N|Y|N|2457024|2457023|2456668|2456941|N|N|N|N|N| +2457034|AAAAAAAAKMNHFCAA|2015-01-11|1380|6002|461|2015|0|1|11|1|2015|461|6002|Sunday|2015Q1|N|N|N|2457024|2457023|2456669|2456942|N|N|N|N|N| +2457035|AAAAAAAALMNHFCAA|2015-01-12|1380|6002|461|2015|1|1|12|1|2015|461|6002|Monday|2015Q1|N|N|N|2457024|2457023|2456670|2456943|N|N|N|N|N| +2457036|AAAAAAAAMMNHFCAA|2015-01-13|1380|6003|461|2015|2|1|13|1|2015|461|6003|Tuesday|2015Q1|N|N|N|2457024|2457023|2456671|2456944|N|N|N|N|N| +2457037|AAAAAAAANMNHFCAA|2015-01-14|1380|6003|461|2015|3|1|14|1|2015|461|6003|Wednesday|2015Q1|N|N|N|2457024|2457023|2456672|2456945|N|N|N|N|N| +2457038|AAAAAAAAOMNHFCAA|2015-01-15|1380|6003|461|2015|4|1|15|1|2015|461|6003|Thursday|2015Q1|N|N|N|2457024|2457023|2456673|2456946|N|N|N|N|N| +2457039|AAAAAAAAPMNHFCAA|2015-01-16|1380|6003|461|2015|5|1|16|1|2015|461|6003|Friday|2015Q1|N|Y|N|2457024|2457023|2456674|2456947|N|N|N|N|N| +2457040|AAAAAAAAANNHFCAA|2015-01-17|1380|6003|461|2015|6|1|17|1|2015|461|6003|Saturday|2015Q1|N|Y|N|2457024|2457023|2456675|2456948|N|N|N|N|N| +2457041|AAAAAAAABNNHFCAA|2015-01-18|1380|6003|461|2015|0|1|18|1|2015|461|6003|Sunday|2015Q1|N|N|N|2457024|2457023|2456676|2456949|N|N|N|N|N| +2457042|AAAAAAAACNNHFCAA|2015-01-19|1380|6003|461|2015|1|1|19|1|2015|461|6003|Monday|2015Q1|N|N|N|2457024|2457023|2456677|2456950|N|N|N|N|N| +2457043|AAAAAAAADNNHFCAA|2015-01-20|1380|6004|461|2015|2|1|20|1|2015|461|6004|Tuesday|2015Q1|N|N|N|2457024|2457023|2456678|2456951|N|N|N|N|N| +2457044|AAAAAAAAENNHFCAA|2015-01-21|1380|6004|461|2015|3|1|21|1|2015|461|6004|Wednesday|2015Q1|N|N|N|2457024|2457023|2456679|2456952|N|N|N|N|N| +2457045|AAAAAAAAFNNHFCAA|2015-01-22|1380|6004|461|2015|4|1|22|1|2015|461|6004|Thursday|2015Q1|N|N|N|2457024|2457023|2456680|2456953|N|N|N|N|N| +2457046|AAAAAAAAGNNHFCAA|2015-01-23|1380|6004|461|2015|5|1|23|1|2015|461|6004|Friday|2015Q1|N|Y|N|2457024|2457023|2456681|2456954|N|N|N|N|N| +2457047|AAAAAAAAHNNHFCAA|2015-01-24|1380|6004|461|2015|6|1|24|1|2015|461|6004|Saturday|2015Q1|N|Y|N|2457024|2457023|2456682|2456955|N|N|N|N|N| +2457048|AAAAAAAAINNHFCAA|2015-01-25|1380|6004|461|2015|0|1|25|1|2015|461|6004|Sunday|2015Q1|N|N|N|2457024|2457023|2456683|2456956|N|N|N|N|N| +2457049|AAAAAAAAJNNHFCAA|2015-01-26|1380|6004|461|2015|1|1|26|1|2015|461|6004|Monday|2015Q1|N|N|N|2457024|2457023|2456684|2456957|N|N|N|N|N| +2457050|AAAAAAAAKNNHFCAA|2015-01-27|1380|6005|461|2015|2|1|27|1|2015|461|6005|Tuesday|2015Q1|N|N|N|2457024|2457023|2456685|2456958|N|N|N|N|N| +2457051|AAAAAAAALNNHFCAA|2015-01-28|1380|6005|461|2015|3|1|28|1|2015|461|6005|Wednesday|2015Q1|N|N|N|2457024|2457023|2456686|2456959|N|N|N|N|N| +2457052|AAAAAAAAMNNHFCAA|2015-01-29|1380|6005|461|2015|4|1|29|1|2015|461|6005|Thursday|2015Q1|N|N|N|2457024|2457023|2456687|2456960|N|N|N|N|N| +2457053|AAAAAAAANNNHFCAA|2015-01-30|1380|6005|461|2015|5|1|30|1|2015|461|6005|Friday|2015Q1|N|Y|N|2457024|2457023|2456688|2456961|N|N|N|N|N| +2457054|AAAAAAAAONNHFCAA|2015-01-31|1380|6005|461|2015|6|1|31|1|2015|461|6005|Saturday|2015Q1|N|Y|N|2457024|2457023|2456689|2456962|N|N|N|N|N| +2457055|AAAAAAAAPNNHFCAA|2015-02-01|1381|6005|461|2015|0|2|1|1|2015|461|6005|Sunday|2015Q1|N|N|N|2457055|2457085|2456690|2456963|N|N|N|N|N| +2457056|AAAAAAAAAONHFCAA|2015-02-02|1381|6005|461|2015|1|2|2|1|2015|461|6005|Monday|2015Q1|N|N|N|2457055|2457085|2456691|2456964|N|N|N|N|N| +2457057|AAAAAAAABONHFCAA|2015-02-03|1381|6006|461|2015|2|2|3|1|2015|461|6006|Tuesday|2015Q1|N|N|N|2457055|2457085|2456692|2456965|N|N|N|N|N| +2457058|AAAAAAAACONHFCAA|2015-02-04|1381|6006|461|2015|3|2|4|1|2015|461|6006|Wednesday|2015Q1|N|N|N|2457055|2457085|2456693|2456966|N|N|N|N|N| +2457059|AAAAAAAADONHFCAA|2015-02-05|1381|6006|461|2015|4|2|5|1|2015|461|6006|Thursday|2015Q1|N|N|N|2457055|2457085|2456694|2456967|N|N|N|N|N| +2457060|AAAAAAAAEONHFCAA|2015-02-06|1381|6006|461|2015|5|2|6|1|2015|461|6006|Friday|2015Q1|N|Y|N|2457055|2457085|2456695|2456968|N|N|N|N|N| +2457061|AAAAAAAAFONHFCAA|2015-02-07|1381|6006|461|2015|6|2|7|1|2015|461|6006|Saturday|2015Q1|N|Y|N|2457055|2457085|2456696|2456969|N|N|N|N|N| +2457062|AAAAAAAAGONHFCAA|2015-02-08|1381|6006|461|2015|0|2|8|1|2015|461|6006|Sunday|2015Q1|N|N|N|2457055|2457085|2456697|2456970|N|N|N|N|N| +2457063|AAAAAAAAHONHFCAA|2015-02-09|1381|6006|461|2015|1|2|9|1|2015|461|6006|Monday|2015Q1|N|N|N|2457055|2457085|2456698|2456971|N|N|N|N|N| +2457064|AAAAAAAAIONHFCAA|2015-02-10|1381|6007|461|2015|2|2|10|1|2015|461|6007|Tuesday|2015Q1|N|N|N|2457055|2457085|2456699|2456972|N|N|N|N|N| +2457065|AAAAAAAAJONHFCAA|2015-02-11|1381|6007|461|2015|3|2|11|1|2015|461|6007|Wednesday|2015Q1|N|N|N|2457055|2457085|2456700|2456973|N|N|N|N|N| +2457066|AAAAAAAAKONHFCAA|2015-02-12|1381|6007|461|2015|4|2|12|1|2015|461|6007|Thursday|2015Q1|N|N|N|2457055|2457085|2456701|2456974|N|N|N|N|N| +2457067|AAAAAAAALONHFCAA|2015-02-13|1381|6007|461|2015|5|2|13|1|2015|461|6007|Friday|2015Q1|N|Y|N|2457055|2457085|2456702|2456975|N|N|N|N|N| +2457068|AAAAAAAAMONHFCAA|2015-02-14|1381|6007|461|2015|6|2|14|1|2015|461|6007|Saturday|2015Q1|N|Y|N|2457055|2457085|2456703|2456976|N|N|N|N|N| +2457069|AAAAAAAANONHFCAA|2015-02-15|1381|6007|461|2015|0|2|15|1|2015|461|6007|Sunday|2015Q1|N|N|N|2457055|2457085|2456704|2456977|N|N|N|N|N| +2457070|AAAAAAAAOONHFCAA|2015-02-16|1381|6007|461|2015|1|2|16|1|2015|461|6007|Monday|2015Q1|N|N|N|2457055|2457085|2456705|2456978|N|N|N|N|N| +2457071|AAAAAAAAPONHFCAA|2015-02-17|1381|6008|461|2015|2|2|17|1|2015|461|6008|Tuesday|2015Q1|N|N|N|2457055|2457085|2456706|2456979|N|N|N|N|N| +2457072|AAAAAAAAAPNHFCAA|2015-02-18|1381|6008|461|2015|3|2|18|1|2015|461|6008|Wednesday|2015Q1|N|N|N|2457055|2457085|2456707|2456980|N|N|N|N|N| +2457073|AAAAAAAABPNHFCAA|2015-02-19|1381|6008|461|2015|4|2|19|1|2015|461|6008|Thursday|2015Q1|N|N|N|2457055|2457085|2456708|2456981|N|N|N|N|N| +2457074|AAAAAAAACPNHFCAA|2015-02-20|1381|6008|461|2015|5|2|20|1|2015|461|6008|Friday|2015Q1|N|Y|N|2457055|2457085|2456709|2456982|N|N|N|N|N| +2457075|AAAAAAAADPNHFCAA|2015-02-21|1381|6008|461|2015|6|2|21|1|2015|461|6008|Saturday|2015Q1|N|Y|N|2457055|2457085|2456710|2456983|N|N|N|N|N| +2457076|AAAAAAAAEPNHFCAA|2015-02-22|1381|6008|461|2015|0|2|22|1|2015|461|6008|Sunday|2015Q1|N|N|N|2457055|2457085|2456711|2456984|N|N|N|N|N| +2457077|AAAAAAAAFPNHFCAA|2015-02-23|1381|6008|461|2015|1|2|23|1|2015|461|6008|Monday|2015Q1|N|N|N|2457055|2457085|2456712|2456985|N|N|N|N|N| +2457078|AAAAAAAAGPNHFCAA|2015-02-24|1381|6009|461|2015|2|2|24|1|2015|461|6009|Tuesday|2015Q1|N|N|N|2457055|2457085|2456713|2456986|N|N|N|N|N| +2457079|AAAAAAAAHPNHFCAA|2015-02-25|1381|6009|461|2015|3|2|25|1|2015|461|6009|Wednesday|2015Q1|N|N|N|2457055|2457085|2456714|2456987|N|N|N|N|N| +2457080|AAAAAAAAIPNHFCAA|2015-02-26|1381|6009|461|2015|4|2|26|1|2015|461|6009|Thursday|2015Q1|N|N|N|2457055|2457085|2456715|2456988|N|N|N|N|N| +2457081|AAAAAAAAJPNHFCAA|2015-02-27|1381|6009|461|2015|5|2|27|1|2015|461|6009|Friday|2015Q1|N|Y|N|2457055|2457085|2456716|2456989|N|N|N|N|N| +2457082|AAAAAAAAKPNHFCAA|2015-02-28|1381|6009|461|2015|6|2|28|1|2015|461|6009|Saturday|2015Q1|N|Y|N|2457055|2457085|2456717|2456990|N|N|N|N|N| +2457083|AAAAAAAALPNHFCAA|2015-03-01|1382|6009|462|2015|0|3|1|1|2015|462|6009|Sunday|2015Q1|N|N|N|2457083|2457141|2456718|2456991|N|N|N|N|N| +2457084|AAAAAAAAMPNHFCAA|2015-03-02|1382|6009|462|2015|1|3|2|1|2015|462|6009|Monday|2015Q1|N|N|N|2457083|2457141|2456719|2456992|N|N|N|N|N| +2457085|AAAAAAAANPNHFCAA|2015-03-03|1382|6010|462|2015|2|3|3|1|2015|462|6010|Tuesday|2015Q1|N|N|N|2457083|2457141|2456720|2456993|N|N|N|N|N| +2457086|AAAAAAAAOPNHFCAA|2015-03-04|1382|6010|462|2015|3|3|4|1|2015|462|6010|Wednesday|2015Q1|N|N|N|2457083|2457141|2456721|2456994|N|N|N|N|N| +2457087|AAAAAAAAPPNHFCAA|2015-03-05|1382|6010|462|2015|4|3|5|1|2015|462|6010|Thursday|2015Q1|N|N|N|2457083|2457141|2456722|2456995|N|N|N|N|N| +2457088|AAAAAAAAAAOHFCAA|2015-03-06|1382|6010|462|2015|5|3|6|1|2015|462|6010|Friday|2015Q1|N|Y|N|2457083|2457141|2456723|2456996|N|N|N|N|N| +2457089|AAAAAAAABAOHFCAA|2015-03-07|1382|6010|462|2015|6|3|7|1|2015|462|6010|Saturday|2015Q1|N|Y|N|2457083|2457141|2456724|2456997|N|N|N|N|N| +2457090|AAAAAAAACAOHFCAA|2015-03-08|1382|6010|462|2015|0|3|8|1|2015|462|6010|Sunday|2015Q1|N|N|N|2457083|2457141|2456725|2456998|N|N|N|N|N| +2457091|AAAAAAAADAOHFCAA|2015-03-09|1382|6010|462|2015|1|3|9|1|2015|462|6010|Monday|2015Q1|N|N|N|2457083|2457141|2456726|2456999|N|N|N|N|N| +2457092|AAAAAAAAEAOHFCAA|2015-03-10|1382|6011|462|2015|2|3|10|1|2015|462|6011|Tuesday|2015Q1|N|N|N|2457083|2457141|2456727|2457000|N|N|N|N|N| +2457093|AAAAAAAAFAOHFCAA|2015-03-11|1382|6011|462|2015|3|3|11|1|2015|462|6011|Wednesday|2015Q1|N|N|N|2457083|2457141|2456728|2457001|N|N|N|N|N| +2457094|AAAAAAAAGAOHFCAA|2015-03-12|1382|6011|462|2015|4|3|12|1|2015|462|6011|Thursday|2015Q1|N|N|N|2457083|2457141|2456729|2457002|N|N|N|N|N| +2457095|AAAAAAAAHAOHFCAA|2015-03-13|1382|6011|462|2015|5|3|13|1|2015|462|6011|Friday|2015Q1|N|Y|N|2457083|2457141|2456730|2457003|N|N|N|N|N| +2457096|AAAAAAAAIAOHFCAA|2015-03-14|1382|6011|462|2015|6|3|14|1|2015|462|6011|Saturday|2015Q1|N|Y|N|2457083|2457141|2456731|2457004|N|N|N|N|N| +2457097|AAAAAAAAJAOHFCAA|2015-03-15|1382|6011|462|2015|0|3|15|1|2015|462|6011|Sunday|2015Q1|N|N|N|2457083|2457141|2456732|2457005|N|N|N|N|N| +2457098|AAAAAAAAKAOHFCAA|2015-03-16|1382|6011|462|2015|1|3|16|1|2015|462|6011|Monday|2015Q1|N|N|N|2457083|2457141|2456733|2457006|N|N|N|N|N| +2457099|AAAAAAAALAOHFCAA|2015-03-17|1382|6012|462|2015|2|3|17|1|2015|462|6012|Tuesday|2015Q1|N|N|N|2457083|2457141|2456734|2457007|N|N|N|N|N| +2457100|AAAAAAAAMAOHFCAA|2015-03-18|1382|6012|462|2015|3|3|18|1|2015|462|6012|Wednesday|2015Q1|N|N|N|2457083|2457141|2456735|2457008|N|N|N|N|N| +2457101|AAAAAAAANAOHFCAA|2015-03-19|1382|6012|462|2015|4|3|19|1|2015|462|6012|Thursday|2015Q1|N|N|N|2457083|2457141|2456736|2457009|N|N|N|N|N| +2457102|AAAAAAAAOAOHFCAA|2015-03-20|1382|6012|462|2015|5|3|20|1|2015|462|6012|Friday|2015Q1|N|Y|N|2457083|2457141|2456737|2457010|N|N|N|N|N| +2457103|AAAAAAAAPAOHFCAA|2015-03-21|1382|6012|462|2015|6|3|21|1|2015|462|6012|Saturday|2015Q1|N|Y|N|2457083|2457141|2456738|2457011|N|N|N|N|N| +2457104|AAAAAAAAABOHFCAA|2015-03-22|1382|6012|462|2015|0|3|22|1|2015|462|6012|Sunday|2015Q1|N|N|N|2457083|2457141|2456739|2457012|N|N|N|N|N| +2457105|AAAAAAAABBOHFCAA|2015-03-23|1382|6012|462|2015|1|3|23|1|2015|462|6012|Monday|2015Q1|N|N|N|2457083|2457141|2456740|2457013|N|N|N|N|N| +2457106|AAAAAAAACBOHFCAA|2015-03-24|1382|6013|462|2015|2|3|24|1|2015|462|6013|Tuesday|2015Q1|N|N|N|2457083|2457141|2456741|2457014|N|N|N|N|N| +2457107|AAAAAAAADBOHFCAA|2015-03-25|1382|6013|462|2015|3|3|25|1|2015|462|6013|Wednesday|2015Q1|N|N|N|2457083|2457141|2456742|2457015|N|N|N|N|N| +2457108|AAAAAAAAEBOHFCAA|2015-03-26|1382|6013|462|2015|4|3|26|1|2015|462|6013|Thursday|2015Q1|N|N|N|2457083|2457141|2456743|2457016|N|N|N|N|N| +2457109|AAAAAAAAFBOHFCAA|2015-03-27|1382|6013|462|2015|5|3|27|1|2015|462|6013|Friday|2015Q1|N|Y|N|2457083|2457141|2456744|2457017|N|N|N|N|N| +2457110|AAAAAAAAGBOHFCAA|2015-03-28|1382|6013|462|2015|6|3|28|1|2015|462|6013|Saturday|2015Q1|N|Y|N|2457083|2457141|2456745|2457018|N|N|N|N|N| +2457111|AAAAAAAAHBOHFCAA|2015-03-29|1382|6013|462|2015|0|3|29|1|2015|462|6013|Sunday|2015Q1|N|N|N|2457083|2457141|2456746|2457019|N|N|N|N|N| +2457112|AAAAAAAAIBOHFCAA|2015-03-30|1382|6013|462|2015|1|3|30|1|2015|462|6013|Monday|2015Q1|N|N|N|2457083|2457141|2456747|2457020|N|N|N|N|N| +2457113|AAAAAAAAJBOHFCAA|2015-03-31|1382|6014|462|2015|2|3|31|1|2015|462|6014|Tuesday|2015Q1|N|N|N|2457083|2457141|2456748|2457021|N|N|N|N|N| +2457114|AAAAAAAAKBOHFCAA|2015-04-01|1383|6014|462|2015|3|4|1|1|2015|462|6014|Wednesday|2015Q1|N|N|N|2457114|2457203|2456749|2457024|N|N|N|N|N| +2457115|AAAAAAAALBOHFCAA|2015-04-02|1383|6014|462|2015|4|4|2|2|2015|462|6014|Thursday|2015Q2|N|N|N|2457114|2457203|2456750|2457025|N|N|N|N|N| +2457116|AAAAAAAAMBOHFCAA|2015-04-03|1383|6014|462|2015|5|4|3|2|2015|462|6014|Friday|2015Q2|N|Y|N|2457114|2457203|2456751|2457026|N|N|N|N|N| +2457117|AAAAAAAANBOHFCAA|2015-04-04|1383|6014|462|2015|6|4|4|2|2015|462|6014|Saturday|2015Q2|N|Y|N|2457114|2457203|2456752|2457027|N|N|N|N|N| +2457118|AAAAAAAAOBOHFCAA|2015-04-05|1383|6014|462|2015|0|4|5|2|2015|462|6014|Sunday|2015Q2|N|N|N|2457114|2457203|2456753|2457028|N|N|N|N|N| +2457119|AAAAAAAAPBOHFCAA|2015-04-06|1383|6014|462|2015|1|4|6|2|2015|462|6014|Monday|2015Q2|N|N|N|2457114|2457203|2456754|2457029|N|N|N|N|N| +2457120|AAAAAAAAACOHFCAA|2015-04-07|1383|6015|462|2015|2|4|7|2|2015|462|6015|Tuesday|2015Q2|N|N|N|2457114|2457203|2456755|2457030|N|N|N|N|N| +2457121|AAAAAAAABCOHFCAA|2015-04-08|1383|6015|462|2015|3|4|8|2|2015|462|6015|Wednesday|2015Q2|N|N|N|2457114|2457203|2456756|2457031|N|N|N|N|N| +2457122|AAAAAAAACCOHFCAA|2015-04-09|1383|6015|462|2015|4|4|9|2|2015|462|6015|Thursday|2015Q2|N|N|N|2457114|2457203|2456757|2457032|N|N|N|N|N| +2457123|AAAAAAAADCOHFCAA|2015-04-10|1383|6015|462|2015|5|4|10|2|2015|462|6015|Friday|2015Q2|N|Y|N|2457114|2457203|2456758|2457033|N|N|N|N|N| +2457124|AAAAAAAAECOHFCAA|2015-04-11|1383|6015|462|2015|6|4|11|2|2015|462|6015|Saturday|2015Q2|N|Y|N|2457114|2457203|2456759|2457034|N|N|N|N|N| +2457125|AAAAAAAAFCOHFCAA|2015-04-12|1383|6015|462|2015|0|4|12|2|2015|462|6015|Sunday|2015Q2|N|N|N|2457114|2457203|2456760|2457035|N|N|N|N|N| +2457126|AAAAAAAAGCOHFCAA|2015-04-13|1383|6015|462|2015|1|4|13|2|2015|462|6015|Monday|2015Q2|N|N|N|2457114|2457203|2456761|2457036|N|N|N|N|N| +2457127|AAAAAAAAHCOHFCAA|2015-04-14|1383|6016|462|2015|2|4|14|2|2015|462|6016|Tuesday|2015Q2|N|N|N|2457114|2457203|2456762|2457037|N|N|N|N|N| +2457128|AAAAAAAAICOHFCAA|2015-04-15|1383|6016|462|2015|3|4|15|2|2015|462|6016|Wednesday|2015Q2|N|N|N|2457114|2457203|2456763|2457038|N|N|N|N|N| +2457129|AAAAAAAAJCOHFCAA|2015-04-16|1383|6016|462|2015|4|4|16|2|2015|462|6016|Thursday|2015Q2|N|N|N|2457114|2457203|2456764|2457039|N|N|N|N|N| +2457130|AAAAAAAAKCOHFCAA|2015-04-17|1383|6016|462|2015|5|4|17|2|2015|462|6016|Friday|2015Q2|N|Y|N|2457114|2457203|2456765|2457040|N|N|N|N|N| +2457131|AAAAAAAALCOHFCAA|2015-04-18|1383|6016|462|2015|6|4|18|2|2015|462|6016|Saturday|2015Q2|N|Y|N|2457114|2457203|2456766|2457041|N|N|N|N|N| +2457132|AAAAAAAAMCOHFCAA|2015-04-19|1383|6016|462|2015|0|4|19|2|2015|462|6016|Sunday|2015Q2|N|N|N|2457114|2457203|2456767|2457042|N|N|N|N|N| +2457133|AAAAAAAANCOHFCAA|2015-04-20|1383|6016|462|2015|1|4|20|2|2015|462|6016|Monday|2015Q2|N|N|N|2457114|2457203|2456768|2457043|N|N|N|N|N| +2457134|AAAAAAAAOCOHFCAA|2015-04-21|1383|6017|462|2015|2|4|21|2|2015|462|6017|Tuesday|2015Q2|N|N|N|2457114|2457203|2456769|2457044|N|N|N|N|N| +2457135|AAAAAAAAPCOHFCAA|2015-04-22|1383|6017|462|2015|3|4|22|2|2015|462|6017|Wednesday|2015Q2|N|N|N|2457114|2457203|2456770|2457045|N|N|N|N|N| +2457136|AAAAAAAAADOHFCAA|2015-04-23|1383|6017|462|2015|4|4|23|2|2015|462|6017|Thursday|2015Q2|N|N|N|2457114|2457203|2456771|2457046|N|N|N|N|N| +2457137|AAAAAAAABDOHFCAA|2015-04-24|1383|6017|462|2015|5|4|24|2|2015|462|6017|Friday|2015Q2|N|Y|N|2457114|2457203|2456772|2457047|N|N|N|N|N| +2457138|AAAAAAAACDOHFCAA|2015-04-25|1383|6017|462|2015|6|4|25|2|2015|462|6017|Saturday|2015Q2|N|Y|N|2457114|2457203|2456773|2457048|N|N|N|N|N| +2457139|AAAAAAAADDOHFCAA|2015-04-26|1383|6017|462|2015|0|4|26|2|2015|462|6017|Sunday|2015Q2|N|N|N|2457114|2457203|2456774|2457049|N|N|N|N|N| +2457140|AAAAAAAAEDOHFCAA|2015-04-27|1383|6017|462|2015|1|4|27|2|2015|462|6017|Monday|2015Q2|N|N|N|2457114|2457203|2456775|2457050|N|N|N|N|N| +2457141|AAAAAAAAFDOHFCAA|2015-04-28|1383|6018|462|2015|2|4|28|2|2015|462|6018|Tuesday|2015Q2|N|N|N|2457114|2457203|2456776|2457051|N|N|N|N|N| +2457142|AAAAAAAAGDOHFCAA|2015-04-29|1383|6018|462|2015|3|4|29|2|2015|462|6018|Wednesday|2015Q2|N|N|N|2457114|2457203|2456777|2457052|N|N|N|N|N| +2457143|AAAAAAAAHDOHFCAA|2015-04-30|1383|6018|462|2015|4|4|30|2|2015|462|6018|Thursday|2015Q2|N|N|N|2457114|2457203|2456778|2457053|N|N|N|N|N| +2457144|AAAAAAAAIDOHFCAA|2015-05-01|1384|6018|462|2015|5|5|1|2|2015|462|6018|Friday|2015Q2|N|Y|N|2457144|2457263|2456779|2457054|N|N|N|N|N| +2457145|AAAAAAAAJDOHFCAA|2015-05-02|1384|6018|462|2015|6|5|2|2|2015|462|6018|Saturday|2015Q2|N|Y|N|2457144|2457263|2456780|2457055|N|N|N|N|N| +2457146|AAAAAAAAKDOHFCAA|2015-05-03|1384|6018|462|2015|0|5|3|2|2015|462|6018|Sunday|2015Q2|N|N|N|2457144|2457263|2456781|2457056|N|N|N|N|N| +2457147|AAAAAAAALDOHFCAA|2015-05-04|1384|6018|462|2015|1|5|4|2|2015|462|6018|Monday|2015Q2|N|N|N|2457144|2457263|2456782|2457057|N|N|N|N|N| +2457148|AAAAAAAAMDOHFCAA|2015-05-05|1384|6019|462|2015|2|5|5|2|2015|462|6019|Tuesday|2015Q2|N|N|N|2457144|2457263|2456783|2457058|N|N|N|N|N| +2457149|AAAAAAAANDOHFCAA|2015-05-06|1384|6019|462|2015|3|5|6|2|2015|462|6019|Wednesday|2015Q2|N|N|N|2457144|2457263|2456784|2457059|N|N|N|N|N| +2457150|AAAAAAAAODOHFCAA|2015-05-07|1384|6019|462|2015|4|5|7|2|2015|462|6019|Thursday|2015Q2|N|N|N|2457144|2457263|2456785|2457060|N|N|N|N|N| +2457151|AAAAAAAAPDOHFCAA|2015-05-08|1384|6019|462|2015|5|5|8|2|2015|462|6019|Friday|2015Q2|N|Y|N|2457144|2457263|2456786|2457061|N|N|N|N|N| +2457152|AAAAAAAAAEOHFCAA|2015-05-09|1384|6019|462|2015|6|5|9|2|2015|462|6019|Saturday|2015Q2|N|Y|N|2457144|2457263|2456787|2457062|N|N|N|N|N| +2457153|AAAAAAAABEOHFCAA|2015-05-10|1384|6019|462|2015|0|5|10|2|2015|462|6019|Sunday|2015Q2|N|N|N|2457144|2457263|2456788|2457063|N|N|N|N|N| +2457154|AAAAAAAACEOHFCAA|2015-05-11|1384|6019|462|2015|1|5|11|2|2015|462|6019|Monday|2015Q2|N|N|N|2457144|2457263|2456789|2457064|N|N|N|N|N| +2457155|AAAAAAAADEOHFCAA|2015-05-12|1384|6020|462|2015|2|5|12|2|2015|462|6020|Tuesday|2015Q2|N|N|N|2457144|2457263|2456790|2457065|N|N|N|N|N| +2457156|AAAAAAAAEEOHFCAA|2015-05-13|1384|6020|462|2015|3|5|13|2|2015|462|6020|Wednesday|2015Q2|N|N|N|2457144|2457263|2456791|2457066|N|N|N|N|N| +2457157|AAAAAAAAFEOHFCAA|2015-05-14|1384|6020|462|2015|4|5|14|2|2015|462|6020|Thursday|2015Q2|N|N|N|2457144|2457263|2456792|2457067|N|N|N|N|N| +2457158|AAAAAAAAGEOHFCAA|2015-05-15|1384|6020|462|2015|5|5|15|2|2015|462|6020|Friday|2015Q2|N|Y|N|2457144|2457263|2456793|2457068|N|N|N|N|N| +2457159|AAAAAAAAHEOHFCAA|2015-05-16|1384|6020|462|2015|6|5|16|2|2015|462|6020|Saturday|2015Q2|N|Y|N|2457144|2457263|2456794|2457069|N|N|N|N|N| +2457160|AAAAAAAAIEOHFCAA|2015-05-17|1384|6020|462|2015|0|5|17|2|2015|462|6020|Sunday|2015Q2|N|N|N|2457144|2457263|2456795|2457070|N|N|N|N|N| +2457161|AAAAAAAAJEOHFCAA|2015-05-18|1384|6020|462|2015|1|5|18|2|2015|462|6020|Monday|2015Q2|N|N|N|2457144|2457263|2456796|2457071|N|N|N|N|N| +2457162|AAAAAAAAKEOHFCAA|2015-05-19|1384|6021|462|2015|2|5|19|2|2015|462|6021|Tuesday|2015Q2|N|N|N|2457144|2457263|2456797|2457072|N|N|N|N|N| +2457163|AAAAAAAALEOHFCAA|2015-05-20|1384|6021|462|2015|3|5|20|2|2015|462|6021|Wednesday|2015Q2|N|N|N|2457144|2457263|2456798|2457073|N|N|N|N|N| +2457164|AAAAAAAAMEOHFCAA|2015-05-21|1384|6021|462|2015|4|5|21|2|2015|462|6021|Thursday|2015Q2|N|N|N|2457144|2457263|2456799|2457074|N|N|N|N|N| +2457165|AAAAAAAANEOHFCAA|2015-05-22|1384|6021|462|2015|5|5|22|2|2015|462|6021|Friday|2015Q2|N|Y|N|2457144|2457263|2456800|2457075|N|N|N|N|N| +2457166|AAAAAAAAOEOHFCAA|2015-05-23|1384|6021|462|2015|6|5|23|2|2015|462|6021|Saturday|2015Q2|N|Y|N|2457144|2457263|2456801|2457076|N|N|N|N|N| +2457167|AAAAAAAAPEOHFCAA|2015-05-24|1384|6021|462|2015|0|5|24|2|2015|462|6021|Sunday|2015Q2|N|N|N|2457144|2457263|2456802|2457077|N|N|N|N|N| +2457168|AAAAAAAAAFOHFCAA|2015-05-25|1384|6021|462|2015|1|5|25|2|2015|462|6021|Monday|2015Q2|N|N|N|2457144|2457263|2456803|2457078|N|N|N|N|N| +2457169|AAAAAAAABFOHFCAA|2015-05-26|1384|6022|462|2015|2|5|26|2|2015|462|6022|Tuesday|2015Q2|N|N|N|2457144|2457263|2456804|2457079|N|N|N|N|N| +2457170|AAAAAAAACFOHFCAA|2015-05-27|1384|6022|462|2015|3|5|27|2|2015|462|6022|Wednesday|2015Q2|N|N|N|2457144|2457263|2456805|2457080|N|N|N|N|N| +2457171|AAAAAAAADFOHFCAA|2015-05-28|1384|6022|462|2015|4|5|28|2|2015|462|6022|Thursday|2015Q2|N|N|N|2457144|2457263|2456806|2457081|N|N|N|N|N| +2457172|AAAAAAAAEFOHFCAA|2015-05-29|1384|6022|462|2015|5|5|29|2|2015|462|6022|Friday|2015Q2|N|Y|N|2457144|2457263|2456807|2457082|N|N|N|N|N| +2457173|AAAAAAAAFFOHFCAA|2015-05-30|1384|6022|462|2015|6|5|30|2|2015|462|6022|Saturday|2015Q2|N|Y|N|2457144|2457263|2456808|2457083|N|N|N|N|N| +2457174|AAAAAAAAGFOHFCAA|2015-05-31|1384|6022|462|2015|0|5|31|2|2015|462|6022|Sunday|2015Q2|N|N|N|2457144|2457263|2456809|2457084|N|N|N|N|N| +2457175|AAAAAAAAHFOHFCAA|2015-06-01|1385|6022|463|2015|1|6|1|2|2015|463|6022|Monday|2015Q2|N|N|N|2457175|2457325|2456810|2457085|N|N|N|N|N| +2457176|AAAAAAAAIFOHFCAA|2015-06-02|1385|6023|463|2015|2|6|2|2|2015|463|6023|Tuesday|2015Q2|N|N|N|2457175|2457325|2456811|2457086|N|N|N|N|N| +2457177|AAAAAAAAJFOHFCAA|2015-06-03|1385|6023|463|2015|3|6|3|2|2015|463|6023|Wednesday|2015Q2|N|N|N|2457175|2457325|2456812|2457087|N|N|N|N|N| +2457178|AAAAAAAAKFOHFCAA|2015-06-04|1385|6023|463|2015|4|6|4|2|2015|463|6023|Thursday|2015Q2|N|N|N|2457175|2457325|2456813|2457088|N|N|N|N|N| +2457179|AAAAAAAALFOHFCAA|2015-06-05|1385|6023|463|2015|5|6|5|2|2015|463|6023|Friday|2015Q2|N|Y|N|2457175|2457325|2456814|2457089|N|N|N|N|N| +2457180|AAAAAAAAMFOHFCAA|2015-06-06|1385|6023|463|2015|6|6|6|2|2015|463|6023|Saturday|2015Q2|N|Y|N|2457175|2457325|2456815|2457090|N|N|N|N|N| +2457181|AAAAAAAANFOHFCAA|2015-06-07|1385|6023|463|2015|0|6|7|2|2015|463|6023|Sunday|2015Q2|N|N|N|2457175|2457325|2456816|2457091|N|N|N|N|N| +2457182|AAAAAAAAOFOHFCAA|2015-06-08|1385|6023|463|2015|1|6|8|2|2015|463|6023|Monday|2015Q2|N|N|N|2457175|2457325|2456817|2457092|N|N|N|N|N| +2457183|AAAAAAAAPFOHFCAA|2015-06-09|1385|6024|463|2015|2|6|9|2|2015|463|6024|Tuesday|2015Q2|N|N|N|2457175|2457325|2456818|2457093|N|N|N|N|N| +2457184|AAAAAAAAAGOHFCAA|2015-06-10|1385|6024|463|2015|3|6|10|2|2015|463|6024|Wednesday|2015Q2|N|N|N|2457175|2457325|2456819|2457094|N|N|N|N|N| +2457185|AAAAAAAABGOHFCAA|2015-06-11|1385|6024|463|2015|4|6|11|2|2015|463|6024|Thursday|2015Q2|N|N|N|2457175|2457325|2456820|2457095|N|N|N|N|N| +2457186|AAAAAAAACGOHFCAA|2015-06-12|1385|6024|463|2015|5|6|12|2|2015|463|6024|Friday|2015Q2|N|Y|N|2457175|2457325|2456821|2457096|N|N|N|N|N| +2457187|AAAAAAAADGOHFCAA|2015-06-13|1385|6024|463|2015|6|6|13|2|2015|463|6024|Saturday|2015Q2|N|Y|N|2457175|2457325|2456822|2457097|N|N|N|N|N| +2457188|AAAAAAAAEGOHFCAA|2015-06-14|1385|6024|463|2015|0|6|14|2|2015|463|6024|Sunday|2015Q2|N|N|N|2457175|2457325|2456823|2457098|N|N|N|N|N| +2457189|AAAAAAAAFGOHFCAA|2015-06-15|1385|6024|463|2015|1|6|15|2|2015|463|6024|Monday|2015Q2|N|N|N|2457175|2457325|2456824|2457099|N|N|N|N|N| +2457190|AAAAAAAAGGOHFCAA|2015-06-16|1385|6025|463|2015|2|6|16|2|2015|463|6025|Tuesday|2015Q2|N|N|N|2457175|2457325|2456825|2457100|N|N|N|N|N| +2457191|AAAAAAAAHGOHFCAA|2015-06-17|1385|6025|463|2015|3|6|17|2|2015|463|6025|Wednesday|2015Q2|N|N|N|2457175|2457325|2456826|2457101|N|N|N|N|N| +2457192|AAAAAAAAIGOHFCAA|2015-06-18|1385|6025|463|2015|4|6|18|2|2015|463|6025|Thursday|2015Q2|N|N|N|2457175|2457325|2456827|2457102|N|N|N|N|N| +2457193|AAAAAAAAJGOHFCAA|2015-06-19|1385|6025|463|2015|5|6|19|2|2015|463|6025|Friday|2015Q2|N|Y|N|2457175|2457325|2456828|2457103|N|N|N|N|N| +2457194|AAAAAAAAKGOHFCAA|2015-06-20|1385|6025|463|2015|6|6|20|2|2015|463|6025|Saturday|2015Q2|N|Y|N|2457175|2457325|2456829|2457104|N|N|N|N|N| +2457195|AAAAAAAALGOHFCAA|2015-06-21|1385|6025|463|2015|0|6|21|2|2015|463|6025|Sunday|2015Q2|N|N|N|2457175|2457325|2456830|2457105|N|N|N|N|N| +2457196|AAAAAAAAMGOHFCAA|2015-06-22|1385|6025|463|2015|1|6|22|2|2015|463|6025|Monday|2015Q2|N|N|N|2457175|2457325|2456831|2457106|N|N|N|N|N| +2457197|AAAAAAAANGOHFCAA|2015-06-23|1385|6026|463|2015|2|6|23|2|2015|463|6026|Tuesday|2015Q2|N|N|N|2457175|2457325|2456832|2457107|N|N|N|N|N| +2457198|AAAAAAAAOGOHFCAA|2015-06-24|1385|6026|463|2015|3|6|24|2|2015|463|6026|Wednesday|2015Q2|N|N|N|2457175|2457325|2456833|2457108|N|N|N|N|N| +2457199|AAAAAAAAPGOHFCAA|2015-06-25|1385|6026|463|2015|4|6|25|2|2015|463|6026|Thursday|2015Q2|N|N|N|2457175|2457325|2456834|2457109|N|N|N|N|N| +2457200|AAAAAAAAAHOHFCAA|2015-06-26|1385|6026|463|2015|5|6|26|2|2015|463|6026|Friday|2015Q2|N|Y|N|2457175|2457325|2456835|2457110|N|N|N|N|N| +2457201|AAAAAAAABHOHFCAA|2015-06-27|1385|6026|463|2015|6|6|27|2|2015|463|6026|Saturday|2015Q2|N|Y|N|2457175|2457325|2456836|2457111|N|N|N|N|N| +2457202|AAAAAAAACHOHFCAA|2015-06-28|1385|6026|463|2015|0|6|28|2|2015|463|6026|Sunday|2015Q2|N|N|N|2457175|2457325|2456837|2457112|N|N|N|N|N| +2457203|AAAAAAAADHOHFCAA|2015-06-29|1385|6026|463|2015|1|6|29|2|2015|463|6026|Monday|2015Q2|N|N|N|2457175|2457325|2456838|2457113|N|N|N|N|N| +2457204|AAAAAAAAEHOHFCAA|2015-06-30|1385|6027|463|2015|2|6|30|2|2015|463|6027|Tuesday|2015Q2|N|N|N|2457175|2457325|2456839|2457114|N|N|N|N|N| +2457205|AAAAAAAAFHOHFCAA|2015-07-01|1386|6027|463|2015|3|7|1|2|2015|463|6027|Wednesday|2015Q2|N|N|N|2457205|2457385|2456840|2457114|N|N|N|N|N| +2457206|AAAAAAAAGHOHFCAA|2015-07-02|1386|6027|463|2015|4|7|2|3|2015|463|6027|Thursday|2015Q3|N|N|N|2457205|2457385|2456841|2457115|N|N|N|N|N| +2457207|AAAAAAAAHHOHFCAA|2015-07-03|1386|6027|463|2015|5|7|3|3|2015|463|6027|Friday|2015Q3|N|Y|N|2457205|2457385|2456842|2457116|N|N|N|N|N| +2457208|AAAAAAAAIHOHFCAA|2015-07-04|1386|6027|463|2015|6|7|4|3|2015|463|6027|Saturday|2015Q3|N|Y|N|2457205|2457385|2456843|2457117|N|N|N|N|N| +2457209|AAAAAAAAJHOHFCAA|2015-07-05|1386|6027|463|2015|0|7|5|3|2015|463|6027|Sunday|2015Q3|Y|N|N|2457205|2457385|2456844|2457118|N|N|N|N|N| +2457210|AAAAAAAAKHOHFCAA|2015-07-06|1386|6027|463|2015|1|7|6|3|2015|463|6027|Monday|2015Q3|N|N|Y|2457205|2457385|2456845|2457119|N|N|N|N|N| +2457211|AAAAAAAALHOHFCAA|2015-07-07|1386|6028|463|2015|2|7|7|3|2015|463|6028|Tuesday|2015Q3|N|N|N|2457205|2457385|2456846|2457120|N|N|N|N|N| +2457212|AAAAAAAAMHOHFCAA|2015-07-08|1386|6028|463|2015|3|7|8|3|2015|463|6028|Wednesday|2015Q3|N|N|N|2457205|2457385|2456847|2457121|N|N|N|N|N| +2457213|AAAAAAAANHOHFCAA|2015-07-09|1386|6028|463|2015|4|7|9|3|2015|463|6028|Thursday|2015Q3|N|N|N|2457205|2457385|2456848|2457122|N|N|N|N|N| +2457214|AAAAAAAAOHOHFCAA|2015-07-10|1386|6028|463|2015|5|7|10|3|2015|463|6028|Friday|2015Q3|N|Y|N|2457205|2457385|2456849|2457123|N|N|N|N|N| +2457215|AAAAAAAAPHOHFCAA|2015-07-11|1386|6028|463|2015|6|7|11|3|2015|463|6028|Saturday|2015Q3|N|Y|N|2457205|2457385|2456850|2457124|N|N|N|N|N| +2457216|AAAAAAAAAIOHFCAA|2015-07-12|1386|6028|463|2015|0|7|12|3|2015|463|6028|Sunday|2015Q3|N|N|N|2457205|2457385|2456851|2457125|N|N|N|N|N| +2457217|AAAAAAAABIOHFCAA|2015-07-13|1386|6028|463|2015|1|7|13|3|2015|463|6028|Monday|2015Q3|N|N|N|2457205|2457385|2456852|2457126|N|N|N|N|N| +2457218|AAAAAAAACIOHFCAA|2015-07-14|1386|6029|463|2015|2|7|14|3|2015|463|6029|Tuesday|2015Q3|N|N|N|2457205|2457385|2456853|2457127|N|N|N|N|N| +2457219|AAAAAAAADIOHFCAA|2015-07-15|1386|6029|463|2015|3|7|15|3|2015|463|6029|Wednesday|2015Q3|N|N|N|2457205|2457385|2456854|2457128|N|N|N|N|N| +2457220|AAAAAAAAEIOHFCAA|2015-07-16|1386|6029|463|2015|4|7|16|3|2015|463|6029|Thursday|2015Q3|N|N|N|2457205|2457385|2456855|2457129|N|N|N|N|N| +2457221|AAAAAAAAFIOHFCAA|2015-07-17|1386|6029|463|2015|5|7|17|3|2015|463|6029|Friday|2015Q3|N|Y|N|2457205|2457385|2456856|2457130|N|N|N|N|N| +2457222|AAAAAAAAGIOHFCAA|2015-07-18|1386|6029|463|2015|6|7|18|3|2015|463|6029|Saturday|2015Q3|N|Y|N|2457205|2457385|2456857|2457131|N|N|N|N|N| +2457223|AAAAAAAAHIOHFCAA|2015-07-19|1386|6029|463|2015|0|7|19|3|2015|463|6029|Sunday|2015Q3|N|N|N|2457205|2457385|2456858|2457132|N|N|N|N|N| +2457224|AAAAAAAAIIOHFCAA|2015-07-20|1386|6029|463|2015|1|7|20|3|2015|463|6029|Monday|2015Q3|N|N|N|2457205|2457385|2456859|2457133|N|N|N|N|N| +2457225|AAAAAAAAJIOHFCAA|2015-07-21|1386|6030|463|2015|2|7|21|3|2015|463|6030|Tuesday|2015Q3|N|N|N|2457205|2457385|2456860|2457134|N|N|N|N|N| +2457226|AAAAAAAAKIOHFCAA|2015-07-22|1386|6030|463|2015|3|7|22|3|2015|463|6030|Wednesday|2015Q3|N|N|N|2457205|2457385|2456861|2457135|N|N|N|N|N| +2457227|AAAAAAAALIOHFCAA|2015-07-23|1386|6030|463|2015|4|7|23|3|2015|463|6030|Thursday|2015Q3|N|N|N|2457205|2457385|2456862|2457136|N|N|N|N|N| +2457228|AAAAAAAAMIOHFCAA|2015-07-24|1386|6030|463|2015|5|7|24|3|2015|463|6030|Friday|2015Q3|N|Y|N|2457205|2457385|2456863|2457137|N|N|N|N|N| +2457229|AAAAAAAANIOHFCAA|2015-07-25|1386|6030|463|2015|6|7|25|3|2015|463|6030|Saturday|2015Q3|N|Y|N|2457205|2457385|2456864|2457138|N|N|N|N|N| +2457230|AAAAAAAAOIOHFCAA|2015-07-26|1386|6030|463|2015|0|7|26|3|2015|463|6030|Sunday|2015Q3|N|N|N|2457205|2457385|2456865|2457139|N|N|N|N|N| +2457231|AAAAAAAAPIOHFCAA|2015-07-27|1386|6030|463|2015|1|7|27|3|2015|463|6030|Monday|2015Q3|N|N|N|2457205|2457385|2456866|2457140|N|N|N|N|N| +2457232|AAAAAAAAAJOHFCAA|2015-07-28|1386|6031|463|2015|2|7|28|3|2015|463|6031|Tuesday|2015Q3|N|N|N|2457205|2457385|2456867|2457141|N|N|N|N|N| +2457233|AAAAAAAABJOHFCAA|2015-07-29|1386|6031|463|2015|3|7|29|3|2015|463|6031|Wednesday|2015Q3|N|N|N|2457205|2457385|2456868|2457142|N|N|N|N|N| +2457234|AAAAAAAACJOHFCAA|2015-07-30|1386|6031|463|2015|4|7|30|3|2015|463|6031|Thursday|2015Q3|N|N|N|2457205|2457385|2456869|2457143|N|N|N|N|N| +2457235|AAAAAAAADJOHFCAA|2015-07-31|1386|6031|463|2015|5|7|31|3|2015|463|6031|Friday|2015Q3|N|Y|N|2457205|2457385|2456870|2457144|N|N|N|N|N| +2457236|AAAAAAAAEJOHFCAA|2015-08-01|1387|6031|463|2015|6|8|1|3|2015|463|6031|Saturday|2015Q3|N|Y|N|2457236|2457447|2456871|2457145|N|N|N|N|N| +2457237|AAAAAAAAFJOHFCAA|2015-08-02|1387|6031|463|2015|0|8|2|3|2015|463|6031|Sunday|2015Q3|N|N|N|2457236|2457447|2456872|2457146|N|N|N|N|N| +2457238|AAAAAAAAGJOHFCAA|2015-08-03|1387|6031|463|2015|1|8|3|3|2015|463|6031|Monday|2015Q3|N|N|N|2457236|2457447|2456873|2457147|N|N|N|N|N| +2457239|AAAAAAAAHJOHFCAA|2015-08-04|1387|6032|463|2015|2|8|4|3|2015|463|6032|Tuesday|2015Q3|N|N|N|2457236|2457447|2456874|2457148|N|N|N|N|N| +2457240|AAAAAAAAIJOHFCAA|2015-08-05|1387|6032|463|2015|3|8|5|3|2015|463|6032|Wednesday|2015Q3|N|N|N|2457236|2457447|2456875|2457149|N|N|N|N|N| +2457241|AAAAAAAAJJOHFCAA|2015-08-06|1387|6032|463|2015|4|8|6|3|2015|463|6032|Thursday|2015Q3|N|N|N|2457236|2457447|2456876|2457150|N|N|N|N|N| +2457242|AAAAAAAAKJOHFCAA|2015-08-07|1387|6032|463|2015|5|8|7|3|2015|463|6032|Friday|2015Q3|N|Y|N|2457236|2457447|2456877|2457151|N|N|N|N|N| +2457243|AAAAAAAALJOHFCAA|2015-08-08|1387|6032|463|2015|6|8|8|3|2015|463|6032|Saturday|2015Q3|N|Y|N|2457236|2457447|2456878|2457152|N|N|N|N|N| +2457244|AAAAAAAAMJOHFCAA|2015-08-09|1387|6032|463|2015|0|8|9|3|2015|463|6032|Sunday|2015Q3|N|N|N|2457236|2457447|2456879|2457153|N|N|N|N|N| +2457245|AAAAAAAANJOHFCAA|2015-08-10|1387|6032|463|2015|1|8|10|3|2015|463|6032|Monday|2015Q3|N|N|N|2457236|2457447|2456880|2457154|N|N|N|N|N| +2457246|AAAAAAAAOJOHFCAA|2015-08-11|1387|6033|463|2015|2|8|11|3|2015|463|6033|Tuesday|2015Q3|N|N|N|2457236|2457447|2456881|2457155|N|N|N|N|N| +2457247|AAAAAAAAPJOHFCAA|2015-08-12|1387|6033|463|2015|3|8|12|3|2015|463|6033|Wednesday|2015Q3|N|N|N|2457236|2457447|2456882|2457156|N|N|N|N|N| +2457248|AAAAAAAAAKOHFCAA|2015-08-13|1387|6033|463|2015|4|8|13|3|2015|463|6033|Thursday|2015Q3|N|N|N|2457236|2457447|2456883|2457157|N|N|N|N|N| +2457249|AAAAAAAABKOHFCAA|2015-08-14|1387|6033|463|2015|5|8|14|3|2015|463|6033|Friday|2015Q3|N|Y|N|2457236|2457447|2456884|2457158|N|N|N|N|N| +2457250|AAAAAAAACKOHFCAA|2015-08-15|1387|6033|463|2015|6|8|15|3|2015|463|6033|Saturday|2015Q3|N|Y|N|2457236|2457447|2456885|2457159|N|N|N|N|N| +2457251|AAAAAAAADKOHFCAA|2015-08-16|1387|6033|463|2015|0|8|16|3|2015|463|6033|Sunday|2015Q3|N|N|N|2457236|2457447|2456886|2457160|N|N|N|N|N| +2457252|AAAAAAAAEKOHFCAA|2015-08-17|1387|6033|463|2015|1|8|17|3|2015|463|6033|Monday|2015Q3|N|N|N|2457236|2457447|2456887|2457161|N|N|N|N|N| +2457253|AAAAAAAAFKOHFCAA|2015-08-18|1387|6034|463|2015|2|8|18|3|2015|463|6034|Tuesday|2015Q3|N|N|N|2457236|2457447|2456888|2457162|N|N|N|N|N| +2457254|AAAAAAAAGKOHFCAA|2015-08-19|1387|6034|463|2015|3|8|19|3|2015|463|6034|Wednesday|2015Q3|N|N|N|2457236|2457447|2456889|2457163|N|N|N|N|N| +2457255|AAAAAAAAHKOHFCAA|2015-08-20|1387|6034|463|2015|4|8|20|3|2015|463|6034|Thursday|2015Q3|N|N|N|2457236|2457447|2456890|2457164|N|N|N|N|N| +2457256|AAAAAAAAIKOHFCAA|2015-08-21|1387|6034|463|2015|5|8|21|3|2015|463|6034|Friday|2015Q3|N|Y|N|2457236|2457447|2456891|2457165|N|N|N|N|N| +2457257|AAAAAAAAJKOHFCAA|2015-08-22|1387|6034|463|2015|6|8|22|3|2015|463|6034|Saturday|2015Q3|N|Y|N|2457236|2457447|2456892|2457166|N|N|N|N|N| +2457258|AAAAAAAAKKOHFCAA|2015-08-23|1387|6034|463|2015|0|8|23|3|2015|463|6034|Sunday|2015Q3|N|N|N|2457236|2457447|2456893|2457167|N|N|N|N|N| +2457259|AAAAAAAALKOHFCAA|2015-08-24|1387|6034|463|2015|1|8|24|3|2015|463|6034|Monday|2015Q3|N|N|N|2457236|2457447|2456894|2457168|N|N|N|N|N| +2457260|AAAAAAAAMKOHFCAA|2015-08-25|1387|6035|463|2015|2|8|25|3|2015|463|6035|Tuesday|2015Q3|N|N|N|2457236|2457447|2456895|2457169|N|N|N|N|N| +2457261|AAAAAAAANKOHFCAA|2015-08-26|1387|6035|463|2015|3|8|26|3|2015|463|6035|Wednesday|2015Q3|N|N|N|2457236|2457447|2456896|2457170|N|N|N|N|N| +2457262|AAAAAAAAOKOHFCAA|2015-08-27|1387|6035|463|2015|4|8|27|3|2015|463|6035|Thursday|2015Q3|N|N|N|2457236|2457447|2456897|2457171|N|N|N|N|N| +2457263|AAAAAAAAPKOHFCAA|2015-08-28|1387|6035|463|2015|5|8|28|3|2015|463|6035|Friday|2015Q3|N|Y|N|2457236|2457447|2456898|2457172|N|N|N|N|N| +2457264|AAAAAAAAALOHFCAA|2015-08-29|1387|6035|463|2015|6|8|29|3|2015|463|6035|Saturday|2015Q3|N|Y|N|2457236|2457447|2456899|2457173|N|N|N|N|N| +2457265|AAAAAAAABLOHFCAA|2015-08-30|1387|6035|463|2015|0|8|30|3|2015|463|6035|Sunday|2015Q3|N|N|N|2457236|2457447|2456900|2457174|N|N|N|N|N| +2457266|AAAAAAAACLOHFCAA|2015-08-31|1387|6035|463|2015|1|8|31|3|2015|463|6035|Monday|2015Q3|N|N|N|2457236|2457447|2456901|2457175|N|N|N|N|N| +2457267|AAAAAAAADLOHFCAA|2015-09-01|1388|6036|464|2015|2|9|1|3|2015|464|6036|Tuesday|2015Q3|N|N|N|2457267|2457509|2456902|2457176|N|N|N|N|N| +2457268|AAAAAAAAELOHFCAA|2015-09-02|1388|6036|464|2015|3|9|2|3|2015|464|6036|Wednesday|2015Q3|N|N|N|2457267|2457509|2456903|2457177|N|N|N|N|N| +2457269|AAAAAAAAFLOHFCAA|2015-09-03|1388|6036|464|2015|4|9|3|3|2015|464|6036|Thursday|2015Q3|N|N|N|2457267|2457509|2456904|2457178|N|N|N|N|N| +2457270|AAAAAAAAGLOHFCAA|2015-09-04|1388|6036|464|2015|5|9|4|3|2015|464|6036|Friday|2015Q3|N|Y|N|2457267|2457509|2456905|2457179|N|N|N|N|N| +2457271|AAAAAAAAHLOHFCAA|2015-09-05|1388|6036|464|2015|6|9|5|3|2015|464|6036|Saturday|2015Q3|N|Y|N|2457267|2457509|2456906|2457180|N|N|N|N|N| +2457272|AAAAAAAAILOHFCAA|2015-09-06|1388|6036|464|2015|0|9|6|3|2015|464|6036|Sunday|2015Q3|N|N|N|2457267|2457509|2456907|2457181|N|N|N|N|N| +2457273|AAAAAAAAJLOHFCAA|2015-09-07|1388|6036|464|2015|1|9|7|3|2015|464|6036|Monday|2015Q3|N|N|N|2457267|2457509|2456908|2457182|N|N|N|N|N| +2457274|AAAAAAAAKLOHFCAA|2015-09-08|1388|6037|464|2015|2|9|8|3|2015|464|6037|Tuesday|2015Q3|N|N|N|2457267|2457509|2456909|2457183|N|N|N|N|N| +2457275|AAAAAAAALLOHFCAA|2015-09-09|1388|6037|464|2015|3|9|9|3|2015|464|6037|Wednesday|2015Q3|N|N|N|2457267|2457509|2456910|2457184|N|N|N|N|N| +2457276|AAAAAAAAMLOHFCAA|2015-09-10|1388|6037|464|2015|4|9|10|3|2015|464|6037|Thursday|2015Q3|N|N|N|2457267|2457509|2456911|2457185|N|N|N|N|N| +2457277|AAAAAAAANLOHFCAA|2015-09-11|1388|6037|464|2015|5|9|11|3|2015|464|6037|Friday|2015Q3|N|Y|N|2457267|2457509|2456912|2457186|N|N|N|N|N| +2457278|AAAAAAAAOLOHFCAA|2015-09-12|1388|6037|464|2015|6|9|12|3|2015|464|6037|Saturday|2015Q3|N|Y|N|2457267|2457509|2456913|2457187|N|N|N|N|N| +2457279|AAAAAAAAPLOHFCAA|2015-09-13|1388|6037|464|2015|0|9|13|3|2015|464|6037|Sunday|2015Q3|N|N|N|2457267|2457509|2456914|2457188|N|N|N|N|N| +2457280|AAAAAAAAAMOHFCAA|2015-09-14|1388|6037|464|2015|1|9|14|3|2015|464|6037|Monday|2015Q3|N|N|N|2457267|2457509|2456915|2457189|N|N|N|N|N| +2457281|AAAAAAAABMOHFCAA|2015-09-15|1388|6038|464|2015|2|9|15|3|2015|464|6038|Tuesday|2015Q3|N|N|N|2457267|2457509|2456916|2457190|N|N|N|N|N| +2457282|AAAAAAAACMOHFCAA|2015-09-16|1388|6038|464|2015|3|9|16|3|2015|464|6038|Wednesday|2015Q3|N|N|N|2457267|2457509|2456917|2457191|N|N|N|N|N| +2457283|AAAAAAAADMOHFCAA|2015-09-17|1388|6038|464|2015|4|9|17|3|2015|464|6038|Thursday|2015Q3|N|N|N|2457267|2457509|2456918|2457192|N|N|N|N|N| +2457284|AAAAAAAAEMOHFCAA|2015-09-18|1388|6038|464|2015|5|9|18|3|2015|464|6038|Friday|2015Q3|N|Y|N|2457267|2457509|2456919|2457193|N|N|N|N|N| +2457285|AAAAAAAAFMOHFCAA|2015-09-19|1388|6038|464|2015|6|9|19|3|2015|464|6038|Saturday|2015Q3|N|Y|N|2457267|2457509|2456920|2457194|N|N|N|N|N| +2457286|AAAAAAAAGMOHFCAA|2015-09-20|1388|6038|464|2015|0|9|20|3|2015|464|6038|Sunday|2015Q3|N|N|N|2457267|2457509|2456921|2457195|N|N|N|N|N| +2457287|AAAAAAAAHMOHFCAA|2015-09-21|1388|6038|464|2015|1|9|21|3|2015|464|6038|Monday|2015Q3|N|N|N|2457267|2457509|2456922|2457196|N|N|N|N|N| +2457288|AAAAAAAAIMOHFCAA|2015-09-22|1388|6039|464|2015|2|9|22|3|2015|464|6039|Tuesday|2015Q3|N|N|N|2457267|2457509|2456923|2457197|N|N|N|N|N| +2457289|AAAAAAAAJMOHFCAA|2015-09-23|1388|6039|464|2015|3|9|23|3|2015|464|6039|Wednesday|2015Q3|N|N|N|2457267|2457509|2456924|2457198|N|N|N|N|N| +2457290|AAAAAAAAKMOHFCAA|2015-09-24|1388|6039|464|2015|4|9|24|3|2015|464|6039|Thursday|2015Q3|N|N|N|2457267|2457509|2456925|2457199|N|N|N|N|N| +2457291|AAAAAAAALMOHFCAA|2015-09-25|1388|6039|464|2015|5|9|25|3|2015|464|6039|Friday|2015Q3|N|Y|N|2457267|2457509|2456926|2457200|N|N|N|N|N| +2457292|AAAAAAAAMMOHFCAA|2015-09-26|1388|6039|464|2015|6|9|26|3|2015|464|6039|Saturday|2015Q3|N|Y|N|2457267|2457509|2456927|2457201|N|N|N|N|N| +2457293|AAAAAAAANMOHFCAA|2015-09-27|1388|6039|464|2015|0|9|27|3|2015|464|6039|Sunday|2015Q3|N|N|N|2457267|2457509|2456928|2457202|N|N|N|N|N| +2457294|AAAAAAAAOMOHFCAA|2015-09-28|1388|6039|464|2015|1|9|28|3|2015|464|6039|Monday|2015Q3|N|N|N|2457267|2457509|2456929|2457203|N|N|N|N|N| +2457295|AAAAAAAAPMOHFCAA|2015-09-29|1388|6040|464|2015|2|9|29|3|2015|464|6040|Tuesday|2015Q3|N|N|N|2457267|2457509|2456930|2457204|N|N|N|N|N| +2457296|AAAAAAAAANOHFCAA|2015-09-30|1388|6040|464|2015|3|9|30|3|2015|464|6040|Wednesday|2015Q3|N|N|N|2457267|2457509|2456931|2457205|N|N|N|N|N| +2457297|AAAAAAAABNOHFCAA|2015-10-01|1389|6040|464|2015|4|10|1|3|2015|464|6040|Thursday|2015Q3|N|N|N|2457297|2457569|2456932|2457205|N|N|N|N|N| +2457298|AAAAAAAACNOHFCAA|2015-10-02|1389|6040|464|2015|5|10|2|4|2015|464|6040|Friday|2015Q4|N|Y|N|2457297|2457569|2456933|2457206|N|N|N|N|N| +2457299|AAAAAAAADNOHFCAA|2015-10-03|1389|6040|464|2015|6|10|3|4|2015|464|6040|Saturday|2015Q4|N|Y|N|2457297|2457569|2456934|2457207|N|N|N|N|N| +2457300|AAAAAAAAENOHFCAA|2015-10-04|1389|6040|464|2015|0|10|4|4|2015|464|6040|Sunday|2015Q4|N|N|N|2457297|2457569|2456935|2457208|N|N|N|N|N| +2457301|AAAAAAAAFNOHFCAA|2015-10-05|1389|6040|464|2015|1|10|5|4|2015|464|6040|Monday|2015Q4|N|N|N|2457297|2457569|2456936|2457209|N|N|N|N|N| +2457302|AAAAAAAAGNOHFCAA|2015-10-06|1389|6041|464|2015|2|10|6|4|2015|464|6041|Tuesday|2015Q4|N|N|N|2457297|2457569|2456937|2457210|N|N|N|N|N| +2457303|AAAAAAAAHNOHFCAA|2015-10-07|1389|6041|464|2015|3|10|7|4|2015|464|6041|Wednesday|2015Q4|N|N|N|2457297|2457569|2456938|2457211|N|N|N|N|N| +2457304|AAAAAAAAINOHFCAA|2015-10-08|1389|6041|464|2015|4|10|8|4|2015|464|6041|Thursday|2015Q4|N|N|N|2457297|2457569|2456939|2457212|N|N|N|N|N| +2457305|AAAAAAAAJNOHFCAA|2015-10-09|1389|6041|464|2015|5|10|9|4|2015|464|6041|Friday|2015Q4|N|Y|N|2457297|2457569|2456940|2457213|N|N|N|N|N| +2457306|AAAAAAAAKNOHFCAA|2015-10-10|1389|6041|464|2015|6|10|10|4|2015|464|6041|Saturday|2015Q4|N|Y|N|2457297|2457569|2456941|2457214|N|N|N|N|N| +2457307|AAAAAAAALNOHFCAA|2015-10-11|1389|6041|464|2015|0|10|11|4|2015|464|6041|Sunday|2015Q4|N|N|N|2457297|2457569|2456942|2457215|N|N|N|N|N| +2457308|AAAAAAAAMNOHFCAA|2015-10-12|1389|6041|464|2015|1|10|12|4|2015|464|6041|Monday|2015Q4|N|N|N|2457297|2457569|2456943|2457216|N|N|N|N|N| +2457309|AAAAAAAANNOHFCAA|2015-10-13|1389|6042|464|2015|2|10|13|4|2015|464|6042|Tuesday|2015Q4|N|N|N|2457297|2457569|2456944|2457217|N|N|N|N|N| +2457310|AAAAAAAAONOHFCAA|2015-10-14|1389|6042|464|2015|3|10|14|4|2015|464|6042|Wednesday|2015Q4|N|N|N|2457297|2457569|2456945|2457218|N|N|N|N|N| +2457311|AAAAAAAAPNOHFCAA|2015-10-15|1389|6042|464|2015|4|10|15|4|2015|464|6042|Thursday|2015Q4|N|N|N|2457297|2457569|2456946|2457219|N|N|N|N|N| +2457312|AAAAAAAAAOOHFCAA|2015-10-16|1389|6042|464|2015|5|10|16|4|2015|464|6042|Friday|2015Q4|N|Y|N|2457297|2457569|2456947|2457220|N|N|N|N|N| +2457313|AAAAAAAABOOHFCAA|2015-10-17|1389|6042|464|2015|6|10|17|4|2015|464|6042|Saturday|2015Q4|N|Y|N|2457297|2457569|2456948|2457221|N|N|N|N|N| +2457314|AAAAAAAACOOHFCAA|2015-10-18|1389|6042|464|2015|0|10|18|4|2015|464|6042|Sunday|2015Q4|N|N|N|2457297|2457569|2456949|2457222|N|N|N|N|N| +2457315|AAAAAAAADOOHFCAA|2015-10-19|1389|6042|464|2015|1|10|19|4|2015|464|6042|Monday|2015Q4|N|N|N|2457297|2457569|2456950|2457223|N|N|N|N|N| +2457316|AAAAAAAAEOOHFCAA|2015-10-20|1389|6043|464|2015|2|10|20|4|2015|464|6043|Tuesday|2015Q4|N|N|N|2457297|2457569|2456951|2457224|N|N|N|N|N| +2457317|AAAAAAAAFOOHFCAA|2015-10-21|1389|6043|464|2015|3|10|21|4|2015|464|6043|Wednesday|2015Q4|N|N|N|2457297|2457569|2456952|2457225|N|N|N|N|N| +2457318|AAAAAAAAGOOHFCAA|2015-10-22|1389|6043|464|2015|4|10|22|4|2015|464|6043|Thursday|2015Q4|N|N|N|2457297|2457569|2456953|2457226|N|N|N|N|N| +2457319|AAAAAAAAHOOHFCAA|2015-10-23|1389|6043|464|2015|5|10|23|4|2015|464|6043|Friday|2015Q4|N|Y|N|2457297|2457569|2456954|2457227|N|N|N|N|N| +2457320|AAAAAAAAIOOHFCAA|2015-10-24|1389|6043|464|2015|6|10|24|4|2015|464|6043|Saturday|2015Q4|N|Y|N|2457297|2457569|2456955|2457228|N|N|N|N|N| +2457321|AAAAAAAAJOOHFCAA|2015-10-25|1389|6043|464|2015|0|10|25|4|2015|464|6043|Sunday|2015Q4|N|N|N|2457297|2457569|2456956|2457229|N|N|N|N|N| +2457322|AAAAAAAAKOOHFCAA|2015-10-26|1389|6043|464|2015|1|10|26|4|2015|464|6043|Monday|2015Q4|N|N|N|2457297|2457569|2456957|2457230|N|N|N|N|N| +2457323|AAAAAAAALOOHFCAA|2015-10-27|1389|6044|464|2015|2|10|27|4|2015|464|6044|Tuesday|2015Q4|N|N|N|2457297|2457569|2456958|2457231|N|N|N|N|N| +2457324|AAAAAAAAMOOHFCAA|2015-10-28|1389|6044|464|2015|3|10|28|4|2015|464|6044|Wednesday|2015Q4|N|N|N|2457297|2457569|2456959|2457232|N|N|N|N|N| +2457325|AAAAAAAANOOHFCAA|2015-10-29|1389|6044|464|2015|4|10|29|4|2015|464|6044|Thursday|2015Q4|N|N|N|2457297|2457569|2456960|2457233|N|N|N|N|N| +2457326|AAAAAAAAOOOHFCAA|2015-10-30|1389|6044|464|2015|5|10|30|4|2015|464|6044|Friday|2015Q4|N|Y|N|2457297|2457569|2456961|2457234|N|N|N|N|N| +2457327|AAAAAAAAPOOHFCAA|2015-10-31|1389|6044|464|2015|6|10|31|4|2015|464|6044|Saturday|2015Q4|N|Y|N|2457297|2457569|2456962|2457235|N|N|N|N|N| +2457328|AAAAAAAAAPOHFCAA|2015-11-01|1390|6044|464|2015|0|11|1|4|2015|464|6044|Sunday|2015Q4|N|N|N|2457328|2457631|2456963|2457236|N|N|N|N|N| +2457329|AAAAAAAABPOHFCAA|2015-11-02|1390|6044|464|2015|1|11|2|4|2015|464|6044|Monday|2015Q4|N|N|N|2457328|2457631|2456964|2457237|N|N|N|N|N| +2457330|AAAAAAAACPOHFCAA|2015-11-03|1390|6045|464|2015|2|11|3|4|2015|464|6045|Tuesday|2015Q4|N|N|N|2457328|2457631|2456965|2457238|N|N|N|N|N| +2457331|AAAAAAAADPOHFCAA|2015-11-04|1390|6045|464|2015|3|11|4|4|2015|464|6045|Wednesday|2015Q4|N|N|N|2457328|2457631|2456966|2457239|N|N|N|N|N| +2457332|AAAAAAAAEPOHFCAA|2015-11-05|1390|6045|464|2015|4|11|5|4|2015|464|6045|Thursday|2015Q4|N|N|N|2457328|2457631|2456967|2457240|N|N|N|N|N| +2457333|AAAAAAAAFPOHFCAA|2015-11-06|1390|6045|464|2015|5|11|6|4|2015|464|6045|Friday|2015Q4|N|Y|N|2457328|2457631|2456968|2457241|N|N|N|N|N| +2457334|AAAAAAAAGPOHFCAA|2015-11-07|1390|6045|464|2015|6|11|7|4|2015|464|6045|Saturday|2015Q4|N|Y|N|2457328|2457631|2456969|2457242|N|N|N|N|N| +2457335|AAAAAAAAHPOHFCAA|2015-11-08|1390|6045|464|2015|0|11|8|4|2015|464|6045|Sunday|2015Q4|N|N|N|2457328|2457631|2456970|2457243|N|N|N|N|N| +2457336|AAAAAAAAIPOHFCAA|2015-11-09|1390|6045|464|2015|1|11|9|4|2015|464|6045|Monday|2015Q4|N|N|N|2457328|2457631|2456971|2457244|N|N|N|N|N| +2457337|AAAAAAAAJPOHFCAA|2015-11-10|1390|6046|464|2015|2|11|10|4|2015|464|6046|Tuesday|2015Q4|N|N|N|2457328|2457631|2456972|2457245|N|N|N|N|N| +2457338|AAAAAAAAKPOHFCAA|2015-11-11|1390|6046|464|2015|3|11|11|4|2015|464|6046|Wednesday|2015Q4|N|N|N|2457328|2457631|2456973|2457246|N|N|N|N|N| +2457339|AAAAAAAALPOHFCAA|2015-11-12|1390|6046|464|2015|4|11|12|4|2015|464|6046|Thursday|2015Q4|N|N|N|2457328|2457631|2456974|2457247|N|N|N|N|N| +2457340|AAAAAAAAMPOHFCAA|2015-11-13|1390|6046|464|2015|5|11|13|4|2015|464|6046|Friday|2015Q4|N|Y|N|2457328|2457631|2456975|2457248|N|N|N|N|N| +2457341|AAAAAAAANPOHFCAA|2015-11-14|1390|6046|464|2015|6|11|14|4|2015|464|6046|Saturday|2015Q4|N|Y|N|2457328|2457631|2456976|2457249|N|N|N|N|N| +2457342|AAAAAAAAOPOHFCAA|2015-11-15|1390|6046|464|2015|0|11|15|4|2015|464|6046|Sunday|2015Q4|N|N|N|2457328|2457631|2456977|2457250|N|N|N|N|N| +2457343|AAAAAAAAPPOHFCAA|2015-11-16|1390|6046|464|2015|1|11|16|4|2015|464|6046|Monday|2015Q4|N|N|N|2457328|2457631|2456978|2457251|N|N|N|N|N| +2457344|AAAAAAAAAAPHFCAA|2015-11-17|1390|6047|464|2015|2|11|17|4|2015|464|6047|Tuesday|2015Q4|N|N|N|2457328|2457631|2456979|2457252|N|N|N|N|N| +2457345|AAAAAAAABAPHFCAA|2015-11-18|1390|6047|464|2015|3|11|18|4|2015|464|6047|Wednesday|2015Q4|N|N|N|2457328|2457631|2456980|2457253|N|N|N|N|N| +2457346|AAAAAAAACAPHFCAA|2015-11-19|1390|6047|464|2015|4|11|19|4|2015|464|6047|Thursday|2015Q4|N|N|N|2457328|2457631|2456981|2457254|N|N|N|N|N| +2457347|AAAAAAAADAPHFCAA|2015-11-20|1390|6047|464|2015|5|11|20|4|2015|464|6047|Friday|2015Q4|N|Y|N|2457328|2457631|2456982|2457255|N|N|N|N|N| +2457348|AAAAAAAAEAPHFCAA|2015-11-21|1390|6047|464|2015|6|11|21|4|2015|464|6047|Saturday|2015Q4|N|Y|N|2457328|2457631|2456983|2457256|N|N|N|N|N| +2457349|AAAAAAAAFAPHFCAA|2015-11-22|1390|6047|464|2015|0|11|22|4|2015|464|6047|Sunday|2015Q4|N|N|N|2457328|2457631|2456984|2457257|N|N|N|N|N| +2457350|AAAAAAAAGAPHFCAA|2015-11-23|1390|6047|464|2015|1|11|23|4|2015|464|6047|Monday|2015Q4|N|N|N|2457328|2457631|2456985|2457258|N|N|N|N|N| +2457351|AAAAAAAAHAPHFCAA|2015-11-24|1390|6048|464|2015|2|11|24|4|2015|464|6048|Tuesday|2015Q4|N|N|N|2457328|2457631|2456986|2457259|N|N|N|N|N| +2457352|AAAAAAAAIAPHFCAA|2015-11-25|1390|6048|464|2015|3|11|25|4|2015|464|6048|Wednesday|2015Q4|N|N|N|2457328|2457631|2456987|2457260|N|N|N|N|N| +2457353|AAAAAAAAJAPHFCAA|2015-11-26|1390|6048|464|2015|4|11|26|4|2015|464|6048|Thursday|2015Q4|N|N|N|2457328|2457631|2456988|2457261|N|N|N|N|N| +2457354|AAAAAAAAKAPHFCAA|2015-11-27|1390|6048|464|2015|5|11|27|4|2015|464|6048|Friday|2015Q4|N|Y|N|2457328|2457631|2456989|2457262|N|N|N|N|N| +2457355|AAAAAAAALAPHFCAA|2015-11-28|1390|6048|464|2015|6|11|28|4|2015|464|6048|Saturday|2015Q4|N|Y|N|2457328|2457631|2456990|2457263|N|N|N|N|N| +2457356|AAAAAAAAMAPHFCAA|2015-11-29|1390|6048|464|2015|0|11|29|4|2015|464|6048|Sunday|2015Q4|N|N|N|2457328|2457631|2456991|2457264|N|N|N|N|N| +2457357|AAAAAAAANAPHFCAA|2015-11-30|1390|6048|464|2015|1|11|30|4|2015|464|6048|Monday|2015Q4|N|N|N|2457328|2457631|2456992|2457265|N|N|N|N|N| +2457358|AAAAAAAAOAPHFCAA|2015-12-01|1391|6049|465|2015|2|12|1|4|2015|465|6049|Tuesday|2015Q4|N|N|N|2457358|2457691|2456993|2457266|N|N|N|N|N| +2457359|AAAAAAAAPAPHFCAA|2015-12-02|1391|6049|465|2015|3|12|2|4|2015|465|6049|Wednesday|2015Q4|N|N|N|2457358|2457691|2456994|2457267|N|N|N|N|N| +2457360|AAAAAAAAABPHFCAA|2015-12-03|1391|6049|465|2015|4|12|3|4|2015|465|6049|Thursday|2015Q4|N|N|N|2457358|2457691|2456995|2457268|N|N|N|N|N| +2457361|AAAAAAAABBPHFCAA|2015-12-04|1391|6049|465|2015|5|12|4|4|2015|465|6049|Friday|2015Q4|N|Y|N|2457358|2457691|2456996|2457269|N|N|N|N|N| +2457362|AAAAAAAACBPHFCAA|2015-12-05|1391|6049|465|2015|6|12|5|4|2015|465|6049|Saturday|2015Q4|N|Y|N|2457358|2457691|2456997|2457270|N|N|N|N|N| +2457363|AAAAAAAADBPHFCAA|2015-12-06|1391|6049|465|2015|0|12|6|4|2015|465|6049|Sunday|2015Q4|N|N|N|2457358|2457691|2456998|2457271|N|N|N|N|N| +2457364|AAAAAAAAEBPHFCAA|2015-12-07|1391|6049|465|2015|1|12|7|4|2015|465|6049|Monday|2015Q4|N|N|N|2457358|2457691|2456999|2457272|N|N|N|N|N| +2457365|AAAAAAAAFBPHFCAA|2015-12-08|1391|6050|465|2015|2|12|8|4|2015|465|6050|Tuesday|2015Q4|N|N|N|2457358|2457691|2457000|2457273|N|N|N|N|N| +2457366|AAAAAAAAGBPHFCAA|2015-12-09|1391|6050|465|2015|3|12|9|4|2015|465|6050|Wednesday|2015Q4|N|N|N|2457358|2457691|2457001|2457274|N|N|N|N|N| +2457367|AAAAAAAAHBPHFCAA|2015-12-10|1391|6050|465|2015|4|12|10|4|2015|465|6050|Thursday|2015Q4|N|N|N|2457358|2457691|2457002|2457275|N|N|N|N|N| +2457368|AAAAAAAAIBPHFCAA|2015-12-11|1391|6050|465|2015|5|12|11|4|2015|465|6050|Friday|2015Q4|N|Y|N|2457358|2457691|2457003|2457276|N|N|N|N|N| +2457369|AAAAAAAAJBPHFCAA|2015-12-12|1391|6050|465|2015|6|12|12|4|2015|465|6050|Saturday|2015Q4|N|Y|N|2457358|2457691|2457004|2457277|N|N|N|N|N| +2457370|AAAAAAAAKBPHFCAA|2015-12-13|1391|6050|465|2015|0|12|13|4|2015|465|6050|Sunday|2015Q4|N|N|N|2457358|2457691|2457005|2457278|N|N|N|N|N| +2457371|AAAAAAAALBPHFCAA|2015-12-14|1391|6050|465|2015|1|12|14|4|2015|465|6050|Monday|2015Q4|N|N|N|2457358|2457691|2457006|2457279|N|N|N|N|N| +2457372|AAAAAAAAMBPHFCAA|2015-12-15|1391|6051|465|2015|2|12|15|4|2015|465|6051|Tuesday|2015Q4|N|N|N|2457358|2457691|2457007|2457280|N|N|N|N|N| +2457373|AAAAAAAANBPHFCAA|2015-12-16|1391|6051|465|2015|3|12|16|4|2015|465|6051|Wednesday|2015Q4|N|N|N|2457358|2457691|2457008|2457281|N|N|N|N|N| +2457374|AAAAAAAAOBPHFCAA|2015-12-17|1391|6051|465|2015|4|12|17|4|2015|465|6051|Thursday|2015Q4|N|N|N|2457358|2457691|2457009|2457282|N|N|N|N|N| +2457375|AAAAAAAAPBPHFCAA|2015-12-18|1391|6051|465|2015|5|12|18|4|2015|465|6051|Friday|2015Q4|N|Y|N|2457358|2457691|2457010|2457283|N|N|N|N|N| +2457376|AAAAAAAAACPHFCAA|2015-12-19|1391|6051|465|2015|6|12|19|4|2015|465|6051|Saturday|2015Q4|N|Y|N|2457358|2457691|2457011|2457284|N|N|N|N|N| +2457377|AAAAAAAABCPHFCAA|2015-12-20|1391|6051|465|2015|0|12|20|4|2015|465|6051|Sunday|2015Q4|N|N|N|2457358|2457691|2457012|2457285|N|N|N|N|N| +2457378|AAAAAAAACCPHFCAA|2015-12-21|1391|6051|465|2015|1|12|21|4|2015|465|6051|Monday|2015Q4|N|N|N|2457358|2457691|2457013|2457286|N|N|N|N|N| +2457379|AAAAAAAADCPHFCAA|2015-12-22|1391|6052|465|2015|2|12|22|4|2015|465|6052|Tuesday|2015Q4|N|N|N|2457358|2457691|2457014|2457287|N|N|N|N|N| +2457380|AAAAAAAAECPHFCAA|2015-12-23|1391|6052|465|2015|3|12|23|4|2015|465|6052|Wednesday|2015Q4|N|N|N|2457358|2457691|2457015|2457288|N|N|N|N|N| +2457381|AAAAAAAAFCPHFCAA|2015-12-24|1391|6052|465|2015|4|12|24|4|2015|465|6052|Thursday|2015Q4|N|N|N|2457358|2457691|2457016|2457289|N|N|N|N|N| +2457382|AAAAAAAAGCPHFCAA|2015-12-25|1391|6052|465|2015|5|12|25|4|2015|465|6052|Friday|2015Q4|N|Y|N|2457358|2457691|2457017|2457290|N|N|N|N|N| +2457383|AAAAAAAAHCPHFCAA|2015-12-26|1391|6052|465|2015|6|12|26|4|2015|465|6052|Saturday|2015Q4|Y|Y|N|2457358|2457691|2457018|2457291|N|N|N|N|N| +2457384|AAAAAAAAICPHFCAA|2015-12-27|1391|6052|465|2015|0|12|27|4|2015|465|6052|Sunday|2015Q4|N|N|Y|2457358|2457691|2457019|2457292|N|N|N|N|N| +2457385|AAAAAAAAJCPHFCAA|2015-12-28|1391|6052|465|2015|1|12|28|4|2015|465|6052|Monday|2015Q4|N|N|N|2457358|2457691|2457020|2457293|N|N|N|N|N| +2457386|AAAAAAAAKCPHFCAA|2015-12-29|1391|6053|465|2015|2|12|29|4|2015|465|6053|Tuesday|2015Q4|N|N|N|2457358|2457691|2457021|2457294|N|N|N|N|N| +2457387|AAAAAAAALCPHFCAA|2015-12-30|1391|6053|465|2015|3|12|30|4|2015|465|6053|Wednesday|2015Q4|N|N|N|2457358|2457691|2457022|2457295|N|N|N|N|N| +2457388|AAAAAAAAMCPHFCAA|2015-12-31|1391|6053|465|2015|4|12|31|4|2015|465|6053|Thursday|2015Q4|N|N|N|2457358|2457691|2457023|2457296|N|N|N|N|N| +2457389|AAAAAAAANCPHFCAA|2016-01-01|1392|6053|465|2016|5|1|1|1|2016|465|6053|Friday|2016Q1|Y|Y|N|2457389|2457388|2457024|2457297|N|N|N|N|N| +2457390|AAAAAAAAOCPHFCAA|2016-01-02|1392|6053|465|2016|6|1|2|1|2016|465|6053|Saturday|2016Q1|N|Y|Y|2457389|2457388|2457025|2457298|N|N|N|N|N| +2457391|AAAAAAAAPCPHFCAA|2016-01-03|1392|6053|465|2016|0|1|3|1|2016|465|6053|Sunday|2016Q1|N|N|N|2457389|2457388|2457026|2457299|N|N|N|N|N| +2457392|AAAAAAAAADPHFCAA|2016-01-04|1392|6053|465|2016|1|1|4|1|2016|465|6053|Monday|2016Q1|N|N|N|2457389|2457388|2457027|2457300|N|N|N|N|N| +2457393|AAAAAAAABDPHFCAA|2016-01-05|1392|6054|465|2016|2|1|5|1|2016|465|6054|Tuesday|2016Q1|N|N|N|2457389|2457388|2457028|2457301|N|N|N|N|N| +2457394|AAAAAAAACDPHFCAA|2016-01-06|1392|6054|465|2016|3|1|6|1|2016|465|6054|Wednesday|2016Q1|N|N|N|2457389|2457388|2457029|2457302|N|N|N|N|N| +2457395|AAAAAAAADDPHFCAA|2016-01-07|1392|6054|465|2016|4|1|7|1|2016|465|6054|Thursday|2016Q1|N|N|N|2457389|2457388|2457030|2457303|N|N|N|N|N| +2457396|AAAAAAAAEDPHFCAA|2016-01-08|1392|6054|465|2016|5|1|8|1|2016|465|6054|Friday|2016Q1|N|Y|N|2457389|2457388|2457031|2457304|N|N|N|N|N| +2457397|AAAAAAAAFDPHFCAA|2016-01-09|1392|6054|465|2016|6|1|9|1|2016|465|6054|Saturday|2016Q1|N|Y|N|2457389|2457388|2457032|2457305|N|N|N|N|N| +2457398|AAAAAAAAGDPHFCAA|2016-01-10|1392|6054|465|2016|0|1|10|1|2016|465|6054|Sunday|2016Q1|N|N|N|2457389|2457388|2457033|2457306|N|N|N|N|N| +2457399|AAAAAAAAHDPHFCAA|2016-01-11|1392|6054|465|2016|1|1|11|1|2016|465|6054|Monday|2016Q1|N|N|N|2457389|2457388|2457034|2457307|N|N|N|N|N| +2457400|AAAAAAAAIDPHFCAA|2016-01-12|1392|6055|465|2016|2|1|12|1|2016|465|6055|Tuesday|2016Q1|N|N|N|2457389|2457388|2457035|2457308|N|N|N|N|N| +2457401|AAAAAAAAJDPHFCAA|2016-01-13|1392|6055|465|2016|3|1|13|1|2016|465|6055|Wednesday|2016Q1|N|N|N|2457389|2457388|2457036|2457309|N|N|N|N|N| +2457402|AAAAAAAAKDPHFCAA|2016-01-14|1392|6055|465|2016|4|1|14|1|2016|465|6055|Thursday|2016Q1|N|N|N|2457389|2457388|2457037|2457310|N|N|N|N|N| +2457403|AAAAAAAALDPHFCAA|2016-01-15|1392|6055|465|2016|5|1|15|1|2016|465|6055|Friday|2016Q1|N|Y|N|2457389|2457388|2457038|2457311|N|N|N|N|N| +2457404|AAAAAAAAMDPHFCAA|2016-01-16|1392|6055|465|2016|6|1|16|1|2016|465|6055|Saturday|2016Q1|N|Y|N|2457389|2457388|2457039|2457312|N|N|N|N|N| +2457405|AAAAAAAANDPHFCAA|2016-01-17|1392|6055|465|2016|0|1|17|1|2016|465|6055|Sunday|2016Q1|N|N|N|2457389|2457388|2457040|2457313|N|N|N|N|N| +2457406|AAAAAAAAODPHFCAA|2016-01-18|1392|6055|465|2016|1|1|18|1|2016|465|6055|Monday|2016Q1|N|N|N|2457389|2457388|2457041|2457314|N|N|N|N|N| +2457407|AAAAAAAAPDPHFCAA|2016-01-19|1392|6056|465|2016|2|1|19|1|2016|465|6056|Tuesday|2016Q1|N|N|N|2457389|2457388|2457042|2457315|N|N|N|N|N| +2457408|AAAAAAAAAEPHFCAA|2016-01-20|1392|6056|465|2016|3|1|20|1|2016|465|6056|Wednesday|2016Q1|N|N|N|2457389|2457388|2457043|2457316|N|N|N|N|N| +2457409|AAAAAAAABEPHFCAA|2016-01-21|1392|6056|465|2016|4|1|21|1|2016|465|6056|Thursday|2016Q1|N|N|N|2457389|2457388|2457044|2457317|N|N|N|N|N| +2457410|AAAAAAAACEPHFCAA|2016-01-22|1392|6056|465|2016|5|1|22|1|2016|465|6056|Friday|2016Q1|N|Y|N|2457389|2457388|2457045|2457318|N|N|N|N|N| +2457411|AAAAAAAADEPHFCAA|2016-01-23|1392|6056|465|2016|6|1|23|1|2016|465|6056|Saturday|2016Q1|N|Y|N|2457389|2457388|2457046|2457319|N|N|N|N|N| +2457412|AAAAAAAAEEPHFCAA|2016-01-24|1392|6056|465|2016|0|1|24|1|2016|465|6056|Sunday|2016Q1|N|N|N|2457389|2457388|2457047|2457320|N|N|N|N|N| +2457413|AAAAAAAAFEPHFCAA|2016-01-25|1392|6056|465|2016|1|1|25|1|2016|465|6056|Monday|2016Q1|N|N|N|2457389|2457388|2457048|2457321|N|N|N|N|N| +2457414|AAAAAAAAGEPHFCAA|2016-01-26|1392|6057|465|2016|2|1|26|1|2016|465|6057|Tuesday|2016Q1|N|N|N|2457389|2457388|2457049|2457322|N|N|N|N|N| +2457415|AAAAAAAAHEPHFCAA|2016-01-27|1392|6057|465|2016|3|1|27|1|2016|465|6057|Wednesday|2016Q1|N|N|N|2457389|2457388|2457050|2457323|N|N|N|N|N| +2457416|AAAAAAAAIEPHFCAA|2016-01-28|1392|6057|465|2016|4|1|28|1|2016|465|6057|Thursday|2016Q1|N|N|N|2457389|2457388|2457051|2457324|N|N|N|N|N| +2457417|AAAAAAAAJEPHFCAA|2016-01-29|1392|6057|465|2016|5|1|29|1|2016|465|6057|Friday|2016Q1|N|Y|N|2457389|2457388|2457052|2457325|N|N|N|N|N| +2457418|AAAAAAAAKEPHFCAA|2016-01-30|1392|6057|465|2016|6|1|30|1|2016|465|6057|Saturday|2016Q1|N|Y|N|2457389|2457388|2457053|2457326|N|N|N|N|N| +2457419|AAAAAAAALEPHFCAA|2016-01-31|1392|6057|465|2016|0|1|31|1|2016|465|6057|Sunday|2016Q1|N|N|N|2457389|2457388|2457054|2457327|N|N|N|N|N| +2457420|AAAAAAAAMEPHFCAA|2016-02-01|1393|6057|465|2016|1|2|1|1|2016|465|6057|Monday|2016Q1|N|N|N|2457420|2457450|2457055|2457328|N|N|N|N|N| +2457421|AAAAAAAANEPHFCAA|2016-02-02|1393|6058|465|2016|2|2|2|1|2016|465|6058|Tuesday|2016Q1|N|N|N|2457420|2457450|2457056|2457329|N|N|N|N|N| +2457422|AAAAAAAAOEPHFCAA|2016-02-03|1393|6058|465|2016|3|2|3|1|2016|465|6058|Wednesday|2016Q1|N|N|N|2457420|2457450|2457057|2457330|N|N|N|N|N| +2457423|AAAAAAAAPEPHFCAA|2016-02-04|1393|6058|465|2016|4|2|4|1|2016|465|6058|Thursday|2016Q1|N|N|N|2457420|2457450|2457058|2457331|N|N|N|N|N| +2457424|AAAAAAAAAFPHFCAA|2016-02-05|1393|6058|465|2016|5|2|5|1|2016|465|6058|Friday|2016Q1|N|Y|N|2457420|2457450|2457059|2457332|N|N|N|N|N| +2457425|AAAAAAAABFPHFCAA|2016-02-06|1393|6058|465|2016|6|2|6|1|2016|465|6058|Saturday|2016Q1|N|Y|N|2457420|2457450|2457060|2457333|N|N|N|N|N| +2457426|AAAAAAAACFPHFCAA|2016-02-07|1393|6058|465|2016|0|2|7|1|2016|465|6058|Sunday|2016Q1|N|N|N|2457420|2457450|2457061|2457334|N|N|N|N|N| +2457427|AAAAAAAADFPHFCAA|2016-02-08|1393|6058|465|2016|1|2|8|1|2016|465|6058|Monday|2016Q1|N|N|N|2457420|2457450|2457062|2457335|N|N|N|N|N| +2457428|AAAAAAAAEFPHFCAA|2016-02-09|1393|6059|465|2016|2|2|9|1|2016|465|6059|Tuesday|2016Q1|N|N|N|2457420|2457450|2457063|2457336|N|N|N|N|N| +2457429|AAAAAAAAFFPHFCAA|2016-02-10|1393|6059|465|2016|3|2|10|1|2016|465|6059|Wednesday|2016Q1|N|N|N|2457420|2457450|2457064|2457337|N|N|N|N|N| +2457430|AAAAAAAAGFPHFCAA|2016-02-11|1393|6059|465|2016|4|2|11|1|2016|465|6059|Thursday|2016Q1|N|N|N|2457420|2457450|2457065|2457338|N|N|N|N|N| +2457431|AAAAAAAAHFPHFCAA|2016-02-12|1393|6059|465|2016|5|2|12|1|2016|465|6059|Friday|2016Q1|N|Y|N|2457420|2457450|2457066|2457339|N|N|N|N|N| +2457432|AAAAAAAAIFPHFCAA|2016-02-13|1393|6059|465|2016|6|2|13|1|2016|465|6059|Saturday|2016Q1|N|Y|N|2457420|2457450|2457067|2457340|N|N|N|N|N| +2457433|AAAAAAAAJFPHFCAA|2016-02-14|1393|6059|465|2016|0|2|14|1|2016|465|6059|Sunday|2016Q1|N|N|N|2457420|2457450|2457068|2457341|N|N|N|N|N| +2457434|AAAAAAAAKFPHFCAA|2016-02-15|1393|6059|465|2016|1|2|15|1|2016|465|6059|Monday|2016Q1|N|N|N|2457420|2457450|2457069|2457342|N|N|N|N|N| +2457435|AAAAAAAALFPHFCAA|2016-02-16|1393|6060|465|2016|2|2|16|1|2016|465|6060|Tuesday|2016Q1|N|N|N|2457420|2457450|2457070|2457343|N|N|N|N|N| +2457436|AAAAAAAAMFPHFCAA|2016-02-17|1393|6060|465|2016|3|2|17|1|2016|465|6060|Wednesday|2016Q1|N|N|N|2457420|2457450|2457071|2457344|N|N|N|N|N| +2457437|AAAAAAAANFPHFCAA|2016-02-18|1393|6060|465|2016|4|2|18|1|2016|465|6060|Thursday|2016Q1|N|N|N|2457420|2457450|2457072|2457345|N|N|N|N|N| +2457438|AAAAAAAAOFPHFCAA|2016-02-19|1393|6060|465|2016|5|2|19|1|2016|465|6060|Friday|2016Q1|N|Y|N|2457420|2457450|2457073|2457346|N|N|N|N|N| +2457439|AAAAAAAAPFPHFCAA|2016-02-20|1393|6060|465|2016|6|2|20|1|2016|465|6060|Saturday|2016Q1|N|Y|N|2457420|2457450|2457074|2457347|N|N|N|N|N| +2457440|AAAAAAAAAGPHFCAA|2016-02-21|1393|6060|465|2016|0|2|21|1|2016|465|6060|Sunday|2016Q1|N|N|N|2457420|2457450|2457075|2457348|N|N|N|N|N| +2457441|AAAAAAAABGPHFCAA|2016-02-22|1393|6060|465|2016|1|2|22|1|2016|465|6060|Monday|2016Q1|N|N|N|2457420|2457450|2457076|2457349|N|N|N|N|N| +2457442|AAAAAAAACGPHFCAA|2016-02-23|1393|6061|465|2016|2|2|23|1|2016|465|6061|Tuesday|2016Q1|N|N|N|2457420|2457450|2457077|2457350|N|N|N|N|N| +2457443|AAAAAAAADGPHFCAA|2016-02-24|1393|6061|465|2016|3|2|24|1|2016|465|6061|Wednesday|2016Q1|N|N|N|2457420|2457450|2457078|2457351|N|N|N|N|N| +2457444|AAAAAAAAEGPHFCAA|2016-02-25|1393|6061|465|2016|4|2|25|1|2016|465|6061|Thursday|2016Q1|N|N|N|2457420|2457450|2457079|2457352|N|N|N|N|N| +2457445|AAAAAAAAFGPHFCAA|2016-02-26|1393|6061|465|2016|5|2|26|1|2016|465|6061|Friday|2016Q1|N|Y|N|2457420|2457450|2457080|2457353|N|N|N|N|N| +2457446|AAAAAAAAGGPHFCAA|2016-02-27|1393|6061|465|2016|6|2|27|1|2016|465|6061|Saturday|2016Q1|N|Y|N|2457420|2457450|2457081|2457354|N|N|N|N|N| +2457447|AAAAAAAAHGPHFCAA|2016-02-28|1393|6061|465|2016|0|2|28|1|2016|465|6061|Sunday|2016Q1|N|N|N|2457420|2457450|2457082|2457355|N|N|N|N|N| +2457448|AAAAAAAAIGPHFCAA|2016-02-29|1393|6061|465|2016|1|2|29|1|2016|465|6061|Monday|2016Q1|N|N|N|2457420|2457450|2457082|2457356|N|N|N|N|N| +2457449|AAAAAAAAJGPHFCAA|2016-03-01|1394|6062|466|2016|2|3|1|1|2016|466|6062|Tuesday|2016Q1|N|N|N|2457449|2457508|2457083|2457357|N|N|N|N|N| +2457450|AAAAAAAAKGPHFCAA|2016-03-02|1394|6062|466|2016|3|3|2|1|2016|466|6062|Wednesday|2016Q1|N|N|N|2457449|2457508|2457084|2457358|N|N|N|N|N| +2457451|AAAAAAAALGPHFCAA|2016-03-03|1394|6062|466|2016|4|3|3|1|2016|466|6062|Thursday|2016Q1|N|N|N|2457449|2457508|2457085|2457359|N|N|N|N|N| +2457452|AAAAAAAAMGPHFCAA|2016-03-04|1394|6062|466|2016|5|3|4|1|2016|466|6062|Friday|2016Q1|N|Y|N|2457449|2457508|2457086|2457360|N|N|N|N|N| +2457453|AAAAAAAANGPHFCAA|2016-03-05|1394|6062|466|2016|6|3|5|1|2016|466|6062|Saturday|2016Q1|N|Y|N|2457449|2457508|2457087|2457361|N|N|N|N|N| +2457454|AAAAAAAAOGPHFCAA|2016-03-06|1394|6062|466|2016|0|3|6|1|2016|466|6062|Sunday|2016Q1|N|N|N|2457449|2457508|2457088|2457362|N|N|N|N|N| +2457455|AAAAAAAAPGPHFCAA|2016-03-07|1394|6062|466|2016|1|3|7|1|2016|466|6062|Monday|2016Q1|N|N|N|2457449|2457508|2457089|2457363|N|N|N|N|N| +2457456|AAAAAAAAAHPHFCAA|2016-03-08|1394|6063|466|2016|2|3|8|1|2016|466|6063|Tuesday|2016Q1|N|N|N|2457449|2457508|2457090|2457364|N|N|N|N|N| +2457457|AAAAAAAABHPHFCAA|2016-03-09|1394|6063|466|2016|3|3|9|1|2016|466|6063|Wednesday|2016Q1|N|N|N|2457449|2457508|2457091|2457365|N|N|N|N|N| +2457458|AAAAAAAACHPHFCAA|2016-03-10|1394|6063|466|2016|4|3|10|1|2016|466|6063|Thursday|2016Q1|N|N|N|2457449|2457508|2457092|2457366|N|N|N|N|N| +2457459|AAAAAAAADHPHFCAA|2016-03-11|1394|6063|466|2016|5|3|11|1|2016|466|6063|Friday|2016Q1|N|Y|N|2457449|2457508|2457093|2457367|N|N|N|N|N| +2457460|AAAAAAAAEHPHFCAA|2016-03-12|1394|6063|466|2016|6|3|12|1|2016|466|6063|Saturday|2016Q1|N|Y|N|2457449|2457508|2457094|2457368|N|N|N|N|N| +2457461|AAAAAAAAFHPHFCAA|2016-03-13|1394|6063|466|2016|0|3|13|1|2016|466|6063|Sunday|2016Q1|N|N|N|2457449|2457508|2457095|2457369|N|N|N|N|N| +2457462|AAAAAAAAGHPHFCAA|2016-03-14|1394|6063|466|2016|1|3|14|1|2016|466|6063|Monday|2016Q1|N|N|N|2457449|2457508|2457096|2457370|N|N|N|N|N| +2457463|AAAAAAAAHHPHFCAA|2016-03-15|1394|6064|466|2016|2|3|15|1|2016|466|6064|Tuesday|2016Q1|N|N|N|2457449|2457508|2457097|2457371|N|N|N|N|N| +2457464|AAAAAAAAIHPHFCAA|2016-03-16|1394|6064|466|2016|3|3|16|1|2016|466|6064|Wednesday|2016Q1|N|N|N|2457449|2457508|2457098|2457372|N|N|N|N|N| +2457465|AAAAAAAAJHPHFCAA|2016-03-17|1394|6064|466|2016|4|3|17|1|2016|466|6064|Thursday|2016Q1|N|N|N|2457449|2457508|2457099|2457373|N|N|N|N|N| +2457466|AAAAAAAAKHPHFCAA|2016-03-18|1394|6064|466|2016|5|3|18|1|2016|466|6064|Friday|2016Q1|N|Y|N|2457449|2457508|2457100|2457374|N|N|N|N|N| +2457467|AAAAAAAALHPHFCAA|2016-03-19|1394|6064|466|2016|6|3|19|1|2016|466|6064|Saturday|2016Q1|N|Y|N|2457449|2457508|2457101|2457375|N|N|N|N|N| +2457468|AAAAAAAAMHPHFCAA|2016-03-20|1394|6064|466|2016|0|3|20|1|2016|466|6064|Sunday|2016Q1|N|N|N|2457449|2457508|2457102|2457376|N|N|N|N|N| +2457469|AAAAAAAANHPHFCAA|2016-03-21|1394|6064|466|2016|1|3|21|1|2016|466|6064|Monday|2016Q1|N|N|N|2457449|2457508|2457103|2457377|N|N|N|N|N| +2457470|AAAAAAAAOHPHFCAA|2016-03-22|1394|6065|466|2016|2|3|22|1|2016|466|6065|Tuesday|2016Q1|N|N|N|2457449|2457508|2457104|2457378|N|N|N|N|N| +2457471|AAAAAAAAPHPHFCAA|2016-03-23|1394|6065|466|2016|3|3|23|1|2016|466|6065|Wednesday|2016Q1|N|N|N|2457449|2457508|2457105|2457379|N|N|N|N|N| +2457472|AAAAAAAAAIPHFCAA|2016-03-24|1394|6065|466|2016|4|3|24|1|2016|466|6065|Thursday|2016Q1|N|N|N|2457449|2457508|2457106|2457380|N|N|N|N|N| +2457473|AAAAAAAABIPHFCAA|2016-03-25|1394|6065|466|2016|5|3|25|1|2016|466|6065|Friday|2016Q1|N|Y|N|2457449|2457508|2457107|2457381|N|N|N|N|N| +2457474|AAAAAAAACIPHFCAA|2016-03-26|1394|6065|466|2016|6|3|26|1|2016|466|6065|Saturday|2016Q1|N|Y|N|2457449|2457508|2457108|2457382|N|N|N|N|N| +2457475|AAAAAAAADIPHFCAA|2016-03-27|1394|6065|466|2016|0|3|27|1|2016|466|6065|Sunday|2016Q1|N|N|N|2457449|2457508|2457109|2457383|N|N|N|N|N| +2457476|AAAAAAAAEIPHFCAA|2016-03-28|1394|6065|466|2016|1|3|28|1|2016|466|6065|Monday|2016Q1|N|N|N|2457449|2457508|2457110|2457384|N|N|N|N|N| +2457477|AAAAAAAAFIPHFCAA|2016-03-29|1394|6066|466|2016|2|3|29|1|2016|466|6066|Tuesday|2016Q1|N|N|N|2457449|2457508|2457111|2457385|N|N|N|N|N| +2457478|AAAAAAAAGIPHFCAA|2016-03-30|1394|6066|466|2016|3|3|30|1|2016|466|6066|Wednesday|2016Q1|N|N|N|2457449|2457508|2457112|2457386|N|N|N|N|N| +2457479|AAAAAAAAHIPHFCAA|2016-03-31|1394|6066|466|2016|4|3|31|1|2016|466|6066|Thursday|2016Q1|N|N|N|2457449|2457508|2457113|2457387|N|N|N|N|N| +2457480|AAAAAAAAIIPHFCAA|2016-04-01|1395|6066|466|2016|5|4|1|2|2016|466|6066|Friday|2016Q2|N|Y|N|2457480|2457570|2457114|2457389|N|N|N|N|N| +2457481|AAAAAAAAJIPHFCAA|2016-04-02|1395|6066|466|2016|6|4|2|2|2016|466|6066|Saturday|2016Q2|N|Y|N|2457480|2457570|2457115|2457390|N|N|N|N|N| +2457482|AAAAAAAAKIPHFCAA|2016-04-03|1395|6066|466|2016|0|4|3|2|2016|466|6066|Sunday|2016Q2|N|N|N|2457480|2457570|2457116|2457391|N|N|N|N|N| +2457483|AAAAAAAALIPHFCAA|2016-04-04|1395|6066|466|2016|1|4|4|2|2016|466|6066|Monday|2016Q2|N|N|N|2457480|2457570|2457117|2457392|N|N|N|N|N| +2457484|AAAAAAAAMIPHFCAA|2016-04-05|1395|6067|466|2016|2|4|5|2|2016|466|6067|Tuesday|2016Q2|N|N|N|2457480|2457570|2457118|2457393|N|N|N|N|N| +2457485|AAAAAAAANIPHFCAA|2016-04-06|1395|6067|466|2016|3|4|6|2|2016|466|6067|Wednesday|2016Q2|N|N|N|2457480|2457570|2457119|2457394|N|N|N|N|N| +2457486|AAAAAAAAOIPHFCAA|2016-04-07|1395|6067|466|2016|4|4|7|2|2016|466|6067|Thursday|2016Q2|N|N|N|2457480|2457570|2457120|2457395|N|N|N|N|N| +2457487|AAAAAAAAPIPHFCAA|2016-04-08|1395|6067|466|2016|5|4|8|2|2016|466|6067|Friday|2016Q2|N|Y|N|2457480|2457570|2457121|2457396|N|N|N|N|N| +2457488|AAAAAAAAAJPHFCAA|2016-04-09|1395|6067|466|2016|6|4|9|2|2016|466|6067|Saturday|2016Q2|N|Y|N|2457480|2457570|2457122|2457397|N|N|N|N|N| +2457489|AAAAAAAABJPHFCAA|2016-04-10|1395|6067|466|2016|0|4|10|2|2016|466|6067|Sunday|2016Q2|N|N|N|2457480|2457570|2457123|2457398|N|N|N|N|N| +2457490|AAAAAAAACJPHFCAA|2016-04-11|1395|6067|466|2016|1|4|11|2|2016|466|6067|Monday|2016Q2|N|N|N|2457480|2457570|2457124|2457399|N|N|N|N|N| +2457491|AAAAAAAADJPHFCAA|2016-04-12|1395|6068|466|2016|2|4|12|2|2016|466|6068|Tuesday|2016Q2|N|N|N|2457480|2457570|2457125|2457400|N|N|N|N|N| +2457492|AAAAAAAAEJPHFCAA|2016-04-13|1395|6068|466|2016|3|4|13|2|2016|466|6068|Wednesday|2016Q2|N|N|N|2457480|2457570|2457126|2457401|N|N|N|N|N| +2457493|AAAAAAAAFJPHFCAA|2016-04-14|1395|6068|466|2016|4|4|14|2|2016|466|6068|Thursday|2016Q2|N|N|N|2457480|2457570|2457127|2457402|N|N|N|N|N| +2457494|AAAAAAAAGJPHFCAA|2016-04-15|1395|6068|466|2016|5|4|15|2|2016|466|6068|Friday|2016Q2|N|Y|N|2457480|2457570|2457128|2457403|N|N|N|N|N| +2457495|AAAAAAAAHJPHFCAA|2016-04-16|1395|6068|466|2016|6|4|16|2|2016|466|6068|Saturday|2016Q2|N|Y|N|2457480|2457570|2457129|2457404|N|N|N|N|N| +2457496|AAAAAAAAIJPHFCAA|2016-04-17|1395|6068|466|2016|0|4|17|2|2016|466|6068|Sunday|2016Q2|N|N|N|2457480|2457570|2457130|2457405|N|N|N|N|N| +2457497|AAAAAAAAJJPHFCAA|2016-04-18|1395|6068|466|2016|1|4|18|2|2016|466|6068|Monday|2016Q2|N|N|N|2457480|2457570|2457131|2457406|N|N|N|N|N| +2457498|AAAAAAAAKJPHFCAA|2016-04-19|1395|6069|466|2016|2|4|19|2|2016|466|6069|Tuesday|2016Q2|N|N|N|2457480|2457570|2457132|2457407|N|N|N|N|N| +2457499|AAAAAAAALJPHFCAA|2016-04-20|1395|6069|466|2016|3|4|20|2|2016|466|6069|Wednesday|2016Q2|N|N|N|2457480|2457570|2457133|2457408|N|N|N|N|N| +2457500|AAAAAAAAMJPHFCAA|2016-04-21|1395|6069|466|2016|4|4|21|2|2016|466|6069|Thursday|2016Q2|N|N|N|2457480|2457570|2457134|2457409|N|N|N|N|N| +2457501|AAAAAAAANJPHFCAA|2016-04-22|1395|6069|466|2016|5|4|22|2|2016|466|6069|Friday|2016Q2|N|Y|N|2457480|2457570|2457135|2457410|N|N|N|N|N| +2457502|AAAAAAAAOJPHFCAA|2016-04-23|1395|6069|466|2016|6|4|23|2|2016|466|6069|Saturday|2016Q2|N|Y|N|2457480|2457570|2457136|2457411|N|N|N|N|N| +2457503|AAAAAAAAPJPHFCAA|2016-04-24|1395|6069|466|2016|0|4|24|2|2016|466|6069|Sunday|2016Q2|N|N|N|2457480|2457570|2457137|2457412|N|N|N|N|N| +2457504|AAAAAAAAAKPHFCAA|2016-04-25|1395|6069|466|2016|1|4|25|2|2016|466|6069|Monday|2016Q2|N|N|N|2457480|2457570|2457138|2457413|N|N|N|N|N| +2457505|AAAAAAAABKPHFCAA|2016-04-26|1395|6070|466|2016|2|4|26|2|2016|466|6070|Tuesday|2016Q2|N|N|N|2457480|2457570|2457139|2457414|N|N|N|N|N| +2457506|AAAAAAAACKPHFCAA|2016-04-27|1395|6070|466|2016|3|4|27|2|2016|466|6070|Wednesday|2016Q2|N|N|N|2457480|2457570|2457140|2457415|N|N|N|N|N| +2457507|AAAAAAAADKPHFCAA|2016-04-28|1395|6070|466|2016|4|4|28|2|2016|466|6070|Thursday|2016Q2|N|N|N|2457480|2457570|2457141|2457416|N|N|N|N|N| +2457508|AAAAAAAAEKPHFCAA|2016-04-29|1395|6070|466|2016|5|4|29|2|2016|466|6070|Friday|2016Q2|N|Y|N|2457480|2457570|2457142|2457417|N|N|N|N|N| +2457509|AAAAAAAAFKPHFCAA|2016-04-30|1395|6070|466|2016|6|4|30|2|2016|466|6070|Saturday|2016Q2|N|Y|N|2457480|2457570|2457143|2457418|N|N|N|N|N| +2457510|AAAAAAAAGKPHFCAA|2016-05-01|1396|6070|466|2016|0|5|1|2|2016|466|6070|Sunday|2016Q2|N|N|N|2457510|2457630|2457144|2457419|N|N|N|N|N| +2457511|AAAAAAAAHKPHFCAA|2016-05-02|1396|6070|466|2016|1|5|2|2|2016|466|6070|Monday|2016Q2|N|N|N|2457510|2457630|2457145|2457420|N|N|N|N|N| +2457512|AAAAAAAAIKPHFCAA|2016-05-03|1396|6071|466|2016|2|5|3|2|2016|466|6071|Tuesday|2016Q2|N|N|N|2457510|2457630|2457146|2457421|N|N|N|N|N| +2457513|AAAAAAAAJKPHFCAA|2016-05-04|1396|6071|466|2016|3|5|4|2|2016|466|6071|Wednesday|2016Q2|N|N|N|2457510|2457630|2457147|2457422|N|N|N|N|N| +2457514|AAAAAAAAKKPHFCAA|2016-05-05|1396|6071|466|2016|4|5|5|2|2016|466|6071|Thursday|2016Q2|N|N|N|2457510|2457630|2457148|2457423|N|N|N|N|N| +2457515|AAAAAAAALKPHFCAA|2016-05-06|1396|6071|466|2016|5|5|6|2|2016|466|6071|Friday|2016Q2|N|Y|N|2457510|2457630|2457149|2457424|N|N|N|N|N| +2457516|AAAAAAAAMKPHFCAA|2016-05-07|1396|6071|466|2016|6|5|7|2|2016|466|6071|Saturday|2016Q2|N|Y|N|2457510|2457630|2457150|2457425|N|N|N|N|N| +2457517|AAAAAAAANKPHFCAA|2016-05-08|1396|6071|466|2016|0|5|8|2|2016|466|6071|Sunday|2016Q2|N|N|N|2457510|2457630|2457151|2457426|N|N|N|N|N| +2457518|AAAAAAAAOKPHFCAA|2016-05-09|1396|6071|466|2016|1|5|9|2|2016|466|6071|Monday|2016Q2|N|N|N|2457510|2457630|2457152|2457427|N|N|N|N|N| +2457519|AAAAAAAAPKPHFCAA|2016-05-10|1396|6072|466|2016|2|5|10|2|2016|466|6072|Tuesday|2016Q2|N|N|N|2457510|2457630|2457153|2457428|N|N|N|N|N| +2457520|AAAAAAAAALPHFCAA|2016-05-11|1396|6072|466|2016|3|5|11|2|2016|466|6072|Wednesday|2016Q2|N|N|N|2457510|2457630|2457154|2457429|N|N|N|N|N| +2457521|AAAAAAAABLPHFCAA|2016-05-12|1396|6072|466|2016|4|5|12|2|2016|466|6072|Thursday|2016Q2|N|N|N|2457510|2457630|2457155|2457430|N|N|N|N|N| +2457522|AAAAAAAACLPHFCAA|2016-05-13|1396|6072|466|2016|5|5|13|2|2016|466|6072|Friday|2016Q2|N|Y|N|2457510|2457630|2457156|2457431|N|N|N|N|N| +2457523|AAAAAAAADLPHFCAA|2016-05-14|1396|6072|466|2016|6|5|14|2|2016|466|6072|Saturday|2016Q2|N|Y|N|2457510|2457630|2457157|2457432|N|N|N|N|N| +2457524|AAAAAAAAELPHFCAA|2016-05-15|1396|6072|466|2016|0|5|15|2|2016|466|6072|Sunday|2016Q2|N|N|N|2457510|2457630|2457158|2457433|N|N|N|N|N| +2457525|AAAAAAAAFLPHFCAA|2016-05-16|1396|6072|466|2016|1|5|16|2|2016|466|6072|Monday|2016Q2|N|N|N|2457510|2457630|2457159|2457434|N|N|N|N|N| +2457526|AAAAAAAAGLPHFCAA|2016-05-17|1396|6073|466|2016|2|5|17|2|2016|466|6073|Tuesday|2016Q2|N|N|N|2457510|2457630|2457160|2457435|N|N|N|N|N| +2457527|AAAAAAAAHLPHFCAA|2016-05-18|1396|6073|466|2016|3|5|18|2|2016|466|6073|Wednesday|2016Q2|N|N|N|2457510|2457630|2457161|2457436|N|N|N|N|N| +2457528|AAAAAAAAILPHFCAA|2016-05-19|1396|6073|466|2016|4|5|19|2|2016|466|6073|Thursday|2016Q2|N|N|N|2457510|2457630|2457162|2457437|N|N|N|N|N| +2457529|AAAAAAAAJLPHFCAA|2016-05-20|1396|6073|466|2016|5|5|20|2|2016|466|6073|Friday|2016Q2|N|Y|N|2457510|2457630|2457163|2457438|N|N|N|N|N| +2457530|AAAAAAAAKLPHFCAA|2016-05-21|1396|6073|466|2016|6|5|21|2|2016|466|6073|Saturday|2016Q2|N|Y|N|2457510|2457630|2457164|2457439|N|N|N|N|N| +2457531|AAAAAAAALLPHFCAA|2016-05-22|1396|6073|466|2016|0|5|22|2|2016|466|6073|Sunday|2016Q2|N|N|N|2457510|2457630|2457165|2457440|N|N|N|N|N| +2457532|AAAAAAAAMLPHFCAA|2016-05-23|1396|6073|466|2016|1|5|23|2|2016|466|6073|Monday|2016Q2|N|N|N|2457510|2457630|2457166|2457441|N|N|N|N|N| +2457533|AAAAAAAANLPHFCAA|2016-05-24|1396|6074|466|2016|2|5|24|2|2016|466|6074|Tuesday|2016Q2|N|N|N|2457510|2457630|2457167|2457442|N|N|N|N|N| +2457534|AAAAAAAAOLPHFCAA|2016-05-25|1396|6074|466|2016|3|5|25|2|2016|466|6074|Wednesday|2016Q2|N|N|N|2457510|2457630|2457168|2457443|N|N|N|N|N| +2457535|AAAAAAAAPLPHFCAA|2016-05-26|1396|6074|466|2016|4|5|26|2|2016|466|6074|Thursday|2016Q2|N|N|N|2457510|2457630|2457169|2457444|N|N|N|N|N| +2457536|AAAAAAAAAMPHFCAA|2016-05-27|1396|6074|466|2016|5|5|27|2|2016|466|6074|Friday|2016Q2|N|Y|N|2457510|2457630|2457170|2457445|N|N|N|N|N| +2457537|AAAAAAAABMPHFCAA|2016-05-28|1396|6074|466|2016|6|5|28|2|2016|466|6074|Saturday|2016Q2|N|Y|N|2457510|2457630|2457171|2457446|N|N|N|N|N| +2457538|AAAAAAAACMPHFCAA|2016-05-29|1396|6074|466|2016|0|5|29|2|2016|466|6074|Sunday|2016Q2|N|N|N|2457510|2457630|2457172|2457447|N|N|N|N|N| +2457539|AAAAAAAADMPHFCAA|2016-05-30|1396|6074|466|2016|1|5|30|2|2016|466|6074|Monday|2016Q2|N|N|N|2457510|2457630|2457173|2457448|N|N|N|N|N| +2457540|AAAAAAAAEMPHFCAA|2016-05-31|1396|6075|466|2016|2|5|31|2|2016|466|6075|Tuesday|2016Q2|N|N|N|2457510|2457630|2457174|2457449|N|N|N|N|N| +2457541|AAAAAAAAFMPHFCAA|2016-06-01|1397|6075|467|2016|3|6|1|2|2016|467|6075|Wednesday|2016Q2|N|N|N|2457541|2457692|2457175|2457450|N|N|N|N|N| +2457542|AAAAAAAAGMPHFCAA|2016-06-02|1397|6075|467|2016|4|6|2|2|2016|467|6075|Thursday|2016Q2|N|N|N|2457541|2457692|2457176|2457451|N|N|N|N|N| +2457543|AAAAAAAAHMPHFCAA|2016-06-03|1397|6075|467|2016|5|6|3|2|2016|467|6075|Friday|2016Q2|N|Y|N|2457541|2457692|2457177|2457452|N|N|N|N|N| +2457544|AAAAAAAAIMPHFCAA|2016-06-04|1397|6075|467|2016|6|6|4|2|2016|467|6075|Saturday|2016Q2|N|Y|N|2457541|2457692|2457178|2457453|N|N|N|N|N| +2457545|AAAAAAAAJMPHFCAA|2016-06-05|1397|6075|467|2016|0|6|5|2|2016|467|6075|Sunday|2016Q2|N|N|N|2457541|2457692|2457179|2457454|N|N|N|N|N| +2457546|AAAAAAAAKMPHFCAA|2016-06-06|1397|6075|467|2016|1|6|6|2|2016|467|6075|Monday|2016Q2|N|N|N|2457541|2457692|2457180|2457455|N|N|N|N|N| +2457547|AAAAAAAALMPHFCAA|2016-06-07|1397|6076|467|2016|2|6|7|2|2016|467|6076|Tuesday|2016Q2|N|N|N|2457541|2457692|2457181|2457456|N|N|N|N|N| +2457548|AAAAAAAAMMPHFCAA|2016-06-08|1397|6076|467|2016|3|6|8|2|2016|467|6076|Wednesday|2016Q2|N|N|N|2457541|2457692|2457182|2457457|N|N|N|N|N| +2457549|AAAAAAAANMPHFCAA|2016-06-09|1397|6076|467|2016|4|6|9|2|2016|467|6076|Thursday|2016Q2|N|N|N|2457541|2457692|2457183|2457458|N|N|N|N|N| +2457550|AAAAAAAAOMPHFCAA|2016-06-10|1397|6076|467|2016|5|6|10|2|2016|467|6076|Friday|2016Q2|N|Y|N|2457541|2457692|2457184|2457459|N|N|N|N|N| +2457551|AAAAAAAAPMPHFCAA|2016-06-11|1397|6076|467|2016|6|6|11|2|2016|467|6076|Saturday|2016Q2|N|Y|N|2457541|2457692|2457185|2457460|N|N|N|N|N| +2457552|AAAAAAAAANPHFCAA|2016-06-12|1397|6076|467|2016|0|6|12|2|2016|467|6076|Sunday|2016Q2|N|N|N|2457541|2457692|2457186|2457461|N|N|N|N|N| +2457553|AAAAAAAABNPHFCAA|2016-06-13|1397|6076|467|2016|1|6|13|2|2016|467|6076|Monday|2016Q2|N|N|N|2457541|2457692|2457187|2457462|N|N|N|N|N| +2457554|AAAAAAAACNPHFCAA|2016-06-14|1397|6077|467|2016|2|6|14|2|2016|467|6077|Tuesday|2016Q2|N|N|N|2457541|2457692|2457188|2457463|N|N|N|N|N| +2457555|AAAAAAAADNPHFCAA|2016-06-15|1397|6077|467|2016|3|6|15|2|2016|467|6077|Wednesday|2016Q2|N|N|N|2457541|2457692|2457189|2457464|N|N|N|N|N| +2457556|AAAAAAAAENPHFCAA|2016-06-16|1397|6077|467|2016|4|6|16|2|2016|467|6077|Thursday|2016Q2|N|N|N|2457541|2457692|2457190|2457465|N|N|N|N|N| +2457557|AAAAAAAAFNPHFCAA|2016-06-17|1397|6077|467|2016|5|6|17|2|2016|467|6077|Friday|2016Q2|N|Y|N|2457541|2457692|2457191|2457466|N|N|N|N|N| +2457558|AAAAAAAAGNPHFCAA|2016-06-18|1397|6077|467|2016|6|6|18|2|2016|467|6077|Saturday|2016Q2|N|Y|N|2457541|2457692|2457192|2457467|N|N|N|N|N| +2457559|AAAAAAAAHNPHFCAA|2016-06-19|1397|6077|467|2016|0|6|19|2|2016|467|6077|Sunday|2016Q2|N|N|N|2457541|2457692|2457193|2457468|N|N|N|N|N| +2457560|AAAAAAAAINPHFCAA|2016-06-20|1397|6077|467|2016|1|6|20|2|2016|467|6077|Monday|2016Q2|N|N|N|2457541|2457692|2457194|2457469|N|N|N|N|N| +2457561|AAAAAAAAJNPHFCAA|2016-06-21|1397|6078|467|2016|2|6|21|2|2016|467|6078|Tuesday|2016Q2|N|N|N|2457541|2457692|2457195|2457470|N|N|N|N|N| +2457562|AAAAAAAAKNPHFCAA|2016-06-22|1397|6078|467|2016|3|6|22|2|2016|467|6078|Wednesday|2016Q2|N|N|N|2457541|2457692|2457196|2457471|N|N|N|N|N| +2457563|AAAAAAAALNPHFCAA|2016-06-23|1397|6078|467|2016|4|6|23|2|2016|467|6078|Thursday|2016Q2|N|N|N|2457541|2457692|2457197|2457472|N|N|N|N|N| +2457564|AAAAAAAAMNPHFCAA|2016-06-24|1397|6078|467|2016|5|6|24|2|2016|467|6078|Friday|2016Q2|N|Y|N|2457541|2457692|2457198|2457473|N|N|N|N|N| +2457565|AAAAAAAANNPHFCAA|2016-06-25|1397|6078|467|2016|6|6|25|2|2016|467|6078|Saturday|2016Q2|N|Y|N|2457541|2457692|2457199|2457474|N|N|N|N|N| +2457566|AAAAAAAAONPHFCAA|2016-06-26|1397|6078|467|2016|0|6|26|2|2016|467|6078|Sunday|2016Q2|N|N|N|2457541|2457692|2457200|2457475|N|N|N|N|N| +2457567|AAAAAAAAPNPHFCAA|2016-06-27|1397|6078|467|2016|1|6|27|2|2016|467|6078|Monday|2016Q2|N|N|N|2457541|2457692|2457201|2457476|N|N|N|N|N| +2457568|AAAAAAAAAOPHFCAA|2016-06-28|1397|6079|467|2016|2|6|28|2|2016|467|6079|Tuesday|2016Q2|N|N|N|2457541|2457692|2457202|2457477|N|N|N|N|N| +2457569|AAAAAAAABOPHFCAA|2016-06-29|1397|6079|467|2016|3|6|29|2|2016|467|6079|Wednesday|2016Q2|N|N|N|2457541|2457692|2457203|2457478|N|N|N|N|N| +2457570|AAAAAAAACOPHFCAA|2016-06-30|1397|6079|467|2016|4|6|30|2|2016|467|6079|Thursday|2016Q2|N|N|N|2457541|2457692|2457204|2457479|N|N|N|N|N| +2457571|AAAAAAAADOPHFCAA|2016-07-01|1398|6079|467|2016|5|7|1|3|2016|467|6079|Friday|2016Q3|N|Y|N|2457571|2457752|2457205|2457480|N|N|N|N|N| +2457572|AAAAAAAAEOPHFCAA|2016-07-02|1398|6079|467|2016|6|7|2|3|2016|467|6079|Saturday|2016Q3|N|Y|N|2457571|2457752|2457206|2457481|N|N|N|N|N| +2457573|AAAAAAAAFOPHFCAA|2016-07-03|1398|6079|467|2016|0|7|3|3|2016|467|6079|Sunday|2016Q3|N|N|N|2457571|2457752|2457207|2457482|N|N|N|N|N| +2457574|AAAAAAAAGOPHFCAA|2016-07-04|1398|6079|467|2016|1|7|4|3|2016|467|6079|Monday|2016Q3|Y|N|N|2457571|2457752|2457208|2457483|N|N|N|N|N| +2457575|AAAAAAAAHOPHFCAA|2016-07-05|1398|6080|467|2016|2|7|5|3|2016|467|6080|Tuesday|2016Q3|N|N|Y|2457571|2457752|2457209|2457484|N|N|N|N|N| +2457576|AAAAAAAAIOPHFCAA|2016-07-06|1398|6080|467|2016|3|7|6|3|2016|467|6080|Wednesday|2016Q3|N|N|N|2457571|2457752|2457210|2457485|N|N|N|N|N| +2457577|AAAAAAAAJOPHFCAA|2016-07-07|1398|6080|467|2016|4|7|7|3|2016|467|6080|Thursday|2016Q3|N|N|N|2457571|2457752|2457211|2457486|N|N|N|N|N| +2457578|AAAAAAAAKOPHFCAA|2016-07-08|1398|6080|467|2016|5|7|8|3|2016|467|6080|Friday|2016Q3|N|Y|N|2457571|2457752|2457212|2457487|N|N|N|N|N| +2457579|AAAAAAAALOPHFCAA|2016-07-09|1398|6080|467|2016|6|7|9|3|2016|467|6080|Saturday|2016Q3|N|Y|N|2457571|2457752|2457213|2457488|N|N|N|N|N| +2457580|AAAAAAAAMOPHFCAA|2016-07-10|1398|6080|467|2016|0|7|10|3|2016|467|6080|Sunday|2016Q3|N|N|N|2457571|2457752|2457214|2457489|N|N|N|N|N| +2457581|AAAAAAAANOPHFCAA|2016-07-11|1398|6080|467|2016|1|7|11|3|2016|467|6080|Monday|2016Q3|N|N|N|2457571|2457752|2457215|2457490|N|N|N|N|N| +2457582|AAAAAAAAOOPHFCAA|2016-07-12|1398|6081|467|2016|2|7|12|3|2016|467|6081|Tuesday|2016Q3|N|N|N|2457571|2457752|2457216|2457491|N|N|N|N|N| +2457583|AAAAAAAAPOPHFCAA|2016-07-13|1398|6081|467|2016|3|7|13|3|2016|467|6081|Wednesday|2016Q3|N|N|N|2457571|2457752|2457217|2457492|N|N|N|N|N| +2457584|AAAAAAAAAPPHFCAA|2016-07-14|1398|6081|467|2016|4|7|14|3|2016|467|6081|Thursday|2016Q3|N|N|N|2457571|2457752|2457218|2457493|N|N|N|N|N| +2457585|AAAAAAAABPPHFCAA|2016-07-15|1398|6081|467|2016|5|7|15|3|2016|467|6081|Friday|2016Q3|N|Y|N|2457571|2457752|2457219|2457494|N|N|N|N|N| +2457586|AAAAAAAACPPHFCAA|2016-07-16|1398|6081|467|2016|6|7|16|3|2016|467|6081|Saturday|2016Q3|N|Y|N|2457571|2457752|2457220|2457495|N|N|N|N|N| +2457587|AAAAAAAADPPHFCAA|2016-07-17|1398|6081|467|2016|0|7|17|3|2016|467|6081|Sunday|2016Q3|N|N|N|2457571|2457752|2457221|2457496|N|N|N|N|N| +2457588|AAAAAAAAEPPHFCAA|2016-07-18|1398|6081|467|2016|1|7|18|3|2016|467|6081|Monday|2016Q3|N|N|N|2457571|2457752|2457222|2457497|N|N|N|N|N| +2457589|AAAAAAAAFPPHFCAA|2016-07-19|1398|6082|467|2016|2|7|19|3|2016|467|6082|Tuesday|2016Q3|N|N|N|2457571|2457752|2457223|2457498|N|N|N|N|N| +2457590|AAAAAAAAGPPHFCAA|2016-07-20|1398|6082|467|2016|3|7|20|3|2016|467|6082|Wednesday|2016Q3|N|N|N|2457571|2457752|2457224|2457499|N|N|N|N|N| +2457591|AAAAAAAAHPPHFCAA|2016-07-21|1398|6082|467|2016|4|7|21|3|2016|467|6082|Thursday|2016Q3|N|N|N|2457571|2457752|2457225|2457500|N|N|N|N|N| +2457592|AAAAAAAAIPPHFCAA|2016-07-22|1398|6082|467|2016|5|7|22|3|2016|467|6082|Friday|2016Q3|N|Y|N|2457571|2457752|2457226|2457501|N|N|N|N|N| +2457593|AAAAAAAAJPPHFCAA|2016-07-23|1398|6082|467|2016|6|7|23|3|2016|467|6082|Saturday|2016Q3|N|Y|N|2457571|2457752|2457227|2457502|N|N|N|N|N| +2457594|AAAAAAAAKPPHFCAA|2016-07-24|1398|6082|467|2016|0|7|24|3|2016|467|6082|Sunday|2016Q3|N|N|N|2457571|2457752|2457228|2457503|N|N|N|N|N| +2457595|AAAAAAAALPPHFCAA|2016-07-25|1398|6082|467|2016|1|7|25|3|2016|467|6082|Monday|2016Q3|N|N|N|2457571|2457752|2457229|2457504|N|N|N|N|N| +2457596|AAAAAAAAMPPHFCAA|2016-07-26|1398|6083|467|2016|2|7|26|3|2016|467|6083|Tuesday|2016Q3|N|N|N|2457571|2457752|2457230|2457505|N|N|N|N|N| +2457597|AAAAAAAANPPHFCAA|2016-07-27|1398|6083|467|2016|3|7|27|3|2016|467|6083|Wednesday|2016Q3|N|N|N|2457571|2457752|2457231|2457506|N|N|N|N|N| +2457598|AAAAAAAAOPPHFCAA|2016-07-28|1398|6083|467|2016|4|7|28|3|2016|467|6083|Thursday|2016Q3|N|N|N|2457571|2457752|2457232|2457507|N|N|N|N|N| +2457599|AAAAAAAAPPPHFCAA|2016-07-29|1398|6083|467|2016|5|7|29|3|2016|467|6083|Friday|2016Q3|N|Y|N|2457571|2457752|2457233|2457508|N|N|N|N|N| +2457600|AAAAAAAAAAAIFCAA|2016-07-30|1398|6083|467|2016|6|7|30|3|2016|467|6083|Saturday|2016Q3|N|Y|N|2457571|2457752|2457234|2457509|N|N|N|N|N| +2457601|AAAAAAAABAAIFCAA|2016-07-31|1398|6083|467|2016|0|7|31|3|2016|467|6083|Sunday|2016Q3|N|N|N|2457571|2457752|2457235|2457510|N|N|N|N|N| +2457602|AAAAAAAACAAIFCAA|2016-08-01|1399|6083|467|2016|1|8|1|3|2016|467|6083|Monday|2016Q3|N|N|N|2457602|2457814|2457236|2457511|N|N|N|N|N| +2457603|AAAAAAAADAAIFCAA|2016-08-02|1399|6084|467|2016|2|8|2|3|2016|467|6084|Tuesday|2016Q3|N|N|N|2457602|2457814|2457237|2457512|N|N|N|N|N| +2457604|AAAAAAAAEAAIFCAA|2016-08-03|1399|6084|467|2016|3|8|3|3|2016|467|6084|Wednesday|2016Q3|N|N|N|2457602|2457814|2457238|2457513|N|N|N|N|N| +2457605|AAAAAAAAFAAIFCAA|2016-08-04|1399|6084|467|2016|4|8|4|3|2016|467|6084|Thursday|2016Q3|N|N|N|2457602|2457814|2457239|2457514|N|N|N|N|N| +2457606|AAAAAAAAGAAIFCAA|2016-08-05|1399|6084|467|2016|5|8|5|3|2016|467|6084|Friday|2016Q3|N|Y|N|2457602|2457814|2457240|2457515|N|N|N|N|N| +2457607|AAAAAAAAHAAIFCAA|2016-08-06|1399|6084|467|2016|6|8|6|3|2016|467|6084|Saturday|2016Q3|N|Y|N|2457602|2457814|2457241|2457516|N|N|N|N|N| +2457608|AAAAAAAAIAAIFCAA|2016-08-07|1399|6084|467|2016|0|8|7|3|2016|467|6084|Sunday|2016Q3|N|N|N|2457602|2457814|2457242|2457517|N|N|N|N|N| +2457609|AAAAAAAAJAAIFCAA|2016-08-08|1399|6084|467|2016|1|8|8|3|2016|467|6084|Monday|2016Q3|N|N|N|2457602|2457814|2457243|2457518|N|N|N|N|N| +2457610|AAAAAAAAKAAIFCAA|2016-08-09|1399|6085|467|2016|2|8|9|3|2016|467|6085|Tuesday|2016Q3|N|N|N|2457602|2457814|2457244|2457519|N|N|N|N|N| +2457611|AAAAAAAALAAIFCAA|2016-08-10|1399|6085|467|2016|3|8|10|3|2016|467|6085|Wednesday|2016Q3|N|N|N|2457602|2457814|2457245|2457520|N|N|N|N|N| +2457612|AAAAAAAAMAAIFCAA|2016-08-11|1399|6085|467|2016|4|8|11|3|2016|467|6085|Thursday|2016Q3|N|N|N|2457602|2457814|2457246|2457521|N|N|N|N|N| +2457613|AAAAAAAANAAIFCAA|2016-08-12|1399|6085|467|2016|5|8|12|3|2016|467|6085|Friday|2016Q3|N|Y|N|2457602|2457814|2457247|2457522|N|N|N|N|N| +2457614|AAAAAAAAOAAIFCAA|2016-08-13|1399|6085|467|2016|6|8|13|3|2016|467|6085|Saturday|2016Q3|N|Y|N|2457602|2457814|2457248|2457523|N|N|N|N|N| +2457615|AAAAAAAAPAAIFCAA|2016-08-14|1399|6085|467|2016|0|8|14|3|2016|467|6085|Sunday|2016Q3|N|N|N|2457602|2457814|2457249|2457524|N|N|N|N|N| +2457616|AAAAAAAAABAIFCAA|2016-08-15|1399|6085|467|2016|1|8|15|3|2016|467|6085|Monday|2016Q3|N|N|N|2457602|2457814|2457250|2457525|N|N|N|N|N| +2457617|AAAAAAAABBAIFCAA|2016-08-16|1399|6086|467|2016|2|8|16|3|2016|467|6086|Tuesday|2016Q3|N|N|N|2457602|2457814|2457251|2457526|N|N|N|N|N| +2457618|AAAAAAAACBAIFCAA|2016-08-17|1399|6086|467|2016|3|8|17|3|2016|467|6086|Wednesday|2016Q3|N|N|N|2457602|2457814|2457252|2457527|N|N|N|N|N| +2457619|AAAAAAAADBAIFCAA|2016-08-18|1399|6086|467|2016|4|8|18|3|2016|467|6086|Thursday|2016Q3|N|N|N|2457602|2457814|2457253|2457528|N|N|N|N|N| +2457620|AAAAAAAAEBAIFCAA|2016-08-19|1399|6086|467|2016|5|8|19|3|2016|467|6086|Friday|2016Q3|N|Y|N|2457602|2457814|2457254|2457529|N|N|N|N|N| +2457621|AAAAAAAAFBAIFCAA|2016-08-20|1399|6086|467|2016|6|8|20|3|2016|467|6086|Saturday|2016Q3|N|Y|N|2457602|2457814|2457255|2457530|N|N|N|N|N| +2457622|AAAAAAAAGBAIFCAA|2016-08-21|1399|6086|467|2016|0|8|21|3|2016|467|6086|Sunday|2016Q3|N|N|N|2457602|2457814|2457256|2457531|N|N|N|N|N| +2457623|AAAAAAAAHBAIFCAA|2016-08-22|1399|6086|467|2016|1|8|22|3|2016|467|6086|Monday|2016Q3|N|N|N|2457602|2457814|2457257|2457532|N|N|N|N|N| +2457624|AAAAAAAAIBAIFCAA|2016-08-23|1399|6087|467|2016|2|8|23|3|2016|467|6087|Tuesday|2016Q3|N|N|N|2457602|2457814|2457258|2457533|N|N|N|N|N| +2457625|AAAAAAAAJBAIFCAA|2016-08-24|1399|6087|467|2016|3|8|24|3|2016|467|6087|Wednesday|2016Q3|N|N|N|2457602|2457814|2457259|2457534|N|N|N|N|N| +2457626|AAAAAAAAKBAIFCAA|2016-08-25|1399|6087|467|2016|4|8|25|3|2016|467|6087|Thursday|2016Q3|N|N|N|2457602|2457814|2457260|2457535|N|N|N|N|N| +2457627|AAAAAAAALBAIFCAA|2016-08-26|1399|6087|467|2016|5|8|26|3|2016|467|6087|Friday|2016Q3|N|Y|N|2457602|2457814|2457261|2457536|N|N|N|N|N| +2457628|AAAAAAAAMBAIFCAA|2016-08-27|1399|6087|467|2016|6|8|27|3|2016|467|6087|Saturday|2016Q3|N|Y|N|2457602|2457814|2457262|2457537|N|N|N|N|N| +2457629|AAAAAAAANBAIFCAA|2016-08-28|1399|6087|467|2016|0|8|28|3|2016|467|6087|Sunday|2016Q3|N|N|N|2457602|2457814|2457263|2457538|N|N|N|N|N| +2457630|AAAAAAAAOBAIFCAA|2016-08-29|1399|6087|467|2016|1|8|29|3|2016|467|6087|Monday|2016Q3|N|N|N|2457602|2457814|2457264|2457539|N|N|N|N|N| +2457631|AAAAAAAAPBAIFCAA|2016-08-30|1399|6088|467|2016|2|8|30|3|2016|467|6088|Tuesday|2016Q3|N|N|N|2457602|2457814|2457265|2457540|N|N|N|N|N| +2457632|AAAAAAAAACAIFCAA|2016-08-31|1399|6088|467|2016|3|8|31|3|2016|467|6088|Wednesday|2016Q3|N|N|N|2457602|2457814|2457266|2457541|N|N|N|N|N| +2457633|AAAAAAAABCAIFCAA|2016-09-01|1400|6088|468|2016|4|9|1|3|2016|468|6088|Thursday|2016Q3|N|N|N|2457633|2457876|2457267|2457542|N|N|N|N|N| +2457634|AAAAAAAACCAIFCAA|2016-09-02|1400|6088|468|2016|5|9|2|3|2016|468|6088|Friday|2016Q3|N|Y|N|2457633|2457876|2457268|2457543|N|N|N|N|N| +2457635|AAAAAAAADCAIFCAA|2016-09-03|1400|6088|468|2016|6|9|3|3|2016|468|6088|Saturday|2016Q3|N|Y|N|2457633|2457876|2457269|2457544|N|N|N|N|N| +2457636|AAAAAAAAECAIFCAA|2016-09-04|1400|6088|468|2016|0|9|4|3|2016|468|6088|Sunday|2016Q3|N|N|N|2457633|2457876|2457270|2457545|N|N|N|N|N| +2457637|AAAAAAAAFCAIFCAA|2016-09-05|1400|6088|468|2016|1|9|5|3|2016|468|6088|Monday|2016Q3|N|N|N|2457633|2457876|2457271|2457546|N|N|N|N|N| +2457638|AAAAAAAAGCAIFCAA|2016-09-06|1400|6089|468|2016|2|9|6|3|2016|468|6089|Tuesday|2016Q3|N|N|N|2457633|2457876|2457272|2457547|N|N|N|N|N| +2457639|AAAAAAAAHCAIFCAA|2016-09-07|1400|6089|468|2016|3|9|7|3|2016|468|6089|Wednesday|2016Q3|N|N|N|2457633|2457876|2457273|2457548|N|N|N|N|N| +2457640|AAAAAAAAICAIFCAA|2016-09-08|1400|6089|468|2016|4|9|8|3|2016|468|6089|Thursday|2016Q3|N|N|N|2457633|2457876|2457274|2457549|N|N|N|N|N| +2457641|AAAAAAAAJCAIFCAA|2016-09-09|1400|6089|468|2016|5|9|9|3|2016|468|6089|Friday|2016Q3|N|Y|N|2457633|2457876|2457275|2457550|N|N|N|N|N| +2457642|AAAAAAAAKCAIFCAA|2016-09-10|1400|6089|468|2016|6|9|10|3|2016|468|6089|Saturday|2016Q3|N|Y|N|2457633|2457876|2457276|2457551|N|N|N|N|N| +2457643|AAAAAAAALCAIFCAA|2016-09-11|1400|6089|468|2016|0|9|11|3|2016|468|6089|Sunday|2016Q3|N|N|N|2457633|2457876|2457277|2457552|N|N|N|N|N| +2457644|AAAAAAAAMCAIFCAA|2016-09-12|1400|6089|468|2016|1|9|12|3|2016|468|6089|Monday|2016Q3|N|N|N|2457633|2457876|2457278|2457553|N|N|N|N|N| +2457645|AAAAAAAANCAIFCAA|2016-09-13|1400|6090|468|2016|2|9|13|3|2016|468|6090|Tuesday|2016Q3|N|N|N|2457633|2457876|2457279|2457554|N|N|N|N|N| +2457646|AAAAAAAAOCAIFCAA|2016-09-14|1400|6090|468|2016|3|9|14|3|2016|468|6090|Wednesday|2016Q3|N|N|N|2457633|2457876|2457280|2457555|N|N|N|N|N| +2457647|AAAAAAAAPCAIFCAA|2016-09-15|1400|6090|468|2016|4|9|15|3|2016|468|6090|Thursday|2016Q3|N|N|N|2457633|2457876|2457281|2457556|N|N|N|N|N| +2457648|AAAAAAAAADAIFCAA|2016-09-16|1400|6090|468|2016|5|9|16|3|2016|468|6090|Friday|2016Q3|N|Y|N|2457633|2457876|2457282|2457557|N|N|N|N|N| +2457649|AAAAAAAABDAIFCAA|2016-09-17|1400|6090|468|2016|6|9|17|3|2016|468|6090|Saturday|2016Q3|N|Y|N|2457633|2457876|2457283|2457558|N|N|N|N|N| +2457650|AAAAAAAACDAIFCAA|2016-09-18|1400|6090|468|2016|0|9|18|3|2016|468|6090|Sunday|2016Q3|N|N|N|2457633|2457876|2457284|2457559|N|N|N|N|N| +2457651|AAAAAAAADDAIFCAA|2016-09-19|1400|6090|468|2016|1|9|19|3|2016|468|6090|Monday|2016Q3|N|N|N|2457633|2457876|2457285|2457560|N|N|N|N|N| +2457652|AAAAAAAAEDAIFCAA|2016-09-20|1400|6091|468|2016|2|9|20|3|2016|468|6091|Tuesday|2016Q3|N|N|N|2457633|2457876|2457286|2457561|N|N|N|N|N| +2457653|AAAAAAAAFDAIFCAA|2016-09-21|1400|6091|468|2016|3|9|21|3|2016|468|6091|Wednesday|2016Q3|N|N|N|2457633|2457876|2457287|2457562|N|N|N|N|N| +2457654|AAAAAAAAGDAIFCAA|2016-09-22|1400|6091|468|2016|4|9|22|3|2016|468|6091|Thursday|2016Q3|N|N|N|2457633|2457876|2457288|2457563|N|N|N|N|N| +2457655|AAAAAAAAHDAIFCAA|2016-09-23|1400|6091|468|2016|5|9|23|3|2016|468|6091|Friday|2016Q3|N|Y|N|2457633|2457876|2457289|2457564|N|N|N|N|N| +2457656|AAAAAAAAIDAIFCAA|2016-09-24|1400|6091|468|2016|6|9|24|3|2016|468|6091|Saturday|2016Q3|N|Y|N|2457633|2457876|2457290|2457565|N|N|N|N|N| +2457657|AAAAAAAAJDAIFCAA|2016-09-25|1400|6091|468|2016|0|9|25|3|2016|468|6091|Sunday|2016Q3|N|N|N|2457633|2457876|2457291|2457566|N|N|N|N|N| +2457658|AAAAAAAAKDAIFCAA|2016-09-26|1400|6091|468|2016|1|9|26|3|2016|468|6091|Monday|2016Q3|N|N|N|2457633|2457876|2457292|2457567|N|N|N|N|N| +2457659|AAAAAAAALDAIFCAA|2016-09-27|1400|6092|468|2016|2|9|27|3|2016|468|6092|Tuesday|2016Q3|N|N|N|2457633|2457876|2457293|2457568|N|N|N|N|N| +2457660|AAAAAAAAMDAIFCAA|2016-09-28|1400|6092|468|2016|3|9|28|3|2016|468|6092|Wednesday|2016Q3|N|N|N|2457633|2457876|2457294|2457569|N|N|N|N|N| +2457661|AAAAAAAANDAIFCAA|2016-09-29|1400|6092|468|2016|4|9|29|3|2016|468|6092|Thursday|2016Q3|N|N|N|2457633|2457876|2457295|2457570|N|N|N|N|N| +2457662|AAAAAAAAODAIFCAA|2016-09-30|1400|6092|468|2016|5|9|30|3|2016|468|6092|Friday|2016Q3|N|Y|N|2457633|2457876|2457296|2457571|N|N|N|N|N| +2457663|AAAAAAAAPDAIFCAA|2016-10-01|1401|6092|468|2016|6|10|1|4|2016|468|6092|Saturday|2016Q4|N|Y|N|2457663|2457936|2457297|2457571|N|N|N|N|N| +2457664|AAAAAAAAAEAIFCAA|2016-10-02|1401|6092|468|2016|0|10|2|4|2016|468|6092|Sunday|2016Q4|N|N|N|2457663|2457936|2457298|2457572|N|N|N|N|N| +2457665|AAAAAAAABEAIFCAA|2016-10-03|1401|6092|468|2016|1|10|3|4|2016|468|6092|Monday|2016Q4|N|N|N|2457663|2457936|2457299|2457573|N|N|N|N|N| +2457666|AAAAAAAACEAIFCAA|2016-10-04|1401|6093|468|2016|2|10|4|4|2016|468|6093|Tuesday|2016Q4|N|N|N|2457663|2457936|2457300|2457574|N|N|N|N|N| +2457667|AAAAAAAADEAIFCAA|2016-10-05|1401|6093|468|2016|3|10|5|4|2016|468|6093|Wednesday|2016Q4|N|N|N|2457663|2457936|2457301|2457575|N|N|N|N|N| +2457668|AAAAAAAAEEAIFCAA|2016-10-06|1401|6093|468|2016|4|10|6|4|2016|468|6093|Thursday|2016Q4|N|N|N|2457663|2457936|2457302|2457576|N|N|N|N|N| +2457669|AAAAAAAAFEAIFCAA|2016-10-07|1401|6093|468|2016|5|10|7|4|2016|468|6093|Friday|2016Q4|N|Y|N|2457663|2457936|2457303|2457577|N|N|N|N|N| +2457670|AAAAAAAAGEAIFCAA|2016-10-08|1401|6093|468|2016|6|10|8|4|2016|468|6093|Saturday|2016Q4|N|Y|N|2457663|2457936|2457304|2457578|N|N|N|N|N| +2457671|AAAAAAAAHEAIFCAA|2016-10-09|1401|6093|468|2016|0|10|9|4|2016|468|6093|Sunday|2016Q4|N|N|N|2457663|2457936|2457305|2457579|N|N|N|N|N| +2457672|AAAAAAAAIEAIFCAA|2016-10-10|1401|6093|468|2016|1|10|10|4|2016|468|6093|Monday|2016Q4|N|N|N|2457663|2457936|2457306|2457580|N|N|N|N|N| +2457673|AAAAAAAAJEAIFCAA|2016-10-11|1401|6094|468|2016|2|10|11|4|2016|468|6094|Tuesday|2016Q4|N|N|N|2457663|2457936|2457307|2457581|N|N|N|N|N| +2457674|AAAAAAAAKEAIFCAA|2016-10-12|1401|6094|468|2016|3|10|12|4|2016|468|6094|Wednesday|2016Q4|N|N|N|2457663|2457936|2457308|2457582|N|N|N|N|N| +2457675|AAAAAAAALEAIFCAA|2016-10-13|1401|6094|468|2016|4|10|13|4|2016|468|6094|Thursday|2016Q4|N|N|N|2457663|2457936|2457309|2457583|N|N|N|N|N| +2457676|AAAAAAAAMEAIFCAA|2016-10-14|1401|6094|468|2016|5|10|14|4|2016|468|6094|Friday|2016Q4|N|Y|N|2457663|2457936|2457310|2457584|N|N|N|N|N| +2457677|AAAAAAAANEAIFCAA|2016-10-15|1401|6094|468|2016|6|10|15|4|2016|468|6094|Saturday|2016Q4|N|Y|N|2457663|2457936|2457311|2457585|N|N|N|N|N| +2457678|AAAAAAAAOEAIFCAA|2016-10-16|1401|6094|468|2016|0|10|16|4|2016|468|6094|Sunday|2016Q4|N|N|N|2457663|2457936|2457312|2457586|N|N|N|N|N| +2457679|AAAAAAAAPEAIFCAA|2016-10-17|1401|6094|468|2016|1|10|17|4|2016|468|6094|Monday|2016Q4|N|N|N|2457663|2457936|2457313|2457587|N|N|N|N|N| +2457680|AAAAAAAAAFAIFCAA|2016-10-18|1401|6095|468|2016|2|10|18|4|2016|468|6095|Tuesday|2016Q4|N|N|N|2457663|2457936|2457314|2457588|N|N|N|N|N| +2457681|AAAAAAAABFAIFCAA|2016-10-19|1401|6095|468|2016|3|10|19|4|2016|468|6095|Wednesday|2016Q4|N|N|N|2457663|2457936|2457315|2457589|N|N|N|N|N| +2457682|AAAAAAAACFAIFCAA|2016-10-20|1401|6095|468|2016|4|10|20|4|2016|468|6095|Thursday|2016Q4|N|N|N|2457663|2457936|2457316|2457590|N|N|N|N|N| +2457683|AAAAAAAADFAIFCAA|2016-10-21|1401|6095|468|2016|5|10|21|4|2016|468|6095|Friday|2016Q4|N|Y|N|2457663|2457936|2457317|2457591|N|N|N|N|N| +2457684|AAAAAAAAEFAIFCAA|2016-10-22|1401|6095|468|2016|6|10|22|4|2016|468|6095|Saturday|2016Q4|N|Y|N|2457663|2457936|2457318|2457592|N|N|N|N|N| +2457685|AAAAAAAAFFAIFCAA|2016-10-23|1401|6095|468|2016|0|10|23|4|2016|468|6095|Sunday|2016Q4|N|N|N|2457663|2457936|2457319|2457593|N|N|N|N|N| +2457686|AAAAAAAAGFAIFCAA|2016-10-24|1401|6095|468|2016|1|10|24|4|2016|468|6095|Monday|2016Q4|N|N|N|2457663|2457936|2457320|2457594|N|N|N|N|N| +2457687|AAAAAAAAHFAIFCAA|2016-10-25|1401|6096|468|2016|2|10|25|4|2016|468|6096|Tuesday|2016Q4|N|N|N|2457663|2457936|2457321|2457595|N|N|N|N|N| +2457688|AAAAAAAAIFAIFCAA|2016-10-26|1401|6096|468|2016|3|10|26|4|2016|468|6096|Wednesday|2016Q4|N|N|N|2457663|2457936|2457322|2457596|N|N|N|N|N| +2457689|AAAAAAAAJFAIFCAA|2016-10-27|1401|6096|468|2016|4|10|27|4|2016|468|6096|Thursday|2016Q4|N|N|N|2457663|2457936|2457323|2457597|N|N|N|N|N| +2457690|AAAAAAAAKFAIFCAA|2016-10-28|1401|6096|468|2016|5|10|28|4|2016|468|6096|Friday|2016Q4|N|Y|N|2457663|2457936|2457324|2457598|N|N|N|N|N| +2457691|AAAAAAAALFAIFCAA|2016-10-29|1401|6096|468|2016|6|10|29|4|2016|468|6096|Saturday|2016Q4|N|Y|N|2457663|2457936|2457325|2457599|N|N|N|N|N| +2457692|AAAAAAAAMFAIFCAA|2016-10-30|1401|6096|468|2016|0|10|30|4|2016|468|6096|Sunday|2016Q4|N|N|N|2457663|2457936|2457326|2457600|N|N|N|N|N| +2457693|AAAAAAAANFAIFCAA|2016-10-31|1401|6096|468|2016|1|10|31|4|2016|468|6096|Monday|2016Q4|N|N|N|2457663|2457936|2457327|2457601|N|N|N|N|N| +2457694|AAAAAAAAOFAIFCAA|2016-11-01|1402|6097|468|2016|2|11|1|4|2016|468|6097|Tuesday|2016Q4|N|N|N|2457694|2457998|2457328|2457602|N|N|N|N|N| +2457695|AAAAAAAAPFAIFCAA|2016-11-02|1402|6097|468|2016|3|11|2|4|2016|468|6097|Wednesday|2016Q4|N|N|N|2457694|2457998|2457329|2457603|N|N|N|N|N| +2457696|AAAAAAAAAGAIFCAA|2016-11-03|1402|6097|468|2016|4|11|3|4|2016|468|6097|Thursday|2016Q4|N|N|N|2457694|2457998|2457330|2457604|N|N|N|N|N| +2457697|AAAAAAAABGAIFCAA|2016-11-04|1402|6097|468|2016|5|11|4|4|2016|468|6097|Friday|2016Q4|N|Y|N|2457694|2457998|2457331|2457605|N|N|N|N|N| +2457698|AAAAAAAACGAIFCAA|2016-11-05|1402|6097|468|2016|6|11|5|4|2016|468|6097|Saturday|2016Q4|N|Y|N|2457694|2457998|2457332|2457606|N|N|N|N|N| +2457699|AAAAAAAADGAIFCAA|2016-11-06|1402|6097|468|2016|0|11|6|4|2016|468|6097|Sunday|2016Q4|N|N|N|2457694|2457998|2457333|2457607|N|N|N|N|N| +2457700|AAAAAAAAEGAIFCAA|2016-11-07|1402|6097|468|2016|1|11|7|4|2016|468|6097|Monday|2016Q4|N|N|N|2457694|2457998|2457334|2457608|N|N|N|N|N| +2457701|AAAAAAAAFGAIFCAA|2016-11-08|1402|6098|468|2016|2|11|8|4|2016|468|6098|Tuesday|2016Q4|N|N|N|2457694|2457998|2457335|2457609|N|N|N|N|N| +2457702|AAAAAAAAGGAIFCAA|2016-11-09|1402|6098|468|2016|3|11|9|4|2016|468|6098|Wednesday|2016Q4|N|N|N|2457694|2457998|2457336|2457610|N|N|N|N|N| +2457703|AAAAAAAAHGAIFCAA|2016-11-10|1402|6098|468|2016|4|11|10|4|2016|468|6098|Thursday|2016Q4|N|N|N|2457694|2457998|2457337|2457611|N|N|N|N|N| +2457704|AAAAAAAAIGAIFCAA|2016-11-11|1402|6098|468|2016|5|11|11|4|2016|468|6098|Friday|2016Q4|N|Y|N|2457694|2457998|2457338|2457612|N|N|N|N|N| +2457705|AAAAAAAAJGAIFCAA|2016-11-12|1402|6098|468|2016|6|11|12|4|2016|468|6098|Saturday|2016Q4|N|Y|N|2457694|2457998|2457339|2457613|N|N|N|N|N| +2457706|AAAAAAAAKGAIFCAA|2016-11-13|1402|6098|468|2016|0|11|13|4|2016|468|6098|Sunday|2016Q4|N|N|N|2457694|2457998|2457340|2457614|N|N|N|N|N| +2457707|AAAAAAAALGAIFCAA|2016-11-14|1402|6098|468|2016|1|11|14|4|2016|468|6098|Monday|2016Q4|N|N|N|2457694|2457998|2457341|2457615|N|N|N|N|N| +2457708|AAAAAAAAMGAIFCAA|2016-11-15|1402|6099|468|2016|2|11|15|4|2016|468|6099|Tuesday|2016Q4|N|N|N|2457694|2457998|2457342|2457616|N|N|N|N|N| +2457709|AAAAAAAANGAIFCAA|2016-11-16|1402|6099|468|2016|3|11|16|4|2016|468|6099|Wednesday|2016Q4|N|N|N|2457694|2457998|2457343|2457617|N|N|N|N|N| +2457710|AAAAAAAAOGAIFCAA|2016-11-17|1402|6099|468|2016|4|11|17|4|2016|468|6099|Thursday|2016Q4|N|N|N|2457694|2457998|2457344|2457618|N|N|N|N|N| +2457711|AAAAAAAAPGAIFCAA|2016-11-18|1402|6099|468|2016|5|11|18|4|2016|468|6099|Friday|2016Q4|N|Y|N|2457694|2457998|2457345|2457619|N|N|N|N|N| +2457712|AAAAAAAAAHAIFCAA|2016-11-19|1402|6099|468|2016|6|11|19|4|2016|468|6099|Saturday|2016Q4|N|Y|N|2457694|2457998|2457346|2457620|N|N|N|N|N| +2457713|AAAAAAAABHAIFCAA|2016-11-20|1402|6099|468|2016|0|11|20|4|2016|468|6099|Sunday|2016Q4|N|N|N|2457694|2457998|2457347|2457621|N|N|N|N|N| +2457714|AAAAAAAACHAIFCAA|2016-11-21|1402|6099|468|2016|1|11|21|4|2016|468|6099|Monday|2016Q4|N|N|N|2457694|2457998|2457348|2457622|N|N|N|N|N| +2457715|AAAAAAAADHAIFCAA|2016-11-22|1402|6100|468|2016|2|11|22|4|2016|468|6100|Tuesday|2016Q4|N|N|N|2457694|2457998|2457349|2457623|N|N|N|N|N| +2457716|AAAAAAAAEHAIFCAA|2016-11-23|1402|6100|468|2016|3|11|23|4|2016|468|6100|Wednesday|2016Q4|N|N|N|2457694|2457998|2457350|2457624|N|N|N|N|N| +2457717|AAAAAAAAFHAIFCAA|2016-11-24|1402|6100|468|2016|4|11|24|4|2016|468|6100|Thursday|2016Q4|N|N|N|2457694|2457998|2457351|2457625|N|N|N|N|N| +2457718|AAAAAAAAGHAIFCAA|2016-11-25|1402|6100|468|2016|5|11|25|4|2016|468|6100|Friday|2016Q4|N|Y|N|2457694|2457998|2457352|2457626|N|N|N|N|N| +2457719|AAAAAAAAHHAIFCAA|2016-11-26|1402|6100|468|2016|6|11|26|4|2016|468|6100|Saturday|2016Q4|N|Y|N|2457694|2457998|2457353|2457627|N|N|N|N|N| +2457720|AAAAAAAAIHAIFCAA|2016-11-27|1402|6100|468|2016|0|11|27|4|2016|468|6100|Sunday|2016Q4|N|N|N|2457694|2457998|2457354|2457628|N|N|N|N|N| +2457721|AAAAAAAAJHAIFCAA|2016-11-28|1402|6100|468|2016|1|11|28|4|2016|468|6100|Monday|2016Q4|N|N|N|2457694|2457998|2457355|2457629|N|N|N|N|N| +2457722|AAAAAAAAKHAIFCAA|2016-11-29|1402|6101|468|2016|2|11|29|4|2016|468|6101|Tuesday|2016Q4|N|N|N|2457694|2457998|2457356|2457630|N|N|N|N|N| +2457723|AAAAAAAALHAIFCAA|2016-11-30|1402|6101|468|2016|3|11|30|4|2016|468|6101|Wednesday|2016Q4|N|N|N|2457694|2457998|2457357|2457631|N|N|N|N|N| +2457724|AAAAAAAAMHAIFCAA|2016-12-01|1403|6101|469|2016|4|12|1|4|2016|469|6101|Thursday|2016Q4|N|N|N|2457724|2458058|2457358|2457632|N|N|N|N|N| +2457725|AAAAAAAANHAIFCAA|2016-12-02|1403|6101|469|2016|5|12|2|4|2016|469|6101|Friday|2016Q4|N|Y|N|2457724|2458058|2457359|2457633|N|N|N|N|N| +2457726|AAAAAAAAOHAIFCAA|2016-12-03|1403|6101|469|2016|6|12|3|4|2016|469|6101|Saturday|2016Q4|N|Y|N|2457724|2458058|2457360|2457634|N|N|N|N|N| +2457727|AAAAAAAAPHAIFCAA|2016-12-04|1403|6101|469|2016|0|12|4|4|2016|469|6101|Sunday|2016Q4|N|N|N|2457724|2458058|2457361|2457635|N|N|N|N|N| +2457728|AAAAAAAAAIAIFCAA|2016-12-05|1403|6101|469|2016|1|12|5|4|2016|469|6101|Monday|2016Q4|N|N|N|2457724|2458058|2457362|2457636|N|N|N|N|N| +2457729|AAAAAAAABIAIFCAA|2016-12-06|1403|6102|469|2016|2|12|6|4|2016|469|6102|Tuesday|2016Q4|N|N|N|2457724|2458058|2457363|2457637|N|N|N|N|N| +2457730|AAAAAAAACIAIFCAA|2016-12-07|1403|6102|469|2016|3|12|7|4|2016|469|6102|Wednesday|2016Q4|N|N|N|2457724|2458058|2457364|2457638|N|N|N|N|N| +2457731|AAAAAAAADIAIFCAA|2016-12-08|1403|6102|469|2016|4|12|8|4|2016|469|6102|Thursday|2016Q4|N|N|N|2457724|2458058|2457365|2457639|N|N|N|N|N| +2457732|AAAAAAAAEIAIFCAA|2016-12-09|1403|6102|469|2016|5|12|9|4|2016|469|6102|Friday|2016Q4|N|Y|N|2457724|2458058|2457366|2457640|N|N|N|N|N| +2457733|AAAAAAAAFIAIFCAA|2016-12-10|1403|6102|469|2016|6|12|10|4|2016|469|6102|Saturday|2016Q4|N|Y|N|2457724|2458058|2457367|2457641|N|N|N|N|N| +2457734|AAAAAAAAGIAIFCAA|2016-12-11|1403|6102|469|2016|0|12|11|4|2016|469|6102|Sunday|2016Q4|N|N|N|2457724|2458058|2457368|2457642|N|N|N|N|N| +2457735|AAAAAAAAHIAIFCAA|2016-12-12|1403|6102|469|2016|1|12|12|4|2016|469|6102|Monday|2016Q4|N|N|N|2457724|2458058|2457369|2457643|N|N|N|N|N| +2457736|AAAAAAAAIIAIFCAA|2016-12-13|1403|6103|469|2016|2|12|13|4|2016|469|6103|Tuesday|2016Q4|N|N|N|2457724|2458058|2457370|2457644|N|N|N|N|N| +2457737|AAAAAAAAJIAIFCAA|2016-12-14|1403|6103|469|2016|3|12|14|4|2016|469|6103|Wednesday|2016Q4|N|N|N|2457724|2458058|2457371|2457645|N|N|N|N|N| +2457738|AAAAAAAAKIAIFCAA|2016-12-15|1403|6103|469|2016|4|12|15|4|2016|469|6103|Thursday|2016Q4|N|N|N|2457724|2458058|2457372|2457646|N|N|N|N|N| +2457739|AAAAAAAALIAIFCAA|2016-12-16|1403|6103|469|2016|5|12|16|4|2016|469|6103|Friday|2016Q4|N|Y|N|2457724|2458058|2457373|2457647|N|N|N|N|N| +2457740|AAAAAAAAMIAIFCAA|2016-12-17|1403|6103|469|2016|6|12|17|4|2016|469|6103|Saturday|2016Q4|N|Y|N|2457724|2458058|2457374|2457648|N|N|N|N|N| +2457741|AAAAAAAANIAIFCAA|2016-12-18|1403|6103|469|2016|0|12|18|4|2016|469|6103|Sunday|2016Q4|N|N|N|2457724|2458058|2457375|2457649|N|N|N|N|N| +2457742|AAAAAAAAOIAIFCAA|2016-12-19|1403|6103|469|2016|1|12|19|4|2016|469|6103|Monday|2016Q4|N|N|N|2457724|2458058|2457376|2457650|N|N|N|N|N| +2457743|AAAAAAAAPIAIFCAA|2016-12-20|1403|6104|469|2016|2|12|20|4|2016|469|6104|Tuesday|2016Q4|N|N|N|2457724|2458058|2457377|2457651|N|N|N|N|N| +2457744|AAAAAAAAAJAIFCAA|2016-12-21|1403|6104|469|2016|3|12|21|4|2016|469|6104|Wednesday|2016Q4|N|N|N|2457724|2458058|2457378|2457652|N|N|N|N|N| +2457745|AAAAAAAABJAIFCAA|2016-12-22|1403|6104|469|2016|4|12|22|4|2016|469|6104|Thursday|2016Q4|N|N|N|2457724|2458058|2457379|2457653|N|N|N|N|N| +2457746|AAAAAAAACJAIFCAA|2016-12-23|1403|6104|469|2016|5|12|23|4|2016|469|6104|Friday|2016Q4|N|Y|N|2457724|2458058|2457380|2457654|N|N|N|N|N| +2457747|AAAAAAAADJAIFCAA|2016-12-24|1403|6104|469|2016|6|12|24|4|2016|469|6104|Saturday|2016Q4|N|Y|N|2457724|2458058|2457381|2457655|N|N|N|N|N| +2457748|AAAAAAAAEJAIFCAA|2016-12-25|1403|6104|469|2016|0|12|25|4|2016|469|6104|Sunday|2016Q4|Y|N|N|2457724|2458058|2457382|2457656|N|N|N|N|N| +2457749|AAAAAAAAFJAIFCAA|2016-12-26|1403|6104|469|2016|1|12|26|4|2016|469|6104|Monday|2016Q4|N|N|Y|2457724|2458058|2457383|2457657|N|N|N|N|N| +2457750|AAAAAAAAGJAIFCAA|2016-12-27|1403|6105|469|2016|2|12|27|4|2016|469|6105|Tuesday|2016Q4|N|N|N|2457724|2458058|2457384|2457658|N|N|N|N|N| +2457751|AAAAAAAAHJAIFCAA|2016-12-28|1403|6105|469|2016|3|12|28|4|2016|469|6105|Wednesday|2016Q4|N|N|N|2457724|2458058|2457385|2457659|N|N|N|N|N| +2457752|AAAAAAAAIJAIFCAA|2016-12-29|1403|6105|469|2016|4|12|29|4|2016|469|6105|Thursday|2016Q4|N|N|N|2457724|2458058|2457386|2457660|N|N|N|N|N| +2457753|AAAAAAAAJJAIFCAA|2016-12-30|1403|6105|469|2016|5|12|30|4|2016|469|6105|Friday|2016Q4|N|Y|N|2457724|2458058|2457387|2457661|N|N|N|N|N| +2457754|AAAAAAAAKJAIFCAA|2016-12-31|1403|6105|469|2016|6|12|31|4|2016|469|6105|Saturday|2016Q4|Y|Y|N|2457724|2458058|2457388|2457662|N|N|N|N|N| +2457755|AAAAAAAALJAIFCAA|2017-01-01|1404|6105|469|2017|0|1|1|1|2017|469|6105|Sunday|2017Q1|Y|N|Y|2457755|2457754|2457389|2457663|N|N|N|N|N| +2457756|AAAAAAAAMJAIFCAA|2017-01-02|1404|6105|469|2017|1|1|2|1|2017|469|6105|Monday|2017Q1|N|N|Y|2457755|2457754|2457390|2457664|N|N|N|N|N| +2457757|AAAAAAAANJAIFCAA|2017-01-03|1404|6106|469|2017|2|1|3|1|2017|469|6106|Tuesday|2017Q1|N|N|N|2457755|2457754|2457391|2457665|N|N|N|N|N| +2457758|AAAAAAAAOJAIFCAA|2017-01-04|1404|6106|469|2017|3|1|4|1|2017|469|6106|Wednesday|2017Q1|N|N|N|2457755|2457754|2457392|2457666|N|N|N|N|N| +2457759|AAAAAAAAPJAIFCAA|2017-01-05|1404|6106|469|2017|4|1|5|1|2017|469|6106|Thursday|2017Q1|N|N|N|2457755|2457754|2457393|2457667|N|N|N|N|N| +2457760|AAAAAAAAAKAIFCAA|2017-01-06|1404|6106|469|2017|5|1|6|1|2017|469|6106|Friday|2017Q1|N|Y|N|2457755|2457754|2457394|2457668|N|N|N|N|N| +2457761|AAAAAAAABKAIFCAA|2017-01-07|1404|6106|469|2017|6|1|7|1|2017|469|6106|Saturday|2017Q1|N|Y|N|2457755|2457754|2457395|2457669|N|N|N|N|N| +2457762|AAAAAAAACKAIFCAA|2017-01-08|1404|6106|469|2017|0|1|8|1|2017|469|6106|Sunday|2017Q1|N|N|N|2457755|2457754|2457396|2457670|N|N|N|N|N| +2457763|AAAAAAAADKAIFCAA|2017-01-09|1404|6106|469|2017|1|1|9|1|2017|469|6106|Monday|2017Q1|N|N|N|2457755|2457754|2457397|2457671|N|N|N|N|N| +2457764|AAAAAAAAEKAIFCAA|2017-01-10|1404|6107|469|2017|2|1|10|1|2017|469|6107|Tuesday|2017Q1|N|N|N|2457755|2457754|2457398|2457672|N|N|N|N|N| +2457765|AAAAAAAAFKAIFCAA|2017-01-11|1404|6107|469|2017|3|1|11|1|2017|469|6107|Wednesday|2017Q1|N|N|N|2457755|2457754|2457399|2457673|N|N|N|N|N| +2457766|AAAAAAAAGKAIFCAA|2017-01-12|1404|6107|469|2017|4|1|12|1|2017|469|6107|Thursday|2017Q1|N|N|N|2457755|2457754|2457400|2457674|N|N|N|N|N| +2457767|AAAAAAAAHKAIFCAA|2017-01-13|1404|6107|469|2017|5|1|13|1|2017|469|6107|Friday|2017Q1|N|Y|N|2457755|2457754|2457401|2457675|N|N|N|N|N| +2457768|AAAAAAAAIKAIFCAA|2017-01-14|1404|6107|469|2017|6|1|14|1|2017|469|6107|Saturday|2017Q1|N|Y|N|2457755|2457754|2457402|2457676|N|N|N|N|N| +2457769|AAAAAAAAJKAIFCAA|2017-01-15|1404|6107|469|2017|0|1|15|1|2017|469|6107|Sunday|2017Q1|N|N|N|2457755|2457754|2457403|2457677|N|N|N|N|N| +2457770|AAAAAAAAKKAIFCAA|2017-01-16|1404|6107|469|2017|1|1|16|1|2017|469|6107|Monday|2017Q1|N|N|N|2457755|2457754|2457404|2457678|N|N|N|N|N| +2457771|AAAAAAAALKAIFCAA|2017-01-17|1404|6108|469|2017|2|1|17|1|2017|469|6108|Tuesday|2017Q1|N|N|N|2457755|2457754|2457405|2457679|N|N|N|N|N| +2457772|AAAAAAAAMKAIFCAA|2017-01-18|1404|6108|469|2017|3|1|18|1|2017|469|6108|Wednesday|2017Q1|N|N|N|2457755|2457754|2457406|2457680|N|N|N|N|N| +2457773|AAAAAAAANKAIFCAA|2017-01-19|1404|6108|469|2017|4|1|19|1|2017|469|6108|Thursday|2017Q1|N|N|N|2457755|2457754|2457407|2457681|N|N|N|N|N| +2457774|AAAAAAAAOKAIFCAA|2017-01-20|1404|6108|469|2017|5|1|20|1|2017|469|6108|Friday|2017Q1|N|Y|N|2457755|2457754|2457408|2457682|N|N|N|N|N| +2457775|AAAAAAAAPKAIFCAA|2017-01-21|1404|6108|469|2017|6|1|21|1|2017|469|6108|Saturday|2017Q1|N|Y|N|2457755|2457754|2457409|2457683|N|N|N|N|N| +2457776|AAAAAAAAALAIFCAA|2017-01-22|1404|6108|469|2017|0|1|22|1|2017|469|6108|Sunday|2017Q1|N|N|N|2457755|2457754|2457410|2457684|N|N|N|N|N| +2457777|AAAAAAAABLAIFCAA|2017-01-23|1404|6108|469|2017|1|1|23|1|2017|469|6108|Monday|2017Q1|N|N|N|2457755|2457754|2457411|2457685|N|N|N|N|N| +2457778|AAAAAAAACLAIFCAA|2017-01-24|1404|6109|469|2017|2|1|24|1|2017|469|6109|Tuesday|2017Q1|N|N|N|2457755|2457754|2457412|2457686|N|N|N|N|N| +2457779|AAAAAAAADLAIFCAA|2017-01-25|1404|6109|469|2017|3|1|25|1|2017|469|6109|Wednesday|2017Q1|N|N|N|2457755|2457754|2457413|2457687|N|N|N|N|N| +2457780|AAAAAAAAELAIFCAA|2017-01-26|1404|6109|469|2017|4|1|26|1|2017|469|6109|Thursday|2017Q1|N|N|N|2457755|2457754|2457414|2457688|N|N|N|N|N| +2457781|AAAAAAAAFLAIFCAA|2017-01-27|1404|6109|469|2017|5|1|27|1|2017|469|6109|Friday|2017Q1|N|Y|N|2457755|2457754|2457415|2457689|N|N|N|N|N| +2457782|AAAAAAAAGLAIFCAA|2017-01-28|1404|6109|469|2017|6|1|28|1|2017|469|6109|Saturday|2017Q1|N|Y|N|2457755|2457754|2457416|2457690|N|N|N|N|N| +2457783|AAAAAAAAHLAIFCAA|2017-01-29|1404|6109|469|2017|0|1|29|1|2017|469|6109|Sunday|2017Q1|N|N|N|2457755|2457754|2457417|2457691|N|N|N|N|N| +2457784|AAAAAAAAILAIFCAA|2017-01-30|1404|6109|469|2017|1|1|30|1|2017|469|6109|Monday|2017Q1|N|N|N|2457755|2457754|2457418|2457692|N|N|N|N|N| +2457785|AAAAAAAAJLAIFCAA|2017-01-31|1404|6110|469|2017|2|1|31|1|2017|469|6110|Tuesday|2017Q1|N|N|N|2457755|2457754|2457419|2457693|N|N|N|N|N| +2457786|AAAAAAAAKLAIFCAA|2017-02-01|1405|6110|469|2017|3|2|1|1|2017|469|6110|Wednesday|2017Q1|N|N|N|2457786|2457816|2457420|2457694|N|N|N|N|N| +2457787|AAAAAAAALLAIFCAA|2017-02-02|1405|6110|469|2017|4|2|2|1|2017|469|6110|Thursday|2017Q1|N|N|N|2457786|2457816|2457421|2457695|N|N|N|N|N| +2457788|AAAAAAAAMLAIFCAA|2017-02-03|1405|6110|469|2017|5|2|3|1|2017|469|6110|Friday|2017Q1|N|Y|N|2457786|2457816|2457422|2457696|N|N|N|N|N| +2457789|AAAAAAAANLAIFCAA|2017-02-04|1405|6110|469|2017|6|2|4|1|2017|469|6110|Saturday|2017Q1|N|Y|N|2457786|2457816|2457423|2457697|N|N|N|N|N| +2457790|AAAAAAAAOLAIFCAA|2017-02-05|1405|6110|469|2017|0|2|5|1|2017|469|6110|Sunday|2017Q1|N|N|N|2457786|2457816|2457424|2457698|N|N|N|N|N| +2457791|AAAAAAAAPLAIFCAA|2017-02-06|1405|6110|469|2017|1|2|6|1|2017|469|6110|Monday|2017Q1|N|N|N|2457786|2457816|2457425|2457699|N|N|N|N|N| +2457792|AAAAAAAAAMAIFCAA|2017-02-07|1405|6111|469|2017|2|2|7|1|2017|469|6111|Tuesday|2017Q1|N|N|N|2457786|2457816|2457426|2457700|N|N|N|N|N| +2457793|AAAAAAAABMAIFCAA|2017-02-08|1405|6111|469|2017|3|2|8|1|2017|469|6111|Wednesday|2017Q1|N|N|N|2457786|2457816|2457427|2457701|N|N|N|N|N| +2457794|AAAAAAAACMAIFCAA|2017-02-09|1405|6111|469|2017|4|2|9|1|2017|469|6111|Thursday|2017Q1|N|N|N|2457786|2457816|2457428|2457702|N|N|N|N|N| +2457795|AAAAAAAADMAIFCAA|2017-02-10|1405|6111|469|2017|5|2|10|1|2017|469|6111|Friday|2017Q1|N|Y|N|2457786|2457816|2457429|2457703|N|N|N|N|N| +2457796|AAAAAAAAEMAIFCAA|2017-02-11|1405|6111|469|2017|6|2|11|1|2017|469|6111|Saturday|2017Q1|N|Y|N|2457786|2457816|2457430|2457704|N|N|N|N|N| +2457797|AAAAAAAAFMAIFCAA|2017-02-12|1405|6111|469|2017|0|2|12|1|2017|469|6111|Sunday|2017Q1|N|N|N|2457786|2457816|2457431|2457705|N|N|N|N|N| +2457798|AAAAAAAAGMAIFCAA|2017-02-13|1405|6111|469|2017|1|2|13|1|2017|469|6111|Monday|2017Q1|N|N|N|2457786|2457816|2457432|2457706|N|N|N|N|N| +2457799|AAAAAAAAHMAIFCAA|2017-02-14|1405|6112|469|2017|2|2|14|1|2017|469|6112|Tuesday|2017Q1|N|N|N|2457786|2457816|2457433|2457707|N|N|N|N|N| +2457800|AAAAAAAAIMAIFCAA|2017-02-15|1405|6112|469|2017|3|2|15|1|2017|469|6112|Wednesday|2017Q1|N|N|N|2457786|2457816|2457434|2457708|N|N|N|N|N| +2457801|AAAAAAAAJMAIFCAA|2017-02-16|1405|6112|469|2017|4|2|16|1|2017|469|6112|Thursday|2017Q1|N|N|N|2457786|2457816|2457435|2457709|N|N|N|N|N| +2457802|AAAAAAAAKMAIFCAA|2017-02-17|1405|6112|469|2017|5|2|17|1|2017|469|6112|Friday|2017Q1|N|Y|N|2457786|2457816|2457436|2457710|N|N|N|N|N| +2457803|AAAAAAAALMAIFCAA|2017-02-18|1405|6112|469|2017|6|2|18|1|2017|469|6112|Saturday|2017Q1|N|Y|N|2457786|2457816|2457437|2457711|N|N|N|N|N| +2457804|AAAAAAAAMMAIFCAA|2017-02-19|1405|6112|469|2017|0|2|19|1|2017|469|6112|Sunday|2017Q1|N|N|N|2457786|2457816|2457438|2457712|N|N|N|N|N| +2457805|AAAAAAAANMAIFCAA|2017-02-20|1405|6112|469|2017|1|2|20|1|2017|469|6112|Monday|2017Q1|N|N|N|2457786|2457816|2457439|2457713|N|N|N|N|N| +2457806|AAAAAAAAOMAIFCAA|2017-02-21|1405|6113|469|2017|2|2|21|1|2017|469|6113|Tuesday|2017Q1|N|N|N|2457786|2457816|2457440|2457714|N|N|N|N|N| +2457807|AAAAAAAAPMAIFCAA|2017-02-22|1405|6113|469|2017|3|2|22|1|2017|469|6113|Wednesday|2017Q1|N|N|N|2457786|2457816|2457441|2457715|N|N|N|N|N| +2457808|AAAAAAAAANAIFCAA|2017-02-23|1405|6113|469|2017|4|2|23|1|2017|469|6113|Thursday|2017Q1|N|N|N|2457786|2457816|2457442|2457716|N|N|N|N|N| +2457809|AAAAAAAABNAIFCAA|2017-02-24|1405|6113|469|2017|5|2|24|1|2017|469|6113|Friday|2017Q1|N|Y|N|2457786|2457816|2457443|2457717|N|N|N|N|N| +2457810|AAAAAAAACNAIFCAA|2017-02-25|1405|6113|469|2017|6|2|25|1|2017|469|6113|Saturday|2017Q1|N|Y|N|2457786|2457816|2457444|2457718|N|N|N|N|N| +2457811|AAAAAAAADNAIFCAA|2017-02-26|1405|6113|469|2017|0|2|26|1|2017|469|6113|Sunday|2017Q1|N|N|N|2457786|2457816|2457445|2457719|N|N|N|N|N| +2457812|AAAAAAAAENAIFCAA|2017-02-27|1405|6113|469|2017|1|2|27|1|2017|469|6113|Monday|2017Q1|N|N|N|2457786|2457816|2457446|2457720|N|N|N|N|N| +2457813|AAAAAAAAFNAIFCAA|2017-02-28|1405|6114|469|2017|2|2|28|1|2017|469|6114|Tuesday|2017Q1|N|N|N|2457786|2457816|2457447|2457721|N|N|N|N|N| +2457814|AAAAAAAAGNAIFCAA|2017-03-01|1406|6114|470|2017|3|3|1|1|2017|470|6114|Wednesday|2017Q1|N|N|N|2457814|2457872|2457449|2457722|N|N|N|N|N| +2457815|AAAAAAAAHNAIFCAA|2017-03-02|1406|6114|470|2017|4|3|2|1|2017|470|6114|Thursday|2017Q1|N|N|N|2457814|2457872|2457450|2457723|N|N|N|N|N| +2457816|AAAAAAAAINAIFCAA|2017-03-03|1406|6114|470|2017|5|3|3|1|2017|470|6114|Friday|2017Q1|N|Y|N|2457814|2457872|2457451|2457724|N|N|N|N|N| +2457817|AAAAAAAAJNAIFCAA|2017-03-04|1406|6114|470|2017|6|3|4|1|2017|470|6114|Saturday|2017Q1|N|Y|N|2457814|2457872|2457452|2457725|N|N|N|N|N| +2457818|AAAAAAAAKNAIFCAA|2017-03-05|1406|6114|470|2017|0|3|5|1|2017|470|6114|Sunday|2017Q1|N|N|N|2457814|2457872|2457453|2457726|N|N|N|N|N| +2457819|AAAAAAAALNAIFCAA|2017-03-06|1406|6114|470|2017|1|3|6|1|2017|470|6114|Monday|2017Q1|N|N|N|2457814|2457872|2457454|2457727|N|N|N|N|N| +2457820|AAAAAAAAMNAIFCAA|2017-03-07|1406|6115|470|2017|2|3|7|1|2017|470|6115|Tuesday|2017Q1|N|N|N|2457814|2457872|2457455|2457728|N|N|N|N|N| +2457821|AAAAAAAANNAIFCAA|2017-03-08|1406|6115|470|2017|3|3|8|1|2017|470|6115|Wednesday|2017Q1|N|N|N|2457814|2457872|2457456|2457729|N|N|N|N|N| +2457822|AAAAAAAAONAIFCAA|2017-03-09|1406|6115|470|2017|4|3|9|1|2017|470|6115|Thursday|2017Q1|N|N|N|2457814|2457872|2457457|2457730|N|N|N|N|N| +2457823|AAAAAAAAPNAIFCAA|2017-03-10|1406|6115|470|2017|5|3|10|1|2017|470|6115|Friday|2017Q1|N|Y|N|2457814|2457872|2457458|2457731|N|N|N|N|N| +2457824|AAAAAAAAAOAIFCAA|2017-03-11|1406|6115|470|2017|6|3|11|1|2017|470|6115|Saturday|2017Q1|N|Y|N|2457814|2457872|2457459|2457732|N|N|N|N|N| +2457825|AAAAAAAABOAIFCAA|2017-03-12|1406|6115|470|2017|0|3|12|1|2017|470|6115|Sunday|2017Q1|N|N|N|2457814|2457872|2457460|2457733|N|N|N|N|N| +2457826|AAAAAAAACOAIFCAA|2017-03-13|1406|6115|470|2017|1|3|13|1|2017|470|6115|Monday|2017Q1|N|N|N|2457814|2457872|2457461|2457734|N|N|N|N|N| +2457827|AAAAAAAADOAIFCAA|2017-03-14|1406|6116|470|2017|2|3|14|1|2017|470|6116|Tuesday|2017Q1|N|N|N|2457814|2457872|2457462|2457735|N|N|N|N|N| +2457828|AAAAAAAAEOAIFCAA|2017-03-15|1406|6116|470|2017|3|3|15|1|2017|470|6116|Wednesday|2017Q1|N|N|N|2457814|2457872|2457463|2457736|N|N|N|N|N| +2457829|AAAAAAAAFOAIFCAA|2017-03-16|1406|6116|470|2017|4|3|16|1|2017|470|6116|Thursday|2017Q1|N|N|N|2457814|2457872|2457464|2457737|N|N|N|N|N| +2457830|AAAAAAAAGOAIFCAA|2017-03-17|1406|6116|470|2017|5|3|17|1|2017|470|6116|Friday|2017Q1|N|Y|N|2457814|2457872|2457465|2457738|N|N|N|N|N| +2457831|AAAAAAAAHOAIFCAA|2017-03-18|1406|6116|470|2017|6|3|18|1|2017|470|6116|Saturday|2017Q1|N|Y|N|2457814|2457872|2457466|2457739|N|N|N|N|N| +2457832|AAAAAAAAIOAIFCAA|2017-03-19|1406|6116|470|2017|0|3|19|1|2017|470|6116|Sunday|2017Q1|N|N|N|2457814|2457872|2457467|2457740|N|N|N|N|N| +2457833|AAAAAAAAJOAIFCAA|2017-03-20|1406|6116|470|2017|1|3|20|1|2017|470|6116|Monday|2017Q1|N|N|N|2457814|2457872|2457468|2457741|N|N|N|N|N| +2457834|AAAAAAAAKOAIFCAA|2017-03-21|1406|6117|470|2017|2|3|21|1|2017|470|6117|Tuesday|2017Q1|N|N|N|2457814|2457872|2457469|2457742|N|N|N|N|N| +2457835|AAAAAAAALOAIFCAA|2017-03-22|1406|6117|470|2017|3|3|22|1|2017|470|6117|Wednesday|2017Q1|N|N|N|2457814|2457872|2457470|2457743|N|N|N|N|N| +2457836|AAAAAAAAMOAIFCAA|2017-03-23|1406|6117|470|2017|4|3|23|1|2017|470|6117|Thursday|2017Q1|N|N|N|2457814|2457872|2457471|2457744|N|N|N|N|N| +2457837|AAAAAAAANOAIFCAA|2017-03-24|1406|6117|470|2017|5|3|24|1|2017|470|6117|Friday|2017Q1|N|Y|N|2457814|2457872|2457472|2457745|N|N|N|N|N| +2457838|AAAAAAAAOOAIFCAA|2017-03-25|1406|6117|470|2017|6|3|25|1|2017|470|6117|Saturday|2017Q1|N|Y|N|2457814|2457872|2457473|2457746|N|N|N|N|N| +2457839|AAAAAAAAPOAIFCAA|2017-03-26|1406|6117|470|2017|0|3|26|1|2017|470|6117|Sunday|2017Q1|N|N|N|2457814|2457872|2457474|2457747|N|N|N|N|N| +2457840|AAAAAAAAAPAIFCAA|2017-03-27|1406|6117|470|2017|1|3|27|1|2017|470|6117|Monday|2017Q1|N|N|N|2457814|2457872|2457475|2457748|N|N|N|N|N| +2457841|AAAAAAAABPAIFCAA|2017-03-28|1406|6118|470|2017|2|3|28|1|2017|470|6118|Tuesday|2017Q1|N|N|N|2457814|2457872|2457476|2457749|N|N|N|N|N| +2457842|AAAAAAAACPAIFCAA|2017-03-29|1406|6118|470|2017|3|3|29|1|2017|470|6118|Wednesday|2017Q1|N|N|N|2457814|2457872|2457477|2457750|N|N|N|N|N| +2457843|AAAAAAAADPAIFCAA|2017-03-30|1406|6118|470|2017|4|3|30|1|2017|470|6118|Thursday|2017Q1|N|N|N|2457814|2457872|2457478|2457751|N|N|N|N|N| +2457844|AAAAAAAAEPAIFCAA|2017-03-31|1406|6118|470|2017|5|3|31|1|2017|470|6118|Friday|2017Q1|N|Y|N|2457814|2457872|2457479|2457752|N|N|N|N|N| +2457845|AAAAAAAAFPAIFCAA|2017-04-01|1407|6118|470|2017|6|4|1|1|2017|470|6118|Saturday|2017Q1|N|Y|N|2457845|2457934|2457480|2457755|N|N|N|N|N| +2457846|AAAAAAAAGPAIFCAA|2017-04-02|1407|6118|470|2017|0|4|2|2|2017|470|6118|Sunday|2017Q2|N|N|N|2457845|2457934|2457481|2457756|N|N|N|N|N| +2457847|AAAAAAAAHPAIFCAA|2017-04-03|1407|6118|470|2017|1|4|3|2|2017|470|6118|Monday|2017Q2|N|N|N|2457845|2457934|2457482|2457757|N|N|N|N|N| +2457848|AAAAAAAAIPAIFCAA|2017-04-04|1407|6119|470|2017|2|4|4|2|2017|470|6119|Tuesday|2017Q2|N|N|N|2457845|2457934|2457483|2457758|N|N|N|N|N| +2457849|AAAAAAAAJPAIFCAA|2017-04-05|1407|6119|470|2017|3|4|5|2|2017|470|6119|Wednesday|2017Q2|N|N|N|2457845|2457934|2457484|2457759|N|N|N|N|N| +2457850|AAAAAAAAKPAIFCAA|2017-04-06|1407|6119|470|2017|4|4|6|2|2017|470|6119|Thursday|2017Q2|N|N|N|2457845|2457934|2457485|2457760|N|N|N|N|N| +2457851|AAAAAAAALPAIFCAA|2017-04-07|1407|6119|470|2017|5|4|7|2|2017|470|6119|Friday|2017Q2|N|Y|N|2457845|2457934|2457486|2457761|N|N|N|N|N| +2457852|AAAAAAAAMPAIFCAA|2017-04-08|1407|6119|470|2017|6|4|8|2|2017|470|6119|Saturday|2017Q2|N|Y|N|2457845|2457934|2457487|2457762|N|N|N|N|N| +2457853|AAAAAAAANPAIFCAA|2017-04-09|1407|6119|470|2017|0|4|9|2|2017|470|6119|Sunday|2017Q2|N|N|N|2457845|2457934|2457488|2457763|N|N|N|N|N| +2457854|AAAAAAAAOPAIFCAA|2017-04-10|1407|6119|470|2017|1|4|10|2|2017|470|6119|Monday|2017Q2|N|N|N|2457845|2457934|2457489|2457764|N|N|N|N|N| +2457855|AAAAAAAAPPAIFCAA|2017-04-11|1407|6120|470|2017|2|4|11|2|2017|470|6120|Tuesday|2017Q2|N|N|N|2457845|2457934|2457490|2457765|N|N|N|N|N| +2457856|AAAAAAAAAABIFCAA|2017-04-12|1407|6120|470|2017|3|4|12|2|2017|470|6120|Wednesday|2017Q2|N|N|N|2457845|2457934|2457491|2457766|N|N|N|N|N| +2457857|AAAAAAAABABIFCAA|2017-04-13|1407|6120|470|2017|4|4|13|2|2017|470|6120|Thursday|2017Q2|N|N|N|2457845|2457934|2457492|2457767|N|N|N|N|N| +2457858|AAAAAAAACABIFCAA|2017-04-14|1407|6120|470|2017|5|4|14|2|2017|470|6120|Friday|2017Q2|N|Y|N|2457845|2457934|2457493|2457768|N|N|N|N|N| +2457859|AAAAAAAADABIFCAA|2017-04-15|1407|6120|470|2017|6|4|15|2|2017|470|6120|Saturday|2017Q2|N|Y|N|2457845|2457934|2457494|2457769|N|N|N|N|N| +2457860|AAAAAAAAEABIFCAA|2017-04-16|1407|6120|470|2017|0|4|16|2|2017|470|6120|Sunday|2017Q2|N|N|N|2457845|2457934|2457495|2457770|N|N|N|N|N| +2457861|AAAAAAAAFABIFCAA|2017-04-17|1407|6120|470|2017|1|4|17|2|2017|470|6120|Monday|2017Q2|N|N|N|2457845|2457934|2457496|2457771|N|N|N|N|N| +2457862|AAAAAAAAGABIFCAA|2017-04-18|1407|6121|470|2017|2|4|18|2|2017|470|6121|Tuesday|2017Q2|N|N|N|2457845|2457934|2457497|2457772|N|N|N|N|N| +2457863|AAAAAAAAHABIFCAA|2017-04-19|1407|6121|470|2017|3|4|19|2|2017|470|6121|Wednesday|2017Q2|N|N|N|2457845|2457934|2457498|2457773|N|N|N|N|N| +2457864|AAAAAAAAIABIFCAA|2017-04-20|1407|6121|470|2017|4|4|20|2|2017|470|6121|Thursday|2017Q2|N|N|N|2457845|2457934|2457499|2457774|N|N|N|N|N| +2457865|AAAAAAAAJABIFCAA|2017-04-21|1407|6121|470|2017|5|4|21|2|2017|470|6121|Friday|2017Q2|N|Y|N|2457845|2457934|2457500|2457775|N|N|N|N|N| +2457866|AAAAAAAAKABIFCAA|2017-04-22|1407|6121|470|2017|6|4|22|2|2017|470|6121|Saturday|2017Q2|N|Y|N|2457845|2457934|2457501|2457776|N|N|N|N|N| +2457867|AAAAAAAALABIFCAA|2017-04-23|1407|6121|470|2017|0|4|23|2|2017|470|6121|Sunday|2017Q2|N|N|N|2457845|2457934|2457502|2457777|N|N|N|N|N| +2457868|AAAAAAAAMABIFCAA|2017-04-24|1407|6121|470|2017|1|4|24|2|2017|470|6121|Monday|2017Q2|N|N|N|2457845|2457934|2457503|2457778|N|N|N|N|N| +2457869|AAAAAAAANABIFCAA|2017-04-25|1407|6122|470|2017|2|4|25|2|2017|470|6122|Tuesday|2017Q2|N|N|N|2457845|2457934|2457504|2457779|N|N|N|N|N| +2457870|AAAAAAAAOABIFCAA|2017-04-26|1407|6122|470|2017|3|4|26|2|2017|470|6122|Wednesday|2017Q2|N|N|N|2457845|2457934|2457505|2457780|N|N|N|N|N| +2457871|AAAAAAAAPABIFCAA|2017-04-27|1407|6122|470|2017|4|4|27|2|2017|470|6122|Thursday|2017Q2|N|N|N|2457845|2457934|2457506|2457781|N|N|N|N|N| +2457872|AAAAAAAAABBIFCAA|2017-04-28|1407|6122|470|2017|5|4|28|2|2017|470|6122|Friday|2017Q2|N|Y|N|2457845|2457934|2457507|2457782|N|N|N|N|N| +2457873|AAAAAAAABBBIFCAA|2017-04-29|1407|6122|470|2017|6|4|29|2|2017|470|6122|Saturday|2017Q2|N|Y|N|2457845|2457934|2457508|2457783|N|N|N|N|N| +2457874|AAAAAAAACBBIFCAA|2017-04-30|1407|6122|470|2017|0|4|30|2|2017|470|6122|Sunday|2017Q2|N|N|N|2457845|2457934|2457509|2457784|N|N|N|N|N| +2457875|AAAAAAAADBBIFCAA|2017-05-01|1408|6122|470|2017|1|5|1|2|2017|470|6122|Monday|2017Q2|N|N|N|2457875|2457994|2457510|2457785|N|N|N|N|N| +2457876|AAAAAAAAEBBIFCAA|2017-05-02|1408|6123|470|2017|2|5|2|2|2017|470|6123|Tuesday|2017Q2|N|N|N|2457875|2457994|2457511|2457786|N|N|N|N|N| +2457877|AAAAAAAAFBBIFCAA|2017-05-03|1408|6123|470|2017|3|5|3|2|2017|470|6123|Wednesday|2017Q2|N|N|N|2457875|2457994|2457512|2457787|N|N|N|N|N| +2457878|AAAAAAAAGBBIFCAA|2017-05-04|1408|6123|470|2017|4|5|4|2|2017|470|6123|Thursday|2017Q2|N|N|N|2457875|2457994|2457513|2457788|N|N|N|N|N| +2457879|AAAAAAAAHBBIFCAA|2017-05-05|1408|6123|470|2017|5|5|5|2|2017|470|6123|Friday|2017Q2|N|Y|N|2457875|2457994|2457514|2457789|N|N|N|N|N| +2457880|AAAAAAAAIBBIFCAA|2017-05-06|1408|6123|470|2017|6|5|6|2|2017|470|6123|Saturday|2017Q2|N|Y|N|2457875|2457994|2457515|2457790|N|N|N|N|N| +2457881|AAAAAAAAJBBIFCAA|2017-05-07|1408|6123|470|2017|0|5|7|2|2017|470|6123|Sunday|2017Q2|N|N|N|2457875|2457994|2457516|2457791|N|N|N|N|N| +2457882|AAAAAAAAKBBIFCAA|2017-05-08|1408|6123|470|2017|1|5|8|2|2017|470|6123|Monday|2017Q2|N|N|N|2457875|2457994|2457517|2457792|N|N|N|N|N| +2457883|AAAAAAAALBBIFCAA|2017-05-09|1408|6124|470|2017|2|5|9|2|2017|470|6124|Tuesday|2017Q2|N|N|N|2457875|2457994|2457518|2457793|N|N|N|N|N| +2457884|AAAAAAAAMBBIFCAA|2017-05-10|1408|6124|470|2017|3|5|10|2|2017|470|6124|Wednesday|2017Q2|N|N|N|2457875|2457994|2457519|2457794|N|N|N|N|N| +2457885|AAAAAAAANBBIFCAA|2017-05-11|1408|6124|470|2017|4|5|11|2|2017|470|6124|Thursday|2017Q2|N|N|N|2457875|2457994|2457520|2457795|N|N|N|N|N| +2457886|AAAAAAAAOBBIFCAA|2017-05-12|1408|6124|470|2017|5|5|12|2|2017|470|6124|Friday|2017Q2|N|Y|N|2457875|2457994|2457521|2457796|N|N|N|N|N| +2457887|AAAAAAAAPBBIFCAA|2017-05-13|1408|6124|470|2017|6|5|13|2|2017|470|6124|Saturday|2017Q2|N|Y|N|2457875|2457994|2457522|2457797|N|N|N|N|N| +2457888|AAAAAAAAACBIFCAA|2017-05-14|1408|6124|470|2017|0|5|14|2|2017|470|6124|Sunday|2017Q2|N|N|N|2457875|2457994|2457523|2457798|N|N|N|N|N| +2457889|AAAAAAAABCBIFCAA|2017-05-15|1408|6124|470|2017|1|5|15|2|2017|470|6124|Monday|2017Q2|N|N|N|2457875|2457994|2457524|2457799|N|N|N|N|N| +2457890|AAAAAAAACCBIFCAA|2017-05-16|1408|6125|470|2017|2|5|16|2|2017|470|6125|Tuesday|2017Q2|N|N|N|2457875|2457994|2457525|2457800|N|N|N|N|N| +2457891|AAAAAAAADCBIFCAA|2017-05-17|1408|6125|470|2017|3|5|17|2|2017|470|6125|Wednesday|2017Q2|N|N|N|2457875|2457994|2457526|2457801|N|N|N|N|N| +2457892|AAAAAAAAECBIFCAA|2017-05-18|1408|6125|470|2017|4|5|18|2|2017|470|6125|Thursday|2017Q2|N|N|N|2457875|2457994|2457527|2457802|N|N|N|N|N| +2457893|AAAAAAAAFCBIFCAA|2017-05-19|1408|6125|470|2017|5|5|19|2|2017|470|6125|Friday|2017Q2|N|Y|N|2457875|2457994|2457528|2457803|N|N|N|N|N| +2457894|AAAAAAAAGCBIFCAA|2017-05-20|1408|6125|470|2017|6|5|20|2|2017|470|6125|Saturday|2017Q2|N|Y|N|2457875|2457994|2457529|2457804|N|N|N|N|N| +2457895|AAAAAAAAHCBIFCAA|2017-05-21|1408|6125|470|2017|0|5|21|2|2017|470|6125|Sunday|2017Q2|N|N|N|2457875|2457994|2457530|2457805|N|N|N|N|N| +2457896|AAAAAAAAICBIFCAA|2017-05-22|1408|6125|470|2017|1|5|22|2|2017|470|6125|Monday|2017Q2|N|N|N|2457875|2457994|2457531|2457806|N|N|N|N|N| +2457897|AAAAAAAAJCBIFCAA|2017-05-23|1408|6126|470|2017|2|5|23|2|2017|470|6126|Tuesday|2017Q2|N|N|N|2457875|2457994|2457532|2457807|N|N|N|N|N| +2457898|AAAAAAAAKCBIFCAA|2017-05-24|1408|6126|470|2017|3|5|24|2|2017|470|6126|Wednesday|2017Q2|N|N|N|2457875|2457994|2457533|2457808|N|N|N|N|N| +2457899|AAAAAAAALCBIFCAA|2017-05-25|1408|6126|470|2017|4|5|25|2|2017|470|6126|Thursday|2017Q2|N|N|N|2457875|2457994|2457534|2457809|N|N|N|N|N| +2457900|AAAAAAAAMCBIFCAA|2017-05-26|1408|6126|470|2017|5|5|26|2|2017|470|6126|Friday|2017Q2|N|Y|N|2457875|2457994|2457535|2457810|N|N|N|N|N| +2457901|AAAAAAAANCBIFCAA|2017-05-27|1408|6126|470|2017|6|5|27|2|2017|470|6126|Saturday|2017Q2|N|Y|N|2457875|2457994|2457536|2457811|N|N|N|N|N| +2457902|AAAAAAAAOCBIFCAA|2017-05-28|1408|6126|470|2017|0|5|28|2|2017|470|6126|Sunday|2017Q2|N|N|N|2457875|2457994|2457537|2457812|N|N|N|N|N| +2457903|AAAAAAAAPCBIFCAA|2017-05-29|1408|6126|470|2017|1|5|29|2|2017|470|6126|Monday|2017Q2|N|N|N|2457875|2457994|2457538|2457813|N|N|N|N|N| +2457904|AAAAAAAAADBIFCAA|2017-05-30|1408|6127|470|2017|2|5|30|2|2017|470|6127|Tuesday|2017Q2|N|N|N|2457875|2457994|2457539|2457814|N|N|N|N|N| +2457905|AAAAAAAABDBIFCAA|2017-05-31|1408|6127|470|2017|3|5|31|2|2017|470|6127|Wednesday|2017Q2|N|N|N|2457875|2457994|2457540|2457815|N|N|N|N|N| +2457906|AAAAAAAACDBIFCAA|2017-06-01|1409|6127|471|2017|4|6|1|2|2017|471|6127|Thursday|2017Q2|N|N|N|2457906|2458056|2457541|2457816|N|N|N|N|N| +2457907|AAAAAAAADDBIFCAA|2017-06-02|1409|6127|471|2017|5|6|2|2|2017|471|6127|Friday|2017Q2|N|Y|N|2457906|2458056|2457542|2457817|N|N|N|N|N| +2457908|AAAAAAAAEDBIFCAA|2017-06-03|1409|6127|471|2017|6|6|3|2|2017|471|6127|Saturday|2017Q2|N|Y|N|2457906|2458056|2457543|2457818|N|N|N|N|N| +2457909|AAAAAAAAFDBIFCAA|2017-06-04|1409|6127|471|2017|0|6|4|2|2017|471|6127|Sunday|2017Q2|N|N|N|2457906|2458056|2457544|2457819|N|N|N|N|N| +2457910|AAAAAAAAGDBIFCAA|2017-06-05|1409|6127|471|2017|1|6|5|2|2017|471|6127|Monday|2017Q2|N|N|N|2457906|2458056|2457545|2457820|N|N|N|N|N| +2457911|AAAAAAAAHDBIFCAA|2017-06-06|1409|6128|471|2017|2|6|6|2|2017|471|6128|Tuesday|2017Q2|N|N|N|2457906|2458056|2457546|2457821|N|N|N|N|N| +2457912|AAAAAAAAIDBIFCAA|2017-06-07|1409|6128|471|2017|3|6|7|2|2017|471|6128|Wednesday|2017Q2|N|N|N|2457906|2458056|2457547|2457822|N|N|N|N|N| +2457913|AAAAAAAAJDBIFCAA|2017-06-08|1409|6128|471|2017|4|6|8|2|2017|471|6128|Thursday|2017Q2|N|N|N|2457906|2458056|2457548|2457823|N|N|N|N|N| +2457914|AAAAAAAAKDBIFCAA|2017-06-09|1409|6128|471|2017|5|6|9|2|2017|471|6128|Friday|2017Q2|N|Y|N|2457906|2458056|2457549|2457824|N|N|N|N|N| +2457915|AAAAAAAALDBIFCAA|2017-06-10|1409|6128|471|2017|6|6|10|2|2017|471|6128|Saturday|2017Q2|N|Y|N|2457906|2458056|2457550|2457825|N|N|N|N|N| +2457916|AAAAAAAAMDBIFCAA|2017-06-11|1409|6128|471|2017|0|6|11|2|2017|471|6128|Sunday|2017Q2|N|N|N|2457906|2458056|2457551|2457826|N|N|N|N|N| +2457917|AAAAAAAANDBIFCAA|2017-06-12|1409|6128|471|2017|1|6|12|2|2017|471|6128|Monday|2017Q2|N|N|N|2457906|2458056|2457552|2457827|N|N|N|N|N| +2457918|AAAAAAAAODBIFCAA|2017-06-13|1409|6129|471|2017|2|6|13|2|2017|471|6129|Tuesday|2017Q2|N|N|N|2457906|2458056|2457553|2457828|N|N|N|N|N| +2457919|AAAAAAAAPDBIFCAA|2017-06-14|1409|6129|471|2017|3|6|14|2|2017|471|6129|Wednesday|2017Q2|N|N|N|2457906|2458056|2457554|2457829|N|N|N|N|N| +2457920|AAAAAAAAAEBIFCAA|2017-06-15|1409|6129|471|2017|4|6|15|2|2017|471|6129|Thursday|2017Q2|N|N|N|2457906|2458056|2457555|2457830|N|N|N|N|N| +2457921|AAAAAAAABEBIFCAA|2017-06-16|1409|6129|471|2017|5|6|16|2|2017|471|6129|Friday|2017Q2|N|Y|N|2457906|2458056|2457556|2457831|N|N|N|N|N| +2457922|AAAAAAAACEBIFCAA|2017-06-17|1409|6129|471|2017|6|6|17|2|2017|471|6129|Saturday|2017Q2|N|Y|N|2457906|2458056|2457557|2457832|N|N|N|N|N| +2457923|AAAAAAAADEBIFCAA|2017-06-18|1409|6129|471|2017|0|6|18|2|2017|471|6129|Sunday|2017Q2|N|N|N|2457906|2458056|2457558|2457833|N|N|N|N|N| +2457924|AAAAAAAAEEBIFCAA|2017-06-19|1409|6129|471|2017|1|6|19|2|2017|471|6129|Monday|2017Q2|N|N|N|2457906|2458056|2457559|2457834|N|N|N|N|N| +2457925|AAAAAAAAFEBIFCAA|2017-06-20|1409|6130|471|2017|2|6|20|2|2017|471|6130|Tuesday|2017Q2|N|N|N|2457906|2458056|2457560|2457835|N|N|N|N|N| +2457926|AAAAAAAAGEBIFCAA|2017-06-21|1409|6130|471|2017|3|6|21|2|2017|471|6130|Wednesday|2017Q2|N|N|N|2457906|2458056|2457561|2457836|N|N|N|N|N| +2457927|AAAAAAAAHEBIFCAA|2017-06-22|1409|6130|471|2017|4|6|22|2|2017|471|6130|Thursday|2017Q2|N|N|N|2457906|2458056|2457562|2457837|N|N|N|N|N| +2457928|AAAAAAAAIEBIFCAA|2017-06-23|1409|6130|471|2017|5|6|23|2|2017|471|6130|Friday|2017Q2|N|Y|N|2457906|2458056|2457563|2457838|N|N|N|N|N| +2457929|AAAAAAAAJEBIFCAA|2017-06-24|1409|6130|471|2017|6|6|24|2|2017|471|6130|Saturday|2017Q2|N|Y|N|2457906|2458056|2457564|2457839|N|N|N|N|N| +2457930|AAAAAAAAKEBIFCAA|2017-06-25|1409|6130|471|2017|0|6|25|2|2017|471|6130|Sunday|2017Q2|N|N|N|2457906|2458056|2457565|2457840|N|N|N|N|N| +2457931|AAAAAAAALEBIFCAA|2017-06-26|1409|6130|471|2017|1|6|26|2|2017|471|6130|Monday|2017Q2|N|N|N|2457906|2458056|2457566|2457841|N|N|N|N|N| +2457932|AAAAAAAAMEBIFCAA|2017-06-27|1409|6131|471|2017|2|6|27|2|2017|471|6131|Tuesday|2017Q2|N|N|N|2457906|2458056|2457567|2457842|N|N|N|N|N| +2457933|AAAAAAAANEBIFCAA|2017-06-28|1409|6131|471|2017|3|6|28|2|2017|471|6131|Wednesday|2017Q2|N|N|N|2457906|2458056|2457568|2457843|N|N|N|N|N| +2457934|AAAAAAAAOEBIFCAA|2017-06-29|1409|6131|471|2017|4|6|29|2|2017|471|6131|Thursday|2017Q2|N|N|N|2457906|2458056|2457569|2457844|N|N|N|N|N| +2457935|AAAAAAAAPEBIFCAA|2017-06-30|1409|6131|471|2017|5|6|30|2|2017|471|6131|Friday|2017Q2|N|Y|N|2457906|2458056|2457570|2457845|N|N|N|N|N| +2457936|AAAAAAAAAFBIFCAA|2017-07-01|1410|6131|471|2017|6|7|1|2|2017|471|6131|Saturday|2017Q2|N|Y|N|2457936|2458116|2457571|2457845|N|N|N|N|N| +2457937|AAAAAAAABFBIFCAA|2017-07-02|1410|6131|471|2017|0|7|2|3|2017|471|6131|Sunday|2017Q3|N|N|N|2457936|2458116|2457572|2457846|N|N|N|N|N| +2457938|AAAAAAAACFBIFCAA|2017-07-03|1410|6131|471|2017|1|7|3|3|2017|471|6131|Monday|2017Q3|N|N|N|2457936|2458116|2457573|2457847|N|N|N|N|N| +2457939|AAAAAAAADFBIFCAA|2017-07-04|1410|6132|471|2017|2|7|4|3|2017|471|6132|Tuesday|2017Q3|N|N|N|2457936|2458116|2457574|2457848|N|N|N|N|N| +2457940|AAAAAAAAEFBIFCAA|2017-07-05|1410|6132|471|2017|3|7|5|3|2017|471|6132|Wednesday|2017Q3|Y|N|N|2457936|2458116|2457575|2457849|N|N|N|N|N| +2457941|AAAAAAAAFFBIFCAA|2017-07-06|1410|6132|471|2017|4|7|6|3|2017|471|6132|Thursday|2017Q3|N|N|Y|2457936|2458116|2457576|2457850|N|N|N|N|N| +2457942|AAAAAAAAGFBIFCAA|2017-07-07|1410|6132|471|2017|5|7|7|3|2017|471|6132|Friday|2017Q3|N|Y|N|2457936|2458116|2457577|2457851|N|N|N|N|N| +2457943|AAAAAAAAHFBIFCAA|2017-07-08|1410|6132|471|2017|6|7|8|3|2017|471|6132|Saturday|2017Q3|N|Y|N|2457936|2458116|2457578|2457852|N|N|N|N|N| +2457944|AAAAAAAAIFBIFCAA|2017-07-09|1410|6132|471|2017|0|7|9|3|2017|471|6132|Sunday|2017Q3|N|N|N|2457936|2458116|2457579|2457853|N|N|N|N|N| +2457945|AAAAAAAAJFBIFCAA|2017-07-10|1410|6132|471|2017|1|7|10|3|2017|471|6132|Monday|2017Q3|N|N|N|2457936|2458116|2457580|2457854|N|N|N|N|N| +2457946|AAAAAAAAKFBIFCAA|2017-07-11|1410|6133|471|2017|2|7|11|3|2017|471|6133|Tuesday|2017Q3|N|N|N|2457936|2458116|2457581|2457855|N|N|N|N|N| +2457947|AAAAAAAALFBIFCAA|2017-07-12|1410|6133|471|2017|3|7|12|3|2017|471|6133|Wednesday|2017Q3|N|N|N|2457936|2458116|2457582|2457856|N|N|N|N|N| +2457948|AAAAAAAAMFBIFCAA|2017-07-13|1410|6133|471|2017|4|7|13|3|2017|471|6133|Thursday|2017Q3|N|N|N|2457936|2458116|2457583|2457857|N|N|N|N|N| +2457949|AAAAAAAANFBIFCAA|2017-07-14|1410|6133|471|2017|5|7|14|3|2017|471|6133|Friday|2017Q3|N|Y|N|2457936|2458116|2457584|2457858|N|N|N|N|N| +2457950|AAAAAAAAOFBIFCAA|2017-07-15|1410|6133|471|2017|6|7|15|3|2017|471|6133|Saturday|2017Q3|N|Y|N|2457936|2458116|2457585|2457859|N|N|N|N|N| +2457951|AAAAAAAAPFBIFCAA|2017-07-16|1410|6133|471|2017|0|7|16|3|2017|471|6133|Sunday|2017Q3|N|N|N|2457936|2458116|2457586|2457860|N|N|N|N|N| +2457952|AAAAAAAAAGBIFCAA|2017-07-17|1410|6133|471|2017|1|7|17|3|2017|471|6133|Monday|2017Q3|N|N|N|2457936|2458116|2457587|2457861|N|N|N|N|N| +2457953|AAAAAAAABGBIFCAA|2017-07-18|1410|6134|471|2017|2|7|18|3|2017|471|6134|Tuesday|2017Q3|N|N|N|2457936|2458116|2457588|2457862|N|N|N|N|N| +2457954|AAAAAAAACGBIFCAA|2017-07-19|1410|6134|471|2017|3|7|19|3|2017|471|6134|Wednesday|2017Q3|N|N|N|2457936|2458116|2457589|2457863|N|N|N|N|N| +2457955|AAAAAAAADGBIFCAA|2017-07-20|1410|6134|471|2017|4|7|20|3|2017|471|6134|Thursday|2017Q3|N|N|N|2457936|2458116|2457590|2457864|N|N|N|N|N| +2457956|AAAAAAAAEGBIFCAA|2017-07-21|1410|6134|471|2017|5|7|21|3|2017|471|6134|Friday|2017Q3|N|Y|N|2457936|2458116|2457591|2457865|N|N|N|N|N| +2457957|AAAAAAAAFGBIFCAA|2017-07-22|1410|6134|471|2017|6|7|22|3|2017|471|6134|Saturday|2017Q3|N|Y|N|2457936|2458116|2457592|2457866|N|N|N|N|N| +2457958|AAAAAAAAGGBIFCAA|2017-07-23|1410|6134|471|2017|0|7|23|3|2017|471|6134|Sunday|2017Q3|N|N|N|2457936|2458116|2457593|2457867|N|N|N|N|N| +2457959|AAAAAAAAHGBIFCAA|2017-07-24|1410|6134|471|2017|1|7|24|3|2017|471|6134|Monday|2017Q3|N|N|N|2457936|2458116|2457594|2457868|N|N|N|N|N| +2457960|AAAAAAAAIGBIFCAA|2017-07-25|1410|6135|471|2017|2|7|25|3|2017|471|6135|Tuesday|2017Q3|N|N|N|2457936|2458116|2457595|2457869|N|N|N|N|N| +2457961|AAAAAAAAJGBIFCAA|2017-07-26|1410|6135|471|2017|3|7|26|3|2017|471|6135|Wednesday|2017Q3|N|N|N|2457936|2458116|2457596|2457870|N|N|N|N|N| +2457962|AAAAAAAAKGBIFCAA|2017-07-27|1410|6135|471|2017|4|7|27|3|2017|471|6135|Thursday|2017Q3|N|N|N|2457936|2458116|2457597|2457871|N|N|N|N|N| +2457963|AAAAAAAALGBIFCAA|2017-07-28|1410|6135|471|2017|5|7|28|3|2017|471|6135|Friday|2017Q3|N|Y|N|2457936|2458116|2457598|2457872|N|N|N|N|N| +2457964|AAAAAAAAMGBIFCAA|2017-07-29|1410|6135|471|2017|6|7|29|3|2017|471|6135|Saturday|2017Q3|N|Y|N|2457936|2458116|2457599|2457873|N|N|N|N|N| +2457965|AAAAAAAANGBIFCAA|2017-07-30|1410|6135|471|2017|0|7|30|3|2017|471|6135|Sunday|2017Q3|N|N|N|2457936|2458116|2457600|2457874|N|N|N|N|N| +2457966|AAAAAAAAOGBIFCAA|2017-07-31|1410|6135|471|2017|1|7|31|3|2017|471|6135|Monday|2017Q3|N|N|N|2457936|2458116|2457601|2457875|N|N|N|N|N| +2457967|AAAAAAAAPGBIFCAA|2017-08-01|1411|6136|471|2017|2|8|1|3|2017|471|6136|Tuesday|2017Q3|N|N|N|2457967|2458178|2457602|2457876|N|N|N|N|N| +2457968|AAAAAAAAAHBIFCAA|2017-08-02|1411|6136|471|2017|3|8|2|3|2017|471|6136|Wednesday|2017Q3|N|N|N|2457967|2458178|2457603|2457877|N|N|N|N|N| +2457969|AAAAAAAABHBIFCAA|2017-08-03|1411|6136|471|2017|4|8|3|3|2017|471|6136|Thursday|2017Q3|N|N|N|2457967|2458178|2457604|2457878|N|N|N|N|N| +2457970|AAAAAAAACHBIFCAA|2017-08-04|1411|6136|471|2017|5|8|4|3|2017|471|6136|Friday|2017Q3|N|Y|N|2457967|2458178|2457605|2457879|N|N|N|N|N| +2457971|AAAAAAAADHBIFCAA|2017-08-05|1411|6136|471|2017|6|8|5|3|2017|471|6136|Saturday|2017Q3|N|Y|N|2457967|2458178|2457606|2457880|N|N|N|N|N| +2457972|AAAAAAAAEHBIFCAA|2017-08-06|1411|6136|471|2017|0|8|6|3|2017|471|6136|Sunday|2017Q3|N|N|N|2457967|2458178|2457607|2457881|N|N|N|N|N| +2457973|AAAAAAAAFHBIFCAA|2017-08-07|1411|6136|471|2017|1|8|7|3|2017|471|6136|Monday|2017Q3|N|N|N|2457967|2458178|2457608|2457882|N|N|N|N|N| +2457974|AAAAAAAAGHBIFCAA|2017-08-08|1411|6137|471|2017|2|8|8|3|2017|471|6137|Tuesday|2017Q3|N|N|N|2457967|2458178|2457609|2457883|N|N|N|N|N| +2457975|AAAAAAAAHHBIFCAA|2017-08-09|1411|6137|471|2017|3|8|9|3|2017|471|6137|Wednesday|2017Q3|N|N|N|2457967|2458178|2457610|2457884|N|N|N|N|N| +2457976|AAAAAAAAIHBIFCAA|2017-08-10|1411|6137|471|2017|4|8|10|3|2017|471|6137|Thursday|2017Q3|N|N|N|2457967|2458178|2457611|2457885|N|N|N|N|N| +2457977|AAAAAAAAJHBIFCAA|2017-08-11|1411|6137|471|2017|5|8|11|3|2017|471|6137|Friday|2017Q3|N|Y|N|2457967|2458178|2457612|2457886|N|N|N|N|N| +2457978|AAAAAAAAKHBIFCAA|2017-08-12|1411|6137|471|2017|6|8|12|3|2017|471|6137|Saturday|2017Q3|N|Y|N|2457967|2458178|2457613|2457887|N|N|N|N|N| +2457979|AAAAAAAALHBIFCAA|2017-08-13|1411|6137|471|2017|0|8|13|3|2017|471|6137|Sunday|2017Q3|N|N|N|2457967|2458178|2457614|2457888|N|N|N|N|N| +2457980|AAAAAAAAMHBIFCAA|2017-08-14|1411|6137|471|2017|1|8|14|3|2017|471|6137|Monday|2017Q3|N|N|N|2457967|2458178|2457615|2457889|N|N|N|N|N| +2457981|AAAAAAAANHBIFCAA|2017-08-15|1411|6138|471|2017|2|8|15|3|2017|471|6138|Tuesday|2017Q3|N|N|N|2457967|2458178|2457616|2457890|N|N|N|N|N| +2457982|AAAAAAAAOHBIFCAA|2017-08-16|1411|6138|471|2017|3|8|16|3|2017|471|6138|Wednesday|2017Q3|N|N|N|2457967|2458178|2457617|2457891|N|N|N|N|N| +2457983|AAAAAAAAPHBIFCAA|2017-08-17|1411|6138|471|2017|4|8|17|3|2017|471|6138|Thursday|2017Q3|N|N|N|2457967|2458178|2457618|2457892|N|N|N|N|N| +2457984|AAAAAAAAAIBIFCAA|2017-08-18|1411|6138|471|2017|5|8|18|3|2017|471|6138|Friday|2017Q3|N|Y|N|2457967|2458178|2457619|2457893|N|N|N|N|N| +2457985|AAAAAAAABIBIFCAA|2017-08-19|1411|6138|471|2017|6|8|19|3|2017|471|6138|Saturday|2017Q3|N|Y|N|2457967|2458178|2457620|2457894|N|N|N|N|N| +2457986|AAAAAAAACIBIFCAA|2017-08-20|1411|6138|471|2017|0|8|20|3|2017|471|6138|Sunday|2017Q3|N|N|N|2457967|2458178|2457621|2457895|N|N|N|N|N| +2457987|AAAAAAAADIBIFCAA|2017-08-21|1411|6138|471|2017|1|8|21|3|2017|471|6138|Monday|2017Q3|N|N|N|2457967|2458178|2457622|2457896|N|N|N|N|N| +2457988|AAAAAAAAEIBIFCAA|2017-08-22|1411|6139|471|2017|2|8|22|3|2017|471|6139|Tuesday|2017Q3|N|N|N|2457967|2458178|2457623|2457897|N|N|N|N|N| +2457989|AAAAAAAAFIBIFCAA|2017-08-23|1411|6139|471|2017|3|8|23|3|2017|471|6139|Wednesday|2017Q3|N|N|N|2457967|2458178|2457624|2457898|N|N|N|N|N| +2457990|AAAAAAAAGIBIFCAA|2017-08-24|1411|6139|471|2017|4|8|24|3|2017|471|6139|Thursday|2017Q3|N|N|N|2457967|2458178|2457625|2457899|N|N|N|N|N| +2457991|AAAAAAAAHIBIFCAA|2017-08-25|1411|6139|471|2017|5|8|25|3|2017|471|6139|Friday|2017Q3|N|Y|N|2457967|2458178|2457626|2457900|N|N|N|N|N| +2457992|AAAAAAAAIIBIFCAA|2017-08-26|1411|6139|471|2017|6|8|26|3|2017|471|6139|Saturday|2017Q3|N|Y|N|2457967|2458178|2457627|2457901|N|N|N|N|N| +2457993|AAAAAAAAJIBIFCAA|2017-08-27|1411|6139|471|2017|0|8|27|3|2017|471|6139|Sunday|2017Q3|N|N|N|2457967|2458178|2457628|2457902|N|N|N|N|N| +2457994|AAAAAAAAKIBIFCAA|2017-08-28|1411|6139|471|2017|1|8|28|3|2017|471|6139|Monday|2017Q3|N|N|N|2457967|2458178|2457629|2457903|N|N|N|N|N| +2457995|AAAAAAAALIBIFCAA|2017-08-29|1411|6140|471|2017|2|8|29|3|2017|471|6140|Tuesday|2017Q3|N|N|N|2457967|2458178|2457630|2457904|N|N|N|N|N| +2457996|AAAAAAAAMIBIFCAA|2017-08-30|1411|6140|471|2017|3|8|30|3|2017|471|6140|Wednesday|2017Q3|N|N|N|2457967|2458178|2457631|2457905|N|N|N|N|N| +2457997|AAAAAAAANIBIFCAA|2017-08-31|1411|6140|471|2017|4|8|31|3|2017|471|6140|Thursday|2017Q3|N|N|N|2457967|2458178|2457632|2457906|N|N|N|N|N| +2457998|AAAAAAAAOIBIFCAA|2017-09-01|1412|6140|472|2017|5|9|1|3|2017|472|6140|Friday|2017Q3|N|Y|N|2457998|2458240|2457633|2457907|N|N|N|N|N| +2457999|AAAAAAAAPIBIFCAA|2017-09-02|1412|6140|472|2017|6|9|2|3|2017|472|6140|Saturday|2017Q3|N|Y|N|2457998|2458240|2457634|2457908|N|N|N|N|N| +2458000|AAAAAAAAAJBIFCAA|2017-09-03|1412|6140|472|2017|0|9|3|3|2017|472|6140|Sunday|2017Q3|N|N|N|2457998|2458240|2457635|2457909|N|N|N|N|N| +2458001|AAAAAAAABJBIFCAA|2017-09-04|1412|6140|472|2017|1|9|4|3|2017|472|6140|Monday|2017Q3|N|N|N|2457998|2458240|2457636|2457910|N|N|N|N|N| +2458002|AAAAAAAACJBIFCAA|2017-09-05|1412|6141|472|2017|2|9|5|3|2017|472|6141|Tuesday|2017Q3|N|N|N|2457998|2458240|2457637|2457911|N|N|N|N|N| +2458003|AAAAAAAADJBIFCAA|2017-09-06|1412|6141|472|2017|3|9|6|3|2017|472|6141|Wednesday|2017Q3|N|N|N|2457998|2458240|2457638|2457912|N|N|N|N|N| +2458004|AAAAAAAAEJBIFCAA|2017-09-07|1412|6141|472|2017|4|9|7|3|2017|472|6141|Thursday|2017Q3|N|N|N|2457998|2458240|2457639|2457913|N|N|N|N|N| +2458005|AAAAAAAAFJBIFCAA|2017-09-08|1412|6141|472|2017|5|9|8|3|2017|472|6141|Friday|2017Q3|N|Y|N|2457998|2458240|2457640|2457914|N|N|N|N|N| +2458006|AAAAAAAAGJBIFCAA|2017-09-09|1412|6141|472|2017|6|9|9|3|2017|472|6141|Saturday|2017Q3|N|Y|N|2457998|2458240|2457641|2457915|N|N|N|N|N| +2458007|AAAAAAAAHJBIFCAA|2017-09-10|1412|6141|472|2017|0|9|10|3|2017|472|6141|Sunday|2017Q3|N|N|N|2457998|2458240|2457642|2457916|N|N|N|N|N| +2458008|AAAAAAAAIJBIFCAA|2017-09-11|1412|6141|472|2017|1|9|11|3|2017|472|6141|Monday|2017Q3|N|N|N|2457998|2458240|2457643|2457917|N|N|N|N|N| +2458009|AAAAAAAAJJBIFCAA|2017-09-12|1412|6142|472|2017|2|9|12|3|2017|472|6142|Tuesday|2017Q3|N|N|N|2457998|2458240|2457644|2457918|N|N|N|N|N| +2458010|AAAAAAAAKJBIFCAA|2017-09-13|1412|6142|472|2017|3|9|13|3|2017|472|6142|Wednesday|2017Q3|N|N|N|2457998|2458240|2457645|2457919|N|N|N|N|N| +2458011|AAAAAAAALJBIFCAA|2017-09-14|1412|6142|472|2017|4|9|14|3|2017|472|6142|Thursday|2017Q3|N|N|N|2457998|2458240|2457646|2457920|N|N|N|N|N| +2458012|AAAAAAAAMJBIFCAA|2017-09-15|1412|6142|472|2017|5|9|15|3|2017|472|6142|Friday|2017Q3|N|Y|N|2457998|2458240|2457647|2457921|N|N|N|N|N| +2458013|AAAAAAAANJBIFCAA|2017-09-16|1412|6142|472|2017|6|9|16|3|2017|472|6142|Saturday|2017Q3|N|Y|N|2457998|2458240|2457648|2457922|N|N|N|N|N| +2458014|AAAAAAAAOJBIFCAA|2017-09-17|1412|6142|472|2017|0|9|17|3|2017|472|6142|Sunday|2017Q3|N|N|N|2457998|2458240|2457649|2457923|N|N|N|N|N| +2458015|AAAAAAAAPJBIFCAA|2017-09-18|1412|6142|472|2017|1|9|18|3|2017|472|6142|Monday|2017Q3|N|N|N|2457998|2458240|2457650|2457924|N|N|N|N|N| +2458016|AAAAAAAAAKBIFCAA|2017-09-19|1412|6143|472|2017|2|9|19|3|2017|472|6143|Tuesday|2017Q3|N|N|N|2457998|2458240|2457651|2457925|N|N|N|N|N| +2458017|AAAAAAAABKBIFCAA|2017-09-20|1412|6143|472|2017|3|9|20|3|2017|472|6143|Wednesday|2017Q3|N|N|N|2457998|2458240|2457652|2457926|N|N|N|N|N| +2458018|AAAAAAAACKBIFCAA|2017-09-21|1412|6143|472|2017|4|9|21|3|2017|472|6143|Thursday|2017Q3|N|N|N|2457998|2458240|2457653|2457927|N|N|N|N|N| +2458019|AAAAAAAADKBIFCAA|2017-09-22|1412|6143|472|2017|5|9|22|3|2017|472|6143|Friday|2017Q3|N|Y|N|2457998|2458240|2457654|2457928|N|N|N|N|N| +2458020|AAAAAAAAEKBIFCAA|2017-09-23|1412|6143|472|2017|6|9|23|3|2017|472|6143|Saturday|2017Q3|N|Y|N|2457998|2458240|2457655|2457929|N|N|N|N|N| +2458021|AAAAAAAAFKBIFCAA|2017-09-24|1412|6143|472|2017|0|9|24|3|2017|472|6143|Sunday|2017Q3|N|N|N|2457998|2458240|2457656|2457930|N|N|N|N|N| +2458022|AAAAAAAAGKBIFCAA|2017-09-25|1412|6143|472|2017|1|9|25|3|2017|472|6143|Monday|2017Q3|N|N|N|2457998|2458240|2457657|2457931|N|N|N|N|N| +2458023|AAAAAAAAHKBIFCAA|2017-09-26|1412|6144|472|2017|2|9|26|3|2017|472|6144|Tuesday|2017Q3|N|N|N|2457998|2458240|2457658|2457932|N|N|N|N|N| +2458024|AAAAAAAAIKBIFCAA|2017-09-27|1412|6144|472|2017|3|9|27|3|2017|472|6144|Wednesday|2017Q3|N|N|N|2457998|2458240|2457659|2457933|N|N|N|N|N| +2458025|AAAAAAAAJKBIFCAA|2017-09-28|1412|6144|472|2017|4|9|28|3|2017|472|6144|Thursday|2017Q3|N|N|N|2457998|2458240|2457660|2457934|N|N|N|N|N| +2458026|AAAAAAAAKKBIFCAA|2017-09-29|1412|6144|472|2017|5|9|29|3|2017|472|6144|Friday|2017Q3|N|Y|N|2457998|2458240|2457661|2457935|N|N|N|N|N| +2458027|AAAAAAAALKBIFCAA|2017-09-30|1412|6144|472|2017|6|9|30|3|2017|472|6144|Saturday|2017Q3|N|Y|N|2457998|2458240|2457662|2457936|N|N|N|N|N| +2458028|AAAAAAAAMKBIFCAA|2017-10-01|1413|6144|472|2017|0|10|1|3|2017|472|6144|Sunday|2017Q3|N|N|N|2458028|2458300|2457663|2457936|N|N|N|N|N| +2458029|AAAAAAAANKBIFCAA|2017-10-02|1413|6144|472|2017|1|10|2|4|2017|472|6144|Monday|2017Q4|N|N|N|2458028|2458300|2457664|2457937|N|N|N|N|N| +2458030|AAAAAAAAOKBIFCAA|2017-10-03|1413|6145|472|2017|2|10|3|4|2017|472|6145|Tuesday|2017Q4|N|N|N|2458028|2458300|2457665|2457938|N|N|N|N|N| +2458031|AAAAAAAAPKBIFCAA|2017-10-04|1413|6145|472|2017|3|10|4|4|2017|472|6145|Wednesday|2017Q4|N|N|N|2458028|2458300|2457666|2457939|N|N|N|N|N| +2458032|AAAAAAAAALBIFCAA|2017-10-05|1413|6145|472|2017|4|10|5|4|2017|472|6145|Thursday|2017Q4|N|N|N|2458028|2458300|2457667|2457940|N|N|N|N|N| +2458033|AAAAAAAABLBIFCAA|2017-10-06|1413|6145|472|2017|5|10|6|4|2017|472|6145|Friday|2017Q4|N|Y|N|2458028|2458300|2457668|2457941|N|N|N|N|N| +2458034|AAAAAAAACLBIFCAA|2017-10-07|1413|6145|472|2017|6|10|7|4|2017|472|6145|Saturday|2017Q4|N|Y|N|2458028|2458300|2457669|2457942|N|N|N|N|N| +2458035|AAAAAAAADLBIFCAA|2017-10-08|1413|6145|472|2017|0|10|8|4|2017|472|6145|Sunday|2017Q4|N|N|N|2458028|2458300|2457670|2457943|N|N|N|N|N| +2458036|AAAAAAAAELBIFCAA|2017-10-09|1413|6145|472|2017|1|10|9|4|2017|472|6145|Monday|2017Q4|N|N|N|2458028|2458300|2457671|2457944|N|N|N|N|N| +2458037|AAAAAAAAFLBIFCAA|2017-10-10|1413|6146|472|2017|2|10|10|4|2017|472|6146|Tuesday|2017Q4|N|N|N|2458028|2458300|2457672|2457945|N|N|N|N|N| +2458038|AAAAAAAAGLBIFCAA|2017-10-11|1413|6146|472|2017|3|10|11|4|2017|472|6146|Wednesday|2017Q4|N|N|N|2458028|2458300|2457673|2457946|N|N|N|N|N| +2458039|AAAAAAAAHLBIFCAA|2017-10-12|1413|6146|472|2017|4|10|12|4|2017|472|6146|Thursday|2017Q4|N|N|N|2458028|2458300|2457674|2457947|N|N|N|N|N| +2458040|AAAAAAAAILBIFCAA|2017-10-13|1413|6146|472|2017|5|10|13|4|2017|472|6146|Friday|2017Q4|N|Y|N|2458028|2458300|2457675|2457948|N|N|N|N|N| +2458041|AAAAAAAAJLBIFCAA|2017-10-14|1413|6146|472|2017|6|10|14|4|2017|472|6146|Saturday|2017Q4|N|Y|N|2458028|2458300|2457676|2457949|N|N|N|N|N| +2458042|AAAAAAAAKLBIFCAA|2017-10-15|1413|6146|472|2017|0|10|15|4|2017|472|6146|Sunday|2017Q4|N|N|N|2458028|2458300|2457677|2457950|N|N|N|N|N| +2458043|AAAAAAAALLBIFCAA|2017-10-16|1413|6146|472|2017|1|10|16|4|2017|472|6146|Monday|2017Q4|N|N|N|2458028|2458300|2457678|2457951|N|N|N|N|N| +2458044|AAAAAAAAMLBIFCAA|2017-10-17|1413|6147|472|2017|2|10|17|4|2017|472|6147|Tuesday|2017Q4|N|N|N|2458028|2458300|2457679|2457952|N|N|N|N|N| +2458045|AAAAAAAANLBIFCAA|2017-10-18|1413|6147|472|2017|3|10|18|4|2017|472|6147|Wednesday|2017Q4|N|N|N|2458028|2458300|2457680|2457953|N|N|N|N|N| +2458046|AAAAAAAAOLBIFCAA|2017-10-19|1413|6147|472|2017|4|10|19|4|2017|472|6147|Thursday|2017Q4|N|N|N|2458028|2458300|2457681|2457954|N|N|N|N|N| +2458047|AAAAAAAAPLBIFCAA|2017-10-20|1413|6147|472|2017|5|10|20|4|2017|472|6147|Friday|2017Q4|N|Y|N|2458028|2458300|2457682|2457955|N|N|N|N|N| +2458048|AAAAAAAAAMBIFCAA|2017-10-21|1413|6147|472|2017|6|10|21|4|2017|472|6147|Saturday|2017Q4|N|Y|N|2458028|2458300|2457683|2457956|N|N|N|N|N| +2458049|AAAAAAAABMBIFCAA|2017-10-22|1413|6147|472|2017|0|10|22|4|2017|472|6147|Sunday|2017Q4|N|N|N|2458028|2458300|2457684|2457957|N|N|N|N|N| +2458050|AAAAAAAACMBIFCAA|2017-10-23|1413|6147|472|2017|1|10|23|4|2017|472|6147|Monday|2017Q4|N|N|N|2458028|2458300|2457685|2457958|N|N|N|N|N| +2458051|AAAAAAAADMBIFCAA|2017-10-24|1413|6148|472|2017|2|10|24|4|2017|472|6148|Tuesday|2017Q4|N|N|N|2458028|2458300|2457686|2457959|N|N|N|N|N| +2458052|AAAAAAAAEMBIFCAA|2017-10-25|1413|6148|472|2017|3|10|25|4|2017|472|6148|Wednesday|2017Q4|N|N|N|2458028|2458300|2457687|2457960|N|N|N|N|N| +2458053|AAAAAAAAFMBIFCAA|2017-10-26|1413|6148|472|2017|4|10|26|4|2017|472|6148|Thursday|2017Q4|N|N|N|2458028|2458300|2457688|2457961|N|N|N|N|N| +2458054|AAAAAAAAGMBIFCAA|2017-10-27|1413|6148|472|2017|5|10|27|4|2017|472|6148|Friday|2017Q4|N|Y|N|2458028|2458300|2457689|2457962|N|N|N|N|N| +2458055|AAAAAAAAHMBIFCAA|2017-10-28|1413|6148|472|2017|6|10|28|4|2017|472|6148|Saturday|2017Q4|N|Y|N|2458028|2458300|2457690|2457963|N|N|N|N|N| +2458056|AAAAAAAAIMBIFCAA|2017-10-29|1413|6148|472|2017|0|10|29|4|2017|472|6148|Sunday|2017Q4|N|N|N|2458028|2458300|2457691|2457964|N|N|N|N|N| +2458057|AAAAAAAAJMBIFCAA|2017-10-30|1413|6148|472|2017|1|10|30|4|2017|472|6148|Monday|2017Q4|N|N|N|2458028|2458300|2457692|2457965|N|N|N|N|N| +2458058|AAAAAAAAKMBIFCAA|2017-10-31|1413|6149|472|2017|2|10|31|4|2017|472|6149|Tuesday|2017Q4|N|N|N|2458028|2458300|2457693|2457966|N|N|N|N|N| +2458059|AAAAAAAALMBIFCAA|2017-11-01|1414|6149|472|2017|3|11|1|4|2017|472|6149|Wednesday|2017Q4|N|N|N|2458059|2458362|2457694|2457967|N|N|N|N|N| +2458060|AAAAAAAAMMBIFCAA|2017-11-02|1414|6149|472|2017|4|11|2|4|2017|472|6149|Thursday|2017Q4|N|N|N|2458059|2458362|2457695|2457968|N|N|N|N|N| +2458061|AAAAAAAANMBIFCAA|2017-11-03|1414|6149|472|2017|5|11|3|4|2017|472|6149|Friday|2017Q4|N|Y|N|2458059|2458362|2457696|2457969|N|N|N|N|N| +2458062|AAAAAAAAOMBIFCAA|2017-11-04|1414|6149|472|2017|6|11|4|4|2017|472|6149|Saturday|2017Q4|N|Y|N|2458059|2458362|2457697|2457970|N|N|N|N|N| +2458063|AAAAAAAAPMBIFCAA|2017-11-05|1414|6149|472|2017|0|11|5|4|2017|472|6149|Sunday|2017Q4|N|N|N|2458059|2458362|2457698|2457971|N|N|N|N|N| +2458064|AAAAAAAAANBIFCAA|2017-11-06|1414|6149|472|2017|1|11|6|4|2017|472|6149|Monday|2017Q4|N|N|N|2458059|2458362|2457699|2457972|N|N|N|N|N| +2458065|AAAAAAAABNBIFCAA|2017-11-07|1414|6150|472|2017|2|11|7|4|2017|472|6150|Tuesday|2017Q4|N|N|N|2458059|2458362|2457700|2457973|N|N|N|N|N| +2458066|AAAAAAAACNBIFCAA|2017-11-08|1414|6150|472|2017|3|11|8|4|2017|472|6150|Wednesday|2017Q4|N|N|N|2458059|2458362|2457701|2457974|N|N|N|N|N| +2458067|AAAAAAAADNBIFCAA|2017-11-09|1414|6150|472|2017|4|11|9|4|2017|472|6150|Thursday|2017Q4|N|N|N|2458059|2458362|2457702|2457975|N|N|N|N|N| +2458068|AAAAAAAAENBIFCAA|2017-11-10|1414|6150|472|2017|5|11|10|4|2017|472|6150|Friday|2017Q4|N|Y|N|2458059|2458362|2457703|2457976|N|N|N|N|N| +2458069|AAAAAAAAFNBIFCAA|2017-11-11|1414|6150|472|2017|6|11|11|4|2017|472|6150|Saturday|2017Q4|N|Y|N|2458059|2458362|2457704|2457977|N|N|N|N|N| +2458070|AAAAAAAAGNBIFCAA|2017-11-12|1414|6150|472|2017|0|11|12|4|2017|472|6150|Sunday|2017Q4|N|N|N|2458059|2458362|2457705|2457978|N|N|N|N|N| +2458071|AAAAAAAAHNBIFCAA|2017-11-13|1414|6150|472|2017|1|11|13|4|2017|472|6150|Monday|2017Q4|N|N|N|2458059|2458362|2457706|2457979|N|N|N|N|N| +2458072|AAAAAAAAINBIFCAA|2017-11-14|1414|6151|472|2017|2|11|14|4|2017|472|6151|Tuesday|2017Q4|N|N|N|2458059|2458362|2457707|2457980|N|N|N|N|N| +2458073|AAAAAAAAJNBIFCAA|2017-11-15|1414|6151|472|2017|3|11|15|4|2017|472|6151|Wednesday|2017Q4|N|N|N|2458059|2458362|2457708|2457981|N|N|N|N|N| +2458074|AAAAAAAAKNBIFCAA|2017-11-16|1414|6151|472|2017|4|11|16|4|2017|472|6151|Thursday|2017Q4|N|N|N|2458059|2458362|2457709|2457982|N|N|N|N|N| +2458075|AAAAAAAALNBIFCAA|2017-11-17|1414|6151|472|2017|5|11|17|4|2017|472|6151|Friday|2017Q4|N|Y|N|2458059|2458362|2457710|2457983|N|N|N|N|N| +2458076|AAAAAAAAMNBIFCAA|2017-11-18|1414|6151|472|2017|6|11|18|4|2017|472|6151|Saturday|2017Q4|N|Y|N|2458059|2458362|2457711|2457984|N|N|N|N|N| +2458077|AAAAAAAANNBIFCAA|2017-11-19|1414|6151|472|2017|0|11|19|4|2017|472|6151|Sunday|2017Q4|N|N|N|2458059|2458362|2457712|2457985|N|N|N|N|N| +2458078|AAAAAAAAONBIFCAA|2017-11-20|1414|6151|472|2017|1|11|20|4|2017|472|6151|Monday|2017Q4|N|N|N|2458059|2458362|2457713|2457986|N|N|N|N|N| +2458079|AAAAAAAAPNBIFCAA|2017-11-21|1414|6152|472|2017|2|11|21|4|2017|472|6152|Tuesday|2017Q4|N|N|N|2458059|2458362|2457714|2457987|N|N|N|N|N| +2458080|AAAAAAAAAOBIFCAA|2017-11-22|1414|6152|472|2017|3|11|22|4|2017|472|6152|Wednesday|2017Q4|N|N|N|2458059|2458362|2457715|2457988|N|N|N|N|N| +2458081|AAAAAAAABOBIFCAA|2017-11-23|1414|6152|472|2017|4|11|23|4|2017|472|6152|Thursday|2017Q4|N|N|N|2458059|2458362|2457716|2457989|N|N|N|N|N| +2458082|AAAAAAAACOBIFCAA|2017-11-24|1414|6152|472|2017|5|11|24|4|2017|472|6152|Friday|2017Q4|N|Y|N|2458059|2458362|2457717|2457990|N|N|N|N|N| +2458083|AAAAAAAADOBIFCAA|2017-11-25|1414|6152|472|2017|6|11|25|4|2017|472|6152|Saturday|2017Q4|N|Y|N|2458059|2458362|2457718|2457991|N|N|N|N|N| +2458084|AAAAAAAAEOBIFCAA|2017-11-26|1414|6152|472|2017|0|11|26|4|2017|472|6152|Sunday|2017Q4|N|N|N|2458059|2458362|2457719|2457992|N|N|N|N|N| +2458085|AAAAAAAAFOBIFCAA|2017-11-27|1414|6152|472|2017|1|11|27|4|2017|472|6152|Monday|2017Q4|N|N|N|2458059|2458362|2457720|2457993|N|N|N|N|N| +2458086|AAAAAAAAGOBIFCAA|2017-11-28|1414|6153|472|2017|2|11|28|4|2017|472|6153|Tuesday|2017Q4|N|N|N|2458059|2458362|2457721|2457994|N|N|N|N|N| +2458087|AAAAAAAAHOBIFCAA|2017-11-29|1414|6153|472|2017|3|11|29|4|2017|472|6153|Wednesday|2017Q4|N|N|N|2458059|2458362|2457722|2457995|N|N|N|N|N| +2458088|AAAAAAAAIOBIFCAA|2017-11-30|1414|6153|472|2017|4|11|30|4|2017|472|6153|Thursday|2017Q4|N|N|N|2458059|2458362|2457723|2457996|N|N|N|N|N| +2458089|AAAAAAAAJOBIFCAA|2017-12-01|1415|6153|473|2017|5|12|1|4|2017|473|6153|Friday|2017Q4|N|Y|N|2458089|2458422|2457724|2457997|N|N|N|N|N| +2458090|AAAAAAAAKOBIFCAA|2017-12-02|1415|6153|473|2017|6|12|2|4|2017|473|6153|Saturday|2017Q4|N|Y|N|2458089|2458422|2457725|2457998|N|N|N|N|N| +2458091|AAAAAAAALOBIFCAA|2017-12-03|1415|6153|473|2017|0|12|3|4|2017|473|6153|Sunday|2017Q4|N|N|N|2458089|2458422|2457726|2457999|N|N|N|N|N| +2458092|AAAAAAAAMOBIFCAA|2017-12-04|1415|6153|473|2017|1|12|4|4|2017|473|6153|Monday|2017Q4|N|N|N|2458089|2458422|2457727|2458000|N|N|N|N|N| +2458093|AAAAAAAANOBIFCAA|2017-12-05|1415|6154|473|2017|2|12|5|4|2017|473|6154|Tuesday|2017Q4|N|N|N|2458089|2458422|2457728|2458001|N|N|N|N|N| +2458094|AAAAAAAAOOBIFCAA|2017-12-06|1415|6154|473|2017|3|12|6|4|2017|473|6154|Wednesday|2017Q4|N|N|N|2458089|2458422|2457729|2458002|N|N|N|N|N| +2458095|AAAAAAAAPOBIFCAA|2017-12-07|1415|6154|473|2017|4|12|7|4|2017|473|6154|Thursday|2017Q4|N|N|N|2458089|2458422|2457730|2458003|N|N|N|N|N| +2458096|AAAAAAAAAPBIFCAA|2017-12-08|1415|6154|473|2017|5|12|8|4|2017|473|6154|Friday|2017Q4|N|Y|N|2458089|2458422|2457731|2458004|N|N|N|N|N| +2458097|AAAAAAAABPBIFCAA|2017-12-09|1415|6154|473|2017|6|12|9|4|2017|473|6154|Saturday|2017Q4|N|Y|N|2458089|2458422|2457732|2458005|N|N|N|N|N| +2458098|AAAAAAAACPBIFCAA|2017-12-10|1415|6154|473|2017|0|12|10|4|2017|473|6154|Sunday|2017Q4|N|N|N|2458089|2458422|2457733|2458006|N|N|N|N|N| +2458099|AAAAAAAADPBIFCAA|2017-12-11|1415|6154|473|2017|1|12|11|4|2017|473|6154|Monday|2017Q4|N|N|N|2458089|2458422|2457734|2458007|N|N|N|N|N| +2458100|AAAAAAAAEPBIFCAA|2017-12-12|1415|6155|473|2017|2|12|12|4|2017|473|6155|Tuesday|2017Q4|N|N|N|2458089|2458422|2457735|2458008|N|N|N|N|N| +2458101|AAAAAAAAFPBIFCAA|2017-12-13|1415|6155|473|2017|3|12|13|4|2017|473|6155|Wednesday|2017Q4|N|N|N|2458089|2458422|2457736|2458009|N|N|N|N|N| +2458102|AAAAAAAAGPBIFCAA|2017-12-14|1415|6155|473|2017|4|12|14|4|2017|473|6155|Thursday|2017Q4|N|N|N|2458089|2458422|2457737|2458010|N|N|N|N|N| +2458103|AAAAAAAAHPBIFCAA|2017-12-15|1415|6155|473|2017|5|12|15|4|2017|473|6155|Friday|2017Q4|N|Y|N|2458089|2458422|2457738|2458011|N|N|N|N|N| +2458104|AAAAAAAAIPBIFCAA|2017-12-16|1415|6155|473|2017|6|12|16|4|2017|473|6155|Saturday|2017Q4|N|Y|N|2458089|2458422|2457739|2458012|N|N|N|N|N| +2458105|AAAAAAAAJPBIFCAA|2017-12-17|1415|6155|473|2017|0|12|17|4|2017|473|6155|Sunday|2017Q4|N|N|N|2458089|2458422|2457740|2458013|N|N|N|N|N| +2458106|AAAAAAAAKPBIFCAA|2017-12-18|1415|6155|473|2017|1|12|18|4|2017|473|6155|Monday|2017Q4|N|N|N|2458089|2458422|2457741|2458014|N|N|N|N|N| +2458107|AAAAAAAALPBIFCAA|2017-12-19|1415|6156|473|2017|2|12|19|4|2017|473|6156|Tuesday|2017Q4|N|N|N|2458089|2458422|2457742|2458015|N|N|N|N|N| +2458108|AAAAAAAAMPBIFCAA|2017-12-20|1415|6156|473|2017|3|12|20|4|2017|473|6156|Wednesday|2017Q4|N|N|N|2458089|2458422|2457743|2458016|N|N|N|N|N| +2458109|AAAAAAAANPBIFCAA|2017-12-21|1415|6156|473|2017|4|12|21|4|2017|473|6156|Thursday|2017Q4|N|N|N|2458089|2458422|2457744|2458017|N|N|N|N|N| +2458110|AAAAAAAAOPBIFCAA|2017-12-22|1415|6156|473|2017|5|12|22|4|2017|473|6156|Friday|2017Q4|N|Y|N|2458089|2458422|2457745|2458018|N|N|N|N|N| +2458111|AAAAAAAAPPBIFCAA|2017-12-23|1415|6156|473|2017|6|12|23|4|2017|473|6156|Saturday|2017Q4|N|Y|N|2458089|2458422|2457746|2458019|N|N|N|N|N| +2458112|AAAAAAAAAACIFCAA|2017-12-24|1415|6156|473|2017|0|12|24|4|2017|473|6156|Sunday|2017Q4|N|N|N|2458089|2458422|2457747|2458020|N|N|N|N|N| +2458113|AAAAAAAABACIFCAA|2017-12-25|1415|6156|473|2017|1|12|25|4|2017|473|6156|Monday|2017Q4|N|N|N|2458089|2458422|2457748|2458021|N|N|N|N|N| +2458114|AAAAAAAACACIFCAA|2017-12-26|1415|6157|473|2017|2|12|26|4|2017|473|6157|Tuesday|2017Q4|Y|N|N|2458089|2458422|2457749|2458022|N|N|N|N|N| +2458115|AAAAAAAADACIFCAA|2017-12-27|1415|6157|473|2017|3|12|27|4|2017|473|6157|Wednesday|2017Q4|N|N|Y|2458089|2458422|2457750|2458023|N|N|N|N|N| +2458116|AAAAAAAAEACIFCAA|2017-12-28|1415|6157|473|2017|4|12|28|4|2017|473|6157|Thursday|2017Q4|N|N|N|2458089|2458422|2457751|2458024|N|N|N|N|N| +2458117|AAAAAAAAFACIFCAA|2017-12-29|1415|6157|473|2017|5|12|29|4|2017|473|6157|Friday|2017Q4|N|Y|N|2458089|2458422|2457752|2458025|N|N|N|N|N| +2458118|AAAAAAAAGACIFCAA|2017-12-30|1415|6157|473|2017|6|12|30|4|2017|473|6157|Saturday|2017Q4|N|Y|N|2458089|2458422|2457753|2458026|N|N|N|N|N| +2458119|AAAAAAAAHACIFCAA|2017-12-31|1415|6157|473|2017|0|12|31|4|2017|473|6157|Sunday|2017Q4|N|N|N|2458089|2458422|2457754|2458027|N|N|N|N|N| +2458120|AAAAAAAAIACIFCAA|2018-01-01|1416|6157|473|2018|1|1|1|1|2018|473|6157|Monday|2018Q1|Y|N|N|2458120|2458119|2457755|2458028|N|N|N|N|N| +2458121|AAAAAAAAJACIFCAA|2018-01-02|1416|6158|473|2018|2|1|2|1|2018|473|6158|Tuesday|2018Q1|N|N|Y|2458120|2458119|2457756|2458029|N|N|N|N|N| +2458122|AAAAAAAAKACIFCAA|2018-01-03|1416|6158|473|2018|3|1|3|1|2018|473|6158|Wednesday|2018Q1|N|N|N|2458120|2458119|2457757|2458030|N|N|N|N|N| +2458123|AAAAAAAALACIFCAA|2018-01-04|1416|6158|473|2018|4|1|4|1|2018|473|6158|Thursday|2018Q1|N|N|N|2458120|2458119|2457758|2458031|N|N|N|N|N| +2458124|AAAAAAAAMACIFCAA|2018-01-05|1416|6158|473|2018|5|1|5|1|2018|473|6158|Friday|2018Q1|N|Y|N|2458120|2458119|2457759|2458032|N|N|N|N|N| +2458125|AAAAAAAANACIFCAA|2018-01-06|1416|6158|473|2018|6|1|6|1|2018|473|6158|Saturday|2018Q1|N|Y|N|2458120|2458119|2457760|2458033|N|N|N|N|N| +2458126|AAAAAAAAOACIFCAA|2018-01-07|1416|6158|473|2018|0|1|7|1|2018|473|6158|Sunday|2018Q1|N|N|N|2458120|2458119|2457761|2458034|N|N|N|N|N| +2458127|AAAAAAAAPACIFCAA|2018-01-08|1416|6158|473|2018|1|1|8|1|2018|473|6158|Monday|2018Q1|N|N|N|2458120|2458119|2457762|2458035|N|N|N|N|N| +2458128|AAAAAAAAABCIFCAA|2018-01-09|1416|6159|473|2018|2|1|9|1|2018|473|6159|Tuesday|2018Q1|N|N|N|2458120|2458119|2457763|2458036|N|N|N|N|N| +2458129|AAAAAAAABBCIFCAA|2018-01-10|1416|6159|473|2018|3|1|10|1|2018|473|6159|Wednesday|2018Q1|N|N|N|2458120|2458119|2457764|2458037|N|N|N|N|N| +2458130|AAAAAAAACBCIFCAA|2018-01-11|1416|6159|473|2018|4|1|11|1|2018|473|6159|Thursday|2018Q1|N|N|N|2458120|2458119|2457765|2458038|N|N|N|N|N| +2458131|AAAAAAAADBCIFCAA|2018-01-12|1416|6159|473|2018|5|1|12|1|2018|473|6159|Friday|2018Q1|N|Y|N|2458120|2458119|2457766|2458039|N|N|N|N|N| +2458132|AAAAAAAAEBCIFCAA|2018-01-13|1416|6159|473|2018|6|1|13|1|2018|473|6159|Saturday|2018Q1|N|Y|N|2458120|2458119|2457767|2458040|N|N|N|N|N| +2458133|AAAAAAAAFBCIFCAA|2018-01-14|1416|6159|473|2018|0|1|14|1|2018|473|6159|Sunday|2018Q1|N|N|N|2458120|2458119|2457768|2458041|N|N|N|N|N| +2458134|AAAAAAAAGBCIFCAA|2018-01-15|1416|6159|473|2018|1|1|15|1|2018|473|6159|Monday|2018Q1|N|N|N|2458120|2458119|2457769|2458042|N|N|N|N|N| +2458135|AAAAAAAAHBCIFCAA|2018-01-16|1416|6160|473|2018|2|1|16|1|2018|473|6160|Tuesday|2018Q1|N|N|N|2458120|2458119|2457770|2458043|N|N|N|N|N| +2458136|AAAAAAAAIBCIFCAA|2018-01-17|1416|6160|473|2018|3|1|17|1|2018|473|6160|Wednesday|2018Q1|N|N|N|2458120|2458119|2457771|2458044|N|N|N|N|N| +2458137|AAAAAAAAJBCIFCAA|2018-01-18|1416|6160|473|2018|4|1|18|1|2018|473|6160|Thursday|2018Q1|N|N|N|2458120|2458119|2457772|2458045|N|N|N|N|N| +2458138|AAAAAAAAKBCIFCAA|2018-01-19|1416|6160|473|2018|5|1|19|1|2018|473|6160|Friday|2018Q1|N|Y|N|2458120|2458119|2457773|2458046|N|N|N|N|N| +2458139|AAAAAAAALBCIFCAA|2018-01-20|1416|6160|473|2018|6|1|20|1|2018|473|6160|Saturday|2018Q1|N|Y|N|2458120|2458119|2457774|2458047|N|N|N|N|N| +2458140|AAAAAAAAMBCIFCAA|2018-01-21|1416|6160|473|2018|0|1|21|1|2018|473|6160|Sunday|2018Q1|N|N|N|2458120|2458119|2457775|2458048|N|N|N|N|N| +2458141|AAAAAAAANBCIFCAA|2018-01-22|1416|6160|473|2018|1|1|22|1|2018|473|6160|Monday|2018Q1|N|N|N|2458120|2458119|2457776|2458049|N|N|N|N|N| +2458142|AAAAAAAAOBCIFCAA|2018-01-23|1416|6161|473|2018|2|1|23|1|2018|473|6161|Tuesday|2018Q1|N|N|N|2458120|2458119|2457777|2458050|N|N|N|N|N| +2458143|AAAAAAAAPBCIFCAA|2018-01-24|1416|6161|473|2018|3|1|24|1|2018|473|6161|Wednesday|2018Q1|N|N|N|2458120|2458119|2457778|2458051|N|N|N|N|N| +2458144|AAAAAAAAACCIFCAA|2018-01-25|1416|6161|473|2018|4|1|25|1|2018|473|6161|Thursday|2018Q1|N|N|N|2458120|2458119|2457779|2458052|N|N|N|N|N| +2458145|AAAAAAAABCCIFCAA|2018-01-26|1416|6161|473|2018|5|1|26|1|2018|473|6161|Friday|2018Q1|N|Y|N|2458120|2458119|2457780|2458053|N|N|N|N|N| +2458146|AAAAAAAACCCIFCAA|2018-01-27|1416|6161|473|2018|6|1|27|1|2018|473|6161|Saturday|2018Q1|N|Y|N|2458120|2458119|2457781|2458054|N|N|N|N|N| +2458147|AAAAAAAADCCIFCAA|2018-01-28|1416|6161|473|2018|0|1|28|1|2018|473|6161|Sunday|2018Q1|N|N|N|2458120|2458119|2457782|2458055|N|N|N|N|N| +2458148|AAAAAAAAECCIFCAA|2018-01-29|1416|6161|473|2018|1|1|29|1|2018|473|6161|Monday|2018Q1|N|N|N|2458120|2458119|2457783|2458056|N|N|N|N|N| +2458149|AAAAAAAAFCCIFCAA|2018-01-30|1416|6162|473|2018|2|1|30|1|2018|473|6162|Tuesday|2018Q1|N|N|N|2458120|2458119|2457784|2458057|N|N|N|N|N| +2458150|AAAAAAAAGCCIFCAA|2018-01-31|1416|6162|473|2018|3|1|31|1|2018|473|6162|Wednesday|2018Q1|N|N|N|2458120|2458119|2457785|2458058|N|N|N|N|N| +2458151|AAAAAAAAHCCIFCAA|2018-02-01|1417|6162|473|2018|4|2|1|1|2018|473|6162|Thursday|2018Q1|N|N|N|2458151|2458181|2457786|2458059|N|N|N|N|N| +2458152|AAAAAAAAICCIFCAA|2018-02-02|1417|6162|473|2018|5|2|2|1|2018|473|6162|Friday|2018Q1|N|Y|N|2458151|2458181|2457787|2458060|N|N|N|N|N| +2458153|AAAAAAAAJCCIFCAA|2018-02-03|1417|6162|473|2018|6|2|3|1|2018|473|6162|Saturday|2018Q1|N|Y|N|2458151|2458181|2457788|2458061|N|N|N|N|N| +2458154|AAAAAAAAKCCIFCAA|2018-02-04|1417|6162|473|2018|0|2|4|1|2018|473|6162|Sunday|2018Q1|N|N|N|2458151|2458181|2457789|2458062|N|N|N|N|N| +2458155|AAAAAAAALCCIFCAA|2018-02-05|1417|6162|473|2018|1|2|5|1|2018|473|6162|Monday|2018Q1|N|N|N|2458151|2458181|2457790|2458063|N|N|N|N|N| +2458156|AAAAAAAAMCCIFCAA|2018-02-06|1417|6163|473|2018|2|2|6|1|2018|473|6163|Tuesday|2018Q1|N|N|N|2458151|2458181|2457791|2458064|N|N|N|N|N| +2458157|AAAAAAAANCCIFCAA|2018-02-07|1417|6163|473|2018|3|2|7|1|2018|473|6163|Wednesday|2018Q1|N|N|N|2458151|2458181|2457792|2458065|N|N|N|N|N| +2458158|AAAAAAAAOCCIFCAA|2018-02-08|1417|6163|473|2018|4|2|8|1|2018|473|6163|Thursday|2018Q1|N|N|N|2458151|2458181|2457793|2458066|N|N|N|N|N| +2458159|AAAAAAAAPCCIFCAA|2018-02-09|1417|6163|473|2018|5|2|9|1|2018|473|6163|Friday|2018Q1|N|Y|N|2458151|2458181|2457794|2458067|N|N|N|N|N| +2458160|AAAAAAAAADCIFCAA|2018-02-10|1417|6163|473|2018|6|2|10|1|2018|473|6163|Saturday|2018Q1|N|Y|N|2458151|2458181|2457795|2458068|N|N|N|N|N| +2458161|AAAAAAAABDCIFCAA|2018-02-11|1417|6163|473|2018|0|2|11|1|2018|473|6163|Sunday|2018Q1|N|N|N|2458151|2458181|2457796|2458069|N|N|N|N|N| +2458162|AAAAAAAACDCIFCAA|2018-02-12|1417|6163|473|2018|1|2|12|1|2018|473|6163|Monday|2018Q1|N|N|N|2458151|2458181|2457797|2458070|N|N|N|N|N| +2458163|AAAAAAAADDCIFCAA|2018-02-13|1417|6164|473|2018|2|2|13|1|2018|473|6164|Tuesday|2018Q1|N|N|N|2458151|2458181|2457798|2458071|N|N|N|N|N| +2458164|AAAAAAAAEDCIFCAA|2018-02-14|1417|6164|473|2018|3|2|14|1|2018|473|6164|Wednesday|2018Q1|N|N|N|2458151|2458181|2457799|2458072|N|N|N|N|N| +2458165|AAAAAAAAFDCIFCAA|2018-02-15|1417|6164|473|2018|4|2|15|1|2018|473|6164|Thursday|2018Q1|N|N|N|2458151|2458181|2457800|2458073|N|N|N|N|N| +2458166|AAAAAAAAGDCIFCAA|2018-02-16|1417|6164|473|2018|5|2|16|1|2018|473|6164|Friday|2018Q1|N|Y|N|2458151|2458181|2457801|2458074|N|N|N|N|N| +2458167|AAAAAAAAHDCIFCAA|2018-02-17|1417|6164|473|2018|6|2|17|1|2018|473|6164|Saturday|2018Q1|N|Y|N|2458151|2458181|2457802|2458075|N|N|N|N|N| +2458168|AAAAAAAAIDCIFCAA|2018-02-18|1417|6164|473|2018|0|2|18|1|2018|473|6164|Sunday|2018Q1|N|N|N|2458151|2458181|2457803|2458076|N|N|N|N|N| +2458169|AAAAAAAAJDCIFCAA|2018-02-19|1417|6164|473|2018|1|2|19|1|2018|473|6164|Monday|2018Q1|N|N|N|2458151|2458181|2457804|2458077|N|N|N|N|N| +2458170|AAAAAAAAKDCIFCAA|2018-02-20|1417|6165|473|2018|2|2|20|1|2018|473|6165|Tuesday|2018Q1|N|N|N|2458151|2458181|2457805|2458078|N|N|N|N|N| +2458171|AAAAAAAALDCIFCAA|2018-02-21|1417|6165|473|2018|3|2|21|1|2018|473|6165|Wednesday|2018Q1|N|N|N|2458151|2458181|2457806|2458079|N|N|N|N|N| +2458172|AAAAAAAAMDCIFCAA|2018-02-22|1417|6165|473|2018|4|2|22|1|2018|473|6165|Thursday|2018Q1|N|N|N|2458151|2458181|2457807|2458080|N|N|N|N|N| +2458173|AAAAAAAANDCIFCAA|2018-02-23|1417|6165|473|2018|5|2|23|1|2018|473|6165|Friday|2018Q1|N|Y|N|2458151|2458181|2457808|2458081|N|N|N|N|N| +2458174|AAAAAAAAODCIFCAA|2018-02-24|1417|6165|473|2018|6|2|24|1|2018|473|6165|Saturday|2018Q1|N|Y|N|2458151|2458181|2457809|2458082|N|N|N|N|N| +2458175|AAAAAAAAPDCIFCAA|2018-02-25|1417|6165|473|2018|0|2|25|1|2018|473|6165|Sunday|2018Q1|N|N|N|2458151|2458181|2457810|2458083|N|N|N|N|N| +2458176|AAAAAAAAAECIFCAA|2018-02-26|1417|6165|473|2018|1|2|26|1|2018|473|6165|Monday|2018Q1|N|N|N|2458151|2458181|2457811|2458084|N|N|N|N|N| +2458177|AAAAAAAABECIFCAA|2018-02-27|1417|6166|473|2018|2|2|27|1|2018|473|6166|Tuesday|2018Q1|N|N|N|2458151|2458181|2457812|2458085|N|N|N|N|N| +2458178|AAAAAAAACECIFCAA|2018-02-28|1417|6166|473|2018|3|2|28|1|2018|473|6166|Wednesday|2018Q1|N|N|N|2458151|2458181|2457813|2458086|N|N|N|N|N| +2458179|AAAAAAAADECIFCAA|2018-03-01|1418|6166|474|2018|4|3|1|1|2018|474|6166|Thursday|2018Q1|N|N|N|2458179|2458237|2457814|2458087|N|N|N|N|N| +2458180|AAAAAAAAEECIFCAA|2018-03-02|1418|6166|474|2018|5|3|2|1|2018|474|6166|Friday|2018Q1|N|Y|N|2458179|2458237|2457815|2458088|N|N|N|N|N| +2458181|AAAAAAAAFECIFCAA|2018-03-03|1418|6166|474|2018|6|3|3|1|2018|474|6166|Saturday|2018Q1|N|Y|N|2458179|2458237|2457816|2458089|N|N|N|N|N| +2458182|AAAAAAAAGECIFCAA|2018-03-04|1418|6166|474|2018|0|3|4|1|2018|474|6166|Sunday|2018Q1|N|N|N|2458179|2458237|2457817|2458090|N|N|N|N|N| +2458183|AAAAAAAAHECIFCAA|2018-03-05|1418|6166|474|2018|1|3|5|1|2018|474|6166|Monday|2018Q1|N|N|N|2458179|2458237|2457818|2458091|N|N|N|N|N| +2458184|AAAAAAAAIECIFCAA|2018-03-06|1418|6167|474|2018|2|3|6|1|2018|474|6167|Tuesday|2018Q1|N|N|N|2458179|2458237|2457819|2458092|N|N|N|N|N| +2458185|AAAAAAAAJECIFCAA|2018-03-07|1418|6167|474|2018|3|3|7|1|2018|474|6167|Wednesday|2018Q1|N|N|N|2458179|2458237|2457820|2458093|N|N|N|N|N| +2458186|AAAAAAAAKECIFCAA|2018-03-08|1418|6167|474|2018|4|3|8|1|2018|474|6167|Thursday|2018Q1|N|N|N|2458179|2458237|2457821|2458094|N|N|N|N|N| +2458187|AAAAAAAALECIFCAA|2018-03-09|1418|6167|474|2018|5|3|9|1|2018|474|6167|Friday|2018Q1|N|Y|N|2458179|2458237|2457822|2458095|N|N|N|N|N| +2458188|AAAAAAAAMECIFCAA|2018-03-10|1418|6167|474|2018|6|3|10|1|2018|474|6167|Saturday|2018Q1|N|Y|N|2458179|2458237|2457823|2458096|N|N|N|N|N| +2458189|AAAAAAAANECIFCAA|2018-03-11|1418|6167|474|2018|0|3|11|1|2018|474|6167|Sunday|2018Q1|N|N|N|2458179|2458237|2457824|2458097|N|N|N|N|N| +2458190|AAAAAAAAOECIFCAA|2018-03-12|1418|6167|474|2018|1|3|12|1|2018|474|6167|Monday|2018Q1|N|N|N|2458179|2458237|2457825|2458098|N|N|N|N|N| +2458191|AAAAAAAAPECIFCAA|2018-03-13|1418|6168|474|2018|2|3|13|1|2018|474|6168|Tuesday|2018Q1|N|N|N|2458179|2458237|2457826|2458099|N|N|N|N|N| +2458192|AAAAAAAAAFCIFCAA|2018-03-14|1418|6168|474|2018|3|3|14|1|2018|474|6168|Wednesday|2018Q1|N|N|N|2458179|2458237|2457827|2458100|N|N|N|N|N| +2458193|AAAAAAAABFCIFCAA|2018-03-15|1418|6168|474|2018|4|3|15|1|2018|474|6168|Thursday|2018Q1|N|N|N|2458179|2458237|2457828|2458101|N|N|N|N|N| +2458194|AAAAAAAACFCIFCAA|2018-03-16|1418|6168|474|2018|5|3|16|1|2018|474|6168|Friday|2018Q1|N|Y|N|2458179|2458237|2457829|2458102|N|N|N|N|N| +2458195|AAAAAAAADFCIFCAA|2018-03-17|1418|6168|474|2018|6|3|17|1|2018|474|6168|Saturday|2018Q1|N|Y|N|2458179|2458237|2457830|2458103|N|N|N|N|N| +2458196|AAAAAAAAEFCIFCAA|2018-03-18|1418|6168|474|2018|0|3|18|1|2018|474|6168|Sunday|2018Q1|N|N|N|2458179|2458237|2457831|2458104|N|N|N|N|N| +2458197|AAAAAAAAFFCIFCAA|2018-03-19|1418|6168|474|2018|1|3|19|1|2018|474|6168|Monday|2018Q1|N|N|N|2458179|2458237|2457832|2458105|N|N|N|N|N| +2458198|AAAAAAAAGFCIFCAA|2018-03-20|1418|6169|474|2018|2|3|20|1|2018|474|6169|Tuesday|2018Q1|N|N|N|2458179|2458237|2457833|2458106|N|N|N|N|N| +2458199|AAAAAAAAHFCIFCAA|2018-03-21|1418|6169|474|2018|3|3|21|1|2018|474|6169|Wednesday|2018Q1|N|N|N|2458179|2458237|2457834|2458107|N|N|N|N|N| +2458200|AAAAAAAAIFCIFCAA|2018-03-22|1418|6169|474|2018|4|3|22|1|2018|474|6169|Thursday|2018Q1|N|N|N|2458179|2458237|2457835|2458108|N|N|N|N|N| +2458201|AAAAAAAAJFCIFCAA|2018-03-23|1418|6169|474|2018|5|3|23|1|2018|474|6169|Friday|2018Q1|N|Y|N|2458179|2458237|2457836|2458109|N|N|N|N|N| +2458202|AAAAAAAAKFCIFCAA|2018-03-24|1418|6169|474|2018|6|3|24|1|2018|474|6169|Saturday|2018Q1|N|Y|N|2458179|2458237|2457837|2458110|N|N|N|N|N| +2458203|AAAAAAAALFCIFCAA|2018-03-25|1418|6169|474|2018|0|3|25|1|2018|474|6169|Sunday|2018Q1|N|N|N|2458179|2458237|2457838|2458111|N|N|N|N|N| +2458204|AAAAAAAAMFCIFCAA|2018-03-26|1418|6169|474|2018|1|3|26|1|2018|474|6169|Monday|2018Q1|N|N|N|2458179|2458237|2457839|2458112|N|N|N|N|N| +2458205|AAAAAAAANFCIFCAA|2018-03-27|1418|6170|474|2018|2|3|27|1|2018|474|6170|Tuesday|2018Q1|N|N|N|2458179|2458237|2457840|2458113|N|N|N|N|N| +2458206|AAAAAAAAOFCIFCAA|2018-03-28|1418|6170|474|2018|3|3|28|1|2018|474|6170|Wednesday|2018Q1|N|N|N|2458179|2458237|2457841|2458114|N|N|N|N|N| +2458207|AAAAAAAAPFCIFCAA|2018-03-29|1418|6170|474|2018|4|3|29|1|2018|474|6170|Thursday|2018Q1|N|N|N|2458179|2458237|2457842|2458115|N|N|N|N|N| +2458208|AAAAAAAAAGCIFCAA|2018-03-30|1418|6170|474|2018|5|3|30|1|2018|474|6170|Friday|2018Q1|N|Y|N|2458179|2458237|2457843|2458116|N|N|N|N|N| +2458209|AAAAAAAABGCIFCAA|2018-03-31|1418|6170|474|2018|6|3|31|1|2018|474|6170|Saturday|2018Q1|N|Y|N|2458179|2458237|2457844|2458117|N|N|N|N|N| +2458210|AAAAAAAACGCIFCAA|2018-04-01|1419|6170|474|2018|0|4|1|1|2018|474|6170|Sunday|2018Q1|N|N|N|2458210|2458299|2457845|2458120|N|N|N|N|N| +2458211|AAAAAAAADGCIFCAA|2018-04-02|1419|6170|474|2018|1|4|2|2|2018|474|6170|Monday|2018Q2|N|N|N|2458210|2458299|2457846|2458121|N|N|N|N|N| +2458212|AAAAAAAAEGCIFCAA|2018-04-03|1419|6171|474|2018|2|4|3|2|2018|474|6171|Tuesday|2018Q2|N|N|N|2458210|2458299|2457847|2458122|N|N|N|N|N| +2458213|AAAAAAAAFGCIFCAA|2018-04-04|1419|6171|474|2018|3|4|4|2|2018|474|6171|Wednesday|2018Q2|N|N|N|2458210|2458299|2457848|2458123|N|N|N|N|N| +2458214|AAAAAAAAGGCIFCAA|2018-04-05|1419|6171|474|2018|4|4|5|2|2018|474|6171|Thursday|2018Q2|N|N|N|2458210|2458299|2457849|2458124|N|N|N|N|N| +2458215|AAAAAAAAHGCIFCAA|2018-04-06|1419|6171|474|2018|5|4|6|2|2018|474|6171|Friday|2018Q2|N|Y|N|2458210|2458299|2457850|2458125|N|N|N|N|N| +2458216|AAAAAAAAIGCIFCAA|2018-04-07|1419|6171|474|2018|6|4|7|2|2018|474|6171|Saturday|2018Q2|N|Y|N|2458210|2458299|2457851|2458126|N|N|N|N|N| +2458217|AAAAAAAAJGCIFCAA|2018-04-08|1419|6171|474|2018|0|4|8|2|2018|474|6171|Sunday|2018Q2|N|N|N|2458210|2458299|2457852|2458127|N|N|N|N|N| +2458218|AAAAAAAAKGCIFCAA|2018-04-09|1419|6171|474|2018|1|4|9|2|2018|474|6171|Monday|2018Q2|N|N|N|2458210|2458299|2457853|2458128|N|N|N|N|N| +2458219|AAAAAAAALGCIFCAA|2018-04-10|1419|6172|474|2018|2|4|10|2|2018|474|6172|Tuesday|2018Q2|N|N|N|2458210|2458299|2457854|2458129|N|N|N|N|N| +2458220|AAAAAAAAMGCIFCAA|2018-04-11|1419|6172|474|2018|3|4|11|2|2018|474|6172|Wednesday|2018Q2|N|N|N|2458210|2458299|2457855|2458130|N|N|N|N|N| +2458221|AAAAAAAANGCIFCAA|2018-04-12|1419|6172|474|2018|4|4|12|2|2018|474|6172|Thursday|2018Q2|N|N|N|2458210|2458299|2457856|2458131|N|N|N|N|N| +2458222|AAAAAAAAOGCIFCAA|2018-04-13|1419|6172|474|2018|5|4|13|2|2018|474|6172|Friday|2018Q2|N|Y|N|2458210|2458299|2457857|2458132|N|N|N|N|N| +2458223|AAAAAAAAPGCIFCAA|2018-04-14|1419|6172|474|2018|6|4|14|2|2018|474|6172|Saturday|2018Q2|N|Y|N|2458210|2458299|2457858|2458133|N|N|N|N|N| +2458224|AAAAAAAAAHCIFCAA|2018-04-15|1419|6172|474|2018|0|4|15|2|2018|474|6172|Sunday|2018Q2|N|N|N|2458210|2458299|2457859|2458134|N|N|N|N|N| +2458225|AAAAAAAABHCIFCAA|2018-04-16|1419|6172|474|2018|1|4|16|2|2018|474|6172|Monday|2018Q2|N|N|N|2458210|2458299|2457860|2458135|N|N|N|N|N| +2458226|AAAAAAAACHCIFCAA|2018-04-17|1419|6173|474|2018|2|4|17|2|2018|474|6173|Tuesday|2018Q2|N|N|N|2458210|2458299|2457861|2458136|N|N|N|N|N| +2458227|AAAAAAAADHCIFCAA|2018-04-18|1419|6173|474|2018|3|4|18|2|2018|474|6173|Wednesday|2018Q2|N|N|N|2458210|2458299|2457862|2458137|N|N|N|N|N| +2458228|AAAAAAAAEHCIFCAA|2018-04-19|1419|6173|474|2018|4|4|19|2|2018|474|6173|Thursday|2018Q2|N|N|N|2458210|2458299|2457863|2458138|N|N|N|N|N| +2458229|AAAAAAAAFHCIFCAA|2018-04-20|1419|6173|474|2018|5|4|20|2|2018|474|6173|Friday|2018Q2|N|Y|N|2458210|2458299|2457864|2458139|N|N|N|N|N| +2458230|AAAAAAAAGHCIFCAA|2018-04-21|1419|6173|474|2018|6|4|21|2|2018|474|6173|Saturday|2018Q2|N|Y|N|2458210|2458299|2457865|2458140|N|N|N|N|N| +2458231|AAAAAAAAHHCIFCAA|2018-04-22|1419|6173|474|2018|0|4|22|2|2018|474|6173|Sunday|2018Q2|N|N|N|2458210|2458299|2457866|2458141|N|N|N|N|N| +2458232|AAAAAAAAIHCIFCAA|2018-04-23|1419|6173|474|2018|1|4|23|2|2018|474|6173|Monday|2018Q2|N|N|N|2458210|2458299|2457867|2458142|N|N|N|N|N| +2458233|AAAAAAAAJHCIFCAA|2018-04-24|1419|6174|474|2018|2|4|24|2|2018|474|6174|Tuesday|2018Q2|N|N|N|2458210|2458299|2457868|2458143|N|N|N|N|N| +2458234|AAAAAAAAKHCIFCAA|2018-04-25|1419|6174|474|2018|3|4|25|2|2018|474|6174|Wednesday|2018Q2|N|N|N|2458210|2458299|2457869|2458144|N|N|N|N|N| +2458235|AAAAAAAALHCIFCAA|2018-04-26|1419|6174|474|2018|4|4|26|2|2018|474|6174|Thursday|2018Q2|N|N|N|2458210|2458299|2457870|2458145|N|N|N|N|N| +2458236|AAAAAAAAMHCIFCAA|2018-04-27|1419|6174|474|2018|5|4|27|2|2018|474|6174|Friday|2018Q2|N|Y|N|2458210|2458299|2457871|2458146|N|N|N|N|N| +2458237|AAAAAAAANHCIFCAA|2018-04-28|1419|6174|474|2018|6|4|28|2|2018|474|6174|Saturday|2018Q2|N|Y|N|2458210|2458299|2457872|2458147|N|N|N|N|N| +2458238|AAAAAAAAOHCIFCAA|2018-04-29|1419|6174|474|2018|0|4|29|2|2018|474|6174|Sunday|2018Q2|N|N|N|2458210|2458299|2457873|2458148|N|N|N|N|N| +2458239|AAAAAAAAPHCIFCAA|2018-04-30|1419|6174|474|2018|1|4|30|2|2018|474|6174|Monday|2018Q2|N|N|N|2458210|2458299|2457874|2458149|N|N|N|N|N| +2458240|AAAAAAAAAICIFCAA|2018-05-01|1420|6175|474|2018|2|5|1|2|2018|474|6175|Tuesday|2018Q2|N|N|N|2458240|2458359|2457875|2458150|N|N|N|N|N| +2458241|AAAAAAAABICIFCAA|2018-05-02|1420|6175|474|2018|3|5|2|2|2018|474|6175|Wednesday|2018Q2|N|N|N|2458240|2458359|2457876|2458151|N|N|N|N|N| +2458242|AAAAAAAACICIFCAA|2018-05-03|1420|6175|474|2018|4|5|3|2|2018|474|6175|Thursday|2018Q2|N|N|N|2458240|2458359|2457877|2458152|N|N|N|N|N| +2458243|AAAAAAAADICIFCAA|2018-05-04|1420|6175|474|2018|5|5|4|2|2018|474|6175|Friday|2018Q2|N|Y|N|2458240|2458359|2457878|2458153|N|N|N|N|N| +2458244|AAAAAAAAEICIFCAA|2018-05-05|1420|6175|474|2018|6|5|5|2|2018|474|6175|Saturday|2018Q2|N|Y|N|2458240|2458359|2457879|2458154|N|N|N|N|N| +2458245|AAAAAAAAFICIFCAA|2018-05-06|1420|6175|474|2018|0|5|6|2|2018|474|6175|Sunday|2018Q2|N|N|N|2458240|2458359|2457880|2458155|N|N|N|N|N| +2458246|AAAAAAAAGICIFCAA|2018-05-07|1420|6175|474|2018|1|5|7|2|2018|474|6175|Monday|2018Q2|N|N|N|2458240|2458359|2457881|2458156|N|N|N|N|N| +2458247|AAAAAAAAHICIFCAA|2018-05-08|1420|6176|474|2018|2|5|8|2|2018|474|6176|Tuesday|2018Q2|N|N|N|2458240|2458359|2457882|2458157|N|N|N|N|N| +2458248|AAAAAAAAIICIFCAA|2018-05-09|1420|6176|474|2018|3|5|9|2|2018|474|6176|Wednesday|2018Q2|N|N|N|2458240|2458359|2457883|2458158|N|N|N|N|N| +2458249|AAAAAAAAJICIFCAA|2018-05-10|1420|6176|474|2018|4|5|10|2|2018|474|6176|Thursday|2018Q2|N|N|N|2458240|2458359|2457884|2458159|N|N|N|N|N| +2458250|AAAAAAAAKICIFCAA|2018-05-11|1420|6176|474|2018|5|5|11|2|2018|474|6176|Friday|2018Q2|N|Y|N|2458240|2458359|2457885|2458160|N|N|N|N|N| +2458251|AAAAAAAALICIFCAA|2018-05-12|1420|6176|474|2018|6|5|12|2|2018|474|6176|Saturday|2018Q2|N|Y|N|2458240|2458359|2457886|2458161|N|N|N|N|N| +2458252|AAAAAAAAMICIFCAA|2018-05-13|1420|6176|474|2018|0|5|13|2|2018|474|6176|Sunday|2018Q2|N|N|N|2458240|2458359|2457887|2458162|N|N|N|N|N| +2458253|AAAAAAAANICIFCAA|2018-05-14|1420|6176|474|2018|1|5|14|2|2018|474|6176|Monday|2018Q2|N|N|N|2458240|2458359|2457888|2458163|N|N|N|N|N| +2458254|AAAAAAAAOICIFCAA|2018-05-15|1420|6177|474|2018|2|5|15|2|2018|474|6177|Tuesday|2018Q2|N|N|N|2458240|2458359|2457889|2458164|N|N|N|N|N| +2458255|AAAAAAAAPICIFCAA|2018-05-16|1420|6177|474|2018|3|5|16|2|2018|474|6177|Wednesday|2018Q2|N|N|N|2458240|2458359|2457890|2458165|N|N|N|N|N| +2458256|AAAAAAAAAJCIFCAA|2018-05-17|1420|6177|474|2018|4|5|17|2|2018|474|6177|Thursday|2018Q2|N|N|N|2458240|2458359|2457891|2458166|N|N|N|N|N| +2458257|AAAAAAAABJCIFCAA|2018-05-18|1420|6177|474|2018|5|5|18|2|2018|474|6177|Friday|2018Q2|N|Y|N|2458240|2458359|2457892|2458167|N|N|N|N|N| +2458258|AAAAAAAACJCIFCAA|2018-05-19|1420|6177|474|2018|6|5|19|2|2018|474|6177|Saturday|2018Q2|N|Y|N|2458240|2458359|2457893|2458168|N|N|N|N|N| +2458259|AAAAAAAADJCIFCAA|2018-05-20|1420|6177|474|2018|0|5|20|2|2018|474|6177|Sunday|2018Q2|N|N|N|2458240|2458359|2457894|2458169|N|N|N|N|N| +2458260|AAAAAAAAEJCIFCAA|2018-05-21|1420|6177|474|2018|1|5|21|2|2018|474|6177|Monday|2018Q2|N|N|N|2458240|2458359|2457895|2458170|N|N|N|N|N| +2458261|AAAAAAAAFJCIFCAA|2018-05-22|1420|6178|474|2018|2|5|22|2|2018|474|6178|Tuesday|2018Q2|N|N|N|2458240|2458359|2457896|2458171|N|N|N|N|N| +2458262|AAAAAAAAGJCIFCAA|2018-05-23|1420|6178|474|2018|3|5|23|2|2018|474|6178|Wednesday|2018Q2|N|N|N|2458240|2458359|2457897|2458172|N|N|N|N|N| +2458263|AAAAAAAAHJCIFCAA|2018-05-24|1420|6178|474|2018|4|5|24|2|2018|474|6178|Thursday|2018Q2|N|N|N|2458240|2458359|2457898|2458173|N|N|N|N|N| +2458264|AAAAAAAAIJCIFCAA|2018-05-25|1420|6178|474|2018|5|5|25|2|2018|474|6178|Friday|2018Q2|N|Y|N|2458240|2458359|2457899|2458174|N|N|N|N|N| +2458265|AAAAAAAAJJCIFCAA|2018-05-26|1420|6178|474|2018|6|5|26|2|2018|474|6178|Saturday|2018Q2|N|Y|N|2458240|2458359|2457900|2458175|N|N|N|N|N| +2458266|AAAAAAAAKJCIFCAA|2018-05-27|1420|6178|474|2018|0|5|27|2|2018|474|6178|Sunday|2018Q2|N|N|N|2458240|2458359|2457901|2458176|N|N|N|N|N| +2458267|AAAAAAAALJCIFCAA|2018-05-28|1420|6178|474|2018|1|5|28|2|2018|474|6178|Monday|2018Q2|N|N|N|2458240|2458359|2457902|2458177|N|N|N|N|N| +2458268|AAAAAAAAMJCIFCAA|2018-05-29|1420|6179|474|2018|2|5|29|2|2018|474|6179|Tuesday|2018Q2|N|N|N|2458240|2458359|2457903|2458178|N|N|N|N|N| +2458269|AAAAAAAANJCIFCAA|2018-05-30|1420|6179|474|2018|3|5|30|2|2018|474|6179|Wednesday|2018Q2|N|N|N|2458240|2458359|2457904|2458179|N|N|N|N|N| +2458270|AAAAAAAAOJCIFCAA|2018-05-31|1420|6179|474|2018|4|5|31|2|2018|474|6179|Thursday|2018Q2|N|N|N|2458240|2458359|2457905|2458180|N|N|N|N|N| +2458271|AAAAAAAAPJCIFCAA|2018-06-01|1421|6179|475|2018|5|6|1|2|2018|475|6179|Friday|2018Q2|N|Y|N|2458271|2458421|2457906|2458181|N|N|N|N|N| +2458272|AAAAAAAAAKCIFCAA|2018-06-02|1421|6179|475|2018|6|6|2|2|2018|475|6179|Saturday|2018Q2|N|Y|N|2458271|2458421|2457907|2458182|N|N|N|N|N| +2458273|AAAAAAAABKCIFCAA|2018-06-03|1421|6179|475|2018|0|6|3|2|2018|475|6179|Sunday|2018Q2|N|N|N|2458271|2458421|2457908|2458183|N|N|N|N|N| +2458274|AAAAAAAACKCIFCAA|2018-06-04|1421|6179|475|2018|1|6|4|2|2018|475|6179|Monday|2018Q2|N|N|N|2458271|2458421|2457909|2458184|N|N|N|N|N| +2458275|AAAAAAAADKCIFCAA|2018-06-05|1421|6180|475|2018|2|6|5|2|2018|475|6180|Tuesday|2018Q2|N|N|N|2458271|2458421|2457910|2458185|N|N|N|N|N| +2458276|AAAAAAAAEKCIFCAA|2018-06-06|1421|6180|475|2018|3|6|6|2|2018|475|6180|Wednesday|2018Q2|N|N|N|2458271|2458421|2457911|2458186|N|N|N|N|N| +2458277|AAAAAAAAFKCIFCAA|2018-06-07|1421|6180|475|2018|4|6|7|2|2018|475|6180|Thursday|2018Q2|N|N|N|2458271|2458421|2457912|2458187|N|N|N|N|N| +2458278|AAAAAAAAGKCIFCAA|2018-06-08|1421|6180|475|2018|5|6|8|2|2018|475|6180|Friday|2018Q2|N|Y|N|2458271|2458421|2457913|2458188|N|N|N|N|N| +2458279|AAAAAAAAHKCIFCAA|2018-06-09|1421|6180|475|2018|6|6|9|2|2018|475|6180|Saturday|2018Q2|N|Y|N|2458271|2458421|2457914|2458189|N|N|N|N|N| +2458280|AAAAAAAAIKCIFCAA|2018-06-10|1421|6180|475|2018|0|6|10|2|2018|475|6180|Sunday|2018Q2|N|N|N|2458271|2458421|2457915|2458190|N|N|N|N|N| +2458281|AAAAAAAAJKCIFCAA|2018-06-11|1421|6180|475|2018|1|6|11|2|2018|475|6180|Monday|2018Q2|N|N|N|2458271|2458421|2457916|2458191|N|N|N|N|N| +2458282|AAAAAAAAKKCIFCAA|2018-06-12|1421|6181|475|2018|2|6|12|2|2018|475|6181|Tuesday|2018Q2|N|N|N|2458271|2458421|2457917|2458192|N|N|N|N|N| +2458283|AAAAAAAALKCIFCAA|2018-06-13|1421|6181|475|2018|3|6|13|2|2018|475|6181|Wednesday|2018Q2|N|N|N|2458271|2458421|2457918|2458193|N|N|N|N|N| +2458284|AAAAAAAAMKCIFCAA|2018-06-14|1421|6181|475|2018|4|6|14|2|2018|475|6181|Thursday|2018Q2|N|N|N|2458271|2458421|2457919|2458194|N|N|N|N|N| +2458285|AAAAAAAANKCIFCAA|2018-06-15|1421|6181|475|2018|5|6|15|2|2018|475|6181|Friday|2018Q2|N|Y|N|2458271|2458421|2457920|2458195|N|N|N|N|N| +2458286|AAAAAAAAOKCIFCAA|2018-06-16|1421|6181|475|2018|6|6|16|2|2018|475|6181|Saturday|2018Q2|N|Y|N|2458271|2458421|2457921|2458196|N|N|N|N|N| +2458287|AAAAAAAAPKCIFCAA|2018-06-17|1421|6181|475|2018|0|6|17|2|2018|475|6181|Sunday|2018Q2|N|N|N|2458271|2458421|2457922|2458197|N|N|N|N|N| +2458288|AAAAAAAAALCIFCAA|2018-06-18|1421|6181|475|2018|1|6|18|2|2018|475|6181|Monday|2018Q2|N|N|N|2458271|2458421|2457923|2458198|N|N|N|N|N| +2458289|AAAAAAAABLCIFCAA|2018-06-19|1421|6182|475|2018|2|6|19|2|2018|475|6182|Tuesday|2018Q2|N|N|N|2458271|2458421|2457924|2458199|N|N|N|N|N| +2458290|AAAAAAAACLCIFCAA|2018-06-20|1421|6182|475|2018|3|6|20|2|2018|475|6182|Wednesday|2018Q2|N|N|N|2458271|2458421|2457925|2458200|N|N|N|N|N| +2458291|AAAAAAAADLCIFCAA|2018-06-21|1421|6182|475|2018|4|6|21|2|2018|475|6182|Thursday|2018Q2|N|N|N|2458271|2458421|2457926|2458201|N|N|N|N|N| +2458292|AAAAAAAAELCIFCAA|2018-06-22|1421|6182|475|2018|5|6|22|2|2018|475|6182|Friday|2018Q2|N|Y|N|2458271|2458421|2457927|2458202|N|N|N|N|N| +2458293|AAAAAAAAFLCIFCAA|2018-06-23|1421|6182|475|2018|6|6|23|2|2018|475|6182|Saturday|2018Q2|N|Y|N|2458271|2458421|2457928|2458203|N|N|N|N|N| +2458294|AAAAAAAAGLCIFCAA|2018-06-24|1421|6182|475|2018|0|6|24|2|2018|475|6182|Sunday|2018Q2|N|N|N|2458271|2458421|2457929|2458204|N|N|N|N|N| +2458295|AAAAAAAAHLCIFCAA|2018-06-25|1421|6182|475|2018|1|6|25|2|2018|475|6182|Monday|2018Q2|N|N|N|2458271|2458421|2457930|2458205|N|N|N|N|N| +2458296|AAAAAAAAILCIFCAA|2018-06-26|1421|6183|475|2018|2|6|26|2|2018|475|6183|Tuesday|2018Q2|N|N|N|2458271|2458421|2457931|2458206|N|N|N|N|N| +2458297|AAAAAAAAJLCIFCAA|2018-06-27|1421|6183|475|2018|3|6|27|2|2018|475|6183|Wednesday|2018Q2|N|N|N|2458271|2458421|2457932|2458207|N|N|N|N|N| +2458298|AAAAAAAAKLCIFCAA|2018-06-28|1421|6183|475|2018|4|6|28|2|2018|475|6183|Thursday|2018Q2|N|N|N|2458271|2458421|2457933|2458208|N|N|N|N|N| +2458299|AAAAAAAALLCIFCAA|2018-06-29|1421|6183|475|2018|5|6|29|2|2018|475|6183|Friday|2018Q2|N|Y|N|2458271|2458421|2457934|2458209|N|N|N|N|N| +2458300|AAAAAAAAMLCIFCAA|2018-06-30|1421|6183|475|2018|6|6|30|2|2018|475|6183|Saturday|2018Q2|N|Y|N|2458271|2458421|2457935|2458210|N|N|N|N|N| +2458301|AAAAAAAANLCIFCAA|2018-07-01|1422|6183|475|2018|0|7|1|2|2018|475|6183|Sunday|2018Q2|N|N|N|2458301|2458481|2457936|2458210|N|N|N|N|N| +2458302|AAAAAAAAOLCIFCAA|2018-07-02|1422|6183|475|2018|1|7|2|3|2018|475|6183|Monday|2018Q3|N|N|N|2458301|2458481|2457937|2458211|N|N|N|N|N| +2458303|AAAAAAAAPLCIFCAA|2018-07-03|1422|6184|475|2018|2|7|3|3|2018|475|6184|Tuesday|2018Q3|N|N|N|2458301|2458481|2457938|2458212|N|N|N|N|N| +2458304|AAAAAAAAAMCIFCAA|2018-07-04|1422|6184|475|2018|3|7|4|3|2018|475|6184|Wednesday|2018Q3|N|N|N|2458301|2458481|2457939|2458213|N|N|N|N|N| +2458305|AAAAAAAABMCIFCAA|2018-07-05|1422|6184|475|2018|4|7|5|3|2018|475|6184|Thursday|2018Q3|Y|N|N|2458301|2458481|2457940|2458214|N|N|N|N|N| +2458306|AAAAAAAACMCIFCAA|2018-07-06|1422|6184|475|2018|5|7|6|3|2018|475|6184|Friday|2018Q3|N|Y|Y|2458301|2458481|2457941|2458215|N|N|N|N|N| +2458307|AAAAAAAADMCIFCAA|2018-07-07|1422|6184|475|2018|6|7|7|3|2018|475|6184|Saturday|2018Q3|N|Y|N|2458301|2458481|2457942|2458216|N|N|N|N|N| +2458308|AAAAAAAAEMCIFCAA|2018-07-08|1422|6184|475|2018|0|7|8|3|2018|475|6184|Sunday|2018Q3|N|N|N|2458301|2458481|2457943|2458217|N|N|N|N|N| +2458309|AAAAAAAAFMCIFCAA|2018-07-09|1422|6184|475|2018|1|7|9|3|2018|475|6184|Monday|2018Q3|N|N|N|2458301|2458481|2457944|2458218|N|N|N|N|N| +2458310|AAAAAAAAGMCIFCAA|2018-07-10|1422|6185|475|2018|2|7|10|3|2018|475|6185|Tuesday|2018Q3|N|N|N|2458301|2458481|2457945|2458219|N|N|N|N|N| +2458311|AAAAAAAAHMCIFCAA|2018-07-11|1422|6185|475|2018|3|7|11|3|2018|475|6185|Wednesday|2018Q3|N|N|N|2458301|2458481|2457946|2458220|N|N|N|N|N| +2458312|AAAAAAAAIMCIFCAA|2018-07-12|1422|6185|475|2018|4|7|12|3|2018|475|6185|Thursday|2018Q3|N|N|N|2458301|2458481|2457947|2458221|N|N|N|N|N| +2458313|AAAAAAAAJMCIFCAA|2018-07-13|1422|6185|475|2018|5|7|13|3|2018|475|6185|Friday|2018Q3|N|Y|N|2458301|2458481|2457948|2458222|N|N|N|N|N| +2458314|AAAAAAAAKMCIFCAA|2018-07-14|1422|6185|475|2018|6|7|14|3|2018|475|6185|Saturday|2018Q3|N|Y|N|2458301|2458481|2457949|2458223|N|N|N|N|N| +2458315|AAAAAAAALMCIFCAA|2018-07-15|1422|6185|475|2018|0|7|15|3|2018|475|6185|Sunday|2018Q3|N|N|N|2458301|2458481|2457950|2458224|N|N|N|N|N| +2458316|AAAAAAAAMMCIFCAA|2018-07-16|1422|6185|475|2018|1|7|16|3|2018|475|6185|Monday|2018Q3|N|N|N|2458301|2458481|2457951|2458225|N|N|N|N|N| +2458317|AAAAAAAANMCIFCAA|2018-07-17|1422|6186|475|2018|2|7|17|3|2018|475|6186|Tuesday|2018Q3|N|N|N|2458301|2458481|2457952|2458226|N|N|N|N|N| +2458318|AAAAAAAAOMCIFCAA|2018-07-18|1422|6186|475|2018|3|7|18|3|2018|475|6186|Wednesday|2018Q3|N|N|N|2458301|2458481|2457953|2458227|N|N|N|N|N| +2458319|AAAAAAAAPMCIFCAA|2018-07-19|1422|6186|475|2018|4|7|19|3|2018|475|6186|Thursday|2018Q3|N|N|N|2458301|2458481|2457954|2458228|N|N|N|N|N| +2458320|AAAAAAAAANCIFCAA|2018-07-20|1422|6186|475|2018|5|7|20|3|2018|475|6186|Friday|2018Q3|N|Y|N|2458301|2458481|2457955|2458229|N|N|N|N|N| +2458321|AAAAAAAABNCIFCAA|2018-07-21|1422|6186|475|2018|6|7|21|3|2018|475|6186|Saturday|2018Q3|N|Y|N|2458301|2458481|2457956|2458230|N|N|N|N|N| +2458322|AAAAAAAACNCIFCAA|2018-07-22|1422|6186|475|2018|0|7|22|3|2018|475|6186|Sunday|2018Q3|N|N|N|2458301|2458481|2457957|2458231|N|N|N|N|N| +2458323|AAAAAAAADNCIFCAA|2018-07-23|1422|6186|475|2018|1|7|23|3|2018|475|6186|Monday|2018Q3|N|N|N|2458301|2458481|2457958|2458232|N|N|N|N|N| +2458324|AAAAAAAAENCIFCAA|2018-07-24|1422|6187|475|2018|2|7|24|3|2018|475|6187|Tuesday|2018Q3|N|N|N|2458301|2458481|2457959|2458233|N|N|N|N|N| +2458325|AAAAAAAAFNCIFCAA|2018-07-25|1422|6187|475|2018|3|7|25|3|2018|475|6187|Wednesday|2018Q3|N|N|N|2458301|2458481|2457960|2458234|N|N|N|N|N| +2458326|AAAAAAAAGNCIFCAA|2018-07-26|1422|6187|475|2018|4|7|26|3|2018|475|6187|Thursday|2018Q3|N|N|N|2458301|2458481|2457961|2458235|N|N|N|N|N| +2458327|AAAAAAAAHNCIFCAA|2018-07-27|1422|6187|475|2018|5|7|27|3|2018|475|6187|Friday|2018Q3|N|Y|N|2458301|2458481|2457962|2458236|N|N|N|N|N| +2458328|AAAAAAAAINCIFCAA|2018-07-28|1422|6187|475|2018|6|7|28|3|2018|475|6187|Saturday|2018Q3|N|Y|N|2458301|2458481|2457963|2458237|N|N|N|N|N| +2458329|AAAAAAAAJNCIFCAA|2018-07-29|1422|6187|475|2018|0|7|29|3|2018|475|6187|Sunday|2018Q3|N|N|N|2458301|2458481|2457964|2458238|N|N|N|N|N| +2458330|AAAAAAAAKNCIFCAA|2018-07-30|1422|6187|475|2018|1|7|30|3|2018|475|6187|Monday|2018Q3|N|N|N|2458301|2458481|2457965|2458239|N|N|N|N|N| +2458331|AAAAAAAALNCIFCAA|2018-07-31|1422|6188|475|2018|2|7|31|3|2018|475|6188|Tuesday|2018Q3|N|N|N|2458301|2458481|2457966|2458240|N|N|N|N|N| +2458332|AAAAAAAAMNCIFCAA|2018-08-01|1423|6188|475|2018|3|8|1|3|2018|475|6188|Wednesday|2018Q3|N|N|N|2458332|2458543|2457967|2458241|N|N|N|N|N| +2458333|AAAAAAAANNCIFCAA|2018-08-02|1423|6188|475|2018|4|8|2|3|2018|475|6188|Thursday|2018Q3|N|N|N|2458332|2458543|2457968|2458242|N|N|N|N|N| +2458334|AAAAAAAAONCIFCAA|2018-08-03|1423|6188|475|2018|5|8|3|3|2018|475|6188|Friday|2018Q3|N|Y|N|2458332|2458543|2457969|2458243|N|N|N|N|N| +2458335|AAAAAAAAPNCIFCAA|2018-08-04|1423|6188|475|2018|6|8|4|3|2018|475|6188|Saturday|2018Q3|N|Y|N|2458332|2458543|2457970|2458244|N|N|N|N|N| +2458336|AAAAAAAAAOCIFCAA|2018-08-05|1423|6188|475|2018|0|8|5|3|2018|475|6188|Sunday|2018Q3|N|N|N|2458332|2458543|2457971|2458245|N|N|N|N|N| +2458337|AAAAAAAABOCIFCAA|2018-08-06|1423|6188|475|2018|1|8|6|3|2018|475|6188|Monday|2018Q3|N|N|N|2458332|2458543|2457972|2458246|N|N|N|N|N| +2458338|AAAAAAAACOCIFCAA|2018-08-07|1423|6189|475|2018|2|8|7|3|2018|475|6189|Tuesday|2018Q3|N|N|N|2458332|2458543|2457973|2458247|N|N|N|N|N| +2458339|AAAAAAAADOCIFCAA|2018-08-08|1423|6189|475|2018|3|8|8|3|2018|475|6189|Wednesday|2018Q3|N|N|N|2458332|2458543|2457974|2458248|N|N|N|N|N| +2458340|AAAAAAAAEOCIFCAA|2018-08-09|1423|6189|475|2018|4|8|9|3|2018|475|6189|Thursday|2018Q3|N|N|N|2458332|2458543|2457975|2458249|N|N|N|N|N| +2458341|AAAAAAAAFOCIFCAA|2018-08-10|1423|6189|475|2018|5|8|10|3|2018|475|6189|Friday|2018Q3|N|Y|N|2458332|2458543|2457976|2458250|N|N|N|N|N| +2458342|AAAAAAAAGOCIFCAA|2018-08-11|1423|6189|475|2018|6|8|11|3|2018|475|6189|Saturday|2018Q3|N|Y|N|2458332|2458543|2457977|2458251|N|N|N|N|N| +2458343|AAAAAAAAHOCIFCAA|2018-08-12|1423|6189|475|2018|0|8|12|3|2018|475|6189|Sunday|2018Q3|N|N|N|2458332|2458543|2457978|2458252|N|N|N|N|N| +2458344|AAAAAAAAIOCIFCAA|2018-08-13|1423|6189|475|2018|1|8|13|3|2018|475|6189|Monday|2018Q3|N|N|N|2458332|2458543|2457979|2458253|N|N|N|N|N| +2458345|AAAAAAAAJOCIFCAA|2018-08-14|1423|6190|475|2018|2|8|14|3|2018|475|6190|Tuesday|2018Q3|N|N|N|2458332|2458543|2457980|2458254|N|N|N|N|N| +2458346|AAAAAAAAKOCIFCAA|2018-08-15|1423|6190|475|2018|3|8|15|3|2018|475|6190|Wednesday|2018Q3|N|N|N|2458332|2458543|2457981|2458255|N|N|N|N|N| +2458347|AAAAAAAALOCIFCAA|2018-08-16|1423|6190|475|2018|4|8|16|3|2018|475|6190|Thursday|2018Q3|N|N|N|2458332|2458543|2457982|2458256|N|N|N|N|N| +2458348|AAAAAAAAMOCIFCAA|2018-08-17|1423|6190|475|2018|5|8|17|3|2018|475|6190|Friday|2018Q3|N|Y|N|2458332|2458543|2457983|2458257|N|N|N|N|N| +2458349|AAAAAAAANOCIFCAA|2018-08-18|1423|6190|475|2018|6|8|18|3|2018|475|6190|Saturday|2018Q3|N|Y|N|2458332|2458543|2457984|2458258|N|N|N|N|N| +2458350|AAAAAAAAOOCIFCAA|2018-08-19|1423|6190|475|2018|0|8|19|3|2018|475|6190|Sunday|2018Q3|N|N|N|2458332|2458543|2457985|2458259|N|N|N|N|N| +2458351|AAAAAAAAPOCIFCAA|2018-08-20|1423|6190|475|2018|1|8|20|3|2018|475|6190|Monday|2018Q3|N|N|N|2458332|2458543|2457986|2458260|N|N|N|N|N| +2458352|AAAAAAAAAPCIFCAA|2018-08-21|1423|6191|475|2018|2|8|21|3|2018|475|6191|Tuesday|2018Q3|N|N|N|2458332|2458543|2457987|2458261|N|N|N|N|N| +2458353|AAAAAAAABPCIFCAA|2018-08-22|1423|6191|475|2018|3|8|22|3|2018|475|6191|Wednesday|2018Q3|N|N|N|2458332|2458543|2457988|2458262|N|N|N|N|N| +2458354|AAAAAAAACPCIFCAA|2018-08-23|1423|6191|475|2018|4|8|23|3|2018|475|6191|Thursday|2018Q3|N|N|N|2458332|2458543|2457989|2458263|N|N|N|N|N| +2458355|AAAAAAAADPCIFCAA|2018-08-24|1423|6191|475|2018|5|8|24|3|2018|475|6191|Friday|2018Q3|N|Y|N|2458332|2458543|2457990|2458264|N|N|N|N|N| +2458356|AAAAAAAAEPCIFCAA|2018-08-25|1423|6191|475|2018|6|8|25|3|2018|475|6191|Saturday|2018Q3|N|Y|N|2458332|2458543|2457991|2458265|N|N|N|N|N| +2458357|AAAAAAAAFPCIFCAA|2018-08-26|1423|6191|475|2018|0|8|26|3|2018|475|6191|Sunday|2018Q3|N|N|N|2458332|2458543|2457992|2458266|N|N|N|N|N| +2458358|AAAAAAAAGPCIFCAA|2018-08-27|1423|6191|475|2018|1|8|27|3|2018|475|6191|Monday|2018Q3|N|N|N|2458332|2458543|2457993|2458267|N|N|N|N|N| +2458359|AAAAAAAAHPCIFCAA|2018-08-28|1423|6192|475|2018|2|8|28|3|2018|475|6192|Tuesday|2018Q3|N|N|N|2458332|2458543|2457994|2458268|N|N|N|N|N| +2458360|AAAAAAAAIPCIFCAA|2018-08-29|1423|6192|475|2018|3|8|29|3|2018|475|6192|Wednesday|2018Q3|N|N|N|2458332|2458543|2457995|2458269|N|N|N|N|N| +2458361|AAAAAAAAJPCIFCAA|2018-08-30|1423|6192|475|2018|4|8|30|3|2018|475|6192|Thursday|2018Q3|N|N|N|2458332|2458543|2457996|2458270|N|N|N|N|N| +2458362|AAAAAAAAKPCIFCAA|2018-08-31|1423|6192|475|2018|5|8|31|3|2018|475|6192|Friday|2018Q3|N|Y|N|2458332|2458543|2457997|2458271|N|N|N|N|N| +2458363|AAAAAAAALPCIFCAA|2018-09-01|1424|6192|476|2018|6|9|1|3|2018|476|6192|Saturday|2018Q3|N|Y|N|2458363|2458605|2457998|2458272|N|N|N|N|N| +2458364|AAAAAAAAMPCIFCAA|2018-09-02|1424|6192|476|2018|0|9|2|3|2018|476|6192|Sunday|2018Q3|N|N|N|2458363|2458605|2457999|2458273|N|N|N|N|N| +2458365|AAAAAAAANPCIFCAA|2018-09-03|1424|6192|476|2018|1|9|3|3|2018|476|6192|Monday|2018Q3|N|N|N|2458363|2458605|2458000|2458274|N|N|N|N|N| +2458366|AAAAAAAAOPCIFCAA|2018-09-04|1424|6193|476|2018|2|9|4|3|2018|476|6193|Tuesday|2018Q3|N|N|N|2458363|2458605|2458001|2458275|N|N|N|N|N| +2458367|AAAAAAAAPPCIFCAA|2018-09-05|1424|6193|476|2018|3|9|5|3|2018|476|6193|Wednesday|2018Q3|N|N|N|2458363|2458605|2458002|2458276|N|N|N|N|N| +2458368|AAAAAAAAAADIFCAA|2018-09-06|1424|6193|476|2018|4|9|6|3|2018|476|6193|Thursday|2018Q3|N|N|N|2458363|2458605|2458003|2458277|N|N|N|N|N| +2458369|AAAAAAAABADIFCAA|2018-09-07|1424|6193|476|2018|5|9|7|3|2018|476|6193|Friday|2018Q3|N|Y|N|2458363|2458605|2458004|2458278|N|N|N|N|N| +2458370|AAAAAAAACADIFCAA|2018-09-08|1424|6193|476|2018|6|9|8|3|2018|476|6193|Saturday|2018Q3|N|Y|N|2458363|2458605|2458005|2458279|N|N|N|N|N| +2458371|AAAAAAAADADIFCAA|2018-09-09|1424|6193|476|2018|0|9|9|3|2018|476|6193|Sunday|2018Q3|N|N|N|2458363|2458605|2458006|2458280|N|N|N|N|N| +2458372|AAAAAAAAEADIFCAA|2018-09-10|1424|6193|476|2018|1|9|10|3|2018|476|6193|Monday|2018Q3|N|N|N|2458363|2458605|2458007|2458281|N|N|N|N|N| +2458373|AAAAAAAAFADIFCAA|2018-09-11|1424|6194|476|2018|2|9|11|3|2018|476|6194|Tuesday|2018Q3|N|N|N|2458363|2458605|2458008|2458282|N|N|N|N|N| +2458374|AAAAAAAAGADIFCAA|2018-09-12|1424|6194|476|2018|3|9|12|3|2018|476|6194|Wednesday|2018Q3|N|N|N|2458363|2458605|2458009|2458283|N|N|N|N|N| +2458375|AAAAAAAAHADIFCAA|2018-09-13|1424|6194|476|2018|4|9|13|3|2018|476|6194|Thursday|2018Q3|N|N|N|2458363|2458605|2458010|2458284|N|N|N|N|N| +2458376|AAAAAAAAIADIFCAA|2018-09-14|1424|6194|476|2018|5|9|14|3|2018|476|6194|Friday|2018Q3|N|Y|N|2458363|2458605|2458011|2458285|N|N|N|N|N| +2458377|AAAAAAAAJADIFCAA|2018-09-15|1424|6194|476|2018|6|9|15|3|2018|476|6194|Saturday|2018Q3|N|Y|N|2458363|2458605|2458012|2458286|N|N|N|N|N| +2458378|AAAAAAAAKADIFCAA|2018-09-16|1424|6194|476|2018|0|9|16|3|2018|476|6194|Sunday|2018Q3|N|N|N|2458363|2458605|2458013|2458287|N|N|N|N|N| +2458379|AAAAAAAALADIFCAA|2018-09-17|1424|6194|476|2018|1|9|17|3|2018|476|6194|Monday|2018Q3|N|N|N|2458363|2458605|2458014|2458288|N|N|N|N|N| +2458380|AAAAAAAAMADIFCAA|2018-09-18|1424|6195|476|2018|2|9|18|3|2018|476|6195|Tuesday|2018Q3|N|N|N|2458363|2458605|2458015|2458289|N|N|N|N|N| +2458381|AAAAAAAANADIFCAA|2018-09-19|1424|6195|476|2018|3|9|19|3|2018|476|6195|Wednesday|2018Q3|N|N|N|2458363|2458605|2458016|2458290|N|N|N|N|N| +2458382|AAAAAAAAOADIFCAA|2018-09-20|1424|6195|476|2018|4|9|20|3|2018|476|6195|Thursday|2018Q3|N|N|N|2458363|2458605|2458017|2458291|N|N|N|N|N| +2458383|AAAAAAAAPADIFCAA|2018-09-21|1424|6195|476|2018|5|9|21|3|2018|476|6195|Friday|2018Q3|N|Y|N|2458363|2458605|2458018|2458292|N|N|N|N|N| +2458384|AAAAAAAAABDIFCAA|2018-09-22|1424|6195|476|2018|6|9|22|3|2018|476|6195|Saturday|2018Q3|N|Y|N|2458363|2458605|2458019|2458293|N|N|N|N|N| +2458385|AAAAAAAABBDIFCAA|2018-09-23|1424|6195|476|2018|0|9|23|3|2018|476|6195|Sunday|2018Q3|N|N|N|2458363|2458605|2458020|2458294|N|N|N|N|N| +2458386|AAAAAAAACBDIFCAA|2018-09-24|1424|6195|476|2018|1|9|24|3|2018|476|6195|Monday|2018Q3|N|N|N|2458363|2458605|2458021|2458295|N|N|N|N|N| +2458387|AAAAAAAADBDIFCAA|2018-09-25|1424|6196|476|2018|2|9|25|3|2018|476|6196|Tuesday|2018Q3|N|N|N|2458363|2458605|2458022|2458296|N|N|N|N|N| +2458388|AAAAAAAAEBDIFCAA|2018-09-26|1424|6196|476|2018|3|9|26|3|2018|476|6196|Wednesday|2018Q3|N|N|N|2458363|2458605|2458023|2458297|N|N|N|N|N| +2458389|AAAAAAAAFBDIFCAA|2018-09-27|1424|6196|476|2018|4|9|27|3|2018|476|6196|Thursday|2018Q3|N|N|N|2458363|2458605|2458024|2458298|N|N|N|N|N| +2458390|AAAAAAAAGBDIFCAA|2018-09-28|1424|6196|476|2018|5|9|28|3|2018|476|6196|Friday|2018Q3|N|Y|N|2458363|2458605|2458025|2458299|N|N|N|N|N| +2458391|AAAAAAAAHBDIFCAA|2018-09-29|1424|6196|476|2018|6|9|29|3|2018|476|6196|Saturday|2018Q3|N|Y|N|2458363|2458605|2458026|2458300|N|N|N|N|N| +2458392|AAAAAAAAIBDIFCAA|2018-09-30|1424|6196|476|2018|0|9|30|3|2018|476|6196|Sunday|2018Q3|N|N|N|2458363|2458605|2458027|2458301|N|N|N|N|N| +2458393|AAAAAAAAJBDIFCAA|2018-10-01|1425|6196|476|2018|1|10|1|3|2018|476|6196|Monday|2018Q3|N|N|N|2458393|2458665|2458028|2458301|N|N|N|N|N| +2458394|AAAAAAAAKBDIFCAA|2018-10-02|1425|6197|476|2018|2|10|2|4|2018|476|6197|Tuesday|2018Q4|N|N|N|2458393|2458665|2458029|2458302|N|N|N|N|N| +2458395|AAAAAAAALBDIFCAA|2018-10-03|1425|6197|476|2018|3|10|3|4|2018|476|6197|Wednesday|2018Q4|N|N|N|2458393|2458665|2458030|2458303|N|N|N|N|N| +2458396|AAAAAAAAMBDIFCAA|2018-10-04|1425|6197|476|2018|4|10|4|4|2018|476|6197|Thursday|2018Q4|N|N|N|2458393|2458665|2458031|2458304|N|N|N|N|N| +2458397|AAAAAAAANBDIFCAA|2018-10-05|1425|6197|476|2018|5|10|5|4|2018|476|6197|Friday|2018Q4|N|Y|N|2458393|2458665|2458032|2458305|N|N|N|N|N| +2458398|AAAAAAAAOBDIFCAA|2018-10-06|1425|6197|476|2018|6|10|6|4|2018|476|6197|Saturday|2018Q4|N|Y|N|2458393|2458665|2458033|2458306|N|N|N|N|N| +2458399|AAAAAAAAPBDIFCAA|2018-10-07|1425|6197|476|2018|0|10|7|4|2018|476|6197|Sunday|2018Q4|N|N|N|2458393|2458665|2458034|2458307|N|N|N|N|N| +2458400|AAAAAAAAACDIFCAA|2018-10-08|1425|6197|476|2018|1|10|8|4|2018|476|6197|Monday|2018Q4|N|N|N|2458393|2458665|2458035|2458308|N|N|N|N|N| +2458401|AAAAAAAABCDIFCAA|2018-10-09|1425|6198|476|2018|2|10|9|4|2018|476|6198|Tuesday|2018Q4|N|N|N|2458393|2458665|2458036|2458309|N|N|N|N|N| +2458402|AAAAAAAACCDIFCAA|2018-10-10|1425|6198|476|2018|3|10|10|4|2018|476|6198|Wednesday|2018Q4|N|N|N|2458393|2458665|2458037|2458310|N|N|N|N|N| +2458403|AAAAAAAADCDIFCAA|2018-10-11|1425|6198|476|2018|4|10|11|4|2018|476|6198|Thursday|2018Q4|N|N|N|2458393|2458665|2458038|2458311|N|N|N|N|N| +2458404|AAAAAAAAECDIFCAA|2018-10-12|1425|6198|476|2018|5|10|12|4|2018|476|6198|Friday|2018Q4|N|Y|N|2458393|2458665|2458039|2458312|N|N|N|N|N| +2458405|AAAAAAAAFCDIFCAA|2018-10-13|1425|6198|476|2018|6|10|13|4|2018|476|6198|Saturday|2018Q4|N|Y|N|2458393|2458665|2458040|2458313|N|N|N|N|N| +2458406|AAAAAAAAGCDIFCAA|2018-10-14|1425|6198|476|2018|0|10|14|4|2018|476|6198|Sunday|2018Q4|N|N|N|2458393|2458665|2458041|2458314|N|N|N|N|N| +2458407|AAAAAAAAHCDIFCAA|2018-10-15|1425|6198|476|2018|1|10|15|4|2018|476|6198|Monday|2018Q4|N|N|N|2458393|2458665|2458042|2458315|N|N|N|N|N| +2458408|AAAAAAAAICDIFCAA|2018-10-16|1425|6199|476|2018|2|10|16|4|2018|476|6199|Tuesday|2018Q4|N|N|N|2458393|2458665|2458043|2458316|N|N|N|N|N| +2458409|AAAAAAAAJCDIFCAA|2018-10-17|1425|6199|476|2018|3|10|17|4|2018|476|6199|Wednesday|2018Q4|N|N|N|2458393|2458665|2458044|2458317|N|N|N|N|N| +2458410|AAAAAAAAKCDIFCAA|2018-10-18|1425|6199|476|2018|4|10|18|4|2018|476|6199|Thursday|2018Q4|N|N|N|2458393|2458665|2458045|2458318|N|N|N|N|N| +2458411|AAAAAAAALCDIFCAA|2018-10-19|1425|6199|476|2018|5|10|19|4|2018|476|6199|Friday|2018Q4|N|Y|N|2458393|2458665|2458046|2458319|N|N|N|N|N| +2458412|AAAAAAAAMCDIFCAA|2018-10-20|1425|6199|476|2018|6|10|20|4|2018|476|6199|Saturday|2018Q4|N|Y|N|2458393|2458665|2458047|2458320|N|N|N|N|N| +2458413|AAAAAAAANCDIFCAA|2018-10-21|1425|6199|476|2018|0|10|21|4|2018|476|6199|Sunday|2018Q4|N|N|N|2458393|2458665|2458048|2458321|N|N|N|N|N| +2458414|AAAAAAAAOCDIFCAA|2018-10-22|1425|6199|476|2018|1|10|22|4|2018|476|6199|Monday|2018Q4|N|N|N|2458393|2458665|2458049|2458322|N|N|N|N|N| +2458415|AAAAAAAAPCDIFCAA|2018-10-23|1425|6200|476|2018|2|10|23|4|2018|476|6200|Tuesday|2018Q4|N|N|N|2458393|2458665|2458050|2458323|N|N|N|N|N| +2458416|AAAAAAAAADDIFCAA|2018-10-24|1425|6200|476|2018|3|10|24|4|2018|476|6200|Wednesday|2018Q4|N|N|N|2458393|2458665|2458051|2458324|N|N|N|N|N| +2458417|AAAAAAAABDDIFCAA|2018-10-25|1425|6200|476|2018|4|10|25|4|2018|476|6200|Thursday|2018Q4|N|N|N|2458393|2458665|2458052|2458325|N|N|N|N|N| +2458418|AAAAAAAACDDIFCAA|2018-10-26|1425|6200|476|2018|5|10|26|4|2018|476|6200|Friday|2018Q4|N|Y|N|2458393|2458665|2458053|2458326|N|N|N|N|N| +2458419|AAAAAAAADDDIFCAA|2018-10-27|1425|6200|476|2018|6|10|27|4|2018|476|6200|Saturday|2018Q4|N|Y|N|2458393|2458665|2458054|2458327|N|N|N|N|N| +2458420|AAAAAAAAEDDIFCAA|2018-10-28|1425|6200|476|2018|0|10|28|4|2018|476|6200|Sunday|2018Q4|N|N|N|2458393|2458665|2458055|2458328|N|N|N|N|N| +2458421|AAAAAAAAFDDIFCAA|2018-10-29|1425|6200|476|2018|1|10|29|4|2018|476|6200|Monday|2018Q4|N|N|N|2458393|2458665|2458056|2458329|N|N|N|N|N| +2458422|AAAAAAAAGDDIFCAA|2018-10-30|1425|6201|476|2018|2|10|30|4|2018|476|6201|Tuesday|2018Q4|N|N|N|2458393|2458665|2458057|2458330|N|N|N|N|N| +2458423|AAAAAAAAHDDIFCAA|2018-10-31|1425|6201|476|2018|3|10|31|4|2018|476|6201|Wednesday|2018Q4|N|N|N|2458393|2458665|2458058|2458331|N|N|N|N|N| +2458424|AAAAAAAAIDDIFCAA|2018-11-01|1426|6201|476|2018|4|11|1|4|2018|476|6201|Thursday|2018Q4|N|N|N|2458424|2458727|2458059|2458332|N|N|N|N|N| +2458425|AAAAAAAAJDDIFCAA|2018-11-02|1426|6201|476|2018|5|11|2|4|2018|476|6201|Friday|2018Q4|N|Y|N|2458424|2458727|2458060|2458333|N|N|N|N|N| +2458426|AAAAAAAAKDDIFCAA|2018-11-03|1426|6201|476|2018|6|11|3|4|2018|476|6201|Saturday|2018Q4|N|Y|N|2458424|2458727|2458061|2458334|N|N|N|N|N| +2458427|AAAAAAAALDDIFCAA|2018-11-04|1426|6201|476|2018|0|11|4|4|2018|476|6201|Sunday|2018Q4|N|N|N|2458424|2458727|2458062|2458335|N|N|N|N|N| +2458428|AAAAAAAAMDDIFCAA|2018-11-05|1426|6201|476|2018|1|11|5|4|2018|476|6201|Monday|2018Q4|N|N|N|2458424|2458727|2458063|2458336|N|N|N|N|N| +2458429|AAAAAAAANDDIFCAA|2018-11-06|1426|6202|476|2018|2|11|6|4|2018|476|6202|Tuesday|2018Q4|N|N|N|2458424|2458727|2458064|2458337|N|N|N|N|N| +2458430|AAAAAAAAODDIFCAA|2018-11-07|1426|6202|476|2018|3|11|7|4|2018|476|6202|Wednesday|2018Q4|N|N|N|2458424|2458727|2458065|2458338|N|N|N|N|N| +2458431|AAAAAAAAPDDIFCAA|2018-11-08|1426|6202|476|2018|4|11|8|4|2018|476|6202|Thursday|2018Q4|N|N|N|2458424|2458727|2458066|2458339|N|N|N|N|N| +2458432|AAAAAAAAAEDIFCAA|2018-11-09|1426|6202|476|2018|5|11|9|4|2018|476|6202|Friday|2018Q4|N|Y|N|2458424|2458727|2458067|2458340|N|N|N|N|N| +2458433|AAAAAAAABEDIFCAA|2018-11-10|1426|6202|476|2018|6|11|10|4|2018|476|6202|Saturday|2018Q4|N|Y|N|2458424|2458727|2458068|2458341|N|N|N|N|N| +2458434|AAAAAAAACEDIFCAA|2018-11-11|1426|6202|476|2018|0|11|11|4|2018|476|6202|Sunday|2018Q4|N|N|N|2458424|2458727|2458069|2458342|N|N|N|N|N| +2458435|AAAAAAAADEDIFCAA|2018-11-12|1426|6202|476|2018|1|11|12|4|2018|476|6202|Monday|2018Q4|N|N|N|2458424|2458727|2458070|2458343|N|N|N|N|N| +2458436|AAAAAAAAEEDIFCAA|2018-11-13|1426|6203|476|2018|2|11|13|4|2018|476|6203|Tuesday|2018Q4|N|N|N|2458424|2458727|2458071|2458344|N|N|N|N|N| +2458437|AAAAAAAAFEDIFCAA|2018-11-14|1426|6203|476|2018|3|11|14|4|2018|476|6203|Wednesday|2018Q4|N|N|N|2458424|2458727|2458072|2458345|N|N|N|N|N| +2458438|AAAAAAAAGEDIFCAA|2018-11-15|1426|6203|476|2018|4|11|15|4|2018|476|6203|Thursday|2018Q4|N|N|N|2458424|2458727|2458073|2458346|N|N|N|N|N| +2458439|AAAAAAAAHEDIFCAA|2018-11-16|1426|6203|476|2018|5|11|16|4|2018|476|6203|Friday|2018Q4|N|Y|N|2458424|2458727|2458074|2458347|N|N|N|N|N| +2458440|AAAAAAAAIEDIFCAA|2018-11-17|1426|6203|476|2018|6|11|17|4|2018|476|6203|Saturday|2018Q4|N|Y|N|2458424|2458727|2458075|2458348|N|N|N|N|N| +2458441|AAAAAAAAJEDIFCAA|2018-11-18|1426|6203|476|2018|0|11|18|4|2018|476|6203|Sunday|2018Q4|N|N|N|2458424|2458727|2458076|2458349|N|N|N|N|N| +2458442|AAAAAAAAKEDIFCAA|2018-11-19|1426|6203|476|2018|1|11|19|4|2018|476|6203|Monday|2018Q4|N|N|N|2458424|2458727|2458077|2458350|N|N|N|N|N| +2458443|AAAAAAAALEDIFCAA|2018-11-20|1426|6204|476|2018|2|11|20|4|2018|476|6204|Tuesday|2018Q4|N|N|N|2458424|2458727|2458078|2458351|N|N|N|N|N| +2458444|AAAAAAAAMEDIFCAA|2018-11-21|1426|6204|476|2018|3|11|21|4|2018|476|6204|Wednesday|2018Q4|N|N|N|2458424|2458727|2458079|2458352|N|N|N|N|N| +2458445|AAAAAAAANEDIFCAA|2018-11-22|1426|6204|476|2018|4|11|22|4|2018|476|6204|Thursday|2018Q4|N|N|N|2458424|2458727|2458080|2458353|N|N|N|N|N| +2458446|AAAAAAAAOEDIFCAA|2018-11-23|1426|6204|476|2018|5|11|23|4|2018|476|6204|Friday|2018Q4|N|Y|N|2458424|2458727|2458081|2458354|N|N|N|N|N| +2458447|AAAAAAAAPEDIFCAA|2018-11-24|1426|6204|476|2018|6|11|24|4|2018|476|6204|Saturday|2018Q4|N|Y|N|2458424|2458727|2458082|2458355|N|N|N|N|N| +2458448|AAAAAAAAAFDIFCAA|2018-11-25|1426|6204|476|2018|0|11|25|4|2018|476|6204|Sunday|2018Q4|N|N|N|2458424|2458727|2458083|2458356|N|N|N|N|N| +2458449|AAAAAAAABFDIFCAA|2018-11-26|1426|6204|476|2018|1|11|26|4|2018|476|6204|Monday|2018Q4|N|N|N|2458424|2458727|2458084|2458357|N|N|N|N|N| +2458450|AAAAAAAACFDIFCAA|2018-11-27|1426|6205|476|2018|2|11|27|4|2018|476|6205|Tuesday|2018Q4|N|N|N|2458424|2458727|2458085|2458358|N|N|N|N|N| +2458451|AAAAAAAADFDIFCAA|2018-11-28|1426|6205|476|2018|3|11|28|4|2018|476|6205|Wednesday|2018Q4|N|N|N|2458424|2458727|2458086|2458359|N|N|N|N|N| +2458452|AAAAAAAAEFDIFCAA|2018-11-29|1426|6205|476|2018|4|11|29|4|2018|476|6205|Thursday|2018Q4|N|N|N|2458424|2458727|2458087|2458360|N|N|N|N|N| +2458453|AAAAAAAAFFDIFCAA|2018-11-30|1426|6205|476|2018|5|11|30|4|2018|476|6205|Friday|2018Q4|N|Y|N|2458424|2458727|2458088|2458361|N|N|N|N|N| +2458454|AAAAAAAAGFDIFCAA|2018-12-01|1427|6205|477|2018|6|12|1|4|2018|477|6205|Saturday|2018Q4|N|Y|N|2458454|2458787|2458089|2458362|N|N|N|N|N| +2458455|AAAAAAAAHFDIFCAA|2018-12-02|1427|6205|477|2018|0|12|2|4|2018|477|6205|Sunday|2018Q4|N|N|N|2458454|2458787|2458090|2458363|N|N|N|N|N| +2458456|AAAAAAAAIFDIFCAA|2018-12-03|1427|6205|477|2018|1|12|3|4|2018|477|6205|Monday|2018Q4|N|N|N|2458454|2458787|2458091|2458364|N|N|N|N|N| +2458457|AAAAAAAAJFDIFCAA|2018-12-04|1427|6206|477|2018|2|12|4|4|2018|477|6206|Tuesday|2018Q4|N|N|N|2458454|2458787|2458092|2458365|N|N|N|N|N| +2458458|AAAAAAAAKFDIFCAA|2018-12-05|1427|6206|477|2018|3|12|5|4|2018|477|6206|Wednesday|2018Q4|N|N|N|2458454|2458787|2458093|2458366|N|N|N|N|N| +2458459|AAAAAAAALFDIFCAA|2018-12-06|1427|6206|477|2018|4|12|6|4|2018|477|6206|Thursday|2018Q4|N|N|N|2458454|2458787|2458094|2458367|N|N|N|N|N| +2458460|AAAAAAAAMFDIFCAA|2018-12-07|1427|6206|477|2018|5|12|7|4|2018|477|6206|Friday|2018Q4|N|Y|N|2458454|2458787|2458095|2458368|N|N|N|N|N| +2458461|AAAAAAAANFDIFCAA|2018-12-08|1427|6206|477|2018|6|12|8|4|2018|477|6206|Saturday|2018Q4|N|Y|N|2458454|2458787|2458096|2458369|N|N|N|N|N| +2458462|AAAAAAAAOFDIFCAA|2018-12-09|1427|6206|477|2018|0|12|9|4|2018|477|6206|Sunday|2018Q4|N|N|N|2458454|2458787|2458097|2458370|N|N|N|N|N| +2458463|AAAAAAAAPFDIFCAA|2018-12-10|1427|6206|477|2018|1|12|10|4|2018|477|6206|Monday|2018Q4|N|N|N|2458454|2458787|2458098|2458371|N|N|N|N|N| +2458464|AAAAAAAAAGDIFCAA|2018-12-11|1427|6207|477|2018|2|12|11|4|2018|477|6207|Tuesday|2018Q4|N|N|N|2458454|2458787|2458099|2458372|N|N|N|N|N| +2458465|AAAAAAAABGDIFCAA|2018-12-12|1427|6207|477|2018|3|12|12|4|2018|477|6207|Wednesday|2018Q4|N|N|N|2458454|2458787|2458100|2458373|N|N|N|N|N| +2458466|AAAAAAAACGDIFCAA|2018-12-13|1427|6207|477|2018|4|12|13|4|2018|477|6207|Thursday|2018Q4|N|N|N|2458454|2458787|2458101|2458374|N|N|N|N|N| +2458467|AAAAAAAADGDIFCAA|2018-12-14|1427|6207|477|2018|5|12|14|4|2018|477|6207|Friday|2018Q4|N|Y|N|2458454|2458787|2458102|2458375|N|N|N|N|N| +2458468|AAAAAAAAEGDIFCAA|2018-12-15|1427|6207|477|2018|6|12|15|4|2018|477|6207|Saturday|2018Q4|N|Y|N|2458454|2458787|2458103|2458376|N|N|N|N|N| +2458469|AAAAAAAAFGDIFCAA|2018-12-16|1427|6207|477|2018|0|12|16|4|2018|477|6207|Sunday|2018Q4|N|N|N|2458454|2458787|2458104|2458377|N|N|N|N|N| +2458470|AAAAAAAAGGDIFCAA|2018-12-17|1427|6207|477|2018|1|12|17|4|2018|477|6207|Monday|2018Q4|N|N|N|2458454|2458787|2458105|2458378|N|N|N|N|N| +2458471|AAAAAAAAHGDIFCAA|2018-12-18|1427|6208|477|2018|2|12|18|4|2018|477|6208|Tuesday|2018Q4|N|N|N|2458454|2458787|2458106|2458379|N|N|N|N|N| +2458472|AAAAAAAAIGDIFCAA|2018-12-19|1427|6208|477|2018|3|12|19|4|2018|477|6208|Wednesday|2018Q4|N|N|N|2458454|2458787|2458107|2458380|N|N|N|N|N| +2458473|AAAAAAAAJGDIFCAA|2018-12-20|1427|6208|477|2018|4|12|20|4|2018|477|6208|Thursday|2018Q4|N|N|N|2458454|2458787|2458108|2458381|N|N|N|N|N| +2458474|AAAAAAAAKGDIFCAA|2018-12-21|1427|6208|477|2018|5|12|21|4|2018|477|6208|Friday|2018Q4|N|Y|N|2458454|2458787|2458109|2458382|N|N|N|N|N| +2458475|AAAAAAAALGDIFCAA|2018-12-22|1427|6208|477|2018|6|12|22|4|2018|477|6208|Saturday|2018Q4|N|Y|N|2458454|2458787|2458110|2458383|N|N|N|N|N| +2458476|AAAAAAAAMGDIFCAA|2018-12-23|1427|6208|477|2018|0|12|23|4|2018|477|6208|Sunday|2018Q4|N|N|N|2458454|2458787|2458111|2458384|N|N|N|N|N| +2458477|AAAAAAAANGDIFCAA|2018-12-24|1427|6208|477|2018|1|12|24|4|2018|477|6208|Monday|2018Q4|N|N|N|2458454|2458787|2458112|2458385|N|N|N|N|N| +2458478|AAAAAAAAOGDIFCAA|2018-12-25|1427|6209|477|2018|2|12|25|4|2018|477|6209|Tuesday|2018Q4|N|N|N|2458454|2458787|2458113|2458386|N|N|N|N|N| +2458479|AAAAAAAAPGDIFCAA|2018-12-26|1427|6209|477|2018|3|12|26|4|2018|477|6209|Wednesday|2018Q4|Y|N|N|2458454|2458787|2458114|2458387|N|N|N|N|N| +2458480|AAAAAAAAAHDIFCAA|2018-12-27|1427|6209|477|2018|4|12|27|4|2018|477|6209|Thursday|2018Q4|N|N|Y|2458454|2458787|2458115|2458388|N|N|N|N|N| +2458481|AAAAAAAABHDIFCAA|2018-12-28|1427|6209|477|2018|5|12|28|4|2018|477|6209|Friday|2018Q4|N|Y|N|2458454|2458787|2458116|2458389|N|N|N|N|N| +2458482|AAAAAAAACHDIFCAA|2018-12-29|1427|6209|477|2018|6|12|29|4|2018|477|6209|Saturday|2018Q4|N|Y|N|2458454|2458787|2458117|2458390|N|N|N|N|N| +2458483|AAAAAAAADHDIFCAA|2018-12-30|1427|6209|477|2018|0|12|30|4|2018|477|6209|Sunday|2018Q4|N|N|N|2458454|2458787|2458118|2458391|N|N|N|N|N| +2458484|AAAAAAAAEHDIFCAA|2018-12-31|1427|6209|477|2018|1|12|31|4|2018|477|6209|Monday|2018Q4|N|N|N|2458454|2458787|2458119|2458392|N|N|N|N|N| +2458485|AAAAAAAAFHDIFCAA|2019-01-01|1428|6210|477|2019|2|1|1|1|2019|477|6210|Tuesday|2019Q1|Y|N|N|2458485|2458484|2458120|2458393|N|N|N|N|N| +2458486|AAAAAAAAGHDIFCAA|2019-01-02|1428|6210|477|2019|3|1|2|1|2019|477|6210|Wednesday|2019Q1|N|N|Y|2458485|2458484|2458121|2458394|N|N|N|N|N| +2458487|AAAAAAAAHHDIFCAA|2019-01-03|1428|6210|477|2019|4|1|3|1|2019|477|6210|Thursday|2019Q1|N|N|N|2458485|2458484|2458122|2458395|N|N|N|N|N| +2458488|AAAAAAAAIHDIFCAA|2019-01-04|1428|6210|477|2019|5|1|4|1|2019|477|6210|Friday|2019Q1|N|Y|N|2458485|2458484|2458123|2458396|N|N|N|N|N| +2458489|AAAAAAAAJHDIFCAA|2019-01-05|1428|6210|477|2019|6|1|5|1|2019|477|6210|Saturday|2019Q1|N|Y|N|2458485|2458484|2458124|2458397|N|N|N|N|N| +2458490|AAAAAAAAKHDIFCAA|2019-01-06|1428|6210|477|2019|0|1|6|1|2019|477|6210|Sunday|2019Q1|N|N|N|2458485|2458484|2458125|2458398|N|N|N|N|N| +2458491|AAAAAAAALHDIFCAA|2019-01-07|1428|6210|477|2019|1|1|7|1|2019|477|6210|Monday|2019Q1|N|N|N|2458485|2458484|2458126|2458399|N|N|N|N|N| +2458492|AAAAAAAAMHDIFCAA|2019-01-08|1428|6211|477|2019|2|1|8|1|2019|477|6211|Tuesday|2019Q1|N|N|N|2458485|2458484|2458127|2458400|N|N|N|N|N| +2458493|AAAAAAAANHDIFCAA|2019-01-09|1428|6211|477|2019|3|1|9|1|2019|477|6211|Wednesday|2019Q1|N|N|N|2458485|2458484|2458128|2458401|N|N|N|N|N| +2458494|AAAAAAAAOHDIFCAA|2019-01-10|1428|6211|477|2019|4|1|10|1|2019|477|6211|Thursday|2019Q1|N|N|N|2458485|2458484|2458129|2458402|N|N|N|N|N| +2458495|AAAAAAAAPHDIFCAA|2019-01-11|1428|6211|477|2019|5|1|11|1|2019|477|6211|Friday|2019Q1|N|Y|N|2458485|2458484|2458130|2458403|N|N|N|N|N| +2458496|AAAAAAAAAIDIFCAA|2019-01-12|1428|6211|477|2019|6|1|12|1|2019|477|6211|Saturday|2019Q1|N|Y|N|2458485|2458484|2458131|2458404|N|N|N|N|N| +2458497|AAAAAAAABIDIFCAA|2019-01-13|1428|6211|477|2019|0|1|13|1|2019|477|6211|Sunday|2019Q1|N|N|N|2458485|2458484|2458132|2458405|N|N|N|N|N| +2458498|AAAAAAAACIDIFCAA|2019-01-14|1428|6211|477|2019|1|1|14|1|2019|477|6211|Monday|2019Q1|N|N|N|2458485|2458484|2458133|2458406|N|N|N|N|N| +2458499|AAAAAAAADIDIFCAA|2019-01-15|1428|6212|477|2019|2|1|15|1|2019|477|6212|Tuesday|2019Q1|N|N|N|2458485|2458484|2458134|2458407|N|N|N|N|N| +2458500|AAAAAAAAEIDIFCAA|2019-01-16|1428|6212|477|2019|3|1|16|1|2019|477|6212|Wednesday|2019Q1|N|N|N|2458485|2458484|2458135|2458408|N|N|N|N|N| +2458501|AAAAAAAAFIDIFCAA|2019-01-17|1428|6212|477|2019|4|1|17|1|2019|477|6212|Thursday|2019Q1|N|N|N|2458485|2458484|2458136|2458409|N|N|N|N|N| +2458502|AAAAAAAAGIDIFCAA|2019-01-18|1428|6212|477|2019|5|1|18|1|2019|477|6212|Friday|2019Q1|N|Y|N|2458485|2458484|2458137|2458410|N|N|N|N|N| +2458503|AAAAAAAAHIDIFCAA|2019-01-19|1428|6212|477|2019|6|1|19|1|2019|477|6212|Saturday|2019Q1|N|Y|N|2458485|2458484|2458138|2458411|N|N|N|N|N| +2458504|AAAAAAAAIIDIFCAA|2019-01-20|1428|6212|477|2019|0|1|20|1|2019|477|6212|Sunday|2019Q1|N|N|N|2458485|2458484|2458139|2458412|N|N|N|N|N| +2458505|AAAAAAAAJIDIFCAA|2019-01-21|1428|6212|477|2019|1|1|21|1|2019|477|6212|Monday|2019Q1|N|N|N|2458485|2458484|2458140|2458413|N|N|N|N|N| +2458506|AAAAAAAAKIDIFCAA|2019-01-22|1428|6213|477|2019|2|1|22|1|2019|477|6213|Tuesday|2019Q1|N|N|N|2458485|2458484|2458141|2458414|N|N|N|N|N| +2458507|AAAAAAAALIDIFCAA|2019-01-23|1428|6213|477|2019|3|1|23|1|2019|477|6213|Wednesday|2019Q1|N|N|N|2458485|2458484|2458142|2458415|N|N|N|N|N| +2458508|AAAAAAAAMIDIFCAA|2019-01-24|1428|6213|477|2019|4|1|24|1|2019|477|6213|Thursday|2019Q1|N|N|N|2458485|2458484|2458143|2458416|N|N|N|N|N| +2458509|AAAAAAAANIDIFCAA|2019-01-25|1428|6213|477|2019|5|1|25|1|2019|477|6213|Friday|2019Q1|N|Y|N|2458485|2458484|2458144|2458417|N|N|N|N|N| +2458510|AAAAAAAAOIDIFCAA|2019-01-26|1428|6213|477|2019|6|1|26|1|2019|477|6213|Saturday|2019Q1|N|Y|N|2458485|2458484|2458145|2458418|N|N|N|N|N| +2458511|AAAAAAAAPIDIFCAA|2019-01-27|1428|6213|477|2019|0|1|27|1|2019|477|6213|Sunday|2019Q1|N|N|N|2458485|2458484|2458146|2458419|N|N|N|N|N| +2458512|AAAAAAAAAJDIFCAA|2019-01-28|1428|6213|477|2019|1|1|28|1|2019|477|6213|Monday|2019Q1|N|N|N|2458485|2458484|2458147|2458420|N|N|N|N|N| +2458513|AAAAAAAABJDIFCAA|2019-01-29|1428|6214|477|2019|2|1|29|1|2019|477|6214|Tuesday|2019Q1|N|N|N|2458485|2458484|2458148|2458421|N|N|N|N|N| +2458514|AAAAAAAACJDIFCAA|2019-01-30|1428|6214|477|2019|3|1|30|1|2019|477|6214|Wednesday|2019Q1|N|N|N|2458485|2458484|2458149|2458422|N|N|N|N|N| +2458515|AAAAAAAADJDIFCAA|2019-01-31|1428|6214|477|2019|4|1|31|1|2019|477|6214|Thursday|2019Q1|N|N|N|2458485|2458484|2458150|2458423|N|N|N|N|N| +2458516|AAAAAAAAEJDIFCAA|2019-02-01|1429|6214|477|2019|5|2|1|1|2019|477|6214|Friday|2019Q1|N|Y|N|2458516|2458546|2458151|2458424|N|N|N|N|N| +2458517|AAAAAAAAFJDIFCAA|2019-02-02|1429|6214|477|2019|6|2|2|1|2019|477|6214|Saturday|2019Q1|N|Y|N|2458516|2458546|2458152|2458425|N|N|N|N|N| +2458518|AAAAAAAAGJDIFCAA|2019-02-03|1429|6214|477|2019|0|2|3|1|2019|477|6214|Sunday|2019Q1|N|N|N|2458516|2458546|2458153|2458426|N|N|N|N|N| +2458519|AAAAAAAAHJDIFCAA|2019-02-04|1429|6214|477|2019|1|2|4|1|2019|477|6214|Monday|2019Q1|N|N|N|2458516|2458546|2458154|2458427|N|N|N|N|N| +2458520|AAAAAAAAIJDIFCAA|2019-02-05|1429|6215|477|2019|2|2|5|1|2019|477|6215|Tuesday|2019Q1|N|N|N|2458516|2458546|2458155|2458428|N|N|N|N|N| +2458521|AAAAAAAAJJDIFCAA|2019-02-06|1429|6215|477|2019|3|2|6|1|2019|477|6215|Wednesday|2019Q1|N|N|N|2458516|2458546|2458156|2458429|N|N|N|N|N| +2458522|AAAAAAAAKJDIFCAA|2019-02-07|1429|6215|477|2019|4|2|7|1|2019|477|6215|Thursday|2019Q1|N|N|N|2458516|2458546|2458157|2458430|N|N|N|N|N| +2458523|AAAAAAAALJDIFCAA|2019-02-08|1429|6215|477|2019|5|2|8|1|2019|477|6215|Friday|2019Q1|N|Y|N|2458516|2458546|2458158|2458431|N|N|N|N|N| +2458524|AAAAAAAAMJDIFCAA|2019-02-09|1429|6215|477|2019|6|2|9|1|2019|477|6215|Saturday|2019Q1|N|Y|N|2458516|2458546|2458159|2458432|N|N|N|N|N| +2458525|AAAAAAAANJDIFCAA|2019-02-10|1429|6215|477|2019|0|2|10|1|2019|477|6215|Sunday|2019Q1|N|N|N|2458516|2458546|2458160|2458433|N|N|N|N|N| +2458526|AAAAAAAAOJDIFCAA|2019-02-11|1429|6215|477|2019|1|2|11|1|2019|477|6215|Monday|2019Q1|N|N|N|2458516|2458546|2458161|2458434|N|N|N|N|N| +2458527|AAAAAAAAPJDIFCAA|2019-02-12|1429|6216|477|2019|2|2|12|1|2019|477|6216|Tuesday|2019Q1|N|N|N|2458516|2458546|2458162|2458435|N|N|N|N|N| +2458528|AAAAAAAAAKDIFCAA|2019-02-13|1429|6216|477|2019|3|2|13|1|2019|477|6216|Wednesday|2019Q1|N|N|N|2458516|2458546|2458163|2458436|N|N|N|N|N| +2458529|AAAAAAAABKDIFCAA|2019-02-14|1429|6216|477|2019|4|2|14|1|2019|477|6216|Thursday|2019Q1|N|N|N|2458516|2458546|2458164|2458437|N|N|N|N|N| +2458530|AAAAAAAACKDIFCAA|2019-02-15|1429|6216|477|2019|5|2|15|1|2019|477|6216|Friday|2019Q1|N|Y|N|2458516|2458546|2458165|2458438|N|N|N|N|N| +2458531|AAAAAAAADKDIFCAA|2019-02-16|1429|6216|477|2019|6|2|16|1|2019|477|6216|Saturday|2019Q1|N|Y|N|2458516|2458546|2458166|2458439|N|N|N|N|N| +2458532|AAAAAAAAEKDIFCAA|2019-02-17|1429|6216|477|2019|0|2|17|1|2019|477|6216|Sunday|2019Q1|N|N|N|2458516|2458546|2458167|2458440|N|N|N|N|N| +2458533|AAAAAAAAFKDIFCAA|2019-02-18|1429|6216|477|2019|1|2|18|1|2019|477|6216|Monday|2019Q1|N|N|N|2458516|2458546|2458168|2458441|N|N|N|N|N| +2458534|AAAAAAAAGKDIFCAA|2019-02-19|1429|6217|477|2019|2|2|19|1|2019|477|6217|Tuesday|2019Q1|N|N|N|2458516|2458546|2458169|2458442|N|N|N|N|N| +2458535|AAAAAAAAHKDIFCAA|2019-02-20|1429|6217|477|2019|3|2|20|1|2019|477|6217|Wednesday|2019Q1|N|N|N|2458516|2458546|2458170|2458443|N|N|N|N|N| +2458536|AAAAAAAAIKDIFCAA|2019-02-21|1429|6217|477|2019|4|2|21|1|2019|477|6217|Thursday|2019Q1|N|N|N|2458516|2458546|2458171|2458444|N|N|N|N|N| +2458537|AAAAAAAAJKDIFCAA|2019-02-22|1429|6217|477|2019|5|2|22|1|2019|477|6217|Friday|2019Q1|N|Y|N|2458516|2458546|2458172|2458445|N|N|N|N|N| +2458538|AAAAAAAAKKDIFCAA|2019-02-23|1429|6217|477|2019|6|2|23|1|2019|477|6217|Saturday|2019Q1|N|Y|N|2458516|2458546|2458173|2458446|N|N|N|N|N| +2458539|AAAAAAAALKDIFCAA|2019-02-24|1429|6217|477|2019|0|2|24|1|2019|477|6217|Sunday|2019Q1|N|N|N|2458516|2458546|2458174|2458447|N|N|N|N|N| +2458540|AAAAAAAAMKDIFCAA|2019-02-25|1429|6217|477|2019|1|2|25|1|2019|477|6217|Monday|2019Q1|N|N|N|2458516|2458546|2458175|2458448|N|N|N|N|N| +2458541|AAAAAAAANKDIFCAA|2019-02-26|1429|6218|477|2019|2|2|26|1|2019|477|6218|Tuesday|2019Q1|N|N|N|2458516|2458546|2458176|2458449|N|N|N|N|N| +2458542|AAAAAAAAOKDIFCAA|2019-02-27|1429|6218|477|2019|3|2|27|1|2019|477|6218|Wednesday|2019Q1|N|N|N|2458516|2458546|2458177|2458450|N|N|N|N|N| +2458543|AAAAAAAAPKDIFCAA|2019-02-28|1429|6218|477|2019|4|2|28|1|2019|477|6218|Thursday|2019Q1|N|N|N|2458516|2458546|2458178|2458451|N|N|N|N|N| +2458544|AAAAAAAAALDIFCAA|2019-03-01|1430|6218|478|2019|5|3|1|1|2019|478|6218|Friday|2019Q1|N|Y|N|2458544|2458602|2458179|2458452|N|N|N|N|N| +2458545|AAAAAAAABLDIFCAA|2019-03-02|1430|6218|478|2019|6|3|2|1|2019|478|6218|Saturday|2019Q1|N|Y|N|2458544|2458602|2458180|2458453|N|N|N|N|N| +2458546|AAAAAAAACLDIFCAA|2019-03-03|1430|6218|478|2019|0|3|3|1|2019|478|6218|Sunday|2019Q1|N|N|N|2458544|2458602|2458181|2458454|N|N|N|N|N| +2458547|AAAAAAAADLDIFCAA|2019-03-04|1430|6218|478|2019|1|3|4|1|2019|478|6218|Monday|2019Q1|N|N|N|2458544|2458602|2458182|2458455|N|N|N|N|N| +2458548|AAAAAAAAELDIFCAA|2019-03-05|1430|6219|478|2019|2|3|5|1|2019|478|6219|Tuesday|2019Q1|N|N|N|2458544|2458602|2458183|2458456|N|N|N|N|N| +2458549|AAAAAAAAFLDIFCAA|2019-03-06|1430|6219|478|2019|3|3|6|1|2019|478|6219|Wednesday|2019Q1|N|N|N|2458544|2458602|2458184|2458457|N|N|N|N|N| +2458550|AAAAAAAAGLDIFCAA|2019-03-07|1430|6219|478|2019|4|3|7|1|2019|478|6219|Thursday|2019Q1|N|N|N|2458544|2458602|2458185|2458458|N|N|N|N|N| +2458551|AAAAAAAAHLDIFCAA|2019-03-08|1430|6219|478|2019|5|3|8|1|2019|478|6219|Friday|2019Q1|N|Y|N|2458544|2458602|2458186|2458459|N|N|N|N|N| +2458552|AAAAAAAAILDIFCAA|2019-03-09|1430|6219|478|2019|6|3|9|1|2019|478|6219|Saturday|2019Q1|N|Y|N|2458544|2458602|2458187|2458460|N|N|N|N|N| +2458553|AAAAAAAAJLDIFCAA|2019-03-10|1430|6219|478|2019|0|3|10|1|2019|478|6219|Sunday|2019Q1|N|N|N|2458544|2458602|2458188|2458461|N|N|N|N|N| +2458554|AAAAAAAAKLDIFCAA|2019-03-11|1430|6219|478|2019|1|3|11|1|2019|478|6219|Monday|2019Q1|N|N|N|2458544|2458602|2458189|2458462|N|N|N|N|N| +2458555|AAAAAAAALLDIFCAA|2019-03-12|1430|6220|478|2019|2|3|12|1|2019|478|6220|Tuesday|2019Q1|N|N|N|2458544|2458602|2458190|2458463|N|N|N|N|N| +2458556|AAAAAAAAMLDIFCAA|2019-03-13|1430|6220|478|2019|3|3|13|1|2019|478|6220|Wednesday|2019Q1|N|N|N|2458544|2458602|2458191|2458464|N|N|N|N|N| +2458557|AAAAAAAANLDIFCAA|2019-03-14|1430|6220|478|2019|4|3|14|1|2019|478|6220|Thursday|2019Q1|N|N|N|2458544|2458602|2458192|2458465|N|N|N|N|N| +2458558|AAAAAAAAOLDIFCAA|2019-03-15|1430|6220|478|2019|5|3|15|1|2019|478|6220|Friday|2019Q1|N|Y|N|2458544|2458602|2458193|2458466|N|N|N|N|N| +2458559|AAAAAAAAPLDIFCAA|2019-03-16|1430|6220|478|2019|6|3|16|1|2019|478|6220|Saturday|2019Q1|N|Y|N|2458544|2458602|2458194|2458467|N|N|N|N|N| +2458560|AAAAAAAAAMDIFCAA|2019-03-17|1430|6220|478|2019|0|3|17|1|2019|478|6220|Sunday|2019Q1|N|N|N|2458544|2458602|2458195|2458468|N|N|N|N|N| +2458561|AAAAAAAABMDIFCAA|2019-03-18|1430|6220|478|2019|1|3|18|1|2019|478|6220|Monday|2019Q1|N|N|N|2458544|2458602|2458196|2458469|N|N|N|N|N| +2458562|AAAAAAAACMDIFCAA|2019-03-19|1430|6221|478|2019|2|3|19|1|2019|478|6221|Tuesday|2019Q1|N|N|N|2458544|2458602|2458197|2458470|N|N|N|N|N| +2458563|AAAAAAAADMDIFCAA|2019-03-20|1430|6221|478|2019|3|3|20|1|2019|478|6221|Wednesday|2019Q1|N|N|N|2458544|2458602|2458198|2458471|N|N|N|N|N| +2458564|AAAAAAAAEMDIFCAA|2019-03-21|1430|6221|478|2019|4|3|21|1|2019|478|6221|Thursday|2019Q1|N|N|N|2458544|2458602|2458199|2458472|N|N|N|N|N| +2458565|AAAAAAAAFMDIFCAA|2019-03-22|1430|6221|478|2019|5|3|22|1|2019|478|6221|Friday|2019Q1|N|Y|N|2458544|2458602|2458200|2458473|N|N|N|N|N| +2458566|AAAAAAAAGMDIFCAA|2019-03-23|1430|6221|478|2019|6|3|23|1|2019|478|6221|Saturday|2019Q1|N|Y|N|2458544|2458602|2458201|2458474|N|N|N|N|N| +2458567|AAAAAAAAHMDIFCAA|2019-03-24|1430|6221|478|2019|0|3|24|1|2019|478|6221|Sunday|2019Q1|N|N|N|2458544|2458602|2458202|2458475|N|N|N|N|N| +2458568|AAAAAAAAIMDIFCAA|2019-03-25|1430|6221|478|2019|1|3|25|1|2019|478|6221|Monday|2019Q1|N|N|N|2458544|2458602|2458203|2458476|N|N|N|N|N| +2458569|AAAAAAAAJMDIFCAA|2019-03-26|1430|6222|478|2019|2|3|26|1|2019|478|6222|Tuesday|2019Q1|N|N|N|2458544|2458602|2458204|2458477|N|N|N|N|N| +2458570|AAAAAAAAKMDIFCAA|2019-03-27|1430|6222|478|2019|3|3|27|1|2019|478|6222|Wednesday|2019Q1|N|N|N|2458544|2458602|2458205|2458478|N|N|N|N|N| +2458571|AAAAAAAALMDIFCAA|2019-03-28|1430|6222|478|2019|4|3|28|1|2019|478|6222|Thursday|2019Q1|N|N|N|2458544|2458602|2458206|2458479|N|N|N|N|N| +2458572|AAAAAAAAMMDIFCAA|2019-03-29|1430|6222|478|2019|5|3|29|1|2019|478|6222|Friday|2019Q1|N|Y|N|2458544|2458602|2458207|2458480|N|N|N|N|N| +2458573|AAAAAAAANMDIFCAA|2019-03-30|1430|6222|478|2019|6|3|30|1|2019|478|6222|Saturday|2019Q1|N|Y|N|2458544|2458602|2458208|2458481|N|N|N|N|N| +2458574|AAAAAAAAOMDIFCAA|2019-03-31|1430|6222|478|2019|0|3|31|1|2019|478|6222|Sunday|2019Q1|N|N|N|2458544|2458602|2458209|2458482|N|N|N|N|N| +2458575|AAAAAAAAPMDIFCAA|2019-04-01|1431|6222|478|2019|1|4|1|1|2019|478|6222|Monday|2019Q1|N|N|N|2458575|2458664|2458210|2458485|N|N|N|N|N| +2458576|AAAAAAAAANDIFCAA|2019-04-02|1431|6223|478|2019|2|4|2|2|2019|478|6223|Tuesday|2019Q2|N|N|N|2458575|2458664|2458211|2458486|N|N|N|N|N| +2458577|AAAAAAAABNDIFCAA|2019-04-03|1431|6223|478|2019|3|4|3|2|2019|478|6223|Wednesday|2019Q2|N|N|N|2458575|2458664|2458212|2458487|N|N|N|N|N| +2458578|AAAAAAAACNDIFCAA|2019-04-04|1431|6223|478|2019|4|4|4|2|2019|478|6223|Thursday|2019Q2|N|N|N|2458575|2458664|2458213|2458488|N|N|N|N|N| +2458579|AAAAAAAADNDIFCAA|2019-04-05|1431|6223|478|2019|5|4|5|2|2019|478|6223|Friday|2019Q2|N|Y|N|2458575|2458664|2458214|2458489|N|N|N|N|N| +2458580|AAAAAAAAENDIFCAA|2019-04-06|1431|6223|478|2019|6|4|6|2|2019|478|6223|Saturday|2019Q2|N|Y|N|2458575|2458664|2458215|2458490|N|N|N|N|N| +2458581|AAAAAAAAFNDIFCAA|2019-04-07|1431|6223|478|2019|0|4|7|2|2019|478|6223|Sunday|2019Q2|N|N|N|2458575|2458664|2458216|2458491|N|N|N|N|N| +2458582|AAAAAAAAGNDIFCAA|2019-04-08|1431|6223|478|2019|1|4|8|2|2019|478|6223|Monday|2019Q2|N|N|N|2458575|2458664|2458217|2458492|N|N|N|N|N| +2458583|AAAAAAAAHNDIFCAA|2019-04-09|1431|6224|478|2019|2|4|9|2|2019|478|6224|Tuesday|2019Q2|N|N|N|2458575|2458664|2458218|2458493|N|N|N|N|N| +2458584|AAAAAAAAINDIFCAA|2019-04-10|1431|6224|478|2019|3|4|10|2|2019|478|6224|Wednesday|2019Q2|N|N|N|2458575|2458664|2458219|2458494|N|N|N|N|N| +2458585|AAAAAAAAJNDIFCAA|2019-04-11|1431|6224|478|2019|4|4|11|2|2019|478|6224|Thursday|2019Q2|N|N|N|2458575|2458664|2458220|2458495|N|N|N|N|N| +2458586|AAAAAAAAKNDIFCAA|2019-04-12|1431|6224|478|2019|5|4|12|2|2019|478|6224|Friday|2019Q2|N|Y|N|2458575|2458664|2458221|2458496|N|N|N|N|N| +2458587|AAAAAAAALNDIFCAA|2019-04-13|1431|6224|478|2019|6|4|13|2|2019|478|6224|Saturday|2019Q2|N|Y|N|2458575|2458664|2458222|2458497|N|N|N|N|N| +2458588|AAAAAAAAMNDIFCAA|2019-04-14|1431|6224|478|2019|0|4|14|2|2019|478|6224|Sunday|2019Q2|N|N|N|2458575|2458664|2458223|2458498|N|N|N|N|N| +2458589|AAAAAAAANNDIFCAA|2019-04-15|1431|6224|478|2019|1|4|15|2|2019|478|6224|Monday|2019Q2|N|N|N|2458575|2458664|2458224|2458499|N|N|N|N|N| +2458590|AAAAAAAAONDIFCAA|2019-04-16|1431|6225|478|2019|2|4|16|2|2019|478|6225|Tuesday|2019Q2|N|N|N|2458575|2458664|2458225|2458500|N|N|N|N|N| +2458591|AAAAAAAAPNDIFCAA|2019-04-17|1431|6225|478|2019|3|4|17|2|2019|478|6225|Wednesday|2019Q2|N|N|N|2458575|2458664|2458226|2458501|N|N|N|N|N| +2458592|AAAAAAAAAODIFCAA|2019-04-18|1431|6225|478|2019|4|4|18|2|2019|478|6225|Thursday|2019Q2|N|N|N|2458575|2458664|2458227|2458502|N|N|N|N|N| +2458593|AAAAAAAABODIFCAA|2019-04-19|1431|6225|478|2019|5|4|19|2|2019|478|6225|Friday|2019Q2|N|Y|N|2458575|2458664|2458228|2458503|N|N|N|N|N| +2458594|AAAAAAAACODIFCAA|2019-04-20|1431|6225|478|2019|6|4|20|2|2019|478|6225|Saturday|2019Q2|N|Y|N|2458575|2458664|2458229|2458504|N|N|N|N|N| +2458595|AAAAAAAADODIFCAA|2019-04-21|1431|6225|478|2019|0|4|21|2|2019|478|6225|Sunday|2019Q2|N|N|N|2458575|2458664|2458230|2458505|N|N|N|N|N| +2458596|AAAAAAAAEODIFCAA|2019-04-22|1431|6225|478|2019|1|4|22|2|2019|478|6225|Monday|2019Q2|N|N|N|2458575|2458664|2458231|2458506|N|N|N|N|N| +2458597|AAAAAAAAFODIFCAA|2019-04-23|1431|6226|478|2019|2|4|23|2|2019|478|6226|Tuesday|2019Q2|N|N|N|2458575|2458664|2458232|2458507|N|N|N|N|N| +2458598|AAAAAAAAGODIFCAA|2019-04-24|1431|6226|478|2019|3|4|24|2|2019|478|6226|Wednesday|2019Q2|N|N|N|2458575|2458664|2458233|2458508|N|N|N|N|N| +2458599|AAAAAAAAHODIFCAA|2019-04-25|1431|6226|478|2019|4|4|25|2|2019|478|6226|Thursday|2019Q2|N|N|N|2458575|2458664|2458234|2458509|N|N|N|N|N| +2458600|AAAAAAAAIODIFCAA|2019-04-26|1431|6226|478|2019|5|4|26|2|2019|478|6226|Friday|2019Q2|N|Y|N|2458575|2458664|2458235|2458510|N|N|N|N|N| +2458601|AAAAAAAAJODIFCAA|2019-04-27|1431|6226|478|2019|6|4|27|2|2019|478|6226|Saturday|2019Q2|N|Y|N|2458575|2458664|2458236|2458511|N|N|N|N|N| +2458602|AAAAAAAAKODIFCAA|2019-04-28|1431|6226|478|2019|0|4|28|2|2019|478|6226|Sunday|2019Q2|N|N|N|2458575|2458664|2458237|2458512|N|N|N|N|N| +2458603|AAAAAAAALODIFCAA|2019-04-29|1431|6226|478|2019|1|4|29|2|2019|478|6226|Monday|2019Q2|N|N|N|2458575|2458664|2458238|2458513|N|N|N|N|N| +2458604|AAAAAAAAMODIFCAA|2019-04-30|1431|6227|478|2019|2|4|30|2|2019|478|6227|Tuesday|2019Q2|N|N|N|2458575|2458664|2458239|2458514|N|N|N|N|N| +2458605|AAAAAAAANODIFCAA|2019-05-01|1432|6227|478|2019|3|5|1|2|2019|478|6227|Wednesday|2019Q2|N|N|N|2458605|2458724|2458240|2458515|N|N|N|N|N| +2458606|AAAAAAAAOODIFCAA|2019-05-02|1432|6227|478|2019|4|5|2|2|2019|478|6227|Thursday|2019Q2|N|N|N|2458605|2458724|2458241|2458516|N|N|N|N|N| +2458607|AAAAAAAAPODIFCAA|2019-05-03|1432|6227|478|2019|5|5|3|2|2019|478|6227|Friday|2019Q2|N|Y|N|2458605|2458724|2458242|2458517|N|N|N|N|N| +2458608|AAAAAAAAAPDIFCAA|2019-05-04|1432|6227|478|2019|6|5|4|2|2019|478|6227|Saturday|2019Q2|N|Y|N|2458605|2458724|2458243|2458518|N|N|N|N|N| +2458609|AAAAAAAABPDIFCAA|2019-05-05|1432|6227|478|2019|0|5|5|2|2019|478|6227|Sunday|2019Q2|N|N|N|2458605|2458724|2458244|2458519|N|N|N|N|N| +2458610|AAAAAAAACPDIFCAA|2019-05-06|1432|6227|478|2019|1|5|6|2|2019|478|6227|Monday|2019Q2|N|N|N|2458605|2458724|2458245|2458520|N|N|N|N|N| +2458611|AAAAAAAADPDIFCAA|2019-05-07|1432|6228|478|2019|2|5|7|2|2019|478|6228|Tuesday|2019Q2|N|N|N|2458605|2458724|2458246|2458521|N|N|N|N|N| +2458612|AAAAAAAAEPDIFCAA|2019-05-08|1432|6228|478|2019|3|5|8|2|2019|478|6228|Wednesday|2019Q2|N|N|N|2458605|2458724|2458247|2458522|N|N|N|N|N| +2458613|AAAAAAAAFPDIFCAA|2019-05-09|1432|6228|478|2019|4|5|9|2|2019|478|6228|Thursday|2019Q2|N|N|N|2458605|2458724|2458248|2458523|N|N|N|N|N| +2458614|AAAAAAAAGPDIFCAA|2019-05-10|1432|6228|478|2019|5|5|10|2|2019|478|6228|Friday|2019Q2|N|Y|N|2458605|2458724|2458249|2458524|N|N|N|N|N| +2458615|AAAAAAAAHPDIFCAA|2019-05-11|1432|6228|478|2019|6|5|11|2|2019|478|6228|Saturday|2019Q2|N|Y|N|2458605|2458724|2458250|2458525|N|N|N|N|N| +2458616|AAAAAAAAIPDIFCAA|2019-05-12|1432|6228|478|2019|0|5|12|2|2019|478|6228|Sunday|2019Q2|N|N|N|2458605|2458724|2458251|2458526|N|N|N|N|N| +2458617|AAAAAAAAJPDIFCAA|2019-05-13|1432|6228|478|2019|1|5|13|2|2019|478|6228|Monday|2019Q2|N|N|N|2458605|2458724|2458252|2458527|N|N|N|N|N| +2458618|AAAAAAAAKPDIFCAA|2019-05-14|1432|6229|478|2019|2|5|14|2|2019|478|6229|Tuesday|2019Q2|N|N|N|2458605|2458724|2458253|2458528|N|N|N|N|N| +2458619|AAAAAAAALPDIFCAA|2019-05-15|1432|6229|478|2019|3|5|15|2|2019|478|6229|Wednesday|2019Q2|N|N|N|2458605|2458724|2458254|2458529|N|N|N|N|N| +2458620|AAAAAAAAMPDIFCAA|2019-05-16|1432|6229|478|2019|4|5|16|2|2019|478|6229|Thursday|2019Q2|N|N|N|2458605|2458724|2458255|2458530|N|N|N|N|N| +2458621|AAAAAAAANPDIFCAA|2019-05-17|1432|6229|478|2019|5|5|17|2|2019|478|6229|Friday|2019Q2|N|Y|N|2458605|2458724|2458256|2458531|N|N|N|N|N| +2458622|AAAAAAAAOPDIFCAA|2019-05-18|1432|6229|478|2019|6|5|18|2|2019|478|6229|Saturday|2019Q2|N|Y|N|2458605|2458724|2458257|2458532|N|N|N|N|N| +2458623|AAAAAAAAPPDIFCAA|2019-05-19|1432|6229|478|2019|0|5|19|2|2019|478|6229|Sunday|2019Q2|N|N|N|2458605|2458724|2458258|2458533|N|N|N|N|N| +2458624|AAAAAAAAAAEIFCAA|2019-05-20|1432|6229|478|2019|1|5|20|2|2019|478|6229|Monday|2019Q2|N|N|N|2458605|2458724|2458259|2458534|N|N|N|N|N| +2458625|AAAAAAAABAEIFCAA|2019-05-21|1432|6230|478|2019|2|5|21|2|2019|478|6230|Tuesday|2019Q2|N|N|N|2458605|2458724|2458260|2458535|N|N|N|N|N| +2458626|AAAAAAAACAEIFCAA|2019-05-22|1432|6230|478|2019|3|5|22|2|2019|478|6230|Wednesday|2019Q2|N|N|N|2458605|2458724|2458261|2458536|N|N|N|N|N| +2458627|AAAAAAAADAEIFCAA|2019-05-23|1432|6230|478|2019|4|5|23|2|2019|478|6230|Thursday|2019Q2|N|N|N|2458605|2458724|2458262|2458537|N|N|N|N|N| +2458628|AAAAAAAAEAEIFCAA|2019-05-24|1432|6230|478|2019|5|5|24|2|2019|478|6230|Friday|2019Q2|N|Y|N|2458605|2458724|2458263|2458538|N|N|N|N|N| +2458629|AAAAAAAAFAEIFCAA|2019-05-25|1432|6230|478|2019|6|5|25|2|2019|478|6230|Saturday|2019Q2|N|Y|N|2458605|2458724|2458264|2458539|N|N|N|N|N| +2458630|AAAAAAAAGAEIFCAA|2019-05-26|1432|6230|478|2019|0|5|26|2|2019|478|6230|Sunday|2019Q2|N|N|N|2458605|2458724|2458265|2458540|N|N|N|N|N| +2458631|AAAAAAAAHAEIFCAA|2019-05-27|1432|6230|478|2019|1|5|27|2|2019|478|6230|Monday|2019Q2|N|N|N|2458605|2458724|2458266|2458541|N|N|N|N|N| +2458632|AAAAAAAAIAEIFCAA|2019-05-28|1432|6231|478|2019|2|5|28|2|2019|478|6231|Tuesday|2019Q2|N|N|N|2458605|2458724|2458267|2458542|N|N|N|N|N| +2458633|AAAAAAAAJAEIFCAA|2019-05-29|1432|6231|478|2019|3|5|29|2|2019|478|6231|Wednesday|2019Q2|N|N|N|2458605|2458724|2458268|2458543|N|N|N|N|N| +2458634|AAAAAAAAKAEIFCAA|2019-05-30|1432|6231|478|2019|4|5|30|2|2019|478|6231|Thursday|2019Q2|N|N|N|2458605|2458724|2458269|2458544|N|N|N|N|N| +2458635|AAAAAAAALAEIFCAA|2019-05-31|1432|6231|478|2019|5|5|31|2|2019|478|6231|Friday|2019Q2|N|Y|N|2458605|2458724|2458270|2458545|N|N|N|N|N| +2458636|AAAAAAAAMAEIFCAA|2019-06-01|1433|6231|479|2019|6|6|1|2|2019|479|6231|Saturday|2019Q2|N|Y|N|2458636|2458786|2458271|2458546|N|N|N|N|N| +2458637|AAAAAAAANAEIFCAA|2019-06-02|1433|6231|479|2019|0|6|2|2|2019|479|6231|Sunday|2019Q2|N|N|N|2458636|2458786|2458272|2458547|N|N|N|N|N| +2458638|AAAAAAAAOAEIFCAA|2019-06-03|1433|6231|479|2019|1|6|3|2|2019|479|6231|Monday|2019Q2|N|N|N|2458636|2458786|2458273|2458548|N|N|N|N|N| +2458639|AAAAAAAAPAEIFCAA|2019-06-04|1433|6232|479|2019|2|6|4|2|2019|479|6232|Tuesday|2019Q2|N|N|N|2458636|2458786|2458274|2458549|N|N|N|N|N| +2458640|AAAAAAAAABEIFCAA|2019-06-05|1433|6232|479|2019|3|6|5|2|2019|479|6232|Wednesday|2019Q2|N|N|N|2458636|2458786|2458275|2458550|N|N|N|N|N| +2458641|AAAAAAAABBEIFCAA|2019-06-06|1433|6232|479|2019|4|6|6|2|2019|479|6232|Thursday|2019Q2|N|N|N|2458636|2458786|2458276|2458551|N|N|N|N|N| +2458642|AAAAAAAACBEIFCAA|2019-06-07|1433|6232|479|2019|5|6|7|2|2019|479|6232|Friday|2019Q2|N|Y|N|2458636|2458786|2458277|2458552|N|N|N|N|N| +2458643|AAAAAAAADBEIFCAA|2019-06-08|1433|6232|479|2019|6|6|8|2|2019|479|6232|Saturday|2019Q2|N|Y|N|2458636|2458786|2458278|2458553|N|N|N|N|N| +2458644|AAAAAAAAEBEIFCAA|2019-06-09|1433|6232|479|2019|0|6|9|2|2019|479|6232|Sunday|2019Q2|N|N|N|2458636|2458786|2458279|2458554|N|N|N|N|N| +2458645|AAAAAAAAFBEIFCAA|2019-06-10|1433|6232|479|2019|1|6|10|2|2019|479|6232|Monday|2019Q2|N|N|N|2458636|2458786|2458280|2458555|N|N|N|N|N| +2458646|AAAAAAAAGBEIFCAA|2019-06-11|1433|6233|479|2019|2|6|11|2|2019|479|6233|Tuesday|2019Q2|N|N|N|2458636|2458786|2458281|2458556|N|N|N|N|N| +2458647|AAAAAAAAHBEIFCAA|2019-06-12|1433|6233|479|2019|3|6|12|2|2019|479|6233|Wednesday|2019Q2|N|N|N|2458636|2458786|2458282|2458557|N|N|N|N|N| +2458648|AAAAAAAAIBEIFCAA|2019-06-13|1433|6233|479|2019|4|6|13|2|2019|479|6233|Thursday|2019Q2|N|N|N|2458636|2458786|2458283|2458558|N|N|N|N|N| +2458649|AAAAAAAAJBEIFCAA|2019-06-14|1433|6233|479|2019|5|6|14|2|2019|479|6233|Friday|2019Q2|N|Y|N|2458636|2458786|2458284|2458559|N|N|N|N|N| +2458650|AAAAAAAAKBEIFCAA|2019-06-15|1433|6233|479|2019|6|6|15|2|2019|479|6233|Saturday|2019Q2|N|Y|N|2458636|2458786|2458285|2458560|N|N|N|N|N| +2458651|AAAAAAAALBEIFCAA|2019-06-16|1433|6233|479|2019|0|6|16|2|2019|479|6233|Sunday|2019Q2|N|N|N|2458636|2458786|2458286|2458561|N|N|N|N|N| +2458652|AAAAAAAAMBEIFCAA|2019-06-17|1433|6233|479|2019|1|6|17|2|2019|479|6233|Monday|2019Q2|N|N|N|2458636|2458786|2458287|2458562|N|N|N|N|N| +2458653|AAAAAAAANBEIFCAA|2019-06-18|1433|6234|479|2019|2|6|18|2|2019|479|6234|Tuesday|2019Q2|N|N|N|2458636|2458786|2458288|2458563|N|N|N|N|N| +2458654|AAAAAAAAOBEIFCAA|2019-06-19|1433|6234|479|2019|3|6|19|2|2019|479|6234|Wednesday|2019Q2|N|N|N|2458636|2458786|2458289|2458564|N|N|N|N|N| +2458655|AAAAAAAAPBEIFCAA|2019-06-20|1433|6234|479|2019|4|6|20|2|2019|479|6234|Thursday|2019Q2|N|N|N|2458636|2458786|2458290|2458565|N|N|N|N|N| +2458656|AAAAAAAAACEIFCAA|2019-06-21|1433|6234|479|2019|5|6|21|2|2019|479|6234|Friday|2019Q2|N|Y|N|2458636|2458786|2458291|2458566|N|N|N|N|N| +2458657|AAAAAAAABCEIFCAA|2019-06-22|1433|6234|479|2019|6|6|22|2|2019|479|6234|Saturday|2019Q2|N|Y|N|2458636|2458786|2458292|2458567|N|N|N|N|N| +2458658|AAAAAAAACCEIFCAA|2019-06-23|1433|6234|479|2019|0|6|23|2|2019|479|6234|Sunday|2019Q2|N|N|N|2458636|2458786|2458293|2458568|N|N|N|N|N| +2458659|AAAAAAAADCEIFCAA|2019-06-24|1433|6234|479|2019|1|6|24|2|2019|479|6234|Monday|2019Q2|N|N|N|2458636|2458786|2458294|2458569|N|N|N|N|N| +2458660|AAAAAAAAECEIFCAA|2019-06-25|1433|6235|479|2019|2|6|25|2|2019|479|6235|Tuesday|2019Q2|N|N|N|2458636|2458786|2458295|2458570|N|N|N|N|N| +2458661|AAAAAAAAFCEIFCAA|2019-06-26|1433|6235|479|2019|3|6|26|2|2019|479|6235|Wednesday|2019Q2|N|N|N|2458636|2458786|2458296|2458571|N|N|N|N|N| +2458662|AAAAAAAAGCEIFCAA|2019-06-27|1433|6235|479|2019|4|6|27|2|2019|479|6235|Thursday|2019Q2|N|N|N|2458636|2458786|2458297|2458572|N|N|N|N|N| +2458663|AAAAAAAAHCEIFCAA|2019-06-28|1433|6235|479|2019|5|6|28|2|2019|479|6235|Friday|2019Q2|N|Y|N|2458636|2458786|2458298|2458573|N|N|N|N|N| +2458664|AAAAAAAAICEIFCAA|2019-06-29|1433|6235|479|2019|6|6|29|2|2019|479|6235|Saturday|2019Q2|N|Y|N|2458636|2458786|2458299|2458574|N|N|N|N|N| +2458665|AAAAAAAAJCEIFCAA|2019-06-30|1433|6235|479|2019|0|6|30|2|2019|479|6235|Sunday|2019Q2|N|N|N|2458636|2458786|2458300|2458575|N|N|N|N|N| +2458666|AAAAAAAAKCEIFCAA|2019-07-01|1434|6235|479|2019|1|7|1|2|2019|479|6235|Monday|2019Q2|N|N|N|2458666|2458846|2458301|2458575|N|N|N|N|N| +2458667|AAAAAAAALCEIFCAA|2019-07-02|1434|6236|479|2019|2|7|2|3|2019|479|6236|Tuesday|2019Q3|N|N|N|2458666|2458846|2458302|2458576|N|N|N|N|N| +2458668|AAAAAAAAMCEIFCAA|2019-07-03|1434|6236|479|2019|3|7|3|3|2019|479|6236|Wednesday|2019Q3|N|N|N|2458666|2458846|2458303|2458577|N|N|N|N|N| +2458669|AAAAAAAANCEIFCAA|2019-07-04|1434|6236|479|2019|4|7|4|3|2019|479|6236|Thursday|2019Q3|N|N|N|2458666|2458846|2458304|2458578|N|N|N|N|N| +2458670|AAAAAAAAOCEIFCAA|2019-07-05|1434|6236|479|2019|5|7|5|3|2019|479|6236|Friday|2019Q3|Y|Y|N|2458666|2458846|2458305|2458579|N|N|N|N|N| +2458671|AAAAAAAAPCEIFCAA|2019-07-06|1434|6236|479|2019|6|7|6|3|2019|479|6236|Saturday|2019Q3|N|Y|Y|2458666|2458846|2458306|2458580|N|N|N|N|N| +2458672|AAAAAAAAADEIFCAA|2019-07-07|1434|6236|479|2019|0|7|7|3|2019|479|6236|Sunday|2019Q3|N|N|N|2458666|2458846|2458307|2458581|N|N|N|N|N| +2458673|AAAAAAAABDEIFCAA|2019-07-08|1434|6236|479|2019|1|7|8|3|2019|479|6236|Monday|2019Q3|N|N|N|2458666|2458846|2458308|2458582|N|N|N|N|N| +2458674|AAAAAAAACDEIFCAA|2019-07-09|1434|6237|479|2019|2|7|9|3|2019|479|6237|Tuesday|2019Q3|N|N|N|2458666|2458846|2458309|2458583|N|N|N|N|N| +2458675|AAAAAAAADDEIFCAA|2019-07-10|1434|6237|479|2019|3|7|10|3|2019|479|6237|Wednesday|2019Q3|N|N|N|2458666|2458846|2458310|2458584|N|N|N|N|N| +2458676|AAAAAAAAEDEIFCAA|2019-07-11|1434|6237|479|2019|4|7|11|3|2019|479|6237|Thursday|2019Q3|N|N|N|2458666|2458846|2458311|2458585|N|N|N|N|N| +2458677|AAAAAAAAFDEIFCAA|2019-07-12|1434|6237|479|2019|5|7|12|3|2019|479|6237|Friday|2019Q3|N|Y|N|2458666|2458846|2458312|2458586|N|N|N|N|N| +2458678|AAAAAAAAGDEIFCAA|2019-07-13|1434|6237|479|2019|6|7|13|3|2019|479|6237|Saturday|2019Q3|N|Y|N|2458666|2458846|2458313|2458587|N|N|N|N|N| +2458679|AAAAAAAAHDEIFCAA|2019-07-14|1434|6237|479|2019|0|7|14|3|2019|479|6237|Sunday|2019Q3|N|N|N|2458666|2458846|2458314|2458588|N|N|N|N|N| +2458680|AAAAAAAAIDEIFCAA|2019-07-15|1434|6237|479|2019|1|7|15|3|2019|479|6237|Monday|2019Q3|N|N|N|2458666|2458846|2458315|2458589|N|N|N|N|N| +2458681|AAAAAAAAJDEIFCAA|2019-07-16|1434|6238|479|2019|2|7|16|3|2019|479|6238|Tuesday|2019Q3|N|N|N|2458666|2458846|2458316|2458590|N|N|N|N|N| +2458682|AAAAAAAAKDEIFCAA|2019-07-17|1434|6238|479|2019|3|7|17|3|2019|479|6238|Wednesday|2019Q3|N|N|N|2458666|2458846|2458317|2458591|N|N|N|N|N| +2458683|AAAAAAAALDEIFCAA|2019-07-18|1434|6238|479|2019|4|7|18|3|2019|479|6238|Thursday|2019Q3|N|N|N|2458666|2458846|2458318|2458592|N|N|N|N|N| +2458684|AAAAAAAAMDEIFCAA|2019-07-19|1434|6238|479|2019|5|7|19|3|2019|479|6238|Friday|2019Q3|N|Y|N|2458666|2458846|2458319|2458593|N|N|N|N|N| +2458685|AAAAAAAANDEIFCAA|2019-07-20|1434|6238|479|2019|6|7|20|3|2019|479|6238|Saturday|2019Q3|N|Y|N|2458666|2458846|2458320|2458594|N|N|N|N|N| +2458686|AAAAAAAAODEIFCAA|2019-07-21|1434|6238|479|2019|0|7|21|3|2019|479|6238|Sunday|2019Q3|N|N|N|2458666|2458846|2458321|2458595|N|N|N|N|N| +2458687|AAAAAAAAPDEIFCAA|2019-07-22|1434|6238|479|2019|1|7|22|3|2019|479|6238|Monday|2019Q3|N|N|N|2458666|2458846|2458322|2458596|N|N|N|N|N| +2458688|AAAAAAAAAEEIFCAA|2019-07-23|1434|6239|479|2019|2|7|23|3|2019|479|6239|Tuesday|2019Q3|N|N|N|2458666|2458846|2458323|2458597|N|N|N|N|N| +2458689|AAAAAAAABEEIFCAA|2019-07-24|1434|6239|479|2019|3|7|24|3|2019|479|6239|Wednesday|2019Q3|N|N|N|2458666|2458846|2458324|2458598|N|N|N|N|N| +2458690|AAAAAAAACEEIFCAA|2019-07-25|1434|6239|479|2019|4|7|25|3|2019|479|6239|Thursday|2019Q3|N|N|N|2458666|2458846|2458325|2458599|N|N|N|N|N| +2458691|AAAAAAAADEEIFCAA|2019-07-26|1434|6239|479|2019|5|7|26|3|2019|479|6239|Friday|2019Q3|N|Y|N|2458666|2458846|2458326|2458600|N|N|N|N|N| +2458692|AAAAAAAAEEEIFCAA|2019-07-27|1434|6239|479|2019|6|7|27|3|2019|479|6239|Saturday|2019Q3|N|Y|N|2458666|2458846|2458327|2458601|N|N|N|N|N| +2458693|AAAAAAAAFEEIFCAA|2019-07-28|1434|6239|479|2019|0|7|28|3|2019|479|6239|Sunday|2019Q3|N|N|N|2458666|2458846|2458328|2458602|N|N|N|N|N| +2458694|AAAAAAAAGEEIFCAA|2019-07-29|1434|6239|479|2019|1|7|29|3|2019|479|6239|Monday|2019Q3|N|N|N|2458666|2458846|2458329|2458603|N|N|N|N|N| +2458695|AAAAAAAAHEEIFCAA|2019-07-30|1434|6240|479|2019|2|7|30|3|2019|479|6240|Tuesday|2019Q3|N|N|N|2458666|2458846|2458330|2458604|N|N|N|N|N| +2458696|AAAAAAAAIEEIFCAA|2019-07-31|1434|6240|479|2019|3|7|31|3|2019|479|6240|Wednesday|2019Q3|N|N|N|2458666|2458846|2458331|2458605|N|N|N|N|N| +2458697|AAAAAAAAJEEIFCAA|2019-08-01|1435|6240|479|2019|4|8|1|3|2019|479|6240|Thursday|2019Q3|N|N|N|2458697|2458908|2458332|2458606|N|N|N|N|N| +2458698|AAAAAAAAKEEIFCAA|2019-08-02|1435|6240|479|2019|5|8|2|3|2019|479|6240|Friday|2019Q3|N|Y|N|2458697|2458908|2458333|2458607|N|N|N|N|N| +2458699|AAAAAAAALEEIFCAA|2019-08-03|1435|6240|479|2019|6|8|3|3|2019|479|6240|Saturday|2019Q3|N|Y|N|2458697|2458908|2458334|2458608|N|N|N|N|N| +2458700|AAAAAAAAMEEIFCAA|2019-08-04|1435|6240|479|2019|0|8|4|3|2019|479|6240|Sunday|2019Q3|N|N|N|2458697|2458908|2458335|2458609|N|N|N|N|N| +2458701|AAAAAAAANEEIFCAA|2019-08-05|1435|6240|479|2019|1|8|5|3|2019|479|6240|Monday|2019Q3|N|N|N|2458697|2458908|2458336|2458610|N|N|N|N|N| +2458702|AAAAAAAAOEEIFCAA|2019-08-06|1435|6241|479|2019|2|8|6|3|2019|479|6241|Tuesday|2019Q3|N|N|N|2458697|2458908|2458337|2458611|N|N|N|N|N| +2458703|AAAAAAAAPEEIFCAA|2019-08-07|1435|6241|479|2019|3|8|7|3|2019|479|6241|Wednesday|2019Q3|N|N|N|2458697|2458908|2458338|2458612|N|N|N|N|N| +2458704|AAAAAAAAAFEIFCAA|2019-08-08|1435|6241|479|2019|4|8|8|3|2019|479|6241|Thursday|2019Q3|N|N|N|2458697|2458908|2458339|2458613|N|N|N|N|N| +2458705|AAAAAAAABFEIFCAA|2019-08-09|1435|6241|479|2019|5|8|9|3|2019|479|6241|Friday|2019Q3|N|Y|N|2458697|2458908|2458340|2458614|N|N|N|N|N| +2458706|AAAAAAAACFEIFCAA|2019-08-10|1435|6241|479|2019|6|8|10|3|2019|479|6241|Saturday|2019Q3|N|Y|N|2458697|2458908|2458341|2458615|N|N|N|N|N| +2458707|AAAAAAAADFEIFCAA|2019-08-11|1435|6241|479|2019|0|8|11|3|2019|479|6241|Sunday|2019Q3|N|N|N|2458697|2458908|2458342|2458616|N|N|N|N|N| +2458708|AAAAAAAAEFEIFCAA|2019-08-12|1435|6241|479|2019|1|8|12|3|2019|479|6241|Monday|2019Q3|N|N|N|2458697|2458908|2458343|2458617|N|N|N|N|N| +2458709|AAAAAAAAFFEIFCAA|2019-08-13|1435|6242|479|2019|2|8|13|3|2019|479|6242|Tuesday|2019Q3|N|N|N|2458697|2458908|2458344|2458618|N|N|N|N|N| +2458710|AAAAAAAAGFEIFCAA|2019-08-14|1435|6242|479|2019|3|8|14|3|2019|479|6242|Wednesday|2019Q3|N|N|N|2458697|2458908|2458345|2458619|N|N|N|N|N| +2458711|AAAAAAAAHFEIFCAA|2019-08-15|1435|6242|479|2019|4|8|15|3|2019|479|6242|Thursday|2019Q3|N|N|N|2458697|2458908|2458346|2458620|N|N|N|N|N| +2458712|AAAAAAAAIFEIFCAA|2019-08-16|1435|6242|479|2019|5|8|16|3|2019|479|6242|Friday|2019Q3|N|Y|N|2458697|2458908|2458347|2458621|N|N|N|N|N| +2458713|AAAAAAAAJFEIFCAA|2019-08-17|1435|6242|479|2019|6|8|17|3|2019|479|6242|Saturday|2019Q3|N|Y|N|2458697|2458908|2458348|2458622|N|N|N|N|N| +2458714|AAAAAAAAKFEIFCAA|2019-08-18|1435|6242|479|2019|0|8|18|3|2019|479|6242|Sunday|2019Q3|N|N|N|2458697|2458908|2458349|2458623|N|N|N|N|N| +2458715|AAAAAAAALFEIFCAA|2019-08-19|1435|6242|479|2019|1|8|19|3|2019|479|6242|Monday|2019Q3|N|N|N|2458697|2458908|2458350|2458624|N|N|N|N|N| +2458716|AAAAAAAAMFEIFCAA|2019-08-20|1435|6243|479|2019|2|8|20|3|2019|479|6243|Tuesday|2019Q3|N|N|N|2458697|2458908|2458351|2458625|N|N|N|N|N| +2458717|AAAAAAAANFEIFCAA|2019-08-21|1435|6243|479|2019|3|8|21|3|2019|479|6243|Wednesday|2019Q3|N|N|N|2458697|2458908|2458352|2458626|N|N|N|N|N| +2458718|AAAAAAAAOFEIFCAA|2019-08-22|1435|6243|479|2019|4|8|22|3|2019|479|6243|Thursday|2019Q3|N|N|N|2458697|2458908|2458353|2458627|N|N|N|N|N| +2458719|AAAAAAAAPFEIFCAA|2019-08-23|1435|6243|479|2019|5|8|23|3|2019|479|6243|Friday|2019Q3|N|Y|N|2458697|2458908|2458354|2458628|N|N|N|N|N| +2458720|AAAAAAAAAGEIFCAA|2019-08-24|1435|6243|479|2019|6|8|24|3|2019|479|6243|Saturday|2019Q3|N|Y|N|2458697|2458908|2458355|2458629|N|N|N|N|N| +2458721|AAAAAAAABGEIFCAA|2019-08-25|1435|6243|479|2019|0|8|25|3|2019|479|6243|Sunday|2019Q3|N|N|N|2458697|2458908|2458356|2458630|N|N|N|N|N| +2458722|AAAAAAAACGEIFCAA|2019-08-26|1435|6243|479|2019|1|8|26|3|2019|479|6243|Monday|2019Q3|N|N|N|2458697|2458908|2458357|2458631|N|N|N|N|N| +2458723|AAAAAAAADGEIFCAA|2019-08-27|1435|6244|479|2019|2|8|27|3|2019|479|6244|Tuesday|2019Q3|N|N|N|2458697|2458908|2458358|2458632|N|N|N|N|N| +2458724|AAAAAAAAEGEIFCAA|2019-08-28|1435|6244|479|2019|3|8|28|3|2019|479|6244|Wednesday|2019Q3|N|N|N|2458697|2458908|2458359|2458633|N|N|N|N|N| +2458725|AAAAAAAAFGEIFCAA|2019-08-29|1435|6244|479|2019|4|8|29|3|2019|479|6244|Thursday|2019Q3|N|N|N|2458697|2458908|2458360|2458634|N|N|N|N|N| +2458726|AAAAAAAAGGEIFCAA|2019-08-30|1435|6244|479|2019|5|8|30|3|2019|479|6244|Friday|2019Q3|N|Y|N|2458697|2458908|2458361|2458635|N|N|N|N|N| +2458727|AAAAAAAAHGEIFCAA|2019-08-31|1435|6244|479|2019|6|8|31|3|2019|479|6244|Saturday|2019Q3|N|Y|N|2458697|2458908|2458362|2458636|N|N|N|N|N| +2458728|AAAAAAAAIGEIFCAA|2019-09-01|1436|6244|480|2019|0|9|1|3|2019|480|6244|Sunday|2019Q3|N|N|N|2458728|2458970|2458363|2458637|N|N|N|N|N| +2458729|AAAAAAAAJGEIFCAA|2019-09-02|1436|6244|480|2019|1|9|2|3|2019|480|6244|Monday|2019Q3|N|N|N|2458728|2458970|2458364|2458638|N|N|N|N|N| +2458730|AAAAAAAAKGEIFCAA|2019-09-03|1436|6245|480|2019|2|9|3|3|2019|480|6245|Tuesday|2019Q3|N|N|N|2458728|2458970|2458365|2458639|N|N|N|N|N| +2458731|AAAAAAAALGEIFCAA|2019-09-04|1436|6245|480|2019|3|9|4|3|2019|480|6245|Wednesday|2019Q3|N|N|N|2458728|2458970|2458366|2458640|N|N|N|N|N| +2458732|AAAAAAAAMGEIFCAA|2019-09-05|1436|6245|480|2019|4|9|5|3|2019|480|6245|Thursday|2019Q3|N|N|N|2458728|2458970|2458367|2458641|N|N|N|N|N| +2458733|AAAAAAAANGEIFCAA|2019-09-06|1436|6245|480|2019|5|9|6|3|2019|480|6245|Friday|2019Q3|N|Y|N|2458728|2458970|2458368|2458642|N|N|N|N|N| +2458734|AAAAAAAAOGEIFCAA|2019-09-07|1436|6245|480|2019|6|9|7|3|2019|480|6245|Saturday|2019Q3|N|Y|N|2458728|2458970|2458369|2458643|N|N|N|N|N| +2458735|AAAAAAAAPGEIFCAA|2019-09-08|1436|6245|480|2019|0|9|8|3|2019|480|6245|Sunday|2019Q3|N|N|N|2458728|2458970|2458370|2458644|N|N|N|N|N| +2458736|AAAAAAAAAHEIFCAA|2019-09-09|1436|6245|480|2019|1|9|9|3|2019|480|6245|Monday|2019Q3|N|N|N|2458728|2458970|2458371|2458645|N|N|N|N|N| +2458737|AAAAAAAABHEIFCAA|2019-09-10|1436|6246|480|2019|2|9|10|3|2019|480|6246|Tuesday|2019Q3|N|N|N|2458728|2458970|2458372|2458646|N|N|N|N|N| +2458738|AAAAAAAACHEIFCAA|2019-09-11|1436|6246|480|2019|3|9|11|3|2019|480|6246|Wednesday|2019Q3|N|N|N|2458728|2458970|2458373|2458647|N|N|N|N|N| +2458739|AAAAAAAADHEIFCAA|2019-09-12|1436|6246|480|2019|4|9|12|3|2019|480|6246|Thursday|2019Q3|N|N|N|2458728|2458970|2458374|2458648|N|N|N|N|N| +2458740|AAAAAAAAEHEIFCAA|2019-09-13|1436|6246|480|2019|5|9|13|3|2019|480|6246|Friday|2019Q3|N|Y|N|2458728|2458970|2458375|2458649|N|N|N|N|N| +2458741|AAAAAAAAFHEIFCAA|2019-09-14|1436|6246|480|2019|6|9|14|3|2019|480|6246|Saturday|2019Q3|N|Y|N|2458728|2458970|2458376|2458650|N|N|N|N|N| +2458742|AAAAAAAAGHEIFCAA|2019-09-15|1436|6246|480|2019|0|9|15|3|2019|480|6246|Sunday|2019Q3|N|N|N|2458728|2458970|2458377|2458651|N|N|N|N|N| +2458743|AAAAAAAAHHEIFCAA|2019-09-16|1436|6246|480|2019|1|9|16|3|2019|480|6246|Monday|2019Q3|N|N|N|2458728|2458970|2458378|2458652|N|N|N|N|N| +2458744|AAAAAAAAIHEIFCAA|2019-09-17|1436|6247|480|2019|2|9|17|3|2019|480|6247|Tuesday|2019Q3|N|N|N|2458728|2458970|2458379|2458653|N|N|N|N|N| +2458745|AAAAAAAAJHEIFCAA|2019-09-18|1436|6247|480|2019|3|9|18|3|2019|480|6247|Wednesday|2019Q3|N|N|N|2458728|2458970|2458380|2458654|N|N|N|N|N| +2458746|AAAAAAAAKHEIFCAA|2019-09-19|1436|6247|480|2019|4|9|19|3|2019|480|6247|Thursday|2019Q3|N|N|N|2458728|2458970|2458381|2458655|N|N|N|N|N| +2458747|AAAAAAAALHEIFCAA|2019-09-20|1436|6247|480|2019|5|9|20|3|2019|480|6247|Friday|2019Q3|N|Y|N|2458728|2458970|2458382|2458656|N|N|N|N|N| +2458748|AAAAAAAAMHEIFCAA|2019-09-21|1436|6247|480|2019|6|9|21|3|2019|480|6247|Saturday|2019Q3|N|Y|N|2458728|2458970|2458383|2458657|N|N|N|N|N| +2458749|AAAAAAAANHEIFCAA|2019-09-22|1436|6247|480|2019|0|9|22|3|2019|480|6247|Sunday|2019Q3|N|N|N|2458728|2458970|2458384|2458658|N|N|N|N|N| +2458750|AAAAAAAAOHEIFCAA|2019-09-23|1436|6247|480|2019|1|9|23|3|2019|480|6247|Monday|2019Q3|N|N|N|2458728|2458970|2458385|2458659|N|N|N|N|N| +2458751|AAAAAAAAPHEIFCAA|2019-09-24|1436|6248|480|2019|2|9|24|3|2019|480|6248|Tuesday|2019Q3|N|N|N|2458728|2458970|2458386|2458660|N|N|N|N|N| +2458752|AAAAAAAAAIEIFCAA|2019-09-25|1436|6248|480|2019|3|9|25|3|2019|480|6248|Wednesday|2019Q3|N|N|N|2458728|2458970|2458387|2458661|N|N|N|N|N| +2458753|AAAAAAAABIEIFCAA|2019-09-26|1436|6248|480|2019|4|9|26|3|2019|480|6248|Thursday|2019Q3|N|N|N|2458728|2458970|2458388|2458662|N|N|N|N|N| +2458754|AAAAAAAACIEIFCAA|2019-09-27|1436|6248|480|2019|5|9|27|3|2019|480|6248|Friday|2019Q3|N|Y|N|2458728|2458970|2458389|2458663|N|N|N|N|N| +2458755|AAAAAAAADIEIFCAA|2019-09-28|1436|6248|480|2019|6|9|28|3|2019|480|6248|Saturday|2019Q3|N|Y|N|2458728|2458970|2458390|2458664|N|N|N|N|N| +2458756|AAAAAAAAEIEIFCAA|2019-09-29|1436|6248|480|2019|0|9|29|3|2019|480|6248|Sunday|2019Q3|N|N|N|2458728|2458970|2458391|2458665|N|N|N|N|N| +2458757|AAAAAAAAFIEIFCAA|2019-09-30|1436|6248|480|2019|1|9|30|3|2019|480|6248|Monday|2019Q3|N|N|N|2458728|2458970|2458392|2458666|N|N|N|N|N| +2458758|AAAAAAAAGIEIFCAA|2019-10-01|1437|6249|480|2019|2|10|1|3|2019|480|6249|Tuesday|2019Q3|N|N|N|2458758|2459030|2458393|2458666|N|N|N|N|N| +2458759|AAAAAAAAHIEIFCAA|2019-10-02|1437|6249|480|2019|3|10|2|4|2019|480|6249|Wednesday|2019Q4|N|N|N|2458758|2459030|2458394|2458667|N|N|N|N|N| +2458760|AAAAAAAAIIEIFCAA|2019-10-03|1437|6249|480|2019|4|10|3|4|2019|480|6249|Thursday|2019Q4|N|N|N|2458758|2459030|2458395|2458668|N|N|N|N|N| +2458761|AAAAAAAAJIEIFCAA|2019-10-04|1437|6249|480|2019|5|10|4|4|2019|480|6249|Friday|2019Q4|N|Y|N|2458758|2459030|2458396|2458669|N|N|N|N|N| +2458762|AAAAAAAAKIEIFCAA|2019-10-05|1437|6249|480|2019|6|10|5|4|2019|480|6249|Saturday|2019Q4|N|Y|N|2458758|2459030|2458397|2458670|N|N|N|N|N| +2458763|AAAAAAAALIEIFCAA|2019-10-06|1437|6249|480|2019|0|10|6|4|2019|480|6249|Sunday|2019Q4|N|N|N|2458758|2459030|2458398|2458671|N|N|N|N|N| +2458764|AAAAAAAAMIEIFCAA|2019-10-07|1437|6249|480|2019|1|10|7|4|2019|480|6249|Monday|2019Q4|N|N|N|2458758|2459030|2458399|2458672|N|N|N|N|N| +2458765|AAAAAAAANIEIFCAA|2019-10-08|1437|6250|480|2019|2|10|8|4|2019|480|6250|Tuesday|2019Q4|N|N|N|2458758|2459030|2458400|2458673|N|N|N|N|N| +2458766|AAAAAAAAOIEIFCAA|2019-10-09|1437|6250|480|2019|3|10|9|4|2019|480|6250|Wednesday|2019Q4|N|N|N|2458758|2459030|2458401|2458674|N|N|N|N|N| +2458767|AAAAAAAAPIEIFCAA|2019-10-10|1437|6250|480|2019|4|10|10|4|2019|480|6250|Thursday|2019Q4|N|N|N|2458758|2459030|2458402|2458675|N|N|N|N|N| +2458768|AAAAAAAAAJEIFCAA|2019-10-11|1437|6250|480|2019|5|10|11|4|2019|480|6250|Friday|2019Q4|N|Y|N|2458758|2459030|2458403|2458676|N|N|N|N|N| +2458769|AAAAAAAABJEIFCAA|2019-10-12|1437|6250|480|2019|6|10|12|4|2019|480|6250|Saturday|2019Q4|N|Y|N|2458758|2459030|2458404|2458677|N|N|N|N|N| +2458770|AAAAAAAACJEIFCAA|2019-10-13|1437|6250|480|2019|0|10|13|4|2019|480|6250|Sunday|2019Q4|N|N|N|2458758|2459030|2458405|2458678|N|N|N|N|N| +2458771|AAAAAAAADJEIFCAA|2019-10-14|1437|6250|480|2019|1|10|14|4|2019|480|6250|Monday|2019Q4|N|N|N|2458758|2459030|2458406|2458679|N|N|N|N|N| +2458772|AAAAAAAAEJEIFCAA|2019-10-15|1437|6251|480|2019|2|10|15|4|2019|480|6251|Tuesday|2019Q4|N|N|N|2458758|2459030|2458407|2458680|N|N|N|N|N| +2458773|AAAAAAAAFJEIFCAA|2019-10-16|1437|6251|480|2019|3|10|16|4|2019|480|6251|Wednesday|2019Q4|N|N|N|2458758|2459030|2458408|2458681|N|N|N|N|N| +2458774|AAAAAAAAGJEIFCAA|2019-10-17|1437|6251|480|2019|4|10|17|4|2019|480|6251|Thursday|2019Q4|N|N|N|2458758|2459030|2458409|2458682|N|N|N|N|N| +2458775|AAAAAAAAHJEIFCAA|2019-10-18|1437|6251|480|2019|5|10|18|4|2019|480|6251|Friday|2019Q4|N|Y|N|2458758|2459030|2458410|2458683|N|N|N|N|N| +2458776|AAAAAAAAIJEIFCAA|2019-10-19|1437|6251|480|2019|6|10|19|4|2019|480|6251|Saturday|2019Q4|N|Y|N|2458758|2459030|2458411|2458684|N|N|N|N|N| +2458777|AAAAAAAAJJEIFCAA|2019-10-20|1437|6251|480|2019|0|10|20|4|2019|480|6251|Sunday|2019Q4|N|N|N|2458758|2459030|2458412|2458685|N|N|N|N|N| +2458778|AAAAAAAAKJEIFCAA|2019-10-21|1437|6251|480|2019|1|10|21|4|2019|480|6251|Monday|2019Q4|N|N|N|2458758|2459030|2458413|2458686|N|N|N|N|N| +2458779|AAAAAAAALJEIFCAA|2019-10-22|1437|6252|480|2019|2|10|22|4|2019|480|6252|Tuesday|2019Q4|N|N|N|2458758|2459030|2458414|2458687|N|N|N|N|N| +2458780|AAAAAAAAMJEIFCAA|2019-10-23|1437|6252|480|2019|3|10|23|4|2019|480|6252|Wednesday|2019Q4|N|N|N|2458758|2459030|2458415|2458688|N|N|N|N|N| +2458781|AAAAAAAANJEIFCAA|2019-10-24|1437|6252|480|2019|4|10|24|4|2019|480|6252|Thursday|2019Q4|N|N|N|2458758|2459030|2458416|2458689|N|N|N|N|N| +2458782|AAAAAAAAOJEIFCAA|2019-10-25|1437|6252|480|2019|5|10|25|4|2019|480|6252|Friday|2019Q4|N|Y|N|2458758|2459030|2458417|2458690|N|N|N|N|N| +2458783|AAAAAAAAPJEIFCAA|2019-10-26|1437|6252|480|2019|6|10|26|4|2019|480|6252|Saturday|2019Q4|N|Y|N|2458758|2459030|2458418|2458691|N|N|N|N|N| +2458784|AAAAAAAAAKEIFCAA|2019-10-27|1437|6252|480|2019|0|10|27|4|2019|480|6252|Sunday|2019Q4|N|N|N|2458758|2459030|2458419|2458692|N|N|N|N|N| +2458785|AAAAAAAABKEIFCAA|2019-10-28|1437|6252|480|2019|1|10|28|4|2019|480|6252|Monday|2019Q4|N|N|N|2458758|2459030|2458420|2458693|N|N|N|N|N| +2458786|AAAAAAAACKEIFCAA|2019-10-29|1437|6253|480|2019|2|10|29|4|2019|480|6253|Tuesday|2019Q4|N|N|N|2458758|2459030|2458421|2458694|N|N|N|N|N| +2458787|AAAAAAAADKEIFCAA|2019-10-30|1437|6253|480|2019|3|10|30|4|2019|480|6253|Wednesday|2019Q4|N|N|N|2458758|2459030|2458422|2458695|N|N|N|N|N| +2458788|AAAAAAAAEKEIFCAA|2019-10-31|1437|6253|480|2019|4|10|31|4|2019|480|6253|Thursday|2019Q4|N|N|N|2458758|2459030|2458423|2458696|N|N|N|N|N| +2458789|AAAAAAAAFKEIFCAA|2019-11-01|1438|6253|480|2019|5|11|1|4|2019|480|6253|Friday|2019Q4|N|Y|N|2458789|2459092|2458424|2458697|N|N|N|N|N| +2458790|AAAAAAAAGKEIFCAA|2019-11-02|1438|6253|480|2019|6|11|2|4|2019|480|6253|Saturday|2019Q4|N|Y|N|2458789|2459092|2458425|2458698|N|N|N|N|N| +2458791|AAAAAAAAHKEIFCAA|2019-11-03|1438|6253|480|2019|0|11|3|4|2019|480|6253|Sunday|2019Q4|N|N|N|2458789|2459092|2458426|2458699|N|N|N|N|N| +2458792|AAAAAAAAIKEIFCAA|2019-11-04|1438|6253|480|2019|1|11|4|4|2019|480|6253|Monday|2019Q4|N|N|N|2458789|2459092|2458427|2458700|N|N|N|N|N| +2458793|AAAAAAAAJKEIFCAA|2019-11-05|1438|6254|480|2019|2|11|5|4|2019|480|6254|Tuesday|2019Q4|N|N|N|2458789|2459092|2458428|2458701|N|N|N|N|N| +2458794|AAAAAAAAKKEIFCAA|2019-11-06|1438|6254|480|2019|3|11|6|4|2019|480|6254|Wednesday|2019Q4|N|N|N|2458789|2459092|2458429|2458702|N|N|N|N|N| +2458795|AAAAAAAALKEIFCAA|2019-11-07|1438|6254|480|2019|4|11|7|4|2019|480|6254|Thursday|2019Q4|N|N|N|2458789|2459092|2458430|2458703|N|N|N|N|N| +2458796|AAAAAAAAMKEIFCAA|2019-11-08|1438|6254|480|2019|5|11|8|4|2019|480|6254|Friday|2019Q4|N|Y|N|2458789|2459092|2458431|2458704|N|N|N|N|N| +2458797|AAAAAAAANKEIFCAA|2019-11-09|1438|6254|480|2019|6|11|9|4|2019|480|6254|Saturday|2019Q4|N|Y|N|2458789|2459092|2458432|2458705|N|N|N|N|N| +2458798|AAAAAAAAOKEIFCAA|2019-11-10|1438|6254|480|2019|0|11|10|4|2019|480|6254|Sunday|2019Q4|N|N|N|2458789|2459092|2458433|2458706|N|N|N|N|N| +2458799|AAAAAAAAPKEIFCAA|2019-11-11|1438|6254|480|2019|1|11|11|4|2019|480|6254|Monday|2019Q4|N|N|N|2458789|2459092|2458434|2458707|N|N|N|N|N| +2458800|AAAAAAAAALEIFCAA|2019-11-12|1438|6255|480|2019|2|11|12|4|2019|480|6255|Tuesday|2019Q4|N|N|N|2458789|2459092|2458435|2458708|N|N|N|N|N| +2458801|AAAAAAAABLEIFCAA|2019-11-13|1438|6255|480|2019|3|11|13|4|2019|480|6255|Wednesday|2019Q4|N|N|N|2458789|2459092|2458436|2458709|N|N|N|N|N| +2458802|AAAAAAAACLEIFCAA|2019-11-14|1438|6255|480|2019|4|11|14|4|2019|480|6255|Thursday|2019Q4|N|N|N|2458789|2459092|2458437|2458710|N|N|N|N|N| +2458803|AAAAAAAADLEIFCAA|2019-11-15|1438|6255|480|2019|5|11|15|4|2019|480|6255|Friday|2019Q4|N|Y|N|2458789|2459092|2458438|2458711|N|N|N|N|N| +2458804|AAAAAAAAELEIFCAA|2019-11-16|1438|6255|480|2019|6|11|16|4|2019|480|6255|Saturday|2019Q4|N|Y|N|2458789|2459092|2458439|2458712|N|N|N|N|N| +2458805|AAAAAAAAFLEIFCAA|2019-11-17|1438|6255|480|2019|0|11|17|4|2019|480|6255|Sunday|2019Q4|N|N|N|2458789|2459092|2458440|2458713|N|N|N|N|N| +2458806|AAAAAAAAGLEIFCAA|2019-11-18|1438|6255|480|2019|1|11|18|4|2019|480|6255|Monday|2019Q4|N|N|N|2458789|2459092|2458441|2458714|N|N|N|N|N| +2458807|AAAAAAAAHLEIFCAA|2019-11-19|1438|6256|480|2019|2|11|19|4|2019|480|6256|Tuesday|2019Q4|N|N|N|2458789|2459092|2458442|2458715|N|N|N|N|N| +2458808|AAAAAAAAILEIFCAA|2019-11-20|1438|6256|480|2019|3|11|20|4|2019|480|6256|Wednesday|2019Q4|N|N|N|2458789|2459092|2458443|2458716|N|N|N|N|N| +2458809|AAAAAAAAJLEIFCAA|2019-11-21|1438|6256|480|2019|4|11|21|4|2019|480|6256|Thursday|2019Q4|N|N|N|2458789|2459092|2458444|2458717|N|N|N|N|N| +2458810|AAAAAAAAKLEIFCAA|2019-11-22|1438|6256|480|2019|5|11|22|4|2019|480|6256|Friday|2019Q4|N|Y|N|2458789|2459092|2458445|2458718|N|N|N|N|N| +2458811|AAAAAAAALLEIFCAA|2019-11-23|1438|6256|480|2019|6|11|23|4|2019|480|6256|Saturday|2019Q4|N|Y|N|2458789|2459092|2458446|2458719|N|N|N|N|N| +2458812|AAAAAAAAMLEIFCAA|2019-11-24|1438|6256|480|2019|0|11|24|4|2019|480|6256|Sunday|2019Q4|N|N|N|2458789|2459092|2458447|2458720|N|N|N|N|N| +2458813|AAAAAAAANLEIFCAA|2019-11-25|1438|6256|480|2019|1|11|25|4|2019|480|6256|Monday|2019Q4|N|N|N|2458789|2459092|2458448|2458721|N|N|N|N|N| +2458814|AAAAAAAAOLEIFCAA|2019-11-26|1438|6257|480|2019|2|11|26|4|2019|480|6257|Tuesday|2019Q4|N|N|N|2458789|2459092|2458449|2458722|N|N|N|N|N| +2458815|AAAAAAAAPLEIFCAA|2019-11-27|1438|6257|480|2019|3|11|27|4|2019|480|6257|Wednesday|2019Q4|N|N|N|2458789|2459092|2458450|2458723|N|N|N|N|N| +2458816|AAAAAAAAAMEIFCAA|2019-11-28|1438|6257|480|2019|4|11|28|4|2019|480|6257|Thursday|2019Q4|N|N|N|2458789|2459092|2458451|2458724|N|N|N|N|N| +2458817|AAAAAAAABMEIFCAA|2019-11-29|1438|6257|480|2019|5|11|29|4|2019|480|6257|Friday|2019Q4|N|Y|N|2458789|2459092|2458452|2458725|N|N|N|N|N| +2458818|AAAAAAAACMEIFCAA|2019-11-30|1438|6257|480|2019|6|11|30|4|2019|480|6257|Saturday|2019Q4|N|Y|N|2458789|2459092|2458453|2458726|N|N|N|N|N| +2458819|AAAAAAAADMEIFCAA|2019-12-01|1439|6257|481|2019|0|12|1|4|2019|481|6257|Sunday|2019Q4|N|N|N|2458819|2459152|2458454|2458727|N|N|N|N|N| +2458820|AAAAAAAAEMEIFCAA|2019-12-02|1439|6257|481|2019|1|12|2|4|2019|481|6257|Monday|2019Q4|N|N|N|2458819|2459152|2458455|2458728|N|N|N|N|N| +2458821|AAAAAAAAFMEIFCAA|2019-12-03|1439|6258|481|2019|2|12|3|4|2019|481|6258|Tuesday|2019Q4|N|N|N|2458819|2459152|2458456|2458729|N|N|N|N|N| +2458822|AAAAAAAAGMEIFCAA|2019-12-04|1439|6258|481|2019|3|12|4|4|2019|481|6258|Wednesday|2019Q4|N|N|N|2458819|2459152|2458457|2458730|N|N|N|N|N| +2458823|AAAAAAAAHMEIFCAA|2019-12-05|1439|6258|481|2019|4|12|5|4|2019|481|6258|Thursday|2019Q4|N|N|N|2458819|2459152|2458458|2458731|N|N|N|N|N| +2458824|AAAAAAAAIMEIFCAA|2019-12-06|1439|6258|481|2019|5|12|6|4|2019|481|6258|Friday|2019Q4|N|Y|N|2458819|2459152|2458459|2458732|N|N|N|N|N| +2458825|AAAAAAAAJMEIFCAA|2019-12-07|1439|6258|481|2019|6|12|7|4|2019|481|6258|Saturday|2019Q4|N|Y|N|2458819|2459152|2458460|2458733|N|N|N|N|N| +2458826|AAAAAAAAKMEIFCAA|2019-12-08|1439|6258|481|2019|0|12|8|4|2019|481|6258|Sunday|2019Q4|N|N|N|2458819|2459152|2458461|2458734|N|N|N|N|N| +2458827|AAAAAAAALMEIFCAA|2019-12-09|1439|6258|481|2019|1|12|9|4|2019|481|6258|Monday|2019Q4|N|N|N|2458819|2459152|2458462|2458735|N|N|N|N|N| +2458828|AAAAAAAAMMEIFCAA|2019-12-10|1439|6259|481|2019|2|12|10|4|2019|481|6259|Tuesday|2019Q4|N|N|N|2458819|2459152|2458463|2458736|N|N|N|N|N| +2458829|AAAAAAAANMEIFCAA|2019-12-11|1439|6259|481|2019|3|12|11|4|2019|481|6259|Wednesday|2019Q4|N|N|N|2458819|2459152|2458464|2458737|N|N|N|N|N| +2458830|AAAAAAAAOMEIFCAA|2019-12-12|1439|6259|481|2019|4|12|12|4|2019|481|6259|Thursday|2019Q4|N|N|N|2458819|2459152|2458465|2458738|N|N|N|N|N| +2458831|AAAAAAAAPMEIFCAA|2019-12-13|1439|6259|481|2019|5|12|13|4|2019|481|6259|Friday|2019Q4|N|Y|N|2458819|2459152|2458466|2458739|N|N|N|N|N| +2458832|AAAAAAAAANEIFCAA|2019-12-14|1439|6259|481|2019|6|12|14|4|2019|481|6259|Saturday|2019Q4|N|Y|N|2458819|2459152|2458467|2458740|N|N|N|N|N| +2458833|AAAAAAAABNEIFCAA|2019-12-15|1439|6259|481|2019|0|12|15|4|2019|481|6259|Sunday|2019Q4|N|N|N|2458819|2459152|2458468|2458741|N|N|N|N|N| +2458834|AAAAAAAACNEIFCAA|2019-12-16|1439|6259|481|2019|1|12|16|4|2019|481|6259|Monday|2019Q4|N|N|N|2458819|2459152|2458469|2458742|N|N|N|N|N| +2458835|AAAAAAAADNEIFCAA|2019-12-17|1439|6260|481|2019|2|12|17|4|2019|481|6260|Tuesday|2019Q4|N|N|N|2458819|2459152|2458470|2458743|N|N|N|N|N| +2458836|AAAAAAAAENEIFCAA|2019-12-18|1439|6260|481|2019|3|12|18|4|2019|481|6260|Wednesday|2019Q4|N|N|N|2458819|2459152|2458471|2458744|N|N|N|N|N| +2458837|AAAAAAAAFNEIFCAA|2019-12-19|1439|6260|481|2019|4|12|19|4|2019|481|6260|Thursday|2019Q4|N|N|N|2458819|2459152|2458472|2458745|N|N|N|N|N| +2458838|AAAAAAAAGNEIFCAA|2019-12-20|1439|6260|481|2019|5|12|20|4|2019|481|6260|Friday|2019Q4|N|Y|N|2458819|2459152|2458473|2458746|N|N|N|N|N| +2458839|AAAAAAAAHNEIFCAA|2019-12-21|1439|6260|481|2019|6|12|21|4|2019|481|6260|Saturday|2019Q4|N|Y|N|2458819|2459152|2458474|2458747|N|N|N|N|N| +2458840|AAAAAAAAINEIFCAA|2019-12-22|1439|6260|481|2019|0|12|22|4|2019|481|6260|Sunday|2019Q4|N|N|N|2458819|2459152|2458475|2458748|N|N|N|N|N| +2458841|AAAAAAAAJNEIFCAA|2019-12-23|1439|6260|481|2019|1|12|23|4|2019|481|6260|Monday|2019Q4|N|N|N|2458819|2459152|2458476|2458749|N|N|N|N|N| +2458842|AAAAAAAAKNEIFCAA|2019-12-24|1439|6261|481|2019|2|12|24|4|2019|481|6261|Tuesday|2019Q4|N|N|N|2458819|2459152|2458477|2458750|N|N|N|N|N| +2458843|AAAAAAAALNEIFCAA|2019-12-25|1439|6261|481|2019|3|12|25|4|2019|481|6261|Wednesday|2019Q4|N|N|N|2458819|2459152|2458478|2458751|N|N|N|N|N| +2458844|AAAAAAAAMNEIFCAA|2019-12-26|1439|6261|481|2019|4|12|26|4|2019|481|6261|Thursday|2019Q4|Y|N|N|2458819|2459152|2458479|2458752|N|N|N|N|N| +2458845|AAAAAAAANNEIFCAA|2019-12-27|1439|6261|481|2019|5|12|27|4|2019|481|6261|Friday|2019Q4|N|Y|Y|2458819|2459152|2458480|2458753|N|N|N|N|N| +2458846|AAAAAAAAONEIFCAA|2019-12-28|1439|6261|481|2019|6|12|28|4|2019|481|6261|Saturday|2019Q4|N|Y|N|2458819|2459152|2458481|2458754|N|N|N|N|N| +2458847|AAAAAAAAPNEIFCAA|2019-12-29|1439|6261|481|2019|0|12|29|4|2019|481|6261|Sunday|2019Q4|N|N|N|2458819|2459152|2458482|2458755|N|N|N|N|N| +2458848|AAAAAAAAAOEIFCAA|2019-12-30|1439|6261|481|2019|1|12|30|4|2019|481|6261|Monday|2019Q4|N|N|N|2458819|2459152|2458483|2458756|N|N|N|N|N| +2458849|AAAAAAAABOEIFCAA|2019-12-31|1439|6262|481|2019|2|12|31|4|2019|481|6262|Tuesday|2019Q4|N|N|N|2458819|2459152|2458484|2458757|N|N|N|N|N| +2458850|AAAAAAAACOEIFCAA|2020-01-01|1440|6262|481|2020|3|1|1|1|2020|481|6262|Wednesday|2020Q1|Y|N|N|2458850|2458849|2458485|2458758|N|N|N|N|N| +2458851|AAAAAAAADOEIFCAA|2020-01-02|1440|6262|481|2020|4|1|2|1|2020|481|6262|Thursday|2020Q1|N|N|Y|2458850|2458849|2458486|2458759|N|N|N|N|N| +2458852|AAAAAAAAEOEIFCAA|2020-01-03|1440|6262|481|2020|5|1|3|1|2020|481|6262|Friday|2020Q1|N|Y|N|2458850|2458849|2458487|2458760|N|N|N|N|N| +2458853|AAAAAAAAFOEIFCAA|2020-01-04|1440|6262|481|2020|6|1|4|1|2020|481|6262|Saturday|2020Q1|N|Y|N|2458850|2458849|2458488|2458761|N|N|N|N|N| +2458854|AAAAAAAAGOEIFCAA|2020-01-05|1440|6262|481|2020|0|1|5|1|2020|481|6262|Sunday|2020Q1|N|N|N|2458850|2458849|2458489|2458762|N|N|N|N|N| +2458855|AAAAAAAAHOEIFCAA|2020-01-06|1440|6262|481|2020|1|1|6|1|2020|481|6262|Monday|2020Q1|N|N|N|2458850|2458849|2458490|2458763|N|N|N|N|N| +2458856|AAAAAAAAIOEIFCAA|2020-01-07|1440|6263|481|2020|2|1|7|1|2020|481|6263|Tuesday|2020Q1|N|N|N|2458850|2458849|2458491|2458764|N|N|N|N|N| +2458857|AAAAAAAAJOEIFCAA|2020-01-08|1440|6263|481|2020|3|1|8|1|2020|481|6263|Wednesday|2020Q1|N|N|N|2458850|2458849|2458492|2458765|N|N|N|N|N| +2458858|AAAAAAAAKOEIFCAA|2020-01-09|1440|6263|481|2020|4|1|9|1|2020|481|6263|Thursday|2020Q1|N|N|N|2458850|2458849|2458493|2458766|N|N|N|N|N| +2458859|AAAAAAAALOEIFCAA|2020-01-10|1440|6263|481|2020|5|1|10|1|2020|481|6263|Friday|2020Q1|N|Y|N|2458850|2458849|2458494|2458767|N|N|N|N|N| +2458860|AAAAAAAAMOEIFCAA|2020-01-11|1440|6263|481|2020|6|1|11|1|2020|481|6263|Saturday|2020Q1|N|Y|N|2458850|2458849|2458495|2458768|N|N|N|N|N| +2458861|AAAAAAAANOEIFCAA|2020-01-12|1440|6263|481|2020|0|1|12|1|2020|481|6263|Sunday|2020Q1|N|N|N|2458850|2458849|2458496|2458769|N|N|N|N|N| +2458862|AAAAAAAAOOEIFCAA|2020-01-13|1440|6263|481|2020|1|1|13|1|2020|481|6263|Monday|2020Q1|N|N|N|2458850|2458849|2458497|2458770|N|N|N|N|N| +2458863|AAAAAAAAPOEIFCAA|2020-01-14|1440|6264|481|2020|2|1|14|1|2020|481|6264|Tuesday|2020Q1|N|N|N|2458850|2458849|2458498|2458771|N|N|N|N|N| +2458864|AAAAAAAAAPEIFCAA|2020-01-15|1440|6264|481|2020|3|1|15|1|2020|481|6264|Wednesday|2020Q1|N|N|N|2458850|2458849|2458499|2458772|N|N|N|N|N| +2458865|AAAAAAAABPEIFCAA|2020-01-16|1440|6264|481|2020|4|1|16|1|2020|481|6264|Thursday|2020Q1|N|N|N|2458850|2458849|2458500|2458773|N|N|N|N|N| +2458866|AAAAAAAACPEIFCAA|2020-01-17|1440|6264|481|2020|5|1|17|1|2020|481|6264|Friday|2020Q1|N|Y|N|2458850|2458849|2458501|2458774|N|N|N|N|N| +2458867|AAAAAAAADPEIFCAA|2020-01-18|1440|6264|481|2020|6|1|18|1|2020|481|6264|Saturday|2020Q1|N|Y|N|2458850|2458849|2458502|2458775|N|N|N|N|N| +2458868|AAAAAAAAEPEIFCAA|2020-01-19|1440|6264|481|2020|0|1|19|1|2020|481|6264|Sunday|2020Q1|N|N|N|2458850|2458849|2458503|2458776|N|N|N|N|N| +2458869|AAAAAAAAFPEIFCAA|2020-01-20|1440|6264|481|2020|1|1|20|1|2020|481|6264|Monday|2020Q1|N|N|N|2458850|2458849|2458504|2458777|N|N|N|N|N| +2458870|AAAAAAAAGPEIFCAA|2020-01-21|1440|6265|481|2020|2|1|21|1|2020|481|6265|Tuesday|2020Q1|N|N|N|2458850|2458849|2458505|2458778|N|N|N|N|N| +2458871|AAAAAAAAHPEIFCAA|2020-01-22|1440|6265|481|2020|3|1|22|1|2020|481|6265|Wednesday|2020Q1|N|N|N|2458850|2458849|2458506|2458779|N|N|N|N|N| +2458872|AAAAAAAAIPEIFCAA|2020-01-23|1440|6265|481|2020|4|1|23|1|2020|481|6265|Thursday|2020Q1|N|N|N|2458850|2458849|2458507|2458780|N|N|N|N|N| +2458873|AAAAAAAAJPEIFCAA|2020-01-24|1440|6265|481|2020|5|1|24|1|2020|481|6265|Friday|2020Q1|N|Y|N|2458850|2458849|2458508|2458781|N|N|N|N|N| +2458874|AAAAAAAAKPEIFCAA|2020-01-25|1440|6265|481|2020|6|1|25|1|2020|481|6265|Saturday|2020Q1|N|Y|N|2458850|2458849|2458509|2458782|N|N|N|N|N| +2458875|AAAAAAAALPEIFCAA|2020-01-26|1440|6265|481|2020|0|1|26|1|2020|481|6265|Sunday|2020Q1|N|N|N|2458850|2458849|2458510|2458783|N|N|N|N|N| +2458876|AAAAAAAAMPEIFCAA|2020-01-27|1440|6265|481|2020|1|1|27|1|2020|481|6265|Monday|2020Q1|N|N|N|2458850|2458849|2458511|2458784|N|N|N|N|N| +2458877|AAAAAAAANPEIFCAA|2020-01-28|1440|6266|481|2020|2|1|28|1|2020|481|6266|Tuesday|2020Q1|N|N|N|2458850|2458849|2458512|2458785|N|N|N|N|N| +2458878|AAAAAAAAOPEIFCAA|2020-01-29|1440|6266|481|2020|3|1|29|1|2020|481|6266|Wednesday|2020Q1|N|N|N|2458850|2458849|2458513|2458786|N|N|N|N|N| +2458879|AAAAAAAAPPEIFCAA|2020-01-30|1440|6266|481|2020|4|1|30|1|2020|481|6266|Thursday|2020Q1|N|N|N|2458850|2458849|2458514|2458787|N|N|N|N|N| +2458880|AAAAAAAAAAFIFCAA|2020-01-31|1440|6266|481|2020|5|1|31|1|2020|481|6266|Friday|2020Q1|N|Y|N|2458850|2458849|2458515|2458788|N|N|N|N|N| +2458881|AAAAAAAABAFIFCAA|2020-02-01|1441|6266|481|2020|6|2|1|1|2020|481|6266|Saturday|2020Q1|N|Y|N|2458881|2458911|2458516|2458789|N|N|N|N|N| +2458882|AAAAAAAACAFIFCAA|2020-02-02|1441|6266|481|2020|0|2|2|1|2020|481|6266|Sunday|2020Q1|N|N|N|2458881|2458911|2458517|2458790|N|N|N|N|N| +2458883|AAAAAAAADAFIFCAA|2020-02-03|1441|6266|481|2020|1|2|3|1|2020|481|6266|Monday|2020Q1|N|N|N|2458881|2458911|2458518|2458791|N|N|N|N|N| +2458884|AAAAAAAAEAFIFCAA|2020-02-04|1441|6267|481|2020|2|2|4|1|2020|481|6267|Tuesday|2020Q1|N|N|N|2458881|2458911|2458519|2458792|N|N|N|N|N| +2458885|AAAAAAAAFAFIFCAA|2020-02-05|1441|6267|481|2020|3|2|5|1|2020|481|6267|Wednesday|2020Q1|N|N|N|2458881|2458911|2458520|2458793|N|N|N|N|N| +2458886|AAAAAAAAGAFIFCAA|2020-02-06|1441|6267|481|2020|4|2|6|1|2020|481|6267|Thursday|2020Q1|N|N|N|2458881|2458911|2458521|2458794|N|N|N|N|N| +2458887|AAAAAAAAHAFIFCAA|2020-02-07|1441|6267|481|2020|5|2|7|1|2020|481|6267|Friday|2020Q1|N|Y|N|2458881|2458911|2458522|2458795|N|N|N|N|N| +2458888|AAAAAAAAIAFIFCAA|2020-02-08|1441|6267|481|2020|6|2|8|1|2020|481|6267|Saturday|2020Q1|N|Y|N|2458881|2458911|2458523|2458796|N|N|N|N|N| +2458889|AAAAAAAAJAFIFCAA|2020-02-09|1441|6267|481|2020|0|2|9|1|2020|481|6267|Sunday|2020Q1|N|N|N|2458881|2458911|2458524|2458797|N|N|N|N|N| +2458890|AAAAAAAAKAFIFCAA|2020-02-10|1441|6267|481|2020|1|2|10|1|2020|481|6267|Monday|2020Q1|N|N|N|2458881|2458911|2458525|2458798|N|N|N|N|N| +2458891|AAAAAAAALAFIFCAA|2020-02-11|1441|6268|481|2020|2|2|11|1|2020|481|6268|Tuesday|2020Q1|N|N|N|2458881|2458911|2458526|2458799|N|N|N|N|N| +2458892|AAAAAAAAMAFIFCAA|2020-02-12|1441|6268|481|2020|3|2|12|1|2020|481|6268|Wednesday|2020Q1|N|N|N|2458881|2458911|2458527|2458800|N|N|N|N|N| +2458893|AAAAAAAANAFIFCAA|2020-02-13|1441|6268|481|2020|4|2|13|1|2020|481|6268|Thursday|2020Q1|N|N|N|2458881|2458911|2458528|2458801|N|N|N|N|N| +2458894|AAAAAAAAOAFIFCAA|2020-02-14|1441|6268|481|2020|5|2|14|1|2020|481|6268|Friday|2020Q1|N|Y|N|2458881|2458911|2458529|2458802|N|N|N|N|N| +2458895|AAAAAAAAPAFIFCAA|2020-02-15|1441|6268|481|2020|6|2|15|1|2020|481|6268|Saturday|2020Q1|N|Y|N|2458881|2458911|2458530|2458803|N|N|N|N|N| +2458896|AAAAAAAAABFIFCAA|2020-02-16|1441|6268|481|2020|0|2|16|1|2020|481|6268|Sunday|2020Q1|N|N|N|2458881|2458911|2458531|2458804|N|N|N|N|N| +2458897|AAAAAAAABBFIFCAA|2020-02-17|1441|6268|481|2020|1|2|17|1|2020|481|6268|Monday|2020Q1|N|N|N|2458881|2458911|2458532|2458805|N|N|N|N|N| +2458898|AAAAAAAACBFIFCAA|2020-02-18|1441|6269|481|2020|2|2|18|1|2020|481|6269|Tuesday|2020Q1|N|N|N|2458881|2458911|2458533|2458806|N|N|N|N|N| +2458899|AAAAAAAADBFIFCAA|2020-02-19|1441|6269|481|2020|3|2|19|1|2020|481|6269|Wednesday|2020Q1|N|N|N|2458881|2458911|2458534|2458807|N|N|N|N|N| +2458900|AAAAAAAAEBFIFCAA|2020-02-20|1441|6269|481|2020|4|2|20|1|2020|481|6269|Thursday|2020Q1|N|N|N|2458881|2458911|2458535|2458808|N|N|N|N|N| +2458901|AAAAAAAAFBFIFCAA|2020-02-21|1441|6269|481|2020|5|2|21|1|2020|481|6269|Friday|2020Q1|N|Y|N|2458881|2458911|2458536|2458809|N|N|N|N|N| +2458902|AAAAAAAAGBFIFCAA|2020-02-22|1441|6269|481|2020|6|2|22|1|2020|481|6269|Saturday|2020Q1|N|Y|N|2458881|2458911|2458537|2458810|N|N|N|N|N| +2458903|AAAAAAAAHBFIFCAA|2020-02-23|1441|6269|481|2020|0|2|23|1|2020|481|6269|Sunday|2020Q1|N|N|N|2458881|2458911|2458538|2458811|N|N|N|N|N| +2458904|AAAAAAAAIBFIFCAA|2020-02-24|1441|6269|481|2020|1|2|24|1|2020|481|6269|Monday|2020Q1|N|N|N|2458881|2458911|2458539|2458812|N|N|N|N|N| +2458905|AAAAAAAAJBFIFCAA|2020-02-25|1441|6270|481|2020|2|2|25|1|2020|481|6270|Tuesday|2020Q1|N|N|N|2458881|2458911|2458540|2458813|N|N|N|N|N| +2458906|AAAAAAAAKBFIFCAA|2020-02-26|1441|6270|481|2020|3|2|26|1|2020|481|6270|Wednesday|2020Q1|N|N|N|2458881|2458911|2458541|2458814|N|N|N|N|N| +2458907|AAAAAAAALBFIFCAA|2020-02-27|1441|6270|481|2020|4|2|27|1|2020|481|6270|Thursday|2020Q1|N|N|N|2458881|2458911|2458542|2458815|N|N|N|N|N| +2458908|AAAAAAAAMBFIFCAA|2020-02-28|1441|6270|481|2020|5|2|28|1|2020|481|6270|Friday|2020Q1|N|Y|N|2458881|2458911|2458543|2458816|N|N|N|N|N| +2458909|AAAAAAAANBFIFCAA|2020-02-29|1441|6270|481|2020|6|2|29|1|2020|481|6270|Saturday|2020Q1|N|Y|N|2458881|2458911|2458543|2458817|N|N|N|N|N| +2458910|AAAAAAAAOBFIFCAA|2020-03-01|1442|6270|482|2020|0|3|1|1|2020|482|6270|Sunday|2020Q1|N|N|N|2458910|2458969|2458544|2458818|N|N|N|N|N| +2458911|AAAAAAAAPBFIFCAA|2020-03-02|1442|6270|482|2020|1|3|2|1|2020|482|6270|Monday|2020Q1|N|N|N|2458910|2458969|2458545|2458819|N|N|N|N|N| +2458912|AAAAAAAAACFIFCAA|2020-03-03|1442|6271|482|2020|2|3|3|1|2020|482|6271|Tuesday|2020Q1|N|N|N|2458910|2458969|2458546|2458820|N|N|N|N|N| +2458913|AAAAAAAABCFIFCAA|2020-03-04|1442|6271|482|2020|3|3|4|1|2020|482|6271|Wednesday|2020Q1|N|N|N|2458910|2458969|2458547|2458821|N|N|N|N|N| +2458914|AAAAAAAACCFIFCAA|2020-03-05|1442|6271|482|2020|4|3|5|1|2020|482|6271|Thursday|2020Q1|N|N|N|2458910|2458969|2458548|2458822|N|N|N|N|N| +2458915|AAAAAAAADCFIFCAA|2020-03-06|1442|6271|482|2020|5|3|6|1|2020|482|6271|Friday|2020Q1|N|Y|N|2458910|2458969|2458549|2458823|N|N|N|N|N| +2458916|AAAAAAAAECFIFCAA|2020-03-07|1442|6271|482|2020|6|3|7|1|2020|482|6271|Saturday|2020Q1|N|Y|N|2458910|2458969|2458550|2458824|N|N|N|N|N| +2458917|AAAAAAAAFCFIFCAA|2020-03-08|1442|6271|482|2020|0|3|8|1|2020|482|6271|Sunday|2020Q1|N|N|N|2458910|2458969|2458551|2458825|N|N|N|N|N| +2458918|AAAAAAAAGCFIFCAA|2020-03-09|1442|6271|482|2020|1|3|9|1|2020|482|6271|Monday|2020Q1|N|N|N|2458910|2458969|2458552|2458826|N|N|N|N|N| +2458919|AAAAAAAAHCFIFCAA|2020-03-10|1442|6272|482|2020|2|3|10|1|2020|482|6272|Tuesday|2020Q1|N|N|N|2458910|2458969|2458553|2458827|N|N|N|N|N| +2458920|AAAAAAAAICFIFCAA|2020-03-11|1442|6272|482|2020|3|3|11|1|2020|482|6272|Wednesday|2020Q1|N|N|N|2458910|2458969|2458554|2458828|N|N|N|N|N| +2458921|AAAAAAAAJCFIFCAA|2020-03-12|1442|6272|482|2020|4|3|12|1|2020|482|6272|Thursday|2020Q1|N|N|N|2458910|2458969|2458555|2458829|N|N|N|N|N| +2458922|AAAAAAAAKCFIFCAA|2020-03-13|1442|6272|482|2020|5|3|13|1|2020|482|6272|Friday|2020Q1|N|Y|N|2458910|2458969|2458556|2458830|N|N|N|N|N| +2458923|AAAAAAAALCFIFCAA|2020-03-14|1442|6272|482|2020|6|3|14|1|2020|482|6272|Saturday|2020Q1|N|Y|N|2458910|2458969|2458557|2458831|N|N|N|N|N| +2458924|AAAAAAAAMCFIFCAA|2020-03-15|1442|6272|482|2020|0|3|15|1|2020|482|6272|Sunday|2020Q1|N|N|N|2458910|2458969|2458558|2458832|N|N|N|N|N| +2458925|AAAAAAAANCFIFCAA|2020-03-16|1442|6272|482|2020|1|3|16|1|2020|482|6272|Monday|2020Q1|N|N|N|2458910|2458969|2458559|2458833|N|N|N|N|N| +2458926|AAAAAAAAOCFIFCAA|2020-03-17|1442|6273|482|2020|2|3|17|1|2020|482|6273|Tuesday|2020Q1|N|N|N|2458910|2458969|2458560|2458834|N|N|N|N|N| +2458927|AAAAAAAAPCFIFCAA|2020-03-18|1442|6273|482|2020|3|3|18|1|2020|482|6273|Wednesday|2020Q1|N|N|N|2458910|2458969|2458561|2458835|N|N|N|N|N| +2458928|AAAAAAAAADFIFCAA|2020-03-19|1442|6273|482|2020|4|3|19|1|2020|482|6273|Thursday|2020Q1|N|N|N|2458910|2458969|2458562|2458836|N|N|N|N|N| +2458929|AAAAAAAABDFIFCAA|2020-03-20|1442|6273|482|2020|5|3|20|1|2020|482|6273|Friday|2020Q1|N|Y|N|2458910|2458969|2458563|2458837|N|N|N|N|N| +2458930|AAAAAAAACDFIFCAA|2020-03-21|1442|6273|482|2020|6|3|21|1|2020|482|6273|Saturday|2020Q1|N|Y|N|2458910|2458969|2458564|2458838|N|N|N|N|N| +2458931|AAAAAAAADDFIFCAA|2020-03-22|1442|6273|482|2020|0|3|22|1|2020|482|6273|Sunday|2020Q1|N|N|N|2458910|2458969|2458565|2458839|N|N|N|N|N| +2458932|AAAAAAAAEDFIFCAA|2020-03-23|1442|6273|482|2020|1|3|23|1|2020|482|6273|Monday|2020Q1|N|N|N|2458910|2458969|2458566|2458840|N|N|N|N|N| +2458933|AAAAAAAAFDFIFCAA|2020-03-24|1442|6274|482|2020|2|3|24|1|2020|482|6274|Tuesday|2020Q1|N|N|N|2458910|2458969|2458567|2458841|N|N|N|N|N| +2458934|AAAAAAAAGDFIFCAA|2020-03-25|1442|6274|482|2020|3|3|25|1|2020|482|6274|Wednesday|2020Q1|N|N|N|2458910|2458969|2458568|2458842|N|N|N|N|N| +2458935|AAAAAAAAHDFIFCAA|2020-03-26|1442|6274|482|2020|4|3|26|1|2020|482|6274|Thursday|2020Q1|N|N|N|2458910|2458969|2458569|2458843|N|N|N|N|N| +2458936|AAAAAAAAIDFIFCAA|2020-03-27|1442|6274|482|2020|5|3|27|1|2020|482|6274|Friday|2020Q1|N|Y|N|2458910|2458969|2458570|2458844|N|N|N|N|N| +2458937|AAAAAAAAJDFIFCAA|2020-03-28|1442|6274|482|2020|6|3|28|1|2020|482|6274|Saturday|2020Q1|N|Y|N|2458910|2458969|2458571|2458845|N|N|N|N|N| +2458938|AAAAAAAAKDFIFCAA|2020-03-29|1442|6274|482|2020|0|3|29|1|2020|482|6274|Sunday|2020Q1|N|N|N|2458910|2458969|2458572|2458846|N|N|N|N|N| +2458939|AAAAAAAALDFIFCAA|2020-03-30|1442|6274|482|2020|1|3|30|1|2020|482|6274|Monday|2020Q1|N|N|N|2458910|2458969|2458573|2458847|N|N|N|N|N| +2458940|AAAAAAAAMDFIFCAA|2020-03-31|1442|6275|482|2020|2|3|31|1|2020|482|6275|Tuesday|2020Q1|N|N|N|2458910|2458969|2458574|2458848|N|N|N|N|N| +2458941|AAAAAAAANDFIFCAA|2020-04-01|1443|6275|482|2020|3|4|1|2|2020|482|6275|Wednesday|2020Q2|N|N|N|2458941|2459031|2458575|2458850|N|N|N|N|N| +2458942|AAAAAAAAODFIFCAA|2020-04-02|1443|6275|482|2020|4|4|2|2|2020|482|6275|Thursday|2020Q2|N|N|N|2458941|2459031|2458576|2458851|N|N|N|N|N| +2458943|AAAAAAAAPDFIFCAA|2020-04-03|1443|6275|482|2020|5|4|3|2|2020|482|6275|Friday|2020Q2|N|Y|N|2458941|2459031|2458577|2458852|N|N|N|N|N| +2458944|AAAAAAAAAEFIFCAA|2020-04-04|1443|6275|482|2020|6|4|4|2|2020|482|6275|Saturday|2020Q2|N|Y|N|2458941|2459031|2458578|2458853|N|N|N|N|N| +2458945|AAAAAAAABEFIFCAA|2020-04-05|1443|6275|482|2020|0|4|5|2|2020|482|6275|Sunday|2020Q2|N|N|N|2458941|2459031|2458579|2458854|N|N|N|N|N| +2458946|AAAAAAAACEFIFCAA|2020-04-06|1443|6275|482|2020|1|4|6|2|2020|482|6275|Monday|2020Q2|N|N|N|2458941|2459031|2458580|2458855|N|N|N|N|N| +2458947|AAAAAAAADEFIFCAA|2020-04-07|1443|6276|482|2020|2|4|7|2|2020|482|6276|Tuesday|2020Q2|N|N|N|2458941|2459031|2458581|2458856|N|N|N|N|N| +2458948|AAAAAAAAEEFIFCAA|2020-04-08|1443|6276|482|2020|3|4|8|2|2020|482|6276|Wednesday|2020Q2|N|N|N|2458941|2459031|2458582|2458857|N|N|N|N|N| +2458949|AAAAAAAAFEFIFCAA|2020-04-09|1443|6276|482|2020|4|4|9|2|2020|482|6276|Thursday|2020Q2|N|N|N|2458941|2459031|2458583|2458858|N|N|N|N|N| +2458950|AAAAAAAAGEFIFCAA|2020-04-10|1443|6276|482|2020|5|4|10|2|2020|482|6276|Friday|2020Q2|N|Y|N|2458941|2459031|2458584|2458859|N|N|N|N|N| +2458951|AAAAAAAAHEFIFCAA|2020-04-11|1443|6276|482|2020|6|4|11|2|2020|482|6276|Saturday|2020Q2|N|Y|N|2458941|2459031|2458585|2458860|N|N|N|N|N| +2458952|AAAAAAAAIEFIFCAA|2020-04-12|1443|6276|482|2020|0|4|12|2|2020|482|6276|Sunday|2020Q2|N|N|N|2458941|2459031|2458586|2458861|N|N|N|N|N| +2458953|AAAAAAAAJEFIFCAA|2020-04-13|1443|6276|482|2020|1|4|13|2|2020|482|6276|Monday|2020Q2|N|N|N|2458941|2459031|2458587|2458862|N|N|N|N|N| +2458954|AAAAAAAAKEFIFCAA|2020-04-14|1443|6277|482|2020|2|4|14|2|2020|482|6277|Tuesday|2020Q2|N|N|N|2458941|2459031|2458588|2458863|N|N|N|N|N| +2458955|AAAAAAAALEFIFCAA|2020-04-15|1443|6277|482|2020|3|4|15|2|2020|482|6277|Wednesday|2020Q2|N|N|N|2458941|2459031|2458589|2458864|N|N|N|N|N| +2458956|AAAAAAAAMEFIFCAA|2020-04-16|1443|6277|482|2020|4|4|16|2|2020|482|6277|Thursday|2020Q2|N|N|N|2458941|2459031|2458590|2458865|N|N|N|N|N| +2458957|AAAAAAAANEFIFCAA|2020-04-17|1443|6277|482|2020|5|4|17|2|2020|482|6277|Friday|2020Q2|N|Y|N|2458941|2459031|2458591|2458866|N|N|N|N|N| +2458958|AAAAAAAAOEFIFCAA|2020-04-18|1443|6277|482|2020|6|4|18|2|2020|482|6277|Saturday|2020Q2|N|Y|N|2458941|2459031|2458592|2458867|N|N|N|N|N| +2458959|AAAAAAAAPEFIFCAA|2020-04-19|1443|6277|482|2020|0|4|19|2|2020|482|6277|Sunday|2020Q2|N|N|N|2458941|2459031|2458593|2458868|N|N|N|N|N| +2458960|AAAAAAAAAFFIFCAA|2020-04-20|1443|6277|482|2020|1|4|20|2|2020|482|6277|Monday|2020Q2|N|N|N|2458941|2459031|2458594|2458869|N|N|N|N|N| +2458961|AAAAAAAABFFIFCAA|2020-04-21|1443|6278|482|2020|2|4|21|2|2020|482|6278|Tuesday|2020Q2|N|N|N|2458941|2459031|2458595|2458870|N|N|N|N|N| +2458962|AAAAAAAACFFIFCAA|2020-04-22|1443|6278|482|2020|3|4|22|2|2020|482|6278|Wednesday|2020Q2|N|N|N|2458941|2459031|2458596|2458871|N|N|N|N|N| +2458963|AAAAAAAADFFIFCAA|2020-04-23|1443|6278|482|2020|4|4|23|2|2020|482|6278|Thursday|2020Q2|N|N|N|2458941|2459031|2458597|2458872|N|N|N|N|N| +2458964|AAAAAAAAEFFIFCAA|2020-04-24|1443|6278|482|2020|5|4|24|2|2020|482|6278|Friday|2020Q2|N|Y|N|2458941|2459031|2458598|2458873|N|N|N|N|N| +2458965|AAAAAAAAFFFIFCAA|2020-04-25|1443|6278|482|2020|6|4|25|2|2020|482|6278|Saturday|2020Q2|N|Y|N|2458941|2459031|2458599|2458874|N|N|N|N|N| +2458966|AAAAAAAAGFFIFCAA|2020-04-26|1443|6278|482|2020|0|4|26|2|2020|482|6278|Sunday|2020Q2|N|N|N|2458941|2459031|2458600|2458875|N|N|N|N|N| +2458967|AAAAAAAAHFFIFCAA|2020-04-27|1443|6278|482|2020|1|4|27|2|2020|482|6278|Monday|2020Q2|N|N|N|2458941|2459031|2458601|2458876|N|N|N|N|N| +2458968|AAAAAAAAIFFIFCAA|2020-04-28|1443|6279|482|2020|2|4|28|2|2020|482|6279|Tuesday|2020Q2|N|N|N|2458941|2459031|2458602|2458877|N|N|N|N|N| +2458969|AAAAAAAAJFFIFCAA|2020-04-29|1443|6279|482|2020|3|4|29|2|2020|482|6279|Wednesday|2020Q2|N|N|N|2458941|2459031|2458603|2458878|N|N|N|N|N| +2458970|AAAAAAAAKFFIFCAA|2020-04-30|1443|6279|482|2020|4|4|30|2|2020|482|6279|Thursday|2020Q2|N|N|N|2458941|2459031|2458604|2458879|N|N|N|N|N| +2458971|AAAAAAAALFFIFCAA|2020-05-01|1444|6279|482|2020|5|5|1|2|2020|482|6279|Friday|2020Q2|N|Y|N|2458971|2459091|2458605|2458880|N|N|N|N|N| +2458972|AAAAAAAAMFFIFCAA|2020-05-02|1444|6279|482|2020|6|5|2|2|2020|482|6279|Saturday|2020Q2|N|Y|N|2458971|2459091|2458606|2458881|N|N|N|N|N| +2458973|AAAAAAAANFFIFCAA|2020-05-03|1444|6279|482|2020|0|5|3|2|2020|482|6279|Sunday|2020Q2|N|N|N|2458971|2459091|2458607|2458882|N|N|N|N|N| +2458974|AAAAAAAAOFFIFCAA|2020-05-04|1444|6279|482|2020|1|5|4|2|2020|482|6279|Monday|2020Q2|N|N|N|2458971|2459091|2458608|2458883|N|N|N|N|N| +2458975|AAAAAAAAPFFIFCAA|2020-05-05|1444|6280|482|2020|2|5|5|2|2020|482|6280|Tuesday|2020Q2|N|N|N|2458971|2459091|2458609|2458884|N|N|N|N|N| +2458976|AAAAAAAAAGFIFCAA|2020-05-06|1444|6280|482|2020|3|5|6|2|2020|482|6280|Wednesday|2020Q2|N|N|N|2458971|2459091|2458610|2458885|N|N|N|N|N| +2458977|AAAAAAAABGFIFCAA|2020-05-07|1444|6280|482|2020|4|5|7|2|2020|482|6280|Thursday|2020Q2|N|N|N|2458971|2459091|2458611|2458886|N|N|N|N|N| +2458978|AAAAAAAACGFIFCAA|2020-05-08|1444|6280|482|2020|5|5|8|2|2020|482|6280|Friday|2020Q2|N|Y|N|2458971|2459091|2458612|2458887|N|N|N|N|N| +2458979|AAAAAAAADGFIFCAA|2020-05-09|1444|6280|482|2020|6|5|9|2|2020|482|6280|Saturday|2020Q2|N|Y|N|2458971|2459091|2458613|2458888|N|N|N|N|N| +2458980|AAAAAAAAEGFIFCAA|2020-05-10|1444|6280|482|2020|0|5|10|2|2020|482|6280|Sunday|2020Q2|N|N|N|2458971|2459091|2458614|2458889|N|N|N|N|N| +2458981|AAAAAAAAFGFIFCAA|2020-05-11|1444|6280|482|2020|1|5|11|2|2020|482|6280|Monday|2020Q2|N|N|N|2458971|2459091|2458615|2458890|N|N|N|N|N| +2458982|AAAAAAAAGGFIFCAA|2020-05-12|1444|6281|482|2020|2|5|12|2|2020|482|6281|Tuesday|2020Q2|N|N|N|2458971|2459091|2458616|2458891|N|N|N|N|N| +2458983|AAAAAAAAHGFIFCAA|2020-05-13|1444|6281|482|2020|3|5|13|2|2020|482|6281|Wednesday|2020Q2|N|N|N|2458971|2459091|2458617|2458892|N|N|N|N|N| +2458984|AAAAAAAAIGFIFCAA|2020-05-14|1444|6281|482|2020|4|5|14|2|2020|482|6281|Thursday|2020Q2|N|N|N|2458971|2459091|2458618|2458893|N|N|N|N|N| +2458985|AAAAAAAAJGFIFCAA|2020-05-15|1444|6281|482|2020|5|5|15|2|2020|482|6281|Friday|2020Q2|N|Y|N|2458971|2459091|2458619|2458894|N|N|N|N|N| +2458986|AAAAAAAAKGFIFCAA|2020-05-16|1444|6281|482|2020|6|5|16|2|2020|482|6281|Saturday|2020Q2|N|Y|N|2458971|2459091|2458620|2458895|N|N|N|N|N| +2458987|AAAAAAAALGFIFCAA|2020-05-17|1444|6281|482|2020|0|5|17|2|2020|482|6281|Sunday|2020Q2|N|N|N|2458971|2459091|2458621|2458896|N|N|N|N|N| +2458988|AAAAAAAAMGFIFCAA|2020-05-18|1444|6281|482|2020|1|5|18|2|2020|482|6281|Monday|2020Q2|N|N|N|2458971|2459091|2458622|2458897|N|N|N|N|N| +2458989|AAAAAAAANGFIFCAA|2020-05-19|1444|6282|482|2020|2|5|19|2|2020|482|6282|Tuesday|2020Q2|N|N|N|2458971|2459091|2458623|2458898|N|N|N|N|N| +2458990|AAAAAAAAOGFIFCAA|2020-05-20|1444|6282|482|2020|3|5|20|2|2020|482|6282|Wednesday|2020Q2|N|N|N|2458971|2459091|2458624|2458899|N|N|N|N|N| +2458991|AAAAAAAAPGFIFCAA|2020-05-21|1444|6282|482|2020|4|5|21|2|2020|482|6282|Thursday|2020Q2|N|N|N|2458971|2459091|2458625|2458900|N|N|N|N|N| +2458992|AAAAAAAAAHFIFCAA|2020-05-22|1444|6282|482|2020|5|5|22|2|2020|482|6282|Friday|2020Q2|N|Y|N|2458971|2459091|2458626|2458901|N|N|N|N|N| +2458993|AAAAAAAABHFIFCAA|2020-05-23|1444|6282|482|2020|6|5|23|2|2020|482|6282|Saturday|2020Q2|N|Y|N|2458971|2459091|2458627|2458902|N|N|N|N|N| +2458994|AAAAAAAACHFIFCAA|2020-05-24|1444|6282|482|2020|0|5|24|2|2020|482|6282|Sunday|2020Q2|N|N|N|2458971|2459091|2458628|2458903|N|N|N|N|N| +2458995|AAAAAAAADHFIFCAA|2020-05-25|1444|6282|482|2020|1|5|25|2|2020|482|6282|Monday|2020Q2|N|N|N|2458971|2459091|2458629|2458904|N|N|N|N|N| +2458996|AAAAAAAAEHFIFCAA|2020-05-26|1444|6283|482|2020|2|5|26|2|2020|482|6283|Tuesday|2020Q2|N|N|N|2458971|2459091|2458630|2458905|N|N|N|N|N| +2458997|AAAAAAAAFHFIFCAA|2020-05-27|1444|6283|482|2020|3|5|27|2|2020|482|6283|Wednesday|2020Q2|N|N|N|2458971|2459091|2458631|2458906|N|N|N|N|N| +2458998|AAAAAAAAGHFIFCAA|2020-05-28|1444|6283|482|2020|4|5|28|2|2020|482|6283|Thursday|2020Q2|N|N|N|2458971|2459091|2458632|2458907|N|N|N|N|N| +2458999|AAAAAAAAHHFIFCAA|2020-05-29|1444|6283|482|2020|5|5|29|2|2020|482|6283|Friday|2020Q2|N|Y|N|2458971|2459091|2458633|2458908|N|N|N|N|N| +2459000|AAAAAAAAIHFIFCAA|2020-05-30|1444|6283|482|2020|6|5|30|2|2020|482|6283|Saturday|2020Q2|N|Y|N|2458971|2459091|2458634|2458909|N|N|N|N|N| +2459001|AAAAAAAAJHFIFCAA|2020-05-31|1444|6283|482|2020|0|5|31|2|2020|482|6283|Sunday|2020Q2|N|N|N|2458971|2459091|2458635|2458910|N|N|N|N|N| +2459002|AAAAAAAAKHFIFCAA|2020-06-01|1445|6283|483|2020|1|6|1|2|2020|483|6283|Monday|2020Q2|N|N|N|2459002|2459153|2458636|2458911|N|N|N|N|N| +2459003|AAAAAAAALHFIFCAA|2020-06-02|1445|6284|483|2020|2|6|2|2|2020|483|6284|Tuesday|2020Q2|N|N|N|2459002|2459153|2458637|2458912|N|N|N|N|N| +2459004|AAAAAAAAMHFIFCAA|2020-06-03|1445|6284|483|2020|3|6|3|2|2020|483|6284|Wednesday|2020Q2|N|N|N|2459002|2459153|2458638|2458913|N|N|N|N|N| +2459005|AAAAAAAANHFIFCAA|2020-06-04|1445|6284|483|2020|4|6|4|2|2020|483|6284|Thursday|2020Q2|N|N|N|2459002|2459153|2458639|2458914|N|N|N|N|N| +2459006|AAAAAAAAOHFIFCAA|2020-06-05|1445|6284|483|2020|5|6|5|2|2020|483|6284|Friday|2020Q2|N|Y|N|2459002|2459153|2458640|2458915|N|N|N|N|N| +2459007|AAAAAAAAPHFIFCAA|2020-06-06|1445|6284|483|2020|6|6|6|2|2020|483|6284|Saturday|2020Q2|N|Y|N|2459002|2459153|2458641|2458916|N|N|N|N|N| +2459008|AAAAAAAAAIFIFCAA|2020-06-07|1445|6284|483|2020|0|6|7|2|2020|483|6284|Sunday|2020Q2|N|N|N|2459002|2459153|2458642|2458917|N|N|N|N|N| +2459009|AAAAAAAABIFIFCAA|2020-06-08|1445|6284|483|2020|1|6|8|2|2020|483|6284|Monday|2020Q2|N|N|N|2459002|2459153|2458643|2458918|N|N|N|N|N| +2459010|AAAAAAAACIFIFCAA|2020-06-09|1445|6285|483|2020|2|6|9|2|2020|483|6285|Tuesday|2020Q2|N|N|N|2459002|2459153|2458644|2458919|N|N|N|N|N| +2459011|AAAAAAAADIFIFCAA|2020-06-10|1445|6285|483|2020|3|6|10|2|2020|483|6285|Wednesday|2020Q2|N|N|N|2459002|2459153|2458645|2458920|N|N|N|N|N| +2459012|AAAAAAAAEIFIFCAA|2020-06-11|1445|6285|483|2020|4|6|11|2|2020|483|6285|Thursday|2020Q2|N|N|N|2459002|2459153|2458646|2458921|N|N|N|N|N| +2459013|AAAAAAAAFIFIFCAA|2020-06-12|1445|6285|483|2020|5|6|12|2|2020|483|6285|Friday|2020Q2|N|Y|N|2459002|2459153|2458647|2458922|N|N|N|N|N| +2459014|AAAAAAAAGIFIFCAA|2020-06-13|1445|6285|483|2020|6|6|13|2|2020|483|6285|Saturday|2020Q2|N|Y|N|2459002|2459153|2458648|2458923|N|N|N|N|N| +2459015|AAAAAAAAHIFIFCAA|2020-06-14|1445|6285|483|2020|0|6|14|2|2020|483|6285|Sunday|2020Q2|N|N|N|2459002|2459153|2458649|2458924|N|N|N|N|N| +2459016|AAAAAAAAIIFIFCAA|2020-06-15|1445|6285|483|2020|1|6|15|2|2020|483|6285|Monday|2020Q2|N|N|N|2459002|2459153|2458650|2458925|N|N|N|N|N| +2459017|AAAAAAAAJIFIFCAA|2020-06-16|1445|6286|483|2020|2|6|16|2|2020|483|6286|Tuesday|2020Q2|N|N|N|2459002|2459153|2458651|2458926|N|N|N|N|N| +2459018|AAAAAAAAKIFIFCAA|2020-06-17|1445|6286|483|2020|3|6|17|2|2020|483|6286|Wednesday|2020Q2|N|N|N|2459002|2459153|2458652|2458927|N|N|N|N|N| +2459019|AAAAAAAALIFIFCAA|2020-06-18|1445|6286|483|2020|4|6|18|2|2020|483|6286|Thursday|2020Q2|N|N|N|2459002|2459153|2458653|2458928|N|N|N|N|N| +2459020|AAAAAAAAMIFIFCAA|2020-06-19|1445|6286|483|2020|5|6|19|2|2020|483|6286|Friday|2020Q2|N|Y|N|2459002|2459153|2458654|2458929|N|N|N|N|N| +2459021|AAAAAAAANIFIFCAA|2020-06-20|1445|6286|483|2020|6|6|20|2|2020|483|6286|Saturday|2020Q2|N|Y|N|2459002|2459153|2458655|2458930|N|N|N|N|N| +2459022|AAAAAAAAOIFIFCAA|2020-06-21|1445|6286|483|2020|0|6|21|2|2020|483|6286|Sunday|2020Q2|N|N|N|2459002|2459153|2458656|2458931|N|N|N|N|N| +2459023|AAAAAAAAPIFIFCAA|2020-06-22|1445|6286|483|2020|1|6|22|2|2020|483|6286|Monday|2020Q2|N|N|N|2459002|2459153|2458657|2458932|N|N|N|N|N| +2459024|AAAAAAAAAJFIFCAA|2020-06-23|1445|6287|483|2020|2|6|23|2|2020|483|6287|Tuesday|2020Q2|N|N|N|2459002|2459153|2458658|2458933|N|N|N|N|N| +2459025|AAAAAAAABJFIFCAA|2020-06-24|1445|6287|483|2020|3|6|24|2|2020|483|6287|Wednesday|2020Q2|N|N|N|2459002|2459153|2458659|2458934|N|N|N|N|N| +2459026|AAAAAAAACJFIFCAA|2020-06-25|1445|6287|483|2020|4|6|25|2|2020|483|6287|Thursday|2020Q2|N|N|N|2459002|2459153|2458660|2458935|N|N|N|N|N| +2459027|AAAAAAAADJFIFCAA|2020-06-26|1445|6287|483|2020|5|6|26|2|2020|483|6287|Friday|2020Q2|N|Y|N|2459002|2459153|2458661|2458936|N|N|N|N|N| +2459028|AAAAAAAAEJFIFCAA|2020-06-27|1445|6287|483|2020|6|6|27|2|2020|483|6287|Saturday|2020Q2|N|Y|N|2459002|2459153|2458662|2458937|N|N|N|N|N| +2459029|AAAAAAAAFJFIFCAA|2020-06-28|1445|6287|483|2020|0|6|28|2|2020|483|6287|Sunday|2020Q2|N|N|N|2459002|2459153|2458663|2458938|N|N|N|N|N| +2459030|AAAAAAAAGJFIFCAA|2020-06-29|1445|6287|483|2020|1|6|29|2|2020|483|6287|Monday|2020Q2|N|N|N|2459002|2459153|2458664|2458939|N|N|N|N|N| +2459031|AAAAAAAAHJFIFCAA|2020-06-30|1445|6288|483|2020|2|6|30|2|2020|483|6288|Tuesday|2020Q2|N|N|N|2459002|2459153|2458665|2458940|N|N|N|N|N| +2459032|AAAAAAAAIJFIFCAA|2020-07-01|1446|6288|483|2020|3|7|1|3|2020|483|6288|Wednesday|2020Q3|N|N|N|2459032|2459213|2458666|2458941|N|N|N|N|N| +2459033|AAAAAAAAJJFIFCAA|2020-07-02|1446|6288|483|2020|4|7|2|3|2020|483|6288|Thursday|2020Q3|N|N|N|2459032|2459213|2458667|2458942|N|N|N|N|N| +2459034|AAAAAAAAKJFIFCAA|2020-07-03|1446|6288|483|2020|5|7|3|3|2020|483|6288|Friday|2020Q3|N|Y|N|2459032|2459213|2458668|2458943|N|N|N|N|N| +2459035|AAAAAAAALJFIFCAA|2020-07-04|1446|6288|483|2020|6|7|4|3|2020|483|6288|Saturday|2020Q3|Y|Y|N|2459032|2459213|2458669|2458944|N|N|N|N|N| +2459036|AAAAAAAAMJFIFCAA|2020-07-05|1446|6288|483|2020|0|7|5|3|2020|483|6288|Sunday|2020Q3|N|N|Y|2459032|2459213|2458670|2458945|N|N|N|N|N| +2459037|AAAAAAAANJFIFCAA|2020-07-06|1446|6288|483|2020|1|7|6|3|2020|483|6288|Monday|2020Q3|N|N|N|2459032|2459213|2458671|2458946|N|N|N|N|N| +2459038|AAAAAAAAOJFIFCAA|2020-07-07|1446|6289|483|2020|2|7|7|3|2020|483|6289|Tuesday|2020Q3|N|N|N|2459032|2459213|2458672|2458947|N|N|N|N|N| +2459039|AAAAAAAAPJFIFCAA|2020-07-08|1446|6289|483|2020|3|7|8|3|2020|483|6289|Wednesday|2020Q3|N|N|N|2459032|2459213|2458673|2458948|N|N|N|N|N| +2459040|AAAAAAAAAKFIFCAA|2020-07-09|1446|6289|483|2020|4|7|9|3|2020|483|6289|Thursday|2020Q3|N|N|N|2459032|2459213|2458674|2458949|N|N|N|N|N| +2459041|AAAAAAAABKFIFCAA|2020-07-10|1446|6289|483|2020|5|7|10|3|2020|483|6289|Friday|2020Q3|N|Y|N|2459032|2459213|2458675|2458950|N|N|N|N|N| +2459042|AAAAAAAACKFIFCAA|2020-07-11|1446|6289|483|2020|6|7|11|3|2020|483|6289|Saturday|2020Q3|N|Y|N|2459032|2459213|2458676|2458951|N|N|N|N|N| +2459043|AAAAAAAADKFIFCAA|2020-07-12|1446|6289|483|2020|0|7|12|3|2020|483|6289|Sunday|2020Q3|N|N|N|2459032|2459213|2458677|2458952|N|N|N|N|N| +2459044|AAAAAAAAEKFIFCAA|2020-07-13|1446|6289|483|2020|1|7|13|3|2020|483|6289|Monday|2020Q3|N|N|N|2459032|2459213|2458678|2458953|N|N|N|N|N| +2459045|AAAAAAAAFKFIFCAA|2020-07-14|1446|6290|483|2020|2|7|14|3|2020|483|6290|Tuesday|2020Q3|N|N|N|2459032|2459213|2458679|2458954|N|N|N|N|N| +2459046|AAAAAAAAGKFIFCAA|2020-07-15|1446|6290|483|2020|3|7|15|3|2020|483|6290|Wednesday|2020Q3|N|N|N|2459032|2459213|2458680|2458955|N|N|N|N|N| +2459047|AAAAAAAAHKFIFCAA|2020-07-16|1446|6290|483|2020|4|7|16|3|2020|483|6290|Thursday|2020Q3|N|N|N|2459032|2459213|2458681|2458956|N|N|N|N|N| +2459048|AAAAAAAAIKFIFCAA|2020-07-17|1446|6290|483|2020|5|7|17|3|2020|483|6290|Friday|2020Q3|N|Y|N|2459032|2459213|2458682|2458957|N|N|N|N|N| +2459049|AAAAAAAAJKFIFCAA|2020-07-18|1446|6290|483|2020|6|7|18|3|2020|483|6290|Saturday|2020Q3|N|Y|N|2459032|2459213|2458683|2458958|N|N|N|N|N| +2459050|AAAAAAAAKKFIFCAA|2020-07-19|1446|6290|483|2020|0|7|19|3|2020|483|6290|Sunday|2020Q3|N|N|N|2459032|2459213|2458684|2458959|N|N|N|N|N| +2459051|AAAAAAAALKFIFCAA|2020-07-20|1446|6290|483|2020|1|7|20|3|2020|483|6290|Monday|2020Q3|N|N|N|2459032|2459213|2458685|2458960|N|N|N|N|N| +2459052|AAAAAAAAMKFIFCAA|2020-07-21|1446|6291|483|2020|2|7|21|3|2020|483|6291|Tuesday|2020Q3|N|N|N|2459032|2459213|2458686|2458961|N|N|N|N|N| +2459053|AAAAAAAANKFIFCAA|2020-07-22|1446|6291|483|2020|3|7|22|3|2020|483|6291|Wednesday|2020Q3|N|N|N|2459032|2459213|2458687|2458962|N|N|N|N|N| +2459054|AAAAAAAAOKFIFCAA|2020-07-23|1446|6291|483|2020|4|7|23|3|2020|483|6291|Thursday|2020Q3|N|N|N|2459032|2459213|2458688|2458963|N|N|N|N|N| +2459055|AAAAAAAAPKFIFCAA|2020-07-24|1446|6291|483|2020|5|7|24|3|2020|483|6291|Friday|2020Q3|N|Y|N|2459032|2459213|2458689|2458964|N|N|N|N|N| +2459056|AAAAAAAAALFIFCAA|2020-07-25|1446|6291|483|2020|6|7|25|3|2020|483|6291|Saturday|2020Q3|N|Y|N|2459032|2459213|2458690|2458965|N|N|N|N|N| +2459057|AAAAAAAABLFIFCAA|2020-07-26|1446|6291|483|2020|0|7|26|3|2020|483|6291|Sunday|2020Q3|N|N|N|2459032|2459213|2458691|2458966|N|N|N|N|N| +2459058|AAAAAAAACLFIFCAA|2020-07-27|1446|6291|483|2020|1|7|27|3|2020|483|6291|Monday|2020Q3|N|N|N|2459032|2459213|2458692|2458967|N|N|N|N|N| +2459059|AAAAAAAADLFIFCAA|2020-07-28|1446|6292|483|2020|2|7|28|3|2020|483|6292|Tuesday|2020Q3|N|N|N|2459032|2459213|2458693|2458968|N|N|N|N|N| +2459060|AAAAAAAAELFIFCAA|2020-07-29|1446|6292|483|2020|3|7|29|3|2020|483|6292|Wednesday|2020Q3|N|N|N|2459032|2459213|2458694|2458969|N|N|N|N|N| +2459061|AAAAAAAAFLFIFCAA|2020-07-30|1446|6292|483|2020|4|7|30|3|2020|483|6292|Thursday|2020Q3|N|N|N|2459032|2459213|2458695|2458970|N|N|N|N|N| +2459062|AAAAAAAAGLFIFCAA|2020-07-31|1446|6292|483|2020|5|7|31|3|2020|483|6292|Friday|2020Q3|N|Y|N|2459032|2459213|2458696|2458971|N|N|N|N|N| +2459063|AAAAAAAAHLFIFCAA|2020-08-01|1447|6292|483|2020|6|8|1|3|2020|483|6292|Saturday|2020Q3|N|Y|N|2459063|2459275|2458697|2458972|N|N|N|N|N| +2459064|AAAAAAAAILFIFCAA|2020-08-02|1447|6292|483|2020|0|8|2|3|2020|483|6292|Sunday|2020Q3|N|N|N|2459063|2459275|2458698|2458973|N|N|N|N|N| +2459065|AAAAAAAAJLFIFCAA|2020-08-03|1447|6292|483|2020|1|8|3|3|2020|483|6292|Monday|2020Q3|N|N|N|2459063|2459275|2458699|2458974|N|N|N|N|N| +2459066|AAAAAAAAKLFIFCAA|2020-08-04|1447|6293|483|2020|2|8|4|3|2020|483|6293|Tuesday|2020Q3|N|N|N|2459063|2459275|2458700|2458975|N|N|N|N|N| +2459067|AAAAAAAALLFIFCAA|2020-08-05|1447|6293|483|2020|3|8|5|3|2020|483|6293|Wednesday|2020Q3|N|N|N|2459063|2459275|2458701|2458976|N|N|N|N|N| +2459068|AAAAAAAAMLFIFCAA|2020-08-06|1447|6293|483|2020|4|8|6|3|2020|483|6293|Thursday|2020Q3|N|N|N|2459063|2459275|2458702|2458977|N|N|N|N|N| +2459069|AAAAAAAANLFIFCAA|2020-08-07|1447|6293|483|2020|5|8|7|3|2020|483|6293|Friday|2020Q3|N|Y|N|2459063|2459275|2458703|2458978|N|N|N|N|N| +2459070|AAAAAAAAOLFIFCAA|2020-08-08|1447|6293|483|2020|6|8|8|3|2020|483|6293|Saturday|2020Q3|N|Y|N|2459063|2459275|2458704|2458979|N|N|N|N|N| +2459071|AAAAAAAAPLFIFCAA|2020-08-09|1447|6293|483|2020|0|8|9|3|2020|483|6293|Sunday|2020Q3|N|N|N|2459063|2459275|2458705|2458980|N|N|N|N|N| +2459072|AAAAAAAAAMFIFCAA|2020-08-10|1447|6293|483|2020|1|8|10|3|2020|483|6293|Monday|2020Q3|N|N|N|2459063|2459275|2458706|2458981|N|N|N|N|N| +2459073|AAAAAAAABMFIFCAA|2020-08-11|1447|6294|483|2020|2|8|11|3|2020|483|6294|Tuesday|2020Q3|N|N|N|2459063|2459275|2458707|2458982|N|N|N|N|N| +2459074|AAAAAAAACMFIFCAA|2020-08-12|1447|6294|483|2020|3|8|12|3|2020|483|6294|Wednesday|2020Q3|N|N|N|2459063|2459275|2458708|2458983|N|N|N|N|N| +2459075|AAAAAAAADMFIFCAA|2020-08-13|1447|6294|483|2020|4|8|13|3|2020|483|6294|Thursday|2020Q3|N|N|N|2459063|2459275|2458709|2458984|N|N|N|N|N| +2459076|AAAAAAAAEMFIFCAA|2020-08-14|1447|6294|483|2020|5|8|14|3|2020|483|6294|Friday|2020Q3|N|Y|N|2459063|2459275|2458710|2458985|N|N|N|N|N| +2459077|AAAAAAAAFMFIFCAA|2020-08-15|1447|6294|483|2020|6|8|15|3|2020|483|6294|Saturday|2020Q3|N|Y|N|2459063|2459275|2458711|2458986|N|N|N|N|N| +2459078|AAAAAAAAGMFIFCAA|2020-08-16|1447|6294|483|2020|0|8|16|3|2020|483|6294|Sunday|2020Q3|N|N|N|2459063|2459275|2458712|2458987|N|N|N|N|N| +2459079|AAAAAAAAHMFIFCAA|2020-08-17|1447|6294|483|2020|1|8|17|3|2020|483|6294|Monday|2020Q3|N|N|N|2459063|2459275|2458713|2458988|N|N|N|N|N| +2459080|AAAAAAAAIMFIFCAA|2020-08-18|1447|6295|483|2020|2|8|18|3|2020|483|6295|Tuesday|2020Q3|N|N|N|2459063|2459275|2458714|2458989|N|N|N|N|N| +2459081|AAAAAAAAJMFIFCAA|2020-08-19|1447|6295|483|2020|3|8|19|3|2020|483|6295|Wednesday|2020Q3|N|N|N|2459063|2459275|2458715|2458990|N|N|N|N|N| +2459082|AAAAAAAAKMFIFCAA|2020-08-20|1447|6295|483|2020|4|8|20|3|2020|483|6295|Thursday|2020Q3|N|N|N|2459063|2459275|2458716|2458991|N|N|N|N|N| +2459083|AAAAAAAALMFIFCAA|2020-08-21|1447|6295|483|2020|5|8|21|3|2020|483|6295|Friday|2020Q3|N|Y|N|2459063|2459275|2458717|2458992|N|N|N|N|N| +2459084|AAAAAAAAMMFIFCAA|2020-08-22|1447|6295|483|2020|6|8|22|3|2020|483|6295|Saturday|2020Q3|N|Y|N|2459063|2459275|2458718|2458993|N|N|N|N|N| +2459085|AAAAAAAANMFIFCAA|2020-08-23|1447|6295|483|2020|0|8|23|3|2020|483|6295|Sunday|2020Q3|N|N|N|2459063|2459275|2458719|2458994|N|N|N|N|N| +2459086|AAAAAAAAOMFIFCAA|2020-08-24|1447|6295|483|2020|1|8|24|3|2020|483|6295|Monday|2020Q3|N|N|N|2459063|2459275|2458720|2458995|N|N|N|N|N| +2459087|AAAAAAAAPMFIFCAA|2020-08-25|1447|6296|483|2020|2|8|25|3|2020|483|6296|Tuesday|2020Q3|N|N|N|2459063|2459275|2458721|2458996|N|N|N|N|N| +2459088|AAAAAAAAANFIFCAA|2020-08-26|1447|6296|483|2020|3|8|26|3|2020|483|6296|Wednesday|2020Q3|N|N|N|2459063|2459275|2458722|2458997|N|N|N|N|N| +2459089|AAAAAAAABNFIFCAA|2020-08-27|1447|6296|483|2020|4|8|27|3|2020|483|6296|Thursday|2020Q3|N|N|N|2459063|2459275|2458723|2458998|N|N|N|N|N| +2459090|AAAAAAAACNFIFCAA|2020-08-28|1447|6296|483|2020|5|8|28|3|2020|483|6296|Friday|2020Q3|N|Y|N|2459063|2459275|2458724|2458999|N|N|N|N|N| +2459091|AAAAAAAADNFIFCAA|2020-08-29|1447|6296|483|2020|6|8|29|3|2020|483|6296|Saturday|2020Q3|N|Y|N|2459063|2459275|2458725|2459000|N|N|N|N|N| +2459092|AAAAAAAAENFIFCAA|2020-08-30|1447|6296|483|2020|0|8|30|3|2020|483|6296|Sunday|2020Q3|N|N|N|2459063|2459275|2458726|2459001|N|N|N|N|N| +2459093|AAAAAAAAFNFIFCAA|2020-08-31|1447|6296|483|2020|1|8|31|3|2020|483|6296|Monday|2020Q3|N|N|N|2459063|2459275|2458727|2459002|N|N|N|N|N| +2459094|AAAAAAAAGNFIFCAA|2020-09-01|1448|6297|484|2020|2|9|1|3|2020|484|6297|Tuesday|2020Q3|N|N|N|2459094|2459337|2458728|2459003|N|N|N|N|N| +2459095|AAAAAAAAHNFIFCAA|2020-09-02|1448|6297|484|2020|3|9|2|3|2020|484|6297|Wednesday|2020Q3|N|N|N|2459094|2459337|2458729|2459004|N|N|N|N|N| +2459096|AAAAAAAAINFIFCAA|2020-09-03|1448|6297|484|2020|4|9|3|3|2020|484|6297|Thursday|2020Q3|N|N|N|2459094|2459337|2458730|2459005|N|N|N|N|N| +2459097|AAAAAAAAJNFIFCAA|2020-09-04|1448|6297|484|2020|5|9|4|3|2020|484|6297|Friday|2020Q3|N|Y|N|2459094|2459337|2458731|2459006|N|N|N|N|N| +2459098|AAAAAAAAKNFIFCAA|2020-09-05|1448|6297|484|2020|6|9|5|3|2020|484|6297|Saturday|2020Q3|N|Y|N|2459094|2459337|2458732|2459007|N|N|N|N|N| +2459099|AAAAAAAALNFIFCAA|2020-09-06|1448|6297|484|2020|0|9|6|3|2020|484|6297|Sunday|2020Q3|N|N|N|2459094|2459337|2458733|2459008|N|N|N|N|N| +2459100|AAAAAAAAMNFIFCAA|2020-09-07|1448|6297|484|2020|1|9|7|3|2020|484|6297|Monday|2020Q3|N|N|N|2459094|2459337|2458734|2459009|N|N|N|N|N| +2459101|AAAAAAAANNFIFCAA|2020-09-08|1448|6298|484|2020|2|9|8|3|2020|484|6298|Tuesday|2020Q3|N|N|N|2459094|2459337|2458735|2459010|N|N|N|N|N| +2459102|AAAAAAAAONFIFCAA|2020-09-09|1448|6298|484|2020|3|9|9|3|2020|484|6298|Wednesday|2020Q3|N|N|N|2459094|2459337|2458736|2459011|N|N|N|N|N| +2459103|AAAAAAAAPNFIFCAA|2020-09-10|1448|6298|484|2020|4|9|10|3|2020|484|6298|Thursday|2020Q3|N|N|N|2459094|2459337|2458737|2459012|N|N|N|N|N| +2459104|AAAAAAAAAOFIFCAA|2020-09-11|1448|6298|484|2020|5|9|11|3|2020|484|6298|Friday|2020Q3|N|Y|N|2459094|2459337|2458738|2459013|N|N|N|N|N| +2459105|AAAAAAAABOFIFCAA|2020-09-12|1448|6298|484|2020|6|9|12|3|2020|484|6298|Saturday|2020Q3|N|Y|N|2459094|2459337|2458739|2459014|N|N|N|N|N| +2459106|AAAAAAAACOFIFCAA|2020-09-13|1448|6298|484|2020|0|9|13|3|2020|484|6298|Sunday|2020Q3|N|N|N|2459094|2459337|2458740|2459015|N|N|N|N|N| +2459107|AAAAAAAADOFIFCAA|2020-09-14|1448|6298|484|2020|1|9|14|3|2020|484|6298|Monday|2020Q3|N|N|N|2459094|2459337|2458741|2459016|N|N|N|N|N| +2459108|AAAAAAAAEOFIFCAA|2020-09-15|1448|6299|484|2020|2|9|15|3|2020|484|6299|Tuesday|2020Q3|N|N|N|2459094|2459337|2458742|2459017|N|N|N|N|N| +2459109|AAAAAAAAFOFIFCAA|2020-09-16|1448|6299|484|2020|3|9|16|3|2020|484|6299|Wednesday|2020Q3|N|N|N|2459094|2459337|2458743|2459018|N|N|N|N|N| +2459110|AAAAAAAAGOFIFCAA|2020-09-17|1448|6299|484|2020|4|9|17|3|2020|484|6299|Thursday|2020Q3|N|N|N|2459094|2459337|2458744|2459019|N|N|N|N|N| +2459111|AAAAAAAAHOFIFCAA|2020-09-18|1448|6299|484|2020|5|9|18|3|2020|484|6299|Friday|2020Q3|N|Y|N|2459094|2459337|2458745|2459020|N|N|N|N|N| +2459112|AAAAAAAAIOFIFCAA|2020-09-19|1448|6299|484|2020|6|9|19|3|2020|484|6299|Saturday|2020Q3|N|Y|N|2459094|2459337|2458746|2459021|N|N|N|N|N| +2459113|AAAAAAAAJOFIFCAA|2020-09-20|1448|6299|484|2020|0|9|20|3|2020|484|6299|Sunday|2020Q3|N|N|N|2459094|2459337|2458747|2459022|N|N|N|N|N| +2459114|AAAAAAAAKOFIFCAA|2020-09-21|1448|6299|484|2020|1|9|21|3|2020|484|6299|Monday|2020Q3|N|N|N|2459094|2459337|2458748|2459023|N|N|N|N|N| +2459115|AAAAAAAALOFIFCAA|2020-09-22|1448|6300|484|2020|2|9|22|3|2020|484|6300|Tuesday|2020Q3|N|N|N|2459094|2459337|2458749|2459024|N|N|N|N|N| +2459116|AAAAAAAAMOFIFCAA|2020-09-23|1448|6300|484|2020|3|9|23|3|2020|484|6300|Wednesday|2020Q3|N|N|N|2459094|2459337|2458750|2459025|N|N|N|N|N| +2459117|AAAAAAAANOFIFCAA|2020-09-24|1448|6300|484|2020|4|9|24|3|2020|484|6300|Thursday|2020Q3|N|N|N|2459094|2459337|2458751|2459026|N|N|N|N|N| +2459118|AAAAAAAAOOFIFCAA|2020-09-25|1448|6300|484|2020|5|9|25|3|2020|484|6300|Friday|2020Q3|N|Y|N|2459094|2459337|2458752|2459027|N|N|N|N|N| +2459119|AAAAAAAAPOFIFCAA|2020-09-26|1448|6300|484|2020|6|9|26|3|2020|484|6300|Saturday|2020Q3|N|Y|N|2459094|2459337|2458753|2459028|N|N|N|N|N| +2459120|AAAAAAAAAPFIFCAA|2020-09-27|1448|6300|484|2020|0|9|27|3|2020|484|6300|Sunday|2020Q3|N|N|N|2459094|2459337|2458754|2459029|N|N|N|N|N| +2459121|AAAAAAAABPFIFCAA|2020-09-28|1448|6300|484|2020|1|9|28|3|2020|484|6300|Monday|2020Q3|N|N|N|2459094|2459337|2458755|2459030|N|N|N|N|N| +2459122|AAAAAAAACPFIFCAA|2020-09-29|1448|6301|484|2020|2|9|29|3|2020|484|6301|Tuesday|2020Q3|N|N|N|2459094|2459337|2458756|2459031|N|N|N|N|N| +2459123|AAAAAAAADPFIFCAA|2020-09-30|1448|6301|484|2020|3|9|30|3|2020|484|6301|Wednesday|2020Q3|N|N|N|2459094|2459337|2458757|2459032|N|N|N|N|N| +2459124|AAAAAAAAEPFIFCAA|2020-10-01|1449|6301|484|2020|4|10|1|4|2020|484|6301|Thursday|2020Q4|N|N|N|2459124|2459397|2458758|2459032|N|N|N|N|N| +2459125|AAAAAAAAFPFIFCAA|2020-10-02|1449|6301|484|2020|5|10|2|4|2020|484|6301|Friday|2020Q4|N|Y|N|2459124|2459397|2458759|2459033|N|N|N|N|N| +2459126|AAAAAAAAGPFIFCAA|2020-10-03|1449|6301|484|2020|6|10|3|4|2020|484|6301|Saturday|2020Q4|N|Y|N|2459124|2459397|2458760|2459034|N|N|N|N|N| +2459127|AAAAAAAAHPFIFCAA|2020-10-04|1449|6301|484|2020|0|10|4|4|2020|484|6301|Sunday|2020Q4|N|N|N|2459124|2459397|2458761|2459035|N|N|N|N|N| +2459128|AAAAAAAAIPFIFCAA|2020-10-05|1449|6301|484|2020|1|10|5|4|2020|484|6301|Monday|2020Q4|N|N|N|2459124|2459397|2458762|2459036|N|N|N|N|N| +2459129|AAAAAAAAJPFIFCAA|2020-10-06|1449|6302|484|2020|2|10|6|4|2020|484|6302|Tuesday|2020Q4|N|N|N|2459124|2459397|2458763|2459037|N|N|N|N|N| +2459130|AAAAAAAAKPFIFCAA|2020-10-07|1449|6302|484|2020|3|10|7|4|2020|484|6302|Wednesday|2020Q4|N|N|N|2459124|2459397|2458764|2459038|N|N|N|N|N| +2459131|AAAAAAAALPFIFCAA|2020-10-08|1449|6302|484|2020|4|10|8|4|2020|484|6302|Thursday|2020Q4|N|N|N|2459124|2459397|2458765|2459039|N|N|N|N|N| +2459132|AAAAAAAAMPFIFCAA|2020-10-09|1449|6302|484|2020|5|10|9|4|2020|484|6302|Friday|2020Q4|N|Y|N|2459124|2459397|2458766|2459040|N|N|N|N|N| +2459133|AAAAAAAANPFIFCAA|2020-10-10|1449|6302|484|2020|6|10|10|4|2020|484|6302|Saturday|2020Q4|N|Y|N|2459124|2459397|2458767|2459041|N|N|N|N|N| +2459134|AAAAAAAAOPFIFCAA|2020-10-11|1449|6302|484|2020|0|10|11|4|2020|484|6302|Sunday|2020Q4|N|N|N|2459124|2459397|2458768|2459042|N|N|N|N|N| +2459135|AAAAAAAAPPFIFCAA|2020-10-12|1449|6302|484|2020|1|10|12|4|2020|484|6302|Monday|2020Q4|N|N|N|2459124|2459397|2458769|2459043|N|N|N|N|N| +2459136|AAAAAAAAAAGIFCAA|2020-10-13|1449|6303|484|2020|2|10|13|4|2020|484|6303|Tuesday|2020Q4|N|N|N|2459124|2459397|2458770|2459044|N|N|N|N|N| +2459137|AAAAAAAABAGIFCAA|2020-10-14|1449|6303|484|2020|3|10|14|4|2020|484|6303|Wednesday|2020Q4|N|N|N|2459124|2459397|2458771|2459045|N|N|N|N|N| +2459138|AAAAAAAACAGIFCAA|2020-10-15|1449|6303|484|2020|4|10|15|4|2020|484|6303|Thursday|2020Q4|N|N|N|2459124|2459397|2458772|2459046|N|N|N|N|N| +2459139|AAAAAAAADAGIFCAA|2020-10-16|1449|6303|484|2020|5|10|16|4|2020|484|6303|Friday|2020Q4|N|Y|N|2459124|2459397|2458773|2459047|N|N|N|N|N| +2459140|AAAAAAAAEAGIFCAA|2020-10-17|1449|6303|484|2020|6|10|17|4|2020|484|6303|Saturday|2020Q4|N|Y|N|2459124|2459397|2458774|2459048|N|N|N|N|N| +2459141|AAAAAAAAFAGIFCAA|2020-10-18|1449|6303|484|2020|0|10|18|4|2020|484|6303|Sunday|2020Q4|N|N|N|2459124|2459397|2458775|2459049|N|N|N|N|N| +2459142|AAAAAAAAGAGIFCAA|2020-10-19|1449|6303|484|2020|1|10|19|4|2020|484|6303|Monday|2020Q4|N|N|N|2459124|2459397|2458776|2459050|N|N|N|N|N| +2459143|AAAAAAAAHAGIFCAA|2020-10-20|1449|6304|484|2020|2|10|20|4|2020|484|6304|Tuesday|2020Q4|N|N|N|2459124|2459397|2458777|2459051|N|N|N|N|N| +2459144|AAAAAAAAIAGIFCAA|2020-10-21|1449|6304|484|2020|3|10|21|4|2020|484|6304|Wednesday|2020Q4|N|N|N|2459124|2459397|2458778|2459052|N|N|N|N|N| +2459145|AAAAAAAAJAGIFCAA|2020-10-22|1449|6304|484|2020|4|10|22|4|2020|484|6304|Thursday|2020Q4|N|N|N|2459124|2459397|2458779|2459053|N|N|N|N|N| +2459146|AAAAAAAAKAGIFCAA|2020-10-23|1449|6304|484|2020|5|10|23|4|2020|484|6304|Friday|2020Q4|N|Y|N|2459124|2459397|2458780|2459054|N|N|N|N|N| +2459147|AAAAAAAALAGIFCAA|2020-10-24|1449|6304|484|2020|6|10|24|4|2020|484|6304|Saturday|2020Q4|N|Y|N|2459124|2459397|2458781|2459055|N|N|N|N|N| +2459148|AAAAAAAAMAGIFCAA|2020-10-25|1449|6304|484|2020|0|10|25|4|2020|484|6304|Sunday|2020Q4|N|N|N|2459124|2459397|2458782|2459056|N|N|N|N|N| +2459149|AAAAAAAANAGIFCAA|2020-10-26|1449|6304|484|2020|1|10|26|4|2020|484|6304|Monday|2020Q4|N|N|N|2459124|2459397|2458783|2459057|N|N|N|N|N| +2459150|AAAAAAAAOAGIFCAA|2020-10-27|1449|6305|484|2020|2|10|27|4|2020|484|6305|Tuesday|2020Q4|N|N|N|2459124|2459397|2458784|2459058|N|N|N|N|N| +2459151|AAAAAAAAPAGIFCAA|2020-10-28|1449|6305|484|2020|3|10|28|4|2020|484|6305|Wednesday|2020Q4|N|N|N|2459124|2459397|2458785|2459059|N|N|N|N|N| +2459152|AAAAAAAAABGIFCAA|2020-10-29|1449|6305|484|2020|4|10|29|4|2020|484|6305|Thursday|2020Q4|N|N|N|2459124|2459397|2458786|2459060|N|N|N|N|N| +2459153|AAAAAAAABBGIFCAA|2020-10-30|1449|6305|484|2020|5|10|30|4|2020|484|6305|Friday|2020Q4|N|Y|N|2459124|2459397|2458787|2459061|N|N|N|N|N| +2459154|AAAAAAAACBGIFCAA|2020-10-31|1449|6305|484|2020|6|10|31|4|2020|484|6305|Saturday|2020Q4|N|Y|N|2459124|2459397|2458788|2459062|N|N|N|N|N| +2459155|AAAAAAAADBGIFCAA|2020-11-01|1450|6305|484|2020|0|11|1|4|2020|484|6305|Sunday|2020Q4|N|N|N|2459155|2459459|2458789|2459063|N|N|N|N|N| +2459156|AAAAAAAAEBGIFCAA|2020-11-02|1450|6305|484|2020|1|11|2|4|2020|484|6305|Monday|2020Q4|N|N|N|2459155|2459459|2458790|2459064|N|N|N|N|N| +2459157|AAAAAAAAFBGIFCAA|2020-11-03|1450|6306|484|2020|2|11|3|4|2020|484|6306|Tuesday|2020Q4|N|N|N|2459155|2459459|2458791|2459065|N|N|N|N|N| +2459158|AAAAAAAAGBGIFCAA|2020-11-04|1450|6306|484|2020|3|11|4|4|2020|484|6306|Wednesday|2020Q4|N|N|N|2459155|2459459|2458792|2459066|N|N|N|N|N| +2459159|AAAAAAAAHBGIFCAA|2020-11-05|1450|6306|484|2020|4|11|5|4|2020|484|6306|Thursday|2020Q4|N|N|N|2459155|2459459|2458793|2459067|N|N|N|N|N| +2459160|AAAAAAAAIBGIFCAA|2020-11-06|1450|6306|484|2020|5|11|6|4|2020|484|6306|Friday|2020Q4|N|Y|N|2459155|2459459|2458794|2459068|N|N|N|N|N| +2459161|AAAAAAAAJBGIFCAA|2020-11-07|1450|6306|484|2020|6|11|7|4|2020|484|6306|Saturday|2020Q4|N|Y|N|2459155|2459459|2458795|2459069|N|N|N|N|N| +2459162|AAAAAAAAKBGIFCAA|2020-11-08|1450|6306|484|2020|0|11|8|4|2020|484|6306|Sunday|2020Q4|N|N|N|2459155|2459459|2458796|2459070|N|N|N|N|N| +2459163|AAAAAAAALBGIFCAA|2020-11-09|1450|6306|484|2020|1|11|9|4|2020|484|6306|Monday|2020Q4|N|N|N|2459155|2459459|2458797|2459071|N|N|N|N|N| +2459164|AAAAAAAAMBGIFCAA|2020-11-10|1450|6307|484|2020|2|11|10|4|2020|484|6307|Tuesday|2020Q4|N|N|N|2459155|2459459|2458798|2459072|N|N|N|N|N| +2459165|AAAAAAAANBGIFCAA|2020-11-11|1450|6307|484|2020|3|11|11|4|2020|484|6307|Wednesday|2020Q4|N|N|N|2459155|2459459|2458799|2459073|N|N|N|N|N| +2459166|AAAAAAAAOBGIFCAA|2020-11-12|1450|6307|484|2020|4|11|12|4|2020|484|6307|Thursday|2020Q4|N|N|N|2459155|2459459|2458800|2459074|N|N|N|N|N| +2459167|AAAAAAAAPBGIFCAA|2020-11-13|1450|6307|484|2020|5|11|13|4|2020|484|6307|Friday|2020Q4|N|Y|N|2459155|2459459|2458801|2459075|N|N|N|N|N| +2459168|AAAAAAAAACGIFCAA|2020-11-14|1450|6307|484|2020|6|11|14|4|2020|484|6307|Saturday|2020Q4|N|Y|N|2459155|2459459|2458802|2459076|N|N|N|N|N| +2459169|AAAAAAAABCGIFCAA|2020-11-15|1450|6307|484|2020|0|11|15|4|2020|484|6307|Sunday|2020Q4|N|N|N|2459155|2459459|2458803|2459077|N|N|N|N|N| +2459170|AAAAAAAACCGIFCAA|2020-11-16|1450|6307|484|2020|1|11|16|4|2020|484|6307|Monday|2020Q4|N|N|N|2459155|2459459|2458804|2459078|N|N|N|N|N| +2459171|AAAAAAAADCGIFCAA|2020-11-17|1450|6308|484|2020|2|11|17|4|2020|484|6308|Tuesday|2020Q4|N|N|N|2459155|2459459|2458805|2459079|N|N|N|N|N| +2459172|AAAAAAAAECGIFCAA|2020-11-18|1450|6308|484|2020|3|11|18|4|2020|484|6308|Wednesday|2020Q4|N|N|N|2459155|2459459|2458806|2459080|N|N|N|N|N| +2459173|AAAAAAAAFCGIFCAA|2020-11-19|1450|6308|484|2020|4|11|19|4|2020|484|6308|Thursday|2020Q4|N|N|N|2459155|2459459|2458807|2459081|N|N|N|N|N| +2459174|AAAAAAAAGCGIFCAA|2020-11-20|1450|6308|484|2020|5|11|20|4|2020|484|6308|Friday|2020Q4|N|Y|N|2459155|2459459|2458808|2459082|N|N|N|N|N| +2459175|AAAAAAAAHCGIFCAA|2020-11-21|1450|6308|484|2020|6|11|21|4|2020|484|6308|Saturday|2020Q4|N|Y|N|2459155|2459459|2458809|2459083|N|N|N|N|N| +2459176|AAAAAAAAICGIFCAA|2020-11-22|1450|6308|484|2020|0|11|22|4|2020|484|6308|Sunday|2020Q4|N|N|N|2459155|2459459|2458810|2459084|N|N|N|N|N| +2459177|AAAAAAAAJCGIFCAA|2020-11-23|1450|6308|484|2020|1|11|23|4|2020|484|6308|Monday|2020Q4|N|N|N|2459155|2459459|2458811|2459085|N|N|N|N|N| +2459178|AAAAAAAAKCGIFCAA|2020-11-24|1450|6309|484|2020|2|11|24|4|2020|484|6309|Tuesday|2020Q4|N|N|N|2459155|2459459|2458812|2459086|N|N|N|N|N| +2459179|AAAAAAAALCGIFCAA|2020-11-25|1450|6309|484|2020|3|11|25|4|2020|484|6309|Wednesday|2020Q4|N|N|N|2459155|2459459|2458813|2459087|N|N|N|N|N| +2459180|AAAAAAAAMCGIFCAA|2020-11-26|1450|6309|484|2020|4|11|26|4|2020|484|6309|Thursday|2020Q4|N|N|N|2459155|2459459|2458814|2459088|N|N|N|N|N| +2459181|AAAAAAAANCGIFCAA|2020-11-27|1450|6309|484|2020|5|11|27|4|2020|484|6309|Friday|2020Q4|N|Y|N|2459155|2459459|2458815|2459089|N|N|N|N|N| +2459182|AAAAAAAAOCGIFCAA|2020-11-28|1450|6309|484|2020|6|11|28|4|2020|484|6309|Saturday|2020Q4|N|Y|N|2459155|2459459|2458816|2459090|N|N|N|N|N| +2459183|AAAAAAAAPCGIFCAA|2020-11-29|1450|6309|484|2020|0|11|29|4|2020|484|6309|Sunday|2020Q4|N|N|N|2459155|2459459|2458817|2459091|N|N|N|N|N| +2459184|AAAAAAAAADGIFCAA|2020-11-30|1450|6309|484|2020|1|11|30|4|2020|484|6309|Monday|2020Q4|N|N|N|2459155|2459459|2458818|2459092|N|N|N|N|N| +2459185|AAAAAAAABDGIFCAA|2020-12-01|1451|6310|485|2020|2|12|1|4|2020|485|6310|Tuesday|2020Q4|N|N|N|2459185|2459519|2458819|2459093|N|N|N|N|N| +2459186|AAAAAAAACDGIFCAA|2020-12-02|1451|6310|485|2020|3|12|2|4|2020|485|6310|Wednesday|2020Q4|N|N|N|2459185|2459519|2458820|2459094|N|N|N|N|N| +2459187|AAAAAAAADDGIFCAA|2020-12-03|1451|6310|485|2020|4|12|3|4|2020|485|6310|Thursday|2020Q4|N|N|N|2459185|2459519|2458821|2459095|N|N|N|N|N| +2459188|AAAAAAAAEDGIFCAA|2020-12-04|1451|6310|485|2020|5|12|4|4|2020|485|6310|Friday|2020Q4|N|Y|N|2459185|2459519|2458822|2459096|N|N|N|N|N| +2459189|AAAAAAAAFDGIFCAA|2020-12-05|1451|6310|485|2020|6|12|5|4|2020|485|6310|Saturday|2020Q4|N|Y|N|2459185|2459519|2458823|2459097|N|N|N|N|N| +2459190|AAAAAAAAGDGIFCAA|2020-12-06|1451|6310|485|2020|0|12|6|4|2020|485|6310|Sunday|2020Q4|N|N|N|2459185|2459519|2458824|2459098|N|N|N|N|N| +2459191|AAAAAAAAHDGIFCAA|2020-12-07|1451|6310|485|2020|1|12|7|4|2020|485|6310|Monday|2020Q4|N|N|N|2459185|2459519|2458825|2459099|N|N|N|N|N| +2459192|AAAAAAAAIDGIFCAA|2020-12-08|1451|6311|485|2020|2|12|8|4|2020|485|6311|Tuesday|2020Q4|N|N|N|2459185|2459519|2458826|2459100|N|N|N|N|N| +2459193|AAAAAAAAJDGIFCAA|2020-12-09|1451|6311|485|2020|3|12|9|4|2020|485|6311|Wednesday|2020Q4|N|N|N|2459185|2459519|2458827|2459101|N|N|N|N|N| +2459194|AAAAAAAAKDGIFCAA|2020-12-10|1451|6311|485|2020|4|12|10|4|2020|485|6311|Thursday|2020Q4|N|N|N|2459185|2459519|2458828|2459102|N|N|N|N|N| +2459195|AAAAAAAALDGIFCAA|2020-12-11|1451|6311|485|2020|5|12|11|4|2020|485|6311|Friday|2020Q4|N|Y|N|2459185|2459519|2458829|2459103|N|N|N|N|N| +2459196|AAAAAAAAMDGIFCAA|2020-12-12|1451|6311|485|2020|6|12|12|4|2020|485|6311|Saturday|2020Q4|N|Y|N|2459185|2459519|2458830|2459104|N|N|N|N|N| +2459197|AAAAAAAANDGIFCAA|2020-12-13|1451|6311|485|2020|0|12|13|4|2020|485|6311|Sunday|2020Q4|N|N|N|2459185|2459519|2458831|2459105|N|N|N|N|N| +2459198|AAAAAAAAODGIFCAA|2020-12-14|1451|6311|485|2020|1|12|14|4|2020|485|6311|Monday|2020Q4|N|N|N|2459185|2459519|2458832|2459106|N|N|N|N|N| +2459199|AAAAAAAAPDGIFCAA|2020-12-15|1451|6312|485|2020|2|12|15|4|2020|485|6312|Tuesday|2020Q4|N|N|N|2459185|2459519|2458833|2459107|N|N|N|N|N| +2459200|AAAAAAAAAEGIFCAA|2020-12-16|1451|6312|485|2020|3|12|16|4|2020|485|6312|Wednesday|2020Q4|N|N|N|2459185|2459519|2458834|2459108|N|N|N|N|N| +2459201|AAAAAAAABEGIFCAA|2020-12-17|1451|6312|485|2020|4|12|17|4|2020|485|6312|Thursday|2020Q4|N|N|N|2459185|2459519|2458835|2459109|N|N|N|N|N| +2459202|AAAAAAAACEGIFCAA|2020-12-18|1451|6312|485|2020|5|12|18|4|2020|485|6312|Friday|2020Q4|N|Y|N|2459185|2459519|2458836|2459110|N|N|N|N|N| +2459203|AAAAAAAADEGIFCAA|2020-12-19|1451|6312|485|2020|6|12|19|4|2020|485|6312|Saturday|2020Q4|N|Y|N|2459185|2459519|2458837|2459111|N|N|N|N|N| +2459204|AAAAAAAAEEGIFCAA|2020-12-20|1451|6312|485|2020|0|12|20|4|2020|485|6312|Sunday|2020Q4|N|N|N|2459185|2459519|2458838|2459112|N|N|N|N|N| +2459205|AAAAAAAAFEGIFCAA|2020-12-21|1451|6312|485|2020|1|12|21|4|2020|485|6312|Monday|2020Q4|N|N|N|2459185|2459519|2458839|2459113|N|N|N|N|N| +2459206|AAAAAAAAGEGIFCAA|2020-12-22|1451|6313|485|2020|2|12|22|4|2020|485|6313|Tuesday|2020Q4|N|N|N|2459185|2459519|2458840|2459114|N|N|N|N|N| +2459207|AAAAAAAAHEGIFCAA|2020-12-23|1451|6313|485|2020|3|12|23|4|2020|485|6313|Wednesday|2020Q4|N|N|N|2459185|2459519|2458841|2459115|N|N|N|N|N| +2459208|AAAAAAAAIEGIFCAA|2020-12-24|1451|6313|485|2020|4|12|24|4|2020|485|6313|Thursday|2020Q4|N|N|N|2459185|2459519|2458842|2459116|N|N|N|N|N| +2459209|AAAAAAAAJEGIFCAA|2020-12-25|1451|6313|485|2020|5|12|25|4|2020|485|6313|Friday|2020Q4|Y|Y|N|2459185|2459519|2458843|2459117|N|N|N|N|N| +2459210|AAAAAAAAKEGIFCAA|2020-12-26|1451|6313|485|2020|6|12|26|4|2020|485|6313|Saturday|2020Q4|N|Y|Y|2459185|2459519|2458844|2459118|N|N|N|N|N| +2459211|AAAAAAAALEGIFCAA|2020-12-27|1451|6313|485|2020|0|12|27|4|2020|485|6313|Sunday|2020Q4|N|N|N|2459185|2459519|2458845|2459119|N|N|N|N|N| +2459212|AAAAAAAAMEGIFCAA|2020-12-28|1451|6313|485|2020|1|12|28|4|2020|485|6313|Monday|2020Q4|N|N|N|2459185|2459519|2458846|2459120|N|N|N|N|N| +2459213|AAAAAAAANEGIFCAA|2020-12-29|1451|6314|485|2020|2|12|29|4|2020|485|6314|Tuesday|2020Q4|N|N|N|2459185|2459519|2458847|2459121|N|N|N|N|N| +2459214|AAAAAAAAOEGIFCAA|2020-12-30|1451|6314|485|2020|3|12|30|4|2020|485|6314|Wednesday|2020Q4|N|N|N|2459185|2459519|2458848|2459122|N|N|N|N|N| +2459215|AAAAAAAAPEGIFCAA|2020-12-31|1451|6314|485|2020|4|12|31|4|2020|485|6314|Thursday|2020Q4|Y|N|N|2459185|2459519|2458849|2459123|N|N|N|N|N| +2459216|AAAAAAAAAFGIFCAA|2021-01-01|1452|6314|485|2021|5|1|1|1|2021|485|6314|Friday|2021Q1|Y|Y|Y|2459216|2459215|2458850|2459124|N|N|N|N|N| +2459217|AAAAAAAABFGIFCAA|2021-01-02|1452|6314|485|2021|6|1|2|1|2021|485|6314|Saturday|2021Q1|N|Y|Y|2459216|2459215|2458851|2459125|N|N|N|N|N| +2459218|AAAAAAAACFGIFCAA|2021-01-03|1452|6314|485|2021|0|1|3|1|2021|485|6314|Sunday|2021Q1|N|N|N|2459216|2459215|2458852|2459126|N|N|N|N|N| +2459219|AAAAAAAADFGIFCAA|2021-01-04|1452|6314|485|2021|1|1|4|1|2021|485|6314|Monday|2021Q1|N|N|N|2459216|2459215|2458853|2459127|N|N|N|N|N| +2459220|AAAAAAAAEFGIFCAA|2021-01-05|1452|6315|485|2021|2|1|5|1|2021|485|6315|Tuesday|2021Q1|N|N|N|2459216|2459215|2458854|2459128|N|N|N|N|N| +2459221|AAAAAAAAFFGIFCAA|2021-01-06|1452|6315|485|2021|3|1|6|1|2021|485|6315|Wednesday|2021Q1|N|N|N|2459216|2459215|2458855|2459129|N|N|N|N|N| +2459222|AAAAAAAAGFGIFCAA|2021-01-07|1452|6315|485|2021|4|1|7|1|2021|485|6315|Thursday|2021Q1|N|N|N|2459216|2459215|2458856|2459130|N|N|N|N|N| +2459223|AAAAAAAAHFGIFCAA|2021-01-08|1452|6315|485|2021|5|1|8|1|2021|485|6315|Friday|2021Q1|N|Y|N|2459216|2459215|2458857|2459131|N|N|N|N|N| +2459224|AAAAAAAAIFGIFCAA|2021-01-09|1452|6315|485|2021|6|1|9|1|2021|485|6315|Saturday|2021Q1|N|Y|N|2459216|2459215|2458858|2459132|N|N|N|N|N| +2459225|AAAAAAAAJFGIFCAA|2021-01-10|1452|6315|485|2021|0|1|10|1|2021|485|6315|Sunday|2021Q1|N|N|N|2459216|2459215|2458859|2459133|N|N|N|N|N| +2459226|AAAAAAAAKFGIFCAA|2021-01-11|1452|6315|485|2021|1|1|11|1|2021|485|6315|Monday|2021Q1|N|N|N|2459216|2459215|2458860|2459134|N|N|N|N|N| +2459227|AAAAAAAALFGIFCAA|2021-01-12|1452|6316|485|2021|2|1|12|1|2021|485|6316|Tuesday|2021Q1|N|N|N|2459216|2459215|2458861|2459135|N|N|N|N|N| +2459228|AAAAAAAAMFGIFCAA|2021-01-13|1452|6316|485|2021|3|1|13|1|2021|485|6316|Wednesday|2021Q1|N|N|N|2459216|2459215|2458862|2459136|N|N|N|N|N| +2459229|AAAAAAAANFGIFCAA|2021-01-14|1452|6316|485|2021|4|1|14|1|2021|485|6316|Thursday|2021Q1|N|N|N|2459216|2459215|2458863|2459137|N|N|N|N|N| +2459230|AAAAAAAAOFGIFCAA|2021-01-15|1452|6316|485|2021|5|1|15|1|2021|485|6316|Friday|2021Q1|N|Y|N|2459216|2459215|2458864|2459138|N|N|N|N|N| +2459231|AAAAAAAAPFGIFCAA|2021-01-16|1452|6316|485|2021|6|1|16|1|2021|485|6316|Saturday|2021Q1|N|Y|N|2459216|2459215|2458865|2459139|N|N|N|N|N| +2459232|AAAAAAAAAGGIFCAA|2021-01-17|1452|6316|485|2021|0|1|17|1|2021|485|6316|Sunday|2021Q1|N|N|N|2459216|2459215|2458866|2459140|N|N|N|N|N| +2459233|AAAAAAAABGGIFCAA|2021-01-18|1452|6316|485|2021|1|1|18|1|2021|485|6316|Monday|2021Q1|N|N|N|2459216|2459215|2458867|2459141|N|N|N|N|N| +2459234|AAAAAAAACGGIFCAA|2021-01-19|1452|6317|485|2021|2|1|19|1|2021|485|6317|Tuesday|2021Q1|N|N|N|2459216|2459215|2458868|2459142|N|N|N|N|N| +2459235|AAAAAAAADGGIFCAA|2021-01-20|1452|6317|485|2021|3|1|20|1|2021|485|6317|Wednesday|2021Q1|N|N|N|2459216|2459215|2458869|2459143|N|N|N|N|N| +2459236|AAAAAAAAEGGIFCAA|2021-01-21|1452|6317|485|2021|4|1|21|1|2021|485|6317|Thursday|2021Q1|N|N|N|2459216|2459215|2458870|2459144|N|N|N|N|N| +2459237|AAAAAAAAFGGIFCAA|2021-01-22|1452|6317|485|2021|5|1|22|1|2021|485|6317|Friday|2021Q1|N|Y|N|2459216|2459215|2458871|2459145|N|N|N|N|N| +2459238|AAAAAAAAGGGIFCAA|2021-01-23|1452|6317|485|2021|6|1|23|1|2021|485|6317|Saturday|2021Q1|N|Y|N|2459216|2459215|2458872|2459146|N|N|N|N|N| +2459239|AAAAAAAAHGGIFCAA|2021-01-24|1452|6317|485|2021|0|1|24|1|2021|485|6317|Sunday|2021Q1|N|N|N|2459216|2459215|2458873|2459147|N|N|N|N|N| +2459240|AAAAAAAAIGGIFCAA|2021-01-25|1452|6317|485|2021|1|1|25|1|2021|485|6317|Monday|2021Q1|N|N|N|2459216|2459215|2458874|2459148|N|N|N|N|N| +2459241|AAAAAAAAJGGIFCAA|2021-01-26|1452|6318|485|2021|2|1|26|1|2021|485|6318|Tuesday|2021Q1|N|N|N|2459216|2459215|2458875|2459149|N|N|N|N|N| +2459242|AAAAAAAAKGGIFCAA|2021-01-27|1452|6318|485|2021|3|1|27|1|2021|485|6318|Wednesday|2021Q1|N|N|N|2459216|2459215|2458876|2459150|N|N|N|N|N| +2459243|AAAAAAAALGGIFCAA|2021-01-28|1452|6318|485|2021|4|1|28|1|2021|485|6318|Thursday|2021Q1|N|N|N|2459216|2459215|2458877|2459151|N|N|N|N|N| +2459244|AAAAAAAAMGGIFCAA|2021-01-29|1452|6318|485|2021|5|1|29|1|2021|485|6318|Friday|2021Q1|N|Y|N|2459216|2459215|2458878|2459152|N|N|N|N|N| +2459245|AAAAAAAANGGIFCAA|2021-01-30|1452|6318|485|2021|6|1|30|1|2021|485|6318|Saturday|2021Q1|N|Y|N|2459216|2459215|2458879|2459153|N|N|N|N|N| +2459246|AAAAAAAAOGGIFCAA|2021-01-31|1452|6318|485|2021|0|1|31|1|2021|485|6318|Sunday|2021Q1|N|N|N|2459216|2459215|2458880|2459154|N|N|N|N|N| +2459247|AAAAAAAAPGGIFCAA|2021-02-01|1453|6318|485|2021|1|2|1|1|2021|485|6318|Monday|2021Q1|N|N|N|2459247|2459277|2458881|2459155|N|N|N|N|N| +2459248|AAAAAAAAAHGIFCAA|2021-02-02|1453|6319|485|2021|2|2|2|1|2021|485|6319|Tuesday|2021Q1|N|N|N|2459247|2459277|2458882|2459156|N|N|N|N|N| +2459249|AAAAAAAABHGIFCAA|2021-02-03|1453|6319|485|2021|3|2|3|1|2021|485|6319|Wednesday|2021Q1|N|N|N|2459247|2459277|2458883|2459157|N|N|N|N|N| +2459250|AAAAAAAACHGIFCAA|2021-02-04|1453|6319|485|2021|4|2|4|1|2021|485|6319|Thursday|2021Q1|N|N|N|2459247|2459277|2458884|2459158|N|N|N|N|N| +2459251|AAAAAAAADHGIFCAA|2021-02-05|1453|6319|485|2021|5|2|5|1|2021|485|6319|Friday|2021Q1|N|Y|N|2459247|2459277|2458885|2459159|N|N|N|N|N| +2459252|AAAAAAAAEHGIFCAA|2021-02-06|1453|6319|485|2021|6|2|6|1|2021|485|6319|Saturday|2021Q1|N|Y|N|2459247|2459277|2458886|2459160|N|N|N|N|N| +2459253|AAAAAAAAFHGIFCAA|2021-02-07|1453|6319|485|2021|0|2|7|1|2021|485|6319|Sunday|2021Q1|N|N|N|2459247|2459277|2458887|2459161|N|N|N|N|N| +2459254|AAAAAAAAGHGIFCAA|2021-02-08|1453|6319|485|2021|1|2|8|1|2021|485|6319|Monday|2021Q1|N|N|N|2459247|2459277|2458888|2459162|N|N|N|N|N| +2459255|AAAAAAAAHHGIFCAA|2021-02-09|1453|6320|485|2021|2|2|9|1|2021|485|6320|Tuesday|2021Q1|N|N|N|2459247|2459277|2458889|2459163|N|N|N|N|N| +2459256|AAAAAAAAIHGIFCAA|2021-02-10|1453|6320|485|2021|3|2|10|1|2021|485|6320|Wednesday|2021Q1|N|N|N|2459247|2459277|2458890|2459164|N|N|N|N|N| +2459257|AAAAAAAAJHGIFCAA|2021-02-11|1453|6320|485|2021|4|2|11|1|2021|485|6320|Thursday|2021Q1|N|N|N|2459247|2459277|2458891|2459165|N|N|N|N|N| +2459258|AAAAAAAAKHGIFCAA|2021-02-12|1453|6320|485|2021|5|2|12|1|2021|485|6320|Friday|2021Q1|N|Y|N|2459247|2459277|2458892|2459166|N|N|N|N|N| +2459259|AAAAAAAALHGIFCAA|2021-02-13|1453|6320|485|2021|6|2|13|1|2021|485|6320|Saturday|2021Q1|N|Y|N|2459247|2459277|2458893|2459167|N|N|N|N|N| +2459260|AAAAAAAAMHGIFCAA|2021-02-14|1453|6320|485|2021|0|2|14|1|2021|485|6320|Sunday|2021Q1|N|N|N|2459247|2459277|2458894|2459168|N|N|N|N|N| +2459261|AAAAAAAANHGIFCAA|2021-02-15|1453|6320|485|2021|1|2|15|1|2021|485|6320|Monday|2021Q1|N|N|N|2459247|2459277|2458895|2459169|N|N|N|N|N| +2459262|AAAAAAAAOHGIFCAA|2021-02-16|1453|6321|485|2021|2|2|16|1|2021|485|6321|Tuesday|2021Q1|N|N|N|2459247|2459277|2458896|2459170|N|N|N|N|N| +2459263|AAAAAAAAPHGIFCAA|2021-02-17|1453|6321|485|2021|3|2|17|1|2021|485|6321|Wednesday|2021Q1|N|N|N|2459247|2459277|2458897|2459171|N|N|N|N|N| +2459264|AAAAAAAAAIGIFCAA|2021-02-18|1453|6321|485|2021|4|2|18|1|2021|485|6321|Thursday|2021Q1|N|N|N|2459247|2459277|2458898|2459172|N|N|N|N|N| +2459265|AAAAAAAABIGIFCAA|2021-02-19|1453|6321|485|2021|5|2|19|1|2021|485|6321|Friday|2021Q1|N|Y|N|2459247|2459277|2458899|2459173|N|N|N|N|N| +2459266|AAAAAAAACIGIFCAA|2021-02-20|1453|6321|485|2021|6|2|20|1|2021|485|6321|Saturday|2021Q1|N|Y|N|2459247|2459277|2458900|2459174|N|N|N|N|N| +2459267|AAAAAAAADIGIFCAA|2021-02-21|1453|6321|485|2021|0|2|21|1|2021|485|6321|Sunday|2021Q1|N|N|N|2459247|2459277|2458901|2459175|N|N|N|N|N| +2459268|AAAAAAAAEIGIFCAA|2021-02-22|1453|6321|485|2021|1|2|22|1|2021|485|6321|Monday|2021Q1|N|N|N|2459247|2459277|2458902|2459176|N|N|N|N|N| +2459269|AAAAAAAAFIGIFCAA|2021-02-23|1453|6322|485|2021|2|2|23|1|2021|485|6322|Tuesday|2021Q1|N|N|N|2459247|2459277|2458903|2459177|N|N|N|N|N| +2459270|AAAAAAAAGIGIFCAA|2021-02-24|1453|6322|485|2021|3|2|24|1|2021|485|6322|Wednesday|2021Q1|N|N|N|2459247|2459277|2458904|2459178|N|N|N|N|N| +2459271|AAAAAAAAHIGIFCAA|2021-02-25|1453|6322|485|2021|4|2|25|1|2021|485|6322|Thursday|2021Q1|N|N|N|2459247|2459277|2458905|2459179|N|N|N|N|N| +2459272|AAAAAAAAIIGIFCAA|2021-02-26|1453|6322|485|2021|5|2|26|1|2021|485|6322|Friday|2021Q1|N|Y|N|2459247|2459277|2458906|2459180|N|N|N|N|N| +2459273|AAAAAAAAJIGIFCAA|2021-02-27|1453|6322|485|2021|6|2|27|1|2021|485|6322|Saturday|2021Q1|N|Y|N|2459247|2459277|2458907|2459181|N|N|N|N|N| +2459274|AAAAAAAAKIGIFCAA|2021-02-28|1453|6322|485|2021|0|2|28|1|2021|485|6322|Sunday|2021Q1|N|N|N|2459247|2459277|2458908|2459182|N|N|N|N|N| +2459275|AAAAAAAALIGIFCAA|2021-03-01|1454|6322|486|2021|1|3|1|1|2021|486|6322|Monday|2021Q1|N|N|N|2459275|2459333|2458910|2459183|N|N|N|N|N| +2459276|AAAAAAAAMIGIFCAA|2021-03-02|1454|6323|486|2021|2|3|2|1|2021|486|6323|Tuesday|2021Q1|N|N|N|2459275|2459333|2458911|2459184|N|N|N|N|N| +2459277|AAAAAAAANIGIFCAA|2021-03-03|1454|6323|486|2021|3|3|3|1|2021|486|6323|Wednesday|2021Q1|N|N|N|2459275|2459333|2458912|2459185|N|N|N|N|N| +2459278|AAAAAAAAOIGIFCAA|2021-03-04|1454|6323|486|2021|4|3|4|1|2021|486|6323|Thursday|2021Q1|N|N|N|2459275|2459333|2458913|2459186|N|N|N|N|N| +2459279|AAAAAAAAPIGIFCAA|2021-03-05|1454|6323|486|2021|5|3|5|1|2021|486|6323|Friday|2021Q1|N|Y|N|2459275|2459333|2458914|2459187|N|N|N|N|N| +2459280|AAAAAAAAAJGIFCAA|2021-03-06|1454|6323|486|2021|6|3|6|1|2021|486|6323|Saturday|2021Q1|N|Y|N|2459275|2459333|2458915|2459188|N|N|N|N|N| +2459281|AAAAAAAABJGIFCAA|2021-03-07|1454|6323|486|2021|0|3|7|1|2021|486|6323|Sunday|2021Q1|N|N|N|2459275|2459333|2458916|2459189|N|N|N|N|N| +2459282|AAAAAAAACJGIFCAA|2021-03-08|1454|6323|486|2021|1|3|8|1|2021|486|6323|Monday|2021Q1|N|N|N|2459275|2459333|2458917|2459190|N|N|N|N|N| +2459283|AAAAAAAADJGIFCAA|2021-03-09|1454|6324|486|2021|2|3|9|1|2021|486|6324|Tuesday|2021Q1|N|N|N|2459275|2459333|2458918|2459191|N|N|N|N|N| +2459284|AAAAAAAAEJGIFCAA|2021-03-10|1454|6324|486|2021|3|3|10|1|2021|486|6324|Wednesday|2021Q1|N|N|N|2459275|2459333|2458919|2459192|N|N|N|N|N| +2459285|AAAAAAAAFJGIFCAA|2021-03-11|1454|6324|486|2021|4|3|11|1|2021|486|6324|Thursday|2021Q1|N|N|N|2459275|2459333|2458920|2459193|N|N|N|N|N| +2459286|AAAAAAAAGJGIFCAA|2021-03-12|1454|6324|486|2021|5|3|12|1|2021|486|6324|Friday|2021Q1|N|Y|N|2459275|2459333|2458921|2459194|N|N|N|N|N| +2459287|AAAAAAAAHJGIFCAA|2021-03-13|1454|6324|486|2021|6|3|13|1|2021|486|6324|Saturday|2021Q1|N|Y|N|2459275|2459333|2458922|2459195|N|N|N|N|N| +2459288|AAAAAAAAIJGIFCAA|2021-03-14|1454|6324|486|2021|0|3|14|1|2021|486|6324|Sunday|2021Q1|N|N|N|2459275|2459333|2458923|2459196|N|N|N|N|N| +2459289|AAAAAAAAJJGIFCAA|2021-03-15|1454|6324|486|2021|1|3|15|1|2021|486|6324|Monday|2021Q1|N|N|N|2459275|2459333|2458924|2459197|N|N|N|N|N| +2459290|AAAAAAAAKJGIFCAA|2021-03-16|1454|6325|486|2021|2|3|16|1|2021|486|6325|Tuesday|2021Q1|N|N|N|2459275|2459333|2458925|2459198|N|N|N|N|N| +2459291|AAAAAAAALJGIFCAA|2021-03-17|1454|6325|486|2021|3|3|17|1|2021|486|6325|Wednesday|2021Q1|N|N|N|2459275|2459333|2458926|2459199|N|N|N|N|N| +2459292|AAAAAAAAMJGIFCAA|2021-03-18|1454|6325|486|2021|4|3|18|1|2021|486|6325|Thursday|2021Q1|N|N|N|2459275|2459333|2458927|2459200|N|N|N|N|N| +2459293|AAAAAAAANJGIFCAA|2021-03-19|1454|6325|486|2021|5|3|19|1|2021|486|6325|Friday|2021Q1|N|Y|N|2459275|2459333|2458928|2459201|N|N|N|N|N| +2459294|AAAAAAAAOJGIFCAA|2021-03-20|1454|6325|486|2021|6|3|20|1|2021|486|6325|Saturday|2021Q1|N|Y|N|2459275|2459333|2458929|2459202|N|N|N|N|N| +2459295|AAAAAAAAPJGIFCAA|2021-03-21|1454|6325|486|2021|0|3|21|1|2021|486|6325|Sunday|2021Q1|N|N|N|2459275|2459333|2458930|2459203|N|N|N|N|N| +2459296|AAAAAAAAAKGIFCAA|2021-03-22|1454|6325|486|2021|1|3|22|1|2021|486|6325|Monday|2021Q1|N|N|N|2459275|2459333|2458931|2459204|N|N|N|N|N| +2459297|AAAAAAAABKGIFCAA|2021-03-23|1454|6326|486|2021|2|3|23|1|2021|486|6326|Tuesday|2021Q1|N|N|N|2459275|2459333|2458932|2459205|N|N|N|N|N| +2459298|AAAAAAAACKGIFCAA|2021-03-24|1454|6326|486|2021|3|3|24|1|2021|486|6326|Wednesday|2021Q1|N|N|N|2459275|2459333|2458933|2459206|N|N|N|N|N| +2459299|AAAAAAAADKGIFCAA|2021-03-25|1454|6326|486|2021|4|3|25|1|2021|486|6326|Thursday|2021Q1|N|N|N|2459275|2459333|2458934|2459207|N|N|N|N|N| +2459300|AAAAAAAAEKGIFCAA|2021-03-26|1454|6326|486|2021|5|3|26|1|2021|486|6326|Friday|2021Q1|N|Y|N|2459275|2459333|2458935|2459208|N|N|N|N|N| +2459301|AAAAAAAAFKGIFCAA|2021-03-27|1454|6326|486|2021|6|3|27|1|2021|486|6326|Saturday|2021Q1|N|Y|N|2459275|2459333|2458936|2459209|N|N|N|N|N| +2459302|AAAAAAAAGKGIFCAA|2021-03-28|1454|6326|486|2021|0|3|28|1|2021|486|6326|Sunday|2021Q1|N|N|N|2459275|2459333|2458937|2459210|N|N|N|N|N| +2459303|AAAAAAAAHKGIFCAA|2021-03-29|1454|6326|486|2021|1|3|29|1|2021|486|6326|Monday|2021Q1|N|N|N|2459275|2459333|2458938|2459211|N|N|N|N|N| +2459304|AAAAAAAAIKGIFCAA|2021-03-30|1454|6327|486|2021|2|3|30|1|2021|486|6327|Tuesday|2021Q1|N|N|N|2459275|2459333|2458939|2459212|N|N|N|N|N| +2459305|AAAAAAAAJKGIFCAA|2021-03-31|1454|6327|486|2021|3|3|31|1|2021|486|6327|Wednesday|2021Q1|N|N|N|2459275|2459333|2458940|2459213|N|N|N|N|N| +2459306|AAAAAAAAKKGIFCAA|2021-04-01|1455|6327|486|2021|4|4|1|1|2021|486|6327|Thursday|2021Q1|N|N|N|2459306|2459395|2458941|2459216|N|N|N|N|N| +2459307|AAAAAAAALKGIFCAA|2021-04-02|1455|6327|486|2021|5|4|2|2|2021|486|6327|Friday|2021Q2|N|Y|N|2459306|2459395|2458942|2459217|N|N|N|N|N| +2459308|AAAAAAAAMKGIFCAA|2021-04-03|1455|6327|486|2021|6|4|3|2|2021|486|6327|Saturday|2021Q2|N|Y|N|2459306|2459395|2458943|2459218|N|N|N|N|N| +2459309|AAAAAAAANKGIFCAA|2021-04-04|1455|6327|486|2021|0|4|4|2|2021|486|6327|Sunday|2021Q2|N|N|N|2459306|2459395|2458944|2459219|N|N|N|N|N| +2459310|AAAAAAAAOKGIFCAA|2021-04-05|1455|6327|486|2021|1|4|5|2|2021|486|6327|Monday|2021Q2|N|N|N|2459306|2459395|2458945|2459220|N|N|N|N|N| +2459311|AAAAAAAAPKGIFCAA|2021-04-06|1455|6328|486|2021|2|4|6|2|2021|486|6328|Tuesday|2021Q2|N|N|N|2459306|2459395|2458946|2459221|N|N|N|N|N| +2459312|AAAAAAAAALGIFCAA|2021-04-07|1455|6328|486|2021|3|4|7|2|2021|486|6328|Wednesday|2021Q2|N|N|N|2459306|2459395|2458947|2459222|N|N|N|N|N| +2459313|AAAAAAAABLGIFCAA|2021-04-08|1455|6328|486|2021|4|4|8|2|2021|486|6328|Thursday|2021Q2|N|N|N|2459306|2459395|2458948|2459223|N|N|N|N|N| +2459314|AAAAAAAACLGIFCAA|2021-04-09|1455|6328|486|2021|5|4|9|2|2021|486|6328|Friday|2021Q2|N|Y|N|2459306|2459395|2458949|2459224|N|N|N|N|N| +2459315|AAAAAAAADLGIFCAA|2021-04-10|1455|6328|486|2021|6|4|10|2|2021|486|6328|Saturday|2021Q2|N|Y|N|2459306|2459395|2458950|2459225|N|N|N|N|N| +2459316|AAAAAAAAELGIFCAA|2021-04-11|1455|6328|486|2021|0|4|11|2|2021|486|6328|Sunday|2021Q2|N|N|N|2459306|2459395|2458951|2459226|N|N|N|N|N| +2459317|AAAAAAAAFLGIFCAA|2021-04-12|1455|6328|486|2021|1|4|12|2|2021|486|6328|Monday|2021Q2|N|N|N|2459306|2459395|2458952|2459227|N|N|N|N|N| +2459318|AAAAAAAAGLGIFCAA|2021-04-13|1455|6329|486|2021|2|4|13|2|2021|486|6329|Tuesday|2021Q2|N|N|N|2459306|2459395|2458953|2459228|N|N|N|N|N| +2459319|AAAAAAAAHLGIFCAA|2021-04-14|1455|6329|486|2021|3|4|14|2|2021|486|6329|Wednesday|2021Q2|N|N|N|2459306|2459395|2458954|2459229|N|N|N|N|N| +2459320|AAAAAAAAILGIFCAA|2021-04-15|1455|6329|486|2021|4|4|15|2|2021|486|6329|Thursday|2021Q2|N|N|N|2459306|2459395|2458955|2459230|N|N|N|N|N| +2459321|AAAAAAAAJLGIFCAA|2021-04-16|1455|6329|486|2021|5|4|16|2|2021|486|6329|Friday|2021Q2|N|Y|N|2459306|2459395|2458956|2459231|N|N|N|N|N| +2459322|AAAAAAAAKLGIFCAA|2021-04-17|1455|6329|486|2021|6|4|17|2|2021|486|6329|Saturday|2021Q2|N|Y|N|2459306|2459395|2458957|2459232|N|N|N|N|N| +2459323|AAAAAAAALLGIFCAA|2021-04-18|1455|6329|486|2021|0|4|18|2|2021|486|6329|Sunday|2021Q2|N|N|N|2459306|2459395|2458958|2459233|N|N|N|N|N| +2459324|AAAAAAAAMLGIFCAA|2021-04-19|1455|6329|486|2021|1|4|19|2|2021|486|6329|Monday|2021Q2|N|N|N|2459306|2459395|2458959|2459234|N|N|N|N|N| +2459325|AAAAAAAANLGIFCAA|2021-04-20|1455|6330|486|2021|2|4|20|2|2021|486|6330|Tuesday|2021Q2|N|N|N|2459306|2459395|2458960|2459235|N|N|N|N|N| +2459326|AAAAAAAAOLGIFCAA|2021-04-21|1455|6330|486|2021|3|4|21|2|2021|486|6330|Wednesday|2021Q2|N|N|N|2459306|2459395|2458961|2459236|N|N|N|N|N| +2459327|AAAAAAAAPLGIFCAA|2021-04-22|1455|6330|486|2021|4|4|22|2|2021|486|6330|Thursday|2021Q2|N|N|N|2459306|2459395|2458962|2459237|N|N|N|N|N| +2459328|AAAAAAAAAMGIFCAA|2021-04-23|1455|6330|486|2021|5|4|23|2|2021|486|6330|Friday|2021Q2|N|Y|N|2459306|2459395|2458963|2459238|N|N|N|N|N| +2459329|AAAAAAAABMGIFCAA|2021-04-24|1455|6330|486|2021|6|4|24|2|2021|486|6330|Saturday|2021Q2|N|Y|N|2459306|2459395|2458964|2459239|N|N|N|N|N| +2459330|AAAAAAAACMGIFCAA|2021-04-25|1455|6330|486|2021|0|4|25|2|2021|486|6330|Sunday|2021Q2|N|N|N|2459306|2459395|2458965|2459240|N|N|N|N|N| +2459331|AAAAAAAADMGIFCAA|2021-04-26|1455|6330|486|2021|1|4|26|2|2021|486|6330|Monday|2021Q2|N|N|N|2459306|2459395|2458966|2459241|N|N|N|N|N| +2459332|AAAAAAAAEMGIFCAA|2021-04-27|1455|6331|486|2021|2|4|27|2|2021|486|6331|Tuesday|2021Q2|N|N|N|2459306|2459395|2458967|2459242|N|N|N|N|N| +2459333|AAAAAAAAFMGIFCAA|2021-04-28|1455|6331|486|2021|3|4|28|2|2021|486|6331|Wednesday|2021Q2|N|N|N|2459306|2459395|2458968|2459243|N|N|N|N|N| +2459334|AAAAAAAAGMGIFCAA|2021-04-29|1455|6331|486|2021|4|4|29|2|2021|486|6331|Thursday|2021Q2|N|N|N|2459306|2459395|2458969|2459244|N|N|N|N|N| +2459335|AAAAAAAAHMGIFCAA|2021-04-30|1455|6331|486|2021|5|4|30|2|2021|486|6331|Friday|2021Q2|N|Y|N|2459306|2459395|2458970|2459245|N|N|N|N|N| +2459336|AAAAAAAAIMGIFCAA|2021-05-01|1456|6331|486|2021|6|5|1|2|2021|486|6331|Saturday|2021Q2|N|Y|N|2459336|2459455|2458971|2459246|N|N|N|N|N| +2459337|AAAAAAAAJMGIFCAA|2021-05-02|1456|6331|486|2021|0|5|2|2|2021|486|6331|Sunday|2021Q2|N|N|N|2459336|2459455|2458972|2459247|N|N|N|N|N| +2459338|AAAAAAAAKMGIFCAA|2021-05-03|1456|6331|486|2021|1|5|3|2|2021|486|6331|Monday|2021Q2|N|N|N|2459336|2459455|2458973|2459248|N|N|N|N|N| +2459339|AAAAAAAALMGIFCAA|2021-05-04|1456|6332|486|2021|2|5|4|2|2021|486|6332|Tuesday|2021Q2|N|N|N|2459336|2459455|2458974|2459249|N|N|N|N|N| +2459340|AAAAAAAAMMGIFCAA|2021-05-05|1456|6332|486|2021|3|5|5|2|2021|486|6332|Wednesday|2021Q2|N|N|N|2459336|2459455|2458975|2459250|N|N|N|N|N| +2459341|AAAAAAAANMGIFCAA|2021-05-06|1456|6332|486|2021|4|5|6|2|2021|486|6332|Thursday|2021Q2|N|N|N|2459336|2459455|2458976|2459251|N|N|N|N|N| +2459342|AAAAAAAAOMGIFCAA|2021-05-07|1456|6332|486|2021|5|5|7|2|2021|486|6332|Friday|2021Q2|N|Y|N|2459336|2459455|2458977|2459252|N|N|N|N|N| +2459343|AAAAAAAAPMGIFCAA|2021-05-08|1456|6332|486|2021|6|5|8|2|2021|486|6332|Saturday|2021Q2|N|Y|N|2459336|2459455|2458978|2459253|N|N|N|N|N| +2459344|AAAAAAAAANGIFCAA|2021-05-09|1456|6332|486|2021|0|5|9|2|2021|486|6332|Sunday|2021Q2|N|N|N|2459336|2459455|2458979|2459254|N|N|N|N|N| +2459345|AAAAAAAABNGIFCAA|2021-05-10|1456|6332|486|2021|1|5|10|2|2021|486|6332|Monday|2021Q2|N|N|N|2459336|2459455|2458980|2459255|N|N|N|N|N| +2459346|AAAAAAAACNGIFCAA|2021-05-11|1456|6333|486|2021|2|5|11|2|2021|486|6333|Tuesday|2021Q2|N|N|N|2459336|2459455|2458981|2459256|N|N|N|N|N| +2459347|AAAAAAAADNGIFCAA|2021-05-12|1456|6333|486|2021|3|5|12|2|2021|486|6333|Wednesday|2021Q2|N|N|N|2459336|2459455|2458982|2459257|N|N|N|N|N| +2459348|AAAAAAAAENGIFCAA|2021-05-13|1456|6333|486|2021|4|5|13|2|2021|486|6333|Thursday|2021Q2|N|N|N|2459336|2459455|2458983|2459258|N|N|N|N|N| +2459349|AAAAAAAAFNGIFCAA|2021-05-14|1456|6333|486|2021|5|5|14|2|2021|486|6333|Friday|2021Q2|N|Y|N|2459336|2459455|2458984|2459259|N|N|N|N|N| +2459350|AAAAAAAAGNGIFCAA|2021-05-15|1456|6333|486|2021|6|5|15|2|2021|486|6333|Saturday|2021Q2|N|Y|N|2459336|2459455|2458985|2459260|N|N|N|N|N| +2459351|AAAAAAAAHNGIFCAA|2021-05-16|1456|6333|486|2021|0|5|16|2|2021|486|6333|Sunday|2021Q2|N|N|N|2459336|2459455|2458986|2459261|N|N|N|N|N| +2459352|AAAAAAAAINGIFCAA|2021-05-17|1456|6333|486|2021|1|5|17|2|2021|486|6333|Monday|2021Q2|N|N|N|2459336|2459455|2458987|2459262|N|N|N|N|N| +2459353|AAAAAAAAJNGIFCAA|2021-05-18|1456|6334|486|2021|2|5|18|2|2021|486|6334|Tuesday|2021Q2|N|N|N|2459336|2459455|2458988|2459263|N|N|N|N|N| +2459354|AAAAAAAAKNGIFCAA|2021-05-19|1456|6334|486|2021|3|5|19|2|2021|486|6334|Wednesday|2021Q2|N|N|N|2459336|2459455|2458989|2459264|N|N|N|N|N| +2459355|AAAAAAAALNGIFCAA|2021-05-20|1456|6334|486|2021|4|5|20|2|2021|486|6334|Thursday|2021Q2|N|N|N|2459336|2459455|2458990|2459265|N|N|N|N|N| +2459356|AAAAAAAAMNGIFCAA|2021-05-21|1456|6334|486|2021|5|5|21|2|2021|486|6334|Friday|2021Q2|N|Y|N|2459336|2459455|2458991|2459266|N|N|N|N|N| +2459357|AAAAAAAANNGIFCAA|2021-05-22|1456|6334|486|2021|6|5|22|2|2021|486|6334|Saturday|2021Q2|N|Y|N|2459336|2459455|2458992|2459267|N|N|N|N|N| +2459358|AAAAAAAAONGIFCAA|2021-05-23|1456|6334|486|2021|0|5|23|2|2021|486|6334|Sunday|2021Q2|N|N|N|2459336|2459455|2458993|2459268|N|N|N|N|N| +2459359|AAAAAAAAPNGIFCAA|2021-05-24|1456|6334|486|2021|1|5|24|2|2021|486|6334|Monday|2021Q2|N|N|N|2459336|2459455|2458994|2459269|N|N|N|N|N| +2459360|AAAAAAAAAOGIFCAA|2021-05-25|1456|6335|486|2021|2|5|25|2|2021|486|6335|Tuesday|2021Q2|N|N|N|2459336|2459455|2458995|2459270|N|N|N|N|N| +2459361|AAAAAAAABOGIFCAA|2021-05-26|1456|6335|486|2021|3|5|26|2|2021|486|6335|Wednesday|2021Q2|N|N|N|2459336|2459455|2458996|2459271|N|N|N|N|N| +2459362|AAAAAAAACOGIFCAA|2021-05-27|1456|6335|486|2021|4|5|27|2|2021|486|6335|Thursday|2021Q2|N|N|N|2459336|2459455|2458997|2459272|N|N|N|N|N| +2459363|AAAAAAAADOGIFCAA|2021-05-28|1456|6335|486|2021|5|5|28|2|2021|486|6335|Friday|2021Q2|N|Y|N|2459336|2459455|2458998|2459273|N|N|N|N|N| +2459364|AAAAAAAAEOGIFCAA|2021-05-29|1456|6335|486|2021|6|5|29|2|2021|486|6335|Saturday|2021Q2|N|Y|N|2459336|2459455|2458999|2459274|N|N|N|N|N| +2459365|AAAAAAAAFOGIFCAA|2021-05-30|1456|6335|486|2021|0|5|30|2|2021|486|6335|Sunday|2021Q2|N|N|N|2459336|2459455|2459000|2459275|N|N|N|N|N| +2459366|AAAAAAAAGOGIFCAA|2021-05-31|1456|6335|486|2021|1|5|31|2|2021|486|6335|Monday|2021Q2|N|N|N|2459336|2459455|2459001|2459276|N|N|N|N|N| +2459367|AAAAAAAAHOGIFCAA|2021-06-01|1457|6336|487|2021|2|6|1|2|2021|487|6336|Tuesday|2021Q2|N|N|N|2459367|2459517|2459002|2459277|N|N|N|N|N| +2459368|AAAAAAAAIOGIFCAA|2021-06-02|1457|6336|487|2021|3|6|2|2|2021|487|6336|Wednesday|2021Q2|N|N|N|2459367|2459517|2459003|2459278|N|N|N|N|N| +2459369|AAAAAAAAJOGIFCAA|2021-06-03|1457|6336|487|2021|4|6|3|2|2021|487|6336|Thursday|2021Q2|N|N|N|2459367|2459517|2459004|2459279|N|N|N|N|N| +2459370|AAAAAAAAKOGIFCAA|2021-06-04|1457|6336|487|2021|5|6|4|2|2021|487|6336|Friday|2021Q2|N|Y|N|2459367|2459517|2459005|2459280|N|N|N|N|N| +2459371|AAAAAAAALOGIFCAA|2021-06-05|1457|6336|487|2021|6|6|5|2|2021|487|6336|Saturday|2021Q2|N|Y|N|2459367|2459517|2459006|2459281|N|N|N|N|N| +2459372|AAAAAAAAMOGIFCAA|2021-06-06|1457|6336|487|2021|0|6|6|2|2021|487|6336|Sunday|2021Q2|N|N|N|2459367|2459517|2459007|2459282|N|N|N|N|N| +2459373|AAAAAAAANOGIFCAA|2021-06-07|1457|6336|487|2021|1|6|7|2|2021|487|6336|Monday|2021Q2|N|N|N|2459367|2459517|2459008|2459283|N|N|N|N|N| +2459374|AAAAAAAAOOGIFCAA|2021-06-08|1457|6337|487|2021|2|6|8|2|2021|487|6337|Tuesday|2021Q2|N|N|N|2459367|2459517|2459009|2459284|N|N|N|N|N| +2459375|AAAAAAAAPOGIFCAA|2021-06-09|1457|6337|487|2021|3|6|9|2|2021|487|6337|Wednesday|2021Q2|N|N|N|2459367|2459517|2459010|2459285|N|N|N|N|N| +2459376|AAAAAAAAAPGIFCAA|2021-06-10|1457|6337|487|2021|4|6|10|2|2021|487|6337|Thursday|2021Q2|N|N|N|2459367|2459517|2459011|2459286|N|N|N|N|N| +2459377|AAAAAAAABPGIFCAA|2021-06-11|1457|6337|487|2021|5|6|11|2|2021|487|6337|Friday|2021Q2|N|Y|N|2459367|2459517|2459012|2459287|N|N|N|N|N| +2459378|AAAAAAAACPGIFCAA|2021-06-12|1457|6337|487|2021|6|6|12|2|2021|487|6337|Saturday|2021Q2|N|Y|N|2459367|2459517|2459013|2459288|N|N|N|N|N| +2459379|AAAAAAAADPGIFCAA|2021-06-13|1457|6337|487|2021|0|6|13|2|2021|487|6337|Sunday|2021Q2|N|N|N|2459367|2459517|2459014|2459289|N|N|N|N|N| +2459380|AAAAAAAAEPGIFCAA|2021-06-14|1457|6337|487|2021|1|6|14|2|2021|487|6337|Monday|2021Q2|N|N|N|2459367|2459517|2459015|2459290|N|N|N|N|N| +2459381|AAAAAAAAFPGIFCAA|2021-06-15|1457|6338|487|2021|2|6|15|2|2021|487|6338|Tuesday|2021Q2|N|N|N|2459367|2459517|2459016|2459291|N|N|N|N|N| +2459382|AAAAAAAAGPGIFCAA|2021-06-16|1457|6338|487|2021|3|6|16|2|2021|487|6338|Wednesday|2021Q2|N|N|N|2459367|2459517|2459017|2459292|N|N|N|N|N| +2459383|AAAAAAAAHPGIFCAA|2021-06-17|1457|6338|487|2021|4|6|17|2|2021|487|6338|Thursday|2021Q2|N|N|N|2459367|2459517|2459018|2459293|N|N|N|N|N| +2459384|AAAAAAAAIPGIFCAA|2021-06-18|1457|6338|487|2021|5|6|18|2|2021|487|6338|Friday|2021Q2|N|Y|N|2459367|2459517|2459019|2459294|N|N|N|N|N| +2459385|AAAAAAAAJPGIFCAA|2021-06-19|1457|6338|487|2021|6|6|19|2|2021|487|6338|Saturday|2021Q2|N|Y|N|2459367|2459517|2459020|2459295|N|N|N|N|N| +2459386|AAAAAAAAKPGIFCAA|2021-06-20|1457|6338|487|2021|0|6|20|2|2021|487|6338|Sunday|2021Q2|N|N|N|2459367|2459517|2459021|2459296|N|N|N|N|N| +2459387|AAAAAAAALPGIFCAA|2021-06-21|1457|6338|487|2021|1|6|21|2|2021|487|6338|Monday|2021Q2|N|N|N|2459367|2459517|2459022|2459297|N|N|N|N|N| +2459388|AAAAAAAAMPGIFCAA|2021-06-22|1457|6339|487|2021|2|6|22|2|2021|487|6339|Tuesday|2021Q2|N|N|N|2459367|2459517|2459023|2459298|N|N|N|N|N| +2459389|AAAAAAAANPGIFCAA|2021-06-23|1457|6339|487|2021|3|6|23|2|2021|487|6339|Wednesday|2021Q2|N|N|N|2459367|2459517|2459024|2459299|N|N|N|N|N| +2459390|AAAAAAAAOPGIFCAA|2021-06-24|1457|6339|487|2021|4|6|24|2|2021|487|6339|Thursday|2021Q2|N|N|N|2459367|2459517|2459025|2459300|N|N|N|N|N| +2459391|AAAAAAAAPPGIFCAA|2021-06-25|1457|6339|487|2021|5|6|25|2|2021|487|6339|Friday|2021Q2|N|Y|N|2459367|2459517|2459026|2459301|N|N|N|N|N| +2459392|AAAAAAAAAAHIFCAA|2021-06-26|1457|6339|487|2021|6|6|26|2|2021|487|6339|Saturday|2021Q2|N|Y|N|2459367|2459517|2459027|2459302|N|N|N|N|N| +2459393|AAAAAAAABAHIFCAA|2021-06-27|1457|6339|487|2021|0|6|27|2|2021|487|6339|Sunday|2021Q2|N|N|N|2459367|2459517|2459028|2459303|N|N|N|N|N| +2459394|AAAAAAAACAHIFCAA|2021-06-28|1457|6339|487|2021|1|6|28|2|2021|487|6339|Monday|2021Q2|N|N|N|2459367|2459517|2459029|2459304|N|N|N|N|N| +2459395|AAAAAAAADAHIFCAA|2021-06-29|1457|6340|487|2021|2|6|29|2|2021|487|6340|Tuesday|2021Q2|N|N|N|2459367|2459517|2459030|2459305|N|N|N|N|N| +2459396|AAAAAAAAEAHIFCAA|2021-06-30|1457|6340|487|2021|3|6|30|2|2021|487|6340|Wednesday|2021Q2|N|N|N|2459367|2459517|2459031|2459306|N|N|N|N|N| +2459397|AAAAAAAAFAHIFCAA|2021-07-01|1458|6340|487|2021|4|7|1|2|2021|487|6340|Thursday|2021Q2|N|N|N|2459397|2459577|2459032|2459306|N|N|N|N|N| +2459398|AAAAAAAAGAHIFCAA|2021-07-02|1458|6340|487|2021|5|7|2|3|2021|487|6340|Friday|2021Q3|N|Y|N|2459397|2459577|2459033|2459307|N|N|N|N|N| +2459399|AAAAAAAAHAHIFCAA|2021-07-03|1458|6340|487|2021|6|7|3|3|2021|487|6340|Saturday|2021Q3|N|Y|N|2459397|2459577|2459034|2459308|N|N|N|N|N| +2459400|AAAAAAAAIAHIFCAA|2021-07-04|1458|6340|487|2021|0|7|4|3|2021|487|6340|Sunday|2021Q3|N|N|N|2459397|2459577|2459035|2459309|N|N|N|N|N| +2459401|AAAAAAAAJAHIFCAA|2021-07-05|1458|6340|487|2021|1|7|5|3|2021|487|6340|Monday|2021Q3|Y|N|N|2459397|2459577|2459036|2459310|N|N|N|N|N| +2459402|AAAAAAAAKAHIFCAA|2021-07-06|1458|6341|487|2021|2|7|6|3|2021|487|6341|Tuesday|2021Q3|N|N|Y|2459397|2459577|2459037|2459311|N|N|N|N|N| +2459403|AAAAAAAALAHIFCAA|2021-07-07|1458|6341|487|2021|3|7|7|3|2021|487|6341|Wednesday|2021Q3|N|N|N|2459397|2459577|2459038|2459312|N|N|N|N|N| +2459404|AAAAAAAAMAHIFCAA|2021-07-08|1458|6341|487|2021|4|7|8|3|2021|487|6341|Thursday|2021Q3|N|N|N|2459397|2459577|2459039|2459313|N|N|N|N|N| +2459405|AAAAAAAANAHIFCAA|2021-07-09|1458|6341|487|2021|5|7|9|3|2021|487|6341|Friday|2021Q3|N|Y|N|2459397|2459577|2459040|2459314|N|N|N|N|N| +2459406|AAAAAAAAOAHIFCAA|2021-07-10|1458|6341|487|2021|6|7|10|3|2021|487|6341|Saturday|2021Q3|N|Y|N|2459397|2459577|2459041|2459315|N|N|N|N|N| +2459407|AAAAAAAAPAHIFCAA|2021-07-11|1458|6341|487|2021|0|7|11|3|2021|487|6341|Sunday|2021Q3|N|N|N|2459397|2459577|2459042|2459316|N|N|N|N|N| +2459408|AAAAAAAAABHIFCAA|2021-07-12|1458|6341|487|2021|1|7|12|3|2021|487|6341|Monday|2021Q3|N|N|N|2459397|2459577|2459043|2459317|N|N|N|N|N| +2459409|AAAAAAAABBHIFCAA|2021-07-13|1458|6342|487|2021|2|7|13|3|2021|487|6342|Tuesday|2021Q3|N|N|N|2459397|2459577|2459044|2459318|N|N|N|N|N| +2459410|AAAAAAAACBHIFCAA|2021-07-14|1458|6342|487|2021|3|7|14|3|2021|487|6342|Wednesday|2021Q3|N|N|N|2459397|2459577|2459045|2459319|N|N|N|N|N| +2459411|AAAAAAAADBHIFCAA|2021-07-15|1458|6342|487|2021|4|7|15|3|2021|487|6342|Thursday|2021Q3|N|N|N|2459397|2459577|2459046|2459320|N|N|N|N|N| +2459412|AAAAAAAAEBHIFCAA|2021-07-16|1458|6342|487|2021|5|7|16|3|2021|487|6342|Friday|2021Q3|N|Y|N|2459397|2459577|2459047|2459321|N|N|N|N|N| +2459413|AAAAAAAAFBHIFCAA|2021-07-17|1458|6342|487|2021|6|7|17|3|2021|487|6342|Saturday|2021Q3|N|Y|N|2459397|2459577|2459048|2459322|N|N|N|N|N| +2459414|AAAAAAAAGBHIFCAA|2021-07-18|1458|6342|487|2021|0|7|18|3|2021|487|6342|Sunday|2021Q3|N|N|N|2459397|2459577|2459049|2459323|N|N|N|N|N| +2459415|AAAAAAAAHBHIFCAA|2021-07-19|1458|6342|487|2021|1|7|19|3|2021|487|6342|Monday|2021Q3|N|N|N|2459397|2459577|2459050|2459324|N|N|N|N|N| +2459416|AAAAAAAAIBHIFCAA|2021-07-20|1458|6343|487|2021|2|7|20|3|2021|487|6343|Tuesday|2021Q3|N|N|N|2459397|2459577|2459051|2459325|N|N|N|N|N| +2459417|AAAAAAAAJBHIFCAA|2021-07-21|1458|6343|487|2021|3|7|21|3|2021|487|6343|Wednesday|2021Q3|N|N|N|2459397|2459577|2459052|2459326|N|N|N|N|N| +2459418|AAAAAAAAKBHIFCAA|2021-07-22|1458|6343|487|2021|4|7|22|3|2021|487|6343|Thursday|2021Q3|N|N|N|2459397|2459577|2459053|2459327|N|N|N|N|N| +2459419|AAAAAAAALBHIFCAA|2021-07-23|1458|6343|487|2021|5|7|23|3|2021|487|6343|Friday|2021Q3|N|Y|N|2459397|2459577|2459054|2459328|N|N|N|N|N| +2459420|AAAAAAAAMBHIFCAA|2021-07-24|1458|6343|487|2021|6|7|24|3|2021|487|6343|Saturday|2021Q3|N|Y|N|2459397|2459577|2459055|2459329|N|N|N|N|N| +2459421|AAAAAAAANBHIFCAA|2021-07-25|1458|6343|487|2021|0|7|25|3|2021|487|6343|Sunday|2021Q3|N|N|N|2459397|2459577|2459056|2459330|N|N|N|N|N| +2459422|AAAAAAAAOBHIFCAA|2021-07-26|1458|6343|487|2021|1|7|26|3|2021|487|6343|Monday|2021Q3|N|N|N|2459397|2459577|2459057|2459331|N|N|N|N|N| +2459423|AAAAAAAAPBHIFCAA|2021-07-27|1458|6344|487|2021|2|7|27|3|2021|487|6344|Tuesday|2021Q3|N|N|N|2459397|2459577|2459058|2459332|N|N|N|N|N| +2459424|AAAAAAAAACHIFCAA|2021-07-28|1458|6344|487|2021|3|7|28|3|2021|487|6344|Wednesday|2021Q3|N|N|N|2459397|2459577|2459059|2459333|N|N|N|N|N| +2459425|AAAAAAAABCHIFCAA|2021-07-29|1458|6344|487|2021|4|7|29|3|2021|487|6344|Thursday|2021Q3|N|N|N|2459397|2459577|2459060|2459334|N|N|N|N|N| +2459426|AAAAAAAACCHIFCAA|2021-07-30|1458|6344|487|2021|5|7|30|3|2021|487|6344|Friday|2021Q3|N|Y|N|2459397|2459577|2459061|2459335|N|N|N|N|N| +2459427|AAAAAAAADCHIFCAA|2021-07-31|1458|6344|487|2021|6|7|31|3|2021|487|6344|Saturday|2021Q3|N|Y|N|2459397|2459577|2459062|2459336|N|N|N|N|N| +2459428|AAAAAAAAECHIFCAA|2021-08-01|1459|6344|487|2021|0|8|1|3|2021|487|6344|Sunday|2021Q3|N|N|N|2459428|2459639|2459063|2459337|N|N|N|N|N| +2459429|AAAAAAAAFCHIFCAA|2021-08-02|1459|6344|487|2021|1|8|2|3|2021|487|6344|Monday|2021Q3|N|N|N|2459428|2459639|2459064|2459338|N|N|N|N|N| +2459430|AAAAAAAAGCHIFCAA|2021-08-03|1459|6345|487|2021|2|8|3|3|2021|487|6345|Tuesday|2021Q3|N|N|N|2459428|2459639|2459065|2459339|N|N|N|N|N| +2459431|AAAAAAAAHCHIFCAA|2021-08-04|1459|6345|487|2021|3|8|4|3|2021|487|6345|Wednesday|2021Q3|N|N|N|2459428|2459639|2459066|2459340|N|N|N|N|N| +2459432|AAAAAAAAICHIFCAA|2021-08-05|1459|6345|487|2021|4|8|5|3|2021|487|6345|Thursday|2021Q3|N|N|N|2459428|2459639|2459067|2459341|N|N|N|N|N| +2459433|AAAAAAAAJCHIFCAA|2021-08-06|1459|6345|487|2021|5|8|6|3|2021|487|6345|Friday|2021Q3|N|Y|N|2459428|2459639|2459068|2459342|N|N|N|N|N| +2459434|AAAAAAAAKCHIFCAA|2021-08-07|1459|6345|487|2021|6|8|7|3|2021|487|6345|Saturday|2021Q3|N|Y|N|2459428|2459639|2459069|2459343|N|N|N|N|N| +2459435|AAAAAAAALCHIFCAA|2021-08-08|1459|6345|487|2021|0|8|8|3|2021|487|6345|Sunday|2021Q3|N|N|N|2459428|2459639|2459070|2459344|N|N|N|N|N| +2459436|AAAAAAAAMCHIFCAA|2021-08-09|1459|6345|487|2021|1|8|9|3|2021|487|6345|Monday|2021Q3|N|N|N|2459428|2459639|2459071|2459345|N|N|N|N|N| +2459437|AAAAAAAANCHIFCAA|2021-08-10|1459|6346|487|2021|2|8|10|3|2021|487|6346|Tuesday|2021Q3|N|N|N|2459428|2459639|2459072|2459346|N|N|N|N|N| +2459438|AAAAAAAAOCHIFCAA|2021-08-11|1459|6346|487|2021|3|8|11|3|2021|487|6346|Wednesday|2021Q3|N|N|N|2459428|2459639|2459073|2459347|N|N|N|N|N| +2459439|AAAAAAAAPCHIFCAA|2021-08-12|1459|6346|487|2021|4|8|12|3|2021|487|6346|Thursday|2021Q3|N|N|N|2459428|2459639|2459074|2459348|N|N|N|N|N| +2459440|AAAAAAAAADHIFCAA|2021-08-13|1459|6346|487|2021|5|8|13|3|2021|487|6346|Friday|2021Q3|N|Y|N|2459428|2459639|2459075|2459349|N|N|N|N|N| +2459441|AAAAAAAABDHIFCAA|2021-08-14|1459|6346|487|2021|6|8|14|3|2021|487|6346|Saturday|2021Q3|N|Y|N|2459428|2459639|2459076|2459350|N|N|N|N|N| +2459442|AAAAAAAACDHIFCAA|2021-08-15|1459|6346|487|2021|0|8|15|3|2021|487|6346|Sunday|2021Q3|N|N|N|2459428|2459639|2459077|2459351|N|N|N|N|N| +2459443|AAAAAAAADDHIFCAA|2021-08-16|1459|6346|487|2021|1|8|16|3|2021|487|6346|Monday|2021Q3|N|N|N|2459428|2459639|2459078|2459352|N|N|N|N|N| +2459444|AAAAAAAAEDHIFCAA|2021-08-17|1459|6347|487|2021|2|8|17|3|2021|487|6347|Tuesday|2021Q3|N|N|N|2459428|2459639|2459079|2459353|N|N|N|N|N| +2459445|AAAAAAAAFDHIFCAA|2021-08-18|1459|6347|487|2021|3|8|18|3|2021|487|6347|Wednesday|2021Q3|N|N|N|2459428|2459639|2459080|2459354|N|N|N|N|N| +2459446|AAAAAAAAGDHIFCAA|2021-08-19|1459|6347|487|2021|4|8|19|3|2021|487|6347|Thursday|2021Q3|N|N|N|2459428|2459639|2459081|2459355|N|N|N|N|N| +2459447|AAAAAAAAHDHIFCAA|2021-08-20|1459|6347|487|2021|5|8|20|3|2021|487|6347|Friday|2021Q3|N|Y|N|2459428|2459639|2459082|2459356|N|N|N|N|N| +2459448|AAAAAAAAIDHIFCAA|2021-08-21|1459|6347|487|2021|6|8|21|3|2021|487|6347|Saturday|2021Q3|N|Y|N|2459428|2459639|2459083|2459357|N|N|N|N|N| +2459449|AAAAAAAAJDHIFCAA|2021-08-22|1459|6347|487|2021|0|8|22|3|2021|487|6347|Sunday|2021Q3|N|N|N|2459428|2459639|2459084|2459358|N|N|N|N|N| +2459450|AAAAAAAAKDHIFCAA|2021-08-23|1459|6347|487|2021|1|8|23|3|2021|487|6347|Monday|2021Q3|N|N|N|2459428|2459639|2459085|2459359|N|N|N|N|N| +2459451|AAAAAAAALDHIFCAA|2021-08-24|1459|6348|487|2021|2|8|24|3|2021|487|6348|Tuesday|2021Q3|N|N|N|2459428|2459639|2459086|2459360|N|N|N|N|N| +2459452|AAAAAAAAMDHIFCAA|2021-08-25|1459|6348|487|2021|3|8|25|3|2021|487|6348|Wednesday|2021Q3|N|N|N|2459428|2459639|2459087|2459361|N|N|N|N|N| +2459453|AAAAAAAANDHIFCAA|2021-08-26|1459|6348|487|2021|4|8|26|3|2021|487|6348|Thursday|2021Q3|N|N|N|2459428|2459639|2459088|2459362|N|N|N|N|N| +2459454|AAAAAAAAODHIFCAA|2021-08-27|1459|6348|487|2021|5|8|27|3|2021|487|6348|Friday|2021Q3|N|Y|N|2459428|2459639|2459089|2459363|N|N|N|N|N| +2459455|AAAAAAAAPDHIFCAA|2021-08-28|1459|6348|487|2021|6|8|28|3|2021|487|6348|Saturday|2021Q3|N|Y|N|2459428|2459639|2459090|2459364|N|N|N|N|N| +2459456|AAAAAAAAAEHIFCAA|2021-08-29|1459|6348|487|2021|0|8|29|3|2021|487|6348|Sunday|2021Q3|N|N|N|2459428|2459639|2459091|2459365|N|N|N|N|N| +2459457|AAAAAAAABEHIFCAA|2021-08-30|1459|6348|487|2021|1|8|30|3|2021|487|6348|Monday|2021Q3|N|N|N|2459428|2459639|2459092|2459366|N|N|N|N|N| +2459458|AAAAAAAACEHIFCAA|2021-08-31|1459|6349|487|2021|2|8|31|3|2021|487|6349|Tuesday|2021Q3|N|N|N|2459428|2459639|2459093|2459367|N|N|N|N|N| +2459459|AAAAAAAADEHIFCAA|2021-09-01|1460|6349|488|2021|3|9|1|3|2021|488|6349|Wednesday|2021Q3|N|N|N|2459459|2459701|2459094|2459368|N|N|N|N|N| +2459460|AAAAAAAAEEHIFCAA|2021-09-02|1460|6349|488|2021|4|9|2|3|2021|488|6349|Thursday|2021Q3|N|N|N|2459459|2459701|2459095|2459369|N|N|N|N|N| +2459461|AAAAAAAAFEHIFCAA|2021-09-03|1460|6349|488|2021|5|9|3|3|2021|488|6349|Friday|2021Q3|N|Y|N|2459459|2459701|2459096|2459370|N|N|N|N|N| +2459462|AAAAAAAAGEHIFCAA|2021-09-04|1460|6349|488|2021|6|9|4|3|2021|488|6349|Saturday|2021Q3|N|Y|N|2459459|2459701|2459097|2459371|N|N|N|N|N| +2459463|AAAAAAAAHEHIFCAA|2021-09-05|1460|6349|488|2021|0|9|5|3|2021|488|6349|Sunday|2021Q3|N|N|N|2459459|2459701|2459098|2459372|N|N|N|N|N| +2459464|AAAAAAAAIEHIFCAA|2021-09-06|1460|6349|488|2021|1|9|6|3|2021|488|6349|Monday|2021Q3|N|N|N|2459459|2459701|2459099|2459373|N|N|N|N|N| +2459465|AAAAAAAAJEHIFCAA|2021-09-07|1460|6350|488|2021|2|9|7|3|2021|488|6350|Tuesday|2021Q3|N|N|N|2459459|2459701|2459100|2459374|N|N|N|N|N| +2459466|AAAAAAAAKEHIFCAA|2021-09-08|1460|6350|488|2021|3|9|8|3|2021|488|6350|Wednesday|2021Q3|N|N|N|2459459|2459701|2459101|2459375|N|N|N|N|N| +2459467|AAAAAAAALEHIFCAA|2021-09-09|1460|6350|488|2021|4|9|9|3|2021|488|6350|Thursday|2021Q3|N|N|N|2459459|2459701|2459102|2459376|N|N|N|N|N| +2459468|AAAAAAAAMEHIFCAA|2021-09-10|1460|6350|488|2021|5|9|10|3|2021|488|6350|Friday|2021Q3|N|Y|N|2459459|2459701|2459103|2459377|N|N|N|N|N| +2459469|AAAAAAAANEHIFCAA|2021-09-11|1460|6350|488|2021|6|9|11|3|2021|488|6350|Saturday|2021Q3|N|Y|N|2459459|2459701|2459104|2459378|N|N|N|N|N| +2459470|AAAAAAAAOEHIFCAA|2021-09-12|1460|6350|488|2021|0|9|12|3|2021|488|6350|Sunday|2021Q3|N|N|N|2459459|2459701|2459105|2459379|N|N|N|N|N| +2459471|AAAAAAAAPEHIFCAA|2021-09-13|1460|6350|488|2021|1|9|13|3|2021|488|6350|Monday|2021Q3|N|N|N|2459459|2459701|2459106|2459380|N|N|N|N|N| +2459472|AAAAAAAAAFHIFCAA|2021-09-14|1460|6351|488|2021|2|9|14|3|2021|488|6351|Tuesday|2021Q3|N|N|N|2459459|2459701|2459107|2459381|N|N|N|N|N| +2459473|AAAAAAAABFHIFCAA|2021-09-15|1460|6351|488|2021|3|9|15|3|2021|488|6351|Wednesday|2021Q3|N|N|N|2459459|2459701|2459108|2459382|N|N|N|N|N| +2459474|AAAAAAAACFHIFCAA|2021-09-16|1460|6351|488|2021|4|9|16|3|2021|488|6351|Thursday|2021Q3|N|N|N|2459459|2459701|2459109|2459383|N|N|N|N|N| +2459475|AAAAAAAADFHIFCAA|2021-09-17|1460|6351|488|2021|5|9|17|3|2021|488|6351|Friday|2021Q3|N|Y|N|2459459|2459701|2459110|2459384|N|N|N|N|N| +2459476|AAAAAAAAEFHIFCAA|2021-09-18|1460|6351|488|2021|6|9|18|3|2021|488|6351|Saturday|2021Q3|N|Y|N|2459459|2459701|2459111|2459385|N|N|N|N|N| +2459477|AAAAAAAAFFHIFCAA|2021-09-19|1460|6351|488|2021|0|9|19|3|2021|488|6351|Sunday|2021Q3|N|N|N|2459459|2459701|2459112|2459386|N|N|N|N|N| +2459478|AAAAAAAAGFHIFCAA|2021-09-20|1460|6351|488|2021|1|9|20|3|2021|488|6351|Monday|2021Q3|N|N|N|2459459|2459701|2459113|2459387|N|N|N|N|N| +2459479|AAAAAAAAHFHIFCAA|2021-09-21|1460|6352|488|2021|2|9|21|3|2021|488|6352|Tuesday|2021Q3|N|N|N|2459459|2459701|2459114|2459388|N|N|N|N|N| +2459480|AAAAAAAAIFHIFCAA|2021-09-22|1460|6352|488|2021|3|9|22|3|2021|488|6352|Wednesday|2021Q3|N|N|N|2459459|2459701|2459115|2459389|N|N|N|N|N| +2459481|AAAAAAAAJFHIFCAA|2021-09-23|1460|6352|488|2021|4|9|23|3|2021|488|6352|Thursday|2021Q3|N|N|N|2459459|2459701|2459116|2459390|N|N|N|N|N| +2459482|AAAAAAAAKFHIFCAA|2021-09-24|1460|6352|488|2021|5|9|24|3|2021|488|6352|Friday|2021Q3|N|Y|N|2459459|2459701|2459117|2459391|N|N|N|N|N| +2459483|AAAAAAAALFHIFCAA|2021-09-25|1460|6352|488|2021|6|9|25|3|2021|488|6352|Saturday|2021Q3|N|Y|N|2459459|2459701|2459118|2459392|N|N|N|N|N| +2459484|AAAAAAAAMFHIFCAA|2021-09-26|1460|6352|488|2021|0|9|26|3|2021|488|6352|Sunday|2021Q3|N|N|N|2459459|2459701|2459119|2459393|N|N|N|N|N| +2459485|AAAAAAAANFHIFCAA|2021-09-27|1460|6352|488|2021|1|9|27|3|2021|488|6352|Monday|2021Q3|N|N|N|2459459|2459701|2459120|2459394|N|N|N|N|N| +2459486|AAAAAAAAOFHIFCAA|2021-09-28|1460|6353|488|2021|2|9|28|3|2021|488|6353|Tuesday|2021Q3|N|N|N|2459459|2459701|2459121|2459395|N|N|N|N|N| +2459487|AAAAAAAAPFHIFCAA|2021-09-29|1460|6353|488|2021|3|9|29|3|2021|488|6353|Wednesday|2021Q3|N|N|N|2459459|2459701|2459122|2459396|N|N|N|N|N| +2459488|AAAAAAAAAGHIFCAA|2021-09-30|1460|6353|488|2021|4|9|30|3|2021|488|6353|Thursday|2021Q3|N|N|N|2459459|2459701|2459123|2459397|N|N|N|N|N| +2459489|AAAAAAAABGHIFCAA|2021-10-01|1461|6353|488|2021|5|10|1|3|2021|488|6353|Friday|2021Q3|N|Y|N|2459489|2459761|2459124|2459397|N|N|N|N|N| +2459490|AAAAAAAACGHIFCAA|2021-10-02|1461|6353|488|2021|6|10|2|4|2021|488|6353|Saturday|2021Q4|N|Y|N|2459489|2459761|2459125|2459398|N|N|N|N|N| +2459491|AAAAAAAADGHIFCAA|2021-10-03|1461|6353|488|2021|0|10|3|4|2021|488|6353|Sunday|2021Q4|N|N|N|2459489|2459761|2459126|2459399|N|N|N|N|N| +2459492|AAAAAAAAEGHIFCAA|2021-10-04|1461|6353|488|2021|1|10|4|4|2021|488|6353|Monday|2021Q4|N|N|N|2459489|2459761|2459127|2459400|N|N|N|N|N| +2459493|AAAAAAAAFGHIFCAA|2021-10-05|1461|6354|488|2021|2|10|5|4|2021|488|6354|Tuesday|2021Q4|N|N|N|2459489|2459761|2459128|2459401|N|N|N|N|N| +2459494|AAAAAAAAGGHIFCAA|2021-10-06|1461|6354|488|2021|3|10|6|4|2021|488|6354|Wednesday|2021Q4|N|N|N|2459489|2459761|2459129|2459402|N|N|N|N|N| +2459495|AAAAAAAAHGHIFCAA|2021-10-07|1461|6354|488|2021|4|10|7|4|2021|488|6354|Thursday|2021Q4|N|N|N|2459489|2459761|2459130|2459403|N|N|N|N|N| +2459496|AAAAAAAAIGHIFCAA|2021-10-08|1461|6354|488|2021|5|10|8|4|2021|488|6354|Friday|2021Q4|N|Y|N|2459489|2459761|2459131|2459404|N|N|N|N|N| +2459497|AAAAAAAAJGHIFCAA|2021-10-09|1461|6354|488|2021|6|10|9|4|2021|488|6354|Saturday|2021Q4|N|Y|N|2459489|2459761|2459132|2459405|N|N|N|N|N| +2459498|AAAAAAAAKGHIFCAA|2021-10-10|1461|6354|488|2021|0|10|10|4|2021|488|6354|Sunday|2021Q4|N|N|N|2459489|2459761|2459133|2459406|N|N|N|N|N| +2459499|AAAAAAAALGHIFCAA|2021-10-11|1461|6354|488|2021|1|10|11|4|2021|488|6354|Monday|2021Q4|N|N|N|2459489|2459761|2459134|2459407|N|N|N|N|N| +2459500|AAAAAAAAMGHIFCAA|2021-10-12|1461|6355|488|2021|2|10|12|4|2021|488|6355|Tuesday|2021Q4|N|N|N|2459489|2459761|2459135|2459408|N|N|N|N|N| +2459501|AAAAAAAANGHIFCAA|2021-10-13|1461|6355|488|2021|3|10|13|4|2021|488|6355|Wednesday|2021Q4|N|N|N|2459489|2459761|2459136|2459409|N|N|N|N|N| +2459502|AAAAAAAAOGHIFCAA|2021-10-14|1461|6355|488|2021|4|10|14|4|2021|488|6355|Thursday|2021Q4|N|N|N|2459489|2459761|2459137|2459410|N|N|N|N|N| +2459503|AAAAAAAAPGHIFCAA|2021-10-15|1461|6355|488|2021|5|10|15|4|2021|488|6355|Friday|2021Q4|N|Y|N|2459489|2459761|2459138|2459411|N|N|N|N|N| +2459504|AAAAAAAAAHHIFCAA|2021-10-16|1461|6355|488|2021|6|10|16|4|2021|488|6355|Saturday|2021Q4|N|Y|N|2459489|2459761|2459139|2459412|N|N|N|N|N| +2459505|AAAAAAAABHHIFCAA|2021-10-17|1461|6355|488|2021|0|10|17|4|2021|488|6355|Sunday|2021Q4|N|N|N|2459489|2459761|2459140|2459413|N|N|N|N|N| +2459506|AAAAAAAACHHIFCAA|2021-10-18|1461|6355|488|2021|1|10|18|4|2021|488|6355|Monday|2021Q4|N|N|N|2459489|2459761|2459141|2459414|N|N|N|N|N| +2459507|AAAAAAAADHHIFCAA|2021-10-19|1461|6356|488|2021|2|10|19|4|2021|488|6356|Tuesday|2021Q4|N|N|N|2459489|2459761|2459142|2459415|N|N|N|N|N| +2459508|AAAAAAAAEHHIFCAA|2021-10-20|1461|6356|488|2021|3|10|20|4|2021|488|6356|Wednesday|2021Q4|N|N|N|2459489|2459761|2459143|2459416|N|N|N|N|N| +2459509|AAAAAAAAFHHIFCAA|2021-10-21|1461|6356|488|2021|4|10|21|4|2021|488|6356|Thursday|2021Q4|N|N|N|2459489|2459761|2459144|2459417|N|N|N|N|N| +2459510|AAAAAAAAGHHIFCAA|2021-10-22|1461|6356|488|2021|5|10|22|4|2021|488|6356|Friday|2021Q4|N|Y|N|2459489|2459761|2459145|2459418|N|N|N|N|N| +2459511|AAAAAAAAHHHIFCAA|2021-10-23|1461|6356|488|2021|6|10|23|4|2021|488|6356|Saturday|2021Q4|N|Y|N|2459489|2459761|2459146|2459419|N|N|N|N|N| +2459512|AAAAAAAAIHHIFCAA|2021-10-24|1461|6356|488|2021|0|10|24|4|2021|488|6356|Sunday|2021Q4|N|N|N|2459489|2459761|2459147|2459420|N|N|N|N|N| +2459513|AAAAAAAAJHHIFCAA|2021-10-25|1461|6356|488|2021|1|10|25|4|2021|488|6356|Monday|2021Q4|N|N|N|2459489|2459761|2459148|2459421|N|N|N|N|N| +2459514|AAAAAAAAKHHIFCAA|2021-10-26|1461|6357|488|2021|2|10|26|4|2021|488|6357|Tuesday|2021Q4|N|N|N|2459489|2459761|2459149|2459422|N|N|N|N|N| +2459515|AAAAAAAALHHIFCAA|2021-10-27|1461|6357|488|2021|3|10|27|4|2021|488|6357|Wednesday|2021Q4|N|N|N|2459489|2459761|2459150|2459423|N|N|N|N|N| +2459516|AAAAAAAAMHHIFCAA|2021-10-28|1461|6357|488|2021|4|10|28|4|2021|488|6357|Thursday|2021Q4|N|N|N|2459489|2459761|2459151|2459424|N|N|N|N|N| +2459517|AAAAAAAANHHIFCAA|2021-10-29|1461|6357|488|2021|5|10|29|4|2021|488|6357|Friday|2021Q4|N|Y|N|2459489|2459761|2459152|2459425|N|N|N|N|N| +2459518|AAAAAAAAOHHIFCAA|2021-10-30|1461|6357|488|2021|6|10|30|4|2021|488|6357|Saturday|2021Q4|N|Y|N|2459489|2459761|2459153|2459426|N|N|N|N|N| +2459519|AAAAAAAAPHHIFCAA|2021-10-31|1461|6357|488|2021|0|10|31|4|2021|488|6357|Sunday|2021Q4|N|N|N|2459489|2459761|2459154|2459427|N|N|N|N|N| +2459520|AAAAAAAAAIHIFCAA|2021-11-01|1462|6357|488|2021|1|11|1|4|2021|488|6357|Monday|2021Q4|N|N|N|2459520|2459823|2459155|2459428|N|N|N|N|N| +2459521|AAAAAAAABIHIFCAA|2021-11-02|1462|6358|488|2021|2|11|2|4|2021|488|6358|Tuesday|2021Q4|N|N|N|2459520|2459823|2459156|2459429|N|N|N|N|N| +2459522|AAAAAAAACIHIFCAA|2021-11-03|1462|6358|488|2021|3|11|3|4|2021|488|6358|Wednesday|2021Q4|N|N|N|2459520|2459823|2459157|2459430|N|N|N|N|N| +2459523|AAAAAAAADIHIFCAA|2021-11-04|1462|6358|488|2021|4|11|4|4|2021|488|6358|Thursday|2021Q4|N|N|N|2459520|2459823|2459158|2459431|N|N|N|N|N| +2459524|AAAAAAAAEIHIFCAA|2021-11-05|1462|6358|488|2021|5|11|5|4|2021|488|6358|Friday|2021Q4|N|Y|N|2459520|2459823|2459159|2459432|N|N|N|N|N| +2459525|AAAAAAAAFIHIFCAA|2021-11-06|1462|6358|488|2021|6|11|6|4|2021|488|6358|Saturday|2021Q4|N|Y|N|2459520|2459823|2459160|2459433|N|N|N|N|N| +2459526|AAAAAAAAGIHIFCAA|2021-11-07|1462|6358|488|2021|0|11|7|4|2021|488|6358|Sunday|2021Q4|N|N|N|2459520|2459823|2459161|2459434|N|N|N|N|N| +2459527|AAAAAAAAHIHIFCAA|2021-11-08|1462|6358|488|2021|1|11|8|4|2021|488|6358|Monday|2021Q4|N|N|N|2459520|2459823|2459162|2459435|N|N|N|N|N| +2459528|AAAAAAAAIIHIFCAA|2021-11-09|1462|6359|488|2021|2|11|9|4|2021|488|6359|Tuesday|2021Q4|N|N|N|2459520|2459823|2459163|2459436|N|N|N|N|N| +2459529|AAAAAAAAJIHIFCAA|2021-11-10|1462|6359|488|2021|3|11|10|4|2021|488|6359|Wednesday|2021Q4|N|N|N|2459520|2459823|2459164|2459437|N|N|N|N|N| +2459530|AAAAAAAAKIHIFCAA|2021-11-11|1462|6359|488|2021|4|11|11|4|2021|488|6359|Thursday|2021Q4|N|N|N|2459520|2459823|2459165|2459438|N|N|N|N|N| +2459531|AAAAAAAALIHIFCAA|2021-11-12|1462|6359|488|2021|5|11|12|4|2021|488|6359|Friday|2021Q4|N|Y|N|2459520|2459823|2459166|2459439|N|N|N|N|N| +2459532|AAAAAAAAMIHIFCAA|2021-11-13|1462|6359|488|2021|6|11|13|4|2021|488|6359|Saturday|2021Q4|N|Y|N|2459520|2459823|2459167|2459440|N|N|N|N|N| +2459533|AAAAAAAANIHIFCAA|2021-11-14|1462|6359|488|2021|0|11|14|4|2021|488|6359|Sunday|2021Q4|N|N|N|2459520|2459823|2459168|2459441|N|N|N|N|N| +2459534|AAAAAAAAOIHIFCAA|2021-11-15|1462|6359|488|2021|1|11|15|4|2021|488|6359|Monday|2021Q4|N|N|N|2459520|2459823|2459169|2459442|N|N|N|N|N| +2459535|AAAAAAAAPIHIFCAA|2021-11-16|1462|6360|488|2021|2|11|16|4|2021|488|6360|Tuesday|2021Q4|N|N|N|2459520|2459823|2459170|2459443|N|N|N|N|N| +2459536|AAAAAAAAAJHIFCAA|2021-11-17|1462|6360|488|2021|3|11|17|4|2021|488|6360|Wednesday|2021Q4|N|N|N|2459520|2459823|2459171|2459444|N|N|N|N|N| +2459537|AAAAAAAABJHIFCAA|2021-11-18|1462|6360|488|2021|4|11|18|4|2021|488|6360|Thursday|2021Q4|N|N|N|2459520|2459823|2459172|2459445|N|N|N|N|N| +2459538|AAAAAAAACJHIFCAA|2021-11-19|1462|6360|488|2021|5|11|19|4|2021|488|6360|Friday|2021Q4|N|Y|N|2459520|2459823|2459173|2459446|N|N|N|N|N| +2459539|AAAAAAAADJHIFCAA|2021-11-20|1462|6360|488|2021|6|11|20|4|2021|488|6360|Saturday|2021Q4|N|Y|N|2459520|2459823|2459174|2459447|N|N|N|N|N| +2459540|AAAAAAAAEJHIFCAA|2021-11-21|1462|6360|488|2021|0|11|21|4|2021|488|6360|Sunday|2021Q4|N|N|N|2459520|2459823|2459175|2459448|N|N|N|N|N| +2459541|AAAAAAAAFJHIFCAA|2021-11-22|1462|6360|488|2021|1|11|22|4|2021|488|6360|Monday|2021Q4|N|N|N|2459520|2459823|2459176|2459449|N|N|N|N|N| +2459542|AAAAAAAAGJHIFCAA|2021-11-23|1462|6361|488|2021|2|11|23|4|2021|488|6361|Tuesday|2021Q4|N|N|N|2459520|2459823|2459177|2459450|N|N|N|N|N| +2459543|AAAAAAAAHJHIFCAA|2021-11-24|1462|6361|488|2021|3|11|24|4|2021|488|6361|Wednesday|2021Q4|N|N|N|2459520|2459823|2459178|2459451|N|N|N|N|N| +2459544|AAAAAAAAIJHIFCAA|2021-11-25|1462|6361|488|2021|4|11|25|4|2021|488|6361|Thursday|2021Q4|N|N|N|2459520|2459823|2459179|2459452|N|N|N|N|N| +2459545|AAAAAAAAJJHIFCAA|2021-11-26|1462|6361|488|2021|5|11|26|4|2021|488|6361|Friday|2021Q4|N|Y|N|2459520|2459823|2459180|2459453|N|N|N|N|N| +2459546|AAAAAAAAKJHIFCAA|2021-11-27|1462|6361|488|2021|6|11|27|4|2021|488|6361|Saturday|2021Q4|N|Y|N|2459520|2459823|2459181|2459454|N|N|N|N|N| +2459547|AAAAAAAALJHIFCAA|2021-11-28|1462|6361|488|2021|0|11|28|4|2021|488|6361|Sunday|2021Q4|N|N|N|2459520|2459823|2459182|2459455|N|N|N|N|N| +2459548|AAAAAAAAMJHIFCAA|2021-11-29|1462|6361|488|2021|1|11|29|4|2021|488|6361|Monday|2021Q4|N|N|N|2459520|2459823|2459183|2459456|N|N|N|N|N| +2459549|AAAAAAAANJHIFCAA|2021-11-30|1462|6362|488|2021|2|11|30|4|2021|488|6362|Tuesday|2021Q4|N|N|N|2459520|2459823|2459184|2459457|N|N|N|N|N| +2459550|AAAAAAAAOJHIFCAA|2021-12-01|1463|6362|489|2021|3|12|1|4|2021|489|6362|Wednesday|2021Q4|N|N|N|2459550|2459883|2459185|2459458|N|N|N|N|N| +2459551|AAAAAAAAPJHIFCAA|2021-12-02|1463|6362|489|2021|4|12|2|4|2021|489|6362|Thursday|2021Q4|N|N|N|2459550|2459883|2459186|2459459|N|N|N|N|N| +2459552|AAAAAAAAAKHIFCAA|2021-12-03|1463|6362|489|2021|5|12|3|4|2021|489|6362|Friday|2021Q4|N|Y|N|2459550|2459883|2459187|2459460|N|N|N|N|N| +2459553|AAAAAAAABKHIFCAA|2021-12-04|1463|6362|489|2021|6|12|4|4|2021|489|6362|Saturday|2021Q4|N|Y|N|2459550|2459883|2459188|2459461|N|N|N|N|N| +2459554|AAAAAAAACKHIFCAA|2021-12-05|1463|6362|489|2021|0|12|5|4|2021|489|6362|Sunday|2021Q4|N|N|N|2459550|2459883|2459189|2459462|N|N|N|N|N| +2459555|AAAAAAAADKHIFCAA|2021-12-06|1463|6362|489|2021|1|12|6|4|2021|489|6362|Monday|2021Q4|N|N|N|2459550|2459883|2459190|2459463|N|N|N|N|N| +2459556|AAAAAAAAEKHIFCAA|2021-12-07|1463|6363|489|2021|2|12|7|4|2021|489|6363|Tuesday|2021Q4|N|N|N|2459550|2459883|2459191|2459464|N|N|N|N|N| +2459557|AAAAAAAAFKHIFCAA|2021-12-08|1463|6363|489|2021|3|12|8|4|2021|489|6363|Wednesday|2021Q4|N|N|N|2459550|2459883|2459192|2459465|N|N|N|N|N| +2459558|AAAAAAAAGKHIFCAA|2021-12-09|1463|6363|489|2021|4|12|9|4|2021|489|6363|Thursday|2021Q4|N|N|N|2459550|2459883|2459193|2459466|N|N|N|N|N| +2459559|AAAAAAAAHKHIFCAA|2021-12-10|1463|6363|489|2021|5|12|10|4|2021|489|6363|Friday|2021Q4|N|Y|N|2459550|2459883|2459194|2459467|N|N|N|N|N| +2459560|AAAAAAAAIKHIFCAA|2021-12-11|1463|6363|489|2021|6|12|11|4|2021|489|6363|Saturday|2021Q4|N|Y|N|2459550|2459883|2459195|2459468|N|N|N|N|N| +2459561|AAAAAAAAJKHIFCAA|2021-12-12|1463|6363|489|2021|0|12|12|4|2021|489|6363|Sunday|2021Q4|N|N|N|2459550|2459883|2459196|2459469|N|N|N|N|N| +2459562|AAAAAAAAKKHIFCAA|2021-12-13|1463|6363|489|2021|1|12|13|4|2021|489|6363|Monday|2021Q4|N|N|N|2459550|2459883|2459197|2459470|N|N|N|N|N| +2459563|AAAAAAAALKHIFCAA|2021-12-14|1463|6364|489|2021|2|12|14|4|2021|489|6364|Tuesday|2021Q4|N|N|N|2459550|2459883|2459198|2459471|N|N|N|N|N| +2459564|AAAAAAAAMKHIFCAA|2021-12-15|1463|6364|489|2021|3|12|15|4|2021|489|6364|Wednesday|2021Q4|N|N|N|2459550|2459883|2459199|2459472|N|N|N|N|N| +2459565|AAAAAAAANKHIFCAA|2021-12-16|1463|6364|489|2021|4|12|16|4|2021|489|6364|Thursday|2021Q4|N|N|N|2459550|2459883|2459200|2459473|N|N|N|N|N| +2459566|AAAAAAAAOKHIFCAA|2021-12-17|1463|6364|489|2021|5|12|17|4|2021|489|6364|Friday|2021Q4|N|Y|N|2459550|2459883|2459201|2459474|N|N|N|N|N| +2459567|AAAAAAAAPKHIFCAA|2021-12-18|1463|6364|489|2021|6|12|18|4|2021|489|6364|Saturday|2021Q4|N|Y|N|2459550|2459883|2459202|2459475|N|N|N|N|N| +2459568|AAAAAAAAALHIFCAA|2021-12-19|1463|6364|489|2021|0|12|19|4|2021|489|6364|Sunday|2021Q4|N|N|N|2459550|2459883|2459203|2459476|N|N|N|N|N| +2459569|AAAAAAAABLHIFCAA|2021-12-20|1463|6364|489|2021|1|12|20|4|2021|489|6364|Monday|2021Q4|N|N|N|2459550|2459883|2459204|2459477|N|N|N|N|N| +2459570|AAAAAAAACLHIFCAA|2021-12-21|1463|6365|489|2021|2|12|21|4|2021|489|6365|Tuesday|2021Q4|N|N|N|2459550|2459883|2459205|2459478|N|N|N|N|N| +2459571|AAAAAAAADLHIFCAA|2021-12-22|1463|6365|489|2021|3|12|22|4|2021|489|6365|Wednesday|2021Q4|N|N|N|2459550|2459883|2459206|2459479|N|N|N|N|N| +2459572|AAAAAAAAELHIFCAA|2021-12-23|1463|6365|489|2021|4|12|23|4|2021|489|6365|Thursday|2021Q4|N|N|N|2459550|2459883|2459207|2459480|N|N|N|N|N| +2459573|AAAAAAAAFLHIFCAA|2021-12-24|1463|6365|489|2021|5|12|24|4|2021|489|6365|Friday|2021Q4|N|Y|N|2459550|2459883|2459208|2459481|N|N|N|N|N| +2459574|AAAAAAAAGLHIFCAA|2021-12-25|1463|6365|489|2021|6|12|25|4|2021|489|6365|Saturday|2021Q4|N|Y|N|2459550|2459883|2459209|2459482|N|N|N|N|N| +2459575|AAAAAAAAHLHIFCAA|2021-12-26|1463|6365|489|2021|0|12|26|4|2021|489|6365|Sunday|2021Q4|Y|N|N|2459550|2459883|2459210|2459483|N|N|N|N|N| +2459576|AAAAAAAAILHIFCAA|2021-12-27|1463|6365|489|2021|1|12|27|4|2021|489|6365|Monday|2021Q4|N|N|Y|2459550|2459883|2459211|2459484|N|N|N|N|N| +2459577|AAAAAAAAJLHIFCAA|2021-12-28|1463|6366|489|2021|2|12|28|4|2021|489|6366|Tuesday|2021Q4|N|N|N|2459550|2459883|2459212|2459485|N|N|N|N|N| +2459578|AAAAAAAAKLHIFCAA|2021-12-29|1463|6366|489|2021|3|12|29|4|2021|489|6366|Wednesday|2021Q4|N|N|N|2459550|2459883|2459213|2459486|N|N|N|N|N| +2459579|AAAAAAAALLHIFCAA|2021-12-30|1463|6366|489|2021|4|12|30|4|2021|489|6366|Thursday|2021Q4|N|N|N|2459550|2459883|2459214|2459487|N|N|N|N|N| +2459580|AAAAAAAAMLHIFCAA|2021-12-31|1463|6366|489|2021|5|12|31|4|2021|489|6366|Friday|2021Q4|N|Y|N|2459550|2459883|2459215|2459488|N|N|N|N|N| +2459581|AAAAAAAANLHIFCAA|2022-01-01|1464|6366|489|2022|6|1|1|1|2022|489|6366|Saturday|2022Q1|Y|Y|N|2459581|2459580|2459216|2459489|N|N|N|N|N| +2459582|AAAAAAAAOLHIFCAA|2022-01-02|1464|6366|489|2022|0|1|2|1|2022|489|6366|Sunday|2022Q1|N|N|Y|2459581|2459580|2459217|2459490|N|N|N|N|N| +2459583|AAAAAAAAPLHIFCAA|2022-01-03|1464|6366|489|2022|1|1|3|1|2022|489|6366|Monday|2022Q1|N|N|N|2459581|2459580|2459218|2459491|N|N|N|N|N| +2459584|AAAAAAAAAMHIFCAA|2022-01-04|1464|6367|489|2022|2|1|4|1|2022|489|6367|Tuesday|2022Q1|N|N|N|2459581|2459580|2459219|2459492|N|N|N|N|N| +2459585|AAAAAAAABMHIFCAA|2022-01-05|1464|6367|489|2022|3|1|5|1|2022|489|6367|Wednesday|2022Q1|N|N|N|2459581|2459580|2459220|2459493|N|N|N|N|N| +2459586|AAAAAAAACMHIFCAA|2022-01-06|1464|6367|489|2022|4|1|6|1|2022|489|6367|Thursday|2022Q1|N|N|N|2459581|2459580|2459221|2459494|N|N|N|N|N| +2459587|AAAAAAAADMHIFCAA|2022-01-07|1464|6367|489|2022|5|1|7|1|2022|489|6367|Friday|2022Q1|N|Y|N|2459581|2459580|2459222|2459495|N|N|N|N|N| +2459588|AAAAAAAAEMHIFCAA|2022-01-08|1464|6367|489|2022|6|1|8|1|2022|489|6367|Saturday|2022Q1|N|Y|N|2459581|2459580|2459223|2459496|N|N|N|N|N| +2459589|AAAAAAAAFMHIFCAA|2022-01-09|1464|6367|489|2022|0|1|9|1|2022|489|6367|Sunday|2022Q1|N|N|N|2459581|2459580|2459224|2459497|N|N|N|N|N| +2459590|AAAAAAAAGMHIFCAA|2022-01-10|1464|6367|489|2022|1|1|10|1|2022|489|6367|Monday|2022Q1|N|N|N|2459581|2459580|2459225|2459498|N|N|N|N|N| +2459591|AAAAAAAAHMHIFCAA|2022-01-11|1464|6368|489|2022|2|1|11|1|2022|489|6368|Tuesday|2022Q1|N|N|N|2459581|2459580|2459226|2459499|N|N|N|N|N| +2459592|AAAAAAAAIMHIFCAA|2022-01-12|1464|6368|489|2022|3|1|12|1|2022|489|6368|Wednesday|2022Q1|N|N|N|2459581|2459580|2459227|2459500|N|N|N|N|N| +2459593|AAAAAAAAJMHIFCAA|2022-01-13|1464|6368|489|2022|4|1|13|1|2022|489|6368|Thursday|2022Q1|N|N|N|2459581|2459580|2459228|2459501|N|N|N|N|N| +2459594|AAAAAAAAKMHIFCAA|2022-01-14|1464|6368|489|2022|5|1|14|1|2022|489|6368|Friday|2022Q1|N|Y|N|2459581|2459580|2459229|2459502|N|N|N|N|N| +2459595|AAAAAAAALMHIFCAA|2022-01-15|1464|6368|489|2022|6|1|15|1|2022|489|6368|Saturday|2022Q1|N|Y|N|2459581|2459580|2459230|2459503|N|N|N|N|N| +2459596|AAAAAAAAMMHIFCAA|2022-01-16|1464|6368|489|2022|0|1|16|1|2022|489|6368|Sunday|2022Q1|N|N|N|2459581|2459580|2459231|2459504|N|N|N|N|N| +2459597|AAAAAAAANMHIFCAA|2022-01-17|1464|6368|489|2022|1|1|17|1|2022|489|6368|Monday|2022Q1|N|N|N|2459581|2459580|2459232|2459505|N|N|N|N|N| +2459598|AAAAAAAAOMHIFCAA|2022-01-18|1464|6369|489|2022|2|1|18|1|2022|489|6369|Tuesday|2022Q1|N|N|N|2459581|2459580|2459233|2459506|N|N|N|N|N| +2459599|AAAAAAAAPMHIFCAA|2022-01-19|1464|6369|489|2022|3|1|19|1|2022|489|6369|Wednesday|2022Q1|N|N|N|2459581|2459580|2459234|2459507|N|N|N|N|N| +2459600|AAAAAAAAANHIFCAA|2022-01-20|1464|6369|489|2022|4|1|20|1|2022|489|6369|Thursday|2022Q1|N|N|N|2459581|2459580|2459235|2459508|N|N|N|N|N| +2459601|AAAAAAAABNHIFCAA|2022-01-21|1464|6369|489|2022|5|1|21|1|2022|489|6369|Friday|2022Q1|N|Y|N|2459581|2459580|2459236|2459509|N|N|N|N|N| +2459602|AAAAAAAACNHIFCAA|2022-01-22|1464|6369|489|2022|6|1|22|1|2022|489|6369|Saturday|2022Q1|N|Y|N|2459581|2459580|2459237|2459510|N|N|N|N|N| +2459603|AAAAAAAADNHIFCAA|2022-01-23|1464|6369|489|2022|0|1|23|1|2022|489|6369|Sunday|2022Q1|N|N|N|2459581|2459580|2459238|2459511|N|N|N|N|N| +2459604|AAAAAAAAENHIFCAA|2022-01-24|1464|6369|489|2022|1|1|24|1|2022|489|6369|Monday|2022Q1|N|N|N|2459581|2459580|2459239|2459512|N|N|N|N|N| +2459605|AAAAAAAAFNHIFCAA|2022-01-25|1464|6370|489|2022|2|1|25|1|2022|489|6370|Tuesday|2022Q1|N|N|N|2459581|2459580|2459240|2459513|N|N|N|N|N| +2459606|AAAAAAAAGNHIFCAA|2022-01-26|1464|6370|489|2022|3|1|26|1|2022|489|6370|Wednesday|2022Q1|N|N|N|2459581|2459580|2459241|2459514|N|N|N|N|N| +2459607|AAAAAAAAHNHIFCAA|2022-01-27|1464|6370|489|2022|4|1|27|1|2022|489|6370|Thursday|2022Q1|N|N|N|2459581|2459580|2459242|2459515|N|N|N|N|N| +2459608|AAAAAAAAINHIFCAA|2022-01-28|1464|6370|489|2022|5|1|28|1|2022|489|6370|Friday|2022Q1|N|Y|N|2459581|2459580|2459243|2459516|N|N|N|N|N| +2459609|AAAAAAAAJNHIFCAA|2022-01-29|1464|6370|489|2022|6|1|29|1|2022|489|6370|Saturday|2022Q1|N|Y|N|2459581|2459580|2459244|2459517|N|N|N|N|N| +2459610|AAAAAAAAKNHIFCAA|2022-01-30|1464|6370|489|2022|0|1|30|1|2022|489|6370|Sunday|2022Q1|N|N|N|2459581|2459580|2459245|2459518|N|N|N|N|N| +2459611|AAAAAAAALNHIFCAA|2022-01-31|1464|6370|489|2022|1|1|31|1|2022|489|6370|Monday|2022Q1|N|N|N|2459581|2459580|2459246|2459519|N|N|N|N|N| +2459612|AAAAAAAAMNHIFCAA|2022-02-01|1465|6371|489|2022|2|2|1|1|2022|489|6371|Tuesday|2022Q1|N|N|N|2459612|2459642|2459247|2459520|N|N|N|N|N| +2459613|AAAAAAAANNHIFCAA|2022-02-02|1465|6371|489|2022|3|2|2|1|2022|489|6371|Wednesday|2022Q1|N|N|N|2459612|2459642|2459248|2459521|N|N|N|N|N| +2459614|AAAAAAAAONHIFCAA|2022-02-03|1465|6371|489|2022|4|2|3|1|2022|489|6371|Thursday|2022Q1|N|N|N|2459612|2459642|2459249|2459522|N|N|N|N|N| +2459615|AAAAAAAAPNHIFCAA|2022-02-04|1465|6371|489|2022|5|2|4|1|2022|489|6371|Friday|2022Q1|N|Y|N|2459612|2459642|2459250|2459523|N|N|N|N|N| +2459616|AAAAAAAAAOHIFCAA|2022-02-05|1465|6371|489|2022|6|2|5|1|2022|489|6371|Saturday|2022Q1|N|Y|N|2459612|2459642|2459251|2459524|N|N|N|N|N| +2459617|AAAAAAAABOHIFCAA|2022-02-06|1465|6371|489|2022|0|2|6|1|2022|489|6371|Sunday|2022Q1|N|N|N|2459612|2459642|2459252|2459525|N|N|N|N|N| +2459618|AAAAAAAACOHIFCAA|2022-02-07|1465|6371|489|2022|1|2|7|1|2022|489|6371|Monday|2022Q1|N|N|N|2459612|2459642|2459253|2459526|N|N|N|N|N| +2459619|AAAAAAAADOHIFCAA|2022-02-08|1465|6372|489|2022|2|2|8|1|2022|489|6372|Tuesday|2022Q1|N|N|N|2459612|2459642|2459254|2459527|N|N|N|N|N| +2459620|AAAAAAAAEOHIFCAA|2022-02-09|1465|6372|489|2022|3|2|9|1|2022|489|6372|Wednesday|2022Q1|N|N|N|2459612|2459642|2459255|2459528|N|N|N|N|N| +2459621|AAAAAAAAFOHIFCAA|2022-02-10|1465|6372|489|2022|4|2|10|1|2022|489|6372|Thursday|2022Q1|N|N|N|2459612|2459642|2459256|2459529|N|N|N|N|N| +2459622|AAAAAAAAGOHIFCAA|2022-02-11|1465|6372|489|2022|5|2|11|1|2022|489|6372|Friday|2022Q1|N|Y|N|2459612|2459642|2459257|2459530|N|N|N|N|N| +2459623|AAAAAAAAHOHIFCAA|2022-02-12|1465|6372|489|2022|6|2|12|1|2022|489|6372|Saturday|2022Q1|N|Y|N|2459612|2459642|2459258|2459531|N|N|N|N|N| +2459624|AAAAAAAAIOHIFCAA|2022-02-13|1465|6372|489|2022|0|2|13|1|2022|489|6372|Sunday|2022Q1|N|N|N|2459612|2459642|2459259|2459532|N|N|N|N|N| +2459625|AAAAAAAAJOHIFCAA|2022-02-14|1465|6372|489|2022|1|2|14|1|2022|489|6372|Monday|2022Q1|N|N|N|2459612|2459642|2459260|2459533|N|N|N|N|N| +2459626|AAAAAAAAKOHIFCAA|2022-02-15|1465|6373|489|2022|2|2|15|1|2022|489|6373|Tuesday|2022Q1|N|N|N|2459612|2459642|2459261|2459534|N|N|N|N|N| +2459627|AAAAAAAALOHIFCAA|2022-02-16|1465|6373|489|2022|3|2|16|1|2022|489|6373|Wednesday|2022Q1|N|N|N|2459612|2459642|2459262|2459535|N|N|N|N|N| +2459628|AAAAAAAAMOHIFCAA|2022-02-17|1465|6373|489|2022|4|2|17|1|2022|489|6373|Thursday|2022Q1|N|N|N|2459612|2459642|2459263|2459536|N|N|N|N|N| +2459629|AAAAAAAANOHIFCAA|2022-02-18|1465|6373|489|2022|5|2|18|1|2022|489|6373|Friday|2022Q1|N|Y|N|2459612|2459642|2459264|2459537|N|N|N|N|N| +2459630|AAAAAAAAOOHIFCAA|2022-02-19|1465|6373|489|2022|6|2|19|1|2022|489|6373|Saturday|2022Q1|N|Y|N|2459612|2459642|2459265|2459538|N|N|N|N|N| +2459631|AAAAAAAAPOHIFCAA|2022-02-20|1465|6373|489|2022|0|2|20|1|2022|489|6373|Sunday|2022Q1|N|N|N|2459612|2459642|2459266|2459539|N|N|N|N|N| +2459632|AAAAAAAAAPHIFCAA|2022-02-21|1465|6373|489|2022|1|2|21|1|2022|489|6373|Monday|2022Q1|N|N|N|2459612|2459642|2459267|2459540|N|N|N|N|N| +2459633|AAAAAAAABPHIFCAA|2022-02-22|1465|6374|489|2022|2|2|22|1|2022|489|6374|Tuesday|2022Q1|N|N|N|2459612|2459642|2459268|2459541|N|N|N|N|N| +2459634|AAAAAAAACPHIFCAA|2022-02-23|1465|6374|489|2022|3|2|23|1|2022|489|6374|Wednesday|2022Q1|N|N|N|2459612|2459642|2459269|2459542|N|N|N|N|N| +2459635|AAAAAAAADPHIFCAA|2022-02-24|1465|6374|489|2022|4|2|24|1|2022|489|6374|Thursday|2022Q1|N|N|N|2459612|2459642|2459270|2459543|N|N|N|N|N| +2459636|AAAAAAAAEPHIFCAA|2022-02-25|1465|6374|489|2022|5|2|25|1|2022|489|6374|Friday|2022Q1|N|Y|N|2459612|2459642|2459271|2459544|N|N|N|N|N| +2459637|AAAAAAAAFPHIFCAA|2022-02-26|1465|6374|489|2022|6|2|26|1|2022|489|6374|Saturday|2022Q1|N|Y|N|2459612|2459642|2459272|2459545|N|N|N|N|N| +2459638|AAAAAAAAGPHIFCAA|2022-02-27|1465|6374|489|2022|0|2|27|1|2022|489|6374|Sunday|2022Q1|N|N|N|2459612|2459642|2459273|2459546|N|N|N|N|N| +2459639|AAAAAAAAHPHIFCAA|2022-02-28|1465|6374|489|2022|1|2|28|1|2022|489|6374|Monday|2022Q1|N|N|N|2459612|2459642|2459274|2459547|N|N|N|N|N| +2459640|AAAAAAAAIPHIFCAA|2022-03-01|1466|6375|490|2022|2|3|1|1|2022|490|6375|Tuesday|2022Q1|N|N|N|2459640|2459698|2459275|2459548|N|N|N|N|N| +2459641|AAAAAAAAJPHIFCAA|2022-03-02|1466|6375|490|2022|3|3|2|1|2022|490|6375|Wednesday|2022Q1|N|N|N|2459640|2459698|2459276|2459549|N|N|N|N|N| +2459642|AAAAAAAAKPHIFCAA|2022-03-03|1466|6375|490|2022|4|3|3|1|2022|490|6375|Thursday|2022Q1|N|N|N|2459640|2459698|2459277|2459550|N|N|N|N|N| +2459643|AAAAAAAALPHIFCAA|2022-03-04|1466|6375|490|2022|5|3|4|1|2022|490|6375|Friday|2022Q1|N|Y|N|2459640|2459698|2459278|2459551|N|N|N|N|N| +2459644|AAAAAAAAMPHIFCAA|2022-03-05|1466|6375|490|2022|6|3|5|1|2022|490|6375|Saturday|2022Q1|N|Y|N|2459640|2459698|2459279|2459552|N|N|N|N|N| +2459645|AAAAAAAANPHIFCAA|2022-03-06|1466|6375|490|2022|0|3|6|1|2022|490|6375|Sunday|2022Q1|N|N|N|2459640|2459698|2459280|2459553|N|N|N|N|N| +2459646|AAAAAAAAOPHIFCAA|2022-03-07|1466|6375|490|2022|1|3|7|1|2022|490|6375|Monday|2022Q1|N|N|N|2459640|2459698|2459281|2459554|N|N|N|N|N| +2459647|AAAAAAAAPPHIFCAA|2022-03-08|1466|6376|490|2022|2|3|8|1|2022|490|6376|Tuesday|2022Q1|N|N|N|2459640|2459698|2459282|2459555|N|N|N|N|N| +2459648|AAAAAAAAAAIIFCAA|2022-03-09|1466|6376|490|2022|3|3|9|1|2022|490|6376|Wednesday|2022Q1|N|N|N|2459640|2459698|2459283|2459556|N|N|N|N|N| +2459649|AAAAAAAABAIIFCAA|2022-03-10|1466|6376|490|2022|4|3|10|1|2022|490|6376|Thursday|2022Q1|N|N|N|2459640|2459698|2459284|2459557|N|N|N|N|N| +2459650|AAAAAAAACAIIFCAA|2022-03-11|1466|6376|490|2022|5|3|11|1|2022|490|6376|Friday|2022Q1|N|Y|N|2459640|2459698|2459285|2459558|N|N|N|N|N| +2459651|AAAAAAAADAIIFCAA|2022-03-12|1466|6376|490|2022|6|3|12|1|2022|490|6376|Saturday|2022Q1|N|Y|N|2459640|2459698|2459286|2459559|N|N|N|N|N| +2459652|AAAAAAAAEAIIFCAA|2022-03-13|1466|6376|490|2022|0|3|13|1|2022|490|6376|Sunday|2022Q1|N|N|N|2459640|2459698|2459287|2459560|N|N|N|N|N| +2459653|AAAAAAAAFAIIFCAA|2022-03-14|1466|6376|490|2022|1|3|14|1|2022|490|6376|Monday|2022Q1|N|N|N|2459640|2459698|2459288|2459561|N|N|N|N|N| +2459654|AAAAAAAAGAIIFCAA|2022-03-15|1466|6377|490|2022|2|3|15|1|2022|490|6377|Tuesday|2022Q1|N|N|N|2459640|2459698|2459289|2459562|N|N|N|N|N| +2459655|AAAAAAAAHAIIFCAA|2022-03-16|1466|6377|490|2022|3|3|16|1|2022|490|6377|Wednesday|2022Q1|N|N|N|2459640|2459698|2459290|2459563|N|N|N|N|N| +2459656|AAAAAAAAIAIIFCAA|2022-03-17|1466|6377|490|2022|4|3|17|1|2022|490|6377|Thursday|2022Q1|N|N|N|2459640|2459698|2459291|2459564|N|N|N|N|N| +2459657|AAAAAAAAJAIIFCAA|2022-03-18|1466|6377|490|2022|5|3|18|1|2022|490|6377|Friday|2022Q1|N|Y|N|2459640|2459698|2459292|2459565|N|N|N|N|N| +2459658|AAAAAAAAKAIIFCAA|2022-03-19|1466|6377|490|2022|6|3|19|1|2022|490|6377|Saturday|2022Q1|N|Y|N|2459640|2459698|2459293|2459566|N|N|N|N|N| +2459659|AAAAAAAALAIIFCAA|2022-03-20|1466|6377|490|2022|0|3|20|1|2022|490|6377|Sunday|2022Q1|N|N|N|2459640|2459698|2459294|2459567|N|N|N|N|N| +2459660|AAAAAAAAMAIIFCAA|2022-03-21|1466|6377|490|2022|1|3|21|1|2022|490|6377|Monday|2022Q1|N|N|N|2459640|2459698|2459295|2459568|N|N|N|N|N| +2459661|AAAAAAAANAIIFCAA|2022-03-22|1466|6378|490|2022|2|3|22|1|2022|490|6378|Tuesday|2022Q1|N|N|N|2459640|2459698|2459296|2459569|N|N|N|N|N| +2459662|AAAAAAAAOAIIFCAA|2022-03-23|1466|6378|490|2022|3|3|23|1|2022|490|6378|Wednesday|2022Q1|N|N|N|2459640|2459698|2459297|2459570|N|N|N|N|N| +2459663|AAAAAAAAPAIIFCAA|2022-03-24|1466|6378|490|2022|4|3|24|1|2022|490|6378|Thursday|2022Q1|N|N|N|2459640|2459698|2459298|2459571|N|N|N|N|N| +2459664|AAAAAAAAABIIFCAA|2022-03-25|1466|6378|490|2022|5|3|25|1|2022|490|6378|Friday|2022Q1|N|Y|N|2459640|2459698|2459299|2459572|N|N|N|N|N| +2459665|AAAAAAAABBIIFCAA|2022-03-26|1466|6378|490|2022|6|3|26|1|2022|490|6378|Saturday|2022Q1|N|Y|N|2459640|2459698|2459300|2459573|N|N|N|N|N| +2459666|AAAAAAAACBIIFCAA|2022-03-27|1466|6378|490|2022|0|3|27|1|2022|490|6378|Sunday|2022Q1|N|N|N|2459640|2459698|2459301|2459574|N|N|N|N|N| +2459667|AAAAAAAADBIIFCAA|2022-03-28|1466|6378|490|2022|1|3|28|1|2022|490|6378|Monday|2022Q1|N|N|N|2459640|2459698|2459302|2459575|N|N|N|N|N| +2459668|AAAAAAAAEBIIFCAA|2022-03-29|1466|6379|490|2022|2|3|29|1|2022|490|6379|Tuesday|2022Q1|N|N|N|2459640|2459698|2459303|2459576|N|N|N|N|N| +2459669|AAAAAAAAFBIIFCAA|2022-03-30|1466|6379|490|2022|3|3|30|1|2022|490|6379|Wednesday|2022Q1|N|N|N|2459640|2459698|2459304|2459577|N|N|N|N|N| +2459670|AAAAAAAAGBIIFCAA|2022-03-31|1466|6379|490|2022|4|3|31|1|2022|490|6379|Thursday|2022Q1|N|N|N|2459640|2459698|2459305|2459578|N|N|N|N|N| +2459671|AAAAAAAAHBIIFCAA|2022-04-01|1467|6379|490|2022|5|4|1|1|2022|490|6379|Friday|2022Q1|N|Y|N|2459671|2459760|2459306|2459581|N|N|N|N|N| +2459672|AAAAAAAAIBIIFCAA|2022-04-02|1467|6379|490|2022|6|4|2|2|2022|490|6379|Saturday|2022Q2|N|Y|N|2459671|2459760|2459307|2459582|N|N|N|N|N| +2459673|AAAAAAAAJBIIFCAA|2022-04-03|1467|6379|490|2022|0|4|3|2|2022|490|6379|Sunday|2022Q2|N|N|N|2459671|2459760|2459308|2459583|N|N|N|N|N| +2459674|AAAAAAAAKBIIFCAA|2022-04-04|1467|6379|490|2022|1|4|4|2|2022|490|6379|Monday|2022Q2|N|N|N|2459671|2459760|2459309|2459584|N|N|N|N|N| +2459675|AAAAAAAALBIIFCAA|2022-04-05|1467|6380|490|2022|2|4|5|2|2022|490|6380|Tuesday|2022Q2|N|N|N|2459671|2459760|2459310|2459585|N|N|N|N|N| +2459676|AAAAAAAAMBIIFCAA|2022-04-06|1467|6380|490|2022|3|4|6|2|2022|490|6380|Wednesday|2022Q2|N|N|N|2459671|2459760|2459311|2459586|N|N|N|N|N| +2459677|AAAAAAAANBIIFCAA|2022-04-07|1467|6380|490|2022|4|4|7|2|2022|490|6380|Thursday|2022Q2|N|N|N|2459671|2459760|2459312|2459587|N|N|N|N|N| +2459678|AAAAAAAAOBIIFCAA|2022-04-08|1467|6380|490|2022|5|4|8|2|2022|490|6380|Friday|2022Q2|N|Y|N|2459671|2459760|2459313|2459588|N|N|N|N|N| +2459679|AAAAAAAAPBIIFCAA|2022-04-09|1467|6380|490|2022|6|4|9|2|2022|490|6380|Saturday|2022Q2|N|Y|N|2459671|2459760|2459314|2459589|N|N|N|N|N| +2459680|AAAAAAAAACIIFCAA|2022-04-10|1467|6380|490|2022|0|4|10|2|2022|490|6380|Sunday|2022Q2|N|N|N|2459671|2459760|2459315|2459590|N|N|N|N|N| +2459681|AAAAAAAABCIIFCAA|2022-04-11|1467|6380|490|2022|1|4|11|2|2022|490|6380|Monday|2022Q2|N|N|N|2459671|2459760|2459316|2459591|N|N|N|N|N| +2459682|AAAAAAAACCIIFCAA|2022-04-12|1467|6381|490|2022|2|4|12|2|2022|490|6381|Tuesday|2022Q2|N|N|N|2459671|2459760|2459317|2459592|N|N|N|N|N| +2459683|AAAAAAAADCIIFCAA|2022-04-13|1467|6381|490|2022|3|4|13|2|2022|490|6381|Wednesday|2022Q2|N|N|N|2459671|2459760|2459318|2459593|N|N|N|N|N| +2459684|AAAAAAAAECIIFCAA|2022-04-14|1467|6381|490|2022|4|4|14|2|2022|490|6381|Thursday|2022Q2|N|N|N|2459671|2459760|2459319|2459594|N|N|N|N|N| +2459685|AAAAAAAAFCIIFCAA|2022-04-15|1467|6381|490|2022|5|4|15|2|2022|490|6381|Friday|2022Q2|N|Y|N|2459671|2459760|2459320|2459595|N|N|N|N|N| +2459686|AAAAAAAAGCIIFCAA|2022-04-16|1467|6381|490|2022|6|4|16|2|2022|490|6381|Saturday|2022Q2|N|Y|N|2459671|2459760|2459321|2459596|N|N|N|N|N| +2459687|AAAAAAAAHCIIFCAA|2022-04-17|1467|6381|490|2022|0|4|17|2|2022|490|6381|Sunday|2022Q2|N|N|N|2459671|2459760|2459322|2459597|N|N|N|N|N| +2459688|AAAAAAAAICIIFCAA|2022-04-18|1467|6381|490|2022|1|4|18|2|2022|490|6381|Monday|2022Q2|N|N|N|2459671|2459760|2459323|2459598|N|N|N|N|N| +2459689|AAAAAAAAJCIIFCAA|2022-04-19|1467|6382|490|2022|2|4|19|2|2022|490|6382|Tuesday|2022Q2|N|N|N|2459671|2459760|2459324|2459599|N|N|N|N|N| +2459690|AAAAAAAAKCIIFCAA|2022-04-20|1467|6382|490|2022|3|4|20|2|2022|490|6382|Wednesday|2022Q2|N|N|N|2459671|2459760|2459325|2459600|N|N|N|N|N| +2459691|AAAAAAAALCIIFCAA|2022-04-21|1467|6382|490|2022|4|4|21|2|2022|490|6382|Thursday|2022Q2|N|N|N|2459671|2459760|2459326|2459601|N|N|N|N|N| +2459692|AAAAAAAAMCIIFCAA|2022-04-22|1467|6382|490|2022|5|4|22|2|2022|490|6382|Friday|2022Q2|N|Y|N|2459671|2459760|2459327|2459602|N|N|N|N|N| +2459693|AAAAAAAANCIIFCAA|2022-04-23|1467|6382|490|2022|6|4|23|2|2022|490|6382|Saturday|2022Q2|N|Y|N|2459671|2459760|2459328|2459603|N|N|N|N|N| +2459694|AAAAAAAAOCIIFCAA|2022-04-24|1467|6382|490|2022|0|4|24|2|2022|490|6382|Sunday|2022Q2|N|N|N|2459671|2459760|2459329|2459604|N|N|N|N|N| +2459695|AAAAAAAAPCIIFCAA|2022-04-25|1467|6382|490|2022|1|4|25|2|2022|490|6382|Monday|2022Q2|N|N|N|2459671|2459760|2459330|2459605|N|N|N|N|N| +2459696|AAAAAAAAADIIFCAA|2022-04-26|1467|6383|490|2022|2|4|26|2|2022|490|6383|Tuesday|2022Q2|N|N|N|2459671|2459760|2459331|2459606|N|N|N|N|N| +2459697|AAAAAAAABDIIFCAA|2022-04-27|1467|6383|490|2022|3|4|27|2|2022|490|6383|Wednesday|2022Q2|N|N|N|2459671|2459760|2459332|2459607|N|N|N|N|N| +2459698|AAAAAAAACDIIFCAA|2022-04-28|1467|6383|490|2022|4|4|28|2|2022|490|6383|Thursday|2022Q2|N|N|N|2459671|2459760|2459333|2459608|N|N|N|N|N| +2459699|AAAAAAAADDIIFCAA|2022-04-29|1467|6383|490|2022|5|4|29|2|2022|490|6383|Friday|2022Q2|N|Y|N|2459671|2459760|2459334|2459609|N|N|N|N|N| +2459700|AAAAAAAAEDIIFCAA|2022-04-30|1467|6383|490|2022|6|4|30|2|2022|490|6383|Saturday|2022Q2|N|Y|N|2459671|2459760|2459335|2459610|N|N|N|N|N| +2459701|AAAAAAAAFDIIFCAA|2022-05-01|1468|6383|490|2022|0|5|1|2|2022|490|6383|Sunday|2022Q2|N|N|N|2459701|2459820|2459336|2459611|N|N|N|N|N| +2459702|AAAAAAAAGDIIFCAA|2022-05-02|1468|6383|490|2022|1|5|2|2|2022|490|6383|Monday|2022Q2|N|N|N|2459701|2459820|2459337|2459612|N|N|N|N|N| +2459703|AAAAAAAAHDIIFCAA|2022-05-03|1468|6384|490|2022|2|5|3|2|2022|490|6384|Tuesday|2022Q2|N|N|N|2459701|2459820|2459338|2459613|N|N|N|N|N| +2459704|AAAAAAAAIDIIFCAA|2022-05-04|1468|6384|490|2022|3|5|4|2|2022|490|6384|Wednesday|2022Q2|N|N|N|2459701|2459820|2459339|2459614|N|N|N|N|N| +2459705|AAAAAAAAJDIIFCAA|2022-05-05|1468|6384|490|2022|4|5|5|2|2022|490|6384|Thursday|2022Q2|N|N|N|2459701|2459820|2459340|2459615|N|N|N|N|N| +2459706|AAAAAAAAKDIIFCAA|2022-05-06|1468|6384|490|2022|5|5|6|2|2022|490|6384|Friday|2022Q2|N|Y|N|2459701|2459820|2459341|2459616|N|N|N|N|N| +2459707|AAAAAAAALDIIFCAA|2022-05-07|1468|6384|490|2022|6|5|7|2|2022|490|6384|Saturday|2022Q2|N|Y|N|2459701|2459820|2459342|2459617|N|N|N|N|N| +2459708|AAAAAAAAMDIIFCAA|2022-05-08|1468|6384|490|2022|0|5|8|2|2022|490|6384|Sunday|2022Q2|N|N|N|2459701|2459820|2459343|2459618|N|N|N|N|N| +2459709|AAAAAAAANDIIFCAA|2022-05-09|1468|6384|490|2022|1|5|9|2|2022|490|6384|Monday|2022Q2|N|N|N|2459701|2459820|2459344|2459619|N|N|N|N|N| +2459710|AAAAAAAAODIIFCAA|2022-05-10|1468|6385|490|2022|2|5|10|2|2022|490|6385|Tuesday|2022Q2|N|N|N|2459701|2459820|2459345|2459620|N|N|N|N|N| +2459711|AAAAAAAAPDIIFCAA|2022-05-11|1468|6385|490|2022|3|5|11|2|2022|490|6385|Wednesday|2022Q2|N|N|N|2459701|2459820|2459346|2459621|N|N|N|N|N| +2459712|AAAAAAAAAEIIFCAA|2022-05-12|1468|6385|490|2022|4|5|12|2|2022|490|6385|Thursday|2022Q2|N|N|N|2459701|2459820|2459347|2459622|N|N|N|N|N| +2459713|AAAAAAAABEIIFCAA|2022-05-13|1468|6385|490|2022|5|5|13|2|2022|490|6385|Friday|2022Q2|N|Y|N|2459701|2459820|2459348|2459623|N|N|N|N|N| +2459714|AAAAAAAACEIIFCAA|2022-05-14|1468|6385|490|2022|6|5|14|2|2022|490|6385|Saturday|2022Q2|N|Y|N|2459701|2459820|2459349|2459624|N|N|N|N|N| +2459715|AAAAAAAADEIIFCAA|2022-05-15|1468|6385|490|2022|0|5|15|2|2022|490|6385|Sunday|2022Q2|N|N|N|2459701|2459820|2459350|2459625|N|N|N|N|N| +2459716|AAAAAAAAEEIIFCAA|2022-05-16|1468|6385|490|2022|1|5|16|2|2022|490|6385|Monday|2022Q2|N|N|N|2459701|2459820|2459351|2459626|N|N|N|N|N| +2459717|AAAAAAAAFEIIFCAA|2022-05-17|1468|6386|490|2022|2|5|17|2|2022|490|6386|Tuesday|2022Q2|N|N|N|2459701|2459820|2459352|2459627|N|N|N|N|N| +2459718|AAAAAAAAGEIIFCAA|2022-05-18|1468|6386|490|2022|3|5|18|2|2022|490|6386|Wednesday|2022Q2|N|N|N|2459701|2459820|2459353|2459628|N|N|N|N|N| +2459719|AAAAAAAAHEIIFCAA|2022-05-19|1468|6386|490|2022|4|5|19|2|2022|490|6386|Thursday|2022Q2|N|N|N|2459701|2459820|2459354|2459629|N|N|N|N|N| +2459720|AAAAAAAAIEIIFCAA|2022-05-20|1468|6386|490|2022|5|5|20|2|2022|490|6386|Friday|2022Q2|N|Y|N|2459701|2459820|2459355|2459630|N|N|N|N|N| +2459721|AAAAAAAAJEIIFCAA|2022-05-21|1468|6386|490|2022|6|5|21|2|2022|490|6386|Saturday|2022Q2|N|Y|N|2459701|2459820|2459356|2459631|N|N|N|N|N| +2459722|AAAAAAAAKEIIFCAA|2022-05-22|1468|6386|490|2022|0|5|22|2|2022|490|6386|Sunday|2022Q2|N|N|N|2459701|2459820|2459357|2459632|N|N|N|N|N| +2459723|AAAAAAAALEIIFCAA|2022-05-23|1468|6386|490|2022|1|5|23|2|2022|490|6386|Monday|2022Q2|N|N|N|2459701|2459820|2459358|2459633|N|N|N|N|N| +2459724|AAAAAAAAMEIIFCAA|2022-05-24|1468|6387|490|2022|2|5|24|2|2022|490|6387|Tuesday|2022Q2|N|N|N|2459701|2459820|2459359|2459634|N|N|N|N|N| +2459725|AAAAAAAANEIIFCAA|2022-05-25|1468|6387|490|2022|3|5|25|2|2022|490|6387|Wednesday|2022Q2|N|N|N|2459701|2459820|2459360|2459635|N|N|N|N|N| +2459726|AAAAAAAAOEIIFCAA|2022-05-26|1468|6387|490|2022|4|5|26|2|2022|490|6387|Thursday|2022Q2|N|N|N|2459701|2459820|2459361|2459636|N|N|N|N|N| +2459727|AAAAAAAAPEIIFCAA|2022-05-27|1468|6387|490|2022|5|5|27|2|2022|490|6387|Friday|2022Q2|N|Y|N|2459701|2459820|2459362|2459637|N|N|N|N|N| +2459728|AAAAAAAAAFIIFCAA|2022-05-28|1468|6387|490|2022|6|5|28|2|2022|490|6387|Saturday|2022Q2|N|Y|N|2459701|2459820|2459363|2459638|N|N|N|N|N| +2459729|AAAAAAAABFIIFCAA|2022-05-29|1468|6387|490|2022|0|5|29|2|2022|490|6387|Sunday|2022Q2|N|N|N|2459701|2459820|2459364|2459639|N|N|N|N|N| +2459730|AAAAAAAACFIIFCAA|2022-05-30|1468|6387|490|2022|1|5|30|2|2022|490|6387|Monday|2022Q2|N|N|N|2459701|2459820|2459365|2459640|N|N|N|N|N| +2459731|AAAAAAAADFIIFCAA|2022-05-31|1468|6388|490|2022|2|5|31|2|2022|490|6388|Tuesday|2022Q2|N|N|N|2459701|2459820|2459366|2459641|N|N|N|N|N| +2459732|AAAAAAAAEFIIFCAA|2022-06-01|1469|6388|491|2022|3|6|1|2|2022|491|6388|Wednesday|2022Q2|N|N|N|2459732|2459882|2459367|2459642|N|N|N|N|N| +2459733|AAAAAAAAFFIIFCAA|2022-06-02|1469|6388|491|2022|4|6|2|2|2022|491|6388|Thursday|2022Q2|N|N|N|2459732|2459882|2459368|2459643|N|N|N|N|N| +2459734|AAAAAAAAGFIIFCAA|2022-06-03|1469|6388|491|2022|5|6|3|2|2022|491|6388|Friday|2022Q2|N|Y|N|2459732|2459882|2459369|2459644|N|N|N|N|N| +2459735|AAAAAAAAHFIIFCAA|2022-06-04|1469|6388|491|2022|6|6|4|2|2022|491|6388|Saturday|2022Q2|N|Y|N|2459732|2459882|2459370|2459645|N|N|N|N|N| +2459736|AAAAAAAAIFIIFCAA|2022-06-05|1469|6388|491|2022|0|6|5|2|2022|491|6388|Sunday|2022Q2|N|N|N|2459732|2459882|2459371|2459646|N|N|N|N|N| +2459737|AAAAAAAAJFIIFCAA|2022-06-06|1469|6388|491|2022|1|6|6|2|2022|491|6388|Monday|2022Q2|N|N|N|2459732|2459882|2459372|2459647|N|N|N|N|N| +2459738|AAAAAAAAKFIIFCAA|2022-06-07|1469|6389|491|2022|2|6|7|2|2022|491|6389|Tuesday|2022Q2|N|N|N|2459732|2459882|2459373|2459648|N|N|N|N|N| +2459739|AAAAAAAALFIIFCAA|2022-06-08|1469|6389|491|2022|3|6|8|2|2022|491|6389|Wednesday|2022Q2|N|N|N|2459732|2459882|2459374|2459649|N|N|N|N|N| +2459740|AAAAAAAAMFIIFCAA|2022-06-09|1469|6389|491|2022|4|6|9|2|2022|491|6389|Thursday|2022Q2|N|N|N|2459732|2459882|2459375|2459650|N|N|N|N|N| +2459741|AAAAAAAANFIIFCAA|2022-06-10|1469|6389|491|2022|5|6|10|2|2022|491|6389|Friday|2022Q2|N|Y|N|2459732|2459882|2459376|2459651|N|N|N|N|N| +2459742|AAAAAAAAOFIIFCAA|2022-06-11|1469|6389|491|2022|6|6|11|2|2022|491|6389|Saturday|2022Q2|N|Y|N|2459732|2459882|2459377|2459652|N|N|N|N|N| +2459743|AAAAAAAAPFIIFCAA|2022-06-12|1469|6389|491|2022|0|6|12|2|2022|491|6389|Sunday|2022Q2|N|N|N|2459732|2459882|2459378|2459653|N|N|N|N|N| +2459744|AAAAAAAAAGIIFCAA|2022-06-13|1469|6389|491|2022|1|6|13|2|2022|491|6389|Monday|2022Q2|N|N|N|2459732|2459882|2459379|2459654|N|N|N|N|N| +2459745|AAAAAAAABGIIFCAA|2022-06-14|1469|6390|491|2022|2|6|14|2|2022|491|6390|Tuesday|2022Q2|N|N|N|2459732|2459882|2459380|2459655|N|N|N|N|N| +2459746|AAAAAAAACGIIFCAA|2022-06-15|1469|6390|491|2022|3|6|15|2|2022|491|6390|Wednesday|2022Q2|N|N|N|2459732|2459882|2459381|2459656|N|N|N|N|N| +2459747|AAAAAAAADGIIFCAA|2022-06-16|1469|6390|491|2022|4|6|16|2|2022|491|6390|Thursday|2022Q2|N|N|N|2459732|2459882|2459382|2459657|N|N|N|N|N| +2459748|AAAAAAAAEGIIFCAA|2022-06-17|1469|6390|491|2022|5|6|17|2|2022|491|6390|Friday|2022Q2|N|Y|N|2459732|2459882|2459383|2459658|N|N|N|N|N| +2459749|AAAAAAAAFGIIFCAA|2022-06-18|1469|6390|491|2022|6|6|18|2|2022|491|6390|Saturday|2022Q2|N|Y|N|2459732|2459882|2459384|2459659|N|N|N|N|N| +2459750|AAAAAAAAGGIIFCAA|2022-06-19|1469|6390|491|2022|0|6|19|2|2022|491|6390|Sunday|2022Q2|N|N|N|2459732|2459882|2459385|2459660|N|N|N|N|N| +2459751|AAAAAAAAHGIIFCAA|2022-06-20|1469|6390|491|2022|1|6|20|2|2022|491|6390|Monday|2022Q2|N|N|N|2459732|2459882|2459386|2459661|N|N|N|N|N| +2459752|AAAAAAAAIGIIFCAA|2022-06-21|1469|6391|491|2022|2|6|21|2|2022|491|6391|Tuesday|2022Q2|N|N|N|2459732|2459882|2459387|2459662|N|N|N|N|N| +2459753|AAAAAAAAJGIIFCAA|2022-06-22|1469|6391|491|2022|3|6|22|2|2022|491|6391|Wednesday|2022Q2|N|N|N|2459732|2459882|2459388|2459663|N|N|N|N|N| +2459754|AAAAAAAAKGIIFCAA|2022-06-23|1469|6391|491|2022|4|6|23|2|2022|491|6391|Thursday|2022Q2|N|N|N|2459732|2459882|2459389|2459664|N|N|N|N|N| +2459755|AAAAAAAALGIIFCAA|2022-06-24|1469|6391|491|2022|5|6|24|2|2022|491|6391|Friday|2022Q2|N|Y|N|2459732|2459882|2459390|2459665|N|N|N|N|N| +2459756|AAAAAAAAMGIIFCAA|2022-06-25|1469|6391|491|2022|6|6|25|2|2022|491|6391|Saturday|2022Q2|N|Y|N|2459732|2459882|2459391|2459666|N|N|N|N|N| +2459757|AAAAAAAANGIIFCAA|2022-06-26|1469|6391|491|2022|0|6|26|2|2022|491|6391|Sunday|2022Q2|N|N|N|2459732|2459882|2459392|2459667|N|N|N|N|N| +2459758|AAAAAAAAOGIIFCAA|2022-06-27|1469|6391|491|2022|1|6|27|2|2022|491|6391|Monday|2022Q2|N|N|N|2459732|2459882|2459393|2459668|N|N|N|N|N| +2459759|AAAAAAAAPGIIFCAA|2022-06-28|1469|6392|491|2022|2|6|28|2|2022|491|6392|Tuesday|2022Q2|N|N|N|2459732|2459882|2459394|2459669|N|N|N|N|N| +2459760|AAAAAAAAAHIIFCAA|2022-06-29|1469|6392|491|2022|3|6|29|2|2022|491|6392|Wednesday|2022Q2|N|N|N|2459732|2459882|2459395|2459670|N|N|N|N|N| +2459761|AAAAAAAABHIIFCAA|2022-06-30|1469|6392|491|2022|4|6|30|2|2022|491|6392|Thursday|2022Q2|N|N|N|2459732|2459882|2459396|2459671|N|N|N|N|N| +2459762|AAAAAAAACHIIFCAA|2022-07-01|1470|6392|491|2022|5|7|1|2|2022|491|6392|Friday|2022Q2|N|Y|N|2459762|2459942|2459397|2459671|N|N|N|N|N| +2459763|AAAAAAAADHIIFCAA|2022-07-02|1470|6392|491|2022|6|7|2|3|2022|491|6392|Saturday|2022Q3|N|Y|N|2459762|2459942|2459398|2459672|N|N|N|N|N| +2459764|AAAAAAAAEHIIFCAA|2022-07-03|1470|6392|491|2022|0|7|3|3|2022|491|6392|Sunday|2022Q3|N|N|N|2459762|2459942|2459399|2459673|N|N|N|N|N| +2459765|AAAAAAAAFHIIFCAA|2022-07-04|1470|6392|491|2022|1|7|4|3|2022|491|6392|Monday|2022Q3|N|N|N|2459762|2459942|2459400|2459674|N|N|N|N|N| +2459766|AAAAAAAAGHIIFCAA|2022-07-05|1470|6393|491|2022|2|7|5|3|2022|491|6393|Tuesday|2022Q3|Y|N|N|2459762|2459942|2459401|2459675|N|N|N|N|N| +2459767|AAAAAAAAHHIIFCAA|2022-07-06|1470|6393|491|2022|3|7|6|3|2022|491|6393|Wednesday|2022Q3|N|N|Y|2459762|2459942|2459402|2459676|N|N|N|N|N| +2459768|AAAAAAAAIHIIFCAA|2022-07-07|1470|6393|491|2022|4|7|7|3|2022|491|6393|Thursday|2022Q3|N|N|N|2459762|2459942|2459403|2459677|N|N|N|N|N| +2459769|AAAAAAAAJHIIFCAA|2022-07-08|1470|6393|491|2022|5|7|8|3|2022|491|6393|Friday|2022Q3|N|Y|N|2459762|2459942|2459404|2459678|N|N|N|N|N| +2459770|AAAAAAAAKHIIFCAA|2022-07-09|1470|6393|491|2022|6|7|9|3|2022|491|6393|Saturday|2022Q3|N|Y|N|2459762|2459942|2459405|2459679|N|N|N|N|N| +2459771|AAAAAAAALHIIFCAA|2022-07-10|1470|6393|491|2022|0|7|10|3|2022|491|6393|Sunday|2022Q3|N|N|N|2459762|2459942|2459406|2459680|N|N|N|N|N| +2459772|AAAAAAAAMHIIFCAA|2022-07-11|1470|6393|491|2022|1|7|11|3|2022|491|6393|Monday|2022Q3|N|N|N|2459762|2459942|2459407|2459681|N|N|N|N|N| +2459773|AAAAAAAANHIIFCAA|2022-07-12|1470|6394|491|2022|2|7|12|3|2022|491|6394|Tuesday|2022Q3|N|N|N|2459762|2459942|2459408|2459682|N|N|N|N|N| +2459774|AAAAAAAAOHIIFCAA|2022-07-13|1470|6394|491|2022|3|7|13|3|2022|491|6394|Wednesday|2022Q3|N|N|N|2459762|2459942|2459409|2459683|N|N|N|N|N| +2459775|AAAAAAAAPHIIFCAA|2022-07-14|1470|6394|491|2022|4|7|14|3|2022|491|6394|Thursday|2022Q3|N|N|N|2459762|2459942|2459410|2459684|N|N|N|N|N| +2459776|AAAAAAAAAIIIFCAA|2022-07-15|1470|6394|491|2022|5|7|15|3|2022|491|6394|Friday|2022Q3|N|Y|N|2459762|2459942|2459411|2459685|N|N|N|N|N| +2459777|AAAAAAAABIIIFCAA|2022-07-16|1470|6394|491|2022|6|7|16|3|2022|491|6394|Saturday|2022Q3|N|Y|N|2459762|2459942|2459412|2459686|N|N|N|N|N| +2459778|AAAAAAAACIIIFCAA|2022-07-17|1470|6394|491|2022|0|7|17|3|2022|491|6394|Sunday|2022Q3|N|N|N|2459762|2459942|2459413|2459687|N|N|N|N|N| +2459779|AAAAAAAADIIIFCAA|2022-07-18|1470|6394|491|2022|1|7|18|3|2022|491|6394|Monday|2022Q3|N|N|N|2459762|2459942|2459414|2459688|N|N|N|N|N| +2459780|AAAAAAAAEIIIFCAA|2022-07-19|1470|6395|491|2022|2|7|19|3|2022|491|6395|Tuesday|2022Q3|N|N|N|2459762|2459942|2459415|2459689|N|N|N|N|N| +2459781|AAAAAAAAFIIIFCAA|2022-07-20|1470|6395|491|2022|3|7|20|3|2022|491|6395|Wednesday|2022Q3|N|N|N|2459762|2459942|2459416|2459690|N|N|N|N|N| +2459782|AAAAAAAAGIIIFCAA|2022-07-21|1470|6395|491|2022|4|7|21|3|2022|491|6395|Thursday|2022Q3|N|N|N|2459762|2459942|2459417|2459691|N|N|N|N|N| +2459783|AAAAAAAAHIIIFCAA|2022-07-22|1470|6395|491|2022|5|7|22|3|2022|491|6395|Friday|2022Q3|N|Y|N|2459762|2459942|2459418|2459692|N|N|N|N|N| +2459784|AAAAAAAAIIIIFCAA|2022-07-23|1470|6395|491|2022|6|7|23|3|2022|491|6395|Saturday|2022Q3|N|Y|N|2459762|2459942|2459419|2459693|N|N|N|N|N| +2459785|AAAAAAAAJIIIFCAA|2022-07-24|1470|6395|491|2022|0|7|24|3|2022|491|6395|Sunday|2022Q3|N|N|N|2459762|2459942|2459420|2459694|N|N|N|N|N| +2459786|AAAAAAAAKIIIFCAA|2022-07-25|1470|6395|491|2022|1|7|25|3|2022|491|6395|Monday|2022Q3|N|N|N|2459762|2459942|2459421|2459695|N|N|N|N|N| +2459787|AAAAAAAALIIIFCAA|2022-07-26|1470|6396|491|2022|2|7|26|3|2022|491|6396|Tuesday|2022Q3|N|N|N|2459762|2459942|2459422|2459696|N|N|N|N|N| +2459788|AAAAAAAAMIIIFCAA|2022-07-27|1470|6396|491|2022|3|7|27|3|2022|491|6396|Wednesday|2022Q3|N|N|N|2459762|2459942|2459423|2459697|N|N|N|N|N| +2459789|AAAAAAAANIIIFCAA|2022-07-28|1470|6396|491|2022|4|7|28|3|2022|491|6396|Thursday|2022Q3|N|N|N|2459762|2459942|2459424|2459698|N|N|N|N|N| +2459790|AAAAAAAAOIIIFCAA|2022-07-29|1470|6396|491|2022|5|7|29|3|2022|491|6396|Friday|2022Q3|N|Y|N|2459762|2459942|2459425|2459699|N|N|N|N|N| +2459791|AAAAAAAAPIIIFCAA|2022-07-30|1470|6396|491|2022|6|7|30|3|2022|491|6396|Saturday|2022Q3|N|Y|N|2459762|2459942|2459426|2459700|N|N|N|N|N| +2459792|AAAAAAAAAJIIFCAA|2022-07-31|1470|6396|491|2022|0|7|31|3|2022|491|6396|Sunday|2022Q3|N|N|N|2459762|2459942|2459427|2459701|N|N|N|N|N| +2459793|AAAAAAAABJIIFCAA|2022-08-01|1471|6396|491|2022|1|8|1|3|2022|491|6396|Monday|2022Q3|N|N|N|2459793|2460004|2459428|2459702|N|N|N|N|N| +2459794|AAAAAAAACJIIFCAA|2022-08-02|1471|6397|491|2022|2|8|2|3|2022|491|6397|Tuesday|2022Q3|N|N|N|2459793|2460004|2459429|2459703|N|N|N|N|N| +2459795|AAAAAAAADJIIFCAA|2022-08-03|1471|6397|491|2022|3|8|3|3|2022|491|6397|Wednesday|2022Q3|N|N|N|2459793|2460004|2459430|2459704|N|N|N|N|N| +2459796|AAAAAAAAEJIIFCAA|2022-08-04|1471|6397|491|2022|4|8|4|3|2022|491|6397|Thursday|2022Q3|N|N|N|2459793|2460004|2459431|2459705|N|N|N|N|N| +2459797|AAAAAAAAFJIIFCAA|2022-08-05|1471|6397|491|2022|5|8|5|3|2022|491|6397|Friday|2022Q3|N|Y|N|2459793|2460004|2459432|2459706|N|N|N|N|N| +2459798|AAAAAAAAGJIIFCAA|2022-08-06|1471|6397|491|2022|6|8|6|3|2022|491|6397|Saturday|2022Q3|N|Y|N|2459793|2460004|2459433|2459707|N|N|N|N|N| +2459799|AAAAAAAAHJIIFCAA|2022-08-07|1471|6397|491|2022|0|8|7|3|2022|491|6397|Sunday|2022Q3|N|N|N|2459793|2460004|2459434|2459708|N|N|N|N|N| +2459800|AAAAAAAAIJIIFCAA|2022-08-08|1471|6397|491|2022|1|8|8|3|2022|491|6397|Monday|2022Q3|N|N|N|2459793|2460004|2459435|2459709|N|N|N|N|N| +2459801|AAAAAAAAJJIIFCAA|2022-08-09|1471|6398|491|2022|2|8|9|3|2022|491|6398|Tuesday|2022Q3|N|N|N|2459793|2460004|2459436|2459710|N|N|N|N|N| +2459802|AAAAAAAAKJIIFCAA|2022-08-10|1471|6398|491|2022|3|8|10|3|2022|491|6398|Wednesday|2022Q3|N|N|N|2459793|2460004|2459437|2459711|N|N|N|N|N| +2459803|AAAAAAAALJIIFCAA|2022-08-11|1471|6398|491|2022|4|8|11|3|2022|491|6398|Thursday|2022Q3|N|N|N|2459793|2460004|2459438|2459712|N|N|N|N|N| +2459804|AAAAAAAAMJIIFCAA|2022-08-12|1471|6398|491|2022|5|8|12|3|2022|491|6398|Friday|2022Q3|N|Y|N|2459793|2460004|2459439|2459713|N|N|N|N|N| +2459805|AAAAAAAANJIIFCAA|2022-08-13|1471|6398|491|2022|6|8|13|3|2022|491|6398|Saturday|2022Q3|N|Y|N|2459793|2460004|2459440|2459714|N|N|N|N|N| +2459806|AAAAAAAAOJIIFCAA|2022-08-14|1471|6398|491|2022|0|8|14|3|2022|491|6398|Sunday|2022Q3|N|N|N|2459793|2460004|2459441|2459715|N|N|N|N|N| +2459807|AAAAAAAAPJIIFCAA|2022-08-15|1471|6398|491|2022|1|8|15|3|2022|491|6398|Monday|2022Q3|N|N|N|2459793|2460004|2459442|2459716|N|N|N|N|N| +2459808|AAAAAAAAAKIIFCAA|2022-08-16|1471|6399|491|2022|2|8|16|3|2022|491|6399|Tuesday|2022Q3|N|N|N|2459793|2460004|2459443|2459717|N|N|N|N|N| +2459809|AAAAAAAABKIIFCAA|2022-08-17|1471|6399|491|2022|3|8|17|3|2022|491|6399|Wednesday|2022Q3|N|N|N|2459793|2460004|2459444|2459718|N|N|N|N|N| +2459810|AAAAAAAACKIIFCAA|2022-08-18|1471|6399|491|2022|4|8|18|3|2022|491|6399|Thursday|2022Q3|N|N|N|2459793|2460004|2459445|2459719|N|N|N|N|N| +2459811|AAAAAAAADKIIFCAA|2022-08-19|1471|6399|491|2022|5|8|19|3|2022|491|6399|Friday|2022Q3|N|Y|N|2459793|2460004|2459446|2459720|N|N|N|N|N| +2459812|AAAAAAAAEKIIFCAA|2022-08-20|1471|6399|491|2022|6|8|20|3|2022|491|6399|Saturday|2022Q3|N|Y|N|2459793|2460004|2459447|2459721|N|N|N|N|N| +2459813|AAAAAAAAFKIIFCAA|2022-08-21|1471|6399|491|2022|0|8|21|3|2022|491|6399|Sunday|2022Q3|N|N|N|2459793|2460004|2459448|2459722|N|N|N|N|N| +2459814|AAAAAAAAGKIIFCAA|2022-08-22|1471|6399|491|2022|1|8|22|3|2022|491|6399|Monday|2022Q3|N|N|N|2459793|2460004|2459449|2459723|N|N|N|N|N| +2459815|AAAAAAAAHKIIFCAA|2022-08-23|1471|6400|491|2022|2|8|23|3|2022|491|6400|Tuesday|2022Q3|N|N|N|2459793|2460004|2459450|2459724|N|N|N|N|N| +2459816|AAAAAAAAIKIIFCAA|2022-08-24|1471|6400|491|2022|3|8|24|3|2022|491|6400|Wednesday|2022Q3|N|N|N|2459793|2460004|2459451|2459725|N|N|N|N|N| +2459817|AAAAAAAAJKIIFCAA|2022-08-25|1471|6400|491|2022|4|8|25|3|2022|491|6400|Thursday|2022Q3|N|N|N|2459793|2460004|2459452|2459726|N|N|N|N|N| +2459818|AAAAAAAAKKIIFCAA|2022-08-26|1471|6400|491|2022|5|8|26|3|2022|491|6400|Friday|2022Q3|N|Y|N|2459793|2460004|2459453|2459727|N|N|N|N|N| +2459819|AAAAAAAALKIIFCAA|2022-08-27|1471|6400|491|2022|6|8|27|3|2022|491|6400|Saturday|2022Q3|N|Y|N|2459793|2460004|2459454|2459728|N|N|N|N|N| +2459820|AAAAAAAAMKIIFCAA|2022-08-28|1471|6400|491|2022|0|8|28|3|2022|491|6400|Sunday|2022Q3|N|N|N|2459793|2460004|2459455|2459729|N|N|N|N|N| +2459821|AAAAAAAANKIIFCAA|2022-08-29|1471|6400|491|2022|1|8|29|3|2022|491|6400|Monday|2022Q3|N|N|N|2459793|2460004|2459456|2459730|N|N|N|N|N| +2459822|AAAAAAAAOKIIFCAA|2022-08-30|1471|6401|491|2022|2|8|30|3|2022|491|6401|Tuesday|2022Q3|N|N|N|2459793|2460004|2459457|2459731|N|N|N|N|N| +2459823|AAAAAAAAPKIIFCAA|2022-08-31|1471|6401|491|2022|3|8|31|3|2022|491|6401|Wednesday|2022Q3|N|N|N|2459793|2460004|2459458|2459732|N|N|N|N|N| +2459824|AAAAAAAAALIIFCAA|2022-09-01|1472|6401|492|2022|4|9|1|3|2022|492|6401|Thursday|2022Q3|N|N|N|2459824|2460066|2459459|2459733|N|N|N|N|N| +2459825|AAAAAAAABLIIFCAA|2022-09-02|1472|6401|492|2022|5|9|2|3|2022|492|6401|Friday|2022Q3|N|Y|N|2459824|2460066|2459460|2459734|N|N|N|N|N| +2459826|AAAAAAAACLIIFCAA|2022-09-03|1472|6401|492|2022|6|9|3|3|2022|492|6401|Saturday|2022Q3|N|Y|N|2459824|2460066|2459461|2459735|N|N|N|N|N| +2459827|AAAAAAAADLIIFCAA|2022-09-04|1472|6401|492|2022|0|9|4|3|2022|492|6401|Sunday|2022Q3|N|N|N|2459824|2460066|2459462|2459736|N|N|N|N|N| +2459828|AAAAAAAAELIIFCAA|2022-09-05|1472|6401|492|2022|1|9|5|3|2022|492|6401|Monday|2022Q3|N|N|N|2459824|2460066|2459463|2459737|N|N|N|N|N| +2459829|AAAAAAAAFLIIFCAA|2022-09-06|1472|6402|492|2022|2|9|6|3|2022|492|6402|Tuesday|2022Q3|N|N|N|2459824|2460066|2459464|2459738|N|N|N|N|N| +2459830|AAAAAAAAGLIIFCAA|2022-09-07|1472|6402|492|2022|3|9|7|3|2022|492|6402|Wednesday|2022Q3|N|N|N|2459824|2460066|2459465|2459739|N|N|N|N|N| +2459831|AAAAAAAAHLIIFCAA|2022-09-08|1472|6402|492|2022|4|9|8|3|2022|492|6402|Thursday|2022Q3|N|N|N|2459824|2460066|2459466|2459740|N|N|N|N|N| +2459832|AAAAAAAAILIIFCAA|2022-09-09|1472|6402|492|2022|5|9|9|3|2022|492|6402|Friday|2022Q3|N|Y|N|2459824|2460066|2459467|2459741|N|N|N|N|N| +2459833|AAAAAAAAJLIIFCAA|2022-09-10|1472|6402|492|2022|6|9|10|3|2022|492|6402|Saturday|2022Q3|N|Y|N|2459824|2460066|2459468|2459742|N|N|N|N|N| +2459834|AAAAAAAAKLIIFCAA|2022-09-11|1472|6402|492|2022|0|9|11|3|2022|492|6402|Sunday|2022Q3|N|N|N|2459824|2460066|2459469|2459743|N|N|N|N|N| +2459835|AAAAAAAALLIIFCAA|2022-09-12|1472|6402|492|2022|1|9|12|3|2022|492|6402|Monday|2022Q3|N|N|N|2459824|2460066|2459470|2459744|N|N|N|N|N| +2459836|AAAAAAAAMLIIFCAA|2022-09-13|1472|6403|492|2022|2|9|13|3|2022|492|6403|Tuesday|2022Q3|N|N|N|2459824|2460066|2459471|2459745|N|N|N|N|N| +2459837|AAAAAAAANLIIFCAA|2022-09-14|1472|6403|492|2022|3|9|14|3|2022|492|6403|Wednesday|2022Q3|N|N|N|2459824|2460066|2459472|2459746|N|N|N|N|N| +2459838|AAAAAAAAOLIIFCAA|2022-09-15|1472|6403|492|2022|4|9|15|3|2022|492|6403|Thursday|2022Q3|N|N|N|2459824|2460066|2459473|2459747|N|N|N|N|N| +2459839|AAAAAAAAPLIIFCAA|2022-09-16|1472|6403|492|2022|5|9|16|3|2022|492|6403|Friday|2022Q3|N|Y|N|2459824|2460066|2459474|2459748|N|N|N|N|N| +2459840|AAAAAAAAAMIIFCAA|2022-09-17|1472|6403|492|2022|6|9|17|3|2022|492|6403|Saturday|2022Q3|N|Y|N|2459824|2460066|2459475|2459749|N|N|N|N|N| +2459841|AAAAAAAABMIIFCAA|2022-09-18|1472|6403|492|2022|0|9|18|3|2022|492|6403|Sunday|2022Q3|N|N|N|2459824|2460066|2459476|2459750|N|N|N|N|N| +2459842|AAAAAAAACMIIFCAA|2022-09-19|1472|6403|492|2022|1|9|19|3|2022|492|6403|Monday|2022Q3|N|N|N|2459824|2460066|2459477|2459751|N|N|N|N|N| +2459843|AAAAAAAADMIIFCAA|2022-09-20|1472|6404|492|2022|2|9|20|3|2022|492|6404|Tuesday|2022Q3|N|N|N|2459824|2460066|2459478|2459752|N|N|N|N|N| +2459844|AAAAAAAAEMIIFCAA|2022-09-21|1472|6404|492|2022|3|9|21|3|2022|492|6404|Wednesday|2022Q3|N|N|N|2459824|2460066|2459479|2459753|N|N|N|N|N| +2459845|AAAAAAAAFMIIFCAA|2022-09-22|1472|6404|492|2022|4|9|22|3|2022|492|6404|Thursday|2022Q3|N|N|N|2459824|2460066|2459480|2459754|N|N|N|N|N| +2459846|AAAAAAAAGMIIFCAA|2022-09-23|1472|6404|492|2022|5|9|23|3|2022|492|6404|Friday|2022Q3|N|Y|N|2459824|2460066|2459481|2459755|N|N|N|N|N| +2459847|AAAAAAAAHMIIFCAA|2022-09-24|1472|6404|492|2022|6|9|24|3|2022|492|6404|Saturday|2022Q3|N|Y|N|2459824|2460066|2459482|2459756|N|N|N|N|N| +2459848|AAAAAAAAIMIIFCAA|2022-09-25|1472|6404|492|2022|0|9|25|3|2022|492|6404|Sunday|2022Q3|N|N|N|2459824|2460066|2459483|2459757|N|N|N|N|N| +2459849|AAAAAAAAJMIIFCAA|2022-09-26|1472|6404|492|2022|1|9|26|3|2022|492|6404|Monday|2022Q3|N|N|N|2459824|2460066|2459484|2459758|N|N|N|N|N| +2459850|AAAAAAAAKMIIFCAA|2022-09-27|1472|6405|492|2022|2|9|27|3|2022|492|6405|Tuesday|2022Q3|N|N|N|2459824|2460066|2459485|2459759|N|N|N|N|N| +2459851|AAAAAAAALMIIFCAA|2022-09-28|1472|6405|492|2022|3|9|28|3|2022|492|6405|Wednesday|2022Q3|N|N|N|2459824|2460066|2459486|2459760|N|N|N|N|N| +2459852|AAAAAAAAMMIIFCAA|2022-09-29|1472|6405|492|2022|4|9|29|3|2022|492|6405|Thursday|2022Q3|N|N|N|2459824|2460066|2459487|2459761|N|N|N|N|N| +2459853|AAAAAAAANMIIFCAA|2022-09-30|1472|6405|492|2022|5|9|30|3|2022|492|6405|Friday|2022Q3|N|Y|N|2459824|2460066|2459488|2459762|N|N|N|N|N| +2459854|AAAAAAAAOMIIFCAA|2022-10-01|1473|6405|492|2022|6|10|1|3|2022|492|6405|Saturday|2022Q3|N|Y|N|2459854|2460126|2459489|2459762|N|N|N|N|N| +2459855|AAAAAAAAPMIIFCAA|2022-10-02|1473|6405|492|2022|0|10|2|4|2022|492|6405|Sunday|2022Q4|N|N|N|2459854|2460126|2459490|2459763|N|N|N|N|N| +2459856|AAAAAAAAANIIFCAA|2022-10-03|1473|6405|492|2022|1|10|3|4|2022|492|6405|Monday|2022Q4|N|N|N|2459854|2460126|2459491|2459764|N|N|N|N|N| +2459857|AAAAAAAABNIIFCAA|2022-10-04|1473|6406|492|2022|2|10|4|4|2022|492|6406|Tuesday|2022Q4|N|N|N|2459854|2460126|2459492|2459765|N|N|N|N|N| +2459858|AAAAAAAACNIIFCAA|2022-10-05|1473|6406|492|2022|3|10|5|4|2022|492|6406|Wednesday|2022Q4|N|N|N|2459854|2460126|2459493|2459766|N|N|N|N|N| +2459859|AAAAAAAADNIIFCAA|2022-10-06|1473|6406|492|2022|4|10|6|4|2022|492|6406|Thursday|2022Q4|N|N|N|2459854|2460126|2459494|2459767|N|N|N|N|N| +2459860|AAAAAAAAENIIFCAA|2022-10-07|1473|6406|492|2022|5|10|7|4|2022|492|6406|Friday|2022Q4|N|Y|N|2459854|2460126|2459495|2459768|N|N|N|N|N| +2459861|AAAAAAAAFNIIFCAA|2022-10-08|1473|6406|492|2022|6|10|8|4|2022|492|6406|Saturday|2022Q4|N|Y|N|2459854|2460126|2459496|2459769|N|N|N|N|N| +2459862|AAAAAAAAGNIIFCAA|2022-10-09|1473|6406|492|2022|0|10|9|4|2022|492|6406|Sunday|2022Q4|N|N|N|2459854|2460126|2459497|2459770|N|N|N|N|N| +2459863|AAAAAAAAHNIIFCAA|2022-10-10|1473|6406|492|2022|1|10|10|4|2022|492|6406|Monday|2022Q4|N|N|N|2459854|2460126|2459498|2459771|N|N|N|N|N| +2459864|AAAAAAAAINIIFCAA|2022-10-11|1473|6407|492|2022|2|10|11|4|2022|492|6407|Tuesday|2022Q4|N|N|N|2459854|2460126|2459499|2459772|N|N|N|N|N| +2459865|AAAAAAAAJNIIFCAA|2022-10-12|1473|6407|492|2022|3|10|12|4|2022|492|6407|Wednesday|2022Q4|N|N|N|2459854|2460126|2459500|2459773|N|N|N|N|N| +2459866|AAAAAAAAKNIIFCAA|2022-10-13|1473|6407|492|2022|4|10|13|4|2022|492|6407|Thursday|2022Q4|N|N|N|2459854|2460126|2459501|2459774|N|N|N|N|N| +2459867|AAAAAAAALNIIFCAA|2022-10-14|1473|6407|492|2022|5|10|14|4|2022|492|6407|Friday|2022Q4|N|Y|N|2459854|2460126|2459502|2459775|N|N|N|N|N| +2459868|AAAAAAAAMNIIFCAA|2022-10-15|1473|6407|492|2022|6|10|15|4|2022|492|6407|Saturday|2022Q4|N|Y|N|2459854|2460126|2459503|2459776|N|N|N|N|N| +2459869|AAAAAAAANNIIFCAA|2022-10-16|1473|6407|492|2022|0|10|16|4|2022|492|6407|Sunday|2022Q4|N|N|N|2459854|2460126|2459504|2459777|N|N|N|N|N| +2459870|AAAAAAAAONIIFCAA|2022-10-17|1473|6407|492|2022|1|10|17|4|2022|492|6407|Monday|2022Q4|N|N|N|2459854|2460126|2459505|2459778|N|N|N|N|N| +2459871|AAAAAAAAPNIIFCAA|2022-10-18|1473|6408|492|2022|2|10|18|4|2022|492|6408|Tuesday|2022Q4|N|N|N|2459854|2460126|2459506|2459779|N|N|N|N|N| +2459872|AAAAAAAAAOIIFCAA|2022-10-19|1473|6408|492|2022|3|10|19|4|2022|492|6408|Wednesday|2022Q4|N|N|N|2459854|2460126|2459507|2459780|N|N|N|N|N| +2459873|AAAAAAAABOIIFCAA|2022-10-20|1473|6408|492|2022|4|10|20|4|2022|492|6408|Thursday|2022Q4|N|N|N|2459854|2460126|2459508|2459781|N|N|N|N|N| +2459874|AAAAAAAACOIIFCAA|2022-10-21|1473|6408|492|2022|5|10|21|4|2022|492|6408|Friday|2022Q4|N|Y|N|2459854|2460126|2459509|2459782|N|N|N|N|N| +2459875|AAAAAAAADOIIFCAA|2022-10-22|1473|6408|492|2022|6|10|22|4|2022|492|6408|Saturday|2022Q4|N|Y|N|2459854|2460126|2459510|2459783|N|N|N|N|N| +2459876|AAAAAAAAEOIIFCAA|2022-10-23|1473|6408|492|2022|0|10|23|4|2022|492|6408|Sunday|2022Q4|N|N|N|2459854|2460126|2459511|2459784|N|N|N|N|N| +2459877|AAAAAAAAFOIIFCAA|2022-10-24|1473|6408|492|2022|1|10|24|4|2022|492|6408|Monday|2022Q4|N|N|N|2459854|2460126|2459512|2459785|N|N|N|N|N| +2459878|AAAAAAAAGOIIFCAA|2022-10-25|1473|6409|492|2022|2|10|25|4|2022|492|6409|Tuesday|2022Q4|N|N|N|2459854|2460126|2459513|2459786|N|N|N|N|N| +2459879|AAAAAAAAHOIIFCAA|2022-10-26|1473|6409|492|2022|3|10|26|4|2022|492|6409|Wednesday|2022Q4|N|N|N|2459854|2460126|2459514|2459787|N|N|N|N|N| +2459880|AAAAAAAAIOIIFCAA|2022-10-27|1473|6409|492|2022|4|10|27|4|2022|492|6409|Thursday|2022Q4|N|N|N|2459854|2460126|2459515|2459788|N|N|N|N|N| +2459881|AAAAAAAAJOIIFCAA|2022-10-28|1473|6409|492|2022|5|10|28|4|2022|492|6409|Friday|2022Q4|N|Y|N|2459854|2460126|2459516|2459789|N|N|N|N|N| +2459882|AAAAAAAAKOIIFCAA|2022-10-29|1473|6409|492|2022|6|10|29|4|2022|492|6409|Saturday|2022Q4|N|Y|N|2459854|2460126|2459517|2459790|N|N|N|N|N| +2459883|AAAAAAAALOIIFCAA|2022-10-30|1473|6409|492|2022|0|10|30|4|2022|492|6409|Sunday|2022Q4|N|N|N|2459854|2460126|2459518|2459791|N|N|N|N|N| +2459884|AAAAAAAAMOIIFCAA|2022-10-31|1473|6409|492|2022|1|10|31|4|2022|492|6409|Monday|2022Q4|N|N|N|2459854|2460126|2459519|2459792|N|N|N|N|N| +2459885|AAAAAAAANOIIFCAA|2022-11-01|1474|6410|492|2022|2|11|1|4|2022|492|6410|Tuesday|2022Q4|N|N|N|2459885|2460188|2459520|2459793|N|N|N|N|N| +2459886|AAAAAAAAOOIIFCAA|2022-11-02|1474|6410|492|2022|3|11|2|4|2022|492|6410|Wednesday|2022Q4|N|N|N|2459885|2460188|2459521|2459794|N|N|N|N|N| +2459887|AAAAAAAAPOIIFCAA|2022-11-03|1474|6410|492|2022|4|11|3|4|2022|492|6410|Thursday|2022Q4|N|N|N|2459885|2460188|2459522|2459795|N|N|N|N|N| +2459888|AAAAAAAAAPIIFCAA|2022-11-04|1474|6410|492|2022|5|11|4|4|2022|492|6410|Friday|2022Q4|N|Y|N|2459885|2460188|2459523|2459796|N|N|N|N|N| +2459889|AAAAAAAABPIIFCAA|2022-11-05|1474|6410|492|2022|6|11|5|4|2022|492|6410|Saturday|2022Q4|N|Y|N|2459885|2460188|2459524|2459797|N|N|N|N|N| +2459890|AAAAAAAACPIIFCAA|2022-11-06|1474|6410|492|2022|0|11|6|4|2022|492|6410|Sunday|2022Q4|N|N|N|2459885|2460188|2459525|2459798|N|N|N|N|N| +2459891|AAAAAAAADPIIFCAA|2022-11-07|1474|6410|492|2022|1|11|7|4|2022|492|6410|Monday|2022Q4|N|N|N|2459885|2460188|2459526|2459799|N|N|N|N|N| +2459892|AAAAAAAAEPIIFCAA|2022-11-08|1474|6411|492|2022|2|11|8|4|2022|492|6411|Tuesday|2022Q4|N|N|N|2459885|2460188|2459527|2459800|N|N|N|N|N| +2459893|AAAAAAAAFPIIFCAA|2022-11-09|1474|6411|492|2022|3|11|9|4|2022|492|6411|Wednesday|2022Q4|N|N|N|2459885|2460188|2459528|2459801|N|N|N|N|N| +2459894|AAAAAAAAGPIIFCAA|2022-11-10|1474|6411|492|2022|4|11|10|4|2022|492|6411|Thursday|2022Q4|N|N|N|2459885|2460188|2459529|2459802|N|N|N|N|N| +2459895|AAAAAAAAHPIIFCAA|2022-11-11|1474|6411|492|2022|5|11|11|4|2022|492|6411|Friday|2022Q4|N|Y|N|2459885|2460188|2459530|2459803|N|N|N|N|N| +2459896|AAAAAAAAIPIIFCAA|2022-11-12|1474|6411|492|2022|6|11|12|4|2022|492|6411|Saturday|2022Q4|N|Y|N|2459885|2460188|2459531|2459804|N|N|N|N|N| +2459897|AAAAAAAAJPIIFCAA|2022-11-13|1474|6411|492|2022|0|11|13|4|2022|492|6411|Sunday|2022Q4|N|N|N|2459885|2460188|2459532|2459805|N|N|N|N|N| +2459898|AAAAAAAAKPIIFCAA|2022-11-14|1474|6411|492|2022|1|11|14|4|2022|492|6411|Monday|2022Q4|N|N|N|2459885|2460188|2459533|2459806|N|N|N|N|N| +2459899|AAAAAAAALPIIFCAA|2022-11-15|1474|6412|492|2022|2|11|15|4|2022|492|6412|Tuesday|2022Q4|N|N|N|2459885|2460188|2459534|2459807|N|N|N|N|N| +2459900|AAAAAAAAMPIIFCAA|2022-11-16|1474|6412|492|2022|3|11|16|4|2022|492|6412|Wednesday|2022Q4|N|N|N|2459885|2460188|2459535|2459808|N|N|N|N|N| +2459901|AAAAAAAANPIIFCAA|2022-11-17|1474|6412|492|2022|4|11|17|4|2022|492|6412|Thursday|2022Q4|N|N|N|2459885|2460188|2459536|2459809|N|N|N|N|N| +2459902|AAAAAAAAOPIIFCAA|2022-11-18|1474|6412|492|2022|5|11|18|4|2022|492|6412|Friday|2022Q4|N|Y|N|2459885|2460188|2459537|2459810|N|N|N|N|N| +2459903|AAAAAAAAPPIIFCAA|2022-11-19|1474|6412|492|2022|6|11|19|4|2022|492|6412|Saturday|2022Q4|N|Y|N|2459885|2460188|2459538|2459811|N|N|N|N|N| +2459904|AAAAAAAAAAJIFCAA|2022-11-20|1474|6412|492|2022|0|11|20|4|2022|492|6412|Sunday|2022Q4|N|N|N|2459885|2460188|2459539|2459812|N|N|N|N|N| +2459905|AAAAAAAABAJIFCAA|2022-11-21|1474|6412|492|2022|1|11|21|4|2022|492|6412|Monday|2022Q4|N|N|N|2459885|2460188|2459540|2459813|N|N|N|N|N| +2459906|AAAAAAAACAJIFCAA|2022-11-22|1474|6413|492|2022|2|11|22|4|2022|492|6413|Tuesday|2022Q4|N|N|N|2459885|2460188|2459541|2459814|N|N|N|N|N| +2459907|AAAAAAAADAJIFCAA|2022-11-23|1474|6413|492|2022|3|11|23|4|2022|492|6413|Wednesday|2022Q4|N|N|N|2459885|2460188|2459542|2459815|N|N|N|N|N| +2459908|AAAAAAAAEAJIFCAA|2022-11-24|1474|6413|492|2022|4|11|24|4|2022|492|6413|Thursday|2022Q4|N|N|N|2459885|2460188|2459543|2459816|N|N|N|N|N| +2459909|AAAAAAAAFAJIFCAA|2022-11-25|1474|6413|492|2022|5|11|25|4|2022|492|6413|Friday|2022Q4|N|Y|N|2459885|2460188|2459544|2459817|N|N|N|N|N| +2459910|AAAAAAAAGAJIFCAA|2022-11-26|1474|6413|492|2022|6|11|26|4|2022|492|6413|Saturday|2022Q4|N|Y|N|2459885|2460188|2459545|2459818|N|N|N|N|N| +2459911|AAAAAAAAHAJIFCAA|2022-11-27|1474|6413|492|2022|0|11|27|4|2022|492|6413|Sunday|2022Q4|N|N|N|2459885|2460188|2459546|2459819|N|N|N|N|N| +2459912|AAAAAAAAIAJIFCAA|2022-11-28|1474|6413|492|2022|1|11|28|4|2022|492|6413|Monday|2022Q4|N|N|N|2459885|2460188|2459547|2459820|N|N|N|N|N| +2459913|AAAAAAAAJAJIFCAA|2022-11-29|1474|6414|492|2022|2|11|29|4|2022|492|6414|Tuesday|2022Q4|N|N|N|2459885|2460188|2459548|2459821|N|N|N|N|N| +2459914|AAAAAAAAKAJIFCAA|2022-11-30|1474|6414|492|2022|3|11|30|4|2022|492|6414|Wednesday|2022Q4|N|N|N|2459885|2460188|2459549|2459822|N|N|N|N|N| +2459915|AAAAAAAALAJIFCAA|2022-12-01|1475|6414|493|2022|4|12|1|4|2022|493|6414|Thursday|2022Q4|N|N|N|2459915|2460248|2459550|2459823|N|N|N|N|N| +2459916|AAAAAAAAMAJIFCAA|2022-12-02|1475|6414|493|2022|5|12|2|4|2022|493|6414|Friday|2022Q4|N|Y|N|2459915|2460248|2459551|2459824|N|N|N|N|N| +2459917|AAAAAAAANAJIFCAA|2022-12-03|1475|6414|493|2022|6|12|3|4|2022|493|6414|Saturday|2022Q4|N|Y|N|2459915|2460248|2459552|2459825|N|N|N|N|N| +2459918|AAAAAAAAOAJIFCAA|2022-12-04|1475|6414|493|2022|0|12|4|4|2022|493|6414|Sunday|2022Q4|N|N|N|2459915|2460248|2459553|2459826|N|N|N|N|N| +2459919|AAAAAAAAPAJIFCAA|2022-12-05|1475|6414|493|2022|1|12|5|4|2022|493|6414|Monday|2022Q4|N|N|N|2459915|2460248|2459554|2459827|N|N|N|N|N| +2459920|AAAAAAAAABJIFCAA|2022-12-06|1475|6415|493|2022|2|12|6|4|2022|493|6415|Tuesday|2022Q4|N|N|N|2459915|2460248|2459555|2459828|N|N|N|N|N| +2459921|AAAAAAAABBJIFCAA|2022-12-07|1475|6415|493|2022|3|12|7|4|2022|493|6415|Wednesday|2022Q4|N|N|N|2459915|2460248|2459556|2459829|N|N|N|N|N| +2459922|AAAAAAAACBJIFCAA|2022-12-08|1475|6415|493|2022|4|12|8|4|2022|493|6415|Thursday|2022Q4|N|N|N|2459915|2460248|2459557|2459830|N|N|N|N|N| +2459923|AAAAAAAADBJIFCAA|2022-12-09|1475|6415|493|2022|5|12|9|4|2022|493|6415|Friday|2022Q4|N|Y|N|2459915|2460248|2459558|2459831|N|N|N|N|N| +2459924|AAAAAAAAEBJIFCAA|2022-12-10|1475|6415|493|2022|6|12|10|4|2022|493|6415|Saturday|2022Q4|N|Y|N|2459915|2460248|2459559|2459832|N|N|N|N|N| +2459925|AAAAAAAAFBJIFCAA|2022-12-11|1475|6415|493|2022|0|12|11|4|2022|493|6415|Sunday|2022Q4|N|N|N|2459915|2460248|2459560|2459833|N|N|N|N|N| +2459926|AAAAAAAAGBJIFCAA|2022-12-12|1475|6415|493|2022|1|12|12|4|2022|493|6415|Monday|2022Q4|N|N|N|2459915|2460248|2459561|2459834|N|N|N|N|N| +2459927|AAAAAAAAHBJIFCAA|2022-12-13|1475|6416|493|2022|2|12|13|4|2022|493|6416|Tuesday|2022Q4|N|N|N|2459915|2460248|2459562|2459835|N|N|N|N|N| +2459928|AAAAAAAAIBJIFCAA|2022-12-14|1475|6416|493|2022|3|12|14|4|2022|493|6416|Wednesday|2022Q4|N|N|N|2459915|2460248|2459563|2459836|N|N|N|N|N| +2459929|AAAAAAAAJBJIFCAA|2022-12-15|1475|6416|493|2022|4|12|15|4|2022|493|6416|Thursday|2022Q4|N|N|N|2459915|2460248|2459564|2459837|N|N|N|N|N| +2459930|AAAAAAAAKBJIFCAA|2022-12-16|1475|6416|493|2022|5|12|16|4|2022|493|6416|Friday|2022Q4|N|Y|N|2459915|2460248|2459565|2459838|N|N|N|N|N| +2459931|AAAAAAAALBJIFCAA|2022-12-17|1475|6416|493|2022|6|12|17|4|2022|493|6416|Saturday|2022Q4|N|Y|N|2459915|2460248|2459566|2459839|N|N|N|N|N| +2459932|AAAAAAAAMBJIFCAA|2022-12-18|1475|6416|493|2022|0|12|18|4|2022|493|6416|Sunday|2022Q4|N|N|N|2459915|2460248|2459567|2459840|N|N|N|N|N| +2459933|AAAAAAAANBJIFCAA|2022-12-19|1475|6416|493|2022|1|12|19|4|2022|493|6416|Monday|2022Q4|N|N|N|2459915|2460248|2459568|2459841|N|N|N|N|N| +2459934|AAAAAAAAOBJIFCAA|2022-12-20|1475|6417|493|2022|2|12|20|4|2022|493|6417|Tuesday|2022Q4|N|N|N|2459915|2460248|2459569|2459842|N|N|N|N|N| +2459935|AAAAAAAAPBJIFCAA|2022-12-21|1475|6417|493|2022|3|12|21|4|2022|493|6417|Wednesday|2022Q4|N|N|N|2459915|2460248|2459570|2459843|N|N|N|N|N| +2459936|AAAAAAAAACJIFCAA|2022-12-22|1475|6417|493|2022|4|12|22|4|2022|493|6417|Thursday|2022Q4|N|N|N|2459915|2460248|2459571|2459844|N|N|N|N|N| +2459937|AAAAAAAABCJIFCAA|2022-12-23|1475|6417|493|2022|5|12|23|4|2022|493|6417|Friday|2022Q4|N|Y|N|2459915|2460248|2459572|2459845|N|N|N|N|N| +2459938|AAAAAAAACCJIFCAA|2022-12-24|1475|6417|493|2022|6|12|24|4|2022|493|6417|Saturday|2022Q4|N|Y|N|2459915|2460248|2459573|2459846|N|N|N|N|N| +2459939|AAAAAAAADCJIFCAA|2022-12-25|1475|6417|493|2022|0|12|25|4|2022|493|6417|Sunday|2022Q4|N|N|N|2459915|2460248|2459574|2459847|N|N|N|N|N| +2459940|AAAAAAAAECJIFCAA|2022-12-26|1475|6417|493|2022|1|12|26|4|2022|493|6417|Monday|2022Q4|Y|N|N|2459915|2460248|2459575|2459848|N|N|N|N|N| +2459941|AAAAAAAAFCJIFCAA|2022-12-27|1475|6418|493|2022|2|12|27|4|2022|493|6418|Tuesday|2022Q4|N|N|Y|2459915|2460248|2459576|2459849|N|N|N|N|N| +2459942|AAAAAAAAGCJIFCAA|2022-12-28|1475|6418|493|2022|3|12|28|4|2022|493|6418|Wednesday|2022Q4|N|N|N|2459915|2460248|2459577|2459850|N|N|N|N|N| +2459943|AAAAAAAAHCJIFCAA|2022-12-29|1475|6418|493|2022|4|12|29|4|2022|493|6418|Thursday|2022Q4|N|N|N|2459915|2460248|2459578|2459851|N|N|N|N|N| +2459944|AAAAAAAAICJIFCAA|2022-12-30|1475|6418|493|2022|5|12|30|4|2022|493|6418|Friday|2022Q4|N|Y|N|2459915|2460248|2459579|2459852|N|N|N|N|N| +2459945|AAAAAAAAJCJIFCAA|2022-12-31|1475|6418|493|2022|6|12|31|4|2022|493|6418|Saturday|2022Q4|N|Y|N|2459915|2460248|2459580|2459853|N|N|N|N|N| +2459946|AAAAAAAAKCJIFCAA|2023-01-01|1476|6418|493|2023|0|1|1|1|2023|493|6418|Sunday|2023Q1|Y|N|N|2459946|2459945|2459581|2459854|N|N|N|N|N| +2459947|AAAAAAAALCJIFCAA|2023-01-02|1476|6418|493|2023|1|1|2|1|2023|493|6418|Monday|2023Q1|N|N|Y|2459946|2459945|2459582|2459855|N|N|N|N|N| +2459948|AAAAAAAAMCJIFCAA|2023-01-03|1476|6419|493|2023|2|1|3|1|2023|493|6419|Tuesday|2023Q1|N|N|N|2459946|2459945|2459583|2459856|N|N|N|N|N| +2459949|AAAAAAAANCJIFCAA|2023-01-04|1476|6419|493|2023|3|1|4|1|2023|493|6419|Wednesday|2023Q1|N|N|N|2459946|2459945|2459584|2459857|N|N|N|N|N| +2459950|AAAAAAAAOCJIFCAA|2023-01-05|1476|6419|493|2023|4|1|5|1|2023|493|6419|Thursday|2023Q1|N|N|N|2459946|2459945|2459585|2459858|N|N|N|N|N| +2459951|AAAAAAAAPCJIFCAA|2023-01-06|1476|6419|493|2023|5|1|6|1|2023|493|6419|Friday|2023Q1|N|Y|N|2459946|2459945|2459586|2459859|N|N|N|N|N| +2459952|AAAAAAAAADJIFCAA|2023-01-07|1476|6419|493|2023|6|1|7|1|2023|493|6419|Saturday|2023Q1|N|Y|N|2459946|2459945|2459587|2459860|N|N|N|N|N| +2459953|AAAAAAAABDJIFCAA|2023-01-08|1476|6419|493|2023|0|1|8|1|2023|493|6419|Sunday|2023Q1|N|N|N|2459946|2459945|2459588|2459861|N|N|N|N|N| +2459954|AAAAAAAACDJIFCAA|2023-01-09|1476|6419|493|2023|1|1|9|1|2023|493|6419|Monday|2023Q1|N|N|N|2459946|2459945|2459589|2459862|N|N|N|N|N| +2459955|AAAAAAAADDJIFCAA|2023-01-10|1476|6420|493|2023|2|1|10|1|2023|493|6420|Tuesday|2023Q1|N|N|N|2459946|2459945|2459590|2459863|N|N|N|N|N| +2459956|AAAAAAAAEDJIFCAA|2023-01-11|1476|6420|493|2023|3|1|11|1|2023|493|6420|Wednesday|2023Q1|N|N|N|2459946|2459945|2459591|2459864|N|N|N|N|N| +2459957|AAAAAAAAFDJIFCAA|2023-01-12|1476|6420|493|2023|4|1|12|1|2023|493|6420|Thursday|2023Q1|N|N|N|2459946|2459945|2459592|2459865|N|N|N|N|N| +2459958|AAAAAAAAGDJIFCAA|2023-01-13|1476|6420|493|2023|5|1|13|1|2023|493|6420|Friday|2023Q1|N|Y|N|2459946|2459945|2459593|2459866|N|N|N|N|N| +2459959|AAAAAAAAHDJIFCAA|2023-01-14|1476|6420|493|2023|6|1|14|1|2023|493|6420|Saturday|2023Q1|N|Y|N|2459946|2459945|2459594|2459867|N|N|N|N|N| +2459960|AAAAAAAAIDJIFCAA|2023-01-15|1476|6420|493|2023|0|1|15|1|2023|493|6420|Sunday|2023Q1|N|N|N|2459946|2459945|2459595|2459868|N|N|N|N|N| +2459961|AAAAAAAAJDJIFCAA|2023-01-16|1476|6420|493|2023|1|1|16|1|2023|493|6420|Monday|2023Q1|N|N|N|2459946|2459945|2459596|2459869|N|N|N|N|N| +2459962|AAAAAAAAKDJIFCAA|2023-01-17|1476|6421|493|2023|2|1|17|1|2023|493|6421|Tuesday|2023Q1|N|N|N|2459946|2459945|2459597|2459870|N|N|N|N|N| +2459963|AAAAAAAALDJIFCAA|2023-01-18|1476|6421|493|2023|3|1|18|1|2023|493|6421|Wednesday|2023Q1|N|N|N|2459946|2459945|2459598|2459871|N|N|N|N|N| +2459964|AAAAAAAAMDJIFCAA|2023-01-19|1476|6421|493|2023|4|1|19|1|2023|493|6421|Thursday|2023Q1|N|N|N|2459946|2459945|2459599|2459872|N|N|N|N|N| +2459965|AAAAAAAANDJIFCAA|2023-01-20|1476|6421|493|2023|5|1|20|1|2023|493|6421|Friday|2023Q1|N|Y|N|2459946|2459945|2459600|2459873|N|N|N|N|N| +2459966|AAAAAAAAODJIFCAA|2023-01-21|1476|6421|493|2023|6|1|21|1|2023|493|6421|Saturday|2023Q1|N|Y|N|2459946|2459945|2459601|2459874|N|N|N|N|N| +2459967|AAAAAAAAPDJIFCAA|2023-01-22|1476|6421|493|2023|0|1|22|1|2023|493|6421|Sunday|2023Q1|N|N|N|2459946|2459945|2459602|2459875|N|N|N|N|N| +2459968|AAAAAAAAAEJIFCAA|2023-01-23|1476|6421|493|2023|1|1|23|1|2023|493|6421|Monday|2023Q1|N|N|N|2459946|2459945|2459603|2459876|N|N|N|N|N| +2459969|AAAAAAAABEJIFCAA|2023-01-24|1476|6422|493|2023|2|1|24|1|2023|493|6422|Tuesday|2023Q1|N|N|N|2459946|2459945|2459604|2459877|N|N|N|N|N| +2459970|AAAAAAAACEJIFCAA|2023-01-25|1476|6422|493|2023|3|1|25|1|2023|493|6422|Wednesday|2023Q1|N|N|N|2459946|2459945|2459605|2459878|N|N|N|N|N| +2459971|AAAAAAAADEJIFCAA|2023-01-26|1476|6422|493|2023|4|1|26|1|2023|493|6422|Thursday|2023Q1|N|N|N|2459946|2459945|2459606|2459879|N|N|N|N|N| +2459972|AAAAAAAAEEJIFCAA|2023-01-27|1476|6422|493|2023|5|1|27|1|2023|493|6422|Friday|2023Q1|N|Y|N|2459946|2459945|2459607|2459880|N|N|N|N|N| +2459973|AAAAAAAAFEJIFCAA|2023-01-28|1476|6422|493|2023|6|1|28|1|2023|493|6422|Saturday|2023Q1|N|Y|N|2459946|2459945|2459608|2459881|N|N|N|N|N| +2459974|AAAAAAAAGEJIFCAA|2023-01-29|1476|6422|493|2023|0|1|29|1|2023|493|6422|Sunday|2023Q1|N|N|N|2459946|2459945|2459609|2459882|N|N|N|N|N| +2459975|AAAAAAAAHEJIFCAA|2023-01-30|1476|6422|493|2023|1|1|30|1|2023|493|6422|Monday|2023Q1|N|N|N|2459946|2459945|2459610|2459883|N|N|N|N|N| +2459976|AAAAAAAAIEJIFCAA|2023-01-31|1476|6423|493|2023|2|1|31|1|2023|493|6423|Tuesday|2023Q1|N|N|N|2459946|2459945|2459611|2459884|N|N|N|N|N| +2459977|AAAAAAAAJEJIFCAA|2023-02-01|1477|6423|493|2023|3|2|1|1|2023|493|6423|Wednesday|2023Q1|N|N|N|2459977|2460007|2459612|2459885|N|N|N|N|N| +2459978|AAAAAAAAKEJIFCAA|2023-02-02|1477|6423|493|2023|4|2|2|1|2023|493|6423|Thursday|2023Q1|N|N|N|2459977|2460007|2459613|2459886|N|N|N|N|N| +2459979|AAAAAAAALEJIFCAA|2023-02-03|1477|6423|493|2023|5|2|3|1|2023|493|6423|Friday|2023Q1|N|Y|N|2459977|2460007|2459614|2459887|N|N|N|N|N| +2459980|AAAAAAAAMEJIFCAA|2023-02-04|1477|6423|493|2023|6|2|4|1|2023|493|6423|Saturday|2023Q1|N|Y|N|2459977|2460007|2459615|2459888|N|N|N|N|N| +2459981|AAAAAAAANEJIFCAA|2023-02-05|1477|6423|493|2023|0|2|5|1|2023|493|6423|Sunday|2023Q1|N|N|N|2459977|2460007|2459616|2459889|N|N|N|N|N| +2459982|AAAAAAAAOEJIFCAA|2023-02-06|1477|6423|493|2023|1|2|6|1|2023|493|6423|Monday|2023Q1|N|N|N|2459977|2460007|2459617|2459890|N|N|N|N|N| +2459983|AAAAAAAAPEJIFCAA|2023-02-07|1477|6424|493|2023|2|2|7|1|2023|493|6424|Tuesday|2023Q1|N|N|N|2459977|2460007|2459618|2459891|N|N|N|N|N| +2459984|AAAAAAAAAFJIFCAA|2023-02-08|1477|6424|493|2023|3|2|8|1|2023|493|6424|Wednesday|2023Q1|N|N|N|2459977|2460007|2459619|2459892|N|N|N|N|N| +2459985|AAAAAAAABFJIFCAA|2023-02-09|1477|6424|493|2023|4|2|9|1|2023|493|6424|Thursday|2023Q1|N|N|N|2459977|2460007|2459620|2459893|N|N|N|N|N| +2459986|AAAAAAAACFJIFCAA|2023-02-10|1477|6424|493|2023|5|2|10|1|2023|493|6424|Friday|2023Q1|N|Y|N|2459977|2460007|2459621|2459894|N|N|N|N|N| +2459987|AAAAAAAADFJIFCAA|2023-02-11|1477|6424|493|2023|6|2|11|1|2023|493|6424|Saturday|2023Q1|N|Y|N|2459977|2460007|2459622|2459895|N|N|N|N|N| +2459988|AAAAAAAAEFJIFCAA|2023-02-12|1477|6424|493|2023|0|2|12|1|2023|493|6424|Sunday|2023Q1|N|N|N|2459977|2460007|2459623|2459896|N|N|N|N|N| +2459989|AAAAAAAAFFJIFCAA|2023-02-13|1477|6424|493|2023|1|2|13|1|2023|493|6424|Monday|2023Q1|N|N|N|2459977|2460007|2459624|2459897|N|N|N|N|N| +2459990|AAAAAAAAGFJIFCAA|2023-02-14|1477|6425|493|2023|2|2|14|1|2023|493|6425|Tuesday|2023Q1|N|N|N|2459977|2460007|2459625|2459898|N|N|N|N|N| +2459991|AAAAAAAAHFJIFCAA|2023-02-15|1477|6425|493|2023|3|2|15|1|2023|493|6425|Wednesday|2023Q1|N|N|N|2459977|2460007|2459626|2459899|N|N|N|N|N| +2459992|AAAAAAAAIFJIFCAA|2023-02-16|1477|6425|493|2023|4|2|16|1|2023|493|6425|Thursday|2023Q1|N|N|N|2459977|2460007|2459627|2459900|N|N|N|N|N| +2459993|AAAAAAAAJFJIFCAA|2023-02-17|1477|6425|493|2023|5|2|17|1|2023|493|6425|Friday|2023Q1|N|Y|N|2459977|2460007|2459628|2459901|N|N|N|N|N| +2459994|AAAAAAAAKFJIFCAA|2023-02-18|1477|6425|493|2023|6|2|18|1|2023|493|6425|Saturday|2023Q1|N|Y|N|2459977|2460007|2459629|2459902|N|N|N|N|N| +2459995|AAAAAAAALFJIFCAA|2023-02-19|1477|6425|493|2023|0|2|19|1|2023|493|6425|Sunday|2023Q1|N|N|N|2459977|2460007|2459630|2459903|N|N|N|N|N| +2459996|AAAAAAAAMFJIFCAA|2023-02-20|1477|6425|493|2023|1|2|20|1|2023|493|6425|Monday|2023Q1|N|N|N|2459977|2460007|2459631|2459904|N|N|N|N|N| +2459997|AAAAAAAANFJIFCAA|2023-02-21|1477|6426|493|2023|2|2|21|1|2023|493|6426|Tuesday|2023Q1|N|N|N|2459977|2460007|2459632|2459905|N|N|N|N|N| +2459998|AAAAAAAAOFJIFCAA|2023-02-22|1477|6426|493|2023|3|2|22|1|2023|493|6426|Wednesday|2023Q1|N|N|N|2459977|2460007|2459633|2459906|N|N|N|N|N| +2459999|AAAAAAAAPFJIFCAA|2023-02-23|1477|6426|493|2023|4|2|23|1|2023|493|6426|Thursday|2023Q1|N|N|N|2459977|2460007|2459634|2459907|N|N|N|N|N| +2460000|AAAAAAAAAGJIFCAA|2023-02-24|1477|6426|493|2023|5|2|24|1|2023|493|6426|Friday|2023Q1|N|Y|N|2459977|2460007|2459635|2459908|N|N|N|N|N| +2460001|AAAAAAAABGJIFCAA|2023-02-25|1477|6426|493|2023|6|2|25|1|2023|493|6426|Saturday|2023Q1|N|Y|N|2459977|2460007|2459636|2459909|N|N|N|N|N| +2460002|AAAAAAAACGJIFCAA|2023-02-26|1477|6426|493|2023|0|2|26|1|2023|493|6426|Sunday|2023Q1|N|N|N|2459977|2460007|2459637|2459910|N|N|N|N|N| +2460003|AAAAAAAADGJIFCAA|2023-02-27|1477|6426|493|2023|1|2|27|1|2023|493|6426|Monday|2023Q1|N|N|N|2459977|2460007|2459638|2459911|N|N|N|N|N| +2460004|AAAAAAAAEGJIFCAA|2023-02-28|1477|6427|493|2023|2|2|28|1|2023|493|6427|Tuesday|2023Q1|N|N|N|2459977|2460007|2459639|2459912|N|N|N|N|N| +2460005|AAAAAAAAFGJIFCAA|2023-03-01|1478|6427|494|2023|3|3|1|1|2023|494|6427|Wednesday|2023Q1|N|N|N|2460005|2460063|2459640|2459913|N|N|N|N|N| +2460006|AAAAAAAAGGJIFCAA|2023-03-02|1478|6427|494|2023|4|3|2|1|2023|494|6427|Thursday|2023Q1|N|N|N|2460005|2460063|2459641|2459914|N|N|N|N|N| +2460007|AAAAAAAAHGJIFCAA|2023-03-03|1478|6427|494|2023|5|3|3|1|2023|494|6427|Friday|2023Q1|N|Y|N|2460005|2460063|2459642|2459915|N|N|N|N|N| +2460008|AAAAAAAAIGJIFCAA|2023-03-04|1478|6427|494|2023|6|3|4|1|2023|494|6427|Saturday|2023Q1|N|Y|N|2460005|2460063|2459643|2459916|N|N|N|N|N| +2460009|AAAAAAAAJGJIFCAA|2023-03-05|1478|6427|494|2023|0|3|5|1|2023|494|6427|Sunday|2023Q1|N|N|N|2460005|2460063|2459644|2459917|N|N|N|N|N| +2460010|AAAAAAAAKGJIFCAA|2023-03-06|1478|6427|494|2023|1|3|6|1|2023|494|6427|Monday|2023Q1|N|N|N|2460005|2460063|2459645|2459918|N|N|N|N|N| +2460011|AAAAAAAALGJIFCAA|2023-03-07|1478|6428|494|2023|2|3|7|1|2023|494|6428|Tuesday|2023Q1|N|N|N|2460005|2460063|2459646|2459919|N|N|N|N|N| +2460012|AAAAAAAAMGJIFCAA|2023-03-08|1478|6428|494|2023|3|3|8|1|2023|494|6428|Wednesday|2023Q1|N|N|N|2460005|2460063|2459647|2459920|N|N|N|N|N| +2460013|AAAAAAAANGJIFCAA|2023-03-09|1478|6428|494|2023|4|3|9|1|2023|494|6428|Thursday|2023Q1|N|N|N|2460005|2460063|2459648|2459921|N|N|N|N|N| +2460014|AAAAAAAAOGJIFCAA|2023-03-10|1478|6428|494|2023|5|3|10|1|2023|494|6428|Friday|2023Q1|N|Y|N|2460005|2460063|2459649|2459922|N|N|N|N|N| +2460015|AAAAAAAAPGJIFCAA|2023-03-11|1478|6428|494|2023|6|3|11|1|2023|494|6428|Saturday|2023Q1|N|Y|N|2460005|2460063|2459650|2459923|N|N|N|N|N| +2460016|AAAAAAAAAHJIFCAA|2023-03-12|1478|6428|494|2023|0|3|12|1|2023|494|6428|Sunday|2023Q1|N|N|N|2460005|2460063|2459651|2459924|N|N|N|N|N| +2460017|AAAAAAAABHJIFCAA|2023-03-13|1478|6428|494|2023|1|3|13|1|2023|494|6428|Monday|2023Q1|N|N|N|2460005|2460063|2459652|2459925|N|N|N|N|N| +2460018|AAAAAAAACHJIFCAA|2023-03-14|1478|6429|494|2023|2|3|14|1|2023|494|6429|Tuesday|2023Q1|N|N|N|2460005|2460063|2459653|2459926|N|N|N|N|N| +2460019|AAAAAAAADHJIFCAA|2023-03-15|1478|6429|494|2023|3|3|15|1|2023|494|6429|Wednesday|2023Q1|N|N|N|2460005|2460063|2459654|2459927|N|N|N|N|N| +2460020|AAAAAAAAEHJIFCAA|2023-03-16|1478|6429|494|2023|4|3|16|1|2023|494|6429|Thursday|2023Q1|N|N|N|2460005|2460063|2459655|2459928|N|N|N|N|N| +2460021|AAAAAAAAFHJIFCAA|2023-03-17|1478|6429|494|2023|5|3|17|1|2023|494|6429|Friday|2023Q1|N|Y|N|2460005|2460063|2459656|2459929|N|N|N|N|N| +2460022|AAAAAAAAGHJIFCAA|2023-03-18|1478|6429|494|2023|6|3|18|1|2023|494|6429|Saturday|2023Q1|N|Y|N|2460005|2460063|2459657|2459930|N|N|N|N|N| +2460023|AAAAAAAAHHJIFCAA|2023-03-19|1478|6429|494|2023|0|3|19|1|2023|494|6429|Sunday|2023Q1|N|N|N|2460005|2460063|2459658|2459931|N|N|N|N|N| +2460024|AAAAAAAAIHJIFCAA|2023-03-20|1478|6429|494|2023|1|3|20|1|2023|494|6429|Monday|2023Q1|N|N|N|2460005|2460063|2459659|2459932|N|N|N|N|N| +2460025|AAAAAAAAJHJIFCAA|2023-03-21|1478|6430|494|2023|2|3|21|1|2023|494|6430|Tuesday|2023Q1|N|N|N|2460005|2460063|2459660|2459933|N|N|N|N|N| +2460026|AAAAAAAAKHJIFCAA|2023-03-22|1478|6430|494|2023|3|3|22|1|2023|494|6430|Wednesday|2023Q1|N|N|N|2460005|2460063|2459661|2459934|N|N|N|N|N| +2460027|AAAAAAAALHJIFCAA|2023-03-23|1478|6430|494|2023|4|3|23|1|2023|494|6430|Thursday|2023Q1|N|N|N|2460005|2460063|2459662|2459935|N|N|N|N|N| +2460028|AAAAAAAAMHJIFCAA|2023-03-24|1478|6430|494|2023|5|3|24|1|2023|494|6430|Friday|2023Q1|N|Y|N|2460005|2460063|2459663|2459936|N|N|N|N|N| +2460029|AAAAAAAANHJIFCAA|2023-03-25|1478|6430|494|2023|6|3|25|1|2023|494|6430|Saturday|2023Q1|N|Y|N|2460005|2460063|2459664|2459937|N|N|N|N|N| +2460030|AAAAAAAAOHJIFCAA|2023-03-26|1478|6430|494|2023|0|3|26|1|2023|494|6430|Sunday|2023Q1|N|N|N|2460005|2460063|2459665|2459938|N|N|N|N|N| +2460031|AAAAAAAAPHJIFCAA|2023-03-27|1478|6430|494|2023|1|3|27|1|2023|494|6430|Monday|2023Q1|N|N|N|2460005|2460063|2459666|2459939|N|N|N|N|N| +2460032|AAAAAAAAAIJIFCAA|2023-03-28|1478|6431|494|2023|2|3|28|1|2023|494|6431|Tuesday|2023Q1|N|N|N|2460005|2460063|2459667|2459940|N|N|N|N|N| +2460033|AAAAAAAABIJIFCAA|2023-03-29|1478|6431|494|2023|3|3|29|1|2023|494|6431|Wednesday|2023Q1|N|N|N|2460005|2460063|2459668|2459941|N|N|N|N|N| +2460034|AAAAAAAACIJIFCAA|2023-03-30|1478|6431|494|2023|4|3|30|1|2023|494|6431|Thursday|2023Q1|N|N|N|2460005|2460063|2459669|2459942|N|N|N|N|N| +2460035|AAAAAAAADIJIFCAA|2023-03-31|1478|6431|494|2023|5|3|31|1|2023|494|6431|Friday|2023Q1|N|Y|N|2460005|2460063|2459670|2459943|N|N|N|N|N| +2460036|AAAAAAAAEIJIFCAA|2023-04-01|1479|6431|494|2023|6|4|1|1|2023|494|6431|Saturday|2023Q1|N|Y|N|2460036|2460125|2459671|2459946|N|N|N|N|N| +2460037|AAAAAAAAFIJIFCAA|2023-04-02|1479|6431|494|2023|0|4|2|2|2023|494|6431|Sunday|2023Q2|N|N|N|2460036|2460125|2459672|2459947|N|N|N|N|N| +2460038|AAAAAAAAGIJIFCAA|2023-04-03|1479|6431|494|2023|1|4|3|2|2023|494|6431|Monday|2023Q2|N|N|N|2460036|2460125|2459673|2459948|N|N|N|N|N| +2460039|AAAAAAAAHIJIFCAA|2023-04-04|1479|6432|494|2023|2|4|4|2|2023|494|6432|Tuesday|2023Q2|N|N|N|2460036|2460125|2459674|2459949|N|N|N|N|N| +2460040|AAAAAAAAIIJIFCAA|2023-04-05|1479|6432|494|2023|3|4|5|2|2023|494|6432|Wednesday|2023Q2|N|N|N|2460036|2460125|2459675|2459950|N|N|N|N|N| +2460041|AAAAAAAAJIJIFCAA|2023-04-06|1479|6432|494|2023|4|4|6|2|2023|494|6432|Thursday|2023Q2|N|N|N|2460036|2460125|2459676|2459951|N|N|N|N|N| +2460042|AAAAAAAAKIJIFCAA|2023-04-07|1479|6432|494|2023|5|4|7|2|2023|494|6432|Friday|2023Q2|N|Y|N|2460036|2460125|2459677|2459952|N|N|N|N|N| +2460043|AAAAAAAALIJIFCAA|2023-04-08|1479|6432|494|2023|6|4|8|2|2023|494|6432|Saturday|2023Q2|N|Y|N|2460036|2460125|2459678|2459953|N|N|N|N|N| +2460044|AAAAAAAAMIJIFCAA|2023-04-09|1479|6432|494|2023|0|4|9|2|2023|494|6432|Sunday|2023Q2|N|N|N|2460036|2460125|2459679|2459954|N|N|N|N|N| +2460045|AAAAAAAANIJIFCAA|2023-04-10|1479|6432|494|2023|1|4|10|2|2023|494|6432|Monday|2023Q2|N|N|N|2460036|2460125|2459680|2459955|N|N|N|N|N| +2460046|AAAAAAAAOIJIFCAA|2023-04-11|1479|6433|494|2023|2|4|11|2|2023|494|6433|Tuesday|2023Q2|N|N|N|2460036|2460125|2459681|2459956|N|N|N|N|N| +2460047|AAAAAAAAPIJIFCAA|2023-04-12|1479|6433|494|2023|3|4|12|2|2023|494|6433|Wednesday|2023Q2|N|N|N|2460036|2460125|2459682|2459957|N|N|N|N|N| +2460048|AAAAAAAAAJJIFCAA|2023-04-13|1479|6433|494|2023|4|4|13|2|2023|494|6433|Thursday|2023Q2|N|N|N|2460036|2460125|2459683|2459958|N|N|N|N|N| +2460049|AAAAAAAABJJIFCAA|2023-04-14|1479|6433|494|2023|5|4|14|2|2023|494|6433|Friday|2023Q2|N|Y|N|2460036|2460125|2459684|2459959|N|N|N|N|N| +2460050|AAAAAAAACJJIFCAA|2023-04-15|1479|6433|494|2023|6|4|15|2|2023|494|6433|Saturday|2023Q2|N|Y|N|2460036|2460125|2459685|2459960|N|N|N|N|N| +2460051|AAAAAAAADJJIFCAA|2023-04-16|1479|6433|494|2023|0|4|16|2|2023|494|6433|Sunday|2023Q2|N|N|N|2460036|2460125|2459686|2459961|N|N|N|N|N| +2460052|AAAAAAAAEJJIFCAA|2023-04-17|1479|6433|494|2023|1|4|17|2|2023|494|6433|Monday|2023Q2|N|N|N|2460036|2460125|2459687|2459962|N|N|N|N|N| +2460053|AAAAAAAAFJJIFCAA|2023-04-18|1479|6434|494|2023|2|4|18|2|2023|494|6434|Tuesday|2023Q2|N|N|N|2460036|2460125|2459688|2459963|N|N|N|N|N| +2460054|AAAAAAAAGJJIFCAA|2023-04-19|1479|6434|494|2023|3|4|19|2|2023|494|6434|Wednesday|2023Q2|N|N|N|2460036|2460125|2459689|2459964|N|N|N|N|N| +2460055|AAAAAAAAHJJIFCAA|2023-04-20|1479|6434|494|2023|4|4|20|2|2023|494|6434|Thursday|2023Q2|N|N|N|2460036|2460125|2459690|2459965|N|N|N|N|N| +2460056|AAAAAAAAIJJIFCAA|2023-04-21|1479|6434|494|2023|5|4|21|2|2023|494|6434|Friday|2023Q2|N|Y|N|2460036|2460125|2459691|2459966|N|N|N|N|N| +2460057|AAAAAAAAJJJIFCAA|2023-04-22|1479|6434|494|2023|6|4|22|2|2023|494|6434|Saturday|2023Q2|N|Y|N|2460036|2460125|2459692|2459967|N|N|N|N|N| +2460058|AAAAAAAAKJJIFCAA|2023-04-23|1479|6434|494|2023|0|4|23|2|2023|494|6434|Sunday|2023Q2|N|N|N|2460036|2460125|2459693|2459968|N|N|N|N|N| +2460059|AAAAAAAALJJIFCAA|2023-04-24|1479|6434|494|2023|1|4|24|2|2023|494|6434|Monday|2023Q2|N|N|N|2460036|2460125|2459694|2459969|N|N|N|N|N| +2460060|AAAAAAAAMJJIFCAA|2023-04-25|1479|6435|494|2023|2|4|25|2|2023|494|6435|Tuesday|2023Q2|N|N|N|2460036|2460125|2459695|2459970|N|N|N|N|N| +2460061|AAAAAAAANJJIFCAA|2023-04-26|1479|6435|494|2023|3|4|26|2|2023|494|6435|Wednesday|2023Q2|N|N|N|2460036|2460125|2459696|2459971|N|N|N|N|N| +2460062|AAAAAAAAOJJIFCAA|2023-04-27|1479|6435|494|2023|4|4|27|2|2023|494|6435|Thursday|2023Q2|N|N|N|2460036|2460125|2459697|2459972|N|N|N|N|N| +2460063|AAAAAAAAPJJIFCAA|2023-04-28|1479|6435|494|2023|5|4|28|2|2023|494|6435|Friday|2023Q2|N|Y|N|2460036|2460125|2459698|2459973|N|N|N|N|N| +2460064|AAAAAAAAAKJIFCAA|2023-04-29|1479|6435|494|2023|6|4|29|2|2023|494|6435|Saturday|2023Q2|N|Y|N|2460036|2460125|2459699|2459974|N|N|N|N|N| +2460065|AAAAAAAABKJIFCAA|2023-04-30|1479|6435|494|2023|0|4|30|2|2023|494|6435|Sunday|2023Q2|N|N|N|2460036|2460125|2459700|2459975|N|N|N|N|N| +2460066|AAAAAAAACKJIFCAA|2023-05-01|1480|6435|494|2023|1|5|1|2|2023|494|6435|Monday|2023Q2|N|N|N|2460066|2460185|2459701|2459976|N|N|N|N|N| +2460067|AAAAAAAADKJIFCAA|2023-05-02|1480|6436|494|2023|2|5|2|2|2023|494|6436|Tuesday|2023Q2|N|N|N|2460066|2460185|2459702|2459977|N|N|N|N|N| +2460068|AAAAAAAAEKJIFCAA|2023-05-03|1480|6436|494|2023|3|5|3|2|2023|494|6436|Wednesday|2023Q2|N|N|N|2460066|2460185|2459703|2459978|N|N|N|N|N| +2460069|AAAAAAAAFKJIFCAA|2023-05-04|1480|6436|494|2023|4|5|4|2|2023|494|6436|Thursday|2023Q2|N|N|N|2460066|2460185|2459704|2459979|N|N|N|N|N| +2460070|AAAAAAAAGKJIFCAA|2023-05-05|1480|6436|494|2023|5|5|5|2|2023|494|6436|Friday|2023Q2|N|Y|N|2460066|2460185|2459705|2459980|N|N|N|N|N| +2460071|AAAAAAAAHKJIFCAA|2023-05-06|1480|6436|494|2023|6|5|6|2|2023|494|6436|Saturday|2023Q2|N|Y|N|2460066|2460185|2459706|2459981|N|N|N|N|N| +2460072|AAAAAAAAIKJIFCAA|2023-05-07|1480|6436|494|2023|0|5|7|2|2023|494|6436|Sunday|2023Q2|N|N|N|2460066|2460185|2459707|2459982|N|N|N|N|N| +2460073|AAAAAAAAJKJIFCAA|2023-05-08|1480|6436|494|2023|1|5|8|2|2023|494|6436|Monday|2023Q2|N|N|N|2460066|2460185|2459708|2459983|N|N|N|N|N| +2460074|AAAAAAAAKKJIFCAA|2023-05-09|1480|6437|494|2023|2|5|9|2|2023|494|6437|Tuesday|2023Q2|N|N|N|2460066|2460185|2459709|2459984|N|N|N|N|N| +2460075|AAAAAAAALKJIFCAA|2023-05-10|1480|6437|494|2023|3|5|10|2|2023|494|6437|Wednesday|2023Q2|N|N|N|2460066|2460185|2459710|2459985|N|N|N|N|N| +2460076|AAAAAAAAMKJIFCAA|2023-05-11|1480|6437|494|2023|4|5|11|2|2023|494|6437|Thursday|2023Q2|N|N|N|2460066|2460185|2459711|2459986|N|N|N|N|N| +2460077|AAAAAAAANKJIFCAA|2023-05-12|1480|6437|494|2023|5|5|12|2|2023|494|6437|Friday|2023Q2|N|Y|N|2460066|2460185|2459712|2459987|N|N|N|N|N| +2460078|AAAAAAAAOKJIFCAA|2023-05-13|1480|6437|494|2023|6|5|13|2|2023|494|6437|Saturday|2023Q2|N|Y|N|2460066|2460185|2459713|2459988|N|N|N|N|N| +2460079|AAAAAAAAPKJIFCAA|2023-05-14|1480|6437|494|2023|0|5|14|2|2023|494|6437|Sunday|2023Q2|N|N|N|2460066|2460185|2459714|2459989|N|N|N|N|N| +2460080|AAAAAAAAALJIFCAA|2023-05-15|1480|6437|494|2023|1|5|15|2|2023|494|6437|Monday|2023Q2|N|N|N|2460066|2460185|2459715|2459990|N|N|N|N|N| +2460081|AAAAAAAABLJIFCAA|2023-05-16|1480|6438|494|2023|2|5|16|2|2023|494|6438|Tuesday|2023Q2|N|N|N|2460066|2460185|2459716|2459991|N|N|N|N|N| +2460082|AAAAAAAACLJIFCAA|2023-05-17|1480|6438|494|2023|3|5|17|2|2023|494|6438|Wednesday|2023Q2|N|N|N|2460066|2460185|2459717|2459992|N|N|N|N|N| +2460083|AAAAAAAADLJIFCAA|2023-05-18|1480|6438|494|2023|4|5|18|2|2023|494|6438|Thursday|2023Q2|N|N|N|2460066|2460185|2459718|2459993|N|N|N|N|N| +2460084|AAAAAAAAELJIFCAA|2023-05-19|1480|6438|494|2023|5|5|19|2|2023|494|6438|Friday|2023Q2|N|Y|N|2460066|2460185|2459719|2459994|N|N|N|N|N| +2460085|AAAAAAAAFLJIFCAA|2023-05-20|1480|6438|494|2023|6|5|20|2|2023|494|6438|Saturday|2023Q2|N|Y|N|2460066|2460185|2459720|2459995|N|N|N|N|N| +2460086|AAAAAAAAGLJIFCAA|2023-05-21|1480|6438|494|2023|0|5|21|2|2023|494|6438|Sunday|2023Q2|N|N|N|2460066|2460185|2459721|2459996|N|N|N|N|N| +2460087|AAAAAAAAHLJIFCAA|2023-05-22|1480|6438|494|2023|1|5|22|2|2023|494|6438|Monday|2023Q2|N|N|N|2460066|2460185|2459722|2459997|N|N|N|N|N| +2460088|AAAAAAAAILJIFCAA|2023-05-23|1480|6439|494|2023|2|5|23|2|2023|494|6439|Tuesday|2023Q2|N|N|N|2460066|2460185|2459723|2459998|N|N|N|N|N| +2460089|AAAAAAAAJLJIFCAA|2023-05-24|1480|6439|494|2023|3|5|24|2|2023|494|6439|Wednesday|2023Q2|N|N|N|2460066|2460185|2459724|2459999|N|N|N|N|N| +2460090|AAAAAAAAKLJIFCAA|2023-05-25|1480|6439|494|2023|4|5|25|2|2023|494|6439|Thursday|2023Q2|N|N|N|2460066|2460185|2459725|2460000|N|N|N|N|N| +2460091|AAAAAAAALLJIFCAA|2023-05-26|1480|6439|494|2023|5|5|26|2|2023|494|6439|Friday|2023Q2|N|Y|N|2460066|2460185|2459726|2460001|N|N|N|N|N| +2460092|AAAAAAAAMLJIFCAA|2023-05-27|1480|6439|494|2023|6|5|27|2|2023|494|6439|Saturday|2023Q2|N|Y|N|2460066|2460185|2459727|2460002|N|N|N|N|N| +2460093|AAAAAAAANLJIFCAA|2023-05-28|1480|6439|494|2023|0|5|28|2|2023|494|6439|Sunday|2023Q2|N|N|N|2460066|2460185|2459728|2460003|N|N|N|N|N| +2460094|AAAAAAAAOLJIFCAA|2023-05-29|1480|6439|494|2023|1|5|29|2|2023|494|6439|Monday|2023Q2|N|N|N|2460066|2460185|2459729|2460004|N|N|N|N|N| +2460095|AAAAAAAAPLJIFCAA|2023-05-30|1480|6440|494|2023|2|5|30|2|2023|494|6440|Tuesday|2023Q2|N|N|N|2460066|2460185|2459730|2460005|N|N|N|N|N| +2460096|AAAAAAAAAMJIFCAA|2023-05-31|1480|6440|494|2023|3|5|31|2|2023|494|6440|Wednesday|2023Q2|N|N|N|2460066|2460185|2459731|2460006|N|N|N|N|N| +2460097|AAAAAAAABMJIFCAA|2023-06-01|1481|6440|495|2023|4|6|1|2|2023|495|6440|Thursday|2023Q2|N|N|N|2460097|2460247|2459732|2460007|N|N|N|N|N| +2460098|AAAAAAAACMJIFCAA|2023-06-02|1481|6440|495|2023|5|6|2|2|2023|495|6440|Friday|2023Q2|N|Y|N|2460097|2460247|2459733|2460008|N|N|N|N|N| +2460099|AAAAAAAADMJIFCAA|2023-06-03|1481|6440|495|2023|6|6|3|2|2023|495|6440|Saturday|2023Q2|N|Y|N|2460097|2460247|2459734|2460009|N|N|N|N|N| +2460100|AAAAAAAAEMJIFCAA|2023-06-04|1481|6440|495|2023|0|6|4|2|2023|495|6440|Sunday|2023Q2|N|N|N|2460097|2460247|2459735|2460010|N|N|N|N|N| +2460101|AAAAAAAAFMJIFCAA|2023-06-05|1481|6440|495|2023|1|6|5|2|2023|495|6440|Monday|2023Q2|N|N|N|2460097|2460247|2459736|2460011|N|N|N|N|N| +2460102|AAAAAAAAGMJIFCAA|2023-06-06|1481|6441|495|2023|2|6|6|2|2023|495|6441|Tuesday|2023Q2|N|N|N|2460097|2460247|2459737|2460012|N|N|N|N|N| +2460103|AAAAAAAAHMJIFCAA|2023-06-07|1481|6441|495|2023|3|6|7|2|2023|495|6441|Wednesday|2023Q2|N|N|N|2460097|2460247|2459738|2460013|N|N|N|N|N| +2460104|AAAAAAAAIMJIFCAA|2023-06-08|1481|6441|495|2023|4|6|8|2|2023|495|6441|Thursday|2023Q2|N|N|N|2460097|2460247|2459739|2460014|N|N|N|N|N| +2460105|AAAAAAAAJMJIFCAA|2023-06-09|1481|6441|495|2023|5|6|9|2|2023|495|6441|Friday|2023Q2|N|Y|N|2460097|2460247|2459740|2460015|N|N|N|N|N| +2460106|AAAAAAAAKMJIFCAA|2023-06-10|1481|6441|495|2023|6|6|10|2|2023|495|6441|Saturday|2023Q2|N|Y|N|2460097|2460247|2459741|2460016|N|N|N|N|N| +2460107|AAAAAAAALMJIFCAA|2023-06-11|1481|6441|495|2023|0|6|11|2|2023|495|6441|Sunday|2023Q2|N|N|N|2460097|2460247|2459742|2460017|N|N|N|N|N| +2460108|AAAAAAAAMMJIFCAA|2023-06-12|1481|6441|495|2023|1|6|12|2|2023|495|6441|Monday|2023Q2|N|N|N|2460097|2460247|2459743|2460018|N|N|N|N|N| +2460109|AAAAAAAANMJIFCAA|2023-06-13|1481|6442|495|2023|2|6|13|2|2023|495|6442|Tuesday|2023Q2|N|N|N|2460097|2460247|2459744|2460019|N|N|N|N|N| +2460110|AAAAAAAAOMJIFCAA|2023-06-14|1481|6442|495|2023|3|6|14|2|2023|495|6442|Wednesday|2023Q2|N|N|N|2460097|2460247|2459745|2460020|N|N|N|N|N| +2460111|AAAAAAAAPMJIFCAA|2023-06-15|1481|6442|495|2023|4|6|15|2|2023|495|6442|Thursday|2023Q2|N|N|N|2460097|2460247|2459746|2460021|N|N|N|N|N| +2460112|AAAAAAAAANJIFCAA|2023-06-16|1481|6442|495|2023|5|6|16|2|2023|495|6442|Friday|2023Q2|N|Y|N|2460097|2460247|2459747|2460022|N|N|N|N|N| +2460113|AAAAAAAABNJIFCAA|2023-06-17|1481|6442|495|2023|6|6|17|2|2023|495|6442|Saturday|2023Q2|N|Y|N|2460097|2460247|2459748|2460023|N|N|N|N|N| +2460114|AAAAAAAACNJIFCAA|2023-06-18|1481|6442|495|2023|0|6|18|2|2023|495|6442|Sunday|2023Q2|N|N|N|2460097|2460247|2459749|2460024|N|N|N|N|N| +2460115|AAAAAAAADNJIFCAA|2023-06-19|1481|6442|495|2023|1|6|19|2|2023|495|6442|Monday|2023Q2|N|N|N|2460097|2460247|2459750|2460025|N|N|N|N|N| +2460116|AAAAAAAAENJIFCAA|2023-06-20|1481|6443|495|2023|2|6|20|2|2023|495|6443|Tuesday|2023Q2|N|N|N|2460097|2460247|2459751|2460026|N|N|N|N|N| +2460117|AAAAAAAAFNJIFCAA|2023-06-21|1481|6443|495|2023|3|6|21|2|2023|495|6443|Wednesday|2023Q2|N|N|N|2460097|2460247|2459752|2460027|N|N|N|N|N| +2460118|AAAAAAAAGNJIFCAA|2023-06-22|1481|6443|495|2023|4|6|22|2|2023|495|6443|Thursday|2023Q2|N|N|N|2460097|2460247|2459753|2460028|N|N|N|N|N| +2460119|AAAAAAAAHNJIFCAA|2023-06-23|1481|6443|495|2023|5|6|23|2|2023|495|6443|Friday|2023Q2|N|Y|N|2460097|2460247|2459754|2460029|N|N|N|N|N| +2460120|AAAAAAAAINJIFCAA|2023-06-24|1481|6443|495|2023|6|6|24|2|2023|495|6443|Saturday|2023Q2|N|Y|N|2460097|2460247|2459755|2460030|N|N|N|N|N| +2460121|AAAAAAAAJNJIFCAA|2023-06-25|1481|6443|495|2023|0|6|25|2|2023|495|6443|Sunday|2023Q2|N|N|N|2460097|2460247|2459756|2460031|N|N|N|N|N| +2460122|AAAAAAAAKNJIFCAA|2023-06-26|1481|6443|495|2023|1|6|26|2|2023|495|6443|Monday|2023Q2|N|N|N|2460097|2460247|2459757|2460032|N|N|N|N|N| +2460123|AAAAAAAALNJIFCAA|2023-06-27|1481|6444|495|2023|2|6|27|2|2023|495|6444|Tuesday|2023Q2|N|N|N|2460097|2460247|2459758|2460033|N|N|N|N|N| +2460124|AAAAAAAAMNJIFCAA|2023-06-28|1481|6444|495|2023|3|6|28|2|2023|495|6444|Wednesday|2023Q2|N|N|N|2460097|2460247|2459759|2460034|N|N|N|N|N| +2460125|AAAAAAAANNJIFCAA|2023-06-29|1481|6444|495|2023|4|6|29|2|2023|495|6444|Thursday|2023Q2|N|N|N|2460097|2460247|2459760|2460035|N|N|N|N|N| +2460126|AAAAAAAAONJIFCAA|2023-06-30|1481|6444|495|2023|5|6|30|2|2023|495|6444|Friday|2023Q2|N|Y|N|2460097|2460247|2459761|2460036|N|N|N|N|N| +2460127|AAAAAAAAPNJIFCAA|2023-07-01|1482|6444|495|2023|6|7|1|2|2023|495|6444|Saturday|2023Q2|N|Y|N|2460127|2460307|2459762|2460036|N|N|N|N|N| +2460128|AAAAAAAAAOJIFCAA|2023-07-02|1482|6444|495|2023|0|7|2|3|2023|495|6444|Sunday|2023Q3|N|N|N|2460127|2460307|2459763|2460037|N|N|N|N|N| +2460129|AAAAAAAABOJIFCAA|2023-07-03|1482|6444|495|2023|1|7|3|3|2023|495|6444|Monday|2023Q3|N|N|N|2460127|2460307|2459764|2460038|N|N|N|N|N| +2460130|AAAAAAAACOJIFCAA|2023-07-04|1482|6445|495|2023|2|7|4|3|2023|495|6445|Tuesday|2023Q3|N|N|N|2460127|2460307|2459765|2460039|N|N|N|N|N| +2460131|AAAAAAAADOJIFCAA|2023-07-05|1482|6445|495|2023|3|7|5|3|2023|495|6445|Wednesday|2023Q3|Y|N|N|2460127|2460307|2459766|2460040|N|N|N|N|N| +2460132|AAAAAAAAEOJIFCAA|2023-07-06|1482|6445|495|2023|4|7|6|3|2023|495|6445|Thursday|2023Q3|N|N|Y|2460127|2460307|2459767|2460041|N|N|N|N|N| +2460133|AAAAAAAAFOJIFCAA|2023-07-07|1482|6445|495|2023|5|7|7|3|2023|495|6445|Friday|2023Q3|N|Y|N|2460127|2460307|2459768|2460042|N|N|N|N|N| +2460134|AAAAAAAAGOJIFCAA|2023-07-08|1482|6445|495|2023|6|7|8|3|2023|495|6445|Saturday|2023Q3|N|Y|N|2460127|2460307|2459769|2460043|N|N|N|N|N| +2460135|AAAAAAAAHOJIFCAA|2023-07-09|1482|6445|495|2023|0|7|9|3|2023|495|6445|Sunday|2023Q3|N|N|N|2460127|2460307|2459770|2460044|N|N|N|N|N| +2460136|AAAAAAAAIOJIFCAA|2023-07-10|1482|6445|495|2023|1|7|10|3|2023|495|6445|Monday|2023Q3|N|N|N|2460127|2460307|2459771|2460045|N|N|N|N|N| +2460137|AAAAAAAAJOJIFCAA|2023-07-11|1482|6446|495|2023|2|7|11|3|2023|495|6446|Tuesday|2023Q3|N|N|N|2460127|2460307|2459772|2460046|N|N|N|N|N| +2460138|AAAAAAAAKOJIFCAA|2023-07-12|1482|6446|495|2023|3|7|12|3|2023|495|6446|Wednesday|2023Q3|N|N|N|2460127|2460307|2459773|2460047|N|N|N|N|N| +2460139|AAAAAAAALOJIFCAA|2023-07-13|1482|6446|495|2023|4|7|13|3|2023|495|6446|Thursday|2023Q3|N|N|N|2460127|2460307|2459774|2460048|N|N|N|N|N| +2460140|AAAAAAAAMOJIFCAA|2023-07-14|1482|6446|495|2023|5|7|14|3|2023|495|6446|Friday|2023Q3|N|Y|N|2460127|2460307|2459775|2460049|N|N|N|N|N| +2460141|AAAAAAAANOJIFCAA|2023-07-15|1482|6446|495|2023|6|7|15|3|2023|495|6446|Saturday|2023Q3|N|Y|N|2460127|2460307|2459776|2460050|N|N|N|N|N| +2460142|AAAAAAAAOOJIFCAA|2023-07-16|1482|6446|495|2023|0|7|16|3|2023|495|6446|Sunday|2023Q3|N|N|N|2460127|2460307|2459777|2460051|N|N|N|N|N| +2460143|AAAAAAAAPOJIFCAA|2023-07-17|1482|6446|495|2023|1|7|17|3|2023|495|6446|Monday|2023Q3|N|N|N|2460127|2460307|2459778|2460052|N|N|N|N|N| +2460144|AAAAAAAAAPJIFCAA|2023-07-18|1482|6447|495|2023|2|7|18|3|2023|495|6447|Tuesday|2023Q3|N|N|N|2460127|2460307|2459779|2460053|N|N|N|N|N| +2460145|AAAAAAAABPJIFCAA|2023-07-19|1482|6447|495|2023|3|7|19|3|2023|495|6447|Wednesday|2023Q3|N|N|N|2460127|2460307|2459780|2460054|N|N|N|N|N| +2460146|AAAAAAAACPJIFCAA|2023-07-20|1482|6447|495|2023|4|7|20|3|2023|495|6447|Thursday|2023Q3|N|N|N|2460127|2460307|2459781|2460055|N|N|N|N|N| +2460147|AAAAAAAADPJIFCAA|2023-07-21|1482|6447|495|2023|5|7|21|3|2023|495|6447|Friday|2023Q3|N|Y|N|2460127|2460307|2459782|2460056|N|N|N|N|N| +2460148|AAAAAAAAEPJIFCAA|2023-07-22|1482|6447|495|2023|6|7|22|3|2023|495|6447|Saturday|2023Q3|N|Y|N|2460127|2460307|2459783|2460057|N|N|N|N|N| +2460149|AAAAAAAAFPJIFCAA|2023-07-23|1482|6447|495|2023|0|7|23|3|2023|495|6447|Sunday|2023Q3|N|N|N|2460127|2460307|2459784|2460058|N|N|N|N|N| +2460150|AAAAAAAAGPJIFCAA|2023-07-24|1482|6447|495|2023|1|7|24|3|2023|495|6447|Monday|2023Q3|N|N|N|2460127|2460307|2459785|2460059|N|N|N|N|N| +2460151|AAAAAAAAHPJIFCAA|2023-07-25|1482|6448|495|2023|2|7|25|3|2023|495|6448|Tuesday|2023Q3|N|N|N|2460127|2460307|2459786|2460060|N|N|N|N|N| +2460152|AAAAAAAAIPJIFCAA|2023-07-26|1482|6448|495|2023|3|7|26|3|2023|495|6448|Wednesday|2023Q3|N|N|N|2460127|2460307|2459787|2460061|N|N|N|N|N| +2460153|AAAAAAAAJPJIFCAA|2023-07-27|1482|6448|495|2023|4|7|27|3|2023|495|6448|Thursday|2023Q3|N|N|N|2460127|2460307|2459788|2460062|N|N|N|N|N| +2460154|AAAAAAAAKPJIFCAA|2023-07-28|1482|6448|495|2023|5|7|28|3|2023|495|6448|Friday|2023Q3|N|Y|N|2460127|2460307|2459789|2460063|N|N|N|N|N| +2460155|AAAAAAAALPJIFCAA|2023-07-29|1482|6448|495|2023|6|7|29|3|2023|495|6448|Saturday|2023Q3|N|Y|N|2460127|2460307|2459790|2460064|N|N|N|N|N| +2460156|AAAAAAAAMPJIFCAA|2023-07-30|1482|6448|495|2023|0|7|30|3|2023|495|6448|Sunday|2023Q3|N|N|N|2460127|2460307|2459791|2460065|N|N|N|N|N| +2460157|AAAAAAAANPJIFCAA|2023-07-31|1482|6448|495|2023|1|7|31|3|2023|495|6448|Monday|2023Q3|N|N|N|2460127|2460307|2459792|2460066|N|N|N|N|N| +2460158|AAAAAAAAOPJIFCAA|2023-08-01|1483|6449|495|2023|2|8|1|3|2023|495|6449|Tuesday|2023Q3|N|N|N|2460158|2460369|2459793|2460067|N|N|N|N|N| +2460159|AAAAAAAAPPJIFCAA|2023-08-02|1483|6449|495|2023|3|8|2|3|2023|495|6449|Wednesday|2023Q3|N|N|N|2460158|2460369|2459794|2460068|N|N|N|N|N| +2460160|AAAAAAAAAAKIFCAA|2023-08-03|1483|6449|495|2023|4|8|3|3|2023|495|6449|Thursday|2023Q3|N|N|N|2460158|2460369|2459795|2460069|N|N|N|N|N| +2460161|AAAAAAAABAKIFCAA|2023-08-04|1483|6449|495|2023|5|8|4|3|2023|495|6449|Friday|2023Q3|N|Y|N|2460158|2460369|2459796|2460070|N|N|N|N|N| +2460162|AAAAAAAACAKIFCAA|2023-08-05|1483|6449|495|2023|6|8|5|3|2023|495|6449|Saturday|2023Q3|N|Y|N|2460158|2460369|2459797|2460071|N|N|N|N|N| +2460163|AAAAAAAADAKIFCAA|2023-08-06|1483|6449|495|2023|0|8|6|3|2023|495|6449|Sunday|2023Q3|N|N|N|2460158|2460369|2459798|2460072|N|N|N|N|N| +2460164|AAAAAAAAEAKIFCAA|2023-08-07|1483|6449|495|2023|1|8|7|3|2023|495|6449|Monday|2023Q3|N|N|N|2460158|2460369|2459799|2460073|N|N|N|N|N| +2460165|AAAAAAAAFAKIFCAA|2023-08-08|1483|6450|495|2023|2|8|8|3|2023|495|6450|Tuesday|2023Q3|N|N|N|2460158|2460369|2459800|2460074|N|N|N|N|N| +2460166|AAAAAAAAGAKIFCAA|2023-08-09|1483|6450|495|2023|3|8|9|3|2023|495|6450|Wednesday|2023Q3|N|N|N|2460158|2460369|2459801|2460075|N|N|N|N|N| +2460167|AAAAAAAAHAKIFCAA|2023-08-10|1483|6450|495|2023|4|8|10|3|2023|495|6450|Thursday|2023Q3|N|N|N|2460158|2460369|2459802|2460076|N|N|N|N|N| +2460168|AAAAAAAAIAKIFCAA|2023-08-11|1483|6450|495|2023|5|8|11|3|2023|495|6450|Friday|2023Q3|N|Y|N|2460158|2460369|2459803|2460077|N|N|N|N|N| +2460169|AAAAAAAAJAKIFCAA|2023-08-12|1483|6450|495|2023|6|8|12|3|2023|495|6450|Saturday|2023Q3|N|Y|N|2460158|2460369|2459804|2460078|N|N|N|N|N| +2460170|AAAAAAAAKAKIFCAA|2023-08-13|1483|6450|495|2023|0|8|13|3|2023|495|6450|Sunday|2023Q3|N|N|N|2460158|2460369|2459805|2460079|N|N|N|N|N| +2460171|AAAAAAAALAKIFCAA|2023-08-14|1483|6450|495|2023|1|8|14|3|2023|495|6450|Monday|2023Q3|N|N|N|2460158|2460369|2459806|2460080|N|N|N|N|N| +2460172|AAAAAAAAMAKIFCAA|2023-08-15|1483|6451|495|2023|2|8|15|3|2023|495|6451|Tuesday|2023Q3|N|N|N|2460158|2460369|2459807|2460081|N|N|N|N|N| +2460173|AAAAAAAANAKIFCAA|2023-08-16|1483|6451|495|2023|3|8|16|3|2023|495|6451|Wednesday|2023Q3|N|N|N|2460158|2460369|2459808|2460082|N|N|N|N|N| +2460174|AAAAAAAAOAKIFCAA|2023-08-17|1483|6451|495|2023|4|8|17|3|2023|495|6451|Thursday|2023Q3|N|N|N|2460158|2460369|2459809|2460083|N|N|N|N|N| +2460175|AAAAAAAAPAKIFCAA|2023-08-18|1483|6451|495|2023|5|8|18|3|2023|495|6451|Friday|2023Q3|N|Y|N|2460158|2460369|2459810|2460084|N|N|N|N|N| +2460176|AAAAAAAAABKIFCAA|2023-08-19|1483|6451|495|2023|6|8|19|3|2023|495|6451|Saturday|2023Q3|N|Y|N|2460158|2460369|2459811|2460085|N|N|N|N|N| +2460177|AAAAAAAABBKIFCAA|2023-08-20|1483|6451|495|2023|0|8|20|3|2023|495|6451|Sunday|2023Q3|N|N|N|2460158|2460369|2459812|2460086|N|N|N|N|N| +2460178|AAAAAAAACBKIFCAA|2023-08-21|1483|6451|495|2023|1|8|21|3|2023|495|6451|Monday|2023Q3|N|N|N|2460158|2460369|2459813|2460087|N|N|N|N|N| +2460179|AAAAAAAADBKIFCAA|2023-08-22|1483|6452|495|2023|2|8|22|3|2023|495|6452|Tuesday|2023Q3|N|N|N|2460158|2460369|2459814|2460088|N|N|N|N|N| +2460180|AAAAAAAAEBKIFCAA|2023-08-23|1483|6452|495|2023|3|8|23|3|2023|495|6452|Wednesday|2023Q3|N|N|N|2460158|2460369|2459815|2460089|N|N|N|N|N| +2460181|AAAAAAAAFBKIFCAA|2023-08-24|1483|6452|495|2023|4|8|24|3|2023|495|6452|Thursday|2023Q3|N|N|N|2460158|2460369|2459816|2460090|N|N|N|N|N| +2460182|AAAAAAAAGBKIFCAA|2023-08-25|1483|6452|495|2023|5|8|25|3|2023|495|6452|Friday|2023Q3|N|Y|N|2460158|2460369|2459817|2460091|N|N|N|N|N| +2460183|AAAAAAAAHBKIFCAA|2023-08-26|1483|6452|495|2023|6|8|26|3|2023|495|6452|Saturday|2023Q3|N|Y|N|2460158|2460369|2459818|2460092|N|N|N|N|N| +2460184|AAAAAAAAIBKIFCAA|2023-08-27|1483|6452|495|2023|0|8|27|3|2023|495|6452|Sunday|2023Q3|N|N|N|2460158|2460369|2459819|2460093|N|N|N|N|N| +2460185|AAAAAAAAJBKIFCAA|2023-08-28|1483|6452|495|2023|1|8|28|3|2023|495|6452|Monday|2023Q3|N|N|N|2460158|2460369|2459820|2460094|N|N|N|N|N| +2460186|AAAAAAAAKBKIFCAA|2023-08-29|1483|6453|495|2023|2|8|29|3|2023|495|6453|Tuesday|2023Q3|N|N|N|2460158|2460369|2459821|2460095|N|N|N|N|N| +2460187|AAAAAAAALBKIFCAA|2023-08-30|1483|6453|495|2023|3|8|30|3|2023|495|6453|Wednesday|2023Q3|N|N|N|2460158|2460369|2459822|2460096|N|N|N|N|N| +2460188|AAAAAAAAMBKIFCAA|2023-08-31|1483|6453|495|2023|4|8|31|3|2023|495|6453|Thursday|2023Q3|N|N|N|2460158|2460369|2459823|2460097|N|N|N|N|N| +2460189|AAAAAAAANBKIFCAA|2023-09-01|1484|6453|496|2023|5|9|1|3|2023|496|6453|Friday|2023Q3|N|Y|N|2460189|2460431|2459824|2460098|N|N|N|N|N| +2460190|AAAAAAAAOBKIFCAA|2023-09-02|1484|6453|496|2023|6|9|2|3|2023|496|6453|Saturday|2023Q3|N|Y|N|2460189|2460431|2459825|2460099|N|N|N|N|N| +2460191|AAAAAAAAPBKIFCAA|2023-09-03|1484|6453|496|2023|0|9|3|3|2023|496|6453|Sunday|2023Q3|N|N|N|2460189|2460431|2459826|2460100|N|N|N|N|N| +2460192|AAAAAAAAACKIFCAA|2023-09-04|1484|6453|496|2023|1|9|4|3|2023|496|6453|Monday|2023Q3|N|N|N|2460189|2460431|2459827|2460101|N|N|N|N|N| +2460193|AAAAAAAABCKIFCAA|2023-09-05|1484|6454|496|2023|2|9|5|3|2023|496|6454|Tuesday|2023Q3|N|N|N|2460189|2460431|2459828|2460102|N|N|N|N|N| +2460194|AAAAAAAACCKIFCAA|2023-09-06|1484|6454|496|2023|3|9|6|3|2023|496|6454|Wednesday|2023Q3|N|N|N|2460189|2460431|2459829|2460103|N|N|N|N|N| +2460195|AAAAAAAADCKIFCAA|2023-09-07|1484|6454|496|2023|4|9|7|3|2023|496|6454|Thursday|2023Q3|N|N|N|2460189|2460431|2459830|2460104|N|N|N|N|N| +2460196|AAAAAAAAECKIFCAA|2023-09-08|1484|6454|496|2023|5|9|8|3|2023|496|6454|Friday|2023Q3|N|Y|N|2460189|2460431|2459831|2460105|N|N|N|N|N| +2460197|AAAAAAAAFCKIFCAA|2023-09-09|1484|6454|496|2023|6|9|9|3|2023|496|6454|Saturday|2023Q3|N|Y|N|2460189|2460431|2459832|2460106|N|N|N|N|N| +2460198|AAAAAAAAGCKIFCAA|2023-09-10|1484|6454|496|2023|0|9|10|3|2023|496|6454|Sunday|2023Q3|N|N|N|2460189|2460431|2459833|2460107|N|N|N|N|N| +2460199|AAAAAAAAHCKIFCAA|2023-09-11|1484|6454|496|2023|1|9|11|3|2023|496|6454|Monday|2023Q3|N|N|N|2460189|2460431|2459834|2460108|N|N|N|N|N| +2460200|AAAAAAAAICKIFCAA|2023-09-12|1484|6455|496|2023|2|9|12|3|2023|496|6455|Tuesday|2023Q3|N|N|N|2460189|2460431|2459835|2460109|N|N|N|N|N| +2460201|AAAAAAAAJCKIFCAA|2023-09-13|1484|6455|496|2023|3|9|13|3|2023|496|6455|Wednesday|2023Q3|N|N|N|2460189|2460431|2459836|2460110|N|N|N|N|N| +2460202|AAAAAAAAKCKIFCAA|2023-09-14|1484|6455|496|2023|4|9|14|3|2023|496|6455|Thursday|2023Q3|N|N|N|2460189|2460431|2459837|2460111|N|N|N|N|N| +2460203|AAAAAAAALCKIFCAA|2023-09-15|1484|6455|496|2023|5|9|15|3|2023|496|6455|Friday|2023Q3|N|Y|N|2460189|2460431|2459838|2460112|N|N|N|N|N| +2460204|AAAAAAAAMCKIFCAA|2023-09-16|1484|6455|496|2023|6|9|16|3|2023|496|6455|Saturday|2023Q3|N|Y|N|2460189|2460431|2459839|2460113|N|N|N|N|N| +2460205|AAAAAAAANCKIFCAA|2023-09-17|1484|6455|496|2023|0|9|17|3|2023|496|6455|Sunday|2023Q3|N|N|N|2460189|2460431|2459840|2460114|N|N|N|N|N| +2460206|AAAAAAAAOCKIFCAA|2023-09-18|1484|6455|496|2023|1|9|18|3|2023|496|6455|Monday|2023Q3|N|N|N|2460189|2460431|2459841|2460115|N|N|N|N|N| +2460207|AAAAAAAAPCKIFCAA|2023-09-19|1484|6456|496|2023|2|9|19|3|2023|496|6456|Tuesday|2023Q3|N|N|N|2460189|2460431|2459842|2460116|N|N|N|N|N| +2460208|AAAAAAAAADKIFCAA|2023-09-20|1484|6456|496|2023|3|9|20|3|2023|496|6456|Wednesday|2023Q3|N|N|N|2460189|2460431|2459843|2460117|N|N|N|N|N| +2460209|AAAAAAAABDKIFCAA|2023-09-21|1484|6456|496|2023|4|9|21|3|2023|496|6456|Thursday|2023Q3|N|N|N|2460189|2460431|2459844|2460118|N|N|N|N|N| +2460210|AAAAAAAACDKIFCAA|2023-09-22|1484|6456|496|2023|5|9|22|3|2023|496|6456|Friday|2023Q3|N|Y|N|2460189|2460431|2459845|2460119|N|N|N|N|N| +2460211|AAAAAAAADDKIFCAA|2023-09-23|1484|6456|496|2023|6|9|23|3|2023|496|6456|Saturday|2023Q3|N|Y|N|2460189|2460431|2459846|2460120|N|N|N|N|N| +2460212|AAAAAAAAEDKIFCAA|2023-09-24|1484|6456|496|2023|0|9|24|3|2023|496|6456|Sunday|2023Q3|N|N|N|2460189|2460431|2459847|2460121|N|N|N|N|N| +2460213|AAAAAAAAFDKIFCAA|2023-09-25|1484|6456|496|2023|1|9|25|3|2023|496|6456|Monday|2023Q3|N|N|N|2460189|2460431|2459848|2460122|N|N|N|N|N| +2460214|AAAAAAAAGDKIFCAA|2023-09-26|1484|6457|496|2023|2|9|26|3|2023|496|6457|Tuesday|2023Q3|N|N|N|2460189|2460431|2459849|2460123|N|N|N|N|N| +2460215|AAAAAAAAHDKIFCAA|2023-09-27|1484|6457|496|2023|3|9|27|3|2023|496|6457|Wednesday|2023Q3|N|N|N|2460189|2460431|2459850|2460124|N|N|N|N|N| +2460216|AAAAAAAAIDKIFCAA|2023-09-28|1484|6457|496|2023|4|9|28|3|2023|496|6457|Thursday|2023Q3|N|N|N|2460189|2460431|2459851|2460125|N|N|N|N|N| +2460217|AAAAAAAAJDKIFCAA|2023-09-29|1484|6457|496|2023|5|9|29|3|2023|496|6457|Friday|2023Q3|N|Y|N|2460189|2460431|2459852|2460126|N|N|N|N|N| +2460218|AAAAAAAAKDKIFCAA|2023-09-30|1484|6457|496|2023|6|9|30|3|2023|496|6457|Saturday|2023Q3|N|Y|N|2460189|2460431|2459853|2460127|N|N|N|N|N| +2460219|AAAAAAAALDKIFCAA|2023-10-01|1485|6457|496|2023|0|10|1|3|2023|496|6457|Sunday|2023Q3|N|N|N|2460219|2460491|2459854|2460127|N|N|N|N|N| +2460220|AAAAAAAAMDKIFCAA|2023-10-02|1485|6457|496|2023|1|10|2|4|2023|496|6457|Monday|2023Q4|N|N|N|2460219|2460491|2459855|2460128|N|N|N|N|N| +2460221|AAAAAAAANDKIFCAA|2023-10-03|1485|6458|496|2023|2|10|3|4|2023|496|6458|Tuesday|2023Q4|N|N|N|2460219|2460491|2459856|2460129|N|N|N|N|N| +2460222|AAAAAAAAODKIFCAA|2023-10-04|1485|6458|496|2023|3|10|4|4|2023|496|6458|Wednesday|2023Q4|N|N|N|2460219|2460491|2459857|2460130|N|N|N|N|N| +2460223|AAAAAAAAPDKIFCAA|2023-10-05|1485|6458|496|2023|4|10|5|4|2023|496|6458|Thursday|2023Q4|N|N|N|2460219|2460491|2459858|2460131|N|N|N|N|N| +2460224|AAAAAAAAAEKIFCAA|2023-10-06|1485|6458|496|2023|5|10|6|4|2023|496|6458|Friday|2023Q4|N|Y|N|2460219|2460491|2459859|2460132|N|N|N|N|N| +2460225|AAAAAAAABEKIFCAA|2023-10-07|1485|6458|496|2023|6|10|7|4|2023|496|6458|Saturday|2023Q4|N|Y|N|2460219|2460491|2459860|2460133|N|N|N|N|N| +2460226|AAAAAAAACEKIFCAA|2023-10-08|1485|6458|496|2023|0|10|8|4|2023|496|6458|Sunday|2023Q4|N|N|N|2460219|2460491|2459861|2460134|N|N|N|N|N| +2460227|AAAAAAAADEKIFCAA|2023-10-09|1485|6458|496|2023|1|10|9|4|2023|496|6458|Monday|2023Q4|N|N|N|2460219|2460491|2459862|2460135|N|N|N|N|N| +2460228|AAAAAAAAEEKIFCAA|2023-10-10|1485|6459|496|2023|2|10|10|4|2023|496|6459|Tuesday|2023Q4|N|N|N|2460219|2460491|2459863|2460136|N|N|N|N|N| +2460229|AAAAAAAAFEKIFCAA|2023-10-11|1485|6459|496|2023|3|10|11|4|2023|496|6459|Wednesday|2023Q4|N|N|N|2460219|2460491|2459864|2460137|N|N|N|N|N| +2460230|AAAAAAAAGEKIFCAA|2023-10-12|1485|6459|496|2023|4|10|12|4|2023|496|6459|Thursday|2023Q4|N|N|N|2460219|2460491|2459865|2460138|N|N|N|N|N| +2460231|AAAAAAAAHEKIFCAA|2023-10-13|1485|6459|496|2023|5|10|13|4|2023|496|6459|Friday|2023Q4|N|Y|N|2460219|2460491|2459866|2460139|N|N|N|N|N| +2460232|AAAAAAAAIEKIFCAA|2023-10-14|1485|6459|496|2023|6|10|14|4|2023|496|6459|Saturday|2023Q4|N|Y|N|2460219|2460491|2459867|2460140|N|N|N|N|N| +2460233|AAAAAAAAJEKIFCAA|2023-10-15|1485|6459|496|2023|0|10|15|4|2023|496|6459|Sunday|2023Q4|N|N|N|2460219|2460491|2459868|2460141|N|N|N|N|N| +2460234|AAAAAAAAKEKIFCAA|2023-10-16|1485|6459|496|2023|1|10|16|4|2023|496|6459|Monday|2023Q4|N|N|N|2460219|2460491|2459869|2460142|N|N|N|N|N| +2460235|AAAAAAAALEKIFCAA|2023-10-17|1485|6460|496|2023|2|10|17|4|2023|496|6460|Tuesday|2023Q4|N|N|N|2460219|2460491|2459870|2460143|N|N|N|N|N| +2460236|AAAAAAAAMEKIFCAA|2023-10-18|1485|6460|496|2023|3|10|18|4|2023|496|6460|Wednesday|2023Q4|N|N|N|2460219|2460491|2459871|2460144|N|N|N|N|N| +2460237|AAAAAAAANEKIFCAA|2023-10-19|1485|6460|496|2023|4|10|19|4|2023|496|6460|Thursday|2023Q4|N|N|N|2460219|2460491|2459872|2460145|N|N|N|N|N| +2460238|AAAAAAAAOEKIFCAA|2023-10-20|1485|6460|496|2023|5|10|20|4|2023|496|6460|Friday|2023Q4|N|Y|N|2460219|2460491|2459873|2460146|N|N|N|N|N| +2460239|AAAAAAAAPEKIFCAA|2023-10-21|1485|6460|496|2023|6|10|21|4|2023|496|6460|Saturday|2023Q4|N|Y|N|2460219|2460491|2459874|2460147|N|N|N|N|N| +2460240|AAAAAAAAAFKIFCAA|2023-10-22|1485|6460|496|2023|0|10|22|4|2023|496|6460|Sunday|2023Q4|N|N|N|2460219|2460491|2459875|2460148|N|N|N|N|N| +2460241|AAAAAAAABFKIFCAA|2023-10-23|1485|6460|496|2023|1|10|23|4|2023|496|6460|Monday|2023Q4|N|N|N|2460219|2460491|2459876|2460149|N|N|N|N|N| +2460242|AAAAAAAACFKIFCAA|2023-10-24|1485|6461|496|2023|2|10|24|4|2023|496|6461|Tuesday|2023Q4|N|N|N|2460219|2460491|2459877|2460150|N|N|N|N|N| +2460243|AAAAAAAADFKIFCAA|2023-10-25|1485|6461|496|2023|3|10|25|4|2023|496|6461|Wednesday|2023Q4|N|N|N|2460219|2460491|2459878|2460151|N|N|N|N|N| +2460244|AAAAAAAAEFKIFCAA|2023-10-26|1485|6461|496|2023|4|10|26|4|2023|496|6461|Thursday|2023Q4|N|N|N|2460219|2460491|2459879|2460152|N|N|N|N|N| +2460245|AAAAAAAAFFKIFCAA|2023-10-27|1485|6461|496|2023|5|10|27|4|2023|496|6461|Friday|2023Q4|N|Y|N|2460219|2460491|2459880|2460153|N|N|N|N|N| +2460246|AAAAAAAAGFKIFCAA|2023-10-28|1485|6461|496|2023|6|10|28|4|2023|496|6461|Saturday|2023Q4|N|Y|N|2460219|2460491|2459881|2460154|N|N|N|N|N| +2460247|AAAAAAAAHFKIFCAA|2023-10-29|1485|6461|496|2023|0|10|29|4|2023|496|6461|Sunday|2023Q4|N|N|N|2460219|2460491|2459882|2460155|N|N|N|N|N| +2460248|AAAAAAAAIFKIFCAA|2023-10-30|1485|6461|496|2023|1|10|30|4|2023|496|6461|Monday|2023Q4|N|N|N|2460219|2460491|2459883|2460156|N|N|N|N|N| +2460249|AAAAAAAAJFKIFCAA|2023-10-31|1485|6462|496|2023|2|10|31|4|2023|496|6462|Tuesday|2023Q4|N|N|N|2460219|2460491|2459884|2460157|N|N|N|N|N| +2460250|AAAAAAAAKFKIFCAA|2023-11-01|1486|6462|496|2023|3|11|1|4|2023|496|6462|Wednesday|2023Q4|N|N|N|2460250|2460553|2459885|2460158|N|N|N|N|N| +2460251|AAAAAAAALFKIFCAA|2023-11-02|1486|6462|496|2023|4|11|2|4|2023|496|6462|Thursday|2023Q4|N|N|N|2460250|2460553|2459886|2460159|N|N|N|N|N| +2460252|AAAAAAAAMFKIFCAA|2023-11-03|1486|6462|496|2023|5|11|3|4|2023|496|6462|Friday|2023Q4|N|Y|N|2460250|2460553|2459887|2460160|N|N|N|N|N| +2460253|AAAAAAAANFKIFCAA|2023-11-04|1486|6462|496|2023|6|11|4|4|2023|496|6462|Saturday|2023Q4|N|Y|N|2460250|2460553|2459888|2460161|N|N|N|N|N| +2460254|AAAAAAAAOFKIFCAA|2023-11-05|1486|6462|496|2023|0|11|5|4|2023|496|6462|Sunday|2023Q4|N|N|N|2460250|2460553|2459889|2460162|N|N|N|N|N| +2460255|AAAAAAAAPFKIFCAA|2023-11-06|1486|6462|496|2023|1|11|6|4|2023|496|6462|Monday|2023Q4|N|N|N|2460250|2460553|2459890|2460163|N|N|N|N|N| +2460256|AAAAAAAAAGKIFCAA|2023-11-07|1486|6463|496|2023|2|11|7|4|2023|496|6463|Tuesday|2023Q4|N|N|N|2460250|2460553|2459891|2460164|N|N|N|N|N| +2460257|AAAAAAAABGKIFCAA|2023-11-08|1486|6463|496|2023|3|11|8|4|2023|496|6463|Wednesday|2023Q4|N|N|N|2460250|2460553|2459892|2460165|N|N|N|N|N| +2460258|AAAAAAAACGKIFCAA|2023-11-09|1486|6463|496|2023|4|11|9|4|2023|496|6463|Thursday|2023Q4|N|N|N|2460250|2460553|2459893|2460166|N|N|N|N|N| +2460259|AAAAAAAADGKIFCAA|2023-11-10|1486|6463|496|2023|5|11|10|4|2023|496|6463|Friday|2023Q4|N|Y|N|2460250|2460553|2459894|2460167|N|N|N|N|N| +2460260|AAAAAAAAEGKIFCAA|2023-11-11|1486|6463|496|2023|6|11|11|4|2023|496|6463|Saturday|2023Q4|N|Y|N|2460250|2460553|2459895|2460168|N|N|N|N|N| +2460261|AAAAAAAAFGKIFCAA|2023-11-12|1486|6463|496|2023|0|11|12|4|2023|496|6463|Sunday|2023Q4|N|N|N|2460250|2460553|2459896|2460169|N|N|N|N|N| +2460262|AAAAAAAAGGKIFCAA|2023-11-13|1486|6463|496|2023|1|11|13|4|2023|496|6463|Monday|2023Q4|N|N|N|2460250|2460553|2459897|2460170|N|N|N|N|N| +2460263|AAAAAAAAHGKIFCAA|2023-11-14|1486|6464|496|2023|2|11|14|4|2023|496|6464|Tuesday|2023Q4|N|N|N|2460250|2460553|2459898|2460171|N|N|N|N|N| +2460264|AAAAAAAAIGKIFCAA|2023-11-15|1486|6464|496|2023|3|11|15|4|2023|496|6464|Wednesday|2023Q4|N|N|N|2460250|2460553|2459899|2460172|N|N|N|N|N| +2460265|AAAAAAAAJGKIFCAA|2023-11-16|1486|6464|496|2023|4|11|16|4|2023|496|6464|Thursday|2023Q4|N|N|N|2460250|2460553|2459900|2460173|N|N|N|N|N| +2460266|AAAAAAAAKGKIFCAA|2023-11-17|1486|6464|496|2023|5|11|17|4|2023|496|6464|Friday|2023Q4|N|Y|N|2460250|2460553|2459901|2460174|N|N|N|N|N| +2460267|AAAAAAAALGKIFCAA|2023-11-18|1486|6464|496|2023|6|11|18|4|2023|496|6464|Saturday|2023Q4|N|Y|N|2460250|2460553|2459902|2460175|N|N|N|N|N| +2460268|AAAAAAAAMGKIFCAA|2023-11-19|1486|6464|496|2023|0|11|19|4|2023|496|6464|Sunday|2023Q4|N|N|N|2460250|2460553|2459903|2460176|N|N|N|N|N| +2460269|AAAAAAAANGKIFCAA|2023-11-20|1486|6464|496|2023|1|11|20|4|2023|496|6464|Monday|2023Q4|N|N|N|2460250|2460553|2459904|2460177|N|N|N|N|N| +2460270|AAAAAAAAOGKIFCAA|2023-11-21|1486|6465|496|2023|2|11|21|4|2023|496|6465|Tuesday|2023Q4|N|N|N|2460250|2460553|2459905|2460178|N|N|N|N|N| +2460271|AAAAAAAAPGKIFCAA|2023-11-22|1486|6465|496|2023|3|11|22|4|2023|496|6465|Wednesday|2023Q4|N|N|N|2460250|2460553|2459906|2460179|N|N|N|N|N| +2460272|AAAAAAAAAHKIFCAA|2023-11-23|1486|6465|496|2023|4|11|23|4|2023|496|6465|Thursday|2023Q4|N|N|N|2460250|2460553|2459907|2460180|N|N|N|N|N| +2460273|AAAAAAAABHKIFCAA|2023-11-24|1486|6465|496|2023|5|11|24|4|2023|496|6465|Friday|2023Q4|N|Y|N|2460250|2460553|2459908|2460181|N|N|N|N|N| +2460274|AAAAAAAACHKIFCAA|2023-11-25|1486|6465|496|2023|6|11|25|4|2023|496|6465|Saturday|2023Q4|N|Y|N|2460250|2460553|2459909|2460182|N|N|N|N|N| +2460275|AAAAAAAADHKIFCAA|2023-11-26|1486|6465|496|2023|0|11|26|4|2023|496|6465|Sunday|2023Q4|N|N|N|2460250|2460553|2459910|2460183|N|N|N|N|N| +2460276|AAAAAAAAEHKIFCAA|2023-11-27|1486|6465|496|2023|1|11|27|4|2023|496|6465|Monday|2023Q4|N|N|N|2460250|2460553|2459911|2460184|N|N|N|N|N| +2460277|AAAAAAAAFHKIFCAA|2023-11-28|1486|6466|496|2023|2|11|28|4|2023|496|6466|Tuesday|2023Q4|N|N|N|2460250|2460553|2459912|2460185|N|N|N|N|N| +2460278|AAAAAAAAGHKIFCAA|2023-11-29|1486|6466|496|2023|3|11|29|4|2023|496|6466|Wednesday|2023Q4|N|N|N|2460250|2460553|2459913|2460186|N|N|N|N|N| +2460279|AAAAAAAAHHKIFCAA|2023-11-30|1486|6466|496|2023|4|11|30|4|2023|496|6466|Thursday|2023Q4|N|N|N|2460250|2460553|2459914|2460187|N|N|N|N|N| +2460280|AAAAAAAAIHKIFCAA|2023-12-01|1487|6466|497|2023|5|12|1|4|2023|497|6466|Friday|2023Q4|N|Y|N|2460280|2460613|2459915|2460188|N|N|N|N|N| +2460281|AAAAAAAAJHKIFCAA|2023-12-02|1487|6466|497|2023|6|12|2|4|2023|497|6466|Saturday|2023Q4|N|Y|N|2460280|2460613|2459916|2460189|N|N|N|N|N| +2460282|AAAAAAAAKHKIFCAA|2023-12-03|1487|6466|497|2023|0|12|3|4|2023|497|6466|Sunday|2023Q4|N|N|N|2460280|2460613|2459917|2460190|N|N|N|N|N| +2460283|AAAAAAAALHKIFCAA|2023-12-04|1487|6466|497|2023|1|12|4|4|2023|497|6466|Monday|2023Q4|N|N|N|2460280|2460613|2459918|2460191|N|N|N|N|N| +2460284|AAAAAAAAMHKIFCAA|2023-12-05|1487|6467|497|2023|2|12|5|4|2023|497|6467|Tuesday|2023Q4|N|N|N|2460280|2460613|2459919|2460192|N|N|N|N|N| +2460285|AAAAAAAANHKIFCAA|2023-12-06|1487|6467|497|2023|3|12|6|4|2023|497|6467|Wednesday|2023Q4|N|N|N|2460280|2460613|2459920|2460193|N|N|N|N|N| +2460286|AAAAAAAAOHKIFCAA|2023-12-07|1487|6467|497|2023|4|12|7|4|2023|497|6467|Thursday|2023Q4|N|N|N|2460280|2460613|2459921|2460194|N|N|N|N|N| +2460287|AAAAAAAAPHKIFCAA|2023-12-08|1487|6467|497|2023|5|12|8|4|2023|497|6467|Friday|2023Q4|N|Y|N|2460280|2460613|2459922|2460195|N|N|N|N|N| +2460288|AAAAAAAAAIKIFCAA|2023-12-09|1487|6467|497|2023|6|12|9|4|2023|497|6467|Saturday|2023Q4|N|Y|N|2460280|2460613|2459923|2460196|N|N|N|N|N| +2460289|AAAAAAAABIKIFCAA|2023-12-10|1487|6467|497|2023|0|12|10|4|2023|497|6467|Sunday|2023Q4|N|N|N|2460280|2460613|2459924|2460197|N|N|N|N|N| +2460290|AAAAAAAACIKIFCAA|2023-12-11|1487|6467|497|2023|1|12|11|4|2023|497|6467|Monday|2023Q4|N|N|N|2460280|2460613|2459925|2460198|N|N|N|N|N| +2460291|AAAAAAAADIKIFCAA|2023-12-12|1487|6468|497|2023|2|12|12|4|2023|497|6468|Tuesday|2023Q4|N|N|N|2460280|2460613|2459926|2460199|N|N|N|N|N| +2460292|AAAAAAAAEIKIFCAA|2023-12-13|1487|6468|497|2023|3|12|13|4|2023|497|6468|Wednesday|2023Q4|N|N|N|2460280|2460613|2459927|2460200|N|N|N|N|N| +2460293|AAAAAAAAFIKIFCAA|2023-12-14|1487|6468|497|2023|4|12|14|4|2023|497|6468|Thursday|2023Q4|N|N|N|2460280|2460613|2459928|2460201|N|N|N|N|N| +2460294|AAAAAAAAGIKIFCAA|2023-12-15|1487|6468|497|2023|5|12|15|4|2023|497|6468|Friday|2023Q4|N|Y|N|2460280|2460613|2459929|2460202|N|N|N|N|N| +2460295|AAAAAAAAHIKIFCAA|2023-12-16|1487|6468|497|2023|6|12|16|4|2023|497|6468|Saturday|2023Q4|N|Y|N|2460280|2460613|2459930|2460203|N|N|N|N|N| +2460296|AAAAAAAAIIKIFCAA|2023-12-17|1487|6468|497|2023|0|12|17|4|2023|497|6468|Sunday|2023Q4|N|N|N|2460280|2460613|2459931|2460204|N|N|N|N|N| +2460297|AAAAAAAAJIKIFCAA|2023-12-18|1487|6468|497|2023|1|12|18|4|2023|497|6468|Monday|2023Q4|N|N|N|2460280|2460613|2459932|2460205|N|N|N|N|N| +2460298|AAAAAAAAKIKIFCAA|2023-12-19|1487|6469|497|2023|2|12|19|4|2023|497|6469|Tuesday|2023Q4|N|N|N|2460280|2460613|2459933|2460206|N|N|N|N|N| +2460299|AAAAAAAALIKIFCAA|2023-12-20|1487|6469|497|2023|3|12|20|4|2023|497|6469|Wednesday|2023Q4|N|N|N|2460280|2460613|2459934|2460207|N|N|N|N|N| +2460300|AAAAAAAAMIKIFCAA|2023-12-21|1487|6469|497|2023|4|12|21|4|2023|497|6469|Thursday|2023Q4|N|N|N|2460280|2460613|2459935|2460208|N|N|N|N|N| +2460301|AAAAAAAANIKIFCAA|2023-12-22|1487|6469|497|2023|5|12|22|4|2023|497|6469|Friday|2023Q4|N|Y|N|2460280|2460613|2459936|2460209|N|N|N|N|N| +2460302|AAAAAAAAOIKIFCAA|2023-12-23|1487|6469|497|2023|6|12|23|4|2023|497|6469|Saturday|2023Q4|N|Y|N|2460280|2460613|2459937|2460210|N|N|N|N|N| +2460303|AAAAAAAAPIKIFCAA|2023-12-24|1487|6469|497|2023|0|12|24|4|2023|497|6469|Sunday|2023Q4|N|N|N|2460280|2460613|2459938|2460211|N|N|N|N|N| +2460304|AAAAAAAAAJKIFCAA|2023-12-25|1487|6469|497|2023|1|12|25|4|2023|497|6469|Monday|2023Q4|N|N|N|2460280|2460613|2459939|2460212|N|N|N|N|N| +2460305|AAAAAAAABJKIFCAA|2023-12-26|1487|6470|497|2023|2|12|26|4|2023|497|6470|Tuesday|2023Q4|Y|N|N|2460280|2460613|2459940|2460213|N|N|N|N|N| +2460306|AAAAAAAACJKIFCAA|2023-12-27|1487|6470|497|2023|3|12|27|4|2023|497|6470|Wednesday|2023Q4|N|N|Y|2460280|2460613|2459941|2460214|N|N|N|N|N| +2460307|AAAAAAAADJKIFCAA|2023-12-28|1487|6470|497|2023|4|12|28|4|2023|497|6470|Thursday|2023Q4|N|N|N|2460280|2460613|2459942|2460215|N|N|N|N|N| +2460308|AAAAAAAAEJKIFCAA|2023-12-29|1487|6470|497|2023|5|12|29|4|2023|497|6470|Friday|2023Q4|N|Y|N|2460280|2460613|2459943|2460216|N|N|N|N|N| +2460309|AAAAAAAAFJKIFCAA|2023-12-30|1487|6470|497|2023|6|12|30|4|2023|497|6470|Saturday|2023Q4|N|Y|N|2460280|2460613|2459944|2460217|N|N|N|N|N| +2460310|AAAAAAAAGJKIFCAA|2023-12-31|1487|6470|497|2023|0|12|31|4|2023|497|6470|Sunday|2023Q4|N|N|N|2460280|2460613|2459945|2460218|N|N|N|N|N| +2460311|AAAAAAAAHJKIFCAA|2024-01-01|1488|6470|497|2024|1|1|1|1|2024|497|6470|Monday|2024Q1|Y|N|N|2460311|2460310|2459946|2460219|N|N|N|N|N| +2460312|AAAAAAAAIJKIFCAA|2024-01-02|1488|6471|497|2024|2|1|2|1|2024|497|6471|Tuesday|2024Q1|N|N|Y|2460311|2460310|2459947|2460220|N|N|N|N|N| +2460313|AAAAAAAAJJKIFCAA|2024-01-03|1488|6471|497|2024|3|1|3|1|2024|497|6471|Wednesday|2024Q1|N|N|N|2460311|2460310|2459948|2460221|N|N|N|N|N| +2460314|AAAAAAAAKJKIFCAA|2024-01-04|1488|6471|497|2024|4|1|4|1|2024|497|6471|Thursday|2024Q1|N|N|N|2460311|2460310|2459949|2460222|N|N|N|N|N| +2460315|AAAAAAAALJKIFCAA|2024-01-05|1488|6471|497|2024|5|1|5|1|2024|497|6471|Friday|2024Q1|N|Y|N|2460311|2460310|2459950|2460223|N|N|N|N|N| +2460316|AAAAAAAAMJKIFCAA|2024-01-06|1488|6471|497|2024|6|1|6|1|2024|497|6471|Saturday|2024Q1|N|Y|N|2460311|2460310|2459951|2460224|N|N|N|N|N| +2460317|AAAAAAAANJKIFCAA|2024-01-07|1488|6471|497|2024|0|1|7|1|2024|497|6471|Sunday|2024Q1|N|N|N|2460311|2460310|2459952|2460225|N|N|N|N|N| +2460318|AAAAAAAAOJKIFCAA|2024-01-08|1488|6471|497|2024|1|1|8|1|2024|497|6471|Monday|2024Q1|N|N|N|2460311|2460310|2459953|2460226|N|N|N|N|N| +2460319|AAAAAAAAPJKIFCAA|2024-01-09|1488|6472|497|2024|2|1|9|1|2024|497|6472|Tuesday|2024Q1|N|N|N|2460311|2460310|2459954|2460227|N|N|N|N|N| +2460320|AAAAAAAAAKKIFCAA|2024-01-10|1488|6472|497|2024|3|1|10|1|2024|497|6472|Wednesday|2024Q1|N|N|N|2460311|2460310|2459955|2460228|N|N|N|N|N| +2460321|AAAAAAAABKKIFCAA|2024-01-11|1488|6472|497|2024|4|1|11|1|2024|497|6472|Thursday|2024Q1|N|N|N|2460311|2460310|2459956|2460229|N|N|N|N|N| +2460322|AAAAAAAACKKIFCAA|2024-01-12|1488|6472|497|2024|5|1|12|1|2024|497|6472|Friday|2024Q1|N|Y|N|2460311|2460310|2459957|2460230|N|N|N|N|N| +2460323|AAAAAAAADKKIFCAA|2024-01-13|1488|6472|497|2024|6|1|13|1|2024|497|6472|Saturday|2024Q1|N|Y|N|2460311|2460310|2459958|2460231|N|N|N|N|N| +2460324|AAAAAAAAEKKIFCAA|2024-01-14|1488|6472|497|2024|0|1|14|1|2024|497|6472|Sunday|2024Q1|N|N|N|2460311|2460310|2459959|2460232|N|N|N|N|N| +2460325|AAAAAAAAFKKIFCAA|2024-01-15|1488|6472|497|2024|1|1|15|1|2024|497|6472|Monday|2024Q1|N|N|N|2460311|2460310|2459960|2460233|N|N|N|N|N| +2460326|AAAAAAAAGKKIFCAA|2024-01-16|1488|6473|497|2024|2|1|16|1|2024|497|6473|Tuesday|2024Q1|N|N|N|2460311|2460310|2459961|2460234|N|N|N|N|N| +2460327|AAAAAAAAHKKIFCAA|2024-01-17|1488|6473|497|2024|3|1|17|1|2024|497|6473|Wednesday|2024Q1|N|N|N|2460311|2460310|2459962|2460235|N|N|N|N|N| +2460328|AAAAAAAAIKKIFCAA|2024-01-18|1488|6473|497|2024|4|1|18|1|2024|497|6473|Thursday|2024Q1|N|N|N|2460311|2460310|2459963|2460236|N|N|N|N|N| +2460329|AAAAAAAAJKKIFCAA|2024-01-19|1488|6473|497|2024|5|1|19|1|2024|497|6473|Friday|2024Q1|N|Y|N|2460311|2460310|2459964|2460237|N|N|N|N|N| +2460330|AAAAAAAAKKKIFCAA|2024-01-20|1488|6473|497|2024|6|1|20|1|2024|497|6473|Saturday|2024Q1|N|Y|N|2460311|2460310|2459965|2460238|N|N|N|N|N| +2460331|AAAAAAAALKKIFCAA|2024-01-21|1488|6473|497|2024|0|1|21|1|2024|497|6473|Sunday|2024Q1|N|N|N|2460311|2460310|2459966|2460239|N|N|N|N|N| +2460332|AAAAAAAAMKKIFCAA|2024-01-22|1488|6473|497|2024|1|1|22|1|2024|497|6473|Monday|2024Q1|N|N|N|2460311|2460310|2459967|2460240|N|N|N|N|N| +2460333|AAAAAAAANKKIFCAA|2024-01-23|1488|6474|497|2024|2|1|23|1|2024|497|6474|Tuesday|2024Q1|N|N|N|2460311|2460310|2459968|2460241|N|N|N|N|N| +2460334|AAAAAAAAOKKIFCAA|2024-01-24|1488|6474|497|2024|3|1|24|1|2024|497|6474|Wednesday|2024Q1|N|N|N|2460311|2460310|2459969|2460242|N|N|N|N|N| +2460335|AAAAAAAAPKKIFCAA|2024-01-25|1488|6474|497|2024|4|1|25|1|2024|497|6474|Thursday|2024Q1|N|N|N|2460311|2460310|2459970|2460243|N|N|N|N|N| +2460336|AAAAAAAAALKIFCAA|2024-01-26|1488|6474|497|2024|5|1|26|1|2024|497|6474|Friday|2024Q1|N|Y|N|2460311|2460310|2459971|2460244|N|N|N|N|N| +2460337|AAAAAAAABLKIFCAA|2024-01-27|1488|6474|497|2024|6|1|27|1|2024|497|6474|Saturday|2024Q1|N|Y|N|2460311|2460310|2459972|2460245|N|N|N|N|N| +2460338|AAAAAAAACLKIFCAA|2024-01-28|1488|6474|497|2024|0|1|28|1|2024|497|6474|Sunday|2024Q1|N|N|N|2460311|2460310|2459973|2460246|N|N|N|N|N| +2460339|AAAAAAAADLKIFCAA|2024-01-29|1488|6474|497|2024|1|1|29|1|2024|497|6474|Monday|2024Q1|N|N|N|2460311|2460310|2459974|2460247|N|N|N|N|N| +2460340|AAAAAAAAELKIFCAA|2024-01-30|1488|6475|497|2024|2|1|30|1|2024|497|6475|Tuesday|2024Q1|N|N|N|2460311|2460310|2459975|2460248|N|N|N|N|N| +2460341|AAAAAAAAFLKIFCAA|2024-01-31|1488|6475|497|2024|3|1|31|1|2024|497|6475|Wednesday|2024Q1|N|N|N|2460311|2460310|2459976|2460249|N|N|N|N|N| +2460342|AAAAAAAAGLKIFCAA|2024-02-01|1489|6475|497|2024|4|2|1|1|2024|497|6475|Thursday|2024Q1|N|N|N|2460342|2460372|2459977|2460250|N|N|N|N|N| +2460343|AAAAAAAAHLKIFCAA|2024-02-02|1489|6475|497|2024|5|2|2|1|2024|497|6475|Friday|2024Q1|N|Y|N|2460342|2460372|2459978|2460251|N|N|N|N|N| +2460344|AAAAAAAAILKIFCAA|2024-02-03|1489|6475|497|2024|6|2|3|1|2024|497|6475|Saturday|2024Q1|N|Y|N|2460342|2460372|2459979|2460252|N|N|N|N|N| +2460345|AAAAAAAAJLKIFCAA|2024-02-04|1489|6475|497|2024|0|2|4|1|2024|497|6475|Sunday|2024Q1|N|N|N|2460342|2460372|2459980|2460253|N|N|N|N|N| +2460346|AAAAAAAAKLKIFCAA|2024-02-05|1489|6475|497|2024|1|2|5|1|2024|497|6475|Monday|2024Q1|N|N|N|2460342|2460372|2459981|2460254|N|N|N|N|N| +2460347|AAAAAAAALLKIFCAA|2024-02-06|1489|6476|497|2024|2|2|6|1|2024|497|6476|Tuesday|2024Q1|N|N|N|2460342|2460372|2459982|2460255|N|N|N|N|N| +2460348|AAAAAAAAMLKIFCAA|2024-02-07|1489|6476|497|2024|3|2|7|1|2024|497|6476|Wednesday|2024Q1|N|N|N|2460342|2460372|2459983|2460256|N|N|N|N|N| +2460349|AAAAAAAANLKIFCAA|2024-02-08|1489|6476|497|2024|4|2|8|1|2024|497|6476|Thursday|2024Q1|N|N|N|2460342|2460372|2459984|2460257|N|N|N|N|N| +2460350|AAAAAAAAOLKIFCAA|2024-02-09|1489|6476|497|2024|5|2|9|1|2024|497|6476|Friday|2024Q1|N|Y|N|2460342|2460372|2459985|2460258|N|N|N|N|N| +2460351|AAAAAAAAPLKIFCAA|2024-02-10|1489|6476|497|2024|6|2|10|1|2024|497|6476|Saturday|2024Q1|N|Y|N|2460342|2460372|2459986|2460259|N|N|N|N|N| +2460352|AAAAAAAAAMKIFCAA|2024-02-11|1489|6476|497|2024|0|2|11|1|2024|497|6476|Sunday|2024Q1|N|N|N|2460342|2460372|2459987|2460260|N|N|N|N|N| +2460353|AAAAAAAABMKIFCAA|2024-02-12|1489|6476|497|2024|1|2|12|1|2024|497|6476|Monday|2024Q1|N|N|N|2460342|2460372|2459988|2460261|N|N|N|N|N| +2460354|AAAAAAAACMKIFCAA|2024-02-13|1489|6477|497|2024|2|2|13|1|2024|497|6477|Tuesday|2024Q1|N|N|N|2460342|2460372|2459989|2460262|N|N|N|N|N| +2460355|AAAAAAAADMKIFCAA|2024-02-14|1489|6477|497|2024|3|2|14|1|2024|497|6477|Wednesday|2024Q1|N|N|N|2460342|2460372|2459990|2460263|N|N|N|N|N| +2460356|AAAAAAAAEMKIFCAA|2024-02-15|1489|6477|497|2024|4|2|15|1|2024|497|6477|Thursday|2024Q1|N|N|N|2460342|2460372|2459991|2460264|N|N|N|N|N| +2460357|AAAAAAAAFMKIFCAA|2024-02-16|1489|6477|497|2024|5|2|16|1|2024|497|6477|Friday|2024Q1|N|Y|N|2460342|2460372|2459992|2460265|N|N|N|N|N| +2460358|AAAAAAAAGMKIFCAA|2024-02-17|1489|6477|497|2024|6|2|17|1|2024|497|6477|Saturday|2024Q1|N|Y|N|2460342|2460372|2459993|2460266|N|N|N|N|N| +2460359|AAAAAAAAHMKIFCAA|2024-02-18|1489|6477|497|2024|0|2|18|1|2024|497|6477|Sunday|2024Q1|N|N|N|2460342|2460372|2459994|2460267|N|N|N|N|N| +2460360|AAAAAAAAIMKIFCAA|2024-02-19|1489|6477|497|2024|1|2|19|1|2024|497|6477|Monday|2024Q1|N|N|N|2460342|2460372|2459995|2460268|N|N|N|N|N| +2460361|AAAAAAAAJMKIFCAA|2024-02-20|1489|6478|497|2024|2|2|20|1|2024|497|6478|Tuesday|2024Q1|N|N|N|2460342|2460372|2459996|2460269|N|N|N|N|N| +2460362|AAAAAAAAKMKIFCAA|2024-02-21|1489|6478|497|2024|3|2|21|1|2024|497|6478|Wednesday|2024Q1|N|N|N|2460342|2460372|2459997|2460270|N|N|N|N|N| +2460363|AAAAAAAALMKIFCAA|2024-02-22|1489|6478|497|2024|4|2|22|1|2024|497|6478|Thursday|2024Q1|N|N|N|2460342|2460372|2459998|2460271|N|N|N|N|N| +2460364|AAAAAAAAMMKIFCAA|2024-02-23|1489|6478|497|2024|5|2|23|1|2024|497|6478|Friday|2024Q1|N|Y|N|2460342|2460372|2459999|2460272|N|N|N|N|N| +2460365|AAAAAAAANMKIFCAA|2024-02-24|1489|6478|497|2024|6|2|24|1|2024|497|6478|Saturday|2024Q1|N|Y|N|2460342|2460372|2460000|2460273|N|N|N|N|N| +2460366|AAAAAAAAOMKIFCAA|2024-02-25|1489|6478|497|2024|0|2|25|1|2024|497|6478|Sunday|2024Q1|N|N|N|2460342|2460372|2460001|2460274|N|N|N|N|N| +2460367|AAAAAAAAPMKIFCAA|2024-02-26|1489|6478|497|2024|1|2|26|1|2024|497|6478|Monday|2024Q1|N|N|N|2460342|2460372|2460002|2460275|N|N|N|N|N| +2460368|AAAAAAAAANKIFCAA|2024-02-27|1489|6479|497|2024|2|2|27|1|2024|497|6479|Tuesday|2024Q1|N|N|N|2460342|2460372|2460003|2460276|N|N|N|N|N| +2460369|AAAAAAAABNKIFCAA|2024-02-28|1489|6479|497|2024|3|2|28|1|2024|497|6479|Wednesday|2024Q1|N|N|N|2460342|2460372|2460004|2460277|N|N|N|N|N| +2460370|AAAAAAAACNKIFCAA|2024-02-29|1489|6479|497|2024|4|2|29|1|2024|497|6479|Thursday|2024Q1|N|N|N|2460342|2460372|2460004|2460278|N|N|N|N|N| +2460371|AAAAAAAADNKIFCAA|2024-03-01|1490|6479|498|2024|5|3|1|1|2024|498|6479|Friday|2024Q1|N|Y|N|2460371|2460430|2460005|2460279|N|N|N|N|N| +2460372|AAAAAAAAENKIFCAA|2024-03-02|1490|6479|498|2024|6|3|2|1|2024|498|6479|Saturday|2024Q1|N|Y|N|2460371|2460430|2460006|2460280|N|N|N|N|N| +2460373|AAAAAAAAFNKIFCAA|2024-03-03|1490|6479|498|2024|0|3|3|1|2024|498|6479|Sunday|2024Q1|N|N|N|2460371|2460430|2460007|2460281|N|N|N|N|N| +2460374|AAAAAAAAGNKIFCAA|2024-03-04|1490|6479|498|2024|1|3|4|1|2024|498|6479|Monday|2024Q1|N|N|N|2460371|2460430|2460008|2460282|N|N|N|N|N| +2460375|AAAAAAAAHNKIFCAA|2024-03-05|1490|6480|498|2024|2|3|5|1|2024|498|6480|Tuesday|2024Q1|N|N|N|2460371|2460430|2460009|2460283|N|N|N|N|N| +2460376|AAAAAAAAINKIFCAA|2024-03-06|1490|6480|498|2024|3|3|6|1|2024|498|6480|Wednesday|2024Q1|N|N|N|2460371|2460430|2460010|2460284|N|N|N|N|N| +2460377|AAAAAAAAJNKIFCAA|2024-03-07|1490|6480|498|2024|4|3|7|1|2024|498|6480|Thursday|2024Q1|N|N|N|2460371|2460430|2460011|2460285|N|N|N|N|N| +2460378|AAAAAAAAKNKIFCAA|2024-03-08|1490|6480|498|2024|5|3|8|1|2024|498|6480|Friday|2024Q1|N|Y|N|2460371|2460430|2460012|2460286|N|N|N|N|N| +2460379|AAAAAAAALNKIFCAA|2024-03-09|1490|6480|498|2024|6|3|9|1|2024|498|6480|Saturday|2024Q1|N|Y|N|2460371|2460430|2460013|2460287|N|N|N|N|N| +2460380|AAAAAAAAMNKIFCAA|2024-03-10|1490|6480|498|2024|0|3|10|1|2024|498|6480|Sunday|2024Q1|N|N|N|2460371|2460430|2460014|2460288|N|N|N|N|N| +2460381|AAAAAAAANNKIFCAA|2024-03-11|1490|6480|498|2024|1|3|11|1|2024|498|6480|Monday|2024Q1|N|N|N|2460371|2460430|2460015|2460289|N|N|N|N|N| +2460382|AAAAAAAAONKIFCAA|2024-03-12|1490|6481|498|2024|2|3|12|1|2024|498|6481|Tuesday|2024Q1|N|N|N|2460371|2460430|2460016|2460290|N|N|N|N|N| +2460383|AAAAAAAAPNKIFCAA|2024-03-13|1490|6481|498|2024|3|3|13|1|2024|498|6481|Wednesday|2024Q1|N|N|N|2460371|2460430|2460017|2460291|N|N|N|N|N| +2460384|AAAAAAAAAOKIFCAA|2024-03-14|1490|6481|498|2024|4|3|14|1|2024|498|6481|Thursday|2024Q1|N|N|N|2460371|2460430|2460018|2460292|N|N|N|N|N| +2460385|AAAAAAAABOKIFCAA|2024-03-15|1490|6481|498|2024|5|3|15|1|2024|498|6481|Friday|2024Q1|N|Y|N|2460371|2460430|2460019|2460293|N|N|N|N|N| +2460386|AAAAAAAACOKIFCAA|2024-03-16|1490|6481|498|2024|6|3|16|1|2024|498|6481|Saturday|2024Q1|N|Y|N|2460371|2460430|2460020|2460294|N|N|N|N|N| +2460387|AAAAAAAADOKIFCAA|2024-03-17|1490|6481|498|2024|0|3|17|1|2024|498|6481|Sunday|2024Q1|N|N|N|2460371|2460430|2460021|2460295|N|N|N|N|N| +2460388|AAAAAAAAEOKIFCAA|2024-03-18|1490|6481|498|2024|1|3|18|1|2024|498|6481|Monday|2024Q1|N|N|N|2460371|2460430|2460022|2460296|N|N|N|N|N| +2460389|AAAAAAAAFOKIFCAA|2024-03-19|1490|6482|498|2024|2|3|19|1|2024|498|6482|Tuesday|2024Q1|N|N|N|2460371|2460430|2460023|2460297|N|N|N|N|N| +2460390|AAAAAAAAGOKIFCAA|2024-03-20|1490|6482|498|2024|3|3|20|1|2024|498|6482|Wednesday|2024Q1|N|N|N|2460371|2460430|2460024|2460298|N|N|N|N|N| +2460391|AAAAAAAAHOKIFCAA|2024-03-21|1490|6482|498|2024|4|3|21|1|2024|498|6482|Thursday|2024Q1|N|N|N|2460371|2460430|2460025|2460299|N|N|N|N|N| +2460392|AAAAAAAAIOKIFCAA|2024-03-22|1490|6482|498|2024|5|3|22|1|2024|498|6482|Friday|2024Q1|N|Y|N|2460371|2460430|2460026|2460300|N|N|N|N|N| +2460393|AAAAAAAAJOKIFCAA|2024-03-23|1490|6482|498|2024|6|3|23|1|2024|498|6482|Saturday|2024Q1|N|Y|N|2460371|2460430|2460027|2460301|N|N|N|N|N| +2460394|AAAAAAAAKOKIFCAA|2024-03-24|1490|6482|498|2024|0|3|24|1|2024|498|6482|Sunday|2024Q1|N|N|N|2460371|2460430|2460028|2460302|N|N|N|N|N| +2460395|AAAAAAAALOKIFCAA|2024-03-25|1490|6482|498|2024|1|3|25|1|2024|498|6482|Monday|2024Q1|N|N|N|2460371|2460430|2460029|2460303|N|N|N|N|N| +2460396|AAAAAAAAMOKIFCAA|2024-03-26|1490|6483|498|2024|2|3|26|1|2024|498|6483|Tuesday|2024Q1|N|N|N|2460371|2460430|2460030|2460304|N|N|N|N|N| +2460397|AAAAAAAANOKIFCAA|2024-03-27|1490|6483|498|2024|3|3|27|1|2024|498|6483|Wednesday|2024Q1|N|N|N|2460371|2460430|2460031|2460305|N|N|N|N|N| +2460398|AAAAAAAAOOKIFCAA|2024-03-28|1490|6483|498|2024|4|3|28|1|2024|498|6483|Thursday|2024Q1|N|N|N|2460371|2460430|2460032|2460306|N|N|N|N|N| +2460399|AAAAAAAAPOKIFCAA|2024-03-29|1490|6483|498|2024|5|3|29|1|2024|498|6483|Friday|2024Q1|N|Y|N|2460371|2460430|2460033|2460307|N|N|N|N|N| +2460400|AAAAAAAAAPKIFCAA|2024-03-30|1490|6483|498|2024|6|3|30|1|2024|498|6483|Saturday|2024Q1|N|Y|N|2460371|2460430|2460034|2460308|N|N|N|N|N| +2460401|AAAAAAAABPKIFCAA|2024-03-31|1490|6483|498|2024|0|3|31|1|2024|498|6483|Sunday|2024Q1|N|N|N|2460371|2460430|2460035|2460309|N|N|N|N|N| +2460402|AAAAAAAACPKIFCAA|2024-04-01|1491|6483|498|2024|1|4|1|2|2024|498|6483|Monday|2024Q2|N|N|N|2460402|2460492|2460036|2460311|N|N|N|N|N| +2460403|AAAAAAAADPKIFCAA|2024-04-02|1491|6484|498|2024|2|4|2|2|2024|498|6484|Tuesday|2024Q2|N|N|N|2460402|2460492|2460037|2460312|N|N|N|N|N| +2460404|AAAAAAAAEPKIFCAA|2024-04-03|1491|6484|498|2024|3|4|3|2|2024|498|6484|Wednesday|2024Q2|N|N|N|2460402|2460492|2460038|2460313|N|N|N|N|N| +2460405|AAAAAAAAFPKIFCAA|2024-04-04|1491|6484|498|2024|4|4|4|2|2024|498|6484|Thursday|2024Q2|N|N|N|2460402|2460492|2460039|2460314|N|N|N|N|N| +2460406|AAAAAAAAGPKIFCAA|2024-04-05|1491|6484|498|2024|5|4|5|2|2024|498|6484|Friday|2024Q2|N|Y|N|2460402|2460492|2460040|2460315|N|N|N|N|N| +2460407|AAAAAAAAHPKIFCAA|2024-04-06|1491|6484|498|2024|6|4|6|2|2024|498|6484|Saturday|2024Q2|N|Y|N|2460402|2460492|2460041|2460316|N|N|N|N|N| +2460408|AAAAAAAAIPKIFCAA|2024-04-07|1491|6484|498|2024|0|4|7|2|2024|498|6484|Sunday|2024Q2|N|N|N|2460402|2460492|2460042|2460317|N|N|N|N|N| +2460409|AAAAAAAAJPKIFCAA|2024-04-08|1491|6484|498|2024|1|4|8|2|2024|498|6484|Monday|2024Q2|N|N|N|2460402|2460492|2460043|2460318|N|N|N|N|N| +2460410|AAAAAAAAKPKIFCAA|2024-04-09|1491|6485|498|2024|2|4|9|2|2024|498|6485|Tuesday|2024Q2|N|N|N|2460402|2460492|2460044|2460319|N|N|N|N|N| +2460411|AAAAAAAALPKIFCAA|2024-04-10|1491|6485|498|2024|3|4|10|2|2024|498|6485|Wednesday|2024Q2|N|N|N|2460402|2460492|2460045|2460320|N|N|N|N|N| +2460412|AAAAAAAAMPKIFCAA|2024-04-11|1491|6485|498|2024|4|4|11|2|2024|498|6485|Thursday|2024Q2|N|N|N|2460402|2460492|2460046|2460321|N|N|N|N|N| +2460413|AAAAAAAANPKIFCAA|2024-04-12|1491|6485|498|2024|5|4|12|2|2024|498|6485|Friday|2024Q2|N|Y|N|2460402|2460492|2460047|2460322|N|N|N|N|N| +2460414|AAAAAAAAOPKIFCAA|2024-04-13|1491|6485|498|2024|6|4|13|2|2024|498|6485|Saturday|2024Q2|N|Y|N|2460402|2460492|2460048|2460323|N|N|N|N|N| +2460415|AAAAAAAAPPKIFCAA|2024-04-14|1491|6485|498|2024|0|4|14|2|2024|498|6485|Sunday|2024Q2|N|N|N|2460402|2460492|2460049|2460324|N|N|N|N|N| +2460416|AAAAAAAAAALIFCAA|2024-04-15|1491|6485|498|2024|1|4|15|2|2024|498|6485|Monday|2024Q2|N|N|N|2460402|2460492|2460050|2460325|N|N|N|N|N| +2460417|AAAAAAAABALIFCAA|2024-04-16|1491|6486|498|2024|2|4|16|2|2024|498|6486|Tuesday|2024Q2|N|N|N|2460402|2460492|2460051|2460326|N|N|N|N|N| +2460418|AAAAAAAACALIFCAA|2024-04-17|1491|6486|498|2024|3|4|17|2|2024|498|6486|Wednesday|2024Q2|N|N|N|2460402|2460492|2460052|2460327|N|N|N|N|N| +2460419|AAAAAAAADALIFCAA|2024-04-18|1491|6486|498|2024|4|4|18|2|2024|498|6486|Thursday|2024Q2|N|N|N|2460402|2460492|2460053|2460328|N|N|N|N|N| +2460420|AAAAAAAAEALIFCAA|2024-04-19|1491|6486|498|2024|5|4|19|2|2024|498|6486|Friday|2024Q2|N|Y|N|2460402|2460492|2460054|2460329|N|N|N|N|N| +2460421|AAAAAAAAFALIFCAA|2024-04-20|1491|6486|498|2024|6|4|20|2|2024|498|6486|Saturday|2024Q2|N|Y|N|2460402|2460492|2460055|2460330|N|N|N|N|N| +2460422|AAAAAAAAGALIFCAA|2024-04-21|1491|6486|498|2024|0|4|21|2|2024|498|6486|Sunday|2024Q2|N|N|N|2460402|2460492|2460056|2460331|N|N|N|N|N| +2460423|AAAAAAAAHALIFCAA|2024-04-22|1491|6486|498|2024|1|4|22|2|2024|498|6486|Monday|2024Q2|N|N|N|2460402|2460492|2460057|2460332|N|N|N|N|N| +2460424|AAAAAAAAIALIFCAA|2024-04-23|1491|6487|498|2024|2|4|23|2|2024|498|6487|Tuesday|2024Q2|N|N|N|2460402|2460492|2460058|2460333|N|N|N|N|N| +2460425|AAAAAAAAJALIFCAA|2024-04-24|1491|6487|498|2024|3|4|24|2|2024|498|6487|Wednesday|2024Q2|N|N|N|2460402|2460492|2460059|2460334|N|N|N|N|N| +2460426|AAAAAAAAKALIFCAA|2024-04-25|1491|6487|498|2024|4|4|25|2|2024|498|6487|Thursday|2024Q2|N|N|N|2460402|2460492|2460060|2460335|N|N|N|N|N| +2460427|AAAAAAAALALIFCAA|2024-04-26|1491|6487|498|2024|5|4|26|2|2024|498|6487|Friday|2024Q2|N|Y|N|2460402|2460492|2460061|2460336|N|N|N|N|N| +2460428|AAAAAAAAMALIFCAA|2024-04-27|1491|6487|498|2024|6|4|27|2|2024|498|6487|Saturday|2024Q2|N|Y|N|2460402|2460492|2460062|2460337|N|N|N|N|N| +2460429|AAAAAAAANALIFCAA|2024-04-28|1491|6487|498|2024|0|4|28|2|2024|498|6487|Sunday|2024Q2|N|N|N|2460402|2460492|2460063|2460338|N|N|N|N|N| +2460430|AAAAAAAAOALIFCAA|2024-04-29|1491|6487|498|2024|1|4|29|2|2024|498|6487|Monday|2024Q2|N|N|N|2460402|2460492|2460064|2460339|N|N|N|N|N| +2460431|AAAAAAAAPALIFCAA|2024-04-30|1491|6488|498|2024|2|4|30|2|2024|498|6488|Tuesday|2024Q2|N|N|N|2460402|2460492|2460065|2460340|N|N|N|N|N| +2460432|AAAAAAAAABLIFCAA|2024-05-01|1492|6488|498|2024|3|5|1|2|2024|498|6488|Wednesday|2024Q2|N|N|N|2460432|2460552|2460066|2460341|N|N|N|N|N| +2460433|AAAAAAAABBLIFCAA|2024-05-02|1492|6488|498|2024|4|5|2|2|2024|498|6488|Thursday|2024Q2|N|N|N|2460432|2460552|2460067|2460342|N|N|N|N|N| +2460434|AAAAAAAACBLIFCAA|2024-05-03|1492|6488|498|2024|5|5|3|2|2024|498|6488|Friday|2024Q2|N|Y|N|2460432|2460552|2460068|2460343|N|N|N|N|N| +2460435|AAAAAAAADBLIFCAA|2024-05-04|1492|6488|498|2024|6|5|4|2|2024|498|6488|Saturday|2024Q2|N|Y|N|2460432|2460552|2460069|2460344|N|N|N|N|N| +2460436|AAAAAAAAEBLIFCAA|2024-05-05|1492|6488|498|2024|0|5|5|2|2024|498|6488|Sunday|2024Q2|N|N|N|2460432|2460552|2460070|2460345|N|N|N|N|N| +2460437|AAAAAAAAFBLIFCAA|2024-05-06|1492|6488|498|2024|1|5|6|2|2024|498|6488|Monday|2024Q2|N|N|N|2460432|2460552|2460071|2460346|N|N|N|N|N| +2460438|AAAAAAAAGBLIFCAA|2024-05-07|1492|6489|498|2024|2|5|7|2|2024|498|6489|Tuesday|2024Q2|N|N|N|2460432|2460552|2460072|2460347|N|N|N|N|N| +2460439|AAAAAAAAHBLIFCAA|2024-05-08|1492|6489|498|2024|3|5|8|2|2024|498|6489|Wednesday|2024Q2|N|N|N|2460432|2460552|2460073|2460348|N|N|N|N|N| +2460440|AAAAAAAAIBLIFCAA|2024-05-09|1492|6489|498|2024|4|5|9|2|2024|498|6489|Thursday|2024Q2|N|N|N|2460432|2460552|2460074|2460349|N|N|N|N|N| +2460441|AAAAAAAAJBLIFCAA|2024-05-10|1492|6489|498|2024|5|5|10|2|2024|498|6489|Friday|2024Q2|N|Y|N|2460432|2460552|2460075|2460350|N|N|N|N|N| +2460442|AAAAAAAAKBLIFCAA|2024-05-11|1492|6489|498|2024|6|5|11|2|2024|498|6489|Saturday|2024Q2|N|Y|N|2460432|2460552|2460076|2460351|N|N|N|N|N| +2460443|AAAAAAAALBLIFCAA|2024-05-12|1492|6489|498|2024|0|5|12|2|2024|498|6489|Sunday|2024Q2|N|N|N|2460432|2460552|2460077|2460352|N|N|N|N|N| +2460444|AAAAAAAAMBLIFCAA|2024-05-13|1492|6489|498|2024|1|5|13|2|2024|498|6489|Monday|2024Q2|N|N|N|2460432|2460552|2460078|2460353|N|N|N|N|N| +2460445|AAAAAAAANBLIFCAA|2024-05-14|1492|6490|498|2024|2|5|14|2|2024|498|6490|Tuesday|2024Q2|N|N|N|2460432|2460552|2460079|2460354|N|N|N|N|N| +2460446|AAAAAAAAOBLIFCAA|2024-05-15|1492|6490|498|2024|3|5|15|2|2024|498|6490|Wednesday|2024Q2|N|N|N|2460432|2460552|2460080|2460355|N|N|N|N|N| +2460447|AAAAAAAAPBLIFCAA|2024-05-16|1492|6490|498|2024|4|5|16|2|2024|498|6490|Thursday|2024Q2|N|N|N|2460432|2460552|2460081|2460356|N|N|N|N|N| +2460448|AAAAAAAAACLIFCAA|2024-05-17|1492|6490|498|2024|5|5|17|2|2024|498|6490|Friday|2024Q2|N|Y|N|2460432|2460552|2460082|2460357|N|N|N|N|N| +2460449|AAAAAAAABCLIFCAA|2024-05-18|1492|6490|498|2024|6|5|18|2|2024|498|6490|Saturday|2024Q2|N|Y|N|2460432|2460552|2460083|2460358|N|N|N|N|N| +2460450|AAAAAAAACCLIFCAA|2024-05-19|1492|6490|498|2024|0|5|19|2|2024|498|6490|Sunday|2024Q2|N|N|N|2460432|2460552|2460084|2460359|N|N|N|N|N| +2460451|AAAAAAAADCLIFCAA|2024-05-20|1492|6490|498|2024|1|5|20|2|2024|498|6490|Monday|2024Q2|N|N|N|2460432|2460552|2460085|2460360|N|N|N|N|N| +2460452|AAAAAAAAECLIFCAA|2024-05-21|1492|6491|498|2024|2|5|21|2|2024|498|6491|Tuesday|2024Q2|N|N|N|2460432|2460552|2460086|2460361|N|N|N|N|N| +2460453|AAAAAAAAFCLIFCAA|2024-05-22|1492|6491|498|2024|3|5|22|2|2024|498|6491|Wednesday|2024Q2|N|N|N|2460432|2460552|2460087|2460362|N|N|N|N|N| +2460454|AAAAAAAAGCLIFCAA|2024-05-23|1492|6491|498|2024|4|5|23|2|2024|498|6491|Thursday|2024Q2|N|N|N|2460432|2460552|2460088|2460363|N|N|N|N|N| +2460455|AAAAAAAAHCLIFCAA|2024-05-24|1492|6491|498|2024|5|5|24|2|2024|498|6491|Friday|2024Q2|N|Y|N|2460432|2460552|2460089|2460364|N|N|N|N|N| +2460456|AAAAAAAAICLIFCAA|2024-05-25|1492|6491|498|2024|6|5|25|2|2024|498|6491|Saturday|2024Q2|N|Y|N|2460432|2460552|2460090|2460365|N|N|N|N|N| +2460457|AAAAAAAAJCLIFCAA|2024-05-26|1492|6491|498|2024|0|5|26|2|2024|498|6491|Sunday|2024Q2|N|N|N|2460432|2460552|2460091|2460366|N|N|N|N|N| +2460458|AAAAAAAAKCLIFCAA|2024-05-27|1492|6491|498|2024|1|5|27|2|2024|498|6491|Monday|2024Q2|N|N|N|2460432|2460552|2460092|2460367|N|N|N|N|N| +2460459|AAAAAAAALCLIFCAA|2024-05-28|1492|6492|498|2024|2|5|28|2|2024|498|6492|Tuesday|2024Q2|N|N|N|2460432|2460552|2460093|2460368|N|N|N|N|N| +2460460|AAAAAAAAMCLIFCAA|2024-05-29|1492|6492|498|2024|3|5|29|2|2024|498|6492|Wednesday|2024Q2|N|N|N|2460432|2460552|2460094|2460369|N|N|N|N|N| +2460461|AAAAAAAANCLIFCAA|2024-05-30|1492|6492|498|2024|4|5|30|2|2024|498|6492|Thursday|2024Q2|N|N|N|2460432|2460552|2460095|2460370|N|N|N|N|N| +2460462|AAAAAAAAOCLIFCAA|2024-05-31|1492|6492|498|2024|5|5|31|2|2024|498|6492|Friday|2024Q2|N|Y|N|2460432|2460552|2460096|2460371|N|N|N|N|N| +2460463|AAAAAAAAPCLIFCAA|2024-06-01|1493|6492|499|2024|6|6|1|2|2024|499|6492|Saturday|2024Q2|N|Y|N|2460463|2460614|2460097|2460372|N|N|N|N|N| +2460464|AAAAAAAAADLIFCAA|2024-06-02|1493|6492|499|2024|0|6|2|2|2024|499|6492|Sunday|2024Q2|N|N|N|2460463|2460614|2460098|2460373|N|N|N|N|N| +2460465|AAAAAAAABDLIFCAA|2024-06-03|1493|6492|499|2024|1|6|3|2|2024|499|6492|Monday|2024Q2|N|N|N|2460463|2460614|2460099|2460374|N|N|N|N|N| +2460466|AAAAAAAACDLIFCAA|2024-06-04|1493|6493|499|2024|2|6|4|2|2024|499|6493|Tuesday|2024Q2|N|N|N|2460463|2460614|2460100|2460375|N|N|N|N|N| +2460467|AAAAAAAADDLIFCAA|2024-06-05|1493|6493|499|2024|3|6|5|2|2024|499|6493|Wednesday|2024Q2|N|N|N|2460463|2460614|2460101|2460376|N|N|N|N|N| +2460468|AAAAAAAAEDLIFCAA|2024-06-06|1493|6493|499|2024|4|6|6|2|2024|499|6493|Thursday|2024Q2|N|N|N|2460463|2460614|2460102|2460377|N|N|N|N|N| +2460469|AAAAAAAAFDLIFCAA|2024-06-07|1493|6493|499|2024|5|6|7|2|2024|499|6493|Friday|2024Q2|N|Y|N|2460463|2460614|2460103|2460378|N|N|N|N|N| +2460470|AAAAAAAAGDLIFCAA|2024-06-08|1493|6493|499|2024|6|6|8|2|2024|499|6493|Saturday|2024Q2|N|Y|N|2460463|2460614|2460104|2460379|N|N|N|N|N| +2460471|AAAAAAAAHDLIFCAA|2024-06-09|1493|6493|499|2024|0|6|9|2|2024|499|6493|Sunday|2024Q2|N|N|N|2460463|2460614|2460105|2460380|N|N|N|N|N| +2460472|AAAAAAAAIDLIFCAA|2024-06-10|1493|6493|499|2024|1|6|10|2|2024|499|6493|Monday|2024Q2|N|N|N|2460463|2460614|2460106|2460381|N|N|N|N|N| +2460473|AAAAAAAAJDLIFCAA|2024-06-11|1493|6494|499|2024|2|6|11|2|2024|499|6494|Tuesday|2024Q2|N|N|N|2460463|2460614|2460107|2460382|N|N|N|N|N| +2460474|AAAAAAAAKDLIFCAA|2024-06-12|1493|6494|499|2024|3|6|12|2|2024|499|6494|Wednesday|2024Q2|N|N|N|2460463|2460614|2460108|2460383|N|N|N|N|N| +2460475|AAAAAAAALDLIFCAA|2024-06-13|1493|6494|499|2024|4|6|13|2|2024|499|6494|Thursday|2024Q2|N|N|N|2460463|2460614|2460109|2460384|N|N|N|N|N| +2460476|AAAAAAAAMDLIFCAA|2024-06-14|1493|6494|499|2024|5|6|14|2|2024|499|6494|Friday|2024Q2|N|Y|N|2460463|2460614|2460110|2460385|N|N|N|N|N| +2460477|AAAAAAAANDLIFCAA|2024-06-15|1493|6494|499|2024|6|6|15|2|2024|499|6494|Saturday|2024Q2|N|Y|N|2460463|2460614|2460111|2460386|N|N|N|N|N| +2460478|AAAAAAAAODLIFCAA|2024-06-16|1493|6494|499|2024|0|6|16|2|2024|499|6494|Sunday|2024Q2|N|N|N|2460463|2460614|2460112|2460387|N|N|N|N|N| +2460479|AAAAAAAAPDLIFCAA|2024-06-17|1493|6494|499|2024|1|6|17|2|2024|499|6494|Monday|2024Q2|N|N|N|2460463|2460614|2460113|2460388|N|N|N|N|N| +2460480|AAAAAAAAAELIFCAA|2024-06-18|1493|6495|499|2024|2|6|18|2|2024|499|6495|Tuesday|2024Q2|N|N|N|2460463|2460614|2460114|2460389|N|N|N|N|N| +2460481|AAAAAAAABELIFCAA|2024-06-19|1493|6495|499|2024|3|6|19|2|2024|499|6495|Wednesday|2024Q2|N|N|N|2460463|2460614|2460115|2460390|N|N|N|N|N| +2460482|AAAAAAAACELIFCAA|2024-06-20|1493|6495|499|2024|4|6|20|2|2024|499|6495|Thursday|2024Q2|N|N|N|2460463|2460614|2460116|2460391|N|N|N|N|N| +2460483|AAAAAAAADELIFCAA|2024-06-21|1493|6495|499|2024|5|6|21|2|2024|499|6495|Friday|2024Q2|N|Y|N|2460463|2460614|2460117|2460392|N|N|N|N|N| +2460484|AAAAAAAAEELIFCAA|2024-06-22|1493|6495|499|2024|6|6|22|2|2024|499|6495|Saturday|2024Q2|N|Y|N|2460463|2460614|2460118|2460393|N|N|N|N|N| +2460485|AAAAAAAAFELIFCAA|2024-06-23|1493|6495|499|2024|0|6|23|2|2024|499|6495|Sunday|2024Q2|N|N|N|2460463|2460614|2460119|2460394|N|N|N|N|N| +2460486|AAAAAAAAGELIFCAA|2024-06-24|1493|6495|499|2024|1|6|24|2|2024|499|6495|Monday|2024Q2|N|N|N|2460463|2460614|2460120|2460395|N|N|N|N|N| +2460487|AAAAAAAAHELIFCAA|2024-06-25|1493|6496|499|2024|2|6|25|2|2024|499|6496|Tuesday|2024Q2|N|N|N|2460463|2460614|2460121|2460396|N|N|N|N|N| +2460488|AAAAAAAAIELIFCAA|2024-06-26|1493|6496|499|2024|3|6|26|2|2024|499|6496|Wednesday|2024Q2|N|N|N|2460463|2460614|2460122|2460397|N|N|N|N|N| +2460489|AAAAAAAAJELIFCAA|2024-06-27|1493|6496|499|2024|4|6|27|2|2024|499|6496|Thursday|2024Q2|N|N|N|2460463|2460614|2460123|2460398|N|N|N|N|N| +2460490|AAAAAAAAKELIFCAA|2024-06-28|1493|6496|499|2024|5|6|28|2|2024|499|6496|Friday|2024Q2|N|Y|N|2460463|2460614|2460124|2460399|N|N|N|N|N| +2460491|AAAAAAAALELIFCAA|2024-06-29|1493|6496|499|2024|6|6|29|2|2024|499|6496|Saturday|2024Q2|N|Y|N|2460463|2460614|2460125|2460400|N|N|N|N|N| +2460492|AAAAAAAAMELIFCAA|2024-06-30|1493|6496|499|2024|0|6|30|2|2024|499|6496|Sunday|2024Q2|N|N|N|2460463|2460614|2460126|2460401|N|N|N|N|N| +2460493|AAAAAAAANELIFCAA|2024-07-01|1494|6496|499|2024|1|7|1|3|2024|499|6496|Monday|2024Q3|N|N|N|2460493|2460674|2460127|2460402|N|N|N|N|N| +2460494|AAAAAAAAOELIFCAA|2024-07-02|1494|6497|499|2024|2|7|2|3|2024|499|6497|Tuesday|2024Q3|N|N|N|2460493|2460674|2460128|2460403|N|N|N|N|N| +2460495|AAAAAAAAPELIFCAA|2024-07-03|1494|6497|499|2024|3|7|3|3|2024|499|6497|Wednesday|2024Q3|N|N|N|2460493|2460674|2460129|2460404|N|N|N|N|N| +2460496|AAAAAAAAAFLIFCAA|2024-07-04|1494|6497|499|2024|4|7|4|3|2024|499|6497|Thursday|2024Q3|Y|N|N|2460493|2460674|2460130|2460405|N|N|N|N|N| +2460497|AAAAAAAABFLIFCAA|2024-07-05|1494|6497|499|2024|5|7|5|3|2024|499|6497|Friday|2024Q3|N|Y|Y|2460493|2460674|2460131|2460406|N|N|N|N|N| +2460498|AAAAAAAACFLIFCAA|2024-07-06|1494|6497|499|2024|6|7|6|3|2024|499|6497|Saturday|2024Q3|N|Y|N|2460493|2460674|2460132|2460407|N|N|N|N|N| +2460499|AAAAAAAADFLIFCAA|2024-07-07|1494|6497|499|2024|0|7|7|3|2024|499|6497|Sunday|2024Q3|N|N|N|2460493|2460674|2460133|2460408|N|N|N|N|N| +2460500|AAAAAAAAEFLIFCAA|2024-07-08|1494|6497|499|2024|1|7|8|3|2024|499|6497|Monday|2024Q3|N|N|N|2460493|2460674|2460134|2460409|N|N|N|N|N| +2460501|AAAAAAAAFFLIFCAA|2024-07-09|1494|6498|499|2024|2|7|9|3|2024|499|6498|Tuesday|2024Q3|N|N|N|2460493|2460674|2460135|2460410|N|N|N|N|N| +2460502|AAAAAAAAGFLIFCAA|2024-07-10|1494|6498|499|2024|3|7|10|3|2024|499|6498|Wednesday|2024Q3|N|N|N|2460493|2460674|2460136|2460411|N|N|N|N|N| +2460503|AAAAAAAAHFLIFCAA|2024-07-11|1494|6498|499|2024|4|7|11|3|2024|499|6498|Thursday|2024Q3|N|N|N|2460493|2460674|2460137|2460412|N|N|N|N|N| +2460504|AAAAAAAAIFLIFCAA|2024-07-12|1494|6498|499|2024|5|7|12|3|2024|499|6498|Friday|2024Q3|N|Y|N|2460493|2460674|2460138|2460413|N|N|N|N|N| +2460505|AAAAAAAAJFLIFCAA|2024-07-13|1494|6498|499|2024|6|7|13|3|2024|499|6498|Saturday|2024Q3|N|Y|N|2460493|2460674|2460139|2460414|N|N|N|N|N| +2460506|AAAAAAAAKFLIFCAA|2024-07-14|1494|6498|499|2024|0|7|14|3|2024|499|6498|Sunday|2024Q3|N|N|N|2460493|2460674|2460140|2460415|N|N|N|N|N| +2460507|AAAAAAAALFLIFCAA|2024-07-15|1494|6498|499|2024|1|7|15|3|2024|499|6498|Monday|2024Q3|N|N|N|2460493|2460674|2460141|2460416|N|N|N|N|N| +2460508|AAAAAAAAMFLIFCAA|2024-07-16|1494|6499|499|2024|2|7|16|3|2024|499|6499|Tuesday|2024Q3|N|N|N|2460493|2460674|2460142|2460417|N|N|N|N|N| +2460509|AAAAAAAANFLIFCAA|2024-07-17|1494|6499|499|2024|3|7|17|3|2024|499|6499|Wednesday|2024Q3|N|N|N|2460493|2460674|2460143|2460418|N|N|N|N|N| +2460510|AAAAAAAAOFLIFCAA|2024-07-18|1494|6499|499|2024|4|7|18|3|2024|499|6499|Thursday|2024Q3|N|N|N|2460493|2460674|2460144|2460419|N|N|N|N|N| +2460511|AAAAAAAAPFLIFCAA|2024-07-19|1494|6499|499|2024|5|7|19|3|2024|499|6499|Friday|2024Q3|N|Y|N|2460493|2460674|2460145|2460420|N|N|N|N|N| +2460512|AAAAAAAAAGLIFCAA|2024-07-20|1494|6499|499|2024|6|7|20|3|2024|499|6499|Saturday|2024Q3|N|Y|N|2460493|2460674|2460146|2460421|N|N|N|N|N| +2460513|AAAAAAAABGLIFCAA|2024-07-21|1494|6499|499|2024|0|7|21|3|2024|499|6499|Sunday|2024Q3|N|N|N|2460493|2460674|2460147|2460422|N|N|N|N|N| +2460514|AAAAAAAACGLIFCAA|2024-07-22|1494|6499|499|2024|1|7|22|3|2024|499|6499|Monday|2024Q3|N|N|N|2460493|2460674|2460148|2460423|N|N|N|N|N| +2460515|AAAAAAAADGLIFCAA|2024-07-23|1494|6500|499|2024|2|7|23|3|2024|499|6500|Tuesday|2024Q3|N|N|N|2460493|2460674|2460149|2460424|N|N|N|N|N| +2460516|AAAAAAAAEGLIFCAA|2024-07-24|1494|6500|499|2024|3|7|24|3|2024|499|6500|Wednesday|2024Q3|N|N|N|2460493|2460674|2460150|2460425|N|N|N|N|N| +2460517|AAAAAAAAFGLIFCAA|2024-07-25|1494|6500|499|2024|4|7|25|3|2024|499|6500|Thursday|2024Q3|N|N|N|2460493|2460674|2460151|2460426|N|N|N|N|N| +2460518|AAAAAAAAGGLIFCAA|2024-07-26|1494|6500|499|2024|5|7|26|3|2024|499|6500|Friday|2024Q3|N|Y|N|2460493|2460674|2460152|2460427|N|N|N|N|N| +2460519|AAAAAAAAHGLIFCAA|2024-07-27|1494|6500|499|2024|6|7|27|3|2024|499|6500|Saturday|2024Q3|N|Y|N|2460493|2460674|2460153|2460428|N|N|N|N|N| +2460520|AAAAAAAAIGLIFCAA|2024-07-28|1494|6500|499|2024|0|7|28|3|2024|499|6500|Sunday|2024Q3|N|N|N|2460493|2460674|2460154|2460429|N|N|N|N|N| +2460521|AAAAAAAAJGLIFCAA|2024-07-29|1494|6500|499|2024|1|7|29|3|2024|499|6500|Monday|2024Q3|N|N|N|2460493|2460674|2460155|2460430|N|N|N|N|N| +2460522|AAAAAAAAKGLIFCAA|2024-07-30|1494|6501|499|2024|2|7|30|3|2024|499|6501|Tuesday|2024Q3|N|N|N|2460493|2460674|2460156|2460431|N|N|N|N|N| +2460523|AAAAAAAALGLIFCAA|2024-07-31|1494|6501|499|2024|3|7|31|3|2024|499|6501|Wednesday|2024Q3|N|N|N|2460493|2460674|2460157|2460432|N|N|N|N|N| +2460524|AAAAAAAAMGLIFCAA|2024-08-01|1495|6501|499|2024|4|8|1|3|2024|499|6501|Thursday|2024Q3|N|N|N|2460524|2460736|2460158|2460433|N|N|N|N|N| +2460525|AAAAAAAANGLIFCAA|2024-08-02|1495|6501|499|2024|5|8|2|3|2024|499|6501|Friday|2024Q3|N|Y|N|2460524|2460736|2460159|2460434|N|N|N|N|N| +2460526|AAAAAAAAOGLIFCAA|2024-08-03|1495|6501|499|2024|6|8|3|3|2024|499|6501|Saturday|2024Q3|N|Y|N|2460524|2460736|2460160|2460435|N|N|N|N|N| +2460527|AAAAAAAAPGLIFCAA|2024-08-04|1495|6501|499|2024|0|8|4|3|2024|499|6501|Sunday|2024Q3|N|N|N|2460524|2460736|2460161|2460436|N|N|N|N|N| +2460528|AAAAAAAAAHLIFCAA|2024-08-05|1495|6501|499|2024|1|8|5|3|2024|499|6501|Monday|2024Q3|N|N|N|2460524|2460736|2460162|2460437|N|N|N|N|N| +2460529|AAAAAAAABHLIFCAA|2024-08-06|1495|6502|499|2024|2|8|6|3|2024|499|6502|Tuesday|2024Q3|N|N|N|2460524|2460736|2460163|2460438|N|N|N|N|N| +2460530|AAAAAAAACHLIFCAA|2024-08-07|1495|6502|499|2024|3|8|7|3|2024|499|6502|Wednesday|2024Q3|N|N|N|2460524|2460736|2460164|2460439|N|N|N|N|N| +2460531|AAAAAAAADHLIFCAA|2024-08-08|1495|6502|499|2024|4|8|8|3|2024|499|6502|Thursday|2024Q3|N|N|N|2460524|2460736|2460165|2460440|N|N|N|N|N| +2460532|AAAAAAAAEHLIFCAA|2024-08-09|1495|6502|499|2024|5|8|9|3|2024|499|6502|Friday|2024Q3|N|Y|N|2460524|2460736|2460166|2460441|N|N|N|N|N| +2460533|AAAAAAAAFHLIFCAA|2024-08-10|1495|6502|499|2024|6|8|10|3|2024|499|6502|Saturday|2024Q3|N|Y|N|2460524|2460736|2460167|2460442|N|N|N|N|N| +2460534|AAAAAAAAGHLIFCAA|2024-08-11|1495|6502|499|2024|0|8|11|3|2024|499|6502|Sunday|2024Q3|N|N|N|2460524|2460736|2460168|2460443|N|N|N|N|N| +2460535|AAAAAAAAHHLIFCAA|2024-08-12|1495|6502|499|2024|1|8|12|3|2024|499|6502|Monday|2024Q3|N|N|N|2460524|2460736|2460169|2460444|N|N|N|N|N| +2460536|AAAAAAAAIHLIFCAA|2024-08-13|1495|6503|499|2024|2|8|13|3|2024|499|6503|Tuesday|2024Q3|N|N|N|2460524|2460736|2460170|2460445|N|N|N|N|N| +2460537|AAAAAAAAJHLIFCAA|2024-08-14|1495|6503|499|2024|3|8|14|3|2024|499|6503|Wednesday|2024Q3|N|N|N|2460524|2460736|2460171|2460446|N|N|N|N|N| +2460538|AAAAAAAAKHLIFCAA|2024-08-15|1495|6503|499|2024|4|8|15|3|2024|499|6503|Thursday|2024Q3|N|N|N|2460524|2460736|2460172|2460447|N|N|N|N|N| +2460539|AAAAAAAALHLIFCAA|2024-08-16|1495|6503|499|2024|5|8|16|3|2024|499|6503|Friday|2024Q3|N|Y|N|2460524|2460736|2460173|2460448|N|N|N|N|N| +2460540|AAAAAAAAMHLIFCAA|2024-08-17|1495|6503|499|2024|6|8|17|3|2024|499|6503|Saturday|2024Q3|N|Y|N|2460524|2460736|2460174|2460449|N|N|N|N|N| +2460541|AAAAAAAANHLIFCAA|2024-08-18|1495|6503|499|2024|0|8|18|3|2024|499|6503|Sunday|2024Q3|N|N|N|2460524|2460736|2460175|2460450|N|N|N|N|N| +2460542|AAAAAAAAOHLIFCAA|2024-08-19|1495|6503|499|2024|1|8|19|3|2024|499|6503|Monday|2024Q3|N|N|N|2460524|2460736|2460176|2460451|N|N|N|N|N| +2460543|AAAAAAAAPHLIFCAA|2024-08-20|1495|6504|499|2024|2|8|20|3|2024|499|6504|Tuesday|2024Q3|N|N|N|2460524|2460736|2460177|2460452|N|N|N|N|N| +2460544|AAAAAAAAAILIFCAA|2024-08-21|1495|6504|499|2024|3|8|21|3|2024|499|6504|Wednesday|2024Q3|N|N|N|2460524|2460736|2460178|2460453|N|N|N|N|N| +2460545|AAAAAAAABILIFCAA|2024-08-22|1495|6504|499|2024|4|8|22|3|2024|499|6504|Thursday|2024Q3|N|N|N|2460524|2460736|2460179|2460454|N|N|N|N|N| +2460546|AAAAAAAACILIFCAA|2024-08-23|1495|6504|499|2024|5|8|23|3|2024|499|6504|Friday|2024Q3|N|Y|N|2460524|2460736|2460180|2460455|N|N|N|N|N| +2460547|AAAAAAAADILIFCAA|2024-08-24|1495|6504|499|2024|6|8|24|3|2024|499|6504|Saturday|2024Q3|N|Y|N|2460524|2460736|2460181|2460456|N|N|N|N|N| +2460548|AAAAAAAAEILIFCAA|2024-08-25|1495|6504|499|2024|0|8|25|3|2024|499|6504|Sunday|2024Q3|N|N|N|2460524|2460736|2460182|2460457|N|N|N|N|N| +2460549|AAAAAAAAFILIFCAA|2024-08-26|1495|6504|499|2024|1|8|26|3|2024|499|6504|Monday|2024Q3|N|N|N|2460524|2460736|2460183|2460458|N|N|N|N|N| +2460550|AAAAAAAAGILIFCAA|2024-08-27|1495|6505|499|2024|2|8|27|3|2024|499|6505|Tuesday|2024Q3|N|N|N|2460524|2460736|2460184|2460459|N|N|N|N|N| +2460551|AAAAAAAAHILIFCAA|2024-08-28|1495|6505|499|2024|3|8|28|3|2024|499|6505|Wednesday|2024Q3|N|N|N|2460524|2460736|2460185|2460460|N|N|N|N|N| +2460552|AAAAAAAAIILIFCAA|2024-08-29|1495|6505|499|2024|4|8|29|3|2024|499|6505|Thursday|2024Q3|N|N|N|2460524|2460736|2460186|2460461|N|N|N|N|N| +2460553|AAAAAAAAJILIFCAA|2024-08-30|1495|6505|499|2024|5|8|30|3|2024|499|6505|Friday|2024Q3|N|Y|N|2460524|2460736|2460187|2460462|N|N|N|N|N| +2460554|AAAAAAAAKILIFCAA|2024-08-31|1495|6505|499|2024|6|8|31|3|2024|499|6505|Saturday|2024Q3|N|Y|N|2460524|2460736|2460188|2460463|N|N|N|N|N| +2460555|AAAAAAAALILIFCAA|2024-09-01|1496|6505|500|2024|0|9|1|3|2024|500|6505|Sunday|2024Q3|N|N|N|2460555|2460798|2460189|2460464|N|N|N|N|N| +2460556|AAAAAAAAMILIFCAA|2024-09-02|1496|6505|500|2024|1|9|2|3|2024|500|6505|Monday|2024Q3|N|N|N|2460555|2460798|2460190|2460465|N|N|N|N|N| +2460557|AAAAAAAANILIFCAA|2024-09-03|1496|6506|500|2024|2|9|3|3|2024|500|6506|Tuesday|2024Q3|N|N|N|2460555|2460798|2460191|2460466|N|N|N|N|N| +2460558|AAAAAAAAOILIFCAA|2024-09-04|1496|6506|500|2024|3|9|4|3|2024|500|6506|Wednesday|2024Q3|N|N|N|2460555|2460798|2460192|2460467|N|N|N|N|N| +2460559|AAAAAAAAPILIFCAA|2024-09-05|1496|6506|500|2024|4|9|5|3|2024|500|6506|Thursday|2024Q3|N|N|N|2460555|2460798|2460193|2460468|N|N|N|N|N| +2460560|AAAAAAAAAJLIFCAA|2024-09-06|1496|6506|500|2024|5|9|6|3|2024|500|6506|Friday|2024Q3|N|Y|N|2460555|2460798|2460194|2460469|N|N|N|N|N| +2460561|AAAAAAAABJLIFCAA|2024-09-07|1496|6506|500|2024|6|9|7|3|2024|500|6506|Saturday|2024Q3|N|Y|N|2460555|2460798|2460195|2460470|N|N|N|N|N| +2460562|AAAAAAAACJLIFCAA|2024-09-08|1496|6506|500|2024|0|9|8|3|2024|500|6506|Sunday|2024Q3|N|N|N|2460555|2460798|2460196|2460471|N|N|N|N|N| +2460563|AAAAAAAADJLIFCAA|2024-09-09|1496|6506|500|2024|1|9|9|3|2024|500|6506|Monday|2024Q3|N|N|N|2460555|2460798|2460197|2460472|N|N|N|N|N| +2460564|AAAAAAAAEJLIFCAA|2024-09-10|1496|6507|500|2024|2|9|10|3|2024|500|6507|Tuesday|2024Q3|N|N|N|2460555|2460798|2460198|2460473|N|N|N|N|N| +2460565|AAAAAAAAFJLIFCAA|2024-09-11|1496|6507|500|2024|3|9|11|3|2024|500|6507|Wednesday|2024Q3|N|N|N|2460555|2460798|2460199|2460474|N|N|N|N|N| +2460566|AAAAAAAAGJLIFCAA|2024-09-12|1496|6507|500|2024|4|9|12|3|2024|500|6507|Thursday|2024Q3|N|N|N|2460555|2460798|2460200|2460475|N|N|N|N|N| +2460567|AAAAAAAAHJLIFCAA|2024-09-13|1496|6507|500|2024|5|9|13|3|2024|500|6507|Friday|2024Q3|N|Y|N|2460555|2460798|2460201|2460476|N|N|N|N|N| +2460568|AAAAAAAAIJLIFCAA|2024-09-14|1496|6507|500|2024|6|9|14|3|2024|500|6507|Saturday|2024Q3|N|Y|N|2460555|2460798|2460202|2460477|N|N|N|N|N| +2460569|AAAAAAAAJJLIFCAA|2024-09-15|1496|6507|500|2024|0|9|15|3|2024|500|6507|Sunday|2024Q3|N|N|N|2460555|2460798|2460203|2460478|N|N|N|N|N| +2460570|AAAAAAAAKJLIFCAA|2024-09-16|1496|6507|500|2024|1|9|16|3|2024|500|6507|Monday|2024Q3|N|N|N|2460555|2460798|2460204|2460479|N|N|N|N|N| +2460571|AAAAAAAALJLIFCAA|2024-09-17|1496|6508|500|2024|2|9|17|3|2024|500|6508|Tuesday|2024Q3|N|N|N|2460555|2460798|2460205|2460480|N|N|N|N|N| +2460572|AAAAAAAAMJLIFCAA|2024-09-18|1496|6508|500|2024|3|9|18|3|2024|500|6508|Wednesday|2024Q3|N|N|N|2460555|2460798|2460206|2460481|N|N|N|N|N| +2460573|AAAAAAAANJLIFCAA|2024-09-19|1496|6508|500|2024|4|9|19|3|2024|500|6508|Thursday|2024Q3|N|N|N|2460555|2460798|2460207|2460482|N|N|N|N|N| +2460574|AAAAAAAAOJLIFCAA|2024-09-20|1496|6508|500|2024|5|9|20|3|2024|500|6508|Friday|2024Q3|N|Y|N|2460555|2460798|2460208|2460483|N|N|N|N|N| +2460575|AAAAAAAAPJLIFCAA|2024-09-21|1496|6508|500|2024|6|9|21|3|2024|500|6508|Saturday|2024Q3|N|Y|N|2460555|2460798|2460209|2460484|N|N|N|N|N| +2460576|AAAAAAAAAKLIFCAA|2024-09-22|1496|6508|500|2024|0|9|22|3|2024|500|6508|Sunday|2024Q3|N|N|N|2460555|2460798|2460210|2460485|N|N|N|N|N| +2460577|AAAAAAAABKLIFCAA|2024-09-23|1496|6508|500|2024|1|9|23|3|2024|500|6508|Monday|2024Q3|N|N|N|2460555|2460798|2460211|2460486|N|N|N|N|N| +2460578|AAAAAAAACKLIFCAA|2024-09-24|1496|6509|500|2024|2|9|24|3|2024|500|6509|Tuesday|2024Q3|N|N|N|2460555|2460798|2460212|2460487|N|N|N|N|N| +2460579|AAAAAAAADKLIFCAA|2024-09-25|1496|6509|500|2024|3|9|25|3|2024|500|6509|Wednesday|2024Q3|N|N|N|2460555|2460798|2460213|2460488|N|N|N|N|N| +2460580|AAAAAAAAEKLIFCAA|2024-09-26|1496|6509|500|2024|4|9|26|3|2024|500|6509|Thursday|2024Q3|N|N|N|2460555|2460798|2460214|2460489|N|N|N|N|N| +2460581|AAAAAAAAFKLIFCAA|2024-09-27|1496|6509|500|2024|5|9|27|3|2024|500|6509|Friday|2024Q3|N|Y|N|2460555|2460798|2460215|2460490|N|N|N|N|N| +2460582|AAAAAAAAGKLIFCAA|2024-09-28|1496|6509|500|2024|6|9|28|3|2024|500|6509|Saturday|2024Q3|N|Y|N|2460555|2460798|2460216|2460491|N|N|N|N|N| +2460583|AAAAAAAAHKLIFCAA|2024-09-29|1496|6509|500|2024|0|9|29|3|2024|500|6509|Sunday|2024Q3|N|N|N|2460555|2460798|2460217|2460492|N|N|N|N|N| +2460584|AAAAAAAAIKLIFCAA|2024-09-30|1496|6509|500|2024|1|9|30|3|2024|500|6509|Monday|2024Q3|N|N|N|2460555|2460798|2460218|2460493|N|N|N|N|N| +2460585|AAAAAAAAJKLIFCAA|2024-10-01|1497|6510|500|2024|2|10|1|4|2024|500|6510|Tuesday|2024Q4|N|N|N|2460585|2460858|2460219|2460493|N|N|N|N|N| +2460586|AAAAAAAAKKLIFCAA|2024-10-02|1497|6510|500|2024|3|10|2|4|2024|500|6510|Wednesday|2024Q4|N|N|N|2460585|2460858|2460220|2460494|N|N|N|N|N| +2460587|AAAAAAAALKLIFCAA|2024-10-03|1497|6510|500|2024|4|10|3|4|2024|500|6510|Thursday|2024Q4|N|N|N|2460585|2460858|2460221|2460495|N|N|N|N|N| +2460588|AAAAAAAAMKLIFCAA|2024-10-04|1497|6510|500|2024|5|10|4|4|2024|500|6510|Friday|2024Q4|N|Y|N|2460585|2460858|2460222|2460496|N|N|N|N|N| +2460589|AAAAAAAANKLIFCAA|2024-10-05|1497|6510|500|2024|6|10|5|4|2024|500|6510|Saturday|2024Q4|N|Y|N|2460585|2460858|2460223|2460497|N|N|N|N|N| +2460590|AAAAAAAAOKLIFCAA|2024-10-06|1497|6510|500|2024|0|10|6|4|2024|500|6510|Sunday|2024Q4|N|N|N|2460585|2460858|2460224|2460498|N|N|N|N|N| +2460591|AAAAAAAAPKLIFCAA|2024-10-07|1497|6510|500|2024|1|10|7|4|2024|500|6510|Monday|2024Q4|N|N|N|2460585|2460858|2460225|2460499|N|N|N|N|N| +2460592|AAAAAAAAALLIFCAA|2024-10-08|1497|6511|500|2024|2|10|8|4|2024|500|6511|Tuesday|2024Q4|N|N|N|2460585|2460858|2460226|2460500|N|N|N|N|N| +2460593|AAAAAAAABLLIFCAA|2024-10-09|1497|6511|500|2024|3|10|9|4|2024|500|6511|Wednesday|2024Q4|N|N|N|2460585|2460858|2460227|2460501|N|N|N|N|N| +2460594|AAAAAAAACLLIFCAA|2024-10-10|1497|6511|500|2024|4|10|10|4|2024|500|6511|Thursday|2024Q4|N|N|N|2460585|2460858|2460228|2460502|N|N|N|N|N| +2460595|AAAAAAAADLLIFCAA|2024-10-11|1497|6511|500|2024|5|10|11|4|2024|500|6511|Friday|2024Q4|N|Y|N|2460585|2460858|2460229|2460503|N|N|N|N|N| +2460596|AAAAAAAAELLIFCAA|2024-10-12|1497|6511|500|2024|6|10|12|4|2024|500|6511|Saturday|2024Q4|N|Y|N|2460585|2460858|2460230|2460504|N|N|N|N|N| +2460597|AAAAAAAAFLLIFCAA|2024-10-13|1497|6511|500|2024|0|10|13|4|2024|500|6511|Sunday|2024Q4|N|N|N|2460585|2460858|2460231|2460505|N|N|N|N|N| +2460598|AAAAAAAAGLLIFCAA|2024-10-14|1497|6511|500|2024|1|10|14|4|2024|500|6511|Monday|2024Q4|N|N|N|2460585|2460858|2460232|2460506|N|N|N|N|N| +2460599|AAAAAAAAHLLIFCAA|2024-10-15|1497|6512|500|2024|2|10|15|4|2024|500|6512|Tuesday|2024Q4|N|N|N|2460585|2460858|2460233|2460507|N|N|N|N|N| +2460600|AAAAAAAAILLIFCAA|2024-10-16|1497|6512|500|2024|3|10|16|4|2024|500|6512|Wednesday|2024Q4|N|N|N|2460585|2460858|2460234|2460508|N|N|N|N|N| +2460601|AAAAAAAAJLLIFCAA|2024-10-17|1497|6512|500|2024|4|10|17|4|2024|500|6512|Thursday|2024Q4|N|N|N|2460585|2460858|2460235|2460509|N|N|N|N|N| +2460602|AAAAAAAAKLLIFCAA|2024-10-18|1497|6512|500|2024|5|10|18|4|2024|500|6512|Friday|2024Q4|N|Y|N|2460585|2460858|2460236|2460510|N|N|N|N|N| +2460603|AAAAAAAALLLIFCAA|2024-10-19|1497|6512|500|2024|6|10|19|4|2024|500|6512|Saturday|2024Q4|N|Y|N|2460585|2460858|2460237|2460511|N|N|N|N|N| +2460604|AAAAAAAAMLLIFCAA|2024-10-20|1497|6512|500|2024|0|10|20|4|2024|500|6512|Sunday|2024Q4|N|N|N|2460585|2460858|2460238|2460512|N|N|N|N|N| +2460605|AAAAAAAANLLIFCAA|2024-10-21|1497|6512|500|2024|1|10|21|4|2024|500|6512|Monday|2024Q4|N|N|N|2460585|2460858|2460239|2460513|N|N|N|N|N| +2460606|AAAAAAAAOLLIFCAA|2024-10-22|1497|6513|500|2024|2|10|22|4|2024|500|6513|Tuesday|2024Q4|N|N|N|2460585|2460858|2460240|2460514|N|N|N|N|N| +2460607|AAAAAAAAPLLIFCAA|2024-10-23|1497|6513|500|2024|3|10|23|4|2024|500|6513|Wednesday|2024Q4|N|N|N|2460585|2460858|2460241|2460515|N|N|N|N|N| +2460608|AAAAAAAAAMLIFCAA|2024-10-24|1497|6513|500|2024|4|10|24|4|2024|500|6513|Thursday|2024Q4|N|N|N|2460585|2460858|2460242|2460516|N|N|N|N|N| +2460609|AAAAAAAABMLIFCAA|2024-10-25|1497|6513|500|2024|5|10|25|4|2024|500|6513|Friday|2024Q4|N|Y|N|2460585|2460858|2460243|2460517|N|N|N|N|N| +2460610|AAAAAAAACMLIFCAA|2024-10-26|1497|6513|500|2024|6|10|26|4|2024|500|6513|Saturday|2024Q4|N|Y|N|2460585|2460858|2460244|2460518|N|N|N|N|N| +2460611|AAAAAAAADMLIFCAA|2024-10-27|1497|6513|500|2024|0|10|27|4|2024|500|6513|Sunday|2024Q4|N|N|N|2460585|2460858|2460245|2460519|N|N|N|N|N| +2460612|AAAAAAAAEMLIFCAA|2024-10-28|1497|6513|500|2024|1|10|28|4|2024|500|6513|Monday|2024Q4|N|N|N|2460585|2460858|2460246|2460520|N|N|N|N|N| +2460613|AAAAAAAAFMLIFCAA|2024-10-29|1497|6514|500|2024|2|10|29|4|2024|500|6514|Tuesday|2024Q4|N|N|N|2460585|2460858|2460247|2460521|N|N|N|N|N| +2460614|AAAAAAAAGMLIFCAA|2024-10-30|1497|6514|500|2024|3|10|30|4|2024|500|6514|Wednesday|2024Q4|N|N|N|2460585|2460858|2460248|2460522|N|N|N|N|N| +2460615|AAAAAAAAHMLIFCAA|2024-10-31|1497|6514|500|2024|4|10|31|4|2024|500|6514|Thursday|2024Q4|N|N|N|2460585|2460858|2460249|2460523|N|N|N|N|N| +2460616|AAAAAAAAIMLIFCAA|2024-11-01|1498|6514|500|2024|5|11|1|4|2024|500|6514|Friday|2024Q4|N|Y|N|2460616|2460920|2460250|2460524|N|N|N|N|N| +2460617|AAAAAAAAJMLIFCAA|2024-11-02|1498|6514|500|2024|6|11|2|4|2024|500|6514|Saturday|2024Q4|N|Y|N|2460616|2460920|2460251|2460525|N|N|N|N|N| +2460618|AAAAAAAAKMLIFCAA|2024-11-03|1498|6514|500|2024|0|11|3|4|2024|500|6514|Sunday|2024Q4|N|N|N|2460616|2460920|2460252|2460526|N|N|N|N|N| +2460619|AAAAAAAALMLIFCAA|2024-11-04|1498|6514|500|2024|1|11|4|4|2024|500|6514|Monday|2024Q4|N|N|N|2460616|2460920|2460253|2460527|N|N|N|N|N| +2460620|AAAAAAAAMMLIFCAA|2024-11-05|1498|6515|500|2024|2|11|5|4|2024|500|6515|Tuesday|2024Q4|N|N|N|2460616|2460920|2460254|2460528|N|N|N|N|N| +2460621|AAAAAAAANMLIFCAA|2024-11-06|1498|6515|500|2024|3|11|6|4|2024|500|6515|Wednesday|2024Q4|N|N|N|2460616|2460920|2460255|2460529|N|N|N|N|N| +2460622|AAAAAAAAOMLIFCAA|2024-11-07|1498|6515|500|2024|4|11|7|4|2024|500|6515|Thursday|2024Q4|N|N|N|2460616|2460920|2460256|2460530|N|N|N|N|N| +2460623|AAAAAAAAPMLIFCAA|2024-11-08|1498|6515|500|2024|5|11|8|4|2024|500|6515|Friday|2024Q4|N|Y|N|2460616|2460920|2460257|2460531|N|N|N|N|N| +2460624|AAAAAAAAANLIFCAA|2024-11-09|1498|6515|500|2024|6|11|9|4|2024|500|6515|Saturday|2024Q4|N|Y|N|2460616|2460920|2460258|2460532|N|N|N|N|N| +2460625|AAAAAAAABNLIFCAA|2024-11-10|1498|6515|500|2024|0|11|10|4|2024|500|6515|Sunday|2024Q4|N|N|N|2460616|2460920|2460259|2460533|N|N|N|N|N| +2460626|AAAAAAAACNLIFCAA|2024-11-11|1498|6515|500|2024|1|11|11|4|2024|500|6515|Monday|2024Q4|N|N|N|2460616|2460920|2460260|2460534|N|N|N|N|N| +2460627|AAAAAAAADNLIFCAA|2024-11-12|1498|6516|500|2024|2|11|12|4|2024|500|6516|Tuesday|2024Q4|N|N|N|2460616|2460920|2460261|2460535|N|N|N|N|N| +2460628|AAAAAAAAENLIFCAA|2024-11-13|1498|6516|500|2024|3|11|13|4|2024|500|6516|Wednesday|2024Q4|N|N|N|2460616|2460920|2460262|2460536|N|N|N|N|N| +2460629|AAAAAAAAFNLIFCAA|2024-11-14|1498|6516|500|2024|4|11|14|4|2024|500|6516|Thursday|2024Q4|N|N|N|2460616|2460920|2460263|2460537|N|N|N|N|N| +2460630|AAAAAAAAGNLIFCAA|2024-11-15|1498|6516|500|2024|5|11|15|4|2024|500|6516|Friday|2024Q4|N|Y|N|2460616|2460920|2460264|2460538|N|N|N|N|N| +2460631|AAAAAAAAHNLIFCAA|2024-11-16|1498|6516|500|2024|6|11|16|4|2024|500|6516|Saturday|2024Q4|N|Y|N|2460616|2460920|2460265|2460539|N|N|N|N|N| +2460632|AAAAAAAAINLIFCAA|2024-11-17|1498|6516|500|2024|0|11|17|4|2024|500|6516|Sunday|2024Q4|N|N|N|2460616|2460920|2460266|2460540|N|N|N|N|N| +2460633|AAAAAAAAJNLIFCAA|2024-11-18|1498|6516|500|2024|1|11|18|4|2024|500|6516|Monday|2024Q4|N|N|N|2460616|2460920|2460267|2460541|N|N|N|N|N| +2460634|AAAAAAAAKNLIFCAA|2024-11-19|1498|6517|500|2024|2|11|19|4|2024|500|6517|Tuesday|2024Q4|N|N|N|2460616|2460920|2460268|2460542|N|N|N|N|N| +2460635|AAAAAAAALNLIFCAA|2024-11-20|1498|6517|500|2024|3|11|20|4|2024|500|6517|Wednesday|2024Q4|N|N|N|2460616|2460920|2460269|2460543|N|N|N|N|N| +2460636|AAAAAAAAMNLIFCAA|2024-11-21|1498|6517|500|2024|4|11|21|4|2024|500|6517|Thursday|2024Q4|N|N|N|2460616|2460920|2460270|2460544|N|N|N|N|N| +2460637|AAAAAAAANNLIFCAA|2024-11-22|1498|6517|500|2024|5|11|22|4|2024|500|6517|Friday|2024Q4|N|Y|N|2460616|2460920|2460271|2460545|N|N|N|N|N| +2460638|AAAAAAAAONLIFCAA|2024-11-23|1498|6517|500|2024|6|11|23|4|2024|500|6517|Saturday|2024Q4|N|Y|N|2460616|2460920|2460272|2460546|N|N|N|N|N| +2460639|AAAAAAAAPNLIFCAA|2024-11-24|1498|6517|500|2024|0|11|24|4|2024|500|6517|Sunday|2024Q4|N|N|N|2460616|2460920|2460273|2460547|N|N|N|N|N| +2460640|AAAAAAAAAOLIFCAA|2024-11-25|1498|6517|500|2024|1|11|25|4|2024|500|6517|Monday|2024Q4|N|N|N|2460616|2460920|2460274|2460548|N|N|N|N|N| +2460641|AAAAAAAABOLIFCAA|2024-11-26|1498|6518|500|2024|2|11|26|4|2024|500|6518|Tuesday|2024Q4|N|N|N|2460616|2460920|2460275|2460549|N|N|N|N|N| +2460642|AAAAAAAACOLIFCAA|2024-11-27|1498|6518|500|2024|3|11|27|4|2024|500|6518|Wednesday|2024Q4|N|N|N|2460616|2460920|2460276|2460550|N|N|N|N|N| +2460643|AAAAAAAADOLIFCAA|2024-11-28|1498|6518|500|2024|4|11|28|4|2024|500|6518|Thursday|2024Q4|N|N|N|2460616|2460920|2460277|2460551|N|N|N|N|N| +2460644|AAAAAAAAEOLIFCAA|2024-11-29|1498|6518|500|2024|5|11|29|4|2024|500|6518|Friday|2024Q4|N|Y|N|2460616|2460920|2460278|2460552|N|N|N|N|N| +2460645|AAAAAAAAFOLIFCAA|2024-11-30|1498|6518|500|2024|6|11|30|4|2024|500|6518|Saturday|2024Q4|N|Y|N|2460616|2460920|2460279|2460553|N|N|N|N|N| +2460646|AAAAAAAAGOLIFCAA|2024-12-01|1499|6518|501|2024|0|12|1|4|2024|501|6518|Sunday|2024Q4|N|N|N|2460646|2460980|2460280|2460554|N|N|N|N|N| +2460647|AAAAAAAAHOLIFCAA|2024-12-02|1499|6518|501|2024|1|12|2|4|2024|501|6518|Monday|2024Q4|N|N|N|2460646|2460980|2460281|2460555|N|N|N|N|N| +2460648|AAAAAAAAIOLIFCAA|2024-12-03|1499|6519|501|2024|2|12|3|4|2024|501|6519|Tuesday|2024Q4|N|N|N|2460646|2460980|2460282|2460556|N|N|N|N|N| +2460649|AAAAAAAAJOLIFCAA|2024-12-04|1499|6519|501|2024|3|12|4|4|2024|501|6519|Wednesday|2024Q4|N|N|N|2460646|2460980|2460283|2460557|N|N|N|N|N| +2460650|AAAAAAAAKOLIFCAA|2024-12-05|1499|6519|501|2024|4|12|5|4|2024|501|6519|Thursday|2024Q4|N|N|N|2460646|2460980|2460284|2460558|N|N|N|N|N| +2460651|AAAAAAAALOLIFCAA|2024-12-06|1499|6519|501|2024|5|12|6|4|2024|501|6519|Friday|2024Q4|N|Y|N|2460646|2460980|2460285|2460559|N|N|N|N|N| +2460652|AAAAAAAAMOLIFCAA|2024-12-07|1499|6519|501|2024|6|12|7|4|2024|501|6519|Saturday|2024Q4|N|Y|N|2460646|2460980|2460286|2460560|N|N|N|N|N| +2460653|AAAAAAAANOLIFCAA|2024-12-08|1499|6519|501|2024|0|12|8|4|2024|501|6519|Sunday|2024Q4|N|N|N|2460646|2460980|2460287|2460561|N|N|N|N|N| +2460654|AAAAAAAAOOLIFCAA|2024-12-09|1499|6519|501|2024|1|12|9|4|2024|501|6519|Monday|2024Q4|N|N|N|2460646|2460980|2460288|2460562|N|N|N|N|N| +2460655|AAAAAAAAPOLIFCAA|2024-12-10|1499|6520|501|2024|2|12|10|4|2024|501|6520|Tuesday|2024Q4|N|N|N|2460646|2460980|2460289|2460563|N|N|N|N|N| +2460656|AAAAAAAAAPLIFCAA|2024-12-11|1499|6520|501|2024|3|12|11|4|2024|501|6520|Wednesday|2024Q4|N|N|N|2460646|2460980|2460290|2460564|N|N|N|N|N| +2460657|AAAAAAAABPLIFCAA|2024-12-12|1499|6520|501|2024|4|12|12|4|2024|501|6520|Thursday|2024Q4|N|N|N|2460646|2460980|2460291|2460565|N|N|N|N|N| +2460658|AAAAAAAACPLIFCAA|2024-12-13|1499|6520|501|2024|5|12|13|4|2024|501|6520|Friday|2024Q4|N|Y|N|2460646|2460980|2460292|2460566|N|N|N|N|N| +2460659|AAAAAAAADPLIFCAA|2024-12-14|1499|6520|501|2024|6|12|14|4|2024|501|6520|Saturday|2024Q4|N|Y|N|2460646|2460980|2460293|2460567|N|N|N|N|N| +2460660|AAAAAAAAEPLIFCAA|2024-12-15|1499|6520|501|2024|0|12|15|4|2024|501|6520|Sunday|2024Q4|N|N|N|2460646|2460980|2460294|2460568|N|N|N|N|N| +2460661|AAAAAAAAFPLIFCAA|2024-12-16|1499|6520|501|2024|1|12|16|4|2024|501|6520|Monday|2024Q4|N|N|N|2460646|2460980|2460295|2460569|N|N|N|N|N| +2460662|AAAAAAAAGPLIFCAA|2024-12-17|1499|6521|501|2024|2|12|17|4|2024|501|6521|Tuesday|2024Q4|N|N|N|2460646|2460980|2460296|2460570|N|N|N|N|N| +2460663|AAAAAAAAHPLIFCAA|2024-12-18|1499|6521|501|2024|3|12|18|4|2024|501|6521|Wednesday|2024Q4|N|N|N|2460646|2460980|2460297|2460571|N|N|N|N|N| +2460664|AAAAAAAAIPLIFCAA|2024-12-19|1499|6521|501|2024|4|12|19|4|2024|501|6521|Thursday|2024Q4|N|N|N|2460646|2460980|2460298|2460572|N|N|N|N|N| +2460665|AAAAAAAAJPLIFCAA|2024-12-20|1499|6521|501|2024|5|12|20|4|2024|501|6521|Friday|2024Q4|N|Y|N|2460646|2460980|2460299|2460573|N|N|N|N|N| +2460666|AAAAAAAAKPLIFCAA|2024-12-21|1499|6521|501|2024|6|12|21|4|2024|501|6521|Saturday|2024Q4|N|Y|N|2460646|2460980|2460300|2460574|N|N|N|N|N| +2460667|AAAAAAAALPLIFCAA|2024-12-22|1499|6521|501|2024|0|12|22|4|2024|501|6521|Sunday|2024Q4|N|N|N|2460646|2460980|2460301|2460575|N|N|N|N|N| +2460668|AAAAAAAAMPLIFCAA|2024-12-23|1499|6521|501|2024|1|12|23|4|2024|501|6521|Monday|2024Q4|N|N|N|2460646|2460980|2460302|2460576|N|N|N|N|N| +2460669|AAAAAAAANPLIFCAA|2024-12-24|1499|6522|501|2024|2|12|24|4|2024|501|6522|Tuesday|2024Q4|N|N|N|2460646|2460980|2460303|2460577|N|N|N|N|N| +2460670|AAAAAAAAOPLIFCAA|2024-12-25|1499|6522|501|2024|3|12|25|4|2024|501|6522|Wednesday|2024Q4|Y|N|N|2460646|2460980|2460304|2460578|N|N|N|N|N| +2460671|AAAAAAAAPPLIFCAA|2024-12-26|1499|6522|501|2024|4|12|26|4|2024|501|6522|Thursday|2024Q4|N|N|Y|2460646|2460980|2460305|2460579|N|N|N|N|N| +2460672|AAAAAAAAAAMIFCAA|2024-12-27|1499|6522|501|2024|5|12|27|4|2024|501|6522|Friday|2024Q4|N|Y|N|2460646|2460980|2460306|2460580|N|N|N|N|N| +2460673|AAAAAAAABAMIFCAA|2024-12-28|1499|6522|501|2024|6|12|28|4|2024|501|6522|Saturday|2024Q4|N|Y|N|2460646|2460980|2460307|2460581|N|N|N|N|N| +2460674|AAAAAAAACAMIFCAA|2024-12-29|1499|6522|501|2024|0|12|29|4|2024|501|6522|Sunday|2024Q4|N|N|N|2460646|2460980|2460308|2460582|N|N|N|N|N| +2460675|AAAAAAAADAMIFCAA|2024-12-30|1499|6522|501|2024|1|12|30|4|2024|501|6522|Monday|2024Q4|N|N|N|2460646|2460980|2460309|2460583|N|N|N|N|N| +2460676|AAAAAAAAEAMIFCAA|2024-12-31|1499|6523|501|2024|2|12|31|4|2024|501|6523|Tuesday|2024Q4|Y|N|N|2460646|2460980|2460310|2460584|N|N|N|N|N| +2460677|AAAAAAAAFAMIFCAA|2025-01-01|1500|6523|501|2025|3|1|1|1|2025|501|6523|Wednesday|2025Q1|Y|N|Y|2460677|2460676|2460311|2460585|N|N|N|N|N| +2460678|AAAAAAAAGAMIFCAA|2025-01-02|1500|6523|501|2025|4|1|2|1|2025|501|6523|Thursday|2025Q1|N|N|Y|2460677|2460676|2460312|2460586|N|N|N|N|N| +2460679|AAAAAAAAHAMIFCAA|2025-01-03|1500|6523|501|2025|5|1|3|1|2025|501|6523|Friday|2025Q1|N|Y|N|2460677|2460676|2460313|2460587|N|N|N|N|N| +2460680|AAAAAAAAIAMIFCAA|2025-01-04|1500|6523|501|2025|6|1|4|1|2025|501|6523|Saturday|2025Q1|N|Y|N|2460677|2460676|2460314|2460588|N|N|N|N|N| +2460681|AAAAAAAAJAMIFCAA|2025-01-05|1500|6523|501|2025|0|1|5|1|2025|501|6523|Sunday|2025Q1|N|N|N|2460677|2460676|2460315|2460589|N|N|N|N|N| +2460682|AAAAAAAAKAMIFCAA|2025-01-06|1500|6523|501|2025|1|1|6|1|2025|501|6523|Monday|2025Q1|N|N|N|2460677|2460676|2460316|2460590|N|N|N|N|N| +2460683|AAAAAAAALAMIFCAA|2025-01-07|1500|6524|501|2025|2|1|7|1|2025|501|6524|Tuesday|2025Q1|N|N|N|2460677|2460676|2460317|2460591|N|N|N|N|N| +2460684|AAAAAAAAMAMIFCAA|2025-01-08|1500|6524|501|2025|3|1|8|1|2025|501|6524|Wednesday|2025Q1|N|N|N|2460677|2460676|2460318|2460592|N|N|N|N|N| +2460685|AAAAAAAANAMIFCAA|2025-01-09|1500|6524|501|2025|4|1|9|1|2025|501|6524|Thursday|2025Q1|N|N|N|2460677|2460676|2460319|2460593|N|N|N|N|N| +2460686|AAAAAAAAOAMIFCAA|2025-01-10|1500|6524|501|2025|5|1|10|1|2025|501|6524|Friday|2025Q1|N|Y|N|2460677|2460676|2460320|2460594|N|N|N|N|N| +2460687|AAAAAAAAPAMIFCAA|2025-01-11|1500|6524|501|2025|6|1|11|1|2025|501|6524|Saturday|2025Q1|N|Y|N|2460677|2460676|2460321|2460595|N|N|N|N|N| +2460688|AAAAAAAAABMIFCAA|2025-01-12|1500|6524|501|2025|0|1|12|1|2025|501|6524|Sunday|2025Q1|N|N|N|2460677|2460676|2460322|2460596|N|N|N|N|N| +2460689|AAAAAAAABBMIFCAA|2025-01-13|1500|6524|501|2025|1|1|13|1|2025|501|6524|Monday|2025Q1|N|N|N|2460677|2460676|2460323|2460597|N|N|N|N|N| +2460690|AAAAAAAACBMIFCAA|2025-01-14|1500|6525|501|2025|2|1|14|1|2025|501|6525|Tuesday|2025Q1|N|N|N|2460677|2460676|2460324|2460598|N|N|N|N|N| +2460691|AAAAAAAADBMIFCAA|2025-01-15|1500|6525|501|2025|3|1|15|1|2025|501|6525|Wednesday|2025Q1|N|N|N|2460677|2460676|2460325|2460599|N|N|N|N|N| +2460692|AAAAAAAAEBMIFCAA|2025-01-16|1500|6525|501|2025|4|1|16|1|2025|501|6525|Thursday|2025Q1|N|N|N|2460677|2460676|2460326|2460600|N|N|N|N|N| +2460693|AAAAAAAAFBMIFCAA|2025-01-17|1500|6525|501|2025|5|1|17|1|2025|501|6525|Friday|2025Q1|N|Y|N|2460677|2460676|2460327|2460601|N|N|N|N|N| +2460694|AAAAAAAAGBMIFCAA|2025-01-18|1500|6525|501|2025|6|1|18|1|2025|501|6525|Saturday|2025Q1|N|Y|N|2460677|2460676|2460328|2460602|N|N|N|N|N| +2460695|AAAAAAAAHBMIFCAA|2025-01-19|1500|6525|501|2025|0|1|19|1|2025|501|6525|Sunday|2025Q1|N|N|N|2460677|2460676|2460329|2460603|N|N|N|N|N| +2460696|AAAAAAAAIBMIFCAA|2025-01-20|1500|6525|501|2025|1|1|20|1|2025|501|6525|Monday|2025Q1|N|N|N|2460677|2460676|2460330|2460604|N|N|N|N|N| +2460697|AAAAAAAAJBMIFCAA|2025-01-21|1500|6526|501|2025|2|1|21|1|2025|501|6526|Tuesday|2025Q1|N|N|N|2460677|2460676|2460331|2460605|N|N|N|N|N| +2460698|AAAAAAAAKBMIFCAA|2025-01-22|1500|6526|501|2025|3|1|22|1|2025|501|6526|Wednesday|2025Q1|N|N|N|2460677|2460676|2460332|2460606|N|N|N|N|N| +2460699|AAAAAAAALBMIFCAA|2025-01-23|1500|6526|501|2025|4|1|23|1|2025|501|6526|Thursday|2025Q1|N|N|N|2460677|2460676|2460333|2460607|N|N|N|N|N| +2460700|AAAAAAAAMBMIFCAA|2025-01-24|1500|6526|501|2025|5|1|24|1|2025|501|6526|Friday|2025Q1|N|Y|N|2460677|2460676|2460334|2460608|N|N|N|N|N| +2460701|AAAAAAAANBMIFCAA|2025-01-25|1500|6526|501|2025|6|1|25|1|2025|501|6526|Saturday|2025Q1|N|Y|N|2460677|2460676|2460335|2460609|N|N|N|N|N| +2460702|AAAAAAAAOBMIFCAA|2025-01-26|1500|6526|501|2025|0|1|26|1|2025|501|6526|Sunday|2025Q1|N|N|N|2460677|2460676|2460336|2460610|N|N|N|N|N| +2460703|AAAAAAAAPBMIFCAA|2025-01-27|1500|6526|501|2025|1|1|27|1|2025|501|6526|Monday|2025Q1|N|N|N|2460677|2460676|2460337|2460611|N|N|N|N|N| +2460704|AAAAAAAAACMIFCAA|2025-01-28|1500|6527|501|2025|2|1|28|1|2025|501|6527|Tuesday|2025Q1|N|N|N|2460677|2460676|2460338|2460612|N|N|N|N|N| +2460705|AAAAAAAABCMIFCAA|2025-01-29|1500|6527|501|2025|3|1|29|1|2025|501|6527|Wednesday|2025Q1|N|N|N|2460677|2460676|2460339|2460613|N|N|N|N|N| +2460706|AAAAAAAACCMIFCAA|2025-01-30|1500|6527|501|2025|4|1|30|1|2025|501|6527|Thursday|2025Q1|N|N|N|2460677|2460676|2460340|2460614|N|N|N|N|N| +2460707|AAAAAAAADCMIFCAA|2025-01-31|1500|6527|501|2025|5|1|31|1|2025|501|6527|Friday|2025Q1|N|Y|N|2460677|2460676|2460341|2460615|N|N|N|N|N| +2460708|AAAAAAAAECMIFCAA|2025-02-01|1501|6527|501|2025|6|2|1|1|2025|501|6527|Saturday|2025Q1|N|Y|N|2460708|2460738|2460342|2460616|N|N|N|N|N| +2460709|AAAAAAAAFCMIFCAA|2025-02-02|1501|6527|501|2025|0|2|2|1|2025|501|6527|Sunday|2025Q1|N|N|N|2460708|2460738|2460343|2460617|N|N|N|N|N| +2460710|AAAAAAAAGCMIFCAA|2025-02-03|1501|6527|501|2025|1|2|3|1|2025|501|6527|Monday|2025Q1|N|N|N|2460708|2460738|2460344|2460618|N|N|N|N|N| +2460711|AAAAAAAAHCMIFCAA|2025-02-04|1501|6528|501|2025|2|2|4|1|2025|501|6528|Tuesday|2025Q1|N|N|N|2460708|2460738|2460345|2460619|N|N|N|N|N| +2460712|AAAAAAAAICMIFCAA|2025-02-05|1501|6528|501|2025|3|2|5|1|2025|501|6528|Wednesday|2025Q1|N|N|N|2460708|2460738|2460346|2460620|N|N|N|N|N| +2460713|AAAAAAAAJCMIFCAA|2025-02-06|1501|6528|501|2025|4|2|6|1|2025|501|6528|Thursday|2025Q1|N|N|N|2460708|2460738|2460347|2460621|N|N|N|N|N| +2460714|AAAAAAAAKCMIFCAA|2025-02-07|1501|6528|501|2025|5|2|7|1|2025|501|6528|Friday|2025Q1|N|Y|N|2460708|2460738|2460348|2460622|N|N|N|N|N| +2460715|AAAAAAAALCMIFCAA|2025-02-08|1501|6528|501|2025|6|2|8|1|2025|501|6528|Saturday|2025Q1|N|Y|N|2460708|2460738|2460349|2460623|N|N|N|N|N| +2460716|AAAAAAAAMCMIFCAA|2025-02-09|1501|6528|501|2025|0|2|9|1|2025|501|6528|Sunday|2025Q1|N|N|N|2460708|2460738|2460350|2460624|N|N|N|N|N| +2460717|AAAAAAAANCMIFCAA|2025-02-10|1501|6528|501|2025|1|2|10|1|2025|501|6528|Monday|2025Q1|N|N|N|2460708|2460738|2460351|2460625|N|N|N|N|N| +2460718|AAAAAAAAOCMIFCAA|2025-02-11|1501|6529|501|2025|2|2|11|1|2025|501|6529|Tuesday|2025Q1|N|N|N|2460708|2460738|2460352|2460626|N|N|N|N|N| +2460719|AAAAAAAAPCMIFCAA|2025-02-12|1501|6529|501|2025|3|2|12|1|2025|501|6529|Wednesday|2025Q1|N|N|N|2460708|2460738|2460353|2460627|N|N|N|N|N| +2460720|AAAAAAAAADMIFCAA|2025-02-13|1501|6529|501|2025|4|2|13|1|2025|501|6529|Thursday|2025Q1|N|N|N|2460708|2460738|2460354|2460628|N|N|N|N|N| +2460721|AAAAAAAABDMIFCAA|2025-02-14|1501|6529|501|2025|5|2|14|1|2025|501|6529|Friday|2025Q1|N|Y|N|2460708|2460738|2460355|2460629|N|N|N|N|N| +2460722|AAAAAAAACDMIFCAA|2025-02-15|1501|6529|501|2025|6|2|15|1|2025|501|6529|Saturday|2025Q1|N|Y|N|2460708|2460738|2460356|2460630|N|N|N|N|N| +2460723|AAAAAAAADDMIFCAA|2025-02-16|1501|6529|501|2025|0|2|16|1|2025|501|6529|Sunday|2025Q1|N|N|N|2460708|2460738|2460357|2460631|N|N|N|N|N| +2460724|AAAAAAAAEDMIFCAA|2025-02-17|1501|6529|501|2025|1|2|17|1|2025|501|6529|Monday|2025Q1|N|N|N|2460708|2460738|2460358|2460632|N|N|N|N|N| +2460725|AAAAAAAAFDMIFCAA|2025-02-18|1501|6530|501|2025|2|2|18|1|2025|501|6530|Tuesday|2025Q1|N|N|N|2460708|2460738|2460359|2460633|N|N|N|N|N| +2460726|AAAAAAAAGDMIFCAA|2025-02-19|1501|6530|501|2025|3|2|19|1|2025|501|6530|Wednesday|2025Q1|N|N|N|2460708|2460738|2460360|2460634|N|N|N|N|N| +2460727|AAAAAAAAHDMIFCAA|2025-02-20|1501|6530|501|2025|4|2|20|1|2025|501|6530|Thursday|2025Q1|N|N|N|2460708|2460738|2460361|2460635|N|N|N|N|N| +2460728|AAAAAAAAIDMIFCAA|2025-02-21|1501|6530|501|2025|5|2|21|1|2025|501|6530|Friday|2025Q1|N|Y|N|2460708|2460738|2460362|2460636|N|N|N|N|N| +2460729|AAAAAAAAJDMIFCAA|2025-02-22|1501|6530|501|2025|6|2|22|1|2025|501|6530|Saturday|2025Q1|N|Y|N|2460708|2460738|2460363|2460637|N|N|N|N|N| +2460730|AAAAAAAAKDMIFCAA|2025-02-23|1501|6530|501|2025|0|2|23|1|2025|501|6530|Sunday|2025Q1|N|N|N|2460708|2460738|2460364|2460638|N|N|N|N|N| +2460731|AAAAAAAALDMIFCAA|2025-02-24|1501|6530|501|2025|1|2|24|1|2025|501|6530|Monday|2025Q1|N|N|N|2460708|2460738|2460365|2460639|N|N|N|N|N| +2460732|AAAAAAAAMDMIFCAA|2025-02-25|1501|6531|501|2025|2|2|25|1|2025|501|6531|Tuesday|2025Q1|N|N|N|2460708|2460738|2460366|2460640|N|N|N|N|N| +2460733|AAAAAAAANDMIFCAA|2025-02-26|1501|6531|501|2025|3|2|26|1|2025|501|6531|Wednesday|2025Q1|N|N|N|2460708|2460738|2460367|2460641|N|N|N|N|N| +2460734|AAAAAAAAODMIFCAA|2025-02-27|1501|6531|501|2025|4|2|27|1|2025|501|6531|Thursday|2025Q1|N|N|N|2460708|2460738|2460368|2460642|N|N|N|N|N| +2460735|AAAAAAAAPDMIFCAA|2025-02-28|1501|6531|501|2025|5|2|28|1|2025|501|6531|Friday|2025Q1|N|Y|N|2460708|2460738|2460369|2460643|N|N|N|N|N| +2460736|AAAAAAAAAEMIFCAA|2025-03-01|1502|6531|502|2025|6|3|1|1|2025|502|6531|Saturday|2025Q1|N|Y|N|2460736|2460794|2460371|2460644|N|N|N|N|N| +2460737|AAAAAAAABEMIFCAA|2025-03-02|1502|6531|502|2025|0|3|2|1|2025|502|6531|Sunday|2025Q1|N|N|N|2460736|2460794|2460372|2460645|N|N|N|N|N| +2460738|AAAAAAAACEMIFCAA|2025-03-03|1502|6531|502|2025|1|3|3|1|2025|502|6531|Monday|2025Q1|N|N|N|2460736|2460794|2460373|2460646|N|N|N|N|N| +2460739|AAAAAAAADEMIFCAA|2025-03-04|1502|6532|502|2025|2|3|4|1|2025|502|6532|Tuesday|2025Q1|N|N|N|2460736|2460794|2460374|2460647|N|N|N|N|N| +2460740|AAAAAAAAEEMIFCAA|2025-03-05|1502|6532|502|2025|3|3|5|1|2025|502|6532|Wednesday|2025Q1|N|N|N|2460736|2460794|2460375|2460648|N|N|N|N|N| +2460741|AAAAAAAAFEMIFCAA|2025-03-06|1502|6532|502|2025|4|3|6|1|2025|502|6532|Thursday|2025Q1|N|N|N|2460736|2460794|2460376|2460649|N|N|N|N|N| +2460742|AAAAAAAAGEMIFCAA|2025-03-07|1502|6532|502|2025|5|3|7|1|2025|502|6532|Friday|2025Q1|N|Y|N|2460736|2460794|2460377|2460650|N|N|N|N|N| +2460743|AAAAAAAAHEMIFCAA|2025-03-08|1502|6532|502|2025|6|3|8|1|2025|502|6532|Saturday|2025Q1|N|Y|N|2460736|2460794|2460378|2460651|N|N|N|N|N| +2460744|AAAAAAAAIEMIFCAA|2025-03-09|1502|6532|502|2025|0|3|9|1|2025|502|6532|Sunday|2025Q1|N|N|N|2460736|2460794|2460379|2460652|N|N|N|N|N| +2460745|AAAAAAAAJEMIFCAA|2025-03-10|1502|6532|502|2025|1|3|10|1|2025|502|6532|Monday|2025Q1|N|N|N|2460736|2460794|2460380|2460653|N|N|N|N|N| +2460746|AAAAAAAAKEMIFCAA|2025-03-11|1502|6533|502|2025|2|3|11|1|2025|502|6533|Tuesday|2025Q1|N|N|N|2460736|2460794|2460381|2460654|N|N|N|N|N| +2460747|AAAAAAAALEMIFCAA|2025-03-12|1502|6533|502|2025|3|3|12|1|2025|502|6533|Wednesday|2025Q1|N|N|N|2460736|2460794|2460382|2460655|N|N|N|N|N| +2460748|AAAAAAAAMEMIFCAA|2025-03-13|1502|6533|502|2025|4|3|13|1|2025|502|6533|Thursday|2025Q1|N|N|N|2460736|2460794|2460383|2460656|N|N|N|N|N| +2460749|AAAAAAAANEMIFCAA|2025-03-14|1502|6533|502|2025|5|3|14|1|2025|502|6533|Friday|2025Q1|N|Y|N|2460736|2460794|2460384|2460657|N|N|N|N|N| +2460750|AAAAAAAAOEMIFCAA|2025-03-15|1502|6533|502|2025|6|3|15|1|2025|502|6533|Saturday|2025Q1|N|Y|N|2460736|2460794|2460385|2460658|N|N|N|N|N| +2460751|AAAAAAAAPEMIFCAA|2025-03-16|1502|6533|502|2025|0|3|16|1|2025|502|6533|Sunday|2025Q1|N|N|N|2460736|2460794|2460386|2460659|N|N|N|N|N| +2460752|AAAAAAAAAFMIFCAA|2025-03-17|1502|6533|502|2025|1|3|17|1|2025|502|6533|Monday|2025Q1|N|N|N|2460736|2460794|2460387|2460660|N|N|N|N|N| +2460753|AAAAAAAABFMIFCAA|2025-03-18|1502|6534|502|2025|2|3|18|1|2025|502|6534|Tuesday|2025Q1|N|N|N|2460736|2460794|2460388|2460661|N|N|N|N|N| +2460754|AAAAAAAACFMIFCAA|2025-03-19|1502|6534|502|2025|3|3|19|1|2025|502|6534|Wednesday|2025Q1|N|N|N|2460736|2460794|2460389|2460662|N|N|N|N|N| +2460755|AAAAAAAADFMIFCAA|2025-03-20|1502|6534|502|2025|4|3|20|1|2025|502|6534|Thursday|2025Q1|N|N|N|2460736|2460794|2460390|2460663|N|N|N|N|N| +2460756|AAAAAAAAEFMIFCAA|2025-03-21|1502|6534|502|2025|5|3|21|1|2025|502|6534|Friday|2025Q1|N|Y|N|2460736|2460794|2460391|2460664|N|N|N|N|N| +2460757|AAAAAAAAFFMIFCAA|2025-03-22|1502|6534|502|2025|6|3|22|1|2025|502|6534|Saturday|2025Q1|N|Y|N|2460736|2460794|2460392|2460665|N|N|N|N|N| +2460758|AAAAAAAAGFMIFCAA|2025-03-23|1502|6534|502|2025|0|3|23|1|2025|502|6534|Sunday|2025Q1|N|N|N|2460736|2460794|2460393|2460666|N|N|N|N|N| +2460759|AAAAAAAAHFMIFCAA|2025-03-24|1502|6534|502|2025|1|3|24|1|2025|502|6534|Monday|2025Q1|N|N|N|2460736|2460794|2460394|2460667|N|N|N|N|N| +2460760|AAAAAAAAIFMIFCAA|2025-03-25|1502|6535|502|2025|2|3|25|1|2025|502|6535|Tuesday|2025Q1|N|N|N|2460736|2460794|2460395|2460668|N|N|N|N|N| +2460761|AAAAAAAAJFMIFCAA|2025-03-26|1502|6535|502|2025|3|3|26|1|2025|502|6535|Wednesday|2025Q1|N|N|N|2460736|2460794|2460396|2460669|N|N|N|N|N| +2460762|AAAAAAAAKFMIFCAA|2025-03-27|1502|6535|502|2025|4|3|27|1|2025|502|6535|Thursday|2025Q1|N|N|N|2460736|2460794|2460397|2460670|N|N|N|N|N| +2460763|AAAAAAAALFMIFCAA|2025-03-28|1502|6535|502|2025|5|3|28|1|2025|502|6535|Friday|2025Q1|N|Y|N|2460736|2460794|2460398|2460671|N|N|N|N|N| +2460764|AAAAAAAAMFMIFCAA|2025-03-29|1502|6535|502|2025|6|3|29|1|2025|502|6535|Saturday|2025Q1|N|Y|N|2460736|2460794|2460399|2460672|N|N|N|N|N| +2460765|AAAAAAAANFMIFCAA|2025-03-30|1502|6535|502|2025|0|3|30|1|2025|502|6535|Sunday|2025Q1|N|N|N|2460736|2460794|2460400|2460673|N|N|N|N|N| +2460766|AAAAAAAAOFMIFCAA|2025-03-31|1502|6535|502|2025|1|3|31|1|2025|502|6535|Monday|2025Q1|N|N|N|2460736|2460794|2460401|2460674|N|N|N|N|N| +2460767|AAAAAAAAPFMIFCAA|2025-04-01|1503|6536|502|2025|2|4|1|1|2025|502|6536|Tuesday|2025Q1|N|N|N|2460767|2460856|2460402|2460677|N|N|N|N|N| +2460768|AAAAAAAAAGMIFCAA|2025-04-02|1503|6536|502|2025|3|4|2|2|2025|502|6536|Wednesday|2025Q2|N|N|N|2460767|2460856|2460403|2460678|N|N|N|N|N| +2460769|AAAAAAAABGMIFCAA|2025-04-03|1503|6536|502|2025|4|4|3|2|2025|502|6536|Thursday|2025Q2|N|N|N|2460767|2460856|2460404|2460679|N|N|N|N|N| +2460770|AAAAAAAACGMIFCAA|2025-04-04|1503|6536|502|2025|5|4|4|2|2025|502|6536|Friday|2025Q2|N|Y|N|2460767|2460856|2460405|2460680|N|N|N|N|N| +2460771|AAAAAAAADGMIFCAA|2025-04-05|1503|6536|502|2025|6|4|5|2|2025|502|6536|Saturday|2025Q2|N|Y|N|2460767|2460856|2460406|2460681|N|N|N|N|N| +2460772|AAAAAAAAEGMIFCAA|2025-04-06|1503|6536|502|2025|0|4|6|2|2025|502|6536|Sunday|2025Q2|N|N|N|2460767|2460856|2460407|2460682|N|N|N|N|N| +2460773|AAAAAAAAFGMIFCAA|2025-04-07|1503|6536|502|2025|1|4|7|2|2025|502|6536|Monday|2025Q2|N|N|N|2460767|2460856|2460408|2460683|N|N|N|N|N| +2460774|AAAAAAAAGGMIFCAA|2025-04-08|1503|6537|502|2025|2|4|8|2|2025|502|6537|Tuesday|2025Q2|N|N|N|2460767|2460856|2460409|2460684|N|N|N|N|N| +2460775|AAAAAAAAHGMIFCAA|2025-04-09|1503|6537|502|2025|3|4|9|2|2025|502|6537|Wednesday|2025Q2|N|N|N|2460767|2460856|2460410|2460685|N|N|N|N|N| +2460776|AAAAAAAAIGMIFCAA|2025-04-10|1503|6537|502|2025|4|4|10|2|2025|502|6537|Thursday|2025Q2|N|N|N|2460767|2460856|2460411|2460686|N|N|N|N|N| +2460777|AAAAAAAAJGMIFCAA|2025-04-11|1503|6537|502|2025|5|4|11|2|2025|502|6537|Friday|2025Q2|N|Y|N|2460767|2460856|2460412|2460687|N|N|N|N|N| +2460778|AAAAAAAAKGMIFCAA|2025-04-12|1503|6537|502|2025|6|4|12|2|2025|502|6537|Saturday|2025Q2|N|Y|N|2460767|2460856|2460413|2460688|N|N|N|N|N| +2460779|AAAAAAAALGMIFCAA|2025-04-13|1503|6537|502|2025|0|4|13|2|2025|502|6537|Sunday|2025Q2|N|N|N|2460767|2460856|2460414|2460689|N|N|N|N|N| +2460780|AAAAAAAAMGMIFCAA|2025-04-14|1503|6537|502|2025|1|4|14|2|2025|502|6537|Monday|2025Q2|N|N|N|2460767|2460856|2460415|2460690|N|N|N|N|N| +2460781|AAAAAAAANGMIFCAA|2025-04-15|1503|6538|502|2025|2|4|15|2|2025|502|6538|Tuesday|2025Q2|N|N|N|2460767|2460856|2460416|2460691|N|N|N|N|N| +2460782|AAAAAAAAOGMIFCAA|2025-04-16|1503|6538|502|2025|3|4|16|2|2025|502|6538|Wednesday|2025Q2|N|N|N|2460767|2460856|2460417|2460692|N|N|N|N|N| +2460783|AAAAAAAAPGMIFCAA|2025-04-17|1503|6538|502|2025|4|4|17|2|2025|502|6538|Thursday|2025Q2|N|N|N|2460767|2460856|2460418|2460693|N|N|N|N|N| +2460784|AAAAAAAAAHMIFCAA|2025-04-18|1503|6538|502|2025|5|4|18|2|2025|502|6538|Friday|2025Q2|N|Y|N|2460767|2460856|2460419|2460694|N|N|N|N|N| +2460785|AAAAAAAABHMIFCAA|2025-04-19|1503|6538|502|2025|6|4|19|2|2025|502|6538|Saturday|2025Q2|N|Y|N|2460767|2460856|2460420|2460695|N|N|N|N|N| +2460786|AAAAAAAACHMIFCAA|2025-04-20|1503|6538|502|2025|0|4|20|2|2025|502|6538|Sunday|2025Q2|N|N|N|2460767|2460856|2460421|2460696|N|N|N|N|N| +2460787|AAAAAAAADHMIFCAA|2025-04-21|1503|6538|502|2025|1|4|21|2|2025|502|6538|Monday|2025Q2|N|N|N|2460767|2460856|2460422|2460697|N|N|N|N|N| +2460788|AAAAAAAAEHMIFCAA|2025-04-22|1503|6539|502|2025|2|4|22|2|2025|502|6539|Tuesday|2025Q2|N|N|N|2460767|2460856|2460423|2460698|N|N|N|N|N| +2460789|AAAAAAAAFHMIFCAA|2025-04-23|1503|6539|502|2025|3|4|23|2|2025|502|6539|Wednesday|2025Q2|N|N|N|2460767|2460856|2460424|2460699|N|N|N|N|N| +2460790|AAAAAAAAGHMIFCAA|2025-04-24|1503|6539|502|2025|4|4|24|2|2025|502|6539|Thursday|2025Q2|N|N|N|2460767|2460856|2460425|2460700|N|N|N|N|N| +2460791|AAAAAAAAHHMIFCAA|2025-04-25|1503|6539|502|2025|5|4|25|2|2025|502|6539|Friday|2025Q2|N|Y|N|2460767|2460856|2460426|2460701|N|N|N|N|N| +2460792|AAAAAAAAIHMIFCAA|2025-04-26|1503|6539|502|2025|6|4|26|2|2025|502|6539|Saturday|2025Q2|N|Y|N|2460767|2460856|2460427|2460702|N|N|N|N|N| +2460793|AAAAAAAAJHMIFCAA|2025-04-27|1503|6539|502|2025|0|4|27|2|2025|502|6539|Sunday|2025Q2|N|N|N|2460767|2460856|2460428|2460703|N|N|N|N|N| +2460794|AAAAAAAAKHMIFCAA|2025-04-28|1503|6539|502|2025|1|4|28|2|2025|502|6539|Monday|2025Q2|N|N|N|2460767|2460856|2460429|2460704|N|N|N|N|N| +2460795|AAAAAAAALHMIFCAA|2025-04-29|1503|6540|502|2025|2|4|29|2|2025|502|6540|Tuesday|2025Q2|N|N|N|2460767|2460856|2460430|2460705|N|N|N|N|N| +2460796|AAAAAAAAMHMIFCAA|2025-04-30|1503|6540|502|2025|3|4|30|2|2025|502|6540|Wednesday|2025Q2|N|N|N|2460767|2460856|2460431|2460706|N|N|N|N|N| +2460797|AAAAAAAANHMIFCAA|2025-05-01|1504|6540|502|2025|4|5|1|2|2025|502|6540|Thursday|2025Q2|N|N|N|2460797|2460916|2460432|2460707|N|N|N|N|N| +2460798|AAAAAAAAOHMIFCAA|2025-05-02|1504|6540|502|2025|5|5|2|2|2025|502|6540|Friday|2025Q2|N|Y|N|2460797|2460916|2460433|2460708|N|N|N|N|N| +2460799|AAAAAAAAPHMIFCAA|2025-05-03|1504|6540|502|2025|6|5|3|2|2025|502|6540|Saturday|2025Q2|N|Y|N|2460797|2460916|2460434|2460709|N|N|N|N|N| +2460800|AAAAAAAAAIMIFCAA|2025-05-04|1504|6540|502|2025|0|5|4|2|2025|502|6540|Sunday|2025Q2|N|N|N|2460797|2460916|2460435|2460710|N|N|N|N|N| +2460801|AAAAAAAABIMIFCAA|2025-05-05|1504|6540|502|2025|1|5|5|2|2025|502|6540|Monday|2025Q2|N|N|N|2460797|2460916|2460436|2460711|N|N|N|N|N| +2460802|AAAAAAAACIMIFCAA|2025-05-06|1504|6541|502|2025|2|5|6|2|2025|502|6541|Tuesday|2025Q2|N|N|N|2460797|2460916|2460437|2460712|N|N|N|N|N| +2460803|AAAAAAAADIMIFCAA|2025-05-07|1504|6541|502|2025|3|5|7|2|2025|502|6541|Wednesday|2025Q2|N|N|N|2460797|2460916|2460438|2460713|N|N|N|N|N| +2460804|AAAAAAAAEIMIFCAA|2025-05-08|1504|6541|502|2025|4|5|8|2|2025|502|6541|Thursday|2025Q2|N|N|N|2460797|2460916|2460439|2460714|N|N|N|N|N| +2460805|AAAAAAAAFIMIFCAA|2025-05-09|1504|6541|502|2025|5|5|9|2|2025|502|6541|Friday|2025Q2|N|Y|N|2460797|2460916|2460440|2460715|N|N|N|N|N| +2460806|AAAAAAAAGIMIFCAA|2025-05-10|1504|6541|502|2025|6|5|10|2|2025|502|6541|Saturday|2025Q2|N|Y|N|2460797|2460916|2460441|2460716|N|N|N|N|N| +2460807|AAAAAAAAHIMIFCAA|2025-05-11|1504|6541|502|2025|0|5|11|2|2025|502|6541|Sunday|2025Q2|N|N|N|2460797|2460916|2460442|2460717|N|N|N|N|N| +2460808|AAAAAAAAIIMIFCAA|2025-05-12|1504|6541|502|2025|1|5|12|2|2025|502|6541|Monday|2025Q2|N|N|N|2460797|2460916|2460443|2460718|N|N|N|N|N| +2460809|AAAAAAAAJIMIFCAA|2025-05-13|1504|6542|502|2025|2|5|13|2|2025|502|6542|Tuesday|2025Q2|N|N|N|2460797|2460916|2460444|2460719|N|N|N|N|N| +2460810|AAAAAAAAKIMIFCAA|2025-05-14|1504|6542|502|2025|3|5|14|2|2025|502|6542|Wednesday|2025Q2|N|N|N|2460797|2460916|2460445|2460720|N|N|N|N|N| +2460811|AAAAAAAALIMIFCAA|2025-05-15|1504|6542|502|2025|4|5|15|2|2025|502|6542|Thursday|2025Q2|N|N|N|2460797|2460916|2460446|2460721|N|N|N|N|N| +2460812|AAAAAAAAMIMIFCAA|2025-05-16|1504|6542|502|2025|5|5|16|2|2025|502|6542|Friday|2025Q2|N|Y|N|2460797|2460916|2460447|2460722|N|N|N|N|N| +2460813|AAAAAAAANIMIFCAA|2025-05-17|1504|6542|502|2025|6|5|17|2|2025|502|6542|Saturday|2025Q2|N|Y|N|2460797|2460916|2460448|2460723|N|N|N|N|N| +2460814|AAAAAAAAOIMIFCAA|2025-05-18|1504|6542|502|2025|0|5|18|2|2025|502|6542|Sunday|2025Q2|N|N|N|2460797|2460916|2460449|2460724|N|N|N|N|N| +2460815|AAAAAAAAPIMIFCAA|2025-05-19|1504|6542|502|2025|1|5|19|2|2025|502|6542|Monday|2025Q2|N|N|N|2460797|2460916|2460450|2460725|N|N|N|N|N| +2460816|AAAAAAAAAJMIFCAA|2025-05-20|1504|6543|502|2025|2|5|20|2|2025|502|6543|Tuesday|2025Q2|N|N|N|2460797|2460916|2460451|2460726|N|N|N|N|N| +2460817|AAAAAAAABJMIFCAA|2025-05-21|1504|6543|502|2025|3|5|21|2|2025|502|6543|Wednesday|2025Q2|N|N|N|2460797|2460916|2460452|2460727|N|N|N|N|N| +2460818|AAAAAAAACJMIFCAA|2025-05-22|1504|6543|502|2025|4|5|22|2|2025|502|6543|Thursday|2025Q2|N|N|N|2460797|2460916|2460453|2460728|N|N|N|N|N| +2460819|AAAAAAAADJMIFCAA|2025-05-23|1504|6543|502|2025|5|5|23|2|2025|502|6543|Friday|2025Q2|N|Y|N|2460797|2460916|2460454|2460729|N|N|N|N|N| +2460820|AAAAAAAAEJMIFCAA|2025-05-24|1504|6543|502|2025|6|5|24|2|2025|502|6543|Saturday|2025Q2|N|Y|N|2460797|2460916|2460455|2460730|N|N|N|N|N| +2460821|AAAAAAAAFJMIFCAA|2025-05-25|1504|6543|502|2025|0|5|25|2|2025|502|6543|Sunday|2025Q2|N|N|N|2460797|2460916|2460456|2460731|N|N|N|N|N| +2460822|AAAAAAAAGJMIFCAA|2025-05-26|1504|6543|502|2025|1|5|26|2|2025|502|6543|Monday|2025Q2|N|N|N|2460797|2460916|2460457|2460732|N|N|N|N|N| +2460823|AAAAAAAAHJMIFCAA|2025-05-27|1504|6544|502|2025|2|5|27|2|2025|502|6544|Tuesday|2025Q2|N|N|N|2460797|2460916|2460458|2460733|N|N|N|N|N| +2460824|AAAAAAAAIJMIFCAA|2025-05-28|1504|6544|502|2025|3|5|28|2|2025|502|6544|Wednesday|2025Q2|N|N|N|2460797|2460916|2460459|2460734|N|N|N|N|N| +2460825|AAAAAAAAJJMIFCAA|2025-05-29|1504|6544|502|2025|4|5|29|2|2025|502|6544|Thursday|2025Q2|N|N|N|2460797|2460916|2460460|2460735|N|N|N|N|N| +2460826|AAAAAAAAKJMIFCAA|2025-05-30|1504|6544|502|2025|5|5|30|2|2025|502|6544|Friday|2025Q2|N|Y|N|2460797|2460916|2460461|2460736|N|N|N|N|N| +2460827|AAAAAAAALJMIFCAA|2025-05-31|1504|6544|502|2025|6|5|31|2|2025|502|6544|Saturday|2025Q2|N|Y|N|2460797|2460916|2460462|2460737|N|N|N|N|N| +2460828|AAAAAAAAMJMIFCAA|2025-06-01|1505|6544|503|2025|0|6|1|2|2025|503|6544|Sunday|2025Q2|N|N|N|2460828|2460978|2460463|2460738|N|N|N|N|N| +2460829|AAAAAAAANJMIFCAA|2025-06-02|1505|6544|503|2025|1|6|2|2|2025|503|6544|Monday|2025Q2|N|N|N|2460828|2460978|2460464|2460739|N|N|N|N|N| +2460830|AAAAAAAAOJMIFCAA|2025-06-03|1505|6545|503|2025|2|6|3|2|2025|503|6545|Tuesday|2025Q2|N|N|N|2460828|2460978|2460465|2460740|N|N|N|N|N| +2460831|AAAAAAAAPJMIFCAA|2025-06-04|1505|6545|503|2025|3|6|4|2|2025|503|6545|Wednesday|2025Q2|N|N|N|2460828|2460978|2460466|2460741|N|N|N|N|N| +2460832|AAAAAAAAAKMIFCAA|2025-06-05|1505|6545|503|2025|4|6|5|2|2025|503|6545|Thursday|2025Q2|N|N|N|2460828|2460978|2460467|2460742|N|N|N|N|N| +2460833|AAAAAAAABKMIFCAA|2025-06-06|1505|6545|503|2025|5|6|6|2|2025|503|6545|Friday|2025Q2|N|Y|N|2460828|2460978|2460468|2460743|N|N|N|N|N| +2460834|AAAAAAAACKMIFCAA|2025-06-07|1505|6545|503|2025|6|6|7|2|2025|503|6545|Saturday|2025Q2|N|Y|N|2460828|2460978|2460469|2460744|N|N|N|N|N| +2460835|AAAAAAAADKMIFCAA|2025-06-08|1505|6545|503|2025|0|6|8|2|2025|503|6545|Sunday|2025Q2|N|N|N|2460828|2460978|2460470|2460745|N|N|N|N|N| +2460836|AAAAAAAAEKMIFCAA|2025-06-09|1505|6545|503|2025|1|6|9|2|2025|503|6545|Monday|2025Q2|N|N|N|2460828|2460978|2460471|2460746|N|N|N|N|N| +2460837|AAAAAAAAFKMIFCAA|2025-06-10|1505|6546|503|2025|2|6|10|2|2025|503|6546|Tuesday|2025Q2|N|N|N|2460828|2460978|2460472|2460747|N|N|N|N|N| +2460838|AAAAAAAAGKMIFCAA|2025-06-11|1505|6546|503|2025|3|6|11|2|2025|503|6546|Wednesday|2025Q2|N|N|N|2460828|2460978|2460473|2460748|N|N|N|N|N| +2460839|AAAAAAAAHKMIFCAA|2025-06-12|1505|6546|503|2025|4|6|12|2|2025|503|6546|Thursday|2025Q2|N|N|N|2460828|2460978|2460474|2460749|N|N|N|N|N| +2460840|AAAAAAAAIKMIFCAA|2025-06-13|1505|6546|503|2025|5|6|13|2|2025|503|6546|Friday|2025Q2|N|Y|N|2460828|2460978|2460475|2460750|N|N|N|N|N| +2460841|AAAAAAAAJKMIFCAA|2025-06-14|1505|6546|503|2025|6|6|14|2|2025|503|6546|Saturday|2025Q2|N|Y|N|2460828|2460978|2460476|2460751|N|N|N|N|N| +2460842|AAAAAAAAKKMIFCAA|2025-06-15|1505|6546|503|2025|0|6|15|2|2025|503|6546|Sunday|2025Q2|N|N|N|2460828|2460978|2460477|2460752|N|N|N|N|N| +2460843|AAAAAAAALKMIFCAA|2025-06-16|1505|6546|503|2025|1|6|16|2|2025|503|6546|Monday|2025Q2|N|N|N|2460828|2460978|2460478|2460753|N|N|N|N|N| +2460844|AAAAAAAAMKMIFCAA|2025-06-17|1505|6547|503|2025|2|6|17|2|2025|503|6547|Tuesday|2025Q2|N|N|N|2460828|2460978|2460479|2460754|N|N|N|N|N| +2460845|AAAAAAAANKMIFCAA|2025-06-18|1505|6547|503|2025|3|6|18|2|2025|503|6547|Wednesday|2025Q2|N|N|N|2460828|2460978|2460480|2460755|N|N|N|N|N| +2460846|AAAAAAAAOKMIFCAA|2025-06-19|1505|6547|503|2025|4|6|19|2|2025|503|6547|Thursday|2025Q2|N|N|N|2460828|2460978|2460481|2460756|N|N|N|N|N| +2460847|AAAAAAAAPKMIFCAA|2025-06-20|1505|6547|503|2025|5|6|20|2|2025|503|6547|Friday|2025Q2|N|Y|N|2460828|2460978|2460482|2460757|N|N|N|N|N| +2460848|AAAAAAAAALMIFCAA|2025-06-21|1505|6547|503|2025|6|6|21|2|2025|503|6547|Saturday|2025Q2|N|Y|N|2460828|2460978|2460483|2460758|N|N|N|N|N| +2460849|AAAAAAAABLMIFCAA|2025-06-22|1505|6547|503|2025|0|6|22|2|2025|503|6547|Sunday|2025Q2|N|N|N|2460828|2460978|2460484|2460759|N|N|N|N|N| +2460850|AAAAAAAACLMIFCAA|2025-06-23|1505|6547|503|2025|1|6|23|2|2025|503|6547|Monday|2025Q2|N|N|N|2460828|2460978|2460485|2460760|N|N|N|N|N| +2460851|AAAAAAAADLMIFCAA|2025-06-24|1505|6548|503|2025|2|6|24|2|2025|503|6548|Tuesday|2025Q2|N|N|N|2460828|2460978|2460486|2460761|N|N|N|N|N| +2460852|AAAAAAAAELMIFCAA|2025-06-25|1505|6548|503|2025|3|6|25|2|2025|503|6548|Wednesday|2025Q2|N|N|N|2460828|2460978|2460487|2460762|N|N|N|N|N| +2460853|AAAAAAAAFLMIFCAA|2025-06-26|1505|6548|503|2025|4|6|26|2|2025|503|6548|Thursday|2025Q2|N|N|N|2460828|2460978|2460488|2460763|N|N|N|N|N| +2460854|AAAAAAAAGLMIFCAA|2025-06-27|1505|6548|503|2025|5|6|27|2|2025|503|6548|Friday|2025Q2|N|Y|N|2460828|2460978|2460489|2460764|N|N|N|N|N| +2460855|AAAAAAAAHLMIFCAA|2025-06-28|1505|6548|503|2025|6|6|28|2|2025|503|6548|Saturday|2025Q2|N|Y|N|2460828|2460978|2460490|2460765|N|N|N|N|N| +2460856|AAAAAAAAILMIFCAA|2025-06-29|1505|6548|503|2025|0|6|29|2|2025|503|6548|Sunday|2025Q2|N|N|N|2460828|2460978|2460491|2460766|N|N|N|N|N| +2460857|AAAAAAAAJLMIFCAA|2025-06-30|1505|6548|503|2025|1|6|30|2|2025|503|6548|Monday|2025Q2|N|N|N|2460828|2460978|2460492|2460767|N|N|N|N|N| +2460858|AAAAAAAAKLMIFCAA|2025-07-01|1506|6549|503|2025|2|7|1|2|2025|503|6549|Tuesday|2025Q2|N|N|N|2460858|2461038|2460493|2460767|N|N|N|N|N| +2460859|AAAAAAAALLMIFCAA|2025-07-02|1506|6549|503|2025|3|7|2|3|2025|503|6549|Wednesday|2025Q3|N|N|N|2460858|2461038|2460494|2460768|N|N|N|N|N| +2460860|AAAAAAAAMLMIFCAA|2025-07-03|1506|6549|503|2025|4|7|3|3|2025|503|6549|Thursday|2025Q3|N|N|N|2460858|2461038|2460495|2460769|N|N|N|N|N| +2460861|AAAAAAAANLMIFCAA|2025-07-04|1506|6549|503|2025|5|7|4|3|2025|503|6549|Friday|2025Q3|N|Y|N|2460858|2461038|2460496|2460770|N|N|N|N|N| +2460862|AAAAAAAAOLMIFCAA|2025-07-05|1506|6549|503|2025|6|7|5|3|2025|503|6549|Saturday|2025Q3|Y|Y|N|2460858|2461038|2460497|2460771|N|N|N|N|N| +2460863|AAAAAAAAPLMIFCAA|2025-07-06|1506|6549|503|2025|0|7|6|3|2025|503|6549|Sunday|2025Q3|N|N|Y|2460858|2461038|2460498|2460772|N|N|N|N|N| +2460864|AAAAAAAAAMMIFCAA|2025-07-07|1506|6549|503|2025|1|7|7|3|2025|503|6549|Monday|2025Q3|N|N|N|2460858|2461038|2460499|2460773|N|N|N|N|N| +2460865|AAAAAAAABMMIFCAA|2025-07-08|1506|6550|503|2025|2|7|8|3|2025|503|6550|Tuesday|2025Q3|N|N|N|2460858|2461038|2460500|2460774|N|N|N|N|N| +2460866|AAAAAAAACMMIFCAA|2025-07-09|1506|6550|503|2025|3|7|9|3|2025|503|6550|Wednesday|2025Q3|N|N|N|2460858|2461038|2460501|2460775|N|N|N|N|N| +2460867|AAAAAAAADMMIFCAA|2025-07-10|1506|6550|503|2025|4|7|10|3|2025|503|6550|Thursday|2025Q3|N|N|N|2460858|2461038|2460502|2460776|N|N|N|N|N| +2460868|AAAAAAAAEMMIFCAA|2025-07-11|1506|6550|503|2025|5|7|11|3|2025|503|6550|Friday|2025Q3|N|Y|N|2460858|2461038|2460503|2460777|N|N|N|N|N| +2460869|AAAAAAAAFMMIFCAA|2025-07-12|1506|6550|503|2025|6|7|12|3|2025|503|6550|Saturday|2025Q3|N|Y|N|2460858|2461038|2460504|2460778|N|N|N|N|N| +2460870|AAAAAAAAGMMIFCAA|2025-07-13|1506|6550|503|2025|0|7|13|3|2025|503|6550|Sunday|2025Q3|N|N|N|2460858|2461038|2460505|2460779|N|N|N|N|N| +2460871|AAAAAAAAHMMIFCAA|2025-07-14|1506|6550|503|2025|1|7|14|3|2025|503|6550|Monday|2025Q3|N|N|N|2460858|2461038|2460506|2460780|N|N|N|N|N| +2460872|AAAAAAAAIMMIFCAA|2025-07-15|1506|6551|503|2025|2|7|15|3|2025|503|6551|Tuesday|2025Q3|N|N|N|2460858|2461038|2460507|2460781|N|N|N|N|N| +2460873|AAAAAAAAJMMIFCAA|2025-07-16|1506|6551|503|2025|3|7|16|3|2025|503|6551|Wednesday|2025Q3|N|N|N|2460858|2461038|2460508|2460782|N|N|N|N|N| +2460874|AAAAAAAAKMMIFCAA|2025-07-17|1506|6551|503|2025|4|7|17|3|2025|503|6551|Thursday|2025Q3|N|N|N|2460858|2461038|2460509|2460783|N|N|N|N|N| +2460875|AAAAAAAALMMIFCAA|2025-07-18|1506|6551|503|2025|5|7|18|3|2025|503|6551|Friday|2025Q3|N|Y|N|2460858|2461038|2460510|2460784|N|N|N|N|N| +2460876|AAAAAAAAMMMIFCAA|2025-07-19|1506|6551|503|2025|6|7|19|3|2025|503|6551|Saturday|2025Q3|N|Y|N|2460858|2461038|2460511|2460785|N|N|N|N|N| +2460877|AAAAAAAANMMIFCAA|2025-07-20|1506|6551|503|2025|0|7|20|3|2025|503|6551|Sunday|2025Q3|N|N|N|2460858|2461038|2460512|2460786|N|N|N|N|N| +2460878|AAAAAAAAOMMIFCAA|2025-07-21|1506|6551|503|2025|1|7|21|3|2025|503|6551|Monday|2025Q3|N|N|N|2460858|2461038|2460513|2460787|N|N|N|N|N| +2460879|AAAAAAAAPMMIFCAA|2025-07-22|1506|6552|503|2025|2|7|22|3|2025|503|6552|Tuesday|2025Q3|N|N|N|2460858|2461038|2460514|2460788|N|N|N|N|N| +2460880|AAAAAAAAANMIFCAA|2025-07-23|1506|6552|503|2025|3|7|23|3|2025|503|6552|Wednesday|2025Q3|N|N|N|2460858|2461038|2460515|2460789|N|N|N|N|N| +2460881|AAAAAAAABNMIFCAA|2025-07-24|1506|6552|503|2025|4|7|24|3|2025|503|6552|Thursday|2025Q3|N|N|N|2460858|2461038|2460516|2460790|N|N|N|N|N| +2460882|AAAAAAAACNMIFCAA|2025-07-25|1506|6552|503|2025|5|7|25|3|2025|503|6552|Friday|2025Q3|N|Y|N|2460858|2461038|2460517|2460791|N|N|N|N|N| +2460883|AAAAAAAADNMIFCAA|2025-07-26|1506|6552|503|2025|6|7|26|3|2025|503|6552|Saturday|2025Q3|N|Y|N|2460858|2461038|2460518|2460792|N|N|N|N|N| +2460884|AAAAAAAAENMIFCAA|2025-07-27|1506|6552|503|2025|0|7|27|3|2025|503|6552|Sunday|2025Q3|N|N|N|2460858|2461038|2460519|2460793|N|N|N|N|N| +2460885|AAAAAAAAFNMIFCAA|2025-07-28|1506|6552|503|2025|1|7|28|3|2025|503|6552|Monday|2025Q3|N|N|N|2460858|2461038|2460520|2460794|N|N|N|N|N| +2460886|AAAAAAAAGNMIFCAA|2025-07-29|1506|6553|503|2025|2|7|29|3|2025|503|6553|Tuesday|2025Q3|N|N|N|2460858|2461038|2460521|2460795|N|N|N|N|N| +2460887|AAAAAAAAHNMIFCAA|2025-07-30|1506|6553|503|2025|3|7|30|3|2025|503|6553|Wednesday|2025Q3|N|N|N|2460858|2461038|2460522|2460796|N|N|N|N|N| +2460888|AAAAAAAAINMIFCAA|2025-07-31|1506|6553|503|2025|4|7|31|3|2025|503|6553|Thursday|2025Q3|N|N|N|2460858|2461038|2460523|2460797|N|N|N|N|N| +2460889|AAAAAAAAJNMIFCAA|2025-08-01|1507|6553|503|2025|5|8|1|3|2025|503|6553|Friday|2025Q3|N|Y|N|2460889|2461100|2460524|2460798|N|N|N|N|N| +2460890|AAAAAAAAKNMIFCAA|2025-08-02|1507|6553|503|2025|6|8|2|3|2025|503|6553|Saturday|2025Q3|N|Y|N|2460889|2461100|2460525|2460799|N|N|N|N|N| +2460891|AAAAAAAALNMIFCAA|2025-08-03|1507|6553|503|2025|0|8|3|3|2025|503|6553|Sunday|2025Q3|N|N|N|2460889|2461100|2460526|2460800|N|N|N|N|N| +2460892|AAAAAAAAMNMIFCAA|2025-08-04|1507|6553|503|2025|1|8|4|3|2025|503|6553|Monday|2025Q3|N|N|N|2460889|2461100|2460527|2460801|N|N|N|N|N| +2460893|AAAAAAAANNMIFCAA|2025-08-05|1507|6554|503|2025|2|8|5|3|2025|503|6554|Tuesday|2025Q3|N|N|N|2460889|2461100|2460528|2460802|N|N|N|N|N| +2460894|AAAAAAAAONMIFCAA|2025-08-06|1507|6554|503|2025|3|8|6|3|2025|503|6554|Wednesday|2025Q3|N|N|N|2460889|2461100|2460529|2460803|N|N|N|N|N| +2460895|AAAAAAAAPNMIFCAA|2025-08-07|1507|6554|503|2025|4|8|7|3|2025|503|6554|Thursday|2025Q3|N|N|N|2460889|2461100|2460530|2460804|N|N|N|N|N| +2460896|AAAAAAAAAOMIFCAA|2025-08-08|1507|6554|503|2025|5|8|8|3|2025|503|6554|Friday|2025Q3|N|Y|N|2460889|2461100|2460531|2460805|N|N|N|N|N| +2460897|AAAAAAAABOMIFCAA|2025-08-09|1507|6554|503|2025|6|8|9|3|2025|503|6554|Saturday|2025Q3|N|Y|N|2460889|2461100|2460532|2460806|N|N|N|N|N| +2460898|AAAAAAAACOMIFCAA|2025-08-10|1507|6554|503|2025|0|8|10|3|2025|503|6554|Sunday|2025Q3|N|N|N|2460889|2461100|2460533|2460807|N|N|N|N|N| +2460899|AAAAAAAADOMIFCAA|2025-08-11|1507|6554|503|2025|1|8|11|3|2025|503|6554|Monday|2025Q3|N|N|N|2460889|2461100|2460534|2460808|N|N|N|N|N| +2460900|AAAAAAAAEOMIFCAA|2025-08-12|1507|6555|503|2025|2|8|12|3|2025|503|6555|Tuesday|2025Q3|N|N|N|2460889|2461100|2460535|2460809|N|N|N|N|N| +2460901|AAAAAAAAFOMIFCAA|2025-08-13|1507|6555|503|2025|3|8|13|3|2025|503|6555|Wednesday|2025Q3|N|N|N|2460889|2461100|2460536|2460810|N|N|N|N|N| +2460902|AAAAAAAAGOMIFCAA|2025-08-14|1507|6555|503|2025|4|8|14|3|2025|503|6555|Thursday|2025Q3|N|N|N|2460889|2461100|2460537|2460811|N|N|N|N|N| +2460903|AAAAAAAAHOMIFCAA|2025-08-15|1507|6555|503|2025|5|8|15|3|2025|503|6555|Friday|2025Q3|N|Y|N|2460889|2461100|2460538|2460812|N|N|N|N|N| +2460904|AAAAAAAAIOMIFCAA|2025-08-16|1507|6555|503|2025|6|8|16|3|2025|503|6555|Saturday|2025Q3|N|Y|N|2460889|2461100|2460539|2460813|N|N|N|N|N| +2460905|AAAAAAAAJOMIFCAA|2025-08-17|1507|6555|503|2025|0|8|17|3|2025|503|6555|Sunday|2025Q3|N|N|N|2460889|2461100|2460540|2460814|N|N|N|N|N| +2460906|AAAAAAAAKOMIFCAA|2025-08-18|1507|6555|503|2025|1|8|18|3|2025|503|6555|Monday|2025Q3|N|N|N|2460889|2461100|2460541|2460815|N|N|N|N|N| +2460907|AAAAAAAALOMIFCAA|2025-08-19|1507|6556|503|2025|2|8|19|3|2025|503|6556|Tuesday|2025Q3|N|N|N|2460889|2461100|2460542|2460816|N|N|N|N|N| +2460908|AAAAAAAAMOMIFCAA|2025-08-20|1507|6556|503|2025|3|8|20|3|2025|503|6556|Wednesday|2025Q3|N|N|N|2460889|2461100|2460543|2460817|N|N|N|N|N| +2460909|AAAAAAAANOMIFCAA|2025-08-21|1507|6556|503|2025|4|8|21|3|2025|503|6556|Thursday|2025Q3|N|N|N|2460889|2461100|2460544|2460818|N|N|N|N|N| +2460910|AAAAAAAAOOMIFCAA|2025-08-22|1507|6556|503|2025|5|8|22|3|2025|503|6556|Friday|2025Q3|N|Y|N|2460889|2461100|2460545|2460819|N|N|N|N|N| +2460911|AAAAAAAAPOMIFCAA|2025-08-23|1507|6556|503|2025|6|8|23|3|2025|503|6556|Saturday|2025Q3|N|Y|N|2460889|2461100|2460546|2460820|N|N|N|N|N| +2460912|AAAAAAAAAPMIFCAA|2025-08-24|1507|6556|503|2025|0|8|24|3|2025|503|6556|Sunday|2025Q3|N|N|N|2460889|2461100|2460547|2460821|N|N|N|N|N| +2460913|AAAAAAAABPMIFCAA|2025-08-25|1507|6556|503|2025|1|8|25|3|2025|503|6556|Monday|2025Q3|N|N|N|2460889|2461100|2460548|2460822|N|N|N|N|N| +2460914|AAAAAAAACPMIFCAA|2025-08-26|1507|6557|503|2025|2|8|26|3|2025|503|6557|Tuesday|2025Q3|N|N|N|2460889|2461100|2460549|2460823|N|N|N|N|N| +2460915|AAAAAAAADPMIFCAA|2025-08-27|1507|6557|503|2025|3|8|27|3|2025|503|6557|Wednesday|2025Q3|N|N|N|2460889|2461100|2460550|2460824|N|N|N|N|N| +2460916|AAAAAAAAEPMIFCAA|2025-08-28|1507|6557|503|2025|4|8|28|3|2025|503|6557|Thursday|2025Q3|N|N|N|2460889|2461100|2460551|2460825|N|N|N|N|N| +2460917|AAAAAAAAFPMIFCAA|2025-08-29|1507|6557|503|2025|5|8|29|3|2025|503|6557|Friday|2025Q3|N|Y|N|2460889|2461100|2460552|2460826|N|N|N|N|N| +2460918|AAAAAAAAGPMIFCAA|2025-08-30|1507|6557|503|2025|6|8|30|3|2025|503|6557|Saturday|2025Q3|N|Y|N|2460889|2461100|2460553|2460827|N|N|N|N|N| +2460919|AAAAAAAAHPMIFCAA|2025-08-31|1507|6557|503|2025|0|8|31|3|2025|503|6557|Sunday|2025Q3|N|N|N|2460889|2461100|2460554|2460828|N|N|N|N|N| +2460920|AAAAAAAAIPMIFCAA|2025-09-01|1508|6557|504|2025|1|9|1|3|2025|504|6557|Monday|2025Q3|N|N|N|2460920|2461162|2460555|2460829|N|N|N|N|N| +2460921|AAAAAAAAJPMIFCAA|2025-09-02|1508|6558|504|2025|2|9|2|3|2025|504|6558|Tuesday|2025Q3|N|N|N|2460920|2461162|2460556|2460830|N|N|N|N|N| +2460922|AAAAAAAAKPMIFCAA|2025-09-03|1508|6558|504|2025|3|9|3|3|2025|504|6558|Wednesday|2025Q3|N|N|N|2460920|2461162|2460557|2460831|N|N|N|N|N| +2460923|AAAAAAAALPMIFCAA|2025-09-04|1508|6558|504|2025|4|9|4|3|2025|504|6558|Thursday|2025Q3|N|N|N|2460920|2461162|2460558|2460832|N|N|N|N|N| +2460924|AAAAAAAAMPMIFCAA|2025-09-05|1508|6558|504|2025|5|9|5|3|2025|504|6558|Friday|2025Q3|N|Y|N|2460920|2461162|2460559|2460833|N|N|N|N|N| +2460925|AAAAAAAANPMIFCAA|2025-09-06|1508|6558|504|2025|6|9|6|3|2025|504|6558|Saturday|2025Q3|N|Y|N|2460920|2461162|2460560|2460834|N|N|N|N|N| +2460926|AAAAAAAAOPMIFCAA|2025-09-07|1508|6558|504|2025|0|9|7|3|2025|504|6558|Sunday|2025Q3|N|N|N|2460920|2461162|2460561|2460835|N|N|N|N|N| +2460927|AAAAAAAAPPMIFCAA|2025-09-08|1508|6558|504|2025|1|9|8|3|2025|504|6558|Monday|2025Q3|N|N|N|2460920|2461162|2460562|2460836|N|N|N|N|N| +2460928|AAAAAAAAAANIFCAA|2025-09-09|1508|6559|504|2025|2|9|9|3|2025|504|6559|Tuesday|2025Q3|N|N|N|2460920|2461162|2460563|2460837|N|N|N|N|N| +2460929|AAAAAAAABANIFCAA|2025-09-10|1508|6559|504|2025|3|9|10|3|2025|504|6559|Wednesday|2025Q3|N|N|N|2460920|2461162|2460564|2460838|N|N|N|N|N| +2460930|AAAAAAAACANIFCAA|2025-09-11|1508|6559|504|2025|4|9|11|3|2025|504|6559|Thursday|2025Q3|N|N|N|2460920|2461162|2460565|2460839|N|N|N|N|N| +2460931|AAAAAAAADANIFCAA|2025-09-12|1508|6559|504|2025|5|9|12|3|2025|504|6559|Friday|2025Q3|N|Y|N|2460920|2461162|2460566|2460840|N|N|N|N|N| +2460932|AAAAAAAAEANIFCAA|2025-09-13|1508|6559|504|2025|6|9|13|3|2025|504|6559|Saturday|2025Q3|N|Y|N|2460920|2461162|2460567|2460841|N|N|N|N|N| +2460933|AAAAAAAAFANIFCAA|2025-09-14|1508|6559|504|2025|0|9|14|3|2025|504|6559|Sunday|2025Q3|N|N|N|2460920|2461162|2460568|2460842|N|N|N|N|N| +2460934|AAAAAAAAGANIFCAA|2025-09-15|1508|6559|504|2025|1|9|15|3|2025|504|6559|Monday|2025Q3|N|N|N|2460920|2461162|2460569|2460843|N|N|N|N|N| +2460935|AAAAAAAAHANIFCAA|2025-09-16|1508|6560|504|2025|2|9|16|3|2025|504|6560|Tuesday|2025Q3|N|N|N|2460920|2461162|2460570|2460844|N|N|N|N|N| +2460936|AAAAAAAAIANIFCAA|2025-09-17|1508|6560|504|2025|3|9|17|3|2025|504|6560|Wednesday|2025Q3|N|N|N|2460920|2461162|2460571|2460845|N|N|N|N|N| +2460937|AAAAAAAAJANIFCAA|2025-09-18|1508|6560|504|2025|4|9|18|3|2025|504|6560|Thursday|2025Q3|N|N|N|2460920|2461162|2460572|2460846|N|N|N|N|N| +2460938|AAAAAAAAKANIFCAA|2025-09-19|1508|6560|504|2025|5|9|19|3|2025|504|6560|Friday|2025Q3|N|Y|N|2460920|2461162|2460573|2460847|N|N|N|N|N| +2460939|AAAAAAAALANIFCAA|2025-09-20|1508|6560|504|2025|6|9|20|3|2025|504|6560|Saturday|2025Q3|N|Y|N|2460920|2461162|2460574|2460848|N|N|N|N|N| +2460940|AAAAAAAAMANIFCAA|2025-09-21|1508|6560|504|2025|0|9|21|3|2025|504|6560|Sunday|2025Q3|N|N|N|2460920|2461162|2460575|2460849|N|N|N|N|N| +2460941|AAAAAAAANANIFCAA|2025-09-22|1508|6560|504|2025|1|9|22|3|2025|504|6560|Monday|2025Q3|N|N|N|2460920|2461162|2460576|2460850|N|N|N|N|N| +2460942|AAAAAAAAOANIFCAA|2025-09-23|1508|6561|504|2025|2|9|23|3|2025|504|6561|Tuesday|2025Q3|N|N|N|2460920|2461162|2460577|2460851|N|N|N|N|N| +2460943|AAAAAAAAPANIFCAA|2025-09-24|1508|6561|504|2025|3|9|24|3|2025|504|6561|Wednesday|2025Q3|N|N|N|2460920|2461162|2460578|2460852|N|N|N|N|N| +2460944|AAAAAAAAABNIFCAA|2025-09-25|1508|6561|504|2025|4|9|25|3|2025|504|6561|Thursday|2025Q3|N|N|N|2460920|2461162|2460579|2460853|N|N|N|N|N| +2460945|AAAAAAAABBNIFCAA|2025-09-26|1508|6561|504|2025|5|9|26|3|2025|504|6561|Friday|2025Q3|N|Y|N|2460920|2461162|2460580|2460854|N|N|N|N|N| +2460946|AAAAAAAACBNIFCAA|2025-09-27|1508|6561|504|2025|6|9|27|3|2025|504|6561|Saturday|2025Q3|N|Y|N|2460920|2461162|2460581|2460855|N|N|N|N|N| +2460947|AAAAAAAADBNIFCAA|2025-09-28|1508|6561|504|2025|0|9|28|3|2025|504|6561|Sunday|2025Q3|N|N|N|2460920|2461162|2460582|2460856|N|N|N|N|N| +2460948|AAAAAAAAEBNIFCAA|2025-09-29|1508|6561|504|2025|1|9|29|3|2025|504|6561|Monday|2025Q3|N|N|N|2460920|2461162|2460583|2460857|N|N|N|N|N| +2460949|AAAAAAAAFBNIFCAA|2025-09-30|1508|6562|504|2025|2|9|30|3|2025|504|6562|Tuesday|2025Q3|N|N|N|2460920|2461162|2460584|2460858|N|N|N|N|N| +2460950|AAAAAAAAGBNIFCAA|2025-10-01|1509|6562|504|2025|3|10|1|3|2025|504|6562|Wednesday|2025Q3|N|N|N|2460950|2461222|2460585|2460858|N|N|N|N|N| +2460951|AAAAAAAAHBNIFCAA|2025-10-02|1509|6562|504|2025|4|10|2|4|2025|504|6562|Thursday|2025Q4|N|N|N|2460950|2461222|2460586|2460859|N|N|N|N|N| +2460952|AAAAAAAAIBNIFCAA|2025-10-03|1509|6562|504|2025|5|10|3|4|2025|504|6562|Friday|2025Q4|N|Y|N|2460950|2461222|2460587|2460860|N|N|N|N|N| +2460953|AAAAAAAAJBNIFCAA|2025-10-04|1509|6562|504|2025|6|10|4|4|2025|504|6562|Saturday|2025Q4|N|Y|N|2460950|2461222|2460588|2460861|N|N|N|N|N| +2460954|AAAAAAAAKBNIFCAA|2025-10-05|1509|6562|504|2025|0|10|5|4|2025|504|6562|Sunday|2025Q4|N|N|N|2460950|2461222|2460589|2460862|N|N|N|N|N| +2460955|AAAAAAAALBNIFCAA|2025-10-06|1509|6562|504|2025|1|10|6|4|2025|504|6562|Monday|2025Q4|N|N|N|2460950|2461222|2460590|2460863|N|N|N|N|N| +2460956|AAAAAAAAMBNIFCAA|2025-10-07|1509|6563|504|2025|2|10|7|4|2025|504|6563|Tuesday|2025Q4|N|N|N|2460950|2461222|2460591|2460864|N|N|N|N|N| +2460957|AAAAAAAANBNIFCAA|2025-10-08|1509|6563|504|2025|3|10|8|4|2025|504|6563|Wednesday|2025Q4|N|N|N|2460950|2461222|2460592|2460865|N|N|N|N|N| +2460958|AAAAAAAAOBNIFCAA|2025-10-09|1509|6563|504|2025|4|10|9|4|2025|504|6563|Thursday|2025Q4|N|N|N|2460950|2461222|2460593|2460866|N|N|N|N|N| +2460959|AAAAAAAAPBNIFCAA|2025-10-10|1509|6563|504|2025|5|10|10|4|2025|504|6563|Friday|2025Q4|N|Y|N|2460950|2461222|2460594|2460867|N|N|N|N|N| +2460960|AAAAAAAAACNIFCAA|2025-10-11|1509|6563|504|2025|6|10|11|4|2025|504|6563|Saturday|2025Q4|N|Y|N|2460950|2461222|2460595|2460868|N|N|N|N|N| +2460961|AAAAAAAABCNIFCAA|2025-10-12|1509|6563|504|2025|0|10|12|4|2025|504|6563|Sunday|2025Q4|N|N|N|2460950|2461222|2460596|2460869|N|N|N|N|N| +2460962|AAAAAAAACCNIFCAA|2025-10-13|1509|6563|504|2025|1|10|13|4|2025|504|6563|Monday|2025Q4|N|N|N|2460950|2461222|2460597|2460870|N|N|N|N|N| +2460963|AAAAAAAADCNIFCAA|2025-10-14|1509|6564|504|2025|2|10|14|4|2025|504|6564|Tuesday|2025Q4|N|N|N|2460950|2461222|2460598|2460871|N|N|N|N|N| +2460964|AAAAAAAAECNIFCAA|2025-10-15|1509|6564|504|2025|3|10|15|4|2025|504|6564|Wednesday|2025Q4|N|N|N|2460950|2461222|2460599|2460872|N|N|N|N|N| +2460965|AAAAAAAAFCNIFCAA|2025-10-16|1509|6564|504|2025|4|10|16|4|2025|504|6564|Thursday|2025Q4|N|N|N|2460950|2461222|2460600|2460873|N|N|N|N|N| +2460966|AAAAAAAAGCNIFCAA|2025-10-17|1509|6564|504|2025|5|10|17|4|2025|504|6564|Friday|2025Q4|N|Y|N|2460950|2461222|2460601|2460874|N|N|N|N|N| +2460967|AAAAAAAAHCNIFCAA|2025-10-18|1509|6564|504|2025|6|10|18|4|2025|504|6564|Saturday|2025Q4|N|Y|N|2460950|2461222|2460602|2460875|N|N|N|N|N| +2460968|AAAAAAAAICNIFCAA|2025-10-19|1509|6564|504|2025|0|10|19|4|2025|504|6564|Sunday|2025Q4|N|N|N|2460950|2461222|2460603|2460876|N|N|N|N|N| +2460969|AAAAAAAAJCNIFCAA|2025-10-20|1509|6564|504|2025|1|10|20|4|2025|504|6564|Monday|2025Q4|N|N|N|2460950|2461222|2460604|2460877|N|N|N|N|N| +2460970|AAAAAAAAKCNIFCAA|2025-10-21|1509|6565|504|2025|2|10|21|4|2025|504|6565|Tuesday|2025Q4|N|N|N|2460950|2461222|2460605|2460878|N|N|N|N|N| +2460971|AAAAAAAALCNIFCAA|2025-10-22|1509|6565|504|2025|3|10|22|4|2025|504|6565|Wednesday|2025Q4|N|N|N|2460950|2461222|2460606|2460879|N|N|N|N|N| +2460972|AAAAAAAAMCNIFCAA|2025-10-23|1509|6565|504|2025|4|10|23|4|2025|504|6565|Thursday|2025Q4|N|N|N|2460950|2461222|2460607|2460880|N|N|N|N|N| +2460973|AAAAAAAANCNIFCAA|2025-10-24|1509|6565|504|2025|5|10|24|4|2025|504|6565|Friday|2025Q4|N|Y|N|2460950|2461222|2460608|2460881|N|N|N|N|N| +2460974|AAAAAAAAOCNIFCAA|2025-10-25|1509|6565|504|2025|6|10|25|4|2025|504|6565|Saturday|2025Q4|N|Y|N|2460950|2461222|2460609|2460882|N|N|N|N|N| +2460975|AAAAAAAAPCNIFCAA|2025-10-26|1509|6565|504|2025|0|10|26|4|2025|504|6565|Sunday|2025Q4|N|N|N|2460950|2461222|2460610|2460883|N|N|N|N|N| +2460976|AAAAAAAAADNIFCAA|2025-10-27|1509|6565|504|2025|1|10|27|4|2025|504|6565|Monday|2025Q4|N|N|N|2460950|2461222|2460611|2460884|N|N|N|N|N| +2460977|AAAAAAAABDNIFCAA|2025-10-28|1509|6566|504|2025|2|10|28|4|2025|504|6566|Tuesday|2025Q4|N|N|N|2460950|2461222|2460612|2460885|N|N|N|N|N| +2460978|AAAAAAAACDNIFCAA|2025-10-29|1509|6566|504|2025|3|10|29|4|2025|504|6566|Wednesday|2025Q4|N|N|N|2460950|2461222|2460613|2460886|N|N|N|N|N| +2460979|AAAAAAAADDNIFCAA|2025-10-30|1509|6566|504|2025|4|10|30|4|2025|504|6566|Thursday|2025Q4|N|N|N|2460950|2461222|2460614|2460887|N|N|N|N|N| +2460980|AAAAAAAAEDNIFCAA|2025-10-31|1509|6566|504|2025|5|10|31|4|2025|504|6566|Friday|2025Q4|N|Y|N|2460950|2461222|2460615|2460888|N|N|N|N|N| +2460981|AAAAAAAAFDNIFCAA|2025-11-01|1510|6566|504|2025|6|11|1|4|2025|504|6566|Saturday|2025Q4|N|Y|N|2460981|2461284|2460616|2460889|N|N|N|N|N| +2460982|AAAAAAAAGDNIFCAA|2025-11-02|1510|6566|504|2025|0|11|2|4|2025|504|6566|Sunday|2025Q4|N|N|N|2460981|2461284|2460617|2460890|N|N|N|N|N| +2460983|AAAAAAAAHDNIFCAA|2025-11-03|1510|6566|504|2025|1|11|3|4|2025|504|6566|Monday|2025Q4|N|N|N|2460981|2461284|2460618|2460891|N|N|N|N|N| +2460984|AAAAAAAAIDNIFCAA|2025-11-04|1510|6567|504|2025|2|11|4|4|2025|504|6567|Tuesday|2025Q4|N|N|N|2460981|2461284|2460619|2460892|N|N|N|N|N| +2460985|AAAAAAAAJDNIFCAA|2025-11-05|1510|6567|504|2025|3|11|5|4|2025|504|6567|Wednesday|2025Q4|N|N|N|2460981|2461284|2460620|2460893|N|N|N|N|N| +2460986|AAAAAAAAKDNIFCAA|2025-11-06|1510|6567|504|2025|4|11|6|4|2025|504|6567|Thursday|2025Q4|N|N|N|2460981|2461284|2460621|2460894|N|N|N|N|N| +2460987|AAAAAAAALDNIFCAA|2025-11-07|1510|6567|504|2025|5|11|7|4|2025|504|6567|Friday|2025Q4|N|Y|N|2460981|2461284|2460622|2460895|N|N|N|N|N| +2460988|AAAAAAAAMDNIFCAA|2025-11-08|1510|6567|504|2025|6|11|8|4|2025|504|6567|Saturday|2025Q4|N|Y|N|2460981|2461284|2460623|2460896|N|N|N|N|N| +2460989|AAAAAAAANDNIFCAA|2025-11-09|1510|6567|504|2025|0|11|9|4|2025|504|6567|Sunday|2025Q4|N|N|N|2460981|2461284|2460624|2460897|N|N|N|N|N| +2460990|AAAAAAAAODNIFCAA|2025-11-10|1510|6567|504|2025|1|11|10|4|2025|504|6567|Monday|2025Q4|N|N|N|2460981|2461284|2460625|2460898|N|N|N|N|N| +2460991|AAAAAAAAPDNIFCAA|2025-11-11|1510|6568|504|2025|2|11|11|4|2025|504|6568|Tuesday|2025Q4|N|N|N|2460981|2461284|2460626|2460899|N|N|N|N|N| +2460992|AAAAAAAAAENIFCAA|2025-11-12|1510|6568|504|2025|3|11|12|4|2025|504|6568|Wednesday|2025Q4|N|N|N|2460981|2461284|2460627|2460900|N|N|N|N|N| +2460993|AAAAAAAABENIFCAA|2025-11-13|1510|6568|504|2025|4|11|13|4|2025|504|6568|Thursday|2025Q4|N|N|N|2460981|2461284|2460628|2460901|N|N|N|N|N| +2460994|AAAAAAAACENIFCAA|2025-11-14|1510|6568|504|2025|5|11|14|4|2025|504|6568|Friday|2025Q4|N|Y|N|2460981|2461284|2460629|2460902|N|N|N|N|N| +2460995|AAAAAAAADENIFCAA|2025-11-15|1510|6568|504|2025|6|11|15|4|2025|504|6568|Saturday|2025Q4|N|Y|N|2460981|2461284|2460630|2460903|N|N|N|N|N| +2460996|AAAAAAAAEENIFCAA|2025-11-16|1510|6568|504|2025|0|11|16|4|2025|504|6568|Sunday|2025Q4|N|N|N|2460981|2461284|2460631|2460904|N|N|N|N|N| +2460997|AAAAAAAAFENIFCAA|2025-11-17|1510|6568|504|2025|1|11|17|4|2025|504|6568|Monday|2025Q4|N|N|N|2460981|2461284|2460632|2460905|N|N|N|N|N| +2460998|AAAAAAAAGENIFCAA|2025-11-18|1510|6569|504|2025|2|11|18|4|2025|504|6569|Tuesday|2025Q4|N|N|N|2460981|2461284|2460633|2460906|N|N|N|N|N| +2460999|AAAAAAAAHENIFCAA|2025-11-19|1510|6569|504|2025|3|11|19|4|2025|504|6569|Wednesday|2025Q4|N|N|N|2460981|2461284|2460634|2460907|N|N|N|N|N| +2461000|AAAAAAAAIENIFCAA|2025-11-20|1510|6569|504|2025|4|11|20|4|2025|504|6569|Thursday|2025Q4|N|N|N|2460981|2461284|2460635|2460908|N|N|N|N|N| +2461001|AAAAAAAAJENIFCAA|2025-11-21|1510|6569|504|2025|5|11|21|4|2025|504|6569|Friday|2025Q4|N|Y|N|2460981|2461284|2460636|2460909|N|N|N|N|N| +2461002|AAAAAAAAKENIFCAA|2025-11-22|1510|6569|504|2025|6|11|22|4|2025|504|6569|Saturday|2025Q4|N|Y|N|2460981|2461284|2460637|2460910|N|N|N|N|N| +2461003|AAAAAAAALENIFCAA|2025-11-23|1510|6569|504|2025|0|11|23|4|2025|504|6569|Sunday|2025Q4|N|N|N|2460981|2461284|2460638|2460911|N|N|N|N|N| +2461004|AAAAAAAAMENIFCAA|2025-11-24|1510|6569|504|2025|1|11|24|4|2025|504|6569|Monday|2025Q4|N|N|N|2460981|2461284|2460639|2460912|N|N|N|N|N| +2461005|AAAAAAAANENIFCAA|2025-11-25|1510|6570|504|2025|2|11|25|4|2025|504|6570|Tuesday|2025Q4|N|N|N|2460981|2461284|2460640|2460913|N|N|N|N|N| +2461006|AAAAAAAAOENIFCAA|2025-11-26|1510|6570|504|2025|3|11|26|4|2025|504|6570|Wednesday|2025Q4|N|N|N|2460981|2461284|2460641|2460914|N|N|N|N|N| +2461007|AAAAAAAAPENIFCAA|2025-11-27|1510|6570|504|2025|4|11|27|4|2025|504|6570|Thursday|2025Q4|N|N|N|2460981|2461284|2460642|2460915|N|N|N|N|N| +2461008|AAAAAAAAAFNIFCAA|2025-11-28|1510|6570|504|2025|5|11|28|4|2025|504|6570|Friday|2025Q4|N|Y|N|2460981|2461284|2460643|2460916|N|N|N|N|N| +2461009|AAAAAAAABFNIFCAA|2025-11-29|1510|6570|504|2025|6|11|29|4|2025|504|6570|Saturday|2025Q4|N|Y|N|2460981|2461284|2460644|2460917|N|N|N|N|N| +2461010|AAAAAAAACFNIFCAA|2025-11-30|1510|6570|504|2025|0|11|30|4|2025|504|6570|Sunday|2025Q4|N|N|N|2460981|2461284|2460645|2460918|N|N|N|N|N| +2461011|AAAAAAAADFNIFCAA|2025-12-01|1511|6570|505|2025|1|12|1|4|2025|505|6570|Monday|2025Q4|N|N|N|2461011|2461344|2460646|2460919|N|N|N|N|N| +2461012|AAAAAAAAEFNIFCAA|2025-12-02|1511|6571|505|2025|2|12|2|4|2025|505|6571|Tuesday|2025Q4|N|N|N|2461011|2461344|2460647|2460920|N|N|N|N|N| +2461013|AAAAAAAAFFNIFCAA|2025-12-03|1511|6571|505|2025|3|12|3|4|2025|505|6571|Wednesday|2025Q4|N|N|N|2461011|2461344|2460648|2460921|N|N|N|N|N| +2461014|AAAAAAAAGFNIFCAA|2025-12-04|1511|6571|505|2025|4|12|4|4|2025|505|6571|Thursday|2025Q4|N|N|N|2461011|2461344|2460649|2460922|N|N|N|N|N| +2461015|AAAAAAAAHFNIFCAA|2025-12-05|1511|6571|505|2025|5|12|5|4|2025|505|6571|Friday|2025Q4|N|Y|N|2461011|2461344|2460650|2460923|N|N|N|N|N| +2461016|AAAAAAAAIFNIFCAA|2025-12-06|1511|6571|505|2025|6|12|6|4|2025|505|6571|Saturday|2025Q4|N|Y|N|2461011|2461344|2460651|2460924|N|N|N|N|N| +2461017|AAAAAAAAJFNIFCAA|2025-12-07|1511|6571|505|2025|0|12|7|4|2025|505|6571|Sunday|2025Q4|N|N|N|2461011|2461344|2460652|2460925|N|N|N|N|N| +2461018|AAAAAAAAKFNIFCAA|2025-12-08|1511|6571|505|2025|1|12|8|4|2025|505|6571|Monday|2025Q4|N|N|N|2461011|2461344|2460653|2460926|N|N|N|N|N| +2461019|AAAAAAAALFNIFCAA|2025-12-09|1511|6572|505|2025|2|12|9|4|2025|505|6572|Tuesday|2025Q4|N|N|N|2461011|2461344|2460654|2460927|N|N|N|N|N| +2461020|AAAAAAAAMFNIFCAA|2025-12-10|1511|6572|505|2025|3|12|10|4|2025|505|6572|Wednesday|2025Q4|N|N|N|2461011|2461344|2460655|2460928|N|N|N|N|N| +2461021|AAAAAAAANFNIFCAA|2025-12-11|1511|6572|505|2025|4|12|11|4|2025|505|6572|Thursday|2025Q4|N|N|N|2461011|2461344|2460656|2460929|N|N|N|N|N| +2461022|AAAAAAAAOFNIFCAA|2025-12-12|1511|6572|505|2025|5|12|12|4|2025|505|6572|Friday|2025Q4|N|Y|N|2461011|2461344|2460657|2460930|N|N|N|N|N| +2461023|AAAAAAAAPFNIFCAA|2025-12-13|1511|6572|505|2025|6|12|13|4|2025|505|6572|Saturday|2025Q4|N|Y|N|2461011|2461344|2460658|2460931|N|N|N|N|N| +2461024|AAAAAAAAAGNIFCAA|2025-12-14|1511|6572|505|2025|0|12|14|4|2025|505|6572|Sunday|2025Q4|N|N|N|2461011|2461344|2460659|2460932|N|N|N|N|N| +2461025|AAAAAAAABGNIFCAA|2025-12-15|1511|6572|505|2025|1|12|15|4|2025|505|6572|Monday|2025Q4|N|N|N|2461011|2461344|2460660|2460933|N|N|N|N|N| +2461026|AAAAAAAACGNIFCAA|2025-12-16|1511|6573|505|2025|2|12|16|4|2025|505|6573|Tuesday|2025Q4|N|N|N|2461011|2461344|2460661|2460934|N|N|N|N|N| +2461027|AAAAAAAADGNIFCAA|2025-12-17|1511|6573|505|2025|3|12|17|4|2025|505|6573|Wednesday|2025Q4|N|N|N|2461011|2461344|2460662|2460935|N|N|N|N|N| +2461028|AAAAAAAAEGNIFCAA|2025-12-18|1511|6573|505|2025|4|12|18|4|2025|505|6573|Thursday|2025Q4|N|N|N|2461011|2461344|2460663|2460936|N|N|N|N|N| +2461029|AAAAAAAAFGNIFCAA|2025-12-19|1511|6573|505|2025|5|12|19|4|2025|505|6573|Friday|2025Q4|N|Y|N|2461011|2461344|2460664|2460937|N|N|N|N|N| +2461030|AAAAAAAAGGNIFCAA|2025-12-20|1511|6573|505|2025|6|12|20|4|2025|505|6573|Saturday|2025Q4|N|Y|N|2461011|2461344|2460665|2460938|N|N|N|N|N| +2461031|AAAAAAAAHGNIFCAA|2025-12-21|1511|6573|505|2025|0|12|21|4|2025|505|6573|Sunday|2025Q4|N|N|N|2461011|2461344|2460666|2460939|N|N|N|N|N| +2461032|AAAAAAAAIGNIFCAA|2025-12-22|1511|6573|505|2025|1|12|22|4|2025|505|6573|Monday|2025Q4|N|N|N|2461011|2461344|2460667|2460940|N|N|N|N|N| +2461033|AAAAAAAAJGNIFCAA|2025-12-23|1511|6574|505|2025|2|12|23|4|2025|505|6574|Tuesday|2025Q4|N|N|N|2461011|2461344|2460668|2460941|N|N|N|N|N| +2461034|AAAAAAAAKGNIFCAA|2025-12-24|1511|6574|505|2025|3|12|24|4|2025|505|6574|Wednesday|2025Q4|N|N|N|2461011|2461344|2460669|2460942|N|N|N|N|N| +2461035|AAAAAAAALGNIFCAA|2025-12-25|1511|6574|505|2025|4|12|25|4|2025|505|6574|Thursday|2025Q4|N|N|N|2461011|2461344|2460670|2460943|N|N|N|N|N| +2461036|AAAAAAAAMGNIFCAA|2025-12-26|1511|6574|505|2025|5|12|26|4|2025|505|6574|Friday|2025Q4|Y|Y|N|2461011|2461344|2460671|2460944|N|N|N|N|N| +2461037|AAAAAAAANGNIFCAA|2025-12-27|1511|6574|505|2025|6|12|27|4|2025|505|6574|Saturday|2025Q4|N|Y|Y|2461011|2461344|2460672|2460945|N|N|N|N|N| +2461038|AAAAAAAAOGNIFCAA|2025-12-28|1511|6574|505|2025|0|12|28|4|2025|505|6574|Sunday|2025Q4|N|N|N|2461011|2461344|2460673|2460946|N|N|N|N|N| +2461039|AAAAAAAAPGNIFCAA|2025-12-29|1511|6574|505|2025|1|12|29|4|2025|505|6574|Monday|2025Q4|N|N|N|2461011|2461344|2460674|2460947|N|N|N|N|N| +2461040|AAAAAAAAAHNIFCAA|2025-12-30|1511|6575|505|2025|2|12|30|4|2025|505|6575|Tuesday|2025Q4|N|N|N|2461011|2461344|2460675|2460948|N|N|N|N|N| +2461041|AAAAAAAABHNIFCAA|2025-12-31|1511|6575|505|2025|3|12|31|4|2025|505|6575|Wednesday|2025Q4|N|N|N|2461011|2461344|2460676|2460949|N|N|N|N|N| +2461042|AAAAAAAACHNIFCAA|2026-01-01|1512|6575|505|2026|4|1|1|1|2026|505|6575|Thursday|2026Q1|Y|N|N|2461042|2461041|2460677|2460950|N|N|N|N|N| +2461043|AAAAAAAADHNIFCAA|2026-01-02|1512|6575|505|2026|5|1|2|1|2026|505|6575|Friday|2026Q1|N|Y|Y|2461042|2461041|2460678|2460951|N|N|N|N|N| +2461044|AAAAAAAAEHNIFCAA|2026-01-03|1512|6575|505|2026|6|1|3|1|2026|505|6575|Saturday|2026Q1|N|Y|N|2461042|2461041|2460679|2460952|N|N|N|N|N| +2461045|AAAAAAAAFHNIFCAA|2026-01-04|1512|6575|505|2026|0|1|4|1|2026|505|6575|Sunday|2026Q1|N|N|N|2461042|2461041|2460680|2460953|N|N|N|N|N| +2461046|AAAAAAAAGHNIFCAA|2026-01-05|1512|6575|505|2026|1|1|5|1|2026|505|6575|Monday|2026Q1|N|N|N|2461042|2461041|2460681|2460954|N|N|N|N|N| +2461047|AAAAAAAAHHNIFCAA|2026-01-06|1512|6576|505|2026|2|1|6|1|2026|505|6576|Tuesday|2026Q1|N|N|N|2461042|2461041|2460682|2460955|N|N|N|N|N| +2461048|AAAAAAAAIHNIFCAA|2026-01-07|1512|6576|505|2026|3|1|7|1|2026|505|6576|Wednesday|2026Q1|N|N|N|2461042|2461041|2460683|2460956|N|N|N|N|N| +2461049|AAAAAAAAJHNIFCAA|2026-01-08|1512|6576|505|2026|4|1|8|1|2026|505|6576|Thursday|2026Q1|N|N|N|2461042|2461041|2460684|2460957|N|N|N|N|N| +2461050|AAAAAAAAKHNIFCAA|2026-01-09|1512|6576|505|2026|5|1|9|1|2026|505|6576|Friday|2026Q1|N|Y|N|2461042|2461041|2460685|2460958|N|N|N|N|N| +2461051|AAAAAAAALHNIFCAA|2026-01-10|1512|6576|505|2026|6|1|10|1|2026|505|6576|Saturday|2026Q1|N|Y|N|2461042|2461041|2460686|2460959|N|N|N|N|N| +2461052|AAAAAAAAMHNIFCAA|2026-01-11|1512|6576|505|2026|0|1|11|1|2026|505|6576|Sunday|2026Q1|N|N|N|2461042|2461041|2460687|2460960|N|N|N|N|N| +2461053|AAAAAAAANHNIFCAA|2026-01-12|1512|6576|505|2026|1|1|12|1|2026|505|6576|Monday|2026Q1|N|N|N|2461042|2461041|2460688|2460961|N|N|N|N|N| +2461054|AAAAAAAAOHNIFCAA|2026-01-13|1512|6577|505|2026|2|1|13|1|2026|505|6577|Tuesday|2026Q1|N|N|N|2461042|2461041|2460689|2460962|N|N|N|N|N| +2461055|AAAAAAAAPHNIFCAA|2026-01-14|1512|6577|505|2026|3|1|14|1|2026|505|6577|Wednesday|2026Q1|N|N|N|2461042|2461041|2460690|2460963|N|N|N|N|N| +2461056|AAAAAAAAAINIFCAA|2026-01-15|1512|6577|505|2026|4|1|15|1|2026|505|6577|Thursday|2026Q1|N|N|N|2461042|2461041|2460691|2460964|N|N|N|N|N| +2461057|AAAAAAAABINIFCAA|2026-01-16|1512|6577|505|2026|5|1|16|1|2026|505|6577|Friday|2026Q1|N|Y|N|2461042|2461041|2460692|2460965|N|N|N|N|N| +2461058|AAAAAAAACINIFCAA|2026-01-17|1512|6577|505|2026|6|1|17|1|2026|505|6577|Saturday|2026Q1|N|Y|N|2461042|2461041|2460693|2460966|N|N|N|N|N| +2461059|AAAAAAAADINIFCAA|2026-01-18|1512|6577|505|2026|0|1|18|1|2026|505|6577|Sunday|2026Q1|N|N|N|2461042|2461041|2460694|2460967|N|N|N|N|N| +2461060|AAAAAAAAEINIFCAA|2026-01-19|1512|6577|505|2026|1|1|19|1|2026|505|6577|Monday|2026Q1|N|N|N|2461042|2461041|2460695|2460968|N|N|N|N|N| +2461061|AAAAAAAAFINIFCAA|2026-01-20|1512|6578|505|2026|2|1|20|1|2026|505|6578|Tuesday|2026Q1|N|N|N|2461042|2461041|2460696|2460969|N|N|N|N|N| +2461062|AAAAAAAAGINIFCAA|2026-01-21|1512|6578|505|2026|3|1|21|1|2026|505|6578|Wednesday|2026Q1|N|N|N|2461042|2461041|2460697|2460970|N|N|N|N|N| +2461063|AAAAAAAAHINIFCAA|2026-01-22|1512|6578|505|2026|4|1|22|1|2026|505|6578|Thursday|2026Q1|N|N|N|2461042|2461041|2460698|2460971|N|N|N|N|N| +2461064|AAAAAAAAIINIFCAA|2026-01-23|1512|6578|505|2026|5|1|23|1|2026|505|6578|Friday|2026Q1|N|Y|N|2461042|2461041|2460699|2460972|N|N|N|N|N| +2461065|AAAAAAAAJINIFCAA|2026-01-24|1512|6578|505|2026|6|1|24|1|2026|505|6578|Saturday|2026Q1|N|Y|N|2461042|2461041|2460700|2460973|N|N|N|N|N| +2461066|AAAAAAAAKINIFCAA|2026-01-25|1512|6578|505|2026|0|1|25|1|2026|505|6578|Sunday|2026Q1|N|N|N|2461042|2461041|2460701|2460974|N|N|N|N|N| +2461067|AAAAAAAALINIFCAA|2026-01-26|1512|6578|505|2026|1|1|26|1|2026|505|6578|Monday|2026Q1|N|N|N|2461042|2461041|2460702|2460975|N|N|N|N|N| +2461068|AAAAAAAAMINIFCAA|2026-01-27|1512|6579|505|2026|2|1|27|1|2026|505|6579|Tuesday|2026Q1|N|N|N|2461042|2461041|2460703|2460976|N|N|N|N|N| +2461069|AAAAAAAANINIFCAA|2026-01-28|1512|6579|505|2026|3|1|28|1|2026|505|6579|Wednesday|2026Q1|N|N|N|2461042|2461041|2460704|2460977|N|N|N|N|N| +2461070|AAAAAAAAOINIFCAA|2026-01-29|1512|6579|505|2026|4|1|29|1|2026|505|6579|Thursday|2026Q1|N|N|N|2461042|2461041|2460705|2460978|N|N|N|N|N| +2461071|AAAAAAAAPINIFCAA|2026-01-30|1512|6579|505|2026|5|1|30|1|2026|505|6579|Friday|2026Q1|N|Y|N|2461042|2461041|2460706|2460979|N|N|N|N|N| +2461072|AAAAAAAAAJNIFCAA|2026-01-31|1512|6579|505|2026|6|1|31|1|2026|505|6579|Saturday|2026Q1|N|Y|N|2461042|2461041|2460707|2460980|N|N|N|N|N| +2461073|AAAAAAAABJNIFCAA|2026-02-01|1513|6579|505|2026|0|2|1|1|2026|505|6579|Sunday|2026Q1|N|N|N|2461073|2461103|2460708|2460981|N|N|N|N|N| +2461074|AAAAAAAACJNIFCAA|2026-02-02|1513|6579|505|2026|1|2|2|1|2026|505|6579|Monday|2026Q1|N|N|N|2461073|2461103|2460709|2460982|N|N|N|N|N| +2461075|AAAAAAAADJNIFCAA|2026-02-03|1513|6580|505|2026|2|2|3|1|2026|505|6580|Tuesday|2026Q1|N|N|N|2461073|2461103|2460710|2460983|N|N|N|N|N| +2461076|AAAAAAAAEJNIFCAA|2026-02-04|1513|6580|505|2026|3|2|4|1|2026|505|6580|Wednesday|2026Q1|N|N|N|2461073|2461103|2460711|2460984|N|N|N|N|N| +2461077|AAAAAAAAFJNIFCAA|2026-02-05|1513|6580|505|2026|4|2|5|1|2026|505|6580|Thursday|2026Q1|N|N|N|2461073|2461103|2460712|2460985|N|N|N|N|N| +2461078|AAAAAAAAGJNIFCAA|2026-02-06|1513|6580|505|2026|5|2|6|1|2026|505|6580|Friday|2026Q1|N|Y|N|2461073|2461103|2460713|2460986|N|N|N|N|N| +2461079|AAAAAAAAHJNIFCAA|2026-02-07|1513|6580|505|2026|6|2|7|1|2026|505|6580|Saturday|2026Q1|N|Y|N|2461073|2461103|2460714|2460987|N|N|N|N|N| +2461080|AAAAAAAAIJNIFCAA|2026-02-08|1513|6580|505|2026|0|2|8|1|2026|505|6580|Sunday|2026Q1|N|N|N|2461073|2461103|2460715|2460988|N|N|N|N|N| +2461081|AAAAAAAAJJNIFCAA|2026-02-09|1513|6580|505|2026|1|2|9|1|2026|505|6580|Monday|2026Q1|N|N|N|2461073|2461103|2460716|2460989|N|N|N|N|N| +2461082|AAAAAAAAKJNIFCAA|2026-02-10|1513|6581|505|2026|2|2|10|1|2026|505|6581|Tuesday|2026Q1|N|N|N|2461073|2461103|2460717|2460990|N|N|N|N|N| +2461083|AAAAAAAALJNIFCAA|2026-02-11|1513|6581|505|2026|3|2|11|1|2026|505|6581|Wednesday|2026Q1|N|N|N|2461073|2461103|2460718|2460991|N|N|N|N|N| +2461084|AAAAAAAAMJNIFCAA|2026-02-12|1513|6581|505|2026|4|2|12|1|2026|505|6581|Thursday|2026Q1|N|N|N|2461073|2461103|2460719|2460992|N|N|N|N|N| +2461085|AAAAAAAANJNIFCAA|2026-02-13|1513|6581|505|2026|5|2|13|1|2026|505|6581|Friday|2026Q1|N|Y|N|2461073|2461103|2460720|2460993|N|N|N|N|N| +2461086|AAAAAAAAOJNIFCAA|2026-02-14|1513|6581|505|2026|6|2|14|1|2026|505|6581|Saturday|2026Q1|N|Y|N|2461073|2461103|2460721|2460994|N|N|N|N|N| +2461087|AAAAAAAAPJNIFCAA|2026-02-15|1513|6581|505|2026|0|2|15|1|2026|505|6581|Sunday|2026Q1|N|N|N|2461073|2461103|2460722|2460995|N|N|N|N|N| +2461088|AAAAAAAAAKNIFCAA|2026-02-16|1513|6581|505|2026|1|2|16|1|2026|505|6581|Monday|2026Q1|N|N|N|2461073|2461103|2460723|2460996|N|N|N|N|N| +2461089|AAAAAAAABKNIFCAA|2026-02-17|1513|6582|505|2026|2|2|17|1|2026|505|6582|Tuesday|2026Q1|N|N|N|2461073|2461103|2460724|2460997|N|N|N|N|N| +2461090|AAAAAAAACKNIFCAA|2026-02-18|1513|6582|505|2026|3|2|18|1|2026|505|6582|Wednesday|2026Q1|N|N|N|2461073|2461103|2460725|2460998|N|N|N|N|N| +2461091|AAAAAAAADKNIFCAA|2026-02-19|1513|6582|505|2026|4|2|19|1|2026|505|6582|Thursday|2026Q1|N|N|N|2461073|2461103|2460726|2460999|N|N|N|N|N| +2461092|AAAAAAAAEKNIFCAA|2026-02-20|1513|6582|505|2026|5|2|20|1|2026|505|6582|Friday|2026Q1|N|Y|N|2461073|2461103|2460727|2461000|N|N|N|N|N| +2461093|AAAAAAAAFKNIFCAA|2026-02-21|1513|6582|505|2026|6|2|21|1|2026|505|6582|Saturday|2026Q1|N|Y|N|2461073|2461103|2460728|2461001|N|N|N|N|N| +2461094|AAAAAAAAGKNIFCAA|2026-02-22|1513|6582|505|2026|0|2|22|1|2026|505|6582|Sunday|2026Q1|N|N|N|2461073|2461103|2460729|2461002|N|N|N|N|N| +2461095|AAAAAAAAHKNIFCAA|2026-02-23|1513|6582|505|2026|1|2|23|1|2026|505|6582|Monday|2026Q1|N|N|N|2461073|2461103|2460730|2461003|N|N|N|N|N| +2461096|AAAAAAAAIKNIFCAA|2026-02-24|1513|6583|505|2026|2|2|24|1|2026|505|6583|Tuesday|2026Q1|N|N|N|2461073|2461103|2460731|2461004|N|N|N|N|N| +2461097|AAAAAAAAJKNIFCAA|2026-02-25|1513|6583|505|2026|3|2|25|1|2026|505|6583|Wednesday|2026Q1|N|N|N|2461073|2461103|2460732|2461005|N|N|N|N|N| +2461098|AAAAAAAAKKNIFCAA|2026-02-26|1513|6583|505|2026|4|2|26|1|2026|505|6583|Thursday|2026Q1|N|N|N|2461073|2461103|2460733|2461006|N|N|N|N|N| +2461099|AAAAAAAALKNIFCAA|2026-02-27|1513|6583|505|2026|5|2|27|1|2026|505|6583|Friday|2026Q1|N|Y|N|2461073|2461103|2460734|2461007|N|N|N|N|N| +2461100|AAAAAAAAMKNIFCAA|2026-02-28|1513|6583|505|2026|6|2|28|1|2026|505|6583|Saturday|2026Q1|N|Y|N|2461073|2461103|2460735|2461008|N|N|N|N|N| +2461101|AAAAAAAANKNIFCAA|2026-03-01|1514|6583|506|2026|0|3|1|1|2026|506|6583|Sunday|2026Q1|N|N|N|2461101|2461159|2460736|2461009|N|N|N|N|N| +2461102|AAAAAAAAOKNIFCAA|2026-03-02|1514|6583|506|2026|1|3|2|1|2026|506|6583|Monday|2026Q1|N|N|N|2461101|2461159|2460737|2461010|N|N|N|N|N| +2461103|AAAAAAAAPKNIFCAA|2026-03-03|1514|6584|506|2026|2|3|3|1|2026|506|6584|Tuesday|2026Q1|N|N|N|2461101|2461159|2460738|2461011|N|N|N|N|N| +2461104|AAAAAAAAALNIFCAA|2026-03-04|1514|6584|506|2026|3|3|4|1|2026|506|6584|Wednesday|2026Q1|N|N|N|2461101|2461159|2460739|2461012|N|N|N|N|N| +2461105|AAAAAAAABLNIFCAA|2026-03-05|1514|6584|506|2026|4|3|5|1|2026|506|6584|Thursday|2026Q1|N|N|N|2461101|2461159|2460740|2461013|N|N|N|N|N| +2461106|AAAAAAAACLNIFCAA|2026-03-06|1514|6584|506|2026|5|3|6|1|2026|506|6584|Friday|2026Q1|N|Y|N|2461101|2461159|2460741|2461014|N|N|N|N|N| +2461107|AAAAAAAADLNIFCAA|2026-03-07|1514|6584|506|2026|6|3|7|1|2026|506|6584|Saturday|2026Q1|N|Y|N|2461101|2461159|2460742|2461015|N|N|N|N|N| +2461108|AAAAAAAAELNIFCAA|2026-03-08|1514|6584|506|2026|0|3|8|1|2026|506|6584|Sunday|2026Q1|N|N|N|2461101|2461159|2460743|2461016|N|N|N|N|N| +2461109|AAAAAAAAFLNIFCAA|2026-03-09|1514|6584|506|2026|1|3|9|1|2026|506|6584|Monday|2026Q1|N|N|N|2461101|2461159|2460744|2461017|N|N|N|N|N| +2461110|AAAAAAAAGLNIFCAA|2026-03-10|1514|6585|506|2026|2|3|10|1|2026|506|6585|Tuesday|2026Q1|N|N|N|2461101|2461159|2460745|2461018|N|N|N|N|N| +2461111|AAAAAAAAHLNIFCAA|2026-03-11|1514|6585|506|2026|3|3|11|1|2026|506|6585|Wednesday|2026Q1|N|N|N|2461101|2461159|2460746|2461019|N|N|N|N|N| +2461112|AAAAAAAAILNIFCAA|2026-03-12|1514|6585|506|2026|4|3|12|1|2026|506|6585|Thursday|2026Q1|N|N|N|2461101|2461159|2460747|2461020|N|N|N|N|N| +2461113|AAAAAAAAJLNIFCAA|2026-03-13|1514|6585|506|2026|5|3|13|1|2026|506|6585|Friday|2026Q1|N|Y|N|2461101|2461159|2460748|2461021|N|N|N|N|N| +2461114|AAAAAAAAKLNIFCAA|2026-03-14|1514|6585|506|2026|6|3|14|1|2026|506|6585|Saturday|2026Q1|N|Y|N|2461101|2461159|2460749|2461022|N|N|N|N|N| +2461115|AAAAAAAALLNIFCAA|2026-03-15|1514|6585|506|2026|0|3|15|1|2026|506|6585|Sunday|2026Q1|N|N|N|2461101|2461159|2460750|2461023|N|N|N|N|N| +2461116|AAAAAAAAMLNIFCAA|2026-03-16|1514|6585|506|2026|1|3|16|1|2026|506|6585|Monday|2026Q1|N|N|N|2461101|2461159|2460751|2461024|N|N|N|N|N| +2461117|AAAAAAAANLNIFCAA|2026-03-17|1514|6586|506|2026|2|3|17|1|2026|506|6586|Tuesday|2026Q1|N|N|N|2461101|2461159|2460752|2461025|N|N|N|N|N| +2461118|AAAAAAAAOLNIFCAA|2026-03-18|1514|6586|506|2026|3|3|18|1|2026|506|6586|Wednesday|2026Q1|N|N|N|2461101|2461159|2460753|2461026|N|N|N|N|N| +2461119|AAAAAAAAPLNIFCAA|2026-03-19|1514|6586|506|2026|4|3|19|1|2026|506|6586|Thursday|2026Q1|N|N|N|2461101|2461159|2460754|2461027|N|N|N|N|N| +2461120|AAAAAAAAAMNIFCAA|2026-03-20|1514|6586|506|2026|5|3|20|1|2026|506|6586|Friday|2026Q1|N|Y|N|2461101|2461159|2460755|2461028|N|N|N|N|N| +2461121|AAAAAAAABMNIFCAA|2026-03-21|1514|6586|506|2026|6|3|21|1|2026|506|6586|Saturday|2026Q1|N|Y|N|2461101|2461159|2460756|2461029|N|N|N|N|N| +2461122|AAAAAAAACMNIFCAA|2026-03-22|1514|6586|506|2026|0|3|22|1|2026|506|6586|Sunday|2026Q1|N|N|N|2461101|2461159|2460757|2461030|N|N|N|N|N| +2461123|AAAAAAAADMNIFCAA|2026-03-23|1514|6586|506|2026|1|3|23|1|2026|506|6586|Monday|2026Q1|N|N|N|2461101|2461159|2460758|2461031|N|N|N|N|N| +2461124|AAAAAAAAEMNIFCAA|2026-03-24|1514|6587|506|2026|2|3|24|1|2026|506|6587|Tuesday|2026Q1|N|N|N|2461101|2461159|2460759|2461032|N|N|N|N|N| +2461125|AAAAAAAAFMNIFCAA|2026-03-25|1514|6587|506|2026|3|3|25|1|2026|506|6587|Wednesday|2026Q1|N|N|N|2461101|2461159|2460760|2461033|N|N|N|N|N| +2461126|AAAAAAAAGMNIFCAA|2026-03-26|1514|6587|506|2026|4|3|26|1|2026|506|6587|Thursday|2026Q1|N|N|N|2461101|2461159|2460761|2461034|N|N|N|N|N| +2461127|AAAAAAAAHMNIFCAA|2026-03-27|1514|6587|506|2026|5|3|27|1|2026|506|6587|Friday|2026Q1|N|Y|N|2461101|2461159|2460762|2461035|N|N|N|N|N| +2461128|AAAAAAAAIMNIFCAA|2026-03-28|1514|6587|506|2026|6|3|28|1|2026|506|6587|Saturday|2026Q1|N|Y|N|2461101|2461159|2460763|2461036|N|N|N|N|N| +2461129|AAAAAAAAJMNIFCAA|2026-03-29|1514|6587|506|2026|0|3|29|1|2026|506|6587|Sunday|2026Q1|N|N|N|2461101|2461159|2460764|2461037|N|N|N|N|N| +2461130|AAAAAAAAKMNIFCAA|2026-03-30|1514|6587|506|2026|1|3|30|1|2026|506|6587|Monday|2026Q1|N|N|N|2461101|2461159|2460765|2461038|N|N|N|N|N| +2461131|AAAAAAAALMNIFCAA|2026-03-31|1514|6588|506|2026|2|3|31|1|2026|506|6588|Tuesday|2026Q1|N|N|N|2461101|2461159|2460766|2461039|N|N|N|N|N| +2461132|AAAAAAAAMMNIFCAA|2026-04-01|1515|6588|506|2026|3|4|1|1|2026|506|6588|Wednesday|2026Q1|N|N|N|2461132|2461221|2460767|2461042|N|N|N|N|N| +2461133|AAAAAAAANMNIFCAA|2026-04-02|1515|6588|506|2026|4|4|2|2|2026|506|6588|Thursday|2026Q2|N|N|N|2461132|2461221|2460768|2461043|N|N|N|N|N| +2461134|AAAAAAAAOMNIFCAA|2026-04-03|1515|6588|506|2026|5|4|3|2|2026|506|6588|Friday|2026Q2|N|Y|N|2461132|2461221|2460769|2461044|N|N|N|N|N| +2461135|AAAAAAAAPMNIFCAA|2026-04-04|1515|6588|506|2026|6|4|4|2|2026|506|6588|Saturday|2026Q2|N|Y|N|2461132|2461221|2460770|2461045|N|N|N|N|N| +2461136|AAAAAAAAANNIFCAA|2026-04-05|1515|6588|506|2026|0|4|5|2|2026|506|6588|Sunday|2026Q2|N|N|N|2461132|2461221|2460771|2461046|N|N|N|N|N| +2461137|AAAAAAAABNNIFCAA|2026-04-06|1515|6588|506|2026|1|4|6|2|2026|506|6588|Monday|2026Q2|N|N|N|2461132|2461221|2460772|2461047|N|N|N|N|N| +2461138|AAAAAAAACNNIFCAA|2026-04-07|1515|6589|506|2026|2|4|7|2|2026|506|6589|Tuesday|2026Q2|N|N|N|2461132|2461221|2460773|2461048|N|N|N|N|N| +2461139|AAAAAAAADNNIFCAA|2026-04-08|1515|6589|506|2026|3|4|8|2|2026|506|6589|Wednesday|2026Q2|N|N|N|2461132|2461221|2460774|2461049|N|N|N|N|N| +2461140|AAAAAAAAENNIFCAA|2026-04-09|1515|6589|506|2026|4|4|9|2|2026|506|6589|Thursday|2026Q2|N|N|N|2461132|2461221|2460775|2461050|N|N|N|N|N| +2461141|AAAAAAAAFNNIFCAA|2026-04-10|1515|6589|506|2026|5|4|10|2|2026|506|6589|Friday|2026Q2|N|Y|N|2461132|2461221|2460776|2461051|N|N|N|N|N| +2461142|AAAAAAAAGNNIFCAA|2026-04-11|1515|6589|506|2026|6|4|11|2|2026|506|6589|Saturday|2026Q2|N|Y|N|2461132|2461221|2460777|2461052|N|N|N|N|N| +2461143|AAAAAAAAHNNIFCAA|2026-04-12|1515|6589|506|2026|0|4|12|2|2026|506|6589|Sunday|2026Q2|N|N|N|2461132|2461221|2460778|2461053|N|N|N|N|N| +2461144|AAAAAAAAINNIFCAA|2026-04-13|1515|6589|506|2026|1|4|13|2|2026|506|6589|Monday|2026Q2|N|N|N|2461132|2461221|2460779|2461054|N|N|N|N|N| +2461145|AAAAAAAAJNNIFCAA|2026-04-14|1515|6590|506|2026|2|4|14|2|2026|506|6590|Tuesday|2026Q2|N|N|N|2461132|2461221|2460780|2461055|N|N|N|N|N| +2461146|AAAAAAAAKNNIFCAA|2026-04-15|1515|6590|506|2026|3|4|15|2|2026|506|6590|Wednesday|2026Q2|N|N|N|2461132|2461221|2460781|2461056|N|N|N|N|N| +2461147|AAAAAAAALNNIFCAA|2026-04-16|1515|6590|506|2026|4|4|16|2|2026|506|6590|Thursday|2026Q2|N|N|N|2461132|2461221|2460782|2461057|N|N|N|N|N| +2461148|AAAAAAAAMNNIFCAA|2026-04-17|1515|6590|506|2026|5|4|17|2|2026|506|6590|Friday|2026Q2|N|Y|N|2461132|2461221|2460783|2461058|N|N|N|N|N| +2461149|AAAAAAAANNNIFCAA|2026-04-18|1515|6590|506|2026|6|4|18|2|2026|506|6590|Saturday|2026Q2|N|Y|N|2461132|2461221|2460784|2461059|N|N|N|N|N| +2461150|AAAAAAAAONNIFCAA|2026-04-19|1515|6590|506|2026|0|4|19|2|2026|506|6590|Sunday|2026Q2|N|N|N|2461132|2461221|2460785|2461060|N|N|N|N|N| +2461151|AAAAAAAAPNNIFCAA|2026-04-20|1515|6590|506|2026|1|4|20|2|2026|506|6590|Monday|2026Q2|N|N|N|2461132|2461221|2460786|2461061|N|N|N|N|N| +2461152|AAAAAAAAAONIFCAA|2026-04-21|1515|6591|506|2026|2|4|21|2|2026|506|6591|Tuesday|2026Q2|N|N|N|2461132|2461221|2460787|2461062|N|N|N|N|N| +2461153|AAAAAAAABONIFCAA|2026-04-22|1515|6591|506|2026|3|4|22|2|2026|506|6591|Wednesday|2026Q2|N|N|N|2461132|2461221|2460788|2461063|N|N|N|N|N| +2461154|AAAAAAAACONIFCAA|2026-04-23|1515|6591|506|2026|4|4|23|2|2026|506|6591|Thursday|2026Q2|N|N|N|2461132|2461221|2460789|2461064|N|N|N|N|N| +2461155|AAAAAAAADONIFCAA|2026-04-24|1515|6591|506|2026|5|4|24|2|2026|506|6591|Friday|2026Q2|N|Y|N|2461132|2461221|2460790|2461065|N|N|N|N|N| +2461156|AAAAAAAAEONIFCAA|2026-04-25|1515|6591|506|2026|6|4|25|2|2026|506|6591|Saturday|2026Q2|N|Y|N|2461132|2461221|2460791|2461066|N|N|N|N|N| +2461157|AAAAAAAAFONIFCAA|2026-04-26|1515|6591|506|2026|0|4|26|2|2026|506|6591|Sunday|2026Q2|N|N|N|2461132|2461221|2460792|2461067|N|N|N|N|N| +2461158|AAAAAAAAGONIFCAA|2026-04-27|1515|6591|506|2026|1|4|27|2|2026|506|6591|Monday|2026Q2|N|N|N|2461132|2461221|2460793|2461068|N|N|N|N|N| +2461159|AAAAAAAAHONIFCAA|2026-04-28|1515|6592|506|2026|2|4|28|2|2026|506|6592|Tuesday|2026Q2|N|N|N|2461132|2461221|2460794|2461069|N|N|N|N|N| +2461160|AAAAAAAAIONIFCAA|2026-04-29|1515|6592|506|2026|3|4|29|2|2026|506|6592|Wednesday|2026Q2|N|N|N|2461132|2461221|2460795|2461070|N|N|N|N|N| +2461161|AAAAAAAAJONIFCAA|2026-04-30|1515|6592|506|2026|4|4|30|2|2026|506|6592|Thursday|2026Q2|N|N|N|2461132|2461221|2460796|2461071|N|N|N|N|N| +2461162|AAAAAAAAKONIFCAA|2026-05-01|1516|6592|506|2026|5|5|1|2|2026|506|6592|Friday|2026Q2|N|Y|N|2461162|2461281|2460797|2461072|N|N|N|N|N| +2461163|AAAAAAAALONIFCAA|2026-05-02|1516|6592|506|2026|6|5|2|2|2026|506|6592|Saturday|2026Q2|N|Y|N|2461162|2461281|2460798|2461073|N|N|N|N|N| +2461164|AAAAAAAAMONIFCAA|2026-05-03|1516|6592|506|2026|0|5|3|2|2026|506|6592|Sunday|2026Q2|N|N|N|2461162|2461281|2460799|2461074|N|N|N|N|N| +2461165|AAAAAAAANONIFCAA|2026-05-04|1516|6592|506|2026|1|5|4|2|2026|506|6592|Monday|2026Q2|N|N|N|2461162|2461281|2460800|2461075|N|N|N|N|N| +2461166|AAAAAAAAOONIFCAA|2026-05-05|1516|6593|506|2026|2|5|5|2|2026|506|6593|Tuesday|2026Q2|N|N|N|2461162|2461281|2460801|2461076|N|N|N|N|N| +2461167|AAAAAAAAPONIFCAA|2026-05-06|1516|6593|506|2026|3|5|6|2|2026|506|6593|Wednesday|2026Q2|N|N|N|2461162|2461281|2460802|2461077|N|N|N|N|N| +2461168|AAAAAAAAAPNIFCAA|2026-05-07|1516|6593|506|2026|4|5|7|2|2026|506|6593|Thursday|2026Q2|N|N|N|2461162|2461281|2460803|2461078|N|N|N|N|N| +2461169|AAAAAAAABPNIFCAA|2026-05-08|1516|6593|506|2026|5|5|8|2|2026|506|6593|Friday|2026Q2|N|Y|N|2461162|2461281|2460804|2461079|N|N|N|N|N| +2461170|AAAAAAAACPNIFCAA|2026-05-09|1516|6593|506|2026|6|5|9|2|2026|506|6593|Saturday|2026Q2|N|Y|N|2461162|2461281|2460805|2461080|N|N|N|N|N| +2461171|AAAAAAAADPNIFCAA|2026-05-10|1516|6593|506|2026|0|5|10|2|2026|506|6593|Sunday|2026Q2|N|N|N|2461162|2461281|2460806|2461081|N|N|N|N|N| +2461172|AAAAAAAAEPNIFCAA|2026-05-11|1516|6593|506|2026|1|5|11|2|2026|506|6593|Monday|2026Q2|N|N|N|2461162|2461281|2460807|2461082|N|N|N|N|N| +2461173|AAAAAAAAFPNIFCAA|2026-05-12|1516|6594|506|2026|2|5|12|2|2026|506|6594|Tuesday|2026Q2|N|N|N|2461162|2461281|2460808|2461083|N|N|N|N|N| +2461174|AAAAAAAAGPNIFCAA|2026-05-13|1516|6594|506|2026|3|5|13|2|2026|506|6594|Wednesday|2026Q2|N|N|N|2461162|2461281|2460809|2461084|N|N|N|N|N| +2461175|AAAAAAAAHPNIFCAA|2026-05-14|1516|6594|506|2026|4|5|14|2|2026|506|6594|Thursday|2026Q2|N|N|N|2461162|2461281|2460810|2461085|N|N|N|N|N| +2461176|AAAAAAAAIPNIFCAA|2026-05-15|1516|6594|506|2026|5|5|15|2|2026|506|6594|Friday|2026Q2|N|Y|N|2461162|2461281|2460811|2461086|N|N|N|N|N| +2461177|AAAAAAAAJPNIFCAA|2026-05-16|1516|6594|506|2026|6|5|16|2|2026|506|6594|Saturday|2026Q2|N|Y|N|2461162|2461281|2460812|2461087|N|N|N|N|N| +2461178|AAAAAAAAKPNIFCAA|2026-05-17|1516|6594|506|2026|0|5|17|2|2026|506|6594|Sunday|2026Q2|N|N|N|2461162|2461281|2460813|2461088|N|N|N|N|N| +2461179|AAAAAAAALPNIFCAA|2026-05-18|1516|6594|506|2026|1|5|18|2|2026|506|6594|Monday|2026Q2|N|N|N|2461162|2461281|2460814|2461089|N|N|N|N|N| +2461180|AAAAAAAAMPNIFCAA|2026-05-19|1516|6595|506|2026|2|5|19|2|2026|506|6595|Tuesday|2026Q2|N|N|N|2461162|2461281|2460815|2461090|N|N|N|N|N| +2461181|AAAAAAAANPNIFCAA|2026-05-20|1516|6595|506|2026|3|5|20|2|2026|506|6595|Wednesday|2026Q2|N|N|N|2461162|2461281|2460816|2461091|N|N|N|N|N| +2461182|AAAAAAAAOPNIFCAA|2026-05-21|1516|6595|506|2026|4|5|21|2|2026|506|6595|Thursday|2026Q2|N|N|N|2461162|2461281|2460817|2461092|N|N|N|N|N| +2461183|AAAAAAAAPPNIFCAA|2026-05-22|1516|6595|506|2026|5|5|22|2|2026|506|6595|Friday|2026Q2|N|Y|N|2461162|2461281|2460818|2461093|N|N|N|N|N| +2461184|AAAAAAAAAAOIFCAA|2026-05-23|1516|6595|506|2026|6|5|23|2|2026|506|6595|Saturday|2026Q2|N|Y|N|2461162|2461281|2460819|2461094|N|N|N|N|N| +2461185|AAAAAAAABAOIFCAA|2026-05-24|1516|6595|506|2026|0|5|24|2|2026|506|6595|Sunday|2026Q2|N|N|N|2461162|2461281|2460820|2461095|N|N|N|N|N| +2461186|AAAAAAAACAOIFCAA|2026-05-25|1516|6595|506|2026|1|5|25|2|2026|506|6595|Monday|2026Q2|N|N|N|2461162|2461281|2460821|2461096|N|N|N|N|N| +2461187|AAAAAAAADAOIFCAA|2026-05-26|1516|6596|506|2026|2|5|26|2|2026|506|6596|Tuesday|2026Q2|N|N|N|2461162|2461281|2460822|2461097|N|N|N|N|N| +2461188|AAAAAAAAEAOIFCAA|2026-05-27|1516|6596|506|2026|3|5|27|2|2026|506|6596|Wednesday|2026Q2|N|N|N|2461162|2461281|2460823|2461098|N|N|N|N|N| +2461189|AAAAAAAAFAOIFCAA|2026-05-28|1516|6596|506|2026|4|5|28|2|2026|506|6596|Thursday|2026Q2|N|N|N|2461162|2461281|2460824|2461099|N|N|N|N|N| +2461190|AAAAAAAAGAOIFCAA|2026-05-29|1516|6596|506|2026|5|5|29|2|2026|506|6596|Friday|2026Q2|N|Y|N|2461162|2461281|2460825|2461100|N|N|N|N|N| +2461191|AAAAAAAAHAOIFCAA|2026-05-30|1516|6596|506|2026|6|5|30|2|2026|506|6596|Saturday|2026Q2|N|Y|N|2461162|2461281|2460826|2461101|N|N|N|N|N| +2461192|AAAAAAAAIAOIFCAA|2026-05-31|1516|6596|506|2026|0|5|31|2|2026|506|6596|Sunday|2026Q2|N|N|N|2461162|2461281|2460827|2461102|N|N|N|N|N| +2461193|AAAAAAAAJAOIFCAA|2026-06-01|1517|6596|507|2026|1|6|1|2|2026|507|6596|Monday|2026Q2|N|N|N|2461193|2461343|2460828|2461103|N|N|N|N|N| +2461194|AAAAAAAAKAOIFCAA|2026-06-02|1517|6597|507|2026|2|6|2|2|2026|507|6597|Tuesday|2026Q2|N|N|N|2461193|2461343|2460829|2461104|N|N|N|N|N| +2461195|AAAAAAAALAOIFCAA|2026-06-03|1517|6597|507|2026|3|6|3|2|2026|507|6597|Wednesday|2026Q2|N|N|N|2461193|2461343|2460830|2461105|N|N|N|N|N| +2461196|AAAAAAAAMAOIFCAA|2026-06-04|1517|6597|507|2026|4|6|4|2|2026|507|6597|Thursday|2026Q2|N|N|N|2461193|2461343|2460831|2461106|N|N|N|N|N| +2461197|AAAAAAAANAOIFCAA|2026-06-05|1517|6597|507|2026|5|6|5|2|2026|507|6597|Friday|2026Q2|N|Y|N|2461193|2461343|2460832|2461107|N|N|N|N|N| +2461198|AAAAAAAAOAOIFCAA|2026-06-06|1517|6597|507|2026|6|6|6|2|2026|507|6597|Saturday|2026Q2|N|Y|N|2461193|2461343|2460833|2461108|N|N|N|N|N| +2461199|AAAAAAAAPAOIFCAA|2026-06-07|1517|6597|507|2026|0|6|7|2|2026|507|6597|Sunday|2026Q2|N|N|N|2461193|2461343|2460834|2461109|N|N|N|N|N| +2461200|AAAAAAAAABOIFCAA|2026-06-08|1517|6597|507|2026|1|6|8|2|2026|507|6597|Monday|2026Q2|N|N|N|2461193|2461343|2460835|2461110|N|N|N|N|N| +2461201|AAAAAAAABBOIFCAA|2026-06-09|1517|6598|507|2026|2|6|9|2|2026|507|6598|Tuesday|2026Q2|N|N|N|2461193|2461343|2460836|2461111|N|N|N|N|N| +2461202|AAAAAAAACBOIFCAA|2026-06-10|1517|6598|507|2026|3|6|10|2|2026|507|6598|Wednesday|2026Q2|N|N|N|2461193|2461343|2460837|2461112|N|N|N|N|N| +2461203|AAAAAAAADBOIFCAA|2026-06-11|1517|6598|507|2026|4|6|11|2|2026|507|6598|Thursday|2026Q2|N|N|N|2461193|2461343|2460838|2461113|N|N|N|N|N| +2461204|AAAAAAAAEBOIFCAA|2026-06-12|1517|6598|507|2026|5|6|12|2|2026|507|6598|Friday|2026Q2|N|Y|N|2461193|2461343|2460839|2461114|N|N|N|N|N| +2461205|AAAAAAAAFBOIFCAA|2026-06-13|1517|6598|507|2026|6|6|13|2|2026|507|6598|Saturday|2026Q2|N|Y|N|2461193|2461343|2460840|2461115|N|N|N|N|N| +2461206|AAAAAAAAGBOIFCAA|2026-06-14|1517|6598|507|2026|0|6|14|2|2026|507|6598|Sunday|2026Q2|N|N|N|2461193|2461343|2460841|2461116|N|N|N|N|N| +2461207|AAAAAAAAHBOIFCAA|2026-06-15|1517|6598|507|2026|1|6|15|2|2026|507|6598|Monday|2026Q2|N|N|N|2461193|2461343|2460842|2461117|N|N|N|N|N| +2461208|AAAAAAAAIBOIFCAA|2026-06-16|1517|6599|507|2026|2|6|16|2|2026|507|6599|Tuesday|2026Q2|N|N|N|2461193|2461343|2460843|2461118|N|N|N|N|N| +2461209|AAAAAAAAJBOIFCAA|2026-06-17|1517|6599|507|2026|3|6|17|2|2026|507|6599|Wednesday|2026Q2|N|N|N|2461193|2461343|2460844|2461119|N|N|N|N|N| +2461210|AAAAAAAAKBOIFCAA|2026-06-18|1517|6599|507|2026|4|6|18|2|2026|507|6599|Thursday|2026Q2|N|N|N|2461193|2461343|2460845|2461120|N|N|N|N|N| +2461211|AAAAAAAALBOIFCAA|2026-06-19|1517|6599|507|2026|5|6|19|2|2026|507|6599|Friday|2026Q2|N|Y|N|2461193|2461343|2460846|2461121|N|N|N|N|N| +2461212|AAAAAAAAMBOIFCAA|2026-06-20|1517|6599|507|2026|6|6|20|2|2026|507|6599|Saturday|2026Q2|N|Y|N|2461193|2461343|2460847|2461122|N|N|N|N|N| +2461213|AAAAAAAANBOIFCAA|2026-06-21|1517|6599|507|2026|0|6|21|2|2026|507|6599|Sunday|2026Q2|N|N|N|2461193|2461343|2460848|2461123|N|N|N|N|N| +2461214|AAAAAAAAOBOIFCAA|2026-06-22|1517|6599|507|2026|1|6|22|2|2026|507|6599|Monday|2026Q2|N|N|N|2461193|2461343|2460849|2461124|N|N|N|N|N| +2461215|AAAAAAAAPBOIFCAA|2026-06-23|1517|6600|507|2026|2|6|23|2|2026|507|6600|Tuesday|2026Q2|N|N|N|2461193|2461343|2460850|2461125|N|N|N|N|N| +2461216|AAAAAAAAACOIFCAA|2026-06-24|1517|6600|507|2026|3|6|24|2|2026|507|6600|Wednesday|2026Q2|N|N|N|2461193|2461343|2460851|2461126|N|N|N|N|N| +2461217|AAAAAAAABCOIFCAA|2026-06-25|1517|6600|507|2026|4|6|25|2|2026|507|6600|Thursday|2026Q2|N|N|N|2461193|2461343|2460852|2461127|N|N|N|N|N| +2461218|AAAAAAAACCOIFCAA|2026-06-26|1517|6600|507|2026|5|6|26|2|2026|507|6600|Friday|2026Q2|N|Y|N|2461193|2461343|2460853|2461128|N|N|N|N|N| +2461219|AAAAAAAADCOIFCAA|2026-06-27|1517|6600|507|2026|6|6|27|2|2026|507|6600|Saturday|2026Q2|N|Y|N|2461193|2461343|2460854|2461129|N|N|N|N|N| +2461220|AAAAAAAAECOIFCAA|2026-06-28|1517|6600|507|2026|0|6|28|2|2026|507|6600|Sunday|2026Q2|N|N|N|2461193|2461343|2460855|2461130|N|N|N|N|N| +2461221|AAAAAAAAFCOIFCAA|2026-06-29|1517|6600|507|2026|1|6|29|2|2026|507|6600|Monday|2026Q2|N|N|N|2461193|2461343|2460856|2461131|N|N|N|N|N| +2461222|AAAAAAAAGCOIFCAA|2026-06-30|1517|6601|507|2026|2|6|30|2|2026|507|6601|Tuesday|2026Q2|N|N|N|2461193|2461343|2460857|2461132|N|N|N|N|N| +2461223|AAAAAAAAHCOIFCAA|2026-07-01|1518|6601|507|2026|3|7|1|2|2026|507|6601|Wednesday|2026Q2|N|N|N|2461223|2461403|2460858|2461132|N|N|N|N|N| +2461224|AAAAAAAAICOIFCAA|2026-07-02|1518|6601|507|2026|4|7|2|3|2026|507|6601|Thursday|2026Q3|N|N|N|2461223|2461403|2460859|2461133|N|N|N|N|N| +2461225|AAAAAAAAJCOIFCAA|2026-07-03|1518|6601|507|2026|5|7|3|3|2026|507|6601|Friday|2026Q3|N|Y|N|2461223|2461403|2460860|2461134|N|N|N|N|N| +2461226|AAAAAAAAKCOIFCAA|2026-07-04|1518|6601|507|2026|6|7|4|3|2026|507|6601|Saturday|2026Q3|N|Y|N|2461223|2461403|2460861|2461135|N|N|N|N|N| +2461227|AAAAAAAALCOIFCAA|2026-07-05|1518|6601|507|2026|0|7|5|3|2026|507|6601|Sunday|2026Q3|Y|N|N|2461223|2461403|2460862|2461136|N|N|N|N|N| +2461228|AAAAAAAAMCOIFCAA|2026-07-06|1518|6601|507|2026|1|7|6|3|2026|507|6601|Monday|2026Q3|N|N|Y|2461223|2461403|2460863|2461137|N|N|N|N|N| +2461229|AAAAAAAANCOIFCAA|2026-07-07|1518|6602|507|2026|2|7|7|3|2026|507|6602|Tuesday|2026Q3|N|N|N|2461223|2461403|2460864|2461138|N|N|N|N|N| +2461230|AAAAAAAAOCOIFCAA|2026-07-08|1518|6602|507|2026|3|7|8|3|2026|507|6602|Wednesday|2026Q3|N|N|N|2461223|2461403|2460865|2461139|N|N|N|N|N| +2461231|AAAAAAAAPCOIFCAA|2026-07-09|1518|6602|507|2026|4|7|9|3|2026|507|6602|Thursday|2026Q3|N|N|N|2461223|2461403|2460866|2461140|N|N|N|N|N| +2461232|AAAAAAAAADOIFCAA|2026-07-10|1518|6602|507|2026|5|7|10|3|2026|507|6602|Friday|2026Q3|N|Y|N|2461223|2461403|2460867|2461141|N|N|N|N|N| +2461233|AAAAAAAABDOIFCAA|2026-07-11|1518|6602|507|2026|6|7|11|3|2026|507|6602|Saturday|2026Q3|N|Y|N|2461223|2461403|2460868|2461142|N|N|N|N|N| +2461234|AAAAAAAACDOIFCAA|2026-07-12|1518|6602|507|2026|0|7|12|3|2026|507|6602|Sunday|2026Q3|N|N|N|2461223|2461403|2460869|2461143|N|N|N|N|N| +2461235|AAAAAAAADDOIFCAA|2026-07-13|1518|6602|507|2026|1|7|13|3|2026|507|6602|Monday|2026Q3|N|N|N|2461223|2461403|2460870|2461144|N|N|N|N|N| +2461236|AAAAAAAAEDOIFCAA|2026-07-14|1518|6603|507|2026|2|7|14|3|2026|507|6603|Tuesday|2026Q3|N|N|N|2461223|2461403|2460871|2461145|N|N|N|N|N| +2461237|AAAAAAAAFDOIFCAA|2026-07-15|1518|6603|507|2026|3|7|15|3|2026|507|6603|Wednesday|2026Q3|N|N|N|2461223|2461403|2460872|2461146|N|N|N|N|N| +2461238|AAAAAAAAGDOIFCAA|2026-07-16|1518|6603|507|2026|4|7|16|3|2026|507|6603|Thursday|2026Q3|N|N|N|2461223|2461403|2460873|2461147|N|N|N|N|N| +2461239|AAAAAAAAHDOIFCAA|2026-07-17|1518|6603|507|2026|5|7|17|3|2026|507|6603|Friday|2026Q3|N|Y|N|2461223|2461403|2460874|2461148|N|N|N|N|N| +2461240|AAAAAAAAIDOIFCAA|2026-07-18|1518|6603|507|2026|6|7|18|3|2026|507|6603|Saturday|2026Q3|N|Y|N|2461223|2461403|2460875|2461149|N|N|N|N|N| +2461241|AAAAAAAAJDOIFCAA|2026-07-19|1518|6603|507|2026|0|7|19|3|2026|507|6603|Sunday|2026Q3|N|N|N|2461223|2461403|2460876|2461150|N|N|N|N|N| +2461242|AAAAAAAAKDOIFCAA|2026-07-20|1518|6603|507|2026|1|7|20|3|2026|507|6603|Monday|2026Q3|N|N|N|2461223|2461403|2460877|2461151|N|N|N|N|N| +2461243|AAAAAAAALDOIFCAA|2026-07-21|1518|6604|507|2026|2|7|21|3|2026|507|6604|Tuesday|2026Q3|N|N|N|2461223|2461403|2460878|2461152|N|N|N|N|N| +2461244|AAAAAAAAMDOIFCAA|2026-07-22|1518|6604|507|2026|3|7|22|3|2026|507|6604|Wednesday|2026Q3|N|N|N|2461223|2461403|2460879|2461153|N|N|N|N|N| +2461245|AAAAAAAANDOIFCAA|2026-07-23|1518|6604|507|2026|4|7|23|3|2026|507|6604|Thursday|2026Q3|N|N|N|2461223|2461403|2460880|2461154|N|N|N|N|N| +2461246|AAAAAAAAODOIFCAA|2026-07-24|1518|6604|507|2026|5|7|24|3|2026|507|6604|Friday|2026Q3|N|Y|N|2461223|2461403|2460881|2461155|N|N|N|N|N| +2461247|AAAAAAAAPDOIFCAA|2026-07-25|1518|6604|507|2026|6|7|25|3|2026|507|6604|Saturday|2026Q3|N|Y|N|2461223|2461403|2460882|2461156|N|N|N|N|N| +2461248|AAAAAAAAAEOIFCAA|2026-07-26|1518|6604|507|2026|0|7|26|3|2026|507|6604|Sunday|2026Q3|N|N|N|2461223|2461403|2460883|2461157|N|N|N|N|N| +2461249|AAAAAAAABEOIFCAA|2026-07-27|1518|6604|507|2026|1|7|27|3|2026|507|6604|Monday|2026Q3|N|N|N|2461223|2461403|2460884|2461158|N|N|N|N|N| +2461250|AAAAAAAACEOIFCAA|2026-07-28|1518|6605|507|2026|2|7|28|3|2026|507|6605|Tuesday|2026Q3|N|N|N|2461223|2461403|2460885|2461159|N|N|N|N|N| +2461251|AAAAAAAADEOIFCAA|2026-07-29|1518|6605|507|2026|3|7|29|3|2026|507|6605|Wednesday|2026Q3|N|N|N|2461223|2461403|2460886|2461160|N|N|N|N|N| +2461252|AAAAAAAAEEOIFCAA|2026-07-30|1518|6605|507|2026|4|7|30|3|2026|507|6605|Thursday|2026Q3|N|N|N|2461223|2461403|2460887|2461161|N|N|N|N|N| +2461253|AAAAAAAAFEOIFCAA|2026-07-31|1518|6605|507|2026|5|7|31|3|2026|507|6605|Friday|2026Q3|N|Y|N|2461223|2461403|2460888|2461162|N|N|N|N|N| +2461254|AAAAAAAAGEOIFCAA|2026-08-01|1519|6605|507|2026|6|8|1|3|2026|507|6605|Saturday|2026Q3|N|Y|N|2461254|2461465|2460889|2461163|N|N|N|N|N| +2461255|AAAAAAAAHEOIFCAA|2026-08-02|1519|6605|507|2026|0|8|2|3|2026|507|6605|Sunday|2026Q3|N|N|N|2461254|2461465|2460890|2461164|N|N|N|N|N| +2461256|AAAAAAAAIEOIFCAA|2026-08-03|1519|6605|507|2026|1|8|3|3|2026|507|6605|Monday|2026Q3|N|N|N|2461254|2461465|2460891|2461165|N|N|N|N|N| +2461257|AAAAAAAAJEOIFCAA|2026-08-04|1519|6606|507|2026|2|8|4|3|2026|507|6606|Tuesday|2026Q3|N|N|N|2461254|2461465|2460892|2461166|N|N|N|N|N| +2461258|AAAAAAAAKEOIFCAA|2026-08-05|1519|6606|507|2026|3|8|5|3|2026|507|6606|Wednesday|2026Q3|N|N|N|2461254|2461465|2460893|2461167|N|N|N|N|N| +2461259|AAAAAAAALEOIFCAA|2026-08-06|1519|6606|507|2026|4|8|6|3|2026|507|6606|Thursday|2026Q3|N|N|N|2461254|2461465|2460894|2461168|N|N|N|N|N| +2461260|AAAAAAAAMEOIFCAA|2026-08-07|1519|6606|507|2026|5|8|7|3|2026|507|6606|Friday|2026Q3|N|Y|N|2461254|2461465|2460895|2461169|N|N|N|N|N| +2461261|AAAAAAAANEOIFCAA|2026-08-08|1519|6606|507|2026|6|8|8|3|2026|507|6606|Saturday|2026Q3|N|Y|N|2461254|2461465|2460896|2461170|N|N|N|N|N| +2461262|AAAAAAAAOEOIFCAA|2026-08-09|1519|6606|507|2026|0|8|9|3|2026|507|6606|Sunday|2026Q3|N|N|N|2461254|2461465|2460897|2461171|N|N|N|N|N| +2461263|AAAAAAAAPEOIFCAA|2026-08-10|1519|6606|507|2026|1|8|10|3|2026|507|6606|Monday|2026Q3|N|N|N|2461254|2461465|2460898|2461172|N|N|N|N|N| +2461264|AAAAAAAAAFOIFCAA|2026-08-11|1519|6607|507|2026|2|8|11|3|2026|507|6607|Tuesday|2026Q3|N|N|N|2461254|2461465|2460899|2461173|N|N|N|N|N| +2461265|AAAAAAAABFOIFCAA|2026-08-12|1519|6607|507|2026|3|8|12|3|2026|507|6607|Wednesday|2026Q3|N|N|N|2461254|2461465|2460900|2461174|N|N|N|N|N| +2461266|AAAAAAAACFOIFCAA|2026-08-13|1519|6607|507|2026|4|8|13|3|2026|507|6607|Thursday|2026Q3|N|N|N|2461254|2461465|2460901|2461175|N|N|N|N|N| +2461267|AAAAAAAADFOIFCAA|2026-08-14|1519|6607|507|2026|5|8|14|3|2026|507|6607|Friday|2026Q3|N|Y|N|2461254|2461465|2460902|2461176|N|N|N|N|N| +2461268|AAAAAAAAEFOIFCAA|2026-08-15|1519|6607|507|2026|6|8|15|3|2026|507|6607|Saturday|2026Q3|N|Y|N|2461254|2461465|2460903|2461177|N|N|N|N|N| +2461269|AAAAAAAAFFOIFCAA|2026-08-16|1519|6607|507|2026|0|8|16|3|2026|507|6607|Sunday|2026Q3|N|N|N|2461254|2461465|2460904|2461178|N|N|N|N|N| +2461270|AAAAAAAAGFOIFCAA|2026-08-17|1519|6607|507|2026|1|8|17|3|2026|507|6607|Monday|2026Q3|N|N|N|2461254|2461465|2460905|2461179|N|N|N|N|N| +2461271|AAAAAAAAHFOIFCAA|2026-08-18|1519|6608|507|2026|2|8|18|3|2026|507|6608|Tuesday|2026Q3|N|N|N|2461254|2461465|2460906|2461180|N|N|N|N|N| +2461272|AAAAAAAAIFOIFCAA|2026-08-19|1519|6608|507|2026|3|8|19|3|2026|507|6608|Wednesday|2026Q3|N|N|N|2461254|2461465|2460907|2461181|N|N|N|N|N| +2461273|AAAAAAAAJFOIFCAA|2026-08-20|1519|6608|507|2026|4|8|20|3|2026|507|6608|Thursday|2026Q3|N|N|N|2461254|2461465|2460908|2461182|N|N|N|N|N| +2461274|AAAAAAAAKFOIFCAA|2026-08-21|1519|6608|507|2026|5|8|21|3|2026|507|6608|Friday|2026Q3|N|Y|N|2461254|2461465|2460909|2461183|N|N|N|N|N| +2461275|AAAAAAAALFOIFCAA|2026-08-22|1519|6608|507|2026|6|8|22|3|2026|507|6608|Saturday|2026Q3|N|Y|N|2461254|2461465|2460910|2461184|N|N|N|N|N| +2461276|AAAAAAAAMFOIFCAA|2026-08-23|1519|6608|507|2026|0|8|23|3|2026|507|6608|Sunday|2026Q3|N|N|N|2461254|2461465|2460911|2461185|N|N|N|N|N| +2461277|AAAAAAAANFOIFCAA|2026-08-24|1519|6608|507|2026|1|8|24|3|2026|507|6608|Monday|2026Q3|N|N|N|2461254|2461465|2460912|2461186|N|N|N|N|N| +2461278|AAAAAAAAOFOIFCAA|2026-08-25|1519|6609|507|2026|2|8|25|3|2026|507|6609|Tuesday|2026Q3|N|N|N|2461254|2461465|2460913|2461187|N|N|N|N|N| +2461279|AAAAAAAAPFOIFCAA|2026-08-26|1519|6609|507|2026|3|8|26|3|2026|507|6609|Wednesday|2026Q3|N|N|N|2461254|2461465|2460914|2461188|N|N|N|N|N| +2461280|AAAAAAAAAGOIFCAA|2026-08-27|1519|6609|507|2026|4|8|27|3|2026|507|6609|Thursday|2026Q3|N|N|N|2461254|2461465|2460915|2461189|N|N|N|N|N| +2461281|AAAAAAAABGOIFCAA|2026-08-28|1519|6609|507|2026|5|8|28|3|2026|507|6609|Friday|2026Q3|N|Y|N|2461254|2461465|2460916|2461190|N|N|N|N|N| +2461282|AAAAAAAACGOIFCAA|2026-08-29|1519|6609|507|2026|6|8|29|3|2026|507|6609|Saturday|2026Q3|N|Y|N|2461254|2461465|2460917|2461191|N|N|N|N|N| +2461283|AAAAAAAADGOIFCAA|2026-08-30|1519|6609|507|2026|0|8|30|3|2026|507|6609|Sunday|2026Q3|N|N|N|2461254|2461465|2460918|2461192|N|N|N|N|N| +2461284|AAAAAAAAEGOIFCAA|2026-08-31|1519|6609|507|2026|1|8|31|3|2026|507|6609|Monday|2026Q3|N|N|N|2461254|2461465|2460919|2461193|N|N|N|N|N| +2461285|AAAAAAAAFGOIFCAA|2026-09-01|1520|6610|508|2026|2|9|1|3|2026|508|6610|Tuesday|2026Q3|N|N|N|2461285|2461527|2460920|2461194|N|N|N|N|N| +2461286|AAAAAAAAGGOIFCAA|2026-09-02|1520|6610|508|2026|3|9|2|3|2026|508|6610|Wednesday|2026Q3|N|N|N|2461285|2461527|2460921|2461195|N|N|N|N|N| +2461287|AAAAAAAAHGOIFCAA|2026-09-03|1520|6610|508|2026|4|9|3|3|2026|508|6610|Thursday|2026Q3|N|N|N|2461285|2461527|2460922|2461196|N|N|N|N|N| +2461288|AAAAAAAAIGOIFCAA|2026-09-04|1520|6610|508|2026|5|9|4|3|2026|508|6610|Friday|2026Q3|N|Y|N|2461285|2461527|2460923|2461197|N|N|N|N|N| +2461289|AAAAAAAAJGOIFCAA|2026-09-05|1520|6610|508|2026|6|9|5|3|2026|508|6610|Saturday|2026Q3|N|Y|N|2461285|2461527|2460924|2461198|N|N|N|N|N| +2461290|AAAAAAAAKGOIFCAA|2026-09-06|1520|6610|508|2026|0|9|6|3|2026|508|6610|Sunday|2026Q3|N|N|N|2461285|2461527|2460925|2461199|N|N|N|N|N| +2461291|AAAAAAAALGOIFCAA|2026-09-07|1520|6610|508|2026|1|9|7|3|2026|508|6610|Monday|2026Q3|N|N|N|2461285|2461527|2460926|2461200|N|N|N|N|N| +2461292|AAAAAAAAMGOIFCAA|2026-09-08|1520|6611|508|2026|2|9|8|3|2026|508|6611|Tuesday|2026Q3|N|N|N|2461285|2461527|2460927|2461201|N|N|N|N|N| +2461293|AAAAAAAANGOIFCAA|2026-09-09|1520|6611|508|2026|3|9|9|3|2026|508|6611|Wednesday|2026Q3|N|N|N|2461285|2461527|2460928|2461202|N|N|N|N|N| +2461294|AAAAAAAAOGOIFCAA|2026-09-10|1520|6611|508|2026|4|9|10|3|2026|508|6611|Thursday|2026Q3|N|N|N|2461285|2461527|2460929|2461203|N|N|N|N|N| +2461295|AAAAAAAAPGOIFCAA|2026-09-11|1520|6611|508|2026|5|9|11|3|2026|508|6611|Friday|2026Q3|N|Y|N|2461285|2461527|2460930|2461204|N|N|N|N|N| +2461296|AAAAAAAAAHOIFCAA|2026-09-12|1520|6611|508|2026|6|9|12|3|2026|508|6611|Saturday|2026Q3|N|Y|N|2461285|2461527|2460931|2461205|N|N|N|N|N| +2461297|AAAAAAAABHOIFCAA|2026-09-13|1520|6611|508|2026|0|9|13|3|2026|508|6611|Sunday|2026Q3|N|N|N|2461285|2461527|2460932|2461206|N|N|N|N|N| +2461298|AAAAAAAACHOIFCAA|2026-09-14|1520|6611|508|2026|1|9|14|3|2026|508|6611|Monday|2026Q3|N|N|N|2461285|2461527|2460933|2461207|N|N|N|N|N| +2461299|AAAAAAAADHOIFCAA|2026-09-15|1520|6612|508|2026|2|9|15|3|2026|508|6612|Tuesday|2026Q3|N|N|N|2461285|2461527|2460934|2461208|N|N|N|N|N| +2461300|AAAAAAAAEHOIFCAA|2026-09-16|1520|6612|508|2026|3|9|16|3|2026|508|6612|Wednesday|2026Q3|N|N|N|2461285|2461527|2460935|2461209|N|N|N|N|N| +2461301|AAAAAAAAFHOIFCAA|2026-09-17|1520|6612|508|2026|4|9|17|3|2026|508|6612|Thursday|2026Q3|N|N|N|2461285|2461527|2460936|2461210|N|N|N|N|N| +2461302|AAAAAAAAGHOIFCAA|2026-09-18|1520|6612|508|2026|5|9|18|3|2026|508|6612|Friday|2026Q3|N|Y|N|2461285|2461527|2460937|2461211|N|N|N|N|N| +2461303|AAAAAAAAHHOIFCAA|2026-09-19|1520|6612|508|2026|6|9|19|3|2026|508|6612|Saturday|2026Q3|N|Y|N|2461285|2461527|2460938|2461212|N|N|N|N|N| +2461304|AAAAAAAAIHOIFCAA|2026-09-20|1520|6612|508|2026|0|9|20|3|2026|508|6612|Sunday|2026Q3|N|N|N|2461285|2461527|2460939|2461213|N|N|N|N|N| +2461305|AAAAAAAAJHOIFCAA|2026-09-21|1520|6612|508|2026|1|9|21|3|2026|508|6612|Monday|2026Q3|N|N|N|2461285|2461527|2460940|2461214|N|N|N|N|N| +2461306|AAAAAAAAKHOIFCAA|2026-09-22|1520|6613|508|2026|2|9|22|3|2026|508|6613|Tuesday|2026Q3|N|N|N|2461285|2461527|2460941|2461215|N|N|N|N|N| +2461307|AAAAAAAALHOIFCAA|2026-09-23|1520|6613|508|2026|3|9|23|3|2026|508|6613|Wednesday|2026Q3|N|N|N|2461285|2461527|2460942|2461216|N|N|N|N|N| +2461308|AAAAAAAAMHOIFCAA|2026-09-24|1520|6613|508|2026|4|9|24|3|2026|508|6613|Thursday|2026Q3|N|N|N|2461285|2461527|2460943|2461217|N|N|N|N|N| +2461309|AAAAAAAANHOIFCAA|2026-09-25|1520|6613|508|2026|5|9|25|3|2026|508|6613|Friday|2026Q3|N|Y|N|2461285|2461527|2460944|2461218|N|N|N|N|N| +2461310|AAAAAAAAOHOIFCAA|2026-09-26|1520|6613|508|2026|6|9|26|3|2026|508|6613|Saturday|2026Q3|N|Y|N|2461285|2461527|2460945|2461219|N|N|N|N|N| +2461311|AAAAAAAAPHOIFCAA|2026-09-27|1520|6613|508|2026|0|9|27|3|2026|508|6613|Sunday|2026Q3|N|N|N|2461285|2461527|2460946|2461220|N|N|N|N|N| +2461312|AAAAAAAAAIOIFCAA|2026-09-28|1520|6613|508|2026|1|9|28|3|2026|508|6613|Monday|2026Q3|N|N|N|2461285|2461527|2460947|2461221|N|N|N|N|N| +2461313|AAAAAAAABIOIFCAA|2026-09-29|1520|6614|508|2026|2|9|29|3|2026|508|6614|Tuesday|2026Q3|N|N|N|2461285|2461527|2460948|2461222|N|N|N|N|N| +2461314|AAAAAAAACIOIFCAA|2026-09-30|1520|6614|508|2026|3|9|30|3|2026|508|6614|Wednesday|2026Q3|N|N|N|2461285|2461527|2460949|2461223|N|N|N|N|N| +2461315|AAAAAAAADIOIFCAA|2026-10-01|1521|6614|508|2026|4|10|1|3|2026|508|6614|Thursday|2026Q3|N|N|N|2461315|2461587|2460950|2461223|N|N|N|N|N| +2461316|AAAAAAAAEIOIFCAA|2026-10-02|1521|6614|508|2026|5|10|2|4|2026|508|6614|Friday|2026Q4|N|Y|N|2461315|2461587|2460951|2461224|N|N|N|N|N| +2461317|AAAAAAAAFIOIFCAA|2026-10-03|1521|6614|508|2026|6|10|3|4|2026|508|6614|Saturday|2026Q4|N|Y|N|2461315|2461587|2460952|2461225|N|N|N|N|N| +2461318|AAAAAAAAGIOIFCAA|2026-10-04|1521|6614|508|2026|0|10|4|4|2026|508|6614|Sunday|2026Q4|N|N|N|2461315|2461587|2460953|2461226|N|N|N|N|N| +2461319|AAAAAAAAHIOIFCAA|2026-10-05|1521|6614|508|2026|1|10|5|4|2026|508|6614|Monday|2026Q4|N|N|N|2461315|2461587|2460954|2461227|N|N|N|N|N| +2461320|AAAAAAAAIIOIFCAA|2026-10-06|1521|6615|508|2026|2|10|6|4|2026|508|6615|Tuesday|2026Q4|N|N|N|2461315|2461587|2460955|2461228|N|N|N|N|N| +2461321|AAAAAAAAJIOIFCAA|2026-10-07|1521|6615|508|2026|3|10|7|4|2026|508|6615|Wednesday|2026Q4|N|N|N|2461315|2461587|2460956|2461229|N|N|N|N|N| +2461322|AAAAAAAAKIOIFCAA|2026-10-08|1521|6615|508|2026|4|10|8|4|2026|508|6615|Thursday|2026Q4|N|N|N|2461315|2461587|2460957|2461230|N|N|N|N|N| +2461323|AAAAAAAALIOIFCAA|2026-10-09|1521|6615|508|2026|5|10|9|4|2026|508|6615|Friday|2026Q4|N|Y|N|2461315|2461587|2460958|2461231|N|N|N|N|N| +2461324|AAAAAAAAMIOIFCAA|2026-10-10|1521|6615|508|2026|6|10|10|4|2026|508|6615|Saturday|2026Q4|N|Y|N|2461315|2461587|2460959|2461232|N|N|N|N|N| +2461325|AAAAAAAANIOIFCAA|2026-10-11|1521|6615|508|2026|0|10|11|4|2026|508|6615|Sunday|2026Q4|N|N|N|2461315|2461587|2460960|2461233|N|N|N|N|N| +2461326|AAAAAAAAOIOIFCAA|2026-10-12|1521|6615|508|2026|1|10|12|4|2026|508|6615|Monday|2026Q4|N|N|N|2461315|2461587|2460961|2461234|N|N|N|N|N| +2461327|AAAAAAAAPIOIFCAA|2026-10-13|1521|6616|508|2026|2|10|13|4|2026|508|6616|Tuesday|2026Q4|N|N|N|2461315|2461587|2460962|2461235|N|N|N|N|N| +2461328|AAAAAAAAAJOIFCAA|2026-10-14|1521|6616|508|2026|3|10|14|4|2026|508|6616|Wednesday|2026Q4|N|N|N|2461315|2461587|2460963|2461236|N|N|N|N|N| +2461329|AAAAAAAABJOIFCAA|2026-10-15|1521|6616|508|2026|4|10|15|4|2026|508|6616|Thursday|2026Q4|N|N|N|2461315|2461587|2460964|2461237|N|N|N|N|N| +2461330|AAAAAAAACJOIFCAA|2026-10-16|1521|6616|508|2026|5|10|16|4|2026|508|6616|Friday|2026Q4|N|Y|N|2461315|2461587|2460965|2461238|N|N|N|N|N| +2461331|AAAAAAAADJOIFCAA|2026-10-17|1521|6616|508|2026|6|10|17|4|2026|508|6616|Saturday|2026Q4|N|Y|N|2461315|2461587|2460966|2461239|N|N|N|N|N| +2461332|AAAAAAAAEJOIFCAA|2026-10-18|1521|6616|508|2026|0|10|18|4|2026|508|6616|Sunday|2026Q4|N|N|N|2461315|2461587|2460967|2461240|N|N|N|N|N| +2461333|AAAAAAAAFJOIFCAA|2026-10-19|1521|6616|508|2026|1|10|19|4|2026|508|6616|Monday|2026Q4|N|N|N|2461315|2461587|2460968|2461241|N|N|N|N|N| +2461334|AAAAAAAAGJOIFCAA|2026-10-20|1521|6617|508|2026|2|10|20|4|2026|508|6617|Tuesday|2026Q4|N|N|N|2461315|2461587|2460969|2461242|N|N|N|N|N| +2461335|AAAAAAAAHJOIFCAA|2026-10-21|1521|6617|508|2026|3|10|21|4|2026|508|6617|Wednesday|2026Q4|N|N|N|2461315|2461587|2460970|2461243|N|N|N|N|N| +2461336|AAAAAAAAIJOIFCAA|2026-10-22|1521|6617|508|2026|4|10|22|4|2026|508|6617|Thursday|2026Q4|N|N|N|2461315|2461587|2460971|2461244|N|N|N|N|N| +2461337|AAAAAAAAJJOIFCAA|2026-10-23|1521|6617|508|2026|5|10|23|4|2026|508|6617|Friday|2026Q4|N|Y|N|2461315|2461587|2460972|2461245|N|N|N|N|N| +2461338|AAAAAAAAKJOIFCAA|2026-10-24|1521|6617|508|2026|6|10|24|4|2026|508|6617|Saturday|2026Q4|N|Y|N|2461315|2461587|2460973|2461246|N|N|N|N|N| +2461339|AAAAAAAALJOIFCAA|2026-10-25|1521|6617|508|2026|0|10|25|4|2026|508|6617|Sunday|2026Q4|N|N|N|2461315|2461587|2460974|2461247|N|N|N|N|N| +2461340|AAAAAAAAMJOIFCAA|2026-10-26|1521|6617|508|2026|1|10|26|4|2026|508|6617|Monday|2026Q4|N|N|N|2461315|2461587|2460975|2461248|N|N|N|N|N| +2461341|AAAAAAAANJOIFCAA|2026-10-27|1521|6618|508|2026|2|10|27|4|2026|508|6618|Tuesday|2026Q4|N|N|N|2461315|2461587|2460976|2461249|N|N|N|N|N| +2461342|AAAAAAAAOJOIFCAA|2026-10-28|1521|6618|508|2026|3|10|28|4|2026|508|6618|Wednesday|2026Q4|N|N|N|2461315|2461587|2460977|2461250|N|N|N|N|N| +2461343|AAAAAAAAPJOIFCAA|2026-10-29|1521|6618|508|2026|4|10|29|4|2026|508|6618|Thursday|2026Q4|N|N|N|2461315|2461587|2460978|2461251|N|N|N|N|N| +2461344|AAAAAAAAAKOIFCAA|2026-10-30|1521|6618|508|2026|5|10|30|4|2026|508|6618|Friday|2026Q4|N|Y|N|2461315|2461587|2460979|2461252|N|N|N|N|N| +2461345|AAAAAAAABKOIFCAA|2026-10-31|1521|6618|508|2026|6|10|31|4|2026|508|6618|Saturday|2026Q4|N|Y|N|2461315|2461587|2460980|2461253|N|N|N|N|N| +2461346|AAAAAAAACKOIFCAA|2026-11-01|1522|6618|508|2026|0|11|1|4|2026|508|6618|Sunday|2026Q4|N|N|N|2461346|2461649|2460981|2461254|N|N|N|N|N| +2461347|AAAAAAAADKOIFCAA|2026-11-02|1522|6618|508|2026|1|11|2|4|2026|508|6618|Monday|2026Q4|N|N|N|2461346|2461649|2460982|2461255|N|N|N|N|N| +2461348|AAAAAAAAEKOIFCAA|2026-11-03|1522|6619|508|2026|2|11|3|4|2026|508|6619|Tuesday|2026Q4|N|N|N|2461346|2461649|2460983|2461256|N|N|N|N|N| +2461349|AAAAAAAAFKOIFCAA|2026-11-04|1522|6619|508|2026|3|11|4|4|2026|508|6619|Wednesday|2026Q4|N|N|N|2461346|2461649|2460984|2461257|N|N|N|N|N| +2461350|AAAAAAAAGKOIFCAA|2026-11-05|1522|6619|508|2026|4|11|5|4|2026|508|6619|Thursday|2026Q4|N|N|N|2461346|2461649|2460985|2461258|N|N|N|N|N| +2461351|AAAAAAAAHKOIFCAA|2026-11-06|1522|6619|508|2026|5|11|6|4|2026|508|6619|Friday|2026Q4|N|Y|N|2461346|2461649|2460986|2461259|N|N|N|N|N| +2461352|AAAAAAAAIKOIFCAA|2026-11-07|1522|6619|508|2026|6|11|7|4|2026|508|6619|Saturday|2026Q4|N|Y|N|2461346|2461649|2460987|2461260|N|N|N|N|N| +2461353|AAAAAAAAJKOIFCAA|2026-11-08|1522|6619|508|2026|0|11|8|4|2026|508|6619|Sunday|2026Q4|N|N|N|2461346|2461649|2460988|2461261|N|N|N|N|N| +2461354|AAAAAAAAKKOIFCAA|2026-11-09|1522|6619|508|2026|1|11|9|4|2026|508|6619|Monday|2026Q4|N|N|N|2461346|2461649|2460989|2461262|N|N|N|N|N| +2461355|AAAAAAAALKOIFCAA|2026-11-10|1522|6620|508|2026|2|11|10|4|2026|508|6620|Tuesday|2026Q4|N|N|N|2461346|2461649|2460990|2461263|N|N|N|N|N| +2461356|AAAAAAAAMKOIFCAA|2026-11-11|1522|6620|508|2026|3|11|11|4|2026|508|6620|Wednesday|2026Q4|N|N|N|2461346|2461649|2460991|2461264|N|N|N|N|N| +2461357|AAAAAAAANKOIFCAA|2026-11-12|1522|6620|508|2026|4|11|12|4|2026|508|6620|Thursday|2026Q4|N|N|N|2461346|2461649|2460992|2461265|N|N|N|N|N| +2461358|AAAAAAAAOKOIFCAA|2026-11-13|1522|6620|508|2026|5|11|13|4|2026|508|6620|Friday|2026Q4|N|Y|N|2461346|2461649|2460993|2461266|N|N|N|N|N| +2461359|AAAAAAAAPKOIFCAA|2026-11-14|1522|6620|508|2026|6|11|14|4|2026|508|6620|Saturday|2026Q4|N|Y|N|2461346|2461649|2460994|2461267|N|N|N|N|N| +2461360|AAAAAAAAALOIFCAA|2026-11-15|1522|6620|508|2026|0|11|15|4|2026|508|6620|Sunday|2026Q4|N|N|N|2461346|2461649|2460995|2461268|N|N|N|N|N| +2461361|AAAAAAAABLOIFCAA|2026-11-16|1522|6620|508|2026|1|11|16|4|2026|508|6620|Monday|2026Q4|N|N|N|2461346|2461649|2460996|2461269|N|N|N|N|N| +2461362|AAAAAAAACLOIFCAA|2026-11-17|1522|6621|508|2026|2|11|17|4|2026|508|6621|Tuesday|2026Q4|N|N|N|2461346|2461649|2460997|2461270|N|N|N|N|N| +2461363|AAAAAAAADLOIFCAA|2026-11-18|1522|6621|508|2026|3|11|18|4|2026|508|6621|Wednesday|2026Q4|N|N|N|2461346|2461649|2460998|2461271|N|N|N|N|N| +2461364|AAAAAAAAELOIFCAA|2026-11-19|1522|6621|508|2026|4|11|19|4|2026|508|6621|Thursday|2026Q4|N|N|N|2461346|2461649|2460999|2461272|N|N|N|N|N| +2461365|AAAAAAAAFLOIFCAA|2026-11-20|1522|6621|508|2026|5|11|20|4|2026|508|6621|Friday|2026Q4|N|Y|N|2461346|2461649|2461000|2461273|N|N|N|N|N| +2461366|AAAAAAAAGLOIFCAA|2026-11-21|1522|6621|508|2026|6|11|21|4|2026|508|6621|Saturday|2026Q4|N|Y|N|2461346|2461649|2461001|2461274|N|N|N|N|N| +2461367|AAAAAAAAHLOIFCAA|2026-11-22|1522|6621|508|2026|0|11|22|4|2026|508|6621|Sunday|2026Q4|N|N|N|2461346|2461649|2461002|2461275|N|N|N|N|N| +2461368|AAAAAAAAILOIFCAA|2026-11-23|1522|6621|508|2026|1|11|23|4|2026|508|6621|Monday|2026Q4|N|N|N|2461346|2461649|2461003|2461276|N|N|N|N|N| +2461369|AAAAAAAAJLOIFCAA|2026-11-24|1522|6622|508|2026|2|11|24|4|2026|508|6622|Tuesday|2026Q4|N|N|N|2461346|2461649|2461004|2461277|N|N|N|N|N| +2461370|AAAAAAAAKLOIFCAA|2026-11-25|1522|6622|508|2026|3|11|25|4|2026|508|6622|Wednesday|2026Q4|N|N|N|2461346|2461649|2461005|2461278|N|N|N|N|N| +2461371|AAAAAAAALLOIFCAA|2026-11-26|1522|6622|508|2026|4|11|26|4|2026|508|6622|Thursday|2026Q4|N|N|N|2461346|2461649|2461006|2461279|N|N|N|N|N| +2461372|AAAAAAAAMLOIFCAA|2026-11-27|1522|6622|508|2026|5|11|27|4|2026|508|6622|Friday|2026Q4|N|Y|N|2461346|2461649|2461007|2461280|N|N|N|N|N| +2461373|AAAAAAAANLOIFCAA|2026-11-28|1522|6622|508|2026|6|11|28|4|2026|508|6622|Saturday|2026Q4|N|Y|N|2461346|2461649|2461008|2461281|N|N|N|N|N| +2461374|AAAAAAAAOLOIFCAA|2026-11-29|1522|6622|508|2026|0|11|29|4|2026|508|6622|Sunday|2026Q4|N|N|N|2461346|2461649|2461009|2461282|N|N|N|N|N| +2461375|AAAAAAAAPLOIFCAA|2026-11-30|1522|6622|508|2026|1|11|30|4|2026|508|6622|Monday|2026Q4|N|N|N|2461346|2461649|2461010|2461283|N|N|N|N|N| +2461376|AAAAAAAAAMOIFCAA|2026-12-01|1523|6623|509|2026|2|12|1|4|2026|509|6623|Tuesday|2026Q4|N|N|N|2461376|2461709|2461011|2461284|N|N|N|N|N| +2461377|AAAAAAAABMOIFCAA|2026-12-02|1523|6623|509|2026|3|12|2|4|2026|509|6623|Wednesday|2026Q4|N|N|N|2461376|2461709|2461012|2461285|N|N|N|N|N| +2461378|AAAAAAAACMOIFCAA|2026-12-03|1523|6623|509|2026|4|12|3|4|2026|509|6623|Thursday|2026Q4|N|N|N|2461376|2461709|2461013|2461286|N|N|N|N|N| +2461379|AAAAAAAADMOIFCAA|2026-12-04|1523|6623|509|2026|5|12|4|4|2026|509|6623|Friday|2026Q4|N|Y|N|2461376|2461709|2461014|2461287|N|N|N|N|N| +2461380|AAAAAAAAEMOIFCAA|2026-12-05|1523|6623|509|2026|6|12|5|4|2026|509|6623|Saturday|2026Q4|N|Y|N|2461376|2461709|2461015|2461288|N|N|N|N|N| +2461381|AAAAAAAAFMOIFCAA|2026-12-06|1523|6623|509|2026|0|12|6|4|2026|509|6623|Sunday|2026Q4|N|N|N|2461376|2461709|2461016|2461289|N|N|N|N|N| +2461382|AAAAAAAAGMOIFCAA|2026-12-07|1523|6623|509|2026|1|12|7|4|2026|509|6623|Monday|2026Q4|N|N|N|2461376|2461709|2461017|2461290|N|N|N|N|N| +2461383|AAAAAAAAHMOIFCAA|2026-12-08|1523|6624|509|2026|2|12|8|4|2026|509|6624|Tuesday|2026Q4|N|N|N|2461376|2461709|2461018|2461291|N|N|N|N|N| +2461384|AAAAAAAAIMOIFCAA|2026-12-09|1523|6624|509|2026|3|12|9|4|2026|509|6624|Wednesday|2026Q4|N|N|N|2461376|2461709|2461019|2461292|N|N|N|N|N| +2461385|AAAAAAAAJMOIFCAA|2026-12-10|1523|6624|509|2026|4|12|10|4|2026|509|6624|Thursday|2026Q4|N|N|N|2461376|2461709|2461020|2461293|N|N|N|N|N| +2461386|AAAAAAAAKMOIFCAA|2026-12-11|1523|6624|509|2026|5|12|11|4|2026|509|6624|Friday|2026Q4|N|Y|N|2461376|2461709|2461021|2461294|N|N|N|N|N| +2461387|AAAAAAAALMOIFCAA|2026-12-12|1523|6624|509|2026|6|12|12|4|2026|509|6624|Saturday|2026Q4|N|Y|N|2461376|2461709|2461022|2461295|N|N|N|N|N| +2461388|AAAAAAAAMMOIFCAA|2026-12-13|1523|6624|509|2026|0|12|13|4|2026|509|6624|Sunday|2026Q4|N|N|N|2461376|2461709|2461023|2461296|N|N|N|N|N| +2461389|AAAAAAAANMOIFCAA|2026-12-14|1523|6624|509|2026|1|12|14|4|2026|509|6624|Monday|2026Q4|N|N|N|2461376|2461709|2461024|2461297|N|N|N|N|N| +2461390|AAAAAAAAOMOIFCAA|2026-12-15|1523|6625|509|2026|2|12|15|4|2026|509|6625|Tuesday|2026Q4|N|N|N|2461376|2461709|2461025|2461298|N|N|N|N|N| +2461391|AAAAAAAAPMOIFCAA|2026-12-16|1523|6625|509|2026|3|12|16|4|2026|509|6625|Wednesday|2026Q4|N|N|N|2461376|2461709|2461026|2461299|N|N|N|N|N| +2461392|AAAAAAAAANOIFCAA|2026-12-17|1523|6625|509|2026|4|12|17|4|2026|509|6625|Thursday|2026Q4|N|N|N|2461376|2461709|2461027|2461300|N|N|N|N|N| +2461393|AAAAAAAABNOIFCAA|2026-12-18|1523|6625|509|2026|5|12|18|4|2026|509|6625|Friday|2026Q4|N|Y|N|2461376|2461709|2461028|2461301|N|N|N|N|N| +2461394|AAAAAAAACNOIFCAA|2026-12-19|1523|6625|509|2026|6|12|19|4|2026|509|6625|Saturday|2026Q4|N|Y|N|2461376|2461709|2461029|2461302|N|N|N|N|N| +2461395|AAAAAAAADNOIFCAA|2026-12-20|1523|6625|509|2026|0|12|20|4|2026|509|6625|Sunday|2026Q4|N|N|N|2461376|2461709|2461030|2461303|N|N|N|N|N| +2461396|AAAAAAAAENOIFCAA|2026-12-21|1523|6625|509|2026|1|12|21|4|2026|509|6625|Monday|2026Q4|N|N|N|2461376|2461709|2461031|2461304|N|N|N|N|N| +2461397|AAAAAAAAFNOIFCAA|2026-12-22|1523|6626|509|2026|2|12|22|4|2026|509|6626|Tuesday|2026Q4|N|N|N|2461376|2461709|2461032|2461305|N|N|N|N|N| +2461398|AAAAAAAAGNOIFCAA|2026-12-23|1523|6626|509|2026|3|12|23|4|2026|509|6626|Wednesday|2026Q4|N|N|N|2461376|2461709|2461033|2461306|N|N|N|N|N| +2461399|AAAAAAAAHNOIFCAA|2026-12-24|1523|6626|509|2026|4|12|24|4|2026|509|6626|Thursday|2026Q4|N|N|N|2461376|2461709|2461034|2461307|N|N|N|N|N| +2461400|AAAAAAAAINOIFCAA|2026-12-25|1523|6626|509|2026|5|12|25|4|2026|509|6626|Friday|2026Q4|N|Y|N|2461376|2461709|2461035|2461308|N|N|N|N|N| +2461401|AAAAAAAAJNOIFCAA|2026-12-26|1523|6626|509|2026|6|12|26|4|2026|509|6626|Saturday|2026Q4|Y|Y|N|2461376|2461709|2461036|2461309|N|N|N|N|N| +2461402|AAAAAAAAKNOIFCAA|2026-12-27|1523|6626|509|2026|0|12|27|4|2026|509|6626|Sunday|2026Q4|N|N|Y|2461376|2461709|2461037|2461310|N|N|N|N|N| +2461403|AAAAAAAALNOIFCAA|2026-12-28|1523|6626|509|2026|1|12|28|4|2026|509|6626|Monday|2026Q4|N|N|N|2461376|2461709|2461038|2461311|N|N|N|N|N| +2461404|AAAAAAAAMNOIFCAA|2026-12-29|1523|6627|509|2026|2|12|29|4|2026|509|6627|Tuesday|2026Q4|N|N|N|2461376|2461709|2461039|2461312|N|N|N|N|N| +2461405|AAAAAAAANNOIFCAA|2026-12-30|1523|6627|509|2026|3|12|30|4|2026|509|6627|Wednesday|2026Q4|N|N|N|2461376|2461709|2461040|2461313|N|N|N|N|N| +2461406|AAAAAAAAONOIFCAA|2026-12-31|1523|6627|509|2026|4|12|31|4|2026|509|6627|Thursday|2026Q4|N|N|N|2461376|2461709|2461041|2461314|N|N|N|N|N| +2461407|AAAAAAAAPNOIFCAA|2027-01-01|1524|6627|509|2027|5|1|1|1|2027|509|6627|Friday|2027Q1|Y|Y|N|2461407|2461406|2461042|2461315|N|N|N|N|N| +2461408|AAAAAAAAAOOIFCAA|2027-01-02|1524|6627|509|2027|6|1|2|1|2027|509|6627|Saturday|2027Q1|N|Y|Y|2461407|2461406|2461043|2461316|N|N|N|N|N| +2461409|AAAAAAAABOOIFCAA|2027-01-03|1524|6627|509|2027|0|1|3|1|2027|509|6627|Sunday|2027Q1|N|N|N|2461407|2461406|2461044|2461317|N|N|N|N|N| +2461410|AAAAAAAACOOIFCAA|2027-01-04|1524|6627|509|2027|1|1|4|1|2027|509|6627|Monday|2027Q1|N|N|N|2461407|2461406|2461045|2461318|N|N|N|N|N| +2461411|AAAAAAAADOOIFCAA|2027-01-05|1524|6628|509|2027|2|1|5|1|2027|509|6628|Tuesday|2027Q1|N|N|N|2461407|2461406|2461046|2461319|N|N|N|N|N| +2461412|AAAAAAAAEOOIFCAA|2027-01-06|1524|6628|509|2027|3|1|6|1|2027|509|6628|Wednesday|2027Q1|N|N|N|2461407|2461406|2461047|2461320|N|N|N|N|N| +2461413|AAAAAAAAFOOIFCAA|2027-01-07|1524|6628|509|2027|4|1|7|1|2027|509|6628|Thursday|2027Q1|N|N|N|2461407|2461406|2461048|2461321|N|N|N|N|N| +2461414|AAAAAAAAGOOIFCAA|2027-01-08|1524|6628|509|2027|5|1|8|1|2027|509|6628|Friday|2027Q1|N|Y|N|2461407|2461406|2461049|2461322|N|N|N|N|N| +2461415|AAAAAAAAHOOIFCAA|2027-01-09|1524|6628|509|2027|6|1|9|1|2027|509|6628|Saturday|2027Q1|N|Y|N|2461407|2461406|2461050|2461323|N|N|N|N|N| +2461416|AAAAAAAAIOOIFCAA|2027-01-10|1524|6628|509|2027|0|1|10|1|2027|509|6628|Sunday|2027Q1|N|N|N|2461407|2461406|2461051|2461324|N|N|N|N|N| +2461417|AAAAAAAAJOOIFCAA|2027-01-11|1524|6628|509|2027|1|1|11|1|2027|509|6628|Monday|2027Q1|N|N|N|2461407|2461406|2461052|2461325|N|N|N|N|N| +2461418|AAAAAAAAKOOIFCAA|2027-01-12|1524|6629|509|2027|2|1|12|1|2027|509|6629|Tuesday|2027Q1|N|N|N|2461407|2461406|2461053|2461326|N|N|N|N|N| +2461419|AAAAAAAALOOIFCAA|2027-01-13|1524|6629|509|2027|3|1|13|1|2027|509|6629|Wednesday|2027Q1|N|N|N|2461407|2461406|2461054|2461327|N|N|N|N|N| +2461420|AAAAAAAAMOOIFCAA|2027-01-14|1524|6629|509|2027|4|1|14|1|2027|509|6629|Thursday|2027Q1|N|N|N|2461407|2461406|2461055|2461328|N|N|N|N|N| +2461421|AAAAAAAANOOIFCAA|2027-01-15|1524|6629|509|2027|5|1|15|1|2027|509|6629|Friday|2027Q1|N|Y|N|2461407|2461406|2461056|2461329|N|N|N|N|N| +2461422|AAAAAAAAOOOIFCAA|2027-01-16|1524|6629|509|2027|6|1|16|1|2027|509|6629|Saturday|2027Q1|N|Y|N|2461407|2461406|2461057|2461330|N|N|N|N|N| +2461423|AAAAAAAAPOOIFCAA|2027-01-17|1524|6629|509|2027|0|1|17|1|2027|509|6629|Sunday|2027Q1|N|N|N|2461407|2461406|2461058|2461331|N|N|N|N|N| +2461424|AAAAAAAAAPOIFCAA|2027-01-18|1524|6629|509|2027|1|1|18|1|2027|509|6629|Monday|2027Q1|N|N|N|2461407|2461406|2461059|2461332|N|N|N|N|N| +2461425|AAAAAAAABPOIFCAA|2027-01-19|1524|6630|509|2027|2|1|19|1|2027|509|6630|Tuesday|2027Q1|N|N|N|2461407|2461406|2461060|2461333|N|N|N|N|N| +2461426|AAAAAAAACPOIFCAA|2027-01-20|1524|6630|509|2027|3|1|20|1|2027|509|6630|Wednesday|2027Q1|N|N|N|2461407|2461406|2461061|2461334|N|N|N|N|N| +2461427|AAAAAAAADPOIFCAA|2027-01-21|1524|6630|509|2027|4|1|21|1|2027|509|6630|Thursday|2027Q1|N|N|N|2461407|2461406|2461062|2461335|N|N|N|N|N| +2461428|AAAAAAAAEPOIFCAA|2027-01-22|1524|6630|509|2027|5|1|22|1|2027|509|6630|Friday|2027Q1|N|Y|N|2461407|2461406|2461063|2461336|N|N|N|N|N| +2461429|AAAAAAAAFPOIFCAA|2027-01-23|1524|6630|509|2027|6|1|23|1|2027|509|6630|Saturday|2027Q1|N|Y|N|2461407|2461406|2461064|2461337|N|N|N|N|N| +2461430|AAAAAAAAGPOIFCAA|2027-01-24|1524|6630|509|2027|0|1|24|1|2027|509|6630|Sunday|2027Q1|N|N|N|2461407|2461406|2461065|2461338|N|N|N|N|N| +2461431|AAAAAAAAHPOIFCAA|2027-01-25|1524|6630|509|2027|1|1|25|1|2027|509|6630|Monday|2027Q1|N|N|N|2461407|2461406|2461066|2461339|N|N|N|N|N| +2461432|AAAAAAAAIPOIFCAA|2027-01-26|1524|6631|509|2027|2|1|26|1|2027|509|6631|Tuesday|2027Q1|N|N|N|2461407|2461406|2461067|2461340|N|N|N|N|N| +2461433|AAAAAAAAJPOIFCAA|2027-01-27|1524|6631|509|2027|3|1|27|1|2027|509|6631|Wednesday|2027Q1|N|N|N|2461407|2461406|2461068|2461341|N|N|N|N|N| +2461434|AAAAAAAAKPOIFCAA|2027-01-28|1524|6631|509|2027|4|1|28|1|2027|509|6631|Thursday|2027Q1|N|N|N|2461407|2461406|2461069|2461342|N|N|N|N|N| +2461435|AAAAAAAALPOIFCAA|2027-01-29|1524|6631|509|2027|5|1|29|1|2027|509|6631|Friday|2027Q1|N|Y|N|2461407|2461406|2461070|2461343|N|N|N|N|N| +2461436|AAAAAAAAMPOIFCAA|2027-01-30|1524|6631|509|2027|6|1|30|1|2027|509|6631|Saturday|2027Q1|N|Y|N|2461407|2461406|2461071|2461344|N|N|N|N|N| +2461437|AAAAAAAANPOIFCAA|2027-01-31|1524|6631|509|2027|0|1|31|1|2027|509|6631|Sunday|2027Q1|N|N|N|2461407|2461406|2461072|2461345|N|N|N|N|N| +2461438|AAAAAAAAOPOIFCAA|2027-02-01|1525|6631|509|2027|1|2|1|1|2027|509|6631|Monday|2027Q1|N|N|N|2461438|2461468|2461073|2461346|N|N|N|N|N| +2461439|AAAAAAAAPPOIFCAA|2027-02-02|1525|6632|509|2027|2|2|2|1|2027|509|6632|Tuesday|2027Q1|N|N|N|2461438|2461468|2461074|2461347|N|N|N|N|N| +2461440|AAAAAAAAAAPIFCAA|2027-02-03|1525|6632|509|2027|3|2|3|1|2027|509|6632|Wednesday|2027Q1|N|N|N|2461438|2461468|2461075|2461348|N|N|N|N|N| +2461441|AAAAAAAABAPIFCAA|2027-02-04|1525|6632|509|2027|4|2|4|1|2027|509|6632|Thursday|2027Q1|N|N|N|2461438|2461468|2461076|2461349|N|N|N|N|N| +2461442|AAAAAAAACAPIFCAA|2027-02-05|1525|6632|509|2027|5|2|5|1|2027|509|6632|Friday|2027Q1|N|Y|N|2461438|2461468|2461077|2461350|N|N|N|N|N| +2461443|AAAAAAAADAPIFCAA|2027-02-06|1525|6632|509|2027|6|2|6|1|2027|509|6632|Saturday|2027Q1|N|Y|N|2461438|2461468|2461078|2461351|N|N|N|N|N| +2461444|AAAAAAAAEAPIFCAA|2027-02-07|1525|6632|509|2027|0|2|7|1|2027|509|6632|Sunday|2027Q1|N|N|N|2461438|2461468|2461079|2461352|N|N|N|N|N| +2461445|AAAAAAAAFAPIFCAA|2027-02-08|1525|6632|509|2027|1|2|8|1|2027|509|6632|Monday|2027Q1|N|N|N|2461438|2461468|2461080|2461353|N|N|N|N|N| +2461446|AAAAAAAAGAPIFCAA|2027-02-09|1525|6633|509|2027|2|2|9|1|2027|509|6633|Tuesday|2027Q1|N|N|N|2461438|2461468|2461081|2461354|N|N|N|N|N| +2461447|AAAAAAAAHAPIFCAA|2027-02-10|1525|6633|509|2027|3|2|10|1|2027|509|6633|Wednesday|2027Q1|N|N|N|2461438|2461468|2461082|2461355|N|N|N|N|N| +2461448|AAAAAAAAIAPIFCAA|2027-02-11|1525|6633|509|2027|4|2|11|1|2027|509|6633|Thursday|2027Q1|N|N|N|2461438|2461468|2461083|2461356|N|N|N|N|N| +2461449|AAAAAAAAJAPIFCAA|2027-02-12|1525|6633|509|2027|5|2|12|1|2027|509|6633|Friday|2027Q1|N|Y|N|2461438|2461468|2461084|2461357|N|N|N|N|N| +2461450|AAAAAAAAKAPIFCAA|2027-02-13|1525|6633|509|2027|6|2|13|1|2027|509|6633|Saturday|2027Q1|N|Y|N|2461438|2461468|2461085|2461358|N|N|N|N|N| +2461451|AAAAAAAALAPIFCAA|2027-02-14|1525|6633|509|2027|0|2|14|1|2027|509|6633|Sunday|2027Q1|N|N|N|2461438|2461468|2461086|2461359|N|N|N|N|N| +2461452|AAAAAAAAMAPIFCAA|2027-02-15|1525|6633|509|2027|1|2|15|1|2027|509|6633|Monday|2027Q1|N|N|N|2461438|2461468|2461087|2461360|N|N|N|N|N| +2461453|AAAAAAAANAPIFCAA|2027-02-16|1525|6634|509|2027|2|2|16|1|2027|509|6634|Tuesday|2027Q1|N|N|N|2461438|2461468|2461088|2461361|N|N|N|N|N| +2461454|AAAAAAAAOAPIFCAA|2027-02-17|1525|6634|509|2027|3|2|17|1|2027|509|6634|Wednesday|2027Q1|N|N|N|2461438|2461468|2461089|2461362|N|N|N|N|N| +2461455|AAAAAAAAPAPIFCAA|2027-02-18|1525|6634|509|2027|4|2|18|1|2027|509|6634|Thursday|2027Q1|N|N|N|2461438|2461468|2461090|2461363|N|N|N|N|N| +2461456|AAAAAAAAABPIFCAA|2027-02-19|1525|6634|509|2027|5|2|19|1|2027|509|6634|Friday|2027Q1|N|Y|N|2461438|2461468|2461091|2461364|N|N|N|N|N| +2461457|AAAAAAAABBPIFCAA|2027-02-20|1525|6634|509|2027|6|2|20|1|2027|509|6634|Saturday|2027Q1|N|Y|N|2461438|2461468|2461092|2461365|N|N|N|N|N| +2461458|AAAAAAAACBPIFCAA|2027-02-21|1525|6634|509|2027|0|2|21|1|2027|509|6634|Sunday|2027Q1|N|N|N|2461438|2461468|2461093|2461366|N|N|N|N|N| +2461459|AAAAAAAADBPIFCAA|2027-02-22|1525|6634|509|2027|1|2|22|1|2027|509|6634|Monday|2027Q1|N|N|N|2461438|2461468|2461094|2461367|N|N|N|N|N| +2461460|AAAAAAAAEBPIFCAA|2027-02-23|1525|6635|509|2027|2|2|23|1|2027|509|6635|Tuesday|2027Q1|N|N|N|2461438|2461468|2461095|2461368|N|N|N|N|N| +2461461|AAAAAAAAFBPIFCAA|2027-02-24|1525|6635|509|2027|3|2|24|1|2027|509|6635|Wednesday|2027Q1|N|N|N|2461438|2461468|2461096|2461369|N|N|N|N|N| +2461462|AAAAAAAAGBPIFCAA|2027-02-25|1525|6635|509|2027|4|2|25|1|2027|509|6635|Thursday|2027Q1|N|N|N|2461438|2461468|2461097|2461370|N|N|N|N|N| +2461463|AAAAAAAAHBPIFCAA|2027-02-26|1525|6635|509|2027|5|2|26|1|2027|509|6635|Friday|2027Q1|N|Y|N|2461438|2461468|2461098|2461371|N|N|N|N|N| +2461464|AAAAAAAAIBPIFCAA|2027-02-27|1525|6635|509|2027|6|2|27|1|2027|509|6635|Saturday|2027Q1|N|Y|N|2461438|2461468|2461099|2461372|N|N|N|N|N| +2461465|AAAAAAAAJBPIFCAA|2027-02-28|1525|6635|509|2027|0|2|28|1|2027|509|6635|Sunday|2027Q1|N|N|N|2461438|2461468|2461100|2461373|N|N|N|N|N| +2461466|AAAAAAAAKBPIFCAA|2027-03-01|1526|6635|510|2027|1|3|1|1|2027|510|6635|Monday|2027Q1|N|N|N|2461466|2461524|2461101|2461374|N|N|N|N|N| +2461467|AAAAAAAALBPIFCAA|2027-03-02|1526|6636|510|2027|2|3|2|1|2027|510|6636|Tuesday|2027Q1|N|N|N|2461466|2461524|2461102|2461375|N|N|N|N|N| +2461468|AAAAAAAAMBPIFCAA|2027-03-03|1526|6636|510|2027|3|3|3|1|2027|510|6636|Wednesday|2027Q1|N|N|N|2461466|2461524|2461103|2461376|N|N|N|N|N| +2461469|AAAAAAAANBPIFCAA|2027-03-04|1526|6636|510|2027|4|3|4|1|2027|510|6636|Thursday|2027Q1|N|N|N|2461466|2461524|2461104|2461377|N|N|N|N|N| +2461470|AAAAAAAAOBPIFCAA|2027-03-05|1526|6636|510|2027|5|3|5|1|2027|510|6636|Friday|2027Q1|N|Y|N|2461466|2461524|2461105|2461378|N|N|N|N|N| +2461471|AAAAAAAAPBPIFCAA|2027-03-06|1526|6636|510|2027|6|3|6|1|2027|510|6636|Saturday|2027Q1|N|Y|N|2461466|2461524|2461106|2461379|N|N|N|N|N| +2461472|AAAAAAAAACPIFCAA|2027-03-07|1526|6636|510|2027|0|3|7|1|2027|510|6636|Sunday|2027Q1|N|N|N|2461466|2461524|2461107|2461380|N|N|N|N|N| +2461473|AAAAAAAABCPIFCAA|2027-03-08|1526|6636|510|2027|1|3|8|1|2027|510|6636|Monday|2027Q1|N|N|N|2461466|2461524|2461108|2461381|N|N|N|N|N| +2461474|AAAAAAAACCPIFCAA|2027-03-09|1526|6637|510|2027|2|3|9|1|2027|510|6637|Tuesday|2027Q1|N|N|N|2461466|2461524|2461109|2461382|N|N|N|N|N| +2461475|AAAAAAAADCPIFCAA|2027-03-10|1526|6637|510|2027|3|3|10|1|2027|510|6637|Wednesday|2027Q1|N|N|N|2461466|2461524|2461110|2461383|N|N|N|N|N| +2461476|AAAAAAAAECPIFCAA|2027-03-11|1526|6637|510|2027|4|3|11|1|2027|510|6637|Thursday|2027Q1|N|N|N|2461466|2461524|2461111|2461384|N|N|N|N|N| +2461477|AAAAAAAAFCPIFCAA|2027-03-12|1526|6637|510|2027|5|3|12|1|2027|510|6637|Friday|2027Q1|N|Y|N|2461466|2461524|2461112|2461385|N|N|N|N|N| +2461478|AAAAAAAAGCPIFCAA|2027-03-13|1526|6637|510|2027|6|3|13|1|2027|510|6637|Saturday|2027Q1|N|Y|N|2461466|2461524|2461113|2461386|N|N|N|N|N| +2461479|AAAAAAAAHCPIFCAA|2027-03-14|1526|6637|510|2027|0|3|14|1|2027|510|6637|Sunday|2027Q1|N|N|N|2461466|2461524|2461114|2461387|N|N|N|N|N| +2461480|AAAAAAAAICPIFCAA|2027-03-15|1526|6637|510|2027|1|3|15|1|2027|510|6637|Monday|2027Q1|N|N|N|2461466|2461524|2461115|2461388|N|N|N|N|N| +2461481|AAAAAAAAJCPIFCAA|2027-03-16|1526|6638|510|2027|2|3|16|1|2027|510|6638|Tuesday|2027Q1|N|N|N|2461466|2461524|2461116|2461389|N|N|N|N|N| +2461482|AAAAAAAAKCPIFCAA|2027-03-17|1526|6638|510|2027|3|3|17|1|2027|510|6638|Wednesday|2027Q1|N|N|N|2461466|2461524|2461117|2461390|N|N|N|N|N| +2461483|AAAAAAAALCPIFCAA|2027-03-18|1526|6638|510|2027|4|3|18|1|2027|510|6638|Thursday|2027Q1|N|N|N|2461466|2461524|2461118|2461391|N|N|N|N|N| +2461484|AAAAAAAAMCPIFCAA|2027-03-19|1526|6638|510|2027|5|3|19|1|2027|510|6638|Friday|2027Q1|N|Y|N|2461466|2461524|2461119|2461392|N|N|N|N|N| +2461485|AAAAAAAANCPIFCAA|2027-03-20|1526|6638|510|2027|6|3|20|1|2027|510|6638|Saturday|2027Q1|N|Y|N|2461466|2461524|2461120|2461393|N|N|N|N|N| +2461486|AAAAAAAAOCPIFCAA|2027-03-21|1526|6638|510|2027|0|3|21|1|2027|510|6638|Sunday|2027Q1|N|N|N|2461466|2461524|2461121|2461394|N|N|N|N|N| +2461487|AAAAAAAAPCPIFCAA|2027-03-22|1526|6638|510|2027|1|3|22|1|2027|510|6638|Monday|2027Q1|N|N|N|2461466|2461524|2461122|2461395|N|N|N|N|N| +2461488|AAAAAAAAADPIFCAA|2027-03-23|1526|6639|510|2027|2|3|23|1|2027|510|6639|Tuesday|2027Q1|N|N|N|2461466|2461524|2461123|2461396|N|N|N|N|N| +2461489|AAAAAAAABDPIFCAA|2027-03-24|1526|6639|510|2027|3|3|24|1|2027|510|6639|Wednesday|2027Q1|N|N|N|2461466|2461524|2461124|2461397|N|N|N|N|N| +2461490|AAAAAAAACDPIFCAA|2027-03-25|1526|6639|510|2027|4|3|25|1|2027|510|6639|Thursday|2027Q1|N|N|N|2461466|2461524|2461125|2461398|N|N|N|N|N| +2461491|AAAAAAAADDPIFCAA|2027-03-26|1526|6639|510|2027|5|3|26|1|2027|510|6639|Friday|2027Q1|N|Y|N|2461466|2461524|2461126|2461399|N|N|N|N|N| +2461492|AAAAAAAAEDPIFCAA|2027-03-27|1526|6639|510|2027|6|3|27|1|2027|510|6639|Saturday|2027Q1|N|Y|N|2461466|2461524|2461127|2461400|N|N|N|N|N| +2461493|AAAAAAAAFDPIFCAA|2027-03-28|1526|6639|510|2027|0|3|28|1|2027|510|6639|Sunday|2027Q1|N|N|N|2461466|2461524|2461128|2461401|N|N|N|N|N| +2461494|AAAAAAAAGDPIFCAA|2027-03-29|1526|6639|510|2027|1|3|29|1|2027|510|6639|Monday|2027Q1|N|N|N|2461466|2461524|2461129|2461402|N|N|N|N|N| +2461495|AAAAAAAAHDPIFCAA|2027-03-30|1526|6640|510|2027|2|3|30|1|2027|510|6640|Tuesday|2027Q1|N|N|N|2461466|2461524|2461130|2461403|N|N|N|N|N| +2461496|AAAAAAAAIDPIFCAA|2027-03-31|1526|6640|510|2027|3|3|31|1|2027|510|6640|Wednesday|2027Q1|N|N|N|2461466|2461524|2461131|2461404|N|N|N|N|N| +2461497|AAAAAAAAJDPIFCAA|2027-04-01|1527|6640|510|2027|4|4|1|1|2027|510|6640|Thursday|2027Q1|N|N|N|2461497|2461586|2461132|2461407|N|N|N|N|N| +2461498|AAAAAAAAKDPIFCAA|2027-04-02|1527|6640|510|2027|5|4|2|2|2027|510|6640|Friday|2027Q2|N|Y|N|2461497|2461586|2461133|2461408|N|N|N|N|N| +2461499|AAAAAAAALDPIFCAA|2027-04-03|1527|6640|510|2027|6|4|3|2|2027|510|6640|Saturday|2027Q2|N|Y|N|2461497|2461586|2461134|2461409|N|N|N|N|N| +2461500|AAAAAAAAMDPIFCAA|2027-04-04|1527|6640|510|2027|0|4|4|2|2027|510|6640|Sunday|2027Q2|N|N|N|2461497|2461586|2461135|2461410|N|N|N|N|N| +2461501|AAAAAAAANDPIFCAA|2027-04-05|1527|6640|510|2027|1|4|5|2|2027|510|6640|Monday|2027Q2|N|N|N|2461497|2461586|2461136|2461411|N|N|N|N|N| +2461502|AAAAAAAAODPIFCAA|2027-04-06|1527|6641|510|2027|2|4|6|2|2027|510|6641|Tuesday|2027Q2|N|N|N|2461497|2461586|2461137|2461412|N|N|N|N|N| +2461503|AAAAAAAAPDPIFCAA|2027-04-07|1527|6641|510|2027|3|4|7|2|2027|510|6641|Wednesday|2027Q2|N|N|N|2461497|2461586|2461138|2461413|N|N|N|N|N| +2461504|AAAAAAAAAEPIFCAA|2027-04-08|1527|6641|510|2027|4|4|8|2|2027|510|6641|Thursday|2027Q2|N|N|N|2461497|2461586|2461139|2461414|N|N|N|N|N| +2461505|AAAAAAAABEPIFCAA|2027-04-09|1527|6641|510|2027|5|4|9|2|2027|510|6641|Friday|2027Q2|N|Y|N|2461497|2461586|2461140|2461415|N|N|N|N|N| +2461506|AAAAAAAACEPIFCAA|2027-04-10|1527|6641|510|2027|6|4|10|2|2027|510|6641|Saturday|2027Q2|N|Y|N|2461497|2461586|2461141|2461416|N|N|N|N|N| +2461507|AAAAAAAADEPIFCAA|2027-04-11|1527|6641|510|2027|0|4|11|2|2027|510|6641|Sunday|2027Q2|N|N|N|2461497|2461586|2461142|2461417|N|N|N|N|N| +2461508|AAAAAAAAEEPIFCAA|2027-04-12|1527|6641|510|2027|1|4|12|2|2027|510|6641|Monday|2027Q2|N|N|N|2461497|2461586|2461143|2461418|N|N|N|N|N| +2461509|AAAAAAAAFEPIFCAA|2027-04-13|1527|6642|510|2027|2|4|13|2|2027|510|6642|Tuesday|2027Q2|N|N|N|2461497|2461586|2461144|2461419|N|N|N|N|N| +2461510|AAAAAAAAGEPIFCAA|2027-04-14|1527|6642|510|2027|3|4|14|2|2027|510|6642|Wednesday|2027Q2|N|N|N|2461497|2461586|2461145|2461420|N|N|N|N|N| +2461511|AAAAAAAAHEPIFCAA|2027-04-15|1527|6642|510|2027|4|4|15|2|2027|510|6642|Thursday|2027Q2|N|N|N|2461497|2461586|2461146|2461421|N|N|N|N|N| +2461512|AAAAAAAAIEPIFCAA|2027-04-16|1527|6642|510|2027|5|4|16|2|2027|510|6642|Friday|2027Q2|N|Y|N|2461497|2461586|2461147|2461422|N|N|N|N|N| +2461513|AAAAAAAAJEPIFCAA|2027-04-17|1527|6642|510|2027|6|4|17|2|2027|510|6642|Saturday|2027Q2|N|Y|N|2461497|2461586|2461148|2461423|N|N|N|N|N| +2461514|AAAAAAAAKEPIFCAA|2027-04-18|1527|6642|510|2027|0|4|18|2|2027|510|6642|Sunday|2027Q2|N|N|N|2461497|2461586|2461149|2461424|N|N|N|N|N| +2461515|AAAAAAAALEPIFCAA|2027-04-19|1527|6642|510|2027|1|4|19|2|2027|510|6642|Monday|2027Q2|N|N|N|2461497|2461586|2461150|2461425|N|N|N|N|N| +2461516|AAAAAAAAMEPIFCAA|2027-04-20|1527|6643|510|2027|2|4|20|2|2027|510|6643|Tuesday|2027Q2|N|N|N|2461497|2461586|2461151|2461426|N|N|N|N|N| +2461517|AAAAAAAANEPIFCAA|2027-04-21|1527|6643|510|2027|3|4|21|2|2027|510|6643|Wednesday|2027Q2|N|N|N|2461497|2461586|2461152|2461427|N|N|N|N|N| +2461518|AAAAAAAAOEPIFCAA|2027-04-22|1527|6643|510|2027|4|4|22|2|2027|510|6643|Thursday|2027Q2|N|N|N|2461497|2461586|2461153|2461428|N|N|N|N|N| +2461519|AAAAAAAAPEPIFCAA|2027-04-23|1527|6643|510|2027|5|4|23|2|2027|510|6643|Friday|2027Q2|N|Y|N|2461497|2461586|2461154|2461429|N|N|N|N|N| +2461520|AAAAAAAAAFPIFCAA|2027-04-24|1527|6643|510|2027|6|4|24|2|2027|510|6643|Saturday|2027Q2|N|Y|N|2461497|2461586|2461155|2461430|N|N|N|N|N| +2461521|AAAAAAAABFPIFCAA|2027-04-25|1527|6643|510|2027|0|4|25|2|2027|510|6643|Sunday|2027Q2|N|N|N|2461497|2461586|2461156|2461431|N|N|N|N|N| +2461522|AAAAAAAACFPIFCAA|2027-04-26|1527|6643|510|2027|1|4|26|2|2027|510|6643|Monday|2027Q2|N|N|N|2461497|2461586|2461157|2461432|N|N|N|N|N| +2461523|AAAAAAAADFPIFCAA|2027-04-27|1527|6644|510|2027|2|4|27|2|2027|510|6644|Tuesday|2027Q2|N|N|N|2461497|2461586|2461158|2461433|N|N|N|N|N| +2461524|AAAAAAAAEFPIFCAA|2027-04-28|1527|6644|510|2027|3|4|28|2|2027|510|6644|Wednesday|2027Q2|N|N|N|2461497|2461586|2461159|2461434|N|N|N|N|N| +2461525|AAAAAAAAFFPIFCAA|2027-04-29|1527|6644|510|2027|4|4|29|2|2027|510|6644|Thursday|2027Q2|N|N|N|2461497|2461586|2461160|2461435|N|N|N|N|N| +2461526|AAAAAAAAGFPIFCAA|2027-04-30|1527|6644|510|2027|5|4|30|2|2027|510|6644|Friday|2027Q2|N|Y|N|2461497|2461586|2461161|2461436|N|N|N|N|N| +2461527|AAAAAAAAHFPIFCAA|2027-05-01|1528|6644|510|2027|6|5|1|2|2027|510|6644|Saturday|2027Q2|N|Y|N|2461527|2461646|2461162|2461437|N|N|N|N|N| +2461528|AAAAAAAAIFPIFCAA|2027-05-02|1528|6644|510|2027|0|5|2|2|2027|510|6644|Sunday|2027Q2|N|N|N|2461527|2461646|2461163|2461438|N|N|N|N|N| +2461529|AAAAAAAAJFPIFCAA|2027-05-03|1528|6644|510|2027|1|5|3|2|2027|510|6644|Monday|2027Q2|N|N|N|2461527|2461646|2461164|2461439|N|N|N|N|N| +2461530|AAAAAAAAKFPIFCAA|2027-05-04|1528|6645|510|2027|2|5|4|2|2027|510|6645|Tuesday|2027Q2|N|N|N|2461527|2461646|2461165|2461440|N|N|N|N|N| +2461531|AAAAAAAALFPIFCAA|2027-05-05|1528|6645|510|2027|3|5|5|2|2027|510|6645|Wednesday|2027Q2|N|N|N|2461527|2461646|2461166|2461441|N|N|N|N|N| +2461532|AAAAAAAAMFPIFCAA|2027-05-06|1528|6645|510|2027|4|5|6|2|2027|510|6645|Thursday|2027Q2|N|N|N|2461527|2461646|2461167|2461442|N|N|N|N|N| +2461533|AAAAAAAANFPIFCAA|2027-05-07|1528|6645|510|2027|5|5|7|2|2027|510|6645|Friday|2027Q2|N|Y|N|2461527|2461646|2461168|2461443|N|N|N|N|N| +2461534|AAAAAAAAOFPIFCAA|2027-05-08|1528|6645|510|2027|6|5|8|2|2027|510|6645|Saturday|2027Q2|N|Y|N|2461527|2461646|2461169|2461444|N|N|N|N|N| +2461535|AAAAAAAAPFPIFCAA|2027-05-09|1528|6645|510|2027|0|5|9|2|2027|510|6645|Sunday|2027Q2|N|N|N|2461527|2461646|2461170|2461445|N|N|N|N|N| +2461536|AAAAAAAAAGPIFCAA|2027-05-10|1528|6645|510|2027|1|5|10|2|2027|510|6645|Monday|2027Q2|N|N|N|2461527|2461646|2461171|2461446|N|N|N|N|N| +2461537|AAAAAAAABGPIFCAA|2027-05-11|1528|6646|510|2027|2|5|11|2|2027|510|6646|Tuesday|2027Q2|N|N|N|2461527|2461646|2461172|2461447|N|N|N|N|N| +2461538|AAAAAAAACGPIFCAA|2027-05-12|1528|6646|510|2027|3|5|12|2|2027|510|6646|Wednesday|2027Q2|N|N|N|2461527|2461646|2461173|2461448|N|N|N|N|N| +2461539|AAAAAAAADGPIFCAA|2027-05-13|1528|6646|510|2027|4|5|13|2|2027|510|6646|Thursday|2027Q2|N|N|N|2461527|2461646|2461174|2461449|N|N|N|N|N| +2461540|AAAAAAAAEGPIFCAA|2027-05-14|1528|6646|510|2027|5|5|14|2|2027|510|6646|Friday|2027Q2|N|Y|N|2461527|2461646|2461175|2461450|N|N|N|N|N| +2461541|AAAAAAAAFGPIFCAA|2027-05-15|1528|6646|510|2027|6|5|15|2|2027|510|6646|Saturday|2027Q2|N|Y|N|2461527|2461646|2461176|2461451|N|N|N|N|N| +2461542|AAAAAAAAGGPIFCAA|2027-05-16|1528|6646|510|2027|0|5|16|2|2027|510|6646|Sunday|2027Q2|N|N|N|2461527|2461646|2461177|2461452|N|N|N|N|N| +2461543|AAAAAAAAHGPIFCAA|2027-05-17|1528|6646|510|2027|1|5|17|2|2027|510|6646|Monday|2027Q2|N|N|N|2461527|2461646|2461178|2461453|N|N|N|N|N| +2461544|AAAAAAAAIGPIFCAA|2027-05-18|1528|6647|510|2027|2|5|18|2|2027|510|6647|Tuesday|2027Q2|N|N|N|2461527|2461646|2461179|2461454|N|N|N|N|N| +2461545|AAAAAAAAJGPIFCAA|2027-05-19|1528|6647|510|2027|3|5|19|2|2027|510|6647|Wednesday|2027Q2|N|N|N|2461527|2461646|2461180|2461455|N|N|N|N|N| +2461546|AAAAAAAAKGPIFCAA|2027-05-20|1528|6647|510|2027|4|5|20|2|2027|510|6647|Thursday|2027Q2|N|N|N|2461527|2461646|2461181|2461456|N|N|N|N|N| +2461547|AAAAAAAALGPIFCAA|2027-05-21|1528|6647|510|2027|5|5|21|2|2027|510|6647|Friday|2027Q2|N|Y|N|2461527|2461646|2461182|2461457|N|N|N|N|N| +2461548|AAAAAAAAMGPIFCAA|2027-05-22|1528|6647|510|2027|6|5|22|2|2027|510|6647|Saturday|2027Q2|N|Y|N|2461527|2461646|2461183|2461458|N|N|N|N|N| +2461549|AAAAAAAANGPIFCAA|2027-05-23|1528|6647|510|2027|0|5|23|2|2027|510|6647|Sunday|2027Q2|N|N|N|2461527|2461646|2461184|2461459|N|N|N|N|N| +2461550|AAAAAAAAOGPIFCAA|2027-05-24|1528|6647|510|2027|1|5|24|2|2027|510|6647|Monday|2027Q2|N|N|N|2461527|2461646|2461185|2461460|N|N|N|N|N| +2461551|AAAAAAAAPGPIFCAA|2027-05-25|1528|6648|510|2027|2|5|25|2|2027|510|6648|Tuesday|2027Q2|N|N|N|2461527|2461646|2461186|2461461|N|N|N|N|N| +2461552|AAAAAAAAAHPIFCAA|2027-05-26|1528|6648|510|2027|3|5|26|2|2027|510|6648|Wednesday|2027Q2|N|N|N|2461527|2461646|2461187|2461462|N|N|N|N|N| +2461553|AAAAAAAABHPIFCAA|2027-05-27|1528|6648|510|2027|4|5|27|2|2027|510|6648|Thursday|2027Q2|N|N|N|2461527|2461646|2461188|2461463|N|N|N|N|N| +2461554|AAAAAAAACHPIFCAA|2027-05-28|1528|6648|510|2027|5|5|28|2|2027|510|6648|Friday|2027Q2|N|Y|N|2461527|2461646|2461189|2461464|N|N|N|N|N| +2461555|AAAAAAAADHPIFCAA|2027-05-29|1528|6648|510|2027|6|5|29|2|2027|510|6648|Saturday|2027Q2|N|Y|N|2461527|2461646|2461190|2461465|N|N|N|N|N| +2461556|AAAAAAAAEHPIFCAA|2027-05-30|1528|6648|510|2027|0|5|30|2|2027|510|6648|Sunday|2027Q2|N|N|N|2461527|2461646|2461191|2461466|N|N|N|N|N| +2461557|AAAAAAAAFHPIFCAA|2027-05-31|1528|6648|510|2027|1|5|31|2|2027|510|6648|Monday|2027Q2|N|N|N|2461527|2461646|2461192|2461467|N|N|N|N|N| +2461558|AAAAAAAAGHPIFCAA|2027-06-01|1529|6649|511|2027|2|6|1|2|2027|511|6649|Tuesday|2027Q2|N|N|N|2461558|2461708|2461193|2461468|N|N|N|N|N| +2461559|AAAAAAAAHHPIFCAA|2027-06-02|1529|6649|511|2027|3|6|2|2|2027|511|6649|Wednesday|2027Q2|N|N|N|2461558|2461708|2461194|2461469|N|N|N|N|N| +2461560|AAAAAAAAIHPIFCAA|2027-06-03|1529|6649|511|2027|4|6|3|2|2027|511|6649|Thursday|2027Q2|N|N|N|2461558|2461708|2461195|2461470|N|N|N|N|N| +2461561|AAAAAAAAJHPIFCAA|2027-06-04|1529|6649|511|2027|5|6|4|2|2027|511|6649|Friday|2027Q2|N|Y|N|2461558|2461708|2461196|2461471|N|N|N|N|N| +2461562|AAAAAAAAKHPIFCAA|2027-06-05|1529|6649|511|2027|6|6|5|2|2027|511|6649|Saturday|2027Q2|N|Y|N|2461558|2461708|2461197|2461472|N|N|N|N|N| +2461563|AAAAAAAALHPIFCAA|2027-06-06|1529|6649|511|2027|0|6|6|2|2027|511|6649|Sunday|2027Q2|N|N|N|2461558|2461708|2461198|2461473|N|N|N|N|N| +2461564|AAAAAAAAMHPIFCAA|2027-06-07|1529|6649|511|2027|1|6|7|2|2027|511|6649|Monday|2027Q2|N|N|N|2461558|2461708|2461199|2461474|N|N|N|N|N| +2461565|AAAAAAAANHPIFCAA|2027-06-08|1529|6650|511|2027|2|6|8|2|2027|511|6650|Tuesday|2027Q2|N|N|N|2461558|2461708|2461200|2461475|N|N|N|N|N| +2461566|AAAAAAAAOHPIFCAA|2027-06-09|1529|6650|511|2027|3|6|9|2|2027|511|6650|Wednesday|2027Q2|N|N|N|2461558|2461708|2461201|2461476|N|N|N|N|N| +2461567|AAAAAAAAPHPIFCAA|2027-06-10|1529|6650|511|2027|4|6|10|2|2027|511|6650|Thursday|2027Q2|N|N|N|2461558|2461708|2461202|2461477|N|N|N|N|N| +2461568|AAAAAAAAAIPIFCAA|2027-06-11|1529|6650|511|2027|5|6|11|2|2027|511|6650|Friday|2027Q2|N|Y|N|2461558|2461708|2461203|2461478|N|N|N|N|N| +2461569|AAAAAAAABIPIFCAA|2027-06-12|1529|6650|511|2027|6|6|12|2|2027|511|6650|Saturday|2027Q2|N|Y|N|2461558|2461708|2461204|2461479|N|N|N|N|N| +2461570|AAAAAAAACIPIFCAA|2027-06-13|1529|6650|511|2027|0|6|13|2|2027|511|6650|Sunday|2027Q2|N|N|N|2461558|2461708|2461205|2461480|N|N|N|N|N| +2461571|AAAAAAAADIPIFCAA|2027-06-14|1529|6650|511|2027|1|6|14|2|2027|511|6650|Monday|2027Q2|N|N|N|2461558|2461708|2461206|2461481|N|N|N|N|N| +2461572|AAAAAAAAEIPIFCAA|2027-06-15|1529|6651|511|2027|2|6|15|2|2027|511|6651|Tuesday|2027Q2|N|N|N|2461558|2461708|2461207|2461482|N|N|N|N|N| +2461573|AAAAAAAAFIPIFCAA|2027-06-16|1529|6651|511|2027|3|6|16|2|2027|511|6651|Wednesday|2027Q2|N|N|N|2461558|2461708|2461208|2461483|N|N|N|N|N| +2461574|AAAAAAAAGIPIFCAA|2027-06-17|1529|6651|511|2027|4|6|17|2|2027|511|6651|Thursday|2027Q2|N|N|N|2461558|2461708|2461209|2461484|N|N|N|N|N| +2461575|AAAAAAAAHIPIFCAA|2027-06-18|1529|6651|511|2027|5|6|18|2|2027|511|6651|Friday|2027Q2|N|Y|N|2461558|2461708|2461210|2461485|N|N|N|N|N| +2461576|AAAAAAAAIIPIFCAA|2027-06-19|1529|6651|511|2027|6|6|19|2|2027|511|6651|Saturday|2027Q2|N|Y|N|2461558|2461708|2461211|2461486|N|N|N|N|N| +2461577|AAAAAAAAJIPIFCAA|2027-06-20|1529|6651|511|2027|0|6|20|2|2027|511|6651|Sunday|2027Q2|N|N|N|2461558|2461708|2461212|2461487|N|N|N|N|N| +2461578|AAAAAAAAKIPIFCAA|2027-06-21|1529|6651|511|2027|1|6|21|2|2027|511|6651|Monday|2027Q2|N|N|N|2461558|2461708|2461213|2461488|N|N|N|N|N| +2461579|AAAAAAAALIPIFCAA|2027-06-22|1529|6652|511|2027|2|6|22|2|2027|511|6652|Tuesday|2027Q2|N|N|N|2461558|2461708|2461214|2461489|N|N|N|N|N| +2461580|AAAAAAAAMIPIFCAA|2027-06-23|1529|6652|511|2027|3|6|23|2|2027|511|6652|Wednesday|2027Q2|N|N|N|2461558|2461708|2461215|2461490|N|N|N|N|N| +2461581|AAAAAAAANIPIFCAA|2027-06-24|1529|6652|511|2027|4|6|24|2|2027|511|6652|Thursday|2027Q2|N|N|N|2461558|2461708|2461216|2461491|N|N|N|N|N| +2461582|AAAAAAAAOIPIFCAA|2027-06-25|1529|6652|511|2027|5|6|25|2|2027|511|6652|Friday|2027Q2|N|Y|N|2461558|2461708|2461217|2461492|N|N|N|N|N| +2461583|AAAAAAAAPIPIFCAA|2027-06-26|1529|6652|511|2027|6|6|26|2|2027|511|6652|Saturday|2027Q2|N|Y|N|2461558|2461708|2461218|2461493|N|N|N|N|N| +2461584|AAAAAAAAAJPIFCAA|2027-06-27|1529|6652|511|2027|0|6|27|2|2027|511|6652|Sunday|2027Q2|N|N|N|2461558|2461708|2461219|2461494|N|N|N|N|N| +2461585|AAAAAAAABJPIFCAA|2027-06-28|1529|6652|511|2027|1|6|28|2|2027|511|6652|Monday|2027Q2|N|N|N|2461558|2461708|2461220|2461495|N|N|N|N|N| +2461586|AAAAAAAACJPIFCAA|2027-06-29|1529|6653|511|2027|2|6|29|2|2027|511|6653|Tuesday|2027Q2|N|N|N|2461558|2461708|2461221|2461496|N|N|N|N|N| +2461587|AAAAAAAADJPIFCAA|2027-06-30|1529|6653|511|2027|3|6|30|2|2027|511|6653|Wednesday|2027Q2|N|N|N|2461558|2461708|2461222|2461497|N|N|N|N|N| +2461588|AAAAAAAAEJPIFCAA|2027-07-01|1530|6653|511|2027|4|7|1|2|2027|511|6653|Thursday|2027Q2|N|N|N|2461588|2461768|2461223|2461497|N|N|N|N|N| +2461589|AAAAAAAAFJPIFCAA|2027-07-02|1530|6653|511|2027|5|7|2|3|2027|511|6653|Friday|2027Q3|N|Y|N|2461588|2461768|2461224|2461498|N|N|N|N|N| +2461590|AAAAAAAAGJPIFCAA|2027-07-03|1530|6653|511|2027|6|7|3|3|2027|511|6653|Saturday|2027Q3|N|Y|N|2461588|2461768|2461225|2461499|N|N|N|N|N| +2461591|AAAAAAAAHJPIFCAA|2027-07-04|1530|6653|511|2027|0|7|4|3|2027|511|6653|Sunday|2027Q3|N|N|N|2461588|2461768|2461226|2461500|N|N|N|N|N| +2461592|AAAAAAAAIJPIFCAA|2027-07-05|1530|6653|511|2027|1|7|5|3|2027|511|6653|Monday|2027Q3|Y|N|N|2461588|2461768|2461227|2461501|N|N|N|N|N| +2461593|AAAAAAAAJJPIFCAA|2027-07-06|1530|6654|511|2027|2|7|6|3|2027|511|6654|Tuesday|2027Q3|N|N|Y|2461588|2461768|2461228|2461502|N|N|N|N|N| +2461594|AAAAAAAAKJPIFCAA|2027-07-07|1530|6654|511|2027|3|7|7|3|2027|511|6654|Wednesday|2027Q3|N|N|N|2461588|2461768|2461229|2461503|N|N|N|N|N| +2461595|AAAAAAAALJPIFCAA|2027-07-08|1530|6654|511|2027|4|7|8|3|2027|511|6654|Thursday|2027Q3|N|N|N|2461588|2461768|2461230|2461504|N|N|N|N|N| +2461596|AAAAAAAAMJPIFCAA|2027-07-09|1530|6654|511|2027|5|7|9|3|2027|511|6654|Friday|2027Q3|N|Y|N|2461588|2461768|2461231|2461505|N|N|N|N|N| +2461597|AAAAAAAANJPIFCAA|2027-07-10|1530|6654|511|2027|6|7|10|3|2027|511|6654|Saturday|2027Q3|N|Y|N|2461588|2461768|2461232|2461506|N|N|N|N|N| +2461598|AAAAAAAAOJPIFCAA|2027-07-11|1530|6654|511|2027|0|7|11|3|2027|511|6654|Sunday|2027Q3|N|N|N|2461588|2461768|2461233|2461507|N|N|N|N|N| +2461599|AAAAAAAAPJPIFCAA|2027-07-12|1530|6654|511|2027|1|7|12|3|2027|511|6654|Monday|2027Q3|N|N|N|2461588|2461768|2461234|2461508|N|N|N|N|N| +2461600|AAAAAAAAAKPIFCAA|2027-07-13|1530|6655|511|2027|2|7|13|3|2027|511|6655|Tuesday|2027Q3|N|N|N|2461588|2461768|2461235|2461509|N|N|N|N|N| +2461601|AAAAAAAABKPIFCAA|2027-07-14|1530|6655|511|2027|3|7|14|3|2027|511|6655|Wednesday|2027Q3|N|N|N|2461588|2461768|2461236|2461510|N|N|N|N|N| +2461602|AAAAAAAACKPIFCAA|2027-07-15|1530|6655|511|2027|4|7|15|3|2027|511|6655|Thursday|2027Q3|N|N|N|2461588|2461768|2461237|2461511|N|N|N|N|N| +2461603|AAAAAAAADKPIFCAA|2027-07-16|1530|6655|511|2027|5|7|16|3|2027|511|6655|Friday|2027Q3|N|Y|N|2461588|2461768|2461238|2461512|N|N|N|N|N| +2461604|AAAAAAAAEKPIFCAA|2027-07-17|1530|6655|511|2027|6|7|17|3|2027|511|6655|Saturday|2027Q3|N|Y|N|2461588|2461768|2461239|2461513|N|N|N|N|N| +2461605|AAAAAAAAFKPIFCAA|2027-07-18|1530|6655|511|2027|0|7|18|3|2027|511|6655|Sunday|2027Q3|N|N|N|2461588|2461768|2461240|2461514|N|N|N|N|N| +2461606|AAAAAAAAGKPIFCAA|2027-07-19|1530|6655|511|2027|1|7|19|3|2027|511|6655|Monday|2027Q3|N|N|N|2461588|2461768|2461241|2461515|N|N|N|N|N| +2461607|AAAAAAAAHKPIFCAA|2027-07-20|1530|6656|511|2027|2|7|20|3|2027|511|6656|Tuesday|2027Q3|N|N|N|2461588|2461768|2461242|2461516|N|N|N|N|N| +2461608|AAAAAAAAIKPIFCAA|2027-07-21|1530|6656|511|2027|3|7|21|3|2027|511|6656|Wednesday|2027Q3|N|N|N|2461588|2461768|2461243|2461517|N|N|N|N|N| +2461609|AAAAAAAAJKPIFCAA|2027-07-22|1530|6656|511|2027|4|7|22|3|2027|511|6656|Thursday|2027Q3|N|N|N|2461588|2461768|2461244|2461518|N|N|N|N|N| +2461610|AAAAAAAAKKPIFCAA|2027-07-23|1530|6656|511|2027|5|7|23|3|2027|511|6656|Friday|2027Q3|N|Y|N|2461588|2461768|2461245|2461519|N|N|N|N|N| +2461611|AAAAAAAALKPIFCAA|2027-07-24|1530|6656|511|2027|6|7|24|3|2027|511|6656|Saturday|2027Q3|N|Y|N|2461588|2461768|2461246|2461520|N|N|N|N|N| +2461612|AAAAAAAAMKPIFCAA|2027-07-25|1530|6656|511|2027|0|7|25|3|2027|511|6656|Sunday|2027Q3|N|N|N|2461588|2461768|2461247|2461521|N|N|N|N|N| +2461613|AAAAAAAANKPIFCAA|2027-07-26|1530|6656|511|2027|1|7|26|3|2027|511|6656|Monday|2027Q3|N|N|N|2461588|2461768|2461248|2461522|N|N|N|N|N| +2461614|AAAAAAAAOKPIFCAA|2027-07-27|1530|6657|511|2027|2|7|27|3|2027|511|6657|Tuesday|2027Q3|N|N|N|2461588|2461768|2461249|2461523|N|N|N|N|N| +2461615|AAAAAAAAPKPIFCAA|2027-07-28|1530|6657|511|2027|3|7|28|3|2027|511|6657|Wednesday|2027Q3|N|N|N|2461588|2461768|2461250|2461524|N|N|N|N|N| +2461616|AAAAAAAAALPIFCAA|2027-07-29|1530|6657|511|2027|4|7|29|3|2027|511|6657|Thursday|2027Q3|N|N|N|2461588|2461768|2461251|2461525|N|N|N|N|N| +2461617|AAAAAAAABLPIFCAA|2027-07-30|1530|6657|511|2027|5|7|30|3|2027|511|6657|Friday|2027Q3|N|Y|N|2461588|2461768|2461252|2461526|N|N|N|N|N| +2461618|AAAAAAAACLPIFCAA|2027-07-31|1530|6657|511|2027|6|7|31|3|2027|511|6657|Saturday|2027Q3|N|Y|N|2461588|2461768|2461253|2461527|N|N|N|N|N| +2461619|AAAAAAAADLPIFCAA|2027-08-01|1531|6657|511|2027|0|8|1|3|2027|511|6657|Sunday|2027Q3|N|N|N|2461619|2461830|2461254|2461528|N|N|N|N|N| +2461620|AAAAAAAAELPIFCAA|2027-08-02|1531|6657|511|2027|1|8|2|3|2027|511|6657|Monday|2027Q3|N|N|N|2461619|2461830|2461255|2461529|N|N|N|N|N| +2461621|AAAAAAAAFLPIFCAA|2027-08-03|1531|6658|511|2027|2|8|3|3|2027|511|6658|Tuesday|2027Q3|N|N|N|2461619|2461830|2461256|2461530|N|N|N|N|N| +2461622|AAAAAAAAGLPIFCAA|2027-08-04|1531|6658|511|2027|3|8|4|3|2027|511|6658|Wednesday|2027Q3|N|N|N|2461619|2461830|2461257|2461531|N|N|N|N|N| +2461623|AAAAAAAAHLPIFCAA|2027-08-05|1531|6658|511|2027|4|8|5|3|2027|511|6658|Thursday|2027Q3|N|N|N|2461619|2461830|2461258|2461532|N|N|N|N|N| +2461624|AAAAAAAAILPIFCAA|2027-08-06|1531|6658|511|2027|5|8|6|3|2027|511|6658|Friday|2027Q3|N|Y|N|2461619|2461830|2461259|2461533|N|N|N|N|N| +2461625|AAAAAAAAJLPIFCAA|2027-08-07|1531|6658|511|2027|6|8|7|3|2027|511|6658|Saturday|2027Q3|N|Y|N|2461619|2461830|2461260|2461534|N|N|N|N|N| +2461626|AAAAAAAAKLPIFCAA|2027-08-08|1531|6658|511|2027|0|8|8|3|2027|511|6658|Sunday|2027Q3|N|N|N|2461619|2461830|2461261|2461535|N|N|N|N|N| +2461627|AAAAAAAALLPIFCAA|2027-08-09|1531|6658|511|2027|1|8|9|3|2027|511|6658|Monday|2027Q3|N|N|N|2461619|2461830|2461262|2461536|N|N|N|N|N| +2461628|AAAAAAAAMLPIFCAA|2027-08-10|1531|6659|511|2027|2|8|10|3|2027|511|6659|Tuesday|2027Q3|N|N|N|2461619|2461830|2461263|2461537|N|N|N|N|N| +2461629|AAAAAAAANLPIFCAA|2027-08-11|1531|6659|511|2027|3|8|11|3|2027|511|6659|Wednesday|2027Q3|N|N|N|2461619|2461830|2461264|2461538|N|N|N|N|N| +2461630|AAAAAAAAOLPIFCAA|2027-08-12|1531|6659|511|2027|4|8|12|3|2027|511|6659|Thursday|2027Q3|N|N|N|2461619|2461830|2461265|2461539|N|N|N|N|N| +2461631|AAAAAAAAPLPIFCAA|2027-08-13|1531|6659|511|2027|5|8|13|3|2027|511|6659|Friday|2027Q3|N|Y|N|2461619|2461830|2461266|2461540|N|N|N|N|N| +2461632|AAAAAAAAAMPIFCAA|2027-08-14|1531|6659|511|2027|6|8|14|3|2027|511|6659|Saturday|2027Q3|N|Y|N|2461619|2461830|2461267|2461541|N|N|N|N|N| +2461633|AAAAAAAABMPIFCAA|2027-08-15|1531|6659|511|2027|0|8|15|3|2027|511|6659|Sunday|2027Q3|N|N|N|2461619|2461830|2461268|2461542|N|N|N|N|N| +2461634|AAAAAAAACMPIFCAA|2027-08-16|1531|6659|511|2027|1|8|16|3|2027|511|6659|Monday|2027Q3|N|N|N|2461619|2461830|2461269|2461543|N|N|N|N|N| +2461635|AAAAAAAADMPIFCAA|2027-08-17|1531|6660|511|2027|2|8|17|3|2027|511|6660|Tuesday|2027Q3|N|N|N|2461619|2461830|2461270|2461544|N|N|N|N|N| +2461636|AAAAAAAAEMPIFCAA|2027-08-18|1531|6660|511|2027|3|8|18|3|2027|511|6660|Wednesday|2027Q3|N|N|N|2461619|2461830|2461271|2461545|N|N|N|N|N| +2461637|AAAAAAAAFMPIFCAA|2027-08-19|1531|6660|511|2027|4|8|19|3|2027|511|6660|Thursday|2027Q3|N|N|N|2461619|2461830|2461272|2461546|N|N|N|N|N| +2461638|AAAAAAAAGMPIFCAA|2027-08-20|1531|6660|511|2027|5|8|20|3|2027|511|6660|Friday|2027Q3|N|Y|N|2461619|2461830|2461273|2461547|N|N|N|N|N| +2461639|AAAAAAAAHMPIFCAA|2027-08-21|1531|6660|511|2027|6|8|21|3|2027|511|6660|Saturday|2027Q3|N|Y|N|2461619|2461830|2461274|2461548|N|N|N|N|N| +2461640|AAAAAAAAIMPIFCAA|2027-08-22|1531|6660|511|2027|0|8|22|3|2027|511|6660|Sunday|2027Q3|N|N|N|2461619|2461830|2461275|2461549|N|N|N|N|N| +2461641|AAAAAAAAJMPIFCAA|2027-08-23|1531|6660|511|2027|1|8|23|3|2027|511|6660|Monday|2027Q3|N|N|N|2461619|2461830|2461276|2461550|N|N|N|N|N| +2461642|AAAAAAAAKMPIFCAA|2027-08-24|1531|6661|511|2027|2|8|24|3|2027|511|6661|Tuesday|2027Q3|N|N|N|2461619|2461830|2461277|2461551|N|N|N|N|N| +2461643|AAAAAAAALMPIFCAA|2027-08-25|1531|6661|511|2027|3|8|25|3|2027|511|6661|Wednesday|2027Q3|N|N|N|2461619|2461830|2461278|2461552|N|N|N|N|N| +2461644|AAAAAAAAMMPIFCAA|2027-08-26|1531|6661|511|2027|4|8|26|3|2027|511|6661|Thursday|2027Q3|N|N|N|2461619|2461830|2461279|2461553|N|N|N|N|N| +2461645|AAAAAAAANMPIFCAA|2027-08-27|1531|6661|511|2027|5|8|27|3|2027|511|6661|Friday|2027Q3|N|Y|N|2461619|2461830|2461280|2461554|N|N|N|N|N| +2461646|AAAAAAAAOMPIFCAA|2027-08-28|1531|6661|511|2027|6|8|28|3|2027|511|6661|Saturday|2027Q3|N|Y|N|2461619|2461830|2461281|2461555|N|N|N|N|N| +2461647|AAAAAAAAPMPIFCAA|2027-08-29|1531|6661|511|2027|0|8|29|3|2027|511|6661|Sunday|2027Q3|N|N|N|2461619|2461830|2461282|2461556|N|N|N|N|N| +2461648|AAAAAAAAANPIFCAA|2027-08-30|1531|6661|511|2027|1|8|30|3|2027|511|6661|Monday|2027Q3|N|N|N|2461619|2461830|2461283|2461557|N|N|N|N|N| +2461649|AAAAAAAABNPIFCAA|2027-08-31|1531|6662|511|2027|2|8|31|3|2027|511|6662|Tuesday|2027Q3|N|N|N|2461619|2461830|2461284|2461558|N|N|N|N|N| +2461650|AAAAAAAACNPIFCAA|2027-09-01|1532|6662|512|2027|3|9|1|3|2027|512|6662|Wednesday|2027Q3|N|N|N|2461650|2461892|2461285|2461559|N|N|N|N|N| +2461651|AAAAAAAADNPIFCAA|2027-09-02|1532|6662|512|2027|4|9|2|3|2027|512|6662|Thursday|2027Q3|N|N|N|2461650|2461892|2461286|2461560|N|N|N|N|N| +2461652|AAAAAAAAENPIFCAA|2027-09-03|1532|6662|512|2027|5|9|3|3|2027|512|6662|Friday|2027Q3|N|Y|N|2461650|2461892|2461287|2461561|N|N|N|N|N| +2461653|AAAAAAAAFNPIFCAA|2027-09-04|1532|6662|512|2027|6|9|4|3|2027|512|6662|Saturday|2027Q3|N|Y|N|2461650|2461892|2461288|2461562|N|N|N|N|N| +2461654|AAAAAAAAGNPIFCAA|2027-09-05|1532|6662|512|2027|0|9|5|3|2027|512|6662|Sunday|2027Q3|N|N|N|2461650|2461892|2461289|2461563|N|N|N|N|N| +2461655|AAAAAAAAHNPIFCAA|2027-09-06|1532|6662|512|2027|1|9|6|3|2027|512|6662|Monday|2027Q3|N|N|N|2461650|2461892|2461290|2461564|N|N|N|N|N| +2461656|AAAAAAAAINPIFCAA|2027-09-07|1532|6663|512|2027|2|9|7|3|2027|512|6663|Tuesday|2027Q3|N|N|N|2461650|2461892|2461291|2461565|N|N|N|N|N| +2461657|AAAAAAAAJNPIFCAA|2027-09-08|1532|6663|512|2027|3|9|8|3|2027|512|6663|Wednesday|2027Q3|N|N|N|2461650|2461892|2461292|2461566|N|N|N|N|N| +2461658|AAAAAAAAKNPIFCAA|2027-09-09|1532|6663|512|2027|4|9|9|3|2027|512|6663|Thursday|2027Q3|N|N|N|2461650|2461892|2461293|2461567|N|N|N|N|N| +2461659|AAAAAAAALNPIFCAA|2027-09-10|1532|6663|512|2027|5|9|10|3|2027|512|6663|Friday|2027Q3|N|Y|N|2461650|2461892|2461294|2461568|N|N|N|N|N| +2461660|AAAAAAAAMNPIFCAA|2027-09-11|1532|6663|512|2027|6|9|11|3|2027|512|6663|Saturday|2027Q3|N|Y|N|2461650|2461892|2461295|2461569|N|N|N|N|N| +2461661|AAAAAAAANNPIFCAA|2027-09-12|1532|6663|512|2027|0|9|12|3|2027|512|6663|Sunday|2027Q3|N|N|N|2461650|2461892|2461296|2461570|N|N|N|N|N| +2461662|AAAAAAAAONPIFCAA|2027-09-13|1532|6663|512|2027|1|9|13|3|2027|512|6663|Monday|2027Q3|N|N|N|2461650|2461892|2461297|2461571|N|N|N|N|N| +2461663|AAAAAAAAPNPIFCAA|2027-09-14|1532|6664|512|2027|2|9|14|3|2027|512|6664|Tuesday|2027Q3|N|N|N|2461650|2461892|2461298|2461572|N|N|N|N|N| +2461664|AAAAAAAAAOPIFCAA|2027-09-15|1532|6664|512|2027|3|9|15|3|2027|512|6664|Wednesday|2027Q3|N|N|N|2461650|2461892|2461299|2461573|N|N|N|N|N| +2461665|AAAAAAAABOPIFCAA|2027-09-16|1532|6664|512|2027|4|9|16|3|2027|512|6664|Thursday|2027Q3|N|N|N|2461650|2461892|2461300|2461574|N|N|N|N|N| +2461666|AAAAAAAACOPIFCAA|2027-09-17|1532|6664|512|2027|5|9|17|3|2027|512|6664|Friday|2027Q3|N|Y|N|2461650|2461892|2461301|2461575|N|N|N|N|N| +2461667|AAAAAAAADOPIFCAA|2027-09-18|1532|6664|512|2027|6|9|18|3|2027|512|6664|Saturday|2027Q3|N|Y|N|2461650|2461892|2461302|2461576|N|N|N|N|N| +2461668|AAAAAAAAEOPIFCAA|2027-09-19|1532|6664|512|2027|0|9|19|3|2027|512|6664|Sunday|2027Q3|N|N|N|2461650|2461892|2461303|2461577|N|N|N|N|N| +2461669|AAAAAAAAFOPIFCAA|2027-09-20|1532|6664|512|2027|1|9|20|3|2027|512|6664|Monday|2027Q3|N|N|N|2461650|2461892|2461304|2461578|N|N|N|N|N| +2461670|AAAAAAAAGOPIFCAA|2027-09-21|1532|6665|512|2027|2|9|21|3|2027|512|6665|Tuesday|2027Q3|N|N|N|2461650|2461892|2461305|2461579|N|N|N|N|N| +2461671|AAAAAAAAHOPIFCAA|2027-09-22|1532|6665|512|2027|3|9|22|3|2027|512|6665|Wednesday|2027Q3|N|N|N|2461650|2461892|2461306|2461580|N|N|N|N|N| +2461672|AAAAAAAAIOPIFCAA|2027-09-23|1532|6665|512|2027|4|9|23|3|2027|512|6665|Thursday|2027Q3|N|N|N|2461650|2461892|2461307|2461581|N|N|N|N|N| +2461673|AAAAAAAAJOPIFCAA|2027-09-24|1532|6665|512|2027|5|9|24|3|2027|512|6665|Friday|2027Q3|N|Y|N|2461650|2461892|2461308|2461582|N|N|N|N|N| +2461674|AAAAAAAAKOPIFCAA|2027-09-25|1532|6665|512|2027|6|9|25|3|2027|512|6665|Saturday|2027Q3|N|Y|N|2461650|2461892|2461309|2461583|N|N|N|N|N| +2461675|AAAAAAAALOPIFCAA|2027-09-26|1532|6665|512|2027|0|9|26|3|2027|512|6665|Sunday|2027Q3|N|N|N|2461650|2461892|2461310|2461584|N|N|N|N|N| +2461676|AAAAAAAAMOPIFCAA|2027-09-27|1532|6665|512|2027|1|9|27|3|2027|512|6665|Monday|2027Q3|N|N|N|2461650|2461892|2461311|2461585|N|N|N|N|N| +2461677|AAAAAAAANOPIFCAA|2027-09-28|1532|6666|512|2027|2|9|28|3|2027|512|6666|Tuesday|2027Q3|N|N|N|2461650|2461892|2461312|2461586|N|N|N|N|N| +2461678|AAAAAAAAOOPIFCAA|2027-09-29|1532|6666|512|2027|3|9|29|3|2027|512|6666|Wednesday|2027Q3|N|N|N|2461650|2461892|2461313|2461587|N|N|N|N|N| +2461679|AAAAAAAAPOPIFCAA|2027-09-30|1532|6666|512|2027|4|9|30|3|2027|512|6666|Thursday|2027Q3|N|N|N|2461650|2461892|2461314|2461588|N|N|N|N|N| +2461680|AAAAAAAAAPPIFCAA|2027-10-01|1533|6666|512|2027|5|10|1|3|2027|512|6666|Friday|2027Q3|N|Y|N|2461680|2461952|2461315|2461588|N|N|N|N|N| +2461681|AAAAAAAABPPIFCAA|2027-10-02|1533|6666|512|2027|6|10|2|4|2027|512|6666|Saturday|2027Q4|N|Y|N|2461680|2461952|2461316|2461589|N|N|N|N|N| +2461682|AAAAAAAACPPIFCAA|2027-10-03|1533|6666|512|2027|0|10|3|4|2027|512|6666|Sunday|2027Q4|N|N|N|2461680|2461952|2461317|2461590|N|N|N|N|N| +2461683|AAAAAAAADPPIFCAA|2027-10-04|1533|6666|512|2027|1|10|4|4|2027|512|6666|Monday|2027Q4|N|N|N|2461680|2461952|2461318|2461591|N|N|N|N|N| +2461684|AAAAAAAAEPPIFCAA|2027-10-05|1533|6667|512|2027|2|10|5|4|2027|512|6667|Tuesday|2027Q4|N|N|N|2461680|2461952|2461319|2461592|N|N|N|N|N| +2461685|AAAAAAAAFPPIFCAA|2027-10-06|1533|6667|512|2027|3|10|6|4|2027|512|6667|Wednesday|2027Q4|N|N|N|2461680|2461952|2461320|2461593|N|N|N|N|N| +2461686|AAAAAAAAGPPIFCAA|2027-10-07|1533|6667|512|2027|4|10|7|4|2027|512|6667|Thursday|2027Q4|N|N|N|2461680|2461952|2461321|2461594|N|N|N|N|N| +2461687|AAAAAAAAHPPIFCAA|2027-10-08|1533|6667|512|2027|5|10|8|4|2027|512|6667|Friday|2027Q4|N|Y|N|2461680|2461952|2461322|2461595|N|N|N|N|N| +2461688|AAAAAAAAIPPIFCAA|2027-10-09|1533|6667|512|2027|6|10|9|4|2027|512|6667|Saturday|2027Q4|N|Y|N|2461680|2461952|2461323|2461596|N|N|N|N|N| +2461689|AAAAAAAAJPPIFCAA|2027-10-10|1533|6667|512|2027|0|10|10|4|2027|512|6667|Sunday|2027Q4|N|N|N|2461680|2461952|2461324|2461597|N|N|N|N|N| +2461690|AAAAAAAAKPPIFCAA|2027-10-11|1533|6667|512|2027|1|10|11|4|2027|512|6667|Monday|2027Q4|N|N|N|2461680|2461952|2461325|2461598|N|N|N|N|N| +2461691|AAAAAAAALPPIFCAA|2027-10-12|1533|6668|512|2027|2|10|12|4|2027|512|6668|Tuesday|2027Q4|N|N|N|2461680|2461952|2461326|2461599|N|N|N|N|N| +2461692|AAAAAAAAMPPIFCAA|2027-10-13|1533|6668|512|2027|3|10|13|4|2027|512|6668|Wednesday|2027Q4|N|N|N|2461680|2461952|2461327|2461600|N|N|N|N|N| +2461693|AAAAAAAANPPIFCAA|2027-10-14|1533|6668|512|2027|4|10|14|4|2027|512|6668|Thursday|2027Q4|N|N|N|2461680|2461952|2461328|2461601|N|N|N|N|N| +2461694|AAAAAAAAOPPIFCAA|2027-10-15|1533|6668|512|2027|5|10|15|4|2027|512|6668|Friday|2027Q4|N|Y|N|2461680|2461952|2461329|2461602|N|N|N|N|N| +2461695|AAAAAAAAPPPIFCAA|2027-10-16|1533|6668|512|2027|6|10|16|4|2027|512|6668|Saturday|2027Q4|N|Y|N|2461680|2461952|2461330|2461603|N|N|N|N|N| +2461696|AAAAAAAAAAAJFCAA|2027-10-17|1533|6668|512|2027|0|10|17|4|2027|512|6668|Sunday|2027Q4|N|N|N|2461680|2461952|2461331|2461604|N|N|N|N|N| +2461697|AAAAAAAABAAJFCAA|2027-10-18|1533|6668|512|2027|1|10|18|4|2027|512|6668|Monday|2027Q4|N|N|N|2461680|2461952|2461332|2461605|N|N|N|N|N| +2461698|AAAAAAAACAAJFCAA|2027-10-19|1533|6669|512|2027|2|10|19|4|2027|512|6669|Tuesday|2027Q4|N|N|N|2461680|2461952|2461333|2461606|N|N|N|N|N| +2461699|AAAAAAAADAAJFCAA|2027-10-20|1533|6669|512|2027|3|10|20|4|2027|512|6669|Wednesday|2027Q4|N|N|N|2461680|2461952|2461334|2461607|N|N|N|N|N| +2461700|AAAAAAAAEAAJFCAA|2027-10-21|1533|6669|512|2027|4|10|21|4|2027|512|6669|Thursday|2027Q4|N|N|N|2461680|2461952|2461335|2461608|N|N|N|N|N| +2461701|AAAAAAAAFAAJFCAA|2027-10-22|1533|6669|512|2027|5|10|22|4|2027|512|6669|Friday|2027Q4|N|Y|N|2461680|2461952|2461336|2461609|N|N|N|N|N| +2461702|AAAAAAAAGAAJFCAA|2027-10-23|1533|6669|512|2027|6|10|23|4|2027|512|6669|Saturday|2027Q4|N|Y|N|2461680|2461952|2461337|2461610|N|N|N|N|N| +2461703|AAAAAAAAHAAJFCAA|2027-10-24|1533|6669|512|2027|0|10|24|4|2027|512|6669|Sunday|2027Q4|N|N|N|2461680|2461952|2461338|2461611|N|N|N|N|N| +2461704|AAAAAAAAIAAJFCAA|2027-10-25|1533|6669|512|2027|1|10|25|4|2027|512|6669|Monday|2027Q4|N|N|N|2461680|2461952|2461339|2461612|N|N|N|N|N| +2461705|AAAAAAAAJAAJFCAA|2027-10-26|1533|6670|512|2027|2|10|26|4|2027|512|6670|Tuesday|2027Q4|N|N|N|2461680|2461952|2461340|2461613|N|N|N|N|N| +2461706|AAAAAAAAKAAJFCAA|2027-10-27|1533|6670|512|2027|3|10|27|4|2027|512|6670|Wednesday|2027Q4|N|N|N|2461680|2461952|2461341|2461614|N|N|N|N|N| +2461707|AAAAAAAALAAJFCAA|2027-10-28|1533|6670|512|2027|4|10|28|4|2027|512|6670|Thursday|2027Q4|N|N|N|2461680|2461952|2461342|2461615|N|N|N|N|N| +2461708|AAAAAAAAMAAJFCAA|2027-10-29|1533|6670|512|2027|5|10|29|4|2027|512|6670|Friday|2027Q4|N|Y|N|2461680|2461952|2461343|2461616|N|N|N|N|N| +2461709|AAAAAAAANAAJFCAA|2027-10-30|1533|6670|512|2027|6|10|30|4|2027|512|6670|Saturday|2027Q4|N|Y|N|2461680|2461952|2461344|2461617|N|N|N|N|N| +2461710|AAAAAAAAOAAJFCAA|2027-10-31|1533|6670|512|2027|0|10|31|4|2027|512|6670|Sunday|2027Q4|N|N|N|2461680|2461952|2461345|2461618|N|N|N|N|N| +2461711|AAAAAAAAPAAJFCAA|2027-11-01|1534|6670|512|2027|1|11|1|4|2027|512|6670|Monday|2027Q4|N|N|N|2461711|2462014|2461346|2461619|N|N|N|N|N| +2461712|AAAAAAAAABAJFCAA|2027-11-02|1534|6671|512|2027|2|11|2|4|2027|512|6671|Tuesday|2027Q4|N|N|N|2461711|2462014|2461347|2461620|N|N|N|N|N| +2461713|AAAAAAAABBAJFCAA|2027-11-03|1534|6671|512|2027|3|11|3|4|2027|512|6671|Wednesday|2027Q4|N|N|N|2461711|2462014|2461348|2461621|N|N|N|N|N| +2461714|AAAAAAAACBAJFCAA|2027-11-04|1534|6671|512|2027|4|11|4|4|2027|512|6671|Thursday|2027Q4|N|N|N|2461711|2462014|2461349|2461622|N|N|N|N|N| +2461715|AAAAAAAADBAJFCAA|2027-11-05|1534|6671|512|2027|5|11|5|4|2027|512|6671|Friday|2027Q4|N|Y|N|2461711|2462014|2461350|2461623|N|N|N|N|N| +2461716|AAAAAAAAEBAJFCAA|2027-11-06|1534|6671|512|2027|6|11|6|4|2027|512|6671|Saturday|2027Q4|N|Y|N|2461711|2462014|2461351|2461624|N|N|N|N|N| +2461717|AAAAAAAAFBAJFCAA|2027-11-07|1534|6671|512|2027|0|11|7|4|2027|512|6671|Sunday|2027Q4|N|N|N|2461711|2462014|2461352|2461625|N|N|N|N|N| +2461718|AAAAAAAAGBAJFCAA|2027-11-08|1534|6671|512|2027|1|11|8|4|2027|512|6671|Monday|2027Q4|N|N|N|2461711|2462014|2461353|2461626|N|N|N|N|N| +2461719|AAAAAAAAHBAJFCAA|2027-11-09|1534|6672|512|2027|2|11|9|4|2027|512|6672|Tuesday|2027Q4|N|N|N|2461711|2462014|2461354|2461627|N|N|N|N|N| +2461720|AAAAAAAAIBAJFCAA|2027-11-10|1534|6672|512|2027|3|11|10|4|2027|512|6672|Wednesday|2027Q4|N|N|N|2461711|2462014|2461355|2461628|N|N|N|N|N| +2461721|AAAAAAAAJBAJFCAA|2027-11-11|1534|6672|512|2027|4|11|11|4|2027|512|6672|Thursday|2027Q4|N|N|N|2461711|2462014|2461356|2461629|N|N|N|N|N| +2461722|AAAAAAAAKBAJFCAA|2027-11-12|1534|6672|512|2027|5|11|12|4|2027|512|6672|Friday|2027Q4|N|Y|N|2461711|2462014|2461357|2461630|N|N|N|N|N| +2461723|AAAAAAAALBAJFCAA|2027-11-13|1534|6672|512|2027|6|11|13|4|2027|512|6672|Saturday|2027Q4|N|Y|N|2461711|2462014|2461358|2461631|N|N|N|N|N| +2461724|AAAAAAAAMBAJFCAA|2027-11-14|1534|6672|512|2027|0|11|14|4|2027|512|6672|Sunday|2027Q4|N|N|N|2461711|2462014|2461359|2461632|N|N|N|N|N| +2461725|AAAAAAAANBAJFCAA|2027-11-15|1534|6672|512|2027|1|11|15|4|2027|512|6672|Monday|2027Q4|N|N|N|2461711|2462014|2461360|2461633|N|N|N|N|N| +2461726|AAAAAAAAOBAJFCAA|2027-11-16|1534|6673|512|2027|2|11|16|4|2027|512|6673|Tuesday|2027Q4|N|N|N|2461711|2462014|2461361|2461634|N|N|N|N|N| +2461727|AAAAAAAAPBAJFCAA|2027-11-17|1534|6673|512|2027|3|11|17|4|2027|512|6673|Wednesday|2027Q4|N|N|N|2461711|2462014|2461362|2461635|N|N|N|N|N| +2461728|AAAAAAAAACAJFCAA|2027-11-18|1534|6673|512|2027|4|11|18|4|2027|512|6673|Thursday|2027Q4|N|N|N|2461711|2462014|2461363|2461636|N|N|N|N|N| +2461729|AAAAAAAABCAJFCAA|2027-11-19|1534|6673|512|2027|5|11|19|4|2027|512|6673|Friday|2027Q4|N|Y|N|2461711|2462014|2461364|2461637|N|N|N|N|N| +2461730|AAAAAAAACCAJFCAA|2027-11-20|1534|6673|512|2027|6|11|20|4|2027|512|6673|Saturday|2027Q4|N|Y|N|2461711|2462014|2461365|2461638|N|N|N|N|N| +2461731|AAAAAAAADCAJFCAA|2027-11-21|1534|6673|512|2027|0|11|21|4|2027|512|6673|Sunday|2027Q4|N|N|N|2461711|2462014|2461366|2461639|N|N|N|N|N| +2461732|AAAAAAAAECAJFCAA|2027-11-22|1534|6673|512|2027|1|11|22|4|2027|512|6673|Monday|2027Q4|N|N|N|2461711|2462014|2461367|2461640|N|N|N|N|N| +2461733|AAAAAAAAFCAJFCAA|2027-11-23|1534|6674|512|2027|2|11|23|4|2027|512|6674|Tuesday|2027Q4|N|N|N|2461711|2462014|2461368|2461641|N|N|N|N|N| +2461734|AAAAAAAAGCAJFCAA|2027-11-24|1534|6674|512|2027|3|11|24|4|2027|512|6674|Wednesday|2027Q4|N|N|N|2461711|2462014|2461369|2461642|N|N|N|N|N| +2461735|AAAAAAAAHCAJFCAA|2027-11-25|1534|6674|512|2027|4|11|25|4|2027|512|6674|Thursday|2027Q4|N|N|N|2461711|2462014|2461370|2461643|N|N|N|N|N| +2461736|AAAAAAAAICAJFCAA|2027-11-26|1534|6674|512|2027|5|11|26|4|2027|512|6674|Friday|2027Q4|N|Y|N|2461711|2462014|2461371|2461644|N|N|N|N|N| +2461737|AAAAAAAAJCAJFCAA|2027-11-27|1534|6674|512|2027|6|11|27|4|2027|512|6674|Saturday|2027Q4|N|Y|N|2461711|2462014|2461372|2461645|N|N|N|N|N| +2461738|AAAAAAAAKCAJFCAA|2027-11-28|1534|6674|512|2027|0|11|28|4|2027|512|6674|Sunday|2027Q4|N|N|N|2461711|2462014|2461373|2461646|N|N|N|N|N| +2461739|AAAAAAAALCAJFCAA|2027-11-29|1534|6674|512|2027|1|11|29|4|2027|512|6674|Monday|2027Q4|N|N|N|2461711|2462014|2461374|2461647|N|N|N|N|N| +2461740|AAAAAAAAMCAJFCAA|2027-11-30|1534|6675|512|2027|2|11|30|4|2027|512|6675|Tuesday|2027Q4|N|N|N|2461711|2462014|2461375|2461648|N|N|N|N|N| +2461741|AAAAAAAANCAJFCAA|2027-12-01|1535|6675|513|2027|3|12|1|4|2027|513|6675|Wednesday|2027Q4|N|N|N|2461741|2462074|2461376|2461649|N|N|N|N|N| +2461742|AAAAAAAAOCAJFCAA|2027-12-02|1535|6675|513|2027|4|12|2|4|2027|513|6675|Thursday|2027Q4|N|N|N|2461741|2462074|2461377|2461650|N|N|N|N|N| +2461743|AAAAAAAAPCAJFCAA|2027-12-03|1535|6675|513|2027|5|12|3|4|2027|513|6675|Friday|2027Q4|N|Y|N|2461741|2462074|2461378|2461651|N|N|N|N|N| +2461744|AAAAAAAAADAJFCAA|2027-12-04|1535|6675|513|2027|6|12|4|4|2027|513|6675|Saturday|2027Q4|N|Y|N|2461741|2462074|2461379|2461652|N|N|N|N|N| +2461745|AAAAAAAABDAJFCAA|2027-12-05|1535|6675|513|2027|0|12|5|4|2027|513|6675|Sunday|2027Q4|N|N|N|2461741|2462074|2461380|2461653|N|N|N|N|N| +2461746|AAAAAAAACDAJFCAA|2027-12-06|1535|6675|513|2027|1|12|6|4|2027|513|6675|Monday|2027Q4|N|N|N|2461741|2462074|2461381|2461654|N|N|N|N|N| +2461747|AAAAAAAADDAJFCAA|2027-12-07|1535|6676|513|2027|2|12|7|4|2027|513|6676|Tuesday|2027Q4|N|N|N|2461741|2462074|2461382|2461655|N|N|N|N|N| +2461748|AAAAAAAAEDAJFCAA|2027-12-08|1535|6676|513|2027|3|12|8|4|2027|513|6676|Wednesday|2027Q4|N|N|N|2461741|2462074|2461383|2461656|N|N|N|N|N| +2461749|AAAAAAAAFDAJFCAA|2027-12-09|1535|6676|513|2027|4|12|9|4|2027|513|6676|Thursday|2027Q4|N|N|N|2461741|2462074|2461384|2461657|N|N|N|N|N| +2461750|AAAAAAAAGDAJFCAA|2027-12-10|1535|6676|513|2027|5|12|10|4|2027|513|6676|Friday|2027Q4|N|Y|N|2461741|2462074|2461385|2461658|N|N|N|N|N| +2461751|AAAAAAAAHDAJFCAA|2027-12-11|1535|6676|513|2027|6|12|11|4|2027|513|6676|Saturday|2027Q4|N|Y|N|2461741|2462074|2461386|2461659|N|N|N|N|N| +2461752|AAAAAAAAIDAJFCAA|2027-12-12|1535|6676|513|2027|0|12|12|4|2027|513|6676|Sunday|2027Q4|N|N|N|2461741|2462074|2461387|2461660|N|N|N|N|N| +2461753|AAAAAAAAJDAJFCAA|2027-12-13|1535|6676|513|2027|1|12|13|4|2027|513|6676|Monday|2027Q4|N|N|N|2461741|2462074|2461388|2461661|N|N|N|N|N| +2461754|AAAAAAAAKDAJFCAA|2027-12-14|1535|6677|513|2027|2|12|14|4|2027|513|6677|Tuesday|2027Q4|N|N|N|2461741|2462074|2461389|2461662|N|N|N|N|N| +2461755|AAAAAAAALDAJFCAA|2027-12-15|1535|6677|513|2027|3|12|15|4|2027|513|6677|Wednesday|2027Q4|N|N|N|2461741|2462074|2461390|2461663|N|N|N|N|N| +2461756|AAAAAAAAMDAJFCAA|2027-12-16|1535|6677|513|2027|4|12|16|4|2027|513|6677|Thursday|2027Q4|N|N|N|2461741|2462074|2461391|2461664|N|N|N|N|N| +2461757|AAAAAAAANDAJFCAA|2027-12-17|1535|6677|513|2027|5|12|17|4|2027|513|6677|Friday|2027Q4|N|Y|N|2461741|2462074|2461392|2461665|N|N|N|N|N| +2461758|AAAAAAAAODAJFCAA|2027-12-18|1535|6677|513|2027|6|12|18|4|2027|513|6677|Saturday|2027Q4|N|Y|N|2461741|2462074|2461393|2461666|N|N|N|N|N| +2461759|AAAAAAAAPDAJFCAA|2027-12-19|1535|6677|513|2027|0|12|19|4|2027|513|6677|Sunday|2027Q4|N|N|N|2461741|2462074|2461394|2461667|N|N|N|N|N| +2461760|AAAAAAAAAEAJFCAA|2027-12-20|1535|6677|513|2027|1|12|20|4|2027|513|6677|Monday|2027Q4|N|N|N|2461741|2462074|2461395|2461668|N|N|N|N|N| +2461761|AAAAAAAABEAJFCAA|2027-12-21|1535|6678|513|2027|2|12|21|4|2027|513|6678|Tuesday|2027Q4|N|N|N|2461741|2462074|2461396|2461669|N|N|N|N|N| +2461762|AAAAAAAACEAJFCAA|2027-12-22|1535|6678|513|2027|3|12|22|4|2027|513|6678|Wednesday|2027Q4|N|N|N|2461741|2462074|2461397|2461670|N|N|N|N|N| +2461763|AAAAAAAADEAJFCAA|2027-12-23|1535|6678|513|2027|4|12|23|4|2027|513|6678|Thursday|2027Q4|N|N|N|2461741|2462074|2461398|2461671|N|N|N|N|N| +2461764|AAAAAAAAEEAJFCAA|2027-12-24|1535|6678|513|2027|5|12|24|4|2027|513|6678|Friday|2027Q4|N|Y|N|2461741|2462074|2461399|2461672|N|N|N|N|N| +2461765|AAAAAAAAFEAJFCAA|2027-12-25|1535|6678|513|2027|6|12|25|4|2027|513|6678|Saturday|2027Q4|N|Y|N|2461741|2462074|2461400|2461673|N|N|N|N|N| +2461766|AAAAAAAAGEAJFCAA|2027-12-26|1535|6678|513|2027|0|12|26|4|2027|513|6678|Sunday|2027Q4|Y|N|N|2461741|2462074|2461401|2461674|N|N|N|N|N| +2461767|AAAAAAAAHEAJFCAA|2027-12-27|1535|6678|513|2027|1|12|27|4|2027|513|6678|Monday|2027Q4|N|N|Y|2461741|2462074|2461402|2461675|N|N|N|N|N| +2461768|AAAAAAAAIEAJFCAA|2027-12-28|1535|6679|513|2027|2|12|28|4|2027|513|6679|Tuesday|2027Q4|N|N|N|2461741|2462074|2461403|2461676|N|N|N|N|N| +2461769|AAAAAAAAJEAJFCAA|2027-12-29|1535|6679|513|2027|3|12|29|4|2027|513|6679|Wednesday|2027Q4|N|N|N|2461741|2462074|2461404|2461677|N|N|N|N|N| +2461770|AAAAAAAAKEAJFCAA|2027-12-30|1535|6679|513|2027|4|12|30|4|2027|513|6679|Thursday|2027Q4|N|N|N|2461741|2462074|2461405|2461678|N|N|N|N|N| +2461771|AAAAAAAALEAJFCAA|2027-12-31|1535|6679|513|2027|5|12|31|4|2027|513|6679|Friday|2027Q4|N|Y|N|2461741|2462074|2461406|2461679|N|N|N|N|N| +2461772|AAAAAAAAMEAJFCAA|2028-01-01|1536|6679|513|2028|6|1|1|1|2028|513|6679|Saturday|2028Q1|Y|Y|N|2461772|2461771|2461407|2461680|N|N|N|N|N| +2461773|AAAAAAAANEAJFCAA|2028-01-02|1536|6679|513|2028|0|1|2|1|2028|513|6679|Sunday|2028Q1|N|N|Y|2461772|2461771|2461408|2461681|N|N|N|N|N| +2461774|AAAAAAAAOEAJFCAA|2028-01-03|1536|6679|513|2028|1|1|3|1|2028|513|6679|Monday|2028Q1|N|N|N|2461772|2461771|2461409|2461682|N|N|N|N|N| +2461775|AAAAAAAAPEAJFCAA|2028-01-04|1536|6680|513|2028|2|1|4|1|2028|513|6680|Tuesday|2028Q1|N|N|N|2461772|2461771|2461410|2461683|N|N|N|N|N| +2461776|AAAAAAAAAFAJFCAA|2028-01-05|1536|6680|513|2028|3|1|5|1|2028|513|6680|Wednesday|2028Q1|N|N|N|2461772|2461771|2461411|2461684|N|N|N|N|N| +2461777|AAAAAAAABFAJFCAA|2028-01-06|1536|6680|513|2028|4|1|6|1|2028|513|6680|Thursday|2028Q1|N|N|N|2461772|2461771|2461412|2461685|N|N|N|N|N| +2461778|AAAAAAAACFAJFCAA|2028-01-07|1536|6680|513|2028|5|1|7|1|2028|513|6680|Friday|2028Q1|N|Y|N|2461772|2461771|2461413|2461686|N|N|N|N|N| +2461779|AAAAAAAADFAJFCAA|2028-01-08|1536|6680|513|2028|6|1|8|1|2028|513|6680|Saturday|2028Q1|N|Y|N|2461772|2461771|2461414|2461687|N|N|N|N|N| +2461780|AAAAAAAAEFAJFCAA|2028-01-09|1536|6680|513|2028|0|1|9|1|2028|513|6680|Sunday|2028Q1|N|N|N|2461772|2461771|2461415|2461688|N|N|N|N|N| +2461781|AAAAAAAAFFAJFCAA|2028-01-10|1536|6680|513|2028|1|1|10|1|2028|513|6680|Monday|2028Q1|N|N|N|2461772|2461771|2461416|2461689|N|N|N|N|N| +2461782|AAAAAAAAGFAJFCAA|2028-01-11|1536|6681|513|2028|2|1|11|1|2028|513|6681|Tuesday|2028Q1|N|N|N|2461772|2461771|2461417|2461690|N|N|N|N|N| +2461783|AAAAAAAAHFAJFCAA|2028-01-12|1536|6681|513|2028|3|1|12|1|2028|513|6681|Wednesday|2028Q1|N|N|N|2461772|2461771|2461418|2461691|N|N|N|N|N| +2461784|AAAAAAAAIFAJFCAA|2028-01-13|1536|6681|513|2028|4|1|13|1|2028|513|6681|Thursday|2028Q1|N|N|N|2461772|2461771|2461419|2461692|N|N|N|N|N| +2461785|AAAAAAAAJFAJFCAA|2028-01-14|1536|6681|513|2028|5|1|14|1|2028|513|6681|Friday|2028Q1|N|Y|N|2461772|2461771|2461420|2461693|N|N|N|N|N| +2461786|AAAAAAAAKFAJFCAA|2028-01-15|1536|6681|513|2028|6|1|15|1|2028|513|6681|Saturday|2028Q1|N|Y|N|2461772|2461771|2461421|2461694|N|N|N|N|N| +2461787|AAAAAAAALFAJFCAA|2028-01-16|1536|6681|513|2028|0|1|16|1|2028|513|6681|Sunday|2028Q1|N|N|N|2461772|2461771|2461422|2461695|N|N|N|N|N| +2461788|AAAAAAAAMFAJFCAA|2028-01-17|1536|6681|513|2028|1|1|17|1|2028|513|6681|Monday|2028Q1|N|N|N|2461772|2461771|2461423|2461696|N|N|N|N|N| +2461789|AAAAAAAANFAJFCAA|2028-01-18|1536|6682|513|2028|2|1|18|1|2028|513|6682|Tuesday|2028Q1|N|N|N|2461772|2461771|2461424|2461697|N|N|N|N|N| +2461790|AAAAAAAAOFAJFCAA|2028-01-19|1536|6682|513|2028|3|1|19|1|2028|513|6682|Wednesday|2028Q1|N|N|N|2461772|2461771|2461425|2461698|N|N|N|N|N| +2461791|AAAAAAAAPFAJFCAA|2028-01-20|1536|6682|513|2028|4|1|20|1|2028|513|6682|Thursday|2028Q1|N|N|N|2461772|2461771|2461426|2461699|N|N|N|N|N| +2461792|AAAAAAAAAGAJFCAA|2028-01-21|1536|6682|513|2028|5|1|21|1|2028|513|6682|Friday|2028Q1|N|Y|N|2461772|2461771|2461427|2461700|N|N|N|N|N| +2461793|AAAAAAAABGAJFCAA|2028-01-22|1536|6682|513|2028|6|1|22|1|2028|513|6682|Saturday|2028Q1|N|Y|N|2461772|2461771|2461428|2461701|N|N|N|N|N| +2461794|AAAAAAAACGAJFCAA|2028-01-23|1536|6682|513|2028|0|1|23|1|2028|513|6682|Sunday|2028Q1|N|N|N|2461772|2461771|2461429|2461702|N|N|N|N|N| +2461795|AAAAAAAADGAJFCAA|2028-01-24|1536|6682|513|2028|1|1|24|1|2028|513|6682|Monday|2028Q1|N|N|N|2461772|2461771|2461430|2461703|N|N|N|N|N| +2461796|AAAAAAAAEGAJFCAA|2028-01-25|1536|6683|513|2028|2|1|25|1|2028|513|6683|Tuesday|2028Q1|N|N|N|2461772|2461771|2461431|2461704|N|N|N|N|N| +2461797|AAAAAAAAFGAJFCAA|2028-01-26|1536|6683|513|2028|3|1|26|1|2028|513|6683|Wednesday|2028Q1|N|N|N|2461772|2461771|2461432|2461705|N|N|N|N|N| +2461798|AAAAAAAAGGAJFCAA|2028-01-27|1536|6683|513|2028|4|1|27|1|2028|513|6683|Thursday|2028Q1|N|N|N|2461772|2461771|2461433|2461706|N|N|N|N|N| +2461799|AAAAAAAAHGAJFCAA|2028-01-28|1536|6683|513|2028|5|1|28|1|2028|513|6683|Friday|2028Q1|N|Y|N|2461772|2461771|2461434|2461707|N|N|N|N|N| +2461800|AAAAAAAAIGAJFCAA|2028-01-29|1536|6683|513|2028|6|1|29|1|2028|513|6683|Saturday|2028Q1|N|Y|N|2461772|2461771|2461435|2461708|N|N|N|N|N| +2461801|AAAAAAAAJGAJFCAA|2028-01-30|1536|6683|513|2028|0|1|30|1|2028|513|6683|Sunday|2028Q1|N|N|N|2461772|2461771|2461436|2461709|N|N|N|N|N| +2461802|AAAAAAAAKGAJFCAA|2028-01-31|1536|6683|513|2028|1|1|31|1|2028|513|6683|Monday|2028Q1|N|N|N|2461772|2461771|2461437|2461710|N|N|N|N|N| +2461803|AAAAAAAALGAJFCAA|2028-02-01|1537|6684|513|2028|2|2|1|1|2028|513|6684|Tuesday|2028Q1|N|N|N|2461803|2461833|2461438|2461711|N|N|N|N|N| +2461804|AAAAAAAAMGAJFCAA|2028-02-02|1537|6684|513|2028|3|2|2|1|2028|513|6684|Wednesday|2028Q1|N|N|N|2461803|2461833|2461439|2461712|N|N|N|N|N| +2461805|AAAAAAAANGAJFCAA|2028-02-03|1537|6684|513|2028|4|2|3|1|2028|513|6684|Thursday|2028Q1|N|N|N|2461803|2461833|2461440|2461713|N|N|N|N|N| +2461806|AAAAAAAAOGAJFCAA|2028-02-04|1537|6684|513|2028|5|2|4|1|2028|513|6684|Friday|2028Q1|N|Y|N|2461803|2461833|2461441|2461714|N|N|N|N|N| +2461807|AAAAAAAAPGAJFCAA|2028-02-05|1537|6684|513|2028|6|2|5|1|2028|513|6684|Saturday|2028Q1|N|Y|N|2461803|2461833|2461442|2461715|N|N|N|N|N| +2461808|AAAAAAAAAHAJFCAA|2028-02-06|1537|6684|513|2028|0|2|6|1|2028|513|6684|Sunday|2028Q1|N|N|N|2461803|2461833|2461443|2461716|N|N|N|N|N| +2461809|AAAAAAAABHAJFCAA|2028-02-07|1537|6684|513|2028|1|2|7|1|2028|513|6684|Monday|2028Q1|N|N|N|2461803|2461833|2461444|2461717|N|N|N|N|N| +2461810|AAAAAAAACHAJFCAA|2028-02-08|1537|6685|513|2028|2|2|8|1|2028|513|6685|Tuesday|2028Q1|N|N|N|2461803|2461833|2461445|2461718|N|N|N|N|N| +2461811|AAAAAAAADHAJFCAA|2028-02-09|1537|6685|513|2028|3|2|9|1|2028|513|6685|Wednesday|2028Q1|N|N|N|2461803|2461833|2461446|2461719|N|N|N|N|N| +2461812|AAAAAAAAEHAJFCAA|2028-02-10|1537|6685|513|2028|4|2|10|1|2028|513|6685|Thursday|2028Q1|N|N|N|2461803|2461833|2461447|2461720|N|N|N|N|N| +2461813|AAAAAAAAFHAJFCAA|2028-02-11|1537|6685|513|2028|5|2|11|1|2028|513|6685|Friday|2028Q1|N|Y|N|2461803|2461833|2461448|2461721|N|N|N|N|N| +2461814|AAAAAAAAGHAJFCAA|2028-02-12|1537|6685|513|2028|6|2|12|1|2028|513|6685|Saturday|2028Q1|N|Y|N|2461803|2461833|2461449|2461722|N|N|N|N|N| +2461815|AAAAAAAAHHAJFCAA|2028-02-13|1537|6685|513|2028|0|2|13|1|2028|513|6685|Sunday|2028Q1|N|N|N|2461803|2461833|2461450|2461723|N|N|N|N|N| +2461816|AAAAAAAAIHAJFCAA|2028-02-14|1537|6685|513|2028|1|2|14|1|2028|513|6685|Monday|2028Q1|N|N|N|2461803|2461833|2461451|2461724|N|N|N|N|N| +2461817|AAAAAAAAJHAJFCAA|2028-02-15|1537|6686|513|2028|2|2|15|1|2028|513|6686|Tuesday|2028Q1|N|N|N|2461803|2461833|2461452|2461725|N|N|N|N|N| +2461818|AAAAAAAAKHAJFCAA|2028-02-16|1537|6686|513|2028|3|2|16|1|2028|513|6686|Wednesday|2028Q1|N|N|N|2461803|2461833|2461453|2461726|N|N|N|N|N| +2461819|AAAAAAAALHAJFCAA|2028-02-17|1537|6686|513|2028|4|2|17|1|2028|513|6686|Thursday|2028Q1|N|N|N|2461803|2461833|2461454|2461727|N|N|N|N|N| +2461820|AAAAAAAAMHAJFCAA|2028-02-18|1537|6686|513|2028|5|2|18|1|2028|513|6686|Friday|2028Q1|N|Y|N|2461803|2461833|2461455|2461728|N|N|N|N|N| +2461821|AAAAAAAANHAJFCAA|2028-02-19|1537|6686|513|2028|6|2|19|1|2028|513|6686|Saturday|2028Q1|N|Y|N|2461803|2461833|2461456|2461729|N|N|N|N|N| +2461822|AAAAAAAAOHAJFCAA|2028-02-20|1537|6686|513|2028|0|2|20|1|2028|513|6686|Sunday|2028Q1|N|N|N|2461803|2461833|2461457|2461730|N|N|N|N|N| +2461823|AAAAAAAAPHAJFCAA|2028-02-21|1537|6686|513|2028|1|2|21|1|2028|513|6686|Monday|2028Q1|N|N|N|2461803|2461833|2461458|2461731|N|N|N|N|N| +2461824|AAAAAAAAAIAJFCAA|2028-02-22|1537|6687|513|2028|2|2|22|1|2028|513|6687|Tuesday|2028Q1|N|N|N|2461803|2461833|2461459|2461732|N|N|N|N|N| +2461825|AAAAAAAABIAJFCAA|2028-02-23|1537|6687|513|2028|3|2|23|1|2028|513|6687|Wednesday|2028Q1|N|N|N|2461803|2461833|2461460|2461733|N|N|N|N|N| +2461826|AAAAAAAACIAJFCAA|2028-02-24|1537|6687|513|2028|4|2|24|1|2028|513|6687|Thursday|2028Q1|N|N|N|2461803|2461833|2461461|2461734|N|N|N|N|N| +2461827|AAAAAAAADIAJFCAA|2028-02-25|1537|6687|513|2028|5|2|25|1|2028|513|6687|Friday|2028Q1|N|Y|N|2461803|2461833|2461462|2461735|N|N|N|N|N| +2461828|AAAAAAAAEIAJFCAA|2028-02-26|1537|6687|513|2028|6|2|26|1|2028|513|6687|Saturday|2028Q1|N|Y|N|2461803|2461833|2461463|2461736|N|N|N|N|N| +2461829|AAAAAAAAFIAJFCAA|2028-02-27|1537|6687|513|2028|0|2|27|1|2028|513|6687|Sunday|2028Q1|N|N|N|2461803|2461833|2461464|2461737|N|N|N|N|N| +2461830|AAAAAAAAGIAJFCAA|2028-02-28|1537|6687|513|2028|1|2|28|1|2028|513|6687|Monday|2028Q1|N|N|N|2461803|2461833|2461465|2461738|N|N|N|N|N| +2461831|AAAAAAAAHIAJFCAA|2028-02-29|1537|6688|513|2028|2|2|29|1|2028|513|6688|Tuesday|2028Q1|N|N|N|2461803|2461833|2461465|2461739|N|N|N|N|N| +2461832|AAAAAAAAIIAJFCAA|2028-03-01|1538|6688|514|2028|3|3|1|1|2028|514|6688|Wednesday|2028Q1|N|N|N|2461832|2461891|2461466|2461740|N|N|N|N|N| +2461833|AAAAAAAAJIAJFCAA|2028-03-02|1538|6688|514|2028|4|3|2|1|2028|514|6688|Thursday|2028Q1|N|N|N|2461832|2461891|2461467|2461741|N|N|N|N|N| +2461834|AAAAAAAAKIAJFCAA|2028-03-03|1538|6688|514|2028|5|3|3|1|2028|514|6688|Friday|2028Q1|N|Y|N|2461832|2461891|2461468|2461742|N|N|N|N|N| +2461835|AAAAAAAALIAJFCAA|2028-03-04|1538|6688|514|2028|6|3|4|1|2028|514|6688|Saturday|2028Q1|N|Y|N|2461832|2461891|2461469|2461743|N|N|N|N|N| +2461836|AAAAAAAAMIAJFCAA|2028-03-05|1538|6688|514|2028|0|3|5|1|2028|514|6688|Sunday|2028Q1|N|N|N|2461832|2461891|2461470|2461744|N|N|N|N|N| +2461837|AAAAAAAANIAJFCAA|2028-03-06|1538|6688|514|2028|1|3|6|1|2028|514|6688|Monday|2028Q1|N|N|N|2461832|2461891|2461471|2461745|N|N|N|N|N| +2461838|AAAAAAAAOIAJFCAA|2028-03-07|1538|6689|514|2028|2|3|7|1|2028|514|6689|Tuesday|2028Q1|N|N|N|2461832|2461891|2461472|2461746|N|N|N|N|N| +2461839|AAAAAAAAPIAJFCAA|2028-03-08|1538|6689|514|2028|3|3|8|1|2028|514|6689|Wednesday|2028Q1|N|N|N|2461832|2461891|2461473|2461747|N|N|N|N|N| +2461840|AAAAAAAAAJAJFCAA|2028-03-09|1538|6689|514|2028|4|3|9|1|2028|514|6689|Thursday|2028Q1|N|N|N|2461832|2461891|2461474|2461748|N|N|N|N|N| +2461841|AAAAAAAABJAJFCAA|2028-03-10|1538|6689|514|2028|5|3|10|1|2028|514|6689|Friday|2028Q1|N|Y|N|2461832|2461891|2461475|2461749|N|N|N|N|N| +2461842|AAAAAAAACJAJFCAA|2028-03-11|1538|6689|514|2028|6|3|11|1|2028|514|6689|Saturday|2028Q1|N|Y|N|2461832|2461891|2461476|2461750|N|N|N|N|N| +2461843|AAAAAAAADJAJFCAA|2028-03-12|1538|6689|514|2028|0|3|12|1|2028|514|6689|Sunday|2028Q1|N|N|N|2461832|2461891|2461477|2461751|N|N|N|N|N| +2461844|AAAAAAAAEJAJFCAA|2028-03-13|1538|6689|514|2028|1|3|13|1|2028|514|6689|Monday|2028Q1|N|N|N|2461832|2461891|2461478|2461752|N|N|N|N|N| +2461845|AAAAAAAAFJAJFCAA|2028-03-14|1538|6690|514|2028|2|3|14|1|2028|514|6690|Tuesday|2028Q1|N|N|N|2461832|2461891|2461479|2461753|N|N|N|N|N| +2461846|AAAAAAAAGJAJFCAA|2028-03-15|1538|6690|514|2028|3|3|15|1|2028|514|6690|Wednesday|2028Q1|N|N|N|2461832|2461891|2461480|2461754|N|N|N|N|N| +2461847|AAAAAAAAHJAJFCAA|2028-03-16|1538|6690|514|2028|4|3|16|1|2028|514|6690|Thursday|2028Q1|N|N|N|2461832|2461891|2461481|2461755|N|N|N|N|N| +2461848|AAAAAAAAIJAJFCAA|2028-03-17|1538|6690|514|2028|5|3|17|1|2028|514|6690|Friday|2028Q1|N|Y|N|2461832|2461891|2461482|2461756|N|N|N|N|N| +2461849|AAAAAAAAJJAJFCAA|2028-03-18|1538|6690|514|2028|6|3|18|1|2028|514|6690|Saturday|2028Q1|N|Y|N|2461832|2461891|2461483|2461757|N|N|N|N|N| +2461850|AAAAAAAAKJAJFCAA|2028-03-19|1538|6690|514|2028|0|3|19|1|2028|514|6690|Sunday|2028Q1|N|N|N|2461832|2461891|2461484|2461758|N|N|N|N|N| +2461851|AAAAAAAALJAJFCAA|2028-03-20|1538|6690|514|2028|1|3|20|1|2028|514|6690|Monday|2028Q1|N|N|N|2461832|2461891|2461485|2461759|N|N|N|N|N| +2461852|AAAAAAAAMJAJFCAA|2028-03-21|1538|6691|514|2028|2|3|21|1|2028|514|6691|Tuesday|2028Q1|N|N|N|2461832|2461891|2461486|2461760|N|N|N|N|N| +2461853|AAAAAAAANJAJFCAA|2028-03-22|1538|6691|514|2028|3|3|22|1|2028|514|6691|Wednesday|2028Q1|N|N|N|2461832|2461891|2461487|2461761|N|N|N|N|N| +2461854|AAAAAAAAOJAJFCAA|2028-03-23|1538|6691|514|2028|4|3|23|1|2028|514|6691|Thursday|2028Q1|N|N|N|2461832|2461891|2461488|2461762|N|N|N|N|N| +2461855|AAAAAAAAPJAJFCAA|2028-03-24|1538|6691|514|2028|5|3|24|1|2028|514|6691|Friday|2028Q1|N|Y|N|2461832|2461891|2461489|2461763|N|N|N|N|N| +2461856|AAAAAAAAAKAJFCAA|2028-03-25|1538|6691|514|2028|6|3|25|1|2028|514|6691|Saturday|2028Q1|N|Y|N|2461832|2461891|2461490|2461764|N|N|N|N|N| +2461857|AAAAAAAABKAJFCAA|2028-03-26|1538|6691|514|2028|0|3|26|1|2028|514|6691|Sunday|2028Q1|N|N|N|2461832|2461891|2461491|2461765|N|N|N|N|N| +2461858|AAAAAAAACKAJFCAA|2028-03-27|1538|6691|514|2028|1|3|27|1|2028|514|6691|Monday|2028Q1|N|N|N|2461832|2461891|2461492|2461766|N|N|N|N|N| +2461859|AAAAAAAADKAJFCAA|2028-03-28|1538|6692|514|2028|2|3|28|1|2028|514|6692|Tuesday|2028Q1|N|N|N|2461832|2461891|2461493|2461767|N|N|N|N|N| +2461860|AAAAAAAAEKAJFCAA|2028-03-29|1538|6692|514|2028|3|3|29|1|2028|514|6692|Wednesday|2028Q1|N|N|N|2461832|2461891|2461494|2461768|N|N|N|N|N| +2461861|AAAAAAAAFKAJFCAA|2028-03-30|1538|6692|514|2028|4|3|30|1|2028|514|6692|Thursday|2028Q1|N|N|N|2461832|2461891|2461495|2461769|N|N|N|N|N| +2461862|AAAAAAAAGKAJFCAA|2028-03-31|1538|6692|514|2028|5|3|31|1|2028|514|6692|Friday|2028Q1|N|Y|N|2461832|2461891|2461496|2461770|N|N|N|N|N| +2461863|AAAAAAAAHKAJFCAA|2028-04-01|1539|6692|514|2028|6|4|1|2|2028|514|6692|Saturday|2028Q2|N|Y|N|2461863|2461953|2461497|2461772|N|N|N|N|N| +2461864|AAAAAAAAIKAJFCAA|2028-04-02|1539|6692|514|2028|0|4|2|2|2028|514|6692|Sunday|2028Q2|N|N|N|2461863|2461953|2461498|2461773|N|N|N|N|N| +2461865|AAAAAAAAJKAJFCAA|2028-04-03|1539|6692|514|2028|1|4|3|2|2028|514|6692|Monday|2028Q2|N|N|N|2461863|2461953|2461499|2461774|N|N|N|N|N| +2461866|AAAAAAAAKKAJFCAA|2028-04-04|1539|6693|514|2028|2|4|4|2|2028|514|6693|Tuesday|2028Q2|N|N|N|2461863|2461953|2461500|2461775|N|N|N|N|N| +2461867|AAAAAAAALKAJFCAA|2028-04-05|1539|6693|514|2028|3|4|5|2|2028|514|6693|Wednesday|2028Q2|N|N|N|2461863|2461953|2461501|2461776|N|N|N|N|N| +2461868|AAAAAAAAMKAJFCAA|2028-04-06|1539|6693|514|2028|4|4|6|2|2028|514|6693|Thursday|2028Q2|N|N|N|2461863|2461953|2461502|2461777|N|N|N|N|N| +2461869|AAAAAAAANKAJFCAA|2028-04-07|1539|6693|514|2028|5|4|7|2|2028|514|6693|Friday|2028Q2|N|Y|N|2461863|2461953|2461503|2461778|N|N|N|N|N| +2461870|AAAAAAAAOKAJFCAA|2028-04-08|1539|6693|514|2028|6|4|8|2|2028|514|6693|Saturday|2028Q2|N|Y|N|2461863|2461953|2461504|2461779|N|N|N|N|N| +2461871|AAAAAAAAPKAJFCAA|2028-04-09|1539|6693|514|2028|0|4|9|2|2028|514|6693|Sunday|2028Q2|N|N|N|2461863|2461953|2461505|2461780|N|N|N|N|N| +2461872|AAAAAAAAALAJFCAA|2028-04-10|1539|6693|514|2028|1|4|10|2|2028|514|6693|Monday|2028Q2|N|N|N|2461863|2461953|2461506|2461781|N|N|N|N|N| +2461873|AAAAAAAABLAJFCAA|2028-04-11|1539|6694|514|2028|2|4|11|2|2028|514|6694|Tuesday|2028Q2|N|N|N|2461863|2461953|2461507|2461782|N|N|N|N|N| +2461874|AAAAAAAACLAJFCAA|2028-04-12|1539|6694|514|2028|3|4|12|2|2028|514|6694|Wednesday|2028Q2|N|N|N|2461863|2461953|2461508|2461783|N|N|N|N|N| +2461875|AAAAAAAADLAJFCAA|2028-04-13|1539|6694|514|2028|4|4|13|2|2028|514|6694|Thursday|2028Q2|N|N|N|2461863|2461953|2461509|2461784|N|N|N|N|N| +2461876|AAAAAAAAELAJFCAA|2028-04-14|1539|6694|514|2028|5|4|14|2|2028|514|6694|Friday|2028Q2|N|Y|N|2461863|2461953|2461510|2461785|N|N|N|N|N| +2461877|AAAAAAAAFLAJFCAA|2028-04-15|1539|6694|514|2028|6|4|15|2|2028|514|6694|Saturday|2028Q2|N|Y|N|2461863|2461953|2461511|2461786|N|N|N|N|N| +2461878|AAAAAAAAGLAJFCAA|2028-04-16|1539|6694|514|2028|0|4|16|2|2028|514|6694|Sunday|2028Q2|N|N|N|2461863|2461953|2461512|2461787|N|N|N|N|N| +2461879|AAAAAAAAHLAJFCAA|2028-04-17|1539|6694|514|2028|1|4|17|2|2028|514|6694|Monday|2028Q2|N|N|N|2461863|2461953|2461513|2461788|N|N|N|N|N| +2461880|AAAAAAAAILAJFCAA|2028-04-18|1539|6695|514|2028|2|4|18|2|2028|514|6695|Tuesday|2028Q2|N|N|N|2461863|2461953|2461514|2461789|N|N|N|N|N| +2461881|AAAAAAAAJLAJFCAA|2028-04-19|1539|6695|514|2028|3|4|19|2|2028|514|6695|Wednesday|2028Q2|N|N|N|2461863|2461953|2461515|2461790|N|N|N|N|N| +2461882|AAAAAAAAKLAJFCAA|2028-04-20|1539|6695|514|2028|4|4|20|2|2028|514|6695|Thursday|2028Q2|N|N|N|2461863|2461953|2461516|2461791|N|N|N|N|N| +2461883|AAAAAAAALLAJFCAA|2028-04-21|1539|6695|514|2028|5|4|21|2|2028|514|6695|Friday|2028Q2|N|Y|N|2461863|2461953|2461517|2461792|N|N|N|N|N| +2461884|AAAAAAAAMLAJFCAA|2028-04-22|1539|6695|514|2028|6|4|22|2|2028|514|6695|Saturday|2028Q2|N|Y|N|2461863|2461953|2461518|2461793|N|N|N|N|N| +2461885|AAAAAAAANLAJFCAA|2028-04-23|1539|6695|514|2028|0|4|23|2|2028|514|6695|Sunday|2028Q2|N|N|N|2461863|2461953|2461519|2461794|N|N|N|N|N| +2461886|AAAAAAAAOLAJFCAA|2028-04-24|1539|6695|514|2028|1|4|24|2|2028|514|6695|Monday|2028Q2|N|N|N|2461863|2461953|2461520|2461795|N|N|N|N|N| +2461887|AAAAAAAAPLAJFCAA|2028-04-25|1539|6696|514|2028|2|4|25|2|2028|514|6696|Tuesday|2028Q2|N|N|N|2461863|2461953|2461521|2461796|N|N|N|N|N| +2461888|AAAAAAAAAMAJFCAA|2028-04-26|1539|6696|514|2028|3|4|26|2|2028|514|6696|Wednesday|2028Q2|N|N|N|2461863|2461953|2461522|2461797|N|N|N|N|N| +2461889|AAAAAAAABMAJFCAA|2028-04-27|1539|6696|514|2028|4|4|27|2|2028|514|6696|Thursday|2028Q2|N|N|N|2461863|2461953|2461523|2461798|N|N|N|N|N| +2461890|AAAAAAAACMAJFCAA|2028-04-28|1539|6696|514|2028|5|4|28|2|2028|514|6696|Friday|2028Q2|N|Y|N|2461863|2461953|2461524|2461799|N|N|N|N|N| +2461891|AAAAAAAADMAJFCAA|2028-04-29|1539|6696|514|2028|6|4|29|2|2028|514|6696|Saturday|2028Q2|N|Y|N|2461863|2461953|2461525|2461800|N|N|N|N|N| +2461892|AAAAAAAAEMAJFCAA|2028-04-30|1539|6696|514|2028|0|4|30|2|2028|514|6696|Sunday|2028Q2|N|N|N|2461863|2461953|2461526|2461801|N|N|N|N|N| +2461893|AAAAAAAAFMAJFCAA|2028-05-01|1540|6696|514|2028|1|5|1|2|2028|514|6696|Monday|2028Q2|N|N|N|2461893|2462013|2461527|2461802|N|N|N|N|N| +2461894|AAAAAAAAGMAJFCAA|2028-05-02|1540|6697|514|2028|2|5|2|2|2028|514|6697|Tuesday|2028Q2|N|N|N|2461893|2462013|2461528|2461803|N|N|N|N|N| +2461895|AAAAAAAAHMAJFCAA|2028-05-03|1540|6697|514|2028|3|5|3|2|2028|514|6697|Wednesday|2028Q2|N|N|N|2461893|2462013|2461529|2461804|N|N|N|N|N| +2461896|AAAAAAAAIMAJFCAA|2028-05-04|1540|6697|514|2028|4|5|4|2|2028|514|6697|Thursday|2028Q2|N|N|N|2461893|2462013|2461530|2461805|N|N|N|N|N| +2461897|AAAAAAAAJMAJFCAA|2028-05-05|1540|6697|514|2028|5|5|5|2|2028|514|6697|Friday|2028Q2|N|Y|N|2461893|2462013|2461531|2461806|N|N|N|N|N| +2461898|AAAAAAAAKMAJFCAA|2028-05-06|1540|6697|514|2028|6|5|6|2|2028|514|6697|Saturday|2028Q2|N|Y|N|2461893|2462013|2461532|2461807|N|N|N|N|N| +2461899|AAAAAAAALMAJFCAA|2028-05-07|1540|6697|514|2028|0|5|7|2|2028|514|6697|Sunday|2028Q2|N|N|N|2461893|2462013|2461533|2461808|N|N|N|N|N| +2461900|AAAAAAAAMMAJFCAA|2028-05-08|1540|6697|514|2028|1|5|8|2|2028|514|6697|Monday|2028Q2|N|N|N|2461893|2462013|2461534|2461809|N|N|N|N|N| +2461901|AAAAAAAANMAJFCAA|2028-05-09|1540|6698|514|2028|2|5|9|2|2028|514|6698|Tuesday|2028Q2|N|N|N|2461893|2462013|2461535|2461810|N|N|N|N|N| +2461902|AAAAAAAAOMAJFCAA|2028-05-10|1540|6698|514|2028|3|5|10|2|2028|514|6698|Wednesday|2028Q2|N|N|N|2461893|2462013|2461536|2461811|N|N|N|N|N| +2461903|AAAAAAAAPMAJFCAA|2028-05-11|1540|6698|514|2028|4|5|11|2|2028|514|6698|Thursday|2028Q2|N|N|N|2461893|2462013|2461537|2461812|N|N|N|N|N| +2461904|AAAAAAAAANAJFCAA|2028-05-12|1540|6698|514|2028|5|5|12|2|2028|514|6698|Friday|2028Q2|N|Y|N|2461893|2462013|2461538|2461813|N|N|N|N|N| +2461905|AAAAAAAABNAJFCAA|2028-05-13|1540|6698|514|2028|6|5|13|2|2028|514|6698|Saturday|2028Q2|N|Y|N|2461893|2462013|2461539|2461814|N|N|N|N|N| +2461906|AAAAAAAACNAJFCAA|2028-05-14|1540|6698|514|2028|0|5|14|2|2028|514|6698|Sunday|2028Q2|N|N|N|2461893|2462013|2461540|2461815|N|N|N|N|N| +2461907|AAAAAAAADNAJFCAA|2028-05-15|1540|6698|514|2028|1|5|15|2|2028|514|6698|Monday|2028Q2|N|N|N|2461893|2462013|2461541|2461816|N|N|N|N|N| +2461908|AAAAAAAAENAJFCAA|2028-05-16|1540|6699|514|2028|2|5|16|2|2028|514|6699|Tuesday|2028Q2|N|N|N|2461893|2462013|2461542|2461817|N|N|N|N|N| +2461909|AAAAAAAAFNAJFCAA|2028-05-17|1540|6699|514|2028|3|5|17|2|2028|514|6699|Wednesday|2028Q2|N|N|N|2461893|2462013|2461543|2461818|N|N|N|N|N| +2461910|AAAAAAAAGNAJFCAA|2028-05-18|1540|6699|514|2028|4|5|18|2|2028|514|6699|Thursday|2028Q2|N|N|N|2461893|2462013|2461544|2461819|N|N|N|N|N| +2461911|AAAAAAAAHNAJFCAA|2028-05-19|1540|6699|514|2028|5|5|19|2|2028|514|6699|Friday|2028Q2|N|Y|N|2461893|2462013|2461545|2461820|N|N|N|N|N| +2461912|AAAAAAAAINAJFCAA|2028-05-20|1540|6699|514|2028|6|5|20|2|2028|514|6699|Saturday|2028Q2|N|Y|N|2461893|2462013|2461546|2461821|N|N|N|N|N| +2461913|AAAAAAAAJNAJFCAA|2028-05-21|1540|6699|514|2028|0|5|21|2|2028|514|6699|Sunday|2028Q2|N|N|N|2461893|2462013|2461547|2461822|N|N|N|N|N| +2461914|AAAAAAAAKNAJFCAA|2028-05-22|1540|6699|514|2028|1|5|22|2|2028|514|6699|Monday|2028Q2|N|N|N|2461893|2462013|2461548|2461823|N|N|N|N|N| +2461915|AAAAAAAALNAJFCAA|2028-05-23|1540|6700|514|2028|2|5|23|2|2028|514|6700|Tuesday|2028Q2|N|N|N|2461893|2462013|2461549|2461824|N|N|N|N|N| +2461916|AAAAAAAAMNAJFCAA|2028-05-24|1540|6700|514|2028|3|5|24|2|2028|514|6700|Wednesday|2028Q2|N|N|N|2461893|2462013|2461550|2461825|N|N|N|N|N| +2461917|AAAAAAAANNAJFCAA|2028-05-25|1540|6700|514|2028|4|5|25|2|2028|514|6700|Thursday|2028Q2|N|N|N|2461893|2462013|2461551|2461826|N|N|N|N|N| +2461918|AAAAAAAAONAJFCAA|2028-05-26|1540|6700|514|2028|5|5|26|2|2028|514|6700|Friday|2028Q2|N|Y|N|2461893|2462013|2461552|2461827|N|N|N|N|N| +2461919|AAAAAAAAPNAJFCAA|2028-05-27|1540|6700|514|2028|6|5|27|2|2028|514|6700|Saturday|2028Q2|N|Y|N|2461893|2462013|2461553|2461828|N|N|N|N|N| +2461920|AAAAAAAAAOAJFCAA|2028-05-28|1540|6700|514|2028|0|5|28|2|2028|514|6700|Sunday|2028Q2|N|N|N|2461893|2462013|2461554|2461829|N|N|N|N|N| +2461921|AAAAAAAABOAJFCAA|2028-05-29|1540|6700|514|2028|1|5|29|2|2028|514|6700|Monday|2028Q2|N|N|N|2461893|2462013|2461555|2461830|N|N|N|N|N| +2461922|AAAAAAAACOAJFCAA|2028-05-30|1540|6701|514|2028|2|5|30|2|2028|514|6701|Tuesday|2028Q2|N|N|N|2461893|2462013|2461556|2461831|N|N|N|N|N| +2461923|AAAAAAAADOAJFCAA|2028-05-31|1540|6701|514|2028|3|5|31|2|2028|514|6701|Wednesday|2028Q2|N|N|N|2461893|2462013|2461557|2461832|N|N|N|N|N| +2461924|AAAAAAAAEOAJFCAA|2028-06-01|1541|6701|515|2028|4|6|1|2|2028|515|6701|Thursday|2028Q2|N|N|N|2461924|2462075|2461558|2461833|N|N|N|N|N| +2461925|AAAAAAAAFOAJFCAA|2028-06-02|1541|6701|515|2028|5|6|2|2|2028|515|6701|Friday|2028Q2|N|Y|N|2461924|2462075|2461559|2461834|N|N|N|N|N| +2461926|AAAAAAAAGOAJFCAA|2028-06-03|1541|6701|515|2028|6|6|3|2|2028|515|6701|Saturday|2028Q2|N|Y|N|2461924|2462075|2461560|2461835|N|N|N|N|N| +2461927|AAAAAAAAHOAJFCAA|2028-06-04|1541|6701|515|2028|0|6|4|2|2028|515|6701|Sunday|2028Q2|N|N|N|2461924|2462075|2461561|2461836|N|N|N|N|N| +2461928|AAAAAAAAIOAJFCAA|2028-06-05|1541|6701|515|2028|1|6|5|2|2028|515|6701|Monday|2028Q2|N|N|N|2461924|2462075|2461562|2461837|N|N|N|N|N| +2461929|AAAAAAAAJOAJFCAA|2028-06-06|1541|6702|515|2028|2|6|6|2|2028|515|6702|Tuesday|2028Q2|N|N|N|2461924|2462075|2461563|2461838|N|N|N|N|N| +2461930|AAAAAAAAKOAJFCAA|2028-06-07|1541|6702|515|2028|3|6|7|2|2028|515|6702|Wednesday|2028Q2|N|N|N|2461924|2462075|2461564|2461839|N|N|N|N|N| +2461931|AAAAAAAALOAJFCAA|2028-06-08|1541|6702|515|2028|4|6|8|2|2028|515|6702|Thursday|2028Q2|N|N|N|2461924|2462075|2461565|2461840|N|N|N|N|N| +2461932|AAAAAAAAMOAJFCAA|2028-06-09|1541|6702|515|2028|5|6|9|2|2028|515|6702|Friday|2028Q2|N|Y|N|2461924|2462075|2461566|2461841|N|N|N|N|N| +2461933|AAAAAAAANOAJFCAA|2028-06-10|1541|6702|515|2028|6|6|10|2|2028|515|6702|Saturday|2028Q2|N|Y|N|2461924|2462075|2461567|2461842|N|N|N|N|N| +2461934|AAAAAAAAOOAJFCAA|2028-06-11|1541|6702|515|2028|0|6|11|2|2028|515|6702|Sunday|2028Q2|N|N|N|2461924|2462075|2461568|2461843|N|N|N|N|N| +2461935|AAAAAAAAPOAJFCAA|2028-06-12|1541|6702|515|2028|1|6|12|2|2028|515|6702|Monday|2028Q2|N|N|N|2461924|2462075|2461569|2461844|N|N|N|N|N| +2461936|AAAAAAAAAPAJFCAA|2028-06-13|1541|6703|515|2028|2|6|13|2|2028|515|6703|Tuesday|2028Q2|N|N|N|2461924|2462075|2461570|2461845|N|N|N|N|N| +2461937|AAAAAAAABPAJFCAA|2028-06-14|1541|6703|515|2028|3|6|14|2|2028|515|6703|Wednesday|2028Q2|N|N|N|2461924|2462075|2461571|2461846|N|N|N|N|N| +2461938|AAAAAAAACPAJFCAA|2028-06-15|1541|6703|515|2028|4|6|15|2|2028|515|6703|Thursday|2028Q2|N|N|N|2461924|2462075|2461572|2461847|N|N|N|N|N| +2461939|AAAAAAAADPAJFCAA|2028-06-16|1541|6703|515|2028|5|6|16|2|2028|515|6703|Friday|2028Q2|N|Y|N|2461924|2462075|2461573|2461848|N|N|N|N|N| +2461940|AAAAAAAAEPAJFCAA|2028-06-17|1541|6703|515|2028|6|6|17|2|2028|515|6703|Saturday|2028Q2|N|Y|N|2461924|2462075|2461574|2461849|N|N|N|N|N| +2461941|AAAAAAAAFPAJFCAA|2028-06-18|1541|6703|515|2028|0|6|18|2|2028|515|6703|Sunday|2028Q2|N|N|N|2461924|2462075|2461575|2461850|N|N|N|N|N| +2461942|AAAAAAAAGPAJFCAA|2028-06-19|1541|6703|515|2028|1|6|19|2|2028|515|6703|Monday|2028Q2|N|N|N|2461924|2462075|2461576|2461851|N|N|N|N|N| +2461943|AAAAAAAAHPAJFCAA|2028-06-20|1541|6704|515|2028|2|6|20|2|2028|515|6704|Tuesday|2028Q2|N|N|N|2461924|2462075|2461577|2461852|N|N|N|N|N| +2461944|AAAAAAAAIPAJFCAA|2028-06-21|1541|6704|515|2028|3|6|21|2|2028|515|6704|Wednesday|2028Q2|N|N|N|2461924|2462075|2461578|2461853|N|N|N|N|N| +2461945|AAAAAAAAJPAJFCAA|2028-06-22|1541|6704|515|2028|4|6|22|2|2028|515|6704|Thursday|2028Q2|N|N|N|2461924|2462075|2461579|2461854|N|N|N|N|N| +2461946|AAAAAAAAKPAJFCAA|2028-06-23|1541|6704|515|2028|5|6|23|2|2028|515|6704|Friday|2028Q2|N|Y|N|2461924|2462075|2461580|2461855|N|N|N|N|N| +2461947|AAAAAAAALPAJFCAA|2028-06-24|1541|6704|515|2028|6|6|24|2|2028|515|6704|Saturday|2028Q2|N|Y|N|2461924|2462075|2461581|2461856|N|N|N|N|N| +2461948|AAAAAAAAMPAJFCAA|2028-06-25|1541|6704|515|2028|0|6|25|2|2028|515|6704|Sunday|2028Q2|N|N|N|2461924|2462075|2461582|2461857|N|N|N|N|N| +2461949|AAAAAAAANPAJFCAA|2028-06-26|1541|6704|515|2028|1|6|26|2|2028|515|6704|Monday|2028Q2|N|N|N|2461924|2462075|2461583|2461858|N|N|N|N|N| +2461950|AAAAAAAAOPAJFCAA|2028-06-27|1541|6705|515|2028|2|6|27|2|2028|515|6705|Tuesday|2028Q2|N|N|N|2461924|2462075|2461584|2461859|N|N|N|N|N| +2461951|AAAAAAAAPPAJFCAA|2028-06-28|1541|6705|515|2028|3|6|28|2|2028|515|6705|Wednesday|2028Q2|N|N|N|2461924|2462075|2461585|2461860|N|N|N|N|N| +2461952|AAAAAAAAAABJFCAA|2028-06-29|1541|6705|515|2028|4|6|29|2|2028|515|6705|Thursday|2028Q2|N|N|N|2461924|2462075|2461586|2461861|N|N|N|N|N| +2461953|AAAAAAAABABJFCAA|2028-06-30|1541|6705|515|2028|5|6|30|2|2028|515|6705|Friday|2028Q2|N|Y|N|2461924|2462075|2461587|2461862|N|N|N|N|N| +2461954|AAAAAAAACABJFCAA|2028-07-01|1542|6705|515|2028|6|7|1|3|2028|515|6705|Saturday|2028Q3|N|Y|N|2461954|2462135|2461588|2461863|N|N|N|N|N| +2461955|AAAAAAAADABJFCAA|2028-07-02|1542|6705|515|2028|0|7|2|3|2028|515|6705|Sunday|2028Q3|N|N|N|2461954|2462135|2461589|2461864|N|N|N|N|N| +2461956|AAAAAAAAEABJFCAA|2028-07-03|1542|6705|515|2028|1|7|3|3|2028|515|6705|Monday|2028Q3|N|N|N|2461954|2462135|2461590|2461865|N|N|N|N|N| +2461957|AAAAAAAAFABJFCAA|2028-07-04|1542|6706|515|2028|2|7|4|3|2028|515|6706|Tuesday|2028Q3|Y|N|N|2461954|2462135|2461591|2461866|N|N|N|N|N| +2461958|AAAAAAAAGABJFCAA|2028-07-05|1542|6706|515|2028|3|7|5|3|2028|515|6706|Wednesday|2028Q3|N|N|Y|2461954|2462135|2461592|2461867|N|N|N|N|N| +2461959|AAAAAAAAHABJFCAA|2028-07-06|1542|6706|515|2028|4|7|6|3|2028|515|6706|Thursday|2028Q3|N|N|N|2461954|2462135|2461593|2461868|N|N|N|N|N| +2461960|AAAAAAAAIABJFCAA|2028-07-07|1542|6706|515|2028|5|7|7|3|2028|515|6706|Friday|2028Q3|N|Y|N|2461954|2462135|2461594|2461869|N|N|N|N|N| +2461961|AAAAAAAAJABJFCAA|2028-07-08|1542|6706|515|2028|6|7|8|3|2028|515|6706|Saturday|2028Q3|N|Y|N|2461954|2462135|2461595|2461870|N|N|N|N|N| +2461962|AAAAAAAAKABJFCAA|2028-07-09|1542|6706|515|2028|0|7|9|3|2028|515|6706|Sunday|2028Q3|N|N|N|2461954|2462135|2461596|2461871|N|N|N|N|N| +2461963|AAAAAAAALABJFCAA|2028-07-10|1542|6706|515|2028|1|7|10|3|2028|515|6706|Monday|2028Q3|N|N|N|2461954|2462135|2461597|2461872|N|N|N|N|N| +2461964|AAAAAAAAMABJFCAA|2028-07-11|1542|6707|515|2028|2|7|11|3|2028|515|6707|Tuesday|2028Q3|N|N|N|2461954|2462135|2461598|2461873|N|N|N|N|N| +2461965|AAAAAAAANABJFCAA|2028-07-12|1542|6707|515|2028|3|7|12|3|2028|515|6707|Wednesday|2028Q3|N|N|N|2461954|2462135|2461599|2461874|N|N|N|N|N| +2461966|AAAAAAAAOABJFCAA|2028-07-13|1542|6707|515|2028|4|7|13|3|2028|515|6707|Thursday|2028Q3|N|N|N|2461954|2462135|2461600|2461875|N|N|N|N|N| +2461967|AAAAAAAAPABJFCAA|2028-07-14|1542|6707|515|2028|5|7|14|3|2028|515|6707|Friday|2028Q3|N|Y|N|2461954|2462135|2461601|2461876|N|N|N|N|N| +2461968|AAAAAAAAABBJFCAA|2028-07-15|1542|6707|515|2028|6|7|15|3|2028|515|6707|Saturday|2028Q3|N|Y|N|2461954|2462135|2461602|2461877|N|N|N|N|N| +2461969|AAAAAAAABBBJFCAA|2028-07-16|1542|6707|515|2028|0|7|16|3|2028|515|6707|Sunday|2028Q3|N|N|N|2461954|2462135|2461603|2461878|N|N|N|N|N| +2461970|AAAAAAAACBBJFCAA|2028-07-17|1542|6707|515|2028|1|7|17|3|2028|515|6707|Monday|2028Q3|N|N|N|2461954|2462135|2461604|2461879|N|N|N|N|N| +2461971|AAAAAAAADBBJFCAA|2028-07-18|1542|6708|515|2028|2|7|18|3|2028|515|6708|Tuesday|2028Q3|N|N|N|2461954|2462135|2461605|2461880|N|N|N|N|N| +2461972|AAAAAAAAEBBJFCAA|2028-07-19|1542|6708|515|2028|3|7|19|3|2028|515|6708|Wednesday|2028Q3|N|N|N|2461954|2462135|2461606|2461881|N|N|N|N|N| +2461973|AAAAAAAAFBBJFCAA|2028-07-20|1542|6708|515|2028|4|7|20|3|2028|515|6708|Thursday|2028Q3|N|N|N|2461954|2462135|2461607|2461882|N|N|N|N|N| +2461974|AAAAAAAAGBBJFCAA|2028-07-21|1542|6708|515|2028|5|7|21|3|2028|515|6708|Friday|2028Q3|N|Y|N|2461954|2462135|2461608|2461883|N|N|N|N|N| +2461975|AAAAAAAAHBBJFCAA|2028-07-22|1542|6708|515|2028|6|7|22|3|2028|515|6708|Saturday|2028Q3|N|Y|N|2461954|2462135|2461609|2461884|N|N|N|N|N| +2461976|AAAAAAAAIBBJFCAA|2028-07-23|1542|6708|515|2028|0|7|23|3|2028|515|6708|Sunday|2028Q3|N|N|N|2461954|2462135|2461610|2461885|N|N|N|N|N| +2461977|AAAAAAAAJBBJFCAA|2028-07-24|1542|6708|515|2028|1|7|24|3|2028|515|6708|Monday|2028Q3|N|N|N|2461954|2462135|2461611|2461886|N|N|N|N|N| +2461978|AAAAAAAAKBBJFCAA|2028-07-25|1542|6709|515|2028|2|7|25|3|2028|515|6709|Tuesday|2028Q3|N|N|N|2461954|2462135|2461612|2461887|N|N|N|N|N| +2461979|AAAAAAAALBBJFCAA|2028-07-26|1542|6709|515|2028|3|7|26|3|2028|515|6709|Wednesday|2028Q3|N|N|N|2461954|2462135|2461613|2461888|N|N|N|N|N| +2461980|AAAAAAAAMBBJFCAA|2028-07-27|1542|6709|515|2028|4|7|27|3|2028|515|6709|Thursday|2028Q3|N|N|N|2461954|2462135|2461614|2461889|N|N|N|N|N| +2461981|AAAAAAAANBBJFCAA|2028-07-28|1542|6709|515|2028|5|7|28|3|2028|515|6709|Friday|2028Q3|N|Y|N|2461954|2462135|2461615|2461890|N|N|N|N|N| +2461982|AAAAAAAAOBBJFCAA|2028-07-29|1542|6709|515|2028|6|7|29|3|2028|515|6709|Saturday|2028Q3|N|Y|N|2461954|2462135|2461616|2461891|N|N|N|N|N| +2461983|AAAAAAAAPBBJFCAA|2028-07-30|1542|6709|515|2028|0|7|30|3|2028|515|6709|Sunday|2028Q3|N|N|N|2461954|2462135|2461617|2461892|N|N|N|N|N| +2461984|AAAAAAAAACBJFCAA|2028-07-31|1542|6709|515|2028|1|7|31|3|2028|515|6709|Monday|2028Q3|N|N|N|2461954|2462135|2461618|2461893|N|N|N|N|N| +2461985|AAAAAAAABCBJFCAA|2028-08-01|1543|6710|515|2028|2|8|1|3|2028|515|6710|Tuesday|2028Q3|N|N|N|2461985|2462197|2461619|2461894|N|N|N|N|N| +2461986|AAAAAAAACCBJFCAA|2028-08-02|1543|6710|515|2028|3|8|2|3|2028|515|6710|Wednesday|2028Q3|N|N|N|2461985|2462197|2461620|2461895|N|N|N|N|N| +2461987|AAAAAAAADCBJFCAA|2028-08-03|1543|6710|515|2028|4|8|3|3|2028|515|6710|Thursday|2028Q3|N|N|N|2461985|2462197|2461621|2461896|N|N|N|N|N| +2461988|AAAAAAAAECBJFCAA|2028-08-04|1543|6710|515|2028|5|8|4|3|2028|515|6710|Friday|2028Q3|N|Y|N|2461985|2462197|2461622|2461897|N|N|N|N|N| +2461989|AAAAAAAAFCBJFCAA|2028-08-05|1543|6710|515|2028|6|8|5|3|2028|515|6710|Saturday|2028Q3|N|Y|N|2461985|2462197|2461623|2461898|N|N|N|N|N| +2461990|AAAAAAAAGCBJFCAA|2028-08-06|1543|6710|515|2028|0|8|6|3|2028|515|6710|Sunday|2028Q3|N|N|N|2461985|2462197|2461624|2461899|N|N|N|N|N| +2461991|AAAAAAAAHCBJFCAA|2028-08-07|1543|6710|515|2028|1|8|7|3|2028|515|6710|Monday|2028Q3|N|N|N|2461985|2462197|2461625|2461900|N|N|N|N|N| +2461992|AAAAAAAAICBJFCAA|2028-08-08|1543|6711|515|2028|2|8|8|3|2028|515|6711|Tuesday|2028Q3|N|N|N|2461985|2462197|2461626|2461901|N|N|N|N|N| +2461993|AAAAAAAAJCBJFCAA|2028-08-09|1543|6711|515|2028|3|8|9|3|2028|515|6711|Wednesday|2028Q3|N|N|N|2461985|2462197|2461627|2461902|N|N|N|N|N| +2461994|AAAAAAAAKCBJFCAA|2028-08-10|1543|6711|515|2028|4|8|10|3|2028|515|6711|Thursday|2028Q3|N|N|N|2461985|2462197|2461628|2461903|N|N|N|N|N| +2461995|AAAAAAAALCBJFCAA|2028-08-11|1543|6711|515|2028|5|8|11|3|2028|515|6711|Friday|2028Q3|N|Y|N|2461985|2462197|2461629|2461904|N|N|N|N|N| +2461996|AAAAAAAAMCBJFCAA|2028-08-12|1543|6711|515|2028|6|8|12|3|2028|515|6711|Saturday|2028Q3|N|Y|N|2461985|2462197|2461630|2461905|N|N|N|N|N| +2461997|AAAAAAAANCBJFCAA|2028-08-13|1543|6711|515|2028|0|8|13|3|2028|515|6711|Sunday|2028Q3|N|N|N|2461985|2462197|2461631|2461906|N|N|N|N|N| +2461998|AAAAAAAAOCBJFCAA|2028-08-14|1543|6711|515|2028|1|8|14|3|2028|515|6711|Monday|2028Q3|N|N|N|2461985|2462197|2461632|2461907|N|N|N|N|N| +2461999|AAAAAAAAPCBJFCAA|2028-08-15|1543|6712|515|2028|2|8|15|3|2028|515|6712|Tuesday|2028Q3|N|N|N|2461985|2462197|2461633|2461908|N|N|N|N|N| +2462000|AAAAAAAAADBJFCAA|2028-08-16|1543|6712|515|2028|3|8|16|3|2028|515|6712|Wednesday|2028Q3|N|N|N|2461985|2462197|2461634|2461909|N|N|N|N|N| +2462001|AAAAAAAABDBJFCAA|2028-08-17|1543|6712|515|2028|4|8|17|3|2028|515|6712|Thursday|2028Q3|N|N|N|2461985|2462197|2461635|2461910|N|N|N|N|N| +2462002|AAAAAAAACDBJFCAA|2028-08-18|1543|6712|515|2028|5|8|18|3|2028|515|6712|Friday|2028Q3|N|Y|N|2461985|2462197|2461636|2461911|N|N|N|N|N| +2462003|AAAAAAAADDBJFCAA|2028-08-19|1543|6712|515|2028|6|8|19|3|2028|515|6712|Saturday|2028Q3|N|Y|N|2461985|2462197|2461637|2461912|N|N|N|N|N| +2462004|AAAAAAAAEDBJFCAA|2028-08-20|1543|6712|515|2028|0|8|20|3|2028|515|6712|Sunday|2028Q3|N|N|N|2461985|2462197|2461638|2461913|N|N|N|N|N| +2462005|AAAAAAAAFDBJFCAA|2028-08-21|1543|6712|515|2028|1|8|21|3|2028|515|6712|Monday|2028Q3|N|N|N|2461985|2462197|2461639|2461914|N|N|N|N|N| +2462006|AAAAAAAAGDBJFCAA|2028-08-22|1543|6713|515|2028|2|8|22|3|2028|515|6713|Tuesday|2028Q3|N|N|N|2461985|2462197|2461640|2461915|N|N|N|N|N| +2462007|AAAAAAAAHDBJFCAA|2028-08-23|1543|6713|515|2028|3|8|23|3|2028|515|6713|Wednesday|2028Q3|N|N|N|2461985|2462197|2461641|2461916|N|N|N|N|N| +2462008|AAAAAAAAIDBJFCAA|2028-08-24|1543|6713|515|2028|4|8|24|3|2028|515|6713|Thursday|2028Q3|N|N|N|2461985|2462197|2461642|2461917|N|N|N|N|N| +2462009|AAAAAAAAJDBJFCAA|2028-08-25|1543|6713|515|2028|5|8|25|3|2028|515|6713|Friday|2028Q3|N|Y|N|2461985|2462197|2461643|2461918|N|N|N|N|N| +2462010|AAAAAAAAKDBJFCAA|2028-08-26|1543|6713|515|2028|6|8|26|3|2028|515|6713|Saturday|2028Q3|N|Y|N|2461985|2462197|2461644|2461919|N|N|N|N|N| +2462011|AAAAAAAALDBJFCAA|2028-08-27|1543|6713|515|2028|0|8|27|3|2028|515|6713|Sunday|2028Q3|N|N|N|2461985|2462197|2461645|2461920|N|N|N|N|N| +2462012|AAAAAAAAMDBJFCAA|2028-08-28|1543|6713|515|2028|1|8|28|3|2028|515|6713|Monday|2028Q3|N|N|N|2461985|2462197|2461646|2461921|N|N|N|N|N| +2462013|AAAAAAAANDBJFCAA|2028-08-29|1543|6714|515|2028|2|8|29|3|2028|515|6714|Tuesday|2028Q3|N|N|N|2461985|2462197|2461647|2461922|N|N|N|N|N| +2462014|AAAAAAAAODBJFCAA|2028-08-30|1543|6714|515|2028|3|8|30|3|2028|515|6714|Wednesday|2028Q3|N|N|N|2461985|2462197|2461648|2461923|N|N|N|N|N| +2462015|AAAAAAAAPDBJFCAA|2028-08-31|1543|6714|515|2028|4|8|31|3|2028|515|6714|Thursday|2028Q3|N|N|N|2461985|2462197|2461649|2461924|N|N|N|N|N| +2462016|AAAAAAAAAEBJFCAA|2028-09-01|1544|6714|516|2028|5|9|1|3|2028|516|6714|Friday|2028Q3|N|Y|N|2462016|2462259|2461650|2461925|N|N|N|N|N| +2462017|AAAAAAAABEBJFCAA|2028-09-02|1544|6714|516|2028|6|9|2|3|2028|516|6714|Saturday|2028Q3|N|Y|N|2462016|2462259|2461651|2461926|N|N|N|N|N| +2462018|AAAAAAAACEBJFCAA|2028-09-03|1544|6714|516|2028|0|9|3|3|2028|516|6714|Sunday|2028Q3|N|N|N|2462016|2462259|2461652|2461927|N|N|N|N|N| +2462019|AAAAAAAADEBJFCAA|2028-09-04|1544|6714|516|2028|1|9|4|3|2028|516|6714|Monday|2028Q3|N|N|N|2462016|2462259|2461653|2461928|N|N|N|N|N| +2462020|AAAAAAAAEEBJFCAA|2028-09-05|1544|6715|516|2028|2|9|5|3|2028|516|6715|Tuesday|2028Q3|N|N|N|2462016|2462259|2461654|2461929|N|N|N|N|N| +2462021|AAAAAAAAFEBJFCAA|2028-09-06|1544|6715|516|2028|3|9|6|3|2028|516|6715|Wednesday|2028Q3|N|N|N|2462016|2462259|2461655|2461930|N|N|N|N|N| +2462022|AAAAAAAAGEBJFCAA|2028-09-07|1544|6715|516|2028|4|9|7|3|2028|516|6715|Thursday|2028Q3|N|N|N|2462016|2462259|2461656|2461931|N|N|N|N|N| +2462023|AAAAAAAAHEBJFCAA|2028-09-08|1544|6715|516|2028|5|9|8|3|2028|516|6715|Friday|2028Q3|N|Y|N|2462016|2462259|2461657|2461932|N|N|N|N|N| +2462024|AAAAAAAAIEBJFCAA|2028-09-09|1544|6715|516|2028|6|9|9|3|2028|516|6715|Saturday|2028Q3|N|Y|N|2462016|2462259|2461658|2461933|N|N|N|N|N| +2462025|AAAAAAAAJEBJFCAA|2028-09-10|1544|6715|516|2028|0|9|10|3|2028|516|6715|Sunday|2028Q3|N|N|N|2462016|2462259|2461659|2461934|N|N|N|N|N| +2462026|AAAAAAAAKEBJFCAA|2028-09-11|1544|6715|516|2028|1|9|11|3|2028|516|6715|Monday|2028Q3|N|N|N|2462016|2462259|2461660|2461935|N|N|N|N|N| +2462027|AAAAAAAALEBJFCAA|2028-09-12|1544|6716|516|2028|2|9|12|3|2028|516|6716|Tuesday|2028Q3|N|N|N|2462016|2462259|2461661|2461936|N|N|N|N|N| +2462028|AAAAAAAAMEBJFCAA|2028-09-13|1544|6716|516|2028|3|9|13|3|2028|516|6716|Wednesday|2028Q3|N|N|N|2462016|2462259|2461662|2461937|N|N|N|N|N| +2462029|AAAAAAAANEBJFCAA|2028-09-14|1544|6716|516|2028|4|9|14|3|2028|516|6716|Thursday|2028Q3|N|N|N|2462016|2462259|2461663|2461938|N|N|N|N|N| +2462030|AAAAAAAAOEBJFCAA|2028-09-15|1544|6716|516|2028|5|9|15|3|2028|516|6716|Friday|2028Q3|N|Y|N|2462016|2462259|2461664|2461939|N|N|N|N|N| +2462031|AAAAAAAAPEBJFCAA|2028-09-16|1544|6716|516|2028|6|9|16|3|2028|516|6716|Saturday|2028Q3|N|Y|N|2462016|2462259|2461665|2461940|N|N|N|N|N| +2462032|AAAAAAAAAFBJFCAA|2028-09-17|1544|6716|516|2028|0|9|17|3|2028|516|6716|Sunday|2028Q3|N|N|N|2462016|2462259|2461666|2461941|N|N|N|N|N| +2462033|AAAAAAAABFBJFCAA|2028-09-18|1544|6716|516|2028|1|9|18|3|2028|516|6716|Monday|2028Q3|N|N|N|2462016|2462259|2461667|2461942|N|N|N|N|N| +2462034|AAAAAAAACFBJFCAA|2028-09-19|1544|6717|516|2028|2|9|19|3|2028|516|6717|Tuesday|2028Q3|N|N|N|2462016|2462259|2461668|2461943|N|N|N|N|N| +2462035|AAAAAAAADFBJFCAA|2028-09-20|1544|6717|516|2028|3|9|20|3|2028|516|6717|Wednesday|2028Q3|N|N|N|2462016|2462259|2461669|2461944|N|N|N|N|N| +2462036|AAAAAAAAEFBJFCAA|2028-09-21|1544|6717|516|2028|4|9|21|3|2028|516|6717|Thursday|2028Q3|N|N|N|2462016|2462259|2461670|2461945|N|N|N|N|N| +2462037|AAAAAAAAFFBJFCAA|2028-09-22|1544|6717|516|2028|5|9|22|3|2028|516|6717|Friday|2028Q3|N|Y|N|2462016|2462259|2461671|2461946|N|N|N|N|N| +2462038|AAAAAAAAGFBJFCAA|2028-09-23|1544|6717|516|2028|6|9|23|3|2028|516|6717|Saturday|2028Q3|N|Y|N|2462016|2462259|2461672|2461947|N|N|N|N|N| +2462039|AAAAAAAAHFBJFCAA|2028-09-24|1544|6717|516|2028|0|9|24|3|2028|516|6717|Sunday|2028Q3|N|N|N|2462016|2462259|2461673|2461948|N|N|N|N|N| +2462040|AAAAAAAAIFBJFCAA|2028-09-25|1544|6717|516|2028|1|9|25|3|2028|516|6717|Monday|2028Q3|N|N|N|2462016|2462259|2461674|2461949|N|N|N|N|N| +2462041|AAAAAAAAJFBJFCAA|2028-09-26|1544|6718|516|2028|2|9|26|3|2028|516|6718|Tuesday|2028Q3|N|N|N|2462016|2462259|2461675|2461950|N|N|N|N|N| +2462042|AAAAAAAAKFBJFCAA|2028-09-27|1544|6718|516|2028|3|9|27|3|2028|516|6718|Wednesday|2028Q3|N|N|N|2462016|2462259|2461676|2461951|N|N|N|N|N| +2462043|AAAAAAAALFBJFCAA|2028-09-28|1544|6718|516|2028|4|9|28|3|2028|516|6718|Thursday|2028Q3|N|N|N|2462016|2462259|2461677|2461952|N|N|N|N|N| +2462044|AAAAAAAAMFBJFCAA|2028-09-29|1544|6718|516|2028|5|9|29|3|2028|516|6718|Friday|2028Q3|N|Y|N|2462016|2462259|2461678|2461953|N|N|N|N|N| +2462045|AAAAAAAANFBJFCAA|2028-09-30|1544|6718|516|2028|6|9|30|3|2028|516|6718|Saturday|2028Q3|N|Y|N|2462016|2462259|2461679|2461954|N|N|N|N|N| +2462046|AAAAAAAAOFBJFCAA|2028-10-01|1545|6718|516|2028|0|10|1|4|2028|516|6718|Sunday|2028Q4|N|N|N|2462046|2462319|2461680|2461954|N|N|N|N|N| +2462047|AAAAAAAAPFBJFCAA|2028-10-02|1545|6718|516|2028|1|10|2|4|2028|516|6718|Monday|2028Q4|N|N|N|2462046|2462319|2461681|2461955|N|N|N|N|N| +2462048|AAAAAAAAAGBJFCAA|2028-10-03|1545|6719|516|2028|2|10|3|4|2028|516|6719|Tuesday|2028Q4|N|N|N|2462046|2462319|2461682|2461956|N|N|N|N|N| +2462049|AAAAAAAABGBJFCAA|2028-10-04|1545|6719|516|2028|3|10|4|4|2028|516|6719|Wednesday|2028Q4|N|N|N|2462046|2462319|2461683|2461957|N|N|N|N|N| +2462050|AAAAAAAACGBJFCAA|2028-10-05|1545|6719|516|2028|4|10|5|4|2028|516|6719|Thursday|2028Q4|N|N|N|2462046|2462319|2461684|2461958|N|N|N|N|N| +2462051|AAAAAAAADGBJFCAA|2028-10-06|1545|6719|516|2028|5|10|6|4|2028|516|6719|Friday|2028Q4|N|Y|N|2462046|2462319|2461685|2461959|N|N|N|N|N| +2462052|AAAAAAAAEGBJFCAA|2028-10-07|1545|6719|516|2028|6|10|7|4|2028|516|6719|Saturday|2028Q4|N|Y|N|2462046|2462319|2461686|2461960|N|N|N|N|N| +2462053|AAAAAAAAFGBJFCAA|2028-10-08|1545|6719|516|2028|0|10|8|4|2028|516|6719|Sunday|2028Q4|N|N|N|2462046|2462319|2461687|2461961|N|N|N|N|N| +2462054|AAAAAAAAGGBJFCAA|2028-10-09|1545|6719|516|2028|1|10|9|4|2028|516|6719|Monday|2028Q4|N|N|N|2462046|2462319|2461688|2461962|N|N|N|N|N| +2462055|AAAAAAAAHGBJFCAA|2028-10-10|1545|6720|516|2028|2|10|10|4|2028|516|6720|Tuesday|2028Q4|N|N|N|2462046|2462319|2461689|2461963|N|N|N|N|N| +2462056|AAAAAAAAIGBJFCAA|2028-10-11|1545|6720|516|2028|3|10|11|4|2028|516|6720|Wednesday|2028Q4|N|N|N|2462046|2462319|2461690|2461964|N|N|N|N|N| +2462057|AAAAAAAAJGBJFCAA|2028-10-12|1545|6720|516|2028|4|10|12|4|2028|516|6720|Thursday|2028Q4|N|N|N|2462046|2462319|2461691|2461965|N|N|N|N|N| +2462058|AAAAAAAAKGBJFCAA|2028-10-13|1545|6720|516|2028|5|10|13|4|2028|516|6720|Friday|2028Q4|N|Y|N|2462046|2462319|2461692|2461966|N|N|N|N|N| +2462059|AAAAAAAALGBJFCAA|2028-10-14|1545|6720|516|2028|6|10|14|4|2028|516|6720|Saturday|2028Q4|N|Y|N|2462046|2462319|2461693|2461967|N|N|N|N|N| +2462060|AAAAAAAAMGBJFCAA|2028-10-15|1545|6720|516|2028|0|10|15|4|2028|516|6720|Sunday|2028Q4|N|N|N|2462046|2462319|2461694|2461968|N|N|N|N|N| +2462061|AAAAAAAANGBJFCAA|2028-10-16|1545|6720|516|2028|1|10|16|4|2028|516|6720|Monday|2028Q4|N|N|N|2462046|2462319|2461695|2461969|N|N|N|N|N| +2462062|AAAAAAAAOGBJFCAA|2028-10-17|1545|6721|516|2028|2|10|17|4|2028|516|6721|Tuesday|2028Q4|N|N|N|2462046|2462319|2461696|2461970|N|N|N|N|N| +2462063|AAAAAAAAPGBJFCAA|2028-10-18|1545|6721|516|2028|3|10|18|4|2028|516|6721|Wednesday|2028Q4|N|N|N|2462046|2462319|2461697|2461971|N|N|N|N|N| +2462064|AAAAAAAAAHBJFCAA|2028-10-19|1545|6721|516|2028|4|10|19|4|2028|516|6721|Thursday|2028Q4|N|N|N|2462046|2462319|2461698|2461972|N|N|N|N|N| +2462065|AAAAAAAABHBJFCAA|2028-10-20|1545|6721|516|2028|5|10|20|4|2028|516|6721|Friday|2028Q4|N|Y|N|2462046|2462319|2461699|2461973|N|N|N|N|N| +2462066|AAAAAAAACHBJFCAA|2028-10-21|1545|6721|516|2028|6|10|21|4|2028|516|6721|Saturday|2028Q4|N|Y|N|2462046|2462319|2461700|2461974|N|N|N|N|N| +2462067|AAAAAAAADHBJFCAA|2028-10-22|1545|6721|516|2028|0|10|22|4|2028|516|6721|Sunday|2028Q4|N|N|N|2462046|2462319|2461701|2461975|N|N|N|N|N| +2462068|AAAAAAAAEHBJFCAA|2028-10-23|1545|6721|516|2028|1|10|23|4|2028|516|6721|Monday|2028Q4|N|N|N|2462046|2462319|2461702|2461976|N|N|N|N|N| +2462069|AAAAAAAAFHBJFCAA|2028-10-24|1545|6722|516|2028|2|10|24|4|2028|516|6722|Tuesday|2028Q4|N|N|N|2462046|2462319|2461703|2461977|N|N|N|N|N| +2462070|AAAAAAAAGHBJFCAA|2028-10-25|1545|6722|516|2028|3|10|25|4|2028|516|6722|Wednesday|2028Q4|N|N|N|2462046|2462319|2461704|2461978|N|N|N|N|N| +2462071|AAAAAAAAHHBJFCAA|2028-10-26|1545|6722|516|2028|4|10|26|4|2028|516|6722|Thursday|2028Q4|N|N|N|2462046|2462319|2461705|2461979|N|N|N|N|N| +2462072|AAAAAAAAIHBJFCAA|2028-10-27|1545|6722|516|2028|5|10|27|4|2028|516|6722|Friday|2028Q4|N|Y|N|2462046|2462319|2461706|2461980|N|N|N|N|N| +2462073|AAAAAAAAJHBJFCAA|2028-10-28|1545|6722|516|2028|6|10|28|4|2028|516|6722|Saturday|2028Q4|N|Y|N|2462046|2462319|2461707|2461981|N|N|N|N|N| +2462074|AAAAAAAAKHBJFCAA|2028-10-29|1545|6722|516|2028|0|10|29|4|2028|516|6722|Sunday|2028Q4|N|N|N|2462046|2462319|2461708|2461982|N|N|N|N|N| +2462075|AAAAAAAALHBJFCAA|2028-10-30|1545|6722|516|2028|1|10|30|4|2028|516|6722|Monday|2028Q4|N|N|N|2462046|2462319|2461709|2461983|N|N|N|N|N| +2462076|AAAAAAAAMHBJFCAA|2028-10-31|1545|6723|516|2028|2|10|31|4|2028|516|6723|Tuesday|2028Q4|N|N|N|2462046|2462319|2461710|2461984|N|N|N|N|N| +2462077|AAAAAAAANHBJFCAA|2028-11-01|1546|6723|516|2028|3|11|1|4|2028|516|6723|Wednesday|2028Q4|N|N|N|2462077|2462381|2461711|2461985|N|N|N|N|N| +2462078|AAAAAAAAOHBJFCAA|2028-11-02|1546|6723|516|2028|4|11|2|4|2028|516|6723|Thursday|2028Q4|N|N|N|2462077|2462381|2461712|2461986|N|N|N|N|N| +2462079|AAAAAAAAPHBJFCAA|2028-11-03|1546|6723|516|2028|5|11|3|4|2028|516|6723|Friday|2028Q4|N|Y|N|2462077|2462381|2461713|2461987|N|N|N|N|N| +2462080|AAAAAAAAAIBJFCAA|2028-11-04|1546|6723|516|2028|6|11|4|4|2028|516|6723|Saturday|2028Q4|N|Y|N|2462077|2462381|2461714|2461988|N|N|N|N|N| +2462081|AAAAAAAABIBJFCAA|2028-11-05|1546|6723|516|2028|0|11|5|4|2028|516|6723|Sunday|2028Q4|N|N|N|2462077|2462381|2461715|2461989|N|N|N|N|N| +2462082|AAAAAAAACIBJFCAA|2028-11-06|1546|6723|516|2028|1|11|6|4|2028|516|6723|Monday|2028Q4|N|N|N|2462077|2462381|2461716|2461990|N|N|N|N|N| +2462083|AAAAAAAADIBJFCAA|2028-11-07|1546|6724|516|2028|2|11|7|4|2028|516|6724|Tuesday|2028Q4|N|N|N|2462077|2462381|2461717|2461991|N|N|N|N|N| +2462084|AAAAAAAAEIBJFCAA|2028-11-08|1546|6724|516|2028|3|11|8|4|2028|516|6724|Wednesday|2028Q4|N|N|N|2462077|2462381|2461718|2461992|N|N|N|N|N| +2462085|AAAAAAAAFIBJFCAA|2028-11-09|1546|6724|516|2028|4|11|9|4|2028|516|6724|Thursday|2028Q4|N|N|N|2462077|2462381|2461719|2461993|N|N|N|N|N| +2462086|AAAAAAAAGIBJFCAA|2028-11-10|1546|6724|516|2028|5|11|10|4|2028|516|6724|Friday|2028Q4|N|Y|N|2462077|2462381|2461720|2461994|N|N|N|N|N| +2462087|AAAAAAAAHIBJFCAA|2028-11-11|1546|6724|516|2028|6|11|11|4|2028|516|6724|Saturday|2028Q4|N|Y|N|2462077|2462381|2461721|2461995|N|N|N|N|N| +2462088|AAAAAAAAIIBJFCAA|2028-11-12|1546|6724|516|2028|0|11|12|4|2028|516|6724|Sunday|2028Q4|N|N|N|2462077|2462381|2461722|2461996|N|N|N|N|N| +2462089|AAAAAAAAJIBJFCAA|2028-11-13|1546|6724|516|2028|1|11|13|4|2028|516|6724|Monday|2028Q4|N|N|N|2462077|2462381|2461723|2461997|N|N|N|N|N| +2462090|AAAAAAAAKIBJFCAA|2028-11-14|1546|6725|516|2028|2|11|14|4|2028|516|6725|Tuesday|2028Q4|N|N|N|2462077|2462381|2461724|2461998|N|N|N|N|N| +2462091|AAAAAAAALIBJFCAA|2028-11-15|1546|6725|516|2028|3|11|15|4|2028|516|6725|Wednesday|2028Q4|N|N|N|2462077|2462381|2461725|2461999|N|N|N|N|N| +2462092|AAAAAAAAMIBJFCAA|2028-11-16|1546|6725|516|2028|4|11|16|4|2028|516|6725|Thursday|2028Q4|N|N|N|2462077|2462381|2461726|2462000|N|N|N|N|N| +2462093|AAAAAAAANIBJFCAA|2028-11-17|1546|6725|516|2028|5|11|17|4|2028|516|6725|Friday|2028Q4|N|Y|N|2462077|2462381|2461727|2462001|N|N|N|N|N| +2462094|AAAAAAAAOIBJFCAA|2028-11-18|1546|6725|516|2028|6|11|18|4|2028|516|6725|Saturday|2028Q4|N|Y|N|2462077|2462381|2461728|2462002|N|N|N|N|N| +2462095|AAAAAAAAPIBJFCAA|2028-11-19|1546|6725|516|2028|0|11|19|4|2028|516|6725|Sunday|2028Q4|N|N|N|2462077|2462381|2461729|2462003|N|N|N|N|N| +2462096|AAAAAAAAAJBJFCAA|2028-11-20|1546|6725|516|2028|1|11|20|4|2028|516|6725|Monday|2028Q4|N|N|N|2462077|2462381|2461730|2462004|N|N|N|N|N| +2462097|AAAAAAAABJBJFCAA|2028-11-21|1546|6726|516|2028|2|11|21|4|2028|516|6726|Tuesday|2028Q4|N|N|N|2462077|2462381|2461731|2462005|N|N|N|N|N| +2462098|AAAAAAAACJBJFCAA|2028-11-22|1546|6726|516|2028|3|11|22|4|2028|516|6726|Wednesday|2028Q4|N|N|N|2462077|2462381|2461732|2462006|N|N|N|N|N| +2462099|AAAAAAAADJBJFCAA|2028-11-23|1546|6726|516|2028|4|11|23|4|2028|516|6726|Thursday|2028Q4|N|N|N|2462077|2462381|2461733|2462007|N|N|N|N|N| +2462100|AAAAAAAAEJBJFCAA|2028-11-24|1546|6726|516|2028|5|11|24|4|2028|516|6726|Friday|2028Q4|N|Y|N|2462077|2462381|2461734|2462008|N|N|N|N|N| +2462101|AAAAAAAAFJBJFCAA|2028-11-25|1546|6726|516|2028|6|11|25|4|2028|516|6726|Saturday|2028Q4|N|Y|N|2462077|2462381|2461735|2462009|N|N|N|N|N| +2462102|AAAAAAAAGJBJFCAA|2028-11-26|1546|6726|516|2028|0|11|26|4|2028|516|6726|Sunday|2028Q4|N|N|N|2462077|2462381|2461736|2462010|N|N|N|N|N| +2462103|AAAAAAAAHJBJFCAA|2028-11-27|1546|6726|516|2028|1|11|27|4|2028|516|6726|Monday|2028Q4|N|N|N|2462077|2462381|2461737|2462011|N|N|N|N|N| +2462104|AAAAAAAAIJBJFCAA|2028-11-28|1546|6727|516|2028|2|11|28|4|2028|516|6727|Tuesday|2028Q4|N|N|N|2462077|2462381|2461738|2462012|N|N|N|N|N| +2462105|AAAAAAAAJJBJFCAA|2028-11-29|1546|6727|516|2028|3|11|29|4|2028|516|6727|Wednesday|2028Q4|N|N|N|2462077|2462381|2461739|2462013|N|N|N|N|N| +2462106|AAAAAAAAKJBJFCAA|2028-11-30|1546|6727|516|2028|4|11|30|4|2028|516|6727|Thursday|2028Q4|N|N|N|2462077|2462381|2461740|2462014|N|N|N|N|N| +2462107|AAAAAAAALJBJFCAA|2028-12-01|1547|6727|517|2028|5|12|1|4|2028|517|6727|Friday|2028Q4|N|Y|N|2462107|2462441|2461741|2462015|N|N|N|N|N| +2462108|AAAAAAAAMJBJFCAA|2028-12-02|1547|6727|517|2028|6|12|2|4|2028|517|6727|Saturday|2028Q4|N|Y|N|2462107|2462441|2461742|2462016|N|N|N|N|N| +2462109|AAAAAAAANJBJFCAA|2028-12-03|1547|6727|517|2028|0|12|3|4|2028|517|6727|Sunday|2028Q4|N|N|N|2462107|2462441|2461743|2462017|N|N|N|N|N| +2462110|AAAAAAAAOJBJFCAA|2028-12-04|1547|6727|517|2028|1|12|4|4|2028|517|6727|Monday|2028Q4|N|N|N|2462107|2462441|2461744|2462018|N|N|N|N|N| +2462111|AAAAAAAAPJBJFCAA|2028-12-05|1547|6728|517|2028|2|12|5|4|2028|517|6728|Tuesday|2028Q4|N|N|N|2462107|2462441|2461745|2462019|N|N|N|N|N| +2462112|AAAAAAAAAKBJFCAA|2028-12-06|1547|6728|517|2028|3|12|6|4|2028|517|6728|Wednesday|2028Q4|N|N|N|2462107|2462441|2461746|2462020|N|N|N|N|N| +2462113|AAAAAAAABKBJFCAA|2028-12-07|1547|6728|517|2028|4|12|7|4|2028|517|6728|Thursday|2028Q4|N|N|N|2462107|2462441|2461747|2462021|N|N|N|N|N| +2462114|AAAAAAAACKBJFCAA|2028-12-08|1547|6728|517|2028|5|12|8|4|2028|517|6728|Friday|2028Q4|N|Y|N|2462107|2462441|2461748|2462022|N|N|N|N|N| +2462115|AAAAAAAADKBJFCAA|2028-12-09|1547|6728|517|2028|6|12|9|4|2028|517|6728|Saturday|2028Q4|N|Y|N|2462107|2462441|2461749|2462023|N|N|N|N|N| +2462116|AAAAAAAAEKBJFCAA|2028-12-10|1547|6728|517|2028|0|12|10|4|2028|517|6728|Sunday|2028Q4|N|N|N|2462107|2462441|2461750|2462024|N|N|N|N|N| +2462117|AAAAAAAAFKBJFCAA|2028-12-11|1547|6728|517|2028|1|12|11|4|2028|517|6728|Monday|2028Q4|N|N|N|2462107|2462441|2461751|2462025|N|N|N|N|N| +2462118|AAAAAAAAGKBJFCAA|2028-12-12|1547|6729|517|2028|2|12|12|4|2028|517|6729|Tuesday|2028Q4|N|N|N|2462107|2462441|2461752|2462026|N|N|N|N|N| +2462119|AAAAAAAAHKBJFCAA|2028-12-13|1547|6729|517|2028|3|12|13|4|2028|517|6729|Wednesday|2028Q4|N|N|N|2462107|2462441|2461753|2462027|N|N|N|N|N| +2462120|AAAAAAAAIKBJFCAA|2028-12-14|1547|6729|517|2028|4|12|14|4|2028|517|6729|Thursday|2028Q4|N|N|N|2462107|2462441|2461754|2462028|N|N|N|N|N| +2462121|AAAAAAAAJKBJFCAA|2028-12-15|1547|6729|517|2028|5|12|15|4|2028|517|6729|Friday|2028Q4|N|Y|N|2462107|2462441|2461755|2462029|N|N|N|N|N| +2462122|AAAAAAAAKKBJFCAA|2028-12-16|1547|6729|517|2028|6|12|16|4|2028|517|6729|Saturday|2028Q4|N|Y|N|2462107|2462441|2461756|2462030|N|N|N|N|N| +2462123|AAAAAAAALKBJFCAA|2028-12-17|1547|6729|517|2028|0|12|17|4|2028|517|6729|Sunday|2028Q4|N|N|N|2462107|2462441|2461757|2462031|N|N|N|N|N| +2462124|AAAAAAAAMKBJFCAA|2028-12-18|1547|6729|517|2028|1|12|18|4|2028|517|6729|Monday|2028Q4|N|N|N|2462107|2462441|2461758|2462032|N|N|N|N|N| +2462125|AAAAAAAANKBJFCAA|2028-12-19|1547|6730|517|2028|2|12|19|4|2028|517|6730|Tuesday|2028Q4|N|N|N|2462107|2462441|2461759|2462033|N|N|N|N|N| +2462126|AAAAAAAAOKBJFCAA|2028-12-20|1547|6730|517|2028|3|12|20|4|2028|517|6730|Wednesday|2028Q4|N|N|N|2462107|2462441|2461760|2462034|N|N|N|N|N| +2462127|AAAAAAAAPKBJFCAA|2028-12-21|1547|6730|517|2028|4|12|21|4|2028|517|6730|Thursday|2028Q4|N|N|N|2462107|2462441|2461761|2462035|N|N|N|N|N| +2462128|AAAAAAAAALBJFCAA|2028-12-22|1547|6730|517|2028|5|12|22|4|2028|517|6730|Friday|2028Q4|N|Y|N|2462107|2462441|2461762|2462036|N|N|N|N|N| +2462129|AAAAAAAABLBJFCAA|2028-12-23|1547|6730|517|2028|6|12|23|4|2028|517|6730|Saturday|2028Q4|N|Y|N|2462107|2462441|2461763|2462037|N|N|N|N|N| +2462130|AAAAAAAACLBJFCAA|2028-12-24|1547|6730|517|2028|0|12|24|4|2028|517|6730|Sunday|2028Q4|N|N|N|2462107|2462441|2461764|2462038|N|N|N|N|N| +2462131|AAAAAAAADLBJFCAA|2028-12-25|1547|6730|517|2028|1|12|25|4|2028|517|6730|Monday|2028Q4|Y|N|N|2462107|2462441|2461765|2462039|N|N|N|N|N| +2462132|AAAAAAAAELBJFCAA|2028-12-26|1547|6731|517|2028|2|12|26|4|2028|517|6731|Tuesday|2028Q4|N|N|Y|2462107|2462441|2461766|2462040|N|N|N|N|N| +2462133|AAAAAAAAFLBJFCAA|2028-12-27|1547|6731|517|2028|3|12|27|4|2028|517|6731|Wednesday|2028Q4|N|N|N|2462107|2462441|2461767|2462041|N|N|N|N|N| +2462134|AAAAAAAAGLBJFCAA|2028-12-28|1547|6731|517|2028|4|12|28|4|2028|517|6731|Thursday|2028Q4|N|N|N|2462107|2462441|2461768|2462042|N|N|N|N|N| +2462135|AAAAAAAAHLBJFCAA|2028-12-29|1547|6731|517|2028|5|12|29|4|2028|517|6731|Friday|2028Q4|N|Y|N|2462107|2462441|2461769|2462043|N|N|N|N|N| +2462136|AAAAAAAAILBJFCAA|2028-12-30|1547|6731|517|2028|6|12|30|4|2028|517|6731|Saturday|2028Q4|N|Y|N|2462107|2462441|2461770|2462044|N|N|N|N|N| +2462137|AAAAAAAAJLBJFCAA|2028-12-31|1547|6731|517|2028|0|12|31|4|2028|517|6731|Sunday|2028Q4|Y|N|N|2462107|2462441|2461771|2462045|N|N|N|N|N| +2462138|AAAAAAAAKLBJFCAA|2029-01-01|1548|6731|517|2029|1|1|1|1|2029|517|6731|Monday|2029Q1|Y|N|Y|2462138|2462137|2461772|2462046|N|N|N|N|N| +2462139|AAAAAAAALLBJFCAA|2029-01-02|1548|6732|517|2029|2|1|2|1|2029|517|6732|Tuesday|2029Q1|N|N|Y|2462138|2462137|2461773|2462047|N|N|N|N|N| +2462140|AAAAAAAAMLBJFCAA|2029-01-03|1548|6732|517|2029|3|1|3|1|2029|517|6732|Wednesday|2029Q1|N|N|N|2462138|2462137|2461774|2462048|N|N|N|N|N| +2462141|AAAAAAAANLBJFCAA|2029-01-04|1548|6732|517|2029|4|1|4|1|2029|517|6732|Thursday|2029Q1|N|N|N|2462138|2462137|2461775|2462049|N|N|N|N|N| +2462142|AAAAAAAAOLBJFCAA|2029-01-05|1548|6732|517|2029|5|1|5|1|2029|517|6732|Friday|2029Q1|N|Y|N|2462138|2462137|2461776|2462050|N|N|N|N|N| +2462143|AAAAAAAAPLBJFCAA|2029-01-06|1548|6732|517|2029|6|1|6|1|2029|517|6732|Saturday|2029Q1|N|Y|N|2462138|2462137|2461777|2462051|N|N|N|N|N| +2462144|AAAAAAAAAMBJFCAA|2029-01-07|1548|6732|517|2029|0|1|7|1|2029|517|6732|Sunday|2029Q1|N|N|N|2462138|2462137|2461778|2462052|N|N|N|N|N| +2462145|AAAAAAAABMBJFCAA|2029-01-08|1548|6732|517|2029|1|1|8|1|2029|517|6732|Monday|2029Q1|N|N|N|2462138|2462137|2461779|2462053|N|N|N|N|N| +2462146|AAAAAAAACMBJFCAA|2029-01-09|1548|6733|517|2029|2|1|9|1|2029|517|6733|Tuesday|2029Q1|N|N|N|2462138|2462137|2461780|2462054|N|N|N|N|N| +2462147|AAAAAAAADMBJFCAA|2029-01-10|1548|6733|517|2029|3|1|10|1|2029|517|6733|Wednesday|2029Q1|N|N|N|2462138|2462137|2461781|2462055|N|N|N|N|N| +2462148|AAAAAAAAEMBJFCAA|2029-01-11|1548|6733|517|2029|4|1|11|1|2029|517|6733|Thursday|2029Q1|N|N|N|2462138|2462137|2461782|2462056|N|N|N|N|N| +2462149|AAAAAAAAFMBJFCAA|2029-01-12|1548|6733|517|2029|5|1|12|1|2029|517|6733|Friday|2029Q1|N|Y|N|2462138|2462137|2461783|2462057|N|N|N|N|N| +2462150|AAAAAAAAGMBJFCAA|2029-01-13|1548|6733|517|2029|6|1|13|1|2029|517|6733|Saturday|2029Q1|N|Y|N|2462138|2462137|2461784|2462058|N|N|N|N|N| +2462151|AAAAAAAAHMBJFCAA|2029-01-14|1548|6733|517|2029|0|1|14|1|2029|517|6733|Sunday|2029Q1|N|N|N|2462138|2462137|2461785|2462059|N|N|N|N|N| +2462152|AAAAAAAAIMBJFCAA|2029-01-15|1548|6733|517|2029|1|1|15|1|2029|517|6733|Monday|2029Q1|N|N|N|2462138|2462137|2461786|2462060|N|N|N|N|N| +2462153|AAAAAAAAJMBJFCAA|2029-01-16|1548|6734|517|2029|2|1|16|1|2029|517|6734|Tuesday|2029Q1|N|N|N|2462138|2462137|2461787|2462061|N|N|N|N|N| +2462154|AAAAAAAAKMBJFCAA|2029-01-17|1548|6734|517|2029|3|1|17|1|2029|517|6734|Wednesday|2029Q1|N|N|N|2462138|2462137|2461788|2462062|N|N|N|N|N| +2462155|AAAAAAAALMBJFCAA|2029-01-18|1548|6734|517|2029|4|1|18|1|2029|517|6734|Thursday|2029Q1|N|N|N|2462138|2462137|2461789|2462063|N|N|N|N|N| +2462156|AAAAAAAAMMBJFCAA|2029-01-19|1548|6734|517|2029|5|1|19|1|2029|517|6734|Friday|2029Q1|N|Y|N|2462138|2462137|2461790|2462064|N|N|N|N|N| +2462157|AAAAAAAANMBJFCAA|2029-01-20|1548|6734|517|2029|6|1|20|1|2029|517|6734|Saturday|2029Q1|N|Y|N|2462138|2462137|2461791|2462065|N|N|N|N|N| +2462158|AAAAAAAAOMBJFCAA|2029-01-21|1548|6734|517|2029|0|1|21|1|2029|517|6734|Sunday|2029Q1|N|N|N|2462138|2462137|2461792|2462066|N|N|N|N|N| +2462159|AAAAAAAAPMBJFCAA|2029-01-22|1548|6734|517|2029|1|1|22|1|2029|517|6734|Monday|2029Q1|N|N|N|2462138|2462137|2461793|2462067|N|N|N|N|N| +2462160|AAAAAAAAANBJFCAA|2029-01-23|1548|6735|517|2029|2|1|23|1|2029|517|6735|Tuesday|2029Q1|N|N|N|2462138|2462137|2461794|2462068|N|N|N|N|N| +2462161|AAAAAAAABNBJFCAA|2029-01-24|1548|6735|517|2029|3|1|24|1|2029|517|6735|Wednesday|2029Q1|N|N|N|2462138|2462137|2461795|2462069|N|N|N|N|N| +2462162|AAAAAAAACNBJFCAA|2029-01-25|1548|6735|517|2029|4|1|25|1|2029|517|6735|Thursday|2029Q1|N|N|N|2462138|2462137|2461796|2462070|N|N|N|N|N| +2462163|AAAAAAAADNBJFCAA|2029-01-26|1548|6735|517|2029|5|1|26|1|2029|517|6735|Friday|2029Q1|N|Y|N|2462138|2462137|2461797|2462071|N|N|N|N|N| +2462164|AAAAAAAAENBJFCAA|2029-01-27|1548|6735|517|2029|6|1|27|1|2029|517|6735|Saturday|2029Q1|N|Y|N|2462138|2462137|2461798|2462072|N|N|N|N|N| +2462165|AAAAAAAAFNBJFCAA|2029-01-28|1548|6735|517|2029|0|1|28|1|2029|517|6735|Sunday|2029Q1|N|N|N|2462138|2462137|2461799|2462073|N|N|N|N|N| +2462166|AAAAAAAAGNBJFCAA|2029-01-29|1548|6735|517|2029|1|1|29|1|2029|517|6735|Monday|2029Q1|N|N|N|2462138|2462137|2461800|2462074|N|N|N|N|N| +2462167|AAAAAAAAHNBJFCAA|2029-01-30|1548|6736|517|2029|2|1|30|1|2029|517|6736|Tuesday|2029Q1|N|N|N|2462138|2462137|2461801|2462075|N|N|N|N|N| +2462168|AAAAAAAAINBJFCAA|2029-01-31|1548|6736|517|2029|3|1|31|1|2029|517|6736|Wednesday|2029Q1|N|N|N|2462138|2462137|2461802|2462076|N|N|N|N|N| +2462169|AAAAAAAAJNBJFCAA|2029-02-01|1549|6736|517|2029|4|2|1|1|2029|517|6736|Thursday|2029Q1|N|N|N|2462169|2462199|2461803|2462077|N|N|N|N|N| +2462170|AAAAAAAAKNBJFCAA|2029-02-02|1549|6736|517|2029|5|2|2|1|2029|517|6736|Friday|2029Q1|N|Y|N|2462169|2462199|2461804|2462078|N|N|N|N|N| +2462171|AAAAAAAALNBJFCAA|2029-02-03|1549|6736|517|2029|6|2|3|1|2029|517|6736|Saturday|2029Q1|N|Y|N|2462169|2462199|2461805|2462079|N|N|N|N|N| +2462172|AAAAAAAAMNBJFCAA|2029-02-04|1549|6736|517|2029|0|2|4|1|2029|517|6736|Sunday|2029Q1|N|N|N|2462169|2462199|2461806|2462080|N|N|N|N|N| +2462173|AAAAAAAANNBJFCAA|2029-02-05|1549|6736|517|2029|1|2|5|1|2029|517|6736|Monday|2029Q1|N|N|N|2462169|2462199|2461807|2462081|N|N|N|N|N| +2462174|AAAAAAAAONBJFCAA|2029-02-06|1549|6737|517|2029|2|2|6|1|2029|517|6737|Tuesday|2029Q1|N|N|N|2462169|2462199|2461808|2462082|N|N|N|N|N| +2462175|AAAAAAAAPNBJFCAA|2029-02-07|1549|6737|517|2029|3|2|7|1|2029|517|6737|Wednesday|2029Q1|N|N|N|2462169|2462199|2461809|2462083|N|N|N|N|N| +2462176|AAAAAAAAAOBJFCAA|2029-02-08|1549|6737|517|2029|4|2|8|1|2029|517|6737|Thursday|2029Q1|N|N|N|2462169|2462199|2461810|2462084|N|N|N|N|N| +2462177|AAAAAAAABOBJFCAA|2029-02-09|1549|6737|517|2029|5|2|9|1|2029|517|6737|Friday|2029Q1|N|Y|N|2462169|2462199|2461811|2462085|N|N|N|N|N| +2462178|AAAAAAAACOBJFCAA|2029-02-10|1549|6737|517|2029|6|2|10|1|2029|517|6737|Saturday|2029Q1|N|Y|N|2462169|2462199|2461812|2462086|N|N|N|N|N| +2462179|AAAAAAAADOBJFCAA|2029-02-11|1549|6737|517|2029|0|2|11|1|2029|517|6737|Sunday|2029Q1|N|N|N|2462169|2462199|2461813|2462087|N|N|N|N|N| +2462180|AAAAAAAAEOBJFCAA|2029-02-12|1549|6737|517|2029|1|2|12|1|2029|517|6737|Monday|2029Q1|N|N|N|2462169|2462199|2461814|2462088|N|N|N|N|N| +2462181|AAAAAAAAFOBJFCAA|2029-02-13|1549|6738|517|2029|2|2|13|1|2029|517|6738|Tuesday|2029Q1|N|N|N|2462169|2462199|2461815|2462089|N|N|N|N|N| +2462182|AAAAAAAAGOBJFCAA|2029-02-14|1549|6738|517|2029|3|2|14|1|2029|517|6738|Wednesday|2029Q1|N|N|N|2462169|2462199|2461816|2462090|N|N|N|N|N| +2462183|AAAAAAAAHOBJFCAA|2029-02-15|1549|6738|517|2029|4|2|15|1|2029|517|6738|Thursday|2029Q1|N|N|N|2462169|2462199|2461817|2462091|N|N|N|N|N| +2462184|AAAAAAAAIOBJFCAA|2029-02-16|1549|6738|517|2029|5|2|16|1|2029|517|6738|Friday|2029Q1|N|Y|N|2462169|2462199|2461818|2462092|N|N|N|N|N| +2462185|AAAAAAAAJOBJFCAA|2029-02-17|1549|6738|517|2029|6|2|17|1|2029|517|6738|Saturday|2029Q1|N|Y|N|2462169|2462199|2461819|2462093|N|N|N|N|N| +2462186|AAAAAAAAKOBJFCAA|2029-02-18|1549|6738|517|2029|0|2|18|1|2029|517|6738|Sunday|2029Q1|N|N|N|2462169|2462199|2461820|2462094|N|N|N|N|N| +2462187|AAAAAAAALOBJFCAA|2029-02-19|1549|6738|517|2029|1|2|19|1|2029|517|6738|Monday|2029Q1|N|N|N|2462169|2462199|2461821|2462095|N|N|N|N|N| +2462188|AAAAAAAAMOBJFCAA|2029-02-20|1549|6739|517|2029|2|2|20|1|2029|517|6739|Tuesday|2029Q1|N|N|N|2462169|2462199|2461822|2462096|N|N|N|N|N| +2462189|AAAAAAAANOBJFCAA|2029-02-21|1549|6739|517|2029|3|2|21|1|2029|517|6739|Wednesday|2029Q1|N|N|N|2462169|2462199|2461823|2462097|N|N|N|N|N| +2462190|AAAAAAAAOOBJFCAA|2029-02-22|1549|6739|517|2029|4|2|22|1|2029|517|6739|Thursday|2029Q1|N|N|N|2462169|2462199|2461824|2462098|N|N|N|N|N| +2462191|AAAAAAAAPOBJFCAA|2029-02-23|1549|6739|517|2029|5|2|23|1|2029|517|6739|Friday|2029Q1|N|Y|N|2462169|2462199|2461825|2462099|N|N|N|N|N| +2462192|AAAAAAAAAPBJFCAA|2029-02-24|1549|6739|517|2029|6|2|24|1|2029|517|6739|Saturday|2029Q1|N|Y|N|2462169|2462199|2461826|2462100|N|N|N|N|N| +2462193|AAAAAAAABPBJFCAA|2029-02-25|1549|6739|517|2029|0|2|25|1|2029|517|6739|Sunday|2029Q1|N|N|N|2462169|2462199|2461827|2462101|N|N|N|N|N| +2462194|AAAAAAAACPBJFCAA|2029-02-26|1549|6739|517|2029|1|2|26|1|2029|517|6739|Monday|2029Q1|N|N|N|2462169|2462199|2461828|2462102|N|N|N|N|N| +2462195|AAAAAAAADPBJFCAA|2029-02-27|1549|6740|517|2029|2|2|27|1|2029|517|6740|Tuesday|2029Q1|N|N|N|2462169|2462199|2461829|2462103|N|N|N|N|N| +2462196|AAAAAAAAEPBJFCAA|2029-02-28|1549|6740|517|2029|3|2|28|1|2029|517|6740|Wednesday|2029Q1|N|N|N|2462169|2462199|2461830|2462104|N|N|N|N|N| +2462197|AAAAAAAAFPBJFCAA|2029-03-01|1550|6740|518|2029|4|3|1|1|2029|518|6740|Thursday|2029Q1|N|N|N|2462197|2462255|2461832|2462105|N|N|N|N|N| +2462198|AAAAAAAAGPBJFCAA|2029-03-02|1550|6740|518|2029|5|3|2|1|2029|518|6740|Friday|2029Q1|N|Y|N|2462197|2462255|2461833|2462106|N|N|N|N|N| +2462199|AAAAAAAAHPBJFCAA|2029-03-03|1550|6740|518|2029|6|3|3|1|2029|518|6740|Saturday|2029Q1|N|Y|N|2462197|2462255|2461834|2462107|N|N|N|N|N| +2462200|AAAAAAAAIPBJFCAA|2029-03-04|1550|6740|518|2029|0|3|4|1|2029|518|6740|Sunday|2029Q1|N|N|N|2462197|2462255|2461835|2462108|N|N|N|N|N| +2462201|AAAAAAAAJPBJFCAA|2029-03-05|1550|6740|518|2029|1|3|5|1|2029|518|6740|Monday|2029Q1|N|N|N|2462197|2462255|2461836|2462109|N|N|N|N|N| +2462202|AAAAAAAAKPBJFCAA|2029-03-06|1550|6741|518|2029|2|3|6|1|2029|518|6741|Tuesday|2029Q1|N|N|N|2462197|2462255|2461837|2462110|N|N|N|N|N| +2462203|AAAAAAAALPBJFCAA|2029-03-07|1550|6741|518|2029|3|3|7|1|2029|518|6741|Wednesday|2029Q1|N|N|N|2462197|2462255|2461838|2462111|N|N|N|N|N| +2462204|AAAAAAAAMPBJFCAA|2029-03-08|1550|6741|518|2029|4|3|8|1|2029|518|6741|Thursday|2029Q1|N|N|N|2462197|2462255|2461839|2462112|N|N|N|N|N| +2462205|AAAAAAAANPBJFCAA|2029-03-09|1550|6741|518|2029|5|3|9|1|2029|518|6741|Friday|2029Q1|N|Y|N|2462197|2462255|2461840|2462113|N|N|N|N|N| +2462206|AAAAAAAAOPBJFCAA|2029-03-10|1550|6741|518|2029|6|3|10|1|2029|518|6741|Saturday|2029Q1|N|Y|N|2462197|2462255|2461841|2462114|N|N|N|N|N| +2462207|AAAAAAAAPPBJFCAA|2029-03-11|1550|6741|518|2029|0|3|11|1|2029|518|6741|Sunday|2029Q1|N|N|N|2462197|2462255|2461842|2462115|N|N|N|N|N| +2462208|AAAAAAAAAACJFCAA|2029-03-12|1550|6741|518|2029|1|3|12|1|2029|518|6741|Monday|2029Q1|N|N|N|2462197|2462255|2461843|2462116|N|N|N|N|N| +2462209|AAAAAAAABACJFCAA|2029-03-13|1550|6742|518|2029|2|3|13|1|2029|518|6742|Tuesday|2029Q1|N|N|N|2462197|2462255|2461844|2462117|N|N|N|N|N| +2462210|AAAAAAAACACJFCAA|2029-03-14|1550|6742|518|2029|3|3|14|1|2029|518|6742|Wednesday|2029Q1|N|N|N|2462197|2462255|2461845|2462118|N|N|N|N|N| +2462211|AAAAAAAADACJFCAA|2029-03-15|1550|6742|518|2029|4|3|15|1|2029|518|6742|Thursday|2029Q1|N|N|N|2462197|2462255|2461846|2462119|N|N|N|N|N| +2462212|AAAAAAAAEACJFCAA|2029-03-16|1550|6742|518|2029|5|3|16|1|2029|518|6742|Friday|2029Q1|N|Y|N|2462197|2462255|2461847|2462120|N|N|N|N|N| +2462213|AAAAAAAAFACJFCAA|2029-03-17|1550|6742|518|2029|6|3|17|1|2029|518|6742|Saturday|2029Q1|N|Y|N|2462197|2462255|2461848|2462121|N|N|N|N|N| +2462214|AAAAAAAAGACJFCAA|2029-03-18|1550|6742|518|2029|0|3|18|1|2029|518|6742|Sunday|2029Q1|N|N|N|2462197|2462255|2461849|2462122|N|N|N|N|N| +2462215|AAAAAAAAHACJFCAA|2029-03-19|1550|6742|518|2029|1|3|19|1|2029|518|6742|Monday|2029Q1|N|N|N|2462197|2462255|2461850|2462123|N|N|N|N|N| +2462216|AAAAAAAAIACJFCAA|2029-03-20|1550|6743|518|2029|2|3|20|1|2029|518|6743|Tuesday|2029Q1|N|N|N|2462197|2462255|2461851|2462124|N|N|N|N|N| +2462217|AAAAAAAAJACJFCAA|2029-03-21|1550|6743|518|2029|3|3|21|1|2029|518|6743|Wednesday|2029Q1|N|N|N|2462197|2462255|2461852|2462125|N|N|N|N|N| +2462218|AAAAAAAAKACJFCAA|2029-03-22|1550|6743|518|2029|4|3|22|1|2029|518|6743|Thursday|2029Q1|N|N|N|2462197|2462255|2461853|2462126|N|N|N|N|N| +2462219|AAAAAAAALACJFCAA|2029-03-23|1550|6743|518|2029|5|3|23|1|2029|518|6743|Friday|2029Q1|N|Y|N|2462197|2462255|2461854|2462127|N|N|N|N|N| +2462220|AAAAAAAAMACJFCAA|2029-03-24|1550|6743|518|2029|6|3|24|1|2029|518|6743|Saturday|2029Q1|N|Y|N|2462197|2462255|2461855|2462128|N|N|N|N|N| +2462221|AAAAAAAANACJFCAA|2029-03-25|1550|6743|518|2029|0|3|25|1|2029|518|6743|Sunday|2029Q1|N|N|N|2462197|2462255|2461856|2462129|N|N|N|N|N| +2462222|AAAAAAAAOACJFCAA|2029-03-26|1550|6743|518|2029|1|3|26|1|2029|518|6743|Monday|2029Q1|N|N|N|2462197|2462255|2461857|2462130|N|N|N|N|N| +2462223|AAAAAAAAPACJFCAA|2029-03-27|1550|6744|518|2029|2|3|27|1|2029|518|6744|Tuesday|2029Q1|N|N|N|2462197|2462255|2461858|2462131|N|N|N|N|N| +2462224|AAAAAAAAABCJFCAA|2029-03-28|1550|6744|518|2029|3|3|28|1|2029|518|6744|Wednesday|2029Q1|N|N|N|2462197|2462255|2461859|2462132|N|N|N|N|N| +2462225|AAAAAAAABBCJFCAA|2029-03-29|1550|6744|518|2029|4|3|29|1|2029|518|6744|Thursday|2029Q1|N|N|N|2462197|2462255|2461860|2462133|N|N|N|N|N| +2462226|AAAAAAAACBCJFCAA|2029-03-30|1550|6744|518|2029|5|3|30|1|2029|518|6744|Friday|2029Q1|N|Y|N|2462197|2462255|2461861|2462134|N|N|N|N|N| +2462227|AAAAAAAADBCJFCAA|2029-03-31|1550|6744|518|2029|6|3|31|1|2029|518|6744|Saturday|2029Q1|N|Y|N|2462197|2462255|2461862|2462135|N|N|N|N|N| +2462228|AAAAAAAAEBCJFCAA|2029-04-01|1551|6744|518|2029|0|4|1|1|2029|518|6744|Sunday|2029Q1|N|N|N|2462228|2462317|2461863|2462138|N|N|N|N|N| +2462229|AAAAAAAAFBCJFCAA|2029-04-02|1551|6744|518|2029|1|4|2|2|2029|518|6744|Monday|2029Q2|N|N|N|2462228|2462317|2461864|2462139|N|N|N|N|N| +2462230|AAAAAAAAGBCJFCAA|2029-04-03|1551|6745|518|2029|2|4|3|2|2029|518|6745|Tuesday|2029Q2|N|N|N|2462228|2462317|2461865|2462140|N|N|N|N|N| +2462231|AAAAAAAAHBCJFCAA|2029-04-04|1551|6745|518|2029|3|4|4|2|2029|518|6745|Wednesday|2029Q2|N|N|N|2462228|2462317|2461866|2462141|N|N|N|N|N| +2462232|AAAAAAAAIBCJFCAA|2029-04-05|1551|6745|518|2029|4|4|5|2|2029|518|6745|Thursday|2029Q2|N|N|N|2462228|2462317|2461867|2462142|N|N|N|N|N| +2462233|AAAAAAAAJBCJFCAA|2029-04-06|1551|6745|518|2029|5|4|6|2|2029|518|6745|Friday|2029Q2|N|Y|N|2462228|2462317|2461868|2462143|N|N|N|N|N| +2462234|AAAAAAAAKBCJFCAA|2029-04-07|1551|6745|518|2029|6|4|7|2|2029|518|6745|Saturday|2029Q2|N|Y|N|2462228|2462317|2461869|2462144|N|N|N|N|N| +2462235|AAAAAAAALBCJFCAA|2029-04-08|1551|6745|518|2029|0|4|8|2|2029|518|6745|Sunday|2029Q2|N|N|N|2462228|2462317|2461870|2462145|N|N|N|N|N| +2462236|AAAAAAAAMBCJFCAA|2029-04-09|1551|6745|518|2029|1|4|9|2|2029|518|6745|Monday|2029Q2|N|N|N|2462228|2462317|2461871|2462146|N|N|N|N|N| +2462237|AAAAAAAANBCJFCAA|2029-04-10|1551|6746|518|2029|2|4|10|2|2029|518|6746|Tuesday|2029Q2|N|N|N|2462228|2462317|2461872|2462147|N|N|N|N|N| +2462238|AAAAAAAAOBCJFCAA|2029-04-11|1551|6746|518|2029|3|4|11|2|2029|518|6746|Wednesday|2029Q2|N|N|N|2462228|2462317|2461873|2462148|N|N|N|N|N| +2462239|AAAAAAAAPBCJFCAA|2029-04-12|1551|6746|518|2029|4|4|12|2|2029|518|6746|Thursday|2029Q2|N|N|N|2462228|2462317|2461874|2462149|N|N|N|N|N| +2462240|AAAAAAAAACCJFCAA|2029-04-13|1551|6746|518|2029|5|4|13|2|2029|518|6746|Friday|2029Q2|N|Y|N|2462228|2462317|2461875|2462150|N|N|N|N|N| +2462241|AAAAAAAABCCJFCAA|2029-04-14|1551|6746|518|2029|6|4|14|2|2029|518|6746|Saturday|2029Q2|N|Y|N|2462228|2462317|2461876|2462151|N|N|N|N|N| +2462242|AAAAAAAACCCJFCAA|2029-04-15|1551|6746|518|2029|0|4|15|2|2029|518|6746|Sunday|2029Q2|N|N|N|2462228|2462317|2461877|2462152|N|N|N|N|N| +2462243|AAAAAAAADCCJFCAA|2029-04-16|1551|6746|518|2029|1|4|16|2|2029|518|6746|Monday|2029Q2|N|N|N|2462228|2462317|2461878|2462153|N|N|N|N|N| +2462244|AAAAAAAAECCJFCAA|2029-04-17|1551|6747|518|2029|2|4|17|2|2029|518|6747|Tuesday|2029Q2|N|N|N|2462228|2462317|2461879|2462154|N|N|N|N|N| +2462245|AAAAAAAAFCCJFCAA|2029-04-18|1551|6747|518|2029|3|4|18|2|2029|518|6747|Wednesday|2029Q2|N|N|N|2462228|2462317|2461880|2462155|N|N|N|N|N| +2462246|AAAAAAAAGCCJFCAA|2029-04-19|1551|6747|518|2029|4|4|19|2|2029|518|6747|Thursday|2029Q2|N|N|N|2462228|2462317|2461881|2462156|N|N|N|N|N| +2462247|AAAAAAAAHCCJFCAA|2029-04-20|1551|6747|518|2029|5|4|20|2|2029|518|6747|Friday|2029Q2|N|Y|N|2462228|2462317|2461882|2462157|N|N|N|N|N| +2462248|AAAAAAAAICCJFCAA|2029-04-21|1551|6747|518|2029|6|4|21|2|2029|518|6747|Saturday|2029Q2|N|Y|N|2462228|2462317|2461883|2462158|N|N|N|N|N| +2462249|AAAAAAAAJCCJFCAA|2029-04-22|1551|6747|518|2029|0|4|22|2|2029|518|6747|Sunday|2029Q2|N|N|N|2462228|2462317|2461884|2462159|N|N|N|N|N| +2462250|AAAAAAAAKCCJFCAA|2029-04-23|1551|6747|518|2029|1|4|23|2|2029|518|6747|Monday|2029Q2|N|N|N|2462228|2462317|2461885|2462160|N|N|N|N|N| +2462251|AAAAAAAALCCJFCAA|2029-04-24|1551|6748|518|2029|2|4|24|2|2029|518|6748|Tuesday|2029Q2|N|N|N|2462228|2462317|2461886|2462161|N|N|N|N|N| +2462252|AAAAAAAAMCCJFCAA|2029-04-25|1551|6748|518|2029|3|4|25|2|2029|518|6748|Wednesday|2029Q2|N|N|N|2462228|2462317|2461887|2462162|N|N|N|N|N| +2462253|AAAAAAAANCCJFCAA|2029-04-26|1551|6748|518|2029|4|4|26|2|2029|518|6748|Thursday|2029Q2|N|N|N|2462228|2462317|2461888|2462163|N|N|N|N|N| +2462254|AAAAAAAAOCCJFCAA|2029-04-27|1551|6748|518|2029|5|4|27|2|2029|518|6748|Friday|2029Q2|N|Y|N|2462228|2462317|2461889|2462164|N|N|N|N|N| +2462255|AAAAAAAAPCCJFCAA|2029-04-28|1551|6748|518|2029|6|4|28|2|2029|518|6748|Saturday|2029Q2|N|Y|N|2462228|2462317|2461890|2462165|N|N|N|N|N| +2462256|AAAAAAAAADCJFCAA|2029-04-29|1551|6748|518|2029|0|4|29|2|2029|518|6748|Sunday|2029Q2|N|N|N|2462228|2462317|2461891|2462166|N|N|N|N|N| +2462257|AAAAAAAABDCJFCAA|2029-04-30|1551|6748|518|2029|1|4|30|2|2029|518|6748|Monday|2029Q2|N|N|N|2462228|2462317|2461892|2462167|N|N|N|N|N| +2462258|AAAAAAAACDCJFCAA|2029-05-01|1552|6749|518|2029|2|5|1|2|2029|518|6749|Tuesday|2029Q2|N|N|N|2462258|2462377|2461893|2462168|N|N|N|N|N| +2462259|AAAAAAAADDCJFCAA|2029-05-02|1552|6749|518|2029|3|5|2|2|2029|518|6749|Wednesday|2029Q2|N|N|N|2462258|2462377|2461894|2462169|N|N|N|N|N| +2462260|AAAAAAAAEDCJFCAA|2029-05-03|1552|6749|518|2029|4|5|3|2|2029|518|6749|Thursday|2029Q2|N|N|N|2462258|2462377|2461895|2462170|N|N|N|N|N| +2462261|AAAAAAAAFDCJFCAA|2029-05-04|1552|6749|518|2029|5|5|4|2|2029|518|6749|Friday|2029Q2|N|Y|N|2462258|2462377|2461896|2462171|N|N|N|N|N| +2462262|AAAAAAAAGDCJFCAA|2029-05-05|1552|6749|518|2029|6|5|5|2|2029|518|6749|Saturday|2029Q2|N|Y|N|2462258|2462377|2461897|2462172|N|N|N|N|N| +2462263|AAAAAAAAHDCJFCAA|2029-05-06|1552|6749|518|2029|0|5|6|2|2029|518|6749|Sunday|2029Q2|N|N|N|2462258|2462377|2461898|2462173|N|N|N|N|N| +2462264|AAAAAAAAIDCJFCAA|2029-05-07|1552|6749|518|2029|1|5|7|2|2029|518|6749|Monday|2029Q2|N|N|N|2462258|2462377|2461899|2462174|N|N|N|N|N| +2462265|AAAAAAAAJDCJFCAA|2029-05-08|1552|6750|518|2029|2|5|8|2|2029|518|6750|Tuesday|2029Q2|N|N|N|2462258|2462377|2461900|2462175|N|N|N|N|N| +2462266|AAAAAAAAKDCJFCAA|2029-05-09|1552|6750|518|2029|3|5|9|2|2029|518|6750|Wednesday|2029Q2|N|N|N|2462258|2462377|2461901|2462176|N|N|N|N|N| +2462267|AAAAAAAALDCJFCAA|2029-05-10|1552|6750|518|2029|4|5|10|2|2029|518|6750|Thursday|2029Q2|N|N|N|2462258|2462377|2461902|2462177|N|N|N|N|N| +2462268|AAAAAAAAMDCJFCAA|2029-05-11|1552|6750|518|2029|5|5|11|2|2029|518|6750|Friday|2029Q2|N|Y|N|2462258|2462377|2461903|2462178|N|N|N|N|N| +2462269|AAAAAAAANDCJFCAA|2029-05-12|1552|6750|518|2029|6|5|12|2|2029|518|6750|Saturday|2029Q2|N|Y|N|2462258|2462377|2461904|2462179|N|N|N|N|N| +2462270|AAAAAAAAODCJFCAA|2029-05-13|1552|6750|518|2029|0|5|13|2|2029|518|6750|Sunday|2029Q2|N|N|N|2462258|2462377|2461905|2462180|N|N|N|N|N| +2462271|AAAAAAAAPDCJFCAA|2029-05-14|1552|6750|518|2029|1|5|14|2|2029|518|6750|Monday|2029Q2|N|N|N|2462258|2462377|2461906|2462181|N|N|N|N|N| +2462272|AAAAAAAAAECJFCAA|2029-05-15|1552|6751|518|2029|2|5|15|2|2029|518|6751|Tuesday|2029Q2|N|N|N|2462258|2462377|2461907|2462182|N|N|N|N|N| +2462273|AAAAAAAABECJFCAA|2029-05-16|1552|6751|518|2029|3|5|16|2|2029|518|6751|Wednesday|2029Q2|N|N|N|2462258|2462377|2461908|2462183|N|N|N|N|N| +2462274|AAAAAAAACECJFCAA|2029-05-17|1552|6751|518|2029|4|5|17|2|2029|518|6751|Thursday|2029Q2|N|N|N|2462258|2462377|2461909|2462184|N|N|N|N|N| +2462275|AAAAAAAADECJFCAA|2029-05-18|1552|6751|518|2029|5|5|18|2|2029|518|6751|Friday|2029Q2|N|Y|N|2462258|2462377|2461910|2462185|N|N|N|N|N| +2462276|AAAAAAAAEECJFCAA|2029-05-19|1552|6751|518|2029|6|5|19|2|2029|518|6751|Saturday|2029Q2|N|Y|N|2462258|2462377|2461911|2462186|N|N|N|N|N| +2462277|AAAAAAAAFECJFCAA|2029-05-20|1552|6751|518|2029|0|5|20|2|2029|518|6751|Sunday|2029Q2|N|N|N|2462258|2462377|2461912|2462187|N|N|N|N|N| +2462278|AAAAAAAAGECJFCAA|2029-05-21|1552|6751|518|2029|1|5|21|2|2029|518|6751|Monday|2029Q2|N|N|N|2462258|2462377|2461913|2462188|N|N|N|N|N| +2462279|AAAAAAAAHECJFCAA|2029-05-22|1552|6752|518|2029|2|5|22|2|2029|518|6752|Tuesday|2029Q2|N|N|N|2462258|2462377|2461914|2462189|N|N|N|N|N| +2462280|AAAAAAAAIECJFCAA|2029-05-23|1552|6752|518|2029|3|5|23|2|2029|518|6752|Wednesday|2029Q2|N|N|N|2462258|2462377|2461915|2462190|N|N|N|N|N| +2462281|AAAAAAAAJECJFCAA|2029-05-24|1552|6752|518|2029|4|5|24|2|2029|518|6752|Thursday|2029Q2|N|N|N|2462258|2462377|2461916|2462191|N|N|N|N|N| +2462282|AAAAAAAAKECJFCAA|2029-05-25|1552|6752|518|2029|5|5|25|2|2029|518|6752|Friday|2029Q2|N|Y|N|2462258|2462377|2461917|2462192|N|N|N|N|N| +2462283|AAAAAAAALECJFCAA|2029-05-26|1552|6752|518|2029|6|5|26|2|2029|518|6752|Saturday|2029Q2|N|Y|N|2462258|2462377|2461918|2462193|N|N|N|N|N| +2462284|AAAAAAAAMECJFCAA|2029-05-27|1552|6752|518|2029|0|5|27|2|2029|518|6752|Sunday|2029Q2|N|N|N|2462258|2462377|2461919|2462194|N|N|N|N|N| +2462285|AAAAAAAANECJFCAA|2029-05-28|1552|6752|518|2029|1|5|28|2|2029|518|6752|Monday|2029Q2|N|N|N|2462258|2462377|2461920|2462195|N|N|N|N|N| +2462286|AAAAAAAAOECJFCAA|2029-05-29|1552|6753|518|2029|2|5|29|2|2029|518|6753|Tuesday|2029Q2|N|N|N|2462258|2462377|2461921|2462196|N|N|N|N|N| +2462287|AAAAAAAAPECJFCAA|2029-05-30|1552|6753|518|2029|3|5|30|2|2029|518|6753|Wednesday|2029Q2|N|N|N|2462258|2462377|2461922|2462197|N|N|N|N|N| +2462288|AAAAAAAAAFCJFCAA|2029-05-31|1552|6753|518|2029|4|5|31|2|2029|518|6753|Thursday|2029Q2|N|N|N|2462258|2462377|2461923|2462198|N|N|N|N|N| +2462289|AAAAAAAABFCJFCAA|2029-06-01|1553|6753|519|2029|5|6|1|2|2029|519|6753|Friday|2029Q2|N|Y|N|2462289|2462439|2461924|2462199|N|N|N|N|N| +2462290|AAAAAAAACFCJFCAA|2029-06-02|1553|6753|519|2029|6|6|2|2|2029|519|6753|Saturday|2029Q2|N|Y|N|2462289|2462439|2461925|2462200|N|N|N|N|N| +2462291|AAAAAAAADFCJFCAA|2029-06-03|1553|6753|519|2029|0|6|3|2|2029|519|6753|Sunday|2029Q2|N|N|N|2462289|2462439|2461926|2462201|N|N|N|N|N| +2462292|AAAAAAAAEFCJFCAA|2029-06-04|1553|6753|519|2029|1|6|4|2|2029|519|6753|Monday|2029Q2|N|N|N|2462289|2462439|2461927|2462202|N|N|N|N|N| +2462293|AAAAAAAAFFCJFCAA|2029-06-05|1553|6754|519|2029|2|6|5|2|2029|519|6754|Tuesday|2029Q2|N|N|N|2462289|2462439|2461928|2462203|N|N|N|N|N| +2462294|AAAAAAAAGFCJFCAA|2029-06-06|1553|6754|519|2029|3|6|6|2|2029|519|6754|Wednesday|2029Q2|N|N|N|2462289|2462439|2461929|2462204|N|N|N|N|N| +2462295|AAAAAAAAHFCJFCAA|2029-06-07|1553|6754|519|2029|4|6|7|2|2029|519|6754|Thursday|2029Q2|N|N|N|2462289|2462439|2461930|2462205|N|N|N|N|N| +2462296|AAAAAAAAIFCJFCAA|2029-06-08|1553|6754|519|2029|5|6|8|2|2029|519|6754|Friday|2029Q2|N|Y|N|2462289|2462439|2461931|2462206|N|N|N|N|N| +2462297|AAAAAAAAJFCJFCAA|2029-06-09|1553|6754|519|2029|6|6|9|2|2029|519|6754|Saturday|2029Q2|N|Y|N|2462289|2462439|2461932|2462207|N|N|N|N|N| +2462298|AAAAAAAAKFCJFCAA|2029-06-10|1553|6754|519|2029|0|6|10|2|2029|519|6754|Sunday|2029Q2|N|N|N|2462289|2462439|2461933|2462208|N|N|N|N|N| +2462299|AAAAAAAALFCJFCAA|2029-06-11|1553|6754|519|2029|1|6|11|2|2029|519|6754|Monday|2029Q2|N|N|N|2462289|2462439|2461934|2462209|N|N|N|N|N| +2462300|AAAAAAAAMFCJFCAA|2029-06-12|1553|6755|519|2029|2|6|12|2|2029|519|6755|Tuesday|2029Q2|N|N|N|2462289|2462439|2461935|2462210|N|N|N|N|N| +2462301|AAAAAAAANFCJFCAA|2029-06-13|1553|6755|519|2029|3|6|13|2|2029|519|6755|Wednesday|2029Q2|N|N|N|2462289|2462439|2461936|2462211|N|N|N|N|N| +2462302|AAAAAAAAOFCJFCAA|2029-06-14|1553|6755|519|2029|4|6|14|2|2029|519|6755|Thursday|2029Q2|N|N|N|2462289|2462439|2461937|2462212|N|N|N|N|N| +2462303|AAAAAAAAPFCJFCAA|2029-06-15|1553|6755|519|2029|5|6|15|2|2029|519|6755|Friday|2029Q2|N|Y|N|2462289|2462439|2461938|2462213|N|N|N|N|N| +2462304|AAAAAAAAAGCJFCAA|2029-06-16|1553|6755|519|2029|6|6|16|2|2029|519|6755|Saturday|2029Q2|N|Y|N|2462289|2462439|2461939|2462214|N|N|N|N|N| +2462305|AAAAAAAABGCJFCAA|2029-06-17|1553|6755|519|2029|0|6|17|2|2029|519|6755|Sunday|2029Q2|N|N|N|2462289|2462439|2461940|2462215|N|N|N|N|N| +2462306|AAAAAAAACGCJFCAA|2029-06-18|1553|6755|519|2029|1|6|18|2|2029|519|6755|Monday|2029Q2|N|N|N|2462289|2462439|2461941|2462216|N|N|N|N|N| +2462307|AAAAAAAADGCJFCAA|2029-06-19|1553|6756|519|2029|2|6|19|2|2029|519|6756|Tuesday|2029Q2|N|N|N|2462289|2462439|2461942|2462217|N|N|N|N|N| +2462308|AAAAAAAAEGCJFCAA|2029-06-20|1553|6756|519|2029|3|6|20|2|2029|519|6756|Wednesday|2029Q2|N|N|N|2462289|2462439|2461943|2462218|N|N|N|N|N| +2462309|AAAAAAAAFGCJFCAA|2029-06-21|1553|6756|519|2029|4|6|21|2|2029|519|6756|Thursday|2029Q2|N|N|N|2462289|2462439|2461944|2462219|N|N|N|N|N| +2462310|AAAAAAAAGGCJFCAA|2029-06-22|1553|6756|519|2029|5|6|22|2|2029|519|6756|Friday|2029Q2|N|Y|N|2462289|2462439|2461945|2462220|N|N|N|N|N| +2462311|AAAAAAAAHGCJFCAA|2029-06-23|1553|6756|519|2029|6|6|23|2|2029|519|6756|Saturday|2029Q2|N|Y|N|2462289|2462439|2461946|2462221|N|N|N|N|N| +2462312|AAAAAAAAIGCJFCAA|2029-06-24|1553|6756|519|2029|0|6|24|2|2029|519|6756|Sunday|2029Q2|N|N|N|2462289|2462439|2461947|2462222|N|N|N|N|N| +2462313|AAAAAAAAJGCJFCAA|2029-06-25|1553|6756|519|2029|1|6|25|2|2029|519|6756|Monday|2029Q2|N|N|N|2462289|2462439|2461948|2462223|N|N|N|N|N| +2462314|AAAAAAAAKGCJFCAA|2029-06-26|1553|6757|519|2029|2|6|26|2|2029|519|6757|Tuesday|2029Q2|N|N|N|2462289|2462439|2461949|2462224|N|N|N|N|N| +2462315|AAAAAAAALGCJFCAA|2029-06-27|1553|6757|519|2029|3|6|27|2|2029|519|6757|Wednesday|2029Q2|N|N|N|2462289|2462439|2461950|2462225|N|N|N|N|N| +2462316|AAAAAAAAMGCJFCAA|2029-06-28|1553|6757|519|2029|4|6|28|2|2029|519|6757|Thursday|2029Q2|N|N|N|2462289|2462439|2461951|2462226|N|N|N|N|N| +2462317|AAAAAAAANGCJFCAA|2029-06-29|1553|6757|519|2029|5|6|29|2|2029|519|6757|Friday|2029Q2|N|Y|N|2462289|2462439|2461952|2462227|N|N|N|N|N| +2462318|AAAAAAAAOGCJFCAA|2029-06-30|1553|6757|519|2029|6|6|30|2|2029|519|6757|Saturday|2029Q2|N|Y|N|2462289|2462439|2461953|2462228|N|N|N|N|N| +2462319|AAAAAAAAPGCJFCAA|2029-07-01|1554|6757|519|2029|0|7|1|2|2029|519|6757|Sunday|2029Q2|N|N|N|2462319|2462499|2461954|2462228|N|N|N|N|N| +2462320|AAAAAAAAAHCJFCAA|2029-07-02|1554|6757|519|2029|1|7|2|3|2029|519|6757|Monday|2029Q3|N|N|N|2462319|2462499|2461955|2462229|N|N|N|N|N| +2462321|AAAAAAAABHCJFCAA|2029-07-03|1554|6758|519|2029|2|7|3|3|2029|519|6758|Tuesday|2029Q3|N|N|N|2462319|2462499|2461956|2462230|N|N|N|N|N| +2462322|AAAAAAAACHCJFCAA|2029-07-04|1554|6758|519|2029|3|7|4|3|2029|519|6758|Wednesday|2029Q3|N|N|N|2462319|2462499|2461957|2462231|N|N|N|N|N| +2462323|AAAAAAAADHCJFCAA|2029-07-05|1554|6758|519|2029|4|7|5|3|2029|519|6758|Thursday|2029Q3|Y|N|N|2462319|2462499|2461958|2462232|N|N|N|N|N| +2462324|AAAAAAAAEHCJFCAA|2029-07-06|1554|6758|519|2029|5|7|6|3|2029|519|6758|Friday|2029Q3|N|Y|Y|2462319|2462499|2461959|2462233|N|N|N|N|N| +2462325|AAAAAAAAFHCJFCAA|2029-07-07|1554|6758|519|2029|6|7|7|3|2029|519|6758|Saturday|2029Q3|N|Y|N|2462319|2462499|2461960|2462234|N|N|N|N|N| +2462326|AAAAAAAAGHCJFCAA|2029-07-08|1554|6758|519|2029|0|7|8|3|2029|519|6758|Sunday|2029Q3|N|N|N|2462319|2462499|2461961|2462235|N|N|N|N|N| +2462327|AAAAAAAAHHCJFCAA|2029-07-09|1554|6758|519|2029|1|7|9|3|2029|519|6758|Monday|2029Q3|N|N|N|2462319|2462499|2461962|2462236|N|N|N|N|N| +2462328|AAAAAAAAIHCJFCAA|2029-07-10|1554|6759|519|2029|2|7|10|3|2029|519|6759|Tuesday|2029Q3|N|N|N|2462319|2462499|2461963|2462237|N|N|N|N|N| +2462329|AAAAAAAAJHCJFCAA|2029-07-11|1554|6759|519|2029|3|7|11|3|2029|519|6759|Wednesday|2029Q3|N|N|N|2462319|2462499|2461964|2462238|N|N|N|N|N| +2462330|AAAAAAAAKHCJFCAA|2029-07-12|1554|6759|519|2029|4|7|12|3|2029|519|6759|Thursday|2029Q3|N|N|N|2462319|2462499|2461965|2462239|N|N|N|N|N| +2462331|AAAAAAAALHCJFCAA|2029-07-13|1554|6759|519|2029|5|7|13|3|2029|519|6759|Friday|2029Q3|N|Y|N|2462319|2462499|2461966|2462240|N|N|N|N|N| +2462332|AAAAAAAAMHCJFCAA|2029-07-14|1554|6759|519|2029|6|7|14|3|2029|519|6759|Saturday|2029Q3|N|Y|N|2462319|2462499|2461967|2462241|N|N|N|N|N| +2462333|AAAAAAAANHCJFCAA|2029-07-15|1554|6759|519|2029|0|7|15|3|2029|519|6759|Sunday|2029Q3|N|N|N|2462319|2462499|2461968|2462242|N|N|N|N|N| +2462334|AAAAAAAAOHCJFCAA|2029-07-16|1554|6759|519|2029|1|7|16|3|2029|519|6759|Monday|2029Q3|N|N|N|2462319|2462499|2461969|2462243|N|N|N|N|N| +2462335|AAAAAAAAPHCJFCAA|2029-07-17|1554|6760|519|2029|2|7|17|3|2029|519|6760|Tuesday|2029Q3|N|N|N|2462319|2462499|2461970|2462244|N|N|N|N|N| +2462336|AAAAAAAAAICJFCAA|2029-07-18|1554|6760|519|2029|3|7|18|3|2029|519|6760|Wednesday|2029Q3|N|N|N|2462319|2462499|2461971|2462245|N|N|N|N|N| +2462337|AAAAAAAABICJFCAA|2029-07-19|1554|6760|519|2029|4|7|19|3|2029|519|6760|Thursday|2029Q3|N|N|N|2462319|2462499|2461972|2462246|N|N|N|N|N| +2462338|AAAAAAAACICJFCAA|2029-07-20|1554|6760|519|2029|5|7|20|3|2029|519|6760|Friday|2029Q3|N|Y|N|2462319|2462499|2461973|2462247|N|N|N|N|N| +2462339|AAAAAAAADICJFCAA|2029-07-21|1554|6760|519|2029|6|7|21|3|2029|519|6760|Saturday|2029Q3|N|Y|N|2462319|2462499|2461974|2462248|N|N|N|N|N| +2462340|AAAAAAAAEICJFCAA|2029-07-22|1554|6760|519|2029|0|7|22|3|2029|519|6760|Sunday|2029Q3|N|N|N|2462319|2462499|2461975|2462249|N|N|N|N|N| +2462341|AAAAAAAAFICJFCAA|2029-07-23|1554|6760|519|2029|1|7|23|3|2029|519|6760|Monday|2029Q3|N|N|N|2462319|2462499|2461976|2462250|N|N|N|N|N| +2462342|AAAAAAAAGICJFCAA|2029-07-24|1554|6761|519|2029|2|7|24|3|2029|519|6761|Tuesday|2029Q3|N|N|N|2462319|2462499|2461977|2462251|N|N|N|N|N| +2462343|AAAAAAAAHICJFCAA|2029-07-25|1554|6761|519|2029|3|7|25|3|2029|519|6761|Wednesday|2029Q3|N|N|N|2462319|2462499|2461978|2462252|N|N|N|N|N| +2462344|AAAAAAAAIICJFCAA|2029-07-26|1554|6761|519|2029|4|7|26|3|2029|519|6761|Thursday|2029Q3|N|N|N|2462319|2462499|2461979|2462253|N|N|N|N|N| +2462345|AAAAAAAAJICJFCAA|2029-07-27|1554|6761|519|2029|5|7|27|3|2029|519|6761|Friday|2029Q3|N|Y|N|2462319|2462499|2461980|2462254|N|N|N|N|N| +2462346|AAAAAAAAKICJFCAA|2029-07-28|1554|6761|519|2029|6|7|28|3|2029|519|6761|Saturday|2029Q3|N|Y|N|2462319|2462499|2461981|2462255|N|N|N|N|N| +2462347|AAAAAAAALICJFCAA|2029-07-29|1554|6761|519|2029|0|7|29|3|2029|519|6761|Sunday|2029Q3|N|N|N|2462319|2462499|2461982|2462256|N|N|N|N|N| +2462348|AAAAAAAAMICJFCAA|2029-07-30|1554|6761|519|2029|1|7|30|3|2029|519|6761|Monday|2029Q3|N|N|N|2462319|2462499|2461983|2462257|N|N|N|N|N| +2462349|AAAAAAAANICJFCAA|2029-07-31|1554|6762|519|2029|2|7|31|3|2029|519|6762|Tuesday|2029Q3|N|N|N|2462319|2462499|2461984|2462258|N|N|N|N|N| +2462350|AAAAAAAAOICJFCAA|2029-08-01|1555|6762|519|2029|3|8|1|3|2029|519|6762|Wednesday|2029Q3|N|N|N|2462350|2462561|2461985|2462259|N|N|N|N|N| +2462351|AAAAAAAAPICJFCAA|2029-08-02|1555|6762|519|2029|4|8|2|3|2029|519|6762|Thursday|2029Q3|N|N|N|2462350|2462561|2461986|2462260|N|N|N|N|N| +2462352|AAAAAAAAAJCJFCAA|2029-08-03|1555|6762|519|2029|5|8|3|3|2029|519|6762|Friday|2029Q3|N|Y|N|2462350|2462561|2461987|2462261|N|N|N|N|N| +2462353|AAAAAAAABJCJFCAA|2029-08-04|1555|6762|519|2029|6|8|4|3|2029|519|6762|Saturday|2029Q3|N|Y|N|2462350|2462561|2461988|2462262|N|N|N|N|N| +2462354|AAAAAAAACJCJFCAA|2029-08-05|1555|6762|519|2029|0|8|5|3|2029|519|6762|Sunday|2029Q3|N|N|N|2462350|2462561|2461989|2462263|N|N|N|N|N| +2462355|AAAAAAAADJCJFCAA|2029-08-06|1555|6762|519|2029|1|8|6|3|2029|519|6762|Monday|2029Q3|N|N|N|2462350|2462561|2461990|2462264|N|N|N|N|N| +2462356|AAAAAAAAEJCJFCAA|2029-08-07|1555|6763|519|2029|2|8|7|3|2029|519|6763|Tuesday|2029Q3|N|N|N|2462350|2462561|2461991|2462265|N|N|N|N|N| +2462357|AAAAAAAAFJCJFCAA|2029-08-08|1555|6763|519|2029|3|8|8|3|2029|519|6763|Wednesday|2029Q3|N|N|N|2462350|2462561|2461992|2462266|N|N|N|N|N| +2462358|AAAAAAAAGJCJFCAA|2029-08-09|1555|6763|519|2029|4|8|9|3|2029|519|6763|Thursday|2029Q3|N|N|N|2462350|2462561|2461993|2462267|N|N|N|N|N| +2462359|AAAAAAAAHJCJFCAA|2029-08-10|1555|6763|519|2029|5|8|10|3|2029|519|6763|Friday|2029Q3|N|Y|N|2462350|2462561|2461994|2462268|N|N|N|N|N| +2462360|AAAAAAAAIJCJFCAA|2029-08-11|1555|6763|519|2029|6|8|11|3|2029|519|6763|Saturday|2029Q3|N|Y|N|2462350|2462561|2461995|2462269|N|N|N|N|N| +2462361|AAAAAAAAJJCJFCAA|2029-08-12|1555|6763|519|2029|0|8|12|3|2029|519|6763|Sunday|2029Q3|N|N|N|2462350|2462561|2461996|2462270|N|N|N|N|N| +2462362|AAAAAAAAKJCJFCAA|2029-08-13|1555|6763|519|2029|1|8|13|3|2029|519|6763|Monday|2029Q3|N|N|N|2462350|2462561|2461997|2462271|N|N|N|N|N| +2462363|AAAAAAAALJCJFCAA|2029-08-14|1555|6764|519|2029|2|8|14|3|2029|519|6764|Tuesday|2029Q3|N|N|N|2462350|2462561|2461998|2462272|N|N|N|N|N| +2462364|AAAAAAAAMJCJFCAA|2029-08-15|1555|6764|519|2029|3|8|15|3|2029|519|6764|Wednesday|2029Q3|N|N|N|2462350|2462561|2461999|2462273|N|N|N|N|N| +2462365|AAAAAAAANJCJFCAA|2029-08-16|1555|6764|519|2029|4|8|16|3|2029|519|6764|Thursday|2029Q3|N|N|N|2462350|2462561|2462000|2462274|N|N|N|N|N| +2462366|AAAAAAAAOJCJFCAA|2029-08-17|1555|6764|519|2029|5|8|17|3|2029|519|6764|Friday|2029Q3|N|Y|N|2462350|2462561|2462001|2462275|N|N|N|N|N| +2462367|AAAAAAAAPJCJFCAA|2029-08-18|1555|6764|519|2029|6|8|18|3|2029|519|6764|Saturday|2029Q3|N|Y|N|2462350|2462561|2462002|2462276|N|N|N|N|N| +2462368|AAAAAAAAAKCJFCAA|2029-08-19|1555|6764|519|2029|0|8|19|3|2029|519|6764|Sunday|2029Q3|N|N|N|2462350|2462561|2462003|2462277|N|N|N|N|N| +2462369|AAAAAAAABKCJFCAA|2029-08-20|1555|6764|519|2029|1|8|20|3|2029|519|6764|Monday|2029Q3|N|N|N|2462350|2462561|2462004|2462278|N|N|N|N|N| +2462370|AAAAAAAACKCJFCAA|2029-08-21|1555|6765|519|2029|2|8|21|3|2029|519|6765|Tuesday|2029Q3|N|N|N|2462350|2462561|2462005|2462279|N|N|N|N|N| +2462371|AAAAAAAADKCJFCAA|2029-08-22|1555|6765|519|2029|3|8|22|3|2029|519|6765|Wednesday|2029Q3|N|N|N|2462350|2462561|2462006|2462280|N|N|N|N|N| +2462372|AAAAAAAAEKCJFCAA|2029-08-23|1555|6765|519|2029|4|8|23|3|2029|519|6765|Thursday|2029Q3|N|N|N|2462350|2462561|2462007|2462281|N|N|N|N|N| +2462373|AAAAAAAAFKCJFCAA|2029-08-24|1555|6765|519|2029|5|8|24|3|2029|519|6765|Friday|2029Q3|N|Y|N|2462350|2462561|2462008|2462282|N|N|N|N|N| +2462374|AAAAAAAAGKCJFCAA|2029-08-25|1555|6765|519|2029|6|8|25|3|2029|519|6765|Saturday|2029Q3|N|Y|N|2462350|2462561|2462009|2462283|N|N|N|N|N| +2462375|AAAAAAAAHKCJFCAA|2029-08-26|1555|6765|519|2029|0|8|26|3|2029|519|6765|Sunday|2029Q3|N|N|N|2462350|2462561|2462010|2462284|N|N|N|N|N| +2462376|AAAAAAAAIKCJFCAA|2029-08-27|1555|6765|519|2029|1|8|27|3|2029|519|6765|Monday|2029Q3|N|N|N|2462350|2462561|2462011|2462285|N|N|N|N|N| +2462377|AAAAAAAAJKCJFCAA|2029-08-28|1555|6766|519|2029|2|8|28|3|2029|519|6766|Tuesday|2029Q3|N|N|N|2462350|2462561|2462012|2462286|N|N|N|N|N| +2462378|AAAAAAAAKKCJFCAA|2029-08-29|1555|6766|519|2029|3|8|29|3|2029|519|6766|Wednesday|2029Q3|N|N|N|2462350|2462561|2462013|2462287|N|N|N|N|N| +2462379|AAAAAAAALKCJFCAA|2029-08-30|1555|6766|519|2029|4|8|30|3|2029|519|6766|Thursday|2029Q3|N|N|N|2462350|2462561|2462014|2462288|N|N|N|N|N| +2462380|AAAAAAAAMKCJFCAA|2029-08-31|1555|6766|519|2029|5|8|31|3|2029|519|6766|Friday|2029Q3|N|Y|N|2462350|2462561|2462015|2462289|N|N|N|N|N| +2462381|AAAAAAAANKCJFCAA|2029-09-01|1556|6766|520|2029|6|9|1|3|2029|520|6766|Saturday|2029Q3|N|Y|N|2462381|2462623|2462016|2462290|N|N|N|N|N| +2462382|AAAAAAAAOKCJFCAA|2029-09-02|1556|6766|520|2029|0|9|2|3|2029|520|6766|Sunday|2029Q3|N|N|N|2462381|2462623|2462017|2462291|N|N|N|N|N| +2462383|AAAAAAAAPKCJFCAA|2029-09-03|1556|6766|520|2029|1|9|3|3|2029|520|6766|Monday|2029Q3|N|N|N|2462381|2462623|2462018|2462292|N|N|N|N|N| +2462384|AAAAAAAAALCJFCAA|2029-09-04|1556|6767|520|2029|2|9|4|3|2029|520|6767|Tuesday|2029Q3|N|N|N|2462381|2462623|2462019|2462293|N|N|N|N|N| +2462385|AAAAAAAABLCJFCAA|2029-09-05|1556|6767|520|2029|3|9|5|3|2029|520|6767|Wednesday|2029Q3|N|N|N|2462381|2462623|2462020|2462294|N|N|N|N|N| +2462386|AAAAAAAACLCJFCAA|2029-09-06|1556|6767|520|2029|4|9|6|3|2029|520|6767|Thursday|2029Q3|N|N|N|2462381|2462623|2462021|2462295|N|N|N|N|N| +2462387|AAAAAAAADLCJFCAA|2029-09-07|1556|6767|520|2029|5|9|7|3|2029|520|6767|Friday|2029Q3|N|Y|N|2462381|2462623|2462022|2462296|N|N|N|N|N| +2462388|AAAAAAAAELCJFCAA|2029-09-08|1556|6767|520|2029|6|9|8|3|2029|520|6767|Saturday|2029Q3|N|Y|N|2462381|2462623|2462023|2462297|N|N|N|N|N| +2462389|AAAAAAAAFLCJFCAA|2029-09-09|1556|6767|520|2029|0|9|9|3|2029|520|6767|Sunday|2029Q3|N|N|N|2462381|2462623|2462024|2462298|N|N|N|N|N| +2462390|AAAAAAAAGLCJFCAA|2029-09-10|1556|6767|520|2029|1|9|10|3|2029|520|6767|Monday|2029Q3|N|N|N|2462381|2462623|2462025|2462299|N|N|N|N|N| +2462391|AAAAAAAAHLCJFCAA|2029-09-11|1556|6768|520|2029|2|9|11|3|2029|520|6768|Tuesday|2029Q3|N|N|N|2462381|2462623|2462026|2462300|N|N|N|N|N| +2462392|AAAAAAAAILCJFCAA|2029-09-12|1556|6768|520|2029|3|9|12|3|2029|520|6768|Wednesday|2029Q3|N|N|N|2462381|2462623|2462027|2462301|N|N|N|N|N| +2462393|AAAAAAAAJLCJFCAA|2029-09-13|1556|6768|520|2029|4|9|13|3|2029|520|6768|Thursday|2029Q3|N|N|N|2462381|2462623|2462028|2462302|N|N|N|N|N| +2462394|AAAAAAAAKLCJFCAA|2029-09-14|1556|6768|520|2029|5|9|14|3|2029|520|6768|Friday|2029Q3|N|Y|N|2462381|2462623|2462029|2462303|N|N|N|N|N| +2462395|AAAAAAAALLCJFCAA|2029-09-15|1556|6768|520|2029|6|9|15|3|2029|520|6768|Saturday|2029Q3|N|Y|N|2462381|2462623|2462030|2462304|N|N|N|N|N| +2462396|AAAAAAAAMLCJFCAA|2029-09-16|1556|6768|520|2029|0|9|16|3|2029|520|6768|Sunday|2029Q3|N|N|N|2462381|2462623|2462031|2462305|N|N|N|N|N| +2462397|AAAAAAAANLCJFCAA|2029-09-17|1556|6768|520|2029|1|9|17|3|2029|520|6768|Monday|2029Q3|N|N|N|2462381|2462623|2462032|2462306|N|N|N|N|N| +2462398|AAAAAAAAOLCJFCAA|2029-09-18|1556|6769|520|2029|2|9|18|3|2029|520|6769|Tuesday|2029Q3|N|N|N|2462381|2462623|2462033|2462307|N|N|N|N|N| +2462399|AAAAAAAAPLCJFCAA|2029-09-19|1556|6769|520|2029|3|9|19|3|2029|520|6769|Wednesday|2029Q3|N|N|N|2462381|2462623|2462034|2462308|N|N|N|N|N| +2462400|AAAAAAAAAMCJFCAA|2029-09-20|1556|6769|520|2029|4|9|20|3|2029|520|6769|Thursday|2029Q3|N|N|N|2462381|2462623|2462035|2462309|N|N|N|N|N| +2462401|AAAAAAAABMCJFCAA|2029-09-21|1556|6769|520|2029|5|9|21|3|2029|520|6769|Friday|2029Q3|N|Y|N|2462381|2462623|2462036|2462310|N|N|N|N|N| +2462402|AAAAAAAACMCJFCAA|2029-09-22|1556|6769|520|2029|6|9|22|3|2029|520|6769|Saturday|2029Q3|N|Y|N|2462381|2462623|2462037|2462311|N|N|N|N|N| +2462403|AAAAAAAADMCJFCAA|2029-09-23|1556|6769|520|2029|0|9|23|3|2029|520|6769|Sunday|2029Q3|N|N|N|2462381|2462623|2462038|2462312|N|N|N|N|N| +2462404|AAAAAAAAEMCJFCAA|2029-09-24|1556|6769|520|2029|1|9|24|3|2029|520|6769|Monday|2029Q3|N|N|N|2462381|2462623|2462039|2462313|N|N|N|N|N| +2462405|AAAAAAAAFMCJFCAA|2029-09-25|1556|6770|520|2029|2|9|25|3|2029|520|6770|Tuesday|2029Q3|N|N|N|2462381|2462623|2462040|2462314|N|N|N|N|N| +2462406|AAAAAAAAGMCJFCAA|2029-09-26|1556|6770|520|2029|3|9|26|3|2029|520|6770|Wednesday|2029Q3|N|N|N|2462381|2462623|2462041|2462315|N|N|N|N|N| +2462407|AAAAAAAAHMCJFCAA|2029-09-27|1556|6770|520|2029|4|9|27|3|2029|520|6770|Thursday|2029Q3|N|N|N|2462381|2462623|2462042|2462316|N|N|N|N|N| +2462408|AAAAAAAAIMCJFCAA|2029-09-28|1556|6770|520|2029|5|9|28|3|2029|520|6770|Friday|2029Q3|N|Y|N|2462381|2462623|2462043|2462317|N|N|N|N|N| +2462409|AAAAAAAAJMCJFCAA|2029-09-29|1556|6770|520|2029|6|9|29|3|2029|520|6770|Saturday|2029Q3|N|Y|N|2462381|2462623|2462044|2462318|N|N|N|N|N| +2462410|AAAAAAAAKMCJFCAA|2029-09-30|1556|6770|520|2029|0|9|30|3|2029|520|6770|Sunday|2029Q3|N|N|N|2462381|2462623|2462045|2462319|N|N|N|N|N| +2462411|AAAAAAAALMCJFCAA|2029-10-01|1557|6770|520|2029|1|10|1|3|2029|520|6770|Monday|2029Q3|N|N|N|2462411|2462683|2462046|2462319|N|N|N|N|N| +2462412|AAAAAAAAMMCJFCAA|2029-10-02|1557|6771|520|2029|2|10|2|4|2029|520|6771|Tuesday|2029Q4|N|N|N|2462411|2462683|2462047|2462320|N|N|N|N|N| +2462413|AAAAAAAANMCJFCAA|2029-10-03|1557|6771|520|2029|3|10|3|4|2029|520|6771|Wednesday|2029Q4|N|N|N|2462411|2462683|2462048|2462321|N|N|N|N|N| +2462414|AAAAAAAAOMCJFCAA|2029-10-04|1557|6771|520|2029|4|10|4|4|2029|520|6771|Thursday|2029Q4|N|N|N|2462411|2462683|2462049|2462322|N|N|N|N|N| +2462415|AAAAAAAAPMCJFCAA|2029-10-05|1557|6771|520|2029|5|10|5|4|2029|520|6771|Friday|2029Q4|N|Y|N|2462411|2462683|2462050|2462323|N|N|N|N|N| +2462416|AAAAAAAAANCJFCAA|2029-10-06|1557|6771|520|2029|6|10|6|4|2029|520|6771|Saturday|2029Q4|N|Y|N|2462411|2462683|2462051|2462324|N|N|N|N|N| +2462417|AAAAAAAABNCJFCAA|2029-10-07|1557|6771|520|2029|0|10|7|4|2029|520|6771|Sunday|2029Q4|N|N|N|2462411|2462683|2462052|2462325|N|N|N|N|N| +2462418|AAAAAAAACNCJFCAA|2029-10-08|1557|6771|520|2029|1|10|8|4|2029|520|6771|Monday|2029Q4|N|N|N|2462411|2462683|2462053|2462326|N|N|N|N|N| +2462419|AAAAAAAADNCJFCAA|2029-10-09|1557|6772|520|2029|2|10|9|4|2029|520|6772|Tuesday|2029Q4|N|N|N|2462411|2462683|2462054|2462327|N|N|N|N|N| +2462420|AAAAAAAAENCJFCAA|2029-10-10|1557|6772|520|2029|3|10|10|4|2029|520|6772|Wednesday|2029Q4|N|N|N|2462411|2462683|2462055|2462328|N|N|N|N|N| +2462421|AAAAAAAAFNCJFCAA|2029-10-11|1557|6772|520|2029|4|10|11|4|2029|520|6772|Thursday|2029Q4|N|N|N|2462411|2462683|2462056|2462329|N|N|N|N|N| +2462422|AAAAAAAAGNCJFCAA|2029-10-12|1557|6772|520|2029|5|10|12|4|2029|520|6772|Friday|2029Q4|N|Y|N|2462411|2462683|2462057|2462330|N|N|N|N|N| +2462423|AAAAAAAAHNCJFCAA|2029-10-13|1557|6772|520|2029|6|10|13|4|2029|520|6772|Saturday|2029Q4|N|Y|N|2462411|2462683|2462058|2462331|N|N|N|N|N| +2462424|AAAAAAAAINCJFCAA|2029-10-14|1557|6772|520|2029|0|10|14|4|2029|520|6772|Sunday|2029Q4|N|N|N|2462411|2462683|2462059|2462332|N|N|N|N|N| +2462425|AAAAAAAAJNCJFCAA|2029-10-15|1557|6772|520|2029|1|10|15|4|2029|520|6772|Monday|2029Q4|N|N|N|2462411|2462683|2462060|2462333|N|N|N|N|N| +2462426|AAAAAAAAKNCJFCAA|2029-10-16|1557|6773|520|2029|2|10|16|4|2029|520|6773|Tuesday|2029Q4|N|N|N|2462411|2462683|2462061|2462334|N|N|N|N|N| +2462427|AAAAAAAALNCJFCAA|2029-10-17|1557|6773|520|2029|3|10|17|4|2029|520|6773|Wednesday|2029Q4|N|N|N|2462411|2462683|2462062|2462335|N|N|N|N|N| +2462428|AAAAAAAAMNCJFCAA|2029-10-18|1557|6773|520|2029|4|10|18|4|2029|520|6773|Thursday|2029Q4|N|N|N|2462411|2462683|2462063|2462336|N|N|N|N|N| +2462429|AAAAAAAANNCJFCAA|2029-10-19|1557|6773|520|2029|5|10|19|4|2029|520|6773|Friday|2029Q4|N|Y|N|2462411|2462683|2462064|2462337|N|N|N|N|N| +2462430|AAAAAAAAONCJFCAA|2029-10-20|1557|6773|520|2029|6|10|20|4|2029|520|6773|Saturday|2029Q4|N|Y|N|2462411|2462683|2462065|2462338|N|N|N|N|N| +2462431|AAAAAAAAPNCJFCAA|2029-10-21|1557|6773|520|2029|0|10|21|4|2029|520|6773|Sunday|2029Q4|N|N|N|2462411|2462683|2462066|2462339|N|N|N|N|N| +2462432|AAAAAAAAAOCJFCAA|2029-10-22|1557|6773|520|2029|1|10|22|4|2029|520|6773|Monday|2029Q4|N|N|N|2462411|2462683|2462067|2462340|N|N|N|N|N| +2462433|AAAAAAAABOCJFCAA|2029-10-23|1557|6774|520|2029|2|10|23|4|2029|520|6774|Tuesday|2029Q4|N|N|N|2462411|2462683|2462068|2462341|N|N|N|N|N| +2462434|AAAAAAAACOCJFCAA|2029-10-24|1557|6774|520|2029|3|10|24|4|2029|520|6774|Wednesday|2029Q4|N|N|N|2462411|2462683|2462069|2462342|N|N|N|N|N| +2462435|AAAAAAAADOCJFCAA|2029-10-25|1557|6774|520|2029|4|10|25|4|2029|520|6774|Thursday|2029Q4|N|N|N|2462411|2462683|2462070|2462343|N|N|N|N|N| +2462436|AAAAAAAAEOCJFCAA|2029-10-26|1557|6774|520|2029|5|10|26|4|2029|520|6774|Friday|2029Q4|N|Y|N|2462411|2462683|2462071|2462344|N|N|N|N|N| +2462437|AAAAAAAAFOCJFCAA|2029-10-27|1557|6774|520|2029|6|10|27|4|2029|520|6774|Saturday|2029Q4|N|Y|N|2462411|2462683|2462072|2462345|N|N|N|N|N| +2462438|AAAAAAAAGOCJFCAA|2029-10-28|1557|6774|520|2029|0|10|28|4|2029|520|6774|Sunday|2029Q4|N|N|N|2462411|2462683|2462073|2462346|N|N|N|N|N| +2462439|AAAAAAAAHOCJFCAA|2029-10-29|1557|6774|520|2029|1|10|29|4|2029|520|6774|Monday|2029Q4|N|N|N|2462411|2462683|2462074|2462347|N|N|N|N|N| +2462440|AAAAAAAAIOCJFCAA|2029-10-30|1557|6775|520|2029|2|10|30|4|2029|520|6775|Tuesday|2029Q4|N|N|N|2462411|2462683|2462075|2462348|N|N|N|N|N| +2462441|AAAAAAAAJOCJFCAA|2029-10-31|1557|6775|520|2029|3|10|31|4|2029|520|6775|Wednesday|2029Q4|N|N|N|2462411|2462683|2462076|2462349|N|N|N|N|N| +2462442|AAAAAAAAKOCJFCAA|2029-11-01|1558|6775|520|2029|4|11|1|4|2029|520|6775|Thursday|2029Q4|N|N|N|2462442|2462745|2462077|2462350|N|N|N|N|N| +2462443|AAAAAAAALOCJFCAA|2029-11-02|1558|6775|520|2029|5|11|2|4|2029|520|6775|Friday|2029Q4|N|Y|N|2462442|2462745|2462078|2462351|N|N|N|N|N| +2462444|AAAAAAAAMOCJFCAA|2029-11-03|1558|6775|520|2029|6|11|3|4|2029|520|6775|Saturday|2029Q4|N|Y|N|2462442|2462745|2462079|2462352|N|N|N|N|N| +2462445|AAAAAAAANOCJFCAA|2029-11-04|1558|6775|520|2029|0|11|4|4|2029|520|6775|Sunday|2029Q4|N|N|N|2462442|2462745|2462080|2462353|N|N|N|N|N| +2462446|AAAAAAAAOOCJFCAA|2029-11-05|1558|6775|520|2029|1|11|5|4|2029|520|6775|Monday|2029Q4|N|N|N|2462442|2462745|2462081|2462354|N|N|N|N|N| +2462447|AAAAAAAAPOCJFCAA|2029-11-06|1558|6776|520|2029|2|11|6|4|2029|520|6776|Tuesday|2029Q4|N|N|N|2462442|2462745|2462082|2462355|N|N|N|N|N| +2462448|AAAAAAAAAPCJFCAA|2029-11-07|1558|6776|520|2029|3|11|7|4|2029|520|6776|Wednesday|2029Q4|N|N|N|2462442|2462745|2462083|2462356|N|N|N|N|N| +2462449|AAAAAAAABPCJFCAA|2029-11-08|1558|6776|520|2029|4|11|8|4|2029|520|6776|Thursday|2029Q4|N|N|N|2462442|2462745|2462084|2462357|N|N|N|N|N| +2462450|AAAAAAAACPCJFCAA|2029-11-09|1558|6776|520|2029|5|11|9|4|2029|520|6776|Friday|2029Q4|N|Y|N|2462442|2462745|2462085|2462358|N|N|N|N|N| +2462451|AAAAAAAADPCJFCAA|2029-11-10|1558|6776|520|2029|6|11|10|4|2029|520|6776|Saturday|2029Q4|N|Y|N|2462442|2462745|2462086|2462359|N|N|N|N|N| +2462452|AAAAAAAAEPCJFCAA|2029-11-11|1558|6776|520|2029|0|11|11|4|2029|520|6776|Sunday|2029Q4|N|N|N|2462442|2462745|2462087|2462360|N|N|N|N|N| +2462453|AAAAAAAAFPCJFCAA|2029-11-12|1558|6776|520|2029|1|11|12|4|2029|520|6776|Monday|2029Q4|N|N|N|2462442|2462745|2462088|2462361|N|N|N|N|N| +2462454|AAAAAAAAGPCJFCAA|2029-11-13|1558|6777|520|2029|2|11|13|4|2029|520|6777|Tuesday|2029Q4|N|N|N|2462442|2462745|2462089|2462362|N|N|N|N|N| +2462455|AAAAAAAAHPCJFCAA|2029-11-14|1558|6777|520|2029|3|11|14|4|2029|520|6777|Wednesday|2029Q4|N|N|N|2462442|2462745|2462090|2462363|N|N|N|N|N| +2462456|AAAAAAAAIPCJFCAA|2029-11-15|1558|6777|520|2029|4|11|15|4|2029|520|6777|Thursday|2029Q4|N|N|N|2462442|2462745|2462091|2462364|N|N|N|N|N| +2462457|AAAAAAAAJPCJFCAA|2029-11-16|1558|6777|520|2029|5|11|16|4|2029|520|6777|Friday|2029Q4|N|Y|N|2462442|2462745|2462092|2462365|N|N|N|N|N| +2462458|AAAAAAAAKPCJFCAA|2029-11-17|1558|6777|520|2029|6|11|17|4|2029|520|6777|Saturday|2029Q4|N|Y|N|2462442|2462745|2462093|2462366|N|N|N|N|N| +2462459|AAAAAAAALPCJFCAA|2029-11-18|1558|6777|520|2029|0|11|18|4|2029|520|6777|Sunday|2029Q4|N|N|N|2462442|2462745|2462094|2462367|N|N|N|N|N| +2462460|AAAAAAAAMPCJFCAA|2029-11-19|1558|6777|520|2029|1|11|19|4|2029|520|6777|Monday|2029Q4|N|N|N|2462442|2462745|2462095|2462368|N|N|N|N|N| +2462461|AAAAAAAANPCJFCAA|2029-11-20|1558|6778|520|2029|2|11|20|4|2029|520|6778|Tuesday|2029Q4|N|N|N|2462442|2462745|2462096|2462369|N|N|N|N|N| +2462462|AAAAAAAAOPCJFCAA|2029-11-21|1558|6778|520|2029|3|11|21|4|2029|520|6778|Wednesday|2029Q4|N|N|N|2462442|2462745|2462097|2462370|N|N|N|N|N| +2462463|AAAAAAAAPPCJFCAA|2029-11-22|1558|6778|520|2029|4|11|22|4|2029|520|6778|Thursday|2029Q4|N|N|N|2462442|2462745|2462098|2462371|N|N|N|N|N| +2462464|AAAAAAAAAADJFCAA|2029-11-23|1558|6778|520|2029|5|11|23|4|2029|520|6778|Friday|2029Q4|N|Y|N|2462442|2462745|2462099|2462372|N|N|N|N|N| +2462465|AAAAAAAABADJFCAA|2029-11-24|1558|6778|520|2029|6|11|24|4|2029|520|6778|Saturday|2029Q4|N|Y|N|2462442|2462745|2462100|2462373|N|N|N|N|N| +2462466|AAAAAAAACADJFCAA|2029-11-25|1558|6778|520|2029|0|11|25|4|2029|520|6778|Sunday|2029Q4|N|N|N|2462442|2462745|2462101|2462374|N|N|N|N|N| +2462467|AAAAAAAADADJFCAA|2029-11-26|1558|6778|520|2029|1|11|26|4|2029|520|6778|Monday|2029Q4|N|N|N|2462442|2462745|2462102|2462375|N|N|N|N|N| +2462468|AAAAAAAAEADJFCAA|2029-11-27|1558|6779|520|2029|2|11|27|4|2029|520|6779|Tuesday|2029Q4|N|N|N|2462442|2462745|2462103|2462376|N|N|N|N|N| +2462469|AAAAAAAAFADJFCAA|2029-11-28|1558|6779|520|2029|3|11|28|4|2029|520|6779|Wednesday|2029Q4|N|N|N|2462442|2462745|2462104|2462377|N|N|N|N|N| +2462470|AAAAAAAAGADJFCAA|2029-11-29|1558|6779|520|2029|4|11|29|4|2029|520|6779|Thursday|2029Q4|N|N|N|2462442|2462745|2462105|2462378|N|N|N|N|N| +2462471|AAAAAAAAHADJFCAA|2029-11-30|1558|6779|520|2029|5|11|30|4|2029|520|6779|Friday|2029Q4|N|Y|N|2462442|2462745|2462106|2462379|N|N|N|N|N| +2462472|AAAAAAAAIADJFCAA|2029-12-01|1559|6779|521|2029|6|12|1|4|2029|521|6779|Saturday|2029Q4|N|Y|N|2462472|2462805|2462107|2462380|N|N|N|N|N| +2462473|AAAAAAAAJADJFCAA|2029-12-02|1559|6779|521|2029|0|12|2|4|2029|521|6779|Sunday|2029Q4|N|N|N|2462472|2462805|2462108|2462381|N|N|N|N|N| +2462474|AAAAAAAAKADJFCAA|2029-12-03|1559|6779|521|2029|1|12|3|4|2029|521|6779|Monday|2029Q4|N|N|N|2462472|2462805|2462109|2462382|N|N|N|N|N| +2462475|AAAAAAAALADJFCAA|2029-12-04|1559|6780|521|2029|2|12|4|4|2029|521|6780|Tuesday|2029Q4|N|N|N|2462472|2462805|2462110|2462383|N|N|N|N|N| +2462476|AAAAAAAAMADJFCAA|2029-12-05|1559|6780|521|2029|3|12|5|4|2029|521|6780|Wednesday|2029Q4|N|N|N|2462472|2462805|2462111|2462384|N|N|N|N|N| +2462477|AAAAAAAANADJFCAA|2029-12-06|1559|6780|521|2029|4|12|6|4|2029|521|6780|Thursday|2029Q4|N|N|N|2462472|2462805|2462112|2462385|N|N|N|N|N| +2462478|AAAAAAAAOADJFCAA|2029-12-07|1559|6780|521|2029|5|12|7|4|2029|521|6780|Friday|2029Q4|N|Y|N|2462472|2462805|2462113|2462386|N|N|N|N|N| +2462479|AAAAAAAAPADJFCAA|2029-12-08|1559|6780|521|2029|6|12|8|4|2029|521|6780|Saturday|2029Q4|N|Y|N|2462472|2462805|2462114|2462387|N|N|N|N|N| +2462480|AAAAAAAAABDJFCAA|2029-12-09|1559|6780|521|2029|0|12|9|4|2029|521|6780|Sunday|2029Q4|N|N|N|2462472|2462805|2462115|2462388|N|N|N|N|N| +2462481|AAAAAAAABBDJFCAA|2029-12-10|1559|6780|521|2029|1|12|10|4|2029|521|6780|Monday|2029Q4|N|N|N|2462472|2462805|2462116|2462389|N|N|N|N|N| +2462482|AAAAAAAACBDJFCAA|2029-12-11|1559|6781|521|2029|2|12|11|4|2029|521|6781|Tuesday|2029Q4|N|N|N|2462472|2462805|2462117|2462390|N|N|N|N|N| +2462483|AAAAAAAADBDJFCAA|2029-12-12|1559|6781|521|2029|3|12|12|4|2029|521|6781|Wednesday|2029Q4|N|N|N|2462472|2462805|2462118|2462391|N|N|N|N|N| +2462484|AAAAAAAAEBDJFCAA|2029-12-13|1559|6781|521|2029|4|12|13|4|2029|521|6781|Thursday|2029Q4|N|N|N|2462472|2462805|2462119|2462392|N|N|N|N|N| +2462485|AAAAAAAAFBDJFCAA|2029-12-14|1559|6781|521|2029|5|12|14|4|2029|521|6781|Friday|2029Q4|N|Y|N|2462472|2462805|2462120|2462393|N|N|N|N|N| +2462486|AAAAAAAAGBDJFCAA|2029-12-15|1559|6781|521|2029|6|12|15|4|2029|521|6781|Saturday|2029Q4|N|Y|N|2462472|2462805|2462121|2462394|N|N|N|N|N| +2462487|AAAAAAAAHBDJFCAA|2029-12-16|1559|6781|521|2029|0|12|16|4|2029|521|6781|Sunday|2029Q4|N|N|N|2462472|2462805|2462122|2462395|N|N|N|N|N| +2462488|AAAAAAAAIBDJFCAA|2029-12-17|1559|6781|521|2029|1|12|17|4|2029|521|6781|Monday|2029Q4|N|N|N|2462472|2462805|2462123|2462396|N|N|N|N|N| +2462489|AAAAAAAAJBDJFCAA|2029-12-18|1559|6782|521|2029|2|12|18|4|2029|521|6782|Tuesday|2029Q4|N|N|N|2462472|2462805|2462124|2462397|N|N|N|N|N| +2462490|AAAAAAAAKBDJFCAA|2029-12-19|1559|6782|521|2029|3|12|19|4|2029|521|6782|Wednesday|2029Q4|N|N|N|2462472|2462805|2462125|2462398|N|N|N|N|N| +2462491|AAAAAAAALBDJFCAA|2029-12-20|1559|6782|521|2029|4|12|20|4|2029|521|6782|Thursday|2029Q4|N|N|N|2462472|2462805|2462126|2462399|N|N|N|N|N| +2462492|AAAAAAAAMBDJFCAA|2029-12-21|1559|6782|521|2029|5|12|21|4|2029|521|6782|Friday|2029Q4|N|Y|N|2462472|2462805|2462127|2462400|N|N|N|N|N| +2462493|AAAAAAAANBDJFCAA|2029-12-22|1559|6782|521|2029|6|12|22|4|2029|521|6782|Saturday|2029Q4|N|Y|N|2462472|2462805|2462128|2462401|N|N|N|N|N| +2462494|AAAAAAAAOBDJFCAA|2029-12-23|1559|6782|521|2029|0|12|23|4|2029|521|6782|Sunday|2029Q4|N|N|N|2462472|2462805|2462129|2462402|N|N|N|N|N| +2462495|AAAAAAAAPBDJFCAA|2029-12-24|1559|6782|521|2029|1|12|24|4|2029|521|6782|Monday|2029Q4|N|N|N|2462472|2462805|2462130|2462403|N|N|N|N|N| +2462496|AAAAAAAAACDJFCAA|2029-12-25|1559|6783|521|2029|2|12|25|4|2029|521|6783|Tuesday|2029Q4|N|N|N|2462472|2462805|2462131|2462404|N|N|N|N|N| +2462497|AAAAAAAABCDJFCAA|2029-12-26|1559|6783|521|2029|3|12|26|4|2029|521|6783|Wednesday|2029Q4|Y|N|N|2462472|2462805|2462132|2462405|N|N|N|N|N| +2462498|AAAAAAAACCDJFCAA|2029-12-27|1559|6783|521|2029|4|12|27|4|2029|521|6783|Thursday|2029Q4|N|N|Y|2462472|2462805|2462133|2462406|N|N|N|N|N| +2462499|AAAAAAAADCDJFCAA|2029-12-28|1559|6783|521|2029|5|12|28|4|2029|521|6783|Friday|2029Q4|N|Y|N|2462472|2462805|2462134|2462407|N|N|N|N|N| +2462500|AAAAAAAAECDJFCAA|2029-12-29|1559|6783|521|2029|6|12|29|4|2029|521|6783|Saturday|2029Q4|N|Y|N|2462472|2462805|2462135|2462408|N|N|N|N|N| +2462501|AAAAAAAAFCDJFCAA|2029-12-30|1559|6783|521|2029|0|12|30|4|2029|521|6783|Sunday|2029Q4|N|N|N|2462472|2462805|2462136|2462409|N|N|N|N|N| +2462502|AAAAAAAAGCDJFCAA|2029-12-31|1559|6783|521|2029|1|12|31|4|2029|521|6783|Monday|2029Q4|N|N|N|2462472|2462805|2462137|2462410|N|N|N|N|N| +2462503|AAAAAAAAHCDJFCAA|2030-01-01|1560|6784|521|2030|2|1|1|1|2030|521|6784|Tuesday|2030Q1|Y|N|N|2462503|2462502|2462138|2462411|N|N|N|N|N| +2462504|AAAAAAAAICDJFCAA|2030-01-02|1560|6784|521|2030|3|1|2|1|2030|521|6784|Wednesday|2030Q1|N|N|Y|2462503|2462502|2462139|2462412|N|N|N|N|N| +2462505|AAAAAAAAJCDJFCAA|2030-01-03|1560|6784|521|2030|4|1|3|1|2030|521|6784|Thursday|2030Q1|N|N|N|2462503|2462502|2462140|2462413|N|N|N|N|N| +2462506|AAAAAAAAKCDJFCAA|2030-01-04|1560|6784|521|2030|5|1|4|1|2030|521|6784|Friday|2030Q1|N|Y|N|2462503|2462502|2462141|2462414|N|N|N|N|N| +2462507|AAAAAAAALCDJFCAA|2030-01-05|1560|6784|521|2030|6|1|5|1|2030|521|6784|Saturday|2030Q1|N|Y|N|2462503|2462502|2462142|2462415|N|N|N|N|N| +2462508|AAAAAAAAMCDJFCAA|2030-01-06|1560|6784|521|2030|0|1|6|1|2030|521|6784|Sunday|2030Q1|N|N|N|2462503|2462502|2462143|2462416|N|N|N|N|N| +2462509|AAAAAAAANCDJFCAA|2030-01-07|1560|6784|521|2030|1|1|7|1|2030|521|6784|Monday|2030Q1|N|N|N|2462503|2462502|2462144|2462417|N|N|N|N|N| +2462510|AAAAAAAAOCDJFCAA|2030-01-08|1560|6785|521|2030|2|1|8|1|2030|521|6785|Tuesday|2030Q1|N|N|N|2462503|2462502|2462145|2462418|N|N|N|N|N| +2462511|AAAAAAAAPCDJFCAA|2030-01-09|1560|6785|521|2030|3|1|9|1|2030|521|6785|Wednesday|2030Q1|N|N|N|2462503|2462502|2462146|2462419|N|N|N|N|N| +2462512|AAAAAAAAADDJFCAA|2030-01-10|1560|6785|521|2030|4|1|10|1|2030|521|6785|Thursday|2030Q1|N|N|N|2462503|2462502|2462147|2462420|N|N|N|N|N| +2462513|AAAAAAAABDDJFCAA|2030-01-11|1560|6785|521|2030|5|1|11|1|2030|521|6785|Friday|2030Q1|N|Y|N|2462503|2462502|2462148|2462421|N|N|N|N|N| +2462514|AAAAAAAACDDJFCAA|2030-01-12|1560|6785|521|2030|6|1|12|1|2030|521|6785|Saturday|2030Q1|N|Y|N|2462503|2462502|2462149|2462422|N|N|N|N|N| +2462515|AAAAAAAADDDJFCAA|2030-01-13|1560|6785|521|2030|0|1|13|1|2030|521|6785|Sunday|2030Q1|N|N|N|2462503|2462502|2462150|2462423|N|N|N|N|N| +2462516|AAAAAAAAEDDJFCAA|2030-01-14|1560|6785|521|2030|1|1|14|1|2030|521|6785|Monday|2030Q1|N|N|N|2462503|2462502|2462151|2462424|N|N|N|N|N| +2462517|AAAAAAAAFDDJFCAA|2030-01-15|1560|6786|521|2030|2|1|15|1|2030|521|6786|Tuesday|2030Q1|N|N|N|2462503|2462502|2462152|2462425|N|N|N|N|N| +2462518|AAAAAAAAGDDJFCAA|2030-01-16|1560|6786|521|2030|3|1|16|1|2030|521|6786|Wednesday|2030Q1|N|N|N|2462503|2462502|2462153|2462426|N|N|N|N|N| +2462519|AAAAAAAAHDDJFCAA|2030-01-17|1560|6786|521|2030|4|1|17|1|2030|521|6786|Thursday|2030Q1|N|N|N|2462503|2462502|2462154|2462427|N|N|N|N|N| +2462520|AAAAAAAAIDDJFCAA|2030-01-18|1560|6786|521|2030|5|1|18|1|2030|521|6786|Friday|2030Q1|N|Y|N|2462503|2462502|2462155|2462428|N|N|N|N|N| +2462521|AAAAAAAAJDDJFCAA|2030-01-19|1560|6786|521|2030|6|1|19|1|2030|521|6786|Saturday|2030Q1|N|Y|N|2462503|2462502|2462156|2462429|N|N|N|N|N| +2462522|AAAAAAAAKDDJFCAA|2030-01-20|1560|6786|521|2030|0|1|20|1|2030|521|6786|Sunday|2030Q1|N|N|N|2462503|2462502|2462157|2462430|N|N|N|N|N| +2462523|AAAAAAAALDDJFCAA|2030-01-21|1560|6786|521|2030|1|1|21|1|2030|521|6786|Monday|2030Q1|N|N|N|2462503|2462502|2462158|2462431|N|N|N|N|N| +2462524|AAAAAAAAMDDJFCAA|2030-01-22|1560|6787|521|2030|2|1|22|1|2030|521|6787|Tuesday|2030Q1|N|N|N|2462503|2462502|2462159|2462432|N|N|N|N|N| +2462525|AAAAAAAANDDJFCAA|2030-01-23|1560|6787|521|2030|3|1|23|1|2030|521|6787|Wednesday|2030Q1|N|N|N|2462503|2462502|2462160|2462433|N|N|N|N|N| +2462526|AAAAAAAAODDJFCAA|2030-01-24|1560|6787|521|2030|4|1|24|1|2030|521|6787|Thursday|2030Q1|N|N|N|2462503|2462502|2462161|2462434|N|N|N|N|N| +2462527|AAAAAAAAPDDJFCAA|2030-01-25|1560|6787|521|2030|5|1|25|1|2030|521|6787|Friday|2030Q1|N|Y|N|2462503|2462502|2462162|2462435|N|N|N|N|N| +2462528|AAAAAAAAAEDJFCAA|2030-01-26|1560|6787|521|2030|6|1|26|1|2030|521|6787|Saturday|2030Q1|N|Y|N|2462503|2462502|2462163|2462436|N|N|N|N|N| +2462529|AAAAAAAABEDJFCAA|2030-01-27|1560|6787|521|2030|0|1|27|1|2030|521|6787|Sunday|2030Q1|N|N|N|2462503|2462502|2462164|2462437|N|N|N|N|N| +2462530|AAAAAAAACEDJFCAA|2030-01-28|1560|6787|521|2030|1|1|28|1|2030|521|6787|Monday|2030Q1|N|N|N|2462503|2462502|2462165|2462438|N|N|N|N|N| +2462531|AAAAAAAADEDJFCAA|2030-01-29|1560|6788|521|2030|2|1|29|1|2030|521|6788|Tuesday|2030Q1|N|N|N|2462503|2462502|2462166|2462439|N|N|N|N|N| +2462532|AAAAAAAAEEDJFCAA|2030-01-30|1560|6788|521|2030|3|1|30|1|2030|521|6788|Wednesday|2030Q1|N|N|N|2462503|2462502|2462167|2462440|N|N|N|N|N| +2462533|AAAAAAAAFEDJFCAA|2030-01-31|1560|6788|521|2030|4|1|31|1|2030|521|6788|Thursday|2030Q1|N|N|N|2462503|2462502|2462168|2462441|N|N|N|N|N| +2462534|AAAAAAAAGEDJFCAA|2030-02-01|1561|6788|521|2030|5|2|1|1|2030|521|6788|Friday|2030Q1|N|Y|N|2462534|2462564|2462169|2462442|N|N|N|N|N| +2462535|AAAAAAAAHEDJFCAA|2030-02-02|1561|6788|521|2030|6|2|2|1|2030|521|6788|Saturday|2030Q1|N|Y|N|2462534|2462564|2462170|2462443|N|N|N|N|N| +2462536|AAAAAAAAIEDJFCAA|2030-02-03|1561|6788|521|2030|0|2|3|1|2030|521|6788|Sunday|2030Q1|N|N|N|2462534|2462564|2462171|2462444|N|N|N|N|N| +2462537|AAAAAAAAJEDJFCAA|2030-02-04|1561|6788|521|2030|1|2|4|1|2030|521|6788|Monday|2030Q1|N|N|N|2462534|2462564|2462172|2462445|N|N|N|N|N| +2462538|AAAAAAAAKEDJFCAA|2030-02-05|1561|6789|521|2030|2|2|5|1|2030|521|6789|Tuesday|2030Q1|N|N|N|2462534|2462564|2462173|2462446|N|N|N|N|N| +2462539|AAAAAAAALEDJFCAA|2030-02-06|1561|6789|521|2030|3|2|6|1|2030|521|6789|Wednesday|2030Q1|N|N|N|2462534|2462564|2462174|2462447|N|N|N|N|N| +2462540|AAAAAAAAMEDJFCAA|2030-02-07|1561|6789|521|2030|4|2|7|1|2030|521|6789|Thursday|2030Q1|N|N|N|2462534|2462564|2462175|2462448|N|N|N|N|N| +2462541|AAAAAAAANEDJFCAA|2030-02-08|1561|6789|521|2030|5|2|8|1|2030|521|6789|Friday|2030Q1|N|Y|N|2462534|2462564|2462176|2462449|N|N|N|N|N| +2462542|AAAAAAAAOEDJFCAA|2030-02-09|1561|6789|521|2030|6|2|9|1|2030|521|6789|Saturday|2030Q1|N|Y|N|2462534|2462564|2462177|2462450|N|N|N|N|N| +2462543|AAAAAAAAPEDJFCAA|2030-02-10|1561|6789|521|2030|0|2|10|1|2030|521|6789|Sunday|2030Q1|N|N|N|2462534|2462564|2462178|2462451|N|N|N|N|N| +2462544|AAAAAAAAAFDJFCAA|2030-02-11|1561|6789|521|2030|1|2|11|1|2030|521|6789|Monday|2030Q1|N|N|N|2462534|2462564|2462179|2462452|N|N|N|N|N| +2462545|AAAAAAAABFDJFCAA|2030-02-12|1561|6790|521|2030|2|2|12|1|2030|521|6790|Tuesday|2030Q1|N|N|N|2462534|2462564|2462180|2462453|N|N|N|N|N| +2462546|AAAAAAAACFDJFCAA|2030-02-13|1561|6790|521|2030|3|2|13|1|2030|521|6790|Wednesday|2030Q1|N|N|N|2462534|2462564|2462181|2462454|N|N|N|N|N| +2462547|AAAAAAAADFDJFCAA|2030-02-14|1561|6790|521|2030|4|2|14|1|2030|521|6790|Thursday|2030Q1|N|N|N|2462534|2462564|2462182|2462455|N|N|N|N|N| +2462548|AAAAAAAAEFDJFCAA|2030-02-15|1561|6790|521|2030|5|2|15|1|2030|521|6790|Friday|2030Q1|N|Y|N|2462534|2462564|2462183|2462456|N|N|N|N|N| +2462549|AAAAAAAAFFDJFCAA|2030-02-16|1561|6790|521|2030|6|2|16|1|2030|521|6790|Saturday|2030Q1|N|Y|N|2462534|2462564|2462184|2462457|N|N|N|N|N| +2462550|AAAAAAAAGFDJFCAA|2030-02-17|1561|6790|521|2030|0|2|17|1|2030|521|6790|Sunday|2030Q1|N|N|N|2462534|2462564|2462185|2462458|N|N|N|N|N| +2462551|AAAAAAAAHFDJFCAA|2030-02-18|1561|6790|521|2030|1|2|18|1|2030|521|6790|Monday|2030Q1|N|N|N|2462534|2462564|2462186|2462459|N|N|N|N|N| +2462552|AAAAAAAAIFDJFCAA|2030-02-19|1561|6791|521|2030|2|2|19|1|2030|521|6791|Tuesday|2030Q1|N|N|N|2462534|2462564|2462187|2462460|N|N|N|N|N| +2462553|AAAAAAAAJFDJFCAA|2030-02-20|1561|6791|521|2030|3|2|20|1|2030|521|6791|Wednesday|2030Q1|N|N|N|2462534|2462564|2462188|2462461|N|N|N|N|N| +2462554|AAAAAAAAKFDJFCAA|2030-02-21|1561|6791|521|2030|4|2|21|1|2030|521|6791|Thursday|2030Q1|N|N|N|2462534|2462564|2462189|2462462|N|N|N|N|N| +2462555|AAAAAAAALFDJFCAA|2030-02-22|1561|6791|521|2030|5|2|22|1|2030|521|6791|Friday|2030Q1|N|Y|N|2462534|2462564|2462190|2462463|N|N|N|N|N| +2462556|AAAAAAAAMFDJFCAA|2030-02-23|1561|6791|521|2030|6|2|23|1|2030|521|6791|Saturday|2030Q1|N|Y|N|2462534|2462564|2462191|2462464|N|N|N|N|N| +2462557|AAAAAAAANFDJFCAA|2030-02-24|1561|6791|521|2030|0|2|24|1|2030|521|6791|Sunday|2030Q1|N|N|N|2462534|2462564|2462192|2462465|N|N|N|N|N| +2462558|AAAAAAAAOFDJFCAA|2030-02-25|1561|6791|521|2030|1|2|25|1|2030|521|6791|Monday|2030Q1|N|N|N|2462534|2462564|2462193|2462466|N|N|N|N|N| +2462559|AAAAAAAAPFDJFCAA|2030-02-26|1561|6792|521|2030|2|2|26|1|2030|521|6792|Tuesday|2030Q1|N|N|N|2462534|2462564|2462194|2462467|N|N|N|N|N| +2462560|AAAAAAAAAGDJFCAA|2030-02-27|1561|6792|521|2030|3|2|27|1|2030|521|6792|Wednesday|2030Q1|N|N|N|2462534|2462564|2462195|2462468|N|N|N|N|N| +2462561|AAAAAAAABGDJFCAA|2030-02-28|1561|6792|521|2030|4|2|28|1|2030|521|6792|Thursday|2030Q1|N|N|N|2462534|2462564|2462196|2462469|N|N|N|N|N| +2462562|AAAAAAAACGDJFCAA|2030-03-01|1562|6792|522|2030|5|3|1|1|2030|522|6792|Friday|2030Q1|N|Y|N|2462562|2462620|2462197|2462470|N|N|N|N|N| +2462563|AAAAAAAADGDJFCAA|2030-03-02|1562|6792|522|2030|6|3|2|1|2030|522|6792|Saturday|2030Q1|N|Y|N|2462562|2462620|2462198|2462471|N|N|N|N|N| +2462564|AAAAAAAAEGDJFCAA|2030-03-03|1562|6792|522|2030|0|3|3|1|2030|522|6792|Sunday|2030Q1|N|N|N|2462562|2462620|2462199|2462472|N|N|N|N|N| +2462565|AAAAAAAAFGDJFCAA|2030-03-04|1562|6792|522|2030|1|3|4|1|2030|522|6792|Monday|2030Q1|N|N|N|2462562|2462620|2462200|2462473|N|N|N|N|N| +2462566|AAAAAAAAGGDJFCAA|2030-03-05|1562|6793|522|2030|2|3|5|1|2030|522|6793|Tuesday|2030Q1|N|N|N|2462562|2462620|2462201|2462474|N|N|N|N|N| +2462567|AAAAAAAAHGDJFCAA|2030-03-06|1562|6793|522|2030|3|3|6|1|2030|522|6793|Wednesday|2030Q1|N|N|N|2462562|2462620|2462202|2462475|N|N|N|N|N| +2462568|AAAAAAAAIGDJFCAA|2030-03-07|1562|6793|522|2030|4|3|7|1|2030|522|6793|Thursday|2030Q1|N|N|N|2462562|2462620|2462203|2462476|N|N|N|N|N| +2462569|AAAAAAAAJGDJFCAA|2030-03-08|1562|6793|522|2030|5|3|8|1|2030|522|6793|Friday|2030Q1|N|Y|N|2462562|2462620|2462204|2462477|N|N|N|N|N| +2462570|AAAAAAAAKGDJFCAA|2030-03-09|1562|6793|522|2030|6|3|9|1|2030|522|6793|Saturday|2030Q1|N|Y|N|2462562|2462620|2462205|2462478|N|N|N|N|N| +2462571|AAAAAAAALGDJFCAA|2030-03-10|1562|6793|522|2030|0|3|10|1|2030|522|6793|Sunday|2030Q1|N|N|N|2462562|2462620|2462206|2462479|N|N|N|N|N| +2462572|AAAAAAAAMGDJFCAA|2030-03-11|1562|6793|522|2030|1|3|11|1|2030|522|6793|Monday|2030Q1|N|N|N|2462562|2462620|2462207|2462480|N|N|N|N|N| +2462573|AAAAAAAANGDJFCAA|2030-03-12|1562|6794|522|2030|2|3|12|1|2030|522|6794|Tuesday|2030Q1|N|N|N|2462562|2462620|2462208|2462481|N|N|N|N|N| +2462574|AAAAAAAAOGDJFCAA|2030-03-13|1562|6794|522|2030|3|3|13|1|2030|522|6794|Wednesday|2030Q1|N|N|N|2462562|2462620|2462209|2462482|N|N|N|N|N| +2462575|AAAAAAAAPGDJFCAA|2030-03-14|1562|6794|522|2030|4|3|14|1|2030|522|6794|Thursday|2030Q1|N|N|N|2462562|2462620|2462210|2462483|N|N|N|N|N| +2462576|AAAAAAAAAHDJFCAA|2030-03-15|1562|6794|522|2030|5|3|15|1|2030|522|6794|Friday|2030Q1|N|Y|N|2462562|2462620|2462211|2462484|N|N|N|N|N| +2462577|AAAAAAAABHDJFCAA|2030-03-16|1562|6794|522|2030|6|3|16|1|2030|522|6794|Saturday|2030Q1|N|Y|N|2462562|2462620|2462212|2462485|N|N|N|N|N| +2462578|AAAAAAAACHDJFCAA|2030-03-17|1562|6794|522|2030|0|3|17|1|2030|522|6794|Sunday|2030Q1|N|N|N|2462562|2462620|2462213|2462486|N|N|N|N|N| +2462579|AAAAAAAADHDJFCAA|2030-03-18|1562|6794|522|2030|1|3|18|1|2030|522|6794|Monday|2030Q1|N|N|N|2462562|2462620|2462214|2462487|N|N|N|N|N| +2462580|AAAAAAAAEHDJFCAA|2030-03-19|1562|6795|522|2030|2|3|19|1|2030|522|6795|Tuesday|2030Q1|N|N|N|2462562|2462620|2462215|2462488|N|N|N|N|N| +2462581|AAAAAAAAFHDJFCAA|2030-03-20|1562|6795|522|2030|3|3|20|1|2030|522|6795|Wednesday|2030Q1|N|N|N|2462562|2462620|2462216|2462489|N|N|N|N|N| +2462582|AAAAAAAAGHDJFCAA|2030-03-21|1562|6795|522|2030|4|3|21|1|2030|522|6795|Thursday|2030Q1|N|N|N|2462562|2462620|2462217|2462490|N|N|N|N|N| +2462583|AAAAAAAAHHDJFCAA|2030-03-22|1562|6795|522|2030|5|3|22|1|2030|522|6795|Friday|2030Q1|N|Y|N|2462562|2462620|2462218|2462491|N|N|N|N|N| +2462584|AAAAAAAAIHDJFCAA|2030-03-23|1562|6795|522|2030|6|3|23|1|2030|522|6795|Saturday|2030Q1|N|Y|N|2462562|2462620|2462219|2462492|N|N|N|N|N| +2462585|AAAAAAAAJHDJFCAA|2030-03-24|1562|6795|522|2030|0|3|24|1|2030|522|6795|Sunday|2030Q1|N|N|N|2462562|2462620|2462220|2462493|N|N|N|N|N| +2462586|AAAAAAAAKHDJFCAA|2030-03-25|1562|6795|522|2030|1|3|25|1|2030|522|6795|Monday|2030Q1|N|N|N|2462562|2462620|2462221|2462494|N|N|N|N|N| +2462587|AAAAAAAALHDJFCAA|2030-03-26|1562|6796|522|2030|2|3|26|1|2030|522|6796|Tuesday|2030Q1|N|N|N|2462562|2462620|2462222|2462495|N|N|N|N|N| +2462588|AAAAAAAAMHDJFCAA|2030-03-27|1562|6796|522|2030|3|3|27|1|2030|522|6796|Wednesday|2030Q1|N|N|N|2462562|2462620|2462223|2462496|N|N|N|N|N| +2462589|AAAAAAAANHDJFCAA|2030-03-28|1562|6796|522|2030|4|3|28|1|2030|522|6796|Thursday|2030Q1|N|N|N|2462562|2462620|2462224|2462497|N|N|N|N|N| +2462590|AAAAAAAAOHDJFCAA|2030-03-29|1562|6796|522|2030|5|3|29|1|2030|522|6796|Friday|2030Q1|N|Y|N|2462562|2462620|2462225|2462498|N|N|N|N|N| +2462591|AAAAAAAAPHDJFCAA|2030-03-30|1562|6796|522|2030|6|3|30|1|2030|522|6796|Saturday|2030Q1|N|Y|N|2462562|2462620|2462226|2462499|N|N|N|N|N| +2462592|AAAAAAAAAIDJFCAA|2030-03-31|1562|6796|522|2030|0|3|31|1|2030|522|6796|Sunday|2030Q1|N|N|N|2462562|2462620|2462227|2462500|N|N|N|N|N| +2462593|AAAAAAAABIDJFCAA|2030-04-01|1563|6796|522|2030|1|4|1|1|2030|522|6796|Monday|2030Q1|N|N|N|2462593|2462682|2462228|2462503|N|N|N|N|N| +2462594|AAAAAAAACIDJFCAA|2030-04-02|1563|6797|522|2030|2|4|2|2|2030|522|6797|Tuesday|2030Q2|N|N|N|2462593|2462682|2462229|2462504|N|N|N|N|N| +2462595|AAAAAAAADIDJFCAA|2030-04-03|1563|6797|522|2030|3|4|3|2|2030|522|6797|Wednesday|2030Q2|N|N|N|2462593|2462682|2462230|2462505|N|N|N|N|N| +2462596|AAAAAAAAEIDJFCAA|2030-04-04|1563|6797|522|2030|4|4|4|2|2030|522|6797|Thursday|2030Q2|N|N|N|2462593|2462682|2462231|2462506|N|N|N|N|N| +2462597|AAAAAAAAFIDJFCAA|2030-04-05|1563|6797|522|2030|5|4|5|2|2030|522|6797|Friday|2030Q2|N|Y|N|2462593|2462682|2462232|2462507|N|N|N|N|N| +2462598|AAAAAAAAGIDJFCAA|2030-04-06|1563|6797|522|2030|6|4|6|2|2030|522|6797|Saturday|2030Q2|N|Y|N|2462593|2462682|2462233|2462508|N|N|N|N|N| +2462599|AAAAAAAAHIDJFCAA|2030-04-07|1563|6797|522|2030|0|4|7|2|2030|522|6797|Sunday|2030Q2|N|N|N|2462593|2462682|2462234|2462509|N|N|N|N|N| +2462600|AAAAAAAAIIDJFCAA|2030-04-08|1563|6797|522|2030|1|4|8|2|2030|522|6797|Monday|2030Q2|N|N|N|2462593|2462682|2462235|2462510|N|N|N|N|N| +2462601|AAAAAAAAJIDJFCAA|2030-04-09|1563|6798|522|2030|2|4|9|2|2030|522|6798|Tuesday|2030Q2|N|N|N|2462593|2462682|2462236|2462511|N|N|N|N|N| +2462602|AAAAAAAAKIDJFCAA|2030-04-10|1563|6798|522|2030|3|4|10|2|2030|522|6798|Wednesday|2030Q2|N|N|N|2462593|2462682|2462237|2462512|N|N|N|N|N| +2462603|AAAAAAAALIDJFCAA|2030-04-11|1563|6798|522|2030|4|4|11|2|2030|522|6798|Thursday|2030Q2|N|N|N|2462593|2462682|2462238|2462513|N|N|N|N|N| +2462604|AAAAAAAAMIDJFCAA|2030-04-12|1563|6798|522|2030|5|4|12|2|2030|522|6798|Friday|2030Q2|N|Y|N|2462593|2462682|2462239|2462514|N|N|N|N|N| +2462605|AAAAAAAANIDJFCAA|2030-04-13|1563|6798|522|2030|6|4|13|2|2030|522|6798|Saturday|2030Q2|N|Y|N|2462593|2462682|2462240|2462515|N|N|N|N|N| +2462606|AAAAAAAAOIDJFCAA|2030-04-14|1563|6798|522|2030|0|4|14|2|2030|522|6798|Sunday|2030Q2|N|N|N|2462593|2462682|2462241|2462516|N|N|N|N|N| +2462607|AAAAAAAAPIDJFCAA|2030-04-15|1563|6798|522|2030|1|4|15|2|2030|522|6798|Monday|2030Q2|N|N|N|2462593|2462682|2462242|2462517|N|N|N|N|N| +2462608|AAAAAAAAAJDJFCAA|2030-04-16|1563|6799|522|2030|2|4|16|2|2030|522|6799|Tuesday|2030Q2|N|N|N|2462593|2462682|2462243|2462518|N|N|N|N|N| +2462609|AAAAAAAABJDJFCAA|2030-04-17|1563|6799|522|2030|3|4|17|2|2030|522|6799|Wednesday|2030Q2|N|N|N|2462593|2462682|2462244|2462519|N|N|N|N|N| +2462610|AAAAAAAACJDJFCAA|2030-04-18|1563|6799|522|2030|4|4|18|2|2030|522|6799|Thursday|2030Q2|N|N|N|2462593|2462682|2462245|2462520|N|N|N|N|N| +2462611|AAAAAAAADJDJFCAA|2030-04-19|1563|6799|522|2030|5|4|19|2|2030|522|6799|Friday|2030Q2|N|Y|N|2462593|2462682|2462246|2462521|N|N|N|N|N| +2462612|AAAAAAAAEJDJFCAA|2030-04-20|1563|6799|522|2030|6|4|20|2|2030|522|6799|Saturday|2030Q2|N|Y|N|2462593|2462682|2462247|2462522|N|N|N|N|N| +2462613|AAAAAAAAFJDJFCAA|2030-04-21|1563|6799|522|2030|0|4|21|2|2030|522|6799|Sunday|2030Q2|N|N|N|2462593|2462682|2462248|2462523|N|N|N|N|N| +2462614|AAAAAAAAGJDJFCAA|2030-04-22|1563|6799|522|2030|1|4|22|2|2030|522|6799|Monday|2030Q2|N|N|N|2462593|2462682|2462249|2462524|N|N|N|N|N| +2462615|AAAAAAAAHJDJFCAA|2030-04-23|1563|6800|522|2030|2|4|23|2|2030|522|6800|Tuesday|2030Q2|N|N|N|2462593|2462682|2462250|2462525|N|N|N|N|N| +2462616|AAAAAAAAIJDJFCAA|2030-04-24|1563|6800|522|2030|3|4|24|2|2030|522|6800|Wednesday|2030Q2|N|N|N|2462593|2462682|2462251|2462526|N|N|N|N|N| +2462617|AAAAAAAAJJDJFCAA|2030-04-25|1563|6800|522|2030|4|4|25|2|2030|522|6800|Thursday|2030Q2|N|N|N|2462593|2462682|2462252|2462527|N|N|N|N|N| +2462618|AAAAAAAAKJDJFCAA|2030-04-26|1563|6800|522|2030|5|4|26|2|2030|522|6800|Friday|2030Q2|N|Y|N|2462593|2462682|2462253|2462528|N|N|N|N|N| +2462619|AAAAAAAALJDJFCAA|2030-04-27|1563|6800|522|2030|6|4|27|2|2030|522|6800|Saturday|2030Q2|N|Y|N|2462593|2462682|2462254|2462529|N|N|N|N|N| +2462620|AAAAAAAAMJDJFCAA|2030-04-28|1563|6800|522|2030|0|4|28|2|2030|522|6800|Sunday|2030Q2|N|N|N|2462593|2462682|2462255|2462530|N|N|N|N|N| +2462621|AAAAAAAANJDJFCAA|2030-04-29|1563|6800|522|2030|1|4|29|2|2030|522|6800|Monday|2030Q2|N|N|N|2462593|2462682|2462256|2462531|N|N|N|N|N| +2462622|AAAAAAAAOJDJFCAA|2030-04-30|1563|6801|522|2030|2|4|30|2|2030|522|6801|Tuesday|2030Q2|N|N|N|2462593|2462682|2462257|2462532|N|N|N|N|N| +2462623|AAAAAAAAPJDJFCAA|2030-05-01|1564|6801|522|2030|3|5|1|2|2030|522|6801|Wednesday|2030Q2|N|N|N|2462623|2462742|2462258|2462533|N|N|N|N|N| +2462624|AAAAAAAAAKDJFCAA|2030-05-02|1564|6801|522|2030|4|5|2|2|2030|522|6801|Thursday|2030Q2|N|N|N|2462623|2462742|2462259|2462534|N|N|N|N|N| +2462625|AAAAAAAABKDJFCAA|2030-05-03|1564|6801|522|2030|5|5|3|2|2030|522|6801|Friday|2030Q2|N|Y|N|2462623|2462742|2462260|2462535|N|N|N|N|N| +2462626|AAAAAAAACKDJFCAA|2030-05-04|1564|6801|522|2030|6|5|4|2|2030|522|6801|Saturday|2030Q2|N|Y|N|2462623|2462742|2462261|2462536|N|N|N|N|N| +2462627|AAAAAAAADKDJFCAA|2030-05-05|1564|6801|522|2030|0|5|5|2|2030|522|6801|Sunday|2030Q2|N|N|N|2462623|2462742|2462262|2462537|N|N|N|N|N| +2462628|AAAAAAAAEKDJFCAA|2030-05-06|1564|6801|522|2030|1|5|6|2|2030|522|6801|Monday|2030Q2|N|N|N|2462623|2462742|2462263|2462538|N|N|N|N|N| +2462629|AAAAAAAAFKDJFCAA|2030-05-07|1564|6802|522|2030|2|5|7|2|2030|522|6802|Tuesday|2030Q2|N|N|N|2462623|2462742|2462264|2462539|N|N|N|N|N| +2462630|AAAAAAAAGKDJFCAA|2030-05-08|1564|6802|522|2030|3|5|8|2|2030|522|6802|Wednesday|2030Q2|N|N|N|2462623|2462742|2462265|2462540|N|N|N|N|N| +2462631|AAAAAAAAHKDJFCAA|2030-05-09|1564|6802|522|2030|4|5|9|2|2030|522|6802|Thursday|2030Q2|N|N|N|2462623|2462742|2462266|2462541|N|N|N|N|N| +2462632|AAAAAAAAIKDJFCAA|2030-05-10|1564|6802|522|2030|5|5|10|2|2030|522|6802|Friday|2030Q2|N|Y|N|2462623|2462742|2462267|2462542|N|N|N|N|N| +2462633|AAAAAAAAJKDJFCAA|2030-05-11|1564|6802|522|2030|6|5|11|2|2030|522|6802|Saturday|2030Q2|N|Y|N|2462623|2462742|2462268|2462543|N|N|N|N|N| +2462634|AAAAAAAAKKDJFCAA|2030-05-12|1564|6802|522|2030|0|5|12|2|2030|522|6802|Sunday|2030Q2|N|N|N|2462623|2462742|2462269|2462544|N|N|N|N|N| +2462635|AAAAAAAALKDJFCAA|2030-05-13|1564|6802|522|2030|1|5|13|2|2030|522|6802|Monday|2030Q2|N|N|N|2462623|2462742|2462270|2462545|N|N|N|N|N| +2462636|AAAAAAAAMKDJFCAA|2030-05-14|1564|6803|522|2030|2|5|14|2|2030|522|6803|Tuesday|2030Q2|N|N|N|2462623|2462742|2462271|2462546|N|N|N|N|N| +2462637|AAAAAAAANKDJFCAA|2030-05-15|1564|6803|522|2030|3|5|15|2|2030|522|6803|Wednesday|2030Q2|N|N|N|2462623|2462742|2462272|2462547|N|N|N|N|N| +2462638|AAAAAAAAOKDJFCAA|2030-05-16|1564|6803|522|2030|4|5|16|2|2030|522|6803|Thursday|2030Q2|N|N|N|2462623|2462742|2462273|2462548|N|N|N|N|N| +2462639|AAAAAAAAPKDJFCAA|2030-05-17|1564|6803|522|2030|5|5|17|2|2030|522|6803|Friday|2030Q2|N|Y|N|2462623|2462742|2462274|2462549|N|N|N|N|N| +2462640|AAAAAAAAALDJFCAA|2030-05-18|1564|6803|522|2030|6|5|18|2|2030|522|6803|Saturday|2030Q2|N|Y|N|2462623|2462742|2462275|2462550|N|N|N|N|N| +2462641|AAAAAAAABLDJFCAA|2030-05-19|1564|6803|522|2030|0|5|19|2|2030|522|6803|Sunday|2030Q2|N|N|N|2462623|2462742|2462276|2462551|N|N|N|N|N| +2462642|AAAAAAAACLDJFCAA|2030-05-20|1564|6803|522|2030|1|5|20|2|2030|522|6803|Monday|2030Q2|N|N|N|2462623|2462742|2462277|2462552|N|N|N|N|N| +2462643|AAAAAAAADLDJFCAA|2030-05-21|1564|6804|522|2030|2|5|21|2|2030|522|6804|Tuesday|2030Q2|N|N|N|2462623|2462742|2462278|2462553|N|N|N|N|N| +2462644|AAAAAAAAELDJFCAA|2030-05-22|1564|6804|522|2030|3|5|22|2|2030|522|6804|Wednesday|2030Q2|N|N|N|2462623|2462742|2462279|2462554|N|N|N|N|N| +2462645|AAAAAAAAFLDJFCAA|2030-05-23|1564|6804|522|2030|4|5|23|2|2030|522|6804|Thursday|2030Q2|N|N|N|2462623|2462742|2462280|2462555|N|N|N|N|N| +2462646|AAAAAAAAGLDJFCAA|2030-05-24|1564|6804|522|2030|5|5|24|2|2030|522|6804|Friday|2030Q2|N|Y|N|2462623|2462742|2462281|2462556|N|N|N|N|N| +2462647|AAAAAAAAHLDJFCAA|2030-05-25|1564|6804|522|2030|6|5|25|2|2030|522|6804|Saturday|2030Q2|N|Y|N|2462623|2462742|2462282|2462557|N|N|N|N|N| +2462648|AAAAAAAAILDJFCAA|2030-05-26|1564|6804|522|2030|0|5|26|2|2030|522|6804|Sunday|2030Q2|N|N|N|2462623|2462742|2462283|2462558|N|N|N|N|N| +2462649|AAAAAAAAJLDJFCAA|2030-05-27|1564|6804|522|2030|1|5|27|2|2030|522|6804|Monday|2030Q2|N|N|N|2462623|2462742|2462284|2462559|N|N|N|N|N| +2462650|AAAAAAAAKLDJFCAA|2030-05-28|1564|6805|522|2030|2|5|28|2|2030|522|6805|Tuesday|2030Q2|N|N|N|2462623|2462742|2462285|2462560|N|N|N|N|N| +2462651|AAAAAAAALLDJFCAA|2030-05-29|1564|6805|522|2030|3|5|29|2|2030|522|6805|Wednesday|2030Q2|N|N|N|2462623|2462742|2462286|2462561|N|N|N|N|N| +2462652|AAAAAAAAMLDJFCAA|2030-05-30|1564|6805|522|2030|4|5|30|2|2030|522|6805|Thursday|2030Q2|N|N|N|2462623|2462742|2462287|2462562|N|N|N|N|N| +2462653|AAAAAAAANLDJFCAA|2030-05-31|1564|6805|522|2030|5|5|31|2|2030|522|6805|Friday|2030Q2|N|Y|N|2462623|2462742|2462288|2462563|N|N|N|N|N| +2462654|AAAAAAAAOLDJFCAA|2030-06-01|1565|6805|523|2030|6|6|1|2|2030|523|6805|Saturday|2030Q2|N|Y|N|2462654|2462804|2462289|2462564|N|N|N|N|N| +2462655|AAAAAAAAPLDJFCAA|2030-06-02|1565|6805|523|2030|0|6|2|2|2030|523|6805|Sunday|2030Q2|N|N|N|2462654|2462804|2462290|2462565|N|N|N|N|N| +2462656|AAAAAAAAAMDJFCAA|2030-06-03|1565|6805|523|2030|1|6|3|2|2030|523|6805|Monday|2030Q2|N|N|N|2462654|2462804|2462291|2462566|N|N|N|N|N| +2462657|AAAAAAAABMDJFCAA|2030-06-04|1565|6806|523|2030|2|6|4|2|2030|523|6806|Tuesday|2030Q2|N|N|N|2462654|2462804|2462292|2462567|N|N|N|N|N| +2462658|AAAAAAAACMDJFCAA|2030-06-05|1565|6806|523|2030|3|6|5|2|2030|523|6806|Wednesday|2030Q2|N|N|N|2462654|2462804|2462293|2462568|N|N|N|N|N| +2462659|AAAAAAAADMDJFCAA|2030-06-06|1565|6806|523|2030|4|6|6|2|2030|523|6806|Thursday|2030Q2|N|N|N|2462654|2462804|2462294|2462569|N|N|N|N|N| +2462660|AAAAAAAAEMDJFCAA|2030-06-07|1565|6806|523|2030|5|6|7|2|2030|523|6806|Friday|2030Q2|N|Y|N|2462654|2462804|2462295|2462570|N|N|N|N|N| +2462661|AAAAAAAAFMDJFCAA|2030-06-08|1565|6806|523|2030|6|6|8|2|2030|523|6806|Saturday|2030Q2|N|Y|N|2462654|2462804|2462296|2462571|N|N|N|N|N| +2462662|AAAAAAAAGMDJFCAA|2030-06-09|1565|6806|523|2030|0|6|9|2|2030|523|6806|Sunday|2030Q2|N|N|N|2462654|2462804|2462297|2462572|N|N|N|N|N| +2462663|AAAAAAAAHMDJFCAA|2030-06-10|1565|6806|523|2030|1|6|10|2|2030|523|6806|Monday|2030Q2|N|N|N|2462654|2462804|2462298|2462573|N|N|N|N|N| +2462664|AAAAAAAAIMDJFCAA|2030-06-11|1565|6807|523|2030|2|6|11|2|2030|523|6807|Tuesday|2030Q2|N|N|N|2462654|2462804|2462299|2462574|N|N|N|N|N| +2462665|AAAAAAAAJMDJFCAA|2030-06-12|1565|6807|523|2030|3|6|12|2|2030|523|6807|Wednesday|2030Q2|N|N|N|2462654|2462804|2462300|2462575|N|N|N|N|N| +2462666|AAAAAAAAKMDJFCAA|2030-06-13|1565|6807|523|2030|4|6|13|2|2030|523|6807|Thursday|2030Q2|N|N|N|2462654|2462804|2462301|2462576|N|N|N|N|N| +2462667|AAAAAAAALMDJFCAA|2030-06-14|1565|6807|523|2030|5|6|14|2|2030|523|6807|Friday|2030Q2|N|Y|N|2462654|2462804|2462302|2462577|N|N|N|N|N| +2462668|AAAAAAAAMMDJFCAA|2030-06-15|1565|6807|523|2030|6|6|15|2|2030|523|6807|Saturday|2030Q2|N|Y|N|2462654|2462804|2462303|2462578|N|N|N|N|N| +2462669|AAAAAAAANMDJFCAA|2030-06-16|1565|6807|523|2030|0|6|16|2|2030|523|6807|Sunday|2030Q2|N|N|N|2462654|2462804|2462304|2462579|N|N|N|N|N| +2462670|AAAAAAAAOMDJFCAA|2030-06-17|1565|6807|523|2030|1|6|17|2|2030|523|6807|Monday|2030Q2|N|N|N|2462654|2462804|2462305|2462580|N|N|N|N|N| +2462671|AAAAAAAAPMDJFCAA|2030-06-18|1565|6808|523|2030|2|6|18|2|2030|523|6808|Tuesday|2030Q2|N|N|N|2462654|2462804|2462306|2462581|N|N|N|N|N| +2462672|AAAAAAAAANDJFCAA|2030-06-19|1565|6808|523|2030|3|6|19|2|2030|523|6808|Wednesday|2030Q2|N|N|N|2462654|2462804|2462307|2462582|N|N|N|N|N| +2462673|AAAAAAAABNDJFCAA|2030-06-20|1565|6808|523|2030|4|6|20|2|2030|523|6808|Thursday|2030Q2|N|N|N|2462654|2462804|2462308|2462583|N|N|N|N|N| +2462674|AAAAAAAACNDJFCAA|2030-06-21|1565|6808|523|2030|5|6|21|2|2030|523|6808|Friday|2030Q2|N|Y|N|2462654|2462804|2462309|2462584|N|N|N|N|N| +2462675|AAAAAAAADNDJFCAA|2030-06-22|1565|6808|523|2030|6|6|22|2|2030|523|6808|Saturday|2030Q2|N|Y|N|2462654|2462804|2462310|2462585|N|N|N|N|N| +2462676|AAAAAAAAENDJFCAA|2030-06-23|1565|6808|523|2030|0|6|23|2|2030|523|6808|Sunday|2030Q2|N|N|N|2462654|2462804|2462311|2462586|N|N|N|N|N| +2462677|AAAAAAAAFNDJFCAA|2030-06-24|1565|6808|523|2030|1|6|24|2|2030|523|6808|Monday|2030Q2|N|N|N|2462654|2462804|2462312|2462587|N|N|N|N|N| +2462678|AAAAAAAAGNDJFCAA|2030-06-25|1565|6809|523|2030|2|6|25|2|2030|523|6809|Tuesday|2030Q2|N|N|N|2462654|2462804|2462313|2462588|N|N|N|N|N| +2462679|AAAAAAAAHNDJFCAA|2030-06-26|1565|6809|523|2030|3|6|26|2|2030|523|6809|Wednesday|2030Q2|N|N|N|2462654|2462804|2462314|2462589|N|N|N|N|N| +2462680|AAAAAAAAINDJFCAA|2030-06-27|1565|6809|523|2030|4|6|27|2|2030|523|6809|Thursday|2030Q2|N|N|N|2462654|2462804|2462315|2462590|N|N|N|N|N| +2462681|AAAAAAAAJNDJFCAA|2030-06-28|1565|6809|523|2030|5|6|28|2|2030|523|6809|Friday|2030Q2|N|Y|N|2462654|2462804|2462316|2462591|N|N|N|N|N| +2462682|AAAAAAAAKNDJFCAA|2030-06-29|1565|6809|523|2030|6|6|29|2|2030|523|6809|Saturday|2030Q2|N|Y|N|2462654|2462804|2462317|2462592|N|N|N|N|N| +2462683|AAAAAAAALNDJFCAA|2030-06-30|1565|6809|523|2030|0|6|30|2|2030|523|6809|Sunday|2030Q2|N|N|N|2462654|2462804|2462318|2462593|N|N|N|N|N| +2462684|AAAAAAAAMNDJFCAA|2030-07-01|1566|6809|523|2030|1|7|1|2|2030|523|6809|Monday|2030Q2|N|N|N|2462684|2462864|2462319|2462593|N|N|N|N|N| +2462685|AAAAAAAANNDJFCAA|2030-07-02|1566|6810|523|2030|2|7|2|3|2030|523|6810|Tuesday|2030Q3|N|N|N|2462684|2462864|2462320|2462594|N|N|N|N|N| +2462686|AAAAAAAAONDJFCAA|2030-07-03|1566|6810|523|2030|3|7|3|3|2030|523|6810|Wednesday|2030Q3|N|N|N|2462684|2462864|2462321|2462595|N|N|N|N|N| +2462687|AAAAAAAAPNDJFCAA|2030-07-04|1566|6810|523|2030|4|7|4|3|2030|523|6810|Thursday|2030Q3|N|N|N|2462684|2462864|2462322|2462596|N|N|N|N|N| +2462688|AAAAAAAAAODJFCAA|2030-07-05|1566|6810|523|2030|5|7|5|3|2030|523|6810|Friday|2030Q3|Y|Y|N|2462684|2462864|2462323|2462597|N|N|N|N|N| +2462689|AAAAAAAABODJFCAA|2030-07-06|1566|6810|523|2030|6|7|6|3|2030|523|6810|Saturday|2030Q3|N|Y|Y|2462684|2462864|2462324|2462598|N|N|N|N|N| +2462690|AAAAAAAACODJFCAA|2030-07-07|1566|6810|523|2030|0|7|7|3|2030|523|6810|Sunday|2030Q3|N|N|N|2462684|2462864|2462325|2462599|N|N|N|N|N| +2462691|AAAAAAAADODJFCAA|2030-07-08|1566|6810|523|2030|1|7|8|3|2030|523|6810|Monday|2030Q3|N|N|N|2462684|2462864|2462326|2462600|N|N|N|N|N| +2462692|AAAAAAAAEODJFCAA|2030-07-09|1566|6811|523|2030|2|7|9|3|2030|523|6811|Tuesday|2030Q3|N|N|N|2462684|2462864|2462327|2462601|N|N|N|N|N| +2462693|AAAAAAAAFODJFCAA|2030-07-10|1566|6811|523|2030|3|7|10|3|2030|523|6811|Wednesday|2030Q3|N|N|N|2462684|2462864|2462328|2462602|N|N|N|N|N| +2462694|AAAAAAAAGODJFCAA|2030-07-11|1566|6811|523|2030|4|7|11|3|2030|523|6811|Thursday|2030Q3|N|N|N|2462684|2462864|2462329|2462603|N|N|N|N|N| +2462695|AAAAAAAAHODJFCAA|2030-07-12|1566|6811|523|2030|5|7|12|3|2030|523|6811|Friday|2030Q3|N|Y|N|2462684|2462864|2462330|2462604|N|N|N|N|N| +2462696|AAAAAAAAIODJFCAA|2030-07-13|1566|6811|523|2030|6|7|13|3|2030|523|6811|Saturday|2030Q3|N|Y|N|2462684|2462864|2462331|2462605|N|N|N|N|N| +2462697|AAAAAAAAJODJFCAA|2030-07-14|1566|6811|523|2030|0|7|14|3|2030|523|6811|Sunday|2030Q3|N|N|N|2462684|2462864|2462332|2462606|N|N|N|N|N| +2462698|AAAAAAAAKODJFCAA|2030-07-15|1566|6811|523|2030|1|7|15|3|2030|523|6811|Monday|2030Q3|N|N|N|2462684|2462864|2462333|2462607|N|N|N|N|N| +2462699|AAAAAAAALODJFCAA|2030-07-16|1566|6812|523|2030|2|7|16|3|2030|523|6812|Tuesday|2030Q3|N|N|N|2462684|2462864|2462334|2462608|N|N|N|N|N| +2462700|AAAAAAAAMODJFCAA|2030-07-17|1566|6812|523|2030|3|7|17|3|2030|523|6812|Wednesday|2030Q3|N|N|N|2462684|2462864|2462335|2462609|N|N|N|N|N| +2462701|AAAAAAAANODJFCAA|2030-07-18|1566|6812|523|2030|4|7|18|3|2030|523|6812|Thursday|2030Q3|N|N|N|2462684|2462864|2462336|2462610|N|N|N|N|N| +2462702|AAAAAAAAOODJFCAA|2030-07-19|1566|6812|523|2030|5|7|19|3|2030|523|6812|Friday|2030Q3|N|Y|N|2462684|2462864|2462337|2462611|N|N|N|N|N| +2462703|AAAAAAAAPODJFCAA|2030-07-20|1566|6812|523|2030|6|7|20|3|2030|523|6812|Saturday|2030Q3|N|Y|N|2462684|2462864|2462338|2462612|N|N|N|N|N| +2462704|AAAAAAAAAPDJFCAA|2030-07-21|1566|6812|523|2030|0|7|21|3|2030|523|6812|Sunday|2030Q3|N|N|N|2462684|2462864|2462339|2462613|N|N|N|N|N| +2462705|AAAAAAAABPDJFCAA|2030-07-22|1566|6812|523|2030|1|7|22|3|2030|523|6812|Monday|2030Q3|N|N|N|2462684|2462864|2462340|2462614|N|N|N|N|N| +2462706|AAAAAAAACPDJFCAA|2030-07-23|1566|6813|523|2030|2|7|23|3|2030|523|6813|Tuesday|2030Q3|N|N|N|2462684|2462864|2462341|2462615|N|N|N|N|N| +2462707|AAAAAAAADPDJFCAA|2030-07-24|1566|6813|523|2030|3|7|24|3|2030|523|6813|Wednesday|2030Q3|N|N|N|2462684|2462864|2462342|2462616|N|N|N|N|N| +2462708|AAAAAAAAEPDJFCAA|2030-07-25|1566|6813|523|2030|4|7|25|3|2030|523|6813|Thursday|2030Q3|N|N|N|2462684|2462864|2462343|2462617|N|N|N|N|N| +2462709|AAAAAAAAFPDJFCAA|2030-07-26|1566|6813|523|2030|5|7|26|3|2030|523|6813|Friday|2030Q3|N|Y|N|2462684|2462864|2462344|2462618|N|N|N|N|N| +2462710|AAAAAAAAGPDJFCAA|2030-07-27|1566|6813|523|2030|6|7|27|3|2030|523|6813|Saturday|2030Q3|N|Y|N|2462684|2462864|2462345|2462619|N|N|N|N|N| +2462711|AAAAAAAAHPDJFCAA|2030-07-28|1566|6813|523|2030|0|7|28|3|2030|523|6813|Sunday|2030Q3|N|N|N|2462684|2462864|2462346|2462620|N|N|N|N|N| +2462712|AAAAAAAAIPDJFCAA|2030-07-29|1566|6813|523|2030|1|7|29|3|2030|523|6813|Monday|2030Q3|N|N|N|2462684|2462864|2462347|2462621|N|N|N|N|N| +2462713|AAAAAAAAJPDJFCAA|2030-07-30|1566|6814|523|2030|2|7|30|3|2030|523|6814|Tuesday|2030Q3|N|N|N|2462684|2462864|2462348|2462622|N|N|N|N|N| +2462714|AAAAAAAAKPDJFCAA|2030-07-31|1566|6814|523|2030|3|7|31|3|2030|523|6814|Wednesday|2030Q3|N|N|N|2462684|2462864|2462349|2462623|N|N|N|N|N| +2462715|AAAAAAAALPDJFCAA|2030-08-01|1567|6814|523|2030|4|8|1|3|2030|523|6814|Thursday|2030Q3|N|N|N|2462715|2462926|2462350|2462624|N|N|N|N|N| +2462716|AAAAAAAAMPDJFCAA|2030-08-02|1567|6814|523|2030|5|8|2|3|2030|523|6814|Friday|2030Q3|N|Y|N|2462715|2462926|2462351|2462625|N|N|N|N|N| +2462717|AAAAAAAANPDJFCAA|2030-08-03|1567|6814|523|2030|6|8|3|3|2030|523|6814|Saturday|2030Q3|N|Y|N|2462715|2462926|2462352|2462626|N|N|N|N|N| +2462718|AAAAAAAAOPDJFCAA|2030-08-04|1567|6814|523|2030|0|8|4|3|2030|523|6814|Sunday|2030Q3|N|N|N|2462715|2462926|2462353|2462627|N|N|N|N|N| +2462719|AAAAAAAAPPDJFCAA|2030-08-05|1567|6814|523|2030|1|8|5|3|2030|523|6814|Monday|2030Q3|N|N|N|2462715|2462926|2462354|2462628|N|N|N|N|N| +2462720|AAAAAAAAAAEJFCAA|2030-08-06|1567|6815|523|2030|2|8|6|3|2030|523|6815|Tuesday|2030Q3|N|N|N|2462715|2462926|2462355|2462629|N|N|N|N|N| +2462721|AAAAAAAABAEJFCAA|2030-08-07|1567|6815|523|2030|3|8|7|3|2030|523|6815|Wednesday|2030Q3|N|N|N|2462715|2462926|2462356|2462630|N|N|N|N|N| +2462722|AAAAAAAACAEJFCAA|2030-08-08|1567|6815|523|2030|4|8|8|3|2030|523|6815|Thursday|2030Q3|N|N|N|2462715|2462926|2462357|2462631|N|N|N|N|N| +2462723|AAAAAAAADAEJFCAA|2030-08-09|1567|6815|523|2030|5|8|9|3|2030|523|6815|Friday|2030Q3|N|Y|N|2462715|2462926|2462358|2462632|N|N|N|N|N| +2462724|AAAAAAAAEAEJFCAA|2030-08-10|1567|6815|523|2030|6|8|10|3|2030|523|6815|Saturday|2030Q3|N|Y|N|2462715|2462926|2462359|2462633|N|N|N|N|N| +2462725|AAAAAAAAFAEJFCAA|2030-08-11|1567|6815|523|2030|0|8|11|3|2030|523|6815|Sunday|2030Q3|N|N|N|2462715|2462926|2462360|2462634|N|N|N|N|N| +2462726|AAAAAAAAGAEJFCAA|2030-08-12|1567|6815|523|2030|1|8|12|3|2030|523|6815|Monday|2030Q3|N|N|N|2462715|2462926|2462361|2462635|N|N|N|N|N| +2462727|AAAAAAAAHAEJFCAA|2030-08-13|1567|6816|523|2030|2|8|13|3|2030|523|6816|Tuesday|2030Q3|N|N|N|2462715|2462926|2462362|2462636|N|N|N|N|N| +2462728|AAAAAAAAIAEJFCAA|2030-08-14|1567|6816|523|2030|3|8|14|3|2030|523|6816|Wednesday|2030Q3|N|N|N|2462715|2462926|2462363|2462637|N|N|N|N|N| +2462729|AAAAAAAAJAEJFCAA|2030-08-15|1567|6816|523|2030|4|8|15|3|2030|523|6816|Thursday|2030Q3|N|N|N|2462715|2462926|2462364|2462638|N|N|N|N|N| +2462730|AAAAAAAAKAEJFCAA|2030-08-16|1567|6816|523|2030|5|8|16|3|2030|523|6816|Friday|2030Q3|N|Y|N|2462715|2462926|2462365|2462639|N|N|N|N|N| +2462731|AAAAAAAALAEJFCAA|2030-08-17|1567|6816|523|2030|6|8|17|3|2030|523|6816|Saturday|2030Q3|N|Y|N|2462715|2462926|2462366|2462640|N|N|N|N|N| +2462732|AAAAAAAAMAEJFCAA|2030-08-18|1567|6816|523|2030|0|8|18|3|2030|523|6816|Sunday|2030Q3|N|N|N|2462715|2462926|2462367|2462641|N|N|N|N|N| +2462733|AAAAAAAANAEJFCAA|2030-08-19|1567|6816|523|2030|1|8|19|3|2030|523|6816|Monday|2030Q3|N|N|N|2462715|2462926|2462368|2462642|N|N|N|N|N| +2462734|AAAAAAAAOAEJFCAA|2030-08-20|1567|6817|523|2030|2|8|20|3|2030|523|6817|Tuesday|2030Q3|N|N|N|2462715|2462926|2462369|2462643|N|N|N|N|N| +2462735|AAAAAAAAPAEJFCAA|2030-08-21|1567|6817|523|2030|3|8|21|3|2030|523|6817|Wednesday|2030Q3|N|N|N|2462715|2462926|2462370|2462644|N|N|N|N|N| +2462736|AAAAAAAAABEJFCAA|2030-08-22|1567|6817|523|2030|4|8|22|3|2030|523|6817|Thursday|2030Q3|N|N|N|2462715|2462926|2462371|2462645|N|N|N|N|N| +2462737|AAAAAAAABBEJFCAA|2030-08-23|1567|6817|523|2030|5|8|23|3|2030|523|6817|Friday|2030Q3|N|Y|N|2462715|2462926|2462372|2462646|N|N|N|N|N| +2462738|AAAAAAAACBEJFCAA|2030-08-24|1567|6817|523|2030|6|8|24|3|2030|523|6817|Saturday|2030Q3|N|Y|N|2462715|2462926|2462373|2462647|N|N|N|N|N| +2462739|AAAAAAAADBEJFCAA|2030-08-25|1567|6817|523|2030|0|8|25|3|2030|523|6817|Sunday|2030Q3|N|N|N|2462715|2462926|2462374|2462648|N|N|N|N|N| +2462740|AAAAAAAAEBEJFCAA|2030-08-26|1567|6817|523|2030|1|8|26|3|2030|523|6817|Monday|2030Q3|N|N|N|2462715|2462926|2462375|2462649|N|N|N|N|N| +2462741|AAAAAAAAFBEJFCAA|2030-08-27|1567|6818|523|2030|2|8|27|3|2030|523|6818|Tuesday|2030Q3|N|N|N|2462715|2462926|2462376|2462650|N|N|N|N|N| +2462742|AAAAAAAAGBEJFCAA|2030-08-28|1567|6818|523|2030|3|8|28|3|2030|523|6818|Wednesday|2030Q3|N|N|N|2462715|2462926|2462377|2462651|N|N|N|N|N| +2462743|AAAAAAAAHBEJFCAA|2030-08-29|1567|6818|523|2030|4|8|29|3|2030|523|6818|Thursday|2030Q3|N|N|N|2462715|2462926|2462378|2462652|N|N|N|N|N| +2462744|AAAAAAAAIBEJFCAA|2030-08-30|1567|6818|523|2030|5|8|30|3|2030|523|6818|Friday|2030Q3|N|Y|N|2462715|2462926|2462379|2462653|N|N|N|N|N| +2462745|AAAAAAAAJBEJFCAA|2030-08-31|1567|6818|523|2030|6|8|31|3|2030|523|6818|Saturday|2030Q3|N|Y|N|2462715|2462926|2462380|2462654|N|N|N|N|N| +2462746|AAAAAAAAKBEJFCAA|2030-09-01|1568|6818|524|2030|0|9|1|3|2030|524|6818|Sunday|2030Q3|N|N|N|2462746|2462988|2462381|2462655|N|N|N|N|N| +2462747|AAAAAAAALBEJFCAA|2030-09-02|1568|6818|524|2030|1|9|2|3|2030|524|6818|Monday|2030Q3|N|N|N|2462746|2462988|2462382|2462656|N|N|N|N|N| +2462748|AAAAAAAAMBEJFCAA|2030-09-03|1568|6819|524|2030|2|9|3|3|2030|524|6819|Tuesday|2030Q3|N|N|N|2462746|2462988|2462383|2462657|N|N|N|N|N| +2462749|AAAAAAAANBEJFCAA|2030-09-04|1568|6819|524|2030|3|9|4|3|2030|524|6819|Wednesday|2030Q3|N|N|N|2462746|2462988|2462384|2462658|N|N|N|N|N| +2462750|AAAAAAAAOBEJFCAA|2030-09-05|1568|6819|524|2030|4|9|5|3|2030|524|6819|Thursday|2030Q3|N|N|N|2462746|2462988|2462385|2462659|N|N|N|N|N| +2462751|AAAAAAAAPBEJFCAA|2030-09-06|1568|6819|524|2030|5|9|6|3|2030|524|6819|Friday|2030Q3|N|Y|N|2462746|2462988|2462386|2462660|N|N|N|N|N| +2462752|AAAAAAAAACEJFCAA|2030-09-07|1568|6819|524|2030|6|9|7|3|2030|524|6819|Saturday|2030Q3|N|Y|N|2462746|2462988|2462387|2462661|N|N|N|N|N| +2462753|AAAAAAAABCEJFCAA|2030-09-08|1568|6819|524|2030|0|9|8|3|2030|524|6819|Sunday|2030Q3|N|N|N|2462746|2462988|2462388|2462662|N|N|N|N|N| +2462754|AAAAAAAACCEJFCAA|2030-09-09|1568|6819|524|2030|1|9|9|3|2030|524|6819|Monday|2030Q3|N|N|N|2462746|2462988|2462389|2462663|N|N|N|N|N| +2462755|AAAAAAAADCEJFCAA|2030-09-10|1568|6820|524|2030|2|9|10|3|2030|524|6820|Tuesday|2030Q3|N|N|N|2462746|2462988|2462390|2462664|N|N|N|N|N| +2462756|AAAAAAAAECEJFCAA|2030-09-11|1568|6820|524|2030|3|9|11|3|2030|524|6820|Wednesday|2030Q3|N|N|N|2462746|2462988|2462391|2462665|N|N|N|N|N| +2462757|AAAAAAAAFCEJFCAA|2030-09-12|1568|6820|524|2030|4|9|12|3|2030|524|6820|Thursday|2030Q3|N|N|N|2462746|2462988|2462392|2462666|N|N|N|N|N| +2462758|AAAAAAAAGCEJFCAA|2030-09-13|1568|6820|524|2030|5|9|13|3|2030|524|6820|Friday|2030Q3|N|Y|N|2462746|2462988|2462393|2462667|N|N|N|N|N| +2462759|AAAAAAAAHCEJFCAA|2030-09-14|1568|6820|524|2030|6|9|14|3|2030|524|6820|Saturday|2030Q3|N|Y|N|2462746|2462988|2462394|2462668|N|N|N|N|N| +2462760|AAAAAAAAICEJFCAA|2030-09-15|1568|6820|524|2030|0|9|15|3|2030|524|6820|Sunday|2030Q3|N|N|N|2462746|2462988|2462395|2462669|N|N|N|N|N| +2462761|AAAAAAAAJCEJFCAA|2030-09-16|1568|6820|524|2030|1|9|16|3|2030|524|6820|Monday|2030Q3|N|N|N|2462746|2462988|2462396|2462670|N|N|N|N|N| +2462762|AAAAAAAAKCEJFCAA|2030-09-17|1568|6821|524|2030|2|9|17|3|2030|524|6821|Tuesday|2030Q3|N|N|N|2462746|2462988|2462397|2462671|N|N|N|N|N| +2462763|AAAAAAAALCEJFCAA|2030-09-18|1568|6821|524|2030|3|9|18|3|2030|524|6821|Wednesday|2030Q3|N|N|N|2462746|2462988|2462398|2462672|N|N|N|N|N| +2462764|AAAAAAAAMCEJFCAA|2030-09-19|1568|6821|524|2030|4|9|19|3|2030|524|6821|Thursday|2030Q3|N|N|N|2462746|2462988|2462399|2462673|N|N|N|N|N| +2462765|AAAAAAAANCEJFCAA|2030-09-20|1568|6821|524|2030|5|9|20|3|2030|524|6821|Friday|2030Q3|N|Y|N|2462746|2462988|2462400|2462674|N|N|N|N|N| +2462766|AAAAAAAAOCEJFCAA|2030-09-21|1568|6821|524|2030|6|9|21|3|2030|524|6821|Saturday|2030Q3|N|Y|N|2462746|2462988|2462401|2462675|N|N|N|N|N| +2462767|AAAAAAAAPCEJFCAA|2030-09-22|1568|6821|524|2030|0|9|22|3|2030|524|6821|Sunday|2030Q3|N|N|N|2462746|2462988|2462402|2462676|N|N|N|N|N| +2462768|AAAAAAAAADEJFCAA|2030-09-23|1568|6821|524|2030|1|9|23|3|2030|524|6821|Monday|2030Q3|N|N|N|2462746|2462988|2462403|2462677|N|N|N|N|N| +2462769|AAAAAAAABDEJFCAA|2030-09-24|1568|6822|524|2030|2|9|24|3|2030|524|6822|Tuesday|2030Q3|N|N|N|2462746|2462988|2462404|2462678|N|N|N|N|N| +2462770|AAAAAAAACDEJFCAA|2030-09-25|1568|6822|524|2030|3|9|25|3|2030|524|6822|Wednesday|2030Q3|N|N|N|2462746|2462988|2462405|2462679|N|N|N|N|N| +2462771|AAAAAAAADDEJFCAA|2030-09-26|1568|6822|524|2030|4|9|26|3|2030|524|6822|Thursday|2030Q3|N|N|N|2462746|2462988|2462406|2462680|N|N|N|N|N| +2462772|AAAAAAAAEDEJFCAA|2030-09-27|1568|6822|524|2030|5|9|27|3|2030|524|6822|Friday|2030Q3|N|Y|N|2462746|2462988|2462407|2462681|N|N|N|N|N| +2462773|AAAAAAAAFDEJFCAA|2030-09-28|1568|6822|524|2030|6|9|28|3|2030|524|6822|Saturday|2030Q3|N|Y|N|2462746|2462988|2462408|2462682|N|N|N|N|N| +2462774|AAAAAAAAGDEJFCAA|2030-09-29|1568|6822|524|2030|0|9|29|3|2030|524|6822|Sunday|2030Q3|N|N|N|2462746|2462988|2462409|2462683|N|N|N|N|N| +2462775|AAAAAAAAHDEJFCAA|2030-09-30|1568|6822|524|2030|1|9|30|3|2030|524|6822|Monday|2030Q3|N|N|N|2462746|2462988|2462410|2462684|N|N|N|N|N| +2462776|AAAAAAAAIDEJFCAA|2030-10-01|1569|6823|524|2030|2|10|1|3|2030|524|6823|Tuesday|2030Q3|N|N|N|2462776|2463048|2462411|2462684|N|N|N|N|N| +2462777|AAAAAAAAJDEJFCAA|2030-10-02|1569|6823|524|2030|3|10|2|4|2030|524|6823|Wednesday|2030Q4|N|N|N|2462776|2463048|2462412|2462685|N|N|N|N|N| +2462778|AAAAAAAAKDEJFCAA|2030-10-03|1569|6823|524|2030|4|10|3|4|2030|524|6823|Thursday|2030Q4|N|N|N|2462776|2463048|2462413|2462686|N|N|N|N|N| +2462779|AAAAAAAALDEJFCAA|2030-10-04|1569|6823|524|2030|5|10|4|4|2030|524|6823|Friday|2030Q4|N|Y|N|2462776|2463048|2462414|2462687|N|N|N|N|N| +2462780|AAAAAAAAMDEJFCAA|2030-10-05|1569|6823|524|2030|6|10|5|4|2030|524|6823|Saturday|2030Q4|N|Y|N|2462776|2463048|2462415|2462688|N|N|N|N|N| +2462781|AAAAAAAANDEJFCAA|2030-10-06|1569|6823|524|2030|0|10|6|4|2030|524|6823|Sunday|2030Q4|N|N|N|2462776|2463048|2462416|2462689|N|N|N|N|N| +2462782|AAAAAAAAODEJFCAA|2030-10-07|1569|6823|524|2030|1|10|7|4|2030|524|6823|Monday|2030Q4|N|N|N|2462776|2463048|2462417|2462690|N|N|N|N|N| +2462783|AAAAAAAAPDEJFCAA|2030-10-08|1569|6824|524|2030|2|10|8|4|2030|524|6824|Tuesday|2030Q4|N|N|N|2462776|2463048|2462418|2462691|N|N|N|N|N| +2462784|AAAAAAAAAEEJFCAA|2030-10-09|1569|6824|524|2030|3|10|9|4|2030|524|6824|Wednesday|2030Q4|N|N|N|2462776|2463048|2462419|2462692|N|N|N|N|N| +2462785|AAAAAAAABEEJFCAA|2030-10-10|1569|6824|524|2030|4|10|10|4|2030|524|6824|Thursday|2030Q4|N|N|N|2462776|2463048|2462420|2462693|N|N|N|N|N| +2462786|AAAAAAAACEEJFCAA|2030-10-11|1569|6824|524|2030|5|10|11|4|2030|524|6824|Friday|2030Q4|N|Y|N|2462776|2463048|2462421|2462694|N|N|N|N|N| +2462787|AAAAAAAADEEJFCAA|2030-10-12|1569|6824|524|2030|6|10|12|4|2030|524|6824|Saturday|2030Q4|N|Y|N|2462776|2463048|2462422|2462695|N|N|N|N|N| +2462788|AAAAAAAAEEEJFCAA|2030-10-13|1569|6824|524|2030|0|10|13|4|2030|524|6824|Sunday|2030Q4|N|N|N|2462776|2463048|2462423|2462696|N|N|N|N|N| +2462789|AAAAAAAAFEEJFCAA|2030-10-14|1569|6824|524|2030|1|10|14|4|2030|524|6824|Monday|2030Q4|N|N|N|2462776|2463048|2462424|2462697|N|N|N|N|N| +2462790|AAAAAAAAGEEJFCAA|2030-10-15|1569|6825|524|2030|2|10|15|4|2030|524|6825|Tuesday|2030Q4|N|N|N|2462776|2463048|2462425|2462698|N|N|N|N|N| +2462791|AAAAAAAAHEEJFCAA|2030-10-16|1569|6825|524|2030|3|10|16|4|2030|524|6825|Wednesday|2030Q4|N|N|N|2462776|2463048|2462426|2462699|N|N|N|N|N| +2462792|AAAAAAAAIEEJFCAA|2030-10-17|1569|6825|524|2030|4|10|17|4|2030|524|6825|Thursday|2030Q4|N|N|N|2462776|2463048|2462427|2462700|N|N|N|N|N| +2462793|AAAAAAAAJEEJFCAA|2030-10-18|1569|6825|524|2030|5|10|18|4|2030|524|6825|Friday|2030Q4|N|Y|N|2462776|2463048|2462428|2462701|N|N|N|N|N| +2462794|AAAAAAAAKEEJFCAA|2030-10-19|1569|6825|524|2030|6|10|19|4|2030|524|6825|Saturday|2030Q4|N|Y|N|2462776|2463048|2462429|2462702|N|N|N|N|N| +2462795|AAAAAAAALEEJFCAA|2030-10-20|1569|6825|524|2030|0|10|20|4|2030|524|6825|Sunday|2030Q4|N|N|N|2462776|2463048|2462430|2462703|N|N|N|N|N| +2462796|AAAAAAAAMEEJFCAA|2030-10-21|1569|6825|524|2030|1|10|21|4|2030|524|6825|Monday|2030Q4|N|N|N|2462776|2463048|2462431|2462704|N|N|N|N|N| +2462797|AAAAAAAANEEJFCAA|2030-10-22|1569|6826|524|2030|2|10|22|4|2030|524|6826|Tuesday|2030Q4|N|N|N|2462776|2463048|2462432|2462705|N|N|N|N|N| +2462798|AAAAAAAAOEEJFCAA|2030-10-23|1569|6826|524|2030|3|10|23|4|2030|524|6826|Wednesday|2030Q4|N|N|N|2462776|2463048|2462433|2462706|N|N|N|N|N| +2462799|AAAAAAAAPEEJFCAA|2030-10-24|1569|6826|524|2030|4|10|24|4|2030|524|6826|Thursday|2030Q4|N|N|N|2462776|2463048|2462434|2462707|N|N|N|N|N| +2462800|AAAAAAAAAFEJFCAA|2030-10-25|1569|6826|524|2030|5|10|25|4|2030|524|6826|Friday|2030Q4|N|Y|N|2462776|2463048|2462435|2462708|N|N|N|N|N| +2462801|AAAAAAAABFEJFCAA|2030-10-26|1569|6826|524|2030|6|10|26|4|2030|524|6826|Saturday|2030Q4|N|Y|N|2462776|2463048|2462436|2462709|N|N|N|N|N| +2462802|AAAAAAAACFEJFCAA|2030-10-27|1569|6826|524|2030|0|10|27|4|2030|524|6826|Sunday|2030Q4|N|N|N|2462776|2463048|2462437|2462710|N|N|N|N|N| +2462803|AAAAAAAADFEJFCAA|2030-10-28|1569|6826|524|2030|1|10|28|4|2030|524|6826|Monday|2030Q4|N|N|N|2462776|2463048|2462438|2462711|N|N|N|N|N| +2462804|AAAAAAAAEFEJFCAA|2030-10-29|1569|6827|524|2030|2|10|29|4|2030|524|6827|Tuesday|2030Q4|N|N|N|2462776|2463048|2462439|2462712|N|N|N|N|N| +2462805|AAAAAAAAFFEJFCAA|2030-10-30|1569|6827|524|2030|3|10|30|4|2030|524|6827|Wednesday|2030Q4|N|N|N|2462776|2463048|2462440|2462713|N|N|N|N|N| +2462806|AAAAAAAAGFEJFCAA|2030-10-31|1569|6827|524|2030|4|10|31|4|2030|524|6827|Thursday|2030Q4|N|N|N|2462776|2463048|2462441|2462714|N|N|N|N|N| +2462807|AAAAAAAAHFEJFCAA|2030-11-01|1570|6827|524|2030|5|11|1|4|2030|524|6827|Friday|2030Q4|N|Y|N|2462807|2463110|2462442|2462715|N|N|N|N|N| +2462808|AAAAAAAAIFEJFCAA|2030-11-02|1570|6827|524|2030|6|11|2|4|2030|524|6827|Saturday|2030Q4|N|Y|N|2462807|2463110|2462443|2462716|N|N|N|N|N| +2462809|AAAAAAAAJFEJFCAA|2030-11-03|1570|6827|524|2030|0|11|3|4|2030|524|6827|Sunday|2030Q4|N|N|N|2462807|2463110|2462444|2462717|N|N|N|N|N| +2462810|AAAAAAAAKFEJFCAA|2030-11-04|1570|6827|524|2030|1|11|4|4|2030|524|6827|Monday|2030Q4|N|N|N|2462807|2463110|2462445|2462718|N|N|N|N|N| +2462811|AAAAAAAALFEJFCAA|2030-11-05|1570|6828|524|2030|2|11|5|4|2030|524|6828|Tuesday|2030Q4|N|N|N|2462807|2463110|2462446|2462719|N|N|N|N|N| +2462812|AAAAAAAAMFEJFCAA|2030-11-06|1570|6828|524|2030|3|11|6|4|2030|524|6828|Wednesday|2030Q4|N|N|N|2462807|2463110|2462447|2462720|N|N|N|N|N| +2462813|AAAAAAAANFEJFCAA|2030-11-07|1570|6828|524|2030|4|11|7|4|2030|524|6828|Thursday|2030Q4|N|N|N|2462807|2463110|2462448|2462721|N|N|N|N|N| +2462814|AAAAAAAAOFEJFCAA|2030-11-08|1570|6828|524|2030|5|11|8|4|2030|524|6828|Friday|2030Q4|N|Y|N|2462807|2463110|2462449|2462722|N|N|N|N|N| +2462815|AAAAAAAAPFEJFCAA|2030-11-09|1570|6828|524|2030|6|11|9|4|2030|524|6828|Saturday|2030Q4|N|Y|N|2462807|2463110|2462450|2462723|N|N|N|N|N| +2462816|AAAAAAAAAGEJFCAA|2030-11-10|1570|6828|524|2030|0|11|10|4|2030|524|6828|Sunday|2030Q4|N|N|N|2462807|2463110|2462451|2462724|N|N|N|N|N| +2462817|AAAAAAAABGEJFCAA|2030-11-11|1570|6828|524|2030|1|11|11|4|2030|524|6828|Monday|2030Q4|N|N|N|2462807|2463110|2462452|2462725|N|N|N|N|N| +2462818|AAAAAAAACGEJFCAA|2030-11-12|1570|6829|524|2030|2|11|12|4|2030|524|6829|Tuesday|2030Q4|N|N|N|2462807|2463110|2462453|2462726|N|N|N|N|N| +2462819|AAAAAAAADGEJFCAA|2030-11-13|1570|6829|524|2030|3|11|13|4|2030|524|6829|Wednesday|2030Q4|N|N|N|2462807|2463110|2462454|2462727|N|N|N|N|N| +2462820|AAAAAAAAEGEJFCAA|2030-11-14|1570|6829|524|2030|4|11|14|4|2030|524|6829|Thursday|2030Q4|N|N|N|2462807|2463110|2462455|2462728|N|N|N|N|N| +2462821|AAAAAAAAFGEJFCAA|2030-11-15|1570|6829|524|2030|5|11|15|4|2030|524|6829|Friday|2030Q4|N|Y|N|2462807|2463110|2462456|2462729|N|N|N|N|N| +2462822|AAAAAAAAGGEJFCAA|2030-11-16|1570|6829|524|2030|6|11|16|4|2030|524|6829|Saturday|2030Q4|N|Y|N|2462807|2463110|2462457|2462730|N|N|N|N|N| +2462823|AAAAAAAAHGEJFCAA|2030-11-17|1570|6829|524|2030|0|11|17|4|2030|524|6829|Sunday|2030Q4|N|N|N|2462807|2463110|2462458|2462731|N|N|N|N|N| +2462824|AAAAAAAAIGEJFCAA|2030-11-18|1570|6829|524|2030|1|11|18|4|2030|524|6829|Monday|2030Q4|N|N|N|2462807|2463110|2462459|2462732|N|N|N|N|N| +2462825|AAAAAAAAJGEJFCAA|2030-11-19|1570|6830|524|2030|2|11|19|4|2030|524|6830|Tuesday|2030Q4|N|N|N|2462807|2463110|2462460|2462733|N|N|N|N|N| +2462826|AAAAAAAAKGEJFCAA|2030-11-20|1570|6830|524|2030|3|11|20|4|2030|524|6830|Wednesday|2030Q4|N|N|N|2462807|2463110|2462461|2462734|N|N|N|N|N| +2462827|AAAAAAAALGEJFCAA|2030-11-21|1570|6830|524|2030|4|11|21|4|2030|524|6830|Thursday|2030Q4|N|N|N|2462807|2463110|2462462|2462735|N|N|N|N|N| +2462828|AAAAAAAAMGEJFCAA|2030-11-22|1570|6830|524|2030|5|11|22|4|2030|524|6830|Friday|2030Q4|N|Y|N|2462807|2463110|2462463|2462736|N|N|N|N|N| +2462829|AAAAAAAANGEJFCAA|2030-11-23|1570|6830|524|2030|6|11|23|4|2030|524|6830|Saturday|2030Q4|N|Y|N|2462807|2463110|2462464|2462737|N|N|N|N|N| +2462830|AAAAAAAAOGEJFCAA|2030-11-24|1570|6830|524|2030|0|11|24|4|2030|524|6830|Sunday|2030Q4|N|N|N|2462807|2463110|2462465|2462738|N|N|N|N|N| +2462831|AAAAAAAAPGEJFCAA|2030-11-25|1570|6830|524|2030|1|11|25|4|2030|524|6830|Monday|2030Q4|N|N|N|2462807|2463110|2462466|2462739|N|N|N|N|N| +2462832|AAAAAAAAAHEJFCAA|2030-11-26|1570|6831|524|2030|2|11|26|4|2030|524|6831|Tuesday|2030Q4|N|N|N|2462807|2463110|2462467|2462740|N|N|N|N|N| +2462833|AAAAAAAABHEJFCAA|2030-11-27|1570|6831|524|2030|3|11|27|4|2030|524|6831|Wednesday|2030Q4|N|N|N|2462807|2463110|2462468|2462741|N|N|N|N|N| +2462834|AAAAAAAACHEJFCAA|2030-11-28|1570|6831|524|2030|4|11|28|4|2030|524|6831|Thursday|2030Q4|N|N|N|2462807|2463110|2462469|2462742|N|N|N|N|N| +2462835|AAAAAAAADHEJFCAA|2030-11-29|1570|6831|524|2030|5|11|29|4|2030|524|6831|Friday|2030Q4|N|Y|N|2462807|2463110|2462470|2462743|N|N|N|N|N| +2462836|AAAAAAAAEHEJFCAA|2030-11-30|1570|6831|524|2030|6|11|30|4|2030|524|6831|Saturday|2030Q4|N|Y|N|2462807|2463110|2462471|2462744|N|N|N|N|N| +2462837|AAAAAAAAFHEJFCAA|2030-12-01|1571|6831|525|2030|0|12|1|4|2030|525|6831|Sunday|2030Q4|N|N|N|2462837|2463170|2462472|2462745|N|N|N|N|N| +2462838|AAAAAAAAGHEJFCAA|2030-12-02|1571|6831|525|2030|1|12|2|4|2030|525|6831|Monday|2030Q4|N|N|N|2462837|2463170|2462473|2462746|N|N|N|N|N| +2462839|AAAAAAAAHHEJFCAA|2030-12-03|1571|6832|525|2030|2|12|3|4|2030|525|6832|Tuesday|2030Q4|N|N|N|2462837|2463170|2462474|2462747|N|N|N|N|N| +2462840|AAAAAAAAIHEJFCAA|2030-12-04|1571|6832|525|2030|3|12|4|4|2030|525|6832|Wednesday|2030Q4|N|N|N|2462837|2463170|2462475|2462748|N|N|N|N|N| +2462841|AAAAAAAAJHEJFCAA|2030-12-05|1571|6832|525|2030|4|12|5|4|2030|525|6832|Thursday|2030Q4|N|N|N|2462837|2463170|2462476|2462749|N|N|N|N|N| +2462842|AAAAAAAAKHEJFCAA|2030-12-06|1571|6832|525|2030|5|12|6|4|2030|525|6832|Friday|2030Q4|N|Y|N|2462837|2463170|2462477|2462750|N|N|N|N|N| +2462843|AAAAAAAALHEJFCAA|2030-12-07|1571|6832|525|2030|6|12|7|4|2030|525|6832|Saturday|2030Q4|N|Y|N|2462837|2463170|2462478|2462751|N|N|N|N|N| +2462844|AAAAAAAAMHEJFCAA|2030-12-08|1571|6832|525|2030|0|12|8|4|2030|525|6832|Sunday|2030Q4|N|N|N|2462837|2463170|2462479|2462752|N|N|N|N|N| +2462845|AAAAAAAANHEJFCAA|2030-12-09|1571|6832|525|2030|1|12|9|4|2030|525|6832|Monday|2030Q4|N|N|N|2462837|2463170|2462480|2462753|N|N|N|N|N| +2462846|AAAAAAAAOHEJFCAA|2030-12-10|1571|6833|525|2030|2|12|10|4|2030|525|6833|Tuesday|2030Q4|N|N|N|2462837|2463170|2462481|2462754|N|N|N|N|N| +2462847|AAAAAAAAPHEJFCAA|2030-12-11|1571|6833|525|2030|3|12|11|4|2030|525|6833|Wednesday|2030Q4|N|N|N|2462837|2463170|2462482|2462755|N|N|N|N|N| +2462848|AAAAAAAAAIEJFCAA|2030-12-12|1571|6833|525|2030|4|12|12|4|2030|525|6833|Thursday|2030Q4|N|N|N|2462837|2463170|2462483|2462756|N|N|N|N|N| +2462849|AAAAAAAABIEJFCAA|2030-12-13|1571|6833|525|2030|5|12|13|4|2030|525|6833|Friday|2030Q4|N|Y|N|2462837|2463170|2462484|2462757|N|N|N|N|N| +2462850|AAAAAAAACIEJFCAA|2030-12-14|1571|6833|525|2030|6|12|14|4|2030|525|6833|Saturday|2030Q4|N|Y|N|2462837|2463170|2462485|2462758|N|N|N|N|N| +2462851|AAAAAAAADIEJFCAA|2030-12-15|1571|6833|525|2030|0|12|15|4|2030|525|6833|Sunday|2030Q4|N|N|N|2462837|2463170|2462486|2462759|N|N|N|N|N| +2462852|AAAAAAAAEIEJFCAA|2030-12-16|1571|6833|525|2030|1|12|16|4|2030|525|6833|Monday|2030Q4|N|N|N|2462837|2463170|2462487|2462760|N|N|N|N|N| +2462853|AAAAAAAAFIEJFCAA|2030-12-17|1571|6834|525|2030|2|12|17|4|2030|525|6834|Tuesday|2030Q4|N|N|N|2462837|2463170|2462488|2462761|N|N|N|N|N| +2462854|AAAAAAAAGIEJFCAA|2030-12-18|1571|6834|525|2030|3|12|18|4|2030|525|6834|Wednesday|2030Q4|N|N|N|2462837|2463170|2462489|2462762|N|N|N|N|N| +2462855|AAAAAAAAHIEJFCAA|2030-12-19|1571|6834|525|2030|4|12|19|4|2030|525|6834|Thursday|2030Q4|N|N|N|2462837|2463170|2462490|2462763|N|N|N|N|N| +2462856|AAAAAAAAIIEJFCAA|2030-12-20|1571|6834|525|2030|5|12|20|4|2030|525|6834|Friday|2030Q4|N|Y|N|2462837|2463170|2462491|2462764|N|N|N|N|N| +2462857|AAAAAAAAJIEJFCAA|2030-12-21|1571|6834|525|2030|6|12|21|4|2030|525|6834|Saturday|2030Q4|N|Y|N|2462837|2463170|2462492|2462765|N|N|N|N|N| +2462858|AAAAAAAAKIEJFCAA|2030-12-22|1571|6834|525|2030|0|12|22|4|2030|525|6834|Sunday|2030Q4|N|N|N|2462837|2463170|2462493|2462766|N|N|N|N|N| +2462859|AAAAAAAALIEJFCAA|2030-12-23|1571|6834|525|2030|1|12|23|4|2030|525|6834|Monday|2030Q4|N|N|N|2462837|2463170|2462494|2462767|N|N|N|N|N| +2462860|AAAAAAAAMIEJFCAA|2030-12-24|1571|6835|525|2030|2|12|24|4|2030|525|6835|Tuesday|2030Q4|N|N|N|2462837|2463170|2462495|2462768|N|N|N|N|N| +2462861|AAAAAAAANIEJFCAA|2030-12-25|1571|6835|525|2030|3|12|25|4|2030|525|6835|Wednesday|2030Q4|N|N|N|2462837|2463170|2462496|2462769|N|N|N|N|N| +2462862|AAAAAAAAOIEJFCAA|2030-12-26|1571|6835|525|2030|4|12|26|4|2030|525|6835|Thursday|2030Q4|Y|N|N|2462837|2463170|2462497|2462770|N|N|N|N|N| +2462863|AAAAAAAAPIEJFCAA|2030-12-27|1571|6835|525|2030|5|12|27|4|2030|525|6835|Friday|2030Q4|N|Y|Y|2462837|2463170|2462498|2462771|N|N|N|N|N| +2462864|AAAAAAAAAJEJFCAA|2030-12-28|1571|6835|525|2030|6|12|28|4|2030|525|6835|Saturday|2030Q4|N|Y|N|2462837|2463170|2462499|2462772|N|N|N|N|N| +2462865|AAAAAAAABJEJFCAA|2030-12-29|1571|6835|525|2030|0|12|29|4|2030|525|6835|Sunday|2030Q4|N|N|N|2462837|2463170|2462500|2462773|N|N|N|N|N| +2462866|AAAAAAAACJEJFCAA|2030-12-30|1571|6835|525|2030|1|12|30|4|2030|525|6835|Monday|2030Q4|N|N|N|2462837|2463170|2462501|2462774|N|N|N|N|N| +2462867|AAAAAAAADJEJFCAA|2030-12-31|1571|6836|525|2030|2|12|31|4|2030|525|6836|Tuesday|2030Q4|N|N|N|2462837|2463170|2462502|2462775|N|N|N|N|N| +2462868|AAAAAAAAEJEJFCAA|2031-01-01|1572|6836|525|2031|3|1|1|1|2031|525|6836|Wednesday|2031Q1|Y|N|N|2462868|2462867|2462503|2462776|N|N|N|N|N| +2462869|AAAAAAAAFJEJFCAA|2031-01-02|1572|6836|525|2031|4|1|2|1|2031|525|6836|Thursday|2031Q1|N|N|Y|2462868|2462867|2462504|2462777|N|N|N|N|N| +2462870|AAAAAAAAGJEJFCAA|2031-01-03|1572|6836|525|2031|5|1|3|1|2031|525|6836|Friday|2031Q1|N|Y|N|2462868|2462867|2462505|2462778|N|N|N|N|N| +2462871|AAAAAAAAHJEJFCAA|2031-01-04|1572|6836|525|2031|6|1|4|1|2031|525|6836|Saturday|2031Q1|N|Y|N|2462868|2462867|2462506|2462779|N|N|N|N|N| +2462872|AAAAAAAAIJEJFCAA|2031-01-05|1572|6836|525|2031|0|1|5|1|2031|525|6836|Sunday|2031Q1|N|N|N|2462868|2462867|2462507|2462780|N|N|N|N|N| +2462873|AAAAAAAAJJEJFCAA|2031-01-06|1572|6836|525|2031|1|1|6|1|2031|525|6836|Monday|2031Q1|N|N|N|2462868|2462867|2462508|2462781|N|N|N|N|N| +2462874|AAAAAAAAKJEJFCAA|2031-01-07|1572|6837|525|2031|2|1|7|1|2031|525|6837|Tuesday|2031Q1|N|N|N|2462868|2462867|2462509|2462782|N|N|N|N|N| +2462875|AAAAAAAALJEJFCAA|2031-01-08|1572|6837|525|2031|3|1|8|1|2031|525|6837|Wednesday|2031Q1|N|N|N|2462868|2462867|2462510|2462783|N|N|N|N|N| +2462876|AAAAAAAAMJEJFCAA|2031-01-09|1572|6837|525|2031|4|1|9|1|2031|525|6837|Thursday|2031Q1|N|N|N|2462868|2462867|2462511|2462784|N|N|N|N|N| +2462877|AAAAAAAANJEJFCAA|2031-01-10|1572|6837|525|2031|5|1|10|1|2031|525|6837|Friday|2031Q1|N|Y|N|2462868|2462867|2462512|2462785|N|N|N|N|N| +2462878|AAAAAAAAOJEJFCAA|2031-01-11|1572|6837|525|2031|6|1|11|1|2031|525|6837|Saturday|2031Q1|N|Y|N|2462868|2462867|2462513|2462786|N|N|N|N|N| +2462879|AAAAAAAAPJEJFCAA|2031-01-12|1572|6837|525|2031|0|1|12|1|2031|525|6837|Sunday|2031Q1|N|N|N|2462868|2462867|2462514|2462787|N|N|N|N|N| +2462880|AAAAAAAAAKEJFCAA|2031-01-13|1572|6837|525|2031|1|1|13|1|2031|525|6837|Monday|2031Q1|N|N|N|2462868|2462867|2462515|2462788|N|N|N|N|N| +2462881|AAAAAAAABKEJFCAA|2031-01-14|1572|6838|525|2031|2|1|14|1|2031|525|6838|Tuesday|2031Q1|N|N|N|2462868|2462867|2462516|2462789|N|N|N|N|N| +2462882|AAAAAAAACKEJFCAA|2031-01-15|1572|6838|525|2031|3|1|15|1|2031|525|6838|Wednesday|2031Q1|N|N|N|2462868|2462867|2462517|2462790|N|N|N|N|N| +2462883|AAAAAAAADKEJFCAA|2031-01-16|1572|6838|525|2031|4|1|16|1|2031|525|6838|Thursday|2031Q1|N|N|N|2462868|2462867|2462518|2462791|N|N|N|N|N| +2462884|AAAAAAAAEKEJFCAA|2031-01-17|1572|6838|525|2031|5|1|17|1|2031|525|6838|Friday|2031Q1|N|Y|N|2462868|2462867|2462519|2462792|N|N|N|N|N| +2462885|AAAAAAAAFKEJFCAA|2031-01-18|1572|6838|525|2031|6|1|18|1|2031|525|6838|Saturday|2031Q1|N|Y|N|2462868|2462867|2462520|2462793|N|N|N|N|N| +2462886|AAAAAAAAGKEJFCAA|2031-01-19|1572|6838|525|2031|0|1|19|1|2031|525|6838|Sunday|2031Q1|N|N|N|2462868|2462867|2462521|2462794|N|N|N|N|N| +2462887|AAAAAAAAHKEJFCAA|2031-01-20|1572|6838|525|2031|1|1|20|1|2031|525|6838|Monday|2031Q1|N|N|N|2462868|2462867|2462522|2462795|N|N|N|N|N| +2462888|AAAAAAAAIKEJFCAA|2031-01-21|1572|6839|525|2031|2|1|21|1|2031|525|6839|Tuesday|2031Q1|N|N|N|2462868|2462867|2462523|2462796|N|N|N|N|N| +2462889|AAAAAAAAJKEJFCAA|2031-01-22|1572|6839|525|2031|3|1|22|1|2031|525|6839|Wednesday|2031Q1|N|N|N|2462868|2462867|2462524|2462797|N|N|N|N|N| +2462890|AAAAAAAAKKEJFCAA|2031-01-23|1572|6839|525|2031|4|1|23|1|2031|525|6839|Thursday|2031Q1|N|N|N|2462868|2462867|2462525|2462798|N|N|N|N|N| +2462891|AAAAAAAALKEJFCAA|2031-01-24|1572|6839|525|2031|5|1|24|1|2031|525|6839|Friday|2031Q1|N|Y|N|2462868|2462867|2462526|2462799|N|N|N|N|N| +2462892|AAAAAAAAMKEJFCAA|2031-01-25|1572|6839|525|2031|6|1|25|1|2031|525|6839|Saturday|2031Q1|N|Y|N|2462868|2462867|2462527|2462800|N|N|N|N|N| +2462893|AAAAAAAANKEJFCAA|2031-01-26|1572|6839|525|2031|0|1|26|1|2031|525|6839|Sunday|2031Q1|N|N|N|2462868|2462867|2462528|2462801|N|N|N|N|N| +2462894|AAAAAAAAOKEJFCAA|2031-01-27|1572|6839|525|2031|1|1|27|1|2031|525|6839|Monday|2031Q1|N|N|N|2462868|2462867|2462529|2462802|N|N|N|N|N| +2462895|AAAAAAAAPKEJFCAA|2031-01-28|1572|6840|525|2031|2|1|28|1|2031|525|6840|Tuesday|2031Q1|N|N|N|2462868|2462867|2462530|2462803|N|N|N|N|N| +2462896|AAAAAAAAALEJFCAA|2031-01-29|1572|6840|525|2031|3|1|29|1|2031|525|6840|Wednesday|2031Q1|N|N|N|2462868|2462867|2462531|2462804|N|N|N|N|N| +2462897|AAAAAAAABLEJFCAA|2031-01-30|1572|6840|525|2031|4|1|30|1|2031|525|6840|Thursday|2031Q1|N|N|N|2462868|2462867|2462532|2462805|N|N|N|N|N| +2462898|AAAAAAAACLEJFCAA|2031-01-31|1572|6840|525|2031|5|1|31|1|2031|525|6840|Friday|2031Q1|N|Y|N|2462868|2462867|2462533|2462806|N|N|N|N|N| +2462899|AAAAAAAADLEJFCAA|2031-02-01|1573|6840|525|2031|6|2|1|1|2031|525|6840|Saturday|2031Q1|N|Y|N|2462899|2462929|2462534|2462807|N|N|N|N|N| +2462900|AAAAAAAAELEJFCAA|2031-02-02|1573|6840|525|2031|0|2|2|1|2031|525|6840|Sunday|2031Q1|N|N|N|2462899|2462929|2462535|2462808|N|N|N|N|N| +2462901|AAAAAAAAFLEJFCAA|2031-02-03|1573|6840|525|2031|1|2|3|1|2031|525|6840|Monday|2031Q1|N|N|N|2462899|2462929|2462536|2462809|N|N|N|N|N| +2462902|AAAAAAAAGLEJFCAA|2031-02-04|1573|6841|525|2031|2|2|4|1|2031|525|6841|Tuesday|2031Q1|N|N|N|2462899|2462929|2462537|2462810|N|N|N|N|N| +2462903|AAAAAAAAHLEJFCAA|2031-02-05|1573|6841|525|2031|3|2|5|1|2031|525|6841|Wednesday|2031Q1|N|N|N|2462899|2462929|2462538|2462811|N|N|N|N|N| +2462904|AAAAAAAAILEJFCAA|2031-02-06|1573|6841|525|2031|4|2|6|1|2031|525|6841|Thursday|2031Q1|N|N|N|2462899|2462929|2462539|2462812|N|N|N|N|N| +2462905|AAAAAAAAJLEJFCAA|2031-02-07|1573|6841|525|2031|5|2|7|1|2031|525|6841|Friday|2031Q1|N|Y|N|2462899|2462929|2462540|2462813|N|N|N|N|N| +2462906|AAAAAAAAKLEJFCAA|2031-02-08|1573|6841|525|2031|6|2|8|1|2031|525|6841|Saturday|2031Q1|N|Y|N|2462899|2462929|2462541|2462814|N|N|N|N|N| +2462907|AAAAAAAALLEJFCAA|2031-02-09|1573|6841|525|2031|0|2|9|1|2031|525|6841|Sunday|2031Q1|N|N|N|2462899|2462929|2462542|2462815|N|N|N|N|N| +2462908|AAAAAAAAMLEJFCAA|2031-02-10|1573|6841|525|2031|1|2|10|1|2031|525|6841|Monday|2031Q1|N|N|N|2462899|2462929|2462543|2462816|N|N|N|N|N| +2462909|AAAAAAAANLEJFCAA|2031-02-11|1573|6842|525|2031|2|2|11|1|2031|525|6842|Tuesday|2031Q1|N|N|N|2462899|2462929|2462544|2462817|N|N|N|N|N| +2462910|AAAAAAAAOLEJFCAA|2031-02-12|1573|6842|525|2031|3|2|12|1|2031|525|6842|Wednesday|2031Q1|N|N|N|2462899|2462929|2462545|2462818|N|N|N|N|N| +2462911|AAAAAAAAPLEJFCAA|2031-02-13|1573|6842|525|2031|4|2|13|1|2031|525|6842|Thursday|2031Q1|N|N|N|2462899|2462929|2462546|2462819|N|N|N|N|N| +2462912|AAAAAAAAAMEJFCAA|2031-02-14|1573|6842|525|2031|5|2|14|1|2031|525|6842|Friday|2031Q1|N|Y|N|2462899|2462929|2462547|2462820|N|N|N|N|N| +2462913|AAAAAAAABMEJFCAA|2031-02-15|1573|6842|525|2031|6|2|15|1|2031|525|6842|Saturday|2031Q1|N|Y|N|2462899|2462929|2462548|2462821|N|N|N|N|N| +2462914|AAAAAAAACMEJFCAA|2031-02-16|1573|6842|525|2031|0|2|16|1|2031|525|6842|Sunday|2031Q1|N|N|N|2462899|2462929|2462549|2462822|N|N|N|N|N| +2462915|AAAAAAAADMEJFCAA|2031-02-17|1573|6842|525|2031|1|2|17|1|2031|525|6842|Monday|2031Q1|N|N|N|2462899|2462929|2462550|2462823|N|N|N|N|N| +2462916|AAAAAAAAEMEJFCAA|2031-02-18|1573|6843|525|2031|2|2|18|1|2031|525|6843|Tuesday|2031Q1|N|N|N|2462899|2462929|2462551|2462824|N|N|N|N|N| +2462917|AAAAAAAAFMEJFCAA|2031-02-19|1573|6843|525|2031|3|2|19|1|2031|525|6843|Wednesday|2031Q1|N|N|N|2462899|2462929|2462552|2462825|N|N|N|N|N| +2462918|AAAAAAAAGMEJFCAA|2031-02-20|1573|6843|525|2031|4|2|20|1|2031|525|6843|Thursday|2031Q1|N|N|N|2462899|2462929|2462553|2462826|N|N|N|N|N| +2462919|AAAAAAAAHMEJFCAA|2031-02-21|1573|6843|525|2031|5|2|21|1|2031|525|6843|Friday|2031Q1|N|Y|N|2462899|2462929|2462554|2462827|N|N|N|N|N| +2462920|AAAAAAAAIMEJFCAA|2031-02-22|1573|6843|525|2031|6|2|22|1|2031|525|6843|Saturday|2031Q1|N|Y|N|2462899|2462929|2462555|2462828|N|N|N|N|N| +2462921|AAAAAAAAJMEJFCAA|2031-02-23|1573|6843|525|2031|0|2|23|1|2031|525|6843|Sunday|2031Q1|N|N|N|2462899|2462929|2462556|2462829|N|N|N|N|N| +2462922|AAAAAAAAKMEJFCAA|2031-02-24|1573|6843|525|2031|1|2|24|1|2031|525|6843|Monday|2031Q1|N|N|N|2462899|2462929|2462557|2462830|N|N|N|N|N| +2462923|AAAAAAAALMEJFCAA|2031-02-25|1573|6844|525|2031|2|2|25|1|2031|525|6844|Tuesday|2031Q1|N|N|N|2462899|2462929|2462558|2462831|N|N|N|N|N| +2462924|AAAAAAAAMMEJFCAA|2031-02-26|1573|6844|525|2031|3|2|26|1|2031|525|6844|Wednesday|2031Q1|N|N|N|2462899|2462929|2462559|2462832|N|N|N|N|N| +2462925|AAAAAAAANMEJFCAA|2031-02-27|1573|6844|525|2031|4|2|27|1|2031|525|6844|Thursday|2031Q1|N|N|N|2462899|2462929|2462560|2462833|N|N|N|N|N| +2462926|AAAAAAAAOMEJFCAA|2031-02-28|1573|6844|525|2031|5|2|28|1|2031|525|6844|Friday|2031Q1|N|Y|N|2462899|2462929|2462561|2462834|N|N|N|N|N| +2462927|AAAAAAAAPMEJFCAA|2031-03-01|1574|6844|526|2031|6|3|1|1|2031|526|6844|Saturday|2031Q1|N|Y|N|2462927|2462985|2462562|2462835|N|N|N|N|N| +2462928|AAAAAAAAANEJFCAA|2031-03-02|1574|6844|526|2031|0|3|2|1|2031|526|6844|Sunday|2031Q1|N|N|N|2462927|2462985|2462563|2462836|N|N|N|N|N| +2462929|AAAAAAAABNEJFCAA|2031-03-03|1574|6844|526|2031|1|3|3|1|2031|526|6844|Monday|2031Q1|N|N|N|2462927|2462985|2462564|2462837|N|N|N|N|N| +2462930|AAAAAAAACNEJFCAA|2031-03-04|1574|6845|526|2031|2|3|4|1|2031|526|6845|Tuesday|2031Q1|N|N|N|2462927|2462985|2462565|2462838|N|N|N|N|N| +2462931|AAAAAAAADNEJFCAA|2031-03-05|1574|6845|526|2031|3|3|5|1|2031|526|6845|Wednesday|2031Q1|N|N|N|2462927|2462985|2462566|2462839|N|N|N|N|N| +2462932|AAAAAAAAENEJFCAA|2031-03-06|1574|6845|526|2031|4|3|6|1|2031|526|6845|Thursday|2031Q1|N|N|N|2462927|2462985|2462567|2462840|N|N|N|N|N| +2462933|AAAAAAAAFNEJFCAA|2031-03-07|1574|6845|526|2031|5|3|7|1|2031|526|6845|Friday|2031Q1|N|Y|N|2462927|2462985|2462568|2462841|N|N|N|N|N| +2462934|AAAAAAAAGNEJFCAA|2031-03-08|1574|6845|526|2031|6|3|8|1|2031|526|6845|Saturday|2031Q1|N|Y|N|2462927|2462985|2462569|2462842|N|N|N|N|N| +2462935|AAAAAAAAHNEJFCAA|2031-03-09|1574|6845|526|2031|0|3|9|1|2031|526|6845|Sunday|2031Q1|N|N|N|2462927|2462985|2462570|2462843|N|N|N|N|N| +2462936|AAAAAAAAINEJFCAA|2031-03-10|1574|6845|526|2031|1|3|10|1|2031|526|6845|Monday|2031Q1|N|N|N|2462927|2462985|2462571|2462844|N|N|N|N|N| +2462937|AAAAAAAAJNEJFCAA|2031-03-11|1574|6846|526|2031|2|3|11|1|2031|526|6846|Tuesday|2031Q1|N|N|N|2462927|2462985|2462572|2462845|N|N|N|N|N| +2462938|AAAAAAAAKNEJFCAA|2031-03-12|1574|6846|526|2031|3|3|12|1|2031|526|6846|Wednesday|2031Q1|N|N|N|2462927|2462985|2462573|2462846|N|N|N|N|N| +2462939|AAAAAAAALNEJFCAA|2031-03-13|1574|6846|526|2031|4|3|13|1|2031|526|6846|Thursday|2031Q1|N|N|N|2462927|2462985|2462574|2462847|N|N|N|N|N| +2462940|AAAAAAAAMNEJFCAA|2031-03-14|1574|6846|526|2031|5|3|14|1|2031|526|6846|Friday|2031Q1|N|Y|N|2462927|2462985|2462575|2462848|N|N|N|N|N| +2462941|AAAAAAAANNEJFCAA|2031-03-15|1574|6846|526|2031|6|3|15|1|2031|526|6846|Saturday|2031Q1|N|Y|N|2462927|2462985|2462576|2462849|N|N|N|N|N| +2462942|AAAAAAAAONEJFCAA|2031-03-16|1574|6846|526|2031|0|3|16|1|2031|526|6846|Sunday|2031Q1|N|N|N|2462927|2462985|2462577|2462850|N|N|N|N|N| +2462943|AAAAAAAAPNEJFCAA|2031-03-17|1574|6846|526|2031|1|3|17|1|2031|526|6846|Monday|2031Q1|N|N|N|2462927|2462985|2462578|2462851|N|N|N|N|N| +2462944|AAAAAAAAAOEJFCAA|2031-03-18|1574|6847|526|2031|2|3|18|1|2031|526|6847|Tuesday|2031Q1|N|N|N|2462927|2462985|2462579|2462852|N|N|N|N|N| +2462945|AAAAAAAABOEJFCAA|2031-03-19|1574|6847|526|2031|3|3|19|1|2031|526|6847|Wednesday|2031Q1|N|N|N|2462927|2462985|2462580|2462853|N|N|N|N|N| +2462946|AAAAAAAACOEJFCAA|2031-03-20|1574|6847|526|2031|4|3|20|1|2031|526|6847|Thursday|2031Q1|N|N|N|2462927|2462985|2462581|2462854|N|N|N|N|N| +2462947|AAAAAAAADOEJFCAA|2031-03-21|1574|6847|526|2031|5|3|21|1|2031|526|6847|Friday|2031Q1|N|Y|N|2462927|2462985|2462582|2462855|N|N|N|N|N| +2462948|AAAAAAAAEOEJFCAA|2031-03-22|1574|6847|526|2031|6|3|22|1|2031|526|6847|Saturday|2031Q1|N|Y|N|2462927|2462985|2462583|2462856|N|N|N|N|N| +2462949|AAAAAAAAFOEJFCAA|2031-03-23|1574|6847|526|2031|0|3|23|1|2031|526|6847|Sunday|2031Q1|N|N|N|2462927|2462985|2462584|2462857|N|N|N|N|N| +2462950|AAAAAAAAGOEJFCAA|2031-03-24|1574|6847|526|2031|1|3|24|1|2031|526|6847|Monday|2031Q1|N|N|N|2462927|2462985|2462585|2462858|N|N|N|N|N| +2462951|AAAAAAAAHOEJFCAA|2031-03-25|1574|6848|526|2031|2|3|25|1|2031|526|6848|Tuesday|2031Q1|N|N|N|2462927|2462985|2462586|2462859|N|N|N|N|N| +2462952|AAAAAAAAIOEJFCAA|2031-03-26|1574|6848|526|2031|3|3|26|1|2031|526|6848|Wednesday|2031Q1|N|N|N|2462927|2462985|2462587|2462860|N|N|N|N|N| +2462953|AAAAAAAAJOEJFCAA|2031-03-27|1574|6848|526|2031|4|3|27|1|2031|526|6848|Thursday|2031Q1|N|N|N|2462927|2462985|2462588|2462861|N|N|N|N|N| +2462954|AAAAAAAAKOEJFCAA|2031-03-28|1574|6848|526|2031|5|3|28|1|2031|526|6848|Friday|2031Q1|N|Y|N|2462927|2462985|2462589|2462862|N|N|N|N|N| +2462955|AAAAAAAALOEJFCAA|2031-03-29|1574|6848|526|2031|6|3|29|1|2031|526|6848|Saturday|2031Q1|N|Y|N|2462927|2462985|2462590|2462863|N|N|N|N|N| +2462956|AAAAAAAAMOEJFCAA|2031-03-30|1574|6848|526|2031|0|3|30|1|2031|526|6848|Sunday|2031Q1|N|N|N|2462927|2462985|2462591|2462864|N|N|N|N|N| +2462957|AAAAAAAANOEJFCAA|2031-03-31|1574|6848|526|2031|1|3|31|1|2031|526|6848|Monday|2031Q1|N|N|N|2462927|2462985|2462592|2462865|N|N|N|N|N| +2462958|AAAAAAAAOOEJFCAA|2031-04-01|1575|6849|526|2031|2|4|1|1|2031|526|6849|Tuesday|2031Q1|N|N|N|2462958|2463047|2462593|2462868|N|N|N|N|N| +2462959|AAAAAAAAPOEJFCAA|2031-04-02|1575|6849|526|2031|3|4|2|2|2031|526|6849|Wednesday|2031Q2|N|N|N|2462958|2463047|2462594|2462869|N|N|N|N|N| +2462960|AAAAAAAAAPEJFCAA|2031-04-03|1575|6849|526|2031|4|4|3|2|2031|526|6849|Thursday|2031Q2|N|N|N|2462958|2463047|2462595|2462870|N|N|N|N|N| +2462961|AAAAAAAABPEJFCAA|2031-04-04|1575|6849|526|2031|5|4|4|2|2031|526|6849|Friday|2031Q2|N|Y|N|2462958|2463047|2462596|2462871|N|N|N|N|N| +2462962|AAAAAAAACPEJFCAA|2031-04-05|1575|6849|526|2031|6|4|5|2|2031|526|6849|Saturday|2031Q2|N|Y|N|2462958|2463047|2462597|2462872|N|N|N|N|N| +2462963|AAAAAAAADPEJFCAA|2031-04-06|1575|6849|526|2031|0|4|6|2|2031|526|6849|Sunday|2031Q2|N|N|N|2462958|2463047|2462598|2462873|N|N|N|N|N| +2462964|AAAAAAAAEPEJFCAA|2031-04-07|1575|6849|526|2031|1|4|7|2|2031|526|6849|Monday|2031Q2|N|N|N|2462958|2463047|2462599|2462874|N|N|N|N|N| +2462965|AAAAAAAAFPEJFCAA|2031-04-08|1575|6850|526|2031|2|4|8|2|2031|526|6850|Tuesday|2031Q2|N|N|N|2462958|2463047|2462600|2462875|N|N|N|N|N| +2462966|AAAAAAAAGPEJFCAA|2031-04-09|1575|6850|526|2031|3|4|9|2|2031|526|6850|Wednesday|2031Q2|N|N|N|2462958|2463047|2462601|2462876|N|N|N|N|N| +2462967|AAAAAAAAHPEJFCAA|2031-04-10|1575|6850|526|2031|4|4|10|2|2031|526|6850|Thursday|2031Q2|N|N|N|2462958|2463047|2462602|2462877|N|N|N|N|N| +2462968|AAAAAAAAIPEJFCAA|2031-04-11|1575|6850|526|2031|5|4|11|2|2031|526|6850|Friday|2031Q2|N|Y|N|2462958|2463047|2462603|2462878|N|N|N|N|N| +2462969|AAAAAAAAJPEJFCAA|2031-04-12|1575|6850|526|2031|6|4|12|2|2031|526|6850|Saturday|2031Q2|N|Y|N|2462958|2463047|2462604|2462879|N|N|N|N|N| +2462970|AAAAAAAAKPEJFCAA|2031-04-13|1575|6850|526|2031|0|4|13|2|2031|526|6850|Sunday|2031Q2|N|N|N|2462958|2463047|2462605|2462880|N|N|N|N|N| +2462971|AAAAAAAALPEJFCAA|2031-04-14|1575|6850|526|2031|1|4|14|2|2031|526|6850|Monday|2031Q2|N|N|N|2462958|2463047|2462606|2462881|N|N|N|N|N| +2462972|AAAAAAAAMPEJFCAA|2031-04-15|1575|6851|526|2031|2|4|15|2|2031|526|6851|Tuesday|2031Q2|N|N|N|2462958|2463047|2462607|2462882|N|N|N|N|N| +2462973|AAAAAAAANPEJFCAA|2031-04-16|1575|6851|526|2031|3|4|16|2|2031|526|6851|Wednesday|2031Q2|N|N|N|2462958|2463047|2462608|2462883|N|N|N|N|N| +2462974|AAAAAAAAOPEJFCAA|2031-04-17|1575|6851|526|2031|4|4|17|2|2031|526|6851|Thursday|2031Q2|N|N|N|2462958|2463047|2462609|2462884|N|N|N|N|N| +2462975|AAAAAAAAPPEJFCAA|2031-04-18|1575|6851|526|2031|5|4|18|2|2031|526|6851|Friday|2031Q2|N|Y|N|2462958|2463047|2462610|2462885|N|N|N|N|N| +2462976|AAAAAAAAAAFJFCAA|2031-04-19|1575|6851|526|2031|6|4|19|2|2031|526|6851|Saturday|2031Q2|N|Y|N|2462958|2463047|2462611|2462886|N|N|N|N|N| +2462977|AAAAAAAABAFJFCAA|2031-04-20|1575|6851|526|2031|0|4|20|2|2031|526|6851|Sunday|2031Q2|N|N|N|2462958|2463047|2462612|2462887|N|N|N|N|N| +2462978|AAAAAAAACAFJFCAA|2031-04-21|1575|6851|526|2031|1|4|21|2|2031|526|6851|Monday|2031Q2|N|N|N|2462958|2463047|2462613|2462888|N|N|N|N|N| +2462979|AAAAAAAADAFJFCAA|2031-04-22|1575|6852|526|2031|2|4|22|2|2031|526|6852|Tuesday|2031Q2|N|N|N|2462958|2463047|2462614|2462889|N|N|N|N|N| +2462980|AAAAAAAAEAFJFCAA|2031-04-23|1575|6852|526|2031|3|4|23|2|2031|526|6852|Wednesday|2031Q2|N|N|N|2462958|2463047|2462615|2462890|N|N|N|N|N| +2462981|AAAAAAAAFAFJFCAA|2031-04-24|1575|6852|526|2031|4|4|24|2|2031|526|6852|Thursday|2031Q2|N|N|N|2462958|2463047|2462616|2462891|N|N|N|N|N| +2462982|AAAAAAAAGAFJFCAA|2031-04-25|1575|6852|526|2031|5|4|25|2|2031|526|6852|Friday|2031Q2|N|Y|N|2462958|2463047|2462617|2462892|N|N|N|N|N| +2462983|AAAAAAAAHAFJFCAA|2031-04-26|1575|6852|526|2031|6|4|26|2|2031|526|6852|Saturday|2031Q2|N|Y|N|2462958|2463047|2462618|2462893|N|N|N|N|N| +2462984|AAAAAAAAIAFJFCAA|2031-04-27|1575|6852|526|2031|0|4|27|2|2031|526|6852|Sunday|2031Q2|N|N|N|2462958|2463047|2462619|2462894|N|N|N|N|N| +2462985|AAAAAAAAJAFJFCAA|2031-04-28|1575|6852|526|2031|1|4|28|2|2031|526|6852|Monday|2031Q2|N|N|N|2462958|2463047|2462620|2462895|N|N|N|N|N| +2462986|AAAAAAAAKAFJFCAA|2031-04-29|1575|6853|526|2031|2|4|29|2|2031|526|6853|Tuesday|2031Q2|N|N|N|2462958|2463047|2462621|2462896|N|N|N|N|N| +2462987|AAAAAAAALAFJFCAA|2031-04-30|1575|6853|526|2031|3|4|30|2|2031|526|6853|Wednesday|2031Q2|N|N|N|2462958|2463047|2462622|2462897|N|N|N|N|N| +2462988|AAAAAAAAMAFJFCAA|2031-05-01|1576|6853|526|2031|4|5|1|2|2031|526|6853|Thursday|2031Q2|N|N|N|2462988|2463107|2462623|2462898|N|N|N|N|N| +2462989|AAAAAAAANAFJFCAA|2031-05-02|1576|6853|526|2031|5|5|2|2|2031|526|6853|Friday|2031Q2|N|Y|N|2462988|2463107|2462624|2462899|N|N|N|N|N| +2462990|AAAAAAAAOAFJFCAA|2031-05-03|1576|6853|526|2031|6|5|3|2|2031|526|6853|Saturday|2031Q2|N|Y|N|2462988|2463107|2462625|2462900|N|N|N|N|N| +2462991|AAAAAAAAPAFJFCAA|2031-05-04|1576|6853|526|2031|0|5|4|2|2031|526|6853|Sunday|2031Q2|N|N|N|2462988|2463107|2462626|2462901|N|N|N|N|N| +2462992|AAAAAAAAABFJFCAA|2031-05-05|1576|6853|526|2031|1|5|5|2|2031|526|6853|Monday|2031Q2|N|N|N|2462988|2463107|2462627|2462902|N|N|N|N|N| +2462993|AAAAAAAABBFJFCAA|2031-05-06|1576|6854|526|2031|2|5|6|2|2031|526|6854|Tuesday|2031Q2|N|N|N|2462988|2463107|2462628|2462903|N|N|N|N|N| +2462994|AAAAAAAACBFJFCAA|2031-05-07|1576|6854|526|2031|3|5|7|2|2031|526|6854|Wednesday|2031Q2|N|N|N|2462988|2463107|2462629|2462904|N|N|N|N|N| +2462995|AAAAAAAADBFJFCAA|2031-05-08|1576|6854|526|2031|4|5|8|2|2031|526|6854|Thursday|2031Q2|N|N|N|2462988|2463107|2462630|2462905|N|N|N|N|N| +2462996|AAAAAAAAEBFJFCAA|2031-05-09|1576|6854|526|2031|5|5|9|2|2031|526|6854|Friday|2031Q2|N|Y|N|2462988|2463107|2462631|2462906|N|N|N|N|N| +2462997|AAAAAAAAFBFJFCAA|2031-05-10|1576|6854|526|2031|6|5|10|2|2031|526|6854|Saturday|2031Q2|N|Y|N|2462988|2463107|2462632|2462907|N|N|N|N|N| +2462998|AAAAAAAAGBFJFCAA|2031-05-11|1576|6854|526|2031|0|5|11|2|2031|526|6854|Sunday|2031Q2|N|N|N|2462988|2463107|2462633|2462908|N|N|N|N|N| +2462999|AAAAAAAAHBFJFCAA|2031-05-12|1576|6854|526|2031|1|5|12|2|2031|526|6854|Monday|2031Q2|N|N|N|2462988|2463107|2462634|2462909|N|N|N|N|N| +2463000|AAAAAAAAIBFJFCAA|2031-05-13|1576|6855|526|2031|2|5|13|2|2031|526|6855|Tuesday|2031Q2|N|N|N|2462988|2463107|2462635|2462910|N|N|N|N|N| +2463001|AAAAAAAAJBFJFCAA|2031-05-14|1576|6855|526|2031|3|5|14|2|2031|526|6855|Wednesday|2031Q2|N|N|N|2462988|2463107|2462636|2462911|N|N|N|N|N| +2463002|AAAAAAAAKBFJFCAA|2031-05-15|1576|6855|526|2031|4|5|15|2|2031|526|6855|Thursday|2031Q2|N|N|N|2462988|2463107|2462637|2462912|N|N|N|N|N| +2463003|AAAAAAAALBFJFCAA|2031-05-16|1576|6855|526|2031|5|5|16|2|2031|526|6855|Friday|2031Q2|N|Y|N|2462988|2463107|2462638|2462913|N|N|N|N|N| +2463004|AAAAAAAAMBFJFCAA|2031-05-17|1576|6855|526|2031|6|5|17|2|2031|526|6855|Saturday|2031Q2|N|Y|N|2462988|2463107|2462639|2462914|N|N|N|N|N| +2463005|AAAAAAAANBFJFCAA|2031-05-18|1576|6855|526|2031|0|5|18|2|2031|526|6855|Sunday|2031Q2|N|N|N|2462988|2463107|2462640|2462915|N|N|N|N|N| +2463006|AAAAAAAAOBFJFCAA|2031-05-19|1576|6855|526|2031|1|5|19|2|2031|526|6855|Monday|2031Q2|N|N|N|2462988|2463107|2462641|2462916|N|N|N|N|N| +2463007|AAAAAAAAPBFJFCAA|2031-05-20|1576|6856|526|2031|2|5|20|2|2031|526|6856|Tuesday|2031Q2|N|N|N|2462988|2463107|2462642|2462917|N|N|N|N|N| +2463008|AAAAAAAAACFJFCAA|2031-05-21|1576|6856|526|2031|3|5|21|2|2031|526|6856|Wednesday|2031Q2|N|N|N|2462988|2463107|2462643|2462918|N|N|N|N|N| +2463009|AAAAAAAABCFJFCAA|2031-05-22|1576|6856|526|2031|4|5|22|2|2031|526|6856|Thursday|2031Q2|N|N|N|2462988|2463107|2462644|2462919|N|N|N|N|N| +2463010|AAAAAAAACCFJFCAA|2031-05-23|1576|6856|526|2031|5|5|23|2|2031|526|6856|Friday|2031Q2|N|Y|N|2462988|2463107|2462645|2462920|N|N|N|N|N| +2463011|AAAAAAAADCFJFCAA|2031-05-24|1576|6856|526|2031|6|5|24|2|2031|526|6856|Saturday|2031Q2|N|Y|N|2462988|2463107|2462646|2462921|N|N|N|N|N| +2463012|AAAAAAAAECFJFCAA|2031-05-25|1576|6856|526|2031|0|5|25|2|2031|526|6856|Sunday|2031Q2|N|N|N|2462988|2463107|2462647|2462922|N|N|N|N|N| +2463013|AAAAAAAAFCFJFCAA|2031-05-26|1576|6856|526|2031|1|5|26|2|2031|526|6856|Monday|2031Q2|N|N|N|2462988|2463107|2462648|2462923|N|N|N|N|N| +2463014|AAAAAAAAGCFJFCAA|2031-05-27|1576|6857|526|2031|2|5|27|2|2031|526|6857|Tuesday|2031Q2|N|N|N|2462988|2463107|2462649|2462924|N|N|N|N|N| +2463015|AAAAAAAAHCFJFCAA|2031-05-28|1576|6857|526|2031|3|5|28|2|2031|526|6857|Wednesday|2031Q2|N|N|N|2462988|2463107|2462650|2462925|N|N|N|N|N| +2463016|AAAAAAAAICFJFCAA|2031-05-29|1576|6857|526|2031|4|5|29|2|2031|526|6857|Thursday|2031Q2|N|N|N|2462988|2463107|2462651|2462926|N|N|N|N|N| +2463017|AAAAAAAAJCFJFCAA|2031-05-30|1576|6857|526|2031|5|5|30|2|2031|526|6857|Friday|2031Q2|N|Y|N|2462988|2463107|2462652|2462927|N|N|N|N|N| +2463018|AAAAAAAAKCFJFCAA|2031-05-31|1576|6857|526|2031|6|5|31|2|2031|526|6857|Saturday|2031Q2|N|Y|N|2462988|2463107|2462653|2462928|N|N|N|N|N| +2463019|AAAAAAAALCFJFCAA|2031-06-01|1577|6857|527|2031|0|6|1|2|2031|527|6857|Sunday|2031Q2|N|N|N|2463019|2463169|2462654|2462929|N|N|N|N|N| +2463020|AAAAAAAAMCFJFCAA|2031-06-02|1577|6857|527|2031|1|6|2|2|2031|527|6857|Monday|2031Q2|N|N|N|2463019|2463169|2462655|2462930|N|N|N|N|N| +2463021|AAAAAAAANCFJFCAA|2031-06-03|1577|6858|527|2031|2|6|3|2|2031|527|6858|Tuesday|2031Q2|N|N|N|2463019|2463169|2462656|2462931|N|N|N|N|N| +2463022|AAAAAAAAOCFJFCAA|2031-06-04|1577|6858|527|2031|3|6|4|2|2031|527|6858|Wednesday|2031Q2|N|N|N|2463019|2463169|2462657|2462932|N|N|N|N|N| +2463023|AAAAAAAAPCFJFCAA|2031-06-05|1577|6858|527|2031|4|6|5|2|2031|527|6858|Thursday|2031Q2|N|N|N|2463019|2463169|2462658|2462933|N|N|N|N|N| +2463024|AAAAAAAAADFJFCAA|2031-06-06|1577|6858|527|2031|5|6|6|2|2031|527|6858|Friday|2031Q2|N|Y|N|2463019|2463169|2462659|2462934|N|N|N|N|N| +2463025|AAAAAAAABDFJFCAA|2031-06-07|1577|6858|527|2031|6|6|7|2|2031|527|6858|Saturday|2031Q2|N|Y|N|2463019|2463169|2462660|2462935|N|N|N|N|N| +2463026|AAAAAAAACDFJFCAA|2031-06-08|1577|6858|527|2031|0|6|8|2|2031|527|6858|Sunday|2031Q2|N|N|N|2463019|2463169|2462661|2462936|N|N|N|N|N| +2463027|AAAAAAAADDFJFCAA|2031-06-09|1577|6858|527|2031|1|6|9|2|2031|527|6858|Monday|2031Q2|N|N|N|2463019|2463169|2462662|2462937|N|N|N|N|N| +2463028|AAAAAAAAEDFJFCAA|2031-06-10|1577|6859|527|2031|2|6|10|2|2031|527|6859|Tuesday|2031Q2|N|N|N|2463019|2463169|2462663|2462938|N|N|N|N|N| +2463029|AAAAAAAAFDFJFCAA|2031-06-11|1577|6859|527|2031|3|6|11|2|2031|527|6859|Wednesday|2031Q2|N|N|N|2463019|2463169|2462664|2462939|N|N|N|N|N| +2463030|AAAAAAAAGDFJFCAA|2031-06-12|1577|6859|527|2031|4|6|12|2|2031|527|6859|Thursday|2031Q2|N|N|N|2463019|2463169|2462665|2462940|N|N|N|N|N| +2463031|AAAAAAAAHDFJFCAA|2031-06-13|1577|6859|527|2031|5|6|13|2|2031|527|6859|Friday|2031Q2|N|Y|N|2463019|2463169|2462666|2462941|N|N|N|N|N| +2463032|AAAAAAAAIDFJFCAA|2031-06-14|1577|6859|527|2031|6|6|14|2|2031|527|6859|Saturday|2031Q2|N|Y|N|2463019|2463169|2462667|2462942|N|N|N|N|N| +2463033|AAAAAAAAJDFJFCAA|2031-06-15|1577|6859|527|2031|0|6|15|2|2031|527|6859|Sunday|2031Q2|N|N|N|2463019|2463169|2462668|2462943|N|N|N|N|N| +2463034|AAAAAAAAKDFJFCAA|2031-06-16|1577|6859|527|2031|1|6|16|2|2031|527|6859|Monday|2031Q2|N|N|N|2463019|2463169|2462669|2462944|N|N|N|N|N| +2463035|AAAAAAAALDFJFCAA|2031-06-17|1577|6860|527|2031|2|6|17|2|2031|527|6860|Tuesday|2031Q2|N|N|N|2463019|2463169|2462670|2462945|N|N|N|N|N| +2463036|AAAAAAAAMDFJFCAA|2031-06-18|1577|6860|527|2031|3|6|18|2|2031|527|6860|Wednesday|2031Q2|N|N|N|2463019|2463169|2462671|2462946|N|N|N|N|N| +2463037|AAAAAAAANDFJFCAA|2031-06-19|1577|6860|527|2031|4|6|19|2|2031|527|6860|Thursday|2031Q2|N|N|N|2463019|2463169|2462672|2462947|N|N|N|N|N| +2463038|AAAAAAAAODFJFCAA|2031-06-20|1577|6860|527|2031|5|6|20|2|2031|527|6860|Friday|2031Q2|N|Y|N|2463019|2463169|2462673|2462948|N|N|N|N|N| +2463039|AAAAAAAAPDFJFCAA|2031-06-21|1577|6860|527|2031|6|6|21|2|2031|527|6860|Saturday|2031Q2|N|Y|N|2463019|2463169|2462674|2462949|N|N|N|N|N| +2463040|AAAAAAAAAEFJFCAA|2031-06-22|1577|6860|527|2031|0|6|22|2|2031|527|6860|Sunday|2031Q2|N|N|N|2463019|2463169|2462675|2462950|N|N|N|N|N| +2463041|AAAAAAAABEFJFCAA|2031-06-23|1577|6860|527|2031|1|6|23|2|2031|527|6860|Monday|2031Q2|N|N|N|2463019|2463169|2462676|2462951|N|N|N|N|N| +2463042|AAAAAAAACEFJFCAA|2031-06-24|1577|6861|527|2031|2|6|24|2|2031|527|6861|Tuesday|2031Q2|N|N|N|2463019|2463169|2462677|2462952|N|N|N|N|N| +2463043|AAAAAAAADEFJFCAA|2031-06-25|1577|6861|527|2031|3|6|25|2|2031|527|6861|Wednesday|2031Q2|N|N|N|2463019|2463169|2462678|2462953|N|N|N|N|N| +2463044|AAAAAAAAEEFJFCAA|2031-06-26|1577|6861|527|2031|4|6|26|2|2031|527|6861|Thursday|2031Q2|N|N|N|2463019|2463169|2462679|2462954|N|N|N|N|N| +2463045|AAAAAAAAFEFJFCAA|2031-06-27|1577|6861|527|2031|5|6|27|2|2031|527|6861|Friday|2031Q2|N|Y|N|2463019|2463169|2462680|2462955|N|N|N|N|N| +2463046|AAAAAAAAGEFJFCAA|2031-06-28|1577|6861|527|2031|6|6|28|2|2031|527|6861|Saturday|2031Q2|N|Y|N|2463019|2463169|2462681|2462956|N|N|N|N|N| +2463047|AAAAAAAAHEFJFCAA|2031-06-29|1577|6861|527|2031|0|6|29|2|2031|527|6861|Sunday|2031Q2|N|N|N|2463019|2463169|2462682|2462957|N|N|N|N|N| +2463048|AAAAAAAAIEFJFCAA|2031-06-30|1577|6861|527|2031|1|6|30|2|2031|527|6861|Monday|2031Q2|N|N|N|2463019|2463169|2462683|2462958|N|N|N|N|N| +2463049|AAAAAAAAJEFJFCAA|2031-07-01|1578|6862|527|2031|2|7|1|2|2031|527|6862|Tuesday|2031Q2|N|N|N|2463049|2463229|2462684|2462958|N|N|N|N|N| +2463050|AAAAAAAAKEFJFCAA|2031-07-02|1578|6862|527|2031|3|7|2|3|2031|527|6862|Wednesday|2031Q3|N|N|N|2463049|2463229|2462685|2462959|N|N|N|N|N| +2463051|AAAAAAAALEFJFCAA|2031-07-03|1578|6862|527|2031|4|7|3|3|2031|527|6862|Thursday|2031Q3|N|N|N|2463049|2463229|2462686|2462960|N|N|N|N|N| +2463052|AAAAAAAAMEFJFCAA|2031-07-04|1578|6862|527|2031|5|7|4|3|2031|527|6862|Friday|2031Q3|N|Y|N|2463049|2463229|2462687|2462961|N|N|N|N|N| +2463053|AAAAAAAANEFJFCAA|2031-07-05|1578|6862|527|2031|6|7|5|3|2031|527|6862|Saturday|2031Q3|Y|Y|N|2463049|2463229|2462688|2462962|N|N|N|N|N| +2463054|AAAAAAAAOEFJFCAA|2031-07-06|1578|6862|527|2031|0|7|6|3|2031|527|6862|Sunday|2031Q3|N|N|Y|2463049|2463229|2462689|2462963|N|N|N|N|N| +2463055|AAAAAAAAPEFJFCAA|2031-07-07|1578|6862|527|2031|1|7|7|3|2031|527|6862|Monday|2031Q3|N|N|N|2463049|2463229|2462690|2462964|N|N|N|N|N| +2463056|AAAAAAAAAFFJFCAA|2031-07-08|1578|6863|527|2031|2|7|8|3|2031|527|6863|Tuesday|2031Q3|N|N|N|2463049|2463229|2462691|2462965|N|N|N|N|N| +2463057|AAAAAAAABFFJFCAA|2031-07-09|1578|6863|527|2031|3|7|9|3|2031|527|6863|Wednesday|2031Q3|N|N|N|2463049|2463229|2462692|2462966|N|N|N|N|N| +2463058|AAAAAAAACFFJFCAA|2031-07-10|1578|6863|527|2031|4|7|10|3|2031|527|6863|Thursday|2031Q3|N|N|N|2463049|2463229|2462693|2462967|N|N|N|N|N| +2463059|AAAAAAAADFFJFCAA|2031-07-11|1578|6863|527|2031|5|7|11|3|2031|527|6863|Friday|2031Q3|N|Y|N|2463049|2463229|2462694|2462968|N|N|N|N|N| +2463060|AAAAAAAAEFFJFCAA|2031-07-12|1578|6863|527|2031|6|7|12|3|2031|527|6863|Saturday|2031Q3|N|Y|N|2463049|2463229|2462695|2462969|N|N|N|N|N| +2463061|AAAAAAAAFFFJFCAA|2031-07-13|1578|6863|527|2031|0|7|13|3|2031|527|6863|Sunday|2031Q3|N|N|N|2463049|2463229|2462696|2462970|N|N|N|N|N| +2463062|AAAAAAAAGFFJFCAA|2031-07-14|1578|6863|527|2031|1|7|14|3|2031|527|6863|Monday|2031Q3|N|N|N|2463049|2463229|2462697|2462971|N|N|N|N|N| +2463063|AAAAAAAAHFFJFCAA|2031-07-15|1578|6864|527|2031|2|7|15|3|2031|527|6864|Tuesday|2031Q3|N|N|N|2463049|2463229|2462698|2462972|N|N|N|N|N| +2463064|AAAAAAAAIFFJFCAA|2031-07-16|1578|6864|527|2031|3|7|16|3|2031|527|6864|Wednesday|2031Q3|N|N|N|2463049|2463229|2462699|2462973|N|N|N|N|N| +2463065|AAAAAAAAJFFJFCAA|2031-07-17|1578|6864|527|2031|4|7|17|3|2031|527|6864|Thursday|2031Q3|N|N|N|2463049|2463229|2462700|2462974|N|N|N|N|N| +2463066|AAAAAAAAKFFJFCAA|2031-07-18|1578|6864|527|2031|5|7|18|3|2031|527|6864|Friday|2031Q3|N|Y|N|2463049|2463229|2462701|2462975|N|N|N|N|N| +2463067|AAAAAAAALFFJFCAA|2031-07-19|1578|6864|527|2031|6|7|19|3|2031|527|6864|Saturday|2031Q3|N|Y|N|2463049|2463229|2462702|2462976|N|N|N|N|N| +2463068|AAAAAAAAMFFJFCAA|2031-07-20|1578|6864|527|2031|0|7|20|3|2031|527|6864|Sunday|2031Q3|N|N|N|2463049|2463229|2462703|2462977|N|N|N|N|N| +2463069|AAAAAAAANFFJFCAA|2031-07-21|1578|6864|527|2031|1|7|21|3|2031|527|6864|Monday|2031Q3|N|N|N|2463049|2463229|2462704|2462978|N|N|N|N|N| +2463070|AAAAAAAAOFFJFCAA|2031-07-22|1578|6865|527|2031|2|7|22|3|2031|527|6865|Tuesday|2031Q3|N|N|N|2463049|2463229|2462705|2462979|N|N|N|N|N| +2463071|AAAAAAAAPFFJFCAA|2031-07-23|1578|6865|527|2031|3|7|23|3|2031|527|6865|Wednesday|2031Q3|N|N|N|2463049|2463229|2462706|2462980|N|N|N|N|N| +2463072|AAAAAAAAAGFJFCAA|2031-07-24|1578|6865|527|2031|4|7|24|3|2031|527|6865|Thursday|2031Q3|N|N|N|2463049|2463229|2462707|2462981|N|N|N|N|N| +2463073|AAAAAAAABGFJFCAA|2031-07-25|1578|6865|527|2031|5|7|25|3|2031|527|6865|Friday|2031Q3|N|Y|N|2463049|2463229|2462708|2462982|N|N|N|N|N| +2463074|AAAAAAAACGFJFCAA|2031-07-26|1578|6865|527|2031|6|7|26|3|2031|527|6865|Saturday|2031Q3|N|Y|N|2463049|2463229|2462709|2462983|N|N|N|N|N| +2463075|AAAAAAAADGFJFCAA|2031-07-27|1578|6865|527|2031|0|7|27|3|2031|527|6865|Sunday|2031Q3|N|N|N|2463049|2463229|2462710|2462984|N|N|N|N|N| +2463076|AAAAAAAAEGFJFCAA|2031-07-28|1578|6865|527|2031|1|7|28|3|2031|527|6865|Monday|2031Q3|N|N|N|2463049|2463229|2462711|2462985|N|N|N|N|N| +2463077|AAAAAAAAFGFJFCAA|2031-07-29|1578|6866|527|2031|2|7|29|3|2031|527|6866|Tuesday|2031Q3|N|N|N|2463049|2463229|2462712|2462986|N|N|N|N|N| +2463078|AAAAAAAAGGFJFCAA|2031-07-30|1578|6866|527|2031|3|7|30|3|2031|527|6866|Wednesday|2031Q3|N|N|N|2463049|2463229|2462713|2462987|N|N|N|N|N| +2463079|AAAAAAAAHGFJFCAA|2031-07-31|1578|6866|527|2031|4|7|31|3|2031|527|6866|Thursday|2031Q3|N|N|N|2463049|2463229|2462714|2462988|N|N|N|N|N| +2463080|AAAAAAAAIGFJFCAA|2031-08-01|1579|6866|527|2031|5|8|1|3|2031|527|6866|Friday|2031Q3|N|Y|N|2463080|2463291|2462715|2462989|N|N|N|N|N| +2463081|AAAAAAAAJGFJFCAA|2031-08-02|1579|6866|527|2031|6|8|2|3|2031|527|6866|Saturday|2031Q3|N|Y|N|2463080|2463291|2462716|2462990|N|N|N|N|N| +2463082|AAAAAAAAKGFJFCAA|2031-08-03|1579|6866|527|2031|0|8|3|3|2031|527|6866|Sunday|2031Q3|N|N|N|2463080|2463291|2462717|2462991|N|N|N|N|N| +2463083|AAAAAAAALGFJFCAA|2031-08-04|1579|6866|527|2031|1|8|4|3|2031|527|6866|Monday|2031Q3|N|N|N|2463080|2463291|2462718|2462992|N|N|N|N|N| +2463084|AAAAAAAAMGFJFCAA|2031-08-05|1579|6867|527|2031|2|8|5|3|2031|527|6867|Tuesday|2031Q3|N|N|N|2463080|2463291|2462719|2462993|N|N|N|N|N| +2463085|AAAAAAAANGFJFCAA|2031-08-06|1579|6867|527|2031|3|8|6|3|2031|527|6867|Wednesday|2031Q3|N|N|N|2463080|2463291|2462720|2462994|N|N|N|N|N| +2463086|AAAAAAAAOGFJFCAA|2031-08-07|1579|6867|527|2031|4|8|7|3|2031|527|6867|Thursday|2031Q3|N|N|N|2463080|2463291|2462721|2462995|N|N|N|N|N| +2463087|AAAAAAAAPGFJFCAA|2031-08-08|1579|6867|527|2031|5|8|8|3|2031|527|6867|Friday|2031Q3|N|Y|N|2463080|2463291|2462722|2462996|N|N|N|N|N| +2463088|AAAAAAAAAHFJFCAA|2031-08-09|1579|6867|527|2031|6|8|9|3|2031|527|6867|Saturday|2031Q3|N|Y|N|2463080|2463291|2462723|2462997|N|N|N|N|N| +2463089|AAAAAAAABHFJFCAA|2031-08-10|1579|6867|527|2031|0|8|10|3|2031|527|6867|Sunday|2031Q3|N|N|N|2463080|2463291|2462724|2462998|N|N|N|N|N| +2463090|AAAAAAAACHFJFCAA|2031-08-11|1579|6867|527|2031|1|8|11|3|2031|527|6867|Monday|2031Q3|N|N|N|2463080|2463291|2462725|2462999|N|N|N|N|N| +2463091|AAAAAAAADHFJFCAA|2031-08-12|1579|6868|527|2031|2|8|12|3|2031|527|6868|Tuesday|2031Q3|N|N|N|2463080|2463291|2462726|2463000|N|N|N|N|N| +2463092|AAAAAAAAEHFJFCAA|2031-08-13|1579|6868|527|2031|3|8|13|3|2031|527|6868|Wednesday|2031Q3|N|N|N|2463080|2463291|2462727|2463001|N|N|N|N|N| +2463093|AAAAAAAAFHFJFCAA|2031-08-14|1579|6868|527|2031|4|8|14|3|2031|527|6868|Thursday|2031Q3|N|N|N|2463080|2463291|2462728|2463002|N|N|N|N|N| +2463094|AAAAAAAAGHFJFCAA|2031-08-15|1579|6868|527|2031|5|8|15|3|2031|527|6868|Friday|2031Q3|N|Y|N|2463080|2463291|2462729|2463003|N|N|N|N|N| +2463095|AAAAAAAAHHFJFCAA|2031-08-16|1579|6868|527|2031|6|8|16|3|2031|527|6868|Saturday|2031Q3|N|Y|N|2463080|2463291|2462730|2463004|N|N|N|N|N| +2463096|AAAAAAAAIHFJFCAA|2031-08-17|1579|6868|527|2031|0|8|17|3|2031|527|6868|Sunday|2031Q3|N|N|N|2463080|2463291|2462731|2463005|N|N|N|N|N| +2463097|AAAAAAAAJHFJFCAA|2031-08-18|1579|6868|527|2031|1|8|18|3|2031|527|6868|Monday|2031Q3|N|N|N|2463080|2463291|2462732|2463006|N|N|N|N|N| +2463098|AAAAAAAAKHFJFCAA|2031-08-19|1579|6869|527|2031|2|8|19|3|2031|527|6869|Tuesday|2031Q3|N|N|N|2463080|2463291|2462733|2463007|N|N|N|N|N| +2463099|AAAAAAAALHFJFCAA|2031-08-20|1579|6869|527|2031|3|8|20|3|2031|527|6869|Wednesday|2031Q3|N|N|N|2463080|2463291|2462734|2463008|N|N|N|N|N| +2463100|AAAAAAAAMHFJFCAA|2031-08-21|1579|6869|527|2031|4|8|21|3|2031|527|6869|Thursday|2031Q3|N|N|N|2463080|2463291|2462735|2463009|N|N|N|N|N| +2463101|AAAAAAAANHFJFCAA|2031-08-22|1579|6869|527|2031|5|8|22|3|2031|527|6869|Friday|2031Q3|N|Y|N|2463080|2463291|2462736|2463010|N|N|N|N|N| +2463102|AAAAAAAAOHFJFCAA|2031-08-23|1579|6869|527|2031|6|8|23|3|2031|527|6869|Saturday|2031Q3|N|Y|N|2463080|2463291|2462737|2463011|N|N|N|N|N| +2463103|AAAAAAAAPHFJFCAA|2031-08-24|1579|6869|527|2031|0|8|24|3|2031|527|6869|Sunday|2031Q3|N|N|N|2463080|2463291|2462738|2463012|N|N|N|N|N| +2463104|AAAAAAAAAIFJFCAA|2031-08-25|1579|6869|527|2031|1|8|25|3|2031|527|6869|Monday|2031Q3|N|N|N|2463080|2463291|2462739|2463013|N|N|N|N|N| +2463105|AAAAAAAABIFJFCAA|2031-08-26|1579|6870|527|2031|2|8|26|3|2031|527|6870|Tuesday|2031Q3|N|N|N|2463080|2463291|2462740|2463014|N|N|N|N|N| +2463106|AAAAAAAACIFJFCAA|2031-08-27|1579|6870|527|2031|3|8|27|3|2031|527|6870|Wednesday|2031Q3|N|N|N|2463080|2463291|2462741|2463015|N|N|N|N|N| +2463107|AAAAAAAADIFJFCAA|2031-08-28|1579|6870|527|2031|4|8|28|3|2031|527|6870|Thursday|2031Q3|N|N|N|2463080|2463291|2462742|2463016|N|N|N|N|N| +2463108|AAAAAAAAEIFJFCAA|2031-08-29|1579|6870|527|2031|5|8|29|3|2031|527|6870|Friday|2031Q3|N|Y|N|2463080|2463291|2462743|2463017|N|N|N|N|N| +2463109|AAAAAAAAFIFJFCAA|2031-08-30|1579|6870|527|2031|6|8|30|3|2031|527|6870|Saturday|2031Q3|N|Y|N|2463080|2463291|2462744|2463018|N|N|N|N|N| +2463110|AAAAAAAAGIFJFCAA|2031-08-31|1579|6870|527|2031|0|8|31|3|2031|527|6870|Sunday|2031Q3|N|N|N|2463080|2463291|2462745|2463019|N|N|N|N|N| +2463111|AAAAAAAAHIFJFCAA|2031-09-01|1580|6870|528|2031|1|9|1|3|2031|528|6870|Monday|2031Q3|N|N|N|2463111|2463353|2462746|2463020|N|N|N|N|N| +2463112|AAAAAAAAIIFJFCAA|2031-09-02|1580|6871|528|2031|2|9|2|3|2031|528|6871|Tuesday|2031Q3|N|N|N|2463111|2463353|2462747|2463021|N|N|N|N|N| +2463113|AAAAAAAAJIFJFCAA|2031-09-03|1580|6871|528|2031|3|9|3|3|2031|528|6871|Wednesday|2031Q3|N|N|N|2463111|2463353|2462748|2463022|N|N|N|N|N| +2463114|AAAAAAAAKIFJFCAA|2031-09-04|1580|6871|528|2031|4|9|4|3|2031|528|6871|Thursday|2031Q3|N|N|N|2463111|2463353|2462749|2463023|N|N|N|N|N| +2463115|AAAAAAAALIFJFCAA|2031-09-05|1580|6871|528|2031|5|9|5|3|2031|528|6871|Friday|2031Q3|N|Y|N|2463111|2463353|2462750|2463024|N|N|N|N|N| +2463116|AAAAAAAAMIFJFCAA|2031-09-06|1580|6871|528|2031|6|9|6|3|2031|528|6871|Saturday|2031Q3|N|Y|N|2463111|2463353|2462751|2463025|N|N|N|N|N| +2463117|AAAAAAAANIFJFCAA|2031-09-07|1580|6871|528|2031|0|9|7|3|2031|528|6871|Sunday|2031Q3|N|N|N|2463111|2463353|2462752|2463026|N|N|N|N|N| +2463118|AAAAAAAAOIFJFCAA|2031-09-08|1580|6871|528|2031|1|9|8|3|2031|528|6871|Monday|2031Q3|N|N|N|2463111|2463353|2462753|2463027|N|N|N|N|N| +2463119|AAAAAAAAPIFJFCAA|2031-09-09|1580|6872|528|2031|2|9|9|3|2031|528|6872|Tuesday|2031Q3|N|N|N|2463111|2463353|2462754|2463028|N|N|N|N|N| +2463120|AAAAAAAAAJFJFCAA|2031-09-10|1580|6872|528|2031|3|9|10|3|2031|528|6872|Wednesday|2031Q3|N|N|N|2463111|2463353|2462755|2463029|N|N|N|N|N| +2463121|AAAAAAAABJFJFCAA|2031-09-11|1580|6872|528|2031|4|9|11|3|2031|528|6872|Thursday|2031Q3|N|N|N|2463111|2463353|2462756|2463030|N|N|N|N|N| +2463122|AAAAAAAACJFJFCAA|2031-09-12|1580|6872|528|2031|5|9|12|3|2031|528|6872|Friday|2031Q3|N|Y|N|2463111|2463353|2462757|2463031|N|N|N|N|N| +2463123|AAAAAAAADJFJFCAA|2031-09-13|1580|6872|528|2031|6|9|13|3|2031|528|6872|Saturday|2031Q3|N|Y|N|2463111|2463353|2462758|2463032|N|N|N|N|N| +2463124|AAAAAAAAEJFJFCAA|2031-09-14|1580|6872|528|2031|0|9|14|3|2031|528|6872|Sunday|2031Q3|N|N|N|2463111|2463353|2462759|2463033|N|N|N|N|N| +2463125|AAAAAAAAFJFJFCAA|2031-09-15|1580|6872|528|2031|1|9|15|3|2031|528|6872|Monday|2031Q3|N|N|N|2463111|2463353|2462760|2463034|N|N|N|N|N| +2463126|AAAAAAAAGJFJFCAA|2031-09-16|1580|6873|528|2031|2|9|16|3|2031|528|6873|Tuesday|2031Q3|N|N|N|2463111|2463353|2462761|2463035|N|N|N|N|N| +2463127|AAAAAAAAHJFJFCAA|2031-09-17|1580|6873|528|2031|3|9|17|3|2031|528|6873|Wednesday|2031Q3|N|N|N|2463111|2463353|2462762|2463036|N|N|N|N|N| +2463128|AAAAAAAAIJFJFCAA|2031-09-18|1580|6873|528|2031|4|9|18|3|2031|528|6873|Thursday|2031Q3|N|N|N|2463111|2463353|2462763|2463037|N|N|N|N|N| +2463129|AAAAAAAAJJFJFCAA|2031-09-19|1580|6873|528|2031|5|9|19|3|2031|528|6873|Friday|2031Q3|N|Y|N|2463111|2463353|2462764|2463038|N|N|N|N|N| +2463130|AAAAAAAAKJFJFCAA|2031-09-20|1580|6873|528|2031|6|9|20|3|2031|528|6873|Saturday|2031Q3|N|Y|N|2463111|2463353|2462765|2463039|N|N|N|N|N| +2463131|AAAAAAAALJFJFCAA|2031-09-21|1580|6873|528|2031|0|9|21|3|2031|528|6873|Sunday|2031Q3|N|N|N|2463111|2463353|2462766|2463040|N|N|N|N|N| +2463132|AAAAAAAAMJFJFCAA|2031-09-22|1580|6873|528|2031|1|9|22|3|2031|528|6873|Monday|2031Q3|N|N|N|2463111|2463353|2462767|2463041|N|N|N|N|N| +2463133|AAAAAAAANJFJFCAA|2031-09-23|1580|6874|528|2031|2|9|23|3|2031|528|6874|Tuesday|2031Q3|N|N|N|2463111|2463353|2462768|2463042|N|N|N|N|N| +2463134|AAAAAAAAOJFJFCAA|2031-09-24|1580|6874|528|2031|3|9|24|3|2031|528|6874|Wednesday|2031Q3|N|N|N|2463111|2463353|2462769|2463043|N|N|N|N|N| +2463135|AAAAAAAAPJFJFCAA|2031-09-25|1580|6874|528|2031|4|9|25|3|2031|528|6874|Thursday|2031Q3|N|N|N|2463111|2463353|2462770|2463044|N|N|N|N|N| +2463136|AAAAAAAAAKFJFCAA|2031-09-26|1580|6874|528|2031|5|9|26|3|2031|528|6874|Friday|2031Q3|N|Y|N|2463111|2463353|2462771|2463045|N|N|N|N|N| +2463137|AAAAAAAABKFJFCAA|2031-09-27|1580|6874|528|2031|6|9|27|3|2031|528|6874|Saturday|2031Q3|N|Y|N|2463111|2463353|2462772|2463046|N|N|N|N|N| +2463138|AAAAAAAACKFJFCAA|2031-09-28|1580|6874|528|2031|0|9|28|3|2031|528|6874|Sunday|2031Q3|N|N|N|2463111|2463353|2462773|2463047|N|N|N|N|N| +2463139|AAAAAAAADKFJFCAA|2031-09-29|1580|6874|528|2031|1|9|29|3|2031|528|6874|Monday|2031Q3|N|N|N|2463111|2463353|2462774|2463048|N|N|N|N|N| +2463140|AAAAAAAAEKFJFCAA|2031-09-30|1580|6875|528|2031|2|9|30|3|2031|528|6875|Tuesday|2031Q3|N|N|N|2463111|2463353|2462775|2463049|N|N|N|N|N| +2463141|AAAAAAAAFKFJFCAA|2031-10-01|1581|6875|528|2031|3|10|1|3|2031|528|6875|Wednesday|2031Q3|N|N|N|2463141|2463413|2462776|2463049|N|N|N|N|N| +2463142|AAAAAAAAGKFJFCAA|2031-10-02|1581|6875|528|2031|4|10|2|4|2031|528|6875|Thursday|2031Q4|N|N|N|2463141|2463413|2462777|2463050|N|N|N|N|N| +2463143|AAAAAAAAHKFJFCAA|2031-10-03|1581|6875|528|2031|5|10|3|4|2031|528|6875|Friday|2031Q4|N|Y|N|2463141|2463413|2462778|2463051|N|N|N|N|N| +2463144|AAAAAAAAIKFJFCAA|2031-10-04|1581|6875|528|2031|6|10|4|4|2031|528|6875|Saturday|2031Q4|N|Y|N|2463141|2463413|2462779|2463052|N|N|N|N|N| +2463145|AAAAAAAAJKFJFCAA|2031-10-05|1581|6875|528|2031|0|10|5|4|2031|528|6875|Sunday|2031Q4|N|N|N|2463141|2463413|2462780|2463053|N|N|N|N|N| +2463146|AAAAAAAAKKFJFCAA|2031-10-06|1581|6875|528|2031|1|10|6|4|2031|528|6875|Monday|2031Q4|N|N|N|2463141|2463413|2462781|2463054|N|N|N|N|N| +2463147|AAAAAAAALKFJFCAA|2031-10-07|1581|6876|528|2031|2|10|7|4|2031|528|6876|Tuesday|2031Q4|N|N|N|2463141|2463413|2462782|2463055|N|N|N|N|N| +2463148|AAAAAAAAMKFJFCAA|2031-10-08|1581|6876|528|2031|3|10|8|4|2031|528|6876|Wednesday|2031Q4|N|N|N|2463141|2463413|2462783|2463056|N|N|N|N|N| +2463149|AAAAAAAANKFJFCAA|2031-10-09|1581|6876|528|2031|4|10|9|4|2031|528|6876|Thursday|2031Q4|N|N|N|2463141|2463413|2462784|2463057|N|N|N|N|N| +2463150|AAAAAAAAOKFJFCAA|2031-10-10|1581|6876|528|2031|5|10|10|4|2031|528|6876|Friday|2031Q4|N|Y|N|2463141|2463413|2462785|2463058|N|N|N|N|N| +2463151|AAAAAAAAPKFJFCAA|2031-10-11|1581|6876|528|2031|6|10|11|4|2031|528|6876|Saturday|2031Q4|N|Y|N|2463141|2463413|2462786|2463059|N|N|N|N|N| +2463152|AAAAAAAAALFJFCAA|2031-10-12|1581|6876|528|2031|0|10|12|4|2031|528|6876|Sunday|2031Q4|N|N|N|2463141|2463413|2462787|2463060|N|N|N|N|N| +2463153|AAAAAAAABLFJFCAA|2031-10-13|1581|6876|528|2031|1|10|13|4|2031|528|6876|Monday|2031Q4|N|N|N|2463141|2463413|2462788|2463061|N|N|N|N|N| +2463154|AAAAAAAACLFJFCAA|2031-10-14|1581|6877|528|2031|2|10|14|4|2031|528|6877|Tuesday|2031Q4|N|N|N|2463141|2463413|2462789|2463062|N|N|N|N|N| +2463155|AAAAAAAADLFJFCAA|2031-10-15|1581|6877|528|2031|3|10|15|4|2031|528|6877|Wednesday|2031Q4|N|N|N|2463141|2463413|2462790|2463063|N|N|N|N|N| +2463156|AAAAAAAAELFJFCAA|2031-10-16|1581|6877|528|2031|4|10|16|4|2031|528|6877|Thursday|2031Q4|N|N|N|2463141|2463413|2462791|2463064|N|N|N|N|N| +2463157|AAAAAAAAFLFJFCAA|2031-10-17|1581|6877|528|2031|5|10|17|4|2031|528|6877|Friday|2031Q4|N|Y|N|2463141|2463413|2462792|2463065|N|N|N|N|N| +2463158|AAAAAAAAGLFJFCAA|2031-10-18|1581|6877|528|2031|6|10|18|4|2031|528|6877|Saturday|2031Q4|N|Y|N|2463141|2463413|2462793|2463066|N|N|N|N|N| +2463159|AAAAAAAAHLFJFCAA|2031-10-19|1581|6877|528|2031|0|10|19|4|2031|528|6877|Sunday|2031Q4|N|N|N|2463141|2463413|2462794|2463067|N|N|N|N|N| +2463160|AAAAAAAAILFJFCAA|2031-10-20|1581|6877|528|2031|1|10|20|4|2031|528|6877|Monday|2031Q4|N|N|N|2463141|2463413|2462795|2463068|N|N|N|N|N| +2463161|AAAAAAAAJLFJFCAA|2031-10-21|1581|6878|528|2031|2|10|21|4|2031|528|6878|Tuesday|2031Q4|N|N|N|2463141|2463413|2462796|2463069|N|N|N|N|N| +2463162|AAAAAAAAKLFJFCAA|2031-10-22|1581|6878|528|2031|3|10|22|4|2031|528|6878|Wednesday|2031Q4|N|N|N|2463141|2463413|2462797|2463070|N|N|N|N|N| +2463163|AAAAAAAALLFJFCAA|2031-10-23|1581|6878|528|2031|4|10|23|4|2031|528|6878|Thursday|2031Q4|N|N|N|2463141|2463413|2462798|2463071|N|N|N|N|N| +2463164|AAAAAAAAMLFJFCAA|2031-10-24|1581|6878|528|2031|5|10|24|4|2031|528|6878|Friday|2031Q4|N|Y|N|2463141|2463413|2462799|2463072|N|N|N|N|N| +2463165|AAAAAAAANLFJFCAA|2031-10-25|1581|6878|528|2031|6|10|25|4|2031|528|6878|Saturday|2031Q4|N|Y|N|2463141|2463413|2462800|2463073|N|N|N|N|N| +2463166|AAAAAAAAOLFJFCAA|2031-10-26|1581|6878|528|2031|0|10|26|4|2031|528|6878|Sunday|2031Q4|N|N|N|2463141|2463413|2462801|2463074|N|N|N|N|N| +2463167|AAAAAAAAPLFJFCAA|2031-10-27|1581|6878|528|2031|1|10|27|4|2031|528|6878|Monday|2031Q4|N|N|N|2463141|2463413|2462802|2463075|N|N|N|N|N| +2463168|AAAAAAAAAMFJFCAA|2031-10-28|1581|6879|528|2031|2|10|28|4|2031|528|6879|Tuesday|2031Q4|N|N|N|2463141|2463413|2462803|2463076|N|N|N|N|N| +2463169|AAAAAAAABMFJFCAA|2031-10-29|1581|6879|528|2031|3|10|29|4|2031|528|6879|Wednesday|2031Q4|N|N|N|2463141|2463413|2462804|2463077|N|N|N|N|N| +2463170|AAAAAAAACMFJFCAA|2031-10-30|1581|6879|528|2031|4|10|30|4|2031|528|6879|Thursday|2031Q4|N|N|N|2463141|2463413|2462805|2463078|N|N|N|N|N| +2463171|AAAAAAAADMFJFCAA|2031-10-31|1581|6879|528|2031|5|10|31|4|2031|528|6879|Friday|2031Q4|N|Y|N|2463141|2463413|2462806|2463079|N|N|N|N|N| +2463172|AAAAAAAAEMFJFCAA|2031-11-01|1582|6879|528|2031|6|11|1|4|2031|528|6879|Saturday|2031Q4|N|Y|N|2463172|2463475|2462807|2463080|N|N|N|N|N| +2463173|AAAAAAAAFMFJFCAA|2031-11-02|1582|6879|528|2031|0|11|2|4|2031|528|6879|Sunday|2031Q4|N|N|N|2463172|2463475|2462808|2463081|N|N|N|N|N| +2463174|AAAAAAAAGMFJFCAA|2031-11-03|1582|6879|528|2031|1|11|3|4|2031|528|6879|Monday|2031Q4|N|N|N|2463172|2463475|2462809|2463082|N|N|N|N|N| +2463175|AAAAAAAAHMFJFCAA|2031-11-04|1582|6880|528|2031|2|11|4|4|2031|528|6880|Tuesday|2031Q4|N|N|N|2463172|2463475|2462810|2463083|N|N|N|N|N| +2463176|AAAAAAAAIMFJFCAA|2031-11-05|1582|6880|528|2031|3|11|5|4|2031|528|6880|Wednesday|2031Q4|N|N|N|2463172|2463475|2462811|2463084|N|N|N|N|N| +2463177|AAAAAAAAJMFJFCAA|2031-11-06|1582|6880|528|2031|4|11|6|4|2031|528|6880|Thursday|2031Q4|N|N|N|2463172|2463475|2462812|2463085|N|N|N|N|N| +2463178|AAAAAAAAKMFJFCAA|2031-11-07|1582|6880|528|2031|5|11|7|4|2031|528|6880|Friday|2031Q4|N|Y|N|2463172|2463475|2462813|2463086|N|N|N|N|N| +2463179|AAAAAAAALMFJFCAA|2031-11-08|1582|6880|528|2031|6|11|8|4|2031|528|6880|Saturday|2031Q4|N|Y|N|2463172|2463475|2462814|2463087|N|N|N|N|N| +2463180|AAAAAAAAMMFJFCAA|2031-11-09|1582|6880|528|2031|0|11|9|4|2031|528|6880|Sunday|2031Q4|N|N|N|2463172|2463475|2462815|2463088|N|N|N|N|N| +2463181|AAAAAAAANMFJFCAA|2031-11-10|1582|6880|528|2031|1|11|10|4|2031|528|6880|Monday|2031Q4|N|N|N|2463172|2463475|2462816|2463089|N|N|N|N|N| +2463182|AAAAAAAAOMFJFCAA|2031-11-11|1582|6881|528|2031|2|11|11|4|2031|528|6881|Tuesday|2031Q4|N|N|N|2463172|2463475|2462817|2463090|N|N|N|N|N| +2463183|AAAAAAAAPMFJFCAA|2031-11-12|1582|6881|528|2031|3|11|12|4|2031|528|6881|Wednesday|2031Q4|N|N|N|2463172|2463475|2462818|2463091|N|N|N|N|N| +2463184|AAAAAAAAANFJFCAA|2031-11-13|1582|6881|528|2031|4|11|13|4|2031|528|6881|Thursday|2031Q4|N|N|N|2463172|2463475|2462819|2463092|N|N|N|N|N| +2463185|AAAAAAAABNFJFCAA|2031-11-14|1582|6881|528|2031|5|11|14|4|2031|528|6881|Friday|2031Q4|N|Y|N|2463172|2463475|2462820|2463093|N|N|N|N|N| +2463186|AAAAAAAACNFJFCAA|2031-11-15|1582|6881|528|2031|6|11|15|4|2031|528|6881|Saturday|2031Q4|N|Y|N|2463172|2463475|2462821|2463094|N|N|N|N|N| +2463187|AAAAAAAADNFJFCAA|2031-11-16|1582|6881|528|2031|0|11|16|4|2031|528|6881|Sunday|2031Q4|N|N|N|2463172|2463475|2462822|2463095|N|N|N|N|N| +2463188|AAAAAAAAENFJFCAA|2031-11-17|1582|6881|528|2031|1|11|17|4|2031|528|6881|Monday|2031Q4|N|N|N|2463172|2463475|2462823|2463096|N|N|N|N|N| +2463189|AAAAAAAAFNFJFCAA|2031-11-18|1582|6882|528|2031|2|11|18|4|2031|528|6882|Tuesday|2031Q4|N|N|N|2463172|2463475|2462824|2463097|N|N|N|N|N| +2463190|AAAAAAAAGNFJFCAA|2031-11-19|1582|6882|528|2031|3|11|19|4|2031|528|6882|Wednesday|2031Q4|N|N|N|2463172|2463475|2462825|2463098|N|N|N|N|N| +2463191|AAAAAAAAHNFJFCAA|2031-11-20|1582|6882|528|2031|4|11|20|4|2031|528|6882|Thursday|2031Q4|N|N|N|2463172|2463475|2462826|2463099|N|N|N|N|N| +2463192|AAAAAAAAINFJFCAA|2031-11-21|1582|6882|528|2031|5|11|21|4|2031|528|6882|Friday|2031Q4|N|Y|N|2463172|2463475|2462827|2463100|N|N|N|N|N| +2463193|AAAAAAAAJNFJFCAA|2031-11-22|1582|6882|528|2031|6|11|22|4|2031|528|6882|Saturday|2031Q4|N|Y|N|2463172|2463475|2462828|2463101|N|N|N|N|N| +2463194|AAAAAAAAKNFJFCAA|2031-11-23|1582|6882|528|2031|0|11|23|4|2031|528|6882|Sunday|2031Q4|N|N|N|2463172|2463475|2462829|2463102|N|N|N|N|N| +2463195|AAAAAAAALNFJFCAA|2031-11-24|1582|6882|528|2031|1|11|24|4|2031|528|6882|Monday|2031Q4|N|N|N|2463172|2463475|2462830|2463103|N|N|N|N|N| +2463196|AAAAAAAAMNFJFCAA|2031-11-25|1582|6883|528|2031|2|11|25|4|2031|528|6883|Tuesday|2031Q4|N|N|N|2463172|2463475|2462831|2463104|N|N|N|N|N| +2463197|AAAAAAAANNFJFCAA|2031-11-26|1582|6883|528|2031|3|11|26|4|2031|528|6883|Wednesday|2031Q4|N|N|N|2463172|2463475|2462832|2463105|N|N|N|N|N| +2463198|AAAAAAAAONFJFCAA|2031-11-27|1582|6883|528|2031|4|11|27|4|2031|528|6883|Thursday|2031Q4|N|N|N|2463172|2463475|2462833|2463106|N|N|N|N|N| +2463199|AAAAAAAAPNFJFCAA|2031-11-28|1582|6883|528|2031|5|11|28|4|2031|528|6883|Friday|2031Q4|N|Y|N|2463172|2463475|2462834|2463107|N|N|N|N|N| +2463200|AAAAAAAAAOFJFCAA|2031-11-29|1582|6883|528|2031|6|11|29|4|2031|528|6883|Saturday|2031Q4|N|Y|N|2463172|2463475|2462835|2463108|N|N|N|N|N| +2463201|AAAAAAAABOFJFCAA|2031-11-30|1582|6883|528|2031|0|11|30|4|2031|528|6883|Sunday|2031Q4|N|N|N|2463172|2463475|2462836|2463109|N|N|N|N|N| +2463202|AAAAAAAACOFJFCAA|2031-12-01|1583|6883|529|2031|1|12|1|4|2031|529|6883|Monday|2031Q4|N|N|N|2463202|2463535|2462837|2463110|N|N|N|N|N| +2463203|AAAAAAAADOFJFCAA|2031-12-02|1583|6884|529|2031|2|12|2|4|2031|529|6884|Tuesday|2031Q4|N|N|N|2463202|2463535|2462838|2463111|N|N|N|N|N| +2463204|AAAAAAAAEOFJFCAA|2031-12-03|1583|6884|529|2031|3|12|3|4|2031|529|6884|Wednesday|2031Q4|N|N|N|2463202|2463535|2462839|2463112|N|N|N|N|N| +2463205|AAAAAAAAFOFJFCAA|2031-12-04|1583|6884|529|2031|4|12|4|4|2031|529|6884|Thursday|2031Q4|N|N|N|2463202|2463535|2462840|2463113|N|N|N|N|N| +2463206|AAAAAAAAGOFJFCAA|2031-12-05|1583|6884|529|2031|5|12|5|4|2031|529|6884|Friday|2031Q4|N|Y|N|2463202|2463535|2462841|2463114|N|N|N|N|N| +2463207|AAAAAAAAHOFJFCAA|2031-12-06|1583|6884|529|2031|6|12|6|4|2031|529|6884|Saturday|2031Q4|N|Y|N|2463202|2463535|2462842|2463115|N|N|N|N|N| +2463208|AAAAAAAAIOFJFCAA|2031-12-07|1583|6884|529|2031|0|12|7|4|2031|529|6884|Sunday|2031Q4|N|N|N|2463202|2463535|2462843|2463116|N|N|N|N|N| +2463209|AAAAAAAAJOFJFCAA|2031-12-08|1583|6884|529|2031|1|12|8|4|2031|529|6884|Monday|2031Q4|N|N|N|2463202|2463535|2462844|2463117|N|N|N|N|N| +2463210|AAAAAAAAKOFJFCAA|2031-12-09|1583|6885|529|2031|2|12|9|4|2031|529|6885|Tuesday|2031Q4|N|N|N|2463202|2463535|2462845|2463118|N|N|N|N|N| +2463211|AAAAAAAALOFJFCAA|2031-12-10|1583|6885|529|2031|3|12|10|4|2031|529|6885|Wednesday|2031Q4|N|N|N|2463202|2463535|2462846|2463119|N|N|N|N|N| +2463212|AAAAAAAAMOFJFCAA|2031-12-11|1583|6885|529|2031|4|12|11|4|2031|529|6885|Thursday|2031Q4|N|N|N|2463202|2463535|2462847|2463120|N|N|N|N|N| +2463213|AAAAAAAANOFJFCAA|2031-12-12|1583|6885|529|2031|5|12|12|4|2031|529|6885|Friday|2031Q4|N|Y|N|2463202|2463535|2462848|2463121|N|N|N|N|N| +2463214|AAAAAAAAOOFJFCAA|2031-12-13|1583|6885|529|2031|6|12|13|4|2031|529|6885|Saturday|2031Q4|N|Y|N|2463202|2463535|2462849|2463122|N|N|N|N|N| +2463215|AAAAAAAAPOFJFCAA|2031-12-14|1583|6885|529|2031|0|12|14|4|2031|529|6885|Sunday|2031Q4|N|N|N|2463202|2463535|2462850|2463123|N|N|N|N|N| +2463216|AAAAAAAAAPFJFCAA|2031-12-15|1583|6885|529|2031|1|12|15|4|2031|529|6885|Monday|2031Q4|N|N|N|2463202|2463535|2462851|2463124|N|N|N|N|N| +2463217|AAAAAAAABPFJFCAA|2031-12-16|1583|6886|529|2031|2|12|16|4|2031|529|6886|Tuesday|2031Q4|N|N|N|2463202|2463535|2462852|2463125|N|N|N|N|N| +2463218|AAAAAAAACPFJFCAA|2031-12-17|1583|6886|529|2031|3|12|17|4|2031|529|6886|Wednesday|2031Q4|N|N|N|2463202|2463535|2462853|2463126|N|N|N|N|N| +2463219|AAAAAAAADPFJFCAA|2031-12-18|1583|6886|529|2031|4|12|18|4|2031|529|6886|Thursday|2031Q4|N|N|N|2463202|2463535|2462854|2463127|N|N|N|N|N| +2463220|AAAAAAAAEPFJFCAA|2031-12-19|1583|6886|529|2031|5|12|19|4|2031|529|6886|Friday|2031Q4|N|Y|N|2463202|2463535|2462855|2463128|N|N|N|N|N| +2463221|AAAAAAAAFPFJFCAA|2031-12-20|1583|6886|529|2031|6|12|20|4|2031|529|6886|Saturday|2031Q4|N|Y|N|2463202|2463535|2462856|2463129|N|N|N|N|N| +2463222|AAAAAAAAGPFJFCAA|2031-12-21|1583|6886|529|2031|0|12|21|4|2031|529|6886|Sunday|2031Q4|N|N|N|2463202|2463535|2462857|2463130|N|N|N|N|N| +2463223|AAAAAAAAHPFJFCAA|2031-12-22|1583|6886|529|2031|1|12|22|4|2031|529|6886|Monday|2031Q4|N|N|N|2463202|2463535|2462858|2463131|N|N|N|N|N| +2463224|AAAAAAAAIPFJFCAA|2031-12-23|1583|6887|529|2031|2|12|23|4|2031|529|6887|Tuesday|2031Q4|N|N|N|2463202|2463535|2462859|2463132|N|N|N|N|N| +2463225|AAAAAAAAJPFJFCAA|2031-12-24|1583|6887|529|2031|3|12|24|4|2031|529|6887|Wednesday|2031Q4|N|N|N|2463202|2463535|2462860|2463133|N|N|N|N|N| +2463226|AAAAAAAAKPFJFCAA|2031-12-25|1583|6887|529|2031|4|12|25|4|2031|529|6887|Thursday|2031Q4|N|N|N|2463202|2463535|2462861|2463134|N|N|N|N|N| +2463227|AAAAAAAALPFJFCAA|2031-12-26|1583|6887|529|2031|5|12|26|4|2031|529|6887|Friday|2031Q4|Y|Y|N|2463202|2463535|2462862|2463135|N|N|N|N|N| +2463228|AAAAAAAAMPFJFCAA|2031-12-27|1583|6887|529|2031|6|12|27|4|2031|529|6887|Saturday|2031Q4|N|Y|Y|2463202|2463535|2462863|2463136|N|N|N|N|N| +2463229|AAAAAAAANPFJFCAA|2031-12-28|1583|6887|529|2031|0|12|28|4|2031|529|6887|Sunday|2031Q4|N|N|N|2463202|2463535|2462864|2463137|N|N|N|N|N| +2463230|AAAAAAAAOPFJFCAA|2031-12-29|1583|6887|529|2031|1|12|29|4|2031|529|6887|Monday|2031Q4|N|N|N|2463202|2463535|2462865|2463138|N|N|N|N|N| +2463231|AAAAAAAAPPFJFCAA|2031-12-30|1583|6888|529|2031|2|12|30|4|2031|529|6888|Tuesday|2031Q4|N|N|N|2463202|2463535|2462866|2463139|N|N|N|N|N| +2463232|AAAAAAAAAAGJFCAA|2031-12-31|1583|6888|529|2031|3|12|31|4|2031|529|6888|Wednesday|2031Q4|N|N|N|2463202|2463535|2462867|2463140|N|N|N|N|N| +2463233|AAAAAAAABAGJFCAA|2032-01-01|1584|6888|529|2032|4|1|1|1|2032|529|6888|Thursday|2032Q1|Y|N|N|2463233|2463232|2462868|2463141|N|N|N|N|N| +2463234|AAAAAAAACAGJFCAA|2032-01-02|1584|6888|529|2032|5|1|2|1|2032|529|6888|Friday|2032Q1|N|Y|Y|2463233|2463232|2462869|2463142|N|N|N|N|N| +2463235|AAAAAAAADAGJFCAA|2032-01-03|1584|6888|529|2032|6|1|3|1|2032|529|6888|Saturday|2032Q1|N|Y|N|2463233|2463232|2462870|2463143|N|N|N|N|N| +2463236|AAAAAAAAEAGJFCAA|2032-01-04|1584|6888|529|2032|0|1|4|1|2032|529|6888|Sunday|2032Q1|N|N|N|2463233|2463232|2462871|2463144|N|N|N|N|N| +2463237|AAAAAAAAFAGJFCAA|2032-01-05|1584|6888|529|2032|1|1|5|1|2032|529|6888|Monday|2032Q1|N|N|N|2463233|2463232|2462872|2463145|N|N|N|N|N| +2463238|AAAAAAAAGAGJFCAA|2032-01-06|1584|6889|529|2032|2|1|6|1|2032|529|6889|Tuesday|2032Q1|N|N|N|2463233|2463232|2462873|2463146|N|N|N|N|N| +2463239|AAAAAAAAHAGJFCAA|2032-01-07|1584|6889|529|2032|3|1|7|1|2032|529|6889|Wednesday|2032Q1|N|N|N|2463233|2463232|2462874|2463147|N|N|N|N|N| +2463240|AAAAAAAAIAGJFCAA|2032-01-08|1584|6889|529|2032|4|1|8|1|2032|529|6889|Thursday|2032Q1|N|N|N|2463233|2463232|2462875|2463148|N|N|N|N|N| +2463241|AAAAAAAAJAGJFCAA|2032-01-09|1584|6889|529|2032|5|1|9|1|2032|529|6889|Friday|2032Q1|N|Y|N|2463233|2463232|2462876|2463149|N|N|N|N|N| +2463242|AAAAAAAAKAGJFCAA|2032-01-10|1584|6889|529|2032|6|1|10|1|2032|529|6889|Saturday|2032Q1|N|Y|N|2463233|2463232|2462877|2463150|N|N|N|N|N| +2463243|AAAAAAAALAGJFCAA|2032-01-11|1584|6889|529|2032|0|1|11|1|2032|529|6889|Sunday|2032Q1|N|N|N|2463233|2463232|2462878|2463151|N|N|N|N|N| +2463244|AAAAAAAAMAGJFCAA|2032-01-12|1584|6889|529|2032|1|1|12|1|2032|529|6889|Monday|2032Q1|N|N|N|2463233|2463232|2462879|2463152|N|N|N|N|N| +2463245|AAAAAAAANAGJFCAA|2032-01-13|1584|6890|529|2032|2|1|13|1|2032|529|6890|Tuesday|2032Q1|N|N|N|2463233|2463232|2462880|2463153|N|N|N|N|N| +2463246|AAAAAAAAOAGJFCAA|2032-01-14|1584|6890|529|2032|3|1|14|1|2032|529|6890|Wednesday|2032Q1|N|N|N|2463233|2463232|2462881|2463154|N|N|N|N|N| +2463247|AAAAAAAAPAGJFCAA|2032-01-15|1584|6890|529|2032|4|1|15|1|2032|529|6890|Thursday|2032Q1|N|N|N|2463233|2463232|2462882|2463155|N|N|N|N|N| +2463248|AAAAAAAAABGJFCAA|2032-01-16|1584|6890|529|2032|5|1|16|1|2032|529|6890|Friday|2032Q1|N|Y|N|2463233|2463232|2462883|2463156|N|N|N|N|N| +2463249|AAAAAAAABBGJFCAA|2032-01-17|1584|6890|529|2032|6|1|17|1|2032|529|6890|Saturday|2032Q1|N|Y|N|2463233|2463232|2462884|2463157|N|N|N|N|N| +2463250|AAAAAAAACBGJFCAA|2032-01-18|1584|6890|529|2032|0|1|18|1|2032|529|6890|Sunday|2032Q1|N|N|N|2463233|2463232|2462885|2463158|N|N|N|N|N| +2463251|AAAAAAAADBGJFCAA|2032-01-19|1584|6890|529|2032|1|1|19|1|2032|529|6890|Monday|2032Q1|N|N|N|2463233|2463232|2462886|2463159|N|N|N|N|N| +2463252|AAAAAAAAEBGJFCAA|2032-01-20|1584|6891|529|2032|2|1|20|1|2032|529|6891|Tuesday|2032Q1|N|N|N|2463233|2463232|2462887|2463160|N|N|N|N|N| +2463253|AAAAAAAAFBGJFCAA|2032-01-21|1584|6891|529|2032|3|1|21|1|2032|529|6891|Wednesday|2032Q1|N|N|N|2463233|2463232|2462888|2463161|N|N|N|N|N| +2463254|AAAAAAAAGBGJFCAA|2032-01-22|1584|6891|529|2032|4|1|22|1|2032|529|6891|Thursday|2032Q1|N|N|N|2463233|2463232|2462889|2463162|N|N|N|N|N| +2463255|AAAAAAAAHBGJFCAA|2032-01-23|1584|6891|529|2032|5|1|23|1|2032|529|6891|Friday|2032Q1|N|Y|N|2463233|2463232|2462890|2463163|N|N|N|N|N| +2463256|AAAAAAAAIBGJFCAA|2032-01-24|1584|6891|529|2032|6|1|24|1|2032|529|6891|Saturday|2032Q1|N|Y|N|2463233|2463232|2462891|2463164|N|N|N|N|N| +2463257|AAAAAAAAJBGJFCAA|2032-01-25|1584|6891|529|2032|0|1|25|1|2032|529|6891|Sunday|2032Q1|N|N|N|2463233|2463232|2462892|2463165|N|N|N|N|N| +2463258|AAAAAAAAKBGJFCAA|2032-01-26|1584|6891|529|2032|1|1|26|1|2032|529|6891|Monday|2032Q1|N|N|N|2463233|2463232|2462893|2463166|N|N|N|N|N| +2463259|AAAAAAAALBGJFCAA|2032-01-27|1584|6892|529|2032|2|1|27|1|2032|529|6892|Tuesday|2032Q1|N|N|N|2463233|2463232|2462894|2463167|N|N|N|N|N| +2463260|AAAAAAAAMBGJFCAA|2032-01-28|1584|6892|529|2032|3|1|28|1|2032|529|6892|Wednesday|2032Q1|N|N|N|2463233|2463232|2462895|2463168|N|N|N|N|N| +2463261|AAAAAAAANBGJFCAA|2032-01-29|1584|6892|529|2032|4|1|29|1|2032|529|6892|Thursday|2032Q1|N|N|N|2463233|2463232|2462896|2463169|N|N|N|N|N| +2463262|AAAAAAAAOBGJFCAA|2032-01-30|1584|6892|529|2032|5|1|30|1|2032|529|6892|Friday|2032Q1|N|Y|N|2463233|2463232|2462897|2463170|N|N|N|N|N| +2463263|AAAAAAAAPBGJFCAA|2032-01-31|1584|6892|529|2032|6|1|31|1|2032|529|6892|Saturday|2032Q1|N|Y|N|2463233|2463232|2462898|2463171|N|N|N|N|N| +2463264|AAAAAAAAACGJFCAA|2032-02-01|1585|6892|529|2032|0|2|1|1|2032|529|6892|Sunday|2032Q1|N|N|N|2463264|2463294|2462899|2463172|N|N|N|N|N| +2463265|AAAAAAAABCGJFCAA|2032-02-02|1585|6892|529|2032|1|2|2|1|2032|529|6892|Monday|2032Q1|N|N|N|2463264|2463294|2462900|2463173|N|N|N|N|N| +2463266|AAAAAAAACCGJFCAA|2032-02-03|1585|6893|529|2032|2|2|3|1|2032|529|6893|Tuesday|2032Q1|N|N|N|2463264|2463294|2462901|2463174|N|N|N|N|N| +2463267|AAAAAAAADCGJFCAA|2032-02-04|1585|6893|529|2032|3|2|4|1|2032|529|6893|Wednesday|2032Q1|N|N|N|2463264|2463294|2462902|2463175|N|N|N|N|N| +2463268|AAAAAAAAECGJFCAA|2032-02-05|1585|6893|529|2032|4|2|5|1|2032|529|6893|Thursday|2032Q1|N|N|N|2463264|2463294|2462903|2463176|N|N|N|N|N| +2463269|AAAAAAAAFCGJFCAA|2032-02-06|1585|6893|529|2032|5|2|6|1|2032|529|6893|Friday|2032Q1|N|Y|N|2463264|2463294|2462904|2463177|N|N|N|N|N| +2463270|AAAAAAAAGCGJFCAA|2032-02-07|1585|6893|529|2032|6|2|7|1|2032|529|6893|Saturday|2032Q1|N|Y|N|2463264|2463294|2462905|2463178|N|N|N|N|N| +2463271|AAAAAAAAHCGJFCAA|2032-02-08|1585|6893|529|2032|0|2|8|1|2032|529|6893|Sunday|2032Q1|N|N|N|2463264|2463294|2462906|2463179|N|N|N|N|N| +2463272|AAAAAAAAICGJFCAA|2032-02-09|1585|6893|529|2032|1|2|9|1|2032|529|6893|Monday|2032Q1|N|N|N|2463264|2463294|2462907|2463180|N|N|N|N|N| +2463273|AAAAAAAAJCGJFCAA|2032-02-10|1585|6894|529|2032|2|2|10|1|2032|529|6894|Tuesday|2032Q1|N|N|N|2463264|2463294|2462908|2463181|N|N|N|N|N| +2463274|AAAAAAAAKCGJFCAA|2032-02-11|1585|6894|529|2032|3|2|11|1|2032|529|6894|Wednesday|2032Q1|N|N|N|2463264|2463294|2462909|2463182|N|N|N|N|N| +2463275|AAAAAAAALCGJFCAA|2032-02-12|1585|6894|529|2032|4|2|12|1|2032|529|6894|Thursday|2032Q1|N|N|N|2463264|2463294|2462910|2463183|N|N|N|N|N| +2463276|AAAAAAAAMCGJFCAA|2032-02-13|1585|6894|529|2032|5|2|13|1|2032|529|6894|Friday|2032Q1|N|Y|N|2463264|2463294|2462911|2463184|N|N|N|N|N| +2463277|AAAAAAAANCGJFCAA|2032-02-14|1585|6894|529|2032|6|2|14|1|2032|529|6894|Saturday|2032Q1|N|Y|N|2463264|2463294|2462912|2463185|N|N|N|N|N| +2463278|AAAAAAAAOCGJFCAA|2032-02-15|1585|6894|529|2032|0|2|15|1|2032|529|6894|Sunday|2032Q1|N|N|N|2463264|2463294|2462913|2463186|N|N|N|N|N| +2463279|AAAAAAAAPCGJFCAA|2032-02-16|1585|6894|529|2032|1|2|16|1|2032|529|6894|Monday|2032Q1|N|N|N|2463264|2463294|2462914|2463187|N|N|N|N|N| +2463280|AAAAAAAAADGJFCAA|2032-02-17|1585|6895|529|2032|2|2|17|1|2032|529|6895|Tuesday|2032Q1|N|N|N|2463264|2463294|2462915|2463188|N|N|N|N|N| +2463281|AAAAAAAABDGJFCAA|2032-02-18|1585|6895|529|2032|3|2|18|1|2032|529|6895|Wednesday|2032Q1|N|N|N|2463264|2463294|2462916|2463189|N|N|N|N|N| +2463282|AAAAAAAACDGJFCAA|2032-02-19|1585|6895|529|2032|4|2|19|1|2032|529|6895|Thursday|2032Q1|N|N|N|2463264|2463294|2462917|2463190|N|N|N|N|N| +2463283|AAAAAAAADDGJFCAA|2032-02-20|1585|6895|529|2032|5|2|20|1|2032|529|6895|Friday|2032Q1|N|Y|N|2463264|2463294|2462918|2463191|N|N|N|N|N| +2463284|AAAAAAAAEDGJFCAA|2032-02-21|1585|6895|529|2032|6|2|21|1|2032|529|6895|Saturday|2032Q1|N|Y|N|2463264|2463294|2462919|2463192|N|N|N|N|N| +2463285|AAAAAAAAFDGJFCAA|2032-02-22|1585|6895|529|2032|0|2|22|1|2032|529|6895|Sunday|2032Q1|N|N|N|2463264|2463294|2462920|2463193|N|N|N|N|N| +2463286|AAAAAAAAGDGJFCAA|2032-02-23|1585|6895|529|2032|1|2|23|1|2032|529|6895|Monday|2032Q1|N|N|N|2463264|2463294|2462921|2463194|N|N|N|N|N| +2463287|AAAAAAAAHDGJFCAA|2032-02-24|1585|6896|529|2032|2|2|24|1|2032|529|6896|Tuesday|2032Q1|N|N|N|2463264|2463294|2462922|2463195|N|N|N|N|N| +2463288|AAAAAAAAIDGJFCAA|2032-02-25|1585|6896|529|2032|3|2|25|1|2032|529|6896|Wednesday|2032Q1|N|N|N|2463264|2463294|2462923|2463196|N|N|N|N|N| +2463289|AAAAAAAAJDGJFCAA|2032-02-26|1585|6896|529|2032|4|2|26|1|2032|529|6896|Thursday|2032Q1|N|N|N|2463264|2463294|2462924|2463197|N|N|N|N|N| +2463290|AAAAAAAAKDGJFCAA|2032-02-27|1585|6896|529|2032|5|2|27|1|2032|529|6896|Friday|2032Q1|N|Y|N|2463264|2463294|2462925|2463198|N|N|N|N|N| +2463291|AAAAAAAALDGJFCAA|2032-02-28|1585|6896|529|2032|6|2|28|1|2032|529|6896|Saturday|2032Q1|N|Y|N|2463264|2463294|2462926|2463199|N|N|N|N|N| +2463292|AAAAAAAAMDGJFCAA|2032-02-29|1585|6896|529|2032|0|2|29|1|2032|529|6896|Sunday|2032Q1|N|N|N|2463264|2463294|2462926|2463200|N|N|N|N|N| +2463293|AAAAAAAANDGJFCAA|2032-03-01|1586|6896|530|2032|1|3|1|1|2032|530|6896|Monday|2032Q1|N|N|N|2463293|2463352|2462927|2463201|N|N|N|N|N| +2463294|AAAAAAAAODGJFCAA|2032-03-02|1586|6897|530|2032|2|3|2|1|2032|530|6897|Tuesday|2032Q1|N|N|N|2463293|2463352|2462928|2463202|N|N|N|N|N| +2463295|AAAAAAAAPDGJFCAA|2032-03-03|1586|6897|530|2032|3|3|3|1|2032|530|6897|Wednesday|2032Q1|N|N|N|2463293|2463352|2462929|2463203|N|N|N|N|N| +2463296|AAAAAAAAAEGJFCAA|2032-03-04|1586|6897|530|2032|4|3|4|1|2032|530|6897|Thursday|2032Q1|N|N|N|2463293|2463352|2462930|2463204|N|N|N|N|N| +2463297|AAAAAAAABEGJFCAA|2032-03-05|1586|6897|530|2032|5|3|5|1|2032|530|6897|Friday|2032Q1|N|Y|N|2463293|2463352|2462931|2463205|N|N|N|N|N| +2463298|AAAAAAAACEGJFCAA|2032-03-06|1586|6897|530|2032|6|3|6|1|2032|530|6897|Saturday|2032Q1|N|Y|N|2463293|2463352|2462932|2463206|N|N|N|N|N| +2463299|AAAAAAAADEGJFCAA|2032-03-07|1586|6897|530|2032|0|3|7|1|2032|530|6897|Sunday|2032Q1|N|N|N|2463293|2463352|2462933|2463207|N|N|N|N|N| +2463300|AAAAAAAAEEGJFCAA|2032-03-08|1586|6897|530|2032|1|3|8|1|2032|530|6897|Monday|2032Q1|N|N|N|2463293|2463352|2462934|2463208|N|N|N|N|N| +2463301|AAAAAAAAFEGJFCAA|2032-03-09|1586|6898|530|2032|2|3|9|1|2032|530|6898|Tuesday|2032Q1|N|N|N|2463293|2463352|2462935|2463209|N|N|N|N|N| +2463302|AAAAAAAAGEGJFCAA|2032-03-10|1586|6898|530|2032|3|3|10|1|2032|530|6898|Wednesday|2032Q1|N|N|N|2463293|2463352|2462936|2463210|N|N|N|N|N| +2463303|AAAAAAAAHEGJFCAA|2032-03-11|1586|6898|530|2032|4|3|11|1|2032|530|6898|Thursday|2032Q1|N|N|N|2463293|2463352|2462937|2463211|N|N|N|N|N| +2463304|AAAAAAAAIEGJFCAA|2032-03-12|1586|6898|530|2032|5|3|12|1|2032|530|6898|Friday|2032Q1|N|Y|N|2463293|2463352|2462938|2463212|N|N|N|N|N| +2463305|AAAAAAAAJEGJFCAA|2032-03-13|1586|6898|530|2032|6|3|13|1|2032|530|6898|Saturday|2032Q1|N|Y|N|2463293|2463352|2462939|2463213|N|N|N|N|N| +2463306|AAAAAAAAKEGJFCAA|2032-03-14|1586|6898|530|2032|0|3|14|1|2032|530|6898|Sunday|2032Q1|N|N|N|2463293|2463352|2462940|2463214|N|N|N|N|N| +2463307|AAAAAAAALEGJFCAA|2032-03-15|1586|6898|530|2032|1|3|15|1|2032|530|6898|Monday|2032Q1|N|N|N|2463293|2463352|2462941|2463215|N|N|N|N|N| +2463308|AAAAAAAAMEGJFCAA|2032-03-16|1586|6899|530|2032|2|3|16|1|2032|530|6899|Tuesday|2032Q1|N|N|N|2463293|2463352|2462942|2463216|N|N|N|N|N| +2463309|AAAAAAAANEGJFCAA|2032-03-17|1586|6899|530|2032|3|3|17|1|2032|530|6899|Wednesday|2032Q1|N|N|N|2463293|2463352|2462943|2463217|N|N|N|N|N| +2463310|AAAAAAAAOEGJFCAA|2032-03-18|1586|6899|530|2032|4|3|18|1|2032|530|6899|Thursday|2032Q1|N|N|N|2463293|2463352|2462944|2463218|N|N|N|N|N| +2463311|AAAAAAAAPEGJFCAA|2032-03-19|1586|6899|530|2032|5|3|19|1|2032|530|6899|Friday|2032Q1|N|Y|N|2463293|2463352|2462945|2463219|N|N|N|N|N| +2463312|AAAAAAAAAFGJFCAA|2032-03-20|1586|6899|530|2032|6|3|20|1|2032|530|6899|Saturday|2032Q1|N|Y|N|2463293|2463352|2462946|2463220|N|N|N|N|N| +2463313|AAAAAAAABFGJFCAA|2032-03-21|1586|6899|530|2032|0|3|21|1|2032|530|6899|Sunday|2032Q1|N|N|N|2463293|2463352|2462947|2463221|N|N|N|N|N| +2463314|AAAAAAAACFGJFCAA|2032-03-22|1586|6899|530|2032|1|3|22|1|2032|530|6899|Monday|2032Q1|N|N|N|2463293|2463352|2462948|2463222|N|N|N|N|N| +2463315|AAAAAAAADFGJFCAA|2032-03-23|1586|6900|530|2032|2|3|23|1|2032|530|6900|Tuesday|2032Q1|N|N|N|2463293|2463352|2462949|2463223|N|N|N|N|N| +2463316|AAAAAAAAEFGJFCAA|2032-03-24|1586|6900|530|2032|3|3|24|1|2032|530|6900|Wednesday|2032Q1|N|N|N|2463293|2463352|2462950|2463224|N|N|N|N|N| +2463317|AAAAAAAAFFGJFCAA|2032-03-25|1586|6900|530|2032|4|3|25|1|2032|530|6900|Thursday|2032Q1|N|N|N|2463293|2463352|2462951|2463225|N|N|N|N|N| +2463318|AAAAAAAAGFGJFCAA|2032-03-26|1586|6900|530|2032|5|3|26|1|2032|530|6900|Friday|2032Q1|N|Y|N|2463293|2463352|2462952|2463226|N|N|N|N|N| +2463319|AAAAAAAAHFGJFCAA|2032-03-27|1586|6900|530|2032|6|3|27|1|2032|530|6900|Saturday|2032Q1|N|Y|N|2463293|2463352|2462953|2463227|N|N|N|N|N| +2463320|AAAAAAAAIFGJFCAA|2032-03-28|1586|6900|530|2032|0|3|28|1|2032|530|6900|Sunday|2032Q1|N|N|N|2463293|2463352|2462954|2463228|N|N|N|N|N| +2463321|AAAAAAAAJFGJFCAA|2032-03-29|1586|6900|530|2032|1|3|29|1|2032|530|6900|Monday|2032Q1|N|N|N|2463293|2463352|2462955|2463229|N|N|N|N|N| +2463322|AAAAAAAAKFGJFCAA|2032-03-30|1586|6901|530|2032|2|3|30|1|2032|530|6901|Tuesday|2032Q1|N|N|N|2463293|2463352|2462956|2463230|N|N|N|N|N| +2463323|AAAAAAAALFGJFCAA|2032-03-31|1586|6901|530|2032|3|3|31|1|2032|530|6901|Wednesday|2032Q1|N|N|N|2463293|2463352|2462957|2463231|N|N|N|N|N| +2463324|AAAAAAAAMFGJFCAA|2032-04-01|1587|6901|530|2032|4|4|1|2|2032|530|6901|Thursday|2032Q2|N|N|N|2463324|2463414|2462958|2463233|N|N|N|N|N| +2463325|AAAAAAAANFGJFCAA|2032-04-02|1587|6901|530|2032|5|4|2|2|2032|530|6901|Friday|2032Q2|N|Y|N|2463324|2463414|2462959|2463234|N|N|N|N|N| +2463326|AAAAAAAAOFGJFCAA|2032-04-03|1587|6901|530|2032|6|4|3|2|2032|530|6901|Saturday|2032Q2|N|Y|N|2463324|2463414|2462960|2463235|N|N|N|N|N| +2463327|AAAAAAAAPFGJFCAA|2032-04-04|1587|6901|530|2032|0|4|4|2|2032|530|6901|Sunday|2032Q2|N|N|N|2463324|2463414|2462961|2463236|N|N|N|N|N| +2463328|AAAAAAAAAGGJFCAA|2032-04-05|1587|6901|530|2032|1|4|5|2|2032|530|6901|Monday|2032Q2|N|N|N|2463324|2463414|2462962|2463237|N|N|N|N|N| +2463329|AAAAAAAABGGJFCAA|2032-04-06|1587|6902|530|2032|2|4|6|2|2032|530|6902|Tuesday|2032Q2|N|N|N|2463324|2463414|2462963|2463238|N|N|N|N|N| +2463330|AAAAAAAACGGJFCAA|2032-04-07|1587|6902|530|2032|3|4|7|2|2032|530|6902|Wednesday|2032Q2|N|N|N|2463324|2463414|2462964|2463239|N|N|N|N|N| +2463331|AAAAAAAADGGJFCAA|2032-04-08|1587|6902|530|2032|4|4|8|2|2032|530|6902|Thursday|2032Q2|N|N|N|2463324|2463414|2462965|2463240|N|N|N|N|N| +2463332|AAAAAAAAEGGJFCAA|2032-04-09|1587|6902|530|2032|5|4|9|2|2032|530|6902|Friday|2032Q2|N|Y|N|2463324|2463414|2462966|2463241|N|N|N|N|N| +2463333|AAAAAAAAFGGJFCAA|2032-04-10|1587|6902|530|2032|6|4|10|2|2032|530|6902|Saturday|2032Q2|N|Y|N|2463324|2463414|2462967|2463242|N|N|N|N|N| +2463334|AAAAAAAAGGGJFCAA|2032-04-11|1587|6902|530|2032|0|4|11|2|2032|530|6902|Sunday|2032Q2|N|N|N|2463324|2463414|2462968|2463243|N|N|N|N|N| +2463335|AAAAAAAAHGGJFCAA|2032-04-12|1587|6902|530|2032|1|4|12|2|2032|530|6902|Monday|2032Q2|N|N|N|2463324|2463414|2462969|2463244|N|N|N|N|N| +2463336|AAAAAAAAIGGJFCAA|2032-04-13|1587|6903|530|2032|2|4|13|2|2032|530|6903|Tuesday|2032Q2|N|N|N|2463324|2463414|2462970|2463245|N|N|N|N|N| +2463337|AAAAAAAAJGGJFCAA|2032-04-14|1587|6903|530|2032|3|4|14|2|2032|530|6903|Wednesday|2032Q2|N|N|N|2463324|2463414|2462971|2463246|N|N|N|N|N| +2463338|AAAAAAAAKGGJFCAA|2032-04-15|1587|6903|530|2032|4|4|15|2|2032|530|6903|Thursday|2032Q2|N|N|N|2463324|2463414|2462972|2463247|N|N|N|N|N| +2463339|AAAAAAAALGGJFCAA|2032-04-16|1587|6903|530|2032|5|4|16|2|2032|530|6903|Friday|2032Q2|N|Y|N|2463324|2463414|2462973|2463248|N|N|N|N|N| +2463340|AAAAAAAAMGGJFCAA|2032-04-17|1587|6903|530|2032|6|4|17|2|2032|530|6903|Saturday|2032Q2|N|Y|N|2463324|2463414|2462974|2463249|N|N|N|N|N| +2463341|AAAAAAAANGGJFCAA|2032-04-18|1587|6903|530|2032|0|4|18|2|2032|530|6903|Sunday|2032Q2|N|N|N|2463324|2463414|2462975|2463250|N|N|N|N|N| +2463342|AAAAAAAAOGGJFCAA|2032-04-19|1587|6903|530|2032|1|4|19|2|2032|530|6903|Monday|2032Q2|N|N|N|2463324|2463414|2462976|2463251|N|N|N|N|N| +2463343|AAAAAAAAPGGJFCAA|2032-04-20|1587|6904|530|2032|2|4|20|2|2032|530|6904|Tuesday|2032Q2|N|N|N|2463324|2463414|2462977|2463252|N|N|N|N|N| +2463344|AAAAAAAAAHGJFCAA|2032-04-21|1587|6904|530|2032|3|4|21|2|2032|530|6904|Wednesday|2032Q2|N|N|N|2463324|2463414|2462978|2463253|N|N|N|N|N| +2463345|AAAAAAAABHGJFCAA|2032-04-22|1587|6904|530|2032|4|4|22|2|2032|530|6904|Thursday|2032Q2|N|N|N|2463324|2463414|2462979|2463254|N|N|N|N|N| +2463346|AAAAAAAACHGJFCAA|2032-04-23|1587|6904|530|2032|5|4|23|2|2032|530|6904|Friday|2032Q2|N|Y|N|2463324|2463414|2462980|2463255|N|N|N|N|N| +2463347|AAAAAAAADHGJFCAA|2032-04-24|1587|6904|530|2032|6|4|24|2|2032|530|6904|Saturday|2032Q2|N|Y|N|2463324|2463414|2462981|2463256|N|N|N|N|N| +2463348|AAAAAAAAEHGJFCAA|2032-04-25|1587|6904|530|2032|0|4|25|2|2032|530|6904|Sunday|2032Q2|N|N|N|2463324|2463414|2462982|2463257|N|N|N|N|N| +2463349|AAAAAAAAFHGJFCAA|2032-04-26|1587|6904|530|2032|1|4|26|2|2032|530|6904|Monday|2032Q2|N|N|N|2463324|2463414|2462983|2463258|N|N|N|N|N| +2463350|AAAAAAAAGHGJFCAA|2032-04-27|1587|6905|530|2032|2|4|27|2|2032|530|6905|Tuesday|2032Q2|N|N|N|2463324|2463414|2462984|2463259|N|N|N|N|N| +2463351|AAAAAAAAHHGJFCAA|2032-04-28|1587|6905|530|2032|3|4|28|2|2032|530|6905|Wednesday|2032Q2|N|N|N|2463324|2463414|2462985|2463260|N|N|N|N|N| +2463352|AAAAAAAAIHGJFCAA|2032-04-29|1587|6905|530|2032|4|4|29|2|2032|530|6905|Thursday|2032Q2|N|N|N|2463324|2463414|2462986|2463261|N|N|N|N|N| +2463353|AAAAAAAAJHGJFCAA|2032-04-30|1587|6905|530|2032|5|4|30|2|2032|530|6905|Friday|2032Q2|N|Y|N|2463324|2463414|2462987|2463262|N|N|N|N|N| +2463354|AAAAAAAAKHGJFCAA|2032-05-01|1588|6905|530|2032|6|5|1|2|2032|530|6905|Saturday|2032Q2|N|Y|N|2463354|2463474|2462988|2463263|N|N|N|N|N| +2463355|AAAAAAAALHGJFCAA|2032-05-02|1588|6905|530|2032|0|5|2|2|2032|530|6905|Sunday|2032Q2|N|N|N|2463354|2463474|2462989|2463264|N|N|N|N|N| +2463356|AAAAAAAAMHGJFCAA|2032-05-03|1588|6905|530|2032|1|5|3|2|2032|530|6905|Monday|2032Q2|N|N|N|2463354|2463474|2462990|2463265|N|N|N|N|N| +2463357|AAAAAAAANHGJFCAA|2032-05-04|1588|6906|530|2032|2|5|4|2|2032|530|6906|Tuesday|2032Q2|N|N|N|2463354|2463474|2462991|2463266|N|N|N|N|N| +2463358|AAAAAAAAOHGJFCAA|2032-05-05|1588|6906|530|2032|3|5|5|2|2032|530|6906|Wednesday|2032Q2|N|N|N|2463354|2463474|2462992|2463267|N|N|N|N|N| +2463359|AAAAAAAAPHGJFCAA|2032-05-06|1588|6906|530|2032|4|5|6|2|2032|530|6906|Thursday|2032Q2|N|N|N|2463354|2463474|2462993|2463268|N|N|N|N|N| +2463360|AAAAAAAAAIGJFCAA|2032-05-07|1588|6906|530|2032|5|5|7|2|2032|530|6906|Friday|2032Q2|N|Y|N|2463354|2463474|2462994|2463269|N|N|N|N|N| +2463361|AAAAAAAABIGJFCAA|2032-05-08|1588|6906|530|2032|6|5|8|2|2032|530|6906|Saturday|2032Q2|N|Y|N|2463354|2463474|2462995|2463270|N|N|N|N|N| +2463362|AAAAAAAACIGJFCAA|2032-05-09|1588|6906|530|2032|0|5|9|2|2032|530|6906|Sunday|2032Q2|N|N|N|2463354|2463474|2462996|2463271|N|N|N|N|N| +2463363|AAAAAAAADIGJFCAA|2032-05-10|1588|6906|530|2032|1|5|10|2|2032|530|6906|Monday|2032Q2|N|N|N|2463354|2463474|2462997|2463272|N|N|N|N|N| +2463364|AAAAAAAAEIGJFCAA|2032-05-11|1588|6907|530|2032|2|5|11|2|2032|530|6907|Tuesday|2032Q2|N|N|N|2463354|2463474|2462998|2463273|N|N|N|N|N| +2463365|AAAAAAAAFIGJFCAA|2032-05-12|1588|6907|530|2032|3|5|12|2|2032|530|6907|Wednesday|2032Q2|N|N|N|2463354|2463474|2462999|2463274|N|N|N|N|N| +2463366|AAAAAAAAGIGJFCAA|2032-05-13|1588|6907|530|2032|4|5|13|2|2032|530|6907|Thursday|2032Q2|N|N|N|2463354|2463474|2463000|2463275|N|N|N|N|N| +2463367|AAAAAAAAHIGJFCAA|2032-05-14|1588|6907|530|2032|5|5|14|2|2032|530|6907|Friday|2032Q2|N|Y|N|2463354|2463474|2463001|2463276|N|N|N|N|N| +2463368|AAAAAAAAIIGJFCAA|2032-05-15|1588|6907|530|2032|6|5|15|2|2032|530|6907|Saturday|2032Q2|N|Y|N|2463354|2463474|2463002|2463277|N|N|N|N|N| +2463369|AAAAAAAAJIGJFCAA|2032-05-16|1588|6907|530|2032|0|5|16|2|2032|530|6907|Sunday|2032Q2|N|N|N|2463354|2463474|2463003|2463278|N|N|N|N|N| +2463370|AAAAAAAAKIGJFCAA|2032-05-17|1588|6907|530|2032|1|5|17|2|2032|530|6907|Monday|2032Q2|N|N|N|2463354|2463474|2463004|2463279|N|N|N|N|N| +2463371|AAAAAAAALIGJFCAA|2032-05-18|1588|6908|530|2032|2|5|18|2|2032|530|6908|Tuesday|2032Q2|N|N|N|2463354|2463474|2463005|2463280|N|N|N|N|N| +2463372|AAAAAAAAMIGJFCAA|2032-05-19|1588|6908|530|2032|3|5|19|2|2032|530|6908|Wednesday|2032Q2|N|N|N|2463354|2463474|2463006|2463281|N|N|N|N|N| +2463373|AAAAAAAANIGJFCAA|2032-05-20|1588|6908|530|2032|4|5|20|2|2032|530|6908|Thursday|2032Q2|N|N|N|2463354|2463474|2463007|2463282|N|N|N|N|N| +2463374|AAAAAAAAOIGJFCAA|2032-05-21|1588|6908|530|2032|5|5|21|2|2032|530|6908|Friday|2032Q2|N|Y|N|2463354|2463474|2463008|2463283|N|N|N|N|N| +2463375|AAAAAAAAPIGJFCAA|2032-05-22|1588|6908|530|2032|6|5|22|2|2032|530|6908|Saturday|2032Q2|N|Y|N|2463354|2463474|2463009|2463284|N|N|N|N|N| +2463376|AAAAAAAAAJGJFCAA|2032-05-23|1588|6908|530|2032|0|5|23|2|2032|530|6908|Sunday|2032Q2|N|N|N|2463354|2463474|2463010|2463285|N|N|N|N|N| +2463377|AAAAAAAABJGJFCAA|2032-05-24|1588|6908|530|2032|1|5|24|2|2032|530|6908|Monday|2032Q2|N|N|N|2463354|2463474|2463011|2463286|N|N|N|N|N| +2463378|AAAAAAAACJGJFCAA|2032-05-25|1588|6909|530|2032|2|5|25|2|2032|530|6909|Tuesday|2032Q2|N|N|N|2463354|2463474|2463012|2463287|N|N|N|N|N| +2463379|AAAAAAAADJGJFCAA|2032-05-26|1588|6909|530|2032|3|5|26|2|2032|530|6909|Wednesday|2032Q2|N|N|N|2463354|2463474|2463013|2463288|N|N|N|N|N| +2463380|AAAAAAAAEJGJFCAA|2032-05-27|1588|6909|530|2032|4|5|27|2|2032|530|6909|Thursday|2032Q2|N|N|N|2463354|2463474|2463014|2463289|N|N|N|N|N| +2463381|AAAAAAAAFJGJFCAA|2032-05-28|1588|6909|530|2032|5|5|28|2|2032|530|6909|Friday|2032Q2|N|Y|N|2463354|2463474|2463015|2463290|N|N|N|N|N| +2463382|AAAAAAAAGJGJFCAA|2032-05-29|1588|6909|530|2032|6|5|29|2|2032|530|6909|Saturday|2032Q2|N|Y|N|2463354|2463474|2463016|2463291|N|N|N|N|N| +2463383|AAAAAAAAHJGJFCAA|2032-05-30|1588|6909|530|2032|0|5|30|2|2032|530|6909|Sunday|2032Q2|N|N|N|2463354|2463474|2463017|2463292|N|N|N|N|N| +2463384|AAAAAAAAIJGJFCAA|2032-05-31|1588|6909|530|2032|1|5|31|2|2032|530|6909|Monday|2032Q2|N|N|N|2463354|2463474|2463018|2463293|N|N|N|N|N| +2463385|AAAAAAAAJJGJFCAA|2032-06-01|1589|6910|531|2032|2|6|1|2|2032|531|6910|Tuesday|2032Q2|N|N|N|2463385|2463536|2463019|2463294|N|N|N|N|N| +2463386|AAAAAAAAKJGJFCAA|2032-06-02|1589|6910|531|2032|3|6|2|2|2032|531|6910|Wednesday|2032Q2|N|N|N|2463385|2463536|2463020|2463295|N|N|N|N|N| +2463387|AAAAAAAALJGJFCAA|2032-06-03|1589|6910|531|2032|4|6|3|2|2032|531|6910|Thursday|2032Q2|N|N|N|2463385|2463536|2463021|2463296|N|N|N|N|N| +2463388|AAAAAAAAMJGJFCAA|2032-06-04|1589|6910|531|2032|5|6|4|2|2032|531|6910|Friday|2032Q2|N|Y|N|2463385|2463536|2463022|2463297|N|N|N|N|N| +2463389|AAAAAAAANJGJFCAA|2032-06-05|1589|6910|531|2032|6|6|5|2|2032|531|6910|Saturday|2032Q2|N|Y|N|2463385|2463536|2463023|2463298|N|N|N|N|N| +2463390|AAAAAAAAOJGJFCAA|2032-06-06|1589|6910|531|2032|0|6|6|2|2032|531|6910|Sunday|2032Q2|N|N|N|2463385|2463536|2463024|2463299|N|N|N|N|N| +2463391|AAAAAAAAPJGJFCAA|2032-06-07|1589|6910|531|2032|1|6|7|2|2032|531|6910|Monday|2032Q2|N|N|N|2463385|2463536|2463025|2463300|N|N|N|N|N| +2463392|AAAAAAAAAKGJFCAA|2032-06-08|1589|6911|531|2032|2|6|8|2|2032|531|6911|Tuesday|2032Q2|N|N|N|2463385|2463536|2463026|2463301|N|N|N|N|N| +2463393|AAAAAAAABKGJFCAA|2032-06-09|1589|6911|531|2032|3|6|9|2|2032|531|6911|Wednesday|2032Q2|N|N|N|2463385|2463536|2463027|2463302|N|N|N|N|N| +2463394|AAAAAAAACKGJFCAA|2032-06-10|1589|6911|531|2032|4|6|10|2|2032|531|6911|Thursday|2032Q2|N|N|N|2463385|2463536|2463028|2463303|N|N|N|N|N| +2463395|AAAAAAAADKGJFCAA|2032-06-11|1589|6911|531|2032|5|6|11|2|2032|531|6911|Friday|2032Q2|N|Y|N|2463385|2463536|2463029|2463304|N|N|N|N|N| +2463396|AAAAAAAAEKGJFCAA|2032-06-12|1589|6911|531|2032|6|6|12|2|2032|531|6911|Saturday|2032Q2|N|Y|N|2463385|2463536|2463030|2463305|N|N|N|N|N| +2463397|AAAAAAAAFKGJFCAA|2032-06-13|1589|6911|531|2032|0|6|13|2|2032|531|6911|Sunday|2032Q2|N|N|N|2463385|2463536|2463031|2463306|N|N|N|N|N| +2463398|AAAAAAAAGKGJFCAA|2032-06-14|1589|6911|531|2032|1|6|14|2|2032|531|6911|Monday|2032Q2|N|N|N|2463385|2463536|2463032|2463307|N|N|N|N|N| +2463399|AAAAAAAAHKGJFCAA|2032-06-15|1589|6912|531|2032|2|6|15|2|2032|531|6912|Tuesday|2032Q2|N|N|N|2463385|2463536|2463033|2463308|N|N|N|N|N| +2463400|AAAAAAAAIKGJFCAA|2032-06-16|1589|6912|531|2032|3|6|16|2|2032|531|6912|Wednesday|2032Q2|N|N|N|2463385|2463536|2463034|2463309|N|N|N|N|N| +2463401|AAAAAAAAJKGJFCAA|2032-06-17|1589|6912|531|2032|4|6|17|2|2032|531|6912|Thursday|2032Q2|N|N|N|2463385|2463536|2463035|2463310|N|N|N|N|N| +2463402|AAAAAAAAKKGJFCAA|2032-06-18|1589|6912|531|2032|5|6|18|2|2032|531|6912|Friday|2032Q2|N|Y|N|2463385|2463536|2463036|2463311|N|N|N|N|N| +2463403|AAAAAAAALKGJFCAA|2032-06-19|1589|6912|531|2032|6|6|19|2|2032|531|6912|Saturday|2032Q2|N|Y|N|2463385|2463536|2463037|2463312|N|N|N|N|N| +2463404|AAAAAAAAMKGJFCAA|2032-06-20|1589|6912|531|2032|0|6|20|2|2032|531|6912|Sunday|2032Q2|N|N|N|2463385|2463536|2463038|2463313|N|N|N|N|N| +2463405|AAAAAAAANKGJFCAA|2032-06-21|1589|6912|531|2032|1|6|21|2|2032|531|6912|Monday|2032Q2|N|N|N|2463385|2463536|2463039|2463314|N|N|N|N|N| +2463406|AAAAAAAAOKGJFCAA|2032-06-22|1589|6913|531|2032|2|6|22|2|2032|531|6913|Tuesday|2032Q2|N|N|N|2463385|2463536|2463040|2463315|N|N|N|N|N| +2463407|AAAAAAAAPKGJFCAA|2032-06-23|1589|6913|531|2032|3|6|23|2|2032|531|6913|Wednesday|2032Q2|N|N|N|2463385|2463536|2463041|2463316|N|N|N|N|N| +2463408|AAAAAAAAALGJFCAA|2032-06-24|1589|6913|531|2032|4|6|24|2|2032|531|6913|Thursday|2032Q2|N|N|N|2463385|2463536|2463042|2463317|N|N|N|N|N| +2463409|AAAAAAAABLGJFCAA|2032-06-25|1589|6913|531|2032|5|6|25|2|2032|531|6913|Friday|2032Q2|N|Y|N|2463385|2463536|2463043|2463318|N|N|N|N|N| +2463410|AAAAAAAACLGJFCAA|2032-06-26|1589|6913|531|2032|6|6|26|2|2032|531|6913|Saturday|2032Q2|N|Y|N|2463385|2463536|2463044|2463319|N|N|N|N|N| +2463411|AAAAAAAADLGJFCAA|2032-06-27|1589|6913|531|2032|0|6|27|2|2032|531|6913|Sunday|2032Q2|N|N|N|2463385|2463536|2463045|2463320|N|N|N|N|N| +2463412|AAAAAAAAELGJFCAA|2032-06-28|1589|6913|531|2032|1|6|28|2|2032|531|6913|Monday|2032Q2|N|N|N|2463385|2463536|2463046|2463321|N|N|N|N|N| +2463413|AAAAAAAAFLGJFCAA|2032-06-29|1589|6914|531|2032|2|6|29|2|2032|531|6914|Tuesday|2032Q2|N|N|N|2463385|2463536|2463047|2463322|N|N|N|N|N| +2463414|AAAAAAAAGLGJFCAA|2032-06-30|1589|6914|531|2032|3|6|30|2|2032|531|6914|Wednesday|2032Q2|N|N|N|2463385|2463536|2463048|2463323|N|N|N|N|N| +2463415|AAAAAAAAHLGJFCAA|2032-07-01|1590|6914|531|2032|4|7|1|3|2032|531|6914|Thursday|2032Q3|N|N|N|2463415|2463596|2463049|2463324|N|N|N|N|N| +2463416|AAAAAAAAILGJFCAA|2032-07-02|1590|6914|531|2032|5|7|2|3|2032|531|6914|Friday|2032Q3|N|Y|N|2463415|2463596|2463050|2463325|N|N|N|N|N| +2463417|AAAAAAAAJLGJFCAA|2032-07-03|1590|6914|531|2032|6|7|3|3|2032|531|6914|Saturday|2032Q3|N|Y|N|2463415|2463596|2463051|2463326|N|N|N|N|N| +2463418|AAAAAAAAKLGJFCAA|2032-07-04|1590|6914|531|2032|0|7|4|3|2032|531|6914|Sunday|2032Q3|Y|N|N|2463415|2463596|2463052|2463327|N|N|N|N|N| +2463419|AAAAAAAALLGJFCAA|2032-07-05|1590|6914|531|2032|1|7|5|3|2032|531|6914|Monday|2032Q3|N|N|Y|2463415|2463596|2463053|2463328|N|N|N|N|N| +2463420|AAAAAAAAMLGJFCAA|2032-07-06|1590|6915|531|2032|2|7|6|3|2032|531|6915|Tuesday|2032Q3|N|N|N|2463415|2463596|2463054|2463329|N|N|N|N|N| +2463421|AAAAAAAANLGJFCAA|2032-07-07|1590|6915|531|2032|3|7|7|3|2032|531|6915|Wednesday|2032Q3|N|N|N|2463415|2463596|2463055|2463330|N|N|N|N|N| +2463422|AAAAAAAAOLGJFCAA|2032-07-08|1590|6915|531|2032|4|7|8|3|2032|531|6915|Thursday|2032Q3|N|N|N|2463415|2463596|2463056|2463331|N|N|N|N|N| +2463423|AAAAAAAAPLGJFCAA|2032-07-09|1590|6915|531|2032|5|7|9|3|2032|531|6915|Friday|2032Q3|N|Y|N|2463415|2463596|2463057|2463332|N|N|N|N|N| +2463424|AAAAAAAAAMGJFCAA|2032-07-10|1590|6915|531|2032|6|7|10|3|2032|531|6915|Saturday|2032Q3|N|Y|N|2463415|2463596|2463058|2463333|N|N|N|N|N| +2463425|AAAAAAAABMGJFCAA|2032-07-11|1590|6915|531|2032|0|7|11|3|2032|531|6915|Sunday|2032Q3|N|N|N|2463415|2463596|2463059|2463334|N|N|N|N|N| +2463426|AAAAAAAACMGJFCAA|2032-07-12|1590|6915|531|2032|1|7|12|3|2032|531|6915|Monday|2032Q3|N|N|N|2463415|2463596|2463060|2463335|N|N|N|N|N| +2463427|AAAAAAAADMGJFCAA|2032-07-13|1590|6916|531|2032|2|7|13|3|2032|531|6916|Tuesday|2032Q3|N|N|N|2463415|2463596|2463061|2463336|N|N|N|N|N| +2463428|AAAAAAAAEMGJFCAA|2032-07-14|1590|6916|531|2032|3|7|14|3|2032|531|6916|Wednesday|2032Q3|N|N|N|2463415|2463596|2463062|2463337|N|N|N|N|N| +2463429|AAAAAAAAFMGJFCAA|2032-07-15|1590|6916|531|2032|4|7|15|3|2032|531|6916|Thursday|2032Q3|N|N|N|2463415|2463596|2463063|2463338|N|N|N|N|N| +2463430|AAAAAAAAGMGJFCAA|2032-07-16|1590|6916|531|2032|5|7|16|3|2032|531|6916|Friday|2032Q3|N|Y|N|2463415|2463596|2463064|2463339|N|N|N|N|N| +2463431|AAAAAAAAHMGJFCAA|2032-07-17|1590|6916|531|2032|6|7|17|3|2032|531|6916|Saturday|2032Q3|N|Y|N|2463415|2463596|2463065|2463340|N|N|N|N|N| +2463432|AAAAAAAAIMGJFCAA|2032-07-18|1590|6916|531|2032|0|7|18|3|2032|531|6916|Sunday|2032Q3|N|N|N|2463415|2463596|2463066|2463341|N|N|N|N|N| +2463433|AAAAAAAAJMGJFCAA|2032-07-19|1590|6916|531|2032|1|7|19|3|2032|531|6916|Monday|2032Q3|N|N|N|2463415|2463596|2463067|2463342|N|N|N|N|N| +2463434|AAAAAAAAKMGJFCAA|2032-07-20|1590|6917|531|2032|2|7|20|3|2032|531|6917|Tuesday|2032Q3|N|N|N|2463415|2463596|2463068|2463343|N|N|N|N|N| +2463435|AAAAAAAALMGJFCAA|2032-07-21|1590|6917|531|2032|3|7|21|3|2032|531|6917|Wednesday|2032Q3|N|N|N|2463415|2463596|2463069|2463344|N|N|N|N|N| +2463436|AAAAAAAAMMGJFCAA|2032-07-22|1590|6917|531|2032|4|7|22|3|2032|531|6917|Thursday|2032Q3|N|N|N|2463415|2463596|2463070|2463345|N|N|N|N|N| +2463437|AAAAAAAANMGJFCAA|2032-07-23|1590|6917|531|2032|5|7|23|3|2032|531|6917|Friday|2032Q3|N|Y|N|2463415|2463596|2463071|2463346|N|N|N|N|N| +2463438|AAAAAAAAOMGJFCAA|2032-07-24|1590|6917|531|2032|6|7|24|3|2032|531|6917|Saturday|2032Q3|N|Y|N|2463415|2463596|2463072|2463347|N|N|N|N|N| +2463439|AAAAAAAAPMGJFCAA|2032-07-25|1590|6917|531|2032|0|7|25|3|2032|531|6917|Sunday|2032Q3|N|N|N|2463415|2463596|2463073|2463348|N|N|N|N|N| +2463440|AAAAAAAAANGJFCAA|2032-07-26|1590|6917|531|2032|1|7|26|3|2032|531|6917|Monday|2032Q3|N|N|N|2463415|2463596|2463074|2463349|N|N|N|N|N| +2463441|AAAAAAAABNGJFCAA|2032-07-27|1590|6918|531|2032|2|7|27|3|2032|531|6918|Tuesday|2032Q3|N|N|N|2463415|2463596|2463075|2463350|N|N|N|N|N| +2463442|AAAAAAAACNGJFCAA|2032-07-28|1590|6918|531|2032|3|7|28|3|2032|531|6918|Wednesday|2032Q3|N|N|N|2463415|2463596|2463076|2463351|N|N|N|N|N| +2463443|AAAAAAAADNGJFCAA|2032-07-29|1590|6918|531|2032|4|7|29|3|2032|531|6918|Thursday|2032Q3|N|N|N|2463415|2463596|2463077|2463352|N|N|N|N|N| +2463444|AAAAAAAAENGJFCAA|2032-07-30|1590|6918|531|2032|5|7|30|3|2032|531|6918|Friday|2032Q3|N|Y|N|2463415|2463596|2463078|2463353|N|N|N|N|N| +2463445|AAAAAAAAFNGJFCAA|2032-07-31|1590|6918|531|2032|6|7|31|3|2032|531|6918|Saturday|2032Q3|N|Y|N|2463415|2463596|2463079|2463354|N|N|N|N|N| +2463446|AAAAAAAAGNGJFCAA|2032-08-01|1591|6918|531|2032|0|8|1|3|2032|531|6918|Sunday|2032Q3|N|N|N|2463446|2463658|2463080|2463355|N|N|N|N|N| +2463447|AAAAAAAAHNGJFCAA|2032-08-02|1591|6918|531|2032|1|8|2|3|2032|531|6918|Monday|2032Q3|N|N|N|2463446|2463658|2463081|2463356|N|N|N|N|N| +2463448|AAAAAAAAINGJFCAA|2032-08-03|1591|6919|531|2032|2|8|3|3|2032|531|6919|Tuesday|2032Q3|N|N|N|2463446|2463658|2463082|2463357|N|N|N|N|N| +2463449|AAAAAAAAJNGJFCAA|2032-08-04|1591|6919|531|2032|3|8|4|3|2032|531|6919|Wednesday|2032Q3|N|N|N|2463446|2463658|2463083|2463358|N|N|N|N|N| +2463450|AAAAAAAAKNGJFCAA|2032-08-05|1591|6919|531|2032|4|8|5|3|2032|531|6919|Thursday|2032Q3|N|N|N|2463446|2463658|2463084|2463359|N|N|N|N|N| +2463451|AAAAAAAALNGJFCAA|2032-08-06|1591|6919|531|2032|5|8|6|3|2032|531|6919|Friday|2032Q3|N|Y|N|2463446|2463658|2463085|2463360|N|N|N|N|N| +2463452|AAAAAAAAMNGJFCAA|2032-08-07|1591|6919|531|2032|6|8|7|3|2032|531|6919|Saturday|2032Q3|N|Y|N|2463446|2463658|2463086|2463361|N|N|N|N|N| +2463453|AAAAAAAANNGJFCAA|2032-08-08|1591|6919|531|2032|0|8|8|3|2032|531|6919|Sunday|2032Q3|N|N|N|2463446|2463658|2463087|2463362|N|N|N|N|N| +2463454|AAAAAAAAONGJFCAA|2032-08-09|1591|6919|531|2032|1|8|9|3|2032|531|6919|Monday|2032Q3|N|N|N|2463446|2463658|2463088|2463363|N|N|N|N|N| +2463455|AAAAAAAAPNGJFCAA|2032-08-10|1591|6920|531|2032|2|8|10|3|2032|531|6920|Tuesday|2032Q3|N|N|N|2463446|2463658|2463089|2463364|N|N|N|N|N| +2463456|AAAAAAAAAOGJFCAA|2032-08-11|1591|6920|531|2032|3|8|11|3|2032|531|6920|Wednesday|2032Q3|N|N|N|2463446|2463658|2463090|2463365|N|N|N|N|N| +2463457|AAAAAAAABOGJFCAA|2032-08-12|1591|6920|531|2032|4|8|12|3|2032|531|6920|Thursday|2032Q3|N|N|N|2463446|2463658|2463091|2463366|N|N|N|N|N| +2463458|AAAAAAAACOGJFCAA|2032-08-13|1591|6920|531|2032|5|8|13|3|2032|531|6920|Friday|2032Q3|N|Y|N|2463446|2463658|2463092|2463367|N|N|N|N|N| +2463459|AAAAAAAADOGJFCAA|2032-08-14|1591|6920|531|2032|6|8|14|3|2032|531|6920|Saturday|2032Q3|N|Y|N|2463446|2463658|2463093|2463368|N|N|N|N|N| +2463460|AAAAAAAAEOGJFCAA|2032-08-15|1591|6920|531|2032|0|8|15|3|2032|531|6920|Sunday|2032Q3|N|N|N|2463446|2463658|2463094|2463369|N|N|N|N|N| +2463461|AAAAAAAAFOGJFCAA|2032-08-16|1591|6920|531|2032|1|8|16|3|2032|531|6920|Monday|2032Q3|N|N|N|2463446|2463658|2463095|2463370|N|N|N|N|N| +2463462|AAAAAAAAGOGJFCAA|2032-08-17|1591|6921|531|2032|2|8|17|3|2032|531|6921|Tuesday|2032Q3|N|N|N|2463446|2463658|2463096|2463371|N|N|N|N|N| +2463463|AAAAAAAAHOGJFCAA|2032-08-18|1591|6921|531|2032|3|8|18|3|2032|531|6921|Wednesday|2032Q3|N|N|N|2463446|2463658|2463097|2463372|N|N|N|N|N| +2463464|AAAAAAAAIOGJFCAA|2032-08-19|1591|6921|531|2032|4|8|19|3|2032|531|6921|Thursday|2032Q3|N|N|N|2463446|2463658|2463098|2463373|N|N|N|N|N| +2463465|AAAAAAAAJOGJFCAA|2032-08-20|1591|6921|531|2032|5|8|20|3|2032|531|6921|Friday|2032Q3|N|Y|N|2463446|2463658|2463099|2463374|N|N|N|N|N| +2463466|AAAAAAAAKOGJFCAA|2032-08-21|1591|6921|531|2032|6|8|21|3|2032|531|6921|Saturday|2032Q3|N|Y|N|2463446|2463658|2463100|2463375|N|N|N|N|N| +2463467|AAAAAAAALOGJFCAA|2032-08-22|1591|6921|531|2032|0|8|22|3|2032|531|6921|Sunday|2032Q3|N|N|N|2463446|2463658|2463101|2463376|N|N|N|N|N| +2463468|AAAAAAAAMOGJFCAA|2032-08-23|1591|6921|531|2032|1|8|23|3|2032|531|6921|Monday|2032Q3|N|N|N|2463446|2463658|2463102|2463377|N|N|N|N|N| +2463469|AAAAAAAANOGJFCAA|2032-08-24|1591|6922|531|2032|2|8|24|3|2032|531|6922|Tuesday|2032Q3|N|N|N|2463446|2463658|2463103|2463378|N|N|N|N|N| +2463470|AAAAAAAAOOGJFCAA|2032-08-25|1591|6922|531|2032|3|8|25|3|2032|531|6922|Wednesday|2032Q3|N|N|N|2463446|2463658|2463104|2463379|N|N|N|N|N| +2463471|AAAAAAAAPOGJFCAA|2032-08-26|1591|6922|531|2032|4|8|26|3|2032|531|6922|Thursday|2032Q3|N|N|N|2463446|2463658|2463105|2463380|N|N|N|N|N| +2463472|AAAAAAAAAPGJFCAA|2032-08-27|1591|6922|531|2032|5|8|27|3|2032|531|6922|Friday|2032Q3|N|Y|N|2463446|2463658|2463106|2463381|N|N|N|N|N| +2463473|AAAAAAAABPGJFCAA|2032-08-28|1591|6922|531|2032|6|8|28|3|2032|531|6922|Saturday|2032Q3|N|Y|N|2463446|2463658|2463107|2463382|N|N|N|N|N| +2463474|AAAAAAAACPGJFCAA|2032-08-29|1591|6922|531|2032|0|8|29|3|2032|531|6922|Sunday|2032Q3|N|N|N|2463446|2463658|2463108|2463383|N|N|N|N|N| +2463475|AAAAAAAADPGJFCAA|2032-08-30|1591|6922|531|2032|1|8|30|3|2032|531|6922|Monday|2032Q3|N|N|N|2463446|2463658|2463109|2463384|N|N|N|N|N| +2463476|AAAAAAAAEPGJFCAA|2032-08-31|1591|6923|531|2032|2|8|31|3|2032|531|6923|Tuesday|2032Q3|N|N|N|2463446|2463658|2463110|2463385|N|N|N|N|N| +2463477|AAAAAAAAFPGJFCAA|2032-09-01|1592|6923|532|2032|3|9|1|3|2032|532|6923|Wednesday|2032Q3|N|N|N|2463477|2463720|2463111|2463386|N|N|N|N|N| +2463478|AAAAAAAAGPGJFCAA|2032-09-02|1592|6923|532|2032|4|9|2|3|2032|532|6923|Thursday|2032Q3|N|N|N|2463477|2463720|2463112|2463387|N|N|N|N|N| +2463479|AAAAAAAAHPGJFCAA|2032-09-03|1592|6923|532|2032|5|9|3|3|2032|532|6923|Friday|2032Q3|N|Y|N|2463477|2463720|2463113|2463388|N|N|N|N|N| +2463480|AAAAAAAAIPGJFCAA|2032-09-04|1592|6923|532|2032|6|9|4|3|2032|532|6923|Saturday|2032Q3|N|Y|N|2463477|2463720|2463114|2463389|N|N|N|N|N| +2463481|AAAAAAAAJPGJFCAA|2032-09-05|1592|6923|532|2032|0|9|5|3|2032|532|6923|Sunday|2032Q3|N|N|N|2463477|2463720|2463115|2463390|N|N|N|N|N| +2463482|AAAAAAAAKPGJFCAA|2032-09-06|1592|6923|532|2032|1|9|6|3|2032|532|6923|Monday|2032Q3|N|N|N|2463477|2463720|2463116|2463391|N|N|N|N|N| +2463483|AAAAAAAALPGJFCAA|2032-09-07|1592|6924|532|2032|2|9|7|3|2032|532|6924|Tuesday|2032Q3|N|N|N|2463477|2463720|2463117|2463392|N|N|N|N|N| +2463484|AAAAAAAAMPGJFCAA|2032-09-08|1592|6924|532|2032|3|9|8|3|2032|532|6924|Wednesday|2032Q3|N|N|N|2463477|2463720|2463118|2463393|N|N|N|N|N| +2463485|AAAAAAAANPGJFCAA|2032-09-09|1592|6924|532|2032|4|9|9|3|2032|532|6924|Thursday|2032Q3|N|N|N|2463477|2463720|2463119|2463394|N|N|N|N|N| +2463486|AAAAAAAAOPGJFCAA|2032-09-10|1592|6924|532|2032|5|9|10|3|2032|532|6924|Friday|2032Q3|N|Y|N|2463477|2463720|2463120|2463395|N|N|N|N|N| +2463487|AAAAAAAAPPGJFCAA|2032-09-11|1592|6924|532|2032|6|9|11|3|2032|532|6924|Saturday|2032Q3|N|Y|N|2463477|2463720|2463121|2463396|N|N|N|N|N| +2463488|AAAAAAAAAAHJFCAA|2032-09-12|1592|6924|532|2032|0|9|12|3|2032|532|6924|Sunday|2032Q3|N|N|N|2463477|2463720|2463122|2463397|N|N|N|N|N| +2463489|AAAAAAAABAHJFCAA|2032-09-13|1592|6924|532|2032|1|9|13|3|2032|532|6924|Monday|2032Q3|N|N|N|2463477|2463720|2463123|2463398|N|N|N|N|N| +2463490|AAAAAAAACAHJFCAA|2032-09-14|1592|6925|532|2032|2|9|14|3|2032|532|6925|Tuesday|2032Q3|N|N|N|2463477|2463720|2463124|2463399|N|N|N|N|N| +2463491|AAAAAAAADAHJFCAA|2032-09-15|1592|6925|532|2032|3|9|15|3|2032|532|6925|Wednesday|2032Q3|N|N|N|2463477|2463720|2463125|2463400|N|N|N|N|N| +2463492|AAAAAAAAEAHJFCAA|2032-09-16|1592|6925|532|2032|4|9|16|3|2032|532|6925|Thursday|2032Q3|N|N|N|2463477|2463720|2463126|2463401|N|N|N|N|N| +2463493|AAAAAAAAFAHJFCAA|2032-09-17|1592|6925|532|2032|5|9|17|3|2032|532|6925|Friday|2032Q3|N|Y|N|2463477|2463720|2463127|2463402|N|N|N|N|N| +2463494|AAAAAAAAGAHJFCAA|2032-09-18|1592|6925|532|2032|6|9|18|3|2032|532|6925|Saturday|2032Q3|N|Y|N|2463477|2463720|2463128|2463403|N|N|N|N|N| +2463495|AAAAAAAAHAHJFCAA|2032-09-19|1592|6925|532|2032|0|9|19|3|2032|532|6925|Sunday|2032Q3|N|N|N|2463477|2463720|2463129|2463404|N|N|N|N|N| +2463496|AAAAAAAAIAHJFCAA|2032-09-20|1592|6925|532|2032|1|9|20|3|2032|532|6925|Monday|2032Q3|N|N|N|2463477|2463720|2463130|2463405|N|N|N|N|N| +2463497|AAAAAAAAJAHJFCAA|2032-09-21|1592|6926|532|2032|2|9|21|3|2032|532|6926|Tuesday|2032Q3|N|N|N|2463477|2463720|2463131|2463406|N|N|N|N|N| +2463498|AAAAAAAAKAHJFCAA|2032-09-22|1592|6926|532|2032|3|9|22|3|2032|532|6926|Wednesday|2032Q3|N|N|N|2463477|2463720|2463132|2463407|N|N|N|N|N| +2463499|AAAAAAAALAHJFCAA|2032-09-23|1592|6926|532|2032|4|9|23|3|2032|532|6926|Thursday|2032Q3|N|N|N|2463477|2463720|2463133|2463408|N|N|N|N|N| +2463500|AAAAAAAAMAHJFCAA|2032-09-24|1592|6926|532|2032|5|9|24|3|2032|532|6926|Friday|2032Q3|N|Y|N|2463477|2463720|2463134|2463409|N|N|N|N|N| +2463501|AAAAAAAANAHJFCAA|2032-09-25|1592|6926|532|2032|6|9|25|3|2032|532|6926|Saturday|2032Q3|N|Y|N|2463477|2463720|2463135|2463410|N|N|N|N|N| +2463502|AAAAAAAAOAHJFCAA|2032-09-26|1592|6926|532|2032|0|9|26|3|2032|532|6926|Sunday|2032Q3|N|N|N|2463477|2463720|2463136|2463411|N|N|N|N|N| +2463503|AAAAAAAAPAHJFCAA|2032-09-27|1592|6926|532|2032|1|9|27|3|2032|532|6926|Monday|2032Q3|N|N|N|2463477|2463720|2463137|2463412|N|N|N|N|N| +2463504|AAAAAAAAABHJFCAA|2032-09-28|1592|6927|532|2032|2|9|28|3|2032|532|6927|Tuesday|2032Q3|N|N|N|2463477|2463720|2463138|2463413|N|N|N|N|N| +2463505|AAAAAAAABBHJFCAA|2032-09-29|1592|6927|532|2032|3|9|29|3|2032|532|6927|Wednesday|2032Q3|N|N|N|2463477|2463720|2463139|2463414|N|N|N|N|N| +2463506|AAAAAAAACBHJFCAA|2032-09-30|1592|6927|532|2032|4|9|30|3|2032|532|6927|Thursday|2032Q3|N|N|N|2463477|2463720|2463140|2463415|N|N|N|N|N| +2463507|AAAAAAAADBHJFCAA|2032-10-01|1593|6927|532|2032|5|10|1|4|2032|532|6927|Friday|2032Q4|N|Y|N|2463507|2463780|2463141|2463415|N|N|N|N|N| +2463508|AAAAAAAAEBHJFCAA|2032-10-02|1593|6927|532|2032|6|10|2|4|2032|532|6927|Saturday|2032Q4|N|Y|N|2463507|2463780|2463142|2463416|N|N|N|N|N| +2463509|AAAAAAAAFBHJFCAA|2032-10-03|1593|6927|532|2032|0|10|3|4|2032|532|6927|Sunday|2032Q4|N|N|N|2463507|2463780|2463143|2463417|N|N|N|N|N| +2463510|AAAAAAAAGBHJFCAA|2032-10-04|1593|6927|532|2032|1|10|4|4|2032|532|6927|Monday|2032Q4|N|N|N|2463507|2463780|2463144|2463418|N|N|N|N|N| +2463511|AAAAAAAAHBHJFCAA|2032-10-05|1593|6928|532|2032|2|10|5|4|2032|532|6928|Tuesday|2032Q4|N|N|N|2463507|2463780|2463145|2463419|N|N|N|N|N| +2463512|AAAAAAAAIBHJFCAA|2032-10-06|1593|6928|532|2032|3|10|6|4|2032|532|6928|Wednesday|2032Q4|N|N|N|2463507|2463780|2463146|2463420|N|N|N|N|N| +2463513|AAAAAAAAJBHJFCAA|2032-10-07|1593|6928|532|2032|4|10|7|4|2032|532|6928|Thursday|2032Q4|N|N|N|2463507|2463780|2463147|2463421|N|N|N|N|N| +2463514|AAAAAAAAKBHJFCAA|2032-10-08|1593|6928|532|2032|5|10|8|4|2032|532|6928|Friday|2032Q4|N|Y|N|2463507|2463780|2463148|2463422|N|N|N|N|N| +2463515|AAAAAAAALBHJFCAA|2032-10-09|1593|6928|532|2032|6|10|9|4|2032|532|6928|Saturday|2032Q4|N|Y|N|2463507|2463780|2463149|2463423|N|N|N|N|N| +2463516|AAAAAAAAMBHJFCAA|2032-10-10|1593|6928|532|2032|0|10|10|4|2032|532|6928|Sunday|2032Q4|N|N|N|2463507|2463780|2463150|2463424|N|N|N|N|N| +2463517|AAAAAAAANBHJFCAA|2032-10-11|1593|6928|532|2032|1|10|11|4|2032|532|6928|Monday|2032Q4|N|N|N|2463507|2463780|2463151|2463425|N|N|N|N|N| +2463518|AAAAAAAAOBHJFCAA|2032-10-12|1593|6929|532|2032|2|10|12|4|2032|532|6929|Tuesday|2032Q4|N|N|N|2463507|2463780|2463152|2463426|N|N|N|N|N| +2463519|AAAAAAAAPBHJFCAA|2032-10-13|1593|6929|532|2032|3|10|13|4|2032|532|6929|Wednesday|2032Q4|N|N|N|2463507|2463780|2463153|2463427|N|N|N|N|N| +2463520|AAAAAAAAACHJFCAA|2032-10-14|1593|6929|532|2032|4|10|14|4|2032|532|6929|Thursday|2032Q4|N|N|N|2463507|2463780|2463154|2463428|N|N|N|N|N| +2463521|AAAAAAAABCHJFCAA|2032-10-15|1593|6929|532|2032|5|10|15|4|2032|532|6929|Friday|2032Q4|N|Y|N|2463507|2463780|2463155|2463429|N|N|N|N|N| +2463522|AAAAAAAACCHJFCAA|2032-10-16|1593|6929|532|2032|6|10|16|4|2032|532|6929|Saturday|2032Q4|N|Y|N|2463507|2463780|2463156|2463430|N|N|N|N|N| +2463523|AAAAAAAADCHJFCAA|2032-10-17|1593|6929|532|2032|0|10|17|4|2032|532|6929|Sunday|2032Q4|N|N|N|2463507|2463780|2463157|2463431|N|N|N|N|N| +2463524|AAAAAAAAECHJFCAA|2032-10-18|1593|6929|532|2032|1|10|18|4|2032|532|6929|Monday|2032Q4|N|N|N|2463507|2463780|2463158|2463432|N|N|N|N|N| +2463525|AAAAAAAAFCHJFCAA|2032-10-19|1593|6930|532|2032|2|10|19|4|2032|532|6930|Tuesday|2032Q4|N|N|N|2463507|2463780|2463159|2463433|N|N|N|N|N| +2463526|AAAAAAAAGCHJFCAA|2032-10-20|1593|6930|532|2032|3|10|20|4|2032|532|6930|Wednesday|2032Q4|N|N|N|2463507|2463780|2463160|2463434|N|N|N|N|N| +2463527|AAAAAAAAHCHJFCAA|2032-10-21|1593|6930|532|2032|4|10|21|4|2032|532|6930|Thursday|2032Q4|N|N|N|2463507|2463780|2463161|2463435|N|N|N|N|N| +2463528|AAAAAAAAICHJFCAA|2032-10-22|1593|6930|532|2032|5|10|22|4|2032|532|6930|Friday|2032Q4|N|Y|N|2463507|2463780|2463162|2463436|N|N|N|N|N| +2463529|AAAAAAAAJCHJFCAA|2032-10-23|1593|6930|532|2032|6|10|23|4|2032|532|6930|Saturday|2032Q4|N|Y|N|2463507|2463780|2463163|2463437|N|N|N|N|N| +2463530|AAAAAAAAKCHJFCAA|2032-10-24|1593|6930|532|2032|0|10|24|4|2032|532|6930|Sunday|2032Q4|N|N|N|2463507|2463780|2463164|2463438|N|N|N|N|N| +2463531|AAAAAAAALCHJFCAA|2032-10-25|1593|6930|532|2032|1|10|25|4|2032|532|6930|Monday|2032Q4|N|N|N|2463507|2463780|2463165|2463439|N|N|N|N|N| +2463532|AAAAAAAAMCHJFCAA|2032-10-26|1593|6931|532|2032|2|10|26|4|2032|532|6931|Tuesday|2032Q4|N|N|N|2463507|2463780|2463166|2463440|N|N|N|N|N| +2463533|AAAAAAAANCHJFCAA|2032-10-27|1593|6931|532|2032|3|10|27|4|2032|532|6931|Wednesday|2032Q4|N|N|N|2463507|2463780|2463167|2463441|N|N|N|N|N| +2463534|AAAAAAAAOCHJFCAA|2032-10-28|1593|6931|532|2032|4|10|28|4|2032|532|6931|Thursday|2032Q4|N|N|N|2463507|2463780|2463168|2463442|N|N|N|N|N| +2463535|AAAAAAAAPCHJFCAA|2032-10-29|1593|6931|532|2032|5|10|29|4|2032|532|6931|Friday|2032Q4|N|Y|N|2463507|2463780|2463169|2463443|N|N|N|N|N| +2463536|AAAAAAAAADHJFCAA|2032-10-30|1593|6931|532|2032|6|10|30|4|2032|532|6931|Saturday|2032Q4|N|Y|N|2463507|2463780|2463170|2463444|N|N|N|N|N| +2463537|AAAAAAAABDHJFCAA|2032-10-31|1593|6931|532|2032|0|10|31|4|2032|532|6931|Sunday|2032Q4|N|N|N|2463507|2463780|2463171|2463445|N|N|N|N|N| +2463538|AAAAAAAACDHJFCAA|2032-11-01|1594|6931|532|2032|1|11|1|4|2032|532|6931|Monday|2032Q4|N|N|N|2463538|2463842|2463172|2463446|N|N|N|N|N| +2463539|AAAAAAAADDHJFCAA|2032-11-02|1594|6932|532|2032|2|11|2|4|2032|532|6932|Tuesday|2032Q4|N|N|N|2463538|2463842|2463173|2463447|N|N|N|N|N| +2463540|AAAAAAAAEDHJFCAA|2032-11-03|1594|6932|532|2032|3|11|3|4|2032|532|6932|Wednesday|2032Q4|N|N|N|2463538|2463842|2463174|2463448|N|N|N|N|N| +2463541|AAAAAAAAFDHJFCAA|2032-11-04|1594|6932|532|2032|4|11|4|4|2032|532|6932|Thursday|2032Q4|N|N|N|2463538|2463842|2463175|2463449|N|N|N|N|N| +2463542|AAAAAAAAGDHJFCAA|2032-11-05|1594|6932|532|2032|5|11|5|4|2032|532|6932|Friday|2032Q4|N|Y|N|2463538|2463842|2463176|2463450|N|N|N|N|N| +2463543|AAAAAAAAHDHJFCAA|2032-11-06|1594|6932|532|2032|6|11|6|4|2032|532|6932|Saturday|2032Q4|N|Y|N|2463538|2463842|2463177|2463451|N|N|N|N|N| +2463544|AAAAAAAAIDHJFCAA|2032-11-07|1594|6932|532|2032|0|11|7|4|2032|532|6932|Sunday|2032Q4|N|N|N|2463538|2463842|2463178|2463452|N|N|N|N|N| +2463545|AAAAAAAAJDHJFCAA|2032-11-08|1594|6932|532|2032|1|11|8|4|2032|532|6932|Monday|2032Q4|N|N|N|2463538|2463842|2463179|2463453|N|N|N|N|N| +2463546|AAAAAAAAKDHJFCAA|2032-11-09|1594|6933|532|2032|2|11|9|4|2032|532|6933|Tuesday|2032Q4|N|N|N|2463538|2463842|2463180|2463454|N|N|N|N|N| +2463547|AAAAAAAALDHJFCAA|2032-11-10|1594|6933|532|2032|3|11|10|4|2032|532|6933|Wednesday|2032Q4|N|N|N|2463538|2463842|2463181|2463455|N|N|N|N|N| +2463548|AAAAAAAAMDHJFCAA|2032-11-11|1594|6933|532|2032|4|11|11|4|2032|532|6933|Thursday|2032Q4|N|N|N|2463538|2463842|2463182|2463456|N|N|N|N|N| +2463549|AAAAAAAANDHJFCAA|2032-11-12|1594|6933|532|2032|5|11|12|4|2032|532|6933|Friday|2032Q4|N|Y|N|2463538|2463842|2463183|2463457|N|N|N|N|N| +2463550|AAAAAAAAODHJFCAA|2032-11-13|1594|6933|532|2032|6|11|13|4|2032|532|6933|Saturday|2032Q4|N|Y|N|2463538|2463842|2463184|2463458|N|N|N|N|N| +2463551|AAAAAAAAPDHJFCAA|2032-11-14|1594|6933|532|2032|0|11|14|4|2032|532|6933|Sunday|2032Q4|N|N|N|2463538|2463842|2463185|2463459|N|N|N|N|N| +2463552|AAAAAAAAAEHJFCAA|2032-11-15|1594|6933|532|2032|1|11|15|4|2032|532|6933|Monday|2032Q4|N|N|N|2463538|2463842|2463186|2463460|N|N|N|N|N| +2463553|AAAAAAAABEHJFCAA|2032-11-16|1594|6934|532|2032|2|11|16|4|2032|532|6934|Tuesday|2032Q4|N|N|N|2463538|2463842|2463187|2463461|N|N|N|N|N| +2463554|AAAAAAAACEHJFCAA|2032-11-17|1594|6934|532|2032|3|11|17|4|2032|532|6934|Wednesday|2032Q4|N|N|N|2463538|2463842|2463188|2463462|N|N|N|N|N| +2463555|AAAAAAAADEHJFCAA|2032-11-18|1594|6934|532|2032|4|11|18|4|2032|532|6934|Thursday|2032Q4|N|N|N|2463538|2463842|2463189|2463463|N|N|N|N|N| +2463556|AAAAAAAAEEHJFCAA|2032-11-19|1594|6934|532|2032|5|11|19|4|2032|532|6934|Friday|2032Q4|N|Y|N|2463538|2463842|2463190|2463464|N|N|N|N|N| +2463557|AAAAAAAAFEHJFCAA|2032-11-20|1594|6934|532|2032|6|11|20|4|2032|532|6934|Saturday|2032Q4|N|Y|N|2463538|2463842|2463191|2463465|N|N|N|N|N| +2463558|AAAAAAAAGEHJFCAA|2032-11-21|1594|6934|532|2032|0|11|21|4|2032|532|6934|Sunday|2032Q4|N|N|N|2463538|2463842|2463192|2463466|N|N|N|N|N| +2463559|AAAAAAAAHEHJFCAA|2032-11-22|1594|6934|532|2032|1|11|22|4|2032|532|6934|Monday|2032Q4|N|N|N|2463538|2463842|2463193|2463467|N|N|N|N|N| +2463560|AAAAAAAAIEHJFCAA|2032-11-23|1594|6935|532|2032|2|11|23|4|2032|532|6935|Tuesday|2032Q4|N|N|N|2463538|2463842|2463194|2463468|N|N|N|N|N| +2463561|AAAAAAAAJEHJFCAA|2032-11-24|1594|6935|532|2032|3|11|24|4|2032|532|6935|Wednesday|2032Q4|N|N|N|2463538|2463842|2463195|2463469|N|N|N|N|N| +2463562|AAAAAAAAKEHJFCAA|2032-11-25|1594|6935|532|2032|4|11|25|4|2032|532|6935|Thursday|2032Q4|N|N|N|2463538|2463842|2463196|2463470|N|N|N|N|N| +2463563|AAAAAAAALEHJFCAA|2032-11-26|1594|6935|532|2032|5|11|26|4|2032|532|6935|Friday|2032Q4|N|Y|N|2463538|2463842|2463197|2463471|N|N|N|N|N| +2463564|AAAAAAAAMEHJFCAA|2032-11-27|1594|6935|532|2032|6|11|27|4|2032|532|6935|Saturday|2032Q4|N|Y|N|2463538|2463842|2463198|2463472|N|N|N|N|N| +2463565|AAAAAAAANEHJFCAA|2032-11-28|1594|6935|532|2032|0|11|28|4|2032|532|6935|Sunday|2032Q4|N|N|N|2463538|2463842|2463199|2463473|N|N|N|N|N| +2463566|AAAAAAAAOEHJFCAA|2032-11-29|1594|6935|532|2032|1|11|29|4|2032|532|6935|Monday|2032Q4|N|N|N|2463538|2463842|2463200|2463474|N|N|N|N|N| +2463567|AAAAAAAAPEHJFCAA|2032-11-30|1594|6936|532|2032|2|11|30|4|2032|532|6936|Tuesday|2032Q4|N|N|N|2463538|2463842|2463201|2463475|N|N|N|N|N| +2463568|AAAAAAAAAFHJFCAA|2032-12-01|1595|6936|533|2032|3|12|1|4|2032|533|6936|Wednesday|2032Q4|N|N|N|2463568|2463902|2463202|2463476|N|N|N|N|N| +2463569|AAAAAAAABFHJFCAA|2032-12-02|1595|6936|533|2032|4|12|2|4|2032|533|6936|Thursday|2032Q4|N|N|N|2463568|2463902|2463203|2463477|N|N|N|N|N| +2463570|AAAAAAAACFHJFCAA|2032-12-03|1595|6936|533|2032|5|12|3|4|2032|533|6936|Friday|2032Q4|N|Y|N|2463568|2463902|2463204|2463478|N|N|N|N|N| +2463571|AAAAAAAADFHJFCAA|2032-12-04|1595|6936|533|2032|6|12|4|4|2032|533|6936|Saturday|2032Q4|N|Y|N|2463568|2463902|2463205|2463479|N|N|N|N|N| +2463572|AAAAAAAAEFHJFCAA|2032-12-05|1595|6936|533|2032|0|12|5|4|2032|533|6936|Sunday|2032Q4|N|N|N|2463568|2463902|2463206|2463480|N|N|N|N|N| +2463573|AAAAAAAAFFHJFCAA|2032-12-06|1595|6936|533|2032|1|12|6|4|2032|533|6936|Monday|2032Q4|N|N|N|2463568|2463902|2463207|2463481|N|N|N|N|N| +2463574|AAAAAAAAGFHJFCAA|2032-12-07|1595|6937|533|2032|2|12|7|4|2032|533|6937|Tuesday|2032Q4|N|N|N|2463568|2463902|2463208|2463482|N|N|N|N|N| +2463575|AAAAAAAAHFHJFCAA|2032-12-08|1595|6937|533|2032|3|12|8|4|2032|533|6937|Wednesday|2032Q4|N|N|N|2463568|2463902|2463209|2463483|N|N|N|N|N| +2463576|AAAAAAAAIFHJFCAA|2032-12-09|1595|6937|533|2032|4|12|9|4|2032|533|6937|Thursday|2032Q4|N|N|N|2463568|2463902|2463210|2463484|N|N|N|N|N| +2463577|AAAAAAAAJFHJFCAA|2032-12-10|1595|6937|533|2032|5|12|10|4|2032|533|6937|Friday|2032Q4|N|Y|N|2463568|2463902|2463211|2463485|N|N|N|N|N| +2463578|AAAAAAAAKFHJFCAA|2032-12-11|1595|6937|533|2032|6|12|11|4|2032|533|6937|Saturday|2032Q4|N|Y|N|2463568|2463902|2463212|2463486|N|N|N|N|N| +2463579|AAAAAAAALFHJFCAA|2032-12-12|1595|6937|533|2032|0|12|12|4|2032|533|6937|Sunday|2032Q4|N|N|N|2463568|2463902|2463213|2463487|N|N|N|N|N| +2463580|AAAAAAAAMFHJFCAA|2032-12-13|1595|6937|533|2032|1|12|13|4|2032|533|6937|Monday|2032Q4|N|N|N|2463568|2463902|2463214|2463488|N|N|N|N|N| +2463581|AAAAAAAANFHJFCAA|2032-12-14|1595|6938|533|2032|2|12|14|4|2032|533|6938|Tuesday|2032Q4|N|N|N|2463568|2463902|2463215|2463489|N|N|N|N|N| +2463582|AAAAAAAAOFHJFCAA|2032-12-15|1595|6938|533|2032|3|12|15|4|2032|533|6938|Wednesday|2032Q4|N|N|N|2463568|2463902|2463216|2463490|N|N|N|N|N| +2463583|AAAAAAAAPFHJFCAA|2032-12-16|1595|6938|533|2032|4|12|16|4|2032|533|6938|Thursday|2032Q4|N|N|N|2463568|2463902|2463217|2463491|N|N|N|N|N| +2463584|AAAAAAAAAGHJFCAA|2032-12-17|1595|6938|533|2032|5|12|17|4|2032|533|6938|Friday|2032Q4|N|Y|N|2463568|2463902|2463218|2463492|N|N|N|N|N| +2463585|AAAAAAAABGHJFCAA|2032-12-18|1595|6938|533|2032|6|12|18|4|2032|533|6938|Saturday|2032Q4|N|Y|N|2463568|2463902|2463219|2463493|N|N|N|N|N| +2463586|AAAAAAAACGHJFCAA|2032-12-19|1595|6938|533|2032|0|12|19|4|2032|533|6938|Sunday|2032Q4|N|N|N|2463568|2463902|2463220|2463494|N|N|N|N|N| +2463587|AAAAAAAADGHJFCAA|2032-12-20|1595|6938|533|2032|1|12|20|4|2032|533|6938|Monday|2032Q4|N|N|N|2463568|2463902|2463221|2463495|N|N|N|N|N| +2463588|AAAAAAAAEGHJFCAA|2032-12-21|1595|6939|533|2032|2|12|21|4|2032|533|6939|Tuesday|2032Q4|N|N|N|2463568|2463902|2463222|2463496|N|N|N|N|N| +2463589|AAAAAAAAFGHJFCAA|2032-12-22|1595|6939|533|2032|3|12|22|4|2032|533|6939|Wednesday|2032Q4|N|N|N|2463568|2463902|2463223|2463497|N|N|N|N|N| +2463590|AAAAAAAAGGHJFCAA|2032-12-23|1595|6939|533|2032|4|12|23|4|2032|533|6939|Thursday|2032Q4|N|N|N|2463568|2463902|2463224|2463498|N|N|N|N|N| +2463591|AAAAAAAAHGHJFCAA|2032-12-24|1595|6939|533|2032|5|12|24|4|2032|533|6939|Friday|2032Q4|N|Y|N|2463568|2463902|2463225|2463499|N|N|N|N|N| +2463592|AAAAAAAAIGHJFCAA|2032-12-25|1595|6939|533|2032|6|12|25|4|2032|533|6939|Saturday|2032Q4|Y|Y|N|2463568|2463902|2463226|2463500|N|N|N|N|N| +2463593|AAAAAAAAJGHJFCAA|2032-12-26|1595|6939|533|2032|0|12|26|4|2032|533|6939|Sunday|2032Q4|N|N|Y|2463568|2463902|2463227|2463501|N|N|N|N|N| +2463594|AAAAAAAAKGHJFCAA|2032-12-27|1595|6939|533|2032|1|12|27|4|2032|533|6939|Monday|2032Q4|N|N|N|2463568|2463902|2463228|2463502|N|N|N|N|N| +2463595|AAAAAAAALGHJFCAA|2032-12-28|1595|6940|533|2032|2|12|28|4|2032|533|6940|Tuesday|2032Q4|N|N|N|2463568|2463902|2463229|2463503|N|N|N|N|N| +2463596|AAAAAAAAMGHJFCAA|2032-12-29|1595|6940|533|2032|3|12|29|4|2032|533|6940|Wednesday|2032Q4|N|N|N|2463568|2463902|2463230|2463504|N|N|N|N|N| +2463597|AAAAAAAANGHJFCAA|2032-12-30|1595|6940|533|2032|4|12|30|4|2032|533|6940|Thursday|2032Q4|N|N|N|2463568|2463902|2463231|2463505|N|N|N|N|N| +2463598|AAAAAAAAOGHJFCAA|2032-12-31|1595|6940|533|2032|5|12|31|4|2032|533|6940|Friday|2032Q4|Y|Y|N|2463568|2463902|2463232|2463506|N|N|N|N|N| +2463599|AAAAAAAAPGHJFCAA|2033-01-01|1596|6940|533|2033|6|1|1|1|2033|533|6940|Saturday|2033Q1|Y|Y|Y|2463599|2463598|2463233|2463507|N|N|N|N|N| +2463600|AAAAAAAAAHHJFCAA|2033-01-02|1596|6940|533|2033|0|1|2|1|2033|533|6940|Sunday|2033Q1|N|N|Y|2463599|2463598|2463234|2463508|N|N|N|N|N| +2463601|AAAAAAAABHHJFCAA|2033-01-03|1596|6940|533|2033|1|1|3|1|2033|533|6940|Monday|2033Q1|N|N|N|2463599|2463598|2463235|2463509|N|N|N|N|N| +2463602|AAAAAAAACHHJFCAA|2033-01-04|1596|6941|533|2033|2|1|4|1|2033|533|6941|Tuesday|2033Q1|N|N|N|2463599|2463598|2463236|2463510|N|N|N|N|N| +2463603|AAAAAAAADHHJFCAA|2033-01-05|1596|6941|533|2033|3|1|5|1|2033|533|6941|Wednesday|2033Q1|N|N|N|2463599|2463598|2463237|2463511|N|N|N|N|N| +2463604|AAAAAAAAEHHJFCAA|2033-01-06|1596|6941|533|2033|4|1|6|1|2033|533|6941|Thursday|2033Q1|N|N|N|2463599|2463598|2463238|2463512|N|N|N|N|N| +2463605|AAAAAAAAFHHJFCAA|2033-01-07|1596|6941|533|2033|5|1|7|1|2033|533|6941|Friday|2033Q1|N|Y|N|2463599|2463598|2463239|2463513|N|N|N|N|N| +2463606|AAAAAAAAGHHJFCAA|2033-01-08|1596|6941|533|2033|6|1|8|1|2033|533|6941|Saturday|2033Q1|N|Y|N|2463599|2463598|2463240|2463514|N|N|N|N|N| +2463607|AAAAAAAAHHHJFCAA|2033-01-09|1596|6941|533|2033|0|1|9|1|2033|533|6941|Sunday|2033Q1|N|N|N|2463599|2463598|2463241|2463515|N|N|N|N|N| +2463608|AAAAAAAAIHHJFCAA|2033-01-10|1596|6941|533|2033|1|1|10|1|2033|533|6941|Monday|2033Q1|N|N|N|2463599|2463598|2463242|2463516|N|N|N|N|N| +2463609|AAAAAAAAJHHJFCAA|2033-01-11|1596|6942|533|2033|2|1|11|1|2033|533|6942|Tuesday|2033Q1|N|N|N|2463599|2463598|2463243|2463517|N|N|N|N|N| +2463610|AAAAAAAAKHHJFCAA|2033-01-12|1596|6942|533|2033|3|1|12|1|2033|533|6942|Wednesday|2033Q1|N|N|N|2463599|2463598|2463244|2463518|N|N|N|N|N| +2463611|AAAAAAAALHHJFCAA|2033-01-13|1596|6942|533|2033|4|1|13|1|2033|533|6942|Thursday|2033Q1|N|N|N|2463599|2463598|2463245|2463519|N|N|N|N|N| +2463612|AAAAAAAAMHHJFCAA|2033-01-14|1596|6942|533|2033|5|1|14|1|2033|533|6942|Friday|2033Q1|N|Y|N|2463599|2463598|2463246|2463520|N|N|N|N|N| +2463613|AAAAAAAANHHJFCAA|2033-01-15|1596|6942|533|2033|6|1|15|1|2033|533|6942|Saturday|2033Q1|N|Y|N|2463599|2463598|2463247|2463521|N|N|N|N|N| +2463614|AAAAAAAAOHHJFCAA|2033-01-16|1596|6942|533|2033|0|1|16|1|2033|533|6942|Sunday|2033Q1|N|N|N|2463599|2463598|2463248|2463522|N|N|N|N|N| +2463615|AAAAAAAAPHHJFCAA|2033-01-17|1596|6942|533|2033|1|1|17|1|2033|533|6942|Monday|2033Q1|N|N|N|2463599|2463598|2463249|2463523|N|N|N|N|N| +2463616|AAAAAAAAAIHJFCAA|2033-01-18|1596|6943|533|2033|2|1|18|1|2033|533|6943|Tuesday|2033Q1|N|N|N|2463599|2463598|2463250|2463524|N|N|N|N|N| +2463617|AAAAAAAABIHJFCAA|2033-01-19|1596|6943|533|2033|3|1|19|1|2033|533|6943|Wednesday|2033Q1|N|N|N|2463599|2463598|2463251|2463525|N|N|N|N|N| +2463618|AAAAAAAACIHJFCAA|2033-01-20|1596|6943|533|2033|4|1|20|1|2033|533|6943|Thursday|2033Q1|N|N|N|2463599|2463598|2463252|2463526|N|N|N|N|N| +2463619|AAAAAAAADIHJFCAA|2033-01-21|1596|6943|533|2033|5|1|21|1|2033|533|6943|Friday|2033Q1|N|Y|N|2463599|2463598|2463253|2463527|N|N|N|N|N| +2463620|AAAAAAAAEIHJFCAA|2033-01-22|1596|6943|533|2033|6|1|22|1|2033|533|6943|Saturday|2033Q1|N|Y|N|2463599|2463598|2463254|2463528|N|N|N|N|N| +2463621|AAAAAAAAFIHJFCAA|2033-01-23|1596|6943|533|2033|0|1|23|1|2033|533|6943|Sunday|2033Q1|N|N|N|2463599|2463598|2463255|2463529|N|N|N|N|N| +2463622|AAAAAAAAGIHJFCAA|2033-01-24|1596|6943|533|2033|1|1|24|1|2033|533|6943|Monday|2033Q1|N|N|N|2463599|2463598|2463256|2463530|N|N|N|N|N| +2463623|AAAAAAAAHIHJFCAA|2033-01-25|1596|6944|533|2033|2|1|25|1|2033|533|6944|Tuesday|2033Q1|N|N|N|2463599|2463598|2463257|2463531|N|N|N|N|N| +2463624|AAAAAAAAIIHJFCAA|2033-01-26|1596|6944|533|2033|3|1|26|1|2033|533|6944|Wednesday|2033Q1|N|N|N|2463599|2463598|2463258|2463532|N|N|N|N|N| +2463625|AAAAAAAAJIHJFCAA|2033-01-27|1596|6944|533|2033|4|1|27|1|2033|533|6944|Thursday|2033Q1|N|N|N|2463599|2463598|2463259|2463533|N|N|N|N|N| +2463626|AAAAAAAAKIHJFCAA|2033-01-28|1596|6944|533|2033|5|1|28|1|2033|533|6944|Friday|2033Q1|N|Y|N|2463599|2463598|2463260|2463534|N|N|N|N|N| +2463627|AAAAAAAALIHJFCAA|2033-01-29|1596|6944|533|2033|6|1|29|1|2033|533|6944|Saturday|2033Q1|N|Y|N|2463599|2463598|2463261|2463535|N|N|N|N|N| +2463628|AAAAAAAAMIHJFCAA|2033-01-30|1596|6944|533|2033|0|1|30|1|2033|533|6944|Sunday|2033Q1|N|N|N|2463599|2463598|2463262|2463536|N|N|N|N|N| +2463629|AAAAAAAANIHJFCAA|2033-01-31|1596|6944|533|2033|1|1|31|1|2033|533|6944|Monday|2033Q1|N|N|N|2463599|2463598|2463263|2463537|N|N|N|N|N| +2463630|AAAAAAAAOIHJFCAA|2033-02-01|1597|6945|533|2033|2|2|1|1|2033|533|6945|Tuesday|2033Q1|N|N|N|2463630|2463660|2463264|2463538|N|N|N|N|N| +2463631|AAAAAAAAPIHJFCAA|2033-02-02|1597|6945|533|2033|3|2|2|1|2033|533|6945|Wednesday|2033Q1|N|N|N|2463630|2463660|2463265|2463539|N|N|N|N|N| +2463632|AAAAAAAAAJHJFCAA|2033-02-03|1597|6945|533|2033|4|2|3|1|2033|533|6945|Thursday|2033Q1|N|N|N|2463630|2463660|2463266|2463540|N|N|N|N|N| +2463633|AAAAAAAABJHJFCAA|2033-02-04|1597|6945|533|2033|5|2|4|1|2033|533|6945|Friday|2033Q1|N|Y|N|2463630|2463660|2463267|2463541|N|N|N|N|N| +2463634|AAAAAAAACJHJFCAA|2033-02-05|1597|6945|533|2033|6|2|5|1|2033|533|6945|Saturday|2033Q1|N|Y|N|2463630|2463660|2463268|2463542|N|N|N|N|N| +2463635|AAAAAAAADJHJFCAA|2033-02-06|1597|6945|533|2033|0|2|6|1|2033|533|6945|Sunday|2033Q1|N|N|N|2463630|2463660|2463269|2463543|N|N|N|N|N| +2463636|AAAAAAAAEJHJFCAA|2033-02-07|1597|6945|533|2033|1|2|7|1|2033|533|6945|Monday|2033Q1|N|N|N|2463630|2463660|2463270|2463544|N|N|N|N|N| +2463637|AAAAAAAAFJHJFCAA|2033-02-08|1597|6946|533|2033|2|2|8|1|2033|533|6946|Tuesday|2033Q1|N|N|N|2463630|2463660|2463271|2463545|N|N|N|N|N| +2463638|AAAAAAAAGJHJFCAA|2033-02-09|1597|6946|533|2033|3|2|9|1|2033|533|6946|Wednesday|2033Q1|N|N|N|2463630|2463660|2463272|2463546|N|N|N|N|N| +2463639|AAAAAAAAHJHJFCAA|2033-02-10|1597|6946|533|2033|4|2|10|1|2033|533|6946|Thursday|2033Q1|N|N|N|2463630|2463660|2463273|2463547|N|N|N|N|N| +2463640|AAAAAAAAIJHJFCAA|2033-02-11|1597|6946|533|2033|5|2|11|1|2033|533|6946|Friday|2033Q1|N|Y|N|2463630|2463660|2463274|2463548|N|N|N|N|N| +2463641|AAAAAAAAJJHJFCAA|2033-02-12|1597|6946|533|2033|6|2|12|1|2033|533|6946|Saturday|2033Q1|N|Y|N|2463630|2463660|2463275|2463549|N|N|N|N|N| +2463642|AAAAAAAAKJHJFCAA|2033-02-13|1597|6946|533|2033|0|2|13|1|2033|533|6946|Sunday|2033Q1|N|N|N|2463630|2463660|2463276|2463550|N|N|N|N|N| +2463643|AAAAAAAALJHJFCAA|2033-02-14|1597|6946|533|2033|1|2|14|1|2033|533|6946|Monday|2033Q1|N|N|N|2463630|2463660|2463277|2463551|N|N|N|N|N| +2463644|AAAAAAAAMJHJFCAA|2033-02-15|1597|6947|533|2033|2|2|15|1|2033|533|6947|Tuesday|2033Q1|N|N|N|2463630|2463660|2463278|2463552|N|N|N|N|N| +2463645|AAAAAAAANJHJFCAA|2033-02-16|1597|6947|533|2033|3|2|16|1|2033|533|6947|Wednesday|2033Q1|N|N|N|2463630|2463660|2463279|2463553|N|N|N|N|N| +2463646|AAAAAAAAOJHJFCAA|2033-02-17|1597|6947|533|2033|4|2|17|1|2033|533|6947|Thursday|2033Q1|N|N|N|2463630|2463660|2463280|2463554|N|N|N|N|N| +2463647|AAAAAAAAPJHJFCAA|2033-02-18|1597|6947|533|2033|5|2|18|1|2033|533|6947|Friday|2033Q1|N|Y|N|2463630|2463660|2463281|2463555|N|N|N|N|N| +2463648|AAAAAAAAAKHJFCAA|2033-02-19|1597|6947|533|2033|6|2|19|1|2033|533|6947|Saturday|2033Q1|N|Y|N|2463630|2463660|2463282|2463556|N|N|N|N|N| +2463649|AAAAAAAABKHJFCAA|2033-02-20|1597|6947|533|2033|0|2|20|1|2033|533|6947|Sunday|2033Q1|N|N|N|2463630|2463660|2463283|2463557|N|N|N|N|N| +2463650|AAAAAAAACKHJFCAA|2033-02-21|1597|6947|533|2033|1|2|21|1|2033|533|6947|Monday|2033Q1|N|N|N|2463630|2463660|2463284|2463558|N|N|N|N|N| +2463651|AAAAAAAADKHJFCAA|2033-02-22|1597|6948|533|2033|2|2|22|1|2033|533|6948|Tuesday|2033Q1|N|N|N|2463630|2463660|2463285|2463559|N|N|N|N|N| +2463652|AAAAAAAAEKHJFCAA|2033-02-23|1597|6948|533|2033|3|2|23|1|2033|533|6948|Wednesday|2033Q1|N|N|N|2463630|2463660|2463286|2463560|N|N|N|N|N| +2463653|AAAAAAAAFKHJFCAA|2033-02-24|1597|6948|533|2033|4|2|24|1|2033|533|6948|Thursday|2033Q1|N|N|N|2463630|2463660|2463287|2463561|N|N|N|N|N| +2463654|AAAAAAAAGKHJFCAA|2033-02-25|1597|6948|533|2033|5|2|25|1|2033|533|6948|Friday|2033Q1|N|Y|N|2463630|2463660|2463288|2463562|N|N|N|N|N| +2463655|AAAAAAAAHKHJFCAA|2033-02-26|1597|6948|533|2033|6|2|26|1|2033|533|6948|Saturday|2033Q1|N|Y|N|2463630|2463660|2463289|2463563|N|N|N|N|N| +2463656|AAAAAAAAIKHJFCAA|2033-02-27|1597|6948|533|2033|0|2|27|1|2033|533|6948|Sunday|2033Q1|N|N|N|2463630|2463660|2463290|2463564|N|N|N|N|N| +2463657|AAAAAAAAJKHJFCAA|2033-02-28|1597|6948|533|2033|1|2|28|1|2033|533|6948|Monday|2033Q1|N|N|N|2463630|2463660|2463291|2463565|N|N|N|N|N| +2463658|AAAAAAAAKKHJFCAA|2033-03-01|1598|6949|534|2033|2|3|1|1|2033|534|6949|Tuesday|2033Q1|N|N|N|2463658|2463716|2463293|2463566|N|N|N|N|N| +2463659|AAAAAAAALKHJFCAA|2033-03-02|1598|6949|534|2033|3|3|2|1|2033|534|6949|Wednesday|2033Q1|N|N|N|2463658|2463716|2463294|2463567|N|N|N|N|N| +2463660|AAAAAAAAMKHJFCAA|2033-03-03|1598|6949|534|2033|4|3|3|1|2033|534|6949|Thursday|2033Q1|N|N|N|2463658|2463716|2463295|2463568|N|N|N|N|N| +2463661|AAAAAAAANKHJFCAA|2033-03-04|1598|6949|534|2033|5|3|4|1|2033|534|6949|Friday|2033Q1|N|Y|N|2463658|2463716|2463296|2463569|N|N|N|N|N| +2463662|AAAAAAAAOKHJFCAA|2033-03-05|1598|6949|534|2033|6|3|5|1|2033|534|6949|Saturday|2033Q1|N|Y|N|2463658|2463716|2463297|2463570|N|N|N|N|N| +2463663|AAAAAAAAPKHJFCAA|2033-03-06|1598|6949|534|2033|0|3|6|1|2033|534|6949|Sunday|2033Q1|N|N|N|2463658|2463716|2463298|2463571|N|N|N|N|N| +2463664|AAAAAAAAALHJFCAA|2033-03-07|1598|6949|534|2033|1|3|7|1|2033|534|6949|Monday|2033Q1|N|N|N|2463658|2463716|2463299|2463572|N|N|N|N|N| +2463665|AAAAAAAABLHJFCAA|2033-03-08|1598|6950|534|2033|2|3|8|1|2033|534|6950|Tuesday|2033Q1|N|N|N|2463658|2463716|2463300|2463573|N|N|N|N|N| +2463666|AAAAAAAACLHJFCAA|2033-03-09|1598|6950|534|2033|3|3|9|1|2033|534|6950|Wednesday|2033Q1|N|N|N|2463658|2463716|2463301|2463574|N|N|N|N|N| +2463667|AAAAAAAADLHJFCAA|2033-03-10|1598|6950|534|2033|4|3|10|1|2033|534|6950|Thursday|2033Q1|N|N|N|2463658|2463716|2463302|2463575|N|N|N|N|N| +2463668|AAAAAAAAELHJFCAA|2033-03-11|1598|6950|534|2033|5|3|11|1|2033|534|6950|Friday|2033Q1|N|Y|N|2463658|2463716|2463303|2463576|N|N|N|N|N| +2463669|AAAAAAAAFLHJFCAA|2033-03-12|1598|6950|534|2033|6|3|12|1|2033|534|6950|Saturday|2033Q1|N|Y|N|2463658|2463716|2463304|2463577|N|N|N|N|N| +2463670|AAAAAAAAGLHJFCAA|2033-03-13|1598|6950|534|2033|0|3|13|1|2033|534|6950|Sunday|2033Q1|N|N|N|2463658|2463716|2463305|2463578|N|N|N|N|N| +2463671|AAAAAAAAHLHJFCAA|2033-03-14|1598|6950|534|2033|1|3|14|1|2033|534|6950|Monday|2033Q1|N|N|N|2463658|2463716|2463306|2463579|N|N|N|N|N| +2463672|AAAAAAAAILHJFCAA|2033-03-15|1598|6951|534|2033|2|3|15|1|2033|534|6951|Tuesday|2033Q1|N|N|N|2463658|2463716|2463307|2463580|N|N|N|N|N| +2463673|AAAAAAAAJLHJFCAA|2033-03-16|1598|6951|534|2033|3|3|16|1|2033|534|6951|Wednesday|2033Q1|N|N|N|2463658|2463716|2463308|2463581|N|N|N|N|N| +2463674|AAAAAAAAKLHJFCAA|2033-03-17|1598|6951|534|2033|4|3|17|1|2033|534|6951|Thursday|2033Q1|N|N|N|2463658|2463716|2463309|2463582|N|N|N|N|N| +2463675|AAAAAAAALLHJFCAA|2033-03-18|1598|6951|534|2033|5|3|18|1|2033|534|6951|Friday|2033Q1|N|Y|N|2463658|2463716|2463310|2463583|N|N|N|N|N| +2463676|AAAAAAAAMLHJFCAA|2033-03-19|1598|6951|534|2033|6|3|19|1|2033|534|6951|Saturday|2033Q1|N|Y|N|2463658|2463716|2463311|2463584|N|N|N|N|N| +2463677|AAAAAAAANLHJFCAA|2033-03-20|1598|6951|534|2033|0|3|20|1|2033|534|6951|Sunday|2033Q1|N|N|N|2463658|2463716|2463312|2463585|N|N|N|N|N| +2463678|AAAAAAAAOLHJFCAA|2033-03-21|1598|6951|534|2033|1|3|21|1|2033|534|6951|Monday|2033Q1|N|N|N|2463658|2463716|2463313|2463586|N|N|N|N|N| +2463679|AAAAAAAAPLHJFCAA|2033-03-22|1598|6952|534|2033|2|3|22|1|2033|534|6952|Tuesday|2033Q1|N|N|N|2463658|2463716|2463314|2463587|N|N|N|N|N| +2463680|AAAAAAAAAMHJFCAA|2033-03-23|1598|6952|534|2033|3|3|23|1|2033|534|6952|Wednesday|2033Q1|N|N|N|2463658|2463716|2463315|2463588|N|N|N|N|N| +2463681|AAAAAAAABMHJFCAA|2033-03-24|1598|6952|534|2033|4|3|24|1|2033|534|6952|Thursday|2033Q1|N|N|N|2463658|2463716|2463316|2463589|N|N|N|N|N| +2463682|AAAAAAAACMHJFCAA|2033-03-25|1598|6952|534|2033|5|3|25|1|2033|534|6952|Friday|2033Q1|N|Y|N|2463658|2463716|2463317|2463590|N|N|N|N|N| +2463683|AAAAAAAADMHJFCAA|2033-03-26|1598|6952|534|2033|6|3|26|1|2033|534|6952|Saturday|2033Q1|N|Y|N|2463658|2463716|2463318|2463591|N|N|N|N|N| +2463684|AAAAAAAAEMHJFCAA|2033-03-27|1598|6952|534|2033|0|3|27|1|2033|534|6952|Sunday|2033Q1|N|N|N|2463658|2463716|2463319|2463592|N|N|N|N|N| +2463685|AAAAAAAAFMHJFCAA|2033-03-28|1598|6952|534|2033|1|3|28|1|2033|534|6952|Monday|2033Q1|N|N|N|2463658|2463716|2463320|2463593|N|N|N|N|N| +2463686|AAAAAAAAGMHJFCAA|2033-03-29|1598|6953|534|2033|2|3|29|1|2033|534|6953|Tuesday|2033Q1|N|N|N|2463658|2463716|2463321|2463594|N|N|N|N|N| +2463687|AAAAAAAAHMHJFCAA|2033-03-30|1598|6953|534|2033|3|3|30|1|2033|534|6953|Wednesday|2033Q1|N|N|N|2463658|2463716|2463322|2463595|N|N|N|N|N| +2463688|AAAAAAAAIMHJFCAA|2033-03-31|1598|6953|534|2033|4|3|31|1|2033|534|6953|Thursday|2033Q1|N|N|N|2463658|2463716|2463323|2463596|N|N|N|N|N| +2463689|AAAAAAAAJMHJFCAA|2033-04-01|1599|6953|534|2033|5|4|1|1|2033|534|6953|Friday|2033Q1|N|Y|N|2463689|2463778|2463324|2463599|N|N|N|N|N| +2463690|AAAAAAAAKMHJFCAA|2033-04-02|1599|6953|534|2033|6|4|2|2|2033|534|6953|Saturday|2033Q2|N|Y|N|2463689|2463778|2463325|2463600|N|N|N|N|N| +2463691|AAAAAAAALMHJFCAA|2033-04-03|1599|6953|534|2033|0|4|3|2|2033|534|6953|Sunday|2033Q2|N|N|N|2463689|2463778|2463326|2463601|N|N|N|N|N| +2463692|AAAAAAAAMMHJFCAA|2033-04-04|1599|6953|534|2033|1|4|4|2|2033|534|6953|Monday|2033Q2|N|N|N|2463689|2463778|2463327|2463602|N|N|N|N|N| +2463693|AAAAAAAANMHJFCAA|2033-04-05|1599|6954|534|2033|2|4|5|2|2033|534|6954|Tuesday|2033Q2|N|N|N|2463689|2463778|2463328|2463603|N|N|N|N|N| +2463694|AAAAAAAAOMHJFCAA|2033-04-06|1599|6954|534|2033|3|4|6|2|2033|534|6954|Wednesday|2033Q2|N|N|N|2463689|2463778|2463329|2463604|N|N|N|N|N| +2463695|AAAAAAAAPMHJFCAA|2033-04-07|1599|6954|534|2033|4|4|7|2|2033|534|6954|Thursday|2033Q2|N|N|N|2463689|2463778|2463330|2463605|N|N|N|N|N| +2463696|AAAAAAAAANHJFCAA|2033-04-08|1599|6954|534|2033|5|4|8|2|2033|534|6954|Friday|2033Q2|N|Y|N|2463689|2463778|2463331|2463606|N|N|N|N|N| +2463697|AAAAAAAABNHJFCAA|2033-04-09|1599|6954|534|2033|6|4|9|2|2033|534|6954|Saturday|2033Q2|N|Y|N|2463689|2463778|2463332|2463607|N|N|N|N|N| +2463698|AAAAAAAACNHJFCAA|2033-04-10|1599|6954|534|2033|0|4|10|2|2033|534|6954|Sunday|2033Q2|N|N|N|2463689|2463778|2463333|2463608|N|N|N|N|N| +2463699|AAAAAAAADNHJFCAA|2033-04-11|1599|6954|534|2033|1|4|11|2|2033|534|6954|Monday|2033Q2|N|N|N|2463689|2463778|2463334|2463609|N|N|N|N|N| +2463700|AAAAAAAAENHJFCAA|2033-04-12|1599|6955|534|2033|2|4|12|2|2033|534|6955|Tuesday|2033Q2|N|N|N|2463689|2463778|2463335|2463610|N|N|N|N|N| +2463701|AAAAAAAAFNHJFCAA|2033-04-13|1599|6955|534|2033|3|4|13|2|2033|534|6955|Wednesday|2033Q2|N|N|N|2463689|2463778|2463336|2463611|N|N|N|N|N| +2463702|AAAAAAAAGNHJFCAA|2033-04-14|1599|6955|534|2033|4|4|14|2|2033|534|6955|Thursday|2033Q2|N|N|N|2463689|2463778|2463337|2463612|N|N|N|N|N| +2463703|AAAAAAAAHNHJFCAA|2033-04-15|1599|6955|534|2033|5|4|15|2|2033|534|6955|Friday|2033Q2|N|Y|N|2463689|2463778|2463338|2463613|N|N|N|N|N| +2463704|AAAAAAAAINHJFCAA|2033-04-16|1599|6955|534|2033|6|4|16|2|2033|534|6955|Saturday|2033Q2|N|Y|N|2463689|2463778|2463339|2463614|N|N|N|N|N| +2463705|AAAAAAAAJNHJFCAA|2033-04-17|1599|6955|534|2033|0|4|17|2|2033|534|6955|Sunday|2033Q2|N|N|N|2463689|2463778|2463340|2463615|N|N|N|N|N| +2463706|AAAAAAAAKNHJFCAA|2033-04-18|1599|6955|534|2033|1|4|18|2|2033|534|6955|Monday|2033Q2|N|N|N|2463689|2463778|2463341|2463616|N|N|N|N|N| +2463707|AAAAAAAALNHJFCAA|2033-04-19|1599|6956|534|2033|2|4|19|2|2033|534|6956|Tuesday|2033Q2|N|N|N|2463689|2463778|2463342|2463617|N|N|N|N|N| +2463708|AAAAAAAAMNHJFCAA|2033-04-20|1599|6956|534|2033|3|4|20|2|2033|534|6956|Wednesday|2033Q2|N|N|N|2463689|2463778|2463343|2463618|N|N|N|N|N| +2463709|AAAAAAAANNHJFCAA|2033-04-21|1599|6956|534|2033|4|4|21|2|2033|534|6956|Thursday|2033Q2|N|N|N|2463689|2463778|2463344|2463619|N|N|N|N|N| +2463710|AAAAAAAAONHJFCAA|2033-04-22|1599|6956|534|2033|5|4|22|2|2033|534|6956|Friday|2033Q2|N|Y|N|2463689|2463778|2463345|2463620|N|N|N|N|N| +2463711|AAAAAAAAPNHJFCAA|2033-04-23|1599|6956|534|2033|6|4|23|2|2033|534|6956|Saturday|2033Q2|N|Y|N|2463689|2463778|2463346|2463621|N|N|N|N|N| +2463712|AAAAAAAAAOHJFCAA|2033-04-24|1599|6956|534|2033|0|4|24|2|2033|534|6956|Sunday|2033Q2|N|N|N|2463689|2463778|2463347|2463622|N|N|N|N|N| +2463713|AAAAAAAABOHJFCAA|2033-04-25|1599|6956|534|2033|1|4|25|2|2033|534|6956|Monday|2033Q2|N|N|N|2463689|2463778|2463348|2463623|N|N|N|N|N| +2463714|AAAAAAAACOHJFCAA|2033-04-26|1599|6957|534|2033|2|4|26|2|2033|534|6957|Tuesday|2033Q2|N|N|N|2463689|2463778|2463349|2463624|N|N|N|N|N| +2463715|AAAAAAAADOHJFCAA|2033-04-27|1599|6957|534|2033|3|4|27|2|2033|534|6957|Wednesday|2033Q2|N|N|N|2463689|2463778|2463350|2463625|N|N|N|N|N| +2463716|AAAAAAAAEOHJFCAA|2033-04-28|1599|6957|534|2033|4|4|28|2|2033|534|6957|Thursday|2033Q2|N|N|N|2463689|2463778|2463351|2463626|N|N|N|N|N| +2463717|AAAAAAAAFOHJFCAA|2033-04-29|1599|6957|534|2033|5|4|29|2|2033|534|6957|Friday|2033Q2|N|Y|N|2463689|2463778|2463352|2463627|N|N|N|N|N| +2463718|AAAAAAAAGOHJFCAA|2033-04-30|1599|6957|534|2033|6|4|30|2|2033|534|6957|Saturday|2033Q2|N|Y|N|2463689|2463778|2463353|2463628|N|N|N|N|N| +2463719|AAAAAAAAHOHJFCAA|2033-05-01|1600|6957|534|2033|0|5|1|2|2033|534|6957|Sunday|2033Q2|N|N|N|2463719|2463838|2463354|2463629|N|N|N|N|N| +2463720|AAAAAAAAIOHJFCAA|2033-05-02|1600|6957|534|2033|1|5|2|2|2033|534|6957|Monday|2033Q2|N|N|N|2463719|2463838|2463355|2463630|N|N|N|N|N| +2463721|AAAAAAAAJOHJFCAA|2033-05-03|1600|6958|534|2033|2|5|3|2|2033|534|6958|Tuesday|2033Q2|N|N|N|2463719|2463838|2463356|2463631|N|N|N|N|N| +2463722|AAAAAAAAKOHJFCAA|2033-05-04|1600|6958|534|2033|3|5|4|2|2033|534|6958|Wednesday|2033Q2|N|N|N|2463719|2463838|2463357|2463632|N|N|N|N|N| +2463723|AAAAAAAALOHJFCAA|2033-05-05|1600|6958|534|2033|4|5|5|2|2033|534|6958|Thursday|2033Q2|N|N|N|2463719|2463838|2463358|2463633|N|N|N|N|N| +2463724|AAAAAAAAMOHJFCAA|2033-05-06|1600|6958|534|2033|5|5|6|2|2033|534|6958|Friday|2033Q2|N|Y|N|2463719|2463838|2463359|2463634|N|N|N|N|N| +2463725|AAAAAAAANOHJFCAA|2033-05-07|1600|6958|534|2033|6|5|7|2|2033|534|6958|Saturday|2033Q2|N|Y|N|2463719|2463838|2463360|2463635|N|N|N|N|N| +2463726|AAAAAAAAOOHJFCAA|2033-05-08|1600|6958|534|2033|0|5|8|2|2033|534|6958|Sunday|2033Q2|N|N|N|2463719|2463838|2463361|2463636|N|N|N|N|N| +2463727|AAAAAAAAPOHJFCAA|2033-05-09|1600|6958|534|2033|1|5|9|2|2033|534|6958|Monday|2033Q2|N|N|N|2463719|2463838|2463362|2463637|N|N|N|N|N| +2463728|AAAAAAAAAPHJFCAA|2033-05-10|1600|6959|534|2033|2|5|10|2|2033|534|6959|Tuesday|2033Q2|N|N|N|2463719|2463838|2463363|2463638|N|N|N|N|N| +2463729|AAAAAAAABPHJFCAA|2033-05-11|1600|6959|534|2033|3|5|11|2|2033|534|6959|Wednesday|2033Q2|N|N|N|2463719|2463838|2463364|2463639|N|N|N|N|N| +2463730|AAAAAAAACPHJFCAA|2033-05-12|1600|6959|534|2033|4|5|12|2|2033|534|6959|Thursday|2033Q2|N|N|N|2463719|2463838|2463365|2463640|N|N|N|N|N| +2463731|AAAAAAAADPHJFCAA|2033-05-13|1600|6959|534|2033|5|5|13|2|2033|534|6959|Friday|2033Q2|N|Y|N|2463719|2463838|2463366|2463641|N|N|N|N|N| +2463732|AAAAAAAAEPHJFCAA|2033-05-14|1600|6959|534|2033|6|5|14|2|2033|534|6959|Saturday|2033Q2|N|Y|N|2463719|2463838|2463367|2463642|N|N|N|N|N| +2463733|AAAAAAAAFPHJFCAA|2033-05-15|1600|6959|534|2033|0|5|15|2|2033|534|6959|Sunday|2033Q2|N|N|N|2463719|2463838|2463368|2463643|N|N|N|N|N| +2463734|AAAAAAAAGPHJFCAA|2033-05-16|1600|6959|534|2033|1|5|16|2|2033|534|6959|Monday|2033Q2|N|N|N|2463719|2463838|2463369|2463644|N|N|N|N|N| +2463735|AAAAAAAAHPHJFCAA|2033-05-17|1600|6960|534|2033|2|5|17|2|2033|534|6960|Tuesday|2033Q2|N|N|N|2463719|2463838|2463370|2463645|N|N|N|N|N| +2463736|AAAAAAAAIPHJFCAA|2033-05-18|1600|6960|534|2033|3|5|18|2|2033|534|6960|Wednesday|2033Q2|N|N|N|2463719|2463838|2463371|2463646|N|N|N|N|N| +2463737|AAAAAAAAJPHJFCAA|2033-05-19|1600|6960|534|2033|4|5|19|2|2033|534|6960|Thursday|2033Q2|N|N|N|2463719|2463838|2463372|2463647|N|N|N|N|N| +2463738|AAAAAAAAKPHJFCAA|2033-05-20|1600|6960|534|2033|5|5|20|2|2033|534|6960|Friday|2033Q2|N|Y|N|2463719|2463838|2463373|2463648|N|N|N|N|N| +2463739|AAAAAAAALPHJFCAA|2033-05-21|1600|6960|534|2033|6|5|21|2|2033|534|6960|Saturday|2033Q2|N|Y|N|2463719|2463838|2463374|2463649|N|N|N|N|N| +2463740|AAAAAAAAMPHJFCAA|2033-05-22|1600|6960|534|2033|0|5|22|2|2033|534|6960|Sunday|2033Q2|N|N|N|2463719|2463838|2463375|2463650|N|N|N|N|N| +2463741|AAAAAAAANPHJFCAA|2033-05-23|1600|6960|534|2033|1|5|23|2|2033|534|6960|Monday|2033Q2|N|N|N|2463719|2463838|2463376|2463651|N|N|N|N|N| +2463742|AAAAAAAAOPHJFCAA|2033-05-24|1600|6961|534|2033|2|5|24|2|2033|534|6961|Tuesday|2033Q2|N|N|N|2463719|2463838|2463377|2463652|N|N|N|N|N| +2463743|AAAAAAAAPPHJFCAA|2033-05-25|1600|6961|534|2033|3|5|25|2|2033|534|6961|Wednesday|2033Q2|N|N|N|2463719|2463838|2463378|2463653|N|N|N|N|N| +2463744|AAAAAAAAAAIJFCAA|2033-05-26|1600|6961|534|2033|4|5|26|2|2033|534|6961|Thursday|2033Q2|N|N|N|2463719|2463838|2463379|2463654|N|N|N|N|N| +2463745|AAAAAAAABAIJFCAA|2033-05-27|1600|6961|534|2033|5|5|27|2|2033|534|6961|Friday|2033Q2|N|Y|N|2463719|2463838|2463380|2463655|N|N|N|N|N| +2463746|AAAAAAAACAIJFCAA|2033-05-28|1600|6961|534|2033|6|5|28|2|2033|534|6961|Saturday|2033Q2|N|Y|N|2463719|2463838|2463381|2463656|N|N|N|N|N| +2463747|AAAAAAAADAIJFCAA|2033-05-29|1600|6961|534|2033|0|5|29|2|2033|534|6961|Sunday|2033Q2|N|N|N|2463719|2463838|2463382|2463657|N|N|N|N|N| +2463748|AAAAAAAAEAIJFCAA|2033-05-30|1600|6961|534|2033|1|5|30|2|2033|534|6961|Monday|2033Q2|N|N|N|2463719|2463838|2463383|2463658|N|N|N|N|N| +2463749|AAAAAAAAFAIJFCAA|2033-05-31|1600|6962|534|2033|2|5|31|2|2033|534|6962|Tuesday|2033Q2|N|N|N|2463719|2463838|2463384|2463659|N|N|N|N|N| +2463750|AAAAAAAAGAIJFCAA|2033-06-01|1601|6962|535|2033|3|6|1|2|2033|535|6962|Wednesday|2033Q2|N|N|N|2463750|2463900|2463385|2463660|N|N|N|N|N| +2463751|AAAAAAAAHAIJFCAA|2033-06-02|1601|6962|535|2033|4|6|2|2|2033|535|6962|Thursday|2033Q2|N|N|N|2463750|2463900|2463386|2463661|N|N|N|N|N| +2463752|AAAAAAAAIAIJFCAA|2033-06-03|1601|6962|535|2033|5|6|3|2|2033|535|6962|Friday|2033Q2|N|Y|N|2463750|2463900|2463387|2463662|N|N|N|N|N| +2463753|AAAAAAAAJAIJFCAA|2033-06-04|1601|6962|535|2033|6|6|4|2|2033|535|6962|Saturday|2033Q2|N|Y|N|2463750|2463900|2463388|2463663|N|N|N|N|N| +2463754|AAAAAAAAKAIJFCAA|2033-06-05|1601|6962|535|2033|0|6|5|2|2033|535|6962|Sunday|2033Q2|N|N|N|2463750|2463900|2463389|2463664|N|N|N|N|N| +2463755|AAAAAAAALAIJFCAA|2033-06-06|1601|6962|535|2033|1|6|6|2|2033|535|6962|Monday|2033Q2|N|N|N|2463750|2463900|2463390|2463665|N|N|N|N|N| +2463756|AAAAAAAAMAIJFCAA|2033-06-07|1601|6963|535|2033|2|6|7|2|2033|535|6963|Tuesday|2033Q2|N|N|N|2463750|2463900|2463391|2463666|N|N|N|N|N| +2463757|AAAAAAAANAIJFCAA|2033-06-08|1601|6963|535|2033|3|6|8|2|2033|535|6963|Wednesday|2033Q2|N|N|N|2463750|2463900|2463392|2463667|N|N|N|N|N| +2463758|AAAAAAAAOAIJFCAA|2033-06-09|1601|6963|535|2033|4|6|9|2|2033|535|6963|Thursday|2033Q2|N|N|N|2463750|2463900|2463393|2463668|N|N|N|N|N| +2463759|AAAAAAAAPAIJFCAA|2033-06-10|1601|6963|535|2033|5|6|10|2|2033|535|6963|Friday|2033Q2|N|Y|N|2463750|2463900|2463394|2463669|N|N|N|N|N| +2463760|AAAAAAAAABIJFCAA|2033-06-11|1601|6963|535|2033|6|6|11|2|2033|535|6963|Saturday|2033Q2|N|Y|N|2463750|2463900|2463395|2463670|N|N|N|N|N| +2463761|AAAAAAAABBIJFCAA|2033-06-12|1601|6963|535|2033|0|6|12|2|2033|535|6963|Sunday|2033Q2|N|N|N|2463750|2463900|2463396|2463671|N|N|N|N|N| +2463762|AAAAAAAACBIJFCAA|2033-06-13|1601|6963|535|2033|1|6|13|2|2033|535|6963|Monday|2033Q2|N|N|N|2463750|2463900|2463397|2463672|N|N|N|N|N| +2463763|AAAAAAAADBIJFCAA|2033-06-14|1601|6964|535|2033|2|6|14|2|2033|535|6964|Tuesday|2033Q2|N|N|N|2463750|2463900|2463398|2463673|N|N|N|N|N| +2463764|AAAAAAAAEBIJFCAA|2033-06-15|1601|6964|535|2033|3|6|15|2|2033|535|6964|Wednesday|2033Q2|N|N|N|2463750|2463900|2463399|2463674|N|N|N|N|N| +2463765|AAAAAAAAFBIJFCAA|2033-06-16|1601|6964|535|2033|4|6|16|2|2033|535|6964|Thursday|2033Q2|N|N|N|2463750|2463900|2463400|2463675|N|N|N|N|N| +2463766|AAAAAAAAGBIJFCAA|2033-06-17|1601|6964|535|2033|5|6|17|2|2033|535|6964|Friday|2033Q2|N|Y|N|2463750|2463900|2463401|2463676|N|N|N|N|N| +2463767|AAAAAAAAHBIJFCAA|2033-06-18|1601|6964|535|2033|6|6|18|2|2033|535|6964|Saturday|2033Q2|N|Y|N|2463750|2463900|2463402|2463677|N|N|N|N|N| +2463768|AAAAAAAAIBIJFCAA|2033-06-19|1601|6964|535|2033|0|6|19|2|2033|535|6964|Sunday|2033Q2|N|N|N|2463750|2463900|2463403|2463678|N|N|N|N|N| +2463769|AAAAAAAAJBIJFCAA|2033-06-20|1601|6964|535|2033|1|6|20|2|2033|535|6964|Monday|2033Q2|N|N|N|2463750|2463900|2463404|2463679|N|N|N|N|N| +2463770|AAAAAAAAKBIJFCAA|2033-06-21|1601|6965|535|2033|2|6|21|2|2033|535|6965|Tuesday|2033Q2|N|N|N|2463750|2463900|2463405|2463680|N|N|N|N|N| +2463771|AAAAAAAALBIJFCAA|2033-06-22|1601|6965|535|2033|3|6|22|2|2033|535|6965|Wednesday|2033Q2|N|N|N|2463750|2463900|2463406|2463681|N|N|N|N|N| +2463772|AAAAAAAAMBIJFCAA|2033-06-23|1601|6965|535|2033|4|6|23|2|2033|535|6965|Thursday|2033Q2|N|N|N|2463750|2463900|2463407|2463682|N|N|N|N|N| +2463773|AAAAAAAANBIJFCAA|2033-06-24|1601|6965|535|2033|5|6|24|2|2033|535|6965|Friday|2033Q2|N|Y|N|2463750|2463900|2463408|2463683|N|N|N|N|N| +2463774|AAAAAAAAOBIJFCAA|2033-06-25|1601|6965|535|2033|6|6|25|2|2033|535|6965|Saturday|2033Q2|N|Y|N|2463750|2463900|2463409|2463684|N|N|N|N|N| +2463775|AAAAAAAAPBIJFCAA|2033-06-26|1601|6965|535|2033|0|6|26|2|2033|535|6965|Sunday|2033Q2|N|N|N|2463750|2463900|2463410|2463685|N|N|N|N|N| +2463776|AAAAAAAAACIJFCAA|2033-06-27|1601|6965|535|2033|1|6|27|2|2033|535|6965|Monday|2033Q2|N|N|N|2463750|2463900|2463411|2463686|N|N|N|N|N| +2463777|AAAAAAAABCIJFCAA|2033-06-28|1601|6966|535|2033|2|6|28|2|2033|535|6966|Tuesday|2033Q2|N|N|N|2463750|2463900|2463412|2463687|N|N|N|N|N| +2463778|AAAAAAAACCIJFCAA|2033-06-29|1601|6966|535|2033|3|6|29|2|2033|535|6966|Wednesday|2033Q2|N|N|N|2463750|2463900|2463413|2463688|N|N|N|N|N| +2463779|AAAAAAAADCIJFCAA|2033-06-30|1601|6966|535|2033|4|6|30|2|2033|535|6966|Thursday|2033Q2|N|N|N|2463750|2463900|2463414|2463689|N|N|N|N|N| +2463780|AAAAAAAAECIJFCAA|2033-07-01|1602|6966|535|2033|5|7|1|2|2033|535|6966|Friday|2033Q2|N|Y|N|2463780|2463960|2463415|2463689|N|N|N|N|N| +2463781|AAAAAAAAFCIJFCAA|2033-07-02|1602|6966|535|2033|6|7|2|3|2033|535|6966|Saturday|2033Q3|N|Y|N|2463780|2463960|2463416|2463690|N|N|N|N|N| +2463782|AAAAAAAAGCIJFCAA|2033-07-03|1602|6966|535|2033|0|7|3|3|2033|535|6966|Sunday|2033Q3|N|N|N|2463780|2463960|2463417|2463691|N|N|N|N|N| +2463783|AAAAAAAAHCIJFCAA|2033-07-04|1602|6966|535|2033|1|7|4|3|2033|535|6966|Monday|2033Q3|N|N|N|2463780|2463960|2463418|2463692|N|N|N|N|N| +2463784|AAAAAAAAICIJFCAA|2033-07-05|1602|6967|535|2033|2|7|5|3|2033|535|6967|Tuesday|2033Q3|Y|N|N|2463780|2463960|2463419|2463693|N|N|N|N|N| +2463785|AAAAAAAAJCIJFCAA|2033-07-06|1602|6967|535|2033|3|7|6|3|2033|535|6967|Wednesday|2033Q3|N|N|Y|2463780|2463960|2463420|2463694|N|N|N|N|N| +2463786|AAAAAAAAKCIJFCAA|2033-07-07|1602|6967|535|2033|4|7|7|3|2033|535|6967|Thursday|2033Q3|N|N|N|2463780|2463960|2463421|2463695|N|N|N|N|N| +2463787|AAAAAAAALCIJFCAA|2033-07-08|1602|6967|535|2033|5|7|8|3|2033|535|6967|Friday|2033Q3|N|Y|N|2463780|2463960|2463422|2463696|N|N|N|N|N| +2463788|AAAAAAAAMCIJFCAA|2033-07-09|1602|6967|535|2033|6|7|9|3|2033|535|6967|Saturday|2033Q3|N|Y|N|2463780|2463960|2463423|2463697|N|N|N|N|N| +2463789|AAAAAAAANCIJFCAA|2033-07-10|1602|6967|535|2033|0|7|10|3|2033|535|6967|Sunday|2033Q3|N|N|N|2463780|2463960|2463424|2463698|N|N|N|N|N| +2463790|AAAAAAAAOCIJFCAA|2033-07-11|1602|6967|535|2033|1|7|11|3|2033|535|6967|Monday|2033Q3|N|N|N|2463780|2463960|2463425|2463699|N|N|N|N|N| +2463791|AAAAAAAAPCIJFCAA|2033-07-12|1602|6968|535|2033|2|7|12|3|2033|535|6968|Tuesday|2033Q3|N|N|N|2463780|2463960|2463426|2463700|N|N|N|N|N| +2463792|AAAAAAAAADIJFCAA|2033-07-13|1602|6968|535|2033|3|7|13|3|2033|535|6968|Wednesday|2033Q3|N|N|N|2463780|2463960|2463427|2463701|N|N|N|N|N| +2463793|AAAAAAAABDIJFCAA|2033-07-14|1602|6968|535|2033|4|7|14|3|2033|535|6968|Thursday|2033Q3|N|N|N|2463780|2463960|2463428|2463702|N|N|N|N|N| +2463794|AAAAAAAACDIJFCAA|2033-07-15|1602|6968|535|2033|5|7|15|3|2033|535|6968|Friday|2033Q3|N|Y|N|2463780|2463960|2463429|2463703|N|N|N|N|N| +2463795|AAAAAAAADDIJFCAA|2033-07-16|1602|6968|535|2033|6|7|16|3|2033|535|6968|Saturday|2033Q3|N|Y|N|2463780|2463960|2463430|2463704|N|N|N|N|N| +2463796|AAAAAAAAEDIJFCAA|2033-07-17|1602|6968|535|2033|0|7|17|3|2033|535|6968|Sunday|2033Q3|N|N|N|2463780|2463960|2463431|2463705|N|N|N|N|N| +2463797|AAAAAAAAFDIJFCAA|2033-07-18|1602|6968|535|2033|1|7|18|3|2033|535|6968|Monday|2033Q3|N|N|N|2463780|2463960|2463432|2463706|N|N|N|N|N| +2463798|AAAAAAAAGDIJFCAA|2033-07-19|1602|6969|535|2033|2|7|19|3|2033|535|6969|Tuesday|2033Q3|N|N|N|2463780|2463960|2463433|2463707|N|N|N|N|N| +2463799|AAAAAAAAHDIJFCAA|2033-07-20|1602|6969|535|2033|3|7|20|3|2033|535|6969|Wednesday|2033Q3|N|N|N|2463780|2463960|2463434|2463708|N|N|N|N|N| +2463800|AAAAAAAAIDIJFCAA|2033-07-21|1602|6969|535|2033|4|7|21|3|2033|535|6969|Thursday|2033Q3|N|N|N|2463780|2463960|2463435|2463709|N|N|N|N|N| +2463801|AAAAAAAAJDIJFCAA|2033-07-22|1602|6969|535|2033|5|7|22|3|2033|535|6969|Friday|2033Q3|N|Y|N|2463780|2463960|2463436|2463710|N|N|N|N|N| +2463802|AAAAAAAAKDIJFCAA|2033-07-23|1602|6969|535|2033|6|7|23|3|2033|535|6969|Saturday|2033Q3|N|Y|N|2463780|2463960|2463437|2463711|N|N|N|N|N| +2463803|AAAAAAAALDIJFCAA|2033-07-24|1602|6969|535|2033|0|7|24|3|2033|535|6969|Sunday|2033Q3|N|N|N|2463780|2463960|2463438|2463712|N|N|N|N|N| +2463804|AAAAAAAAMDIJFCAA|2033-07-25|1602|6969|535|2033|1|7|25|3|2033|535|6969|Monday|2033Q3|N|N|N|2463780|2463960|2463439|2463713|N|N|N|N|N| +2463805|AAAAAAAANDIJFCAA|2033-07-26|1602|6970|535|2033|2|7|26|3|2033|535|6970|Tuesday|2033Q3|N|N|N|2463780|2463960|2463440|2463714|N|N|N|N|N| +2463806|AAAAAAAAODIJFCAA|2033-07-27|1602|6970|535|2033|3|7|27|3|2033|535|6970|Wednesday|2033Q3|N|N|N|2463780|2463960|2463441|2463715|N|N|N|N|N| +2463807|AAAAAAAAPDIJFCAA|2033-07-28|1602|6970|535|2033|4|7|28|3|2033|535|6970|Thursday|2033Q3|N|N|N|2463780|2463960|2463442|2463716|N|N|N|N|N| +2463808|AAAAAAAAAEIJFCAA|2033-07-29|1602|6970|535|2033|5|7|29|3|2033|535|6970|Friday|2033Q3|N|Y|N|2463780|2463960|2463443|2463717|N|N|N|N|N| +2463809|AAAAAAAABEIJFCAA|2033-07-30|1602|6970|535|2033|6|7|30|3|2033|535|6970|Saturday|2033Q3|N|Y|N|2463780|2463960|2463444|2463718|N|N|N|N|N| +2463810|AAAAAAAACEIJFCAA|2033-07-31|1602|6970|535|2033|0|7|31|3|2033|535|6970|Sunday|2033Q3|N|N|N|2463780|2463960|2463445|2463719|N|N|N|N|N| +2463811|AAAAAAAADEIJFCAA|2033-08-01|1603|6970|535|2033|1|8|1|3|2033|535|6970|Monday|2033Q3|N|N|N|2463811|2464022|2463446|2463720|N|N|N|N|N| +2463812|AAAAAAAAEEIJFCAA|2033-08-02|1603|6971|535|2033|2|8|2|3|2033|535|6971|Tuesday|2033Q3|N|N|N|2463811|2464022|2463447|2463721|N|N|N|N|N| +2463813|AAAAAAAAFEIJFCAA|2033-08-03|1603|6971|535|2033|3|8|3|3|2033|535|6971|Wednesday|2033Q3|N|N|N|2463811|2464022|2463448|2463722|N|N|N|N|N| +2463814|AAAAAAAAGEIJFCAA|2033-08-04|1603|6971|535|2033|4|8|4|3|2033|535|6971|Thursday|2033Q3|N|N|N|2463811|2464022|2463449|2463723|N|N|N|N|N| +2463815|AAAAAAAAHEIJFCAA|2033-08-05|1603|6971|535|2033|5|8|5|3|2033|535|6971|Friday|2033Q3|N|Y|N|2463811|2464022|2463450|2463724|N|N|N|N|N| +2463816|AAAAAAAAIEIJFCAA|2033-08-06|1603|6971|535|2033|6|8|6|3|2033|535|6971|Saturday|2033Q3|N|Y|N|2463811|2464022|2463451|2463725|N|N|N|N|N| +2463817|AAAAAAAAJEIJFCAA|2033-08-07|1603|6971|535|2033|0|8|7|3|2033|535|6971|Sunday|2033Q3|N|N|N|2463811|2464022|2463452|2463726|N|N|N|N|N| +2463818|AAAAAAAAKEIJFCAA|2033-08-08|1603|6971|535|2033|1|8|8|3|2033|535|6971|Monday|2033Q3|N|N|N|2463811|2464022|2463453|2463727|N|N|N|N|N| +2463819|AAAAAAAALEIJFCAA|2033-08-09|1603|6972|535|2033|2|8|9|3|2033|535|6972|Tuesday|2033Q3|N|N|N|2463811|2464022|2463454|2463728|N|N|N|N|N| +2463820|AAAAAAAAMEIJFCAA|2033-08-10|1603|6972|535|2033|3|8|10|3|2033|535|6972|Wednesday|2033Q3|N|N|N|2463811|2464022|2463455|2463729|N|N|N|N|N| +2463821|AAAAAAAANEIJFCAA|2033-08-11|1603|6972|535|2033|4|8|11|3|2033|535|6972|Thursday|2033Q3|N|N|N|2463811|2464022|2463456|2463730|N|N|N|N|N| +2463822|AAAAAAAAOEIJFCAA|2033-08-12|1603|6972|535|2033|5|8|12|3|2033|535|6972|Friday|2033Q3|N|Y|N|2463811|2464022|2463457|2463731|N|N|N|N|N| +2463823|AAAAAAAAPEIJFCAA|2033-08-13|1603|6972|535|2033|6|8|13|3|2033|535|6972|Saturday|2033Q3|N|Y|N|2463811|2464022|2463458|2463732|N|N|N|N|N| +2463824|AAAAAAAAAFIJFCAA|2033-08-14|1603|6972|535|2033|0|8|14|3|2033|535|6972|Sunday|2033Q3|N|N|N|2463811|2464022|2463459|2463733|N|N|N|N|N| +2463825|AAAAAAAABFIJFCAA|2033-08-15|1603|6972|535|2033|1|8|15|3|2033|535|6972|Monday|2033Q3|N|N|N|2463811|2464022|2463460|2463734|N|N|N|N|N| +2463826|AAAAAAAACFIJFCAA|2033-08-16|1603|6973|535|2033|2|8|16|3|2033|535|6973|Tuesday|2033Q3|N|N|N|2463811|2464022|2463461|2463735|N|N|N|N|N| +2463827|AAAAAAAADFIJFCAA|2033-08-17|1603|6973|535|2033|3|8|17|3|2033|535|6973|Wednesday|2033Q3|N|N|N|2463811|2464022|2463462|2463736|N|N|N|N|N| +2463828|AAAAAAAAEFIJFCAA|2033-08-18|1603|6973|535|2033|4|8|18|3|2033|535|6973|Thursday|2033Q3|N|N|N|2463811|2464022|2463463|2463737|N|N|N|N|N| +2463829|AAAAAAAAFFIJFCAA|2033-08-19|1603|6973|535|2033|5|8|19|3|2033|535|6973|Friday|2033Q3|N|Y|N|2463811|2464022|2463464|2463738|N|N|N|N|N| +2463830|AAAAAAAAGFIJFCAA|2033-08-20|1603|6973|535|2033|6|8|20|3|2033|535|6973|Saturday|2033Q3|N|Y|N|2463811|2464022|2463465|2463739|N|N|N|N|N| +2463831|AAAAAAAAHFIJFCAA|2033-08-21|1603|6973|535|2033|0|8|21|3|2033|535|6973|Sunday|2033Q3|N|N|N|2463811|2464022|2463466|2463740|N|N|N|N|N| +2463832|AAAAAAAAIFIJFCAA|2033-08-22|1603|6973|535|2033|1|8|22|3|2033|535|6973|Monday|2033Q3|N|N|N|2463811|2464022|2463467|2463741|N|N|N|N|N| +2463833|AAAAAAAAJFIJFCAA|2033-08-23|1603|6974|535|2033|2|8|23|3|2033|535|6974|Tuesday|2033Q3|N|N|N|2463811|2464022|2463468|2463742|N|N|N|N|N| +2463834|AAAAAAAAKFIJFCAA|2033-08-24|1603|6974|535|2033|3|8|24|3|2033|535|6974|Wednesday|2033Q3|N|N|N|2463811|2464022|2463469|2463743|N|N|N|N|N| +2463835|AAAAAAAALFIJFCAA|2033-08-25|1603|6974|535|2033|4|8|25|3|2033|535|6974|Thursday|2033Q3|N|N|N|2463811|2464022|2463470|2463744|N|N|N|N|N| +2463836|AAAAAAAAMFIJFCAA|2033-08-26|1603|6974|535|2033|5|8|26|3|2033|535|6974|Friday|2033Q3|N|Y|N|2463811|2464022|2463471|2463745|N|N|N|N|N| +2463837|AAAAAAAANFIJFCAA|2033-08-27|1603|6974|535|2033|6|8|27|3|2033|535|6974|Saturday|2033Q3|N|Y|N|2463811|2464022|2463472|2463746|N|N|N|N|N| +2463838|AAAAAAAAOFIJFCAA|2033-08-28|1603|6974|535|2033|0|8|28|3|2033|535|6974|Sunday|2033Q3|N|N|N|2463811|2464022|2463473|2463747|N|N|N|N|N| +2463839|AAAAAAAAPFIJFCAA|2033-08-29|1603|6974|535|2033|1|8|29|3|2033|535|6974|Monday|2033Q3|N|N|N|2463811|2464022|2463474|2463748|N|N|N|N|N| +2463840|AAAAAAAAAGIJFCAA|2033-08-30|1603|6975|535|2033|2|8|30|3|2033|535|6975|Tuesday|2033Q3|N|N|N|2463811|2464022|2463475|2463749|N|N|N|N|N| +2463841|AAAAAAAABGIJFCAA|2033-08-31|1603|6975|535|2033|3|8|31|3|2033|535|6975|Wednesday|2033Q3|N|N|N|2463811|2464022|2463476|2463750|N|N|N|N|N| +2463842|AAAAAAAACGIJFCAA|2033-09-01|1604|6975|536|2033|4|9|1|3|2033|536|6975|Thursday|2033Q3|N|N|N|2463842|2464084|2463477|2463751|N|N|N|N|N| +2463843|AAAAAAAADGIJFCAA|2033-09-02|1604|6975|536|2033|5|9|2|3|2033|536|6975|Friday|2033Q3|N|Y|N|2463842|2464084|2463478|2463752|N|N|N|N|N| +2463844|AAAAAAAAEGIJFCAA|2033-09-03|1604|6975|536|2033|6|9|3|3|2033|536|6975|Saturday|2033Q3|N|Y|N|2463842|2464084|2463479|2463753|N|N|N|N|N| +2463845|AAAAAAAAFGIJFCAA|2033-09-04|1604|6975|536|2033|0|9|4|3|2033|536|6975|Sunday|2033Q3|N|N|N|2463842|2464084|2463480|2463754|N|N|N|N|N| +2463846|AAAAAAAAGGIJFCAA|2033-09-05|1604|6975|536|2033|1|9|5|3|2033|536|6975|Monday|2033Q3|N|N|N|2463842|2464084|2463481|2463755|N|N|N|N|N| +2463847|AAAAAAAAHGIJFCAA|2033-09-06|1604|6976|536|2033|2|9|6|3|2033|536|6976|Tuesday|2033Q3|N|N|N|2463842|2464084|2463482|2463756|N|N|N|N|N| +2463848|AAAAAAAAIGIJFCAA|2033-09-07|1604|6976|536|2033|3|9|7|3|2033|536|6976|Wednesday|2033Q3|N|N|N|2463842|2464084|2463483|2463757|N|N|N|N|N| +2463849|AAAAAAAAJGIJFCAA|2033-09-08|1604|6976|536|2033|4|9|8|3|2033|536|6976|Thursday|2033Q3|N|N|N|2463842|2464084|2463484|2463758|N|N|N|N|N| +2463850|AAAAAAAAKGIJFCAA|2033-09-09|1604|6976|536|2033|5|9|9|3|2033|536|6976|Friday|2033Q3|N|Y|N|2463842|2464084|2463485|2463759|N|N|N|N|N| +2463851|AAAAAAAALGIJFCAA|2033-09-10|1604|6976|536|2033|6|9|10|3|2033|536|6976|Saturday|2033Q3|N|Y|N|2463842|2464084|2463486|2463760|N|N|N|N|N| +2463852|AAAAAAAAMGIJFCAA|2033-09-11|1604|6976|536|2033|0|9|11|3|2033|536|6976|Sunday|2033Q3|N|N|N|2463842|2464084|2463487|2463761|N|N|N|N|N| +2463853|AAAAAAAANGIJFCAA|2033-09-12|1604|6976|536|2033|1|9|12|3|2033|536|6976|Monday|2033Q3|N|N|N|2463842|2464084|2463488|2463762|N|N|N|N|N| +2463854|AAAAAAAAOGIJFCAA|2033-09-13|1604|6977|536|2033|2|9|13|3|2033|536|6977|Tuesday|2033Q3|N|N|N|2463842|2464084|2463489|2463763|N|N|N|N|N| +2463855|AAAAAAAAPGIJFCAA|2033-09-14|1604|6977|536|2033|3|9|14|3|2033|536|6977|Wednesday|2033Q3|N|N|N|2463842|2464084|2463490|2463764|N|N|N|N|N| +2463856|AAAAAAAAAHIJFCAA|2033-09-15|1604|6977|536|2033|4|9|15|3|2033|536|6977|Thursday|2033Q3|N|N|N|2463842|2464084|2463491|2463765|N|N|N|N|N| +2463857|AAAAAAAABHIJFCAA|2033-09-16|1604|6977|536|2033|5|9|16|3|2033|536|6977|Friday|2033Q3|N|Y|N|2463842|2464084|2463492|2463766|N|N|N|N|N| +2463858|AAAAAAAACHIJFCAA|2033-09-17|1604|6977|536|2033|6|9|17|3|2033|536|6977|Saturday|2033Q3|N|Y|N|2463842|2464084|2463493|2463767|N|N|N|N|N| +2463859|AAAAAAAADHIJFCAA|2033-09-18|1604|6977|536|2033|0|9|18|3|2033|536|6977|Sunday|2033Q3|N|N|N|2463842|2464084|2463494|2463768|N|N|N|N|N| +2463860|AAAAAAAAEHIJFCAA|2033-09-19|1604|6977|536|2033|1|9|19|3|2033|536|6977|Monday|2033Q3|N|N|N|2463842|2464084|2463495|2463769|N|N|N|N|N| +2463861|AAAAAAAAFHIJFCAA|2033-09-20|1604|6978|536|2033|2|9|20|3|2033|536|6978|Tuesday|2033Q3|N|N|N|2463842|2464084|2463496|2463770|N|N|N|N|N| +2463862|AAAAAAAAGHIJFCAA|2033-09-21|1604|6978|536|2033|3|9|21|3|2033|536|6978|Wednesday|2033Q3|N|N|N|2463842|2464084|2463497|2463771|N|N|N|N|N| +2463863|AAAAAAAAHHIJFCAA|2033-09-22|1604|6978|536|2033|4|9|22|3|2033|536|6978|Thursday|2033Q3|N|N|N|2463842|2464084|2463498|2463772|N|N|N|N|N| +2463864|AAAAAAAAIHIJFCAA|2033-09-23|1604|6978|536|2033|5|9|23|3|2033|536|6978|Friday|2033Q3|N|Y|N|2463842|2464084|2463499|2463773|N|N|N|N|N| +2463865|AAAAAAAAJHIJFCAA|2033-09-24|1604|6978|536|2033|6|9|24|3|2033|536|6978|Saturday|2033Q3|N|Y|N|2463842|2464084|2463500|2463774|N|N|N|N|N| +2463866|AAAAAAAAKHIJFCAA|2033-09-25|1604|6978|536|2033|0|9|25|3|2033|536|6978|Sunday|2033Q3|N|N|N|2463842|2464084|2463501|2463775|N|N|N|N|N| +2463867|AAAAAAAALHIJFCAA|2033-09-26|1604|6978|536|2033|1|9|26|3|2033|536|6978|Monday|2033Q3|N|N|N|2463842|2464084|2463502|2463776|N|N|N|N|N| +2463868|AAAAAAAAMHIJFCAA|2033-09-27|1604|6979|536|2033|2|9|27|3|2033|536|6979|Tuesday|2033Q3|N|N|N|2463842|2464084|2463503|2463777|N|N|N|N|N| +2463869|AAAAAAAANHIJFCAA|2033-09-28|1604|6979|536|2033|3|9|28|3|2033|536|6979|Wednesday|2033Q3|N|N|N|2463842|2464084|2463504|2463778|N|N|N|N|N| +2463870|AAAAAAAAOHIJFCAA|2033-09-29|1604|6979|536|2033|4|9|29|3|2033|536|6979|Thursday|2033Q3|N|N|N|2463842|2464084|2463505|2463779|N|N|N|N|N| +2463871|AAAAAAAAPHIJFCAA|2033-09-30|1604|6979|536|2033|5|9|30|3|2033|536|6979|Friday|2033Q3|N|Y|N|2463842|2464084|2463506|2463780|N|N|N|N|N| +2463872|AAAAAAAAAIIJFCAA|2033-10-01|1605|6979|536|2033|6|10|1|3|2033|536|6979|Saturday|2033Q3|N|Y|N|2463872|2464144|2463507|2463780|N|N|N|N|N| +2463873|AAAAAAAABIIJFCAA|2033-10-02|1605|6979|536|2033|0|10|2|4|2033|536|6979|Sunday|2033Q4|N|N|N|2463872|2464144|2463508|2463781|N|N|N|N|N| +2463874|AAAAAAAACIIJFCAA|2033-10-03|1605|6979|536|2033|1|10|3|4|2033|536|6979|Monday|2033Q4|N|N|N|2463872|2464144|2463509|2463782|N|N|N|N|N| +2463875|AAAAAAAADIIJFCAA|2033-10-04|1605|6980|536|2033|2|10|4|4|2033|536|6980|Tuesday|2033Q4|N|N|N|2463872|2464144|2463510|2463783|N|N|N|N|N| +2463876|AAAAAAAAEIIJFCAA|2033-10-05|1605|6980|536|2033|3|10|5|4|2033|536|6980|Wednesday|2033Q4|N|N|N|2463872|2464144|2463511|2463784|N|N|N|N|N| +2463877|AAAAAAAAFIIJFCAA|2033-10-06|1605|6980|536|2033|4|10|6|4|2033|536|6980|Thursday|2033Q4|N|N|N|2463872|2464144|2463512|2463785|N|N|N|N|N| +2463878|AAAAAAAAGIIJFCAA|2033-10-07|1605|6980|536|2033|5|10|7|4|2033|536|6980|Friday|2033Q4|N|Y|N|2463872|2464144|2463513|2463786|N|N|N|N|N| +2463879|AAAAAAAAHIIJFCAA|2033-10-08|1605|6980|536|2033|6|10|8|4|2033|536|6980|Saturday|2033Q4|N|Y|N|2463872|2464144|2463514|2463787|N|N|N|N|N| +2463880|AAAAAAAAIIIJFCAA|2033-10-09|1605|6980|536|2033|0|10|9|4|2033|536|6980|Sunday|2033Q4|N|N|N|2463872|2464144|2463515|2463788|N|N|N|N|N| +2463881|AAAAAAAAJIIJFCAA|2033-10-10|1605|6980|536|2033|1|10|10|4|2033|536|6980|Monday|2033Q4|N|N|N|2463872|2464144|2463516|2463789|N|N|N|N|N| +2463882|AAAAAAAAKIIJFCAA|2033-10-11|1605|6981|536|2033|2|10|11|4|2033|536|6981|Tuesday|2033Q4|N|N|N|2463872|2464144|2463517|2463790|N|N|N|N|N| +2463883|AAAAAAAALIIJFCAA|2033-10-12|1605|6981|536|2033|3|10|12|4|2033|536|6981|Wednesday|2033Q4|N|N|N|2463872|2464144|2463518|2463791|N|N|N|N|N| +2463884|AAAAAAAAMIIJFCAA|2033-10-13|1605|6981|536|2033|4|10|13|4|2033|536|6981|Thursday|2033Q4|N|N|N|2463872|2464144|2463519|2463792|N|N|N|N|N| +2463885|AAAAAAAANIIJFCAA|2033-10-14|1605|6981|536|2033|5|10|14|4|2033|536|6981|Friday|2033Q4|N|Y|N|2463872|2464144|2463520|2463793|N|N|N|N|N| +2463886|AAAAAAAAOIIJFCAA|2033-10-15|1605|6981|536|2033|6|10|15|4|2033|536|6981|Saturday|2033Q4|N|Y|N|2463872|2464144|2463521|2463794|N|N|N|N|N| +2463887|AAAAAAAAPIIJFCAA|2033-10-16|1605|6981|536|2033|0|10|16|4|2033|536|6981|Sunday|2033Q4|N|N|N|2463872|2464144|2463522|2463795|N|N|N|N|N| +2463888|AAAAAAAAAJIJFCAA|2033-10-17|1605|6981|536|2033|1|10|17|4|2033|536|6981|Monday|2033Q4|N|N|N|2463872|2464144|2463523|2463796|N|N|N|N|N| +2463889|AAAAAAAABJIJFCAA|2033-10-18|1605|6982|536|2033|2|10|18|4|2033|536|6982|Tuesday|2033Q4|N|N|N|2463872|2464144|2463524|2463797|N|N|N|N|N| +2463890|AAAAAAAACJIJFCAA|2033-10-19|1605|6982|536|2033|3|10|19|4|2033|536|6982|Wednesday|2033Q4|N|N|N|2463872|2464144|2463525|2463798|N|N|N|N|N| +2463891|AAAAAAAADJIJFCAA|2033-10-20|1605|6982|536|2033|4|10|20|4|2033|536|6982|Thursday|2033Q4|N|N|N|2463872|2464144|2463526|2463799|N|N|N|N|N| +2463892|AAAAAAAAEJIJFCAA|2033-10-21|1605|6982|536|2033|5|10|21|4|2033|536|6982|Friday|2033Q4|N|Y|N|2463872|2464144|2463527|2463800|N|N|N|N|N| +2463893|AAAAAAAAFJIJFCAA|2033-10-22|1605|6982|536|2033|6|10|22|4|2033|536|6982|Saturday|2033Q4|N|Y|N|2463872|2464144|2463528|2463801|N|N|N|N|N| +2463894|AAAAAAAAGJIJFCAA|2033-10-23|1605|6982|536|2033|0|10|23|4|2033|536|6982|Sunday|2033Q4|N|N|N|2463872|2464144|2463529|2463802|N|N|N|N|N| +2463895|AAAAAAAAHJIJFCAA|2033-10-24|1605|6982|536|2033|1|10|24|4|2033|536|6982|Monday|2033Q4|N|N|N|2463872|2464144|2463530|2463803|N|N|N|N|N| +2463896|AAAAAAAAIJIJFCAA|2033-10-25|1605|6983|536|2033|2|10|25|4|2033|536|6983|Tuesday|2033Q4|N|N|N|2463872|2464144|2463531|2463804|N|N|N|N|N| +2463897|AAAAAAAAJJIJFCAA|2033-10-26|1605|6983|536|2033|3|10|26|4|2033|536|6983|Wednesday|2033Q4|N|N|N|2463872|2464144|2463532|2463805|N|N|N|N|N| +2463898|AAAAAAAAKJIJFCAA|2033-10-27|1605|6983|536|2033|4|10|27|4|2033|536|6983|Thursday|2033Q4|N|N|N|2463872|2464144|2463533|2463806|N|N|N|N|N| +2463899|AAAAAAAALJIJFCAA|2033-10-28|1605|6983|536|2033|5|10|28|4|2033|536|6983|Friday|2033Q4|N|Y|N|2463872|2464144|2463534|2463807|N|N|N|N|N| +2463900|AAAAAAAAMJIJFCAA|2033-10-29|1605|6983|536|2033|6|10|29|4|2033|536|6983|Saturday|2033Q4|N|Y|N|2463872|2464144|2463535|2463808|N|N|N|N|N| +2463901|AAAAAAAANJIJFCAA|2033-10-30|1605|6983|536|2033|0|10|30|4|2033|536|6983|Sunday|2033Q4|N|N|N|2463872|2464144|2463536|2463809|N|N|N|N|N| +2463902|AAAAAAAAOJIJFCAA|2033-10-31|1605|6983|536|2033|1|10|31|4|2033|536|6983|Monday|2033Q4|N|N|N|2463872|2464144|2463537|2463810|N|N|N|N|N| +2463903|AAAAAAAAPJIJFCAA|2033-11-01|1606|6984|536|2033|2|11|1|4|2033|536|6984|Tuesday|2033Q4|N|N|N|2463903|2464206|2463538|2463811|N|N|N|N|N| +2463904|AAAAAAAAAKIJFCAA|2033-11-02|1606|6984|536|2033|3|11|2|4|2033|536|6984|Wednesday|2033Q4|N|N|N|2463903|2464206|2463539|2463812|N|N|N|N|N| +2463905|AAAAAAAABKIJFCAA|2033-11-03|1606|6984|536|2033|4|11|3|4|2033|536|6984|Thursday|2033Q4|N|N|N|2463903|2464206|2463540|2463813|N|N|N|N|N| +2463906|AAAAAAAACKIJFCAA|2033-11-04|1606|6984|536|2033|5|11|4|4|2033|536|6984|Friday|2033Q4|N|Y|N|2463903|2464206|2463541|2463814|N|N|N|N|N| +2463907|AAAAAAAADKIJFCAA|2033-11-05|1606|6984|536|2033|6|11|5|4|2033|536|6984|Saturday|2033Q4|N|Y|N|2463903|2464206|2463542|2463815|N|N|N|N|N| +2463908|AAAAAAAAEKIJFCAA|2033-11-06|1606|6984|536|2033|0|11|6|4|2033|536|6984|Sunday|2033Q4|N|N|N|2463903|2464206|2463543|2463816|N|N|N|N|N| +2463909|AAAAAAAAFKIJFCAA|2033-11-07|1606|6984|536|2033|1|11|7|4|2033|536|6984|Monday|2033Q4|N|N|N|2463903|2464206|2463544|2463817|N|N|N|N|N| +2463910|AAAAAAAAGKIJFCAA|2033-11-08|1606|6985|536|2033|2|11|8|4|2033|536|6985|Tuesday|2033Q4|N|N|N|2463903|2464206|2463545|2463818|N|N|N|N|N| +2463911|AAAAAAAAHKIJFCAA|2033-11-09|1606|6985|536|2033|3|11|9|4|2033|536|6985|Wednesday|2033Q4|N|N|N|2463903|2464206|2463546|2463819|N|N|N|N|N| +2463912|AAAAAAAAIKIJFCAA|2033-11-10|1606|6985|536|2033|4|11|10|4|2033|536|6985|Thursday|2033Q4|N|N|N|2463903|2464206|2463547|2463820|N|N|N|N|N| +2463913|AAAAAAAAJKIJFCAA|2033-11-11|1606|6985|536|2033|5|11|11|4|2033|536|6985|Friday|2033Q4|N|Y|N|2463903|2464206|2463548|2463821|N|N|N|N|N| +2463914|AAAAAAAAKKIJFCAA|2033-11-12|1606|6985|536|2033|6|11|12|4|2033|536|6985|Saturday|2033Q4|N|Y|N|2463903|2464206|2463549|2463822|N|N|N|N|N| +2463915|AAAAAAAALKIJFCAA|2033-11-13|1606|6985|536|2033|0|11|13|4|2033|536|6985|Sunday|2033Q4|N|N|N|2463903|2464206|2463550|2463823|N|N|N|N|N| +2463916|AAAAAAAAMKIJFCAA|2033-11-14|1606|6985|536|2033|1|11|14|4|2033|536|6985|Monday|2033Q4|N|N|N|2463903|2464206|2463551|2463824|N|N|N|N|N| +2463917|AAAAAAAANKIJFCAA|2033-11-15|1606|6986|536|2033|2|11|15|4|2033|536|6986|Tuesday|2033Q4|N|N|N|2463903|2464206|2463552|2463825|N|N|N|N|N| +2463918|AAAAAAAAOKIJFCAA|2033-11-16|1606|6986|536|2033|3|11|16|4|2033|536|6986|Wednesday|2033Q4|N|N|N|2463903|2464206|2463553|2463826|N|N|N|N|N| +2463919|AAAAAAAAPKIJFCAA|2033-11-17|1606|6986|536|2033|4|11|17|4|2033|536|6986|Thursday|2033Q4|N|N|N|2463903|2464206|2463554|2463827|N|N|N|N|N| +2463920|AAAAAAAAALIJFCAA|2033-11-18|1606|6986|536|2033|5|11|18|4|2033|536|6986|Friday|2033Q4|N|Y|N|2463903|2464206|2463555|2463828|N|N|N|N|N| +2463921|AAAAAAAABLIJFCAA|2033-11-19|1606|6986|536|2033|6|11|19|4|2033|536|6986|Saturday|2033Q4|N|Y|N|2463903|2464206|2463556|2463829|N|N|N|N|N| +2463922|AAAAAAAACLIJFCAA|2033-11-20|1606|6986|536|2033|0|11|20|4|2033|536|6986|Sunday|2033Q4|N|N|N|2463903|2464206|2463557|2463830|N|N|N|N|N| +2463923|AAAAAAAADLIJFCAA|2033-11-21|1606|6986|536|2033|1|11|21|4|2033|536|6986|Monday|2033Q4|N|N|N|2463903|2464206|2463558|2463831|N|N|N|N|N| +2463924|AAAAAAAAELIJFCAA|2033-11-22|1606|6987|536|2033|2|11|22|4|2033|536|6987|Tuesday|2033Q4|N|N|N|2463903|2464206|2463559|2463832|N|N|N|N|N| +2463925|AAAAAAAAFLIJFCAA|2033-11-23|1606|6987|536|2033|3|11|23|4|2033|536|6987|Wednesday|2033Q4|N|N|N|2463903|2464206|2463560|2463833|N|N|N|N|N| +2463926|AAAAAAAAGLIJFCAA|2033-11-24|1606|6987|536|2033|4|11|24|4|2033|536|6987|Thursday|2033Q4|N|N|N|2463903|2464206|2463561|2463834|N|N|N|N|N| +2463927|AAAAAAAAHLIJFCAA|2033-11-25|1606|6987|536|2033|5|11|25|4|2033|536|6987|Friday|2033Q4|N|Y|N|2463903|2464206|2463562|2463835|N|N|N|N|N| +2463928|AAAAAAAAILIJFCAA|2033-11-26|1606|6987|536|2033|6|11|26|4|2033|536|6987|Saturday|2033Q4|N|Y|N|2463903|2464206|2463563|2463836|N|N|N|N|N| +2463929|AAAAAAAAJLIJFCAA|2033-11-27|1606|6987|536|2033|0|11|27|4|2033|536|6987|Sunday|2033Q4|N|N|N|2463903|2464206|2463564|2463837|N|N|N|N|N| +2463930|AAAAAAAAKLIJFCAA|2033-11-28|1606|6987|536|2033|1|11|28|4|2033|536|6987|Monday|2033Q4|N|N|N|2463903|2464206|2463565|2463838|N|N|N|N|N| +2463931|AAAAAAAALLIJFCAA|2033-11-29|1606|6988|536|2033|2|11|29|4|2033|536|6988|Tuesday|2033Q4|N|N|N|2463903|2464206|2463566|2463839|N|N|N|N|N| +2463932|AAAAAAAAMLIJFCAA|2033-11-30|1606|6988|536|2033|3|11|30|4|2033|536|6988|Wednesday|2033Q4|N|N|N|2463903|2464206|2463567|2463840|N|N|N|N|N| +2463933|AAAAAAAANLIJFCAA|2033-12-01|1607|6988|537|2033|4|12|1|4|2033|537|6988|Thursday|2033Q4|N|N|N|2463933|2464266|2463568|2463841|N|N|N|N|N| +2463934|AAAAAAAAOLIJFCAA|2033-12-02|1607|6988|537|2033|5|12|2|4|2033|537|6988|Friday|2033Q4|N|Y|N|2463933|2464266|2463569|2463842|N|N|N|N|N| +2463935|AAAAAAAAPLIJFCAA|2033-12-03|1607|6988|537|2033|6|12|3|4|2033|537|6988|Saturday|2033Q4|N|Y|N|2463933|2464266|2463570|2463843|N|N|N|N|N| +2463936|AAAAAAAAAMIJFCAA|2033-12-04|1607|6988|537|2033|0|12|4|4|2033|537|6988|Sunday|2033Q4|N|N|N|2463933|2464266|2463571|2463844|N|N|N|N|N| +2463937|AAAAAAAABMIJFCAA|2033-12-05|1607|6988|537|2033|1|12|5|4|2033|537|6988|Monday|2033Q4|N|N|N|2463933|2464266|2463572|2463845|N|N|N|N|N| +2463938|AAAAAAAACMIJFCAA|2033-12-06|1607|6989|537|2033|2|12|6|4|2033|537|6989|Tuesday|2033Q4|N|N|N|2463933|2464266|2463573|2463846|N|N|N|N|N| +2463939|AAAAAAAADMIJFCAA|2033-12-07|1607|6989|537|2033|3|12|7|4|2033|537|6989|Wednesday|2033Q4|N|N|N|2463933|2464266|2463574|2463847|N|N|N|N|N| +2463940|AAAAAAAAEMIJFCAA|2033-12-08|1607|6989|537|2033|4|12|8|4|2033|537|6989|Thursday|2033Q4|N|N|N|2463933|2464266|2463575|2463848|N|N|N|N|N| +2463941|AAAAAAAAFMIJFCAA|2033-12-09|1607|6989|537|2033|5|12|9|4|2033|537|6989|Friday|2033Q4|N|Y|N|2463933|2464266|2463576|2463849|N|N|N|N|N| +2463942|AAAAAAAAGMIJFCAA|2033-12-10|1607|6989|537|2033|6|12|10|4|2033|537|6989|Saturday|2033Q4|N|Y|N|2463933|2464266|2463577|2463850|N|N|N|N|N| +2463943|AAAAAAAAHMIJFCAA|2033-12-11|1607|6989|537|2033|0|12|11|4|2033|537|6989|Sunday|2033Q4|N|N|N|2463933|2464266|2463578|2463851|N|N|N|N|N| +2463944|AAAAAAAAIMIJFCAA|2033-12-12|1607|6989|537|2033|1|12|12|4|2033|537|6989|Monday|2033Q4|N|N|N|2463933|2464266|2463579|2463852|N|N|N|N|N| +2463945|AAAAAAAAJMIJFCAA|2033-12-13|1607|6990|537|2033|2|12|13|4|2033|537|6990|Tuesday|2033Q4|N|N|N|2463933|2464266|2463580|2463853|N|N|N|N|N| +2463946|AAAAAAAAKMIJFCAA|2033-12-14|1607|6990|537|2033|3|12|14|4|2033|537|6990|Wednesday|2033Q4|N|N|N|2463933|2464266|2463581|2463854|N|N|N|N|N| +2463947|AAAAAAAALMIJFCAA|2033-12-15|1607|6990|537|2033|4|12|15|4|2033|537|6990|Thursday|2033Q4|N|N|N|2463933|2464266|2463582|2463855|N|N|N|N|N| +2463948|AAAAAAAAMMIJFCAA|2033-12-16|1607|6990|537|2033|5|12|16|4|2033|537|6990|Friday|2033Q4|N|Y|N|2463933|2464266|2463583|2463856|N|N|N|N|N| +2463949|AAAAAAAANMIJFCAA|2033-12-17|1607|6990|537|2033|6|12|17|4|2033|537|6990|Saturday|2033Q4|N|Y|N|2463933|2464266|2463584|2463857|N|N|N|N|N| +2463950|AAAAAAAAOMIJFCAA|2033-12-18|1607|6990|537|2033|0|12|18|4|2033|537|6990|Sunday|2033Q4|N|N|N|2463933|2464266|2463585|2463858|N|N|N|N|N| +2463951|AAAAAAAAPMIJFCAA|2033-12-19|1607|6990|537|2033|1|12|19|4|2033|537|6990|Monday|2033Q4|N|N|N|2463933|2464266|2463586|2463859|N|N|N|N|N| +2463952|AAAAAAAAANIJFCAA|2033-12-20|1607|6991|537|2033|2|12|20|4|2033|537|6991|Tuesday|2033Q4|N|N|N|2463933|2464266|2463587|2463860|N|N|N|N|N| +2463953|AAAAAAAABNIJFCAA|2033-12-21|1607|6991|537|2033|3|12|21|4|2033|537|6991|Wednesday|2033Q4|N|N|N|2463933|2464266|2463588|2463861|N|N|N|N|N| +2463954|AAAAAAAACNIJFCAA|2033-12-22|1607|6991|537|2033|4|12|22|4|2033|537|6991|Thursday|2033Q4|N|N|N|2463933|2464266|2463589|2463862|N|N|N|N|N| +2463955|AAAAAAAADNIJFCAA|2033-12-23|1607|6991|537|2033|5|12|23|4|2033|537|6991|Friday|2033Q4|N|Y|N|2463933|2464266|2463590|2463863|N|N|N|N|N| +2463956|AAAAAAAAENIJFCAA|2033-12-24|1607|6991|537|2033|6|12|24|4|2033|537|6991|Saturday|2033Q4|N|Y|N|2463933|2464266|2463591|2463864|N|N|N|N|N| +2463957|AAAAAAAAFNIJFCAA|2033-12-25|1607|6991|537|2033|0|12|25|4|2033|537|6991|Sunday|2033Q4|N|N|N|2463933|2464266|2463592|2463865|N|N|N|N|N| +2463958|AAAAAAAAGNIJFCAA|2033-12-26|1607|6991|537|2033|1|12|26|4|2033|537|6991|Monday|2033Q4|Y|N|N|2463933|2464266|2463593|2463866|N|N|N|N|N| +2463959|AAAAAAAAHNIJFCAA|2033-12-27|1607|6992|537|2033|2|12|27|4|2033|537|6992|Tuesday|2033Q4|N|N|Y|2463933|2464266|2463594|2463867|N|N|N|N|N| +2463960|AAAAAAAAINIJFCAA|2033-12-28|1607|6992|537|2033|3|12|28|4|2033|537|6992|Wednesday|2033Q4|N|N|N|2463933|2464266|2463595|2463868|N|N|N|N|N| +2463961|AAAAAAAAJNIJFCAA|2033-12-29|1607|6992|537|2033|4|12|29|4|2033|537|6992|Thursday|2033Q4|N|N|N|2463933|2464266|2463596|2463869|N|N|N|N|N| +2463962|AAAAAAAAKNIJFCAA|2033-12-30|1607|6992|537|2033|5|12|30|4|2033|537|6992|Friday|2033Q4|N|Y|N|2463933|2464266|2463597|2463870|N|N|N|N|N| +2463963|AAAAAAAALNIJFCAA|2033-12-31|1607|6992|537|2033|6|12|31|4|2033|537|6992|Saturday|2033Q4|N|Y|N|2463933|2464266|2463598|2463871|N|N|N|N|N| +2463964|AAAAAAAAMNIJFCAA|2034-01-01|1608|6992|537|2034|0|1|1|1|2034|537|6992|Sunday|2034Q1|Y|N|N|2463964|2463963|2463599|2463872|N|N|N|N|N| +2463965|AAAAAAAANNIJFCAA|2034-01-02|1608|6992|537|2034|1|1|2|1|2034|537|6992|Monday|2034Q1|N|N|Y|2463964|2463963|2463600|2463873|N|N|N|N|N| +2463966|AAAAAAAAONIJFCAA|2034-01-03|1608|6993|537|2034|2|1|3|1|2034|537|6993|Tuesday|2034Q1|N|N|N|2463964|2463963|2463601|2463874|N|N|N|N|N| +2463967|AAAAAAAAPNIJFCAA|2034-01-04|1608|6993|537|2034|3|1|4|1|2034|537|6993|Wednesday|2034Q1|N|N|N|2463964|2463963|2463602|2463875|N|N|N|N|N| +2463968|AAAAAAAAAOIJFCAA|2034-01-05|1608|6993|537|2034|4|1|5|1|2034|537|6993|Thursday|2034Q1|N|N|N|2463964|2463963|2463603|2463876|N|N|N|N|N| +2463969|AAAAAAAABOIJFCAA|2034-01-06|1608|6993|537|2034|5|1|6|1|2034|537|6993|Friday|2034Q1|N|Y|N|2463964|2463963|2463604|2463877|N|N|N|N|N| +2463970|AAAAAAAACOIJFCAA|2034-01-07|1608|6993|537|2034|6|1|7|1|2034|537|6993|Saturday|2034Q1|N|Y|N|2463964|2463963|2463605|2463878|N|N|N|N|N| +2463971|AAAAAAAADOIJFCAA|2034-01-08|1608|6993|537|2034|0|1|8|1|2034|537|6993|Sunday|2034Q1|N|N|N|2463964|2463963|2463606|2463879|N|N|N|N|N| +2463972|AAAAAAAAEOIJFCAA|2034-01-09|1608|6993|537|2034|1|1|9|1|2034|537|6993|Monday|2034Q1|N|N|N|2463964|2463963|2463607|2463880|N|N|N|N|N| +2463973|AAAAAAAAFOIJFCAA|2034-01-10|1608|6994|537|2034|2|1|10|1|2034|537|6994|Tuesday|2034Q1|N|N|N|2463964|2463963|2463608|2463881|N|N|N|N|N| +2463974|AAAAAAAAGOIJFCAA|2034-01-11|1608|6994|537|2034|3|1|11|1|2034|537|6994|Wednesday|2034Q1|N|N|N|2463964|2463963|2463609|2463882|N|N|N|N|N| +2463975|AAAAAAAAHOIJFCAA|2034-01-12|1608|6994|537|2034|4|1|12|1|2034|537|6994|Thursday|2034Q1|N|N|N|2463964|2463963|2463610|2463883|N|N|N|N|N| +2463976|AAAAAAAAIOIJFCAA|2034-01-13|1608|6994|537|2034|5|1|13|1|2034|537|6994|Friday|2034Q1|N|Y|N|2463964|2463963|2463611|2463884|N|N|N|N|N| +2463977|AAAAAAAAJOIJFCAA|2034-01-14|1608|6994|537|2034|6|1|14|1|2034|537|6994|Saturday|2034Q1|N|Y|N|2463964|2463963|2463612|2463885|N|N|N|N|N| +2463978|AAAAAAAAKOIJFCAA|2034-01-15|1608|6994|537|2034|0|1|15|1|2034|537|6994|Sunday|2034Q1|N|N|N|2463964|2463963|2463613|2463886|N|N|N|N|N| +2463979|AAAAAAAALOIJFCAA|2034-01-16|1608|6994|537|2034|1|1|16|1|2034|537|6994|Monday|2034Q1|N|N|N|2463964|2463963|2463614|2463887|N|N|N|N|N| +2463980|AAAAAAAAMOIJFCAA|2034-01-17|1608|6995|537|2034|2|1|17|1|2034|537|6995|Tuesday|2034Q1|N|N|N|2463964|2463963|2463615|2463888|N|N|N|N|N| +2463981|AAAAAAAANOIJFCAA|2034-01-18|1608|6995|537|2034|3|1|18|1|2034|537|6995|Wednesday|2034Q1|N|N|N|2463964|2463963|2463616|2463889|N|N|N|N|N| +2463982|AAAAAAAAOOIJFCAA|2034-01-19|1608|6995|537|2034|4|1|19|1|2034|537|6995|Thursday|2034Q1|N|N|N|2463964|2463963|2463617|2463890|N|N|N|N|N| +2463983|AAAAAAAAPOIJFCAA|2034-01-20|1608|6995|537|2034|5|1|20|1|2034|537|6995|Friday|2034Q1|N|Y|N|2463964|2463963|2463618|2463891|N|N|N|N|N| +2463984|AAAAAAAAAPIJFCAA|2034-01-21|1608|6995|537|2034|6|1|21|1|2034|537|6995|Saturday|2034Q1|N|Y|N|2463964|2463963|2463619|2463892|N|N|N|N|N| +2463985|AAAAAAAABPIJFCAA|2034-01-22|1608|6995|537|2034|0|1|22|1|2034|537|6995|Sunday|2034Q1|N|N|N|2463964|2463963|2463620|2463893|N|N|N|N|N| +2463986|AAAAAAAACPIJFCAA|2034-01-23|1608|6995|537|2034|1|1|23|1|2034|537|6995|Monday|2034Q1|N|N|N|2463964|2463963|2463621|2463894|N|N|N|N|N| +2463987|AAAAAAAADPIJFCAA|2034-01-24|1608|6996|537|2034|2|1|24|1|2034|537|6996|Tuesday|2034Q1|N|N|N|2463964|2463963|2463622|2463895|N|N|N|N|N| +2463988|AAAAAAAAEPIJFCAA|2034-01-25|1608|6996|537|2034|3|1|25|1|2034|537|6996|Wednesday|2034Q1|N|N|N|2463964|2463963|2463623|2463896|N|N|N|N|N| +2463989|AAAAAAAAFPIJFCAA|2034-01-26|1608|6996|537|2034|4|1|26|1|2034|537|6996|Thursday|2034Q1|N|N|N|2463964|2463963|2463624|2463897|N|N|N|N|N| +2463990|AAAAAAAAGPIJFCAA|2034-01-27|1608|6996|537|2034|5|1|27|1|2034|537|6996|Friday|2034Q1|N|Y|N|2463964|2463963|2463625|2463898|N|N|N|N|N| +2463991|AAAAAAAAHPIJFCAA|2034-01-28|1608|6996|537|2034|6|1|28|1|2034|537|6996|Saturday|2034Q1|N|Y|N|2463964|2463963|2463626|2463899|N|N|N|N|N| +2463992|AAAAAAAAIPIJFCAA|2034-01-29|1608|6996|537|2034|0|1|29|1|2034|537|6996|Sunday|2034Q1|N|N|N|2463964|2463963|2463627|2463900|N|N|N|N|N| +2463993|AAAAAAAAJPIJFCAA|2034-01-30|1608|6996|537|2034|1|1|30|1|2034|537|6996|Monday|2034Q1|N|N|N|2463964|2463963|2463628|2463901|N|N|N|N|N| +2463994|AAAAAAAAKPIJFCAA|2034-01-31|1608|6997|537|2034|2|1|31|1|2034|537|6997|Tuesday|2034Q1|N|N|N|2463964|2463963|2463629|2463902|N|N|N|N|N| +2463995|AAAAAAAALPIJFCAA|2034-02-01|1609|6997|537|2034|3|2|1|1|2034|537|6997|Wednesday|2034Q1|N|N|N|2463995|2464025|2463630|2463903|N|N|N|N|N| +2463996|AAAAAAAAMPIJFCAA|2034-02-02|1609|6997|537|2034|4|2|2|1|2034|537|6997|Thursday|2034Q1|N|N|N|2463995|2464025|2463631|2463904|N|N|N|N|N| +2463997|AAAAAAAANPIJFCAA|2034-02-03|1609|6997|537|2034|5|2|3|1|2034|537|6997|Friday|2034Q1|N|Y|N|2463995|2464025|2463632|2463905|N|N|N|N|N| +2463998|AAAAAAAAOPIJFCAA|2034-02-04|1609|6997|537|2034|6|2|4|1|2034|537|6997|Saturday|2034Q1|N|Y|N|2463995|2464025|2463633|2463906|N|N|N|N|N| +2463999|AAAAAAAAPPIJFCAA|2034-02-05|1609|6997|537|2034|0|2|5|1|2034|537|6997|Sunday|2034Q1|N|N|N|2463995|2464025|2463634|2463907|N|N|N|N|N| +2464000|AAAAAAAAAAJJFCAA|2034-02-06|1609|6997|537|2034|1|2|6|1|2034|537|6997|Monday|2034Q1|N|N|N|2463995|2464025|2463635|2463908|N|N|N|N|N| +2464001|AAAAAAAABAJJFCAA|2034-02-07|1609|6998|537|2034|2|2|7|1|2034|537|6998|Tuesday|2034Q1|N|N|N|2463995|2464025|2463636|2463909|N|N|N|N|N| +2464002|AAAAAAAACAJJFCAA|2034-02-08|1609|6998|537|2034|3|2|8|1|2034|537|6998|Wednesday|2034Q1|N|N|N|2463995|2464025|2463637|2463910|N|N|N|N|N| +2464003|AAAAAAAADAJJFCAA|2034-02-09|1609|6998|537|2034|4|2|9|1|2034|537|6998|Thursday|2034Q1|N|N|N|2463995|2464025|2463638|2463911|N|N|N|N|N| +2464004|AAAAAAAAEAJJFCAA|2034-02-10|1609|6998|537|2034|5|2|10|1|2034|537|6998|Friday|2034Q1|N|Y|N|2463995|2464025|2463639|2463912|N|N|N|N|N| +2464005|AAAAAAAAFAJJFCAA|2034-02-11|1609|6998|537|2034|6|2|11|1|2034|537|6998|Saturday|2034Q1|N|Y|N|2463995|2464025|2463640|2463913|N|N|N|N|N| +2464006|AAAAAAAAGAJJFCAA|2034-02-12|1609|6998|537|2034|0|2|12|1|2034|537|6998|Sunday|2034Q1|N|N|N|2463995|2464025|2463641|2463914|N|N|N|N|N| +2464007|AAAAAAAAHAJJFCAA|2034-02-13|1609|6998|537|2034|1|2|13|1|2034|537|6998|Monday|2034Q1|N|N|N|2463995|2464025|2463642|2463915|N|N|N|N|N| +2464008|AAAAAAAAIAJJFCAA|2034-02-14|1609|6999|537|2034|2|2|14|1|2034|537|6999|Tuesday|2034Q1|N|N|N|2463995|2464025|2463643|2463916|N|N|N|N|N| +2464009|AAAAAAAAJAJJFCAA|2034-02-15|1609|6999|537|2034|3|2|15|1|2034|537|6999|Wednesday|2034Q1|N|N|N|2463995|2464025|2463644|2463917|N|N|N|N|N| +2464010|AAAAAAAAKAJJFCAA|2034-02-16|1609|6999|537|2034|4|2|16|1|2034|537|6999|Thursday|2034Q1|N|N|N|2463995|2464025|2463645|2463918|N|N|N|N|N| +2464011|AAAAAAAALAJJFCAA|2034-02-17|1609|6999|537|2034|5|2|17|1|2034|537|6999|Friday|2034Q1|N|Y|N|2463995|2464025|2463646|2463919|N|N|N|N|N| +2464012|AAAAAAAAMAJJFCAA|2034-02-18|1609|6999|537|2034|6|2|18|1|2034|537|6999|Saturday|2034Q1|N|Y|N|2463995|2464025|2463647|2463920|N|N|N|N|N| +2464013|AAAAAAAANAJJFCAA|2034-02-19|1609|6999|537|2034|0|2|19|1|2034|537|6999|Sunday|2034Q1|N|N|N|2463995|2464025|2463648|2463921|N|N|N|N|N| +2464014|AAAAAAAAOAJJFCAA|2034-02-20|1609|6999|537|2034|1|2|20|1|2034|537|6999|Monday|2034Q1|N|N|N|2463995|2464025|2463649|2463922|N|N|N|N|N| +2464015|AAAAAAAAPAJJFCAA|2034-02-21|1609|7000|537|2034|2|2|21|1|2034|537|7000|Tuesday|2034Q1|N|N|N|2463995|2464025|2463650|2463923|N|N|N|N|N| +2464016|AAAAAAAAABJJFCAA|2034-02-22|1609|7000|537|2034|3|2|22|1|2034|537|7000|Wednesday|2034Q1|N|N|N|2463995|2464025|2463651|2463924|N|N|N|N|N| +2464017|AAAAAAAABBJJFCAA|2034-02-23|1609|7000|537|2034|4|2|23|1|2034|537|7000|Thursday|2034Q1|N|N|N|2463995|2464025|2463652|2463925|N|N|N|N|N| +2464018|AAAAAAAACBJJFCAA|2034-02-24|1609|7000|537|2034|5|2|24|1|2034|537|7000|Friday|2034Q1|N|Y|N|2463995|2464025|2463653|2463926|N|N|N|N|N| +2464019|AAAAAAAADBJJFCAA|2034-02-25|1609|7000|537|2034|6|2|25|1|2034|537|7000|Saturday|2034Q1|N|Y|N|2463995|2464025|2463654|2463927|N|N|N|N|N| +2464020|AAAAAAAAEBJJFCAA|2034-02-26|1609|7000|537|2034|0|2|26|1|2034|537|7000|Sunday|2034Q1|N|N|N|2463995|2464025|2463655|2463928|N|N|N|N|N| +2464021|AAAAAAAAFBJJFCAA|2034-02-27|1609|7000|537|2034|1|2|27|1|2034|537|7000|Monday|2034Q1|N|N|N|2463995|2464025|2463656|2463929|N|N|N|N|N| +2464022|AAAAAAAAGBJJFCAA|2034-02-28|1609|7001|537|2034|2|2|28|1|2034|537|7001|Tuesday|2034Q1|N|N|N|2463995|2464025|2463657|2463930|N|N|N|N|N| +2464023|AAAAAAAAHBJJFCAA|2034-03-01|1610|7001|538|2034|3|3|1|1|2034|538|7001|Wednesday|2034Q1|N|N|N|2464023|2464081|2463658|2463931|N|N|N|N|N| +2464024|AAAAAAAAIBJJFCAA|2034-03-02|1610|7001|538|2034|4|3|2|1|2034|538|7001|Thursday|2034Q1|N|N|N|2464023|2464081|2463659|2463932|N|N|N|N|N| +2464025|AAAAAAAAJBJJFCAA|2034-03-03|1610|7001|538|2034|5|3|3|1|2034|538|7001|Friday|2034Q1|N|Y|N|2464023|2464081|2463660|2463933|N|N|N|N|N| +2464026|AAAAAAAAKBJJFCAA|2034-03-04|1610|7001|538|2034|6|3|4|1|2034|538|7001|Saturday|2034Q1|N|Y|N|2464023|2464081|2463661|2463934|N|N|N|N|N| +2464027|AAAAAAAALBJJFCAA|2034-03-05|1610|7001|538|2034|0|3|5|1|2034|538|7001|Sunday|2034Q1|N|N|N|2464023|2464081|2463662|2463935|N|N|N|N|N| +2464028|AAAAAAAAMBJJFCAA|2034-03-06|1610|7001|538|2034|1|3|6|1|2034|538|7001|Monday|2034Q1|N|N|N|2464023|2464081|2463663|2463936|N|N|N|N|N| +2464029|AAAAAAAANBJJFCAA|2034-03-07|1610|7002|538|2034|2|3|7|1|2034|538|7002|Tuesday|2034Q1|N|N|N|2464023|2464081|2463664|2463937|N|N|N|N|N| +2464030|AAAAAAAAOBJJFCAA|2034-03-08|1610|7002|538|2034|3|3|8|1|2034|538|7002|Wednesday|2034Q1|N|N|N|2464023|2464081|2463665|2463938|N|N|N|N|N| +2464031|AAAAAAAAPBJJFCAA|2034-03-09|1610|7002|538|2034|4|3|9|1|2034|538|7002|Thursday|2034Q1|N|N|N|2464023|2464081|2463666|2463939|N|N|N|N|N| +2464032|AAAAAAAAACJJFCAA|2034-03-10|1610|7002|538|2034|5|3|10|1|2034|538|7002|Friday|2034Q1|N|Y|N|2464023|2464081|2463667|2463940|N|N|N|N|N| +2464033|AAAAAAAABCJJFCAA|2034-03-11|1610|7002|538|2034|6|3|11|1|2034|538|7002|Saturday|2034Q1|N|Y|N|2464023|2464081|2463668|2463941|N|N|N|N|N| +2464034|AAAAAAAACCJJFCAA|2034-03-12|1610|7002|538|2034|0|3|12|1|2034|538|7002|Sunday|2034Q1|N|N|N|2464023|2464081|2463669|2463942|N|N|N|N|N| +2464035|AAAAAAAADCJJFCAA|2034-03-13|1610|7002|538|2034|1|3|13|1|2034|538|7002|Monday|2034Q1|N|N|N|2464023|2464081|2463670|2463943|N|N|N|N|N| +2464036|AAAAAAAAECJJFCAA|2034-03-14|1610|7003|538|2034|2|3|14|1|2034|538|7003|Tuesday|2034Q1|N|N|N|2464023|2464081|2463671|2463944|N|N|N|N|N| +2464037|AAAAAAAAFCJJFCAA|2034-03-15|1610|7003|538|2034|3|3|15|1|2034|538|7003|Wednesday|2034Q1|N|N|N|2464023|2464081|2463672|2463945|N|N|N|N|N| +2464038|AAAAAAAAGCJJFCAA|2034-03-16|1610|7003|538|2034|4|3|16|1|2034|538|7003|Thursday|2034Q1|N|N|N|2464023|2464081|2463673|2463946|N|N|N|N|N| +2464039|AAAAAAAAHCJJFCAA|2034-03-17|1610|7003|538|2034|5|3|17|1|2034|538|7003|Friday|2034Q1|N|Y|N|2464023|2464081|2463674|2463947|N|N|N|N|N| +2464040|AAAAAAAAICJJFCAA|2034-03-18|1610|7003|538|2034|6|3|18|1|2034|538|7003|Saturday|2034Q1|N|Y|N|2464023|2464081|2463675|2463948|N|N|N|N|N| +2464041|AAAAAAAAJCJJFCAA|2034-03-19|1610|7003|538|2034|0|3|19|1|2034|538|7003|Sunday|2034Q1|N|N|N|2464023|2464081|2463676|2463949|N|N|N|N|N| +2464042|AAAAAAAAKCJJFCAA|2034-03-20|1610|7003|538|2034|1|3|20|1|2034|538|7003|Monday|2034Q1|N|N|N|2464023|2464081|2463677|2463950|N|N|N|N|N| +2464043|AAAAAAAALCJJFCAA|2034-03-21|1610|7004|538|2034|2|3|21|1|2034|538|7004|Tuesday|2034Q1|N|N|N|2464023|2464081|2463678|2463951|N|N|N|N|N| +2464044|AAAAAAAAMCJJFCAA|2034-03-22|1610|7004|538|2034|3|3|22|1|2034|538|7004|Wednesday|2034Q1|N|N|N|2464023|2464081|2463679|2463952|N|N|N|N|N| +2464045|AAAAAAAANCJJFCAA|2034-03-23|1610|7004|538|2034|4|3|23|1|2034|538|7004|Thursday|2034Q1|N|N|N|2464023|2464081|2463680|2463953|N|N|N|N|N| +2464046|AAAAAAAAOCJJFCAA|2034-03-24|1610|7004|538|2034|5|3|24|1|2034|538|7004|Friday|2034Q1|N|Y|N|2464023|2464081|2463681|2463954|N|N|N|N|N| +2464047|AAAAAAAAPCJJFCAA|2034-03-25|1610|7004|538|2034|6|3|25|1|2034|538|7004|Saturday|2034Q1|N|Y|N|2464023|2464081|2463682|2463955|N|N|N|N|N| +2464048|AAAAAAAAADJJFCAA|2034-03-26|1610|7004|538|2034|0|3|26|1|2034|538|7004|Sunday|2034Q1|N|N|N|2464023|2464081|2463683|2463956|N|N|N|N|N| +2464049|AAAAAAAABDJJFCAA|2034-03-27|1610|7004|538|2034|1|3|27|1|2034|538|7004|Monday|2034Q1|N|N|N|2464023|2464081|2463684|2463957|N|N|N|N|N| +2464050|AAAAAAAACDJJFCAA|2034-03-28|1610|7005|538|2034|2|3|28|1|2034|538|7005|Tuesday|2034Q1|N|N|N|2464023|2464081|2463685|2463958|N|N|N|N|N| +2464051|AAAAAAAADDJJFCAA|2034-03-29|1610|7005|538|2034|3|3|29|1|2034|538|7005|Wednesday|2034Q1|N|N|N|2464023|2464081|2463686|2463959|N|N|N|N|N| +2464052|AAAAAAAAEDJJFCAA|2034-03-30|1610|7005|538|2034|4|3|30|1|2034|538|7005|Thursday|2034Q1|N|N|N|2464023|2464081|2463687|2463960|N|N|N|N|N| +2464053|AAAAAAAAFDJJFCAA|2034-03-31|1610|7005|538|2034|5|3|31|1|2034|538|7005|Friday|2034Q1|N|Y|N|2464023|2464081|2463688|2463961|N|N|N|N|N| +2464054|AAAAAAAAGDJJFCAA|2034-04-01|1611|7005|538|2034|6|4|1|1|2034|538|7005|Saturday|2034Q1|N|Y|N|2464054|2464143|2463689|2463964|N|N|N|N|N| +2464055|AAAAAAAAHDJJFCAA|2034-04-02|1611|7005|538|2034|0|4|2|2|2034|538|7005|Sunday|2034Q2|N|N|N|2464054|2464143|2463690|2463965|N|N|N|N|N| +2464056|AAAAAAAAIDJJFCAA|2034-04-03|1611|7005|538|2034|1|4|3|2|2034|538|7005|Monday|2034Q2|N|N|N|2464054|2464143|2463691|2463966|N|N|N|N|N| +2464057|AAAAAAAAJDJJFCAA|2034-04-04|1611|7006|538|2034|2|4|4|2|2034|538|7006|Tuesday|2034Q2|N|N|N|2464054|2464143|2463692|2463967|N|N|N|N|N| +2464058|AAAAAAAAKDJJFCAA|2034-04-05|1611|7006|538|2034|3|4|5|2|2034|538|7006|Wednesday|2034Q2|N|N|N|2464054|2464143|2463693|2463968|N|N|N|N|N| +2464059|AAAAAAAALDJJFCAA|2034-04-06|1611|7006|538|2034|4|4|6|2|2034|538|7006|Thursday|2034Q2|N|N|N|2464054|2464143|2463694|2463969|N|N|N|N|N| +2464060|AAAAAAAAMDJJFCAA|2034-04-07|1611|7006|538|2034|5|4|7|2|2034|538|7006|Friday|2034Q2|N|Y|N|2464054|2464143|2463695|2463970|N|N|N|N|N| +2464061|AAAAAAAANDJJFCAA|2034-04-08|1611|7006|538|2034|6|4|8|2|2034|538|7006|Saturday|2034Q2|N|Y|N|2464054|2464143|2463696|2463971|N|N|N|N|N| +2464062|AAAAAAAAODJJFCAA|2034-04-09|1611|7006|538|2034|0|4|9|2|2034|538|7006|Sunday|2034Q2|N|N|N|2464054|2464143|2463697|2463972|N|N|N|N|N| +2464063|AAAAAAAAPDJJFCAA|2034-04-10|1611|7006|538|2034|1|4|10|2|2034|538|7006|Monday|2034Q2|N|N|N|2464054|2464143|2463698|2463973|N|N|N|N|N| +2464064|AAAAAAAAAEJJFCAA|2034-04-11|1611|7007|538|2034|2|4|11|2|2034|538|7007|Tuesday|2034Q2|N|N|N|2464054|2464143|2463699|2463974|N|N|N|N|N| +2464065|AAAAAAAABEJJFCAA|2034-04-12|1611|7007|538|2034|3|4|12|2|2034|538|7007|Wednesday|2034Q2|N|N|N|2464054|2464143|2463700|2463975|N|N|N|N|N| +2464066|AAAAAAAACEJJFCAA|2034-04-13|1611|7007|538|2034|4|4|13|2|2034|538|7007|Thursday|2034Q2|N|N|N|2464054|2464143|2463701|2463976|N|N|N|N|N| +2464067|AAAAAAAADEJJFCAA|2034-04-14|1611|7007|538|2034|5|4|14|2|2034|538|7007|Friday|2034Q2|N|Y|N|2464054|2464143|2463702|2463977|N|N|N|N|N| +2464068|AAAAAAAAEEJJFCAA|2034-04-15|1611|7007|538|2034|6|4|15|2|2034|538|7007|Saturday|2034Q2|N|Y|N|2464054|2464143|2463703|2463978|N|N|N|N|N| +2464069|AAAAAAAAFEJJFCAA|2034-04-16|1611|7007|538|2034|0|4|16|2|2034|538|7007|Sunday|2034Q2|N|N|N|2464054|2464143|2463704|2463979|N|N|N|N|N| +2464070|AAAAAAAAGEJJFCAA|2034-04-17|1611|7007|538|2034|1|4|17|2|2034|538|7007|Monday|2034Q2|N|N|N|2464054|2464143|2463705|2463980|N|N|N|N|N| +2464071|AAAAAAAAHEJJFCAA|2034-04-18|1611|7008|538|2034|2|4|18|2|2034|538|7008|Tuesday|2034Q2|N|N|N|2464054|2464143|2463706|2463981|N|N|N|N|N| +2464072|AAAAAAAAIEJJFCAA|2034-04-19|1611|7008|538|2034|3|4|19|2|2034|538|7008|Wednesday|2034Q2|N|N|N|2464054|2464143|2463707|2463982|N|N|N|N|N| +2464073|AAAAAAAAJEJJFCAA|2034-04-20|1611|7008|538|2034|4|4|20|2|2034|538|7008|Thursday|2034Q2|N|N|N|2464054|2464143|2463708|2463983|N|N|N|N|N| +2464074|AAAAAAAAKEJJFCAA|2034-04-21|1611|7008|538|2034|5|4|21|2|2034|538|7008|Friday|2034Q2|N|Y|N|2464054|2464143|2463709|2463984|N|N|N|N|N| +2464075|AAAAAAAALEJJFCAA|2034-04-22|1611|7008|538|2034|6|4|22|2|2034|538|7008|Saturday|2034Q2|N|Y|N|2464054|2464143|2463710|2463985|N|N|N|N|N| +2464076|AAAAAAAAMEJJFCAA|2034-04-23|1611|7008|538|2034|0|4|23|2|2034|538|7008|Sunday|2034Q2|N|N|N|2464054|2464143|2463711|2463986|N|N|N|N|N| +2464077|AAAAAAAANEJJFCAA|2034-04-24|1611|7008|538|2034|1|4|24|2|2034|538|7008|Monday|2034Q2|N|N|N|2464054|2464143|2463712|2463987|N|N|N|N|N| +2464078|AAAAAAAAOEJJFCAA|2034-04-25|1611|7009|538|2034|2|4|25|2|2034|538|7009|Tuesday|2034Q2|N|N|N|2464054|2464143|2463713|2463988|N|N|N|N|N| +2464079|AAAAAAAAPEJJFCAA|2034-04-26|1611|7009|538|2034|3|4|26|2|2034|538|7009|Wednesday|2034Q2|N|N|N|2464054|2464143|2463714|2463989|N|N|N|N|N| +2464080|AAAAAAAAAFJJFCAA|2034-04-27|1611|7009|538|2034|4|4|27|2|2034|538|7009|Thursday|2034Q2|N|N|N|2464054|2464143|2463715|2463990|N|N|N|N|N| +2464081|AAAAAAAABFJJFCAA|2034-04-28|1611|7009|538|2034|5|4|28|2|2034|538|7009|Friday|2034Q2|N|Y|N|2464054|2464143|2463716|2463991|N|N|N|N|N| +2464082|AAAAAAAACFJJFCAA|2034-04-29|1611|7009|538|2034|6|4|29|2|2034|538|7009|Saturday|2034Q2|N|Y|N|2464054|2464143|2463717|2463992|N|N|N|N|N| +2464083|AAAAAAAADFJJFCAA|2034-04-30|1611|7009|538|2034|0|4|30|2|2034|538|7009|Sunday|2034Q2|N|N|N|2464054|2464143|2463718|2463993|N|N|N|N|N| +2464084|AAAAAAAAEFJJFCAA|2034-05-01|1612|7009|538|2034|1|5|1|2|2034|538|7009|Monday|2034Q2|N|N|N|2464084|2464203|2463719|2463994|N|N|N|N|N| +2464085|AAAAAAAAFFJJFCAA|2034-05-02|1612|7010|538|2034|2|5|2|2|2034|538|7010|Tuesday|2034Q2|N|N|N|2464084|2464203|2463720|2463995|N|N|N|N|N| +2464086|AAAAAAAAGFJJFCAA|2034-05-03|1612|7010|538|2034|3|5|3|2|2034|538|7010|Wednesday|2034Q2|N|N|N|2464084|2464203|2463721|2463996|N|N|N|N|N| +2464087|AAAAAAAAHFJJFCAA|2034-05-04|1612|7010|538|2034|4|5|4|2|2034|538|7010|Thursday|2034Q2|N|N|N|2464084|2464203|2463722|2463997|N|N|N|N|N| +2464088|AAAAAAAAIFJJFCAA|2034-05-05|1612|7010|538|2034|5|5|5|2|2034|538|7010|Friday|2034Q2|N|Y|N|2464084|2464203|2463723|2463998|N|N|N|N|N| +2464089|AAAAAAAAJFJJFCAA|2034-05-06|1612|7010|538|2034|6|5|6|2|2034|538|7010|Saturday|2034Q2|N|Y|N|2464084|2464203|2463724|2463999|N|N|N|N|N| +2464090|AAAAAAAAKFJJFCAA|2034-05-07|1612|7010|538|2034|0|5|7|2|2034|538|7010|Sunday|2034Q2|N|N|N|2464084|2464203|2463725|2464000|N|N|N|N|N| +2464091|AAAAAAAALFJJFCAA|2034-05-08|1612|7010|538|2034|1|5|8|2|2034|538|7010|Monday|2034Q2|N|N|N|2464084|2464203|2463726|2464001|N|N|N|N|N| +2464092|AAAAAAAAMFJJFCAA|2034-05-09|1612|7011|538|2034|2|5|9|2|2034|538|7011|Tuesday|2034Q2|N|N|N|2464084|2464203|2463727|2464002|N|N|N|N|N| +2464093|AAAAAAAANFJJFCAA|2034-05-10|1612|7011|538|2034|3|5|10|2|2034|538|7011|Wednesday|2034Q2|N|N|N|2464084|2464203|2463728|2464003|N|N|N|N|N| +2464094|AAAAAAAAOFJJFCAA|2034-05-11|1612|7011|538|2034|4|5|11|2|2034|538|7011|Thursday|2034Q2|N|N|N|2464084|2464203|2463729|2464004|N|N|N|N|N| +2464095|AAAAAAAAPFJJFCAA|2034-05-12|1612|7011|538|2034|5|5|12|2|2034|538|7011|Friday|2034Q2|N|Y|N|2464084|2464203|2463730|2464005|N|N|N|N|N| +2464096|AAAAAAAAAGJJFCAA|2034-05-13|1612|7011|538|2034|6|5|13|2|2034|538|7011|Saturday|2034Q2|N|Y|N|2464084|2464203|2463731|2464006|N|N|N|N|N| +2464097|AAAAAAAABGJJFCAA|2034-05-14|1612|7011|538|2034|0|5|14|2|2034|538|7011|Sunday|2034Q2|N|N|N|2464084|2464203|2463732|2464007|N|N|N|N|N| +2464098|AAAAAAAACGJJFCAA|2034-05-15|1612|7011|538|2034|1|5|15|2|2034|538|7011|Monday|2034Q2|N|N|N|2464084|2464203|2463733|2464008|N|N|N|N|N| +2464099|AAAAAAAADGJJFCAA|2034-05-16|1612|7012|538|2034|2|5|16|2|2034|538|7012|Tuesday|2034Q2|N|N|N|2464084|2464203|2463734|2464009|N|N|N|N|N| +2464100|AAAAAAAAEGJJFCAA|2034-05-17|1612|7012|538|2034|3|5|17|2|2034|538|7012|Wednesday|2034Q2|N|N|N|2464084|2464203|2463735|2464010|N|N|N|N|N| +2464101|AAAAAAAAFGJJFCAA|2034-05-18|1612|7012|538|2034|4|5|18|2|2034|538|7012|Thursday|2034Q2|N|N|N|2464084|2464203|2463736|2464011|N|N|N|N|N| +2464102|AAAAAAAAGGJJFCAA|2034-05-19|1612|7012|538|2034|5|5|19|2|2034|538|7012|Friday|2034Q2|N|Y|N|2464084|2464203|2463737|2464012|N|N|N|N|N| +2464103|AAAAAAAAHGJJFCAA|2034-05-20|1612|7012|538|2034|6|5|20|2|2034|538|7012|Saturday|2034Q2|N|Y|N|2464084|2464203|2463738|2464013|N|N|N|N|N| +2464104|AAAAAAAAIGJJFCAA|2034-05-21|1612|7012|538|2034|0|5|21|2|2034|538|7012|Sunday|2034Q2|N|N|N|2464084|2464203|2463739|2464014|N|N|N|N|N| +2464105|AAAAAAAAJGJJFCAA|2034-05-22|1612|7012|538|2034|1|5|22|2|2034|538|7012|Monday|2034Q2|N|N|N|2464084|2464203|2463740|2464015|N|N|N|N|N| +2464106|AAAAAAAAKGJJFCAA|2034-05-23|1612|7013|538|2034|2|5|23|2|2034|538|7013|Tuesday|2034Q2|N|N|N|2464084|2464203|2463741|2464016|N|N|N|N|N| +2464107|AAAAAAAALGJJFCAA|2034-05-24|1612|7013|538|2034|3|5|24|2|2034|538|7013|Wednesday|2034Q2|N|N|N|2464084|2464203|2463742|2464017|N|N|N|N|N| +2464108|AAAAAAAAMGJJFCAA|2034-05-25|1612|7013|538|2034|4|5|25|2|2034|538|7013|Thursday|2034Q2|N|N|N|2464084|2464203|2463743|2464018|N|N|N|N|N| +2464109|AAAAAAAANGJJFCAA|2034-05-26|1612|7013|538|2034|5|5|26|2|2034|538|7013|Friday|2034Q2|N|Y|N|2464084|2464203|2463744|2464019|N|N|N|N|N| +2464110|AAAAAAAAOGJJFCAA|2034-05-27|1612|7013|538|2034|6|5|27|2|2034|538|7013|Saturday|2034Q2|N|Y|N|2464084|2464203|2463745|2464020|N|N|N|N|N| +2464111|AAAAAAAAPGJJFCAA|2034-05-28|1612|7013|538|2034|0|5|28|2|2034|538|7013|Sunday|2034Q2|N|N|N|2464084|2464203|2463746|2464021|N|N|N|N|N| +2464112|AAAAAAAAAHJJFCAA|2034-05-29|1612|7013|538|2034|1|5|29|2|2034|538|7013|Monday|2034Q2|N|N|N|2464084|2464203|2463747|2464022|N|N|N|N|N| +2464113|AAAAAAAABHJJFCAA|2034-05-30|1612|7014|538|2034|2|5|30|2|2034|538|7014|Tuesday|2034Q2|N|N|N|2464084|2464203|2463748|2464023|N|N|N|N|N| +2464114|AAAAAAAACHJJFCAA|2034-05-31|1612|7014|538|2034|3|5|31|2|2034|538|7014|Wednesday|2034Q2|N|N|N|2464084|2464203|2463749|2464024|N|N|N|N|N| +2464115|AAAAAAAADHJJFCAA|2034-06-01|1613|7014|539|2034|4|6|1|2|2034|539|7014|Thursday|2034Q2|N|N|N|2464115|2464265|2463750|2464025|N|N|N|N|N| +2464116|AAAAAAAAEHJJFCAA|2034-06-02|1613|7014|539|2034|5|6|2|2|2034|539|7014|Friday|2034Q2|N|Y|N|2464115|2464265|2463751|2464026|N|N|N|N|N| +2464117|AAAAAAAAFHJJFCAA|2034-06-03|1613|7014|539|2034|6|6|3|2|2034|539|7014|Saturday|2034Q2|N|Y|N|2464115|2464265|2463752|2464027|N|N|N|N|N| +2464118|AAAAAAAAGHJJFCAA|2034-06-04|1613|7014|539|2034|0|6|4|2|2034|539|7014|Sunday|2034Q2|N|N|N|2464115|2464265|2463753|2464028|N|N|N|N|N| +2464119|AAAAAAAAHHJJFCAA|2034-06-05|1613|7014|539|2034|1|6|5|2|2034|539|7014|Monday|2034Q2|N|N|N|2464115|2464265|2463754|2464029|N|N|N|N|N| +2464120|AAAAAAAAIHJJFCAA|2034-06-06|1613|7015|539|2034|2|6|6|2|2034|539|7015|Tuesday|2034Q2|N|N|N|2464115|2464265|2463755|2464030|N|N|N|N|N| +2464121|AAAAAAAAJHJJFCAA|2034-06-07|1613|7015|539|2034|3|6|7|2|2034|539|7015|Wednesday|2034Q2|N|N|N|2464115|2464265|2463756|2464031|N|N|N|N|N| +2464122|AAAAAAAAKHJJFCAA|2034-06-08|1613|7015|539|2034|4|6|8|2|2034|539|7015|Thursday|2034Q2|N|N|N|2464115|2464265|2463757|2464032|N|N|N|N|N| +2464123|AAAAAAAALHJJFCAA|2034-06-09|1613|7015|539|2034|5|6|9|2|2034|539|7015|Friday|2034Q2|N|Y|N|2464115|2464265|2463758|2464033|N|N|N|N|N| +2464124|AAAAAAAAMHJJFCAA|2034-06-10|1613|7015|539|2034|6|6|10|2|2034|539|7015|Saturday|2034Q2|N|Y|N|2464115|2464265|2463759|2464034|N|N|N|N|N| +2464125|AAAAAAAANHJJFCAA|2034-06-11|1613|7015|539|2034|0|6|11|2|2034|539|7015|Sunday|2034Q2|N|N|N|2464115|2464265|2463760|2464035|N|N|N|N|N| +2464126|AAAAAAAAOHJJFCAA|2034-06-12|1613|7015|539|2034|1|6|12|2|2034|539|7015|Monday|2034Q2|N|N|N|2464115|2464265|2463761|2464036|N|N|N|N|N| +2464127|AAAAAAAAPHJJFCAA|2034-06-13|1613|7016|539|2034|2|6|13|2|2034|539|7016|Tuesday|2034Q2|N|N|N|2464115|2464265|2463762|2464037|N|N|N|N|N| +2464128|AAAAAAAAAIJJFCAA|2034-06-14|1613|7016|539|2034|3|6|14|2|2034|539|7016|Wednesday|2034Q2|N|N|N|2464115|2464265|2463763|2464038|N|N|N|N|N| +2464129|AAAAAAAABIJJFCAA|2034-06-15|1613|7016|539|2034|4|6|15|2|2034|539|7016|Thursday|2034Q2|N|N|N|2464115|2464265|2463764|2464039|N|N|N|N|N| +2464130|AAAAAAAACIJJFCAA|2034-06-16|1613|7016|539|2034|5|6|16|2|2034|539|7016|Friday|2034Q2|N|Y|N|2464115|2464265|2463765|2464040|N|N|N|N|N| +2464131|AAAAAAAADIJJFCAA|2034-06-17|1613|7016|539|2034|6|6|17|2|2034|539|7016|Saturday|2034Q2|N|Y|N|2464115|2464265|2463766|2464041|N|N|N|N|N| +2464132|AAAAAAAAEIJJFCAA|2034-06-18|1613|7016|539|2034|0|6|18|2|2034|539|7016|Sunday|2034Q2|N|N|N|2464115|2464265|2463767|2464042|N|N|N|N|N| +2464133|AAAAAAAAFIJJFCAA|2034-06-19|1613|7016|539|2034|1|6|19|2|2034|539|7016|Monday|2034Q2|N|N|N|2464115|2464265|2463768|2464043|N|N|N|N|N| +2464134|AAAAAAAAGIJJFCAA|2034-06-20|1613|7017|539|2034|2|6|20|2|2034|539|7017|Tuesday|2034Q2|N|N|N|2464115|2464265|2463769|2464044|N|N|N|N|N| +2464135|AAAAAAAAHIJJFCAA|2034-06-21|1613|7017|539|2034|3|6|21|2|2034|539|7017|Wednesday|2034Q2|N|N|N|2464115|2464265|2463770|2464045|N|N|N|N|N| +2464136|AAAAAAAAIIJJFCAA|2034-06-22|1613|7017|539|2034|4|6|22|2|2034|539|7017|Thursday|2034Q2|N|N|N|2464115|2464265|2463771|2464046|N|N|N|N|N| +2464137|AAAAAAAAJIJJFCAA|2034-06-23|1613|7017|539|2034|5|6|23|2|2034|539|7017|Friday|2034Q2|N|Y|N|2464115|2464265|2463772|2464047|N|N|N|N|N| +2464138|AAAAAAAAKIJJFCAA|2034-06-24|1613|7017|539|2034|6|6|24|2|2034|539|7017|Saturday|2034Q2|N|Y|N|2464115|2464265|2463773|2464048|N|N|N|N|N| +2464139|AAAAAAAALIJJFCAA|2034-06-25|1613|7017|539|2034|0|6|25|2|2034|539|7017|Sunday|2034Q2|N|N|N|2464115|2464265|2463774|2464049|N|N|N|N|N| +2464140|AAAAAAAAMIJJFCAA|2034-06-26|1613|7017|539|2034|1|6|26|2|2034|539|7017|Monday|2034Q2|N|N|N|2464115|2464265|2463775|2464050|N|N|N|N|N| +2464141|AAAAAAAANIJJFCAA|2034-06-27|1613|7018|539|2034|2|6|27|2|2034|539|7018|Tuesday|2034Q2|N|N|N|2464115|2464265|2463776|2464051|N|N|N|N|N| +2464142|AAAAAAAAOIJJFCAA|2034-06-28|1613|7018|539|2034|3|6|28|2|2034|539|7018|Wednesday|2034Q2|N|N|N|2464115|2464265|2463777|2464052|N|N|N|N|N| +2464143|AAAAAAAAPIJJFCAA|2034-06-29|1613|7018|539|2034|4|6|29|2|2034|539|7018|Thursday|2034Q2|N|N|N|2464115|2464265|2463778|2464053|N|N|N|N|N| +2464144|AAAAAAAAAJJJFCAA|2034-06-30|1613|7018|539|2034|5|6|30|2|2034|539|7018|Friday|2034Q2|N|Y|N|2464115|2464265|2463779|2464054|N|N|N|N|N| +2464145|AAAAAAAABJJJFCAA|2034-07-01|1614|7018|539|2034|6|7|1|2|2034|539|7018|Saturday|2034Q2|N|Y|N|2464145|2464325|2463780|2464054|N|N|N|N|N| +2464146|AAAAAAAACJJJFCAA|2034-07-02|1614|7018|539|2034|0|7|2|3|2034|539|7018|Sunday|2034Q3|N|N|N|2464145|2464325|2463781|2464055|N|N|N|N|N| +2464147|AAAAAAAADJJJFCAA|2034-07-03|1614|7018|539|2034|1|7|3|3|2034|539|7018|Monday|2034Q3|N|N|N|2464145|2464325|2463782|2464056|N|N|N|N|N| +2464148|AAAAAAAAEJJJFCAA|2034-07-04|1614|7019|539|2034|2|7|4|3|2034|539|7019|Tuesday|2034Q3|N|N|N|2464145|2464325|2463783|2464057|N|N|N|N|N| +2464149|AAAAAAAAFJJJFCAA|2034-07-05|1614|7019|539|2034|3|7|5|3|2034|539|7019|Wednesday|2034Q3|Y|N|N|2464145|2464325|2463784|2464058|N|N|N|N|N| +2464150|AAAAAAAAGJJJFCAA|2034-07-06|1614|7019|539|2034|4|7|6|3|2034|539|7019|Thursday|2034Q3|N|N|Y|2464145|2464325|2463785|2464059|N|N|N|N|N| +2464151|AAAAAAAAHJJJFCAA|2034-07-07|1614|7019|539|2034|5|7|7|3|2034|539|7019|Friday|2034Q3|N|Y|N|2464145|2464325|2463786|2464060|N|N|N|N|N| +2464152|AAAAAAAAIJJJFCAA|2034-07-08|1614|7019|539|2034|6|7|8|3|2034|539|7019|Saturday|2034Q3|N|Y|N|2464145|2464325|2463787|2464061|N|N|N|N|N| +2464153|AAAAAAAAJJJJFCAA|2034-07-09|1614|7019|539|2034|0|7|9|3|2034|539|7019|Sunday|2034Q3|N|N|N|2464145|2464325|2463788|2464062|N|N|N|N|N| +2464154|AAAAAAAAKJJJFCAA|2034-07-10|1614|7019|539|2034|1|7|10|3|2034|539|7019|Monday|2034Q3|N|N|N|2464145|2464325|2463789|2464063|N|N|N|N|N| +2464155|AAAAAAAALJJJFCAA|2034-07-11|1614|7020|539|2034|2|7|11|3|2034|539|7020|Tuesday|2034Q3|N|N|N|2464145|2464325|2463790|2464064|N|N|N|N|N| +2464156|AAAAAAAAMJJJFCAA|2034-07-12|1614|7020|539|2034|3|7|12|3|2034|539|7020|Wednesday|2034Q3|N|N|N|2464145|2464325|2463791|2464065|N|N|N|N|N| +2464157|AAAAAAAANJJJFCAA|2034-07-13|1614|7020|539|2034|4|7|13|3|2034|539|7020|Thursday|2034Q3|N|N|N|2464145|2464325|2463792|2464066|N|N|N|N|N| +2464158|AAAAAAAAOJJJFCAA|2034-07-14|1614|7020|539|2034|5|7|14|3|2034|539|7020|Friday|2034Q3|N|Y|N|2464145|2464325|2463793|2464067|N|N|N|N|N| +2464159|AAAAAAAAPJJJFCAA|2034-07-15|1614|7020|539|2034|6|7|15|3|2034|539|7020|Saturday|2034Q3|N|Y|N|2464145|2464325|2463794|2464068|N|N|N|N|N| +2464160|AAAAAAAAAKJJFCAA|2034-07-16|1614|7020|539|2034|0|7|16|3|2034|539|7020|Sunday|2034Q3|N|N|N|2464145|2464325|2463795|2464069|N|N|N|N|N| +2464161|AAAAAAAABKJJFCAA|2034-07-17|1614|7020|539|2034|1|7|17|3|2034|539|7020|Monday|2034Q3|N|N|N|2464145|2464325|2463796|2464070|N|N|N|N|N| +2464162|AAAAAAAACKJJFCAA|2034-07-18|1614|7021|539|2034|2|7|18|3|2034|539|7021|Tuesday|2034Q3|N|N|N|2464145|2464325|2463797|2464071|N|N|N|N|N| +2464163|AAAAAAAADKJJFCAA|2034-07-19|1614|7021|539|2034|3|7|19|3|2034|539|7021|Wednesday|2034Q3|N|N|N|2464145|2464325|2463798|2464072|N|N|N|N|N| +2464164|AAAAAAAAEKJJFCAA|2034-07-20|1614|7021|539|2034|4|7|20|3|2034|539|7021|Thursday|2034Q3|N|N|N|2464145|2464325|2463799|2464073|N|N|N|N|N| +2464165|AAAAAAAAFKJJFCAA|2034-07-21|1614|7021|539|2034|5|7|21|3|2034|539|7021|Friday|2034Q3|N|Y|N|2464145|2464325|2463800|2464074|N|N|N|N|N| +2464166|AAAAAAAAGKJJFCAA|2034-07-22|1614|7021|539|2034|6|7|22|3|2034|539|7021|Saturday|2034Q3|N|Y|N|2464145|2464325|2463801|2464075|N|N|N|N|N| +2464167|AAAAAAAAHKJJFCAA|2034-07-23|1614|7021|539|2034|0|7|23|3|2034|539|7021|Sunday|2034Q3|N|N|N|2464145|2464325|2463802|2464076|N|N|N|N|N| +2464168|AAAAAAAAIKJJFCAA|2034-07-24|1614|7021|539|2034|1|7|24|3|2034|539|7021|Monday|2034Q3|N|N|N|2464145|2464325|2463803|2464077|N|N|N|N|N| +2464169|AAAAAAAAJKJJFCAA|2034-07-25|1614|7022|539|2034|2|7|25|3|2034|539|7022|Tuesday|2034Q3|N|N|N|2464145|2464325|2463804|2464078|N|N|N|N|N| +2464170|AAAAAAAAKKJJFCAA|2034-07-26|1614|7022|539|2034|3|7|26|3|2034|539|7022|Wednesday|2034Q3|N|N|N|2464145|2464325|2463805|2464079|N|N|N|N|N| +2464171|AAAAAAAALKJJFCAA|2034-07-27|1614|7022|539|2034|4|7|27|3|2034|539|7022|Thursday|2034Q3|N|N|N|2464145|2464325|2463806|2464080|N|N|N|N|N| +2464172|AAAAAAAAMKJJFCAA|2034-07-28|1614|7022|539|2034|5|7|28|3|2034|539|7022|Friday|2034Q3|N|Y|N|2464145|2464325|2463807|2464081|N|N|N|N|N| +2464173|AAAAAAAANKJJFCAA|2034-07-29|1614|7022|539|2034|6|7|29|3|2034|539|7022|Saturday|2034Q3|N|Y|N|2464145|2464325|2463808|2464082|N|N|N|N|N| +2464174|AAAAAAAAOKJJFCAA|2034-07-30|1614|7022|539|2034|0|7|30|3|2034|539|7022|Sunday|2034Q3|N|N|N|2464145|2464325|2463809|2464083|N|N|N|N|N| +2464175|AAAAAAAAPKJJFCAA|2034-07-31|1614|7022|539|2034|1|7|31|3|2034|539|7022|Monday|2034Q3|N|N|N|2464145|2464325|2463810|2464084|N|N|N|N|N| +2464176|AAAAAAAAALJJFCAA|2034-08-01|1615|7023|539|2034|2|8|1|3|2034|539|7023|Tuesday|2034Q3|N|N|N|2464176|2464387|2463811|2464085|N|N|N|N|N| +2464177|AAAAAAAABLJJFCAA|2034-08-02|1615|7023|539|2034|3|8|2|3|2034|539|7023|Wednesday|2034Q3|N|N|N|2464176|2464387|2463812|2464086|N|N|N|N|N| +2464178|AAAAAAAACLJJFCAA|2034-08-03|1615|7023|539|2034|4|8|3|3|2034|539|7023|Thursday|2034Q3|N|N|N|2464176|2464387|2463813|2464087|N|N|N|N|N| +2464179|AAAAAAAADLJJFCAA|2034-08-04|1615|7023|539|2034|5|8|4|3|2034|539|7023|Friday|2034Q3|N|Y|N|2464176|2464387|2463814|2464088|N|N|N|N|N| +2464180|AAAAAAAAELJJFCAA|2034-08-05|1615|7023|539|2034|6|8|5|3|2034|539|7023|Saturday|2034Q3|N|Y|N|2464176|2464387|2463815|2464089|N|N|N|N|N| +2464181|AAAAAAAAFLJJFCAA|2034-08-06|1615|7023|539|2034|0|8|6|3|2034|539|7023|Sunday|2034Q3|N|N|N|2464176|2464387|2463816|2464090|N|N|N|N|N| +2464182|AAAAAAAAGLJJFCAA|2034-08-07|1615|7023|539|2034|1|8|7|3|2034|539|7023|Monday|2034Q3|N|N|N|2464176|2464387|2463817|2464091|N|N|N|N|N| +2464183|AAAAAAAAHLJJFCAA|2034-08-08|1615|7024|539|2034|2|8|8|3|2034|539|7024|Tuesday|2034Q3|N|N|N|2464176|2464387|2463818|2464092|N|N|N|N|N| +2464184|AAAAAAAAILJJFCAA|2034-08-09|1615|7024|539|2034|3|8|9|3|2034|539|7024|Wednesday|2034Q3|N|N|N|2464176|2464387|2463819|2464093|N|N|N|N|N| +2464185|AAAAAAAAJLJJFCAA|2034-08-10|1615|7024|539|2034|4|8|10|3|2034|539|7024|Thursday|2034Q3|N|N|N|2464176|2464387|2463820|2464094|N|N|N|N|N| +2464186|AAAAAAAAKLJJFCAA|2034-08-11|1615|7024|539|2034|5|8|11|3|2034|539|7024|Friday|2034Q3|N|Y|N|2464176|2464387|2463821|2464095|N|N|N|N|N| +2464187|AAAAAAAALLJJFCAA|2034-08-12|1615|7024|539|2034|6|8|12|3|2034|539|7024|Saturday|2034Q3|N|Y|N|2464176|2464387|2463822|2464096|N|N|N|N|N| +2464188|AAAAAAAAMLJJFCAA|2034-08-13|1615|7024|539|2034|0|8|13|3|2034|539|7024|Sunday|2034Q3|N|N|N|2464176|2464387|2463823|2464097|N|N|N|N|N| +2464189|AAAAAAAANLJJFCAA|2034-08-14|1615|7024|539|2034|1|8|14|3|2034|539|7024|Monday|2034Q3|N|N|N|2464176|2464387|2463824|2464098|N|N|N|N|N| +2464190|AAAAAAAAOLJJFCAA|2034-08-15|1615|7025|539|2034|2|8|15|3|2034|539|7025|Tuesday|2034Q3|N|N|N|2464176|2464387|2463825|2464099|N|N|N|N|N| +2464191|AAAAAAAAPLJJFCAA|2034-08-16|1615|7025|539|2034|3|8|16|3|2034|539|7025|Wednesday|2034Q3|N|N|N|2464176|2464387|2463826|2464100|N|N|N|N|N| +2464192|AAAAAAAAAMJJFCAA|2034-08-17|1615|7025|539|2034|4|8|17|3|2034|539|7025|Thursday|2034Q3|N|N|N|2464176|2464387|2463827|2464101|N|N|N|N|N| +2464193|AAAAAAAABMJJFCAA|2034-08-18|1615|7025|539|2034|5|8|18|3|2034|539|7025|Friday|2034Q3|N|Y|N|2464176|2464387|2463828|2464102|N|N|N|N|N| +2464194|AAAAAAAACMJJFCAA|2034-08-19|1615|7025|539|2034|6|8|19|3|2034|539|7025|Saturday|2034Q3|N|Y|N|2464176|2464387|2463829|2464103|N|N|N|N|N| +2464195|AAAAAAAADMJJFCAA|2034-08-20|1615|7025|539|2034|0|8|20|3|2034|539|7025|Sunday|2034Q3|N|N|N|2464176|2464387|2463830|2464104|N|N|N|N|N| +2464196|AAAAAAAAEMJJFCAA|2034-08-21|1615|7025|539|2034|1|8|21|3|2034|539|7025|Monday|2034Q3|N|N|N|2464176|2464387|2463831|2464105|N|N|N|N|N| +2464197|AAAAAAAAFMJJFCAA|2034-08-22|1615|7026|539|2034|2|8|22|3|2034|539|7026|Tuesday|2034Q3|N|N|N|2464176|2464387|2463832|2464106|N|N|N|N|N| +2464198|AAAAAAAAGMJJFCAA|2034-08-23|1615|7026|539|2034|3|8|23|3|2034|539|7026|Wednesday|2034Q3|N|N|N|2464176|2464387|2463833|2464107|N|N|N|N|N| +2464199|AAAAAAAAHMJJFCAA|2034-08-24|1615|7026|539|2034|4|8|24|3|2034|539|7026|Thursday|2034Q3|N|N|N|2464176|2464387|2463834|2464108|N|N|N|N|N| +2464200|AAAAAAAAIMJJFCAA|2034-08-25|1615|7026|539|2034|5|8|25|3|2034|539|7026|Friday|2034Q3|N|Y|N|2464176|2464387|2463835|2464109|N|N|N|N|N| +2464201|AAAAAAAAJMJJFCAA|2034-08-26|1615|7026|539|2034|6|8|26|3|2034|539|7026|Saturday|2034Q3|N|Y|N|2464176|2464387|2463836|2464110|N|N|N|N|N| +2464202|AAAAAAAAKMJJFCAA|2034-08-27|1615|7026|539|2034|0|8|27|3|2034|539|7026|Sunday|2034Q3|N|N|N|2464176|2464387|2463837|2464111|N|N|N|N|N| +2464203|AAAAAAAALMJJFCAA|2034-08-28|1615|7026|539|2034|1|8|28|3|2034|539|7026|Monday|2034Q3|N|N|N|2464176|2464387|2463838|2464112|N|N|N|N|N| +2464204|AAAAAAAAMMJJFCAA|2034-08-29|1615|7027|539|2034|2|8|29|3|2034|539|7027|Tuesday|2034Q3|N|N|N|2464176|2464387|2463839|2464113|N|N|N|N|N| +2464205|AAAAAAAANMJJFCAA|2034-08-30|1615|7027|539|2034|3|8|30|3|2034|539|7027|Wednesday|2034Q3|N|N|N|2464176|2464387|2463840|2464114|N|N|N|N|N| +2464206|AAAAAAAAOMJJFCAA|2034-08-31|1615|7027|539|2034|4|8|31|3|2034|539|7027|Thursday|2034Q3|N|N|N|2464176|2464387|2463841|2464115|N|N|N|N|N| +2464207|AAAAAAAAPMJJFCAA|2034-09-01|1616|7027|540|2034|5|9|1|3|2034|540|7027|Friday|2034Q3|N|Y|N|2464207|2464449|2463842|2464116|N|N|N|N|N| +2464208|AAAAAAAAANJJFCAA|2034-09-02|1616|7027|540|2034|6|9|2|3|2034|540|7027|Saturday|2034Q3|N|Y|N|2464207|2464449|2463843|2464117|N|N|N|N|N| +2464209|AAAAAAAABNJJFCAA|2034-09-03|1616|7027|540|2034|0|9|3|3|2034|540|7027|Sunday|2034Q3|N|N|N|2464207|2464449|2463844|2464118|N|N|N|N|N| +2464210|AAAAAAAACNJJFCAA|2034-09-04|1616|7027|540|2034|1|9|4|3|2034|540|7027|Monday|2034Q3|N|N|N|2464207|2464449|2463845|2464119|N|N|N|N|N| +2464211|AAAAAAAADNJJFCAA|2034-09-05|1616|7028|540|2034|2|9|5|3|2034|540|7028|Tuesday|2034Q3|N|N|N|2464207|2464449|2463846|2464120|N|N|N|N|N| +2464212|AAAAAAAAENJJFCAA|2034-09-06|1616|7028|540|2034|3|9|6|3|2034|540|7028|Wednesday|2034Q3|N|N|N|2464207|2464449|2463847|2464121|N|N|N|N|N| +2464213|AAAAAAAAFNJJFCAA|2034-09-07|1616|7028|540|2034|4|9|7|3|2034|540|7028|Thursday|2034Q3|N|N|N|2464207|2464449|2463848|2464122|N|N|N|N|N| +2464214|AAAAAAAAGNJJFCAA|2034-09-08|1616|7028|540|2034|5|9|8|3|2034|540|7028|Friday|2034Q3|N|Y|N|2464207|2464449|2463849|2464123|N|N|N|N|N| +2464215|AAAAAAAAHNJJFCAA|2034-09-09|1616|7028|540|2034|6|9|9|3|2034|540|7028|Saturday|2034Q3|N|Y|N|2464207|2464449|2463850|2464124|N|N|N|N|N| +2464216|AAAAAAAAINJJFCAA|2034-09-10|1616|7028|540|2034|0|9|10|3|2034|540|7028|Sunday|2034Q3|N|N|N|2464207|2464449|2463851|2464125|N|N|N|N|N| +2464217|AAAAAAAAJNJJFCAA|2034-09-11|1616|7028|540|2034|1|9|11|3|2034|540|7028|Monday|2034Q3|N|N|N|2464207|2464449|2463852|2464126|N|N|N|N|N| +2464218|AAAAAAAAKNJJFCAA|2034-09-12|1616|7029|540|2034|2|9|12|3|2034|540|7029|Tuesday|2034Q3|N|N|N|2464207|2464449|2463853|2464127|N|N|N|N|N| +2464219|AAAAAAAALNJJFCAA|2034-09-13|1616|7029|540|2034|3|9|13|3|2034|540|7029|Wednesday|2034Q3|N|N|N|2464207|2464449|2463854|2464128|N|N|N|N|N| +2464220|AAAAAAAAMNJJFCAA|2034-09-14|1616|7029|540|2034|4|9|14|3|2034|540|7029|Thursday|2034Q3|N|N|N|2464207|2464449|2463855|2464129|N|N|N|N|N| +2464221|AAAAAAAANNJJFCAA|2034-09-15|1616|7029|540|2034|5|9|15|3|2034|540|7029|Friday|2034Q3|N|Y|N|2464207|2464449|2463856|2464130|N|N|N|N|N| +2464222|AAAAAAAAONJJFCAA|2034-09-16|1616|7029|540|2034|6|9|16|3|2034|540|7029|Saturday|2034Q3|N|Y|N|2464207|2464449|2463857|2464131|N|N|N|N|N| +2464223|AAAAAAAAPNJJFCAA|2034-09-17|1616|7029|540|2034|0|9|17|3|2034|540|7029|Sunday|2034Q3|N|N|N|2464207|2464449|2463858|2464132|N|N|N|N|N| +2464224|AAAAAAAAAOJJFCAA|2034-09-18|1616|7029|540|2034|1|9|18|3|2034|540|7029|Monday|2034Q3|N|N|N|2464207|2464449|2463859|2464133|N|N|N|N|N| +2464225|AAAAAAAABOJJFCAA|2034-09-19|1616|7030|540|2034|2|9|19|3|2034|540|7030|Tuesday|2034Q3|N|N|N|2464207|2464449|2463860|2464134|N|N|N|N|N| +2464226|AAAAAAAACOJJFCAA|2034-09-20|1616|7030|540|2034|3|9|20|3|2034|540|7030|Wednesday|2034Q3|N|N|N|2464207|2464449|2463861|2464135|N|N|N|N|N| +2464227|AAAAAAAADOJJFCAA|2034-09-21|1616|7030|540|2034|4|9|21|3|2034|540|7030|Thursday|2034Q3|N|N|N|2464207|2464449|2463862|2464136|N|N|N|N|N| +2464228|AAAAAAAAEOJJFCAA|2034-09-22|1616|7030|540|2034|5|9|22|3|2034|540|7030|Friday|2034Q3|N|Y|N|2464207|2464449|2463863|2464137|N|N|N|N|N| +2464229|AAAAAAAAFOJJFCAA|2034-09-23|1616|7030|540|2034|6|9|23|3|2034|540|7030|Saturday|2034Q3|N|Y|N|2464207|2464449|2463864|2464138|N|N|N|N|N| +2464230|AAAAAAAAGOJJFCAA|2034-09-24|1616|7030|540|2034|0|9|24|3|2034|540|7030|Sunday|2034Q3|N|N|N|2464207|2464449|2463865|2464139|N|N|N|N|N| +2464231|AAAAAAAAHOJJFCAA|2034-09-25|1616|7030|540|2034|1|9|25|3|2034|540|7030|Monday|2034Q3|N|N|N|2464207|2464449|2463866|2464140|N|N|N|N|N| +2464232|AAAAAAAAIOJJFCAA|2034-09-26|1616|7031|540|2034|2|9|26|3|2034|540|7031|Tuesday|2034Q3|N|N|N|2464207|2464449|2463867|2464141|N|N|N|N|N| +2464233|AAAAAAAAJOJJFCAA|2034-09-27|1616|7031|540|2034|3|9|27|3|2034|540|7031|Wednesday|2034Q3|N|N|N|2464207|2464449|2463868|2464142|N|N|N|N|N| +2464234|AAAAAAAAKOJJFCAA|2034-09-28|1616|7031|540|2034|4|9|28|3|2034|540|7031|Thursday|2034Q3|N|N|N|2464207|2464449|2463869|2464143|N|N|N|N|N| +2464235|AAAAAAAALOJJFCAA|2034-09-29|1616|7031|540|2034|5|9|29|3|2034|540|7031|Friday|2034Q3|N|Y|N|2464207|2464449|2463870|2464144|N|N|N|N|N| +2464236|AAAAAAAAMOJJFCAA|2034-09-30|1616|7031|540|2034|6|9|30|3|2034|540|7031|Saturday|2034Q3|N|Y|N|2464207|2464449|2463871|2464145|N|N|N|N|N| +2464237|AAAAAAAANOJJFCAA|2034-10-01|1617|7031|540|2034|0|10|1|3|2034|540|7031|Sunday|2034Q3|N|N|N|2464237|2464509|2463872|2464145|N|N|N|N|N| +2464238|AAAAAAAAOOJJFCAA|2034-10-02|1617|7031|540|2034|1|10|2|4|2034|540|7031|Monday|2034Q4|N|N|N|2464237|2464509|2463873|2464146|N|N|N|N|N| +2464239|AAAAAAAAPOJJFCAA|2034-10-03|1617|7032|540|2034|2|10|3|4|2034|540|7032|Tuesday|2034Q4|N|N|N|2464237|2464509|2463874|2464147|N|N|N|N|N| +2464240|AAAAAAAAAPJJFCAA|2034-10-04|1617|7032|540|2034|3|10|4|4|2034|540|7032|Wednesday|2034Q4|N|N|N|2464237|2464509|2463875|2464148|N|N|N|N|N| +2464241|AAAAAAAABPJJFCAA|2034-10-05|1617|7032|540|2034|4|10|5|4|2034|540|7032|Thursday|2034Q4|N|N|N|2464237|2464509|2463876|2464149|N|N|N|N|N| +2464242|AAAAAAAACPJJFCAA|2034-10-06|1617|7032|540|2034|5|10|6|4|2034|540|7032|Friday|2034Q4|N|Y|N|2464237|2464509|2463877|2464150|N|N|N|N|N| +2464243|AAAAAAAADPJJFCAA|2034-10-07|1617|7032|540|2034|6|10|7|4|2034|540|7032|Saturday|2034Q4|N|Y|N|2464237|2464509|2463878|2464151|N|N|N|N|N| +2464244|AAAAAAAAEPJJFCAA|2034-10-08|1617|7032|540|2034|0|10|8|4|2034|540|7032|Sunday|2034Q4|N|N|N|2464237|2464509|2463879|2464152|N|N|N|N|N| +2464245|AAAAAAAAFPJJFCAA|2034-10-09|1617|7032|540|2034|1|10|9|4|2034|540|7032|Monday|2034Q4|N|N|N|2464237|2464509|2463880|2464153|N|N|N|N|N| +2464246|AAAAAAAAGPJJFCAA|2034-10-10|1617|7033|540|2034|2|10|10|4|2034|540|7033|Tuesday|2034Q4|N|N|N|2464237|2464509|2463881|2464154|N|N|N|N|N| +2464247|AAAAAAAAHPJJFCAA|2034-10-11|1617|7033|540|2034|3|10|11|4|2034|540|7033|Wednesday|2034Q4|N|N|N|2464237|2464509|2463882|2464155|N|N|N|N|N| +2464248|AAAAAAAAIPJJFCAA|2034-10-12|1617|7033|540|2034|4|10|12|4|2034|540|7033|Thursday|2034Q4|N|N|N|2464237|2464509|2463883|2464156|N|N|N|N|N| +2464249|AAAAAAAAJPJJFCAA|2034-10-13|1617|7033|540|2034|5|10|13|4|2034|540|7033|Friday|2034Q4|N|Y|N|2464237|2464509|2463884|2464157|N|N|N|N|N| +2464250|AAAAAAAAKPJJFCAA|2034-10-14|1617|7033|540|2034|6|10|14|4|2034|540|7033|Saturday|2034Q4|N|Y|N|2464237|2464509|2463885|2464158|N|N|N|N|N| +2464251|AAAAAAAALPJJFCAA|2034-10-15|1617|7033|540|2034|0|10|15|4|2034|540|7033|Sunday|2034Q4|N|N|N|2464237|2464509|2463886|2464159|N|N|N|N|N| +2464252|AAAAAAAAMPJJFCAA|2034-10-16|1617|7033|540|2034|1|10|16|4|2034|540|7033|Monday|2034Q4|N|N|N|2464237|2464509|2463887|2464160|N|N|N|N|N| +2464253|AAAAAAAANPJJFCAA|2034-10-17|1617|7034|540|2034|2|10|17|4|2034|540|7034|Tuesday|2034Q4|N|N|N|2464237|2464509|2463888|2464161|N|N|N|N|N| +2464254|AAAAAAAAOPJJFCAA|2034-10-18|1617|7034|540|2034|3|10|18|4|2034|540|7034|Wednesday|2034Q4|N|N|N|2464237|2464509|2463889|2464162|N|N|N|N|N| +2464255|AAAAAAAAPPJJFCAA|2034-10-19|1617|7034|540|2034|4|10|19|4|2034|540|7034|Thursday|2034Q4|N|N|N|2464237|2464509|2463890|2464163|N|N|N|N|N| +2464256|AAAAAAAAAAKJFCAA|2034-10-20|1617|7034|540|2034|5|10|20|4|2034|540|7034|Friday|2034Q4|N|Y|N|2464237|2464509|2463891|2464164|N|N|N|N|N| +2464257|AAAAAAAABAKJFCAA|2034-10-21|1617|7034|540|2034|6|10|21|4|2034|540|7034|Saturday|2034Q4|N|Y|N|2464237|2464509|2463892|2464165|N|N|N|N|N| +2464258|AAAAAAAACAKJFCAA|2034-10-22|1617|7034|540|2034|0|10|22|4|2034|540|7034|Sunday|2034Q4|N|N|N|2464237|2464509|2463893|2464166|N|N|N|N|N| +2464259|AAAAAAAADAKJFCAA|2034-10-23|1617|7034|540|2034|1|10|23|4|2034|540|7034|Monday|2034Q4|N|N|N|2464237|2464509|2463894|2464167|N|N|N|N|N| +2464260|AAAAAAAAEAKJFCAA|2034-10-24|1617|7035|540|2034|2|10|24|4|2034|540|7035|Tuesday|2034Q4|N|N|N|2464237|2464509|2463895|2464168|N|N|N|N|N| +2464261|AAAAAAAAFAKJFCAA|2034-10-25|1617|7035|540|2034|3|10|25|4|2034|540|7035|Wednesday|2034Q4|N|N|N|2464237|2464509|2463896|2464169|N|N|N|N|N| +2464262|AAAAAAAAGAKJFCAA|2034-10-26|1617|7035|540|2034|4|10|26|4|2034|540|7035|Thursday|2034Q4|N|N|N|2464237|2464509|2463897|2464170|N|N|N|N|N| +2464263|AAAAAAAAHAKJFCAA|2034-10-27|1617|7035|540|2034|5|10|27|4|2034|540|7035|Friday|2034Q4|N|Y|N|2464237|2464509|2463898|2464171|N|N|N|N|N| +2464264|AAAAAAAAIAKJFCAA|2034-10-28|1617|7035|540|2034|6|10|28|4|2034|540|7035|Saturday|2034Q4|N|Y|N|2464237|2464509|2463899|2464172|N|N|N|N|N| +2464265|AAAAAAAAJAKJFCAA|2034-10-29|1617|7035|540|2034|0|10|29|4|2034|540|7035|Sunday|2034Q4|N|N|N|2464237|2464509|2463900|2464173|N|N|N|N|N| +2464266|AAAAAAAAKAKJFCAA|2034-10-30|1617|7035|540|2034|1|10|30|4|2034|540|7035|Monday|2034Q4|N|N|N|2464237|2464509|2463901|2464174|N|N|N|N|N| +2464267|AAAAAAAALAKJFCAA|2034-10-31|1617|7036|540|2034|2|10|31|4|2034|540|7036|Tuesday|2034Q4|N|N|N|2464237|2464509|2463902|2464175|N|N|N|N|N| +2464268|AAAAAAAAMAKJFCAA|2034-11-01|1618|7036|540|2034|3|11|1|4|2034|540|7036|Wednesday|2034Q4|N|N|N|2464268|2464571|2463903|2464176|N|N|N|N|N| +2464269|AAAAAAAANAKJFCAA|2034-11-02|1618|7036|540|2034|4|11|2|4|2034|540|7036|Thursday|2034Q4|N|N|N|2464268|2464571|2463904|2464177|N|N|N|N|N| +2464270|AAAAAAAAOAKJFCAA|2034-11-03|1618|7036|540|2034|5|11|3|4|2034|540|7036|Friday|2034Q4|N|Y|N|2464268|2464571|2463905|2464178|N|N|N|N|N| +2464271|AAAAAAAAPAKJFCAA|2034-11-04|1618|7036|540|2034|6|11|4|4|2034|540|7036|Saturday|2034Q4|N|Y|N|2464268|2464571|2463906|2464179|N|N|N|N|N| +2464272|AAAAAAAAABKJFCAA|2034-11-05|1618|7036|540|2034|0|11|5|4|2034|540|7036|Sunday|2034Q4|N|N|N|2464268|2464571|2463907|2464180|N|N|N|N|N| +2464273|AAAAAAAABBKJFCAA|2034-11-06|1618|7036|540|2034|1|11|6|4|2034|540|7036|Monday|2034Q4|N|N|N|2464268|2464571|2463908|2464181|N|N|N|N|N| +2464274|AAAAAAAACBKJFCAA|2034-11-07|1618|7037|540|2034|2|11|7|4|2034|540|7037|Tuesday|2034Q4|N|N|N|2464268|2464571|2463909|2464182|N|N|N|N|N| +2464275|AAAAAAAADBKJFCAA|2034-11-08|1618|7037|540|2034|3|11|8|4|2034|540|7037|Wednesday|2034Q4|N|N|N|2464268|2464571|2463910|2464183|N|N|N|N|N| +2464276|AAAAAAAAEBKJFCAA|2034-11-09|1618|7037|540|2034|4|11|9|4|2034|540|7037|Thursday|2034Q4|N|N|N|2464268|2464571|2463911|2464184|N|N|N|N|N| +2464277|AAAAAAAAFBKJFCAA|2034-11-10|1618|7037|540|2034|5|11|10|4|2034|540|7037|Friday|2034Q4|N|Y|N|2464268|2464571|2463912|2464185|N|N|N|N|N| +2464278|AAAAAAAAGBKJFCAA|2034-11-11|1618|7037|540|2034|6|11|11|4|2034|540|7037|Saturday|2034Q4|N|Y|N|2464268|2464571|2463913|2464186|N|N|N|N|N| +2464279|AAAAAAAAHBKJFCAA|2034-11-12|1618|7037|540|2034|0|11|12|4|2034|540|7037|Sunday|2034Q4|N|N|N|2464268|2464571|2463914|2464187|N|N|N|N|N| +2464280|AAAAAAAAIBKJFCAA|2034-11-13|1618|7037|540|2034|1|11|13|4|2034|540|7037|Monday|2034Q4|N|N|N|2464268|2464571|2463915|2464188|N|N|N|N|N| +2464281|AAAAAAAAJBKJFCAA|2034-11-14|1618|7038|540|2034|2|11|14|4|2034|540|7038|Tuesday|2034Q4|N|N|N|2464268|2464571|2463916|2464189|N|N|N|N|N| +2464282|AAAAAAAAKBKJFCAA|2034-11-15|1618|7038|540|2034|3|11|15|4|2034|540|7038|Wednesday|2034Q4|N|N|N|2464268|2464571|2463917|2464190|N|N|N|N|N| +2464283|AAAAAAAALBKJFCAA|2034-11-16|1618|7038|540|2034|4|11|16|4|2034|540|7038|Thursday|2034Q4|N|N|N|2464268|2464571|2463918|2464191|N|N|N|N|N| +2464284|AAAAAAAAMBKJFCAA|2034-11-17|1618|7038|540|2034|5|11|17|4|2034|540|7038|Friday|2034Q4|N|Y|N|2464268|2464571|2463919|2464192|N|N|N|N|N| +2464285|AAAAAAAANBKJFCAA|2034-11-18|1618|7038|540|2034|6|11|18|4|2034|540|7038|Saturday|2034Q4|N|Y|N|2464268|2464571|2463920|2464193|N|N|N|N|N| +2464286|AAAAAAAAOBKJFCAA|2034-11-19|1618|7038|540|2034|0|11|19|4|2034|540|7038|Sunday|2034Q4|N|N|N|2464268|2464571|2463921|2464194|N|N|N|N|N| +2464287|AAAAAAAAPBKJFCAA|2034-11-20|1618|7038|540|2034|1|11|20|4|2034|540|7038|Monday|2034Q4|N|N|N|2464268|2464571|2463922|2464195|N|N|N|N|N| +2464288|AAAAAAAAACKJFCAA|2034-11-21|1618|7039|540|2034|2|11|21|4|2034|540|7039|Tuesday|2034Q4|N|N|N|2464268|2464571|2463923|2464196|N|N|N|N|N| +2464289|AAAAAAAABCKJFCAA|2034-11-22|1618|7039|540|2034|3|11|22|4|2034|540|7039|Wednesday|2034Q4|N|N|N|2464268|2464571|2463924|2464197|N|N|N|N|N| +2464290|AAAAAAAACCKJFCAA|2034-11-23|1618|7039|540|2034|4|11|23|4|2034|540|7039|Thursday|2034Q4|N|N|N|2464268|2464571|2463925|2464198|N|N|N|N|N| +2464291|AAAAAAAADCKJFCAA|2034-11-24|1618|7039|540|2034|5|11|24|4|2034|540|7039|Friday|2034Q4|N|Y|N|2464268|2464571|2463926|2464199|N|N|N|N|N| +2464292|AAAAAAAAECKJFCAA|2034-11-25|1618|7039|540|2034|6|11|25|4|2034|540|7039|Saturday|2034Q4|N|Y|N|2464268|2464571|2463927|2464200|N|N|N|N|N| +2464293|AAAAAAAAFCKJFCAA|2034-11-26|1618|7039|540|2034|0|11|26|4|2034|540|7039|Sunday|2034Q4|N|N|N|2464268|2464571|2463928|2464201|N|N|N|N|N| +2464294|AAAAAAAAGCKJFCAA|2034-11-27|1618|7039|540|2034|1|11|27|4|2034|540|7039|Monday|2034Q4|N|N|N|2464268|2464571|2463929|2464202|N|N|N|N|N| +2464295|AAAAAAAAHCKJFCAA|2034-11-28|1618|7040|540|2034|2|11|28|4|2034|540|7040|Tuesday|2034Q4|N|N|N|2464268|2464571|2463930|2464203|N|N|N|N|N| +2464296|AAAAAAAAICKJFCAA|2034-11-29|1618|7040|540|2034|3|11|29|4|2034|540|7040|Wednesday|2034Q4|N|N|N|2464268|2464571|2463931|2464204|N|N|N|N|N| +2464297|AAAAAAAAJCKJFCAA|2034-11-30|1618|7040|540|2034|4|11|30|4|2034|540|7040|Thursday|2034Q4|N|N|N|2464268|2464571|2463932|2464205|N|N|N|N|N| +2464298|AAAAAAAAKCKJFCAA|2034-12-01|1619|7040|541|2034|5|12|1|4|2034|541|7040|Friday|2034Q4|N|Y|N|2464298|2464631|2463933|2464206|N|N|N|N|N| +2464299|AAAAAAAALCKJFCAA|2034-12-02|1619|7040|541|2034|6|12|2|4|2034|541|7040|Saturday|2034Q4|N|Y|N|2464298|2464631|2463934|2464207|N|N|N|N|N| +2464300|AAAAAAAAMCKJFCAA|2034-12-03|1619|7040|541|2034|0|12|3|4|2034|541|7040|Sunday|2034Q4|N|N|N|2464298|2464631|2463935|2464208|N|N|N|N|N| +2464301|AAAAAAAANCKJFCAA|2034-12-04|1619|7040|541|2034|1|12|4|4|2034|541|7040|Monday|2034Q4|N|N|N|2464298|2464631|2463936|2464209|N|N|N|N|N| +2464302|AAAAAAAAOCKJFCAA|2034-12-05|1619|7041|541|2034|2|12|5|4|2034|541|7041|Tuesday|2034Q4|N|N|N|2464298|2464631|2463937|2464210|N|N|N|N|N| +2464303|AAAAAAAAPCKJFCAA|2034-12-06|1619|7041|541|2034|3|12|6|4|2034|541|7041|Wednesday|2034Q4|N|N|N|2464298|2464631|2463938|2464211|N|N|N|N|N| +2464304|AAAAAAAAADKJFCAA|2034-12-07|1619|7041|541|2034|4|12|7|4|2034|541|7041|Thursday|2034Q4|N|N|N|2464298|2464631|2463939|2464212|N|N|N|N|N| +2464305|AAAAAAAABDKJFCAA|2034-12-08|1619|7041|541|2034|5|12|8|4|2034|541|7041|Friday|2034Q4|N|Y|N|2464298|2464631|2463940|2464213|N|N|N|N|N| +2464306|AAAAAAAACDKJFCAA|2034-12-09|1619|7041|541|2034|6|12|9|4|2034|541|7041|Saturday|2034Q4|N|Y|N|2464298|2464631|2463941|2464214|N|N|N|N|N| +2464307|AAAAAAAADDKJFCAA|2034-12-10|1619|7041|541|2034|0|12|10|4|2034|541|7041|Sunday|2034Q4|N|N|N|2464298|2464631|2463942|2464215|N|N|N|N|N| +2464308|AAAAAAAAEDKJFCAA|2034-12-11|1619|7041|541|2034|1|12|11|4|2034|541|7041|Monday|2034Q4|N|N|N|2464298|2464631|2463943|2464216|N|N|N|N|N| +2464309|AAAAAAAAFDKJFCAA|2034-12-12|1619|7042|541|2034|2|12|12|4|2034|541|7042|Tuesday|2034Q4|N|N|N|2464298|2464631|2463944|2464217|N|N|N|N|N| +2464310|AAAAAAAAGDKJFCAA|2034-12-13|1619|7042|541|2034|3|12|13|4|2034|541|7042|Wednesday|2034Q4|N|N|N|2464298|2464631|2463945|2464218|N|N|N|N|N| +2464311|AAAAAAAAHDKJFCAA|2034-12-14|1619|7042|541|2034|4|12|14|4|2034|541|7042|Thursday|2034Q4|N|N|N|2464298|2464631|2463946|2464219|N|N|N|N|N| +2464312|AAAAAAAAIDKJFCAA|2034-12-15|1619|7042|541|2034|5|12|15|4|2034|541|7042|Friday|2034Q4|N|Y|N|2464298|2464631|2463947|2464220|N|N|N|N|N| +2464313|AAAAAAAAJDKJFCAA|2034-12-16|1619|7042|541|2034|6|12|16|4|2034|541|7042|Saturday|2034Q4|N|Y|N|2464298|2464631|2463948|2464221|N|N|N|N|N| +2464314|AAAAAAAAKDKJFCAA|2034-12-17|1619|7042|541|2034|0|12|17|4|2034|541|7042|Sunday|2034Q4|N|N|N|2464298|2464631|2463949|2464222|N|N|N|N|N| +2464315|AAAAAAAALDKJFCAA|2034-12-18|1619|7042|541|2034|1|12|18|4|2034|541|7042|Monday|2034Q4|N|N|N|2464298|2464631|2463950|2464223|N|N|N|N|N| +2464316|AAAAAAAAMDKJFCAA|2034-12-19|1619|7043|541|2034|2|12|19|4|2034|541|7043|Tuesday|2034Q4|N|N|N|2464298|2464631|2463951|2464224|N|N|N|N|N| +2464317|AAAAAAAANDKJFCAA|2034-12-20|1619|7043|541|2034|3|12|20|4|2034|541|7043|Wednesday|2034Q4|N|N|N|2464298|2464631|2463952|2464225|N|N|N|N|N| +2464318|AAAAAAAAODKJFCAA|2034-12-21|1619|7043|541|2034|4|12|21|4|2034|541|7043|Thursday|2034Q4|N|N|N|2464298|2464631|2463953|2464226|N|N|N|N|N| +2464319|AAAAAAAAPDKJFCAA|2034-12-22|1619|7043|541|2034|5|12|22|4|2034|541|7043|Friday|2034Q4|N|Y|N|2464298|2464631|2463954|2464227|N|N|N|N|N| +2464320|AAAAAAAAAEKJFCAA|2034-12-23|1619|7043|541|2034|6|12|23|4|2034|541|7043|Saturday|2034Q4|N|Y|N|2464298|2464631|2463955|2464228|N|N|N|N|N| +2464321|AAAAAAAABEKJFCAA|2034-12-24|1619|7043|541|2034|0|12|24|4|2034|541|7043|Sunday|2034Q4|N|N|N|2464298|2464631|2463956|2464229|N|N|N|N|N| +2464322|AAAAAAAACEKJFCAA|2034-12-25|1619|7043|541|2034|1|12|25|4|2034|541|7043|Monday|2034Q4|N|N|N|2464298|2464631|2463957|2464230|N|N|N|N|N| +2464323|AAAAAAAADEKJFCAA|2034-12-26|1619|7044|541|2034|2|12|26|4|2034|541|7044|Tuesday|2034Q4|Y|N|N|2464298|2464631|2463958|2464231|N|N|N|N|N| +2464324|AAAAAAAAEEKJFCAA|2034-12-27|1619|7044|541|2034|3|12|27|4|2034|541|7044|Wednesday|2034Q4|N|N|Y|2464298|2464631|2463959|2464232|N|N|N|N|N| +2464325|AAAAAAAAFEKJFCAA|2034-12-28|1619|7044|541|2034|4|12|28|4|2034|541|7044|Thursday|2034Q4|N|N|N|2464298|2464631|2463960|2464233|N|N|N|N|N| +2464326|AAAAAAAAGEKJFCAA|2034-12-29|1619|7044|541|2034|5|12|29|4|2034|541|7044|Friday|2034Q4|N|Y|N|2464298|2464631|2463961|2464234|N|N|N|N|N| +2464327|AAAAAAAAHEKJFCAA|2034-12-30|1619|7044|541|2034|6|12|30|4|2034|541|7044|Saturday|2034Q4|N|Y|N|2464298|2464631|2463962|2464235|N|N|N|N|N| +2464328|AAAAAAAAIEKJFCAA|2034-12-31|1619|7044|541|2034|0|12|31|4|2034|541|7044|Sunday|2034Q4|N|N|N|2464298|2464631|2463963|2464236|N|N|N|N|N| +2464329|AAAAAAAAJEKJFCAA|2035-01-01|1620|7044|541|2035|1|1|1|1|2035|541|7044|Monday|2035Q1|Y|N|N|2464329|2464328|2463964|2464237|N|N|N|N|N| +2464330|AAAAAAAAKEKJFCAA|2035-01-02|1620|7045|541|2035|2|1|2|1|2035|541|7045|Tuesday|2035Q1|N|N|Y|2464329|2464328|2463965|2464238|N|N|N|N|N| +2464331|AAAAAAAALEKJFCAA|2035-01-03|1620|7045|541|2035|3|1|3|1|2035|541|7045|Wednesday|2035Q1|N|N|N|2464329|2464328|2463966|2464239|N|N|N|N|N| +2464332|AAAAAAAAMEKJFCAA|2035-01-04|1620|7045|541|2035|4|1|4|1|2035|541|7045|Thursday|2035Q1|N|N|N|2464329|2464328|2463967|2464240|N|N|N|N|N| +2464333|AAAAAAAANEKJFCAA|2035-01-05|1620|7045|541|2035|5|1|5|1|2035|541|7045|Friday|2035Q1|N|Y|N|2464329|2464328|2463968|2464241|N|N|N|N|N| +2464334|AAAAAAAAOEKJFCAA|2035-01-06|1620|7045|541|2035|6|1|6|1|2035|541|7045|Saturday|2035Q1|N|Y|N|2464329|2464328|2463969|2464242|N|N|N|N|N| +2464335|AAAAAAAAPEKJFCAA|2035-01-07|1620|7045|541|2035|0|1|7|1|2035|541|7045|Sunday|2035Q1|N|N|N|2464329|2464328|2463970|2464243|N|N|N|N|N| +2464336|AAAAAAAAAFKJFCAA|2035-01-08|1620|7045|541|2035|1|1|8|1|2035|541|7045|Monday|2035Q1|N|N|N|2464329|2464328|2463971|2464244|N|N|N|N|N| +2464337|AAAAAAAABFKJFCAA|2035-01-09|1620|7046|541|2035|2|1|9|1|2035|541|7046|Tuesday|2035Q1|N|N|N|2464329|2464328|2463972|2464245|N|N|N|N|N| +2464338|AAAAAAAACFKJFCAA|2035-01-10|1620|7046|541|2035|3|1|10|1|2035|541|7046|Wednesday|2035Q1|N|N|N|2464329|2464328|2463973|2464246|N|N|N|N|N| +2464339|AAAAAAAADFKJFCAA|2035-01-11|1620|7046|541|2035|4|1|11|1|2035|541|7046|Thursday|2035Q1|N|N|N|2464329|2464328|2463974|2464247|N|N|N|N|N| +2464340|AAAAAAAAEFKJFCAA|2035-01-12|1620|7046|541|2035|5|1|12|1|2035|541|7046|Friday|2035Q1|N|Y|N|2464329|2464328|2463975|2464248|N|N|N|N|N| +2464341|AAAAAAAAFFKJFCAA|2035-01-13|1620|7046|541|2035|6|1|13|1|2035|541|7046|Saturday|2035Q1|N|Y|N|2464329|2464328|2463976|2464249|N|N|N|N|N| +2464342|AAAAAAAAGFKJFCAA|2035-01-14|1620|7046|541|2035|0|1|14|1|2035|541|7046|Sunday|2035Q1|N|N|N|2464329|2464328|2463977|2464250|N|N|N|N|N| +2464343|AAAAAAAAHFKJFCAA|2035-01-15|1620|7046|541|2035|1|1|15|1|2035|541|7046|Monday|2035Q1|N|N|N|2464329|2464328|2463978|2464251|N|N|N|N|N| +2464344|AAAAAAAAIFKJFCAA|2035-01-16|1620|7047|541|2035|2|1|16|1|2035|541|7047|Tuesday|2035Q1|N|N|N|2464329|2464328|2463979|2464252|N|N|N|N|N| +2464345|AAAAAAAAJFKJFCAA|2035-01-17|1620|7047|541|2035|3|1|17|1|2035|541|7047|Wednesday|2035Q1|N|N|N|2464329|2464328|2463980|2464253|N|N|N|N|N| +2464346|AAAAAAAAKFKJFCAA|2035-01-18|1620|7047|541|2035|4|1|18|1|2035|541|7047|Thursday|2035Q1|N|N|N|2464329|2464328|2463981|2464254|N|N|N|N|N| +2464347|AAAAAAAALFKJFCAA|2035-01-19|1620|7047|541|2035|5|1|19|1|2035|541|7047|Friday|2035Q1|N|Y|N|2464329|2464328|2463982|2464255|N|N|N|N|N| +2464348|AAAAAAAAMFKJFCAA|2035-01-20|1620|7047|541|2035|6|1|20|1|2035|541|7047|Saturday|2035Q1|N|Y|N|2464329|2464328|2463983|2464256|N|N|N|N|N| +2464349|AAAAAAAANFKJFCAA|2035-01-21|1620|7047|541|2035|0|1|21|1|2035|541|7047|Sunday|2035Q1|N|N|N|2464329|2464328|2463984|2464257|N|N|N|N|N| +2464350|AAAAAAAAOFKJFCAA|2035-01-22|1620|7047|541|2035|1|1|22|1|2035|541|7047|Monday|2035Q1|N|N|N|2464329|2464328|2463985|2464258|N|N|N|N|N| +2464351|AAAAAAAAPFKJFCAA|2035-01-23|1620|7048|541|2035|2|1|23|1|2035|541|7048|Tuesday|2035Q1|N|N|N|2464329|2464328|2463986|2464259|N|N|N|N|N| +2464352|AAAAAAAAAGKJFCAA|2035-01-24|1620|7048|541|2035|3|1|24|1|2035|541|7048|Wednesday|2035Q1|N|N|N|2464329|2464328|2463987|2464260|N|N|N|N|N| +2464353|AAAAAAAABGKJFCAA|2035-01-25|1620|7048|541|2035|4|1|25|1|2035|541|7048|Thursday|2035Q1|N|N|N|2464329|2464328|2463988|2464261|N|N|N|N|N| +2464354|AAAAAAAACGKJFCAA|2035-01-26|1620|7048|541|2035|5|1|26|1|2035|541|7048|Friday|2035Q1|N|Y|N|2464329|2464328|2463989|2464262|N|N|N|N|N| +2464355|AAAAAAAADGKJFCAA|2035-01-27|1620|7048|541|2035|6|1|27|1|2035|541|7048|Saturday|2035Q1|N|Y|N|2464329|2464328|2463990|2464263|N|N|N|N|N| +2464356|AAAAAAAAEGKJFCAA|2035-01-28|1620|7048|541|2035|0|1|28|1|2035|541|7048|Sunday|2035Q1|N|N|N|2464329|2464328|2463991|2464264|N|N|N|N|N| +2464357|AAAAAAAAFGKJFCAA|2035-01-29|1620|7048|541|2035|1|1|29|1|2035|541|7048|Monday|2035Q1|N|N|N|2464329|2464328|2463992|2464265|N|N|N|N|N| +2464358|AAAAAAAAGGKJFCAA|2035-01-30|1620|7049|541|2035|2|1|30|1|2035|541|7049|Tuesday|2035Q1|N|N|N|2464329|2464328|2463993|2464266|N|N|N|N|N| +2464359|AAAAAAAAHGKJFCAA|2035-01-31|1620|7049|541|2035|3|1|31|1|2035|541|7049|Wednesday|2035Q1|N|N|N|2464329|2464328|2463994|2464267|N|N|N|N|N| +2464360|AAAAAAAAIGKJFCAA|2035-02-01|1621|7049|541|2035|4|2|1|1|2035|541|7049|Thursday|2035Q1|N|N|N|2464360|2464390|2463995|2464268|N|N|N|N|N| +2464361|AAAAAAAAJGKJFCAA|2035-02-02|1621|7049|541|2035|5|2|2|1|2035|541|7049|Friday|2035Q1|N|Y|N|2464360|2464390|2463996|2464269|N|N|N|N|N| +2464362|AAAAAAAAKGKJFCAA|2035-02-03|1621|7049|541|2035|6|2|3|1|2035|541|7049|Saturday|2035Q1|N|Y|N|2464360|2464390|2463997|2464270|N|N|N|N|N| +2464363|AAAAAAAALGKJFCAA|2035-02-04|1621|7049|541|2035|0|2|4|1|2035|541|7049|Sunday|2035Q1|N|N|N|2464360|2464390|2463998|2464271|N|N|N|N|N| +2464364|AAAAAAAAMGKJFCAA|2035-02-05|1621|7049|541|2035|1|2|5|1|2035|541|7049|Monday|2035Q1|N|N|N|2464360|2464390|2463999|2464272|N|N|N|N|N| +2464365|AAAAAAAANGKJFCAA|2035-02-06|1621|7050|541|2035|2|2|6|1|2035|541|7050|Tuesday|2035Q1|N|N|N|2464360|2464390|2464000|2464273|N|N|N|N|N| +2464366|AAAAAAAAOGKJFCAA|2035-02-07|1621|7050|541|2035|3|2|7|1|2035|541|7050|Wednesday|2035Q1|N|N|N|2464360|2464390|2464001|2464274|N|N|N|N|N| +2464367|AAAAAAAAPGKJFCAA|2035-02-08|1621|7050|541|2035|4|2|8|1|2035|541|7050|Thursday|2035Q1|N|N|N|2464360|2464390|2464002|2464275|N|N|N|N|N| +2464368|AAAAAAAAAHKJFCAA|2035-02-09|1621|7050|541|2035|5|2|9|1|2035|541|7050|Friday|2035Q1|N|Y|N|2464360|2464390|2464003|2464276|N|N|N|N|N| +2464369|AAAAAAAABHKJFCAA|2035-02-10|1621|7050|541|2035|6|2|10|1|2035|541|7050|Saturday|2035Q1|N|Y|N|2464360|2464390|2464004|2464277|N|N|N|N|N| +2464370|AAAAAAAACHKJFCAA|2035-02-11|1621|7050|541|2035|0|2|11|1|2035|541|7050|Sunday|2035Q1|N|N|N|2464360|2464390|2464005|2464278|N|N|N|N|N| +2464371|AAAAAAAADHKJFCAA|2035-02-12|1621|7050|541|2035|1|2|12|1|2035|541|7050|Monday|2035Q1|N|N|N|2464360|2464390|2464006|2464279|N|N|N|N|N| +2464372|AAAAAAAAEHKJFCAA|2035-02-13|1621|7051|541|2035|2|2|13|1|2035|541|7051|Tuesday|2035Q1|N|N|N|2464360|2464390|2464007|2464280|N|N|N|N|N| +2464373|AAAAAAAAFHKJFCAA|2035-02-14|1621|7051|541|2035|3|2|14|1|2035|541|7051|Wednesday|2035Q1|N|N|N|2464360|2464390|2464008|2464281|N|N|N|N|N| +2464374|AAAAAAAAGHKJFCAA|2035-02-15|1621|7051|541|2035|4|2|15|1|2035|541|7051|Thursday|2035Q1|N|N|N|2464360|2464390|2464009|2464282|N|N|N|N|N| +2464375|AAAAAAAAHHKJFCAA|2035-02-16|1621|7051|541|2035|5|2|16|1|2035|541|7051|Friday|2035Q1|N|Y|N|2464360|2464390|2464010|2464283|N|N|N|N|N| +2464376|AAAAAAAAIHKJFCAA|2035-02-17|1621|7051|541|2035|6|2|17|1|2035|541|7051|Saturday|2035Q1|N|Y|N|2464360|2464390|2464011|2464284|N|N|N|N|N| +2464377|AAAAAAAAJHKJFCAA|2035-02-18|1621|7051|541|2035|0|2|18|1|2035|541|7051|Sunday|2035Q1|N|N|N|2464360|2464390|2464012|2464285|N|N|N|N|N| +2464378|AAAAAAAAKHKJFCAA|2035-02-19|1621|7051|541|2035|1|2|19|1|2035|541|7051|Monday|2035Q1|N|N|N|2464360|2464390|2464013|2464286|N|N|N|N|N| +2464379|AAAAAAAALHKJFCAA|2035-02-20|1621|7052|541|2035|2|2|20|1|2035|541|7052|Tuesday|2035Q1|N|N|N|2464360|2464390|2464014|2464287|N|N|N|N|N| +2464380|AAAAAAAAMHKJFCAA|2035-02-21|1621|7052|541|2035|3|2|21|1|2035|541|7052|Wednesday|2035Q1|N|N|N|2464360|2464390|2464015|2464288|N|N|N|N|N| +2464381|AAAAAAAANHKJFCAA|2035-02-22|1621|7052|541|2035|4|2|22|1|2035|541|7052|Thursday|2035Q1|N|N|N|2464360|2464390|2464016|2464289|N|N|N|N|N| +2464382|AAAAAAAAOHKJFCAA|2035-02-23|1621|7052|541|2035|5|2|23|1|2035|541|7052|Friday|2035Q1|N|Y|N|2464360|2464390|2464017|2464290|N|N|N|N|N| +2464383|AAAAAAAAPHKJFCAA|2035-02-24|1621|7052|541|2035|6|2|24|1|2035|541|7052|Saturday|2035Q1|N|Y|N|2464360|2464390|2464018|2464291|N|N|N|N|N| +2464384|AAAAAAAAAIKJFCAA|2035-02-25|1621|7052|541|2035|0|2|25|1|2035|541|7052|Sunday|2035Q1|N|N|N|2464360|2464390|2464019|2464292|N|N|N|N|N| +2464385|AAAAAAAABIKJFCAA|2035-02-26|1621|7052|541|2035|1|2|26|1|2035|541|7052|Monday|2035Q1|N|N|N|2464360|2464390|2464020|2464293|N|N|N|N|N| +2464386|AAAAAAAACIKJFCAA|2035-02-27|1621|7053|541|2035|2|2|27|1|2035|541|7053|Tuesday|2035Q1|N|N|N|2464360|2464390|2464021|2464294|N|N|N|N|N| +2464387|AAAAAAAADIKJFCAA|2035-02-28|1621|7053|541|2035|3|2|28|1|2035|541|7053|Wednesday|2035Q1|N|N|N|2464360|2464390|2464022|2464295|N|N|N|N|N| +2464388|AAAAAAAAEIKJFCAA|2035-03-01|1622|7053|542|2035|4|3|1|1|2035|542|7053|Thursday|2035Q1|N|N|N|2464388|2464446|2464023|2464296|N|N|N|N|N| +2464389|AAAAAAAAFIKJFCAA|2035-03-02|1622|7053|542|2035|5|3|2|1|2035|542|7053|Friday|2035Q1|N|Y|N|2464388|2464446|2464024|2464297|N|N|N|N|N| +2464390|AAAAAAAAGIKJFCAA|2035-03-03|1622|7053|542|2035|6|3|3|1|2035|542|7053|Saturday|2035Q1|N|Y|N|2464388|2464446|2464025|2464298|N|N|N|N|N| +2464391|AAAAAAAAHIKJFCAA|2035-03-04|1622|7053|542|2035|0|3|4|1|2035|542|7053|Sunday|2035Q1|N|N|N|2464388|2464446|2464026|2464299|N|N|N|N|N| +2464392|AAAAAAAAIIKJFCAA|2035-03-05|1622|7053|542|2035|1|3|5|1|2035|542|7053|Monday|2035Q1|N|N|N|2464388|2464446|2464027|2464300|N|N|N|N|N| +2464393|AAAAAAAAJIKJFCAA|2035-03-06|1622|7054|542|2035|2|3|6|1|2035|542|7054|Tuesday|2035Q1|N|N|N|2464388|2464446|2464028|2464301|N|N|N|N|N| +2464394|AAAAAAAAKIKJFCAA|2035-03-07|1622|7054|542|2035|3|3|7|1|2035|542|7054|Wednesday|2035Q1|N|N|N|2464388|2464446|2464029|2464302|N|N|N|N|N| +2464395|AAAAAAAALIKJFCAA|2035-03-08|1622|7054|542|2035|4|3|8|1|2035|542|7054|Thursday|2035Q1|N|N|N|2464388|2464446|2464030|2464303|N|N|N|N|N| +2464396|AAAAAAAAMIKJFCAA|2035-03-09|1622|7054|542|2035|5|3|9|1|2035|542|7054|Friday|2035Q1|N|Y|N|2464388|2464446|2464031|2464304|N|N|N|N|N| +2464397|AAAAAAAANIKJFCAA|2035-03-10|1622|7054|542|2035|6|3|10|1|2035|542|7054|Saturday|2035Q1|N|Y|N|2464388|2464446|2464032|2464305|N|N|N|N|N| +2464398|AAAAAAAAOIKJFCAA|2035-03-11|1622|7054|542|2035|0|3|11|1|2035|542|7054|Sunday|2035Q1|N|N|N|2464388|2464446|2464033|2464306|N|N|N|N|N| +2464399|AAAAAAAAPIKJFCAA|2035-03-12|1622|7054|542|2035|1|3|12|1|2035|542|7054|Monday|2035Q1|N|N|N|2464388|2464446|2464034|2464307|N|N|N|N|N| +2464400|AAAAAAAAAJKJFCAA|2035-03-13|1622|7055|542|2035|2|3|13|1|2035|542|7055|Tuesday|2035Q1|N|N|N|2464388|2464446|2464035|2464308|N|N|N|N|N| +2464401|AAAAAAAABJKJFCAA|2035-03-14|1622|7055|542|2035|3|3|14|1|2035|542|7055|Wednesday|2035Q1|N|N|N|2464388|2464446|2464036|2464309|N|N|N|N|N| +2464402|AAAAAAAACJKJFCAA|2035-03-15|1622|7055|542|2035|4|3|15|1|2035|542|7055|Thursday|2035Q1|N|N|N|2464388|2464446|2464037|2464310|N|N|N|N|N| +2464403|AAAAAAAADJKJFCAA|2035-03-16|1622|7055|542|2035|5|3|16|1|2035|542|7055|Friday|2035Q1|N|Y|N|2464388|2464446|2464038|2464311|N|N|N|N|N| +2464404|AAAAAAAAEJKJFCAA|2035-03-17|1622|7055|542|2035|6|3|17|1|2035|542|7055|Saturday|2035Q1|N|Y|N|2464388|2464446|2464039|2464312|N|N|N|N|N| +2464405|AAAAAAAAFJKJFCAA|2035-03-18|1622|7055|542|2035|0|3|18|1|2035|542|7055|Sunday|2035Q1|N|N|N|2464388|2464446|2464040|2464313|N|N|N|N|N| +2464406|AAAAAAAAGJKJFCAA|2035-03-19|1622|7055|542|2035|1|3|19|1|2035|542|7055|Monday|2035Q1|N|N|N|2464388|2464446|2464041|2464314|N|N|N|N|N| +2464407|AAAAAAAAHJKJFCAA|2035-03-20|1622|7056|542|2035|2|3|20|1|2035|542|7056|Tuesday|2035Q1|N|N|N|2464388|2464446|2464042|2464315|N|N|N|N|N| +2464408|AAAAAAAAIJKJFCAA|2035-03-21|1622|7056|542|2035|3|3|21|1|2035|542|7056|Wednesday|2035Q1|N|N|N|2464388|2464446|2464043|2464316|N|N|N|N|N| +2464409|AAAAAAAAJJKJFCAA|2035-03-22|1622|7056|542|2035|4|3|22|1|2035|542|7056|Thursday|2035Q1|N|N|N|2464388|2464446|2464044|2464317|N|N|N|N|N| +2464410|AAAAAAAAKJKJFCAA|2035-03-23|1622|7056|542|2035|5|3|23|1|2035|542|7056|Friday|2035Q1|N|Y|N|2464388|2464446|2464045|2464318|N|N|N|N|N| +2464411|AAAAAAAALJKJFCAA|2035-03-24|1622|7056|542|2035|6|3|24|1|2035|542|7056|Saturday|2035Q1|N|Y|N|2464388|2464446|2464046|2464319|N|N|N|N|N| +2464412|AAAAAAAAMJKJFCAA|2035-03-25|1622|7056|542|2035|0|3|25|1|2035|542|7056|Sunday|2035Q1|N|N|N|2464388|2464446|2464047|2464320|N|N|N|N|N| +2464413|AAAAAAAANJKJFCAA|2035-03-26|1622|7056|542|2035|1|3|26|1|2035|542|7056|Monday|2035Q1|N|N|N|2464388|2464446|2464048|2464321|N|N|N|N|N| +2464414|AAAAAAAAOJKJFCAA|2035-03-27|1622|7057|542|2035|2|3|27|1|2035|542|7057|Tuesday|2035Q1|N|N|N|2464388|2464446|2464049|2464322|N|N|N|N|N| +2464415|AAAAAAAAPJKJFCAA|2035-03-28|1622|7057|542|2035|3|3|28|1|2035|542|7057|Wednesday|2035Q1|N|N|N|2464388|2464446|2464050|2464323|N|N|N|N|N| +2464416|AAAAAAAAAKKJFCAA|2035-03-29|1622|7057|542|2035|4|3|29|1|2035|542|7057|Thursday|2035Q1|N|N|N|2464388|2464446|2464051|2464324|N|N|N|N|N| +2464417|AAAAAAAABKKJFCAA|2035-03-30|1622|7057|542|2035|5|3|30|1|2035|542|7057|Friday|2035Q1|N|Y|N|2464388|2464446|2464052|2464325|N|N|N|N|N| +2464418|AAAAAAAACKKJFCAA|2035-03-31|1622|7057|542|2035|6|3|31|1|2035|542|7057|Saturday|2035Q1|N|Y|N|2464388|2464446|2464053|2464326|N|N|N|N|N| +2464419|AAAAAAAADKKJFCAA|2035-04-01|1623|7057|542|2035|0|4|1|1|2035|542|7057|Sunday|2035Q1|N|N|N|2464419|2464508|2464054|2464329|N|N|N|N|N| +2464420|AAAAAAAAEKKJFCAA|2035-04-02|1623|7057|542|2035|1|4|2|2|2035|542|7057|Monday|2035Q2|N|N|N|2464419|2464508|2464055|2464330|N|N|N|N|N| +2464421|AAAAAAAAFKKJFCAA|2035-04-03|1623|7058|542|2035|2|4|3|2|2035|542|7058|Tuesday|2035Q2|N|N|N|2464419|2464508|2464056|2464331|N|N|N|N|N| +2464422|AAAAAAAAGKKJFCAA|2035-04-04|1623|7058|542|2035|3|4|4|2|2035|542|7058|Wednesday|2035Q2|N|N|N|2464419|2464508|2464057|2464332|N|N|N|N|N| +2464423|AAAAAAAAHKKJFCAA|2035-04-05|1623|7058|542|2035|4|4|5|2|2035|542|7058|Thursday|2035Q2|N|N|N|2464419|2464508|2464058|2464333|N|N|N|N|N| +2464424|AAAAAAAAIKKJFCAA|2035-04-06|1623|7058|542|2035|5|4|6|2|2035|542|7058|Friday|2035Q2|N|Y|N|2464419|2464508|2464059|2464334|N|N|N|N|N| +2464425|AAAAAAAAJKKJFCAA|2035-04-07|1623|7058|542|2035|6|4|7|2|2035|542|7058|Saturday|2035Q2|N|Y|N|2464419|2464508|2464060|2464335|N|N|N|N|N| +2464426|AAAAAAAAKKKJFCAA|2035-04-08|1623|7058|542|2035|0|4|8|2|2035|542|7058|Sunday|2035Q2|N|N|N|2464419|2464508|2464061|2464336|N|N|N|N|N| +2464427|AAAAAAAALKKJFCAA|2035-04-09|1623|7058|542|2035|1|4|9|2|2035|542|7058|Monday|2035Q2|N|N|N|2464419|2464508|2464062|2464337|N|N|N|N|N| +2464428|AAAAAAAAMKKJFCAA|2035-04-10|1623|7059|542|2035|2|4|10|2|2035|542|7059|Tuesday|2035Q2|N|N|N|2464419|2464508|2464063|2464338|N|N|N|N|N| +2464429|AAAAAAAANKKJFCAA|2035-04-11|1623|7059|542|2035|3|4|11|2|2035|542|7059|Wednesday|2035Q2|N|N|N|2464419|2464508|2464064|2464339|N|N|N|N|N| +2464430|AAAAAAAAOKKJFCAA|2035-04-12|1623|7059|542|2035|4|4|12|2|2035|542|7059|Thursday|2035Q2|N|N|N|2464419|2464508|2464065|2464340|N|N|N|N|N| +2464431|AAAAAAAAPKKJFCAA|2035-04-13|1623|7059|542|2035|5|4|13|2|2035|542|7059|Friday|2035Q2|N|Y|N|2464419|2464508|2464066|2464341|N|N|N|N|N| +2464432|AAAAAAAAALKJFCAA|2035-04-14|1623|7059|542|2035|6|4|14|2|2035|542|7059|Saturday|2035Q2|N|Y|N|2464419|2464508|2464067|2464342|N|N|N|N|N| +2464433|AAAAAAAABLKJFCAA|2035-04-15|1623|7059|542|2035|0|4|15|2|2035|542|7059|Sunday|2035Q2|N|N|N|2464419|2464508|2464068|2464343|N|N|N|N|N| +2464434|AAAAAAAACLKJFCAA|2035-04-16|1623|7059|542|2035|1|4|16|2|2035|542|7059|Monday|2035Q2|N|N|N|2464419|2464508|2464069|2464344|N|N|N|N|N| +2464435|AAAAAAAADLKJFCAA|2035-04-17|1623|7060|542|2035|2|4|17|2|2035|542|7060|Tuesday|2035Q2|N|N|N|2464419|2464508|2464070|2464345|N|N|N|N|N| +2464436|AAAAAAAAELKJFCAA|2035-04-18|1623|7060|542|2035|3|4|18|2|2035|542|7060|Wednesday|2035Q2|N|N|N|2464419|2464508|2464071|2464346|N|N|N|N|N| +2464437|AAAAAAAAFLKJFCAA|2035-04-19|1623|7060|542|2035|4|4|19|2|2035|542|7060|Thursday|2035Q2|N|N|N|2464419|2464508|2464072|2464347|N|N|N|N|N| +2464438|AAAAAAAAGLKJFCAA|2035-04-20|1623|7060|542|2035|5|4|20|2|2035|542|7060|Friday|2035Q2|N|Y|N|2464419|2464508|2464073|2464348|N|N|N|N|N| +2464439|AAAAAAAAHLKJFCAA|2035-04-21|1623|7060|542|2035|6|4|21|2|2035|542|7060|Saturday|2035Q2|N|Y|N|2464419|2464508|2464074|2464349|N|N|N|N|N| +2464440|AAAAAAAAILKJFCAA|2035-04-22|1623|7060|542|2035|0|4|22|2|2035|542|7060|Sunday|2035Q2|N|N|N|2464419|2464508|2464075|2464350|N|N|N|N|N| +2464441|AAAAAAAAJLKJFCAA|2035-04-23|1623|7060|542|2035|1|4|23|2|2035|542|7060|Monday|2035Q2|N|N|N|2464419|2464508|2464076|2464351|N|N|N|N|N| +2464442|AAAAAAAAKLKJFCAA|2035-04-24|1623|7061|542|2035|2|4|24|2|2035|542|7061|Tuesday|2035Q2|N|N|N|2464419|2464508|2464077|2464352|N|N|N|N|N| +2464443|AAAAAAAALLKJFCAA|2035-04-25|1623|7061|542|2035|3|4|25|2|2035|542|7061|Wednesday|2035Q2|N|N|N|2464419|2464508|2464078|2464353|N|N|N|N|N| +2464444|AAAAAAAAMLKJFCAA|2035-04-26|1623|7061|542|2035|4|4|26|2|2035|542|7061|Thursday|2035Q2|N|N|N|2464419|2464508|2464079|2464354|N|N|N|N|N| +2464445|AAAAAAAANLKJFCAA|2035-04-27|1623|7061|542|2035|5|4|27|2|2035|542|7061|Friday|2035Q2|N|Y|N|2464419|2464508|2464080|2464355|N|N|N|N|N| +2464446|AAAAAAAAOLKJFCAA|2035-04-28|1623|7061|542|2035|6|4|28|2|2035|542|7061|Saturday|2035Q2|N|Y|N|2464419|2464508|2464081|2464356|N|N|N|N|N| +2464447|AAAAAAAAPLKJFCAA|2035-04-29|1623|7061|542|2035|0|4|29|2|2035|542|7061|Sunday|2035Q2|N|N|N|2464419|2464508|2464082|2464357|N|N|N|N|N| +2464448|AAAAAAAAAMKJFCAA|2035-04-30|1623|7061|542|2035|1|4|30|2|2035|542|7061|Monday|2035Q2|N|N|N|2464419|2464508|2464083|2464358|N|N|N|N|N| +2464449|AAAAAAAABMKJFCAA|2035-05-01|1624|7062|542|2035|2|5|1|2|2035|542|7062|Tuesday|2035Q2|N|N|N|2464449|2464568|2464084|2464359|N|N|N|N|N| +2464450|AAAAAAAACMKJFCAA|2035-05-02|1624|7062|542|2035|3|5|2|2|2035|542|7062|Wednesday|2035Q2|N|N|N|2464449|2464568|2464085|2464360|N|N|N|N|N| +2464451|AAAAAAAADMKJFCAA|2035-05-03|1624|7062|542|2035|4|5|3|2|2035|542|7062|Thursday|2035Q2|N|N|N|2464449|2464568|2464086|2464361|N|N|N|N|N| +2464452|AAAAAAAAEMKJFCAA|2035-05-04|1624|7062|542|2035|5|5|4|2|2035|542|7062|Friday|2035Q2|N|Y|N|2464449|2464568|2464087|2464362|N|N|N|N|N| +2464453|AAAAAAAAFMKJFCAA|2035-05-05|1624|7062|542|2035|6|5|5|2|2035|542|7062|Saturday|2035Q2|N|Y|N|2464449|2464568|2464088|2464363|N|N|N|N|N| +2464454|AAAAAAAAGMKJFCAA|2035-05-06|1624|7062|542|2035|0|5|6|2|2035|542|7062|Sunday|2035Q2|N|N|N|2464449|2464568|2464089|2464364|N|N|N|N|N| +2464455|AAAAAAAAHMKJFCAA|2035-05-07|1624|7062|542|2035|1|5|7|2|2035|542|7062|Monday|2035Q2|N|N|N|2464449|2464568|2464090|2464365|N|N|N|N|N| +2464456|AAAAAAAAIMKJFCAA|2035-05-08|1624|7063|542|2035|2|5|8|2|2035|542|7063|Tuesday|2035Q2|N|N|N|2464449|2464568|2464091|2464366|N|N|N|N|N| +2464457|AAAAAAAAJMKJFCAA|2035-05-09|1624|7063|542|2035|3|5|9|2|2035|542|7063|Wednesday|2035Q2|N|N|N|2464449|2464568|2464092|2464367|N|N|N|N|N| +2464458|AAAAAAAAKMKJFCAA|2035-05-10|1624|7063|542|2035|4|5|10|2|2035|542|7063|Thursday|2035Q2|N|N|N|2464449|2464568|2464093|2464368|N|N|N|N|N| +2464459|AAAAAAAALMKJFCAA|2035-05-11|1624|7063|542|2035|5|5|11|2|2035|542|7063|Friday|2035Q2|N|Y|N|2464449|2464568|2464094|2464369|N|N|N|N|N| +2464460|AAAAAAAAMMKJFCAA|2035-05-12|1624|7063|542|2035|6|5|12|2|2035|542|7063|Saturday|2035Q2|N|Y|N|2464449|2464568|2464095|2464370|N|N|N|N|N| +2464461|AAAAAAAANMKJFCAA|2035-05-13|1624|7063|542|2035|0|5|13|2|2035|542|7063|Sunday|2035Q2|N|N|N|2464449|2464568|2464096|2464371|N|N|N|N|N| +2464462|AAAAAAAAOMKJFCAA|2035-05-14|1624|7063|542|2035|1|5|14|2|2035|542|7063|Monday|2035Q2|N|N|N|2464449|2464568|2464097|2464372|N|N|N|N|N| +2464463|AAAAAAAAPMKJFCAA|2035-05-15|1624|7064|542|2035|2|5|15|2|2035|542|7064|Tuesday|2035Q2|N|N|N|2464449|2464568|2464098|2464373|N|N|N|N|N| +2464464|AAAAAAAAANKJFCAA|2035-05-16|1624|7064|542|2035|3|5|16|2|2035|542|7064|Wednesday|2035Q2|N|N|N|2464449|2464568|2464099|2464374|N|N|N|N|N| +2464465|AAAAAAAABNKJFCAA|2035-05-17|1624|7064|542|2035|4|5|17|2|2035|542|7064|Thursday|2035Q2|N|N|N|2464449|2464568|2464100|2464375|N|N|N|N|N| +2464466|AAAAAAAACNKJFCAA|2035-05-18|1624|7064|542|2035|5|5|18|2|2035|542|7064|Friday|2035Q2|N|Y|N|2464449|2464568|2464101|2464376|N|N|N|N|N| +2464467|AAAAAAAADNKJFCAA|2035-05-19|1624|7064|542|2035|6|5|19|2|2035|542|7064|Saturday|2035Q2|N|Y|N|2464449|2464568|2464102|2464377|N|N|N|N|N| +2464468|AAAAAAAAENKJFCAA|2035-05-20|1624|7064|542|2035|0|5|20|2|2035|542|7064|Sunday|2035Q2|N|N|N|2464449|2464568|2464103|2464378|N|N|N|N|N| +2464469|AAAAAAAAFNKJFCAA|2035-05-21|1624|7064|542|2035|1|5|21|2|2035|542|7064|Monday|2035Q2|N|N|N|2464449|2464568|2464104|2464379|N|N|N|N|N| +2464470|AAAAAAAAGNKJFCAA|2035-05-22|1624|7065|542|2035|2|5|22|2|2035|542|7065|Tuesday|2035Q2|N|N|N|2464449|2464568|2464105|2464380|N|N|N|N|N| +2464471|AAAAAAAAHNKJFCAA|2035-05-23|1624|7065|542|2035|3|5|23|2|2035|542|7065|Wednesday|2035Q2|N|N|N|2464449|2464568|2464106|2464381|N|N|N|N|N| +2464472|AAAAAAAAINKJFCAA|2035-05-24|1624|7065|542|2035|4|5|24|2|2035|542|7065|Thursday|2035Q2|N|N|N|2464449|2464568|2464107|2464382|N|N|N|N|N| +2464473|AAAAAAAAJNKJFCAA|2035-05-25|1624|7065|542|2035|5|5|25|2|2035|542|7065|Friday|2035Q2|N|Y|N|2464449|2464568|2464108|2464383|N|N|N|N|N| +2464474|AAAAAAAAKNKJFCAA|2035-05-26|1624|7065|542|2035|6|5|26|2|2035|542|7065|Saturday|2035Q2|N|Y|N|2464449|2464568|2464109|2464384|N|N|N|N|N| +2464475|AAAAAAAALNKJFCAA|2035-05-27|1624|7065|542|2035|0|5|27|2|2035|542|7065|Sunday|2035Q2|N|N|N|2464449|2464568|2464110|2464385|N|N|N|N|N| +2464476|AAAAAAAAMNKJFCAA|2035-05-28|1624|7065|542|2035|1|5|28|2|2035|542|7065|Monday|2035Q2|N|N|N|2464449|2464568|2464111|2464386|N|N|N|N|N| +2464477|AAAAAAAANNKJFCAA|2035-05-29|1624|7066|542|2035|2|5|29|2|2035|542|7066|Tuesday|2035Q2|N|N|N|2464449|2464568|2464112|2464387|N|N|N|N|N| +2464478|AAAAAAAAONKJFCAA|2035-05-30|1624|7066|542|2035|3|5|30|2|2035|542|7066|Wednesday|2035Q2|N|N|N|2464449|2464568|2464113|2464388|N|N|N|N|N| +2464479|AAAAAAAAPNKJFCAA|2035-05-31|1624|7066|542|2035|4|5|31|2|2035|542|7066|Thursday|2035Q2|N|N|N|2464449|2464568|2464114|2464389|N|N|N|N|N| +2464480|AAAAAAAAAOKJFCAA|2035-06-01|1625|7066|543|2035|5|6|1|2|2035|543|7066|Friday|2035Q2|N|Y|N|2464480|2464630|2464115|2464390|N|N|N|N|N| +2464481|AAAAAAAABOKJFCAA|2035-06-02|1625|7066|543|2035|6|6|2|2|2035|543|7066|Saturday|2035Q2|N|Y|N|2464480|2464630|2464116|2464391|N|N|N|N|N| +2464482|AAAAAAAACOKJFCAA|2035-06-03|1625|7066|543|2035|0|6|3|2|2035|543|7066|Sunday|2035Q2|N|N|N|2464480|2464630|2464117|2464392|N|N|N|N|N| +2464483|AAAAAAAADOKJFCAA|2035-06-04|1625|7066|543|2035|1|6|4|2|2035|543|7066|Monday|2035Q2|N|N|N|2464480|2464630|2464118|2464393|N|N|N|N|N| +2464484|AAAAAAAAEOKJFCAA|2035-06-05|1625|7067|543|2035|2|6|5|2|2035|543|7067|Tuesday|2035Q2|N|N|N|2464480|2464630|2464119|2464394|N|N|N|N|N| +2464485|AAAAAAAAFOKJFCAA|2035-06-06|1625|7067|543|2035|3|6|6|2|2035|543|7067|Wednesday|2035Q2|N|N|N|2464480|2464630|2464120|2464395|N|N|N|N|N| +2464486|AAAAAAAAGOKJFCAA|2035-06-07|1625|7067|543|2035|4|6|7|2|2035|543|7067|Thursday|2035Q2|N|N|N|2464480|2464630|2464121|2464396|N|N|N|N|N| +2464487|AAAAAAAAHOKJFCAA|2035-06-08|1625|7067|543|2035|5|6|8|2|2035|543|7067|Friday|2035Q2|N|Y|N|2464480|2464630|2464122|2464397|N|N|N|N|N| +2464488|AAAAAAAAIOKJFCAA|2035-06-09|1625|7067|543|2035|6|6|9|2|2035|543|7067|Saturday|2035Q2|N|Y|N|2464480|2464630|2464123|2464398|N|N|N|N|N| +2464489|AAAAAAAAJOKJFCAA|2035-06-10|1625|7067|543|2035|0|6|10|2|2035|543|7067|Sunday|2035Q2|N|N|N|2464480|2464630|2464124|2464399|N|N|N|N|N| +2464490|AAAAAAAAKOKJFCAA|2035-06-11|1625|7067|543|2035|1|6|11|2|2035|543|7067|Monday|2035Q2|N|N|N|2464480|2464630|2464125|2464400|N|N|N|N|N| +2464491|AAAAAAAALOKJFCAA|2035-06-12|1625|7068|543|2035|2|6|12|2|2035|543|7068|Tuesday|2035Q2|N|N|N|2464480|2464630|2464126|2464401|N|N|N|N|N| +2464492|AAAAAAAAMOKJFCAA|2035-06-13|1625|7068|543|2035|3|6|13|2|2035|543|7068|Wednesday|2035Q2|N|N|N|2464480|2464630|2464127|2464402|N|N|N|N|N| +2464493|AAAAAAAANOKJFCAA|2035-06-14|1625|7068|543|2035|4|6|14|2|2035|543|7068|Thursday|2035Q2|N|N|N|2464480|2464630|2464128|2464403|N|N|N|N|N| +2464494|AAAAAAAAOOKJFCAA|2035-06-15|1625|7068|543|2035|5|6|15|2|2035|543|7068|Friday|2035Q2|N|Y|N|2464480|2464630|2464129|2464404|N|N|N|N|N| +2464495|AAAAAAAAPOKJFCAA|2035-06-16|1625|7068|543|2035|6|6|16|2|2035|543|7068|Saturday|2035Q2|N|Y|N|2464480|2464630|2464130|2464405|N|N|N|N|N| +2464496|AAAAAAAAAPKJFCAA|2035-06-17|1625|7068|543|2035|0|6|17|2|2035|543|7068|Sunday|2035Q2|N|N|N|2464480|2464630|2464131|2464406|N|N|N|N|N| +2464497|AAAAAAAABPKJFCAA|2035-06-18|1625|7068|543|2035|1|6|18|2|2035|543|7068|Monday|2035Q2|N|N|N|2464480|2464630|2464132|2464407|N|N|N|N|N| +2464498|AAAAAAAACPKJFCAA|2035-06-19|1625|7069|543|2035|2|6|19|2|2035|543|7069|Tuesday|2035Q2|N|N|N|2464480|2464630|2464133|2464408|N|N|N|N|N| +2464499|AAAAAAAADPKJFCAA|2035-06-20|1625|7069|543|2035|3|6|20|2|2035|543|7069|Wednesday|2035Q2|N|N|N|2464480|2464630|2464134|2464409|N|N|N|N|N| +2464500|AAAAAAAAEPKJFCAA|2035-06-21|1625|7069|543|2035|4|6|21|2|2035|543|7069|Thursday|2035Q2|N|N|N|2464480|2464630|2464135|2464410|N|N|N|N|N| +2464501|AAAAAAAAFPKJFCAA|2035-06-22|1625|7069|543|2035|5|6|22|2|2035|543|7069|Friday|2035Q2|N|Y|N|2464480|2464630|2464136|2464411|N|N|N|N|N| +2464502|AAAAAAAAGPKJFCAA|2035-06-23|1625|7069|543|2035|6|6|23|2|2035|543|7069|Saturday|2035Q2|N|Y|N|2464480|2464630|2464137|2464412|N|N|N|N|N| +2464503|AAAAAAAAHPKJFCAA|2035-06-24|1625|7069|543|2035|0|6|24|2|2035|543|7069|Sunday|2035Q2|N|N|N|2464480|2464630|2464138|2464413|N|N|N|N|N| +2464504|AAAAAAAAIPKJFCAA|2035-06-25|1625|7069|543|2035|1|6|25|2|2035|543|7069|Monday|2035Q2|N|N|N|2464480|2464630|2464139|2464414|N|N|N|N|N| +2464505|AAAAAAAAJPKJFCAA|2035-06-26|1625|7070|543|2035|2|6|26|2|2035|543|7070|Tuesday|2035Q2|N|N|N|2464480|2464630|2464140|2464415|N|N|N|N|N| +2464506|AAAAAAAAKPKJFCAA|2035-06-27|1625|7070|543|2035|3|6|27|2|2035|543|7070|Wednesday|2035Q2|N|N|N|2464480|2464630|2464141|2464416|N|N|N|N|N| +2464507|AAAAAAAALPKJFCAA|2035-06-28|1625|7070|543|2035|4|6|28|2|2035|543|7070|Thursday|2035Q2|N|N|N|2464480|2464630|2464142|2464417|N|N|N|N|N| +2464508|AAAAAAAAMPKJFCAA|2035-06-29|1625|7070|543|2035|5|6|29|2|2035|543|7070|Friday|2035Q2|N|Y|N|2464480|2464630|2464143|2464418|N|N|N|N|N| +2464509|AAAAAAAANPKJFCAA|2035-06-30|1625|7070|543|2035|6|6|30|2|2035|543|7070|Saturday|2035Q2|N|Y|N|2464480|2464630|2464144|2464419|N|N|N|N|N| +2464510|AAAAAAAAOPKJFCAA|2035-07-01|1626|7070|543|2035|0|7|1|2|2035|543|7070|Sunday|2035Q2|N|N|N|2464510|2464690|2464145|2464419|N|N|N|N|N| +2464511|AAAAAAAAPPKJFCAA|2035-07-02|1626|7070|543|2035|1|7|2|3|2035|543|7070|Monday|2035Q3|N|N|N|2464510|2464690|2464146|2464420|N|N|N|N|N| +2464512|AAAAAAAAAALJFCAA|2035-07-03|1626|7071|543|2035|2|7|3|3|2035|543|7071|Tuesday|2035Q3|N|N|N|2464510|2464690|2464147|2464421|N|N|N|N|N| +2464513|AAAAAAAABALJFCAA|2035-07-04|1626|7071|543|2035|3|7|4|3|2035|543|7071|Wednesday|2035Q3|N|N|N|2464510|2464690|2464148|2464422|N|N|N|N|N| +2464514|AAAAAAAACALJFCAA|2035-07-05|1626|7071|543|2035|4|7|5|3|2035|543|7071|Thursday|2035Q3|Y|N|N|2464510|2464690|2464149|2464423|N|N|N|N|N| +2464515|AAAAAAAADALJFCAA|2035-07-06|1626|7071|543|2035|5|7|6|3|2035|543|7071|Friday|2035Q3|N|Y|Y|2464510|2464690|2464150|2464424|N|N|N|N|N| +2464516|AAAAAAAAEALJFCAA|2035-07-07|1626|7071|543|2035|6|7|7|3|2035|543|7071|Saturday|2035Q3|N|Y|N|2464510|2464690|2464151|2464425|N|N|N|N|N| +2464517|AAAAAAAAFALJFCAA|2035-07-08|1626|7071|543|2035|0|7|8|3|2035|543|7071|Sunday|2035Q3|N|N|N|2464510|2464690|2464152|2464426|N|N|N|N|N| +2464518|AAAAAAAAGALJFCAA|2035-07-09|1626|7071|543|2035|1|7|9|3|2035|543|7071|Monday|2035Q3|N|N|N|2464510|2464690|2464153|2464427|N|N|N|N|N| +2464519|AAAAAAAAHALJFCAA|2035-07-10|1626|7072|543|2035|2|7|10|3|2035|543|7072|Tuesday|2035Q3|N|N|N|2464510|2464690|2464154|2464428|N|N|N|N|N| +2464520|AAAAAAAAIALJFCAA|2035-07-11|1626|7072|543|2035|3|7|11|3|2035|543|7072|Wednesday|2035Q3|N|N|N|2464510|2464690|2464155|2464429|N|N|N|N|N| +2464521|AAAAAAAAJALJFCAA|2035-07-12|1626|7072|543|2035|4|7|12|3|2035|543|7072|Thursday|2035Q3|N|N|N|2464510|2464690|2464156|2464430|N|N|N|N|N| +2464522|AAAAAAAAKALJFCAA|2035-07-13|1626|7072|543|2035|5|7|13|3|2035|543|7072|Friday|2035Q3|N|Y|N|2464510|2464690|2464157|2464431|N|N|N|N|N| +2464523|AAAAAAAALALJFCAA|2035-07-14|1626|7072|543|2035|6|7|14|3|2035|543|7072|Saturday|2035Q3|N|Y|N|2464510|2464690|2464158|2464432|N|N|N|N|N| +2464524|AAAAAAAAMALJFCAA|2035-07-15|1626|7072|543|2035|0|7|15|3|2035|543|7072|Sunday|2035Q3|N|N|N|2464510|2464690|2464159|2464433|N|N|N|N|N| +2464525|AAAAAAAANALJFCAA|2035-07-16|1626|7072|543|2035|1|7|16|3|2035|543|7072|Monday|2035Q3|N|N|N|2464510|2464690|2464160|2464434|N|N|N|N|N| +2464526|AAAAAAAAOALJFCAA|2035-07-17|1626|7073|543|2035|2|7|17|3|2035|543|7073|Tuesday|2035Q3|N|N|N|2464510|2464690|2464161|2464435|N|N|N|N|N| +2464527|AAAAAAAAPALJFCAA|2035-07-18|1626|7073|543|2035|3|7|18|3|2035|543|7073|Wednesday|2035Q3|N|N|N|2464510|2464690|2464162|2464436|N|N|N|N|N| +2464528|AAAAAAAAABLJFCAA|2035-07-19|1626|7073|543|2035|4|7|19|3|2035|543|7073|Thursday|2035Q3|N|N|N|2464510|2464690|2464163|2464437|N|N|N|N|N| +2464529|AAAAAAAABBLJFCAA|2035-07-20|1626|7073|543|2035|5|7|20|3|2035|543|7073|Friday|2035Q3|N|Y|N|2464510|2464690|2464164|2464438|N|N|N|N|N| +2464530|AAAAAAAACBLJFCAA|2035-07-21|1626|7073|543|2035|6|7|21|3|2035|543|7073|Saturday|2035Q3|N|Y|N|2464510|2464690|2464165|2464439|N|N|N|N|N| +2464531|AAAAAAAADBLJFCAA|2035-07-22|1626|7073|543|2035|0|7|22|3|2035|543|7073|Sunday|2035Q3|N|N|N|2464510|2464690|2464166|2464440|N|N|N|N|N| +2464532|AAAAAAAAEBLJFCAA|2035-07-23|1626|7073|543|2035|1|7|23|3|2035|543|7073|Monday|2035Q3|N|N|N|2464510|2464690|2464167|2464441|N|N|N|N|N| +2464533|AAAAAAAAFBLJFCAA|2035-07-24|1626|7074|543|2035|2|7|24|3|2035|543|7074|Tuesday|2035Q3|N|N|N|2464510|2464690|2464168|2464442|N|N|N|N|N| +2464534|AAAAAAAAGBLJFCAA|2035-07-25|1626|7074|543|2035|3|7|25|3|2035|543|7074|Wednesday|2035Q3|N|N|N|2464510|2464690|2464169|2464443|N|N|N|N|N| +2464535|AAAAAAAAHBLJFCAA|2035-07-26|1626|7074|543|2035|4|7|26|3|2035|543|7074|Thursday|2035Q3|N|N|N|2464510|2464690|2464170|2464444|N|N|N|N|N| +2464536|AAAAAAAAIBLJFCAA|2035-07-27|1626|7074|543|2035|5|7|27|3|2035|543|7074|Friday|2035Q3|N|Y|N|2464510|2464690|2464171|2464445|N|N|N|N|N| +2464537|AAAAAAAAJBLJFCAA|2035-07-28|1626|7074|543|2035|6|7|28|3|2035|543|7074|Saturday|2035Q3|N|Y|N|2464510|2464690|2464172|2464446|N|N|N|N|N| +2464538|AAAAAAAAKBLJFCAA|2035-07-29|1626|7074|543|2035|0|7|29|3|2035|543|7074|Sunday|2035Q3|N|N|N|2464510|2464690|2464173|2464447|N|N|N|N|N| +2464539|AAAAAAAALBLJFCAA|2035-07-30|1626|7074|543|2035|1|7|30|3|2035|543|7074|Monday|2035Q3|N|N|N|2464510|2464690|2464174|2464448|N|N|N|N|N| +2464540|AAAAAAAAMBLJFCAA|2035-07-31|1626|7075|543|2035|2|7|31|3|2035|543|7075|Tuesday|2035Q3|N|N|N|2464510|2464690|2464175|2464449|N|N|N|N|N| +2464541|AAAAAAAANBLJFCAA|2035-08-01|1627|7075|543|2035|3|8|1|3|2035|543|7075|Wednesday|2035Q3|N|N|N|2464541|2464752|2464176|2464450|N|N|N|N|N| +2464542|AAAAAAAAOBLJFCAA|2035-08-02|1627|7075|543|2035|4|8|2|3|2035|543|7075|Thursday|2035Q3|N|N|N|2464541|2464752|2464177|2464451|N|N|N|N|N| +2464543|AAAAAAAAPBLJFCAA|2035-08-03|1627|7075|543|2035|5|8|3|3|2035|543|7075|Friday|2035Q3|N|Y|N|2464541|2464752|2464178|2464452|N|N|N|N|N| +2464544|AAAAAAAAACLJFCAA|2035-08-04|1627|7075|543|2035|6|8|4|3|2035|543|7075|Saturday|2035Q3|N|Y|N|2464541|2464752|2464179|2464453|N|N|N|N|N| +2464545|AAAAAAAABCLJFCAA|2035-08-05|1627|7075|543|2035|0|8|5|3|2035|543|7075|Sunday|2035Q3|N|N|N|2464541|2464752|2464180|2464454|N|N|N|N|N| +2464546|AAAAAAAACCLJFCAA|2035-08-06|1627|7075|543|2035|1|8|6|3|2035|543|7075|Monday|2035Q3|N|N|N|2464541|2464752|2464181|2464455|N|N|N|N|N| +2464547|AAAAAAAADCLJFCAA|2035-08-07|1627|7076|543|2035|2|8|7|3|2035|543|7076|Tuesday|2035Q3|N|N|N|2464541|2464752|2464182|2464456|N|N|N|N|N| +2464548|AAAAAAAAECLJFCAA|2035-08-08|1627|7076|543|2035|3|8|8|3|2035|543|7076|Wednesday|2035Q3|N|N|N|2464541|2464752|2464183|2464457|N|N|N|N|N| +2464549|AAAAAAAAFCLJFCAA|2035-08-09|1627|7076|543|2035|4|8|9|3|2035|543|7076|Thursday|2035Q3|N|N|N|2464541|2464752|2464184|2464458|N|N|N|N|N| +2464550|AAAAAAAAGCLJFCAA|2035-08-10|1627|7076|543|2035|5|8|10|3|2035|543|7076|Friday|2035Q3|N|Y|N|2464541|2464752|2464185|2464459|N|N|N|N|N| +2464551|AAAAAAAAHCLJFCAA|2035-08-11|1627|7076|543|2035|6|8|11|3|2035|543|7076|Saturday|2035Q3|N|Y|N|2464541|2464752|2464186|2464460|N|N|N|N|N| +2464552|AAAAAAAAICLJFCAA|2035-08-12|1627|7076|543|2035|0|8|12|3|2035|543|7076|Sunday|2035Q3|N|N|N|2464541|2464752|2464187|2464461|N|N|N|N|N| +2464553|AAAAAAAAJCLJFCAA|2035-08-13|1627|7076|543|2035|1|8|13|3|2035|543|7076|Monday|2035Q3|N|N|N|2464541|2464752|2464188|2464462|N|N|N|N|N| +2464554|AAAAAAAAKCLJFCAA|2035-08-14|1627|7077|543|2035|2|8|14|3|2035|543|7077|Tuesday|2035Q3|N|N|N|2464541|2464752|2464189|2464463|N|N|N|N|N| +2464555|AAAAAAAALCLJFCAA|2035-08-15|1627|7077|543|2035|3|8|15|3|2035|543|7077|Wednesday|2035Q3|N|N|N|2464541|2464752|2464190|2464464|N|N|N|N|N| +2464556|AAAAAAAAMCLJFCAA|2035-08-16|1627|7077|543|2035|4|8|16|3|2035|543|7077|Thursday|2035Q3|N|N|N|2464541|2464752|2464191|2464465|N|N|N|N|N| +2464557|AAAAAAAANCLJFCAA|2035-08-17|1627|7077|543|2035|5|8|17|3|2035|543|7077|Friday|2035Q3|N|Y|N|2464541|2464752|2464192|2464466|N|N|N|N|N| +2464558|AAAAAAAAOCLJFCAA|2035-08-18|1627|7077|543|2035|6|8|18|3|2035|543|7077|Saturday|2035Q3|N|Y|N|2464541|2464752|2464193|2464467|N|N|N|N|N| +2464559|AAAAAAAAPCLJFCAA|2035-08-19|1627|7077|543|2035|0|8|19|3|2035|543|7077|Sunday|2035Q3|N|N|N|2464541|2464752|2464194|2464468|N|N|N|N|N| +2464560|AAAAAAAAADLJFCAA|2035-08-20|1627|7077|543|2035|1|8|20|3|2035|543|7077|Monday|2035Q3|N|N|N|2464541|2464752|2464195|2464469|N|N|N|N|N| +2464561|AAAAAAAABDLJFCAA|2035-08-21|1627|7078|543|2035|2|8|21|3|2035|543|7078|Tuesday|2035Q3|N|N|N|2464541|2464752|2464196|2464470|N|N|N|N|N| +2464562|AAAAAAAACDLJFCAA|2035-08-22|1627|7078|543|2035|3|8|22|3|2035|543|7078|Wednesday|2035Q3|N|N|N|2464541|2464752|2464197|2464471|N|N|N|N|N| +2464563|AAAAAAAADDLJFCAA|2035-08-23|1627|7078|543|2035|4|8|23|3|2035|543|7078|Thursday|2035Q3|N|N|N|2464541|2464752|2464198|2464472|N|N|N|N|N| +2464564|AAAAAAAAEDLJFCAA|2035-08-24|1627|7078|543|2035|5|8|24|3|2035|543|7078|Friday|2035Q3|N|Y|N|2464541|2464752|2464199|2464473|N|N|N|N|N| +2464565|AAAAAAAAFDLJFCAA|2035-08-25|1627|7078|543|2035|6|8|25|3|2035|543|7078|Saturday|2035Q3|N|Y|N|2464541|2464752|2464200|2464474|N|N|N|N|N| +2464566|AAAAAAAAGDLJFCAA|2035-08-26|1627|7078|543|2035|0|8|26|3|2035|543|7078|Sunday|2035Q3|N|N|N|2464541|2464752|2464201|2464475|N|N|N|N|N| +2464567|AAAAAAAAHDLJFCAA|2035-08-27|1627|7078|543|2035|1|8|27|3|2035|543|7078|Monday|2035Q3|N|N|N|2464541|2464752|2464202|2464476|N|N|N|N|N| +2464568|AAAAAAAAIDLJFCAA|2035-08-28|1627|7079|543|2035|2|8|28|3|2035|543|7079|Tuesday|2035Q3|N|N|N|2464541|2464752|2464203|2464477|N|N|N|N|N| +2464569|AAAAAAAAJDLJFCAA|2035-08-29|1627|7079|543|2035|3|8|29|3|2035|543|7079|Wednesday|2035Q3|N|N|N|2464541|2464752|2464204|2464478|N|N|N|N|N| +2464570|AAAAAAAAKDLJFCAA|2035-08-30|1627|7079|543|2035|4|8|30|3|2035|543|7079|Thursday|2035Q3|N|N|N|2464541|2464752|2464205|2464479|N|N|N|N|N| +2464571|AAAAAAAALDLJFCAA|2035-08-31|1627|7079|543|2035|5|8|31|3|2035|543|7079|Friday|2035Q3|N|Y|N|2464541|2464752|2464206|2464480|N|N|N|N|N| +2464572|AAAAAAAAMDLJFCAA|2035-09-01|1628|7079|544|2035|6|9|1|3|2035|544|7079|Saturday|2035Q3|N|Y|N|2464572|2464814|2464207|2464481|N|N|N|N|N| +2464573|AAAAAAAANDLJFCAA|2035-09-02|1628|7079|544|2035|0|9|2|3|2035|544|7079|Sunday|2035Q3|N|N|N|2464572|2464814|2464208|2464482|N|N|N|N|N| +2464574|AAAAAAAAODLJFCAA|2035-09-03|1628|7079|544|2035|1|9|3|3|2035|544|7079|Monday|2035Q3|N|N|N|2464572|2464814|2464209|2464483|N|N|N|N|N| +2464575|AAAAAAAAPDLJFCAA|2035-09-04|1628|7080|544|2035|2|9|4|3|2035|544|7080|Tuesday|2035Q3|N|N|N|2464572|2464814|2464210|2464484|N|N|N|N|N| +2464576|AAAAAAAAAELJFCAA|2035-09-05|1628|7080|544|2035|3|9|5|3|2035|544|7080|Wednesday|2035Q3|N|N|N|2464572|2464814|2464211|2464485|N|N|N|N|N| +2464577|AAAAAAAABELJFCAA|2035-09-06|1628|7080|544|2035|4|9|6|3|2035|544|7080|Thursday|2035Q3|N|N|N|2464572|2464814|2464212|2464486|N|N|N|N|N| +2464578|AAAAAAAACELJFCAA|2035-09-07|1628|7080|544|2035|5|9|7|3|2035|544|7080|Friday|2035Q3|N|Y|N|2464572|2464814|2464213|2464487|N|N|N|N|N| +2464579|AAAAAAAADELJFCAA|2035-09-08|1628|7080|544|2035|6|9|8|3|2035|544|7080|Saturday|2035Q3|N|Y|N|2464572|2464814|2464214|2464488|N|N|N|N|N| +2464580|AAAAAAAAEELJFCAA|2035-09-09|1628|7080|544|2035|0|9|9|3|2035|544|7080|Sunday|2035Q3|N|N|N|2464572|2464814|2464215|2464489|N|N|N|N|N| +2464581|AAAAAAAAFELJFCAA|2035-09-10|1628|7080|544|2035|1|9|10|3|2035|544|7080|Monday|2035Q3|N|N|N|2464572|2464814|2464216|2464490|N|N|N|N|N| +2464582|AAAAAAAAGELJFCAA|2035-09-11|1628|7081|544|2035|2|9|11|3|2035|544|7081|Tuesday|2035Q3|N|N|N|2464572|2464814|2464217|2464491|N|N|N|N|N| +2464583|AAAAAAAAHELJFCAA|2035-09-12|1628|7081|544|2035|3|9|12|3|2035|544|7081|Wednesday|2035Q3|N|N|N|2464572|2464814|2464218|2464492|N|N|N|N|N| +2464584|AAAAAAAAIELJFCAA|2035-09-13|1628|7081|544|2035|4|9|13|3|2035|544|7081|Thursday|2035Q3|N|N|N|2464572|2464814|2464219|2464493|N|N|N|N|N| +2464585|AAAAAAAAJELJFCAA|2035-09-14|1628|7081|544|2035|5|9|14|3|2035|544|7081|Friday|2035Q3|N|Y|N|2464572|2464814|2464220|2464494|N|N|N|N|N| +2464586|AAAAAAAAKELJFCAA|2035-09-15|1628|7081|544|2035|6|9|15|3|2035|544|7081|Saturday|2035Q3|N|Y|N|2464572|2464814|2464221|2464495|N|N|N|N|N| +2464587|AAAAAAAALELJFCAA|2035-09-16|1628|7081|544|2035|0|9|16|3|2035|544|7081|Sunday|2035Q3|N|N|N|2464572|2464814|2464222|2464496|N|N|N|N|N| +2464588|AAAAAAAAMELJFCAA|2035-09-17|1628|7081|544|2035|1|9|17|3|2035|544|7081|Monday|2035Q3|N|N|N|2464572|2464814|2464223|2464497|N|N|N|N|N| +2464589|AAAAAAAANELJFCAA|2035-09-18|1628|7082|544|2035|2|9|18|3|2035|544|7082|Tuesday|2035Q3|N|N|N|2464572|2464814|2464224|2464498|N|N|N|N|N| +2464590|AAAAAAAAOELJFCAA|2035-09-19|1628|7082|544|2035|3|9|19|3|2035|544|7082|Wednesday|2035Q3|N|N|N|2464572|2464814|2464225|2464499|N|N|N|N|N| +2464591|AAAAAAAAPELJFCAA|2035-09-20|1628|7082|544|2035|4|9|20|3|2035|544|7082|Thursday|2035Q3|N|N|N|2464572|2464814|2464226|2464500|N|N|N|N|N| +2464592|AAAAAAAAAFLJFCAA|2035-09-21|1628|7082|544|2035|5|9|21|3|2035|544|7082|Friday|2035Q3|N|Y|N|2464572|2464814|2464227|2464501|N|N|N|N|N| +2464593|AAAAAAAABFLJFCAA|2035-09-22|1628|7082|544|2035|6|9|22|3|2035|544|7082|Saturday|2035Q3|N|Y|N|2464572|2464814|2464228|2464502|N|N|N|N|N| +2464594|AAAAAAAACFLJFCAA|2035-09-23|1628|7082|544|2035|0|9|23|3|2035|544|7082|Sunday|2035Q3|N|N|N|2464572|2464814|2464229|2464503|N|N|N|N|N| +2464595|AAAAAAAADFLJFCAA|2035-09-24|1628|7082|544|2035|1|9|24|3|2035|544|7082|Monday|2035Q3|N|N|N|2464572|2464814|2464230|2464504|N|N|N|N|N| +2464596|AAAAAAAAEFLJFCAA|2035-09-25|1628|7083|544|2035|2|9|25|3|2035|544|7083|Tuesday|2035Q3|N|N|N|2464572|2464814|2464231|2464505|N|N|N|N|N| +2464597|AAAAAAAAFFLJFCAA|2035-09-26|1628|7083|544|2035|3|9|26|3|2035|544|7083|Wednesday|2035Q3|N|N|N|2464572|2464814|2464232|2464506|N|N|N|N|N| +2464598|AAAAAAAAGFLJFCAA|2035-09-27|1628|7083|544|2035|4|9|27|3|2035|544|7083|Thursday|2035Q3|N|N|N|2464572|2464814|2464233|2464507|N|N|N|N|N| +2464599|AAAAAAAAHFLJFCAA|2035-09-28|1628|7083|544|2035|5|9|28|3|2035|544|7083|Friday|2035Q3|N|Y|N|2464572|2464814|2464234|2464508|N|N|N|N|N| +2464600|AAAAAAAAIFLJFCAA|2035-09-29|1628|7083|544|2035|6|9|29|3|2035|544|7083|Saturday|2035Q3|N|Y|N|2464572|2464814|2464235|2464509|N|N|N|N|N| +2464601|AAAAAAAAJFLJFCAA|2035-09-30|1628|7083|544|2035|0|9|30|3|2035|544|7083|Sunday|2035Q3|N|N|N|2464572|2464814|2464236|2464510|N|N|N|N|N| +2464602|AAAAAAAAKFLJFCAA|2035-10-01|1629|7083|544|2035|1|10|1|3|2035|544|7083|Monday|2035Q3|N|N|N|2464602|2464874|2464237|2464510|N|N|N|N|N| +2464603|AAAAAAAALFLJFCAA|2035-10-02|1629|7084|544|2035|2|10|2|4|2035|544|7084|Tuesday|2035Q4|N|N|N|2464602|2464874|2464238|2464511|N|N|N|N|N| +2464604|AAAAAAAAMFLJFCAA|2035-10-03|1629|7084|544|2035|3|10|3|4|2035|544|7084|Wednesday|2035Q4|N|N|N|2464602|2464874|2464239|2464512|N|N|N|N|N| +2464605|AAAAAAAANFLJFCAA|2035-10-04|1629|7084|544|2035|4|10|4|4|2035|544|7084|Thursday|2035Q4|N|N|N|2464602|2464874|2464240|2464513|N|N|N|N|N| +2464606|AAAAAAAAOFLJFCAA|2035-10-05|1629|7084|544|2035|5|10|5|4|2035|544|7084|Friday|2035Q4|N|Y|N|2464602|2464874|2464241|2464514|N|N|N|N|N| +2464607|AAAAAAAAPFLJFCAA|2035-10-06|1629|7084|544|2035|6|10|6|4|2035|544|7084|Saturday|2035Q4|N|Y|N|2464602|2464874|2464242|2464515|N|N|N|N|N| +2464608|AAAAAAAAAGLJFCAA|2035-10-07|1629|7084|544|2035|0|10|7|4|2035|544|7084|Sunday|2035Q4|N|N|N|2464602|2464874|2464243|2464516|N|N|N|N|N| +2464609|AAAAAAAABGLJFCAA|2035-10-08|1629|7084|544|2035|1|10|8|4|2035|544|7084|Monday|2035Q4|N|N|N|2464602|2464874|2464244|2464517|N|N|N|N|N| +2464610|AAAAAAAACGLJFCAA|2035-10-09|1629|7085|544|2035|2|10|9|4|2035|544|7085|Tuesday|2035Q4|N|N|N|2464602|2464874|2464245|2464518|N|N|N|N|N| +2464611|AAAAAAAADGLJFCAA|2035-10-10|1629|7085|544|2035|3|10|10|4|2035|544|7085|Wednesday|2035Q4|N|N|N|2464602|2464874|2464246|2464519|N|N|N|N|N| +2464612|AAAAAAAAEGLJFCAA|2035-10-11|1629|7085|544|2035|4|10|11|4|2035|544|7085|Thursday|2035Q4|N|N|N|2464602|2464874|2464247|2464520|N|N|N|N|N| +2464613|AAAAAAAAFGLJFCAA|2035-10-12|1629|7085|544|2035|5|10|12|4|2035|544|7085|Friday|2035Q4|N|Y|N|2464602|2464874|2464248|2464521|N|N|N|N|N| +2464614|AAAAAAAAGGLJFCAA|2035-10-13|1629|7085|544|2035|6|10|13|4|2035|544|7085|Saturday|2035Q4|N|Y|N|2464602|2464874|2464249|2464522|N|N|N|N|N| +2464615|AAAAAAAAHGLJFCAA|2035-10-14|1629|7085|544|2035|0|10|14|4|2035|544|7085|Sunday|2035Q4|N|N|N|2464602|2464874|2464250|2464523|N|N|N|N|N| +2464616|AAAAAAAAIGLJFCAA|2035-10-15|1629|7085|544|2035|1|10|15|4|2035|544|7085|Monday|2035Q4|N|N|N|2464602|2464874|2464251|2464524|N|N|N|N|N| +2464617|AAAAAAAAJGLJFCAA|2035-10-16|1629|7086|544|2035|2|10|16|4|2035|544|7086|Tuesday|2035Q4|N|N|N|2464602|2464874|2464252|2464525|N|N|N|N|N| +2464618|AAAAAAAAKGLJFCAA|2035-10-17|1629|7086|544|2035|3|10|17|4|2035|544|7086|Wednesday|2035Q4|N|N|N|2464602|2464874|2464253|2464526|N|N|N|N|N| +2464619|AAAAAAAALGLJFCAA|2035-10-18|1629|7086|544|2035|4|10|18|4|2035|544|7086|Thursday|2035Q4|N|N|N|2464602|2464874|2464254|2464527|N|N|N|N|N| +2464620|AAAAAAAAMGLJFCAA|2035-10-19|1629|7086|544|2035|5|10|19|4|2035|544|7086|Friday|2035Q4|N|Y|N|2464602|2464874|2464255|2464528|N|N|N|N|N| +2464621|AAAAAAAANGLJFCAA|2035-10-20|1629|7086|544|2035|6|10|20|4|2035|544|7086|Saturday|2035Q4|N|Y|N|2464602|2464874|2464256|2464529|N|N|N|N|N| +2464622|AAAAAAAAOGLJFCAA|2035-10-21|1629|7086|544|2035|0|10|21|4|2035|544|7086|Sunday|2035Q4|N|N|N|2464602|2464874|2464257|2464530|N|N|N|N|N| +2464623|AAAAAAAAPGLJFCAA|2035-10-22|1629|7086|544|2035|1|10|22|4|2035|544|7086|Monday|2035Q4|N|N|N|2464602|2464874|2464258|2464531|N|N|N|N|N| +2464624|AAAAAAAAAHLJFCAA|2035-10-23|1629|7087|544|2035|2|10|23|4|2035|544|7087|Tuesday|2035Q4|N|N|N|2464602|2464874|2464259|2464532|N|N|N|N|N| +2464625|AAAAAAAABHLJFCAA|2035-10-24|1629|7087|544|2035|3|10|24|4|2035|544|7087|Wednesday|2035Q4|N|N|N|2464602|2464874|2464260|2464533|N|N|N|N|N| +2464626|AAAAAAAACHLJFCAA|2035-10-25|1629|7087|544|2035|4|10|25|4|2035|544|7087|Thursday|2035Q4|N|N|N|2464602|2464874|2464261|2464534|N|N|N|N|N| +2464627|AAAAAAAADHLJFCAA|2035-10-26|1629|7087|544|2035|5|10|26|4|2035|544|7087|Friday|2035Q4|N|Y|N|2464602|2464874|2464262|2464535|N|N|N|N|N| +2464628|AAAAAAAAEHLJFCAA|2035-10-27|1629|7087|544|2035|6|10|27|4|2035|544|7087|Saturday|2035Q4|N|Y|N|2464602|2464874|2464263|2464536|N|N|N|N|N| +2464629|AAAAAAAAFHLJFCAA|2035-10-28|1629|7087|544|2035|0|10|28|4|2035|544|7087|Sunday|2035Q4|N|N|N|2464602|2464874|2464264|2464537|N|N|N|N|N| +2464630|AAAAAAAAGHLJFCAA|2035-10-29|1629|7087|544|2035|1|10|29|4|2035|544|7087|Monday|2035Q4|N|N|N|2464602|2464874|2464265|2464538|N|N|N|N|N| +2464631|AAAAAAAAHHLJFCAA|2035-10-30|1629|7088|544|2035|2|10|30|4|2035|544|7088|Tuesday|2035Q4|N|N|N|2464602|2464874|2464266|2464539|N|N|N|N|N| +2464632|AAAAAAAAIHLJFCAA|2035-10-31|1629|7088|544|2035|3|10|31|4|2035|544|7088|Wednesday|2035Q4|N|N|N|2464602|2464874|2464267|2464540|N|N|N|N|N| +2464633|AAAAAAAAJHLJFCAA|2035-11-01|1630|7088|544|2035|4|11|1|4|2035|544|7088|Thursday|2035Q4|N|N|N|2464633|2464936|2464268|2464541|N|N|N|N|N| +2464634|AAAAAAAAKHLJFCAA|2035-11-02|1630|7088|544|2035|5|11|2|4|2035|544|7088|Friday|2035Q4|N|Y|N|2464633|2464936|2464269|2464542|N|N|N|N|N| +2464635|AAAAAAAALHLJFCAA|2035-11-03|1630|7088|544|2035|6|11|3|4|2035|544|7088|Saturday|2035Q4|N|Y|N|2464633|2464936|2464270|2464543|N|N|N|N|N| +2464636|AAAAAAAAMHLJFCAA|2035-11-04|1630|7088|544|2035|0|11|4|4|2035|544|7088|Sunday|2035Q4|N|N|N|2464633|2464936|2464271|2464544|N|N|N|N|N| +2464637|AAAAAAAANHLJFCAA|2035-11-05|1630|7088|544|2035|1|11|5|4|2035|544|7088|Monday|2035Q4|N|N|N|2464633|2464936|2464272|2464545|N|N|N|N|N| +2464638|AAAAAAAAOHLJFCAA|2035-11-06|1630|7089|544|2035|2|11|6|4|2035|544|7089|Tuesday|2035Q4|N|N|N|2464633|2464936|2464273|2464546|N|N|N|N|N| +2464639|AAAAAAAAPHLJFCAA|2035-11-07|1630|7089|544|2035|3|11|7|4|2035|544|7089|Wednesday|2035Q4|N|N|N|2464633|2464936|2464274|2464547|N|N|N|N|N| +2464640|AAAAAAAAAILJFCAA|2035-11-08|1630|7089|544|2035|4|11|8|4|2035|544|7089|Thursday|2035Q4|N|N|N|2464633|2464936|2464275|2464548|N|N|N|N|N| +2464641|AAAAAAAABILJFCAA|2035-11-09|1630|7089|544|2035|5|11|9|4|2035|544|7089|Friday|2035Q4|N|Y|N|2464633|2464936|2464276|2464549|N|N|N|N|N| +2464642|AAAAAAAACILJFCAA|2035-11-10|1630|7089|544|2035|6|11|10|4|2035|544|7089|Saturday|2035Q4|N|Y|N|2464633|2464936|2464277|2464550|N|N|N|N|N| +2464643|AAAAAAAADILJFCAA|2035-11-11|1630|7089|544|2035|0|11|11|4|2035|544|7089|Sunday|2035Q4|N|N|N|2464633|2464936|2464278|2464551|N|N|N|N|N| +2464644|AAAAAAAAEILJFCAA|2035-11-12|1630|7089|544|2035|1|11|12|4|2035|544|7089|Monday|2035Q4|N|N|N|2464633|2464936|2464279|2464552|N|N|N|N|N| +2464645|AAAAAAAAFILJFCAA|2035-11-13|1630|7090|544|2035|2|11|13|4|2035|544|7090|Tuesday|2035Q4|N|N|N|2464633|2464936|2464280|2464553|N|N|N|N|N| +2464646|AAAAAAAAGILJFCAA|2035-11-14|1630|7090|544|2035|3|11|14|4|2035|544|7090|Wednesday|2035Q4|N|N|N|2464633|2464936|2464281|2464554|N|N|N|N|N| +2464647|AAAAAAAAHILJFCAA|2035-11-15|1630|7090|544|2035|4|11|15|4|2035|544|7090|Thursday|2035Q4|N|N|N|2464633|2464936|2464282|2464555|N|N|N|N|N| +2464648|AAAAAAAAIILJFCAA|2035-11-16|1630|7090|544|2035|5|11|16|4|2035|544|7090|Friday|2035Q4|N|Y|N|2464633|2464936|2464283|2464556|N|N|N|N|N| +2464649|AAAAAAAAJILJFCAA|2035-11-17|1630|7090|544|2035|6|11|17|4|2035|544|7090|Saturday|2035Q4|N|Y|N|2464633|2464936|2464284|2464557|N|N|N|N|N| +2464650|AAAAAAAAKILJFCAA|2035-11-18|1630|7090|544|2035|0|11|18|4|2035|544|7090|Sunday|2035Q4|N|N|N|2464633|2464936|2464285|2464558|N|N|N|N|N| +2464651|AAAAAAAALILJFCAA|2035-11-19|1630|7090|544|2035|1|11|19|4|2035|544|7090|Monday|2035Q4|N|N|N|2464633|2464936|2464286|2464559|N|N|N|N|N| +2464652|AAAAAAAAMILJFCAA|2035-11-20|1630|7091|544|2035|2|11|20|4|2035|544|7091|Tuesday|2035Q4|N|N|N|2464633|2464936|2464287|2464560|N|N|N|N|N| +2464653|AAAAAAAANILJFCAA|2035-11-21|1630|7091|544|2035|3|11|21|4|2035|544|7091|Wednesday|2035Q4|N|N|N|2464633|2464936|2464288|2464561|N|N|N|N|N| +2464654|AAAAAAAAOILJFCAA|2035-11-22|1630|7091|544|2035|4|11|22|4|2035|544|7091|Thursday|2035Q4|N|N|N|2464633|2464936|2464289|2464562|N|N|N|N|N| +2464655|AAAAAAAAPILJFCAA|2035-11-23|1630|7091|544|2035|5|11|23|4|2035|544|7091|Friday|2035Q4|N|Y|N|2464633|2464936|2464290|2464563|N|N|N|N|N| +2464656|AAAAAAAAAJLJFCAA|2035-11-24|1630|7091|544|2035|6|11|24|4|2035|544|7091|Saturday|2035Q4|N|Y|N|2464633|2464936|2464291|2464564|N|N|N|N|N| +2464657|AAAAAAAABJLJFCAA|2035-11-25|1630|7091|544|2035|0|11|25|4|2035|544|7091|Sunday|2035Q4|N|N|N|2464633|2464936|2464292|2464565|N|N|N|N|N| +2464658|AAAAAAAACJLJFCAA|2035-11-26|1630|7091|544|2035|1|11|26|4|2035|544|7091|Monday|2035Q4|N|N|N|2464633|2464936|2464293|2464566|N|N|N|N|N| +2464659|AAAAAAAADJLJFCAA|2035-11-27|1630|7092|544|2035|2|11|27|4|2035|544|7092|Tuesday|2035Q4|N|N|N|2464633|2464936|2464294|2464567|N|N|N|N|N| +2464660|AAAAAAAAEJLJFCAA|2035-11-28|1630|7092|544|2035|3|11|28|4|2035|544|7092|Wednesday|2035Q4|N|N|N|2464633|2464936|2464295|2464568|N|N|N|N|N| +2464661|AAAAAAAAFJLJFCAA|2035-11-29|1630|7092|544|2035|4|11|29|4|2035|544|7092|Thursday|2035Q4|N|N|N|2464633|2464936|2464296|2464569|N|N|N|N|N| +2464662|AAAAAAAAGJLJFCAA|2035-11-30|1630|7092|544|2035|5|11|30|4|2035|544|7092|Friday|2035Q4|N|Y|N|2464633|2464936|2464297|2464570|N|N|N|N|N| +2464663|AAAAAAAAHJLJFCAA|2035-12-01|1631|7092|545|2035|6|12|1|4|2035|545|7092|Saturday|2035Q4|N|Y|N|2464663|2464996|2464298|2464571|N|N|N|N|N| +2464664|AAAAAAAAIJLJFCAA|2035-12-02|1631|7092|545|2035|0|12|2|4|2035|545|7092|Sunday|2035Q4|N|N|N|2464663|2464996|2464299|2464572|N|N|N|N|N| +2464665|AAAAAAAAJJLJFCAA|2035-12-03|1631|7092|545|2035|1|12|3|4|2035|545|7092|Monday|2035Q4|N|N|N|2464663|2464996|2464300|2464573|N|N|N|N|N| +2464666|AAAAAAAAKJLJFCAA|2035-12-04|1631|7093|545|2035|2|12|4|4|2035|545|7093|Tuesday|2035Q4|N|N|N|2464663|2464996|2464301|2464574|N|N|N|N|N| +2464667|AAAAAAAALJLJFCAA|2035-12-05|1631|7093|545|2035|3|12|5|4|2035|545|7093|Wednesday|2035Q4|N|N|N|2464663|2464996|2464302|2464575|N|N|N|N|N| +2464668|AAAAAAAAMJLJFCAA|2035-12-06|1631|7093|545|2035|4|12|6|4|2035|545|7093|Thursday|2035Q4|N|N|N|2464663|2464996|2464303|2464576|N|N|N|N|N| +2464669|AAAAAAAANJLJFCAA|2035-12-07|1631|7093|545|2035|5|12|7|4|2035|545|7093|Friday|2035Q4|N|Y|N|2464663|2464996|2464304|2464577|N|N|N|N|N| +2464670|AAAAAAAAOJLJFCAA|2035-12-08|1631|7093|545|2035|6|12|8|4|2035|545|7093|Saturday|2035Q4|N|Y|N|2464663|2464996|2464305|2464578|N|N|N|N|N| +2464671|AAAAAAAAPJLJFCAA|2035-12-09|1631|7093|545|2035|0|12|9|4|2035|545|7093|Sunday|2035Q4|N|N|N|2464663|2464996|2464306|2464579|N|N|N|N|N| +2464672|AAAAAAAAAKLJFCAA|2035-12-10|1631|7093|545|2035|1|12|10|4|2035|545|7093|Monday|2035Q4|N|N|N|2464663|2464996|2464307|2464580|N|N|N|N|N| +2464673|AAAAAAAABKLJFCAA|2035-12-11|1631|7094|545|2035|2|12|11|4|2035|545|7094|Tuesday|2035Q4|N|N|N|2464663|2464996|2464308|2464581|N|N|N|N|N| +2464674|AAAAAAAACKLJFCAA|2035-12-12|1631|7094|545|2035|3|12|12|4|2035|545|7094|Wednesday|2035Q4|N|N|N|2464663|2464996|2464309|2464582|N|N|N|N|N| +2464675|AAAAAAAADKLJFCAA|2035-12-13|1631|7094|545|2035|4|12|13|4|2035|545|7094|Thursday|2035Q4|N|N|N|2464663|2464996|2464310|2464583|N|N|N|N|N| +2464676|AAAAAAAAEKLJFCAA|2035-12-14|1631|7094|545|2035|5|12|14|4|2035|545|7094|Friday|2035Q4|N|Y|N|2464663|2464996|2464311|2464584|N|N|N|N|N| +2464677|AAAAAAAAFKLJFCAA|2035-12-15|1631|7094|545|2035|6|12|15|4|2035|545|7094|Saturday|2035Q4|N|Y|N|2464663|2464996|2464312|2464585|N|N|N|N|N| +2464678|AAAAAAAAGKLJFCAA|2035-12-16|1631|7094|545|2035|0|12|16|4|2035|545|7094|Sunday|2035Q4|N|N|N|2464663|2464996|2464313|2464586|N|N|N|N|N| +2464679|AAAAAAAAHKLJFCAA|2035-12-17|1631|7094|545|2035|1|12|17|4|2035|545|7094|Monday|2035Q4|N|N|N|2464663|2464996|2464314|2464587|N|N|N|N|N| +2464680|AAAAAAAAIKLJFCAA|2035-12-18|1631|7095|545|2035|2|12|18|4|2035|545|7095|Tuesday|2035Q4|N|N|N|2464663|2464996|2464315|2464588|N|N|N|N|N| +2464681|AAAAAAAAJKLJFCAA|2035-12-19|1631|7095|545|2035|3|12|19|4|2035|545|7095|Wednesday|2035Q4|N|N|N|2464663|2464996|2464316|2464589|N|N|N|N|N| +2464682|AAAAAAAAKKLJFCAA|2035-12-20|1631|7095|545|2035|4|12|20|4|2035|545|7095|Thursday|2035Q4|N|N|N|2464663|2464996|2464317|2464590|N|N|N|N|N| +2464683|AAAAAAAALKLJFCAA|2035-12-21|1631|7095|545|2035|5|12|21|4|2035|545|7095|Friday|2035Q4|N|Y|N|2464663|2464996|2464318|2464591|N|N|N|N|N| +2464684|AAAAAAAAMKLJFCAA|2035-12-22|1631|7095|545|2035|6|12|22|4|2035|545|7095|Saturday|2035Q4|N|Y|N|2464663|2464996|2464319|2464592|N|N|N|N|N| +2464685|AAAAAAAANKLJFCAA|2035-12-23|1631|7095|545|2035|0|12|23|4|2035|545|7095|Sunday|2035Q4|N|N|N|2464663|2464996|2464320|2464593|N|N|N|N|N| +2464686|AAAAAAAAOKLJFCAA|2035-12-24|1631|7095|545|2035|1|12|24|4|2035|545|7095|Monday|2035Q4|N|N|N|2464663|2464996|2464321|2464594|N|N|N|N|N| +2464687|AAAAAAAAPKLJFCAA|2035-12-25|1631|7096|545|2035|2|12|25|4|2035|545|7096|Tuesday|2035Q4|N|N|N|2464663|2464996|2464322|2464595|N|N|N|N|N| +2464688|AAAAAAAAALLJFCAA|2035-12-26|1631|7096|545|2035|3|12|26|4|2035|545|7096|Wednesday|2035Q4|Y|N|N|2464663|2464996|2464323|2464596|N|N|N|N|N| +2464689|AAAAAAAABLLJFCAA|2035-12-27|1631|7096|545|2035|4|12|27|4|2035|545|7096|Thursday|2035Q4|N|N|Y|2464663|2464996|2464324|2464597|N|N|N|N|N| +2464690|AAAAAAAACLLJFCAA|2035-12-28|1631|7096|545|2035|5|12|28|4|2035|545|7096|Friday|2035Q4|N|Y|N|2464663|2464996|2464325|2464598|N|N|N|N|N| +2464691|AAAAAAAADLLJFCAA|2035-12-29|1631|7096|545|2035|6|12|29|4|2035|545|7096|Saturday|2035Q4|N|Y|N|2464663|2464996|2464326|2464599|N|N|N|N|N| +2464692|AAAAAAAAELLJFCAA|2035-12-30|1631|7096|545|2035|0|12|30|4|2035|545|7096|Sunday|2035Q4|N|N|N|2464663|2464996|2464327|2464600|N|N|N|N|N| +2464693|AAAAAAAAFLLJFCAA|2035-12-31|1631|7096|545|2035|1|12|31|4|2035|545|7096|Monday|2035Q4|N|N|N|2464663|2464996|2464328|2464601|N|N|N|N|N| +2464694|AAAAAAAAGLLJFCAA|2036-01-01|1632|7097|545|2036|2|1|1|1|2036|545|7097|Tuesday|2036Q1|Y|N|N|2464694|2464693|2464329|2464602|N|N|N|N|N| +2464695|AAAAAAAAHLLJFCAA|2036-01-02|1632|7097|545|2036|3|1|2|1|2036|545|7097|Wednesday|2036Q1|N|N|Y|2464694|2464693|2464330|2464603|N|N|N|N|N| +2464696|AAAAAAAAILLJFCAA|2036-01-03|1632|7097|545|2036|4|1|3|1|2036|545|7097|Thursday|2036Q1|N|N|N|2464694|2464693|2464331|2464604|N|N|N|N|N| +2464697|AAAAAAAAJLLJFCAA|2036-01-04|1632|7097|545|2036|5|1|4|1|2036|545|7097|Friday|2036Q1|N|Y|N|2464694|2464693|2464332|2464605|N|N|N|N|N| +2464698|AAAAAAAAKLLJFCAA|2036-01-05|1632|7097|545|2036|6|1|5|1|2036|545|7097|Saturday|2036Q1|N|Y|N|2464694|2464693|2464333|2464606|N|N|N|N|N| +2464699|AAAAAAAALLLJFCAA|2036-01-06|1632|7097|545|2036|0|1|6|1|2036|545|7097|Sunday|2036Q1|N|N|N|2464694|2464693|2464334|2464607|N|N|N|N|N| +2464700|AAAAAAAAMLLJFCAA|2036-01-07|1632|7097|545|2036|1|1|7|1|2036|545|7097|Monday|2036Q1|N|N|N|2464694|2464693|2464335|2464608|N|N|N|N|N| +2464701|AAAAAAAANLLJFCAA|2036-01-08|1632|7098|545|2036|2|1|8|1|2036|545|7098|Tuesday|2036Q1|N|N|N|2464694|2464693|2464336|2464609|N|N|N|N|N| +2464702|AAAAAAAAOLLJFCAA|2036-01-09|1632|7098|545|2036|3|1|9|1|2036|545|7098|Wednesday|2036Q1|N|N|N|2464694|2464693|2464337|2464610|N|N|N|N|N| +2464703|AAAAAAAAPLLJFCAA|2036-01-10|1632|7098|545|2036|4|1|10|1|2036|545|7098|Thursday|2036Q1|N|N|N|2464694|2464693|2464338|2464611|N|N|N|N|N| +2464704|AAAAAAAAAMLJFCAA|2036-01-11|1632|7098|545|2036|5|1|11|1|2036|545|7098|Friday|2036Q1|N|Y|N|2464694|2464693|2464339|2464612|N|N|N|N|N| +2464705|AAAAAAAABMLJFCAA|2036-01-12|1632|7098|545|2036|6|1|12|1|2036|545|7098|Saturday|2036Q1|N|Y|N|2464694|2464693|2464340|2464613|N|N|N|N|N| +2464706|AAAAAAAACMLJFCAA|2036-01-13|1632|7098|545|2036|0|1|13|1|2036|545|7098|Sunday|2036Q1|N|N|N|2464694|2464693|2464341|2464614|N|N|N|N|N| +2464707|AAAAAAAADMLJFCAA|2036-01-14|1632|7098|545|2036|1|1|14|1|2036|545|7098|Monday|2036Q1|N|N|N|2464694|2464693|2464342|2464615|N|N|N|N|N| +2464708|AAAAAAAAEMLJFCAA|2036-01-15|1632|7099|545|2036|2|1|15|1|2036|545|7099|Tuesday|2036Q1|N|N|N|2464694|2464693|2464343|2464616|N|N|N|N|N| +2464709|AAAAAAAAFMLJFCAA|2036-01-16|1632|7099|545|2036|3|1|16|1|2036|545|7099|Wednesday|2036Q1|N|N|N|2464694|2464693|2464344|2464617|N|N|N|N|N| +2464710|AAAAAAAAGMLJFCAA|2036-01-17|1632|7099|545|2036|4|1|17|1|2036|545|7099|Thursday|2036Q1|N|N|N|2464694|2464693|2464345|2464618|N|N|N|N|N| +2464711|AAAAAAAAHMLJFCAA|2036-01-18|1632|7099|545|2036|5|1|18|1|2036|545|7099|Friday|2036Q1|N|Y|N|2464694|2464693|2464346|2464619|N|N|N|N|N| +2464712|AAAAAAAAIMLJFCAA|2036-01-19|1632|7099|545|2036|6|1|19|1|2036|545|7099|Saturday|2036Q1|N|Y|N|2464694|2464693|2464347|2464620|N|N|N|N|N| +2464713|AAAAAAAAJMLJFCAA|2036-01-20|1632|7099|545|2036|0|1|20|1|2036|545|7099|Sunday|2036Q1|N|N|N|2464694|2464693|2464348|2464621|N|N|N|N|N| +2464714|AAAAAAAAKMLJFCAA|2036-01-21|1632|7099|545|2036|1|1|21|1|2036|545|7099|Monday|2036Q1|N|N|N|2464694|2464693|2464349|2464622|N|N|N|N|N| +2464715|AAAAAAAALMLJFCAA|2036-01-22|1632|7100|545|2036|2|1|22|1|2036|545|7100|Tuesday|2036Q1|N|N|N|2464694|2464693|2464350|2464623|N|N|N|N|N| +2464716|AAAAAAAAMMLJFCAA|2036-01-23|1632|7100|545|2036|3|1|23|1|2036|545|7100|Wednesday|2036Q1|N|N|N|2464694|2464693|2464351|2464624|N|N|N|N|N| +2464717|AAAAAAAANMLJFCAA|2036-01-24|1632|7100|545|2036|4|1|24|1|2036|545|7100|Thursday|2036Q1|N|N|N|2464694|2464693|2464352|2464625|N|N|N|N|N| +2464718|AAAAAAAAOMLJFCAA|2036-01-25|1632|7100|545|2036|5|1|25|1|2036|545|7100|Friday|2036Q1|N|Y|N|2464694|2464693|2464353|2464626|N|N|N|N|N| +2464719|AAAAAAAAPMLJFCAA|2036-01-26|1632|7100|545|2036|6|1|26|1|2036|545|7100|Saturday|2036Q1|N|Y|N|2464694|2464693|2464354|2464627|N|N|N|N|N| +2464720|AAAAAAAAANLJFCAA|2036-01-27|1632|7100|545|2036|0|1|27|1|2036|545|7100|Sunday|2036Q1|N|N|N|2464694|2464693|2464355|2464628|N|N|N|N|N| +2464721|AAAAAAAABNLJFCAA|2036-01-28|1632|7100|545|2036|1|1|28|1|2036|545|7100|Monday|2036Q1|N|N|N|2464694|2464693|2464356|2464629|N|N|N|N|N| +2464722|AAAAAAAACNLJFCAA|2036-01-29|1632|7101|545|2036|2|1|29|1|2036|545|7101|Tuesday|2036Q1|N|N|N|2464694|2464693|2464357|2464630|N|N|N|N|N| +2464723|AAAAAAAADNLJFCAA|2036-01-30|1632|7101|545|2036|3|1|30|1|2036|545|7101|Wednesday|2036Q1|N|N|N|2464694|2464693|2464358|2464631|N|N|N|N|N| +2464724|AAAAAAAAENLJFCAA|2036-01-31|1632|7101|545|2036|4|1|31|1|2036|545|7101|Thursday|2036Q1|N|N|N|2464694|2464693|2464359|2464632|N|N|N|N|N| +2464725|AAAAAAAAFNLJFCAA|2036-02-01|1633|7101|545|2036|5|2|1|1|2036|545|7101|Friday|2036Q1|N|Y|N|2464725|2464755|2464360|2464633|N|N|N|N|N| +2464726|AAAAAAAAGNLJFCAA|2036-02-02|1633|7101|545|2036|6|2|2|1|2036|545|7101|Saturday|2036Q1|N|Y|N|2464725|2464755|2464361|2464634|N|N|N|N|N| +2464727|AAAAAAAAHNLJFCAA|2036-02-03|1633|7101|545|2036|0|2|3|1|2036|545|7101|Sunday|2036Q1|N|N|N|2464725|2464755|2464362|2464635|N|N|N|N|N| +2464728|AAAAAAAAINLJFCAA|2036-02-04|1633|7101|545|2036|1|2|4|1|2036|545|7101|Monday|2036Q1|N|N|N|2464725|2464755|2464363|2464636|N|N|N|N|N| +2464729|AAAAAAAAJNLJFCAA|2036-02-05|1633|7102|545|2036|2|2|5|1|2036|545|7102|Tuesday|2036Q1|N|N|N|2464725|2464755|2464364|2464637|N|N|N|N|N| +2464730|AAAAAAAAKNLJFCAA|2036-02-06|1633|7102|545|2036|3|2|6|1|2036|545|7102|Wednesday|2036Q1|N|N|N|2464725|2464755|2464365|2464638|N|N|N|N|N| +2464731|AAAAAAAALNLJFCAA|2036-02-07|1633|7102|545|2036|4|2|7|1|2036|545|7102|Thursday|2036Q1|N|N|N|2464725|2464755|2464366|2464639|N|N|N|N|N| +2464732|AAAAAAAAMNLJFCAA|2036-02-08|1633|7102|545|2036|5|2|8|1|2036|545|7102|Friday|2036Q1|N|Y|N|2464725|2464755|2464367|2464640|N|N|N|N|N| +2464733|AAAAAAAANNLJFCAA|2036-02-09|1633|7102|545|2036|6|2|9|1|2036|545|7102|Saturday|2036Q1|N|Y|N|2464725|2464755|2464368|2464641|N|N|N|N|N| +2464734|AAAAAAAAONLJFCAA|2036-02-10|1633|7102|545|2036|0|2|10|1|2036|545|7102|Sunday|2036Q1|N|N|N|2464725|2464755|2464369|2464642|N|N|N|N|N| +2464735|AAAAAAAAPNLJFCAA|2036-02-11|1633|7102|545|2036|1|2|11|1|2036|545|7102|Monday|2036Q1|N|N|N|2464725|2464755|2464370|2464643|N|N|N|N|N| +2464736|AAAAAAAAAOLJFCAA|2036-02-12|1633|7103|545|2036|2|2|12|1|2036|545|7103|Tuesday|2036Q1|N|N|N|2464725|2464755|2464371|2464644|N|N|N|N|N| +2464737|AAAAAAAABOLJFCAA|2036-02-13|1633|7103|545|2036|3|2|13|1|2036|545|7103|Wednesday|2036Q1|N|N|N|2464725|2464755|2464372|2464645|N|N|N|N|N| +2464738|AAAAAAAACOLJFCAA|2036-02-14|1633|7103|545|2036|4|2|14|1|2036|545|7103|Thursday|2036Q1|N|N|N|2464725|2464755|2464373|2464646|N|N|N|N|N| +2464739|AAAAAAAADOLJFCAA|2036-02-15|1633|7103|545|2036|5|2|15|1|2036|545|7103|Friday|2036Q1|N|Y|N|2464725|2464755|2464374|2464647|N|N|N|N|N| +2464740|AAAAAAAAEOLJFCAA|2036-02-16|1633|7103|545|2036|6|2|16|1|2036|545|7103|Saturday|2036Q1|N|Y|N|2464725|2464755|2464375|2464648|N|N|N|N|N| +2464741|AAAAAAAAFOLJFCAA|2036-02-17|1633|7103|545|2036|0|2|17|1|2036|545|7103|Sunday|2036Q1|N|N|N|2464725|2464755|2464376|2464649|N|N|N|N|N| +2464742|AAAAAAAAGOLJFCAA|2036-02-18|1633|7103|545|2036|1|2|18|1|2036|545|7103|Monday|2036Q1|N|N|N|2464725|2464755|2464377|2464650|N|N|N|N|N| +2464743|AAAAAAAAHOLJFCAA|2036-02-19|1633|7104|545|2036|2|2|19|1|2036|545|7104|Tuesday|2036Q1|N|N|N|2464725|2464755|2464378|2464651|N|N|N|N|N| +2464744|AAAAAAAAIOLJFCAA|2036-02-20|1633|7104|545|2036|3|2|20|1|2036|545|7104|Wednesday|2036Q1|N|N|N|2464725|2464755|2464379|2464652|N|N|N|N|N| +2464745|AAAAAAAAJOLJFCAA|2036-02-21|1633|7104|545|2036|4|2|21|1|2036|545|7104|Thursday|2036Q1|N|N|N|2464725|2464755|2464380|2464653|N|N|N|N|N| +2464746|AAAAAAAAKOLJFCAA|2036-02-22|1633|7104|545|2036|5|2|22|1|2036|545|7104|Friday|2036Q1|N|Y|N|2464725|2464755|2464381|2464654|N|N|N|N|N| +2464747|AAAAAAAALOLJFCAA|2036-02-23|1633|7104|545|2036|6|2|23|1|2036|545|7104|Saturday|2036Q1|N|Y|N|2464725|2464755|2464382|2464655|N|N|N|N|N| +2464748|AAAAAAAAMOLJFCAA|2036-02-24|1633|7104|545|2036|0|2|24|1|2036|545|7104|Sunday|2036Q1|N|N|N|2464725|2464755|2464383|2464656|N|N|N|N|N| +2464749|AAAAAAAANOLJFCAA|2036-02-25|1633|7104|545|2036|1|2|25|1|2036|545|7104|Monday|2036Q1|N|N|N|2464725|2464755|2464384|2464657|N|N|N|N|N| +2464750|AAAAAAAAOOLJFCAA|2036-02-26|1633|7105|545|2036|2|2|26|1|2036|545|7105|Tuesday|2036Q1|N|N|N|2464725|2464755|2464385|2464658|N|N|N|N|N| +2464751|AAAAAAAAPOLJFCAA|2036-02-27|1633|7105|545|2036|3|2|27|1|2036|545|7105|Wednesday|2036Q1|N|N|N|2464725|2464755|2464386|2464659|N|N|N|N|N| +2464752|AAAAAAAAAPLJFCAA|2036-02-28|1633|7105|545|2036|4|2|28|1|2036|545|7105|Thursday|2036Q1|N|N|N|2464725|2464755|2464387|2464660|N|N|N|N|N| +2464753|AAAAAAAABPLJFCAA|2036-02-29|1633|7105|545|2036|5|2|29|1|2036|545|7105|Friday|2036Q1|N|Y|N|2464725|2464755|2464387|2464661|N|N|N|N|N| +2464754|AAAAAAAACPLJFCAA|2036-03-01|1634|7105|546|2036|6|3|1|1|2036|546|7105|Saturday|2036Q1|N|Y|N|2464754|2464813|2464388|2464662|N|N|N|N|N| +2464755|AAAAAAAADPLJFCAA|2036-03-02|1634|7105|546|2036|0|3|2|1|2036|546|7105|Sunday|2036Q1|N|N|N|2464754|2464813|2464389|2464663|N|N|N|N|N| +2464756|AAAAAAAAEPLJFCAA|2036-03-03|1634|7105|546|2036|1|3|3|1|2036|546|7105|Monday|2036Q1|N|N|N|2464754|2464813|2464390|2464664|N|N|N|N|N| +2464757|AAAAAAAAFPLJFCAA|2036-03-04|1634|7106|546|2036|2|3|4|1|2036|546|7106|Tuesday|2036Q1|N|N|N|2464754|2464813|2464391|2464665|N|N|N|N|N| +2464758|AAAAAAAAGPLJFCAA|2036-03-05|1634|7106|546|2036|3|3|5|1|2036|546|7106|Wednesday|2036Q1|N|N|N|2464754|2464813|2464392|2464666|N|N|N|N|N| +2464759|AAAAAAAAHPLJFCAA|2036-03-06|1634|7106|546|2036|4|3|6|1|2036|546|7106|Thursday|2036Q1|N|N|N|2464754|2464813|2464393|2464667|N|N|N|N|N| +2464760|AAAAAAAAIPLJFCAA|2036-03-07|1634|7106|546|2036|5|3|7|1|2036|546|7106|Friday|2036Q1|N|Y|N|2464754|2464813|2464394|2464668|N|N|N|N|N| +2464761|AAAAAAAAJPLJFCAA|2036-03-08|1634|7106|546|2036|6|3|8|1|2036|546|7106|Saturday|2036Q1|N|Y|N|2464754|2464813|2464395|2464669|N|N|N|N|N| +2464762|AAAAAAAAKPLJFCAA|2036-03-09|1634|7106|546|2036|0|3|9|1|2036|546|7106|Sunday|2036Q1|N|N|N|2464754|2464813|2464396|2464670|N|N|N|N|N| +2464763|AAAAAAAALPLJFCAA|2036-03-10|1634|7106|546|2036|1|3|10|1|2036|546|7106|Monday|2036Q1|N|N|N|2464754|2464813|2464397|2464671|N|N|N|N|N| +2464764|AAAAAAAAMPLJFCAA|2036-03-11|1634|7107|546|2036|2|3|11|1|2036|546|7107|Tuesday|2036Q1|N|N|N|2464754|2464813|2464398|2464672|N|N|N|N|N| +2464765|AAAAAAAANPLJFCAA|2036-03-12|1634|7107|546|2036|3|3|12|1|2036|546|7107|Wednesday|2036Q1|N|N|N|2464754|2464813|2464399|2464673|N|N|N|N|N| +2464766|AAAAAAAAOPLJFCAA|2036-03-13|1634|7107|546|2036|4|3|13|1|2036|546|7107|Thursday|2036Q1|N|N|N|2464754|2464813|2464400|2464674|N|N|N|N|N| +2464767|AAAAAAAAPPLJFCAA|2036-03-14|1634|7107|546|2036|5|3|14|1|2036|546|7107|Friday|2036Q1|N|Y|N|2464754|2464813|2464401|2464675|N|N|N|N|N| +2464768|AAAAAAAAAAMJFCAA|2036-03-15|1634|7107|546|2036|6|3|15|1|2036|546|7107|Saturday|2036Q1|N|Y|N|2464754|2464813|2464402|2464676|N|N|N|N|N| +2464769|AAAAAAAABAMJFCAA|2036-03-16|1634|7107|546|2036|0|3|16|1|2036|546|7107|Sunday|2036Q1|N|N|N|2464754|2464813|2464403|2464677|N|N|N|N|N| +2464770|AAAAAAAACAMJFCAA|2036-03-17|1634|7107|546|2036|1|3|17|1|2036|546|7107|Monday|2036Q1|N|N|N|2464754|2464813|2464404|2464678|N|N|N|N|N| +2464771|AAAAAAAADAMJFCAA|2036-03-18|1634|7108|546|2036|2|3|18|1|2036|546|7108|Tuesday|2036Q1|N|N|N|2464754|2464813|2464405|2464679|N|N|N|N|N| +2464772|AAAAAAAAEAMJFCAA|2036-03-19|1634|7108|546|2036|3|3|19|1|2036|546|7108|Wednesday|2036Q1|N|N|N|2464754|2464813|2464406|2464680|N|N|N|N|N| +2464773|AAAAAAAAFAMJFCAA|2036-03-20|1634|7108|546|2036|4|3|20|1|2036|546|7108|Thursday|2036Q1|N|N|N|2464754|2464813|2464407|2464681|N|N|N|N|N| +2464774|AAAAAAAAGAMJFCAA|2036-03-21|1634|7108|546|2036|5|3|21|1|2036|546|7108|Friday|2036Q1|N|Y|N|2464754|2464813|2464408|2464682|N|N|N|N|N| +2464775|AAAAAAAAHAMJFCAA|2036-03-22|1634|7108|546|2036|6|3|22|1|2036|546|7108|Saturday|2036Q1|N|Y|N|2464754|2464813|2464409|2464683|N|N|N|N|N| +2464776|AAAAAAAAIAMJFCAA|2036-03-23|1634|7108|546|2036|0|3|23|1|2036|546|7108|Sunday|2036Q1|N|N|N|2464754|2464813|2464410|2464684|N|N|N|N|N| +2464777|AAAAAAAAJAMJFCAA|2036-03-24|1634|7108|546|2036|1|3|24|1|2036|546|7108|Monday|2036Q1|N|N|N|2464754|2464813|2464411|2464685|N|N|N|N|N| +2464778|AAAAAAAAKAMJFCAA|2036-03-25|1634|7109|546|2036|2|3|25|1|2036|546|7109|Tuesday|2036Q1|N|N|N|2464754|2464813|2464412|2464686|N|N|N|N|N| +2464779|AAAAAAAALAMJFCAA|2036-03-26|1634|7109|546|2036|3|3|26|1|2036|546|7109|Wednesday|2036Q1|N|N|N|2464754|2464813|2464413|2464687|N|N|N|N|N| +2464780|AAAAAAAAMAMJFCAA|2036-03-27|1634|7109|546|2036|4|3|27|1|2036|546|7109|Thursday|2036Q1|N|N|N|2464754|2464813|2464414|2464688|N|N|N|N|N| +2464781|AAAAAAAANAMJFCAA|2036-03-28|1634|7109|546|2036|5|3|28|1|2036|546|7109|Friday|2036Q1|N|Y|N|2464754|2464813|2464415|2464689|N|N|N|N|N| +2464782|AAAAAAAAOAMJFCAA|2036-03-29|1634|7109|546|2036|6|3|29|1|2036|546|7109|Saturday|2036Q1|N|Y|N|2464754|2464813|2464416|2464690|N|N|N|N|N| +2464783|AAAAAAAAPAMJFCAA|2036-03-30|1634|7109|546|2036|0|3|30|1|2036|546|7109|Sunday|2036Q1|N|N|N|2464754|2464813|2464417|2464691|N|N|N|N|N| +2464784|AAAAAAAAABMJFCAA|2036-03-31|1634|7109|546|2036|1|3|31|1|2036|546|7109|Monday|2036Q1|N|N|N|2464754|2464813|2464418|2464692|N|N|N|N|N| +2464785|AAAAAAAABBMJFCAA|2036-04-01|1635|7110|546|2036|2|4|1|2|2036|546|7110|Tuesday|2036Q2|N|N|N|2464785|2464875|2464419|2464694|N|N|N|N|N| +2464786|AAAAAAAACBMJFCAA|2036-04-02|1635|7110|546|2036|3|4|2|2|2036|546|7110|Wednesday|2036Q2|N|N|N|2464785|2464875|2464420|2464695|N|N|N|N|N| +2464787|AAAAAAAADBMJFCAA|2036-04-03|1635|7110|546|2036|4|4|3|2|2036|546|7110|Thursday|2036Q2|N|N|N|2464785|2464875|2464421|2464696|N|N|N|N|N| +2464788|AAAAAAAAEBMJFCAA|2036-04-04|1635|7110|546|2036|5|4|4|2|2036|546|7110|Friday|2036Q2|N|Y|N|2464785|2464875|2464422|2464697|N|N|N|N|N| +2464789|AAAAAAAAFBMJFCAA|2036-04-05|1635|7110|546|2036|6|4|5|2|2036|546|7110|Saturday|2036Q2|N|Y|N|2464785|2464875|2464423|2464698|N|N|N|N|N| +2464790|AAAAAAAAGBMJFCAA|2036-04-06|1635|7110|546|2036|0|4|6|2|2036|546|7110|Sunday|2036Q2|N|N|N|2464785|2464875|2464424|2464699|N|N|N|N|N| +2464791|AAAAAAAAHBMJFCAA|2036-04-07|1635|7110|546|2036|1|4|7|2|2036|546|7110|Monday|2036Q2|N|N|N|2464785|2464875|2464425|2464700|N|N|N|N|N| +2464792|AAAAAAAAIBMJFCAA|2036-04-08|1635|7111|546|2036|2|4|8|2|2036|546|7111|Tuesday|2036Q2|N|N|N|2464785|2464875|2464426|2464701|N|N|N|N|N| +2464793|AAAAAAAAJBMJFCAA|2036-04-09|1635|7111|546|2036|3|4|9|2|2036|546|7111|Wednesday|2036Q2|N|N|N|2464785|2464875|2464427|2464702|N|N|N|N|N| +2464794|AAAAAAAAKBMJFCAA|2036-04-10|1635|7111|546|2036|4|4|10|2|2036|546|7111|Thursday|2036Q2|N|N|N|2464785|2464875|2464428|2464703|N|N|N|N|N| +2464795|AAAAAAAALBMJFCAA|2036-04-11|1635|7111|546|2036|5|4|11|2|2036|546|7111|Friday|2036Q2|N|Y|N|2464785|2464875|2464429|2464704|N|N|N|N|N| +2464796|AAAAAAAAMBMJFCAA|2036-04-12|1635|7111|546|2036|6|4|12|2|2036|546|7111|Saturday|2036Q2|N|Y|N|2464785|2464875|2464430|2464705|N|N|N|N|N| +2464797|AAAAAAAANBMJFCAA|2036-04-13|1635|7111|546|2036|0|4|13|2|2036|546|7111|Sunday|2036Q2|N|N|N|2464785|2464875|2464431|2464706|N|N|N|N|N| +2464798|AAAAAAAAOBMJFCAA|2036-04-14|1635|7111|546|2036|1|4|14|2|2036|546|7111|Monday|2036Q2|N|N|N|2464785|2464875|2464432|2464707|N|N|N|N|N| +2464799|AAAAAAAAPBMJFCAA|2036-04-15|1635|7112|546|2036|2|4|15|2|2036|546|7112|Tuesday|2036Q2|N|N|N|2464785|2464875|2464433|2464708|N|N|N|N|N| +2464800|AAAAAAAAACMJFCAA|2036-04-16|1635|7112|546|2036|3|4|16|2|2036|546|7112|Wednesday|2036Q2|N|N|N|2464785|2464875|2464434|2464709|N|N|N|N|N| +2464801|AAAAAAAABCMJFCAA|2036-04-17|1635|7112|546|2036|4|4|17|2|2036|546|7112|Thursday|2036Q2|N|N|N|2464785|2464875|2464435|2464710|N|N|N|N|N| +2464802|AAAAAAAACCMJFCAA|2036-04-18|1635|7112|546|2036|5|4|18|2|2036|546|7112|Friday|2036Q2|N|Y|N|2464785|2464875|2464436|2464711|N|N|N|N|N| +2464803|AAAAAAAADCMJFCAA|2036-04-19|1635|7112|546|2036|6|4|19|2|2036|546|7112|Saturday|2036Q2|N|Y|N|2464785|2464875|2464437|2464712|N|N|N|N|N| +2464804|AAAAAAAAECMJFCAA|2036-04-20|1635|7112|546|2036|0|4|20|2|2036|546|7112|Sunday|2036Q2|N|N|N|2464785|2464875|2464438|2464713|N|N|N|N|N| +2464805|AAAAAAAAFCMJFCAA|2036-04-21|1635|7112|546|2036|1|4|21|2|2036|546|7112|Monday|2036Q2|N|N|N|2464785|2464875|2464439|2464714|N|N|N|N|N| +2464806|AAAAAAAAGCMJFCAA|2036-04-22|1635|7113|546|2036|2|4|22|2|2036|546|7113|Tuesday|2036Q2|N|N|N|2464785|2464875|2464440|2464715|N|N|N|N|N| +2464807|AAAAAAAAHCMJFCAA|2036-04-23|1635|7113|546|2036|3|4|23|2|2036|546|7113|Wednesday|2036Q2|N|N|N|2464785|2464875|2464441|2464716|N|N|N|N|N| +2464808|AAAAAAAAICMJFCAA|2036-04-24|1635|7113|546|2036|4|4|24|2|2036|546|7113|Thursday|2036Q2|N|N|N|2464785|2464875|2464442|2464717|N|N|N|N|N| +2464809|AAAAAAAAJCMJFCAA|2036-04-25|1635|7113|546|2036|5|4|25|2|2036|546|7113|Friday|2036Q2|N|Y|N|2464785|2464875|2464443|2464718|N|N|N|N|N| +2464810|AAAAAAAAKCMJFCAA|2036-04-26|1635|7113|546|2036|6|4|26|2|2036|546|7113|Saturday|2036Q2|N|Y|N|2464785|2464875|2464444|2464719|N|N|N|N|N| +2464811|AAAAAAAALCMJFCAA|2036-04-27|1635|7113|546|2036|0|4|27|2|2036|546|7113|Sunday|2036Q2|N|N|N|2464785|2464875|2464445|2464720|N|N|N|N|N| +2464812|AAAAAAAAMCMJFCAA|2036-04-28|1635|7113|546|2036|1|4|28|2|2036|546|7113|Monday|2036Q2|N|N|N|2464785|2464875|2464446|2464721|N|N|N|N|N| +2464813|AAAAAAAANCMJFCAA|2036-04-29|1635|7114|546|2036|2|4|29|2|2036|546|7114|Tuesday|2036Q2|N|N|N|2464785|2464875|2464447|2464722|N|N|N|N|N| +2464814|AAAAAAAAOCMJFCAA|2036-04-30|1635|7114|546|2036|3|4|30|2|2036|546|7114|Wednesday|2036Q2|N|N|N|2464785|2464875|2464448|2464723|N|N|N|N|N| +2464815|AAAAAAAAPCMJFCAA|2036-05-01|1636|7114|546|2036|4|5|1|2|2036|546|7114|Thursday|2036Q2|N|N|N|2464815|2464935|2464449|2464724|N|N|N|N|N| +2464816|AAAAAAAAADMJFCAA|2036-05-02|1636|7114|546|2036|5|5|2|2|2036|546|7114|Friday|2036Q2|N|Y|N|2464815|2464935|2464450|2464725|N|N|N|N|N| +2464817|AAAAAAAABDMJFCAA|2036-05-03|1636|7114|546|2036|6|5|3|2|2036|546|7114|Saturday|2036Q2|N|Y|N|2464815|2464935|2464451|2464726|N|N|N|N|N| +2464818|AAAAAAAACDMJFCAA|2036-05-04|1636|7114|546|2036|0|5|4|2|2036|546|7114|Sunday|2036Q2|N|N|N|2464815|2464935|2464452|2464727|N|N|N|N|N| +2464819|AAAAAAAADDMJFCAA|2036-05-05|1636|7114|546|2036|1|5|5|2|2036|546|7114|Monday|2036Q2|N|N|N|2464815|2464935|2464453|2464728|N|N|N|N|N| +2464820|AAAAAAAAEDMJFCAA|2036-05-06|1636|7115|546|2036|2|5|6|2|2036|546|7115|Tuesday|2036Q2|N|N|N|2464815|2464935|2464454|2464729|N|N|N|N|N| +2464821|AAAAAAAAFDMJFCAA|2036-05-07|1636|7115|546|2036|3|5|7|2|2036|546|7115|Wednesday|2036Q2|N|N|N|2464815|2464935|2464455|2464730|N|N|N|N|N| +2464822|AAAAAAAAGDMJFCAA|2036-05-08|1636|7115|546|2036|4|5|8|2|2036|546|7115|Thursday|2036Q2|N|N|N|2464815|2464935|2464456|2464731|N|N|N|N|N| +2464823|AAAAAAAAHDMJFCAA|2036-05-09|1636|7115|546|2036|5|5|9|2|2036|546|7115|Friday|2036Q2|N|Y|N|2464815|2464935|2464457|2464732|N|N|N|N|N| +2464824|AAAAAAAAIDMJFCAA|2036-05-10|1636|7115|546|2036|6|5|10|2|2036|546|7115|Saturday|2036Q2|N|Y|N|2464815|2464935|2464458|2464733|N|N|N|N|N| +2464825|AAAAAAAAJDMJFCAA|2036-05-11|1636|7115|546|2036|0|5|11|2|2036|546|7115|Sunday|2036Q2|N|N|N|2464815|2464935|2464459|2464734|N|N|N|N|N| +2464826|AAAAAAAAKDMJFCAA|2036-05-12|1636|7115|546|2036|1|5|12|2|2036|546|7115|Monday|2036Q2|N|N|N|2464815|2464935|2464460|2464735|N|N|N|N|N| +2464827|AAAAAAAALDMJFCAA|2036-05-13|1636|7116|546|2036|2|5|13|2|2036|546|7116|Tuesday|2036Q2|N|N|N|2464815|2464935|2464461|2464736|N|N|N|N|N| +2464828|AAAAAAAAMDMJFCAA|2036-05-14|1636|7116|546|2036|3|5|14|2|2036|546|7116|Wednesday|2036Q2|N|N|N|2464815|2464935|2464462|2464737|N|N|N|N|N| +2464829|AAAAAAAANDMJFCAA|2036-05-15|1636|7116|546|2036|4|5|15|2|2036|546|7116|Thursday|2036Q2|N|N|N|2464815|2464935|2464463|2464738|N|N|N|N|N| +2464830|AAAAAAAAODMJFCAA|2036-05-16|1636|7116|546|2036|5|5|16|2|2036|546|7116|Friday|2036Q2|N|Y|N|2464815|2464935|2464464|2464739|N|N|N|N|N| +2464831|AAAAAAAAPDMJFCAA|2036-05-17|1636|7116|546|2036|6|5|17|2|2036|546|7116|Saturday|2036Q2|N|Y|N|2464815|2464935|2464465|2464740|N|N|N|N|N| +2464832|AAAAAAAAAEMJFCAA|2036-05-18|1636|7116|546|2036|0|5|18|2|2036|546|7116|Sunday|2036Q2|N|N|N|2464815|2464935|2464466|2464741|N|N|N|N|N| +2464833|AAAAAAAABEMJFCAA|2036-05-19|1636|7116|546|2036|1|5|19|2|2036|546|7116|Monday|2036Q2|N|N|N|2464815|2464935|2464467|2464742|N|N|N|N|N| +2464834|AAAAAAAACEMJFCAA|2036-05-20|1636|7117|546|2036|2|5|20|2|2036|546|7117|Tuesday|2036Q2|N|N|N|2464815|2464935|2464468|2464743|N|N|N|N|N| +2464835|AAAAAAAADEMJFCAA|2036-05-21|1636|7117|546|2036|3|5|21|2|2036|546|7117|Wednesday|2036Q2|N|N|N|2464815|2464935|2464469|2464744|N|N|N|N|N| +2464836|AAAAAAAAEEMJFCAA|2036-05-22|1636|7117|546|2036|4|5|22|2|2036|546|7117|Thursday|2036Q2|N|N|N|2464815|2464935|2464470|2464745|N|N|N|N|N| +2464837|AAAAAAAAFEMJFCAA|2036-05-23|1636|7117|546|2036|5|5|23|2|2036|546|7117|Friday|2036Q2|N|Y|N|2464815|2464935|2464471|2464746|N|N|N|N|N| +2464838|AAAAAAAAGEMJFCAA|2036-05-24|1636|7117|546|2036|6|5|24|2|2036|546|7117|Saturday|2036Q2|N|Y|N|2464815|2464935|2464472|2464747|N|N|N|N|N| +2464839|AAAAAAAAHEMJFCAA|2036-05-25|1636|7117|546|2036|0|5|25|2|2036|546|7117|Sunday|2036Q2|N|N|N|2464815|2464935|2464473|2464748|N|N|N|N|N| +2464840|AAAAAAAAIEMJFCAA|2036-05-26|1636|7117|546|2036|1|5|26|2|2036|546|7117|Monday|2036Q2|N|N|N|2464815|2464935|2464474|2464749|N|N|N|N|N| +2464841|AAAAAAAAJEMJFCAA|2036-05-27|1636|7118|546|2036|2|5|27|2|2036|546|7118|Tuesday|2036Q2|N|N|N|2464815|2464935|2464475|2464750|N|N|N|N|N| +2464842|AAAAAAAAKEMJFCAA|2036-05-28|1636|7118|546|2036|3|5|28|2|2036|546|7118|Wednesday|2036Q2|N|N|N|2464815|2464935|2464476|2464751|N|N|N|N|N| +2464843|AAAAAAAALEMJFCAA|2036-05-29|1636|7118|546|2036|4|5|29|2|2036|546|7118|Thursday|2036Q2|N|N|N|2464815|2464935|2464477|2464752|N|N|N|N|N| +2464844|AAAAAAAAMEMJFCAA|2036-05-30|1636|7118|546|2036|5|5|30|2|2036|546|7118|Friday|2036Q2|N|Y|N|2464815|2464935|2464478|2464753|N|N|N|N|N| +2464845|AAAAAAAANEMJFCAA|2036-05-31|1636|7118|546|2036|6|5|31|2|2036|546|7118|Saturday|2036Q2|N|Y|N|2464815|2464935|2464479|2464754|N|N|N|N|N| +2464846|AAAAAAAAOEMJFCAA|2036-06-01|1637|7118|547|2036|0|6|1|2|2036|547|7118|Sunday|2036Q2|N|N|N|2464846|2464997|2464480|2464755|N|N|N|N|N| +2464847|AAAAAAAAPEMJFCAA|2036-06-02|1637|7118|547|2036|1|6|2|2|2036|547|7118|Monday|2036Q2|N|N|N|2464846|2464997|2464481|2464756|N|N|N|N|N| +2464848|AAAAAAAAAFMJFCAA|2036-06-03|1637|7119|547|2036|2|6|3|2|2036|547|7119|Tuesday|2036Q2|N|N|N|2464846|2464997|2464482|2464757|N|N|N|N|N| +2464849|AAAAAAAABFMJFCAA|2036-06-04|1637|7119|547|2036|3|6|4|2|2036|547|7119|Wednesday|2036Q2|N|N|N|2464846|2464997|2464483|2464758|N|N|N|N|N| +2464850|AAAAAAAACFMJFCAA|2036-06-05|1637|7119|547|2036|4|6|5|2|2036|547|7119|Thursday|2036Q2|N|N|N|2464846|2464997|2464484|2464759|N|N|N|N|N| +2464851|AAAAAAAADFMJFCAA|2036-06-06|1637|7119|547|2036|5|6|6|2|2036|547|7119|Friday|2036Q2|N|Y|N|2464846|2464997|2464485|2464760|N|N|N|N|N| +2464852|AAAAAAAAEFMJFCAA|2036-06-07|1637|7119|547|2036|6|6|7|2|2036|547|7119|Saturday|2036Q2|N|Y|N|2464846|2464997|2464486|2464761|N|N|N|N|N| +2464853|AAAAAAAAFFMJFCAA|2036-06-08|1637|7119|547|2036|0|6|8|2|2036|547|7119|Sunday|2036Q2|N|N|N|2464846|2464997|2464487|2464762|N|N|N|N|N| +2464854|AAAAAAAAGFMJFCAA|2036-06-09|1637|7119|547|2036|1|6|9|2|2036|547|7119|Monday|2036Q2|N|N|N|2464846|2464997|2464488|2464763|N|N|N|N|N| +2464855|AAAAAAAAHFMJFCAA|2036-06-10|1637|7120|547|2036|2|6|10|2|2036|547|7120|Tuesday|2036Q2|N|N|N|2464846|2464997|2464489|2464764|N|N|N|N|N| +2464856|AAAAAAAAIFMJFCAA|2036-06-11|1637|7120|547|2036|3|6|11|2|2036|547|7120|Wednesday|2036Q2|N|N|N|2464846|2464997|2464490|2464765|N|N|N|N|N| +2464857|AAAAAAAAJFMJFCAA|2036-06-12|1637|7120|547|2036|4|6|12|2|2036|547|7120|Thursday|2036Q2|N|N|N|2464846|2464997|2464491|2464766|N|N|N|N|N| +2464858|AAAAAAAAKFMJFCAA|2036-06-13|1637|7120|547|2036|5|6|13|2|2036|547|7120|Friday|2036Q2|N|Y|N|2464846|2464997|2464492|2464767|N|N|N|N|N| +2464859|AAAAAAAALFMJFCAA|2036-06-14|1637|7120|547|2036|6|6|14|2|2036|547|7120|Saturday|2036Q2|N|Y|N|2464846|2464997|2464493|2464768|N|N|N|N|N| +2464860|AAAAAAAAMFMJFCAA|2036-06-15|1637|7120|547|2036|0|6|15|2|2036|547|7120|Sunday|2036Q2|N|N|N|2464846|2464997|2464494|2464769|N|N|N|N|N| +2464861|AAAAAAAANFMJFCAA|2036-06-16|1637|7120|547|2036|1|6|16|2|2036|547|7120|Monday|2036Q2|N|N|N|2464846|2464997|2464495|2464770|N|N|N|N|N| +2464862|AAAAAAAAOFMJFCAA|2036-06-17|1637|7121|547|2036|2|6|17|2|2036|547|7121|Tuesday|2036Q2|N|N|N|2464846|2464997|2464496|2464771|N|N|N|N|N| +2464863|AAAAAAAAPFMJFCAA|2036-06-18|1637|7121|547|2036|3|6|18|2|2036|547|7121|Wednesday|2036Q2|N|N|N|2464846|2464997|2464497|2464772|N|N|N|N|N| +2464864|AAAAAAAAAGMJFCAA|2036-06-19|1637|7121|547|2036|4|6|19|2|2036|547|7121|Thursday|2036Q2|N|N|N|2464846|2464997|2464498|2464773|N|N|N|N|N| +2464865|AAAAAAAABGMJFCAA|2036-06-20|1637|7121|547|2036|5|6|20|2|2036|547|7121|Friday|2036Q2|N|Y|N|2464846|2464997|2464499|2464774|N|N|N|N|N| +2464866|AAAAAAAACGMJFCAA|2036-06-21|1637|7121|547|2036|6|6|21|2|2036|547|7121|Saturday|2036Q2|N|Y|N|2464846|2464997|2464500|2464775|N|N|N|N|N| +2464867|AAAAAAAADGMJFCAA|2036-06-22|1637|7121|547|2036|0|6|22|2|2036|547|7121|Sunday|2036Q2|N|N|N|2464846|2464997|2464501|2464776|N|N|N|N|N| +2464868|AAAAAAAAEGMJFCAA|2036-06-23|1637|7121|547|2036|1|6|23|2|2036|547|7121|Monday|2036Q2|N|N|N|2464846|2464997|2464502|2464777|N|N|N|N|N| +2464869|AAAAAAAAFGMJFCAA|2036-06-24|1637|7122|547|2036|2|6|24|2|2036|547|7122|Tuesday|2036Q2|N|N|N|2464846|2464997|2464503|2464778|N|N|N|N|N| +2464870|AAAAAAAAGGMJFCAA|2036-06-25|1637|7122|547|2036|3|6|25|2|2036|547|7122|Wednesday|2036Q2|N|N|N|2464846|2464997|2464504|2464779|N|N|N|N|N| +2464871|AAAAAAAAHGMJFCAA|2036-06-26|1637|7122|547|2036|4|6|26|2|2036|547|7122|Thursday|2036Q2|N|N|N|2464846|2464997|2464505|2464780|N|N|N|N|N| +2464872|AAAAAAAAIGMJFCAA|2036-06-27|1637|7122|547|2036|5|6|27|2|2036|547|7122|Friday|2036Q2|N|Y|N|2464846|2464997|2464506|2464781|N|N|N|N|N| +2464873|AAAAAAAAJGMJFCAA|2036-06-28|1637|7122|547|2036|6|6|28|2|2036|547|7122|Saturday|2036Q2|N|Y|N|2464846|2464997|2464507|2464782|N|N|N|N|N| +2464874|AAAAAAAAKGMJFCAA|2036-06-29|1637|7122|547|2036|0|6|29|2|2036|547|7122|Sunday|2036Q2|N|N|N|2464846|2464997|2464508|2464783|N|N|N|N|N| +2464875|AAAAAAAALGMJFCAA|2036-06-30|1637|7122|547|2036|1|6|30|2|2036|547|7122|Monday|2036Q2|N|N|N|2464846|2464997|2464509|2464784|N|N|N|N|N| +2464876|AAAAAAAAMGMJFCAA|2036-07-01|1638|7123|547|2036|2|7|1|3|2036|547|7123|Tuesday|2036Q3|N|N|N|2464876|2465057|2464510|2464785|N|N|N|N|N| +2464877|AAAAAAAANGMJFCAA|2036-07-02|1638|7123|547|2036|3|7|2|3|2036|547|7123|Wednesday|2036Q3|N|N|N|2464876|2465057|2464511|2464786|N|N|N|N|N| +2464878|AAAAAAAAOGMJFCAA|2036-07-03|1638|7123|547|2036|4|7|3|3|2036|547|7123|Thursday|2036Q3|N|N|N|2464876|2465057|2464512|2464787|N|N|N|N|N| +2464879|AAAAAAAAPGMJFCAA|2036-07-04|1638|7123|547|2036|5|7|4|3|2036|547|7123|Friday|2036Q3|Y|Y|N|2464876|2465057|2464513|2464788|N|N|N|N|N| +2464880|AAAAAAAAAHMJFCAA|2036-07-05|1638|7123|547|2036|6|7|5|3|2036|547|7123|Saturday|2036Q3|N|Y|Y|2464876|2465057|2464514|2464789|N|N|N|N|N| +2464881|AAAAAAAABHMJFCAA|2036-07-06|1638|7123|547|2036|0|7|6|3|2036|547|7123|Sunday|2036Q3|N|N|N|2464876|2465057|2464515|2464790|N|N|N|N|N| +2464882|AAAAAAAACHMJFCAA|2036-07-07|1638|7123|547|2036|1|7|7|3|2036|547|7123|Monday|2036Q3|N|N|N|2464876|2465057|2464516|2464791|N|N|N|N|N| +2464883|AAAAAAAADHMJFCAA|2036-07-08|1638|7124|547|2036|2|7|8|3|2036|547|7124|Tuesday|2036Q3|N|N|N|2464876|2465057|2464517|2464792|N|N|N|N|N| +2464884|AAAAAAAAEHMJFCAA|2036-07-09|1638|7124|547|2036|3|7|9|3|2036|547|7124|Wednesday|2036Q3|N|N|N|2464876|2465057|2464518|2464793|N|N|N|N|N| +2464885|AAAAAAAAFHMJFCAA|2036-07-10|1638|7124|547|2036|4|7|10|3|2036|547|7124|Thursday|2036Q3|N|N|N|2464876|2465057|2464519|2464794|N|N|N|N|N| +2464886|AAAAAAAAGHMJFCAA|2036-07-11|1638|7124|547|2036|5|7|11|3|2036|547|7124|Friday|2036Q3|N|Y|N|2464876|2465057|2464520|2464795|N|N|N|N|N| +2464887|AAAAAAAAHHMJFCAA|2036-07-12|1638|7124|547|2036|6|7|12|3|2036|547|7124|Saturday|2036Q3|N|Y|N|2464876|2465057|2464521|2464796|N|N|N|N|N| +2464888|AAAAAAAAIHMJFCAA|2036-07-13|1638|7124|547|2036|0|7|13|3|2036|547|7124|Sunday|2036Q3|N|N|N|2464876|2465057|2464522|2464797|N|N|N|N|N| +2464889|AAAAAAAAJHMJFCAA|2036-07-14|1638|7124|547|2036|1|7|14|3|2036|547|7124|Monday|2036Q3|N|N|N|2464876|2465057|2464523|2464798|N|N|N|N|N| +2464890|AAAAAAAAKHMJFCAA|2036-07-15|1638|7125|547|2036|2|7|15|3|2036|547|7125|Tuesday|2036Q3|N|N|N|2464876|2465057|2464524|2464799|N|N|N|N|N| +2464891|AAAAAAAALHMJFCAA|2036-07-16|1638|7125|547|2036|3|7|16|3|2036|547|7125|Wednesday|2036Q3|N|N|N|2464876|2465057|2464525|2464800|N|N|N|N|N| +2464892|AAAAAAAAMHMJFCAA|2036-07-17|1638|7125|547|2036|4|7|17|3|2036|547|7125|Thursday|2036Q3|N|N|N|2464876|2465057|2464526|2464801|N|N|N|N|N| +2464893|AAAAAAAANHMJFCAA|2036-07-18|1638|7125|547|2036|5|7|18|3|2036|547|7125|Friday|2036Q3|N|Y|N|2464876|2465057|2464527|2464802|N|N|N|N|N| +2464894|AAAAAAAAOHMJFCAA|2036-07-19|1638|7125|547|2036|6|7|19|3|2036|547|7125|Saturday|2036Q3|N|Y|N|2464876|2465057|2464528|2464803|N|N|N|N|N| +2464895|AAAAAAAAPHMJFCAA|2036-07-20|1638|7125|547|2036|0|7|20|3|2036|547|7125|Sunday|2036Q3|N|N|N|2464876|2465057|2464529|2464804|N|N|N|N|N| +2464896|AAAAAAAAAIMJFCAA|2036-07-21|1638|7125|547|2036|1|7|21|3|2036|547|7125|Monday|2036Q3|N|N|N|2464876|2465057|2464530|2464805|N|N|N|N|N| +2464897|AAAAAAAABIMJFCAA|2036-07-22|1638|7126|547|2036|2|7|22|3|2036|547|7126|Tuesday|2036Q3|N|N|N|2464876|2465057|2464531|2464806|N|N|N|N|N| +2464898|AAAAAAAACIMJFCAA|2036-07-23|1638|7126|547|2036|3|7|23|3|2036|547|7126|Wednesday|2036Q3|N|N|N|2464876|2465057|2464532|2464807|N|N|N|N|N| +2464899|AAAAAAAADIMJFCAA|2036-07-24|1638|7126|547|2036|4|7|24|3|2036|547|7126|Thursday|2036Q3|N|N|N|2464876|2465057|2464533|2464808|N|N|N|N|N| +2464900|AAAAAAAAEIMJFCAA|2036-07-25|1638|7126|547|2036|5|7|25|3|2036|547|7126|Friday|2036Q3|N|Y|N|2464876|2465057|2464534|2464809|N|N|N|N|N| +2464901|AAAAAAAAFIMJFCAA|2036-07-26|1638|7126|547|2036|6|7|26|3|2036|547|7126|Saturday|2036Q3|N|Y|N|2464876|2465057|2464535|2464810|N|N|N|N|N| +2464902|AAAAAAAAGIMJFCAA|2036-07-27|1638|7126|547|2036|0|7|27|3|2036|547|7126|Sunday|2036Q3|N|N|N|2464876|2465057|2464536|2464811|N|N|N|N|N| +2464903|AAAAAAAAHIMJFCAA|2036-07-28|1638|7126|547|2036|1|7|28|3|2036|547|7126|Monday|2036Q3|N|N|N|2464876|2465057|2464537|2464812|N|N|N|N|N| +2464904|AAAAAAAAIIMJFCAA|2036-07-29|1638|7127|547|2036|2|7|29|3|2036|547|7127|Tuesday|2036Q3|N|N|N|2464876|2465057|2464538|2464813|N|N|N|N|N| +2464905|AAAAAAAAJIMJFCAA|2036-07-30|1638|7127|547|2036|3|7|30|3|2036|547|7127|Wednesday|2036Q3|N|N|N|2464876|2465057|2464539|2464814|N|N|N|N|N| +2464906|AAAAAAAAKIMJFCAA|2036-07-31|1638|7127|547|2036|4|7|31|3|2036|547|7127|Thursday|2036Q3|N|N|N|2464876|2465057|2464540|2464815|N|N|N|N|N| +2464907|AAAAAAAALIMJFCAA|2036-08-01|1639|7127|547|2036|5|8|1|3|2036|547|7127|Friday|2036Q3|N|Y|N|2464907|2465119|2464541|2464816|N|N|N|N|N| +2464908|AAAAAAAAMIMJFCAA|2036-08-02|1639|7127|547|2036|6|8|2|3|2036|547|7127|Saturday|2036Q3|N|Y|N|2464907|2465119|2464542|2464817|N|N|N|N|N| +2464909|AAAAAAAANIMJFCAA|2036-08-03|1639|7127|547|2036|0|8|3|3|2036|547|7127|Sunday|2036Q3|N|N|N|2464907|2465119|2464543|2464818|N|N|N|N|N| +2464910|AAAAAAAAOIMJFCAA|2036-08-04|1639|7127|547|2036|1|8|4|3|2036|547|7127|Monday|2036Q3|N|N|N|2464907|2465119|2464544|2464819|N|N|N|N|N| +2464911|AAAAAAAAPIMJFCAA|2036-08-05|1639|7128|547|2036|2|8|5|3|2036|547|7128|Tuesday|2036Q3|N|N|N|2464907|2465119|2464545|2464820|N|N|N|N|N| +2464912|AAAAAAAAAJMJFCAA|2036-08-06|1639|7128|547|2036|3|8|6|3|2036|547|7128|Wednesday|2036Q3|N|N|N|2464907|2465119|2464546|2464821|N|N|N|N|N| +2464913|AAAAAAAABJMJFCAA|2036-08-07|1639|7128|547|2036|4|8|7|3|2036|547|7128|Thursday|2036Q3|N|N|N|2464907|2465119|2464547|2464822|N|N|N|N|N| +2464914|AAAAAAAACJMJFCAA|2036-08-08|1639|7128|547|2036|5|8|8|3|2036|547|7128|Friday|2036Q3|N|Y|N|2464907|2465119|2464548|2464823|N|N|N|N|N| +2464915|AAAAAAAADJMJFCAA|2036-08-09|1639|7128|547|2036|6|8|9|3|2036|547|7128|Saturday|2036Q3|N|Y|N|2464907|2465119|2464549|2464824|N|N|N|N|N| +2464916|AAAAAAAAEJMJFCAA|2036-08-10|1639|7128|547|2036|0|8|10|3|2036|547|7128|Sunday|2036Q3|N|N|N|2464907|2465119|2464550|2464825|N|N|N|N|N| +2464917|AAAAAAAAFJMJFCAA|2036-08-11|1639|7128|547|2036|1|8|11|3|2036|547|7128|Monday|2036Q3|N|N|N|2464907|2465119|2464551|2464826|N|N|N|N|N| +2464918|AAAAAAAAGJMJFCAA|2036-08-12|1639|7129|547|2036|2|8|12|3|2036|547|7129|Tuesday|2036Q3|N|N|N|2464907|2465119|2464552|2464827|N|N|N|N|N| +2464919|AAAAAAAAHJMJFCAA|2036-08-13|1639|7129|547|2036|3|8|13|3|2036|547|7129|Wednesday|2036Q3|N|N|N|2464907|2465119|2464553|2464828|N|N|N|N|N| +2464920|AAAAAAAAIJMJFCAA|2036-08-14|1639|7129|547|2036|4|8|14|3|2036|547|7129|Thursday|2036Q3|N|N|N|2464907|2465119|2464554|2464829|N|N|N|N|N| +2464921|AAAAAAAAJJMJFCAA|2036-08-15|1639|7129|547|2036|5|8|15|3|2036|547|7129|Friday|2036Q3|N|Y|N|2464907|2465119|2464555|2464830|N|N|N|N|N| +2464922|AAAAAAAAKJMJFCAA|2036-08-16|1639|7129|547|2036|6|8|16|3|2036|547|7129|Saturday|2036Q3|N|Y|N|2464907|2465119|2464556|2464831|N|N|N|N|N| +2464923|AAAAAAAALJMJFCAA|2036-08-17|1639|7129|547|2036|0|8|17|3|2036|547|7129|Sunday|2036Q3|N|N|N|2464907|2465119|2464557|2464832|N|N|N|N|N| +2464924|AAAAAAAAMJMJFCAA|2036-08-18|1639|7129|547|2036|1|8|18|3|2036|547|7129|Monday|2036Q3|N|N|N|2464907|2465119|2464558|2464833|N|N|N|N|N| +2464925|AAAAAAAANJMJFCAA|2036-08-19|1639|7130|547|2036|2|8|19|3|2036|547|7130|Tuesday|2036Q3|N|N|N|2464907|2465119|2464559|2464834|N|N|N|N|N| +2464926|AAAAAAAAOJMJFCAA|2036-08-20|1639|7130|547|2036|3|8|20|3|2036|547|7130|Wednesday|2036Q3|N|N|N|2464907|2465119|2464560|2464835|N|N|N|N|N| +2464927|AAAAAAAAPJMJFCAA|2036-08-21|1639|7130|547|2036|4|8|21|3|2036|547|7130|Thursday|2036Q3|N|N|N|2464907|2465119|2464561|2464836|N|N|N|N|N| +2464928|AAAAAAAAAKMJFCAA|2036-08-22|1639|7130|547|2036|5|8|22|3|2036|547|7130|Friday|2036Q3|N|Y|N|2464907|2465119|2464562|2464837|N|N|N|N|N| +2464929|AAAAAAAABKMJFCAA|2036-08-23|1639|7130|547|2036|6|8|23|3|2036|547|7130|Saturday|2036Q3|N|Y|N|2464907|2465119|2464563|2464838|N|N|N|N|N| +2464930|AAAAAAAACKMJFCAA|2036-08-24|1639|7130|547|2036|0|8|24|3|2036|547|7130|Sunday|2036Q3|N|N|N|2464907|2465119|2464564|2464839|N|N|N|N|N| +2464931|AAAAAAAADKMJFCAA|2036-08-25|1639|7130|547|2036|1|8|25|3|2036|547|7130|Monday|2036Q3|N|N|N|2464907|2465119|2464565|2464840|N|N|N|N|N| +2464932|AAAAAAAAEKMJFCAA|2036-08-26|1639|7131|547|2036|2|8|26|3|2036|547|7131|Tuesday|2036Q3|N|N|N|2464907|2465119|2464566|2464841|N|N|N|N|N| +2464933|AAAAAAAAFKMJFCAA|2036-08-27|1639|7131|547|2036|3|8|27|3|2036|547|7131|Wednesday|2036Q3|N|N|N|2464907|2465119|2464567|2464842|N|N|N|N|N| +2464934|AAAAAAAAGKMJFCAA|2036-08-28|1639|7131|547|2036|4|8|28|3|2036|547|7131|Thursday|2036Q3|N|N|N|2464907|2465119|2464568|2464843|N|N|N|N|N| +2464935|AAAAAAAAHKMJFCAA|2036-08-29|1639|7131|547|2036|5|8|29|3|2036|547|7131|Friday|2036Q3|N|Y|N|2464907|2465119|2464569|2464844|N|N|N|N|N| +2464936|AAAAAAAAIKMJFCAA|2036-08-30|1639|7131|547|2036|6|8|30|3|2036|547|7131|Saturday|2036Q3|N|Y|N|2464907|2465119|2464570|2464845|N|N|N|N|N| +2464937|AAAAAAAAJKMJFCAA|2036-08-31|1639|7131|547|2036|0|8|31|3|2036|547|7131|Sunday|2036Q3|N|N|N|2464907|2465119|2464571|2464846|N|N|N|N|N| +2464938|AAAAAAAAKKMJFCAA|2036-09-01|1640|7131|548|2036|1|9|1|3|2036|548|7131|Monday|2036Q3|N|N|N|2464938|2465181|2464572|2464847|N|N|N|N|N| +2464939|AAAAAAAALKMJFCAA|2036-09-02|1640|7132|548|2036|2|9|2|3|2036|548|7132|Tuesday|2036Q3|N|N|N|2464938|2465181|2464573|2464848|N|N|N|N|N| +2464940|AAAAAAAAMKMJFCAA|2036-09-03|1640|7132|548|2036|3|9|3|3|2036|548|7132|Wednesday|2036Q3|N|N|N|2464938|2465181|2464574|2464849|N|N|N|N|N| +2464941|AAAAAAAANKMJFCAA|2036-09-04|1640|7132|548|2036|4|9|4|3|2036|548|7132|Thursday|2036Q3|N|N|N|2464938|2465181|2464575|2464850|N|N|N|N|N| +2464942|AAAAAAAAOKMJFCAA|2036-09-05|1640|7132|548|2036|5|9|5|3|2036|548|7132|Friday|2036Q3|N|Y|N|2464938|2465181|2464576|2464851|N|N|N|N|N| +2464943|AAAAAAAAPKMJFCAA|2036-09-06|1640|7132|548|2036|6|9|6|3|2036|548|7132|Saturday|2036Q3|N|Y|N|2464938|2465181|2464577|2464852|N|N|N|N|N| +2464944|AAAAAAAAALMJFCAA|2036-09-07|1640|7132|548|2036|0|9|7|3|2036|548|7132|Sunday|2036Q3|N|N|N|2464938|2465181|2464578|2464853|N|N|N|N|N| +2464945|AAAAAAAABLMJFCAA|2036-09-08|1640|7132|548|2036|1|9|8|3|2036|548|7132|Monday|2036Q3|N|N|N|2464938|2465181|2464579|2464854|N|N|N|N|N| +2464946|AAAAAAAACLMJFCAA|2036-09-09|1640|7133|548|2036|2|9|9|3|2036|548|7133|Tuesday|2036Q3|N|N|N|2464938|2465181|2464580|2464855|N|N|N|N|N| +2464947|AAAAAAAADLMJFCAA|2036-09-10|1640|7133|548|2036|3|9|10|3|2036|548|7133|Wednesday|2036Q3|N|N|N|2464938|2465181|2464581|2464856|N|N|N|N|N| +2464948|AAAAAAAAELMJFCAA|2036-09-11|1640|7133|548|2036|4|9|11|3|2036|548|7133|Thursday|2036Q3|N|N|N|2464938|2465181|2464582|2464857|N|N|N|N|N| +2464949|AAAAAAAAFLMJFCAA|2036-09-12|1640|7133|548|2036|5|9|12|3|2036|548|7133|Friday|2036Q3|N|Y|N|2464938|2465181|2464583|2464858|N|N|N|N|N| +2464950|AAAAAAAAGLMJFCAA|2036-09-13|1640|7133|548|2036|6|9|13|3|2036|548|7133|Saturday|2036Q3|N|Y|N|2464938|2465181|2464584|2464859|N|N|N|N|N| +2464951|AAAAAAAAHLMJFCAA|2036-09-14|1640|7133|548|2036|0|9|14|3|2036|548|7133|Sunday|2036Q3|N|N|N|2464938|2465181|2464585|2464860|N|N|N|N|N| +2464952|AAAAAAAAILMJFCAA|2036-09-15|1640|7133|548|2036|1|9|15|3|2036|548|7133|Monday|2036Q3|N|N|N|2464938|2465181|2464586|2464861|N|N|N|N|N| +2464953|AAAAAAAAJLMJFCAA|2036-09-16|1640|7134|548|2036|2|9|16|3|2036|548|7134|Tuesday|2036Q3|N|N|N|2464938|2465181|2464587|2464862|N|N|N|N|N| +2464954|AAAAAAAAKLMJFCAA|2036-09-17|1640|7134|548|2036|3|9|17|3|2036|548|7134|Wednesday|2036Q3|N|N|N|2464938|2465181|2464588|2464863|N|N|N|N|N| +2464955|AAAAAAAALLMJFCAA|2036-09-18|1640|7134|548|2036|4|9|18|3|2036|548|7134|Thursday|2036Q3|N|N|N|2464938|2465181|2464589|2464864|N|N|N|N|N| +2464956|AAAAAAAAMLMJFCAA|2036-09-19|1640|7134|548|2036|5|9|19|3|2036|548|7134|Friday|2036Q3|N|Y|N|2464938|2465181|2464590|2464865|N|N|N|N|N| +2464957|AAAAAAAANLMJFCAA|2036-09-20|1640|7134|548|2036|6|9|20|3|2036|548|7134|Saturday|2036Q3|N|Y|N|2464938|2465181|2464591|2464866|N|N|N|N|N| +2464958|AAAAAAAAOLMJFCAA|2036-09-21|1640|7134|548|2036|0|9|21|3|2036|548|7134|Sunday|2036Q3|N|N|N|2464938|2465181|2464592|2464867|N|N|N|N|N| +2464959|AAAAAAAAPLMJFCAA|2036-09-22|1640|7134|548|2036|1|9|22|3|2036|548|7134|Monday|2036Q3|N|N|N|2464938|2465181|2464593|2464868|N|N|N|N|N| +2464960|AAAAAAAAAMMJFCAA|2036-09-23|1640|7135|548|2036|2|9|23|3|2036|548|7135|Tuesday|2036Q3|N|N|N|2464938|2465181|2464594|2464869|N|N|N|N|N| +2464961|AAAAAAAABMMJFCAA|2036-09-24|1640|7135|548|2036|3|9|24|3|2036|548|7135|Wednesday|2036Q3|N|N|N|2464938|2465181|2464595|2464870|N|N|N|N|N| +2464962|AAAAAAAACMMJFCAA|2036-09-25|1640|7135|548|2036|4|9|25|3|2036|548|7135|Thursday|2036Q3|N|N|N|2464938|2465181|2464596|2464871|N|N|N|N|N| +2464963|AAAAAAAADMMJFCAA|2036-09-26|1640|7135|548|2036|5|9|26|3|2036|548|7135|Friday|2036Q3|N|Y|N|2464938|2465181|2464597|2464872|N|N|N|N|N| +2464964|AAAAAAAAEMMJFCAA|2036-09-27|1640|7135|548|2036|6|9|27|3|2036|548|7135|Saturday|2036Q3|N|Y|N|2464938|2465181|2464598|2464873|N|N|N|N|N| +2464965|AAAAAAAAFMMJFCAA|2036-09-28|1640|7135|548|2036|0|9|28|3|2036|548|7135|Sunday|2036Q3|N|N|N|2464938|2465181|2464599|2464874|N|N|N|N|N| +2464966|AAAAAAAAGMMJFCAA|2036-09-29|1640|7135|548|2036|1|9|29|3|2036|548|7135|Monday|2036Q3|N|N|N|2464938|2465181|2464600|2464875|N|N|N|N|N| +2464967|AAAAAAAAHMMJFCAA|2036-09-30|1640|7136|548|2036|2|9|30|3|2036|548|7136|Tuesday|2036Q3|N|N|N|2464938|2465181|2464601|2464876|N|N|N|N|N| +2464968|AAAAAAAAIMMJFCAA|2036-10-01|1641|7136|548|2036|3|10|1|4|2036|548|7136|Wednesday|2036Q4|N|N|N|2464968|2465241|2464602|2464876|N|N|N|N|N| +2464969|AAAAAAAAJMMJFCAA|2036-10-02|1641|7136|548|2036|4|10|2|4|2036|548|7136|Thursday|2036Q4|N|N|N|2464968|2465241|2464603|2464877|N|N|N|N|N| +2464970|AAAAAAAAKMMJFCAA|2036-10-03|1641|7136|548|2036|5|10|3|4|2036|548|7136|Friday|2036Q4|N|Y|N|2464968|2465241|2464604|2464878|N|N|N|N|N| +2464971|AAAAAAAALMMJFCAA|2036-10-04|1641|7136|548|2036|6|10|4|4|2036|548|7136|Saturday|2036Q4|N|Y|N|2464968|2465241|2464605|2464879|N|N|N|N|N| +2464972|AAAAAAAAMMMJFCAA|2036-10-05|1641|7136|548|2036|0|10|5|4|2036|548|7136|Sunday|2036Q4|N|N|N|2464968|2465241|2464606|2464880|N|N|N|N|N| +2464973|AAAAAAAANMMJFCAA|2036-10-06|1641|7136|548|2036|1|10|6|4|2036|548|7136|Monday|2036Q4|N|N|N|2464968|2465241|2464607|2464881|N|N|N|N|N| +2464974|AAAAAAAAOMMJFCAA|2036-10-07|1641|7137|548|2036|2|10|7|4|2036|548|7137|Tuesday|2036Q4|N|N|N|2464968|2465241|2464608|2464882|N|N|N|N|N| +2464975|AAAAAAAAPMMJFCAA|2036-10-08|1641|7137|548|2036|3|10|8|4|2036|548|7137|Wednesday|2036Q4|N|N|N|2464968|2465241|2464609|2464883|N|N|N|N|N| +2464976|AAAAAAAAANMJFCAA|2036-10-09|1641|7137|548|2036|4|10|9|4|2036|548|7137|Thursday|2036Q4|N|N|N|2464968|2465241|2464610|2464884|N|N|N|N|N| +2464977|AAAAAAAABNMJFCAA|2036-10-10|1641|7137|548|2036|5|10|10|4|2036|548|7137|Friday|2036Q4|N|Y|N|2464968|2465241|2464611|2464885|N|N|N|N|N| +2464978|AAAAAAAACNMJFCAA|2036-10-11|1641|7137|548|2036|6|10|11|4|2036|548|7137|Saturday|2036Q4|N|Y|N|2464968|2465241|2464612|2464886|N|N|N|N|N| +2464979|AAAAAAAADNMJFCAA|2036-10-12|1641|7137|548|2036|0|10|12|4|2036|548|7137|Sunday|2036Q4|N|N|N|2464968|2465241|2464613|2464887|N|N|N|N|N| +2464980|AAAAAAAAENMJFCAA|2036-10-13|1641|7137|548|2036|1|10|13|4|2036|548|7137|Monday|2036Q4|N|N|N|2464968|2465241|2464614|2464888|N|N|N|N|N| +2464981|AAAAAAAAFNMJFCAA|2036-10-14|1641|7138|548|2036|2|10|14|4|2036|548|7138|Tuesday|2036Q4|N|N|N|2464968|2465241|2464615|2464889|N|N|N|N|N| +2464982|AAAAAAAAGNMJFCAA|2036-10-15|1641|7138|548|2036|3|10|15|4|2036|548|7138|Wednesday|2036Q4|N|N|N|2464968|2465241|2464616|2464890|N|N|N|N|N| +2464983|AAAAAAAAHNMJFCAA|2036-10-16|1641|7138|548|2036|4|10|16|4|2036|548|7138|Thursday|2036Q4|N|N|N|2464968|2465241|2464617|2464891|N|N|N|N|N| +2464984|AAAAAAAAINMJFCAA|2036-10-17|1641|7138|548|2036|5|10|17|4|2036|548|7138|Friday|2036Q4|N|Y|N|2464968|2465241|2464618|2464892|N|N|N|N|N| +2464985|AAAAAAAAJNMJFCAA|2036-10-18|1641|7138|548|2036|6|10|18|4|2036|548|7138|Saturday|2036Q4|N|Y|N|2464968|2465241|2464619|2464893|N|N|N|N|N| +2464986|AAAAAAAAKNMJFCAA|2036-10-19|1641|7138|548|2036|0|10|19|4|2036|548|7138|Sunday|2036Q4|N|N|N|2464968|2465241|2464620|2464894|N|N|N|N|N| +2464987|AAAAAAAALNMJFCAA|2036-10-20|1641|7138|548|2036|1|10|20|4|2036|548|7138|Monday|2036Q4|N|N|N|2464968|2465241|2464621|2464895|N|N|N|N|N| +2464988|AAAAAAAAMNMJFCAA|2036-10-21|1641|7139|548|2036|2|10|21|4|2036|548|7139|Tuesday|2036Q4|N|N|N|2464968|2465241|2464622|2464896|N|N|N|N|N| +2464989|AAAAAAAANNMJFCAA|2036-10-22|1641|7139|548|2036|3|10|22|4|2036|548|7139|Wednesday|2036Q4|N|N|N|2464968|2465241|2464623|2464897|N|N|N|N|N| +2464990|AAAAAAAAONMJFCAA|2036-10-23|1641|7139|548|2036|4|10|23|4|2036|548|7139|Thursday|2036Q4|N|N|N|2464968|2465241|2464624|2464898|N|N|N|N|N| +2464991|AAAAAAAAPNMJFCAA|2036-10-24|1641|7139|548|2036|5|10|24|4|2036|548|7139|Friday|2036Q4|N|Y|N|2464968|2465241|2464625|2464899|N|N|N|N|N| +2464992|AAAAAAAAAOMJFCAA|2036-10-25|1641|7139|548|2036|6|10|25|4|2036|548|7139|Saturday|2036Q4|N|Y|N|2464968|2465241|2464626|2464900|N|N|N|N|N| +2464993|AAAAAAAABOMJFCAA|2036-10-26|1641|7139|548|2036|0|10|26|4|2036|548|7139|Sunday|2036Q4|N|N|N|2464968|2465241|2464627|2464901|N|N|N|N|N| +2464994|AAAAAAAACOMJFCAA|2036-10-27|1641|7139|548|2036|1|10|27|4|2036|548|7139|Monday|2036Q4|N|N|N|2464968|2465241|2464628|2464902|N|N|N|N|N| +2464995|AAAAAAAADOMJFCAA|2036-10-28|1641|7140|548|2036|2|10|28|4|2036|548|7140|Tuesday|2036Q4|N|N|N|2464968|2465241|2464629|2464903|N|N|N|N|N| +2464996|AAAAAAAAEOMJFCAA|2036-10-29|1641|7140|548|2036|3|10|29|4|2036|548|7140|Wednesday|2036Q4|N|N|N|2464968|2465241|2464630|2464904|N|N|N|N|N| +2464997|AAAAAAAAFOMJFCAA|2036-10-30|1641|7140|548|2036|4|10|30|4|2036|548|7140|Thursday|2036Q4|N|N|N|2464968|2465241|2464631|2464905|N|N|N|N|N| +2464998|AAAAAAAAGOMJFCAA|2036-10-31|1641|7140|548|2036|5|10|31|4|2036|548|7140|Friday|2036Q4|N|Y|N|2464968|2465241|2464632|2464906|N|N|N|N|N| +2464999|AAAAAAAAHOMJFCAA|2036-11-01|1642|7140|548|2036|6|11|1|4|2036|548|7140|Saturday|2036Q4|N|Y|N|2464999|2465303|2464633|2464907|N|N|N|N|N| +2465000|AAAAAAAAIOMJFCAA|2036-11-02|1642|7140|548|2036|0|11|2|4|2036|548|7140|Sunday|2036Q4|N|N|N|2464999|2465303|2464634|2464908|N|N|N|N|N| +2465001|AAAAAAAAJOMJFCAA|2036-11-03|1642|7140|548|2036|1|11|3|4|2036|548|7140|Monday|2036Q4|N|N|N|2464999|2465303|2464635|2464909|N|N|N|N|N| +2465002|AAAAAAAAKOMJFCAA|2036-11-04|1642|7141|548|2036|2|11|4|4|2036|548|7141|Tuesday|2036Q4|N|N|N|2464999|2465303|2464636|2464910|N|N|N|N|N| +2465003|AAAAAAAALOMJFCAA|2036-11-05|1642|7141|548|2036|3|11|5|4|2036|548|7141|Wednesday|2036Q4|N|N|N|2464999|2465303|2464637|2464911|N|N|N|N|N| +2465004|AAAAAAAAMOMJFCAA|2036-11-06|1642|7141|548|2036|4|11|6|4|2036|548|7141|Thursday|2036Q4|N|N|N|2464999|2465303|2464638|2464912|N|N|N|N|N| +2465005|AAAAAAAANOMJFCAA|2036-11-07|1642|7141|548|2036|5|11|7|4|2036|548|7141|Friday|2036Q4|N|Y|N|2464999|2465303|2464639|2464913|N|N|N|N|N| +2465006|AAAAAAAAOOMJFCAA|2036-11-08|1642|7141|548|2036|6|11|8|4|2036|548|7141|Saturday|2036Q4|N|Y|N|2464999|2465303|2464640|2464914|N|N|N|N|N| +2465007|AAAAAAAAPOMJFCAA|2036-11-09|1642|7141|548|2036|0|11|9|4|2036|548|7141|Sunday|2036Q4|N|N|N|2464999|2465303|2464641|2464915|N|N|N|N|N| +2465008|AAAAAAAAAPMJFCAA|2036-11-10|1642|7141|548|2036|1|11|10|4|2036|548|7141|Monday|2036Q4|N|N|N|2464999|2465303|2464642|2464916|N|N|N|N|N| +2465009|AAAAAAAABPMJFCAA|2036-11-11|1642|7142|548|2036|2|11|11|4|2036|548|7142|Tuesday|2036Q4|N|N|N|2464999|2465303|2464643|2464917|N|N|N|N|N| +2465010|AAAAAAAACPMJFCAA|2036-11-12|1642|7142|548|2036|3|11|12|4|2036|548|7142|Wednesday|2036Q4|N|N|N|2464999|2465303|2464644|2464918|N|N|N|N|N| +2465011|AAAAAAAADPMJFCAA|2036-11-13|1642|7142|548|2036|4|11|13|4|2036|548|7142|Thursday|2036Q4|N|N|N|2464999|2465303|2464645|2464919|N|N|N|N|N| +2465012|AAAAAAAAEPMJFCAA|2036-11-14|1642|7142|548|2036|5|11|14|4|2036|548|7142|Friday|2036Q4|N|Y|N|2464999|2465303|2464646|2464920|N|N|N|N|N| +2465013|AAAAAAAAFPMJFCAA|2036-11-15|1642|7142|548|2036|6|11|15|4|2036|548|7142|Saturday|2036Q4|N|Y|N|2464999|2465303|2464647|2464921|N|N|N|N|N| +2465014|AAAAAAAAGPMJFCAA|2036-11-16|1642|7142|548|2036|0|11|16|4|2036|548|7142|Sunday|2036Q4|N|N|N|2464999|2465303|2464648|2464922|N|N|N|N|N| +2465015|AAAAAAAAHPMJFCAA|2036-11-17|1642|7142|548|2036|1|11|17|4|2036|548|7142|Monday|2036Q4|N|N|N|2464999|2465303|2464649|2464923|N|N|N|N|N| +2465016|AAAAAAAAIPMJFCAA|2036-11-18|1642|7143|548|2036|2|11|18|4|2036|548|7143|Tuesday|2036Q4|N|N|N|2464999|2465303|2464650|2464924|N|N|N|N|N| +2465017|AAAAAAAAJPMJFCAA|2036-11-19|1642|7143|548|2036|3|11|19|4|2036|548|7143|Wednesday|2036Q4|N|N|N|2464999|2465303|2464651|2464925|N|N|N|N|N| +2465018|AAAAAAAAKPMJFCAA|2036-11-20|1642|7143|548|2036|4|11|20|4|2036|548|7143|Thursday|2036Q4|N|N|N|2464999|2465303|2464652|2464926|N|N|N|N|N| +2465019|AAAAAAAALPMJFCAA|2036-11-21|1642|7143|548|2036|5|11|21|4|2036|548|7143|Friday|2036Q4|N|Y|N|2464999|2465303|2464653|2464927|N|N|N|N|N| +2465020|AAAAAAAAMPMJFCAA|2036-11-22|1642|7143|548|2036|6|11|22|4|2036|548|7143|Saturday|2036Q4|N|Y|N|2464999|2465303|2464654|2464928|N|N|N|N|N| +2465021|AAAAAAAANPMJFCAA|2036-11-23|1642|7143|548|2036|0|11|23|4|2036|548|7143|Sunday|2036Q4|N|N|N|2464999|2465303|2464655|2464929|N|N|N|N|N| +2465022|AAAAAAAAOPMJFCAA|2036-11-24|1642|7143|548|2036|1|11|24|4|2036|548|7143|Monday|2036Q4|N|N|N|2464999|2465303|2464656|2464930|N|N|N|N|N| +2465023|AAAAAAAAPPMJFCAA|2036-11-25|1642|7144|548|2036|2|11|25|4|2036|548|7144|Tuesday|2036Q4|N|N|N|2464999|2465303|2464657|2464931|N|N|N|N|N| +2465024|AAAAAAAAAANJFCAA|2036-11-26|1642|7144|548|2036|3|11|26|4|2036|548|7144|Wednesday|2036Q4|N|N|N|2464999|2465303|2464658|2464932|N|N|N|N|N| +2465025|AAAAAAAABANJFCAA|2036-11-27|1642|7144|548|2036|4|11|27|4|2036|548|7144|Thursday|2036Q4|N|N|N|2464999|2465303|2464659|2464933|N|N|N|N|N| +2465026|AAAAAAAACANJFCAA|2036-11-28|1642|7144|548|2036|5|11|28|4|2036|548|7144|Friday|2036Q4|N|Y|N|2464999|2465303|2464660|2464934|N|N|N|N|N| +2465027|AAAAAAAADANJFCAA|2036-11-29|1642|7144|548|2036|6|11|29|4|2036|548|7144|Saturday|2036Q4|N|Y|N|2464999|2465303|2464661|2464935|N|N|N|N|N| +2465028|AAAAAAAAEANJFCAA|2036-11-30|1642|7144|548|2036|0|11|30|4|2036|548|7144|Sunday|2036Q4|N|N|N|2464999|2465303|2464662|2464936|N|N|N|N|N| +2465029|AAAAAAAAFANJFCAA|2036-12-01|1643|7144|549|2036|1|12|1|4|2036|549|7144|Monday|2036Q4|N|N|N|2465029|2465363|2464663|2464937|N|N|N|N|N| +2465030|AAAAAAAAGANJFCAA|2036-12-02|1643|7145|549|2036|2|12|2|4|2036|549|7145|Tuesday|2036Q4|N|N|N|2465029|2465363|2464664|2464938|N|N|N|N|N| +2465031|AAAAAAAAHANJFCAA|2036-12-03|1643|7145|549|2036|3|12|3|4|2036|549|7145|Wednesday|2036Q4|N|N|N|2465029|2465363|2464665|2464939|N|N|N|N|N| +2465032|AAAAAAAAIANJFCAA|2036-12-04|1643|7145|549|2036|4|12|4|4|2036|549|7145|Thursday|2036Q4|N|N|N|2465029|2465363|2464666|2464940|N|N|N|N|N| +2465033|AAAAAAAAJANJFCAA|2036-12-05|1643|7145|549|2036|5|12|5|4|2036|549|7145|Friday|2036Q4|N|Y|N|2465029|2465363|2464667|2464941|N|N|N|N|N| +2465034|AAAAAAAAKANJFCAA|2036-12-06|1643|7145|549|2036|6|12|6|4|2036|549|7145|Saturday|2036Q4|N|Y|N|2465029|2465363|2464668|2464942|N|N|N|N|N| +2465035|AAAAAAAALANJFCAA|2036-12-07|1643|7145|549|2036|0|12|7|4|2036|549|7145|Sunday|2036Q4|N|N|N|2465029|2465363|2464669|2464943|N|N|N|N|N| +2465036|AAAAAAAAMANJFCAA|2036-12-08|1643|7145|549|2036|1|12|8|4|2036|549|7145|Monday|2036Q4|N|N|N|2465029|2465363|2464670|2464944|N|N|N|N|N| +2465037|AAAAAAAANANJFCAA|2036-12-09|1643|7146|549|2036|2|12|9|4|2036|549|7146|Tuesday|2036Q4|N|N|N|2465029|2465363|2464671|2464945|N|N|N|N|N| +2465038|AAAAAAAAOANJFCAA|2036-12-10|1643|7146|549|2036|3|12|10|4|2036|549|7146|Wednesday|2036Q4|N|N|N|2465029|2465363|2464672|2464946|N|N|N|N|N| +2465039|AAAAAAAAPANJFCAA|2036-12-11|1643|7146|549|2036|4|12|11|4|2036|549|7146|Thursday|2036Q4|N|N|N|2465029|2465363|2464673|2464947|N|N|N|N|N| +2465040|AAAAAAAAABNJFCAA|2036-12-12|1643|7146|549|2036|5|12|12|4|2036|549|7146|Friday|2036Q4|N|Y|N|2465029|2465363|2464674|2464948|N|N|N|N|N| +2465041|AAAAAAAABBNJFCAA|2036-12-13|1643|7146|549|2036|6|12|13|4|2036|549|7146|Saturday|2036Q4|N|Y|N|2465029|2465363|2464675|2464949|N|N|N|N|N| +2465042|AAAAAAAACBNJFCAA|2036-12-14|1643|7146|549|2036|0|12|14|4|2036|549|7146|Sunday|2036Q4|N|N|N|2465029|2465363|2464676|2464950|N|N|N|N|N| +2465043|AAAAAAAADBNJFCAA|2036-12-15|1643|7146|549|2036|1|12|15|4|2036|549|7146|Monday|2036Q4|N|N|N|2465029|2465363|2464677|2464951|N|N|N|N|N| +2465044|AAAAAAAAEBNJFCAA|2036-12-16|1643|7147|549|2036|2|12|16|4|2036|549|7147|Tuesday|2036Q4|N|N|N|2465029|2465363|2464678|2464952|N|N|N|N|N| +2465045|AAAAAAAAFBNJFCAA|2036-12-17|1643|7147|549|2036|3|12|17|4|2036|549|7147|Wednesday|2036Q4|N|N|N|2465029|2465363|2464679|2464953|N|N|N|N|N| +2465046|AAAAAAAAGBNJFCAA|2036-12-18|1643|7147|549|2036|4|12|18|4|2036|549|7147|Thursday|2036Q4|N|N|N|2465029|2465363|2464680|2464954|N|N|N|N|N| +2465047|AAAAAAAAHBNJFCAA|2036-12-19|1643|7147|549|2036|5|12|19|4|2036|549|7147|Friday|2036Q4|N|Y|N|2465029|2465363|2464681|2464955|N|N|N|N|N| +2465048|AAAAAAAAIBNJFCAA|2036-12-20|1643|7147|549|2036|6|12|20|4|2036|549|7147|Saturday|2036Q4|N|Y|N|2465029|2465363|2464682|2464956|N|N|N|N|N| +2465049|AAAAAAAAJBNJFCAA|2036-12-21|1643|7147|549|2036|0|12|21|4|2036|549|7147|Sunday|2036Q4|N|N|N|2465029|2465363|2464683|2464957|N|N|N|N|N| +2465050|AAAAAAAAKBNJFCAA|2036-12-22|1643|7147|549|2036|1|12|22|4|2036|549|7147|Monday|2036Q4|N|N|N|2465029|2465363|2464684|2464958|N|N|N|N|N| +2465051|AAAAAAAALBNJFCAA|2036-12-23|1643|7148|549|2036|2|12|23|4|2036|549|7148|Tuesday|2036Q4|N|N|N|2465029|2465363|2464685|2464959|N|N|N|N|N| +2465052|AAAAAAAAMBNJFCAA|2036-12-24|1643|7148|549|2036|3|12|24|4|2036|549|7148|Wednesday|2036Q4|N|N|N|2465029|2465363|2464686|2464960|N|N|N|N|N| +2465053|AAAAAAAANBNJFCAA|2036-12-25|1643|7148|549|2036|4|12|25|4|2036|549|7148|Thursday|2036Q4|Y|N|N|2465029|2465363|2464687|2464961|N|N|N|N|N| +2465054|AAAAAAAAOBNJFCAA|2036-12-26|1643|7148|549|2036|5|12|26|4|2036|549|7148|Friday|2036Q4|N|Y|Y|2465029|2465363|2464688|2464962|N|N|N|N|N| +2465055|AAAAAAAAPBNJFCAA|2036-12-27|1643|7148|549|2036|6|12|27|4|2036|549|7148|Saturday|2036Q4|N|Y|N|2465029|2465363|2464689|2464963|N|N|N|N|N| +2465056|AAAAAAAAACNJFCAA|2036-12-28|1643|7148|549|2036|0|12|28|4|2036|549|7148|Sunday|2036Q4|N|N|N|2465029|2465363|2464690|2464964|N|N|N|N|N| +2465057|AAAAAAAABCNJFCAA|2036-12-29|1643|7148|549|2036|1|12|29|4|2036|549|7148|Monday|2036Q4|N|N|N|2465029|2465363|2464691|2464965|N|N|N|N|N| +2465058|AAAAAAAACCNJFCAA|2036-12-30|1643|7149|549|2036|2|12|30|4|2036|549|7149|Tuesday|2036Q4|N|N|N|2465029|2465363|2464692|2464966|N|N|N|N|N| +2465059|AAAAAAAADCNJFCAA|2036-12-31|1643|7149|549|2036|3|12|31|4|2036|549|7149|Wednesday|2036Q4|Y|N|N|2465029|2465363|2464693|2464967|N|N|N|N|N| +2465060|AAAAAAAAECNJFCAA|2037-01-01|1644|7149|549|2037|4|1|1|1|2037|549|7149|Thursday|2037Q1|Y|N|Y|2465060|2465059|2464694|2464968|N|N|N|N|N| +2465061|AAAAAAAAFCNJFCAA|2037-01-02|1644|7149|549|2037|5|1|2|1|2037|549|7149|Friday|2037Q1|N|Y|Y|2465060|2465059|2464695|2464969|N|N|N|N|N| +2465062|AAAAAAAAGCNJFCAA|2037-01-03|1644|7149|549|2037|6|1|3|1|2037|549|7149|Saturday|2037Q1|N|Y|N|2465060|2465059|2464696|2464970|N|N|N|N|N| +2465063|AAAAAAAAHCNJFCAA|2037-01-04|1644|7149|549|2037|0|1|4|1|2037|549|7149|Sunday|2037Q1|N|N|N|2465060|2465059|2464697|2464971|N|N|N|N|N| +2465064|AAAAAAAAICNJFCAA|2037-01-05|1644|7149|549|2037|1|1|5|1|2037|549|7149|Monday|2037Q1|N|N|N|2465060|2465059|2464698|2464972|N|N|N|N|N| +2465065|AAAAAAAAJCNJFCAA|2037-01-06|1644|7150|549|2037|2|1|6|1|2037|549|7150|Tuesday|2037Q1|N|N|N|2465060|2465059|2464699|2464973|N|N|N|N|N| +2465066|AAAAAAAAKCNJFCAA|2037-01-07|1644|7150|549|2037|3|1|7|1|2037|549|7150|Wednesday|2037Q1|N|N|N|2465060|2465059|2464700|2464974|N|N|N|N|N| +2465067|AAAAAAAALCNJFCAA|2037-01-08|1644|7150|549|2037|4|1|8|1|2037|549|7150|Thursday|2037Q1|N|N|N|2465060|2465059|2464701|2464975|N|N|N|N|N| +2465068|AAAAAAAAMCNJFCAA|2037-01-09|1644|7150|549|2037|5|1|9|1|2037|549|7150|Friday|2037Q1|N|Y|N|2465060|2465059|2464702|2464976|N|N|N|N|N| +2465069|AAAAAAAANCNJFCAA|2037-01-10|1644|7150|549|2037|6|1|10|1|2037|549|7150|Saturday|2037Q1|N|Y|N|2465060|2465059|2464703|2464977|N|N|N|N|N| +2465070|AAAAAAAAOCNJFCAA|2037-01-11|1644|7150|549|2037|0|1|11|1|2037|549|7150|Sunday|2037Q1|N|N|N|2465060|2465059|2464704|2464978|N|N|N|N|N| +2465071|AAAAAAAAPCNJFCAA|2037-01-12|1644|7150|549|2037|1|1|12|1|2037|549|7150|Monday|2037Q1|N|N|N|2465060|2465059|2464705|2464979|N|N|N|N|N| +2465072|AAAAAAAAADNJFCAA|2037-01-13|1644|7151|549|2037|2|1|13|1|2037|549|7151|Tuesday|2037Q1|N|N|N|2465060|2465059|2464706|2464980|N|N|N|N|N| +2465073|AAAAAAAABDNJFCAA|2037-01-14|1644|7151|549|2037|3|1|14|1|2037|549|7151|Wednesday|2037Q1|N|N|N|2465060|2465059|2464707|2464981|N|N|N|N|N| +2465074|AAAAAAAACDNJFCAA|2037-01-15|1644|7151|549|2037|4|1|15|1|2037|549|7151|Thursday|2037Q1|N|N|N|2465060|2465059|2464708|2464982|N|N|N|N|N| +2465075|AAAAAAAADDNJFCAA|2037-01-16|1644|7151|549|2037|5|1|16|1|2037|549|7151|Friday|2037Q1|N|Y|N|2465060|2465059|2464709|2464983|N|N|N|N|N| +2465076|AAAAAAAAEDNJFCAA|2037-01-17|1644|7151|549|2037|6|1|17|1|2037|549|7151|Saturday|2037Q1|N|Y|N|2465060|2465059|2464710|2464984|N|N|N|N|N| +2465077|AAAAAAAAFDNJFCAA|2037-01-18|1644|7151|549|2037|0|1|18|1|2037|549|7151|Sunday|2037Q1|N|N|N|2465060|2465059|2464711|2464985|N|N|N|N|N| +2465078|AAAAAAAAGDNJFCAA|2037-01-19|1644|7151|549|2037|1|1|19|1|2037|549|7151|Monday|2037Q1|N|N|N|2465060|2465059|2464712|2464986|N|N|N|N|N| +2465079|AAAAAAAAHDNJFCAA|2037-01-20|1644|7152|549|2037|2|1|20|1|2037|549|7152|Tuesday|2037Q1|N|N|N|2465060|2465059|2464713|2464987|N|N|N|N|N| +2465080|AAAAAAAAIDNJFCAA|2037-01-21|1644|7152|549|2037|3|1|21|1|2037|549|7152|Wednesday|2037Q1|N|N|N|2465060|2465059|2464714|2464988|N|N|N|N|N| +2465081|AAAAAAAAJDNJFCAA|2037-01-22|1644|7152|549|2037|4|1|22|1|2037|549|7152|Thursday|2037Q1|N|N|N|2465060|2465059|2464715|2464989|N|N|N|N|N| +2465082|AAAAAAAAKDNJFCAA|2037-01-23|1644|7152|549|2037|5|1|23|1|2037|549|7152|Friday|2037Q1|N|Y|N|2465060|2465059|2464716|2464990|N|N|N|N|N| +2465083|AAAAAAAALDNJFCAA|2037-01-24|1644|7152|549|2037|6|1|24|1|2037|549|7152|Saturday|2037Q1|N|Y|N|2465060|2465059|2464717|2464991|N|N|N|N|N| +2465084|AAAAAAAAMDNJFCAA|2037-01-25|1644|7152|549|2037|0|1|25|1|2037|549|7152|Sunday|2037Q1|N|N|N|2465060|2465059|2464718|2464992|N|N|N|N|N| +2465085|AAAAAAAANDNJFCAA|2037-01-26|1644|7152|549|2037|1|1|26|1|2037|549|7152|Monday|2037Q1|N|N|N|2465060|2465059|2464719|2464993|N|N|N|N|N| +2465086|AAAAAAAAODNJFCAA|2037-01-27|1644|7153|549|2037|2|1|27|1|2037|549|7153|Tuesday|2037Q1|N|N|N|2465060|2465059|2464720|2464994|N|N|N|N|N| +2465087|AAAAAAAAPDNJFCAA|2037-01-28|1644|7153|549|2037|3|1|28|1|2037|549|7153|Wednesday|2037Q1|N|N|N|2465060|2465059|2464721|2464995|N|N|N|N|N| +2465088|AAAAAAAAAENJFCAA|2037-01-29|1644|7153|549|2037|4|1|29|1|2037|549|7153|Thursday|2037Q1|N|N|N|2465060|2465059|2464722|2464996|N|N|N|N|N| +2465089|AAAAAAAABENJFCAA|2037-01-30|1644|7153|549|2037|5|1|30|1|2037|549|7153|Friday|2037Q1|N|Y|N|2465060|2465059|2464723|2464997|N|N|N|N|N| +2465090|AAAAAAAACENJFCAA|2037-01-31|1644|7153|549|2037|6|1|31|1|2037|549|7153|Saturday|2037Q1|N|Y|N|2465060|2465059|2464724|2464998|N|N|N|N|N| +2465091|AAAAAAAADENJFCAA|2037-02-01|1645|7153|549|2037|0|2|1|1|2037|549|7153|Sunday|2037Q1|N|N|N|2465091|2465121|2464725|2464999|N|N|N|N|N| +2465092|AAAAAAAAEENJFCAA|2037-02-02|1645|7153|549|2037|1|2|2|1|2037|549|7153|Monday|2037Q1|N|N|N|2465091|2465121|2464726|2465000|N|N|N|N|N| +2465093|AAAAAAAAFENJFCAA|2037-02-03|1645|7154|549|2037|2|2|3|1|2037|549|7154|Tuesday|2037Q1|N|N|N|2465091|2465121|2464727|2465001|N|N|N|N|N| +2465094|AAAAAAAAGENJFCAA|2037-02-04|1645|7154|549|2037|3|2|4|1|2037|549|7154|Wednesday|2037Q1|N|N|N|2465091|2465121|2464728|2465002|N|N|N|N|N| +2465095|AAAAAAAAHENJFCAA|2037-02-05|1645|7154|549|2037|4|2|5|1|2037|549|7154|Thursday|2037Q1|N|N|N|2465091|2465121|2464729|2465003|N|N|N|N|N| +2465096|AAAAAAAAIENJFCAA|2037-02-06|1645|7154|549|2037|5|2|6|1|2037|549|7154|Friday|2037Q1|N|Y|N|2465091|2465121|2464730|2465004|N|N|N|N|N| +2465097|AAAAAAAAJENJFCAA|2037-02-07|1645|7154|549|2037|6|2|7|1|2037|549|7154|Saturday|2037Q1|N|Y|N|2465091|2465121|2464731|2465005|N|N|N|N|N| +2465098|AAAAAAAAKENJFCAA|2037-02-08|1645|7154|549|2037|0|2|8|1|2037|549|7154|Sunday|2037Q1|N|N|N|2465091|2465121|2464732|2465006|N|N|N|N|N| +2465099|AAAAAAAALENJFCAA|2037-02-09|1645|7154|549|2037|1|2|9|1|2037|549|7154|Monday|2037Q1|N|N|N|2465091|2465121|2464733|2465007|N|N|N|N|N| +2465100|AAAAAAAAMENJFCAA|2037-02-10|1645|7155|549|2037|2|2|10|1|2037|549|7155|Tuesday|2037Q1|N|N|N|2465091|2465121|2464734|2465008|N|N|N|N|N| +2465101|AAAAAAAANENJFCAA|2037-02-11|1645|7155|549|2037|3|2|11|1|2037|549|7155|Wednesday|2037Q1|N|N|N|2465091|2465121|2464735|2465009|N|N|N|N|N| +2465102|AAAAAAAAOENJFCAA|2037-02-12|1645|7155|549|2037|4|2|12|1|2037|549|7155|Thursday|2037Q1|N|N|N|2465091|2465121|2464736|2465010|N|N|N|N|N| +2465103|AAAAAAAAPENJFCAA|2037-02-13|1645|7155|549|2037|5|2|13|1|2037|549|7155|Friday|2037Q1|N|Y|N|2465091|2465121|2464737|2465011|N|N|N|N|N| +2465104|AAAAAAAAAFNJFCAA|2037-02-14|1645|7155|549|2037|6|2|14|1|2037|549|7155|Saturday|2037Q1|N|Y|N|2465091|2465121|2464738|2465012|N|N|N|N|N| +2465105|AAAAAAAABFNJFCAA|2037-02-15|1645|7155|549|2037|0|2|15|1|2037|549|7155|Sunday|2037Q1|N|N|N|2465091|2465121|2464739|2465013|N|N|N|N|N| +2465106|AAAAAAAACFNJFCAA|2037-02-16|1645|7155|549|2037|1|2|16|1|2037|549|7155|Monday|2037Q1|N|N|N|2465091|2465121|2464740|2465014|N|N|N|N|N| +2465107|AAAAAAAADFNJFCAA|2037-02-17|1645|7156|549|2037|2|2|17|1|2037|549|7156|Tuesday|2037Q1|N|N|N|2465091|2465121|2464741|2465015|N|N|N|N|N| +2465108|AAAAAAAAEFNJFCAA|2037-02-18|1645|7156|549|2037|3|2|18|1|2037|549|7156|Wednesday|2037Q1|N|N|N|2465091|2465121|2464742|2465016|N|N|N|N|N| +2465109|AAAAAAAAFFNJFCAA|2037-02-19|1645|7156|549|2037|4|2|19|1|2037|549|7156|Thursday|2037Q1|N|N|N|2465091|2465121|2464743|2465017|N|N|N|N|N| +2465110|AAAAAAAAGFNJFCAA|2037-02-20|1645|7156|549|2037|5|2|20|1|2037|549|7156|Friday|2037Q1|N|Y|N|2465091|2465121|2464744|2465018|N|N|N|N|N| +2465111|AAAAAAAAHFNJFCAA|2037-02-21|1645|7156|549|2037|6|2|21|1|2037|549|7156|Saturday|2037Q1|N|Y|N|2465091|2465121|2464745|2465019|N|N|N|N|N| +2465112|AAAAAAAAIFNJFCAA|2037-02-22|1645|7156|549|2037|0|2|22|1|2037|549|7156|Sunday|2037Q1|N|N|N|2465091|2465121|2464746|2465020|N|N|N|N|N| +2465113|AAAAAAAAJFNJFCAA|2037-02-23|1645|7156|549|2037|1|2|23|1|2037|549|7156|Monday|2037Q1|N|N|N|2465091|2465121|2464747|2465021|N|N|N|N|N| +2465114|AAAAAAAAKFNJFCAA|2037-02-24|1645|7157|549|2037|2|2|24|1|2037|549|7157|Tuesday|2037Q1|N|N|N|2465091|2465121|2464748|2465022|N|N|N|N|N| +2465115|AAAAAAAALFNJFCAA|2037-02-25|1645|7157|549|2037|3|2|25|1|2037|549|7157|Wednesday|2037Q1|N|N|N|2465091|2465121|2464749|2465023|N|N|N|N|N| +2465116|AAAAAAAAMFNJFCAA|2037-02-26|1645|7157|549|2037|4|2|26|1|2037|549|7157|Thursday|2037Q1|N|N|N|2465091|2465121|2464750|2465024|N|N|N|N|N| +2465117|AAAAAAAANFNJFCAA|2037-02-27|1645|7157|549|2037|5|2|27|1|2037|549|7157|Friday|2037Q1|N|Y|N|2465091|2465121|2464751|2465025|N|N|N|N|N| +2465118|AAAAAAAAOFNJFCAA|2037-02-28|1645|7157|549|2037|6|2|28|1|2037|549|7157|Saturday|2037Q1|N|Y|N|2465091|2465121|2464752|2465026|N|N|N|N|N| +2465119|AAAAAAAAPFNJFCAA|2037-03-01|1646|7157|550|2037|0|3|1|1|2037|550|7157|Sunday|2037Q1|N|N|N|2465119|2465177|2464754|2465027|N|N|N|N|N| +2465120|AAAAAAAAAGNJFCAA|2037-03-02|1646|7157|550|2037|1|3|2|1|2037|550|7157|Monday|2037Q1|N|N|N|2465119|2465177|2464755|2465028|N|N|N|N|N| +2465121|AAAAAAAABGNJFCAA|2037-03-03|1646|7158|550|2037|2|3|3|1|2037|550|7158|Tuesday|2037Q1|N|N|N|2465119|2465177|2464756|2465029|N|N|N|N|N| +2465122|AAAAAAAACGNJFCAA|2037-03-04|1646|7158|550|2037|3|3|4|1|2037|550|7158|Wednesday|2037Q1|N|N|N|2465119|2465177|2464757|2465030|N|N|N|N|N| +2465123|AAAAAAAADGNJFCAA|2037-03-05|1646|7158|550|2037|4|3|5|1|2037|550|7158|Thursday|2037Q1|N|N|N|2465119|2465177|2464758|2465031|N|N|N|N|N| +2465124|AAAAAAAAEGNJFCAA|2037-03-06|1646|7158|550|2037|5|3|6|1|2037|550|7158|Friday|2037Q1|N|Y|N|2465119|2465177|2464759|2465032|N|N|N|N|N| +2465125|AAAAAAAAFGNJFCAA|2037-03-07|1646|7158|550|2037|6|3|7|1|2037|550|7158|Saturday|2037Q1|N|Y|N|2465119|2465177|2464760|2465033|N|N|N|N|N| +2465126|AAAAAAAAGGNJFCAA|2037-03-08|1646|7158|550|2037|0|3|8|1|2037|550|7158|Sunday|2037Q1|N|N|N|2465119|2465177|2464761|2465034|N|N|N|N|N| +2465127|AAAAAAAAHGNJFCAA|2037-03-09|1646|7158|550|2037|1|3|9|1|2037|550|7158|Monday|2037Q1|N|N|N|2465119|2465177|2464762|2465035|N|N|N|N|N| +2465128|AAAAAAAAIGNJFCAA|2037-03-10|1646|7159|550|2037|2|3|10|1|2037|550|7159|Tuesday|2037Q1|N|N|N|2465119|2465177|2464763|2465036|N|N|N|N|N| +2465129|AAAAAAAAJGNJFCAA|2037-03-11|1646|7159|550|2037|3|3|11|1|2037|550|7159|Wednesday|2037Q1|N|N|N|2465119|2465177|2464764|2465037|N|N|N|N|N| +2465130|AAAAAAAAKGNJFCAA|2037-03-12|1646|7159|550|2037|4|3|12|1|2037|550|7159|Thursday|2037Q1|N|N|N|2465119|2465177|2464765|2465038|N|N|N|N|N| +2465131|AAAAAAAALGNJFCAA|2037-03-13|1646|7159|550|2037|5|3|13|1|2037|550|7159|Friday|2037Q1|N|Y|N|2465119|2465177|2464766|2465039|N|N|N|N|N| +2465132|AAAAAAAAMGNJFCAA|2037-03-14|1646|7159|550|2037|6|3|14|1|2037|550|7159|Saturday|2037Q1|N|Y|N|2465119|2465177|2464767|2465040|N|N|N|N|N| +2465133|AAAAAAAANGNJFCAA|2037-03-15|1646|7159|550|2037|0|3|15|1|2037|550|7159|Sunday|2037Q1|N|N|N|2465119|2465177|2464768|2465041|N|N|N|N|N| +2465134|AAAAAAAAOGNJFCAA|2037-03-16|1646|7159|550|2037|1|3|16|1|2037|550|7159|Monday|2037Q1|N|N|N|2465119|2465177|2464769|2465042|N|N|N|N|N| +2465135|AAAAAAAAPGNJFCAA|2037-03-17|1646|7160|550|2037|2|3|17|1|2037|550|7160|Tuesday|2037Q1|N|N|N|2465119|2465177|2464770|2465043|N|N|N|N|N| +2465136|AAAAAAAAAHNJFCAA|2037-03-18|1646|7160|550|2037|3|3|18|1|2037|550|7160|Wednesday|2037Q1|N|N|N|2465119|2465177|2464771|2465044|N|N|N|N|N| +2465137|AAAAAAAABHNJFCAA|2037-03-19|1646|7160|550|2037|4|3|19|1|2037|550|7160|Thursday|2037Q1|N|N|N|2465119|2465177|2464772|2465045|N|N|N|N|N| +2465138|AAAAAAAACHNJFCAA|2037-03-20|1646|7160|550|2037|5|3|20|1|2037|550|7160|Friday|2037Q1|N|Y|N|2465119|2465177|2464773|2465046|N|N|N|N|N| +2465139|AAAAAAAADHNJFCAA|2037-03-21|1646|7160|550|2037|6|3|21|1|2037|550|7160|Saturday|2037Q1|N|Y|N|2465119|2465177|2464774|2465047|N|N|N|N|N| +2465140|AAAAAAAAEHNJFCAA|2037-03-22|1646|7160|550|2037|0|3|22|1|2037|550|7160|Sunday|2037Q1|N|N|N|2465119|2465177|2464775|2465048|N|N|N|N|N| +2465141|AAAAAAAAFHNJFCAA|2037-03-23|1646|7160|550|2037|1|3|23|1|2037|550|7160|Monday|2037Q1|N|N|N|2465119|2465177|2464776|2465049|N|N|N|N|N| +2465142|AAAAAAAAGHNJFCAA|2037-03-24|1646|7161|550|2037|2|3|24|1|2037|550|7161|Tuesday|2037Q1|N|N|N|2465119|2465177|2464777|2465050|N|N|N|N|N| +2465143|AAAAAAAAHHNJFCAA|2037-03-25|1646|7161|550|2037|3|3|25|1|2037|550|7161|Wednesday|2037Q1|N|N|N|2465119|2465177|2464778|2465051|N|N|N|N|N| +2465144|AAAAAAAAIHNJFCAA|2037-03-26|1646|7161|550|2037|4|3|26|1|2037|550|7161|Thursday|2037Q1|N|N|N|2465119|2465177|2464779|2465052|N|N|N|N|N| +2465145|AAAAAAAAJHNJFCAA|2037-03-27|1646|7161|550|2037|5|3|27|1|2037|550|7161|Friday|2037Q1|N|Y|N|2465119|2465177|2464780|2465053|N|N|N|N|N| +2465146|AAAAAAAAKHNJFCAA|2037-03-28|1646|7161|550|2037|6|3|28|1|2037|550|7161|Saturday|2037Q1|N|Y|N|2465119|2465177|2464781|2465054|N|N|N|N|N| +2465147|AAAAAAAALHNJFCAA|2037-03-29|1646|7161|550|2037|0|3|29|1|2037|550|7161|Sunday|2037Q1|N|N|N|2465119|2465177|2464782|2465055|N|N|N|N|N| +2465148|AAAAAAAAMHNJFCAA|2037-03-30|1646|7161|550|2037|1|3|30|1|2037|550|7161|Monday|2037Q1|N|N|N|2465119|2465177|2464783|2465056|N|N|N|N|N| +2465149|AAAAAAAANHNJFCAA|2037-03-31|1646|7162|550|2037|2|3|31|1|2037|550|7162|Tuesday|2037Q1|N|N|N|2465119|2465177|2464784|2465057|N|N|N|N|N| +2465150|AAAAAAAAOHNJFCAA|2037-04-01|1647|7162|550|2037|3|4|1|1|2037|550|7162|Wednesday|2037Q1|N|N|N|2465150|2465239|2464785|2465060|N|N|N|N|N| +2465151|AAAAAAAAPHNJFCAA|2037-04-02|1647|7162|550|2037|4|4|2|2|2037|550|7162|Thursday|2037Q2|N|N|N|2465150|2465239|2464786|2465061|N|N|N|N|N| +2465152|AAAAAAAAAINJFCAA|2037-04-03|1647|7162|550|2037|5|4|3|2|2037|550|7162|Friday|2037Q2|N|Y|N|2465150|2465239|2464787|2465062|N|N|N|N|N| +2465153|AAAAAAAABINJFCAA|2037-04-04|1647|7162|550|2037|6|4|4|2|2037|550|7162|Saturday|2037Q2|N|Y|N|2465150|2465239|2464788|2465063|N|N|N|N|N| +2465154|AAAAAAAACINJFCAA|2037-04-05|1647|7162|550|2037|0|4|5|2|2037|550|7162|Sunday|2037Q2|N|N|N|2465150|2465239|2464789|2465064|N|N|N|N|N| +2465155|AAAAAAAADINJFCAA|2037-04-06|1647|7162|550|2037|1|4|6|2|2037|550|7162|Monday|2037Q2|N|N|N|2465150|2465239|2464790|2465065|N|N|N|N|N| +2465156|AAAAAAAAEINJFCAA|2037-04-07|1647|7163|550|2037|2|4|7|2|2037|550|7163|Tuesday|2037Q2|N|N|N|2465150|2465239|2464791|2465066|N|N|N|N|N| +2465157|AAAAAAAAFINJFCAA|2037-04-08|1647|7163|550|2037|3|4|8|2|2037|550|7163|Wednesday|2037Q2|N|N|N|2465150|2465239|2464792|2465067|N|N|N|N|N| +2465158|AAAAAAAAGINJFCAA|2037-04-09|1647|7163|550|2037|4|4|9|2|2037|550|7163|Thursday|2037Q2|N|N|N|2465150|2465239|2464793|2465068|N|N|N|N|N| +2465159|AAAAAAAAHINJFCAA|2037-04-10|1647|7163|550|2037|5|4|10|2|2037|550|7163|Friday|2037Q2|N|Y|N|2465150|2465239|2464794|2465069|N|N|N|N|N| +2465160|AAAAAAAAIINJFCAA|2037-04-11|1647|7163|550|2037|6|4|11|2|2037|550|7163|Saturday|2037Q2|N|Y|N|2465150|2465239|2464795|2465070|N|N|N|N|N| +2465161|AAAAAAAAJINJFCAA|2037-04-12|1647|7163|550|2037|0|4|12|2|2037|550|7163|Sunday|2037Q2|N|N|N|2465150|2465239|2464796|2465071|N|N|N|N|N| +2465162|AAAAAAAAKINJFCAA|2037-04-13|1647|7163|550|2037|1|4|13|2|2037|550|7163|Monday|2037Q2|N|N|N|2465150|2465239|2464797|2465072|N|N|N|N|N| +2465163|AAAAAAAALINJFCAA|2037-04-14|1647|7164|550|2037|2|4|14|2|2037|550|7164|Tuesday|2037Q2|N|N|N|2465150|2465239|2464798|2465073|N|N|N|N|N| +2465164|AAAAAAAAMINJFCAA|2037-04-15|1647|7164|550|2037|3|4|15|2|2037|550|7164|Wednesday|2037Q2|N|N|N|2465150|2465239|2464799|2465074|N|N|N|N|N| +2465165|AAAAAAAANINJFCAA|2037-04-16|1647|7164|550|2037|4|4|16|2|2037|550|7164|Thursday|2037Q2|N|N|N|2465150|2465239|2464800|2465075|N|N|N|N|N| +2465166|AAAAAAAAOINJFCAA|2037-04-17|1647|7164|550|2037|5|4|17|2|2037|550|7164|Friday|2037Q2|N|Y|N|2465150|2465239|2464801|2465076|N|N|N|N|N| +2465167|AAAAAAAAPINJFCAA|2037-04-18|1647|7164|550|2037|6|4|18|2|2037|550|7164|Saturday|2037Q2|N|Y|N|2465150|2465239|2464802|2465077|N|N|N|N|N| +2465168|AAAAAAAAAJNJFCAA|2037-04-19|1647|7164|550|2037|0|4|19|2|2037|550|7164|Sunday|2037Q2|N|N|N|2465150|2465239|2464803|2465078|N|N|N|N|N| +2465169|AAAAAAAABJNJFCAA|2037-04-20|1647|7164|550|2037|1|4|20|2|2037|550|7164|Monday|2037Q2|N|N|N|2465150|2465239|2464804|2465079|N|N|N|N|N| +2465170|AAAAAAAACJNJFCAA|2037-04-21|1647|7165|550|2037|2|4|21|2|2037|550|7165|Tuesday|2037Q2|N|N|N|2465150|2465239|2464805|2465080|N|N|N|N|N| +2465171|AAAAAAAADJNJFCAA|2037-04-22|1647|7165|550|2037|3|4|22|2|2037|550|7165|Wednesday|2037Q2|N|N|N|2465150|2465239|2464806|2465081|N|N|N|N|N| +2465172|AAAAAAAAEJNJFCAA|2037-04-23|1647|7165|550|2037|4|4|23|2|2037|550|7165|Thursday|2037Q2|N|N|N|2465150|2465239|2464807|2465082|N|N|N|N|N| +2465173|AAAAAAAAFJNJFCAA|2037-04-24|1647|7165|550|2037|5|4|24|2|2037|550|7165|Friday|2037Q2|N|Y|N|2465150|2465239|2464808|2465083|N|N|N|N|N| +2465174|AAAAAAAAGJNJFCAA|2037-04-25|1647|7165|550|2037|6|4|25|2|2037|550|7165|Saturday|2037Q2|N|Y|N|2465150|2465239|2464809|2465084|N|N|N|N|N| +2465175|AAAAAAAAHJNJFCAA|2037-04-26|1647|7165|550|2037|0|4|26|2|2037|550|7165|Sunday|2037Q2|N|N|N|2465150|2465239|2464810|2465085|N|N|N|N|N| +2465176|AAAAAAAAIJNJFCAA|2037-04-27|1647|7165|550|2037|1|4|27|2|2037|550|7165|Monday|2037Q2|N|N|N|2465150|2465239|2464811|2465086|N|N|N|N|N| +2465177|AAAAAAAAJJNJFCAA|2037-04-28|1647|7166|550|2037|2|4|28|2|2037|550|7166|Tuesday|2037Q2|N|N|N|2465150|2465239|2464812|2465087|N|N|N|N|N| +2465178|AAAAAAAAKJNJFCAA|2037-04-29|1647|7166|550|2037|3|4|29|2|2037|550|7166|Wednesday|2037Q2|N|N|N|2465150|2465239|2464813|2465088|N|N|N|N|N| +2465179|AAAAAAAALJNJFCAA|2037-04-30|1647|7166|550|2037|4|4|30|2|2037|550|7166|Thursday|2037Q2|N|N|N|2465150|2465239|2464814|2465089|N|N|N|N|N| +2465180|AAAAAAAAMJNJFCAA|2037-05-01|1648|7166|550|2037|5|5|1|2|2037|550|7166|Friday|2037Q2|N|Y|N|2465180|2465299|2464815|2465090|N|N|N|N|N| +2465181|AAAAAAAANJNJFCAA|2037-05-02|1648|7166|550|2037|6|5|2|2|2037|550|7166|Saturday|2037Q2|N|Y|N|2465180|2465299|2464816|2465091|N|N|N|N|N| +2465182|AAAAAAAAOJNJFCAA|2037-05-03|1648|7166|550|2037|0|5|3|2|2037|550|7166|Sunday|2037Q2|N|N|N|2465180|2465299|2464817|2465092|N|N|N|N|N| +2465183|AAAAAAAAPJNJFCAA|2037-05-04|1648|7166|550|2037|1|5|4|2|2037|550|7166|Monday|2037Q2|N|N|N|2465180|2465299|2464818|2465093|N|N|N|N|N| +2465184|AAAAAAAAAKNJFCAA|2037-05-05|1648|7167|550|2037|2|5|5|2|2037|550|7167|Tuesday|2037Q2|N|N|N|2465180|2465299|2464819|2465094|N|N|N|N|N| +2465185|AAAAAAAABKNJFCAA|2037-05-06|1648|7167|550|2037|3|5|6|2|2037|550|7167|Wednesday|2037Q2|N|N|N|2465180|2465299|2464820|2465095|N|N|N|N|N| +2465186|AAAAAAAACKNJFCAA|2037-05-07|1648|7167|550|2037|4|5|7|2|2037|550|7167|Thursday|2037Q2|N|N|N|2465180|2465299|2464821|2465096|N|N|N|N|N| +2465187|AAAAAAAADKNJFCAA|2037-05-08|1648|7167|550|2037|5|5|8|2|2037|550|7167|Friday|2037Q2|N|Y|N|2465180|2465299|2464822|2465097|N|N|N|N|N| +2465188|AAAAAAAAEKNJFCAA|2037-05-09|1648|7167|550|2037|6|5|9|2|2037|550|7167|Saturday|2037Q2|N|Y|N|2465180|2465299|2464823|2465098|N|N|N|N|N| +2465189|AAAAAAAAFKNJFCAA|2037-05-10|1648|7167|550|2037|0|5|10|2|2037|550|7167|Sunday|2037Q2|N|N|N|2465180|2465299|2464824|2465099|N|N|N|N|N| +2465190|AAAAAAAAGKNJFCAA|2037-05-11|1648|7167|550|2037|1|5|11|2|2037|550|7167|Monday|2037Q2|N|N|N|2465180|2465299|2464825|2465100|N|N|N|N|N| +2465191|AAAAAAAAHKNJFCAA|2037-05-12|1648|7168|550|2037|2|5|12|2|2037|550|7168|Tuesday|2037Q2|N|N|N|2465180|2465299|2464826|2465101|N|N|N|N|N| +2465192|AAAAAAAAIKNJFCAA|2037-05-13|1648|7168|550|2037|3|5|13|2|2037|550|7168|Wednesday|2037Q2|N|N|N|2465180|2465299|2464827|2465102|N|N|N|N|N| +2465193|AAAAAAAAJKNJFCAA|2037-05-14|1648|7168|550|2037|4|5|14|2|2037|550|7168|Thursday|2037Q2|N|N|N|2465180|2465299|2464828|2465103|N|N|N|N|N| +2465194|AAAAAAAAKKNJFCAA|2037-05-15|1648|7168|550|2037|5|5|15|2|2037|550|7168|Friday|2037Q2|N|Y|N|2465180|2465299|2464829|2465104|N|N|N|N|N| +2465195|AAAAAAAALKNJFCAA|2037-05-16|1648|7168|550|2037|6|5|16|2|2037|550|7168|Saturday|2037Q2|N|Y|N|2465180|2465299|2464830|2465105|N|N|N|N|N| +2465196|AAAAAAAAMKNJFCAA|2037-05-17|1648|7168|550|2037|0|5|17|2|2037|550|7168|Sunday|2037Q2|N|N|N|2465180|2465299|2464831|2465106|N|N|N|N|N| +2465197|AAAAAAAANKNJFCAA|2037-05-18|1648|7168|550|2037|1|5|18|2|2037|550|7168|Monday|2037Q2|N|N|N|2465180|2465299|2464832|2465107|N|N|N|N|N| +2465198|AAAAAAAAOKNJFCAA|2037-05-19|1648|7169|550|2037|2|5|19|2|2037|550|7169|Tuesday|2037Q2|N|N|N|2465180|2465299|2464833|2465108|N|N|N|N|N| +2465199|AAAAAAAAPKNJFCAA|2037-05-20|1648|7169|550|2037|3|5|20|2|2037|550|7169|Wednesday|2037Q2|N|N|N|2465180|2465299|2464834|2465109|N|N|N|N|N| +2465200|AAAAAAAAALNJFCAA|2037-05-21|1648|7169|550|2037|4|5|21|2|2037|550|7169|Thursday|2037Q2|N|N|N|2465180|2465299|2464835|2465110|N|N|N|N|N| +2465201|AAAAAAAABLNJFCAA|2037-05-22|1648|7169|550|2037|5|5|22|2|2037|550|7169|Friday|2037Q2|N|Y|N|2465180|2465299|2464836|2465111|N|N|N|N|N| +2465202|AAAAAAAACLNJFCAA|2037-05-23|1648|7169|550|2037|6|5|23|2|2037|550|7169|Saturday|2037Q2|N|Y|N|2465180|2465299|2464837|2465112|N|N|N|N|N| +2465203|AAAAAAAADLNJFCAA|2037-05-24|1648|7169|550|2037|0|5|24|2|2037|550|7169|Sunday|2037Q2|N|N|N|2465180|2465299|2464838|2465113|N|N|N|N|N| +2465204|AAAAAAAAELNJFCAA|2037-05-25|1648|7169|550|2037|1|5|25|2|2037|550|7169|Monday|2037Q2|N|N|N|2465180|2465299|2464839|2465114|N|N|N|N|N| +2465205|AAAAAAAAFLNJFCAA|2037-05-26|1648|7170|550|2037|2|5|26|2|2037|550|7170|Tuesday|2037Q2|N|N|N|2465180|2465299|2464840|2465115|N|N|N|N|N| +2465206|AAAAAAAAGLNJFCAA|2037-05-27|1648|7170|550|2037|3|5|27|2|2037|550|7170|Wednesday|2037Q2|N|N|N|2465180|2465299|2464841|2465116|N|N|N|N|N| +2465207|AAAAAAAAHLNJFCAA|2037-05-28|1648|7170|550|2037|4|5|28|2|2037|550|7170|Thursday|2037Q2|N|N|N|2465180|2465299|2464842|2465117|N|N|N|N|N| +2465208|AAAAAAAAILNJFCAA|2037-05-29|1648|7170|550|2037|5|5|29|2|2037|550|7170|Friday|2037Q2|N|Y|N|2465180|2465299|2464843|2465118|N|N|N|N|N| +2465209|AAAAAAAAJLNJFCAA|2037-05-30|1648|7170|550|2037|6|5|30|2|2037|550|7170|Saturday|2037Q2|N|Y|N|2465180|2465299|2464844|2465119|N|N|N|N|N| +2465210|AAAAAAAAKLNJFCAA|2037-05-31|1648|7170|550|2037|0|5|31|2|2037|550|7170|Sunday|2037Q2|N|N|N|2465180|2465299|2464845|2465120|N|N|N|N|N| +2465211|AAAAAAAALLNJFCAA|2037-06-01|1649|7170|551|2037|1|6|1|2|2037|551|7170|Monday|2037Q2|N|N|N|2465211|2465361|2464846|2465121|N|N|N|N|N| +2465212|AAAAAAAAMLNJFCAA|2037-06-02|1649|7171|551|2037|2|6|2|2|2037|551|7171|Tuesday|2037Q2|N|N|N|2465211|2465361|2464847|2465122|N|N|N|N|N| +2465213|AAAAAAAANLNJFCAA|2037-06-03|1649|7171|551|2037|3|6|3|2|2037|551|7171|Wednesday|2037Q2|N|N|N|2465211|2465361|2464848|2465123|N|N|N|N|N| +2465214|AAAAAAAAOLNJFCAA|2037-06-04|1649|7171|551|2037|4|6|4|2|2037|551|7171|Thursday|2037Q2|N|N|N|2465211|2465361|2464849|2465124|N|N|N|N|N| +2465215|AAAAAAAAPLNJFCAA|2037-06-05|1649|7171|551|2037|5|6|5|2|2037|551|7171|Friday|2037Q2|N|Y|N|2465211|2465361|2464850|2465125|N|N|N|N|N| +2465216|AAAAAAAAAMNJFCAA|2037-06-06|1649|7171|551|2037|6|6|6|2|2037|551|7171|Saturday|2037Q2|N|Y|N|2465211|2465361|2464851|2465126|N|N|N|N|N| +2465217|AAAAAAAABMNJFCAA|2037-06-07|1649|7171|551|2037|0|6|7|2|2037|551|7171|Sunday|2037Q2|N|N|N|2465211|2465361|2464852|2465127|N|N|N|N|N| +2465218|AAAAAAAACMNJFCAA|2037-06-08|1649|7171|551|2037|1|6|8|2|2037|551|7171|Monday|2037Q2|N|N|N|2465211|2465361|2464853|2465128|N|N|N|N|N| +2465219|AAAAAAAADMNJFCAA|2037-06-09|1649|7172|551|2037|2|6|9|2|2037|551|7172|Tuesday|2037Q2|N|N|N|2465211|2465361|2464854|2465129|N|N|N|N|N| +2465220|AAAAAAAAEMNJFCAA|2037-06-10|1649|7172|551|2037|3|6|10|2|2037|551|7172|Wednesday|2037Q2|N|N|N|2465211|2465361|2464855|2465130|N|N|N|N|N| +2465221|AAAAAAAAFMNJFCAA|2037-06-11|1649|7172|551|2037|4|6|11|2|2037|551|7172|Thursday|2037Q2|N|N|N|2465211|2465361|2464856|2465131|N|N|N|N|N| +2465222|AAAAAAAAGMNJFCAA|2037-06-12|1649|7172|551|2037|5|6|12|2|2037|551|7172|Friday|2037Q2|N|Y|N|2465211|2465361|2464857|2465132|N|N|N|N|N| +2465223|AAAAAAAAHMNJFCAA|2037-06-13|1649|7172|551|2037|6|6|13|2|2037|551|7172|Saturday|2037Q2|N|Y|N|2465211|2465361|2464858|2465133|N|N|N|N|N| +2465224|AAAAAAAAIMNJFCAA|2037-06-14|1649|7172|551|2037|0|6|14|2|2037|551|7172|Sunday|2037Q2|N|N|N|2465211|2465361|2464859|2465134|N|N|N|N|N| +2465225|AAAAAAAAJMNJFCAA|2037-06-15|1649|7172|551|2037|1|6|15|2|2037|551|7172|Monday|2037Q2|N|N|N|2465211|2465361|2464860|2465135|N|N|N|N|N| +2465226|AAAAAAAAKMNJFCAA|2037-06-16|1649|7173|551|2037|2|6|16|2|2037|551|7173|Tuesday|2037Q2|N|N|N|2465211|2465361|2464861|2465136|N|N|N|N|N| +2465227|AAAAAAAALMNJFCAA|2037-06-17|1649|7173|551|2037|3|6|17|2|2037|551|7173|Wednesday|2037Q2|N|N|N|2465211|2465361|2464862|2465137|N|N|N|N|N| +2465228|AAAAAAAAMMNJFCAA|2037-06-18|1649|7173|551|2037|4|6|18|2|2037|551|7173|Thursday|2037Q2|N|N|N|2465211|2465361|2464863|2465138|N|N|N|N|N| +2465229|AAAAAAAANMNJFCAA|2037-06-19|1649|7173|551|2037|5|6|19|2|2037|551|7173|Friday|2037Q2|N|Y|N|2465211|2465361|2464864|2465139|N|N|N|N|N| +2465230|AAAAAAAAOMNJFCAA|2037-06-20|1649|7173|551|2037|6|6|20|2|2037|551|7173|Saturday|2037Q2|N|Y|N|2465211|2465361|2464865|2465140|N|N|N|N|N| +2465231|AAAAAAAAPMNJFCAA|2037-06-21|1649|7173|551|2037|0|6|21|2|2037|551|7173|Sunday|2037Q2|N|N|N|2465211|2465361|2464866|2465141|N|N|N|N|N| +2465232|AAAAAAAAANNJFCAA|2037-06-22|1649|7173|551|2037|1|6|22|2|2037|551|7173|Monday|2037Q2|N|N|N|2465211|2465361|2464867|2465142|N|N|N|N|N| +2465233|AAAAAAAABNNJFCAA|2037-06-23|1649|7174|551|2037|2|6|23|2|2037|551|7174|Tuesday|2037Q2|N|N|N|2465211|2465361|2464868|2465143|N|N|N|N|N| +2465234|AAAAAAAACNNJFCAA|2037-06-24|1649|7174|551|2037|3|6|24|2|2037|551|7174|Wednesday|2037Q2|N|N|N|2465211|2465361|2464869|2465144|N|N|N|N|N| +2465235|AAAAAAAADNNJFCAA|2037-06-25|1649|7174|551|2037|4|6|25|2|2037|551|7174|Thursday|2037Q2|N|N|N|2465211|2465361|2464870|2465145|N|N|N|N|N| +2465236|AAAAAAAAENNJFCAA|2037-06-26|1649|7174|551|2037|5|6|26|2|2037|551|7174|Friday|2037Q2|N|Y|N|2465211|2465361|2464871|2465146|N|N|N|N|N| +2465237|AAAAAAAAFNNJFCAA|2037-06-27|1649|7174|551|2037|6|6|27|2|2037|551|7174|Saturday|2037Q2|N|Y|N|2465211|2465361|2464872|2465147|N|N|N|N|N| +2465238|AAAAAAAAGNNJFCAA|2037-06-28|1649|7174|551|2037|0|6|28|2|2037|551|7174|Sunday|2037Q2|N|N|N|2465211|2465361|2464873|2465148|N|N|N|N|N| +2465239|AAAAAAAAHNNJFCAA|2037-06-29|1649|7174|551|2037|1|6|29|2|2037|551|7174|Monday|2037Q2|N|N|N|2465211|2465361|2464874|2465149|N|N|N|N|N| +2465240|AAAAAAAAINNJFCAA|2037-06-30|1649|7175|551|2037|2|6|30|2|2037|551|7175|Tuesday|2037Q2|N|N|N|2465211|2465361|2464875|2465150|N|N|N|N|N| +2465241|AAAAAAAAJNNJFCAA|2037-07-01|1650|7175|551|2037|3|7|1|2|2037|551|7175|Wednesday|2037Q2|N|N|N|2465241|2465421|2464876|2465150|N|N|N|N|N| +2465242|AAAAAAAAKNNJFCAA|2037-07-02|1650|7175|551|2037|4|7|2|3|2037|551|7175|Thursday|2037Q3|N|N|N|2465241|2465421|2464877|2465151|N|N|N|N|N| +2465243|AAAAAAAALNNJFCAA|2037-07-03|1650|7175|551|2037|5|7|3|3|2037|551|7175|Friday|2037Q3|N|Y|N|2465241|2465421|2464878|2465152|N|N|N|N|N| +2465244|AAAAAAAAMNNJFCAA|2037-07-04|1650|7175|551|2037|6|7|4|3|2037|551|7175|Saturday|2037Q3|N|Y|N|2465241|2465421|2464879|2465153|N|N|N|N|N| +2465245|AAAAAAAANNNJFCAA|2037-07-05|1650|7175|551|2037|0|7|5|3|2037|551|7175|Sunday|2037Q3|Y|N|N|2465241|2465421|2464880|2465154|N|N|N|N|N| +2465246|AAAAAAAAONNJFCAA|2037-07-06|1650|7175|551|2037|1|7|6|3|2037|551|7175|Monday|2037Q3|N|N|Y|2465241|2465421|2464881|2465155|N|N|N|N|N| +2465247|AAAAAAAAPNNJFCAA|2037-07-07|1650|7176|551|2037|2|7|7|3|2037|551|7176|Tuesday|2037Q3|N|N|N|2465241|2465421|2464882|2465156|N|N|N|N|N| +2465248|AAAAAAAAAONJFCAA|2037-07-08|1650|7176|551|2037|3|7|8|3|2037|551|7176|Wednesday|2037Q3|N|N|N|2465241|2465421|2464883|2465157|N|N|N|N|N| +2465249|AAAAAAAABONJFCAA|2037-07-09|1650|7176|551|2037|4|7|9|3|2037|551|7176|Thursday|2037Q3|N|N|N|2465241|2465421|2464884|2465158|N|N|N|N|N| +2465250|AAAAAAAACONJFCAA|2037-07-10|1650|7176|551|2037|5|7|10|3|2037|551|7176|Friday|2037Q3|N|Y|N|2465241|2465421|2464885|2465159|N|N|N|N|N| +2465251|AAAAAAAADONJFCAA|2037-07-11|1650|7176|551|2037|6|7|11|3|2037|551|7176|Saturday|2037Q3|N|Y|N|2465241|2465421|2464886|2465160|N|N|N|N|N| +2465252|AAAAAAAAEONJFCAA|2037-07-12|1650|7176|551|2037|0|7|12|3|2037|551|7176|Sunday|2037Q3|N|N|N|2465241|2465421|2464887|2465161|N|N|N|N|N| +2465253|AAAAAAAAFONJFCAA|2037-07-13|1650|7176|551|2037|1|7|13|3|2037|551|7176|Monday|2037Q3|N|N|N|2465241|2465421|2464888|2465162|N|N|N|N|N| +2465254|AAAAAAAAGONJFCAA|2037-07-14|1650|7177|551|2037|2|7|14|3|2037|551|7177|Tuesday|2037Q3|N|N|N|2465241|2465421|2464889|2465163|N|N|N|N|N| +2465255|AAAAAAAAHONJFCAA|2037-07-15|1650|7177|551|2037|3|7|15|3|2037|551|7177|Wednesday|2037Q3|N|N|N|2465241|2465421|2464890|2465164|N|N|N|N|N| +2465256|AAAAAAAAIONJFCAA|2037-07-16|1650|7177|551|2037|4|7|16|3|2037|551|7177|Thursday|2037Q3|N|N|N|2465241|2465421|2464891|2465165|N|N|N|N|N| +2465257|AAAAAAAAJONJFCAA|2037-07-17|1650|7177|551|2037|5|7|17|3|2037|551|7177|Friday|2037Q3|N|Y|N|2465241|2465421|2464892|2465166|N|N|N|N|N| +2465258|AAAAAAAAKONJFCAA|2037-07-18|1650|7177|551|2037|6|7|18|3|2037|551|7177|Saturday|2037Q3|N|Y|N|2465241|2465421|2464893|2465167|N|N|N|N|N| +2465259|AAAAAAAALONJFCAA|2037-07-19|1650|7177|551|2037|0|7|19|3|2037|551|7177|Sunday|2037Q3|N|N|N|2465241|2465421|2464894|2465168|N|N|N|N|N| +2465260|AAAAAAAAMONJFCAA|2037-07-20|1650|7177|551|2037|1|7|20|3|2037|551|7177|Monday|2037Q3|N|N|N|2465241|2465421|2464895|2465169|N|N|N|N|N| +2465261|AAAAAAAANONJFCAA|2037-07-21|1650|7178|551|2037|2|7|21|3|2037|551|7178|Tuesday|2037Q3|N|N|N|2465241|2465421|2464896|2465170|N|N|N|N|N| +2465262|AAAAAAAAOONJFCAA|2037-07-22|1650|7178|551|2037|3|7|22|3|2037|551|7178|Wednesday|2037Q3|N|N|N|2465241|2465421|2464897|2465171|N|N|N|N|N| +2465263|AAAAAAAAPONJFCAA|2037-07-23|1650|7178|551|2037|4|7|23|3|2037|551|7178|Thursday|2037Q3|N|N|N|2465241|2465421|2464898|2465172|N|N|N|N|N| +2465264|AAAAAAAAAPNJFCAA|2037-07-24|1650|7178|551|2037|5|7|24|3|2037|551|7178|Friday|2037Q3|N|Y|N|2465241|2465421|2464899|2465173|N|N|N|N|N| +2465265|AAAAAAAABPNJFCAA|2037-07-25|1650|7178|551|2037|6|7|25|3|2037|551|7178|Saturday|2037Q3|N|Y|N|2465241|2465421|2464900|2465174|N|N|N|N|N| +2465266|AAAAAAAACPNJFCAA|2037-07-26|1650|7178|551|2037|0|7|26|3|2037|551|7178|Sunday|2037Q3|N|N|N|2465241|2465421|2464901|2465175|N|N|N|N|N| +2465267|AAAAAAAADPNJFCAA|2037-07-27|1650|7178|551|2037|1|7|27|3|2037|551|7178|Monday|2037Q3|N|N|N|2465241|2465421|2464902|2465176|N|N|N|N|N| +2465268|AAAAAAAAEPNJFCAA|2037-07-28|1650|7179|551|2037|2|7|28|3|2037|551|7179|Tuesday|2037Q3|N|N|N|2465241|2465421|2464903|2465177|N|N|N|N|N| +2465269|AAAAAAAAFPNJFCAA|2037-07-29|1650|7179|551|2037|3|7|29|3|2037|551|7179|Wednesday|2037Q3|N|N|N|2465241|2465421|2464904|2465178|N|N|N|N|N| +2465270|AAAAAAAAGPNJFCAA|2037-07-30|1650|7179|551|2037|4|7|30|3|2037|551|7179|Thursday|2037Q3|N|N|N|2465241|2465421|2464905|2465179|N|N|N|N|N| +2465271|AAAAAAAAHPNJFCAA|2037-07-31|1650|7179|551|2037|5|7|31|3|2037|551|7179|Friday|2037Q3|N|Y|N|2465241|2465421|2464906|2465180|N|N|N|N|N| +2465272|AAAAAAAAIPNJFCAA|2037-08-01|1651|7179|551|2037|6|8|1|3|2037|551|7179|Saturday|2037Q3|N|Y|N|2465272|2465483|2464907|2465181|N|N|N|N|N| +2465273|AAAAAAAAJPNJFCAA|2037-08-02|1651|7179|551|2037|0|8|2|3|2037|551|7179|Sunday|2037Q3|N|N|N|2465272|2465483|2464908|2465182|N|N|N|N|N| +2465274|AAAAAAAAKPNJFCAA|2037-08-03|1651|7179|551|2037|1|8|3|3|2037|551|7179|Monday|2037Q3|N|N|N|2465272|2465483|2464909|2465183|N|N|N|N|N| +2465275|AAAAAAAALPNJFCAA|2037-08-04|1651|7180|551|2037|2|8|4|3|2037|551|7180|Tuesday|2037Q3|N|N|N|2465272|2465483|2464910|2465184|N|N|N|N|N| +2465276|AAAAAAAAMPNJFCAA|2037-08-05|1651|7180|551|2037|3|8|5|3|2037|551|7180|Wednesday|2037Q3|N|N|N|2465272|2465483|2464911|2465185|N|N|N|N|N| +2465277|AAAAAAAANPNJFCAA|2037-08-06|1651|7180|551|2037|4|8|6|3|2037|551|7180|Thursday|2037Q3|N|N|N|2465272|2465483|2464912|2465186|N|N|N|N|N| +2465278|AAAAAAAAOPNJFCAA|2037-08-07|1651|7180|551|2037|5|8|7|3|2037|551|7180|Friday|2037Q3|N|Y|N|2465272|2465483|2464913|2465187|N|N|N|N|N| +2465279|AAAAAAAAPPNJFCAA|2037-08-08|1651|7180|551|2037|6|8|8|3|2037|551|7180|Saturday|2037Q3|N|Y|N|2465272|2465483|2464914|2465188|N|N|N|N|N| +2465280|AAAAAAAAAAOJFCAA|2037-08-09|1651|7180|551|2037|0|8|9|3|2037|551|7180|Sunday|2037Q3|N|N|N|2465272|2465483|2464915|2465189|N|N|N|N|N| +2465281|AAAAAAAABAOJFCAA|2037-08-10|1651|7180|551|2037|1|8|10|3|2037|551|7180|Monday|2037Q3|N|N|N|2465272|2465483|2464916|2465190|N|N|N|N|N| +2465282|AAAAAAAACAOJFCAA|2037-08-11|1651|7181|551|2037|2|8|11|3|2037|551|7181|Tuesday|2037Q3|N|N|N|2465272|2465483|2464917|2465191|N|N|N|N|N| +2465283|AAAAAAAADAOJFCAA|2037-08-12|1651|7181|551|2037|3|8|12|3|2037|551|7181|Wednesday|2037Q3|N|N|N|2465272|2465483|2464918|2465192|N|N|N|N|N| +2465284|AAAAAAAAEAOJFCAA|2037-08-13|1651|7181|551|2037|4|8|13|3|2037|551|7181|Thursday|2037Q3|N|N|N|2465272|2465483|2464919|2465193|N|N|N|N|N| +2465285|AAAAAAAAFAOJFCAA|2037-08-14|1651|7181|551|2037|5|8|14|3|2037|551|7181|Friday|2037Q3|N|Y|N|2465272|2465483|2464920|2465194|N|N|N|N|N| +2465286|AAAAAAAAGAOJFCAA|2037-08-15|1651|7181|551|2037|6|8|15|3|2037|551|7181|Saturday|2037Q3|N|Y|N|2465272|2465483|2464921|2465195|N|N|N|N|N| +2465287|AAAAAAAAHAOJFCAA|2037-08-16|1651|7181|551|2037|0|8|16|3|2037|551|7181|Sunday|2037Q3|N|N|N|2465272|2465483|2464922|2465196|N|N|N|N|N| +2465288|AAAAAAAAIAOJFCAA|2037-08-17|1651|7181|551|2037|1|8|17|3|2037|551|7181|Monday|2037Q3|N|N|N|2465272|2465483|2464923|2465197|N|N|N|N|N| +2465289|AAAAAAAAJAOJFCAA|2037-08-18|1651|7182|551|2037|2|8|18|3|2037|551|7182|Tuesday|2037Q3|N|N|N|2465272|2465483|2464924|2465198|N|N|N|N|N| +2465290|AAAAAAAAKAOJFCAA|2037-08-19|1651|7182|551|2037|3|8|19|3|2037|551|7182|Wednesday|2037Q3|N|N|N|2465272|2465483|2464925|2465199|N|N|N|N|N| +2465291|AAAAAAAALAOJFCAA|2037-08-20|1651|7182|551|2037|4|8|20|3|2037|551|7182|Thursday|2037Q3|N|N|N|2465272|2465483|2464926|2465200|N|N|N|N|N| +2465292|AAAAAAAAMAOJFCAA|2037-08-21|1651|7182|551|2037|5|8|21|3|2037|551|7182|Friday|2037Q3|N|Y|N|2465272|2465483|2464927|2465201|N|N|N|N|N| +2465293|AAAAAAAANAOJFCAA|2037-08-22|1651|7182|551|2037|6|8|22|3|2037|551|7182|Saturday|2037Q3|N|Y|N|2465272|2465483|2464928|2465202|N|N|N|N|N| +2465294|AAAAAAAAOAOJFCAA|2037-08-23|1651|7182|551|2037|0|8|23|3|2037|551|7182|Sunday|2037Q3|N|N|N|2465272|2465483|2464929|2465203|N|N|N|N|N| +2465295|AAAAAAAAPAOJFCAA|2037-08-24|1651|7182|551|2037|1|8|24|3|2037|551|7182|Monday|2037Q3|N|N|N|2465272|2465483|2464930|2465204|N|N|N|N|N| +2465296|AAAAAAAAABOJFCAA|2037-08-25|1651|7183|551|2037|2|8|25|3|2037|551|7183|Tuesday|2037Q3|N|N|N|2465272|2465483|2464931|2465205|N|N|N|N|N| +2465297|AAAAAAAABBOJFCAA|2037-08-26|1651|7183|551|2037|3|8|26|3|2037|551|7183|Wednesday|2037Q3|N|N|N|2465272|2465483|2464932|2465206|N|N|N|N|N| +2465298|AAAAAAAACBOJFCAA|2037-08-27|1651|7183|551|2037|4|8|27|3|2037|551|7183|Thursday|2037Q3|N|N|N|2465272|2465483|2464933|2465207|N|N|N|N|N| +2465299|AAAAAAAADBOJFCAA|2037-08-28|1651|7183|551|2037|5|8|28|3|2037|551|7183|Friday|2037Q3|N|Y|N|2465272|2465483|2464934|2465208|N|N|N|N|N| +2465300|AAAAAAAAEBOJFCAA|2037-08-29|1651|7183|551|2037|6|8|29|3|2037|551|7183|Saturday|2037Q3|N|Y|N|2465272|2465483|2464935|2465209|N|N|N|N|N| +2465301|AAAAAAAAFBOJFCAA|2037-08-30|1651|7183|551|2037|0|8|30|3|2037|551|7183|Sunday|2037Q3|N|N|N|2465272|2465483|2464936|2465210|N|N|N|N|N| +2465302|AAAAAAAAGBOJFCAA|2037-08-31|1651|7183|551|2037|1|8|31|3|2037|551|7183|Monday|2037Q3|N|N|N|2465272|2465483|2464937|2465211|N|N|N|N|N| +2465303|AAAAAAAAHBOJFCAA|2037-09-01|1652|7184|552|2037|2|9|1|3|2037|552|7184|Tuesday|2037Q3|N|N|N|2465303|2465545|2464938|2465212|N|N|N|N|N| +2465304|AAAAAAAAIBOJFCAA|2037-09-02|1652|7184|552|2037|3|9|2|3|2037|552|7184|Wednesday|2037Q3|N|N|N|2465303|2465545|2464939|2465213|N|N|N|N|N| +2465305|AAAAAAAAJBOJFCAA|2037-09-03|1652|7184|552|2037|4|9|3|3|2037|552|7184|Thursday|2037Q3|N|N|N|2465303|2465545|2464940|2465214|N|N|N|N|N| +2465306|AAAAAAAAKBOJFCAA|2037-09-04|1652|7184|552|2037|5|9|4|3|2037|552|7184|Friday|2037Q3|N|Y|N|2465303|2465545|2464941|2465215|N|N|N|N|N| +2465307|AAAAAAAALBOJFCAA|2037-09-05|1652|7184|552|2037|6|9|5|3|2037|552|7184|Saturday|2037Q3|N|Y|N|2465303|2465545|2464942|2465216|N|N|N|N|N| +2465308|AAAAAAAAMBOJFCAA|2037-09-06|1652|7184|552|2037|0|9|6|3|2037|552|7184|Sunday|2037Q3|N|N|N|2465303|2465545|2464943|2465217|N|N|N|N|N| +2465309|AAAAAAAANBOJFCAA|2037-09-07|1652|7184|552|2037|1|9|7|3|2037|552|7184|Monday|2037Q3|N|N|N|2465303|2465545|2464944|2465218|N|N|N|N|N| +2465310|AAAAAAAAOBOJFCAA|2037-09-08|1652|7185|552|2037|2|9|8|3|2037|552|7185|Tuesday|2037Q3|N|N|N|2465303|2465545|2464945|2465219|N|N|N|N|N| +2465311|AAAAAAAAPBOJFCAA|2037-09-09|1652|7185|552|2037|3|9|9|3|2037|552|7185|Wednesday|2037Q3|N|N|N|2465303|2465545|2464946|2465220|N|N|N|N|N| +2465312|AAAAAAAAACOJFCAA|2037-09-10|1652|7185|552|2037|4|9|10|3|2037|552|7185|Thursday|2037Q3|N|N|N|2465303|2465545|2464947|2465221|N|N|N|N|N| +2465313|AAAAAAAABCOJFCAA|2037-09-11|1652|7185|552|2037|5|9|11|3|2037|552|7185|Friday|2037Q3|N|Y|N|2465303|2465545|2464948|2465222|N|N|N|N|N| +2465314|AAAAAAAACCOJFCAA|2037-09-12|1652|7185|552|2037|6|9|12|3|2037|552|7185|Saturday|2037Q3|N|Y|N|2465303|2465545|2464949|2465223|N|N|N|N|N| +2465315|AAAAAAAADCOJFCAA|2037-09-13|1652|7185|552|2037|0|9|13|3|2037|552|7185|Sunday|2037Q3|N|N|N|2465303|2465545|2464950|2465224|N|N|N|N|N| +2465316|AAAAAAAAECOJFCAA|2037-09-14|1652|7185|552|2037|1|9|14|3|2037|552|7185|Monday|2037Q3|N|N|N|2465303|2465545|2464951|2465225|N|N|N|N|N| +2465317|AAAAAAAAFCOJFCAA|2037-09-15|1652|7186|552|2037|2|9|15|3|2037|552|7186|Tuesday|2037Q3|N|N|N|2465303|2465545|2464952|2465226|N|N|N|N|N| +2465318|AAAAAAAAGCOJFCAA|2037-09-16|1652|7186|552|2037|3|9|16|3|2037|552|7186|Wednesday|2037Q3|N|N|N|2465303|2465545|2464953|2465227|N|N|N|N|N| +2465319|AAAAAAAAHCOJFCAA|2037-09-17|1652|7186|552|2037|4|9|17|3|2037|552|7186|Thursday|2037Q3|N|N|N|2465303|2465545|2464954|2465228|N|N|N|N|N| +2465320|AAAAAAAAICOJFCAA|2037-09-18|1652|7186|552|2037|5|9|18|3|2037|552|7186|Friday|2037Q3|N|Y|N|2465303|2465545|2464955|2465229|N|N|N|N|N| +2465321|AAAAAAAAJCOJFCAA|2037-09-19|1652|7186|552|2037|6|9|19|3|2037|552|7186|Saturday|2037Q3|N|Y|N|2465303|2465545|2464956|2465230|N|N|N|N|N| +2465322|AAAAAAAAKCOJFCAA|2037-09-20|1652|7186|552|2037|0|9|20|3|2037|552|7186|Sunday|2037Q3|N|N|N|2465303|2465545|2464957|2465231|N|N|N|N|N| +2465323|AAAAAAAALCOJFCAA|2037-09-21|1652|7186|552|2037|1|9|21|3|2037|552|7186|Monday|2037Q3|N|N|N|2465303|2465545|2464958|2465232|N|N|N|N|N| +2465324|AAAAAAAAMCOJFCAA|2037-09-22|1652|7187|552|2037|2|9|22|3|2037|552|7187|Tuesday|2037Q3|N|N|N|2465303|2465545|2464959|2465233|N|N|N|N|N| +2465325|AAAAAAAANCOJFCAA|2037-09-23|1652|7187|552|2037|3|9|23|3|2037|552|7187|Wednesday|2037Q3|N|N|N|2465303|2465545|2464960|2465234|N|N|N|N|N| +2465326|AAAAAAAAOCOJFCAA|2037-09-24|1652|7187|552|2037|4|9|24|3|2037|552|7187|Thursday|2037Q3|N|N|N|2465303|2465545|2464961|2465235|N|N|N|N|N| +2465327|AAAAAAAAPCOJFCAA|2037-09-25|1652|7187|552|2037|5|9|25|3|2037|552|7187|Friday|2037Q3|N|Y|N|2465303|2465545|2464962|2465236|N|N|N|N|N| +2465328|AAAAAAAAADOJFCAA|2037-09-26|1652|7187|552|2037|6|9|26|3|2037|552|7187|Saturday|2037Q3|N|Y|N|2465303|2465545|2464963|2465237|N|N|N|N|N| +2465329|AAAAAAAABDOJFCAA|2037-09-27|1652|7187|552|2037|0|9|27|3|2037|552|7187|Sunday|2037Q3|N|N|N|2465303|2465545|2464964|2465238|N|N|N|N|N| +2465330|AAAAAAAACDOJFCAA|2037-09-28|1652|7187|552|2037|1|9|28|3|2037|552|7187|Monday|2037Q3|N|N|N|2465303|2465545|2464965|2465239|N|N|N|N|N| +2465331|AAAAAAAADDOJFCAA|2037-09-29|1652|7188|552|2037|2|9|29|3|2037|552|7188|Tuesday|2037Q3|N|N|N|2465303|2465545|2464966|2465240|N|N|N|N|N| +2465332|AAAAAAAAEDOJFCAA|2037-09-30|1652|7188|552|2037|3|9|30|3|2037|552|7188|Wednesday|2037Q3|N|N|N|2465303|2465545|2464967|2465241|N|N|N|N|N| +2465333|AAAAAAAAFDOJFCAA|2037-10-01|1653|7188|552|2037|4|10|1|3|2037|552|7188|Thursday|2037Q3|N|N|N|2465333|2465605|2464968|2465241|N|N|N|N|N| +2465334|AAAAAAAAGDOJFCAA|2037-10-02|1653|7188|552|2037|5|10|2|4|2037|552|7188|Friday|2037Q4|N|Y|N|2465333|2465605|2464969|2465242|N|N|N|N|N| +2465335|AAAAAAAAHDOJFCAA|2037-10-03|1653|7188|552|2037|6|10|3|4|2037|552|7188|Saturday|2037Q4|N|Y|N|2465333|2465605|2464970|2465243|N|N|N|N|N| +2465336|AAAAAAAAIDOJFCAA|2037-10-04|1653|7188|552|2037|0|10|4|4|2037|552|7188|Sunday|2037Q4|N|N|N|2465333|2465605|2464971|2465244|N|N|N|N|N| +2465337|AAAAAAAAJDOJFCAA|2037-10-05|1653|7188|552|2037|1|10|5|4|2037|552|7188|Monday|2037Q4|N|N|N|2465333|2465605|2464972|2465245|N|N|N|N|N| +2465338|AAAAAAAAKDOJFCAA|2037-10-06|1653|7189|552|2037|2|10|6|4|2037|552|7189|Tuesday|2037Q4|N|N|N|2465333|2465605|2464973|2465246|N|N|N|N|N| +2465339|AAAAAAAALDOJFCAA|2037-10-07|1653|7189|552|2037|3|10|7|4|2037|552|7189|Wednesday|2037Q4|N|N|N|2465333|2465605|2464974|2465247|N|N|N|N|N| +2465340|AAAAAAAAMDOJFCAA|2037-10-08|1653|7189|552|2037|4|10|8|4|2037|552|7189|Thursday|2037Q4|N|N|N|2465333|2465605|2464975|2465248|N|N|N|N|N| +2465341|AAAAAAAANDOJFCAA|2037-10-09|1653|7189|552|2037|5|10|9|4|2037|552|7189|Friday|2037Q4|N|Y|N|2465333|2465605|2464976|2465249|N|N|N|N|N| +2465342|AAAAAAAAODOJFCAA|2037-10-10|1653|7189|552|2037|6|10|10|4|2037|552|7189|Saturday|2037Q4|N|Y|N|2465333|2465605|2464977|2465250|N|N|N|N|N| +2465343|AAAAAAAAPDOJFCAA|2037-10-11|1653|7189|552|2037|0|10|11|4|2037|552|7189|Sunday|2037Q4|N|N|N|2465333|2465605|2464978|2465251|N|N|N|N|N| +2465344|AAAAAAAAAEOJFCAA|2037-10-12|1653|7189|552|2037|1|10|12|4|2037|552|7189|Monday|2037Q4|N|N|N|2465333|2465605|2464979|2465252|N|N|N|N|N| +2465345|AAAAAAAABEOJFCAA|2037-10-13|1653|7190|552|2037|2|10|13|4|2037|552|7190|Tuesday|2037Q4|N|N|N|2465333|2465605|2464980|2465253|N|N|N|N|N| +2465346|AAAAAAAACEOJFCAA|2037-10-14|1653|7190|552|2037|3|10|14|4|2037|552|7190|Wednesday|2037Q4|N|N|N|2465333|2465605|2464981|2465254|N|N|N|N|N| +2465347|AAAAAAAADEOJFCAA|2037-10-15|1653|7190|552|2037|4|10|15|4|2037|552|7190|Thursday|2037Q4|N|N|N|2465333|2465605|2464982|2465255|N|N|N|N|N| +2465348|AAAAAAAAEEOJFCAA|2037-10-16|1653|7190|552|2037|5|10|16|4|2037|552|7190|Friday|2037Q4|N|Y|N|2465333|2465605|2464983|2465256|N|N|N|N|N| +2465349|AAAAAAAAFEOJFCAA|2037-10-17|1653|7190|552|2037|6|10|17|4|2037|552|7190|Saturday|2037Q4|N|Y|N|2465333|2465605|2464984|2465257|N|N|N|N|N| +2465350|AAAAAAAAGEOJFCAA|2037-10-18|1653|7190|552|2037|0|10|18|4|2037|552|7190|Sunday|2037Q4|N|N|N|2465333|2465605|2464985|2465258|N|N|N|N|N| +2465351|AAAAAAAAHEOJFCAA|2037-10-19|1653|7190|552|2037|1|10|19|4|2037|552|7190|Monday|2037Q4|N|N|N|2465333|2465605|2464986|2465259|N|N|N|N|N| +2465352|AAAAAAAAIEOJFCAA|2037-10-20|1653|7191|552|2037|2|10|20|4|2037|552|7191|Tuesday|2037Q4|N|N|N|2465333|2465605|2464987|2465260|N|N|N|N|N| +2465353|AAAAAAAAJEOJFCAA|2037-10-21|1653|7191|552|2037|3|10|21|4|2037|552|7191|Wednesday|2037Q4|N|N|N|2465333|2465605|2464988|2465261|N|N|N|N|N| +2465354|AAAAAAAAKEOJFCAA|2037-10-22|1653|7191|552|2037|4|10|22|4|2037|552|7191|Thursday|2037Q4|N|N|N|2465333|2465605|2464989|2465262|N|N|N|N|N| +2465355|AAAAAAAALEOJFCAA|2037-10-23|1653|7191|552|2037|5|10|23|4|2037|552|7191|Friday|2037Q4|N|Y|N|2465333|2465605|2464990|2465263|N|N|N|N|N| +2465356|AAAAAAAAMEOJFCAA|2037-10-24|1653|7191|552|2037|6|10|24|4|2037|552|7191|Saturday|2037Q4|N|Y|N|2465333|2465605|2464991|2465264|N|N|N|N|N| +2465357|AAAAAAAANEOJFCAA|2037-10-25|1653|7191|552|2037|0|10|25|4|2037|552|7191|Sunday|2037Q4|N|N|N|2465333|2465605|2464992|2465265|N|N|N|N|N| +2465358|AAAAAAAAOEOJFCAA|2037-10-26|1653|7191|552|2037|1|10|26|4|2037|552|7191|Monday|2037Q4|N|N|N|2465333|2465605|2464993|2465266|N|N|N|N|N| +2465359|AAAAAAAAPEOJFCAA|2037-10-27|1653|7192|552|2037|2|10|27|4|2037|552|7192|Tuesday|2037Q4|N|N|N|2465333|2465605|2464994|2465267|N|N|N|N|N| +2465360|AAAAAAAAAFOJFCAA|2037-10-28|1653|7192|552|2037|3|10|28|4|2037|552|7192|Wednesday|2037Q4|N|N|N|2465333|2465605|2464995|2465268|N|N|N|N|N| +2465361|AAAAAAAABFOJFCAA|2037-10-29|1653|7192|552|2037|4|10|29|4|2037|552|7192|Thursday|2037Q4|N|N|N|2465333|2465605|2464996|2465269|N|N|N|N|N| +2465362|AAAAAAAACFOJFCAA|2037-10-30|1653|7192|552|2037|5|10|30|4|2037|552|7192|Friday|2037Q4|N|Y|N|2465333|2465605|2464997|2465270|N|N|N|N|N| +2465363|AAAAAAAADFOJFCAA|2037-10-31|1653|7192|552|2037|6|10|31|4|2037|552|7192|Saturday|2037Q4|N|Y|N|2465333|2465605|2464998|2465271|N|N|N|N|N| +2465364|AAAAAAAAEFOJFCAA|2037-11-01|1654|7192|552|2037|0|11|1|4|2037|552|7192|Sunday|2037Q4|N|N|N|2465364|2465667|2464999|2465272|N|N|N|N|N| +2465365|AAAAAAAAFFOJFCAA|2037-11-02|1654|7192|552|2037|1|11|2|4|2037|552|7192|Monday|2037Q4|N|N|N|2465364|2465667|2465000|2465273|N|N|N|N|N| +2465366|AAAAAAAAGFOJFCAA|2037-11-03|1654|7193|552|2037|2|11|3|4|2037|552|7193|Tuesday|2037Q4|N|N|N|2465364|2465667|2465001|2465274|N|N|N|N|N| +2465367|AAAAAAAAHFOJFCAA|2037-11-04|1654|7193|552|2037|3|11|4|4|2037|552|7193|Wednesday|2037Q4|N|N|N|2465364|2465667|2465002|2465275|N|N|N|N|N| +2465368|AAAAAAAAIFOJFCAA|2037-11-05|1654|7193|552|2037|4|11|5|4|2037|552|7193|Thursday|2037Q4|N|N|N|2465364|2465667|2465003|2465276|N|N|N|N|N| +2465369|AAAAAAAAJFOJFCAA|2037-11-06|1654|7193|552|2037|5|11|6|4|2037|552|7193|Friday|2037Q4|N|Y|N|2465364|2465667|2465004|2465277|N|N|N|N|N| +2465370|AAAAAAAAKFOJFCAA|2037-11-07|1654|7193|552|2037|6|11|7|4|2037|552|7193|Saturday|2037Q4|N|Y|N|2465364|2465667|2465005|2465278|N|N|N|N|N| +2465371|AAAAAAAALFOJFCAA|2037-11-08|1654|7193|552|2037|0|11|8|4|2037|552|7193|Sunday|2037Q4|N|N|N|2465364|2465667|2465006|2465279|N|N|N|N|N| +2465372|AAAAAAAAMFOJFCAA|2037-11-09|1654|7193|552|2037|1|11|9|4|2037|552|7193|Monday|2037Q4|N|N|N|2465364|2465667|2465007|2465280|N|N|N|N|N| +2465373|AAAAAAAANFOJFCAA|2037-11-10|1654|7194|552|2037|2|11|10|4|2037|552|7194|Tuesday|2037Q4|N|N|N|2465364|2465667|2465008|2465281|N|N|N|N|N| +2465374|AAAAAAAAOFOJFCAA|2037-11-11|1654|7194|552|2037|3|11|11|4|2037|552|7194|Wednesday|2037Q4|N|N|N|2465364|2465667|2465009|2465282|N|N|N|N|N| +2465375|AAAAAAAAPFOJFCAA|2037-11-12|1654|7194|552|2037|4|11|12|4|2037|552|7194|Thursday|2037Q4|N|N|N|2465364|2465667|2465010|2465283|N|N|N|N|N| +2465376|AAAAAAAAAGOJFCAA|2037-11-13|1654|7194|552|2037|5|11|13|4|2037|552|7194|Friday|2037Q4|N|Y|N|2465364|2465667|2465011|2465284|N|N|N|N|N| +2465377|AAAAAAAABGOJFCAA|2037-11-14|1654|7194|552|2037|6|11|14|4|2037|552|7194|Saturday|2037Q4|N|Y|N|2465364|2465667|2465012|2465285|N|N|N|N|N| +2465378|AAAAAAAACGOJFCAA|2037-11-15|1654|7194|552|2037|0|11|15|4|2037|552|7194|Sunday|2037Q4|N|N|N|2465364|2465667|2465013|2465286|N|N|N|N|N| +2465379|AAAAAAAADGOJFCAA|2037-11-16|1654|7194|552|2037|1|11|16|4|2037|552|7194|Monday|2037Q4|N|N|N|2465364|2465667|2465014|2465287|N|N|N|N|N| +2465380|AAAAAAAAEGOJFCAA|2037-11-17|1654|7195|552|2037|2|11|17|4|2037|552|7195|Tuesday|2037Q4|N|N|N|2465364|2465667|2465015|2465288|N|N|N|N|N| +2465381|AAAAAAAAFGOJFCAA|2037-11-18|1654|7195|552|2037|3|11|18|4|2037|552|7195|Wednesday|2037Q4|N|N|N|2465364|2465667|2465016|2465289|N|N|N|N|N| +2465382|AAAAAAAAGGOJFCAA|2037-11-19|1654|7195|552|2037|4|11|19|4|2037|552|7195|Thursday|2037Q4|N|N|N|2465364|2465667|2465017|2465290|N|N|N|N|N| +2465383|AAAAAAAAHGOJFCAA|2037-11-20|1654|7195|552|2037|5|11|20|4|2037|552|7195|Friday|2037Q4|N|Y|N|2465364|2465667|2465018|2465291|N|N|N|N|N| +2465384|AAAAAAAAIGOJFCAA|2037-11-21|1654|7195|552|2037|6|11|21|4|2037|552|7195|Saturday|2037Q4|N|Y|N|2465364|2465667|2465019|2465292|N|N|N|N|N| +2465385|AAAAAAAAJGOJFCAA|2037-11-22|1654|7195|552|2037|0|11|22|4|2037|552|7195|Sunday|2037Q4|N|N|N|2465364|2465667|2465020|2465293|N|N|N|N|N| +2465386|AAAAAAAAKGOJFCAA|2037-11-23|1654|7195|552|2037|1|11|23|4|2037|552|7195|Monday|2037Q4|N|N|N|2465364|2465667|2465021|2465294|N|N|N|N|N| +2465387|AAAAAAAALGOJFCAA|2037-11-24|1654|7196|552|2037|2|11|24|4|2037|552|7196|Tuesday|2037Q4|N|N|N|2465364|2465667|2465022|2465295|N|N|N|N|N| +2465388|AAAAAAAAMGOJFCAA|2037-11-25|1654|7196|552|2037|3|11|25|4|2037|552|7196|Wednesday|2037Q4|N|N|N|2465364|2465667|2465023|2465296|N|N|N|N|N| +2465389|AAAAAAAANGOJFCAA|2037-11-26|1654|7196|552|2037|4|11|26|4|2037|552|7196|Thursday|2037Q4|N|N|N|2465364|2465667|2465024|2465297|N|N|N|N|N| +2465390|AAAAAAAAOGOJFCAA|2037-11-27|1654|7196|552|2037|5|11|27|4|2037|552|7196|Friday|2037Q4|N|Y|N|2465364|2465667|2465025|2465298|N|N|N|N|N| +2465391|AAAAAAAAPGOJFCAA|2037-11-28|1654|7196|552|2037|6|11|28|4|2037|552|7196|Saturday|2037Q4|N|Y|N|2465364|2465667|2465026|2465299|N|N|N|N|N| +2465392|AAAAAAAAAHOJFCAA|2037-11-29|1654|7196|552|2037|0|11|29|4|2037|552|7196|Sunday|2037Q4|N|N|N|2465364|2465667|2465027|2465300|N|N|N|N|N| +2465393|AAAAAAAABHOJFCAA|2037-11-30|1654|7196|552|2037|1|11|30|4|2037|552|7196|Monday|2037Q4|N|N|N|2465364|2465667|2465028|2465301|N|N|N|N|N| +2465394|AAAAAAAACHOJFCAA|2037-12-01|1655|7197|553|2037|2|12|1|4|2037|553|7197|Tuesday|2037Q4|N|N|N|2465394|2465727|2465029|2465302|N|N|N|N|N| +2465395|AAAAAAAADHOJFCAA|2037-12-02|1655|7197|553|2037|3|12|2|4|2037|553|7197|Wednesday|2037Q4|N|N|N|2465394|2465727|2465030|2465303|N|N|N|N|N| +2465396|AAAAAAAAEHOJFCAA|2037-12-03|1655|7197|553|2037|4|12|3|4|2037|553|7197|Thursday|2037Q4|N|N|N|2465394|2465727|2465031|2465304|N|N|N|N|N| +2465397|AAAAAAAAFHOJFCAA|2037-12-04|1655|7197|553|2037|5|12|4|4|2037|553|7197|Friday|2037Q4|N|Y|N|2465394|2465727|2465032|2465305|N|N|N|N|N| +2465398|AAAAAAAAGHOJFCAA|2037-12-05|1655|7197|553|2037|6|12|5|4|2037|553|7197|Saturday|2037Q4|N|Y|N|2465394|2465727|2465033|2465306|N|N|N|N|N| +2465399|AAAAAAAAHHOJFCAA|2037-12-06|1655|7197|553|2037|0|12|6|4|2037|553|7197|Sunday|2037Q4|N|N|N|2465394|2465727|2465034|2465307|N|N|N|N|N| +2465400|AAAAAAAAIHOJFCAA|2037-12-07|1655|7197|553|2037|1|12|7|4|2037|553|7197|Monday|2037Q4|N|N|N|2465394|2465727|2465035|2465308|N|N|N|N|N| +2465401|AAAAAAAAJHOJFCAA|2037-12-08|1655|7198|553|2037|2|12|8|4|2037|553|7198|Tuesday|2037Q4|N|N|N|2465394|2465727|2465036|2465309|N|N|N|N|N| +2465402|AAAAAAAAKHOJFCAA|2037-12-09|1655|7198|553|2037|3|12|9|4|2037|553|7198|Wednesday|2037Q4|N|N|N|2465394|2465727|2465037|2465310|N|N|N|N|N| +2465403|AAAAAAAALHOJFCAA|2037-12-10|1655|7198|553|2037|4|12|10|4|2037|553|7198|Thursday|2037Q4|N|N|N|2465394|2465727|2465038|2465311|N|N|N|N|N| +2465404|AAAAAAAAMHOJFCAA|2037-12-11|1655|7198|553|2037|5|12|11|4|2037|553|7198|Friday|2037Q4|N|Y|N|2465394|2465727|2465039|2465312|N|N|N|N|N| +2465405|AAAAAAAANHOJFCAA|2037-12-12|1655|7198|553|2037|6|12|12|4|2037|553|7198|Saturday|2037Q4|N|Y|N|2465394|2465727|2465040|2465313|N|N|N|N|N| +2465406|AAAAAAAAOHOJFCAA|2037-12-13|1655|7198|553|2037|0|12|13|4|2037|553|7198|Sunday|2037Q4|N|N|N|2465394|2465727|2465041|2465314|N|N|N|N|N| +2465407|AAAAAAAAPHOJFCAA|2037-12-14|1655|7198|553|2037|1|12|14|4|2037|553|7198|Monday|2037Q4|N|N|N|2465394|2465727|2465042|2465315|N|N|N|N|N| +2465408|AAAAAAAAAIOJFCAA|2037-12-15|1655|7199|553|2037|2|12|15|4|2037|553|7199|Tuesday|2037Q4|N|N|N|2465394|2465727|2465043|2465316|N|N|N|N|N| +2465409|AAAAAAAABIOJFCAA|2037-12-16|1655|7199|553|2037|3|12|16|4|2037|553|7199|Wednesday|2037Q4|N|N|N|2465394|2465727|2465044|2465317|N|N|N|N|N| +2465410|AAAAAAAACIOJFCAA|2037-12-17|1655|7199|553|2037|4|12|17|4|2037|553|7199|Thursday|2037Q4|N|N|N|2465394|2465727|2465045|2465318|N|N|N|N|N| +2465411|AAAAAAAADIOJFCAA|2037-12-18|1655|7199|553|2037|5|12|18|4|2037|553|7199|Friday|2037Q4|N|Y|N|2465394|2465727|2465046|2465319|N|N|N|N|N| +2465412|AAAAAAAAEIOJFCAA|2037-12-19|1655|7199|553|2037|6|12|19|4|2037|553|7199|Saturday|2037Q4|N|Y|N|2465394|2465727|2465047|2465320|N|N|N|N|N| +2465413|AAAAAAAAFIOJFCAA|2037-12-20|1655|7199|553|2037|0|12|20|4|2037|553|7199|Sunday|2037Q4|N|N|N|2465394|2465727|2465048|2465321|N|N|N|N|N| +2465414|AAAAAAAAGIOJFCAA|2037-12-21|1655|7199|553|2037|1|12|21|4|2037|553|7199|Monday|2037Q4|N|N|N|2465394|2465727|2465049|2465322|N|N|N|N|N| +2465415|AAAAAAAAHIOJFCAA|2037-12-22|1655|7200|553|2037|2|12|22|4|2037|553|7200|Tuesday|2037Q4|N|N|N|2465394|2465727|2465050|2465323|N|N|N|N|N| +2465416|AAAAAAAAIIOJFCAA|2037-12-23|1655|7200|553|2037|3|12|23|4|2037|553|7200|Wednesday|2037Q4|N|N|N|2465394|2465727|2465051|2465324|N|N|N|N|N| +2465417|AAAAAAAAJIOJFCAA|2037-12-24|1655|7200|553|2037|4|12|24|4|2037|553|7200|Thursday|2037Q4|N|N|N|2465394|2465727|2465052|2465325|N|N|N|N|N| +2465418|AAAAAAAAKIOJFCAA|2037-12-25|1655|7200|553|2037|5|12|25|4|2037|553|7200|Friday|2037Q4|N|Y|N|2465394|2465727|2465053|2465326|N|N|N|N|N| +2465419|AAAAAAAALIOJFCAA|2037-12-26|1655|7200|553|2037|6|12|26|4|2037|553|7200|Saturday|2037Q4|Y|Y|N|2465394|2465727|2465054|2465327|N|N|N|N|N| +2465420|AAAAAAAAMIOJFCAA|2037-12-27|1655|7200|553|2037|0|12|27|4|2037|553|7200|Sunday|2037Q4|N|N|Y|2465394|2465727|2465055|2465328|N|N|N|N|N| +2465421|AAAAAAAANIOJFCAA|2037-12-28|1655|7200|553|2037|1|12|28|4|2037|553|7200|Monday|2037Q4|N|N|N|2465394|2465727|2465056|2465329|N|N|N|N|N| +2465422|AAAAAAAAOIOJFCAA|2037-12-29|1655|7201|553|2037|2|12|29|4|2037|553|7201|Tuesday|2037Q4|N|N|N|2465394|2465727|2465057|2465330|N|N|N|N|N| +2465423|AAAAAAAAPIOJFCAA|2037-12-30|1655|7201|553|2037|3|12|30|4|2037|553|7201|Wednesday|2037Q4|N|N|N|2465394|2465727|2465058|2465331|N|N|N|N|N| +2465424|AAAAAAAAAJOJFCAA|2037-12-31|1655|7201|553|2037|4|12|31|4|2037|553|7201|Thursday|2037Q4|N|N|N|2465394|2465727|2465059|2465332|N|N|N|N|N| +2465425|AAAAAAAABJOJFCAA|2038-01-01|1656|7201|553|2038|5|1|1|1|2038|553|7201|Friday|2038Q1|Y|Y|N|2465425|2465424|2465060|2465333|N|N|N|N|N| +2465426|AAAAAAAACJOJFCAA|2038-01-02|1656|7201|553|2038|6|1|2|1|2038|553|7201|Saturday|2038Q1|N|Y|Y|2465425|2465424|2465061|2465334|N|N|N|N|N| +2465427|AAAAAAAADJOJFCAA|2038-01-03|1656|7201|553|2038|0|1|3|1|2038|553|7201|Sunday|2038Q1|N|N|N|2465425|2465424|2465062|2465335|N|N|N|N|N| +2465428|AAAAAAAAEJOJFCAA|2038-01-04|1656|7201|553|2038|1|1|4|1|2038|553|7201|Monday|2038Q1|N|N|N|2465425|2465424|2465063|2465336|N|N|N|N|N| +2465429|AAAAAAAAFJOJFCAA|2038-01-05|1656|7202|553|2038|2|1|5|1|2038|553|7202|Tuesday|2038Q1|N|N|N|2465425|2465424|2465064|2465337|N|N|N|N|N| +2465430|AAAAAAAAGJOJFCAA|2038-01-06|1656|7202|553|2038|3|1|6|1|2038|553|7202|Wednesday|2038Q1|N|N|N|2465425|2465424|2465065|2465338|N|N|N|N|N| +2465431|AAAAAAAAHJOJFCAA|2038-01-07|1656|7202|553|2038|4|1|7|1|2038|553|7202|Thursday|2038Q1|N|N|N|2465425|2465424|2465066|2465339|N|N|N|N|N| +2465432|AAAAAAAAIJOJFCAA|2038-01-08|1656|7202|553|2038|5|1|8|1|2038|553|7202|Friday|2038Q1|N|Y|N|2465425|2465424|2465067|2465340|N|N|N|N|N| +2465433|AAAAAAAAJJOJFCAA|2038-01-09|1656|7202|553|2038|6|1|9|1|2038|553|7202|Saturday|2038Q1|N|Y|N|2465425|2465424|2465068|2465341|N|N|N|N|N| +2465434|AAAAAAAAKJOJFCAA|2038-01-10|1656|7202|553|2038|0|1|10|1|2038|553|7202|Sunday|2038Q1|N|N|N|2465425|2465424|2465069|2465342|N|N|N|N|N| +2465435|AAAAAAAALJOJFCAA|2038-01-11|1656|7202|553|2038|1|1|11|1|2038|553|7202|Monday|2038Q1|N|N|N|2465425|2465424|2465070|2465343|N|N|N|N|N| +2465436|AAAAAAAAMJOJFCAA|2038-01-12|1656|7203|553|2038|2|1|12|1|2038|553|7203|Tuesday|2038Q1|N|N|N|2465425|2465424|2465071|2465344|N|N|N|N|N| +2465437|AAAAAAAANJOJFCAA|2038-01-13|1656|7203|553|2038|3|1|13|1|2038|553|7203|Wednesday|2038Q1|N|N|N|2465425|2465424|2465072|2465345|N|N|N|N|N| +2465438|AAAAAAAAOJOJFCAA|2038-01-14|1656|7203|553|2038|4|1|14|1|2038|553|7203|Thursday|2038Q1|N|N|N|2465425|2465424|2465073|2465346|N|N|N|N|N| +2465439|AAAAAAAAPJOJFCAA|2038-01-15|1656|7203|553|2038|5|1|15|1|2038|553|7203|Friday|2038Q1|N|Y|N|2465425|2465424|2465074|2465347|N|N|N|N|N| +2465440|AAAAAAAAAKOJFCAA|2038-01-16|1656|7203|553|2038|6|1|16|1|2038|553|7203|Saturday|2038Q1|N|Y|N|2465425|2465424|2465075|2465348|N|N|N|N|N| +2465441|AAAAAAAABKOJFCAA|2038-01-17|1656|7203|553|2038|0|1|17|1|2038|553|7203|Sunday|2038Q1|N|N|N|2465425|2465424|2465076|2465349|N|N|N|N|N| +2465442|AAAAAAAACKOJFCAA|2038-01-18|1656|7203|553|2038|1|1|18|1|2038|553|7203|Monday|2038Q1|N|N|N|2465425|2465424|2465077|2465350|N|N|N|N|N| +2465443|AAAAAAAADKOJFCAA|2038-01-19|1656|7204|553|2038|2|1|19|1|2038|553|7204|Tuesday|2038Q1|N|N|N|2465425|2465424|2465078|2465351|N|N|N|N|N| +2465444|AAAAAAAAEKOJFCAA|2038-01-20|1656|7204|553|2038|3|1|20|1|2038|553|7204|Wednesday|2038Q1|N|N|N|2465425|2465424|2465079|2465352|N|N|N|N|N| +2465445|AAAAAAAAFKOJFCAA|2038-01-21|1656|7204|553|2038|4|1|21|1|2038|553|7204|Thursday|2038Q1|N|N|N|2465425|2465424|2465080|2465353|N|N|N|N|N| +2465446|AAAAAAAAGKOJFCAA|2038-01-22|1656|7204|553|2038|5|1|22|1|2038|553|7204|Friday|2038Q1|N|Y|N|2465425|2465424|2465081|2465354|N|N|N|N|N| +2465447|AAAAAAAAHKOJFCAA|2038-01-23|1656|7204|553|2038|6|1|23|1|2038|553|7204|Saturday|2038Q1|N|Y|N|2465425|2465424|2465082|2465355|N|N|N|N|N| +2465448|AAAAAAAAIKOJFCAA|2038-01-24|1656|7204|553|2038|0|1|24|1|2038|553|7204|Sunday|2038Q1|N|N|N|2465425|2465424|2465083|2465356|N|N|N|N|N| +2465449|AAAAAAAAJKOJFCAA|2038-01-25|1656|7204|553|2038|1|1|25|1|2038|553|7204|Monday|2038Q1|N|N|N|2465425|2465424|2465084|2465357|N|N|N|N|N| +2465450|AAAAAAAAKKOJFCAA|2038-01-26|1656|7205|553|2038|2|1|26|1|2038|553|7205|Tuesday|2038Q1|N|N|N|2465425|2465424|2465085|2465358|N|N|N|N|N| +2465451|AAAAAAAALKOJFCAA|2038-01-27|1656|7205|553|2038|3|1|27|1|2038|553|7205|Wednesday|2038Q1|N|N|N|2465425|2465424|2465086|2465359|N|N|N|N|N| +2465452|AAAAAAAAMKOJFCAA|2038-01-28|1656|7205|553|2038|4|1|28|1|2038|553|7205|Thursday|2038Q1|N|N|N|2465425|2465424|2465087|2465360|N|N|N|N|N| +2465453|AAAAAAAANKOJFCAA|2038-01-29|1656|7205|553|2038|5|1|29|1|2038|553|7205|Friday|2038Q1|N|Y|N|2465425|2465424|2465088|2465361|N|N|N|N|N| +2465454|AAAAAAAAOKOJFCAA|2038-01-30|1656|7205|553|2038|6|1|30|1|2038|553|7205|Saturday|2038Q1|N|Y|N|2465425|2465424|2465089|2465362|N|N|N|N|N| +2465455|AAAAAAAAPKOJFCAA|2038-01-31|1656|7205|553|2038|0|1|31|1|2038|553|7205|Sunday|2038Q1|N|N|N|2465425|2465424|2465090|2465363|N|N|N|N|N| +2465456|AAAAAAAAALOJFCAA|2038-02-01|1657|7205|553|2038|1|2|1|1|2038|553|7205|Monday|2038Q1|N|N|N|2465456|2465486|2465091|2465364|N|N|N|N|N| +2465457|AAAAAAAABLOJFCAA|2038-02-02|1657|7206|553|2038|2|2|2|1|2038|553|7206|Tuesday|2038Q1|N|N|N|2465456|2465486|2465092|2465365|N|N|N|N|N| +2465458|AAAAAAAACLOJFCAA|2038-02-03|1657|7206|553|2038|3|2|3|1|2038|553|7206|Wednesday|2038Q1|N|N|N|2465456|2465486|2465093|2465366|N|N|N|N|N| +2465459|AAAAAAAADLOJFCAA|2038-02-04|1657|7206|553|2038|4|2|4|1|2038|553|7206|Thursday|2038Q1|N|N|N|2465456|2465486|2465094|2465367|N|N|N|N|N| +2465460|AAAAAAAAELOJFCAA|2038-02-05|1657|7206|553|2038|5|2|5|1|2038|553|7206|Friday|2038Q1|N|Y|N|2465456|2465486|2465095|2465368|N|N|N|N|N| +2465461|AAAAAAAAFLOJFCAA|2038-02-06|1657|7206|553|2038|6|2|6|1|2038|553|7206|Saturday|2038Q1|N|Y|N|2465456|2465486|2465096|2465369|N|N|N|N|N| +2465462|AAAAAAAAGLOJFCAA|2038-02-07|1657|7206|553|2038|0|2|7|1|2038|553|7206|Sunday|2038Q1|N|N|N|2465456|2465486|2465097|2465370|N|N|N|N|N| +2465463|AAAAAAAAHLOJFCAA|2038-02-08|1657|7206|553|2038|1|2|8|1|2038|553|7206|Monday|2038Q1|N|N|N|2465456|2465486|2465098|2465371|N|N|N|N|N| +2465464|AAAAAAAAILOJFCAA|2038-02-09|1657|7207|553|2038|2|2|9|1|2038|553|7207|Tuesday|2038Q1|N|N|N|2465456|2465486|2465099|2465372|N|N|N|N|N| +2465465|AAAAAAAAJLOJFCAA|2038-02-10|1657|7207|553|2038|3|2|10|1|2038|553|7207|Wednesday|2038Q1|N|N|N|2465456|2465486|2465100|2465373|N|N|N|N|N| +2465466|AAAAAAAAKLOJFCAA|2038-02-11|1657|7207|553|2038|4|2|11|1|2038|553|7207|Thursday|2038Q1|N|N|N|2465456|2465486|2465101|2465374|N|N|N|N|N| +2465467|AAAAAAAALLOJFCAA|2038-02-12|1657|7207|553|2038|5|2|12|1|2038|553|7207|Friday|2038Q1|N|Y|N|2465456|2465486|2465102|2465375|N|N|N|N|N| +2465468|AAAAAAAAMLOJFCAA|2038-02-13|1657|7207|553|2038|6|2|13|1|2038|553|7207|Saturday|2038Q1|N|Y|N|2465456|2465486|2465103|2465376|N|N|N|N|N| +2465469|AAAAAAAANLOJFCAA|2038-02-14|1657|7207|553|2038|0|2|14|1|2038|553|7207|Sunday|2038Q1|N|N|N|2465456|2465486|2465104|2465377|N|N|N|N|N| +2465470|AAAAAAAAOLOJFCAA|2038-02-15|1657|7207|553|2038|1|2|15|1|2038|553|7207|Monday|2038Q1|N|N|N|2465456|2465486|2465105|2465378|N|N|N|N|N| +2465471|AAAAAAAAPLOJFCAA|2038-02-16|1657|7208|553|2038|2|2|16|1|2038|553|7208|Tuesday|2038Q1|N|N|N|2465456|2465486|2465106|2465379|N|N|N|N|N| +2465472|AAAAAAAAAMOJFCAA|2038-02-17|1657|7208|553|2038|3|2|17|1|2038|553|7208|Wednesday|2038Q1|N|N|N|2465456|2465486|2465107|2465380|N|N|N|N|N| +2465473|AAAAAAAABMOJFCAA|2038-02-18|1657|7208|553|2038|4|2|18|1|2038|553|7208|Thursday|2038Q1|N|N|N|2465456|2465486|2465108|2465381|N|N|N|N|N| +2465474|AAAAAAAACMOJFCAA|2038-02-19|1657|7208|553|2038|5|2|19|1|2038|553|7208|Friday|2038Q1|N|Y|N|2465456|2465486|2465109|2465382|N|N|N|N|N| +2465475|AAAAAAAADMOJFCAA|2038-02-20|1657|7208|553|2038|6|2|20|1|2038|553|7208|Saturday|2038Q1|N|Y|N|2465456|2465486|2465110|2465383|N|N|N|N|N| +2465476|AAAAAAAAEMOJFCAA|2038-02-21|1657|7208|553|2038|0|2|21|1|2038|553|7208|Sunday|2038Q1|N|N|N|2465456|2465486|2465111|2465384|N|N|N|N|N| +2465477|AAAAAAAAFMOJFCAA|2038-02-22|1657|7208|553|2038|1|2|22|1|2038|553|7208|Monday|2038Q1|N|N|N|2465456|2465486|2465112|2465385|N|N|N|N|N| +2465478|AAAAAAAAGMOJFCAA|2038-02-23|1657|7209|553|2038|2|2|23|1|2038|553|7209|Tuesday|2038Q1|N|N|N|2465456|2465486|2465113|2465386|N|N|N|N|N| +2465479|AAAAAAAAHMOJFCAA|2038-02-24|1657|7209|553|2038|3|2|24|1|2038|553|7209|Wednesday|2038Q1|N|N|N|2465456|2465486|2465114|2465387|N|N|N|N|N| +2465480|AAAAAAAAIMOJFCAA|2038-02-25|1657|7209|553|2038|4|2|25|1|2038|553|7209|Thursday|2038Q1|N|N|N|2465456|2465486|2465115|2465388|N|N|N|N|N| +2465481|AAAAAAAAJMOJFCAA|2038-02-26|1657|7209|553|2038|5|2|26|1|2038|553|7209|Friday|2038Q1|N|Y|N|2465456|2465486|2465116|2465389|N|N|N|N|N| +2465482|AAAAAAAAKMOJFCAA|2038-02-27|1657|7209|553|2038|6|2|27|1|2038|553|7209|Saturday|2038Q1|N|Y|N|2465456|2465486|2465117|2465390|N|N|N|N|N| +2465483|AAAAAAAALMOJFCAA|2038-02-28|1657|7209|553|2038|0|2|28|1|2038|553|7209|Sunday|2038Q1|N|N|N|2465456|2465486|2465118|2465391|N|N|N|N|N| +2465484|AAAAAAAAMMOJFCAA|2038-03-01|1658|7209|554|2038|1|3|1|1|2038|554|7209|Monday|2038Q1|N|N|N|2465484|2465542|2465119|2465392|N|N|N|N|N| +2465485|AAAAAAAANMOJFCAA|2038-03-02|1658|7210|554|2038|2|3|2|1|2038|554|7210|Tuesday|2038Q1|N|N|N|2465484|2465542|2465120|2465393|N|N|N|N|N| +2465486|AAAAAAAAOMOJFCAA|2038-03-03|1658|7210|554|2038|3|3|3|1|2038|554|7210|Wednesday|2038Q1|N|N|N|2465484|2465542|2465121|2465394|N|N|N|N|N| +2465487|AAAAAAAAPMOJFCAA|2038-03-04|1658|7210|554|2038|4|3|4|1|2038|554|7210|Thursday|2038Q1|N|N|N|2465484|2465542|2465122|2465395|N|N|N|N|N| +2465488|AAAAAAAAANOJFCAA|2038-03-05|1658|7210|554|2038|5|3|5|1|2038|554|7210|Friday|2038Q1|N|Y|N|2465484|2465542|2465123|2465396|N|N|N|N|N| +2465489|AAAAAAAABNOJFCAA|2038-03-06|1658|7210|554|2038|6|3|6|1|2038|554|7210|Saturday|2038Q1|N|Y|N|2465484|2465542|2465124|2465397|N|N|N|N|N| +2465490|AAAAAAAACNOJFCAA|2038-03-07|1658|7210|554|2038|0|3|7|1|2038|554|7210|Sunday|2038Q1|N|N|N|2465484|2465542|2465125|2465398|N|N|N|N|N| +2465491|AAAAAAAADNOJFCAA|2038-03-08|1658|7210|554|2038|1|3|8|1|2038|554|7210|Monday|2038Q1|N|N|N|2465484|2465542|2465126|2465399|N|N|N|N|N| +2465492|AAAAAAAAENOJFCAA|2038-03-09|1658|7211|554|2038|2|3|9|1|2038|554|7211|Tuesday|2038Q1|N|N|N|2465484|2465542|2465127|2465400|N|N|N|N|N| +2465493|AAAAAAAAFNOJFCAA|2038-03-10|1658|7211|554|2038|3|3|10|1|2038|554|7211|Wednesday|2038Q1|N|N|N|2465484|2465542|2465128|2465401|N|N|N|N|N| +2465494|AAAAAAAAGNOJFCAA|2038-03-11|1658|7211|554|2038|4|3|11|1|2038|554|7211|Thursday|2038Q1|N|N|N|2465484|2465542|2465129|2465402|N|N|N|N|N| +2465495|AAAAAAAAHNOJFCAA|2038-03-12|1658|7211|554|2038|5|3|12|1|2038|554|7211|Friday|2038Q1|N|Y|N|2465484|2465542|2465130|2465403|N|N|N|N|N| +2465496|AAAAAAAAINOJFCAA|2038-03-13|1658|7211|554|2038|6|3|13|1|2038|554|7211|Saturday|2038Q1|N|Y|N|2465484|2465542|2465131|2465404|N|N|N|N|N| +2465497|AAAAAAAAJNOJFCAA|2038-03-14|1658|7211|554|2038|0|3|14|1|2038|554|7211|Sunday|2038Q1|N|N|N|2465484|2465542|2465132|2465405|N|N|N|N|N| +2465498|AAAAAAAAKNOJFCAA|2038-03-15|1658|7211|554|2038|1|3|15|1|2038|554|7211|Monday|2038Q1|N|N|N|2465484|2465542|2465133|2465406|N|N|N|N|N| +2465499|AAAAAAAALNOJFCAA|2038-03-16|1658|7212|554|2038|2|3|16|1|2038|554|7212|Tuesday|2038Q1|N|N|N|2465484|2465542|2465134|2465407|N|N|N|N|N| +2465500|AAAAAAAAMNOJFCAA|2038-03-17|1658|7212|554|2038|3|3|17|1|2038|554|7212|Wednesday|2038Q1|N|N|N|2465484|2465542|2465135|2465408|N|N|N|N|N| +2465501|AAAAAAAANNOJFCAA|2038-03-18|1658|7212|554|2038|4|3|18|1|2038|554|7212|Thursday|2038Q1|N|N|N|2465484|2465542|2465136|2465409|N|N|N|N|N| +2465502|AAAAAAAAONOJFCAA|2038-03-19|1658|7212|554|2038|5|3|19|1|2038|554|7212|Friday|2038Q1|N|Y|N|2465484|2465542|2465137|2465410|N|N|N|N|N| +2465503|AAAAAAAAPNOJFCAA|2038-03-20|1658|7212|554|2038|6|3|20|1|2038|554|7212|Saturday|2038Q1|N|Y|N|2465484|2465542|2465138|2465411|N|N|N|N|N| +2465504|AAAAAAAAAOOJFCAA|2038-03-21|1658|7212|554|2038|0|3|21|1|2038|554|7212|Sunday|2038Q1|N|N|N|2465484|2465542|2465139|2465412|N|N|N|N|N| +2465505|AAAAAAAABOOJFCAA|2038-03-22|1658|7212|554|2038|1|3|22|1|2038|554|7212|Monday|2038Q1|N|N|N|2465484|2465542|2465140|2465413|N|N|N|N|N| +2465506|AAAAAAAACOOJFCAA|2038-03-23|1658|7213|554|2038|2|3|23|1|2038|554|7213|Tuesday|2038Q1|N|N|N|2465484|2465542|2465141|2465414|N|N|N|N|N| +2465507|AAAAAAAADOOJFCAA|2038-03-24|1658|7213|554|2038|3|3|24|1|2038|554|7213|Wednesday|2038Q1|N|N|N|2465484|2465542|2465142|2465415|N|N|N|N|N| +2465508|AAAAAAAAEOOJFCAA|2038-03-25|1658|7213|554|2038|4|3|25|1|2038|554|7213|Thursday|2038Q1|N|N|N|2465484|2465542|2465143|2465416|N|N|N|N|N| +2465509|AAAAAAAAFOOJFCAA|2038-03-26|1658|7213|554|2038|5|3|26|1|2038|554|7213|Friday|2038Q1|N|Y|N|2465484|2465542|2465144|2465417|N|N|N|N|N| +2465510|AAAAAAAAGOOJFCAA|2038-03-27|1658|7213|554|2038|6|3|27|1|2038|554|7213|Saturday|2038Q1|N|Y|N|2465484|2465542|2465145|2465418|N|N|N|N|N| +2465511|AAAAAAAAHOOJFCAA|2038-03-28|1658|7213|554|2038|0|3|28|1|2038|554|7213|Sunday|2038Q1|N|N|N|2465484|2465542|2465146|2465419|N|N|N|N|N| +2465512|AAAAAAAAIOOJFCAA|2038-03-29|1658|7213|554|2038|1|3|29|1|2038|554|7213|Monday|2038Q1|N|N|N|2465484|2465542|2465147|2465420|N|N|N|N|N| +2465513|AAAAAAAAJOOJFCAA|2038-03-30|1658|7214|554|2038|2|3|30|1|2038|554|7214|Tuesday|2038Q1|N|N|N|2465484|2465542|2465148|2465421|N|N|N|N|N| +2465514|AAAAAAAAKOOJFCAA|2038-03-31|1658|7214|554|2038|3|3|31|1|2038|554|7214|Wednesday|2038Q1|N|N|N|2465484|2465542|2465149|2465422|N|N|N|N|N| +2465515|AAAAAAAALOOJFCAA|2038-04-01|1659|7214|554|2038|4|4|1|1|2038|554|7214|Thursday|2038Q1|N|N|N|2465515|2465604|2465150|2465425|N|N|N|N|N| +2465516|AAAAAAAAMOOJFCAA|2038-04-02|1659|7214|554|2038|5|4|2|2|2038|554|7214|Friday|2038Q2|N|Y|N|2465515|2465604|2465151|2465426|N|N|N|N|N| +2465517|AAAAAAAANOOJFCAA|2038-04-03|1659|7214|554|2038|6|4|3|2|2038|554|7214|Saturday|2038Q2|N|Y|N|2465515|2465604|2465152|2465427|N|N|N|N|N| +2465518|AAAAAAAAOOOJFCAA|2038-04-04|1659|7214|554|2038|0|4|4|2|2038|554|7214|Sunday|2038Q2|N|N|N|2465515|2465604|2465153|2465428|N|N|N|N|N| +2465519|AAAAAAAAPOOJFCAA|2038-04-05|1659|7214|554|2038|1|4|5|2|2038|554|7214|Monday|2038Q2|N|N|N|2465515|2465604|2465154|2465429|N|N|N|N|N| +2465520|AAAAAAAAAPOJFCAA|2038-04-06|1659|7215|554|2038|2|4|6|2|2038|554|7215|Tuesday|2038Q2|N|N|N|2465515|2465604|2465155|2465430|N|N|N|N|N| +2465521|AAAAAAAABPOJFCAA|2038-04-07|1659|7215|554|2038|3|4|7|2|2038|554|7215|Wednesday|2038Q2|N|N|N|2465515|2465604|2465156|2465431|N|N|N|N|N| +2465522|AAAAAAAACPOJFCAA|2038-04-08|1659|7215|554|2038|4|4|8|2|2038|554|7215|Thursday|2038Q2|N|N|N|2465515|2465604|2465157|2465432|N|N|N|N|N| +2465523|AAAAAAAADPOJFCAA|2038-04-09|1659|7215|554|2038|5|4|9|2|2038|554|7215|Friday|2038Q2|N|Y|N|2465515|2465604|2465158|2465433|N|N|N|N|N| +2465524|AAAAAAAAEPOJFCAA|2038-04-10|1659|7215|554|2038|6|4|10|2|2038|554|7215|Saturday|2038Q2|N|Y|N|2465515|2465604|2465159|2465434|N|N|N|N|N| +2465525|AAAAAAAAFPOJFCAA|2038-04-11|1659|7215|554|2038|0|4|11|2|2038|554|7215|Sunday|2038Q2|N|N|N|2465515|2465604|2465160|2465435|N|N|N|N|N| +2465526|AAAAAAAAGPOJFCAA|2038-04-12|1659|7215|554|2038|1|4|12|2|2038|554|7215|Monday|2038Q2|N|N|N|2465515|2465604|2465161|2465436|N|N|N|N|N| +2465527|AAAAAAAAHPOJFCAA|2038-04-13|1659|7216|554|2038|2|4|13|2|2038|554|7216|Tuesday|2038Q2|N|N|N|2465515|2465604|2465162|2465437|N|N|N|N|N| +2465528|AAAAAAAAIPOJFCAA|2038-04-14|1659|7216|554|2038|3|4|14|2|2038|554|7216|Wednesday|2038Q2|N|N|N|2465515|2465604|2465163|2465438|N|N|N|N|N| +2465529|AAAAAAAAJPOJFCAA|2038-04-15|1659|7216|554|2038|4|4|15|2|2038|554|7216|Thursday|2038Q2|N|N|N|2465515|2465604|2465164|2465439|N|N|N|N|N| +2465530|AAAAAAAAKPOJFCAA|2038-04-16|1659|7216|554|2038|5|4|16|2|2038|554|7216|Friday|2038Q2|N|Y|N|2465515|2465604|2465165|2465440|N|N|N|N|N| +2465531|AAAAAAAALPOJFCAA|2038-04-17|1659|7216|554|2038|6|4|17|2|2038|554|7216|Saturday|2038Q2|N|Y|N|2465515|2465604|2465166|2465441|N|N|N|N|N| +2465532|AAAAAAAAMPOJFCAA|2038-04-18|1659|7216|554|2038|0|4|18|2|2038|554|7216|Sunday|2038Q2|N|N|N|2465515|2465604|2465167|2465442|N|N|N|N|N| +2465533|AAAAAAAANPOJFCAA|2038-04-19|1659|7216|554|2038|1|4|19|2|2038|554|7216|Monday|2038Q2|N|N|N|2465515|2465604|2465168|2465443|N|N|N|N|N| +2465534|AAAAAAAAOPOJFCAA|2038-04-20|1659|7217|554|2038|2|4|20|2|2038|554|7217|Tuesday|2038Q2|N|N|N|2465515|2465604|2465169|2465444|N|N|N|N|N| +2465535|AAAAAAAAPPOJFCAA|2038-04-21|1659|7217|554|2038|3|4|21|2|2038|554|7217|Wednesday|2038Q2|N|N|N|2465515|2465604|2465170|2465445|N|N|N|N|N| +2465536|AAAAAAAAAAPJFCAA|2038-04-22|1659|7217|554|2038|4|4|22|2|2038|554|7217|Thursday|2038Q2|N|N|N|2465515|2465604|2465171|2465446|N|N|N|N|N| +2465537|AAAAAAAABAPJFCAA|2038-04-23|1659|7217|554|2038|5|4|23|2|2038|554|7217|Friday|2038Q2|N|Y|N|2465515|2465604|2465172|2465447|N|N|N|N|N| +2465538|AAAAAAAACAPJFCAA|2038-04-24|1659|7217|554|2038|6|4|24|2|2038|554|7217|Saturday|2038Q2|N|Y|N|2465515|2465604|2465173|2465448|N|N|N|N|N| +2465539|AAAAAAAADAPJFCAA|2038-04-25|1659|7217|554|2038|0|4|25|2|2038|554|7217|Sunday|2038Q2|N|N|N|2465515|2465604|2465174|2465449|N|N|N|N|N| +2465540|AAAAAAAAEAPJFCAA|2038-04-26|1659|7217|554|2038|1|4|26|2|2038|554|7217|Monday|2038Q2|N|N|N|2465515|2465604|2465175|2465450|N|N|N|N|N| +2465541|AAAAAAAAFAPJFCAA|2038-04-27|1659|7218|554|2038|2|4|27|2|2038|554|7218|Tuesday|2038Q2|N|N|N|2465515|2465604|2465176|2465451|N|N|N|N|N| +2465542|AAAAAAAAGAPJFCAA|2038-04-28|1659|7218|554|2038|3|4|28|2|2038|554|7218|Wednesday|2038Q2|N|N|N|2465515|2465604|2465177|2465452|N|N|N|N|N| +2465543|AAAAAAAAHAPJFCAA|2038-04-29|1659|7218|554|2038|4|4|29|2|2038|554|7218|Thursday|2038Q2|N|N|N|2465515|2465604|2465178|2465453|N|N|N|N|N| +2465544|AAAAAAAAIAPJFCAA|2038-04-30|1659|7218|554|2038|5|4|30|2|2038|554|7218|Friday|2038Q2|N|Y|N|2465515|2465604|2465179|2465454|N|N|N|N|N| +2465545|AAAAAAAAJAPJFCAA|2038-05-01|1660|7218|554|2038|6|5|1|2|2038|554|7218|Saturday|2038Q2|N|Y|N|2465545|2465664|2465180|2465455|N|N|N|N|N| +2465546|AAAAAAAAKAPJFCAA|2038-05-02|1660|7218|554|2038|0|5|2|2|2038|554|7218|Sunday|2038Q2|N|N|N|2465545|2465664|2465181|2465456|N|N|N|N|N| +2465547|AAAAAAAALAPJFCAA|2038-05-03|1660|7218|554|2038|1|5|3|2|2038|554|7218|Monday|2038Q2|N|N|N|2465545|2465664|2465182|2465457|N|N|N|N|N| +2465548|AAAAAAAAMAPJFCAA|2038-05-04|1660|7219|554|2038|2|5|4|2|2038|554|7219|Tuesday|2038Q2|N|N|N|2465545|2465664|2465183|2465458|N|N|N|N|N| +2465549|AAAAAAAANAPJFCAA|2038-05-05|1660|7219|554|2038|3|5|5|2|2038|554|7219|Wednesday|2038Q2|N|N|N|2465545|2465664|2465184|2465459|N|N|N|N|N| +2465550|AAAAAAAAOAPJFCAA|2038-05-06|1660|7219|554|2038|4|5|6|2|2038|554|7219|Thursday|2038Q2|N|N|N|2465545|2465664|2465185|2465460|N|N|N|N|N| +2465551|AAAAAAAAPAPJFCAA|2038-05-07|1660|7219|554|2038|5|5|7|2|2038|554|7219|Friday|2038Q2|N|Y|N|2465545|2465664|2465186|2465461|N|N|N|N|N| +2465552|AAAAAAAAABPJFCAA|2038-05-08|1660|7219|554|2038|6|5|8|2|2038|554|7219|Saturday|2038Q2|N|Y|N|2465545|2465664|2465187|2465462|N|N|N|N|N| +2465553|AAAAAAAABBPJFCAA|2038-05-09|1660|7219|554|2038|0|5|9|2|2038|554|7219|Sunday|2038Q2|N|N|N|2465545|2465664|2465188|2465463|N|N|N|N|N| +2465554|AAAAAAAACBPJFCAA|2038-05-10|1660|7219|554|2038|1|5|10|2|2038|554|7219|Monday|2038Q2|N|N|N|2465545|2465664|2465189|2465464|N|N|N|N|N| +2465555|AAAAAAAADBPJFCAA|2038-05-11|1660|7220|554|2038|2|5|11|2|2038|554|7220|Tuesday|2038Q2|N|N|N|2465545|2465664|2465190|2465465|N|N|N|N|N| +2465556|AAAAAAAAEBPJFCAA|2038-05-12|1660|7220|554|2038|3|5|12|2|2038|554|7220|Wednesday|2038Q2|N|N|N|2465545|2465664|2465191|2465466|N|N|N|N|N| +2465557|AAAAAAAAFBPJFCAA|2038-05-13|1660|7220|554|2038|4|5|13|2|2038|554|7220|Thursday|2038Q2|N|N|N|2465545|2465664|2465192|2465467|N|N|N|N|N| +2465558|AAAAAAAAGBPJFCAA|2038-05-14|1660|7220|554|2038|5|5|14|2|2038|554|7220|Friday|2038Q2|N|Y|N|2465545|2465664|2465193|2465468|N|N|N|N|N| +2465559|AAAAAAAAHBPJFCAA|2038-05-15|1660|7220|554|2038|6|5|15|2|2038|554|7220|Saturday|2038Q2|N|Y|N|2465545|2465664|2465194|2465469|N|N|N|N|N| +2465560|AAAAAAAAIBPJFCAA|2038-05-16|1660|7220|554|2038|0|5|16|2|2038|554|7220|Sunday|2038Q2|N|N|N|2465545|2465664|2465195|2465470|N|N|N|N|N| +2465561|AAAAAAAAJBPJFCAA|2038-05-17|1660|7220|554|2038|1|5|17|2|2038|554|7220|Monday|2038Q2|N|N|N|2465545|2465664|2465196|2465471|N|N|N|N|N| +2465562|AAAAAAAAKBPJFCAA|2038-05-18|1660|7221|554|2038|2|5|18|2|2038|554|7221|Tuesday|2038Q2|N|N|N|2465545|2465664|2465197|2465472|N|N|N|N|N| +2465563|AAAAAAAALBPJFCAA|2038-05-19|1660|7221|554|2038|3|5|19|2|2038|554|7221|Wednesday|2038Q2|N|N|N|2465545|2465664|2465198|2465473|N|N|N|N|N| +2465564|AAAAAAAAMBPJFCAA|2038-05-20|1660|7221|554|2038|4|5|20|2|2038|554|7221|Thursday|2038Q2|N|N|N|2465545|2465664|2465199|2465474|N|N|N|N|N| +2465565|AAAAAAAANBPJFCAA|2038-05-21|1660|7221|554|2038|5|5|21|2|2038|554|7221|Friday|2038Q2|N|Y|N|2465545|2465664|2465200|2465475|N|N|N|N|N| +2465566|AAAAAAAAOBPJFCAA|2038-05-22|1660|7221|554|2038|6|5|22|2|2038|554|7221|Saturday|2038Q2|N|Y|N|2465545|2465664|2465201|2465476|N|N|N|N|N| +2465567|AAAAAAAAPBPJFCAA|2038-05-23|1660|7221|554|2038|0|5|23|2|2038|554|7221|Sunday|2038Q2|N|N|N|2465545|2465664|2465202|2465477|N|N|N|N|N| +2465568|AAAAAAAAACPJFCAA|2038-05-24|1660|7221|554|2038|1|5|24|2|2038|554|7221|Monday|2038Q2|N|N|N|2465545|2465664|2465203|2465478|N|N|N|N|N| +2465569|AAAAAAAABCPJFCAA|2038-05-25|1660|7222|554|2038|2|5|25|2|2038|554|7222|Tuesday|2038Q2|N|N|N|2465545|2465664|2465204|2465479|N|N|N|N|N| +2465570|AAAAAAAACCPJFCAA|2038-05-26|1660|7222|554|2038|3|5|26|2|2038|554|7222|Wednesday|2038Q2|N|N|N|2465545|2465664|2465205|2465480|N|N|N|N|N| +2465571|AAAAAAAADCPJFCAA|2038-05-27|1660|7222|554|2038|4|5|27|2|2038|554|7222|Thursday|2038Q2|N|N|N|2465545|2465664|2465206|2465481|N|N|N|N|N| +2465572|AAAAAAAAECPJFCAA|2038-05-28|1660|7222|554|2038|5|5|28|2|2038|554|7222|Friday|2038Q2|N|Y|N|2465545|2465664|2465207|2465482|N|N|N|N|N| +2465573|AAAAAAAAFCPJFCAA|2038-05-29|1660|7222|554|2038|6|5|29|2|2038|554|7222|Saturday|2038Q2|N|Y|N|2465545|2465664|2465208|2465483|N|N|N|N|N| +2465574|AAAAAAAAGCPJFCAA|2038-05-30|1660|7222|554|2038|0|5|30|2|2038|554|7222|Sunday|2038Q2|N|N|N|2465545|2465664|2465209|2465484|N|N|N|N|N| +2465575|AAAAAAAAHCPJFCAA|2038-05-31|1660|7222|554|2038|1|5|31|2|2038|554|7222|Monday|2038Q2|N|N|N|2465545|2465664|2465210|2465485|N|N|N|N|N| +2465576|AAAAAAAAICPJFCAA|2038-06-01|1661|7223|555|2038|2|6|1|2|2038|555|7223|Tuesday|2038Q2|N|N|N|2465576|2465726|2465211|2465486|N|N|N|N|N| +2465577|AAAAAAAAJCPJFCAA|2038-06-02|1661|7223|555|2038|3|6|2|2|2038|555|7223|Wednesday|2038Q2|N|N|N|2465576|2465726|2465212|2465487|N|N|N|N|N| +2465578|AAAAAAAAKCPJFCAA|2038-06-03|1661|7223|555|2038|4|6|3|2|2038|555|7223|Thursday|2038Q2|N|N|N|2465576|2465726|2465213|2465488|N|N|N|N|N| +2465579|AAAAAAAALCPJFCAA|2038-06-04|1661|7223|555|2038|5|6|4|2|2038|555|7223|Friday|2038Q2|N|Y|N|2465576|2465726|2465214|2465489|N|N|N|N|N| +2465580|AAAAAAAAMCPJFCAA|2038-06-05|1661|7223|555|2038|6|6|5|2|2038|555|7223|Saturday|2038Q2|N|Y|N|2465576|2465726|2465215|2465490|N|N|N|N|N| +2465581|AAAAAAAANCPJFCAA|2038-06-06|1661|7223|555|2038|0|6|6|2|2038|555|7223|Sunday|2038Q2|N|N|N|2465576|2465726|2465216|2465491|N|N|N|N|N| +2465582|AAAAAAAAOCPJFCAA|2038-06-07|1661|7223|555|2038|1|6|7|2|2038|555|7223|Monday|2038Q2|N|N|N|2465576|2465726|2465217|2465492|N|N|N|N|N| +2465583|AAAAAAAAPCPJFCAA|2038-06-08|1661|7224|555|2038|2|6|8|2|2038|555|7224|Tuesday|2038Q2|N|N|N|2465576|2465726|2465218|2465493|N|N|N|N|N| +2465584|AAAAAAAAADPJFCAA|2038-06-09|1661|7224|555|2038|3|6|9|2|2038|555|7224|Wednesday|2038Q2|N|N|N|2465576|2465726|2465219|2465494|N|N|N|N|N| +2465585|AAAAAAAABDPJFCAA|2038-06-10|1661|7224|555|2038|4|6|10|2|2038|555|7224|Thursday|2038Q2|N|N|N|2465576|2465726|2465220|2465495|N|N|N|N|N| +2465586|AAAAAAAACDPJFCAA|2038-06-11|1661|7224|555|2038|5|6|11|2|2038|555|7224|Friday|2038Q2|N|Y|N|2465576|2465726|2465221|2465496|N|N|N|N|N| +2465587|AAAAAAAADDPJFCAA|2038-06-12|1661|7224|555|2038|6|6|12|2|2038|555|7224|Saturday|2038Q2|N|Y|N|2465576|2465726|2465222|2465497|N|N|N|N|N| +2465588|AAAAAAAAEDPJFCAA|2038-06-13|1661|7224|555|2038|0|6|13|2|2038|555|7224|Sunday|2038Q2|N|N|N|2465576|2465726|2465223|2465498|N|N|N|N|N| +2465589|AAAAAAAAFDPJFCAA|2038-06-14|1661|7224|555|2038|1|6|14|2|2038|555|7224|Monday|2038Q2|N|N|N|2465576|2465726|2465224|2465499|N|N|N|N|N| +2465590|AAAAAAAAGDPJFCAA|2038-06-15|1661|7225|555|2038|2|6|15|2|2038|555|7225|Tuesday|2038Q2|N|N|N|2465576|2465726|2465225|2465500|N|N|N|N|N| +2465591|AAAAAAAAHDPJFCAA|2038-06-16|1661|7225|555|2038|3|6|16|2|2038|555|7225|Wednesday|2038Q2|N|N|N|2465576|2465726|2465226|2465501|N|N|N|N|N| +2465592|AAAAAAAAIDPJFCAA|2038-06-17|1661|7225|555|2038|4|6|17|2|2038|555|7225|Thursday|2038Q2|N|N|N|2465576|2465726|2465227|2465502|N|N|N|N|N| +2465593|AAAAAAAAJDPJFCAA|2038-06-18|1661|7225|555|2038|5|6|18|2|2038|555|7225|Friday|2038Q2|N|Y|N|2465576|2465726|2465228|2465503|N|N|N|N|N| +2465594|AAAAAAAAKDPJFCAA|2038-06-19|1661|7225|555|2038|6|6|19|2|2038|555|7225|Saturday|2038Q2|N|Y|N|2465576|2465726|2465229|2465504|N|N|N|N|N| +2465595|AAAAAAAALDPJFCAA|2038-06-20|1661|7225|555|2038|0|6|20|2|2038|555|7225|Sunday|2038Q2|N|N|N|2465576|2465726|2465230|2465505|N|N|N|N|N| +2465596|AAAAAAAAMDPJFCAA|2038-06-21|1661|7225|555|2038|1|6|21|2|2038|555|7225|Monday|2038Q2|N|N|N|2465576|2465726|2465231|2465506|N|N|N|N|N| +2465597|AAAAAAAANDPJFCAA|2038-06-22|1661|7226|555|2038|2|6|22|2|2038|555|7226|Tuesday|2038Q2|N|N|N|2465576|2465726|2465232|2465507|N|N|N|N|N| +2465598|AAAAAAAAODPJFCAA|2038-06-23|1661|7226|555|2038|3|6|23|2|2038|555|7226|Wednesday|2038Q2|N|N|N|2465576|2465726|2465233|2465508|N|N|N|N|N| +2465599|AAAAAAAAPDPJFCAA|2038-06-24|1661|7226|555|2038|4|6|24|2|2038|555|7226|Thursday|2038Q2|N|N|N|2465576|2465726|2465234|2465509|N|N|N|N|N| +2465600|AAAAAAAAAEPJFCAA|2038-06-25|1661|7226|555|2038|5|6|25|2|2038|555|7226|Friday|2038Q2|N|Y|N|2465576|2465726|2465235|2465510|N|N|N|N|N| +2465601|AAAAAAAABEPJFCAA|2038-06-26|1661|7226|555|2038|6|6|26|2|2038|555|7226|Saturday|2038Q2|N|Y|N|2465576|2465726|2465236|2465511|N|N|N|N|N| +2465602|AAAAAAAACEPJFCAA|2038-06-27|1661|7226|555|2038|0|6|27|2|2038|555|7226|Sunday|2038Q2|N|N|N|2465576|2465726|2465237|2465512|N|N|N|N|N| +2465603|AAAAAAAADEPJFCAA|2038-06-28|1661|7226|555|2038|1|6|28|2|2038|555|7226|Monday|2038Q2|N|N|N|2465576|2465726|2465238|2465513|N|N|N|N|N| +2465604|AAAAAAAAEEPJFCAA|2038-06-29|1661|7227|555|2038|2|6|29|2|2038|555|7227|Tuesday|2038Q2|N|N|N|2465576|2465726|2465239|2465514|N|N|N|N|N| +2465605|AAAAAAAAFEPJFCAA|2038-06-30|1661|7227|555|2038|3|6|30|2|2038|555|7227|Wednesday|2038Q2|N|N|N|2465576|2465726|2465240|2465515|N|N|N|N|N| +2465606|AAAAAAAAGEPJFCAA|2038-07-01|1662|7227|555|2038|4|7|1|2|2038|555|7227|Thursday|2038Q2|N|N|N|2465606|2465786|2465241|2465515|N|N|N|N|N| +2465607|AAAAAAAAHEPJFCAA|2038-07-02|1662|7227|555|2038|5|7|2|3|2038|555|7227|Friday|2038Q3|N|Y|N|2465606|2465786|2465242|2465516|N|N|N|N|N| +2465608|AAAAAAAAIEPJFCAA|2038-07-03|1662|7227|555|2038|6|7|3|3|2038|555|7227|Saturday|2038Q3|N|Y|N|2465606|2465786|2465243|2465517|N|N|N|N|N| +2465609|AAAAAAAAJEPJFCAA|2038-07-04|1662|7227|555|2038|0|7|4|3|2038|555|7227|Sunday|2038Q3|N|N|N|2465606|2465786|2465244|2465518|N|N|N|N|N| +2465610|AAAAAAAAKEPJFCAA|2038-07-05|1662|7227|555|2038|1|7|5|3|2038|555|7227|Monday|2038Q3|Y|N|N|2465606|2465786|2465245|2465519|N|N|N|N|N| +2465611|AAAAAAAALEPJFCAA|2038-07-06|1662|7228|555|2038|2|7|6|3|2038|555|7228|Tuesday|2038Q3|N|N|Y|2465606|2465786|2465246|2465520|N|N|N|N|N| +2465612|AAAAAAAAMEPJFCAA|2038-07-07|1662|7228|555|2038|3|7|7|3|2038|555|7228|Wednesday|2038Q3|N|N|N|2465606|2465786|2465247|2465521|N|N|N|N|N| +2465613|AAAAAAAANEPJFCAA|2038-07-08|1662|7228|555|2038|4|7|8|3|2038|555|7228|Thursday|2038Q3|N|N|N|2465606|2465786|2465248|2465522|N|N|N|N|N| +2465614|AAAAAAAAOEPJFCAA|2038-07-09|1662|7228|555|2038|5|7|9|3|2038|555|7228|Friday|2038Q3|N|Y|N|2465606|2465786|2465249|2465523|N|N|N|N|N| +2465615|AAAAAAAAPEPJFCAA|2038-07-10|1662|7228|555|2038|6|7|10|3|2038|555|7228|Saturday|2038Q3|N|Y|N|2465606|2465786|2465250|2465524|N|N|N|N|N| +2465616|AAAAAAAAAFPJFCAA|2038-07-11|1662|7228|555|2038|0|7|11|3|2038|555|7228|Sunday|2038Q3|N|N|N|2465606|2465786|2465251|2465525|N|N|N|N|N| +2465617|AAAAAAAABFPJFCAA|2038-07-12|1662|7228|555|2038|1|7|12|3|2038|555|7228|Monday|2038Q3|N|N|N|2465606|2465786|2465252|2465526|N|N|N|N|N| +2465618|AAAAAAAACFPJFCAA|2038-07-13|1662|7229|555|2038|2|7|13|3|2038|555|7229|Tuesday|2038Q3|N|N|N|2465606|2465786|2465253|2465527|N|N|N|N|N| +2465619|AAAAAAAADFPJFCAA|2038-07-14|1662|7229|555|2038|3|7|14|3|2038|555|7229|Wednesday|2038Q3|N|N|N|2465606|2465786|2465254|2465528|N|N|N|N|N| +2465620|AAAAAAAAEFPJFCAA|2038-07-15|1662|7229|555|2038|4|7|15|3|2038|555|7229|Thursday|2038Q3|N|N|N|2465606|2465786|2465255|2465529|N|N|N|N|N| +2465621|AAAAAAAAFFPJFCAA|2038-07-16|1662|7229|555|2038|5|7|16|3|2038|555|7229|Friday|2038Q3|N|Y|N|2465606|2465786|2465256|2465530|N|N|N|N|N| +2465622|AAAAAAAAGFPJFCAA|2038-07-17|1662|7229|555|2038|6|7|17|3|2038|555|7229|Saturday|2038Q3|N|Y|N|2465606|2465786|2465257|2465531|N|N|N|N|N| +2465623|AAAAAAAAHFPJFCAA|2038-07-18|1662|7229|555|2038|0|7|18|3|2038|555|7229|Sunday|2038Q3|N|N|N|2465606|2465786|2465258|2465532|N|N|N|N|N| +2465624|AAAAAAAAIFPJFCAA|2038-07-19|1662|7229|555|2038|1|7|19|3|2038|555|7229|Monday|2038Q3|N|N|N|2465606|2465786|2465259|2465533|N|N|N|N|N| +2465625|AAAAAAAAJFPJFCAA|2038-07-20|1662|7230|555|2038|2|7|20|3|2038|555|7230|Tuesday|2038Q3|N|N|N|2465606|2465786|2465260|2465534|N|N|N|N|N| +2465626|AAAAAAAAKFPJFCAA|2038-07-21|1662|7230|555|2038|3|7|21|3|2038|555|7230|Wednesday|2038Q3|N|N|N|2465606|2465786|2465261|2465535|N|N|N|N|N| +2465627|AAAAAAAALFPJFCAA|2038-07-22|1662|7230|555|2038|4|7|22|3|2038|555|7230|Thursday|2038Q3|N|N|N|2465606|2465786|2465262|2465536|N|N|N|N|N| +2465628|AAAAAAAAMFPJFCAA|2038-07-23|1662|7230|555|2038|5|7|23|3|2038|555|7230|Friday|2038Q3|N|Y|N|2465606|2465786|2465263|2465537|N|N|N|N|N| +2465629|AAAAAAAANFPJFCAA|2038-07-24|1662|7230|555|2038|6|7|24|3|2038|555|7230|Saturday|2038Q3|N|Y|N|2465606|2465786|2465264|2465538|N|N|N|N|N| +2465630|AAAAAAAAOFPJFCAA|2038-07-25|1662|7230|555|2038|0|7|25|3|2038|555|7230|Sunday|2038Q3|N|N|N|2465606|2465786|2465265|2465539|N|N|N|N|N| +2465631|AAAAAAAAPFPJFCAA|2038-07-26|1662|7230|555|2038|1|7|26|3|2038|555|7230|Monday|2038Q3|N|N|N|2465606|2465786|2465266|2465540|N|N|N|N|N| +2465632|AAAAAAAAAGPJFCAA|2038-07-27|1662|7231|555|2038|2|7|27|3|2038|555|7231|Tuesday|2038Q3|N|N|N|2465606|2465786|2465267|2465541|N|N|N|N|N| +2465633|AAAAAAAABGPJFCAA|2038-07-28|1662|7231|555|2038|3|7|28|3|2038|555|7231|Wednesday|2038Q3|N|N|N|2465606|2465786|2465268|2465542|N|N|N|N|N| +2465634|AAAAAAAACGPJFCAA|2038-07-29|1662|7231|555|2038|4|7|29|3|2038|555|7231|Thursday|2038Q3|N|N|N|2465606|2465786|2465269|2465543|N|N|N|N|N| +2465635|AAAAAAAADGPJFCAA|2038-07-30|1662|7231|555|2038|5|7|30|3|2038|555|7231|Friday|2038Q3|N|Y|N|2465606|2465786|2465270|2465544|N|N|N|N|N| +2465636|AAAAAAAAEGPJFCAA|2038-07-31|1662|7231|555|2038|6|7|31|3|2038|555|7231|Saturday|2038Q3|N|Y|N|2465606|2465786|2465271|2465545|N|N|N|N|N| +2465637|AAAAAAAAFGPJFCAA|2038-08-01|1663|7231|555|2038|0|8|1|3|2038|555|7231|Sunday|2038Q3|N|N|N|2465637|2465848|2465272|2465546|N|N|N|N|N| +2465638|AAAAAAAAGGPJFCAA|2038-08-02|1663|7231|555|2038|1|8|2|3|2038|555|7231|Monday|2038Q3|N|N|N|2465637|2465848|2465273|2465547|N|N|N|N|N| +2465639|AAAAAAAAHGPJFCAA|2038-08-03|1663|7232|555|2038|2|8|3|3|2038|555|7232|Tuesday|2038Q3|N|N|N|2465637|2465848|2465274|2465548|N|N|N|N|N| +2465640|AAAAAAAAIGPJFCAA|2038-08-04|1663|7232|555|2038|3|8|4|3|2038|555|7232|Wednesday|2038Q3|N|N|N|2465637|2465848|2465275|2465549|N|N|N|N|N| +2465641|AAAAAAAAJGPJFCAA|2038-08-05|1663|7232|555|2038|4|8|5|3|2038|555|7232|Thursday|2038Q3|N|N|N|2465637|2465848|2465276|2465550|N|N|N|N|N| +2465642|AAAAAAAAKGPJFCAA|2038-08-06|1663|7232|555|2038|5|8|6|3|2038|555|7232|Friday|2038Q3|N|Y|N|2465637|2465848|2465277|2465551|N|N|N|N|N| +2465643|AAAAAAAALGPJFCAA|2038-08-07|1663|7232|555|2038|6|8|7|3|2038|555|7232|Saturday|2038Q3|N|Y|N|2465637|2465848|2465278|2465552|N|N|N|N|N| +2465644|AAAAAAAAMGPJFCAA|2038-08-08|1663|7232|555|2038|0|8|8|3|2038|555|7232|Sunday|2038Q3|N|N|N|2465637|2465848|2465279|2465553|N|N|N|N|N| +2465645|AAAAAAAANGPJFCAA|2038-08-09|1663|7232|555|2038|1|8|9|3|2038|555|7232|Monday|2038Q3|N|N|N|2465637|2465848|2465280|2465554|N|N|N|N|N| +2465646|AAAAAAAAOGPJFCAA|2038-08-10|1663|7233|555|2038|2|8|10|3|2038|555|7233|Tuesday|2038Q3|N|N|N|2465637|2465848|2465281|2465555|N|N|N|N|N| +2465647|AAAAAAAAPGPJFCAA|2038-08-11|1663|7233|555|2038|3|8|11|3|2038|555|7233|Wednesday|2038Q3|N|N|N|2465637|2465848|2465282|2465556|N|N|N|N|N| +2465648|AAAAAAAAAHPJFCAA|2038-08-12|1663|7233|555|2038|4|8|12|3|2038|555|7233|Thursday|2038Q3|N|N|N|2465637|2465848|2465283|2465557|N|N|N|N|N| +2465649|AAAAAAAABHPJFCAA|2038-08-13|1663|7233|555|2038|5|8|13|3|2038|555|7233|Friday|2038Q3|N|Y|N|2465637|2465848|2465284|2465558|N|N|N|N|N| +2465650|AAAAAAAACHPJFCAA|2038-08-14|1663|7233|555|2038|6|8|14|3|2038|555|7233|Saturday|2038Q3|N|Y|N|2465637|2465848|2465285|2465559|N|N|N|N|N| +2465651|AAAAAAAADHPJFCAA|2038-08-15|1663|7233|555|2038|0|8|15|3|2038|555|7233|Sunday|2038Q3|N|N|N|2465637|2465848|2465286|2465560|N|N|N|N|N| +2465652|AAAAAAAAEHPJFCAA|2038-08-16|1663|7233|555|2038|1|8|16|3|2038|555|7233|Monday|2038Q3|N|N|N|2465637|2465848|2465287|2465561|N|N|N|N|N| +2465653|AAAAAAAAFHPJFCAA|2038-08-17|1663|7234|555|2038|2|8|17|3|2038|555|7234|Tuesday|2038Q3|N|N|N|2465637|2465848|2465288|2465562|N|N|N|N|N| +2465654|AAAAAAAAGHPJFCAA|2038-08-18|1663|7234|555|2038|3|8|18|3|2038|555|7234|Wednesday|2038Q3|N|N|N|2465637|2465848|2465289|2465563|N|N|N|N|N| +2465655|AAAAAAAAHHPJFCAA|2038-08-19|1663|7234|555|2038|4|8|19|3|2038|555|7234|Thursday|2038Q3|N|N|N|2465637|2465848|2465290|2465564|N|N|N|N|N| +2465656|AAAAAAAAIHPJFCAA|2038-08-20|1663|7234|555|2038|5|8|20|3|2038|555|7234|Friday|2038Q3|N|Y|N|2465637|2465848|2465291|2465565|N|N|N|N|N| +2465657|AAAAAAAAJHPJFCAA|2038-08-21|1663|7234|555|2038|6|8|21|3|2038|555|7234|Saturday|2038Q3|N|Y|N|2465637|2465848|2465292|2465566|N|N|N|N|N| +2465658|AAAAAAAAKHPJFCAA|2038-08-22|1663|7234|555|2038|0|8|22|3|2038|555|7234|Sunday|2038Q3|N|N|N|2465637|2465848|2465293|2465567|N|N|N|N|N| +2465659|AAAAAAAALHPJFCAA|2038-08-23|1663|7234|555|2038|1|8|23|3|2038|555|7234|Monday|2038Q3|N|N|N|2465637|2465848|2465294|2465568|N|N|N|N|N| +2465660|AAAAAAAAMHPJFCAA|2038-08-24|1663|7235|555|2038|2|8|24|3|2038|555|7235|Tuesday|2038Q3|N|N|N|2465637|2465848|2465295|2465569|N|N|N|N|N| +2465661|AAAAAAAANHPJFCAA|2038-08-25|1663|7235|555|2038|3|8|25|3|2038|555|7235|Wednesday|2038Q3|N|N|N|2465637|2465848|2465296|2465570|N|N|N|N|N| +2465662|AAAAAAAAOHPJFCAA|2038-08-26|1663|7235|555|2038|4|8|26|3|2038|555|7235|Thursday|2038Q3|N|N|N|2465637|2465848|2465297|2465571|N|N|N|N|N| +2465663|AAAAAAAAPHPJFCAA|2038-08-27|1663|7235|555|2038|5|8|27|3|2038|555|7235|Friday|2038Q3|N|Y|N|2465637|2465848|2465298|2465572|N|N|N|N|N| +2465664|AAAAAAAAAIPJFCAA|2038-08-28|1663|7235|555|2038|6|8|28|3|2038|555|7235|Saturday|2038Q3|N|Y|N|2465637|2465848|2465299|2465573|N|N|N|N|N| +2465665|AAAAAAAABIPJFCAA|2038-08-29|1663|7235|555|2038|0|8|29|3|2038|555|7235|Sunday|2038Q3|N|N|N|2465637|2465848|2465300|2465574|N|N|N|N|N| +2465666|AAAAAAAACIPJFCAA|2038-08-30|1663|7235|555|2038|1|8|30|3|2038|555|7235|Monday|2038Q3|N|N|N|2465637|2465848|2465301|2465575|N|N|N|N|N| +2465667|AAAAAAAADIPJFCAA|2038-08-31|1663|7236|555|2038|2|8|31|3|2038|555|7236|Tuesday|2038Q3|N|N|N|2465637|2465848|2465302|2465576|N|N|N|N|N| +2465668|AAAAAAAAEIPJFCAA|2038-09-01|1664|7236|556|2038|3|9|1|3|2038|556|7236|Wednesday|2038Q3|N|N|N|2465668|2465910|2465303|2465577|N|N|N|N|N| +2465669|AAAAAAAAFIPJFCAA|2038-09-02|1664|7236|556|2038|4|9|2|3|2038|556|7236|Thursday|2038Q3|N|N|N|2465668|2465910|2465304|2465578|N|N|N|N|N| +2465670|AAAAAAAAGIPJFCAA|2038-09-03|1664|7236|556|2038|5|9|3|3|2038|556|7236|Friday|2038Q3|N|Y|N|2465668|2465910|2465305|2465579|N|N|N|N|N| +2465671|AAAAAAAAHIPJFCAA|2038-09-04|1664|7236|556|2038|6|9|4|3|2038|556|7236|Saturday|2038Q3|N|Y|N|2465668|2465910|2465306|2465580|N|N|N|N|N| +2465672|AAAAAAAAIIPJFCAA|2038-09-05|1664|7236|556|2038|0|9|5|3|2038|556|7236|Sunday|2038Q3|N|N|N|2465668|2465910|2465307|2465581|N|N|N|N|N| +2465673|AAAAAAAAJIPJFCAA|2038-09-06|1664|7236|556|2038|1|9|6|3|2038|556|7236|Monday|2038Q3|N|N|N|2465668|2465910|2465308|2465582|N|N|N|N|N| +2465674|AAAAAAAAKIPJFCAA|2038-09-07|1664|7237|556|2038|2|9|7|3|2038|556|7237|Tuesday|2038Q3|N|N|N|2465668|2465910|2465309|2465583|N|N|N|N|N| +2465675|AAAAAAAALIPJFCAA|2038-09-08|1664|7237|556|2038|3|9|8|3|2038|556|7237|Wednesday|2038Q3|N|N|N|2465668|2465910|2465310|2465584|N|N|N|N|N| +2465676|AAAAAAAAMIPJFCAA|2038-09-09|1664|7237|556|2038|4|9|9|3|2038|556|7237|Thursday|2038Q3|N|N|N|2465668|2465910|2465311|2465585|N|N|N|N|N| +2465677|AAAAAAAANIPJFCAA|2038-09-10|1664|7237|556|2038|5|9|10|3|2038|556|7237|Friday|2038Q3|N|Y|N|2465668|2465910|2465312|2465586|N|N|N|N|N| +2465678|AAAAAAAAOIPJFCAA|2038-09-11|1664|7237|556|2038|6|9|11|3|2038|556|7237|Saturday|2038Q3|N|Y|N|2465668|2465910|2465313|2465587|N|N|N|N|N| +2465679|AAAAAAAAPIPJFCAA|2038-09-12|1664|7237|556|2038|0|9|12|3|2038|556|7237|Sunday|2038Q3|N|N|N|2465668|2465910|2465314|2465588|N|N|N|N|N| +2465680|AAAAAAAAAJPJFCAA|2038-09-13|1664|7237|556|2038|1|9|13|3|2038|556|7237|Monday|2038Q3|N|N|N|2465668|2465910|2465315|2465589|N|N|N|N|N| +2465681|AAAAAAAABJPJFCAA|2038-09-14|1664|7238|556|2038|2|9|14|3|2038|556|7238|Tuesday|2038Q3|N|N|N|2465668|2465910|2465316|2465590|N|N|N|N|N| +2465682|AAAAAAAACJPJFCAA|2038-09-15|1664|7238|556|2038|3|9|15|3|2038|556|7238|Wednesday|2038Q3|N|N|N|2465668|2465910|2465317|2465591|N|N|N|N|N| +2465683|AAAAAAAADJPJFCAA|2038-09-16|1664|7238|556|2038|4|9|16|3|2038|556|7238|Thursday|2038Q3|N|N|N|2465668|2465910|2465318|2465592|N|N|N|N|N| +2465684|AAAAAAAAEJPJFCAA|2038-09-17|1664|7238|556|2038|5|9|17|3|2038|556|7238|Friday|2038Q3|N|Y|N|2465668|2465910|2465319|2465593|N|N|N|N|N| +2465685|AAAAAAAAFJPJFCAA|2038-09-18|1664|7238|556|2038|6|9|18|3|2038|556|7238|Saturday|2038Q3|N|Y|N|2465668|2465910|2465320|2465594|N|N|N|N|N| +2465686|AAAAAAAAGJPJFCAA|2038-09-19|1664|7238|556|2038|0|9|19|3|2038|556|7238|Sunday|2038Q3|N|N|N|2465668|2465910|2465321|2465595|N|N|N|N|N| +2465687|AAAAAAAAHJPJFCAA|2038-09-20|1664|7238|556|2038|1|9|20|3|2038|556|7238|Monday|2038Q3|N|N|N|2465668|2465910|2465322|2465596|N|N|N|N|N| +2465688|AAAAAAAAIJPJFCAA|2038-09-21|1664|7239|556|2038|2|9|21|3|2038|556|7239|Tuesday|2038Q3|N|N|N|2465668|2465910|2465323|2465597|N|N|N|N|N| +2465689|AAAAAAAAJJPJFCAA|2038-09-22|1664|7239|556|2038|3|9|22|3|2038|556|7239|Wednesday|2038Q3|N|N|N|2465668|2465910|2465324|2465598|N|N|N|N|N| +2465690|AAAAAAAAKJPJFCAA|2038-09-23|1664|7239|556|2038|4|9|23|3|2038|556|7239|Thursday|2038Q3|N|N|N|2465668|2465910|2465325|2465599|N|N|N|N|N| +2465691|AAAAAAAALJPJFCAA|2038-09-24|1664|7239|556|2038|5|9|24|3|2038|556|7239|Friday|2038Q3|N|Y|N|2465668|2465910|2465326|2465600|N|N|N|N|N| +2465692|AAAAAAAAMJPJFCAA|2038-09-25|1664|7239|556|2038|6|9|25|3|2038|556|7239|Saturday|2038Q3|N|Y|N|2465668|2465910|2465327|2465601|N|N|N|N|N| +2465693|AAAAAAAANJPJFCAA|2038-09-26|1664|7239|556|2038|0|9|26|3|2038|556|7239|Sunday|2038Q3|N|N|N|2465668|2465910|2465328|2465602|N|N|N|N|N| +2465694|AAAAAAAAOJPJFCAA|2038-09-27|1664|7239|556|2038|1|9|27|3|2038|556|7239|Monday|2038Q3|N|N|N|2465668|2465910|2465329|2465603|N|N|N|N|N| +2465695|AAAAAAAAPJPJFCAA|2038-09-28|1664|7240|556|2038|2|9|28|3|2038|556|7240|Tuesday|2038Q3|N|N|N|2465668|2465910|2465330|2465604|N|N|N|N|N| +2465696|AAAAAAAAAKPJFCAA|2038-09-29|1664|7240|556|2038|3|9|29|3|2038|556|7240|Wednesday|2038Q3|N|N|N|2465668|2465910|2465331|2465605|N|N|N|N|N| +2465697|AAAAAAAABKPJFCAA|2038-09-30|1664|7240|556|2038|4|9|30|3|2038|556|7240|Thursday|2038Q3|N|N|N|2465668|2465910|2465332|2465606|N|N|N|N|N| +2465698|AAAAAAAACKPJFCAA|2038-10-01|1665|7240|556|2038|5|10|1|3|2038|556|7240|Friday|2038Q3|N|Y|N|2465698|2465970|2465333|2465606|N|N|N|N|N| +2465699|AAAAAAAADKPJFCAA|2038-10-02|1665|7240|556|2038|6|10|2|4|2038|556|7240|Saturday|2038Q4|N|Y|N|2465698|2465970|2465334|2465607|N|N|N|N|N| +2465700|AAAAAAAAEKPJFCAA|2038-10-03|1665|7240|556|2038|0|10|3|4|2038|556|7240|Sunday|2038Q4|N|N|N|2465698|2465970|2465335|2465608|N|N|N|N|N| +2465701|AAAAAAAAFKPJFCAA|2038-10-04|1665|7240|556|2038|1|10|4|4|2038|556|7240|Monday|2038Q4|N|N|N|2465698|2465970|2465336|2465609|N|N|N|N|N| +2465702|AAAAAAAAGKPJFCAA|2038-10-05|1665|7241|556|2038|2|10|5|4|2038|556|7241|Tuesday|2038Q4|N|N|N|2465698|2465970|2465337|2465610|N|N|N|N|N| +2465703|AAAAAAAAHKPJFCAA|2038-10-06|1665|7241|556|2038|3|10|6|4|2038|556|7241|Wednesday|2038Q4|N|N|N|2465698|2465970|2465338|2465611|N|N|N|N|N| +2465704|AAAAAAAAIKPJFCAA|2038-10-07|1665|7241|556|2038|4|10|7|4|2038|556|7241|Thursday|2038Q4|N|N|N|2465698|2465970|2465339|2465612|N|N|N|N|N| +2465705|AAAAAAAAJKPJFCAA|2038-10-08|1665|7241|556|2038|5|10|8|4|2038|556|7241|Friday|2038Q4|N|Y|N|2465698|2465970|2465340|2465613|N|N|N|N|N| +2465706|AAAAAAAAKKPJFCAA|2038-10-09|1665|7241|556|2038|6|10|9|4|2038|556|7241|Saturday|2038Q4|N|Y|N|2465698|2465970|2465341|2465614|N|N|N|N|N| +2465707|AAAAAAAALKPJFCAA|2038-10-10|1665|7241|556|2038|0|10|10|4|2038|556|7241|Sunday|2038Q4|N|N|N|2465698|2465970|2465342|2465615|N|N|N|N|N| +2465708|AAAAAAAAMKPJFCAA|2038-10-11|1665|7241|556|2038|1|10|11|4|2038|556|7241|Monday|2038Q4|N|N|N|2465698|2465970|2465343|2465616|N|N|N|N|N| +2465709|AAAAAAAANKPJFCAA|2038-10-12|1665|7242|556|2038|2|10|12|4|2038|556|7242|Tuesday|2038Q4|N|N|N|2465698|2465970|2465344|2465617|N|N|N|N|N| +2465710|AAAAAAAAOKPJFCAA|2038-10-13|1665|7242|556|2038|3|10|13|4|2038|556|7242|Wednesday|2038Q4|N|N|N|2465698|2465970|2465345|2465618|N|N|N|N|N| +2465711|AAAAAAAAPKPJFCAA|2038-10-14|1665|7242|556|2038|4|10|14|4|2038|556|7242|Thursday|2038Q4|N|N|N|2465698|2465970|2465346|2465619|N|N|N|N|N| +2465712|AAAAAAAAALPJFCAA|2038-10-15|1665|7242|556|2038|5|10|15|4|2038|556|7242|Friday|2038Q4|N|Y|N|2465698|2465970|2465347|2465620|N|N|N|N|N| +2465713|AAAAAAAABLPJFCAA|2038-10-16|1665|7242|556|2038|6|10|16|4|2038|556|7242|Saturday|2038Q4|N|Y|N|2465698|2465970|2465348|2465621|N|N|N|N|N| +2465714|AAAAAAAACLPJFCAA|2038-10-17|1665|7242|556|2038|0|10|17|4|2038|556|7242|Sunday|2038Q4|N|N|N|2465698|2465970|2465349|2465622|N|N|N|N|N| +2465715|AAAAAAAADLPJFCAA|2038-10-18|1665|7242|556|2038|1|10|18|4|2038|556|7242|Monday|2038Q4|N|N|N|2465698|2465970|2465350|2465623|N|N|N|N|N| +2465716|AAAAAAAAELPJFCAA|2038-10-19|1665|7243|556|2038|2|10|19|4|2038|556|7243|Tuesday|2038Q4|N|N|N|2465698|2465970|2465351|2465624|N|N|N|N|N| +2465717|AAAAAAAAFLPJFCAA|2038-10-20|1665|7243|556|2038|3|10|20|4|2038|556|7243|Wednesday|2038Q4|N|N|N|2465698|2465970|2465352|2465625|N|N|N|N|N| +2465718|AAAAAAAAGLPJFCAA|2038-10-21|1665|7243|556|2038|4|10|21|4|2038|556|7243|Thursday|2038Q4|N|N|N|2465698|2465970|2465353|2465626|N|N|N|N|N| +2465719|AAAAAAAAHLPJFCAA|2038-10-22|1665|7243|556|2038|5|10|22|4|2038|556|7243|Friday|2038Q4|N|Y|N|2465698|2465970|2465354|2465627|N|N|N|N|N| +2465720|AAAAAAAAILPJFCAA|2038-10-23|1665|7243|556|2038|6|10|23|4|2038|556|7243|Saturday|2038Q4|N|Y|N|2465698|2465970|2465355|2465628|N|N|N|N|N| +2465721|AAAAAAAAJLPJFCAA|2038-10-24|1665|7243|556|2038|0|10|24|4|2038|556|7243|Sunday|2038Q4|N|N|N|2465698|2465970|2465356|2465629|N|N|N|N|N| +2465722|AAAAAAAAKLPJFCAA|2038-10-25|1665|7243|556|2038|1|10|25|4|2038|556|7243|Monday|2038Q4|N|N|N|2465698|2465970|2465357|2465630|N|N|N|N|N| +2465723|AAAAAAAALLPJFCAA|2038-10-26|1665|7244|556|2038|2|10|26|4|2038|556|7244|Tuesday|2038Q4|N|N|N|2465698|2465970|2465358|2465631|N|N|N|N|N| +2465724|AAAAAAAAMLPJFCAA|2038-10-27|1665|7244|556|2038|3|10|27|4|2038|556|7244|Wednesday|2038Q4|N|N|N|2465698|2465970|2465359|2465632|N|N|N|N|N| +2465725|AAAAAAAANLPJFCAA|2038-10-28|1665|7244|556|2038|4|10|28|4|2038|556|7244|Thursday|2038Q4|N|N|N|2465698|2465970|2465360|2465633|N|N|N|N|N| +2465726|AAAAAAAAOLPJFCAA|2038-10-29|1665|7244|556|2038|5|10|29|4|2038|556|7244|Friday|2038Q4|N|Y|N|2465698|2465970|2465361|2465634|N|N|N|N|N| +2465727|AAAAAAAAPLPJFCAA|2038-10-30|1665|7244|556|2038|6|10|30|4|2038|556|7244|Saturday|2038Q4|N|Y|N|2465698|2465970|2465362|2465635|N|N|N|N|N| +2465728|AAAAAAAAAMPJFCAA|2038-10-31|1665|7244|556|2038|0|10|31|4|2038|556|7244|Sunday|2038Q4|N|N|N|2465698|2465970|2465363|2465636|N|N|N|N|N| +2465729|AAAAAAAABMPJFCAA|2038-11-01|1666|7244|556|2038|1|11|1|4|2038|556|7244|Monday|2038Q4|N|N|N|2465729|2466032|2465364|2465637|N|N|N|N|N| +2465730|AAAAAAAACMPJFCAA|2038-11-02|1666|7245|556|2038|2|11|2|4|2038|556|7245|Tuesday|2038Q4|N|N|N|2465729|2466032|2465365|2465638|N|N|N|N|N| +2465731|AAAAAAAADMPJFCAA|2038-11-03|1666|7245|556|2038|3|11|3|4|2038|556|7245|Wednesday|2038Q4|N|N|N|2465729|2466032|2465366|2465639|N|N|N|N|N| +2465732|AAAAAAAAEMPJFCAA|2038-11-04|1666|7245|556|2038|4|11|4|4|2038|556|7245|Thursday|2038Q4|N|N|N|2465729|2466032|2465367|2465640|N|N|N|N|N| +2465733|AAAAAAAAFMPJFCAA|2038-11-05|1666|7245|556|2038|5|11|5|4|2038|556|7245|Friday|2038Q4|N|Y|N|2465729|2466032|2465368|2465641|N|N|N|N|N| +2465734|AAAAAAAAGMPJFCAA|2038-11-06|1666|7245|556|2038|6|11|6|4|2038|556|7245|Saturday|2038Q4|N|Y|N|2465729|2466032|2465369|2465642|N|N|N|N|N| +2465735|AAAAAAAAHMPJFCAA|2038-11-07|1666|7245|556|2038|0|11|7|4|2038|556|7245|Sunday|2038Q4|N|N|N|2465729|2466032|2465370|2465643|N|N|N|N|N| +2465736|AAAAAAAAIMPJFCAA|2038-11-08|1666|7245|556|2038|1|11|8|4|2038|556|7245|Monday|2038Q4|N|N|N|2465729|2466032|2465371|2465644|N|N|N|N|N| +2465737|AAAAAAAAJMPJFCAA|2038-11-09|1666|7246|556|2038|2|11|9|4|2038|556|7246|Tuesday|2038Q4|N|N|N|2465729|2466032|2465372|2465645|N|N|N|N|N| +2465738|AAAAAAAAKMPJFCAA|2038-11-10|1666|7246|556|2038|3|11|10|4|2038|556|7246|Wednesday|2038Q4|N|N|N|2465729|2466032|2465373|2465646|N|N|N|N|N| +2465739|AAAAAAAALMPJFCAA|2038-11-11|1666|7246|556|2038|4|11|11|4|2038|556|7246|Thursday|2038Q4|N|N|N|2465729|2466032|2465374|2465647|N|N|N|N|N| +2465740|AAAAAAAAMMPJFCAA|2038-11-12|1666|7246|556|2038|5|11|12|4|2038|556|7246|Friday|2038Q4|N|Y|N|2465729|2466032|2465375|2465648|N|N|N|N|N| +2465741|AAAAAAAANMPJFCAA|2038-11-13|1666|7246|556|2038|6|11|13|4|2038|556|7246|Saturday|2038Q4|N|Y|N|2465729|2466032|2465376|2465649|N|N|N|N|N| +2465742|AAAAAAAAOMPJFCAA|2038-11-14|1666|7246|556|2038|0|11|14|4|2038|556|7246|Sunday|2038Q4|N|N|N|2465729|2466032|2465377|2465650|N|N|N|N|N| +2465743|AAAAAAAAPMPJFCAA|2038-11-15|1666|7246|556|2038|1|11|15|4|2038|556|7246|Monday|2038Q4|N|N|N|2465729|2466032|2465378|2465651|N|N|N|N|N| +2465744|AAAAAAAAANPJFCAA|2038-11-16|1666|7247|556|2038|2|11|16|4|2038|556|7247|Tuesday|2038Q4|N|N|N|2465729|2466032|2465379|2465652|N|N|N|N|N| +2465745|AAAAAAAABNPJFCAA|2038-11-17|1666|7247|556|2038|3|11|17|4|2038|556|7247|Wednesday|2038Q4|N|N|N|2465729|2466032|2465380|2465653|N|N|N|N|N| +2465746|AAAAAAAACNPJFCAA|2038-11-18|1666|7247|556|2038|4|11|18|4|2038|556|7247|Thursday|2038Q4|N|N|N|2465729|2466032|2465381|2465654|N|N|N|N|N| +2465747|AAAAAAAADNPJFCAA|2038-11-19|1666|7247|556|2038|5|11|19|4|2038|556|7247|Friday|2038Q4|N|Y|N|2465729|2466032|2465382|2465655|N|N|N|N|N| +2465748|AAAAAAAAENPJFCAA|2038-11-20|1666|7247|556|2038|6|11|20|4|2038|556|7247|Saturday|2038Q4|N|Y|N|2465729|2466032|2465383|2465656|N|N|N|N|N| +2465749|AAAAAAAAFNPJFCAA|2038-11-21|1666|7247|556|2038|0|11|21|4|2038|556|7247|Sunday|2038Q4|N|N|N|2465729|2466032|2465384|2465657|N|N|N|N|N| +2465750|AAAAAAAAGNPJFCAA|2038-11-22|1666|7247|556|2038|1|11|22|4|2038|556|7247|Monday|2038Q4|N|N|N|2465729|2466032|2465385|2465658|N|N|N|N|N| +2465751|AAAAAAAAHNPJFCAA|2038-11-23|1666|7248|556|2038|2|11|23|4|2038|556|7248|Tuesday|2038Q4|N|N|N|2465729|2466032|2465386|2465659|N|N|N|N|N| +2465752|AAAAAAAAINPJFCAA|2038-11-24|1666|7248|556|2038|3|11|24|4|2038|556|7248|Wednesday|2038Q4|N|N|N|2465729|2466032|2465387|2465660|N|N|N|N|N| +2465753|AAAAAAAAJNPJFCAA|2038-11-25|1666|7248|556|2038|4|11|25|4|2038|556|7248|Thursday|2038Q4|N|N|N|2465729|2466032|2465388|2465661|N|N|N|N|N| +2465754|AAAAAAAAKNPJFCAA|2038-11-26|1666|7248|556|2038|5|11|26|4|2038|556|7248|Friday|2038Q4|N|Y|N|2465729|2466032|2465389|2465662|N|N|N|N|N| +2465755|AAAAAAAALNPJFCAA|2038-11-27|1666|7248|556|2038|6|11|27|4|2038|556|7248|Saturday|2038Q4|N|Y|N|2465729|2466032|2465390|2465663|N|N|N|N|N| +2465756|AAAAAAAAMNPJFCAA|2038-11-28|1666|7248|556|2038|0|11|28|4|2038|556|7248|Sunday|2038Q4|N|N|N|2465729|2466032|2465391|2465664|N|N|N|N|N| +2465757|AAAAAAAANNPJFCAA|2038-11-29|1666|7248|556|2038|1|11|29|4|2038|556|7248|Monday|2038Q4|N|N|N|2465729|2466032|2465392|2465665|N|N|N|N|N| +2465758|AAAAAAAAONPJFCAA|2038-11-30|1666|7249|556|2038|2|11|30|4|2038|556|7249|Tuesday|2038Q4|N|N|N|2465729|2466032|2465393|2465666|N|N|N|N|N| +2465759|AAAAAAAAPNPJFCAA|2038-12-01|1667|7249|557|2038|3|12|1|4|2038|557|7249|Wednesday|2038Q4|N|N|N|2465759|2466092|2465394|2465667|N|N|N|N|N| +2465760|AAAAAAAAAOPJFCAA|2038-12-02|1667|7249|557|2038|4|12|2|4|2038|557|7249|Thursday|2038Q4|N|N|N|2465759|2466092|2465395|2465668|N|N|N|N|N| +2465761|AAAAAAAABOPJFCAA|2038-12-03|1667|7249|557|2038|5|12|3|4|2038|557|7249|Friday|2038Q4|N|Y|N|2465759|2466092|2465396|2465669|N|N|N|N|N| +2465762|AAAAAAAACOPJFCAA|2038-12-04|1667|7249|557|2038|6|12|4|4|2038|557|7249|Saturday|2038Q4|N|Y|N|2465759|2466092|2465397|2465670|N|N|N|N|N| +2465763|AAAAAAAADOPJFCAA|2038-12-05|1667|7249|557|2038|0|12|5|4|2038|557|7249|Sunday|2038Q4|N|N|N|2465759|2466092|2465398|2465671|N|N|N|N|N| +2465764|AAAAAAAAEOPJFCAA|2038-12-06|1667|7249|557|2038|1|12|6|4|2038|557|7249|Monday|2038Q4|N|N|N|2465759|2466092|2465399|2465672|N|N|N|N|N| +2465765|AAAAAAAAFOPJFCAA|2038-12-07|1667|7250|557|2038|2|12|7|4|2038|557|7250|Tuesday|2038Q4|N|N|N|2465759|2466092|2465400|2465673|N|N|N|N|N| +2465766|AAAAAAAAGOPJFCAA|2038-12-08|1667|7250|557|2038|3|12|8|4|2038|557|7250|Wednesday|2038Q4|N|N|N|2465759|2466092|2465401|2465674|N|N|N|N|N| +2465767|AAAAAAAAHOPJFCAA|2038-12-09|1667|7250|557|2038|4|12|9|4|2038|557|7250|Thursday|2038Q4|N|N|N|2465759|2466092|2465402|2465675|N|N|N|N|N| +2465768|AAAAAAAAIOPJFCAA|2038-12-10|1667|7250|557|2038|5|12|10|4|2038|557|7250|Friday|2038Q4|N|Y|N|2465759|2466092|2465403|2465676|N|N|N|N|N| +2465769|AAAAAAAAJOPJFCAA|2038-12-11|1667|7250|557|2038|6|12|11|4|2038|557|7250|Saturday|2038Q4|N|Y|N|2465759|2466092|2465404|2465677|N|N|N|N|N| +2465770|AAAAAAAAKOPJFCAA|2038-12-12|1667|7250|557|2038|0|12|12|4|2038|557|7250|Sunday|2038Q4|N|N|N|2465759|2466092|2465405|2465678|N|N|N|N|N| +2465771|AAAAAAAALOPJFCAA|2038-12-13|1667|7250|557|2038|1|12|13|4|2038|557|7250|Monday|2038Q4|N|N|N|2465759|2466092|2465406|2465679|N|N|N|N|N| +2465772|AAAAAAAAMOPJFCAA|2038-12-14|1667|7251|557|2038|2|12|14|4|2038|557|7251|Tuesday|2038Q4|N|N|N|2465759|2466092|2465407|2465680|N|N|N|N|N| +2465773|AAAAAAAANOPJFCAA|2038-12-15|1667|7251|557|2038|3|12|15|4|2038|557|7251|Wednesday|2038Q4|N|N|N|2465759|2466092|2465408|2465681|N|N|N|N|N| +2465774|AAAAAAAAOOPJFCAA|2038-12-16|1667|7251|557|2038|4|12|16|4|2038|557|7251|Thursday|2038Q4|N|N|N|2465759|2466092|2465409|2465682|N|N|N|N|N| +2465775|AAAAAAAAPOPJFCAA|2038-12-17|1667|7251|557|2038|5|12|17|4|2038|557|7251|Friday|2038Q4|N|Y|N|2465759|2466092|2465410|2465683|N|N|N|N|N| +2465776|AAAAAAAAAPPJFCAA|2038-12-18|1667|7251|557|2038|6|12|18|4|2038|557|7251|Saturday|2038Q4|N|Y|N|2465759|2466092|2465411|2465684|N|N|N|N|N| +2465777|AAAAAAAABPPJFCAA|2038-12-19|1667|7251|557|2038|0|12|19|4|2038|557|7251|Sunday|2038Q4|N|N|N|2465759|2466092|2465412|2465685|N|N|N|N|N| +2465778|AAAAAAAACPPJFCAA|2038-12-20|1667|7251|557|2038|1|12|20|4|2038|557|7251|Monday|2038Q4|N|N|N|2465759|2466092|2465413|2465686|N|N|N|N|N| +2465779|AAAAAAAADPPJFCAA|2038-12-21|1667|7252|557|2038|2|12|21|4|2038|557|7252|Tuesday|2038Q4|N|N|N|2465759|2466092|2465414|2465687|N|N|N|N|N| +2465780|AAAAAAAAEPPJFCAA|2038-12-22|1667|7252|557|2038|3|12|22|4|2038|557|7252|Wednesday|2038Q4|N|N|N|2465759|2466092|2465415|2465688|N|N|N|N|N| +2465781|AAAAAAAAFPPJFCAA|2038-12-23|1667|7252|557|2038|4|12|23|4|2038|557|7252|Thursday|2038Q4|N|N|N|2465759|2466092|2465416|2465689|N|N|N|N|N| +2465782|AAAAAAAAGPPJFCAA|2038-12-24|1667|7252|557|2038|5|12|24|4|2038|557|7252|Friday|2038Q4|N|Y|N|2465759|2466092|2465417|2465690|N|N|N|N|N| +2465783|AAAAAAAAHPPJFCAA|2038-12-25|1667|7252|557|2038|6|12|25|4|2038|557|7252|Saturday|2038Q4|N|Y|N|2465759|2466092|2465418|2465691|N|N|N|N|N| +2465784|AAAAAAAAIPPJFCAA|2038-12-26|1667|7252|557|2038|0|12|26|4|2038|557|7252|Sunday|2038Q4|Y|N|N|2465759|2466092|2465419|2465692|N|N|N|N|N| +2465785|AAAAAAAAJPPJFCAA|2038-12-27|1667|7252|557|2038|1|12|27|4|2038|557|7252|Monday|2038Q4|N|N|Y|2465759|2466092|2465420|2465693|N|N|N|N|N| +2465786|AAAAAAAAKPPJFCAA|2038-12-28|1667|7253|557|2038|2|12|28|4|2038|557|7253|Tuesday|2038Q4|N|N|N|2465759|2466092|2465421|2465694|N|N|N|N|N| +2465787|AAAAAAAALPPJFCAA|2038-12-29|1667|7253|557|2038|3|12|29|4|2038|557|7253|Wednesday|2038Q4|N|N|N|2465759|2466092|2465422|2465695|N|N|N|N|N| +2465788|AAAAAAAAMPPJFCAA|2038-12-30|1667|7253|557|2038|4|12|30|4|2038|557|7253|Thursday|2038Q4|N|N|N|2465759|2466092|2465423|2465696|N|N|N|N|N| +2465789|AAAAAAAANPPJFCAA|2038-12-31|1667|7253|557|2038|5|12|31|4|2038|557|7253|Friday|2038Q4|N|Y|N|2465759|2466092|2465424|2465697|N|N|N|N|N| +2465790|AAAAAAAAOPPJFCAA|2039-01-01|1668|7253|557|2039|6|1|1|1|2039|557|7253|Saturday|2039Q1|Y|Y|N|2465790|2465789|2465425|2465698|N|N|N|N|N| +2465791|AAAAAAAAPPPJFCAA|2039-01-02|1668|7253|557|2039|0|1|2|1|2039|557|7253|Sunday|2039Q1|N|N|Y|2465790|2465789|2465426|2465699|N|N|N|N|N| +2465792|AAAAAAAAAAAKFCAA|2039-01-03|1668|7253|557|2039|1|1|3|1|2039|557|7253|Monday|2039Q1|N|N|N|2465790|2465789|2465427|2465700|N|N|N|N|N| +2465793|AAAAAAAABAAKFCAA|2039-01-04|1668|7254|557|2039|2|1|4|1|2039|557|7254|Tuesday|2039Q1|N|N|N|2465790|2465789|2465428|2465701|N|N|N|N|N| +2465794|AAAAAAAACAAKFCAA|2039-01-05|1668|7254|557|2039|3|1|5|1|2039|557|7254|Wednesday|2039Q1|N|N|N|2465790|2465789|2465429|2465702|N|N|N|N|N| +2465795|AAAAAAAADAAKFCAA|2039-01-06|1668|7254|557|2039|4|1|6|1|2039|557|7254|Thursday|2039Q1|N|N|N|2465790|2465789|2465430|2465703|N|N|N|N|N| +2465796|AAAAAAAAEAAKFCAA|2039-01-07|1668|7254|557|2039|5|1|7|1|2039|557|7254|Friday|2039Q1|N|Y|N|2465790|2465789|2465431|2465704|N|N|N|N|N| +2465797|AAAAAAAAFAAKFCAA|2039-01-08|1668|7254|557|2039|6|1|8|1|2039|557|7254|Saturday|2039Q1|N|Y|N|2465790|2465789|2465432|2465705|N|N|N|N|N| +2465798|AAAAAAAAGAAKFCAA|2039-01-09|1668|7254|557|2039|0|1|9|1|2039|557|7254|Sunday|2039Q1|N|N|N|2465790|2465789|2465433|2465706|N|N|N|N|N| +2465799|AAAAAAAAHAAKFCAA|2039-01-10|1668|7254|557|2039|1|1|10|1|2039|557|7254|Monday|2039Q1|N|N|N|2465790|2465789|2465434|2465707|N|N|N|N|N| +2465800|AAAAAAAAIAAKFCAA|2039-01-11|1668|7255|557|2039|2|1|11|1|2039|557|7255|Tuesday|2039Q1|N|N|N|2465790|2465789|2465435|2465708|N|N|N|N|N| +2465801|AAAAAAAAJAAKFCAA|2039-01-12|1668|7255|557|2039|3|1|12|1|2039|557|7255|Wednesday|2039Q1|N|N|N|2465790|2465789|2465436|2465709|N|N|N|N|N| +2465802|AAAAAAAAKAAKFCAA|2039-01-13|1668|7255|557|2039|4|1|13|1|2039|557|7255|Thursday|2039Q1|N|N|N|2465790|2465789|2465437|2465710|N|N|N|N|N| +2465803|AAAAAAAALAAKFCAA|2039-01-14|1668|7255|557|2039|5|1|14|1|2039|557|7255|Friday|2039Q1|N|Y|N|2465790|2465789|2465438|2465711|N|N|N|N|N| +2465804|AAAAAAAAMAAKFCAA|2039-01-15|1668|7255|557|2039|6|1|15|1|2039|557|7255|Saturday|2039Q1|N|Y|N|2465790|2465789|2465439|2465712|N|N|N|N|N| +2465805|AAAAAAAANAAKFCAA|2039-01-16|1668|7255|557|2039|0|1|16|1|2039|557|7255|Sunday|2039Q1|N|N|N|2465790|2465789|2465440|2465713|N|N|N|N|N| +2465806|AAAAAAAAOAAKFCAA|2039-01-17|1668|7255|557|2039|1|1|17|1|2039|557|7255|Monday|2039Q1|N|N|N|2465790|2465789|2465441|2465714|N|N|N|N|N| +2465807|AAAAAAAAPAAKFCAA|2039-01-18|1668|7256|557|2039|2|1|18|1|2039|557|7256|Tuesday|2039Q1|N|N|N|2465790|2465789|2465442|2465715|N|N|N|N|N| +2465808|AAAAAAAAABAKFCAA|2039-01-19|1668|7256|557|2039|3|1|19|1|2039|557|7256|Wednesday|2039Q1|N|N|N|2465790|2465789|2465443|2465716|N|N|N|N|N| +2465809|AAAAAAAABBAKFCAA|2039-01-20|1668|7256|557|2039|4|1|20|1|2039|557|7256|Thursday|2039Q1|N|N|N|2465790|2465789|2465444|2465717|N|N|N|N|N| +2465810|AAAAAAAACBAKFCAA|2039-01-21|1668|7256|557|2039|5|1|21|1|2039|557|7256|Friday|2039Q1|N|Y|N|2465790|2465789|2465445|2465718|N|N|N|N|N| +2465811|AAAAAAAADBAKFCAA|2039-01-22|1668|7256|557|2039|6|1|22|1|2039|557|7256|Saturday|2039Q1|N|Y|N|2465790|2465789|2465446|2465719|N|N|N|N|N| +2465812|AAAAAAAAEBAKFCAA|2039-01-23|1668|7256|557|2039|0|1|23|1|2039|557|7256|Sunday|2039Q1|N|N|N|2465790|2465789|2465447|2465720|N|N|N|N|N| +2465813|AAAAAAAAFBAKFCAA|2039-01-24|1668|7256|557|2039|1|1|24|1|2039|557|7256|Monday|2039Q1|N|N|N|2465790|2465789|2465448|2465721|N|N|N|N|N| +2465814|AAAAAAAAGBAKFCAA|2039-01-25|1668|7257|557|2039|2|1|25|1|2039|557|7257|Tuesday|2039Q1|N|N|N|2465790|2465789|2465449|2465722|N|N|N|N|N| +2465815|AAAAAAAAHBAKFCAA|2039-01-26|1668|7257|557|2039|3|1|26|1|2039|557|7257|Wednesday|2039Q1|N|N|N|2465790|2465789|2465450|2465723|N|N|N|N|N| +2465816|AAAAAAAAIBAKFCAA|2039-01-27|1668|7257|557|2039|4|1|27|1|2039|557|7257|Thursday|2039Q1|N|N|N|2465790|2465789|2465451|2465724|N|N|N|N|N| +2465817|AAAAAAAAJBAKFCAA|2039-01-28|1668|7257|557|2039|5|1|28|1|2039|557|7257|Friday|2039Q1|N|Y|N|2465790|2465789|2465452|2465725|N|N|N|N|N| +2465818|AAAAAAAAKBAKFCAA|2039-01-29|1668|7257|557|2039|6|1|29|1|2039|557|7257|Saturday|2039Q1|N|Y|N|2465790|2465789|2465453|2465726|N|N|N|N|N| +2465819|AAAAAAAALBAKFCAA|2039-01-30|1668|7257|557|2039|0|1|30|1|2039|557|7257|Sunday|2039Q1|N|N|N|2465790|2465789|2465454|2465727|N|N|N|N|N| +2465820|AAAAAAAAMBAKFCAA|2039-01-31|1668|7257|557|2039|1|1|31|1|2039|557|7257|Monday|2039Q1|N|N|N|2465790|2465789|2465455|2465728|N|N|N|N|N| +2465821|AAAAAAAANBAKFCAA|2039-02-01|1669|7258|557|2039|2|2|1|1|2039|557|7258|Tuesday|2039Q1|N|N|N|2465821|2465851|2465456|2465729|N|N|N|N|N| +2465822|AAAAAAAAOBAKFCAA|2039-02-02|1669|7258|557|2039|3|2|2|1|2039|557|7258|Wednesday|2039Q1|N|N|N|2465821|2465851|2465457|2465730|N|N|N|N|N| +2465823|AAAAAAAAPBAKFCAA|2039-02-03|1669|7258|557|2039|4|2|3|1|2039|557|7258|Thursday|2039Q1|N|N|N|2465821|2465851|2465458|2465731|N|N|N|N|N| +2465824|AAAAAAAAACAKFCAA|2039-02-04|1669|7258|557|2039|5|2|4|1|2039|557|7258|Friday|2039Q1|N|Y|N|2465821|2465851|2465459|2465732|N|N|N|N|N| +2465825|AAAAAAAABCAKFCAA|2039-02-05|1669|7258|557|2039|6|2|5|1|2039|557|7258|Saturday|2039Q1|N|Y|N|2465821|2465851|2465460|2465733|N|N|N|N|N| +2465826|AAAAAAAACCAKFCAA|2039-02-06|1669|7258|557|2039|0|2|6|1|2039|557|7258|Sunday|2039Q1|N|N|N|2465821|2465851|2465461|2465734|N|N|N|N|N| +2465827|AAAAAAAADCAKFCAA|2039-02-07|1669|7258|557|2039|1|2|7|1|2039|557|7258|Monday|2039Q1|N|N|N|2465821|2465851|2465462|2465735|N|N|N|N|N| +2465828|AAAAAAAAECAKFCAA|2039-02-08|1669|7259|557|2039|2|2|8|1|2039|557|7259|Tuesday|2039Q1|N|N|N|2465821|2465851|2465463|2465736|N|N|N|N|N| +2465829|AAAAAAAAFCAKFCAA|2039-02-09|1669|7259|557|2039|3|2|9|1|2039|557|7259|Wednesday|2039Q1|N|N|N|2465821|2465851|2465464|2465737|N|N|N|N|N| +2465830|AAAAAAAAGCAKFCAA|2039-02-10|1669|7259|557|2039|4|2|10|1|2039|557|7259|Thursday|2039Q1|N|N|N|2465821|2465851|2465465|2465738|N|N|N|N|N| +2465831|AAAAAAAAHCAKFCAA|2039-02-11|1669|7259|557|2039|5|2|11|1|2039|557|7259|Friday|2039Q1|N|Y|N|2465821|2465851|2465466|2465739|N|N|N|N|N| +2465832|AAAAAAAAICAKFCAA|2039-02-12|1669|7259|557|2039|6|2|12|1|2039|557|7259|Saturday|2039Q1|N|Y|N|2465821|2465851|2465467|2465740|N|N|N|N|N| +2465833|AAAAAAAAJCAKFCAA|2039-02-13|1669|7259|557|2039|0|2|13|1|2039|557|7259|Sunday|2039Q1|N|N|N|2465821|2465851|2465468|2465741|N|N|N|N|N| +2465834|AAAAAAAAKCAKFCAA|2039-02-14|1669|7259|557|2039|1|2|14|1|2039|557|7259|Monday|2039Q1|N|N|N|2465821|2465851|2465469|2465742|N|N|N|N|N| +2465835|AAAAAAAALCAKFCAA|2039-02-15|1669|7260|557|2039|2|2|15|1|2039|557|7260|Tuesday|2039Q1|N|N|N|2465821|2465851|2465470|2465743|N|N|N|N|N| +2465836|AAAAAAAAMCAKFCAA|2039-02-16|1669|7260|557|2039|3|2|16|1|2039|557|7260|Wednesday|2039Q1|N|N|N|2465821|2465851|2465471|2465744|N|N|N|N|N| +2465837|AAAAAAAANCAKFCAA|2039-02-17|1669|7260|557|2039|4|2|17|1|2039|557|7260|Thursday|2039Q1|N|N|N|2465821|2465851|2465472|2465745|N|N|N|N|N| +2465838|AAAAAAAAOCAKFCAA|2039-02-18|1669|7260|557|2039|5|2|18|1|2039|557|7260|Friday|2039Q1|N|Y|N|2465821|2465851|2465473|2465746|N|N|N|N|N| +2465839|AAAAAAAAPCAKFCAA|2039-02-19|1669|7260|557|2039|6|2|19|1|2039|557|7260|Saturday|2039Q1|N|Y|N|2465821|2465851|2465474|2465747|N|N|N|N|N| +2465840|AAAAAAAAADAKFCAA|2039-02-20|1669|7260|557|2039|0|2|20|1|2039|557|7260|Sunday|2039Q1|N|N|N|2465821|2465851|2465475|2465748|N|N|N|N|N| +2465841|AAAAAAAABDAKFCAA|2039-02-21|1669|7260|557|2039|1|2|21|1|2039|557|7260|Monday|2039Q1|N|N|N|2465821|2465851|2465476|2465749|N|N|N|N|N| +2465842|AAAAAAAACDAKFCAA|2039-02-22|1669|7261|557|2039|2|2|22|1|2039|557|7261|Tuesday|2039Q1|N|N|N|2465821|2465851|2465477|2465750|N|N|N|N|N| +2465843|AAAAAAAADDAKFCAA|2039-02-23|1669|7261|557|2039|3|2|23|1|2039|557|7261|Wednesday|2039Q1|N|N|N|2465821|2465851|2465478|2465751|N|N|N|N|N| +2465844|AAAAAAAAEDAKFCAA|2039-02-24|1669|7261|557|2039|4|2|24|1|2039|557|7261|Thursday|2039Q1|N|N|N|2465821|2465851|2465479|2465752|N|N|N|N|N| +2465845|AAAAAAAAFDAKFCAA|2039-02-25|1669|7261|557|2039|5|2|25|1|2039|557|7261|Friday|2039Q1|N|Y|N|2465821|2465851|2465480|2465753|N|N|N|N|N| +2465846|AAAAAAAAGDAKFCAA|2039-02-26|1669|7261|557|2039|6|2|26|1|2039|557|7261|Saturday|2039Q1|N|Y|N|2465821|2465851|2465481|2465754|N|N|N|N|N| +2465847|AAAAAAAAHDAKFCAA|2039-02-27|1669|7261|557|2039|0|2|27|1|2039|557|7261|Sunday|2039Q1|N|N|N|2465821|2465851|2465482|2465755|N|N|N|N|N| +2465848|AAAAAAAAIDAKFCAA|2039-02-28|1669|7261|557|2039|1|2|28|1|2039|557|7261|Monday|2039Q1|N|N|N|2465821|2465851|2465483|2465756|N|N|N|N|N| +2465849|AAAAAAAAJDAKFCAA|2039-03-01|1670|7262|558|2039|2|3|1|1|2039|558|7262|Tuesday|2039Q1|N|N|N|2465849|2465907|2465484|2465757|N|N|N|N|N| +2465850|AAAAAAAAKDAKFCAA|2039-03-02|1670|7262|558|2039|3|3|2|1|2039|558|7262|Wednesday|2039Q1|N|N|N|2465849|2465907|2465485|2465758|N|N|N|N|N| +2465851|AAAAAAAALDAKFCAA|2039-03-03|1670|7262|558|2039|4|3|3|1|2039|558|7262|Thursday|2039Q1|N|N|N|2465849|2465907|2465486|2465759|N|N|N|N|N| +2465852|AAAAAAAAMDAKFCAA|2039-03-04|1670|7262|558|2039|5|3|4|1|2039|558|7262|Friday|2039Q1|N|Y|N|2465849|2465907|2465487|2465760|N|N|N|N|N| +2465853|AAAAAAAANDAKFCAA|2039-03-05|1670|7262|558|2039|6|3|5|1|2039|558|7262|Saturday|2039Q1|N|Y|N|2465849|2465907|2465488|2465761|N|N|N|N|N| +2465854|AAAAAAAAODAKFCAA|2039-03-06|1670|7262|558|2039|0|3|6|1|2039|558|7262|Sunday|2039Q1|N|N|N|2465849|2465907|2465489|2465762|N|N|N|N|N| +2465855|AAAAAAAAPDAKFCAA|2039-03-07|1670|7262|558|2039|1|3|7|1|2039|558|7262|Monday|2039Q1|N|N|N|2465849|2465907|2465490|2465763|N|N|N|N|N| +2465856|AAAAAAAAAEAKFCAA|2039-03-08|1670|7263|558|2039|2|3|8|1|2039|558|7263|Tuesday|2039Q1|N|N|N|2465849|2465907|2465491|2465764|N|N|N|N|N| +2465857|AAAAAAAABEAKFCAA|2039-03-09|1670|7263|558|2039|3|3|9|1|2039|558|7263|Wednesday|2039Q1|N|N|N|2465849|2465907|2465492|2465765|N|N|N|N|N| +2465858|AAAAAAAACEAKFCAA|2039-03-10|1670|7263|558|2039|4|3|10|1|2039|558|7263|Thursday|2039Q1|N|N|N|2465849|2465907|2465493|2465766|N|N|N|N|N| +2465859|AAAAAAAADEAKFCAA|2039-03-11|1670|7263|558|2039|5|3|11|1|2039|558|7263|Friday|2039Q1|N|Y|N|2465849|2465907|2465494|2465767|N|N|N|N|N| +2465860|AAAAAAAAEEAKFCAA|2039-03-12|1670|7263|558|2039|6|3|12|1|2039|558|7263|Saturday|2039Q1|N|Y|N|2465849|2465907|2465495|2465768|N|N|N|N|N| +2465861|AAAAAAAAFEAKFCAA|2039-03-13|1670|7263|558|2039|0|3|13|1|2039|558|7263|Sunday|2039Q1|N|N|N|2465849|2465907|2465496|2465769|N|N|N|N|N| +2465862|AAAAAAAAGEAKFCAA|2039-03-14|1670|7263|558|2039|1|3|14|1|2039|558|7263|Monday|2039Q1|N|N|N|2465849|2465907|2465497|2465770|N|N|N|N|N| +2465863|AAAAAAAAHEAKFCAA|2039-03-15|1670|7264|558|2039|2|3|15|1|2039|558|7264|Tuesday|2039Q1|N|N|N|2465849|2465907|2465498|2465771|N|N|N|N|N| +2465864|AAAAAAAAIEAKFCAA|2039-03-16|1670|7264|558|2039|3|3|16|1|2039|558|7264|Wednesday|2039Q1|N|N|N|2465849|2465907|2465499|2465772|N|N|N|N|N| +2465865|AAAAAAAAJEAKFCAA|2039-03-17|1670|7264|558|2039|4|3|17|1|2039|558|7264|Thursday|2039Q1|N|N|N|2465849|2465907|2465500|2465773|N|N|N|N|N| +2465866|AAAAAAAAKEAKFCAA|2039-03-18|1670|7264|558|2039|5|3|18|1|2039|558|7264|Friday|2039Q1|N|Y|N|2465849|2465907|2465501|2465774|N|N|N|N|N| +2465867|AAAAAAAALEAKFCAA|2039-03-19|1670|7264|558|2039|6|3|19|1|2039|558|7264|Saturday|2039Q1|N|Y|N|2465849|2465907|2465502|2465775|N|N|N|N|N| +2465868|AAAAAAAAMEAKFCAA|2039-03-20|1670|7264|558|2039|0|3|20|1|2039|558|7264|Sunday|2039Q1|N|N|N|2465849|2465907|2465503|2465776|N|N|N|N|N| +2465869|AAAAAAAANEAKFCAA|2039-03-21|1670|7264|558|2039|1|3|21|1|2039|558|7264|Monday|2039Q1|N|N|N|2465849|2465907|2465504|2465777|N|N|N|N|N| +2465870|AAAAAAAAOEAKFCAA|2039-03-22|1670|7265|558|2039|2|3|22|1|2039|558|7265|Tuesday|2039Q1|N|N|N|2465849|2465907|2465505|2465778|N|N|N|N|N| +2465871|AAAAAAAAPEAKFCAA|2039-03-23|1670|7265|558|2039|3|3|23|1|2039|558|7265|Wednesday|2039Q1|N|N|N|2465849|2465907|2465506|2465779|N|N|N|N|N| +2465872|AAAAAAAAAFAKFCAA|2039-03-24|1670|7265|558|2039|4|3|24|1|2039|558|7265|Thursday|2039Q1|N|N|N|2465849|2465907|2465507|2465780|N|N|N|N|N| +2465873|AAAAAAAABFAKFCAA|2039-03-25|1670|7265|558|2039|5|3|25|1|2039|558|7265|Friday|2039Q1|N|Y|N|2465849|2465907|2465508|2465781|N|N|N|N|N| +2465874|AAAAAAAACFAKFCAA|2039-03-26|1670|7265|558|2039|6|3|26|1|2039|558|7265|Saturday|2039Q1|N|Y|N|2465849|2465907|2465509|2465782|N|N|N|N|N| +2465875|AAAAAAAADFAKFCAA|2039-03-27|1670|7265|558|2039|0|3|27|1|2039|558|7265|Sunday|2039Q1|N|N|N|2465849|2465907|2465510|2465783|N|N|N|N|N| +2465876|AAAAAAAAEFAKFCAA|2039-03-28|1670|7265|558|2039|1|3|28|1|2039|558|7265|Monday|2039Q1|N|N|N|2465849|2465907|2465511|2465784|N|N|N|N|N| +2465877|AAAAAAAAFFAKFCAA|2039-03-29|1670|7266|558|2039|2|3|29|1|2039|558|7266|Tuesday|2039Q1|N|N|N|2465849|2465907|2465512|2465785|N|N|N|N|N| +2465878|AAAAAAAAGFAKFCAA|2039-03-30|1670|7266|558|2039|3|3|30|1|2039|558|7266|Wednesday|2039Q1|N|N|N|2465849|2465907|2465513|2465786|N|N|N|N|N| +2465879|AAAAAAAAHFAKFCAA|2039-03-31|1670|7266|558|2039|4|3|31|1|2039|558|7266|Thursday|2039Q1|N|N|N|2465849|2465907|2465514|2465787|N|N|N|N|N| +2465880|AAAAAAAAIFAKFCAA|2039-04-01|1671|7266|558|2039|5|4|1|1|2039|558|7266|Friday|2039Q1|N|Y|N|2465880|2465969|2465515|2465790|N|N|N|N|N| +2465881|AAAAAAAAJFAKFCAA|2039-04-02|1671|7266|558|2039|6|4|2|2|2039|558|7266|Saturday|2039Q2|N|Y|N|2465880|2465969|2465516|2465791|N|N|N|N|N| +2465882|AAAAAAAAKFAKFCAA|2039-04-03|1671|7266|558|2039|0|4|3|2|2039|558|7266|Sunday|2039Q2|N|N|N|2465880|2465969|2465517|2465792|N|N|N|N|N| +2465883|AAAAAAAALFAKFCAA|2039-04-04|1671|7266|558|2039|1|4|4|2|2039|558|7266|Monday|2039Q2|N|N|N|2465880|2465969|2465518|2465793|N|N|N|N|N| +2465884|AAAAAAAAMFAKFCAA|2039-04-05|1671|7267|558|2039|2|4|5|2|2039|558|7267|Tuesday|2039Q2|N|N|N|2465880|2465969|2465519|2465794|N|N|N|N|N| +2465885|AAAAAAAANFAKFCAA|2039-04-06|1671|7267|558|2039|3|4|6|2|2039|558|7267|Wednesday|2039Q2|N|N|N|2465880|2465969|2465520|2465795|N|N|N|N|N| +2465886|AAAAAAAAOFAKFCAA|2039-04-07|1671|7267|558|2039|4|4|7|2|2039|558|7267|Thursday|2039Q2|N|N|N|2465880|2465969|2465521|2465796|N|N|N|N|N| +2465887|AAAAAAAAPFAKFCAA|2039-04-08|1671|7267|558|2039|5|4|8|2|2039|558|7267|Friday|2039Q2|N|Y|N|2465880|2465969|2465522|2465797|N|N|N|N|N| +2465888|AAAAAAAAAGAKFCAA|2039-04-09|1671|7267|558|2039|6|4|9|2|2039|558|7267|Saturday|2039Q2|N|Y|N|2465880|2465969|2465523|2465798|N|N|N|N|N| +2465889|AAAAAAAABGAKFCAA|2039-04-10|1671|7267|558|2039|0|4|10|2|2039|558|7267|Sunday|2039Q2|N|N|N|2465880|2465969|2465524|2465799|N|N|N|N|N| +2465890|AAAAAAAACGAKFCAA|2039-04-11|1671|7267|558|2039|1|4|11|2|2039|558|7267|Monday|2039Q2|N|N|N|2465880|2465969|2465525|2465800|N|N|N|N|N| +2465891|AAAAAAAADGAKFCAA|2039-04-12|1671|7268|558|2039|2|4|12|2|2039|558|7268|Tuesday|2039Q2|N|N|N|2465880|2465969|2465526|2465801|N|N|N|N|N| +2465892|AAAAAAAAEGAKFCAA|2039-04-13|1671|7268|558|2039|3|4|13|2|2039|558|7268|Wednesday|2039Q2|N|N|N|2465880|2465969|2465527|2465802|N|N|N|N|N| +2465893|AAAAAAAAFGAKFCAA|2039-04-14|1671|7268|558|2039|4|4|14|2|2039|558|7268|Thursday|2039Q2|N|N|N|2465880|2465969|2465528|2465803|N|N|N|N|N| +2465894|AAAAAAAAGGAKFCAA|2039-04-15|1671|7268|558|2039|5|4|15|2|2039|558|7268|Friday|2039Q2|N|Y|N|2465880|2465969|2465529|2465804|N|N|N|N|N| +2465895|AAAAAAAAHGAKFCAA|2039-04-16|1671|7268|558|2039|6|4|16|2|2039|558|7268|Saturday|2039Q2|N|Y|N|2465880|2465969|2465530|2465805|N|N|N|N|N| +2465896|AAAAAAAAIGAKFCAA|2039-04-17|1671|7268|558|2039|0|4|17|2|2039|558|7268|Sunday|2039Q2|N|N|N|2465880|2465969|2465531|2465806|N|N|N|N|N| +2465897|AAAAAAAAJGAKFCAA|2039-04-18|1671|7268|558|2039|1|4|18|2|2039|558|7268|Monday|2039Q2|N|N|N|2465880|2465969|2465532|2465807|N|N|N|N|N| +2465898|AAAAAAAAKGAKFCAA|2039-04-19|1671|7269|558|2039|2|4|19|2|2039|558|7269|Tuesday|2039Q2|N|N|N|2465880|2465969|2465533|2465808|N|N|N|N|N| +2465899|AAAAAAAALGAKFCAA|2039-04-20|1671|7269|558|2039|3|4|20|2|2039|558|7269|Wednesday|2039Q2|N|N|N|2465880|2465969|2465534|2465809|N|N|N|N|N| +2465900|AAAAAAAAMGAKFCAA|2039-04-21|1671|7269|558|2039|4|4|21|2|2039|558|7269|Thursday|2039Q2|N|N|N|2465880|2465969|2465535|2465810|N|N|N|N|N| +2465901|AAAAAAAANGAKFCAA|2039-04-22|1671|7269|558|2039|5|4|22|2|2039|558|7269|Friday|2039Q2|N|Y|N|2465880|2465969|2465536|2465811|N|N|N|N|N| +2465902|AAAAAAAAOGAKFCAA|2039-04-23|1671|7269|558|2039|6|4|23|2|2039|558|7269|Saturday|2039Q2|N|Y|N|2465880|2465969|2465537|2465812|N|N|N|N|N| +2465903|AAAAAAAAPGAKFCAA|2039-04-24|1671|7269|558|2039|0|4|24|2|2039|558|7269|Sunday|2039Q2|N|N|N|2465880|2465969|2465538|2465813|N|N|N|N|N| +2465904|AAAAAAAAAHAKFCAA|2039-04-25|1671|7269|558|2039|1|4|25|2|2039|558|7269|Monday|2039Q2|N|N|N|2465880|2465969|2465539|2465814|N|N|N|N|N| +2465905|AAAAAAAABHAKFCAA|2039-04-26|1671|7270|558|2039|2|4|26|2|2039|558|7270|Tuesday|2039Q2|N|N|N|2465880|2465969|2465540|2465815|N|N|N|N|N| +2465906|AAAAAAAACHAKFCAA|2039-04-27|1671|7270|558|2039|3|4|27|2|2039|558|7270|Wednesday|2039Q2|N|N|N|2465880|2465969|2465541|2465816|N|N|N|N|N| +2465907|AAAAAAAADHAKFCAA|2039-04-28|1671|7270|558|2039|4|4|28|2|2039|558|7270|Thursday|2039Q2|N|N|N|2465880|2465969|2465542|2465817|N|N|N|N|N| +2465908|AAAAAAAAEHAKFCAA|2039-04-29|1671|7270|558|2039|5|4|29|2|2039|558|7270|Friday|2039Q2|N|Y|N|2465880|2465969|2465543|2465818|N|N|N|N|N| +2465909|AAAAAAAAFHAKFCAA|2039-04-30|1671|7270|558|2039|6|4|30|2|2039|558|7270|Saturday|2039Q2|N|Y|N|2465880|2465969|2465544|2465819|N|N|N|N|N| +2465910|AAAAAAAAGHAKFCAA|2039-05-01|1672|7270|558|2039|0|5|1|2|2039|558|7270|Sunday|2039Q2|N|N|N|2465910|2466029|2465545|2465820|N|N|N|N|N| +2465911|AAAAAAAAHHAKFCAA|2039-05-02|1672|7270|558|2039|1|5|2|2|2039|558|7270|Monday|2039Q2|N|N|N|2465910|2466029|2465546|2465821|N|N|N|N|N| +2465912|AAAAAAAAIHAKFCAA|2039-05-03|1672|7271|558|2039|2|5|3|2|2039|558|7271|Tuesday|2039Q2|N|N|N|2465910|2466029|2465547|2465822|N|N|N|N|N| +2465913|AAAAAAAAJHAKFCAA|2039-05-04|1672|7271|558|2039|3|5|4|2|2039|558|7271|Wednesday|2039Q2|N|N|N|2465910|2466029|2465548|2465823|N|N|N|N|N| +2465914|AAAAAAAAKHAKFCAA|2039-05-05|1672|7271|558|2039|4|5|5|2|2039|558|7271|Thursday|2039Q2|N|N|N|2465910|2466029|2465549|2465824|N|N|N|N|N| +2465915|AAAAAAAALHAKFCAA|2039-05-06|1672|7271|558|2039|5|5|6|2|2039|558|7271|Friday|2039Q2|N|Y|N|2465910|2466029|2465550|2465825|N|N|N|N|N| +2465916|AAAAAAAAMHAKFCAA|2039-05-07|1672|7271|558|2039|6|5|7|2|2039|558|7271|Saturday|2039Q2|N|Y|N|2465910|2466029|2465551|2465826|N|N|N|N|N| +2465917|AAAAAAAANHAKFCAA|2039-05-08|1672|7271|558|2039|0|5|8|2|2039|558|7271|Sunday|2039Q2|N|N|N|2465910|2466029|2465552|2465827|N|N|N|N|N| +2465918|AAAAAAAAOHAKFCAA|2039-05-09|1672|7271|558|2039|1|5|9|2|2039|558|7271|Monday|2039Q2|N|N|N|2465910|2466029|2465553|2465828|N|N|N|N|N| +2465919|AAAAAAAAPHAKFCAA|2039-05-10|1672|7272|558|2039|2|5|10|2|2039|558|7272|Tuesday|2039Q2|N|N|N|2465910|2466029|2465554|2465829|N|N|N|N|N| +2465920|AAAAAAAAAIAKFCAA|2039-05-11|1672|7272|558|2039|3|5|11|2|2039|558|7272|Wednesday|2039Q2|N|N|N|2465910|2466029|2465555|2465830|N|N|N|N|N| +2465921|AAAAAAAABIAKFCAA|2039-05-12|1672|7272|558|2039|4|5|12|2|2039|558|7272|Thursday|2039Q2|N|N|N|2465910|2466029|2465556|2465831|N|N|N|N|N| +2465922|AAAAAAAACIAKFCAA|2039-05-13|1672|7272|558|2039|5|5|13|2|2039|558|7272|Friday|2039Q2|N|Y|N|2465910|2466029|2465557|2465832|N|N|N|N|N| +2465923|AAAAAAAADIAKFCAA|2039-05-14|1672|7272|558|2039|6|5|14|2|2039|558|7272|Saturday|2039Q2|N|Y|N|2465910|2466029|2465558|2465833|N|N|N|N|N| +2465924|AAAAAAAAEIAKFCAA|2039-05-15|1672|7272|558|2039|0|5|15|2|2039|558|7272|Sunday|2039Q2|N|N|N|2465910|2466029|2465559|2465834|N|N|N|N|N| +2465925|AAAAAAAAFIAKFCAA|2039-05-16|1672|7272|558|2039|1|5|16|2|2039|558|7272|Monday|2039Q2|N|N|N|2465910|2466029|2465560|2465835|N|N|N|N|N| +2465926|AAAAAAAAGIAKFCAA|2039-05-17|1672|7273|558|2039|2|5|17|2|2039|558|7273|Tuesday|2039Q2|N|N|N|2465910|2466029|2465561|2465836|N|N|N|N|N| +2465927|AAAAAAAAHIAKFCAA|2039-05-18|1672|7273|558|2039|3|5|18|2|2039|558|7273|Wednesday|2039Q2|N|N|N|2465910|2466029|2465562|2465837|N|N|N|N|N| +2465928|AAAAAAAAIIAKFCAA|2039-05-19|1672|7273|558|2039|4|5|19|2|2039|558|7273|Thursday|2039Q2|N|N|N|2465910|2466029|2465563|2465838|N|N|N|N|N| +2465929|AAAAAAAAJIAKFCAA|2039-05-20|1672|7273|558|2039|5|5|20|2|2039|558|7273|Friday|2039Q2|N|Y|N|2465910|2466029|2465564|2465839|N|N|N|N|N| +2465930|AAAAAAAAKIAKFCAA|2039-05-21|1672|7273|558|2039|6|5|21|2|2039|558|7273|Saturday|2039Q2|N|Y|N|2465910|2466029|2465565|2465840|N|N|N|N|N| +2465931|AAAAAAAALIAKFCAA|2039-05-22|1672|7273|558|2039|0|5|22|2|2039|558|7273|Sunday|2039Q2|N|N|N|2465910|2466029|2465566|2465841|N|N|N|N|N| +2465932|AAAAAAAAMIAKFCAA|2039-05-23|1672|7273|558|2039|1|5|23|2|2039|558|7273|Monday|2039Q2|N|N|N|2465910|2466029|2465567|2465842|N|N|N|N|N| +2465933|AAAAAAAANIAKFCAA|2039-05-24|1672|7274|558|2039|2|5|24|2|2039|558|7274|Tuesday|2039Q2|N|N|N|2465910|2466029|2465568|2465843|N|N|N|N|N| +2465934|AAAAAAAAOIAKFCAA|2039-05-25|1672|7274|558|2039|3|5|25|2|2039|558|7274|Wednesday|2039Q2|N|N|N|2465910|2466029|2465569|2465844|N|N|N|N|N| +2465935|AAAAAAAAPIAKFCAA|2039-05-26|1672|7274|558|2039|4|5|26|2|2039|558|7274|Thursday|2039Q2|N|N|N|2465910|2466029|2465570|2465845|N|N|N|N|N| +2465936|AAAAAAAAAJAKFCAA|2039-05-27|1672|7274|558|2039|5|5|27|2|2039|558|7274|Friday|2039Q2|N|Y|N|2465910|2466029|2465571|2465846|N|N|N|N|N| +2465937|AAAAAAAABJAKFCAA|2039-05-28|1672|7274|558|2039|6|5|28|2|2039|558|7274|Saturday|2039Q2|N|Y|N|2465910|2466029|2465572|2465847|N|N|N|N|N| +2465938|AAAAAAAACJAKFCAA|2039-05-29|1672|7274|558|2039|0|5|29|2|2039|558|7274|Sunday|2039Q2|N|N|N|2465910|2466029|2465573|2465848|N|N|N|N|N| +2465939|AAAAAAAADJAKFCAA|2039-05-30|1672|7274|558|2039|1|5|30|2|2039|558|7274|Monday|2039Q2|N|N|N|2465910|2466029|2465574|2465849|N|N|N|N|N| +2465940|AAAAAAAAEJAKFCAA|2039-05-31|1672|7275|558|2039|2|5|31|2|2039|558|7275|Tuesday|2039Q2|N|N|N|2465910|2466029|2465575|2465850|N|N|N|N|N| +2465941|AAAAAAAAFJAKFCAA|2039-06-01|1673|7275|559|2039|3|6|1|2|2039|559|7275|Wednesday|2039Q2|N|N|N|2465941|2466091|2465576|2465851|N|N|N|N|N| +2465942|AAAAAAAAGJAKFCAA|2039-06-02|1673|7275|559|2039|4|6|2|2|2039|559|7275|Thursday|2039Q2|N|N|N|2465941|2466091|2465577|2465852|N|N|N|N|N| +2465943|AAAAAAAAHJAKFCAA|2039-06-03|1673|7275|559|2039|5|6|3|2|2039|559|7275|Friday|2039Q2|N|Y|N|2465941|2466091|2465578|2465853|N|N|N|N|N| +2465944|AAAAAAAAIJAKFCAA|2039-06-04|1673|7275|559|2039|6|6|4|2|2039|559|7275|Saturday|2039Q2|N|Y|N|2465941|2466091|2465579|2465854|N|N|N|N|N| +2465945|AAAAAAAAJJAKFCAA|2039-06-05|1673|7275|559|2039|0|6|5|2|2039|559|7275|Sunday|2039Q2|N|N|N|2465941|2466091|2465580|2465855|N|N|N|N|N| +2465946|AAAAAAAAKJAKFCAA|2039-06-06|1673|7275|559|2039|1|6|6|2|2039|559|7275|Monday|2039Q2|N|N|N|2465941|2466091|2465581|2465856|N|N|N|N|N| +2465947|AAAAAAAALJAKFCAA|2039-06-07|1673|7276|559|2039|2|6|7|2|2039|559|7276|Tuesday|2039Q2|N|N|N|2465941|2466091|2465582|2465857|N|N|N|N|N| +2465948|AAAAAAAAMJAKFCAA|2039-06-08|1673|7276|559|2039|3|6|8|2|2039|559|7276|Wednesday|2039Q2|N|N|N|2465941|2466091|2465583|2465858|N|N|N|N|N| +2465949|AAAAAAAANJAKFCAA|2039-06-09|1673|7276|559|2039|4|6|9|2|2039|559|7276|Thursday|2039Q2|N|N|N|2465941|2466091|2465584|2465859|N|N|N|N|N| +2465950|AAAAAAAAOJAKFCAA|2039-06-10|1673|7276|559|2039|5|6|10|2|2039|559|7276|Friday|2039Q2|N|Y|N|2465941|2466091|2465585|2465860|N|N|N|N|N| +2465951|AAAAAAAAPJAKFCAA|2039-06-11|1673|7276|559|2039|6|6|11|2|2039|559|7276|Saturday|2039Q2|N|Y|N|2465941|2466091|2465586|2465861|N|N|N|N|N| +2465952|AAAAAAAAAKAKFCAA|2039-06-12|1673|7276|559|2039|0|6|12|2|2039|559|7276|Sunday|2039Q2|N|N|N|2465941|2466091|2465587|2465862|N|N|N|N|N| +2465953|AAAAAAAABKAKFCAA|2039-06-13|1673|7276|559|2039|1|6|13|2|2039|559|7276|Monday|2039Q2|N|N|N|2465941|2466091|2465588|2465863|N|N|N|N|N| +2465954|AAAAAAAACKAKFCAA|2039-06-14|1673|7277|559|2039|2|6|14|2|2039|559|7277|Tuesday|2039Q2|N|N|N|2465941|2466091|2465589|2465864|N|N|N|N|N| +2465955|AAAAAAAADKAKFCAA|2039-06-15|1673|7277|559|2039|3|6|15|2|2039|559|7277|Wednesday|2039Q2|N|N|N|2465941|2466091|2465590|2465865|N|N|N|N|N| +2465956|AAAAAAAAEKAKFCAA|2039-06-16|1673|7277|559|2039|4|6|16|2|2039|559|7277|Thursday|2039Q2|N|N|N|2465941|2466091|2465591|2465866|N|N|N|N|N| +2465957|AAAAAAAAFKAKFCAA|2039-06-17|1673|7277|559|2039|5|6|17|2|2039|559|7277|Friday|2039Q2|N|Y|N|2465941|2466091|2465592|2465867|N|N|N|N|N| +2465958|AAAAAAAAGKAKFCAA|2039-06-18|1673|7277|559|2039|6|6|18|2|2039|559|7277|Saturday|2039Q2|N|Y|N|2465941|2466091|2465593|2465868|N|N|N|N|N| +2465959|AAAAAAAAHKAKFCAA|2039-06-19|1673|7277|559|2039|0|6|19|2|2039|559|7277|Sunday|2039Q2|N|N|N|2465941|2466091|2465594|2465869|N|N|N|N|N| +2465960|AAAAAAAAIKAKFCAA|2039-06-20|1673|7277|559|2039|1|6|20|2|2039|559|7277|Monday|2039Q2|N|N|N|2465941|2466091|2465595|2465870|N|N|N|N|N| +2465961|AAAAAAAAJKAKFCAA|2039-06-21|1673|7278|559|2039|2|6|21|2|2039|559|7278|Tuesday|2039Q2|N|N|N|2465941|2466091|2465596|2465871|N|N|N|N|N| +2465962|AAAAAAAAKKAKFCAA|2039-06-22|1673|7278|559|2039|3|6|22|2|2039|559|7278|Wednesday|2039Q2|N|N|N|2465941|2466091|2465597|2465872|N|N|N|N|N| +2465963|AAAAAAAALKAKFCAA|2039-06-23|1673|7278|559|2039|4|6|23|2|2039|559|7278|Thursday|2039Q2|N|N|N|2465941|2466091|2465598|2465873|N|N|N|N|N| +2465964|AAAAAAAAMKAKFCAA|2039-06-24|1673|7278|559|2039|5|6|24|2|2039|559|7278|Friday|2039Q2|N|Y|N|2465941|2466091|2465599|2465874|N|N|N|N|N| +2465965|AAAAAAAANKAKFCAA|2039-06-25|1673|7278|559|2039|6|6|25|2|2039|559|7278|Saturday|2039Q2|N|Y|N|2465941|2466091|2465600|2465875|N|N|N|N|N| +2465966|AAAAAAAAOKAKFCAA|2039-06-26|1673|7278|559|2039|0|6|26|2|2039|559|7278|Sunday|2039Q2|N|N|N|2465941|2466091|2465601|2465876|N|N|N|N|N| +2465967|AAAAAAAAPKAKFCAA|2039-06-27|1673|7278|559|2039|1|6|27|2|2039|559|7278|Monday|2039Q2|N|N|N|2465941|2466091|2465602|2465877|N|N|N|N|N| +2465968|AAAAAAAAALAKFCAA|2039-06-28|1673|7279|559|2039|2|6|28|2|2039|559|7279|Tuesday|2039Q2|N|N|N|2465941|2466091|2465603|2465878|N|N|N|N|N| +2465969|AAAAAAAABLAKFCAA|2039-06-29|1673|7279|559|2039|3|6|29|2|2039|559|7279|Wednesday|2039Q2|N|N|N|2465941|2466091|2465604|2465879|N|N|N|N|N| +2465970|AAAAAAAACLAKFCAA|2039-06-30|1673|7279|559|2039|4|6|30|2|2039|559|7279|Thursday|2039Q2|N|N|N|2465941|2466091|2465605|2465880|N|N|N|N|N| +2465971|AAAAAAAADLAKFCAA|2039-07-01|1674|7279|559|2039|5|7|1|2|2039|559|7279|Friday|2039Q2|N|Y|N|2465971|2466151|2465606|2465880|N|N|N|N|N| +2465972|AAAAAAAAELAKFCAA|2039-07-02|1674|7279|559|2039|6|7|2|3|2039|559|7279|Saturday|2039Q3|N|Y|N|2465971|2466151|2465607|2465881|N|N|N|N|N| +2465973|AAAAAAAAFLAKFCAA|2039-07-03|1674|7279|559|2039|0|7|3|3|2039|559|7279|Sunday|2039Q3|N|N|N|2465971|2466151|2465608|2465882|N|N|N|N|N| +2465974|AAAAAAAAGLAKFCAA|2039-07-04|1674|7279|559|2039|1|7|4|3|2039|559|7279|Monday|2039Q3|N|N|N|2465971|2466151|2465609|2465883|N|N|N|N|N| +2465975|AAAAAAAAHLAKFCAA|2039-07-05|1674|7280|559|2039|2|7|5|3|2039|559|7280|Tuesday|2039Q3|Y|N|N|2465971|2466151|2465610|2465884|N|N|N|N|N| +2465976|AAAAAAAAILAKFCAA|2039-07-06|1674|7280|559|2039|3|7|6|3|2039|559|7280|Wednesday|2039Q3|N|N|Y|2465971|2466151|2465611|2465885|N|N|N|N|N| +2465977|AAAAAAAAJLAKFCAA|2039-07-07|1674|7280|559|2039|4|7|7|3|2039|559|7280|Thursday|2039Q3|N|N|N|2465971|2466151|2465612|2465886|N|N|N|N|N| +2465978|AAAAAAAAKLAKFCAA|2039-07-08|1674|7280|559|2039|5|7|8|3|2039|559|7280|Friday|2039Q3|N|Y|N|2465971|2466151|2465613|2465887|N|N|N|N|N| +2465979|AAAAAAAALLAKFCAA|2039-07-09|1674|7280|559|2039|6|7|9|3|2039|559|7280|Saturday|2039Q3|N|Y|N|2465971|2466151|2465614|2465888|N|N|N|N|N| +2465980|AAAAAAAAMLAKFCAA|2039-07-10|1674|7280|559|2039|0|7|10|3|2039|559|7280|Sunday|2039Q3|N|N|N|2465971|2466151|2465615|2465889|N|N|N|N|N| +2465981|AAAAAAAANLAKFCAA|2039-07-11|1674|7280|559|2039|1|7|11|3|2039|559|7280|Monday|2039Q3|N|N|N|2465971|2466151|2465616|2465890|N|N|N|N|N| +2465982|AAAAAAAAOLAKFCAA|2039-07-12|1674|7281|559|2039|2|7|12|3|2039|559|7281|Tuesday|2039Q3|N|N|N|2465971|2466151|2465617|2465891|N|N|N|N|N| +2465983|AAAAAAAAPLAKFCAA|2039-07-13|1674|7281|559|2039|3|7|13|3|2039|559|7281|Wednesday|2039Q3|N|N|N|2465971|2466151|2465618|2465892|N|N|N|N|N| +2465984|AAAAAAAAAMAKFCAA|2039-07-14|1674|7281|559|2039|4|7|14|3|2039|559|7281|Thursday|2039Q3|N|N|N|2465971|2466151|2465619|2465893|N|N|N|N|N| +2465985|AAAAAAAABMAKFCAA|2039-07-15|1674|7281|559|2039|5|7|15|3|2039|559|7281|Friday|2039Q3|N|Y|N|2465971|2466151|2465620|2465894|N|N|N|N|N| +2465986|AAAAAAAACMAKFCAA|2039-07-16|1674|7281|559|2039|6|7|16|3|2039|559|7281|Saturday|2039Q3|N|Y|N|2465971|2466151|2465621|2465895|N|N|N|N|N| +2465987|AAAAAAAADMAKFCAA|2039-07-17|1674|7281|559|2039|0|7|17|3|2039|559|7281|Sunday|2039Q3|N|N|N|2465971|2466151|2465622|2465896|N|N|N|N|N| +2465988|AAAAAAAAEMAKFCAA|2039-07-18|1674|7281|559|2039|1|7|18|3|2039|559|7281|Monday|2039Q3|N|N|N|2465971|2466151|2465623|2465897|N|N|N|N|N| +2465989|AAAAAAAAFMAKFCAA|2039-07-19|1674|7282|559|2039|2|7|19|3|2039|559|7282|Tuesday|2039Q3|N|N|N|2465971|2466151|2465624|2465898|N|N|N|N|N| +2465990|AAAAAAAAGMAKFCAA|2039-07-20|1674|7282|559|2039|3|7|20|3|2039|559|7282|Wednesday|2039Q3|N|N|N|2465971|2466151|2465625|2465899|N|N|N|N|N| +2465991|AAAAAAAAHMAKFCAA|2039-07-21|1674|7282|559|2039|4|7|21|3|2039|559|7282|Thursday|2039Q3|N|N|N|2465971|2466151|2465626|2465900|N|N|N|N|N| +2465992|AAAAAAAAIMAKFCAA|2039-07-22|1674|7282|559|2039|5|7|22|3|2039|559|7282|Friday|2039Q3|N|Y|N|2465971|2466151|2465627|2465901|N|N|N|N|N| +2465993|AAAAAAAAJMAKFCAA|2039-07-23|1674|7282|559|2039|6|7|23|3|2039|559|7282|Saturday|2039Q3|N|Y|N|2465971|2466151|2465628|2465902|N|N|N|N|N| +2465994|AAAAAAAAKMAKFCAA|2039-07-24|1674|7282|559|2039|0|7|24|3|2039|559|7282|Sunday|2039Q3|N|N|N|2465971|2466151|2465629|2465903|N|N|N|N|N| +2465995|AAAAAAAALMAKFCAA|2039-07-25|1674|7282|559|2039|1|7|25|3|2039|559|7282|Monday|2039Q3|N|N|N|2465971|2466151|2465630|2465904|N|N|N|N|N| +2465996|AAAAAAAAMMAKFCAA|2039-07-26|1674|7283|559|2039|2|7|26|3|2039|559|7283|Tuesday|2039Q3|N|N|N|2465971|2466151|2465631|2465905|N|N|N|N|N| +2465997|AAAAAAAANMAKFCAA|2039-07-27|1674|7283|559|2039|3|7|27|3|2039|559|7283|Wednesday|2039Q3|N|N|N|2465971|2466151|2465632|2465906|N|N|N|N|N| +2465998|AAAAAAAAOMAKFCAA|2039-07-28|1674|7283|559|2039|4|7|28|3|2039|559|7283|Thursday|2039Q3|N|N|N|2465971|2466151|2465633|2465907|N|N|N|N|N| +2465999|AAAAAAAAPMAKFCAA|2039-07-29|1674|7283|559|2039|5|7|29|3|2039|559|7283|Friday|2039Q3|N|Y|N|2465971|2466151|2465634|2465908|N|N|N|N|N| +2466000|AAAAAAAAANAKFCAA|2039-07-30|1674|7283|559|2039|6|7|30|3|2039|559|7283|Saturday|2039Q3|N|Y|N|2465971|2466151|2465635|2465909|N|N|N|N|N| +2466001|AAAAAAAABNAKFCAA|2039-07-31|1674|7283|559|2039|0|7|31|3|2039|559|7283|Sunday|2039Q3|N|N|N|2465971|2466151|2465636|2465910|N|N|N|N|N| +2466002|AAAAAAAACNAKFCAA|2039-08-01|1675|7283|559|2039|1|8|1|3|2039|559|7283|Monday|2039Q3|N|N|N|2466002|2466213|2465637|2465911|N|N|N|N|N| +2466003|AAAAAAAADNAKFCAA|2039-08-02|1675|7284|559|2039|2|8|2|3|2039|559|7284|Tuesday|2039Q3|N|N|N|2466002|2466213|2465638|2465912|N|N|N|N|N| +2466004|AAAAAAAAENAKFCAA|2039-08-03|1675|7284|559|2039|3|8|3|3|2039|559|7284|Wednesday|2039Q3|N|N|N|2466002|2466213|2465639|2465913|N|N|N|N|N| +2466005|AAAAAAAAFNAKFCAA|2039-08-04|1675|7284|559|2039|4|8|4|3|2039|559|7284|Thursday|2039Q3|N|N|N|2466002|2466213|2465640|2465914|N|N|N|N|N| +2466006|AAAAAAAAGNAKFCAA|2039-08-05|1675|7284|559|2039|5|8|5|3|2039|559|7284|Friday|2039Q3|N|Y|N|2466002|2466213|2465641|2465915|N|N|N|N|N| +2466007|AAAAAAAAHNAKFCAA|2039-08-06|1675|7284|559|2039|6|8|6|3|2039|559|7284|Saturday|2039Q3|N|Y|N|2466002|2466213|2465642|2465916|N|N|N|N|N| +2466008|AAAAAAAAINAKFCAA|2039-08-07|1675|7284|559|2039|0|8|7|3|2039|559|7284|Sunday|2039Q3|N|N|N|2466002|2466213|2465643|2465917|N|N|N|N|N| +2466009|AAAAAAAAJNAKFCAA|2039-08-08|1675|7284|559|2039|1|8|8|3|2039|559|7284|Monday|2039Q3|N|N|N|2466002|2466213|2465644|2465918|N|N|N|N|N| +2466010|AAAAAAAAKNAKFCAA|2039-08-09|1675|7285|559|2039|2|8|9|3|2039|559|7285|Tuesday|2039Q3|N|N|N|2466002|2466213|2465645|2465919|N|N|N|N|N| +2466011|AAAAAAAALNAKFCAA|2039-08-10|1675|7285|559|2039|3|8|10|3|2039|559|7285|Wednesday|2039Q3|N|N|N|2466002|2466213|2465646|2465920|N|N|N|N|N| +2466012|AAAAAAAAMNAKFCAA|2039-08-11|1675|7285|559|2039|4|8|11|3|2039|559|7285|Thursday|2039Q3|N|N|N|2466002|2466213|2465647|2465921|N|N|N|N|N| +2466013|AAAAAAAANNAKFCAA|2039-08-12|1675|7285|559|2039|5|8|12|3|2039|559|7285|Friday|2039Q3|N|Y|N|2466002|2466213|2465648|2465922|N|N|N|N|N| +2466014|AAAAAAAAONAKFCAA|2039-08-13|1675|7285|559|2039|6|8|13|3|2039|559|7285|Saturday|2039Q3|N|Y|N|2466002|2466213|2465649|2465923|N|N|N|N|N| +2466015|AAAAAAAAPNAKFCAA|2039-08-14|1675|7285|559|2039|0|8|14|3|2039|559|7285|Sunday|2039Q3|N|N|N|2466002|2466213|2465650|2465924|N|N|N|N|N| +2466016|AAAAAAAAAOAKFCAA|2039-08-15|1675|7285|559|2039|1|8|15|3|2039|559|7285|Monday|2039Q3|N|N|N|2466002|2466213|2465651|2465925|N|N|N|N|N| +2466017|AAAAAAAABOAKFCAA|2039-08-16|1675|7286|559|2039|2|8|16|3|2039|559|7286|Tuesday|2039Q3|N|N|N|2466002|2466213|2465652|2465926|N|N|N|N|N| +2466018|AAAAAAAACOAKFCAA|2039-08-17|1675|7286|559|2039|3|8|17|3|2039|559|7286|Wednesday|2039Q3|N|N|N|2466002|2466213|2465653|2465927|N|N|N|N|N| +2466019|AAAAAAAADOAKFCAA|2039-08-18|1675|7286|559|2039|4|8|18|3|2039|559|7286|Thursday|2039Q3|N|N|N|2466002|2466213|2465654|2465928|N|N|N|N|N| +2466020|AAAAAAAAEOAKFCAA|2039-08-19|1675|7286|559|2039|5|8|19|3|2039|559|7286|Friday|2039Q3|N|Y|N|2466002|2466213|2465655|2465929|N|N|N|N|N| +2466021|AAAAAAAAFOAKFCAA|2039-08-20|1675|7286|559|2039|6|8|20|3|2039|559|7286|Saturday|2039Q3|N|Y|N|2466002|2466213|2465656|2465930|N|N|N|N|N| +2466022|AAAAAAAAGOAKFCAA|2039-08-21|1675|7286|559|2039|0|8|21|3|2039|559|7286|Sunday|2039Q3|N|N|N|2466002|2466213|2465657|2465931|N|N|N|N|N| +2466023|AAAAAAAAHOAKFCAA|2039-08-22|1675|7286|559|2039|1|8|22|3|2039|559|7286|Monday|2039Q3|N|N|N|2466002|2466213|2465658|2465932|N|N|N|N|N| +2466024|AAAAAAAAIOAKFCAA|2039-08-23|1675|7287|559|2039|2|8|23|3|2039|559|7287|Tuesday|2039Q3|N|N|N|2466002|2466213|2465659|2465933|N|N|N|N|N| +2466025|AAAAAAAAJOAKFCAA|2039-08-24|1675|7287|559|2039|3|8|24|3|2039|559|7287|Wednesday|2039Q3|N|N|N|2466002|2466213|2465660|2465934|N|N|N|N|N| +2466026|AAAAAAAAKOAKFCAA|2039-08-25|1675|7287|559|2039|4|8|25|3|2039|559|7287|Thursday|2039Q3|N|N|N|2466002|2466213|2465661|2465935|N|N|N|N|N| +2466027|AAAAAAAALOAKFCAA|2039-08-26|1675|7287|559|2039|5|8|26|3|2039|559|7287|Friday|2039Q3|N|Y|N|2466002|2466213|2465662|2465936|N|N|N|N|N| +2466028|AAAAAAAAMOAKFCAA|2039-08-27|1675|7287|559|2039|6|8|27|3|2039|559|7287|Saturday|2039Q3|N|Y|N|2466002|2466213|2465663|2465937|N|N|N|N|N| +2466029|AAAAAAAANOAKFCAA|2039-08-28|1675|7287|559|2039|0|8|28|3|2039|559|7287|Sunday|2039Q3|N|N|N|2466002|2466213|2465664|2465938|N|N|N|N|N| +2466030|AAAAAAAAOOAKFCAA|2039-08-29|1675|7287|559|2039|1|8|29|3|2039|559|7287|Monday|2039Q3|N|N|N|2466002|2466213|2465665|2465939|N|N|N|N|N| +2466031|AAAAAAAAPOAKFCAA|2039-08-30|1675|7288|559|2039|2|8|30|3|2039|559|7288|Tuesday|2039Q3|N|N|N|2466002|2466213|2465666|2465940|N|N|N|N|N| +2466032|AAAAAAAAAPAKFCAA|2039-08-31|1675|7288|559|2039|3|8|31|3|2039|559|7288|Wednesday|2039Q3|N|N|N|2466002|2466213|2465667|2465941|N|N|N|N|N| +2466033|AAAAAAAABPAKFCAA|2039-09-01|1676|7288|560|2039|4|9|1|3|2039|560|7288|Thursday|2039Q3|N|N|N|2466033|2466275|2465668|2465942|N|N|N|N|N| +2466034|AAAAAAAACPAKFCAA|2039-09-02|1676|7288|560|2039|5|9|2|3|2039|560|7288|Friday|2039Q3|N|Y|N|2466033|2466275|2465669|2465943|N|N|N|N|N| +2466035|AAAAAAAADPAKFCAA|2039-09-03|1676|7288|560|2039|6|9|3|3|2039|560|7288|Saturday|2039Q3|N|Y|N|2466033|2466275|2465670|2465944|N|N|N|N|N| +2466036|AAAAAAAAEPAKFCAA|2039-09-04|1676|7288|560|2039|0|9|4|3|2039|560|7288|Sunday|2039Q3|N|N|N|2466033|2466275|2465671|2465945|N|N|N|N|N| +2466037|AAAAAAAAFPAKFCAA|2039-09-05|1676|7288|560|2039|1|9|5|3|2039|560|7288|Monday|2039Q3|N|N|N|2466033|2466275|2465672|2465946|N|N|N|N|N| +2466038|AAAAAAAAGPAKFCAA|2039-09-06|1676|7289|560|2039|2|9|6|3|2039|560|7289|Tuesday|2039Q3|N|N|N|2466033|2466275|2465673|2465947|N|N|N|N|N| +2466039|AAAAAAAAHPAKFCAA|2039-09-07|1676|7289|560|2039|3|9|7|3|2039|560|7289|Wednesday|2039Q3|N|N|N|2466033|2466275|2465674|2465948|N|N|N|N|N| +2466040|AAAAAAAAIPAKFCAA|2039-09-08|1676|7289|560|2039|4|9|8|3|2039|560|7289|Thursday|2039Q3|N|N|N|2466033|2466275|2465675|2465949|N|N|N|N|N| +2466041|AAAAAAAAJPAKFCAA|2039-09-09|1676|7289|560|2039|5|9|9|3|2039|560|7289|Friday|2039Q3|N|Y|N|2466033|2466275|2465676|2465950|N|N|N|N|N| +2466042|AAAAAAAAKPAKFCAA|2039-09-10|1676|7289|560|2039|6|9|10|3|2039|560|7289|Saturday|2039Q3|N|Y|N|2466033|2466275|2465677|2465951|N|N|N|N|N| +2466043|AAAAAAAALPAKFCAA|2039-09-11|1676|7289|560|2039|0|9|11|3|2039|560|7289|Sunday|2039Q3|N|N|N|2466033|2466275|2465678|2465952|N|N|N|N|N| +2466044|AAAAAAAAMPAKFCAA|2039-09-12|1676|7289|560|2039|1|9|12|3|2039|560|7289|Monday|2039Q3|N|N|N|2466033|2466275|2465679|2465953|N|N|N|N|N| +2466045|AAAAAAAANPAKFCAA|2039-09-13|1676|7290|560|2039|2|9|13|3|2039|560|7290|Tuesday|2039Q3|N|N|N|2466033|2466275|2465680|2465954|N|N|N|N|N| +2466046|AAAAAAAAOPAKFCAA|2039-09-14|1676|7290|560|2039|3|9|14|3|2039|560|7290|Wednesday|2039Q3|N|N|N|2466033|2466275|2465681|2465955|N|N|N|N|N| +2466047|AAAAAAAAPPAKFCAA|2039-09-15|1676|7290|560|2039|4|9|15|3|2039|560|7290|Thursday|2039Q3|N|N|N|2466033|2466275|2465682|2465956|N|N|N|N|N| +2466048|AAAAAAAAAABKFCAA|2039-09-16|1676|7290|560|2039|5|9|16|3|2039|560|7290|Friday|2039Q3|N|Y|N|2466033|2466275|2465683|2465957|N|N|N|N|N| +2466049|AAAAAAAABABKFCAA|2039-09-17|1676|7290|560|2039|6|9|17|3|2039|560|7290|Saturday|2039Q3|N|Y|N|2466033|2466275|2465684|2465958|N|N|N|N|N| +2466050|AAAAAAAACABKFCAA|2039-09-18|1676|7290|560|2039|0|9|18|3|2039|560|7290|Sunday|2039Q3|N|N|N|2466033|2466275|2465685|2465959|N|N|N|N|N| +2466051|AAAAAAAADABKFCAA|2039-09-19|1676|7290|560|2039|1|9|19|3|2039|560|7290|Monday|2039Q3|N|N|N|2466033|2466275|2465686|2465960|N|N|N|N|N| +2466052|AAAAAAAAEABKFCAA|2039-09-20|1676|7291|560|2039|2|9|20|3|2039|560|7291|Tuesday|2039Q3|N|N|N|2466033|2466275|2465687|2465961|N|N|N|N|N| +2466053|AAAAAAAAFABKFCAA|2039-09-21|1676|7291|560|2039|3|9|21|3|2039|560|7291|Wednesday|2039Q3|N|N|N|2466033|2466275|2465688|2465962|N|N|N|N|N| +2466054|AAAAAAAAGABKFCAA|2039-09-22|1676|7291|560|2039|4|9|22|3|2039|560|7291|Thursday|2039Q3|N|N|N|2466033|2466275|2465689|2465963|N|N|N|N|N| +2466055|AAAAAAAAHABKFCAA|2039-09-23|1676|7291|560|2039|5|9|23|3|2039|560|7291|Friday|2039Q3|N|Y|N|2466033|2466275|2465690|2465964|N|N|N|N|N| +2466056|AAAAAAAAIABKFCAA|2039-09-24|1676|7291|560|2039|6|9|24|3|2039|560|7291|Saturday|2039Q3|N|Y|N|2466033|2466275|2465691|2465965|N|N|N|N|N| +2466057|AAAAAAAAJABKFCAA|2039-09-25|1676|7291|560|2039|0|9|25|3|2039|560|7291|Sunday|2039Q3|N|N|N|2466033|2466275|2465692|2465966|N|N|N|N|N| +2466058|AAAAAAAAKABKFCAA|2039-09-26|1676|7291|560|2039|1|9|26|3|2039|560|7291|Monday|2039Q3|N|N|N|2466033|2466275|2465693|2465967|N|N|N|N|N| +2466059|AAAAAAAALABKFCAA|2039-09-27|1676|7292|560|2039|2|9|27|3|2039|560|7292|Tuesday|2039Q3|N|N|N|2466033|2466275|2465694|2465968|N|N|N|N|N| +2466060|AAAAAAAAMABKFCAA|2039-09-28|1676|7292|560|2039|3|9|28|3|2039|560|7292|Wednesday|2039Q3|N|N|N|2466033|2466275|2465695|2465969|N|N|N|N|N| +2466061|AAAAAAAANABKFCAA|2039-09-29|1676|7292|560|2039|4|9|29|3|2039|560|7292|Thursday|2039Q3|N|N|N|2466033|2466275|2465696|2465970|N|N|N|N|N| +2466062|AAAAAAAAOABKFCAA|2039-09-30|1676|7292|560|2039|5|9|30|3|2039|560|7292|Friday|2039Q3|N|Y|N|2466033|2466275|2465697|2465971|N|N|N|N|N| +2466063|AAAAAAAAPABKFCAA|2039-10-01|1677|7292|560|2039|6|10|1|3|2039|560|7292|Saturday|2039Q3|N|Y|N|2466063|2466335|2465698|2465971|N|N|N|N|N| +2466064|AAAAAAAAABBKFCAA|2039-10-02|1677|7292|560|2039|0|10|2|4|2039|560|7292|Sunday|2039Q4|N|N|N|2466063|2466335|2465699|2465972|N|N|N|N|N| +2466065|AAAAAAAABBBKFCAA|2039-10-03|1677|7292|560|2039|1|10|3|4|2039|560|7292|Monday|2039Q4|N|N|N|2466063|2466335|2465700|2465973|N|N|N|N|N| +2466066|AAAAAAAACBBKFCAA|2039-10-04|1677|7293|560|2039|2|10|4|4|2039|560|7293|Tuesday|2039Q4|N|N|N|2466063|2466335|2465701|2465974|N|N|N|N|N| +2466067|AAAAAAAADBBKFCAA|2039-10-05|1677|7293|560|2039|3|10|5|4|2039|560|7293|Wednesday|2039Q4|N|N|N|2466063|2466335|2465702|2465975|N|N|N|N|N| +2466068|AAAAAAAAEBBKFCAA|2039-10-06|1677|7293|560|2039|4|10|6|4|2039|560|7293|Thursday|2039Q4|N|N|N|2466063|2466335|2465703|2465976|N|N|N|N|N| +2466069|AAAAAAAAFBBKFCAA|2039-10-07|1677|7293|560|2039|5|10|7|4|2039|560|7293|Friday|2039Q4|N|Y|N|2466063|2466335|2465704|2465977|N|N|N|N|N| +2466070|AAAAAAAAGBBKFCAA|2039-10-08|1677|7293|560|2039|6|10|8|4|2039|560|7293|Saturday|2039Q4|N|Y|N|2466063|2466335|2465705|2465978|N|N|N|N|N| +2466071|AAAAAAAAHBBKFCAA|2039-10-09|1677|7293|560|2039|0|10|9|4|2039|560|7293|Sunday|2039Q4|N|N|N|2466063|2466335|2465706|2465979|N|N|N|N|N| +2466072|AAAAAAAAIBBKFCAA|2039-10-10|1677|7293|560|2039|1|10|10|4|2039|560|7293|Monday|2039Q4|N|N|N|2466063|2466335|2465707|2465980|N|N|N|N|N| +2466073|AAAAAAAAJBBKFCAA|2039-10-11|1677|7294|560|2039|2|10|11|4|2039|560|7294|Tuesday|2039Q4|N|N|N|2466063|2466335|2465708|2465981|N|N|N|N|N| +2466074|AAAAAAAAKBBKFCAA|2039-10-12|1677|7294|560|2039|3|10|12|4|2039|560|7294|Wednesday|2039Q4|N|N|N|2466063|2466335|2465709|2465982|N|N|N|N|N| +2466075|AAAAAAAALBBKFCAA|2039-10-13|1677|7294|560|2039|4|10|13|4|2039|560|7294|Thursday|2039Q4|N|N|N|2466063|2466335|2465710|2465983|N|N|N|N|N| +2466076|AAAAAAAAMBBKFCAA|2039-10-14|1677|7294|560|2039|5|10|14|4|2039|560|7294|Friday|2039Q4|N|Y|N|2466063|2466335|2465711|2465984|N|N|N|N|N| +2466077|AAAAAAAANBBKFCAA|2039-10-15|1677|7294|560|2039|6|10|15|4|2039|560|7294|Saturday|2039Q4|N|Y|N|2466063|2466335|2465712|2465985|N|N|N|N|N| +2466078|AAAAAAAAOBBKFCAA|2039-10-16|1677|7294|560|2039|0|10|16|4|2039|560|7294|Sunday|2039Q4|N|N|N|2466063|2466335|2465713|2465986|N|N|N|N|N| +2466079|AAAAAAAAPBBKFCAA|2039-10-17|1677|7294|560|2039|1|10|17|4|2039|560|7294|Monday|2039Q4|N|N|N|2466063|2466335|2465714|2465987|N|N|N|N|N| +2466080|AAAAAAAAACBKFCAA|2039-10-18|1677|7295|560|2039|2|10|18|4|2039|560|7295|Tuesday|2039Q4|N|N|N|2466063|2466335|2465715|2465988|N|N|N|N|N| +2466081|AAAAAAAABCBKFCAA|2039-10-19|1677|7295|560|2039|3|10|19|4|2039|560|7295|Wednesday|2039Q4|N|N|N|2466063|2466335|2465716|2465989|N|N|N|N|N| +2466082|AAAAAAAACCBKFCAA|2039-10-20|1677|7295|560|2039|4|10|20|4|2039|560|7295|Thursday|2039Q4|N|N|N|2466063|2466335|2465717|2465990|N|N|N|N|N| +2466083|AAAAAAAADCBKFCAA|2039-10-21|1677|7295|560|2039|5|10|21|4|2039|560|7295|Friday|2039Q4|N|Y|N|2466063|2466335|2465718|2465991|N|N|N|N|N| +2466084|AAAAAAAAECBKFCAA|2039-10-22|1677|7295|560|2039|6|10|22|4|2039|560|7295|Saturday|2039Q4|N|Y|N|2466063|2466335|2465719|2465992|N|N|N|N|N| +2466085|AAAAAAAAFCBKFCAA|2039-10-23|1677|7295|560|2039|0|10|23|4|2039|560|7295|Sunday|2039Q4|N|N|N|2466063|2466335|2465720|2465993|N|N|N|N|N| +2466086|AAAAAAAAGCBKFCAA|2039-10-24|1677|7295|560|2039|1|10|24|4|2039|560|7295|Monday|2039Q4|N|N|N|2466063|2466335|2465721|2465994|N|N|N|N|N| +2466087|AAAAAAAAHCBKFCAA|2039-10-25|1677|7296|560|2039|2|10|25|4|2039|560|7296|Tuesday|2039Q4|N|N|N|2466063|2466335|2465722|2465995|N|N|N|N|N| +2466088|AAAAAAAAICBKFCAA|2039-10-26|1677|7296|560|2039|3|10|26|4|2039|560|7296|Wednesday|2039Q4|N|N|N|2466063|2466335|2465723|2465996|N|N|N|N|N| +2466089|AAAAAAAAJCBKFCAA|2039-10-27|1677|7296|560|2039|4|10|27|4|2039|560|7296|Thursday|2039Q4|N|N|N|2466063|2466335|2465724|2465997|N|N|N|N|N| +2466090|AAAAAAAAKCBKFCAA|2039-10-28|1677|7296|560|2039|5|10|28|4|2039|560|7296|Friday|2039Q4|N|Y|N|2466063|2466335|2465725|2465998|N|N|N|N|N| +2466091|AAAAAAAALCBKFCAA|2039-10-29|1677|7296|560|2039|6|10|29|4|2039|560|7296|Saturday|2039Q4|N|Y|N|2466063|2466335|2465726|2465999|N|N|N|N|N| +2466092|AAAAAAAAMCBKFCAA|2039-10-30|1677|7296|560|2039|0|10|30|4|2039|560|7296|Sunday|2039Q4|N|N|N|2466063|2466335|2465727|2466000|N|N|N|N|N| +2466093|AAAAAAAANCBKFCAA|2039-10-31|1677|7296|560|2039|1|10|31|4|2039|560|7296|Monday|2039Q4|N|N|N|2466063|2466335|2465728|2466001|N|N|N|N|N| +2466094|AAAAAAAAOCBKFCAA|2039-11-01|1678|7297|560|2039|2|11|1|4|2039|560|7297|Tuesday|2039Q4|N|N|N|2466094|2466397|2465729|2466002|N|N|N|N|N| +2466095|AAAAAAAAPCBKFCAA|2039-11-02|1678|7297|560|2039|3|11|2|4|2039|560|7297|Wednesday|2039Q4|N|N|N|2466094|2466397|2465730|2466003|N|N|N|N|N| +2466096|AAAAAAAAADBKFCAA|2039-11-03|1678|7297|560|2039|4|11|3|4|2039|560|7297|Thursday|2039Q4|N|N|N|2466094|2466397|2465731|2466004|N|N|N|N|N| +2466097|AAAAAAAABDBKFCAA|2039-11-04|1678|7297|560|2039|5|11|4|4|2039|560|7297|Friday|2039Q4|N|Y|N|2466094|2466397|2465732|2466005|N|N|N|N|N| +2466098|AAAAAAAACDBKFCAA|2039-11-05|1678|7297|560|2039|6|11|5|4|2039|560|7297|Saturday|2039Q4|N|Y|N|2466094|2466397|2465733|2466006|N|N|N|N|N| +2466099|AAAAAAAADDBKFCAA|2039-11-06|1678|7297|560|2039|0|11|6|4|2039|560|7297|Sunday|2039Q4|N|N|N|2466094|2466397|2465734|2466007|N|N|N|N|N| +2466100|AAAAAAAAEDBKFCAA|2039-11-07|1678|7297|560|2039|1|11|7|4|2039|560|7297|Monday|2039Q4|N|N|N|2466094|2466397|2465735|2466008|N|N|N|N|N| +2466101|AAAAAAAAFDBKFCAA|2039-11-08|1678|7298|560|2039|2|11|8|4|2039|560|7298|Tuesday|2039Q4|N|N|N|2466094|2466397|2465736|2466009|N|N|N|N|N| +2466102|AAAAAAAAGDBKFCAA|2039-11-09|1678|7298|560|2039|3|11|9|4|2039|560|7298|Wednesday|2039Q4|N|N|N|2466094|2466397|2465737|2466010|N|N|N|N|N| +2466103|AAAAAAAAHDBKFCAA|2039-11-10|1678|7298|560|2039|4|11|10|4|2039|560|7298|Thursday|2039Q4|N|N|N|2466094|2466397|2465738|2466011|N|N|N|N|N| +2466104|AAAAAAAAIDBKFCAA|2039-11-11|1678|7298|560|2039|5|11|11|4|2039|560|7298|Friday|2039Q4|N|Y|N|2466094|2466397|2465739|2466012|N|N|N|N|N| +2466105|AAAAAAAAJDBKFCAA|2039-11-12|1678|7298|560|2039|6|11|12|4|2039|560|7298|Saturday|2039Q4|N|Y|N|2466094|2466397|2465740|2466013|N|N|N|N|N| +2466106|AAAAAAAAKDBKFCAA|2039-11-13|1678|7298|560|2039|0|11|13|4|2039|560|7298|Sunday|2039Q4|N|N|N|2466094|2466397|2465741|2466014|N|N|N|N|N| +2466107|AAAAAAAALDBKFCAA|2039-11-14|1678|7298|560|2039|1|11|14|4|2039|560|7298|Monday|2039Q4|N|N|N|2466094|2466397|2465742|2466015|N|N|N|N|N| +2466108|AAAAAAAAMDBKFCAA|2039-11-15|1678|7299|560|2039|2|11|15|4|2039|560|7299|Tuesday|2039Q4|N|N|N|2466094|2466397|2465743|2466016|N|N|N|N|N| +2466109|AAAAAAAANDBKFCAA|2039-11-16|1678|7299|560|2039|3|11|16|4|2039|560|7299|Wednesday|2039Q4|N|N|N|2466094|2466397|2465744|2466017|N|N|N|N|N| +2466110|AAAAAAAAODBKFCAA|2039-11-17|1678|7299|560|2039|4|11|17|4|2039|560|7299|Thursday|2039Q4|N|N|N|2466094|2466397|2465745|2466018|N|N|N|N|N| +2466111|AAAAAAAAPDBKFCAA|2039-11-18|1678|7299|560|2039|5|11|18|4|2039|560|7299|Friday|2039Q4|N|Y|N|2466094|2466397|2465746|2466019|N|N|N|N|N| +2466112|AAAAAAAAAEBKFCAA|2039-11-19|1678|7299|560|2039|6|11|19|4|2039|560|7299|Saturday|2039Q4|N|Y|N|2466094|2466397|2465747|2466020|N|N|N|N|N| +2466113|AAAAAAAABEBKFCAA|2039-11-20|1678|7299|560|2039|0|11|20|4|2039|560|7299|Sunday|2039Q4|N|N|N|2466094|2466397|2465748|2466021|N|N|N|N|N| +2466114|AAAAAAAACEBKFCAA|2039-11-21|1678|7299|560|2039|1|11|21|4|2039|560|7299|Monday|2039Q4|N|N|N|2466094|2466397|2465749|2466022|N|N|N|N|N| +2466115|AAAAAAAADEBKFCAA|2039-11-22|1678|7300|560|2039|2|11|22|4|2039|560|7300|Tuesday|2039Q4|N|N|N|2466094|2466397|2465750|2466023|N|N|N|N|N| +2466116|AAAAAAAAEEBKFCAA|2039-11-23|1678|7300|560|2039|3|11|23|4|2039|560|7300|Wednesday|2039Q4|N|N|N|2466094|2466397|2465751|2466024|N|N|N|N|N| +2466117|AAAAAAAAFEBKFCAA|2039-11-24|1678|7300|560|2039|4|11|24|4|2039|560|7300|Thursday|2039Q4|N|N|N|2466094|2466397|2465752|2466025|N|N|N|N|N| +2466118|AAAAAAAAGEBKFCAA|2039-11-25|1678|7300|560|2039|5|11|25|4|2039|560|7300|Friday|2039Q4|N|Y|N|2466094|2466397|2465753|2466026|N|N|N|N|N| +2466119|AAAAAAAAHEBKFCAA|2039-11-26|1678|7300|560|2039|6|11|26|4|2039|560|7300|Saturday|2039Q4|N|Y|N|2466094|2466397|2465754|2466027|N|N|N|N|N| +2466120|AAAAAAAAIEBKFCAA|2039-11-27|1678|7300|560|2039|0|11|27|4|2039|560|7300|Sunday|2039Q4|N|N|N|2466094|2466397|2465755|2466028|N|N|N|N|N| +2466121|AAAAAAAAJEBKFCAA|2039-11-28|1678|7300|560|2039|1|11|28|4|2039|560|7300|Monday|2039Q4|N|N|N|2466094|2466397|2465756|2466029|N|N|N|N|N| +2466122|AAAAAAAAKEBKFCAA|2039-11-29|1678|7301|560|2039|2|11|29|4|2039|560|7301|Tuesday|2039Q4|N|N|N|2466094|2466397|2465757|2466030|N|N|N|N|N| +2466123|AAAAAAAALEBKFCAA|2039-11-30|1678|7301|560|2039|3|11|30|4|2039|560|7301|Wednesday|2039Q4|N|N|N|2466094|2466397|2465758|2466031|N|N|N|N|N| +2466124|AAAAAAAAMEBKFCAA|2039-12-01|1679|7301|561|2039|4|12|1|4|2039|561|7301|Thursday|2039Q4|N|N|N|2466124|2466457|2465759|2466032|N|N|N|N|N| +2466125|AAAAAAAANEBKFCAA|2039-12-02|1679|7301|561|2039|5|12|2|4|2039|561|7301|Friday|2039Q4|N|Y|N|2466124|2466457|2465760|2466033|N|N|N|N|N| +2466126|AAAAAAAAOEBKFCAA|2039-12-03|1679|7301|561|2039|6|12|3|4|2039|561|7301|Saturday|2039Q4|N|Y|N|2466124|2466457|2465761|2466034|N|N|N|N|N| +2466127|AAAAAAAAPEBKFCAA|2039-12-04|1679|7301|561|2039|0|12|4|4|2039|561|7301|Sunday|2039Q4|N|N|N|2466124|2466457|2465762|2466035|N|N|N|N|N| +2466128|AAAAAAAAAFBKFCAA|2039-12-05|1679|7301|561|2039|1|12|5|4|2039|561|7301|Monday|2039Q4|N|N|N|2466124|2466457|2465763|2466036|N|N|N|N|N| +2466129|AAAAAAAABFBKFCAA|2039-12-06|1679|7302|561|2039|2|12|6|4|2039|561|7302|Tuesday|2039Q4|N|N|N|2466124|2466457|2465764|2466037|N|N|N|N|N| +2466130|AAAAAAAACFBKFCAA|2039-12-07|1679|7302|561|2039|3|12|7|4|2039|561|7302|Wednesday|2039Q4|N|N|N|2466124|2466457|2465765|2466038|N|N|N|N|N| +2466131|AAAAAAAADFBKFCAA|2039-12-08|1679|7302|561|2039|4|12|8|4|2039|561|7302|Thursday|2039Q4|N|N|N|2466124|2466457|2465766|2466039|N|N|N|N|N| +2466132|AAAAAAAAEFBKFCAA|2039-12-09|1679|7302|561|2039|5|12|9|4|2039|561|7302|Friday|2039Q4|N|Y|N|2466124|2466457|2465767|2466040|N|N|N|N|N| +2466133|AAAAAAAAFFBKFCAA|2039-12-10|1679|7302|561|2039|6|12|10|4|2039|561|7302|Saturday|2039Q4|N|Y|N|2466124|2466457|2465768|2466041|N|N|N|N|N| +2466134|AAAAAAAAGFBKFCAA|2039-12-11|1679|7302|561|2039|0|12|11|4|2039|561|7302|Sunday|2039Q4|N|N|N|2466124|2466457|2465769|2466042|N|N|N|N|N| +2466135|AAAAAAAAHFBKFCAA|2039-12-12|1679|7302|561|2039|1|12|12|4|2039|561|7302|Monday|2039Q4|N|N|N|2466124|2466457|2465770|2466043|N|N|N|N|N| +2466136|AAAAAAAAIFBKFCAA|2039-12-13|1679|7303|561|2039|2|12|13|4|2039|561|7303|Tuesday|2039Q4|N|N|N|2466124|2466457|2465771|2466044|N|N|N|N|N| +2466137|AAAAAAAAJFBKFCAA|2039-12-14|1679|7303|561|2039|3|12|14|4|2039|561|7303|Wednesday|2039Q4|N|N|N|2466124|2466457|2465772|2466045|N|N|N|N|N| +2466138|AAAAAAAAKFBKFCAA|2039-12-15|1679|7303|561|2039|4|12|15|4|2039|561|7303|Thursday|2039Q4|N|N|N|2466124|2466457|2465773|2466046|N|N|N|N|N| +2466139|AAAAAAAALFBKFCAA|2039-12-16|1679|7303|561|2039|5|12|16|4|2039|561|7303|Friday|2039Q4|N|Y|N|2466124|2466457|2465774|2466047|N|N|N|N|N| +2466140|AAAAAAAAMFBKFCAA|2039-12-17|1679|7303|561|2039|6|12|17|4|2039|561|7303|Saturday|2039Q4|N|Y|N|2466124|2466457|2465775|2466048|N|N|N|N|N| +2466141|AAAAAAAANFBKFCAA|2039-12-18|1679|7303|561|2039|0|12|18|4|2039|561|7303|Sunday|2039Q4|N|N|N|2466124|2466457|2465776|2466049|N|N|N|N|N| +2466142|AAAAAAAAOFBKFCAA|2039-12-19|1679|7303|561|2039|1|12|19|4|2039|561|7303|Monday|2039Q4|N|N|N|2466124|2466457|2465777|2466050|N|N|N|N|N| +2466143|AAAAAAAAPFBKFCAA|2039-12-20|1679|7304|561|2039|2|12|20|4|2039|561|7304|Tuesday|2039Q4|N|N|N|2466124|2466457|2465778|2466051|N|N|N|N|N| +2466144|AAAAAAAAAGBKFCAA|2039-12-21|1679|7304|561|2039|3|12|21|4|2039|561|7304|Wednesday|2039Q4|N|N|N|2466124|2466457|2465779|2466052|N|N|N|N|N| +2466145|AAAAAAAABGBKFCAA|2039-12-22|1679|7304|561|2039|4|12|22|4|2039|561|7304|Thursday|2039Q4|N|N|N|2466124|2466457|2465780|2466053|N|N|N|N|N| +2466146|AAAAAAAACGBKFCAA|2039-12-23|1679|7304|561|2039|5|12|23|4|2039|561|7304|Friday|2039Q4|N|Y|N|2466124|2466457|2465781|2466054|N|N|N|N|N| +2466147|AAAAAAAADGBKFCAA|2039-12-24|1679|7304|561|2039|6|12|24|4|2039|561|7304|Saturday|2039Q4|N|Y|N|2466124|2466457|2465782|2466055|N|N|N|N|N| +2466148|AAAAAAAAEGBKFCAA|2039-12-25|1679|7304|561|2039|0|12|25|4|2039|561|7304|Sunday|2039Q4|N|N|N|2466124|2466457|2465783|2466056|N|N|N|N|N| +2466149|AAAAAAAAFGBKFCAA|2039-12-26|1679|7304|561|2039|1|12|26|4|2039|561|7304|Monday|2039Q4|Y|N|N|2466124|2466457|2465784|2466057|N|N|N|N|N| +2466150|AAAAAAAAGGBKFCAA|2039-12-27|1679|7305|561|2039|2|12|27|4|2039|561|7305|Tuesday|2039Q4|N|N|Y|2466124|2466457|2465785|2466058|N|N|N|N|N| +2466151|AAAAAAAAHGBKFCAA|2039-12-28|1679|7305|561|2039|3|12|28|4|2039|561|7305|Wednesday|2039Q4|N|N|N|2466124|2466457|2465786|2466059|N|N|N|N|N| +2466152|AAAAAAAAIGBKFCAA|2039-12-29|1679|7305|561|2039|4|12|29|4|2039|561|7305|Thursday|2039Q4|N|N|N|2466124|2466457|2465787|2466060|N|N|N|N|N| +2466153|AAAAAAAAJGBKFCAA|2039-12-30|1679|7305|561|2039|5|12|30|4|2039|561|7305|Friday|2039Q4|N|Y|N|2466124|2466457|2465788|2466061|N|N|N|N|N| +2466154|AAAAAAAAKGBKFCAA|2039-12-31|1679|7305|561|2039|6|12|31|4|2039|561|7305|Saturday|2039Q4|N|Y|N|2466124|2466457|2465789|2466062|N|N|N|N|N| +2466155|AAAAAAAALGBKFCAA|2040-01-01|1680|7305|561|2040|0|1|1|1|2040|561|7305|Sunday|2040Q1|Y|N|N|2466155|2466154|2465790|2466063|N|N|N|N|N| +2466156|AAAAAAAAMGBKFCAA|2040-01-02|1680|7305|561|2040|1|1|2|1|2040|561|7305|Monday|2040Q1|N|N|Y|2466155|2466154|2465791|2466064|N|N|N|N|N| +2466157|AAAAAAAANGBKFCAA|2040-01-03|1680|7306|561|2040|2|1|3|1|2040|561|7306|Tuesday|2040Q1|N|N|N|2466155|2466154|2465792|2466065|N|N|N|N|N| +2466158|AAAAAAAAOGBKFCAA|2040-01-04|1680|7306|561|2040|3|1|4|1|2040|561|7306|Wednesday|2040Q1|N|N|N|2466155|2466154|2465793|2466066|N|N|N|N|N| +2466159|AAAAAAAAPGBKFCAA|2040-01-05|1680|7306|561|2040|4|1|5|1|2040|561|7306|Thursday|2040Q1|N|N|N|2466155|2466154|2465794|2466067|N|N|N|N|N| +2466160|AAAAAAAAAHBKFCAA|2040-01-06|1680|7306|561|2040|5|1|6|1|2040|561|7306|Friday|2040Q1|N|Y|N|2466155|2466154|2465795|2466068|N|N|N|N|N| +2466161|AAAAAAAABHBKFCAA|2040-01-07|1680|7306|561|2040|6|1|7|1|2040|561|7306|Saturday|2040Q1|N|Y|N|2466155|2466154|2465796|2466069|N|N|N|N|N| +2466162|AAAAAAAACHBKFCAA|2040-01-08|1680|7306|561|2040|0|1|8|1|2040|561|7306|Sunday|2040Q1|N|N|N|2466155|2466154|2465797|2466070|N|N|N|N|N| +2466163|AAAAAAAADHBKFCAA|2040-01-09|1680|7306|561|2040|1|1|9|1|2040|561|7306|Monday|2040Q1|N|N|N|2466155|2466154|2465798|2466071|N|N|N|N|N| +2466164|AAAAAAAAEHBKFCAA|2040-01-10|1680|7307|561|2040|2|1|10|1|2040|561|7307|Tuesday|2040Q1|N|N|N|2466155|2466154|2465799|2466072|N|N|N|N|N| +2466165|AAAAAAAAFHBKFCAA|2040-01-11|1680|7307|561|2040|3|1|11|1|2040|561|7307|Wednesday|2040Q1|N|N|N|2466155|2466154|2465800|2466073|N|N|N|N|N| +2466166|AAAAAAAAGHBKFCAA|2040-01-12|1680|7307|561|2040|4|1|12|1|2040|561|7307|Thursday|2040Q1|N|N|N|2466155|2466154|2465801|2466074|N|N|N|N|N| +2466167|AAAAAAAAHHBKFCAA|2040-01-13|1680|7307|561|2040|5|1|13|1|2040|561|7307|Friday|2040Q1|N|Y|N|2466155|2466154|2465802|2466075|N|N|N|N|N| +2466168|AAAAAAAAIHBKFCAA|2040-01-14|1680|7307|561|2040|6|1|14|1|2040|561|7307|Saturday|2040Q1|N|Y|N|2466155|2466154|2465803|2466076|N|N|N|N|N| +2466169|AAAAAAAAJHBKFCAA|2040-01-15|1680|7307|561|2040|0|1|15|1|2040|561|7307|Sunday|2040Q1|N|N|N|2466155|2466154|2465804|2466077|N|N|N|N|N| +2466170|AAAAAAAAKHBKFCAA|2040-01-16|1680|7307|561|2040|1|1|16|1|2040|561|7307|Monday|2040Q1|N|N|N|2466155|2466154|2465805|2466078|N|N|N|N|N| +2466171|AAAAAAAALHBKFCAA|2040-01-17|1680|7308|561|2040|2|1|17|1|2040|561|7308|Tuesday|2040Q1|N|N|N|2466155|2466154|2465806|2466079|N|N|N|N|N| +2466172|AAAAAAAAMHBKFCAA|2040-01-18|1680|7308|561|2040|3|1|18|1|2040|561|7308|Wednesday|2040Q1|N|N|N|2466155|2466154|2465807|2466080|N|N|N|N|N| +2466173|AAAAAAAANHBKFCAA|2040-01-19|1680|7308|561|2040|4|1|19|1|2040|561|7308|Thursday|2040Q1|N|N|N|2466155|2466154|2465808|2466081|N|N|N|N|N| +2466174|AAAAAAAAOHBKFCAA|2040-01-20|1680|7308|561|2040|5|1|20|1|2040|561|7308|Friday|2040Q1|N|Y|N|2466155|2466154|2465809|2466082|N|N|N|N|N| +2466175|AAAAAAAAPHBKFCAA|2040-01-21|1680|7308|561|2040|6|1|21|1|2040|561|7308|Saturday|2040Q1|N|Y|N|2466155|2466154|2465810|2466083|N|N|N|N|N| +2466176|AAAAAAAAAIBKFCAA|2040-01-22|1680|7308|561|2040|0|1|22|1|2040|561|7308|Sunday|2040Q1|N|N|N|2466155|2466154|2465811|2466084|N|N|N|N|N| +2466177|AAAAAAAABIBKFCAA|2040-01-23|1680|7308|561|2040|1|1|23|1|2040|561|7308|Monday|2040Q1|N|N|N|2466155|2466154|2465812|2466085|N|N|N|N|N| +2466178|AAAAAAAACIBKFCAA|2040-01-24|1680|7309|561|2040|2|1|24|1|2040|561|7309|Tuesday|2040Q1|N|N|N|2466155|2466154|2465813|2466086|N|N|N|N|N| +2466179|AAAAAAAADIBKFCAA|2040-01-25|1680|7309|561|2040|3|1|25|1|2040|561|7309|Wednesday|2040Q1|N|N|N|2466155|2466154|2465814|2466087|N|N|N|N|N| +2466180|AAAAAAAAEIBKFCAA|2040-01-26|1680|7309|561|2040|4|1|26|1|2040|561|7309|Thursday|2040Q1|N|N|N|2466155|2466154|2465815|2466088|N|N|N|N|N| +2466181|AAAAAAAAFIBKFCAA|2040-01-27|1680|7309|561|2040|5|1|27|1|2040|561|7309|Friday|2040Q1|N|Y|N|2466155|2466154|2465816|2466089|N|N|N|N|N| +2466182|AAAAAAAAGIBKFCAA|2040-01-28|1680|7309|561|2040|6|1|28|1|2040|561|7309|Saturday|2040Q1|N|Y|N|2466155|2466154|2465817|2466090|N|N|N|N|N| +2466183|AAAAAAAAHIBKFCAA|2040-01-29|1680|7309|561|2040|0|1|29|1|2040|561|7309|Sunday|2040Q1|N|N|N|2466155|2466154|2465818|2466091|N|N|N|N|N| +2466184|AAAAAAAAIIBKFCAA|2040-01-30|1680|7309|561|2040|1|1|30|1|2040|561|7309|Monday|2040Q1|N|N|N|2466155|2466154|2465819|2466092|N|N|N|N|N| +2466185|AAAAAAAAJIBKFCAA|2040-01-31|1680|7310|561|2040|2|1|31|1|2040|561|7310|Tuesday|2040Q1|N|N|N|2466155|2466154|2465820|2466093|N|N|N|N|N| +2466186|AAAAAAAAKIBKFCAA|2040-02-01|1681|7310|561|2040|3|2|1|1|2040|561|7310|Wednesday|2040Q1|N|N|N|2466186|2466216|2465821|2466094|N|N|N|N|N| +2466187|AAAAAAAALIBKFCAA|2040-02-02|1681|7310|561|2040|4|2|2|1|2040|561|7310|Thursday|2040Q1|N|N|N|2466186|2466216|2465822|2466095|N|N|N|N|N| +2466188|AAAAAAAAMIBKFCAA|2040-02-03|1681|7310|561|2040|5|2|3|1|2040|561|7310|Friday|2040Q1|N|Y|N|2466186|2466216|2465823|2466096|N|N|N|N|N| +2466189|AAAAAAAANIBKFCAA|2040-02-04|1681|7310|561|2040|6|2|4|1|2040|561|7310|Saturday|2040Q1|N|Y|N|2466186|2466216|2465824|2466097|N|N|N|N|N| +2466190|AAAAAAAAOIBKFCAA|2040-02-05|1681|7310|561|2040|0|2|5|1|2040|561|7310|Sunday|2040Q1|N|N|N|2466186|2466216|2465825|2466098|N|N|N|N|N| +2466191|AAAAAAAAPIBKFCAA|2040-02-06|1681|7310|561|2040|1|2|6|1|2040|561|7310|Monday|2040Q1|N|N|N|2466186|2466216|2465826|2466099|N|N|N|N|N| +2466192|AAAAAAAAAJBKFCAA|2040-02-07|1681|7311|561|2040|2|2|7|1|2040|561|7311|Tuesday|2040Q1|N|N|N|2466186|2466216|2465827|2466100|N|N|N|N|N| +2466193|AAAAAAAABJBKFCAA|2040-02-08|1681|7311|561|2040|3|2|8|1|2040|561|7311|Wednesday|2040Q1|N|N|N|2466186|2466216|2465828|2466101|N|N|N|N|N| +2466194|AAAAAAAACJBKFCAA|2040-02-09|1681|7311|561|2040|4|2|9|1|2040|561|7311|Thursday|2040Q1|N|N|N|2466186|2466216|2465829|2466102|N|N|N|N|N| +2466195|AAAAAAAADJBKFCAA|2040-02-10|1681|7311|561|2040|5|2|10|1|2040|561|7311|Friday|2040Q1|N|Y|N|2466186|2466216|2465830|2466103|N|N|N|N|N| +2466196|AAAAAAAAEJBKFCAA|2040-02-11|1681|7311|561|2040|6|2|11|1|2040|561|7311|Saturday|2040Q1|N|Y|N|2466186|2466216|2465831|2466104|N|N|N|N|N| +2466197|AAAAAAAAFJBKFCAA|2040-02-12|1681|7311|561|2040|0|2|12|1|2040|561|7311|Sunday|2040Q1|N|N|N|2466186|2466216|2465832|2466105|N|N|N|N|N| +2466198|AAAAAAAAGJBKFCAA|2040-02-13|1681|7311|561|2040|1|2|13|1|2040|561|7311|Monday|2040Q1|N|N|N|2466186|2466216|2465833|2466106|N|N|N|N|N| +2466199|AAAAAAAAHJBKFCAA|2040-02-14|1681|7312|561|2040|2|2|14|1|2040|561|7312|Tuesday|2040Q1|N|N|N|2466186|2466216|2465834|2466107|N|N|N|N|N| +2466200|AAAAAAAAIJBKFCAA|2040-02-15|1681|7312|561|2040|3|2|15|1|2040|561|7312|Wednesday|2040Q1|N|N|N|2466186|2466216|2465835|2466108|N|N|N|N|N| +2466201|AAAAAAAAJJBKFCAA|2040-02-16|1681|7312|561|2040|4|2|16|1|2040|561|7312|Thursday|2040Q1|N|N|N|2466186|2466216|2465836|2466109|N|N|N|N|N| +2466202|AAAAAAAAKJBKFCAA|2040-02-17|1681|7312|561|2040|5|2|17|1|2040|561|7312|Friday|2040Q1|N|Y|N|2466186|2466216|2465837|2466110|N|N|N|N|N| +2466203|AAAAAAAALJBKFCAA|2040-02-18|1681|7312|561|2040|6|2|18|1|2040|561|7312|Saturday|2040Q1|N|Y|N|2466186|2466216|2465838|2466111|N|N|N|N|N| +2466204|AAAAAAAAMJBKFCAA|2040-02-19|1681|7312|561|2040|0|2|19|1|2040|561|7312|Sunday|2040Q1|N|N|N|2466186|2466216|2465839|2466112|N|N|N|N|N| +2466205|AAAAAAAANJBKFCAA|2040-02-20|1681|7312|561|2040|1|2|20|1|2040|561|7312|Monday|2040Q1|N|N|N|2466186|2466216|2465840|2466113|N|N|N|N|N| +2466206|AAAAAAAAOJBKFCAA|2040-02-21|1681|7313|561|2040|2|2|21|1|2040|561|7313|Tuesday|2040Q1|N|N|N|2466186|2466216|2465841|2466114|N|N|N|N|N| +2466207|AAAAAAAAPJBKFCAA|2040-02-22|1681|7313|561|2040|3|2|22|1|2040|561|7313|Wednesday|2040Q1|N|N|N|2466186|2466216|2465842|2466115|N|N|N|N|N| +2466208|AAAAAAAAAKBKFCAA|2040-02-23|1681|7313|561|2040|4|2|23|1|2040|561|7313|Thursday|2040Q1|N|N|N|2466186|2466216|2465843|2466116|N|N|N|N|N| +2466209|AAAAAAAABKBKFCAA|2040-02-24|1681|7313|561|2040|5|2|24|1|2040|561|7313|Friday|2040Q1|N|Y|N|2466186|2466216|2465844|2466117|N|N|N|N|N| +2466210|AAAAAAAACKBKFCAA|2040-02-25|1681|7313|561|2040|6|2|25|1|2040|561|7313|Saturday|2040Q1|N|Y|N|2466186|2466216|2465845|2466118|N|N|N|N|N| +2466211|AAAAAAAADKBKFCAA|2040-02-26|1681|7313|561|2040|0|2|26|1|2040|561|7313|Sunday|2040Q1|N|N|N|2466186|2466216|2465846|2466119|N|N|N|N|N| +2466212|AAAAAAAAEKBKFCAA|2040-02-27|1681|7313|561|2040|1|2|27|1|2040|561|7313|Monday|2040Q1|N|N|N|2466186|2466216|2465847|2466120|N|N|N|N|N| +2466213|AAAAAAAAFKBKFCAA|2040-02-28|1681|7314|561|2040|2|2|28|1|2040|561|7314|Tuesday|2040Q1|N|N|N|2466186|2466216|2465848|2466121|N|N|N|N|N| +2466214|AAAAAAAAGKBKFCAA|2040-02-29|1681|7314|561|2040|3|2|29|1|2040|561|7314|Wednesday|2040Q1|N|N|N|2466186|2466216|2465848|2466122|N|N|N|N|N| +2466215|AAAAAAAAHKBKFCAA|2040-03-01|1682|7314|562|2040|4|3|1|1|2040|562|7314|Thursday|2040Q1|N|N|N|2466215|2466274|2465849|2466123|N|N|N|N|N| +2466216|AAAAAAAAIKBKFCAA|2040-03-02|1682|7314|562|2040|5|3|2|1|2040|562|7314|Friday|2040Q1|N|Y|N|2466215|2466274|2465850|2466124|N|N|N|N|N| +2466217|AAAAAAAAJKBKFCAA|2040-03-03|1682|7314|562|2040|6|3|3|1|2040|562|7314|Saturday|2040Q1|N|Y|N|2466215|2466274|2465851|2466125|N|N|N|N|N| +2466218|AAAAAAAAKKBKFCAA|2040-03-04|1682|7314|562|2040|0|3|4|1|2040|562|7314|Sunday|2040Q1|N|N|N|2466215|2466274|2465852|2466126|N|N|N|N|N| +2466219|AAAAAAAALKBKFCAA|2040-03-05|1682|7314|562|2040|1|3|5|1|2040|562|7314|Monday|2040Q1|N|N|N|2466215|2466274|2465853|2466127|N|N|N|N|N| +2466220|AAAAAAAAMKBKFCAA|2040-03-06|1682|7315|562|2040|2|3|6|1|2040|562|7315|Tuesday|2040Q1|N|N|N|2466215|2466274|2465854|2466128|N|N|N|N|N| +2466221|AAAAAAAANKBKFCAA|2040-03-07|1682|7315|562|2040|3|3|7|1|2040|562|7315|Wednesday|2040Q1|N|N|N|2466215|2466274|2465855|2466129|N|N|N|N|N| +2466222|AAAAAAAAOKBKFCAA|2040-03-08|1682|7315|562|2040|4|3|8|1|2040|562|7315|Thursday|2040Q1|N|N|N|2466215|2466274|2465856|2466130|N|N|N|N|N| +2466223|AAAAAAAAPKBKFCAA|2040-03-09|1682|7315|562|2040|5|3|9|1|2040|562|7315|Friday|2040Q1|N|Y|N|2466215|2466274|2465857|2466131|N|N|N|N|N| +2466224|AAAAAAAAALBKFCAA|2040-03-10|1682|7315|562|2040|6|3|10|1|2040|562|7315|Saturday|2040Q1|N|Y|N|2466215|2466274|2465858|2466132|N|N|N|N|N| +2466225|AAAAAAAABLBKFCAA|2040-03-11|1682|7315|562|2040|0|3|11|1|2040|562|7315|Sunday|2040Q1|N|N|N|2466215|2466274|2465859|2466133|N|N|N|N|N| +2466226|AAAAAAAACLBKFCAA|2040-03-12|1682|7315|562|2040|1|3|12|1|2040|562|7315|Monday|2040Q1|N|N|N|2466215|2466274|2465860|2466134|N|N|N|N|N| +2466227|AAAAAAAADLBKFCAA|2040-03-13|1682|7316|562|2040|2|3|13|1|2040|562|7316|Tuesday|2040Q1|N|N|N|2466215|2466274|2465861|2466135|N|N|N|N|N| +2466228|AAAAAAAAELBKFCAA|2040-03-14|1682|7316|562|2040|3|3|14|1|2040|562|7316|Wednesday|2040Q1|N|N|N|2466215|2466274|2465862|2466136|N|N|N|N|N| +2466229|AAAAAAAAFLBKFCAA|2040-03-15|1682|7316|562|2040|4|3|15|1|2040|562|7316|Thursday|2040Q1|N|N|N|2466215|2466274|2465863|2466137|N|N|N|N|N| +2466230|AAAAAAAAGLBKFCAA|2040-03-16|1682|7316|562|2040|5|3|16|1|2040|562|7316|Friday|2040Q1|N|Y|N|2466215|2466274|2465864|2466138|N|N|N|N|N| +2466231|AAAAAAAAHLBKFCAA|2040-03-17|1682|7316|562|2040|6|3|17|1|2040|562|7316|Saturday|2040Q1|N|Y|N|2466215|2466274|2465865|2466139|N|N|N|N|N| +2466232|AAAAAAAAILBKFCAA|2040-03-18|1682|7316|562|2040|0|3|18|1|2040|562|7316|Sunday|2040Q1|N|N|N|2466215|2466274|2465866|2466140|N|N|N|N|N| +2466233|AAAAAAAAJLBKFCAA|2040-03-19|1682|7316|562|2040|1|3|19|1|2040|562|7316|Monday|2040Q1|N|N|N|2466215|2466274|2465867|2466141|N|N|N|N|N| +2466234|AAAAAAAAKLBKFCAA|2040-03-20|1682|7317|562|2040|2|3|20|1|2040|562|7317|Tuesday|2040Q1|N|N|N|2466215|2466274|2465868|2466142|N|N|N|N|N| +2466235|AAAAAAAALLBKFCAA|2040-03-21|1682|7317|562|2040|3|3|21|1|2040|562|7317|Wednesday|2040Q1|N|N|N|2466215|2466274|2465869|2466143|N|N|N|N|N| +2466236|AAAAAAAAMLBKFCAA|2040-03-22|1682|7317|562|2040|4|3|22|1|2040|562|7317|Thursday|2040Q1|N|N|N|2466215|2466274|2465870|2466144|N|N|N|N|N| +2466237|AAAAAAAANLBKFCAA|2040-03-23|1682|7317|562|2040|5|3|23|1|2040|562|7317|Friday|2040Q1|N|Y|N|2466215|2466274|2465871|2466145|N|N|N|N|N| +2466238|AAAAAAAAOLBKFCAA|2040-03-24|1682|7317|562|2040|6|3|24|1|2040|562|7317|Saturday|2040Q1|N|Y|N|2466215|2466274|2465872|2466146|N|N|N|N|N| +2466239|AAAAAAAAPLBKFCAA|2040-03-25|1682|7317|562|2040|0|3|25|1|2040|562|7317|Sunday|2040Q1|N|N|N|2466215|2466274|2465873|2466147|N|N|N|N|N| +2466240|AAAAAAAAAMBKFCAA|2040-03-26|1682|7317|562|2040|1|3|26|1|2040|562|7317|Monday|2040Q1|N|N|N|2466215|2466274|2465874|2466148|N|N|N|N|N| +2466241|AAAAAAAABMBKFCAA|2040-03-27|1682|7318|562|2040|2|3|27|1|2040|562|7318|Tuesday|2040Q1|N|N|N|2466215|2466274|2465875|2466149|N|N|N|N|N| +2466242|AAAAAAAACMBKFCAA|2040-03-28|1682|7318|562|2040|3|3|28|1|2040|562|7318|Wednesday|2040Q1|N|N|N|2466215|2466274|2465876|2466150|N|N|N|N|N| +2466243|AAAAAAAADMBKFCAA|2040-03-29|1682|7318|562|2040|4|3|29|1|2040|562|7318|Thursday|2040Q1|N|N|N|2466215|2466274|2465877|2466151|N|N|N|N|N| +2466244|AAAAAAAAEMBKFCAA|2040-03-30|1682|7318|562|2040|5|3|30|1|2040|562|7318|Friday|2040Q1|N|Y|N|2466215|2466274|2465878|2466152|N|N|N|N|N| +2466245|AAAAAAAAFMBKFCAA|2040-03-31|1682|7318|562|2040|6|3|31|1|2040|562|7318|Saturday|2040Q1|N|Y|N|2466215|2466274|2465879|2466153|N|N|N|N|N| +2466246|AAAAAAAAGMBKFCAA|2040-04-01|1683|7318|562|2040|0|4|1|2|2040|562|7318|Sunday|2040Q2|N|N|N|2466246|2466336|2465880|2466155|N|N|N|N|N| +2466247|AAAAAAAAHMBKFCAA|2040-04-02|1683|7318|562|2040|1|4|2|2|2040|562|7318|Monday|2040Q2|N|N|N|2466246|2466336|2465881|2466156|N|N|N|N|N| +2466248|AAAAAAAAIMBKFCAA|2040-04-03|1683|7319|562|2040|2|4|3|2|2040|562|7319|Tuesday|2040Q2|N|N|N|2466246|2466336|2465882|2466157|N|N|N|N|N| +2466249|AAAAAAAAJMBKFCAA|2040-04-04|1683|7319|562|2040|3|4|4|2|2040|562|7319|Wednesday|2040Q2|N|N|N|2466246|2466336|2465883|2466158|N|N|N|N|N| +2466250|AAAAAAAAKMBKFCAA|2040-04-05|1683|7319|562|2040|4|4|5|2|2040|562|7319|Thursday|2040Q2|N|N|N|2466246|2466336|2465884|2466159|N|N|N|N|N| +2466251|AAAAAAAALMBKFCAA|2040-04-06|1683|7319|562|2040|5|4|6|2|2040|562|7319|Friday|2040Q2|N|Y|N|2466246|2466336|2465885|2466160|N|N|N|N|N| +2466252|AAAAAAAAMMBKFCAA|2040-04-07|1683|7319|562|2040|6|4|7|2|2040|562|7319|Saturday|2040Q2|N|Y|N|2466246|2466336|2465886|2466161|N|N|N|N|N| +2466253|AAAAAAAANMBKFCAA|2040-04-08|1683|7319|562|2040|0|4|8|2|2040|562|7319|Sunday|2040Q2|N|N|N|2466246|2466336|2465887|2466162|N|N|N|N|N| +2466254|AAAAAAAAOMBKFCAA|2040-04-09|1683|7319|562|2040|1|4|9|2|2040|562|7319|Monday|2040Q2|N|N|N|2466246|2466336|2465888|2466163|N|N|N|N|N| +2466255|AAAAAAAAPMBKFCAA|2040-04-10|1683|7320|562|2040|2|4|10|2|2040|562|7320|Tuesday|2040Q2|N|N|N|2466246|2466336|2465889|2466164|N|N|N|N|N| +2466256|AAAAAAAAANBKFCAA|2040-04-11|1683|7320|562|2040|3|4|11|2|2040|562|7320|Wednesday|2040Q2|N|N|N|2466246|2466336|2465890|2466165|N|N|N|N|N| +2466257|AAAAAAAABNBKFCAA|2040-04-12|1683|7320|562|2040|4|4|12|2|2040|562|7320|Thursday|2040Q2|N|N|N|2466246|2466336|2465891|2466166|N|N|N|N|N| +2466258|AAAAAAAACNBKFCAA|2040-04-13|1683|7320|562|2040|5|4|13|2|2040|562|7320|Friday|2040Q2|N|Y|N|2466246|2466336|2465892|2466167|N|N|N|N|N| +2466259|AAAAAAAADNBKFCAA|2040-04-14|1683|7320|562|2040|6|4|14|2|2040|562|7320|Saturday|2040Q2|N|Y|N|2466246|2466336|2465893|2466168|N|N|N|N|N| +2466260|AAAAAAAAENBKFCAA|2040-04-15|1683|7320|562|2040|0|4|15|2|2040|562|7320|Sunday|2040Q2|N|N|N|2466246|2466336|2465894|2466169|N|N|N|N|N| +2466261|AAAAAAAAFNBKFCAA|2040-04-16|1683|7320|562|2040|1|4|16|2|2040|562|7320|Monday|2040Q2|N|N|N|2466246|2466336|2465895|2466170|N|N|N|N|N| +2466262|AAAAAAAAGNBKFCAA|2040-04-17|1683|7321|562|2040|2|4|17|2|2040|562|7321|Tuesday|2040Q2|N|N|N|2466246|2466336|2465896|2466171|N|N|N|N|N| +2466263|AAAAAAAAHNBKFCAA|2040-04-18|1683|7321|562|2040|3|4|18|2|2040|562|7321|Wednesday|2040Q2|N|N|N|2466246|2466336|2465897|2466172|N|N|N|N|N| +2466264|AAAAAAAAINBKFCAA|2040-04-19|1683|7321|562|2040|4|4|19|2|2040|562|7321|Thursday|2040Q2|N|N|N|2466246|2466336|2465898|2466173|N|N|N|N|N| +2466265|AAAAAAAAJNBKFCAA|2040-04-20|1683|7321|562|2040|5|4|20|2|2040|562|7321|Friday|2040Q2|N|Y|N|2466246|2466336|2465899|2466174|N|N|N|N|N| +2466266|AAAAAAAAKNBKFCAA|2040-04-21|1683|7321|562|2040|6|4|21|2|2040|562|7321|Saturday|2040Q2|N|Y|N|2466246|2466336|2465900|2466175|N|N|N|N|N| +2466267|AAAAAAAALNBKFCAA|2040-04-22|1683|7321|562|2040|0|4|22|2|2040|562|7321|Sunday|2040Q2|N|N|N|2466246|2466336|2465901|2466176|N|N|N|N|N| +2466268|AAAAAAAAMNBKFCAA|2040-04-23|1683|7321|562|2040|1|4|23|2|2040|562|7321|Monday|2040Q2|N|N|N|2466246|2466336|2465902|2466177|N|N|N|N|N| +2466269|AAAAAAAANNBKFCAA|2040-04-24|1683|7322|562|2040|2|4|24|2|2040|562|7322|Tuesday|2040Q2|N|N|N|2466246|2466336|2465903|2466178|N|N|N|N|N| +2466270|AAAAAAAAONBKFCAA|2040-04-25|1683|7322|562|2040|3|4|25|2|2040|562|7322|Wednesday|2040Q2|N|N|N|2466246|2466336|2465904|2466179|N|N|N|N|N| +2466271|AAAAAAAAPNBKFCAA|2040-04-26|1683|7322|562|2040|4|4|26|2|2040|562|7322|Thursday|2040Q2|N|N|N|2466246|2466336|2465905|2466180|N|N|N|N|N| +2466272|AAAAAAAAAOBKFCAA|2040-04-27|1683|7322|562|2040|5|4|27|2|2040|562|7322|Friday|2040Q2|N|Y|N|2466246|2466336|2465906|2466181|N|N|N|N|N| +2466273|AAAAAAAABOBKFCAA|2040-04-28|1683|7322|562|2040|6|4|28|2|2040|562|7322|Saturday|2040Q2|N|Y|N|2466246|2466336|2465907|2466182|N|N|N|N|N| +2466274|AAAAAAAACOBKFCAA|2040-04-29|1683|7322|562|2040|0|4|29|2|2040|562|7322|Sunday|2040Q2|N|N|N|2466246|2466336|2465908|2466183|N|N|N|N|N| +2466275|AAAAAAAADOBKFCAA|2040-04-30|1683|7322|562|2040|1|4|30|2|2040|562|7322|Monday|2040Q2|N|N|N|2466246|2466336|2465909|2466184|N|N|N|N|N| +2466276|AAAAAAAAEOBKFCAA|2040-05-01|1684|7323|562|2040|2|5|1|2|2040|562|7323|Tuesday|2040Q2|N|N|N|2466276|2466396|2465910|2466185|N|N|N|N|N| +2466277|AAAAAAAAFOBKFCAA|2040-05-02|1684|7323|562|2040|3|5|2|2|2040|562|7323|Wednesday|2040Q2|N|N|N|2466276|2466396|2465911|2466186|N|N|N|N|N| +2466278|AAAAAAAAGOBKFCAA|2040-05-03|1684|7323|562|2040|4|5|3|2|2040|562|7323|Thursday|2040Q2|N|N|N|2466276|2466396|2465912|2466187|N|N|N|N|N| +2466279|AAAAAAAAHOBKFCAA|2040-05-04|1684|7323|562|2040|5|5|4|2|2040|562|7323|Friday|2040Q2|N|Y|N|2466276|2466396|2465913|2466188|N|N|N|N|N| +2466280|AAAAAAAAIOBKFCAA|2040-05-05|1684|7323|562|2040|6|5|5|2|2040|562|7323|Saturday|2040Q2|N|Y|N|2466276|2466396|2465914|2466189|N|N|N|N|N| +2466281|AAAAAAAAJOBKFCAA|2040-05-06|1684|7323|562|2040|0|5|6|2|2040|562|7323|Sunday|2040Q2|N|N|N|2466276|2466396|2465915|2466190|N|N|N|N|N| +2466282|AAAAAAAAKOBKFCAA|2040-05-07|1684|7323|562|2040|1|5|7|2|2040|562|7323|Monday|2040Q2|N|N|N|2466276|2466396|2465916|2466191|N|N|N|N|N| +2466283|AAAAAAAALOBKFCAA|2040-05-08|1684|7324|562|2040|2|5|8|2|2040|562|7324|Tuesday|2040Q2|N|N|N|2466276|2466396|2465917|2466192|N|N|N|N|N| +2466284|AAAAAAAAMOBKFCAA|2040-05-09|1684|7324|562|2040|3|5|9|2|2040|562|7324|Wednesday|2040Q2|N|N|N|2466276|2466396|2465918|2466193|N|N|N|N|N| +2466285|AAAAAAAANOBKFCAA|2040-05-10|1684|7324|562|2040|4|5|10|2|2040|562|7324|Thursday|2040Q2|N|N|N|2466276|2466396|2465919|2466194|N|N|N|N|N| +2466286|AAAAAAAAOOBKFCAA|2040-05-11|1684|7324|562|2040|5|5|11|2|2040|562|7324|Friday|2040Q2|N|Y|N|2466276|2466396|2465920|2466195|N|N|N|N|N| +2466287|AAAAAAAAPOBKFCAA|2040-05-12|1684|7324|562|2040|6|5|12|2|2040|562|7324|Saturday|2040Q2|N|Y|N|2466276|2466396|2465921|2466196|N|N|N|N|N| +2466288|AAAAAAAAAPBKFCAA|2040-05-13|1684|7324|562|2040|0|5|13|2|2040|562|7324|Sunday|2040Q2|N|N|N|2466276|2466396|2465922|2466197|N|N|N|N|N| +2466289|AAAAAAAABPBKFCAA|2040-05-14|1684|7324|562|2040|1|5|14|2|2040|562|7324|Monday|2040Q2|N|N|N|2466276|2466396|2465923|2466198|N|N|N|N|N| +2466290|AAAAAAAACPBKFCAA|2040-05-15|1684|7325|562|2040|2|5|15|2|2040|562|7325|Tuesday|2040Q2|N|N|N|2466276|2466396|2465924|2466199|N|N|N|N|N| +2466291|AAAAAAAADPBKFCAA|2040-05-16|1684|7325|562|2040|3|5|16|2|2040|562|7325|Wednesday|2040Q2|N|N|N|2466276|2466396|2465925|2466200|N|N|N|N|N| +2466292|AAAAAAAAEPBKFCAA|2040-05-17|1684|7325|562|2040|4|5|17|2|2040|562|7325|Thursday|2040Q2|N|N|N|2466276|2466396|2465926|2466201|N|N|N|N|N| +2466293|AAAAAAAAFPBKFCAA|2040-05-18|1684|7325|562|2040|5|5|18|2|2040|562|7325|Friday|2040Q2|N|Y|N|2466276|2466396|2465927|2466202|N|N|N|N|N| +2466294|AAAAAAAAGPBKFCAA|2040-05-19|1684|7325|562|2040|6|5|19|2|2040|562|7325|Saturday|2040Q2|N|Y|N|2466276|2466396|2465928|2466203|N|N|N|N|N| +2466295|AAAAAAAAHPBKFCAA|2040-05-20|1684|7325|562|2040|0|5|20|2|2040|562|7325|Sunday|2040Q2|N|N|N|2466276|2466396|2465929|2466204|N|N|N|N|N| +2466296|AAAAAAAAIPBKFCAA|2040-05-21|1684|7325|562|2040|1|5|21|2|2040|562|7325|Monday|2040Q2|N|N|N|2466276|2466396|2465930|2466205|N|N|N|N|N| +2466297|AAAAAAAAJPBKFCAA|2040-05-22|1684|7326|562|2040|2|5|22|2|2040|562|7326|Tuesday|2040Q2|N|N|N|2466276|2466396|2465931|2466206|N|N|N|N|N| +2466298|AAAAAAAAKPBKFCAA|2040-05-23|1684|7326|562|2040|3|5|23|2|2040|562|7326|Wednesday|2040Q2|N|N|N|2466276|2466396|2465932|2466207|N|N|N|N|N| +2466299|AAAAAAAALPBKFCAA|2040-05-24|1684|7326|562|2040|4|5|24|2|2040|562|7326|Thursday|2040Q2|N|N|N|2466276|2466396|2465933|2466208|N|N|N|N|N| +2466300|AAAAAAAAMPBKFCAA|2040-05-25|1684|7326|562|2040|5|5|25|2|2040|562|7326|Friday|2040Q2|N|Y|N|2466276|2466396|2465934|2466209|N|N|N|N|N| +2466301|AAAAAAAANPBKFCAA|2040-05-26|1684|7326|562|2040|6|5|26|2|2040|562|7326|Saturday|2040Q2|N|Y|N|2466276|2466396|2465935|2466210|N|N|N|N|N| +2466302|AAAAAAAAOPBKFCAA|2040-05-27|1684|7326|562|2040|0|5|27|2|2040|562|7326|Sunday|2040Q2|N|N|N|2466276|2466396|2465936|2466211|N|N|N|N|N| +2466303|AAAAAAAAPPBKFCAA|2040-05-28|1684|7326|562|2040|1|5|28|2|2040|562|7326|Monday|2040Q2|N|N|N|2466276|2466396|2465937|2466212|N|N|N|N|N| +2466304|AAAAAAAAAACKFCAA|2040-05-29|1684|7327|562|2040|2|5|29|2|2040|562|7327|Tuesday|2040Q2|N|N|N|2466276|2466396|2465938|2466213|N|N|N|N|N| +2466305|AAAAAAAABACKFCAA|2040-05-30|1684|7327|562|2040|3|5|30|2|2040|562|7327|Wednesday|2040Q2|N|N|N|2466276|2466396|2465939|2466214|N|N|N|N|N| +2466306|AAAAAAAACACKFCAA|2040-05-31|1684|7327|562|2040|4|5|31|2|2040|562|7327|Thursday|2040Q2|N|N|N|2466276|2466396|2465940|2466215|N|N|N|N|N| +2466307|AAAAAAAADACKFCAA|2040-06-01|1685|7327|563|2040|5|6|1|2|2040|563|7327|Friday|2040Q2|N|Y|N|2466307|2466458|2465941|2466216|N|N|N|N|N| +2466308|AAAAAAAAEACKFCAA|2040-06-02|1685|7327|563|2040|6|6|2|2|2040|563|7327|Saturday|2040Q2|N|Y|N|2466307|2466458|2465942|2466217|N|N|N|N|N| +2466309|AAAAAAAAFACKFCAA|2040-06-03|1685|7327|563|2040|0|6|3|2|2040|563|7327|Sunday|2040Q2|N|N|N|2466307|2466458|2465943|2466218|N|N|N|N|N| +2466310|AAAAAAAAGACKFCAA|2040-06-04|1685|7327|563|2040|1|6|4|2|2040|563|7327|Monday|2040Q2|N|N|N|2466307|2466458|2465944|2466219|N|N|N|N|N| +2466311|AAAAAAAAHACKFCAA|2040-06-05|1685|7328|563|2040|2|6|5|2|2040|563|7328|Tuesday|2040Q2|N|N|N|2466307|2466458|2465945|2466220|N|N|N|N|N| +2466312|AAAAAAAAIACKFCAA|2040-06-06|1685|7328|563|2040|3|6|6|2|2040|563|7328|Wednesday|2040Q2|N|N|N|2466307|2466458|2465946|2466221|N|N|N|N|N| +2466313|AAAAAAAAJACKFCAA|2040-06-07|1685|7328|563|2040|4|6|7|2|2040|563|7328|Thursday|2040Q2|N|N|N|2466307|2466458|2465947|2466222|N|N|N|N|N| +2466314|AAAAAAAAKACKFCAA|2040-06-08|1685|7328|563|2040|5|6|8|2|2040|563|7328|Friday|2040Q2|N|Y|N|2466307|2466458|2465948|2466223|N|N|N|N|N| +2466315|AAAAAAAALACKFCAA|2040-06-09|1685|7328|563|2040|6|6|9|2|2040|563|7328|Saturday|2040Q2|N|Y|N|2466307|2466458|2465949|2466224|N|N|N|N|N| +2466316|AAAAAAAAMACKFCAA|2040-06-10|1685|7328|563|2040|0|6|10|2|2040|563|7328|Sunday|2040Q2|N|N|N|2466307|2466458|2465950|2466225|N|N|N|N|N| +2466317|AAAAAAAANACKFCAA|2040-06-11|1685|7328|563|2040|1|6|11|2|2040|563|7328|Monday|2040Q2|N|N|N|2466307|2466458|2465951|2466226|N|N|N|N|N| +2466318|AAAAAAAAOACKFCAA|2040-06-12|1685|7329|563|2040|2|6|12|2|2040|563|7329|Tuesday|2040Q2|N|N|N|2466307|2466458|2465952|2466227|N|N|N|N|N| +2466319|AAAAAAAAPACKFCAA|2040-06-13|1685|7329|563|2040|3|6|13|2|2040|563|7329|Wednesday|2040Q2|N|N|N|2466307|2466458|2465953|2466228|N|N|N|N|N| +2466320|AAAAAAAAABCKFCAA|2040-06-14|1685|7329|563|2040|4|6|14|2|2040|563|7329|Thursday|2040Q2|N|N|N|2466307|2466458|2465954|2466229|N|N|N|N|N| +2466321|AAAAAAAABBCKFCAA|2040-06-15|1685|7329|563|2040|5|6|15|2|2040|563|7329|Friday|2040Q2|N|Y|N|2466307|2466458|2465955|2466230|N|N|N|N|N| +2466322|AAAAAAAACBCKFCAA|2040-06-16|1685|7329|563|2040|6|6|16|2|2040|563|7329|Saturday|2040Q2|N|Y|N|2466307|2466458|2465956|2466231|N|N|N|N|N| +2466323|AAAAAAAADBCKFCAA|2040-06-17|1685|7329|563|2040|0|6|17|2|2040|563|7329|Sunday|2040Q2|N|N|N|2466307|2466458|2465957|2466232|N|N|N|N|N| +2466324|AAAAAAAAEBCKFCAA|2040-06-18|1685|7329|563|2040|1|6|18|2|2040|563|7329|Monday|2040Q2|N|N|N|2466307|2466458|2465958|2466233|N|N|N|N|N| +2466325|AAAAAAAAFBCKFCAA|2040-06-19|1685|7330|563|2040|2|6|19|2|2040|563|7330|Tuesday|2040Q2|N|N|N|2466307|2466458|2465959|2466234|N|N|N|N|N| +2466326|AAAAAAAAGBCKFCAA|2040-06-20|1685|7330|563|2040|3|6|20|2|2040|563|7330|Wednesday|2040Q2|N|N|N|2466307|2466458|2465960|2466235|N|N|N|N|N| +2466327|AAAAAAAAHBCKFCAA|2040-06-21|1685|7330|563|2040|4|6|21|2|2040|563|7330|Thursday|2040Q2|N|N|N|2466307|2466458|2465961|2466236|N|N|N|N|N| +2466328|AAAAAAAAIBCKFCAA|2040-06-22|1685|7330|563|2040|5|6|22|2|2040|563|7330|Friday|2040Q2|N|Y|N|2466307|2466458|2465962|2466237|N|N|N|N|N| +2466329|AAAAAAAAJBCKFCAA|2040-06-23|1685|7330|563|2040|6|6|23|2|2040|563|7330|Saturday|2040Q2|N|Y|N|2466307|2466458|2465963|2466238|N|N|N|N|N| +2466330|AAAAAAAAKBCKFCAA|2040-06-24|1685|7330|563|2040|0|6|24|2|2040|563|7330|Sunday|2040Q2|N|N|N|2466307|2466458|2465964|2466239|N|N|N|N|N| +2466331|AAAAAAAALBCKFCAA|2040-06-25|1685|7330|563|2040|1|6|25|2|2040|563|7330|Monday|2040Q2|N|N|N|2466307|2466458|2465965|2466240|N|N|N|N|N| +2466332|AAAAAAAAMBCKFCAA|2040-06-26|1685|7331|563|2040|2|6|26|2|2040|563|7331|Tuesday|2040Q2|N|N|N|2466307|2466458|2465966|2466241|N|N|N|N|N| +2466333|AAAAAAAANBCKFCAA|2040-06-27|1685|7331|563|2040|3|6|27|2|2040|563|7331|Wednesday|2040Q2|N|N|N|2466307|2466458|2465967|2466242|N|N|N|N|N| +2466334|AAAAAAAAOBCKFCAA|2040-06-28|1685|7331|563|2040|4|6|28|2|2040|563|7331|Thursday|2040Q2|N|N|N|2466307|2466458|2465968|2466243|N|N|N|N|N| +2466335|AAAAAAAAPBCKFCAA|2040-06-29|1685|7331|563|2040|5|6|29|2|2040|563|7331|Friday|2040Q2|N|Y|N|2466307|2466458|2465969|2466244|N|N|N|N|N| +2466336|AAAAAAAAACCKFCAA|2040-06-30|1685|7331|563|2040|6|6|30|2|2040|563|7331|Saturday|2040Q2|N|Y|N|2466307|2466458|2465970|2466245|N|N|N|N|N| +2466337|AAAAAAAABCCKFCAA|2040-07-01|1686|7331|563|2040|0|7|1|3|2040|563|7331|Sunday|2040Q3|N|N|N|2466337|2466518|2465971|2466246|N|N|N|N|N| +2466338|AAAAAAAACCCKFCAA|2040-07-02|1686|7331|563|2040|1|7|2|3|2040|563|7331|Monday|2040Q3|N|N|N|2466337|2466518|2465972|2466247|N|N|N|N|N| +2466339|AAAAAAAADCCKFCAA|2040-07-03|1686|7332|563|2040|2|7|3|3|2040|563|7332|Tuesday|2040Q3|N|N|N|2466337|2466518|2465973|2466248|N|N|N|N|N| +2466340|AAAAAAAAECCKFCAA|2040-07-04|1686|7332|563|2040|3|7|4|3|2040|563|7332|Wednesday|2040Q3|Y|N|N|2466337|2466518|2465974|2466249|N|N|N|N|N| +2466341|AAAAAAAAFCCKFCAA|2040-07-05|1686|7332|563|2040|4|7|5|3|2040|563|7332|Thursday|2040Q3|N|N|Y|2466337|2466518|2465975|2466250|N|N|N|N|N| +2466342|AAAAAAAAGCCKFCAA|2040-07-06|1686|7332|563|2040|5|7|6|3|2040|563|7332|Friday|2040Q3|N|Y|N|2466337|2466518|2465976|2466251|N|N|N|N|N| +2466343|AAAAAAAAHCCKFCAA|2040-07-07|1686|7332|563|2040|6|7|7|3|2040|563|7332|Saturday|2040Q3|N|Y|N|2466337|2466518|2465977|2466252|N|N|N|N|N| +2466344|AAAAAAAAICCKFCAA|2040-07-08|1686|7332|563|2040|0|7|8|3|2040|563|7332|Sunday|2040Q3|N|N|N|2466337|2466518|2465978|2466253|N|N|N|N|N| +2466345|AAAAAAAAJCCKFCAA|2040-07-09|1686|7332|563|2040|1|7|9|3|2040|563|7332|Monday|2040Q3|N|N|N|2466337|2466518|2465979|2466254|N|N|N|N|N| +2466346|AAAAAAAAKCCKFCAA|2040-07-10|1686|7333|563|2040|2|7|10|3|2040|563|7333|Tuesday|2040Q3|N|N|N|2466337|2466518|2465980|2466255|N|N|N|N|N| +2466347|AAAAAAAALCCKFCAA|2040-07-11|1686|7333|563|2040|3|7|11|3|2040|563|7333|Wednesday|2040Q3|N|N|N|2466337|2466518|2465981|2466256|N|N|N|N|N| +2466348|AAAAAAAAMCCKFCAA|2040-07-12|1686|7333|563|2040|4|7|12|3|2040|563|7333|Thursday|2040Q3|N|N|N|2466337|2466518|2465982|2466257|N|N|N|N|N| +2466349|AAAAAAAANCCKFCAA|2040-07-13|1686|7333|563|2040|5|7|13|3|2040|563|7333|Friday|2040Q3|N|Y|N|2466337|2466518|2465983|2466258|N|N|N|N|N| +2466350|AAAAAAAAOCCKFCAA|2040-07-14|1686|7333|563|2040|6|7|14|3|2040|563|7333|Saturday|2040Q3|N|Y|N|2466337|2466518|2465984|2466259|N|N|N|N|N| +2466351|AAAAAAAAPCCKFCAA|2040-07-15|1686|7333|563|2040|0|7|15|3|2040|563|7333|Sunday|2040Q3|N|N|N|2466337|2466518|2465985|2466260|N|N|N|N|N| +2466352|AAAAAAAAADCKFCAA|2040-07-16|1686|7333|563|2040|1|7|16|3|2040|563|7333|Monday|2040Q3|N|N|N|2466337|2466518|2465986|2466261|N|N|N|N|N| +2466353|AAAAAAAABDCKFCAA|2040-07-17|1686|7334|563|2040|2|7|17|3|2040|563|7334|Tuesday|2040Q3|N|N|N|2466337|2466518|2465987|2466262|N|N|N|N|N| +2466354|AAAAAAAACDCKFCAA|2040-07-18|1686|7334|563|2040|3|7|18|3|2040|563|7334|Wednesday|2040Q3|N|N|N|2466337|2466518|2465988|2466263|N|N|N|N|N| +2466355|AAAAAAAADDCKFCAA|2040-07-19|1686|7334|563|2040|4|7|19|3|2040|563|7334|Thursday|2040Q3|N|N|N|2466337|2466518|2465989|2466264|N|N|N|N|N| +2466356|AAAAAAAAEDCKFCAA|2040-07-20|1686|7334|563|2040|5|7|20|3|2040|563|7334|Friday|2040Q3|N|Y|N|2466337|2466518|2465990|2466265|N|N|N|N|N| +2466357|AAAAAAAAFDCKFCAA|2040-07-21|1686|7334|563|2040|6|7|21|3|2040|563|7334|Saturday|2040Q3|N|Y|N|2466337|2466518|2465991|2466266|N|N|N|N|N| +2466358|AAAAAAAAGDCKFCAA|2040-07-22|1686|7334|563|2040|0|7|22|3|2040|563|7334|Sunday|2040Q3|N|N|N|2466337|2466518|2465992|2466267|N|N|N|N|N| +2466359|AAAAAAAAHDCKFCAA|2040-07-23|1686|7334|563|2040|1|7|23|3|2040|563|7334|Monday|2040Q3|N|N|N|2466337|2466518|2465993|2466268|N|N|N|N|N| +2466360|AAAAAAAAIDCKFCAA|2040-07-24|1686|7335|563|2040|2|7|24|3|2040|563|7335|Tuesday|2040Q3|N|N|N|2466337|2466518|2465994|2466269|N|N|N|N|N| +2466361|AAAAAAAAJDCKFCAA|2040-07-25|1686|7335|563|2040|3|7|25|3|2040|563|7335|Wednesday|2040Q3|N|N|N|2466337|2466518|2465995|2466270|N|N|N|N|N| +2466362|AAAAAAAAKDCKFCAA|2040-07-26|1686|7335|563|2040|4|7|26|3|2040|563|7335|Thursday|2040Q3|N|N|N|2466337|2466518|2465996|2466271|N|N|N|N|N| +2466363|AAAAAAAALDCKFCAA|2040-07-27|1686|7335|563|2040|5|7|27|3|2040|563|7335|Friday|2040Q3|N|Y|N|2466337|2466518|2465997|2466272|N|N|N|N|N| +2466364|AAAAAAAAMDCKFCAA|2040-07-28|1686|7335|563|2040|6|7|28|3|2040|563|7335|Saturday|2040Q3|N|Y|N|2466337|2466518|2465998|2466273|N|N|N|N|N| +2466365|AAAAAAAANDCKFCAA|2040-07-29|1686|7335|563|2040|0|7|29|3|2040|563|7335|Sunday|2040Q3|N|N|N|2466337|2466518|2465999|2466274|N|N|N|N|N| +2466366|AAAAAAAAODCKFCAA|2040-07-30|1686|7335|563|2040|1|7|30|3|2040|563|7335|Monday|2040Q3|N|N|N|2466337|2466518|2466000|2466275|N|N|N|N|N| +2466367|AAAAAAAAPDCKFCAA|2040-07-31|1686|7336|563|2040|2|7|31|3|2040|563|7336|Tuesday|2040Q3|N|N|N|2466337|2466518|2466001|2466276|N|N|N|N|N| +2466368|AAAAAAAAAECKFCAA|2040-08-01|1687|7336|563|2040|3|8|1|3|2040|563|7336|Wednesday|2040Q3|N|N|N|2466368|2466580|2466002|2466277|N|N|N|N|N| +2466369|AAAAAAAABECKFCAA|2040-08-02|1687|7336|563|2040|4|8|2|3|2040|563|7336|Thursday|2040Q3|N|N|N|2466368|2466580|2466003|2466278|N|N|N|N|N| +2466370|AAAAAAAACECKFCAA|2040-08-03|1687|7336|563|2040|5|8|3|3|2040|563|7336|Friday|2040Q3|N|Y|N|2466368|2466580|2466004|2466279|N|N|N|N|N| +2466371|AAAAAAAADECKFCAA|2040-08-04|1687|7336|563|2040|6|8|4|3|2040|563|7336|Saturday|2040Q3|N|Y|N|2466368|2466580|2466005|2466280|N|N|N|N|N| +2466372|AAAAAAAAEECKFCAA|2040-08-05|1687|7336|563|2040|0|8|5|3|2040|563|7336|Sunday|2040Q3|N|N|N|2466368|2466580|2466006|2466281|N|N|N|N|N| +2466373|AAAAAAAAFECKFCAA|2040-08-06|1687|7336|563|2040|1|8|6|3|2040|563|7336|Monday|2040Q3|N|N|N|2466368|2466580|2466007|2466282|N|N|N|N|N| +2466374|AAAAAAAAGECKFCAA|2040-08-07|1687|7337|563|2040|2|8|7|3|2040|563|7337|Tuesday|2040Q3|N|N|N|2466368|2466580|2466008|2466283|N|N|N|N|N| +2466375|AAAAAAAAHECKFCAA|2040-08-08|1687|7337|563|2040|3|8|8|3|2040|563|7337|Wednesday|2040Q3|N|N|N|2466368|2466580|2466009|2466284|N|N|N|N|N| +2466376|AAAAAAAAIECKFCAA|2040-08-09|1687|7337|563|2040|4|8|9|3|2040|563|7337|Thursday|2040Q3|N|N|N|2466368|2466580|2466010|2466285|N|N|N|N|N| +2466377|AAAAAAAAJECKFCAA|2040-08-10|1687|7337|563|2040|5|8|10|3|2040|563|7337|Friday|2040Q3|N|Y|N|2466368|2466580|2466011|2466286|N|N|N|N|N| +2466378|AAAAAAAAKECKFCAA|2040-08-11|1687|7337|563|2040|6|8|11|3|2040|563|7337|Saturday|2040Q3|N|Y|N|2466368|2466580|2466012|2466287|N|N|N|N|N| +2466379|AAAAAAAALECKFCAA|2040-08-12|1687|7337|563|2040|0|8|12|3|2040|563|7337|Sunday|2040Q3|N|N|N|2466368|2466580|2466013|2466288|N|N|N|N|N| +2466380|AAAAAAAAMECKFCAA|2040-08-13|1687|7337|563|2040|1|8|13|3|2040|563|7337|Monday|2040Q3|N|N|N|2466368|2466580|2466014|2466289|N|N|N|N|N| +2466381|AAAAAAAANECKFCAA|2040-08-14|1687|7338|563|2040|2|8|14|3|2040|563|7338|Tuesday|2040Q3|N|N|N|2466368|2466580|2466015|2466290|N|N|N|N|N| +2466382|AAAAAAAAOECKFCAA|2040-08-15|1687|7338|563|2040|3|8|15|3|2040|563|7338|Wednesday|2040Q3|N|N|N|2466368|2466580|2466016|2466291|N|N|N|N|N| +2466383|AAAAAAAAPECKFCAA|2040-08-16|1687|7338|563|2040|4|8|16|3|2040|563|7338|Thursday|2040Q3|N|N|N|2466368|2466580|2466017|2466292|N|N|N|N|N| +2466384|AAAAAAAAAFCKFCAA|2040-08-17|1687|7338|563|2040|5|8|17|3|2040|563|7338|Friday|2040Q3|N|Y|N|2466368|2466580|2466018|2466293|N|N|N|N|N| +2466385|AAAAAAAABFCKFCAA|2040-08-18|1687|7338|563|2040|6|8|18|3|2040|563|7338|Saturday|2040Q3|N|Y|N|2466368|2466580|2466019|2466294|N|N|N|N|N| +2466386|AAAAAAAACFCKFCAA|2040-08-19|1687|7338|563|2040|0|8|19|3|2040|563|7338|Sunday|2040Q3|N|N|N|2466368|2466580|2466020|2466295|N|N|N|N|N| +2466387|AAAAAAAADFCKFCAA|2040-08-20|1687|7338|563|2040|1|8|20|3|2040|563|7338|Monday|2040Q3|N|N|N|2466368|2466580|2466021|2466296|N|N|N|N|N| +2466388|AAAAAAAAEFCKFCAA|2040-08-21|1687|7339|563|2040|2|8|21|3|2040|563|7339|Tuesday|2040Q3|N|N|N|2466368|2466580|2466022|2466297|N|N|N|N|N| +2466389|AAAAAAAAFFCKFCAA|2040-08-22|1687|7339|563|2040|3|8|22|3|2040|563|7339|Wednesday|2040Q3|N|N|N|2466368|2466580|2466023|2466298|N|N|N|N|N| +2466390|AAAAAAAAGFCKFCAA|2040-08-23|1687|7339|563|2040|4|8|23|3|2040|563|7339|Thursday|2040Q3|N|N|N|2466368|2466580|2466024|2466299|N|N|N|N|N| +2466391|AAAAAAAAHFCKFCAA|2040-08-24|1687|7339|563|2040|5|8|24|3|2040|563|7339|Friday|2040Q3|N|Y|N|2466368|2466580|2466025|2466300|N|N|N|N|N| +2466392|AAAAAAAAIFCKFCAA|2040-08-25|1687|7339|563|2040|6|8|25|3|2040|563|7339|Saturday|2040Q3|N|Y|N|2466368|2466580|2466026|2466301|N|N|N|N|N| +2466393|AAAAAAAAJFCKFCAA|2040-08-26|1687|7339|563|2040|0|8|26|3|2040|563|7339|Sunday|2040Q3|N|N|N|2466368|2466580|2466027|2466302|N|N|N|N|N| +2466394|AAAAAAAAKFCKFCAA|2040-08-27|1687|7339|563|2040|1|8|27|3|2040|563|7339|Monday|2040Q3|N|N|N|2466368|2466580|2466028|2466303|N|N|N|N|N| +2466395|AAAAAAAALFCKFCAA|2040-08-28|1687|7340|563|2040|2|8|28|3|2040|563|7340|Tuesday|2040Q3|N|N|N|2466368|2466580|2466029|2466304|N|N|N|N|N| +2466396|AAAAAAAAMFCKFCAA|2040-08-29|1687|7340|563|2040|3|8|29|3|2040|563|7340|Wednesday|2040Q3|N|N|N|2466368|2466580|2466030|2466305|N|N|N|N|N| +2466397|AAAAAAAANFCKFCAA|2040-08-30|1687|7340|563|2040|4|8|30|3|2040|563|7340|Thursday|2040Q3|N|N|N|2466368|2466580|2466031|2466306|N|N|N|N|N| +2466398|AAAAAAAAOFCKFCAA|2040-08-31|1687|7340|563|2040|5|8|31|3|2040|563|7340|Friday|2040Q3|N|Y|N|2466368|2466580|2466032|2466307|N|N|N|N|N| +2466399|AAAAAAAAPFCKFCAA|2040-09-01|1688|7340|564|2040|6|9|1|3|2040|564|7340|Saturday|2040Q3|N|Y|N|2466399|2466642|2466033|2466308|N|N|N|N|N| +2466400|AAAAAAAAAGCKFCAA|2040-09-02|1688|7340|564|2040|0|9|2|3|2040|564|7340|Sunday|2040Q3|N|N|N|2466399|2466642|2466034|2466309|N|N|N|N|N| +2466401|AAAAAAAABGCKFCAA|2040-09-03|1688|7340|564|2040|1|9|3|3|2040|564|7340|Monday|2040Q3|N|N|N|2466399|2466642|2466035|2466310|N|N|N|N|N| +2466402|AAAAAAAACGCKFCAA|2040-09-04|1688|7341|564|2040|2|9|4|3|2040|564|7341|Tuesday|2040Q3|N|N|N|2466399|2466642|2466036|2466311|N|N|N|N|N| +2466403|AAAAAAAADGCKFCAA|2040-09-05|1688|7341|564|2040|3|9|5|3|2040|564|7341|Wednesday|2040Q3|N|N|N|2466399|2466642|2466037|2466312|N|N|N|N|N| +2466404|AAAAAAAAEGCKFCAA|2040-09-06|1688|7341|564|2040|4|9|6|3|2040|564|7341|Thursday|2040Q3|N|N|N|2466399|2466642|2466038|2466313|N|N|N|N|N| +2466405|AAAAAAAAFGCKFCAA|2040-09-07|1688|7341|564|2040|5|9|7|3|2040|564|7341|Friday|2040Q3|N|Y|N|2466399|2466642|2466039|2466314|N|N|N|N|N| +2466406|AAAAAAAAGGCKFCAA|2040-09-08|1688|7341|564|2040|6|9|8|3|2040|564|7341|Saturday|2040Q3|N|Y|N|2466399|2466642|2466040|2466315|N|N|N|N|N| +2466407|AAAAAAAAHGCKFCAA|2040-09-09|1688|7341|564|2040|0|9|9|3|2040|564|7341|Sunday|2040Q3|N|N|N|2466399|2466642|2466041|2466316|N|N|N|N|N| +2466408|AAAAAAAAIGCKFCAA|2040-09-10|1688|7341|564|2040|1|9|10|3|2040|564|7341|Monday|2040Q3|N|N|N|2466399|2466642|2466042|2466317|N|N|N|N|N| +2466409|AAAAAAAAJGCKFCAA|2040-09-11|1688|7342|564|2040|2|9|11|3|2040|564|7342|Tuesday|2040Q3|N|N|N|2466399|2466642|2466043|2466318|N|N|N|N|N| +2466410|AAAAAAAAKGCKFCAA|2040-09-12|1688|7342|564|2040|3|9|12|3|2040|564|7342|Wednesday|2040Q3|N|N|N|2466399|2466642|2466044|2466319|N|N|N|N|N| +2466411|AAAAAAAALGCKFCAA|2040-09-13|1688|7342|564|2040|4|9|13|3|2040|564|7342|Thursday|2040Q3|N|N|N|2466399|2466642|2466045|2466320|N|N|N|N|N| +2466412|AAAAAAAAMGCKFCAA|2040-09-14|1688|7342|564|2040|5|9|14|3|2040|564|7342|Friday|2040Q3|N|Y|N|2466399|2466642|2466046|2466321|N|N|N|N|N| +2466413|AAAAAAAANGCKFCAA|2040-09-15|1688|7342|564|2040|6|9|15|3|2040|564|7342|Saturday|2040Q3|N|Y|N|2466399|2466642|2466047|2466322|N|N|N|N|N| +2466414|AAAAAAAAOGCKFCAA|2040-09-16|1688|7342|564|2040|0|9|16|3|2040|564|7342|Sunday|2040Q3|N|N|N|2466399|2466642|2466048|2466323|N|N|N|N|N| +2466415|AAAAAAAAPGCKFCAA|2040-09-17|1688|7342|564|2040|1|9|17|3|2040|564|7342|Monday|2040Q3|N|N|N|2466399|2466642|2466049|2466324|N|N|N|N|N| +2466416|AAAAAAAAAHCKFCAA|2040-09-18|1688|7343|564|2040|2|9|18|3|2040|564|7343|Tuesday|2040Q3|N|N|N|2466399|2466642|2466050|2466325|N|N|N|N|N| +2466417|AAAAAAAABHCKFCAA|2040-09-19|1688|7343|564|2040|3|9|19|3|2040|564|7343|Wednesday|2040Q3|N|N|N|2466399|2466642|2466051|2466326|N|N|N|N|N| +2466418|AAAAAAAACHCKFCAA|2040-09-20|1688|7343|564|2040|4|9|20|3|2040|564|7343|Thursday|2040Q3|N|N|N|2466399|2466642|2466052|2466327|N|N|N|N|N| +2466419|AAAAAAAADHCKFCAA|2040-09-21|1688|7343|564|2040|5|9|21|3|2040|564|7343|Friday|2040Q3|N|Y|N|2466399|2466642|2466053|2466328|N|N|N|N|N| +2466420|AAAAAAAAEHCKFCAA|2040-09-22|1688|7343|564|2040|6|9|22|3|2040|564|7343|Saturday|2040Q3|N|Y|N|2466399|2466642|2466054|2466329|N|N|N|N|N| +2466421|AAAAAAAAFHCKFCAA|2040-09-23|1688|7343|564|2040|0|9|23|3|2040|564|7343|Sunday|2040Q3|N|N|N|2466399|2466642|2466055|2466330|N|N|N|N|N| +2466422|AAAAAAAAGHCKFCAA|2040-09-24|1688|7343|564|2040|1|9|24|3|2040|564|7343|Monday|2040Q3|N|N|N|2466399|2466642|2466056|2466331|N|N|N|N|N| +2466423|AAAAAAAAHHCKFCAA|2040-09-25|1688|7344|564|2040|2|9|25|3|2040|564|7344|Tuesday|2040Q3|N|N|N|2466399|2466642|2466057|2466332|N|N|N|N|N| +2466424|AAAAAAAAIHCKFCAA|2040-09-26|1688|7344|564|2040|3|9|26|3|2040|564|7344|Wednesday|2040Q3|N|N|N|2466399|2466642|2466058|2466333|N|N|N|N|N| +2466425|AAAAAAAAJHCKFCAA|2040-09-27|1688|7344|564|2040|4|9|27|3|2040|564|7344|Thursday|2040Q3|N|N|N|2466399|2466642|2466059|2466334|N|N|N|N|N| +2466426|AAAAAAAAKHCKFCAA|2040-09-28|1688|7344|564|2040|5|9|28|3|2040|564|7344|Friday|2040Q3|N|Y|N|2466399|2466642|2466060|2466335|N|N|N|N|N| +2466427|AAAAAAAALHCKFCAA|2040-09-29|1688|7344|564|2040|6|9|29|3|2040|564|7344|Saturday|2040Q3|N|Y|N|2466399|2466642|2466061|2466336|N|N|N|N|N| +2466428|AAAAAAAAMHCKFCAA|2040-09-30|1688|7344|564|2040|0|9|30|3|2040|564|7344|Sunday|2040Q3|N|N|N|2466399|2466642|2466062|2466337|N|N|N|N|N| +2466429|AAAAAAAANHCKFCAA|2040-10-01|1689|7344|564|2040|1|10|1|4|2040|564|7344|Monday|2040Q4|N|N|N|2466429|2466702|2466063|2466337|N|N|N|N|N| +2466430|AAAAAAAAOHCKFCAA|2040-10-02|1689|7345|564|2040|2|10|2|4|2040|564|7345|Tuesday|2040Q4|N|N|N|2466429|2466702|2466064|2466338|N|N|N|N|N| +2466431|AAAAAAAAPHCKFCAA|2040-10-03|1689|7345|564|2040|3|10|3|4|2040|564|7345|Wednesday|2040Q4|N|N|N|2466429|2466702|2466065|2466339|N|N|N|N|N| +2466432|AAAAAAAAAICKFCAA|2040-10-04|1689|7345|564|2040|4|10|4|4|2040|564|7345|Thursday|2040Q4|N|N|N|2466429|2466702|2466066|2466340|N|N|N|N|N| +2466433|AAAAAAAABICKFCAA|2040-10-05|1689|7345|564|2040|5|10|5|4|2040|564|7345|Friday|2040Q4|N|Y|N|2466429|2466702|2466067|2466341|N|N|N|N|N| +2466434|AAAAAAAACICKFCAA|2040-10-06|1689|7345|564|2040|6|10|6|4|2040|564|7345|Saturday|2040Q4|N|Y|N|2466429|2466702|2466068|2466342|N|N|N|N|N| +2466435|AAAAAAAADICKFCAA|2040-10-07|1689|7345|564|2040|0|10|7|4|2040|564|7345|Sunday|2040Q4|N|N|N|2466429|2466702|2466069|2466343|N|N|N|N|N| +2466436|AAAAAAAAEICKFCAA|2040-10-08|1689|7345|564|2040|1|10|8|4|2040|564|7345|Monday|2040Q4|N|N|N|2466429|2466702|2466070|2466344|N|N|N|N|N| +2466437|AAAAAAAAFICKFCAA|2040-10-09|1689|7346|564|2040|2|10|9|4|2040|564|7346|Tuesday|2040Q4|N|N|N|2466429|2466702|2466071|2466345|N|N|N|N|N| +2466438|AAAAAAAAGICKFCAA|2040-10-10|1689|7346|564|2040|3|10|10|4|2040|564|7346|Wednesday|2040Q4|N|N|N|2466429|2466702|2466072|2466346|N|N|N|N|N| +2466439|AAAAAAAAHICKFCAA|2040-10-11|1689|7346|564|2040|4|10|11|4|2040|564|7346|Thursday|2040Q4|N|N|N|2466429|2466702|2466073|2466347|N|N|N|N|N| +2466440|AAAAAAAAIICKFCAA|2040-10-12|1689|7346|564|2040|5|10|12|4|2040|564|7346|Friday|2040Q4|N|Y|N|2466429|2466702|2466074|2466348|N|N|N|N|N| +2466441|AAAAAAAAJICKFCAA|2040-10-13|1689|7346|564|2040|6|10|13|4|2040|564|7346|Saturday|2040Q4|N|Y|N|2466429|2466702|2466075|2466349|N|N|N|N|N| +2466442|AAAAAAAAKICKFCAA|2040-10-14|1689|7346|564|2040|0|10|14|4|2040|564|7346|Sunday|2040Q4|N|N|N|2466429|2466702|2466076|2466350|N|N|N|N|N| +2466443|AAAAAAAALICKFCAA|2040-10-15|1689|7346|564|2040|1|10|15|4|2040|564|7346|Monday|2040Q4|N|N|N|2466429|2466702|2466077|2466351|N|N|N|N|N| +2466444|AAAAAAAAMICKFCAA|2040-10-16|1689|7347|564|2040|2|10|16|4|2040|564|7347|Tuesday|2040Q4|N|N|N|2466429|2466702|2466078|2466352|N|N|N|N|N| +2466445|AAAAAAAANICKFCAA|2040-10-17|1689|7347|564|2040|3|10|17|4|2040|564|7347|Wednesday|2040Q4|N|N|N|2466429|2466702|2466079|2466353|N|N|N|N|N| +2466446|AAAAAAAAOICKFCAA|2040-10-18|1689|7347|564|2040|4|10|18|4|2040|564|7347|Thursday|2040Q4|N|N|N|2466429|2466702|2466080|2466354|N|N|N|N|N| +2466447|AAAAAAAAPICKFCAA|2040-10-19|1689|7347|564|2040|5|10|19|4|2040|564|7347|Friday|2040Q4|N|Y|N|2466429|2466702|2466081|2466355|N|N|N|N|N| +2466448|AAAAAAAAAJCKFCAA|2040-10-20|1689|7347|564|2040|6|10|20|4|2040|564|7347|Saturday|2040Q4|N|Y|N|2466429|2466702|2466082|2466356|N|N|N|N|N| +2466449|AAAAAAAABJCKFCAA|2040-10-21|1689|7347|564|2040|0|10|21|4|2040|564|7347|Sunday|2040Q4|N|N|N|2466429|2466702|2466083|2466357|N|N|N|N|N| +2466450|AAAAAAAACJCKFCAA|2040-10-22|1689|7347|564|2040|1|10|22|4|2040|564|7347|Monday|2040Q4|N|N|N|2466429|2466702|2466084|2466358|N|N|N|N|N| +2466451|AAAAAAAADJCKFCAA|2040-10-23|1689|7348|564|2040|2|10|23|4|2040|564|7348|Tuesday|2040Q4|N|N|N|2466429|2466702|2466085|2466359|N|N|N|N|N| +2466452|AAAAAAAAEJCKFCAA|2040-10-24|1689|7348|564|2040|3|10|24|4|2040|564|7348|Wednesday|2040Q4|N|N|N|2466429|2466702|2466086|2466360|N|N|N|N|N| +2466453|AAAAAAAAFJCKFCAA|2040-10-25|1689|7348|564|2040|4|10|25|4|2040|564|7348|Thursday|2040Q4|N|N|N|2466429|2466702|2466087|2466361|N|N|N|N|N| +2466454|AAAAAAAAGJCKFCAA|2040-10-26|1689|7348|564|2040|5|10|26|4|2040|564|7348|Friday|2040Q4|N|Y|N|2466429|2466702|2466088|2466362|N|N|N|N|N| +2466455|AAAAAAAAHJCKFCAA|2040-10-27|1689|7348|564|2040|6|10|27|4|2040|564|7348|Saturday|2040Q4|N|Y|N|2466429|2466702|2466089|2466363|N|N|N|N|N| +2466456|AAAAAAAAIJCKFCAA|2040-10-28|1689|7348|564|2040|0|10|28|4|2040|564|7348|Sunday|2040Q4|N|N|N|2466429|2466702|2466090|2466364|N|N|N|N|N| +2466457|AAAAAAAAJJCKFCAA|2040-10-29|1689|7348|564|2040|1|10|29|4|2040|564|7348|Monday|2040Q4|N|N|N|2466429|2466702|2466091|2466365|N|N|N|N|N| +2466458|AAAAAAAAKJCKFCAA|2040-10-30|1689|7349|564|2040|2|10|30|4|2040|564|7349|Tuesday|2040Q4|N|N|N|2466429|2466702|2466092|2466366|N|N|N|N|N| +2466459|AAAAAAAALJCKFCAA|2040-10-31|1689|7349|564|2040|3|10|31|4|2040|564|7349|Wednesday|2040Q4|N|N|N|2466429|2466702|2466093|2466367|N|N|N|N|N| +2466460|AAAAAAAAMJCKFCAA|2040-11-01|1690|7349|564|2040|4|11|1|4|2040|564|7349|Thursday|2040Q4|N|N|N|2466460|2466764|2466094|2466368|N|N|N|N|N| +2466461|AAAAAAAANJCKFCAA|2040-11-02|1690|7349|564|2040|5|11|2|4|2040|564|7349|Friday|2040Q4|N|Y|N|2466460|2466764|2466095|2466369|N|N|N|N|N| +2466462|AAAAAAAAOJCKFCAA|2040-11-03|1690|7349|564|2040|6|11|3|4|2040|564|7349|Saturday|2040Q4|N|Y|N|2466460|2466764|2466096|2466370|N|N|N|N|N| +2466463|AAAAAAAAPJCKFCAA|2040-11-04|1690|7349|564|2040|0|11|4|4|2040|564|7349|Sunday|2040Q4|N|N|N|2466460|2466764|2466097|2466371|N|N|N|N|N| +2466464|AAAAAAAAAKCKFCAA|2040-11-05|1690|7349|564|2040|1|11|5|4|2040|564|7349|Monday|2040Q4|N|N|N|2466460|2466764|2466098|2466372|N|N|N|N|N| +2466465|AAAAAAAABKCKFCAA|2040-11-06|1690|7350|564|2040|2|11|6|4|2040|564|7350|Tuesday|2040Q4|N|N|N|2466460|2466764|2466099|2466373|N|N|N|N|N| +2466466|AAAAAAAACKCKFCAA|2040-11-07|1690|7350|564|2040|3|11|7|4|2040|564|7350|Wednesday|2040Q4|N|N|N|2466460|2466764|2466100|2466374|N|N|N|N|N| +2466467|AAAAAAAADKCKFCAA|2040-11-08|1690|7350|564|2040|4|11|8|4|2040|564|7350|Thursday|2040Q4|N|N|N|2466460|2466764|2466101|2466375|N|N|N|N|N| +2466468|AAAAAAAAEKCKFCAA|2040-11-09|1690|7350|564|2040|5|11|9|4|2040|564|7350|Friday|2040Q4|N|Y|N|2466460|2466764|2466102|2466376|N|N|N|N|N| +2466469|AAAAAAAAFKCKFCAA|2040-11-10|1690|7350|564|2040|6|11|10|4|2040|564|7350|Saturday|2040Q4|N|Y|N|2466460|2466764|2466103|2466377|N|N|N|N|N| +2466470|AAAAAAAAGKCKFCAA|2040-11-11|1690|7350|564|2040|0|11|11|4|2040|564|7350|Sunday|2040Q4|N|N|N|2466460|2466764|2466104|2466378|N|N|N|N|N| +2466471|AAAAAAAAHKCKFCAA|2040-11-12|1690|7350|564|2040|1|11|12|4|2040|564|7350|Monday|2040Q4|N|N|N|2466460|2466764|2466105|2466379|N|N|N|N|N| +2466472|AAAAAAAAIKCKFCAA|2040-11-13|1690|7351|564|2040|2|11|13|4|2040|564|7351|Tuesday|2040Q4|N|N|N|2466460|2466764|2466106|2466380|N|N|N|N|N| +2466473|AAAAAAAAJKCKFCAA|2040-11-14|1690|7351|564|2040|3|11|14|4|2040|564|7351|Wednesday|2040Q4|N|N|N|2466460|2466764|2466107|2466381|N|N|N|N|N| +2466474|AAAAAAAAKKCKFCAA|2040-11-15|1690|7351|564|2040|4|11|15|4|2040|564|7351|Thursday|2040Q4|N|N|N|2466460|2466764|2466108|2466382|N|N|N|N|N| +2466475|AAAAAAAALKCKFCAA|2040-11-16|1690|7351|564|2040|5|11|16|4|2040|564|7351|Friday|2040Q4|N|Y|N|2466460|2466764|2466109|2466383|N|N|N|N|N| +2466476|AAAAAAAAMKCKFCAA|2040-11-17|1690|7351|564|2040|6|11|17|4|2040|564|7351|Saturday|2040Q4|N|Y|N|2466460|2466764|2466110|2466384|N|N|N|N|N| +2466477|AAAAAAAANKCKFCAA|2040-11-18|1690|7351|564|2040|0|11|18|4|2040|564|7351|Sunday|2040Q4|N|N|N|2466460|2466764|2466111|2466385|N|N|N|N|N| +2466478|AAAAAAAAOKCKFCAA|2040-11-19|1690|7351|564|2040|1|11|19|4|2040|564|7351|Monday|2040Q4|N|N|N|2466460|2466764|2466112|2466386|N|N|N|N|N| +2466479|AAAAAAAAPKCKFCAA|2040-11-20|1690|7352|564|2040|2|11|20|4|2040|564|7352|Tuesday|2040Q4|N|N|N|2466460|2466764|2466113|2466387|N|N|N|N|N| +2466480|AAAAAAAAALCKFCAA|2040-11-21|1690|7352|564|2040|3|11|21|4|2040|564|7352|Wednesday|2040Q4|N|N|N|2466460|2466764|2466114|2466388|N|N|N|N|N| +2466481|AAAAAAAABLCKFCAA|2040-11-22|1690|7352|564|2040|4|11|22|4|2040|564|7352|Thursday|2040Q4|N|N|N|2466460|2466764|2466115|2466389|N|N|N|N|N| +2466482|AAAAAAAACLCKFCAA|2040-11-23|1690|7352|564|2040|5|11|23|4|2040|564|7352|Friday|2040Q4|N|Y|N|2466460|2466764|2466116|2466390|N|N|N|N|N| +2466483|AAAAAAAADLCKFCAA|2040-11-24|1690|7352|564|2040|6|11|24|4|2040|564|7352|Saturday|2040Q4|N|Y|N|2466460|2466764|2466117|2466391|N|N|N|N|N| +2466484|AAAAAAAAELCKFCAA|2040-11-25|1690|7352|564|2040|0|11|25|4|2040|564|7352|Sunday|2040Q4|N|N|N|2466460|2466764|2466118|2466392|N|N|N|N|N| +2466485|AAAAAAAAFLCKFCAA|2040-11-26|1690|7352|564|2040|1|11|26|4|2040|564|7352|Monday|2040Q4|N|N|N|2466460|2466764|2466119|2466393|N|N|N|N|N| +2466486|AAAAAAAAGLCKFCAA|2040-11-27|1690|7353|564|2040|2|11|27|4|2040|564|7353|Tuesday|2040Q4|N|N|N|2466460|2466764|2466120|2466394|N|N|N|N|N| +2466487|AAAAAAAAHLCKFCAA|2040-11-28|1690|7353|564|2040|3|11|28|4|2040|564|7353|Wednesday|2040Q4|N|N|N|2466460|2466764|2466121|2466395|N|N|N|N|N| +2466488|AAAAAAAAILCKFCAA|2040-11-29|1690|7353|564|2040|4|11|29|4|2040|564|7353|Thursday|2040Q4|N|N|N|2466460|2466764|2466122|2466396|N|N|N|N|N| +2466489|AAAAAAAAJLCKFCAA|2040-11-30|1690|7353|564|2040|5|11|30|4|2040|564|7353|Friday|2040Q4|N|Y|N|2466460|2466764|2466123|2466397|N|N|N|N|N| +2466490|AAAAAAAAKLCKFCAA|2040-12-01|1691|7353|565|2040|6|12|1|4|2040|565|7353|Saturday|2040Q4|N|Y|N|2466490|2466824|2466124|2466398|N|N|N|N|N| +2466491|AAAAAAAALLCKFCAA|2040-12-02|1691|7353|565|2040|0|12|2|4|2040|565|7353|Sunday|2040Q4|N|N|N|2466490|2466824|2466125|2466399|N|N|N|N|N| +2466492|AAAAAAAAMLCKFCAA|2040-12-03|1691|7353|565|2040|1|12|3|4|2040|565|7353|Monday|2040Q4|N|N|N|2466490|2466824|2466126|2466400|N|N|N|N|N| +2466493|AAAAAAAANLCKFCAA|2040-12-04|1691|7354|565|2040|2|12|4|4|2040|565|7354|Tuesday|2040Q4|N|N|N|2466490|2466824|2466127|2466401|N|N|N|N|N| +2466494|AAAAAAAAOLCKFCAA|2040-12-05|1691|7354|565|2040|3|12|5|4|2040|565|7354|Wednesday|2040Q4|N|N|N|2466490|2466824|2466128|2466402|N|N|N|N|N| +2466495|AAAAAAAAPLCKFCAA|2040-12-06|1691|7354|565|2040|4|12|6|4|2040|565|7354|Thursday|2040Q4|N|N|N|2466490|2466824|2466129|2466403|N|N|N|N|N| +2466496|AAAAAAAAAMCKFCAA|2040-12-07|1691|7354|565|2040|5|12|7|4|2040|565|7354|Friday|2040Q4|N|Y|N|2466490|2466824|2466130|2466404|N|N|N|N|N| +2466497|AAAAAAAABMCKFCAA|2040-12-08|1691|7354|565|2040|6|12|8|4|2040|565|7354|Saturday|2040Q4|N|Y|N|2466490|2466824|2466131|2466405|N|N|N|N|N| +2466498|AAAAAAAACMCKFCAA|2040-12-09|1691|7354|565|2040|0|12|9|4|2040|565|7354|Sunday|2040Q4|N|N|N|2466490|2466824|2466132|2466406|N|N|N|N|N| +2466499|AAAAAAAADMCKFCAA|2040-12-10|1691|7354|565|2040|1|12|10|4|2040|565|7354|Monday|2040Q4|N|N|N|2466490|2466824|2466133|2466407|N|N|N|N|N| +2466500|AAAAAAAAEMCKFCAA|2040-12-11|1691|7355|565|2040|2|12|11|4|2040|565|7355|Tuesday|2040Q4|N|N|N|2466490|2466824|2466134|2466408|N|N|N|N|N| +2466501|AAAAAAAAFMCKFCAA|2040-12-12|1691|7355|565|2040|3|12|12|4|2040|565|7355|Wednesday|2040Q4|N|N|N|2466490|2466824|2466135|2466409|N|N|N|N|N| +2466502|AAAAAAAAGMCKFCAA|2040-12-13|1691|7355|565|2040|4|12|13|4|2040|565|7355|Thursday|2040Q4|N|N|N|2466490|2466824|2466136|2466410|N|N|N|N|N| +2466503|AAAAAAAAHMCKFCAA|2040-12-14|1691|7355|565|2040|5|12|14|4|2040|565|7355|Friday|2040Q4|N|Y|N|2466490|2466824|2466137|2466411|N|N|N|N|N| +2466504|AAAAAAAAIMCKFCAA|2040-12-15|1691|7355|565|2040|6|12|15|4|2040|565|7355|Saturday|2040Q4|N|Y|N|2466490|2466824|2466138|2466412|N|N|N|N|N| +2466505|AAAAAAAAJMCKFCAA|2040-12-16|1691|7355|565|2040|0|12|16|4|2040|565|7355|Sunday|2040Q4|N|N|N|2466490|2466824|2466139|2466413|N|N|N|N|N| +2466506|AAAAAAAAKMCKFCAA|2040-12-17|1691|7355|565|2040|1|12|17|4|2040|565|7355|Monday|2040Q4|N|N|N|2466490|2466824|2466140|2466414|N|N|N|N|N| +2466507|AAAAAAAALMCKFCAA|2040-12-18|1691|7356|565|2040|2|12|18|4|2040|565|7356|Tuesday|2040Q4|N|N|N|2466490|2466824|2466141|2466415|N|N|N|N|N| +2466508|AAAAAAAAMMCKFCAA|2040-12-19|1691|7356|565|2040|3|12|19|4|2040|565|7356|Wednesday|2040Q4|N|N|N|2466490|2466824|2466142|2466416|N|N|N|N|N| +2466509|AAAAAAAANMCKFCAA|2040-12-20|1691|7356|565|2040|4|12|20|4|2040|565|7356|Thursday|2040Q4|N|N|N|2466490|2466824|2466143|2466417|N|N|N|N|N| +2466510|AAAAAAAAOMCKFCAA|2040-12-21|1691|7356|565|2040|5|12|21|4|2040|565|7356|Friday|2040Q4|N|Y|N|2466490|2466824|2466144|2466418|N|N|N|N|N| +2466511|AAAAAAAAPMCKFCAA|2040-12-22|1691|7356|565|2040|6|12|22|4|2040|565|7356|Saturday|2040Q4|N|Y|N|2466490|2466824|2466145|2466419|N|N|N|N|N| +2466512|AAAAAAAAANCKFCAA|2040-12-23|1691|7356|565|2040|0|12|23|4|2040|565|7356|Sunday|2040Q4|N|N|N|2466490|2466824|2466146|2466420|N|N|N|N|N| +2466513|AAAAAAAABNCKFCAA|2040-12-24|1691|7356|565|2040|1|12|24|4|2040|565|7356|Monday|2040Q4|N|N|N|2466490|2466824|2466147|2466421|N|N|N|N|N| +2466514|AAAAAAAACNCKFCAA|2040-12-25|1691|7357|565|2040|2|12|25|4|2040|565|7357|Tuesday|2040Q4|Y|N|N|2466490|2466824|2466148|2466422|N|N|N|N|N| +2466515|AAAAAAAADNCKFCAA|2040-12-26|1691|7357|565|2040|3|12|26|4|2040|565|7357|Wednesday|2040Q4|N|N|Y|2466490|2466824|2466149|2466423|N|N|N|N|N| +2466516|AAAAAAAAENCKFCAA|2040-12-27|1691|7357|565|2040|4|12|27|4|2040|565|7357|Thursday|2040Q4|N|N|N|2466490|2466824|2466150|2466424|N|N|N|N|N| +2466517|AAAAAAAAFNCKFCAA|2040-12-28|1691|7357|565|2040|5|12|28|4|2040|565|7357|Friday|2040Q4|N|Y|N|2466490|2466824|2466151|2466425|N|N|N|N|N| +2466518|AAAAAAAAGNCKFCAA|2040-12-29|1691|7357|565|2040|6|12|29|4|2040|565|7357|Saturday|2040Q4|N|Y|N|2466490|2466824|2466152|2466426|N|N|N|N|N| +2466519|AAAAAAAAHNCKFCAA|2040-12-30|1691|7357|565|2040|0|12|30|4|2040|565|7357|Sunday|2040Q4|N|N|N|2466490|2466824|2466153|2466427|N|N|N|N|N| +2466520|AAAAAAAAINCKFCAA|2040-12-31|1691|7357|565|2040|1|12|31|4|2040|565|7357|Monday|2040Q4|Y|N|N|2466490|2466824|2466154|2466428|N|N|N|N|N| +2466521|AAAAAAAAJNCKFCAA|2041-01-01|1692|7358|565|2041|2|1|1|1|2041|565|7358|Tuesday|2041Q1|Y|N|Y|2466521|2466520|2466155|2466429|N|N|N|N|N| +2466522|AAAAAAAAKNCKFCAA|2041-01-02|1692|7358|565|2041|3|1|2|1|2041|565|7358|Wednesday|2041Q1|N|N|Y|2466521|2466520|2466156|2466430|N|N|N|N|N| +2466523|AAAAAAAALNCKFCAA|2041-01-03|1692|7358|565|2041|4|1|3|1|2041|565|7358|Thursday|2041Q1|N|N|N|2466521|2466520|2466157|2466431|N|N|N|N|N| +2466524|AAAAAAAAMNCKFCAA|2041-01-04|1692|7358|565|2041|5|1|4|1|2041|565|7358|Friday|2041Q1|N|Y|N|2466521|2466520|2466158|2466432|N|N|N|N|N| +2466525|AAAAAAAANNCKFCAA|2041-01-05|1692|7358|565|2041|6|1|5|1|2041|565|7358|Saturday|2041Q1|N|Y|N|2466521|2466520|2466159|2466433|N|N|N|N|N| +2466526|AAAAAAAAONCKFCAA|2041-01-06|1692|7358|565|2041|0|1|6|1|2041|565|7358|Sunday|2041Q1|N|N|N|2466521|2466520|2466160|2466434|N|N|N|N|N| +2466527|AAAAAAAAPNCKFCAA|2041-01-07|1692|7358|565|2041|1|1|7|1|2041|565|7358|Monday|2041Q1|N|N|N|2466521|2466520|2466161|2466435|N|N|N|N|N| +2466528|AAAAAAAAAOCKFCAA|2041-01-08|1692|7359|565|2041|2|1|8|1|2041|565|7359|Tuesday|2041Q1|N|N|N|2466521|2466520|2466162|2466436|N|N|N|N|N| +2466529|AAAAAAAABOCKFCAA|2041-01-09|1692|7359|565|2041|3|1|9|1|2041|565|7359|Wednesday|2041Q1|N|N|N|2466521|2466520|2466163|2466437|N|N|N|N|N| +2466530|AAAAAAAACOCKFCAA|2041-01-10|1692|7359|565|2041|4|1|10|1|2041|565|7359|Thursday|2041Q1|N|N|N|2466521|2466520|2466164|2466438|N|N|N|N|N| +2466531|AAAAAAAADOCKFCAA|2041-01-11|1692|7359|565|2041|5|1|11|1|2041|565|7359|Friday|2041Q1|N|Y|N|2466521|2466520|2466165|2466439|N|N|N|N|N| +2466532|AAAAAAAAEOCKFCAA|2041-01-12|1692|7359|565|2041|6|1|12|1|2041|565|7359|Saturday|2041Q1|N|Y|N|2466521|2466520|2466166|2466440|N|N|N|N|N| +2466533|AAAAAAAAFOCKFCAA|2041-01-13|1692|7359|565|2041|0|1|13|1|2041|565|7359|Sunday|2041Q1|N|N|N|2466521|2466520|2466167|2466441|N|N|N|N|N| +2466534|AAAAAAAAGOCKFCAA|2041-01-14|1692|7359|565|2041|1|1|14|1|2041|565|7359|Monday|2041Q1|N|N|N|2466521|2466520|2466168|2466442|N|N|N|N|N| +2466535|AAAAAAAAHOCKFCAA|2041-01-15|1692|7360|565|2041|2|1|15|1|2041|565|7360|Tuesday|2041Q1|N|N|N|2466521|2466520|2466169|2466443|N|N|N|N|N| +2466536|AAAAAAAAIOCKFCAA|2041-01-16|1692|7360|565|2041|3|1|16|1|2041|565|7360|Wednesday|2041Q1|N|N|N|2466521|2466520|2466170|2466444|N|N|N|N|N| +2466537|AAAAAAAAJOCKFCAA|2041-01-17|1692|7360|565|2041|4|1|17|1|2041|565|7360|Thursday|2041Q1|N|N|N|2466521|2466520|2466171|2466445|N|N|N|N|N| +2466538|AAAAAAAAKOCKFCAA|2041-01-18|1692|7360|565|2041|5|1|18|1|2041|565|7360|Friday|2041Q1|N|Y|N|2466521|2466520|2466172|2466446|N|N|N|N|N| +2466539|AAAAAAAALOCKFCAA|2041-01-19|1692|7360|565|2041|6|1|19|1|2041|565|7360|Saturday|2041Q1|N|Y|N|2466521|2466520|2466173|2466447|N|N|N|N|N| +2466540|AAAAAAAAMOCKFCAA|2041-01-20|1692|7360|565|2041|0|1|20|1|2041|565|7360|Sunday|2041Q1|N|N|N|2466521|2466520|2466174|2466448|N|N|N|N|N| +2466541|AAAAAAAANOCKFCAA|2041-01-21|1692|7360|565|2041|1|1|21|1|2041|565|7360|Monday|2041Q1|N|N|N|2466521|2466520|2466175|2466449|N|N|N|N|N| +2466542|AAAAAAAAOOCKFCAA|2041-01-22|1692|7361|565|2041|2|1|22|1|2041|565|7361|Tuesday|2041Q1|N|N|N|2466521|2466520|2466176|2466450|N|N|N|N|N| +2466543|AAAAAAAAPOCKFCAA|2041-01-23|1692|7361|565|2041|3|1|23|1|2041|565|7361|Wednesday|2041Q1|N|N|N|2466521|2466520|2466177|2466451|N|N|N|N|N| +2466544|AAAAAAAAAPCKFCAA|2041-01-24|1692|7361|565|2041|4|1|24|1|2041|565|7361|Thursday|2041Q1|N|N|N|2466521|2466520|2466178|2466452|N|N|N|N|N| +2466545|AAAAAAAABPCKFCAA|2041-01-25|1692|7361|565|2041|5|1|25|1|2041|565|7361|Friday|2041Q1|N|Y|N|2466521|2466520|2466179|2466453|N|N|N|N|N| +2466546|AAAAAAAACPCKFCAA|2041-01-26|1692|7361|565|2041|6|1|26|1|2041|565|7361|Saturday|2041Q1|N|Y|N|2466521|2466520|2466180|2466454|N|N|N|N|N| +2466547|AAAAAAAADPCKFCAA|2041-01-27|1692|7361|565|2041|0|1|27|1|2041|565|7361|Sunday|2041Q1|N|N|N|2466521|2466520|2466181|2466455|N|N|N|N|N| +2466548|AAAAAAAAEPCKFCAA|2041-01-28|1692|7361|565|2041|1|1|28|1|2041|565|7361|Monday|2041Q1|N|N|N|2466521|2466520|2466182|2466456|N|N|N|N|N| +2466549|AAAAAAAAFPCKFCAA|2041-01-29|1692|7362|565|2041|2|1|29|1|2041|565|7362|Tuesday|2041Q1|N|N|N|2466521|2466520|2466183|2466457|N|N|N|N|N| +2466550|AAAAAAAAGPCKFCAA|2041-01-30|1692|7362|565|2041|3|1|30|1|2041|565|7362|Wednesday|2041Q1|N|N|N|2466521|2466520|2466184|2466458|N|N|N|N|N| +2466551|AAAAAAAAHPCKFCAA|2041-01-31|1692|7362|565|2041|4|1|31|1|2041|565|7362|Thursday|2041Q1|N|N|N|2466521|2466520|2466185|2466459|N|N|N|N|N| +2466552|AAAAAAAAIPCKFCAA|2041-02-01|1693|7362|565|2041|5|2|1|1|2041|565|7362|Friday|2041Q1|N|Y|N|2466552|2466582|2466186|2466460|N|N|N|N|N| +2466553|AAAAAAAAJPCKFCAA|2041-02-02|1693|7362|565|2041|6|2|2|1|2041|565|7362|Saturday|2041Q1|N|Y|N|2466552|2466582|2466187|2466461|N|N|N|N|N| +2466554|AAAAAAAAKPCKFCAA|2041-02-03|1693|7362|565|2041|0|2|3|1|2041|565|7362|Sunday|2041Q1|N|N|N|2466552|2466582|2466188|2466462|N|N|N|N|N| +2466555|AAAAAAAALPCKFCAA|2041-02-04|1693|7362|565|2041|1|2|4|1|2041|565|7362|Monday|2041Q1|N|N|N|2466552|2466582|2466189|2466463|N|N|N|N|N| +2466556|AAAAAAAAMPCKFCAA|2041-02-05|1693|7363|565|2041|2|2|5|1|2041|565|7363|Tuesday|2041Q1|N|N|N|2466552|2466582|2466190|2466464|N|N|N|N|N| +2466557|AAAAAAAANPCKFCAA|2041-02-06|1693|7363|565|2041|3|2|6|1|2041|565|7363|Wednesday|2041Q1|N|N|N|2466552|2466582|2466191|2466465|N|N|N|N|N| +2466558|AAAAAAAAOPCKFCAA|2041-02-07|1693|7363|565|2041|4|2|7|1|2041|565|7363|Thursday|2041Q1|N|N|N|2466552|2466582|2466192|2466466|N|N|N|N|N| +2466559|AAAAAAAAPPCKFCAA|2041-02-08|1693|7363|565|2041|5|2|8|1|2041|565|7363|Friday|2041Q1|N|Y|N|2466552|2466582|2466193|2466467|N|N|N|N|N| +2466560|AAAAAAAAAADKFCAA|2041-02-09|1693|7363|565|2041|6|2|9|1|2041|565|7363|Saturday|2041Q1|N|Y|N|2466552|2466582|2466194|2466468|N|N|N|N|N| +2466561|AAAAAAAABADKFCAA|2041-02-10|1693|7363|565|2041|0|2|10|1|2041|565|7363|Sunday|2041Q1|N|N|N|2466552|2466582|2466195|2466469|N|N|N|N|N| +2466562|AAAAAAAACADKFCAA|2041-02-11|1693|7363|565|2041|1|2|11|1|2041|565|7363|Monday|2041Q1|N|N|N|2466552|2466582|2466196|2466470|N|N|N|N|N| +2466563|AAAAAAAADADKFCAA|2041-02-12|1693|7364|565|2041|2|2|12|1|2041|565|7364|Tuesday|2041Q1|N|N|N|2466552|2466582|2466197|2466471|N|N|N|N|N| +2466564|AAAAAAAAEADKFCAA|2041-02-13|1693|7364|565|2041|3|2|13|1|2041|565|7364|Wednesday|2041Q1|N|N|N|2466552|2466582|2466198|2466472|N|N|N|N|N| +2466565|AAAAAAAAFADKFCAA|2041-02-14|1693|7364|565|2041|4|2|14|1|2041|565|7364|Thursday|2041Q1|N|N|N|2466552|2466582|2466199|2466473|N|N|N|N|N| +2466566|AAAAAAAAGADKFCAA|2041-02-15|1693|7364|565|2041|5|2|15|1|2041|565|7364|Friday|2041Q1|N|Y|N|2466552|2466582|2466200|2466474|N|N|N|N|N| +2466567|AAAAAAAAHADKFCAA|2041-02-16|1693|7364|565|2041|6|2|16|1|2041|565|7364|Saturday|2041Q1|N|Y|N|2466552|2466582|2466201|2466475|N|N|N|N|N| +2466568|AAAAAAAAIADKFCAA|2041-02-17|1693|7364|565|2041|0|2|17|1|2041|565|7364|Sunday|2041Q1|N|N|N|2466552|2466582|2466202|2466476|N|N|N|N|N| +2466569|AAAAAAAAJADKFCAA|2041-02-18|1693|7364|565|2041|1|2|18|1|2041|565|7364|Monday|2041Q1|N|N|N|2466552|2466582|2466203|2466477|N|N|N|N|N| +2466570|AAAAAAAAKADKFCAA|2041-02-19|1693|7365|565|2041|2|2|19|1|2041|565|7365|Tuesday|2041Q1|N|N|N|2466552|2466582|2466204|2466478|N|N|N|N|N| +2466571|AAAAAAAALADKFCAA|2041-02-20|1693|7365|565|2041|3|2|20|1|2041|565|7365|Wednesday|2041Q1|N|N|N|2466552|2466582|2466205|2466479|N|N|N|N|N| +2466572|AAAAAAAAMADKFCAA|2041-02-21|1693|7365|565|2041|4|2|21|1|2041|565|7365|Thursday|2041Q1|N|N|N|2466552|2466582|2466206|2466480|N|N|N|N|N| +2466573|AAAAAAAANADKFCAA|2041-02-22|1693|7365|565|2041|5|2|22|1|2041|565|7365|Friday|2041Q1|N|Y|N|2466552|2466582|2466207|2466481|N|N|N|N|N| +2466574|AAAAAAAAOADKFCAA|2041-02-23|1693|7365|565|2041|6|2|23|1|2041|565|7365|Saturday|2041Q1|N|Y|N|2466552|2466582|2466208|2466482|N|N|N|N|N| +2466575|AAAAAAAAPADKFCAA|2041-02-24|1693|7365|565|2041|0|2|24|1|2041|565|7365|Sunday|2041Q1|N|N|N|2466552|2466582|2466209|2466483|N|N|N|N|N| +2466576|AAAAAAAAABDKFCAA|2041-02-25|1693|7365|565|2041|1|2|25|1|2041|565|7365|Monday|2041Q1|N|N|N|2466552|2466582|2466210|2466484|N|N|N|N|N| +2466577|AAAAAAAABBDKFCAA|2041-02-26|1693|7366|565|2041|2|2|26|1|2041|565|7366|Tuesday|2041Q1|N|N|N|2466552|2466582|2466211|2466485|N|N|N|N|N| +2466578|AAAAAAAACBDKFCAA|2041-02-27|1693|7366|565|2041|3|2|27|1|2041|565|7366|Wednesday|2041Q1|N|N|N|2466552|2466582|2466212|2466486|N|N|N|N|N| +2466579|AAAAAAAADBDKFCAA|2041-02-28|1693|7366|565|2041|4|2|28|1|2041|565|7366|Thursday|2041Q1|N|N|N|2466552|2466582|2466213|2466487|N|N|N|N|N| +2466580|AAAAAAAAEBDKFCAA|2041-03-01|1694|7366|566|2041|5|3|1|1|2041|566|7366|Friday|2041Q1|N|Y|N|2466580|2466638|2466215|2466488|N|N|N|N|N| +2466581|AAAAAAAAFBDKFCAA|2041-03-02|1694|7366|566|2041|6|3|2|1|2041|566|7366|Saturday|2041Q1|N|Y|N|2466580|2466638|2466216|2466489|N|N|N|N|N| +2466582|AAAAAAAAGBDKFCAA|2041-03-03|1694|7366|566|2041|0|3|3|1|2041|566|7366|Sunday|2041Q1|N|N|N|2466580|2466638|2466217|2466490|N|N|N|N|N| +2466583|AAAAAAAAHBDKFCAA|2041-03-04|1694|7366|566|2041|1|3|4|1|2041|566|7366|Monday|2041Q1|N|N|N|2466580|2466638|2466218|2466491|N|N|N|N|N| +2466584|AAAAAAAAIBDKFCAA|2041-03-05|1694|7367|566|2041|2|3|5|1|2041|566|7367|Tuesday|2041Q1|N|N|N|2466580|2466638|2466219|2466492|N|N|N|N|N| +2466585|AAAAAAAAJBDKFCAA|2041-03-06|1694|7367|566|2041|3|3|6|1|2041|566|7367|Wednesday|2041Q1|N|N|N|2466580|2466638|2466220|2466493|N|N|N|N|N| +2466586|AAAAAAAAKBDKFCAA|2041-03-07|1694|7367|566|2041|4|3|7|1|2041|566|7367|Thursday|2041Q1|N|N|N|2466580|2466638|2466221|2466494|N|N|N|N|N| +2466587|AAAAAAAALBDKFCAA|2041-03-08|1694|7367|566|2041|5|3|8|1|2041|566|7367|Friday|2041Q1|N|Y|N|2466580|2466638|2466222|2466495|N|N|N|N|N| +2466588|AAAAAAAAMBDKFCAA|2041-03-09|1694|7367|566|2041|6|3|9|1|2041|566|7367|Saturday|2041Q1|N|Y|N|2466580|2466638|2466223|2466496|N|N|N|N|N| +2466589|AAAAAAAANBDKFCAA|2041-03-10|1694|7367|566|2041|0|3|10|1|2041|566|7367|Sunday|2041Q1|N|N|N|2466580|2466638|2466224|2466497|N|N|N|N|N| +2466590|AAAAAAAAOBDKFCAA|2041-03-11|1694|7367|566|2041|1|3|11|1|2041|566|7367|Monday|2041Q1|N|N|N|2466580|2466638|2466225|2466498|N|N|N|N|N| +2466591|AAAAAAAAPBDKFCAA|2041-03-12|1694|7368|566|2041|2|3|12|1|2041|566|7368|Tuesday|2041Q1|N|N|N|2466580|2466638|2466226|2466499|N|N|N|N|N| +2466592|AAAAAAAAACDKFCAA|2041-03-13|1694|7368|566|2041|3|3|13|1|2041|566|7368|Wednesday|2041Q1|N|N|N|2466580|2466638|2466227|2466500|N|N|N|N|N| +2466593|AAAAAAAABCDKFCAA|2041-03-14|1694|7368|566|2041|4|3|14|1|2041|566|7368|Thursday|2041Q1|N|N|N|2466580|2466638|2466228|2466501|N|N|N|N|N| +2466594|AAAAAAAACCDKFCAA|2041-03-15|1694|7368|566|2041|5|3|15|1|2041|566|7368|Friday|2041Q1|N|Y|N|2466580|2466638|2466229|2466502|N|N|N|N|N| +2466595|AAAAAAAADCDKFCAA|2041-03-16|1694|7368|566|2041|6|3|16|1|2041|566|7368|Saturday|2041Q1|N|Y|N|2466580|2466638|2466230|2466503|N|N|N|N|N| +2466596|AAAAAAAAECDKFCAA|2041-03-17|1694|7368|566|2041|0|3|17|1|2041|566|7368|Sunday|2041Q1|N|N|N|2466580|2466638|2466231|2466504|N|N|N|N|N| +2466597|AAAAAAAAFCDKFCAA|2041-03-18|1694|7368|566|2041|1|3|18|1|2041|566|7368|Monday|2041Q1|N|N|N|2466580|2466638|2466232|2466505|N|N|N|N|N| +2466598|AAAAAAAAGCDKFCAA|2041-03-19|1694|7369|566|2041|2|3|19|1|2041|566|7369|Tuesday|2041Q1|N|N|N|2466580|2466638|2466233|2466506|N|N|N|N|N| +2466599|AAAAAAAAHCDKFCAA|2041-03-20|1694|7369|566|2041|3|3|20|1|2041|566|7369|Wednesday|2041Q1|N|N|N|2466580|2466638|2466234|2466507|N|N|N|N|N| +2466600|AAAAAAAAICDKFCAA|2041-03-21|1694|7369|566|2041|4|3|21|1|2041|566|7369|Thursday|2041Q1|N|N|N|2466580|2466638|2466235|2466508|N|N|N|N|N| +2466601|AAAAAAAAJCDKFCAA|2041-03-22|1694|7369|566|2041|5|3|22|1|2041|566|7369|Friday|2041Q1|N|Y|N|2466580|2466638|2466236|2466509|N|N|N|N|N| +2466602|AAAAAAAAKCDKFCAA|2041-03-23|1694|7369|566|2041|6|3|23|1|2041|566|7369|Saturday|2041Q1|N|Y|N|2466580|2466638|2466237|2466510|N|N|N|N|N| +2466603|AAAAAAAALCDKFCAA|2041-03-24|1694|7369|566|2041|0|3|24|1|2041|566|7369|Sunday|2041Q1|N|N|N|2466580|2466638|2466238|2466511|N|N|N|N|N| +2466604|AAAAAAAAMCDKFCAA|2041-03-25|1694|7369|566|2041|1|3|25|1|2041|566|7369|Monday|2041Q1|N|N|N|2466580|2466638|2466239|2466512|N|N|N|N|N| +2466605|AAAAAAAANCDKFCAA|2041-03-26|1694|7370|566|2041|2|3|26|1|2041|566|7370|Tuesday|2041Q1|N|N|N|2466580|2466638|2466240|2466513|N|N|N|N|N| +2466606|AAAAAAAAOCDKFCAA|2041-03-27|1694|7370|566|2041|3|3|27|1|2041|566|7370|Wednesday|2041Q1|N|N|N|2466580|2466638|2466241|2466514|N|N|N|N|N| +2466607|AAAAAAAAPCDKFCAA|2041-03-28|1694|7370|566|2041|4|3|28|1|2041|566|7370|Thursday|2041Q1|N|N|N|2466580|2466638|2466242|2466515|N|N|N|N|N| +2466608|AAAAAAAAADDKFCAA|2041-03-29|1694|7370|566|2041|5|3|29|1|2041|566|7370|Friday|2041Q1|N|Y|N|2466580|2466638|2466243|2466516|N|N|N|N|N| +2466609|AAAAAAAABDDKFCAA|2041-03-30|1694|7370|566|2041|6|3|30|1|2041|566|7370|Saturday|2041Q1|N|Y|N|2466580|2466638|2466244|2466517|N|N|N|N|N| +2466610|AAAAAAAACDDKFCAA|2041-03-31|1694|7370|566|2041|0|3|31|1|2041|566|7370|Sunday|2041Q1|N|N|N|2466580|2466638|2466245|2466518|N|N|N|N|N| +2466611|AAAAAAAADDDKFCAA|2041-04-01|1695|7370|566|2041|1|4|1|1|2041|566|7370|Monday|2041Q1|N|N|N|2466611|2466700|2466246|2466521|N|N|N|N|N| +2466612|AAAAAAAAEDDKFCAA|2041-04-02|1695|7371|566|2041|2|4|2|2|2041|566|7371|Tuesday|2041Q2|N|N|N|2466611|2466700|2466247|2466522|N|N|N|N|N| +2466613|AAAAAAAAFDDKFCAA|2041-04-03|1695|7371|566|2041|3|4|3|2|2041|566|7371|Wednesday|2041Q2|N|N|N|2466611|2466700|2466248|2466523|N|N|N|N|N| +2466614|AAAAAAAAGDDKFCAA|2041-04-04|1695|7371|566|2041|4|4|4|2|2041|566|7371|Thursday|2041Q2|N|N|N|2466611|2466700|2466249|2466524|N|N|N|N|N| +2466615|AAAAAAAAHDDKFCAA|2041-04-05|1695|7371|566|2041|5|4|5|2|2041|566|7371|Friday|2041Q2|N|Y|N|2466611|2466700|2466250|2466525|N|N|N|N|N| +2466616|AAAAAAAAIDDKFCAA|2041-04-06|1695|7371|566|2041|6|4|6|2|2041|566|7371|Saturday|2041Q2|N|Y|N|2466611|2466700|2466251|2466526|N|N|N|N|N| +2466617|AAAAAAAAJDDKFCAA|2041-04-07|1695|7371|566|2041|0|4|7|2|2041|566|7371|Sunday|2041Q2|N|N|N|2466611|2466700|2466252|2466527|N|N|N|N|N| +2466618|AAAAAAAAKDDKFCAA|2041-04-08|1695|7371|566|2041|1|4|8|2|2041|566|7371|Monday|2041Q2|N|N|N|2466611|2466700|2466253|2466528|N|N|N|N|N| +2466619|AAAAAAAALDDKFCAA|2041-04-09|1695|7372|566|2041|2|4|9|2|2041|566|7372|Tuesday|2041Q2|N|N|N|2466611|2466700|2466254|2466529|N|N|N|N|N| +2466620|AAAAAAAAMDDKFCAA|2041-04-10|1695|7372|566|2041|3|4|10|2|2041|566|7372|Wednesday|2041Q2|N|N|N|2466611|2466700|2466255|2466530|N|N|N|N|N| +2466621|AAAAAAAANDDKFCAA|2041-04-11|1695|7372|566|2041|4|4|11|2|2041|566|7372|Thursday|2041Q2|N|N|N|2466611|2466700|2466256|2466531|N|N|N|N|N| +2466622|AAAAAAAAODDKFCAA|2041-04-12|1695|7372|566|2041|5|4|12|2|2041|566|7372|Friday|2041Q2|N|Y|N|2466611|2466700|2466257|2466532|N|N|N|N|N| +2466623|AAAAAAAAPDDKFCAA|2041-04-13|1695|7372|566|2041|6|4|13|2|2041|566|7372|Saturday|2041Q2|N|Y|N|2466611|2466700|2466258|2466533|N|N|N|N|N| +2466624|AAAAAAAAAEDKFCAA|2041-04-14|1695|7372|566|2041|0|4|14|2|2041|566|7372|Sunday|2041Q2|N|N|N|2466611|2466700|2466259|2466534|N|N|N|N|N| +2466625|AAAAAAAABEDKFCAA|2041-04-15|1695|7372|566|2041|1|4|15|2|2041|566|7372|Monday|2041Q2|N|N|N|2466611|2466700|2466260|2466535|N|N|N|N|N| +2466626|AAAAAAAACEDKFCAA|2041-04-16|1695|7373|566|2041|2|4|16|2|2041|566|7373|Tuesday|2041Q2|N|N|N|2466611|2466700|2466261|2466536|N|N|N|N|N| +2466627|AAAAAAAADEDKFCAA|2041-04-17|1695|7373|566|2041|3|4|17|2|2041|566|7373|Wednesday|2041Q2|N|N|N|2466611|2466700|2466262|2466537|N|N|N|N|N| +2466628|AAAAAAAAEEDKFCAA|2041-04-18|1695|7373|566|2041|4|4|18|2|2041|566|7373|Thursday|2041Q2|N|N|N|2466611|2466700|2466263|2466538|N|N|N|N|N| +2466629|AAAAAAAAFEDKFCAA|2041-04-19|1695|7373|566|2041|5|4|19|2|2041|566|7373|Friday|2041Q2|N|Y|N|2466611|2466700|2466264|2466539|N|N|N|N|N| +2466630|AAAAAAAAGEDKFCAA|2041-04-20|1695|7373|566|2041|6|4|20|2|2041|566|7373|Saturday|2041Q2|N|Y|N|2466611|2466700|2466265|2466540|N|N|N|N|N| +2466631|AAAAAAAAHEDKFCAA|2041-04-21|1695|7373|566|2041|0|4|21|2|2041|566|7373|Sunday|2041Q2|N|N|N|2466611|2466700|2466266|2466541|N|N|N|N|N| +2466632|AAAAAAAAIEDKFCAA|2041-04-22|1695|7373|566|2041|1|4|22|2|2041|566|7373|Monday|2041Q2|N|N|N|2466611|2466700|2466267|2466542|N|N|N|N|N| +2466633|AAAAAAAAJEDKFCAA|2041-04-23|1695|7374|566|2041|2|4|23|2|2041|566|7374|Tuesday|2041Q2|N|N|N|2466611|2466700|2466268|2466543|N|N|N|N|N| +2466634|AAAAAAAAKEDKFCAA|2041-04-24|1695|7374|566|2041|3|4|24|2|2041|566|7374|Wednesday|2041Q2|N|N|N|2466611|2466700|2466269|2466544|N|N|N|N|N| +2466635|AAAAAAAALEDKFCAA|2041-04-25|1695|7374|566|2041|4|4|25|2|2041|566|7374|Thursday|2041Q2|N|N|N|2466611|2466700|2466270|2466545|N|N|N|N|N| +2466636|AAAAAAAAMEDKFCAA|2041-04-26|1695|7374|566|2041|5|4|26|2|2041|566|7374|Friday|2041Q2|N|Y|N|2466611|2466700|2466271|2466546|N|N|N|N|N| +2466637|AAAAAAAANEDKFCAA|2041-04-27|1695|7374|566|2041|6|4|27|2|2041|566|7374|Saturday|2041Q2|N|Y|N|2466611|2466700|2466272|2466547|N|N|N|N|N| +2466638|AAAAAAAAOEDKFCAA|2041-04-28|1695|7374|566|2041|0|4|28|2|2041|566|7374|Sunday|2041Q2|N|N|N|2466611|2466700|2466273|2466548|N|N|N|N|N| +2466639|AAAAAAAAPEDKFCAA|2041-04-29|1695|7374|566|2041|1|4|29|2|2041|566|7374|Monday|2041Q2|N|N|N|2466611|2466700|2466274|2466549|N|N|N|N|N| +2466640|AAAAAAAAAFDKFCAA|2041-04-30|1695|7375|566|2041|2|4|30|2|2041|566|7375|Tuesday|2041Q2|N|N|N|2466611|2466700|2466275|2466550|N|N|N|N|N| +2466641|AAAAAAAABFDKFCAA|2041-05-01|1696|7375|566|2041|3|5|1|2|2041|566|7375|Wednesday|2041Q2|N|N|N|2466641|2466760|2466276|2466551|N|N|N|N|N| +2466642|AAAAAAAACFDKFCAA|2041-05-02|1696|7375|566|2041|4|5|2|2|2041|566|7375|Thursday|2041Q2|N|N|N|2466641|2466760|2466277|2466552|N|N|N|N|N| +2466643|AAAAAAAADFDKFCAA|2041-05-03|1696|7375|566|2041|5|5|3|2|2041|566|7375|Friday|2041Q2|N|Y|N|2466641|2466760|2466278|2466553|N|N|N|N|N| +2466644|AAAAAAAAEFDKFCAA|2041-05-04|1696|7375|566|2041|6|5|4|2|2041|566|7375|Saturday|2041Q2|N|Y|N|2466641|2466760|2466279|2466554|N|N|N|N|N| +2466645|AAAAAAAAFFDKFCAA|2041-05-05|1696|7375|566|2041|0|5|5|2|2041|566|7375|Sunday|2041Q2|N|N|N|2466641|2466760|2466280|2466555|N|N|N|N|N| +2466646|AAAAAAAAGFDKFCAA|2041-05-06|1696|7375|566|2041|1|5|6|2|2041|566|7375|Monday|2041Q2|N|N|N|2466641|2466760|2466281|2466556|N|N|N|N|N| +2466647|AAAAAAAAHFDKFCAA|2041-05-07|1696|7376|566|2041|2|5|7|2|2041|566|7376|Tuesday|2041Q2|N|N|N|2466641|2466760|2466282|2466557|N|N|N|N|N| +2466648|AAAAAAAAIFDKFCAA|2041-05-08|1696|7376|566|2041|3|5|8|2|2041|566|7376|Wednesday|2041Q2|N|N|N|2466641|2466760|2466283|2466558|N|N|N|N|N| +2466649|AAAAAAAAJFDKFCAA|2041-05-09|1696|7376|566|2041|4|5|9|2|2041|566|7376|Thursday|2041Q2|N|N|N|2466641|2466760|2466284|2466559|N|N|N|N|N| +2466650|AAAAAAAAKFDKFCAA|2041-05-10|1696|7376|566|2041|5|5|10|2|2041|566|7376|Friday|2041Q2|N|Y|N|2466641|2466760|2466285|2466560|N|N|N|N|N| +2466651|AAAAAAAALFDKFCAA|2041-05-11|1696|7376|566|2041|6|5|11|2|2041|566|7376|Saturday|2041Q2|N|Y|N|2466641|2466760|2466286|2466561|N|N|N|N|N| +2466652|AAAAAAAAMFDKFCAA|2041-05-12|1696|7376|566|2041|0|5|12|2|2041|566|7376|Sunday|2041Q2|N|N|N|2466641|2466760|2466287|2466562|N|N|N|N|N| +2466653|AAAAAAAANFDKFCAA|2041-05-13|1696|7376|566|2041|1|5|13|2|2041|566|7376|Monday|2041Q2|N|N|N|2466641|2466760|2466288|2466563|N|N|N|N|N| +2466654|AAAAAAAAOFDKFCAA|2041-05-14|1696|7377|566|2041|2|5|14|2|2041|566|7377|Tuesday|2041Q2|N|N|N|2466641|2466760|2466289|2466564|N|N|N|N|N| +2466655|AAAAAAAAPFDKFCAA|2041-05-15|1696|7377|566|2041|3|5|15|2|2041|566|7377|Wednesday|2041Q2|N|N|N|2466641|2466760|2466290|2466565|N|N|N|N|N| +2466656|AAAAAAAAAGDKFCAA|2041-05-16|1696|7377|566|2041|4|5|16|2|2041|566|7377|Thursday|2041Q2|N|N|N|2466641|2466760|2466291|2466566|N|N|N|N|N| +2466657|AAAAAAAABGDKFCAA|2041-05-17|1696|7377|566|2041|5|5|17|2|2041|566|7377|Friday|2041Q2|N|Y|N|2466641|2466760|2466292|2466567|N|N|N|N|N| +2466658|AAAAAAAACGDKFCAA|2041-05-18|1696|7377|566|2041|6|5|18|2|2041|566|7377|Saturday|2041Q2|N|Y|N|2466641|2466760|2466293|2466568|N|N|N|N|N| +2466659|AAAAAAAADGDKFCAA|2041-05-19|1696|7377|566|2041|0|5|19|2|2041|566|7377|Sunday|2041Q2|N|N|N|2466641|2466760|2466294|2466569|N|N|N|N|N| +2466660|AAAAAAAAEGDKFCAA|2041-05-20|1696|7377|566|2041|1|5|20|2|2041|566|7377|Monday|2041Q2|N|N|N|2466641|2466760|2466295|2466570|N|N|N|N|N| +2466661|AAAAAAAAFGDKFCAA|2041-05-21|1696|7378|566|2041|2|5|21|2|2041|566|7378|Tuesday|2041Q2|N|N|N|2466641|2466760|2466296|2466571|N|N|N|N|N| +2466662|AAAAAAAAGGDKFCAA|2041-05-22|1696|7378|566|2041|3|5|22|2|2041|566|7378|Wednesday|2041Q2|N|N|N|2466641|2466760|2466297|2466572|N|N|N|N|N| +2466663|AAAAAAAAHGDKFCAA|2041-05-23|1696|7378|566|2041|4|5|23|2|2041|566|7378|Thursday|2041Q2|N|N|N|2466641|2466760|2466298|2466573|N|N|N|N|N| +2466664|AAAAAAAAIGDKFCAA|2041-05-24|1696|7378|566|2041|5|5|24|2|2041|566|7378|Friday|2041Q2|N|Y|N|2466641|2466760|2466299|2466574|N|N|N|N|N| +2466665|AAAAAAAAJGDKFCAA|2041-05-25|1696|7378|566|2041|6|5|25|2|2041|566|7378|Saturday|2041Q2|N|Y|N|2466641|2466760|2466300|2466575|N|N|N|N|N| +2466666|AAAAAAAAKGDKFCAA|2041-05-26|1696|7378|566|2041|0|5|26|2|2041|566|7378|Sunday|2041Q2|N|N|N|2466641|2466760|2466301|2466576|N|N|N|N|N| +2466667|AAAAAAAALGDKFCAA|2041-05-27|1696|7378|566|2041|1|5|27|2|2041|566|7378|Monday|2041Q2|N|N|N|2466641|2466760|2466302|2466577|N|N|N|N|N| +2466668|AAAAAAAAMGDKFCAA|2041-05-28|1696|7379|566|2041|2|5|28|2|2041|566|7379|Tuesday|2041Q2|N|N|N|2466641|2466760|2466303|2466578|N|N|N|N|N| +2466669|AAAAAAAANGDKFCAA|2041-05-29|1696|7379|566|2041|3|5|29|2|2041|566|7379|Wednesday|2041Q2|N|N|N|2466641|2466760|2466304|2466579|N|N|N|N|N| +2466670|AAAAAAAAOGDKFCAA|2041-05-30|1696|7379|566|2041|4|5|30|2|2041|566|7379|Thursday|2041Q2|N|N|N|2466641|2466760|2466305|2466580|N|N|N|N|N| +2466671|AAAAAAAAPGDKFCAA|2041-05-31|1696|7379|566|2041|5|5|31|2|2041|566|7379|Friday|2041Q2|N|Y|N|2466641|2466760|2466306|2466581|N|N|N|N|N| +2466672|AAAAAAAAAHDKFCAA|2041-06-01|1697|7379|567|2041|6|6|1|2|2041|567|7379|Saturday|2041Q2|N|Y|N|2466672|2466822|2466307|2466582|N|N|N|N|N| +2466673|AAAAAAAABHDKFCAA|2041-06-02|1697|7379|567|2041|0|6|2|2|2041|567|7379|Sunday|2041Q2|N|N|N|2466672|2466822|2466308|2466583|N|N|N|N|N| +2466674|AAAAAAAACHDKFCAA|2041-06-03|1697|7379|567|2041|1|6|3|2|2041|567|7379|Monday|2041Q2|N|N|N|2466672|2466822|2466309|2466584|N|N|N|N|N| +2466675|AAAAAAAADHDKFCAA|2041-06-04|1697|7380|567|2041|2|6|4|2|2041|567|7380|Tuesday|2041Q2|N|N|N|2466672|2466822|2466310|2466585|N|N|N|N|N| +2466676|AAAAAAAAEHDKFCAA|2041-06-05|1697|7380|567|2041|3|6|5|2|2041|567|7380|Wednesday|2041Q2|N|N|N|2466672|2466822|2466311|2466586|N|N|N|N|N| +2466677|AAAAAAAAFHDKFCAA|2041-06-06|1697|7380|567|2041|4|6|6|2|2041|567|7380|Thursday|2041Q2|N|N|N|2466672|2466822|2466312|2466587|N|N|N|N|N| +2466678|AAAAAAAAGHDKFCAA|2041-06-07|1697|7380|567|2041|5|6|7|2|2041|567|7380|Friday|2041Q2|N|Y|N|2466672|2466822|2466313|2466588|N|N|N|N|N| +2466679|AAAAAAAAHHDKFCAA|2041-06-08|1697|7380|567|2041|6|6|8|2|2041|567|7380|Saturday|2041Q2|N|Y|N|2466672|2466822|2466314|2466589|N|N|N|N|N| +2466680|AAAAAAAAIHDKFCAA|2041-06-09|1697|7380|567|2041|0|6|9|2|2041|567|7380|Sunday|2041Q2|N|N|N|2466672|2466822|2466315|2466590|N|N|N|N|N| +2466681|AAAAAAAAJHDKFCAA|2041-06-10|1697|7380|567|2041|1|6|10|2|2041|567|7380|Monday|2041Q2|N|N|N|2466672|2466822|2466316|2466591|N|N|N|N|N| +2466682|AAAAAAAAKHDKFCAA|2041-06-11|1697|7381|567|2041|2|6|11|2|2041|567|7381|Tuesday|2041Q2|N|N|N|2466672|2466822|2466317|2466592|N|N|N|N|N| +2466683|AAAAAAAALHDKFCAA|2041-06-12|1697|7381|567|2041|3|6|12|2|2041|567|7381|Wednesday|2041Q2|N|N|N|2466672|2466822|2466318|2466593|N|N|N|N|N| +2466684|AAAAAAAAMHDKFCAA|2041-06-13|1697|7381|567|2041|4|6|13|2|2041|567|7381|Thursday|2041Q2|N|N|N|2466672|2466822|2466319|2466594|N|N|N|N|N| +2466685|AAAAAAAANHDKFCAA|2041-06-14|1697|7381|567|2041|5|6|14|2|2041|567|7381|Friday|2041Q2|N|Y|N|2466672|2466822|2466320|2466595|N|N|N|N|N| +2466686|AAAAAAAAOHDKFCAA|2041-06-15|1697|7381|567|2041|6|6|15|2|2041|567|7381|Saturday|2041Q2|N|Y|N|2466672|2466822|2466321|2466596|N|N|N|N|N| +2466687|AAAAAAAAPHDKFCAA|2041-06-16|1697|7381|567|2041|0|6|16|2|2041|567|7381|Sunday|2041Q2|N|N|N|2466672|2466822|2466322|2466597|N|N|N|N|N| +2466688|AAAAAAAAAIDKFCAA|2041-06-17|1697|7381|567|2041|1|6|17|2|2041|567|7381|Monday|2041Q2|N|N|N|2466672|2466822|2466323|2466598|N|N|N|N|N| +2466689|AAAAAAAABIDKFCAA|2041-06-18|1697|7382|567|2041|2|6|18|2|2041|567|7382|Tuesday|2041Q2|N|N|N|2466672|2466822|2466324|2466599|N|N|N|N|N| +2466690|AAAAAAAACIDKFCAA|2041-06-19|1697|7382|567|2041|3|6|19|2|2041|567|7382|Wednesday|2041Q2|N|N|N|2466672|2466822|2466325|2466600|N|N|N|N|N| +2466691|AAAAAAAADIDKFCAA|2041-06-20|1697|7382|567|2041|4|6|20|2|2041|567|7382|Thursday|2041Q2|N|N|N|2466672|2466822|2466326|2466601|N|N|N|N|N| +2466692|AAAAAAAAEIDKFCAA|2041-06-21|1697|7382|567|2041|5|6|21|2|2041|567|7382|Friday|2041Q2|N|Y|N|2466672|2466822|2466327|2466602|N|N|N|N|N| +2466693|AAAAAAAAFIDKFCAA|2041-06-22|1697|7382|567|2041|6|6|22|2|2041|567|7382|Saturday|2041Q2|N|Y|N|2466672|2466822|2466328|2466603|N|N|N|N|N| +2466694|AAAAAAAAGIDKFCAA|2041-06-23|1697|7382|567|2041|0|6|23|2|2041|567|7382|Sunday|2041Q2|N|N|N|2466672|2466822|2466329|2466604|N|N|N|N|N| +2466695|AAAAAAAAHIDKFCAA|2041-06-24|1697|7382|567|2041|1|6|24|2|2041|567|7382|Monday|2041Q2|N|N|N|2466672|2466822|2466330|2466605|N|N|N|N|N| +2466696|AAAAAAAAIIDKFCAA|2041-06-25|1697|7383|567|2041|2|6|25|2|2041|567|7383|Tuesday|2041Q2|N|N|N|2466672|2466822|2466331|2466606|N|N|N|N|N| +2466697|AAAAAAAAJIDKFCAA|2041-06-26|1697|7383|567|2041|3|6|26|2|2041|567|7383|Wednesday|2041Q2|N|N|N|2466672|2466822|2466332|2466607|N|N|N|N|N| +2466698|AAAAAAAAKIDKFCAA|2041-06-27|1697|7383|567|2041|4|6|27|2|2041|567|7383|Thursday|2041Q2|N|N|N|2466672|2466822|2466333|2466608|N|N|N|N|N| +2466699|AAAAAAAALIDKFCAA|2041-06-28|1697|7383|567|2041|5|6|28|2|2041|567|7383|Friday|2041Q2|N|Y|N|2466672|2466822|2466334|2466609|N|N|N|N|N| +2466700|AAAAAAAAMIDKFCAA|2041-06-29|1697|7383|567|2041|6|6|29|2|2041|567|7383|Saturday|2041Q2|N|Y|N|2466672|2466822|2466335|2466610|N|N|N|N|N| +2466701|AAAAAAAANIDKFCAA|2041-06-30|1697|7383|567|2041|0|6|30|2|2041|567|7383|Sunday|2041Q2|N|N|N|2466672|2466822|2466336|2466611|N|N|N|N|N| +2466702|AAAAAAAAOIDKFCAA|2041-07-01|1698|7383|567|2041|1|7|1|2|2041|567|7383|Monday|2041Q2|N|N|N|2466702|2466882|2466337|2466611|N|N|N|N|N| +2466703|AAAAAAAAPIDKFCAA|2041-07-02|1698|7384|567|2041|2|7|2|3|2041|567|7384|Tuesday|2041Q3|N|N|N|2466702|2466882|2466338|2466612|N|N|N|N|N| +2466704|AAAAAAAAAJDKFCAA|2041-07-03|1698|7384|567|2041|3|7|3|3|2041|567|7384|Wednesday|2041Q3|N|N|N|2466702|2466882|2466339|2466613|N|N|N|N|N| +2466705|AAAAAAAABJDKFCAA|2041-07-04|1698|7384|567|2041|4|7|4|3|2041|567|7384|Thursday|2041Q3|N|N|N|2466702|2466882|2466340|2466614|N|N|N|N|N| +2466706|AAAAAAAACJDKFCAA|2041-07-05|1698|7384|567|2041|5|7|5|3|2041|567|7384|Friday|2041Q3|Y|Y|N|2466702|2466882|2466341|2466615|N|N|N|N|N| +2466707|AAAAAAAADJDKFCAA|2041-07-06|1698|7384|567|2041|6|7|6|3|2041|567|7384|Saturday|2041Q3|N|Y|Y|2466702|2466882|2466342|2466616|N|N|N|N|N| +2466708|AAAAAAAAEJDKFCAA|2041-07-07|1698|7384|567|2041|0|7|7|3|2041|567|7384|Sunday|2041Q3|N|N|N|2466702|2466882|2466343|2466617|N|N|N|N|N| +2466709|AAAAAAAAFJDKFCAA|2041-07-08|1698|7384|567|2041|1|7|8|3|2041|567|7384|Monday|2041Q3|N|N|N|2466702|2466882|2466344|2466618|N|N|N|N|N| +2466710|AAAAAAAAGJDKFCAA|2041-07-09|1698|7385|567|2041|2|7|9|3|2041|567|7385|Tuesday|2041Q3|N|N|N|2466702|2466882|2466345|2466619|N|N|N|N|N| +2466711|AAAAAAAAHJDKFCAA|2041-07-10|1698|7385|567|2041|3|7|10|3|2041|567|7385|Wednesday|2041Q3|N|N|N|2466702|2466882|2466346|2466620|N|N|N|N|N| +2466712|AAAAAAAAIJDKFCAA|2041-07-11|1698|7385|567|2041|4|7|11|3|2041|567|7385|Thursday|2041Q3|N|N|N|2466702|2466882|2466347|2466621|N|N|N|N|N| +2466713|AAAAAAAAJJDKFCAA|2041-07-12|1698|7385|567|2041|5|7|12|3|2041|567|7385|Friday|2041Q3|N|Y|N|2466702|2466882|2466348|2466622|N|N|N|N|N| +2466714|AAAAAAAAKJDKFCAA|2041-07-13|1698|7385|567|2041|6|7|13|3|2041|567|7385|Saturday|2041Q3|N|Y|N|2466702|2466882|2466349|2466623|N|N|N|N|N| +2466715|AAAAAAAALJDKFCAA|2041-07-14|1698|7385|567|2041|0|7|14|3|2041|567|7385|Sunday|2041Q3|N|N|N|2466702|2466882|2466350|2466624|N|N|N|N|N| +2466716|AAAAAAAAMJDKFCAA|2041-07-15|1698|7385|567|2041|1|7|15|3|2041|567|7385|Monday|2041Q3|N|N|N|2466702|2466882|2466351|2466625|N|N|N|N|N| +2466717|AAAAAAAANJDKFCAA|2041-07-16|1698|7386|567|2041|2|7|16|3|2041|567|7386|Tuesday|2041Q3|N|N|N|2466702|2466882|2466352|2466626|N|N|N|N|N| +2466718|AAAAAAAAOJDKFCAA|2041-07-17|1698|7386|567|2041|3|7|17|3|2041|567|7386|Wednesday|2041Q3|N|N|N|2466702|2466882|2466353|2466627|N|N|N|N|N| +2466719|AAAAAAAAPJDKFCAA|2041-07-18|1698|7386|567|2041|4|7|18|3|2041|567|7386|Thursday|2041Q3|N|N|N|2466702|2466882|2466354|2466628|N|N|N|N|N| +2466720|AAAAAAAAAKDKFCAA|2041-07-19|1698|7386|567|2041|5|7|19|3|2041|567|7386|Friday|2041Q3|N|Y|N|2466702|2466882|2466355|2466629|N|N|N|N|N| +2466721|AAAAAAAABKDKFCAA|2041-07-20|1698|7386|567|2041|6|7|20|3|2041|567|7386|Saturday|2041Q3|N|Y|N|2466702|2466882|2466356|2466630|N|N|N|N|N| +2466722|AAAAAAAACKDKFCAA|2041-07-21|1698|7386|567|2041|0|7|21|3|2041|567|7386|Sunday|2041Q3|N|N|N|2466702|2466882|2466357|2466631|N|N|N|N|N| +2466723|AAAAAAAADKDKFCAA|2041-07-22|1698|7386|567|2041|1|7|22|3|2041|567|7386|Monday|2041Q3|N|N|N|2466702|2466882|2466358|2466632|N|N|N|N|N| +2466724|AAAAAAAAEKDKFCAA|2041-07-23|1698|7387|567|2041|2|7|23|3|2041|567|7387|Tuesday|2041Q3|N|N|N|2466702|2466882|2466359|2466633|N|N|N|N|N| +2466725|AAAAAAAAFKDKFCAA|2041-07-24|1698|7387|567|2041|3|7|24|3|2041|567|7387|Wednesday|2041Q3|N|N|N|2466702|2466882|2466360|2466634|N|N|N|N|N| +2466726|AAAAAAAAGKDKFCAA|2041-07-25|1698|7387|567|2041|4|7|25|3|2041|567|7387|Thursday|2041Q3|N|N|N|2466702|2466882|2466361|2466635|N|N|N|N|N| +2466727|AAAAAAAAHKDKFCAA|2041-07-26|1698|7387|567|2041|5|7|26|3|2041|567|7387|Friday|2041Q3|N|Y|N|2466702|2466882|2466362|2466636|N|N|N|N|N| +2466728|AAAAAAAAIKDKFCAA|2041-07-27|1698|7387|567|2041|6|7|27|3|2041|567|7387|Saturday|2041Q3|N|Y|N|2466702|2466882|2466363|2466637|N|N|N|N|N| +2466729|AAAAAAAAJKDKFCAA|2041-07-28|1698|7387|567|2041|0|7|28|3|2041|567|7387|Sunday|2041Q3|N|N|N|2466702|2466882|2466364|2466638|N|N|N|N|N| +2466730|AAAAAAAAKKDKFCAA|2041-07-29|1698|7387|567|2041|1|7|29|3|2041|567|7387|Monday|2041Q3|N|N|N|2466702|2466882|2466365|2466639|N|N|N|N|N| +2466731|AAAAAAAALKDKFCAA|2041-07-30|1698|7388|567|2041|2|7|30|3|2041|567|7388|Tuesday|2041Q3|N|N|N|2466702|2466882|2466366|2466640|N|N|N|N|N| +2466732|AAAAAAAAMKDKFCAA|2041-07-31|1698|7388|567|2041|3|7|31|3|2041|567|7388|Wednesday|2041Q3|N|N|N|2466702|2466882|2466367|2466641|N|N|N|N|N| +2466733|AAAAAAAANKDKFCAA|2041-08-01|1699|7388|567|2041|4|8|1|3|2041|567|7388|Thursday|2041Q3|N|N|N|2466733|2466944|2466368|2466642|N|N|N|N|N| +2466734|AAAAAAAAOKDKFCAA|2041-08-02|1699|7388|567|2041|5|8|2|3|2041|567|7388|Friday|2041Q3|N|Y|N|2466733|2466944|2466369|2466643|N|N|N|N|N| +2466735|AAAAAAAAPKDKFCAA|2041-08-03|1699|7388|567|2041|6|8|3|3|2041|567|7388|Saturday|2041Q3|N|Y|N|2466733|2466944|2466370|2466644|N|N|N|N|N| +2466736|AAAAAAAAALDKFCAA|2041-08-04|1699|7388|567|2041|0|8|4|3|2041|567|7388|Sunday|2041Q3|N|N|N|2466733|2466944|2466371|2466645|N|N|N|N|N| +2466737|AAAAAAAABLDKFCAA|2041-08-05|1699|7388|567|2041|1|8|5|3|2041|567|7388|Monday|2041Q3|N|N|N|2466733|2466944|2466372|2466646|N|N|N|N|N| +2466738|AAAAAAAACLDKFCAA|2041-08-06|1699|7389|567|2041|2|8|6|3|2041|567|7389|Tuesday|2041Q3|N|N|N|2466733|2466944|2466373|2466647|N|N|N|N|N| +2466739|AAAAAAAADLDKFCAA|2041-08-07|1699|7389|567|2041|3|8|7|3|2041|567|7389|Wednesday|2041Q3|N|N|N|2466733|2466944|2466374|2466648|N|N|N|N|N| +2466740|AAAAAAAAELDKFCAA|2041-08-08|1699|7389|567|2041|4|8|8|3|2041|567|7389|Thursday|2041Q3|N|N|N|2466733|2466944|2466375|2466649|N|N|N|N|N| +2466741|AAAAAAAAFLDKFCAA|2041-08-09|1699|7389|567|2041|5|8|9|3|2041|567|7389|Friday|2041Q3|N|Y|N|2466733|2466944|2466376|2466650|N|N|N|N|N| +2466742|AAAAAAAAGLDKFCAA|2041-08-10|1699|7389|567|2041|6|8|10|3|2041|567|7389|Saturday|2041Q3|N|Y|N|2466733|2466944|2466377|2466651|N|N|N|N|N| +2466743|AAAAAAAAHLDKFCAA|2041-08-11|1699|7389|567|2041|0|8|11|3|2041|567|7389|Sunday|2041Q3|N|N|N|2466733|2466944|2466378|2466652|N|N|N|N|N| +2466744|AAAAAAAAILDKFCAA|2041-08-12|1699|7389|567|2041|1|8|12|3|2041|567|7389|Monday|2041Q3|N|N|N|2466733|2466944|2466379|2466653|N|N|N|N|N| +2466745|AAAAAAAAJLDKFCAA|2041-08-13|1699|7390|567|2041|2|8|13|3|2041|567|7390|Tuesday|2041Q3|N|N|N|2466733|2466944|2466380|2466654|N|N|N|N|N| +2466746|AAAAAAAAKLDKFCAA|2041-08-14|1699|7390|567|2041|3|8|14|3|2041|567|7390|Wednesday|2041Q3|N|N|N|2466733|2466944|2466381|2466655|N|N|N|N|N| +2466747|AAAAAAAALLDKFCAA|2041-08-15|1699|7390|567|2041|4|8|15|3|2041|567|7390|Thursday|2041Q3|N|N|N|2466733|2466944|2466382|2466656|N|N|N|N|N| +2466748|AAAAAAAAMLDKFCAA|2041-08-16|1699|7390|567|2041|5|8|16|3|2041|567|7390|Friday|2041Q3|N|Y|N|2466733|2466944|2466383|2466657|N|N|N|N|N| +2466749|AAAAAAAANLDKFCAA|2041-08-17|1699|7390|567|2041|6|8|17|3|2041|567|7390|Saturday|2041Q3|N|Y|N|2466733|2466944|2466384|2466658|N|N|N|N|N| +2466750|AAAAAAAAOLDKFCAA|2041-08-18|1699|7390|567|2041|0|8|18|3|2041|567|7390|Sunday|2041Q3|N|N|N|2466733|2466944|2466385|2466659|N|N|N|N|N| +2466751|AAAAAAAAPLDKFCAA|2041-08-19|1699|7390|567|2041|1|8|19|3|2041|567|7390|Monday|2041Q3|N|N|N|2466733|2466944|2466386|2466660|N|N|N|N|N| +2466752|AAAAAAAAAMDKFCAA|2041-08-20|1699|7391|567|2041|2|8|20|3|2041|567|7391|Tuesday|2041Q3|N|N|N|2466733|2466944|2466387|2466661|N|N|N|N|N| +2466753|AAAAAAAABMDKFCAA|2041-08-21|1699|7391|567|2041|3|8|21|3|2041|567|7391|Wednesday|2041Q3|N|N|N|2466733|2466944|2466388|2466662|N|N|N|N|N| +2466754|AAAAAAAACMDKFCAA|2041-08-22|1699|7391|567|2041|4|8|22|3|2041|567|7391|Thursday|2041Q3|N|N|N|2466733|2466944|2466389|2466663|N|N|N|N|N| +2466755|AAAAAAAADMDKFCAA|2041-08-23|1699|7391|567|2041|5|8|23|3|2041|567|7391|Friday|2041Q3|N|Y|N|2466733|2466944|2466390|2466664|N|N|N|N|N| +2466756|AAAAAAAAEMDKFCAA|2041-08-24|1699|7391|567|2041|6|8|24|3|2041|567|7391|Saturday|2041Q3|N|Y|N|2466733|2466944|2466391|2466665|N|N|N|N|N| +2466757|AAAAAAAAFMDKFCAA|2041-08-25|1699|7391|567|2041|0|8|25|3|2041|567|7391|Sunday|2041Q3|N|N|N|2466733|2466944|2466392|2466666|N|N|N|N|N| +2466758|AAAAAAAAGMDKFCAA|2041-08-26|1699|7391|567|2041|1|8|26|3|2041|567|7391|Monday|2041Q3|N|N|N|2466733|2466944|2466393|2466667|N|N|N|N|N| +2466759|AAAAAAAAHMDKFCAA|2041-08-27|1699|7392|567|2041|2|8|27|3|2041|567|7392|Tuesday|2041Q3|N|N|N|2466733|2466944|2466394|2466668|N|N|N|N|N| +2466760|AAAAAAAAIMDKFCAA|2041-08-28|1699|7392|567|2041|3|8|28|3|2041|567|7392|Wednesday|2041Q3|N|N|N|2466733|2466944|2466395|2466669|N|N|N|N|N| +2466761|AAAAAAAAJMDKFCAA|2041-08-29|1699|7392|567|2041|4|8|29|3|2041|567|7392|Thursday|2041Q3|N|N|N|2466733|2466944|2466396|2466670|N|N|N|N|N| +2466762|AAAAAAAAKMDKFCAA|2041-08-30|1699|7392|567|2041|5|8|30|3|2041|567|7392|Friday|2041Q3|N|Y|N|2466733|2466944|2466397|2466671|N|N|N|N|N| +2466763|AAAAAAAALMDKFCAA|2041-08-31|1699|7392|567|2041|6|8|31|3|2041|567|7392|Saturday|2041Q3|N|Y|N|2466733|2466944|2466398|2466672|N|N|N|N|N| +2466764|AAAAAAAAMMDKFCAA|2041-09-01|1700|7392|568|2041|0|9|1|3|2041|568|7392|Sunday|2041Q3|N|N|N|2466764|2467006|2466399|2466673|N|N|N|N|N| +2466765|AAAAAAAANMDKFCAA|2041-09-02|1700|7392|568|2041|1|9|2|3|2041|568|7392|Monday|2041Q3|N|N|N|2466764|2467006|2466400|2466674|N|N|N|N|N| +2466766|AAAAAAAAOMDKFCAA|2041-09-03|1700|7393|568|2041|2|9|3|3|2041|568|7393|Tuesday|2041Q3|N|N|N|2466764|2467006|2466401|2466675|N|N|N|N|N| +2466767|AAAAAAAAPMDKFCAA|2041-09-04|1700|7393|568|2041|3|9|4|3|2041|568|7393|Wednesday|2041Q3|N|N|N|2466764|2467006|2466402|2466676|N|N|N|N|N| +2466768|AAAAAAAAANDKFCAA|2041-09-05|1700|7393|568|2041|4|9|5|3|2041|568|7393|Thursday|2041Q3|N|N|N|2466764|2467006|2466403|2466677|N|N|N|N|N| +2466769|AAAAAAAABNDKFCAA|2041-09-06|1700|7393|568|2041|5|9|6|3|2041|568|7393|Friday|2041Q3|N|Y|N|2466764|2467006|2466404|2466678|N|N|N|N|N| +2466770|AAAAAAAACNDKFCAA|2041-09-07|1700|7393|568|2041|6|9|7|3|2041|568|7393|Saturday|2041Q3|N|Y|N|2466764|2467006|2466405|2466679|N|N|N|N|N| +2466771|AAAAAAAADNDKFCAA|2041-09-08|1700|7393|568|2041|0|9|8|3|2041|568|7393|Sunday|2041Q3|N|N|N|2466764|2467006|2466406|2466680|N|N|N|N|N| +2466772|AAAAAAAAENDKFCAA|2041-09-09|1700|7393|568|2041|1|9|9|3|2041|568|7393|Monday|2041Q3|N|N|N|2466764|2467006|2466407|2466681|N|N|N|N|N| +2466773|AAAAAAAAFNDKFCAA|2041-09-10|1700|7394|568|2041|2|9|10|3|2041|568|7394|Tuesday|2041Q3|N|N|N|2466764|2467006|2466408|2466682|N|N|N|N|N| +2466774|AAAAAAAAGNDKFCAA|2041-09-11|1700|7394|568|2041|3|9|11|3|2041|568|7394|Wednesday|2041Q3|N|N|N|2466764|2467006|2466409|2466683|N|N|N|N|N| +2466775|AAAAAAAAHNDKFCAA|2041-09-12|1700|7394|568|2041|4|9|12|3|2041|568|7394|Thursday|2041Q3|N|N|N|2466764|2467006|2466410|2466684|N|N|N|N|N| +2466776|AAAAAAAAINDKFCAA|2041-09-13|1700|7394|568|2041|5|9|13|3|2041|568|7394|Friday|2041Q3|N|Y|N|2466764|2467006|2466411|2466685|N|N|N|N|N| +2466777|AAAAAAAAJNDKFCAA|2041-09-14|1700|7394|568|2041|6|9|14|3|2041|568|7394|Saturday|2041Q3|N|Y|N|2466764|2467006|2466412|2466686|N|N|N|N|N| +2466778|AAAAAAAAKNDKFCAA|2041-09-15|1700|7394|568|2041|0|9|15|3|2041|568|7394|Sunday|2041Q3|N|N|N|2466764|2467006|2466413|2466687|N|N|N|N|N| +2466779|AAAAAAAALNDKFCAA|2041-09-16|1700|7394|568|2041|1|9|16|3|2041|568|7394|Monday|2041Q3|N|N|N|2466764|2467006|2466414|2466688|N|N|N|N|N| +2466780|AAAAAAAAMNDKFCAA|2041-09-17|1700|7395|568|2041|2|9|17|3|2041|568|7395|Tuesday|2041Q3|N|N|N|2466764|2467006|2466415|2466689|N|N|N|N|N| +2466781|AAAAAAAANNDKFCAA|2041-09-18|1700|7395|568|2041|3|9|18|3|2041|568|7395|Wednesday|2041Q3|N|N|N|2466764|2467006|2466416|2466690|N|N|N|N|N| +2466782|AAAAAAAAONDKFCAA|2041-09-19|1700|7395|568|2041|4|9|19|3|2041|568|7395|Thursday|2041Q3|N|N|N|2466764|2467006|2466417|2466691|N|N|N|N|N| +2466783|AAAAAAAAPNDKFCAA|2041-09-20|1700|7395|568|2041|5|9|20|3|2041|568|7395|Friday|2041Q3|N|Y|N|2466764|2467006|2466418|2466692|N|N|N|N|N| +2466784|AAAAAAAAAODKFCAA|2041-09-21|1700|7395|568|2041|6|9|21|3|2041|568|7395|Saturday|2041Q3|N|Y|N|2466764|2467006|2466419|2466693|N|N|N|N|N| +2466785|AAAAAAAABODKFCAA|2041-09-22|1700|7395|568|2041|0|9|22|3|2041|568|7395|Sunday|2041Q3|N|N|N|2466764|2467006|2466420|2466694|N|N|N|N|N| +2466786|AAAAAAAACODKFCAA|2041-09-23|1700|7395|568|2041|1|9|23|3|2041|568|7395|Monday|2041Q3|N|N|N|2466764|2467006|2466421|2466695|N|N|N|N|N| +2466787|AAAAAAAADODKFCAA|2041-09-24|1700|7396|568|2041|2|9|24|3|2041|568|7396|Tuesday|2041Q3|N|N|N|2466764|2467006|2466422|2466696|N|N|N|N|N| +2466788|AAAAAAAAEODKFCAA|2041-09-25|1700|7396|568|2041|3|9|25|3|2041|568|7396|Wednesday|2041Q3|N|N|N|2466764|2467006|2466423|2466697|N|N|N|N|N| +2466789|AAAAAAAAFODKFCAA|2041-09-26|1700|7396|568|2041|4|9|26|3|2041|568|7396|Thursday|2041Q3|N|N|N|2466764|2467006|2466424|2466698|N|N|N|N|N| +2466790|AAAAAAAAGODKFCAA|2041-09-27|1700|7396|568|2041|5|9|27|3|2041|568|7396|Friday|2041Q3|N|Y|N|2466764|2467006|2466425|2466699|N|N|N|N|N| +2466791|AAAAAAAAHODKFCAA|2041-09-28|1700|7396|568|2041|6|9|28|3|2041|568|7396|Saturday|2041Q3|N|Y|N|2466764|2467006|2466426|2466700|N|N|N|N|N| +2466792|AAAAAAAAIODKFCAA|2041-09-29|1700|7396|568|2041|0|9|29|3|2041|568|7396|Sunday|2041Q3|N|N|N|2466764|2467006|2466427|2466701|N|N|N|N|N| +2466793|AAAAAAAAJODKFCAA|2041-09-30|1700|7396|568|2041|1|9|30|3|2041|568|7396|Monday|2041Q3|N|N|N|2466764|2467006|2466428|2466702|N|N|N|N|N| +2466794|AAAAAAAAKODKFCAA|2041-10-01|1701|7397|568|2041|2|10|1|3|2041|568|7397|Tuesday|2041Q3|N|N|N|2466794|2467066|2466429|2466702|N|N|N|N|N| +2466795|AAAAAAAALODKFCAA|2041-10-02|1701|7397|568|2041|3|10|2|4|2041|568|7397|Wednesday|2041Q4|N|N|N|2466794|2467066|2466430|2466703|N|N|N|N|N| +2466796|AAAAAAAAMODKFCAA|2041-10-03|1701|7397|568|2041|4|10|3|4|2041|568|7397|Thursday|2041Q4|N|N|N|2466794|2467066|2466431|2466704|N|N|N|N|N| +2466797|AAAAAAAANODKFCAA|2041-10-04|1701|7397|568|2041|5|10|4|4|2041|568|7397|Friday|2041Q4|N|Y|N|2466794|2467066|2466432|2466705|N|N|N|N|N| +2466798|AAAAAAAAOODKFCAA|2041-10-05|1701|7397|568|2041|6|10|5|4|2041|568|7397|Saturday|2041Q4|N|Y|N|2466794|2467066|2466433|2466706|N|N|N|N|N| +2466799|AAAAAAAAPODKFCAA|2041-10-06|1701|7397|568|2041|0|10|6|4|2041|568|7397|Sunday|2041Q4|N|N|N|2466794|2467066|2466434|2466707|N|N|N|N|N| +2466800|AAAAAAAAAPDKFCAA|2041-10-07|1701|7397|568|2041|1|10|7|4|2041|568|7397|Monday|2041Q4|N|N|N|2466794|2467066|2466435|2466708|N|N|N|N|N| +2466801|AAAAAAAABPDKFCAA|2041-10-08|1701|7398|568|2041|2|10|8|4|2041|568|7398|Tuesday|2041Q4|N|N|N|2466794|2467066|2466436|2466709|N|N|N|N|N| +2466802|AAAAAAAACPDKFCAA|2041-10-09|1701|7398|568|2041|3|10|9|4|2041|568|7398|Wednesday|2041Q4|N|N|N|2466794|2467066|2466437|2466710|N|N|N|N|N| +2466803|AAAAAAAADPDKFCAA|2041-10-10|1701|7398|568|2041|4|10|10|4|2041|568|7398|Thursday|2041Q4|N|N|N|2466794|2467066|2466438|2466711|N|N|N|N|N| +2466804|AAAAAAAAEPDKFCAA|2041-10-11|1701|7398|568|2041|5|10|11|4|2041|568|7398|Friday|2041Q4|N|Y|N|2466794|2467066|2466439|2466712|N|N|N|N|N| +2466805|AAAAAAAAFPDKFCAA|2041-10-12|1701|7398|568|2041|6|10|12|4|2041|568|7398|Saturday|2041Q4|N|Y|N|2466794|2467066|2466440|2466713|N|N|N|N|N| +2466806|AAAAAAAAGPDKFCAA|2041-10-13|1701|7398|568|2041|0|10|13|4|2041|568|7398|Sunday|2041Q4|N|N|N|2466794|2467066|2466441|2466714|N|N|N|N|N| +2466807|AAAAAAAAHPDKFCAA|2041-10-14|1701|7398|568|2041|1|10|14|4|2041|568|7398|Monday|2041Q4|N|N|N|2466794|2467066|2466442|2466715|N|N|N|N|N| +2466808|AAAAAAAAIPDKFCAA|2041-10-15|1701|7399|568|2041|2|10|15|4|2041|568|7399|Tuesday|2041Q4|N|N|N|2466794|2467066|2466443|2466716|N|N|N|N|N| +2466809|AAAAAAAAJPDKFCAA|2041-10-16|1701|7399|568|2041|3|10|16|4|2041|568|7399|Wednesday|2041Q4|N|N|N|2466794|2467066|2466444|2466717|N|N|N|N|N| +2466810|AAAAAAAAKPDKFCAA|2041-10-17|1701|7399|568|2041|4|10|17|4|2041|568|7399|Thursday|2041Q4|N|N|N|2466794|2467066|2466445|2466718|N|N|N|N|N| +2466811|AAAAAAAALPDKFCAA|2041-10-18|1701|7399|568|2041|5|10|18|4|2041|568|7399|Friday|2041Q4|N|Y|N|2466794|2467066|2466446|2466719|N|N|N|N|N| +2466812|AAAAAAAAMPDKFCAA|2041-10-19|1701|7399|568|2041|6|10|19|4|2041|568|7399|Saturday|2041Q4|N|Y|N|2466794|2467066|2466447|2466720|N|N|N|N|N| +2466813|AAAAAAAANPDKFCAA|2041-10-20|1701|7399|568|2041|0|10|20|4|2041|568|7399|Sunday|2041Q4|N|N|N|2466794|2467066|2466448|2466721|N|N|N|N|N| +2466814|AAAAAAAAOPDKFCAA|2041-10-21|1701|7399|568|2041|1|10|21|4|2041|568|7399|Monday|2041Q4|N|N|N|2466794|2467066|2466449|2466722|N|N|N|N|N| +2466815|AAAAAAAAPPDKFCAA|2041-10-22|1701|7400|568|2041|2|10|22|4|2041|568|7400|Tuesday|2041Q4|N|N|N|2466794|2467066|2466450|2466723|N|N|N|N|N| +2466816|AAAAAAAAAAEKFCAA|2041-10-23|1701|7400|568|2041|3|10|23|4|2041|568|7400|Wednesday|2041Q4|N|N|N|2466794|2467066|2466451|2466724|N|N|N|N|N| +2466817|AAAAAAAABAEKFCAA|2041-10-24|1701|7400|568|2041|4|10|24|4|2041|568|7400|Thursday|2041Q4|N|N|N|2466794|2467066|2466452|2466725|N|N|N|N|N| +2466818|AAAAAAAACAEKFCAA|2041-10-25|1701|7400|568|2041|5|10|25|4|2041|568|7400|Friday|2041Q4|N|Y|N|2466794|2467066|2466453|2466726|N|N|N|N|N| +2466819|AAAAAAAADAEKFCAA|2041-10-26|1701|7400|568|2041|6|10|26|4|2041|568|7400|Saturday|2041Q4|N|Y|N|2466794|2467066|2466454|2466727|N|N|N|N|N| +2466820|AAAAAAAAEAEKFCAA|2041-10-27|1701|7400|568|2041|0|10|27|4|2041|568|7400|Sunday|2041Q4|N|N|N|2466794|2467066|2466455|2466728|N|N|N|N|N| +2466821|AAAAAAAAFAEKFCAA|2041-10-28|1701|7400|568|2041|1|10|28|4|2041|568|7400|Monday|2041Q4|N|N|N|2466794|2467066|2466456|2466729|N|N|N|N|N| +2466822|AAAAAAAAGAEKFCAA|2041-10-29|1701|7401|568|2041|2|10|29|4|2041|568|7401|Tuesday|2041Q4|N|N|N|2466794|2467066|2466457|2466730|N|N|N|N|N| +2466823|AAAAAAAAHAEKFCAA|2041-10-30|1701|7401|568|2041|3|10|30|4|2041|568|7401|Wednesday|2041Q4|N|N|N|2466794|2467066|2466458|2466731|N|N|N|N|N| +2466824|AAAAAAAAIAEKFCAA|2041-10-31|1701|7401|568|2041|4|10|31|4|2041|568|7401|Thursday|2041Q4|N|N|N|2466794|2467066|2466459|2466732|N|N|N|N|N| +2466825|AAAAAAAAJAEKFCAA|2041-11-01|1702|7401|568|2041|5|11|1|4|2041|568|7401|Friday|2041Q4|N|Y|N|2466825|2467128|2466460|2466733|N|N|N|N|N| +2466826|AAAAAAAAKAEKFCAA|2041-11-02|1702|7401|568|2041|6|11|2|4|2041|568|7401|Saturday|2041Q4|N|Y|N|2466825|2467128|2466461|2466734|N|N|N|N|N| +2466827|AAAAAAAALAEKFCAA|2041-11-03|1702|7401|568|2041|0|11|3|4|2041|568|7401|Sunday|2041Q4|N|N|N|2466825|2467128|2466462|2466735|N|N|N|N|N| +2466828|AAAAAAAAMAEKFCAA|2041-11-04|1702|7401|568|2041|1|11|4|4|2041|568|7401|Monday|2041Q4|N|N|N|2466825|2467128|2466463|2466736|N|N|N|N|N| +2466829|AAAAAAAANAEKFCAA|2041-11-05|1702|7402|568|2041|2|11|5|4|2041|568|7402|Tuesday|2041Q4|N|N|N|2466825|2467128|2466464|2466737|N|N|N|N|N| +2466830|AAAAAAAAOAEKFCAA|2041-11-06|1702|7402|568|2041|3|11|6|4|2041|568|7402|Wednesday|2041Q4|N|N|N|2466825|2467128|2466465|2466738|N|N|N|N|N| +2466831|AAAAAAAAPAEKFCAA|2041-11-07|1702|7402|568|2041|4|11|7|4|2041|568|7402|Thursday|2041Q4|N|N|N|2466825|2467128|2466466|2466739|N|N|N|N|N| +2466832|AAAAAAAAABEKFCAA|2041-11-08|1702|7402|568|2041|5|11|8|4|2041|568|7402|Friday|2041Q4|N|Y|N|2466825|2467128|2466467|2466740|N|N|N|N|N| +2466833|AAAAAAAABBEKFCAA|2041-11-09|1702|7402|568|2041|6|11|9|4|2041|568|7402|Saturday|2041Q4|N|Y|N|2466825|2467128|2466468|2466741|N|N|N|N|N| +2466834|AAAAAAAACBEKFCAA|2041-11-10|1702|7402|568|2041|0|11|10|4|2041|568|7402|Sunday|2041Q4|N|N|N|2466825|2467128|2466469|2466742|N|N|N|N|N| +2466835|AAAAAAAADBEKFCAA|2041-11-11|1702|7402|568|2041|1|11|11|4|2041|568|7402|Monday|2041Q4|N|N|N|2466825|2467128|2466470|2466743|N|N|N|N|N| +2466836|AAAAAAAAEBEKFCAA|2041-11-12|1702|7403|568|2041|2|11|12|4|2041|568|7403|Tuesday|2041Q4|N|N|N|2466825|2467128|2466471|2466744|N|N|N|N|N| +2466837|AAAAAAAAFBEKFCAA|2041-11-13|1702|7403|568|2041|3|11|13|4|2041|568|7403|Wednesday|2041Q4|N|N|N|2466825|2467128|2466472|2466745|N|N|N|N|N| +2466838|AAAAAAAAGBEKFCAA|2041-11-14|1702|7403|568|2041|4|11|14|4|2041|568|7403|Thursday|2041Q4|N|N|N|2466825|2467128|2466473|2466746|N|N|N|N|N| +2466839|AAAAAAAAHBEKFCAA|2041-11-15|1702|7403|568|2041|5|11|15|4|2041|568|7403|Friday|2041Q4|N|Y|N|2466825|2467128|2466474|2466747|N|N|N|N|N| +2466840|AAAAAAAAIBEKFCAA|2041-11-16|1702|7403|568|2041|6|11|16|4|2041|568|7403|Saturday|2041Q4|N|Y|N|2466825|2467128|2466475|2466748|N|N|N|N|N| +2466841|AAAAAAAAJBEKFCAA|2041-11-17|1702|7403|568|2041|0|11|17|4|2041|568|7403|Sunday|2041Q4|N|N|N|2466825|2467128|2466476|2466749|N|N|N|N|N| +2466842|AAAAAAAAKBEKFCAA|2041-11-18|1702|7403|568|2041|1|11|18|4|2041|568|7403|Monday|2041Q4|N|N|N|2466825|2467128|2466477|2466750|N|N|N|N|N| +2466843|AAAAAAAALBEKFCAA|2041-11-19|1702|7404|568|2041|2|11|19|4|2041|568|7404|Tuesday|2041Q4|N|N|N|2466825|2467128|2466478|2466751|N|N|N|N|N| +2466844|AAAAAAAAMBEKFCAA|2041-11-20|1702|7404|568|2041|3|11|20|4|2041|568|7404|Wednesday|2041Q4|N|N|N|2466825|2467128|2466479|2466752|N|N|N|N|N| +2466845|AAAAAAAANBEKFCAA|2041-11-21|1702|7404|568|2041|4|11|21|4|2041|568|7404|Thursday|2041Q4|N|N|N|2466825|2467128|2466480|2466753|N|N|N|N|N| +2466846|AAAAAAAAOBEKFCAA|2041-11-22|1702|7404|568|2041|5|11|22|4|2041|568|7404|Friday|2041Q4|N|Y|N|2466825|2467128|2466481|2466754|N|N|N|N|N| +2466847|AAAAAAAAPBEKFCAA|2041-11-23|1702|7404|568|2041|6|11|23|4|2041|568|7404|Saturday|2041Q4|N|Y|N|2466825|2467128|2466482|2466755|N|N|N|N|N| +2466848|AAAAAAAAACEKFCAA|2041-11-24|1702|7404|568|2041|0|11|24|4|2041|568|7404|Sunday|2041Q4|N|N|N|2466825|2467128|2466483|2466756|N|N|N|N|N| +2466849|AAAAAAAABCEKFCAA|2041-11-25|1702|7404|568|2041|1|11|25|4|2041|568|7404|Monday|2041Q4|N|N|N|2466825|2467128|2466484|2466757|N|N|N|N|N| +2466850|AAAAAAAACCEKFCAA|2041-11-26|1702|7405|568|2041|2|11|26|4|2041|568|7405|Tuesday|2041Q4|N|N|N|2466825|2467128|2466485|2466758|N|N|N|N|N| +2466851|AAAAAAAADCEKFCAA|2041-11-27|1702|7405|568|2041|3|11|27|4|2041|568|7405|Wednesday|2041Q4|N|N|N|2466825|2467128|2466486|2466759|N|N|N|N|N| +2466852|AAAAAAAAECEKFCAA|2041-11-28|1702|7405|568|2041|4|11|28|4|2041|568|7405|Thursday|2041Q4|N|N|N|2466825|2467128|2466487|2466760|N|N|N|N|N| +2466853|AAAAAAAAFCEKFCAA|2041-11-29|1702|7405|568|2041|5|11|29|4|2041|568|7405|Friday|2041Q4|N|Y|N|2466825|2467128|2466488|2466761|N|N|N|N|N| +2466854|AAAAAAAAGCEKFCAA|2041-11-30|1702|7405|568|2041|6|11|30|4|2041|568|7405|Saturday|2041Q4|N|Y|N|2466825|2467128|2466489|2466762|N|N|N|N|N| +2466855|AAAAAAAAHCEKFCAA|2041-12-01|1703|7405|569|2041|0|12|1|4|2041|569|7405|Sunday|2041Q4|N|N|N|2466855|2467188|2466490|2466763|N|N|N|N|N| +2466856|AAAAAAAAICEKFCAA|2041-12-02|1703|7405|569|2041|1|12|2|4|2041|569|7405|Monday|2041Q4|N|N|N|2466855|2467188|2466491|2466764|N|N|N|N|N| +2466857|AAAAAAAAJCEKFCAA|2041-12-03|1703|7406|569|2041|2|12|3|4|2041|569|7406|Tuesday|2041Q4|N|N|N|2466855|2467188|2466492|2466765|N|N|N|N|N| +2466858|AAAAAAAAKCEKFCAA|2041-12-04|1703|7406|569|2041|3|12|4|4|2041|569|7406|Wednesday|2041Q4|N|N|N|2466855|2467188|2466493|2466766|N|N|N|N|N| +2466859|AAAAAAAALCEKFCAA|2041-12-05|1703|7406|569|2041|4|12|5|4|2041|569|7406|Thursday|2041Q4|N|N|N|2466855|2467188|2466494|2466767|N|N|N|N|N| +2466860|AAAAAAAAMCEKFCAA|2041-12-06|1703|7406|569|2041|5|12|6|4|2041|569|7406|Friday|2041Q4|N|Y|N|2466855|2467188|2466495|2466768|N|N|N|N|N| +2466861|AAAAAAAANCEKFCAA|2041-12-07|1703|7406|569|2041|6|12|7|4|2041|569|7406|Saturday|2041Q4|N|Y|N|2466855|2467188|2466496|2466769|N|N|N|N|N| +2466862|AAAAAAAAOCEKFCAA|2041-12-08|1703|7406|569|2041|0|12|8|4|2041|569|7406|Sunday|2041Q4|N|N|N|2466855|2467188|2466497|2466770|N|N|N|N|N| +2466863|AAAAAAAAPCEKFCAA|2041-12-09|1703|7406|569|2041|1|12|9|4|2041|569|7406|Monday|2041Q4|N|N|N|2466855|2467188|2466498|2466771|N|N|N|N|N| +2466864|AAAAAAAAADEKFCAA|2041-12-10|1703|7407|569|2041|2|12|10|4|2041|569|7407|Tuesday|2041Q4|N|N|N|2466855|2467188|2466499|2466772|N|N|N|N|N| +2466865|AAAAAAAABDEKFCAA|2041-12-11|1703|7407|569|2041|3|12|11|4|2041|569|7407|Wednesday|2041Q4|N|N|N|2466855|2467188|2466500|2466773|N|N|N|N|N| +2466866|AAAAAAAACDEKFCAA|2041-12-12|1703|7407|569|2041|4|12|12|4|2041|569|7407|Thursday|2041Q4|N|N|N|2466855|2467188|2466501|2466774|N|N|N|N|N| +2466867|AAAAAAAADDEKFCAA|2041-12-13|1703|7407|569|2041|5|12|13|4|2041|569|7407|Friday|2041Q4|N|Y|N|2466855|2467188|2466502|2466775|N|N|N|N|N| +2466868|AAAAAAAAEDEKFCAA|2041-12-14|1703|7407|569|2041|6|12|14|4|2041|569|7407|Saturday|2041Q4|N|Y|N|2466855|2467188|2466503|2466776|N|N|N|N|N| +2466869|AAAAAAAAFDEKFCAA|2041-12-15|1703|7407|569|2041|0|12|15|4|2041|569|7407|Sunday|2041Q4|N|N|N|2466855|2467188|2466504|2466777|N|N|N|N|N| +2466870|AAAAAAAAGDEKFCAA|2041-12-16|1703|7407|569|2041|1|12|16|4|2041|569|7407|Monday|2041Q4|N|N|N|2466855|2467188|2466505|2466778|N|N|N|N|N| +2466871|AAAAAAAAHDEKFCAA|2041-12-17|1703|7408|569|2041|2|12|17|4|2041|569|7408|Tuesday|2041Q4|N|N|N|2466855|2467188|2466506|2466779|N|N|N|N|N| +2466872|AAAAAAAAIDEKFCAA|2041-12-18|1703|7408|569|2041|3|12|18|4|2041|569|7408|Wednesday|2041Q4|N|N|N|2466855|2467188|2466507|2466780|N|N|N|N|N| +2466873|AAAAAAAAJDEKFCAA|2041-12-19|1703|7408|569|2041|4|12|19|4|2041|569|7408|Thursday|2041Q4|N|N|N|2466855|2467188|2466508|2466781|N|N|N|N|N| +2466874|AAAAAAAAKDEKFCAA|2041-12-20|1703|7408|569|2041|5|12|20|4|2041|569|7408|Friday|2041Q4|N|Y|N|2466855|2467188|2466509|2466782|N|N|N|N|N| +2466875|AAAAAAAALDEKFCAA|2041-12-21|1703|7408|569|2041|6|12|21|4|2041|569|7408|Saturday|2041Q4|N|Y|N|2466855|2467188|2466510|2466783|N|N|N|N|N| +2466876|AAAAAAAAMDEKFCAA|2041-12-22|1703|7408|569|2041|0|12|22|4|2041|569|7408|Sunday|2041Q4|N|N|N|2466855|2467188|2466511|2466784|N|N|N|N|N| +2466877|AAAAAAAANDEKFCAA|2041-12-23|1703|7408|569|2041|1|12|23|4|2041|569|7408|Monday|2041Q4|N|N|N|2466855|2467188|2466512|2466785|N|N|N|N|N| +2466878|AAAAAAAAODEKFCAA|2041-12-24|1703|7409|569|2041|2|12|24|4|2041|569|7409|Tuesday|2041Q4|N|N|N|2466855|2467188|2466513|2466786|N|N|N|N|N| +2466879|AAAAAAAAPDEKFCAA|2041-12-25|1703|7409|569|2041|3|12|25|4|2041|569|7409|Wednesday|2041Q4|N|N|N|2466855|2467188|2466514|2466787|N|N|N|N|N| +2466880|AAAAAAAAAEEKFCAA|2041-12-26|1703|7409|569|2041|4|12|26|4|2041|569|7409|Thursday|2041Q4|Y|N|N|2466855|2467188|2466515|2466788|N|N|N|N|N| +2466881|AAAAAAAABEEKFCAA|2041-12-27|1703|7409|569|2041|5|12|27|4|2041|569|7409|Friday|2041Q4|N|Y|Y|2466855|2467188|2466516|2466789|N|N|N|N|N| +2466882|AAAAAAAACEEKFCAA|2041-12-28|1703|7409|569|2041|6|12|28|4|2041|569|7409|Saturday|2041Q4|N|Y|N|2466855|2467188|2466517|2466790|N|N|N|N|N| +2466883|AAAAAAAADEEKFCAA|2041-12-29|1703|7409|569|2041|0|12|29|4|2041|569|7409|Sunday|2041Q4|N|N|N|2466855|2467188|2466518|2466791|N|N|N|N|N| +2466884|AAAAAAAAEEEKFCAA|2041-12-30|1703|7409|569|2041|1|12|30|4|2041|569|7409|Monday|2041Q4|N|N|N|2466855|2467188|2466519|2466792|N|N|N|N|N| +2466885|AAAAAAAAFEEKFCAA|2041-12-31|1703|7410|569|2041|2|12|31|4|2041|569|7410|Tuesday|2041Q4|N|N|N|2466855|2467188|2466520|2466793|N|N|N|N|N| +2466886|AAAAAAAAGEEKFCAA|2042-01-01|1704|7410|569|2042|3|1|1|1|2042|569|7410|Wednesday|2042Q1|Y|N|N|2466886|2466885|2466521|2466794|N|N|N|N|N| +2466887|AAAAAAAAHEEKFCAA|2042-01-02|1704|7410|569|2042|4|1|2|1|2042|569|7410|Thursday|2042Q1|N|N|Y|2466886|2466885|2466522|2466795|N|N|N|N|N| +2466888|AAAAAAAAIEEKFCAA|2042-01-03|1704|7410|569|2042|5|1|3|1|2042|569|7410|Friday|2042Q1|N|Y|N|2466886|2466885|2466523|2466796|N|N|N|N|N| +2466889|AAAAAAAAJEEKFCAA|2042-01-04|1704|7410|569|2042|6|1|4|1|2042|569|7410|Saturday|2042Q1|N|Y|N|2466886|2466885|2466524|2466797|N|N|N|N|N| +2466890|AAAAAAAAKEEKFCAA|2042-01-05|1704|7410|569|2042|0|1|5|1|2042|569|7410|Sunday|2042Q1|N|N|N|2466886|2466885|2466525|2466798|N|N|N|N|N| +2466891|AAAAAAAALEEKFCAA|2042-01-06|1704|7410|569|2042|1|1|6|1|2042|569|7410|Monday|2042Q1|N|N|N|2466886|2466885|2466526|2466799|N|N|N|N|N| +2466892|AAAAAAAAMEEKFCAA|2042-01-07|1704|7411|569|2042|2|1|7|1|2042|569|7411|Tuesday|2042Q1|N|N|N|2466886|2466885|2466527|2466800|N|N|N|N|N| +2466893|AAAAAAAANEEKFCAA|2042-01-08|1704|7411|569|2042|3|1|8|1|2042|569|7411|Wednesday|2042Q1|N|N|N|2466886|2466885|2466528|2466801|N|N|N|N|N| +2466894|AAAAAAAAOEEKFCAA|2042-01-09|1704|7411|569|2042|4|1|9|1|2042|569|7411|Thursday|2042Q1|N|N|N|2466886|2466885|2466529|2466802|N|N|N|N|N| +2466895|AAAAAAAAPEEKFCAA|2042-01-10|1704|7411|569|2042|5|1|10|1|2042|569|7411|Friday|2042Q1|N|Y|N|2466886|2466885|2466530|2466803|N|N|N|N|N| +2466896|AAAAAAAAAFEKFCAA|2042-01-11|1704|7411|569|2042|6|1|11|1|2042|569|7411|Saturday|2042Q1|N|Y|N|2466886|2466885|2466531|2466804|N|N|N|N|N| +2466897|AAAAAAAABFEKFCAA|2042-01-12|1704|7411|569|2042|0|1|12|1|2042|569|7411|Sunday|2042Q1|N|N|N|2466886|2466885|2466532|2466805|N|N|N|N|N| +2466898|AAAAAAAACFEKFCAA|2042-01-13|1704|7411|569|2042|1|1|13|1|2042|569|7411|Monday|2042Q1|N|N|N|2466886|2466885|2466533|2466806|N|N|N|N|N| +2466899|AAAAAAAADFEKFCAA|2042-01-14|1704|7412|569|2042|2|1|14|1|2042|569|7412|Tuesday|2042Q1|N|N|N|2466886|2466885|2466534|2466807|N|N|N|N|N| +2466900|AAAAAAAAEFEKFCAA|2042-01-15|1704|7412|569|2042|3|1|15|1|2042|569|7412|Wednesday|2042Q1|N|N|N|2466886|2466885|2466535|2466808|N|N|N|N|N| +2466901|AAAAAAAAFFEKFCAA|2042-01-16|1704|7412|569|2042|4|1|16|1|2042|569|7412|Thursday|2042Q1|N|N|N|2466886|2466885|2466536|2466809|N|N|N|N|N| +2466902|AAAAAAAAGFEKFCAA|2042-01-17|1704|7412|569|2042|5|1|17|1|2042|569|7412|Friday|2042Q1|N|Y|N|2466886|2466885|2466537|2466810|N|N|N|N|N| +2466903|AAAAAAAAHFEKFCAA|2042-01-18|1704|7412|569|2042|6|1|18|1|2042|569|7412|Saturday|2042Q1|N|Y|N|2466886|2466885|2466538|2466811|N|N|N|N|N| +2466904|AAAAAAAAIFEKFCAA|2042-01-19|1704|7412|569|2042|0|1|19|1|2042|569|7412|Sunday|2042Q1|N|N|N|2466886|2466885|2466539|2466812|N|N|N|N|N| +2466905|AAAAAAAAJFEKFCAA|2042-01-20|1704|7412|569|2042|1|1|20|1|2042|569|7412|Monday|2042Q1|N|N|N|2466886|2466885|2466540|2466813|N|N|N|N|N| +2466906|AAAAAAAAKFEKFCAA|2042-01-21|1704|7413|569|2042|2|1|21|1|2042|569|7413|Tuesday|2042Q1|N|N|N|2466886|2466885|2466541|2466814|N|N|N|N|N| +2466907|AAAAAAAALFEKFCAA|2042-01-22|1704|7413|569|2042|3|1|22|1|2042|569|7413|Wednesday|2042Q1|N|N|N|2466886|2466885|2466542|2466815|N|N|N|N|N| +2466908|AAAAAAAAMFEKFCAA|2042-01-23|1704|7413|569|2042|4|1|23|1|2042|569|7413|Thursday|2042Q1|N|N|N|2466886|2466885|2466543|2466816|N|N|N|N|N| +2466909|AAAAAAAANFEKFCAA|2042-01-24|1704|7413|569|2042|5|1|24|1|2042|569|7413|Friday|2042Q1|N|Y|N|2466886|2466885|2466544|2466817|N|N|N|N|N| +2466910|AAAAAAAAOFEKFCAA|2042-01-25|1704|7413|569|2042|6|1|25|1|2042|569|7413|Saturday|2042Q1|N|Y|N|2466886|2466885|2466545|2466818|N|N|N|N|N| +2466911|AAAAAAAAPFEKFCAA|2042-01-26|1704|7413|569|2042|0|1|26|1|2042|569|7413|Sunday|2042Q1|N|N|N|2466886|2466885|2466546|2466819|N|N|N|N|N| +2466912|AAAAAAAAAGEKFCAA|2042-01-27|1704|7413|569|2042|1|1|27|1|2042|569|7413|Monday|2042Q1|N|N|N|2466886|2466885|2466547|2466820|N|N|N|N|N| +2466913|AAAAAAAABGEKFCAA|2042-01-28|1704|7414|569|2042|2|1|28|1|2042|569|7414|Tuesday|2042Q1|N|N|N|2466886|2466885|2466548|2466821|N|N|N|N|N| +2466914|AAAAAAAACGEKFCAA|2042-01-29|1704|7414|569|2042|3|1|29|1|2042|569|7414|Wednesday|2042Q1|N|N|N|2466886|2466885|2466549|2466822|N|N|N|N|N| +2466915|AAAAAAAADGEKFCAA|2042-01-30|1704|7414|569|2042|4|1|30|1|2042|569|7414|Thursday|2042Q1|N|N|N|2466886|2466885|2466550|2466823|N|N|N|N|N| +2466916|AAAAAAAAEGEKFCAA|2042-01-31|1704|7414|569|2042|5|1|31|1|2042|569|7414|Friday|2042Q1|N|Y|N|2466886|2466885|2466551|2466824|N|N|N|N|N| +2466917|AAAAAAAAFGEKFCAA|2042-02-01|1705|7414|569|2042|6|2|1|1|2042|569|7414|Saturday|2042Q1|N|Y|N|2466917|2466947|2466552|2466825|N|N|N|N|N| +2466918|AAAAAAAAGGEKFCAA|2042-02-02|1705|7414|569|2042|0|2|2|1|2042|569|7414|Sunday|2042Q1|N|N|N|2466917|2466947|2466553|2466826|N|N|N|N|N| +2466919|AAAAAAAAHGEKFCAA|2042-02-03|1705|7414|569|2042|1|2|3|1|2042|569|7414|Monday|2042Q1|N|N|N|2466917|2466947|2466554|2466827|N|N|N|N|N| +2466920|AAAAAAAAIGEKFCAA|2042-02-04|1705|7415|569|2042|2|2|4|1|2042|569|7415|Tuesday|2042Q1|N|N|N|2466917|2466947|2466555|2466828|N|N|N|N|N| +2466921|AAAAAAAAJGEKFCAA|2042-02-05|1705|7415|569|2042|3|2|5|1|2042|569|7415|Wednesday|2042Q1|N|N|N|2466917|2466947|2466556|2466829|N|N|N|N|N| +2466922|AAAAAAAAKGEKFCAA|2042-02-06|1705|7415|569|2042|4|2|6|1|2042|569|7415|Thursday|2042Q1|N|N|N|2466917|2466947|2466557|2466830|N|N|N|N|N| +2466923|AAAAAAAALGEKFCAA|2042-02-07|1705|7415|569|2042|5|2|7|1|2042|569|7415|Friday|2042Q1|N|Y|N|2466917|2466947|2466558|2466831|N|N|N|N|N| +2466924|AAAAAAAAMGEKFCAA|2042-02-08|1705|7415|569|2042|6|2|8|1|2042|569|7415|Saturday|2042Q1|N|Y|N|2466917|2466947|2466559|2466832|N|N|N|N|N| +2466925|AAAAAAAANGEKFCAA|2042-02-09|1705|7415|569|2042|0|2|9|1|2042|569|7415|Sunday|2042Q1|N|N|N|2466917|2466947|2466560|2466833|N|N|N|N|N| +2466926|AAAAAAAAOGEKFCAA|2042-02-10|1705|7415|569|2042|1|2|10|1|2042|569|7415|Monday|2042Q1|N|N|N|2466917|2466947|2466561|2466834|N|N|N|N|N| +2466927|AAAAAAAAPGEKFCAA|2042-02-11|1705|7416|569|2042|2|2|11|1|2042|569|7416|Tuesday|2042Q1|N|N|N|2466917|2466947|2466562|2466835|N|N|N|N|N| +2466928|AAAAAAAAAHEKFCAA|2042-02-12|1705|7416|569|2042|3|2|12|1|2042|569|7416|Wednesday|2042Q1|N|N|N|2466917|2466947|2466563|2466836|N|N|N|N|N| +2466929|AAAAAAAABHEKFCAA|2042-02-13|1705|7416|569|2042|4|2|13|1|2042|569|7416|Thursday|2042Q1|N|N|N|2466917|2466947|2466564|2466837|N|N|N|N|N| +2466930|AAAAAAAACHEKFCAA|2042-02-14|1705|7416|569|2042|5|2|14|1|2042|569|7416|Friday|2042Q1|N|Y|N|2466917|2466947|2466565|2466838|N|N|N|N|N| +2466931|AAAAAAAADHEKFCAA|2042-02-15|1705|7416|569|2042|6|2|15|1|2042|569|7416|Saturday|2042Q1|N|Y|N|2466917|2466947|2466566|2466839|N|N|N|N|N| +2466932|AAAAAAAAEHEKFCAA|2042-02-16|1705|7416|569|2042|0|2|16|1|2042|569|7416|Sunday|2042Q1|N|N|N|2466917|2466947|2466567|2466840|N|N|N|N|N| +2466933|AAAAAAAAFHEKFCAA|2042-02-17|1705|7416|569|2042|1|2|17|1|2042|569|7416|Monday|2042Q1|N|N|N|2466917|2466947|2466568|2466841|N|N|N|N|N| +2466934|AAAAAAAAGHEKFCAA|2042-02-18|1705|7417|569|2042|2|2|18|1|2042|569|7417|Tuesday|2042Q1|N|N|N|2466917|2466947|2466569|2466842|N|N|N|N|N| +2466935|AAAAAAAAHHEKFCAA|2042-02-19|1705|7417|569|2042|3|2|19|1|2042|569|7417|Wednesday|2042Q1|N|N|N|2466917|2466947|2466570|2466843|N|N|N|N|N| +2466936|AAAAAAAAIHEKFCAA|2042-02-20|1705|7417|569|2042|4|2|20|1|2042|569|7417|Thursday|2042Q1|N|N|N|2466917|2466947|2466571|2466844|N|N|N|N|N| +2466937|AAAAAAAAJHEKFCAA|2042-02-21|1705|7417|569|2042|5|2|21|1|2042|569|7417|Friday|2042Q1|N|Y|N|2466917|2466947|2466572|2466845|N|N|N|N|N| +2466938|AAAAAAAAKHEKFCAA|2042-02-22|1705|7417|569|2042|6|2|22|1|2042|569|7417|Saturday|2042Q1|N|Y|N|2466917|2466947|2466573|2466846|N|N|N|N|N| +2466939|AAAAAAAALHEKFCAA|2042-02-23|1705|7417|569|2042|0|2|23|1|2042|569|7417|Sunday|2042Q1|N|N|N|2466917|2466947|2466574|2466847|N|N|N|N|N| +2466940|AAAAAAAAMHEKFCAA|2042-02-24|1705|7417|569|2042|1|2|24|1|2042|569|7417|Monday|2042Q1|N|N|N|2466917|2466947|2466575|2466848|N|N|N|N|N| +2466941|AAAAAAAANHEKFCAA|2042-02-25|1705|7418|569|2042|2|2|25|1|2042|569|7418|Tuesday|2042Q1|N|N|N|2466917|2466947|2466576|2466849|N|N|N|N|N| +2466942|AAAAAAAAOHEKFCAA|2042-02-26|1705|7418|569|2042|3|2|26|1|2042|569|7418|Wednesday|2042Q1|N|N|N|2466917|2466947|2466577|2466850|N|N|N|N|N| +2466943|AAAAAAAAPHEKFCAA|2042-02-27|1705|7418|569|2042|4|2|27|1|2042|569|7418|Thursday|2042Q1|N|N|N|2466917|2466947|2466578|2466851|N|N|N|N|N| +2466944|AAAAAAAAAIEKFCAA|2042-02-28|1705|7418|569|2042|5|2|28|1|2042|569|7418|Friday|2042Q1|N|Y|N|2466917|2466947|2466579|2466852|N|N|N|N|N| +2466945|AAAAAAAABIEKFCAA|2042-03-01|1706|7418|570|2042|6|3|1|1|2042|570|7418|Saturday|2042Q1|N|Y|N|2466945|2467003|2466580|2466853|N|N|N|N|N| +2466946|AAAAAAAACIEKFCAA|2042-03-02|1706|7418|570|2042|0|3|2|1|2042|570|7418|Sunday|2042Q1|N|N|N|2466945|2467003|2466581|2466854|N|N|N|N|N| +2466947|AAAAAAAADIEKFCAA|2042-03-03|1706|7418|570|2042|1|3|3|1|2042|570|7418|Monday|2042Q1|N|N|N|2466945|2467003|2466582|2466855|N|N|N|N|N| +2466948|AAAAAAAAEIEKFCAA|2042-03-04|1706|7419|570|2042|2|3|4|1|2042|570|7419|Tuesday|2042Q1|N|N|N|2466945|2467003|2466583|2466856|N|N|N|N|N| +2466949|AAAAAAAAFIEKFCAA|2042-03-05|1706|7419|570|2042|3|3|5|1|2042|570|7419|Wednesday|2042Q1|N|N|N|2466945|2467003|2466584|2466857|N|N|N|N|N| +2466950|AAAAAAAAGIEKFCAA|2042-03-06|1706|7419|570|2042|4|3|6|1|2042|570|7419|Thursday|2042Q1|N|N|N|2466945|2467003|2466585|2466858|N|N|N|N|N| +2466951|AAAAAAAAHIEKFCAA|2042-03-07|1706|7419|570|2042|5|3|7|1|2042|570|7419|Friday|2042Q1|N|Y|N|2466945|2467003|2466586|2466859|N|N|N|N|N| +2466952|AAAAAAAAIIEKFCAA|2042-03-08|1706|7419|570|2042|6|3|8|1|2042|570|7419|Saturday|2042Q1|N|Y|N|2466945|2467003|2466587|2466860|N|N|N|N|N| +2466953|AAAAAAAAJIEKFCAA|2042-03-09|1706|7419|570|2042|0|3|9|1|2042|570|7419|Sunday|2042Q1|N|N|N|2466945|2467003|2466588|2466861|N|N|N|N|N| +2466954|AAAAAAAAKIEKFCAA|2042-03-10|1706|7419|570|2042|1|3|10|1|2042|570|7419|Monday|2042Q1|N|N|N|2466945|2467003|2466589|2466862|N|N|N|N|N| +2466955|AAAAAAAALIEKFCAA|2042-03-11|1706|7420|570|2042|2|3|11|1|2042|570|7420|Tuesday|2042Q1|N|N|N|2466945|2467003|2466590|2466863|N|N|N|N|N| +2466956|AAAAAAAAMIEKFCAA|2042-03-12|1706|7420|570|2042|3|3|12|1|2042|570|7420|Wednesday|2042Q1|N|N|N|2466945|2467003|2466591|2466864|N|N|N|N|N| +2466957|AAAAAAAANIEKFCAA|2042-03-13|1706|7420|570|2042|4|3|13|1|2042|570|7420|Thursday|2042Q1|N|N|N|2466945|2467003|2466592|2466865|N|N|N|N|N| +2466958|AAAAAAAAOIEKFCAA|2042-03-14|1706|7420|570|2042|5|3|14|1|2042|570|7420|Friday|2042Q1|N|Y|N|2466945|2467003|2466593|2466866|N|N|N|N|N| +2466959|AAAAAAAAPIEKFCAA|2042-03-15|1706|7420|570|2042|6|3|15|1|2042|570|7420|Saturday|2042Q1|N|Y|N|2466945|2467003|2466594|2466867|N|N|N|N|N| +2466960|AAAAAAAAAJEKFCAA|2042-03-16|1706|7420|570|2042|0|3|16|1|2042|570|7420|Sunday|2042Q1|N|N|N|2466945|2467003|2466595|2466868|N|N|N|N|N| +2466961|AAAAAAAABJEKFCAA|2042-03-17|1706|7420|570|2042|1|3|17|1|2042|570|7420|Monday|2042Q1|N|N|N|2466945|2467003|2466596|2466869|N|N|N|N|N| +2466962|AAAAAAAACJEKFCAA|2042-03-18|1706|7421|570|2042|2|3|18|1|2042|570|7421|Tuesday|2042Q1|N|N|N|2466945|2467003|2466597|2466870|N|N|N|N|N| +2466963|AAAAAAAADJEKFCAA|2042-03-19|1706|7421|570|2042|3|3|19|1|2042|570|7421|Wednesday|2042Q1|N|N|N|2466945|2467003|2466598|2466871|N|N|N|N|N| +2466964|AAAAAAAAEJEKFCAA|2042-03-20|1706|7421|570|2042|4|3|20|1|2042|570|7421|Thursday|2042Q1|N|N|N|2466945|2467003|2466599|2466872|N|N|N|N|N| +2466965|AAAAAAAAFJEKFCAA|2042-03-21|1706|7421|570|2042|5|3|21|1|2042|570|7421|Friday|2042Q1|N|Y|N|2466945|2467003|2466600|2466873|N|N|N|N|N| +2466966|AAAAAAAAGJEKFCAA|2042-03-22|1706|7421|570|2042|6|3|22|1|2042|570|7421|Saturday|2042Q1|N|Y|N|2466945|2467003|2466601|2466874|N|N|N|N|N| +2466967|AAAAAAAAHJEKFCAA|2042-03-23|1706|7421|570|2042|0|3|23|1|2042|570|7421|Sunday|2042Q1|N|N|N|2466945|2467003|2466602|2466875|N|N|N|N|N| +2466968|AAAAAAAAIJEKFCAA|2042-03-24|1706|7421|570|2042|1|3|24|1|2042|570|7421|Monday|2042Q1|N|N|N|2466945|2467003|2466603|2466876|N|N|N|N|N| +2466969|AAAAAAAAJJEKFCAA|2042-03-25|1706|7422|570|2042|2|3|25|1|2042|570|7422|Tuesday|2042Q1|N|N|N|2466945|2467003|2466604|2466877|N|N|N|N|N| +2466970|AAAAAAAAKJEKFCAA|2042-03-26|1706|7422|570|2042|3|3|26|1|2042|570|7422|Wednesday|2042Q1|N|N|N|2466945|2467003|2466605|2466878|N|N|N|N|N| +2466971|AAAAAAAALJEKFCAA|2042-03-27|1706|7422|570|2042|4|3|27|1|2042|570|7422|Thursday|2042Q1|N|N|N|2466945|2467003|2466606|2466879|N|N|N|N|N| +2466972|AAAAAAAAMJEKFCAA|2042-03-28|1706|7422|570|2042|5|3|28|1|2042|570|7422|Friday|2042Q1|N|Y|N|2466945|2467003|2466607|2466880|N|N|N|N|N| +2466973|AAAAAAAANJEKFCAA|2042-03-29|1706|7422|570|2042|6|3|29|1|2042|570|7422|Saturday|2042Q1|N|Y|N|2466945|2467003|2466608|2466881|N|N|N|N|N| +2466974|AAAAAAAAOJEKFCAA|2042-03-30|1706|7422|570|2042|0|3|30|1|2042|570|7422|Sunday|2042Q1|N|N|N|2466945|2467003|2466609|2466882|N|N|N|N|N| +2466975|AAAAAAAAPJEKFCAA|2042-03-31|1706|7422|570|2042|1|3|31|1|2042|570|7422|Monday|2042Q1|N|N|N|2466945|2467003|2466610|2466883|N|N|N|N|N| +2466976|AAAAAAAAAKEKFCAA|2042-04-01|1707|7423|570|2042|2|4|1|1|2042|570|7423|Tuesday|2042Q1|N|N|N|2466976|2467065|2466611|2466886|N|N|N|N|N| +2466977|AAAAAAAABKEKFCAA|2042-04-02|1707|7423|570|2042|3|4|2|2|2042|570|7423|Wednesday|2042Q2|N|N|N|2466976|2467065|2466612|2466887|N|N|N|N|N| +2466978|AAAAAAAACKEKFCAA|2042-04-03|1707|7423|570|2042|4|4|3|2|2042|570|7423|Thursday|2042Q2|N|N|N|2466976|2467065|2466613|2466888|N|N|N|N|N| +2466979|AAAAAAAADKEKFCAA|2042-04-04|1707|7423|570|2042|5|4|4|2|2042|570|7423|Friday|2042Q2|N|Y|N|2466976|2467065|2466614|2466889|N|N|N|N|N| +2466980|AAAAAAAAEKEKFCAA|2042-04-05|1707|7423|570|2042|6|4|5|2|2042|570|7423|Saturday|2042Q2|N|Y|N|2466976|2467065|2466615|2466890|N|N|N|N|N| +2466981|AAAAAAAAFKEKFCAA|2042-04-06|1707|7423|570|2042|0|4|6|2|2042|570|7423|Sunday|2042Q2|N|N|N|2466976|2467065|2466616|2466891|N|N|N|N|N| +2466982|AAAAAAAAGKEKFCAA|2042-04-07|1707|7423|570|2042|1|4|7|2|2042|570|7423|Monday|2042Q2|N|N|N|2466976|2467065|2466617|2466892|N|N|N|N|N| +2466983|AAAAAAAAHKEKFCAA|2042-04-08|1707|7424|570|2042|2|4|8|2|2042|570|7424|Tuesday|2042Q2|N|N|N|2466976|2467065|2466618|2466893|N|N|N|N|N| +2466984|AAAAAAAAIKEKFCAA|2042-04-09|1707|7424|570|2042|3|4|9|2|2042|570|7424|Wednesday|2042Q2|N|N|N|2466976|2467065|2466619|2466894|N|N|N|N|N| +2466985|AAAAAAAAJKEKFCAA|2042-04-10|1707|7424|570|2042|4|4|10|2|2042|570|7424|Thursday|2042Q2|N|N|N|2466976|2467065|2466620|2466895|N|N|N|N|N| +2466986|AAAAAAAAKKEKFCAA|2042-04-11|1707|7424|570|2042|5|4|11|2|2042|570|7424|Friday|2042Q2|N|Y|N|2466976|2467065|2466621|2466896|N|N|N|N|N| +2466987|AAAAAAAALKEKFCAA|2042-04-12|1707|7424|570|2042|6|4|12|2|2042|570|7424|Saturday|2042Q2|N|Y|N|2466976|2467065|2466622|2466897|N|N|N|N|N| +2466988|AAAAAAAAMKEKFCAA|2042-04-13|1707|7424|570|2042|0|4|13|2|2042|570|7424|Sunday|2042Q2|N|N|N|2466976|2467065|2466623|2466898|N|N|N|N|N| +2466989|AAAAAAAANKEKFCAA|2042-04-14|1707|7424|570|2042|1|4|14|2|2042|570|7424|Monday|2042Q2|N|N|N|2466976|2467065|2466624|2466899|N|N|N|N|N| +2466990|AAAAAAAAOKEKFCAA|2042-04-15|1707|7425|570|2042|2|4|15|2|2042|570|7425|Tuesday|2042Q2|N|N|N|2466976|2467065|2466625|2466900|N|N|N|N|N| +2466991|AAAAAAAAPKEKFCAA|2042-04-16|1707|7425|570|2042|3|4|16|2|2042|570|7425|Wednesday|2042Q2|N|N|N|2466976|2467065|2466626|2466901|N|N|N|N|N| +2466992|AAAAAAAAALEKFCAA|2042-04-17|1707|7425|570|2042|4|4|17|2|2042|570|7425|Thursday|2042Q2|N|N|N|2466976|2467065|2466627|2466902|N|N|N|N|N| +2466993|AAAAAAAABLEKFCAA|2042-04-18|1707|7425|570|2042|5|4|18|2|2042|570|7425|Friday|2042Q2|N|Y|N|2466976|2467065|2466628|2466903|N|N|N|N|N| +2466994|AAAAAAAACLEKFCAA|2042-04-19|1707|7425|570|2042|6|4|19|2|2042|570|7425|Saturday|2042Q2|N|Y|N|2466976|2467065|2466629|2466904|N|N|N|N|N| +2466995|AAAAAAAADLEKFCAA|2042-04-20|1707|7425|570|2042|0|4|20|2|2042|570|7425|Sunday|2042Q2|N|N|N|2466976|2467065|2466630|2466905|N|N|N|N|N| +2466996|AAAAAAAAELEKFCAA|2042-04-21|1707|7425|570|2042|1|4|21|2|2042|570|7425|Monday|2042Q2|N|N|N|2466976|2467065|2466631|2466906|N|N|N|N|N| +2466997|AAAAAAAAFLEKFCAA|2042-04-22|1707|7426|570|2042|2|4|22|2|2042|570|7426|Tuesday|2042Q2|N|N|N|2466976|2467065|2466632|2466907|N|N|N|N|N| +2466998|AAAAAAAAGLEKFCAA|2042-04-23|1707|7426|570|2042|3|4|23|2|2042|570|7426|Wednesday|2042Q2|N|N|N|2466976|2467065|2466633|2466908|N|N|N|N|N| +2466999|AAAAAAAAHLEKFCAA|2042-04-24|1707|7426|570|2042|4|4|24|2|2042|570|7426|Thursday|2042Q2|N|N|N|2466976|2467065|2466634|2466909|N|N|N|N|N| +2467000|AAAAAAAAILEKFCAA|2042-04-25|1707|7426|570|2042|5|4|25|2|2042|570|7426|Friday|2042Q2|N|Y|N|2466976|2467065|2466635|2466910|N|N|N|N|N| +2467001|AAAAAAAAJLEKFCAA|2042-04-26|1707|7426|570|2042|6|4|26|2|2042|570|7426|Saturday|2042Q2|N|Y|N|2466976|2467065|2466636|2466911|N|N|N|N|N| +2467002|AAAAAAAAKLEKFCAA|2042-04-27|1707|7426|570|2042|0|4|27|2|2042|570|7426|Sunday|2042Q2|N|N|N|2466976|2467065|2466637|2466912|N|N|N|N|N| +2467003|AAAAAAAALLEKFCAA|2042-04-28|1707|7426|570|2042|1|4|28|2|2042|570|7426|Monday|2042Q2|N|N|N|2466976|2467065|2466638|2466913|N|N|N|N|N| +2467004|AAAAAAAAMLEKFCAA|2042-04-29|1707|7427|570|2042|2|4|29|2|2042|570|7427|Tuesday|2042Q2|N|N|N|2466976|2467065|2466639|2466914|N|N|N|N|N| +2467005|AAAAAAAANLEKFCAA|2042-04-30|1707|7427|570|2042|3|4|30|2|2042|570|7427|Wednesday|2042Q2|N|N|N|2466976|2467065|2466640|2466915|N|N|N|N|N| +2467006|AAAAAAAAOLEKFCAA|2042-05-01|1708|7427|570|2042|4|5|1|2|2042|570|7427|Thursday|2042Q2|N|N|N|2467006|2467125|2466641|2466916|N|N|N|N|N| +2467007|AAAAAAAAPLEKFCAA|2042-05-02|1708|7427|570|2042|5|5|2|2|2042|570|7427|Friday|2042Q2|N|Y|N|2467006|2467125|2466642|2466917|N|N|N|N|N| +2467008|AAAAAAAAAMEKFCAA|2042-05-03|1708|7427|570|2042|6|5|3|2|2042|570|7427|Saturday|2042Q2|N|Y|N|2467006|2467125|2466643|2466918|N|N|N|N|N| +2467009|AAAAAAAABMEKFCAA|2042-05-04|1708|7427|570|2042|0|5|4|2|2042|570|7427|Sunday|2042Q2|N|N|N|2467006|2467125|2466644|2466919|N|N|N|N|N| +2467010|AAAAAAAACMEKFCAA|2042-05-05|1708|7427|570|2042|1|5|5|2|2042|570|7427|Monday|2042Q2|N|N|N|2467006|2467125|2466645|2466920|N|N|N|N|N| +2467011|AAAAAAAADMEKFCAA|2042-05-06|1708|7428|570|2042|2|5|6|2|2042|570|7428|Tuesday|2042Q2|N|N|N|2467006|2467125|2466646|2466921|N|N|N|N|N| +2467012|AAAAAAAAEMEKFCAA|2042-05-07|1708|7428|570|2042|3|5|7|2|2042|570|7428|Wednesday|2042Q2|N|N|N|2467006|2467125|2466647|2466922|N|N|N|N|N| +2467013|AAAAAAAAFMEKFCAA|2042-05-08|1708|7428|570|2042|4|5|8|2|2042|570|7428|Thursday|2042Q2|N|N|N|2467006|2467125|2466648|2466923|N|N|N|N|N| +2467014|AAAAAAAAGMEKFCAA|2042-05-09|1708|7428|570|2042|5|5|9|2|2042|570|7428|Friday|2042Q2|N|Y|N|2467006|2467125|2466649|2466924|N|N|N|N|N| +2467015|AAAAAAAAHMEKFCAA|2042-05-10|1708|7428|570|2042|6|5|10|2|2042|570|7428|Saturday|2042Q2|N|Y|N|2467006|2467125|2466650|2466925|N|N|N|N|N| +2467016|AAAAAAAAIMEKFCAA|2042-05-11|1708|7428|570|2042|0|5|11|2|2042|570|7428|Sunday|2042Q2|N|N|N|2467006|2467125|2466651|2466926|N|N|N|N|N| +2467017|AAAAAAAAJMEKFCAA|2042-05-12|1708|7428|570|2042|1|5|12|2|2042|570|7428|Monday|2042Q2|N|N|N|2467006|2467125|2466652|2466927|N|N|N|N|N| +2467018|AAAAAAAAKMEKFCAA|2042-05-13|1708|7429|570|2042|2|5|13|2|2042|570|7429|Tuesday|2042Q2|N|N|N|2467006|2467125|2466653|2466928|N|N|N|N|N| +2467019|AAAAAAAALMEKFCAA|2042-05-14|1708|7429|570|2042|3|5|14|2|2042|570|7429|Wednesday|2042Q2|N|N|N|2467006|2467125|2466654|2466929|N|N|N|N|N| +2467020|AAAAAAAAMMEKFCAA|2042-05-15|1708|7429|570|2042|4|5|15|2|2042|570|7429|Thursday|2042Q2|N|N|N|2467006|2467125|2466655|2466930|N|N|N|N|N| +2467021|AAAAAAAANMEKFCAA|2042-05-16|1708|7429|570|2042|5|5|16|2|2042|570|7429|Friday|2042Q2|N|Y|N|2467006|2467125|2466656|2466931|N|N|N|N|N| +2467022|AAAAAAAAOMEKFCAA|2042-05-17|1708|7429|570|2042|6|5|17|2|2042|570|7429|Saturday|2042Q2|N|Y|N|2467006|2467125|2466657|2466932|N|N|N|N|N| +2467023|AAAAAAAAPMEKFCAA|2042-05-18|1708|7429|570|2042|0|5|18|2|2042|570|7429|Sunday|2042Q2|N|N|N|2467006|2467125|2466658|2466933|N|N|N|N|N| +2467024|AAAAAAAAANEKFCAA|2042-05-19|1708|7429|570|2042|1|5|19|2|2042|570|7429|Monday|2042Q2|N|N|N|2467006|2467125|2466659|2466934|N|N|N|N|N| +2467025|AAAAAAAABNEKFCAA|2042-05-20|1708|7430|570|2042|2|5|20|2|2042|570|7430|Tuesday|2042Q2|N|N|N|2467006|2467125|2466660|2466935|N|N|N|N|N| +2467026|AAAAAAAACNEKFCAA|2042-05-21|1708|7430|570|2042|3|5|21|2|2042|570|7430|Wednesday|2042Q2|N|N|N|2467006|2467125|2466661|2466936|N|N|N|N|N| +2467027|AAAAAAAADNEKFCAA|2042-05-22|1708|7430|570|2042|4|5|22|2|2042|570|7430|Thursday|2042Q2|N|N|N|2467006|2467125|2466662|2466937|N|N|N|N|N| +2467028|AAAAAAAAENEKFCAA|2042-05-23|1708|7430|570|2042|5|5|23|2|2042|570|7430|Friday|2042Q2|N|Y|N|2467006|2467125|2466663|2466938|N|N|N|N|N| +2467029|AAAAAAAAFNEKFCAA|2042-05-24|1708|7430|570|2042|6|5|24|2|2042|570|7430|Saturday|2042Q2|N|Y|N|2467006|2467125|2466664|2466939|N|N|N|N|N| +2467030|AAAAAAAAGNEKFCAA|2042-05-25|1708|7430|570|2042|0|5|25|2|2042|570|7430|Sunday|2042Q2|N|N|N|2467006|2467125|2466665|2466940|N|N|N|N|N| +2467031|AAAAAAAAHNEKFCAA|2042-05-26|1708|7430|570|2042|1|5|26|2|2042|570|7430|Monday|2042Q2|N|N|N|2467006|2467125|2466666|2466941|N|N|N|N|N| +2467032|AAAAAAAAINEKFCAA|2042-05-27|1708|7431|570|2042|2|5|27|2|2042|570|7431|Tuesday|2042Q2|N|N|N|2467006|2467125|2466667|2466942|N|N|N|N|N| +2467033|AAAAAAAAJNEKFCAA|2042-05-28|1708|7431|570|2042|3|5|28|2|2042|570|7431|Wednesday|2042Q2|N|N|N|2467006|2467125|2466668|2466943|N|N|N|N|N| +2467034|AAAAAAAAKNEKFCAA|2042-05-29|1708|7431|570|2042|4|5|29|2|2042|570|7431|Thursday|2042Q2|N|N|N|2467006|2467125|2466669|2466944|N|N|N|N|N| +2467035|AAAAAAAALNEKFCAA|2042-05-30|1708|7431|570|2042|5|5|30|2|2042|570|7431|Friday|2042Q2|N|Y|N|2467006|2467125|2466670|2466945|N|N|N|N|N| +2467036|AAAAAAAAMNEKFCAA|2042-05-31|1708|7431|570|2042|6|5|31|2|2042|570|7431|Saturday|2042Q2|N|Y|N|2467006|2467125|2466671|2466946|N|N|N|N|N| +2467037|AAAAAAAANNEKFCAA|2042-06-01|1709|7431|571|2042|0|6|1|2|2042|571|7431|Sunday|2042Q2|N|N|N|2467037|2467187|2466672|2466947|N|N|N|N|N| +2467038|AAAAAAAAONEKFCAA|2042-06-02|1709|7431|571|2042|1|6|2|2|2042|571|7431|Monday|2042Q2|N|N|N|2467037|2467187|2466673|2466948|N|N|N|N|N| +2467039|AAAAAAAAPNEKFCAA|2042-06-03|1709|7432|571|2042|2|6|3|2|2042|571|7432|Tuesday|2042Q2|N|N|N|2467037|2467187|2466674|2466949|N|N|N|N|N| +2467040|AAAAAAAAAOEKFCAA|2042-06-04|1709|7432|571|2042|3|6|4|2|2042|571|7432|Wednesday|2042Q2|N|N|N|2467037|2467187|2466675|2466950|N|N|N|N|N| +2467041|AAAAAAAABOEKFCAA|2042-06-05|1709|7432|571|2042|4|6|5|2|2042|571|7432|Thursday|2042Q2|N|N|N|2467037|2467187|2466676|2466951|N|N|N|N|N| +2467042|AAAAAAAACOEKFCAA|2042-06-06|1709|7432|571|2042|5|6|6|2|2042|571|7432|Friday|2042Q2|N|Y|N|2467037|2467187|2466677|2466952|N|N|N|N|N| +2467043|AAAAAAAADOEKFCAA|2042-06-07|1709|7432|571|2042|6|6|7|2|2042|571|7432|Saturday|2042Q2|N|Y|N|2467037|2467187|2466678|2466953|N|N|N|N|N| +2467044|AAAAAAAAEOEKFCAA|2042-06-08|1709|7432|571|2042|0|6|8|2|2042|571|7432|Sunday|2042Q2|N|N|N|2467037|2467187|2466679|2466954|N|N|N|N|N| +2467045|AAAAAAAAFOEKFCAA|2042-06-09|1709|7432|571|2042|1|6|9|2|2042|571|7432|Monday|2042Q2|N|N|N|2467037|2467187|2466680|2466955|N|N|N|N|N| +2467046|AAAAAAAAGOEKFCAA|2042-06-10|1709|7433|571|2042|2|6|10|2|2042|571|7433|Tuesday|2042Q2|N|N|N|2467037|2467187|2466681|2466956|N|N|N|N|N| +2467047|AAAAAAAAHOEKFCAA|2042-06-11|1709|7433|571|2042|3|6|11|2|2042|571|7433|Wednesday|2042Q2|N|N|N|2467037|2467187|2466682|2466957|N|N|N|N|N| +2467048|AAAAAAAAIOEKFCAA|2042-06-12|1709|7433|571|2042|4|6|12|2|2042|571|7433|Thursday|2042Q2|N|N|N|2467037|2467187|2466683|2466958|N|N|N|N|N| +2467049|AAAAAAAAJOEKFCAA|2042-06-13|1709|7433|571|2042|5|6|13|2|2042|571|7433|Friday|2042Q2|N|Y|N|2467037|2467187|2466684|2466959|N|N|N|N|N| +2467050|AAAAAAAAKOEKFCAA|2042-06-14|1709|7433|571|2042|6|6|14|2|2042|571|7433|Saturday|2042Q2|N|Y|N|2467037|2467187|2466685|2466960|N|N|N|N|N| +2467051|AAAAAAAALOEKFCAA|2042-06-15|1709|7433|571|2042|0|6|15|2|2042|571|7433|Sunday|2042Q2|N|N|N|2467037|2467187|2466686|2466961|N|N|N|N|N| +2467052|AAAAAAAAMOEKFCAA|2042-06-16|1709|7433|571|2042|1|6|16|2|2042|571|7433|Monday|2042Q2|N|N|N|2467037|2467187|2466687|2466962|N|N|N|N|N| +2467053|AAAAAAAANOEKFCAA|2042-06-17|1709|7434|571|2042|2|6|17|2|2042|571|7434|Tuesday|2042Q2|N|N|N|2467037|2467187|2466688|2466963|N|N|N|N|N| +2467054|AAAAAAAAOOEKFCAA|2042-06-18|1709|7434|571|2042|3|6|18|2|2042|571|7434|Wednesday|2042Q2|N|N|N|2467037|2467187|2466689|2466964|N|N|N|N|N| +2467055|AAAAAAAAPOEKFCAA|2042-06-19|1709|7434|571|2042|4|6|19|2|2042|571|7434|Thursday|2042Q2|N|N|N|2467037|2467187|2466690|2466965|N|N|N|N|N| +2467056|AAAAAAAAAPEKFCAA|2042-06-20|1709|7434|571|2042|5|6|20|2|2042|571|7434|Friday|2042Q2|N|Y|N|2467037|2467187|2466691|2466966|N|N|N|N|N| +2467057|AAAAAAAABPEKFCAA|2042-06-21|1709|7434|571|2042|6|6|21|2|2042|571|7434|Saturday|2042Q2|N|Y|N|2467037|2467187|2466692|2466967|N|N|N|N|N| +2467058|AAAAAAAACPEKFCAA|2042-06-22|1709|7434|571|2042|0|6|22|2|2042|571|7434|Sunday|2042Q2|N|N|N|2467037|2467187|2466693|2466968|N|N|N|N|N| +2467059|AAAAAAAADPEKFCAA|2042-06-23|1709|7434|571|2042|1|6|23|2|2042|571|7434|Monday|2042Q2|N|N|N|2467037|2467187|2466694|2466969|N|N|N|N|N| +2467060|AAAAAAAAEPEKFCAA|2042-06-24|1709|7435|571|2042|2|6|24|2|2042|571|7435|Tuesday|2042Q2|N|N|N|2467037|2467187|2466695|2466970|N|N|N|N|N| +2467061|AAAAAAAAFPEKFCAA|2042-06-25|1709|7435|571|2042|3|6|25|2|2042|571|7435|Wednesday|2042Q2|N|N|N|2467037|2467187|2466696|2466971|N|N|N|N|N| +2467062|AAAAAAAAGPEKFCAA|2042-06-26|1709|7435|571|2042|4|6|26|2|2042|571|7435|Thursday|2042Q2|N|N|N|2467037|2467187|2466697|2466972|N|N|N|N|N| +2467063|AAAAAAAAHPEKFCAA|2042-06-27|1709|7435|571|2042|5|6|27|2|2042|571|7435|Friday|2042Q2|N|Y|N|2467037|2467187|2466698|2466973|N|N|N|N|N| +2467064|AAAAAAAAIPEKFCAA|2042-06-28|1709|7435|571|2042|6|6|28|2|2042|571|7435|Saturday|2042Q2|N|Y|N|2467037|2467187|2466699|2466974|N|N|N|N|N| +2467065|AAAAAAAAJPEKFCAA|2042-06-29|1709|7435|571|2042|0|6|29|2|2042|571|7435|Sunday|2042Q2|N|N|N|2467037|2467187|2466700|2466975|N|N|N|N|N| +2467066|AAAAAAAAKPEKFCAA|2042-06-30|1709|7435|571|2042|1|6|30|2|2042|571|7435|Monday|2042Q2|N|N|N|2467037|2467187|2466701|2466976|N|N|N|N|N| +2467067|AAAAAAAALPEKFCAA|2042-07-01|1710|7436|571|2042|2|7|1|2|2042|571|7436|Tuesday|2042Q2|N|N|N|2467067|2467247|2466702|2466976|N|N|N|N|N| +2467068|AAAAAAAAMPEKFCAA|2042-07-02|1710|7436|571|2042|3|7|2|3|2042|571|7436|Wednesday|2042Q3|N|N|N|2467067|2467247|2466703|2466977|N|N|N|N|N| +2467069|AAAAAAAANPEKFCAA|2042-07-03|1710|7436|571|2042|4|7|3|3|2042|571|7436|Thursday|2042Q3|N|N|N|2467067|2467247|2466704|2466978|N|N|N|N|N| +2467070|AAAAAAAAOPEKFCAA|2042-07-04|1710|7436|571|2042|5|7|4|3|2042|571|7436|Friday|2042Q3|N|Y|N|2467067|2467247|2466705|2466979|N|N|N|N|N| +2467071|AAAAAAAAPPEKFCAA|2042-07-05|1710|7436|571|2042|6|7|5|3|2042|571|7436|Saturday|2042Q3|Y|Y|N|2467067|2467247|2466706|2466980|N|N|N|N|N| +2467072|AAAAAAAAAAFKFCAA|2042-07-06|1710|7436|571|2042|0|7|6|3|2042|571|7436|Sunday|2042Q3|N|N|Y|2467067|2467247|2466707|2466981|N|N|N|N|N| +2467073|AAAAAAAABAFKFCAA|2042-07-07|1710|7436|571|2042|1|7|7|3|2042|571|7436|Monday|2042Q3|N|N|N|2467067|2467247|2466708|2466982|N|N|N|N|N| +2467074|AAAAAAAACAFKFCAA|2042-07-08|1710|7437|571|2042|2|7|8|3|2042|571|7437|Tuesday|2042Q3|N|N|N|2467067|2467247|2466709|2466983|N|N|N|N|N| +2467075|AAAAAAAADAFKFCAA|2042-07-09|1710|7437|571|2042|3|7|9|3|2042|571|7437|Wednesday|2042Q3|N|N|N|2467067|2467247|2466710|2466984|N|N|N|N|N| +2467076|AAAAAAAAEAFKFCAA|2042-07-10|1710|7437|571|2042|4|7|10|3|2042|571|7437|Thursday|2042Q3|N|N|N|2467067|2467247|2466711|2466985|N|N|N|N|N| +2467077|AAAAAAAAFAFKFCAA|2042-07-11|1710|7437|571|2042|5|7|11|3|2042|571|7437|Friday|2042Q3|N|Y|N|2467067|2467247|2466712|2466986|N|N|N|N|N| +2467078|AAAAAAAAGAFKFCAA|2042-07-12|1710|7437|571|2042|6|7|12|3|2042|571|7437|Saturday|2042Q3|N|Y|N|2467067|2467247|2466713|2466987|N|N|N|N|N| +2467079|AAAAAAAAHAFKFCAA|2042-07-13|1710|7437|571|2042|0|7|13|3|2042|571|7437|Sunday|2042Q3|N|N|N|2467067|2467247|2466714|2466988|N|N|N|N|N| +2467080|AAAAAAAAIAFKFCAA|2042-07-14|1710|7437|571|2042|1|7|14|3|2042|571|7437|Monday|2042Q3|N|N|N|2467067|2467247|2466715|2466989|N|N|N|N|N| +2467081|AAAAAAAAJAFKFCAA|2042-07-15|1710|7438|571|2042|2|7|15|3|2042|571|7438|Tuesday|2042Q3|N|N|N|2467067|2467247|2466716|2466990|N|N|N|N|N| +2467082|AAAAAAAAKAFKFCAA|2042-07-16|1710|7438|571|2042|3|7|16|3|2042|571|7438|Wednesday|2042Q3|N|N|N|2467067|2467247|2466717|2466991|N|N|N|N|N| +2467083|AAAAAAAALAFKFCAA|2042-07-17|1710|7438|571|2042|4|7|17|3|2042|571|7438|Thursday|2042Q3|N|N|N|2467067|2467247|2466718|2466992|N|N|N|N|N| +2467084|AAAAAAAAMAFKFCAA|2042-07-18|1710|7438|571|2042|5|7|18|3|2042|571|7438|Friday|2042Q3|N|Y|N|2467067|2467247|2466719|2466993|N|N|N|N|N| +2467085|AAAAAAAANAFKFCAA|2042-07-19|1710|7438|571|2042|6|7|19|3|2042|571|7438|Saturday|2042Q3|N|Y|N|2467067|2467247|2466720|2466994|N|N|N|N|N| +2467086|AAAAAAAAOAFKFCAA|2042-07-20|1710|7438|571|2042|0|7|20|3|2042|571|7438|Sunday|2042Q3|N|N|N|2467067|2467247|2466721|2466995|N|N|N|N|N| +2467087|AAAAAAAAPAFKFCAA|2042-07-21|1710|7438|571|2042|1|7|21|3|2042|571|7438|Monday|2042Q3|N|N|N|2467067|2467247|2466722|2466996|N|N|N|N|N| +2467088|AAAAAAAAABFKFCAA|2042-07-22|1710|7439|571|2042|2|7|22|3|2042|571|7439|Tuesday|2042Q3|N|N|N|2467067|2467247|2466723|2466997|N|N|N|N|N| +2467089|AAAAAAAABBFKFCAA|2042-07-23|1710|7439|571|2042|3|7|23|3|2042|571|7439|Wednesday|2042Q3|N|N|N|2467067|2467247|2466724|2466998|N|N|N|N|N| +2467090|AAAAAAAACBFKFCAA|2042-07-24|1710|7439|571|2042|4|7|24|3|2042|571|7439|Thursday|2042Q3|N|N|N|2467067|2467247|2466725|2466999|N|N|N|N|N| +2467091|AAAAAAAADBFKFCAA|2042-07-25|1710|7439|571|2042|5|7|25|3|2042|571|7439|Friday|2042Q3|N|Y|N|2467067|2467247|2466726|2467000|N|N|N|N|N| +2467092|AAAAAAAAEBFKFCAA|2042-07-26|1710|7439|571|2042|6|7|26|3|2042|571|7439|Saturday|2042Q3|N|Y|N|2467067|2467247|2466727|2467001|N|N|N|N|N| +2467093|AAAAAAAAFBFKFCAA|2042-07-27|1710|7439|571|2042|0|7|27|3|2042|571|7439|Sunday|2042Q3|N|N|N|2467067|2467247|2466728|2467002|N|N|N|N|N| +2467094|AAAAAAAAGBFKFCAA|2042-07-28|1710|7439|571|2042|1|7|28|3|2042|571|7439|Monday|2042Q3|N|N|N|2467067|2467247|2466729|2467003|N|N|N|N|N| +2467095|AAAAAAAAHBFKFCAA|2042-07-29|1710|7440|571|2042|2|7|29|3|2042|571|7440|Tuesday|2042Q3|N|N|N|2467067|2467247|2466730|2467004|N|N|N|N|N| +2467096|AAAAAAAAIBFKFCAA|2042-07-30|1710|7440|571|2042|3|7|30|3|2042|571|7440|Wednesday|2042Q3|N|N|N|2467067|2467247|2466731|2467005|N|N|N|N|N| +2467097|AAAAAAAAJBFKFCAA|2042-07-31|1710|7440|571|2042|4|7|31|3|2042|571|7440|Thursday|2042Q3|N|N|N|2467067|2467247|2466732|2467006|N|N|N|N|N| +2467098|AAAAAAAAKBFKFCAA|2042-08-01|1711|7440|571|2042|5|8|1|3|2042|571|7440|Friday|2042Q3|N|Y|N|2467098|2467309|2466733|2467007|N|N|N|N|N| +2467099|AAAAAAAALBFKFCAA|2042-08-02|1711|7440|571|2042|6|8|2|3|2042|571|7440|Saturday|2042Q3|N|Y|N|2467098|2467309|2466734|2467008|N|N|N|N|N| +2467100|AAAAAAAAMBFKFCAA|2042-08-03|1711|7440|571|2042|0|8|3|3|2042|571|7440|Sunday|2042Q3|N|N|N|2467098|2467309|2466735|2467009|N|N|N|N|N| +2467101|AAAAAAAANBFKFCAA|2042-08-04|1711|7440|571|2042|1|8|4|3|2042|571|7440|Monday|2042Q3|N|N|N|2467098|2467309|2466736|2467010|N|N|N|N|N| +2467102|AAAAAAAAOBFKFCAA|2042-08-05|1711|7441|571|2042|2|8|5|3|2042|571|7441|Tuesday|2042Q3|N|N|N|2467098|2467309|2466737|2467011|N|N|N|N|N| +2467103|AAAAAAAAPBFKFCAA|2042-08-06|1711|7441|571|2042|3|8|6|3|2042|571|7441|Wednesday|2042Q3|N|N|N|2467098|2467309|2466738|2467012|N|N|N|N|N| +2467104|AAAAAAAAACFKFCAA|2042-08-07|1711|7441|571|2042|4|8|7|3|2042|571|7441|Thursday|2042Q3|N|N|N|2467098|2467309|2466739|2467013|N|N|N|N|N| +2467105|AAAAAAAABCFKFCAA|2042-08-08|1711|7441|571|2042|5|8|8|3|2042|571|7441|Friday|2042Q3|N|Y|N|2467098|2467309|2466740|2467014|N|N|N|N|N| +2467106|AAAAAAAACCFKFCAA|2042-08-09|1711|7441|571|2042|6|8|9|3|2042|571|7441|Saturday|2042Q3|N|Y|N|2467098|2467309|2466741|2467015|N|N|N|N|N| +2467107|AAAAAAAADCFKFCAA|2042-08-10|1711|7441|571|2042|0|8|10|3|2042|571|7441|Sunday|2042Q3|N|N|N|2467098|2467309|2466742|2467016|N|N|N|N|N| +2467108|AAAAAAAAECFKFCAA|2042-08-11|1711|7441|571|2042|1|8|11|3|2042|571|7441|Monday|2042Q3|N|N|N|2467098|2467309|2466743|2467017|N|N|N|N|N| +2467109|AAAAAAAAFCFKFCAA|2042-08-12|1711|7442|571|2042|2|8|12|3|2042|571|7442|Tuesday|2042Q3|N|N|N|2467098|2467309|2466744|2467018|N|N|N|N|N| +2467110|AAAAAAAAGCFKFCAA|2042-08-13|1711|7442|571|2042|3|8|13|3|2042|571|7442|Wednesday|2042Q3|N|N|N|2467098|2467309|2466745|2467019|N|N|N|N|N| +2467111|AAAAAAAAHCFKFCAA|2042-08-14|1711|7442|571|2042|4|8|14|3|2042|571|7442|Thursday|2042Q3|N|N|N|2467098|2467309|2466746|2467020|N|N|N|N|N| +2467112|AAAAAAAAICFKFCAA|2042-08-15|1711|7442|571|2042|5|8|15|3|2042|571|7442|Friday|2042Q3|N|Y|N|2467098|2467309|2466747|2467021|N|N|N|N|N| +2467113|AAAAAAAAJCFKFCAA|2042-08-16|1711|7442|571|2042|6|8|16|3|2042|571|7442|Saturday|2042Q3|N|Y|N|2467098|2467309|2466748|2467022|N|N|N|N|N| +2467114|AAAAAAAAKCFKFCAA|2042-08-17|1711|7442|571|2042|0|8|17|3|2042|571|7442|Sunday|2042Q3|N|N|N|2467098|2467309|2466749|2467023|N|N|N|N|N| +2467115|AAAAAAAALCFKFCAA|2042-08-18|1711|7442|571|2042|1|8|18|3|2042|571|7442|Monday|2042Q3|N|N|N|2467098|2467309|2466750|2467024|N|N|N|N|N| +2467116|AAAAAAAAMCFKFCAA|2042-08-19|1711|7443|571|2042|2|8|19|3|2042|571|7443|Tuesday|2042Q3|N|N|N|2467098|2467309|2466751|2467025|N|N|N|N|N| +2467117|AAAAAAAANCFKFCAA|2042-08-20|1711|7443|571|2042|3|8|20|3|2042|571|7443|Wednesday|2042Q3|N|N|N|2467098|2467309|2466752|2467026|N|N|N|N|N| +2467118|AAAAAAAAOCFKFCAA|2042-08-21|1711|7443|571|2042|4|8|21|3|2042|571|7443|Thursday|2042Q3|N|N|N|2467098|2467309|2466753|2467027|N|N|N|N|N| +2467119|AAAAAAAAPCFKFCAA|2042-08-22|1711|7443|571|2042|5|8|22|3|2042|571|7443|Friday|2042Q3|N|Y|N|2467098|2467309|2466754|2467028|N|N|N|N|N| +2467120|AAAAAAAAADFKFCAA|2042-08-23|1711|7443|571|2042|6|8|23|3|2042|571|7443|Saturday|2042Q3|N|Y|N|2467098|2467309|2466755|2467029|N|N|N|N|N| +2467121|AAAAAAAABDFKFCAA|2042-08-24|1711|7443|571|2042|0|8|24|3|2042|571|7443|Sunday|2042Q3|N|N|N|2467098|2467309|2466756|2467030|N|N|N|N|N| +2467122|AAAAAAAACDFKFCAA|2042-08-25|1711|7443|571|2042|1|8|25|3|2042|571|7443|Monday|2042Q3|N|N|N|2467098|2467309|2466757|2467031|N|N|N|N|N| +2467123|AAAAAAAADDFKFCAA|2042-08-26|1711|7444|571|2042|2|8|26|3|2042|571|7444|Tuesday|2042Q3|N|N|N|2467098|2467309|2466758|2467032|N|N|N|N|N| +2467124|AAAAAAAAEDFKFCAA|2042-08-27|1711|7444|571|2042|3|8|27|3|2042|571|7444|Wednesday|2042Q3|N|N|N|2467098|2467309|2466759|2467033|N|N|N|N|N| +2467125|AAAAAAAAFDFKFCAA|2042-08-28|1711|7444|571|2042|4|8|28|3|2042|571|7444|Thursday|2042Q3|N|N|N|2467098|2467309|2466760|2467034|N|N|N|N|N| +2467126|AAAAAAAAGDFKFCAA|2042-08-29|1711|7444|571|2042|5|8|29|3|2042|571|7444|Friday|2042Q3|N|Y|N|2467098|2467309|2466761|2467035|N|N|N|N|N| +2467127|AAAAAAAAHDFKFCAA|2042-08-30|1711|7444|571|2042|6|8|30|3|2042|571|7444|Saturday|2042Q3|N|Y|N|2467098|2467309|2466762|2467036|N|N|N|N|N| +2467128|AAAAAAAAIDFKFCAA|2042-08-31|1711|7444|571|2042|0|8|31|3|2042|571|7444|Sunday|2042Q3|N|N|N|2467098|2467309|2466763|2467037|N|N|N|N|N| +2467129|AAAAAAAAJDFKFCAA|2042-09-01|1712|7444|572|2042|1|9|1|3|2042|572|7444|Monday|2042Q3|N|N|N|2467129|2467371|2466764|2467038|N|N|N|N|N| +2467130|AAAAAAAAKDFKFCAA|2042-09-02|1712|7445|572|2042|2|9|2|3|2042|572|7445|Tuesday|2042Q3|N|N|N|2467129|2467371|2466765|2467039|N|N|N|N|N| +2467131|AAAAAAAALDFKFCAA|2042-09-03|1712|7445|572|2042|3|9|3|3|2042|572|7445|Wednesday|2042Q3|N|N|N|2467129|2467371|2466766|2467040|N|N|N|N|N| +2467132|AAAAAAAAMDFKFCAA|2042-09-04|1712|7445|572|2042|4|9|4|3|2042|572|7445|Thursday|2042Q3|N|N|N|2467129|2467371|2466767|2467041|N|N|N|N|N| +2467133|AAAAAAAANDFKFCAA|2042-09-05|1712|7445|572|2042|5|9|5|3|2042|572|7445|Friday|2042Q3|N|Y|N|2467129|2467371|2466768|2467042|N|N|N|N|N| +2467134|AAAAAAAAODFKFCAA|2042-09-06|1712|7445|572|2042|6|9|6|3|2042|572|7445|Saturday|2042Q3|N|Y|N|2467129|2467371|2466769|2467043|N|N|N|N|N| +2467135|AAAAAAAAPDFKFCAA|2042-09-07|1712|7445|572|2042|0|9|7|3|2042|572|7445|Sunday|2042Q3|N|N|N|2467129|2467371|2466770|2467044|N|N|N|N|N| +2467136|AAAAAAAAAEFKFCAA|2042-09-08|1712|7445|572|2042|1|9|8|3|2042|572|7445|Monday|2042Q3|N|N|N|2467129|2467371|2466771|2467045|N|N|N|N|N| +2467137|AAAAAAAABEFKFCAA|2042-09-09|1712|7446|572|2042|2|9|9|3|2042|572|7446|Tuesday|2042Q3|N|N|N|2467129|2467371|2466772|2467046|N|N|N|N|N| +2467138|AAAAAAAACEFKFCAA|2042-09-10|1712|7446|572|2042|3|9|10|3|2042|572|7446|Wednesday|2042Q3|N|N|N|2467129|2467371|2466773|2467047|N|N|N|N|N| +2467139|AAAAAAAADEFKFCAA|2042-09-11|1712|7446|572|2042|4|9|11|3|2042|572|7446|Thursday|2042Q3|N|N|N|2467129|2467371|2466774|2467048|N|N|N|N|N| +2467140|AAAAAAAAEEFKFCAA|2042-09-12|1712|7446|572|2042|5|9|12|3|2042|572|7446|Friday|2042Q3|N|Y|N|2467129|2467371|2466775|2467049|N|N|N|N|N| +2467141|AAAAAAAAFEFKFCAA|2042-09-13|1712|7446|572|2042|6|9|13|3|2042|572|7446|Saturday|2042Q3|N|Y|N|2467129|2467371|2466776|2467050|N|N|N|N|N| +2467142|AAAAAAAAGEFKFCAA|2042-09-14|1712|7446|572|2042|0|9|14|3|2042|572|7446|Sunday|2042Q3|N|N|N|2467129|2467371|2466777|2467051|N|N|N|N|N| +2467143|AAAAAAAAHEFKFCAA|2042-09-15|1712|7446|572|2042|1|9|15|3|2042|572|7446|Monday|2042Q3|N|N|N|2467129|2467371|2466778|2467052|N|N|N|N|N| +2467144|AAAAAAAAIEFKFCAA|2042-09-16|1712|7447|572|2042|2|9|16|3|2042|572|7447|Tuesday|2042Q3|N|N|N|2467129|2467371|2466779|2467053|N|N|N|N|N| +2467145|AAAAAAAAJEFKFCAA|2042-09-17|1712|7447|572|2042|3|9|17|3|2042|572|7447|Wednesday|2042Q3|N|N|N|2467129|2467371|2466780|2467054|N|N|N|N|N| +2467146|AAAAAAAAKEFKFCAA|2042-09-18|1712|7447|572|2042|4|9|18|3|2042|572|7447|Thursday|2042Q3|N|N|N|2467129|2467371|2466781|2467055|N|N|N|N|N| +2467147|AAAAAAAALEFKFCAA|2042-09-19|1712|7447|572|2042|5|9|19|3|2042|572|7447|Friday|2042Q3|N|Y|N|2467129|2467371|2466782|2467056|N|N|N|N|N| +2467148|AAAAAAAAMEFKFCAA|2042-09-20|1712|7447|572|2042|6|9|20|3|2042|572|7447|Saturday|2042Q3|N|Y|N|2467129|2467371|2466783|2467057|N|N|N|N|N| +2467149|AAAAAAAANEFKFCAA|2042-09-21|1712|7447|572|2042|0|9|21|3|2042|572|7447|Sunday|2042Q3|N|N|N|2467129|2467371|2466784|2467058|N|N|N|N|N| +2467150|AAAAAAAAOEFKFCAA|2042-09-22|1712|7447|572|2042|1|9|22|3|2042|572|7447|Monday|2042Q3|N|N|N|2467129|2467371|2466785|2467059|N|N|N|N|N| +2467151|AAAAAAAAPEFKFCAA|2042-09-23|1712|7448|572|2042|2|9|23|3|2042|572|7448|Tuesday|2042Q3|N|N|N|2467129|2467371|2466786|2467060|N|N|N|N|N| +2467152|AAAAAAAAAFFKFCAA|2042-09-24|1712|7448|572|2042|3|9|24|3|2042|572|7448|Wednesday|2042Q3|N|N|N|2467129|2467371|2466787|2467061|N|N|N|N|N| +2467153|AAAAAAAABFFKFCAA|2042-09-25|1712|7448|572|2042|4|9|25|3|2042|572|7448|Thursday|2042Q3|N|N|N|2467129|2467371|2466788|2467062|N|N|N|N|N| +2467154|AAAAAAAACFFKFCAA|2042-09-26|1712|7448|572|2042|5|9|26|3|2042|572|7448|Friday|2042Q3|N|Y|N|2467129|2467371|2466789|2467063|N|N|N|N|N| +2467155|AAAAAAAADFFKFCAA|2042-09-27|1712|7448|572|2042|6|9|27|3|2042|572|7448|Saturday|2042Q3|N|Y|N|2467129|2467371|2466790|2467064|N|N|N|N|N| +2467156|AAAAAAAAEFFKFCAA|2042-09-28|1712|7448|572|2042|0|9|28|3|2042|572|7448|Sunday|2042Q3|N|N|N|2467129|2467371|2466791|2467065|N|N|N|N|N| +2467157|AAAAAAAAFFFKFCAA|2042-09-29|1712|7448|572|2042|1|9|29|3|2042|572|7448|Monday|2042Q3|N|N|N|2467129|2467371|2466792|2467066|N|N|N|N|N| +2467158|AAAAAAAAGFFKFCAA|2042-09-30|1712|7449|572|2042|2|9|30|3|2042|572|7449|Tuesday|2042Q3|N|N|N|2467129|2467371|2466793|2467067|N|N|N|N|N| +2467159|AAAAAAAAHFFKFCAA|2042-10-01|1713|7449|572|2042|3|10|1|3|2042|572|7449|Wednesday|2042Q3|N|N|N|2467159|2467431|2466794|2467067|N|N|N|N|N| +2467160|AAAAAAAAIFFKFCAA|2042-10-02|1713|7449|572|2042|4|10|2|4|2042|572|7449|Thursday|2042Q4|N|N|N|2467159|2467431|2466795|2467068|N|N|N|N|N| +2467161|AAAAAAAAJFFKFCAA|2042-10-03|1713|7449|572|2042|5|10|3|4|2042|572|7449|Friday|2042Q4|N|Y|N|2467159|2467431|2466796|2467069|N|N|N|N|N| +2467162|AAAAAAAAKFFKFCAA|2042-10-04|1713|7449|572|2042|6|10|4|4|2042|572|7449|Saturday|2042Q4|N|Y|N|2467159|2467431|2466797|2467070|N|N|N|N|N| +2467163|AAAAAAAALFFKFCAA|2042-10-05|1713|7449|572|2042|0|10|5|4|2042|572|7449|Sunday|2042Q4|N|N|N|2467159|2467431|2466798|2467071|N|N|N|N|N| +2467164|AAAAAAAAMFFKFCAA|2042-10-06|1713|7449|572|2042|1|10|6|4|2042|572|7449|Monday|2042Q4|N|N|N|2467159|2467431|2466799|2467072|N|N|N|N|N| +2467165|AAAAAAAANFFKFCAA|2042-10-07|1713|7450|572|2042|2|10|7|4|2042|572|7450|Tuesday|2042Q4|N|N|N|2467159|2467431|2466800|2467073|N|N|N|N|N| +2467166|AAAAAAAAOFFKFCAA|2042-10-08|1713|7450|572|2042|3|10|8|4|2042|572|7450|Wednesday|2042Q4|N|N|N|2467159|2467431|2466801|2467074|N|N|N|N|N| +2467167|AAAAAAAAPFFKFCAA|2042-10-09|1713|7450|572|2042|4|10|9|4|2042|572|7450|Thursday|2042Q4|N|N|N|2467159|2467431|2466802|2467075|N|N|N|N|N| +2467168|AAAAAAAAAGFKFCAA|2042-10-10|1713|7450|572|2042|5|10|10|4|2042|572|7450|Friday|2042Q4|N|Y|N|2467159|2467431|2466803|2467076|N|N|N|N|N| +2467169|AAAAAAAABGFKFCAA|2042-10-11|1713|7450|572|2042|6|10|11|4|2042|572|7450|Saturday|2042Q4|N|Y|N|2467159|2467431|2466804|2467077|N|N|N|N|N| +2467170|AAAAAAAACGFKFCAA|2042-10-12|1713|7450|572|2042|0|10|12|4|2042|572|7450|Sunday|2042Q4|N|N|N|2467159|2467431|2466805|2467078|N|N|N|N|N| +2467171|AAAAAAAADGFKFCAA|2042-10-13|1713|7450|572|2042|1|10|13|4|2042|572|7450|Monday|2042Q4|N|N|N|2467159|2467431|2466806|2467079|N|N|N|N|N| +2467172|AAAAAAAAEGFKFCAA|2042-10-14|1713|7451|572|2042|2|10|14|4|2042|572|7451|Tuesday|2042Q4|N|N|N|2467159|2467431|2466807|2467080|N|N|N|N|N| +2467173|AAAAAAAAFGFKFCAA|2042-10-15|1713|7451|572|2042|3|10|15|4|2042|572|7451|Wednesday|2042Q4|N|N|N|2467159|2467431|2466808|2467081|N|N|N|N|N| +2467174|AAAAAAAAGGFKFCAA|2042-10-16|1713|7451|572|2042|4|10|16|4|2042|572|7451|Thursday|2042Q4|N|N|N|2467159|2467431|2466809|2467082|N|N|N|N|N| +2467175|AAAAAAAAHGFKFCAA|2042-10-17|1713|7451|572|2042|5|10|17|4|2042|572|7451|Friday|2042Q4|N|Y|N|2467159|2467431|2466810|2467083|N|N|N|N|N| +2467176|AAAAAAAAIGFKFCAA|2042-10-18|1713|7451|572|2042|6|10|18|4|2042|572|7451|Saturday|2042Q4|N|Y|N|2467159|2467431|2466811|2467084|N|N|N|N|N| +2467177|AAAAAAAAJGFKFCAA|2042-10-19|1713|7451|572|2042|0|10|19|4|2042|572|7451|Sunday|2042Q4|N|N|N|2467159|2467431|2466812|2467085|N|N|N|N|N| +2467178|AAAAAAAAKGFKFCAA|2042-10-20|1713|7451|572|2042|1|10|20|4|2042|572|7451|Monday|2042Q4|N|N|N|2467159|2467431|2466813|2467086|N|N|N|N|N| +2467179|AAAAAAAALGFKFCAA|2042-10-21|1713|7452|572|2042|2|10|21|4|2042|572|7452|Tuesday|2042Q4|N|N|N|2467159|2467431|2466814|2467087|N|N|N|N|N| +2467180|AAAAAAAAMGFKFCAA|2042-10-22|1713|7452|572|2042|3|10|22|4|2042|572|7452|Wednesday|2042Q4|N|N|N|2467159|2467431|2466815|2467088|N|N|N|N|N| +2467181|AAAAAAAANGFKFCAA|2042-10-23|1713|7452|572|2042|4|10|23|4|2042|572|7452|Thursday|2042Q4|N|N|N|2467159|2467431|2466816|2467089|N|N|N|N|N| +2467182|AAAAAAAAOGFKFCAA|2042-10-24|1713|7452|572|2042|5|10|24|4|2042|572|7452|Friday|2042Q4|N|Y|N|2467159|2467431|2466817|2467090|N|N|N|N|N| +2467183|AAAAAAAAPGFKFCAA|2042-10-25|1713|7452|572|2042|6|10|25|4|2042|572|7452|Saturday|2042Q4|N|Y|N|2467159|2467431|2466818|2467091|N|N|N|N|N| +2467184|AAAAAAAAAHFKFCAA|2042-10-26|1713|7452|572|2042|0|10|26|4|2042|572|7452|Sunday|2042Q4|N|N|N|2467159|2467431|2466819|2467092|N|N|N|N|N| +2467185|AAAAAAAABHFKFCAA|2042-10-27|1713|7452|572|2042|1|10|27|4|2042|572|7452|Monday|2042Q4|N|N|N|2467159|2467431|2466820|2467093|N|N|N|N|N| +2467186|AAAAAAAACHFKFCAA|2042-10-28|1713|7453|572|2042|2|10|28|4|2042|572|7453|Tuesday|2042Q4|N|N|N|2467159|2467431|2466821|2467094|N|N|N|N|N| +2467187|AAAAAAAADHFKFCAA|2042-10-29|1713|7453|572|2042|3|10|29|4|2042|572|7453|Wednesday|2042Q4|N|N|N|2467159|2467431|2466822|2467095|N|N|N|N|N| +2467188|AAAAAAAAEHFKFCAA|2042-10-30|1713|7453|572|2042|4|10|30|4|2042|572|7453|Thursday|2042Q4|N|N|N|2467159|2467431|2466823|2467096|N|N|N|N|N| +2467189|AAAAAAAAFHFKFCAA|2042-10-31|1713|7453|572|2042|5|10|31|4|2042|572|7453|Friday|2042Q4|N|Y|N|2467159|2467431|2466824|2467097|N|N|N|N|N| +2467190|AAAAAAAAGHFKFCAA|2042-11-01|1714|7453|572|2042|6|11|1|4|2042|572|7453|Saturday|2042Q4|N|Y|N|2467190|2467493|2466825|2467098|N|N|N|N|N| +2467191|AAAAAAAAHHFKFCAA|2042-11-02|1714|7453|572|2042|0|11|2|4|2042|572|7453|Sunday|2042Q4|N|N|N|2467190|2467493|2466826|2467099|N|N|N|N|N| +2467192|AAAAAAAAIHFKFCAA|2042-11-03|1714|7453|572|2042|1|11|3|4|2042|572|7453|Monday|2042Q4|N|N|N|2467190|2467493|2466827|2467100|N|N|N|N|N| +2467193|AAAAAAAAJHFKFCAA|2042-11-04|1714|7454|572|2042|2|11|4|4|2042|572|7454|Tuesday|2042Q4|N|N|N|2467190|2467493|2466828|2467101|N|N|N|N|N| +2467194|AAAAAAAAKHFKFCAA|2042-11-05|1714|7454|572|2042|3|11|5|4|2042|572|7454|Wednesday|2042Q4|N|N|N|2467190|2467493|2466829|2467102|N|N|N|N|N| +2467195|AAAAAAAALHFKFCAA|2042-11-06|1714|7454|572|2042|4|11|6|4|2042|572|7454|Thursday|2042Q4|N|N|N|2467190|2467493|2466830|2467103|N|N|N|N|N| +2467196|AAAAAAAAMHFKFCAA|2042-11-07|1714|7454|572|2042|5|11|7|4|2042|572|7454|Friday|2042Q4|N|Y|N|2467190|2467493|2466831|2467104|N|N|N|N|N| +2467197|AAAAAAAANHFKFCAA|2042-11-08|1714|7454|572|2042|6|11|8|4|2042|572|7454|Saturday|2042Q4|N|Y|N|2467190|2467493|2466832|2467105|N|N|N|N|N| +2467198|AAAAAAAAOHFKFCAA|2042-11-09|1714|7454|572|2042|0|11|9|4|2042|572|7454|Sunday|2042Q4|N|N|N|2467190|2467493|2466833|2467106|N|N|N|N|N| +2467199|AAAAAAAAPHFKFCAA|2042-11-10|1714|7454|572|2042|1|11|10|4|2042|572|7454|Monday|2042Q4|N|N|N|2467190|2467493|2466834|2467107|N|N|N|N|N| +2467200|AAAAAAAAAIFKFCAA|2042-11-11|1714|7455|572|2042|2|11|11|4|2042|572|7455|Tuesday|2042Q4|N|N|N|2467190|2467493|2466835|2467108|N|N|N|N|N| +2467201|AAAAAAAABIFKFCAA|2042-11-12|1714|7455|572|2042|3|11|12|4|2042|572|7455|Wednesday|2042Q4|N|N|N|2467190|2467493|2466836|2467109|N|N|N|N|N| +2467202|AAAAAAAACIFKFCAA|2042-11-13|1714|7455|572|2042|4|11|13|4|2042|572|7455|Thursday|2042Q4|N|N|N|2467190|2467493|2466837|2467110|N|N|N|N|N| +2467203|AAAAAAAADIFKFCAA|2042-11-14|1714|7455|572|2042|5|11|14|4|2042|572|7455|Friday|2042Q4|N|Y|N|2467190|2467493|2466838|2467111|N|N|N|N|N| +2467204|AAAAAAAAEIFKFCAA|2042-11-15|1714|7455|572|2042|6|11|15|4|2042|572|7455|Saturday|2042Q4|N|Y|N|2467190|2467493|2466839|2467112|N|N|N|N|N| +2467205|AAAAAAAAFIFKFCAA|2042-11-16|1714|7455|572|2042|0|11|16|4|2042|572|7455|Sunday|2042Q4|N|N|N|2467190|2467493|2466840|2467113|N|N|N|N|N| +2467206|AAAAAAAAGIFKFCAA|2042-11-17|1714|7455|572|2042|1|11|17|4|2042|572|7455|Monday|2042Q4|N|N|N|2467190|2467493|2466841|2467114|N|N|N|N|N| +2467207|AAAAAAAAHIFKFCAA|2042-11-18|1714|7456|572|2042|2|11|18|4|2042|572|7456|Tuesday|2042Q4|N|N|N|2467190|2467493|2466842|2467115|N|N|N|N|N| +2467208|AAAAAAAAIIFKFCAA|2042-11-19|1714|7456|572|2042|3|11|19|4|2042|572|7456|Wednesday|2042Q4|N|N|N|2467190|2467493|2466843|2467116|N|N|N|N|N| +2467209|AAAAAAAAJIFKFCAA|2042-11-20|1714|7456|572|2042|4|11|20|4|2042|572|7456|Thursday|2042Q4|N|N|N|2467190|2467493|2466844|2467117|N|N|N|N|N| +2467210|AAAAAAAAKIFKFCAA|2042-11-21|1714|7456|572|2042|5|11|21|4|2042|572|7456|Friday|2042Q4|N|Y|N|2467190|2467493|2466845|2467118|N|N|N|N|N| +2467211|AAAAAAAALIFKFCAA|2042-11-22|1714|7456|572|2042|6|11|22|4|2042|572|7456|Saturday|2042Q4|N|Y|N|2467190|2467493|2466846|2467119|N|N|N|N|N| +2467212|AAAAAAAAMIFKFCAA|2042-11-23|1714|7456|572|2042|0|11|23|4|2042|572|7456|Sunday|2042Q4|N|N|N|2467190|2467493|2466847|2467120|N|N|N|N|N| +2467213|AAAAAAAANIFKFCAA|2042-11-24|1714|7456|572|2042|1|11|24|4|2042|572|7456|Monday|2042Q4|N|N|N|2467190|2467493|2466848|2467121|N|N|N|N|N| +2467214|AAAAAAAAOIFKFCAA|2042-11-25|1714|7457|572|2042|2|11|25|4|2042|572|7457|Tuesday|2042Q4|N|N|N|2467190|2467493|2466849|2467122|N|N|N|N|N| +2467215|AAAAAAAAPIFKFCAA|2042-11-26|1714|7457|572|2042|3|11|26|4|2042|572|7457|Wednesday|2042Q4|N|N|N|2467190|2467493|2466850|2467123|N|N|N|N|N| +2467216|AAAAAAAAAJFKFCAA|2042-11-27|1714|7457|572|2042|4|11|27|4|2042|572|7457|Thursday|2042Q4|N|N|N|2467190|2467493|2466851|2467124|N|N|N|N|N| +2467217|AAAAAAAABJFKFCAA|2042-11-28|1714|7457|572|2042|5|11|28|4|2042|572|7457|Friday|2042Q4|N|Y|N|2467190|2467493|2466852|2467125|N|N|N|N|N| +2467218|AAAAAAAACJFKFCAA|2042-11-29|1714|7457|572|2042|6|11|29|4|2042|572|7457|Saturday|2042Q4|N|Y|N|2467190|2467493|2466853|2467126|N|N|N|N|N| +2467219|AAAAAAAADJFKFCAA|2042-11-30|1714|7457|572|2042|0|11|30|4|2042|572|7457|Sunday|2042Q4|N|N|N|2467190|2467493|2466854|2467127|N|N|N|N|N| +2467220|AAAAAAAAEJFKFCAA|2042-12-01|1715|7457|573|2042|1|12|1|4|2042|573|7457|Monday|2042Q4|N|N|N|2467220|2467553|2466855|2467128|N|N|N|N|N| +2467221|AAAAAAAAFJFKFCAA|2042-12-02|1715|7458|573|2042|2|12|2|4|2042|573|7458|Tuesday|2042Q4|N|N|N|2467220|2467553|2466856|2467129|N|N|N|N|N| +2467222|AAAAAAAAGJFKFCAA|2042-12-03|1715|7458|573|2042|3|12|3|4|2042|573|7458|Wednesday|2042Q4|N|N|N|2467220|2467553|2466857|2467130|N|N|N|N|N| +2467223|AAAAAAAAHJFKFCAA|2042-12-04|1715|7458|573|2042|4|12|4|4|2042|573|7458|Thursday|2042Q4|N|N|N|2467220|2467553|2466858|2467131|N|N|N|N|N| +2467224|AAAAAAAAIJFKFCAA|2042-12-05|1715|7458|573|2042|5|12|5|4|2042|573|7458|Friday|2042Q4|N|Y|N|2467220|2467553|2466859|2467132|N|N|N|N|N| +2467225|AAAAAAAAJJFKFCAA|2042-12-06|1715|7458|573|2042|6|12|6|4|2042|573|7458|Saturday|2042Q4|N|Y|N|2467220|2467553|2466860|2467133|N|N|N|N|N| +2467226|AAAAAAAAKJFKFCAA|2042-12-07|1715|7458|573|2042|0|12|7|4|2042|573|7458|Sunday|2042Q4|N|N|N|2467220|2467553|2466861|2467134|N|N|N|N|N| +2467227|AAAAAAAALJFKFCAA|2042-12-08|1715|7458|573|2042|1|12|8|4|2042|573|7458|Monday|2042Q4|N|N|N|2467220|2467553|2466862|2467135|N|N|N|N|N| +2467228|AAAAAAAAMJFKFCAA|2042-12-09|1715|7459|573|2042|2|12|9|4|2042|573|7459|Tuesday|2042Q4|N|N|N|2467220|2467553|2466863|2467136|N|N|N|N|N| +2467229|AAAAAAAANJFKFCAA|2042-12-10|1715|7459|573|2042|3|12|10|4|2042|573|7459|Wednesday|2042Q4|N|N|N|2467220|2467553|2466864|2467137|N|N|N|N|N| +2467230|AAAAAAAAOJFKFCAA|2042-12-11|1715|7459|573|2042|4|12|11|4|2042|573|7459|Thursday|2042Q4|N|N|N|2467220|2467553|2466865|2467138|N|N|N|N|N| +2467231|AAAAAAAAPJFKFCAA|2042-12-12|1715|7459|573|2042|5|12|12|4|2042|573|7459|Friday|2042Q4|N|Y|N|2467220|2467553|2466866|2467139|N|N|N|N|N| +2467232|AAAAAAAAAKFKFCAA|2042-12-13|1715|7459|573|2042|6|12|13|4|2042|573|7459|Saturday|2042Q4|N|Y|N|2467220|2467553|2466867|2467140|N|N|N|N|N| +2467233|AAAAAAAABKFKFCAA|2042-12-14|1715|7459|573|2042|0|12|14|4|2042|573|7459|Sunday|2042Q4|N|N|N|2467220|2467553|2466868|2467141|N|N|N|N|N| +2467234|AAAAAAAACKFKFCAA|2042-12-15|1715|7459|573|2042|1|12|15|4|2042|573|7459|Monday|2042Q4|N|N|N|2467220|2467553|2466869|2467142|N|N|N|N|N| +2467235|AAAAAAAADKFKFCAA|2042-12-16|1715|7460|573|2042|2|12|16|4|2042|573|7460|Tuesday|2042Q4|N|N|N|2467220|2467553|2466870|2467143|N|N|N|N|N| +2467236|AAAAAAAAEKFKFCAA|2042-12-17|1715|7460|573|2042|3|12|17|4|2042|573|7460|Wednesday|2042Q4|N|N|N|2467220|2467553|2466871|2467144|N|N|N|N|N| +2467237|AAAAAAAAFKFKFCAA|2042-12-18|1715|7460|573|2042|4|12|18|4|2042|573|7460|Thursday|2042Q4|N|N|N|2467220|2467553|2466872|2467145|N|N|N|N|N| +2467238|AAAAAAAAGKFKFCAA|2042-12-19|1715|7460|573|2042|5|12|19|4|2042|573|7460|Friday|2042Q4|N|Y|N|2467220|2467553|2466873|2467146|N|N|N|N|N| +2467239|AAAAAAAAHKFKFCAA|2042-12-20|1715|7460|573|2042|6|12|20|4|2042|573|7460|Saturday|2042Q4|N|Y|N|2467220|2467553|2466874|2467147|N|N|N|N|N| +2467240|AAAAAAAAIKFKFCAA|2042-12-21|1715|7460|573|2042|0|12|21|4|2042|573|7460|Sunday|2042Q4|N|N|N|2467220|2467553|2466875|2467148|N|N|N|N|N| +2467241|AAAAAAAAJKFKFCAA|2042-12-22|1715|7460|573|2042|1|12|22|4|2042|573|7460|Monday|2042Q4|N|N|N|2467220|2467553|2466876|2467149|N|N|N|N|N| +2467242|AAAAAAAAKKFKFCAA|2042-12-23|1715|7461|573|2042|2|12|23|4|2042|573|7461|Tuesday|2042Q4|N|N|N|2467220|2467553|2466877|2467150|N|N|N|N|N| +2467243|AAAAAAAALKFKFCAA|2042-12-24|1715|7461|573|2042|3|12|24|4|2042|573|7461|Wednesday|2042Q4|N|N|N|2467220|2467553|2466878|2467151|N|N|N|N|N| +2467244|AAAAAAAAMKFKFCAA|2042-12-25|1715|7461|573|2042|4|12|25|4|2042|573|7461|Thursday|2042Q4|N|N|N|2467220|2467553|2466879|2467152|N|N|N|N|N| +2467245|AAAAAAAANKFKFCAA|2042-12-26|1715|7461|573|2042|5|12|26|4|2042|573|7461|Friday|2042Q4|Y|Y|N|2467220|2467553|2466880|2467153|N|N|N|N|N| +2467246|AAAAAAAAOKFKFCAA|2042-12-27|1715|7461|573|2042|6|12|27|4|2042|573|7461|Saturday|2042Q4|N|Y|Y|2467220|2467553|2466881|2467154|N|N|N|N|N| +2467247|AAAAAAAAPKFKFCAA|2042-12-28|1715|7461|573|2042|0|12|28|4|2042|573|7461|Sunday|2042Q4|N|N|N|2467220|2467553|2466882|2467155|N|N|N|N|N| +2467248|AAAAAAAAALFKFCAA|2042-12-29|1715|7461|573|2042|1|12|29|4|2042|573|7461|Monday|2042Q4|N|N|N|2467220|2467553|2466883|2467156|N|N|N|N|N| +2467249|AAAAAAAABLFKFCAA|2042-12-30|1715|7462|573|2042|2|12|30|4|2042|573|7462|Tuesday|2042Q4|N|N|N|2467220|2467553|2466884|2467157|N|N|N|N|N| +2467250|AAAAAAAACLFKFCAA|2042-12-31|1715|7462|573|2042|3|12|31|4|2042|573|7462|Wednesday|2042Q4|N|N|N|2467220|2467553|2466885|2467158|N|N|N|N|N| +2467251|AAAAAAAADLFKFCAA|2043-01-01|1716|7462|573|2043|4|1|1|1|2043|573|7462|Thursday|2043Q1|Y|N|N|2467251|2467250|2466886|2467159|N|N|N|N|N| +2467252|AAAAAAAAELFKFCAA|2043-01-02|1716|7462|573|2043|5|1|2|1|2043|573|7462|Friday|2043Q1|N|Y|Y|2467251|2467250|2466887|2467160|N|N|N|N|N| +2467253|AAAAAAAAFLFKFCAA|2043-01-03|1716|7462|573|2043|6|1|3|1|2043|573|7462|Saturday|2043Q1|N|Y|N|2467251|2467250|2466888|2467161|N|N|N|N|N| +2467254|AAAAAAAAGLFKFCAA|2043-01-04|1716|7462|573|2043|0|1|4|1|2043|573|7462|Sunday|2043Q1|N|N|N|2467251|2467250|2466889|2467162|N|N|N|N|N| +2467255|AAAAAAAAHLFKFCAA|2043-01-05|1716|7462|573|2043|1|1|5|1|2043|573|7462|Monday|2043Q1|N|N|N|2467251|2467250|2466890|2467163|N|N|N|N|N| +2467256|AAAAAAAAILFKFCAA|2043-01-06|1716|7463|573|2043|2|1|6|1|2043|573|7463|Tuesday|2043Q1|N|N|N|2467251|2467250|2466891|2467164|N|N|N|N|N| +2467257|AAAAAAAAJLFKFCAA|2043-01-07|1716|7463|573|2043|3|1|7|1|2043|573|7463|Wednesday|2043Q1|N|N|N|2467251|2467250|2466892|2467165|N|N|N|N|N| +2467258|AAAAAAAAKLFKFCAA|2043-01-08|1716|7463|573|2043|4|1|8|1|2043|573|7463|Thursday|2043Q1|N|N|N|2467251|2467250|2466893|2467166|N|N|N|N|N| +2467259|AAAAAAAALLFKFCAA|2043-01-09|1716|7463|573|2043|5|1|9|1|2043|573|7463|Friday|2043Q1|N|Y|N|2467251|2467250|2466894|2467167|N|N|N|N|N| +2467260|AAAAAAAAMLFKFCAA|2043-01-10|1716|7463|573|2043|6|1|10|1|2043|573|7463|Saturday|2043Q1|N|Y|N|2467251|2467250|2466895|2467168|N|N|N|N|N| +2467261|AAAAAAAANLFKFCAA|2043-01-11|1716|7463|573|2043|0|1|11|1|2043|573|7463|Sunday|2043Q1|N|N|N|2467251|2467250|2466896|2467169|N|N|N|N|N| +2467262|AAAAAAAAOLFKFCAA|2043-01-12|1716|7463|573|2043|1|1|12|1|2043|573|7463|Monday|2043Q1|N|N|N|2467251|2467250|2466897|2467170|N|N|N|N|N| +2467263|AAAAAAAAPLFKFCAA|2043-01-13|1716|7464|573|2043|2|1|13|1|2043|573|7464|Tuesday|2043Q1|N|N|N|2467251|2467250|2466898|2467171|N|N|N|N|N| +2467264|AAAAAAAAAMFKFCAA|2043-01-14|1716|7464|573|2043|3|1|14|1|2043|573|7464|Wednesday|2043Q1|N|N|N|2467251|2467250|2466899|2467172|N|N|N|N|N| +2467265|AAAAAAAABMFKFCAA|2043-01-15|1716|7464|573|2043|4|1|15|1|2043|573|7464|Thursday|2043Q1|N|N|N|2467251|2467250|2466900|2467173|N|N|N|N|N| +2467266|AAAAAAAACMFKFCAA|2043-01-16|1716|7464|573|2043|5|1|16|1|2043|573|7464|Friday|2043Q1|N|Y|N|2467251|2467250|2466901|2467174|N|N|N|N|N| +2467267|AAAAAAAADMFKFCAA|2043-01-17|1716|7464|573|2043|6|1|17|1|2043|573|7464|Saturday|2043Q1|N|Y|N|2467251|2467250|2466902|2467175|N|N|N|N|N| +2467268|AAAAAAAAEMFKFCAA|2043-01-18|1716|7464|573|2043|0|1|18|1|2043|573|7464|Sunday|2043Q1|N|N|N|2467251|2467250|2466903|2467176|N|N|N|N|N| +2467269|AAAAAAAAFMFKFCAA|2043-01-19|1716|7464|573|2043|1|1|19|1|2043|573|7464|Monday|2043Q1|N|N|N|2467251|2467250|2466904|2467177|N|N|N|N|N| +2467270|AAAAAAAAGMFKFCAA|2043-01-20|1716|7465|573|2043|2|1|20|1|2043|573|7465|Tuesday|2043Q1|N|N|N|2467251|2467250|2466905|2467178|N|N|N|N|N| +2467271|AAAAAAAAHMFKFCAA|2043-01-21|1716|7465|573|2043|3|1|21|1|2043|573|7465|Wednesday|2043Q1|N|N|N|2467251|2467250|2466906|2467179|N|N|N|N|N| +2467272|AAAAAAAAIMFKFCAA|2043-01-22|1716|7465|573|2043|4|1|22|1|2043|573|7465|Thursday|2043Q1|N|N|N|2467251|2467250|2466907|2467180|N|N|N|N|N| +2467273|AAAAAAAAJMFKFCAA|2043-01-23|1716|7465|573|2043|5|1|23|1|2043|573|7465|Friday|2043Q1|N|Y|N|2467251|2467250|2466908|2467181|N|N|N|N|N| +2467274|AAAAAAAAKMFKFCAA|2043-01-24|1716|7465|573|2043|6|1|24|1|2043|573|7465|Saturday|2043Q1|N|Y|N|2467251|2467250|2466909|2467182|N|N|N|N|N| +2467275|AAAAAAAALMFKFCAA|2043-01-25|1716|7465|573|2043|0|1|25|1|2043|573|7465|Sunday|2043Q1|N|N|N|2467251|2467250|2466910|2467183|N|N|N|N|N| +2467276|AAAAAAAAMMFKFCAA|2043-01-26|1716|7465|573|2043|1|1|26|1|2043|573|7465|Monday|2043Q1|N|N|N|2467251|2467250|2466911|2467184|N|N|N|N|N| +2467277|AAAAAAAANMFKFCAA|2043-01-27|1716|7466|573|2043|2|1|27|1|2043|573|7466|Tuesday|2043Q1|N|N|N|2467251|2467250|2466912|2467185|N|N|N|N|N| +2467278|AAAAAAAAOMFKFCAA|2043-01-28|1716|7466|573|2043|3|1|28|1|2043|573|7466|Wednesday|2043Q1|N|N|N|2467251|2467250|2466913|2467186|N|N|N|N|N| +2467279|AAAAAAAAPMFKFCAA|2043-01-29|1716|7466|573|2043|4|1|29|1|2043|573|7466|Thursday|2043Q1|N|N|N|2467251|2467250|2466914|2467187|N|N|N|N|N| +2467280|AAAAAAAAANFKFCAA|2043-01-30|1716|7466|573|2043|5|1|30|1|2043|573|7466|Friday|2043Q1|N|Y|N|2467251|2467250|2466915|2467188|N|N|N|N|N| +2467281|AAAAAAAABNFKFCAA|2043-01-31|1716|7466|573|2043|6|1|31|1|2043|573|7466|Saturday|2043Q1|N|Y|N|2467251|2467250|2466916|2467189|N|N|N|N|N| +2467282|AAAAAAAACNFKFCAA|2043-02-01|1717|7466|573|2043|0|2|1|1|2043|573|7466|Sunday|2043Q1|N|N|N|2467282|2467312|2466917|2467190|N|N|N|N|N| +2467283|AAAAAAAADNFKFCAA|2043-02-02|1717|7466|573|2043|1|2|2|1|2043|573|7466|Monday|2043Q1|N|N|N|2467282|2467312|2466918|2467191|N|N|N|N|N| +2467284|AAAAAAAAENFKFCAA|2043-02-03|1717|7467|573|2043|2|2|3|1|2043|573|7467|Tuesday|2043Q1|N|N|N|2467282|2467312|2466919|2467192|N|N|N|N|N| +2467285|AAAAAAAAFNFKFCAA|2043-02-04|1717|7467|573|2043|3|2|4|1|2043|573|7467|Wednesday|2043Q1|N|N|N|2467282|2467312|2466920|2467193|N|N|N|N|N| +2467286|AAAAAAAAGNFKFCAA|2043-02-05|1717|7467|573|2043|4|2|5|1|2043|573|7467|Thursday|2043Q1|N|N|N|2467282|2467312|2466921|2467194|N|N|N|N|N| +2467287|AAAAAAAAHNFKFCAA|2043-02-06|1717|7467|573|2043|5|2|6|1|2043|573|7467|Friday|2043Q1|N|Y|N|2467282|2467312|2466922|2467195|N|N|N|N|N| +2467288|AAAAAAAAINFKFCAA|2043-02-07|1717|7467|573|2043|6|2|7|1|2043|573|7467|Saturday|2043Q1|N|Y|N|2467282|2467312|2466923|2467196|N|N|N|N|N| +2467289|AAAAAAAAJNFKFCAA|2043-02-08|1717|7467|573|2043|0|2|8|1|2043|573|7467|Sunday|2043Q1|N|N|N|2467282|2467312|2466924|2467197|N|N|N|N|N| +2467290|AAAAAAAAKNFKFCAA|2043-02-09|1717|7467|573|2043|1|2|9|1|2043|573|7467|Monday|2043Q1|N|N|N|2467282|2467312|2466925|2467198|N|N|N|N|N| +2467291|AAAAAAAALNFKFCAA|2043-02-10|1717|7468|573|2043|2|2|10|1|2043|573|7468|Tuesday|2043Q1|N|N|N|2467282|2467312|2466926|2467199|N|N|N|N|N| +2467292|AAAAAAAAMNFKFCAA|2043-02-11|1717|7468|573|2043|3|2|11|1|2043|573|7468|Wednesday|2043Q1|N|N|N|2467282|2467312|2466927|2467200|N|N|N|N|N| +2467293|AAAAAAAANNFKFCAA|2043-02-12|1717|7468|573|2043|4|2|12|1|2043|573|7468|Thursday|2043Q1|N|N|N|2467282|2467312|2466928|2467201|N|N|N|N|N| +2467294|AAAAAAAAONFKFCAA|2043-02-13|1717|7468|573|2043|5|2|13|1|2043|573|7468|Friday|2043Q1|N|Y|N|2467282|2467312|2466929|2467202|N|N|N|N|N| +2467295|AAAAAAAAPNFKFCAA|2043-02-14|1717|7468|573|2043|6|2|14|1|2043|573|7468|Saturday|2043Q1|N|Y|N|2467282|2467312|2466930|2467203|N|N|N|N|N| +2467296|AAAAAAAAAOFKFCAA|2043-02-15|1717|7468|573|2043|0|2|15|1|2043|573|7468|Sunday|2043Q1|N|N|N|2467282|2467312|2466931|2467204|N|N|N|N|N| +2467297|AAAAAAAABOFKFCAA|2043-02-16|1717|7468|573|2043|1|2|16|1|2043|573|7468|Monday|2043Q1|N|N|N|2467282|2467312|2466932|2467205|N|N|N|N|N| +2467298|AAAAAAAACOFKFCAA|2043-02-17|1717|7469|573|2043|2|2|17|1|2043|573|7469|Tuesday|2043Q1|N|N|N|2467282|2467312|2466933|2467206|N|N|N|N|N| +2467299|AAAAAAAADOFKFCAA|2043-02-18|1717|7469|573|2043|3|2|18|1|2043|573|7469|Wednesday|2043Q1|N|N|N|2467282|2467312|2466934|2467207|N|N|N|N|N| +2467300|AAAAAAAAEOFKFCAA|2043-02-19|1717|7469|573|2043|4|2|19|1|2043|573|7469|Thursday|2043Q1|N|N|N|2467282|2467312|2466935|2467208|N|N|N|N|N| +2467301|AAAAAAAAFOFKFCAA|2043-02-20|1717|7469|573|2043|5|2|20|1|2043|573|7469|Friday|2043Q1|N|Y|N|2467282|2467312|2466936|2467209|N|N|N|N|N| +2467302|AAAAAAAAGOFKFCAA|2043-02-21|1717|7469|573|2043|6|2|21|1|2043|573|7469|Saturday|2043Q1|N|Y|N|2467282|2467312|2466937|2467210|N|N|N|N|N| +2467303|AAAAAAAAHOFKFCAA|2043-02-22|1717|7469|573|2043|0|2|22|1|2043|573|7469|Sunday|2043Q1|N|N|N|2467282|2467312|2466938|2467211|N|N|N|N|N| +2467304|AAAAAAAAIOFKFCAA|2043-02-23|1717|7469|573|2043|1|2|23|1|2043|573|7469|Monday|2043Q1|N|N|N|2467282|2467312|2466939|2467212|N|N|N|N|N| +2467305|AAAAAAAAJOFKFCAA|2043-02-24|1717|7470|573|2043|2|2|24|1|2043|573|7470|Tuesday|2043Q1|N|N|N|2467282|2467312|2466940|2467213|N|N|N|N|N| +2467306|AAAAAAAAKOFKFCAA|2043-02-25|1717|7470|573|2043|3|2|25|1|2043|573|7470|Wednesday|2043Q1|N|N|N|2467282|2467312|2466941|2467214|N|N|N|N|N| +2467307|AAAAAAAALOFKFCAA|2043-02-26|1717|7470|573|2043|4|2|26|1|2043|573|7470|Thursday|2043Q1|N|N|N|2467282|2467312|2466942|2467215|N|N|N|N|N| +2467308|AAAAAAAAMOFKFCAA|2043-02-27|1717|7470|573|2043|5|2|27|1|2043|573|7470|Friday|2043Q1|N|Y|N|2467282|2467312|2466943|2467216|N|N|N|N|N| +2467309|AAAAAAAANOFKFCAA|2043-02-28|1717|7470|573|2043|6|2|28|1|2043|573|7470|Saturday|2043Q1|N|Y|N|2467282|2467312|2466944|2467217|N|N|N|N|N| +2467310|AAAAAAAAOOFKFCAA|2043-03-01|1718|7470|574|2043|0|3|1|1|2043|574|7470|Sunday|2043Q1|N|N|N|2467310|2467368|2466945|2467218|N|N|N|N|N| +2467311|AAAAAAAAPOFKFCAA|2043-03-02|1718|7470|574|2043|1|3|2|1|2043|574|7470|Monday|2043Q1|N|N|N|2467310|2467368|2466946|2467219|N|N|N|N|N| +2467312|AAAAAAAAAPFKFCAA|2043-03-03|1718|7471|574|2043|2|3|3|1|2043|574|7471|Tuesday|2043Q1|N|N|N|2467310|2467368|2466947|2467220|N|N|N|N|N| +2467313|AAAAAAAABPFKFCAA|2043-03-04|1718|7471|574|2043|3|3|4|1|2043|574|7471|Wednesday|2043Q1|N|N|N|2467310|2467368|2466948|2467221|N|N|N|N|N| +2467314|AAAAAAAACPFKFCAA|2043-03-05|1718|7471|574|2043|4|3|5|1|2043|574|7471|Thursday|2043Q1|N|N|N|2467310|2467368|2466949|2467222|N|N|N|N|N| +2467315|AAAAAAAADPFKFCAA|2043-03-06|1718|7471|574|2043|5|3|6|1|2043|574|7471|Friday|2043Q1|N|Y|N|2467310|2467368|2466950|2467223|N|N|N|N|N| +2467316|AAAAAAAAEPFKFCAA|2043-03-07|1718|7471|574|2043|6|3|7|1|2043|574|7471|Saturday|2043Q1|N|Y|N|2467310|2467368|2466951|2467224|N|N|N|N|N| +2467317|AAAAAAAAFPFKFCAA|2043-03-08|1718|7471|574|2043|0|3|8|1|2043|574|7471|Sunday|2043Q1|N|N|N|2467310|2467368|2466952|2467225|N|N|N|N|N| +2467318|AAAAAAAAGPFKFCAA|2043-03-09|1718|7471|574|2043|1|3|9|1|2043|574|7471|Monday|2043Q1|N|N|N|2467310|2467368|2466953|2467226|N|N|N|N|N| +2467319|AAAAAAAAHPFKFCAA|2043-03-10|1718|7472|574|2043|2|3|10|1|2043|574|7472|Tuesday|2043Q1|N|N|N|2467310|2467368|2466954|2467227|N|N|N|N|N| +2467320|AAAAAAAAIPFKFCAA|2043-03-11|1718|7472|574|2043|3|3|11|1|2043|574|7472|Wednesday|2043Q1|N|N|N|2467310|2467368|2466955|2467228|N|N|N|N|N| +2467321|AAAAAAAAJPFKFCAA|2043-03-12|1718|7472|574|2043|4|3|12|1|2043|574|7472|Thursday|2043Q1|N|N|N|2467310|2467368|2466956|2467229|N|N|N|N|N| +2467322|AAAAAAAAKPFKFCAA|2043-03-13|1718|7472|574|2043|5|3|13|1|2043|574|7472|Friday|2043Q1|N|Y|N|2467310|2467368|2466957|2467230|N|N|N|N|N| +2467323|AAAAAAAALPFKFCAA|2043-03-14|1718|7472|574|2043|6|3|14|1|2043|574|7472|Saturday|2043Q1|N|Y|N|2467310|2467368|2466958|2467231|N|N|N|N|N| +2467324|AAAAAAAAMPFKFCAA|2043-03-15|1718|7472|574|2043|0|3|15|1|2043|574|7472|Sunday|2043Q1|N|N|N|2467310|2467368|2466959|2467232|N|N|N|N|N| +2467325|AAAAAAAANPFKFCAA|2043-03-16|1718|7472|574|2043|1|3|16|1|2043|574|7472|Monday|2043Q1|N|N|N|2467310|2467368|2466960|2467233|N|N|N|N|N| +2467326|AAAAAAAAOPFKFCAA|2043-03-17|1718|7473|574|2043|2|3|17|1|2043|574|7473|Tuesday|2043Q1|N|N|N|2467310|2467368|2466961|2467234|N|N|N|N|N| +2467327|AAAAAAAAPPFKFCAA|2043-03-18|1718|7473|574|2043|3|3|18|1|2043|574|7473|Wednesday|2043Q1|N|N|N|2467310|2467368|2466962|2467235|N|N|N|N|N| +2467328|AAAAAAAAAAGKFCAA|2043-03-19|1718|7473|574|2043|4|3|19|1|2043|574|7473|Thursday|2043Q1|N|N|N|2467310|2467368|2466963|2467236|N|N|N|N|N| +2467329|AAAAAAAABAGKFCAA|2043-03-20|1718|7473|574|2043|5|3|20|1|2043|574|7473|Friday|2043Q1|N|Y|N|2467310|2467368|2466964|2467237|N|N|N|N|N| +2467330|AAAAAAAACAGKFCAA|2043-03-21|1718|7473|574|2043|6|3|21|1|2043|574|7473|Saturday|2043Q1|N|Y|N|2467310|2467368|2466965|2467238|N|N|N|N|N| +2467331|AAAAAAAADAGKFCAA|2043-03-22|1718|7473|574|2043|0|3|22|1|2043|574|7473|Sunday|2043Q1|N|N|N|2467310|2467368|2466966|2467239|N|N|N|N|N| +2467332|AAAAAAAAEAGKFCAA|2043-03-23|1718|7473|574|2043|1|3|23|1|2043|574|7473|Monday|2043Q1|N|N|N|2467310|2467368|2466967|2467240|N|N|N|N|N| +2467333|AAAAAAAAFAGKFCAA|2043-03-24|1718|7474|574|2043|2|3|24|1|2043|574|7474|Tuesday|2043Q1|N|N|N|2467310|2467368|2466968|2467241|N|N|N|N|N| +2467334|AAAAAAAAGAGKFCAA|2043-03-25|1718|7474|574|2043|3|3|25|1|2043|574|7474|Wednesday|2043Q1|N|N|N|2467310|2467368|2466969|2467242|N|N|N|N|N| +2467335|AAAAAAAAHAGKFCAA|2043-03-26|1718|7474|574|2043|4|3|26|1|2043|574|7474|Thursday|2043Q1|N|N|N|2467310|2467368|2466970|2467243|N|N|N|N|N| +2467336|AAAAAAAAIAGKFCAA|2043-03-27|1718|7474|574|2043|5|3|27|1|2043|574|7474|Friday|2043Q1|N|Y|N|2467310|2467368|2466971|2467244|N|N|N|N|N| +2467337|AAAAAAAAJAGKFCAA|2043-03-28|1718|7474|574|2043|6|3|28|1|2043|574|7474|Saturday|2043Q1|N|Y|N|2467310|2467368|2466972|2467245|N|N|N|N|N| +2467338|AAAAAAAAKAGKFCAA|2043-03-29|1718|7474|574|2043|0|3|29|1|2043|574|7474|Sunday|2043Q1|N|N|N|2467310|2467368|2466973|2467246|N|N|N|N|N| +2467339|AAAAAAAALAGKFCAA|2043-03-30|1718|7474|574|2043|1|3|30|1|2043|574|7474|Monday|2043Q1|N|N|N|2467310|2467368|2466974|2467247|N|N|N|N|N| +2467340|AAAAAAAAMAGKFCAA|2043-03-31|1718|7475|574|2043|2|3|31|1|2043|574|7475|Tuesday|2043Q1|N|N|N|2467310|2467368|2466975|2467248|N|N|N|N|N| +2467341|AAAAAAAANAGKFCAA|2043-04-01|1719|7475|574|2043|3|4|1|1|2043|574|7475|Wednesday|2043Q1|N|N|N|2467341|2467430|2466976|2467251|N|N|N|N|N| +2467342|AAAAAAAAOAGKFCAA|2043-04-02|1719|7475|574|2043|4|4|2|2|2043|574|7475|Thursday|2043Q2|N|N|N|2467341|2467430|2466977|2467252|N|N|N|N|N| +2467343|AAAAAAAAPAGKFCAA|2043-04-03|1719|7475|574|2043|5|4|3|2|2043|574|7475|Friday|2043Q2|N|Y|N|2467341|2467430|2466978|2467253|N|N|N|N|N| +2467344|AAAAAAAAABGKFCAA|2043-04-04|1719|7475|574|2043|6|4|4|2|2043|574|7475|Saturday|2043Q2|N|Y|N|2467341|2467430|2466979|2467254|N|N|N|N|N| +2467345|AAAAAAAABBGKFCAA|2043-04-05|1719|7475|574|2043|0|4|5|2|2043|574|7475|Sunday|2043Q2|N|N|N|2467341|2467430|2466980|2467255|N|N|N|N|N| +2467346|AAAAAAAACBGKFCAA|2043-04-06|1719|7475|574|2043|1|4|6|2|2043|574|7475|Monday|2043Q2|N|N|N|2467341|2467430|2466981|2467256|N|N|N|N|N| +2467347|AAAAAAAADBGKFCAA|2043-04-07|1719|7476|574|2043|2|4|7|2|2043|574|7476|Tuesday|2043Q2|N|N|N|2467341|2467430|2466982|2467257|N|N|N|N|N| +2467348|AAAAAAAAEBGKFCAA|2043-04-08|1719|7476|574|2043|3|4|8|2|2043|574|7476|Wednesday|2043Q2|N|N|N|2467341|2467430|2466983|2467258|N|N|N|N|N| +2467349|AAAAAAAAFBGKFCAA|2043-04-09|1719|7476|574|2043|4|4|9|2|2043|574|7476|Thursday|2043Q2|N|N|N|2467341|2467430|2466984|2467259|N|N|N|N|N| +2467350|AAAAAAAAGBGKFCAA|2043-04-10|1719|7476|574|2043|5|4|10|2|2043|574|7476|Friday|2043Q2|N|Y|N|2467341|2467430|2466985|2467260|N|N|N|N|N| +2467351|AAAAAAAAHBGKFCAA|2043-04-11|1719|7476|574|2043|6|4|11|2|2043|574|7476|Saturday|2043Q2|N|Y|N|2467341|2467430|2466986|2467261|N|N|N|N|N| +2467352|AAAAAAAAIBGKFCAA|2043-04-12|1719|7476|574|2043|0|4|12|2|2043|574|7476|Sunday|2043Q2|N|N|N|2467341|2467430|2466987|2467262|N|N|N|N|N| +2467353|AAAAAAAAJBGKFCAA|2043-04-13|1719|7476|574|2043|1|4|13|2|2043|574|7476|Monday|2043Q2|N|N|N|2467341|2467430|2466988|2467263|N|N|N|N|N| +2467354|AAAAAAAAKBGKFCAA|2043-04-14|1719|7477|574|2043|2|4|14|2|2043|574|7477|Tuesday|2043Q2|N|N|N|2467341|2467430|2466989|2467264|N|N|N|N|N| +2467355|AAAAAAAALBGKFCAA|2043-04-15|1719|7477|574|2043|3|4|15|2|2043|574|7477|Wednesday|2043Q2|N|N|N|2467341|2467430|2466990|2467265|N|N|N|N|N| +2467356|AAAAAAAAMBGKFCAA|2043-04-16|1719|7477|574|2043|4|4|16|2|2043|574|7477|Thursday|2043Q2|N|N|N|2467341|2467430|2466991|2467266|N|N|N|N|N| +2467357|AAAAAAAANBGKFCAA|2043-04-17|1719|7477|574|2043|5|4|17|2|2043|574|7477|Friday|2043Q2|N|Y|N|2467341|2467430|2466992|2467267|N|N|N|N|N| +2467358|AAAAAAAAOBGKFCAA|2043-04-18|1719|7477|574|2043|6|4|18|2|2043|574|7477|Saturday|2043Q2|N|Y|N|2467341|2467430|2466993|2467268|N|N|N|N|N| +2467359|AAAAAAAAPBGKFCAA|2043-04-19|1719|7477|574|2043|0|4|19|2|2043|574|7477|Sunday|2043Q2|N|N|N|2467341|2467430|2466994|2467269|N|N|N|N|N| +2467360|AAAAAAAAACGKFCAA|2043-04-20|1719|7477|574|2043|1|4|20|2|2043|574|7477|Monday|2043Q2|N|N|N|2467341|2467430|2466995|2467270|N|N|N|N|N| +2467361|AAAAAAAABCGKFCAA|2043-04-21|1719|7478|574|2043|2|4|21|2|2043|574|7478|Tuesday|2043Q2|N|N|N|2467341|2467430|2466996|2467271|N|N|N|N|N| +2467362|AAAAAAAACCGKFCAA|2043-04-22|1719|7478|574|2043|3|4|22|2|2043|574|7478|Wednesday|2043Q2|N|N|N|2467341|2467430|2466997|2467272|N|N|N|N|N| +2467363|AAAAAAAADCGKFCAA|2043-04-23|1719|7478|574|2043|4|4|23|2|2043|574|7478|Thursday|2043Q2|N|N|N|2467341|2467430|2466998|2467273|N|N|N|N|N| +2467364|AAAAAAAAECGKFCAA|2043-04-24|1719|7478|574|2043|5|4|24|2|2043|574|7478|Friday|2043Q2|N|Y|N|2467341|2467430|2466999|2467274|N|N|N|N|N| +2467365|AAAAAAAAFCGKFCAA|2043-04-25|1719|7478|574|2043|6|4|25|2|2043|574|7478|Saturday|2043Q2|N|Y|N|2467341|2467430|2467000|2467275|N|N|N|N|N| +2467366|AAAAAAAAGCGKFCAA|2043-04-26|1719|7478|574|2043|0|4|26|2|2043|574|7478|Sunday|2043Q2|N|N|N|2467341|2467430|2467001|2467276|N|N|N|N|N| +2467367|AAAAAAAAHCGKFCAA|2043-04-27|1719|7478|574|2043|1|4|27|2|2043|574|7478|Monday|2043Q2|N|N|N|2467341|2467430|2467002|2467277|N|N|N|N|N| +2467368|AAAAAAAAICGKFCAA|2043-04-28|1719|7479|574|2043|2|4|28|2|2043|574|7479|Tuesday|2043Q2|N|N|N|2467341|2467430|2467003|2467278|N|N|N|N|N| +2467369|AAAAAAAAJCGKFCAA|2043-04-29|1719|7479|574|2043|3|4|29|2|2043|574|7479|Wednesday|2043Q2|N|N|N|2467341|2467430|2467004|2467279|N|N|N|N|N| +2467370|AAAAAAAAKCGKFCAA|2043-04-30|1719|7479|574|2043|4|4|30|2|2043|574|7479|Thursday|2043Q2|N|N|N|2467341|2467430|2467005|2467280|N|N|N|N|N| +2467371|AAAAAAAALCGKFCAA|2043-05-01|1720|7479|574|2043|5|5|1|2|2043|574|7479|Friday|2043Q2|N|Y|N|2467371|2467490|2467006|2467281|N|N|N|N|N| +2467372|AAAAAAAAMCGKFCAA|2043-05-02|1720|7479|574|2043|6|5|2|2|2043|574|7479|Saturday|2043Q2|N|Y|N|2467371|2467490|2467007|2467282|N|N|N|N|N| +2467373|AAAAAAAANCGKFCAA|2043-05-03|1720|7479|574|2043|0|5|3|2|2043|574|7479|Sunday|2043Q2|N|N|N|2467371|2467490|2467008|2467283|N|N|N|N|N| +2467374|AAAAAAAAOCGKFCAA|2043-05-04|1720|7479|574|2043|1|5|4|2|2043|574|7479|Monday|2043Q2|N|N|N|2467371|2467490|2467009|2467284|N|N|N|N|N| +2467375|AAAAAAAAPCGKFCAA|2043-05-05|1720|7480|574|2043|2|5|5|2|2043|574|7480|Tuesday|2043Q2|N|N|N|2467371|2467490|2467010|2467285|N|N|N|N|N| +2467376|AAAAAAAAADGKFCAA|2043-05-06|1720|7480|574|2043|3|5|6|2|2043|574|7480|Wednesday|2043Q2|N|N|N|2467371|2467490|2467011|2467286|N|N|N|N|N| +2467377|AAAAAAAABDGKFCAA|2043-05-07|1720|7480|574|2043|4|5|7|2|2043|574|7480|Thursday|2043Q2|N|N|N|2467371|2467490|2467012|2467287|N|N|N|N|N| +2467378|AAAAAAAACDGKFCAA|2043-05-08|1720|7480|574|2043|5|5|8|2|2043|574|7480|Friday|2043Q2|N|Y|N|2467371|2467490|2467013|2467288|N|N|N|N|N| +2467379|AAAAAAAADDGKFCAA|2043-05-09|1720|7480|574|2043|6|5|9|2|2043|574|7480|Saturday|2043Q2|N|Y|N|2467371|2467490|2467014|2467289|N|N|N|N|N| +2467380|AAAAAAAAEDGKFCAA|2043-05-10|1720|7480|574|2043|0|5|10|2|2043|574|7480|Sunday|2043Q2|N|N|N|2467371|2467490|2467015|2467290|N|N|N|N|N| +2467381|AAAAAAAAFDGKFCAA|2043-05-11|1720|7480|574|2043|1|5|11|2|2043|574|7480|Monday|2043Q2|N|N|N|2467371|2467490|2467016|2467291|N|N|N|N|N| +2467382|AAAAAAAAGDGKFCAA|2043-05-12|1720|7481|574|2043|2|5|12|2|2043|574|7481|Tuesday|2043Q2|N|N|N|2467371|2467490|2467017|2467292|N|N|N|N|N| +2467383|AAAAAAAAHDGKFCAA|2043-05-13|1720|7481|574|2043|3|5|13|2|2043|574|7481|Wednesday|2043Q2|N|N|N|2467371|2467490|2467018|2467293|N|N|N|N|N| +2467384|AAAAAAAAIDGKFCAA|2043-05-14|1720|7481|574|2043|4|5|14|2|2043|574|7481|Thursday|2043Q2|N|N|N|2467371|2467490|2467019|2467294|N|N|N|N|N| +2467385|AAAAAAAAJDGKFCAA|2043-05-15|1720|7481|574|2043|5|5|15|2|2043|574|7481|Friday|2043Q2|N|Y|N|2467371|2467490|2467020|2467295|N|N|N|N|N| +2467386|AAAAAAAAKDGKFCAA|2043-05-16|1720|7481|574|2043|6|5|16|2|2043|574|7481|Saturday|2043Q2|N|Y|N|2467371|2467490|2467021|2467296|N|N|N|N|N| +2467387|AAAAAAAALDGKFCAA|2043-05-17|1720|7481|574|2043|0|5|17|2|2043|574|7481|Sunday|2043Q2|N|N|N|2467371|2467490|2467022|2467297|N|N|N|N|N| +2467388|AAAAAAAAMDGKFCAA|2043-05-18|1720|7481|574|2043|1|5|18|2|2043|574|7481|Monday|2043Q2|N|N|N|2467371|2467490|2467023|2467298|N|N|N|N|N| +2467389|AAAAAAAANDGKFCAA|2043-05-19|1720|7482|574|2043|2|5|19|2|2043|574|7482|Tuesday|2043Q2|N|N|N|2467371|2467490|2467024|2467299|N|N|N|N|N| +2467390|AAAAAAAAODGKFCAA|2043-05-20|1720|7482|574|2043|3|5|20|2|2043|574|7482|Wednesday|2043Q2|N|N|N|2467371|2467490|2467025|2467300|N|N|N|N|N| +2467391|AAAAAAAAPDGKFCAA|2043-05-21|1720|7482|574|2043|4|5|21|2|2043|574|7482|Thursday|2043Q2|N|N|N|2467371|2467490|2467026|2467301|N|N|N|N|N| +2467392|AAAAAAAAAEGKFCAA|2043-05-22|1720|7482|574|2043|5|5|22|2|2043|574|7482|Friday|2043Q2|N|Y|N|2467371|2467490|2467027|2467302|N|N|N|N|N| +2467393|AAAAAAAABEGKFCAA|2043-05-23|1720|7482|574|2043|6|5|23|2|2043|574|7482|Saturday|2043Q2|N|Y|N|2467371|2467490|2467028|2467303|N|N|N|N|N| +2467394|AAAAAAAACEGKFCAA|2043-05-24|1720|7482|574|2043|0|5|24|2|2043|574|7482|Sunday|2043Q2|N|N|N|2467371|2467490|2467029|2467304|N|N|N|N|N| +2467395|AAAAAAAADEGKFCAA|2043-05-25|1720|7482|574|2043|1|5|25|2|2043|574|7482|Monday|2043Q2|N|N|N|2467371|2467490|2467030|2467305|N|N|N|N|N| +2467396|AAAAAAAAEEGKFCAA|2043-05-26|1720|7483|574|2043|2|5|26|2|2043|574|7483|Tuesday|2043Q2|N|N|N|2467371|2467490|2467031|2467306|N|N|N|N|N| +2467397|AAAAAAAAFEGKFCAA|2043-05-27|1720|7483|574|2043|3|5|27|2|2043|574|7483|Wednesday|2043Q2|N|N|N|2467371|2467490|2467032|2467307|N|N|N|N|N| +2467398|AAAAAAAAGEGKFCAA|2043-05-28|1720|7483|574|2043|4|5|28|2|2043|574|7483|Thursday|2043Q2|N|N|N|2467371|2467490|2467033|2467308|N|N|N|N|N| +2467399|AAAAAAAAHEGKFCAA|2043-05-29|1720|7483|574|2043|5|5|29|2|2043|574|7483|Friday|2043Q2|N|Y|N|2467371|2467490|2467034|2467309|N|N|N|N|N| +2467400|AAAAAAAAIEGKFCAA|2043-05-30|1720|7483|574|2043|6|5|30|2|2043|574|7483|Saturday|2043Q2|N|Y|N|2467371|2467490|2467035|2467310|N|N|N|N|N| +2467401|AAAAAAAAJEGKFCAA|2043-05-31|1720|7483|574|2043|0|5|31|2|2043|574|7483|Sunday|2043Q2|N|N|N|2467371|2467490|2467036|2467311|N|N|N|N|N| +2467402|AAAAAAAAKEGKFCAA|2043-06-01|1721|7483|575|2043|1|6|1|2|2043|575|7483|Monday|2043Q2|N|N|N|2467402|2467552|2467037|2467312|N|N|N|N|N| +2467403|AAAAAAAALEGKFCAA|2043-06-02|1721|7484|575|2043|2|6|2|2|2043|575|7484|Tuesday|2043Q2|N|N|N|2467402|2467552|2467038|2467313|N|N|N|N|N| +2467404|AAAAAAAAMEGKFCAA|2043-06-03|1721|7484|575|2043|3|6|3|2|2043|575|7484|Wednesday|2043Q2|N|N|N|2467402|2467552|2467039|2467314|N|N|N|N|N| +2467405|AAAAAAAANEGKFCAA|2043-06-04|1721|7484|575|2043|4|6|4|2|2043|575|7484|Thursday|2043Q2|N|N|N|2467402|2467552|2467040|2467315|N|N|N|N|N| +2467406|AAAAAAAAOEGKFCAA|2043-06-05|1721|7484|575|2043|5|6|5|2|2043|575|7484|Friday|2043Q2|N|Y|N|2467402|2467552|2467041|2467316|N|N|N|N|N| +2467407|AAAAAAAAPEGKFCAA|2043-06-06|1721|7484|575|2043|6|6|6|2|2043|575|7484|Saturday|2043Q2|N|Y|N|2467402|2467552|2467042|2467317|N|N|N|N|N| +2467408|AAAAAAAAAFGKFCAA|2043-06-07|1721|7484|575|2043|0|6|7|2|2043|575|7484|Sunday|2043Q2|N|N|N|2467402|2467552|2467043|2467318|N|N|N|N|N| +2467409|AAAAAAAABFGKFCAA|2043-06-08|1721|7484|575|2043|1|6|8|2|2043|575|7484|Monday|2043Q2|N|N|N|2467402|2467552|2467044|2467319|N|N|N|N|N| +2467410|AAAAAAAACFGKFCAA|2043-06-09|1721|7485|575|2043|2|6|9|2|2043|575|7485|Tuesday|2043Q2|N|N|N|2467402|2467552|2467045|2467320|N|N|N|N|N| +2467411|AAAAAAAADFGKFCAA|2043-06-10|1721|7485|575|2043|3|6|10|2|2043|575|7485|Wednesday|2043Q2|N|N|N|2467402|2467552|2467046|2467321|N|N|N|N|N| +2467412|AAAAAAAAEFGKFCAA|2043-06-11|1721|7485|575|2043|4|6|11|2|2043|575|7485|Thursday|2043Q2|N|N|N|2467402|2467552|2467047|2467322|N|N|N|N|N| +2467413|AAAAAAAAFFGKFCAA|2043-06-12|1721|7485|575|2043|5|6|12|2|2043|575|7485|Friday|2043Q2|N|Y|N|2467402|2467552|2467048|2467323|N|N|N|N|N| +2467414|AAAAAAAAGFGKFCAA|2043-06-13|1721|7485|575|2043|6|6|13|2|2043|575|7485|Saturday|2043Q2|N|Y|N|2467402|2467552|2467049|2467324|N|N|N|N|N| +2467415|AAAAAAAAHFGKFCAA|2043-06-14|1721|7485|575|2043|0|6|14|2|2043|575|7485|Sunday|2043Q2|N|N|N|2467402|2467552|2467050|2467325|N|N|N|N|N| +2467416|AAAAAAAAIFGKFCAA|2043-06-15|1721|7485|575|2043|1|6|15|2|2043|575|7485|Monday|2043Q2|N|N|N|2467402|2467552|2467051|2467326|N|N|N|N|N| +2467417|AAAAAAAAJFGKFCAA|2043-06-16|1721|7486|575|2043|2|6|16|2|2043|575|7486|Tuesday|2043Q2|N|N|N|2467402|2467552|2467052|2467327|N|N|N|N|N| +2467418|AAAAAAAAKFGKFCAA|2043-06-17|1721|7486|575|2043|3|6|17|2|2043|575|7486|Wednesday|2043Q2|N|N|N|2467402|2467552|2467053|2467328|N|N|N|N|N| +2467419|AAAAAAAALFGKFCAA|2043-06-18|1721|7486|575|2043|4|6|18|2|2043|575|7486|Thursday|2043Q2|N|N|N|2467402|2467552|2467054|2467329|N|N|N|N|N| +2467420|AAAAAAAAMFGKFCAA|2043-06-19|1721|7486|575|2043|5|6|19|2|2043|575|7486|Friday|2043Q2|N|Y|N|2467402|2467552|2467055|2467330|N|N|N|N|N| +2467421|AAAAAAAANFGKFCAA|2043-06-20|1721|7486|575|2043|6|6|20|2|2043|575|7486|Saturday|2043Q2|N|Y|N|2467402|2467552|2467056|2467331|N|N|N|N|N| +2467422|AAAAAAAAOFGKFCAA|2043-06-21|1721|7486|575|2043|0|6|21|2|2043|575|7486|Sunday|2043Q2|N|N|N|2467402|2467552|2467057|2467332|N|N|N|N|N| +2467423|AAAAAAAAPFGKFCAA|2043-06-22|1721|7486|575|2043|1|6|22|2|2043|575|7486|Monday|2043Q2|N|N|N|2467402|2467552|2467058|2467333|N|N|N|N|N| +2467424|AAAAAAAAAGGKFCAA|2043-06-23|1721|7487|575|2043|2|6|23|2|2043|575|7487|Tuesday|2043Q2|N|N|N|2467402|2467552|2467059|2467334|N|N|N|N|N| +2467425|AAAAAAAABGGKFCAA|2043-06-24|1721|7487|575|2043|3|6|24|2|2043|575|7487|Wednesday|2043Q2|N|N|N|2467402|2467552|2467060|2467335|N|N|N|N|N| +2467426|AAAAAAAACGGKFCAA|2043-06-25|1721|7487|575|2043|4|6|25|2|2043|575|7487|Thursday|2043Q2|N|N|N|2467402|2467552|2467061|2467336|N|N|N|N|N| +2467427|AAAAAAAADGGKFCAA|2043-06-26|1721|7487|575|2043|5|6|26|2|2043|575|7487|Friday|2043Q2|N|Y|N|2467402|2467552|2467062|2467337|N|N|N|N|N| +2467428|AAAAAAAAEGGKFCAA|2043-06-27|1721|7487|575|2043|6|6|27|2|2043|575|7487|Saturday|2043Q2|N|Y|N|2467402|2467552|2467063|2467338|N|N|N|N|N| +2467429|AAAAAAAAFGGKFCAA|2043-06-28|1721|7487|575|2043|0|6|28|2|2043|575|7487|Sunday|2043Q2|N|N|N|2467402|2467552|2467064|2467339|N|N|N|N|N| +2467430|AAAAAAAAGGGKFCAA|2043-06-29|1721|7487|575|2043|1|6|29|2|2043|575|7487|Monday|2043Q2|N|N|N|2467402|2467552|2467065|2467340|N|N|N|N|N| +2467431|AAAAAAAAHGGKFCAA|2043-06-30|1721|7488|575|2043|2|6|30|2|2043|575|7488|Tuesday|2043Q2|N|N|N|2467402|2467552|2467066|2467341|N|N|N|N|N| +2467432|AAAAAAAAIGGKFCAA|2043-07-01|1722|7488|575|2043|3|7|1|2|2043|575|7488|Wednesday|2043Q2|N|N|N|2467432|2467612|2467067|2467341|N|N|N|N|N| +2467433|AAAAAAAAJGGKFCAA|2043-07-02|1722|7488|575|2043|4|7|2|3|2043|575|7488|Thursday|2043Q3|N|N|N|2467432|2467612|2467068|2467342|N|N|N|N|N| +2467434|AAAAAAAAKGGKFCAA|2043-07-03|1722|7488|575|2043|5|7|3|3|2043|575|7488|Friday|2043Q3|N|Y|N|2467432|2467612|2467069|2467343|N|N|N|N|N| +2467435|AAAAAAAALGGKFCAA|2043-07-04|1722|7488|575|2043|6|7|4|3|2043|575|7488|Saturday|2043Q3|N|Y|N|2467432|2467612|2467070|2467344|N|N|N|N|N| +2467436|AAAAAAAAMGGKFCAA|2043-07-05|1722|7488|575|2043|0|7|5|3|2043|575|7488|Sunday|2043Q3|Y|N|N|2467432|2467612|2467071|2467345|N|N|N|N|N| +2467437|AAAAAAAANGGKFCAA|2043-07-06|1722|7488|575|2043|1|7|6|3|2043|575|7488|Monday|2043Q3|N|N|Y|2467432|2467612|2467072|2467346|N|N|N|N|N| +2467438|AAAAAAAAOGGKFCAA|2043-07-07|1722|7489|575|2043|2|7|7|3|2043|575|7489|Tuesday|2043Q3|N|N|N|2467432|2467612|2467073|2467347|N|N|N|N|N| +2467439|AAAAAAAAPGGKFCAA|2043-07-08|1722|7489|575|2043|3|7|8|3|2043|575|7489|Wednesday|2043Q3|N|N|N|2467432|2467612|2467074|2467348|N|N|N|N|N| +2467440|AAAAAAAAAHGKFCAA|2043-07-09|1722|7489|575|2043|4|7|9|3|2043|575|7489|Thursday|2043Q3|N|N|N|2467432|2467612|2467075|2467349|N|N|N|N|N| +2467441|AAAAAAAABHGKFCAA|2043-07-10|1722|7489|575|2043|5|7|10|3|2043|575|7489|Friday|2043Q3|N|Y|N|2467432|2467612|2467076|2467350|N|N|N|N|N| +2467442|AAAAAAAACHGKFCAA|2043-07-11|1722|7489|575|2043|6|7|11|3|2043|575|7489|Saturday|2043Q3|N|Y|N|2467432|2467612|2467077|2467351|N|N|N|N|N| +2467443|AAAAAAAADHGKFCAA|2043-07-12|1722|7489|575|2043|0|7|12|3|2043|575|7489|Sunday|2043Q3|N|N|N|2467432|2467612|2467078|2467352|N|N|N|N|N| +2467444|AAAAAAAAEHGKFCAA|2043-07-13|1722|7489|575|2043|1|7|13|3|2043|575|7489|Monday|2043Q3|N|N|N|2467432|2467612|2467079|2467353|N|N|N|N|N| +2467445|AAAAAAAAFHGKFCAA|2043-07-14|1722|7490|575|2043|2|7|14|3|2043|575|7490|Tuesday|2043Q3|N|N|N|2467432|2467612|2467080|2467354|N|N|N|N|N| +2467446|AAAAAAAAGHGKFCAA|2043-07-15|1722|7490|575|2043|3|7|15|3|2043|575|7490|Wednesday|2043Q3|N|N|N|2467432|2467612|2467081|2467355|N|N|N|N|N| +2467447|AAAAAAAAHHGKFCAA|2043-07-16|1722|7490|575|2043|4|7|16|3|2043|575|7490|Thursday|2043Q3|N|N|N|2467432|2467612|2467082|2467356|N|N|N|N|N| +2467448|AAAAAAAAIHGKFCAA|2043-07-17|1722|7490|575|2043|5|7|17|3|2043|575|7490|Friday|2043Q3|N|Y|N|2467432|2467612|2467083|2467357|N|N|N|N|N| +2467449|AAAAAAAAJHGKFCAA|2043-07-18|1722|7490|575|2043|6|7|18|3|2043|575|7490|Saturday|2043Q3|N|Y|N|2467432|2467612|2467084|2467358|N|N|N|N|N| +2467450|AAAAAAAAKHGKFCAA|2043-07-19|1722|7490|575|2043|0|7|19|3|2043|575|7490|Sunday|2043Q3|N|N|N|2467432|2467612|2467085|2467359|N|N|N|N|N| +2467451|AAAAAAAALHGKFCAA|2043-07-20|1722|7490|575|2043|1|7|20|3|2043|575|7490|Monday|2043Q3|N|N|N|2467432|2467612|2467086|2467360|N|N|N|N|N| +2467452|AAAAAAAAMHGKFCAA|2043-07-21|1722|7491|575|2043|2|7|21|3|2043|575|7491|Tuesday|2043Q3|N|N|N|2467432|2467612|2467087|2467361|N|N|N|N|N| +2467453|AAAAAAAANHGKFCAA|2043-07-22|1722|7491|575|2043|3|7|22|3|2043|575|7491|Wednesday|2043Q3|N|N|N|2467432|2467612|2467088|2467362|N|N|N|N|N| +2467454|AAAAAAAAOHGKFCAA|2043-07-23|1722|7491|575|2043|4|7|23|3|2043|575|7491|Thursday|2043Q3|N|N|N|2467432|2467612|2467089|2467363|N|N|N|N|N| +2467455|AAAAAAAAPHGKFCAA|2043-07-24|1722|7491|575|2043|5|7|24|3|2043|575|7491|Friday|2043Q3|N|Y|N|2467432|2467612|2467090|2467364|N|N|N|N|N| +2467456|AAAAAAAAAIGKFCAA|2043-07-25|1722|7491|575|2043|6|7|25|3|2043|575|7491|Saturday|2043Q3|N|Y|N|2467432|2467612|2467091|2467365|N|N|N|N|N| +2467457|AAAAAAAABIGKFCAA|2043-07-26|1722|7491|575|2043|0|7|26|3|2043|575|7491|Sunday|2043Q3|N|N|N|2467432|2467612|2467092|2467366|N|N|N|N|N| +2467458|AAAAAAAACIGKFCAA|2043-07-27|1722|7491|575|2043|1|7|27|3|2043|575|7491|Monday|2043Q3|N|N|N|2467432|2467612|2467093|2467367|N|N|N|N|N| +2467459|AAAAAAAADIGKFCAA|2043-07-28|1722|7492|575|2043|2|7|28|3|2043|575|7492|Tuesday|2043Q3|N|N|N|2467432|2467612|2467094|2467368|N|N|N|N|N| +2467460|AAAAAAAAEIGKFCAA|2043-07-29|1722|7492|575|2043|3|7|29|3|2043|575|7492|Wednesday|2043Q3|N|N|N|2467432|2467612|2467095|2467369|N|N|N|N|N| +2467461|AAAAAAAAFIGKFCAA|2043-07-30|1722|7492|575|2043|4|7|30|3|2043|575|7492|Thursday|2043Q3|N|N|N|2467432|2467612|2467096|2467370|N|N|N|N|N| +2467462|AAAAAAAAGIGKFCAA|2043-07-31|1722|7492|575|2043|5|7|31|3|2043|575|7492|Friday|2043Q3|N|Y|N|2467432|2467612|2467097|2467371|N|N|N|N|N| +2467463|AAAAAAAAHIGKFCAA|2043-08-01|1723|7492|575|2043|6|8|1|3|2043|575|7492|Saturday|2043Q3|N|Y|N|2467463|2467674|2467098|2467372|N|N|N|N|N| +2467464|AAAAAAAAIIGKFCAA|2043-08-02|1723|7492|575|2043|0|8|2|3|2043|575|7492|Sunday|2043Q3|N|N|N|2467463|2467674|2467099|2467373|N|N|N|N|N| +2467465|AAAAAAAAJIGKFCAA|2043-08-03|1723|7492|575|2043|1|8|3|3|2043|575|7492|Monday|2043Q3|N|N|N|2467463|2467674|2467100|2467374|N|N|N|N|N| +2467466|AAAAAAAAKIGKFCAA|2043-08-04|1723|7493|575|2043|2|8|4|3|2043|575|7493|Tuesday|2043Q3|N|N|N|2467463|2467674|2467101|2467375|N|N|N|N|N| +2467467|AAAAAAAALIGKFCAA|2043-08-05|1723|7493|575|2043|3|8|5|3|2043|575|7493|Wednesday|2043Q3|N|N|N|2467463|2467674|2467102|2467376|N|N|N|N|N| +2467468|AAAAAAAAMIGKFCAA|2043-08-06|1723|7493|575|2043|4|8|6|3|2043|575|7493|Thursday|2043Q3|N|N|N|2467463|2467674|2467103|2467377|N|N|N|N|N| +2467469|AAAAAAAANIGKFCAA|2043-08-07|1723|7493|575|2043|5|8|7|3|2043|575|7493|Friday|2043Q3|N|Y|N|2467463|2467674|2467104|2467378|N|N|N|N|N| +2467470|AAAAAAAAOIGKFCAA|2043-08-08|1723|7493|575|2043|6|8|8|3|2043|575|7493|Saturday|2043Q3|N|Y|N|2467463|2467674|2467105|2467379|N|N|N|N|N| +2467471|AAAAAAAAPIGKFCAA|2043-08-09|1723|7493|575|2043|0|8|9|3|2043|575|7493|Sunday|2043Q3|N|N|N|2467463|2467674|2467106|2467380|N|N|N|N|N| +2467472|AAAAAAAAAJGKFCAA|2043-08-10|1723|7493|575|2043|1|8|10|3|2043|575|7493|Monday|2043Q3|N|N|N|2467463|2467674|2467107|2467381|N|N|N|N|N| +2467473|AAAAAAAABJGKFCAA|2043-08-11|1723|7494|575|2043|2|8|11|3|2043|575|7494|Tuesday|2043Q3|N|N|N|2467463|2467674|2467108|2467382|N|N|N|N|N| +2467474|AAAAAAAACJGKFCAA|2043-08-12|1723|7494|575|2043|3|8|12|3|2043|575|7494|Wednesday|2043Q3|N|N|N|2467463|2467674|2467109|2467383|N|N|N|N|N| +2467475|AAAAAAAADJGKFCAA|2043-08-13|1723|7494|575|2043|4|8|13|3|2043|575|7494|Thursday|2043Q3|N|N|N|2467463|2467674|2467110|2467384|N|N|N|N|N| +2467476|AAAAAAAAEJGKFCAA|2043-08-14|1723|7494|575|2043|5|8|14|3|2043|575|7494|Friday|2043Q3|N|Y|N|2467463|2467674|2467111|2467385|N|N|N|N|N| +2467477|AAAAAAAAFJGKFCAA|2043-08-15|1723|7494|575|2043|6|8|15|3|2043|575|7494|Saturday|2043Q3|N|Y|N|2467463|2467674|2467112|2467386|N|N|N|N|N| +2467478|AAAAAAAAGJGKFCAA|2043-08-16|1723|7494|575|2043|0|8|16|3|2043|575|7494|Sunday|2043Q3|N|N|N|2467463|2467674|2467113|2467387|N|N|N|N|N| +2467479|AAAAAAAAHJGKFCAA|2043-08-17|1723|7494|575|2043|1|8|17|3|2043|575|7494|Monday|2043Q3|N|N|N|2467463|2467674|2467114|2467388|N|N|N|N|N| +2467480|AAAAAAAAIJGKFCAA|2043-08-18|1723|7495|575|2043|2|8|18|3|2043|575|7495|Tuesday|2043Q3|N|N|N|2467463|2467674|2467115|2467389|N|N|N|N|N| +2467481|AAAAAAAAJJGKFCAA|2043-08-19|1723|7495|575|2043|3|8|19|3|2043|575|7495|Wednesday|2043Q3|N|N|N|2467463|2467674|2467116|2467390|N|N|N|N|N| +2467482|AAAAAAAAKJGKFCAA|2043-08-20|1723|7495|575|2043|4|8|20|3|2043|575|7495|Thursday|2043Q3|N|N|N|2467463|2467674|2467117|2467391|N|N|N|N|N| +2467483|AAAAAAAALJGKFCAA|2043-08-21|1723|7495|575|2043|5|8|21|3|2043|575|7495|Friday|2043Q3|N|Y|N|2467463|2467674|2467118|2467392|N|N|N|N|N| +2467484|AAAAAAAAMJGKFCAA|2043-08-22|1723|7495|575|2043|6|8|22|3|2043|575|7495|Saturday|2043Q3|N|Y|N|2467463|2467674|2467119|2467393|N|N|N|N|N| +2467485|AAAAAAAANJGKFCAA|2043-08-23|1723|7495|575|2043|0|8|23|3|2043|575|7495|Sunday|2043Q3|N|N|N|2467463|2467674|2467120|2467394|N|N|N|N|N| +2467486|AAAAAAAAOJGKFCAA|2043-08-24|1723|7495|575|2043|1|8|24|3|2043|575|7495|Monday|2043Q3|N|N|N|2467463|2467674|2467121|2467395|N|N|N|N|N| +2467487|AAAAAAAAPJGKFCAA|2043-08-25|1723|7496|575|2043|2|8|25|3|2043|575|7496|Tuesday|2043Q3|N|N|N|2467463|2467674|2467122|2467396|N|N|N|N|N| +2467488|AAAAAAAAAKGKFCAA|2043-08-26|1723|7496|575|2043|3|8|26|3|2043|575|7496|Wednesday|2043Q3|N|N|N|2467463|2467674|2467123|2467397|N|N|N|N|N| +2467489|AAAAAAAABKGKFCAA|2043-08-27|1723|7496|575|2043|4|8|27|3|2043|575|7496|Thursday|2043Q3|N|N|N|2467463|2467674|2467124|2467398|N|N|N|N|N| +2467490|AAAAAAAACKGKFCAA|2043-08-28|1723|7496|575|2043|5|8|28|3|2043|575|7496|Friday|2043Q3|N|Y|N|2467463|2467674|2467125|2467399|N|N|N|N|N| +2467491|AAAAAAAADKGKFCAA|2043-08-29|1723|7496|575|2043|6|8|29|3|2043|575|7496|Saturday|2043Q3|N|Y|N|2467463|2467674|2467126|2467400|N|N|N|N|N| +2467492|AAAAAAAAEKGKFCAA|2043-08-30|1723|7496|575|2043|0|8|30|3|2043|575|7496|Sunday|2043Q3|N|N|N|2467463|2467674|2467127|2467401|N|N|N|N|N| +2467493|AAAAAAAAFKGKFCAA|2043-08-31|1723|7496|575|2043|1|8|31|3|2043|575|7496|Monday|2043Q3|N|N|N|2467463|2467674|2467128|2467402|N|N|N|N|N| +2467494|AAAAAAAAGKGKFCAA|2043-09-01|1724|7497|576|2043|2|9|1|3|2043|576|7497|Tuesday|2043Q3|N|N|N|2467494|2467736|2467129|2467403|N|N|N|N|N| +2467495|AAAAAAAAHKGKFCAA|2043-09-02|1724|7497|576|2043|3|9|2|3|2043|576|7497|Wednesday|2043Q3|N|N|N|2467494|2467736|2467130|2467404|N|N|N|N|N| +2467496|AAAAAAAAIKGKFCAA|2043-09-03|1724|7497|576|2043|4|9|3|3|2043|576|7497|Thursday|2043Q3|N|N|N|2467494|2467736|2467131|2467405|N|N|N|N|N| +2467497|AAAAAAAAJKGKFCAA|2043-09-04|1724|7497|576|2043|5|9|4|3|2043|576|7497|Friday|2043Q3|N|Y|N|2467494|2467736|2467132|2467406|N|N|N|N|N| +2467498|AAAAAAAAKKGKFCAA|2043-09-05|1724|7497|576|2043|6|9|5|3|2043|576|7497|Saturday|2043Q3|N|Y|N|2467494|2467736|2467133|2467407|N|N|N|N|N| +2467499|AAAAAAAALKGKFCAA|2043-09-06|1724|7497|576|2043|0|9|6|3|2043|576|7497|Sunday|2043Q3|N|N|N|2467494|2467736|2467134|2467408|N|N|N|N|N| +2467500|AAAAAAAAMKGKFCAA|2043-09-07|1724|7497|576|2043|1|9|7|3|2043|576|7497|Monday|2043Q3|N|N|N|2467494|2467736|2467135|2467409|N|N|N|N|N| +2467501|AAAAAAAANKGKFCAA|2043-09-08|1724|7498|576|2043|2|9|8|3|2043|576|7498|Tuesday|2043Q3|N|N|N|2467494|2467736|2467136|2467410|N|N|N|N|N| +2467502|AAAAAAAAOKGKFCAA|2043-09-09|1724|7498|576|2043|3|9|9|3|2043|576|7498|Wednesday|2043Q3|N|N|N|2467494|2467736|2467137|2467411|N|N|N|N|N| +2467503|AAAAAAAAPKGKFCAA|2043-09-10|1724|7498|576|2043|4|9|10|3|2043|576|7498|Thursday|2043Q3|N|N|N|2467494|2467736|2467138|2467412|N|N|N|N|N| +2467504|AAAAAAAAALGKFCAA|2043-09-11|1724|7498|576|2043|5|9|11|3|2043|576|7498|Friday|2043Q3|N|Y|N|2467494|2467736|2467139|2467413|N|N|N|N|N| +2467505|AAAAAAAABLGKFCAA|2043-09-12|1724|7498|576|2043|6|9|12|3|2043|576|7498|Saturday|2043Q3|N|Y|N|2467494|2467736|2467140|2467414|N|N|N|N|N| +2467506|AAAAAAAACLGKFCAA|2043-09-13|1724|7498|576|2043|0|9|13|3|2043|576|7498|Sunday|2043Q3|N|N|N|2467494|2467736|2467141|2467415|N|N|N|N|N| +2467507|AAAAAAAADLGKFCAA|2043-09-14|1724|7498|576|2043|1|9|14|3|2043|576|7498|Monday|2043Q3|N|N|N|2467494|2467736|2467142|2467416|N|N|N|N|N| +2467508|AAAAAAAAELGKFCAA|2043-09-15|1724|7499|576|2043|2|9|15|3|2043|576|7499|Tuesday|2043Q3|N|N|N|2467494|2467736|2467143|2467417|N|N|N|N|N| +2467509|AAAAAAAAFLGKFCAA|2043-09-16|1724|7499|576|2043|3|9|16|3|2043|576|7499|Wednesday|2043Q3|N|N|N|2467494|2467736|2467144|2467418|N|N|N|N|N| +2467510|AAAAAAAAGLGKFCAA|2043-09-17|1724|7499|576|2043|4|9|17|3|2043|576|7499|Thursday|2043Q3|N|N|N|2467494|2467736|2467145|2467419|N|N|N|N|N| +2467511|AAAAAAAAHLGKFCAA|2043-09-18|1724|7499|576|2043|5|9|18|3|2043|576|7499|Friday|2043Q3|N|Y|N|2467494|2467736|2467146|2467420|N|N|N|N|N| +2467512|AAAAAAAAILGKFCAA|2043-09-19|1724|7499|576|2043|6|9|19|3|2043|576|7499|Saturday|2043Q3|N|Y|N|2467494|2467736|2467147|2467421|N|N|N|N|N| +2467513|AAAAAAAAJLGKFCAA|2043-09-20|1724|7499|576|2043|0|9|20|3|2043|576|7499|Sunday|2043Q3|N|N|N|2467494|2467736|2467148|2467422|N|N|N|N|N| +2467514|AAAAAAAAKLGKFCAA|2043-09-21|1724|7499|576|2043|1|9|21|3|2043|576|7499|Monday|2043Q3|N|N|N|2467494|2467736|2467149|2467423|N|N|N|N|N| +2467515|AAAAAAAALLGKFCAA|2043-09-22|1724|7500|576|2043|2|9|22|3|2043|576|7500|Tuesday|2043Q3|N|N|N|2467494|2467736|2467150|2467424|N|N|N|N|N| +2467516|AAAAAAAAMLGKFCAA|2043-09-23|1724|7500|576|2043|3|9|23|3|2043|576|7500|Wednesday|2043Q3|N|N|N|2467494|2467736|2467151|2467425|N|N|N|N|N| +2467517|AAAAAAAANLGKFCAA|2043-09-24|1724|7500|576|2043|4|9|24|3|2043|576|7500|Thursday|2043Q3|N|N|N|2467494|2467736|2467152|2467426|N|N|N|N|N| +2467518|AAAAAAAAOLGKFCAA|2043-09-25|1724|7500|576|2043|5|9|25|3|2043|576|7500|Friday|2043Q3|N|Y|N|2467494|2467736|2467153|2467427|N|N|N|N|N| +2467519|AAAAAAAAPLGKFCAA|2043-09-26|1724|7500|576|2043|6|9|26|3|2043|576|7500|Saturday|2043Q3|N|Y|N|2467494|2467736|2467154|2467428|N|N|N|N|N| +2467520|AAAAAAAAAMGKFCAA|2043-09-27|1724|7500|576|2043|0|9|27|3|2043|576|7500|Sunday|2043Q3|N|N|N|2467494|2467736|2467155|2467429|N|N|N|N|N| +2467521|AAAAAAAABMGKFCAA|2043-09-28|1724|7500|576|2043|1|9|28|3|2043|576|7500|Monday|2043Q3|N|N|N|2467494|2467736|2467156|2467430|N|N|N|N|N| +2467522|AAAAAAAACMGKFCAA|2043-09-29|1724|7501|576|2043|2|9|29|3|2043|576|7501|Tuesday|2043Q3|N|N|N|2467494|2467736|2467157|2467431|N|N|N|N|N| +2467523|AAAAAAAADMGKFCAA|2043-09-30|1724|7501|576|2043|3|9|30|3|2043|576|7501|Wednesday|2043Q3|N|N|N|2467494|2467736|2467158|2467432|N|N|N|N|N| +2467524|AAAAAAAAEMGKFCAA|2043-10-01|1725|7501|576|2043|4|10|1|3|2043|576|7501|Thursday|2043Q3|N|N|N|2467524|2467796|2467159|2467432|N|N|N|N|N| +2467525|AAAAAAAAFMGKFCAA|2043-10-02|1725|7501|576|2043|5|10|2|4|2043|576|7501|Friday|2043Q4|N|Y|N|2467524|2467796|2467160|2467433|N|N|N|N|N| +2467526|AAAAAAAAGMGKFCAA|2043-10-03|1725|7501|576|2043|6|10|3|4|2043|576|7501|Saturday|2043Q4|N|Y|N|2467524|2467796|2467161|2467434|N|N|N|N|N| +2467527|AAAAAAAAHMGKFCAA|2043-10-04|1725|7501|576|2043|0|10|4|4|2043|576|7501|Sunday|2043Q4|N|N|N|2467524|2467796|2467162|2467435|N|N|N|N|N| +2467528|AAAAAAAAIMGKFCAA|2043-10-05|1725|7501|576|2043|1|10|5|4|2043|576|7501|Monday|2043Q4|N|N|N|2467524|2467796|2467163|2467436|N|N|N|N|N| +2467529|AAAAAAAAJMGKFCAA|2043-10-06|1725|7502|576|2043|2|10|6|4|2043|576|7502|Tuesday|2043Q4|N|N|N|2467524|2467796|2467164|2467437|N|N|N|N|N| +2467530|AAAAAAAAKMGKFCAA|2043-10-07|1725|7502|576|2043|3|10|7|4|2043|576|7502|Wednesday|2043Q4|N|N|N|2467524|2467796|2467165|2467438|N|N|N|N|N| +2467531|AAAAAAAALMGKFCAA|2043-10-08|1725|7502|576|2043|4|10|8|4|2043|576|7502|Thursday|2043Q4|N|N|N|2467524|2467796|2467166|2467439|N|N|N|N|N| +2467532|AAAAAAAAMMGKFCAA|2043-10-09|1725|7502|576|2043|5|10|9|4|2043|576|7502|Friday|2043Q4|N|Y|N|2467524|2467796|2467167|2467440|N|N|N|N|N| +2467533|AAAAAAAANMGKFCAA|2043-10-10|1725|7502|576|2043|6|10|10|4|2043|576|7502|Saturday|2043Q4|N|Y|N|2467524|2467796|2467168|2467441|N|N|N|N|N| +2467534|AAAAAAAAOMGKFCAA|2043-10-11|1725|7502|576|2043|0|10|11|4|2043|576|7502|Sunday|2043Q4|N|N|N|2467524|2467796|2467169|2467442|N|N|N|N|N| +2467535|AAAAAAAAPMGKFCAA|2043-10-12|1725|7502|576|2043|1|10|12|4|2043|576|7502|Monday|2043Q4|N|N|N|2467524|2467796|2467170|2467443|N|N|N|N|N| +2467536|AAAAAAAAANGKFCAA|2043-10-13|1725|7503|576|2043|2|10|13|4|2043|576|7503|Tuesday|2043Q4|N|N|N|2467524|2467796|2467171|2467444|N|N|N|N|N| +2467537|AAAAAAAABNGKFCAA|2043-10-14|1725|7503|576|2043|3|10|14|4|2043|576|7503|Wednesday|2043Q4|N|N|N|2467524|2467796|2467172|2467445|N|N|N|N|N| +2467538|AAAAAAAACNGKFCAA|2043-10-15|1725|7503|576|2043|4|10|15|4|2043|576|7503|Thursday|2043Q4|N|N|N|2467524|2467796|2467173|2467446|N|N|N|N|N| +2467539|AAAAAAAADNGKFCAA|2043-10-16|1725|7503|576|2043|5|10|16|4|2043|576|7503|Friday|2043Q4|N|Y|N|2467524|2467796|2467174|2467447|N|N|N|N|N| +2467540|AAAAAAAAENGKFCAA|2043-10-17|1725|7503|576|2043|6|10|17|4|2043|576|7503|Saturday|2043Q4|N|Y|N|2467524|2467796|2467175|2467448|N|N|N|N|N| +2467541|AAAAAAAAFNGKFCAA|2043-10-18|1725|7503|576|2043|0|10|18|4|2043|576|7503|Sunday|2043Q4|N|N|N|2467524|2467796|2467176|2467449|N|N|N|N|N| +2467542|AAAAAAAAGNGKFCAA|2043-10-19|1725|7503|576|2043|1|10|19|4|2043|576|7503|Monday|2043Q4|N|N|N|2467524|2467796|2467177|2467450|N|N|N|N|N| +2467543|AAAAAAAAHNGKFCAA|2043-10-20|1725|7504|576|2043|2|10|20|4|2043|576|7504|Tuesday|2043Q4|N|N|N|2467524|2467796|2467178|2467451|N|N|N|N|N| +2467544|AAAAAAAAINGKFCAA|2043-10-21|1725|7504|576|2043|3|10|21|4|2043|576|7504|Wednesday|2043Q4|N|N|N|2467524|2467796|2467179|2467452|N|N|N|N|N| +2467545|AAAAAAAAJNGKFCAA|2043-10-22|1725|7504|576|2043|4|10|22|4|2043|576|7504|Thursday|2043Q4|N|N|N|2467524|2467796|2467180|2467453|N|N|N|N|N| +2467546|AAAAAAAAKNGKFCAA|2043-10-23|1725|7504|576|2043|5|10|23|4|2043|576|7504|Friday|2043Q4|N|Y|N|2467524|2467796|2467181|2467454|N|N|N|N|N| +2467547|AAAAAAAALNGKFCAA|2043-10-24|1725|7504|576|2043|6|10|24|4|2043|576|7504|Saturday|2043Q4|N|Y|N|2467524|2467796|2467182|2467455|N|N|N|N|N| +2467548|AAAAAAAAMNGKFCAA|2043-10-25|1725|7504|576|2043|0|10|25|4|2043|576|7504|Sunday|2043Q4|N|N|N|2467524|2467796|2467183|2467456|N|N|N|N|N| +2467549|AAAAAAAANNGKFCAA|2043-10-26|1725|7504|576|2043|1|10|26|4|2043|576|7504|Monday|2043Q4|N|N|N|2467524|2467796|2467184|2467457|N|N|N|N|N| +2467550|AAAAAAAAONGKFCAA|2043-10-27|1725|7505|576|2043|2|10|27|4|2043|576|7505|Tuesday|2043Q4|N|N|N|2467524|2467796|2467185|2467458|N|N|N|N|N| +2467551|AAAAAAAAPNGKFCAA|2043-10-28|1725|7505|576|2043|3|10|28|4|2043|576|7505|Wednesday|2043Q4|N|N|N|2467524|2467796|2467186|2467459|N|N|N|N|N| +2467552|AAAAAAAAAOGKFCAA|2043-10-29|1725|7505|576|2043|4|10|29|4|2043|576|7505|Thursday|2043Q4|N|N|N|2467524|2467796|2467187|2467460|N|N|N|N|N| +2467553|AAAAAAAABOGKFCAA|2043-10-30|1725|7505|576|2043|5|10|30|4|2043|576|7505|Friday|2043Q4|N|Y|N|2467524|2467796|2467188|2467461|N|N|N|N|N| +2467554|AAAAAAAACOGKFCAA|2043-10-31|1725|7505|576|2043|6|10|31|4|2043|576|7505|Saturday|2043Q4|N|Y|N|2467524|2467796|2467189|2467462|N|N|N|N|N| +2467555|AAAAAAAADOGKFCAA|2043-11-01|1726|7505|576|2043|0|11|1|4|2043|576|7505|Sunday|2043Q4|N|N|N|2467555|2467858|2467190|2467463|N|N|N|N|N| +2467556|AAAAAAAAEOGKFCAA|2043-11-02|1726|7505|576|2043|1|11|2|4|2043|576|7505|Monday|2043Q4|N|N|N|2467555|2467858|2467191|2467464|N|N|N|N|N| +2467557|AAAAAAAAFOGKFCAA|2043-11-03|1726|7506|576|2043|2|11|3|4|2043|576|7506|Tuesday|2043Q4|N|N|N|2467555|2467858|2467192|2467465|N|N|N|N|N| +2467558|AAAAAAAAGOGKFCAA|2043-11-04|1726|7506|576|2043|3|11|4|4|2043|576|7506|Wednesday|2043Q4|N|N|N|2467555|2467858|2467193|2467466|N|N|N|N|N| +2467559|AAAAAAAAHOGKFCAA|2043-11-05|1726|7506|576|2043|4|11|5|4|2043|576|7506|Thursday|2043Q4|N|N|N|2467555|2467858|2467194|2467467|N|N|N|N|N| +2467560|AAAAAAAAIOGKFCAA|2043-11-06|1726|7506|576|2043|5|11|6|4|2043|576|7506|Friday|2043Q4|N|Y|N|2467555|2467858|2467195|2467468|N|N|N|N|N| +2467561|AAAAAAAAJOGKFCAA|2043-11-07|1726|7506|576|2043|6|11|7|4|2043|576|7506|Saturday|2043Q4|N|Y|N|2467555|2467858|2467196|2467469|N|N|N|N|N| +2467562|AAAAAAAAKOGKFCAA|2043-11-08|1726|7506|576|2043|0|11|8|4|2043|576|7506|Sunday|2043Q4|N|N|N|2467555|2467858|2467197|2467470|N|N|N|N|N| +2467563|AAAAAAAALOGKFCAA|2043-11-09|1726|7506|576|2043|1|11|9|4|2043|576|7506|Monday|2043Q4|N|N|N|2467555|2467858|2467198|2467471|N|N|N|N|N| +2467564|AAAAAAAAMOGKFCAA|2043-11-10|1726|7507|576|2043|2|11|10|4|2043|576|7507|Tuesday|2043Q4|N|N|N|2467555|2467858|2467199|2467472|N|N|N|N|N| +2467565|AAAAAAAANOGKFCAA|2043-11-11|1726|7507|576|2043|3|11|11|4|2043|576|7507|Wednesday|2043Q4|N|N|N|2467555|2467858|2467200|2467473|N|N|N|N|N| +2467566|AAAAAAAAOOGKFCAA|2043-11-12|1726|7507|576|2043|4|11|12|4|2043|576|7507|Thursday|2043Q4|N|N|N|2467555|2467858|2467201|2467474|N|N|N|N|N| +2467567|AAAAAAAAPOGKFCAA|2043-11-13|1726|7507|576|2043|5|11|13|4|2043|576|7507|Friday|2043Q4|N|Y|N|2467555|2467858|2467202|2467475|N|N|N|N|N| +2467568|AAAAAAAAAPGKFCAA|2043-11-14|1726|7507|576|2043|6|11|14|4|2043|576|7507|Saturday|2043Q4|N|Y|N|2467555|2467858|2467203|2467476|N|N|N|N|N| +2467569|AAAAAAAABPGKFCAA|2043-11-15|1726|7507|576|2043|0|11|15|4|2043|576|7507|Sunday|2043Q4|N|N|N|2467555|2467858|2467204|2467477|N|N|N|N|N| +2467570|AAAAAAAACPGKFCAA|2043-11-16|1726|7507|576|2043|1|11|16|4|2043|576|7507|Monday|2043Q4|N|N|N|2467555|2467858|2467205|2467478|N|N|N|N|N| +2467571|AAAAAAAADPGKFCAA|2043-11-17|1726|7508|576|2043|2|11|17|4|2043|576|7508|Tuesday|2043Q4|N|N|N|2467555|2467858|2467206|2467479|N|N|N|N|N| +2467572|AAAAAAAAEPGKFCAA|2043-11-18|1726|7508|576|2043|3|11|18|4|2043|576|7508|Wednesday|2043Q4|N|N|N|2467555|2467858|2467207|2467480|N|N|N|N|N| +2467573|AAAAAAAAFPGKFCAA|2043-11-19|1726|7508|576|2043|4|11|19|4|2043|576|7508|Thursday|2043Q4|N|N|N|2467555|2467858|2467208|2467481|N|N|N|N|N| +2467574|AAAAAAAAGPGKFCAA|2043-11-20|1726|7508|576|2043|5|11|20|4|2043|576|7508|Friday|2043Q4|N|Y|N|2467555|2467858|2467209|2467482|N|N|N|N|N| +2467575|AAAAAAAAHPGKFCAA|2043-11-21|1726|7508|576|2043|6|11|21|4|2043|576|7508|Saturday|2043Q4|N|Y|N|2467555|2467858|2467210|2467483|N|N|N|N|N| +2467576|AAAAAAAAIPGKFCAA|2043-11-22|1726|7508|576|2043|0|11|22|4|2043|576|7508|Sunday|2043Q4|N|N|N|2467555|2467858|2467211|2467484|N|N|N|N|N| +2467577|AAAAAAAAJPGKFCAA|2043-11-23|1726|7508|576|2043|1|11|23|4|2043|576|7508|Monday|2043Q4|N|N|N|2467555|2467858|2467212|2467485|N|N|N|N|N| +2467578|AAAAAAAAKPGKFCAA|2043-11-24|1726|7509|576|2043|2|11|24|4|2043|576|7509|Tuesday|2043Q4|N|N|N|2467555|2467858|2467213|2467486|N|N|N|N|N| +2467579|AAAAAAAALPGKFCAA|2043-11-25|1726|7509|576|2043|3|11|25|4|2043|576|7509|Wednesday|2043Q4|N|N|N|2467555|2467858|2467214|2467487|N|N|N|N|N| +2467580|AAAAAAAAMPGKFCAA|2043-11-26|1726|7509|576|2043|4|11|26|4|2043|576|7509|Thursday|2043Q4|N|N|N|2467555|2467858|2467215|2467488|N|N|N|N|N| +2467581|AAAAAAAANPGKFCAA|2043-11-27|1726|7509|576|2043|5|11|27|4|2043|576|7509|Friday|2043Q4|N|Y|N|2467555|2467858|2467216|2467489|N|N|N|N|N| +2467582|AAAAAAAAOPGKFCAA|2043-11-28|1726|7509|576|2043|6|11|28|4|2043|576|7509|Saturday|2043Q4|N|Y|N|2467555|2467858|2467217|2467490|N|N|N|N|N| +2467583|AAAAAAAAPPGKFCAA|2043-11-29|1726|7509|576|2043|0|11|29|4|2043|576|7509|Sunday|2043Q4|N|N|N|2467555|2467858|2467218|2467491|N|N|N|N|N| +2467584|AAAAAAAAAAHKFCAA|2043-11-30|1726|7509|576|2043|1|11|30|4|2043|576|7509|Monday|2043Q4|N|N|N|2467555|2467858|2467219|2467492|N|N|N|N|N| +2467585|AAAAAAAABAHKFCAA|2043-12-01|1727|7510|577|2043|2|12|1|4|2043|577|7510|Tuesday|2043Q4|N|N|N|2467585|2467918|2467220|2467493|N|N|N|N|N| +2467586|AAAAAAAACAHKFCAA|2043-12-02|1727|7510|577|2043|3|12|2|4|2043|577|7510|Wednesday|2043Q4|N|N|N|2467585|2467918|2467221|2467494|N|N|N|N|N| +2467587|AAAAAAAADAHKFCAA|2043-12-03|1727|7510|577|2043|4|12|3|4|2043|577|7510|Thursday|2043Q4|N|N|N|2467585|2467918|2467222|2467495|N|N|N|N|N| +2467588|AAAAAAAAEAHKFCAA|2043-12-04|1727|7510|577|2043|5|12|4|4|2043|577|7510|Friday|2043Q4|N|Y|N|2467585|2467918|2467223|2467496|N|N|N|N|N| +2467589|AAAAAAAAFAHKFCAA|2043-12-05|1727|7510|577|2043|6|12|5|4|2043|577|7510|Saturday|2043Q4|N|Y|N|2467585|2467918|2467224|2467497|N|N|N|N|N| +2467590|AAAAAAAAGAHKFCAA|2043-12-06|1727|7510|577|2043|0|12|6|4|2043|577|7510|Sunday|2043Q4|N|N|N|2467585|2467918|2467225|2467498|N|N|N|N|N| +2467591|AAAAAAAAHAHKFCAA|2043-12-07|1727|7510|577|2043|1|12|7|4|2043|577|7510|Monday|2043Q4|N|N|N|2467585|2467918|2467226|2467499|N|N|N|N|N| +2467592|AAAAAAAAIAHKFCAA|2043-12-08|1727|7511|577|2043|2|12|8|4|2043|577|7511|Tuesday|2043Q4|N|N|N|2467585|2467918|2467227|2467500|N|N|N|N|N| +2467593|AAAAAAAAJAHKFCAA|2043-12-09|1727|7511|577|2043|3|12|9|4|2043|577|7511|Wednesday|2043Q4|N|N|N|2467585|2467918|2467228|2467501|N|N|N|N|N| +2467594|AAAAAAAAKAHKFCAA|2043-12-10|1727|7511|577|2043|4|12|10|4|2043|577|7511|Thursday|2043Q4|N|N|N|2467585|2467918|2467229|2467502|N|N|N|N|N| +2467595|AAAAAAAALAHKFCAA|2043-12-11|1727|7511|577|2043|5|12|11|4|2043|577|7511|Friday|2043Q4|N|Y|N|2467585|2467918|2467230|2467503|N|N|N|N|N| +2467596|AAAAAAAAMAHKFCAA|2043-12-12|1727|7511|577|2043|6|12|12|4|2043|577|7511|Saturday|2043Q4|N|Y|N|2467585|2467918|2467231|2467504|N|N|N|N|N| +2467597|AAAAAAAANAHKFCAA|2043-12-13|1727|7511|577|2043|0|12|13|4|2043|577|7511|Sunday|2043Q4|N|N|N|2467585|2467918|2467232|2467505|N|N|N|N|N| +2467598|AAAAAAAAOAHKFCAA|2043-12-14|1727|7511|577|2043|1|12|14|4|2043|577|7511|Monday|2043Q4|N|N|N|2467585|2467918|2467233|2467506|N|N|N|N|N| +2467599|AAAAAAAAPAHKFCAA|2043-12-15|1727|7512|577|2043|2|12|15|4|2043|577|7512|Tuesday|2043Q4|N|N|N|2467585|2467918|2467234|2467507|N|N|N|N|N| +2467600|AAAAAAAAABHKFCAA|2043-12-16|1727|7512|577|2043|3|12|16|4|2043|577|7512|Wednesday|2043Q4|N|N|N|2467585|2467918|2467235|2467508|N|N|N|N|N| +2467601|AAAAAAAABBHKFCAA|2043-12-17|1727|7512|577|2043|4|12|17|4|2043|577|7512|Thursday|2043Q4|N|N|N|2467585|2467918|2467236|2467509|N|N|N|N|N| +2467602|AAAAAAAACBHKFCAA|2043-12-18|1727|7512|577|2043|5|12|18|4|2043|577|7512|Friday|2043Q4|N|Y|N|2467585|2467918|2467237|2467510|N|N|N|N|N| +2467603|AAAAAAAADBHKFCAA|2043-12-19|1727|7512|577|2043|6|12|19|4|2043|577|7512|Saturday|2043Q4|N|Y|N|2467585|2467918|2467238|2467511|N|N|N|N|N| +2467604|AAAAAAAAEBHKFCAA|2043-12-20|1727|7512|577|2043|0|12|20|4|2043|577|7512|Sunday|2043Q4|N|N|N|2467585|2467918|2467239|2467512|N|N|N|N|N| +2467605|AAAAAAAAFBHKFCAA|2043-12-21|1727|7512|577|2043|1|12|21|4|2043|577|7512|Monday|2043Q4|N|N|N|2467585|2467918|2467240|2467513|N|N|N|N|N| +2467606|AAAAAAAAGBHKFCAA|2043-12-22|1727|7513|577|2043|2|12|22|4|2043|577|7513|Tuesday|2043Q4|N|N|N|2467585|2467918|2467241|2467514|N|N|N|N|N| +2467607|AAAAAAAAHBHKFCAA|2043-12-23|1727|7513|577|2043|3|12|23|4|2043|577|7513|Wednesday|2043Q4|N|N|N|2467585|2467918|2467242|2467515|N|N|N|N|N| +2467608|AAAAAAAAIBHKFCAA|2043-12-24|1727|7513|577|2043|4|12|24|4|2043|577|7513|Thursday|2043Q4|N|N|N|2467585|2467918|2467243|2467516|N|N|N|N|N| +2467609|AAAAAAAAJBHKFCAA|2043-12-25|1727|7513|577|2043|5|12|25|4|2043|577|7513|Friday|2043Q4|N|Y|N|2467585|2467918|2467244|2467517|N|N|N|N|N| +2467610|AAAAAAAAKBHKFCAA|2043-12-26|1727|7513|577|2043|6|12|26|4|2043|577|7513|Saturday|2043Q4|Y|Y|N|2467585|2467918|2467245|2467518|N|N|N|N|N| +2467611|AAAAAAAALBHKFCAA|2043-12-27|1727|7513|577|2043|0|12|27|4|2043|577|7513|Sunday|2043Q4|N|N|Y|2467585|2467918|2467246|2467519|N|N|N|N|N| +2467612|AAAAAAAAMBHKFCAA|2043-12-28|1727|7513|577|2043|1|12|28|4|2043|577|7513|Monday|2043Q4|N|N|N|2467585|2467918|2467247|2467520|N|N|N|N|N| +2467613|AAAAAAAANBHKFCAA|2043-12-29|1727|7514|577|2043|2|12|29|4|2043|577|7514|Tuesday|2043Q4|N|N|N|2467585|2467918|2467248|2467521|N|N|N|N|N| +2467614|AAAAAAAAOBHKFCAA|2043-12-30|1727|7514|577|2043|3|12|30|4|2043|577|7514|Wednesday|2043Q4|N|N|N|2467585|2467918|2467249|2467522|N|N|N|N|N| +2467615|AAAAAAAAPBHKFCAA|2043-12-31|1727|7514|577|2043|4|12|31|4|2043|577|7514|Thursday|2043Q4|N|N|N|2467585|2467918|2467250|2467523|N|N|N|N|N| +2467616|AAAAAAAAACHKFCAA|2044-01-01|1728|7514|577|2044|5|1|1|1|2044|577|7514|Friday|2044Q1|Y|Y|N|2467616|2467615|2467251|2467524|N|N|N|N|N| +2467617|AAAAAAAABCHKFCAA|2044-01-02|1728|7514|577|2044|6|1|2|1|2044|577|7514|Saturday|2044Q1|N|Y|Y|2467616|2467615|2467252|2467525|N|N|N|N|N| +2467618|AAAAAAAACCHKFCAA|2044-01-03|1728|7514|577|2044|0|1|3|1|2044|577|7514|Sunday|2044Q1|N|N|N|2467616|2467615|2467253|2467526|N|N|N|N|N| +2467619|AAAAAAAADCHKFCAA|2044-01-04|1728|7514|577|2044|1|1|4|1|2044|577|7514|Monday|2044Q1|N|N|N|2467616|2467615|2467254|2467527|N|N|N|N|N| +2467620|AAAAAAAAECHKFCAA|2044-01-05|1728|7515|577|2044|2|1|5|1|2044|577|7515|Tuesday|2044Q1|N|N|N|2467616|2467615|2467255|2467528|N|N|N|N|N| +2467621|AAAAAAAAFCHKFCAA|2044-01-06|1728|7515|577|2044|3|1|6|1|2044|577|7515|Wednesday|2044Q1|N|N|N|2467616|2467615|2467256|2467529|N|N|N|N|N| +2467622|AAAAAAAAGCHKFCAA|2044-01-07|1728|7515|577|2044|4|1|7|1|2044|577|7515|Thursday|2044Q1|N|N|N|2467616|2467615|2467257|2467530|N|N|N|N|N| +2467623|AAAAAAAAHCHKFCAA|2044-01-08|1728|7515|577|2044|5|1|8|1|2044|577|7515|Friday|2044Q1|N|Y|N|2467616|2467615|2467258|2467531|N|N|N|N|N| +2467624|AAAAAAAAICHKFCAA|2044-01-09|1728|7515|577|2044|6|1|9|1|2044|577|7515|Saturday|2044Q1|N|Y|N|2467616|2467615|2467259|2467532|N|N|N|N|N| +2467625|AAAAAAAAJCHKFCAA|2044-01-10|1728|7515|577|2044|0|1|10|1|2044|577|7515|Sunday|2044Q1|N|N|N|2467616|2467615|2467260|2467533|N|N|N|N|N| +2467626|AAAAAAAAKCHKFCAA|2044-01-11|1728|7515|577|2044|1|1|11|1|2044|577|7515|Monday|2044Q1|N|N|N|2467616|2467615|2467261|2467534|N|N|N|N|N| +2467627|AAAAAAAALCHKFCAA|2044-01-12|1728|7516|577|2044|2|1|12|1|2044|577|7516|Tuesday|2044Q1|N|N|N|2467616|2467615|2467262|2467535|N|N|N|N|N| +2467628|AAAAAAAAMCHKFCAA|2044-01-13|1728|7516|577|2044|3|1|13|1|2044|577|7516|Wednesday|2044Q1|N|N|N|2467616|2467615|2467263|2467536|N|N|N|N|N| +2467629|AAAAAAAANCHKFCAA|2044-01-14|1728|7516|577|2044|4|1|14|1|2044|577|7516|Thursday|2044Q1|N|N|N|2467616|2467615|2467264|2467537|N|N|N|N|N| +2467630|AAAAAAAAOCHKFCAA|2044-01-15|1728|7516|577|2044|5|1|15|1|2044|577|7516|Friday|2044Q1|N|Y|N|2467616|2467615|2467265|2467538|N|N|N|N|N| +2467631|AAAAAAAAPCHKFCAA|2044-01-16|1728|7516|577|2044|6|1|16|1|2044|577|7516|Saturday|2044Q1|N|Y|N|2467616|2467615|2467266|2467539|N|N|N|N|N| +2467632|AAAAAAAAADHKFCAA|2044-01-17|1728|7516|577|2044|0|1|17|1|2044|577|7516|Sunday|2044Q1|N|N|N|2467616|2467615|2467267|2467540|N|N|N|N|N| +2467633|AAAAAAAABDHKFCAA|2044-01-18|1728|7516|577|2044|1|1|18|1|2044|577|7516|Monday|2044Q1|N|N|N|2467616|2467615|2467268|2467541|N|N|N|N|N| +2467634|AAAAAAAACDHKFCAA|2044-01-19|1728|7517|577|2044|2|1|19|1|2044|577|7517|Tuesday|2044Q1|N|N|N|2467616|2467615|2467269|2467542|N|N|N|N|N| +2467635|AAAAAAAADDHKFCAA|2044-01-20|1728|7517|577|2044|3|1|20|1|2044|577|7517|Wednesday|2044Q1|N|N|N|2467616|2467615|2467270|2467543|N|N|N|N|N| +2467636|AAAAAAAAEDHKFCAA|2044-01-21|1728|7517|577|2044|4|1|21|1|2044|577|7517|Thursday|2044Q1|N|N|N|2467616|2467615|2467271|2467544|N|N|N|N|N| +2467637|AAAAAAAAFDHKFCAA|2044-01-22|1728|7517|577|2044|5|1|22|1|2044|577|7517|Friday|2044Q1|N|Y|N|2467616|2467615|2467272|2467545|N|N|N|N|N| +2467638|AAAAAAAAGDHKFCAA|2044-01-23|1728|7517|577|2044|6|1|23|1|2044|577|7517|Saturday|2044Q1|N|Y|N|2467616|2467615|2467273|2467546|N|N|N|N|N| +2467639|AAAAAAAAHDHKFCAA|2044-01-24|1728|7517|577|2044|0|1|24|1|2044|577|7517|Sunday|2044Q1|N|N|N|2467616|2467615|2467274|2467547|N|N|N|N|N| +2467640|AAAAAAAAIDHKFCAA|2044-01-25|1728|7517|577|2044|1|1|25|1|2044|577|7517|Monday|2044Q1|N|N|N|2467616|2467615|2467275|2467548|N|N|N|N|N| +2467641|AAAAAAAAJDHKFCAA|2044-01-26|1728|7518|577|2044|2|1|26|1|2044|577|7518|Tuesday|2044Q1|N|N|N|2467616|2467615|2467276|2467549|N|N|N|N|N| +2467642|AAAAAAAAKDHKFCAA|2044-01-27|1728|7518|577|2044|3|1|27|1|2044|577|7518|Wednesday|2044Q1|N|N|N|2467616|2467615|2467277|2467550|N|N|N|N|N| +2467643|AAAAAAAALDHKFCAA|2044-01-28|1728|7518|577|2044|4|1|28|1|2044|577|7518|Thursday|2044Q1|N|N|N|2467616|2467615|2467278|2467551|N|N|N|N|N| +2467644|AAAAAAAAMDHKFCAA|2044-01-29|1728|7518|577|2044|5|1|29|1|2044|577|7518|Friday|2044Q1|N|Y|N|2467616|2467615|2467279|2467552|N|N|N|N|N| +2467645|AAAAAAAANDHKFCAA|2044-01-30|1728|7518|577|2044|6|1|30|1|2044|577|7518|Saturday|2044Q1|N|Y|N|2467616|2467615|2467280|2467553|N|N|N|N|N| +2467646|AAAAAAAAODHKFCAA|2044-01-31|1728|7518|577|2044|0|1|31|1|2044|577|7518|Sunday|2044Q1|N|N|N|2467616|2467615|2467281|2467554|N|N|N|N|N| +2467647|AAAAAAAAPDHKFCAA|2044-02-01|1729|7518|577|2044|1|2|1|1|2044|577|7518|Monday|2044Q1|N|N|N|2467647|2467677|2467282|2467555|N|N|N|N|N| +2467648|AAAAAAAAAEHKFCAA|2044-02-02|1729|7519|577|2044|2|2|2|1|2044|577|7519|Tuesday|2044Q1|N|N|N|2467647|2467677|2467283|2467556|N|N|N|N|N| +2467649|AAAAAAAABEHKFCAA|2044-02-03|1729|7519|577|2044|3|2|3|1|2044|577|7519|Wednesday|2044Q1|N|N|N|2467647|2467677|2467284|2467557|N|N|N|N|N| +2467650|AAAAAAAACEHKFCAA|2044-02-04|1729|7519|577|2044|4|2|4|1|2044|577|7519|Thursday|2044Q1|N|N|N|2467647|2467677|2467285|2467558|N|N|N|N|N| +2467651|AAAAAAAADEHKFCAA|2044-02-05|1729|7519|577|2044|5|2|5|1|2044|577|7519|Friday|2044Q1|N|Y|N|2467647|2467677|2467286|2467559|N|N|N|N|N| +2467652|AAAAAAAAEEHKFCAA|2044-02-06|1729|7519|577|2044|6|2|6|1|2044|577|7519|Saturday|2044Q1|N|Y|N|2467647|2467677|2467287|2467560|N|N|N|N|N| +2467653|AAAAAAAAFEHKFCAA|2044-02-07|1729|7519|577|2044|0|2|7|1|2044|577|7519|Sunday|2044Q1|N|N|N|2467647|2467677|2467288|2467561|N|N|N|N|N| +2467654|AAAAAAAAGEHKFCAA|2044-02-08|1729|7519|577|2044|1|2|8|1|2044|577|7519|Monday|2044Q1|N|N|N|2467647|2467677|2467289|2467562|N|N|N|N|N| +2467655|AAAAAAAAHEHKFCAA|2044-02-09|1729|7520|577|2044|2|2|9|1|2044|577|7520|Tuesday|2044Q1|N|N|N|2467647|2467677|2467290|2467563|N|N|N|N|N| +2467656|AAAAAAAAIEHKFCAA|2044-02-10|1729|7520|577|2044|3|2|10|1|2044|577|7520|Wednesday|2044Q1|N|N|N|2467647|2467677|2467291|2467564|N|N|N|N|N| +2467657|AAAAAAAAJEHKFCAA|2044-02-11|1729|7520|577|2044|4|2|11|1|2044|577|7520|Thursday|2044Q1|N|N|N|2467647|2467677|2467292|2467565|N|N|N|N|N| +2467658|AAAAAAAAKEHKFCAA|2044-02-12|1729|7520|577|2044|5|2|12|1|2044|577|7520|Friday|2044Q1|N|Y|N|2467647|2467677|2467293|2467566|N|N|N|N|N| +2467659|AAAAAAAALEHKFCAA|2044-02-13|1729|7520|577|2044|6|2|13|1|2044|577|7520|Saturday|2044Q1|N|Y|N|2467647|2467677|2467294|2467567|N|N|N|N|N| +2467660|AAAAAAAAMEHKFCAA|2044-02-14|1729|7520|577|2044|0|2|14|1|2044|577|7520|Sunday|2044Q1|N|N|N|2467647|2467677|2467295|2467568|N|N|N|N|N| +2467661|AAAAAAAANEHKFCAA|2044-02-15|1729|7520|577|2044|1|2|15|1|2044|577|7520|Monday|2044Q1|N|N|N|2467647|2467677|2467296|2467569|N|N|N|N|N| +2467662|AAAAAAAAOEHKFCAA|2044-02-16|1729|7521|577|2044|2|2|16|1|2044|577|7521|Tuesday|2044Q1|N|N|N|2467647|2467677|2467297|2467570|N|N|N|N|N| +2467663|AAAAAAAAPEHKFCAA|2044-02-17|1729|7521|577|2044|3|2|17|1|2044|577|7521|Wednesday|2044Q1|N|N|N|2467647|2467677|2467298|2467571|N|N|N|N|N| +2467664|AAAAAAAAAFHKFCAA|2044-02-18|1729|7521|577|2044|4|2|18|1|2044|577|7521|Thursday|2044Q1|N|N|N|2467647|2467677|2467299|2467572|N|N|N|N|N| +2467665|AAAAAAAABFHKFCAA|2044-02-19|1729|7521|577|2044|5|2|19|1|2044|577|7521|Friday|2044Q1|N|Y|N|2467647|2467677|2467300|2467573|N|N|N|N|N| +2467666|AAAAAAAACFHKFCAA|2044-02-20|1729|7521|577|2044|6|2|20|1|2044|577|7521|Saturday|2044Q1|N|Y|N|2467647|2467677|2467301|2467574|N|N|N|N|N| +2467667|AAAAAAAADFHKFCAA|2044-02-21|1729|7521|577|2044|0|2|21|1|2044|577|7521|Sunday|2044Q1|N|N|N|2467647|2467677|2467302|2467575|N|N|N|N|N| +2467668|AAAAAAAAEFHKFCAA|2044-02-22|1729|7521|577|2044|1|2|22|1|2044|577|7521|Monday|2044Q1|N|N|N|2467647|2467677|2467303|2467576|N|N|N|N|N| +2467669|AAAAAAAAFFHKFCAA|2044-02-23|1729|7522|577|2044|2|2|23|1|2044|577|7522|Tuesday|2044Q1|N|N|N|2467647|2467677|2467304|2467577|N|N|N|N|N| +2467670|AAAAAAAAGFHKFCAA|2044-02-24|1729|7522|577|2044|3|2|24|1|2044|577|7522|Wednesday|2044Q1|N|N|N|2467647|2467677|2467305|2467578|N|N|N|N|N| +2467671|AAAAAAAAHFHKFCAA|2044-02-25|1729|7522|577|2044|4|2|25|1|2044|577|7522|Thursday|2044Q1|N|N|N|2467647|2467677|2467306|2467579|N|N|N|N|N| +2467672|AAAAAAAAIFHKFCAA|2044-02-26|1729|7522|577|2044|5|2|26|1|2044|577|7522|Friday|2044Q1|N|Y|N|2467647|2467677|2467307|2467580|N|N|N|N|N| +2467673|AAAAAAAAJFHKFCAA|2044-02-27|1729|7522|577|2044|6|2|27|1|2044|577|7522|Saturday|2044Q1|N|Y|N|2467647|2467677|2467308|2467581|N|N|N|N|N| +2467674|AAAAAAAAKFHKFCAA|2044-02-28|1729|7522|577|2044|0|2|28|1|2044|577|7522|Sunday|2044Q1|N|N|N|2467647|2467677|2467309|2467582|N|N|N|N|N| +2467675|AAAAAAAALFHKFCAA|2044-02-29|1729|7522|577|2044|1|2|29|1|2044|577|7522|Monday|2044Q1|N|N|N|2467647|2467677|2467309|2467583|N|N|N|N|N| +2467676|AAAAAAAAMFHKFCAA|2044-03-01|1730|7523|578|2044|2|3|1|1|2044|578|7523|Tuesday|2044Q1|N|N|N|2467676|2467735|2467310|2467584|N|N|N|N|N| +2467677|AAAAAAAANFHKFCAA|2044-03-02|1730|7523|578|2044|3|3|2|1|2044|578|7523|Wednesday|2044Q1|N|N|N|2467676|2467735|2467311|2467585|N|N|N|N|N| +2467678|AAAAAAAAOFHKFCAA|2044-03-03|1730|7523|578|2044|4|3|3|1|2044|578|7523|Thursday|2044Q1|N|N|N|2467676|2467735|2467312|2467586|N|N|N|N|N| +2467679|AAAAAAAAPFHKFCAA|2044-03-04|1730|7523|578|2044|5|3|4|1|2044|578|7523|Friday|2044Q1|N|Y|N|2467676|2467735|2467313|2467587|N|N|N|N|N| +2467680|AAAAAAAAAGHKFCAA|2044-03-05|1730|7523|578|2044|6|3|5|1|2044|578|7523|Saturday|2044Q1|N|Y|N|2467676|2467735|2467314|2467588|N|N|N|N|N| +2467681|AAAAAAAABGHKFCAA|2044-03-06|1730|7523|578|2044|0|3|6|1|2044|578|7523|Sunday|2044Q1|N|N|N|2467676|2467735|2467315|2467589|N|N|N|N|N| +2467682|AAAAAAAACGHKFCAA|2044-03-07|1730|7523|578|2044|1|3|7|1|2044|578|7523|Monday|2044Q1|N|N|N|2467676|2467735|2467316|2467590|N|N|N|N|N| +2467683|AAAAAAAADGHKFCAA|2044-03-08|1730|7524|578|2044|2|3|8|1|2044|578|7524|Tuesday|2044Q1|N|N|N|2467676|2467735|2467317|2467591|N|N|N|N|N| +2467684|AAAAAAAAEGHKFCAA|2044-03-09|1730|7524|578|2044|3|3|9|1|2044|578|7524|Wednesday|2044Q1|N|N|N|2467676|2467735|2467318|2467592|N|N|N|N|N| +2467685|AAAAAAAAFGHKFCAA|2044-03-10|1730|7524|578|2044|4|3|10|1|2044|578|7524|Thursday|2044Q1|N|N|N|2467676|2467735|2467319|2467593|N|N|N|N|N| +2467686|AAAAAAAAGGHKFCAA|2044-03-11|1730|7524|578|2044|5|3|11|1|2044|578|7524|Friday|2044Q1|N|Y|N|2467676|2467735|2467320|2467594|N|N|N|N|N| +2467687|AAAAAAAAHGHKFCAA|2044-03-12|1730|7524|578|2044|6|3|12|1|2044|578|7524|Saturday|2044Q1|N|Y|N|2467676|2467735|2467321|2467595|N|N|N|N|N| +2467688|AAAAAAAAIGHKFCAA|2044-03-13|1730|7524|578|2044|0|3|13|1|2044|578|7524|Sunday|2044Q1|N|N|N|2467676|2467735|2467322|2467596|N|N|N|N|N| +2467689|AAAAAAAAJGHKFCAA|2044-03-14|1730|7524|578|2044|1|3|14|1|2044|578|7524|Monday|2044Q1|N|N|N|2467676|2467735|2467323|2467597|N|N|N|N|N| +2467690|AAAAAAAAKGHKFCAA|2044-03-15|1730|7525|578|2044|2|3|15|1|2044|578|7525|Tuesday|2044Q1|N|N|N|2467676|2467735|2467324|2467598|N|N|N|N|N| +2467691|AAAAAAAALGHKFCAA|2044-03-16|1730|7525|578|2044|3|3|16|1|2044|578|7525|Wednesday|2044Q1|N|N|N|2467676|2467735|2467325|2467599|N|N|N|N|N| +2467692|AAAAAAAAMGHKFCAA|2044-03-17|1730|7525|578|2044|4|3|17|1|2044|578|7525|Thursday|2044Q1|N|N|N|2467676|2467735|2467326|2467600|N|N|N|N|N| +2467693|AAAAAAAANGHKFCAA|2044-03-18|1730|7525|578|2044|5|3|18|1|2044|578|7525|Friday|2044Q1|N|Y|N|2467676|2467735|2467327|2467601|N|N|N|N|N| +2467694|AAAAAAAAOGHKFCAA|2044-03-19|1730|7525|578|2044|6|3|19|1|2044|578|7525|Saturday|2044Q1|N|Y|N|2467676|2467735|2467328|2467602|N|N|N|N|N| +2467695|AAAAAAAAPGHKFCAA|2044-03-20|1730|7525|578|2044|0|3|20|1|2044|578|7525|Sunday|2044Q1|N|N|N|2467676|2467735|2467329|2467603|N|N|N|N|N| +2467696|AAAAAAAAAHHKFCAA|2044-03-21|1730|7525|578|2044|1|3|21|1|2044|578|7525|Monday|2044Q1|N|N|N|2467676|2467735|2467330|2467604|N|N|N|N|N| +2467697|AAAAAAAABHHKFCAA|2044-03-22|1730|7526|578|2044|2|3|22|1|2044|578|7526|Tuesday|2044Q1|N|N|N|2467676|2467735|2467331|2467605|N|N|N|N|N| +2467698|AAAAAAAACHHKFCAA|2044-03-23|1730|7526|578|2044|3|3|23|1|2044|578|7526|Wednesday|2044Q1|N|N|N|2467676|2467735|2467332|2467606|N|N|N|N|N| +2467699|AAAAAAAADHHKFCAA|2044-03-24|1730|7526|578|2044|4|3|24|1|2044|578|7526|Thursday|2044Q1|N|N|N|2467676|2467735|2467333|2467607|N|N|N|N|N| +2467700|AAAAAAAAEHHKFCAA|2044-03-25|1730|7526|578|2044|5|3|25|1|2044|578|7526|Friday|2044Q1|N|Y|N|2467676|2467735|2467334|2467608|N|N|N|N|N| +2467701|AAAAAAAAFHHKFCAA|2044-03-26|1730|7526|578|2044|6|3|26|1|2044|578|7526|Saturday|2044Q1|N|Y|N|2467676|2467735|2467335|2467609|N|N|N|N|N| +2467702|AAAAAAAAGHHKFCAA|2044-03-27|1730|7526|578|2044|0|3|27|1|2044|578|7526|Sunday|2044Q1|N|N|N|2467676|2467735|2467336|2467610|N|N|N|N|N| +2467703|AAAAAAAAHHHKFCAA|2044-03-28|1730|7526|578|2044|1|3|28|1|2044|578|7526|Monday|2044Q1|N|N|N|2467676|2467735|2467337|2467611|N|N|N|N|N| +2467704|AAAAAAAAIHHKFCAA|2044-03-29|1730|7527|578|2044|2|3|29|1|2044|578|7527|Tuesday|2044Q1|N|N|N|2467676|2467735|2467338|2467612|N|N|N|N|N| +2467705|AAAAAAAAJHHKFCAA|2044-03-30|1730|7527|578|2044|3|3|30|1|2044|578|7527|Wednesday|2044Q1|N|N|N|2467676|2467735|2467339|2467613|N|N|N|N|N| +2467706|AAAAAAAAKHHKFCAA|2044-03-31|1730|7527|578|2044|4|3|31|1|2044|578|7527|Thursday|2044Q1|N|N|N|2467676|2467735|2467340|2467614|N|N|N|N|N| +2467707|AAAAAAAALHHKFCAA|2044-04-01|1731|7527|578|2044|5|4|1|2|2044|578|7527|Friday|2044Q2|N|Y|N|2467707|2467797|2467341|2467616|N|N|N|N|N| +2467708|AAAAAAAAMHHKFCAA|2044-04-02|1731|7527|578|2044|6|4|2|2|2044|578|7527|Saturday|2044Q2|N|Y|N|2467707|2467797|2467342|2467617|N|N|N|N|N| +2467709|AAAAAAAANHHKFCAA|2044-04-03|1731|7527|578|2044|0|4|3|2|2044|578|7527|Sunday|2044Q2|N|N|N|2467707|2467797|2467343|2467618|N|N|N|N|N| +2467710|AAAAAAAAOHHKFCAA|2044-04-04|1731|7527|578|2044|1|4|4|2|2044|578|7527|Monday|2044Q2|N|N|N|2467707|2467797|2467344|2467619|N|N|N|N|N| +2467711|AAAAAAAAPHHKFCAA|2044-04-05|1731|7528|578|2044|2|4|5|2|2044|578|7528|Tuesday|2044Q2|N|N|N|2467707|2467797|2467345|2467620|N|N|N|N|N| +2467712|AAAAAAAAAIHKFCAA|2044-04-06|1731|7528|578|2044|3|4|6|2|2044|578|7528|Wednesday|2044Q2|N|N|N|2467707|2467797|2467346|2467621|N|N|N|N|N| +2467713|AAAAAAAABIHKFCAA|2044-04-07|1731|7528|578|2044|4|4|7|2|2044|578|7528|Thursday|2044Q2|N|N|N|2467707|2467797|2467347|2467622|N|N|N|N|N| +2467714|AAAAAAAACIHKFCAA|2044-04-08|1731|7528|578|2044|5|4|8|2|2044|578|7528|Friday|2044Q2|N|Y|N|2467707|2467797|2467348|2467623|N|N|N|N|N| +2467715|AAAAAAAADIHKFCAA|2044-04-09|1731|7528|578|2044|6|4|9|2|2044|578|7528|Saturday|2044Q2|N|Y|N|2467707|2467797|2467349|2467624|N|N|N|N|N| +2467716|AAAAAAAAEIHKFCAA|2044-04-10|1731|7528|578|2044|0|4|10|2|2044|578|7528|Sunday|2044Q2|N|N|N|2467707|2467797|2467350|2467625|N|N|N|N|N| +2467717|AAAAAAAAFIHKFCAA|2044-04-11|1731|7528|578|2044|1|4|11|2|2044|578|7528|Monday|2044Q2|N|N|N|2467707|2467797|2467351|2467626|N|N|N|N|N| +2467718|AAAAAAAAGIHKFCAA|2044-04-12|1731|7529|578|2044|2|4|12|2|2044|578|7529|Tuesday|2044Q2|N|N|N|2467707|2467797|2467352|2467627|N|N|N|N|N| +2467719|AAAAAAAAHIHKFCAA|2044-04-13|1731|7529|578|2044|3|4|13|2|2044|578|7529|Wednesday|2044Q2|N|N|N|2467707|2467797|2467353|2467628|N|N|N|N|N| +2467720|AAAAAAAAIIHKFCAA|2044-04-14|1731|7529|578|2044|4|4|14|2|2044|578|7529|Thursday|2044Q2|N|N|N|2467707|2467797|2467354|2467629|N|N|N|N|N| +2467721|AAAAAAAAJIHKFCAA|2044-04-15|1731|7529|578|2044|5|4|15|2|2044|578|7529|Friday|2044Q2|N|Y|N|2467707|2467797|2467355|2467630|N|N|N|N|N| +2467722|AAAAAAAAKIHKFCAA|2044-04-16|1731|7529|578|2044|6|4|16|2|2044|578|7529|Saturday|2044Q2|N|Y|N|2467707|2467797|2467356|2467631|N|N|N|N|N| +2467723|AAAAAAAALIHKFCAA|2044-04-17|1731|7529|578|2044|0|4|17|2|2044|578|7529|Sunday|2044Q2|N|N|N|2467707|2467797|2467357|2467632|N|N|N|N|N| +2467724|AAAAAAAAMIHKFCAA|2044-04-18|1731|7529|578|2044|1|4|18|2|2044|578|7529|Monday|2044Q2|N|N|N|2467707|2467797|2467358|2467633|N|N|N|N|N| +2467725|AAAAAAAANIHKFCAA|2044-04-19|1731|7530|578|2044|2|4|19|2|2044|578|7530|Tuesday|2044Q2|N|N|N|2467707|2467797|2467359|2467634|N|N|N|N|N| +2467726|AAAAAAAAOIHKFCAA|2044-04-20|1731|7530|578|2044|3|4|20|2|2044|578|7530|Wednesday|2044Q2|N|N|N|2467707|2467797|2467360|2467635|N|N|N|N|N| +2467727|AAAAAAAAPIHKFCAA|2044-04-21|1731|7530|578|2044|4|4|21|2|2044|578|7530|Thursday|2044Q2|N|N|N|2467707|2467797|2467361|2467636|N|N|N|N|N| +2467728|AAAAAAAAAJHKFCAA|2044-04-22|1731|7530|578|2044|5|4|22|2|2044|578|7530|Friday|2044Q2|N|Y|N|2467707|2467797|2467362|2467637|N|N|N|N|N| +2467729|AAAAAAAABJHKFCAA|2044-04-23|1731|7530|578|2044|6|4|23|2|2044|578|7530|Saturday|2044Q2|N|Y|N|2467707|2467797|2467363|2467638|N|N|N|N|N| +2467730|AAAAAAAACJHKFCAA|2044-04-24|1731|7530|578|2044|0|4|24|2|2044|578|7530|Sunday|2044Q2|N|N|N|2467707|2467797|2467364|2467639|N|N|N|N|N| +2467731|AAAAAAAADJHKFCAA|2044-04-25|1731|7530|578|2044|1|4|25|2|2044|578|7530|Monday|2044Q2|N|N|N|2467707|2467797|2467365|2467640|N|N|N|N|N| +2467732|AAAAAAAAEJHKFCAA|2044-04-26|1731|7531|578|2044|2|4|26|2|2044|578|7531|Tuesday|2044Q2|N|N|N|2467707|2467797|2467366|2467641|N|N|N|N|N| +2467733|AAAAAAAAFJHKFCAA|2044-04-27|1731|7531|578|2044|3|4|27|2|2044|578|7531|Wednesday|2044Q2|N|N|N|2467707|2467797|2467367|2467642|N|N|N|N|N| +2467734|AAAAAAAAGJHKFCAA|2044-04-28|1731|7531|578|2044|4|4|28|2|2044|578|7531|Thursday|2044Q2|N|N|N|2467707|2467797|2467368|2467643|N|N|N|N|N| +2467735|AAAAAAAAHJHKFCAA|2044-04-29|1731|7531|578|2044|5|4|29|2|2044|578|7531|Friday|2044Q2|N|Y|N|2467707|2467797|2467369|2467644|N|N|N|N|N| +2467736|AAAAAAAAIJHKFCAA|2044-04-30|1731|7531|578|2044|6|4|30|2|2044|578|7531|Saturday|2044Q2|N|Y|N|2467707|2467797|2467370|2467645|N|N|N|N|N| +2467737|AAAAAAAAJJHKFCAA|2044-05-01|1732|7531|578|2044|0|5|1|2|2044|578|7531|Sunday|2044Q2|N|N|N|2467737|2467857|2467371|2467646|N|N|N|N|N| +2467738|AAAAAAAAKJHKFCAA|2044-05-02|1732|7531|578|2044|1|5|2|2|2044|578|7531|Monday|2044Q2|N|N|N|2467737|2467857|2467372|2467647|N|N|N|N|N| +2467739|AAAAAAAALJHKFCAA|2044-05-03|1732|7532|578|2044|2|5|3|2|2044|578|7532|Tuesday|2044Q2|N|N|N|2467737|2467857|2467373|2467648|N|N|N|N|N| +2467740|AAAAAAAAMJHKFCAA|2044-05-04|1732|7532|578|2044|3|5|4|2|2044|578|7532|Wednesday|2044Q2|N|N|N|2467737|2467857|2467374|2467649|N|N|N|N|N| +2467741|AAAAAAAANJHKFCAA|2044-05-05|1732|7532|578|2044|4|5|5|2|2044|578|7532|Thursday|2044Q2|N|N|N|2467737|2467857|2467375|2467650|N|N|N|N|N| +2467742|AAAAAAAAOJHKFCAA|2044-05-06|1732|7532|578|2044|5|5|6|2|2044|578|7532|Friday|2044Q2|N|Y|N|2467737|2467857|2467376|2467651|N|N|N|N|N| +2467743|AAAAAAAAPJHKFCAA|2044-05-07|1732|7532|578|2044|6|5|7|2|2044|578|7532|Saturday|2044Q2|N|Y|N|2467737|2467857|2467377|2467652|N|N|N|N|N| +2467744|AAAAAAAAAKHKFCAA|2044-05-08|1732|7532|578|2044|0|5|8|2|2044|578|7532|Sunday|2044Q2|N|N|N|2467737|2467857|2467378|2467653|N|N|N|N|N| +2467745|AAAAAAAABKHKFCAA|2044-05-09|1732|7532|578|2044|1|5|9|2|2044|578|7532|Monday|2044Q2|N|N|N|2467737|2467857|2467379|2467654|N|N|N|N|N| +2467746|AAAAAAAACKHKFCAA|2044-05-10|1732|7533|578|2044|2|5|10|2|2044|578|7533|Tuesday|2044Q2|N|N|N|2467737|2467857|2467380|2467655|N|N|N|N|N| +2467747|AAAAAAAADKHKFCAA|2044-05-11|1732|7533|578|2044|3|5|11|2|2044|578|7533|Wednesday|2044Q2|N|N|N|2467737|2467857|2467381|2467656|N|N|N|N|N| +2467748|AAAAAAAAEKHKFCAA|2044-05-12|1732|7533|578|2044|4|5|12|2|2044|578|7533|Thursday|2044Q2|N|N|N|2467737|2467857|2467382|2467657|N|N|N|N|N| +2467749|AAAAAAAAFKHKFCAA|2044-05-13|1732|7533|578|2044|5|5|13|2|2044|578|7533|Friday|2044Q2|N|Y|N|2467737|2467857|2467383|2467658|N|N|N|N|N| +2467750|AAAAAAAAGKHKFCAA|2044-05-14|1732|7533|578|2044|6|5|14|2|2044|578|7533|Saturday|2044Q2|N|Y|N|2467737|2467857|2467384|2467659|N|N|N|N|N| +2467751|AAAAAAAAHKHKFCAA|2044-05-15|1732|7533|578|2044|0|5|15|2|2044|578|7533|Sunday|2044Q2|N|N|N|2467737|2467857|2467385|2467660|N|N|N|N|N| +2467752|AAAAAAAAIKHKFCAA|2044-05-16|1732|7533|578|2044|1|5|16|2|2044|578|7533|Monday|2044Q2|N|N|N|2467737|2467857|2467386|2467661|N|N|N|N|N| +2467753|AAAAAAAAJKHKFCAA|2044-05-17|1732|7534|578|2044|2|5|17|2|2044|578|7534|Tuesday|2044Q2|N|N|N|2467737|2467857|2467387|2467662|N|N|N|N|N| +2467754|AAAAAAAAKKHKFCAA|2044-05-18|1732|7534|578|2044|3|5|18|2|2044|578|7534|Wednesday|2044Q2|N|N|N|2467737|2467857|2467388|2467663|N|N|N|N|N| +2467755|AAAAAAAALKHKFCAA|2044-05-19|1732|7534|578|2044|4|5|19|2|2044|578|7534|Thursday|2044Q2|N|N|N|2467737|2467857|2467389|2467664|N|N|N|N|N| +2467756|AAAAAAAAMKHKFCAA|2044-05-20|1732|7534|578|2044|5|5|20|2|2044|578|7534|Friday|2044Q2|N|Y|N|2467737|2467857|2467390|2467665|N|N|N|N|N| +2467757|AAAAAAAANKHKFCAA|2044-05-21|1732|7534|578|2044|6|5|21|2|2044|578|7534|Saturday|2044Q2|N|Y|N|2467737|2467857|2467391|2467666|N|N|N|N|N| +2467758|AAAAAAAAOKHKFCAA|2044-05-22|1732|7534|578|2044|0|5|22|2|2044|578|7534|Sunday|2044Q2|N|N|N|2467737|2467857|2467392|2467667|N|N|N|N|N| +2467759|AAAAAAAAPKHKFCAA|2044-05-23|1732|7534|578|2044|1|5|23|2|2044|578|7534|Monday|2044Q2|N|N|N|2467737|2467857|2467393|2467668|N|N|N|N|N| +2467760|AAAAAAAAALHKFCAA|2044-05-24|1732|7535|578|2044|2|5|24|2|2044|578|7535|Tuesday|2044Q2|N|N|N|2467737|2467857|2467394|2467669|N|N|N|N|N| +2467761|AAAAAAAABLHKFCAA|2044-05-25|1732|7535|578|2044|3|5|25|2|2044|578|7535|Wednesday|2044Q2|N|N|N|2467737|2467857|2467395|2467670|N|N|N|N|N| +2467762|AAAAAAAACLHKFCAA|2044-05-26|1732|7535|578|2044|4|5|26|2|2044|578|7535|Thursday|2044Q2|N|N|N|2467737|2467857|2467396|2467671|N|N|N|N|N| +2467763|AAAAAAAADLHKFCAA|2044-05-27|1732|7535|578|2044|5|5|27|2|2044|578|7535|Friday|2044Q2|N|Y|N|2467737|2467857|2467397|2467672|N|N|N|N|N| +2467764|AAAAAAAAELHKFCAA|2044-05-28|1732|7535|578|2044|6|5|28|2|2044|578|7535|Saturday|2044Q2|N|Y|N|2467737|2467857|2467398|2467673|N|N|N|N|N| +2467765|AAAAAAAAFLHKFCAA|2044-05-29|1732|7535|578|2044|0|5|29|2|2044|578|7535|Sunday|2044Q2|N|N|N|2467737|2467857|2467399|2467674|N|N|N|N|N| +2467766|AAAAAAAAGLHKFCAA|2044-05-30|1732|7535|578|2044|1|5|30|2|2044|578|7535|Monday|2044Q2|N|N|N|2467737|2467857|2467400|2467675|N|N|N|N|N| +2467767|AAAAAAAAHLHKFCAA|2044-05-31|1732|7536|578|2044|2|5|31|2|2044|578|7536|Tuesday|2044Q2|N|N|N|2467737|2467857|2467401|2467676|N|N|N|N|N| +2467768|AAAAAAAAILHKFCAA|2044-06-01|1733|7536|579|2044|3|6|1|2|2044|579|7536|Wednesday|2044Q2|N|N|N|2467768|2467919|2467402|2467677|N|N|N|N|N| +2467769|AAAAAAAAJLHKFCAA|2044-06-02|1733|7536|579|2044|4|6|2|2|2044|579|7536|Thursday|2044Q2|N|N|N|2467768|2467919|2467403|2467678|N|N|N|N|N| +2467770|AAAAAAAAKLHKFCAA|2044-06-03|1733|7536|579|2044|5|6|3|2|2044|579|7536|Friday|2044Q2|N|Y|N|2467768|2467919|2467404|2467679|N|N|N|N|N| +2467771|AAAAAAAALLHKFCAA|2044-06-04|1733|7536|579|2044|6|6|4|2|2044|579|7536|Saturday|2044Q2|N|Y|N|2467768|2467919|2467405|2467680|N|N|N|N|N| +2467772|AAAAAAAAMLHKFCAA|2044-06-05|1733|7536|579|2044|0|6|5|2|2044|579|7536|Sunday|2044Q2|N|N|N|2467768|2467919|2467406|2467681|N|N|N|N|N| +2467773|AAAAAAAANLHKFCAA|2044-06-06|1733|7536|579|2044|1|6|6|2|2044|579|7536|Monday|2044Q2|N|N|N|2467768|2467919|2467407|2467682|N|N|N|N|N| +2467774|AAAAAAAAOLHKFCAA|2044-06-07|1733|7537|579|2044|2|6|7|2|2044|579|7537|Tuesday|2044Q2|N|N|N|2467768|2467919|2467408|2467683|N|N|N|N|N| +2467775|AAAAAAAAPLHKFCAA|2044-06-08|1733|7537|579|2044|3|6|8|2|2044|579|7537|Wednesday|2044Q2|N|N|N|2467768|2467919|2467409|2467684|N|N|N|N|N| +2467776|AAAAAAAAAMHKFCAA|2044-06-09|1733|7537|579|2044|4|6|9|2|2044|579|7537|Thursday|2044Q2|N|N|N|2467768|2467919|2467410|2467685|N|N|N|N|N| +2467777|AAAAAAAABMHKFCAA|2044-06-10|1733|7537|579|2044|5|6|10|2|2044|579|7537|Friday|2044Q2|N|Y|N|2467768|2467919|2467411|2467686|N|N|N|N|N| +2467778|AAAAAAAACMHKFCAA|2044-06-11|1733|7537|579|2044|6|6|11|2|2044|579|7537|Saturday|2044Q2|N|Y|N|2467768|2467919|2467412|2467687|N|N|N|N|N| +2467779|AAAAAAAADMHKFCAA|2044-06-12|1733|7537|579|2044|0|6|12|2|2044|579|7537|Sunday|2044Q2|N|N|N|2467768|2467919|2467413|2467688|N|N|N|N|N| +2467780|AAAAAAAAEMHKFCAA|2044-06-13|1733|7537|579|2044|1|6|13|2|2044|579|7537|Monday|2044Q2|N|N|N|2467768|2467919|2467414|2467689|N|N|N|N|N| +2467781|AAAAAAAAFMHKFCAA|2044-06-14|1733|7538|579|2044|2|6|14|2|2044|579|7538|Tuesday|2044Q2|N|N|N|2467768|2467919|2467415|2467690|N|N|N|N|N| +2467782|AAAAAAAAGMHKFCAA|2044-06-15|1733|7538|579|2044|3|6|15|2|2044|579|7538|Wednesday|2044Q2|N|N|N|2467768|2467919|2467416|2467691|N|N|N|N|N| +2467783|AAAAAAAAHMHKFCAA|2044-06-16|1733|7538|579|2044|4|6|16|2|2044|579|7538|Thursday|2044Q2|N|N|N|2467768|2467919|2467417|2467692|N|N|N|N|N| +2467784|AAAAAAAAIMHKFCAA|2044-06-17|1733|7538|579|2044|5|6|17|2|2044|579|7538|Friday|2044Q2|N|Y|N|2467768|2467919|2467418|2467693|N|N|N|N|N| +2467785|AAAAAAAAJMHKFCAA|2044-06-18|1733|7538|579|2044|6|6|18|2|2044|579|7538|Saturday|2044Q2|N|Y|N|2467768|2467919|2467419|2467694|N|N|N|N|N| +2467786|AAAAAAAAKMHKFCAA|2044-06-19|1733|7538|579|2044|0|6|19|2|2044|579|7538|Sunday|2044Q2|N|N|N|2467768|2467919|2467420|2467695|N|N|N|N|N| +2467787|AAAAAAAALMHKFCAA|2044-06-20|1733|7538|579|2044|1|6|20|2|2044|579|7538|Monday|2044Q2|N|N|N|2467768|2467919|2467421|2467696|N|N|N|N|N| +2467788|AAAAAAAAMMHKFCAA|2044-06-21|1733|7539|579|2044|2|6|21|2|2044|579|7539|Tuesday|2044Q2|N|N|N|2467768|2467919|2467422|2467697|N|N|N|N|N| +2467789|AAAAAAAANMHKFCAA|2044-06-22|1733|7539|579|2044|3|6|22|2|2044|579|7539|Wednesday|2044Q2|N|N|N|2467768|2467919|2467423|2467698|N|N|N|N|N| +2467790|AAAAAAAAOMHKFCAA|2044-06-23|1733|7539|579|2044|4|6|23|2|2044|579|7539|Thursday|2044Q2|N|N|N|2467768|2467919|2467424|2467699|N|N|N|N|N| +2467791|AAAAAAAAPMHKFCAA|2044-06-24|1733|7539|579|2044|5|6|24|2|2044|579|7539|Friday|2044Q2|N|Y|N|2467768|2467919|2467425|2467700|N|N|N|N|N| +2467792|AAAAAAAAANHKFCAA|2044-06-25|1733|7539|579|2044|6|6|25|2|2044|579|7539|Saturday|2044Q2|N|Y|N|2467768|2467919|2467426|2467701|N|N|N|N|N| +2467793|AAAAAAAABNHKFCAA|2044-06-26|1733|7539|579|2044|0|6|26|2|2044|579|7539|Sunday|2044Q2|N|N|N|2467768|2467919|2467427|2467702|N|N|N|N|N| +2467794|AAAAAAAACNHKFCAA|2044-06-27|1733|7539|579|2044|1|6|27|2|2044|579|7539|Monday|2044Q2|N|N|N|2467768|2467919|2467428|2467703|N|N|N|N|N| +2467795|AAAAAAAADNHKFCAA|2044-06-28|1733|7540|579|2044|2|6|28|2|2044|579|7540|Tuesday|2044Q2|N|N|N|2467768|2467919|2467429|2467704|N|N|N|N|N| +2467796|AAAAAAAAENHKFCAA|2044-06-29|1733|7540|579|2044|3|6|29|2|2044|579|7540|Wednesday|2044Q2|N|N|N|2467768|2467919|2467430|2467705|N|N|N|N|N| +2467797|AAAAAAAAFNHKFCAA|2044-06-30|1733|7540|579|2044|4|6|30|2|2044|579|7540|Thursday|2044Q2|N|N|N|2467768|2467919|2467431|2467706|N|N|N|N|N| +2467798|AAAAAAAAGNHKFCAA|2044-07-01|1734|7540|579|2044|5|7|1|3|2044|579|7540|Friday|2044Q3|N|Y|N|2467798|2467979|2467432|2467707|N|N|N|N|N| +2467799|AAAAAAAAHNHKFCAA|2044-07-02|1734|7540|579|2044|6|7|2|3|2044|579|7540|Saturday|2044Q3|N|Y|N|2467798|2467979|2467433|2467708|N|N|N|N|N| +2467800|AAAAAAAAINHKFCAA|2044-07-03|1734|7540|579|2044|0|7|3|3|2044|579|7540|Sunday|2044Q3|N|N|N|2467798|2467979|2467434|2467709|N|N|N|N|N| +2467801|AAAAAAAAJNHKFCAA|2044-07-04|1734|7540|579|2044|1|7|4|3|2044|579|7540|Monday|2044Q3|Y|N|N|2467798|2467979|2467435|2467710|N|N|N|N|N| +2467802|AAAAAAAAKNHKFCAA|2044-07-05|1734|7541|579|2044|2|7|5|3|2044|579|7541|Tuesday|2044Q3|N|N|Y|2467798|2467979|2467436|2467711|N|N|N|N|N| +2467803|AAAAAAAALNHKFCAA|2044-07-06|1734|7541|579|2044|3|7|6|3|2044|579|7541|Wednesday|2044Q3|N|N|N|2467798|2467979|2467437|2467712|N|N|N|N|N| +2467804|AAAAAAAAMNHKFCAA|2044-07-07|1734|7541|579|2044|4|7|7|3|2044|579|7541|Thursday|2044Q3|N|N|N|2467798|2467979|2467438|2467713|N|N|N|N|N| +2467805|AAAAAAAANNHKFCAA|2044-07-08|1734|7541|579|2044|5|7|8|3|2044|579|7541|Friday|2044Q3|N|Y|N|2467798|2467979|2467439|2467714|N|N|N|N|N| +2467806|AAAAAAAAONHKFCAA|2044-07-09|1734|7541|579|2044|6|7|9|3|2044|579|7541|Saturday|2044Q3|N|Y|N|2467798|2467979|2467440|2467715|N|N|N|N|N| +2467807|AAAAAAAAPNHKFCAA|2044-07-10|1734|7541|579|2044|0|7|10|3|2044|579|7541|Sunday|2044Q3|N|N|N|2467798|2467979|2467441|2467716|N|N|N|N|N| +2467808|AAAAAAAAAOHKFCAA|2044-07-11|1734|7541|579|2044|1|7|11|3|2044|579|7541|Monday|2044Q3|N|N|N|2467798|2467979|2467442|2467717|N|N|N|N|N| +2467809|AAAAAAAABOHKFCAA|2044-07-12|1734|7542|579|2044|2|7|12|3|2044|579|7542|Tuesday|2044Q3|N|N|N|2467798|2467979|2467443|2467718|N|N|N|N|N| +2467810|AAAAAAAACOHKFCAA|2044-07-13|1734|7542|579|2044|3|7|13|3|2044|579|7542|Wednesday|2044Q3|N|N|N|2467798|2467979|2467444|2467719|N|N|N|N|N| +2467811|AAAAAAAADOHKFCAA|2044-07-14|1734|7542|579|2044|4|7|14|3|2044|579|7542|Thursday|2044Q3|N|N|N|2467798|2467979|2467445|2467720|N|N|N|N|N| +2467812|AAAAAAAAEOHKFCAA|2044-07-15|1734|7542|579|2044|5|7|15|3|2044|579|7542|Friday|2044Q3|N|Y|N|2467798|2467979|2467446|2467721|N|N|N|N|N| +2467813|AAAAAAAAFOHKFCAA|2044-07-16|1734|7542|579|2044|6|7|16|3|2044|579|7542|Saturday|2044Q3|N|Y|N|2467798|2467979|2467447|2467722|N|N|N|N|N| +2467814|AAAAAAAAGOHKFCAA|2044-07-17|1734|7542|579|2044|0|7|17|3|2044|579|7542|Sunday|2044Q3|N|N|N|2467798|2467979|2467448|2467723|N|N|N|N|N| +2467815|AAAAAAAAHOHKFCAA|2044-07-18|1734|7542|579|2044|1|7|18|3|2044|579|7542|Monday|2044Q3|N|N|N|2467798|2467979|2467449|2467724|N|N|N|N|N| +2467816|AAAAAAAAIOHKFCAA|2044-07-19|1734|7543|579|2044|2|7|19|3|2044|579|7543|Tuesday|2044Q3|N|N|N|2467798|2467979|2467450|2467725|N|N|N|N|N| +2467817|AAAAAAAAJOHKFCAA|2044-07-20|1734|7543|579|2044|3|7|20|3|2044|579|7543|Wednesday|2044Q3|N|N|N|2467798|2467979|2467451|2467726|N|N|N|N|N| +2467818|AAAAAAAAKOHKFCAA|2044-07-21|1734|7543|579|2044|4|7|21|3|2044|579|7543|Thursday|2044Q3|N|N|N|2467798|2467979|2467452|2467727|N|N|N|N|N| +2467819|AAAAAAAALOHKFCAA|2044-07-22|1734|7543|579|2044|5|7|22|3|2044|579|7543|Friday|2044Q3|N|Y|N|2467798|2467979|2467453|2467728|N|N|N|N|N| +2467820|AAAAAAAAMOHKFCAA|2044-07-23|1734|7543|579|2044|6|7|23|3|2044|579|7543|Saturday|2044Q3|N|Y|N|2467798|2467979|2467454|2467729|N|N|N|N|N| +2467821|AAAAAAAANOHKFCAA|2044-07-24|1734|7543|579|2044|0|7|24|3|2044|579|7543|Sunday|2044Q3|N|N|N|2467798|2467979|2467455|2467730|N|N|N|N|N| +2467822|AAAAAAAAOOHKFCAA|2044-07-25|1734|7543|579|2044|1|7|25|3|2044|579|7543|Monday|2044Q3|N|N|N|2467798|2467979|2467456|2467731|N|N|N|N|N| +2467823|AAAAAAAAPOHKFCAA|2044-07-26|1734|7544|579|2044|2|7|26|3|2044|579|7544|Tuesday|2044Q3|N|N|N|2467798|2467979|2467457|2467732|N|N|N|N|N| +2467824|AAAAAAAAAPHKFCAA|2044-07-27|1734|7544|579|2044|3|7|27|3|2044|579|7544|Wednesday|2044Q3|N|N|N|2467798|2467979|2467458|2467733|N|N|N|N|N| +2467825|AAAAAAAABPHKFCAA|2044-07-28|1734|7544|579|2044|4|7|28|3|2044|579|7544|Thursday|2044Q3|N|N|N|2467798|2467979|2467459|2467734|N|N|N|N|N| +2467826|AAAAAAAACPHKFCAA|2044-07-29|1734|7544|579|2044|5|7|29|3|2044|579|7544|Friday|2044Q3|N|Y|N|2467798|2467979|2467460|2467735|N|N|N|N|N| +2467827|AAAAAAAADPHKFCAA|2044-07-30|1734|7544|579|2044|6|7|30|3|2044|579|7544|Saturday|2044Q3|N|Y|N|2467798|2467979|2467461|2467736|N|N|N|N|N| +2467828|AAAAAAAAEPHKFCAA|2044-07-31|1734|7544|579|2044|0|7|31|3|2044|579|7544|Sunday|2044Q3|N|N|N|2467798|2467979|2467462|2467737|N|N|N|N|N| +2467829|AAAAAAAAFPHKFCAA|2044-08-01|1735|7544|579|2044|1|8|1|3|2044|579|7544|Monday|2044Q3|N|N|N|2467829|2468041|2467463|2467738|N|N|N|N|N| +2467830|AAAAAAAAGPHKFCAA|2044-08-02|1735|7545|579|2044|2|8|2|3|2044|579|7545|Tuesday|2044Q3|N|N|N|2467829|2468041|2467464|2467739|N|N|N|N|N| +2467831|AAAAAAAAHPHKFCAA|2044-08-03|1735|7545|579|2044|3|8|3|3|2044|579|7545|Wednesday|2044Q3|N|N|N|2467829|2468041|2467465|2467740|N|N|N|N|N| +2467832|AAAAAAAAIPHKFCAA|2044-08-04|1735|7545|579|2044|4|8|4|3|2044|579|7545|Thursday|2044Q3|N|N|N|2467829|2468041|2467466|2467741|N|N|N|N|N| +2467833|AAAAAAAAJPHKFCAA|2044-08-05|1735|7545|579|2044|5|8|5|3|2044|579|7545|Friday|2044Q3|N|Y|N|2467829|2468041|2467467|2467742|N|N|N|N|N| +2467834|AAAAAAAAKPHKFCAA|2044-08-06|1735|7545|579|2044|6|8|6|3|2044|579|7545|Saturday|2044Q3|N|Y|N|2467829|2468041|2467468|2467743|N|N|N|N|N| +2467835|AAAAAAAALPHKFCAA|2044-08-07|1735|7545|579|2044|0|8|7|3|2044|579|7545|Sunday|2044Q3|N|N|N|2467829|2468041|2467469|2467744|N|N|N|N|N| +2467836|AAAAAAAAMPHKFCAA|2044-08-08|1735|7545|579|2044|1|8|8|3|2044|579|7545|Monday|2044Q3|N|N|N|2467829|2468041|2467470|2467745|N|N|N|N|N| +2467837|AAAAAAAANPHKFCAA|2044-08-09|1735|7546|579|2044|2|8|9|3|2044|579|7546|Tuesday|2044Q3|N|N|N|2467829|2468041|2467471|2467746|N|N|N|N|N| +2467838|AAAAAAAAOPHKFCAA|2044-08-10|1735|7546|579|2044|3|8|10|3|2044|579|7546|Wednesday|2044Q3|N|N|N|2467829|2468041|2467472|2467747|N|N|N|N|N| +2467839|AAAAAAAAPPHKFCAA|2044-08-11|1735|7546|579|2044|4|8|11|3|2044|579|7546|Thursday|2044Q3|N|N|N|2467829|2468041|2467473|2467748|N|N|N|N|N| +2467840|AAAAAAAAAAIKFCAA|2044-08-12|1735|7546|579|2044|5|8|12|3|2044|579|7546|Friday|2044Q3|N|Y|N|2467829|2468041|2467474|2467749|N|N|N|N|N| +2467841|AAAAAAAABAIKFCAA|2044-08-13|1735|7546|579|2044|6|8|13|3|2044|579|7546|Saturday|2044Q3|N|Y|N|2467829|2468041|2467475|2467750|N|N|N|N|N| +2467842|AAAAAAAACAIKFCAA|2044-08-14|1735|7546|579|2044|0|8|14|3|2044|579|7546|Sunday|2044Q3|N|N|N|2467829|2468041|2467476|2467751|N|N|N|N|N| +2467843|AAAAAAAADAIKFCAA|2044-08-15|1735|7546|579|2044|1|8|15|3|2044|579|7546|Monday|2044Q3|N|N|N|2467829|2468041|2467477|2467752|N|N|N|N|N| +2467844|AAAAAAAAEAIKFCAA|2044-08-16|1735|7547|579|2044|2|8|16|3|2044|579|7547|Tuesday|2044Q3|N|N|N|2467829|2468041|2467478|2467753|N|N|N|N|N| +2467845|AAAAAAAAFAIKFCAA|2044-08-17|1735|7547|579|2044|3|8|17|3|2044|579|7547|Wednesday|2044Q3|N|N|N|2467829|2468041|2467479|2467754|N|N|N|N|N| +2467846|AAAAAAAAGAIKFCAA|2044-08-18|1735|7547|579|2044|4|8|18|3|2044|579|7547|Thursday|2044Q3|N|N|N|2467829|2468041|2467480|2467755|N|N|N|N|N| +2467847|AAAAAAAAHAIKFCAA|2044-08-19|1735|7547|579|2044|5|8|19|3|2044|579|7547|Friday|2044Q3|N|Y|N|2467829|2468041|2467481|2467756|N|N|N|N|N| +2467848|AAAAAAAAIAIKFCAA|2044-08-20|1735|7547|579|2044|6|8|20|3|2044|579|7547|Saturday|2044Q3|N|Y|N|2467829|2468041|2467482|2467757|N|N|N|N|N| +2467849|AAAAAAAAJAIKFCAA|2044-08-21|1735|7547|579|2044|0|8|21|3|2044|579|7547|Sunday|2044Q3|N|N|N|2467829|2468041|2467483|2467758|N|N|N|N|N| +2467850|AAAAAAAAKAIKFCAA|2044-08-22|1735|7547|579|2044|1|8|22|3|2044|579|7547|Monday|2044Q3|N|N|N|2467829|2468041|2467484|2467759|N|N|N|N|N| +2467851|AAAAAAAALAIKFCAA|2044-08-23|1735|7548|579|2044|2|8|23|3|2044|579|7548|Tuesday|2044Q3|N|N|N|2467829|2468041|2467485|2467760|N|N|N|N|N| +2467852|AAAAAAAAMAIKFCAA|2044-08-24|1735|7548|579|2044|3|8|24|3|2044|579|7548|Wednesday|2044Q3|N|N|N|2467829|2468041|2467486|2467761|N|N|N|N|N| +2467853|AAAAAAAANAIKFCAA|2044-08-25|1735|7548|579|2044|4|8|25|3|2044|579|7548|Thursday|2044Q3|N|N|N|2467829|2468041|2467487|2467762|N|N|N|N|N| +2467854|AAAAAAAAOAIKFCAA|2044-08-26|1735|7548|579|2044|5|8|26|3|2044|579|7548|Friday|2044Q3|N|Y|N|2467829|2468041|2467488|2467763|N|N|N|N|N| +2467855|AAAAAAAAPAIKFCAA|2044-08-27|1735|7548|579|2044|6|8|27|3|2044|579|7548|Saturday|2044Q3|N|Y|N|2467829|2468041|2467489|2467764|N|N|N|N|N| +2467856|AAAAAAAAABIKFCAA|2044-08-28|1735|7548|579|2044|0|8|28|3|2044|579|7548|Sunday|2044Q3|N|N|N|2467829|2468041|2467490|2467765|N|N|N|N|N| +2467857|AAAAAAAABBIKFCAA|2044-08-29|1735|7548|579|2044|1|8|29|3|2044|579|7548|Monday|2044Q3|N|N|N|2467829|2468041|2467491|2467766|N|N|N|N|N| +2467858|AAAAAAAACBIKFCAA|2044-08-30|1735|7549|579|2044|2|8|30|3|2044|579|7549|Tuesday|2044Q3|N|N|N|2467829|2468041|2467492|2467767|N|N|N|N|N| +2467859|AAAAAAAADBIKFCAA|2044-08-31|1735|7549|579|2044|3|8|31|3|2044|579|7549|Wednesday|2044Q3|N|N|N|2467829|2468041|2467493|2467768|N|N|N|N|N| +2467860|AAAAAAAAEBIKFCAA|2044-09-01|1736|7549|580|2044|4|9|1|3|2044|580|7549|Thursday|2044Q3|N|N|N|2467860|2468103|2467494|2467769|N|N|N|N|N| +2467861|AAAAAAAAFBIKFCAA|2044-09-02|1736|7549|580|2044|5|9|2|3|2044|580|7549|Friday|2044Q3|N|Y|N|2467860|2468103|2467495|2467770|N|N|N|N|N| +2467862|AAAAAAAAGBIKFCAA|2044-09-03|1736|7549|580|2044|6|9|3|3|2044|580|7549|Saturday|2044Q3|N|Y|N|2467860|2468103|2467496|2467771|N|N|N|N|N| +2467863|AAAAAAAAHBIKFCAA|2044-09-04|1736|7549|580|2044|0|9|4|3|2044|580|7549|Sunday|2044Q3|N|N|N|2467860|2468103|2467497|2467772|N|N|N|N|N| +2467864|AAAAAAAAIBIKFCAA|2044-09-05|1736|7549|580|2044|1|9|5|3|2044|580|7549|Monday|2044Q3|N|N|N|2467860|2468103|2467498|2467773|N|N|N|N|N| +2467865|AAAAAAAAJBIKFCAA|2044-09-06|1736|7550|580|2044|2|9|6|3|2044|580|7550|Tuesday|2044Q3|N|N|N|2467860|2468103|2467499|2467774|N|N|N|N|N| +2467866|AAAAAAAAKBIKFCAA|2044-09-07|1736|7550|580|2044|3|9|7|3|2044|580|7550|Wednesday|2044Q3|N|N|N|2467860|2468103|2467500|2467775|N|N|N|N|N| +2467867|AAAAAAAALBIKFCAA|2044-09-08|1736|7550|580|2044|4|9|8|3|2044|580|7550|Thursday|2044Q3|N|N|N|2467860|2468103|2467501|2467776|N|N|N|N|N| +2467868|AAAAAAAAMBIKFCAA|2044-09-09|1736|7550|580|2044|5|9|9|3|2044|580|7550|Friday|2044Q3|N|Y|N|2467860|2468103|2467502|2467777|N|N|N|N|N| +2467869|AAAAAAAANBIKFCAA|2044-09-10|1736|7550|580|2044|6|9|10|3|2044|580|7550|Saturday|2044Q3|N|Y|N|2467860|2468103|2467503|2467778|N|N|N|N|N| +2467870|AAAAAAAAOBIKFCAA|2044-09-11|1736|7550|580|2044|0|9|11|3|2044|580|7550|Sunday|2044Q3|N|N|N|2467860|2468103|2467504|2467779|N|N|N|N|N| +2467871|AAAAAAAAPBIKFCAA|2044-09-12|1736|7550|580|2044|1|9|12|3|2044|580|7550|Monday|2044Q3|N|N|N|2467860|2468103|2467505|2467780|N|N|N|N|N| +2467872|AAAAAAAAACIKFCAA|2044-09-13|1736|7551|580|2044|2|9|13|3|2044|580|7551|Tuesday|2044Q3|N|N|N|2467860|2468103|2467506|2467781|N|N|N|N|N| +2467873|AAAAAAAABCIKFCAA|2044-09-14|1736|7551|580|2044|3|9|14|3|2044|580|7551|Wednesday|2044Q3|N|N|N|2467860|2468103|2467507|2467782|N|N|N|N|N| +2467874|AAAAAAAACCIKFCAA|2044-09-15|1736|7551|580|2044|4|9|15|3|2044|580|7551|Thursday|2044Q3|N|N|N|2467860|2468103|2467508|2467783|N|N|N|N|N| +2467875|AAAAAAAADCIKFCAA|2044-09-16|1736|7551|580|2044|5|9|16|3|2044|580|7551|Friday|2044Q3|N|Y|N|2467860|2468103|2467509|2467784|N|N|N|N|N| +2467876|AAAAAAAAECIKFCAA|2044-09-17|1736|7551|580|2044|6|9|17|3|2044|580|7551|Saturday|2044Q3|N|Y|N|2467860|2468103|2467510|2467785|N|N|N|N|N| +2467877|AAAAAAAAFCIKFCAA|2044-09-18|1736|7551|580|2044|0|9|18|3|2044|580|7551|Sunday|2044Q3|N|N|N|2467860|2468103|2467511|2467786|N|N|N|N|N| +2467878|AAAAAAAAGCIKFCAA|2044-09-19|1736|7551|580|2044|1|9|19|3|2044|580|7551|Monday|2044Q3|N|N|N|2467860|2468103|2467512|2467787|N|N|N|N|N| +2467879|AAAAAAAAHCIKFCAA|2044-09-20|1736|7552|580|2044|2|9|20|3|2044|580|7552|Tuesday|2044Q3|N|N|N|2467860|2468103|2467513|2467788|N|N|N|N|N| +2467880|AAAAAAAAICIKFCAA|2044-09-21|1736|7552|580|2044|3|9|21|3|2044|580|7552|Wednesday|2044Q3|N|N|N|2467860|2468103|2467514|2467789|N|N|N|N|N| +2467881|AAAAAAAAJCIKFCAA|2044-09-22|1736|7552|580|2044|4|9|22|3|2044|580|7552|Thursday|2044Q3|N|N|N|2467860|2468103|2467515|2467790|N|N|N|N|N| +2467882|AAAAAAAAKCIKFCAA|2044-09-23|1736|7552|580|2044|5|9|23|3|2044|580|7552|Friday|2044Q3|N|Y|N|2467860|2468103|2467516|2467791|N|N|N|N|N| +2467883|AAAAAAAALCIKFCAA|2044-09-24|1736|7552|580|2044|6|9|24|3|2044|580|7552|Saturday|2044Q3|N|Y|N|2467860|2468103|2467517|2467792|N|N|N|N|N| +2467884|AAAAAAAAMCIKFCAA|2044-09-25|1736|7552|580|2044|0|9|25|3|2044|580|7552|Sunday|2044Q3|N|N|N|2467860|2468103|2467518|2467793|N|N|N|N|N| +2467885|AAAAAAAANCIKFCAA|2044-09-26|1736|7552|580|2044|1|9|26|3|2044|580|7552|Monday|2044Q3|N|N|N|2467860|2468103|2467519|2467794|N|N|N|N|N| +2467886|AAAAAAAAOCIKFCAA|2044-09-27|1736|7553|580|2044|2|9|27|3|2044|580|7553|Tuesday|2044Q3|N|N|N|2467860|2468103|2467520|2467795|N|N|N|N|N| +2467887|AAAAAAAAPCIKFCAA|2044-09-28|1736|7553|580|2044|3|9|28|3|2044|580|7553|Wednesday|2044Q3|N|N|N|2467860|2468103|2467521|2467796|N|N|N|N|N| +2467888|AAAAAAAAADIKFCAA|2044-09-29|1736|7553|580|2044|4|9|29|3|2044|580|7553|Thursday|2044Q3|N|N|N|2467860|2468103|2467522|2467797|N|N|N|N|N| +2467889|AAAAAAAABDIKFCAA|2044-09-30|1736|7553|580|2044|5|9|30|3|2044|580|7553|Friday|2044Q3|N|Y|N|2467860|2468103|2467523|2467798|N|N|N|N|N| +2467890|AAAAAAAACDIKFCAA|2044-10-01|1737|7553|580|2044|6|10|1|4|2044|580|7553|Saturday|2044Q4|N|Y|N|2467890|2468163|2467524|2467798|N|N|N|N|N| +2467891|AAAAAAAADDIKFCAA|2044-10-02|1737|7553|580|2044|0|10|2|4|2044|580|7553|Sunday|2044Q4|N|N|N|2467890|2468163|2467525|2467799|N|N|N|N|N| +2467892|AAAAAAAAEDIKFCAA|2044-10-03|1737|7553|580|2044|1|10|3|4|2044|580|7553|Monday|2044Q4|N|N|N|2467890|2468163|2467526|2467800|N|N|N|N|N| +2467893|AAAAAAAAFDIKFCAA|2044-10-04|1737|7554|580|2044|2|10|4|4|2044|580|7554|Tuesday|2044Q4|N|N|N|2467890|2468163|2467527|2467801|N|N|N|N|N| +2467894|AAAAAAAAGDIKFCAA|2044-10-05|1737|7554|580|2044|3|10|5|4|2044|580|7554|Wednesday|2044Q4|N|N|N|2467890|2468163|2467528|2467802|N|N|N|N|N| +2467895|AAAAAAAAHDIKFCAA|2044-10-06|1737|7554|580|2044|4|10|6|4|2044|580|7554|Thursday|2044Q4|N|N|N|2467890|2468163|2467529|2467803|N|N|N|N|N| +2467896|AAAAAAAAIDIKFCAA|2044-10-07|1737|7554|580|2044|5|10|7|4|2044|580|7554|Friday|2044Q4|N|Y|N|2467890|2468163|2467530|2467804|N|N|N|N|N| +2467897|AAAAAAAAJDIKFCAA|2044-10-08|1737|7554|580|2044|6|10|8|4|2044|580|7554|Saturday|2044Q4|N|Y|N|2467890|2468163|2467531|2467805|N|N|N|N|N| +2467898|AAAAAAAAKDIKFCAA|2044-10-09|1737|7554|580|2044|0|10|9|4|2044|580|7554|Sunday|2044Q4|N|N|N|2467890|2468163|2467532|2467806|N|N|N|N|N| +2467899|AAAAAAAALDIKFCAA|2044-10-10|1737|7554|580|2044|1|10|10|4|2044|580|7554|Monday|2044Q4|N|N|N|2467890|2468163|2467533|2467807|N|N|N|N|N| +2467900|AAAAAAAAMDIKFCAA|2044-10-11|1737|7555|580|2044|2|10|11|4|2044|580|7555|Tuesday|2044Q4|N|N|N|2467890|2468163|2467534|2467808|N|N|N|N|N| +2467901|AAAAAAAANDIKFCAA|2044-10-12|1737|7555|580|2044|3|10|12|4|2044|580|7555|Wednesday|2044Q4|N|N|N|2467890|2468163|2467535|2467809|N|N|N|N|N| +2467902|AAAAAAAAODIKFCAA|2044-10-13|1737|7555|580|2044|4|10|13|4|2044|580|7555|Thursday|2044Q4|N|N|N|2467890|2468163|2467536|2467810|N|N|N|N|N| +2467903|AAAAAAAAPDIKFCAA|2044-10-14|1737|7555|580|2044|5|10|14|4|2044|580|7555|Friday|2044Q4|N|Y|N|2467890|2468163|2467537|2467811|N|N|N|N|N| +2467904|AAAAAAAAAEIKFCAA|2044-10-15|1737|7555|580|2044|6|10|15|4|2044|580|7555|Saturday|2044Q4|N|Y|N|2467890|2468163|2467538|2467812|N|N|N|N|N| +2467905|AAAAAAAABEIKFCAA|2044-10-16|1737|7555|580|2044|0|10|16|4|2044|580|7555|Sunday|2044Q4|N|N|N|2467890|2468163|2467539|2467813|N|N|N|N|N| +2467906|AAAAAAAACEIKFCAA|2044-10-17|1737|7555|580|2044|1|10|17|4|2044|580|7555|Monday|2044Q4|N|N|N|2467890|2468163|2467540|2467814|N|N|N|N|N| +2467907|AAAAAAAADEIKFCAA|2044-10-18|1737|7556|580|2044|2|10|18|4|2044|580|7556|Tuesday|2044Q4|N|N|N|2467890|2468163|2467541|2467815|N|N|N|N|N| +2467908|AAAAAAAAEEIKFCAA|2044-10-19|1737|7556|580|2044|3|10|19|4|2044|580|7556|Wednesday|2044Q4|N|N|N|2467890|2468163|2467542|2467816|N|N|N|N|N| +2467909|AAAAAAAAFEIKFCAA|2044-10-20|1737|7556|580|2044|4|10|20|4|2044|580|7556|Thursday|2044Q4|N|N|N|2467890|2468163|2467543|2467817|N|N|N|N|N| +2467910|AAAAAAAAGEIKFCAA|2044-10-21|1737|7556|580|2044|5|10|21|4|2044|580|7556|Friday|2044Q4|N|Y|N|2467890|2468163|2467544|2467818|N|N|N|N|N| +2467911|AAAAAAAAHEIKFCAA|2044-10-22|1737|7556|580|2044|6|10|22|4|2044|580|7556|Saturday|2044Q4|N|Y|N|2467890|2468163|2467545|2467819|N|N|N|N|N| +2467912|AAAAAAAAIEIKFCAA|2044-10-23|1737|7556|580|2044|0|10|23|4|2044|580|7556|Sunday|2044Q4|N|N|N|2467890|2468163|2467546|2467820|N|N|N|N|N| +2467913|AAAAAAAAJEIKFCAA|2044-10-24|1737|7556|580|2044|1|10|24|4|2044|580|7556|Monday|2044Q4|N|N|N|2467890|2468163|2467547|2467821|N|N|N|N|N| +2467914|AAAAAAAAKEIKFCAA|2044-10-25|1737|7557|580|2044|2|10|25|4|2044|580|7557|Tuesday|2044Q4|N|N|N|2467890|2468163|2467548|2467822|N|N|N|N|N| +2467915|AAAAAAAALEIKFCAA|2044-10-26|1737|7557|580|2044|3|10|26|4|2044|580|7557|Wednesday|2044Q4|N|N|N|2467890|2468163|2467549|2467823|N|N|N|N|N| +2467916|AAAAAAAAMEIKFCAA|2044-10-27|1737|7557|580|2044|4|10|27|4|2044|580|7557|Thursday|2044Q4|N|N|N|2467890|2468163|2467550|2467824|N|N|N|N|N| +2467917|AAAAAAAANEIKFCAA|2044-10-28|1737|7557|580|2044|5|10|28|4|2044|580|7557|Friday|2044Q4|N|Y|N|2467890|2468163|2467551|2467825|N|N|N|N|N| +2467918|AAAAAAAAOEIKFCAA|2044-10-29|1737|7557|580|2044|6|10|29|4|2044|580|7557|Saturday|2044Q4|N|Y|N|2467890|2468163|2467552|2467826|N|N|N|N|N| +2467919|AAAAAAAAPEIKFCAA|2044-10-30|1737|7557|580|2044|0|10|30|4|2044|580|7557|Sunday|2044Q4|N|N|N|2467890|2468163|2467553|2467827|N|N|N|N|N| +2467920|AAAAAAAAAFIKFCAA|2044-10-31|1737|7557|580|2044|1|10|31|4|2044|580|7557|Monday|2044Q4|N|N|N|2467890|2468163|2467554|2467828|N|N|N|N|N| +2467921|AAAAAAAABFIKFCAA|2044-11-01|1738|7558|580|2044|2|11|1|4|2044|580|7558|Tuesday|2044Q4|N|N|N|2467921|2468225|2467555|2467829|N|N|N|N|N| +2467922|AAAAAAAACFIKFCAA|2044-11-02|1738|7558|580|2044|3|11|2|4|2044|580|7558|Wednesday|2044Q4|N|N|N|2467921|2468225|2467556|2467830|N|N|N|N|N| +2467923|AAAAAAAADFIKFCAA|2044-11-03|1738|7558|580|2044|4|11|3|4|2044|580|7558|Thursday|2044Q4|N|N|N|2467921|2468225|2467557|2467831|N|N|N|N|N| +2467924|AAAAAAAAEFIKFCAA|2044-11-04|1738|7558|580|2044|5|11|4|4|2044|580|7558|Friday|2044Q4|N|Y|N|2467921|2468225|2467558|2467832|N|N|N|N|N| +2467925|AAAAAAAAFFIKFCAA|2044-11-05|1738|7558|580|2044|6|11|5|4|2044|580|7558|Saturday|2044Q4|N|Y|N|2467921|2468225|2467559|2467833|N|N|N|N|N| +2467926|AAAAAAAAGFIKFCAA|2044-11-06|1738|7558|580|2044|0|11|6|4|2044|580|7558|Sunday|2044Q4|N|N|N|2467921|2468225|2467560|2467834|N|N|N|N|N| +2467927|AAAAAAAAHFIKFCAA|2044-11-07|1738|7558|580|2044|1|11|7|4|2044|580|7558|Monday|2044Q4|N|N|N|2467921|2468225|2467561|2467835|N|N|N|N|N| +2467928|AAAAAAAAIFIKFCAA|2044-11-08|1738|7559|580|2044|2|11|8|4|2044|580|7559|Tuesday|2044Q4|N|N|N|2467921|2468225|2467562|2467836|N|N|N|N|N| +2467929|AAAAAAAAJFIKFCAA|2044-11-09|1738|7559|580|2044|3|11|9|4|2044|580|7559|Wednesday|2044Q4|N|N|N|2467921|2468225|2467563|2467837|N|N|N|N|N| +2467930|AAAAAAAAKFIKFCAA|2044-11-10|1738|7559|580|2044|4|11|10|4|2044|580|7559|Thursday|2044Q4|N|N|N|2467921|2468225|2467564|2467838|N|N|N|N|N| +2467931|AAAAAAAALFIKFCAA|2044-11-11|1738|7559|580|2044|5|11|11|4|2044|580|7559|Friday|2044Q4|N|Y|N|2467921|2468225|2467565|2467839|N|N|N|N|N| +2467932|AAAAAAAAMFIKFCAA|2044-11-12|1738|7559|580|2044|6|11|12|4|2044|580|7559|Saturday|2044Q4|N|Y|N|2467921|2468225|2467566|2467840|N|N|N|N|N| +2467933|AAAAAAAANFIKFCAA|2044-11-13|1738|7559|580|2044|0|11|13|4|2044|580|7559|Sunday|2044Q4|N|N|N|2467921|2468225|2467567|2467841|N|N|N|N|N| +2467934|AAAAAAAAOFIKFCAA|2044-11-14|1738|7559|580|2044|1|11|14|4|2044|580|7559|Monday|2044Q4|N|N|N|2467921|2468225|2467568|2467842|N|N|N|N|N| +2467935|AAAAAAAAPFIKFCAA|2044-11-15|1738|7560|580|2044|2|11|15|4|2044|580|7560|Tuesday|2044Q4|N|N|N|2467921|2468225|2467569|2467843|N|N|N|N|N| +2467936|AAAAAAAAAGIKFCAA|2044-11-16|1738|7560|580|2044|3|11|16|4|2044|580|7560|Wednesday|2044Q4|N|N|N|2467921|2468225|2467570|2467844|N|N|N|N|N| +2467937|AAAAAAAABGIKFCAA|2044-11-17|1738|7560|580|2044|4|11|17|4|2044|580|7560|Thursday|2044Q4|N|N|N|2467921|2468225|2467571|2467845|N|N|N|N|N| +2467938|AAAAAAAACGIKFCAA|2044-11-18|1738|7560|580|2044|5|11|18|4|2044|580|7560|Friday|2044Q4|N|Y|N|2467921|2468225|2467572|2467846|N|N|N|N|N| +2467939|AAAAAAAADGIKFCAA|2044-11-19|1738|7560|580|2044|6|11|19|4|2044|580|7560|Saturday|2044Q4|N|Y|N|2467921|2468225|2467573|2467847|N|N|N|N|N| +2467940|AAAAAAAAEGIKFCAA|2044-11-20|1738|7560|580|2044|0|11|20|4|2044|580|7560|Sunday|2044Q4|N|N|N|2467921|2468225|2467574|2467848|N|N|N|N|N| +2467941|AAAAAAAAFGIKFCAA|2044-11-21|1738|7560|580|2044|1|11|21|4|2044|580|7560|Monday|2044Q4|N|N|N|2467921|2468225|2467575|2467849|N|N|N|N|N| +2467942|AAAAAAAAGGIKFCAA|2044-11-22|1738|7561|580|2044|2|11|22|4|2044|580|7561|Tuesday|2044Q4|N|N|N|2467921|2468225|2467576|2467850|N|N|N|N|N| +2467943|AAAAAAAAHGIKFCAA|2044-11-23|1738|7561|580|2044|3|11|23|4|2044|580|7561|Wednesday|2044Q4|N|N|N|2467921|2468225|2467577|2467851|N|N|N|N|N| +2467944|AAAAAAAAIGIKFCAA|2044-11-24|1738|7561|580|2044|4|11|24|4|2044|580|7561|Thursday|2044Q4|N|N|N|2467921|2468225|2467578|2467852|N|N|N|N|N| +2467945|AAAAAAAAJGIKFCAA|2044-11-25|1738|7561|580|2044|5|11|25|4|2044|580|7561|Friday|2044Q4|N|Y|N|2467921|2468225|2467579|2467853|N|N|N|N|N| +2467946|AAAAAAAAKGIKFCAA|2044-11-26|1738|7561|580|2044|6|11|26|4|2044|580|7561|Saturday|2044Q4|N|Y|N|2467921|2468225|2467580|2467854|N|N|N|N|N| +2467947|AAAAAAAALGIKFCAA|2044-11-27|1738|7561|580|2044|0|11|27|4|2044|580|7561|Sunday|2044Q4|N|N|N|2467921|2468225|2467581|2467855|N|N|N|N|N| +2467948|AAAAAAAAMGIKFCAA|2044-11-28|1738|7561|580|2044|1|11|28|4|2044|580|7561|Monday|2044Q4|N|N|N|2467921|2468225|2467582|2467856|N|N|N|N|N| +2467949|AAAAAAAANGIKFCAA|2044-11-29|1738|7562|580|2044|2|11|29|4|2044|580|7562|Tuesday|2044Q4|N|N|N|2467921|2468225|2467583|2467857|N|N|N|N|N| +2467950|AAAAAAAAOGIKFCAA|2044-11-30|1738|7562|580|2044|3|11|30|4|2044|580|7562|Wednesday|2044Q4|N|N|N|2467921|2468225|2467584|2467858|N|N|N|N|N| +2467951|AAAAAAAAPGIKFCAA|2044-12-01|1739|7562|581|2044|4|12|1|4|2044|581|7562|Thursday|2044Q4|N|N|N|2467951|2468285|2467585|2467859|N|N|N|N|N| +2467952|AAAAAAAAAHIKFCAA|2044-12-02|1739|7562|581|2044|5|12|2|4|2044|581|7562|Friday|2044Q4|N|Y|N|2467951|2468285|2467586|2467860|N|N|N|N|N| +2467953|AAAAAAAABHIKFCAA|2044-12-03|1739|7562|581|2044|6|12|3|4|2044|581|7562|Saturday|2044Q4|N|Y|N|2467951|2468285|2467587|2467861|N|N|N|N|N| +2467954|AAAAAAAACHIKFCAA|2044-12-04|1739|7562|581|2044|0|12|4|4|2044|581|7562|Sunday|2044Q4|N|N|N|2467951|2468285|2467588|2467862|N|N|N|N|N| +2467955|AAAAAAAADHIKFCAA|2044-12-05|1739|7562|581|2044|1|12|5|4|2044|581|7562|Monday|2044Q4|N|N|N|2467951|2468285|2467589|2467863|N|N|N|N|N| +2467956|AAAAAAAAEHIKFCAA|2044-12-06|1739|7563|581|2044|2|12|6|4|2044|581|7563|Tuesday|2044Q4|N|N|N|2467951|2468285|2467590|2467864|N|N|N|N|N| +2467957|AAAAAAAAFHIKFCAA|2044-12-07|1739|7563|581|2044|3|12|7|4|2044|581|7563|Wednesday|2044Q4|N|N|N|2467951|2468285|2467591|2467865|N|N|N|N|N| +2467958|AAAAAAAAGHIKFCAA|2044-12-08|1739|7563|581|2044|4|12|8|4|2044|581|7563|Thursday|2044Q4|N|N|N|2467951|2468285|2467592|2467866|N|N|N|N|N| +2467959|AAAAAAAAHHIKFCAA|2044-12-09|1739|7563|581|2044|5|12|9|4|2044|581|7563|Friday|2044Q4|N|Y|N|2467951|2468285|2467593|2467867|N|N|N|N|N| +2467960|AAAAAAAAIHIKFCAA|2044-12-10|1739|7563|581|2044|6|12|10|4|2044|581|7563|Saturday|2044Q4|N|Y|N|2467951|2468285|2467594|2467868|N|N|N|N|N| +2467961|AAAAAAAAJHIKFCAA|2044-12-11|1739|7563|581|2044|0|12|11|4|2044|581|7563|Sunday|2044Q4|N|N|N|2467951|2468285|2467595|2467869|N|N|N|N|N| +2467962|AAAAAAAAKHIKFCAA|2044-12-12|1739|7563|581|2044|1|12|12|4|2044|581|7563|Monday|2044Q4|N|N|N|2467951|2468285|2467596|2467870|N|N|N|N|N| +2467963|AAAAAAAALHIKFCAA|2044-12-13|1739|7564|581|2044|2|12|13|4|2044|581|7564|Tuesday|2044Q4|N|N|N|2467951|2468285|2467597|2467871|N|N|N|N|N| +2467964|AAAAAAAAMHIKFCAA|2044-12-14|1739|7564|581|2044|3|12|14|4|2044|581|7564|Wednesday|2044Q4|N|N|N|2467951|2468285|2467598|2467872|N|N|N|N|N| +2467965|AAAAAAAANHIKFCAA|2044-12-15|1739|7564|581|2044|4|12|15|4|2044|581|7564|Thursday|2044Q4|N|N|N|2467951|2468285|2467599|2467873|N|N|N|N|N| +2467966|AAAAAAAAOHIKFCAA|2044-12-16|1739|7564|581|2044|5|12|16|4|2044|581|7564|Friday|2044Q4|N|Y|N|2467951|2468285|2467600|2467874|N|N|N|N|N| +2467967|AAAAAAAAPHIKFCAA|2044-12-17|1739|7564|581|2044|6|12|17|4|2044|581|7564|Saturday|2044Q4|N|Y|N|2467951|2468285|2467601|2467875|N|N|N|N|N| +2467968|AAAAAAAAAIIKFCAA|2044-12-18|1739|7564|581|2044|0|12|18|4|2044|581|7564|Sunday|2044Q4|N|N|N|2467951|2468285|2467602|2467876|N|N|N|N|N| +2467969|AAAAAAAABIIKFCAA|2044-12-19|1739|7564|581|2044|1|12|19|4|2044|581|7564|Monday|2044Q4|N|N|N|2467951|2468285|2467603|2467877|N|N|N|N|N| +2467970|AAAAAAAACIIKFCAA|2044-12-20|1739|7565|581|2044|2|12|20|4|2044|581|7565|Tuesday|2044Q4|N|N|N|2467951|2468285|2467604|2467878|N|N|N|N|N| +2467971|AAAAAAAADIIKFCAA|2044-12-21|1739|7565|581|2044|3|12|21|4|2044|581|7565|Wednesday|2044Q4|N|N|N|2467951|2468285|2467605|2467879|N|N|N|N|N| +2467972|AAAAAAAAEIIKFCAA|2044-12-22|1739|7565|581|2044|4|12|22|4|2044|581|7565|Thursday|2044Q4|N|N|N|2467951|2468285|2467606|2467880|N|N|N|N|N| +2467973|AAAAAAAAFIIKFCAA|2044-12-23|1739|7565|581|2044|5|12|23|4|2044|581|7565|Friday|2044Q4|N|Y|N|2467951|2468285|2467607|2467881|N|N|N|N|N| +2467974|AAAAAAAAGIIKFCAA|2044-12-24|1739|7565|581|2044|6|12|24|4|2044|581|7565|Saturday|2044Q4|N|Y|N|2467951|2468285|2467608|2467882|N|N|N|N|N| +2467975|AAAAAAAAHIIKFCAA|2044-12-25|1739|7565|581|2044|0|12|25|4|2044|581|7565|Sunday|2044Q4|Y|N|N|2467951|2468285|2467609|2467883|N|N|N|N|N| +2467976|AAAAAAAAIIIKFCAA|2044-12-26|1739|7565|581|2044|1|12|26|4|2044|581|7565|Monday|2044Q4|N|N|Y|2467951|2468285|2467610|2467884|N|N|N|N|N| +2467977|AAAAAAAAJIIKFCAA|2044-12-27|1739|7566|581|2044|2|12|27|4|2044|581|7566|Tuesday|2044Q4|N|N|N|2467951|2468285|2467611|2467885|N|N|N|N|N| +2467978|AAAAAAAAKIIKFCAA|2044-12-28|1739|7566|581|2044|3|12|28|4|2044|581|7566|Wednesday|2044Q4|N|N|N|2467951|2468285|2467612|2467886|N|N|N|N|N| +2467979|AAAAAAAALIIKFCAA|2044-12-29|1739|7566|581|2044|4|12|29|4|2044|581|7566|Thursday|2044Q4|N|N|N|2467951|2468285|2467613|2467887|N|N|N|N|N| +2467980|AAAAAAAAMIIKFCAA|2044-12-30|1739|7566|581|2044|5|12|30|4|2044|581|7566|Friday|2044Q4|N|Y|N|2467951|2468285|2467614|2467888|N|N|N|N|N| +2467981|AAAAAAAANIIKFCAA|2044-12-31|1739|7566|581|2044|6|12|31|4|2044|581|7566|Saturday|2044Q4|Y|Y|N|2467951|2468285|2467615|2467889|N|N|N|N|N| +2467982|AAAAAAAAOIIKFCAA|2045-01-01|1740|7566|581|2045|0|1|1|1|2045|581|7566|Sunday|2045Q1|Y|N|Y|2467982|2467981|2467616|2467890|N|N|N|N|N| +2467983|AAAAAAAAPIIKFCAA|2045-01-02|1740|7566|581|2045|1|1|2|1|2045|581|7566|Monday|2045Q1|N|N|Y|2467982|2467981|2467617|2467891|N|N|N|N|N| +2467984|AAAAAAAAAJIKFCAA|2045-01-03|1740|7567|581|2045|2|1|3|1|2045|581|7567|Tuesday|2045Q1|N|N|N|2467982|2467981|2467618|2467892|N|N|N|N|N| +2467985|AAAAAAAABJIKFCAA|2045-01-04|1740|7567|581|2045|3|1|4|1|2045|581|7567|Wednesday|2045Q1|N|N|N|2467982|2467981|2467619|2467893|N|N|N|N|N| +2467986|AAAAAAAACJIKFCAA|2045-01-05|1740|7567|581|2045|4|1|5|1|2045|581|7567|Thursday|2045Q1|N|N|N|2467982|2467981|2467620|2467894|N|N|N|N|N| +2467987|AAAAAAAADJIKFCAA|2045-01-06|1740|7567|581|2045|5|1|6|1|2045|581|7567|Friday|2045Q1|N|Y|N|2467982|2467981|2467621|2467895|N|N|N|N|N| +2467988|AAAAAAAAEJIKFCAA|2045-01-07|1740|7567|581|2045|6|1|7|1|2045|581|7567|Saturday|2045Q1|N|Y|N|2467982|2467981|2467622|2467896|N|N|N|N|N| +2467989|AAAAAAAAFJIKFCAA|2045-01-08|1740|7567|581|2045|0|1|8|1|2045|581|7567|Sunday|2045Q1|N|N|N|2467982|2467981|2467623|2467897|N|N|N|N|N| +2467990|AAAAAAAAGJIKFCAA|2045-01-09|1740|7567|581|2045|1|1|9|1|2045|581|7567|Monday|2045Q1|N|N|N|2467982|2467981|2467624|2467898|N|N|N|N|N| +2467991|AAAAAAAAHJIKFCAA|2045-01-10|1740|7568|581|2045|2|1|10|1|2045|581|7568|Tuesday|2045Q1|N|N|N|2467982|2467981|2467625|2467899|N|N|N|N|N| +2467992|AAAAAAAAIJIKFCAA|2045-01-11|1740|7568|581|2045|3|1|11|1|2045|581|7568|Wednesday|2045Q1|N|N|N|2467982|2467981|2467626|2467900|N|N|N|N|N| +2467993|AAAAAAAAJJIKFCAA|2045-01-12|1740|7568|581|2045|4|1|12|1|2045|581|7568|Thursday|2045Q1|N|N|N|2467982|2467981|2467627|2467901|N|N|N|N|N| +2467994|AAAAAAAAKJIKFCAA|2045-01-13|1740|7568|581|2045|5|1|13|1|2045|581|7568|Friday|2045Q1|N|Y|N|2467982|2467981|2467628|2467902|N|N|N|N|N| +2467995|AAAAAAAALJIKFCAA|2045-01-14|1740|7568|581|2045|6|1|14|1|2045|581|7568|Saturday|2045Q1|N|Y|N|2467982|2467981|2467629|2467903|N|N|N|N|N| +2467996|AAAAAAAAMJIKFCAA|2045-01-15|1740|7568|581|2045|0|1|15|1|2045|581|7568|Sunday|2045Q1|N|N|N|2467982|2467981|2467630|2467904|N|N|N|N|N| +2467997|AAAAAAAANJIKFCAA|2045-01-16|1740|7568|581|2045|1|1|16|1|2045|581|7568|Monday|2045Q1|N|N|N|2467982|2467981|2467631|2467905|N|N|N|N|N| +2467998|AAAAAAAAOJIKFCAA|2045-01-17|1740|7569|581|2045|2|1|17|1|2045|581|7569|Tuesday|2045Q1|N|N|N|2467982|2467981|2467632|2467906|N|N|N|N|N| +2467999|AAAAAAAAPJIKFCAA|2045-01-18|1740|7569|581|2045|3|1|18|1|2045|581|7569|Wednesday|2045Q1|N|N|N|2467982|2467981|2467633|2467907|N|N|N|N|N| +2468000|AAAAAAAAAKIKFCAA|2045-01-19|1740|7569|581|2045|4|1|19|1|2045|581|7569|Thursday|2045Q1|N|N|N|2467982|2467981|2467634|2467908|N|N|N|N|N| +2468001|AAAAAAAABKIKFCAA|2045-01-20|1740|7569|581|2045|5|1|20|1|2045|581|7569|Friday|2045Q1|N|Y|N|2467982|2467981|2467635|2467909|N|N|N|N|N| +2468002|AAAAAAAACKIKFCAA|2045-01-21|1740|7569|581|2045|6|1|21|1|2045|581|7569|Saturday|2045Q1|N|Y|N|2467982|2467981|2467636|2467910|N|N|N|N|N| +2468003|AAAAAAAADKIKFCAA|2045-01-22|1740|7569|581|2045|0|1|22|1|2045|581|7569|Sunday|2045Q1|N|N|N|2467982|2467981|2467637|2467911|N|N|N|N|N| +2468004|AAAAAAAAEKIKFCAA|2045-01-23|1740|7569|581|2045|1|1|23|1|2045|581|7569|Monday|2045Q1|N|N|N|2467982|2467981|2467638|2467912|N|N|N|N|N| +2468005|AAAAAAAAFKIKFCAA|2045-01-24|1740|7570|581|2045|2|1|24|1|2045|581|7570|Tuesday|2045Q1|N|N|N|2467982|2467981|2467639|2467913|N|N|N|N|N| +2468006|AAAAAAAAGKIKFCAA|2045-01-25|1740|7570|581|2045|3|1|25|1|2045|581|7570|Wednesday|2045Q1|N|N|N|2467982|2467981|2467640|2467914|N|N|N|N|N| +2468007|AAAAAAAAHKIKFCAA|2045-01-26|1740|7570|581|2045|4|1|26|1|2045|581|7570|Thursday|2045Q1|N|N|N|2467982|2467981|2467641|2467915|N|N|N|N|N| +2468008|AAAAAAAAIKIKFCAA|2045-01-27|1740|7570|581|2045|5|1|27|1|2045|581|7570|Friday|2045Q1|N|Y|N|2467982|2467981|2467642|2467916|N|N|N|N|N| +2468009|AAAAAAAAJKIKFCAA|2045-01-28|1740|7570|581|2045|6|1|28|1|2045|581|7570|Saturday|2045Q1|N|Y|N|2467982|2467981|2467643|2467917|N|N|N|N|N| +2468010|AAAAAAAAKKIKFCAA|2045-01-29|1740|7570|581|2045|0|1|29|1|2045|581|7570|Sunday|2045Q1|N|N|N|2467982|2467981|2467644|2467918|N|N|N|N|N| +2468011|AAAAAAAALKIKFCAA|2045-01-30|1740|7570|581|2045|1|1|30|1|2045|581|7570|Monday|2045Q1|N|N|N|2467982|2467981|2467645|2467919|N|N|N|N|N| +2468012|AAAAAAAAMKIKFCAA|2045-01-31|1740|7571|581|2045|2|1|31|1|2045|581|7571|Tuesday|2045Q1|N|N|N|2467982|2467981|2467646|2467920|N|N|N|N|N| +2468013|AAAAAAAANKIKFCAA|2045-02-01|1741|7571|581|2045|3|2|1|1|2045|581|7571|Wednesday|2045Q1|N|N|N|2468013|2468043|2467647|2467921|N|N|N|N|N| +2468014|AAAAAAAAOKIKFCAA|2045-02-02|1741|7571|581|2045|4|2|2|1|2045|581|7571|Thursday|2045Q1|N|N|N|2468013|2468043|2467648|2467922|N|N|N|N|N| +2468015|AAAAAAAAPKIKFCAA|2045-02-03|1741|7571|581|2045|5|2|3|1|2045|581|7571|Friday|2045Q1|N|Y|N|2468013|2468043|2467649|2467923|N|N|N|N|N| +2468016|AAAAAAAAALIKFCAA|2045-02-04|1741|7571|581|2045|6|2|4|1|2045|581|7571|Saturday|2045Q1|N|Y|N|2468013|2468043|2467650|2467924|N|N|N|N|N| +2468017|AAAAAAAABLIKFCAA|2045-02-05|1741|7571|581|2045|0|2|5|1|2045|581|7571|Sunday|2045Q1|N|N|N|2468013|2468043|2467651|2467925|N|N|N|N|N| +2468018|AAAAAAAACLIKFCAA|2045-02-06|1741|7571|581|2045|1|2|6|1|2045|581|7571|Monday|2045Q1|N|N|N|2468013|2468043|2467652|2467926|N|N|N|N|N| +2468019|AAAAAAAADLIKFCAA|2045-02-07|1741|7572|581|2045|2|2|7|1|2045|581|7572|Tuesday|2045Q1|N|N|N|2468013|2468043|2467653|2467927|N|N|N|N|N| +2468020|AAAAAAAAELIKFCAA|2045-02-08|1741|7572|581|2045|3|2|8|1|2045|581|7572|Wednesday|2045Q1|N|N|N|2468013|2468043|2467654|2467928|N|N|N|N|N| +2468021|AAAAAAAAFLIKFCAA|2045-02-09|1741|7572|581|2045|4|2|9|1|2045|581|7572|Thursday|2045Q1|N|N|N|2468013|2468043|2467655|2467929|N|N|N|N|N| +2468022|AAAAAAAAGLIKFCAA|2045-02-10|1741|7572|581|2045|5|2|10|1|2045|581|7572|Friday|2045Q1|N|Y|N|2468013|2468043|2467656|2467930|N|N|N|N|N| +2468023|AAAAAAAAHLIKFCAA|2045-02-11|1741|7572|581|2045|6|2|11|1|2045|581|7572|Saturday|2045Q1|N|Y|N|2468013|2468043|2467657|2467931|N|N|N|N|N| +2468024|AAAAAAAAILIKFCAA|2045-02-12|1741|7572|581|2045|0|2|12|1|2045|581|7572|Sunday|2045Q1|N|N|N|2468013|2468043|2467658|2467932|N|N|N|N|N| +2468025|AAAAAAAAJLIKFCAA|2045-02-13|1741|7572|581|2045|1|2|13|1|2045|581|7572|Monday|2045Q1|N|N|N|2468013|2468043|2467659|2467933|N|N|N|N|N| +2468026|AAAAAAAAKLIKFCAA|2045-02-14|1741|7573|581|2045|2|2|14|1|2045|581|7573|Tuesday|2045Q1|N|N|N|2468013|2468043|2467660|2467934|N|N|N|N|N| +2468027|AAAAAAAALLIKFCAA|2045-02-15|1741|7573|581|2045|3|2|15|1|2045|581|7573|Wednesday|2045Q1|N|N|N|2468013|2468043|2467661|2467935|N|N|N|N|N| +2468028|AAAAAAAAMLIKFCAA|2045-02-16|1741|7573|581|2045|4|2|16|1|2045|581|7573|Thursday|2045Q1|N|N|N|2468013|2468043|2467662|2467936|N|N|N|N|N| +2468029|AAAAAAAANLIKFCAA|2045-02-17|1741|7573|581|2045|5|2|17|1|2045|581|7573|Friday|2045Q1|N|Y|N|2468013|2468043|2467663|2467937|N|N|N|N|N| +2468030|AAAAAAAAOLIKFCAA|2045-02-18|1741|7573|581|2045|6|2|18|1|2045|581|7573|Saturday|2045Q1|N|Y|N|2468013|2468043|2467664|2467938|N|N|N|N|N| +2468031|AAAAAAAAPLIKFCAA|2045-02-19|1741|7573|581|2045|0|2|19|1|2045|581|7573|Sunday|2045Q1|N|N|N|2468013|2468043|2467665|2467939|N|N|N|N|N| +2468032|AAAAAAAAAMIKFCAA|2045-02-20|1741|7573|581|2045|1|2|20|1|2045|581|7573|Monday|2045Q1|N|N|N|2468013|2468043|2467666|2467940|N|N|N|N|N| +2468033|AAAAAAAABMIKFCAA|2045-02-21|1741|7574|581|2045|2|2|21|1|2045|581|7574|Tuesday|2045Q1|N|N|N|2468013|2468043|2467667|2467941|N|N|N|N|N| +2468034|AAAAAAAACMIKFCAA|2045-02-22|1741|7574|581|2045|3|2|22|1|2045|581|7574|Wednesday|2045Q1|N|N|N|2468013|2468043|2467668|2467942|N|N|N|N|N| +2468035|AAAAAAAADMIKFCAA|2045-02-23|1741|7574|581|2045|4|2|23|1|2045|581|7574|Thursday|2045Q1|N|N|N|2468013|2468043|2467669|2467943|N|N|N|N|N| +2468036|AAAAAAAAEMIKFCAA|2045-02-24|1741|7574|581|2045|5|2|24|1|2045|581|7574|Friday|2045Q1|N|Y|N|2468013|2468043|2467670|2467944|N|N|N|N|N| +2468037|AAAAAAAAFMIKFCAA|2045-02-25|1741|7574|581|2045|6|2|25|1|2045|581|7574|Saturday|2045Q1|N|Y|N|2468013|2468043|2467671|2467945|N|N|N|N|N| +2468038|AAAAAAAAGMIKFCAA|2045-02-26|1741|7574|581|2045|0|2|26|1|2045|581|7574|Sunday|2045Q1|N|N|N|2468013|2468043|2467672|2467946|N|N|N|N|N| +2468039|AAAAAAAAHMIKFCAA|2045-02-27|1741|7574|581|2045|1|2|27|1|2045|581|7574|Monday|2045Q1|N|N|N|2468013|2468043|2467673|2467947|N|N|N|N|N| +2468040|AAAAAAAAIMIKFCAA|2045-02-28|1741|7575|581|2045|2|2|28|1|2045|581|7575|Tuesday|2045Q1|N|N|N|2468013|2468043|2467674|2467948|N|N|N|N|N| +2468041|AAAAAAAAJMIKFCAA|2045-03-01|1742|7575|582|2045|3|3|1|1|2045|582|7575|Wednesday|2045Q1|N|N|N|2468041|2468099|2467676|2467949|N|N|N|N|N| +2468042|AAAAAAAAKMIKFCAA|2045-03-02|1742|7575|582|2045|4|3|2|1|2045|582|7575|Thursday|2045Q1|N|N|N|2468041|2468099|2467677|2467950|N|N|N|N|N| +2468043|AAAAAAAALMIKFCAA|2045-03-03|1742|7575|582|2045|5|3|3|1|2045|582|7575|Friday|2045Q1|N|Y|N|2468041|2468099|2467678|2467951|N|N|N|N|N| +2468044|AAAAAAAAMMIKFCAA|2045-03-04|1742|7575|582|2045|6|3|4|1|2045|582|7575|Saturday|2045Q1|N|Y|N|2468041|2468099|2467679|2467952|N|N|N|N|N| +2468045|AAAAAAAANMIKFCAA|2045-03-05|1742|7575|582|2045|0|3|5|1|2045|582|7575|Sunday|2045Q1|N|N|N|2468041|2468099|2467680|2467953|N|N|N|N|N| +2468046|AAAAAAAAOMIKFCAA|2045-03-06|1742|7575|582|2045|1|3|6|1|2045|582|7575|Monday|2045Q1|N|N|N|2468041|2468099|2467681|2467954|N|N|N|N|N| +2468047|AAAAAAAAPMIKFCAA|2045-03-07|1742|7576|582|2045|2|3|7|1|2045|582|7576|Tuesday|2045Q1|N|N|N|2468041|2468099|2467682|2467955|N|N|N|N|N| +2468048|AAAAAAAAANIKFCAA|2045-03-08|1742|7576|582|2045|3|3|8|1|2045|582|7576|Wednesday|2045Q1|N|N|N|2468041|2468099|2467683|2467956|N|N|N|N|N| +2468049|AAAAAAAABNIKFCAA|2045-03-09|1742|7576|582|2045|4|3|9|1|2045|582|7576|Thursday|2045Q1|N|N|N|2468041|2468099|2467684|2467957|N|N|N|N|N| +2468050|AAAAAAAACNIKFCAA|2045-03-10|1742|7576|582|2045|5|3|10|1|2045|582|7576|Friday|2045Q1|N|Y|N|2468041|2468099|2467685|2467958|N|N|N|N|N| +2468051|AAAAAAAADNIKFCAA|2045-03-11|1742|7576|582|2045|6|3|11|1|2045|582|7576|Saturday|2045Q1|N|Y|N|2468041|2468099|2467686|2467959|N|N|N|N|N| +2468052|AAAAAAAAENIKFCAA|2045-03-12|1742|7576|582|2045|0|3|12|1|2045|582|7576|Sunday|2045Q1|N|N|N|2468041|2468099|2467687|2467960|N|N|N|N|N| +2468053|AAAAAAAAFNIKFCAA|2045-03-13|1742|7576|582|2045|1|3|13|1|2045|582|7576|Monday|2045Q1|N|N|N|2468041|2468099|2467688|2467961|N|N|N|N|N| +2468054|AAAAAAAAGNIKFCAA|2045-03-14|1742|7577|582|2045|2|3|14|1|2045|582|7577|Tuesday|2045Q1|N|N|N|2468041|2468099|2467689|2467962|N|N|N|N|N| +2468055|AAAAAAAAHNIKFCAA|2045-03-15|1742|7577|582|2045|3|3|15|1|2045|582|7577|Wednesday|2045Q1|N|N|N|2468041|2468099|2467690|2467963|N|N|N|N|N| +2468056|AAAAAAAAINIKFCAA|2045-03-16|1742|7577|582|2045|4|3|16|1|2045|582|7577|Thursday|2045Q1|N|N|N|2468041|2468099|2467691|2467964|N|N|N|N|N| +2468057|AAAAAAAAJNIKFCAA|2045-03-17|1742|7577|582|2045|5|3|17|1|2045|582|7577|Friday|2045Q1|N|Y|N|2468041|2468099|2467692|2467965|N|N|N|N|N| +2468058|AAAAAAAAKNIKFCAA|2045-03-18|1742|7577|582|2045|6|3|18|1|2045|582|7577|Saturday|2045Q1|N|Y|N|2468041|2468099|2467693|2467966|N|N|N|N|N| +2468059|AAAAAAAALNIKFCAA|2045-03-19|1742|7577|582|2045|0|3|19|1|2045|582|7577|Sunday|2045Q1|N|N|N|2468041|2468099|2467694|2467967|N|N|N|N|N| +2468060|AAAAAAAAMNIKFCAA|2045-03-20|1742|7577|582|2045|1|3|20|1|2045|582|7577|Monday|2045Q1|N|N|N|2468041|2468099|2467695|2467968|N|N|N|N|N| +2468061|AAAAAAAANNIKFCAA|2045-03-21|1742|7578|582|2045|2|3|21|1|2045|582|7578|Tuesday|2045Q1|N|N|N|2468041|2468099|2467696|2467969|N|N|N|N|N| +2468062|AAAAAAAAONIKFCAA|2045-03-22|1742|7578|582|2045|3|3|22|1|2045|582|7578|Wednesday|2045Q1|N|N|N|2468041|2468099|2467697|2467970|N|N|N|N|N| +2468063|AAAAAAAAPNIKFCAA|2045-03-23|1742|7578|582|2045|4|3|23|1|2045|582|7578|Thursday|2045Q1|N|N|N|2468041|2468099|2467698|2467971|N|N|N|N|N| +2468064|AAAAAAAAAOIKFCAA|2045-03-24|1742|7578|582|2045|5|3|24|1|2045|582|7578|Friday|2045Q1|N|Y|N|2468041|2468099|2467699|2467972|N|N|N|N|N| +2468065|AAAAAAAABOIKFCAA|2045-03-25|1742|7578|582|2045|6|3|25|1|2045|582|7578|Saturday|2045Q1|N|Y|N|2468041|2468099|2467700|2467973|N|N|N|N|N| +2468066|AAAAAAAACOIKFCAA|2045-03-26|1742|7578|582|2045|0|3|26|1|2045|582|7578|Sunday|2045Q1|N|N|N|2468041|2468099|2467701|2467974|N|N|N|N|N| +2468067|AAAAAAAADOIKFCAA|2045-03-27|1742|7578|582|2045|1|3|27|1|2045|582|7578|Monday|2045Q1|N|N|N|2468041|2468099|2467702|2467975|N|N|N|N|N| +2468068|AAAAAAAAEOIKFCAA|2045-03-28|1742|7579|582|2045|2|3|28|1|2045|582|7579|Tuesday|2045Q1|N|N|N|2468041|2468099|2467703|2467976|N|N|N|N|N| +2468069|AAAAAAAAFOIKFCAA|2045-03-29|1742|7579|582|2045|3|3|29|1|2045|582|7579|Wednesday|2045Q1|N|N|N|2468041|2468099|2467704|2467977|N|N|N|N|N| +2468070|AAAAAAAAGOIKFCAA|2045-03-30|1742|7579|582|2045|4|3|30|1|2045|582|7579|Thursday|2045Q1|N|N|N|2468041|2468099|2467705|2467978|N|N|N|N|N| +2468071|AAAAAAAAHOIKFCAA|2045-03-31|1742|7579|582|2045|5|3|31|1|2045|582|7579|Friday|2045Q1|N|Y|N|2468041|2468099|2467706|2467979|N|N|N|N|N| +2468072|AAAAAAAAIOIKFCAA|2045-04-01|1743|7579|582|2045|6|4|1|1|2045|582|7579|Saturday|2045Q1|N|Y|N|2468072|2468161|2467707|2467982|N|N|N|N|N| +2468073|AAAAAAAAJOIKFCAA|2045-04-02|1743|7579|582|2045|0|4|2|2|2045|582|7579|Sunday|2045Q2|N|N|N|2468072|2468161|2467708|2467983|N|N|N|N|N| +2468074|AAAAAAAAKOIKFCAA|2045-04-03|1743|7579|582|2045|1|4|3|2|2045|582|7579|Monday|2045Q2|N|N|N|2468072|2468161|2467709|2467984|N|N|N|N|N| +2468075|AAAAAAAALOIKFCAA|2045-04-04|1743|7580|582|2045|2|4|4|2|2045|582|7580|Tuesday|2045Q2|N|N|N|2468072|2468161|2467710|2467985|N|N|N|N|N| +2468076|AAAAAAAAMOIKFCAA|2045-04-05|1743|7580|582|2045|3|4|5|2|2045|582|7580|Wednesday|2045Q2|N|N|N|2468072|2468161|2467711|2467986|N|N|N|N|N| +2468077|AAAAAAAANOIKFCAA|2045-04-06|1743|7580|582|2045|4|4|6|2|2045|582|7580|Thursday|2045Q2|N|N|N|2468072|2468161|2467712|2467987|N|N|N|N|N| +2468078|AAAAAAAAOOIKFCAA|2045-04-07|1743|7580|582|2045|5|4|7|2|2045|582|7580|Friday|2045Q2|N|Y|N|2468072|2468161|2467713|2467988|N|N|N|N|N| +2468079|AAAAAAAAPOIKFCAA|2045-04-08|1743|7580|582|2045|6|4|8|2|2045|582|7580|Saturday|2045Q2|N|Y|N|2468072|2468161|2467714|2467989|N|N|N|N|N| +2468080|AAAAAAAAAPIKFCAA|2045-04-09|1743|7580|582|2045|0|4|9|2|2045|582|7580|Sunday|2045Q2|N|N|N|2468072|2468161|2467715|2467990|N|N|N|N|N| +2468081|AAAAAAAABPIKFCAA|2045-04-10|1743|7580|582|2045|1|4|10|2|2045|582|7580|Monday|2045Q2|N|N|N|2468072|2468161|2467716|2467991|N|N|N|N|N| +2468082|AAAAAAAACPIKFCAA|2045-04-11|1743|7581|582|2045|2|4|11|2|2045|582|7581|Tuesday|2045Q2|N|N|N|2468072|2468161|2467717|2467992|N|N|N|N|N| +2468083|AAAAAAAADPIKFCAA|2045-04-12|1743|7581|582|2045|3|4|12|2|2045|582|7581|Wednesday|2045Q2|N|N|N|2468072|2468161|2467718|2467993|N|N|N|N|N| +2468084|AAAAAAAAEPIKFCAA|2045-04-13|1743|7581|582|2045|4|4|13|2|2045|582|7581|Thursday|2045Q2|N|N|N|2468072|2468161|2467719|2467994|N|N|N|N|N| +2468085|AAAAAAAAFPIKFCAA|2045-04-14|1743|7581|582|2045|5|4|14|2|2045|582|7581|Friday|2045Q2|N|Y|N|2468072|2468161|2467720|2467995|N|N|N|N|N| +2468086|AAAAAAAAGPIKFCAA|2045-04-15|1743|7581|582|2045|6|4|15|2|2045|582|7581|Saturday|2045Q2|N|Y|N|2468072|2468161|2467721|2467996|N|N|N|N|N| +2468087|AAAAAAAAHPIKFCAA|2045-04-16|1743|7581|582|2045|0|4|16|2|2045|582|7581|Sunday|2045Q2|N|N|N|2468072|2468161|2467722|2467997|N|N|N|N|N| +2468088|AAAAAAAAIPIKFCAA|2045-04-17|1743|7581|582|2045|1|4|17|2|2045|582|7581|Monday|2045Q2|N|N|N|2468072|2468161|2467723|2467998|N|N|N|N|N| +2468089|AAAAAAAAJPIKFCAA|2045-04-18|1743|7582|582|2045|2|4|18|2|2045|582|7582|Tuesday|2045Q2|N|N|N|2468072|2468161|2467724|2467999|N|N|N|N|N| +2468090|AAAAAAAAKPIKFCAA|2045-04-19|1743|7582|582|2045|3|4|19|2|2045|582|7582|Wednesday|2045Q2|N|N|N|2468072|2468161|2467725|2468000|N|N|N|N|N| +2468091|AAAAAAAALPIKFCAA|2045-04-20|1743|7582|582|2045|4|4|20|2|2045|582|7582|Thursday|2045Q2|N|N|N|2468072|2468161|2467726|2468001|N|N|N|N|N| +2468092|AAAAAAAAMPIKFCAA|2045-04-21|1743|7582|582|2045|5|4|21|2|2045|582|7582|Friday|2045Q2|N|Y|N|2468072|2468161|2467727|2468002|N|N|N|N|N| +2468093|AAAAAAAANPIKFCAA|2045-04-22|1743|7582|582|2045|6|4|22|2|2045|582|7582|Saturday|2045Q2|N|Y|N|2468072|2468161|2467728|2468003|N|N|N|N|N| +2468094|AAAAAAAAOPIKFCAA|2045-04-23|1743|7582|582|2045|0|4|23|2|2045|582|7582|Sunday|2045Q2|N|N|N|2468072|2468161|2467729|2468004|N|N|N|N|N| +2468095|AAAAAAAAPPIKFCAA|2045-04-24|1743|7582|582|2045|1|4|24|2|2045|582|7582|Monday|2045Q2|N|N|N|2468072|2468161|2467730|2468005|N|N|N|N|N| +2468096|AAAAAAAAAAJKFCAA|2045-04-25|1743|7583|582|2045|2|4|25|2|2045|582|7583|Tuesday|2045Q2|N|N|N|2468072|2468161|2467731|2468006|N|N|N|N|N| +2468097|AAAAAAAABAJKFCAA|2045-04-26|1743|7583|582|2045|3|4|26|2|2045|582|7583|Wednesday|2045Q2|N|N|N|2468072|2468161|2467732|2468007|N|N|N|N|N| +2468098|AAAAAAAACAJKFCAA|2045-04-27|1743|7583|582|2045|4|4|27|2|2045|582|7583|Thursday|2045Q2|N|N|N|2468072|2468161|2467733|2468008|N|N|N|N|N| +2468099|AAAAAAAADAJKFCAA|2045-04-28|1743|7583|582|2045|5|4|28|2|2045|582|7583|Friday|2045Q2|N|Y|N|2468072|2468161|2467734|2468009|N|N|N|N|N| +2468100|AAAAAAAAEAJKFCAA|2045-04-29|1743|7583|582|2045|6|4|29|2|2045|582|7583|Saturday|2045Q2|N|Y|N|2468072|2468161|2467735|2468010|N|N|N|N|N| +2468101|AAAAAAAAFAJKFCAA|2045-04-30|1743|7583|582|2045|0|4|30|2|2045|582|7583|Sunday|2045Q2|N|N|N|2468072|2468161|2467736|2468011|N|N|N|N|N| +2468102|AAAAAAAAGAJKFCAA|2045-05-01|1744|7583|582|2045|1|5|1|2|2045|582|7583|Monday|2045Q2|N|N|N|2468102|2468221|2467737|2468012|N|N|N|N|N| +2468103|AAAAAAAAHAJKFCAA|2045-05-02|1744|7584|582|2045|2|5|2|2|2045|582|7584|Tuesday|2045Q2|N|N|N|2468102|2468221|2467738|2468013|N|N|N|N|N| +2468104|AAAAAAAAIAJKFCAA|2045-05-03|1744|7584|582|2045|3|5|3|2|2045|582|7584|Wednesday|2045Q2|N|N|N|2468102|2468221|2467739|2468014|N|N|N|N|N| +2468105|AAAAAAAAJAJKFCAA|2045-05-04|1744|7584|582|2045|4|5|4|2|2045|582|7584|Thursday|2045Q2|N|N|N|2468102|2468221|2467740|2468015|N|N|N|N|N| +2468106|AAAAAAAAKAJKFCAA|2045-05-05|1744|7584|582|2045|5|5|5|2|2045|582|7584|Friday|2045Q2|N|Y|N|2468102|2468221|2467741|2468016|N|N|N|N|N| +2468107|AAAAAAAALAJKFCAA|2045-05-06|1744|7584|582|2045|6|5|6|2|2045|582|7584|Saturday|2045Q2|N|Y|N|2468102|2468221|2467742|2468017|N|N|N|N|N| +2468108|AAAAAAAAMAJKFCAA|2045-05-07|1744|7584|582|2045|0|5|7|2|2045|582|7584|Sunday|2045Q2|N|N|N|2468102|2468221|2467743|2468018|N|N|N|N|N| +2468109|AAAAAAAANAJKFCAA|2045-05-08|1744|7584|582|2045|1|5|8|2|2045|582|7584|Monday|2045Q2|N|N|N|2468102|2468221|2467744|2468019|N|N|N|N|N| +2468110|AAAAAAAAOAJKFCAA|2045-05-09|1744|7585|582|2045|2|5|9|2|2045|582|7585|Tuesday|2045Q2|N|N|N|2468102|2468221|2467745|2468020|N|N|N|N|N| +2468111|AAAAAAAAPAJKFCAA|2045-05-10|1744|7585|582|2045|3|5|10|2|2045|582|7585|Wednesday|2045Q2|N|N|N|2468102|2468221|2467746|2468021|N|N|N|N|N| +2468112|AAAAAAAAABJKFCAA|2045-05-11|1744|7585|582|2045|4|5|11|2|2045|582|7585|Thursday|2045Q2|N|N|N|2468102|2468221|2467747|2468022|N|N|N|N|N| +2468113|AAAAAAAABBJKFCAA|2045-05-12|1744|7585|582|2045|5|5|12|2|2045|582|7585|Friday|2045Q2|N|Y|N|2468102|2468221|2467748|2468023|N|N|N|N|N| +2468114|AAAAAAAACBJKFCAA|2045-05-13|1744|7585|582|2045|6|5|13|2|2045|582|7585|Saturday|2045Q2|N|Y|N|2468102|2468221|2467749|2468024|N|N|N|N|N| +2468115|AAAAAAAADBJKFCAA|2045-05-14|1744|7585|582|2045|0|5|14|2|2045|582|7585|Sunday|2045Q2|N|N|N|2468102|2468221|2467750|2468025|N|N|N|N|N| +2468116|AAAAAAAAEBJKFCAA|2045-05-15|1744|7585|582|2045|1|5|15|2|2045|582|7585|Monday|2045Q2|N|N|N|2468102|2468221|2467751|2468026|N|N|N|N|N| +2468117|AAAAAAAAFBJKFCAA|2045-05-16|1744|7586|582|2045|2|5|16|2|2045|582|7586|Tuesday|2045Q2|N|N|N|2468102|2468221|2467752|2468027|N|N|N|N|N| +2468118|AAAAAAAAGBJKFCAA|2045-05-17|1744|7586|582|2045|3|5|17|2|2045|582|7586|Wednesday|2045Q2|N|N|N|2468102|2468221|2467753|2468028|N|N|N|N|N| +2468119|AAAAAAAAHBJKFCAA|2045-05-18|1744|7586|582|2045|4|5|18|2|2045|582|7586|Thursday|2045Q2|N|N|N|2468102|2468221|2467754|2468029|N|N|N|N|N| +2468120|AAAAAAAAIBJKFCAA|2045-05-19|1744|7586|582|2045|5|5|19|2|2045|582|7586|Friday|2045Q2|N|Y|N|2468102|2468221|2467755|2468030|N|N|N|N|N| +2468121|AAAAAAAAJBJKFCAA|2045-05-20|1744|7586|582|2045|6|5|20|2|2045|582|7586|Saturday|2045Q2|N|Y|N|2468102|2468221|2467756|2468031|N|N|N|N|N| +2468122|AAAAAAAAKBJKFCAA|2045-05-21|1744|7586|582|2045|0|5|21|2|2045|582|7586|Sunday|2045Q2|N|N|N|2468102|2468221|2467757|2468032|N|N|N|N|N| +2468123|AAAAAAAALBJKFCAA|2045-05-22|1744|7586|582|2045|1|5|22|2|2045|582|7586|Monday|2045Q2|N|N|N|2468102|2468221|2467758|2468033|N|N|N|N|N| +2468124|AAAAAAAAMBJKFCAA|2045-05-23|1744|7587|582|2045|2|5|23|2|2045|582|7587|Tuesday|2045Q2|N|N|N|2468102|2468221|2467759|2468034|N|N|N|N|N| +2468125|AAAAAAAANBJKFCAA|2045-05-24|1744|7587|582|2045|3|5|24|2|2045|582|7587|Wednesday|2045Q2|N|N|N|2468102|2468221|2467760|2468035|N|N|N|N|N| +2468126|AAAAAAAAOBJKFCAA|2045-05-25|1744|7587|582|2045|4|5|25|2|2045|582|7587|Thursday|2045Q2|N|N|N|2468102|2468221|2467761|2468036|N|N|N|N|N| +2468127|AAAAAAAAPBJKFCAA|2045-05-26|1744|7587|582|2045|5|5|26|2|2045|582|7587|Friday|2045Q2|N|Y|N|2468102|2468221|2467762|2468037|N|N|N|N|N| +2468128|AAAAAAAAACJKFCAA|2045-05-27|1744|7587|582|2045|6|5|27|2|2045|582|7587|Saturday|2045Q2|N|Y|N|2468102|2468221|2467763|2468038|N|N|N|N|N| +2468129|AAAAAAAABCJKFCAA|2045-05-28|1744|7587|582|2045|0|5|28|2|2045|582|7587|Sunday|2045Q2|N|N|N|2468102|2468221|2467764|2468039|N|N|N|N|N| +2468130|AAAAAAAACCJKFCAA|2045-05-29|1744|7587|582|2045|1|5|29|2|2045|582|7587|Monday|2045Q2|N|N|N|2468102|2468221|2467765|2468040|N|N|N|N|N| +2468131|AAAAAAAADCJKFCAA|2045-05-30|1744|7588|582|2045|2|5|30|2|2045|582|7588|Tuesday|2045Q2|N|N|N|2468102|2468221|2467766|2468041|N|N|N|N|N| +2468132|AAAAAAAAECJKFCAA|2045-05-31|1744|7588|582|2045|3|5|31|2|2045|582|7588|Wednesday|2045Q2|N|N|N|2468102|2468221|2467767|2468042|N|N|N|N|N| +2468133|AAAAAAAAFCJKFCAA|2045-06-01|1745|7588|583|2045|4|6|1|2|2045|583|7588|Thursday|2045Q2|N|N|N|2468133|2468283|2467768|2468043|N|N|N|N|N| +2468134|AAAAAAAAGCJKFCAA|2045-06-02|1745|7588|583|2045|5|6|2|2|2045|583|7588|Friday|2045Q2|N|Y|N|2468133|2468283|2467769|2468044|N|N|N|N|N| +2468135|AAAAAAAAHCJKFCAA|2045-06-03|1745|7588|583|2045|6|6|3|2|2045|583|7588|Saturday|2045Q2|N|Y|N|2468133|2468283|2467770|2468045|N|N|N|N|N| +2468136|AAAAAAAAICJKFCAA|2045-06-04|1745|7588|583|2045|0|6|4|2|2045|583|7588|Sunday|2045Q2|N|N|N|2468133|2468283|2467771|2468046|N|N|N|N|N| +2468137|AAAAAAAAJCJKFCAA|2045-06-05|1745|7588|583|2045|1|6|5|2|2045|583|7588|Monday|2045Q2|N|N|N|2468133|2468283|2467772|2468047|N|N|N|N|N| +2468138|AAAAAAAAKCJKFCAA|2045-06-06|1745|7589|583|2045|2|6|6|2|2045|583|7589|Tuesday|2045Q2|N|N|N|2468133|2468283|2467773|2468048|N|N|N|N|N| +2468139|AAAAAAAALCJKFCAA|2045-06-07|1745|7589|583|2045|3|6|7|2|2045|583|7589|Wednesday|2045Q2|N|N|N|2468133|2468283|2467774|2468049|N|N|N|N|N| +2468140|AAAAAAAAMCJKFCAA|2045-06-08|1745|7589|583|2045|4|6|8|2|2045|583|7589|Thursday|2045Q2|N|N|N|2468133|2468283|2467775|2468050|N|N|N|N|N| +2468141|AAAAAAAANCJKFCAA|2045-06-09|1745|7589|583|2045|5|6|9|2|2045|583|7589|Friday|2045Q2|N|Y|N|2468133|2468283|2467776|2468051|N|N|N|N|N| +2468142|AAAAAAAAOCJKFCAA|2045-06-10|1745|7589|583|2045|6|6|10|2|2045|583|7589|Saturday|2045Q2|N|Y|N|2468133|2468283|2467777|2468052|N|N|N|N|N| +2468143|AAAAAAAAPCJKFCAA|2045-06-11|1745|7589|583|2045|0|6|11|2|2045|583|7589|Sunday|2045Q2|N|N|N|2468133|2468283|2467778|2468053|N|N|N|N|N| +2468144|AAAAAAAAADJKFCAA|2045-06-12|1745|7589|583|2045|1|6|12|2|2045|583|7589|Monday|2045Q2|N|N|N|2468133|2468283|2467779|2468054|N|N|N|N|N| +2468145|AAAAAAAABDJKFCAA|2045-06-13|1745|7590|583|2045|2|6|13|2|2045|583|7590|Tuesday|2045Q2|N|N|N|2468133|2468283|2467780|2468055|N|N|N|N|N| +2468146|AAAAAAAACDJKFCAA|2045-06-14|1745|7590|583|2045|3|6|14|2|2045|583|7590|Wednesday|2045Q2|N|N|N|2468133|2468283|2467781|2468056|N|N|N|N|N| +2468147|AAAAAAAADDJKFCAA|2045-06-15|1745|7590|583|2045|4|6|15|2|2045|583|7590|Thursday|2045Q2|N|N|N|2468133|2468283|2467782|2468057|N|N|N|N|N| +2468148|AAAAAAAAEDJKFCAA|2045-06-16|1745|7590|583|2045|5|6|16|2|2045|583|7590|Friday|2045Q2|N|Y|N|2468133|2468283|2467783|2468058|N|N|N|N|N| +2468149|AAAAAAAAFDJKFCAA|2045-06-17|1745|7590|583|2045|6|6|17|2|2045|583|7590|Saturday|2045Q2|N|Y|N|2468133|2468283|2467784|2468059|N|N|N|N|N| +2468150|AAAAAAAAGDJKFCAA|2045-06-18|1745|7590|583|2045|0|6|18|2|2045|583|7590|Sunday|2045Q2|N|N|N|2468133|2468283|2467785|2468060|N|N|N|N|N| +2468151|AAAAAAAAHDJKFCAA|2045-06-19|1745|7590|583|2045|1|6|19|2|2045|583|7590|Monday|2045Q2|N|N|N|2468133|2468283|2467786|2468061|N|N|N|N|N| +2468152|AAAAAAAAIDJKFCAA|2045-06-20|1745|7591|583|2045|2|6|20|2|2045|583|7591|Tuesday|2045Q2|N|N|N|2468133|2468283|2467787|2468062|N|N|N|N|N| +2468153|AAAAAAAAJDJKFCAA|2045-06-21|1745|7591|583|2045|3|6|21|2|2045|583|7591|Wednesday|2045Q2|N|N|N|2468133|2468283|2467788|2468063|N|N|N|N|N| +2468154|AAAAAAAAKDJKFCAA|2045-06-22|1745|7591|583|2045|4|6|22|2|2045|583|7591|Thursday|2045Q2|N|N|N|2468133|2468283|2467789|2468064|N|N|N|N|N| +2468155|AAAAAAAALDJKFCAA|2045-06-23|1745|7591|583|2045|5|6|23|2|2045|583|7591|Friday|2045Q2|N|Y|N|2468133|2468283|2467790|2468065|N|N|N|N|N| +2468156|AAAAAAAAMDJKFCAA|2045-06-24|1745|7591|583|2045|6|6|24|2|2045|583|7591|Saturday|2045Q2|N|Y|N|2468133|2468283|2467791|2468066|N|N|N|N|N| +2468157|AAAAAAAANDJKFCAA|2045-06-25|1745|7591|583|2045|0|6|25|2|2045|583|7591|Sunday|2045Q2|N|N|N|2468133|2468283|2467792|2468067|N|N|N|N|N| +2468158|AAAAAAAAODJKFCAA|2045-06-26|1745|7591|583|2045|1|6|26|2|2045|583|7591|Monday|2045Q2|N|N|N|2468133|2468283|2467793|2468068|N|N|N|N|N| +2468159|AAAAAAAAPDJKFCAA|2045-06-27|1745|7592|583|2045|2|6|27|2|2045|583|7592|Tuesday|2045Q2|N|N|N|2468133|2468283|2467794|2468069|N|N|N|N|N| +2468160|AAAAAAAAAEJKFCAA|2045-06-28|1745|7592|583|2045|3|6|28|2|2045|583|7592|Wednesday|2045Q2|N|N|N|2468133|2468283|2467795|2468070|N|N|N|N|N| +2468161|AAAAAAAABEJKFCAA|2045-06-29|1745|7592|583|2045|4|6|29|2|2045|583|7592|Thursday|2045Q2|N|N|N|2468133|2468283|2467796|2468071|N|N|N|N|N| +2468162|AAAAAAAACEJKFCAA|2045-06-30|1745|7592|583|2045|5|6|30|2|2045|583|7592|Friday|2045Q2|N|Y|N|2468133|2468283|2467797|2468072|N|N|N|N|N| +2468163|AAAAAAAADEJKFCAA|2045-07-01|1746|7592|583|2045|6|7|1|2|2045|583|7592|Saturday|2045Q2|N|Y|N|2468163|2468343|2467798|2468072|N|N|N|N|N| +2468164|AAAAAAAAEEJKFCAA|2045-07-02|1746|7592|583|2045|0|7|2|3|2045|583|7592|Sunday|2045Q3|N|N|N|2468163|2468343|2467799|2468073|N|N|N|N|N| +2468165|AAAAAAAAFEJKFCAA|2045-07-03|1746|7592|583|2045|1|7|3|3|2045|583|7592|Monday|2045Q3|N|N|N|2468163|2468343|2467800|2468074|N|N|N|N|N| +2468166|AAAAAAAAGEJKFCAA|2045-07-04|1746|7593|583|2045|2|7|4|3|2045|583|7593|Tuesday|2045Q3|N|N|N|2468163|2468343|2467801|2468075|N|N|N|N|N| +2468167|AAAAAAAAHEJKFCAA|2045-07-05|1746|7593|583|2045|3|7|5|3|2045|583|7593|Wednesday|2045Q3|Y|N|N|2468163|2468343|2467802|2468076|N|N|N|N|N| +2468168|AAAAAAAAIEJKFCAA|2045-07-06|1746|7593|583|2045|4|7|6|3|2045|583|7593|Thursday|2045Q3|N|N|Y|2468163|2468343|2467803|2468077|N|N|N|N|N| +2468169|AAAAAAAAJEJKFCAA|2045-07-07|1746|7593|583|2045|5|7|7|3|2045|583|7593|Friday|2045Q3|N|Y|N|2468163|2468343|2467804|2468078|N|N|N|N|N| +2468170|AAAAAAAAKEJKFCAA|2045-07-08|1746|7593|583|2045|6|7|8|3|2045|583|7593|Saturday|2045Q3|N|Y|N|2468163|2468343|2467805|2468079|N|N|N|N|N| +2468171|AAAAAAAALEJKFCAA|2045-07-09|1746|7593|583|2045|0|7|9|3|2045|583|7593|Sunday|2045Q3|N|N|N|2468163|2468343|2467806|2468080|N|N|N|N|N| +2468172|AAAAAAAAMEJKFCAA|2045-07-10|1746|7593|583|2045|1|7|10|3|2045|583|7593|Monday|2045Q3|N|N|N|2468163|2468343|2467807|2468081|N|N|N|N|N| +2468173|AAAAAAAANEJKFCAA|2045-07-11|1746|7594|583|2045|2|7|11|3|2045|583|7594|Tuesday|2045Q3|N|N|N|2468163|2468343|2467808|2468082|N|N|N|N|N| +2468174|AAAAAAAAOEJKFCAA|2045-07-12|1746|7594|583|2045|3|7|12|3|2045|583|7594|Wednesday|2045Q3|N|N|N|2468163|2468343|2467809|2468083|N|N|N|N|N| +2468175|AAAAAAAAPEJKFCAA|2045-07-13|1746|7594|583|2045|4|7|13|3|2045|583|7594|Thursday|2045Q3|N|N|N|2468163|2468343|2467810|2468084|N|N|N|N|N| +2468176|AAAAAAAAAFJKFCAA|2045-07-14|1746|7594|583|2045|5|7|14|3|2045|583|7594|Friday|2045Q3|N|Y|N|2468163|2468343|2467811|2468085|N|N|N|N|N| +2468177|AAAAAAAABFJKFCAA|2045-07-15|1746|7594|583|2045|6|7|15|3|2045|583|7594|Saturday|2045Q3|N|Y|N|2468163|2468343|2467812|2468086|N|N|N|N|N| +2468178|AAAAAAAACFJKFCAA|2045-07-16|1746|7594|583|2045|0|7|16|3|2045|583|7594|Sunday|2045Q3|N|N|N|2468163|2468343|2467813|2468087|N|N|N|N|N| +2468179|AAAAAAAADFJKFCAA|2045-07-17|1746|7594|583|2045|1|7|17|3|2045|583|7594|Monday|2045Q3|N|N|N|2468163|2468343|2467814|2468088|N|N|N|N|N| +2468180|AAAAAAAAEFJKFCAA|2045-07-18|1746|7595|583|2045|2|7|18|3|2045|583|7595|Tuesday|2045Q3|N|N|N|2468163|2468343|2467815|2468089|N|N|N|N|N| +2468181|AAAAAAAAFFJKFCAA|2045-07-19|1746|7595|583|2045|3|7|19|3|2045|583|7595|Wednesday|2045Q3|N|N|N|2468163|2468343|2467816|2468090|N|N|N|N|N| +2468182|AAAAAAAAGFJKFCAA|2045-07-20|1746|7595|583|2045|4|7|20|3|2045|583|7595|Thursday|2045Q3|N|N|N|2468163|2468343|2467817|2468091|N|N|N|N|N| +2468183|AAAAAAAAHFJKFCAA|2045-07-21|1746|7595|583|2045|5|7|21|3|2045|583|7595|Friday|2045Q3|N|Y|N|2468163|2468343|2467818|2468092|N|N|N|N|N| +2468184|AAAAAAAAIFJKFCAA|2045-07-22|1746|7595|583|2045|6|7|22|3|2045|583|7595|Saturday|2045Q3|N|Y|N|2468163|2468343|2467819|2468093|N|N|N|N|N| +2468185|AAAAAAAAJFJKFCAA|2045-07-23|1746|7595|583|2045|0|7|23|3|2045|583|7595|Sunday|2045Q3|N|N|N|2468163|2468343|2467820|2468094|N|N|N|N|N| +2468186|AAAAAAAAKFJKFCAA|2045-07-24|1746|7595|583|2045|1|7|24|3|2045|583|7595|Monday|2045Q3|N|N|N|2468163|2468343|2467821|2468095|N|N|N|N|N| +2468187|AAAAAAAALFJKFCAA|2045-07-25|1746|7596|583|2045|2|7|25|3|2045|583|7596|Tuesday|2045Q3|N|N|N|2468163|2468343|2467822|2468096|N|N|N|N|N| +2468188|AAAAAAAAMFJKFCAA|2045-07-26|1746|7596|583|2045|3|7|26|3|2045|583|7596|Wednesday|2045Q3|N|N|N|2468163|2468343|2467823|2468097|N|N|N|N|N| +2468189|AAAAAAAANFJKFCAA|2045-07-27|1746|7596|583|2045|4|7|27|3|2045|583|7596|Thursday|2045Q3|N|N|N|2468163|2468343|2467824|2468098|N|N|N|N|N| +2468190|AAAAAAAAOFJKFCAA|2045-07-28|1746|7596|583|2045|5|7|28|3|2045|583|7596|Friday|2045Q3|N|Y|N|2468163|2468343|2467825|2468099|N|N|N|N|N| +2468191|AAAAAAAAPFJKFCAA|2045-07-29|1746|7596|583|2045|6|7|29|3|2045|583|7596|Saturday|2045Q3|N|Y|N|2468163|2468343|2467826|2468100|N|N|N|N|N| +2468192|AAAAAAAAAGJKFCAA|2045-07-30|1746|7596|583|2045|0|7|30|3|2045|583|7596|Sunday|2045Q3|N|N|N|2468163|2468343|2467827|2468101|N|N|N|N|N| +2468193|AAAAAAAABGJKFCAA|2045-07-31|1746|7596|583|2045|1|7|31|3|2045|583|7596|Monday|2045Q3|N|N|N|2468163|2468343|2467828|2468102|N|N|N|N|N| +2468194|AAAAAAAACGJKFCAA|2045-08-01|1747|7597|583|2045|2|8|1|3|2045|583|7597|Tuesday|2045Q3|N|N|N|2468194|2468405|2467829|2468103|N|N|N|N|N| +2468195|AAAAAAAADGJKFCAA|2045-08-02|1747|7597|583|2045|3|8|2|3|2045|583|7597|Wednesday|2045Q3|N|N|N|2468194|2468405|2467830|2468104|N|N|N|N|N| +2468196|AAAAAAAAEGJKFCAA|2045-08-03|1747|7597|583|2045|4|8|3|3|2045|583|7597|Thursday|2045Q3|N|N|N|2468194|2468405|2467831|2468105|N|N|N|N|N| +2468197|AAAAAAAAFGJKFCAA|2045-08-04|1747|7597|583|2045|5|8|4|3|2045|583|7597|Friday|2045Q3|N|Y|N|2468194|2468405|2467832|2468106|N|N|N|N|N| +2468198|AAAAAAAAGGJKFCAA|2045-08-05|1747|7597|583|2045|6|8|5|3|2045|583|7597|Saturday|2045Q3|N|Y|N|2468194|2468405|2467833|2468107|N|N|N|N|N| +2468199|AAAAAAAAHGJKFCAA|2045-08-06|1747|7597|583|2045|0|8|6|3|2045|583|7597|Sunday|2045Q3|N|N|N|2468194|2468405|2467834|2468108|N|N|N|N|N| +2468200|AAAAAAAAIGJKFCAA|2045-08-07|1747|7597|583|2045|1|8|7|3|2045|583|7597|Monday|2045Q3|N|N|N|2468194|2468405|2467835|2468109|N|N|N|N|N| +2468201|AAAAAAAAJGJKFCAA|2045-08-08|1747|7598|583|2045|2|8|8|3|2045|583|7598|Tuesday|2045Q3|N|N|N|2468194|2468405|2467836|2468110|N|N|N|N|N| +2468202|AAAAAAAAKGJKFCAA|2045-08-09|1747|7598|583|2045|3|8|9|3|2045|583|7598|Wednesday|2045Q3|N|N|N|2468194|2468405|2467837|2468111|N|N|N|N|N| +2468203|AAAAAAAALGJKFCAA|2045-08-10|1747|7598|583|2045|4|8|10|3|2045|583|7598|Thursday|2045Q3|N|N|N|2468194|2468405|2467838|2468112|N|N|N|N|N| +2468204|AAAAAAAAMGJKFCAA|2045-08-11|1747|7598|583|2045|5|8|11|3|2045|583|7598|Friday|2045Q3|N|Y|N|2468194|2468405|2467839|2468113|N|N|N|N|N| +2468205|AAAAAAAANGJKFCAA|2045-08-12|1747|7598|583|2045|6|8|12|3|2045|583|7598|Saturday|2045Q3|N|Y|N|2468194|2468405|2467840|2468114|N|N|N|N|N| +2468206|AAAAAAAAOGJKFCAA|2045-08-13|1747|7598|583|2045|0|8|13|3|2045|583|7598|Sunday|2045Q3|N|N|N|2468194|2468405|2467841|2468115|N|N|N|N|N| +2468207|AAAAAAAAPGJKFCAA|2045-08-14|1747|7598|583|2045|1|8|14|3|2045|583|7598|Monday|2045Q3|N|N|N|2468194|2468405|2467842|2468116|N|N|N|N|N| +2468208|AAAAAAAAAHJKFCAA|2045-08-15|1747|7599|583|2045|2|8|15|3|2045|583|7599|Tuesday|2045Q3|N|N|N|2468194|2468405|2467843|2468117|N|N|N|N|N| +2468209|AAAAAAAABHJKFCAA|2045-08-16|1747|7599|583|2045|3|8|16|3|2045|583|7599|Wednesday|2045Q3|N|N|N|2468194|2468405|2467844|2468118|N|N|N|N|N| +2468210|AAAAAAAACHJKFCAA|2045-08-17|1747|7599|583|2045|4|8|17|3|2045|583|7599|Thursday|2045Q3|N|N|N|2468194|2468405|2467845|2468119|N|N|N|N|N| +2468211|AAAAAAAADHJKFCAA|2045-08-18|1747|7599|583|2045|5|8|18|3|2045|583|7599|Friday|2045Q3|N|Y|N|2468194|2468405|2467846|2468120|N|N|N|N|N| +2468212|AAAAAAAAEHJKFCAA|2045-08-19|1747|7599|583|2045|6|8|19|3|2045|583|7599|Saturday|2045Q3|N|Y|N|2468194|2468405|2467847|2468121|N|N|N|N|N| +2468213|AAAAAAAAFHJKFCAA|2045-08-20|1747|7599|583|2045|0|8|20|3|2045|583|7599|Sunday|2045Q3|N|N|N|2468194|2468405|2467848|2468122|N|N|N|N|N| +2468214|AAAAAAAAGHJKFCAA|2045-08-21|1747|7599|583|2045|1|8|21|3|2045|583|7599|Monday|2045Q3|N|N|N|2468194|2468405|2467849|2468123|N|N|N|N|N| +2468215|AAAAAAAAHHJKFCAA|2045-08-22|1747|7600|583|2045|2|8|22|3|2045|583|7600|Tuesday|2045Q3|N|N|N|2468194|2468405|2467850|2468124|N|N|N|N|N| +2468216|AAAAAAAAIHJKFCAA|2045-08-23|1747|7600|583|2045|3|8|23|3|2045|583|7600|Wednesday|2045Q3|N|N|N|2468194|2468405|2467851|2468125|N|N|N|N|N| +2468217|AAAAAAAAJHJKFCAA|2045-08-24|1747|7600|583|2045|4|8|24|3|2045|583|7600|Thursday|2045Q3|N|N|N|2468194|2468405|2467852|2468126|N|N|N|N|N| +2468218|AAAAAAAAKHJKFCAA|2045-08-25|1747|7600|583|2045|5|8|25|3|2045|583|7600|Friday|2045Q3|N|Y|N|2468194|2468405|2467853|2468127|N|N|N|N|N| +2468219|AAAAAAAALHJKFCAA|2045-08-26|1747|7600|583|2045|6|8|26|3|2045|583|7600|Saturday|2045Q3|N|Y|N|2468194|2468405|2467854|2468128|N|N|N|N|N| +2468220|AAAAAAAAMHJKFCAA|2045-08-27|1747|7600|583|2045|0|8|27|3|2045|583|7600|Sunday|2045Q3|N|N|N|2468194|2468405|2467855|2468129|N|N|N|N|N| +2468221|AAAAAAAANHJKFCAA|2045-08-28|1747|7600|583|2045|1|8|28|3|2045|583|7600|Monday|2045Q3|N|N|N|2468194|2468405|2467856|2468130|N|N|N|N|N| +2468222|AAAAAAAAOHJKFCAA|2045-08-29|1747|7601|583|2045|2|8|29|3|2045|583|7601|Tuesday|2045Q3|N|N|N|2468194|2468405|2467857|2468131|N|N|N|N|N| +2468223|AAAAAAAAPHJKFCAA|2045-08-30|1747|7601|583|2045|3|8|30|3|2045|583|7601|Wednesday|2045Q3|N|N|N|2468194|2468405|2467858|2468132|N|N|N|N|N| +2468224|AAAAAAAAAIJKFCAA|2045-08-31|1747|7601|583|2045|4|8|31|3|2045|583|7601|Thursday|2045Q3|N|N|N|2468194|2468405|2467859|2468133|N|N|N|N|N| +2468225|AAAAAAAABIJKFCAA|2045-09-01|1748|7601|584|2045|5|9|1|3|2045|584|7601|Friday|2045Q3|N|Y|N|2468225|2468467|2467860|2468134|N|N|N|N|N| +2468226|AAAAAAAACIJKFCAA|2045-09-02|1748|7601|584|2045|6|9|2|3|2045|584|7601|Saturday|2045Q3|N|Y|N|2468225|2468467|2467861|2468135|N|N|N|N|N| +2468227|AAAAAAAADIJKFCAA|2045-09-03|1748|7601|584|2045|0|9|3|3|2045|584|7601|Sunday|2045Q3|N|N|N|2468225|2468467|2467862|2468136|N|N|N|N|N| +2468228|AAAAAAAAEIJKFCAA|2045-09-04|1748|7601|584|2045|1|9|4|3|2045|584|7601|Monday|2045Q3|N|N|N|2468225|2468467|2467863|2468137|N|N|N|N|N| +2468229|AAAAAAAAFIJKFCAA|2045-09-05|1748|7602|584|2045|2|9|5|3|2045|584|7602|Tuesday|2045Q3|N|N|N|2468225|2468467|2467864|2468138|N|N|N|N|N| +2468230|AAAAAAAAGIJKFCAA|2045-09-06|1748|7602|584|2045|3|9|6|3|2045|584|7602|Wednesday|2045Q3|N|N|N|2468225|2468467|2467865|2468139|N|N|N|N|N| +2468231|AAAAAAAAHIJKFCAA|2045-09-07|1748|7602|584|2045|4|9|7|3|2045|584|7602|Thursday|2045Q3|N|N|N|2468225|2468467|2467866|2468140|N|N|N|N|N| +2468232|AAAAAAAAIIJKFCAA|2045-09-08|1748|7602|584|2045|5|9|8|3|2045|584|7602|Friday|2045Q3|N|Y|N|2468225|2468467|2467867|2468141|N|N|N|N|N| +2468233|AAAAAAAAJIJKFCAA|2045-09-09|1748|7602|584|2045|6|9|9|3|2045|584|7602|Saturday|2045Q3|N|Y|N|2468225|2468467|2467868|2468142|N|N|N|N|N| +2468234|AAAAAAAAKIJKFCAA|2045-09-10|1748|7602|584|2045|0|9|10|3|2045|584|7602|Sunday|2045Q3|N|N|N|2468225|2468467|2467869|2468143|N|N|N|N|N| +2468235|AAAAAAAALIJKFCAA|2045-09-11|1748|7602|584|2045|1|9|11|3|2045|584|7602|Monday|2045Q3|N|N|N|2468225|2468467|2467870|2468144|N|N|N|N|N| +2468236|AAAAAAAAMIJKFCAA|2045-09-12|1748|7603|584|2045|2|9|12|3|2045|584|7603|Tuesday|2045Q3|N|N|N|2468225|2468467|2467871|2468145|N|N|N|N|N| +2468237|AAAAAAAANIJKFCAA|2045-09-13|1748|7603|584|2045|3|9|13|3|2045|584|7603|Wednesday|2045Q3|N|N|N|2468225|2468467|2467872|2468146|N|N|N|N|N| +2468238|AAAAAAAAOIJKFCAA|2045-09-14|1748|7603|584|2045|4|9|14|3|2045|584|7603|Thursday|2045Q3|N|N|N|2468225|2468467|2467873|2468147|N|N|N|N|N| +2468239|AAAAAAAAPIJKFCAA|2045-09-15|1748|7603|584|2045|5|9|15|3|2045|584|7603|Friday|2045Q3|N|Y|N|2468225|2468467|2467874|2468148|N|N|N|N|N| +2468240|AAAAAAAAAJJKFCAA|2045-09-16|1748|7603|584|2045|6|9|16|3|2045|584|7603|Saturday|2045Q3|N|Y|N|2468225|2468467|2467875|2468149|N|N|N|N|N| +2468241|AAAAAAAABJJKFCAA|2045-09-17|1748|7603|584|2045|0|9|17|3|2045|584|7603|Sunday|2045Q3|N|N|N|2468225|2468467|2467876|2468150|N|N|N|N|N| +2468242|AAAAAAAACJJKFCAA|2045-09-18|1748|7603|584|2045|1|9|18|3|2045|584|7603|Monday|2045Q3|N|N|N|2468225|2468467|2467877|2468151|N|N|N|N|N| +2468243|AAAAAAAADJJKFCAA|2045-09-19|1748|7604|584|2045|2|9|19|3|2045|584|7604|Tuesday|2045Q3|N|N|N|2468225|2468467|2467878|2468152|N|N|N|N|N| +2468244|AAAAAAAAEJJKFCAA|2045-09-20|1748|7604|584|2045|3|9|20|3|2045|584|7604|Wednesday|2045Q3|N|N|N|2468225|2468467|2467879|2468153|N|N|N|N|N| +2468245|AAAAAAAAFJJKFCAA|2045-09-21|1748|7604|584|2045|4|9|21|3|2045|584|7604|Thursday|2045Q3|N|N|N|2468225|2468467|2467880|2468154|N|N|N|N|N| +2468246|AAAAAAAAGJJKFCAA|2045-09-22|1748|7604|584|2045|5|9|22|3|2045|584|7604|Friday|2045Q3|N|Y|N|2468225|2468467|2467881|2468155|N|N|N|N|N| +2468247|AAAAAAAAHJJKFCAA|2045-09-23|1748|7604|584|2045|6|9|23|3|2045|584|7604|Saturday|2045Q3|N|Y|N|2468225|2468467|2467882|2468156|N|N|N|N|N| +2468248|AAAAAAAAIJJKFCAA|2045-09-24|1748|7604|584|2045|0|9|24|3|2045|584|7604|Sunday|2045Q3|N|N|N|2468225|2468467|2467883|2468157|N|N|N|N|N| +2468249|AAAAAAAAJJJKFCAA|2045-09-25|1748|7604|584|2045|1|9|25|3|2045|584|7604|Monday|2045Q3|N|N|N|2468225|2468467|2467884|2468158|N|N|N|N|N| +2468250|AAAAAAAAKJJKFCAA|2045-09-26|1748|7605|584|2045|2|9|26|3|2045|584|7605|Tuesday|2045Q3|N|N|N|2468225|2468467|2467885|2468159|N|N|N|N|N| +2468251|AAAAAAAALJJKFCAA|2045-09-27|1748|7605|584|2045|3|9|27|3|2045|584|7605|Wednesday|2045Q3|N|N|N|2468225|2468467|2467886|2468160|N|N|N|N|N| +2468252|AAAAAAAAMJJKFCAA|2045-09-28|1748|7605|584|2045|4|9|28|3|2045|584|7605|Thursday|2045Q3|N|N|N|2468225|2468467|2467887|2468161|N|N|N|N|N| +2468253|AAAAAAAANJJKFCAA|2045-09-29|1748|7605|584|2045|5|9|29|3|2045|584|7605|Friday|2045Q3|N|Y|N|2468225|2468467|2467888|2468162|N|N|N|N|N| +2468254|AAAAAAAAOJJKFCAA|2045-09-30|1748|7605|584|2045|6|9|30|3|2045|584|7605|Saturday|2045Q3|N|Y|N|2468225|2468467|2467889|2468163|N|N|N|N|N| +2468255|AAAAAAAAPJJKFCAA|2045-10-01|1749|7605|584|2045|0|10|1|3|2045|584|7605|Sunday|2045Q3|N|N|N|2468255|2468527|2467890|2468163|N|N|N|N|N| +2468256|AAAAAAAAAKJKFCAA|2045-10-02|1749|7605|584|2045|1|10|2|4|2045|584|7605|Monday|2045Q4|N|N|N|2468255|2468527|2467891|2468164|N|N|N|N|N| +2468257|AAAAAAAABKJKFCAA|2045-10-03|1749|7606|584|2045|2|10|3|4|2045|584|7606|Tuesday|2045Q4|N|N|N|2468255|2468527|2467892|2468165|N|N|N|N|N| +2468258|AAAAAAAACKJKFCAA|2045-10-04|1749|7606|584|2045|3|10|4|4|2045|584|7606|Wednesday|2045Q4|N|N|N|2468255|2468527|2467893|2468166|N|N|N|N|N| +2468259|AAAAAAAADKJKFCAA|2045-10-05|1749|7606|584|2045|4|10|5|4|2045|584|7606|Thursday|2045Q4|N|N|N|2468255|2468527|2467894|2468167|N|N|N|N|N| +2468260|AAAAAAAAEKJKFCAA|2045-10-06|1749|7606|584|2045|5|10|6|4|2045|584|7606|Friday|2045Q4|N|Y|N|2468255|2468527|2467895|2468168|N|N|N|N|N| +2468261|AAAAAAAAFKJKFCAA|2045-10-07|1749|7606|584|2045|6|10|7|4|2045|584|7606|Saturday|2045Q4|N|Y|N|2468255|2468527|2467896|2468169|N|N|N|N|N| +2468262|AAAAAAAAGKJKFCAA|2045-10-08|1749|7606|584|2045|0|10|8|4|2045|584|7606|Sunday|2045Q4|N|N|N|2468255|2468527|2467897|2468170|N|N|N|N|N| +2468263|AAAAAAAAHKJKFCAA|2045-10-09|1749|7606|584|2045|1|10|9|4|2045|584|7606|Monday|2045Q4|N|N|N|2468255|2468527|2467898|2468171|N|N|N|N|N| +2468264|AAAAAAAAIKJKFCAA|2045-10-10|1749|7607|584|2045|2|10|10|4|2045|584|7607|Tuesday|2045Q4|N|N|N|2468255|2468527|2467899|2468172|N|N|N|N|N| +2468265|AAAAAAAAJKJKFCAA|2045-10-11|1749|7607|584|2045|3|10|11|4|2045|584|7607|Wednesday|2045Q4|N|N|N|2468255|2468527|2467900|2468173|N|N|N|N|N| +2468266|AAAAAAAAKKJKFCAA|2045-10-12|1749|7607|584|2045|4|10|12|4|2045|584|7607|Thursday|2045Q4|N|N|N|2468255|2468527|2467901|2468174|N|N|N|N|N| +2468267|AAAAAAAALKJKFCAA|2045-10-13|1749|7607|584|2045|5|10|13|4|2045|584|7607|Friday|2045Q4|N|Y|N|2468255|2468527|2467902|2468175|N|N|N|N|N| +2468268|AAAAAAAAMKJKFCAA|2045-10-14|1749|7607|584|2045|6|10|14|4|2045|584|7607|Saturday|2045Q4|N|Y|N|2468255|2468527|2467903|2468176|N|N|N|N|N| +2468269|AAAAAAAANKJKFCAA|2045-10-15|1749|7607|584|2045|0|10|15|4|2045|584|7607|Sunday|2045Q4|N|N|N|2468255|2468527|2467904|2468177|N|N|N|N|N| +2468270|AAAAAAAAOKJKFCAA|2045-10-16|1749|7607|584|2045|1|10|16|4|2045|584|7607|Monday|2045Q4|N|N|N|2468255|2468527|2467905|2468178|N|N|N|N|N| +2468271|AAAAAAAAPKJKFCAA|2045-10-17|1749|7608|584|2045|2|10|17|4|2045|584|7608|Tuesday|2045Q4|N|N|N|2468255|2468527|2467906|2468179|N|N|N|N|N| +2468272|AAAAAAAAALJKFCAA|2045-10-18|1749|7608|584|2045|3|10|18|4|2045|584|7608|Wednesday|2045Q4|N|N|N|2468255|2468527|2467907|2468180|N|N|N|N|N| +2468273|AAAAAAAABLJKFCAA|2045-10-19|1749|7608|584|2045|4|10|19|4|2045|584|7608|Thursday|2045Q4|N|N|N|2468255|2468527|2467908|2468181|N|N|N|N|N| +2468274|AAAAAAAACLJKFCAA|2045-10-20|1749|7608|584|2045|5|10|20|4|2045|584|7608|Friday|2045Q4|N|Y|N|2468255|2468527|2467909|2468182|N|N|N|N|N| +2468275|AAAAAAAADLJKFCAA|2045-10-21|1749|7608|584|2045|6|10|21|4|2045|584|7608|Saturday|2045Q4|N|Y|N|2468255|2468527|2467910|2468183|N|N|N|N|N| +2468276|AAAAAAAAELJKFCAA|2045-10-22|1749|7608|584|2045|0|10|22|4|2045|584|7608|Sunday|2045Q4|N|N|N|2468255|2468527|2467911|2468184|N|N|N|N|N| +2468277|AAAAAAAAFLJKFCAA|2045-10-23|1749|7608|584|2045|1|10|23|4|2045|584|7608|Monday|2045Q4|N|N|N|2468255|2468527|2467912|2468185|N|N|N|N|N| +2468278|AAAAAAAAGLJKFCAA|2045-10-24|1749|7609|584|2045|2|10|24|4|2045|584|7609|Tuesday|2045Q4|N|N|N|2468255|2468527|2467913|2468186|N|N|N|N|N| +2468279|AAAAAAAAHLJKFCAA|2045-10-25|1749|7609|584|2045|3|10|25|4|2045|584|7609|Wednesday|2045Q4|N|N|N|2468255|2468527|2467914|2468187|N|N|N|N|N| +2468280|AAAAAAAAILJKFCAA|2045-10-26|1749|7609|584|2045|4|10|26|4|2045|584|7609|Thursday|2045Q4|N|N|N|2468255|2468527|2467915|2468188|N|N|N|N|N| +2468281|AAAAAAAAJLJKFCAA|2045-10-27|1749|7609|584|2045|5|10|27|4|2045|584|7609|Friday|2045Q4|N|Y|N|2468255|2468527|2467916|2468189|N|N|N|N|N| +2468282|AAAAAAAAKLJKFCAA|2045-10-28|1749|7609|584|2045|6|10|28|4|2045|584|7609|Saturday|2045Q4|N|Y|N|2468255|2468527|2467917|2468190|N|N|N|N|N| +2468283|AAAAAAAALLJKFCAA|2045-10-29|1749|7609|584|2045|0|10|29|4|2045|584|7609|Sunday|2045Q4|N|N|N|2468255|2468527|2467918|2468191|N|N|N|N|N| +2468284|AAAAAAAAMLJKFCAA|2045-10-30|1749|7609|584|2045|1|10|30|4|2045|584|7609|Monday|2045Q4|N|N|N|2468255|2468527|2467919|2468192|N|N|N|N|N| +2468285|AAAAAAAANLJKFCAA|2045-10-31|1749|7610|584|2045|2|10|31|4|2045|584|7610|Tuesday|2045Q4|N|N|N|2468255|2468527|2467920|2468193|N|N|N|N|N| +2468286|AAAAAAAAOLJKFCAA|2045-11-01|1750|7610|584|2045|3|11|1|4|2045|584|7610|Wednesday|2045Q4|N|N|N|2468286|2468589|2467921|2468194|N|N|N|N|N| +2468287|AAAAAAAAPLJKFCAA|2045-11-02|1750|7610|584|2045|4|11|2|4|2045|584|7610|Thursday|2045Q4|N|N|N|2468286|2468589|2467922|2468195|N|N|N|N|N| +2468288|AAAAAAAAAMJKFCAA|2045-11-03|1750|7610|584|2045|5|11|3|4|2045|584|7610|Friday|2045Q4|N|Y|N|2468286|2468589|2467923|2468196|N|N|N|N|N| +2468289|AAAAAAAABMJKFCAA|2045-11-04|1750|7610|584|2045|6|11|4|4|2045|584|7610|Saturday|2045Q4|N|Y|N|2468286|2468589|2467924|2468197|N|N|N|N|N| +2468290|AAAAAAAACMJKFCAA|2045-11-05|1750|7610|584|2045|0|11|5|4|2045|584|7610|Sunday|2045Q4|N|N|N|2468286|2468589|2467925|2468198|N|N|N|N|N| +2468291|AAAAAAAADMJKFCAA|2045-11-06|1750|7610|584|2045|1|11|6|4|2045|584|7610|Monday|2045Q4|N|N|N|2468286|2468589|2467926|2468199|N|N|N|N|N| +2468292|AAAAAAAAEMJKFCAA|2045-11-07|1750|7611|584|2045|2|11|7|4|2045|584|7611|Tuesday|2045Q4|N|N|N|2468286|2468589|2467927|2468200|N|N|N|N|N| +2468293|AAAAAAAAFMJKFCAA|2045-11-08|1750|7611|584|2045|3|11|8|4|2045|584|7611|Wednesday|2045Q4|N|N|N|2468286|2468589|2467928|2468201|N|N|N|N|N| +2468294|AAAAAAAAGMJKFCAA|2045-11-09|1750|7611|584|2045|4|11|9|4|2045|584|7611|Thursday|2045Q4|N|N|N|2468286|2468589|2467929|2468202|N|N|N|N|N| +2468295|AAAAAAAAHMJKFCAA|2045-11-10|1750|7611|584|2045|5|11|10|4|2045|584|7611|Friday|2045Q4|N|Y|N|2468286|2468589|2467930|2468203|N|N|N|N|N| +2468296|AAAAAAAAIMJKFCAA|2045-11-11|1750|7611|584|2045|6|11|11|4|2045|584|7611|Saturday|2045Q4|N|Y|N|2468286|2468589|2467931|2468204|N|N|N|N|N| +2468297|AAAAAAAAJMJKFCAA|2045-11-12|1750|7611|584|2045|0|11|12|4|2045|584|7611|Sunday|2045Q4|N|N|N|2468286|2468589|2467932|2468205|N|N|N|N|N| +2468298|AAAAAAAAKMJKFCAA|2045-11-13|1750|7611|584|2045|1|11|13|4|2045|584|7611|Monday|2045Q4|N|N|N|2468286|2468589|2467933|2468206|N|N|N|N|N| +2468299|AAAAAAAALMJKFCAA|2045-11-14|1750|7612|584|2045|2|11|14|4|2045|584|7612|Tuesday|2045Q4|N|N|N|2468286|2468589|2467934|2468207|N|N|N|N|N| +2468300|AAAAAAAAMMJKFCAA|2045-11-15|1750|7612|584|2045|3|11|15|4|2045|584|7612|Wednesday|2045Q4|N|N|N|2468286|2468589|2467935|2468208|N|N|N|N|N| +2468301|AAAAAAAANMJKFCAA|2045-11-16|1750|7612|584|2045|4|11|16|4|2045|584|7612|Thursday|2045Q4|N|N|N|2468286|2468589|2467936|2468209|N|N|N|N|N| +2468302|AAAAAAAAOMJKFCAA|2045-11-17|1750|7612|584|2045|5|11|17|4|2045|584|7612|Friday|2045Q4|N|Y|N|2468286|2468589|2467937|2468210|N|N|N|N|N| +2468303|AAAAAAAAPMJKFCAA|2045-11-18|1750|7612|584|2045|6|11|18|4|2045|584|7612|Saturday|2045Q4|N|Y|N|2468286|2468589|2467938|2468211|N|N|N|N|N| +2468304|AAAAAAAAANJKFCAA|2045-11-19|1750|7612|584|2045|0|11|19|4|2045|584|7612|Sunday|2045Q4|N|N|N|2468286|2468589|2467939|2468212|N|N|N|N|N| +2468305|AAAAAAAABNJKFCAA|2045-11-20|1750|7612|584|2045|1|11|20|4|2045|584|7612|Monday|2045Q4|N|N|N|2468286|2468589|2467940|2468213|N|N|N|N|N| +2468306|AAAAAAAACNJKFCAA|2045-11-21|1750|7613|584|2045|2|11|21|4|2045|584|7613|Tuesday|2045Q4|N|N|N|2468286|2468589|2467941|2468214|N|N|N|N|N| +2468307|AAAAAAAADNJKFCAA|2045-11-22|1750|7613|584|2045|3|11|22|4|2045|584|7613|Wednesday|2045Q4|N|N|N|2468286|2468589|2467942|2468215|N|N|N|N|N| +2468308|AAAAAAAAENJKFCAA|2045-11-23|1750|7613|584|2045|4|11|23|4|2045|584|7613|Thursday|2045Q4|N|N|N|2468286|2468589|2467943|2468216|N|N|N|N|N| +2468309|AAAAAAAAFNJKFCAA|2045-11-24|1750|7613|584|2045|5|11|24|4|2045|584|7613|Friday|2045Q4|N|Y|N|2468286|2468589|2467944|2468217|N|N|N|N|N| +2468310|AAAAAAAAGNJKFCAA|2045-11-25|1750|7613|584|2045|6|11|25|4|2045|584|7613|Saturday|2045Q4|N|Y|N|2468286|2468589|2467945|2468218|N|N|N|N|N| +2468311|AAAAAAAAHNJKFCAA|2045-11-26|1750|7613|584|2045|0|11|26|4|2045|584|7613|Sunday|2045Q4|N|N|N|2468286|2468589|2467946|2468219|N|N|N|N|N| +2468312|AAAAAAAAINJKFCAA|2045-11-27|1750|7613|584|2045|1|11|27|4|2045|584|7613|Monday|2045Q4|N|N|N|2468286|2468589|2467947|2468220|N|N|N|N|N| +2468313|AAAAAAAAJNJKFCAA|2045-11-28|1750|7614|584|2045|2|11|28|4|2045|584|7614|Tuesday|2045Q4|N|N|N|2468286|2468589|2467948|2468221|N|N|N|N|N| +2468314|AAAAAAAAKNJKFCAA|2045-11-29|1750|7614|584|2045|3|11|29|4|2045|584|7614|Wednesday|2045Q4|N|N|N|2468286|2468589|2467949|2468222|N|N|N|N|N| +2468315|AAAAAAAALNJKFCAA|2045-11-30|1750|7614|584|2045|4|11|30|4|2045|584|7614|Thursday|2045Q4|N|N|N|2468286|2468589|2467950|2468223|N|N|N|N|N| +2468316|AAAAAAAAMNJKFCAA|2045-12-01|1751|7614|585|2045|5|12|1|4|2045|585|7614|Friday|2045Q4|N|Y|N|2468316|2468649|2467951|2468224|N|N|N|N|N| +2468317|AAAAAAAANNJKFCAA|2045-12-02|1751|7614|585|2045|6|12|2|4|2045|585|7614|Saturday|2045Q4|N|Y|N|2468316|2468649|2467952|2468225|N|N|N|N|N| +2468318|AAAAAAAAONJKFCAA|2045-12-03|1751|7614|585|2045|0|12|3|4|2045|585|7614|Sunday|2045Q4|N|N|N|2468316|2468649|2467953|2468226|N|N|N|N|N| +2468319|AAAAAAAAPNJKFCAA|2045-12-04|1751|7614|585|2045|1|12|4|4|2045|585|7614|Monday|2045Q4|N|N|N|2468316|2468649|2467954|2468227|N|N|N|N|N| +2468320|AAAAAAAAAOJKFCAA|2045-12-05|1751|7615|585|2045|2|12|5|4|2045|585|7615|Tuesday|2045Q4|N|N|N|2468316|2468649|2467955|2468228|N|N|N|N|N| +2468321|AAAAAAAABOJKFCAA|2045-12-06|1751|7615|585|2045|3|12|6|4|2045|585|7615|Wednesday|2045Q4|N|N|N|2468316|2468649|2467956|2468229|N|N|N|N|N| +2468322|AAAAAAAACOJKFCAA|2045-12-07|1751|7615|585|2045|4|12|7|4|2045|585|7615|Thursday|2045Q4|N|N|N|2468316|2468649|2467957|2468230|N|N|N|N|N| +2468323|AAAAAAAADOJKFCAA|2045-12-08|1751|7615|585|2045|5|12|8|4|2045|585|7615|Friday|2045Q4|N|Y|N|2468316|2468649|2467958|2468231|N|N|N|N|N| +2468324|AAAAAAAAEOJKFCAA|2045-12-09|1751|7615|585|2045|6|12|9|4|2045|585|7615|Saturday|2045Q4|N|Y|N|2468316|2468649|2467959|2468232|N|N|N|N|N| +2468325|AAAAAAAAFOJKFCAA|2045-12-10|1751|7615|585|2045|0|12|10|4|2045|585|7615|Sunday|2045Q4|N|N|N|2468316|2468649|2467960|2468233|N|N|N|N|N| +2468326|AAAAAAAAGOJKFCAA|2045-12-11|1751|7615|585|2045|1|12|11|4|2045|585|7615|Monday|2045Q4|N|N|N|2468316|2468649|2467961|2468234|N|N|N|N|N| +2468327|AAAAAAAAHOJKFCAA|2045-12-12|1751|7616|585|2045|2|12|12|4|2045|585|7616|Tuesday|2045Q4|N|N|N|2468316|2468649|2467962|2468235|N|N|N|N|N| +2468328|AAAAAAAAIOJKFCAA|2045-12-13|1751|7616|585|2045|3|12|13|4|2045|585|7616|Wednesday|2045Q4|N|N|N|2468316|2468649|2467963|2468236|N|N|N|N|N| +2468329|AAAAAAAAJOJKFCAA|2045-12-14|1751|7616|585|2045|4|12|14|4|2045|585|7616|Thursday|2045Q4|N|N|N|2468316|2468649|2467964|2468237|N|N|N|N|N| +2468330|AAAAAAAAKOJKFCAA|2045-12-15|1751|7616|585|2045|5|12|15|4|2045|585|7616|Friday|2045Q4|N|Y|N|2468316|2468649|2467965|2468238|N|N|N|N|N| +2468331|AAAAAAAALOJKFCAA|2045-12-16|1751|7616|585|2045|6|12|16|4|2045|585|7616|Saturday|2045Q4|N|Y|N|2468316|2468649|2467966|2468239|N|N|N|N|N| +2468332|AAAAAAAAMOJKFCAA|2045-12-17|1751|7616|585|2045|0|12|17|4|2045|585|7616|Sunday|2045Q4|N|N|N|2468316|2468649|2467967|2468240|N|N|N|N|N| +2468333|AAAAAAAANOJKFCAA|2045-12-18|1751|7616|585|2045|1|12|18|4|2045|585|7616|Monday|2045Q4|N|N|N|2468316|2468649|2467968|2468241|N|N|N|N|N| +2468334|AAAAAAAAOOJKFCAA|2045-12-19|1751|7617|585|2045|2|12|19|4|2045|585|7617|Tuesday|2045Q4|N|N|N|2468316|2468649|2467969|2468242|N|N|N|N|N| +2468335|AAAAAAAAPOJKFCAA|2045-12-20|1751|7617|585|2045|3|12|20|4|2045|585|7617|Wednesday|2045Q4|N|N|N|2468316|2468649|2467970|2468243|N|N|N|N|N| +2468336|AAAAAAAAAPJKFCAA|2045-12-21|1751|7617|585|2045|4|12|21|4|2045|585|7617|Thursday|2045Q4|N|N|N|2468316|2468649|2467971|2468244|N|N|N|N|N| +2468337|AAAAAAAABPJKFCAA|2045-12-22|1751|7617|585|2045|5|12|22|4|2045|585|7617|Friday|2045Q4|N|Y|N|2468316|2468649|2467972|2468245|N|N|N|N|N| +2468338|AAAAAAAACPJKFCAA|2045-12-23|1751|7617|585|2045|6|12|23|4|2045|585|7617|Saturday|2045Q4|N|Y|N|2468316|2468649|2467973|2468246|N|N|N|N|N| +2468339|AAAAAAAADPJKFCAA|2045-12-24|1751|7617|585|2045|0|12|24|4|2045|585|7617|Sunday|2045Q4|N|N|N|2468316|2468649|2467974|2468247|N|N|N|N|N| +2468340|AAAAAAAAEPJKFCAA|2045-12-25|1751|7617|585|2045|1|12|25|4|2045|585|7617|Monday|2045Q4|N|N|N|2468316|2468649|2467975|2468248|N|N|N|N|N| +2468341|AAAAAAAAFPJKFCAA|2045-12-26|1751|7618|585|2045|2|12|26|4|2045|585|7618|Tuesday|2045Q4|Y|N|N|2468316|2468649|2467976|2468249|N|N|N|N|N| +2468342|AAAAAAAAGPJKFCAA|2045-12-27|1751|7618|585|2045|3|12|27|4|2045|585|7618|Wednesday|2045Q4|N|N|Y|2468316|2468649|2467977|2468250|N|N|N|N|N| +2468343|AAAAAAAAHPJKFCAA|2045-12-28|1751|7618|585|2045|4|12|28|4|2045|585|7618|Thursday|2045Q4|N|N|N|2468316|2468649|2467978|2468251|N|N|N|N|N| +2468344|AAAAAAAAIPJKFCAA|2045-12-29|1751|7618|585|2045|5|12|29|4|2045|585|7618|Friday|2045Q4|N|Y|N|2468316|2468649|2467979|2468252|N|N|N|N|N| +2468345|AAAAAAAAJPJKFCAA|2045-12-30|1751|7618|585|2045|6|12|30|4|2045|585|7618|Saturday|2045Q4|N|Y|N|2468316|2468649|2467980|2468253|N|N|N|N|N| +2468346|AAAAAAAAKPJKFCAA|2045-12-31|1751|7618|585|2045|0|12|31|4|2045|585|7618|Sunday|2045Q4|N|N|N|2468316|2468649|2467981|2468254|N|N|N|N|N| +2468347|AAAAAAAALPJKFCAA|2046-01-01|1752|7618|585|2046|1|1|1|1|2046|585|7618|Monday|2046Q1|Y|N|N|2468347|2468346|2467982|2468255|N|N|N|N|N| +2468348|AAAAAAAAMPJKFCAA|2046-01-02|1752|7619|585|2046|2|1|2|1|2046|585|7619|Tuesday|2046Q1|N|N|Y|2468347|2468346|2467983|2468256|N|N|N|N|N| +2468349|AAAAAAAANPJKFCAA|2046-01-03|1752|7619|585|2046|3|1|3|1|2046|585|7619|Wednesday|2046Q1|N|N|N|2468347|2468346|2467984|2468257|N|N|N|N|N| +2468350|AAAAAAAAOPJKFCAA|2046-01-04|1752|7619|585|2046|4|1|4|1|2046|585|7619|Thursday|2046Q1|N|N|N|2468347|2468346|2467985|2468258|N|N|N|N|N| +2468351|AAAAAAAAPPJKFCAA|2046-01-05|1752|7619|585|2046|5|1|5|1|2046|585|7619|Friday|2046Q1|N|Y|N|2468347|2468346|2467986|2468259|N|N|N|N|N| +2468352|AAAAAAAAAAKKFCAA|2046-01-06|1752|7619|585|2046|6|1|6|1|2046|585|7619|Saturday|2046Q1|N|Y|N|2468347|2468346|2467987|2468260|N|N|N|N|N| +2468353|AAAAAAAABAKKFCAA|2046-01-07|1752|7619|585|2046|0|1|7|1|2046|585|7619|Sunday|2046Q1|N|N|N|2468347|2468346|2467988|2468261|N|N|N|N|N| +2468354|AAAAAAAACAKKFCAA|2046-01-08|1752|7619|585|2046|1|1|8|1|2046|585|7619|Monday|2046Q1|N|N|N|2468347|2468346|2467989|2468262|N|N|N|N|N| +2468355|AAAAAAAADAKKFCAA|2046-01-09|1752|7620|585|2046|2|1|9|1|2046|585|7620|Tuesday|2046Q1|N|N|N|2468347|2468346|2467990|2468263|N|N|N|N|N| +2468356|AAAAAAAAEAKKFCAA|2046-01-10|1752|7620|585|2046|3|1|10|1|2046|585|7620|Wednesday|2046Q1|N|N|N|2468347|2468346|2467991|2468264|N|N|N|N|N| +2468357|AAAAAAAAFAKKFCAA|2046-01-11|1752|7620|585|2046|4|1|11|1|2046|585|7620|Thursday|2046Q1|N|N|N|2468347|2468346|2467992|2468265|N|N|N|N|N| +2468358|AAAAAAAAGAKKFCAA|2046-01-12|1752|7620|585|2046|5|1|12|1|2046|585|7620|Friday|2046Q1|N|Y|N|2468347|2468346|2467993|2468266|N|N|N|N|N| +2468359|AAAAAAAAHAKKFCAA|2046-01-13|1752|7620|585|2046|6|1|13|1|2046|585|7620|Saturday|2046Q1|N|Y|N|2468347|2468346|2467994|2468267|N|N|N|N|N| +2468360|AAAAAAAAIAKKFCAA|2046-01-14|1752|7620|585|2046|0|1|14|1|2046|585|7620|Sunday|2046Q1|N|N|N|2468347|2468346|2467995|2468268|N|N|N|N|N| +2468361|AAAAAAAAJAKKFCAA|2046-01-15|1752|7620|585|2046|1|1|15|1|2046|585|7620|Monday|2046Q1|N|N|N|2468347|2468346|2467996|2468269|N|N|N|N|N| +2468362|AAAAAAAAKAKKFCAA|2046-01-16|1752|7621|585|2046|2|1|16|1|2046|585|7621|Tuesday|2046Q1|N|N|N|2468347|2468346|2467997|2468270|N|N|N|N|N| +2468363|AAAAAAAALAKKFCAA|2046-01-17|1752|7621|585|2046|3|1|17|1|2046|585|7621|Wednesday|2046Q1|N|N|N|2468347|2468346|2467998|2468271|N|N|N|N|N| +2468364|AAAAAAAAMAKKFCAA|2046-01-18|1752|7621|585|2046|4|1|18|1|2046|585|7621|Thursday|2046Q1|N|N|N|2468347|2468346|2467999|2468272|N|N|N|N|N| +2468365|AAAAAAAANAKKFCAA|2046-01-19|1752|7621|585|2046|5|1|19|1|2046|585|7621|Friday|2046Q1|N|Y|N|2468347|2468346|2468000|2468273|N|N|N|N|N| +2468366|AAAAAAAAOAKKFCAA|2046-01-20|1752|7621|585|2046|6|1|20|1|2046|585|7621|Saturday|2046Q1|N|Y|N|2468347|2468346|2468001|2468274|N|N|N|N|N| +2468367|AAAAAAAAPAKKFCAA|2046-01-21|1752|7621|585|2046|0|1|21|1|2046|585|7621|Sunday|2046Q1|N|N|N|2468347|2468346|2468002|2468275|N|N|N|N|N| +2468368|AAAAAAAAABKKFCAA|2046-01-22|1752|7621|585|2046|1|1|22|1|2046|585|7621|Monday|2046Q1|N|N|N|2468347|2468346|2468003|2468276|N|N|N|N|N| +2468369|AAAAAAAABBKKFCAA|2046-01-23|1752|7622|585|2046|2|1|23|1|2046|585|7622|Tuesday|2046Q1|N|N|N|2468347|2468346|2468004|2468277|N|N|N|N|N| +2468370|AAAAAAAACBKKFCAA|2046-01-24|1752|7622|585|2046|3|1|24|1|2046|585|7622|Wednesday|2046Q1|N|N|N|2468347|2468346|2468005|2468278|N|N|N|N|N| +2468371|AAAAAAAADBKKFCAA|2046-01-25|1752|7622|585|2046|4|1|25|1|2046|585|7622|Thursday|2046Q1|N|N|N|2468347|2468346|2468006|2468279|N|N|N|N|N| +2468372|AAAAAAAAEBKKFCAA|2046-01-26|1752|7622|585|2046|5|1|26|1|2046|585|7622|Friday|2046Q1|N|Y|N|2468347|2468346|2468007|2468280|N|N|N|N|N| +2468373|AAAAAAAAFBKKFCAA|2046-01-27|1752|7622|585|2046|6|1|27|1|2046|585|7622|Saturday|2046Q1|N|Y|N|2468347|2468346|2468008|2468281|N|N|N|N|N| +2468374|AAAAAAAAGBKKFCAA|2046-01-28|1752|7622|585|2046|0|1|28|1|2046|585|7622|Sunday|2046Q1|N|N|N|2468347|2468346|2468009|2468282|N|N|N|N|N| +2468375|AAAAAAAAHBKKFCAA|2046-01-29|1752|7622|585|2046|1|1|29|1|2046|585|7622|Monday|2046Q1|N|N|N|2468347|2468346|2468010|2468283|N|N|N|N|N| +2468376|AAAAAAAAIBKKFCAA|2046-01-30|1752|7623|585|2046|2|1|30|1|2046|585|7623|Tuesday|2046Q1|N|N|N|2468347|2468346|2468011|2468284|N|N|N|N|N| +2468377|AAAAAAAAJBKKFCAA|2046-01-31|1752|7623|585|2046|3|1|31|1|2046|585|7623|Wednesday|2046Q1|N|N|N|2468347|2468346|2468012|2468285|N|N|N|N|N| +2468378|AAAAAAAAKBKKFCAA|2046-02-01|1753|7623|585|2046|4|2|1|1|2046|585|7623|Thursday|2046Q1|N|N|N|2468378|2468408|2468013|2468286|N|N|N|N|N| +2468379|AAAAAAAALBKKFCAA|2046-02-02|1753|7623|585|2046|5|2|2|1|2046|585|7623|Friday|2046Q1|N|Y|N|2468378|2468408|2468014|2468287|N|N|N|N|N| +2468380|AAAAAAAAMBKKFCAA|2046-02-03|1753|7623|585|2046|6|2|3|1|2046|585|7623|Saturday|2046Q1|N|Y|N|2468378|2468408|2468015|2468288|N|N|N|N|N| +2468381|AAAAAAAANBKKFCAA|2046-02-04|1753|7623|585|2046|0|2|4|1|2046|585|7623|Sunday|2046Q1|N|N|N|2468378|2468408|2468016|2468289|N|N|N|N|N| +2468382|AAAAAAAAOBKKFCAA|2046-02-05|1753|7623|585|2046|1|2|5|1|2046|585|7623|Monday|2046Q1|N|N|N|2468378|2468408|2468017|2468290|N|N|N|N|N| +2468383|AAAAAAAAPBKKFCAA|2046-02-06|1753|7624|585|2046|2|2|6|1|2046|585|7624|Tuesday|2046Q1|N|N|N|2468378|2468408|2468018|2468291|N|N|N|N|N| +2468384|AAAAAAAAACKKFCAA|2046-02-07|1753|7624|585|2046|3|2|7|1|2046|585|7624|Wednesday|2046Q1|N|N|N|2468378|2468408|2468019|2468292|N|N|N|N|N| +2468385|AAAAAAAABCKKFCAA|2046-02-08|1753|7624|585|2046|4|2|8|1|2046|585|7624|Thursday|2046Q1|N|N|N|2468378|2468408|2468020|2468293|N|N|N|N|N| +2468386|AAAAAAAACCKKFCAA|2046-02-09|1753|7624|585|2046|5|2|9|1|2046|585|7624|Friday|2046Q1|N|Y|N|2468378|2468408|2468021|2468294|N|N|N|N|N| +2468387|AAAAAAAADCKKFCAA|2046-02-10|1753|7624|585|2046|6|2|10|1|2046|585|7624|Saturday|2046Q1|N|Y|N|2468378|2468408|2468022|2468295|N|N|N|N|N| +2468388|AAAAAAAAECKKFCAA|2046-02-11|1753|7624|585|2046|0|2|11|1|2046|585|7624|Sunday|2046Q1|N|N|N|2468378|2468408|2468023|2468296|N|N|N|N|N| +2468389|AAAAAAAAFCKKFCAA|2046-02-12|1753|7624|585|2046|1|2|12|1|2046|585|7624|Monday|2046Q1|N|N|N|2468378|2468408|2468024|2468297|N|N|N|N|N| +2468390|AAAAAAAAGCKKFCAA|2046-02-13|1753|7625|585|2046|2|2|13|1|2046|585|7625|Tuesday|2046Q1|N|N|N|2468378|2468408|2468025|2468298|N|N|N|N|N| +2468391|AAAAAAAAHCKKFCAA|2046-02-14|1753|7625|585|2046|3|2|14|1|2046|585|7625|Wednesday|2046Q1|N|N|N|2468378|2468408|2468026|2468299|N|N|N|N|N| +2468392|AAAAAAAAICKKFCAA|2046-02-15|1753|7625|585|2046|4|2|15|1|2046|585|7625|Thursday|2046Q1|N|N|N|2468378|2468408|2468027|2468300|N|N|N|N|N| +2468393|AAAAAAAAJCKKFCAA|2046-02-16|1753|7625|585|2046|5|2|16|1|2046|585|7625|Friday|2046Q1|N|Y|N|2468378|2468408|2468028|2468301|N|N|N|N|N| +2468394|AAAAAAAAKCKKFCAA|2046-02-17|1753|7625|585|2046|6|2|17|1|2046|585|7625|Saturday|2046Q1|N|Y|N|2468378|2468408|2468029|2468302|N|N|N|N|N| +2468395|AAAAAAAALCKKFCAA|2046-02-18|1753|7625|585|2046|0|2|18|1|2046|585|7625|Sunday|2046Q1|N|N|N|2468378|2468408|2468030|2468303|N|N|N|N|N| +2468396|AAAAAAAAMCKKFCAA|2046-02-19|1753|7625|585|2046|1|2|19|1|2046|585|7625|Monday|2046Q1|N|N|N|2468378|2468408|2468031|2468304|N|N|N|N|N| +2468397|AAAAAAAANCKKFCAA|2046-02-20|1753|7626|585|2046|2|2|20|1|2046|585|7626|Tuesday|2046Q1|N|N|N|2468378|2468408|2468032|2468305|N|N|N|N|N| +2468398|AAAAAAAAOCKKFCAA|2046-02-21|1753|7626|585|2046|3|2|21|1|2046|585|7626|Wednesday|2046Q1|N|N|N|2468378|2468408|2468033|2468306|N|N|N|N|N| +2468399|AAAAAAAAPCKKFCAA|2046-02-22|1753|7626|585|2046|4|2|22|1|2046|585|7626|Thursday|2046Q1|N|N|N|2468378|2468408|2468034|2468307|N|N|N|N|N| +2468400|AAAAAAAAADKKFCAA|2046-02-23|1753|7626|585|2046|5|2|23|1|2046|585|7626|Friday|2046Q1|N|Y|N|2468378|2468408|2468035|2468308|N|N|N|N|N| +2468401|AAAAAAAABDKKFCAA|2046-02-24|1753|7626|585|2046|6|2|24|1|2046|585|7626|Saturday|2046Q1|N|Y|N|2468378|2468408|2468036|2468309|N|N|N|N|N| +2468402|AAAAAAAACDKKFCAA|2046-02-25|1753|7626|585|2046|0|2|25|1|2046|585|7626|Sunday|2046Q1|N|N|N|2468378|2468408|2468037|2468310|N|N|N|N|N| +2468403|AAAAAAAADDKKFCAA|2046-02-26|1753|7626|585|2046|1|2|26|1|2046|585|7626|Monday|2046Q1|N|N|N|2468378|2468408|2468038|2468311|N|N|N|N|N| +2468404|AAAAAAAAEDKKFCAA|2046-02-27|1753|7627|585|2046|2|2|27|1|2046|585|7627|Tuesday|2046Q1|N|N|N|2468378|2468408|2468039|2468312|N|N|N|N|N| +2468405|AAAAAAAAFDKKFCAA|2046-02-28|1753|7627|585|2046|3|2|28|1|2046|585|7627|Wednesday|2046Q1|N|N|N|2468378|2468408|2468040|2468313|N|N|N|N|N| +2468406|AAAAAAAAGDKKFCAA|2046-03-01|1754|7627|586|2046|4|3|1|1|2046|586|7627|Thursday|2046Q1|N|N|N|2468406|2468464|2468041|2468314|N|N|N|N|N| +2468407|AAAAAAAAHDKKFCAA|2046-03-02|1754|7627|586|2046|5|3|2|1|2046|586|7627|Friday|2046Q1|N|Y|N|2468406|2468464|2468042|2468315|N|N|N|N|N| +2468408|AAAAAAAAIDKKFCAA|2046-03-03|1754|7627|586|2046|6|3|3|1|2046|586|7627|Saturday|2046Q1|N|Y|N|2468406|2468464|2468043|2468316|N|N|N|N|N| +2468409|AAAAAAAAJDKKFCAA|2046-03-04|1754|7627|586|2046|0|3|4|1|2046|586|7627|Sunday|2046Q1|N|N|N|2468406|2468464|2468044|2468317|N|N|N|N|N| +2468410|AAAAAAAAKDKKFCAA|2046-03-05|1754|7627|586|2046|1|3|5|1|2046|586|7627|Monday|2046Q1|N|N|N|2468406|2468464|2468045|2468318|N|N|N|N|N| +2468411|AAAAAAAALDKKFCAA|2046-03-06|1754|7628|586|2046|2|3|6|1|2046|586|7628|Tuesday|2046Q1|N|N|N|2468406|2468464|2468046|2468319|N|N|N|N|N| +2468412|AAAAAAAAMDKKFCAA|2046-03-07|1754|7628|586|2046|3|3|7|1|2046|586|7628|Wednesday|2046Q1|N|N|N|2468406|2468464|2468047|2468320|N|N|N|N|N| +2468413|AAAAAAAANDKKFCAA|2046-03-08|1754|7628|586|2046|4|3|8|1|2046|586|7628|Thursday|2046Q1|N|N|N|2468406|2468464|2468048|2468321|N|N|N|N|N| +2468414|AAAAAAAAODKKFCAA|2046-03-09|1754|7628|586|2046|5|3|9|1|2046|586|7628|Friday|2046Q1|N|Y|N|2468406|2468464|2468049|2468322|N|N|N|N|N| +2468415|AAAAAAAAPDKKFCAA|2046-03-10|1754|7628|586|2046|6|3|10|1|2046|586|7628|Saturday|2046Q1|N|Y|N|2468406|2468464|2468050|2468323|N|N|N|N|N| +2468416|AAAAAAAAAEKKFCAA|2046-03-11|1754|7628|586|2046|0|3|11|1|2046|586|7628|Sunday|2046Q1|N|N|N|2468406|2468464|2468051|2468324|N|N|N|N|N| +2468417|AAAAAAAABEKKFCAA|2046-03-12|1754|7628|586|2046|1|3|12|1|2046|586|7628|Monday|2046Q1|N|N|N|2468406|2468464|2468052|2468325|N|N|N|N|N| +2468418|AAAAAAAACEKKFCAA|2046-03-13|1754|7629|586|2046|2|3|13|1|2046|586|7629|Tuesday|2046Q1|N|N|N|2468406|2468464|2468053|2468326|N|N|N|N|N| +2468419|AAAAAAAADEKKFCAA|2046-03-14|1754|7629|586|2046|3|3|14|1|2046|586|7629|Wednesday|2046Q1|N|N|N|2468406|2468464|2468054|2468327|N|N|N|N|N| +2468420|AAAAAAAAEEKKFCAA|2046-03-15|1754|7629|586|2046|4|3|15|1|2046|586|7629|Thursday|2046Q1|N|N|N|2468406|2468464|2468055|2468328|N|N|N|N|N| +2468421|AAAAAAAAFEKKFCAA|2046-03-16|1754|7629|586|2046|5|3|16|1|2046|586|7629|Friday|2046Q1|N|Y|N|2468406|2468464|2468056|2468329|N|N|N|N|N| +2468422|AAAAAAAAGEKKFCAA|2046-03-17|1754|7629|586|2046|6|3|17|1|2046|586|7629|Saturday|2046Q1|N|Y|N|2468406|2468464|2468057|2468330|N|N|N|N|N| +2468423|AAAAAAAAHEKKFCAA|2046-03-18|1754|7629|586|2046|0|3|18|1|2046|586|7629|Sunday|2046Q1|N|N|N|2468406|2468464|2468058|2468331|N|N|N|N|N| +2468424|AAAAAAAAIEKKFCAA|2046-03-19|1754|7629|586|2046|1|3|19|1|2046|586|7629|Monday|2046Q1|N|N|N|2468406|2468464|2468059|2468332|N|N|N|N|N| +2468425|AAAAAAAAJEKKFCAA|2046-03-20|1754|7630|586|2046|2|3|20|1|2046|586|7630|Tuesday|2046Q1|N|N|N|2468406|2468464|2468060|2468333|N|N|N|N|N| +2468426|AAAAAAAAKEKKFCAA|2046-03-21|1754|7630|586|2046|3|3|21|1|2046|586|7630|Wednesday|2046Q1|N|N|N|2468406|2468464|2468061|2468334|N|N|N|N|N| +2468427|AAAAAAAALEKKFCAA|2046-03-22|1754|7630|586|2046|4|3|22|1|2046|586|7630|Thursday|2046Q1|N|N|N|2468406|2468464|2468062|2468335|N|N|N|N|N| +2468428|AAAAAAAAMEKKFCAA|2046-03-23|1754|7630|586|2046|5|3|23|1|2046|586|7630|Friday|2046Q1|N|Y|N|2468406|2468464|2468063|2468336|N|N|N|N|N| +2468429|AAAAAAAANEKKFCAA|2046-03-24|1754|7630|586|2046|6|3|24|1|2046|586|7630|Saturday|2046Q1|N|Y|N|2468406|2468464|2468064|2468337|N|N|N|N|N| +2468430|AAAAAAAAOEKKFCAA|2046-03-25|1754|7630|586|2046|0|3|25|1|2046|586|7630|Sunday|2046Q1|N|N|N|2468406|2468464|2468065|2468338|N|N|N|N|N| +2468431|AAAAAAAAPEKKFCAA|2046-03-26|1754|7630|586|2046|1|3|26|1|2046|586|7630|Monday|2046Q1|N|N|N|2468406|2468464|2468066|2468339|N|N|N|N|N| +2468432|AAAAAAAAAFKKFCAA|2046-03-27|1754|7631|586|2046|2|3|27|1|2046|586|7631|Tuesday|2046Q1|N|N|N|2468406|2468464|2468067|2468340|N|N|N|N|N| +2468433|AAAAAAAABFKKFCAA|2046-03-28|1754|7631|586|2046|3|3|28|1|2046|586|7631|Wednesday|2046Q1|N|N|N|2468406|2468464|2468068|2468341|N|N|N|N|N| +2468434|AAAAAAAACFKKFCAA|2046-03-29|1754|7631|586|2046|4|3|29|1|2046|586|7631|Thursday|2046Q1|N|N|N|2468406|2468464|2468069|2468342|N|N|N|N|N| +2468435|AAAAAAAADFKKFCAA|2046-03-30|1754|7631|586|2046|5|3|30|1|2046|586|7631|Friday|2046Q1|N|Y|N|2468406|2468464|2468070|2468343|N|N|N|N|N| +2468436|AAAAAAAAEFKKFCAA|2046-03-31|1754|7631|586|2046|6|3|31|1|2046|586|7631|Saturday|2046Q1|N|Y|N|2468406|2468464|2468071|2468344|N|N|N|N|N| +2468437|AAAAAAAAFFKKFCAA|2046-04-01|1755|7631|586|2046|0|4|1|1|2046|586|7631|Sunday|2046Q1|N|N|N|2468437|2468526|2468072|2468347|N|N|N|N|N| +2468438|AAAAAAAAGFKKFCAA|2046-04-02|1755|7631|586|2046|1|4|2|2|2046|586|7631|Monday|2046Q2|N|N|N|2468437|2468526|2468073|2468348|N|N|N|N|N| +2468439|AAAAAAAAHFKKFCAA|2046-04-03|1755|7632|586|2046|2|4|3|2|2046|586|7632|Tuesday|2046Q2|N|N|N|2468437|2468526|2468074|2468349|N|N|N|N|N| +2468440|AAAAAAAAIFKKFCAA|2046-04-04|1755|7632|586|2046|3|4|4|2|2046|586|7632|Wednesday|2046Q2|N|N|N|2468437|2468526|2468075|2468350|N|N|N|N|N| +2468441|AAAAAAAAJFKKFCAA|2046-04-05|1755|7632|586|2046|4|4|5|2|2046|586|7632|Thursday|2046Q2|N|N|N|2468437|2468526|2468076|2468351|N|N|N|N|N| +2468442|AAAAAAAAKFKKFCAA|2046-04-06|1755|7632|586|2046|5|4|6|2|2046|586|7632|Friday|2046Q2|N|Y|N|2468437|2468526|2468077|2468352|N|N|N|N|N| +2468443|AAAAAAAALFKKFCAA|2046-04-07|1755|7632|586|2046|6|4|7|2|2046|586|7632|Saturday|2046Q2|N|Y|N|2468437|2468526|2468078|2468353|N|N|N|N|N| +2468444|AAAAAAAAMFKKFCAA|2046-04-08|1755|7632|586|2046|0|4|8|2|2046|586|7632|Sunday|2046Q2|N|N|N|2468437|2468526|2468079|2468354|N|N|N|N|N| +2468445|AAAAAAAANFKKFCAA|2046-04-09|1755|7632|586|2046|1|4|9|2|2046|586|7632|Monday|2046Q2|N|N|N|2468437|2468526|2468080|2468355|N|N|N|N|N| +2468446|AAAAAAAAOFKKFCAA|2046-04-10|1755|7633|586|2046|2|4|10|2|2046|586|7633|Tuesday|2046Q2|N|N|N|2468437|2468526|2468081|2468356|N|N|N|N|N| +2468447|AAAAAAAAPFKKFCAA|2046-04-11|1755|7633|586|2046|3|4|11|2|2046|586|7633|Wednesday|2046Q2|N|N|N|2468437|2468526|2468082|2468357|N|N|N|N|N| +2468448|AAAAAAAAAGKKFCAA|2046-04-12|1755|7633|586|2046|4|4|12|2|2046|586|7633|Thursday|2046Q2|N|N|N|2468437|2468526|2468083|2468358|N|N|N|N|N| +2468449|AAAAAAAABGKKFCAA|2046-04-13|1755|7633|586|2046|5|4|13|2|2046|586|7633|Friday|2046Q2|N|Y|N|2468437|2468526|2468084|2468359|N|N|N|N|N| +2468450|AAAAAAAACGKKFCAA|2046-04-14|1755|7633|586|2046|6|4|14|2|2046|586|7633|Saturday|2046Q2|N|Y|N|2468437|2468526|2468085|2468360|N|N|N|N|N| +2468451|AAAAAAAADGKKFCAA|2046-04-15|1755|7633|586|2046|0|4|15|2|2046|586|7633|Sunday|2046Q2|N|N|N|2468437|2468526|2468086|2468361|N|N|N|N|N| +2468452|AAAAAAAAEGKKFCAA|2046-04-16|1755|7633|586|2046|1|4|16|2|2046|586|7633|Monday|2046Q2|N|N|N|2468437|2468526|2468087|2468362|N|N|N|N|N| +2468453|AAAAAAAAFGKKFCAA|2046-04-17|1755|7634|586|2046|2|4|17|2|2046|586|7634|Tuesday|2046Q2|N|N|N|2468437|2468526|2468088|2468363|N|N|N|N|N| +2468454|AAAAAAAAGGKKFCAA|2046-04-18|1755|7634|586|2046|3|4|18|2|2046|586|7634|Wednesday|2046Q2|N|N|N|2468437|2468526|2468089|2468364|N|N|N|N|N| +2468455|AAAAAAAAHGKKFCAA|2046-04-19|1755|7634|586|2046|4|4|19|2|2046|586|7634|Thursday|2046Q2|N|N|N|2468437|2468526|2468090|2468365|N|N|N|N|N| +2468456|AAAAAAAAIGKKFCAA|2046-04-20|1755|7634|586|2046|5|4|20|2|2046|586|7634|Friday|2046Q2|N|Y|N|2468437|2468526|2468091|2468366|N|N|N|N|N| +2468457|AAAAAAAAJGKKFCAA|2046-04-21|1755|7634|586|2046|6|4|21|2|2046|586|7634|Saturday|2046Q2|N|Y|N|2468437|2468526|2468092|2468367|N|N|N|N|N| +2468458|AAAAAAAAKGKKFCAA|2046-04-22|1755|7634|586|2046|0|4|22|2|2046|586|7634|Sunday|2046Q2|N|N|N|2468437|2468526|2468093|2468368|N|N|N|N|N| +2468459|AAAAAAAALGKKFCAA|2046-04-23|1755|7634|586|2046|1|4|23|2|2046|586|7634|Monday|2046Q2|N|N|N|2468437|2468526|2468094|2468369|N|N|N|N|N| +2468460|AAAAAAAAMGKKFCAA|2046-04-24|1755|7635|586|2046|2|4|24|2|2046|586|7635|Tuesday|2046Q2|N|N|N|2468437|2468526|2468095|2468370|N|N|N|N|N| +2468461|AAAAAAAANGKKFCAA|2046-04-25|1755|7635|586|2046|3|4|25|2|2046|586|7635|Wednesday|2046Q2|N|N|N|2468437|2468526|2468096|2468371|N|N|N|N|N| +2468462|AAAAAAAAOGKKFCAA|2046-04-26|1755|7635|586|2046|4|4|26|2|2046|586|7635|Thursday|2046Q2|N|N|N|2468437|2468526|2468097|2468372|N|N|N|N|N| +2468463|AAAAAAAAPGKKFCAA|2046-04-27|1755|7635|586|2046|5|4|27|2|2046|586|7635|Friday|2046Q2|N|Y|N|2468437|2468526|2468098|2468373|N|N|N|N|N| +2468464|AAAAAAAAAHKKFCAA|2046-04-28|1755|7635|586|2046|6|4|28|2|2046|586|7635|Saturday|2046Q2|N|Y|N|2468437|2468526|2468099|2468374|N|N|N|N|N| +2468465|AAAAAAAABHKKFCAA|2046-04-29|1755|7635|586|2046|0|4|29|2|2046|586|7635|Sunday|2046Q2|N|N|N|2468437|2468526|2468100|2468375|N|N|N|N|N| +2468466|AAAAAAAACHKKFCAA|2046-04-30|1755|7635|586|2046|1|4|30|2|2046|586|7635|Monday|2046Q2|N|N|N|2468437|2468526|2468101|2468376|N|N|N|N|N| +2468467|AAAAAAAADHKKFCAA|2046-05-01|1756|7636|586|2046|2|5|1|2|2046|586|7636|Tuesday|2046Q2|N|N|N|2468467|2468586|2468102|2468377|N|N|N|N|N| +2468468|AAAAAAAAEHKKFCAA|2046-05-02|1756|7636|586|2046|3|5|2|2|2046|586|7636|Wednesday|2046Q2|N|N|N|2468467|2468586|2468103|2468378|N|N|N|N|N| +2468469|AAAAAAAAFHKKFCAA|2046-05-03|1756|7636|586|2046|4|5|3|2|2046|586|7636|Thursday|2046Q2|N|N|N|2468467|2468586|2468104|2468379|N|N|N|N|N| +2468470|AAAAAAAAGHKKFCAA|2046-05-04|1756|7636|586|2046|5|5|4|2|2046|586|7636|Friday|2046Q2|N|Y|N|2468467|2468586|2468105|2468380|N|N|N|N|N| +2468471|AAAAAAAAHHKKFCAA|2046-05-05|1756|7636|586|2046|6|5|5|2|2046|586|7636|Saturday|2046Q2|N|Y|N|2468467|2468586|2468106|2468381|N|N|N|N|N| +2468472|AAAAAAAAIHKKFCAA|2046-05-06|1756|7636|586|2046|0|5|6|2|2046|586|7636|Sunday|2046Q2|N|N|N|2468467|2468586|2468107|2468382|N|N|N|N|N| +2468473|AAAAAAAAJHKKFCAA|2046-05-07|1756|7636|586|2046|1|5|7|2|2046|586|7636|Monday|2046Q2|N|N|N|2468467|2468586|2468108|2468383|N|N|N|N|N| +2468474|AAAAAAAAKHKKFCAA|2046-05-08|1756|7637|586|2046|2|5|8|2|2046|586|7637|Tuesday|2046Q2|N|N|N|2468467|2468586|2468109|2468384|N|N|N|N|N| +2468475|AAAAAAAALHKKFCAA|2046-05-09|1756|7637|586|2046|3|5|9|2|2046|586|7637|Wednesday|2046Q2|N|N|N|2468467|2468586|2468110|2468385|N|N|N|N|N| +2468476|AAAAAAAAMHKKFCAA|2046-05-10|1756|7637|586|2046|4|5|10|2|2046|586|7637|Thursday|2046Q2|N|N|N|2468467|2468586|2468111|2468386|N|N|N|N|N| +2468477|AAAAAAAANHKKFCAA|2046-05-11|1756|7637|586|2046|5|5|11|2|2046|586|7637|Friday|2046Q2|N|Y|N|2468467|2468586|2468112|2468387|N|N|N|N|N| +2468478|AAAAAAAAOHKKFCAA|2046-05-12|1756|7637|586|2046|6|5|12|2|2046|586|7637|Saturday|2046Q2|N|Y|N|2468467|2468586|2468113|2468388|N|N|N|N|N| +2468479|AAAAAAAAPHKKFCAA|2046-05-13|1756|7637|586|2046|0|5|13|2|2046|586|7637|Sunday|2046Q2|N|N|N|2468467|2468586|2468114|2468389|N|N|N|N|N| +2468480|AAAAAAAAAIKKFCAA|2046-05-14|1756|7637|586|2046|1|5|14|2|2046|586|7637|Monday|2046Q2|N|N|N|2468467|2468586|2468115|2468390|N|N|N|N|N| +2468481|AAAAAAAABIKKFCAA|2046-05-15|1756|7638|586|2046|2|5|15|2|2046|586|7638|Tuesday|2046Q2|N|N|N|2468467|2468586|2468116|2468391|N|N|N|N|N| +2468482|AAAAAAAACIKKFCAA|2046-05-16|1756|7638|586|2046|3|5|16|2|2046|586|7638|Wednesday|2046Q2|N|N|N|2468467|2468586|2468117|2468392|N|N|N|N|N| +2468483|AAAAAAAADIKKFCAA|2046-05-17|1756|7638|586|2046|4|5|17|2|2046|586|7638|Thursday|2046Q2|N|N|N|2468467|2468586|2468118|2468393|N|N|N|N|N| +2468484|AAAAAAAAEIKKFCAA|2046-05-18|1756|7638|586|2046|5|5|18|2|2046|586|7638|Friday|2046Q2|N|Y|N|2468467|2468586|2468119|2468394|N|N|N|N|N| +2468485|AAAAAAAAFIKKFCAA|2046-05-19|1756|7638|586|2046|6|5|19|2|2046|586|7638|Saturday|2046Q2|N|Y|N|2468467|2468586|2468120|2468395|N|N|N|N|N| +2468486|AAAAAAAAGIKKFCAA|2046-05-20|1756|7638|586|2046|0|5|20|2|2046|586|7638|Sunday|2046Q2|N|N|N|2468467|2468586|2468121|2468396|N|N|N|N|N| +2468487|AAAAAAAAHIKKFCAA|2046-05-21|1756|7638|586|2046|1|5|21|2|2046|586|7638|Monday|2046Q2|N|N|N|2468467|2468586|2468122|2468397|N|N|N|N|N| +2468488|AAAAAAAAIIKKFCAA|2046-05-22|1756|7639|586|2046|2|5|22|2|2046|586|7639|Tuesday|2046Q2|N|N|N|2468467|2468586|2468123|2468398|N|N|N|N|N| +2468489|AAAAAAAAJIKKFCAA|2046-05-23|1756|7639|586|2046|3|5|23|2|2046|586|7639|Wednesday|2046Q2|N|N|N|2468467|2468586|2468124|2468399|N|N|N|N|N| +2468490|AAAAAAAAKIKKFCAA|2046-05-24|1756|7639|586|2046|4|5|24|2|2046|586|7639|Thursday|2046Q2|N|N|N|2468467|2468586|2468125|2468400|N|N|N|N|N| +2468491|AAAAAAAALIKKFCAA|2046-05-25|1756|7639|586|2046|5|5|25|2|2046|586|7639|Friday|2046Q2|N|Y|N|2468467|2468586|2468126|2468401|N|N|N|N|N| +2468492|AAAAAAAAMIKKFCAA|2046-05-26|1756|7639|586|2046|6|5|26|2|2046|586|7639|Saturday|2046Q2|N|Y|N|2468467|2468586|2468127|2468402|N|N|N|N|N| +2468493|AAAAAAAANIKKFCAA|2046-05-27|1756|7639|586|2046|0|5|27|2|2046|586|7639|Sunday|2046Q2|N|N|N|2468467|2468586|2468128|2468403|N|N|N|N|N| +2468494|AAAAAAAAOIKKFCAA|2046-05-28|1756|7639|586|2046|1|5|28|2|2046|586|7639|Monday|2046Q2|N|N|N|2468467|2468586|2468129|2468404|N|N|N|N|N| +2468495|AAAAAAAAPIKKFCAA|2046-05-29|1756|7640|586|2046|2|5|29|2|2046|586|7640|Tuesday|2046Q2|N|N|N|2468467|2468586|2468130|2468405|N|N|N|N|N| +2468496|AAAAAAAAAJKKFCAA|2046-05-30|1756|7640|586|2046|3|5|30|2|2046|586|7640|Wednesday|2046Q2|N|N|N|2468467|2468586|2468131|2468406|N|N|N|N|N| +2468497|AAAAAAAABJKKFCAA|2046-05-31|1756|7640|586|2046|4|5|31|2|2046|586|7640|Thursday|2046Q2|N|N|N|2468467|2468586|2468132|2468407|N|N|N|N|N| +2468498|AAAAAAAACJKKFCAA|2046-06-01|1757|7640|587|2046|5|6|1|2|2046|587|7640|Friday|2046Q2|N|Y|N|2468498|2468648|2468133|2468408|N|N|N|N|N| +2468499|AAAAAAAADJKKFCAA|2046-06-02|1757|7640|587|2046|6|6|2|2|2046|587|7640|Saturday|2046Q2|N|Y|N|2468498|2468648|2468134|2468409|N|N|N|N|N| +2468500|AAAAAAAAEJKKFCAA|2046-06-03|1757|7640|587|2046|0|6|3|2|2046|587|7640|Sunday|2046Q2|N|N|N|2468498|2468648|2468135|2468410|N|N|N|N|N| +2468501|AAAAAAAAFJKKFCAA|2046-06-04|1757|7640|587|2046|1|6|4|2|2046|587|7640|Monday|2046Q2|N|N|N|2468498|2468648|2468136|2468411|N|N|N|N|N| +2468502|AAAAAAAAGJKKFCAA|2046-06-05|1757|7641|587|2046|2|6|5|2|2046|587|7641|Tuesday|2046Q2|N|N|N|2468498|2468648|2468137|2468412|N|N|N|N|N| +2468503|AAAAAAAAHJKKFCAA|2046-06-06|1757|7641|587|2046|3|6|6|2|2046|587|7641|Wednesday|2046Q2|N|N|N|2468498|2468648|2468138|2468413|N|N|N|N|N| +2468504|AAAAAAAAIJKKFCAA|2046-06-07|1757|7641|587|2046|4|6|7|2|2046|587|7641|Thursday|2046Q2|N|N|N|2468498|2468648|2468139|2468414|N|N|N|N|N| +2468505|AAAAAAAAJJKKFCAA|2046-06-08|1757|7641|587|2046|5|6|8|2|2046|587|7641|Friday|2046Q2|N|Y|N|2468498|2468648|2468140|2468415|N|N|N|N|N| +2468506|AAAAAAAAKJKKFCAA|2046-06-09|1757|7641|587|2046|6|6|9|2|2046|587|7641|Saturday|2046Q2|N|Y|N|2468498|2468648|2468141|2468416|N|N|N|N|N| +2468507|AAAAAAAALJKKFCAA|2046-06-10|1757|7641|587|2046|0|6|10|2|2046|587|7641|Sunday|2046Q2|N|N|N|2468498|2468648|2468142|2468417|N|N|N|N|N| +2468508|AAAAAAAAMJKKFCAA|2046-06-11|1757|7641|587|2046|1|6|11|2|2046|587|7641|Monday|2046Q2|N|N|N|2468498|2468648|2468143|2468418|N|N|N|N|N| +2468509|AAAAAAAANJKKFCAA|2046-06-12|1757|7642|587|2046|2|6|12|2|2046|587|7642|Tuesday|2046Q2|N|N|N|2468498|2468648|2468144|2468419|N|N|N|N|N| +2468510|AAAAAAAAOJKKFCAA|2046-06-13|1757|7642|587|2046|3|6|13|2|2046|587|7642|Wednesday|2046Q2|N|N|N|2468498|2468648|2468145|2468420|N|N|N|N|N| +2468511|AAAAAAAAPJKKFCAA|2046-06-14|1757|7642|587|2046|4|6|14|2|2046|587|7642|Thursday|2046Q2|N|N|N|2468498|2468648|2468146|2468421|N|N|N|N|N| +2468512|AAAAAAAAAKKKFCAA|2046-06-15|1757|7642|587|2046|5|6|15|2|2046|587|7642|Friday|2046Q2|N|Y|N|2468498|2468648|2468147|2468422|N|N|N|N|N| +2468513|AAAAAAAABKKKFCAA|2046-06-16|1757|7642|587|2046|6|6|16|2|2046|587|7642|Saturday|2046Q2|N|Y|N|2468498|2468648|2468148|2468423|N|N|N|N|N| +2468514|AAAAAAAACKKKFCAA|2046-06-17|1757|7642|587|2046|0|6|17|2|2046|587|7642|Sunday|2046Q2|N|N|N|2468498|2468648|2468149|2468424|N|N|N|N|N| +2468515|AAAAAAAADKKKFCAA|2046-06-18|1757|7642|587|2046|1|6|18|2|2046|587|7642|Monday|2046Q2|N|N|N|2468498|2468648|2468150|2468425|N|N|N|N|N| +2468516|AAAAAAAAEKKKFCAA|2046-06-19|1757|7643|587|2046|2|6|19|2|2046|587|7643|Tuesday|2046Q2|N|N|N|2468498|2468648|2468151|2468426|N|N|N|N|N| +2468517|AAAAAAAAFKKKFCAA|2046-06-20|1757|7643|587|2046|3|6|20|2|2046|587|7643|Wednesday|2046Q2|N|N|N|2468498|2468648|2468152|2468427|N|N|N|N|N| +2468518|AAAAAAAAGKKKFCAA|2046-06-21|1757|7643|587|2046|4|6|21|2|2046|587|7643|Thursday|2046Q2|N|N|N|2468498|2468648|2468153|2468428|N|N|N|N|N| +2468519|AAAAAAAAHKKKFCAA|2046-06-22|1757|7643|587|2046|5|6|22|2|2046|587|7643|Friday|2046Q2|N|Y|N|2468498|2468648|2468154|2468429|N|N|N|N|N| +2468520|AAAAAAAAIKKKFCAA|2046-06-23|1757|7643|587|2046|6|6|23|2|2046|587|7643|Saturday|2046Q2|N|Y|N|2468498|2468648|2468155|2468430|N|N|N|N|N| +2468521|AAAAAAAAJKKKFCAA|2046-06-24|1757|7643|587|2046|0|6|24|2|2046|587|7643|Sunday|2046Q2|N|N|N|2468498|2468648|2468156|2468431|N|N|N|N|N| +2468522|AAAAAAAAKKKKFCAA|2046-06-25|1757|7643|587|2046|1|6|25|2|2046|587|7643|Monday|2046Q2|N|N|N|2468498|2468648|2468157|2468432|N|N|N|N|N| +2468523|AAAAAAAALKKKFCAA|2046-06-26|1757|7644|587|2046|2|6|26|2|2046|587|7644|Tuesday|2046Q2|N|N|N|2468498|2468648|2468158|2468433|N|N|N|N|N| +2468524|AAAAAAAAMKKKFCAA|2046-06-27|1757|7644|587|2046|3|6|27|2|2046|587|7644|Wednesday|2046Q2|N|N|N|2468498|2468648|2468159|2468434|N|N|N|N|N| +2468525|AAAAAAAANKKKFCAA|2046-06-28|1757|7644|587|2046|4|6|28|2|2046|587|7644|Thursday|2046Q2|N|N|N|2468498|2468648|2468160|2468435|N|N|N|N|N| +2468526|AAAAAAAAOKKKFCAA|2046-06-29|1757|7644|587|2046|5|6|29|2|2046|587|7644|Friday|2046Q2|N|Y|N|2468498|2468648|2468161|2468436|N|N|N|N|N| +2468527|AAAAAAAAPKKKFCAA|2046-06-30|1757|7644|587|2046|6|6|30|2|2046|587|7644|Saturday|2046Q2|N|Y|N|2468498|2468648|2468162|2468437|N|N|N|N|N| +2468528|AAAAAAAAALKKFCAA|2046-07-01|1758|7644|587|2046|0|7|1|2|2046|587|7644|Sunday|2046Q2|N|N|N|2468528|2468708|2468163|2468437|N|N|N|N|N| +2468529|AAAAAAAABLKKFCAA|2046-07-02|1758|7644|587|2046|1|7|2|3|2046|587|7644|Monday|2046Q3|N|N|N|2468528|2468708|2468164|2468438|N|N|N|N|N| +2468530|AAAAAAAACLKKFCAA|2046-07-03|1758|7645|587|2046|2|7|3|3|2046|587|7645|Tuesday|2046Q3|N|N|N|2468528|2468708|2468165|2468439|N|N|N|N|N| +2468531|AAAAAAAADLKKFCAA|2046-07-04|1758|7645|587|2046|3|7|4|3|2046|587|7645|Wednesday|2046Q3|N|N|N|2468528|2468708|2468166|2468440|N|N|N|N|N| +2468532|AAAAAAAAELKKFCAA|2046-07-05|1758|7645|587|2046|4|7|5|3|2046|587|7645|Thursday|2046Q3|Y|N|N|2468528|2468708|2468167|2468441|N|N|N|N|N| +2468533|AAAAAAAAFLKKFCAA|2046-07-06|1758|7645|587|2046|5|7|6|3|2046|587|7645|Friday|2046Q3|N|Y|Y|2468528|2468708|2468168|2468442|N|N|N|N|N| +2468534|AAAAAAAAGLKKFCAA|2046-07-07|1758|7645|587|2046|6|7|7|3|2046|587|7645|Saturday|2046Q3|N|Y|N|2468528|2468708|2468169|2468443|N|N|N|N|N| +2468535|AAAAAAAAHLKKFCAA|2046-07-08|1758|7645|587|2046|0|7|8|3|2046|587|7645|Sunday|2046Q3|N|N|N|2468528|2468708|2468170|2468444|N|N|N|N|N| +2468536|AAAAAAAAILKKFCAA|2046-07-09|1758|7645|587|2046|1|7|9|3|2046|587|7645|Monday|2046Q3|N|N|N|2468528|2468708|2468171|2468445|N|N|N|N|N| +2468537|AAAAAAAAJLKKFCAA|2046-07-10|1758|7646|587|2046|2|7|10|3|2046|587|7646|Tuesday|2046Q3|N|N|N|2468528|2468708|2468172|2468446|N|N|N|N|N| +2468538|AAAAAAAAKLKKFCAA|2046-07-11|1758|7646|587|2046|3|7|11|3|2046|587|7646|Wednesday|2046Q3|N|N|N|2468528|2468708|2468173|2468447|N|N|N|N|N| +2468539|AAAAAAAALLKKFCAA|2046-07-12|1758|7646|587|2046|4|7|12|3|2046|587|7646|Thursday|2046Q3|N|N|N|2468528|2468708|2468174|2468448|N|N|N|N|N| +2468540|AAAAAAAAMLKKFCAA|2046-07-13|1758|7646|587|2046|5|7|13|3|2046|587|7646|Friday|2046Q3|N|Y|N|2468528|2468708|2468175|2468449|N|N|N|N|N| +2468541|AAAAAAAANLKKFCAA|2046-07-14|1758|7646|587|2046|6|7|14|3|2046|587|7646|Saturday|2046Q3|N|Y|N|2468528|2468708|2468176|2468450|N|N|N|N|N| +2468542|AAAAAAAAOLKKFCAA|2046-07-15|1758|7646|587|2046|0|7|15|3|2046|587|7646|Sunday|2046Q3|N|N|N|2468528|2468708|2468177|2468451|N|N|N|N|N| +2468543|AAAAAAAAPLKKFCAA|2046-07-16|1758|7646|587|2046|1|7|16|3|2046|587|7646|Monday|2046Q3|N|N|N|2468528|2468708|2468178|2468452|N|N|N|N|N| +2468544|AAAAAAAAAMKKFCAA|2046-07-17|1758|7647|587|2046|2|7|17|3|2046|587|7647|Tuesday|2046Q3|N|N|N|2468528|2468708|2468179|2468453|N|N|N|N|N| +2468545|AAAAAAAABMKKFCAA|2046-07-18|1758|7647|587|2046|3|7|18|3|2046|587|7647|Wednesday|2046Q3|N|N|N|2468528|2468708|2468180|2468454|N|N|N|N|N| +2468546|AAAAAAAACMKKFCAA|2046-07-19|1758|7647|587|2046|4|7|19|3|2046|587|7647|Thursday|2046Q3|N|N|N|2468528|2468708|2468181|2468455|N|N|N|N|N| +2468547|AAAAAAAADMKKFCAA|2046-07-20|1758|7647|587|2046|5|7|20|3|2046|587|7647|Friday|2046Q3|N|Y|N|2468528|2468708|2468182|2468456|N|N|N|N|N| +2468548|AAAAAAAAEMKKFCAA|2046-07-21|1758|7647|587|2046|6|7|21|3|2046|587|7647|Saturday|2046Q3|N|Y|N|2468528|2468708|2468183|2468457|N|N|N|N|N| +2468549|AAAAAAAAFMKKFCAA|2046-07-22|1758|7647|587|2046|0|7|22|3|2046|587|7647|Sunday|2046Q3|N|N|N|2468528|2468708|2468184|2468458|N|N|N|N|N| +2468550|AAAAAAAAGMKKFCAA|2046-07-23|1758|7647|587|2046|1|7|23|3|2046|587|7647|Monday|2046Q3|N|N|N|2468528|2468708|2468185|2468459|N|N|N|N|N| +2468551|AAAAAAAAHMKKFCAA|2046-07-24|1758|7648|587|2046|2|7|24|3|2046|587|7648|Tuesday|2046Q3|N|N|N|2468528|2468708|2468186|2468460|N|N|N|N|N| +2468552|AAAAAAAAIMKKFCAA|2046-07-25|1758|7648|587|2046|3|7|25|3|2046|587|7648|Wednesday|2046Q3|N|N|N|2468528|2468708|2468187|2468461|N|N|N|N|N| +2468553|AAAAAAAAJMKKFCAA|2046-07-26|1758|7648|587|2046|4|7|26|3|2046|587|7648|Thursday|2046Q3|N|N|N|2468528|2468708|2468188|2468462|N|N|N|N|N| +2468554|AAAAAAAAKMKKFCAA|2046-07-27|1758|7648|587|2046|5|7|27|3|2046|587|7648|Friday|2046Q3|N|Y|N|2468528|2468708|2468189|2468463|N|N|N|N|N| +2468555|AAAAAAAALMKKFCAA|2046-07-28|1758|7648|587|2046|6|7|28|3|2046|587|7648|Saturday|2046Q3|N|Y|N|2468528|2468708|2468190|2468464|N|N|N|N|N| +2468556|AAAAAAAAMMKKFCAA|2046-07-29|1758|7648|587|2046|0|7|29|3|2046|587|7648|Sunday|2046Q3|N|N|N|2468528|2468708|2468191|2468465|N|N|N|N|N| +2468557|AAAAAAAANMKKFCAA|2046-07-30|1758|7648|587|2046|1|7|30|3|2046|587|7648|Monday|2046Q3|N|N|N|2468528|2468708|2468192|2468466|N|N|N|N|N| +2468558|AAAAAAAAOMKKFCAA|2046-07-31|1758|7649|587|2046|2|7|31|3|2046|587|7649|Tuesday|2046Q3|N|N|N|2468528|2468708|2468193|2468467|N|N|N|N|N| +2468559|AAAAAAAAPMKKFCAA|2046-08-01|1759|7649|587|2046|3|8|1|3|2046|587|7649|Wednesday|2046Q3|N|N|N|2468559|2468770|2468194|2468468|N|N|N|N|N| +2468560|AAAAAAAAANKKFCAA|2046-08-02|1759|7649|587|2046|4|8|2|3|2046|587|7649|Thursday|2046Q3|N|N|N|2468559|2468770|2468195|2468469|N|N|N|N|N| +2468561|AAAAAAAABNKKFCAA|2046-08-03|1759|7649|587|2046|5|8|3|3|2046|587|7649|Friday|2046Q3|N|Y|N|2468559|2468770|2468196|2468470|N|N|N|N|N| +2468562|AAAAAAAACNKKFCAA|2046-08-04|1759|7649|587|2046|6|8|4|3|2046|587|7649|Saturday|2046Q3|N|Y|N|2468559|2468770|2468197|2468471|N|N|N|N|N| +2468563|AAAAAAAADNKKFCAA|2046-08-05|1759|7649|587|2046|0|8|5|3|2046|587|7649|Sunday|2046Q3|N|N|N|2468559|2468770|2468198|2468472|N|N|N|N|N| +2468564|AAAAAAAAENKKFCAA|2046-08-06|1759|7649|587|2046|1|8|6|3|2046|587|7649|Monday|2046Q3|N|N|N|2468559|2468770|2468199|2468473|N|N|N|N|N| +2468565|AAAAAAAAFNKKFCAA|2046-08-07|1759|7650|587|2046|2|8|7|3|2046|587|7650|Tuesday|2046Q3|N|N|N|2468559|2468770|2468200|2468474|N|N|N|N|N| +2468566|AAAAAAAAGNKKFCAA|2046-08-08|1759|7650|587|2046|3|8|8|3|2046|587|7650|Wednesday|2046Q3|N|N|N|2468559|2468770|2468201|2468475|N|N|N|N|N| +2468567|AAAAAAAAHNKKFCAA|2046-08-09|1759|7650|587|2046|4|8|9|3|2046|587|7650|Thursday|2046Q3|N|N|N|2468559|2468770|2468202|2468476|N|N|N|N|N| +2468568|AAAAAAAAINKKFCAA|2046-08-10|1759|7650|587|2046|5|8|10|3|2046|587|7650|Friday|2046Q3|N|Y|N|2468559|2468770|2468203|2468477|N|N|N|N|N| +2468569|AAAAAAAAJNKKFCAA|2046-08-11|1759|7650|587|2046|6|8|11|3|2046|587|7650|Saturday|2046Q3|N|Y|N|2468559|2468770|2468204|2468478|N|N|N|N|N| +2468570|AAAAAAAAKNKKFCAA|2046-08-12|1759|7650|587|2046|0|8|12|3|2046|587|7650|Sunday|2046Q3|N|N|N|2468559|2468770|2468205|2468479|N|N|N|N|N| +2468571|AAAAAAAALNKKFCAA|2046-08-13|1759|7650|587|2046|1|8|13|3|2046|587|7650|Monday|2046Q3|N|N|N|2468559|2468770|2468206|2468480|N|N|N|N|N| +2468572|AAAAAAAAMNKKFCAA|2046-08-14|1759|7651|587|2046|2|8|14|3|2046|587|7651|Tuesday|2046Q3|N|N|N|2468559|2468770|2468207|2468481|N|N|N|N|N| +2468573|AAAAAAAANNKKFCAA|2046-08-15|1759|7651|587|2046|3|8|15|3|2046|587|7651|Wednesday|2046Q3|N|N|N|2468559|2468770|2468208|2468482|N|N|N|N|N| +2468574|AAAAAAAAONKKFCAA|2046-08-16|1759|7651|587|2046|4|8|16|3|2046|587|7651|Thursday|2046Q3|N|N|N|2468559|2468770|2468209|2468483|N|N|N|N|N| +2468575|AAAAAAAAPNKKFCAA|2046-08-17|1759|7651|587|2046|5|8|17|3|2046|587|7651|Friday|2046Q3|N|Y|N|2468559|2468770|2468210|2468484|N|N|N|N|N| +2468576|AAAAAAAAAOKKFCAA|2046-08-18|1759|7651|587|2046|6|8|18|3|2046|587|7651|Saturday|2046Q3|N|Y|N|2468559|2468770|2468211|2468485|N|N|N|N|N| +2468577|AAAAAAAABOKKFCAA|2046-08-19|1759|7651|587|2046|0|8|19|3|2046|587|7651|Sunday|2046Q3|N|N|N|2468559|2468770|2468212|2468486|N|N|N|N|N| +2468578|AAAAAAAACOKKFCAA|2046-08-20|1759|7651|587|2046|1|8|20|3|2046|587|7651|Monday|2046Q3|N|N|N|2468559|2468770|2468213|2468487|N|N|N|N|N| +2468579|AAAAAAAADOKKFCAA|2046-08-21|1759|7652|587|2046|2|8|21|3|2046|587|7652|Tuesday|2046Q3|N|N|N|2468559|2468770|2468214|2468488|N|N|N|N|N| +2468580|AAAAAAAAEOKKFCAA|2046-08-22|1759|7652|587|2046|3|8|22|3|2046|587|7652|Wednesday|2046Q3|N|N|N|2468559|2468770|2468215|2468489|N|N|N|N|N| +2468581|AAAAAAAAFOKKFCAA|2046-08-23|1759|7652|587|2046|4|8|23|3|2046|587|7652|Thursday|2046Q3|N|N|N|2468559|2468770|2468216|2468490|N|N|N|N|N| +2468582|AAAAAAAAGOKKFCAA|2046-08-24|1759|7652|587|2046|5|8|24|3|2046|587|7652|Friday|2046Q3|N|Y|N|2468559|2468770|2468217|2468491|N|N|N|N|N| +2468583|AAAAAAAAHOKKFCAA|2046-08-25|1759|7652|587|2046|6|8|25|3|2046|587|7652|Saturday|2046Q3|N|Y|N|2468559|2468770|2468218|2468492|N|N|N|N|N| +2468584|AAAAAAAAIOKKFCAA|2046-08-26|1759|7652|587|2046|0|8|26|3|2046|587|7652|Sunday|2046Q3|N|N|N|2468559|2468770|2468219|2468493|N|N|N|N|N| +2468585|AAAAAAAAJOKKFCAA|2046-08-27|1759|7652|587|2046|1|8|27|3|2046|587|7652|Monday|2046Q3|N|N|N|2468559|2468770|2468220|2468494|N|N|N|N|N| +2468586|AAAAAAAAKOKKFCAA|2046-08-28|1759|7653|587|2046|2|8|28|3|2046|587|7653|Tuesday|2046Q3|N|N|N|2468559|2468770|2468221|2468495|N|N|N|N|N| +2468587|AAAAAAAALOKKFCAA|2046-08-29|1759|7653|587|2046|3|8|29|3|2046|587|7653|Wednesday|2046Q3|N|N|N|2468559|2468770|2468222|2468496|N|N|N|N|N| +2468588|AAAAAAAAMOKKFCAA|2046-08-30|1759|7653|587|2046|4|8|30|3|2046|587|7653|Thursday|2046Q3|N|N|N|2468559|2468770|2468223|2468497|N|N|N|N|N| +2468589|AAAAAAAANOKKFCAA|2046-08-31|1759|7653|587|2046|5|8|31|3|2046|587|7653|Friday|2046Q3|N|Y|N|2468559|2468770|2468224|2468498|N|N|N|N|N| +2468590|AAAAAAAAOOKKFCAA|2046-09-01|1760|7653|588|2046|6|9|1|3|2046|588|7653|Saturday|2046Q3|N|Y|N|2468590|2468832|2468225|2468499|N|N|N|N|N| +2468591|AAAAAAAAPOKKFCAA|2046-09-02|1760|7653|588|2046|0|9|2|3|2046|588|7653|Sunday|2046Q3|N|N|N|2468590|2468832|2468226|2468500|N|N|N|N|N| +2468592|AAAAAAAAAPKKFCAA|2046-09-03|1760|7653|588|2046|1|9|3|3|2046|588|7653|Monday|2046Q3|N|N|N|2468590|2468832|2468227|2468501|N|N|N|N|N| +2468593|AAAAAAAABPKKFCAA|2046-09-04|1760|7654|588|2046|2|9|4|3|2046|588|7654|Tuesday|2046Q3|N|N|N|2468590|2468832|2468228|2468502|N|N|N|N|N| +2468594|AAAAAAAACPKKFCAA|2046-09-05|1760|7654|588|2046|3|9|5|3|2046|588|7654|Wednesday|2046Q3|N|N|N|2468590|2468832|2468229|2468503|N|N|N|N|N| +2468595|AAAAAAAADPKKFCAA|2046-09-06|1760|7654|588|2046|4|9|6|3|2046|588|7654|Thursday|2046Q3|N|N|N|2468590|2468832|2468230|2468504|N|N|N|N|N| +2468596|AAAAAAAAEPKKFCAA|2046-09-07|1760|7654|588|2046|5|9|7|3|2046|588|7654|Friday|2046Q3|N|Y|N|2468590|2468832|2468231|2468505|N|N|N|N|N| +2468597|AAAAAAAAFPKKFCAA|2046-09-08|1760|7654|588|2046|6|9|8|3|2046|588|7654|Saturday|2046Q3|N|Y|N|2468590|2468832|2468232|2468506|N|N|N|N|N| +2468598|AAAAAAAAGPKKFCAA|2046-09-09|1760|7654|588|2046|0|9|9|3|2046|588|7654|Sunday|2046Q3|N|N|N|2468590|2468832|2468233|2468507|N|N|N|N|N| +2468599|AAAAAAAAHPKKFCAA|2046-09-10|1760|7654|588|2046|1|9|10|3|2046|588|7654|Monday|2046Q3|N|N|N|2468590|2468832|2468234|2468508|N|N|N|N|N| +2468600|AAAAAAAAIPKKFCAA|2046-09-11|1760|7655|588|2046|2|9|11|3|2046|588|7655|Tuesday|2046Q3|N|N|N|2468590|2468832|2468235|2468509|N|N|N|N|N| +2468601|AAAAAAAAJPKKFCAA|2046-09-12|1760|7655|588|2046|3|9|12|3|2046|588|7655|Wednesday|2046Q3|N|N|N|2468590|2468832|2468236|2468510|N|N|N|N|N| +2468602|AAAAAAAAKPKKFCAA|2046-09-13|1760|7655|588|2046|4|9|13|3|2046|588|7655|Thursday|2046Q3|N|N|N|2468590|2468832|2468237|2468511|N|N|N|N|N| +2468603|AAAAAAAALPKKFCAA|2046-09-14|1760|7655|588|2046|5|9|14|3|2046|588|7655|Friday|2046Q3|N|Y|N|2468590|2468832|2468238|2468512|N|N|N|N|N| +2468604|AAAAAAAAMPKKFCAA|2046-09-15|1760|7655|588|2046|6|9|15|3|2046|588|7655|Saturday|2046Q3|N|Y|N|2468590|2468832|2468239|2468513|N|N|N|N|N| +2468605|AAAAAAAANPKKFCAA|2046-09-16|1760|7655|588|2046|0|9|16|3|2046|588|7655|Sunday|2046Q3|N|N|N|2468590|2468832|2468240|2468514|N|N|N|N|N| +2468606|AAAAAAAAOPKKFCAA|2046-09-17|1760|7655|588|2046|1|9|17|3|2046|588|7655|Monday|2046Q3|N|N|N|2468590|2468832|2468241|2468515|N|N|N|N|N| +2468607|AAAAAAAAPPKKFCAA|2046-09-18|1760|7656|588|2046|2|9|18|3|2046|588|7656|Tuesday|2046Q3|N|N|N|2468590|2468832|2468242|2468516|N|N|N|N|N| +2468608|AAAAAAAAAALKFCAA|2046-09-19|1760|7656|588|2046|3|9|19|3|2046|588|7656|Wednesday|2046Q3|N|N|N|2468590|2468832|2468243|2468517|N|N|N|N|N| +2468609|AAAAAAAABALKFCAA|2046-09-20|1760|7656|588|2046|4|9|20|3|2046|588|7656|Thursday|2046Q3|N|N|N|2468590|2468832|2468244|2468518|N|N|N|N|N| +2468610|AAAAAAAACALKFCAA|2046-09-21|1760|7656|588|2046|5|9|21|3|2046|588|7656|Friday|2046Q3|N|Y|N|2468590|2468832|2468245|2468519|N|N|N|N|N| +2468611|AAAAAAAADALKFCAA|2046-09-22|1760|7656|588|2046|6|9|22|3|2046|588|7656|Saturday|2046Q3|N|Y|N|2468590|2468832|2468246|2468520|N|N|N|N|N| +2468612|AAAAAAAAEALKFCAA|2046-09-23|1760|7656|588|2046|0|9|23|3|2046|588|7656|Sunday|2046Q3|N|N|N|2468590|2468832|2468247|2468521|N|N|N|N|N| +2468613|AAAAAAAAFALKFCAA|2046-09-24|1760|7656|588|2046|1|9|24|3|2046|588|7656|Monday|2046Q3|N|N|N|2468590|2468832|2468248|2468522|N|N|N|N|N| +2468614|AAAAAAAAGALKFCAA|2046-09-25|1760|7657|588|2046|2|9|25|3|2046|588|7657|Tuesday|2046Q3|N|N|N|2468590|2468832|2468249|2468523|N|N|N|N|N| +2468615|AAAAAAAAHALKFCAA|2046-09-26|1760|7657|588|2046|3|9|26|3|2046|588|7657|Wednesday|2046Q3|N|N|N|2468590|2468832|2468250|2468524|N|N|N|N|N| +2468616|AAAAAAAAIALKFCAA|2046-09-27|1760|7657|588|2046|4|9|27|3|2046|588|7657|Thursday|2046Q3|N|N|N|2468590|2468832|2468251|2468525|N|N|N|N|N| +2468617|AAAAAAAAJALKFCAA|2046-09-28|1760|7657|588|2046|5|9|28|3|2046|588|7657|Friday|2046Q3|N|Y|N|2468590|2468832|2468252|2468526|N|N|N|N|N| +2468618|AAAAAAAAKALKFCAA|2046-09-29|1760|7657|588|2046|6|9|29|3|2046|588|7657|Saturday|2046Q3|N|Y|N|2468590|2468832|2468253|2468527|N|N|N|N|N| +2468619|AAAAAAAALALKFCAA|2046-09-30|1760|7657|588|2046|0|9|30|3|2046|588|7657|Sunday|2046Q3|N|N|N|2468590|2468832|2468254|2468528|N|N|N|N|N| +2468620|AAAAAAAAMALKFCAA|2046-10-01|1761|7657|588|2046|1|10|1|3|2046|588|7657|Monday|2046Q3|N|N|N|2468620|2468892|2468255|2468528|N|N|N|N|N| +2468621|AAAAAAAANALKFCAA|2046-10-02|1761|7658|588|2046|2|10|2|4|2046|588|7658|Tuesday|2046Q4|N|N|N|2468620|2468892|2468256|2468529|N|N|N|N|N| +2468622|AAAAAAAAOALKFCAA|2046-10-03|1761|7658|588|2046|3|10|3|4|2046|588|7658|Wednesday|2046Q4|N|N|N|2468620|2468892|2468257|2468530|N|N|N|N|N| +2468623|AAAAAAAAPALKFCAA|2046-10-04|1761|7658|588|2046|4|10|4|4|2046|588|7658|Thursday|2046Q4|N|N|N|2468620|2468892|2468258|2468531|N|N|N|N|N| +2468624|AAAAAAAAABLKFCAA|2046-10-05|1761|7658|588|2046|5|10|5|4|2046|588|7658|Friday|2046Q4|N|Y|N|2468620|2468892|2468259|2468532|N|N|N|N|N| +2468625|AAAAAAAABBLKFCAA|2046-10-06|1761|7658|588|2046|6|10|6|4|2046|588|7658|Saturday|2046Q4|N|Y|N|2468620|2468892|2468260|2468533|N|N|N|N|N| +2468626|AAAAAAAACBLKFCAA|2046-10-07|1761|7658|588|2046|0|10|7|4|2046|588|7658|Sunday|2046Q4|N|N|N|2468620|2468892|2468261|2468534|N|N|N|N|N| +2468627|AAAAAAAADBLKFCAA|2046-10-08|1761|7658|588|2046|1|10|8|4|2046|588|7658|Monday|2046Q4|N|N|N|2468620|2468892|2468262|2468535|N|N|N|N|N| +2468628|AAAAAAAAEBLKFCAA|2046-10-09|1761|7659|588|2046|2|10|9|4|2046|588|7659|Tuesday|2046Q4|N|N|N|2468620|2468892|2468263|2468536|N|N|N|N|N| +2468629|AAAAAAAAFBLKFCAA|2046-10-10|1761|7659|588|2046|3|10|10|4|2046|588|7659|Wednesday|2046Q4|N|N|N|2468620|2468892|2468264|2468537|N|N|N|N|N| +2468630|AAAAAAAAGBLKFCAA|2046-10-11|1761|7659|588|2046|4|10|11|4|2046|588|7659|Thursday|2046Q4|N|N|N|2468620|2468892|2468265|2468538|N|N|N|N|N| +2468631|AAAAAAAAHBLKFCAA|2046-10-12|1761|7659|588|2046|5|10|12|4|2046|588|7659|Friday|2046Q4|N|Y|N|2468620|2468892|2468266|2468539|N|N|N|N|N| +2468632|AAAAAAAAIBLKFCAA|2046-10-13|1761|7659|588|2046|6|10|13|4|2046|588|7659|Saturday|2046Q4|N|Y|N|2468620|2468892|2468267|2468540|N|N|N|N|N| +2468633|AAAAAAAAJBLKFCAA|2046-10-14|1761|7659|588|2046|0|10|14|4|2046|588|7659|Sunday|2046Q4|N|N|N|2468620|2468892|2468268|2468541|N|N|N|N|N| +2468634|AAAAAAAAKBLKFCAA|2046-10-15|1761|7659|588|2046|1|10|15|4|2046|588|7659|Monday|2046Q4|N|N|N|2468620|2468892|2468269|2468542|N|N|N|N|N| +2468635|AAAAAAAALBLKFCAA|2046-10-16|1761|7660|588|2046|2|10|16|4|2046|588|7660|Tuesday|2046Q4|N|N|N|2468620|2468892|2468270|2468543|N|N|N|N|N| +2468636|AAAAAAAAMBLKFCAA|2046-10-17|1761|7660|588|2046|3|10|17|4|2046|588|7660|Wednesday|2046Q4|N|N|N|2468620|2468892|2468271|2468544|N|N|N|N|N| +2468637|AAAAAAAANBLKFCAA|2046-10-18|1761|7660|588|2046|4|10|18|4|2046|588|7660|Thursday|2046Q4|N|N|N|2468620|2468892|2468272|2468545|N|N|N|N|N| +2468638|AAAAAAAAOBLKFCAA|2046-10-19|1761|7660|588|2046|5|10|19|4|2046|588|7660|Friday|2046Q4|N|Y|N|2468620|2468892|2468273|2468546|N|N|N|N|N| +2468639|AAAAAAAAPBLKFCAA|2046-10-20|1761|7660|588|2046|6|10|20|4|2046|588|7660|Saturday|2046Q4|N|Y|N|2468620|2468892|2468274|2468547|N|N|N|N|N| +2468640|AAAAAAAAACLKFCAA|2046-10-21|1761|7660|588|2046|0|10|21|4|2046|588|7660|Sunday|2046Q4|N|N|N|2468620|2468892|2468275|2468548|N|N|N|N|N| +2468641|AAAAAAAABCLKFCAA|2046-10-22|1761|7660|588|2046|1|10|22|4|2046|588|7660|Monday|2046Q4|N|N|N|2468620|2468892|2468276|2468549|N|N|N|N|N| +2468642|AAAAAAAACCLKFCAA|2046-10-23|1761|7661|588|2046|2|10|23|4|2046|588|7661|Tuesday|2046Q4|N|N|N|2468620|2468892|2468277|2468550|N|N|N|N|N| +2468643|AAAAAAAADCLKFCAA|2046-10-24|1761|7661|588|2046|3|10|24|4|2046|588|7661|Wednesday|2046Q4|N|N|N|2468620|2468892|2468278|2468551|N|N|N|N|N| +2468644|AAAAAAAAECLKFCAA|2046-10-25|1761|7661|588|2046|4|10|25|4|2046|588|7661|Thursday|2046Q4|N|N|N|2468620|2468892|2468279|2468552|N|N|N|N|N| +2468645|AAAAAAAAFCLKFCAA|2046-10-26|1761|7661|588|2046|5|10|26|4|2046|588|7661|Friday|2046Q4|N|Y|N|2468620|2468892|2468280|2468553|N|N|N|N|N| +2468646|AAAAAAAAGCLKFCAA|2046-10-27|1761|7661|588|2046|6|10|27|4|2046|588|7661|Saturday|2046Q4|N|Y|N|2468620|2468892|2468281|2468554|N|N|N|N|N| +2468647|AAAAAAAAHCLKFCAA|2046-10-28|1761|7661|588|2046|0|10|28|4|2046|588|7661|Sunday|2046Q4|N|N|N|2468620|2468892|2468282|2468555|N|N|N|N|N| +2468648|AAAAAAAAICLKFCAA|2046-10-29|1761|7661|588|2046|1|10|29|4|2046|588|7661|Monday|2046Q4|N|N|N|2468620|2468892|2468283|2468556|N|N|N|N|N| +2468649|AAAAAAAAJCLKFCAA|2046-10-30|1761|7662|588|2046|2|10|30|4|2046|588|7662|Tuesday|2046Q4|N|N|N|2468620|2468892|2468284|2468557|N|N|N|N|N| +2468650|AAAAAAAAKCLKFCAA|2046-10-31|1761|7662|588|2046|3|10|31|4|2046|588|7662|Wednesday|2046Q4|N|N|N|2468620|2468892|2468285|2468558|N|N|N|N|N| +2468651|AAAAAAAALCLKFCAA|2046-11-01|1762|7662|588|2046|4|11|1|4|2046|588|7662|Thursday|2046Q4|N|N|N|2468651|2468954|2468286|2468559|N|N|N|N|N| +2468652|AAAAAAAAMCLKFCAA|2046-11-02|1762|7662|588|2046|5|11|2|4|2046|588|7662|Friday|2046Q4|N|Y|N|2468651|2468954|2468287|2468560|N|N|N|N|N| +2468653|AAAAAAAANCLKFCAA|2046-11-03|1762|7662|588|2046|6|11|3|4|2046|588|7662|Saturday|2046Q4|N|Y|N|2468651|2468954|2468288|2468561|N|N|N|N|N| +2468654|AAAAAAAAOCLKFCAA|2046-11-04|1762|7662|588|2046|0|11|4|4|2046|588|7662|Sunday|2046Q4|N|N|N|2468651|2468954|2468289|2468562|N|N|N|N|N| +2468655|AAAAAAAAPCLKFCAA|2046-11-05|1762|7662|588|2046|1|11|5|4|2046|588|7662|Monday|2046Q4|N|N|N|2468651|2468954|2468290|2468563|N|N|N|N|N| +2468656|AAAAAAAAADLKFCAA|2046-11-06|1762|7663|588|2046|2|11|6|4|2046|588|7663|Tuesday|2046Q4|N|N|N|2468651|2468954|2468291|2468564|N|N|N|N|N| +2468657|AAAAAAAABDLKFCAA|2046-11-07|1762|7663|588|2046|3|11|7|4|2046|588|7663|Wednesday|2046Q4|N|N|N|2468651|2468954|2468292|2468565|N|N|N|N|N| +2468658|AAAAAAAACDLKFCAA|2046-11-08|1762|7663|588|2046|4|11|8|4|2046|588|7663|Thursday|2046Q4|N|N|N|2468651|2468954|2468293|2468566|N|N|N|N|N| +2468659|AAAAAAAADDLKFCAA|2046-11-09|1762|7663|588|2046|5|11|9|4|2046|588|7663|Friday|2046Q4|N|Y|N|2468651|2468954|2468294|2468567|N|N|N|N|N| +2468660|AAAAAAAAEDLKFCAA|2046-11-10|1762|7663|588|2046|6|11|10|4|2046|588|7663|Saturday|2046Q4|N|Y|N|2468651|2468954|2468295|2468568|N|N|N|N|N| +2468661|AAAAAAAAFDLKFCAA|2046-11-11|1762|7663|588|2046|0|11|11|4|2046|588|7663|Sunday|2046Q4|N|N|N|2468651|2468954|2468296|2468569|N|N|N|N|N| +2468662|AAAAAAAAGDLKFCAA|2046-11-12|1762|7663|588|2046|1|11|12|4|2046|588|7663|Monday|2046Q4|N|N|N|2468651|2468954|2468297|2468570|N|N|N|N|N| +2468663|AAAAAAAAHDLKFCAA|2046-11-13|1762|7664|588|2046|2|11|13|4|2046|588|7664|Tuesday|2046Q4|N|N|N|2468651|2468954|2468298|2468571|N|N|N|N|N| +2468664|AAAAAAAAIDLKFCAA|2046-11-14|1762|7664|588|2046|3|11|14|4|2046|588|7664|Wednesday|2046Q4|N|N|N|2468651|2468954|2468299|2468572|N|N|N|N|N| +2468665|AAAAAAAAJDLKFCAA|2046-11-15|1762|7664|588|2046|4|11|15|4|2046|588|7664|Thursday|2046Q4|N|N|N|2468651|2468954|2468300|2468573|N|N|N|N|N| +2468666|AAAAAAAAKDLKFCAA|2046-11-16|1762|7664|588|2046|5|11|16|4|2046|588|7664|Friday|2046Q4|N|Y|N|2468651|2468954|2468301|2468574|N|N|N|N|N| +2468667|AAAAAAAALDLKFCAA|2046-11-17|1762|7664|588|2046|6|11|17|4|2046|588|7664|Saturday|2046Q4|N|Y|N|2468651|2468954|2468302|2468575|N|N|N|N|N| +2468668|AAAAAAAAMDLKFCAA|2046-11-18|1762|7664|588|2046|0|11|18|4|2046|588|7664|Sunday|2046Q4|N|N|N|2468651|2468954|2468303|2468576|N|N|N|N|N| +2468669|AAAAAAAANDLKFCAA|2046-11-19|1762|7664|588|2046|1|11|19|4|2046|588|7664|Monday|2046Q4|N|N|N|2468651|2468954|2468304|2468577|N|N|N|N|N| +2468670|AAAAAAAAODLKFCAA|2046-11-20|1762|7665|588|2046|2|11|20|4|2046|588|7665|Tuesday|2046Q4|N|N|N|2468651|2468954|2468305|2468578|N|N|N|N|N| +2468671|AAAAAAAAPDLKFCAA|2046-11-21|1762|7665|588|2046|3|11|21|4|2046|588|7665|Wednesday|2046Q4|N|N|N|2468651|2468954|2468306|2468579|N|N|N|N|N| +2468672|AAAAAAAAAELKFCAA|2046-11-22|1762|7665|588|2046|4|11|22|4|2046|588|7665|Thursday|2046Q4|N|N|N|2468651|2468954|2468307|2468580|N|N|N|N|N| +2468673|AAAAAAAABELKFCAA|2046-11-23|1762|7665|588|2046|5|11|23|4|2046|588|7665|Friday|2046Q4|N|Y|N|2468651|2468954|2468308|2468581|N|N|N|N|N| +2468674|AAAAAAAACELKFCAA|2046-11-24|1762|7665|588|2046|6|11|24|4|2046|588|7665|Saturday|2046Q4|N|Y|N|2468651|2468954|2468309|2468582|N|N|N|N|N| +2468675|AAAAAAAADELKFCAA|2046-11-25|1762|7665|588|2046|0|11|25|4|2046|588|7665|Sunday|2046Q4|N|N|N|2468651|2468954|2468310|2468583|N|N|N|N|N| +2468676|AAAAAAAAEELKFCAA|2046-11-26|1762|7665|588|2046|1|11|26|4|2046|588|7665|Monday|2046Q4|N|N|N|2468651|2468954|2468311|2468584|N|N|N|N|N| +2468677|AAAAAAAAFELKFCAA|2046-11-27|1762|7666|588|2046|2|11|27|4|2046|588|7666|Tuesday|2046Q4|N|N|N|2468651|2468954|2468312|2468585|N|N|N|N|N| +2468678|AAAAAAAAGELKFCAA|2046-11-28|1762|7666|588|2046|3|11|28|4|2046|588|7666|Wednesday|2046Q4|N|N|N|2468651|2468954|2468313|2468586|N|N|N|N|N| +2468679|AAAAAAAAHELKFCAA|2046-11-29|1762|7666|588|2046|4|11|29|4|2046|588|7666|Thursday|2046Q4|N|N|N|2468651|2468954|2468314|2468587|N|N|N|N|N| +2468680|AAAAAAAAIELKFCAA|2046-11-30|1762|7666|588|2046|5|11|30|4|2046|588|7666|Friday|2046Q4|N|Y|N|2468651|2468954|2468315|2468588|N|N|N|N|N| +2468681|AAAAAAAAJELKFCAA|2046-12-01|1763|7666|589|2046|6|12|1|4|2046|589|7666|Saturday|2046Q4|N|Y|N|2468681|2469014|2468316|2468589|N|N|N|N|N| +2468682|AAAAAAAAKELKFCAA|2046-12-02|1763|7666|589|2046|0|12|2|4|2046|589|7666|Sunday|2046Q4|N|N|N|2468681|2469014|2468317|2468590|N|N|N|N|N| +2468683|AAAAAAAALELKFCAA|2046-12-03|1763|7666|589|2046|1|12|3|4|2046|589|7666|Monday|2046Q4|N|N|N|2468681|2469014|2468318|2468591|N|N|N|N|N| +2468684|AAAAAAAAMELKFCAA|2046-12-04|1763|7667|589|2046|2|12|4|4|2046|589|7667|Tuesday|2046Q4|N|N|N|2468681|2469014|2468319|2468592|N|N|N|N|N| +2468685|AAAAAAAANELKFCAA|2046-12-05|1763|7667|589|2046|3|12|5|4|2046|589|7667|Wednesday|2046Q4|N|N|N|2468681|2469014|2468320|2468593|N|N|N|N|N| +2468686|AAAAAAAAOELKFCAA|2046-12-06|1763|7667|589|2046|4|12|6|4|2046|589|7667|Thursday|2046Q4|N|N|N|2468681|2469014|2468321|2468594|N|N|N|N|N| +2468687|AAAAAAAAPELKFCAA|2046-12-07|1763|7667|589|2046|5|12|7|4|2046|589|7667|Friday|2046Q4|N|Y|N|2468681|2469014|2468322|2468595|N|N|N|N|N| +2468688|AAAAAAAAAFLKFCAA|2046-12-08|1763|7667|589|2046|6|12|8|4|2046|589|7667|Saturday|2046Q4|N|Y|N|2468681|2469014|2468323|2468596|N|N|N|N|N| +2468689|AAAAAAAABFLKFCAA|2046-12-09|1763|7667|589|2046|0|12|9|4|2046|589|7667|Sunday|2046Q4|N|N|N|2468681|2469014|2468324|2468597|N|N|N|N|N| +2468690|AAAAAAAACFLKFCAA|2046-12-10|1763|7667|589|2046|1|12|10|4|2046|589|7667|Monday|2046Q4|N|N|N|2468681|2469014|2468325|2468598|N|N|N|N|N| +2468691|AAAAAAAADFLKFCAA|2046-12-11|1763|7668|589|2046|2|12|11|4|2046|589|7668|Tuesday|2046Q4|N|N|N|2468681|2469014|2468326|2468599|N|N|N|N|N| +2468692|AAAAAAAAEFLKFCAA|2046-12-12|1763|7668|589|2046|3|12|12|4|2046|589|7668|Wednesday|2046Q4|N|N|N|2468681|2469014|2468327|2468600|N|N|N|N|N| +2468693|AAAAAAAAFFLKFCAA|2046-12-13|1763|7668|589|2046|4|12|13|4|2046|589|7668|Thursday|2046Q4|N|N|N|2468681|2469014|2468328|2468601|N|N|N|N|N| +2468694|AAAAAAAAGFLKFCAA|2046-12-14|1763|7668|589|2046|5|12|14|4|2046|589|7668|Friday|2046Q4|N|Y|N|2468681|2469014|2468329|2468602|N|N|N|N|N| +2468695|AAAAAAAAHFLKFCAA|2046-12-15|1763|7668|589|2046|6|12|15|4|2046|589|7668|Saturday|2046Q4|N|Y|N|2468681|2469014|2468330|2468603|N|N|N|N|N| +2468696|AAAAAAAAIFLKFCAA|2046-12-16|1763|7668|589|2046|0|12|16|4|2046|589|7668|Sunday|2046Q4|N|N|N|2468681|2469014|2468331|2468604|N|N|N|N|N| +2468697|AAAAAAAAJFLKFCAA|2046-12-17|1763|7668|589|2046|1|12|17|4|2046|589|7668|Monday|2046Q4|N|N|N|2468681|2469014|2468332|2468605|N|N|N|N|N| +2468698|AAAAAAAAKFLKFCAA|2046-12-18|1763|7669|589|2046|2|12|18|4|2046|589|7669|Tuesday|2046Q4|N|N|N|2468681|2469014|2468333|2468606|N|N|N|N|N| +2468699|AAAAAAAALFLKFCAA|2046-12-19|1763|7669|589|2046|3|12|19|4|2046|589|7669|Wednesday|2046Q4|N|N|N|2468681|2469014|2468334|2468607|N|N|N|N|N| +2468700|AAAAAAAAMFLKFCAA|2046-12-20|1763|7669|589|2046|4|12|20|4|2046|589|7669|Thursday|2046Q4|N|N|N|2468681|2469014|2468335|2468608|N|N|N|N|N| +2468701|AAAAAAAANFLKFCAA|2046-12-21|1763|7669|589|2046|5|12|21|4|2046|589|7669|Friday|2046Q4|N|Y|N|2468681|2469014|2468336|2468609|N|N|N|N|N| +2468702|AAAAAAAAOFLKFCAA|2046-12-22|1763|7669|589|2046|6|12|22|4|2046|589|7669|Saturday|2046Q4|N|Y|N|2468681|2469014|2468337|2468610|N|N|N|N|N| +2468703|AAAAAAAAPFLKFCAA|2046-12-23|1763|7669|589|2046|0|12|23|4|2046|589|7669|Sunday|2046Q4|N|N|N|2468681|2469014|2468338|2468611|N|N|N|N|N| +2468704|AAAAAAAAAGLKFCAA|2046-12-24|1763|7669|589|2046|1|12|24|4|2046|589|7669|Monday|2046Q4|N|N|N|2468681|2469014|2468339|2468612|N|N|N|N|N| +2468705|AAAAAAAABGLKFCAA|2046-12-25|1763|7670|589|2046|2|12|25|4|2046|589|7670|Tuesday|2046Q4|N|N|N|2468681|2469014|2468340|2468613|N|N|N|N|N| +2468706|AAAAAAAACGLKFCAA|2046-12-26|1763|7670|589|2046|3|12|26|4|2046|589|7670|Wednesday|2046Q4|Y|N|N|2468681|2469014|2468341|2468614|N|N|N|N|N| +2468707|AAAAAAAADGLKFCAA|2046-12-27|1763|7670|589|2046|4|12|27|4|2046|589|7670|Thursday|2046Q4|N|N|Y|2468681|2469014|2468342|2468615|N|N|N|N|N| +2468708|AAAAAAAAEGLKFCAA|2046-12-28|1763|7670|589|2046|5|12|28|4|2046|589|7670|Friday|2046Q4|N|Y|N|2468681|2469014|2468343|2468616|N|N|N|N|N| +2468709|AAAAAAAAFGLKFCAA|2046-12-29|1763|7670|589|2046|6|12|29|4|2046|589|7670|Saturday|2046Q4|N|Y|N|2468681|2469014|2468344|2468617|N|N|N|N|N| +2468710|AAAAAAAAGGLKFCAA|2046-12-30|1763|7670|589|2046|0|12|30|4|2046|589|7670|Sunday|2046Q4|N|N|N|2468681|2469014|2468345|2468618|N|N|N|N|N| +2468711|AAAAAAAAHGLKFCAA|2046-12-31|1763|7670|589|2046|1|12|31|4|2046|589|7670|Monday|2046Q4|N|N|N|2468681|2469014|2468346|2468619|N|N|N|N|N| +2468712|AAAAAAAAIGLKFCAA|2047-01-01|1764|7671|589|2047|2|1|1|1|2047|589|7671|Tuesday|2047Q1|Y|N|N|2468712|2468711|2468347|2468620|N|N|N|N|N| +2468713|AAAAAAAAJGLKFCAA|2047-01-02|1764|7671|589|2047|3|1|2|1|2047|589|7671|Wednesday|2047Q1|N|N|Y|2468712|2468711|2468348|2468621|N|N|N|N|N| +2468714|AAAAAAAAKGLKFCAA|2047-01-03|1764|7671|589|2047|4|1|3|1|2047|589|7671|Thursday|2047Q1|N|N|N|2468712|2468711|2468349|2468622|N|N|N|N|N| +2468715|AAAAAAAALGLKFCAA|2047-01-04|1764|7671|589|2047|5|1|4|1|2047|589|7671|Friday|2047Q1|N|Y|N|2468712|2468711|2468350|2468623|N|N|N|N|N| +2468716|AAAAAAAAMGLKFCAA|2047-01-05|1764|7671|589|2047|6|1|5|1|2047|589|7671|Saturday|2047Q1|N|Y|N|2468712|2468711|2468351|2468624|N|N|N|N|N| +2468717|AAAAAAAANGLKFCAA|2047-01-06|1764|7671|589|2047|0|1|6|1|2047|589|7671|Sunday|2047Q1|N|N|N|2468712|2468711|2468352|2468625|N|N|N|N|N| +2468718|AAAAAAAAOGLKFCAA|2047-01-07|1764|7671|589|2047|1|1|7|1|2047|589|7671|Monday|2047Q1|N|N|N|2468712|2468711|2468353|2468626|N|N|N|N|N| +2468719|AAAAAAAAPGLKFCAA|2047-01-08|1764|7672|589|2047|2|1|8|1|2047|589|7672|Tuesday|2047Q1|N|N|N|2468712|2468711|2468354|2468627|N|N|N|N|N| +2468720|AAAAAAAAAHLKFCAA|2047-01-09|1764|7672|589|2047|3|1|9|1|2047|589|7672|Wednesday|2047Q1|N|N|N|2468712|2468711|2468355|2468628|N|N|N|N|N| +2468721|AAAAAAAABHLKFCAA|2047-01-10|1764|7672|589|2047|4|1|10|1|2047|589|7672|Thursday|2047Q1|N|N|N|2468712|2468711|2468356|2468629|N|N|N|N|N| +2468722|AAAAAAAACHLKFCAA|2047-01-11|1764|7672|589|2047|5|1|11|1|2047|589|7672|Friday|2047Q1|N|Y|N|2468712|2468711|2468357|2468630|N|N|N|N|N| +2468723|AAAAAAAADHLKFCAA|2047-01-12|1764|7672|589|2047|6|1|12|1|2047|589|7672|Saturday|2047Q1|N|Y|N|2468712|2468711|2468358|2468631|N|N|N|N|N| +2468724|AAAAAAAAEHLKFCAA|2047-01-13|1764|7672|589|2047|0|1|13|1|2047|589|7672|Sunday|2047Q1|N|N|N|2468712|2468711|2468359|2468632|N|N|N|N|N| +2468725|AAAAAAAAFHLKFCAA|2047-01-14|1764|7672|589|2047|1|1|14|1|2047|589|7672|Monday|2047Q1|N|N|N|2468712|2468711|2468360|2468633|N|N|N|N|N| +2468726|AAAAAAAAGHLKFCAA|2047-01-15|1764|7673|589|2047|2|1|15|1|2047|589|7673|Tuesday|2047Q1|N|N|N|2468712|2468711|2468361|2468634|N|N|N|N|N| +2468727|AAAAAAAAHHLKFCAA|2047-01-16|1764|7673|589|2047|3|1|16|1|2047|589|7673|Wednesday|2047Q1|N|N|N|2468712|2468711|2468362|2468635|N|N|N|N|N| +2468728|AAAAAAAAIHLKFCAA|2047-01-17|1764|7673|589|2047|4|1|17|1|2047|589|7673|Thursday|2047Q1|N|N|N|2468712|2468711|2468363|2468636|N|N|N|N|N| +2468729|AAAAAAAAJHLKFCAA|2047-01-18|1764|7673|589|2047|5|1|18|1|2047|589|7673|Friday|2047Q1|N|Y|N|2468712|2468711|2468364|2468637|N|N|N|N|N| +2468730|AAAAAAAAKHLKFCAA|2047-01-19|1764|7673|589|2047|6|1|19|1|2047|589|7673|Saturday|2047Q1|N|Y|N|2468712|2468711|2468365|2468638|N|N|N|N|N| +2468731|AAAAAAAALHLKFCAA|2047-01-20|1764|7673|589|2047|0|1|20|1|2047|589|7673|Sunday|2047Q1|N|N|N|2468712|2468711|2468366|2468639|N|N|N|N|N| +2468732|AAAAAAAAMHLKFCAA|2047-01-21|1764|7673|589|2047|1|1|21|1|2047|589|7673|Monday|2047Q1|N|N|N|2468712|2468711|2468367|2468640|N|N|N|N|N| +2468733|AAAAAAAANHLKFCAA|2047-01-22|1764|7674|589|2047|2|1|22|1|2047|589|7674|Tuesday|2047Q1|N|N|N|2468712|2468711|2468368|2468641|N|N|N|N|N| +2468734|AAAAAAAAOHLKFCAA|2047-01-23|1764|7674|589|2047|3|1|23|1|2047|589|7674|Wednesday|2047Q1|N|N|N|2468712|2468711|2468369|2468642|N|N|N|N|N| +2468735|AAAAAAAAPHLKFCAA|2047-01-24|1764|7674|589|2047|4|1|24|1|2047|589|7674|Thursday|2047Q1|N|N|N|2468712|2468711|2468370|2468643|N|N|N|N|N| +2468736|AAAAAAAAAILKFCAA|2047-01-25|1764|7674|589|2047|5|1|25|1|2047|589|7674|Friday|2047Q1|N|Y|N|2468712|2468711|2468371|2468644|N|N|N|N|N| +2468737|AAAAAAAABILKFCAA|2047-01-26|1764|7674|589|2047|6|1|26|1|2047|589|7674|Saturday|2047Q1|N|Y|N|2468712|2468711|2468372|2468645|N|N|N|N|N| +2468738|AAAAAAAACILKFCAA|2047-01-27|1764|7674|589|2047|0|1|27|1|2047|589|7674|Sunday|2047Q1|N|N|N|2468712|2468711|2468373|2468646|N|N|N|N|N| +2468739|AAAAAAAADILKFCAA|2047-01-28|1764|7674|589|2047|1|1|28|1|2047|589|7674|Monday|2047Q1|N|N|N|2468712|2468711|2468374|2468647|N|N|N|N|N| +2468740|AAAAAAAAEILKFCAA|2047-01-29|1764|7675|589|2047|2|1|29|1|2047|589|7675|Tuesday|2047Q1|N|N|N|2468712|2468711|2468375|2468648|N|N|N|N|N| +2468741|AAAAAAAAFILKFCAA|2047-01-30|1764|7675|589|2047|3|1|30|1|2047|589|7675|Wednesday|2047Q1|N|N|N|2468712|2468711|2468376|2468649|N|N|N|N|N| +2468742|AAAAAAAAGILKFCAA|2047-01-31|1764|7675|589|2047|4|1|31|1|2047|589|7675|Thursday|2047Q1|N|N|N|2468712|2468711|2468377|2468650|N|N|N|N|N| +2468743|AAAAAAAAHILKFCAA|2047-02-01|1765|7675|589|2047|5|2|1|1|2047|589|7675|Friday|2047Q1|N|Y|N|2468743|2468773|2468378|2468651|N|N|N|N|N| +2468744|AAAAAAAAIILKFCAA|2047-02-02|1765|7675|589|2047|6|2|2|1|2047|589|7675|Saturday|2047Q1|N|Y|N|2468743|2468773|2468379|2468652|N|N|N|N|N| +2468745|AAAAAAAAJILKFCAA|2047-02-03|1765|7675|589|2047|0|2|3|1|2047|589|7675|Sunday|2047Q1|N|N|N|2468743|2468773|2468380|2468653|N|N|N|N|N| +2468746|AAAAAAAAKILKFCAA|2047-02-04|1765|7675|589|2047|1|2|4|1|2047|589|7675|Monday|2047Q1|N|N|N|2468743|2468773|2468381|2468654|N|N|N|N|N| +2468747|AAAAAAAALILKFCAA|2047-02-05|1765|7676|589|2047|2|2|5|1|2047|589|7676|Tuesday|2047Q1|N|N|N|2468743|2468773|2468382|2468655|N|N|N|N|N| +2468748|AAAAAAAAMILKFCAA|2047-02-06|1765|7676|589|2047|3|2|6|1|2047|589|7676|Wednesday|2047Q1|N|N|N|2468743|2468773|2468383|2468656|N|N|N|N|N| +2468749|AAAAAAAANILKFCAA|2047-02-07|1765|7676|589|2047|4|2|7|1|2047|589|7676|Thursday|2047Q1|N|N|N|2468743|2468773|2468384|2468657|N|N|N|N|N| +2468750|AAAAAAAAOILKFCAA|2047-02-08|1765|7676|589|2047|5|2|8|1|2047|589|7676|Friday|2047Q1|N|Y|N|2468743|2468773|2468385|2468658|N|N|N|N|N| +2468751|AAAAAAAAPILKFCAA|2047-02-09|1765|7676|589|2047|6|2|9|1|2047|589|7676|Saturday|2047Q1|N|Y|N|2468743|2468773|2468386|2468659|N|N|N|N|N| +2468752|AAAAAAAAAJLKFCAA|2047-02-10|1765|7676|589|2047|0|2|10|1|2047|589|7676|Sunday|2047Q1|N|N|N|2468743|2468773|2468387|2468660|N|N|N|N|N| +2468753|AAAAAAAABJLKFCAA|2047-02-11|1765|7676|589|2047|1|2|11|1|2047|589|7676|Monday|2047Q1|N|N|N|2468743|2468773|2468388|2468661|N|N|N|N|N| +2468754|AAAAAAAACJLKFCAA|2047-02-12|1765|7677|589|2047|2|2|12|1|2047|589|7677|Tuesday|2047Q1|N|N|N|2468743|2468773|2468389|2468662|N|N|N|N|N| +2468755|AAAAAAAADJLKFCAA|2047-02-13|1765|7677|589|2047|3|2|13|1|2047|589|7677|Wednesday|2047Q1|N|N|N|2468743|2468773|2468390|2468663|N|N|N|N|N| +2468756|AAAAAAAAEJLKFCAA|2047-02-14|1765|7677|589|2047|4|2|14|1|2047|589|7677|Thursday|2047Q1|N|N|N|2468743|2468773|2468391|2468664|N|N|N|N|N| +2468757|AAAAAAAAFJLKFCAA|2047-02-15|1765|7677|589|2047|5|2|15|1|2047|589|7677|Friday|2047Q1|N|Y|N|2468743|2468773|2468392|2468665|N|N|N|N|N| +2468758|AAAAAAAAGJLKFCAA|2047-02-16|1765|7677|589|2047|6|2|16|1|2047|589|7677|Saturday|2047Q1|N|Y|N|2468743|2468773|2468393|2468666|N|N|N|N|N| +2468759|AAAAAAAAHJLKFCAA|2047-02-17|1765|7677|589|2047|0|2|17|1|2047|589|7677|Sunday|2047Q1|N|N|N|2468743|2468773|2468394|2468667|N|N|N|N|N| +2468760|AAAAAAAAIJLKFCAA|2047-02-18|1765|7677|589|2047|1|2|18|1|2047|589|7677|Monday|2047Q1|N|N|N|2468743|2468773|2468395|2468668|N|N|N|N|N| +2468761|AAAAAAAAJJLKFCAA|2047-02-19|1765|7678|589|2047|2|2|19|1|2047|589|7678|Tuesday|2047Q1|N|N|N|2468743|2468773|2468396|2468669|N|N|N|N|N| +2468762|AAAAAAAAKJLKFCAA|2047-02-20|1765|7678|589|2047|3|2|20|1|2047|589|7678|Wednesday|2047Q1|N|N|N|2468743|2468773|2468397|2468670|N|N|N|N|N| +2468763|AAAAAAAALJLKFCAA|2047-02-21|1765|7678|589|2047|4|2|21|1|2047|589|7678|Thursday|2047Q1|N|N|N|2468743|2468773|2468398|2468671|N|N|N|N|N| +2468764|AAAAAAAAMJLKFCAA|2047-02-22|1765|7678|589|2047|5|2|22|1|2047|589|7678|Friday|2047Q1|N|Y|N|2468743|2468773|2468399|2468672|N|N|N|N|N| +2468765|AAAAAAAANJLKFCAA|2047-02-23|1765|7678|589|2047|6|2|23|1|2047|589|7678|Saturday|2047Q1|N|Y|N|2468743|2468773|2468400|2468673|N|N|N|N|N| +2468766|AAAAAAAAOJLKFCAA|2047-02-24|1765|7678|589|2047|0|2|24|1|2047|589|7678|Sunday|2047Q1|N|N|N|2468743|2468773|2468401|2468674|N|N|N|N|N| +2468767|AAAAAAAAPJLKFCAA|2047-02-25|1765|7678|589|2047|1|2|25|1|2047|589|7678|Monday|2047Q1|N|N|N|2468743|2468773|2468402|2468675|N|N|N|N|N| +2468768|AAAAAAAAAKLKFCAA|2047-02-26|1765|7679|589|2047|2|2|26|1|2047|589|7679|Tuesday|2047Q1|N|N|N|2468743|2468773|2468403|2468676|N|N|N|N|N| +2468769|AAAAAAAABKLKFCAA|2047-02-27|1765|7679|589|2047|3|2|27|1|2047|589|7679|Wednesday|2047Q1|N|N|N|2468743|2468773|2468404|2468677|N|N|N|N|N| +2468770|AAAAAAAACKLKFCAA|2047-02-28|1765|7679|589|2047|4|2|28|1|2047|589|7679|Thursday|2047Q1|N|N|N|2468743|2468773|2468405|2468678|N|N|N|N|N| +2468771|AAAAAAAADKLKFCAA|2047-03-01|1766|7679|590|2047|5|3|1|1|2047|590|7679|Friday|2047Q1|N|Y|N|2468771|2468829|2468406|2468679|N|N|N|N|N| +2468772|AAAAAAAAEKLKFCAA|2047-03-02|1766|7679|590|2047|6|3|2|1|2047|590|7679|Saturday|2047Q1|N|Y|N|2468771|2468829|2468407|2468680|N|N|N|N|N| +2468773|AAAAAAAAFKLKFCAA|2047-03-03|1766|7679|590|2047|0|3|3|1|2047|590|7679|Sunday|2047Q1|N|N|N|2468771|2468829|2468408|2468681|N|N|N|N|N| +2468774|AAAAAAAAGKLKFCAA|2047-03-04|1766|7679|590|2047|1|3|4|1|2047|590|7679|Monday|2047Q1|N|N|N|2468771|2468829|2468409|2468682|N|N|N|N|N| +2468775|AAAAAAAAHKLKFCAA|2047-03-05|1766|7680|590|2047|2|3|5|1|2047|590|7680|Tuesday|2047Q1|N|N|N|2468771|2468829|2468410|2468683|N|N|N|N|N| +2468776|AAAAAAAAIKLKFCAA|2047-03-06|1766|7680|590|2047|3|3|6|1|2047|590|7680|Wednesday|2047Q1|N|N|N|2468771|2468829|2468411|2468684|N|N|N|N|N| +2468777|AAAAAAAAJKLKFCAA|2047-03-07|1766|7680|590|2047|4|3|7|1|2047|590|7680|Thursday|2047Q1|N|N|N|2468771|2468829|2468412|2468685|N|N|N|N|N| +2468778|AAAAAAAAKKLKFCAA|2047-03-08|1766|7680|590|2047|5|3|8|1|2047|590|7680|Friday|2047Q1|N|Y|N|2468771|2468829|2468413|2468686|N|N|N|N|N| +2468779|AAAAAAAALKLKFCAA|2047-03-09|1766|7680|590|2047|6|3|9|1|2047|590|7680|Saturday|2047Q1|N|Y|N|2468771|2468829|2468414|2468687|N|N|N|N|N| +2468780|AAAAAAAAMKLKFCAA|2047-03-10|1766|7680|590|2047|0|3|10|1|2047|590|7680|Sunday|2047Q1|N|N|N|2468771|2468829|2468415|2468688|N|N|N|N|N| +2468781|AAAAAAAANKLKFCAA|2047-03-11|1766|7680|590|2047|1|3|11|1|2047|590|7680|Monday|2047Q1|N|N|N|2468771|2468829|2468416|2468689|N|N|N|N|N| +2468782|AAAAAAAAOKLKFCAA|2047-03-12|1766|7681|590|2047|2|3|12|1|2047|590|7681|Tuesday|2047Q1|N|N|N|2468771|2468829|2468417|2468690|N|N|N|N|N| +2468783|AAAAAAAAPKLKFCAA|2047-03-13|1766|7681|590|2047|3|3|13|1|2047|590|7681|Wednesday|2047Q1|N|N|N|2468771|2468829|2468418|2468691|N|N|N|N|N| +2468784|AAAAAAAAALLKFCAA|2047-03-14|1766|7681|590|2047|4|3|14|1|2047|590|7681|Thursday|2047Q1|N|N|N|2468771|2468829|2468419|2468692|N|N|N|N|N| +2468785|AAAAAAAABLLKFCAA|2047-03-15|1766|7681|590|2047|5|3|15|1|2047|590|7681|Friday|2047Q1|N|Y|N|2468771|2468829|2468420|2468693|N|N|N|N|N| +2468786|AAAAAAAACLLKFCAA|2047-03-16|1766|7681|590|2047|6|3|16|1|2047|590|7681|Saturday|2047Q1|N|Y|N|2468771|2468829|2468421|2468694|N|N|N|N|N| +2468787|AAAAAAAADLLKFCAA|2047-03-17|1766|7681|590|2047|0|3|17|1|2047|590|7681|Sunday|2047Q1|N|N|N|2468771|2468829|2468422|2468695|N|N|N|N|N| +2468788|AAAAAAAAELLKFCAA|2047-03-18|1766|7681|590|2047|1|3|18|1|2047|590|7681|Monday|2047Q1|N|N|N|2468771|2468829|2468423|2468696|N|N|N|N|N| +2468789|AAAAAAAAFLLKFCAA|2047-03-19|1766|7682|590|2047|2|3|19|1|2047|590|7682|Tuesday|2047Q1|N|N|N|2468771|2468829|2468424|2468697|N|N|N|N|N| +2468790|AAAAAAAAGLLKFCAA|2047-03-20|1766|7682|590|2047|3|3|20|1|2047|590|7682|Wednesday|2047Q1|N|N|N|2468771|2468829|2468425|2468698|N|N|N|N|N| +2468791|AAAAAAAAHLLKFCAA|2047-03-21|1766|7682|590|2047|4|3|21|1|2047|590|7682|Thursday|2047Q1|N|N|N|2468771|2468829|2468426|2468699|N|N|N|N|N| +2468792|AAAAAAAAILLKFCAA|2047-03-22|1766|7682|590|2047|5|3|22|1|2047|590|7682|Friday|2047Q1|N|Y|N|2468771|2468829|2468427|2468700|N|N|N|N|N| +2468793|AAAAAAAAJLLKFCAA|2047-03-23|1766|7682|590|2047|6|3|23|1|2047|590|7682|Saturday|2047Q1|N|Y|N|2468771|2468829|2468428|2468701|N|N|N|N|N| +2468794|AAAAAAAAKLLKFCAA|2047-03-24|1766|7682|590|2047|0|3|24|1|2047|590|7682|Sunday|2047Q1|N|N|N|2468771|2468829|2468429|2468702|N|N|N|N|N| +2468795|AAAAAAAALLLKFCAA|2047-03-25|1766|7682|590|2047|1|3|25|1|2047|590|7682|Monday|2047Q1|N|N|N|2468771|2468829|2468430|2468703|N|N|N|N|N| +2468796|AAAAAAAAMLLKFCAA|2047-03-26|1766|7683|590|2047|2|3|26|1|2047|590|7683|Tuesday|2047Q1|N|N|N|2468771|2468829|2468431|2468704|N|N|N|N|N| +2468797|AAAAAAAANLLKFCAA|2047-03-27|1766|7683|590|2047|3|3|27|1|2047|590|7683|Wednesday|2047Q1|N|N|N|2468771|2468829|2468432|2468705|N|N|N|N|N| +2468798|AAAAAAAAOLLKFCAA|2047-03-28|1766|7683|590|2047|4|3|28|1|2047|590|7683|Thursday|2047Q1|N|N|N|2468771|2468829|2468433|2468706|N|N|N|N|N| +2468799|AAAAAAAAPLLKFCAA|2047-03-29|1766|7683|590|2047|5|3|29|1|2047|590|7683|Friday|2047Q1|N|Y|N|2468771|2468829|2468434|2468707|N|N|N|N|N| +2468800|AAAAAAAAAMLKFCAA|2047-03-30|1766|7683|590|2047|6|3|30|1|2047|590|7683|Saturday|2047Q1|N|Y|N|2468771|2468829|2468435|2468708|N|N|N|N|N| +2468801|AAAAAAAABMLKFCAA|2047-03-31|1766|7683|590|2047|0|3|31|1|2047|590|7683|Sunday|2047Q1|N|N|N|2468771|2468829|2468436|2468709|N|N|N|N|N| +2468802|AAAAAAAACMLKFCAA|2047-04-01|1767|7683|590|2047|1|4|1|1|2047|590|7683|Monday|2047Q1|N|N|N|2468802|2468891|2468437|2468712|N|N|N|N|N| +2468803|AAAAAAAADMLKFCAA|2047-04-02|1767|7684|590|2047|2|4|2|2|2047|590|7684|Tuesday|2047Q2|N|N|N|2468802|2468891|2468438|2468713|N|N|N|N|N| +2468804|AAAAAAAAEMLKFCAA|2047-04-03|1767|7684|590|2047|3|4|3|2|2047|590|7684|Wednesday|2047Q2|N|N|N|2468802|2468891|2468439|2468714|N|N|N|N|N| +2468805|AAAAAAAAFMLKFCAA|2047-04-04|1767|7684|590|2047|4|4|4|2|2047|590|7684|Thursday|2047Q2|N|N|N|2468802|2468891|2468440|2468715|N|N|N|N|N| +2468806|AAAAAAAAGMLKFCAA|2047-04-05|1767|7684|590|2047|5|4|5|2|2047|590|7684|Friday|2047Q2|N|Y|N|2468802|2468891|2468441|2468716|N|N|N|N|N| +2468807|AAAAAAAAHMLKFCAA|2047-04-06|1767|7684|590|2047|6|4|6|2|2047|590|7684|Saturday|2047Q2|N|Y|N|2468802|2468891|2468442|2468717|N|N|N|N|N| +2468808|AAAAAAAAIMLKFCAA|2047-04-07|1767|7684|590|2047|0|4|7|2|2047|590|7684|Sunday|2047Q2|N|N|N|2468802|2468891|2468443|2468718|N|N|N|N|N| +2468809|AAAAAAAAJMLKFCAA|2047-04-08|1767|7684|590|2047|1|4|8|2|2047|590|7684|Monday|2047Q2|N|N|N|2468802|2468891|2468444|2468719|N|N|N|N|N| +2468810|AAAAAAAAKMLKFCAA|2047-04-09|1767|7685|590|2047|2|4|9|2|2047|590|7685|Tuesday|2047Q2|N|N|N|2468802|2468891|2468445|2468720|N|N|N|N|N| +2468811|AAAAAAAALMLKFCAA|2047-04-10|1767|7685|590|2047|3|4|10|2|2047|590|7685|Wednesday|2047Q2|N|N|N|2468802|2468891|2468446|2468721|N|N|N|N|N| +2468812|AAAAAAAAMMLKFCAA|2047-04-11|1767|7685|590|2047|4|4|11|2|2047|590|7685|Thursday|2047Q2|N|N|N|2468802|2468891|2468447|2468722|N|N|N|N|N| +2468813|AAAAAAAANMLKFCAA|2047-04-12|1767|7685|590|2047|5|4|12|2|2047|590|7685|Friday|2047Q2|N|Y|N|2468802|2468891|2468448|2468723|N|N|N|N|N| +2468814|AAAAAAAAOMLKFCAA|2047-04-13|1767|7685|590|2047|6|4|13|2|2047|590|7685|Saturday|2047Q2|N|Y|N|2468802|2468891|2468449|2468724|N|N|N|N|N| +2468815|AAAAAAAAPMLKFCAA|2047-04-14|1767|7685|590|2047|0|4|14|2|2047|590|7685|Sunday|2047Q2|N|N|N|2468802|2468891|2468450|2468725|N|N|N|N|N| +2468816|AAAAAAAAANLKFCAA|2047-04-15|1767|7685|590|2047|1|4|15|2|2047|590|7685|Monday|2047Q2|N|N|N|2468802|2468891|2468451|2468726|N|N|N|N|N| +2468817|AAAAAAAABNLKFCAA|2047-04-16|1767|7686|590|2047|2|4|16|2|2047|590|7686|Tuesday|2047Q2|N|N|N|2468802|2468891|2468452|2468727|N|N|N|N|N| +2468818|AAAAAAAACNLKFCAA|2047-04-17|1767|7686|590|2047|3|4|17|2|2047|590|7686|Wednesday|2047Q2|N|N|N|2468802|2468891|2468453|2468728|N|N|N|N|N| +2468819|AAAAAAAADNLKFCAA|2047-04-18|1767|7686|590|2047|4|4|18|2|2047|590|7686|Thursday|2047Q2|N|N|N|2468802|2468891|2468454|2468729|N|N|N|N|N| +2468820|AAAAAAAAENLKFCAA|2047-04-19|1767|7686|590|2047|5|4|19|2|2047|590|7686|Friday|2047Q2|N|Y|N|2468802|2468891|2468455|2468730|N|N|N|N|N| +2468821|AAAAAAAAFNLKFCAA|2047-04-20|1767|7686|590|2047|6|4|20|2|2047|590|7686|Saturday|2047Q2|N|Y|N|2468802|2468891|2468456|2468731|N|N|N|N|N| +2468822|AAAAAAAAGNLKFCAA|2047-04-21|1767|7686|590|2047|0|4|21|2|2047|590|7686|Sunday|2047Q2|N|N|N|2468802|2468891|2468457|2468732|N|N|N|N|N| +2468823|AAAAAAAAHNLKFCAA|2047-04-22|1767|7686|590|2047|1|4|22|2|2047|590|7686|Monday|2047Q2|N|N|N|2468802|2468891|2468458|2468733|N|N|N|N|N| +2468824|AAAAAAAAINLKFCAA|2047-04-23|1767|7687|590|2047|2|4|23|2|2047|590|7687|Tuesday|2047Q2|N|N|N|2468802|2468891|2468459|2468734|N|N|N|N|N| +2468825|AAAAAAAAJNLKFCAA|2047-04-24|1767|7687|590|2047|3|4|24|2|2047|590|7687|Wednesday|2047Q2|N|N|N|2468802|2468891|2468460|2468735|N|N|N|N|N| +2468826|AAAAAAAAKNLKFCAA|2047-04-25|1767|7687|590|2047|4|4|25|2|2047|590|7687|Thursday|2047Q2|N|N|N|2468802|2468891|2468461|2468736|N|N|N|N|N| +2468827|AAAAAAAALNLKFCAA|2047-04-26|1767|7687|590|2047|5|4|26|2|2047|590|7687|Friday|2047Q2|N|Y|N|2468802|2468891|2468462|2468737|N|N|N|N|N| +2468828|AAAAAAAAMNLKFCAA|2047-04-27|1767|7687|590|2047|6|4|27|2|2047|590|7687|Saturday|2047Q2|N|Y|N|2468802|2468891|2468463|2468738|N|N|N|N|N| +2468829|AAAAAAAANNLKFCAA|2047-04-28|1767|7687|590|2047|0|4|28|2|2047|590|7687|Sunday|2047Q2|N|N|N|2468802|2468891|2468464|2468739|N|N|N|N|N| +2468830|AAAAAAAAONLKFCAA|2047-04-29|1767|7687|590|2047|1|4|29|2|2047|590|7687|Monday|2047Q2|N|N|N|2468802|2468891|2468465|2468740|N|N|N|N|N| +2468831|AAAAAAAAPNLKFCAA|2047-04-30|1767|7688|590|2047|2|4|30|2|2047|590|7688|Tuesday|2047Q2|N|N|N|2468802|2468891|2468466|2468741|N|N|N|N|N| +2468832|AAAAAAAAAOLKFCAA|2047-05-01|1768|7688|590|2047|3|5|1|2|2047|590|7688|Wednesday|2047Q2|N|N|N|2468832|2468951|2468467|2468742|N|N|N|N|N| +2468833|AAAAAAAABOLKFCAA|2047-05-02|1768|7688|590|2047|4|5|2|2|2047|590|7688|Thursday|2047Q2|N|N|N|2468832|2468951|2468468|2468743|N|N|N|N|N| +2468834|AAAAAAAACOLKFCAA|2047-05-03|1768|7688|590|2047|5|5|3|2|2047|590|7688|Friday|2047Q2|N|Y|N|2468832|2468951|2468469|2468744|N|N|N|N|N| +2468835|AAAAAAAADOLKFCAA|2047-05-04|1768|7688|590|2047|6|5|4|2|2047|590|7688|Saturday|2047Q2|N|Y|N|2468832|2468951|2468470|2468745|N|N|N|N|N| +2468836|AAAAAAAAEOLKFCAA|2047-05-05|1768|7688|590|2047|0|5|5|2|2047|590|7688|Sunday|2047Q2|N|N|N|2468832|2468951|2468471|2468746|N|N|N|N|N| +2468837|AAAAAAAAFOLKFCAA|2047-05-06|1768|7688|590|2047|1|5|6|2|2047|590|7688|Monday|2047Q2|N|N|N|2468832|2468951|2468472|2468747|N|N|N|N|N| +2468838|AAAAAAAAGOLKFCAA|2047-05-07|1768|7689|590|2047|2|5|7|2|2047|590|7689|Tuesday|2047Q2|N|N|N|2468832|2468951|2468473|2468748|N|N|N|N|N| +2468839|AAAAAAAAHOLKFCAA|2047-05-08|1768|7689|590|2047|3|5|8|2|2047|590|7689|Wednesday|2047Q2|N|N|N|2468832|2468951|2468474|2468749|N|N|N|N|N| +2468840|AAAAAAAAIOLKFCAA|2047-05-09|1768|7689|590|2047|4|5|9|2|2047|590|7689|Thursday|2047Q2|N|N|N|2468832|2468951|2468475|2468750|N|N|N|N|N| +2468841|AAAAAAAAJOLKFCAA|2047-05-10|1768|7689|590|2047|5|5|10|2|2047|590|7689|Friday|2047Q2|N|Y|N|2468832|2468951|2468476|2468751|N|N|N|N|N| +2468842|AAAAAAAAKOLKFCAA|2047-05-11|1768|7689|590|2047|6|5|11|2|2047|590|7689|Saturday|2047Q2|N|Y|N|2468832|2468951|2468477|2468752|N|N|N|N|N| +2468843|AAAAAAAALOLKFCAA|2047-05-12|1768|7689|590|2047|0|5|12|2|2047|590|7689|Sunday|2047Q2|N|N|N|2468832|2468951|2468478|2468753|N|N|N|N|N| +2468844|AAAAAAAAMOLKFCAA|2047-05-13|1768|7689|590|2047|1|5|13|2|2047|590|7689|Monday|2047Q2|N|N|N|2468832|2468951|2468479|2468754|N|N|N|N|N| +2468845|AAAAAAAANOLKFCAA|2047-05-14|1768|7690|590|2047|2|5|14|2|2047|590|7690|Tuesday|2047Q2|N|N|N|2468832|2468951|2468480|2468755|N|N|N|N|N| +2468846|AAAAAAAAOOLKFCAA|2047-05-15|1768|7690|590|2047|3|5|15|2|2047|590|7690|Wednesday|2047Q2|N|N|N|2468832|2468951|2468481|2468756|N|N|N|N|N| +2468847|AAAAAAAAPOLKFCAA|2047-05-16|1768|7690|590|2047|4|5|16|2|2047|590|7690|Thursday|2047Q2|N|N|N|2468832|2468951|2468482|2468757|N|N|N|N|N| +2468848|AAAAAAAAAPLKFCAA|2047-05-17|1768|7690|590|2047|5|5|17|2|2047|590|7690|Friday|2047Q2|N|Y|N|2468832|2468951|2468483|2468758|N|N|N|N|N| +2468849|AAAAAAAABPLKFCAA|2047-05-18|1768|7690|590|2047|6|5|18|2|2047|590|7690|Saturday|2047Q2|N|Y|N|2468832|2468951|2468484|2468759|N|N|N|N|N| +2468850|AAAAAAAACPLKFCAA|2047-05-19|1768|7690|590|2047|0|5|19|2|2047|590|7690|Sunday|2047Q2|N|N|N|2468832|2468951|2468485|2468760|N|N|N|N|N| +2468851|AAAAAAAADPLKFCAA|2047-05-20|1768|7690|590|2047|1|5|20|2|2047|590|7690|Monday|2047Q2|N|N|N|2468832|2468951|2468486|2468761|N|N|N|N|N| +2468852|AAAAAAAAEPLKFCAA|2047-05-21|1768|7691|590|2047|2|5|21|2|2047|590|7691|Tuesday|2047Q2|N|N|N|2468832|2468951|2468487|2468762|N|N|N|N|N| +2468853|AAAAAAAAFPLKFCAA|2047-05-22|1768|7691|590|2047|3|5|22|2|2047|590|7691|Wednesday|2047Q2|N|N|N|2468832|2468951|2468488|2468763|N|N|N|N|N| +2468854|AAAAAAAAGPLKFCAA|2047-05-23|1768|7691|590|2047|4|5|23|2|2047|590|7691|Thursday|2047Q2|N|N|N|2468832|2468951|2468489|2468764|N|N|N|N|N| +2468855|AAAAAAAAHPLKFCAA|2047-05-24|1768|7691|590|2047|5|5|24|2|2047|590|7691|Friday|2047Q2|N|Y|N|2468832|2468951|2468490|2468765|N|N|N|N|N| +2468856|AAAAAAAAIPLKFCAA|2047-05-25|1768|7691|590|2047|6|5|25|2|2047|590|7691|Saturday|2047Q2|N|Y|N|2468832|2468951|2468491|2468766|N|N|N|N|N| +2468857|AAAAAAAAJPLKFCAA|2047-05-26|1768|7691|590|2047|0|5|26|2|2047|590|7691|Sunday|2047Q2|N|N|N|2468832|2468951|2468492|2468767|N|N|N|N|N| +2468858|AAAAAAAAKPLKFCAA|2047-05-27|1768|7691|590|2047|1|5|27|2|2047|590|7691|Monday|2047Q2|N|N|N|2468832|2468951|2468493|2468768|N|N|N|N|N| +2468859|AAAAAAAALPLKFCAA|2047-05-28|1768|7692|590|2047|2|5|28|2|2047|590|7692|Tuesday|2047Q2|N|N|N|2468832|2468951|2468494|2468769|N|N|N|N|N| +2468860|AAAAAAAAMPLKFCAA|2047-05-29|1768|7692|590|2047|3|5|29|2|2047|590|7692|Wednesday|2047Q2|N|N|N|2468832|2468951|2468495|2468770|N|N|N|N|N| +2468861|AAAAAAAANPLKFCAA|2047-05-30|1768|7692|590|2047|4|5|30|2|2047|590|7692|Thursday|2047Q2|N|N|N|2468832|2468951|2468496|2468771|N|N|N|N|N| +2468862|AAAAAAAAOPLKFCAA|2047-05-31|1768|7692|590|2047|5|5|31|2|2047|590|7692|Friday|2047Q2|N|Y|N|2468832|2468951|2468497|2468772|N|N|N|N|N| +2468863|AAAAAAAAPPLKFCAA|2047-06-01|1769|7692|591|2047|6|6|1|2|2047|591|7692|Saturday|2047Q2|N|Y|N|2468863|2469013|2468498|2468773|N|N|N|N|N| +2468864|AAAAAAAAAAMKFCAA|2047-06-02|1769|7692|591|2047|0|6|2|2|2047|591|7692|Sunday|2047Q2|N|N|N|2468863|2469013|2468499|2468774|N|N|N|N|N| +2468865|AAAAAAAABAMKFCAA|2047-06-03|1769|7692|591|2047|1|6|3|2|2047|591|7692|Monday|2047Q2|N|N|N|2468863|2469013|2468500|2468775|N|N|N|N|N| +2468866|AAAAAAAACAMKFCAA|2047-06-04|1769|7693|591|2047|2|6|4|2|2047|591|7693|Tuesday|2047Q2|N|N|N|2468863|2469013|2468501|2468776|N|N|N|N|N| +2468867|AAAAAAAADAMKFCAA|2047-06-05|1769|7693|591|2047|3|6|5|2|2047|591|7693|Wednesday|2047Q2|N|N|N|2468863|2469013|2468502|2468777|N|N|N|N|N| +2468868|AAAAAAAAEAMKFCAA|2047-06-06|1769|7693|591|2047|4|6|6|2|2047|591|7693|Thursday|2047Q2|N|N|N|2468863|2469013|2468503|2468778|N|N|N|N|N| +2468869|AAAAAAAAFAMKFCAA|2047-06-07|1769|7693|591|2047|5|6|7|2|2047|591|7693|Friday|2047Q2|N|Y|N|2468863|2469013|2468504|2468779|N|N|N|N|N| +2468870|AAAAAAAAGAMKFCAA|2047-06-08|1769|7693|591|2047|6|6|8|2|2047|591|7693|Saturday|2047Q2|N|Y|N|2468863|2469013|2468505|2468780|N|N|N|N|N| +2468871|AAAAAAAAHAMKFCAA|2047-06-09|1769|7693|591|2047|0|6|9|2|2047|591|7693|Sunday|2047Q2|N|N|N|2468863|2469013|2468506|2468781|N|N|N|N|N| +2468872|AAAAAAAAIAMKFCAA|2047-06-10|1769|7693|591|2047|1|6|10|2|2047|591|7693|Monday|2047Q2|N|N|N|2468863|2469013|2468507|2468782|N|N|N|N|N| +2468873|AAAAAAAAJAMKFCAA|2047-06-11|1769|7694|591|2047|2|6|11|2|2047|591|7694|Tuesday|2047Q2|N|N|N|2468863|2469013|2468508|2468783|N|N|N|N|N| +2468874|AAAAAAAAKAMKFCAA|2047-06-12|1769|7694|591|2047|3|6|12|2|2047|591|7694|Wednesday|2047Q2|N|N|N|2468863|2469013|2468509|2468784|N|N|N|N|N| +2468875|AAAAAAAALAMKFCAA|2047-06-13|1769|7694|591|2047|4|6|13|2|2047|591|7694|Thursday|2047Q2|N|N|N|2468863|2469013|2468510|2468785|N|N|N|N|N| +2468876|AAAAAAAAMAMKFCAA|2047-06-14|1769|7694|591|2047|5|6|14|2|2047|591|7694|Friday|2047Q2|N|Y|N|2468863|2469013|2468511|2468786|N|N|N|N|N| +2468877|AAAAAAAANAMKFCAA|2047-06-15|1769|7694|591|2047|6|6|15|2|2047|591|7694|Saturday|2047Q2|N|Y|N|2468863|2469013|2468512|2468787|N|N|N|N|N| +2468878|AAAAAAAAOAMKFCAA|2047-06-16|1769|7694|591|2047|0|6|16|2|2047|591|7694|Sunday|2047Q2|N|N|N|2468863|2469013|2468513|2468788|N|N|N|N|N| +2468879|AAAAAAAAPAMKFCAA|2047-06-17|1769|7694|591|2047|1|6|17|2|2047|591|7694|Monday|2047Q2|N|N|N|2468863|2469013|2468514|2468789|N|N|N|N|N| +2468880|AAAAAAAAABMKFCAA|2047-06-18|1769|7695|591|2047|2|6|18|2|2047|591|7695|Tuesday|2047Q2|N|N|N|2468863|2469013|2468515|2468790|N|N|N|N|N| +2468881|AAAAAAAABBMKFCAA|2047-06-19|1769|7695|591|2047|3|6|19|2|2047|591|7695|Wednesday|2047Q2|N|N|N|2468863|2469013|2468516|2468791|N|N|N|N|N| +2468882|AAAAAAAACBMKFCAA|2047-06-20|1769|7695|591|2047|4|6|20|2|2047|591|7695|Thursday|2047Q2|N|N|N|2468863|2469013|2468517|2468792|N|N|N|N|N| +2468883|AAAAAAAADBMKFCAA|2047-06-21|1769|7695|591|2047|5|6|21|2|2047|591|7695|Friday|2047Q2|N|Y|N|2468863|2469013|2468518|2468793|N|N|N|N|N| +2468884|AAAAAAAAEBMKFCAA|2047-06-22|1769|7695|591|2047|6|6|22|2|2047|591|7695|Saturday|2047Q2|N|Y|N|2468863|2469013|2468519|2468794|N|N|N|N|N| +2468885|AAAAAAAAFBMKFCAA|2047-06-23|1769|7695|591|2047|0|6|23|2|2047|591|7695|Sunday|2047Q2|N|N|N|2468863|2469013|2468520|2468795|N|N|N|N|N| +2468886|AAAAAAAAGBMKFCAA|2047-06-24|1769|7695|591|2047|1|6|24|2|2047|591|7695|Monday|2047Q2|N|N|N|2468863|2469013|2468521|2468796|N|N|N|N|N| +2468887|AAAAAAAAHBMKFCAA|2047-06-25|1769|7696|591|2047|2|6|25|2|2047|591|7696|Tuesday|2047Q2|N|N|N|2468863|2469013|2468522|2468797|N|N|N|N|N| +2468888|AAAAAAAAIBMKFCAA|2047-06-26|1769|7696|591|2047|3|6|26|2|2047|591|7696|Wednesday|2047Q2|N|N|N|2468863|2469013|2468523|2468798|N|N|N|N|N| +2468889|AAAAAAAAJBMKFCAA|2047-06-27|1769|7696|591|2047|4|6|27|2|2047|591|7696|Thursday|2047Q2|N|N|N|2468863|2469013|2468524|2468799|N|N|N|N|N| +2468890|AAAAAAAAKBMKFCAA|2047-06-28|1769|7696|591|2047|5|6|28|2|2047|591|7696|Friday|2047Q2|N|Y|N|2468863|2469013|2468525|2468800|N|N|N|N|N| +2468891|AAAAAAAALBMKFCAA|2047-06-29|1769|7696|591|2047|6|6|29|2|2047|591|7696|Saturday|2047Q2|N|Y|N|2468863|2469013|2468526|2468801|N|N|N|N|N| +2468892|AAAAAAAAMBMKFCAA|2047-06-30|1769|7696|591|2047|0|6|30|2|2047|591|7696|Sunday|2047Q2|N|N|N|2468863|2469013|2468527|2468802|N|N|N|N|N| +2468893|AAAAAAAANBMKFCAA|2047-07-01|1770|7696|591|2047|1|7|1|2|2047|591|7696|Monday|2047Q2|N|N|N|2468893|2469073|2468528|2468802|N|N|N|N|N| +2468894|AAAAAAAAOBMKFCAA|2047-07-02|1770|7697|591|2047|2|7|2|3|2047|591|7697|Tuesday|2047Q3|N|N|N|2468893|2469073|2468529|2468803|N|N|N|N|N| +2468895|AAAAAAAAPBMKFCAA|2047-07-03|1770|7697|591|2047|3|7|3|3|2047|591|7697|Wednesday|2047Q3|N|N|N|2468893|2469073|2468530|2468804|N|N|N|N|N| +2468896|AAAAAAAAACMKFCAA|2047-07-04|1770|7697|591|2047|4|7|4|3|2047|591|7697|Thursday|2047Q3|N|N|N|2468893|2469073|2468531|2468805|N|N|N|N|N| +2468897|AAAAAAAABCMKFCAA|2047-07-05|1770|7697|591|2047|5|7|5|3|2047|591|7697|Friday|2047Q3|Y|Y|N|2468893|2469073|2468532|2468806|N|N|N|N|N| +2468898|AAAAAAAACCMKFCAA|2047-07-06|1770|7697|591|2047|6|7|6|3|2047|591|7697|Saturday|2047Q3|N|Y|Y|2468893|2469073|2468533|2468807|N|N|N|N|N| +2468899|AAAAAAAADCMKFCAA|2047-07-07|1770|7697|591|2047|0|7|7|3|2047|591|7697|Sunday|2047Q3|N|N|N|2468893|2469073|2468534|2468808|N|N|N|N|N| +2468900|AAAAAAAAECMKFCAA|2047-07-08|1770|7697|591|2047|1|7|8|3|2047|591|7697|Monday|2047Q3|N|N|N|2468893|2469073|2468535|2468809|N|N|N|N|N| +2468901|AAAAAAAAFCMKFCAA|2047-07-09|1770|7698|591|2047|2|7|9|3|2047|591|7698|Tuesday|2047Q3|N|N|N|2468893|2469073|2468536|2468810|N|N|N|N|N| +2468902|AAAAAAAAGCMKFCAA|2047-07-10|1770|7698|591|2047|3|7|10|3|2047|591|7698|Wednesday|2047Q3|N|N|N|2468893|2469073|2468537|2468811|N|N|N|N|N| +2468903|AAAAAAAAHCMKFCAA|2047-07-11|1770|7698|591|2047|4|7|11|3|2047|591|7698|Thursday|2047Q3|N|N|N|2468893|2469073|2468538|2468812|N|N|N|N|N| +2468904|AAAAAAAAICMKFCAA|2047-07-12|1770|7698|591|2047|5|7|12|3|2047|591|7698|Friday|2047Q3|N|Y|N|2468893|2469073|2468539|2468813|N|N|N|N|N| +2468905|AAAAAAAAJCMKFCAA|2047-07-13|1770|7698|591|2047|6|7|13|3|2047|591|7698|Saturday|2047Q3|N|Y|N|2468893|2469073|2468540|2468814|N|N|N|N|N| +2468906|AAAAAAAAKCMKFCAA|2047-07-14|1770|7698|591|2047|0|7|14|3|2047|591|7698|Sunday|2047Q3|N|N|N|2468893|2469073|2468541|2468815|N|N|N|N|N| +2468907|AAAAAAAALCMKFCAA|2047-07-15|1770|7698|591|2047|1|7|15|3|2047|591|7698|Monday|2047Q3|N|N|N|2468893|2469073|2468542|2468816|N|N|N|N|N| +2468908|AAAAAAAAMCMKFCAA|2047-07-16|1770|7699|591|2047|2|7|16|3|2047|591|7699|Tuesday|2047Q3|N|N|N|2468893|2469073|2468543|2468817|N|N|N|N|N| +2468909|AAAAAAAANCMKFCAA|2047-07-17|1770|7699|591|2047|3|7|17|3|2047|591|7699|Wednesday|2047Q3|N|N|N|2468893|2469073|2468544|2468818|N|N|N|N|N| +2468910|AAAAAAAAOCMKFCAA|2047-07-18|1770|7699|591|2047|4|7|18|3|2047|591|7699|Thursday|2047Q3|N|N|N|2468893|2469073|2468545|2468819|N|N|N|N|N| +2468911|AAAAAAAAPCMKFCAA|2047-07-19|1770|7699|591|2047|5|7|19|3|2047|591|7699|Friday|2047Q3|N|Y|N|2468893|2469073|2468546|2468820|N|N|N|N|N| +2468912|AAAAAAAAADMKFCAA|2047-07-20|1770|7699|591|2047|6|7|20|3|2047|591|7699|Saturday|2047Q3|N|Y|N|2468893|2469073|2468547|2468821|N|N|N|N|N| +2468913|AAAAAAAABDMKFCAA|2047-07-21|1770|7699|591|2047|0|7|21|3|2047|591|7699|Sunday|2047Q3|N|N|N|2468893|2469073|2468548|2468822|N|N|N|N|N| +2468914|AAAAAAAACDMKFCAA|2047-07-22|1770|7699|591|2047|1|7|22|3|2047|591|7699|Monday|2047Q3|N|N|N|2468893|2469073|2468549|2468823|N|N|N|N|N| +2468915|AAAAAAAADDMKFCAA|2047-07-23|1770|7700|591|2047|2|7|23|3|2047|591|7700|Tuesday|2047Q3|N|N|N|2468893|2469073|2468550|2468824|N|N|N|N|N| +2468916|AAAAAAAAEDMKFCAA|2047-07-24|1770|7700|591|2047|3|7|24|3|2047|591|7700|Wednesday|2047Q3|N|N|N|2468893|2469073|2468551|2468825|N|N|N|N|N| +2468917|AAAAAAAAFDMKFCAA|2047-07-25|1770|7700|591|2047|4|7|25|3|2047|591|7700|Thursday|2047Q3|N|N|N|2468893|2469073|2468552|2468826|N|N|N|N|N| +2468918|AAAAAAAAGDMKFCAA|2047-07-26|1770|7700|591|2047|5|7|26|3|2047|591|7700|Friday|2047Q3|N|Y|N|2468893|2469073|2468553|2468827|N|N|N|N|N| +2468919|AAAAAAAAHDMKFCAA|2047-07-27|1770|7700|591|2047|6|7|27|3|2047|591|7700|Saturday|2047Q3|N|Y|N|2468893|2469073|2468554|2468828|N|N|N|N|N| +2468920|AAAAAAAAIDMKFCAA|2047-07-28|1770|7700|591|2047|0|7|28|3|2047|591|7700|Sunday|2047Q3|N|N|N|2468893|2469073|2468555|2468829|N|N|N|N|N| +2468921|AAAAAAAAJDMKFCAA|2047-07-29|1770|7700|591|2047|1|7|29|3|2047|591|7700|Monday|2047Q3|N|N|N|2468893|2469073|2468556|2468830|N|N|N|N|N| +2468922|AAAAAAAAKDMKFCAA|2047-07-30|1770|7701|591|2047|2|7|30|3|2047|591|7701|Tuesday|2047Q3|N|N|N|2468893|2469073|2468557|2468831|N|N|N|N|N| +2468923|AAAAAAAALDMKFCAA|2047-07-31|1770|7701|591|2047|3|7|31|3|2047|591|7701|Wednesday|2047Q3|N|N|N|2468893|2469073|2468558|2468832|N|N|N|N|N| +2468924|AAAAAAAAMDMKFCAA|2047-08-01|1771|7701|591|2047|4|8|1|3|2047|591|7701|Thursday|2047Q3|N|N|N|2468924|2469135|2468559|2468833|N|N|N|N|N| +2468925|AAAAAAAANDMKFCAA|2047-08-02|1771|7701|591|2047|5|8|2|3|2047|591|7701|Friday|2047Q3|N|Y|N|2468924|2469135|2468560|2468834|N|N|N|N|N| +2468926|AAAAAAAAODMKFCAA|2047-08-03|1771|7701|591|2047|6|8|3|3|2047|591|7701|Saturday|2047Q3|N|Y|N|2468924|2469135|2468561|2468835|N|N|N|N|N| +2468927|AAAAAAAAPDMKFCAA|2047-08-04|1771|7701|591|2047|0|8|4|3|2047|591|7701|Sunday|2047Q3|N|N|N|2468924|2469135|2468562|2468836|N|N|N|N|N| +2468928|AAAAAAAAAEMKFCAA|2047-08-05|1771|7701|591|2047|1|8|5|3|2047|591|7701|Monday|2047Q3|N|N|N|2468924|2469135|2468563|2468837|N|N|N|N|N| +2468929|AAAAAAAABEMKFCAA|2047-08-06|1771|7702|591|2047|2|8|6|3|2047|591|7702|Tuesday|2047Q3|N|N|N|2468924|2469135|2468564|2468838|N|N|N|N|N| +2468930|AAAAAAAACEMKFCAA|2047-08-07|1771|7702|591|2047|3|8|7|3|2047|591|7702|Wednesday|2047Q3|N|N|N|2468924|2469135|2468565|2468839|N|N|N|N|N| +2468931|AAAAAAAADEMKFCAA|2047-08-08|1771|7702|591|2047|4|8|8|3|2047|591|7702|Thursday|2047Q3|N|N|N|2468924|2469135|2468566|2468840|N|N|N|N|N| +2468932|AAAAAAAAEEMKFCAA|2047-08-09|1771|7702|591|2047|5|8|9|3|2047|591|7702|Friday|2047Q3|N|Y|N|2468924|2469135|2468567|2468841|N|N|N|N|N| +2468933|AAAAAAAAFEMKFCAA|2047-08-10|1771|7702|591|2047|6|8|10|3|2047|591|7702|Saturday|2047Q3|N|Y|N|2468924|2469135|2468568|2468842|N|N|N|N|N| +2468934|AAAAAAAAGEMKFCAA|2047-08-11|1771|7702|591|2047|0|8|11|3|2047|591|7702|Sunday|2047Q3|N|N|N|2468924|2469135|2468569|2468843|N|N|N|N|N| +2468935|AAAAAAAAHEMKFCAA|2047-08-12|1771|7702|591|2047|1|8|12|3|2047|591|7702|Monday|2047Q3|N|N|N|2468924|2469135|2468570|2468844|N|N|N|N|N| +2468936|AAAAAAAAIEMKFCAA|2047-08-13|1771|7703|591|2047|2|8|13|3|2047|591|7703|Tuesday|2047Q3|N|N|N|2468924|2469135|2468571|2468845|N|N|N|N|N| +2468937|AAAAAAAAJEMKFCAA|2047-08-14|1771|7703|591|2047|3|8|14|3|2047|591|7703|Wednesday|2047Q3|N|N|N|2468924|2469135|2468572|2468846|N|N|N|N|N| +2468938|AAAAAAAAKEMKFCAA|2047-08-15|1771|7703|591|2047|4|8|15|3|2047|591|7703|Thursday|2047Q3|N|N|N|2468924|2469135|2468573|2468847|N|N|N|N|N| +2468939|AAAAAAAALEMKFCAA|2047-08-16|1771|7703|591|2047|5|8|16|3|2047|591|7703|Friday|2047Q3|N|Y|N|2468924|2469135|2468574|2468848|N|N|N|N|N| +2468940|AAAAAAAAMEMKFCAA|2047-08-17|1771|7703|591|2047|6|8|17|3|2047|591|7703|Saturday|2047Q3|N|Y|N|2468924|2469135|2468575|2468849|N|N|N|N|N| +2468941|AAAAAAAANEMKFCAA|2047-08-18|1771|7703|591|2047|0|8|18|3|2047|591|7703|Sunday|2047Q3|N|N|N|2468924|2469135|2468576|2468850|N|N|N|N|N| +2468942|AAAAAAAAOEMKFCAA|2047-08-19|1771|7703|591|2047|1|8|19|3|2047|591|7703|Monday|2047Q3|N|N|N|2468924|2469135|2468577|2468851|N|N|N|N|N| +2468943|AAAAAAAAPEMKFCAA|2047-08-20|1771|7704|591|2047|2|8|20|3|2047|591|7704|Tuesday|2047Q3|N|N|N|2468924|2469135|2468578|2468852|N|N|N|N|N| +2468944|AAAAAAAAAFMKFCAA|2047-08-21|1771|7704|591|2047|3|8|21|3|2047|591|7704|Wednesday|2047Q3|N|N|N|2468924|2469135|2468579|2468853|N|N|N|N|N| +2468945|AAAAAAAABFMKFCAA|2047-08-22|1771|7704|591|2047|4|8|22|3|2047|591|7704|Thursday|2047Q3|N|N|N|2468924|2469135|2468580|2468854|N|N|N|N|N| +2468946|AAAAAAAACFMKFCAA|2047-08-23|1771|7704|591|2047|5|8|23|3|2047|591|7704|Friday|2047Q3|N|Y|N|2468924|2469135|2468581|2468855|N|N|N|N|N| +2468947|AAAAAAAADFMKFCAA|2047-08-24|1771|7704|591|2047|6|8|24|3|2047|591|7704|Saturday|2047Q3|N|Y|N|2468924|2469135|2468582|2468856|N|N|N|N|N| +2468948|AAAAAAAAEFMKFCAA|2047-08-25|1771|7704|591|2047|0|8|25|3|2047|591|7704|Sunday|2047Q3|N|N|N|2468924|2469135|2468583|2468857|N|N|N|N|N| +2468949|AAAAAAAAFFMKFCAA|2047-08-26|1771|7704|591|2047|1|8|26|3|2047|591|7704|Monday|2047Q3|N|N|N|2468924|2469135|2468584|2468858|N|N|N|N|N| +2468950|AAAAAAAAGFMKFCAA|2047-08-27|1771|7705|591|2047|2|8|27|3|2047|591|7705|Tuesday|2047Q3|N|N|N|2468924|2469135|2468585|2468859|N|N|N|N|N| +2468951|AAAAAAAAHFMKFCAA|2047-08-28|1771|7705|591|2047|3|8|28|3|2047|591|7705|Wednesday|2047Q3|N|N|N|2468924|2469135|2468586|2468860|N|N|N|N|N| +2468952|AAAAAAAAIFMKFCAA|2047-08-29|1771|7705|591|2047|4|8|29|3|2047|591|7705|Thursday|2047Q3|N|N|N|2468924|2469135|2468587|2468861|N|N|N|N|N| +2468953|AAAAAAAAJFMKFCAA|2047-08-30|1771|7705|591|2047|5|8|30|3|2047|591|7705|Friday|2047Q3|N|Y|N|2468924|2469135|2468588|2468862|N|N|N|N|N| +2468954|AAAAAAAAKFMKFCAA|2047-08-31|1771|7705|591|2047|6|8|31|3|2047|591|7705|Saturday|2047Q3|N|Y|N|2468924|2469135|2468589|2468863|N|N|N|N|N| +2468955|AAAAAAAALFMKFCAA|2047-09-01|1772|7705|592|2047|0|9|1|3|2047|592|7705|Sunday|2047Q3|N|N|N|2468955|2469197|2468590|2468864|N|N|N|N|N| +2468956|AAAAAAAAMFMKFCAA|2047-09-02|1772|7705|592|2047|1|9|2|3|2047|592|7705|Monday|2047Q3|N|N|N|2468955|2469197|2468591|2468865|N|N|N|N|N| +2468957|AAAAAAAANFMKFCAA|2047-09-03|1772|7706|592|2047|2|9|3|3|2047|592|7706|Tuesday|2047Q3|N|N|N|2468955|2469197|2468592|2468866|N|N|N|N|N| +2468958|AAAAAAAAOFMKFCAA|2047-09-04|1772|7706|592|2047|3|9|4|3|2047|592|7706|Wednesday|2047Q3|N|N|N|2468955|2469197|2468593|2468867|N|N|N|N|N| +2468959|AAAAAAAAPFMKFCAA|2047-09-05|1772|7706|592|2047|4|9|5|3|2047|592|7706|Thursday|2047Q3|N|N|N|2468955|2469197|2468594|2468868|N|N|N|N|N| +2468960|AAAAAAAAAGMKFCAA|2047-09-06|1772|7706|592|2047|5|9|6|3|2047|592|7706|Friday|2047Q3|N|Y|N|2468955|2469197|2468595|2468869|N|N|N|N|N| +2468961|AAAAAAAABGMKFCAA|2047-09-07|1772|7706|592|2047|6|9|7|3|2047|592|7706|Saturday|2047Q3|N|Y|N|2468955|2469197|2468596|2468870|N|N|N|N|N| +2468962|AAAAAAAACGMKFCAA|2047-09-08|1772|7706|592|2047|0|9|8|3|2047|592|7706|Sunday|2047Q3|N|N|N|2468955|2469197|2468597|2468871|N|N|N|N|N| +2468963|AAAAAAAADGMKFCAA|2047-09-09|1772|7706|592|2047|1|9|9|3|2047|592|7706|Monday|2047Q3|N|N|N|2468955|2469197|2468598|2468872|N|N|N|N|N| +2468964|AAAAAAAAEGMKFCAA|2047-09-10|1772|7707|592|2047|2|9|10|3|2047|592|7707|Tuesday|2047Q3|N|N|N|2468955|2469197|2468599|2468873|N|N|N|N|N| +2468965|AAAAAAAAFGMKFCAA|2047-09-11|1772|7707|592|2047|3|9|11|3|2047|592|7707|Wednesday|2047Q3|N|N|N|2468955|2469197|2468600|2468874|N|N|N|N|N| +2468966|AAAAAAAAGGMKFCAA|2047-09-12|1772|7707|592|2047|4|9|12|3|2047|592|7707|Thursday|2047Q3|N|N|N|2468955|2469197|2468601|2468875|N|N|N|N|N| +2468967|AAAAAAAAHGMKFCAA|2047-09-13|1772|7707|592|2047|5|9|13|3|2047|592|7707|Friday|2047Q3|N|Y|N|2468955|2469197|2468602|2468876|N|N|N|N|N| +2468968|AAAAAAAAIGMKFCAA|2047-09-14|1772|7707|592|2047|6|9|14|3|2047|592|7707|Saturday|2047Q3|N|Y|N|2468955|2469197|2468603|2468877|N|N|N|N|N| +2468969|AAAAAAAAJGMKFCAA|2047-09-15|1772|7707|592|2047|0|9|15|3|2047|592|7707|Sunday|2047Q3|N|N|N|2468955|2469197|2468604|2468878|N|N|N|N|N| +2468970|AAAAAAAAKGMKFCAA|2047-09-16|1772|7707|592|2047|1|9|16|3|2047|592|7707|Monday|2047Q3|N|N|N|2468955|2469197|2468605|2468879|N|N|N|N|N| +2468971|AAAAAAAALGMKFCAA|2047-09-17|1772|7708|592|2047|2|9|17|3|2047|592|7708|Tuesday|2047Q3|N|N|N|2468955|2469197|2468606|2468880|N|N|N|N|N| +2468972|AAAAAAAAMGMKFCAA|2047-09-18|1772|7708|592|2047|3|9|18|3|2047|592|7708|Wednesday|2047Q3|N|N|N|2468955|2469197|2468607|2468881|N|N|N|N|N| +2468973|AAAAAAAANGMKFCAA|2047-09-19|1772|7708|592|2047|4|9|19|3|2047|592|7708|Thursday|2047Q3|N|N|N|2468955|2469197|2468608|2468882|N|N|N|N|N| +2468974|AAAAAAAAOGMKFCAA|2047-09-20|1772|7708|592|2047|5|9|20|3|2047|592|7708|Friday|2047Q3|N|Y|N|2468955|2469197|2468609|2468883|N|N|N|N|N| +2468975|AAAAAAAAPGMKFCAA|2047-09-21|1772|7708|592|2047|6|9|21|3|2047|592|7708|Saturday|2047Q3|N|Y|N|2468955|2469197|2468610|2468884|N|N|N|N|N| +2468976|AAAAAAAAAHMKFCAA|2047-09-22|1772|7708|592|2047|0|9|22|3|2047|592|7708|Sunday|2047Q3|N|N|N|2468955|2469197|2468611|2468885|N|N|N|N|N| +2468977|AAAAAAAABHMKFCAA|2047-09-23|1772|7708|592|2047|1|9|23|3|2047|592|7708|Monday|2047Q3|N|N|N|2468955|2469197|2468612|2468886|N|N|N|N|N| +2468978|AAAAAAAACHMKFCAA|2047-09-24|1772|7709|592|2047|2|9|24|3|2047|592|7709|Tuesday|2047Q3|N|N|N|2468955|2469197|2468613|2468887|N|N|N|N|N| +2468979|AAAAAAAADHMKFCAA|2047-09-25|1772|7709|592|2047|3|9|25|3|2047|592|7709|Wednesday|2047Q3|N|N|N|2468955|2469197|2468614|2468888|N|N|N|N|N| +2468980|AAAAAAAAEHMKFCAA|2047-09-26|1772|7709|592|2047|4|9|26|3|2047|592|7709|Thursday|2047Q3|N|N|N|2468955|2469197|2468615|2468889|N|N|N|N|N| +2468981|AAAAAAAAFHMKFCAA|2047-09-27|1772|7709|592|2047|5|9|27|3|2047|592|7709|Friday|2047Q3|N|Y|N|2468955|2469197|2468616|2468890|N|N|N|N|N| +2468982|AAAAAAAAGHMKFCAA|2047-09-28|1772|7709|592|2047|6|9|28|3|2047|592|7709|Saturday|2047Q3|N|Y|N|2468955|2469197|2468617|2468891|N|N|N|N|N| +2468983|AAAAAAAAHHMKFCAA|2047-09-29|1772|7709|592|2047|0|9|29|3|2047|592|7709|Sunday|2047Q3|N|N|N|2468955|2469197|2468618|2468892|N|N|N|N|N| +2468984|AAAAAAAAIHMKFCAA|2047-09-30|1772|7709|592|2047|1|9|30|3|2047|592|7709|Monday|2047Q3|N|N|N|2468955|2469197|2468619|2468893|N|N|N|N|N| +2468985|AAAAAAAAJHMKFCAA|2047-10-01|1773|7710|592|2047|2|10|1|3|2047|592|7710|Tuesday|2047Q3|N|N|N|2468985|2469257|2468620|2468893|N|N|N|N|N| +2468986|AAAAAAAAKHMKFCAA|2047-10-02|1773|7710|592|2047|3|10|2|4|2047|592|7710|Wednesday|2047Q4|N|N|N|2468985|2469257|2468621|2468894|N|N|N|N|N| +2468987|AAAAAAAALHMKFCAA|2047-10-03|1773|7710|592|2047|4|10|3|4|2047|592|7710|Thursday|2047Q4|N|N|N|2468985|2469257|2468622|2468895|N|N|N|N|N| +2468988|AAAAAAAAMHMKFCAA|2047-10-04|1773|7710|592|2047|5|10|4|4|2047|592|7710|Friday|2047Q4|N|Y|N|2468985|2469257|2468623|2468896|N|N|N|N|N| +2468989|AAAAAAAANHMKFCAA|2047-10-05|1773|7710|592|2047|6|10|5|4|2047|592|7710|Saturday|2047Q4|N|Y|N|2468985|2469257|2468624|2468897|N|N|N|N|N| +2468990|AAAAAAAAOHMKFCAA|2047-10-06|1773|7710|592|2047|0|10|6|4|2047|592|7710|Sunday|2047Q4|N|N|N|2468985|2469257|2468625|2468898|N|N|N|N|N| +2468991|AAAAAAAAPHMKFCAA|2047-10-07|1773|7710|592|2047|1|10|7|4|2047|592|7710|Monday|2047Q4|N|N|N|2468985|2469257|2468626|2468899|N|N|N|N|N| +2468992|AAAAAAAAAIMKFCAA|2047-10-08|1773|7711|592|2047|2|10|8|4|2047|592|7711|Tuesday|2047Q4|N|N|N|2468985|2469257|2468627|2468900|N|N|N|N|N| +2468993|AAAAAAAABIMKFCAA|2047-10-09|1773|7711|592|2047|3|10|9|4|2047|592|7711|Wednesday|2047Q4|N|N|N|2468985|2469257|2468628|2468901|N|N|N|N|N| +2468994|AAAAAAAACIMKFCAA|2047-10-10|1773|7711|592|2047|4|10|10|4|2047|592|7711|Thursday|2047Q4|N|N|N|2468985|2469257|2468629|2468902|N|N|N|N|N| +2468995|AAAAAAAADIMKFCAA|2047-10-11|1773|7711|592|2047|5|10|11|4|2047|592|7711|Friday|2047Q4|N|Y|N|2468985|2469257|2468630|2468903|N|N|N|N|N| +2468996|AAAAAAAAEIMKFCAA|2047-10-12|1773|7711|592|2047|6|10|12|4|2047|592|7711|Saturday|2047Q4|N|Y|N|2468985|2469257|2468631|2468904|N|N|N|N|N| +2468997|AAAAAAAAFIMKFCAA|2047-10-13|1773|7711|592|2047|0|10|13|4|2047|592|7711|Sunday|2047Q4|N|N|N|2468985|2469257|2468632|2468905|N|N|N|N|N| +2468998|AAAAAAAAGIMKFCAA|2047-10-14|1773|7711|592|2047|1|10|14|4|2047|592|7711|Monday|2047Q4|N|N|N|2468985|2469257|2468633|2468906|N|N|N|N|N| +2468999|AAAAAAAAHIMKFCAA|2047-10-15|1773|7712|592|2047|2|10|15|4|2047|592|7712|Tuesday|2047Q4|N|N|N|2468985|2469257|2468634|2468907|N|N|N|N|N| +2469000|AAAAAAAAIIMKFCAA|2047-10-16|1773|7712|592|2047|3|10|16|4|2047|592|7712|Wednesday|2047Q4|N|N|N|2468985|2469257|2468635|2468908|N|N|N|N|N| +2469001|AAAAAAAAJIMKFCAA|2047-10-17|1773|7712|592|2047|4|10|17|4|2047|592|7712|Thursday|2047Q4|N|N|N|2468985|2469257|2468636|2468909|N|N|N|N|N| +2469002|AAAAAAAAKIMKFCAA|2047-10-18|1773|7712|592|2047|5|10|18|4|2047|592|7712|Friday|2047Q4|N|Y|N|2468985|2469257|2468637|2468910|N|N|N|N|N| +2469003|AAAAAAAALIMKFCAA|2047-10-19|1773|7712|592|2047|6|10|19|4|2047|592|7712|Saturday|2047Q4|N|Y|N|2468985|2469257|2468638|2468911|N|N|N|N|N| +2469004|AAAAAAAAMIMKFCAA|2047-10-20|1773|7712|592|2047|0|10|20|4|2047|592|7712|Sunday|2047Q4|N|N|N|2468985|2469257|2468639|2468912|N|N|N|N|N| +2469005|AAAAAAAANIMKFCAA|2047-10-21|1773|7712|592|2047|1|10|21|4|2047|592|7712|Monday|2047Q4|N|N|N|2468985|2469257|2468640|2468913|N|N|N|N|N| +2469006|AAAAAAAAOIMKFCAA|2047-10-22|1773|7713|592|2047|2|10|22|4|2047|592|7713|Tuesday|2047Q4|N|N|N|2468985|2469257|2468641|2468914|N|N|N|N|N| +2469007|AAAAAAAAPIMKFCAA|2047-10-23|1773|7713|592|2047|3|10|23|4|2047|592|7713|Wednesday|2047Q4|N|N|N|2468985|2469257|2468642|2468915|N|N|N|N|N| +2469008|AAAAAAAAAJMKFCAA|2047-10-24|1773|7713|592|2047|4|10|24|4|2047|592|7713|Thursday|2047Q4|N|N|N|2468985|2469257|2468643|2468916|N|N|N|N|N| +2469009|AAAAAAAABJMKFCAA|2047-10-25|1773|7713|592|2047|5|10|25|4|2047|592|7713|Friday|2047Q4|N|Y|N|2468985|2469257|2468644|2468917|N|N|N|N|N| +2469010|AAAAAAAACJMKFCAA|2047-10-26|1773|7713|592|2047|6|10|26|4|2047|592|7713|Saturday|2047Q4|N|Y|N|2468985|2469257|2468645|2468918|N|N|N|N|N| +2469011|AAAAAAAADJMKFCAA|2047-10-27|1773|7713|592|2047|0|10|27|4|2047|592|7713|Sunday|2047Q4|N|N|N|2468985|2469257|2468646|2468919|N|N|N|N|N| +2469012|AAAAAAAAEJMKFCAA|2047-10-28|1773|7713|592|2047|1|10|28|4|2047|592|7713|Monday|2047Q4|N|N|N|2468985|2469257|2468647|2468920|N|N|N|N|N| +2469013|AAAAAAAAFJMKFCAA|2047-10-29|1773|7714|592|2047|2|10|29|4|2047|592|7714|Tuesday|2047Q4|N|N|N|2468985|2469257|2468648|2468921|N|N|N|N|N| +2469014|AAAAAAAAGJMKFCAA|2047-10-30|1773|7714|592|2047|3|10|30|4|2047|592|7714|Wednesday|2047Q4|N|N|N|2468985|2469257|2468649|2468922|N|N|N|N|N| +2469015|AAAAAAAAHJMKFCAA|2047-10-31|1773|7714|592|2047|4|10|31|4|2047|592|7714|Thursday|2047Q4|N|N|N|2468985|2469257|2468650|2468923|N|N|N|N|N| +2469016|AAAAAAAAIJMKFCAA|2047-11-01|1774|7714|592|2047|5|11|1|4|2047|592|7714|Friday|2047Q4|N|Y|N|2469016|2469319|2468651|2468924|N|N|N|N|N| +2469017|AAAAAAAAJJMKFCAA|2047-11-02|1774|7714|592|2047|6|11|2|4|2047|592|7714|Saturday|2047Q4|N|Y|N|2469016|2469319|2468652|2468925|N|N|N|N|N| +2469018|AAAAAAAAKJMKFCAA|2047-11-03|1774|7714|592|2047|0|11|3|4|2047|592|7714|Sunday|2047Q4|N|N|N|2469016|2469319|2468653|2468926|N|N|N|N|N| +2469019|AAAAAAAALJMKFCAA|2047-11-04|1774|7714|592|2047|1|11|4|4|2047|592|7714|Monday|2047Q4|N|N|N|2469016|2469319|2468654|2468927|N|N|N|N|N| +2469020|AAAAAAAAMJMKFCAA|2047-11-05|1774|7715|592|2047|2|11|5|4|2047|592|7715|Tuesday|2047Q4|N|N|N|2469016|2469319|2468655|2468928|N|N|N|N|N| +2469021|AAAAAAAANJMKFCAA|2047-11-06|1774|7715|592|2047|3|11|6|4|2047|592|7715|Wednesday|2047Q4|N|N|N|2469016|2469319|2468656|2468929|N|N|N|N|N| +2469022|AAAAAAAAOJMKFCAA|2047-11-07|1774|7715|592|2047|4|11|7|4|2047|592|7715|Thursday|2047Q4|N|N|N|2469016|2469319|2468657|2468930|N|N|N|N|N| +2469023|AAAAAAAAPJMKFCAA|2047-11-08|1774|7715|592|2047|5|11|8|4|2047|592|7715|Friday|2047Q4|N|Y|N|2469016|2469319|2468658|2468931|N|N|N|N|N| +2469024|AAAAAAAAAKMKFCAA|2047-11-09|1774|7715|592|2047|6|11|9|4|2047|592|7715|Saturday|2047Q4|N|Y|N|2469016|2469319|2468659|2468932|N|N|N|N|N| +2469025|AAAAAAAABKMKFCAA|2047-11-10|1774|7715|592|2047|0|11|10|4|2047|592|7715|Sunday|2047Q4|N|N|N|2469016|2469319|2468660|2468933|N|N|N|N|N| +2469026|AAAAAAAACKMKFCAA|2047-11-11|1774|7715|592|2047|1|11|11|4|2047|592|7715|Monday|2047Q4|N|N|N|2469016|2469319|2468661|2468934|N|N|N|N|N| +2469027|AAAAAAAADKMKFCAA|2047-11-12|1774|7716|592|2047|2|11|12|4|2047|592|7716|Tuesday|2047Q4|N|N|N|2469016|2469319|2468662|2468935|N|N|N|N|N| +2469028|AAAAAAAAEKMKFCAA|2047-11-13|1774|7716|592|2047|3|11|13|4|2047|592|7716|Wednesday|2047Q4|N|N|N|2469016|2469319|2468663|2468936|N|N|N|N|N| +2469029|AAAAAAAAFKMKFCAA|2047-11-14|1774|7716|592|2047|4|11|14|4|2047|592|7716|Thursday|2047Q4|N|N|N|2469016|2469319|2468664|2468937|N|N|N|N|N| +2469030|AAAAAAAAGKMKFCAA|2047-11-15|1774|7716|592|2047|5|11|15|4|2047|592|7716|Friday|2047Q4|N|Y|N|2469016|2469319|2468665|2468938|N|N|N|N|N| +2469031|AAAAAAAAHKMKFCAA|2047-11-16|1774|7716|592|2047|6|11|16|4|2047|592|7716|Saturday|2047Q4|N|Y|N|2469016|2469319|2468666|2468939|N|N|N|N|N| +2469032|AAAAAAAAIKMKFCAA|2047-11-17|1774|7716|592|2047|0|11|17|4|2047|592|7716|Sunday|2047Q4|N|N|N|2469016|2469319|2468667|2468940|N|N|N|N|N| +2469033|AAAAAAAAJKMKFCAA|2047-11-18|1774|7716|592|2047|1|11|18|4|2047|592|7716|Monday|2047Q4|N|N|N|2469016|2469319|2468668|2468941|N|N|N|N|N| +2469034|AAAAAAAAKKMKFCAA|2047-11-19|1774|7717|592|2047|2|11|19|4|2047|592|7717|Tuesday|2047Q4|N|N|N|2469016|2469319|2468669|2468942|N|N|N|N|N| +2469035|AAAAAAAALKMKFCAA|2047-11-20|1774|7717|592|2047|3|11|20|4|2047|592|7717|Wednesday|2047Q4|N|N|N|2469016|2469319|2468670|2468943|N|N|N|N|N| +2469036|AAAAAAAAMKMKFCAA|2047-11-21|1774|7717|592|2047|4|11|21|4|2047|592|7717|Thursday|2047Q4|N|N|N|2469016|2469319|2468671|2468944|N|N|N|N|N| +2469037|AAAAAAAANKMKFCAA|2047-11-22|1774|7717|592|2047|5|11|22|4|2047|592|7717|Friday|2047Q4|N|Y|N|2469016|2469319|2468672|2468945|N|N|N|N|N| +2469038|AAAAAAAAOKMKFCAA|2047-11-23|1774|7717|592|2047|6|11|23|4|2047|592|7717|Saturday|2047Q4|N|Y|N|2469016|2469319|2468673|2468946|N|N|N|N|N| +2469039|AAAAAAAAPKMKFCAA|2047-11-24|1774|7717|592|2047|0|11|24|4|2047|592|7717|Sunday|2047Q4|N|N|N|2469016|2469319|2468674|2468947|N|N|N|N|N| +2469040|AAAAAAAAALMKFCAA|2047-11-25|1774|7717|592|2047|1|11|25|4|2047|592|7717|Monday|2047Q4|N|N|N|2469016|2469319|2468675|2468948|N|N|N|N|N| +2469041|AAAAAAAABLMKFCAA|2047-11-26|1774|7718|592|2047|2|11|26|4|2047|592|7718|Tuesday|2047Q4|N|N|N|2469016|2469319|2468676|2468949|N|N|N|N|N| +2469042|AAAAAAAACLMKFCAA|2047-11-27|1774|7718|592|2047|3|11|27|4|2047|592|7718|Wednesday|2047Q4|N|N|N|2469016|2469319|2468677|2468950|N|N|N|N|N| +2469043|AAAAAAAADLMKFCAA|2047-11-28|1774|7718|592|2047|4|11|28|4|2047|592|7718|Thursday|2047Q4|N|N|N|2469016|2469319|2468678|2468951|N|N|N|N|N| +2469044|AAAAAAAAELMKFCAA|2047-11-29|1774|7718|592|2047|5|11|29|4|2047|592|7718|Friday|2047Q4|N|Y|N|2469016|2469319|2468679|2468952|N|N|N|N|N| +2469045|AAAAAAAAFLMKFCAA|2047-11-30|1774|7718|592|2047|6|11|30|4|2047|592|7718|Saturday|2047Q4|N|Y|N|2469016|2469319|2468680|2468953|N|N|N|N|N| +2469046|AAAAAAAAGLMKFCAA|2047-12-01|1775|7718|593|2047|0|12|1|4|2047|593|7718|Sunday|2047Q4|N|N|N|2469046|2469379|2468681|2468954|N|N|N|N|N| +2469047|AAAAAAAAHLMKFCAA|2047-12-02|1775|7718|593|2047|1|12|2|4|2047|593|7718|Monday|2047Q4|N|N|N|2469046|2469379|2468682|2468955|N|N|N|N|N| +2469048|AAAAAAAAILMKFCAA|2047-12-03|1775|7719|593|2047|2|12|3|4|2047|593|7719|Tuesday|2047Q4|N|N|N|2469046|2469379|2468683|2468956|N|N|N|N|N| +2469049|AAAAAAAAJLMKFCAA|2047-12-04|1775|7719|593|2047|3|12|4|4|2047|593|7719|Wednesday|2047Q4|N|N|N|2469046|2469379|2468684|2468957|N|N|N|N|N| +2469050|AAAAAAAAKLMKFCAA|2047-12-05|1775|7719|593|2047|4|12|5|4|2047|593|7719|Thursday|2047Q4|N|N|N|2469046|2469379|2468685|2468958|N|N|N|N|N| +2469051|AAAAAAAALLMKFCAA|2047-12-06|1775|7719|593|2047|5|12|6|4|2047|593|7719|Friday|2047Q4|N|Y|N|2469046|2469379|2468686|2468959|N|N|N|N|N| +2469052|AAAAAAAAMLMKFCAA|2047-12-07|1775|7719|593|2047|6|12|7|4|2047|593|7719|Saturday|2047Q4|N|Y|N|2469046|2469379|2468687|2468960|N|N|N|N|N| +2469053|AAAAAAAANLMKFCAA|2047-12-08|1775|7719|593|2047|0|12|8|4|2047|593|7719|Sunday|2047Q4|N|N|N|2469046|2469379|2468688|2468961|N|N|N|N|N| +2469054|AAAAAAAAOLMKFCAA|2047-12-09|1775|7719|593|2047|1|12|9|4|2047|593|7719|Monday|2047Q4|N|N|N|2469046|2469379|2468689|2468962|N|N|N|N|N| +2469055|AAAAAAAAPLMKFCAA|2047-12-10|1775|7720|593|2047|2|12|10|4|2047|593|7720|Tuesday|2047Q4|N|N|N|2469046|2469379|2468690|2468963|N|N|N|N|N| +2469056|AAAAAAAAAMMKFCAA|2047-12-11|1775|7720|593|2047|3|12|11|4|2047|593|7720|Wednesday|2047Q4|N|N|N|2469046|2469379|2468691|2468964|N|N|N|N|N| +2469057|AAAAAAAABMMKFCAA|2047-12-12|1775|7720|593|2047|4|12|12|4|2047|593|7720|Thursday|2047Q4|N|N|N|2469046|2469379|2468692|2468965|N|N|N|N|N| +2469058|AAAAAAAACMMKFCAA|2047-12-13|1775|7720|593|2047|5|12|13|4|2047|593|7720|Friday|2047Q4|N|Y|N|2469046|2469379|2468693|2468966|N|N|N|N|N| +2469059|AAAAAAAADMMKFCAA|2047-12-14|1775|7720|593|2047|6|12|14|4|2047|593|7720|Saturday|2047Q4|N|Y|N|2469046|2469379|2468694|2468967|N|N|N|N|N| +2469060|AAAAAAAAEMMKFCAA|2047-12-15|1775|7720|593|2047|0|12|15|4|2047|593|7720|Sunday|2047Q4|N|N|N|2469046|2469379|2468695|2468968|N|N|N|N|N| +2469061|AAAAAAAAFMMKFCAA|2047-12-16|1775|7720|593|2047|1|12|16|4|2047|593|7720|Monday|2047Q4|N|N|N|2469046|2469379|2468696|2468969|N|N|N|N|N| +2469062|AAAAAAAAGMMKFCAA|2047-12-17|1775|7721|593|2047|2|12|17|4|2047|593|7721|Tuesday|2047Q4|N|N|N|2469046|2469379|2468697|2468970|N|N|N|N|N| +2469063|AAAAAAAAHMMKFCAA|2047-12-18|1775|7721|593|2047|3|12|18|4|2047|593|7721|Wednesday|2047Q4|N|N|N|2469046|2469379|2468698|2468971|N|N|N|N|N| +2469064|AAAAAAAAIMMKFCAA|2047-12-19|1775|7721|593|2047|4|12|19|4|2047|593|7721|Thursday|2047Q4|N|N|N|2469046|2469379|2468699|2468972|N|N|N|N|N| +2469065|AAAAAAAAJMMKFCAA|2047-12-20|1775|7721|593|2047|5|12|20|4|2047|593|7721|Friday|2047Q4|N|Y|N|2469046|2469379|2468700|2468973|N|N|N|N|N| +2469066|AAAAAAAAKMMKFCAA|2047-12-21|1775|7721|593|2047|6|12|21|4|2047|593|7721|Saturday|2047Q4|N|Y|N|2469046|2469379|2468701|2468974|N|N|N|N|N| +2469067|AAAAAAAALMMKFCAA|2047-12-22|1775|7721|593|2047|0|12|22|4|2047|593|7721|Sunday|2047Q4|N|N|N|2469046|2469379|2468702|2468975|N|N|N|N|N| +2469068|AAAAAAAAMMMKFCAA|2047-12-23|1775|7721|593|2047|1|12|23|4|2047|593|7721|Monday|2047Q4|N|N|N|2469046|2469379|2468703|2468976|N|N|N|N|N| +2469069|AAAAAAAANMMKFCAA|2047-12-24|1775|7722|593|2047|2|12|24|4|2047|593|7722|Tuesday|2047Q4|N|N|N|2469046|2469379|2468704|2468977|N|N|N|N|N| +2469070|AAAAAAAAOMMKFCAA|2047-12-25|1775|7722|593|2047|3|12|25|4|2047|593|7722|Wednesday|2047Q4|N|N|N|2469046|2469379|2468705|2468978|N|N|N|N|N| +2469071|AAAAAAAAPMMKFCAA|2047-12-26|1775|7722|593|2047|4|12|26|4|2047|593|7722|Thursday|2047Q4|Y|N|N|2469046|2469379|2468706|2468979|N|N|N|N|N| +2469072|AAAAAAAAANMKFCAA|2047-12-27|1775|7722|593|2047|5|12|27|4|2047|593|7722|Friday|2047Q4|N|Y|Y|2469046|2469379|2468707|2468980|N|N|N|N|N| +2469073|AAAAAAAABNMKFCAA|2047-12-28|1775|7722|593|2047|6|12|28|4|2047|593|7722|Saturday|2047Q4|N|Y|N|2469046|2469379|2468708|2468981|N|N|N|N|N| +2469074|AAAAAAAACNMKFCAA|2047-12-29|1775|7722|593|2047|0|12|29|4|2047|593|7722|Sunday|2047Q4|N|N|N|2469046|2469379|2468709|2468982|N|N|N|N|N| +2469075|AAAAAAAADNMKFCAA|2047-12-30|1775|7722|593|2047|1|12|30|4|2047|593|7722|Monday|2047Q4|N|N|N|2469046|2469379|2468710|2468983|N|N|N|N|N| +2469076|AAAAAAAAENMKFCAA|2047-12-31|1775|7723|593|2047|2|12|31|4|2047|593|7723|Tuesday|2047Q4|N|N|N|2469046|2469379|2468711|2468984|N|N|N|N|N| +2469077|AAAAAAAAFNMKFCAA|2048-01-01|1776|7723|593|2048|3|1|1|1|2048|593|7723|Wednesday|2048Q1|Y|N|N|2469077|2469076|2468712|2468985|N|N|N|N|N| +2469078|AAAAAAAAGNMKFCAA|2048-01-02|1776|7723|593|2048|4|1|2|1|2048|593|7723|Thursday|2048Q1|N|N|Y|2469077|2469076|2468713|2468986|N|N|N|N|N| +2469079|AAAAAAAAHNMKFCAA|2048-01-03|1776|7723|593|2048|5|1|3|1|2048|593|7723|Friday|2048Q1|N|Y|N|2469077|2469076|2468714|2468987|N|N|N|N|N| +2469080|AAAAAAAAINMKFCAA|2048-01-04|1776|7723|593|2048|6|1|4|1|2048|593|7723|Saturday|2048Q1|N|Y|N|2469077|2469076|2468715|2468988|N|N|N|N|N| +2469081|AAAAAAAAJNMKFCAA|2048-01-05|1776|7723|593|2048|0|1|5|1|2048|593|7723|Sunday|2048Q1|N|N|N|2469077|2469076|2468716|2468989|N|N|N|N|N| +2469082|AAAAAAAAKNMKFCAA|2048-01-06|1776|7723|593|2048|1|1|6|1|2048|593|7723|Monday|2048Q1|N|N|N|2469077|2469076|2468717|2468990|N|N|N|N|N| +2469083|AAAAAAAALNMKFCAA|2048-01-07|1776|7724|593|2048|2|1|7|1|2048|593|7724|Tuesday|2048Q1|N|N|N|2469077|2469076|2468718|2468991|N|N|N|N|N| +2469084|AAAAAAAAMNMKFCAA|2048-01-08|1776|7724|593|2048|3|1|8|1|2048|593|7724|Wednesday|2048Q1|N|N|N|2469077|2469076|2468719|2468992|N|N|N|N|N| +2469085|AAAAAAAANNMKFCAA|2048-01-09|1776|7724|593|2048|4|1|9|1|2048|593|7724|Thursday|2048Q1|N|N|N|2469077|2469076|2468720|2468993|N|N|N|N|N| +2469086|AAAAAAAAONMKFCAA|2048-01-10|1776|7724|593|2048|5|1|10|1|2048|593|7724|Friday|2048Q1|N|Y|N|2469077|2469076|2468721|2468994|N|N|N|N|N| +2469087|AAAAAAAAPNMKFCAA|2048-01-11|1776|7724|593|2048|6|1|11|1|2048|593|7724|Saturday|2048Q1|N|Y|N|2469077|2469076|2468722|2468995|N|N|N|N|N| +2469088|AAAAAAAAAOMKFCAA|2048-01-12|1776|7724|593|2048|0|1|12|1|2048|593|7724|Sunday|2048Q1|N|N|N|2469077|2469076|2468723|2468996|N|N|N|N|N| +2469089|AAAAAAAABOMKFCAA|2048-01-13|1776|7724|593|2048|1|1|13|1|2048|593|7724|Monday|2048Q1|N|N|N|2469077|2469076|2468724|2468997|N|N|N|N|N| +2469090|AAAAAAAACOMKFCAA|2048-01-14|1776|7725|593|2048|2|1|14|1|2048|593|7725|Tuesday|2048Q1|N|N|N|2469077|2469076|2468725|2468998|N|N|N|N|N| +2469091|AAAAAAAADOMKFCAA|2048-01-15|1776|7725|593|2048|3|1|15|1|2048|593|7725|Wednesday|2048Q1|N|N|N|2469077|2469076|2468726|2468999|N|N|N|N|N| +2469092|AAAAAAAAEOMKFCAA|2048-01-16|1776|7725|593|2048|4|1|16|1|2048|593|7725|Thursday|2048Q1|N|N|N|2469077|2469076|2468727|2469000|N|N|N|N|N| +2469093|AAAAAAAAFOMKFCAA|2048-01-17|1776|7725|593|2048|5|1|17|1|2048|593|7725|Friday|2048Q1|N|Y|N|2469077|2469076|2468728|2469001|N|N|N|N|N| +2469094|AAAAAAAAGOMKFCAA|2048-01-18|1776|7725|593|2048|6|1|18|1|2048|593|7725|Saturday|2048Q1|N|Y|N|2469077|2469076|2468729|2469002|N|N|N|N|N| +2469095|AAAAAAAAHOMKFCAA|2048-01-19|1776|7725|593|2048|0|1|19|1|2048|593|7725|Sunday|2048Q1|N|N|N|2469077|2469076|2468730|2469003|N|N|N|N|N| +2469096|AAAAAAAAIOMKFCAA|2048-01-20|1776|7725|593|2048|1|1|20|1|2048|593|7725|Monday|2048Q1|N|N|N|2469077|2469076|2468731|2469004|N|N|N|N|N| +2469097|AAAAAAAAJOMKFCAA|2048-01-21|1776|7726|593|2048|2|1|21|1|2048|593|7726|Tuesday|2048Q1|N|N|N|2469077|2469076|2468732|2469005|N|N|N|N|N| +2469098|AAAAAAAAKOMKFCAA|2048-01-22|1776|7726|593|2048|3|1|22|1|2048|593|7726|Wednesday|2048Q1|N|N|N|2469077|2469076|2468733|2469006|N|N|N|N|N| +2469099|AAAAAAAALOMKFCAA|2048-01-23|1776|7726|593|2048|4|1|23|1|2048|593|7726|Thursday|2048Q1|N|N|N|2469077|2469076|2468734|2469007|N|N|N|N|N| +2469100|AAAAAAAAMOMKFCAA|2048-01-24|1776|7726|593|2048|5|1|24|1|2048|593|7726|Friday|2048Q1|N|Y|N|2469077|2469076|2468735|2469008|N|N|N|N|N| +2469101|AAAAAAAANOMKFCAA|2048-01-25|1776|7726|593|2048|6|1|25|1|2048|593|7726|Saturday|2048Q1|N|Y|N|2469077|2469076|2468736|2469009|N|N|N|N|N| +2469102|AAAAAAAAOOMKFCAA|2048-01-26|1776|7726|593|2048|0|1|26|1|2048|593|7726|Sunday|2048Q1|N|N|N|2469077|2469076|2468737|2469010|N|N|N|N|N| +2469103|AAAAAAAAPOMKFCAA|2048-01-27|1776|7726|593|2048|1|1|27|1|2048|593|7726|Monday|2048Q1|N|N|N|2469077|2469076|2468738|2469011|N|N|N|N|N| +2469104|AAAAAAAAAPMKFCAA|2048-01-28|1776|7727|593|2048|2|1|28|1|2048|593|7727|Tuesday|2048Q1|N|N|N|2469077|2469076|2468739|2469012|N|N|N|N|N| +2469105|AAAAAAAABPMKFCAA|2048-01-29|1776|7727|593|2048|3|1|29|1|2048|593|7727|Wednesday|2048Q1|N|N|N|2469077|2469076|2468740|2469013|N|N|N|N|N| +2469106|AAAAAAAACPMKFCAA|2048-01-30|1776|7727|593|2048|4|1|30|1|2048|593|7727|Thursday|2048Q1|N|N|N|2469077|2469076|2468741|2469014|N|N|N|N|N| +2469107|AAAAAAAADPMKFCAA|2048-01-31|1776|7727|593|2048|5|1|31|1|2048|593|7727|Friday|2048Q1|N|Y|N|2469077|2469076|2468742|2469015|N|N|N|N|N| +2469108|AAAAAAAAEPMKFCAA|2048-02-01|1777|7727|593|2048|6|2|1|1|2048|593|7727|Saturday|2048Q1|N|Y|N|2469108|2469138|2468743|2469016|N|N|N|N|N| +2469109|AAAAAAAAFPMKFCAA|2048-02-02|1777|7727|593|2048|0|2|2|1|2048|593|7727|Sunday|2048Q1|N|N|N|2469108|2469138|2468744|2469017|N|N|N|N|N| +2469110|AAAAAAAAGPMKFCAA|2048-02-03|1777|7727|593|2048|1|2|3|1|2048|593|7727|Monday|2048Q1|N|N|N|2469108|2469138|2468745|2469018|N|N|N|N|N| +2469111|AAAAAAAAHPMKFCAA|2048-02-04|1777|7728|593|2048|2|2|4|1|2048|593|7728|Tuesday|2048Q1|N|N|N|2469108|2469138|2468746|2469019|N|N|N|N|N| +2469112|AAAAAAAAIPMKFCAA|2048-02-05|1777|7728|593|2048|3|2|5|1|2048|593|7728|Wednesday|2048Q1|N|N|N|2469108|2469138|2468747|2469020|N|N|N|N|N| +2469113|AAAAAAAAJPMKFCAA|2048-02-06|1777|7728|593|2048|4|2|6|1|2048|593|7728|Thursday|2048Q1|N|N|N|2469108|2469138|2468748|2469021|N|N|N|N|N| +2469114|AAAAAAAAKPMKFCAA|2048-02-07|1777|7728|593|2048|5|2|7|1|2048|593|7728|Friday|2048Q1|N|Y|N|2469108|2469138|2468749|2469022|N|N|N|N|N| +2469115|AAAAAAAALPMKFCAA|2048-02-08|1777|7728|593|2048|6|2|8|1|2048|593|7728|Saturday|2048Q1|N|Y|N|2469108|2469138|2468750|2469023|N|N|N|N|N| +2469116|AAAAAAAAMPMKFCAA|2048-02-09|1777|7728|593|2048|0|2|9|1|2048|593|7728|Sunday|2048Q1|N|N|N|2469108|2469138|2468751|2469024|N|N|N|N|N| +2469117|AAAAAAAANPMKFCAA|2048-02-10|1777|7728|593|2048|1|2|10|1|2048|593|7728|Monday|2048Q1|N|N|N|2469108|2469138|2468752|2469025|N|N|N|N|N| +2469118|AAAAAAAAOPMKFCAA|2048-02-11|1777|7729|593|2048|2|2|11|1|2048|593|7729|Tuesday|2048Q1|N|N|N|2469108|2469138|2468753|2469026|N|N|N|N|N| +2469119|AAAAAAAAPPMKFCAA|2048-02-12|1777|7729|593|2048|3|2|12|1|2048|593|7729|Wednesday|2048Q1|N|N|N|2469108|2469138|2468754|2469027|N|N|N|N|N| +2469120|AAAAAAAAAANKFCAA|2048-02-13|1777|7729|593|2048|4|2|13|1|2048|593|7729|Thursday|2048Q1|N|N|N|2469108|2469138|2468755|2469028|N|N|N|N|N| +2469121|AAAAAAAABANKFCAA|2048-02-14|1777|7729|593|2048|5|2|14|1|2048|593|7729|Friday|2048Q1|N|Y|N|2469108|2469138|2468756|2469029|N|N|N|N|N| +2469122|AAAAAAAACANKFCAA|2048-02-15|1777|7729|593|2048|6|2|15|1|2048|593|7729|Saturday|2048Q1|N|Y|N|2469108|2469138|2468757|2469030|N|N|N|N|N| +2469123|AAAAAAAADANKFCAA|2048-02-16|1777|7729|593|2048|0|2|16|1|2048|593|7729|Sunday|2048Q1|N|N|N|2469108|2469138|2468758|2469031|N|N|N|N|N| +2469124|AAAAAAAAEANKFCAA|2048-02-17|1777|7729|593|2048|1|2|17|1|2048|593|7729|Monday|2048Q1|N|N|N|2469108|2469138|2468759|2469032|N|N|N|N|N| +2469125|AAAAAAAAFANKFCAA|2048-02-18|1777|7730|593|2048|2|2|18|1|2048|593|7730|Tuesday|2048Q1|N|N|N|2469108|2469138|2468760|2469033|N|N|N|N|N| +2469126|AAAAAAAAGANKFCAA|2048-02-19|1777|7730|593|2048|3|2|19|1|2048|593|7730|Wednesday|2048Q1|N|N|N|2469108|2469138|2468761|2469034|N|N|N|N|N| +2469127|AAAAAAAAHANKFCAA|2048-02-20|1777|7730|593|2048|4|2|20|1|2048|593|7730|Thursday|2048Q1|N|N|N|2469108|2469138|2468762|2469035|N|N|N|N|N| +2469128|AAAAAAAAIANKFCAA|2048-02-21|1777|7730|593|2048|5|2|21|1|2048|593|7730|Friday|2048Q1|N|Y|N|2469108|2469138|2468763|2469036|N|N|N|N|N| +2469129|AAAAAAAAJANKFCAA|2048-02-22|1777|7730|593|2048|6|2|22|1|2048|593|7730|Saturday|2048Q1|N|Y|N|2469108|2469138|2468764|2469037|N|N|N|N|N| +2469130|AAAAAAAAKANKFCAA|2048-02-23|1777|7730|593|2048|0|2|23|1|2048|593|7730|Sunday|2048Q1|N|N|N|2469108|2469138|2468765|2469038|N|N|N|N|N| +2469131|AAAAAAAALANKFCAA|2048-02-24|1777|7730|593|2048|1|2|24|1|2048|593|7730|Monday|2048Q1|N|N|N|2469108|2469138|2468766|2469039|N|N|N|N|N| +2469132|AAAAAAAAMANKFCAA|2048-02-25|1777|7731|593|2048|2|2|25|1|2048|593|7731|Tuesday|2048Q1|N|N|N|2469108|2469138|2468767|2469040|N|N|N|N|N| +2469133|AAAAAAAANANKFCAA|2048-02-26|1777|7731|593|2048|3|2|26|1|2048|593|7731|Wednesday|2048Q1|N|N|N|2469108|2469138|2468768|2469041|N|N|N|N|N| +2469134|AAAAAAAAOANKFCAA|2048-02-27|1777|7731|593|2048|4|2|27|1|2048|593|7731|Thursday|2048Q1|N|N|N|2469108|2469138|2468769|2469042|N|N|N|N|N| +2469135|AAAAAAAAPANKFCAA|2048-02-28|1777|7731|593|2048|5|2|28|1|2048|593|7731|Friday|2048Q1|N|Y|N|2469108|2469138|2468770|2469043|N|N|N|N|N| +2469136|AAAAAAAAABNKFCAA|2048-02-29|1777|7731|593|2048|6|2|29|1|2048|593|7731|Saturday|2048Q1|N|Y|N|2469108|2469138|2468770|2469044|N|N|N|N|N| +2469137|AAAAAAAABBNKFCAA|2048-03-01|1778|7731|594|2048|0|3|1|1|2048|594|7731|Sunday|2048Q1|N|N|N|2469137|2469196|2468771|2469045|N|N|N|N|N| +2469138|AAAAAAAACBNKFCAA|2048-03-02|1778|7731|594|2048|1|3|2|1|2048|594|7731|Monday|2048Q1|N|N|N|2469137|2469196|2468772|2469046|N|N|N|N|N| +2469139|AAAAAAAADBNKFCAA|2048-03-03|1778|7732|594|2048|2|3|3|1|2048|594|7732|Tuesday|2048Q1|N|N|N|2469137|2469196|2468773|2469047|N|N|N|N|N| +2469140|AAAAAAAAEBNKFCAA|2048-03-04|1778|7732|594|2048|3|3|4|1|2048|594|7732|Wednesday|2048Q1|N|N|N|2469137|2469196|2468774|2469048|N|N|N|N|N| +2469141|AAAAAAAAFBNKFCAA|2048-03-05|1778|7732|594|2048|4|3|5|1|2048|594|7732|Thursday|2048Q1|N|N|N|2469137|2469196|2468775|2469049|N|N|N|N|N| +2469142|AAAAAAAAGBNKFCAA|2048-03-06|1778|7732|594|2048|5|3|6|1|2048|594|7732|Friday|2048Q1|N|Y|N|2469137|2469196|2468776|2469050|N|N|N|N|N| +2469143|AAAAAAAAHBNKFCAA|2048-03-07|1778|7732|594|2048|6|3|7|1|2048|594|7732|Saturday|2048Q1|N|Y|N|2469137|2469196|2468777|2469051|N|N|N|N|N| +2469144|AAAAAAAAIBNKFCAA|2048-03-08|1778|7732|594|2048|0|3|8|1|2048|594|7732|Sunday|2048Q1|N|N|N|2469137|2469196|2468778|2469052|N|N|N|N|N| +2469145|AAAAAAAAJBNKFCAA|2048-03-09|1778|7732|594|2048|1|3|9|1|2048|594|7732|Monday|2048Q1|N|N|N|2469137|2469196|2468779|2469053|N|N|N|N|N| +2469146|AAAAAAAAKBNKFCAA|2048-03-10|1778|7733|594|2048|2|3|10|1|2048|594|7733|Tuesday|2048Q1|N|N|N|2469137|2469196|2468780|2469054|N|N|N|N|N| +2469147|AAAAAAAALBNKFCAA|2048-03-11|1778|7733|594|2048|3|3|11|1|2048|594|7733|Wednesday|2048Q1|N|N|N|2469137|2469196|2468781|2469055|N|N|N|N|N| +2469148|AAAAAAAAMBNKFCAA|2048-03-12|1778|7733|594|2048|4|3|12|1|2048|594|7733|Thursday|2048Q1|N|N|N|2469137|2469196|2468782|2469056|N|N|N|N|N| +2469149|AAAAAAAANBNKFCAA|2048-03-13|1778|7733|594|2048|5|3|13|1|2048|594|7733|Friday|2048Q1|N|Y|N|2469137|2469196|2468783|2469057|N|N|N|N|N| +2469150|AAAAAAAAOBNKFCAA|2048-03-14|1778|7733|594|2048|6|3|14|1|2048|594|7733|Saturday|2048Q1|N|Y|N|2469137|2469196|2468784|2469058|N|N|N|N|N| +2469151|AAAAAAAAPBNKFCAA|2048-03-15|1778|7733|594|2048|0|3|15|1|2048|594|7733|Sunday|2048Q1|N|N|N|2469137|2469196|2468785|2469059|N|N|N|N|N| +2469152|AAAAAAAAACNKFCAA|2048-03-16|1778|7733|594|2048|1|3|16|1|2048|594|7733|Monday|2048Q1|N|N|N|2469137|2469196|2468786|2469060|N|N|N|N|N| +2469153|AAAAAAAABCNKFCAA|2048-03-17|1778|7734|594|2048|2|3|17|1|2048|594|7734|Tuesday|2048Q1|N|N|N|2469137|2469196|2468787|2469061|N|N|N|N|N| +2469154|AAAAAAAACCNKFCAA|2048-03-18|1778|7734|594|2048|3|3|18|1|2048|594|7734|Wednesday|2048Q1|N|N|N|2469137|2469196|2468788|2469062|N|N|N|N|N| +2469155|AAAAAAAADCNKFCAA|2048-03-19|1778|7734|594|2048|4|3|19|1|2048|594|7734|Thursday|2048Q1|N|N|N|2469137|2469196|2468789|2469063|N|N|N|N|N| +2469156|AAAAAAAAECNKFCAA|2048-03-20|1778|7734|594|2048|5|3|20|1|2048|594|7734|Friday|2048Q1|N|Y|N|2469137|2469196|2468790|2469064|N|N|N|N|N| +2469157|AAAAAAAAFCNKFCAA|2048-03-21|1778|7734|594|2048|6|3|21|1|2048|594|7734|Saturday|2048Q1|N|Y|N|2469137|2469196|2468791|2469065|N|N|N|N|N| +2469158|AAAAAAAAGCNKFCAA|2048-03-22|1778|7734|594|2048|0|3|22|1|2048|594|7734|Sunday|2048Q1|N|N|N|2469137|2469196|2468792|2469066|N|N|N|N|N| +2469159|AAAAAAAAHCNKFCAA|2048-03-23|1778|7734|594|2048|1|3|23|1|2048|594|7734|Monday|2048Q1|N|N|N|2469137|2469196|2468793|2469067|N|N|N|N|N| +2469160|AAAAAAAAICNKFCAA|2048-03-24|1778|7735|594|2048|2|3|24|1|2048|594|7735|Tuesday|2048Q1|N|N|N|2469137|2469196|2468794|2469068|N|N|N|N|N| +2469161|AAAAAAAAJCNKFCAA|2048-03-25|1778|7735|594|2048|3|3|25|1|2048|594|7735|Wednesday|2048Q1|N|N|N|2469137|2469196|2468795|2469069|N|N|N|N|N| +2469162|AAAAAAAAKCNKFCAA|2048-03-26|1778|7735|594|2048|4|3|26|1|2048|594|7735|Thursday|2048Q1|N|N|N|2469137|2469196|2468796|2469070|N|N|N|N|N| +2469163|AAAAAAAALCNKFCAA|2048-03-27|1778|7735|594|2048|5|3|27|1|2048|594|7735|Friday|2048Q1|N|Y|N|2469137|2469196|2468797|2469071|N|N|N|N|N| +2469164|AAAAAAAAMCNKFCAA|2048-03-28|1778|7735|594|2048|6|3|28|1|2048|594|7735|Saturday|2048Q1|N|Y|N|2469137|2469196|2468798|2469072|N|N|N|N|N| +2469165|AAAAAAAANCNKFCAA|2048-03-29|1778|7735|594|2048|0|3|29|1|2048|594|7735|Sunday|2048Q1|N|N|N|2469137|2469196|2468799|2469073|N|N|N|N|N| +2469166|AAAAAAAAOCNKFCAA|2048-03-30|1778|7735|594|2048|1|3|30|1|2048|594|7735|Monday|2048Q1|N|N|N|2469137|2469196|2468800|2469074|N|N|N|N|N| +2469167|AAAAAAAAPCNKFCAA|2048-03-31|1778|7736|594|2048|2|3|31|1|2048|594|7736|Tuesday|2048Q1|N|N|N|2469137|2469196|2468801|2469075|N|N|N|N|N| +2469168|AAAAAAAAADNKFCAA|2048-04-01|1779|7736|594|2048|3|4|1|2|2048|594|7736|Wednesday|2048Q2|N|N|N|2469168|2469258|2468802|2469077|N|N|N|N|N| +2469169|AAAAAAAABDNKFCAA|2048-04-02|1779|7736|594|2048|4|4|2|2|2048|594|7736|Thursday|2048Q2|N|N|N|2469168|2469258|2468803|2469078|N|N|N|N|N| +2469170|AAAAAAAACDNKFCAA|2048-04-03|1779|7736|594|2048|5|4|3|2|2048|594|7736|Friday|2048Q2|N|Y|N|2469168|2469258|2468804|2469079|N|N|N|N|N| +2469171|AAAAAAAADDNKFCAA|2048-04-04|1779|7736|594|2048|6|4|4|2|2048|594|7736|Saturday|2048Q2|N|Y|N|2469168|2469258|2468805|2469080|N|N|N|N|N| +2469172|AAAAAAAAEDNKFCAA|2048-04-05|1779|7736|594|2048|0|4|5|2|2048|594|7736|Sunday|2048Q2|N|N|N|2469168|2469258|2468806|2469081|N|N|N|N|N| +2469173|AAAAAAAAFDNKFCAA|2048-04-06|1779|7736|594|2048|1|4|6|2|2048|594|7736|Monday|2048Q2|N|N|N|2469168|2469258|2468807|2469082|N|N|N|N|N| +2469174|AAAAAAAAGDNKFCAA|2048-04-07|1779|7737|594|2048|2|4|7|2|2048|594|7737|Tuesday|2048Q2|N|N|N|2469168|2469258|2468808|2469083|N|N|N|N|N| +2469175|AAAAAAAAHDNKFCAA|2048-04-08|1779|7737|594|2048|3|4|8|2|2048|594|7737|Wednesday|2048Q2|N|N|N|2469168|2469258|2468809|2469084|N|N|N|N|N| +2469176|AAAAAAAAIDNKFCAA|2048-04-09|1779|7737|594|2048|4|4|9|2|2048|594|7737|Thursday|2048Q2|N|N|N|2469168|2469258|2468810|2469085|N|N|N|N|N| +2469177|AAAAAAAAJDNKFCAA|2048-04-10|1779|7737|594|2048|5|4|10|2|2048|594|7737|Friday|2048Q2|N|Y|N|2469168|2469258|2468811|2469086|N|N|N|N|N| +2469178|AAAAAAAAKDNKFCAA|2048-04-11|1779|7737|594|2048|6|4|11|2|2048|594|7737|Saturday|2048Q2|N|Y|N|2469168|2469258|2468812|2469087|N|N|N|N|N| +2469179|AAAAAAAALDNKFCAA|2048-04-12|1779|7737|594|2048|0|4|12|2|2048|594|7737|Sunday|2048Q2|N|N|N|2469168|2469258|2468813|2469088|N|N|N|N|N| +2469180|AAAAAAAAMDNKFCAA|2048-04-13|1779|7737|594|2048|1|4|13|2|2048|594|7737|Monday|2048Q2|N|N|N|2469168|2469258|2468814|2469089|N|N|N|N|N| +2469181|AAAAAAAANDNKFCAA|2048-04-14|1779|7738|594|2048|2|4|14|2|2048|594|7738|Tuesday|2048Q2|N|N|N|2469168|2469258|2468815|2469090|N|N|N|N|N| +2469182|AAAAAAAAODNKFCAA|2048-04-15|1779|7738|594|2048|3|4|15|2|2048|594|7738|Wednesday|2048Q2|N|N|N|2469168|2469258|2468816|2469091|N|N|N|N|N| +2469183|AAAAAAAAPDNKFCAA|2048-04-16|1779|7738|594|2048|4|4|16|2|2048|594|7738|Thursday|2048Q2|N|N|N|2469168|2469258|2468817|2469092|N|N|N|N|N| +2469184|AAAAAAAAAENKFCAA|2048-04-17|1779|7738|594|2048|5|4|17|2|2048|594|7738|Friday|2048Q2|N|Y|N|2469168|2469258|2468818|2469093|N|N|N|N|N| +2469185|AAAAAAAABENKFCAA|2048-04-18|1779|7738|594|2048|6|4|18|2|2048|594|7738|Saturday|2048Q2|N|Y|N|2469168|2469258|2468819|2469094|N|N|N|N|N| +2469186|AAAAAAAACENKFCAA|2048-04-19|1779|7738|594|2048|0|4|19|2|2048|594|7738|Sunday|2048Q2|N|N|N|2469168|2469258|2468820|2469095|N|N|N|N|N| +2469187|AAAAAAAADENKFCAA|2048-04-20|1779|7738|594|2048|1|4|20|2|2048|594|7738|Monday|2048Q2|N|N|N|2469168|2469258|2468821|2469096|N|N|N|N|N| +2469188|AAAAAAAAEENKFCAA|2048-04-21|1779|7739|594|2048|2|4|21|2|2048|594|7739|Tuesday|2048Q2|N|N|N|2469168|2469258|2468822|2469097|N|N|N|N|N| +2469189|AAAAAAAAFENKFCAA|2048-04-22|1779|7739|594|2048|3|4|22|2|2048|594|7739|Wednesday|2048Q2|N|N|N|2469168|2469258|2468823|2469098|N|N|N|N|N| +2469190|AAAAAAAAGENKFCAA|2048-04-23|1779|7739|594|2048|4|4|23|2|2048|594|7739|Thursday|2048Q2|N|N|N|2469168|2469258|2468824|2469099|N|N|N|N|N| +2469191|AAAAAAAAHENKFCAA|2048-04-24|1779|7739|594|2048|5|4|24|2|2048|594|7739|Friday|2048Q2|N|Y|N|2469168|2469258|2468825|2469100|N|N|N|N|N| +2469192|AAAAAAAAIENKFCAA|2048-04-25|1779|7739|594|2048|6|4|25|2|2048|594|7739|Saturday|2048Q2|N|Y|N|2469168|2469258|2468826|2469101|N|N|N|N|N| +2469193|AAAAAAAAJENKFCAA|2048-04-26|1779|7739|594|2048|0|4|26|2|2048|594|7739|Sunday|2048Q2|N|N|N|2469168|2469258|2468827|2469102|N|N|N|N|N| +2469194|AAAAAAAAKENKFCAA|2048-04-27|1779|7739|594|2048|1|4|27|2|2048|594|7739|Monday|2048Q2|N|N|N|2469168|2469258|2468828|2469103|N|N|N|N|N| +2469195|AAAAAAAALENKFCAA|2048-04-28|1779|7740|594|2048|2|4|28|2|2048|594|7740|Tuesday|2048Q2|N|N|N|2469168|2469258|2468829|2469104|N|N|N|N|N| +2469196|AAAAAAAAMENKFCAA|2048-04-29|1779|7740|594|2048|3|4|29|2|2048|594|7740|Wednesday|2048Q2|N|N|N|2469168|2469258|2468830|2469105|N|N|N|N|N| +2469197|AAAAAAAANENKFCAA|2048-04-30|1779|7740|594|2048|4|4|30|2|2048|594|7740|Thursday|2048Q2|N|N|N|2469168|2469258|2468831|2469106|N|N|N|N|N| +2469198|AAAAAAAAOENKFCAA|2048-05-01|1780|7740|594|2048|5|5|1|2|2048|594|7740|Friday|2048Q2|N|Y|N|2469198|2469318|2468832|2469107|N|N|N|N|N| +2469199|AAAAAAAAPENKFCAA|2048-05-02|1780|7740|594|2048|6|5|2|2|2048|594|7740|Saturday|2048Q2|N|Y|N|2469198|2469318|2468833|2469108|N|N|N|N|N| +2469200|AAAAAAAAAFNKFCAA|2048-05-03|1780|7740|594|2048|0|5|3|2|2048|594|7740|Sunday|2048Q2|N|N|N|2469198|2469318|2468834|2469109|N|N|N|N|N| +2469201|AAAAAAAABFNKFCAA|2048-05-04|1780|7740|594|2048|1|5|4|2|2048|594|7740|Monday|2048Q2|N|N|N|2469198|2469318|2468835|2469110|N|N|N|N|N| +2469202|AAAAAAAACFNKFCAA|2048-05-05|1780|7741|594|2048|2|5|5|2|2048|594|7741|Tuesday|2048Q2|N|N|N|2469198|2469318|2468836|2469111|N|N|N|N|N| +2469203|AAAAAAAADFNKFCAA|2048-05-06|1780|7741|594|2048|3|5|6|2|2048|594|7741|Wednesday|2048Q2|N|N|N|2469198|2469318|2468837|2469112|N|N|N|N|N| +2469204|AAAAAAAAEFNKFCAA|2048-05-07|1780|7741|594|2048|4|5|7|2|2048|594|7741|Thursday|2048Q2|N|N|N|2469198|2469318|2468838|2469113|N|N|N|N|N| +2469205|AAAAAAAAFFNKFCAA|2048-05-08|1780|7741|594|2048|5|5|8|2|2048|594|7741|Friday|2048Q2|N|Y|N|2469198|2469318|2468839|2469114|N|N|N|N|N| +2469206|AAAAAAAAGFNKFCAA|2048-05-09|1780|7741|594|2048|6|5|9|2|2048|594|7741|Saturday|2048Q2|N|Y|N|2469198|2469318|2468840|2469115|N|N|N|N|N| +2469207|AAAAAAAAHFNKFCAA|2048-05-10|1780|7741|594|2048|0|5|10|2|2048|594|7741|Sunday|2048Q2|N|N|N|2469198|2469318|2468841|2469116|N|N|N|N|N| +2469208|AAAAAAAAIFNKFCAA|2048-05-11|1780|7741|594|2048|1|5|11|2|2048|594|7741|Monday|2048Q2|N|N|N|2469198|2469318|2468842|2469117|N|N|N|N|N| +2469209|AAAAAAAAJFNKFCAA|2048-05-12|1780|7742|594|2048|2|5|12|2|2048|594|7742|Tuesday|2048Q2|N|N|N|2469198|2469318|2468843|2469118|N|N|N|N|N| +2469210|AAAAAAAAKFNKFCAA|2048-05-13|1780|7742|594|2048|3|5|13|2|2048|594|7742|Wednesday|2048Q2|N|N|N|2469198|2469318|2468844|2469119|N|N|N|N|N| +2469211|AAAAAAAALFNKFCAA|2048-05-14|1780|7742|594|2048|4|5|14|2|2048|594|7742|Thursday|2048Q2|N|N|N|2469198|2469318|2468845|2469120|N|N|N|N|N| +2469212|AAAAAAAAMFNKFCAA|2048-05-15|1780|7742|594|2048|5|5|15|2|2048|594|7742|Friday|2048Q2|N|Y|N|2469198|2469318|2468846|2469121|N|N|N|N|N| +2469213|AAAAAAAANFNKFCAA|2048-05-16|1780|7742|594|2048|6|5|16|2|2048|594|7742|Saturday|2048Q2|N|Y|N|2469198|2469318|2468847|2469122|N|N|N|N|N| +2469214|AAAAAAAAOFNKFCAA|2048-05-17|1780|7742|594|2048|0|5|17|2|2048|594|7742|Sunday|2048Q2|N|N|N|2469198|2469318|2468848|2469123|N|N|N|N|N| +2469215|AAAAAAAAPFNKFCAA|2048-05-18|1780|7742|594|2048|1|5|18|2|2048|594|7742|Monday|2048Q2|N|N|N|2469198|2469318|2468849|2469124|N|N|N|N|N| +2469216|AAAAAAAAAGNKFCAA|2048-05-19|1780|7743|594|2048|2|5|19|2|2048|594|7743|Tuesday|2048Q2|N|N|N|2469198|2469318|2468850|2469125|N|N|N|N|N| +2469217|AAAAAAAABGNKFCAA|2048-05-20|1780|7743|594|2048|3|5|20|2|2048|594|7743|Wednesday|2048Q2|N|N|N|2469198|2469318|2468851|2469126|N|N|N|N|N| +2469218|AAAAAAAACGNKFCAA|2048-05-21|1780|7743|594|2048|4|5|21|2|2048|594|7743|Thursday|2048Q2|N|N|N|2469198|2469318|2468852|2469127|N|N|N|N|N| +2469219|AAAAAAAADGNKFCAA|2048-05-22|1780|7743|594|2048|5|5|22|2|2048|594|7743|Friday|2048Q2|N|Y|N|2469198|2469318|2468853|2469128|N|N|N|N|N| +2469220|AAAAAAAAEGNKFCAA|2048-05-23|1780|7743|594|2048|6|5|23|2|2048|594|7743|Saturday|2048Q2|N|Y|N|2469198|2469318|2468854|2469129|N|N|N|N|N| +2469221|AAAAAAAAFGNKFCAA|2048-05-24|1780|7743|594|2048|0|5|24|2|2048|594|7743|Sunday|2048Q2|N|N|N|2469198|2469318|2468855|2469130|N|N|N|N|N| +2469222|AAAAAAAAGGNKFCAA|2048-05-25|1780|7743|594|2048|1|5|25|2|2048|594|7743|Monday|2048Q2|N|N|N|2469198|2469318|2468856|2469131|N|N|N|N|N| +2469223|AAAAAAAAHGNKFCAA|2048-05-26|1780|7744|594|2048|2|5|26|2|2048|594|7744|Tuesday|2048Q2|N|N|N|2469198|2469318|2468857|2469132|N|N|N|N|N| +2469224|AAAAAAAAIGNKFCAA|2048-05-27|1780|7744|594|2048|3|5|27|2|2048|594|7744|Wednesday|2048Q2|N|N|N|2469198|2469318|2468858|2469133|N|N|N|N|N| +2469225|AAAAAAAAJGNKFCAA|2048-05-28|1780|7744|594|2048|4|5|28|2|2048|594|7744|Thursday|2048Q2|N|N|N|2469198|2469318|2468859|2469134|N|N|N|N|N| +2469226|AAAAAAAAKGNKFCAA|2048-05-29|1780|7744|594|2048|5|5|29|2|2048|594|7744|Friday|2048Q2|N|Y|N|2469198|2469318|2468860|2469135|N|N|N|N|N| +2469227|AAAAAAAALGNKFCAA|2048-05-30|1780|7744|594|2048|6|5|30|2|2048|594|7744|Saturday|2048Q2|N|Y|N|2469198|2469318|2468861|2469136|N|N|N|N|N| +2469228|AAAAAAAAMGNKFCAA|2048-05-31|1780|7744|594|2048|0|5|31|2|2048|594|7744|Sunday|2048Q2|N|N|N|2469198|2469318|2468862|2469137|N|N|N|N|N| +2469229|AAAAAAAANGNKFCAA|2048-06-01|1781|7744|595|2048|1|6|1|2|2048|595|7744|Monday|2048Q2|N|N|N|2469229|2469380|2468863|2469138|N|N|N|N|N| +2469230|AAAAAAAAOGNKFCAA|2048-06-02|1781|7745|595|2048|2|6|2|2|2048|595|7745|Tuesday|2048Q2|N|N|N|2469229|2469380|2468864|2469139|N|N|N|N|N| +2469231|AAAAAAAAPGNKFCAA|2048-06-03|1781|7745|595|2048|3|6|3|2|2048|595|7745|Wednesday|2048Q2|N|N|N|2469229|2469380|2468865|2469140|N|N|N|N|N| +2469232|AAAAAAAAAHNKFCAA|2048-06-04|1781|7745|595|2048|4|6|4|2|2048|595|7745|Thursday|2048Q2|N|N|N|2469229|2469380|2468866|2469141|N|N|N|N|N| +2469233|AAAAAAAABHNKFCAA|2048-06-05|1781|7745|595|2048|5|6|5|2|2048|595|7745|Friday|2048Q2|N|Y|N|2469229|2469380|2468867|2469142|N|N|N|N|N| +2469234|AAAAAAAACHNKFCAA|2048-06-06|1781|7745|595|2048|6|6|6|2|2048|595|7745|Saturday|2048Q2|N|Y|N|2469229|2469380|2468868|2469143|N|N|N|N|N| +2469235|AAAAAAAADHNKFCAA|2048-06-07|1781|7745|595|2048|0|6|7|2|2048|595|7745|Sunday|2048Q2|N|N|N|2469229|2469380|2468869|2469144|N|N|N|N|N| +2469236|AAAAAAAAEHNKFCAA|2048-06-08|1781|7745|595|2048|1|6|8|2|2048|595|7745|Monday|2048Q2|N|N|N|2469229|2469380|2468870|2469145|N|N|N|N|N| +2469237|AAAAAAAAFHNKFCAA|2048-06-09|1781|7746|595|2048|2|6|9|2|2048|595|7746|Tuesday|2048Q2|N|N|N|2469229|2469380|2468871|2469146|N|N|N|N|N| +2469238|AAAAAAAAGHNKFCAA|2048-06-10|1781|7746|595|2048|3|6|10|2|2048|595|7746|Wednesday|2048Q2|N|N|N|2469229|2469380|2468872|2469147|N|N|N|N|N| +2469239|AAAAAAAAHHNKFCAA|2048-06-11|1781|7746|595|2048|4|6|11|2|2048|595|7746|Thursday|2048Q2|N|N|N|2469229|2469380|2468873|2469148|N|N|N|N|N| +2469240|AAAAAAAAIHNKFCAA|2048-06-12|1781|7746|595|2048|5|6|12|2|2048|595|7746|Friday|2048Q2|N|Y|N|2469229|2469380|2468874|2469149|N|N|N|N|N| +2469241|AAAAAAAAJHNKFCAA|2048-06-13|1781|7746|595|2048|6|6|13|2|2048|595|7746|Saturday|2048Q2|N|Y|N|2469229|2469380|2468875|2469150|N|N|N|N|N| +2469242|AAAAAAAAKHNKFCAA|2048-06-14|1781|7746|595|2048|0|6|14|2|2048|595|7746|Sunday|2048Q2|N|N|N|2469229|2469380|2468876|2469151|N|N|N|N|N| +2469243|AAAAAAAALHNKFCAA|2048-06-15|1781|7746|595|2048|1|6|15|2|2048|595|7746|Monday|2048Q2|N|N|N|2469229|2469380|2468877|2469152|N|N|N|N|N| +2469244|AAAAAAAAMHNKFCAA|2048-06-16|1781|7747|595|2048|2|6|16|2|2048|595|7747|Tuesday|2048Q2|N|N|N|2469229|2469380|2468878|2469153|N|N|N|N|N| +2469245|AAAAAAAANHNKFCAA|2048-06-17|1781|7747|595|2048|3|6|17|2|2048|595|7747|Wednesday|2048Q2|N|N|N|2469229|2469380|2468879|2469154|N|N|N|N|N| +2469246|AAAAAAAAOHNKFCAA|2048-06-18|1781|7747|595|2048|4|6|18|2|2048|595|7747|Thursday|2048Q2|N|N|N|2469229|2469380|2468880|2469155|N|N|N|N|N| +2469247|AAAAAAAAPHNKFCAA|2048-06-19|1781|7747|595|2048|5|6|19|2|2048|595|7747|Friday|2048Q2|N|Y|N|2469229|2469380|2468881|2469156|N|N|N|N|N| +2469248|AAAAAAAAAINKFCAA|2048-06-20|1781|7747|595|2048|6|6|20|2|2048|595|7747|Saturday|2048Q2|N|Y|N|2469229|2469380|2468882|2469157|N|N|N|N|N| +2469249|AAAAAAAABINKFCAA|2048-06-21|1781|7747|595|2048|0|6|21|2|2048|595|7747|Sunday|2048Q2|N|N|N|2469229|2469380|2468883|2469158|N|N|N|N|N| +2469250|AAAAAAAACINKFCAA|2048-06-22|1781|7747|595|2048|1|6|22|2|2048|595|7747|Monday|2048Q2|N|N|N|2469229|2469380|2468884|2469159|N|N|N|N|N| +2469251|AAAAAAAADINKFCAA|2048-06-23|1781|7748|595|2048|2|6|23|2|2048|595|7748|Tuesday|2048Q2|N|N|N|2469229|2469380|2468885|2469160|N|N|N|N|N| +2469252|AAAAAAAAEINKFCAA|2048-06-24|1781|7748|595|2048|3|6|24|2|2048|595|7748|Wednesday|2048Q2|N|N|N|2469229|2469380|2468886|2469161|N|N|N|N|N| +2469253|AAAAAAAAFINKFCAA|2048-06-25|1781|7748|595|2048|4|6|25|2|2048|595|7748|Thursday|2048Q2|N|N|N|2469229|2469380|2468887|2469162|N|N|N|N|N| +2469254|AAAAAAAAGINKFCAA|2048-06-26|1781|7748|595|2048|5|6|26|2|2048|595|7748|Friday|2048Q2|N|Y|N|2469229|2469380|2468888|2469163|N|N|N|N|N| +2469255|AAAAAAAAHINKFCAA|2048-06-27|1781|7748|595|2048|6|6|27|2|2048|595|7748|Saturday|2048Q2|N|Y|N|2469229|2469380|2468889|2469164|N|N|N|N|N| +2469256|AAAAAAAAIINKFCAA|2048-06-28|1781|7748|595|2048|0|6|28|2|2048|595|7748|Sunday|2048Q2|N|N|N|2469229|2469380|2468890|2469165|N|N|N|N|N| +2469257|AAAAAAAAJINKFCAA|2048-06-29|1781|7748|595|2048|1|6|29|2|2048|595|7748|Monday|2048Q2|N|N|N|2469229|2469380|2468891|2469166|N|N|N|N|N| +2469258|AAAAAAAAKINKFCAA|2048-06-30|1781|7749|595|2048|2|6|30|2|2048|595|7749|Tuesday|2048Q2|N|N|N|2469229|2469380|2468892|2469167|N|N|N|N|N| +2469259|AAAAAAAALINKFCAA|2048-07-01|1782|7749|595|2048|3|7|1|3|2048|595|7749|Wednesday|2048Q3|N|N|N|2469259|2469440|2468893|2469168|N|N|N|N|N| +2469260|AAAAAAAAMINKFCAA|2048-07-02|1782|7749|595|2048|4|7|2|3|2048|595|7749|Thursday|2048Q3|N|N|N|2469259|2469440|2468894|2469169|N|N|N|N|N| +2469261|AAAAAAAANINKFCAA|2048-07-03|1782|7749|595|2048|5|7|3|3|2048|595|7749|Friday|2048Q3|N|Y|N|2469259|2469440|2468895|2469170|N|N|N|N|N| +2469262|AAAAAAAAOINKFCAA|2048-07-04|1782|7749|595|2048|6|7|4|3|2048|595|7749|Saturday|2048Q3|Y|Y|N|2469259|2469440|2468896|2469171|N|N|N|N|N| +2469263|AAAAAAAAPINKFCAA|2048-07-05|1782|7749|595|2048|0|7|5|3|2048|595|7749|Sunday|2048Q3|N|N|Y|2469259|2469440|2468897|2469172|N|N|N|N|N| +2469264|AAAAAAAAAJNKFCAA|2048-07-06|1782|7749|595|2048|1|7|6|3|2048|595|7749|Monday|2048Q3|N|N|N|2469259|2469440|2468898|2469173|N|N|N|N|N| +2469265|AAAAAAAABJNKFCAA|2048-07-07|1782|7750|595|2048|2|7|7|3|2048|595|7750|Tuesday|2048Q3|N|N|N|2469259|2469440|2468899|2469174|N|N|N|N|N| +2469266|AAAAAAAACJNKFCAA|2048-07-08|1782|7750|595|2048|3|7|8|3|2048|595|7750|Wednesday|2048Q3|N|N|N|2469259|2469440|2468900|2469175|N|N|N|N|N| +2469267|AAAAAAAADJNKFCAA|2048-07-09|1782|7750|595|2048|4|7|9|3|2048|595|7750|Thursday|2048Q3|N|N|N|2469259|2469440|2468901|2469176|N|N|N|N|N| +2469268|AAAAAAAAEJNKFCAA|2048-07-10|1782|7750|595|2048|5|7|10|3|2048|595|7750|Friday|2048Q3|N|Y|N|2469259|2469440|2468902|2469177|N|N|N|N|N| +2469269|AAAAAAAAFJNKFCAA|2048-07-11|1782|7750|595|2048|6|7|11|3|2048|595|7750|Saturday|2048Q3|N|Y|N|2469259|2469440|2468903|2469178|N|N|N|N|N| +2469270|AAAAAAAAGJNKFCAA|2048-07-12|1782|7750|595|2048|0|7|12|3|2048|595|7750|Sunday|2048Q3|N|N|N|2469259|2469440|2468904|2469179|N|N|N|N|N| +2469271|AAAAAAAAHJNKFCAA|2048-07-13|1782|7750|595|2048|1|7|13|3|2048|595|7750|Monday|2048Q3|N|N|N|2469259|2469440|2468905|2469180|N|N|N|N|N| +2469272|AAAAAAAAIJNKFCAA|2048-07-14|1782|7751|595|2048|2|7|14|3|2048|595|7751|Tuesday|2048Q3|N|N|N|2469259|2469440|2468906|2469181|N|N|N|N|N| +2469273|AAAAAAAAJJNKFCAA|2048-07-15|1782|7751|595|2048|3|7|15|3|2048|595|7751|Wednesday|2048Q3|N|N|N|2469259|2469440|2468907|2469182|N|N|N|N|N| +2469274|AAAAAAAAKJNKFCAA|2048-07-16|1782|7751|595|2048|4|7|16|3|2048|595|7751|Thursday|2048Q3|N|N|N|2469259|2469440|2468908|2469183|N|N|N|N|N| +2469275|AAAAAAAALJNKFCAA|2048-07-17|1782|7751|595|2048|5|7|17|3|2048|595|7751|Friday|2048Q3|N|Y|N|2469259|2469440|2468909|2469184|N|N|N|N|N| +2469276|AAAAAAAAMJNKFCAA|2048-07-18|1782|7751|595|2048|6|7|18|3|2048|595|7751|Saturday|2048Q3|N|Y|N|2469259|2469440|2468910|2469185|N|N|N|N|N| +2469277|AAAAAAAANJNKFCAA|2048-07-19|1782|7751|595|2048|0|7|19|3|2048|595|7751|Sunday|2048Q3|N|N|N|2469259|2469440|2468911|2469186|N|N|N|N|N| +2469278|AAAAAAAAOJNKFCAA|2048-07-20|1782|7751|595|2048|1|7|20|3|2048|595|7751|Monday|2048Q3|N|N|N|2469259|2469440|2468912|2469187|N|N|N|N|N| +2469279|AAAAAAAAPJNKFCAA|2048-07-21|1782|7752|595|2048|2|7|21|3|2048|595|7752|Tuesday|2048Q3|N|N|N|2469259|2469440|2468913|2469188|N|N|N|N|N| +2469280|AAAAAAAAAKNKFCAA|2048-07-22|1782|7752|595|2048|3|7|22|3|2048|595|7752|Wednesday|2048Q3|N|N|N|2469259|2469440|2468914|2469189|N|N|N|N|N| +2469281|AAAAAAAABKNKFCAA|2048-07-23|1782|7752|595|2048|4|7|23|3|2048|595|7752|Thursday|2048Q3|N|N|N|2469259|2469440|2468915|2469190|N|N|N|N|N| +2469282|AAAAAAAACKNKFCAA|2048-07-24|1782|7752|595|2048|5|7|24|3|2048|595|7752|Friday|2048Q3|N|Y|N|2469259|2469440|2468916|2469191|N|N|N|N|N| +2469283|AAAAAAAADKNKFCAA|2048-07-25|1782|7752|595|2048|6|7|25|3|2048|595|7752|Saturday|2048Q3|N|Y|N|2469259|2469440|2468917|2469192|N|N|N|N|N| +2469284|AAAAAAAAEKNKFCAA|2048-07-26|1782|7752|595|2048|0|7|26|3|2048|595|7752|Sunday|2048Q3|N|N|N|2469259|2469440|2468918|2469193|N|N|N|N|N| +2469285|AAAAAAAAFKNKFCAA|2048-07-27|1782|7752|595|2048|1|7|27|3|2048|595|7752|Monday|2048Q3|N|N|N|2469259|2469440|2468919|2469194|N|N|N|N|N| +2469286|AAAAAAAAGKNKFCAA|2048-07-28|1782|7753|595|2048|2|7|28|3|2048|595|7753|Tuesday|2048Q3|N|N|N|2469259|2469440|2468920|2469195|N|N|N|N|N| +2469287|AAAAAAAAHKNKFCAA|2048-07-29|1782|7753|595|2048|3|7|29|3|2048|595|7753|Wednesday|2048Q3|N|N|N|2469259|2469440|2468921|2469196|N|N|N|N|N| +2469288|AAAAAAAAIKNKFCAA|2048-07-30|1782|7753|595|2048|4|7|30|3|2048|595|7753|Thursday|2048Q3|N|N|N|2469259|2469440|2468922|2469197|N|N|N|N|N| +2469289|AAAAAAAAJKNKFCAA|2048-07-31|1782|7753|595|2048|5|7|31|3|2048|595|7753|Friday|2048Q3|N|Y|N|2469259|2469440|2468923|2469198|N|N|N|N|N| +2469290|AAAAAAAAKKNKFCAA|2048-08-01|1783|7753|595|2048|6|8|1|3|2048|595|7753|Saturday|2048Q3|N|Y|N|2469290|2469502|2468924|2469199|N|N|N|N|N| +2469291|AAAAAAAALKNKFCAA|2048-08-02|1783|7753|595|2048|0|8|2|3|2048|595|7753|Sunday|2048Q3|N|N|N|2469290|2469502|2468925|2469200|N|N|N|N|N| +2469292|AAAAAAAAMKNKFCAA|2048-08-03|1783|7753|595|2048|1|8|3|3|2048|595|7753|Monday|2048Q3|N|N|N|2469290|2469502|2468926|2469201|N|N|N|N|N| +2469293|AAAAAAAANKNKFCAA|2048-08-04|1783|7754|595|2048|2|8|4|3|2048|595|7754|Tuesday|2048Q3|N|N|N|2469290|2469502|2468927|2469202|N|N|N|N|N| +2469294|AAAAAAAAOKNKFCAA|2048-08-05|1783|7754|595|2048|3|8|5|3|2048|595|7754|Wednesday|2048Q3|N|N|N|2469290|2469502|2468928|2469203|N|N|N|N|N| +2469295|AAAAAAAAPKNKFCAA|2048-08-06|1783|7754|595|2048|4|8|6|3|2048|595|7754|Thursday|2048Q3|N|N|N|2469290|2469502|2468929|2469204|N|N|N|N|N| +2469296|AAAAAAAAALNKFCAA|2048-08-07|1783|7754|595|2048|5|8|7|3|2048|595|7754|Friday|2048Q3|N|Y|N|2469290|2469502|2468930|2469205|N|N|N|N|N| +2469297|AAAAAAAABLNKFCAA|2048-08-08|1783|7754|595|2048|6|8|8|3|2048|595|7754|Saturday|2048Q3|N|Y|N|2469290|2469502|2468931|2469206|N|N|N|N|N| +2469298|AAAAAAAACLNKFCAA|2048-08-09|1783|7754|595|2048|0|8|9|3|2048|595|7754|Sunday|2048Q3|N|N|N|2469290|2469502|2468932|2469207|N|N|N|N|N| +2469299|AAAAAAAADLNKFCAA|2048-08-10|1783|7754|595|2048|1|8|10|3|2048|595|7754|Monday|2048Q3|N|N|N|2469290|2469502|2468933|2469208|N|N|N|N|N| +2469300|AAAAAAAAELNKFCAA|2048-08-11|1783|7755|595|2048|2|8|11|3|2048|595|7755|Tuesday|2048Q3|N|N|N|2469290|2469502|2468934|2469209|N|N|N|N|N| +2469301|AAAAAAAAFLNKFCAA|2048-08-12|1783|7755|595|2048|3|8|12|3|2048|595|7755|Wednesday|2048Q3|N|N|N|2469290|2469502|2468935|2469210|N|N|N|N|N| +2469302|AAAAAAAAGLNKFCAA|2048-08-13|1783|7755|595|2048|4|8|13|3|2048|595|7755|Thursday|2048Q3|N|N|N|2469290|2469502|2468936|2469211|N|N|N|N|N| +2469303|AAAAAAAAHLNKFCAA|2048-08-14|1783|7755|595|2048|5|8|14|3|2048|595|7755|Friday|2048Q3|N|Y|N|2469290|2469502|2468937|2469212|N|N|N|N|N| +2469304|AAAAAAAAILNKFCAA|2048-08-15|1783|7755|595|2048|6|8|15|3|2048|595|7755|Saturday|2048Q3|N|Y|N|2469290|2469502|2468938|2469213|N|N|N|N|N| +2469305|AAAAAAAAJLNKFCAA|2048-08-16|1783|7755|595|2048|0|8|16|3|2048|595|7755|Sunday|2048Q3|N|N|N|2469290|2469502|2468939|2469214|N|N|N|N|N| +2469306|AAAAAAAAKLNKFCAA|2048-08-17|1783|7755|595|2048|1|8|17|3|2048|595|7755|Monday|2048Q3|N|N|N|2469290|2469502|2468940|2469215|N|N|N|N|N| +2469307|AAAAAAAALLNKFCAA|2048-08-18|1783|7756|595|2048|2|8|18|3|2048|595|7756|Tuesday|2048Q3|N|N|N|2469290|2469502|2468941|2469216|N|N|N|N|N| +2469308|AAAAAAAAMLNKFCAA|2048-08-19|1783|7756|595|2048|3|8|19|3|2048|595|7756|Wednesday|2048Q3|N|N|N|2469290|2469502|2468942|2469217|N|N|N|N|N| +2469309|AAAAAAAANLNKFCAA|2048-08-20|1783|7756|595|2048|4|8|20|3|2048|595|7756|Thursday|2048Q3|N|N|N|2469290|2469502|2468943|2469218|N|N|N|N|N| +2469310|AAAAAAAAOLNKFCAA|2048-08-21|1783|7756|595|2048|5|8|21|3|2048|595|7756|Friday|2048Q3|N|Y|N|2469290|2469502|2468944|2469219|N|N|N|N|N| +2469311|AAAAAAAAPLNKFCAA|2048-08-22|1783|7756|595|2048|6|8|22|3|2048|595|7756|Saturday|2048Q3|N|Y|N|2469290|2469502|2468945|2469220|N|N|N|N|N| +2469312|AAAAAAAAAMNKFCAA|2048-08-23|1783|7756|595|2048|0|8|23|3|2048|595|7756|Sunday|2048Q3|N|N|N|2469290|2469502|2468946|2469221|N|N|N|N|N| +2469313|AAAAAAAABMNKFCAA|2048-08-24|1783|7756|595|2048|1|8|24|3|2048|595|7756|Monday|2048Q3|N|N|N|2469290|2469502|2468947|2469222|N|N|N|N|N| +2469314|AAAAAAAACMNKFCAA|2048-08-25|1783|7757|595|2048|2|8|25|3|2048|595|7757|Tuesday|2048Q3|N|N|N|2469290|2469502|2468948|2469223|N|N|N|N|N| +2469315|AAAAAAAADMNKFCAA|2048-08-26|1783|7757|595|2048|3|8|26|3|2048|595|7757|Wednesday|2048Q3|N|N|N|2469290|2469502|2468949|2469224|N|N|N|N|N| +2469316|AAAAAAAAEMNKFCAA|2048-08-27|1783|7757|595|2048|4|8|27|3|2048|595|7757|Thursday|2048Q3|N|N|N|2469290|2469502|2468950|2469225|N|N|N|N|N| +2469317|AAAAAAAAFMNKFCAA|2048-08-28|1783|7757|595|2048|5|8|28|3|2048|595|7757|Friday|2048Q3|N|Y|N|2469290|2469502|2468951|2469226|N|N|N|N|N| +2469318|AAAAAAAAGMNKFCAA|2048-08-29|1783|7757|595|2048|6|8|29|3|2048|595|7757|Saturday|2048Q3|N|Y|N|2469290|2469502|2468952|2469227|N|N|N|N|N| +2469319|AAAAAAAAHMNKFCAA|2048-08-30|1783|7757|595|2048|0|8|30|3|2048|595|7757|Sunday|2048Q3|N|N|N|2469290|2469502|2468953|2469228|N|N|N|N|N| +2469320|AAAAAAAAIMNKFCAA|2048-08-31|1783|7757|595|2048|1|8|31|3|2048|595|7757|Monday|2048Q3|N|N|N|2469290|2469502|2468954|2469229|N|N|N|N|N| +2469321|AAAAAAAAJMNKFCAA|2048-09-01|1784|7758|596|2048|2|9|1|3|2048|596|7758|Tuesday|2048Q3|N|N|N|2469321|2469564|2468955|2469230|N|N|N|N|N| +2469322|AAAAAAAAKMNKFCAA|2048-09-02|1784|7758|596|2048|3|9|2|3|2048|596|7758|Wednesday|2048Q3|N|N|N|2469321|2469564|2468956|2469231|N|N|N|N|N| +2469323|AAAAAAAALMNKFCAA|2048-09-03|1784|7758|596|2048|4|9|3|3|2048|596|7758|Thursday|2048Q3|N|N|N|2469321|2469564|2468957|2469232|N|N|N|N|N| +2469324|AAAAAAAAMMNKFCAA|2048-09-04|1784|7758|596|2048|5|9|4|3|2048|596|7758|Friday|2048Q3|N|Y|N|2469321|2469564|2468958|2469233|N|N|N|N|N| +2469325|AAAAAAAANMNKFCAA|2048-09-05|1784|7758|596|2048|6|9|5|3|2048|596|7758|Saturday|2048Q3|N|Y|N|2469321|2469564|2468959|2469234|N|N|N|N|N| +2469326|AAAAAAAAOMNKFCAA|2048-09-06|1784|7758|596|2048|0|9|6|3|2048|596|7758|Sunday|2048Q3|N|N|N|2469321|2469564|2468960|2469235|N|N|N|N|N| +2469327|AAAAAAAAPMNKFCAA|2048-09-07|1784|7758|596|2048|1|9|7|3|2048|596|7758|Monday|2048Q3|N|N|N|2469321|2469564|2468961|2469236|N|N|N|N|N| +2469328|AAAAAAAAANNKFCAA|2048-09-08|1784|7759|596|2048|2|9|8|3|2048|596|7759|Tuesday|2048Q3|N|N|N|2469321|2469564|2468962|2469237|N|N|N|N|N| +2469329|AAAAAAAABNNKFCAA|2048-09-09|1784|7759|596|2048|3|9|9|3|2048|596|7759|Wednesday|2048Q3|N|N|N|2469321|2469564|2468963|2469238|N|N|N|N|N| +2469330|AAAAAAAACNNKFCAA|2048-09-10|1784|7759|596|2048|4|9|10|3|2048|596|7759|Thursday|2048Q3|N|N|N|2469321|2469564|2468964|2469239|N|N|N|N|N| +2469331|AAAAAAAADNNKFCAA|2048-09-11|1784|7759|596|2048|5|9|11|3|2048|596|7759|Friday|2048Q3|N|Y|N|2469321|2469564|2468965|2469240|N|N|N|N|N| +2469332|AAAAAAAAENNKFCAA|2048-09-12|1784|7759|596|2048|6|9|12|3|2048|596|7759|Saturday|2048Q3|N|Y|N|2469321|2469564|2468966|2469241|N|N|N|N|N| +2469333|AAAAAAAAFNNKFCAA|2048-09-13|1784|7759|596|2048|0|9|13|3|2048|596|7759|Sunday|2048Q3|N|N|N|2469321|2469564|2468967|2469242|N|N|N|N|N| +2469334|AAAAAAAAGNNKFCAA|2048-09-14|1784|7759|596|2048|1|9|14|3|2048|596|7759|Monday|2048Q3|N|N|N|2469321|2469564|2468968|2469243|N|N|N|N|N| +2469335|AAAAAAAAHNNKFCAA|2048-09-15|1784|7760|596|2048|2|9|15|3|2048|596|7760|Tuesday|2048Q3|N|N|N|2469321|2469564|2468969|2469244|N|N|N|N|N| +2469336|AAAAAAAAINNKFCAA|2048-09-16|1784|7760|596|2048|3|9|16|3|2048|596|7760|Wednesday|2048Q3|N|N|N|2469321|2469564|2468970|2469245|N|N|N|N|N| +2469337|AAAAAAAAJNNKFCAA|2048-09-17|1784|7760|596|2048|4|9|17|3|2048|596|7760|Thursday|2048Q3|N|N|N|2469321|2469564|2468971|2469246|N|N|N|N|N| +2469338|AAAAAAAAKNNKFCAA|2048-09-18|1784|7760|596|2048|5|9|18|3|2048|596|7760|Friday|2048Q3|N|Y|N|2469321|2469564|2468972|2469247|N|N|N|N|N| +2469339|AAAAAAAALNNKFCAA|2048-09-19|1784|7760|596|2048|6|9|19|3|2048|596|7760|Saturday|2048Q3|N|Y|N|2469321|2469564|2468973|2469248|N|N|N|N|N| +2469340|AAAAAAAAMNNKFCAA|2048-09-20|1784|7760|596|2048|0|9|20|3|2048|596|7760|Sunday|2048Q3|N|N|N|2469321|2469564|2468974|2469249|N|N|N|N|N| +2469341|AAAAAAAANNNKFCAA|2048-09-21|1784|7760|596|2048|1|9|21|3|2048|596|7760|Monday|2048Q3|N|N|N|2469321|2469564|2468975|2469250|N|N|N|N|N| +2469342|AAAAAAAAONNKFCAA|2048-09-22|1784|7761|596|2048|2|9|22|3|2048|596|7761|Tuesday|2048Q3|N|N|N|2469321|2469564|2468976|2469251|N|N|N|N|N| +2469343|AAAAAAAAPNNKFCAA|2048-09-23|1784|7761|596|2048|3|9|23|3|2048|596|7761|Wednesday|2048Q3|N|N|N|2469321|2469564|2468977|2469252|N|N|N|N|N| +2469344|AAAAAAAAAONKFCAA|2048-09-24|1784|7761|596|2048|4|9|24|3|2048|596|7761|Thursday|2048Q3|N|N|N|2469321|2469564|2468978|2469253|N|N|N|N|N| +2469345|AAAAAAAABONKFCAA|2048-09-25|1784|7761|596|2048|5|9|25|3|2048|596|7761|Friday|2048Q3|N|Y|N|2469321|2469564|2468979|2469254|N|N|N|N|N| +2469346|AAAAAAAACONKFCAA|2048-09-26|1784|7761|596|2048|6|9|26|3|2048|596|7761|Saturday|2048Q3|N|Y|N|2469321|2469564|2468980|2469255|N|N|N|N|N| +2469347|AAAAAAAADONKFCAA|2048-09-27|1784|7761|596|2048|0|9|27|3|2048|596|7761|Sunday|2048Q3|N|N|N|2469321|2469564|2468981|2469256|N|N|N|N|N| +2469348|AAAAAAAAEONKFCAA|2048-09-28|1784|7761|596|2048|1|9|28|3|2048|596|7761|Monday|2048Q3|N|N|N|2469321|2469564|2468982|2469257|N|N|N|N|N| +2469349|AAAAAAAAFONKFCAA|2048-09-29|1784|7762|596|2048|2|9|29|3|2048|596|7762|Tuesday|2048Q3|N|N|N|2469321|2469564|2468983|2469258|N|N|N|N|N| +2469350|AAAAAAAAGONKFCAA|2048-09-30|1784|7762|596|2048|3|9|30|3|2048|596|7762|Wednesday|2048Q3|N|N|N|2469321|2469564|2468984|2469259|N|N|N|N|N| +2469351|AAAAAAAAHONKFCAA|2048-10-01|1785|7762|596|2048|4|10|1|4|2048|596|7762|Thursday|2048Q4|N|N|N|2469351|2469624|2468985|2469259|N|N|N|N|N| +2469352|AAAAAAAAIONKFCAA|2048-10-02|1785|7762|596|2048|5|10|2|4|2048|596|7762|Friday|2048Q4|N|Y|N|2469351|2469624|2468986|2469260|N|N|N|N|N| +2469353|AAAAAAAAJONKFCAA|2048-10-03|1785|7762|596|2048|6|10|3|4|2048|596|7762|Saturday|2048Q4|N|Y|N|2469351|2469624|2468987|2469261|N|N|N|N|N| +2469354|AAAAAAAAKONKFCAA|2048-10-04|1785|7762|596|2048|0|10|4|4|2048|596|7762|Sunday|2048Q4|N|N|N|2469351|2469624|2468988|2469262|N|N|N|N|N| +2469355|AAAAAAAALONKFCAA|2048-10-05|1785|7762|596|2048|1|10|5|4|2048|596|7762|Monday|2048Q4|N|N|N|2469351|2469624|2468989|2469263|N|N|N|N|N| +2469356|AAAAAAAAMONKFCAA|2048-10-06|1785|7763|596|2048|2|10|6|4|2048|596|7763|Tuesday|2048Q4|N|N|N|2469351|2469624|2468990|2469264|N|N|N|N|N| +2469357|AAAAAAAANONKFCAA|2048-10-07|1785|7763|596|2048|3|10|7|4|2048|596|7763|Wednesday|2048Q4|N|N|N|2469351|2469624|2468991|2469265|N|N|N|N|N| +2469358|AAAAAAAAOONKFCAA|2048-10-08|1785|7763|596|2048|4|10|8|4|2048|596|7763|Thursday|2048Q4|N|N|N|2469351|2469624|2468992|2469266|N|N|N|N|N| +2469359|AAAAAAAAPONKFCAA|2048-10-09|1785|7763|596|2048|5|10|9|4|2048|596|7763|Friday|2048Q4|N|Y|N|2469351|2469624|2468993|2469267|N|N|N|N|N| +2469360|AAAAAAAAAPNKFCAA|2048-10-10|1785|7763|596|2048|6|10|10|4|2048|596|7763|Saturday|2048Q4|N|Y|N|2469351|2469624|2468994|2469268|N|N|N|N|N| +2469361|AAAAAAAABPNKFCAA|2048-10-11|1785|7763|596|2048|0|10|11|4|2048|596|7763|Sunday|2048Q4|N|N|N|2469351|2469624|2468995|2469269|N|N|N|N|N| +2469362|AAAAAAAACPNKFCAA|2048-10-12|1785|7763|596|2048|1|10|12|4|2048|596|7763|Monday|2048Q4|N|N|N|2469351|2469624|2468996|2469270|N|N|N|N|N| +2469363|AAAAAAAADPNKFCAA|2048-10-13|1785|7764|596|2048|2|10|13|4|2048|596|7764|Tuesday|2048Q4|N|N|N|2469351|2469624|2468997|2469271|N|N|N|N|N| +2469364|AAAAAAAAEPNKFCAA|2048-10-14|1785|7764|596|2048|3|10|14|4|2048|596|7764|Wednesday|2048Q4|N|N|N|2469351|2469624|2468998|2469272|N|N|N|N|N| +2469365|AAAAAAAAFPNKFCAA|2048-10-15|1785|7764|596|2048|4|10|15|4|2048|596|7764|Thursday|2048Q4|N|N|N|2469351|2469624|2468999|2469273|N|N|N|N|N| +2469366|AAAAAAAAGPNKFCAA|2048-10-16|1785|7764|596|2048|5|10|16|4|2048|596|7764|Friday|2048Q4|N|Y|N|2469351|2469624|2469000|2469274|N|N|N|N|N| +2469367|AAAAAAAAHPNKFCAA|2048-10-17|1785|7764|596|2048|6|10|17|4|2048|596|7764|Saturday|2048Q4|N|Y|N|2469351|2469624|2469001|2469275|N|N|N|N|N| +2469368|AAAAAAAAIPNKFCAA|2048-10-18|1785|7764|596|2048|0|10|18|4|2048|596|7764|Sunday|2048Q4|N|N|N|2469351|2469624|2469002|2469276|N|N|N|N|N| +2469369|AAAAAAAAJPNKFCAA|2048-10-19|1785|7764|596|2048|1|10|19|4|2048|596|7764|Monday|2048Q4|N|N|N|2469351|2469624|2469003|2469277|N|N|N|N|N| +2469370|AAAAAAAAKPNKFCAA|2048-10-20|1785|7765|596|2048|2|10|20|4|2048|596|7765|Tuesday|2048Q4|N|N|N|2469351|2469624|2469004|2469278|N|N|N|N|N| +2469371|AAAAAAAALPNKFCAA|2048-10-21|1785|7765|596|2048|3|10|21|4|2048|596|7765|Wednesday|2048Q4|N|N|N|2469351|2469624|2469005|2469279|N|N|N|N|N| +2469372|AAAAAAAAMPNKFCAA|2048-10-22|1785|7765|596|2048|4|10|22|4|2048|596|7765|Thursday|2048Q4|N|N|N|2469351|2469624|2469006|2469280|N|N|N|N|N| +2469373|AAAAAAAANPNKFCAA|2048-10-23|1785|7765|596|2048|5|10|23|4|2048|596|7765|Friday|2048Q4|N|Y|N|2469351|2469624|2469007|2469281|N|N|N|N|N| +2469374|AAAAAAAAOPNKFCAA|2048-10-24|1785|7765|596|2048|6|10|24|4|2048|596|7765|Saturday|2048Q4|N|Y|N|2469351|2469624|2469008|2469282|N|N|N|N|N| +2469375|AAAAAAAAPPNKFCAA|2048-10-25|1785|7765|596|2048|0|10|25|4|2048|596|7765|Sunday|2048Q4|N|N|N|2469351|2469624|2469009|2469283|N|N|N|N|N| +2469376|AAAAAAAAAAOKFCAA|2048-10-26|1785|7765|596|2048|1|10|26|4|2048|596|7765|Monday|2048Q4|N|N|N|2469351|2469624|2469010|2469284|N|N|N|N|N| +2469377|AAAAAAAABAOKFCAA|2048-10-27|1785|7766|596|2048|2|10|27|4|2048|596|7766|Tuesday|2048Q4|N|N|N|2469351|2469624|2469011|2469285|N|N|N|N|N| +2469378|AAAAAAAACAOKFCAA|2048-10-28|1785|7766|596|2048|3|10|28|4|2048|596|7766|Wednesday|2048Q4|N|N|N|2469351|2469624|2469012|2469286|N|N|N|N|N| +2469379|AAAAAAAADAOKFCAA|2048-10-29|1785|7766|596|2048|4|10|29|4|2048|596|7766|Thursday|2048Q4|N|N|N|2469351|2469624|2469013|2469287|N|N|N|N|N| +2469380|AAAAAAAAEAOKFCAA|2048-10-30|1785|7766|596|2048|5|10|30|4|2048|596|7766|Friday|2048Q4|N|Y|N|2469351|2469624|2469014|2469288|N|N|N|N|N| +2469381|AAAAAAAAFAOKFCAA|2048-10-31|1785|7766|596|2048|6|10|31|4|2048|596|7766|Saturday|2048Q4|N|Y|N|2469351|2469624|2469015|2469289|N|N|N|N|N| +2469382|AAAAAAAAGAOKFCAA|2048-11-01|1786|7766|596|2048|0|11|1|4|2048|596|7766|Sunday|2048Q4|N|N|N|2469382|2469686|2469016|2469290|N|N|N|N|N| +2469383|AAAAAAAAHAOKFCAA|2048-11-02|1786|7766|596|2048|1|11|2|4|2048|596|7766|Monday|2048Q4|N|N|N|2469382|2469686|2469017|2469291|N|N|N|N|N| +2469384|AAAAAAAAIAOKFCAA|2048-11-03|1786|7767|596|2048|2|11|3|4|2048|596|7767|Tuesday|2048Q4|N|N|N|2469382|2469686|2469018|2469292|N|N|N|N|N| +2469385|AAAAAAAAJAOKFCAA|2048-11-04|1786|7767|596|2048|3|11|4|4|2048|596|7767|Wednesday|2048Q4|N|N|N|2469382|2469686|2469019|2469293|N|N|N|N|N| +2469386|AAAAAAAAKAOKFCAA|2048-11-05|1786|7767|596|2048|4|11|5|4|2048|596|7767|Thursday|2048Q4|N|N|N|2469382|2469686|2469020|2469294|N|N|N|N|N| +2469387|AAAAAAAALAOKFCAA|2048-11-06|1786|7767|596|2048|5|11|6|4|2048|596|7767|Friday|2048Q4|N|Y|N|2469382|2469686|2469021|2469295|N|N|N|N|N| +2469388|AAAAAAAAMAOKFCAA|2048-11-07|1786|7767|596|2048|6|11|7|4|2048|596|7767|Saturday|2048Q4|N|Y|N|2469382|2469686|2469022|2469296|N|N|N|N|N| +2469389|AAAAAAAANAOKFCAA|2048-11-08|1786|7767|596|2048|0|11|8|4|2048|596|7767|Sunday|2048Q4|N|N|N|2469382|2469686|2469023|2469297|N|N|N|N|N| +2469390|AAAAAAAAOAOKFCAA|2048-11-09|1786|7767|596|2048|1|11|9|4|2048|596|7767|Monday|2048Q4|N|N|N|2469382|2469686|2469024|2469298|N|N|N|N|N| +2469391|AAAAAAAAPAOKFCAA|2048-11-10|1786|7768|596|2048|2|11|10|4|2048|596|7768|Tuesday|2048Q4|N|N|N|2469382|2469686|2469025|2469299|N|N|N|N|N| +2469392|AAAAAAAAABOKFCAA|2048-11-11|1786|7768|596|2048|3|11|11|4|2048|596|7768|Wednesday|2048Q4|N|N|N|2469382|2469686|2469026|2469300|N|N|N|N|N| +2469393|AAAAAAAABBOKFCAA|2048-11-12|1786|7768|596|2048|4|11|12|4|2048|596|7768|Thursday|2048Q4|N|N|N|2469382|2469686|2469027|2469301|N|N|N|N|N| +2469394|AAAAAAAACBOKFCAA|2048-11-13|1786|7768|596|2048|5|11|13|4|2048|596|7768|Friday|2048Q4|N|Y|N|2469382|2469686|2469028|2469302|N|N|N|N|N| +2469395|AAAAAAAADBOKFCAA|2048-11-14|1786|7768|596|2048|6|11|14|4|2048|596|7768|Saturday|2048Q4|N|Y|N|2469382|2469686|2469029|2469303|N|N|N|N|N| +2469396|AAAAAAAAEBOKFCAA|2048-11-15|1786|7768|596|2048|0|11|15|4|2048|596|7768|Sunday|2048Q4|N|N|N|2469382|2469686|2469030|2469304|N|N|N|N|N| +2469397|AAAAAAAAFBOKFCAA|2048-11-16|1786|7768|596|2048|1|11|16|4|2048|596|7768|Monday|2048Q4|N|N|N|2469382|2469686|2469031|2469305|N|N|N|N|N| +2469398|AAAAAAAAGBOKFCAA|2048-11-17|1786|7769|596|2048|2|11|17|4|2048|596|7769|Tuesday|2048Q4|N|N|N|2469382|2469686|2469032|2469306|N|N|N|N|N| +2469399|AAAAAAAAHBOKFCAA|2048-11-18|1786|7769|596|2048|3|11|18|4|2048|596|7769|Wednesday|2048Q4|N|N|N|2469382|2469686|2469033|2469307|N|N|N|N|N| +2469400|AAAAAAAAIBOKFCAA|2048-11-19|1786|7769|596|2048|4|11|19|4|2048|596|7769|Thursday|2048Q4|N|N|N|2469382|2469686|2469034|2469308|N|N|N|N|N| +2469401|AAAAAAAAJBOKFCAA|2048-11-20|1786|7769|596|2048|5|11|20|4|2048|596|7769|Friday|2048Q4|N|Y|N|2469382|2469686|2469035|2469309|N|N|N|N|N| +2469402|AAAAAAAAKBOKFCAA|2048-11-21|1786|7769|596|2048|6|11|21|4|2048|596|7769|Saturday|2048Q4|N|Y|N|2469382|2469686|2469036|2469310|N|N|N|N|N| +2469403|AAAAAAAALBOKFCAA|2048-11-22|1786|7769|596|2048|0|11|22|4|2048|596|7769|Sunday|2048Q4|N|N|N|2469382|2469686|2469037|2469311|N|N|N|N|N| +2469404|AAAAAAAAMBOKFCAA|2048-11-23|1786|7769|596|2048|1|11|23|4|2048|596|7769|Monday|2048Q4|N|N|N|2469382|2469686|2469038|2469312|N|N|N|N|N| +2469405|AAAAAAAANBOKFCAA|2048-11-24|1786|7770|596|2048|2|11|24|4|2048|596|7770|Tuesday|2048Q4|N|N|N|2469382|2469686|2469039|2469313|N|N|N|N|N| +2469406|AAAAAAAAOBOKFCAA|2048-11-25|1786|7770|596|2048|3|11|25|4|2048|596|7770|Wednesday|2048Q4|N|N|N|2469382|2469686|2469040|2469314|N|N|N|N|N| +2469407|AAAAAAAAPBOKFCAA|2048-11-26|1786|7770|596|2048|4|11|26|4|2048|596|7770|Thursday|2048Q4|N|N|N|2469382|2469686|2469041|2469315|N|N|N|N|N| +2469408|AAAAAAAAACOKFCAA|2048-11-27|1786|7770|596|2048|5|11|27|4|2048|596|7770|Friday|2048Q4|N|Y|N|2469382|2469686|2469042|2469316|N|N|N|N|N| +2469409|AAAAAAAABCOKFCAA|2048-11-28|1786|7770|596|2048|6|11|28|4|2048|596|7770|Saturday|2048Q4|N|Y|N|2469382|2469686|2469043|2469317|N|N|N|N|N| +2469410|AAAAAAAACCOKFCAA|2048-11-29|1786|7770|596|2048|0|11|29|4|2048|596|7770|Sunday|2048Q4|N|N|N|2469382|2469686|2469044|2469318|N|N|N|N|N| +2469411|AAAAAAAADCOKFCAA|2048-11-30|1786|7770|596|2048|1|11|30|4|2048|596|7770|Monday|2048Q4|N|N|N|2469382|2469686|2469045|2469319|N|N|N|N|N| +2469412|AAAAAAAAECOKFCAA|2048-12-01|1787|7771|597|2048|2|12|1|4|2048|597|7771|Tuesday|2048Q4|N|N|N|2469412|2469746|2469046|2469320|N|N|N|N|N| +2469413|AAAAAAAAFCOKFCAA|2048-12-02|1787|7771|597|2048|3|12|2|4|2048|597|7771|Wednesday|2048Q4|N|N|N|2469412|2469746|2469047|2469321|N|N|N|N|N| +2469414|AAAAAAAAGCOKFCAA|2048-12-03|1787|7771|597|2048|4|12|3|4|2048|597|7771|Thursday|2048Q4|N|N|N|2469412|2469746|2469048|2469322|N|N|N|N|N| +2469415|AAAAAAAAHCOKFCAA|2048-12-04|1787|7771|597|2048|5|12|4|4|2048|597|7771|Friday|2048Q4|N|Y|N|2469412|2469746|2469049|2469323|N|N|N|N|N| +2469416|AAAAAAAAICOKFCAA|2048-12-05|1787|7771|597|2048|6|12|5|4|2048|597|7771|Saturday|2048Q4|N|Y|N|2469412|2469746|2469050|2469324|N|N|N|N|N| +2469417|AAAAAAAAJCOKFCAA|2048-12-06|1787|7771|597|2048|0|12|6|4|2048|597|7771|Sunday|2048Q4|N|N|N|2469412|2469746|2469051|2469325|N|N|N|N|N| +2469418|AAAAAAAAKCOKFCAA|2048-12-07|1787|7771|597|2048|1|12|7|4|2048|597|7771|Monday|2048Q4|N|N|N|2469412|2469746|2469052|2469326|N|N|N|N|N| +2469419|AAAAAAAALCOKFCAA|2048-12-08|1787|7772|597|2048|2|12|8|4|2048|597|7772|Tuesday|2048Q4|N|N|N|2469412|2469746|2469053|2469327|N|N|N|N|N| +2469420|AAAAAAAAMCOKFCAA|2048-12-09|1787|7772|597|2048|3|12|9|4|2048|597|7772|Wednesday|2048Q4|N|N|N|2469412|2469746|2469054|2469328|N|N|N|N|N| +2469421|AAAAAAAANCOKFCAA|2048-12-10|1787|7772|597|2048|4|12|10|4|2048|597|7772|Thursday|2048Q4|N|N|N|2469412|2469746|2469055|2469329|N|N|N|N|N| +2469422|AAAAAAAAOCOKFCAA|2048-12-11|1787|7772|597|2048|5|12|11|4|2048|597|7772|Friday|2048Q4|N|Y|N|2469412|2469746|2469056|2469330|N|N|N|N|N| +2469423|AAAAAAAAPCOKFCAA|2048-12-12|1787|7772|597|2048|6|12|12|4|2048|597|7772|Saturday|2048Q4|N|Y|N|2469412|2469746|2469057|2469331|N|N|N|N|N| +2469424|AAAAAAAAADOKFCAA|2048-12-13|1787|7772|597|2048|0|12|13|4|2048|597|7772|Sunday|2048Q4|N|N|N|2469412|2469746|2469058|2469332|N|N|N|N|N| +2469425|AAAAAAAABDOKFCAA|2048-12-14|1787|7772|597|2048|1|12|14|4|2048|597|7772|Monday|2048Q4|N|N|N|2469412|2469746|2469059|2469333|N|N|N|N|N| +2469426|AAAAAAAACDOKFCAA|2048-12-15|1787|7773|597|2048|2|12|15|4|2048|597|7773|Tuesday|2048Q4|N|N|N|2469412|2469746|2469060|2469334|N|N|N|N|N| +2469427|AAAAAAAADDOKFCAA|2048-12-16|1787|7773|597|2048|3|12|16|4|2048|597|7773|Wednesday|2048Q4|N|N|N|2469412|2469746|2469061|2469335|N|N|N|N|N| +2469428|AAAAAAAAEDOKFCAA|2048-12-17|1787|7773|597|2048|4|12|17|4|2048|597|7773|Thursday|2048Q4|N|N|N|2469412|2469746|2469062|2469336|N|N|N|N|N| +2469429|AAAAAAAAFDOKFCAA|2048-12-18|1787|7773|597|2048|5|12|18|4|2048|597|7773|Friday|2048Q4|N|Y|N|2469412|2469746|2469063|2469337|N|N|N|N|N| +2469430|AAAAAAAAGDOKFCAA|2048-12-19|1787|7773|597|2048|6|12|19|4|2048|597|7773|Saturday|2048Q4|N|Y|N|2469412|2469746|2469064|2469338|N|N|N|N|N| +2469431|AAAAAAAAHDOKFCAA|2048-12-20|1787|7773|597|2048|0|12|20|4|2048|597|7773|Sunday|2048Q4|N|N|N|2469412|2469746|2469065|2469339|N|N|N|N|N| +2469432|AAAAAAAAIDOKFCAA|2048-12-21|1787|7773|597|2048|1|12|21|4|2048|597|7773|Monday|2048Q4|N|N|N|2469412|2469746|2469066|2469340|N|N|N|N|N| +2469433|AAAAAAAAJDOKFCAA|2048-12-22|1787|7774|597|2048|2|12|22|4|2048|597|7774|Tuesday|2048Q4|N|N|N|2469412|2469746|2469067|2469341|N|N|N|N|N| +2469434|AAAAAAAAKDOKFCAA|2048-12-23|1787|7774|597|2048|3|12|23|4|2048|597|7774|Wednesday|2048Q4|N|N|N|2469412|2469746|2469068|2469342|N|N|N|N|N| +2469435|AAAAAAAALDOKFCAA|2048-12-24|1787|7774|597|2048|4|12|24|4|2048|597|7774|Thursday|2048Q4|N|N|N|2469412|2469746|2469069|2469343|N|N|N|N|N| +2469436|AAAAAAAAMDOKFCAA|2048-12-25|1787|7774|597|2048|5|12|25|4|2048|597|7774|Friday|2048Q4|Y|Y|N|2469412|2469746|2469070|2469344|N|N|N|N|N| +2469437|AAAAAAAANDOKFCAA|2048-12-26|1787|7774|597|2048|6|12|26|4|2048|597|7774|Saturday|2048Q4|N|Y|Y|2469412|2469746|2469071|2469345|N|N|N|N|N| +2469438|AAAAAAAAODOKFCAA|2048-12-27|1787|7774|597|2048|0|12|27|4|2048|597|7774|Sunday|2048Q4|N|N|N|2469412|2469746|2469072|2469346|N|N|N|N|N| +2469439|AAAAAAAAPDOKFCAA|2048-12-28|1787|7774|597|2048|1|12|28|4|2048|597|7774|Monday|2048Q4|N|N|N|2469412|2469746|2469073|2469347|N|N|N|N|N| +2469440|AAAAAAAAAEOKFCAA|2048-12-29|1787|7775|597|2048|2|12|29|4|2048|597|7775|Tuesday|2048Q4|N|N|N|2469412|2469746|2469074|2469348|N|N|N|N|N| +2469441|AAAAAAAABEOKFCAA|2048-12-30|1787|7775|597|2048|3|12|30|4|2048|597|7775|Wednesday|2048Q4|N|N|N|2469412|2469746|2469075|2469349|N|N|N|N|N| +2469442|AAAAAAAACEOKFCAA|2048-12-31|1787|7775|597|2048|4|12|31|4|2048|597|7775|Thursday|2048Q4|Y|N|N|2469412|2469746|2469076|2469350|N|N|N|N|N| +2469443|AAAAAAAADEOKFCAA|2049-01-01|1788|7775|597|2049|5|1|1|1|2049|597|7775|Friday|2049Q1|Y|Y|Y|2469443|2469442|2469077|2469351|N|N|N|N|N| +2469444|AAAAAAAAEEOKFCAA|2049-01-02|1788|7775|597|2049|6|1|2|1|2049|597|7775|Saturday|2049Q1|N|Y|Y|2469443|2469442|2469078|2469352|N|N|N|N|N| +2469445|AAAAAAAAFEOKFCAA|2049-01-03|1788|7775|597|2049|0|1|3|1|2049|597|7775|Sunday|2049Q1|N|N|N|2469443|2469442|2469079|2469353|N|N|N|N|N| +2469446|AAAAAAAAGEOKFCAA|2049-01-04|1788|7775|597|2049|1|1|4|1|2049|597|7775|Monday|2049Q1|N|N|N|2469443|2469442|2469080|2469354|N|N|N|N|N| +2469447|AAAAAAAAHEOKFCAA|2049-01-05|1788|7776|597|2049|2|1|5|1|2049|597|7776|Tuesday|2049Q1|N|N|N|2469443|2469442|2469081|2469355|N|N|N|N|N| +2469448|AAAAAAAAIEOKFCAA|2049-01-06|1788|7776|597|2049|3|1|6|1|2049|597|7776|Wednesday|2049Q1|N|N|N|2469443|2469442|2469082|2469356|N|N|N|N|N| +2469449|AAAAAAAAJEOKFCAA|2049-01-07|1788|7776|597|2049|4|1|7|1|2049|597|7776|Thursday|2049Q1|N|N|N|2469443|2469442|2469083|2469357|N|N|N|N|N| +2469450|AAAAAAAAKEOKFCAA|2049-01-08|1788|7776|597|2049|5|1|8|1|2049|597|7776|Friday|2049Q1|N|Y|N|2469443|2469442|2469084|2469358|N|N|N|N|N| +2469451|AAAAAAAALEOKFCAA|2049-01-09|1788|7776|597|2049|6|1|9|1|2049|597|7776|Saturday|2049Q1|N|Y|N|2469443|2469442|2469085|2469359|N|N|N|N|N| +2469452|AAAAAAAAMEOKFCAA|2049-01-10|1788|7776|597|2049|0|1|10|1|2049|597|7776|Sunday|2049Q1|N|N|N|2469443|2469442|2469086|2469360|N|N|N|N|N| +2469453|AAAAAAAANEOKFCAA|2049-01-11|1788|7776|597|2049|1|1|11|1|2049|597|7776|Monday|2049Q1|N|N|N|2469443|2469442|2469087|2469361|N|N|N|N|N| +2469454|AAAAAAAAOEOKFCAA|2049-01-12|1788|7777|597|2049|2|1|12|1|2049|597|7777|Tuesday|2049Q1|N|N|N|2469443|2469442|2469088|2469362|N|N|N|N|N| +2469455|AAAAAAAAPEOKFCAA|2049-01-13|1788|7777|597|2049|3|1|13|1|2049|597|7777|Wednesday|2049Q1|N|N|N|2469443|2469442|2469089|2469363|N|N|N|N|N| +2469456|AAAAAAAAAFOKFCAA|2049-01-14|1788|7777|597|2049|4|1|14|1|2049|597|7777|Thursday|2049Q1|N|N|N|2469443|2469442|2469090|2469364|N|N|N|N|N| +2469457|AAAAAAAABFOKFCAA|2049-01-15|1788|7777|597|2049|5|1|15|1|2049|597|7777|Friday|2049Q1|N|Y|N|2469443|2469442|2469091|2469365|N|N|N|N|N| +2469458|AAAAAAAACFOKFCAA|2049-01-16|1788|7777|597|2049|6|1|16|1|2049|597|7777|Saturday|2049Q1|N|Y|N|2469443|2469442|2469092|2469366|N|N|N|N|N| +2469459|AAAAAAAADFOKFCAA|2049-01-17|1788|7777|597|2049|0|1|17|1|2049|597|7777|Sunday|2049Q1|N|N|N|2469443|2469442|2469093|2469367|N|N|N|N|N| +2469460|AAAAAAAAEFOKFCAA|2049-01-18|1788|7777|597|2049|1|1|18|1|2049|597|7777|Monday|2049Q1|N|N|N|2469443|2469442|2469094|2469368|N|N|N|N|N| +2469461|AAAAAAAAFFOKFCAA|2049-01-19|1788|7778|597|2049|2|1|19|1|2049|597|7778|Tuesday|2049Q1|N|N|N|2469443|2469442|2469095|2469369|N|N|N|N|N| +2469462|AAAAAAAAGFOKFCAA|2049-01-20|1788|7778|597|2049|3|1|20|1|2049|597|7778|Wednesday|2049Q1|N|N|N|2469443|2469442|2469096|2469370|N|N|N|N|N| +2469463|AAAAAAAAHFOKFCAA|2049-01-21|1788|7778|597|2049|4|1|21|1|2049|597|7778|Thursday|2049Q1|N|N|N|2469443|2469442|2469097|2469371|N|N|N|N|N| +2469464|AAAAAAAAIFOKFCAA|2049-01-22|1788|7778|597|2049|5|1|22|1|2049|597|7778|Friday|2049Q1|N|Y|N|2469443|2469442|2469098|2469372|N|N|N|N|N| +2469465|AAAAAAAAJFOKFCAA|2049-01-23|1788|7778|597|2049|6|1|23|1|2049|597|7778|Saturday|2049Q1|N|Y|N|2469443|2469442|2469099|2469373|N|N|N|N|N| +2469466|AAAAAAAAKFOKFCAA|2049-01-24|1788|7778|597|2049|0|1|24|1|2049|597|7778|Sunday|2049Q1|N|N|N|2469443|2469442|2469100|2469374|N|N|N|N|N| +2469467|AAAAAAAALFOKFCAA|2049-01-25|1788|7778|597|2049|1|1|25|1|2049|597|7778|Monday|2049Q1|N|N|N|2469443|2469442|2469101|2469375|N|N|N|N|N| +2469468|AAAAAAAAMFOKFCAA|2049-01-26|1788|7779|597|2049|2|1|26|1|2049|597|7779|Tuesday|2049Q1|N|N|N|2469443|2469442|2469102|2469376|N|N|N|N|N| +2469469|AAAAAAAANFOKFCAA|2049-01-27|1788|7779|597|2049|3|1|27|1|2049|597|7779|Wednesday|2049Q1|N|N|N|2469443|2469442|2469103|2469377|N|N|N|N|N| +2469470|AAAAAAAAOFOKFCAA|2049-01-28|1788|7779|597|2049|4|1|28|1|2049|597|7779|Thursday|2049Q1|N|N|N|2469443|2469442|2469104|2469378|N|N|N|N|N| +2469471|AAAAAAAAPFOKFCAA|2049-01-29|1788|7779|597|2049|5|1|29|1|2049|597|7779|Friday|2049Q1|N|Y|N|2469443|2469442|2469105|2469379|N|N|N|N|N| +2469472|AAAAAAAAAGOKFCAA|2049-01-30|1788|7779|597|2049|6|1|30|1|2049|597|7779|Saturday|2049Q1|N|Y|N|2469443|2469442|2469106|2469380|N|N|N|N|N| +2469473|AAAAAAAABGOKFCAA|2049-01-31|1788|7779|597|2049|0|1|31|1|2049|597|7779|Sunday|2049Q1|N|N|N|2469443|2469442|2469107|2469381|N|N|N|N|N| +2469474|AAAAAAAACGOKFCAA|2049-02-01|1789|7779|597|2049|1|2|1|1|2049|597|7779|Monday|2049Q1|N|N|N|2469474|2469504|2469108|2469382|N|N|N|N|N| +2469475|AAAAAAAADGOKFCAA|2049-02-02|1789|7780|597|2049|2|2|2|1|2049|597|7780|Tuesday|2049Q1|N|N|N|2469474|2469504|2469109|2469383|N|N|N|N|N| +2469476|AAAAAAAAEGOKFCAA|2049-02-03|1789|7780|597|2049|3|2|3|1|2049|597|7780|Wednesday|2049Q1|N|N|N|2469474|2469504|2469110|2469384|N|N|N|N|N| +2469477|AAAAAAAAFGOKFCAA|2049-02-04|1789|7780|597|2049|4|2|4|1|2049|597|7780|Thursday|2049Q1|N|N|N|2469474|2469504|2469111|2469385|N|N|N|N|N| +2469478|AAAAAAAAGGOKFCAA|2049-02-05|1789|7780|597|2049|5|2|5|1|2049|597|7780|Friday|2049Q1|N|Y|N|2469474|2469504|2469112|2469386|N|N|N|N|N| +2469479|AAAAAAAAHGOKFCAA|2049-02-06|1789|7780|597|2049|6|2|6|1|2049|597|7780|Saturday|2049Q1|N|Y|N|2469474|2469504|2469113|2469387|N|N|N|N|N| +2469480|AAAAAAAAIGOKFCAA|2049-02-07|1789|7780|597|2049|0|2|7|1|2049|597|7780|Sunday|2049Q1|N|N|N|2469474|2469504|2469114|2469388|N|N|N|N|N| +2469481|AAAAAAAAJGOKFCAA|2049-02-08|1789|7780|597|2049|1|2|8|1|2049|597|7780|Monday|2049Q1|N|N|N|2469474|2469504|2469115|2469389|N|N|N|N|N| +2469482|AAAAAAAAKGOKFCAA|2049-02-09|1789|7781|597|2049|2|2|9|1|2049|597|7781|Tuesday|2049Q1|N|N|N|2469474|2469504|2469116|2469390|N|N|N|N|N| +2469483|AAAAAAAALGOKFCAA|2049-02-10|1789|7781|597|2049|3|2|10|1|2049|597|7781|Wednesday|2049Q1|N|N|N|2469474|2469504|2469117|2469391|N|N|N|N|N| +2469484|AAAAAAAAMGOKFCAA|2049-02-11|1789|7781|597|2049|4|2|11|1|2049|597|7781|Thursday|2049Q1|N|N|N|2469474|2469504|2469118|2469392|N|N|N|N|N| +2469485|AAAAAAAANGOKFCAA|2049-02-12|1789|7781|597|2049|5|2|12|1|2049|597|7781|Friday|2049Q1|N|Y|N|2469474|2469504|2469119|2469393|N|N|N|N|N| +2469486|AAAAAAAAOGOKFCAA|2049-02-13|1789|7781|597|2049|6|2|13|1|2049|597|7781|Saturday|2049Q1|N|Y|N|2469474|2469504|2469120|2469394|N|N|N|N|N| +2469487|AAAAAAAAPGOKFCAA|2049-02-14|1789|7781|597|2049|0|2|14|1|2049|597|7781|Sunday|2049Q1|N|N|N|2469474|2469504|2469121|2469395|N|N|N|N|N| +2469488|AAAAAAAAAHOKFCAA|2049-02-15|1789|7781|597|2049|1|2|15|1|2049|597|7781|Monday|2049Q1|N|N|N|2469474|2469504|2469122|2469396|N|N|N|N|N| +2469489|AAAAAAAABHOKFCAA|2049-02-16|1789|7782|597|2049|2|2|16|1|2049|597|7782|Tuesday|2049Q1|N|N|N|2469474|2469504|2469123|2469397|N|N|N|N|N| +2469490|AAAAAAAACHOKFCAA|2049-02-17|1789|7782|597|2049|3|2|17|1|2049|597|7782|Wednesday|2049Q1|N|N|N|2469474|2469504|2469124|2469398|N|N|N|N|N| +2469491|AAAAAAAADHOKFCAA|2049-02-18|1789|7782|597|2049|4|2|18|1|2049|597|7782|Thursday|2049Q1|N|N|N|2469474|2469504|2469125|2469399|N|N|N|N|N| +2469492|AAAAAAAAEHOKFCAA|2049-02-19|1789|7782|597|2049|5|2|19|1|2049|597|7782|Friday|2049Q1|N|Y|N|2469474|2469504|2469126|2469400|N|N|N|N|N| +2469493|AAAAAAAAFHOKFCAA|2049-02-20|1789|7782|597|2049|6|2|20|1|2049|597|7782|Saturday|2049Q1|N|Y|N|2469474|2469504|2469127|2469401|N|N|N|N|N| +2469494|AAAAAAAAGHOKFCAA|2049-02-21|1789|7782|597|2049|0|2|21|1|2049|597|7782|Sunday|2049Q1|N|N|N|2469474|2469504|2469128|2469402|N|N|N|N|N| +2469495|AAAAAAAAHHOKFCAA|2049-02-22|1789|7782|597|2049|1|2|22|1|2049|597|7782|Monday|2049Q1|N|N|N|2469474|2469504|2469129|2469403|N|N|N|N|N| +2469496|AAAAAAAAIHOKFCAA|2049-02-23|1789|7783|597|2049|2|2|23|1|2049|597|7783|Tuesday|2049Q1|N|N|N|2469474|2469504|2469130|2469404|N|N|N|N|N| +2469497|AAAAAAAAJHOKFCAA|2049-02-24|1789|7783|597|2049|3|2|24|1|2049|597|7783|Wednesday|2049Q1|N|N|N|2469474|2469504|2469131|2469405|N|N|N|N|N| +2469498|AAAAAAAAKHOKFCAA|2049-02-25|1789|7783|597|2049|4|2|25|1|2049|597|7783|Thursday|2049Q1|N|N|N|2469474|2469504|2469132|2469406|N|N|N|N|N| +2469499|AAAAAAAALHOKFCAA|2049-02-26|1789|7783|597|2049|5|2|26|1|2049|597|7783|Friday|2049Q1|N|Y|N|2469474|2469504|2469133|2469407|N|N|N|N|N| +2469500|AAAAAAAAMHOKFCAA|2049-02-27|1789|7783|597|2049|6|2|27|1|2049|597|7783|Saturday|2049Q1|N|Y|N|2469474|2469504|2469134|2469408|N|N|N|N|N| +2469501|AAAAAAAANHOKFCAA|2049-02-28|1789|7783|597|2049|0|2|28|1|2049|597|7783|Sunday|2049Q1|N|N|N|2469474|2469504|2469135|2469409|N|N|N|N|N| +2469502|AAAAAAAAOHOKFCAA|2049-03-01|1790|7783|598|2049|1|3|1|1|2049|598|7783|Monday|2049Q1|N|N|N|2469502|2469560|2469137|2469410|N|N|N|N|N| +2469503|AAAAAAAAPHOKFCAA|2049-03-02|1790|7784|598|2049|2|3|2|1|2049|598|7784|Tuesday|2049Q1|N|N|N|2469502|2469560|2469138|2469411|N|N|N|N|N| +2469504|AAAAAAAAAIOKFCAA|2049-03-03|1790|7784|598|2049|3|3|3|1|2049|598|7784|Wednesday|2049Q1|N|N|N|2469502|2469560|2469139|2469412|N|N|N|N|N| +2469505|AAAAAAAABIOKFCAA|2049-03-04|1790|7784|598|2049|4|3|4|1|2049|598|7784|Thursday|2049Q1|N|N|N|2469502|2469560|2469140|2469413|N|N|N|N|N| +2469506|AAAAAAAACIOKFCAA|2049-03-05|1790|7784|598|2049|5|3|5|1|2049|598|7784|Friday|2049Q1|N|Y|N|2469502|2469560|2469141|2469414|N|N|N|N|N| +2469507|AAAAAAAADIOKFCAA|2049-03-06|1790|7784|598|2049|6|3|6|1|2049|598|7784|Saturday|2049Q1|N|Y|N|2469502|2469560|2469142|2469415|N|N|N|N|N| +2469508|AAAAAAAAEIOKFCAA|2049-03-07|1790|7784|598|2049|0|3|7|1|2049|598|7784|Sunday|2049Q1|N|N|N|2469502|2469560|2469143|2469416|N|N|N|N|N| +2469509|AAAAAAAAFIOKFCAA|2049-03-08|1790|7784|598|2049|1|3|8|1|2049|598|7784|Monday|2049Q1|N|N|N|2469502|2469560|2469144|2469417|N|N|N|N|N| +2469510|AAAAAAAAGIOKFCAA|2049-03-09|1790|7785|598|2049|2|3|9|1|2049|598|7785|Tuesday|2049Q1|N|N|N|2469502|2469560|2469145|2469418|N|N|N|N|N| +2469511|AAAAAAAAHIOKFCAA|2049-03-10|1790|7785|598|2049|3|3|10|1|2049|598|7785|Wednesday|2049Q1|N|N|N|2469502|2469560|2469146|2469419|N|N|N|N|N| +2469512|AAAAAAAAIIOKFCAA|2049-03-11|1790|7785|598|2049|4|3|11|1|2049|598|7785|Thursday|2049Q1|N|N|N|2469502|2469560|2469147|2469420|N|N|N|N|N| +2469513|AAAAAAAAJIOKFCAA|2049-03-12|1790|7785|598|2049|5|3|12|1|2049|598|7785|Friday|2049Q1|N|Y|N|2469502|2469560|2469148|2469421|N|N|N|N|N| +2469514|AAAAAAAAKIOKFCAA|2049-03-13|1790|7785|598|2049|6|3|13|1|2049|598|7785|Saturday|2049Q1|N|Y|N|2469502|2469560|2469149|2469422|N|N|N|N|N| +2469515|AAAAAAAALIOKFCAA|2049-03-14|1790|7785|598|2049|0|3|14|1|2049|598|7785|Sunday|2049Q1|N|N|N|2469502|2469560|2469150|2469423|N|N|N|N|N| +2469516|AAAAAAAAMIOKFCAA|2049-03-15|1790|7785|598|2049|1|3|15|1|2049|598|7785|Monday|2049Q1|N|N|N|2469502|2469560|2469151|2469424|N|N|N|N|N| +2469517|AAAAAAAANIOKFCAA|2049-03-16|1790|7786|598|2049|2|3|16|1|2049|598|7786|Tuesday|2049Q1|N|N|N|2469502|2469560|2469152|2469425|N|N|N|N|N| +2469518|AAAAAAAAOIOKFCAA|2049-03-17|1790|7786|598|2049|3|3|17|1|2049|598|7786|Wednesday|2049Q1|N|N|N|2469502|2469560|2469153|2469426|N|N|N|N|N| +2469519|AAAAAAAAPIOKFCAA|2049-03-18|1790|7786|598|2049|4|3|18|1|2049|598|7786|Thursday|2049Q1|N|N|N|2469502|2469560|2469154|2469427|N|N|N|N|N| +2469520|AAAAAAAAAJOKFCAA|2049-03-19|1790|7786|598|2049|5|3|19|1|2049|598|7786|Friday|2049Q1|N|Y|N|2469502|2469560|2469155|2469428|N|N|N|N|N| +2469521|AAAAAAAABJOKFCAA|2049-03-20|1790|7786|598|2049|6|3|20|1|2049|598|7786|Saturday|2049Q1|N|Y|N|2469502|2469560|2469156|2469429|N|N|N|N|N| +2469522|AAAAAAAACJOKFCAA|2049-03-21|1790|7786|598|2049|0|3|21|1|2049|598|7786|Sunday|2049Q1|N|N|N|2469502|2469560|2469157|2469430|N|N|N|N|N| +2469523|AAAAAAAADJOKFCAA|2049-03-22|1790|7786|598|2049|1|3|22|1|2049|598|7786|Monday|2049Q1|N|N|N|2469502|2469560|2469158|2469431|N|N|N|N|N| +2469524|AAAAAAAAEJOKFCAA|2049-03-23|1790|7787|598|2049|2|3|23|1|2049|598|7787|Tuesday|2049Q1|N|N|N|2469502|2469560|2469159|2469432|N|N|N|N|N| +2469525|AAAAAAAAFJOKFCAA|2049-03-24|1790|7787|598|2049|3|3|24|1|2049|598|7787|Wednesday|2049Q1|N|N|N|2469502|2469560|2469160|2469433|N|N|N|N|N| +2469526|AAAAAAAAGJOKFCAA|2049-03-25|1790|7787|598|2049|4|3|25|1|2049|598|7787|Thursday|2049Q1|N|N|N|2469502|2469560|2469161|2469434|N|N|N|N|N| +2469527|AAAAAAAAHJOKFCAA|2049-03-26|1790|7787|598|2049|5|3|26|1|2049|598|7787|Friday|2049Q1|N|Y|N|2469502|2469560|2469162|2469435|N|N|N|N|N| +2469528|AAAAAAAAIJOKFCAA|2049-03-27|1790|7787|598|2049|6|3|27|1|2049|598|7787|Saturday|2049Q1|N|Y|N|2469502|2469560|2469163|2469436|N|N|N|N|N| +2469529|AAAAAAAAJJOKFCAA|2049-03-28|1790|7787|598|2049|0|3|28|1|2049|598|7787|Sunday|2049Q1|N|N|N|2469502|2469560|2469164|2469437|N|N|N|N|N| +2469530|AAAAAAAAKJOKFCAA|2049-03-29|1790|7787|598|2049|1|3|29|1|2049|598|7787|Monday|2049Q1|N|N|N|2469502|2469560|2469165|2469438|N|N|N|N|N| +2469531|AAAAAAAALJOKFCAA|2049-03-30|1790|7788|598|2049|2|3|30|1|2049|598|7788|Tuesday|2049Q1|N|N|N|2469502|2469560|2469166|2469439|N|N|N|N|N| +2469532|AAAAAAAAMJOKFCAA|2049-03-31|1790|7788|598|2049|3|3|31|1|2049|598|7788|Wednesday|2049Q1|N|N|N|2469502|2469560|2469167|2469440|N|N|N|N|N| +2469533|AAAAAAAANJOKFCAA|2049-04-01|1791|7788|598|2049|4|4|1|1|2049|598|7788|Thursday|2049Q1|N|N|N|2469533|2469622|2469168|2469443|N|N|N|N|N| +2469534|AAAAAAAAOJOKFCAA|2049-04-02|1791|7788|598|2049|5|4|2|2|2049|598|7788|Friday|2049Q2|N|Y|N|2469533|2469622|2469169|2469444|N|N|N|N|N| +2469535|AAAAAAAAPJOKFCAA|2049-04-03|1791|7788|598|2049|6|4|3|2|2049|598|7788|Saturday|2049Q2|N|Y|N|2469533|2469622|2469170|2469445|N|N|N|N|N| +2469536|AAAAAAAAAKOKFCAA|2049-04-04|1791|7788|598|2049|0|4|4|2|2049|598|7788|Sunday|2049Q2|N|N|N|2469533|2469622|2469171|2469446|N|N|N|N|N| +2469537|AAAAAAAABKOKFCAA|2049-04-05|1791|7788|598|2049|1|4|5|2|2049|598|7788|Monday|2049Q2|N|N|N|2469533|2469622|2469172|2469447|N|N|N|N|N| +2469538|AAAAAAAACKOKFCAA|2049-04-06|1791|7789|598|2049|2|4|6|2|2049|598|7789|Tuesday|2049Q2|N|N|N|2469533|2469622|2469173|2469448|N|N|N|N|N| +2469539|AAAAAAAADKOKFCAA|2049-04-07|1791|7789|598|2049|3|4|7|2|2049|598|7789|Wednesday|2049Q2|N|N|N|2469533|2469622|2469174|2469449|N|N|N|N|N| +2469540|AAAAAAAAEKOKFCAA|2049-04-08|1791|7789|598|2049|4|4|8|2|2049|598|7789|Thursday|2049Q2|N|N|N|2469533|2469622|2469175|2469450|N|N|N|N|N| +2469541|AAAAAAAAFKOKFCAA|2049-04-09|1791|7789|598|2049|5|4|9|2|2049|598|7789|Friday|2049Q2|N|Y|N|2469533|2469622|2469176|2469451|N|N|N|N|N| +2469542|AAAAAAAAGKOKFCAA|2049-04-10|1791|7789|598|2049|6|4|10|2|2049|598|7789|Saturday|2049Q2|N|Y|N|2469533|2469622|2469177|2469452|N|N|N|N|N| +2469543|AAAAAAAAHKOKFCAA|2049-04-11|1791|7789|598|2049|0|4|11|2|2049|598|7789|Sunday|2049Q2|N|N|N|2469533|2469622|2469178|2469453|N|N|N|N|N| +2469544|AAAAAAAAIKOKFCAA|2049-04-12|1791|7789|598|2049|1|4|12|2|2049|598|7789|Monday|2049Q2|N|N|N|2469533|2469622|2469179|2469454|N|N|N|N|N| +2469545|AAAAAAAAJKOKFCAA|2049-04-13|1791|7790|598|2049|2|4|13|2|2049|598|7790|Tuesday|2049Q2|N|N|N|2469533|2469622|2469180|2469455|N|N|N|N|N| +2469546|AAAAAAAAKKOKFCAA|2049-04-14|1791|7790|598|2049|3|4|14|2|2049|598|7790|Wednesday|2049Q2|N|N|N|2469533|2469622|2469181|2469456|N|N|N|N|N| +2469547|AAAAAAAALKOKFCAA|2049-04-15|1791|7790|598|2049|4|4|15|2|2049|598|7790|Thursday|2049Q2|N|N|N|2469533|2469622|2469182|2469457|N|N|N|N|N| +2469548|AAAAAAAAMKOKFCAA|2049-04-16|1791|7790|598|2049|5|4|16|2|2049|598|7790|Friday|2049Q2|N|Y|N|2469533|2469622|2469183|2469458|N|N|N|N|N| +2469549|AAAAAAAANKOKFCAA|2049-04-17|1791|7790|598|2049|6|4|17|2|2049|598|7790|Saturday|2049Q2|N|Y|N|2469533|2469622|2469184|2469459|N|N|N|N|N| +2469550|AAAAAAAAOKOKFCAA|2049-04-18|1791|7790|598|2049|0|4|18|2|2049|598|7790|Sunday|2049Q2|N|N|N|2469533|2469622|2469185|2469460|N|N|N|N|N| +2469551|AAAAAAAAPKOKFCAA|2049-04-19|1791|7790|598|2049|1|4|19|2|2049|598|7790|Monday|2049Q2|N|N|N|2469533|2469622|2469186|2469461|N|N|N|N|N| +2469552|AAAAAAAAALOKFCAA|2049-04-20|1791|7791|598|2049|2|4|20|2|2049|598|7791|Tuesday|2049Q2|N|N|N|2469533|2469622|2469187|2469462|N|N|N|N|N| +2469553|AAAAAAAABLOKFCAA|2049-04-21|1791|7791|598|2049|3|4|21|2|2049|598|7791|Wednesday|2049Q2|N|N|N|2469533|2469622|2469188|2469463|N|N|N|N|N| +2469554|AAAAAAAACLOKFCAA|2049-04-22|1791|7791|598|2049|4|4|22|2|2049|598|7791|Thursday|2049Q2|N|N|N|2469533|2469622|2469189|2469464|N|N|N|N|N| +2469555|AAAAAAAADLOKFCAA|2049-04-23|1791|7791|598|2049|5|4|23|2|2049|598|7791|Friday|2049Q2|N|Y|N|2469533|2469622|2469190|2469465|N|N|N|N|N| +2469556|AAAAAAAAELOKFCAA|2049-04-24|1791|7791|598|2049|6|4|24|2|2049|598|7791|Saturday|2049Q2|N|Y|N|2469533|2469622|2469191|2469466|N|N|N|N|N| +2469557|AAAAAAAAFLOKFCAA|2049-04-25|1791|7791|598|2049|0|4|25|2|2049|598|7791|Sunday|2049Q2|N|N|N|2469533|2469622|2469192|2469467|N|N|N|N|N| +2469558|AAAAAAAAGLOKFCAA|2049-04-26|1791|7791|598|2049|1|4|26|2|2049|598|7791|Monday|2049Q2|N|N|N|2469533|2469622|2469193|2469468|N|N|N|N|N| +2469559|AAAAAAAAHLOKFCAA|2049-04-27|1791|7792|598|2049|2|4|27|2|2049|598|7792|Tuesday|2049Q2|N|N|N|2469533|2469622|2469194|2469469|N|N|N|N|N| +2469560|AAAAAAAAILOKFCAA|2049-04-28|1791|7792|598|2049|3|4|28|2|2049|598|7792|Wednesday|2049Q2|N|N|N|2469533|2469622|2469195|2469470|N|N|N|N|N| +2469561|AAAAAAAAJLOKFCAA|2049-04-29|1791|7792|598|2049|4|4|29|2|2049|598|7792|Thursday|2049Q2|N|N|N|2469533|2469622|2469196|2469471|N|N|N|N|N| +2469562|AAAAAAAAKLOKFCAA|2049-04-30|1791|7792|598|2049|5|4|30|2|2049|598|7792|Friday|2049Q2|N|Y|N|2469533|2469622|2469197|2469472|N|N|N|N|N| +2469563|AAAAAAAALLOKFCAA|2049-05-01|1792|7792|598|2049|6|5|1|2|2049|598|7792|Saturday|2049Q2|N|Y|N|2469563|2469682|2469198|2469473|N|N|N|N|N| +2469564|AAAAAAAAMLOKFCAA|2049-05-02|1792|7792|598|2049|0|5|2|2|2049|598|7792|Sunday|2049Q2|N|N|N|2469563|2469682|2469199|2469474|N|N|N|N|N| +2469565|AAAAAAAANLOKFCAA|2049-05-03|1792|7792|598|2049|1|5|3|2|2049|598|7792|Monday|2049Q2|N|N|N|2469563|2469682|2469200|2469475|N|N|N|N|N| +2469566|AAAAAAAAOLOKFCAA|2049-05-04|1792|7793|598|2049|2|5|4|2|2049|598|7793|Tuesday|2049Q2|N|N|N|2469563|2469682|2469201|2469476|N|N|N|N|N| +2469567|AAAAAAAAPLOKFCAA|2049-05-05|1792|7793|598|2049|3|5|5|2|2049|598|7793|Wednesday|2049Q2|N|N|N|2469563|2469682|2469202|2469477|N|N|N|N|N| +2469568|AAAAAAAAAMOKFCAA|2049-05-06|1792|7793|598|2049|4|5|6|2|2049|598|7793|Thursday|2049Q2|N|N|N|2469563|2469682|2469203|2469478|N|N|N|N|N| +2469569|AAAAAAAABMOKFCAA|2049-05-07|1792|7793|598|2049|5|5|7|2|2049|598|7793|Friday|2049Q2|N|Y|N|2469563|2469682|2469204|2469479|N|N|N|N|N| +2469570|AAAAAAAACMOKFCAA|2049-05-08|1792|7793|598|2049|6|5|8|2|2049|598|7793|Saturday|2049Q2|N|Y|N|2469563|2469682|2469205|2469480|N|N|N|N|N| +2469571|AAAAAAAADMOKFCAA|2049-05-09|1792|7793|598|2049|0|5|9|2|2049|598|7793|Sunday|2049Q2|N|N|N|2469563|2469682|2469206|2469481|N|N|N|N|N| +2469572|AAAAAAAAEMOKFCAA|2049-05-10|1792|7793|598|2049|1|5|10|2|2049|598|7793|Monday|2049Q2|N|N|N|2469563|2469682|2469207|2469482|N|N|N|N|N| +2469573|AAAAAAAAFMOKFCAA|2049-05-11|1792|7794|598|2049|2|5|11|2|2049|598|7794|Tuesday|2049Q2|N|N|N|2469563|2469682|2469208|2469483|N|N|N|N|N| +2469574|AAAAAAAAGMOKFCAA|2049-05-12|1792|7794|598|2049|3|5|12|2|2049|598|7794|Wednesday|2049Q2|N|N|N|2469563|2469682|2469209|2469484|N|N|N|N|N| +2469575|AAAAAAAAHMOKFCAA|2049-05-13|1792|7794|598|2049|4|5|13|2|2049|598|7794|Thursday|2049Q2|N|N|N|2469563|2469682|2469210|2469485|N|N|N|N|N| +2469576|AAAAAAAAIMOKFCAA|2049-05-14|1792|7794|598|2049|5|5|14|2|2049|598|7794|Friday|2049Q2|N|Y|N|2469563|2469682|2469211|2469486|N|N|N|N|N| +2469577|AAAAAAAAJMOKFCAA|2049-05-15|1792|7794|598|2049|6|5|15|2|2049|598|7794|Saturday|2049Q2|N|Y|N|2469563|2469682|2469212|2469487|N|N|N|N|N| +2469578|AAAAAAAAKMOKFCAA|2049-05-16|1792|7794|598|2049|0|5|16|2|2049|598|7794|Sunday|2049Q2|N|N|N|2469563|2469682|2469213|2469488|N|N|N|N|N| +2469579|AAAAAAAALMOKFCAA|2049-05-17|1792|7794|598|2049|1|5|17|2|2049|598|7794|Monday|2049Q2|N|N|N|2469563|2469682|2469214|2469489|N|N|N|N|N| +2469580|AAAAAAAAMMOKFCAA|2049-05-18|1792|7795|598|2049|2|5|18|2|2049|598|7795|Tuesday|2049Q2|N|N|N|2469563|2469682|2469215|2469490|N|N|N|N|N| +2469581|AAAAAAAANMOKFCAA|2049-05-19|1792|7795|598|2049|3|5|19|2|2049|598|7795|Wednesday|2049Q2|N|N|N|2469563|2469682|2469216|2469491|N|N|N|N|N| +2469582|AAAAAAAAOMOKFCAA|2049-05-20|1792|7795|598|2049|4|5|20|2|2049|598|7795|Thursday|2049Q2|N|N|N|2469563|2469682|2469217|2469492|N|N|N|N|N| +2469583|AAAAAAAAPMOKFCAA|2049-05-21|1792|7795|598|2049|5|5|21|2|2049|598|7795|Friday|2049Q2|N|Y|N|2469563|2469682|2469218|2469493|N|N|N|N|N| +2469584|AAAAAAAAANOKFCAA|2049-05-22|1792|7795|598|2049|6|5|22|2|2049|598|7795|Saturday|2049Q2|N|Y|N|2469563|2469682|2469219|2469494|N|N|N|N|N| +2469585|AAAAAAAABNOKFCAA|2049-05-23|1792|7795|598|2049|0|5|23|2|2049|598|7795|Sunday|2049Q2|N|N|N|2469563|2469682|2469220|2469495|N|N|N|N|N| +2469586|AAAAAAAACNOKFCAA|2049-05-24|1792|7795|598|2049|1|5|24|2|2049|598|7795|Monday|2049Q2|N|N|N|2469563|2469682|2469221|2469496|N|N|N|N|N| +2469587|AAAAAAAADNOKFCAA|2049-05-25|1792|7796|598|2049|2|5|25|2|2049|598|7796|Tuesday|2049Q2|N|N|N|2469563|2469682|2469222|2469497|N|N|N|N|N| +2469588|AAAAAAAAENOKFCAA|2049-05-26|1792|7796|598|2049|3|5|26|2|2049|598|7796|Wednesday|2049Q2|N|N|N|2469563|2469682|2469223|2469498|N|N|N|N|N| +2469589|AAAAAAAAFNOKFCAA|2049-05-27|1792|7796|598|2049|4|5|27|2|2049|598|7796|Thursday|2049Q2|N|N|N|2469563|2469682|2469224|2469499|N|N|N|N|N| +2469590|AAAAAAAAGNOKFCAA|2049-05-28|1792|7796|598|2049|5|5|28|2|2049|598|7796|Friday|2049Q2|N|Y|N|2469563|2469682|2469225|2469500|N|N|N|N|N| +2469591|AAAAAAAAHNOKFCAA|2049-05-29|1792|7796|598|2049|6|5|29|2|2049|598|7796|Saturday|2049Q2|N|Y|N|2469563|2469682|2469226|2469501|N|N|N|N|N| +2469592|AAAAAAAAINOKFCAA|2049-05-30|1792|7796|598|2049|0|5|30|2|2049|598|7796|Sunday|2049Q2|N|N|N|2469563|2469682|2469227|2469502|N|N|N|N|N| +2469593|AAAAAAAAJNOKFCAA|2049-05-31|1792|7796|598|2049|1|5|31|2|2049|598|7796|Monday|2049Q2|N|N|N|2469563|2469682|2469228|2469503|N|N|N|N|N| +2469594|AAAAAAAAKNOKFCAA|2049-06-01|1793|7797|599|2049|2|6|1|2|2049|599|7797|Tuesday|2049Q2|N|N|N|2469594|2469744|2469229|2469504|N|N|N|N|N| +2469595|AAAAAAAALNOKFCAA|2049-06-02|1793|7797|599|2049|3|6|2|2|2049|599|7797|Wednesday|2049Q2|N|N|N|2469594|2469744|2469230|2469505|N|N|N|N|N| +2469596|AAAAAAAAMNOKFCAA|2049-06-03|1793|7797|599|2049|4|6|3|2|2049|599|7797|Thursday|2049Q2|N|N|N|2469594|2469744|2469231|2469506|N|N|N|N|N| +2469597|AAAAAAAANNOKFCAA|2049-06-04|1793|7797|599|2049|5|6|4|2|2049|599|7797|Friday|2049Q2|N|Y|N|2469594|2469744|2469232|2469507|N|N|N|N|N| +2469598|AAAAAAAAONOKFCAA|2049-06-05|1793|7797|599|2049|6|6|5|2|2049|599|7797|Saturday|2049Q2|N|Y|N|2469594|2469744|2469233|2469508|N|N|N|N|N| +2469599|AAAAAAAAPNOKFCAA|2049-06-06|1793|7797|599|2049|0|6|6|2|2049|599|7797|Sunday|2049Q2|N|N|N|2469594|2469744|2469234|2469509|N|N|N|N|N| +2469600|AAAAAAAAAOOKFCAA|2049-06-07|1793|7797|599|2049|1|6|7|2|2049|599|7797|Monday|2049Q2|N|N|N|2469594|2469744|2469235|2469510|N|N|N|N|N| +2469601|AAAAAAAABOOKFCAA|2049-06-08|1793|7798|599|2049|2|6|8|2|2049|599|7798|Tuesday|2049Q2|N|N|N|2469594|2469744|2469236|2469511|N|N|N|N|N| +2469602|AAAAAAAACOOKFCAA|2049-06-09|1793|7798|599|2049|3|6|9|2|2049|599|7798|Wednesday|2049Q2|N|N|N|2469594|2469744|2469237|2469512|N|N|N|N|N| +2469603|AAAAAAAADOOKFCAA|2049-06-10|1793|7798|599|2049|4|6|10|2|2049|599|7798|Thursday|2049Q2|N|N|N|2469594|2469744|2469238|2469513|N|N|N|N|N| +2469604|AAAAAAAAEOOKFCAA|2049-06-11|1793|7798|599|2049|5|6|11|2|2049|599|7798|Friday|2049Q2|N|Y|N|2469594|2469744|2469239|2469514|N|N|N|N|N| +2469605|AAAAAAAAFOOKFCAA|2049-06-12|1793|7798|599|2049|6|6|12|2|2049|599|7798|Saturday|2049Q2|N|Y|N|2469594|2469744|2469240|2469515|N|N|N|N|N| +2469606|AAAAAAAAGOOKFCAA|2049-06-13|1793|7798|599|2049|0|6|13|2|2049|599|7798|Sunday|2049Q2|N|N|N|2469594|2469744|2469241|2469516|N|N|N|N|N| +2469607|AAAAAAAAHOOKFCAA|2049-06-14|1793|7798|599|2049|1|6|14|2|2049|599|7798|Monday|2049Q2|N|N|N|2469594|2469744|2469242|2469517|N|N|N|N|N| +2469608|AAAAAAAAIOOKFCAA|2049-06-15|1793|7799|599|2049|2|6|15|2|2049|599|7799|Tuesday|2049Q2|N|N|N|2469594|2469744|2469243|2469518|N|N|N|N|N| +2469609|AAAAAAAAJOOKFCAA|2049-06-16|1793|7799|599|2049|3|6|16|2|2049|599|7799|Wednesday|2049Q2|N|N|N|2469594|2469744|2469244|2469519|N|N|N|N|N| +2469610|AAAAAAAAKOOKFCAA|2049-06-17|1793|7799|599|2049|4|6|17|2|2049|599|7799|Thursday|2049Q2|N|N|N|2469594|2469744|2469245|2469520|N|N|N|N|N| +2469611|AAAAAAAALOOKFCAA|2049-06-18|1793|7799|599|2049|5|6|18|2|2049|599|7799|Friday|2049Q2|N|Y|N|2469594|2469744|2469246|2469521|N|N|N|N|N| +2469612|AAAAAAAAMOOKFCAA|2049-06-19|1793|7799|599|2049|6|6|19|2|2049|599|7799|Saturday|2049Q2|N|Y|N|2469594|2469744|2469247|2469522|N|N|N|N|N| +2469613|AAAAAAAANOOKFCAA|2049-06-20|1793|7799|599|2049|0|6|20|2|2049|599|7799|Sunday|2049Q2|N|N|N|2469594|2469744|2469248|2469523|N|N|N|N|N| +2469614|AAAAAAAAOOOKFCAA|2049-06-21|1793|7799|599|2049|1|6|21|2|2049|599|7799|Monday|2049Q2|N|N|N|2469594|2469744|2469249|2469524|N|N|N|N|N| +2469615|AAAAAAAAPOOKFCAA|2049-06-22|1793|7800|599|2049|2|6|22|2|2049|599|7800|Tuesday|2049Q2|N|N|N|2469594|2469744|2469250|2469525|N|N|N|N|N| +2469616|AAAAAAAAAPOKFCAA|2049-06-23|1793|7800|599|2049|3|6|23|2|2049|599|7800|Wednesday|2049Q2|N|N|N|2469594|2469744|2469251|2469526|N|N|N|N|N| +2469617|AAAAAAAABPOKFCAA|2049-06-24|1793|7800|599|2049|4|6|24|2|2049|599|7800|Thursday|2049Q2|N|N|N|2469594|2469744|2469252|2469527|N|N|N|N|N| +2469618|AAAAAAAACPOKFCAA|2049-06-25|1793|7800|599|2049|5|6|25|2|2049|599|7800|Friday|2049Q2|N|Y|N|2469594|2469744|2469253|2469528|N|N|N|N|N| +2469619|AAAAAAAADPOKFCAA|2049-06-26|1793|7800|599|2049|6|6|26|2|2049|599|7800|Saturday|2049Q2|N|Y|N|2469594|2469744|2469254|2469529|N|N|N|N|N| +2469620|AAAAAAAAEPOKFCAA|2049-06-27|1793|7800|599|2049|0|6|27|2|2049|599|7800|Sunday|2049Q2|N|N|N|2469594|2469744|2469255|2469530|N|N|N|N|N| +2469621|AAAAAAAAFPOKFCAA|2049-06-28|1793|7800|599|2049|1|6|28|2|2049|599|7800|Monday|2049Q2|N|N|N|2469594|2469744|2469256|2469531|N|N|N|N|N| +2469622|AAAAAAAAGPOKFCAA|2049-06-29|1793|7801|599|2049|2|6|29|2|2049|599|7801|Tuesday|2049Q2|N|N|N|2469594|2469744|2469257|2469532|N|N|N|N|N| +2469623|AAAAAAAAHPOKFCAA|2049-06-30|1793|7801|599|2049|3|6|30|2|2049|599|7801|Wednesday|2049Q2|N|N|N|2469594|2469744|2469258|2469533|N|N|N|N|N| +2469624|AAAAAAAAIPOKFCAA|2049-07-01|1794|7801|599|2049|4|7|1|2|2049|599|7801|Thursday|2049Q2|N|N|N|2469624|2469804|2469259|2469533|N|N|N|N|N| +2469625|AAAAAAAAJPOKFCAA|2049-07-02|1794|7801|599|2049|5|7|2|3|2049|599|7801|Friday|2049Q3|N|Y|N|2469624|2469804|2469260|2469534|N|N|N|N|N| +2469626|AAAAAAAAKPOKFCAA|2049-07-03|1794|7801|599|2049|6|7|3|3|2049|599|7801|Saturday|2049Q3|N|Y|N|2469624|2469804|2469261|2469535|N|N|N|N|N| +2469627|AAAAAAAALPOKFCAA|2049-07-04|1794|7801|599|2049|0|7|4|3|2049|599|7801|Sunday|2049Q3|N|N|N|2469624|2469804|2469262|2469536|N|N|N|N|N| +2469628|AAAAAAAAMPOKFCAA|2049-07-05|1794|7801|599|2049|1|7|5|3|2049|599|7801|Monday|2049Q3|Y|N|N|2469624|2469804|2469263|2469537|N|N|N|N|N| +2469629|AAAAAAAANPOKFCAA|2049-07-06|1794|7802|599|2049|2|7|6|3|2049|599|7802|Tuesday|2049Q3|N|N|Y|2469624|2469804|2469264|2469538|N|N|N|N|N| +2469630|AAAAAAAAOPOKFCAA|2049-07-07|1794|7802|599|2049|3|7|7|3|2049|599|7802|Wednesday|2049Q3|N|N|N|2469624|2469804|2469265|2469539|N|N|N|N|N| +2469631|AAAAAAAAPPOKFCAA|2049-07-08|1794|7802|599|2049|4|7|8|3|2049|599|7802|Thursday|2049Q3|N|N|N|2469624|2469804|2469266|2469540|N|N|N|N|N| +2469632|AAAAAAAAAAPKFCAA|2049-07-09|1794|7802|599|2049|5|7|9|3|2049|599|7802|Friday|2049Q3|N|Y|N|2469624|2469804|2469267|2469541|N|N|N|N|N| +2469633|AAAAAAAABAPKFCAA|2049-07-10|1794|7802|599|2049|6|7|10|3|2049|599|7802|Saturday|2049Q3|N|Y|N|2469624|2469804|2469268|2469542|N|N|N|N|N| +2469634|AAAAAAAACAPKFCAA|2049-07-11|1794|7802|599|2049|0|7|11|3|2049|599|7802|Sunday|2049Q3|N|N|N|2469624|2469804|2469269|2469543|N|N|N|N|N| +2469635|AAAAAAAADAPKFCAA|2049-07-12|1794|7802|599|2049|1|7|12|3|2049|599|7802|Monday|2049Q3|N|N|N|2469624|2469804|2469270|2469544|N|N|N|N|N| +2469636|AAAAAAAAEAPKFCAA|2049-07-13|1794|7803|599|2049|2|7|13|3|2049|599|7803|Tuesday|2049Q3|N|N|N|2469624|2469804|2469271|2469545|N|N|N|N|N| +2469637|AAAAAAAAFAPKFCAA|2049-07-14|1794|7803|599|2049|3|7|14|3|2049|599|7803|Wednesday|2049Q3|N|N|N|2469624|2469804|2469272|2469546|N|N|N|N|N| +2469638|AAAAAAAAGAPKFCAA|2049-07-15|1794|7803|599|2049|4|7|15|3|2049|599|7803|Thursday|2049Q3|N|N|N|2469624|2469804|2469273|2469547|N|N|N|N|N| +2469639|AAAAAAAAHAPKFCAA|2049-07-16|1794|7803|599|2049|5|7|16|3|2049|599|7803|Friday|2049Q3|N|Y|N|2469624|2469804|2469274|2469548|N|N|N|N|N| +2469640|AAAAAAAAIAPKFCAA|2049-07-17|1794|7803|599|2049|6|7|17|3|2049|599|7803|Saturday|2049Q3|N|Y|N|2469624|2469804|2469275|2469549|N|N|N|N|N| +2469641|AAAAAAAAJAPKFCAA|2049-07-18|1794|7803|599|2049|0|7|18|3|2049|599|7803|Sunday|2049Q3|N|N|N|2469624|2469804|2469276|2469550|N|N|N|N|N| +2469642|AAAAAAAAKAPKFCAA|2049-07-19|1794|7803|599|2049|1|7|19|3|2049|599|7803|Monday|2049Q3|N|N|N|2469624|2469804|2469277|2469551|N|N|N|N|N| +2469643|AAAAAAAALAPKFCAA|2049-07-20|1794|7804|599|2049|2|7|20|3|2049|599|7804|Tuesday|2049Q3|N|N|N|2469624|2469804|2469278|2469552|N|N|N|N|N| +2469644|AAAAAAAAMAPKFCAA|2049-07-21|1794|7804|599|2049|3|7|21|3|2049|599|7804|Wednesday|2049Q3|N|N|N|2469624|2469804|2469279|2469553|N|N|N|N|N| +2469645|AAAAAAAANAPKFCAA|2049-07-22|1794|7804|599|2049|4|7|22|3|2049|599|7804|Thursday|2049Q3|N|N|N|2469624|2469804|2469280|2469554|N|N|N|N|N| +2469646|AAAAAAAAOAPKFCAA|2049-07-23|1794|7804|599|2049|5|7|23|3|2049|599|7804|Friday|2049Q3|N|Y|N|2469624|2469804|2469281|2469555|N|N|N|N|N| +2469647|AAAAAAAAPAPKFCAA|2049-07-24|1794|7804|599|2049|6|7|24|3|2049|599|7804|Saturday|2049Q3|N|Y|N|2469624|2469804|2469282|2469556|N|N|N|N|N| +2469648|AAAAAAAAABPKFCAA|2049-07-25|1794|7804|599|2049|0|7|25|3|2049|599|7804|Sunday|2049Q3|N|N|N|2469624|2469804|2469283|2469557|N|N|N|N|N| +2469649|AAAAAAAABBPKFCAA|2049-07-26|1794|7804|599|2049|1|7|26|3|2049|599|7804|Monday|2049Q3|N|N|N|2469624|2469804|2469284|2469558|N|N|N|N|N| +2469650|AAAAAAAACBPKFCAA|2049-07-27|1794|7805|599|2049|2|7|27|3|2049|599|7805|Tuesday|2049Q3|N|N|N|2469624|2469804|2469285|2469559|N|N|N|N|N| +2469651|AAAAAAAADBPKFCAA|2049-07-28|1794|7805|599|2049|3|7|28|3|2049|599|7805|Wednesday|2049Q3|N|N|N|2469624|2469804|2469286|2469560|N|N|N|N|N| +2469652|AAAAAAAAEBPKFCAA|2049-07-29|1794|7805|599|2049|4|7|29|3|2049|599|7805|Thursday|2049Q3|N|N|N|2469624|2469804|2469287|2469561|N|N|N|N|N| +2469653|AAAAAAAAFBPKFCAA|2049-07-30|1794|7805|599|2049|5|7|30|3|2049|599|7805|Friday|2049Q3|N|Y|N|2469624|2469804|2469288|2469562|N|N|N|N|N| +2469654|AAAAAAAAGBPKFCAA|2049-07-31|1794|7805|599|2049|6|7|31|3|2049|599|7805|Saturday|2049Q3|N|Y|N|2469624|2469804|2469289|2469563|N|N|N|N|N| +2469655|AAAAAAAAHBPKFCAA|2049-08-01|1795|7805|599|2049|0|8|1|3|2049|599|7805|Sunday|2049Q3|N|N|N|2469655|2469866|2469290|2469564|N|N|N|N|N| +2469656|AAAAAAAAIBPKFCAA|2049-08-02|1795|7805|599|2049|1|8|2|3|2049|599|7805|Monday|2049Q3|N|N|N|2469655|2469866|2469291|2469565|N|N|N|N|N| +2469657|AAAAAAAAJBPKFCAA|2049-08-03|1795|7806|599|2049|2|8|3|3|2049|599|7806|Tuesday|2049Q3|N|N|N|2469655|2469866|2469292|2469566|N|N|N|N|N| +2469658|AAAAAAAAKBPKFCAA|2049-08-04|1795|7806|599|2049|3|8|4|3|2049|599|7806|Wednesday|2049Q3|N|N|N|2469655|2469866|2469293|2469567|N|N|N|N|N| +2469659|AAAAAAAALBPKFCAA|2049-08-05|1795|7806|599|2049|4|8|5|3|2049|599|7806|Thursday|2049Q3|N|N|N|2469655|2469866|2469294|2469568|N|N|N|N|N| +2469660|AAAAAAAAMBPKFCAA|2049-08-06|1795|7806|599|2049|5|8|6|3|2049|599|7806|Friday|2049Q3|N|Y|N|2469655|2469866|2469295|2469569|N|N|N|N|N| +2469661|AAAAAAAANBPKFCAA|2049-08-07|1795|7806|599|2049|6|8|7|3|2049|599|7806|Saturday|2049Q3|N|Y|N|2469655|2469866|2469296|2469570|N|N|N|N|N| +2469662|AAAAAAAAOBPKFCAA|2049-08-08|1795|7806|599|2049|0|8|8|3|2049|599|7806|Sunday|2049Q3|N|N|N|2469655|2469866|2469297|2469571|N|N|N|N|N| +2469663|AAAAAAAAPBPKFCAA|2049-08-09|1795|7806|599|2049|1|8|9|3|2049|599|7806|Monday|2049Q3|N|N|N|2469655|2469866|2469298|2469572|N|N|N|N|N| +2469664|AAAAAAAAACPKFCAA|2049-08-10|1795|7807|599|2049|2|8|10|3|2049|599|7807|Tuesday|2049Q3|N|N|N|2469655|2469866|2469299|2469573|N|N|N|N|N| +2469665|AAAAAAAABCPKFCAA|2049-08-11|1795|7807|599|2049|3|8|11|3|2049|599|7807|Wednesday|2049Q3|N|N|N|2469655|2469866|2469300|2469574|N|N|N|N|N| +2469666|AAAAAAAACCPKFCAA|2049-08-12|1795|7807|599|2049|4|8|12|3|2049|599|7807|Thursday|2049Q3|N|N|N|2469655|2469866|2469301|2469575|N|N|N|N|N| +2469667|AAAAAAAADCPKFCAA|2049-08-13|1795|7807|599|2049|5|8|13|3|2049|599|7807|Friday|2049Q3|N|Y|N|2469655|2469866|2469302|2469576|N|N|N|N|N| +2469668|AAAAAAAAECPKFCAA|2049-08-14|1795|7807|599|2049|6|8|14|3|2049|599|7807|Saturday|2049Q3|N|Y|N|2469655|2469866|2469303|2469577|N|N|N|N|N| +2469669|AAAAAAAAFCPKFCAA|2049-08-15|1795|7807|599|2049|0|8|15|3|2049|599|7807|Sunday|2049Q3|N|N|N|2469655|2469866|2469304|2469578|N|N|N|N|N| +2469670|AAAAAAAAGCPKFCAA|2049-08-16|1795|7807|599|2049|1|8|16|3|2049|599|7807|Monday|2049Q3|N|N|N|2469655|2469866|2469305|2469579|N|N|N|N|N| +2469671|AAAAAAAAHCPKFCAA|2049-08-17|1795|7808|599|2049|2|8|17|3|2049|599|7808|Tuesday|2049Q3|N|N|N|2469655|2469866|2469306|2469580|N|N|N|N|N| +2469672|AAAAAAAAICPKFCAA|2049-08-18|1795|7808|599|2049|3|8|18|3|2049|599|7808|Wednesday|2049Q3|N|N|N|2469655|2469866|2469307|2469581|N|N|N|N|N| +2469673|AAAAAAAAJCPKFCAA|2049-08-19|1795|7808|599|2049|4|8|19|3|2049|599|7808|Thursday|2049Q3|N|N|N|2469655|2469866|2469308|2469582|N|N|N|N|N| +2469674|AAAAAAAAKCPKFCAA|2049-08-20|1795|7808|599|2049|5|8|20|3|2049|599|7808|Friday|2049Q3|N|Y|N|2469655|2469866|2469309|2469583|N|N|N|N|N| +2469675|AAAAAAAALCPKFCAA|2049-08-21|1795|7808|599|2049|6|8|21|3|2049|599|7808|Saturday|2049Q3|N|Y|N|2469655|2469866|2469310|2469584|N|N|N|N|N| +2469676|AAAAAAAAMCPKFCAA|2049-08-22|1795|7808|599|2049|0|8|22|3|2049|599|7808|Sunday|2049Q3|N|N|N|2469655|2469866|2469311|2469585|N|N|N|N|N| +2469677|AAAAAAAANCPKFCAA|2049-08-23|1795|7808|599|2049|1|8|23|3|2049|599|7808|Monday|2049Q3|N|N|N|2469655|2469866|2469312|2469586|N|N|N|N|N| +2469678|AAAAAAAAOCPKFCAA|2049-08-24|1795|7809|599|2049|2|8|24|3|2049|599|7809|Tuesday|2049Q3|N|N|N|2469655|2469866|2469313|2469587|N|N|N|N|N| +2469679|AAAAAAAAPCPKFCAA|2049-08-25|1795|7809|599|2049|3|8|25|3|2049|599|7809|Wednesday|2049Q3|N|N|N|2469655|2469866|2469314|2469588|N|N|N|N|N| +2469680|AAAAAAAAADPKFCAA|2049-08-26|1795|7809|599|2049|4|8|26|3|2049|599|7809|Thursday|2049Q3|N|N|N|2469655|2469866|2469315|2469589|N|N|N|N|N| +2469681|AAAAAAAABDPKFCAA|2049-08-27|1795|7809|599|2049|5|8|27|3|2049|599|7809|Friday|2049Q3|N|Y|N|2469655|2469866|2469316|2469590|N|N|N|N|N| +2469682|AAAAAAAACDPKFCAA|2049-08-28|1795|7809|599|2049|6|8|28|3|2049|599|7809|Saturday|2049Q3|N|Y|N|2469655|2469866|2469317|2469591|N|N|N|N|N| +2469683|AAAAAAAADDPKFCAA|2049-08-29|1795|7809|599|2049|0|8|29|3|2049|599|7809|Sunday|2049Q3|N|N|N|2469655|2469866|2469318|2469592|N|N|N|N|N| +2469684|AAAAAAAAEDPKFCAA|2049-08-30|1795|7809|599|2049|1|8|30|3|2049|599|7809|Monday|2049Q3|N|N|N|2469655|2469866|2469319|2469593|N|N|N|N|N| +2469685|AAAAAAAAFDPKFCAA|2049-08-31|1795|7810|599|2049|2|8|31|3|2049|599|7810|Tuesday|2049Q3|N|N|N|2469655|2469866|2469320|2469594|N|N|N|N|N| +2469686|AAAAAAAAGDPKFCAA|2049-09-01|1796|7810|600|2049|3|9|1|3|2049|600|7810|Wednesday|2049Q3|N|N|N|2469686|2469928|2469321|2469595|N|N|N|N|N| +2469687|AAAAAAAAHDPKFCAA|2049-09-02|1796|7810|600|2049|4|9|2|3|2049|600|7810|Thursday|2049Q3|N|N|N|2469686|2469928|2469322|2469596|N|N|N|N|N| +2469688|AAAAAAAAIDPKFCAA|2049-09-03|1796|7810|600|2049|5|9|3|3|2049|600|7810|Friday|2049Q3|N|Y|N|2469686|2469928|2469323|2469597|N|N|N|N|N| +2469689|AAAAAAAAJDPKFCAA|2049-09-04|1796|7810|600|2049|6|9|4|3|2049|600|7810|Saturday|2049Q3|N|Y|N|2469686|2469928|2469324|2469598|N|N|N|N|N| +2469690|AAAAAAAAKDPKFCAA|2049-09-05|1796|7810|600|2049|0|9|5|3|2049|600|7810|Sunday|2049Q3|N|N|N|2469686|2469928|2469325|2469599|N|N|N|N|N| +2469691|AAAAAAAALDPKFCAA|2049-09-06|1796|7810|600|2049|1|9|6|3|2049|600|7810|Monday|2049Q3|N|N|N|2469686|2469928|2469326|2469600|N|N|N|N|N| +2469692|AAAAAAAAMDPKFCAA|2049-09-07|1796|7811|600|2049|2|9|7|3|2049|600|7811|Tuesday|2049Q3|N|N|N|2469686|2469928|2469327|2469601|N|N|N|N|N| +2469693|AAAAAAAANDPKFCAA|2049-09-08|1796|7811|600|2049|3|9|8|3|2049|600|7811|Wednesday|2049Q3|N|N|N|2469686|2469928|2469328|2469602|N|N|N|N|N| +2469694|AAAAAAAAODPKFCAA|2049-09-09|1796|7811|600|2049|4|9|9|3|2049|600|7811|Thursday|2049Q3|N|N|N|2469686|2469928|2469329|2469603|N|N|N|N|N| +2469695|AAAAAAAAPDPKFCAA|2049-09-10|1796|7811|600|2049|5|9|10|3|2049|600|7811|Friday|2049Q3|N|Y|N|2469686|2469928|2469330|2469604|N|N|N|N|N| +2469696|AAAAAAAAAEPKFCAA|2049-09-11|1796|7811|600|2049|6|9|11|3|2049|600|7811|Saturday|2049Q3|N|Y|N|2469686|2469928|2469331|2469605|N|N|N|N|N| +2469697|AAAAAAAABEPKFCAA|2049-09-12|1796|7811|600|2049|0|9|12|3|2049|600|7811|Sunday|2049Q3|N|N|N|2469686|2469928|2469332|2469606|N|N|N|N|N| +2469698|AAAAAAAACEPKFCAA|2049-09-13|1796|7811|600|2049|1|9|13|3|2049|600|7811|Monday|2049Q3|N|N|N|2469686|2469928|2469333|2469607|N|N|N|N|N| +2469699|AAAAAAAADEPKFCAA|2049-09-14|1796|7812|600|2049|2|9|14|3|2049|600|7812|Tuesday|2049Q3|N|N|N|2469686|2469928|2469334|2469608|N|N|N|N|N| +2469700|AAAAAAAAEEPKFCAA|2049-09-15|1796|7812|600|2049|3|9|15|3|2049|600|7812|Wednesday|2049Q3|N|N|N|2469686|2469928|2469335|2469609|N|N|N|N|N| +2469701|AAAAAAAAFEPKFCAA|2049-09-16|1796|7812|600|2049|4|9|16|3|2049|600|7812|Thursday|2049Q3|N|N|N|2469686|2469928|2469336|2469610|N|N|N|N|N| +2469702|AAAAAAAAGEPKFCAA|2049-09-17|1796|7812|600|2049|5|9|17|3|2049|600|7812|Friday|2049Q3|N|Y|N|2469686|2469928|2469337|2469611|N|N|N|N|N| +2469703|AAAAAAAAHEPKFCAA|2049-09-18|1796|7812|600|2049|6|9|18|3|2049|600|7812|Saturday|2049Q3|N|Y|N|2469686|2469928|2469338|2469612|N|N|N|N|N| +2469704|AAAAAAAAIEPKFCAA|2049-09-19|1796|7812|600|2049|0|9|19|3|2049|600|7812|Sunday|2049Q3|N|N|N|2469686|2469928|2469339|2469613|N|N|N|N|N| +2469705|AAAAAAAAJEPKFCAA|2049-09-20|1796|7812|600|2049|1|9|20|3|2049|600|7812|Monday|2049Q3|N|N|N|2469686|2469928|2469340|2469614|N|N|N|N|N| +2469706|AAAAAAAAKEPKFCAA|2049-09-21|1796|7813|600|2049|2|9|21|3|2049|600|7813|Tuesday|2049Q3|N|N|N|2469686|2469928|2469341|2469615|N|N|N|N|N| +2469707|AAAAAAAALEPKFCAA|2049-09-22|1796|7813|600|2049|3|9|22|3|2049|600|7813|Wednesday|2049Q3|N|N|N|2469686|2469928|2469342|2469616|N|N|N|N|N| +2469708|AAAAAAAAMEPKFCAA|2049-09-23|1796|7813|600|2049|4|9|23|3|2049|600|7813|Thursday|2049Q3|N|N|N|2469686|2469928|2469343|2469617|N|N|N|N|N| +2469709|AAAAAAAANEPKFCAA|2049-09-24|1796|7813|600|2049|5|9|24|3|2049|600|7813|Friday|2049Q3|N|Y|N|2469686|2469928|2469344|2469618|N|N|N|N|N| +2469710|AAAAAAAAOEPKFCAA|2049-09-25|1796|7813|600|2049|6|9|25|3|2049|600|7813|Saturday|2049Q3|N|Y|N|2469686|2469928|2469345|2469619|N|N|N|N|N| +2469711|AAAAAAAAPEPKFCAA|2049-09-26|1796|7813|600|2049|0|9|26|3|2049|600|7813|Sunday|2049Q3|N|N|N|2469686|2469928|2469346|2469620|N|N|N|N|N| +2469712|AAAAAAAAAFPKFCAA|2049-09-27|1796|7813|600|2049|1|9|27|3|2049|600|7813|Monday|2049Q3|N|N|N|2469686|2469928|2469347|2469621|N|N|N|N|N| +2469713|AAAAAAAABFPKFCAA|2049-09-28|1796|7814|600|2049|2|9|28|3|2049|600|7814|Tuesday|2049Q3|N|N|N|2469686|2469928|2469348|2469622|N|N|N|N|N| +2469714|AAAAAAAACFPKFCAA|2049-09-29|1796|7814|600|2049|3|9|29|3|2049|600|7814|Wednesday|2049Q3|N|N|N|2469686|2469928|2469349|2469623|N|N|N|N|N| +2469715|AAAAAAAADFPKFCAA|2049-09-30|1796|7814|600|2049|4|9|30|3|2049|600|7814|Thursday|2049Q3|N|N|N|2469686|2469928|2469350|2469624|N|N|N|N|N| +2469716|AAAAAAAAEFPKFCAA|2049-10-01|1797|7814|600|2049|5|10|1|3|2049|600|7814|Friday|2049Q3|N|Y|N|2469716|2469988|2469351|2469624|N|N|N|N|N| +2469717|AAAAAAAAFFPKFCAA|2049-10-02|1797|7814|600|2049|6|10|2|4|2049|600|7814|Saturday|2049Q4|N|Y|N|2469716|2469988|2469352|2469625|N|N|N|N|N| +2469718|AAAAAAAAGFPKFCAA|2049-10-03|1797|7814|600|2049|0|10|3|4|2049|600|7814|Sunday|2049Q4|N|N|N|2469716|2469988|2469353|2469626|N|N|N|N|N| +2469719|AAAAAAAAHFPKFCAA|2049-10-04|1797|7814|600|2049|1|10|4|4|2049|600|7814|Monday|2049Q4|N|N|N|2469716|2469988|2469354|2469627|N|N|N|N|N| +2469720|AAAAAAAAIFPKFCAA|2049-10-05|1797|7815|600|2049|2|10|5|4|2049|600|7815|Tuesday|2049Q4|N|N|N|2469716|2469988|2469355|2469628|N|N|N|N|N| +2469721|AAAAAAAAJFPKFCAA|2049-10-06|1797|7815|600|2049|3|10|6|4|2049|600|7815|Wednesday|2049Q4|N|N|N|2469716|2469988|2469356|2469629|N|N|N|N|N| +2469722|AAAAAAAAKFPKFCAA|2049-10-07|1797|7815|600|2049|4|10|7|4|2049|600|7815|Thursday|2049Q4|N|N|N|2469716|2469988|2469357|2469630|N|N|N|N|N| +2469723|AAAAAAAALFPKFCAA|2049-10-08|1797|7815|600|2049|5|10|8|4|2049|600|7815|Friday|2049Q4|N|Y|N|2469716|2469988|2469358|2469631|N|N|N|N|N| +2469724|AAAAAAAAMFPKFCAA|2049-10-09|1797|7815|600|2049|6|10|9|4|2049|600|7815|Saturday|2049Q4|N|Y|N|2469716|2469988|2469359|2469632|N|N|N|N|N| +2469725|AAAAAAAANFPKFCAA|2049-10-10|1797|7815|600|2049|0|10|10|4|2049|600|7815|Sunday|2049Q4|N|N|N|2469716|2469988|2469360|2469633|N|N|N|N|N| +2469726|AAAAAAAAOFPKFCAA|2049-10-11|1797|7815|600|2049|1|10|11|4|2049|600|7815|Monday|2049Q4|N|N|N|2469716|2469988|2469361|2469634|N|N|N|N|N| +2469727|AAAAAAAAPFPKFCAA|2049-10-12|1797|7816|600|2049|2|10|12|4|2049|600|7816|Tuesday|2049Q4|N|N|N|2469716|2469988|2469362|2469635|N|N|N|N|N| +2469728|AAAAAAAAAGPKFCAA|2049-10-13|1797|7816|600|2049|3|10|13|4|2049|600|7816|Wednesday|2049Q4|N|N|N|2469716|2469988|2469363|2469636|N|N|N|N|N| +2469729|AAAAAAAABGPKFCAA|2049-10-14|1797|7816|600|2049|4|10|14|4|2049|600|7816|Thursday|2049Q4|N|N|N|2469716|2469988|2469364|2469637|N|N|N|N|N| +2469730|AAAAAAAACGPKFCAA|2049-10-15|1797|7816|600|2049|5|10|15|4|2049|600|7816|Friday|2049Q4|N|Y|N|2469716|2469988|2469365|2469638|N|N|N|N|N| +2469731|AAAAAAAADGPKFCAA|2049-10-16|1797|7816|600|2049|6|10|16|4|2049|600|7816|Saturday|2049Q4|N|Y|N|2469716|2469988|2469366|2469639|N|N|N|N|N| +2469732|AAAAAAAAEGPKFCAA|2049-10-17|1797|7816|600|2049|0|10|17|4|2049|600|7816|Sunday|2049Q4|N|N|N|2469716|2469988|2469367|2469640|N|N|N|N|N| +2469733|AAAAAAAAFGPKFCAA|2049-10-18|1797|7816|600|2049|1|10|18|4|2049|600|7816|Monday|2049Q4|N|N|N|2469716|2469988|2469368|2469641|N|N|N|N|N| +2469734|AAAAAAAAGGPKFCAA|2049-10-19|1797|7817|600|2049|2|10|19|4|2049|600|7817|Tuesday|2049Q4|N|N|N|2469716|2469988|2469369|2469642|N|N|N|N|N| +2469735|AAAAAAAAHGPKFCAA|2049-10-20|1797|7817|600|2049|3|10|20|4|2049|600|7817|Wednesday|2049Q4|N|N|N|2469716|2469988|2469370|2469643|N|N|N|N|N| +2469736|AAAAAAAAIGPKFCAA|2049-10-21|1797|7817|600|2049|4|10|21|4|2049|600|7817|Thursday|2049Q4|N|N|N|2469716|2469988|2469371|2469644|N|N|N|N|N| +2469737|AAAAAAAAJGPKFCAA|2049-10-22|1797|7817|600|2049|5|10|22|4|2049|600|7817|Friday|2049Q4|N|Y|N|2469716|2469988|2469372|2469645|N|N|N|N|N| +2469738|AAAAAAAAKGPKFCAA|2049-10-23|1797|7817|600|2049|6|10|23|4|2049|600|7817|Saturday|2049Q4|N|Y|N|2469716|2469988|2469373|2469646|N|N|N|N|N| +2469739|AAAAAAAALGPKFCAA|2049-10-24|1797|7817|600|2049|0|10|24|4|2049|600|7817|Sunday|2049Q4|N|N|N|2469716|2469988|2469374|2469647|N|N|N|N|N| +2469740|AAAAAAAAMGPKFCAA|2049-10-25|1797|7817|600|2049|1|10|25|4|2049|600|7817|Monday|2049Q4|N|N|N|2469716|2469988|2469375|2469648|N|N|N|N|N| +2469741|AAAAAAAANGPKFCAA|2049-10-26|1797|7818|600|2049|2|10|26|4|2049|600|7818|Tuesday|2049Q4|N|N|N|2469716|2469988|2469376|2469649|N|N|N|N|N| +2469742|AAAAAAAAOGPKFCAA|2049-10-27|1797|7818|600|2049|3|10|27|4|2049|600|7818|Wednesday|2049Q4|N|N|N|2469716|2469988|2469377|2469650|N|N|N|N|N| +2469743|AAAAAAAAPGPKFCAA|2049-10-28|1797|7818|600|2049|4|10|28|4|2049|600|7818|Thursday|2049Q4|N|N|N|2469716|2469988|2469378|2469651|N|N|N|N|N| +2469744|AAAAAAAAAHPKFCAA|2049-10-29|1797|7818|600|2049|5|10|29|4|2049|600|7818|Friday|2049Q4|N|Y|N|2469716|2469988|2469379|2469652|N|N|N|N|N| +2469745|AAAAAAAABHPKFCAA|2049-10-30|1797|7818|600|2049|6|10|30|4|2049|600|7818|Saturday|2049Q4|N|Y|N|2469716|2469988|2469380|2469653|N|N|N|N|N| +2469746|AAAAAAAACHPKFCAA|2049-10-31|1797|7818|600|2049|0|10|31|4|2049|600|7818|Sunday|2049Q4|N|N|N|2469716|2469988|2469381|2469654|N|N|N|N|N| +2469747|AAAAAAAADHPKFCAA|2049-11-01|1798|7818|600|2049|1|11|1|4|2049|600|7818|Monday|2049Q4|N|N|N|2469747|2470050|2469382|2469655|N|N|N|N|N| +2469748|AAAAAAAAEHPKFCAA|2049-11-02|1798|7819|600|2049|2|11|2|4|2049|600|7819|Tuesday|2049Q4|N|N|N|2469747|2470050|2469383|2469656|N|N|N|N|N| +2469749|AAAAAAAAFHPKFCAA|2049-11-03|1798|7819|600|2049|3|11|3|4|2049|600|7819|Wednesday|2049Q4|N|N|N|2469747|2470050|2469384|2469657|N|N|N|N|N| +2469750|AAAAAAAAGHPKFCAA|2049-11-04|1798|7819|600|2049|4|11|4|4|2049|600|7819|Thursday|2049Q4|N|N|N|2469747|2470050|2469385|2469658|N|N|N|N|N| +2469751|AAAAAAAAHHPKFCAA|2049-11-05|1798|7819|600|2049|5|11|5|4|2049|600|7819|Friday|2049Q4|N|Y|N|2469747|2470050|2469386|2469659|N|N|N|N|N| +2469752|AAAAAAAAIHPKFCAA|2049-11-06|1798|7819|600|2049|6|11|6|4|2049|600|7819|Saturday|2049Q4|N|Y|N|2469747|2470050|2469387|2469660|N|N|N|N|N| +2469753|AAAAAAAAJHPKFCAA|2049-11-07|1798|7819|600|2049|0|11|7|4|2049|600|7819|Sunday|2049Q4|N|N|N|2469747|2470050|2469388|2469661|N|N|N|N|N| +2469754|AAAAAAAAKHPKFCAA|2049-11-08|1798|7819|600|2049|1|11|8|4|2049|600|7819|Monday|2049Q4|N|N|N|2469747|2470050|2469389|2469662|N|N|N|N|N| +2469755|AAAAAAAALHPKFCAA|2049-11-09|1798|7820|600|2049|2|11|9|4|2049|600|7820|Tuesday|2049Q4|N|N|N|2469747|2470050|2469390|2469663|N|N|N|N|N| +2469756|AAAAAAAAMHPKFCAA|2049-11-10|1798|7820|600|2049|3|11|10|4|2049|600|7820|Wednesday|2049Q4|N|N|N|2469747|2470050|2469391|2469664|N|N|N|N|N| +2469757|AAAAAAAANHPKFCAA|2049-11-11|1798|7820|600|2049|4|11|11|4|2049|600|7820|Thursday|2049Q4|N|N|N|2469747|2470050|2469392|2469665|N|N|N|N|N| +2469758|AAAAAAAAOHPKFCAA|2049-11-12|1798|7820|600|2049|5|11|12|4|2049|600|7820|Friday|2049Q4|N|Y|N|2469747|2470050|2469393|2469666|N|N|N|N|N| +2469759|AAAAAAAAPHPKFCAA|2049-11-13|1798|7820|600|2049|6|11|13|4|2049|600|7820|Saturday|2049Q4|N|Y|N|2469747|2470050|2469394|2469667|N|N|N|N|N| +2469760|AAAAAAAAAIPKFCAA|2049-11-14|1798|7820|600|2049|0|11|14|4|2049|600|7820|Sunday|2049Q4|N|N|N|2469747|2470050|2469395|2469668|N|N|N|N|N| +2469761|AAAAAAAABIPKFCAA|2049-11-15|1798|7820|600|2049|1|11|15|4|2049|600|7820|Monday|2049Q4|N|N|N|2469747|2470050|2469396|2469669|N|N|N|N|N| +2469762|AAAAAAAACIPKFCAA|2049-11-16|1798|7821|600|2049|2|11|16|4|2049|600|7821|Tuesday|2049Q4|N|N|N|2469747|2470050|2469397|2469670|N|N|N|N|N| +2469763|AAAAAAAADIPKFCAA|2049-11-17|1798|7821|600|2049|3|11|17|4|2049|600|7821|Wednesday|2049Q4|N|N|N|2469747|2470050|2469398|2469671|N|N|N|N|N| +2469764|AAAAAAAAEIPKFCAA|2049-11-18|1798|7821|600|2049|4|11|18|4|2049|600|7821|Thursday|2049Q4|N|N|N|2469747|2470050|2469399|2469672|N|N|N|N|N| +2469765|AAAAAAAAFIPKFCAA|2049-11-19|1798|7821|600|2049|5|11|19|4|2049|600|7821|Friday|2049Q4|N|Y|N|2469747|2470050|2469400|2469673|N|N|N|N|N| +2469766|AAAAAAAAGIPKFCAA|2049-11-20|1798|7821|600|2049|6|11|20|4|2049|600|7821|Saturday|2049Q4|N|Y|N|2469747|2470050|2469401|2469674|N|N|N|N|N| +2469767|AAAAAAAAHIPKFCAA|2049-11-21|1798|7821|600|2049|0|11|21|4|2049|600|7821|Sunday|2049Q4|N|N|N|2469747|2470050|2469402|2469675|N|N|N|N|N| +2469768|AAAAAAAAIIPKFCAA|2049-11-22|1798|7821|600|2049|1|11|22|4|2049|600|7821|Monday|2049Q4|N|N|N|2469747|2470050|2469403|2469676|N|N|N|N|N| +2469769|AAAAAAAAJIPKFCAA|2049-11-23|1798|7822|600|2049|2|11|23|4|2049|600|7822|Tuesday|2049Q4|N|N|N|2469747|2470050|2469404|2469677|N|N|N|N|N| +2469770|AAAAAAAAKIPKFCAA|2049-11-24|1798|7822|600|2049|3|11|24|4|2049|600|7822|Wednesday|2049Q4|N|N|N|2469747|2470050|2469405|2469678|N|N|N|N|N| +2469771|AAAAAAAALIPKFCAA|2049-11-25|1798|7822|600|2049|4|11|25|4|2049|600|7822|Thursday|2049Q4|N|N|N|2469747|2470050|2469406|2469679|N|N|N|N|N| +2469772|AAAAAAAAMIPKFCAA|2049-11-26|1798|7822|600|2049|5|11|26|4|2049|600|7822|Friday|2049Q4|N|Y|N|2469747|2470050|2469407|2469680|N|N|N|N|N| +2469773|AAAAAAAANIPKFCAA|2049-11-27|1798|7822|600|2049|6|11|27|4|2049|600|7822|Saturday|2049Q4|N|Y|N|2469747|2470050|2469408|2469681|N|N|N|N|N| +2469774|AAAAAAAAOIPKFCAA|2049-11-28|1798|7822|600|2049|0|11|28|4|2049|600|7822|Sunday|2049Q4|N|N|N|2469747|2470050|2469409|2469682|N|N|N|N|N| +2469775|AAAAAAAAPIPKFCAA|2049-11-29|1798|7822|600|2049|1|11|29|4|2049|600|7822|Monday|2049Q4|N|N|N|2469747|2470050|2469410|2469683|N|N|N|N|N| +2469776|AAAAAAAAAJPKFCAA|2049-11-30|1798|7823|600|2049|2|11|30|4|2049|600|7823|Tuesday|2049Q4|N|N|N|2469747|2470050|2469411|2469684|N|N|N|N|N| +2469777|AAAAAAAABJPKFCAA|2049-12-01|1799|7823|601|2049|3|12|1|4|2049|601|7823|Wednesday|2049Q4|N|N|N|2469777|2470110|2469412|2469685|N|N|N|N|N| +2469778|AAAAAAAACJPKFCAA|2049-12-02|1799|7823|601|2049|4|12|2|4|2049|601|7823|Thursday|2049Q4|N|N|N|2469777|2470110|2469413|2469686|N|N|N|N|N| +2469779|AAAAAAAADJPKFCAA|2049-12-03|1799|7823|601|2049|5|12|3|4|2049|601|7823|Friday|2049Q4|N|Y|N|2469777|2470110|2469414|2469687|N|N|N|N|N| +2469780|AAAAAAAAEJPKFCAA|2049-12-04|1799|7823|601|2049|6|12|4|4|2049|601|7823|Saturday|2049Q4|N|Y|N|2469777|2470110|2469415|2469688|N|N|N|N|N| +2469781|AAAAAAAAFJPKFCAA|2049-12-05|1799|7823|601|2049|0|12|5|4|2049|601|7823|Sunday|2049Q4|N|N|N|2469777|2470110|2469416|2469689|N|N|N|N|N| +2469782|AAAAAAAAGJPKFCAA|2049-12-06|1799|7823|601|2049|1|12|6|4|2049|601|7823|Monday|2049Q4|N|N|N|2469777|2470110|2469417|2469690|N|N|N|N|N| +2469783|AAAAAAAAHJPKFCAA|2049-12-07|1799|7824|601|2049|2|12|7|4|2049|601|7824|Tuesday|2049Q4|N|N|N|2469777|2470110|2469418|2469691|N|N|N|N|N| +2469784|AAAAAAAAIJPKFCAA|2049-12-08|1799|7824|601|2049|3|12|8|4|2049|601|7824|Wednesday|2049Q4|N|N|N|2469777|2470110|2469419|2469692|N|N|N|N|N| +2469785|AAAAAAAAJJPKFCAA|2049-12-09|1799|7824|601|2049|4|12|9|4|2049|601|7824|Thursday|2049Q4|N|N|N|2469777|2470110|2469420|2469693|N|N|N|N|N| +2469786|AAAAAAAAKJPKFCAA|2049-12-10|1799|7824|601|2049|5|12|10|4|2049|601|7824|Friday|2049Q4|N|Y|N|2469777|2470110|2469421|2469694|N|N|N|N|N| +2469787|AAAAAAAALJPKFCAA|2049-12-11|1799|7824|601|2049|6|12|11|4|2049|601|7824|Saturday|2049Q4|N|Y|N|2469777|2470110|2469422|2469695|N|N|N|N|N| +2469788|AAAAAAAAMJPKFCAA|2049-12-12|1799|7824|601|2049|0|12|12|4|2049|601|7824|Sunday|2049Q4|N|N|N|2469777|2470110|2469423|2469696|N|N|N|N|N| +2469789|AAAAAAAANJPKFCAA|2049-12-13|1799|7824|601|2049|1|12|13|4|2049|601|7824|Monday|2049Q4|N|N|N|2469777|2470110|2469424|2469697|N|N|N|N|N| +2469790|AAAAAAAAOJPKFCAA|2049-12-14|1799|7825|601|2049|2|12|14|4|2049|601|7825|Tuesday|2049Q4|N|N|N|2469777|2470110|2469425|2469698|N|N|N|N|N| +2469791|AAAAAAAAPJPKFCAA|2049-12-15|1799|7825|601|2049|3|12|15|4|2049|601|7825|Wednesday|2049Q4|N|N|N|2469777|2470110|2469426|2469699|N|N|N|N|N| +2469792|AAAAAAAAAKPKFCAA|2049-12-16|1799|7825|601|2049|4|12|16|4|2049|601|7825|Thursday|2049Q4|N|N|N|2469777|2470110|2469427|2469700|N|N|N|N|N| +2469793|AAAAAAAABKPKFCAA|2049-12-17|1799|7825|601|2049|5|12|17|4|2049|601|7825|Friday|2049Q4|N|Y|N|2469777|2470110|2469428|2469701|N|N|N|N|N| +2469794|AAAAAAAACKPKFCAA|2049-12-18|1799|7825|601|2049|6|12|18|4|2049|601|7825|Saturday|2049Q4|N|Y|N|2469777|2470110|2469429|2469702|N|N|N|N|N| +2469795|AAAAAAAADKPKFCAA|2049-12-19|1799|7825|601|2049|0|12|19|4|2049|601|7825|Sunday|2049Q4|N|N|N|2469777|2470110|2469430|2469703|N|N|N|N|N| +2469796|AAAAAAAAEKPKFCAA|2049-12-20|1799|7825|601|2049|1|12|20|4|2049|601|7825|Monday|2049Q4|N|N|N|2469777|2470110|2469431|2469704|N|N|N|N|N| +2469797|AAAAAAAAFKPKFCAA|2049-12-21|1799|7826|601|2049|2|12|21|4|2049|601|7826|Tuesday|2049Q4|N|N|N|2469777|2470110|2469432|2469705|N|N|N|N|N| +2469798|AAAAAAAAGKPKFCAA|2049-12-22|1799|7826|601|2049|3|12|22|4|2049|601|7826|Wednesday|2049Q4|N|N|N|2469777|2470110|2469433|2469706|N|N|N|N|N| +2469799|AAAAAAAAHKPKFCAA|2049-12-23|1799|7826|601|2049|4|12|23|4|2049|601|7826|Thursday|2049Q4|N|N|N|2469777|2470110|2469434|2469707|N|N|N|N|N| +2469800|AAAAAAAAIKPKFCAA|2049-12-24|1799|7826|601|2049|5|12|24|4|2049|601|7826|Friday|2049Q4|N|Y|N|2469777|2470110|2469435|2469708|N|N|N|N|N| +2469801|AAAAAAAAJKPKFCAA|2049-12-25|1799|7826|601|2049|6|12|25|4|2049|601|7826|Saturday|2049Q4|N|Y|N|2469777|2470110|2469436|2469709|N|N|N|N|N| +2469802|AAAAAAAAKKPKFCAA|2049-12-26|1799|7826|601|2049|0|12|26|4|2049|601|7826|Sunday|2049Q4|Y|N|N|2469777|2470110|2469437|2469710|N|N|N|N|N| +2469803|AAAAAAAALKPKFCAA|2049-12-27|1799|7826|601|2049|1|12|27|4|2049|601|7826|Monday|2049Q4|N|N|Y|2469777|2470110|2469438|2469711|N|N|N|N|N| +2469804|AAAAAAAAMKPKFCAA|2049-12-28|1799|7827|601|2049|2|12|28|4|2049|601|7827|Tuesday|2049Q4|N|N|N|2469777|2470110|2469439|2469712|N|N|N|N|N| +2469805|AAAAAAAANKPKFCAA|2049-12-29|1799|7827|601|2049|3|12|29|4|2049|601|7827|Wednesday|2049Q4|N|N|N|2469777|2470110|2469440|2469713|N|N|N|N|N| +2469806|AAAAAAAAOKPKFCAA|2049-12-30|1799|7827|601|2049|4|12|30|4|2049|601|7827|Thursday|2049Q4|N|N|N|2469777|2470110|2469441|2469714|N|N|N|N|N| +2469807|AAAAAAAAPKPKFCAA|2049-12-31|1799|7827|601|2049|5|12|31|4|2049|601|7827|Friday|2049Q4|N|Y|N|2469777|2470110|2469442|2469715|N|N|N|N|N| +2469808|AAAAAAAAALPKFCAA|2050-01-01|1800|7827|601|2050|6|1|1|1|2050|601|7827|Saturday|2050Q1|Y|Y|N|2469808|2469807|2469443|2469716|N|N|N|N|N| +2469809|AAAAAAAABLPKFCAA|2050-01-02|1800|7827|601|2050|0|1|2|1|2050|601|7827|Sunday|2050Q1|N|N|Y|2469808|2469807|2469444|2469717|N|N|N|N|N| +2469810|AAAAAAAACLPKFCAA|2050-01-03|1800|7827|601|2050|1|1|3|1|2050|601|7827|Monday|2050Q1|N|N|N|2469808|2469807|2469445|2469718|N|N|N|N|N| +2469811|AAAAAAAADLPKFCAA|2050-01-04|1800|7828|601|2050|2|1|4|1|2050|601|7828|Tuesday|2050Q1|N|N|N|2469808|2469807|2469446|2469719|N|N|N|N|N| +2469812|AAAAAAAAELPKFCAA|2050-01-05|1800|7828|601|2050|3|1|5|1|2050|601|7828|Wednesday|2050Q1|N|N|N|2469808|2469807|2469447|2469720|N|N|N|N|N| +2469813|AAAAAAAAFLPKFCAA|2050-01-06|1800|7828|601|2050|4|1|6|1|2050|601|7828|Thursday|2050Q1|N|N|N|2469808|2469807|2469448|2469721|N|N|N|N|N| +2469814|AAAAAAAAGLPKFCAA|2050-01-07|1800|7828|601|2050|5|1|7|1|2050|601|7828|Friday|2050Q1|N|Y|N|2469808|2469807|2469449|2469722|N|N|N|N|N| +2469815|AAAAAAAAHLPKFCAA|2050-01-08|1800|7828|601|2050|6|1|8|1|2050|601|7828|Saturday|2050Q1|N|Y|N|2469808|2469807|2469450|2469723|N|N|N|N|N| +2469816|AAAAAAAAILPKFCAA|2050-01-09|1800|7828|601|2050|0|1|9|1|2050|601|7828|Sunday|2050Q1|N|N|N|2469808|2469807|2469451|2469724|N|N|N|N|N| +2469817|AAAAAAAAJLPKFCAA|2050-01-10|1800|7828|601|2050|1|1|10|1|2050|601|7828|Monday|2050Q1|N|N|N|2469808|2469807|2469452|2469725|N|N|N|N|N| +2469818|AAAAAAAAKLPKFCAA|2050-01-11|1800|7829|601|2050|2|1|11|1|2050|601|7829|Tuesday|2050Q1|N|N|N|2469808|2469807|2469453|2469726|N|N|N|N|N| +2469819|AAAAAAAALLPKFCAA|2050-01-12|1800|7829|601|2050|3|1|12|1|2050|601|7829|Wednesday|2050Q1|N|N|N|2469808|2469807|2469454|2469727|N|N|N|N|N| +2469820|AAAAAAAAMLPKFCAA|2050-01-13|1800|7829|601|2050|4|1|13|1|2050|601|7829|Thursday|2050Q1|N|N|N|2469808|2469807|2469455|2469728|N|N|N|N|N| +2469821|AAAAAAAANLPKFCAA|2050-01-14|1800|7829|601|2050|5|1|14|1|2050|601|7829|Friday|2050Q1|N|Y|N|2469808|2469807|2469456|2469729|N|N|N|N|N| +2469822|AAAAAAAAOLPKFCAA|2050-01-15|1800|7829|601|2050|6|1|15|1|2050|601|7829|Saturday|2050Q1|N|Y|N|2469808|2469807|2469457|2469730|N|N|N|N|N| +2469823|AAAAAAAAPLPKFCAA|2050-01-16|1800|7829|601|2050|0|1|16|1|2050|601|7829|Sunday|2050Q1|N|N|N|2469808|2469807|2469458|2469731|N|N|N|N|N| +2469824|AAAAAAAAAMPKFCAA|2050-01-17|1800|7829|601|2050|1|1|17|1|2050|601|7829|Monday|2050Q1|N|N|N|2469808|2469807|2469459|2469732|N|N|N|N|N| +2469825|AAAAAAAABMPKFCAA|2050-01-18|1800|7830|601|2050|2|1|18|1|2050|601|7830|Tuesday|2050Q1|N|N|N|2469808|2469807|2469460|2469733|N|N|N|N|N| +2469826|AAAAAAAACMPKFCAA|2050-01-19|1800|7830|601|2050|3|1|19|1|2050|601|7830|Wednesday|2050Q1|N|N|N|2469808|2469807|2469461|2469734|N|N|N|N|N| +2469827|AAAAAAAADMPKFCAA|2050-01-20|1800|7830|601|2050|4|1|20|1|2050|601|7830|Thursday|2050Q1|N|N|N|2469808|2469807|2469462|2469735|N|N|N|N|N| +2469828|AAAAAAAAEMPKFCAA|2050-01-21|1800|7830|601|2050|5|1|21|1|2050|601|7830|Friday|2050Q1|N|Y|N|2469808|2469807|2469463|2469736|N|N|N|N|N| +2469829|AAAAAAAAFMPKFCAA|2050-01-22|1800|7830|601|2050|6|1|22|1|2050|601|7830|Saturday|2050Q1|N|Y|N|2469808|2469807|2469464|2469737|N|N|N|N|N| +2469830|AAAAAAAAGMPKFCAA|2050-01-23|1800|7830|601|2050|0|1|23|1|2050|601|7830|Sunday|2050Q1|N|N|N|2469808|2469807|2469465|2469738|N|N|N|N|N| +2469831|AAAAAAAAHMPKFCAA|2050-01-24|1800|7830|601|2050|1|1|24|1|2050|601|7830|Monday|2050Q1|N|N|N|2469808|2469807|2469466|2469739|N|N|N|N|N| +2469832|AAAAAAAAIMPKFCAA|2050-01-25|1800|7831|601|2050|2|1|25|1|2050|601|7831|Tuesday|2050Q1|N|N|N|2469808|2469807|2469467|2469740|N|N|N|N|N| +2469833|AAAAAAAAJMPKFCAA|2050-01-26|1800|7831|601|2050|3|1|26|1|2050|601|7831|Wednesday|2050Q1|N|N|N|2469808|2469807|2469468|2469741|N|N|N|N|N| +2469834|AAAAAAAAKMPKFCAA|2050-01-27|1800|7831|601|2050|4|1|27|1|2050|601|7831|Thursday|2050Q1|N|N|N|2469808|2469807|2469469|2469742|N|N|N|N|N| +2469835|AAAAAAAALMPKFCAA|2050-01-28|1800|7831|601|2050|5|1|28|1|2050|601|7831|Friday|2050Q1|N|Y|N|2469808|2469807|2469470|2469743|N|N|N|N|N| +2469836|AAAAAAAAMMPKFCAA|2050-01-29|1800|7831|601|2050|6|1|29|1|2050|601|7831|Saturday|2050Q1|N|Y|N|2469808|2469807|2469471|2469744|N|N|N|N|N| +2469837|AAAAAAAANMPKFCAA|2050-01-30|1800|7831|601|2050|0|1|30|1|2050|601|7831|Sunday|2050Q1|N|N|N|2469808|2469807|2469472|2469745|N|N|N|N|N| +2469838|AAAAAAAAOMPKFCAA|2050-01-31|1800|7831|601|2050|1|1|31|1|2050|601|7831|Monday|2050Q1|N|N|N|2469808|2469807|2469473|2469746|N|N|N|N|N| +2469839|AAAAAAAAPMPKFCAA|2050-02-01|1801|7832|601|2050|2|2|1|1|2050|601|7832|Tuesday|2050Q1|N|N|N|2469839|2469869|2469474|2469747|N|N|N|N|N| +2469840|AAAAAAAAANPKFCAA|2050-02-02|1801|7832|601|2050|3|2|2|1|2050|601|7832|Wednesday|2050Q1|N|N|N|2469839|2469869|2469475|2469748|N|N|N|N|N| +2469841|AAAAAAAABNPKFCAA|2050-02-03|1801|7832|601|2050|4|2|3|1|2050|601|7832|Thursday|2050Q1|N|N|N|2469839|2469869|2469476|2469749|N|N|N|N|N| +2469842|AAAAAAAACNPKFCAA|2050-02-04|1801|7832|601|2050|5|2|4|1|2050|601|7832|Friday|2050Q1|N|Y|N|2469839|2469869|2469477|2469750|N|N|N|N|N| +2469843|AAAAAAAADNPKFCAA|2050-02-05|1801|7832|601|2050|6|2|5|1|2050|601|7832|Saturday|2050Q1|N|Y|N|2469839|2469869|2469478|2469751|N|N|N|N|N| +2469844|AAAAAAAAENPKFCAA|2050-02-06|1801|7832|601|2050|0|2|6|1|2050|601|7832|Sunday|2050Q1|N|N|N|2469839|2469869|2469479|2469752|N|N|N|N|N| +2469845|AAAAAAAAFNPKFCAA|2050-02-07|1801|7832|601|2050|1|2|7|1|2050|601|7832|Monday|2050Q1|N|N|N|2469839|2469869|2469480|2469753|N|N|N|N|N| +2469846|AAAAAAAAGNPKFCAA|2050-02-08|1801|7833|601|2050|2|2|8|1|2050|601|7833|Tuesday|2050Q1|N|N|N|2469839|2469869|2469481|2469754|N|N|N|N|N| +2469847|AAAAAAAAHNPKFCAA|2050-02-09|1801|7833|601|2050|3|2|9|1|2050|601|7833|Wednesday|2050Q1|N|N|N|2469839|2469869|2469482|2469755|N|N|N|N|N| +2469848|AAAAAAAAINPKFCAA|2050-02-10|1801|7833|601|2050|4|2|10|1|2050|601|7833|Thursday|2050Q1|N|N|N|2469839|2469869|2469483|2469756|N|N|N|N|N| +2469849|AAAAAAAAJNPKFCAA|2050-02-11|1801|7833|601|2050|5|2|11|1|2050|601|7833|Friday|2050Q1|N|Y|N|2469839|2469869|2469484|2469757|N|N|N|N|N| +2469850|AAAAAAAAKNPKFCAA|2050-02-12|1801|7833|601|2050|6|2|12|1|2050|601|7833|Saturday|2050Q1|N|Y|N|2469839|2469869|2469485|2469758|N|N|N|N|N| +2469851|AAAAAAAALNPKFCAA|2050-02-13|1801|7833|601|2050|0|2|13|1|2050|601|7833|Sunday|2050Q1|N|N|N|2469839|2469869|2469486|2469759|N|N|N|N|N| +2469852|AAAAAAAAMNPKFCAA|2050-02-14|1801|7833|601|2050|1|2|14|1|2050|601|7833|Monday|2050Q1|N|N|N|2469839|2469869|2469487|2469760|N|N|N|N|N| +2469853|AAAAAAAANNPKFCAA|2050-02-15|1801|7834|601|2050|2|2|15|1|2050|601|7834|Tuesday|2050Q1|N|N|N|2469839|2469869|2469488|2469761|N|N|N|N|N| +2469854|AAAAAAAAONPKFCAA|2050-02-16|1801|7834|601|2050|3|2|16|1|2050|601|7834|Wednesday|2050Q1|N|N|N|2469839|2469869|2469489|2469762|N|N|N|N|N| +2469855|AAAAAAAAPNPKFCAA|2050-02-17|1801|7834|601|2050|4|2|17|1|2050|601|7834|Thursday|2050Q1|N|N|N|2469839|2469869|2469490|2469763|N|N|N|N|N| +2469856|AAAAAAAAAOPKFCAA|2050-02-18|1801|7834|601|2050|5|2|18|1|2050|601|7834|Friday|2050Q1|N|Y|N|2469839|2469869|2469491|2469764|N|N|N|N|N| +2469857|AAAAAAAABOPKFCAA|2050-02-19|1801|7834|601|2050|6|2|19|1|2050|601|7834|Saturday|2050Q1|N|Y|N|2469839|2469869|2469492|2469765|N|N|N|N|N| +2469858|AAAAAAAACOPKFCAA|2050-02-20|1801|7834|601|2050|0|2|20|1|2050|601|7834|Sunday|2050Q1|N|N|N|2469839|2469869|2469493|2469766|N|N|N|N|N| +2469859|AAAAAAAADOPKFCAA|2050-02-21|1801|7834|601|2050|1|2|21|1|2050|601|7834|Monday|2050Q1|N|N|N|2469839|2469869|2469494|2469767|N|N|N|N|N| +2469860|AAAAAAAAEOPKFCAA|2050-02-22|1801|7835|601|2050|2|2|22|1|2050|601|7835|Tuesday|2050Q1|N|N|N|2469839|2469869|2469495|2469768|N|N|N|N|N| +2469861|AAAAAAAAFOPKFCAA|2050-02-23|1801|7835|601|2050|3|2|23|1|2050|601|7835|Wednesday|2050Q1|N|N|N|2469839|2469869|2469496|2469769|N|N|N|N|N| +2469862|AAAAAAAAGOPKFCAA|2050-02-24|1801|7835|601|2050|4|2|24|1|2050|601|7835|Thursday|2050Q1|N|N|N|2469839|2469869|2469497|2469770|N|N|N|N|N| +2469863|AAAAAAAAHOPKFCAA|2050-02-25|1801|7835|601|2050|5|2|25|1|2050|601|7835|Friday|2050Q1|N|Y|N|2469839|2469869|2469498|2469771|N|N|N|N|N| +2469864|AAAAAAAAIOPKFCAA|2050-02-26|1801|7835|601|2050|6|2|26|1|2050|601|7835|Saturday|2050Q1|N|Y|N|2469839|2469869|2469499|2469772|N|N|N|N|N| +2469865|AAAAAAAAJOPKFCAA|2050-02-27|1801|7835|601|2050|0|2|27|1|2050|601|7835|Sunday|2050Q1|N|N|N|2469839|2469869|2469500|2469773|N|N|N|N|N| +2469866|AAAAAAAAKOPKFCAA|2050-02-28|1801|7835|601|2050|1|2|28|1|2050|601|7835|Monday|2050Q1|N|N|N|2469839|2469869|2469501|2469774|N|N|N|N|N| +2469867|AAAAAAAALOPKFCAA|2050-03-01|1802|7836|602|2050|2|3|1|1|2050|602|7836|Tuesday|2050Q1|N|N|N|2469867|2469925|2469502|2469775|N|N|N|N|N| +2469868|AAAAAAAAMOPKFCAA|2050-03-02|1802|7836|602|2050|3|3|2|1|2050|602|7836|Wednesday|2050Q1|N|N|N|2469867|2469925|2469503|2469776|N|N|N|N|N| +2469869|AAAAAAAANOPKFCAA|2050-03-03|1802|7836|602|2050|4|3|3|1|2050|602|7836|Thursday|2050Q1|N|N|N|2469867|2469925|2469504|2469777|N|N|N|N|N| +2469870|AAAAAAAAOOPKFCAA|2050-03-04|1802|7836|602|2050|5|3|4|1|2050|602|7836|Friday|2050Q1|N|Y|N|2469867|2469925|2469505|2469778|N|N|N|N|N| +2469871|AAAAAAAAPOPKFCAA|2050-03-05|1802|7836|602|2050|6|3|5|1|2050|602|7836|Saturday|2050Q1|N|Y|N|2469867|2469925|2469506|2469779|N|N|N|N|N| +2469872|AAAAAAAAAPPKFCAA|2050-03-06|1802|7836|602|2050|0|3|6|1|2050|602|7836|Sunday|2050Q1|N|N|N|2469867|2469925|2469507|2469780|N|N|N|N|N| +2469873|AAAAAAAABPPKFCAA|2050-03-07|1802|7836|602|2050|1|3|7|1|2050|602|7836|Monday|2050Q1|N|N|N|2469867|2469925|2469508|2469781|N|N|N|N|N| +2469874|AAAAAAAACPPKFCAA|2050-03-08|1802|7837|602|2050|2|3|8|1|2050|602|7837|Tuesday|2050Q1|N|N|N|2469867|2469925|2469509|2469782|N|N|N|N|N| +2469875|AAAAAAAADPPKFCAA|2050-03-09|1802|7837|602|2050|3|3|9|1|2050|602|7837|Wednesday|2050Q1|N|N|N|2469867|2469925|2469510|2469783|N|N|N|N|N| +2469876|AAAAAAAAEPPKFCAA|2050-03-10|1802|7837|602|2050|4|3|10|1|2050|602|7837|Thursday|2050Q1|N|N|N|2469867|2469925|2469511|2469784|N|N|N|N|N| +2469877|AAAAAAAAFPPKFCAA|2050-03-11|1802|7837|602|2050|5|3|11|1|2050|602|7837|Friday|2050Q1|N|Y|N|2469867|2469925|2469512|2469785|N|N|N|N|N| +2469878|AAAAAAAAGPPKFCAA|2050-03-12|1802|7837|602|2050|6|3|12|1|2050|602|7837|Saturday|2050Q1|N|Y|N|2469867|2469925|2469513|2469786|N|N|N|N|N| +2469879|AAAAAAAAHPPKFCAA|2050-03-13|1802|7837|602|2050|0|3|13|1|2050|602|7837|Sunday|2050Q1|N|N|N|2469867|2469925|2469514|2469787|N|N|N|N|N| +2469880|AAAAAAAAIPPKFCAA|2050-03-14|1802|7837|602|2050|1|3|14|1|2050|602|7837|Monday|2050Q1|N|N|N|2469867|2469925|2469515|2469788|N|N|N|N|N| +2469881|AAAAAAAAJPPKFCAA|2050-03-15|1802|7838|602|2050|2|3|15|1|2050|602|7838|Tuesday|2050Q1|N|N|N|2469867|2469925|2469516|2469789|N|N|N|N|N| +2469882|AAAAAAAAKPPKFCAA|2050-03-16|1802|7838|602|2050|3|3|16|1|2050|602|7838|Wednesday|2050Q1|N|N|N|2469867|2469925|2469517|2469790|N|N|N|N|N| +2469883|AAAAAAAALPPKFCAA|2050-03-17|1802|7838|602|2050|4|3|17|1|2050|602|7838|Thursday|2050Q1|N|N|N|2469867|2469925|2469518|2469791|N|N|N|N|N| +2469884|AAAAAAAAMPPKFCAA|2050-03-18|1802|7838|602|2050|5|3|18|1|2050|602|7838|Friday|2050Q1|N|Y|N|2469867|2469925|2469519|2469792|N|N|N|N|N| +2469885|AAAAAAAANPPKFCAA|2050-03-19|1802|7838|602|2050|6|3|19|1|2050|602|7838|Saturday|2050Q1|N|Y|N|2469867|2469925|2469520|2469793|N|N|N|N|N| +2469886|AAAAAAAAOPPKFCAA|2050-03-20|1802|7838|602|2050|0|3|20|1|2050|602|7838|Sunday|2050Q1|N|N|N|2469867|2469925|2469521|2469794|N|N|N|N|N| +2469887|AAAAAAAAPPPKFCAA|2050-03-21|1802|7838|602|2050|1|3|21|1|2050|602|7838|Monday|2050Q1|N|N|N|2469867|2469925|2469522|2469795|N|N|N|N|N| +2469888|AAAAAAAAAAALFCAA|2050-03-22|1802|7839|602|2050|2|3|22|1|2050|602|7839|Tuesday|2050Q1|N|N|N|2469867|2469925|2469523|2469796|N|N|N|N|N| +2469889|AAAAAAAABAALFCAA|2050-03-23|1802|7839|602|2050|3|3|23|1|2050|602|7839|Wednesday|2050Q1|N|N|N|2469867|2469925|2469524|2469797|N|N|N|N|N| +2469890|AAAAAAAACAALFCAA|2050-03-24|1802|7839|602|2050|4|3|24|1|2050|602|7839|Thursday|2050Q1|N|N|N|2469867|2469925|2469525|2469798|N|N|N|N|N| +2469891|AAAAAAAADAALFCAA|2050-03-25|1802|7839|602|2050|5|3|25|1|2050|602|7839|Friday|2050Q1|N|Y|N|2469867|2469925|2469526|2469799|N|N|N|N|N| +2469892|AAAAAAAAEAALFCAA|2050-03-26|1802|7839|602|2050|6|3|26|1|2050|602|7839|Saturday|2050Q1|N|Y|N|2469867|2469925|2469527|2469800|N|N|N|N|N| +2469893|AAAAAAAAFAALFCAA|2050-03-27|1802|7839|602|2050|0|3|27|1|2050|602|7839|Sunday|2050Q1|N|N|N|2469867|2469925|2469528|2469801|N|N|N|N|N| +2469894|AAAAAAAAGAALFCAA|2050-03-28|1802|7839|602|2050|1|3|28|1|2050|602|7839|Monday|2050Q1|N|N|N|2469867|2469925|2469529|2469802|N|N|N|N|N| +2469895|AAAAAAAAHAALFCAA|2050-03-29|1802|7840|602|2050|2|3|29|1|2050|602|7840|Tuesday|2050Q1|N|N|N|2469867|2469925|2469530|2469803|N|N|N|N|N| +2469896|AAAAAAAAIAALFCAA|2050-03-30|1802|7840|602|2050|3|3|30|1|2050|602|7840|Wednesday|2050Q1|N|N|N|2469867|2469925|2469531|2469804|N|N|N|N|N| +2469897|AAAAAAAAJAALFCAA|2050-03-31|1802|7840|602|2050|4|3|31|1|2050|602|7840|Thursday|2050Q1|N|N|N|2469867|2469925|2469532|2469805|N|N|N|N|N| +2469898|AAAAAAAAKAALFCAA|2050-04-01|1803|7840|602|2050|5|4|1|1|2050|602|7840|Friday|2050Q1|N|Y|N|2469898|2469987|2469533|2469808|N|N|N|N|N| +2469899|AAAAAAAALAALFCAA|2050-04-02|1803|7840|602|2050|6|4|2|2|2050|602|7840|Saturday|2050Q2|N|Y|N|2469898|2469987|2469534|2469809|N|N|N|N|N| +2469900|AAAAAAAAMAALFCAA|2050-04-03|1803|7840|602|2050|0|4|3|2|2050|602|7840|Sunday|2050Q2|N|N|N|2469898|2469987|2469535|2469810|N|N|N|N|N| +2469901|AAAAAAAANAALFCAA|2050-04-04|1803|7840|602|2050|1|4|4|2|2050|602|7840|Monday|2050Q2|N|N|N|2469898|2469987|2469536|2469811|N|N|N|N|N| +2469902|AAAAAAAAOAALFCAA|2050-04-05|1803|7841|602|2050|2|4|5|2|2050|602|7841|Tuesday|2050Q2|N|N|N|2469898|2469987|2469537|2469812|N|N|N|N|N| +2469903|AAAAAAAAPAALFCAA|2050-04-06|1803|7841|602|2050|3|4|6|2|2050|602|7841|Wednesday|2050Q2|N|N|N|2469898|2469987|2469538|2469813|N|N|N|N|N| +2469904|AAAAAAAAABALFCAA|2050-04-07|1803|7841|602|2050|4|4|7|2|2050|602|7841|Thursday|2050Q2|N|N|N|2469898|2469987|2469539|2469814|N|N|N|N|N| +2469905|AAAAAAAABBALFCAA|2050-04-08|1803|7841|602|2050|5|4|8|2|2050|602|7841|Friday|2050Q2|N|Y|N|2469898|2469987|2469540|2469815|N|N|N|N|N| +2469906|AAAAAAAACBALFCAA|2050-04-09|1803|7841|602|2050|6|4|9|2|2050|602|7841|Saturday|2050Q2|N|Y|N|2469898|2469987|2469541|2469816|N|N|N|N|N| +2469907|AAAAAAAADBALFCAA|2050-04-10|1803|7841|602|2050|0|4|10|2|2050|602|7841|Sunday|2050Q2|N|N|N|2469898|2469987|2469542|2469817|N|N|N|N|N| +2469908|AAAAAAAAEBALFCAA|2050-04-11|1803|7841|602|2050|1|4|11|2|2050|602|7841|Monday|2050Q2|N|N|N|2469898|2469987|2469543|2469818|N|N|N|N|N| +2469909|AAAAAAAAFBALFCAA|2050-04-12|1803|7842|602|2050|2|4|12|2|2050|602|7842|Tuesday|2050Q2|N|N|N|2469898|2469987|2469544|2469819|N|N|N|N|N| +2469910|AAAAAAAAGBALFCAA|2050-04-13|1803|7842|602|2050|3|4|13|2|2050|602|7842|Wednesday|2050Q2|N|N|N|2469898|2469987|2469545|2469820|N|N|N|N|N| +2469911|AAAAAAAAHBALFCAA|2050-04-14|1803|7842|602|2050|4|4|14|2|2050|602|7842|Thursday|2050Q2|N|N|N|2469898|2469987|2469546|2469821|N|N|N|N|N| +2469912|AAAAAAAAIBALFCAA|2050-04-15|1803|7842|602|2050|5|4|15|2|2050|602|7842|Friday|2050Q2|N|Y|N|2469898|2469987|2469547|2469822|N|N|N|N|N| +2469913|AAAAAAAAJBALFCAA|2050-04-16|1803|7842|602|2050|6|4|16|2|2050|602|7842|Saturday|2050Q2|N|Y|N|2469898|2469987|2469548|2469823|N|N|N|N|N| +2469914|AAAAAAAAKBALFCAA|2050-04-17|1803|7842|602|2050|0|4|17|2|2050|602|7842|Sunday|2050Q2|N|N|N|2469898|2469987|2469549|2469824|N|N|N|N|N| +2469915|AAAAAAAALBALFCAA|2050-04-18|1803|7842|602|2050|1|4|18|2|2050|602|7842|Monday|2050Q2|N|N|N|2469898|2469987|2469550|2469825|N|N|N|N|N| +2469916|AAAAAAAAMBALFCAA|2050-04-19|1803|7843|602|2050|2|4|19|2|2050|602|7843|Tuesday|2050Q2|N|N|N|2469898|2469987|2469551|2469826|N|N|N|N|N| +2469917|AAAAAAAANBALFCAA|2050-04-20|1803|7843|602|2050|3|4|20|2|2050|602|7843|Wednesday|2050Q2|N|N|N|2469898|2469987|2469552|2469827|N|N|N|N|N| +2469918|AAAAAAAAOBALFCAA|2050-04-21|1803|7843|602|2050|4|4|21|2|2050|602|7843|Thursday|2050Q2|N|N|N|2469898|2469987|2469553|2469828|N|N|N|N|N| +2469919|AAAAAAAAPBALFCAA|2050-04-22|1803|7843|602|2050|5|4|22|2|2050|602|7843|Friday|2050Q2|N|Y|N|2469898|2469987|2469554|2469829|N|N|N|N|N| +2469920|AAAAAAAAACALFCAA|2050-04-23|1803|7843|602|2050|6|4|23|2|2050|602|7843|Saturday|2050Q2|N|Y|N|2469898|2469987|2469555|2469830|N|N|N|N|N| +2469921|AAAAAAAABCALFCAA|2050-04-24|1803|7843|602|2050|0|4|24|2|2050|602|7843|Sunday|2050Q2|N|N|N|2469898|2469987|2469556|2469831|N|N|N|N|N| +2469922|AAAAAAAACCALFCAA|2050-04-25|1803|7843|602|2050|1|4|25|2|2050|602|7843|Monday|2050Q2|N|N|N|2469898|2469987|2469557|2469832|N|N|N|N|N| +2469923|AAAAAAAADCALFCAA|2050-04-26|1803|7844|602|2050|2|4|26|2|2050|602|7844|Tuesday|2050Q2|N|N|N|2469898|2469987|2469558|2469833|N|N|N|N|N| +2469924|AAAAAAAAECALFCAA|2050-04-27|1803|7844|602|2050|3|4|27|2|2050|602|7844|Wednesday|2050Q2|N|N|N|2469898|2469987|2469559|2469834|N|N|N|N|N| +2469925|AAAAAAAAFCALFCAA|2050-04-28|1803|7844|602|2050|4|4|28|2|2050|602|7844|Thursday|2050Q2|N|N|N|2469898|2469987|2469560|2469835|N|N|N|N|N| +2469926|AAAAAAAAGCALFCAA|2050-04-29|1803|7844|602|2050|5|4|29|2|2050|602|7844|Friday|2050Q2|N|Y|N|2469898|2469987|2469561|2469836|N|N|N|N|N| +2469927|AAAAAAAAHCALFCAA|2050-04-30|1803|7844|602|2050|6|4|30|2|2050|602|7844|Saturday|2050Q2|N|Y|N|2469898|2469987|2469562|2469837|N|N|N|N|N| +2469928|AAAAAAAAICALFCAA|2050-05-01|1804|7844|602|2050|0|5|1|2|2050|602|7844|Sunday|2050Q2|N|N|N|2469928|2470047|2469563|2469838|N|N|N|N|N| +2469929|AAAAAAAAJCALFCAA|2050-05-02|1804|7844|602|2050|1|5|2|2|2050|602|7844|Monday|2050Q2|N|N|N|2469928|2470047|2469564|2469839|N|N|N|N|N| +2469930|AAAAAAAAKCALFCAA|2050-05-03|1804|7845|602|2050|2|5|3|2|2050|602|7845|Tuesday|2050Q2|N|N|N|2469928|2470047|2469565|2469840|N|N|N|N|N| +2469931|AAAAAAAALCALFCAA|2050-05-04|1804|7845|602|2050|3|5|4|2|2050|602|7845|Wednesday|2050Q2|N|N|N|2469928|2470047|2469566|2469841|N|N|N|N|N| +2469932|AAAAAAAAMCALFCAA|2050-05-05|1804|7845|602|2050|4|5|5|2|2050|602|7845|Thursday|2050Q2|N|N|N|2469928|2470047|2469567|2469842|N|N|N|N|N| +2469933|AAAAAAAANCALFCAA|2050-05-06|1804|7845|602|2050|5|5|6|2|2050|602|7845|Friday|2050Q2|N|Y|N|2469928|2470047|2469568|2469843|N|N|N|N|N| +2469934|AAAAAAAAOCALFCAA|2050-05-07|1804|7845|602|2050|6|5|7|2|2050|602|7845|Saturday|2050Q2|N|Y|N|2469928|2470047|2469569|2469844|N|N|N|N|N| +2469935|AAAAAAAAPCALFCAA|2050-05-08|1804|7845|602|2050|0|5|8|2|2050|602|7845|Sunday|2050Q2|N|N|N|2469928|2470047|2469570|2469845|N|N|N|N|N| +2469936|AAAAAAAAADALFCAA|2050-05-09|1804|7845|602|2050|1|5|9|2|2050|602|7845|Monday|2050Q2|N|N|N|2469928|2470047|2469571|2469846|N|N|N|N|N| +2469937|AAAAAAAABDALFCAA|2050-05-10|1804|7846|602|2050|2|5|10|2|2050|602|7846|Tuesday|2050Q2|N|N|N|2469928|2470047|2469572|2469847|N|N|N|N|N| +2469938|AAAAAAAACDALFCAA|2050-05-11|1804|7846|602|2050|3|5|11|2|2050|602|7846|Wednesday|2050Q2|N|N|N|2469928|2470047|2469573|2469848|N|N|N|N|N| +2469939|AAAAAAAADDALFCAA|2050-05-12|1804|7846|602|2050|4|5|12|2|2050|602|7846|Thursday|2050Q2|N|N|N|2469928|2470047|2469574|2469849|N|N|N|N|N| +2469940|AAAAAAAAEDALFCAA|2050-05-13|1804|7846|602|2050|5|5|13|2|2050|602|7846|Friday|2050Q2|N|Y|N|2469928|2470047|2469575|2469850|N|N|N|N|N| +2469941|AAAAAAAAFDALFCAA|2050-05-14|1804|7846|602|2050|6|5|14|2|2050|602|7846|Saturday|2050Q2|N|Y|N|2469928|2470047|2469576|2469851|N|N|N|N|N| +2469942|AAAAAAAAGDALFCAA|2050-05-15|1804|7846|602|2050|0|5|15|2|2050|602|7846|Sunday|2050Q2|N|N|N|2469928|2470047|2469577|2469852|N|N|N|N|N| +2469943|AAAAAAAAHDALFCAA|2050-05-16|1804|7846|602|2050|1|5|16|2|2050|602|7846|Monday|2050Q2|N|N|N|2469928|2470047|2469578|2469853|N|N|N|N|N| +2469944|AAAAAAAAIDALFCAA|2050-05-17|1804|7847|602|2050|2|5|17|2|2050|602|7847|Tuesday|2050Q2|N|N|N|2469928|2470047|2469579|2469854|N|N|N|N|N| +2469945|AAAAAAAAJDALFCAA|2050-05-18|1804|7847|602|2050|3|5|18|2|2050|602|7847|Wednesday|2050Q2|N|N|N|2469928|2470047|2469580|2469855|N|N|N|N|N| +2469946|AAAAAAAAKDALFCAA|2050-05-19|1804|7847|602|2050|4|5|19|2|2050|602|7847|Thursday|2050Q2|N|N|N|2469928|2470047|2469581|2469856|N|N|N|N|N| +2469947|AAAAAAAALDALFCAA|2050-05-20|1804|7847|602|2050|5|5|20|2|2050|602|7847|Friday|2050Q2|N|Y|N|2469928|2470047|2469582|2469857|N|N|N|N|N| +2469948|AAAAAAAAMDALFCAA|2050-05-21|1804|7847|602|2050|6|5|21|2|2050|602|7847|Saturday|2050Q2|N|Y|N|2469928|2470047|2469583|2469858|N|N|N|N|N| +2469949|AAAAAAAANDALFCAA|2050-05-22|1804|7847|602|2050|0|5|22|2|2050|602|7847|Sunday|2050Q2|N|N|N|2469928|2470047|2469584|2469859|N|N|N|N|N| +2469950|AAAAAAAAODALFCAA|2050-05-23|1804|7847|602|2050|1|5|23|2|2050|602|7847|Monday|2050Q2|N|N|N|2469928|2470047|2469585|2469860|N|N|N|N|N| +2469951|AAAAAAAAPDALFCAA|2050-05-24|1804|7848|602|2050|2|5|24|2|2050|602|7848|Tuesday|2050Q2|N|N|N|2469928|2470047|2469586|2469861|N|N|N|N|N| +2469952|AAAAAAAAAEALFCAA|2050-05-25|1804|7848|602|2050|3|5|25|2|2050|602|7848|Wednesday|2050Q2|N|N|N|2469928|2470047|2469587|2469862|N|N|N|N|N| +2469953|AAAAAAAABEALFCAA|2050-05-26|1804|7848|602|2050|4|5|26|2|2050|602|7848|Thursday|2050Q2|N|N|N|2469928|2470047|2469588|2469863|N|N|N|N|N| +2469954|AAAAAAAACEALFCAA|2050-05-27|1804|7848|602|2050|5|5|27|2|2050|602|7848|Friday|2050Q2|N|Y|N|2469928|2470047|2469589|2469864|N|N|N|N|N| +2469955|AAAAAAAADEALFCAA|2050-05-28|1804|7848|602|2050|6|5|28|2|2050|602|7848|Saturday|2050Q2|N|Y|N|2469928|2470047|2469590|2469865|N|N|N|N|N| +2469956|AAAAAAAAEEALFCAA|2050-05-29|1804|7848|602|2050|0|5|29|2|2050|602|7848|Sunday|2050Q2|N|N|N|2469928|2470047|2469591|2469866|N|N|N|N|N| +2469957|AAAAAAAAFEALFCAA|2050-05-30|1804|7848|602|2050|1|5|30|2|2050|602|7848|Monday|2050Q2|N|N|N|2469928|2470047|2469592|2469867|N|N|N|N|N| +2469958|AAAAAAAAGEALFCAA|2050-05-31|1804|7849|602|2050|2|5|31|2|2050|602|7849|Tuesday|2050Q2|N|N|N|2469928|2470047|2469593|2469868|N|N|N|N|N| +2469959|AAAAAAAAHEALFCAA|2050-06-01|1805|7849|603|2050|3|6|1|2|2050|603|7849|Wednesday|2050Q2|N|N|N|2469959|2470109|2469594|2469869|N|N|N|N|N| +2469960|AAAAAAAAIEALFCAA|2050-06-02|1805|7849|603|2050|4|6|2|2|2050|603|7849|Thursday|2050Q2|N|N|N|2469959|2470109|2469595|2469870|N|N|N|N|N| +2469961|AAAAAAAAJEALFCAA|2050-06-03|1805|7849|603|2050|5|6|3|2|2050|603|7849|Friday|2050Q2|N|Y|N|2469959|2470109|2469596|2469871|N|N|N|N|N| +2469962|AAAAAAAAKEALFCAA|2050-06-04|1805|7849|603|2050|6|6|4|2|2050|603|7849|Saturday|2050Q2|N|Y|N|2469959|2470109|2469597|2469872|N|N|N|N|N| +2469963|AAAAAAAALEALFCAA|2050-06-05|1805|7849|603|2050|0|6|5|2|2050|603|7849|Sunday|2050Q2|N|N|N|2469959|2470109|2469598|2469873|N|N|N|N|N| +2469964|AAAAAAAAMEALFCAA|2050-06-06|1805|7849|603|2050|1|6|6|2|2050|603|7849|Monday|2050Q2|N|N|N|2469959|2470109|2469599|2469874|N|N|N|N|N| +2469965|AAAAAAAANEALFCAA|2050-06-07|1805|7850|603|2050|2|6|7|2|2050|603|7850|Tuesday|2050Q2|N|N|N|2469959|2470109|2469600|2469875|N|N|N|N|N| +2469966|AAAAAAAAOEALFCAA|2050-06-08|1805|7850|603|2050|3|6|8|2|2050|603|7850|Wednesday|2050Q2|N|N|N|2469959|2470109|2469601|2469876|N|N|N|N|N| +2469967|AAAAAAAAPEALFCAA|2050-06-09|1805|7850|603|2050|4|6|9|2|2050|603|7850|Thursday|2050Q2|N|N|N|2469959|2470109|2469602|2469877|N|N|N|N|N| +2469968|AAAAAAAAAFALFCAA|2050-06-10|1805|7850|603|2050|5|6|10|2|2050|603|7850|Friday|2050Q2|N|Y|N|2469959|2470109|2469603|2469878|N|N|N|N|N| +2469969|AAAAAAAABFALFCAA|2050-06-11|1805|7850|603|2050|6|6|11|2|2050|603|7850|Saturday|2050Q2|N|Y|N|2469959|2470109|2469604|2469879|N|N|N|N|N| +2469970|AAAAAAAACFALFCAA|2050-06-12|1805|7850|603|2050|0|6|12|2|2050|603|7850|Sunday|2050Q2|N|N|N|2469959|2470109|2469605|2469880|N|N|N|N|N| +2469971|AAAAAAAADFALFCAA|2050-06-13|1805|7850|603|2050|1|6|13|2|2050|603|7850|Monday|2050Q2|N|N|N|2469959|2470109|2469606|2469881|N|N|N|N|N| +2469972|AAAAAAAAEFALFCAA|2050-06-14|1805|7851|603|2050|2|6|14|2|2050|603|7851|Tuesday|2050Q2|N|N|N|2469959|2470109|2469607|2469882|N|N|N|N|N| +2469973|AAAAAAAAFFALFCAA|2050-06-15|1805|7851|603|2050|3|6|15|2|2050|603|7851|Wednesday|2050Q2|N|N|N|2469959|2470109|2469608|2469883|N|N|N|N|N| +2469974|AAAAAAAAGFALFCAA|2050-06-16|1805|7851|603|2050|4|6|16|2|2050|603|7851|Thursday|2050Q2|N|N|N|2469959|2470109|2469609|2469884|N|N|N|N|N| +2469975|AAAAAAAAHFALFCAA|2050-06-17|1805|7851|603|2050|5|6|17|2|2050|603|7851|Friday|2050Q2|N|Y|N|2469959|2470109|2469610|2469885|N|N|N|N|N| +2469976|AAAAAAAAIFALFCAA|2050-06-18|1805|7851|603|2050|6|6|18|2|2050|603|7851|Saturday|2050Q2|N|Y|N|2469959|2470109|2469611|2469886|N|N|N|N|N| +2469977|AAAAAAAAJFALFCAA|2050-06-19|1805|7851|603|2050|0|6|19|2|2050|603|7851|Sunday|2050Q2|N|N|N|2469959|2470109|2469612|2469887|N|N|N|N|N| +2469978|AAAAAAAAKFALFCAA|2050-06-20|1805|7851|603|2050|1|6|20|2|2050|603|7851|Monday|2050Q2|N|N|N|2469959|2470109|2469613|2469888|N|N|N|N|N| +2469979|AAAAAAAALFALFCAA|2050-06-21|1805|7852|603|2050|2|6|21|2|2050|603|7852|Tuesday|2050Q2|N|N|N|2469959|2470109|2469614|2469889|N|N|N|N|N| +2469980|AAAAAAAAMFALFCAA|2050-06-22|1805|7852|603|2050|3|6|22|2|2050|603|7852|Wednesday|2050Q2|N|N|N|2469959|2470109|2469615|2469890|N|N|N|N|N| +2469981|AAAAAAAANFALFCAA|2050-06-23|1805|7852|603|2050|4|6|23|2|2050|603|7852|Thursday|2050Q2|N|N|N|2469959|2470109|2469616|2469891|N|N|N|N|N| +2469982|AAAAAAAAOFALFCAA|2050-06-24|1805|7852|603|2050|5|6|24|2|2050|603|7852|Friday|2050Q2|N|Y|N|2469959|2470109|2469617|2469892|N|N|N|N|N| +2469983|AAAAAAAAPFALFCAA|2050-06-25|1805|7852|603|2050|6|6|25|2|2050|603|7852|Saturday|2050Q2|N|Y|N|2469959|2470109|2469618|2469893|N|N|N|N|N| +2469984|AAAAAAAAAGALFCAA|2050-06-26|1805|7852|603|2050|0|6|26|2|2050|603|7852|Sunday|2050Q2|N|N|N|2469959|2470109|2469619|2469894|N|N|N|N|N| +2469985|AAAAAAAABGALFCAA|2050-06-27|1805|7852|603|2050|1|6|27|2|2050|603|7852|Monday|2050Q2|N|N|N|2469959|2470109|2469620|2469895|N|N|N|N|N| +2469986|AAAAAAAACGALFCAA|2050-06-28|1805|7853|603|2050|2|6|28|2|2050|603|7853|Tuesday|2050Q2|N|N|N|2469959|2470109|2469621|2469896|N|N|N|N|N| +2469987|AAAAAAAADGALFCAA|2050-06-29|1805|7853|603|2050|3|6|29|2|2050|603|7853|Wednesday|2050Q2|N|N|N|2469959|2470109|2469622|2469897|N|N|N|N|N| +2469988|AAAAAAAAEGALFCAA|2050-06-30|1805|7853|603|2050|4|6|30|2|2050|603|7853|Thursday|2050Q2|N|N|N|2469959|2470109|2469623|2469898|N|N|N|N|N| +2469989|AAAAAAAAFGALFCAA|2050-07-01|1806|7853|603|2050|5|7|1|2|2050|603|7853|Friday|2050Q2|N|Y|N|2469989|2470169|2469624|2469898|N|N|N|N|N| +2469990|AAAAAAAAGGALFCAA|2050-07-02|1806|7853|603|2050|6|7|2|3|2050|603|7853|Saturday|2050Q3|N|Y|N|2469989|2470169|2469625|2469899|N|N|N|N|N| +2469991|AAAAAAAAHGALFCAA|2050-07-03|1806|7853|603|2050|0|7|3|3|2050|603|7853|Sunday|2050Q3|N|N|N|2469989|2470169|2469626|2469900|N|N|N|N|N| +2469992|AAAAAAAAIGALFCAA|2050-07-04|1806|7853|603|2050|1|7|4|3|2050|603|7853|Monday|2050Q3|N|N|N|2469989|2470169|2469627|2469901|N|N|N|N|N| +2469993|AAAAAAAAJGALFCAA|2050-07-05|1806|7854|603|2050|2|7|5|3|2050|603|7854|Tuesday|2050Q3|Y|N|N|2469989|2470169|2469628|2469902|N|N|N|N|N| +2469994|AAAAAAAAKGALFCAA|2050-07-06|1806|7854|603|2050|3|7|6|3|2050|603|7854|Wednesday|2050Q3|N|N|Y|2469989|2470169|2469629|2469903|N|N|N|N|N| +2469995|AAAAAAAALGALFCAA|2050-07-07|1806|7854|603|2050|4|7|7|3|2050|603|7854|Thursday|2050Q3|N|N|N|2469989|2470169|2469630|2469904|N|N|N|N|N| +2469996|AAAAAAAAMGALFCAA|2050-07-08|1806|7854|603|2050|5|7|8|3|2050|603|7854|Friday|2050Q3|N|Y|N|2469989|2470169|2469631|2469905|N|N|N|N|N| +2469997|AAAAAAAANGALFCAA|2050-07-09|1806|7854|603|2050|6|7|9|3|2050|603|7854|Saturday|2050Q3|N|Y|N|2469989|2470169|2469632|2469906|N|N|N|N|N| +2469998|AAAAAAAAOGALFCAA|2050-07-10|1806|7854|603|2050|0|7|10|3|2050|603|7854|Sunday|2050Q3|N|N|N|2469989|2470169|2469633|2469907|N|N|N|N|N| +2469999|AAAAAAAAPGALFCAA|2050-07-11|1806|7854|603|2050|1|7|11|3|2050|603|7854|Monday|2050Q3|N|N|N|2469989|2470169|2469634|2469908|N|N|N|N|N| +2470000|AAAAAAAAAHALFCAA|2050-07-12|1806|7855|603|2050|2|7|12|3|2050|603|7855|Tuesday|2050Q3|N|N|N|2469989|2470169|2469635|2469909|N|N|N|N|N| +2470001|AAAAAAAABHALFCAA|2050-07-13|1806|7855|603|2050|3|7|13|3|2050|603|7855|Wednesday|2050Q3|N|N|N|2469989|2470169|2469636|2469910|N|N|N|N|N| +2470002|AAAAAAAACHALFCAA|2050-07-14|1806|7855|603|2050|4|7|14|3|2050|603|7855|Thursday|2050Q3|N|N|N|2469989|2470169|2469637|2469911|N|N|N|N|N| +2470003|AAAAAAAADHALFCAA|2050-07-15|1806|7855|603|2050|5|7|15|3|2050|603|7855|Friday|2050Q3|N|Y|N|2469989|2470169|2469638|2469912|N|N|N|N|N| +2470004|AAAAAAAAEHALFCAA|2050-07-16|1806|7855|603|2050|6|7|16|3|2050|603|7855|Saturday|2050Q3|N|Y|N|2469989|2470169|2469639|2469913|N|N|N|N|N| +2470005|AAAAAAAAFHALFCAA|2050-07-17|1806|7855|603|2050|0|7|17|3|2050|603|7855|Sunday|2050Q3|N|N|N|2469989|2470169|2469640|2469914|N|N|N|N|N| +2470006|AAAAAAAAGHALFCAA|2050-07-18|1806|7855|603|2050|1|7|18|3|2050|603|7855|Monday|2050Q3|N|N|N|2469989|2470169|2469641|2469915|N|N|N|N|N| +2470007|AAAAAAAAHHALFCAA|2050-07-19|1806|7856|603|2050|2|7|19|3|2050|603|7856|Tuesday|2050Q3|N|N|N|2469989|2470169|2469642|2469916|N|N|N|N|N| +2470008|AAAAAAAAIHALFCAA|2050-07-20|1806|7856|603|2050|3|7|20|3|2050|603|7856|Wednesday|2050Q3|N|N|N|2469989|2470169|2469643|2469917|N|N|N|N|N| +2470009|AAAAAAAAJHALFCAA|2050-07-21|1806|7856|603|2050|4|7|21|3|2050|603|7856|Thursday|2050Q3|N|N|N|2469989|2470169|2469644|2469918|N|N|N|N|N| +2470010|AAAAAAAAKHALFCAA|2050-07-22|1806|7856|603|2050|5|7|22|3|2050|603|7856|Friday|2050Q3|N|Y|N|2469989|2470169|2469645|2469919|N|N|N|N|N| +2470011|AAAAAAAALHALFCAA|2050-07-23|1806|7856|603|2050|6|7|23|3|2050|603|7856|Saturday|2050Q3|N|Y|N|2469989|2470169|2469646|2469920|N|N|N|N|N| +2470012|AAAAAAAAMHALFCAA|2050-07-24|1806|7856|603|2050|0|7|24|3|2050|603|7856|Sunday|2050Q3|N|N|N|2469989|2470169|2469647|2469921|N|N|N|N|N| +2470013|AAAAAAAANHALFCAA|2050-07-25|1806|7856|603|2050|1|7|25|3|2050|603|7856|Monday|2050Q3|N|N|N|2469989|2470169|2469648|2469922|N|N|N|N|N| +2470014|AAAAAAAAOHALFCAA|2050-07-26|1806|7857|603|2050|2|7|26|3|2050|603|7857|Tuesday|2050Q3|N|N|N|2469989|2470169|2469649|2469923|N|N|N|N|N| +2470015|AAAAAAAAPHALFCAA|2050-07-27|1806|7857|603|2050|3|7|27|3|2050|603|7857|Wednesday|2050Q3|N|N|N|2469989|2470169|2469650|2469924|N|N|N|N|N| +2470016|AAAAAAAAAIALFCAA|2050-07-28|1806|7857|603|2050|4|7|28|3|2050|603|7857|Thursday|2050Q3|N|N|N|2469989|2470169|2469651|2469925|N|N|N|N|N| +2470017|AAAAAAAABIALFCAA|2050-07-29|1806|7857|603|2050|5|7|29|3|2050|603|7857|Friday|2050Q3|N|Y|N|2469989|2470169|2469652|2469926|N|N|N|N|N| +2470018|AAAAAAAACIALFCAA|2050-07-30|1806|7857|603|2050|6|7|30|3|2050|603|7857|Saturday|2050Q3|N|Y|N|2469989|2470169|2469653|2469927|N|N|N|N|N| +2470019|AAAAAAAADIALFCAA|2050-07-31|1806|7857|603|2050|0|7|31|3|2050|603|7857|Sunday|2050Q3|N|N|N|2469989|2470169|2469654|2469928|N|N|N|N|N| +2470020|AAAAAAAAEIALFCAA|2050-08-01|1807|7857|603|2050|1|8|1|3|2050|603|7857|Monday|2050Q3|N|N|N|2470020|2470231|2469655|2469929|N|N|N|N|N| +2470021|AAAAAAAAFIALFCAA|2050-08-02|1807|7858|603|2050|2|8|2|3|2050|603|7858|Tuesday|2050Q3|N|N|N|2470020|2470231|2469656|2469930|N|N|N|N|N| +2470022|AAAAAAAAGIALFCAA|2050-08-03|1807|7858|603|2050|3|8|3|3|2050|603|7858|Wednesday|2050Q3|N|N|N|2470020|2470231|2469657|2469931|N|N|N|N|N| +2470023|AAAAAAAAHIALFCAA|2050-08-04|1807|7858|603|2050|4|8|4|3|2050|603|7858|Thursday|2050Q3|N|N|N|2470020|2470231|2469658|2469932|N|N|N|N|N| +2470024|AAAAAAAAIIALFCAA|2050-08-05|1807|7858|603|2050|5|8|5|3|2050|603|7858|Friday|2050Q3|N|Y|N|2470020|2470231|2469659|2469933|N|N|N|N|N| +2470025|AAAAAAAAJIALFCAA|2050-08-06|1807|7858|603|2050|6|8|6|3|2050|603|7858|Saturday|2050Q3|N|Y|N|2470020|2470231|2469660|2469934|N|N|N|N|N| +2470026|AAAAAAAAKIALFCAA|2050-08-07|1807|7858|603|2050|0|8|7|3|2050|603|7858|Sunday|2050Q3|N|N|N|2470020|2470231|2469661|2469935|N|N|N|N|N| +2470027|AAAAAAAALIALFCAA|2050-08-08|1807|7858|603|2050|1|8|8|3|2050|603|7858|Monday|2050Q3|N|N|N|2470020|2470231|2469662|2469936|N|N|N|N|N| +2470028|AAAAAAAAMIALFCAA|2050-08-09|1807|7859|603|2050|2|8|9|3|2050|603|7859|Tuesday|2050Q3|N|N|N|2470020|2470231|2469663|2469937|N|N|N|N|N| +2470029|AAAAAAAANIALFCAA|2050-08-10|1807|7859|603|2050|3|8|10|3|2050|603|7859|Wednesday|2050Q3|N|N|N|2470020|2470231|2469664|2469938|N|N|N|N|N| +2470030|AAAAAAAAOIALFCAA|2050-08-11|1807|7859|603|2050|4|8|11|3|2050|603|7859|Thursday|2050Q3|N|N|N|2470020|2470231|2469665|2469939|N|N|N|N|N| +2470031|AAAAAAAAPIALFCAA|2050-08-12|1807|7859|603|2050|5|8|12|3|2050|603|7859|Friday|2050Q3|N|Y|N|2470020|2470231|2469666|2469940|N|N|N|N|N| +2470032|AAAAAAAAAJALFCAA|2050-08-13|1807|7859|603|2050|6|8|13|3|2050|603|7859|Saturday|2050Q3|N|Y|N|2470020|2470231|2469667|2469941|N|N|N|N|N| +2470033|AAAAAAAABJALFCAA|2050-08-14|1807|7859|603|2050|0|8|14|3|2050|603|7859|Sunday|2050Q3|N|N|N|2470020|2470231|2469668|2469942|N|N|N|N|N| +2470034|AAAAAAAACJALFCAA|2050-08-15|1807|7859|603|2050|1|8|15|3|2050|603|7859|Monday|2050Q3|N|N|N|2470020|2470231|2469669|2469943|N|N|N|N|N| +2470035|AAAAAAAADJALFCAA|2050-08-16|1807|7860|603|2050|2|8|16|3|2050|603|7860|Tuesday|2050Q3|N|N|N|2470020|2470231|2469670|2469944|N|N|N|N|N| +2470036|AAAAAAAAEJALFCAA|2050-08-17|1807|7860|603|2050|3|8|17|3|2050|603|7860|Wednesday|2050Q3|N|N|N|2470020|2470231|2469671|2469945|N|N|N|N|N| +2470037|AAAAAAAAFJALFCAA|2050-08-18|1807|7860|603|2050|4|8|18|3|2050|603|7860|Thursday|2050Q3|N|N|N|2470020|2470231|2469672|2469946|N|N|N|N|N| +2470038|AAAAAAAAGJALFCAA|2050-08-19|1807|7860|603|2050|5|8|19|3|2050|603|7860|Friday|2050Q3|N|Y|N|2470020|2470231|2469673|2469947|N|N|N|N|N| +2470039|AAAAAAAAHJALFCAA|2050-08-20|1807|7860|603|2050|6|8|20|3|2050|603|7860|Saturday|2050Q3|N|Y|N|2470020|2470231|2469674|2469948|N|N|N|N|N| +2470040|AAAAAAAAIJALFCAA|2050-08-21|1807|7860|603|2050|0|8|21|3|2050|603|7860|Sunday|2050Q3|N|N|N|2470020|2470231|2469675|2469949|N|N|N|N|N| +2470041|AAAAAAAAJJALFCAA|2050-08-22|1807|7860|603|2050|1|8|22|3|2050|603|7860|Monday|2050Q3|N|N|N|2470020|2470231|2469676|2469950|N|N|N|N|N| +2470042|AAAAAAAAKJALFCAA|2050-08-23|1807|7861|603|2050|2|8|23|3|2050|603|7861|Tuesday|2050Q3|N|N|N|2470020|2470231|2469677|2469951|N|N|N|N|N| +2470043|AAAAAAAALJALFCAA|2050-08-24|1807|7861|603|2050|3|8|24|3|2050|603|7861|Wednesday|2050Q3|N|N|N|2470020|2470231|2469678|2469952|N|N|N|N|N| +2470044|AAAAAAAAMJALFCAA|2050-08-25|1807|7861|603|2050|4|8|25|3|2050|603|7861|Thursday|2050Q3|N|N|N|2470020|2470231|2469679|2469953|N|N|N|N|N| +2470045|AAAAAAAANJALFCAA|2050-08-26|1807|7861|603|2050|5|8|26|3|2050|603|7861|Friday|2050Q3|N|Y|N|2470020|2470231|2469680|2469954|N|N|N|N|N| +2470046|AAAAAAAAOJALFCAA|2050-08-27|1807|7861|603|2050|6|8|27|3|2050|603|7861|Saturday|2050Q3|N|Y|N|2470020|2470231|2469681|2469955|N|N|N|N|N| +2470047|AAAAAAAAPJALFCAA|2050-08-28|1807|7861|603|2050|0|8|28|3|2050|603|7861|Sunday|2050Q3|N|N|N|2470020|2470231|2469682|2469956|N|N|N|N|N| +2470048|AAAAAAAAAKALFCAA|2050-08-29|1807|7861|603|2050|1|8|29|3|2050|603|7861|Monday|2050Q3|N|N|N|2470020|2470231|2469683|2469957|N|N|N|N|N| +2470049|AAAAAAAABKALFCAA|2050-08-30|1807|7862|603|2050|2|8|30|3|2050|603|7862|Tuesday|2050Q3|N|N|N|2470020|2470231|2469684|2469958|N|N|N|N|N| +2470050|AAAAAAAACKALFCAA|2050-08-31|1807|7862|603|2050|3|8|31|3|2050|603|7862|Wednesday|2050Q3|N|N|N|2470020|2470231|2469685|2469959|N|N|N|N|N| +2470051|AAAAAAAADKALFCAA|2050-09-01|1808|7862|604|2050|4|9|1|3|2050|604|7862|Thursday|2050Q3|N|N|N|2470051|2470293|2469686|2469960|N|N|N|N|N| +2470052|AAAAAAAAEKALFCAA|2050-09-02|1808|7862|604|2050|5|9|2|3|2050|604|7862|Friday|2050Q3|N|Y|N|2470051|2470293|2469687|2469961|N|N|N|N|N| +2470053|AAAAAAAAFKALFCAA|2050-09-03|1808|7862|604|2050|6|9|3|3|2050|604|7862|Saturday|2050Q3|N|Y|N|2470051|2470293|2469688|2469962|N|N|N|N|N| +2470054|AAAAAAAAGKALFCAA|2050-09-04|1808|7862|604|2050|0|9|4|3|2050|604|7862|Sunday|2050Q3|N|N|N|2470051|2470293|2469689|2469963|N|N|N|N|N| +2470055|AAAAAAAAHKALFCAA|2050-09-05|1808|7862|604|2050|1|9|5|3|2050|604|7862|Monday|2050Q3|N|N|N|2470051|2470293|2469690|2469964|N|N|N|N|N| +2470056|AAAAAAAAIKALFCAA|2050-09-06|1808|7863|604|2050|2|9|6|3|2050|604|7863|Tuesday|2050Q3|N|N|N|2470051|2470293|2469691|2469965|N|N|N|N|N| +2470057|AAAAAAAAJKALFCAA|2050-09-07|1808|7863|604|2050|3|9|7|3|2050|604|7863|Wednesday|2050Q3|N|N|N|2470051|2470293|2469692|2469966|N|N|N|N|N| +2470058|AAAAAAAAKKALFCAA|2050-09-08|1808|7863|604|2050|4|9|8|3|2050|604|7863|Thursday|2050Q3|N|N|N|2470051|2470293|2469693|2469967|N|N|N|N|N| +2470059|AAAAAAAALKALFCAA|2050-09-09|1808|7863|604|2050|5|9|9|3|2050|604|7863|Friday|2050Q3|N|Y|N|2470051|2470293|2469694|2469968|N|N|N|N|N| +2470060|AAAAAAAAMKALFCAA|2050-09-10|1808|7863|604|2050|6|9|10|3|2050|604|7863|Saturday|2050Q3|N|Y|N|2470051|2470293|2469695|2469969|N|N|N|N|N| +2470061|AAAAAAAANKALFCAA|2050-09-11|1808|7863|604|2050|0|9|11|3|2050|604|7863|Sunday|2050Q3|N|N|N|2470051|2470293|2469696|2469970|N|N|N|N|N| +2470062|AAAAAAAAOKALFCAA|2050-09-12|1808|7863|604|2050|1|9|12|3|2050|604|7863|Monday|2050Q3|N|N|N|2470051|2470293|2469697|2469971|N|N|N|N|N| +2470063|AAAAAAAAPKALFCAA|2050-09-13|1808|7864|604|2050|2|9|13|3|2050|604|7864|Tuesday|2050Q3|N|N|N|2470051|2470293|2469698|2469972|N|N|N|N|N| +2470064|AAAAAAAAALALFCAA|2050-09-14|1808|7864|604|2050|3|9|14|3|2050|604|7864|Wednesday|2050Q3|N|N|N|2470051|2470293|2469699|2469973|N|N|N|N|N| +2470065|AAAAAAAABLALFCAA|2050-09-15|1808|7864|604|2050|4|9|15|3|2050|604|7864|Thursday|2050Q3|N|N|N|2470051|2470293|2469700|2469974|N|N|N|N|N| +2470066|AAAAAAAACLALFCAA|2050-09-16|1808|7864|604|2050|5|9|16|3|2050|604|7864|Friday|2050Q3|N|Y|N|2470051|2470293|2469701|2469975|N|N|N|N|N| +2470067|AAAAAAAADLALFCAA|2050-09-17|1808|7864|604|2050|6|9|17|3|2050|604|7864|Saturday|2050Q3|N|Y|N|2470051|2470293|2469702|2469976|N|N|N|N|N| +2470068|AAAAAAAAELALFCAA|2050-09-18|1808|7864|604|2050|0|9|18|3|2050|604|7864|Sunday|2050Q3|N|N|N|2470051|2470293|2469703|2469977|N|N|N|N|N| +2470069|AAAAAAAAFLALFCAA|2050-09-19|1808|7864|604|2050|1|9|19|3|2050|604|7864|Monday|2050Q3|N|N|N|2470051|2470293|2469704|2469978|N|N|N|N|N| +2470070|AAAAAAAAGLALFCAA|2050-09-20|1808|7865|604|2050|2|9|20|3|2050|604|7865|Tuesday|2050Q3|N|N|N|2470051|2470293|2469705|2469979|N|N|N|N|N| +2470071|AAAAAAAAHLALFCAA|2050-09-21|1808|7865|604|2050|3|9|21|3|2050|604|7865|Wednesday|2050Q3|N|N|N|2470051|2470293|2469706|2469980|N|N|N|N|N| +2470072|AAAAAAAAILALFCAA|2050-09-22|1808|7865|604|2050|4|9|22|3|2050|604|7865|Thursday|2050Q3|N|N|N|2470051|2470293|2469707|2469981|N|N|N|N|N| +2470073|AAAAAAAAJLALFCAA|2050-09-23|1808|7865|604|2050|5|9|23|3|2050|604|7865|Friday|2050Q3|N|Y|N|2470051|2470293|2469708|2469982|N|N|N|N|N| +2470074|AAAAAAAAKLALFCAA|2050-09-24|1808|7865|604|2050|6|9|24|3|2050|604|7865|Saturday|2050Q3|N|Y|N|2470051|2470293|2469709|2469983|N|N|N|N|N| +2470075|AAAAAAAALLALFCAA|2050-09-25|1808|7865|604|2050|0|9|25|3|2050|604|7865|Sunday|2050Q3|N|N|N|2470051|2470293|2469710|2469984|N|N|N|N|N| +2470076|AAAAAAAAMLALFCAA|2050-09-26|1808|7865|604|2050|1|9|26|3|2050|604|7865|Monday|2050Q3|N|N|N|2470051|2470293|2469711|2469985|N|N|N|N|N| +2470077|AAAAAAAANLALFCAA|2050-09-27|1808|7866|604|2050|2|9|27|3|2050|604|7866|Tuesday|2050Q3|N|N|N|2470051|2470293|2469712|2469986|N|N|N|N|N| +2470078|AAAAAAAAOLALFCAA|2050-09-28|1808|7866|604|2050|3|9|28|3|2050|604|7866|Wednesday|2050Q3|N|N|N|2470051|2470293|2469713|2469987|N|N|N|N|N| +2470079|AAAAAAAAPLALFCAA|2050-09-29|1808|7866|604|2050|4|9|29|3|2050|604|7866|Thursday|2050Q3|N|N|N|2470051|2470293|2469714|2469988|N|N|N|N|N| +2470080|AAAAAAAAAMALFCAA|2050-09-30|1808|7866|604|2050|5|9|30|3|2050|604|7866|Friday|2050Q3|N|Y|N|2470051|2470293|2469715|2469989|N|N|N|N|N| +2470081|AAAAAAAABMALFCAA|2050-10-01|1809|7866|604|2050|6|10|1|3|2050|604|7866|Saturday|2050Q3|N|Y|N|2470081|2470353|2469716|2469989|N|N|N|N|N| +2470082|AAAAAAAACMALFCAA|2050-10-02|1809|7866|604|2050|0|10|2|4|2050|604|7866|Sunday|2050Q4|N|N|N|2470081|2470353|2469717|2469990|N|N|N|N|N| +2470083|AAAAAAAADMALFCAA|2050-10-03|1809|7866|604|2050|1|10|3|4|2050|604|7866|Monday|2050Q4|N|N|N|2470081|2470353|2469718|2469991|N|N|N|N|N| +2470084|AAAAAAAAEMALFCAA|2050-10-04|1809|7867|604|2050|2|10|4|4|2050|604|7867|Tuesday|2050Q4|N|N|N|2470081|2470353|2469719|2469992|N|N|N|N|N| +2470085|AAAAAAAAFMALFCAA|2050-10-05|1809|7867|604|2050|3|10|5|4|2050|604|7867|Wednesday|2050Q4|N|N|N|2470081|2470353|2469720|2469993|N|N|N|N|N| +2470086|AAAAAAAAGMALFCAA|2050-10-06|1809|7867|604|2050|4|10|6|4|2050|604|7867|Thursday|2050Q4|N|N|N|2470081|2470353|2469721|2469994|N|N|N|N|N| +2470087|AAAAAAAAHMALFCAA|2050-10-07|1809|7867|604|2050|5|10|7|4|2050|604|7867|Friday|2050Q4|N|Y|N|2470081|2470353|2469722|2469995|N|N|N|N|N| +2470088|AAAAAAAAIMALFCAA|2050-10-08|1809|7867|604|2050|6|10|8|4|2050|604|7867|Saturday|2050Q4|N|Y|N|2470081|2470353|2469723|2469996|N|N|N|N|N| +2470089|AAAAAAAAJMALFCAA|2050-10-09|1809|7867|604|2050|0|10|9|4|2050|604|7867|Sunday|2050Q4|N|N|N|2470081|2470353|2469724|2469997|N|N|N|N|N| +2470090|AAAAAAAAKMALFCAA|2050-10-10|1809|7867|604|2050|1|10|10|4|2050|604|7867|Monday|2050Q4|N|N|N|2470081|2470353|2469725|2469998|N|N|N|N|N| +2470091|AAAAAAAALMALFCAA|2050-10-11|1809|7868|604|2050|2|10|11|4|2050|604|7868|Tuesday|2050Q4|N|N|N|2470081|2470353|2469726|2469999|N|N|N|N|N| +2470092|AAAAAAAAMMALFCAA|2050-10-12|1809|7868|604|2050|3|10|12|4|2050|604|7868|Wednesday|2050Q4|N|N|N|2470081|2470353|2469727|2470000|N|N|N|N|N| +2470093|AAAAAAAANMALFCAA|2050-10-13|1809|7868|604|2050|4|10|13|4|2050|604|7868|Thursday|2050Q4|N|N|N|2470081|2470353|2469728|2470001|N|N|N|N|N| +2470094|AAAAAAAAOMALFCAA|2050-10-14|1809|7868|604|2050|5|10|14|4|2050|604|7868|Friday|2050Q4|N|Y|N|2470081|2470353|2469729|2470002|N|N|N|N|N| +2470095|AAAAAAAAPMALFCAA|2050-10-15|1809|7868|604|2050|6|10|15|4|2050|604|7868|Saturday|2050Q4|N|Y|N|2470081|2470353|2469730|2470003|N|N|N|N|N| +2470096|AAAAAAAAANALFCAA|2050-10-16|1809|7868|604|2050|0|10|16|4|2050|604|7868|Sunday|2050Q4|N|N|N|2470081|2470353|2469731|2470004|N|N|N|N|N| +2470097|AAAAAAAABNALFCAA|2050-10-17|1809|7868|604|2050|1|10|17|4|2050|604|7868|Monday|2050Q4|N|N|N|2470081|2470353|2469732|2470005|N|N|N|N|N| +2470098|AAAAAAAACNALFCAA|2050-10-18|1809|7869|604|2050|2|10|18|4|2050|604|7869|Tuesday|2050Q4|N|N|N|2470081|2470353|2469733|2470006|N|N|N|N|N| +2470099|AAAAAAAADNALFCAA|2050-10-19|1809|7869|604|2050|3|10|19|4|2050|604|7869|Wednesday|2050Q4|N|N|N|2470081|2470353|2469734|2470007|N|N|N|N|N| +2470100|AAAAAAAAENALFCAA|2050-10-20|1809|7869|604|2050|4|10|20|4|2050|604|7869|Thursday|2050Q4|N|N|N|2470081|2470353|2469735|2470008|N|N|N|N|N| +2470101|AAAAAAAAFNALFCAA|2050-10-21|1809|7869|604|2050|5|10|21|4|2050|604|7869|Friday|2050Q4|N|Y|N|2470081|2470353|2469736|2470009|N|N|N|N|N| +2470102|AAAAAAAAGNALFCAA|2050-10-22|1809|7869|604|2050|6|10|22|4|2050|604|7869|Saturday|2050Q4|N|Y|N|2470081|2470353|2469737|2470010|N|N|N|N|N| +2470103|AAAAAAAAHNALFCAA|2050-10-23|1809|7869|604|2050|0|10|23|4|2050|604|7869|Sunday|2050Q4|N|N|N|2470081|2470353|2469738|2470011|N|N|N|N|N| +2470104|AAAAAAAAINALFCAA|2050-10-24|1809|7869|604|2050|1|10|24|4|2050|604|7869|Monday|2050Q4|N|N|N|2470081|2470353|2469739|2470012|N|N|N|N|N| +2470105|AAAAAAAAJNALFCAA|2050-10-25|1809|7870|604|2050|2|10|25|4|2050|604|7870|Tuesday|2050Q4|N|N|N|2470081|2470353|2469740|2470013|N|N|N|N|N| +2470106|AAAAAAAAKNALFCAA|2050-10-26|1809|7870|604|2050|3|10|26|4|2050|604|7870|Wednesday|2050Q4|N|N|N|2470081|2470353|2469741|2470014|N|N|N|N|N| +2470107|AAAAAAAALNALFCAA|2050-10-27|1809|7870|604|2050|4|10|27|4|2050|604|7870|Thursday|2050Q4|N|N|N|2470081|2470353|2469742|2470015|N|N|N|N|N| +2470108|AAAAAAAAMNALFCAA|2050-10-28|1809|7870|604|2050|5|10|28|4|2050|604|7870|Friday|2050Q4|N|Y|N|2470081|2470353|2469743|2470016|N|N|N|N|N| +2470109|AAAAAAAANNALFCAA|2050-10-29|1809|7870|604|2050|6|10|29|4|2050|604|7870|Saturday|2050Q4|N|Y|N|2470081|2470353|2469744|2470017|N|N|N|N|N| +2470110|AAAAAAAAONALFCAA|2050-10-30|1809|7870|604|2050|0|10|30|4|2050|604|7870|Sunday|2050Q4|N|N|N|2470081|2470353|2469745|2470018|N|N|N|N|N| +2470111|AAAAAAAAPNALFCAA|2050-10-31|1809|7870|604|2050|1|10|31|4|2050|604|7870|Monday|2050Q4|N|N|N|2470081|2470353|2469746|2470019|N|N|N|N|N| +2470112|AAAAAAAAAOALFCAA|2050-11-01|1810|7871|604|2050|2|11|1|4|2050|604|7871|Tuesday|2050Q4|N|N|N|2470112|2470415|2469747|2470020|N|N|N|N|N| +2470113|AAAAAAAABOALFCAA|2050-11-02|1810|7871|604|2050|3|11|2|4|2050|604|7871|Wednesday|2050Q4|N|N|N|2470112|2470415|2469748|2470021|N|N|N|N|N| +2470114|AAAAAAAACOALFCAA|2050-11-03|1810|7871|604|2050|4|11|3|4|2050|604|7871|Thursday|2050Q4|N|N|N|2470112|2470415|2469749|2470022|N|N|N|N|N| +2470115|AAAAAAAADOALFCAA|2050-11-04|1810|7871|604|2050|5|11|4|4|2050|604|7871|Friday|2050Q4|N|Y|N|2470112|2470415|2469750|2470023|N|N|N|N|N| +2470116|AAAAAAAAEOALFCAA|2050-11-05|1810|7871|604|2050|6|11|5|4|2050|604|7871|Saturday|2050Q4|N|Y|N|2470112|2470415|2469751|2470024|N|N|N|N|N| +2470117|AAAAAAAAFOALFCAA|2050-11-06|1810|7871|604|2050|0|11|6|4|2050|604|7871|Sunday|2050Q4|N|N|N|2470112|2470415|2469752|2470025|N|N|N|N|N| +2470118|AAAAAAAAGOALFCAA|2050-11-07|1810|7871|604|2050|1|11|7|4|2050|604|7871|Monday|2050Q4|N|N|N|2470112|2470415|2469753|2470026|N|N|N|N|N| +2470119|AAAAAAAAHOALFCAA|2050-11-08|1810|7872|604|2050|2|11|8|4|2050|604|7872|Tuesday|2050Q4|N|N|N|2470112|2470415|2469754|2470027|N|N|N|N|N| +2470120|AAAAAAAAIOALFCAA|2050-11-09|1810|7872|604|2050|3|11|9|4|2050|604|7872|Wednesday|2050Q4|N|N|N|2470112|2470415|2469755|2470028|N|N|N|N|N| +2470121|AAAAAAAAJOALFCAA|2050-11-10|1810|7872|604|2050|4|11|10|4|2050|604|7872|Thursday|2050Q4|N|N|N|2470112|2470415|2469756|2470029|N|N|N|N|N| +2470122|AAAAAAAAKOALFCAA|2050-11-11|1810|7872|604|2050|5|11|11|4|2050|604|7872|Friday|2050Q4|N|Y|N|2470112|2470415|2469757|2470030|N|N|N|N|N| +2470123|AAAAAAAALOALFCAA|2050-11-12|1810|7872|604|2050|6|11|12|4|2050|604|7872|Saturday|2050Q4|N|Y|N|2470112|2470415|2469758|2470031|N|N|N|N|N| +2470124|AAAAAAAAMOALFCAA|2050-11-13|1810|7872|604|2050|0|11|13|4|2050|604|7872|Sunday|2050Q4|N|N|N|2470112|2470415|2469759|2470032|N|N|N|N|N| +2470125|AAAAAAAANOALFCAA|2050-11-14|1810|7872|604|2050|1|11|14|4|2050|604|7872|Monday|2050Q4|N|N|N|2470112|2470415|2469760|2470033|N|N|N|N|N| +2470126|AAAAAAAAOOALFCAA|2050-11-15|1810|7873|604|2050|2|11|15|4|2050|604|7873|Tuesday|2050Q4|N|N|N|2470112|2470415|2469761|2470034|N|N|N|N|N| +2470127|AAAAAAAAPOALFCAA|2050-11-16|1810|7873|604|2050|3|11|16|4|2050|604|7873|Wednesday|2050Q4|N|N|N|2470112|2470415|2469762|2470035|N|N|N|N|N| +2470128|AAAAAAAAAPALFCAA|2050-11-17|1810|7873|604|2050|4|11|17|4|2050|604|7873|Thursday|2050Q4|N|N|N|2470112|2470415|2469763|2470036|N|N|N|N|N| +2470129|AAAAAAAABPALFCAA|2050-11-18|1810|7873|604|2050|5|11|18|4|2050|604|7873|Friday|2050Q4|N|Y|N|2470112|2470415|2469764|2470037|N|N|N|N|N| +2470130|AAAAAAAACPALFCAA|2050-11-19|1810|7873|604|2050|6|11|19|4|2050|604|7873|Saturday|2050Q4|N|Y|N|2470112|2470415|2469765|2470038|N|N|N|N|N| +2470131|AAAAAAAADPALFCAA|2050-11-20|1810|7873|604|2050|0|11|20|4|2050|604|7873|Sunday|2050Q4|N|N|N|2470112|2470415|2469766|2470039|N|N|N|N|N| +2470132|AAAAAAAAEPALFCAA|2050-11-21|1810|7873|604|2050|1|11|21|4|2050|604|7873|Monday|2050Q4|N|N|N|2470112|2470415|2469767|2470040|N|N|N|N|N| +2470133|AAAAAAAAFPALFCAA|2050-11-22|1810|7874|604|2050|2|11|22|4|2050|604|7874|Tuesday|2050Q4|N|N|N|2470112|2470415|2469768|2470041|N|N|N|N|N| +2470134|AAAAAAAAGPALFCAA|2050-11-23|1810|7874|604|2050|3|11|23|4|2050|604|7874|Wednesday|2050Q4|N|N|N|2470112|2470415|2469769|2470042|N|N|N|N|N| +2470135|AAAAAAAAHPALFCAA|2050-11-24|1810|7874|604|2050|4|11|24|4|2050|604|7874|Thursday|2050Q4|N|N|N|2470112|2470415|2469770|2470043|N|N|N|N|N| +2470136|AAAAAAAAIPALFCAA|2050-11-25|1810|7874|604|2050|5|11|25|4|2050|604|7874|Friday|2050Q4|N|Y|N|2470112|2470415|2469771|2470044|N|N|N|N|N| +2470137|AAAAAAAAJPALFCAA|2050-11-26|1810|7874|604|2050|6|11|26|4|2050|604|7874|Saturday|2050Q4|N|Y|N|2470112|2470415|2469772|2470045|N|N|N|N|N| +2470138|AAAAAAAAKPALFCAA|2050-11-27|1810|7874|604|2050|0|11|27|4|2050|604|7874|Sunday|2050Q4|N|N|N|2470112|2470415|2469773|2470046|N|N|N|N|N| +2470139|AAAAAAAALPALFCAA|2050-11-28|1810|7874|604|2050|1|11|28|4|2050|604|7874|Monday|2050Q4|N|N|N|2470112|2470415|2469774|2470047|N|N|N|N|N| +2470140|AAAAAAAAMPALFCAA|2050-11-29|1810|7875|604|2050|2|11|29|4|2050|604|7875|Tuesday|2050Q4|N|N|N|2470112|2470415|2469775|2470048|N|N|N|N|N| +2470141|AAAAAAAANPALFCAA|2050-11-30|1810|7875|604|2050|3|11|30|4|2050|604|7875|Wednesday|2050Q4|N|N|N|2470112|2470415|2469776|2470049|N|N|N|N|N| +2470142|AAAAAAAAOPALFCAA|2050-12-01|1811|7875|605|2050|4|12|1|4|2050|605|7875|Thursday|2050Q4|N|N|N|2470142|2470475|2469777|2470050|N|N|N|N|N| +2470143|AAAAAAAAPPALFCAA|2050-12-02|1811|7875|605|2050|5|12|2|4|2050|605|7875|Friday|2050Q4|N|Y|N|2470142|2470475|2469778|2470051|N|N|N|N|N| +2470144|AAAAAAAAAABLFCAA|2050-12-03|1811|7875|605|2050|6|12|3|4|2050|605|7875|Saturday|2050Q4|N|Y|N|2470142|2470475|2469779|2470052|N|N|N|N|N| +2470145|AAAAAAAABABLFCAA|2050-12-04|1811|7875|605|2050|0|12|4|4|2050|605|7875|Sunday|2050Q4|N|N|N|2470142|2470475|2469780|2470053|N|N|N|N|N| +2470146|AAAAAAAACABLFCAA|2050-12-05|1811|7875|605|2050|1|12|5|4|2050|605|7875|Monday|2050Q4|N|N|N|2470142|2470475|2469781|2470054|N|N|N|N|N| +2470147|AAAAAAAADABLFCAA|2050-12-06|1811|7876|605|2050|2|12|6|4|2050|605|7876|Tuesday|2050Q4|N|N|N|2470142|2470475|2469782|2470055|N|N|N|N|N| +2470148|AAAAAAAAEABLFCAA|2050-12-07|1811|7876|605|2050|3|12|7|4|2050|605|7876|Wednesday|2050Q4|N|N|N|2470142|2470475|2469783|2470056|N|N|N|N|N| +2470149|AAAAAAAAFABLFCAA|2050-12-08|1811|7876|605|2050|4|12|8|4|2050|605|7876|Thursday|2050Q4|N|N|N|2470142|2470475|2469784|2470057|N|N|N|N|N| +2470150|AAAAAAAAGABLFCAA|2050-12-09|1811|7876|605|2050|5|12|9|4|2050|605|7876|Friday|2050Q4|N|Y|N|2470142|2470475|2469785|2470058|N|N|N|N|N| +2470151|AAAAAAAAHABLFCAA|2050-12-10|1811|7876|605|2050|6|12|10|4|2050|605|7876|Saturday|2050Q4|N|Y|N|2470142|2470475|2469786|2470059|N|N|N|N|N| +2470152|AAAAAAAAIABLFCAA|2050-12-11|1811|7876|605|2050|0|12|11|4|2050|605|7876|Sunday|2050Q4|N|N|N|2470142|2470475|2469787|2470060|N|N|N|N|N| +2470153|AAAAAAAAJABLFCAA|2050-12-12|1811|7876|605|2050|1|12|12|4|2050|605|7876|Monday|2050Q4|N|N|N|2470142|2470475|2469788|2470061|N|N|N|N|N| +2470154|AAAAAAAAKABLFCAA|2050-12-13|1811|7877|605|2050|2|12|13|4|2050|605|7877|Tuesday|2050Q4|N|N|N|2470142|2470475|2469789|2470062|N|N|N|N|N| +2470155|AAAAAAAALABLFCAA|2050-12-14|1811|7877|605|2050|3|12|14|4|2050|605|7877|Wednesday|2050Q4|N|N|N|2470142|2470475|2469790|2470063|N|N|N|N|N| +2470156|AAAAAAAAMABLFCAA|2050-12-15|1811|7877|605|2050|4|12|15|4|2050|605|7877|Thursday|2050Q4|N|N|N|2470142|2470475|2469791|2470064|N|N|N|N|N| +2470157|AAAAAAAANABLFCAA|2050-12-16|1811|7877|605|2050|5|12|16|4|2050|605|7877|Friday|2050Q4|N|Y|N|2470142|2470475|2469792|2470065|N|N|N|N|N| +2470158|AAAAAAAAOABLFCAA|2050-12-17|1811|7877|605|2050|6|12|17|4|2050|605|7877|Saturday|2050Q4|N|Y|N|2470142|2470475|2469793|2470066|N|N|N|N|N| +2470159|AAAAAAAAPABLFCAA|2050-12-18|1811|7877|605|2050|0|12|18|4|2050|605|7877|Sunday|2050Q4|N|N|N|2470142|2470475|2469794|2470067|N|N|N|N|N| +2470160|AAAAAAAAABBLFCAA|2050-12-19|1811|7877|605|2050|1|12|19|4|2050|605|7877|Monday|2050Q4|N|N|N|2470142|2470475|2469795|2470068|N|N|N|N|N| +2470161|AAAAAAAABBBLFCAA|2050-12-20|1811|7878|605|2050|2|12|20|4|2050|605|7878|Tuesday|2050Q4|N|N|N|2470142|2470475|2469796|2470069|N|N|N|N|N| +2470162|AAAAAAAACBBLFCAA|2050-12-21|1811|7878|605|2050|3|12|21|4|2050|605|7878|Wednesday|2050Q4|N|N|N|2470142|2470475|2469797|2470070|N|N|N|N|N| +2470163|AAAAAAAADBBLFCAA|2050-12-22|1811|7878|605|2050|4|12|22|4|2050|605|7878|Thursday|2050Q4|N|N|N|2470142|2470475|2469798|2470071|N|N|N|N|N| +2470164|AAAAAAAAEBBLFCAA|2050-12-23|1811|7878|605|2050|5|12|23|4|2050|605|7878|Friday|2050Q4|N|Y|N|2470142|2470475|2469799|2470072|N|N|N|N|N| +2470165|AAAAAAAAFBBLFCAA|2050-12-24|1811|7878|605|2050|6|12|24|4|2050|605|7878|Saturday|2050Q4|N|Y|N|2470142|2470475|2469800|2470073|N|N|N|N|N| +2470166|AAAAAAAAGBBLFCAA|2050-12-25|1811|7878|605|2050|0|12|25|4|2050|605|7878|Sunday|2050Q4|N|N|N|2470142|2470475|2469801|2470074|N|N|N|N|N| +2470167|AAAAAAAAHBBLFCAA|2050-12-26|1811|7878|605|2050|1|12|26|4|2050|605|7878|Monday|2050Q4|Y|N|N|2470142|2470475|2469802|2470075|N|N|N|N|N| +2470168|AAAAAAAAIBBLFCAA|2050-12-27|1811|7879|605|2050|2|12|27|4|2050|605|7879|Tuesday|2050Q4|N|N|Y|2470142|2470475|2469803|2470076|N|N|N|N|N| +2470169|AAAAAAAAJBBLFCAA|2050-12-28|1811|7879|605|2050|3|12|28|4|2050|605|7879|Wednesday|2050Q4|N|N|N|2470142|2470475|2469804|2470077|N|N|N|N|N| +2470170|AAAAAAAAKBBLFCAA|2050-12-29|1811|7879|605|2050|4|12|29|4|2050|605|7879|Thursday|2050Q4|N|N|N|2470142|2470475|2469805|2470078|N|N|N|N|N| +2470171|AAAAAAAALBBLFCAA|2050-12-30|1811|7879|605|2050|5|12|30|4|2050|605|7879|Friday|2050Q4|N|Y|N|2470142|2470475|2469806|2470079|N|N|N|N|N| +2470172|AAAAAAAAMBBLFCAA|2050-12-31|1811|7879|605|2050|6|12|31|4|2050|605|7879|Saturday|2050Q4|N|Y|N|2470142|2470475|2469807|2470080|N|N|N|N|N| +2470173|AAAAAAAANBBLFCAA|2051-01-01|1812|7879|605|2051|0|1|1|1|2051|605|7879|Sunday|2051Q1|Y|N|N|2470173|2470172|2469808|2470081|N|N|N|N|N| +2470174|AAAAAAAAOBBLFCAA|2051-01-02|1812|7879|605|2051|1|1|2|1|2051|605|7879|Monday|2051Q1|N|N|Y|2470173|2470172|2469809|2470082|N|N|N|N|N| +2470175|AAAAAAAAPBBLFCAA|2051-01-03|1812|7880|605|2051|2|1|3|1|2051|605|7880|Tuesday|2051Q1|N|N|N|2470173|2470172|2469810|2470083|N|N|N|N|N| +2470176|AAAAAAAAACBLFCAA|2051-01-04|1812|7880|605|2051|3|1|4|1|2051|605|7880|Wednesday|2051Q1|N|N|N|2470173|2470172|2469811|2470084|N|N|N|N|N| +2470177|AAAAAAAABCBLFCAA|2051-01-05|1812|7880|605|2051|4|1|5|1|2051|605|7880|Thursday|2051Q1|N|N|N|2470173|2470172|2469812|2470085|N|N|N|N|N| +2470178|AAAAAAAACCBLFCAA|2051-01-06|1812|7880|605|2051|5|1|6|1|2051|605|7880|Friday|2051Q1|N|Y|N|2470173|2470172|2469813|2470086|N|N|N|N|N| +2470179|AAAAAAAADCBLFCAA|2051-01-07|1812|7880|605|2051|6|1|7|1|2051|605|7880|Saturday|2051Q1|N|Y|N|2470173|2470172|2469814|2470087|N|N|N|N|N| +2470180|AAAAAAAAECBLFCAA|2051-01-08|1812|7880|605|2051|0|1|8|1|2051|605|7880|Sunday|2051Q1|N|N|N|2470173|2470172|2469815|2470088|N|N|N|N|N| +2470181|AAAAAAAAFCBLFCAA|2051-01-09|1812|7880|605|2051|1|1|9|1|2051|605|7880|Monday|2051Q1|N|N|N|2470173|2470172|2469816|2470089|N|N|N|N|N| +2470182|AAAAAAAAGCBLFCAA|2051-01-10|1812|7881|605|2051|2|1|10|1|2051|605|7881|Tuesday|2051Q1|N|N|N|2470173|2470172|2469817|2470090|N|N|N|N|N| +2470183|AAAAAAAAHCBLFCAA|2051-01-11|1812|7881|605|2051|3|1|11|1|2051|605|7881|Wednesday|2051Q1|N|N|N|2470173|2470172|2469818|2470091|N|N|N|N|N| +2470184|AAAAAAAAICBLFCAA|2051-01-12|1812|7881|605|2051|4|1|12|1|2051|605|7881|Thursday|2051Q1|N|N|N|2470173|2470172|2469819|2470092|N|N|N|N|N| +2470185|AAAAAAAAJCBLFCAA|2051-01-13|1812|7881|605|2051|5|1|13|1|2051|605|7881|Friday|2051Q1|N|Y|N|2470173|2470172|2469820|2470093|N|N|N|N|N| +2470186|AAAAAAAAKCBLFCAA|2051-01-14|1812|7881|605|2051|6|1|14|1|2051|605|7881|Saturday|2051Q1|N|Y|N|2470173|2470172|2469821|2470094|N|N|N|N|N| +2470187|AAAAAAAALCBLFCAA|2051-01-15|1812|7881|605|2051|0|1|15|1|2051|605|7881|Sunday|2051Q1|N|N|N|2470173|2470172|2469822|2470095|N|N|N|N|N| +2470188|AAAAAAAAMCBLFCAA|2051-01-16|1812|7881|605|2051|1|1|16|1|2051|605|7881|Monday|2051Q1|N|N|N|2470173|2470172|2469823|2470096|N|N|N|N|N| +2470189|AAAAAAAANCBLFCAA|2051-01-17|1812|7882|605|2051|2|1|17|1|2051|605|7882|Tuesday|2051Q1|N|N|N|2470173|2470172|2469824|2470097|N|N|N|N|N| +2470190|AAAAAAAAOCBLFCAA|2051-01-18|1812|7882|605|2051|3|1|18|1|2051|605|7882|Wednesday|2051Q1|N|N|N|2470173|2470172|2469825|2470098|N|N|N|N|N| +2470191|AAAAAAAAPCBLFCAA|2051-01-19|1812|7882|605|2051|4|1|19|1|2051|605|7882|Thursday|2051Q1|N|N|N|2470173|2470172|2469826|2470099|N|N|N|N|N| +2470192|AAAAAAAAADBLFCAA|2051-01-20|1812|7882|605|2051|5|1|20|1|2051|605|7882|Friday|2051Q1|N|Y|N|2470173|2470172|2469827|2470100|N|N|N|N|N| +2470193|AAAAAAAABDBLFCAA|2051-01-21|1812|7882|605|2051|6|1|21|1|2051|605|7882|Saturday|2051Q1|N|Y|N|2470173|2470172|2469828|2470101|N|N|N|N|N| +2470194|AAAAAAAACDBLFCAA|2051-01-22|1812|7882|605|2051|0|1|22|1|2051|605|7882|Sunday|2051Q1|N|N|N|2470173|2470172|2469829|2470102|N|N|N|N|N| +2470195|AAAAAAAADDBLFCAA|2051-01-23|1812|7882|605|2051|1|1|23|1|2051|605|7882|Monday|2051Q1|N|N|N|2470173|2470172|2469830|2470103|N|N|N|N|N| +2470196|AAAAAAAAEDBLFCAA|2051-01-24|1812|7883|605|2051|2|1|24|1|2051|605|7883|Tuesday|2051Q1|N|N|N|2470173|2470172|2469831|2470104|N|N|N|N|N| +2470197|AAAAAAAAFDBLFCAA|2051-01-25|1812|7883|605|2051|3|1|25|1|2051|605|7883|Wednesday|2051Q1|N|N|N|2470173|2470172|2469832|2470105|N|N|N|N|N| +2470198|AAAAAAAAGDBLFCAA|2051-01-26|1812|7883|605|2051|4|1|26|1|2051|605|7883|Thursday|2051Q1|N|N|N|2470173|2470172|2469833|2470106|N|N|N|N|N| +2470199|AAAAAAAAHDBLFCAA|2051-01-27|1812|7883|605|2051|5|1|27|1|2051|605|7883|Friday|2051Q1|N|Y|N|2470173|2470172|2469834|2470107|N|N|N|N|N| +2470200|AAAAAAAAIDBLFCAA|2051-01-28|1812|7883|605|2051|6|1|28|1|2051|605|7883|Saturday|2051Q1|N|Y|N|2470173|2470172|2469835|2470108|N|N|N|N|N| +2470201|AAAAAAAAJDBLFCAA|2051-01-29|1812|7883|605|2051|0|1|29|1|2051|605|7883|Sunday|2051Q1|N|N|N|2470173|2470172|2469836|2470109|N|N|N|N|N| +2470202|AAAAAAAAKDBLFCAA|2051-01-30|1812|7883|605|2051|1|1|30|1|2051|605|7883|Monday|2051Q1|N|N|N|2470173|2470172|2469837|2470110|N|N|N|N|N| +2470203|AAAAAAAALDBLFCAA|2051-01-31|1812|7884|605|2051|2|1|31|1|2051|605|7884|Tuesday|2051Q1|N|N|N|2470173|2470172|2469838|2470111|N|N|N|N|N| +2470204|AAAAAAAAMDBLFCAA|2051-02-01|1813|7884|605|2051|3|2|1|1|2051|605|7884|Wednesday|2051Q1|N|N|N|2470204|2470234|2469839|2470112|N|N|N|N|N| +2470205|AAAAAAAANDBLFCAA|2051-02-02|1813|7884|605|2051|4|2|2|1|2051|605|7884|Thursday|2051Q1|N|N|N|2470204|2470234|2469840|2470113|N|N|N|N|N| +2470206|AAAAAAAAODBLFCAA|2051-02-03|1813|7884|605|2051|5|2|3|1|2051|605|7884|Friday|2051Q1|N|Y|N|2470204|2470234|2469841|2470114|N|N|N|N|N| +2470207|AAAAAAAAPDBLFCAA|2051-02-04|1813|7884|605|2051|6|2|4|1|2051|605|7884|Saturday|2051Q1|N|Y|N|2470204|2470234|2469842|2470115|N|N|N|N|N| +2470208|AAAAAAAAAEBLFCAA|2051-02-05|1813|7884|605|2051|0|2|5|1|2051|605|7884|Sunday|2051Q1|N|N|N|2470204|2470234|2469843|2470116|N|N|N|N|N| +2470209|AAAAAAAABEBLFCAA|2051-02-06|1813|7884|605|2051|1|2|6|1|2051|605|7884|Monday|2051Q1|N|N|N|2470204|2470234|2469844|2470117|N|N|N|N|N| +2470210|AAAAAAAACEBLFCAA|2051-02-07|1813|7885|605|2051|2|2|7|1|2051|605|7885|Tuesday|2051Q1|N|N|N|2470204|2470234|2469845|2470118|N|N|N|N|N| +2470211|AAAAAAAADEBLFCAA|2051-02-08|1813|7885|605|2051|3|2|8|1|2051|605|7885|Wednesday|2051Q1|N|N|N|2470204|2470234|2469846|2470119|N|N|N|N|N| +2470212|AAAAAAAAEEBLFCAA|2051-02-09|1813|7885|605|2051|4|2|9|1|2051|605|7885|Thursday|2051Q1|N|N|N|2470204|2470234|2469847|2470120|N|N|N|N|N| +2470213|AAAAAAAAFEBLFCAA|2051-02-10|1813|7885|605|2051|5|2|10|1|2051|605|7885|Friday|2051Q1|N|Y|N|2470204|2470234|2469848|2470121|N|N|N|N|N| +2470214|AAAAAAAAGEBLFCAA|2051-02-11|1813|7885|605|2051|6|2|11|1|2051|605|7885|Saturday|2051Q1|N|Y|N|2470204|2470234|2469849|2470122|N|N|N|N|N| +2470215|AAAAAAAAHEBLFCAA|2051-02-12|1813|7885|605|2051|0|2|12|1|2051|605|7885|Sunday|2051Q1|N|N|N|2470204|2470234|2469850|2470123|N|N|N|N|N| +2470216|AAAAAAAAIEBLFCAA|2051-02-13|1813|7885|605|2051|1|2|13|1|2051|605|7885|Monday|2051Q1|N|N|N|2470204|2470234|2469851|2470124|N|N|N|N|N| +2470217|AAAAAAAAJEBLFCAA|2051-02-14|1813|7886|605|2051|2|2|14|1|2051|605|7886|Tuesday|2051Q1|N|N|N|2470204|2470234|2469852|2470125|N|N|N|N|N| +2470218|AAAAAAAAKEBLFCAA|2051-02-15|1813|7886|605|2051|3|2|15|1|2051|605|7886|Wednesday|2051Q1|N|N|N|2470204|2470234|2469853|2470126|N|N|N|N|N| +2470219|AAAAAAAALEBLFCAA|2051-02-16|1813|7886|605|2051|4|2|16|1|2051|605|7886|Thursday|2051Q1|N|N|N|2470204|2470234|2469854|2470127|N|N|N|N|N| +2470220|AAAAAAAAMEBLFCAA|2051-02-17|1813|7886|605|2051|5|2|17|1|2051|605|7886|Friday|2051Q1|N|Y|N|2470204|2470234|2469855|2470128|N|N|N|N|N| +2470221|AAAAAAAANEBLFCAA|2051-02-18|1813|7886|605|2051|6|2|18|1|2051|605|7886|Saturday|2051Q1|N|Y|N|2470204|2470234|2469856|2470129|N|N|N|N|N| +2470222|AAAAAAAAOEBLFCAA|2051-02-19|1813|7886|605|2051|0|2|19|1|2051|605|7886|Sunday|2051Q1|N|N|N|2470204|2470234|2469857|2470130|N|N|N|N|N| +2470223|AAAAAAAAPEBLFCAA|2051-02-20|1813|7886|605|2051|1|2|20|1|2051|605|7886|Monday|2051Q1|N|N|N|2470204|2470234|2469858|2470131|N|N|N|N|N| +2470224|AAAAAAAAAFBLFCAA|2051-02-21|1813|7887|605|2051|2|2|21|1|2051|605|7887|Tuesday|2051Q1|N|N|N|2470204|2470234|2469859|2470132|N|N|N|N|N| +2470225|AAAAAAAABFBLFCAA|2051-02-22|1813|7887|605|2051|3|2|22|1|2051|605|7887|Wednesday|2051Q1|N|N|N|2470204|2470234|2469860|2470133|N|N|N|N|N| +2470226|AAAAAAAACFBLFCAA|2051-02-23|1813|7887|605|2051|4|2|23|1|2051|605|7887|Thursday|2051Q1|N|N|N|2470204|2470234|2469861|2470134|N|N|N|N|N| +2470227|AAAAAAAADFBLFCAA|2051-02-24|1813|7887|605|2051|5|2|24|1|2051|605|7887|Friday|2051Q1|N|Y|N|2470204|2470234|2469862|2470135|N|N|N|N|N| +2470228|AAAAAAAAEFBLFCAA|2051-02-25|1813|7887|605|2051|6|2|25|1|2051|605|7887|Saturday|2051Q1|N|Y|N|2470204|2470234|2469863|2470136|N|N|N|N|N| +2470229|AAAAAAAAFFBLFCAA|2051-02-26|1813|7887|605|2051|0|2|26|1|2051|605|7887|Sunday|2051Q1|N|N|N|2470204|2470234|2469864|2470137|N|N|N|N|N| +2470230|AAAAAAAAGFBLFCAA|2051-02-27|1813|7887|605|2051|1|2|27|1|2051|605|7887|Monday|2051Q1|N|N|N|2470204|2470234|2469865|2470138|N|N|N|N|N| +2470231|AAAAAAAAHFBLFCAA|2051-02-28|1813|7888|605|2051|2|2|28|1|2051|605|7888|Tuesday|2051Q1|N|N|N|2470204|2470234|2469866|2470139|N|N|N|N|N| +2470232|AAAAAAAAIFBLFCAA|2051-03-01|1814|7888|606|2051|3|3|1|1|2051|606|7888|Wednesday|2051Q1|N|N|N|2470232|2470290|2469867|2470140|N|N|N|N|N| +2470233|AAAAAAAAJFBLFCAA|2051-03-02|1814|7888|606|2051|4|3|2|1|2051|606|7888|Thursday|2051Q1|N|N|N|2470232|2470290|2469868|2470141|N|N|N|N|N| +2470234|AAAAAAAAKFBLFCAA|2051-03-03|1814|7888|606|2051|5|3|3|1|2051|606|7888|Friday|2051Q1|N|Y|N|2470232|2470290|2469869|2470142|N|N|N|N|N| +2470235|AAAAAAAALFBLFCAA|2051-03-04|1814|7888|606|2051|6|3|4|1|2051|606|7888|Saturday|2051Q1|N|Y|N|2470232|2470290|2469870|2470143|N|N|N|N|N| +2470236|AAAAAAAAMFBLFCAA|2051-03-05|1814|7888|606|2051|0|3|5|1|2051|606|7888|Sunday|2051Q1|N|N|N|2470232|2470290|2469871|2470144|N|N|N|N|N| +2470237|AAAAAAAANFBLFCAA|2051-03-06|1814|7888|606|2051|1|3|6|1|2051|606|7888|Monday|2051Q1|N|N|N|2470232|2470290|2469872|2470145|N|N|N|N|N| +2470238|AAAAAAAAOFBLFCAA|2051-03-07|1814|7889|606|2051|2|3|7|1|2051|606|7889|Tuesday|2051Q1|N|N|N|2470232|2470290|2469873|2470146|N|N|N|N|N| +2470239|AAAAAAAAPFBLFCAA|2051-03-08|1814|7889|606|2051|3|3|8|1|2051|606|7889|Wednesday|2051Q1|N|N|N|2470232|2470290|2469874|2470147|N|N|N|N|N| +2470240|AAAAAAAAAGBLFCAA|2051-03-09|1814|7889|606|2051|4|3|9|1|2051|606|7889|Thursday|2051Q1|N|N|N|2470232|2470290|2469875|2470148|N|N|N|N|N| +2470241|AAAAAAAABGBLFCAA|2051-03-10|1814|7889|606|2051|5|3|10|1|2051|606|7889|Friday|2051Q1|N|Y|N|2470232|2470290|2469876|2470149|N|N|N|N|N| +2470242|AAAAAAAACGBLFCAA|2051-03-11|1814|7889|606|2051|6|3|11|1|2051|606|7889|Saturday|2051Q1|N|Y|N|2470232|2470290|2469877|2470150|N|N|N|N|N| +2470243|AAAAAAAADGBLFCAA|2051-03-12|1814|7889|606|2051|0|3|12|1|2051|606|7889|Sunday|2051Q1|N|N|N|2470232|2470290|2469878|2470151|N|N|N|N|N| +2470244|AAAAAAAAEGBLFCAA|2051-03-13|1814|7889|606|2051|1|3|13|1|2051|606|7889|Monday|2051Q1|N|N|N|2470232|2470290|2469879|2470152|N|N|N|N|N| +2470245|AAAAAAAAFGBLFCAA|2051-03-14|1814|7890|606|2051|2|3|14|1|2051|606|7890|Tuesday|2051Q1|N|N|N|2470232|2470290|2469880|2470153|N|N|N|N|N| +2470246|AAAAAAAAGGBLFCAA|2051-03-15|1814|7890|606|2051|3|3|15|1|2051|606|7890|Wednesday|2051Q1|N|N|N|2470232|2470290|2469881|2470154|N|N|N|N|N| +2470247|AAAAAAAAHGBLFCAA|2051-03-16|1814|7890|606|2051|4|3|16|1|2051|606|7890|Thursday|2051Q1|N|N|N|2470232|2470290|2469882|2470155|N|N|N|N|N| +2470248|AAAAAAAAIGBLFCAA|2051-03-17|1814|7890|606|2051|5|3|17|1|2051|606|7890|Friday|2051Q1|N|Y|N|2470232|2470290|2469883|2470156|N|N|N|N|N| +2470249|AAAAAAAAJGBLFCAA|2051-03-18|1814|7890|606|2051|6|3|18|1|2051|606|7890|Saturday|2051Q1|N|Y|N|2470232|2470290|2469884|2470157|N|N|N|N|N| +2470250|AAAAAAAAKGBLFCAA|2051-03-19|1814|7890|606|2051|0|3|19|1|2051|606|7890|Sunday|2051Q1|N|N|N|2470232|2470290|2469885|2470158|N|N|N|N|N| +2470251|AAAAAAAALGBLFCAA|2051-03-20|1814|7890|606|2051|1|3|20|1|2051|606|7890|Monday|2051Q1|N|N|N|2470232|2470290|2469886|2470159|N|N|N|N|N| +2470252|AAAAAAAAMGBLFCAA|2051-03-21|1814|7891|606|2051|2|3|21|1|2051|606|7891|Tuesday|2051Q1|N|N|N|2470232|2470290|2469887|2470160|N|N|N|N|N| +2470253|AAAAAAAANGBLFCAA|2051-03-22|1814|7891|606|2051|3|3|22|1|2051|606|7891|Wednesday|2051Q1|N|N|N|2470232|2470290|2469888|2470161|N|N|N|N|N| +2470254|AAAAAAAAOGBLFCAA|2051-03-23|1814|7891|606|2051|4|3|23|1|2051|606|7891|Thursday|2051Q1|N|N|N|2470232|2470290|2469889|2470162|N|N|N|N|N| +2470255|AAAAAAAAPGBLFCAA|2051-03-24|1814|7891|606|2051|5|3|24|1|2051|606|7891|Friday|2051Q1|N|Y|N|2470232|2470290|2469890|2470163|N|N|N|N|N| +2470256|AAAAAAAAAHBLFCAA|2051-03-25|1814|7891|606|2051|6|3|25|1|2051|606|7891|Saturday|2051Q1|N|Y|N|2470232|2470290|2469891|2470164|N|N|N|N|N| +2470257|AAAAAAAABHBLFCAA|2051-03-26|1814|7891|606|2051|0|3|26|1|2051|606|7891|Sunday|2051Q1|N|N|N|2470232|2470290|2469892|2470165|N|N|N|N|N| +2470258|AAAAAAAACHBLFCAA|2051-03-27|1814|7891|606|2051|1|3|27|1|2051|606|7891|Monday|2051Q1|N|N|N|2470232|2470290|2469893|2470166|N|N|N|N|N| +2470259|AAAAAAAADHBLFCAA|2051-03-28|1814|7892|606|2051|2|3|28|1|2051|606|7892|Tuesday|2051Q1|N|N|N|2470232|2470290|2469894|2470167|N|N|N|N|N| +2470260|AAAAAAAAEHBLFCAA|2051-03-29|1814|7892|606|2051|3|3|29|1|2051|606|7892|Wednesday|2051Q1|N|N|N|2470232|2470290|2469895|2470168|N|N|N|N|N| +2470261|AAAAAAAAFHBLFCAA|2051-03-30|1814|7892|606|2051|4|3|30|1|2051|606|7892|Thursday|2051Q1|N|N|N|2470232|2470290|2469896|2470169|N|N|N|N|N| +2470262|AAAAAAAAGHBLFCAA|2051-03-31|1814|7892|606|2051|5|3|31|1|2051|606|7892|Friday|2051Q1|N|Y|N|2470232|2470290|2469897|2470170|N|N|N|N|N| +2470263|AAAAAAAAHHBLFCAA|2051-04-01|1815|7892|606|2051|6|4|1|1|2051|606|7892|Saturday|2051Q1|N|Y|N|2470263|2470352|2469898|2470173|N|N|N|N|N| +2470264|AAAAAAAAIHBLFCAA|2051-04-02|1815|7892|606|2051|0|4|2|2|2051|606|7892|Sunday|2051Q2|N|N|N|2470263|2470352|2469899|2470174|N|N|N|N|N| +2470265|AAAAAAAAJHBLFCAA|2051-04-03|1815|7892|606|2051|1|4|3|2|2051|606|7892|Monday|2051Q2|N|N|N|2470263|2470352|2469900|2470175|N|N|N|N|N| +2470266|AAAAAAAAKHBLFCAA|2051-04-04|1815|7893|606|2051|2|4|4|2|2051|606|7893|Tuesday|2051Q2|N|N|N|2470263|2470352|2469901|2470176|N|N|N|N|N| +2470267|AAAAAAAALHBLFCAA|2051-04-05|1815|7893|606|2051|3|4|5|2|2051|606|7893|Wednesday|2051Q2|N|N|N|2470263|2470352|2469902|2470177|N|N|N|N|N| +2470268|AAAAAAAAMHBLFCAA|2051-04-06|1815|7893|606|2051|4|4|6|2|2051|606|7893|Thursday|2051Q2|N|N|N|2470263|2470352|2469903|2470178|N|N|N|N|N| +2470269|AAAAAAAANHBLFCAA|2051-04-07|1815|7893|606|2051|5|4|7|2|2051|606|7893|Friday|2051Q2|N|Y|N|2470263|2470352|2469904|2470179|N|N|N|N|N| +2470270|AAAAAAAAOHBLFCAA|2051-04-08|1815|7893|606|2051|6|4|8|2|2051|606|7893|Saturday|2051Q2|N|Y|N|2470263|2470352|2469905|2470180|N|N|N|N|N| +2470271|AAAAAAAAPHBLFCAA|2051-04-09|1815|7893|606|2051|0|4|9|2|2051|606|7893|Sunday|2051Q2|N|N|N|2470263|2470352|2469906|2470181|N|N|N|N|N| +2470272|AAAAAAAAAIBLFCAA|2051-04-10|1815|7893|606|2051|1|4|10|2|2051|606|7893|Monday|2051Q2|N|N|N|2470263|2470352|2469907|2470182|N|N|N|N|N| +2470273|AAAAAAAABIBLFCAA|2051-04-11|1815|7894|606|2051|2|4|11|2|2051|606|7894|Tuesday|2051Q2|N|N|N|2470263|2470352|2469908|2470183|N|N|N|N|N| +2470274|AAAAAAAACIBLFCAA|2051-04-12|1815|7894|606|2051|3|4|12|2|2051|606|7894|Wednesday|2051Q2|N|N|N|2470263|2470352|2469909|2470184|N|N|N|N|N| +2470275|AAAAAAAADIBLFCAA|2051-04-13|1815|7894|606|2051|4|4|13|2|2051|606|7894|Thursday|2051Q2|N|N|N|2470263|2470352|2469910|2470185|N|N|N|N|N| +2470276|AAAAAAAAEIBLFCAA|2051-04-14|1815|7894|606|2051|5|4|14|2|2051|606|7894|Friday|2051Q2|N|Y|N|2470263|2470352|2469911|2470186|N|N|N|N|N| +2470277|AAAAAAAAFIBLFCAA|2051-04-15|1815|7894|606|2051|6|4|15|2|2051|606|7894|Saturday|2051Q2|N|Y|N|2470263|2470352|2469912|2470187|N|N|N|N|N| +2470278|AAAAAAAAGIBLFCAA|2051-04-16|1815|7894|606|2051|0|4|16|2|2051|606|7894|Sunday|2051Q2|N|N|N|2470263|2470352|2469913|2470188|N|N|N|N|N| +2470279|AAAAAAAAHIBLFCAA|2051-04-17|1815|7894|606|2051|1|4|17|2|2051|606|7894|Monday|2051Q2|N|N|N|2470263|2470352|2469914|2470189|N|N|N|N|N| +2470280|AAAAAAAAIIBLFCAA|2051-04-18|1815|7895|606|2051|2|4|18|2|2051|606|7895|Tuesday|2051Q2|N|N|N|2470263|2470352|2469915|2470190|N|N|N|N|N| +2470281|AAAAAAAAJIBLFCAA|2051-04-19|1815|7895|606|2051|3|4|19|2|2051|606|7895|Wednesday|2051Q2|N|N|N|2470263|2470352|2469916|2470191|N|N|N|N|N| +2470282|AAAAAAAAKIBLFCAA|2051-04-20|1815|7895|606|2051|4|4|20|2|2051|606|7895|Thursday|2051Q2|N|N|N|2470263|2470352|2469917|2470192|N|N|N|N|N| +2470283|AAAAAAAALIBLFCAA|2051-04-21|1815|7895|606|2051|5|4|21|2|2051|606|7895|Friday|2051Q2|N|Y|N|2470263|2470352|2469918|2470193|N|N|N|N|N| +2470284|AAAAAAAAMIBLFCAA|2051-04-22|1815|7895|606|2051|6|4|22|2|2051|606|7895|Saturday|2051Q2|N|Y|N|2470263|2470352|2469919|2470194|N|N|N|N|N| +2470285|AAAAAAAANIBLFCAA|2051-04-23|1815|7895|606|2051|0|4|23|2|2051|606|7895|Sunday|2051Q2|N|N|N|2470263|2470352|2469920|2470195|N|N|N|N|N| +2470286|AAAAAAAAOIBLFCAA|2051-04-24|1815|7895|606|2051|1|4|24|2|2051|606|7895|Monday|2051Q2|N|N|N|2470263|2470352|2469921|2470196|N|N|N|N|N| +2470287|AAAAAAAAPIBLFCAA|2051-04-25|1815|7896|606|2051|2|4|25|2|2051|606|7896|Tuesday|2051Q2|N|N|N|2470263|2470352|2469922|2470197|N|N|N|N|N| +2470288|AAAAAAAAAJBLFCAA|2051-04-26|1815|7896|606|2051|3|4|26|2|2051|606|7896|Wednesday|2051Q2|N|N|N|2470263|2470352|2469923|2470198|N|N|N|N|N| +2470289|AAAAAAAABJBLFCAA|2051-04-27|1815|7896|606|2051|4|4|27|2|2051|606|7896|Thursday|2051Q2|N|N|N|2470263|2470352|2469924|2470199|N|N|N|N|N| +2470290|AAAAAAAACJBLFCAA|2051-04-28|1815|7896|606|2051|5|4|28|2|2051|606|7896|Friday|2051Q2|N|Y|N|2470263|2470352|2469925|2470200|N|N|N|N|N| +2470291|AAAAAAAADJBLFCAA|2051-04-29|1815|7896|606|2051|6|4|29|2|2051|606|7896|Saturday|2051Q2|N|Y|N|2470263|2470352|2469926|2470201|N|N|N|N|N| +2470292|AAAAAAAAEJBLFCAA|2051-04-30|1815|7896|606|2051|0|4|30|2|2051|606|7896|Sunday|2051Q2|N|N|N|2470263|2470352|2469927|2470202|N|N|N|N|N| +2470293|AAAAAAAAFJBLFCAA|2051-05-01|1816|7896|606|2051|1|5|1|2|2051|606|7896|Monday|2051Q2|N|N|N|2470293|2470412|2469928|2470203|N|N|N|N|N| +2470294|AAAAAAAAGJBLFCAA|2051-05-02|1816|7897|606|2051|2|5|2|2|2051|606|7897|Tuesday|2051Q2|N|N|N|2470293|2470412|2469929|2470204|N|N|N|N|N| +2470295|AAAAAAAAHJBLFCAA|2051-05-03|1816|7897|606|2051|3|5|3|2|2051|606|7897|Wednesday|2051Q2|N|N|N|2470293|2470412|2469930|2470205|N|N|N|N|N| +2470296|AAAAAAAAIJBLFCAA|2051-05-04|1816|7897|606|2051|4|5|4|2|2051|606|7897|Thursday|2051Q2|N|N|N|2470293|2470412|2469931|2470206|N|N|N|N|N| +2470297|AAAAAAAAJJBLFCAA|2051-05-05|1816|7897|606|2051|5|5|5|2|2051|606|7897|Friday|2051Q2|N|Y|N|2470293|2470412|2469932|2470207|N|N|N|N|N| +2470298|AAAAAAAAKJBLFCAA|2051-05-06|1816|7897|606|2051|6|5|6|2|2051|606|7897|Saturday|2051Q2|N|Y|N|2470293|2470412|2469933|2470208|N|N|N|N|N| +2470299|AAAAAAAALJBLFCAA|2051-05-07|1816|7897|606|2051|0|5|7|2|2051|606|7897|Sunday|2051Q2|N|N|N|2470293|2470412|2469934|2470209|N|N|N|N|N| +2470300|AAAAAAAAMJBLFCAA|2051-05-08|1816|7897|606|2051|1|5|8|2|2051|606|7897|Monday|2051Q2|N|N|N|2470293|2470412|2469935|2470210|N|N|N|N|N| +2470301|AAAAAAAANJBLFCAA|2051-05-09|1816|7898|606|2051|2|5|9|2|2051|606|7898|Tuesday|2051Q2|N|N|N|2470293|2470412|2469936|2470211|N|N|N|N|N| +2470302|AAAAAAAAOJBLFCAA|2051-05-10|1816|7898|606|2051|3|5|10|2|2051|606|7898|Wednesday|2051Q2|N|N|N|2470293|2470412|2469937|2470212|N|N|N|N|N| +2470303|AAAAAAAAPJBLFCAA|2051-05-11|1816|7898|606|2051|4|5|11|2|2051|606|7898|Thursday|2051Q2|N|N|N|2470293|2470412|2469938|2470213|N|N|N|N|N| +2470304|AAAAAAAAAKBLFCAA|2051-05-12|1816|7898|606|2051|5|5|12|2|2051|606|7898|Friday|2051Q2|N|Y|N|2470293|2470412|2469939|2470214|N|N|N|N|N| +2470305|AAAAAAAABKBLFCAA|2051-05-13|1816|7898|606|2051|6|5|13|2|2051|606|7898|Saturday|2051Q2|N|Y|N|2470293|2470412|2469940|2470215|N|N|N|N|N| +2470306|AAAAAAAACKBLFCAA|2051-05-14|1816|7898|606|2051|0|5|14|2|2051|606|7898|Sunday|2051Q2|N|N|N|2470293|2470412|2469941|2470216|N|N|N|N|N| +2470307|AAAAAAAADKBLFCAA|2051-05-15|1816|7898|606|2051|1|5|15|2|2051|606|7898|Monday|2051Q2|N|N|N|2470293|2470412|2469942|2470217|N|N|N|N|N| +2470308|AAAAAAAAEKBLFCAA|2051-05-16|1816|7899|606|2051|2|5|16|2|2051|606|7899|Tuesday|2051Q2|N|N|N|2470293|2470412|2469943|2470218|N|N|N|N|N| +2470309|AAAAAAAAFKBLFCAA|2051-05-17|1816|7899|606|2051|3|5|17|2|2051|606|7899|Wednesday|2051Q2|N|N|N|2470293|2470412|2469944|2470219|N|N|N|N|N| +2470310|AAAAAAAAGKBLFCAA|2051-05-18|1816|7899|606|2051|4|5|18|2|2051|606|7899|Thursday|2051Q2|N|N|N|2470293|2470412|2469945|2470220|N|N|N|N|N| +2470311|AAAAAAAAHKBLFCAA|2051-05-19|1816|7899|606|2051|5|5|19|2|2051|606|7899|Friday|2051Q2|N|Y|N|2470293|2470412|2469946|2470221|N|N|N|N|N| +2470312|AAAAAAAAIKBLFCAA|2051-05-20|1816|7899|606|2051|6|5|20|2|2051|606|7899|Saturday|2051Q2|N|Y|N|2470293|2470412|2469947|2470222|N|N|N|N|N| +2470313|AAAAAAAAJKBLFCAA|2051-05-21|1816|7899|606|2051|0|5|21|2|2051|606|7899|Sunday|2051Q2|N|N|N|2470293|2470412|2469948|2470223|N|N|N|N|N| +2470314|AAAAAAAAKKBLFCAA|2051-05-22|1816|7899|606|2051|1|5|22|2|2051|606|7899|Monday|2051Q2|N|N|N|2470293|2470412|2469949|2470224|N|N|N|N|N| +2470315|AAAAAAAALKBLFCAA|2051-05-23|1816|7900|606|2051|2|5|23|2|2051|606|7900|Tuesday|2051Q2|N|N|N|2470293|2470412|2469950|2470225|N|N|N|N|N| +2470316|AAAAAAAAMKBLFCAA|2051-05-24|1816|7900|606|2051|3|5|24|2|2051|606|7900|Wednesday|2051Q2|N|N|N|2470293|2470412|2469951|2470226|N|N|N|N|N| +2470317|AAAAAAAANKBLFCAA|2051-05-25|1816|7900|606|2051|4|5|25|2|2051|606|7900|Thursday|2051Q2|N|N|N|2470293|2470412|2469952|2470227|N|N|N|N|N| +2470318|AAAAAAAAOKBLFCAA|2051-05-26|1816|7900|606|2051|5|5|26|2|2051|606|7900|Friday|2051Q2|N|Y|N|2470293|2470412|2469953|2470228|N|N|N|N|N| +2470319|AAAAAAAAPKBLFCAA|2051-05-27|1816|7900|606|2051|6|5|27|2|2051|606|7900|Saturday|2051Q2|N|Y|N|2470293|2470412|2469954|2470229|N|N|N|N|N| +2470320|AAAAAAAAALBLFCAA|2051-05-28|1816|7900|606|2051|0|5|28|2|2051|606|7900|Sunday|2051Q2|N|N|N|2470293|2470412|2469955|2470230|N|N|N|N|N| +2470321|AAAAAAAABLBLFCAA|2051-05-29|1816|7900|606|2051|1|5|29|2|2051|606|7900|Monday|2051Q2|N|N|N|2470293|2470412|2469956|2470231|N|N|N|N|N| +2470322|AAAAAAAACLBLFCAA|2051-05-30|1816|7901|606|2051|2|5|30|2|2051|606|7901|Tuesday|2051Q2|N|N|N|2470293|2470412|2469957|2470232|N|N|N|N|N| +2470323|AAAAAAAADLBLFCAA|2051-05-31|1816|7901|606|2051|3|5|31|2|2051|606|7901|Wednesday|2051Q2|N|N|N|2470293|2470412|2469958|2470233|N|N|N|N|N| +2470324|AAAAAAAAELBLFCAA|2051-06-01|1817|7901|607|2051|4|6|1|2|2051|607|7901|Thursday|2051Q2|N|N|N|2470324|2470474|2469959|2470234|N|N|N|N|N| +2470325|AAAAAAAAFLBLFCAA|2051-06-02|1817|7901|607|2051|5|6|2|2|2051|607|7901|Friday|2051Q2|N|Y|N|2470324|2470474|2469960|2470235|N|N|N|N|N| +2470326|AAAAAAAAGLBLFCAA|2051-06-03|1817|7901|607|2051|6|6|3|2|2051|607|7901|Saturday|2051Q2|N|Y|N|2470324|2470474|2469961|2470236|N|N|N|N|N| +2470327|AAAAAAAAHLBLFCAA|2051-06-04|1817|7901|607|2051|0|6|4|2|2051|607|7901|Sunday|2051Q2|N|N|N|2470324|2470474|2469962|2470237|N|N|N|N|N| +2470328|AAAAAAAAILBLFCAA|2051-06-05|1817|7901|607|2051|1|6|5|2|2051|607|7901|Monday|2051Q2|N|N|N|2470324|2470474|2469963|2470238|N|N|N|N|N| +2470329|AAAAAAAAJLBLFCAA|2051-06-06|1817|7902|607|2051|2|6|6|2|2051|607|7902|Tuesday|2051Q2|N|N|N|2470324|2470474|2469964|2470239|N|N|N|N|N| +2470330|AAAAAAAAKLBLFCAA|2051-06-07|1817|7902|607|2051|3|6|7|2|2051|607|7902|Wednesday|2051Q2|N|N|N|2470324|2470474|2469965|2470240|N|N|N|N|N| +2470331|AAAAAAAALLBLFCAA|2051-06-08|1817|7902|607|2051|4|6|8|2|2051|607|7902|Thursday|2051Q2|N|N|N|2470324|2470474|2469966|2470241|N|N|N|N|N| +2470332|AAAAAAAAMLBLFCAA|2051-06-09|1817|7902|607|2051|5|6|9|2|2051|607|7902|Friday|2051Q2|N|Y|N|2470324|2470474|2469967|2470242|N|N|N|N|N| +2470333|AAAAAAAANLBLFCAA|2051-06-10|1817|7902|607|2051|6|6|10|2|2051|607|7902|Saturday|2051Q2|N|Y|N|2470324|2470474|2469968|2470243|N|N|N|N|N| +2470334|AAAAAAAAOLBLFCAA|2051-06-11|1817|7902|607|2051|0|6|11|2|2051|607|7902|Sunday|2051Q2|N|N|N|2470324|2470474|2469969|2470244|N|N|N|N|N| +2470335|AAAAAAAAPLBLFCAA|2051-06-12|1817|7902|607|2051|1|6|12|2|2051|607|7902|Monday|2051Q2|N|N|N|2470324|2470474|2469970|2470245|N|N|N|N|N| +2470336|AAAAAAAAAMBLFCAA|2051-06-13|1817|7903|607|2051|2|6|13|2|2051|607|7903|Tuesday|2051Q2|N|N|N|2470324|2470474|2469971|2470246|N|N|N|N|N| +2470337|AAAAAAAABMBLFCAA|2051-06-14|1817|7903|607|2051|3|6|14|2|2051|607|7903|Wednesday|2051Q2|N|N|N|2470324|2470474|2469972|2470247|N|N|N|N|N| +2470338|AAAAAAAACMBLFCAA|2051-06-15|1817|7903|607|2051|4|6|15|2|2051|607|7903|Thursday|2051Q2|N|N|N|2470324|2470474|2469973|2470248|N|N|N|N|N| +2470339|AAAAAAAADMBLFCAA|2051-06-16|1817|7903|607|2051|5|6|16|2|2051|607|7903|Friday|2051Q2|N|Y|N|2470324|2470474|2469974|2470249|N|N|N|N|N| +2470340|AAAAAAAAEMBLFCAA|2051-06-17|1817|7903|607|2051|6|6|17|2|2051|607|7903|Saturday|2051Q2|N|Y|N|2470324|2470474|2469975|2470250|N|N|N|N|N| +2470341|AAAAAAAAFMBLFCAA|2051-06-18|1817|7903|607|2051|0|6|18|2|2051|607|7903|Sunday|2051Q2|N|N|N|2470324|2470474|2469976|2470251|N|N|N|N|N| +2470342|AAAAAAAAGMBLFCAA|2051-06-19|1817|7903|607|2051|1|6|19|2|2051|607|7903|Monday|2051Q2|N|N|N|2470324|2470474|2469977|2470252|N|N|N|N|N| +2470343|AAAAAAAAHMBLFCAA|2051-06-20|1817|7904|607|2051|2|6|20|2|2051|607|7904|Tuesday|2051Q2|N|N|N|2470324|2470474|2469978|2470253|N|N|N|N|N| +2470344|AAAAAAAAIMBLFCAA|2051-06-21|1817|7904|607|2051|3|6|21|2|2051|607|7904|Wednesday|2051Q2|N|N|N|2470324|2470474|2469979|2470254|N|N|N|N|N| +2470345|AAAAAAAAJMBLFCAA|2051-06-22|1817|7904|607|2051|4|6|22|2|2051|607|7904|Thursday|2051Q2|N|N|N|2470324|2470474|2469980|2470255|N|N|N|N|N| +2470346|AAAAAAAAKMBLFCAA|2051-06-23|1817|7904|607|2051|5|6|23|2|2051|607|7904|Friday|2051Q2|N|Y|N|2470324|2470474|2469981|2470256|N|N|N|N|N| +2470347|AAAAAAAALMBLFCAA|2051-06-24|1817|7904|607|2051|6|6|24|2|2051|607|7904|Saturday|2051Q2|N|Y|N|2470324|2470474|2469982|2470257|N|N|N|N|N| +2470348|AAAAAAAAMMBLFCAA|2051-06-25|1817|7904|607|2051|0|6|25|2|2051|607|7904|Sunday|2051Q2|N|N|N|2470324|2470474|2469983|2470258|N|N|N|N|N| +2470349|AAAAAAAANMBLFCAA|2051-06-26|1817|7904|607|2051|1|6|26|2|2051|607|7904|Monday|2051Q2|N|N|N|2470324|2470474|2469984|2470259|N|N|N|N|N| +2470350|AAAAAAAAOMBLFCAA|2051-06-27|1817|7905|607|2051|2|6|27|2|2051|607|7905|Tuesday|2051Q2|N|N|N|2470324|2470474|2469985|2470260|N|N|N|N|N| +2470351|AAAAAAAAPMBLFCAA|2051-06-28|1817|7905|607|2051|3|6|28|2|2051|607|7905|Wednesday|2051Q2|N|N|N|2470324|2470474|2469986|2470261|N|N|N|N|N| +2470352|AAAAAAAAANBLFCAA|2051-06-29|1817|7905|607|2051|4|6|29|2|2051|607|7905|Thursday|2051Q2|N|N|N|2470324|2470474|2469987|2470262|N|N|N|N|N| +2470353|AAAAAAAABNBLFCAA|2051-06-30|1817|7905|607|2051|5|6|30|2|2051|607|7905|Friday|2051Q2|N|Y|N|2470324|2470474|2469988|2470263|N|N|N|N|N| +2470354|AAAAAAAACNBLFCAA|2051-07-01|1818|7905|607|2051|6|7|1|2|2051|607|7905|Saturday|2051Q2|N|Y|N|2470354|2470534|2469989|2470263|N|N|N|N|N| +2470355|AAAAAAAADNBLFCAA|2051-07-02|1818|7905|607|2051|0|7|2|3|2051|607|7905|Sunday|2051Q3|N|N|N|2470354|2470534|2469990|2470264|N|N|N|N|N| +2470356|AAAAAAAAENBLFCAA|2051-07-03|1818|7905|607|2051|1|7|3|3|2051|607|7905|Monday|2051Q3|N|N|N|2470354|2470534|2469991|2470265|N|N|N|N|N| +2470357|AAAAAAAAFNBLFCAA|2051-07-04|1818|7906|607|2051|2|7|4|3|2051|607|7906|Tuesday|2051Q3|N|N|N|2470354|2470534|2469992|2470266|N|N|N|N|N| +2470358|AAAAAAAAGNBLFCAA|2051-07-05|1818|7906|607|2051|3|7|5|3|2051|607|7906|Wednesday|2051Q3|Y|N|N|2470354|2470534|2469993|2470267|N|N|N|N|N| +2470359|AAAAAAAAHNBLFCAA|2051-07-06|1818|7906|607|2051|4|7|6|3|2051|607|7906|Thursday|2051Q3|N|N|Y|2470354|2470534|2469994|2470268|N|N|N|N|N| +2470360|AAAAAAAAINBLFCAA|2051-07-07|1818|7906|607|2051|5|7|7|3|2051|607|7906|Friday|2051Q3|N|Y|N|2470354|2470534|2469995|2470269|N|N|N|N|N| +2470361|AAAAAAAAJNBLFCAA|2051-07-08|1818|7906|607|2051|6|7|8|3|2051|607|7906|Saturday|2051Q3|N|Y|N|2470354|2470534|2469996|2470270|N|N|N|N|N| +2470362|AAAAAAAAKNBLFCAA|2051-07-09|1818|7906|607|2051|0|7|9|3|2051|607|7906|Sunday|2051Q3|N|N|N|2470354|2470534|2469997|2470271|N|N|N|N|N| +2470363|AAAAAAAALNBLFCAA|2051-07-10|1818|7906|607|2051|1|7|10|3|2051|607|7906|Monday|2051Q3|N|N|N|2470354|2470534|2469998|2470272|N|N|N|N|N| +2470364|AAAAAAAAMNBLFCAA|2051-07-11|1818|7907|607|2051|2|7|11|3|2051|607|7907|Tuesday|2051Q3|N|N|N|2470354|2470534|2469999|2470273|N|N|N|N|N| +2470365|AAAAAAAANNBLFCAA|2051-07-12|1818|7907|607|2051|3|7|12|3|2051|607|7907|Wednesday|2051Q3|N|N|N|2470354|2470534|2470000|2470274|N|N|N|N|N| +2470366|AAAAAAAAONBLFCAA|2051-07-13|1818|7907|607|2051|4|7|13|3|2051|607|7907|Thursday|2051Q3|N|N|N|2470354|2470534|2470001|2470275|N|N|N|N|N| +2470367|AAAAAAAAPNBLFCAA|2051-07-14|1818|7907|607|2051|5|7|14|3|2051|607|7907|Friday|2051Q3|N|Y|N|2470354|2470534|2470002|2470276|N|N|N|N|N| +2470368|AAAAAAAAAOBLFCAA|2051-07-15|1818|7907|607|2051|6|7|15|3|2051|607|7907|Saturday|2051Q3|N|Y|N|2470354|2470534|2470003|2470277|N|N|N|N|N| +2470369|AAAAAAAABOBLFCAA|2051-07-16|1818|7907|607|2051|0|7|16|3|2051|607|7907|Sunday|2051Q3|N|N|N|2470354|2470534|2470004|2470278|N|N|N|N|N| +2470370|AAAAAAAACOBLFCAA|2051-07-17|1818|7907|607|2051|1|7|17|3|2051|607|7907|Monday|2051Q3|N|N|N|2470354|2470534|2470005|2470279|N|N|N|N|N| +2470371|AAAAAAAADOBLFCAA|2051-07-18|1818|7908|607|2051|2|7|18|3|2051|607|7908|Tuesday|2051Q3|N|N|N|2470354|2470534|2470006|2470280|N|N|N|N|N| +2470372|AAAAAAAAEOBLFCAA|2051-07-19|1818|7908|607|2051|3|7|19|3|2051|607|7908|Wednesday|2051Q3|N|N|N|2470354|2470534|2470007|2470281|N|N|N|N|N| +2470373|AAAAAAAAFOBLFCAA|2051-07-20|1818|7908|607|2051|4|7|20|3|2051|607|7908|Thursday|2051Q3|N|N|N|2470354|2470534|2470008|2470282|N|N|N|N|N| +2470374|AAAAAAAAGOBLFCAA|2051-07-21|1818|7908|607|2051|5|7|21|3|2051|607|7908|Friday|2051Q3|N|Y|N|2470354|2470534|2470009|2470283|N|N|N|N|N| +2470375|AAAAAAAAHOBLFCAA|2051-07-22|1818|7908|607|2051|6|7|22|3|2051|607|7908|Saturday|2051Q3|N|Y|N|2470354|2470534|2470010|2470284|N|N|N|N|N| +2470376|AAAAAAAAIOBLFCAA|2051-07-23|1818|7908|607|2051|0|7|23|3|2051|607|7908|Sunday|2051Q3|N|N|N|2470354|2470534|2470011|2470285|N|N|N|N|N| +2470377|AAAAAAAAJOBLFCAA|2051-07-24|1818|7908|607|2051|1|7|24|3|2051|607|7908|Monday|2051Q3|N|N|N|2470354|2470534|2470012|2470286|N|N|N|N|N| +2470378|AAAAAAAAKOBLFCAA|2051-07-25|1818|7909|607|2051|2|7|25|3|2051|607|7909|Tuesday|2051Q3|N|N|N|2470354|2470534|2470013|2470287|N|N|N|N|N| +2470379|AAAAAAAALOBLFCAA|2051-07-26|1818|7909|607|2051|3|7|26|3|2051|607|7909|Wednesday|2051Q3|N|N|N|2470354|2470534|2470014|2470288|N|N|N|N|N| +2470380|AAAAAAAAMOBLFCAA|2051-07-27|1818|7909|607|2051|4|7|27|3|2051|607|7909|Thursday|2051Q3|N|N|N|2470354|2470534|2470015|2470289|N|N|N|N|N| +2470381|AAAAAAAANOBLFCAA|2051-07-28|1818|7909|607|2051|5|7|28|3|2051|607|7909|Friday|2051Q3|N|Y|N|2470354|2470534|2470016|2470290|N|N|N|N|N| +2470382|AAAAAAAAOOBLFCAA|2051-07-29|1818|7909|607|2051|6|7|29|3|2051|607|7909|Saturday|2051Q3|N|Y|N|2470354|2470534|2470017|2470291|N|N|N|N|N| +2470383|AAAAAAAAPOBLFCAA|2051-07-30|1818|7909|607|2051|0|7|30|3|2051|607|7909|Sunday|2051Q3|N|N|N|2470354|2470534|2470018|2470292|N|N|N|N|N| +2470384|AAAAAAAAAPBLFCAA|2051-07-31|1818|7909|607|2051|1|7|31|3|2051|607|7909|Monday|2051Q3|N|N|N|2470354|2470534|2470019|2470293|N|N|N|N|N| +2470385|AAAAAAAABPBLFCAA|2051-08-01|1819|7910|607|2051|2|8|1|3|2051|607|7910|Tuesday|2051Q3|N|N|N|2470385|2470596|2470020|2470294|N|N|N|N|N| +2470386|AAAAAAAACPBLFCAA|2051-08-02|1819|7910|607|2051|3|8|2|3|2051|607|7910|Wednesday|2051Q3|N|N|N|2470385|2470596|2470021|2470295|N|N|N|N|N| +2470387|AAAAAAAADPBLFCAA|2051-08-03|1819|7910|607|2051|4|8|3|3|2051|607|7910|Thursday|2051Q3|N|N|N|2470385|2470596|2470022|2470296|N|N|N|N|N| +2470388|AAAAAAAAEPBLFCAA|2051-08-04|1819|7910|607|2051|5|8|4|3|2051|607|7910|Friday|2051Q3|N|Y|N|2470385|2470596|2470023|2470297|N|N|N|N|N| +2470389|AAAAAAAAFPBLFCAA|2051-08-05|1819|7910|607|2051|6|8|5|3|2051|607|7910|Saturday|2051Q3|N|Y|N|2470385|2470596|2470024|2470298|N|N|N|N|N| +2470390|AAAAAAAAGPBLFCAA|2051-08-06|1819|7910|607|2051|0|8|6|3|2051|607|7910|Sunday|2051Q3|N|N|N|2470385|2470596|2470025|2470299|N|N|N|N|N| +2470391|AAAAAAAAHPBLFCAA|2051-08-07|1819|7910|607|2051|1|8|7|3|2051|607|7910|Monday|2051Q3|N|N|N|2470385|2470596|2470026|2470300|N|N|N|N|N| +2470392|AAAAAAAAIPBLFCAA|2051-08-08|1819|7911|607|2051|2|8|8|3|2051|607|7911|Tuesday|2051Q3|N|N|N|2470385|2470596|2470027|2470301|N|N|N|N|N| +2470393|AAAAAAAAJPBLFCAA|2051-08-09|1819|7911|607|2051|3|8|9|3|2051|607|7911|Wednesday|2051Q3|N|N|N|2470385|2470596|2470028|2470302|N|N|N|N|N| +2470394|AAAAAAAAKPBLFCAA|2051-08-10|1819|7911|607|2051|4|8|10|3|2051|607|7911|Thursday|2051Q3|N|N|N|2470385|2470596|2470029|2470303|N|N|N|N|N| +2470395|AAAAAAAALPBLFCAA|2051-08-11|1819|7911|607|2051|5|8|11|3|2051|607|7911|Friday|2051Q3|N|Y|N|2470385|2470596|2470030|2470304|N|N|N|N|N| +2470396|AAAAAAAAMPBLFCAA|2051-08-12|1819|7911|607|2051|6|8|12|3|2051|607|7911|Saturday|2051Q3|N|Y|N|2470385|2470596|2470031|2470305|N|N|N|N|N| +2470397|AAAAAAAANPBLFCAA|2051-08-13|1819|7911|607|2051|0|8|13|3|2051|607|7911|Sunday|2051Q3|N|N|N|2470385|2470596|2470032|2470306|N|N|N|N|N| +2470398|AAAAAAAAOPBLFCAA|2051-08-14|1819|7911|607|2051|1|8|14|3|2051|607|7911|Monday|2051Q3|N|N|N|2470385|2470596|2470033|2470307|N|N|N|N|N| +2470399|AAAAAAAAPPBLFCAA|2051-08-15|1819|7912|607|2051|2|8|15|3|2051|607|7912|Tuesday|2051Q3|N|N|N|2470385|2470596|2470034|2470308|N|N|N|N|N| +2470400|AAAAAAAAAACLFCAA|2051-08-16|1819|7912|607|2051|3|8|16|3|2051|607|7912|Wednesday|2051Q3|N|N|N|2470385|2470596|2470035|2470309|N|N|N|N|N| +2470401|AAAAAAAABACLFCAA|2051-08-17|1819|7912|607|2051|4|8|17|3|2051|607|7912|Thursday|2051Q3|N|N|N|2470385|2470596|2470036|2470310|N|N|N|N|N| +2470402|AAAAAAAACACLFCAA|2051-08-18|1819|7912|607|2051|5|8|18|3|2051|607|7912|Friday|2051Q3|N|Y|N|2470385|2470596|2470037|2470311|N|N|N|N|N| +2470403|AAAAAAAADACLFCAA|2051-08-19|1819|7912|607|2051|6|8|19|3|2051|607|7912|Saturday|2051Q3|N|Y|N|2470385|2470596|2470038|2470312|N|N|N|N|N| +2470404|AAAAAAAAEACLFCAA|2051-08-20|1819|7912|607|2051|0|8|20|3|2051|607|7912|Sunday|2051Q3|N|N|N|2470385|2470596|2470039|2470313|N|N|N|N|N| +2470405|AAAAAAAAFACLFCAA|2051-08-21|1819|7912|607|2051|1|8|21|3|2051|607|7912|Monday|2051Q3|N|N|N|2470385|2470596|2470040|2470314|N|N|N|N|N| +2470406|AAAAAAAAGACLFCAA|2051-08-22|1819|7913|607|2051|2|8|22|3|2051|607|7913|Tuesday|2051Q3|N|N|N|2470385|2470596|2470041|2470315|N|N|N|N|N| +2470407|AAAAAAAAHACLFCAA|2051-08-23|1819|7913|607|2051|3|8|23|3|2051|607|7913|Wednesday|2051Q3|N|N|N|2470385|2470596|2470042|2470316|N|N|N|N|N| +2470408|AAAAAAAAIACLFCAA|2051-08-24|1819|7913|607|2051|4|8|24|3|2051|607|7913|Thursday|2051Q3|N|N|N|2470385|2470596|2470043|2470317|N|N|N|N|N| +2470409|AAAAAAAAJACLFCAA|2051-08-25|1819|7913|607|2051|5|8|25|3|2051|607|7913|Friday|2051Q3|N|Y|N|2470385|2470596|2470044|2470318|N|N|N|N|N| +2470410|AAAAAAAAKACLFCAA|2051-08-26|1819|7913|607|2051|6|8|26|3|2051|607|7913|Saturday|2051Q3|N|Y|N|2470385|2470596|2470045|2470319|N|N|N|N|N| +2470411|AAAAAAAALACLFCAA|2051-08-27|1819|7913|607|2051|0|8|27|3|2051|607|7913|Sunday|2051Q3|N|N|N|2470385|2470596|2470046|2470320|N|N|N|N|N| +2470412|AAAAAAAAMACLFCAA|2051-08-28|1819|7913|607|2051|1|8|28|3|2051|607|7913|Monday|2051Q3|N|N|N|2470385|2470596|2470047|2470321|N|N|N|N|N| +2470413|AAAAAAAANACLFCAA|2051-08-29|1819|7914|607|2051|2|8|29|3|2051|607|7914|Tuesday|2051Q3|N|N|N|2470385|2470596|2470048|2470322|N|N|N|N|N| +2470414|AAAAAAAAOACLFCAA|2051-08-30|1819|7914|607|2051|3|8|30|3|2051|607|7914|Wednesday|2051Q3|N|N|N|2470385|2470596|2470049|2470323|N|N|N|N|N| +2470415|AAAAAAAAPACLFCAA|2051-08-31|1819|7914|607|2051|4|8|31|3|2051|607|7914|Thursday|2051Q3|N|N|N|2470385|2470596|2470050|2470324|N|N|N|N|N| +2470416|AAAAAAAAABCLFCAA|2051-09-01|1820|7914|608|2051|5|9|1|3|2051|608|7914|Friday|2051Q3|N|Y|N|2470416|2470658|2470051|2470325|N|N|N|N|N| +2470417|AAAAAAAABBCLFCAA|2051-09-02|1820|7914|608|2051|6|9|2|3|2051|608|7914|Saturday|2051Q3|N|Y|N|2470416|2470658|2470052|2470326|N|N|N|N|N| +2470418|AAAAAAAACBCLFCAA|2051-09-03|1820|7914|608|2051|0|9|3|3|2051|608|7914|Sunday|2051Q3|N|N|N|2470416|2470658|2470053|2470327|N|N|N|N|N| +2470419|AAAAAAAADBCLFCAA|2051-09-04|1820|7914|608|2051|1|9|4|3|2051|608|7914|Monday|2051Q3|N|N|N|2470416|2470658|2470054|2470328|N|N|N|N|N| +2470420|AAAAAAAAEBCLFCAA|2051-09-05|1820|7915|608|2051|2|9|5|3|2051|608|7915|Tuesday|2051Q3|N|N|N|2470416|2470658|2470055|2470329|N|N|N|N|N| +2470421|AAAAAAAAFBCLFCAA|2051-09-06|1820|7915|608|2051|3|9|6|3|2051|608|7915|Wednesday|2051Q3|N|N|N|2470416|2470658|2470056|2470330|N|N|N|N|N| +2470422|AAAAAAAAGBCLFCAA|2051-09-07|1820|7915|608|2051|4|9|7|3|2051|608|7915|Thursday|2051Q3|N|N|N|2470416|2470658|2470057|2470331|N|N|N|N|N| +2470423|AAAAAAAAHBCLFCAA|2051-09-08|1820|7915|608|2051|5|9|8|3|2051|608|7915|Friday|2051Q3|N|Y|N|2470416|2470658|2470058|2470332|N|N|N|N|N| +2470424|AAAAAAAAIBCLFCAA|2051-09-09|1820|7915|608|2051|6|9|9|3|2051|608|7915|Saturday|2051Q3|N|Y|N|2470416|2470658|2470059|2470333|N|N|N|N|N| +2470425|AAAAAAAAJBCLFCAA|2051-09-10|1820|7915|608|2051|0|9|10|3|2051|608|7915|Sunday|2051Q3|N|N|N|2470416|2470658|2470060|2470334|N|N|N|N|N| +2470426|AAAAAAAAKBCLFCAA|2051-09-11|1820|7915|608|2051|1|9|11|3|2051|608|7915|Monday|2051Q3|N|N|N|2470416|2470658|2470061|2470335|N|N|N|N|N| +2470427|AAAAAAAALBCLFCAA|2051-09-12|1820|7916|608|2051|2|9|12|3|2051|608|7916|Tuesday|2051Q3|N|N|N|2470416|2470658|2470062|2470336|N|N|N|N|N| +2470428|AAAAAAAAMBCLFCAA|2051-09-13|1820|7916|608|2051|3|9|13|3|2051|608|7916|Wednesday|2051Q3|N|N|N|2470416|2470658|2470063|2470337|N|N|N|N|N| +2470429|AAAAAAAANBCLFCAA|2051-09-14|1820|7916|608|2051|4|9|14|3|2051|608|7916|Thursday|2051Q3|N|N|N|2470416|2470658|2470064|2470338|N|N|N|N|N| +2470430|AAAAAAAAOBCLFCAA|2051-09-15|1820|7916|608|2051|5|9|15|3|2051|608|7916|Friday|2051Q3|N|Y|N|2470416|2470658|2470065|2470339|N|N|N|N|N| +2470431|AAAAAAAAPBCLFCAA|2051-09-16|1820|7916|608|2051|6|9|16|3|2051|608|7916|Saturday|2051Q3|N|Y|N|2470416|2470658|2470066|2470340|N|N|N|N|N| +2470432|AAAAAAAAACCLFCAA|2051-09-17|1820|7916|608|2051|0|9|17|3|2051|608|7916|Sunday|2051Q3|N|N|N|2470416|2470658|2470067|2470341|N|N|N|N|N| +2470433|AAAAAAAABCCLFCAA|2051-09-18|1820|7916|608|2051|1|9|18|3|2051|608|7916|Monday|2051Q3|N|N|N|2470416|2470658|2470068|2470342|N|N|N|N|N| +2470434|AAAAAAAACCCLFCAA|2051-09-19|1820|7917|608|2051|2|9|19|3|2051|608|7917|Tuesday|2051Q3|N|N|N|2470416|2470658|2470069|2470343|N|N|N|N|N| +2470435|AAAAAAAADCCLFCAA|2051-09-20|1820|7917|608|2051|3|9|20|3|2051|608|7917|Wednesday|2051Q3|N|N|N|2470416|2470658|2470070|2470344|N|N|N|N|N| +2470436|AAAAAAAAECCLFCAA|2051-09-21|1820|7917|608|2051|4|9|21|3|2051|608|7917|Thursday|2051Q3|N|N|N|2470416|2470658|2470071|2470345|N|N|N|N|N| +2470437|AAAAAAAAFCCLFCAA|2051-09-22|1820|7917|608|2051|5|9|22|3|2051|608|7917|Friday|2051Q3|N|Y|N|2470416|2470658|2470072|2470346|N|N|N|N|N| +2470438|AAAAAAAAGCCLFCAA|2051-09-23|1820|7917|608|2051|6|9|23|3|2051|608|7917|Saturday|2051Q3|N|Y|N|2470416|2470658|2470073|2470347|N|N|N|N|N| +2470439|AAAAAAAAHCCLFCAA|2051-09-24|1820|7917|608|2051|0|9|24|3|2051|608|7917|Sunday|2051Q3|N|N|N|2470416|2470658|2470074|2470348|N|N|N|N|N| +2470440|AAAAAAAAICCLFCAA|2051-09-25|1820|7917|608|2051|1|9|25|3|2051|608|7917|Monday|2051Q3|N|N|N|2470416|2470658|2470075|2470349|N|N|N|N|N| +2470441|AAAAAAAAJCCLFCAA|2051-09-26|1820|7918|608|2051|2|9|26|3|2051|608|7918|Tuesday|2051Q3|N|N|N|2470416|2470658|2470076|2470350|N|N|N|N|N| +2470442|AAAAAAAAKCCLFCAA|2051-09-27|1820|7918|608|2051|3|9|27|3|2051|608|7918|Wednesday|2051Q3|N|N|N|2470416|2470658|2470077|2470351|N|N|N|N|N| +2470443|AAAAAAAALCCLFCAA|2051-09-28|1820|7918|608|2051|4|9|28|3|2051|608|7918|Thursday|2051Q3|N|N|N|2470416|2470658|2470078|2470352|N|N|N|N|N| +2470444|AAAAAAAAMCCLFCAA|2051-09-29|1820|7918|608|2051|5|9|29|3|2051|608|7918|Friday|2051Q3|N|Y|N|2470416|2470658|2470079|2470353|N|N|N|N|N| +2470445|AAAAAAAANCCLFCAA|2051-09-30|1820|7918|608|2051|6|9|30|3|2051|608|7918|Saturday|2051Q3|N|Y|N|2470416|2470658|2470080|2470354|N|N|N|N|N| +2470446|AAAAAAAAOCCLFCAA|2051-10-01|1821|7918|608|2051|0|10|1|3|2051|608|7918|Sunday|2051Q3|N|N|N|2470446|2470718|2470081|2470354|N|N|N|N|N| +2470447|AAAAAAAAPCCLFCAA|2051-10-02|1821|7918|608|2051|1|10|2|4|2051|608|7918|Monday|2051Q4|N|N|N|2470446|2470718|2470082|2470355|N|N|N|N|N| +2470448|AAAAAAAAADCLFCAA|2051-10-03|1821|7919|608|2051|2|10|3|4|2051|608|7919|Tuesday|2051Q4|N|N|N|2470446|2470718|2470083|2470356|N|N|N|N|N| +2470449|AAAAAAAABDCLFCAA|2051-10-04|1821|7919|608|2051|3|10|4|4|2051|608|7919|Wednesday|2051Q4|N|N|N|2470446|2470718|2470084|2470357|N|N|N|N|N| +2470450|AAAAAAAACDCLFCAA|2051-10-05|1821|7919|608|2051|4|10|5|4|2051|608|7919|Thursday|2051Q4|N|N|N|2470446|2470718|2470085|2470358|N|N|N|N|N| +2470451|AAAAAAAADDCLFCAA|2051-10-06|1821|7919|608|2051|5|10|6|4|2051|608|7919|Friday|2051Q4|N|Y|N|2470446|2470718|2470086|2470359|N|N|N|N|N| +2470452|AAAAAAAAEDCLFCAA|2051-10-07|1821|7919|608|2051|6|10|7|4|2051|608|7919|Saturday|2051Q4|N|Y|N|2470446|2470718|2470087|2470360|N|N|N|N|N| +2470453|AAAAAAAAFDCLFCAA|2051-10-08|1821|7919|608|2051|0|10|8|4|2051|608|7919|Sunday|2051Q4|N|N|N|2470446|2470718|2470088|2470361|N|N|N|N|N| +2470454|AAAAAAAAGDCLFCAA|2051-10-09|1821|7919|608|2051|1|10|9|4|2051|608|7919|Monday|2051Q4|N|N|N|2470446|2470718|2470089|2470362|N|N|N|N|N| +2470455|AAAAAAAAHDCLFCAA|2051-10-10|1821|7920|608|2051|2|10|10|4|2051|608|7920|Tuesday|2051Q4|N|N|N|2470446|2470718|2470090|2470363|N|N|N|N|N| +2470456|AAAAAAAAIDCLFCAA|2051-10-11|1821|7920|608|2051|3|10|11|4|2051|608|7920|Wednesday|2051Q4|N|N|N|2470446|2470718|2470091|2470364|N|N|N|N|N| +2470457|AAAAAAAAJDCLFCAA|2051-10-12|1821|7920|608|2051|4|10|12|4|2051|608|7920|Thursday|2051Q4|N|N|N|2470446|2470718|2470092|2470365|N|N|N|N|N| +2470458|AAAAAAAAKDCLFCAA|2051-10-13|1821|7920|608|2051|5|10|13|4|2051|608|7920|Friday|2051Q4|N|Y|N|2470446|2470718|2470093|2470366|N|N|N|N|N| +2470459|AAAAAAAALDCLFCAA|2051-10-14|1821|7920|608|2051|6|10|14|4|2051|608|7920|Saturday|2051Q4|N|Y|N|2470446|2470718|2470094|2470367|N|N|N|N|N| +2470460|AAAAAAAAMDCLFCAA|2051-10-15|1821|7920|608|2051|0|10|15|4|2051|608|7920|Sunday|2051Q4|N|N|N|2470446|2470718|2470095|2470368|N|N|N|N|N| +2470461|AAAAAAAANDCLFCAA|2051-10-16|1821|7920|608|2051|1|10|16|4|2051|608|7920|Monday|2051Q4|N|N|N|2470446|2470718|2470096|2470369|N|N|N|N|N| +2470462|AAAAAAAAODCLFCAA|2051-10-17|1821|7921|608|2051|2|10|17|4|2051|608|7921|Tuesday|2051Q4|N|N|N|2470446|2470718|2470097|2470370|N|N|N|N|N| +2470463|AAAAAAAAPDCLFCAA|2051-10-18|1821|7921|608|2051|3|10|18|4|2051|608|7921|Wednesday|2051Q4|N|N|N|2470446|2470718|2470098|2470371|N|N|N|N|N| +2470464|AAAAAAAAAECLFCAA|2051-10-19|1821|7921|608|2051|4|10|19|4|2051|608|7921|Thursday|2051Q4|N|N|N|2470446|2470718|2470099|2470372|N|N|N|N|N| +2470465|AAAAAAAABECLFCAA|2051-10-20|1821|7921|608|2051|5|10|20|4|2051|608|7921|Friday|2051Q4|N|Y|N|2470446|2470718|2470100|2470373|N|N|N|N|N| +2470466|AAAAAAAACECLFCAA|2051-10-21|1821|7921|608|2051|6|10|21|4|2051|608|7921|Saturday|2051Q4|N|Y|N|2470446|2470718|2470101|2470374|N|N|N|N|N| +2470467|AAAAAAAADECLFCAA|2051-10-22|1821|7921|608|2051|0|10|22|4|2051|608|7921|Sunday|2051Q4|N|N|N|2470446|2470718|2470102|2470375|N|N|N|N|N| +2470468|AAAAAAAAEECLFCAA|2051-10-23|1821|7921|608|2051|1|10|23|4|2051|608|7921|Monday|2051Q4|N|N|N|2470446|2470718|2470103|2470376|N|N|N|N|N| +2470469|AAAAAAAAFECLFCAA|2051-10-24|1821|7922|608|2051|2|10|24|4|2051|608|7922|Tuesday|2051Q4|N|N|N|2470446|2470718|2470104|2470377|N|N|N|N|N| +2470470|AAAAAAAAGECLFCAA|2051-10-25|1821|7922|608|2051|3|10|25|4|2051|608|7922|Wednesday|2051Q4|N|N|N|2470446|2470718|2470105|2470378|N|N|N|N|N| +2470471|AAAAAAAAHECLFCAA|2051-10-26|1821|7922|608|2051|4|10|26|4|2051|608|7922|Thursday|2051Q4|N|N|N|2470446|2470718|2470106|2470379|N|N|N|N|N| +2470472|AAAAAAAAIECLFCAA|2051-10-27|1821|7922|608|2051|5|10|27|4|2051|608|7922|Friday|2051Q4|N|Y|N|2470446|2470718|2470107|2470380|N|N|N|N|N| +2470473|AAAAAAAAJECLFCAA|2051-10-28|1821|7922|608|2051|6|10|28|4|2051|608|7922|Saturday|2051Q4|N|Y|N|2470446|2470718|2470108|2470381|N|N|N|N|N| +2470474|AAAAAAAAKECLFCAA|2051-10-29|1821|7922|608|2051|0|10|29|4|2051|608|7922|Sunday|2051Q4|N|N|N|2470446|2470718|2470109|2470382|N|N|N|N|N| +2470475|AAAAAAAALECLFCAA|2051-10-30|1821|7922|608|2051|1|10|30|4|2051|608|7922|Monday|2051Q4|N|N|N|2470446|2470718|2470110|2470383|N|N|N|N|N| +2470476|AAAAAAAAMECLFCAA|2051-10-31|1821|7923|608|2051|2|10|31|4|2051|608|7923|Tuesday|2051Q4|N|N|N|2470446|2470718|2470111|2470384|N|N|N|N|N| +2470477|AAAAAAAANECLFCAA|2051-11-01|1822|7923|608|2051|3|11|1|4|2051|608|7923|Wednesday|2051Q4|N|N|N|2470477|2470780|2470112|2470385|N|N|N|N|N| +2470478|AAAAAAAAOECLFCAA|2051-11-02|1822|7923|608|2051|4|11|2|4|2051|608|7923|Thursday|2051Q4|N|N|N|2470477|2470780|2470113|2470386|N|N|N|N|N| +2470479|AAAAAAAAPECLFCAA|2051-11-03|1822|7923|608|2051|5|11|3|4|2051|608|7923|Friday|2051Q4|N|Y|N|2470477|2470780|2470114|2470387|N|N|N|N|N| +2470480|AAAAAAAAAFCLFCAA|2051-11-04|1822|7923|608|2051|6|11|4|4|2051|608|7923|Saturday|2051Q4|N|Y|N|2470477|2470780|2470115|2470388|N|N|N|N|N| +2470481|AAAAAAAABFCLFCAA|2051-11-05|1822|7923|608|2051|0|11|5|4|2051|608|7923|Sunday|2051Q4|N|N|N|2470477|2470780|2470116|2470389|N|N|N|N|N| +2470482|AAAAAAAACFCLFCAA|2051-11-06|1822|7923|608|2051|1|11|6|4|2051|608|7923|Monday|2051Q4|N|N|N|2470477|2470780|2470117|2470390|N|N|N|N|N| +2470483|AAAAAAAADFCLFCAA|2051-11-07|1822|7924|608|2051|2|11|7|4|2051|608|7924|Tuesday|2051Q4|N|N|N|2470477|2470780|2470118|2470391|N|N|N|N|N| +2470484|AAAAAAAAEFCLFCAA|2051-11-08|1822|7924|608|2051|3|11|8|4|2051|608|7924|Wednesday|2051Q4|N|N|N|2470477|2470780|2470119|2470392|N|N|N|N|N| +2470485|AAAAAAAAFFCLFCAA|2051-11-09|1822|7924|608|2051|4|11|9|4|2051|608|7924|Thursday|2051Q4|N|N|N|2470477|2470780|2470120|2470393|N|N|N|N|N| +2470486|AAAAAAAAGFCLFCAA|2051-11-10|1822|7924|608|2051|5|11|10|4|2051|608|7924|Friday|2051Q4|N|Y|N|2470477|2470780|2470121|2470394|N|N|N|N|N| +2470487|AAAAAAAAHFCLFCAA|2051-11-11|1822|7924|608|2051|6|11|11|4|2051|608|7924|Saturday|2051Q4|N|Y|N|2470477|2470780|2470122|2470395|N|N|N|N|N| +2470488|AAAAAAAAIFCLFCAA|2051-11-12|1822|7924|608|2051|0|11|12|4|2051|608|7924|Sunday|2051Q4|N|N|N|2470477|2470780|2470123|2470396|N|N|N|N|N| +2470489|AAAAAAAAJFCLFCAA|2051-11-13|1822|7924|608|2051|1|11|13|4|2051|608|7924|Monday|2051Q4|N|N|N|2470477|2470780|2470124|2470397|N|N|N|N|N| +2470490|AAAAAAAAKFCLFCAA|2051-11-14|1822|7925|608|2051|2|11|14|4|2051|608|7925|Tuesday|2051Q4|N|N|N|2470477|2470780|2470125|2470398|N|N|N|N|N| +2470491|AAAAAAAALFCLFCAA|2051-11-15|1822|7925|608|2051|3|11|15|4|2051|608|7925|Wednesday|2051Q4|N|N|N|2470477|2470780|2470126|2470399|N|N|N|N|N| +2470492|AAAAAAAAMFCLFCAA|2051-11-16|1822|7925|608|2051|4|11|16|4|2051|608|7925|Thursday|2051Q4|N|N|N|2470477|2470780|2470127|2470400|N|N|N|N|N| +2470493|AAAAAAAANFCLFCAA|2051-11-17|1822|7925|608|2051|5|11|17|4|2051|608|7925|Friday|2051Q4|N|Y|N|2470477|2470780|2470128|2470401|N|N|N|N|N| +2470494|AAAAAAAAOFCLFCAA|2051-11-18|1822|7925|608|2051|6|11|18|4|2051|608|7925|Saturday|2051Q4|N|Y|N|2470477|2470780|2470129|2470402|N|N|N|N|N| +2470495|AAAAAAAAPFCLFCAA|2051-11-19|1822|7925|608|2051|0|11|19|4|2051|608|7925|Sunday|2051Q4|N|N|N|2470477|2470780|2470130|2470403|N|N|N|N|N| +2470496|AAAAAAAAAGCLFCAA|2051-11-20|1822|7925|608|2051|1|11|20|4|2051|608|7925|Monday|2051Q4|N|N|N|2470477|2470780|2470131|2470404|N|N|N|N|N| +2470497|AAAAAAAABGCLFCAA|2051-11-21|1822|7926|608|2051|2|11|21|4|2051|608|7926|Tuesday|2051Q4|N|N|N|2470477|2470780|2470132|2470405|N|N|N|N|N| +2470498|AAAAAAAACGCLFCAA|2051-11-22|1822|7926|608|2051|3|11|22|4|2051|608|7926|Wednesday|2051Q4|N|N|N|2470477|2470780|2470133|2470406|N|N|N|N|N| +2470499|AAAAAAAADGCLFCAA|2051-11-23|1822|7926|608|2051|4|11|23|4|2051|608|7926|Thursday|2051Q4|N|N|N|2470477|2470780|2470134|2470407|N|N|N|N|N| +2470500|AAAAAAAAEGCLFCAA|2051-11-24|1822|7926|608|2051|5|11|24|4|2051|608|7926|Friday|2051Q4|N|Y|N|2470477|2470780|2470135|2470408|N|N|N|N|N| +2470501|AAAAAAAAFGCLFCAA|2051-11-25|1822|7926|608|2051|6|11|25|4|2051|608|7926|Saturday|2051Q4|N|Y|N|2470477|2470780|2470136|2470409|N|N|N|N|N| +2470502|AAAAAAAAGGCLFCAA|2051-11-26|1822|7926|608|2051|0|11|26|4|2051|608|7926|Sunday|2051Q4|N|N|N|2470477|2470780|2470137|2470410|N|N|N|N|N| +2470503|AAAAAAAAHGCLFCAA|2051-11-27|1822|7926|608|2051|1|11|27|4|2051|608|7926|Monday|2051Q4|N|N|N|2470477|2470780|2470138|2470411|N|N|N|N|N| +2470504|AAAAAAAAIGCLFCAA|2051-11-28|1822|7927|608|2051|2|11|28|4|2051|608|7927|Tuesday|2051Q4|N|N|N|2470477|2470780|2470139|2470412|N|N|N|N|N| +2470505|AAAAAAAAJGCLFCAA|2051-11-29|1822|7927|608|2051|3|11|29|4|2051|608|7927|Wednesday|2051Q4|N|N|N|2470477|2470780|2470140|2470413|N|N|N|N|N| +2470506|AAAAAAAAKGCLFCAA|2051-11-30|1822|7927|608|2051|4|11|30|4|2051|608|7927|Thursday|2051Q4|N|N|N|2470477|2470780|2470141|2470414|N|N|N|N|N| +2470507|AAAAAAAALGCLFCAA|2051-12-01|1823|7927|609|2051|5|12|1|4|2051|609|7927|Friday|2051Q4|N|Y|N|2470507|2470840|2470142|2470415|N|N|N|N|N| +2470508|AAAAAAAAMGCLFCAA|2051-12-02|1823|7927|609|2051|6|12|2|4|2051|609|7927|Saturday|2051Q4|N|Y|N|2470507|2470840|2470143|2470416|N|N|N|N|N| +2470509|AAAAAAAANGCLFCAA|2051-12-03|1823|7927|609|2051|0|12|3|4|2051|609|7927|Sunday|2051Q4|N|N|N|2470507|2470840|2470144|2470417|N|N|N|N|N| +2470510|AAAAAAAAOGCLFCAA|2051-12-04|1823|7927|609|2051|1|12|4|4|2051|609|7927|Monday|2051Q4|N|N|N|2470507|2470840|2470145|2470418|N|N|N|N|N| +2470511|AAAAAAAAPGCLFCAA|2051-12-05|1823|7928|609|2051|2|12|5|4|2051|609|7928|Tuesday|2051Q4|N|N|N|2470507|2470840|2470146|2470419|N|N|N|N|N| +2470512|AAAAAAAAAHCLFCAA|2051-12-06|1823|7928|609|2051|3|12|6|4|2051|609|7928|Wednesday|2051Q4|N|N|N|2470507|2470840|2470147|2470420|N|N|N|N|N| +2470513|AAAAAAAABHCLFCAA|2051-12-07|1823|7928|609|2051|4|12|7|4|2051|609|7928|Thursday|2051Q4|N|N|N|2470507|2470840|2470148|2470421|N|N|N|N|N| +2470514|AAAAAAAACHCLFCAA|2051-12-08|1823|7928|609|2051|5|12|8|4|2051|609|7928|Friday|2051Q4|N|Y|N|2470507|2470840|2470149|2470422|N|N|N|N|N| +2470515|AAAAAAAADHCLFCAA|2051-12-09|1823|7928|609|2051|6|12|9|4|2051|609|7928|Saturday|2051Q4|N|Y|N|2470507|2470840|2470150|2470423|N|N|N|N|N| +2470516|AAAAAAAAEHCLFCAA|2051-12-10|1823|7928|609|2051|0|12|10|4|2051|609|7928|Sunday|2051Q4|N|N|N|2470507|2470840|2470151|2470424|N|N|N|N|N| +2470517|AAAAAAAAFHCLFCAA|2051-12-11|1823|7928|609|2051|1|12|11|4|2051|609|7928|Monday|2051Q4|N|N|N|2470507|2470840|2470152|2470425|N|N|N|N|N| +2470518|AAAAAAAAGHCLFCAA|2051-12-12|1823|7929|609|2051|2|12|12|4|2051|609|7929|Tuesday|2051Q4|N|N|N|2470507|2470840|2470153|2470426|N|N|N|N|N| +2470519|AAAAAAAAHHCLFCAA|2051-12-13|1823|7929|609|2051|3|12|13|4|2051|609|7929|Wednesday|2051Q4|N|N|N|2470507|2470840|2470154|2470427|N|N|N|N|N| +2470520|AAAAAAAAIHCLFCAA|2051-12-14|1823|7929|609|2051|4|12|14|4|2051|609|7929|Thursday|2051Q4|N|N|N|2470507|2470840|2470155|2470428|N|N|N|N|N| +2470521|AAAAAAAAJHCLFCAA|2051-12-15|1823|7929|609|2051|5|12|15|4|2051|609|7929|Friday|2051Q4|N|Y|N|2470507|2470840|2470156|2470429|N|N|N|N|N| +2470522|AAAAAAAAKHCLFCAA|2051-12-16|1823|7929|609|2051|6|12|16|4|2051|609|7929|Saturday|2051Q4|N|Y|N|2470507|2470840|2470157|2470430|N|N|N|N|N| +2470523|AAAAAAAALHCLFCAA|2051-12-17|1823|7929|609|2051|0|12|17|4|2051|609|7929|Sunday|2051Q4|N|N|N|2470507|2470840|2470158|2470431|N|N|N|N|N| +2470524|AAAAAAAAMHCLFCAA|2051-12-18|1823|7929|609|2051|1|12|18|4|2051|609|7929|Monday|2051Q4|N|N|N|2470507|2470840|2470159|2470432|N|N|N|N|N| +2470525|AAAAAAAANHCLFCAA|2051-12-19|1823|7930|609|2051|2|12|19|4|2051|609|7930|Tuesday|2051Q4|N|N|N|2470507|2470840|2470160|2470433|N|N|N|N|N| +2470526|AAAAAAAAOHCLFCAA|2051-12-20|1823|7930|609|2051|3|12|20|4|2051|609|7930|Wednesday|2051Q4|N|N|N|2470507|2470840|2470161|2470434|N|N|N|N|N| +2470527|AAAAAAAAPHCLFCAA|2051-12-21|1823|7930|609|2051|4|12|21|4|2051|609|7930|Thursday|2051Q4|N|N|N|2470507|2470840|2470162|2470435|N|N|N|N|N| +2470528|AAAAAAAAAICLFCAA|2051-12-22|1823|7930|609|2051|5|12|22|4|2051|609|7930|Friday|2051Q4|N|Y|N|2470507|2470840|2470163|2470436|N|N|N|N|N| +2470529|AAAAAAAABICLFCAA|2051-12-23|1823|7930|609|2051|6|12|23|4|2051|609|7930|Saturday|2051Q4|N|Y|N|2470507|2470840|2470164|2470437|N|N|N|N|N| +2470530|AAAAAAAACICLFCAA|2051-12-24|1823|7930|609|2051|0|12|24|4|2051|609|7930|Sunday|2051Q4|N|N|N|2470507|2470840|2470165|2470438|N|N|N|N|N| +2470531|AAAAAAAADICLFCAA|2051-12-25|1823|7930|609|2051|1|12|25|4|2051|609|7930|Monday|2051Q4|N|N|N|2470507|2470840|2470166|2470439|N|N|N|N|N| +2470532|AAAAAAAAEICLFCAA|2051-12-26|1823|7931|609|2051|2|12|26|4|2051|609|7931|Tuesday|2051Q4|Y|N|N|2470507|2470840|2470167|2470440|N|N|N|N|N| +2470533|AAAAAAAAFICLFCAA|2051-12-27|1823|7931|609|2051|3|12|27|4|2051|609|7931|Wednesday|2051Q4|N|N|Y|2470507|2470840|2470168|2470441|N|N|N|N|N| +2470534|AAAAAAAAGICLFCAA|2051-12-28|1823|7931|609|2051|4|12|28|4|2051|609|7931|Thursday|2051Q4|N|N|N|2470507|2470840|2470169|2470442|N|N|N|N|N| +2470535|AAAAAAAAHICLFCAA|2051-12-29|1823|7931|609|2051|5|12|29|4|2051|609|7931|Friday|2051Q4|N|Y|N|2470507|2470840|2470170|2470443|N|N|N|N|N| +2470536|AAAAAAAAIICLFCAA|2051-12-30|1823|7931|609|2051|6|12|30|4|2051|609|7931|Saturday|2051Q4|N|Y|N|2470507|2470840|2470171|2470444|N|N|N|N|N| +2470537|AAAAAAAAJICLFCAA|2051-12-31|1823|7931|609|2051|0|12|31|4|2051|609|7931|Sunday|2051Q4|N|N|N|2470507|2470840|2470172|2470445|N|N|N|N|N| +2470538|AAAAAAAAKICLFCAA|2052-01-01|1824|7931|609|2052|1|1|1|1|2052|609|7931|Monday|2052Q1|Y|N|N|2470538|2470537|2470173|2470446|N|N|N|N|N| +2470539|AAAAAAAALICLFCAA|2052-01-02|1824|7932|609|2052|2|1|2|1|2052|609|7932|Tuesday|2052Q1|N|N|Y|2470538|2470537|2470174|2470447|N|N|N|N|N| +2470540|AAAAAAAAMICLFCAA|2052-01-03|1824|7932|609|2052|3|1|3|1|2052|609|7932|Wednesday|2052Q1|N|N|N|2470538|2470537|2470175|2470448|N|N|N|N|N| +2470541|AAAAAAAANICLFCAA|2052-01-04|1824|7932|609|2052|4|1|4|1|2052|609|7932|Thursday|2052Q1|N|N|N|2470538|2470537|2470176|2470449|N|N|N|N|N| +2470542|AAAAAAAAOICLFCAA|2052-01-05|1824|7932|609|2052|5|1|5|1|2052|609|7932|Friday|2052Q1|N|Y|N|2470538|2470537|2470177|2470450|N|N|N|N|N| +2470543|AAAAAAAAPICLFCAA|2052-01-06|1824|7932|609|2052|6|1|6|1|2052|609|7932|Saturday|2052Q1|N|Y|N|2470538|2470537|2470178|2470451|N|N|N|N|N| +2470544|AAAAAAAAAJCLFCAA|2052-01-07|1824|7932|609|2052|0|1|7|1|2052|609|7932|Sunday|2052Q1|N|N|N|2470538|2470537|2470179|2470452|N|N|N|N|N| +2470545|AAAAAAAABJCLFCAA|2052-01-08|1824|7932|609|2052|1|1|8|1|2052|609|7932|Monday|2052Q1|N|N|N|2470538|2470537|2470180|2470453|N|N|N|N|N| +2470546|AAAAAAAACJCLFCAA|2052-01-09|1824|7933|609|2052|2|1|9|1|2052|609|7933|Tuesday|2052Q1|N|N|N|2470538|2470537|2470181|2470454|N|N|N|N|N| +2470547|AAAAAAAADJCLFCAA|2052-01-10|1824|7933|609|2052|3|1|10|1|2052|609|7933|Wednesday|2052Q1|N|N|N|2470538|2470537|2470182|2470455|N|N|N|N|N| +2470548|AAAAAAAAEJCLFCAA|2052-01-11|1824|7933|609|2052|4|1|11|1|2052|609|7933|Thursday|2052Q1|N|N|N|2470538|2470537|2470183|2470456|N|N|N|N|N| +2470549|AAAAAAAAFJCLFCAA|2052-01-12|1824|7933|609|2052|5|1|12|1|2052|609|7933|Friday|2052Q1|N|Y|N|2470538|2470537|2470184|2470457|N|N|N|N|N| +2470550|AAAAAAAAGJCLFCAA|2052-01-13|1824|7933|609|2052|6|1|13|1|2052|609|7933|Saturday|2052Q1|N|Y|N|2470538|2470537|2470185|2470458|N|N|N|N|N| +2470551|AAAAAAAAHJCLFCAA|2052-01-14|1824|7933|609|2052|0|1|14|1|2052|609|7933|Sunday|2052Q1|N|N|N|2470538|2470537|2470186|2470459|N|N|N|N|N| +2470552|AAAAAAAAIJCLFCAA|2052-01-15|1824|7933|609|2052|1|1|15|1|2052|609|7933|Monday|2052Q1|N|N|N|2470538|2470537|2470187|2470460|N|N|N|N|N| +2470553|AAAAAAAAJJCLFCAA|2052-01-16|1824|7934|609|2052|2|1|16|1|2052|609|7934|Tuesday|2052Q1|N|N|N|2470538|2470537|2470188|2470461|N|N|N|N|N| +2470554|AAAAAAAAKJCLFCAA|2052-01-17|1824|7934|609|2052|3|1|17|1|2052|609|7934|Wednesday|2052Q1|N|N|N|2470538|2470537|2470189|2470462|N|N|N|N|N| +2470555|AAAAAAAALJCLFCAA|2052-01-18|1824|7934|609|2052|4|1|18|1|2052|609|7934|Thursday|2052Q1|N|N|N|2470538|2470537|2470190|2470463|N|N|N|N|N| +2470556|AAAAAAAAMJCLFCAA|2052-01-19|1824|7934|609|2052|5|1|19|1|2052|609|7934|Friday|2052Q1|N|Y|N|2470538|2470537|2470191|2470464|N|N|N|N|N| +2470557|AAAAAAAANJCLFCAA|2052-01-20|1824|7934|609|2052|6|1|20|1|2052|609|7934|Saturday|2052Q1|N|Y|N|2470538|2470537|2470192|2470465|N|N|N|N|N| +2470558|AAAAAAAAOJCLFCAA|2052-01-21|1824|7934|609|2052|0|1|21|1|2052|609|7934|Sunday|2052Q1|N|N|N|2470538|2470537|2470193|2470466|N|N|N|N|N| +2470559|AAAAAAAAPJCLFCAA|2052-01-22|1824|7934|609|2052|1|1|22|1|2052|609|7934|Monday|2052Q1|N|N|N|2470538|2470537|2470194|2470467|N|N|N|N|N| +2470560|AAAAAAAAAKCLFCAA|2052-01-23|1824|7935|609|2052|2|1|23|1|2052|609|7935|Tuesday|2052Q1|N|N|N|2470538|2470537|2470195|2470468|N|N|N|N|N| +2470561|AAAAAAAABKCLFCAA|2052-01-24|1824|7935|609|2052|3|1|24|1|2052|609|7935|Wednesday|2052Q1|N|N|N|2470538|2470537|2470196|2470469|N|N|N|N|N| +2470562|AAAAAAAACKCLFCAA|2052-01-25|1824|7935|609|2052|4|1|25|1|2052|609|7935|Thursday|2052Q1|N|N|N|2470538|2470537|2470197|2470470|N|N|N|N|N| +2470563|AAAAAAAADKCLFCAA|2052-01-26|1824|7935|609|2052|5|1|26|1|2052|609|7935|Friday|2052Q1|N|Y|N|2470538|2470537|2470198|2470471|N|N|N|N|N| +2470564|AAAAAAAAEKCLFCAA|2052-01-27|1824|7935|609|2052|6|1|27|1|2052|609|7935|Saturday|2052Q1|N|Y|N|2470538|2470537|2470199|2470472|N|N|N|N|N| +2470565|AAAAAAAAFKCLFCAA|2052-01-28|1824|7935|609|2052|0|1|28|1|2052|609|7935|Sunday|2052Q1|N|N|N|2470538|2470537|2470200|2470473|N|N|N|N|N| +2470566|AAAAAAAAGKCLFCAA|2052-01-29|1824|7935|609|2052|1|1|29|1|2052|609|7935|Monday|2052Q1|N|N|N|2470538|2470537|2470201|2470474|N|N|N|N|N| +2470567|AAAAAAAAHKCLFCAA|2052-01-30|1824|7936|609|2052|2|1|30|1|2052|609|7936|Tuesday|2052Q1|N|N|N|2470538|2470537|2470202|2470475|N|N|N|N|N| +2470568|AAAAAAAAIKCLFCAA|2052-01-31|1824|7936|609|2052|3|1|31|1|2052|609|7936|Wednesday|2052Q1|N|N|N|2470538|2470537|2470203|2470476|N|N|N|N|N| +2470569|AAAAAAAAJKCLFCAA|2052-02-01|1825|7936|609|2052|4|2|1|1|2052|609|7936|Thursday|2052Q1|N|N|N|2470569|2470599|2470204|2470477|N|N|N|N|N| +2470570|AAAAAAAAKKCLFCAA|2052-02-02|1825|7936|609|2052|5|2|2|1|2052|609|7936|Friday|2052Q1|N|Y|N|2470569|2470599|2470205|2470478|N|N|N|N|N| +2470571|AAAAAAAALKCLFCAA|2052-02-03|1825|7936|609|2052|6|2|3|1|2052|609|7936|Saturday|2052Q1|N|Y|N|2470569|2470599|2470206|2470479|N|N|N|N|N| +2470572|AAAAAAAAMKCLFCAA|2052-02-04|1825|7936|609|2052|0|2|4|1|2052|609|7936|Sunday|2052Q1|N|N|N|2470569|2470599|2470207|2470480|N|N|N|N|N| +2470573|AAAAAAAANKCLFCAA|2052-02-05|1825|7936|609|2052|1|2|5|1|2052|609|7936|Monday|2052Q1|N|N|N|2470569|2470599|2470208|2470481|N|N|N|N|N| +2470574|AAAAAAAAOKCLFCAA|2052-02-06|1825|7937|609|2052|2|2|6|1|2052|609|7937|Tuesday|2052Q1|N|N|N|2470569|2470599|2470209|2470482|N|N|N|N|N| +2470575|AAAAAAAAPKCLFCAA|2052-02-07|1825|7937|609|2052|3|2|7|1|2052|609|7937|Wednesday|2052Q1|N|N|N|2470569|2470599|2470210|2470483|N|N|N|N|N| +2470576|AAAAAAAAALCLFCAA|2052-02-08|1825|7937|609|2052|4|2|8|1|2052|609|7937|Thursday|2052Q1|N|N|N|2470569|2470599|2470211|2470484|N|N|N|N|N| +2470577|AAAAAAAABLCLFCAA|2052-02-09|1825|7937|609|2052|5|2|9|1|2052|609|7937|Friday|2052Q1|N|Y|N|2470569|2470599|2470212|2470485|N|N|N|N|N| +2470578|AAAAAAAACLCLFCAA|2052-02-10|1825|7937|609|2052|6|2|10|1|2052|609|7937|Saturday|2052Q1|N|Y|N|2470569|2470599|2470213|2470486|N|N|N|N|N| +2470579|AAAAAAAADLCLFCAA|2052-02-11|1825|7937|609|2052|0|2|11|1|2052|609|7937|Sunday|2052Q1|N|N|N|2470569|2470599|2470214|2470487|N|N|N|N|N| +2470580|AAAAAAAAELCLFCAA|2052-02-12|1825|7937|609|2052|1|2|12|1|2052|609|7937|Monday|2052Q1|N|N|N|2470569|2470599|2470215|2470488|N|N|N|N|N| +2470581|AAAAAAAAFLCLFCAA|2052-02-13|1825|7938|609|2052|2|2|13|1|2052|609|7938|Tuesday|2052Q1|N|N|N|2470569|2470599|2470216|2470489|N|N|N|N|N| +2470582|AAAAAAAAGLCLFCAA|2052-02-14|1825|7938|609|2052|3|2|14|1|2052|609|7938|Wednesday|2052Q1|N|N|N|2470569|2470599|2470217|2470490|N|N|N|N|N| +2470583|AAAAAAAAHLCLFCAA|2052-02-15|1825|7938|609|2052|4|2|15|1|2052|609|7938|Thursday|2052Q1|N|N|N|2470569|2470599|2470218|2470491|N|N|N|N|N| +2470584|AAAAAAAAILCLFCAA|2052-02-16|1825|7938|609|2052|5|2|16|1|2052|609|7938|Friday|2052Q1|N|Y|N|2470569|2470599|2470219|2470492|N|N|N|N|N| +2470585|AAAAAAAAJLCLFCAA|2052-02-17|1825|7938|609|2052|6|2|17|1|2052|609|7938|Saturday|2052Q1|N|Y|N|2470569|2470599|2470220|2470493|N|N|N|N|N| +2470586|AAAAAAAAKLCLFCAA|2052-02-18|1825|7938|609|2052|0|2|18|1|2052|609|7938|Sunday|2052Q1|N|N|N|2470569|2470599|2470221|2470494|N|N|N|N|N| +2470587|AAAAAAAALLCLFCAA|2052-02-19|1825|7938|609|2052|1|2|19|1|2052|609|7938|Monday|2052Q1|N|N|N|2470569|2470599|2470222|2470495|N|N|N|N|N| +2470588|AAAAAAAAMLCLFCAA|2052-02-20|1825|7939|609|2052|2|2|20|1|2052|609|7939|Tuesday|2052Q1|N|N|N|2470569|2470599|2470223|2470496|N|N|N|N|N| +2470589|AAAAAAAANLCLFCAA|2052-02-21|1825|7939|609|2052|3|2|21|1|2052|609|7939|Wednesday|2052Q1|N|N|N|2470569|2470599|2470224|2470497|N|N|N|N|N| +2470590|AAAAAAAAOLCLFCAA|2052-02-22|1825|7939|609|2052|4|2|22|1|2052|609|7939|Thursday|2052Q1|N|N|N|2470569|2470599|2470225|2470498|N|N|N|N|N| +2470591|AAAAAAAAPLCLFCAA|2052-02-23|1825|7939|609|2052|5|2|23|1|2052|609|7939|Friday|2052Q1|N|Y|N|2470569|2470599|2470226|2470499|N|N|N|N|N| +2470592|AAAAAAAAAMCLFCAA|2052-02-24|1825|7939|609|2052|6|2|24|1|2052|609|7939|Saturday|2052Q1|N|Y|N|2470569|2470599|2470227|2470500|N|N|N|N|N| +2470593|AAAAAAAABMCLFCAA|2052-02-25|1825|7939|609|2052|0|2|25|1|2052|609|7939|Sunday|2052Q1|N|N|N|2470569|2470599|2470228|2470501|N|N|N|N|N| +2470594|AAAAAAAACMCLFCAA|2052-02-26|1825|7939|609|2052|1|2|26|1|2052|609|7939|Monday|2052Q1|N|N|N|2470569|2470599|2470229|2470502|N|N|N|N|N| +2470595|AAAAAAAADMCLFCAA|2052-02-27|1825|7940|609|2052|2|2|27|1|2052|609|7940|Tuesday|2052Q1|N|N|N|2470569|2470599|2470230|2470503|N|N|N|N|N| +2470596|AAAAAAAAEMCLFCAA|2052-02-28|1825|7940|609|2052|3|2|28|1|2052|609|7940|Wednesday|2052Q1|N|N|N|2470569|2470599|2470231|2470504|N|N|N|N|N| +2470597|AAAAAAAAFMCLFCAA|2052-02-29|1825|7940|609|2052|4|2|29|1|2052|609|7940|Thursday|2052Q1|N|N|N|2470569|2470599|2470231|2470505|N|N|N|N|N| +2470598|AAAAAAAAGMCLFCAA|2052-03-01|1826|7940|610|2052|5|3|1|1|2052|610|7940|Friday|2052Q1|N|Y|N|2470598|2470657|2470232|2470506|N|N|N|N|N| +2470599|AAAAAAAAHMCLFCAA|2052-03-02|1826|7940|610|2052|6|3|2|1|2052|610|7940|Saturday|2052Q1|N|Y|N|2470598|2470657|2470233|2470507|N|N|N|N|N| +2470600|AAAAAAAAIMCLFCAA|2052-03-03|1826|7940|610|2052|0|3|3|1|2052|610|7940|Sunday|2052Q1|N|N|N|2470598|2470657|2470234|2470508|N|N|N|N|N| +2470601|AAAAAAAAJMCLFCAA|2052-03-04|1826|7940|610|2052|1|3|4|1|2052|610|7940|Monday|2052Q1|N|N|N|2470598|2470657|2470235|2470509|N|N|N|N|N| +2470602|AAAAAAAAKMCLFCAA|2052-03-05|1826|7941|610|2052|2|3|5|1|2052|610|7941|Tuesday|2052Q1|N|N|N|2470598|2470657|2470236|2470510|N|N|N|N|N| +2470603|AAAAAAAALMCLFCAA|2052-03-06|1826|7941|610|2052|3|3|6|1|2052|610|7941|Wednesday|2052Q1|N|N|N|2470598|2470657|2470237|2470511|N|N|N|N|N| +2470604|AAAAAAAAMMCLFCAA|2052-03-07|1826|7941|610|2052|4|3|7|1|2052|610|7941|Thursday|2052Q1|N|N|N|2470598|2470657|2470238|2470512|N|N|N|N|N| +2470605|AAAAAAAANMCLFCAA|2052-03-08|1826|7941|610|2052|5|3|8|1|2052|610|7941|Friday|2052Q1|N|Y|N|2470598|2470657|2470239|2470513|N|N|N|N|N| +2470606|AAAAAAAAOMCLFCAA|2052-03-09|1826|7941|610|2052|6|3|9|1|2052|610|7941|Saturday|2052Q1|N|Y|N|2470598|2470657|2470240|2470514|N|N|N|N|N| +2470607|AAAAAAAAPMCLFCAA|2052-03-10|1826|7941|610|2052|0|3|10|1|2052|610|7941|Sunday|2052Q1|N|N|N|2470598|2470657|2470241|2470515|N|N|N|N|N| +2470608|AAAAAAAAANCLFCAA|2052-03-11|1826|7941|610|2052|1|3|11|1|2052|610|7941|Monday|2052Q1|N|N|N|2470598|2470657|2470242|2470516|N|N|N|N|N| +2470609|AAAAAAAABNCLFCAA|2052-03-12|1826|7942|610|2052|2|3|12|1|2052|610|7942|Tuesday|2052Q1|N|N|N|2470598|2470657|2470243|2470517|N|N|N|N|N| +2470610|AAAAAAAACNCLFCAA|2052-03-13|1826|7942|610|2052|3|3|13|1|2052|610|7942|Wednesday|2052Q1|N|N|N|2470598|2470657|2470244|2470518|N|N|N|N|N| +2470611|AAAAAAAADNCLFCAA|2052-03-14|1826|7942|610|2052|4|3|14|1|2052|610|7942|Thursday|2052Q1|N|N|N|2470598|2470657|2470245|2470519|N|N|N|N|N| +2470612|AAAAAAAAENCLFCAA|2052-03-15|1826|7942|610|2052|5|3|15|1|2052|610|7942|Friday|2052Q1|N|Y|N|2470598|2470657|2470246|2470520|N|N|N|N|N| +2470613|AAAAAAAAFNCLFCAA|2052-03-16|1826|7942|610|2052|6|3|16|1|2052|610|7942|Saturday|2052Q1|N|Y|N|2470598|2470657|2470247|2470521|N|N|N|N|N| +2470614|AAAAAAAAGNCLFCAA|2052-03-17|1826|7942|610|2052|0|3|17|1|2052|610|7942|Sunday|2052Q1|N|N|N|2470598|2470657|2470248|2470522|N|N|N|N|N| +2470615|AAAAAAAAHNCLFCAA|2052-03-18|1826|7942|610|2052|1|3|18|1|2052|610|7942|Monday|2052Q1|N|N|N|2470598|2470657|2470249|2470523|N|N|N|N|N| +2470616|AAAAAAAAINCLFCAA|2052-03-19|1826|7943|610|2052|2|3|19|1|2052|610|7943|Tuesday|2052Q1|N|N|N|2470598|2470657|2470250|2470524|N|N|N|N|N| +2470617|AAAAAAAAJNCLFCAA|2052-03-20|1826|7943|610|2052|3|3|20|1|2052|610|7943|Wednesday|2052Q1|N|N|N|2470598|2470657|2470251|2470525|N|N|N|N|N| +2470618|AAAAAAAAKNCLFCAA|2052-03-21|1826|7943|610|2052|4|3|21|1|2052|610|7943|Thursday|2052Q1|N|N|N|2470598|2470657|2470252|2470526|N|N|N|N|N| +2470619|AAAAAAAALNCLFCAA|2052-03-22|1826|7943|610|2052|5|3|22|1|2052|610|7943|Friday|2052Q1|N|Y|N|2470598|2470657|2470253|2470527|N|N|N|N|N| +2470620|AAAAAAAAMNCLFCAA|2052-03-23|1826|7943|610|2052|6|3|23|1|2052|610|7943|Saturday|2052Q1|N|Y|N|2470598|2470657|2470254|2470528|N|N|N|N|N| +2470621|AAAAAAAANNCLFCAA|2052-03-24|1826|7943|610|2052|0|3|24|1|2052|610|7943|Sunday|2052Q1|N|N|N|2470598|2470657|2470255|2470529|N|N|N|N|N| +2470622|AAAAAAAAONCLFCAA|2052-03-25|1826|7943|610|2052|1|3|25|1|2052|610|7943|Monday|2052Q1|N|N|N|2470598|2470657|2470256|2470530|N|N|N|N|N| +2470623|AAAAAAAAPNCLFCAA|2052-03-26|1826|7944|610|2052|2|3|26|1|2052|610|7944|Tuesday|2052Q1|N|N|N|2470598|2470657|2470257|2470531|N|N|N|N|N| +2470624|AAAAAAAAAOCLFCAA|2052-03-27|1826|7944|610|2052|3|3|27|1|2052|610|7944|Wednesday|2052Q1|N|N|N|2470598|2470657|2470258|2470532|N|N|N|N|N| +2470625|AAAAAAAABOCLFCAA|2052-03-28|1826|7944|610|2052|4|3|28|1|2052|610|7944|Thursday|2052Q1|N|N|N|2470598|2470657|2470259|2470533|N|N|N|N|N| +2470626|AAAAAAAACOCLFCAA|2052-03-29|1826|7944|610|2052|5|3|29|1|2052|610|7944|Friday|2052Q1|N|Y|N|2470598|2470657|2470260|2470534|N|N|N|N|N| +2470627|AAAAAAAADOCLFCAA|2052-03-30|1826|7944|610|2052|6|3|30|1|2052|610|7944|Saturday|2052Q1|N|Y|N|2470598|2470657|2470261|2470535|N|N|N|N|N| +2470628|AAAAAAAAEOCLFCAA|2052-03-31|1826|7944|610|2052|0|3|31|1|2052|610|7944|Sunday|2052Q1|N|N|N|2470598|2470657|2470262|2470536|N|N|N|N|N| +2470629|AAAAAAAAFOCLFCAA|2052-04-01|1827|7944|610|2052|1|4|1|2|2052|610|7944|Monday|2052Q2|N|N|N|2470629|2470719|2470263|2470538|N|N|N|N|N| +2470630|AAAAAAAAGOCLFCAA|2052-04-02|1827|7945|610|2052|2|4|2|2|2052|610|7945|Tuesday|2052Q2|N|N|N|2470629|2470719|2470264|2470539|N|N|N|N|N| +2470631|AAAAAAAAHOCLFCAA|2052-04-03|1827|7945|610|2052|3|4|3|2|2052|610|7945|Wednesday|2052Q2|N|N|N|2470629|2470719|2470265|2470540|N|N|N|N|N| +2470632|AAAAAAAAIOCLFCAA|2052-04-04|1827|7945|610|2052|4|4|4|2|2052|610|7945|Thursday|2052Q2|N|N|N|2470629|2470719|2470266|2470541|N|N|N|N|N| +2470633|AAAAAAAAJOCLFCAA|2052-04-05|1827|7945|610|2052|5|4|5|2|2052|610|7945|Friday|2052Q2|N|Y|N|2470629|2470719|2470267|2470542|N|N|N|N|N| +2470634|AAAAAAAAKOCLFCAA|2052-04-06|1827|7945|610|2052|6|4|6|2|2052|610|7945|Saturday|2052Q2|N|Y|N|2470629|2470719|2470268|2470543|N|N|N|N|N| +2470635|AAAAAAAALOCLFCAA|2052-04-07|1827|7945|610|2052|0|4|7|2|2052|610|7945|Sunday|2052Q2|N|N|N|2470629|2470719|2470269|2470544|N|N|N|N|N| +2470636|AAAAAAAAMOCLFCAA|2052-04-08|1827|7945|610|2052|1|4|8|2|2052|610|7945|Monday|2052Q2|N|N|N|2470629|2470719|2470270|2470545|N|N|N|N|N| +2470637|AAAAAAAANOCLFCAA|2052-04-09|1827|7946|610|2052|2|4|9|2|2052|610|7946|Tuesday|2052Q2|N|N|N|2470629|2470719|2470271|2470546|N|N|N|N|N| +2470638|AAAAAAAAOOCLFCAA|2052-04-10|1827|7946|610|2052|3|4|10|2|2052|610|7946|Wednesday|2052Q2|N|N|N|2470629|2470719|2470272|2470547|N|N|N|N|N| +2470639|AAAAAAAAPOCLFCAA|2052-04-11|1827|7946|610|2052|4|4|11|2|2052|610|7946|Thursday|2052Q2|N|N|N|2470629|2470719|2470273|2470548|N|N|N|N|N| +2470640|AAAAAAAAAPCLFCAA|2052-04-12|1827|7946|610|2052|5|4|12|2|2052|610|7946|Friday|2052Q2|N|Y|N|2470629|2470719|2470274|2470549|N|N|N|N|N| +2470641|AAAAAAAABPCLFCAA|2052-04-13|1827|7946|610|2052|6|4|13|2|2052|610|7946|Saturday|2052Q2|N|Y|N|2470629|2470719|2470275|2470550|N|N|N|N|N| +2470642|AAAAAAAACPCLFCAA|2052-04-14|1827|7946|610|2052|0|4|14|2|2052|610|7946|Sunday|2052Q2|N|N|N|2470629|2470719|2470276|2470551|N|N|N|N|N| +2470643|AAAAAAAADPCLFCAA|2052-04-15|1827|7946|610|2052|1|4|15|2|2052|610|7946|Monday|2052Q2|N|N|N|2470629|2470719|2470277|2470552|N|N|N|N|N| +2470644|AAAAAAAAEPCLFCAA|2052-04-16|1827|7947|610|2052|2|4|16|2|2052|610|7947|Tuesday|2052Q2|N|N|N|2470629|2470719|2470278|2470553|N|N|N|N|N| +2470645|AAAAAAAAFPCLFCAA|2052-04-17|1827|7947|610|2052|3|4|17|2|2052|610|7947|Wednesday|2052Q2|N|N|N|2470629|2470719|2470279|2470554|N|N|N|N|N| +2470646|AAAAAAAAGPCLFCAA|2052-04-18|1827|7947|610|2052|4|4|18|2|2052|610|7947|Thursday|2052Q2|N|N|N|2470629|2470719|2470280|2470555|N|N|N|N|N| +2470647|AAAAAAAAHPCLFCAA|2052-04-19|1827|7947|610|2052|5|4|19|2|2052|610|7947|Friday|2052Q2|N|Y|N|2470629|2470719|2470281|2470556|N|N|N|N|N| +2470648|AAAAAAAAIPCLFCAA|2052-04-20|1827|7947|610|2052|6|4|20|2|2052|610|7947|Saturday|2052Q2|N|Y|N|2470629|2470719|2470282|2470557|N|N|N|N|N| +2470649|AAAAAAAAJPCLFCAA|2052-04-21|1827|7947|610|2052|0|4|21|2|2052|610|7947|Sunday|2052Q2|N|N|N|2470629|2470719|2470283|2470558|N|N|N|N|N| +2470650|AAAAAAAAKPCLFCAA|2052-04-22|1827|7947|610|2052|1|4|22|2|2052|610|7947|Monday|2052Q2|N|N|N|2470629|2470719|2470284|2470559|N|N|N|N|N| +2470651|AAAAAAAALPCLFCAA|2052-04-23|1827|7948|610|2052|2|4|23|2|2052|610|7948|Tuesday|2052Q2|N|N|N|2470629|2470719|2470285|2470560|N|N|N|N|N| +2470652|AAAAAAAAMPCLFCAA|2052-04-24|1827|7948|610|2052|3|4|24|2|2052|610|7948|Wednesday|2052Q2|N|N|N|2470629|2470719|2470286|2470561|N|N|N|N|N| +2470653|AAAAAAAANPCLFCAA|2052-04-25|1827|7948|610|2052|4|4|25|2|2052|610|7948|Thursday|2052Q2|N|N|N|2470629|2470719|2470287|2470562|N|N|N|N|N| +2470654|AAAAAAAAOPCLFCAA|2052-04-26|1827|7948|610|2052|5|4|26|2|2052|610|7948|Friday|2052Q2|N|Y|N|2470629|2470719|2470288|2470563|N|N|N|N|N| +2470655|AAAAAAAAPPCLFCAA|2052-04-27|1827|7948|610|2052|6|4|27|2|2052|610|7948|Saturday|2052Q2|N|Y|N|2470629|2470719|2470289|2470564|N|N|N|N|N| +2470656|AAAAAAAAAADLFCAA|2052-04-28|1827|7948|610|2052|0|4|28|2|2052|610|7948|Sunday|2052Q2|N|N|N|2470629|2470719|2470290|2470565|N|N|N|N|N| +2470657|AAAAAAAABADLFCAA|2052-04-29|1827|7948|610|2052|1|4|29|2|2052|610|7948|Monday|2052Q2|N|N|N|2470629|2470719|2470291|2470566|N|N|N|N|N| +2470658|AAAAAAAACADLFCAA|2052-04-30|1827|7949|610|2052|2|4|30|2|2052|610|7949|Tuesday|2052Q2|N|N|N|2470629|2470719|2470292|2470567|N|N|N|N|N| +2470659|AAAAAAAADADLFCAA|2052-05-01|1828|7949|610|2052|3|5|1|2|2052|610|7949|Wednesday|2052Q2|N|N|N|2470659|2470779|2470293|2470568|N|N|N|N|N| +2470660|AAAAAAAAEADLFCAA|2052-05-02|1828|7949|610|2052|4|5|2|2|2052|610|7949|Thursday|2052Q2|N|N|N|2470659|2470779|2470294|2470569|N|N|N|N|N| +2470661|AAAAAAAAFADLFCAA|2052-05-03|1828|7949|610|2052|5|5|3|2|2052|610|7949|Friday|2052Q2|N|Y|N|2470659|2470779|2470295|2470570|N|N|N|N|N| +2470662|AAAAAAAAGADLFCAA|2052-05-04|1828|7949|610|2052|6|5|4|2|2052|610|7949|Saturday|2052Q2|N|Y|N|2470659|2470779|2470296|2470571|N|N|N|N|N| +2470663|AAAAAAAAHADLFCAA|2052-05-05|1828|7949|610|2052|0|5|5|2|2052|610|7949|Sunday|2052Q2|N|N|N|2470659|2470779|2470297|2470572|N|N|N|N|N| +2470664|AAAAAAAAIADLFCAA|2052-05-06|1828|7949|610|2052|1|5|6|2|2052|610|7949|Monday|2052Q2|N|N|N|2470659|2470779|2470298|2470573|N|N|N|N|N| +2470665|AAAAAAAAJADLFCAA|2052-05-07|1828|7950|610|2052|2|5|7|2|2052|610|7950|Tuesday|2052Q2|N|N|N|2470659|2470779|2470299|2470574|N|N|N|N|N| +2470666|AAAAAAAAKADLFCAA|2052-05-08|1828|7950|610|2052|3|5|8|2|2052|610|7950|Wednesday|2052Q2|N|N|N|2470659|2470779|2470300|2470575|N|N|N|N|N| +2470667|AAAAAAAALADLFCAA|2052-05-09|1828|7950|610|2052|4|5|9|2|2052|610|7950|Thursday|2052Q2|N|N|N|2470659|2470779|2470301|2470576|N|N|N|N|N| +2470668|AAAAAAAAMADLFCAA|2052-05-10|1828|7950|610|2052|5|5|10|2|2052|610|7950|Friday|2052Q2|N|Y|N|2470659|2470779|2470302|2470577|N|N|N|N|N| +2470669|AAAAAAAANADLFCAA|2052-05-11|1828|7950|610|2052|6|5|11|2|2052|610|7950|Saturday|2052Q2|N|Y|N|2470659|2470779|2470303|2470578|N|N|N|N|N| +2470670|AAAAAAAAOADLFCAA|2052-05-12|1828|7950|610|2052|0|5|12|2|2052|610|7950|Sunday|2052Q2|N|N|N|2470659|2470779|2470304|2470579|N|N|N|N|N| +2470671|AAAAAAAAPADLFCAA|2052-05-13|1828|7950|610|2052|1|5|13|2|2052|610|7950|Monday|2052Q2|N|N|N|2470659|2470779|2470305|2470580|N|N|N|N|N| +2470672|AAAAAAAAABDLFCAA|2052-05-14|1828|7951|610|2052|2|5|14|2|2052|610|7951|Tuesday|2052Q2|N|N|N|2470659|2470779|2470306|2470581|N|N|N|N|N| +2470673|AAAAAAAABBDLFCAA|2052-05-15|1828|7951|610|2052|3|5|15|2|2052|610|7951|Wednesday|2052Q2|N|N|N|2470659|2470779|2470307|2470582|N|N|N|N|N| +2470674|AAAAAAAACBDLFCAA|2052-05-16|1828|7951|610|2052|4|5|16|2|2052|610|7951|Thursday|2052Q2|N|N|N|2470659|2470779|2470308|2470583|N|N|N|N|N| +2470675|AAAAAAAADBDLFCAA|2052-05-17|1828|7951|610|2052|5|5|17|2|2052|610|7951|Friday|2052Q2|N|Y|N|2470659|2470779|2470309|2470584|N|N|N|N|N| +2470676|AAAAAAAAEBDLFCAA|2052-05-18|1828|7951|610|2052|6|5|18|2|2052|610|7951|Saturday|2052Q2|N|Y|N|2470659|2470779|2470310|2470585|N|N|N|N|N| +2470677|AAAAAAAAFBDLFCAA|2052-05-19|1828|7951|610|2052|0|5|19|2|2052|610|7951|Sunday|2052Q2|N|N|N|2470659|2470779|2470311|2470586|N|N|N|N|N| +2470678|AAAAAAAAGBDLFCAA|2052-05-20|1828|7951|610|2052|1|5|20|2|2052|610|7951|Monday|2052Q2|N|N|N|2470659|2470779|2470312|2470587|N|N|N|N|N| +2470679|AAAAAAAAHBDLFCAA|2052-05-21|1828|7952|610|2052|2|5|21|2|2052|610|7952|Tuesday|2052Q2|N|N|N|2470659|2470779|2470313|2470588|N|N|N|N|N| +2470680|AAAAAAAAIBDLFCAA|2052-05-22|1828|7952|610|2052|3|5|22|2|2052|610|7952|Wednesday|2052Q2|N|N|N|2470659|2470779|2470314|2470589|N|N|N|N|N| +2470681|AAAAAAAAJBDLFCAA|2052-05-23|1828|7952|610|2052|4|5|23|2|2052|610|7952|Thursday|2052Q2|N|N|N|2470659|2470779|2470315|2470590|N|N|N|N|N| +2470682|AAAAAAAAKBDLFCAA|2052-05-24|1828|7952|610|2052|5|5|24|2|2052|610|7952|Friday|2052Q2|N|Y|N|2470659|2470779|2470316|2470591|N|N|N|N|N| +2470683|AAAAAAAALBDLFCAA|2052-05-25|1828|7952|610|2052|6|5|25|2|2052|610|7952|Saturday|2052Q2|N|Y|N|2470659|2470779|2470317|2470592|N|N|N|N|N| +2470684|AAAAAAAAMBDLFCAA|2052-05-26|1828|7952|610|2052|0|5|26|2|2052|610|7952|Sunday|2052Q2|N|N|N|2470659|2470779|2470318|2470593|N|N|N|N|N| +2470685|AAAAAAAANBDLFCAA|2052-05-27|1828|7952|610|2052|1|5|27|2|2052|610|7952|Monday|2052Q2|N|N|N|2470659|2470779|2470319|2470594|N|N|N|N|N| +2470686|AAAAAAAAOBDLFCAA|2052-05-28|1828|7953|610|2052|2|5|28|2|2052|610|7953|Tuesday|2052Q2|N|N|N|2470659|2470779|2470320|2470595|N|N|N|N|N| +2470687|AAAAAAAAPBDLFCAA|2052-05-29|1828|7953|610|2052|3|5|29|2|2052|610|7953|Wednesday|2052Q2|N|N|N|2470659|2470779|2470321|2470596|N|N|N|N|N| +2470688|AAAAAAAAACDLFCAA|2052-05-30|1828|7953|610|2052|4|5|30|2|2052|610|7953|Thursday|2052Q2|N|N|N|2470659|2470779|2470322|2470597|N|N|N|N|N| +2470689|AAAAAAAABCDLFCAA|2052-05-31|1828|7953|610|2052|5|5|31|2|2052|610|7953|Friday|2052Q2|N|Y|N|2470659|2470779|2470323|2470598|N|N|N|N|N| +2470690|AAAAAAAACCDLFCAA|2052-06-01|1829|7953|611|2052|6|6|1|2|2052|611|7953|Saturday|2052Q2|N|Y|N|2470690|2470841|2470324|2470599|N|N|N|N|N| +2470691|AAAAAAAADCDLFCAA|2052-06-02|1829|7953|611|2052|0|6|2|2|2052|611|7953|Sunday|2052Q2|N|N|N|2470690|2470841|2470325|2470600|N|N|N|N|N| +2470692|AAAAAAAAECDLFCAA|2052-06-03|1829|7953|611|2052|1|6|3|2|2052|611|7953|Monday|2052Q2|N|N|N|2470690|2470841|2470326|2470601|N|N|N|N|N| +2470693|AAAAAAAAFCDLFCAA|2052-06-04|1829|7954|611|2052|2|6|4|2|2052|611|7954|Tuesday|2052Q2|N|N|N|2470690|2470841|2470327|2470602|N|N|N|N|N| +2470694|AAAAAAAAGCDLFCAA|2052-06-05|1829|7954|611|2052|3|6|5|2|2052|611|7954|Wednesday|2052Q2|N|N|N|2470690|2470841|2470328|2470603|N|N|N|N|N| +2470695|AAAAAAAAHCDLFCAA|2052-06-06|1829|7954|611|2052|4|6|6|2|2052|611|7954|Thursday|2052Q2|N|N|N|2470690|2470841|2470329|2470604|N|N|N|N|N| +2470696|AAAAAAAAICDLFCAA|2052-06-07|1829|7954|611|2052|5|6|7|2|2052|611|7954|Friday|2052Q2|N|Y|N|2470690|2470841|2470330|2470605|N|N|N|N|N| +2470697|AAAAAAAAJCDLFCAA|2052-06-08|1829|7954|611|2052|6|6|8|2|2052|611|7954|Saturday|2052Q2|N|Y|N|2470690|2470841|2470331|2470606|N|N|N|N|N| +2470698|AAAAAAAAKCDLFCAA|2052-06-09|1829|7954|611|2052|0|6|9|2|2052|611|7954|Sunday|2052Q2|N|N|N|2470690|2470841|2470332|2470607|N|N|N|N|N| +2470699|AAAAAAAALCDLFCAA|2052-06-10|1829|7954|611|2052|1|6|10|2|2052|611|7954|Monday|2052Q2|N|N|N|2470690|2470841|2470333|2470608|N|N|N|N|N| +2470700|AAAAAAAAMCDLFCAA|2052-06-11|1829|7955|611|2052|2|6|11|2|2052|611|7955|Tuesday|2052Q2|N|N|N|2470690|2470841|2470334|2470609|N|N|N|N|N| +2470701|AAAAAAAANCDLFCAA|2052-06-12|1829|7955|611|2052|3|6|12|2|2052|611|7955|Wednesday|2052Q2|N|N|N|2470690|2470841|2470335|2470610|N|N|N|N|N| +2470702|AAAAAAAAOCDLFCAA|2052-06-13|1829|7955|611|2052|4|6|13|2|2052|611|7955|Thursday|2052Q2|N|N|N|2470690|2470841|2470336|2470611|N|N|N|N|N| +2470703|AAAAAAAAPCDLFCAA|2052-06-14|1829|7955|611|2052|5|6|14|2|2052|611|7955|Friday|2052Q2|N|Y|N|2470690|2470841|2470337|2470612|N|N|N|N|N| +2470704|AAAAAAAAADDLFCAA|2052-06-15|1829|7955|611|2052|6|6|15|2|2052|611|7955|Saturday|2052Q2|N|Y|N|2470690|2470841|2470338|2470613|N|N|N|N|N| +2470705|AAAAAAAABDDLFCAA|2052-06-16|1829|7955|611|2052|0|6|16|2|2052|611|7955|Sunday|2052Q2|N|N|N|2470690|2470841|2470339|2470614|N|N|N|N|N| +2470706|AAAAAAAACDDLFCAA|2052-06-17|1829|7955|611|2052|1|6|17|2|2052|611|7955|Monday|2052Q2|N|N|N|2470690|2470841|2470340|2470615|N|N|N|N|N| +2470707|AAAAAAAADDDLFCAA|2052-06-18|1829|7956|611|2052|2|6|18|2|2052|611|7956|Tuesday|2052Q2|N|N|N|2470690|2470841|2470341|2470616|N|N|N|N|N| +2470708|AAAAAAAAEDDLFCAA|2052-06-19|1829|7956|611|2052|3|6|19|2|2052|611|7956|Wednesday|2052Q2|N|N|N|2470690|2470841|2470342|2470617|N|N|N|N|N| +2470709|AAAAAAAAFDDLFCAA|2052-06-20|1829|7956|611|2052|4|6|20|2|2052|611|7956|Thursday|2052Q2|N|N|N|2470690|2470841|2470343|2470618|N|N|N|N|N| +2470710|AAAAAAAAGDDLFCAA|2052-06-21|1829|7956|611|2052|5|6|21|2|2052|611|7956|Friday|2052Q2|N|Y|N|2470690|2470841|2470344|2470619|N|N|N|N|N| +2470711|AAAAAAAAHDDLFCAA|2052-06-22|1829|7956|611|2052|6|6|22|2|2052|611|7956|Saturday|2052Q2|N|Y|N|2470690|2470841|2470345|2470620|N|N|N|N|N| +2470712|AAAAAAAAIDDLFCAA|2052-06-23|1829|7956|611|2052|0|6|23|2|2052|611|7956|Sunday|2052Q2|N|N|N|2470690|2470841|2470346|2470621|N|N|N|N|N| +2470713|AAAAAAAAJDDLFCAA|2052-06-24|1829|7956|611|2052|1|6|24|2|2052|611|7956|Monday|2052Q2|N|N|N|2470690|2470841|2470347|2470622|N|N|N|N|N| +2470714|AAAAAAAAKDDLFCAA|2052-06-25|1829|7957|611|2052|2|6|25|2|2052|611|7957|Tuesday|2052Q2|N|N|N|2470690|2470841|2470348|2470623|N|N|N|N|N| +2470715|AAAAAAAALDDLFCAA|2052-06-26|1829|7957|611|2052|3|6|26|2|2052|611|7957|Wednesday|2052Q2|N|N|N|2470690|2470841|2470349|2470624|N|N|N|N|N| +2470716|AAAAAAAAMDDLFCAA|2052-06-27|1829|7957|611|2052|4|6|27|2|2052|611|7957|Thursday|2052Q2|N|N|N|2470690|2470841|2470350|2470625|N|N|N|N|N| +2470717|AAAAAAAANDDLFCAA|2052-06-28|1829|7957|611|2052|5|6|28|2|2052|611|7957|Friday|2052Q2|N|Y|N|2470690|2470841|2470351|2470626|N|N|N|N|N| +2470718|AAAAAAAAODDLFCAA|2052-06-29|1829|7957|611|2052|6|6|29|2|2052|611|7957|Saturday|2052Q2|N|Y|N|2470690|2470841|2470352|2470627|N|N|N|N|N| +2470719|AAAAAAAAPDDLFCAA|2052-06-30|1829|7957|611|2052|0|6|30|2|2052|611|7957|Sunday|2052Q2|N|N|N|2470690|2470841|2470353|2470628|N|N|N|N|N| +2470720|AAAAAAAAAEDLFCAA|2052-07-01|1830|7957|611|2052|1|7|1|3|2052|611|7957|Monday|2052Q3|N|N|N|2470720|2470901|2470354|2470629|N|N|N|N|N| +2470721|AAAAAAAABEDLFCAA|2052-07-02|1830|7958|611|2052|2|7|2|3|2052|611|7958|Tuesday|2052Q3|N|N|N|2470720|2470901|2470355|2470630|N|N|N|N|N| +2470722|AAAAAAAACEDLFCAA|2052-07-03|1830|7958|611|2052|3|7|3|3|2052|611|7958|Wednesday|2052Q3|N|N|N|2470720|2470901|2470356|2470631|N|N|N|N|N| +2470723|AAAAAAAADEDLFCAA|2052-07-04|1830|7958|611|2052|4|7|4|3|2052|611|7958|Thursday|2052Q3|Y|N|N|2470720|2470901|2470357|2470632|N|N|N|N|N| +2470724|AAAAAAAAEEDLFCAA|2052-07-05|1830|7958|611|2052|5|7|5|3|2052|611|7958|Friday|2052Q3|N|Y|Y|2470720|2470901|2470358|2470633|N|N|N|N|N| +2470725|AAAAAAAAFEDLFCAA|2052-07-06|1830|7958|611|2052|6|7|6|3|2052|611|7958|Saturday|2052Q3|N|Y|N|2470720|2470901|2470359|2470634|N|N|N|N|N| +2470726|AAAAAAAAGEDLFCAA|2052-07-07|1830|7958|611|2052|0|7|7|3|2052|611|7958|Sunday|2052Q3|N|N|N|2470720|2470901|2470360|2470635|N|N|N|N|N| +2470727|AAAAAAAAHEDLFCAA|2052-07-08|1830|7958|611|2052|1|7|8|3|2052|611|7958|Monday|2052Q3|N|N|N|2470720|2470901|2470361|2470636|N|N|N|N|N| +2470728|AAAAAAAAIEDLFCAA|2052-07-09|1830|7959|611|2052|2|7|9|3|2052|611|7959|Tuesday|2052Q3|N|N|N|2470720|2470901|2470362|2470637|N|N|N|N|N| +2470729|AAAAAAAAJEDLFCAA|2052-07-10|1830|7959|611|2052|3|7|10|3|2052|611|7959|Wednesday|2052Q3|N|N|N|2470720|2470901|2470363|2470638|N|N|N|N|N| +2470730|AAAAAAAAKEDLFCAA|2052-07-11|1830|7959|611|2052|4|7|11|3|2052|611|7959|Thursday|2052Q3|N|N|N|2470720|2470901|2470364|2470639|N|N|N|N|N| +2470731|AAAAAAAALEDLFCAA|2052-07-12|1830|7959|611|2052|5|7|12|3|2052|611|7959|Friday|2052Q3|N|Y|N|2470720|2470901|2470365|2470640|N|N|N|N|N| +2470732|AAAAAAAAMEDLFCAA|2052-07-13|1830|7959|611|2052|6|7|13|3|2052|611|7959|Saturday|2052Q3|N|Y|N|2470720|2470901|2470366|2470641|N|N|N|N|N| +2470733|AAAAAAAANEDLFCAA|2052-07-14|1830|7959|611|2052|0|7|14|3|2052|611|7959|Sunday|2052Q3|N|N|N|2470720|2470901|2470367|2470642|N|N|N|N|N| +2470734|AAAAAAAAOEDLFCAA|2052-07-15|1830|7959|611|2052|1|7|15|3|2052|611|7959|Monday|2052Q3|N|N|N|2470720|2470901|2470368|2470643|N|N|N|N|N| +2470735|AAAAAAAAPEDLFCAA|2052-07-16|1830|7960|611|2052|2|7|16|3|2052|611|7960|Tuesday|2052Q3|N|N|N|2470720|2470901|2470369|2470644|N|N|N|N|N| +2470736|AAAAAAAAAFDLFCAA|2052-07-17|1830|7960|611|2052|3|7|17|3|2052|611|7960|Wednesday|2052Q3|N|N|N|2470720|2470901|2470370|2470645|N|N|N|N|N| +2470737|AAAAAAAABFDLFCAA|2052-07-18|1830|7960|611|2052|4|7|18|3|2052|611|7960|Thursday|2052Q3|N|N|N|2470720|2470901|2470371|2470646|N|N|N|N|N| +2470738|AAAAAAAACFDLFCAA|2052-07-19|1830|7960|611|2052|5|7|19|3|2052|611|7960|Friday|2052Q3|N|Y|N|2470720|2470901|2470372|2470647|N|N|N|N|N| +2470739|AAAAAAAADFDLFCAA|2052-07-20|1830|7960|611|2052|6|7|20|3|2052|611|7960|Saturday|2052Q3|N|Y|N|2470720|2470901|2470373|2470648|N|N|N|N|N| +2470740|AAAAAAAAEFDLFCAA|2052-07-21|1830|7960|611|2052|0|7|21|3|2052|611|7960|Sunday|2052Q3|N|N|N|2470720|2470901|2470374|2470649|N|N|N|N|N| +2470741|AAAAAAAAFFDLFCAA|2052-07-22|1830|7960|611|2052|1|7|22|3|2052|611|7960|Monday|2052Q3|N|N|N|2470720|2470901|2470375|2470650|N|N|N|N|N| +2470742|AAAAAAAAGFDLFCAA|2052-07-23|1830|7961|611|2052|2|7|23|3|2052|611|7961|Tuesday|2052Q3|N|N|N|2470720|2470901|2470376|2470651|N|N|N|N|N| +2470743|AAAAAAAAHFDLFCAA|2052-07-24|1830|7961|611|2052|3|7|24|3|2052|611|7961|Wednesday|2052Q3|N|N|N|2470720|2470901|2470377|2470652|N|N|N|N|N| +2470744|AAAAAAAAIFDLFCAA|2052-07-25|1830|7961|611|2052|4|7|25|3|2052|611|7961|Thursday|2052Q3|N|N|N|2470720|2470901|2470378|2470653|N|N|N|N|N| +2470745|AAAAAAAAJFDLFCAA|2052-07-26|1830|7961|611|2052|5|7|26|3|2052|611|7961|Friday|2052Q3|N|Y|N|2470720|2470901|2470379|2470654|N|N|N|N|N| +2470746|AAAAAAAAKFDLFCAA|2052-07-27|1830|7961|611|2052|6|7|27|3|2052|611|7961|Saturday|2052Q3|N|Y|N|2470720|2470901|2470380|2470655|N|N|N|N|N| +2470747|AAAAAAAALFDLFCAA|2052-07-28|1830|7961|611|2052|0|7|28|3|2052|611|7961|Sunday|2052Q3|N|N|N|2470720|2470901|2470381|2470656|N|N|N|N|N| +2470748|AAAAAAAAMFDLFCAA|2052-07-29|1830|7961|611|2052|1|7|29|3|2052|611|7961|Monday|2052Q3|N|N|N|2470720|2470901|2470382|2470657|N|N|N|N|N| +2470749|AAAAAAAANFDLFCAA|2052-07-30|1830|7962|611|2052|2|7|30|3|2052|611|7962|Tuesday|2052Q3|N|N|N|2470720|2470901|2470383|2470658|N|N|N|N|N| +2470750|AAAAAAAAOFDLFCAA|2052-07-31|1830|7962|611|2052|3|7|31|3|2052|611|7962|Wednesday|2052Q3|N|N|N|2470720|2470901|2470384|2470659|N|N|N|N|N| +2470751|AAAAAAAAPFDLFCAA|2052-08-01|1831|7962|611|2052|4|8|1|3|2052|611|7962|Thursday|2052Q3|N|N|N|2470751|2470963|2470385|2470660|N|N|N|N|N| +2470752|AAAAAAAAAGDLFCAA|2052-08-02|1831|7962|611|2052|5|8|2|3|2052|611|7962|Friday|2052Q3|N|Y|N|2470751|2470963|2470386|2470661|N|N|N|N|N| +2470753|AAAAAAAABGDLFCAA|2052-08-03|1831|7962|611|2052|6|8|3|3|2052|611|7962|Saturday|2052Q3|N|Y|N|2470751|2470963|2470387|2470662|N|N|N|N|N| +2470754|AAAAAAAACGDLFCAA|2052-08-04|1831|7962|611|2052|0|8|4|3|2052|611|7962|Sunday|2052Q3|N|N|N|2470751|2470963|2470388|2470663|N|N|N|N|N| +2470755|AAAAAAAADGDLFCAA|2052-08-05|1831|7962|611|2052|1|8|5|3|2052|611|7962|Monday|2052Q3|N|N|N|2470751|2470963|2470389|2470664|N|N|N|N|N| +2470756|AAAAAAAAEGDLFCAA|2052-08-06|1831|7963|611|2052|2|8|6|3|2052|611|7963|Tuesday|2052Q3|N|N|N|2470751|2470963|2470390|2470665|N|N|N|N|N| +2470757|AAAAAAAAFGDLFCAA|2052-08-07|1831|7963|611|2052|3|8|7|3|2052|611|7963|Wednesday|2052Q3|N|N|N|2470751|2470963|2470391|2470666|N|N|N|N|N| +2470758|AAAAAAAAGGDLFCAA|2052-08-08|1831|7963|611|2052|4|8|8|3|2052|611|7963|Thursday|2052Q3|N|N|N|2470751|2470963|2470392|2470667|N|N|N|N|N| +2470759|AAAAAAAAHGDLFCAA|2052-08-09|1831|7963|611|2052|5|8|9|3|2052|611|7963|Friday|2052Q3|N|Y|N|2470751|2470963|2470393|2470668|N|N|N|N|N| +2470760|AAAAAAAAIGDLFCAA|2052-08-10|1831|7963|611|2052|6|8|10|3|2052|611|7963|Saturday|2052Q3|N|Y|N|2470751|2470963|2470394|2470669|N|N|N|N|N| +2470761|AAAAAAAAJGDLFCAA|2052-08-11|1831|7963|611|2052|0|8|11|3|2052|611|7963|Sunday|2052Q3|N|N|N|2470751|2470963|2470395|2470670|N|N|N|N|N| +2470762|AAAAAAAAKGDLFCAA|2052-08-12|1831|7963|611|2052|1|8|12|3|2052|611|7963|Monday|2052Q3|N|N|N|2470751|2470963|2470396|2470671|N|N|N|N|N| +2470763|AAAAAAAALGDLFCAA|2052-08-13|1831|7964|611|2052|2|8|13|3|2052|611|7964|Tuesday|2052Q3|N|N|N|2470751|2470963|2470397|2470672|N|N|N|N|N| +2470764|AAAAAAAAMGDLFCAA|2052-08-14|1831|7964|611|2052|3|8|14|3|2052|611|7964|Wednesday|2052Q3|N|N|N|2470751|2470963|2470398|2470673|N|N|N|N|N| +2470765|AAAAAAAANGDLFCAA|2052-08-15|1831|7964|611|2052|4|8|15|3|2052|611|7964|Thursday|2052Q3|N|N|N|2470751|2470963|2470399|2470674|N|N|N|N|N| +2470766|AAAAAAAAOGDLFCAA|2052-08-16|1831|7964|611|2052|5|8|16|3|2052|611|7964|Friday|2052Q3|N|Y|N|2470751|2470963|2470400|2470675|N|N|N|N|N| +2470767|AAAAAAAAPGDLFCAA|2052-08-17|1831|7964|611|2052|6|8|17|3|2052|611|7964|Saturday|2052Q3|N|Y|N|2470751|2470963|2470401|2470676|N|N|N|N|N| +2470768|AAAAAAAAAHDLFCAA|2052-08-18|1831|7964|611|2052|0|8|18|3|2052|611|7964|Sunday|2052Q3|N|N|N|2470751|2470963|2470402|2470677|N|N|N|N|N| +2470769|AAAAAAAABHDLFCAA|2052-08-19|1831|7964|611|2052|1|8|19|3|2052|611|7964|Monday|2052Q3|N|N|N|2470751|2470963|2470403|2470678|N|N|N|N|N| +2470770|AAAAAAAACHDLFCAA|2052-08-20|1831|7965|611|2052|2|8|20|3|2052|611|7965|Tuesday|2052Q3|N|N|N|2470751|2470963|2470404|2470679|N|N|N|N|N| +2470771|AAAAAAAADHDLFCAA|2052-08-21|1831|7965|611|2052|3|8|21|3|2052|611|7965|Wednesday|2052Q3|N|N|N|2470751|2470963|2470405|2470680|N|N|N|N|N| +2470772|AAAAAAAAEHDLFCAA|2052-08-22|1831|7965|611|2052|4|8|22|3|2052|611|7965|Thursday|2052Q3|N|N|N|2470751|2470963|2470406|2470681|N|N|N|N|N| +2470773|AAAAAAAAFHDLFCAA|2052-08-23|1831|7965|611|2052|5|8|23|3|2052|611|7965|Friday|2052Q3|N|Y|N|2470751|2470963|2470407|2470682|N|N|N|N|N| +2470774|AAAAAAAAGHDLFCAA|2052-08-24|1831|7965|611|2052|6|8|24|3|2052|611|7965|Saturday|2052Q3|N|Y|N|2470751|2470963|2470408|2470683|N|N|N|N|N| +2470775|AAAAAAAAHHDLFCAA|2052-08-25|1831|7965|611|2052|0|8|25|3|2052|611|7965|Sunday|2052Q3|N|N|N|2470751|2470963|2470409|2470684|N|N|N|N|N| +2470776|AAAAAAAAIHDLFCAA|2052-08-26|1831|7965|611|2052|1|8|26|3|2052|611|7965|Monday|2052Q3|N|N|N|2470751|2470963|2470410|2470685|N|N|N|N|N| +2470777|AAAAAAAAJHDLFCAA|2052-08-27|1831|7966|611|2052|2|8|27|3|2052|611|7966|Tuesday|2052Q3|N|N|N|2470751|2470963|2470411|2470686|N|N|N|N|N| +2470778|AAAAAAAAKHDLFCAA|2052-08-28|1831|7966|611|2052|3|8|28|3|2052|611|7966|Wednesday|2052Q3|N|N|N|2470751|2470963|2470412|2470687|N|N|N|N|N| +2470779|AAAAAAAALHDLFCAA|2052-08-29|1831|7966|611|2052|4|8|29|3|2052|611|7966|Thursday|2052Q3|N|N|N|2470751|2470963|2470413|2470688|N|N|N|N|N| +2470780|AAAAAAAAMHDLFCAA|2052-08-30|1831|7966|611|2052|5|8|30|3|2052|611|7966|Friday|2052Q3|N|Y|N|2470751|2470963|2470414|2470689|N|N|N|N|N| +2470781|AAAAAAAANHDLFCAA|2052-08-31|1831|7966|611|2052|6|8|31|3|2052|611|7966|Saturday|2052Q3|N|Y|N|2470751|2470963|2470415|2470690|N|N|N|N|N| +2470782|AAAAAAAAOHDLFCAA|2052-09-01|1832|7966|612|2052|0|9|1|3|2052|612|7966|Sunday|2052Q3|N|N|N|2470782|2471025|2470416|2470691|N|N|N|N|N| +2470783|AAAAAAAAPHDLFCAA|2052-09-02|1832|7966|612|2052|1|9|2|3|2052|612|7966|Monday|2052Q3|N|N|N|2470782|2471025|2470417|2470692|N|N|N|N|N| +2470784|AAAAAAAAAIDLFCAA|2052-09-03|1832|7967|612|2052|2|9|3|3|2052|612|7967|Tuesday|2052Q3|N|N|N|2470782|2471025|2470418|2470693|N|N|N|N|N| +2470785|AAAAAAAABIDLFCAA|2052-09-04|1832|7967|612|2052|3|9|4|3|2052|612|7967|Wednesday|2052Q3|N|N|N|2470782|2471025|2470419|2470694|N|N|N|N|N| +2470786|AAAAAAAACIDLFCAA|2052-09-05|1832|7967|612|2052|4|9|5|3|2052|612|7967|Thursday|2052Q3|N|N|N|2470782|2471025|2470420|2470695|N|N|N|N|N| +2470787|AAAAAAAADIDLFCAA|2052-09-06|1832|7967|612|2052|5|9|6|3|2052|612|7967|Friday|2052Q3|N|Y|N|2470782|2471025|2470421|2470696|N|N|N|N|N| +2470788|AAAAAAAAEIDLFCAA|2052-09-07|1832|7967|612|2052|6|9|7|3|2052|612|7967|Saturday|2052Q3|N|Y|N|2470782|2471025|2470422|2470697|N|N|N|N|N| +2470789|AAAAAAAAFIDLFCAA|2052-09-08|1832|7967|612|2052|0|9|8|3|2052|612|7967|Sunday|2052Q3|N|N|N|2470782|2471025|2470423|2470698|N|N|N|N|N| +2470790|AAAAAAAAGIDLFCAA|2052-09-09|1832|7967|612|2052|1|9|9|3|2052|612|7967|Monday|2052Q3|N|N|N|2470782|2471025|2470424|2470699|N|N|N|N|N| +2470791|AAAAAAAAHIDLFCAA|2052-09-10|1832|7968|612|2052|2|9|10|3|2052|612|7968|Tuesday|2052Q3|N|N|N|2470782|2471025|2470425|2470700|N|N|N|N|N| +2470792|AAAAAAAAIIDLFCAA|2052-09-11|1832|7968|612|2052|3|9|11|3|2052|612|7968|Wednesday|2052Q3|N|N|N|2470782|2471025|2470426|2470701|N|N|N|N|N| +2470793|AAAAAAAAJIDLFCAA|2052-09-12|1832|7968|612|2052|4|9|12|3|2052|612|7968|Thursday|2052Q3|N|N|N|2470782|2471025|2470427|2470702|N|N|N|N|N| +2470794|AAAAAAAAKIDLFCAA|2052-09-13|1832|7968|612|2052|5|9|13|3|2052|612|7968|Friday|2052Q3|N|Y|N|2470782|2471025|2470428|2470703|N|N|N|N|N| +2470795|AAAAAAAALIDLFCAA|2052-09-14|1832|7968|612|2052|6|9|14|3|2052|612|7968|Saturday|2052Q3|N|Y|N|2470782|2471025|2470429|2470704|N|N|N|N|N| +2470796|AAAAAAAAMIDLFCAA|2052-09-15|1832|7968|612|2052|0|9|15|3|2052|612|7968|Sunday|2052Q3|N|N|N|2470782|2471025|2470430|2470705|N|N|N|N|N| +2470797|AAAAAAAANIDLFCAA|2052-09-16|1832|7968|612|2052|1|9|16|3|2052|612|7968|Monday|2052Q3|N|N|N|2470782|2471025|2470431|2470706|N|N|N|N|N| +2470798|AAAAAAAAOIDLFCAA|2052-09-17|1832|7969|612|2052|2|9|17|3|2052|612|7969|Tuesday|2052Q3|N|N|N|2470782|2471025|2470432|2470707|N|N|N|N|N| +2470799|AAAAAAAAPIDLFCAA|2052-09-18|1832|7969|612|2052|3|9|18|3|2052|612|7969|Wednesday|2052Q3|N|N|N|2470782|2471025|2470433|2470708|N|N|N|N|N| +2470800|AAAAAAAAAJDLFCAA|2052-09-19|1832|7969|612|2052|4|9|19|3|2052|612|7969|Thursday|2052Q3|N|N|N|2470782|2471025|2470434|2470709|N|N|N|N|N| +2470801|AAAAAAAABJDLFCAA|2052-09-20|1832|7969|612|2052|5|9|20|3|2052|612|7969|Friday|2052Q3|N|Y|N|2470782|2471025|2470435|2470710|N|N|N|N|N| +2470802|AAAAAAAACJDLFCAA|2052-09-21|1832|7969|612|2052|6|9|21|3|2052|612|7969|Saturday|2052Q3|N|Y|N|2470782|2471025|2470436|2470711|N|N|N|N|N| +2470803|AAAAAAAADJDLFCAA|2052-09-22|1832|7969|612|2052|0|9|22|3|2052|612|7969|Sunday|2052Q3|N|N|N|2470782|2471025|2470437|2470712|N|N|N|N|N| +2470804|AAAAAAAAEJDLFCAA|2052-09-23|1832|7969|612|2052|1|9|23|3|2052|612|7969|Monday|2052Q3|N|N|N|2470782|2471025|2470438|2470713|N|N|N|N|N| +2470805|AAAAAAAAFJDLFCAA|2052-09-24|1832|7970|612|2052|2|9|24|3|2052|612|7970|Tuesday|2052Q3|N|N|N|2470782|2471025|2470439|2470714|N|N|N|N|N| +2470806|AAAAAAAAGJDLFCAA|2052-09-25|1832|7970|612|2052|3|9|25|3|2052|612|7970|Wednesday|2052Q3|N|N|N|2470782|2471025|2470440|2470715|N|N|N|N|N| +2470807|AAAAAAAAHJDLFCAA|2052-09-26|1832|7970|612|2052|4|9|26|3|2052|612|7970|Thursday|2052Q3|N|N|N|2470782|2471025|2470441|2470716|N|N|N|N|N| +2470808|AAAAAAAAIJDLFCAA|2052-09-27|1832|7970|612|2052|5|9|27|3|2052|612|7970|Friday|2052Q3|N|Y|N|2470782|2471025|2470442|2470717|N|N|N|N|N| +2470809|AAAAAAAAJJDLFCAA|2052-09-28|1832|7970|612|2052|6|9|28|3|2052|612|7970|Saturday|2052Q3|N|Y|N|2470782|2471025|2470443|2470718|N|N|N|N|N| +2470810|AAAAAAAAKJDLFCAA|2052-09-29|1832|7970|612|2052|0|9|29|3|2052|612|7970|Sunday|2052Q3|N|N|N|2470782|2471025|2470444|2470719|N|N|N|N|N| +2470811|AAAAAAAALJDLFCAA|2052-09-30|1832|7970|612|2052|1|9|30|3|2052|612|7970|Monday|2052Q3|N|N|N|2470782|2471025|2470445|2470720|N|N|N|N|N| +2470812|AAAAAAAAMJDLFCAA|2052-10-01|1833|7971|612|2052|2|10|1|4|2052|612|7971|Tuesday|2052Q4|N|N|N|2470812|2471085|2470446|2470720|N|N|N|N|N| +2470813|AAAAAAAANJDLFCAA|2052-10-02|1833|7971|612|2052|3|10|2|4|2052|612|7971|Wednesday|2052Q4|N|N|N|2470812|2471085|2470447|2470721|N|N|N|N|N| +2470814|AAAAAAAAOJDLFCAA|2052-10-03|1833|7971|612|2052|4|10|3|4|2052|612|7971|Thursday|2052Q4|N|N|N|2470812|2471085|2470448|2470722|N|N|N|N|N| +2470815|AAAAAAAAPJDLFCAA|2052-10-04|1833|7971|612|2052|5|10|4|4|2052|612|7971|Friday|2052Q4|N|Y|N|2470812|2471085|2470449|2470723|N|N|N|N|N| +2470816|AAAAAAAAAKDLFCAA|2052-10-05|1833|7971|612|2052|6|10|5|4|2052|612|7971|Saturday|2052Q4|N|Y|N|2470812|2471085|2470450|2470724|N|N|N|N|N| +2470817|AAAAAAAABKDLFCAA|2052-10-06|1833|7971|612|2052|0|10|6|4|2052|612|7971|Sunday|2052Q4|N|N|N|2470812|2471085|2470451|2470725|N|N|N|N|N| +2470818|AAAAAAAACKDLFCAA|2052-10-07|1833|7971|612|2052|1|10|7|4|2052|612|7971|Monday|2052Q4|N|N|N|2470812|2471085|2470452|2470726|N|N|N|N|N| +2470819|AAAAAAAADKDLFCAA|2052-10-08|1833|7972|612|2052|2|10|8|4|2052|612|7972|Tuesday|2052Q4|N|N|N|2470812|2471085|2470453|2470727|N|N|N|N|N| +2470820|AAAAAAAAEKDLFCAA|2052-10-09|1833|7972|612|2052|3|10|9|4|2052|612|7972|Wednesday|2052Q4|N|N|N|2470812|2471085|2470454|2470728|N|N|N|N|N| +2470821|AAAAAAAAFKDLFCAA|2052-10-10|1833|7972|612|2052|4|10|10|4|2052|612|7972|Thursday|2052Q4|N|N|N|2470812|2471085|2470455|2470729|N|N|N|N|N| +2470822|AAAAAAAAGKDLFCAA|2052-10-11|1833|7972|612|2052|5|10|11|4|2052|612|7972|Friday|2052Q4|N|Y|N|2470812|2471085|2470456|2470730|N|N|N|N|N| +2470823|AAAAAAAAHKDLFCAA|2052-10-12|1833|7972|612|2052|6|10|12|4|2052|612|7972|Saturday|2052Q4|N|Y|N|2470812|2471085|2470457|2470731|N|N|N|N|N| +2470824|AAAAAAAAIKDLFCAA|2052-10-13|1833|7972|612|2052|0|10|13|4|2052|612|7972|Sunday|2052Q4|N|N|N|2470812|2471085|2470458|2470732|N|N|N|N|N| +2470825|AAAAAAAAJKDLFCAA|2052-10-14|1833|7972|612|2052|1|10|14|4|2052|612|7972|Monday|2052Q4|N|N|N|2470812|2471085|2470459|2470733|N|N|N|N|N| +2470826|AAAAAAAAKKDLFCAA|2052-10-15|1833|7973|612|2052|2|10|15|4|2052|612|7973|Tuesday|2052Q4|N|N|N|2470812|2471085|2470460|2470734|N|N|N|N|N| +2470827|AAAAAAAALKDLFCAA|2052-10-16|1833|7973|612|2052|3|10|16|4|2052|612|7973|Wednesday|2052Q4|N|N|N|2470812|2471085|2470461|2470735|N|N|N|N|N| +2470828|AAAAAAAAMKDLFCAA|2052-10-17|1833|7973|612|2052|4|10|17|4|2052|612|7973|Thursday|2052Q4|N|N|N|2470812|2471085|2470462|2470736|N|N|N|N|N| +2470829|AAAAAAAANKDLFCAA|2052-10-18|1833|7973|612|2052|5|10|18|4|2052|612|7973|Friday|2052Q4|N|Y|N|2470812|2471085|2470463|2470737|N|N|N|N|N| +2470830|AAAAAAAAOKDLFCAA|2052-10-19|1833|7973|612|2052|6|10|19|4|2052|612|7973|Saturday|2052Q4|N|Y|N|2470812|2471085|2470464|2470738|N|N|N|N|N| +2470831|AAAAAAAAPKDLFCAA|2052-10-20|1833|7973|612|2052|0|10|20|4|2052|612|7973|Sunday|2052Q4|N|N|N|2470812|2471085|2470465|2470739|N|N|N|N|N| +2470832|AAAAAAAAALDLFCAA|2052-10-21|1833|7973|612|2052|1|10|21|4|2052|612|7973|Monday|2052Q4|N|N|N|2470812|2471085|2470466|2470740|N|N|N|N|N| +2470833|AAAAAAAABLDLFCAA|2052-10-22|1833|7974|612|2052|2|10|22|4|2052|612|7974|Tuesday|2052Q4|N|N|N|2470812|2471085|2470467|2470741|N|N|N|N|N| +2470834|AAAAAAAACLDLFCAA|2052-10-23|1833|7974|612|2052|3|10|23|4|2052|612|7974|Wednesday|2052Q4|N|N|N|2470812|2471085|2470468|2470742|N|N|N|N|N| +2470835|AAAAAAAADLDLFCAA|2052-10-24|1833|7974|612|2052|4|10|24|4|2052|612|7974|Thursday|2052Q4|N|N|N|2470812|2471085|2470469|2470743|N|N|N|N|N| +2470836|AAAAAAAAELDLFCAA|2052-10-25|1833|7974|612|2052|5|10|25|4|2052|612|7974|Friday|2052Q4|N|Y|N|2470812|2471085|2470470|2470744|N|N|N|N|N| +2470837|AAAAAAAAFLDLFCAA|2052-10-26|1833|7974|612|2052|6|10|26|4|2052|612|7974|Saturday|2052Q4|N|Y|N|2470812|2471085|2470471|2470745|N|N|N|N|N| +2470838|AAAAAAAAGLDLFCAA|2052-10-27|1833|7974|612|2052|0|10|27|4|2052|612|7974|Sunday|2052Q4|N|N|N|2470812|2471085|2470472|2470746|N|N|N|N|N| +2470839|AAAAAAAAHLDLFCAA|2052-10-28|1833|7974|612|2052|1|10|28|4|2052|612|7974|Monday|2052Q4|N|N|N|2470812|2471085|2470473|2470747|N|N|N|N|N| +2470840|AAAAAAAAILDLFCAA|2052-10-29|1833|7975|612|2052|2|10|29|4|2052|612|7975|Tuesday|2052Q4|N|N|N|2470812|2471085|2470474|2470748|N|N|N|N|N| +2470841|AAAAAAAAJLDLFCAA|2052-10-30|1833|7975|612|2052|3|10|30|4|2052|612|7975|Wednesday|2052Q4|N|N|N|2470812|2471085|2470475|2470749|N|N|N|N|N| +2470842|AAAAAAAAKLDLFCAA|2052-10-31|1833|7975|612|2052|4|10|31|4|2052|612|7975|Thursday|2052Q4|N|N|N|2470812|2471085|2470476|2470750|N|N|N|N|N| +2470843|AAAAAAAALLDLFCAA|2052-11-01|1834|7975|612|2052|5|11|1|4|2052|612|7975|Friday|2052Q4|N|Y|N|2470843|2471147|2470477|2470751|N|N|N|N|N| +2470844|AAAAAAAAMLDLFCAA|2052-11-02|1834|7975|612|2052|6|11|2|4|2052|612|7975|Saturday|2052Q4|N|Y|N|2470843|2471147|2470478|2470752|N|N|N|N|N| +2470845|AAAAAAAANLDLFCAA|2052-11-03|1834|7975|612|2052|0|11|3|4|2052|612|7975|Sunday|2052Q4|N|N|N|2470843|2471147|2470479|2470753|N|N|N|N|N| +2470846|AAAAAAAAOLDLFCAA|2052-11-04|1834|7975|612|2052|1|11|4|4|2052|612|7975|Monday|2052Q4|N|N|N|2470843|2471147|2470480|2470754|N|N|N|N|N| +2470847|AAAAAAAAPLDLFCAA|2052-11-05|1834|7976|612|2052|2|11|5|4|2052|612|7976|Tuesday|2052Q4|N|N|N|2470843|2471147|2470481|2470755|N|N|N|N|N| +2470848|AAAAAAAAAMDLFCAA|2052-11-06|1834|7976|612|2052|3|11|6|4|2052|612|7976|Wednesday|2052Q4|N|N|N|2470843|2471147|2470482|2470756|N|N|N|N|N| +2470849|AAAAAAAABMDLFCAA|2052-11-07|1834|7976|612|2052|4|11|7|4|2052|612|7976|Thursday|2052Q4|N|N|N|2470843|2471147|2470483|2470757|N|N|N|N|N| +2470850|AAAAAAAACMDLFCAA|2052-11-08|1834|7976|612|2052|5|11|8|4|2052|612|7976|Friday|2052Q4|N|Y|N|2470843|2471147|2470484|2470758|N|N|N|N|N| +2470851|AAAAAAAADMDLFCAA|2052-11-09|1834|7976|612|2052|6|11|9|4|2052|612|7976|Saturday|2052Q4|N|Y|N|2470843|2471147|2470485|2470759|N|N|N|N|N| +2470852|AAAAAAAAEMDLFCAA|2052-11-10|1834|7976|612|2052|0|11|10|4|2052|612|7976|Sunday|2052Q4|N|N|N|2470843|2471147|2470486|2470760|N|N|N|N|N| +2470853|AAAAAAAAFMDLFCAA|2052-11-11|1834|7976|612|2052|1|11|11|4|2052|612|7976|Monday|2052Q4|N|N|N|2470843|2471147|2470487|2470761|N|N|N|N|N| +2470854|AAAAAAAAGMDLFCAA|2052-11-12|1834|7977|612|2052|2|11|12|4|2052|612|7977|Tuesday|2052Q4|N|N|N|2470843|2471147|2470488|2470762|N|N|N|N|N| +2470855|AAAAAAAAHMDLFCAA|2052-11-13|1834|7977|612|2052|3|11|13|4|2052|612|7977|Wednesday|2052Q4|N|N|N|2470843|2471147|2470489|2470763|N|N|N|N|N| +2470856|AAAAAAAAIMDLFCAA|2052-11-14|1834|7977|612|2052|4|11|14|4|2052|612|7977|Thursday|2052Q4|N|N|N|2470843|2471147|2470490|2470764|N|N|N|N|N| +2470857|AAAAAAAAJMDLFCAA|2052-11-15|1834|7977|612|2052|5|11|15|4|2052|612|7977|Friday|2052Q4|N|Y|N|2470843|2471147|2470491|2470765|N|N|N|N|N| +2470858|AAAAAAAAKMDLFCAA|2052-11-16|1834|7977|612|2052|6|11|16|4|2052|612|7977|Saturday|2052Q4|N|Y|N|2470843|2471147|2470492|2470766|N|N|N|N|N| +2470859|AAAAAAAALMDLFCAA|2052-11-17|1834|7977|612|2052|0|11|17|4|2052|612|7977|Sunday|2052Q4|N|N|N|2470843|2471147|2470493|2470767|N|N|N|N|N| +2470860|AAAAAAAAMMDLFCAA|2052-11-18|1834|7977|612|2052|1|11|18|4|2052|612|7977|Monday|2052Q4|N|N|N|2470843|2471147|2470494|2470768|N|N|N|N|N| +2470861|AAAAAAAANMDLFCAA|2052-11-19|1834|7978|612|2052|2|11|19|4|2052|612|7978|Tuesday|2052Q4|N|N|N|2470843|2471147|2470495|2470769|N|N|N|N|N| +2470862|AAAAAAAAOMDLFCAA|2052-11-20|1834|7978|612|2052|3|11|20|4|2052|612|7978|Wednesday|2052Q4|N|N|N|2470843|2471147|2470496|2470770|N|N|N|N|N| +2470863|AAAAAAAAPMDLFCAA|2052-11-21|1834|7978|612|2052|4|11|21|4|2052|612|7978|Thursday|2052Q4|N|N|N|2470843|2471147|2470497|2470771|N|N|N|N|N| +2470864|AAAAAAAAANDLFCAA|2052-11-22|1834|7978|612|2052|5|11|22|4|2052|612|7978|Friday|2052Q4|N|Y|N|2470843|2471147|2470498|2470772|N|N|N|N|N| +2470865|AAAAAAAABNDLFCAA|2052-11-23|1834|7978|612|2052|6|11|23|4|2052|612|7978|Saturday|2052Q4|N|Y|N|2470843|2471147|2470499|2470773|N|N|N|N|N| +2470866|AAAAAAAACNDLFCAA|2052-11-24|1834|7978|612|2052|0|11|24|4|2052|612|7978|Sunday|2052Q4|N|N|N|2470843|2471147|2470500|2470774|N|N|N|N|N| +2470867|AAAAAAAADNDLFCAA|2052-11-25|1834|7978|612|2052|1|11|25|4|2052|612|7978|Monday|2052Q4|N|N|N|2470843|2471147|2470501|2470775|N|N|N|N|N| +2470868|AAAAAAAAENDLFCAA|2052-11-26|1834|7979|612|2052|2|11|26|4|2052|612|7979|Tuesday|2052Q4|N|N|N|2470843|2471147|2470502|2470776|N|N|N|N|N| +2470869|AAAAAAAAFNDLFCAA|2052-11-27|1834|7979|612|2052|3|11|27|4|2052|612|7979|Wednesday|2052Q4|N|N|N|2470843|2471147|2470503|2470777|N|N|N|N|N| +2470870|AAAAAAAAGNDLFCAA|2052-11-28|1834|7979|612|2052|4|11|28|4|2052|612|7979|Thursday|2052Q4|N|N|N|2470843|2471147|2470504|2470778|N|N|N|N|N| +2470871|AAAAAAAAHNDLFCAA|2052-11-29|1834|7979|612|2052|5|11|29|4|2052|612|7979|Friday|2052Q4|N|Y|N|2470843|2471147|2470505|2470779|N|N|N|N|N| +2470872|AAAAAAAAINDLFCAA|2052-11-30|1834|7979|612|2052|6|11|30|4|2052|612|7979|Saturday|2052Q4|N|Y|N|2470843|2471147|2470506|2470780|N|N|N|N|N| +2470873|AAAAAAAAJNDLFCAA|2052-12-01|1835|7979|613|2052|0|12|1|4|2052|613|7979|Sunday|2052Q4|N|N|N|2470873|2471207|2470507|2470781|N|N|N|N|N| +2470874|AAAAAAAAKNDLFCAA|2052-12-02|1835|7979|613|2052|1|12|2|4|2052|613|7979|Monday|2052Q4|N|N|N|2470873|2471207|2470508|2470782|N|N|N|N|N| +2470875|AAAAAAAALNDLFCAA|2052-12-03|1835|7980|613|2052|2|12|3|4|2052|613|7980|Tuesday|2052Q4|N|N|N|2470873|2471207|2470509|2470783|N|N|N|N|N| +2470876|AAAAAAAAMNDLFCAA|2052-12-04|1835|7980|613|2052|3|12|4|4|2052|613|7980|Wednesday|2052Q4|N|N|N|2470873|2471207|2470510|2470784|N|N|N|N|N| +2470877|AAAAAAAANNDLFCAA|2052-12-05|1835|7980|613|2052|4|12|5|4|2052|613|7980|Thursday|2052Q4|N|N|N|2470873|2471207|2470511|2470785|N|N|N|N|N| +2470878|AAAAAAAAONDLFCAA|2052-12-06|1835|7980|613|2052|5|12|6|4|2052|613|7980|Friday|2052Q4|N|Y|N|2470873|2471207|2470512|2470786|N|N|N|N|N| +2470879|AAAAAAAAPNDLFCAA|2052-12-07|1835|7980|613|2052|6|12|7|4|2052|613|7980|Saturday|2052Q4|N|Y|N|2470873|2471207|2470513|2470787|N|N|N|N|N| +2470880|AAAAAAAAAODLFCAA|2052-12-08|1835|7980|613|2052|0|12|8|4|2052|613|7980|Sunday|2052Q4|N|N|N|2470873|2471207|2470514|2470788|N|N|N|N|N| +2470881|AAAAAAAABODLFCAA|2052-12-09|1835|7980|613|2052|1|12|9|4|2052|613|7980|Monday|2052Q4|N|N|N|2470873|2471207|2470515|2470789|N|N|N|N|N| +2470882|AAAAAAAACODLFCAA|2052-12-10|1835|7981|613|2052|2|12|10|4|2052|613|7981|Tuesday|2052Q4|N|N|N|2470873|2471207|2470516|2470790|N|N|N|N|N| +2470883|AAAAAAAADODLFCAA|2052-12-11|1835|7981|613|2052|3|12|11|4|2052|613|7981|Wednesday|2052Q4|N|N|N|2470873|2471207|2470517|2470791|N|N|N|N|N| +2470884|AAAAAAAAEODLFCAA|2052-12-12|1835|7981|613|2052|4|12|12|4|2052|613|7981|Thursday|2052Q4|N|N|N|2470873|2471207|2470518|2470792|N|N|N|N|N| +2470885|AAAAAAAAFODLFCAA|2052-12-13|1835|7981|613|2052|5|12|13|4|2052|613|7981|Friday|2052Q4|N|Y|N|2470873|2471207|2470519|2470793|N|N|N|N|N| +2470886|AAAAAAAAGODLFCAA|2052-12-14|1835|7981|613|2052|6|12|14|4|2052|613|7981|Saturday|2052Q4|N|Y|N|2470873|2471207|2470520|2470794|N|N|N|N|N| +2470887|AAAAAAAAHODLFCAA|2052-12-15|1835|7981|613|2052|0|12|15|4|2052|613|7981|Sunday|2052Q4|N|N|N|2470873|2471207|2470521|2470795|N|N|N|N|N| +2470888|AAAAAAAAIODLFCAA|2052-12-16|1835|7981|613|2052|1|12|16|4|2052|613|7981|Monday|2052Q4|N|N|N|2470873|2471207|2470522|2470796|N|N|N|N|N| +2470889|AAAAAAAAJODLFCAA|2052-12-17|1835|7982|613|2052|2|12|17|4|2052|613|7982|Tuesday|2052Q4|N|N|N|2470873|2471207|2470523|2470797|N|N|N|N|N| +2470890|AAAAAAAAKODLFCAA|2052-12-18|1835|7982|613|2052|3|12|18|4|2052|613|7982|Wednesday|2052Q4|N|N|N|2470873|2471207|2470524|2470798|N|N|N|N|N| +2470891|AAAAAAAALODLFCAA|2052-12-19|1835|7982|613|2052|4|12|19|4|2052|613|7982|Thursday|2052Q4|N|N|N|2470873|2471207|2470525|2470799|N|N|N|N|N| +2470892|AAAAAAAAMODLFCAA|2052-12-20|1835|7982|613|2052|5|12|20|4|2052|613|7982|Friday|2052Q4|N|Y|N|2470873|2471207|2470526|2470800|N|N|N|N|N| +2470893|AAAAAAAANODLFCAA|2052-12-21|1835|7982|613|2052|6|12|21|4|2052|613|7982|Saturday|2052Q4|N|Y|N|2470873|2471207|2470527|2470801|N|N|N|N|N| +2470894|AAAAAAAAOODLFCAA|2052-12-22|1835|7982|613|2052|0|12|22|4|2052|613|7982|Sunday|2052Q4|N|N|N|2470873|2471207|2470528|2470802|N|N|N|N|N| +2470895|AAAAAAAAPODLFCAA|2052-12-23|1835|7982|613|2052|1|12|23|4|2052|613|7982|Monday|2052Q4|N|N|N|2470873|2471207|2470529|2470803|N|N|N|N|N| +2470896|AAAAAAAAAPDLFCAA|2052-12-24|1835|7983|613|2052|2|12|24|4|2052|613|7983|Tuesday|2052Q4|N|N|N|2470873|2471207|2470530|2470804|N|N|N|N|N| +2470897|AAAAAAAABPDLFCAA|2052-12-25|1835|7983|613|2052|3|12|25|4|2052|613|7983|Wednesday|2052Q4|Y|N|N|2470873|2471207|2470531|2470805|N|N|N|N|N| +2470898|AAAAAAAACPDLFCAA|2052-12-26|1835|7983|613|2052|4|12|26|4|2052|613|7983|Thursday|2052Q4|N|N|Y|2470873|2471207|2470532|2470806|N|N|N|N|N| +2470899|AAAAAAAADPDLFCAA|2052-12-27|1835|7983|613|2052|5|12|27|4|2052|613|7983|Friday|2052Q4|N|Y|N|2470873|2471207|2470533|2470807|N|N|N|N|N| +2470900|AAAAAAAAEPDLFCAA|2052-12-28|1835|7983|613|2052|6|12|28|4|2052|613|7983|Saturday|2052Q4|N|Y|N|2470873|2471207|2470534|2470808|N|N|N|N|N| +2470901|AAAAAAAAFPDLFCAA|2052-12-29|1835|7983|613|2052|0|12|29|4|2052|613|7983|Sunday|2052Q4|N|N|N|2470873|2471207|2470535|2470809|N|N|N|N|N| +2470902|AAAAAAAAGPDLFCAA|2052-12-30|1835|7983|613|2052|1|12|30|4|2052|613|7983|Monday|2052Q4|N|N|N|2470873|2471207|2470536|2470810|N|N|N|N|N| +2470903|AAAAAAAAHPDLFCAA|2052-12-31|1835|7984|613|2052|2|12|31|4|2052|613|7984|Tuesday|2052Q4|Y|N|N|2470873|2471207|2470537|2470811|N|N|N|N|N| +2470904|AAAAAAAAIPDLFCAA|2053-01-01|1836|7984|613|2053|3|1|1|1|2053|613|7984|Wednesday|2053Q1|Y|N|Y|2470904|2470903|2470538|2470812|N|N|N|N|N| +2470905|AAAAAAAAJPDLFCAA|2053-01-02|1836|7984|613|2053|4|1|2|1|2053|613|7984|Thursday|2053Q1|N|N|Y|2470904|2470903|2470539|2470813|N|N|N|N|N| +2470906|AAAAAAAAKPDLFCAA|2053-01-03|1836|7984|613|2053|5|1|3|1|2053|613|7984|Friday|2053Q1|N|Y|N|2470904|2470903|2470540|2470814|N|N|N|N|N| +2470907|AAAAAAAALPDLFCAA|2053-01-04|1836|7984|613|2053|6|1|4|1|2053|613|7984|Saturday|2053Q1|N|Y|N|2470904|2470903|2470541|2470815|N|N|N|N|N| +2470908|AAAAAAAAMPDLFCAA|2053-01-05|1836|7984|613|2053|0|1|5|1|2053|613|7984|Sunday|2053Q1|N|N|N|2470904|2470903|2470542|2470816|N|N|N|N|N| +2470909|AAAAAAAANPDLFCAA|2053-01-06|1836|7984|613|2053|1|1|6|1|2053|613|7984|Monday|2053Q1|N|N|N|2470904|2470903|2470543|2470817|N|N|N|N|N| +2470910|AAAAAAAAOPDLFCAA|2053-01-07|1836|7985|613|2053|2|1|7|1|2053|613|7985|Tuesday|2053Q1|N|N|N|2470904|2470903|2470544|2470818|N|N|N|N|N| +2470911|AAAAAAAAPPDLFCAA|2053-01-08|1836|7985|613|2053|3|1|8|1|2053|613|7985|Wednesday|2053Q1|N|N|N|2470904|2470903|2470545|2470819|N|N|N|N|N| +2470912|AAAAAAAAAAELFCAA|2053-01-09|1836|7985|613|2053|4|1|9|1|2053|613|7985|Thursday|2053Q1|N|N|N|2470904|2470903|2470546|2470820|N|N|N|N|N| +2470913|AAAAAAAABAELFCAA|2053-01-10|1836|7985|613|2053|5|1|10|1|2053|613|7985|Friday|2053Q1|N|Y|N|2470904|2470903|2470547|2470821|N|N|N|N|N| +2470914|AAAAAAAACAELFCAA|2053-01-11|1836|7985|613|2053|6|1|11|1|2053|613|7985|Saturday|2053Q1|N|Y|N|2470904|2470903|2470548|2470822|N|N|N|N|N| +2470915|AAAAAAAADAELFCAA|2053-01-12|1836|7985|613|2053|0|1|12|1|2053|613|7985|Sunday|2053Q1|N|N|N|2470904|2470903|2470549|2470823|N|N|N|N|N| +2470916|AAAAAAAAEAELFCAA|2053-01-13|1836|7985|613|2053|1|1|13|1|2053|613|7985|Monday|2053Q1|N|N|N|2470904|2470903|2470550|2470824|N|N|N|N|N| +2470917|AAAAAAAAFAELFCAA|2053-01-14|1836|7986|613|2053|2|1|14|1|2053|613|7986|Tuesday|2053Q1|N|N|N|2470904|2470903|2470551|2470825|N|N|N|N|N| +2470918|AAAAAAAAGAELFCAA|2053-01-15|1836|7986|613|2053|3|1|15|1|2053|613|7986|Wednesday|2053Q1|N|N|N|2470904|2470903|2470552|2470826|N|N|N|N|N| +2470919|AAAAAAAAHAELFCAA|2053-01-16|1836|7986|613|2053|4|1|16|1|2053|613|7986|Thursday|2053Q1|N|N|N|2470904|2470903|2470553|2470827|N|N|N|N|N| +2470920|AAAAAAAAIAELFCAA|2053-01-17|1836|7986|613|2053|5|1|17|1|2053|613|7986|Friday|2053Q1|N|Y|N|2470904|2470903|2470554|2470828|N|N|N|N|N| +2470921|AAAAAAAAJAELFCAA|2053-01-18|1836|7986|613|2053|6|1|18|1|2053|613|7986|Saturday|2053Q1|N|Y|N|2470904|2470903|2470555|2470829|N|N|N|N|N| +2470922|AAAAAAAAKAELFCAA|2053-01-19|1836|7986|613|2053|0|1|19|1|2053|613|7986|Sunday|2053Q1|N|N|N|2470904|2470903|2470556|2470830|N|N|N|N|N| +2470923|AAAAAAAALAELFCAA|2053-01-20|1836|7986|613|2053|1|1|20|1|2053|613|7986|Monday|2053Q1|N|N|N|2470904|2470903|2470557|2470831|N|N|N|N|N| +2470924|AAAAAAAAMAELFCAA|2053-01-21|1836|7987|613|2053|2|1|21|1|2053|613|7987|Tuesday|2053Q1|N|N|N|2470904|2470903|2470558|2470832|N|N|N|N|N| +2470925|AAAAAAAANAELFCAA|2053-01-22|1836|7987|613|2053|3|1|22|1|2053|613|7987|Wednesday|2053Q1|N|N|N|2470904|2470903|2470559|2470833|N|N|N|N|N| +2470926|AAAAAAAAOAELFCAA|2053-01-23|1836|7987|613|2053|4|1|23|1|2053|613|7987|Thursday|2053Q1|N|N|N|2470904|2470903|2470560|2470834|N|N|N|N|N| +2470927|AAAAAAAAPAELFCAA|2053-01-24|1836|7987|613|2053|5|1|24|1|2053|613|7987|Friday|2053Q1|N|Y|N|2470904|2470903|2470561|2470835|N|N|N|N|N| +2470928|AAAAAAAAABELFCAA|2053-01-25|1836|7987|613|2053|6|1|25|1|2053|613|7987|Saturday|2053Q1|N|Y|N|2470904|2470903|2470562|2470836|N|N|N|N|N| +2470929|AAAAAAAABBELFCAA|2053-01-26|1836|7987|613|2053|0|1|26|1|2053|613|7987|Sunday|2053Q1|N|N|N|2470904|2470903|2470563|2470837|N|N|N|N|N| +2470930|AAAAAAAACBELFCAA|2053-01-27|1836|7987|613|2053|1|1|27|1|2053|613|7987|Monday|2053Q1|N|N|N|2470904|2470903|2470564|2470838|N|N|N|N|N| +2470931|AAAAAAAADBELFCAA|2053-01-28|1836|7988|613|2053|2|1|28|1|2053|613|7988|Tuesday|2053Q1|N|N|N|2470904|2470903|2470565|2470839|N|N|N|N|N| +2470932|AAAAAAAAEBELFCAA|2053-01-29|1836|7988|613|2053|3|1|29|1|2053|613|7988|Wednesday|2053Q1|N|N|N|2470904|2470903|2470566|2470840|N|N|N|N|N| +2470933|AAAAAAAAFBELFCAA|2053-01-30|1836|7988|613|2053|4|1|30|1|2053|613|7988|Thursday|2053Q1|N|N|N|2470904|2470903|2470567|2470841|N|N|N|N|N| +2470934|AAAAAAAAGBELFCAA|2053-01-31|1836|7988|613|2053|5|1|31|1|2053|613|7988|Friday|2053Q1|N|Y|N|2470904|2470903|2470568|2470842|N|N|N|N|N| +2470935|AAAAAAAAHBELFCAA|2053-02-01|1837|7988|613|2053|6|2|1|1|2053|613|7988|Saturday|2053Q1|N|Y|N|2470935|2470965|2470569|2470843|N|N|N|N|N| +2470936|AAAAAAAAIBELFCAA|2053-02-02|1837|7988|613|2053|0|2|2|1|2053|613|7988|Sunday|2053Q1|N|N|N|2470935|2470965|2470570|2470844|N|N|N|N|N| +2470937|AAAAAAAAJBELFCAA|2053-02-03|1837|7988|613|2053|1|2|3|1|2053|613|7988|Monday|2053Q1|N|N|N|2470935|2470965|2470571|2470845|N|N|N|N|N| +2470938|AAAAAAAAKBELFCAA|2053-02-04|1837|7989|613|2053|2|2|4|1|2053|613|7989|Tuesday|2053Q1|N|N|N|2470935|2470965|2470572|2470846|N|N|N|N|N| +2470939|AAAAAAAALBELFCAA|2053-02-05|1837|7989|613|2053|3|2|5|1|2053|613|7989|Wednesday|2053Q1|N|N|N|2470935|2470965|2470573|2470847|N|N|N|N|N| +2470940|AAAAAAAAMBELFCAA|2053-02-06|1837|7989|613|2053|4|2|6|1|2053|613|7989|Thursday|2053Q1|N|N|N|2470935|2470965|2470574|2470848|N|N|N|N|N| +2470941|AAAAAAAANBELFCAA|2053-02-07|1837|7989|613|2053|5|2|7|1|2053|613|7989|Friday|2053Q1|N|Y|N|2470935|2470965|2470575|2470849|N|N|N|N|N| +2470942|AAAAAAAAOBELFCAA|2053-02-08|1837|7989|613|2053|6|2|8|1|2053|613|7989|Saturday|2053Q1|N|Y|N|2470935|2470965|2470576|2470850|N|N|N|N|N| +2470943|AAAAAAAAPBELFCAA|2053-02-09|1837|7989|613|2053|0|2|9|1|2053|613|7989|Sunday|2053Q1|N|N|N|2470935|2470965|2470577|2470851|N|N|N|N|N| +2470944|AAAAAAAAACELFCAA|2053-02-10|1837|7989|613|2053|1|2|10|1|2053|613|7989|Monday|2053Q1|N|N|N|2470935|2470965|2470578|2470852|N|N|N|N|N| +2470945|AAAAAAAABCELFCAA|2053-02-11|1837|7990|613|2053|2|2|11|1|2053|613|7990|Tuesday|2053Q1|N|N|N|2470935|2470965|2470579|2470853|N|N|N|N|N| +2470946|AAAAAAAACCELFCAA|2053-02-12|1837|7990|613|2053|3|2|12|1|2053|613|7990|Wednesday|2053Q1|N|N|N|2470935|2470965|2470580|2470854|N|N|N|N|N| +2470947|AAAAAAAADCELFCAA|2053-02-13|1837|7990|613|2053|4|2|13|1|2053|613|7990|Thursday|2053Q1|N|N|N|2470935|2470965|2470581|2470855|N|N|N|N|N| +2470948|AAAAAAAAECELFCAA|2053-02-14|1837|7990|613|2053|5|2|14|1|2053|613|7990|Friday|2053Q1|N|Y|N|2470935|2470965|2470582|2470856|N|N|N|N|N| +2470949|AAAAAAAAFCELFCAA|2053-02-15|1837|7990|613|2053|6|2|15|1|2053|613|7990|Saturday|2053Q1|N|Y|N|2470935|2470965|2470583|2470857|N|N|N|N|N| +2470950|AAAAAAAAGCELFCAA|2053-02-16|1837|7990|613|2053|0|2|16|1|2053|613|7990|Sunday|2053Q1|N|N|N|2470935|2470965|2470584|2470858|N|N|N|N|N| +2470951|AAAAAAAAHCELFCAA|2053-02-17|1837|7990|613|2053|1|2|17|1|2053|613|7990|Monday|2053Q1|N|N|N|2470935|2470965|2470585|2470859|N|N|N|N|N| +2470952|AAAAAAAAICELFCAA|2053-02-18|1837|7991|613|2053|2|2|18|1|2053|613|7991|Tuesday|2053Q1|N|N|N|2470935|2470965|2470586|2470860|N|N|N|N|N| +2470953|AAAAAAAAJCELFCAA|2053-02-19|1837|7991|613|2053|3|2|19|1|2053|613|7991|Wednesday|2053Q1|N|N|N|2470935|2470965|2470587|2470861|N|N|N|N|N| +2470954|AAAAAAAAKCELFCAA|2053-02-20|1837|7991|613|2053|4|2|20|1|2053|613|7991|Thursday|2053Q1|N|N|N|2470935|2470965|2470588|2470862|N|N|N|N|N| +2470955|AAAAAAAALCELFCAA|2053-02-21|1837|7991|613|2053|5|2|21|1|2053|613|7991|Friday|2053Q1|N|Y|N|2470935|2470965|2470589|2470863|N|N|N|N|N| +2470956|AAAAAAAAMCELFCAA|2053-02-22|1837|7991|613|2053|6|2|22|1|2053|613|7991|Saturday|2053Q1|N|Y|N|2470935|2470965|2470590|2470864|N|N|N|N|N| +2470957|AAAAAAAANCELFCAA|2053-02-23|1837|7991|613|2053|0|2|23|1|2053|613|7991|Sunday|2053Q1|N|N|N|2470935|2470965|2470591|2470865|N|N|N|N|N| +2470958|AAAAAAAAOCELFCAA|2053-02-24|1837|7991|613|2053|1|2|24|1|2053|613|7991|Monday|2053Q1|N|N|N|2470935|2470965|2470592|2470866|N|N|N|N|N| +2470959|AAAAAAAAPCELFCAA|2053-02-25|1837|7992|613|2053|2|2|25|1|2053|613|7992|Tuesday|2053Q1|N|N|N|2470935|2470965|2470593|2470867|N|N|N|N|N| +2470960|AAAAAAAAADELFCAA|2053-02-26|1837|7992|613|2053|3|2|26|1|2053|613|7992|Wednesday|2053Q1|N|N|N|2470935|2470965|2470594|2470868|N|N|N|N|N| +2470961|AAAAAAAABDELFCAA|2053-02-27|1837|7992|613|2053|4|2|27|1|2053|613|7992|Thursday|2053Q1|N|N|N|2470935|2470965|2470595|2470869|N|N|N|N|N| +2470962|AAAAAAAACDELFCAA|2053-02-28|1837|7992|613|2053|5|2|28|1|2053|613|7992|Friday|2053Q1|N|Y|N|2470935|2470965|2470596|2470870|N|N|N|N|N| +2470963|AAAAAAAADDELFCAA|2053-03-01|1838|7992|614|2053|6|3|1|1|2053|614|7992|Saturday|2053Q1|N|Y|N|2470963|2471021|2470598|2470871|N|N|N|N|N| +2470964|AAAAAAAAEDELFCAA|2053-03-02|1838|7992|614|2053|0|3|2|1|2053|614|7992|Sunday|2053Q1|N|N|N|2470963|2471021|2470599|2470872|N|N|N|N|N| +2470965|AAAAAAAAFDELFCAA|2053-03-03|1838|7992|614|2053|1|3|3|1|2053|614|7992|Monday|2053Q1|N|N|N|2470963|2471021|2470600|2470873|N|N|N|N|N| +2470966|AAAAAAAAGDELFCAA|2053-03-04|1838|7993|614|2053|2|3|4|1|2053|614|7993|Tuesday|2053Q1|N|N|N|2470963|2471021|2470601|2470874|N|N|N|N|N| +2470967|AAAAAAAAHDELFCAA|2053-03-05|1838|7993|614|2053|3|3|5|1|2053|614|7993|Wednesday|2053Q1|N|N|N|2470963|2471021|2470602|2470875|N|N|N|N|N| +2470968|AAAAAAAAIDELFCAA|2053-03-06|1838|7993|614|2053|4|3|6|1|2053|614|7993|Thursday|2053Q1|N|N|N|2470963|2471021|2470603|2470876|N|N|N|N|N| +2470969|AAAAAAAAJDELFCAA|2053-03-07|1838|7993|614|2053|5|3|7|1|2053|614|7993|Friday|2053Q1|N|Y|N|2470963|2471021|2470604|2470877|N|N|N|N|N| +2470970|AAAAAAAAKDELFCAA|2053-03-08|1838|7993|614|2053|6|3|8|1|2053|614|7993|Saturday|2053Q1|N|Y|N|2470963|2471021|2470605|2470878|N|N|N|N|N| +2470971|AAAAAAAALDELFCAA|2053-03-09|1838|7993|614|2053|0|3|9|1|2053|614|7993|Sunday|2053Q1|N|N|N|2470963|2471021|2470606|2470879|N|N|N|N|N| +2470972|AAAAAAAAMDELFCAA|2053-03-10|1838|7993|614|2053|1|3|10|1|2053|614|7993|Monday|2053Q1|N|N|N|2470963|2471021|2470607|2470880|N|N|N|N|N| +2470973|AAAAAAAANDELFCAA|2053-03-11|1838|7994|614|2053|2|3|11|1|2053|614|7994|Tuesday|2053Q1|N|N|N|2470963|2471021|2470608|2470881|N|N|N|N|N| +2470974|AAAAAAAAODELFCAA|2053-03-12|1838|7994|614|2053|3|3|12|1|2053|614|7994|Wednesday|2053Q1|N|N|N|2470963|2471021|2470609|2470882|N|N|N|N|N| +2470975|AAAAAAAAPDELFCAA|2053-03-13|1838|7994|614|2053|4|3|13|1|2053|614|7994|Thursday|2053Q1|N|N|N|2470963|2471021|2470610|2470883|N|N|N|N|N| +2470976|AAAAAAAAAEELFCAA|2053-03-14|1838|7994|614|2053|5|3|14|1|2053|614|7994|Friday|2053Q1|N|Y|N|2470963|2471021|2470611|2470884|N|N|N|N|N| +2470977|AAAAAAAABEELFCAA|2053-03-15|1838|7994|614|2053|6|3|15|1|2053|614|7994|Saturday|2053Q1|N|Y|N|2470963|2471021|2470612|2470885|N|N|N|N|N| +2470978|AAAAAAAACEELFCAA|2053-03-16|1838|7994|614|2053|0|3|16|1|2053|614|7994|Sunday|2053Q1|N|N|N|2470963|2471021|2470613|2470886|N|N|N|N|N| +2470979|AAAAAAAADEELFCAA|2053-03-17|1838|7994|614|2053|1|3|17|1|2053|614|7994|Monday|2053Q1|N|N|N|2470963|2471021|2470614|2470887|N|N|N|N|N| +2470980|AAAAAAAAEEELFCAA|2053-03-18|1838|7995|614|2053|2|3|18|1|2053|614|7995|Tuesday|2053Q1|N|N|N|2470963|2471021|2470615|2470888|N|N|N|N|N| +2470981|AAAAAAAAFEELFCAA|2053-03-19|1838|7995|614|2053|3|3|19|1|2053|614|7995|Wednesday|2053Q1|N|N|N|2470963|2471021|2470616|2470889|N|N|N|N|N| +2470982|AAAAAAAAGEELFCAA|2053-03-20|1838|7995|614|2053|4|3|20|1|2053|614|7995|Thursday|2053Q1|N|N|N|2470963|2471021|2470617|2470890|N|N|N|N|N| +2470983|AAAAAAAAHEELFCAA|2053-03-21|1838|7995|614|2053|5|3|21|1|2053|614|7995|Friday|2053Q1|N|Y|N|2470963|2471021|2470618|2470891|N|N|N|N|N| +2470984|AAAAAAAAIEELFCAA|2053-03-22|1838|7995|614|2053|6|3|22|1|2053|614|7995|Saturday|2053Q1|N|Y|N|2470963|2471021|2470619|2470892|N|N|N|N|N| +2470985|AAAAAAAAJEELFCAA|2053-03-23|1838|7995|614|2053|0|3|23|1|2053|614|7995|Sunday|2053Q1|N|N|N|2470963|2471021|2470620|2470893|N|N|N|N|N| +2470986|AAAAAAAAKEELFCAA|2053-03-24|1838|7995|614|2053|1|3|24|1|2053|614|7995|Monday|2053Q1|N|N|N|2470963|2471021|2470621|2470894|N|N|N|N|N| +2470987|AAAAAAAALEELFCAA|2053-03-25|1838|7996|614|2053|2|3|25|1|2053|614|7996|Tuesday|2053Q1|N|N|N|2470963|2471021|2470622|2470895|N|N|N|N|N| +2470988|AAAAAAAAMEELFCAA|2053-03-26|1838|7996|614|2053|3|3|26|1|2053|614|7996|Wednesday|2053Q1|N|N|N|2470963|2471021|2470623|2470896|N|N|N|N|N| +2470989|AAAAAAAANEELFCAA|2053-03-27|1838|7996|614|2053|4|3|27|1|2053|614|7996|Thursday|2053Q1|N|N|N|2470963|2471021|2470624|2470897|N|N|N|N|N| +2470990|AAAAAAAAOEELFCAA|2053-03-28|1838|7996|614|2053|5|3|28|1|2053|614|7996|Friday|2053Q1|N|Y|N|2470963|2471021|2470625|2470898|N|N|N|N|N| +2470991|AAAAAAAAPEELFCAA|2053-03-29|1838|7996|614|2053|6|3|29|1|2053|614|7996|Saturday|2053Q1|N|Y|N|2470963|2471021|2470626|2470899|N|N|N|N|N| +2470992|AAAAAAAAAFELFCAA|2053-03-30|1838|7996|614|2053|0|3|30|1|2053|614|7996|Sunday|2053Q1|N|N|N|2470963|2471021|2470627|2470900|N|N|N|N|N| +2470993|AAAAAAAABFELFCAA|2053-03-31|1838|7996|614|2053|1|3|31|1|2053|614|7996|Monday|2053Q1|N|N|N|2470963|2471021|2470628|2470901|N|N|N|N|N| +2470994|AAAAAAAACFELFCAA|2053-04-01|1839|7997|614|2053|2|4|1|1|2053|614|7997|Tuesday|2053Q1|N|N|N|2470994|2471083|2470629|2470904|N|N|N|N|N| +2470995|AAAAAAAADFELFCAA|2053-04-02|1839|7997|614|2053|3|4|2|2|2053|614|7997|Wednesday|2053Q2|N|N|N|2470994|2471083|2470630|2470905|N|N|N|N|N| +2470996|AAAAAAAAEFELFCAA|2053-04-03|1839|7997|614|2053|4|4|3|2|2053|614|7997|Thursday|2053Q2|N|N|N|2470994|2471083|2470631|2470906|N|N|N|N|N| +2470997|AAAAAAAAFFELFCAA|2053-04-04|1839|7997|614|2053|5|4|4|2|2053|614|7997|Friday|2053Q2|N|Y|N|2470994|2471083|2470632|2470907|N|N|N|N|N| +2470998|AAAAAAAAGFELFCAA|2053-04-05|1839|7997|614|2053|6|4|5|2|2053|614|7997|Saturday|2053Q2|N|Y|N|2470994|2471083|2470633|2470908|N|N|N|N|N| +2470999|AAAAAAAAHFELFCAA|2053-04-06|1839|7997|614|2053|0|4|6|2|2053|614|7997|Sunday|2053Q2|N|N|N|2470994|2471083|2470634|2470909|N|N|N|N|N| +2471000|AAAAAAAAIFELFCAA|2053-04-07|1839|7997|614|2053|1|4|7|2|2053|614|7997|Monday|2053Q2|N|N|N|2470994|2471083|2470635|2470910|N|N|N|N|N| +2471001|AAAAAAAAJFELFCAA|2053-04-08|1839|7998|614|2053|2|4|8|2|2053|614|7998|Tuesday|2053Q2|N|N|N|2470994|2471083|2470636|2470911|N|N|N|N|N| +2471002|AAAAAAAAKFELFCAA|2053-04-09|1839|7998|614|2053|3|4|9|2|2053|614|7998|Wednesday|2053Q2|N|N|N|2470994|2471083|2470637|2470912|N|N|N|N|N| +2471003|AAAAAAAALFELFCAA|2053-04-10|1839|7998|614|2053|4|4|10|2|2053|614|7998|Thursday|2053Q2|N|N|N|2470994|2471083|2470638|2470913|N|N|N|N|N| +2471004|AAAAAAAAMFELFCAA|2053-04-11|1839|7998|614|2053|5|4|11|2|2053|614|7998|Friday|2053Q2|N|Y|N|2470994|2471083|2470639|2470914|N|N|N|N|N| +2471005|AAAAAAAANFELFCAA|2053-04-12|1839|7998|614|2053|6|4|12|2|2053|614|7998|Saturday|2053Q2|N|Y|N|2470994|2471083|2470640|2470915|N|N|N|N|N| +2471006|AAAAAAAAOFELFCAA|2053-04-13|1839|7998|614|2053|0|4|13|2|2053|614|7998|Sunday|2053Q2|N|N|N|2470994|2471083|2470641|2470916|N|N|N|N|N| +2471007|AAAAAAAAPFELFCAA|2053-04-14|1839|7998|614|2053|1|4|14|2|2053|614|7998|Monday|2053Q2|N|N|N|2470994|2471083|2470642|2470917|N|N|N|N|N| +2471008|AAAAAAAAAGELFCAA|2053-04-15|1839|7999|614|2053|2|4|15|2|2053|614|7999|Tuesday|2053Q2|N|N|N|2470994|2471083|2470643|2470918|N|N|N|N|N| +2471009|AAAAAAAABGELFCAA|2053-04-16|1839|7999|614|2053|3|4|16|2|2053|614|7999|Wednesday|2053Q2|N|N|N|2470994|2471083|2470644|2470919|N|N|N|N|N| +2471010|AAAAAAAACGELFCAA|2053-04-17|1839|7999|614|2053|4|4|17|2|2053|614|7999|Thursday|2053Q2|N|N|N|2470994|2471083|2470645|2470920|N|N|N|N|N| +2471011|AAAAAAAADGELFCAA|2053-04-18|1839|7999|614|2053|5|4|18|2|2053|614|7999|Friday|2053Q2|N|Y|N|2470994|2471083|2470646|2470921|N|N|N|N|N| +2471012|AAAAAAAAEGELFCAA|2053-04-19|1839|7999|614|2053|6|4|19|2|2053|614|7999|Saturday|2053Q2|N|Y|N|2470994|2471083|2470647|2470922|N|N|N|N|N| +2471013|AAAAAAAAFGELFCAA|2053-04-20|1839|7999|614|2053|0|4|20|2|2053|614|7999|Sunday|2053Q2|N|N|N|2470994|2471083|2470648|2470923|N|N|N|N|N| +2471014|AAAAAAAAGGELFCAA|2053-04-21|1839|7999|614|2053|1|4|21|2|2053|614|7999|Monday|2053Q2|N|N|N|2470994|2471083|2470649|2470924|N|N|N|N|N| +2471015|AAAAAAAAHGELFCAA|2053-04-22|1839|8000|614|2053|2|4|22|2|2053|614|8000|Tuesday|2053Q2|N|N|N|2470994|2471083|2470650|2470925|N|N|N|N|N| +2471016|AAAAAAAAIGELFCAA|2053-04-23|1839|8000|614|2053|3|4|23|2|2053|614|8000|Wednesday|2053Q2|N|N|N|2470994|2471083|2470651|2470926|N|N|N|N|N| +2471017|AAAAAAAAJGELFCAA|2053-04-24|1839|8000|614|2053|4|4|24|2|2053|614|8000|Thursday|2053Q2|N|N|N|2470994|2471083|2470652|2470927|N|N|N|N|N| +2471018|AAAAAAAAKGELFCAA|2053-04-25|1839|8000|614|2053|5|4|25|2|2053|614|8000|Friday|2053Q2|N|Y|N|2470994|2471083|2470653|2470928|N|N|N|N|N| +2471019|AAAAAAAALGELFCAA|2053-04-26|1839|8000|614|2053|6|4|26|2|2053|614|8000|Saturday|2053Q2|N|Y|N|2470994|2471083|2470654|2470929|N|N|N|N|N| +2471020|AAAAAAAAMGELFCAA|2053-04-27|1839|8000|614|2053|0|4|27|2|2053|614|8000|Sunday|2053Q2|N|N|N|2470994|2471083|2470655|2470930|N|N|N|N|N| +2471021|AAAAAAAANGELFCAA|2053-04-28|1839|8000|614|2053|1|4|28|2|2053|614|8000|Monday|2053Q2|N|N|N|2470994|2471083|2470656|2470931|N|N|N|N|N| +2471022|AAAAAAAAOGELFCAA|2053-04-29|1839|8001|614|2053|2|4|29|2|2053|614|8001|Tuesday|2053Q2|N|N|N|2470994|2471083|2470657|2470932|N|N|N|N|N| +2471023|AAAAAAAAPGELFCAA|2053-04-30|1839|8001|614|2053|3|4|30|2|2053|614|8001|Wednesday|2053Q2|N|N|N|2470994|2471083|2470658|2470933|N|N|N|N|N| +2471024|AAAAAAAAAHELFCAA|2053-05-01|1840|8001|614|2053|4|5|1|2|2053|614|8001|Thursday|2053Q2|N|N|N|2471024|2471143|2470659|2470934|N|N|N|N|N| +2471025|AAAAAAAABHELFCAA|2053-05-02|1840|8001|614|2053|5|5|2|2|2053|614|8001|Friday|2053Q2|N|Y|N|2471024|2471143|2470660|2470935|N|N|N|N|N| +2471026|AAAAAAAACHELFCAA|2053-05-03|1840|8001|614|2053|6|5|3|2|2053|614|8001|Saturday|2053Q2|N|Y|N|2471024|2471143|2470661|2470936|N|N|N|N|N| +2471027|AAAAAAAADHELFCAA|2053-05-04|1840|8001|614|2053|0|5|4|2|2053|614|8001|Sunday|2053Q2|N|N|N|2471024|2471143|2470662|2470937|N|N|N|N|N| +2471028|AAAAAAAAEHELFCAA|2053-05-05|1840|8001|614|2053|1|5|5|2|2053|614|8001|Monday|2053Q2|N|N|N|2471024|2471143|2470663|2470938|N|N|N|N|N| +2471029|AAAAAAAAFHELFCAA|2053-05-06|1840|8002|614|2053|2|5|6|2|2053|614|8002|Tuesday|2053Q2|N|N|N|2471024|2471143|2470664|2470939|N|N|N|N|N| +2471030|AAAAAAAAGHELFCAA|2053-05-07|1840|8002|614|2053|3|5|7|2|2053|614|8002|Wednesday|2053Q2|N|N|N|2471024|2471143|2470665|2470940|N|N|N|N|N| +2471031|AAAAAAAAHHELFCAA|2053-05-08|1840|8002|614|2053|4|5|8|2|2053|614|8002|Thursday|2053Q2|N|N|N|2471024|2471143|2470666|2470941|N|N|N|N|N| +2471032|AAAAAAAAIHELFCAA|2053-05-09|1840|8002|614|2053|5|5|9|2|2053|614|8002|Friday|2053Q2|N|Y|N|2471024|2471143|2470667|2470942|N|N|N|N|N| +2471033|AAAAAAAAJHELFCAA|2053-05-10|1840|8002|614|2053|6|5|10|2|2053|614|8002|Saturday|2053Q2|N|Y|N|2471024|2471143|2470668|2470943|N|N|N|N|N| +2471034|AAAAAAAAKHELFCAA|2053-05-11|1840|8002|614|2053|0|5|11|2|2053|614|8002|Sunday|2053Q2|N|N|N|2471024|2471143|2470669|2470944|N|N|N|N|N| +2471035|AAAAAAAALHELFCAA|2053-05-12|1840|8002|614|2053|1|5|12|2|2053|614|8002|Monday|2053Q2|N|N|N|2471024|2471143|2470670|2470945|N|N|N|N|N| +2471036|AAAAAAAAMHELFCAA|2053-05-13|1840|8003|614|2053|2|5|13|2|2053|614|8003|Tuesday|2053Q2|N|N|N|2471024|2471143|2470671|2470946|N|N|N|N|N| +2471037|AAAAAAAANHELFCAA|2053-05-14|1840|8003|614|2053|3|5|14|2|2053|614|8003|Wednesday|2053Q2|N|N|N|2471024|2471143|2470672|2470947|N|N|N|N|N| +2471038|AAAAAAAAOHELFCAA|2053-05-15|1840|8003|614|2053|4|5|15|2|2053|614|8003|Thursday|2053Q2|N|N|N|2471024|2471143|2470673|2470948|N|N|N|N|N| +2471039|AAAAAAAAPHELFCAA|2053-05-16|1840|8003|614|2053|5|5|16|2|2053|614|8003|Friday|2053Q2|N|Y|N|2471024|2471143|2470674|2470949|N|N|N|N|N| +2471040|AAAAAAAAAIELFCAA|2053-05-17|1840|8003|614|2053|6|5|17|2|2053|614|8003|Saturday|2053Q2|N|Y|N|2471024|2471143|2470675|2470950|N|N|N|N|N| +2471041|AAAAAAAABIELFCAA|2053-05-18|1840|8003|614|2053|0|5|18|2|2053|614|8003|Sunday|2053Q2|N|N|N|2471024|2471143|2470676|2470951|N|N|N|N|N| +2471042|AAAAAAAACIELFCAA|2053-05-19|1840|8003|614|2053|1|5|19|2|2053|614|8003|Monday|2053Q2|N|N|N|2471024|2471143|2470677|2470952|N|N|N|N|N| +2471043|AAAAAAAADIELFCAA|2053-05-20|1840|8004|614|2053|2|5|20|2|2053|614|8004|Tuesday|2053Q2|N|N|N|2471024|2471143|2470678|2470953|N|N|N|N|N| +2471044|AAAAAAAAEIELFCAA|2053-05-21|1840|8004|614|2053|3|5|21|2|2053|614|8004|Wednesday|2053Q2|N|N|N|2471024|2471143|2470679|2470954|N|N|N|N|N| +2471045|AAAAAAAAFIELFCAA|2053-05-22|1840|8004|614|2053|4|5|22|2|2053|614|8004|Thursday|2053Q2|N|N|N|2471024|2471143|2470680|2470955|N|N|N|N|N| +2471046|AAAAAAAAGIELFCAA|2053-05-23|1840|8004|614|2053|5|5|23|2|2053|614|8004|Friday|2053Q2|N|Y|N|2471024|2471143|2470681|2470956|N|N|N|N|N| +2471047|AAAAAAAAHIELFCAA|2053-05-24|1840|8004|614|2053|6|5|24|2|2053|614|8004|Saturday|2053Q2|N|Y|N|2471024|2471143|2470682|2470957|N|N|N|N|N| +2471048|AAAAAAAAIIELFCAA|2053-05-25|1840|8004|614|2053|0|5|25|2|2053|614|8004|Sunday|2053Q2|N|N|N|2471024|2471143|2470683|2470958|N|N|N|N|N| +2471049|AAAAAAAAJIELFCAA|2053-05-26|1840|8004|614|2053|1|5|26|2|2053|614|8004|Monday|2053Q2|N|N|N|2471024|2471143|2470684|2470959|N|N|N|N|N| +2471050|AAAAAAAAKIELFCAA|2053-05-27|1840|8005|614|2053|2|5|27|2|2053|614|8005|Tuesday|2053Q2|N|N|N|2471024|2471143|2470685|2470960|N|N|N|N|N| +2471051|AAAAAAAALIELFCAA|2053-05-28|1840|8005|614|2053|3|5|28|2|2053|614|8005|Wednesday|2053Q2|N|N|N|2471024|2471143|2470686|2470961|N|N|N|N|N| +2471052|AAAAAAAAMIELFCAA|2053-05-29|1840|8005|614|2053|4|5|29|2|2053|614|8005|Thursday|2053Q2|N|N|N|2471024|2471143|2470687|2470962|N|N|N|N|N| +2471053|AAAAAAAANIELFCAA|2053-05-30|1840|8005|614|2053|5|5|30|2|2053|614|8005|Friday|2053Q2|N|Y|N|2471024|2471143|2470688|2470963|N|N|N|N|N| +2471054|AAAAAAAAOIELFCAA|2053-05-31|1840|8005|614|2053|6|5|31|2|2053|614|8005|Saturday|2053Q2|N|Y|N|2471024|2471143|2470689|2470964|N|N|N|N|N| +2471055|AAAAAAAAPIELFCAA|2053-06-01|1841|8005|615|2053|0|6|1|2|2053|615|8005|Sunday|2053Q2|N|N|N|2471055|2471205|2470690|2470965|N|N|N|N|N| +2471056|AAAAAAAAAJELFCAA|2053-06-02|1841|8005|615|2053|1|6|2|2|2053|615|8005|Monday|2053Q2|N|N|N|2471055|2471205|2470691|2470966|N|N|N|N|N| +2471057|AAAAAAAABJELFCAA|2053-06-03|1841|8006|615|2053|2|6|3|2|2053|615|8006|Tuesday|2053Q2|N|N|N|2471055|2471205|2470692|2470967|N|N|N|N|N| +2471058|AAAAAAAACJELFCAA|2053-06-04|1841|8006|615|2053|3|6|4|2|2053|615|8006|Wednesday|2053Q2|N|N|N|2471055|2471205|2470693|2470968|N|N|N|N|N| +2471059|AAAAAAAADJELFCAA|2053-06-05|1841|8006|615|2053|4|6|5|2|2053|615|8006|Thursday|2053Q2|N|N|N|2471055|2471205|2470694|2470969|N|N|N|N|N| +2471060|AAAAAAAAEJELFCAA|2053-06-06|1841|8006|615|2053|5|6|6|2|2053|615|8006|Friday|2053Q2|N|Y|N|2471055|2471205|2470695|2470970|N|N|N|N|N| +2471061|AAAAAAAAFJELFCAA|2053-06-07|1841|8006|615|2053|6|6|7|2|2053|615|8006|Saturday|2053Q2|N|Y|N|2471055|2471205|2470696|2470971|N|N|N|N|N| +2471062|AAAAAAAAGJELFCAA|2053-06-08|1841|8006|615|2053|0|6|8|2|2053|615|8006|Sunday|2053Q2|N|N|N|2471055|2471205|2470697|2470972|N|N|N|N|N| +2471063|AAAAAAAAHJELFCAA|2053-06-09|1841|8006|615|2053|1|6|9|2|2053|615|8006|Monday|2053Q2|N|N|N|2471055|2471205|2470698|2470973|N|N|N|N|N| +2471064|AAAAAAAAIJELFCAA|2053-06-10|1841|8007|615|2053|2|6|10|2|2053|615|8007|Tuesday|2053Q2|N|N|N|2471055|2471205|2470699|2470974|N|N|N|N|N| +2471065|AAAAAAAAJJELFCAA|2053-06-11|1841|8007|615|2053|3|6|11|2|2053|615|8007|Wednesday|2053Q2|N|N|N|2471055|2471205|2470700|2470975|N|N|N|N|N| +2471066|AAAAAAAAKJELFCAA|2053-06-12|1841|8007|615|2053|4|6|12|2|2053|615|8007|Thursday|2053Q2|N|N|N|2471055|2471205|2470701|2470976|N|N|N|N|N| +2471067|AAAAAAAALJELFCAA|2053-06-13|1841|8007|615|2053|5|6|13|2|2053|615|8007|Friday|2053Q2|N|Y|N|2471055|2471205|2470702|2470977|N|N|N|N|N| +2471068|AAAAAAAAMJELFCAA|2053-06-14|1841|8007|615|2053|6|6|14|2|2053|615|8007|Saturday|2053Q2|N|Y|N|2471055|2471205|2470703|2470978|N|N|N|N|N| +2471069|AAAAAAAANJELFCAA|2053-06-15|1841|8007|615|2053|0|6|15|2|2053|615|8007|Sunday|2053Q2|N|N|N|2471055|2471205|2470704|2470979|N|N|N|N|N| +2471070|AAAAAAAAOJELFCAA|2053-06-16|1841|8007|615|2053|1|6|16|2|2053|615|8007|Monday|2053Q2|N|N|N|2471055|2471205|2470705|2470980|N|N|N|N|N| +2471071|AAAAAAAAPJELFCAA|2053-06-17|1841|8008|615|2053|2|6|17|2|2053|615|8008|Tuesday|2053Q2|N|N|N|2471055|2471205|2470706|2470981|N|N|N|N|N| +2471072|AAAAAAAAAKELFCAA|2053-06-18|1841|8008|615|2053|3|6|18|2|2053|615|8008|Wednesday|2053Q2|N|N|N|2471055|2471205|2470707|2470982|N|N|N|N|N| +2471073|AAAAAAAABKELFCAA|2053-06-19|1841|8008|615|2053|4|6|19|2|2053|615|8008|Thursday|2053Q2|N|N|N|2471055|2471205|2470708|2470983|N|N|N|N|N| +2471074|AAAAAAAACKELFCAA|2053-06-20|1841|8008|615|2053|5|6|20|2|2053|615|8008|Friday|2053Q2|N|Y|N|2471055|2471205|2470709|2470984|N|N|N|N|N| +2471075|AAAAAAAADKELFCAA|2053-06-21|1841|8008|615|2053|6|6|21|2|2053|615|8008|Saturday|2053Q2|N|Y|N|2471055|2471205|2470710|2470985|N|N|N|N|N| +2471076|AAAAAAAAEKELFCAA|2053-06-22|1841|8008|615|2053|0|6|22|2|2053|615|8008|Sunday|2053Q2|N|N|N|2471055|2471205|2470711|2470986|N|N|N|N|N| +2471077|AAAAAAAAFKELFCAA|2053-06-23|1841|8008|615|2053|1|6|23|2|2053|615|8008|Monday|2053Q2|N|N|N|2471055|2471205|2470712|2470987|N|N|N|N|N| +2471078|AAAAAAAAGKELFCAA|2053-06-24|1841|8009|615|2053|2|6|24|2|2053|615|8009|Tuesday|2053Q2|N|N|N|2471055|2471205|2470713|2470988|N|N|N|N|N| +2471079|AAAAAAAAHKELFCAA|2053-06-25|1841|8009|615|2053|3|6|25|2|2053|615|8009|Wednesday|2053Q2|N|N|N|2471055|2471205|2470714|2470989|N|N|N|N|N| +2471080|AAAAAAAAIKELFCAA|2053-06-26|1841|8009|615|2053|4|6|26|2|2053|615|8009|Thursday|2053Q2|N|N|N|2471055|2471205|2470715|2470990|N|N|N|N|N| +2471081|AAAAAAAAJKELFCAA|2053-06-27|1841|8009|615|2053|5|6|27|2|2053|615|8009|Friday|2053Q2|N|Y|N|2471055|2471205|2470716|2470991|N|N|N|N|N| +2471082|AAAAAAAAKKELFCAA|2053-06-28|1841|8009|615|2053|6|6|28|2|2053|615|8009|Saturday|2053Q2|N|Y|N|2471055|2471205|2470717|2470992|N|N|N|N|N| +2471083|AAAAAAAALKELFCAA|2053-06-29|1841|8009|615|2053|0|6|29|2|2053|615|8009|Sunday|2053Q2|N|N|N|2471055|2471205|2470718|2470993|N|N|N|N|N| +2471084|AAAAAAAAMKELFCAA|2053-06-30|1841|8009|615|2053|1|6|30|2|2053|615|8009|Monday|2053Q2|N|N|N|2471055|2471205|2470719|2470994|N|N|N|N|N| +2471085|AAAAAAAANKELFCAA|2053-07-01|1842|8010|615|2053|2|7|1|2|2053|615|8010|Tuesday|2053Q2|N|N|N|2471085|2471265|2470720|2470994|N|N|N|N|N| +2471086|AAAAAAAAOKELFCAA|2053-07-02|1842|8010|615|2053|3|7|2|3|2053|615|8010|Wednesday|2053Q3|N|N|N|2471085|2471265|2470721|2470995|N|N|N|N|N| +2471087|AAAAAAAAPKELFCAA|2053-07-03|1842|8010|615|2053|4|7|3|3|2053|615|8010|Thursday|2053Q3|N|N|N|2471085|2471265|2470722|2470996|N|N|N|N|N| +2471088|AAAAAAAAALELFCAA|2053-07-04|1842|8010|615|2053|5|7|4|3|2053|615|8010|Friday|2053Q3|N|Y|N|2471085|2471265|2470723|2470997|N|N|N|N|N| +2471089|AAAAAAAABLELFCAA|2053-07-05|1842|8010|615|2053|6|7|5|3|2053|615|8010|Saturday|2053Q3|Y|Y|N|2471085|2471265|2470724|2470998|N|N|N|N|N| +2471090|AAAAAAAACLELFCAA|2053-07-06|1842|8010|615|2053|0|7|6|3|2053|615|8010|Sunday|2053Q3|N|N|Y|2471085|2471265|2470725|2470999|N|N|N|N|N| +2471091|AAAAAAAADLELFCAA|2053-07-07|1842|8010|615|2053|1|7|7|3|2053|615|8010|Monday|2053Q3|N|N|N|2471085|2471265|2470726|2471000|N|N|N|N|N| +2471092|AAAAAAAAELELFCAA|2053-07-08|1842|8011|615|2053|2|7|8|3|2053|615|8011|Tuesday|2053Q3|N|N|N|2471085|2471265|2470727|2471001|N|N|N|N|N| +2471093|AAAAAAAAFLELFCAA|2053-07-09|1842|8011|615|2053|3|7|9|3|2053|615|8011|Wednesday|2053Q3|N|N|N|2471085|2471265|2470728|2471002|N|N|N|N|N| +2471094|AAAAAAAAGLELFCAA|2053-07-10|1842|8011|615|2053|4|7|10|3|2053|615|8011|Thursday|2053Q3|N|N|N|2471085|2471265|2470729|2471003|N|N|N|N|N| +2471095|AAAAAAAAHLELFCAA|2053-07-11|1842|8011|615|2053|5|7|11|3|2053|615|8011|Friday|2053Q3|N|Y|N|2471085|2471265|2470730|2471004|N|N|N|N|N| +2471096|AAAAAAAAILELFCAA|2053-07-12|1842|8011|615|2053|6|7|12|3|2053|615|8011|Saturday|2053Q3|N|Y|N|2471085|2471265|2470731|2471005|N|N|N|N|N| +2471097|AAAAAAAAJLELFCAA|2053-07-13|1842|8011|615|2053|0|7|13|3|2053|615|8011|Sunday|2053Q3|N|N|N|2471085|2471265|2470732|2471006|N|N|N|N|N| +2471098|AAAAAAAAKLELFCAA|2053-07-14|1842|8011|615|2053|1|7|14|3|2053|615|8011|Monday|2053Q3|N|N|N|2471085|2471265|2470733|2471007|N|N|N|N|N| +2471099|AAAAAAAALLELFCAA|2053-07-15|1842|8012|615|2053|2|7|15|3|2053|615|8012|Tuesday|2053Q3|N|N|N|2471085|2471265|2470734|2471008|N|N|N|N|N| +2471100|AAAAAAAAMLELFCAA|2053-07-16|1842|8012|615|2053|3|7|16|3|2053|615|8012|Wednesday|2053Q3|N|N|N|2471085|2471265|2470735|2471009|N|N|N|N|N| +2471101|AAAAAAAANLELFCAA|2053-07-17|1842|8012|615|2053|4|7|17|3|2053|615|8012|Thursday|2053Q3|N|N|N|2471085|2471265|2470736|2471010|N|N|N|N|N| +2471102|AAAAAAAAOLELFCAA|2053-07-18|1842|8012|615|2053|5|7|18|3|2053|615|8012|Friday|2053Q3|N|Y|N|2471085|2471265|2470737|2471011|N|N|N|N|N| +2471103|AAAAAAAAPLELFCAA|2053-07-19|1842|8012|615|2053|6|7|19|3|2053|615|8012|Saturday|2053Q3|N|Y|N|2471085|2471265|2470738|2471012|N|N|N|N|N| +2471104|AAAAAAAAAMELFCAA|2053-07-20|1842|8012|615|2053|0|7|20|3|2053|615|8012|Sunday|2053Q3|N|N|N|2471085|2471265|2470739|2471013|N|N|N|N|N| +2471105|AAAAAAAABMELFCAA|2053-07-21|1842|8012|615|2053|1|7|21|3|2053|615|8012|Monday|2053Q3|N|N|N|2471085|2471265|2470740|2471014|N|N|N|N|N| +2471106|AAAAAAAACMELFCAA|2053-07-22|1842|8013|615|2053|2|7|22|3|2053|615|8013|Tuesday|2053Q3|N|N|N|2471085|2471265|2470741|2471015|N|N|N|N|N| +2471107|AAAAAAAADMELFCAA|2053-07-23|1842|8013|615|2053|3|7|23|3|2053|615|8013|Wednesday|2053Q3|N|N|N|2471085|2471265|2470742|2471016|N|N|N|N|N| +2471108|AAAAAAAAEMELFCAA|2053-07-24|1842|8013|615|2053|4|7|24|3|2053|615|8013|Thursday|2053Q3|N|N|N|2471085|2471265|2470743|2471017|N|N|N|N|N| +2471109|AAAAAAAAFMELFCAA|2053-07-25|1842|8013|615|2053|5|7|25|3|2053|615|8013|Friday|2053Q3|N|Y|N|2471085|2471265|2470744|2471018|N|N|N|N|N| +2471110|AAAAAAAAGMELFCAA|2053-07-26|1842|8013|615|2053|6|7|26|3|2053|615|8013|Saturday|2053Q3|N|Y|N|2471085|2471265|2470745|2471019|N|N|N|N|N| +2471111|AAAAAAAAHMELFCAA|2053-07-27|1842|8013|615|2053|0|7|27|3|2053|615|8013|Sunday|2053Q3|N|N|N|2471085|2471265|2470746|2471020|N|N|N|N|N| +2471112|AAAAAAAAIMELFCAA|2053-07-28|1842|8013|615|2053|1|7|28|3|2053|615|8013|Monday|2053Q3|N|N|N|2471085|2471265|2470747|2471021|N|N|N|N|N| +2471113|AAAAAAAAJMELFCAA|2053-07-29|1842|8014|615|2053|2|7|29|3|2053|615|8014|Tuesday|2053Q3|N|N|N|2471085|2471265|2470748|2471022|N|N|N|N|N| +2471114|AAAAAAAAKMELFCAA|2053-07-30|1842|8014|615|2053|3|7|30|3|2053|615|8014|Wednesday|2053Q3|N|N|N|2471085|2471265|2470749|2471023|N|N|N|N|N| +2471115|AAAAAAAALMELFCAA|2053-07-31|1842|8014|615|2053|4|7|31|3|2053|615|8014|Thursday|2053Q3|N|N|N|2471085|2471265|2470750|2471024|N|N|N|N|N| +2471116|AAAAAAAAMMELFCAA|2053-08-01|1843|8014|615|2053|5|8|1|3|2053|615|8014|Friday|2053Q3|N|Y|N|2471116|2471327|2470751|2471025|N|N|N|N|N| +2471117|AAAAAAAANMELFCAA|2053-08-02|1843|8014|615|2053|6|8|2|3|2053|615|8014|Saturday|2053Q3|N|Y|N|2471116|2471327|2470752|2471026|N|N|N|N|N| +2471118|AAAAAAAAOMELFCAA|2053-08-03|1843|8014|615|2053|0|8|3|3|2053|615|8014|Sunday|2053Q3|N|N|N|2471116|2471327|2470753|2471027|N|N|N|N|N| +2471119|AAAAAAAAPMELFCAA|2053-08-04|1843|8014|615|2053|1|8|4|3|2053|615|8014|Monday|2053Q3|N|N|N|2471116|2471327|2470754|2471028|N|N|N|N|N| +2471120|AAAAAAAAANELFCAA|2053-08-05|1843|8015|615|2053|2|8|5|3|2053|615|8015|Tuesday|2053Q3|N|N|N|2471116|2471327|2470755|2471029|N|N|N|N|N| +2471121|AAAAAAAABNELFCAA|2053-08-06|1843|8015|615|2053|3|8|6|3|2053|615|8015|Wednesday|2053Q3|N|N|N|2471116|2471327|2470756|2471030|N|N|N|N|N| +2471122|AAAAAAAACNELFCAA|2053-08-07|1843|8015|615|2053|4|8|7|3|2053|615|8015|Thursday|2053Q3|N|N|N|2471116|2471327|2470757|2471031|N|N|N|N|N| +2471123|AAAAAAAADNELFCAA|2053-08-08|1843|8015|615|2053|5|8|8|3|2053|615|8015|Friday|2053Q3|N|Y|N|2471116|2471327|2470758|2471032|N|N|N|N|N| +2471124|AAAAAAAAENELFCAA|2053-08-09|1843|8015|615|2053|6|8|9|3|2053|615|8015|Saturday|2053Q3|N|Y|N|2471116|2471327|2470759|2471033|N|N|N|N|N| +2471125|AAAAAAAAFNELFCAA|2053-08-10|1843|8015|615|2053|0|8|10|3|2053|615|8015|Sunday|2053Q3|N|N|N|2471116|2471327|2470760|2471034|N|N|N|N|N| +2471126|AAAAAAAAGNELFCAA|2053-08-11|1843|8015|615|2053|1|8|11|3|2053|615|8015|Monday|2053Q3|N|N|N|2471116|2471327|2470761|2471035|N|N|N|N|N| +2471127|AAAAAAAAHNELFCAA|2053-08-12|1843|8016|615|2053|2|8|12|3|2053|615|8016|Tuesday|2053Q3|N|N|N|2471116|2471327|2470762|2471036|N|N|N|N|N| +2471128|AAAAAAAAINELFCAA|2053-08-13|1843|8016|615|2053|3|8|13|3|2053|615|8016|Wednesday|2053Q3|N|N|N|2471116|2471327|2470763|2471037|N|N|N|N|N| +2471129|AAAAAAAAJNELFCAA|2053-08-14|1843|8016|615|2053|4|8|14|3|2053|615|8016|Thursday|2053Q3|N|N|N|2471116|2471327|2470764|2471038|N|N|N|N|N| +2471130|AAAAAAAAKNELFCAA|2053-08-15|1843|8016|615|2053|5|8|15|3|2053|615|8016|Friday|2053Q3|N|Y|N|2471116|2471327|2470765|2471039|N|N|N|N|N| +2471131|AAAAAAAALNELFCAA|2053-08-16|1843|8016|615|2053|6|8|16|3|2053|615|8016|Saturday|2053Q3|N|Y|N|2471116|2471327|2470766|2471040|N|N|N|N|N| +2471132|AAAAAAAAMNELFCAA|2053-08-17|1843|8016|615|2053|0|8|17|3|2053|615|8016|Sunday|2053Q3|N|N|N|2471116|2471327|2470767|2471041|N|N|N|N|N| +2471133|AAAAAAAANNELFCAA|2053-08-18|1843|8016|615|2053|1|8|18|3|2053|615|8016|Monday|2053Q3|N|N|N|2471116|2471327|2470768|2471042|N|N|N|N|N| +2471134|AAAAAAAAONELFCAA|2053-08-19|1843|8017|615|2053|2|8|19|3|2053|615|8017|Tuesday|2053Q3|N|N|N|2471116|2471327|2470769|2471043|N|N|N|N|N| +2471135|AAAAAAAAPNELFCAA|2053-08-20|1843|8017|615|2053|3|8|20|3|2053|615|8017|Wednesday|2053Q3|N|N|N|2471116|2471327|2470770|2471044|N|N|N|N|N| +2471136|AAAAAAAAAOELFCAA|2053-08-21|1843|8017|615|2053|4|8|21|3|2053|615|8017|Thursday|2053Q3|N|N|N|2471116|2471327|2470771|2471045|N|N|N|N|N| +2471137|AAAAAAAABOELFCAA|2053-08-22|1843|8017|615|2053|5|8|22|3|2053|615|8017|Friday|2053Q3|N|Y|N|2471116|2471327|2470772|2471046|N|N|N|N|N| +2471138|AAAAAAAACOELFCAA|2053-08-23|1843|8017|615|2053|6|8|23|3|2053|615|8017|Saturday|2053Q3|N|Y|N|2471116|2471327|2470773|2471047|N|N|N|N|N| +2471139|AAAAAAAADOELFCAA|2053-08-24|1843|8017|615|2053|0|8|24|3|2053|615|8017|Sunday|2053Q3|N|N|N|2471116|2471327|2470774|2471048|N|N|N|N|N| +2471140|AAAAAAAAEOELFCAA|2053-08-25|1843|8017|615|2053|1|8|25|3|2053|615|8017|Monday|2053Q3|N|N|N|2471116|2471327|2470775|2471049|N|N|N|N|N| +2471141|AAAAAAAAFOELFCAA|2053-08-26|1843|8018|615|2053|2|8|26|3|2053|615|8018|Tuesday|2053Q3|N|N|N|2471116|2471327|2470776|2471050|N|N|N|N|N| +2471142|AAAAAAAAGOELFCAA|2053-08-27|1843|8018|615|2053|3|8|27|3|2053|615|8018|Wednesday|2053Q3|N|N|N|2471116|2471327|2470777|2471051|N|N|N|N|N| +2471143|AAAAAAAAHOELFCAA|2053-08-28|1843|8018|615|2053|4|8|28|3|2053|615|8018|Thursday|2053Q3|N|N|N|2471116|2471327|2470778|2471052|N|N|N|N|N| +2471144|AAAAAAAAIOELFCAA|2053-08-29|1843|8018|615|2053|5|8|29|3|2053|615|8018|Friday|2053Q3|N|Y|N|2471116|2471327|2470779|2471053|N|N|N|N|N| +2471145|AAAAAAAAJOELFCAA|2053-08-30|1843|8018|615|2053|6|8|30|3|2053|615|8018|Saturday|2053Q3|N|Y|N|2471116|2471327|2470780|2471054|N|N|N|N|N| +2471146|AAAAAAAAKOELFCAA|2053-08-31|1843|8018|615|2053|0|8|31|3|2053|615|8018|Sunday|2053Q3|N|N|N|2471116|2471327|2470781|2471055|N|N|N|N|N| +2471147|AAAAAAAALOELFCAA|2053-09-01|1844|8018|616|2053|1|9|1|3|2053|616|8018|Monday|2053Q3|N|N|N|2471147|2471389|2470782|2471056|N|N|N|N|N| +2471148|AAAAAAAAMOELFCAA|2053-09-02|1844|8019|616|2053|2|9|2|3|2053|616|8019|Tuesday|2053Q3|N|N|N|2471147|2471389|2470783|2471057|N|N|N|N|N| +2471149|AAAAAAAANOELFCAA|2053-09-03|1844|8019|616|2053|3|9|3|3|2053|616|8019|Wednesday|2053Q3|N|N|N|2471147|2471389|2470784|2471058|N|N|N|N|N| +2471150|AAAAAAAAOOELFCAA|2053-09-04|1844|8019|616|2053|4|9|4|3|2053|616|8019|Thursday|2053Q3|N|N|N|2471147|2471389|2470785|2471059|N|N|N|N|N| +2471151|AAAAAAAAPOELFCAA|2053-09-05|1844|8019|616|2053|5|9|5|3|2053|616|8019|Friday|2053Q3|N|Y|N|2471147|2471389|2470786|2471060|N|N|N|N|N| +2471152|AAAAAAAAAPELFCAA|2053-09-06|1844|8019|616|2053|6|9|6|3|2053|616|8019|Saturday|2053Q3|N|Y|N|2471147|2471389|2470787|2471061|N|N|N|N|N| +2471153|AAAAAAAABPELFCAA|2053-09-07|1844|8019|616|2053|0|9|7|3|2053|616|8019|Sunday|2053Q3|N|N|N|2471147|2471389|2470788|2471062|N|N|N|N|N| +2471154|AAAAAAAACPELFCAA|2053-09-08|1844|8019|616|2053|1|9|8|3|2053|616|8019|Monday|2053Q3|N|N|N|2471147|2471389|2470789|2471063|N|N|N|N|N| +2471155|AAAAAAAADPELFCAA|2053-09-09|1844|8020|616|2053|2|9|9|3|2053|616|8020|Tuesday|2053Q3|N|N|N|2471147|2471389|2470790|2471064|N|N|N|N|N| +2471156|AAAAAAAAEPELFCAA|2053-09-10|1844|8020|616|2053|3|9|10|3|2053|616|8020|Wednesday|2053Q3|N|N|N|2471147|2471389|2470791|2471065|N|N|N|N|N| +2471157|AAAAAAAAFPELFCAA|2053-09-11|1844|8020|616|2053|4|9|11|3|2053|616|8020|Thursday|2053Q3|N|N|N|2471147|2471389|2470792|2471066|N|N|N|N|N| +2471158|AAAAAAAAGPELFCAA|2053-09-12|1844|8020|616|2053|5|9|12|3|2053|616|8020|Friday|2053Q3|N|Y|N|2471147|2471389|2470793|2471067|N|N|N|N|N| +2471159|AAAAAAAAHPELFCAA|2053-09-13|1844|8020|616|2053|6|9|13|3|2053|616|8020|Saturday|2053Q3|N|Y|N|2471147|2471389|2470794|2471068|N|N|N|N|N| +2471160|AAAAAAAAIPELFCAA|2053-09-14|1844|8020|616|2053|0|9|14|3|2053|616|8020|Sunday|2053Q3|N|N|N|2471147|2471389|2470795|2471069|N|N|N|N|N| +2471161|AAAAAAAAJPELFCAA|2053-09-15|1844|8020|616|2053|1|9|15|3|2053|616|8020|Monday|2053Q3|N|N|N|2471147|2471389|2470796|2471070|N|N|N|N|N| +2471162|AAAAAAAAKPELFCAA|2053-09-16|1844|8021|616|2053|2|9|16|3|2053|616|8021|Tuesday|2053Q3|N|N|N|2471147|2471389|2470797|2471071|N|N|N|N|N| +2471163|AAAAAAAALPELFCAA|2053-09-17|1844|8021|616|2053|3|9|17|3|2053|616|8021|Wednesday|2053Q3|N|N|N|2471147|2471389|2470798|2471072|N|N|N|N|N| +2471164|AAAAAAAAMPELFCAA|2053-09-18|1844|8021|616|2053|4|9|18|3|2053|616|8021|Thursday|2053Q3|N|N|N|2471147|2471389|2470799|2471073|N|N|N|N|N| +2471165|AAAAAAAANPELFCAA|2053-09-19|1844|8021|616|2053|5|9|19|3|2053|616|8021|Friday|2053Q3|N|Y|N|2471147|2471389|2470800|2471074|N|N|N|N|N| +2471166|AAAAAAAAOPELFCAA|2053-09-20|1844|8021|616|2053|6|9|20|3|2053|616|8021|Saturday|2053Q3|N|Y|N|2471147|2471389|2470801|2471075|N|N|N|N|N| +2471167|AAAAAAAAPPELFCAA|2053-09-21|1844|8021|616|2053|0|9|21|3|2053|616|8021|Sunday|2053Q3|N|N|N|2471147|2471389|2470802|2471076|N|N|N|N|N| +2471168|AAAAAAAAAAFLFCAA|2053-09-22|1844|8021|616|2053|1|9|22|3|2053|616|8021|Monday|2053Q3|N|N|N|2471147|2471389|2470803|2471077|N|N|N|N|N| +2471169|AAAAAAAABAFLFCAA|2053-09-23|1844|8022|616|2053|2|9|23|3|2053|616|8022|Tuesday|2053Q3|N|N|N|2471147|2471389|2470804|2471078|N|N|N|N|N| +2471170|AAAAAAAACAFLFCAA|2053-09-24|1844|8022|616|2053|3|9|24|3|2053|616|8022|Wednesday|2053Q3|N|N|N|2471147|2471389|2470805|2471079|N|N|N|N|N| +2471171|AAAAAAAADAFLFCAA|2053-09-25|1844|8022|616|2053|4|9|25|3|2053|616|8022|Thursday|2053Q3|N|N|N|2471147|2471389|2470806|2471080|N|N|N|N|N| +2471172|AAAAAAAAEAFLFCAA|2053-09-26|1844|8022|616|2053|5|9|26|3|2053|616|8022|Friday|2053Q3|N|Y|N|2471147|2471389|2470807|2471081|N|N|N|N|N| +2471173|AAAAAAAAFAFLFCAA|2053-09-27|1844|8022|616|2053|6|9|27|3|2053|616|8022|Saturday|2053Q3|N|Y|N|2471147|2471389|2470808|2471082|N|N|N|N|N| +2471174|AAAAAAAAGAFLFCAA|2053-09-28|1844|8022|616|2053|0|9|28|3|2053|616|8022|Sunday|2053Q3|N|N|N|2471147|2471389|2470809|2471083|N|N|N|N|N| +2471175|AAAAAAAAHAFLFCAA|2053-09-29|1844|8022|616|2053|1|9|29|3|2053|616|8022|Monday|2053Q3|N|N|N|2471147|2471389|2470810|2471084|N|N|N|N|N| +2471176|AAAAAAAAIAFLFCAA|2053-09-30|1844|8023|616|2053|2|9|30|3|2053|616|8023|Tuesday|2053Q3|N|N|N|2471147|2471389|2470811|2471085|N|N|N|N|N| +2471177|AAAAAAAAJAFLFCAA|2053-10-01|1845|8023|616|2053|3|10|1|3|2053|616|8023|Wednesday|2053Q3|N|N|N|2471177|2471449|2470812|2471085|N|N|N|N|N| +2471178|AAAAAAAAKAFLFCAA|2053-10-02|1845|8023|616|2053|4|10|2|4|2053|616|8023|Thursday|2053Q4|N|N|N|2471177|2471449|2470813|2471086|N|N|N|N|N| +2471179|AAAAAAAALAFLFCAA|2053-10-03|1845|8023|616|2053|5|10|3|4|2053|616|8023|Friday|2053Q4|N|Y|N|2471177|2471449|2470814|2471087|N|N|N|N|N| +2471180|AAAAAAAAMAFLFCAA|2053-10-04|1845|8023|616|2053|6|10|4|4|2053|616|8023|Saturday|2053Q4|N|Y|N|2471177|2471449|2470815|2471088|N|N|N|N|N| +2471181|AAAAAAAANAFLFCAA|2053-10-05|1845|8023|616|2053|0|10|5|4|2053|616|8023|Sunday|2053Q4|N|N|N|2471177|2471449|2470816|2471089|N|N|N|N|N| +2471182|AAAAAAAAOAFLFCAA|2053-10-06|1845|8023|616|2053|1|10|6|4|2053|616|8023|Monday|2053Q4|N|N|N|2471177|2471449|2470817|2471090|N|N|N|N|N| +2471183|AAAAAAAAPAFLFCAA|2053-10-07|1845|8024|616|2053|2|10|7|4|2053|616|8024|Tuesday|2053Q4|N|N|N|2471177|2471449|2470818|2471091|N|N|N|N|N| +2471184|AAAAAAAAABFLFCAA|2053-10-08|1845|8024|616|2053|3|10|8|4|2053|616|8024|Wednesday|2053Q4|N|N|N|2471177|2471449|2470819|2471092|N|N|N|N|N| +2471185|AAAAAAAABBFLFCAA|2053-10-09|1845|8024|616|2053|4|10|9|4|2053|616|8024|Thursday|2053Q4|N|N|N|2471177|2471449|2470820|2471093|N|N|N|N|N| +2471186|AAAAAAAACBFLFCAA|2053-10-10|1845|8024|616|2053|5|10|10|4|2053|616|8024|Friday|2053Q4|N|Y|N|2471177|2471449|2470821|2471094|N|N|N|N|N| +2471187|AAAAAAAADBFLFCAA|2053-10-11|1845|8024|616|2053|6|10|11|4|2053|616|8024|Saturday|2053Q4|N|Y|N|2471177|2471449|2470822|2471095|N|N|N|N|N| +2471188|AAAAAAAAEBFLFCAA|2053-10-12|1845|8024|616|2053|0|10|12|4|2053|616|8024|Sunday|2053Q4|N|N|N|2471177|2471449|2470823|2471096|N|N|N|N|N| +2471189|AAAAAAAAFBFLFCAA|2053-10-13|1845|8024|616|2053|1|10|13|4|2053|616|8024|Monday|2053Q4|N|N|N|2471177|2471449|2470824|2471097|N|N|N|N|N| +2471190|AAAAAAAAGBFLFCAA|2053-10-14|1845|8025|616|2053|2|10|14|4|2053|616|8025|Tuesday|2053Q4|N|N|N|2471177|2471449|2470825|2471098|N|N|N|N|N| +2471191|AAAAAAAAHBFLFCAA|2053-10-15|1845|8025|616|2053|3|10|15|4|2053|616|8025|Wednesday|2053Q4|N|N|N|2471177|2471449|2470826|2471099|N|N|N|N|N| +2471192|AAAAAAAAIBFLFCAA|2053-10-16|1845|8025|616|2053|4|10|16|4|2053|616|8025|Thursday|2053Q4|N|N|N|2471177|2471449|2470827|2471100|N|N|N|N|N| +2471193|AAAAAAAAJBFLFCAA|2053-10-17|1845|8025|616|2053|5|10|17|4|2053|616|8025|Friday|2053Q4|N|Y|N|2471177|2471449|2470828|2471101|N|N|N|N|N| +2471194|AAAAAAAAKBFLFCAA|2053-10-18|1845|8025|616|2053|6|10|18|4|2053|616|8025|Saturday|2053Q4|N|Y|N|2471177|2471449|2470829|2471102|N|N|N|N|N| +2471195|AAAAAAAALBFLFCAA|2053-10-19|1845|8025|616|2053|0|10|19|4|2053|616|8025|Sunday|2053Q4|N|N|N|2471177|2471449|2470830|2471103|N|N|N|N|N| +2471196|AAAAAAAAMBFLFCAA|2053-10-20|1845|8025|616|2053|1|10|20|4|2053|616|8025|Monday|2053Q4|N|N|N|2471177|2471449|2470831|2471104|N|N|N|N|N| +2471197|AAAAAAAANBFLFCAA|2053-10-21|1845|8026|616|2053|2|10|21|4|2053|616|8026|Tuesday|2053Q4|N|N|N|2471177|2471449|2470832|2471105|N|N|N|N|N| +2471198|AAAAAAAAOBFLFCAA|2053-10-22|1845|8026|616|2053|3|10|22|4|2053|616|8026|Wednesday|2053Q4|N|N|N|2471177|2471449|2470833|2471106|N|N|N|N|N| +2471199|AAAAAAAAPBFLFCAA|2053-10-23|1845|8026|616|2053|4|10|23|4|2053|616|8026|Thursday|2053Q4|N|N|N|2471177|2471449|2470834|2471107|N|N|N|N|N| +2471200|AAAAAAAAACFLFCAA|2053-10-24|1845|8026|616|2053|5|10|24|4|2053|616|8026|Friday|2053Q4|N|Y|N|2471177|2471449|2470835|2471108|N|N|N|N|N| +2471201|AAAAAAAABCFLFCAA|2053-10-25|1845|8026|616|2053|6|10|25|4|2053|616|8026|Saturday|2053Q4|N|Y|N|2471177|2471449|2470836|2471109|N|N|N|N|N| +2471202|AAAAAAAACCFLFCAA|2053-10-26|1845|8026|616|2053|0|10|26|4|2053|616|8026|Sunday|2053Q4|N|N|N|2471177|2471449|2470837|2471110|N|N|N|N|N| +2471203|AAAAAAAADCFLFCAA|2053-10-27|1845|8026|616|2053|1|10|27|4|2053|616|8026|Monday|2053Q4|N|N|N|2471177|2471449|2470838|2471111|N|N|N|N|N| +2471204|AAAAAAAAECFLFCAA|2053-10-28|1845|8027|616|2053|2|10|28|4|2053|616|8027|Tuesday|2053Q4|N|N|N|2471177|2471449|2470839|2471112|N|N|N|N|N| +2471205|AAAAAAAAFCFLFCAA|2053-10-29|1845|8027|616|2053|3|10|29|4|2053|616|8027|Wednesday|2053Q4|N|N|N|2471177|2471449|2470840|2471113|N|N|N|N|N| +2471206|AAAAAAAAGCFLFCAA|2053-10-30|1845|8027|616|2053|4|10|30|4|2053|616|8027|Thursday|2053Q4|N|N|N|2471177|2471449|2470841|2471114|N|N|N|N|N| +2471207|AAAAAAAAHCFLFCAA|2053-10-31|1845|8027|616|2053|5|10|31|4|2053|616|8027|Friday|2053Q4|N|Y|N|2471177|2471449|2470842|2471115|N|N|N|N|N| +2471208|AAAAAAAAICFLFCAA|2053-11-01|1846|8027|616|2053|6|11|1|4|2053|616|8027|Saturday|2053Q4|N|Y|N|2471208|2471511|2470843|2471116|N|N|N|N|N| +2471209|AAAAAAAAJCFLFCAA|2053-11-02|1846|8027|616|2053|0|11|2|4|2053|616|8027|Sunday|2053Q4|N|N|N|2471208|2471511|2470844|2471117|N|N|N|N|N| +2471210|AAAAAAAAKCFLFCAA|2053-11-03|1846|8027|616|2053|1|11|3|4|2053|616|8027|Monday|2053Q4|N|N|N|2471208|2471511|2470845|2471118|N|N|N|N|N| +2471211|AAAAAAAALCFLFCAA|2053-11-04|1846|8028|616|2053|2|11|4|4|2053|616|8028|Tuesday|2053Q4|N|N|N|2471208|2471511|2470846|2471119|N|N|N|N|N| +2471212|AAAAAAAAMCFLFCAA|2053-11-05|1846|8028|616|2053|3|11|5|4|2053|616|8028|Wednesday|2053Q4|N|N|N|2471208|2471511|2470847|2471120|N|N|N|N|N| +2471213|AAAAAAAANCFLFCAA|2053-11-06|1846|8028|616|2053|4|11|6|4|2053|616|8028|Thursday|2053Q4|N|N|N|2471208|2471511|2470848|2471121|N|N|N|N|N| +2471214|AAAAAAAAOCFLFCAA|2053-11-07|1846|8028|616|2053|5|11|7|4|2053|616|8028|Friday|2053Q4|N|Y|N|2471208|2471511|2470849|2471122|N|N|N|N|N| +2471215|AAAAAAAAPCFLFCAA|2053-11-08|1846|8028|616|2053|6|11|8|4|2053|616|8028|Saturday|2053Q4|N|Y|N|2471208|2471511|2470850|2471123|N|N|N|N|N| +2471216|AAAAAAAAADFLFCAA|2053-11-09|1846|8028|616|2053|0|11|9|4|2053|616|8028|Sunday|2053Q4|N|N|N|2471208|2471511|2470851|2471124|N|N|N|N|N| +2471217|AAAAAAAABDFLFCAA|2053-11-10|1846|8028|616|2053|1|11|10|4|2053|616|8028|Monday|2053Q4|N|N|N|2471208|2471511|2470852|2471125|N|N|N|N|N| +2471218|AAAAAAAACDFLFCAA|2053-11-11|1846|8029|616|2053|2|11|11|4|2053|616|8029|Tuesday|2053Q4|N|N|N|2471208|2471511|2470853|2471126|N|N|N|N|N| +2471219|AAAAAAAADDFLFCAA|2053-11-12|1846|8029|616|2053|3|11|12|4|2053|616|8029|Wednesday|2053Q4|N|N|N|2471208|2471511|2470854|2471127|N|N|N|N|N| +2471220|AAAAAAAAEDFLFCAA|2053-11-13|1846|8029|616|2053|4|11|13|4|2053|616|8029|Thursday|2053Q4|N|N|N|2471208|2471511|2470855|2471128|N|N|N|N|N| +2471221|AAAAAAAAFDFLFCAA|2053-11-14|1846|8029|616|2053|5|11|14|4|2053|616|8029|Friday|2053Q4|N|Y|N|2471208|2471511|2470856|2471129|N|N|N|N|N| +2471222|AAAAAAAAGDFLFCAA|2053-11-15|1846|8029|616|2053|6|11|15|4|2053|616|8029|Saturday|2053Q4|N|Y|N|2471208|2471511|2470857|2471130|N|N|N|N|N| +2471223|AAAAAAAAHDFLFCAA|2053-11-16|1846|8029|616|2053|0|11|16|4|2053|616|8029|Sunday|2053Q4|N|N|N|2471208|2471511|2470858|2471131|N|N|N|N|N| +2471224|AAAAAAAAIDFLFCAA|2053-11-17|1846|8029|616|2053|1|11|17|4|2053|616|8029|Monday|2053Q4|N|N|N|2471208|2471511|2470859|2471132|N|N|N|N|N| +2471225|AAAAAAAAJDFLFCAA|2053-11-18|1846|8030|616|2053|2|11|18|4|2053|616|8030|Tuesday|2053Q4|N|N|N|2471208|2471511|2470860|2471133|N|N|N|N|N| +2471226|AAAAAAAAKDFLFCAA|2053-11-19|1846|8030|616|2053|3|11|19|4|2053|616|8030|Wednesday|2053Q4|N|N|N|2471208|2471511|2470861|2471134|N|N|N|N|N| +2471227|AAAAAAAALDFLFCAA|2053-11-20|1846|8030|616|2053|4|11|20|4|2053|616|8030|Thursday|2053Q4|N|N|N|2471208|2471511|2470862|2471135|N|N|N|N|N| +2471228|AAAAAAAAMDFLFCAA|2053-11-21|1846|8030|616|2053|5|11|21|4|2053|616|8030|Friday|2053Q4|N|Y|N|2471208|2471511|2470863|2471136|N|N|N|N|N| +2471229|AAAAAAAANDFLFCAA|2053-11-22|1846|8030|616|2053|6|11|22|4|2053|616|8030|Saturday|2053Q4|N|Y|N|2471208|2471511|2470864|2471137|N|N|N|N|N| +2471230|AAAAAAAAODFLFCAA|2053-11-23|1846|8030|616|2053|0|11|23|4|2053|616|8030|Sunday|2053Q4|N|N|N|2471208|2471511|2470865|2471138|N|N|N|N|N| +2471231|AAAAAAAAPDFLFCAA|2053-11-24|1846|8030|616|2053|1|11|24|4|2053|616|8030|Monday|2053Q4|N|N|N|2471208|2471511|2470866|2471139|N|N|N|N|N| +2471232|AAAAAAAAAEFLFCAA|2053-11-25|1846|8031|616|2053|2|11|25|4|2053|616|8031|Tuesday|2053Q4|N|N|N|2471208|2471511|2470867|2471140|N|N|N|N|N| +2471233|AAAAAAAABEFLFCAA|2053-11-26|1846|8031|616|2053|3|11|26|4|2053|616|8031|Wednesday|2053Q4|N|N|N|2471208|2471511|2470868|2471141|N|N|N|N|N| +2471234|AAAAAAAACEFLFCAA|2053-11-27|1846|8031|616|2053|4|11|27|4|2053|616|8031|Thursday|2053Q4|N|N|N|2471208|2471511|2470869|2471142|N|N|N|N|N| +2471235|AAAAAAAADEFLFCAA|2053-11-28|1846|8031|616|2053|5|11|28|4|2053|616|8031|Friday|2053Q4|N|Y|N|2471208|2471511|2470870|2471143|N|N|N|N|N| +2471236|AAAAAAAAEEFLFCAA|2053-11-29|1846|8031|616|2053|6|11|29|4|2053|616|8031|Saturday|2053Q4|N|Y|N|2471208|2471511|2470871|2471144|N|N|N|N|N| +2471237|AAAAAAAAFEFLFCAA|2053-11-30|1846|8031|616|2053|0|11|30|4|2053|616|8031|Sunday|2053Q4|N|N|N|2471208|2471511|2470872|2471145|N|N|N|N|N| +2471238|AAAAAAAAGEFLFCAA|2053-12-01|1847|8031|617|2053|1|12|1|4|2053|617|8031|Monday|2053Q4|N|N|N|2471238|2471571|2470873|2471146|N|N|N|N|N| +2471239|AAAAAAAAHEFLFCAA|2053-12-02|1847|8032|617|2053|2|12|2|4|2053|617|8032|Tuesday|2053Q4|N|N|N|2471238|2471571|2470874|2471147|N|N|N|N|N| +2471240|AAAAAAAAIEFLFCAA|2053-12-03|1847|8032|617|2053|3|12|3|4|2053|617|8032|Wednesday|2053Q4|N|N|N|2471238|2471571|2470875|2471148|N|N|N|N|N| +2471241|AAAAAAAAJEFLFCAA|2053-12-04|1847|8032|617|2053|4|12|4|4|2053|617|8032|Thursday|2053Q4|N|N|N|2471238|2471571|2470876|2471149|N|N|N|N|N| +2471242|AAAAAAAAKEFLFCAA|2053-12-05|1847|8032|617|2053|5|12|5|4|2053|617|8032|Friday|2053Q4|N|Y|N|2471238|2471571|2470877|2471150|N|N|N|N|N| +2471243|AAAAAAAALEFLFCAA|2053-12-06|1847|8032|617|2053|6|12|6|4|2053|617|8032|Saturday|2053Q4|N|Y|N|2471238|2471571|2470878|2471151|N|N|N|N|N| +2471244|AAAAAAAAMEFLFCAA|2053-12-07|1847|8032|617|2053|0|12|7|4|2053|617|8032|Sunday|2053Q4|N|N|N|2471238|2471571|2470879|2471152|N|N|N|N|N| +2471245|AAAAAAAANEFLFCAA|2053-12-08|1847|8032|617|2053|1|12|8|4|2053|617|8032|Monday|2053Q4|N|N|N|2471238|2471571|2470880|2471153|N|N|N|N|N| +2471246|AAAAAAAAOEFLFCAA|2053-12-09|1847|8033|617|2053|2|12|9|4|2053|617|8033|Tuesday|2053Q4|N|N|N|2471238|2471571|2470881|2471154|N|N|N|N|N| +2471247|AAAAAAAAPEFLFCAA|2053-12-10|1847|8033|617|2053|3|12|10|4|2053|617|8033|Wednesday|2053Q4|N|N|N|2471238|2471571|2470882|2471155|N|N|N|N|N| +2471248|AAAAAAAAAFFLFCAA|2053-12-11|1847|8033|617|2053|4|12|11|4|2053|617|8033|Thursday|2053Q4|N|N|N|2471238|2471571|2470883|2471156|N|N|N|N|N| +2471249|AAAAAAAABFFLFCAA|2053-12-12|1847|8033|617|2053|5|12|12|4|2053|617|8033|Friday|2053Q4|N|Y|N|2471238|2471571|2470884|2471157|N|N|N|N|N| +2471250|AAAAAAAACFFLFCAA|2053-12-13|1847|8033|617|2053|6|12|13|4|2053|617|8033|Saturday|2053Q4|N|Y|N|2471238|2471571|2470885|2471158|N|N|N|N|N| +2471251|AAAAAAAADFFLFCAA|2053-12-14|1847|8033|617|2053|0|12|14|4|2053|617|8033|Sunday|2053Q4|N|N|N|2471238|2471571|2470886|2471159|N|N|N|N|N| +2471252|AAAAAAAAEFFLFCAA|2053-12-15|1847|8033|617|2053|1|12|15|4|2053|617|8033|Monday|2053Q4|N|N|N|2471238|2471571|2470887|2471160|N|N|N|N|N| +2471253|AAAAAAAAFFFLFCAA|2053-12-16|1847|8034|617|2053|2|12|16|4|2053|617|8034|Tuesday|2053Q4|N|N|N|2471238|2471571|2470888|2471161|N|N|N|N|N| +2471254|AAAAAAAAGFFLFCAA|2053-12-17|1847|8034|617|2053|3|12|17|4|2053|617|8034|Wednesday|2053Q4|N|N|N|2471238|2471571|2470889|2471162|N|N|N|N|N| +2471255|AAAAAAAAHFFLFCAA|2053-12-18|1847|8034|617|2053|4|12|18|4|2053|617|8034|Thursday|2053Q4|N|N|N|2471238|2471571|2470890|2471163|N|N|N|N|N| +2471256|AAAAAAAAIFFLFCAA|2053-12-19|1847|8034|617|2053|5|12|19|4|2053|617|8034|Friday|2053Q4|N|Y|N|2471238|2471571|2470891|2471164|N|N|N|N|N| +2471257|AAAAAAAAJFFLFCAA|2053-12-20|1847|8034|617|2053|6|12|20|4|2053|617|8034|Saturday|2053Q4|N|Y|N|2471238|2471571|2470892|2471165|N|N|N|N|N| +2471258|AAAAAAAAKFFLFCAA|2053-12-21|1847|8034|617|2053|0|12|21|4|2053|617|8034|Sunday|2053Q4|N|N|N|2471238|2471571|2470893|2471166|N|N|N|N|N| +2471259|AAAAAAAALFFLFCAA|2053-12-22|1847|8034|617|2053|1|12|22|4|2053|617|8034|Monday|2053Q4|N|N|N|2471238|2471571|2470894|2471167|N|N|N|N|N| +2471260|AAAAAAAAMFFLFCAA|2053-12-23|1847|8035|617|2053|2|12|23|4|2053|617|8035|Tuesday|2053Q4|N|N|N|2471238|2471571|2470895|2471168|N|N|N|N|N| +2471261|AAAAAAAANFFLFCAA|2053-12-24|1847|8035|617|2053|3|12|24|4|2053|617|8035|Wednesday|2053Q4|N|N|N|2471238|2471571|2470896|2471169|N|N|N|N|N| +2471262|AAAAAAAAOFFLFCAA|2053-12-25|1847|8035|617|2053|4|12|25|4|2053|617|8035|Thursday|2053Q4|N|N|N|2471238|2471571|2470897|2471170|N|N|N|N|N| +2471263|AAAAAAAAPFFLFCAA|2053-12-26|1847|8035|617|2053|5|12|26|4|2053|617|8035|Friday|2053Q4|Y|Y|N|2471238|2471571|2470898|2471171|N|N|N|N|N| +2471264|AAAAAAAAAGFLFCAA|2053-12-27|1847|8035|617|2053|6|12|27|4|2053|617|8035|Saturday|2053Q4|N|Y|Y|2471238|2471571|2470899|2471172|N|N|N|N|N| +2471265|AAAAAAAABGFLFCAA|2053-12-28|1847|8035|617|2053|0|12|28|4|2053|617|8035|Sunday|2053Q4|N|N|N|2471238|2471571|2470900|2471173|N|N|N|N|N| +2471266|AAAAAAAACGFLFCAA|2053-12-29|1847|8035|617|2053|1|12|29|4|2053|617|8035|Monday|2053Q4|N|N|N|2471238|2471571|2470901|2471174|N|N|N|N|N| +2471267|AAAAAAAADGFLFCAA|2053-12-30|1847|8036|617|2053|2|12|30|4|2053|617|8036|Tuesday|2053Q4|N|N|N|2471238|2471571|2470902|2471175|N|N|N|N|N| +2471268|AAAAAAAAEGFLFCAA|2053-12-31|1847|8036|617|2053|3|12|31|4|2053|617|8036|Wednesday|2053Q4|N|N|N|2471238|2471571|2470903|2471176|N|N|N|N|N| +2471269|AAAAAAAAFGFLFCAA|2054-01-01|1848|8036|617|2054|4|1|1|1|2054|617|8036|Thursday|2054Q1|Y|N|N|2471269|2471268|2470904|2471177|N|N|N|N|N| +2471270|AAAAAAAAGGFLFCAA|2054-01-02|1848|8036|617|2054|5|1|2|1|2054|617|8036|Friday|2054Q1|N|Y|Y|2471269|2471268|2470905|2471178|N|N|N|N|N| +2471271|AAAAAAAAHGFLFCAA|2054-01-03|1848|8036|617|2054|6|1|3|1|2054|617|8036|Saturday|2054Q1|N|Y|N|2471269|2471268|2470906|2471179|N|N|N|N|N| +2471272|AAAAAAAAIGFLFCAA|2054-01-04|1848|8036|617|2054|0|1|4|1|2054|617|8036|Sunday|2054Q1|N|N|N|2471269|2471268|2470907|2471180|N|N|N|N|N| +2471273|AAAAAAAAJGFLFCAA|2054-01-05|1848|8036|617|2054|1|1|5|1|2054|617|8036|Monday|2054Q1|N|N|N|2471269|2471268|2470908|2471181|N|N|N|N|N| +2471274|AAAAAAAAKGFLFCAA|2054-01-06|1848|8037|617|2054|2|1|6|1|2054|617|8037|Tuesday|2054Q1|N|N|N|2471269|2471268|2470909|2471182|N|N|N|N|N| +2471275|AAAAAAAALGFLFCAA|2054-01-07|1848|8037|617|2054|3|1|7|1|2054|617|8037|Wednesday|2054Q1|N|N|N|2471269|2471268|2470910|2471183|N|N|N|N|N| +2471276|AAAAAAAAMGFLFCAA|2054-01-08|1848|8037|617|2054|4|1|8|1|2054|617|8037|Thursday|2054Q1|N|N|N|2471269|2471268|2470911|2471184|N|N|N|N|N| +2471277|AAAAAAAANGFLFCAA|2054-01-09|1848|8037|617|2054|5|1|9|1|2054|617|8037|Friday|2054Q1|N|Y|N|2471269|2471268|2470912|2471185|N|N|N|N|N| +2471278|AAAAAAAAOGFLFCAA|2054-01-10|1848|8037|617|2054|6|1|10|1|2054|617|8037|Saturday|2054Q1|N|Y|N|2471269|2471268|2470913|2471186|N|N|N|N|N| +2471279|AAAAAAAAPGFLFCAA|2054-01-11|1848|8037|617|2054|0|1|11|1|2054|617|8037|Sunday|2054Q1|N|N|N|2471269|2471268|2470914|2471187|N|N|N|N|N| +2471280|AAAAAAAAAHFLFCAA|2054-01-12|1848|8037|617|2054|1|1|12|1|2054|617|8037|Monday|2054Q1|N|N|N|2471269|2471268|2470915|2471188|N|N|N|N|N| +2471281|AAAAAAAABHFLFCAA|2054-01-13|1848|8038|617|2054|2|1|13|1|2054|617|8038|Tuesday|2054Q1|N|N|N|2471269|2471268|2470916|2471189|N|N|N|N|N| +2471282|AAAAAAAACHFLFCAA|2054-01-14|1848|8038|617|2054|3|1|14|1|2054|617|8038|Wednesday|2054Q1|N|N|N|2471269|2471268|2470917|2471190|N|N|N|N|N| +2471283|AAAAAAAADHFLFCAA|2054-01-15|1848|8038|617|2054|4|1|15|1|2054|617|8038|Thursday|2054Q1|N|N|N|2471269|2471268|2470918|2471191|N|N|N|N|N| +2471284|AAAAAAAAEHFLFCAA|2054-01-16|1848|8038|617|2054|5|1|16|1|2054|617|8038|Friday|2054Q1|N|Y|N|2471269|2471268|2470919|2471192|N|N|N|N|N| +2471285|AAAAAAAAFHFLFCAA|2054-01-17|1848|8038|617|2054|6|1|17|1|2054|617|8038|Saturday|2054Q1|N|Y|N|2471269|2471268|2470920|2471193|N|N|N|N|N| +2471286|AAAAAAAAGHFLFCAA|2054-01-18|1848|8038|617|2054|0|1|18|1|2054|617|8038|Sunday|2054Q1|N|N|N|2471269|2471268|2470921|2471194|N|N|N|N|N| +2471287|AAAAAAAAHHFLFCAA|2054-01-19|1848|8038|617|2054|1|1|19|1|2054|617|8038|Monday|2054Q1|N|N|N|2471269|2471268|2470922|2471195|N|N|N|N|N| +2471288|AAAAAAAAIHFLFCAA|2054-01-20|1848|8039|617|2054|2|1|20|1|2054|617|8039|Tuesday|2054Q1|N|N|N|2471269|2471268|2470923|2471196|N|N|N|N|N| +2471289|AAAAAAAAJHFLFCAA|2054-01-21|1848|8039|617|2054|3|1|21|1|2054|617|8039|Wednesday|2054Q1|N|N|N|2471269|2471268|2470924|2471197|N|N|N|N|N| +2471290|AAAAAAAAKHFLFCAA|2054-01-22|1848|8039|617|2054|4|1|22|1|2054|617|8039|Thursday|2054Q1|N|N|N|2471269|2471268|2470925|2471198|N|N|N|N|N| +2471291|AAAAAAAALHFLFCAA|2054-01-23|1848|8039|617|2054|5|1|23|1|2054|617|8039|Friday|2054Q1|N|Y|N|2471269|2471268|2470926|2471199|N|N|N|N|N| +2471292|AAAAAAAAMHFLFCAA|2054-01-24|1848|8039|617|2054|6|1|24|1|2054|617|8039|Saturday|2054Q1|N|Y|N|2471269|2471268|2470927|2471200|N|N|N|N|N| +2471293|AAAAAAAANHFLFCAA|2054-01-25|1848|8039|617|2054|0|1|25|1|2054|617|8039|Sunday|2054Q1|N|N|N|2471269|2471268|2470928|2471201|N|N|N|N|N| +2471294|AAAAAAAAOHFLFCAA|2054-01-26|1848|8039|617|2054|1|1|26|1|2054|617|8039|Monday|2054Q1|N|N|N|2471269|2471268|2470929|2471202|N|N|N|N|N| +2471295|AAAAAAAAPHFLFCAA|2054-01-27|1848|8040|617|2054|2|1|27|1|2054|617|8040|Tuesday|2054Q1|N|N|N|2471269|2471268|2470930|2471203|N|N|N|N|N| +2471296|AAAAAAAAAIFLFCAA|2054-01-28|1848|8040|617|2054|3|1|28|1|2054|617|8040|Wednesday|2054Q1|N|N|N|2471269|2471268|2470931|2471204|N|N|N|N|N| +2471297|AAAAAAAABIFLFCAA|2054-01-29|1848|8040|617|2054|4|1|29|1|2054|617|8040|Thursday|2054Q1|N|N|N|2471269|2471268|2470932|2471205|N|N|N|N|N| +2471298|AAAAAAAACIFLFCAA|2054-01-30|1848|8040|617|2054|5|1|30|1|2054|617|8040|Friday|2054Q1|N|Y|N|2471269|2471268|2470933|2471206|N|N|N|N|N| +2471299|AAAAAAAADIFLFCAA|2054-01-31|1848|8040|617|2054|6|1|31|1|2054|617|8040|Saturday|2054Q1|N|Y|N|2471269|2471268|2470934|2471207|N|N|N|N|N| +2471300|AAAAAAAAEIFLFCAA|2054-02-01|1849|8040|617|2054|0|2|1|1|2054|617|8040|Sunday|2054Q1|N|N|N|2471300|2471330|2470935|2471208|N|N|N|N|N| +2471301|AAAAAAAAFIFLFCAA|2054-02-02|1849|8040|617|2054|1|2|2|1|2054|617|8040|Monday|2054Q1|N|N|N|2471300|2471330|2470936|2471209|N|N|N|N|N| +2471302|AAAAAAAAGIFLFCAA|2054-02-03|1849|8041|617|2054|2|2|3|1|2054|617|8041|Tuesday|2054Q1|N|N|N|2471300|2471330|2470937|2471210|N|N|N|N|N| +2471303|AAAAAAAAHIFLFCAA|2054-02-04|1849|8041|617|2054|3|2|4|1|2054|617|8041|Wednesday|2054Q1|N|N|N|2471300|2471330|2470938|2471211|N|N|N|N|N| +2471304|AAAAAAAAIIFLFCAA|2054-02-05|1849|8041|617|2054|4|2|5|1|2054|617|8041|Thursday|2054Q1|N|N|N|2471300|2471330|2470939|2471212|N|N|N|N|N| +2471305|AAAAAAAAJIFLFCAA|2054-02-06|1849|8041|617|2054|5|2|6|1|2054|617|8041|Friday|2054Q1|N|Y|N|2471300|2471330|2470940|2471213|N|N|N|N|N| +2471306|AAAAAAAAKIFLFCAA|2054-02-07|1849|8041|617|2054|6|2|7|1|2054|617|8041|Saturday|2054Q1|N|Y|N|2471300|2471330|2470941|2471214|N|N|N|N|N| +2471307|AAAAAAAALIFLFCAA|2054-02-08|1849|8041|617|2054|0|2|8|1|2054|617|8041|Sunday|2054Q1|N|N|N|2471300|2471330|2470942|2471215|N|N|N|N|N| +2471308|AAAAAAAAMIFLFCAA|2054-02-09|1849|8041|617|2054|1|2|9|1|2054|617|8041|Monday|2054Q1|N|N|N|2471300|2471330|2470943|2471216|N|N|N|N|N| +2471309|AAAAAAAANIFLFCAA|2054-02-10|1849|8042|617|2054|2|2|10|1|2054|617|8042|Tuesday|2054Q1|N|N|N|2471300|2471330|2470944|2471217|N|N|N|N|N| +2471310|AAAAAAAAOIFLFCAA|2054-02-11|1849|8042|617|2054|3|2|11|1|2054|617|8042|Wednesday|2054Q1|N|N|N|2471300|2471330|2470945|2471218|N|N|N|N|N| +2471311|AAAAAAAAPIFLFCAA|2054-02-12|1849|8042|617|2054|4|2|12|1|2054|617|8042|Thursday|2054Q1|N|N|N|2471300|2471330|2470946|2471219|N|N|N|N|N| +2471312|AAAAAAAAAJFLFCAA|2054-02-13|1849|8042|617|2054|5|2|13|1|2054|617|8042|Friday|2054Q1|N|Y|N|2471300|2471330|2470947|2471220|N|N|N|N|N| +2471313|AAAAAAAABJFLFCAA|2054-02-14|1849|8042|617|2054|6|2|14|1|2054|617|8042|Saturday|2054Q1|N|Y|N|2471300|2471330|2470948|2471221|N|N|N|N|N| +2471314|AAAAAAAACJFLFCAA|2054-02-15|1849|8042|617|2054|0|2|15|1|2054|617|8042|Sunday|2054Q1|N|N|N|2471300|2471330|2470949|2471222|N|N|N|N|N| +2471315|AAAAAAAADJFLFCAA|2054-02-16|1849|8042|617|2054|1|2|16|1|2054|617|8042|Monday|2054Q1|N|N|N|2471300|2471330|2470950|2471223|N|N|N|N|N| +2471316|AAAAAAAAEJFLFCAA|2054-02-17|1849|8043|617|2054|2|2|17|1|2054|617|8043|Tuesday|2054Q1|N|N|N|2471300|2471330|2470951|2471224|N|N|N|N|N| +2471317|AAAAAAAAFJFLFCAA|2054-02-18|1849|8043|617|2054|3|2|18|1|2054|617|8043|Wednesday|2054Q1|N|N|N|2471300|2471330|2470952|2471225|N|N|N|N|N| +2471318|AAAAAAAAGJFLFCAA|2054-02-19|1849|8043|617|2054|4|2|19|1|2054|617|8043|Thursday|2054Q1|N|N|N|2471300|2471330|2470953|2471226|N|N|N|N|N| +2471319|AAAAAAAAHJFLFCAA|2054-02-20|1849|8043|617|2054|5|2|20|1|2054|617|8043|Friday|2054Q1|N|Y|N|2471300|2471330|2470954|2471227|N|N|N|N|N| +2471320|AAAAAAAAIJFLFCAA|2054-02-21|1849|8043|617|2054|6|2|21|1|2054|617|8043|Saturday|2054Q1|N|Y|N|2471300|2471330|2470955|2471228|N|N|N|N|N| +2471321|AAAAAAAAJJFLFCAA|2054-02-22|1849|8043|617|2054|0|2|22|1|2054|617|8043|Sunday|2054Q1|N|N|N|2471300|2471330|2470956|2471229|N|N|N|N|N| +2471322|AAAAAAAAKJFLFCAA|2054-02-23|1849|8043|617|2054|1|2|23|1|2054|617|8043|Monday|2054Q1|N|N|N|2471300|2471330|2470957|2471230|N|N|N|N|N| +2471323|AAAAAAAALJFLFCAA|2054-02-24|1849|8044|617|2054|2|2|24|1|2054|617|8044|Tuesday|2054Q1|N|N|N|2471300|2471330|2470958|2471231|N|N|N|N|N| +2471324|AAAAAAAAMJFLFCAA|2054-02-25|1849|8044|617|2054|3|2|25|1|2054|617|8044|Wednesday|2054Q1|N|N|N|2471300|2471330|2470959|2471232|N|N|N|N|N| +2471325|AAAAAAAANJFLFCAA|2054-02-26|1849|8044|617|2054|4|2|26|1|2054|617|8044|Thursday|2054Q1|N|N|N|2471300|2471330|2470960|2471233|N|N|N|N|N| +2471326|AAAAAAAAOJFLFCAA|2054-02-27|1849|8044|617|2054|5|2|27|1|2054|617|8044|Friday|2054Q1|N|Y|N|2471300|2471330|2470961|2471234|N|N|N|N|N| +2471327|AAAAAAAAPJFLFCAA|2054-02-28|1849|8044|617|2054|6|2|28|1|2054|617|8044|Saturday|2054Q1|N|Y|N|2471300|2471330|2470962|2471235|N|N|N|N|N| +2471328|AAAAAAAAAKFLFCAA|2054-03-01|1850|8044|618|2054|0|3|1|1|2054|618|8044|Sunday|2054Q1|N|N|N|2471328|2471386|2470963|2471236|N|N|N|N|N| +2471329|AAAAAAAABKFLFCAA|2054-03-02|1850|8044|618|2054|1|3|2|1|2054|618|8044|Monday|2054Q1|N|N|N|2471328|2471386|2470964|2471237|N|N|N|N|N| +2471330|AAAAAAAACKFLFCAA|2054-03-03|1850|8045|618|2054|2|3|3|1|2054|618|8045|Tuesday|2054Q1|N|N|N|2471328|2471386|2470965|2471238|N|N|N|N|N| +2471331|AAAAAAAADKFLFCAA|2054-03-04|1850|8045|618|2054|3|3|4|1|2054|618|8045|Wednesday|2054Q1|N|N|N|2471328|2471386|2470966|2471239|N|N|N|N|N| +2471332|AAAAAAAAEKFLFCAA|2054-03-05|1850|8045|618|2054|4|3|5|1|2054|618|8045|Thursday|2054Q1|N|N|N|2471328|2471386|2470967|2471240|N|N|N|N|N| +2471333|AAAAAAAAFKFLFCAA|2054-03-06|1850|8045|618|2054|5|3|6|1|2054|618|8045|Friday|2054Q1|N|Y|N|2471328|2471386|2470968|2471241|N|N|N|N|N| +2471334|AAAAAAAAGKFLFCAA|2054-03-07|1850|8045|618|2054|6|3|7|1|2054|618|8045|Saturday|2054Q1|N|Y|N|2471328|2471386|2470969|2471242|N|N|N|N|N| +2471335|AAAAAAAAHKFLFCAA|2054-03-08|1850|8045|618|2054|0|3|8|1|2054|618|8045|Sunday|2054Q1|N|N|N|2471328|2471386|2470970|2471243|N|N|N|N|N| +2471336|AAAAAAAAIKFLFCAA|2054-03-09|1850|8045|618|2054|1|3|9|1|2054|618|8045|Monday|2054Q1|N|N|N|2471328|2471386|2470971|2471244|N|N|N|N|N| +2471337|AAAAAAAAJKFLFCAA|2054-03-10|1850|8046|618|2054|2|3|10|1|2054|618|8046|Tuesday|2054Q1|N|N|N|2471328|2471386|2470972|2471245|N|N|N|N|N| +2471338|AAAAAAAAKKFLFCAA|2054-03-11|1850|8046|618|2054|3|3|11|1|2054|618|8046|Wednesday|2054Q1|N|N|N|2471328|2471386|2470973|2471246|N|N|N|N|N| +2471339|AAAAAAAALKFLFCAA|2054-03-12|1850|8046|618|2054|4|3|12|1|2054|618|8046|Thursday|2054Q1|N|N|N|2471328|2471386|2470974|2471247|N|N|N|N|N| +2471340|AAAAAAAAMKFLFCAA|2054-03-13|1850|8046|618|2054|5|3|13|1|2054|618|8046|Friday|2054Q1|N|Y|N|2471328|2471386|2470975|2471248|N|N|N|N|N| +2471341|AAAAAAAANKFLFCAA|2054-03-14|1850|8046|618|2054|6|3|14|1|2054|618|8046|Saturday|2054Q1|N|Y|N|2471328|2471386|2470976|2471249|N|N|N|N|N| +2471342|AAAAAAAAOKFLFCAA|2054-03-15|1850|8046|618|2054|0|3|15|1|2054|618|8046|Sunday|2054Q1|N|N|N|2471328|2471386|2470977|2471250|N|N|N|N|N| +2471343|AAAAAAAAPKFLFCAA|2054-03-16|1850|8046|618|2054|1|3|16|1|2054|618|8046|Monday|2054Q1|N|N|N|2471328|2471386|2470978|2471251|N|N|N|N|N| +2471344|AAAAAAAAALFLFCAA|2054-03-17|1850|8047|618|2054|2|3|17|1|2054|618|8047|Tuesday|2054Q1|N|N|N|2471328|2471386|2470979|2471252|N|N|N|N|N| +2471345|AAAAAAAABLFLFCAA|2054-03-18|1850|8047|618|2054|3|3|18|1|2054|618|8047|Wednesday|2054Q1|N|N|N|2471328|2471386|2470980|2471253|N|N|N|N|N| +2471346|AAAAAAAACLFLFCAA|2054-03-19|1850|8047|618|2054|4|3|19|1|2054|618|8047|Thursday|2054Q1|N|N|N|2471328|2471386|2470981|2471254|N|N|N|N|N| +2471347|AAAAAAAADLFLFCAA|2054-03-20|1850|8047|618|2054|5|3|20|1|2054|618|8047|Friday|2054Q1|N|Y|N|2471328|2471386|2470982|2471255|N|N|N|N|N| +2471348|AAAAAAAAELFLFCAA|2054-03-21|1850|8047|618|2054|6|3|21|1|2054|618|8047|Saturday|2054Q1|N|Y|N|2471328|2471386|2470983|2471256|N|N|N|N|N| +2471349|AAAAAAAAFLFLFCAA|2054-03-22|1850|8047|618|2054|0|3|22|1|2054|618|8047|Sunday|2054Q1|N|N|N|2471328|2471386|2470984|2471257|N|N|N|N|N| +2471350|AAAAAAAAGLFLFCAA|2054-03-23|1850|8047|618|2054|1|3|23|1|2054|618|8047|Monday|2054Q1|N|N|N|2471328|2471386|2470985|2471258|N|N|N|N|N| +2471351|AAAAAAAAHLFLFCAA|2054-03-24|1850|8048|618|2054|2|3|24|1|2054|618|8048|Tuesday|2054Q1|N|N|N|2471328|2471386|2470986|2471259|N|N|N|N|N| +2471352|AAAAAAAAILFLFCAA|2054-03-25|1850|8048|618|2054|3|3|25|1|2054|618|8048|Wednesday|2054Q1|N|N|N|2471328|2471386|2470987|2471260|N|N|N|N|N| +2471353|AAAAAAAAJLFLFCAA|2054-03-26|1850|8048|618|2054|4|3|26|1|2054|618|8048|Thursday|2054Q1|N|N|N|2471328|2471386|2470988|2471261|N|N|N|N|N| +2471354|AAAAAAAAKLFLFCAA|2054-03-27|1850|8048|618|2054|5|3|27|1|2054|618|8048|Friday|2054Q1|N|Y|N|2471328|2471386|2470989|2471262|N|N|N|N|N| +2471355|AAAAAAAALLFLFCAA|2054-03-28|1850|8048|618|2054|6|3|28|1|2054|618|8048|Saturday|2054Q1|N|Y|N|2471328|2471386|2470990|2471263|N|N|N|N|N| +2471356|AAAAAAAAMLFLFCAA|2054-03-29|1850|8048|618|2054|0|3|29|1|2054|618|8048|Sunday|2054Q1|N|N|N|2471328|2471386|2470991|2471264|N|N|N|N|N| +2471357|AAAAAAAANLFLFCAA|2054-03-30|1850|8048|618|2054|1|3|30|1|2054|618|8048|Monday|2054Q1|N|N|N|2471328|2471386|2470992|2471265|N|N|N|N|N| +2471358|AAAAAAAAOLFLFCAA|2054-03-31|1850|8049|618|2054|2|3|31|1|2054|618|8049|Tuesday|2054Q1|N|N|N|2471328|2471386|2470993|2471266|N|N|N|N|N| +2471359|AAAAAAAAPLFLFCAA|2054-04-01|1851|8049|618|2054|3|4|1|1|2054|618|8049|Wednesday|2054Q1|N|N|N|2471359|2471448|2470994|2471269|N|N|N|N|N| +2471360|AAAAAAAAAMFLFCAA|2054-04-02|1851|8049|618|2054|4|4|2|2|2054|618|8049|Thursday|2054Q2|N|N|N|2471359|2471448|2470995|2471270|N|N|N|N|N| +2471361|AAAAAAAABMFLFCAA|2054-04-03|1851|8049|618|2054|5|4|3|2|2054|618|8049|Friday|2054Q2|N|Y|N|2471359|2471448|2470996|2471271|N|N|N|N|N| +2471362|AAAAAAAACMFLFCAA|2054-04-04|1851|8049|618|2054|6|4|4|2|2054|618|8049|Saturday|2054Q2|N|Y|N|2471359|2471448|2470997|2471272|N|N|N|N|N| +2471363|AAAAAAAADMFLFCAA|2054-04-05|1851|8049|618|2054|0|4|5|2|2054|618|8049|Sunday|2054Q2|N|N|N|2471359|2471448|2470998|2471273|N|N|N|N|N| +2471364|AAAAAAAAEMFLFCAA|2054-04-06|1851|8049|618|2054|1|4|6|2|2054|618|8049|Monday|2054Q2|N|N|N|2471359|2471448|2470999|2471274|N|N|N|N|N| +2471365|AAAAAAAAFMFLFCAA|2054-04-07|1851|8050|618|2054|2|4|7|2|2054|618|8050|Tuesday|2054Q2|N|N|N|2471359|2471448|2471000|2471275|N|N|N|N|N| +2471366|AAAAAAAAGMFLFCAA|2054-04-08|1851|8050|618|2054|3|4|8|2|2054|618|8050|Wednesday|2054Q2|N|N|N|2471359|2471448|2471001|2471276|N|N|N|N|N| +2471367|AAAAAAAAHMFLFCAA|2054-04-09|1851|8050|618|2054|4|4|9|2|2054|618|8050|Thursday|2054Q2|N|N|N|2471359|2471448|2471002|2471277|N|N|N|N|N| +2471368|AAAAAAAAIMFLFCAA|2054-04-10|1851|8050|618|2054|5|4|10|2|2054|618|8050|Friday|2054Q2|N|Y|N|2471359|2471448|2471003|2471278|N|N|N|N|N| +2471369|AAAAAAAAJMFLFCAA|2054-04-11|1851|8050|618|2054|6|4|11|2|2054|618|8050|Saturday|2054Q2|N|Y|N|2471359|2471448|2471004|2471279|N|N|N|N|N| +2471370|AAAAAAAAKMFLFCAA|2054-04-12|1851|8050|618|2054|0|4|12|2|2054|618|8050|Sunday|2054Q2|N|N|N|2471359|2471448|2471005|2471280|N|N|N|N|N| +2471371|AAAAAAAALMFLFCAA|2054-04-13|1851|8050|618|2054|1|4|13|2|2054|618|8050|Monday|2054Q2|N|N|N|2471359|2471448|2471006|2471281|N|N|N|N|N| +2471372|AAAAAAAAMMFLFCAA|2054-04-14|1851|8051|618|2054|2|4|14|2|2054|618|8051|Tuesday|2054Q2|N|N|N|2471359|2471448|2471007|2471282|N|N|N|N|N| +2471373|AAAAAAAANMFLFCAA|2054-04-15|1851|8051|618|2054|3|4|15|2|2054|618|8051|Wednesday|2054Q2|N|N|N|2471359|2471448|2471008|2471283|N|N|N|N|N| +2471374|AAAAAAAAOMFLFCAA|2054-04-16|1851|8051|618|2054|4|4|16|2|2054|618|8051|Thursday|2054Q2|N|N|N|2471359|2471448|2471009|2471284|N|N|N|N|N| +2471375|AAAAAAAAPMFLFCAA|2054-04-17|1851|8051|618|2054|5|4|17|2|2054|618|8051|Friday|2054Q2|N|Y|N|2471359|2471448|2471010|2471285|N|N|N|N|N| +2471376|AAAAAAAAANFLFCAA|2054-04-18|1851|8051|618|2054|6|4|18|2|2054|618|8051|Saturday|2054Q2|N|Y|N|2471359|2471448|2471011|2471286|N|N|N|N|N| +2471377|AAAAAAAABNFLFCAA|2054-04-19|1851|8051|618|2054|0|4|19|2|2054|618|8051|Sunday|2054Q2|N|N|N|2471359|2471448|2471012|2471287|N|N|N|N|N| +2471378|AAAAAAAACNFLFCAA|2054-04-20|1851|8051|618|2054|1|4|20|2|2054|618|8051|Monday|2054Q2|N|N|N|2471359|2471448|2471013|2471288|N|N|N|N|N| +2471379|AAAAAAAADNFLFCAA|2054-04-21|1851|8052|618|2054|2|4|21|2|2054|618|8052|Tuesday|2054Q2|N|N|N|2471359|2471448|2471014|2471289|N|N|N|N|N| +2471380|AAAAAAAAENFLFCAA|2054-04-22|1851|8052|618|2054|3|4|22|2|2054|618|8052|Wednesday|2054Q2|N|N|N|2471359|2471448|2471015|2471290|N|N|N|N|N| +2471381|AAAAAAAAFNFLFCAA|2054-04-23|1851|8052|618|2054|4|4|23|2|2054|618|8052|Thursday|2054Q2|N|N|N|2471359|2471448|2471016|2471291|N|N|N|N|N| +2471382|AAAAAAAAGNFLFCAA|2054-04-24|1851|8052|618|2054|5|4|24|2|2054|618|8052|Friday|2054Q2|N|Y|N|2471359|2471448|2471017|2471292|N|N|N|N|N| +2471383|AAAAAAAAHNFLFCAA|2054-04-25|1851|8052|618|2054|6|4|25|2|2054|618|8052|Saturday|2054Q2|N|Y|N|2471359|2471448|2471018|2471293|N|N|N|N|N| +2471384|AAAAAAAAINFLFCAA|2054-04-26|1851|8052|618|2054|0|4|26|2|2054|618|8052|Sunday|2054Q2|N|N|N|2471359|2471448|2471019|2471294|N|N|N|N|N| +2471385|AAAAAAAAJNFLFCAA|2054-04-27|1851|8052|618|2054|1|4|27|2|2054|618|8052|Monday|2054Q2|N|N|N|2471359|2471448|2471020|2471295|N|N|N|N|N| +2471386|AAAAAAAAKNFLFCAA|2054-04-28|1851|8053|618|2054|2|4|28|2|2054|618|8053|Tuesday|2054Q2|N|N|N|2471359|2471448|2471021|2471296|N|N|N|N|N| +2471387|AAAAAAAALNFLFCAA|2054-04-29|1851|8053|618|2054|3|4|29|2|2054|618|8053|Wednesday|2054Q2|N|N|N|2471359|2471448|2471022|2471297|N|N|N|N|N| +2471388|AAAAAAAAMNFLFCAA|2054-04-30|1851|8053|618|2054|4|4|30|2|2054|618|8053|Thursday|2054Q2|N|N|N|2471359|2471448|2471023|2471298|N|N|N|N|N| +2471389|AAAAAAAANNFLFCAA|2054-05-01|1852|8053|618|2054|5|5|1|2|2054|618|8053|Friday|2054Q2|N|Y|N|2471389|2471508|2471024|2471299|N|N|N|N|N| +2471390|AAAAAAAAONFLFCAA|2054-05-02|1852|8053|618|2054|6|5|2|2|2054|618|8053|Saturday|2054Q2|N|Y|N|2471389|2471508|2471025|2471300|N|N|N|N|N| +2471391|AAAAAAAAPNFLFCAA|2054-05-03|1852|8053|618|2054|0|5|3|2|2054|618|8053|Sunday|2054Q2|N|N|N|2471389|2471508|2471026|2471301|N|N|N|N|N| +2471392|AAAAAAAAAOFLFCAA|2054-05-04|1852|8053|618|2054|1|5|4|2|2054|618|8053|Monday|2054Q2|N|N|N|2471389|2471508|2471027|2471302|N|N|N|N|N| +2471393|AAAAAAAABOFLFCAA|2054-05-05|1852|8054|618|2054|2|5|5|2|2054|618|8054|Tuesday|2054Q2|N|N|N|2471389|2471508|2471028|2471303|N|N|N|N|N| +2471394|AAAAAAAACOFLFCAA|2054-05-06|1852|8054|618|2054|3|5|6|2|2054|618|8054|Wednesday|2054Q2|N|N|N|2471389|2471508|2471029|2471304|N|N|N|N|N| +2471395|AAAAAAAADOFLFCAA|2054-05-07|1852|8054|618|2054|4|5|7|2|2054|618|8054|Thursday|2054Q2|N|N|N|2471389|2471508|2471030|2471305|N|N|N|N|N| +2471396|AAAAAAAAEOFLFCAA|2054-05-08|1852|8054|618|2054|5|5|8|2|2054|618|8054|Friday|2054Q2|N|Y|N|2471389|2471508|2471031|2471306|N|N|N|N|N| +2471397|AAAAAAAAFOFLFCAA|2054-05-09|1852|8054|618|2054|6|5|9|2|2054|618|8054|Saturday|2054Q2|N|Y|N|2471389|2471508|2471032|2471307|N|N|N|N|N| +2471398|AAAAAAAAGOFLFCAA|2054-05-10|1852|8054|618|2054|0|5|10|2|2054|618|8054|Sunday|2054Q2|N|N|N|2471389|2471508|2471033|2471308|N|N|N|N|N| +2471399|AAAAAAAAHOFLFCAA|2054-05-11|1852|8054|618|2054|1|5|11|2|2054|618|8054|Monday|2054Q2|N|N|N|2471389|2471508|2471034|2471309|N|N|N|N|N| +2471400|AAAAAAAAIOFLFCAA|2054-05-12|1852|8055|618|2054|2|5|12|2|2054|618|8055|Tuesday|2054Q2|N|N|N|2471389|2471508|2471035|2471310|N|N|N|N|N| +2471401|AAAAAAAAJOFLFCAA|2054-05-13|1852|8055|618|2054|3|5|13|2|2054|618|8055|Wednesday|2054Q2|N|N|N|2471389|2471508|2471036|2471311|N|N|N|N|N| +2471402|AAAAAAAAKOFLFCAA|2054-05-14|1852|8055|618|2054|4|5|14|2|2054|618|8055|Thursday|2054Q2|N|N|N|2471389|2471508|2471037|2471312|N|N|N|N|N| +2471403|AAAAAAAALOFLFCAA|2054-05-15|1852|8055|618|2054|5|5|15|2|2054|618|8055|Friday|2054Q2|N|Y|N|2471389|2471508|2471038|2471313|N|N|N|N|N| +2471404|AAAAAAAAMOFLFCAA|2054-05-16|1852|8055|618|2054|6|5|16|2|2054|618|8055|Saturday|2054Q2|N|Y|N|2471389|2471508|2471039|2471314|N|N|N|N|N| +2471405|AAAAAAAANOFLFCAA|2054-05-17|1852|8055|618|2054|0|5|17|2|2054|618|8055|Sunday|2054Q2|N|N|N|2471389|2471508|2471040|2471315|N|N|N|N|N| +2471406|AAAAAAAAOOFLFCAA|2054-05-18|1852|8055|618|2054|1|5|18|2|2054|618|8055|Monday|2054Q2|N|N|N|2471389|2471508|2471041|2471316|N|N|N|N|N| +2471407|AAAAAAAAPOFLFCAA|2054-05-19|1852|8056|618|2054|2|5|19|2|2054|618|8056|Tuesday|2054Q2|N|N|N|2471389|2471508|2471042|2471317|N|N|N|N|N| +2471408|AAAAAAAAAPFLFCAA|2054-05-20|1852|8056|618|2054|3|5|20|2|2054|618|8056|Wednesday|2054Q2|N|N|N|2471389|2471508|2471043|2471318|N|N|N|N|N| +2471409|AAAAAAAABPFLFCAA|2054-05-21|1852|8056|618|2054|4|5|21|2|2054|618|8056|Thursday|2054Q2|N|N|N|2471389|2471508|2471044|2471319|N|N|N|N|N| +2471410|AAAAAAAACPFLFCAA|2054-05-22|1852|8056|618|2054|5|5|22|2|2054|618|8056|Friday|2054Q2|N|Y|N|2471389|2471508|2471045|2471320|N|N|N|N|N| +2471411|AAAAAAAADPFLFCAA|2054-05-23|1852|8056|618|2054|6|5|23|2|2054|618|8056|Saturday|2054Q2|N|Y|N|2471389|2471508|2471046|2471321|N|N|N|N|N| +2471412|AAAAAAAAEPFLFCAA|2054-05-24|1852|8056|618|2054|0|5|24|2|2054|618|8056|Sunday|2054Q2|N|N|N|2471389|2471508|2471047|2471322|N|N|N|N|N| +2471413|AAAAAAAAFPFLFCAA|2054-05-25|1852|8056|618|2054|1|5|25|2|2054|618|8056|Monday|2054Q2|N|N|N|2471389|2471508|2471048|2471323|N|N|N|N|N| +2471414|AAAAAAAAGPFLFCAA|2054-05-26|1852|8057|618|2054|2|5|26|2|2054|618|8057|Tuesday|2054Q2|N|N|N|2471389|2471508|2471049|2471324|N|N|N|N|N| +2471415|AAAAAAAAHPFLFCAA|2054-05-27|1852|8057|618|2054|3|5|27|2|2054|618|8057|Wednesday|2054Q2|N|N|N|2471389|2471508|2471050|2471325|N|N|N|N|N| +2471416|AAAAAAAAIPFLFCAA|2054-05-28|1852|8057|618|2054|4|5|28|2|2054|618|8057|Thursday|2054Q2|N|N|N|2471389|2471508|2471051|2471326|N|N|N|N|N| +2471417|AAAAAAAAJPFLFCAA|2054-05-29|1852|8057|618|2054|5|5|29|2|2054|618|8057|Friday|2054Q2|N|Y|N|2471389|2471508|2471052|2471327|N|N|N|N|N| +2471418|AAAAAAAAKPFLFCAA|2054-05-30|1852|8057|618|2054|6|5|30|2|2054|618|8057|Saturday|2054Q2|N|Y|N|2471389|2471508|2471053|2471328|N|N|N|N|N| +2471419|AAAAAAAALPFLFCAA|2054-05-31|1852|8057|618|2054|0|5|31|2|2054|618|8057|Sunday|2054Q2|N|N|N|2471389|2471508|2471054|2471329|N|N|N|N|N| +2471420|AAAAAAAAMPFLFCAA|2054-06-01|1853|8057|619|2054|1|6|1|2|2054|619|8057|Monday|2054Q2|N|N|N|2471420|2471570|2471055|2471330|N|N|N|N|N| +2471421|AAAAAAAANPFLFCAA|2054-06-02|1853|8058|619|2054|2|6|2|2|2054|619|8058|Tuesday|2054Q2|N|N|N|2471420|2471570|2471056|2471331|N|N|N|N|N| +2471422|AAAAAAAAOPFLFCAA|2054-06-03|1853|8058|619|2054|3|6|3|2|2054|619|8058|Wednesday|2054Q2|N|N|N|2471420|2471570|2471057|2471332|N|N|N|N|N| +2471423|AAAAAAAAPPFLFCAA|2054-06-04|1853|8058|619|2054|4|6|4|2|2054|619|8058|Thursday|2054Q2|N|N|N|2471420|2471570|2471058|2471333|N|N|N|N|N| +2471424|AAAAAAAAAAGLFCAA|2054-06-05|1853|8058|619|2054|5|6|5|2|2054|619|8058|Friday|2054Q2|N|Y|N|2471420|2471570|2471059|2471334|N|N|N|N|N| +2471425|AAAAAAAABAGLFCAA|2054-06-06|1853|8058|619|2054|6|6|6|2|2054|619|8058|Saturday|2054Q2|N|Y|N|2471420|2471570|2471060|2471335|N|N|N|N|N| +2471426|AAAAAAAACAGLFCAA|2054-06-07|1853|8058|619|2054|0|6|7|2|2054|619|8058|Sunday|2054Q2|N|N|N|2471420|2471570|2471061|2471336|N|N|N|N|N| +2471427|AAAAAAAADAGLFCAA|2054-06-08|1853|8058|619|2054|1|6|8|2|2054|619|8058|Monday|2054Q2|N|N|N|2471420|2471570|2471062|2471337|N|N|N|N|N| +2471428|AAAAAAAAEAGLFCAA|2054-06-09|1853|8059|619|2054|2|6|9|2|2054|619|8059|Tuesday|2054Q2|N|N|N|2471420|2471570|2471063|2471338|N|N|N|N|N| +2471429|AAAAAAAAFAGLFCAA|2054-06-10|1853|8059|619|2054|3|6|10|2|2054|619|8059|Wednesday|2054Q2|N|N|N|2471420|2471570|2471064|2471339|N|N|N|N|N| +2471430|AAAAAAAAGAGLFCAA|2054-06-11|1853|8059|619|2054|4|6|11|2|2054|619|8059|Thursday|2054Q2|N|N|N|2471420|2471570|2471065|2471340|N|N|N|N|N| +2471431|AAAAAAAAHAGLFCAA|2054-06-12|1853|8059|619|2054|5|6|12|2|2054|619|8059|Friday|2054Q2|N|Y|N|2471420|2471570|2471066|2471341|N|N|N|N|N| +2471432|AAAAAAAAIAGLFCAA|2054-06-13|1853|8059|619|2054|6|6|13|2|2054|619|8059|Saturday|2054Q2|N|Y|N|2471420|2471570|2471067|2471342|N|N|N|N|N| +2471433|AAAAAAAAJAGLFCAA|2054-06-14|1853|8059|619|2054|0|6|14|2|2054|619|8059|Sunday|2054Q2|N|N|N|2471420|2471570|2471068|2471343|N|N|N|N|N| +2471434|AAAAAAAAKAGLFCAA|2054-06-15|1853|8059|619|2054|1|6|15|2|2054|619|8059|Monday|2054Q2|N|N|N|2471420|2471570|2471069|2471344|N|N|N|N|N| +2471435|AAAAAAAALAGLFCAA|2054-06-16|1853|8060|619|2054|2|6|16|2|2054|619|8060|Tuesday|2054Q2|N|N|N|2471420|2471570|2471070|2471345|N|N|N|N|N| +2471436|AAAAAAAAMAGLFCAA|2054-06-17|1853|8060|619|2054|3|6|17|2|2054|619|8060|Wednesday|2054Q2|N|N|N|2471420|2471570|2471071|2471346|N|N|N|N|N| +2471437|AAAAAAAANAGLFCAA|2054-06-18|1853|8060|619|2054|4|6|18|2|2054|619|8060|Thursday|2054Q2|N|N|N|2471420|2471570|2471072|2471347|N|N|N|N|N| +2471438|AAAAAAAAOAGLFCAA|2054-06-19|1853|8060|619|2054|5|6|19|2|2054|619|8060|Friday|2054Q2|N|Y|N|2471420|2471570|2471073|2471348|N|N|N|N|N| +2471439|AAAAAAAAPAGLFCAA|2054-06-20|1853|8060|619|2054|6|6|20|2|2054|619|8060|Saturday|2054Q2|N|Y|N|2471420|2471570|2471074|2471349|N|N|N|N|N| +2471440|AAAAAAAAABGLFCAA|2054-06-21|1853|8060|619|2054|0|6|21|2|2054|619|8060|Sunday|2054Q2|N|N|N|2471420|2471570|2471075|2471350|N|N|N|N|N| +2471441|AAAAAAAABBGLFCAA|2054-06-22|1853|8060|619|2054|1|6|22|2|2054|619|8060|Monday|2054Q2|N|N|N|2471420|2471570|2471076|2471351|N|N|N|N|N| +2471442|AAAAAAAACBGLFCAA|2054-06-23|1853|8061|619|2054|2|6|23|2|2054|619|8061|Tuesday|2054Q2|N|N|N|2471420|2471570|2471077|2471352|N|N|N|N|N| +2471443|AAAAAAAADBGLFCAA|2054-06-24|1853|8061|619|2054|3|6|24|2|2054|619|8061|Wednesday|2054Q2|N|N|N|2471420|2471570|2471078|2471353|N|N|N|N|N| +2471444|AAAAAAAAEBGLFCAA|2054-06-25|1853|8061|619|2054|4|6|25|2|2054|619|8061|Thursday|2054Q2|N|N|N|2471420|2471570|2471079|2471354|N|N|N|N|N| +2471445|AAAAAAAAFBGLFCAA|2054-06-26|1853|8061|619|2054|5|6|26|2|2054|619|8061|Friday|2054Q2|N|Y|N|2471420|2471570|2471080|2471355|N|N|N|N|N| +2471446|AAAAAAAAGBGLFCAA|2054-06-27|1853|8061|619|2054|6|6|27|2|2054|619|8061|Saturday|2054Q2|N|Y|N|2471420|2471570|2471081|2471356|N|N|N|N|N| +2471447|AAAAAAAAHBGLFCAA|2054-06-28|1853|8061|619|2054|0|6|28|2|2054|619|8061|Sunday|2054Q2|N|N|N|2471420|2471570|2471082|2471357|N|N|N|N|N| +2471448|AAAAAAAAIBGLFCAA|2054-06-29|1853|8061|619|2054|1|6|29|2|2054|619|8061|Monday|2054Q2|N|N|N|2471420|2471570|2471083|2471358|N|N|N|N|N| +2471449|AAAAAAAAJBGLFCAA|2054-06-30|1853|8062|619|2054|2|6|30|2|2054|619|8062|Tuesday|2054Q2|N|N|N|2471420|2471570|2471084|2471359|N|N|N|N|N| +2471450|AAAAAAAAKBGLFCAA|2054-07-01|1854|8062|619|2054|3|7|1|2|2054|619|8062|Wednesday|2054Q2|N|N|N|2471450|2471630|2471085|2471359|N|N|N|N|N| +2471451|AAAAAAAALBGLFCAA|2054-07-02|1854|8062|619|2054|4|7|2|3|2054|619|8062|Thursday|2054Q3|N|N|N|2471450|2471630|2471086|2471360|N|N|N|N|N| +2471452|AAAAAAAAMBGLFCAA|2054-07-03|1854|8062|619|2054|5|7|3|3|2054|619|8062|Friday|2054Q3|N|Y|N|2471450|2471630|2471087|2471361|N|N|N|N|N| +2471453|AAAAAAAANBGLFCAA|2054-07-04|1854|8062|619|2054|6|7|4|3|2054|619|8062|Saturday|2054Q3|N|Y|N|2471450|2471630|2471088|2471362|N|N|N|N|N| +2471454|AAAAAAAAOBGLFCAA|2054-07-05|1854|8062|619|2054|0|7|5|3|2054|619|8062|Sunday|2054Q3|Y|N|N|2471450|2471630|2471089|2471363|N|N|N|N|N| +2471455|AAAAAAAAPBGLFCAA|2054-07-06|1854|8062|619|2054|1|7|6|3|2054|619|8062|Monday|2054Q3|N|N|Y|2471450|2471630|2471090|2471364|N|N|N|N|N| +2471456|AAAAAAAAACGLFCAA|2054-07-07|1854|8063|619|2054|2|7|7|3|2054|619|8063|Tuesday|2054Q3|N|N|N|2471450|2471630|2471091|2471365|N|N|N|N|N| +2471457|AAAAAAAABCGLFCAA|2054-07-08|1854|8063|619|2054|3|7|8|3|2054|619|8063|Wednesday|2054Q3|N|N|N|2471450|2471630|2471092|2471366|N|N|N|N|N| +2471458|AAAAAAAACCGLFCAA|2054-07-09|1854|8063|619|2054|4|7|9|3|2054|619|8063|Thursday|2054Q3|N|N|N|2471450|2471630|2471093|2471367|N|N|N|N|N| +2471459|AAAAAAAADCGLFCAA|2054-07-10|1854|8063|619|2054|5|7|10|3|2054|619|8063|Friday|2054Q3|N|Y|N|2471450|2471630|2471094|2471368|N|N|N|N|N| +2471460|AAAAAAAAECGLFCAA|2054-07-11|1854|8063|619|2054|6|7|11|3|2054|619|8063|Saturday|2054Q3|N|Y|N|2471450|2471630|2471095|2471369|N|N|N|N|N| +2471461|AAAAAAAAFCGLFCAA|2054-07-12|1854|8063|619|2054|0|7|12|3|2054|619|8063|Sunday|2054Q3|N|N|N|2471450|2471630|2471096|2471370|N|N|N|N|N| +2471462|AAAAAAAAGCGLFCAA|2054-07-13|1854|8063|619|2054|1|7|13|3|2054|619|8063|Monday|2054Q3|N|N|N|2471450|2471630|2471097|2471371|N|N|N|N|N| +2471463|AAAAAAAAHCGLFCAA|2054-07-14|1854|8064|619|2054|2|7|14|3|2054|619|8064|Tuesday|2054Q3|N|N|N|2471450|2471630|2471098|2471372|N|N|N|N|N| +2471464|AAAAAAAAICGLFCAA|2054-07-15|1854|8064|619|2054|3|7|15|3|2054|619|8064|Wednesday|2054Q3|N|N|N|2471450|2471630|2471099|2471373|N|N|N|N|N| +2471465|AAAAAAAAJCGLFCAA|2054-07-16|1854|8064|619|2054|4|7|16|3|2054|619|8064|Thursday|2054Q3|N|N|N|2471450|2471630|2471100|2471374|N|N|N|N|N| +2471466|AAAAAAAAKCGLFCAA|2054-07-17|1854|8064|619|2054|5|7|17|3|2054|619|8064|Friday|2054Q3|N|Y|N|2471450|2471630|2471101|2471375|N|N|N|N|N| +2471467|AAAAAAAALCGLFCAA|2054-07-18|1854|8064|619|2054|6|7|18|3|2054|619|8064|Saturday|2054Q3|N|Y|N|2471450|2471630|2471102|2471376|N|N|N|N|N| +2471468|AAAAAAAAMCGLFCAA|2054-07-19|1854|8064|619|2054|0|7|19|3|2054|619|8064|Sunday|2054Q3|N|N|N|2471450|2471630|2471103|2471377|N|N|N|N|N| +2471469|AAAAAAAANCGLFCAA|2054-07-20|1854|8064|619|2054|1|7|20|3|2054|619|8064|Monday|2054Q3|N|N|N|2471450|2471630|2471104|2471378|N|N|N|N|N| +2471470|AAAAAAAAOCGLFCAA|2054-07-21|1854|8065|619|2054|2|7|21|3|2054|619|8065|Tuesday|2054Q3|N|N|N|2471450|2471630|2471105|2471379|N|N|N|N|N| +2471471|AAAAAAAAPCGLFCAA|2054-07-22|1854|8065|619|2054|3|7|22|3|2054|619|8065|Wednesday|2054Q3|N|N|N|2471450|2471630|2471106|2471380|N|N|N|N|N| +2471472|AAAAAAAAADGLFCAA|2054-07-23|1854|8065|619|2054|4|7|23|3|2054|619|8065|Thursday|2054Q3|N|N|N|2471450|2471630|2471107|2471381|N|N|N|N|N| +2471473|AAAAAAAABDGLFCAA|2054-07-24|1854|8065|619|2054|5|7|24|3|2054|619|8065|Friday|2054Q3|N|Y|N|2471450|2471630|2471108|2471382|N|N|N|N|N| +2471474|AAAAAAAACDGLFCAA|2054-07-25|1854|8065|619|2054|6|7|25|3|2054|619|8065|Saturday|2054Q3|N|Y|N|2471450|2471630|2471109|2471383|N|N|N|N|N| +2471475|AAAAAAAADDGLFCAA|2054-07-26|1854|8065|619|2054|0|7|26|3|2054|619|8065|Sunday|2054Q3|N|N|N|2471450|2471630|2471110|2471384|N|N|N|N|N| +2471476|AAAAAAAAEDGLFCAA|2054-07-27|1854|8065|619|2054|1|7|27|3|2054|619|8065|Monday|2054Q3|N|N|N|2471450|2471630|2471111|2471385|N|N|N|N|N| +2471477|AAAAAAAAFDGLFCAA|2054-07-28|1854|8066|619|2054|2|7|28|3|2054|619|8066|Tuesday|2054Q3|N|N|N|2471450|2471630|2471112|2471386|N|N|N|N|N| +2471478|AAAAAAAAGDGLFCAA|2054-07-29|1854|8066|619|2054|3|7|29|3|2054|619|8066|Wednesday|2054Q3|N|N|N|2471450|2471630|2471113|2471387|N|N|N|N|N| +2471479|AAAAAAAAHDGLFCAA|2054-07-30|1854|8066|619|2054|4|7|30|3|2054|619|8066|Thursday|2054Q3|N|N|N|2471450|2471630|2471114|2471388|N|N|N|N|N| +2471480|AAAAAAAAIDGLFCAA|2054-07-31|1854|8066|619|2054|5|7|31|3|2054|619|8066|Friday|2054Q3|N|Y|N|2471450|2471630|2471115|2471389|N|N|N|N|N| +2471481|AAAAAAAAJDGLFCAA|2054-08-01|1855|8066|619|2054|6|8|1|3|2054|619|8066|Saturday|2054Q3|N|Y|N|2471481|2471692|2471116|2471390|N|N|N|N|N| +2471482|AAAAAAAAKDGLFCAA|2054-08-02|1855|8066|619|2054|0|8|2|3|2054|619|8066|Sunday|2054Q3|N|N|N|2471481|2471692|2471117|2471391|N|N|N|N|N| +2471483|AAAAAAAALDGLFCAA|2054-08-03|1855|8066|619|2054|1|8|3|3|2054|619|8066|Monday|2054Q3|N|N|N|2471481|2471692|2471118|2471392|N|N|N|N|N| +2471484|AAAAAAAAMDGLFCAA|2054-08-04|1855|8067|619|2054|2|8|4|3|2054|619|8067|Tuesday|2054Q3|N|N|N|2471481|2471692|2471119|2471393|N|N|N|N|N| +2471485|AAAAAAAANDGLFCAA|2054-08-05|1855|8067|619|2054|3|8|5|3|2054|619|8067|Wednesday|2054Q3|N|N|N|2471481|2471692|2471120|2471394|N|N|N|N|N| +2471486|AAAAAAAAODGLFCAA|2054-08-06|1855|8067|619|2054|4|8|6|3|2054|619|8067|Thursday|2054Q3|N|N|N|2471481|2471692|2471121|2471395|N|N|N|N|N| +2471487|AAAAAAAAPDGLFCAA|2054-08-07|1855|8067|619|2054|5|8|7|3|2054|619|8067|Friday|2054Q3|N|Y|N|2471481|2471692|2471122|2471396|N|N|N|N|N| +2471488|AAAAAAAAAEGLFCAA|2054-08-08|1855|8067|619|2054|6|8|8|3|2054|619|8067|Saturday|2054Q3|N|Y|N|2471481|2471692|2471123|2471397|N|N|N|N|N| +2471489|AAAAAAAABEGLFCAA|2054-08-09|1855|8067|619|2054|0|8|9|3|2054|619|8067|Sunday|2054Q3|N|N|N|2471481|2471692|2471124|2471398|N|N|N|N|N| +2471490|AAAAAAAACEGLFCAA|2054-08-10|1855|8067|619|2054|1|8|10|3|2054|619|8067|Monday|2054Q3|N|N|N|2471481|2471692|2471125|2471399|N|N|N|N|N| +2471491|AAAAAAAADEGLFCAA|2054-08-11|1855|8068|619|2054|2|8|11|3|2054|619|8068|Tuesday|2054Q3|N|N|N|2471481|2471692|2471126|2471400|N|N|N|N|N| +2471492|AAAAAAAAEEGLFCAA|2054-08-12|1855|8068|619|2054|3|8|12|3|2054|619|8068|Wednesday|2054Q3|N|N|N|2471481|2471692|2471127|2471401|N|N|N|N|N| +2471493|AAAAAAAAFEGLFCAA|2054-08-13|1855|8068|619|2054|4|8|13|3|2054|619|8068|Thursday|2054Q3|N|N|N|2471481|2471692|2471128|2471402|N|N|N|N|N| +2471494|AAAAAAAAGEGLFCAA|2054-08-14|1855|8068|619|2054|5|8|14|3|2054|619|8068|Friday|2054Q3|N|Y|N|2471481|2471692|2471129|2471403|N|N|N|N|N| +2471495|AAAAAAAAHEGLFCAA|2054-08-15|1855|8068|619|2054|6|8|15|3|2054|619|8068|Saturday|2054Q3|N|Y|N|2471481|2471692|2471130|2471404|N|N|N|N|N| +2471496|AAAAAAAAIEGLFCAA|2054-08-16|1855|8068|619|2054|0|8|16|3|2054|619|8068|Sunday|2054Q3|N|N|N|2471481|2471692|2471131|2471405|N|N|N|N|N| +2471497|AAAAAAAAJEGLFCAA|2054-08-17|1855|8068|619|2054|1|8|17|3|2054|619|8068|Monday|2054Q3|N|N|N|2471481|2471692|2471132|2471406|N|N|N|N|N| +2471498|AAAAAAAAKEGLFCAA|2054-08-18|1855|8069|619|2054|2|8|18|3|2054|619|8069|Tuesday|2054Q3|N|N|N|2471481|2471692|2471133|2471407|N|N|N|N|N| +2471499|AAAAAAAALEGLFCAA|2054-08-19|1855|8069|619|2054|3|8|19|3|2054|619|8069|Wednesday|2054Q3|N|N|N|2471481|2471692|2471134|2471408|N|N|N|N|N| +2471500|AAAAAAAAMEGLFCAA|2054-08-20|1855|8069|619|2054|4|8|20|3|2054|619|8069|Thursday|2054Q3|N|N|N|2471481|2471692|2471135|2471409|N|N|N|N|N| +2471501|AAAAAAAANEGLFCAA|2054-08-21|1855|8069|619|2054|5|8|21|3|2054|619|8069|Friday|2054Q3|N|Y|N|2471481|2471692|2471136|2471410|N|N|N|N|N| +2471502|AAAAAAAAOEGLFCAA|2054-08-22|1855|8069|619|2054|6|8|22|3|2054|619|8069|Saturday|2054Q3|N|Y|N|2471481|2471692|2471137|2471411|N|N|N|N|N| +2471503|AAAAAAAAPEGLFCAA|2054-08-23|1855|8069|619|2054|0|8|23|3|2054|619|8069|Sunday|2054Q3|N|N|N|2471481|2471692|2471138|2471412|N|N|N|N|N| +2471504|AAAAAAAAAFGLFCAA|2054-08-24|1855|8069|619|2054|1|8|24|3|2054|619|8069|Monday|2054Q3|N|N|N|2471481|2471692|2471139|2471413|N|N|N|N|N| +2471505|AAAAAAAABFGLFCAA|2054-08-25|1855|8070|619|2054|2|8|25|3|2054|619|8070|Tuesday|2054Q3|N|N|N|2471481|2471692|2471140|2471414|N|N|N|N|N| +2471506|AAAAAAAACFGLFCAA|2054-08-26|1855|8070|619|2054|3|8|26|3|2054|619|8070|Wednesday|2054Q3|N|N|N|2471481|2471692|2471141|2471415|N|N|N|N|N| +2471507|AAAAAAAADFGLFCAA|2054-08-27|1855|8070|619|2054|4|8|27|3|2054|619|8070|Thursday|2054Q3|N|N|N|2471481|2471692|2471142|2471416|N|N|N|N|N| +2471508|AAAAAAAAEFGLFCAA|2054-08-28|1855|8070|619|2054|5|8|28|3|2054|619|8070|Friday|2054Q3|N|Y|N|2471481|2471692|2471143|2471417|N|N|N|N|N| +2471509|AAAAAAAAFFGLFCAA|2054-08-29|1855|8070|619|2054|6|8|29|3|2054|619|8070|Saturday|2054Q3|N|Y|N|2471481|2471692|2471144|2471418|N|N|N|N|N| +2471510|AAAAAAAAGFGLFCAA|2054-08-30|1855|8070|619|2054|0|8|30|3|2054|619|8070|Sunday|2054Q3|N|N|N|2471481|2471692|2471145|2471419|N|N|N|N|N| +2471511|AAAAAAAAHFGLFCAA|2054-08-31|1855|8070|619|2054|1|8|31|3|2054|619|8070|Monday|2054Q3|N|N|N|2471481|2471692|2471146|2471420|N|N|N|N|N| +2471512|AAAAAAAAIFGLFCAA|2054-09-01|1856|8071|620|2054|2|9|1|3|2054|620|8071|Tuesday|2054Q3|N|N|N|2471512|2471754|2471147|2471421|N|N|N|N|N| +2471513|AAAAAAAAJFGLFCAA|2054-09-02|1856|8071|620|2054|3|9|2|3|2054|620|8071|Wednesday|2054Q3|N|N|N|2471512|2471754|2471148|2471422|N|N|N|N|N| +2471514|AAAAAAAAKFGLFCAA|2054-09-03|1856|8071|620|2054|4|9|3|3|2054|620|8071|Thursday|2054Q3|N|N|N|2471512|2471754|2471149|2471423|N|N|N|N|N| +2471515|AAAAAAAALFGLFCAA|2054-09-04|1856|8071|620|2054|5|9|4|3|2054|620|8071|Friday|2054Q3|N|Y|N|2471512|2471754|2471150|2471424|N|N|N|N|N| +2471516|AAAAAAAAMFGLFCAA|2054-09-05|1856|8071|620|2054|6|9|5|3|2054|620|8071|Saturday|2054Q3|N|Y|N|2471512|2471754|2471151|2471425|N|N|N|N|N| +2471517|AAAAAAAANFGLFCAA|2054-09-06|1856|8071|620|2054|0|9|6|3|2054|620|8071|Sunday|2054Q3|N|N|N|2471512|2471754|2471152|2471426|N|N|N|N|N| +2471518|AAAAAAAAOFGLFCAA|2054-09-07|1856|8071|620|2054|1|9|7|3|2054|620|8071|Monday|2054Q3|N|N|N|2471512|2471754|2471153|2471427|N|N|N|N|N| +2471519|AAAAAAAAPFGLFCAA|2054-09-08|1856|8072|620|2054|2|9|8|3|2054|620|8072|Tuesday|2054Q3|N|N|N|2471512|2471754|2471154|2471428|N|N|N|N|N| +2471520|AAAAAAAAAGGLFCAA|2054-09-09|1856|8072|620|2054|3|9|9|3|2054|620|8072|Wednesday|2054Q3|N|N|N|2471512|2471754|2471155|2471429|N|N|N|N|N| +2471521|AAAAAAAABGGLFCAA|2054-09-10|1856|8072|620|2054|4|9|10|3|2054|620|8072|Thursday|2054Q3|N|N|N|2471512|2471754|2471156|2471430|N|N|N|N|N| +2471522|AAAAAAAACGGLFCAA|2054-09-11|1856|8072|620|2054|5|9|11|3|2054|620|8072|Friday|2054Q3|N|Y|N|2471512|2471754|2471157|2471431|N|N|N|N|N| +2471523|AAAAAAAADGGLFCAA|2054-09-12|1856|8072|620|2054|6|9|12|3|2054|620|8072|Saturday|2054Q3|N|Y|N|2471512|2471754|2471158|2471432|N|N|N|N|N| +2471524|AAAAAAAAEGGLFCAA|2054-09-13|1856|8072|620|2054|0|9|13|3|2054|620|8072|Sunday|2054Q3|N|N|N|2471512|2471754|2471159|2471433|N|N|N|N|N| +2471525|AAAAAAAAFGGLFCAA|2054-09-14|1856|8072|620|2054|1|9|14|3|2054|620|8072|Monday|2054Q3|N|N|N|2471512|2471754|2471160|2471434|N|N|N|N|N| +2471526|AAAAAAAAGGGLFCAA|2054-09-15|1856|8073|620|2054|2|9|15|3|2054|620|8073|Tuesday|2054Q3|N|N|N|2471512|2471754|2471161|2471435|N|N|N|N|N| +2471527|AAAAAAAAHGGLFCAA|2054-09-16|1856|8073|620|2054|3|9|16|3|2054|620|8073|Wednesday|2054Q3|N|N|N|2471512|2471754|2471162|2471436|N|N|N|N|N| +2471528|AAAAAAAAIGGLFCAA|2054-09-17|1856|8073|620|2054|4|9|17|3|2054|620|8073|Thursday|2054Q3|N|N|N|2471512|2471754|2471163|2471437|N|N|N|N|N| +2471529|AAAAAAAAJGGLFCAA|2054-09-18|1856|8073|620|2054|5|9|18|3|2054|620|8073|Friday|2054Q3|N|Y|N|2471512|2471754|2471164|2471438|N|N|N|N|N| +2471530|AAAAAAAAKGGLFCAA|2054-09-19|1856|8073|620|2054|6|9|19|3|2054|620|8073|Saturday|2054Q3|N|Y|N|2471512|2471754|2471165|2471439|N|N|N|N|N| +2471531|AAAAAAAALGGLFCAA|2054-09-20|1856|8073|620|2054|0|9|20|3|2054|620|8073|Sunday|2054Q3|N|N|N|2471512|2471754|2471166|2471440|N|N|N|N|N| +2471532|AAAAAAAAMGGLFCAA|2054-09-21|1856|8073|620|2054|1|9|21|3|2054|620|8073|Monday|2054Q3|N|N|N|2471512|2471754|2471167|2471441|N|N|N|N|N| +2471533|AAAAAAAANGGLFCAA|2054-09-22|1856|8074|620|2054|2|9|22|3|2054|620|8074|Tuesday|2054Q3|N|N|N|2471512|2471754|2471168|2471442|N|N|N|N|N| +2471534|AAAAAAAAOGGLFCAA|2054-09-23|1856|8074|620|2054|3|9|23|3|2054|620|8074|Wednesday|2054Q3|N|N|N|2471512|2471754|2471169|2471443|N|N|N|N|N| +2471535|AAAAAAAAPGGLFCAA|2054-09-24|1856|8074|620|2054|4|9|24|3|2054|620|8074|Thursday|2054Q3|N|N|N|2471512|2471754|2471170|2471444|N|N|N|N|N| +2471536|AAAAAAAAAHGLFCAA|2054-09-25|1856|8074|620|2054|5|9|25|3|2054|620|8074|Friday|2054Q3|N|Y|N|2471512|2471754|2471171|2471445|N|N|N|N|N| +2471537|AAAAAAAABHGLFCAA|2054-09-26|1856|8074|620|2054|6|9|26|3|2054|620|8074|Saturday|2054Q3|N|Y|N|2471512|2471754|2471172|2471446|N|N|N|N|N| +2471538|AAAAAAAACHGLFCAA|2054-09-27|1856|8074|620|2054|0|9|27|3|2054|620|8074|Sunday|2054Q3|N|N|N|2471512|2471754|2471173|2471447|N|N|N|N|N| +2471539|AAAAAAAADHGLFCAA|2054-09-28|1856|8074|620|2054|1|9|28|3|2054|620|8074|Monday|2054Q3|N|N|N|2471512|2471754|2471174|2471448|N|N|N|N|N| +2471540|AAAAAAAAEHGLFCAA|2054-09-29|1856|8075|620|2054|2|9|29|3|2054|620|8075|Tuesday|2054Q3|N|N|N|2471512|2471754|2471175|2471449|N|N|N|N|N| +2471541|AAAAAAAAFHGLFCAA|2054-09-30|1856|8075|620|2054|3|9|30|3|2054|620|8075|Wednesday|2054Q3|N|N|N|2471512|2471754|2471176|2471450|N|N|N|N|N| +2471542|AAAAAAAAGHGLFCAA|2054-10-01|1857|8075|620|2054|4|10|1|3|2054|620|8075|Thursday|2054Q3|N|N|N|2471542|2471814|2471177|2471450|N|N|N|N|N| +2471543|AAAAAAAAHHGLFCAA|2054-10-02|1857|8075|620|2054|5|10|2|4|2054|620|8075|Friday|2054Q4|N|Y|N|2471542|2471814|2471178|2471451|N|N|N|N|N| +2471544|AAAAAAAAIHGLFCAA|2054-10-03|1857|8075|620|2054|6|10|3|4|2054|620|8075|Saturday|2054Q4|N|Y|N|2471542|2471814|2471179|2471452|N|N|N|N|N| +2471545|AAAAAAAAJHGLFCAA|2054-10-04|1857|8075|620|2054|0|10|4|4|2054|620|8075|Sunday|2054Q4|N|N|N|2471542|2471814|2471180|2471453|N|N|N|N|N| +2471546|AAAAAAAAKHGLFCAA|2054-10-05|1857|8075|620|2054|1|10|5|4|2054|620|8075|Monday|2054Q4|N|N|N|2471542|2471814|2471181|2471454|N|N|N|N|N| +2471547|AAAAAAAALHGLFCAA|2054-10-06|1857|8076|620|2054|2|10|6|4|2054|620|8076|Tuesday|2054Q4|N|N|N|2471542|2471814|2471182|2471455|N|N|N|N|N| +2471548|AAAAAAAAMHGLFCAA|2054-10-07|1857|8076|620|2054|3|10|7|4|2054|620|8076|Wednesday|2054Q4|N|N|N|2471542|2471814|2471183|2471456|N|N|N|N|N| +2471549|AAAAAAAANHGLFCAA|2054-10-08|1857|8076|620|2054|4|10|8|4|2054|620|8076|Thursday|2054Q4|N|N|N|2471542|2471814|2471184|2471457|N|N|N|N|N| +2471550|AAAAAAAAOHGLFCAA|2054-10-09|1857|8076|620|2054|5|10|9|4|2054|620|8076|Friday|2054Q4|N|Y|N|2471542|2471814|2471185|2471458|N|N|N|N|N| +2471551|AAAAAAAAPHGLFCAA|2054-10-10|1857|8076|620|2054|6|10|10|4|2054|620|8076|Saturday|2054Q4|N|Y|N|2471542|2471814|2471186|2471459|N|N|N|N|N| +2471552|AAAAAAAAAIGLFCAA|2054-10-11|1857|8076|620|2054|0|10|11|4|2054|620|8076|Sunday|2054Q4|N|N|N|2471542|2471814|2471187|2471460|N|N|N|N|N| +2471553|AAAAAAAABIGLFCAA|2054-10-12|1857|8076|620|2054|1|10|12|4|2054|620|8076|Monday|2054Q4|N|N|N|2471542|2471814|2471188|2471461|N|N|N|N|N| +2471554|AAAAAAAACIGLFCAA|2054-10-13|1857|8077|620|2054|2|10|13|4|2054|620|8077|Tuesday|2054Q4|N|N|N|2471542|2471814|2471189|2471462|N|N|N|N|N| +2471555|AAAAAAAADIGLFCAA|2054-10-14|1857|8077|620|2054|3|10|14|4|2054|620|8077|Wednesday|2054Q4|N|N|N|2471542|2471814|2471190|2471463|N|N|N|N|N| +2471556|AAAAAAAAEIGLFCAA|2054-10-15|1857|8077|620|2054|4|10|15|4|2054|620|8077|Thursday|2054Q4|N|N|N|2471542|2471814|2471191|2471464|N|N|N|N|N| +2471557|AAAAAAAAFIGLFCAA|2054-10-16|1857|8077|620|2054|5|10|16|4|2054|620|8077|Friday|2054Q4|N|Y|N|2471542|2471814|2471192|2471465|N|N|N|N|N| +2471558|AAAAAAAAGIGLFCAA|2054-10-17|1857|8077|620|2054|6|10|17|4|2054|620|8077|Saturday|2054Q4|N|Y|N|2471542|2471814|2471193|2471466|N|N|N|N|N| +2471559|AAAAAAAAHIGLFCAA|2054-10-18|1857|8077|620|2054|0|10|18|4|2054|620|8077|Sunday|2054Q4|N|N|N|2471542|2471814|2471194|2471467|N|N|N|N|N| +2471560|AAAAAAAAIIGLFCAA|2054-10-19|1857|8077|620|2054|1|10|19|4|2054|620|8077|Monday|2054Q4|N|N|N|2471542|2471814|2471195|2471468|N|N|N|N|N| +2471561|AAAAAAAAJIGLFCAA|2054-10-20|1857|8078|620|2054|2|10|20|4|2054|620|8078|Tuesday|2054Q4|N|N|N|2471542|2471814|2471196|2471469|N|N|N|N|N| +2471562|AAAAAAAAKIGLFCAA|2054-10-21|1857|8078|620|2054|3|10|21|4|2054|620|8078|Wednesday|2054Q4|N|N|N|2471542|2471814|2471197|2471470|N|N|N|N|N| +2471563|AAAAAAAALIGLFCAA|2054-10-22|1857|8078|620|2054|4|10|22|4|2054|620|8078|Thursday|2054Q4|N|N|N|2471542|2471814|2471198|2471471|N|N|N|N|N| +2471564|AAAAAAAAMIGLFCAA|2054-10-23|1857|8078|620|2054|5|10|23|4|2054|620|8078|Friday|2054Q4|N|Y|N|2471542|2471814|2471199|2471472|N|N|N|N|N| +2471565|AAAAAAAANIGLFCAA|2054-10-24|1857|8078|620|2054|6|10|24|4|2054|620|8078|Saturday|2054Q4|N|Y|N|2471542|2471814|2471200|2471473|N|N|N|N|N| +2471566|AAAAAAAAOIGLFCAA|2054-10-25|1857|8078|620|2054|0|10|25|4|2054|620|8078|Sunday|2054Q4|N|N|N|2471542|2471814|2471201|2471474|N|N|N|N|N| +2471567|AAAAAAAAPIGLFCAA|2054-10-26|1857|8078|620|2054|1|10|26|4|2054|620|8078|Monday|2054Q4|N|N|N|2471542|2471814|2471202|2471475|N|N|N|N|N| +2471568|AAAAAAAAAJGLFCAA|2054-10-27|1857|8079|620|2054|2|10|27|4|2054|620|8079|Tuesday|2054Q4|N|N|N|2471542|2471814|2471203|2471476|N|N|N|N|N| +2471569|AAAAAAAABJGLFCAA|2054-10-28|1857|8079|620|2054|3|10|28|4|2054|620|8079|Wednesday|2054Q4|N|N|N|2471542|2471814|2471204|2471477|N|N|N|N|N| +2471570|AAAAAAAACJGLFCAA|2054-10-29|1857|8079|620|2054|4|10|29|4|2054|620|8079|Thursday|2054Q4|N|N|N|2471542|2471814|2471205|2471478|N|N|N|N|N| +2471571|AAAAAAAADJGLFCAA|2054-10-30|1857|8079|620|2054|5|10|30|4|2054|620|8079|Friday|2054Q4|N|Y|N|2471542|2471814|2471206|2471479|N|N|N|N|N| +2471572|AAAAAAAAEJGLFCAA|2054-10-31|1857|8079|620|2054|6|10|31|4|2054|620|8079|Saturday|2054Q4|N|Y|N|2471542|2471814|2471207|2471480|N|N|N|N|N| +2471573|AAAAAAAAFJGLFCAA|2054-11-01|1858|8079|620|2054|0|11|1|4|2054|620|8079|Sunday|2054Q4|N|N|N|2471573|2471876|2471208|2471481|N|N|N|N|N| +2471574|AAAAAAAAGJGLFCAA|2054-11-02|1858|8079|620|2054|1|11|2|4|2054|620|8079|Monday|2054Q4|N|N|N|2471573|2471876|2471209|2471482|N|N|N|N|N| +2471575|AAAAAAAAHJGLFCAA|2054-11-03|1858|8080|620|2054|2|11|3|4|2054|620|8080|Tuesday|2054Q4|N|N|N|2471573|2471876|2471210|2471483|N|N|N|N|N| +2471576|AAAAAAAAIJGLFCAA|2054-11-04|1858|8080|620|2054|3|11|4|4|2054|620|8080|Wednesday|2054Q4|N|N|N|2471573|2471876|2471211|2471484|N|N|N|N|N| +2471577|AAAAAAAAJJGLFCAA|2054-11-05|1858|8080|620|2054|4|11|5|4|2054|620|8080|Thursday|2054Q4|N|N|N|2471573|2471876|2471212|2471485|N|N|N|N|N| +2471578|AAAAAAAAKJGLFCAA|2054-11-06|1858|8080|620|2054|5|11|6|4|2054|620|8080|Friday|2054Q4|N|Y|N|2471573|2471876|2471213|2471486|N|N|N|N|N| +2471579|AAAAAAAALJGLFCAA|2054-11-07|1858|8080|620|2054|6|11|7|4|2054|620|8080|Saturday|2054Q4|N|Y|N|2471573|2471876|2471214|2471487|N|N|N|N|N| +2471580|AAAAAAAAMJGLFCAA|2054-11-08|1858|8080|620|2054|0|11|8|4|2054|620|8080|Sunday|2054Q4|N|N|N|2471573|2471876|2471215|2471488|N|N|N|N|N| +2471581|AAAAAAAANJGLFCAA|2054-11-09|1858|8080|620|2054|1|11|9|4|2054|620|8080|Monday|2054Q4|N|N|N|2471573|2471876|2471216|2471489|N|N|N|N|N| +2471582|AAAAAAAAOJGLFCAA|2054-11-10|1858|8081|620|2054|2|11|10|4|2054|620|8081|Tuesday|2054Q4|N|N|N|2471573|2471876|2471217|2471490|N|N|N|N|N| +2471583|AAAAAAAAPJGLFCAA|2054-11-11|1858|8081|620|2054|3|11|11|4|2054|620|8081|Wednesday|2054Q4|N|N|N|2471573|2471876|2471218|2471491|N|N|N|N|N| +2471584|AAAAAAAAAKGLFCAA|2054-11-12|1858|8081|620|2054|4|11|12|4|2054|620|8081|Thursday|2054Q4|N|N|N|2471573|2471876|2471219|2471492|N|N|N|N|N| +2471585|AAAAAAAABKGLFCAA|2054-11-13|1858|8081|620|2054|5|11|13|4|2054|620|8081|Friday|2054Q4|N|Y|N|2471573|2471876|2471220|2471493|N|N|N|N|N| +2471586|AAAAAAAACKGLFCAA|2054-11-14|1858|8081|620|2054|6|11|14|4|2054|620|8081|Saturday|2054Q4|N|Y|N|2471573|2471876|2471221|2471494|N|N|N|N|N| +2471587|AAAAAAAADKGLFCAA|2054-11-15|1858|8081|620|2054|0|11|15|4|2054|620|8081|Sunday|2054Q4|N|N|N|2471573|2471876|2471222|2471495|N|N|N|N|N| +2471588|AAAAAAAAEKGLFCAA|2054-11-16|1858|8081|620|2054|1|11|16|4|2054|620|8081|Monday|2054Q4|N|N|N|2471573|2471876|2471223|2471496|N|N|N|N|N| +2471589|AAAAAAAAFKGLFCAA|2054-11-17|1858|8082|620|2054|2|11|17|4|2054|620|8082|Tuesday|2054Q4|N|N|N|2471573|2471876|2471224|2471497|N|N|N|N|N| +2471590|AAAAAAAAGKGLFCAA|2054-11-18|1858|8082|620|2054|3|11|18|4|2054|620|8082|Wednesday|2054Q4|N|N|N|2471573|2471876|2471225|2471498|N|N|N|N|N| +2471591|AAAAAAAAHKGLFCAA|2054-11-19|1858|8082|620|2054|4|11|19|4|2054|620|8082|Thursday|2054Q4|N|N|N|2471573|2471876|2471226|2471499|N|N|N|N|N| +2471592|AAAAAAAAIKGLFCAA|2054-11-20|1858|8082|620|2054|5|11|20|4|2054|620|8082|Friday|2054Q4|N|Y|N|2471573|2471876|2471227|2471500|N|N|N|N|N| +2471593|AAAAAAAAJKGLFCAA|2054-11-21|1858|8082|620|2054|6|11|21|4|2054|620|8082|Saturday|2054Q4|N|Y|N|2471573|2471876|2471228|2471501|N|N|N|N|N| +2471594|AAAAAAAAKKGLFCAA|2054-11-22|1858|8082|620|2054|0|11|22|4|2054|620|8082|Sunday|2054Q4|N|N|N|2471573|2471876|2471229|2471502|N|N|N|N|N| +2471595|AAAAAAAALKGLFCAA|2054-11-23|1858|8082|620|2054|1|11|23|4|2054|620|8082|Monday|2054Q4|N|N|N|2471573|2471876|2471230|2471503|N|N|N|N|N| +2471596|AAAAAAAAMKGLFCAA|2054-11-24|1858|8083|620|2054|2|11|24|4|2054|620|8083|Tuesday|2054Q4|N|N|N|2471573|2471876|2471231|2471504|N|N|N|N|N| +2471597|AAAAAAAANKGLFCAA|2054-11-25|1858|8083|620|2054|3|11|25|4|2054|620|8083|Wednesday|2054Q4|N|N|N|2471573|2471876|2471232|2471505|N|N|N|N|N| +2471598|AAAAAAAAOKGLFCAA|2054-11-26|1858|8083|620|2054|4|11|26|4|2054|620|8083|Thursday|2054Q4|N|N|N|2471573|2471876|2471233|2471506|N|N|N|N|N| +2471599|AAAAAAAAPKGLFCAA|2054-11-27|1858|8083|620|2054|5|11|27|4|2054|620|8083|Friday|2054Q4|N|Y|N|2471573|2471876|2471234|2471507|N|N|N|N|N| +2471600|AAAAAAAAALGLFCAA|2054-11-28|1858|8083|620|2054|6|11|28|4|2054|620|8083|Saturday|2054Q4|N|Y|N|2471573|2471876|2471235|2471508|N|N|N|N|N| +2471601|AAAAAAAABLGLFCAA|2054-11-29|1858|8083|620|2054|0|11|29|4|2054|620|8083|Sunday|2054Q4|N|N|N|2471573|2471876|2471236|2471509|N|N|N|N|N| +2471602|AAAAAAAACLGLFCAA|2054-11-30|1858|8083|620|2054|1|11|30|4|2054|620|8083|Monday|2054Q4|N|N|N|2471573|2471876|2471237|2471510|N|N|N|N|N| +2471603|AAAAAAAADLGLFCAA|2054-12-01|1859|8084|621|2054|2|12|1|4|2054|621|8084|Tuesday|2054Q4|N|N|N|2471603|2471936|2471238|2471511|N|N|N|N|N| +2471604|AAAAAAAAELGLFCAA|2054-12-02|1859|8084|621|2054|3|12|2|4|2054|621|8084|Wednesday|2054Q4|N|N|N|2471603|2471936|2471239|2471512|N|N|N|N|N| +2471605|AAAAAAAAFLGLFCAA|2054-12-03|1859|8084|621|2054|4|12|3|4|2054|621|8084|Thursday|2054Q4|N|N|N|2471603|2471936|2471240|2471513|N|N|N|N|N| +2471606|AAAAAAAAGLGLFCAA|2054-12-04|1859|8084|621|2054|5|12|4|4|2054|621|8084|Friday|2054Q4|N|Y|N|2471603|2471936|2471241|2471514|N|N|N|N|N| +2471607|AAAAAAAAHLGLFCAA|2054-12-05|1859|8084|621|2054|6|12|5|4|2054|621|8084|Saturday|2054Q4|N|Y|N|2471603|2471936|2471242|2471515|N|N|N|N|N| +2471608|AAAAAAAAILGLFCAA|2054-12-06|1859|8084|621|2054|0|12|6|4|2054|621|8084|Sunday|2054Q4|N|N|N|2471603|2471936|2471243|2471516|N|N|N|N|N| +2471609|AAAAAAAAJLGLFCAA|2054-12-07|1859|8084|621|2054|1|12|7|4|2054|621|8084|Monday|2054Q4|N|N|N|2471603|2471936|2471244|2471517|N|N|N|N|N| +2471610|AAAAAAAAKLGLFCAA|2054-12-08|1859|8085|621|2054|2|12|8|4|2054|621|8085|Tuesday|2054Q4|N|N|N|2471603|2471936|2471245|2471518|N|N|N|N|N| +2471611|AAAAAAAALLGLFCAA|2054-12-09|1859|8085|621|2054|3|12|9|4|2054|621|8085|Wednesday|2054Q4|N|N|N|2471603|2471936|2471246|2471519|N|N|N|N|N| +2471612|AAAAAAAAMLGLFCAA|2054-12-10|1859|8085|621|2054|4|12|10|4|2054|621|8085|Thursday|2054Q4|N|N|N|2471603|2471936|2471247|2471520|N|N|N|N|N| +2471613|AAAAAAAANLGLFCAA|2054-12-11|1859|8085|621|2054|5|12|11|4|2054|621|8085|Friday|2054Q4|N|Y|N|2471603|2471936|2471248|2471521|N|N|N|N|N| +2471614|AAAAAAAAOLGLFCAA|2054-12-12|1859|8085|621|2054|6|12|12|4|2054|621|8085|Saturday|2054Q4|N|Y|N|2471603|2471936|2471249|2471522|N|N|N|N|N| +2471615|AAAAAAAAPLGLFCAA|2054-12-13|1859|8085|621|2054|0|12|13|4|2054|621|8085|Sunday|2054Q4|N|N|N|2471603|2471936|2471250|2471523|N|N|N|N|N| +2471616|AAAAAAAAAMGLFCAA|2054-12-14|1859|8085|621|2054|1|12|14|4|2054|621|8085|Monday|2054Q4|N|N|N|2471603|2471936|2471251|2471524|N|N|N|N|N| +2471617|AAAAAAAABMGLFCAA|2054-12-15|1859|8086|621|2054|2|12|15|4|2054|621|8086|Tuesday|2054Q4|N|N|N|2471603|2471936|2471252|2471525|N|N|N|N|N| +2471618|AAAAAAAACMGLFCAA|2054-12-16|1859|8086|621|2054|3|12|16|4|2054|621|8086|Wednesday|2054Q4|N|N|N|2471603|2471936|2471253|2471526|N|N|N|N|N| +2471619|AAAAAAAADMGLFCAA|2054-12-17|1859|8086|621|2054|4|12|17|4|2054|621|8086|Thursday|2054Q4|N|N|N|2471603|2471936|2471254|2471527|N|N|N|N|N| +2471620|AAAAAAAAEMGLFCAA|2054-12-18|1859|8086|621|2054|5|12|18|4|2054|621|8086|Friday|2054Q4|N|Y|N|2471603|2471936|2471255|2471528|N|N|N|N|N| +2471621|AAAAAAAAFMGLFCAA|2054-12-19|1859|8086|621|2054|6|12|19|4|2054|621|8086|Saturday|2054Q4|N|Y|N|2471603|2471936|2471256|2471529|N|N|N|N|N| +2471622|AAAAAAAAGMGLFCAA|2054-12-20|1859|8086|621|2054|0|12|20|4|2054|621|8086|Sunday|2054Q4|N|N|N|2471603|2471936|2471257|2471530|N|N|N|N|N| +2471623|AAAAAAAAHMGLFCAA|2054-12-21|1859|8086|621|2054|1|12|21|4|2054|621|8086|Monday|2054Q4|N|N|N|2471603|2471936|2471258|2471531|N|N|N|N|N| +2471624|AAAAAAAAIMGLFCAA|2054-12-22|1859|8087|621|2054|2|12|22|4|2054|621|8087|Tuesday|2054Q4|N|N|N|2471603|2471936|2471259|2471532|N|N|N|N|N| +2471625|AAAAAAAAJMGLFCAA|2054-12-23|1859|8087|621|2054|3|12|23|4|2054|621|8087|Wednesday|2054Q4|N|N|N|2471603|2471936|2471260|2471533|N|N|N|N|N| +2471626|AAAAAAAAKMGLFCAA|2054-12-24|1859|8087|621|2054|4|12|24|4|2054|621|8087|Thursday|2054Q4|N|N|N|2471603|2471936|2471261|2471534|N|N|N|N|N| +2471627|AAAAAAAALMGLFCAA|2054-12-25|1859|8087|621|2054|5|12|25|4|2054|621|8087|Friday|2054Q4|N|Y|N|2471603|2471936|2471262|2471535|N|N|N|N|N| +2471628|AAAAAAAAMMGLFCAA|2054-12-26|1859|8087|621|2054|6|12|26|4|2054|621|8087|Saturday|2054Q4|Y|Y|N|2471603|2471936|2471263|2471536|N|N|N|N|N| +2471629|AAAAAAAANMGLFCAA|2054-12-27|1859|8087|621|2054|0|12|27|4|2054|621|8087|Sunday|2054Q4|N|N|Y|2471603|2471936|2471264|2471537|N|N|N|N|N| +2471630|AAAAAAAAOMGLFCAA|2054-12-28|1859|8087|621|2054|1|12|28|4|2054|621|8087|Monday|2054Q4|N|N|N|2471603|2471936|2471265|2471538|N|N|N|N|N| +2471631|AAAAAAAAPMGLFCAA|2054-12-29|1859|8088|621|2054|2|12|29|4|2054|621|8088|Tuesday|2054Q4|N|N|N|2471603|2471936|2471266|2471539|N|N|N|N|N| +2471632|AAAAAAAAANGLFCAA|2054-12-30|1859|8088|621|2054|3|12|30|4|2054|621|8088|Wednesday|2054Q4|N|N|N|2471603|2471936|2471267|2471540|N|N|N|N|N| +2471633|AAAAAAAABNGLFCAA|2054-12-31|1859|8088|621|2054|4|12|31|4|2054|621|8088|Thursday|2054Q4|N|N|N|2471603|2471936|2471268|2471541|N|N|N|N|N| +2471634|AAAAAAAACNGLFCAA|2055-01-01|1860|8088|621|2055|5|1|1|1|2055|621|8088|Friday|2055Q1|Y|Y|N|2471634|2471633|2471269|2471542|N|N|N|N|N| +2471635|AAAAAAAADNGLFCAA|2055-01-02|1860|8088|621|2055|6|1|2|1|2055|621|8088|Saturday|2055Q1|N|Y|Y|2471634|2471633|2471270|2471543|N|N|N|N|N| +2471636|AAAAAAAAENGLFCAA|2055-01-03|1860|8088|621|2055|0|1|3|1|2055|621|8088|Sunday|2055Q1|N|N|N|2471634|2471633|2471271|2471544|N|N|N|N|N| +2471637|AAAAAAAAFNGLFCAA|2055-01-04|1860|8088|621|2055|1|1|4|1|2055|621|8088|Monday|2055Q1|N|N|N|2471634|2471633|2471272|2471545|N|N|N|N|N| +2471638|AAAAAAAAGNGLFCAA|2055-01-05|1860|8089|621|2055|2|1|5|1|2055|621|8089|Tuesday|2055Q1|N|N|N|2471634|2471633|2471273|2471546|N|N|N|N|N| +2471639|AAAAAAAAHNGLFCAA|2055-01-06|1860|8089|621|2055|3|1|6|1|2055|621|8089|Wednesday|2055Q1|N|N|N|2471634|2471633|2471274|2471547|N|N|N|N|N| +2471640|AAAAAAAAINGLFCAA|2055-01-07|1860|8089|621|2055|4|1|7|1|2055|621|8089|Thursday|2055Q1|N|N|N|2471634|2471633|2471275|2471548|N|N|N|N|N| +2471641|AAAAAAAAJNGLFCAA|2055-01-08|1860|8089|621|2055|5|1|8|1|2055|621|8089|Friday|2055Q1|N|Y|N|2471634|2471633|2471276|2471549|N|N|N|N|N| +2471642|AAAAAAAAKNGLFCAA|2055-01-09|1860|8089|621|2055|6|1|9|1|2055|621|8089|Saturday|2055Q1|N|Y|N|2471634|2471633|2471277|2471550|N|N|N|N|N| +2471643|AAAAAAAALNGLFCAA|2055-01-10|1860|8089|621|2055|0|1|10|1|2055|621|8089|Sunday|2055Q1|N|N|N|2471634|2471633|2471278|2471551|N|N|N|N|N| +2471644|AAAAAAAAMNGLFCAA|2055-01-11|1860|8089|621|2055|1|1|11|1|2055|621|8089|Monday|2055Q1|N|N|N|2471634|2471633|2471279|2471552|N|N|N|N|N| +2471645|AAAAAAAANNGLFCAA|2055-01-12|1860|8090|621|2055|2|1|12|1|2055|621|8090|Tuesday|2055Q1|N|N|N|2471634|2471633|2471280|2471553|N|N|N|N|N| +2471646|AAAAAAAAONGLFCAA|2055-01-13|1860|8090|621|2055|3|1|13|1|2055|621|8090|Wednesday|2055Q1|N|N|N|2471634|2471633|2471281|2471554|N|N|N|N|N| +2471647|AAAAAAAAPNGLFCAA|2055-01-14|1860|8090|621|2055|4|1|14|1|2055|621|8090|Thursday|2055Q1|N|N|N|2471634|2471633|2471282|2471555|N|N|N|N|N| +2471648|AAAAAAAAAOGLFCAA|2055-01-15|1860|8090|621|2055|5|1|15|1|2055|621|8090|Friday|2055Q1|N|Y|N|2471634|2471633|2471283|2471556|N|N|N|N|N| +2471649|AAAAAAAABOGLFCAA|2055-01-16|1860|8090|621|2055|6|1|16|1|2055|621|8090|Saturday|2055Q1|N|Y|N|2471634|2471633|2471284|2471557|N|N|N|N|N| +2471650|AAAAAAAACOGLFCAA|2055-01-17|1860|8090|621|2055|0|1|17|1|2055|621|8090|Sunday|2055Q1|N|N|N|2471634|2471633|2471285|2471558|N|N|N|N|N| +2471651|AAAAAAAADOGLFCAA|2055-01-18|1860|8090|621|2055|1|1|18|1|2055|621|8090|Monday|2055Q1|N|N|N|2471634|2471633|2471286|2471559|N|N|N|N|N| +2471652|AAAAAAAAEOGLFCAA|2055-01-19|1860|8091|621|2055|2|1|19|1|2055|621|8091|Tuesday|2055Q1|N|N|N|2471634|2471633|2471287|2471560|N|N|N|N|N| +2471653|AAAAAAAAFOGLFCAA|2055-01-20|1860|8091|621|2055|3|1|20|1|2055|621|8091|Wednesday|2055Q1|N|N|N|2471634|2471633|2471288|2471561|N|N|N|N|N| +2471654|AAAAAAAAGOGLFCAA|2055-01-21|1860|8091|621|2055|4|1|21|1|2055|621|8091|Thursday|2055Q1|N|N|N|2471634|2471633|2471289|2471562|N|N|N|N|N| +2471655|AAAAAAAAHOGLFCAA|2055-01-22|1860|8091|621|2055|5|1|22|1|2055|621|8091|Friday|2055Q1|N|Y|N|2471634|2471633|2471290|2471563|N|N|N|N|N| +2471656|AAAAAAAAIOGLFCAA|2055-01-23|1860|8091|621|2055|6|1|23|1|2055|621|8091|Saturday|2055Q1|N|Y|N|2471634|2471633|2471291|2471564|N|N|N|N|N| +2471657|AAAAAAAAJOGLFCAA|2055-01-24|1860|8091|621|2055|0|1|24|1|2055|621|8091|Sunday|2055Q1|N|N|N|2471634|2471633|2471292|2471565|N|N|N|N|N| +2471658|AAAAAAAAKOGLFCAA|2055-01-25|1860|8091|621|2055|1|1|25|1|2055|621|8091|Monday|2055Q1|N|N|N|2471634|2471633|2471293|2471566|N|N|N|N|N| +2471659|AAAAAAAALOGLFCAA|2055-01-26|1860|8092|621|2055|2|1|26|1|2055|621|8092|Tuesday|2055Q1|N|N|N|2471634|2471633|2471294|2471567|N|N|N|N|N| +2471660|AAAAAAAAMOGLFCAA|2055-01-27|1860|8092|621|2055|3|1|27|1|2055|621|8092|Wednesday|2055Q1|N|N|N|2471634|2471633|2471295|2471568|N|N|N|N|N| +2471661|AAAAAAAANOGLFCAA|2055-01-28|1860|8092|621|2055|4|1|28|1|2055|621|8092|Thursday|2055Q1|N|N|N|2471634|2471633|2471296|2471569|N|N|N|N|N| +2471662|AAAAAAAAOOGLFCAA|2055-01-29|1860|8092|621|2055|5|1|29|1|2055|621|8092|Friday|2055Q1|N|Y|N|2471634|2471633|2471297|2471570|N|N|N|N|N| +2471663|AAAAAAAAPOGLFCAA|2055-01-30|1860|8092|621|2055|6|1|30|1|2055|621|8092|Saturday|2055Q1|N|Y|N|2471634|2471633|2471298|2471571|N|N|N|N|N| +2471664|AAAAAAAAAPGLFCAA|2055-01-31|1860|8092|621|2055|0|1|31|1|2055|621|8092|Sunday|2055Q1|N|N|N|2471634|2471633|2471299|2471572|N|N|N|N|N| +2471665|AAAAAAAABPGLFCAA|2055-02-01|1861|8092|621|2055|1|2|1|1|2055|621|8092|Monday|2055Q1|N|N|N|2471665|2471695|2471300|2471573|N|N|N|N|N| +2471666|AAAAAAAACPGLFCAA|2055-02-02|1861|8093|621|2055|2|2|2|1|2055|621|8093|Tuesday|2055Q1|N|N|N|2471665|2471695|2471301|2471574|N|N|N|N|N| +2471667|AAAAAAAADPGLFCAA|2055-02-03|1861|8093|621|2055|3|2|3|1|2055|621|8093|Wednesday|2055Q1|N|N|N|2471665|2471695|2471302|2471575|N|N|N|N|N| +2471668|AAAAAAAAEPGLFCAA|2055-02-04|1861|8093|621|2055|4|2|4|1|2055|621|8093|Thursday|2055Q1|N|N|N|2471665|2471695|2471303|2471576|N|N|N|N|N| +2471669|AAAAAAAAFPGLFCAA|2055-02-05|1861|8093|621|2055|5|2|5|1|2055|621|8093|Friday|2055Q1|N|Y|N|2471665|2471695|2471304|2471577|N|N|N|N|N| +2471670|AAAAAAAAGPGLFCAA|2055-02-06|1861|8093|621|2055|6|2|6|1|2055|621|8093|Saturday|2055Q1|N|Y|N|2471665|2471695|2471305|2471578|N|N|N|N|N| +2471671|AAAAAAAAHPGLFCAA|2055-02-07|1861|8093|621|2055|0|2|7|1|2055|621|8093|Sunday|2055Q1|N|N|N|2471665|2471695|2471306|2471579|N|N|N|N|N| +2471672|AAAAAAAAIPGLFCAA|2055-02-08|1861|8093|621|2055|1|2|8|1|2055|621|8093|Monday|2055Q1|N|N|N|2471665|2471695|2471307|2471580|N|N|N|N|N| +2471673|AAAAAAAAJPGLFCAA|2055-02-09|1861|8094|621|2055|2|2|9|1|2055|621|8094|Tuesday|2055Q1|N|N|N|2471665|2471695|2471308|2471581|N|N|N|N|N| +2471674|AAAAAAAAKPGLFCAA|2055-02-10|1861|8094|621|2055|3|2|10|1|2055|621|8094|Wednesday|2055Q1|N|N|N|2471665|2471695|2471309|2471582|N|N|N|N|N| +2471675|AAAAAAAALPGLFCAA|2055-02-11|1861|8094|621|2055|4|2|11|1|2055|621|8094|Thursday|2055Q1|N|N|N|2471665|2471695|2471310|2471583|N|N|N|N|N| +2471676|AAAAAAAAMPGLFCAA|2055-02-12|1861|8094|621|2055|5|2|12|1|2055|621|8094|Friday|2055Q1|N|Y|N|2471665|2471695|2471311|2471584|N|N|N|N|N| +2471677|AAAAAAAANPGLFCAA|2055-02-13|1861|8094|621|2055|6|2|13|1|2055|621|8094|Saturday|2055Q1|N|Y|N|2471665|2471695|2471312|2471585|N|N|N|N|N| +2471678|AAAAAAAAOPGLFCAA|2055-02-14|1861|8094|621|2055|0|2|14|1|2055|621|8094|Sunday|2055Q1|N|N|N|2471665|2471695|2471313|2471586|N|N|N|N|N| +2471679|AAAAAAAAPPGLFCAA|2055-02-15|1861|8094|621|2055|1|2|15|1|2055|621|8094|Monday|2055Q1|N|N|N|2471665|2471695|2471314|2471587|N|N|N|N|N| +2471680|AAAAAAAAAAHLFCAA|2055-02-16|1861|8095|621|2055|2|2|16|1|2055|621|8095|Tuesday|2055Q1|N|N|N|2471665|2471695|2471315|2471588|N|N|N|N|N| +2471681|AAAAAAAABAHLFCAA|2055-02-17|1861|8095|621|2055|3|2|17|1|2055|621|8095|Wednesday|2055Q1|N|N|N|2471665|2471695|2471316|2471589|N|N|N|N|N| +2471682|AAAAAAAACAHLFCAA|2055-02-18|1861|8095|621|2055|4|2|18|1|2055|621|8095|Thursday|2055Q1|N|N|N|2471665|2471695|2471317|2471590|N|N|N|N|N| +2471683|AAAAAAAADAHLFCAA|2055-02-19|1861|8095|621|2055|5|2|19|1|2055|621|8095|Friday|2055Q1|N|Y|N|2471665|2471695|2471318|2471591|N|N|N|N|N| +2471684|AAAAAAAAEAHLFCAA|2055-02-20|1861|8095|621|2055|6|2|20|1|2055|621|8095|Saturday|2055Q1|N|Y|N|2471665|2471695|2471319|2471592|N|N|N|N|N| +2471685|AAAAAAAAFAHLFCAA|2055-02-21|1861|8095|621|2055|0|2|21|1|2055|621|8095|Sunday|2055Q1|N|N|N|2471665|2471695|2471320|2471593|N|N|N|N|N| +2471686|AAAAAAAAGAHLFCAA|2055-02-22|1861|8095|621|2055|1|2|22|1|2055|621|8095|Monday|2055Q1|N|N|N|2471665|2471695|2471321|2471594|N|N|N|N|N| +2471687|AAAAAAAAHAHLFCAA|2055-02-23|1861|8096|621|2055|2|2|23|1|2055|621|8096|Tuesday|2055Q1|N|N|N|2471665|2471695|2471322|2471595|N|N|N|N|N| +2471688|AAAAAAAAIAHLFCAA|2055-02-24|1861|8096|621|2055|3|2|24|1|2055|621|8096|Wednesday|2055Q1|N|N|N|2471665|2471695|2471323|2471596|N|N|N|N|N| +2471689|AAAAAAAAJAHLFCAA|2055-02-25|1861|8096|621|2055|4|2|25|1|2055|621|8096|Thursday|2055Q1|N|N|N|2471665|2471695|2471324|2471597|N|N|N|N|N| +2471690|AAAAAAAAKAHLFCAA|2055-02-26|1861|8096|621|2055|5|2|26|1|2055|621|8096|Friday|2055Q1|N|Y|N|2471665|2471695|2471325|2471598|N|N|N|N|N| +2471691|AAAAAAAALAHLFCAA|2055-02-27|1861|8096|621|2055|6|2|27|1|2055|621|8096|Saturday|2055Q1|N|Y|N|2471665|2471695|2471326|2471599|N|N|N|N|N| +2471692|AAAAAAAAMAHLFCAA|2055-02-28|1861|8096|621|2055|0|2|28|1|2055|621|8096|Sunday|2055Q1|N|N|N|2471665|2471695|2471327|2471600|N|N|N|N|N| +2471693|AAAAAAAANAHLFCAA|2055-03-01|1862|8096|622|2055|1|3|1|1|2055|622|8096|Monday|2055Q1|N|N|N|2471693|2471751|2471328|2471601|N|N|N|N|N| +2471694|AAAAAAAAOAHLFCAA|2055-03-02|1862|8097|622|2055|2|3|2|1|2055|622|8097|Tuesday|2055Q1|N|N|N|2471693|2471751|2471329|2471602|N|N|N|N|N| +2471695|AAAAAAAAPAHLFCAA|2055-03-03|1862|8097|622|2055|3|3|3|1|2055|622|8097|Wednesday|2055Q1|N|N|N|2471693|2471751|2471330|2471603|N|N|N|N|N| +2471696|AAAAAAAAABHLFCAA|2055-03-04|1862|8097|622|2055|4|3|4|1|2055|622|8097|Thursday|2055Q1|N|N|N|2471693|2471751|2471331|2471604|N|N|N|N|N| +2471697|AAAAAAAABBHLFCAA|2055-03-05|1862|8097|622|2055|5|3|5|1|2055|622|8097|Friday|2055Q1|N|Y|N|2471693|2471751|2471332|2471605|N|N|N|N|N| +2471698|AAAAAAAACBHLFCAA|2055-03-06|1862|8097|622|2055|6|3|6|1|2055|622|8097|Saturday|2055Q1|N|Y|N|2471693|2471751|2471333|2471606|N|N|N|N|N| +2471699|AAAAAAAADBHLFCAA|2055-03-07|1862|8097|622|2055|0|3|7|1|2055|622|8097|Sunday|2055Q1|N|N|N|2471693|2471751|2471334|2471607|N|N|N|N|N| +2471700|AAAAAAAAEBHLFCAA|2055-03-08|1862|8097|622|2055|1|3|8|1|2055|622|8097|Monday|2055Q1|N|N|N|2471693|2471751|2471335|2471608|N|N|N|N|N| +2471701|AAAAAAAAFBHLFCAA|2055-03-09|1862|8098|622|2055|2|3|9|1|2055|622|8098|Tuesday|2055Q1|N|N|N|2471693|2471751|2471336|2471609|N|N|N|N|N| +2471702|AAAAAAAAGBHLFCAA|2055-03-10|1862|8098|622|2055|3|3|10|1|2055|622|8098|Wednesday|2055Q1|N|N|N|2471693|2471751|2471337|2471610|N|N|N|N|N| +2471703|AAAAAAAAHBHLFCAA|2055-03-11|1862|8098|622|2055|4|3|11|1|2055|622|8098|Thursday|2055Q1|N|N|N|2471693|2471751|2471338|2471611|N|N|N|N|N| +2471704|AAAAAAAAIBHLFCAA|2055-03-12|1862|8098|622|2055|5|3|12|1|2055|622|8098|Friday|2055Q1|N|Y|N|2471693|2471751|2471339|2471612|N|N|N|N|N| +2471705|AAAAAAAAJBHLFCAA|2055-03-13|1862|8098|622|2055|6|3|13|1|2055|622|8098|Saturday|2055Q1|N|Y|N|2471693|2471751|2471340|2471613|N|N|N|N|N| +2471706|AAAAAAAAKBHLFCAA|2055-03-14|1862|8098|622|2055|0|3|14|1|2055|622|8098|Sunday|2055Q1|N|N|N|2471693|2471751|2471341|2471614|N|N|N|N|N| +2471707|AAAAAAAALBHLFCAA|2055-03-15|1862|8098|622|2055|1|3|15|1|2055|622|8098|Monday|2055Q1|N|N|N|2471693|2471751|2471342|2471615|N|N|N|N|N| +2471708|AAAAAAAAMBHLFCAA|2055-03-16|1862|8099|622|2055|2|3|16|1|2055|622|8099|Tuesday|2055Q1|N|N|N|2471693|2471751|2471343|2471616|N|N|N|N|N| +2471709|AAAAAAAANBHLFCAA|2055-03-17|1862|8099|622|2055|3|3|17|1|2055|622|8099|Wednesday|2055Q1|N|N|N|2471693|2471751|2471344|2471617|N|N|N|N|N| +2471710|AAAAAAAAOBHLFCAA|2055-03-18|1862|8099|622|2055|4|3|18|1|2055|622|8099|Thursday|2055Q1|N|N|N|2471693|2471751|2471345|2471618|N|N|N|N|N| +2471711|AAAAAAAAPBHLFCAA|2055-03-19|1862|8099|622|2055|5|3|19|1|2055|622|8099|Friday|2055Q1|N|Y|N|2471693|2471751|2471346|2471619|N|N|N|N|N| +2471712|AAAAAAAAACHLFCAA|2055-03-20|1862|8099|622|2055|6|3|20|1|2055|622|8099|Saturday|2055Q1|N|Y|N|2471693|2471751|2471347|2471620|N|N|N|N|N| +2471713|AAAAAAAABCHLFCAA|2055-03-21|1862|8099|622|2055|0|3|21|1|2055|622|8099|Sunday|2055Q1|N|N|N|2471693|2471751|2471348|2471621|N|N|N|N|N| +2471714|AAAAAAAACCHLFCAA|2055-03-22|1862|8099|622|2055|1|3|22|1|2055|622|8099|Monday|2055Q1|N|N|N|2471693|2471751|2471349|2471622|N|N|N|N|N| +2471715|AAAAAAAADCHLFCAA|2055-03-23|1862|8100|622|2055|2|3|23|1|2055|622|8100|Tuesday|2055Q1|N|N|N|2471693|2471751|2471350|2471623|N|N|N|N|N| +2471716|AAAAAAAAECHLFCAA|2055-03-24|1862|8100|622|2055|3|3|24|1|2055|622|8100|Wednesday|2055Q1|N|N|N|2471693|2471751|2471351|2471624|N|N|N|N|N| +2471717|AAAAAAAAFCHLFCAA|2055-03-25|1862|8100|622|2055|4|3|25|1|2055|622|8100|Thursday|2055Q1|N|N|N|2471693|2471751|2471352|2471625|N|N|N|N|N| +2471718|AAAAAAAAGCHLFCAA|2055-03-26|1862|8100|622|2055|5|3|26|1|2055|622|8100|Friday|2055Q1|N|Y|N|2471693|2471751|2471353|2471626|N|N|N|N|N| +2471719|AAAAAAAAHCHLFCAA|2055-03-27|1862|8100|622|2055|6|3|27|1|2055|622|8100|Saturday|2055Q1|N|Y|N|2471693|2471751|2471354|2471627|N|N|N|N|N| +2471720|AAAAAAAAICHLFCAA|2055-03-28|1862|8100|622|2055|0|3|28|1|2055|622|8100|Sunday|2055Q1|N|N|N|2471693|2471751|2471355|2471628|N|N|N|N|N| +2471721|AAAAAAAAJCHLFCAA|2055-03-29|1862|8100|622|2055|1|3|29|1|2055|622|8100|Monday|2055Q1|N|N|N|2471693|2471751|2471356|2471629|N|N|N|N|N| +2471722|AAAAAAAAKCHLFCAA|2055-03-30|1862|8101|622|2055|2|3|30|1|2055|622|8101|Tuesday|2055Q1|N|N|N|2471693|2471751|2471357|2471630|N|N|N|N|N| +2471723|AAAAAAAALCHLFCAA|2055-03-31|1862|8101|622|2055|3|3|31|1|2055|622|8101|Wednesday|2055Q1|N|N|N|2471693|2471751|2471358|2471631|N|N|N|N|N| +2471724|AAAAAAAAMCHLFCAA|2055-04-01|1863|8101|622|2055|4|4|1|1|2055|622|8101|Thursday|2055Q1|N|N|N|2471724|2471813|2471359|2471634|N|N|N|N|N| +2471725|AAAAAAAANCHLFCAA|2055-04-02|1863|8101|622|2055|5|4|2|2|2055|622|8101|Friday|2055Q2|N|Y|N|2471724|2471813|2471360|2471635|N|N|N|N|N| +2471726|AAAAAAAAOCHLFCAA|2055-04-03|1863|8101|622|2055|6|4|3|2|2055|622|8101|Saturday|2055Q2|N|Y|N|2471724|2471813|2471361|2471636|N|N|N|N|N| +2471727|AAAAAAAAPCHLFCAA|2055-04-04|1863|8101|622|2055|0|4|4|2|2055|622|8101|Sunday|2055Q2|N|N|N|2471724|2471813|2471362|2471637|N|N|N|N|N| +2471728|AAAAAAAAADHLFCAA|2055-04-05|1863|8101|622|2055|1|4|5|2|2055|622|8101|Monday|2055Q2|N|N|N|2471724|2471813|2471363|2471638|N|N|N|N|N| +2471729|AAAAAAAABDHLFCAA|2055-04-06|1863|8102|622|2055|2|4|6|2|2055|622|8102|Tuesday|2055Q2|N|N|N|2471724|2471813|2471364|2471639|N|N|N|N|N| +2471730|AAAAAAAACDHLFCAA|2055-04-07|1863|8102|622|2055|3|4|7|2|2055|622|8102|Wednesday|2055Q2|N|N|N|2471724|2471813|2471365|2471640|N|N|N|N|N| +2471731|AAAAAAAADDHLFCAA|2055-04-08|1863|8102|622|2055|4|4|8|2|2055|622|8102|Thursday|2055Q2|N|N|N|2471724|2471813|2471366|2471641|N|N|N|N|N| +2471732|AAAAAAAAEDHLFCAA|2055-04-09|1863|8102|622|2055|5|4|9|2|2055|622|8102|Friday|2055Q2|N|Y|N|2471724|2471813|2471367|2471642|N|N|N|N|N| +2471733|AAAAAAAAFDHLFCAA|2055-04-10|1863|8102|622|2055|6|4|10|2|2055|622|8102|Saturday|2055Q2|N|Y|N|2471724|2471813|2471368|2471643|N|N|N|N|N| +2471734|AAAAAAAAGDHLFCAA|2055-04-11|1863|8102|622|2055|0|4|11|2|2055|622|8102|Sunday|2055Q2|N|N|N|2471724|2471813|2471369|2471644|N|N|N|N|N| +2471735|AAAAAAAAHDHLFCAA|2055-04-12|1863|8102|622|2055|1|4|12|2|2055|622|8102|Monday|2055Q2|N|N|N|2471724|2471813|2471370|2471645|N|N|N|N|N| +2471736|AAAAAAAAIDHLFCAA|2055-04-13|1863|8103|622|2055|2|4|13|2|2055|622|8103|Tuesday|2055Q2|N|N|N|2471724|2471813|2471371|2471646|N|N|N|N|N| +2471737|AAAAAAAAJDHLFCAA|2055-04-14|1863|8103|622|2055|3|4|14|2|2055|622|8103|Wednesday|2055Q2|N|N|N|2471724|2471813|2471372|2471647|N|N|N|N|N| +2471738|AAAAAAAAKDHLFCAA|2055-04-15|1863|8103|622|2055|4|4|15|2|2055|622|8103|Thursday|2055Q2|N|N|N|2471724|2471813|2471373|2471648|N|N|N|N|N| +2471739|AAAAAAAALDHLFCAA|2055-04-16|1863|8103|622|2055|5|4|16|2|2055|622|8103|Friday|2055Q2|N|Y|N|2471724|2471813|2471374|2471649|N|N|N|N|N| +2471740|AAAAAAAAMDHLFCAA|2055-04-17|1863|8103|622|2055|6|4|17|2|2055|622|8103|Saturday|2055Q2|N|Y|N|2471724|2471813|2471375|2471650|N|N|N|N|N| +2471741|AAAAAAAANDHLFCAA|2055-04-18|1863|8103|622|2055|0|4|18|2|2055|622|8103|Sunday|2055Q2|N|N|N|2471724|2471813|2471376|2471651|N|N|N|N|N| +2471742|AAAAAAAAODHLFCAA|2055-04-19|1863|8103|622|2055|1|4|19|2|2055|622|8103|Monday|2055Q2|N|N|N|2471724|2471813|2471377|2471652|N|N|N|N|N| +2471743|AAAAAAAAPDHLFCAA|2055-04-20|1863|8104|622|2055|2|4|20|2|2055|622|8104|Tuesday|2055Q2|N|N|N|2471724|2471813|2471378|2471653|N|N|N|N|N| +2471744|AAAAAAAAAEHLFCAA|2055-04-21|1863|8104|622|2055|3|4|21|2|2055|622|8104|Wednesday|2055Q2|N|N|N|2471724|2471813|2471379|2471654|N|N|N|N|N| +2471745|AAAAAAAABEHLFCAA|2055-04-22|1863|8104|622|2055|4|4|22|2|2055|622|8104|Thursday|2055Q2|N|N|N|2471724|2471813|2471380|2471655|N|N|N|N|N| +2471746|AAAAAAAACEHLFCAA|2055-04-23|1863|8104|622|2055|5|4|23|2|2055|622|8104|Friday|2055Q2|N|Y|N|2471724|2471813|2471381|2471656|N|N|N|N|N| +2471747|AAAAAAAADEHLFCAA|2055-04-24|1863|8104|622|2055|6|4|24|2|2055|622|8104|Saturday|2055Q2|N|Y|N|2471724|2471813|2471382|2471657|N|N|N|N|N| +2471748|AAAAAAAAEEHLFCAA|2055-04-25|1863|8104|622|2055|0|4|25|2|2055|622|8104|Sunday|2055Q2|N|N|N|2471724|2471813|2471383|2471658|N|N|N|N|N| +2471749|AAAAAAAAFEHLFCAA|2055-04-26|1863|8104|622|2055|1|4|26|2|2055|622|8104|Monday|2055Q2|N|N|N|2471724|2471813|2471384|2471659|N|N|N|N|N| +2471750|AAAAAAAAGEHLFCAA|2055-04-27|1863|8105|622|2055|2|4|27|2|2055|622|8105|Tuesday|2055Q2|N|N|N|2471724|2471813|2471385|2471660|N|N|N|N|N| +2471751|AAAAAAAAHEHLFCAA|2055-04-28|1863|8105|622|2055|3|4|28|2|2055|622|8105|Wednesday|2055Q2|N|N|N|2471724|2471813|2471386|2471661|N|N|N|N|N| +2471752|AAAAAAAAIEHLFCAA|2055-04-29|1863|8105|622|2055|4|4|29|2|2055|622|8105|Thursday|2055Q2|N|N|N|2471724|2471813|2471387|2471662|N|N|N|N|N| +2471753|AAAAAAAAJEHLFCAA|2055-04-30|1863|8105|622|2055|5|4|30|2|2055|622|8105|Friday|2055Q2|N|Y|N|2471724|2471813|2471388|2471663|N|N|N|N|N| +2471754|AAAAAAAAKEHLFCAA|2055-05-01|1864|8105|622|2055|6|5|1|2|2055|622|8105|Saturday|2055Q2|N|Y|N|2471754|2471873|2471389|2471664|N|N|N|N|N| +2471755|AAAAAAAALEHLFCAA|2055-05-02|1864|8105|622|2055|0|5|2|2|2055|622|8105|Sunday|2055Q2|N|N|N|2471754|2471873|2471390|2471665|N|N|N|N|N| +2471756|AAAAAAAAMEHLFCAA|2055-05-03|1864|8105|622|2055|1|5|3|2|2055|622|8105|Monday|2055Q2|N|N|N|2471754|2471873|2471391|2471666|N|N|N|N|N| +2471757|AAAAAAAANEHLFCAA|2055-05-04|1864|8106|622|2055|2|5|4|2|2055|622|8106|Tuesday|2055Q2|N|N|N|2471754|2471873|2471392|2471667|N|N|N|N|N| +2471758|AAAAAAAAOEHLFCAA|2055-05-05|1864|8106|622|2055|3|5|5|2|2055|622|8106|Wednesday|2055Q2|N|N|N|2471754|2471873|2471393|2471668|N|N|N|N|N| +2471759|AAAAAAAAPEHLFCAA|2055-05-06|1864|8106|622|2055|4|5|6|2|2055|622|8106|Thursday|2055Q2|N|N|N|2471754|2471873|2471394|2471669|N|N|N|N|N| +2471760|AAAAAAAAAFHLFCAA|2055-05-07|1864|8106|622|2055|5|5|7|2|2055|622|8106|Friday|2055Q2|N|Y|N|2471754|2471873|2471395|2471670|N|N|N|N|N| +2471761|AAAAAAAABFHLFCAA|2055-05-08|1864|8106|622|2055|6|5|8|2|2055|622|8106|Saturday|2055Q2|N|Y|N|2471754|2471873|2471396|2471671|N|N|N|N|N| +2471762|AAAAAAAACFHLFCAA|2055-05-09|1864|8106|622|2055|0|5|9|2|2055|622|8106|Sunday|2055Q2|N|N|N|2471754|2471873|2471397|2471672|N|N|N|N|N| +2471763|AAAAAAAADFHLFCAA|2055-05-10|1864|8106|622|2055|1|5|10|2|2055|622|8106|Monday|2055Q2|N|N|N|2471754|2471873|2471398|2471673|N|N|N|N|N| +2471764|AAAAAAAAEFHLFCAA|2055-05-11|1864|8107|622|2055|2|5|11|2|2055|622|8107|Tuesday|2055Q2|N|N|N|2471754|2471873|2471399|2471674|N|N|N|N|N| +2471765|AAAAAAAAFFHLFCAA|2055-05-12|1864|8107|622|2055|3|5|12|2|2055|622|8107|Wednesday|2055Q2|N|N|N|2471754|2471873|2471400|2471675|N|N|N|N|N| +2471766|AAAAAAAAGFHLFCAA|2055-05-13|1864|8107|622|2055|4|5|13|2|2055|622|8107|Thursday|2055Q2|N|N|N|2471754|2471873|2471401|2471676|N|N|N|N|N| +2471767|AAAAAAAAHFHLFCAA|2055-05-14|1864|8107|622|2055|5|5|14|2|2055|622|8107|Friday|2055Q2|N|Y|N|2471754|2471873|2471402|2471677|N|N|N|N|N| +2471768|AAAAAAAAIFHLFCAA|2055-05-15|1864|8107|622|2055|6|5|15|2|2055|622|8107|Saturday|2055Q2|N|Y|N|2471754|2471873|2471403|2471678|N|N|N|N|N| +2471769|AAAAAAAAJFHLFCAA|2055-05-16|1864|8107|622|2055|0|5|16|2|2055|622|8107|Sunday|2055Q2|N|N|N|2471754|2471873|2471404|2471679|N|N|N|N|N| +2471770|AAAAAAAAKFHLFCAA|2055-05-17|1864|8107|622|2055|1|5|17|2|2055|622|8107|Monday|2055Q2|N|N|N|2471754|2471873|2471405|2471680|N|N|N|N|N| +2471771|AAAAAAAALFHLFCAA|2055-05-18|1864|8108|622|2055|2|5|18|2|2055|622|8108|Tuesday|2055Q2|N|N|N|2471754|2471873|2471406|2471681|N|N|N|N|N| +2471772|AAAAAAAAMFHLFCAA|2055-05-19|1864|8108|622|2055|3|5|19|2|2055|622|8108|Wednesday|2055Q2|N|N|N|2471754|2471873|2471407|2471682|N|N|N|N|N| +2471773|AAAAAAAANFHLFCAA|2055-05-20|1864|8108|622|2055|4|5|20|2|2055|622|8108|Thursday|2055Q2|N|N|N|2471754|2471873|2471408|2471683|N|N|N|N|N| +2471774|AAAAAAAAOFHLFCAA|2055-05-21|1864|8108|622|2055|5|5|21|2|2055|622|8108|Friday|2055Q2|N|Y|N|2471754|2471873|2471409|2471684|N|N|N|N|N| +2471775|AAAAAAAAPFHLFCAA|2055-05-22|1864|8108|622|2055|6|5|22|2|2055|622|8108|Saturday|2055Q2|N|Y|N|2471754|2471873|2471410|2471685|N|N|N|N|N| +2471776|AAAAAAAAAGHLFCAA|2055-05-23|1864|8108|622|2055|0|5|23|2|2055|622|8108|Sunday|2055Q2|N|N|N|2471754|2471873|2471411|2471686|N|N|N|N|N| +2471777|AAAAAAAABGHLFCAA|2055-05-24|1864|8108|622|2055|1|5|24|2|2055|622|8108|Monday|2055Q2|N|N|N|2471754|2471873|2471412|2471687|N|N|N|N|N| +2471778|AAAAAAAACGHLFCAA|2055-05-25|1864|8109|622|2055|2|5|25|2|2055|622|8109|Tuesday|2055Q2|N|N|N|2471754|2471873|2471413|2471688|N|N|N|N|N| +2471779|AAAAAAAADGHLFCAA|2055-05-26|1864|8109|622|2055|3|5|26|2|2055|622|8109|Wednesday|2055Q2|N|N|N|2471754|2471873|2471414|2471689|N|N|N|N|N| +2471780|AAAAAAAAEGHLFCAA|2055-05-27|1864|8109|622|2055|4|5|27|2|2055|622|8109|Thursday|2055Q2|N|N|N|2471754|2471873|2471415|2471690|N|N|N|N|N| +2471781|AAAAAAAAFGHLFCAA|2055-05-28|1864|8109|622|2055|5|5|28|2|2055|622|8109|Friday|2055Q2|N|Y|N|2471754|2471873|2471416|2471691|N|N|N|N|N| +2471782|AAAAAAAAGGHLFCAA|2055-05-29|1864|8109|622|2055|6|5|29|2|2055|622|8109|Saturday|2055Q2|N|Y|N|2471754|2471873|2471417|2471692|N|N|N|N|N| +2471783|AAAAAAAAHGHLFCAA|2055-05-30|1864|8109|622|2055|0|5|30|2|2055|622|8109|Sunday|2055Q2|N|N|N|2471754|2471873|2471418|2471693|N|N|N|N|N| +2471784|AAAAAAAAIGHLFCAA|2055-05-31|1864|8109|622|2055|1|5|31|2|2055|622|8109|Monday|2055Q2|N|N|N|2471754|2471873|2471419|2471694|N|N|N|N|N| +2471785|AAAAAAAAJGHLFCAA|2055-06-01|1865|8110|623|2055|2|6|1|2|2055|623|8110|Tuesday|2055Q2|N|N|N|2471785|2471935|2471420|2471695|N|N|N|N|N| +2471786|AAAAAAAAKGHLFCAA|2055-06-02|1865|8110|623|2055|3|6|2|2|2055|623|8110|Wednesday|2055Q2|N|N|N|2471785|2471935|2471421|2471696|N|N|N|N|N| +2471787|AAAAAAAALGHLFCAA|2055-06-03|1865|8110|623|2055|4|6|3|2|2055|623|8110|Thursday|2055Q2|N|N|N|2471785|2471935|2471422|2471697|N|N|N|N|N| +2471788|AAAAAAAAMGHLFCAA|2055-06-04|1865|8110|623|2055|5|6|4|2|2055|623|8110|Friday|2055Q2|N|Y|N|2471785|2471935|2471423|2471698|N|N|N|N|N| +2471789|AAAAAAAANGHLFCAA|2055-06-05|1865|8110|623|2055|6|6|5|2|2055|623|8110|Saturday|2055Q2|N|Y|N|2471785|2471935|2471424|2471699|N|N|N|N|N| +2471790|AAAAAAAAOGHLFCAA|2055-06-06|1865|8110|623|2055|0|6|6|2|2055|623|8110|Sunday|2055Q2|N|N|N|2471785|2471935|2471425|2471700|N|N|N|N|N| +2471791|AAAAAAAAPGHLFCAA|2055-06-07|1865|8110|623|2055|1|6|7|2|2055|623|8110|Monday|2055Q2|N|N|N|2471785|2471935|2471426|2471701|N|N|N|N|N| +2471792|AAAAAAAAAHHLFCAA|2055-06-08|1865|8111|623|2055|2|6|8|2|2055|623|8111|Tuesday|2055Q2|N|N|N|2471785|2471935|2471427|2471702|N|N|N|N|N| +2471793|AAAAAAAABHHLFCAA|2055-06-09|1865|8111|623|2055|3|6|9|2|2055|623|8111|Wednesday|2055Q2|N|N|N|2471785|2471935|2471428|2471703|N|N|N|N|N| +2471794|AAAAAAAACHHLFCAA|2055-06-10|1865|8111|623|2055|4|6|10|2|2055|623|8111|Thursday|2055Q2|N|N|N|2471785|2471935|2471429|2471704|N|N|N|N|N| +2471795|AAAAAAAADHHLFCAA|2055-06-11|1865|8111|623|2055|5|6|11|2|2055|623|8111|Friday|2055Q2|N|Y|N|2471785|2471935|2471430|2471705|N|N|N|N|N| +2471796|AAAAAAAAEHHLFCAA|2055-06-12|1865|8111|623|2055|6|6|12|2|2055|623|8111|Saturday|2055Q2|N|Y|N|2471785|2471935|2471431|2471706|N|N|N|N|N| +2471797|AAAAAAAAFHHLFCAA|2055-06-13|1865|8111|623|2055|0|6|13|2|2055|623|8111|Sunday|2055Q2|N|N|N|2471785|2471935|2471432|2471707|N|N|N|N|N| +2471798|AAAAAAAAGHHLFCAA|2055-06-14|1865|8111|623|2055|1|6|14|2|2055|623|8111|Monday|2055Q2|N|N|N|2471785|2471935|2471433|2471708|N|N|N|N|N| +2471799|AAAAAAAAHHHLFCAA|2055-06-15|1865|8112|623|2055|2|6|15|2|2055|623|8112|Tuesday|2055Q2|N|N|N|2471785|2471935|2471434|2471709|N|N|N|N|N| +2471800|AAAAAAAAIHHLFCAA|2055-06-16|1865|8112|623|2055|3|6|16|2|2055|623|8112|Wednesday|2055Q2|N|N|N|2471785|2471935|2471435|2471710|N|N|N|N|N| +2471801|AAAAAAAAJHHLFCAA|2055-06-17|1865|8112|623|2055|4|6|17|2|2055|623|8112|Thursday|2055Q2|N|N|N|2471785|2471935|2471436|2471711|N|N|N|N|N| +2471802|AAAAAAAAKHHLFCAA|2055-06-18|1865|8112|623|2055|5|6|18|2|2055|623|8112|Friday|2055Q2|N|Y|N|2471785|2471935|2471437|2471712|N|N|N|N|N| +2471803|AAAAAAAALHHLFCAA|2055-06-19|1865|8112|623|2055|6|6|19|2|2055|623|8112|Saturday|2055Q2|N|Y|N|2471785|2471935|2471438|2471713|N|N|N|N|N| +2471804|AAAAAAAAMHHLFCAA|2055-06-20|1865|8112|623|2055|0|6|20|2|2055|623|8112|Sunday|2055Q2|N|N|N|2471785|2471935|2471439|2471714|N|N|N|N|N| +2471805|AAAAAAAANHHLFCAA|2055-06-21|1865|8112|623|2055|1|6|21|2|2055|623|8112|Monday|2055Q2|N|N|N|2471785|2471935|2471440|2471715|N|N|N|N|N| +2471806|AAAAAAAAOHHLFCAA|2055-06-22|1865|8113|623|2055|2|6|22|2|2055|623|8113|Tuesday|2055Q2|N|N|N|2471785|2471935|2471441|2471716|N|N|N|N|N| +2471807|AAAAAAAAPHHLFCAA|2055-06-23|1865|8113|623|2055|3|6|23|2|2055|623|8113|Wednesday|2055Q2|N|N|N|2471785|2471935|2471442|2471717|N|N|N|N|N| +2471808|AAAAAAAAAIHLFCAA|2055-06-24|1865|8113|623|2055|4|6|24|2|2055|623|8113|Thursday|2055Q2|N|N|N|2471785|2471935|2471443|2471718|N|N|N|N|N| +2471809|AAAAAAAABIHLFCAA|2055-06-25|1865|8113|623|2055|5|6|25|2|2055|623|8113|Friday|2055Q2|N|Y|N|2471785|2471935|2471444|2471719|N|N|N|N|N| +2471810|AAAAAAAACIHLFCAA|2055-06-26|1865|8113|623|2055|6|6|26|2|2055|623|8113|Saturday|2055Q2|N|Y|N|2471785|2471935|2471445|2471720|N|N|N|N|N| +2471811|AAAAAAAADIHLFCAA|2055-06-27|1865|8113|623|2055|0|6|27|2|2055|623|8113|Sunday|2055Q2|N|N|N|2471785|2471935|2471446|2471721|N|N|N|N|N| +2471812|AAAAAAAAEIHLFCAA|2055-06-28|1865|8113|623|2055|1|6|28|2|2055|623|8113|Monday|2055Q2|N|N|N|2471785|2471935|2471447|2471722|N|N|N|N|N| +2471813|AAAAAAAAFIHLFCAA|2055-06-29|1865|8114|623|2055|2|6|29|2|2055|623|8114|Tuesday|2055Q2|N|N|N|2471785|2471935|2471448|2471723|N|N|N|N|N| +2471814|AAAAAAAAGIHLFCAA|2055-06-30|1865|8114|623|2055|3|6|30|2|2055|623|8114|Wednesday|2055Q2|N|N|N|2471785|2471935|2471449|2471724|N|N|N|N|N| +2471815|AAAAAAAAHIHLFCAA|2055-07-01|1866|8114|623|2055|4|7|1|2|2055|623|8114|Thursday|2055Q2|N|N|N|2471815|2471995|2471450|2471724|N|N|N|N|N| +2471816|AAAAAAAAIIHLFCAA|2055-07-02|1866|8114|623|2055|5|7|2|3|2055|623|8114|Friday|2055Q3|N|Y|N|2471815|2471995|2471451|2471725|N|N|N|N|N| +2471817|AAAAAAAAJIHLFCAA|2055-07-03|1866|8114|623|2055|6|7|3|3|2055|623|8114|Saturday|2055Q3|N|Y|N|2471815|2471995|2471452|2471726|N|N|N|N|N| +2471818|AAAAAAAAKIHLFCAA|2055-07-04|1866|8114|623|2055|0|7|4|3|2055|623|8114|Sunday|2055Q3|N|N|N|2471815|2471995|2471453|2471727|N|N|N|N|N| +2471819|AAAAAAAALIHLFCAA|2055-07-05|1866|8114|623|2055|1|7|5|3|2055|623|8114|Monday|2055Q3|Y|N|N|2471815|2471995|2471454|2471728|N|N|N|N|N| +2471820|AAAAAAAAMIHLFCAA|2055-07-06|1866|8115|623|2055|2|7|6|3|2055|623|8115|Tuesday|2055Q3|N|N|Y|2471815|2471995|2471455|2471729|N|N|N|N|N| +2471821|AAAAAAAANIHLFCAA|2055-07-07|1866|8115|623|2055|3|7|7|3|2055|623|8115|Wednesday|2055Q3|N|N|N|2471815|2471995|2471456|2471730|N|N|N|N|N| +2471822|AAAAAAAAOIHLFCAA|2055-07-08|1866|8115|623|2055|4|7|8|3|2055|623|8115|Thursday|2055Q3|N|N|N|2471815|2471995|2471457|2471731|N|N|N|N|N| +2471823|AAAAAAAAPIHLFCAA|2055-07-09|1866|8115|623|2055|5|7|9|3|2055|623|8115|Friday|2055Q3|N|Y|N|2471815|2471995|2471458|2471732|N|N|N|N|N| +2471824|AAAAAAAAAJHLFCAA|2055-07-10|1866|8115|623|2055|6|7|10|3|2055|623|8115|Saturday|2055Q3|N|Y|N|2471815|2471995|2471459|2471733|N|N|N|N|N| +2471825|AAAAAAAABJHLFCAA|2055-07-11|1866|8115|623|2055|0|7|11|3|2055|623|8115|Sunday|2055Q3|N|N|N|2471815|2471995|2471460|2471734|N|N|N|N|N| +2471826|AAAAAAAACJHLFCAA|2055-07-12|1866|8115|623|2055|1|7|12|3|2055|623|8115|Monday|2055Q3|N|N|N|2471815|2471995|2471461|2471735|N|N|N|N|N| +2471827|AAAAAAAADJHLFCAA|2055-07-13|1866|8116|623|2055|2|7|13|3|2055|623|8116|Tuesday|2055Q3|N|N|N|2471815|2471995|2471462|2471736|N|N|N|N|N| +2471828|AAAAAAAAEJHLFCAA|2055-07-14|1866|8116|623|2055|3|7|14|3|2055|623|8116|Wednesday|2055Q3|N|N|N|2471815|2471995|2471463|2471737|N|N|N|N|N| +2471829|AAAAAAAAFJHLFCAA|2055-07-15|1866|8116|623|2055|4|7|15|3|2055|623|8116|Thursday|2055Q3|N|N|N|2471815|2471995|2471464|2471738|N|N|N|N|N| +2471830|AAAAAAAAGJHLFCAA|2055-07-16|1866|8116|623|2055|5|7|16|3|2055|623|8116|Friday|2055Q3|N|Y|N|2471815|2471995|2471465|2471739|N|N|N|N|N| +2471831|AAAAAAAAHJHLFCAA|2055-07-17|1866|8116|623|2055|6|7|17|3|2055|623|8116|Saturday|2055Q3|N|Y|N|2471815|2471995|2471466|2471740|N|N|N|N|N| +2471832|AAAAAAAAIJHLFCAA|2055-07-18|1866|8116|623|2055|0|7|18|3|2055|623|8116|Sunday|2055Q3|N|N|N|2471815|2471995|2471467|2471741|N|N|N|N|N| +2471833|AAAAAAAAJJHLFCAA|2055-07-19|1866|8116|623|2055|1|7|19|3|2055|623|8116|Monday|2055Q3|N|N|N|2471815|2471995|2471468|2471742|N|N|N|N|N| +2471834|AAAAAAAAKJHLFCAA|2055-07-20|1866|8117|623|2055|2|7|20|3|2055|623|8117|Tuesday|2055Q3|N|N|N|2471815|2471995|2471469|2471743|N|N|N|N|N| +2471835|AAAAAAAALJHLFCAA|2055-07-21|1866|8117|623|2055|3|7|21|3|2055|623|8117|Wednesday|2055Q3|N|N|N|2471815|2471995|2471470|2471744|N|N|N|N|N| +2471836|AAAAAAAAMJHLFCAA|2055-07-22|1866|8117|623|2055|4|7|22|3|2055|623|8117|Thursday|2055Q3|N|N|N|2471815|2471995|2471471|2471745|N|N|N|N|N| +2471837|AAAAAAAANJHLFCAA|2055-07-23|1866|8117|623|2055|5|7|23|3|2055|623|8117|Friday|2055Q3|N|Y|N|2471815|2471995|2471472|2471746|N|N|N|N|N| +2471838|AAAAAAAAOJHLFCAA|2055-07-24|1866|8117|623|2055|6|7|24|3|2055|623|8117|Saturday|2055Q3|N|Y|N|2471815|2471995|2471473|2471747|N|N|N|N|N| +2471839|AAAAAAAAPJHLFCAA|2055-07-25|1866|8117|623|2055|0|7|25|3|2055|623|8117|Sunday|2055Q3|N|N|N|2471815|2471995|2471474|2471748|N|N|N|N|N| +2471840|AAAAAAAAAKHLFCAA|2055-07-26|1866|8117|623|2055|1|7|26|3|2055|623|8117|Monday|2055Q3|N|N|N|2471815|2471995|2471475|2471749|N|N|N|N|N| +2471841|AAAAAAAABKHLFCAA|2055-07-27|1866|8118|623|2055|2|7|27|3|2055|623|8118|Tuesday|2055Q3|N|N|N|2471815|2471995|2471476|2471750|N|N|N|N|N| +2471842|AAAAAAAACKHLFCAA|2055-07-28|1866|8118|623|2055|3|7|28|3|2055|623|8118|Wednesday|2055Q3|N|N|N|2471815|2471995|2471477|2471751|N|N|N|N|N| +2471843|AAAAAAAADKHLFCAA|2055-07-29|1866|8118|623|2055|4|7|29|3|2055|623|8118|Thursday|2055Q3|N|N|N|2471815|2471995|2471478|2471752|N|N|N|N|N| +2471844|AAAAAAAAEKHLFCAA|2055-07-30|1866|8118|623|2055|5|7|30|3|2055|623|8118|Friday|2055Q3|N|Y|N|2471815|2471995|2471479|2471753|N|N|N|N|N| +2471845|AAAAAAAAFKHLFCAA|2055-07-31|1866|8118|623|2055|6|7|31|3|2055|623|8118|Saturday|2055Q3|N|Y|N|2471815|2471995|2471480|2471754|N|N|N|N|N| +2471846|AAAAAAAAGKHLFCAA|2055-08-01|1867|8118|623|2055|0|8|1|3|2055|623|8118|Sunday|2055Q3|N|N|N|2471846|2472057|2471481|2471755|N|N|N|N|N| +2471847|AAAAAAAAHKHLFCAA|2055-08-02|1867|8118|623|2055|1|8|2|3|2055|623|8118|Monday|2055Q3|N|N|N|2471846|2472057|2471482|2471756|N|N|N|N|N| +2471848|AAAAAAAAIKHLFCAA|2055-08-03|1867|8119|623|2055|2|8|3|3|2055|623|8119|Tuesday|2055Q3|N|N|N|2471846|2472057|2471483|2471757|N|N|N|N|N| +2471849|AAAAAAAAJKHLFCAA|2055-08-04|1867|8119|623|2055|3|8|4|3|2055|623|8119|Wednesday|2055Q3|N|N|N|2471846|2472057|2471484|2471758|N|N|N|N|N| +2471850|AAAAAAAAKKHLFCAA|2055-08-05|1867|8119|623|2055|4|8|5|3|2055|623|8119|Thursday|2055Q3|N|N|N|2471846|2472057|2471485|2471759|N|N|N|N|N| +2471851|AAAAAAAALKHLFCAA|2055-08-06|1867|8119|623|2055|5|8|6|3|2055|623|8119|Friday|2055Q3|N|Y|N|2471846|2472057|2471486|2471760|N|N|N|N|N| +2471852|AAAAAAAAMKHLFCAA|2055-08-07|1867|8119|623|2055|6|8|7|3|2055|623|8119|Saturday|2055Q3|N|Y|N|2471846|2472057|2471487|2471761|N|N|N|N|N| +2471853|AAAAAAAANKHLFCAA|2055-08-08|1867|8119|623|2055|0|8|8|3|2055|623|8119|Sunday|2055Q3|N|N|N|2471846|2472057|2471488|2471762|N|N|N|N|N| +2471854|AAAAAAAAOKHLFCAA|2055-08-09|1867|8119|623|2055|1|8|9|3|2055|623|8119|Monday|2055Q3|N|N|N|2471846|2472057|2471489|2471763|N|N|N|N|N| +2471855|AAAAAAAAPKHLFCAA|2055-08-10|1867|8120|623|2055|2|8|10|3|2055|623|8120|Tuesday|2055Q3|N|N|N|2471846|2472057|2471490|2471764|N|N|N|N|N| +2471856|AAAAAAAAALHLFCAA|2055-08-11|1867|8120|623|2055|3|8|11|3|2055|623|8120|Wednesday|2055Q3|N|N|N|2471846|2472057|2471491|2471765|N|N|N|N|N| +2471857|AAAAAAAABLHLFCAA|2055-08-12|1867|8120|623|2055|4|8|12|3|2055|623|8120|Thursday|2055Q3|N|N|N|2471846|2472057|2471492|2471766|N|N|N|N|N| +2471858|AAAAAAAACLHLFCAA|2055-08-13|1867|8120|623|2055|5|8|13|3|2055|623|8120|Friday|2055Q3|N|Y|N|2471846|2472057|2471493|2471767|N|N|N|N|N| +2471859|AAAAAAAADLHLFCAA|2055-08-14|1867|8120|623|2055|6|8|14|3|2055|623|8120|Saturday|2055Q3|N|Y|N|2471846|2472057|2471494|2471768|N|N|N|N|N| +2471860|AAAAAAAAELHLFCAA|2055-08-15|1867|8120|623|2055|0|8|15|3|2055|623|8120|Sunday|2055Q3|N|N|N|2471846|2472057|2471495|2471769|N|N|N|N|N| +2471861|AAAAAAAAFLHLFCAA|2055-08-16|1867|8120|623|2055|1|8|16|3|2055|623|8120|Monday|2055Q3|N|N|N|2471846|2472057|2471496|2471770|N|N|N|N|N| +2471862|AAAAAAAAGLHLFCAA|2055-08-17|1867|8121|623|2055|2|8|17|3|2055|623|8121|Tuesday|2055Q3|N|N|N|2471846|2472057|2471497|2471771|N|N|N|N|N| +2471863|AAAAAAAAHLHLFCAA|2055-08-18|1867|8121|623|2055|3|8|18|3|2055|623|8121|Wednesday|2055Q3|N|N|N|2471846|2472057|2471498|2471772|N|N|N|N|N| +2471864|AAAAAAAAILHLFCAA|2055-08-19|1867|8121|623|2055|4|8|19|3|2055|623|8121|Thursday|2055Q3|N|N|N|2471846|2472057|2471499|2471773|N|N|N|N|N| +2471865|AAAAAAAAJLHLFCAA|2055-08-20|1867|8121|623|2055|5|8|20|3|2055|623|8121|Friday|2055Q3|N|Y|N|2471846|2472057|2471500|2471774|N|N|N|N|N| +2471866|AAAAAAAAKLHLFCAA|2055-08-21|1867|8121|623|2055|6|8|21|3|2055|623|8121|Saturday|2055Q3|N|Y|N|2471846|2472057|2471501|2471775|N|N|N|N|N| +2471867|AAAAAAAALLHLFCAA|2055-08-22|1867|8121|623|2055|0|8|22|3|2055|623|8121|Sunday|2055Q3|N|N|N|2471846|2472057|2471502|2471776|N|N|N|N|N| +2471868|AAAAAAAAMLHLFCAA|2055-08-23|1867|8121|623|2055|1|8|23|3|2055|623|8121|Monday|2055Q3|N|N|N|2471846|2472057|2471503|2471777|N|N|N|N|N| +2471869|AAAAAAAANLHLFCAA|2055-08-24|1867|8122|623|2055|2|8|24|3|2055|623|8122|Tuesday|2055Q3|N|N|N|2471846|2472057|2471504|2471778|N|N|N|N|N| +2471870|AAAAAAAAOLHLFCAA|2055-08-25|1867|8122|623|2055|3|8|25|3|2055|623|8122|Wednesday|2055Q3|N|N|N|2471846|2472057|2471505|2471779|N|N|N|N|N| +2471871|AAAAAAAAPLHLFCAA|2055-08-26|1867|8122|623|2055|4|8|26|3|2055|623|8122|Thursday|2055Q3|N|N|N|2471846|2472057|2471506|2471780|N|N|N|N|N| +2471872|AAAAAAAAAMHLFCAA|2055-08-27|1867|8122|623|2055|5|8|27|3|2055|623|8122|Friday|2055Q3|N|Y|N|2471846|2472057|2471507|2471781|N|N|N|N|N| +2471873|AAAAAAAABMHLFCAA|2055-08-28|1867|8122|623|2055|6|8|28|3|2055|623|8122|Saturday|2055Q3|N|Y|N|2471846|2472057|2471508|2471782|N|N|N|N|N| +2471874|AAAAAAAACMHLFCAA|2055-08-29|1867|8122|623|2055|0|8|29|3|2055|623|8122|Sunday|2055Q3|N|N|N|2471846|2472057|2471509|2471783|N|N|N|N|N| +2471875|AAAAAAAADMHLFCAA|2055-08-30|1867|8122|623|2055|1|8|30|3|2055|623|8122|Monday|2055Q3|N|N|N|2471846|2472057|2471510|2471784|N|N|N|N|N| +2471876|AAAAAAAAEMHLFCAA|2055-08-31|1867|8123|623|2055|2|8|31|3|2055|623|8123|Tuesday|2055Q3|N|N|N|2471846|2472057|2471511|2471785|N|N|N|N|N| +2471877|AAAAAAAAFMHLFCAA|2055-09-01|1868|8123|624|2055|3|9|1|3|2055|624|8123|Wednesday|2055Q3|N|N|N|2471877|2472119|2471512|2471786|N|N|N|N|N| +2471878|AAAAAAAAGMHLFCAA|2055-09-02|1868|8123|624|2055|4|9|2|3|2055|624|8123|Thursday|2055Q3|N|N|N|2471877|2472119|2471513|2471787|N|N|N|N|N| +2471879|AAAAAAAAHMHLFCAA|2055-09-03|1868|8123|624|2055|5|9|3|3|2055|624|8123|Friday|2055Q3|N|Y|N|2471877|2472119|2471514|2471788|N|N|N|N|N| +2471880|AAAAAAAAIMHLFCAA|2055-09-04|1868|8123|624|2055|6|9|4|3|2055|624|8123|Saturday|2055Q3|N|Y|N|2471877|2472119|2471515|2471789|N|N|N|N|N| +2471881|AAAAAAAAJMHLFCAA|2055-09-05|1868|8123|624|2055|0|9|5|3|2055|624|8123|Sunday|2055Q3|N|N|N|2471877|2472119|2471516|2471790|N|N|N|N|N| +2471882|AAAAAAAAKMHLFCAA|2055-09-06|1868|8123|624|2055|1|9|6|3|2055|624|8123|Monday|2055Q3|N|N|N|2471877|2472119|2471517|2471791|N|N|N|N|N| +2471883|AAAAAAAALMHLFCAA|2055-09-07|1868|8124|624|2055|2|9|7|3|2055|624|8124|Tuesday|2055Q3|N|N|N|2471877|2472119|2471518|2471792|N|N|N|N|N| +2471884|AAAAAAAAMMHLFCAA|2055-09-08|1868|8124|624|2055|3|9|8|3|2055|624|8124|Wednesday|2055Q3|N|N|N|2471877|2472119|2471519|2471793|N|N|N|N|N| +2471885|AAAAAAAANMHLFCAA|2055-09-09|1868|8124|624|2055|4|9|9|3|2055|624|8124|Thursday|2055Q3|N|N|N|2471877|2472119|2471520|2471794|N|N|N|N|N| +2471886|AAAAAAAAOMHLFCAA|2055-09-10|1868|8124|624|2055|5|9|10|3|2055|624|8124|Friday|2055Q3|N|Y|N|2471877|2472119|2471521|2471795|N|N|N|N|N| +2471887|AAAAAAAAPMHLFCAA|2055-09-11|1868|8124|624|2055|6|9|11|3|2055|624|8124|Saturday|2055Q3|N|Y|N|2471877|2472119|2471522|2471796|N|N|N|N|N| +2471888|AAAAAAAAANHLFCAA|2055-09-12|1868|8124|624|2055|0|9|12|3|2055|624|8124|Sunday|2055Q3|N|N|N|2471877|2472119|2471523|2471797|N|N|N|N|N| +2471889|AAAAAAAABNHLFCAA|2055-09-13|1868|8124|624|2055|1|9|13|3|2055|624|8124|Monday|2055Q3|N|N|N|2471877|2472119|2471524|2471798|N|N|N|N|N| +2471890|AAAAAAAACNHLFCAA|2055-09-14|1868|8125|624|2055|2|9|14|3|2055|624|8125|Tuesday|2055Q3|N|N|N|2471877|2472119|2471525|2471799|N|N|N|N|N| +2471891|AAAAAAAADNHLFCAA|2055-09-15|1868|8125|624|2055|3|9|15|3|2055|624|8125|Wednesday|2055Q3|N|N|N|2471877|2472119|2471526|2471800|N|N|N|N|N| +2471892|AAAAAAAAENHLFCAA|2055-09-16|1868|8125|624|2055|4|9|16|3|2055|624|8125|Thursday|2055Q3|N|N|N|2471877|2472119|2471527|2471801|N|N|N|N|N| +2471893|AAAAAAAAFNHLFCAA|2055-09-17|1868|8125|624|2055|5|9|17|3|2055|624|8125|Friday|2055Q3|N|Y|N|2471877|2472119|2471528|2471802|N|N|N|N|N| +2471894|AAAAAAAAGNHLFCAA|2055-09-18|1868|8125|624|2055|6|9|18|3|2055|624|8125|Saturday|2055Q3|N|Y|N|2471877|2472119|2471529|2471803|N|N|N|N|N| +2471895|AAAAAAAAHNHLFCAA|2055-09-19|1868|8125|624|2055|0|9|19|3|2055|624|8125|Sunday|2055Q3|N|N|N|2471877|2472119|2471530|2471804|N|N|N|N|N| +2471896|AAAAAAAAINHLFCAA|2055-09-20|1868|8125|624|2055|1|9|20|3|2055|624|8125|Monday|2055Q3|N|N|N|2471877|2472119|2471531|2471805|N|N|N|N|N| +2471897|AAAAAAAAJNHLFCAA|2055-09-21|1868|8126|624|2055|2|9|21|3|2055|624|8126|Tuesday|2055Q3|N|N|N|2471877|2472119|2471532|2471806|N|N|N|N|N| +2471898|AAAAAAAAKNHLFCAA|2055-09-22|1868|8126|624|2055|3|9|22|3|2055|624|8126|Wednesday|2055Q3|N|N|N|2471877|2472119|2471533|2471807|N|N|N|N|N| +2471899|AAAAAAAALNHLFCAA|2055-09-23|1868|8126|624|2055|4|9|23|3|2055|624|8126|Thursday|2055Q3|N|N|N|2471877|2472119|2471534|2471808|N|N|N|N|N| +2471900|AAAAAAAAMNHLFCAA|2055-09-24|1868|8126|624|2055|5|9|24|3|2055|624|8126|Friday|2055Q3|N|Y|N|2471877|2472119|2471535|2471809|N|N|N|N|N| +2471901|AAAAAAAANNHLFCAA|2055-09-25|1868|8126|624|2055|6|9|25|3|2055|624|8126|Saturday|2055Q3|N|Y|N|2471877|2472119|2471536|2471810|N|N|N|N|N| +2471902|AAAAAAAAONHLFCAA|2055-09-26|1868|8126|624|2055|0|9|26|3|2055|624|8126|Sunday|2055Q3|N|N|N|2471877|2472119|2471537|2471811|N|N|N|N|N| +2471903|AAAAAAAAPNHLFCAA|2055-09-27|1868|8126|624|2055|1|9|27|3|2055|624|8126|Monday|2055Q3|N|N|N|2471877|2472119|2471538|2471812|N|N|N|N|N| +2471904|AAAAAAAAAOHLFCAA|2055-09-28|1868|8127|624|2055|2|9|28|3|2055|624|8127|Tuesday|2055Q3|N|N|N|2471877|2472119|2471539|2471813|N|N|N|N|N| +2471905|AAAAAAAABOHLFCAA|2055-09-29|1868|8127|624|2055|3|9|29|3|2055|624|8127|Wednesday|2055Q3|N|N|N|2471877|2472119|2471540|2471814|N|N|N|N|N| +2471906|AAAAAAAACOHLFCAA|2055-09-30|1868|8127|624|2055|4|9|30|3|2055|624|8127|Thursday|2055Q3|N|N|N|2471877|2472119|2471541|2471815|N|N|N|N|N| +2471907|AAAAAAAADOHLFCAA|2055-10-01|1869|8127|624|2055|5|10|1|3|2055|624|8127|Friday|2055Q3|N|Y|N|2471907|2472179|2471542|2471815|N|N|N|N|N| +2471908|AAAAAAAAEOHLFCAA|2055-10-02|1869|8127|624|2055|6|10|2|4|2055|624|8127|Saturday|2055Q4|N|Y|N|2471907|2472179|2471543|2471816|N|N|N|N|N| +2471909|AAAAAAAAFOHLFCAA|2055-10-03|1869|8127|624|2055|0|10|3|4|2055|624|8127|Sunday|2055Q4|N|N|N|2471907|2472179|2471544|2471817|N|N|N|N|N| +2471910|AAAAAAAAGOHLFCAA|2055-10-04|1869|8127|624|2055|1|10|4|4|2055|624|8127|Monday|2055Q4|N|N|N|2471907|2472179|2471545|2471818|N|N|N|N|N| +2471911|AAAAAAAAHOHLFCAA|2055-10-05|1869|8128|624|2055|2|10|5|4|2055|624|8128|Tuesday|2055Q4|N|N|N|2471907|2472179|2471546|2471819|N|N|N|N|N| +2471912|AAAAAAAAIOHLFCAA|2055-10-06|1869|8128|624|2055|3|10|6|4|2055|624|8128|Wednesday|2055Q4|N|N|N|2471907|2472179|2471547|2471820|N|N|N|N|N| +2471913|AAAAAAAAJOHLFCAA|2055-10-07|1869|8128|624|2055|4|10|7|4|2055|624|8128|Thursday|2055Q4|N|N|N|2471907|2472179|2471548|2471821|N|N|N|N|N| +2471914|AAAAAAAAKOHLFCAA|2055-10-08|1869|8128|624|2055|5|10|8|4|2055|624|8128|Friday|2055Q4|N|Y|N|2471907|2472179|2471549|2471822|N|N|N|N|N| +2471915|AAAAAAAALOHLFCAA|2055-10-09|1869|8128|624|2055|6|10|9|4|2055|624|8128|Saturday|2055Q4|N|Y|N|2471907|2472179|2471550|2471823|N|N|N|N|N| +2471916|AAAAAAAAMOHLFCAA|2055-10-10|1869|8128|624|2055|0|10|10|4|2055|624|8128|Sunday|2055Q4|N|N|N|2471907|2472179|2471551|2471824|N|N|N|N|N| +2471917|AAAAAAAANOHLFCAA|2055-10-11|1869|8128|624|2055|1|10|11|4|2055|624|8128|Monday|2055Q4|N|N|N|2471907|2472179|2471552|2471825|N|N|N|N|N| +2471918|AAAAAAAAOOHLFCAA|2055-10-12|1869|8129|624|2055|2|10|12|4|2055|624|8129|Tuesday|2055Q4|N|N|N|2471907|2472179|2471553|2471826|N|N|N|N|N| +2471919|AAAAAAAAPOHLFCAA|2055-10-13|1869|8129|624|2055|3|10|13|4|2055|624|8129|Wednesday|2055Q4|N|N|N|2471907|2472179|2471554|2471827|N|N|N|N|N| +2471920|AAAAAAAAAPHLFCAA|2055-10-14|1869|8129|624|2055|4|10|14|4|2055|624|8129|Thursday|2055Q4|N|N|N|2471907|2472179|2471555|2471828|N|N|N|N|N| +2471921|AAAAAAAABPHLFCAA|2055-10-15|1869|8129|624|2055|5|10|15|4|2055|624|8129|Friday|2055Q4|N|Y|N|2471907|2472179|2471556|2471829|N|N|N|N|N| +2471922|AAAAAAAACPHLFCAA|2055-10-16|1869|8129|624|2055|6|10|16|4|2055|624|8129|Saturday|2055Q4|N|Y|N|2471907|2472179|2471557|2471830|N|N|N|N|N| +2471923|AAAAAAAADPHLFCAA|2055-10-17|1869|8129|624|2055|0|10|17|4|2055|624|8129|Sunday|2055Q4|N|N|N|2471907|2472179|2471558|2471831|N|N|N|N|N| +2471924|AAAAAAAAEPHLFCAA|2055-10-18|1869|8129|624|2055|1|10|18|4|2055|624|8129|Monday|2055Q4|N|N|N|2471907|2472179|2471559|2471832|N|N|N|N|N| +2471925|AAAAAAAAFPHLFCAA|2055-10-19|1869|8130|624|2055|2|10|19|4|2055|624|8130|Tuesday|2055Q4|N|N|N|2471907|2472179|2471560|2471833|N|N|N|N|N| +2471926|AAAAAAAAGPHLFCAA|2055-10-20|1869|8130|624|2055|3|10|20|4|2055|624|8130|Wednesday|2055Q4|N|N|N|2471907|2472179|2471561|2471834|N|N|N|N|N| +2471927|AAAAAAAAHPHLFCAA|2055-10-21|1869|8130|624|2055|4|10|21|4|2055|624|8130|Thursday|2055Q4|N|N|N|2471907|2472179|2471562|2471835|N|N|N|N|N| +2471928|AAAAAAAAIPHLFCAA|2055-10-22|1869|8130|624|2055|5|10|22|4|2055|624|8130|Friday|2055Q4|N|Y|N|2471907|2472179|2471563|2471836|N|N|N|N|N| +2471929|AAAAAAAAJPHLFCAA|2055-10-23|1869|8130|624|2055|6|10|23|4|2055|624|8130|Saturday|2055Q4|N|Y|N|2471907|2472179|2471564|2471837|N|N|N|N|N| +2471930|AAAAAAAAKPHLFCAA|2055-10-24|1869|8130|624|2055|0|10|24|4|2055|624|8130|Sunday|2055Q4|N|N|N|2471907|2472179|2471565|2471838|N|N|N|N|N| +2471931|AAAAAAAALPHLFCAA|2055-10-25|1869|8130|624|2055|1|10|25|4|2055|624|8130|Monday|2055Q4|N|N|N|2471907|2472179|2471566|2471839|N|N|N|N|N| +2471932|AAAAAAAAMPHLFCAA|2055-10-26|1869|8131|624|2055|2|10|26|4|2055|624|8131|Tuesday|2055Q4|N|N|N|2471907|2472179|2471567|2471840|N|N|N|N|N| +2471933|AAAAAAAANPHLFCAA|2055-10-27|1869|8131|624|2055|3|10|27|4|2055|624|8131|Wednesday|2055Q4|N|N|N|2471907|2472179|2471568|2471841|N|N|N|N|N| +2471934|AAAAAAAAOPHLFCAA|2055-10-28|1869|8131|624|2055|4|10|28|4|2055|624|8131|Thursday|2055Q4|N|N|N|2471907|2472179|2471569|2471842|N|N|N|N|N| +2471935|AAAAAAAAPPHLFCAA|2055-10-29|1869|8131|624|2055|5|10|29|4|2055|624|8131|Friday|2055Q4|N|Y|N|2471907|2472179|2471570|2471843|N|N|N|N|N| +2471936|AAAAAAAAAAILFCAA|2055-10-30|1869|8131|624|2055|6|10|30|4|2055|624|8131|Saturday|2055Q4|N|Y|N|2471907|2472179|2471571|2471844|N|N|N|N|N| +2471937|AAAAAAAABAILFCAA|2055-10-31|1869|8131|624|2055|0|10|31|4|2055|624|8131|Sunday|2055Q4|N|N|N|2471907|2472179|2471572|2471845|N|N|N|N|N| +2471938|AAAAAAAACAILFCAA|2055-11-01|1870|8131|624|2055|1|11|1|4|2055|624|8131|Monday|2055Q4|N|N|N|2471938|2472241|2471573|2471846|N|N|N|N|N| +2471939|AAAAAAAADAILFCAA|2055-11-02|1870|8132|624|2055|2|11|2|4|2055|624|8132|Tuesday|2055Q4|N|N|N|2471938|2472241|2471574|2471847|N|N|N|N|N| +2471940|AAAAAAAAEAILFCAA|2055-11-03|1870|8132|624|2055|3|11|3|4|2055|624|8132|Wednesday|2055Q4|N|N|N|2471938|2472241|2471575|2471848|N|N|N|N|N| +2471941|AAAAAAAAFAILFCAA|2055-11-04|1870|8132|624|2055|4|11|4|4|2055|624|8132|Thursday|2055Q4|N|N|N|2471938|2472241|2471576|2471849|N|N|N|N|N| +2471942|AAAAAAAAGAILFCAA|2055-11-05|1870|8132|624|2055|5|11|5|4|2055|624|8132|Friday|2055Q4|N|Y|N|2471938|2472241|2471577|2471850|N|N|N|N|N| +2471943|AAAAAAAAHAILFCAA|2055-11-06|1870|8132|624|2055|6|11|6|4|2055|624|8132|Saturday|2055Q4|N|Y|N|2471938|2472241|2471578|2471851|N|N|N|N|N| +2471944|AAAAAAAAIAILFCAA|2055-11-07|1870|8132|624|2055|0|11|7|4|2055|624|8132|Sunday|2055Q4|N|N|N|2471938|2472241|2471579|2471852|N|N|N|N|N| +2471945|AAAAAAAAJAILFCAA|2055-11-08|1870|8132|624|2055|1|11|8|4|2055|624|8132|Monday|2055Q4|N|N|N|2471938|2472241|2471580|2471853|N|N|N|N|N| +2471946|AAAAAAAAKAILFCAA|2055-11-09|1870|8133|624|2055|2|11|9|4|2055|624|8133|Tuesday|2055Q4|N|N|N|2471938|2472241|2471581|2471854|N|N|N|N|N| +2471947|AAAAAAAALAILFCAA|2055-11-10|1870|8133|624|2055|3|11|10|4|2055|624|8133|Wednesday|2055Q4|N|N|N|2471938|2472241|2471582|2471855|N|N|N|N|N| +2471948|AAAAAAAAMAILFCAA|2055-11-11|1870|8133|624|2055|4|11|11|4|2055|624|8133|Thursday|2055Q4|N|N|N|2471938|2472241|2471583|2471856|N|N|N|N|N| +2471949|AAAAAAAANAILFCAA|2055-11-12|1870|8133|624|2055|5|11|12|4|2055|624|8133|Friday|2055Q4|N|Y|N|2471938|2472241|2471584|2471857|N|N|N|N|N| +2471950|AAAAAAAAOAILFCAA|2055-11-13|1870|8133|624|2055|6|11|13|4|2055|624|8133|Saturday|2055Q4|N|Y|N|2471938|2472241|2471585|2471858|N|N|N|N|N| +2471951|AAAAAAAAPAILFCAA|2055-11-14|1870|8133|624|2055|0|11|14|4|2055|624|8133|Sunday|2055Q4|N|N|N|2471938|2472241|2471586|2471859|N|N|N|N|N| +2471952|AAAAAAAAABILFCAA|2055-11-15|1870|8133|624|2055|1|11|15|4|2055|624|8133|Monday|2055Q4|N|N|N|2471938|2472241|2471587|2471860|N|N|N|N|N| +2471953|AAAAAAAABBILFCAA|2055-11-16|1870|8134|624|2055|2|11|16|4|2055|624|8134|Tuesday|2055Q4|N|N|N|2471938|2472241|2471588|2471861|N|N|N|N|N| +2471954|AAAAAAAACBILFCAA|2055-11-17|1870|8134|624|2055|3|11|17|4|2055|624|8134|Wednesday|2055Q4|N|N|N|2471938|2472241|2471589|2471862|N|N|N|N|N| +2471955|AAAAAAAADBILFCAA|2055-11-18|1870|8134|624|2055|4|11|18|4|2055|624|8134|Thursday|2055Q4|N|N|N|2471938|2472241|2471590|2471863|N|N|N|N|N| +2471956|AAAAAAAAEBILFCAA|2055-11-19|1870|8134|624|2055|5|11|19|4|2055|624|8134|Friday|2055Q4|N|Y|N|2471938|2472241|2471591|2471864|N|N|N|N|N| +2471957|AAAAAAAAFBILFCAA|2055-11-20|1870|8134|624|2055|6|11|20|4|2055|624|8134|Saturday|2055Q4|N|Y|N|2471938|2472241|2471592|2471865|N|N|N|N|N| +2471958|AAAAAAAAGBILFCAA|2055-11-21|1870|8134|624|2055|0|11|21|4|2055|624|8134|Sunday|2055Q4|N|N|N|2471938|2472241|2471593|2471866|N|N|N|N|N| +2471959|AAAAAAAAHBILFCAA|2055-11-22|1870|8134|624|2055|1|11|22|4|2055|624|8134|Monday|2055Q4|N|N|N|2471938|2472241|2471594|2471867|N|N|N|N|N| +2471960|AAAAAAAAIBILFCAA|2055-11-23|1870|8135|624|2055|2|11|23|4|2055|624|8135|Tuesday|2055Q4|N|N|N|2471938|2472241|2471595|2471868|N|N|N|N|N| +2471961|AAAAAAAAJBILFCAA|2055-11-24|1870|8135|624|2055|3|11|24|4|2055|624|8135|Wednesday|2055Q4|N|N|N|2471938|2472241|2471596|2471869|N|N|N|N|N| +2471962|AAAAAAAAKBILFCAA|2055-11-25|1870|8135|624|2055|4|11|25|4|2055|624|8135|Thursday|2055Q4|N|N|N|2471938|2472241|2471597|2471870|N|N|N|N|N| +2471963|AAAAAAAALBILFCAA|2055-11-26|1870|8135|624|2055|5|11|26|4|2055|624|8135|Friday|2055Q4|N|Y|N|2471938|2472241|2471598|2471871|N|N|N|N|N| +2471964|AAAAAAAAMBILFCAA|2055-11-27|1870|8135|624|2055|6|11|27|4|2055|624|8135|Saturday|2055Q4|N|Y|N|2471938|2472241|2471599|2471872|N|N|N|N|N| +2471965|AAAAAAAANBILFCAA|2055-11-28|1870|8135|624|2055|0|11|28|4|2055|624|8135|Sunday|2055Q4|N|N|N|2471938|2472241|2471600|2471873|N|N|N|N|N| +2471966|AAAAAAAAOBILFCAA|2055-11-29|1870|8135|624|2055|1|11|29|4|2055|624|8135|Monday|2055Q4|N|N|N|2471938|2472241|2471601|2471874|N|N|N|N|N| +2471967|AAAAAAAAPBILFCAA|2055-11-30|1870|8136|624|2055|2|11|30|4|2055|624|8136|Tuesday|2055Q4|N|N|N|2471938|2472241|2471602|2471875|N|N|N|N|N| +2471968|AAAAAAAAACILFCAA|2055-12-01|1871|8136|625|2055|3|12|1|4|2055|625|8136|Wednesday|2055Q4|N|N|N|2471968|2472301|2471603|2471876|N|N|N|N|N| +2471969|AAAAAAAABCILFCAA|2055-12-02|1871|8136|625|2055|4|12|2|4|2055|625|8136|Thursday|2055Q4|N|N|N|2471968|2472301|2471604|2471877|N|N|N|N|N| +2471970|AAAAAAAACCILFCAA|2055-12-03|1871|8136|625|2055|5|12|3|4|2055|625|8136|Friday|2055Q4|N|Y|N|2471968|2472301|2471605|2471878|N|N|N|N|N| +2471971|AAAAAAAADCILFCAA|2055-12-04|1871|8136|625|2055|6|12|4|4|2055|625|8136|Saturday|2055Q4|N|Y|N|2471968|2472301|2471606|2471879|N|N|N|N|N| +2471972|AAAAAAAAECILFCAA|2055-12-05|1871|8136|625|2055|0|12|5|4|2055|625|8136|Sunday|2055Q4|N|N|N|2471968|2472301|2471607|2471880|N|N|N|N|N| +2471973|AAAAAAAAFCILFCAA|2055-12-06|1871|8136|625|2055|1|12|6|4|2055|625|8136|Monday|2055Q4|N|N|N|2471968|2472301|2471608|2471881|N|N|N|N|N| +2471974|AAAAAAAAGCILFCAA|2055-12-07|1871|8137|625|2055|2|12|7|4|2055|625|8137|Tuesday|2055Q4|N|N|N|2471968|2472301|2471609|2471882|N|N|N|N|N| +2471975|AAAAAAAAHCILFCAA|2055-12-08|1871|8137|625|2055|3|12|8|4|2055|625|8137|Wednesday|2055Q4|N|N|N|2471968|2472301|2471610|2471883|N|N|N|N|N| +2471976|AAAAAAAAICILFCAA|2055-12-09|1871|8137|625|2055|4|12|9|4|2055|625|8137|Thursday|2055Q4|N|N|N|2471968|2472301|2471611|2471884|N|N|N|N|N| +2471977|AAAAAAAAJCILFCAA|2055-12-10|1871|8137|625|2055|5|12|10|4|2055|625|8137|Friday|2055Q4|N|Y|N|2471968|2472301|2471612|2471885|N|N|N|N|N| +2471978|AAAAAAAAKCILFCAA|2055-12-11|1871|8137|625|2055|6|12|11|4|2055|625|8137|Saturday|2055Q4|N|Y|N|2471968|2472301|2471613|2471886|N|N|N|N|N| +2471979|AAAAAAAALCILFCAA|2055-12-12|1871|8137|625|2055|0|12|12|4|2055|625|8137|Sunday|2055Q4|N|N|N|2471968|2472301|2471614|2471887|N|N|N|N|N| +2471980|AAAAAAAAMCILFCAA|2055-12-13|1871|8137|625|2055|1|12|13|4|2055|625|8137|Monday|2055Q4|N|N|N|2471968|2472301|2471615|2471888|N|N|N|N|N| +2471981|AAAAAAAANCILFCAA|2055-12-14|1871|8138|625|2055|2|12|14|4|2055|625|8138|Tuesday|2055Q4|N|N|N|2471968|2472301|2471616|2471889|N|N|N|N|N| +2471982|AAAAAAAAOCILFCAA|2055-12-15|1871|8138|625|2055|3|12|15|4|2055|625|8138|Wednesday|2055Q4|N|N|N|2471968|2472301|2471617|2471890|N|N|N|N|N| +2471983|AAAAAAAAPCILFCAA|2055-12-16|1871|8138|625|2055|4|12|16|4|2055|625|8138|Thursday|2055Q4|N|N|N|2471968|2472301|2471618|2471891|N|N|N|N|N| +2471984|AAAAAAAAADILFCAA|2055-12-17|1871|8138|625|2055|5|12|17|4|2055|625|8138|Friday|2055Q4|N|Y|N|2471968|2472301|2471619|2471892|N|N|N|N|N| +2471985|AAAAAAAABDILFCAA|2055-12-18|1871|8138|625|2055|6|12|18|4|2055|625|8138|Saturday|2055Q4|N|Y|N|2471968|2472301|2471620|2471893|N|N|N|N|N| +2471986|AAAAAAAACDILFCAA|2055-12-19|1871|8138|625|2055|0|12|19|4|2055|625|8138|Sunday|2055Q4|N|N|N|2471968|2472301|2471621|2471894|N|N|N|N|N| +2471987|AAAAAAAADDILFCAA|2055-12-20|1871|8138|625|2055|1|12|20|4|2055|625|8138|Monday|2055Q4|N|N|N|2471968|2472301|2471622|2471895|N|N|N|N|N| +2471988|AAAAAAAAEDILFCAA|2055-12-21|1871|8139|625|2055|2|12|21|4|2055|625|8139|Tuesday|2055Q4|N|N|N|2471968|2472301|2471623|2471896|N|N|N|N|N| +2471989|AAAAAAAAFDILFCAA|2055-12-22|1871|8139|625|2055|3|12|22|4|2055|625|8139|Wednesday|2055Q4|N|N|N|2471968|2472301|2471624|2471897|N|N|N|N|N| +2471990|AAAAAAAAGDILFCAA|2055-12-23|1871|8139|625|2055|4|12|23|4|2055|625|8139|Thursday|2055Q4|N|N|N|2471968|2472301|2471625|2471898|N|N|N|N|N| +2471991|AAAAAAAAHDILFCAA|2055-12-24|1871|8139|625|2055|5|12|24|4|2055|625|8139|Friday|2055Q4|N|Y|N|2471968|2472301|2471626|2471899|N|N|N|N|N| +2471992|AAAAAAAAIDILFCAA|2055-12-25|1871|8139|625|2055|6|12|25|4|2055|625|8139|Saturday|2055Q4|N|Y|N|2471968|2472301|2471627|2471900|N|N|N|N|N| +2471993|AAAAAAAAJDILFCAA|2055-12-26|1871|8139|625|2055|0|12|26|4|2055|625|8139|Sunday|2055Q4|Y|N|N|2471968|2472301|2471628|2471901|N|N|N|N|N| +2471994|AAAAAAAAKDILFCAA|2055-12-27|1871|8139|625|2055|1|12|27|4|2055|625|8139|Monday|2055Q4|N|N|Y|2471968|2472301|2471629|2471902|N|N|N|N|N| +2471995|AAAAAAAALDILFCAA|2055-12-28|1871|8140|625|2055|2|12|28|4|2055|625|8140|Tuesday|2055Q4|N|N|N|2471968|2472301|2471630|2471903|N|N|N|N|N| +2471996|AAAAAAAAMDILFCAA|2055-12-29|1871|8140|625|2055|3|12|29|4|2055|625|8140|Wednesday|2055Q4|N|N|N|2471968|2472301|2471631|2471904|N|N|N|N|N| +2471997|AAAAAAAANDILFCAA|2055-12-30|1871|8140|625|2055|4|12|30|4|2055|625|8140|Thursday|2055Q4|N|N|N|2471968|2472301|2471632|2471905|N|N|N|N|N| +2471998|AAAAAAAAODILFCAA|2055-12-31|1871|8140|625|2055|5|12|31|4|2055|625|8140|Friday|2055Q4|N|Y|N|2471968|2472301|2471633|2471906|N|N|N|N|N| +2471999|AAAAAAAAPDILFCAA|2056-01-01|1872|8140|625|2056|6|1|1|1|2056|625|8140|Saturday|2056Q1|Y|Y|N|2471999|2471998|2471634|2471907|N|N|N|N|N| +2472000|AAAAAAAAAEILFCAA|2056-01-02|1872|8140|625|2056|0|1|2|1|2056|625|8140|Sunday|2056Q1|N|N|Y|2471999|2471998|2471635|2471908|N|N|N|N|N| +2472001|AAAAAAAABEILFCAA|2056-01-03|1872|8140|625|2056|1|1|3|1|2056|625|8140|Monday|2056Q1|N|N|N|2471999|2471998|2471636|2471909|N|N|N|N|N| +2472002|AAAAAAAACEILFCAA|2056-01-04|1872|8141|625|2056|2|1|4|1|2056|625|8141|Tuesday|2056Q1|N|N|N|2471999|2471998|2471637|2471910|N|N|N|N|N| +2472003|AAAAAAAADEILFCAA|2056-01-05|1872|8141|625|2056|3|1|5|1|2056|625|8141|Wednesday|2056Q1|N|N|N|2471999|2471998|2471638|2471911|N|N|N|N|N| +2472004|AAAAAAAAEEILFCAA|2056-01-06|1872|8141|625|2056|4|1|6|1|2056|625|8141|Thursday|2056Q1|N|N|N|2471999|2471998|2471639|2471912|N|N|N|N|N| +2472005|AAAAAAAAFEILFCAA|2056-01-07|1872|8141|625|2056|5|1|7|1|2056|625|8141|Friday|2056Q1|N|Y|N|2471999|2471998|2471640|2471913|N|N|N|N|N| +2472006|AAAAAAAAGEILFCAA|2056-01-08|1872|8141|625|2056|6|1|8|1|2056|625|8141|Saturday|2056Q1|N|Y|N|2471999|2471998|2471641|2471914|N|N|N|N|N| +2472007|AAAAAAAAHEILFCAA|2056-01-09|1872|8141|625|2056|0|1|9|1|2056|625|8141|Sunday|2056Q1|N|N|N|2471999|2471998|2471642|2471915|N|N|N|N|N| +2472008|AAAAAAAAIEILFCAA|2056-01-10|1872|8141|625|2056|1|1|10|1|2056|625|8141|Monday|2056Q1|N|N|N|2471999|2471998|2471643|2471916|N|N|N|N|N| +2472009|AAAAAAAAJEILFCAA|2056-01-11|1872|8142|625|2056|2|1|11|1|2056|625|8142|Tuesday|2056Q1|N|N|N|2471999|2471998|2471644|2471917|N|N|N|N|N| +2472010|AAAAAAAAKEILFCAA|2056-01-12|1872|8142|625|2056|3|1|12|1|2056|625|8142|Wednesday|2056Q1|N|N|N|2471999|2471998|2471645|2471918|N|N|N|N|N| +2472011|AAAAAAAALEILFCAA|2056-01-13|1872|8142|625|2056|4|1|13|1|2056|625|8142|Thursday|2056Q1|N|N|N|2471999|2471998|2471646|2471919|N|N|N|N|N| +2472012|AAAAAAAAMEILFCAA|2056-01-14|1872|8142|625|2056|5|1|14|1|2056|625|8142|Friday|2056Q1|N|Y|N|2471999|2471998|2471647|2471920|N|N|N|N|N| +2472013|AAAAAAAANEILFCAA|2056-01-15|1872|8142|625|2056|6|1|15|1|2056|625|8142|Saturday|2056Q1|N|Y|N|2471999|2471998|2471648|2471921|N|N|N|N|N| +2472014|AAAAAAAAOEILFCAA|2056-01-16|1872|8142|625|2056|0|1|16|1|2056|625|8142|Sunday|2056Q1|N|N|N|2471999|2471998|2471649|2471922|N|N|N|N|N| +2472015|AAAAAAAAPEILFCAA|2056-01-17|1872|8142|625|2056|1|1|17|1|2056|625|8142|Monday|2056Q1|N|N|N|2471999|2471998|2471650|2471923|N|N|N|N|N| +2472016|AAAAAAAAAFILFCAA|2056-01-18|1872|8143|625|2056|2|1|18|1|2056|625|8143|Tuesday|2056Q1|N|N|N|2471999|2471998|2471651|2471924|N|N|N|N|N| +2472017|AAAAAAAABFILFCAA|2056-01-19|1872|8143|625|2056|3|1|19|1|2056|625|8143|Wednesday|2056Q1|N|N|N|2471999|2471998|2471652|2471925|N|N|N|N|N| +2472018|AAAAAAAACFILFCAA|2056-01-20|1872|8143|625|2056|4|1|20|1|2056|625|8143|Thursday|2056Q1|N|N|N|2471999|2471998|2471653|2471926|N|N|N|N|N| +2472019|AAAAAAAADFILFCAA|2056-01-21|1872|8143|625|2056|5|1|21|1|2056|625|8143|Friday|2056Q1|N|Y|N|2471999|2471998|2471654|2471927|N|N|N|N|N| +2472020|AAAAAAAAEFILFCAA|2056-01-22|1872|8143|625|2056|6|1|22|1|2056|625|8143|Saturday|2056Q1|N|Y|N|2471999|2471998|2471655|2471928|N|N|N|N|N| +2472021|AAAAAAAAFFILFCAA|2056-01-23|1872|8143|625|2056|0|1|23|1|2056|625|8143|Sunday|2056Q1|N|N|N|2471999|2471998|2471656|2471929|N|N|N|N|N| +2472022|AAAAAAAAGFILFCAA|2056-01-24|1872|8143|625|2056|1|1|24|1|2056|625|8143|Monday|2056Q1|N|N|N|2471999|2471998|2471657|2471930|N|N|N|N|N| +2472023|AAAAAAAAHFILFCAA|2056-01-25|1872|8144|625|2056|2|1|25|1|2056|625|8144|Tuesday|2056Q1|N|N|N|2471999|2471998|2471658|2471931|N|N|N|N|N| +2472024|AAAAAAAAIFILFCAA|2056-01-26|1872|8144|625|2056|3|1|26|1|2056|625|8144|Wednesday|2056Q1|N|N|N|2471999|2471998|2471659|2471932|N|N|N|N|N| +2472025|AAAAAAAAJFILFCAA|2056-01-27|1872|8144|625|2056|4|1|27|1|2056|625|8144|Thursday|2056Q1|N|N|N|2471999|2471998|2471660|2471933|N|N|N|N|N| +2472026|AAAAAAAAKFILFCAA|2056-01-28|1872|8144|625|2056|5|1|28|1|2056|625|8144|Friday|2056Q1|N|Y|N|2471999|2471998|2471661|2471934|N|N|N|N|N| +2472027|AAAAAAAALFILFCAA|2056-01-29|1872|8144|625|2056|6|1|29|1|2056|625|8144|Saturday|2056Q1|N|Y|N|2471999|2471998|2471662|2471935|N|N|N|N|N| +2472028|AAAAAAAAMFILFCAA|2056-01-30|1872|8144|625|2056|0|1|30|1|2056|625|8144|Sunday|2056Q1|N|N|N|2471999|2471998|2471663|2471936|N|N|N|N|N| +2472029|AAAAAAAANFILFCAA|2056-01-31|1872|8144|625|2056|1|1|31|1|2056|625|8144|Monday|2056Q1|N|N|N|2471999|2471998|2471664|2471937|N|N|N|N|N| +2472030|AAAAAAAAOFILFCAA|2056-02-01|1873|8145|625|2056|2|2|1|1|2056|625|8145|Tuesday|2056Q1|N|N|N|2472030|2472060|2471665|2471938|N|N|N|N|N| +2472031|AAAAAAAAPFILFCAA|2056-02-02|1873|8145|625|2056|3|2|2|1|2056|625|8145|Wednesday|2056Q1|N|N|N|2472030|2472060|2471666|2471939|N|N|N|N|N| +2472032|AAAAAAAAAGILFCAA|2056-02-03|1873|8145|625|2056|4|2|3|1|2056|625|8145|Thursday|2056Q1|N|N|N|2472030|2472060|2471667|2471940|N|N|N|N|N| +2472033|AAAAAAAABGILFCAA|2056-02-04|1873|8145|625|2056|5|2|4|1|2056|625|8145|Friday|2056Q1|N|Y|N|2472030|2472060|2471668|2471941|N|N|N|N|N| +2472034|AAAAAAAACGILFCAA|2056-02-05|1873|8145|625|2056|6|2|5|1|2056|625|8145|Saturday|2056Q1|N|Y|N|2472030|2472060|2471669|2471942|N|N|N|N|N| +2472035|AAAAAAAADGILFCAA|2056-02-06|1873|8145|625|2056|0|2|6|1|2056|625|8145|Sunday|2056Q1|N|N|N|2472030|2472060|2471670|2471943|N|N|N|N|N| +2472036|AAAAAAAAEGILFCAA|2056-02-07|1873|8145|625|2056|1|2|7|1|2056|625|8145|Monday|2056Q1|N|N|N|2472030|2472060|2471671|2471944|N|N|N|N|N| +2472037|AAAAAAAAFGILFCAA|2056-02-08|1873|8146|625|2056|2|2|8|1|2056|625|8146|Tuesday|2056Q1|N|N|N|2472030|2472060|2471672|2471945|N|N|N|N|N| +2472038|AAAAAAAAGGILFCAA|2056-02-09|1873|8146|625|2056|3|2|9|1|2056|625|8146|Wednesday|2056Q1|N|N|N|2472030|2472060|2471673|2471946|N|N|N|N|N| +2472039|AAAAAAAAHGILFCAA|2056-02-10|1873|8146|625|2056|4|2|10|1|2056|625|8146|Thursday|2056Q1|N|N|N|2472030|2472060|2471674|2471947|N|N|N|N|N| +2472040|AAAAAAAAIGILFCAA|2056-02-11|1873|8146|625|2056|5|2|11|1|2056|625|8146|Friday|2056Q1|N|Y|N|2472030|2472060|2471675|2471948|N|N|N|N|N| +2472041|AAAAAAAAJGILFCAA|2056-02-12|1873|8146|625|2056|6|2|12|1|2056|625|8146|Saturday|2056Q1|N|Y|N|2472030|2472060|2471676|2471949|N|N|N|N|N| +2472042|AAAAAAAAKGILFCAA|2056-02-13|1873|8146|625|2056|0|2|13|1|2056|625|8146|Sunday|2056Q1|N|N|N|2472030|2472060|2471677|2471950|N|N|N|N|N| +2472043|AAAAAAAALGILFCAA|2056-02-14|1873|8146|625|2056|1|2|14|1|2056|625|8146|Monday|2056Q1|N|N|N|2472030|2472060|2471678|2471951|N|N|N|N|N| +2472044|AAAAAAAAMGILFCAA|2056-02-15|1873|8147|625|2056|2|2|15|1|2056|625|8147|Tuesday|2056Q1|N|N|N|2472030|2472060|2471679|2471952|N|N|N|N|N| +2472045|AAAAAAAANGILFCAA|2056-02-16|1873|8147|625|2056|3|2|16|1|2056|625|8147|Wednesday|2056Q1|N|N|N|2472030|2472060|2471680|2471953|N|N|N|N|N| +2472046|AAAAAAAAOGILFCAA|2056-02-17|1873|8147|625|2056|4|2|17|1|2056|625|8147|Thursday|2056Q1|N|N|N|2472030|2472060|2471681|2471954|N|N|N|N|N| +2472047|AAAAAAAAPGILFCAA|2056-02-18|1873|8147|625|2056|5|2|18|1|2056|625|8147|Friday|2056Q1|N|Y|N|2472030|2472060|2471682|2471955|N|N|N|N|N| +2472048|AAAAAAAAAHILFCAA|2056-02-19|1873|8147|625|2056|6|2|19|1|2056|625|8147|Saturday|2056Q1|N|Y|N|2472030|2472060|2471683|2471956|N|N|N|N|N| +2472049|AAAAAAAABHILFCAA|2056-02-20|1873|8147|625|2056|0|2|20|1|2056|625|8147|Sunday|2056Q1|N|N|N|2472030|2472060|2471684|2471957|N|N|N|N|N| +2472050|AAAAAAAACHILFCAA|2056-02-21|1873|8147|625|2056|1|2|21|1|2056|625|8147|Monday|2056Q1|N|N|N|2472030|2472060|2471685|2471958|N|N|N|N|N| +2472051|AAAAAAAADHILFCAA|2056-02-22|1873|8148|625|2056|2|2|22|1|2056|625|8148|Tuesday|2056Q1|N|N|N|2472030|2472060|2471686|2471959|N|N|N|N|N| +2472052|AAAAAAAAEHILFCAA|2056-02-23|1873|8148|625|2056|3|2|23|1|2056|625|8148|Wednesday|2056Q1|N|N|N|2472030|2472060|2471687|2471960|N|N|N|N|N| +2472053|AAAAAAAAFHILFCAA|2056-02-24|1873|8148|625|2056|4|2|24|1|2056|625|8148|Thursday|2056Q1|N|N|N|2472030|2472060|2471688|2471961|N|N|N|N|N| +2472054|AAAAAAAAGHILFCAA|2056-02-25|1873|8148|625|2056|5|2|25|1|2056|625|8148|Friday|2056Q1|N|Y|N|2472030|2472060|2471689|2471962|N|N|N|N|N| +2472055|AAAAAAAAHHILFCAA|2056-02-26|1873|8148|625|2056|6|2|26|1|2056|625|8148|Saturday|2056Q1|N|Y|N|2472030|2472060|2471690|2471963|N|N|N|N|N| +2472056|AAAAAAAAIHILFCAA|2056-02-27|1873|8148|625|2056|0|2|27|1|2056|625|8148|Sunday|2056Q1|N|N|N|2472030|2472060|2471691|2471964|N|N|N|N|N| +2472057|AAAAAAAAJHILFCAA|2056-02-28|1873|8148|625|2056|1|2|28|1|2056|625|8148|Monday|2056Q1|N|N|N|2472030|2472060|2471692|2471965|N|N|N|N|N| +2472058|AAAAAAAAKHILFCAA|2056-02-29|1873|8149|625|2056|2|2|29|1|2056|625|8149|Tuesday|2056Q1|N|N|N|2472030|2472060|2471692|2471966|N|N|N|N|N| +2472059|AAAAAAAALHILFCAA|2056-03-01|1874|8149|626|2056|3|3|1|1|2056|626|8149|Wednesday|2056Q1|N|N|N|2472059|2472118|2471693|2471967|N|N|N|N|N| +2472060|AAAAAAAAMHILFCAA|2056-03-02|1874|8149|626|2056|4|3|2|1|2056|626|8149|Thursday|2056Q1|N|N|N|2472059|2472118|2471694|2471968|N|N|N|N|N| +2472061|AAAAAAAANHILFCAA|2056-03-03|1874|8149|626|2056|5|3|3|1|2056|626|8149|Friday|2056Q1|N|Y|N|2472059|2472118|2471695|2471969|N|N|N|N|N| +2472062|AAAAAAAAOHILFCAA|2056-03-04|1874|8149|626|2056|6|3|4|1|2056|626|8149|Saturday|2056Q1|N|Y|N|2472059|2472118|2471696|2471970|N|N|N|N|N| +2472063|AAAAAAAAPHILFCAA|2056-03-05|1874|8149|626|2056|0|3|5|1|2056|626|8149|Sunday|2056Q1|N|N|N|2472059|2472118|2471697|2471971|N|N|N|N|N| +2472064|AAAAAAAAAIILFCAA|2056-03-06|1874|8149|626|2056|1|3|6|1|2056|626|8149|Monday|2056Q1|N|N|N|2472059|2472118|2471698|2471972|N|N|N|N|N| +2472065|AAAAAAAABIILFCAA|2056-03-07|1874|8150|626|2056|2|3|7|1|2056|626|8150|Tuesday|2056Q1|N|N|N|2472059|2472118|2471699|2471973|N|N|N|N|N| +2472066|AAAAAAAACIILFCAA|2056-03-08|1874|8150|626|2056|3|3|8|1|2056|626|8150|Wednesday|2056Q1|N|N|N|2472059|2472118|2471700|2471974|N|N|N|N|N| +2472067|AAAAAAAADIILFCAA|2056-03-09|1874|8150|626|2056|4|3|9|1|2056|626|8150|Thursday|2056Q1|N|N|N|2472059|2472118|2471701|2471975|N|N|N|N|N| +2472068|AAAAAAAAEIILFCAA|2056-03-10|1874|8150|626|2056|5|3|10|1|2056|626|8150|Friday|2056Q1|N|Y|N|2472059|2472118|2471702|2471976|N|N|N|N|N| +2472069|AAAAAAAAFIILFCAA|2056-03-11|1874|8150|626|2056|6|3|11|1|2056|626|8150|Saturday|2056Q1|N|Y|N|2472059|2472118|2471703|2471977|N|N|N|N|N| +2472070|AAAAAAAAGIILFCAA|2056-03-12|1874|8150|626|2056|0|3|12|1|2056|626|8150|Sunday|2056Q1|N|N|N|2472059|2472118|2471704|2471978|N|N|N|N|N| +2472071|AAAAAAAAHIILFCAA|2056-03-13|1874|8150|626|2056|1|3|13|1|2056|626|8150|Monday|2056Q1|N|N|N|2472059|2472118|2471705|2471979|N|N|N|N|N| +2472072|AAAAAAAAIIILFCAA|2056-03-14|1874|8151|626|2056|2|3|14|1|2056|626|8151|Tuesday|2056Q1|N|N|N|2472059|2472118|2471706|2471980|N|N|N|N|N| +2472073|AAAAAAAAJIILFCAA|2056-03-15|1874|8151|626|2056|3|3|15|1|2056|626|8151|Wednesday|2056Q1|N|N|N|2472059|2472118|2471707|2471981|N|N|N|N|N| +2472074|AAAAAAAAKIILFCAA|2056-03-16|1874|8151|626|2056|4|3|16|1|2056|626|8151|Thursday|2056Q1|N|N|N|2472059|2472118|2471708|2471982|N|N|N|N|N| +2472075|AAAAAAAALIILFCAA|2056-03-17|1874|8151|626|2056|5|3|17|1|2056|626|8151|Friday|2056Q1|N|Y|N|2472059|2472118|2471709|2471983|N|N|N|N|N| +2472076|AAAAAAAAMIILFCAA|2056-03-18|1874|8151|626|2056|6|3|18|1|2056|626|8151|Saturday|2056Q1|N|Y|N|2472059|2472118|2471710|2471984|N|N|N|N|N| +2472077|AAAAAAAANIILFCAA|2056-03-19|1874|8151|626|2056|0|3|19|1|2056|626|8151|Sunday|2056Q1|N|N|N|2472059|2472118|2471711|2471985|N|N|N|N|N| +2472078|AAAAAAAAOIILFCAA|2056-03-20|1874|8151|626|2056|1|3|20|1|2056|626|8151|Monday|2056Q1|N|N|N|2472059|2472118|2471712|2471986|N|N|N|N|N| +2472079|AAAAAAAAPIILFCAA|2056-03-21|1874|8152|626|2056|2|3|21|1|2056|626|8152|Tuesday|2056Q1|N|N|N|2472059|2472118|2471713|2471987|N|N|N|N|N| +2472080|AAAAAAAAAJILFCAA|2056-03-22|1874|8152|626|2056|3|3|22|1|2056|626|8152|Wednesday|2056Q1|N|N|N|2472059|2472118|2471714|2471988|N|N|N|N|N| +2472081|AAAAAAAABJILFCAA|2056-03-23|1874|8152|626|2056|4|3|23|1|2056|626|8152|Thursday|2056Q1|N|N|N|2472059|2472118|2471715|2471989|N|N|N|N|N| +2472082|AAAAAAAACJILFCAA|2056-03-24|1874|8152|626|2056|5|3|24|1|2056|626|8152|Friday|2056Q1|N|Y|N|2472059|2472118|2471716|2471990|N|N|N|N|N| +2472083|AAAAAAAADJILFCAA|2056-03-25|1874|8152|626|2056|6|3|25|1|2056|626|8152|Saturday|2056Q1|N|Y|N|2472059|2472118|2471717|2471991|N|N|N|N|N| +2472084|AAAAAAAAEJILFCAA|2056-03-26|1874|8152|626|2056|0|3|26|1|2056|626|8152|Sunday|2056Q1|N|N|N|2472059|2472118|2471718|2471992|N|N|N|N|N| +2472085|AAAAAAAAFJILFCAA|2056-03-27|1874|8152|626|2056|1|3|27|1|2056|626|8152|Monday|2056Q1|N|N|N|2472059|2472118|2471719|2471993|N|N|N|N|N| +2472086|AAAAAAAAGJILFCAA|2056-03-28|1874|8153|626|2056|2|3|28|1|2056|626|8153|Tuesday|2056Q1|N|N|N|2472059|2472118|2471720|2471994|N|N|N|N|N| +2472087|AAAAAAAAHJILFCAA|2056-03-29|1874|8153|626|2056|3|3|29|1|2056|626|8153|Wednesday|2056Q1|N|N|N|2472059|2472118|2471721|2471995|N|N|N|N|N| +2472088|AAAAAAAAIJILFCAA|2056-03-30|1874|8153|626|2056|4|3|30|1|2056|626|8153|Thursday|2056Q1|N|N|N|2472059|2472118|2471722|2471996|N|N|N|N|N| +2472089|AAAAAAAAJJILFCAA|2056-03-31|1874|8153|626|2056|5|3|31|1|2056|626|8153|Friday|2056Q1|N|Y|N|2472059|2472118|2471723|2471997|N|N|N|N|N| +2472090|AAAAAAAAKJILFCAA|2056-04-01|1875|8153|626|2056|6|4|1|2|2056|626|8153|Saturday|2056Q2|N|Y|N|2472090|2472180|2471724|2471999|N|N|N|N|N| +2472091|AAAAAAAALJILFCAA|2056-04-02|1875|8153|626|2056|0|4|2|2|2056|626|8153|Sunday|2056Q2|N|N|N|2472090|2472180|2471725|2472000|N|N|N|N|N| +2472092|AAAAAAAAMJILFCAA|2056-04-03|1875|8153|626|2056|1|4|3|2|2056|626|8153|Monday|2056Q2|N|N|N|2472090|2472180|2471726|2472001|N|N|N|N|N| +2472093|AAAAAAAANJILFCAA|2056-04-04|1875|8154|626|2056|2|4|4|2|2056|626|8154|Tuesday|2056Q2|N|N|N|2472090|2472180|2471727|2472002|N|N|N|N|N| +2472094|AAAAAAAAOJILFCAA|2056-04-05|1875|8154|626|2056|3|4|5|2|2056|626|8154|Wednesday|2056Q2|N|N|N|2472090|2472180|2471728|2472003|N|N|N|N|N| +2472095|AAAAAAAAPJILFCAA|2056-04-06|1875|8154|626|2056|4|4|6|2|2056|626|8154|Thursday|2056Q2|N|N|N|2472090|2472180|2471729|2472004|N|N|N|N|N| +2472096|AAAAAAAAAKILFCAA|2056-04-07|1875|8154|626|2056|5|4|7|2|2056|626|8154|Friday|2056Q2|N|Y|N|2472090|2472180|2471730|2472005|N|N|N|N|N| +2472097|AAAAAAAABKILFCAA|2056-04-08|1875|8154|626|2056|6|4|8|2|2056|626|8154|Saturday|2056Q2|N|Y|N|2472090|2472180|2471731|2472006|N|N|N|N|N| +2472098|AAAAAAAACKILFCAA|2056-04-09|1875|8154|626|2056|0|4|9|2|2056|626|8154|Sunday|2056Q2|N|N|N|2472090|2472180|2471732|2472007|N|N|N|N|N| +2472099|AAAAAAAADKILFCAA|2056-04-10|1875|8154|626|2056|1|4|10|2|2056|626|8154|Monday|2056Q2|N|N|N|2472090|2472180|2471733|2472008|N|N|N|N|N| +2472100|AAAAAAAAEKILFCAA|2056-04-11|1875|8155|626|2056|2|4|11|2|2056|626|8155|Tuesday|2056Q2|N|N|N|2472090|2472180|2471734|2472009|N|N|N|N|N| +2472101|AAAAAAAAFKILFCAA|2056-04-12|1875|8155|626|2056|3|4|12|2|2056|626|8155|Wednesday|2056Q2|N|N|N|2472090|2472180|2471735|2472010|N|N|N|N|N| +2472102|AAAAAAAAGKILFCAA|2056-04-13|1875|8155|626|2056|4|4|13|2|2056|626|8155|Thursday|2056Q2|N|N|N|2472090|2472180|2471736|2472011|N|N|N|N|N| +2472103|AAAAAAAAHKILFCAA|2056-04-14|1875|8155|626|2056|5|4|14|2|2056|626|8155|Friday|2056Q2|N|Y|N|2472090|2472180|2471737|2472012|N|N|N|N|N| +2472104|AAAAAAAAIKILFCAA|2056-04-15|1875|8155|626|2056|6|4|15|2|2056|626|8155|Saturday|2056Q2|N|Y|N|2472090|2472180|2471738|2472013|N|N|N|N|N| +2472105|AAAAAAAAJKILFCAA|2056-04-16|1875|8155|626|2056|0|4|16|2|2056|626|8155|Sunday|2056Q2|N|N|N|2472090|2472180|2471739|2472014|N|N|N|N|N| +2472106|AAAAAAAAKKILFCAA|2056-04-17|1875|8155|626|2056|1|4|17|2|2056|626|8155|Monday|2056Q2|N|N|N|2472090|2472180|2471740|2472015|N|N|N|N|N| +2472107|AAAAAAAALKILFCAA|2056-04-18|1875|8156|626|2056|2|4|18|2|2056|626|8156|Tuesday|2056Q2|N|N|N|2472090|2472180|2471741|2472016|N|N|N|N|N| +2472108|AAAAAAAAMKILFCAA|2056-04-19|1875|8156|626|2056|3|4|19|2|2056|626|8156|Wednesday|2056Q2|N|N|N|2472090|2472180|2471742|2472017|N|N|N|N|N| +2472109|AAAAAAAANKILFCAA|2056-04-20|1875|8156|626|2056|4|4|20|2|2056|626|8156|Thursday|2056Q2|N|N|N|2472090|2472180|2471743|2472018|N|N|N|N|N| +2472110|AAAAAAAAOKILFCAA|2056-04-21|1875|8156|626|2056|5|4|21|2|2056|626|8156|Friday|2056Q2|N|Y|N|2472090|2472180|2471744|2472019|N|N|N|N|N| +2472111|AAAAAAAAPKILFCAA|2056-04-22|1875|8156|626|2056|6|4|22|2|2056|626|8156|Saturday|2056Q2|N|Y|N|2472090|2472180|2471745|2472020|N|N|N|N|N| +2472112|AAAAAAAAALILFCAA|2056-04-23|1875|8156|626|2056|0|4|23|2|2056|626|8156|Sunday|2056Q2|N|N|N|2472090|2472180|2471746|2472021|N|N|N|N|N| +2472113|AAAAAAAABLILFCAA|2056-04-24|1875|8156|626|2056|1|4|24|2|2056|626|8156|Monday|2056Q2|N|N|N|2472090|2472180|2471747|2472022|N|N|N|N|N| +2472114|AAAAAAAACLILFCAA|2056-04-25|1875|8157|626|2056|2|4|25|2|2056|626|8157|Tuesday|2056Q2|N|N|N|2472090|2472180|2471748|2472023|N|N|N|N|N| +2472115|AAAAAAAADLILFCAA|2056-04-26|1875|8157|626|2056|3|4|26|2|2056|626|8157|Wednesday|2056Q2|N|N|N|2472090|2472180|2471749|2472024|N|N|N|N|N| +2472116|AAAAAAAAELILFCAA|2056-04-27|1875|8157|626|2056|4|4|27|2|2056|626|8157|Thursday|2056Q2|N|N|N|2472090|2472180|2471750|2472025|N|N|N|N|N| +2472117|AAAAAAAAFLILFCAA|2056-04-28|1875|8157|626|2056|5|4|28|2|2056|626|8157|Friday|2056Q2|N|Y|N|2472090|2472180|2471751|2472026|N|N|N|N|N| +2472118|AAAAAAAAGLILFCAA|2056-04-29|1875|8157|626|2056|6|4|29|2|2056|626|8157|Saturday|2056Q2|N|Y|N|2472090|2472180|2471752|2472027|N|N|N|N|N| +2472119|AAAAAAAAHLILFCAA|2056-04-30|1875|8157|626|2056|0|4|30|2|2056|626|8157|Sunday|2056Q2|N|N|N|2472090|2472180|2471753|2472028|N|N|N|N|N| +2472120|AAAAAAAAILILFCAA|2056-05-01|1876|8157|626|2056|1|5|1|2|2056|626|8157|Monday|2056Q2|N|N|N|2472120|2472240|2471754|2472029|N|N|N|N|N| +2472121|AAAAAAAAJLILFCAA|2056-05-02|1876|8158|626|2056|2|5|2|2|2056|626|8158|Tuesday|2056Q2|N|N|N|2472120|2472240|2471755|2472030|N|N|N|N|N| +2472122|AAAAAAAAKLILFCAA|2056-05-03|1876|8158|626|2056|3|5|3|2|2056|626|8158|Wednesday|2056Q2|N|N|N|2472120|2472240|2471756|2472031|N|N|N|N|N| +2472123|AAAAAAAALLILFCAA|2056-05-04|1876|8158|626|2056|4|5|4|2|2056|626|8158|Thursday|2056Q2|N|N|N|2472120|2472240|2471757|2472032|N|N|N|N|N| +2472124|AAAAAAAAMLILFCAA|2056-05-05|1876|8158|626|2056|5|5|5|2|2056|626|8158|Friday|2056Q2|N|Y|N|2472120|2472240|2471758|2472033|N|N|N|N|N| +2472125|AAAAAAAANLILFCAA|2056-05-06|1876|8158|626|2056|6|5|6|2|2056|626|8158|Saturday|2056Q2|N|Y|N|2472120|2472240|2471759|2472034|N|N|N|N|N| +2472126|AAAAAAAAOLILFCAA|2056-05-07|1876|8158|626|2056|0|5|7|2|2056|626|8158|Sunday|2056Q2|N|N|N|2472120|2472240|2471760|2472035|N|N|N|N|N| +2472127|AAAAAAAAPLILFCAA|2056-05-08|1876|8158|626|2056|1|5|8|2|2056|626|8158|Monday|2056Q2|N|N|N|2472120|2472240|2471761|2472036|N|N|N|N|N| +2472128|AAAAAAAAAMILFCAA|2056-05-09|1876|8159|626|2056|2|5|9|2|2056|626|8159|Tuesday|2056Q2|N|N|N|2472120|2472240|2471762|2472037|N|N|N|N|N| +2472129|AAAAAAAABMILFCAA|2056-05-10|1876|8159|626|2056|3|5|10|2|2056|626|8159|Wednesday|2056Q2|N|N|N|2472120|2472240|2471763|2472038|N|N|N|N|N| +2472130|AAAAAAAACMILFCAA|2056-05-11|1876|8159|626|2056|4|5|11|2|2056|626|8159|Thursday|2056Q2|N|N|N|2472120|2472240|2471764|2472039|N|N|N|N|N| +2472131|AAAAAAAADMILFCAA|2056-05-12|1876|8159|626|2056|5|5|12|2|2056|626|8159|Friday|2056Q2|N|Y|N|2472120|2472240|2471765|2472040|N|N|N|N|N| +2472132|AAAAAAAAEMILFCAA|2056-05-13|1876|8159|626|2056|6|5|13|2|2056|626|8159|Saturday|2056Q2|N|Y|N|2472120|2472240|2471766|2472041|N|N|N|N|N| +2472133|AAAAAAAAFMILFCAA|2056-05-14|1876|8159|626|2056|0|5|14|2|2056|626|8159|Sunday|2056Q2|N|N|N|2472120|2472240|2471767|2472042|N|N|N|N|N| +2472134|AAAAAAAAGMILFCAA|2056-05-15|1876|8159|626|2056|1|5|15|2|2056|626|8159|Monday|2056Q2|N|N|N|2472120|2472240|2471768|2472043|N|N|N|N|N| +2472135|AAAAAAAAHMILFCAA|2056-05-16|1876|8160|626|2056|2|5|16|2|2056|626|8160|Tuesday|2056Q2|N|N|N|2472120|2472240|2471769|2472044|N|N|N|N|N| +2472136|AAAAAAAAIMILFCAA|2056-05-17|1876|8160|626|2056|3|5|17|2|2056|626|8160|Wednesday|2056Q2|N|N|N|2472120|2472240|2471770|2472045|N|N|N|N|N| +2472137|AAAAAAAAJMILFCAA|2056-05-18|1876|8160|626|2056|4|5|18|2|2056|626|8160|Thursday|2056Q2|N|N|N|2472120|2472240|2471771|2472046|N|N|N|N|N| +2472138|AAAAAAAAKMILFCAA|2056-05-19|1876|8160|626|2056|5|5|19|2|2056|626|8160|Friday|2056Q2|N|Y|N|2472120|2472240|2471772|2472047|N|N|N|N|N| +2472139|AAAAAAAALMILFCAA|2056-05-20|1876|8160|626|2056|6|5|20|2|2056|626|8160|Saturday|2056Q2|N|Y|N|2472120|2472240|2471773|2472048|N|N|N|N|N| +2472140|AAAAAAAAMMILFCAA|2056-05-21|1876|8160|626|2056|0|5|21|2|2056|626|8160|Sunday|2056Q2|N|N|N|2472120|2472240|2471774|2472049|N|N|N|N|N| +2472141|AAAAAAAANMILFCAA|2056-05-22|1876|8160|626|2056|1|5|22|2|2056|626|8160|Monday|2056Q2|N|N|N|2472120|2472240|2471775|2472050|N|N|N|N|N| +2472142|AAAAAAAAOMILFCAA|2056-05-23|1876|8161|626|2056|2|5|23|2|2056|626|8161|Tuesday|2056Q2|N|N|N|2472120|2472240|2471776|2472051|N|N|N|N|N| +2472143|AAAAAAAAPMILFCAA|2056-05-24|1876|8161|626|2056|3|5|24|2|2056|626|8161|Wednesday|2056Q2|N|N|N|2472120|2472240|2471777|2472052|N|N|N|N|N| +2472144|AAAAAAAAANILFCAA|2056-05-25|1876|8161|626|2056|4|5|25|2|2056|626|8161|Thursday|2056Q2|N|N|N|2472120|2472240|2471778|2472053|N|N|N|N|N| +2472145|AAAAAAAABNILFCAA|2056-05-26|1876|8161|626|2056|5|5|26|2|2056|626|8161|Friday|2056Q2|N|Y|N|2472120|2472240|2471779|2472054|N|N|N|N|N| +2472146|AAAAAAAACNILFCAA|2056-05-27|1876|8161|626|2056|6|5|27|2|2056|626|8161|Saturday|2056Q2|N|Y|N|2472120|2472240|2471780|2472055|N|N|N|N|N| +2472147|AAAAAAAADNILFCAA|2056-05-28|1876|8161|626|2056|0|5|28|2|2056|626|8161|Sunday|2056Q2|N|N|N|2472120|2472240|2471781|2472056|N|N|N|N|N| +2472148|AAAAAAAAENILFCAA|2056-05-29|1876|8161|626|2056|1|5|29|2|2056|626|8161|Monday|2056Q2|N|N|N|2472120|2472240|2471782|2472057|N|N|N|N|N| +2472149|AAAAAAAAFNILFCAA|2056-05-30|1876|8162|626|2056|2|5|30|2|2056|626|8162|Tuesday|2056Q2|N|N|N|2472120|2472240|2471783|2472058|N|N|N|N|N| +2472150|AAAAAAAAGNILFCAA|2056-05-31|1876|8162|626|2056|3|5|31|2|2056|626|8162|Wednesday|2056Q2|N|N|N|2472120|2472240|2471784|2472059|N|N|N|N|N| +2472151|AAAAAAAAHNILFCAA|2056-06-01|1877|8162|627|2056|4|6|1|2|2056|627|8162|Thursday|2056Q2|N|N|N|2472151|2472302|2471785|2472060|N|N|N|N|N| +2472152|AAAAAAAAINILFCAA|2056-06-02|1877|8162|627|2056|5|6|2|2|2056|627|8162|Friday|2056Q2|N|Y|N|2472151|2472302|2471786|2472061|N|N|N|N|N| +2472153|AAAAAAAAJNILFCAA|2056-06-03|1877|8162|627|2056|6|6|3|2|2056|627|8162|Saturday|2056Q2|N|Y|N|2472151|2472302|2471787|2472062|N|N|N|N|N| +2472154|AAAAAAAAKNILFCAA|2056-06-04|1877|8162|627|2056|0|6|4|2|2056|627|8162|Sunday|2056Q2|N|N|N|2472151|2472302|2471788|2472063|N|N|N|N|N| +2472155|AAAAAAAALNILFCAA|2056-06-05|1877|8162|627|2056|1|6|5|2|2056|627|8162|Monday|2056Q2|N|N|N|2472151|2472302|2471789|2472064|N|N|N|N|N| +2472156|AAAAAAAAMNILFCAA|2056-06-06|1877|8163|627|2056|2|6|6|2|2056|627|8163|Tuesday|2056Q2|N|N|N|2472151|2472302|2471790|2472065|N|N|N|N|N| +2472157|AAAAAAAANNILFCAA|2056-06-07|1877|8163|627|2056|3|6|7|2|2056|627|8163|Wednesday|2056Q2|N|N|N|2472151|2472302|2471791|2472066|N|N|N|N|N| +2472158|AAAAAAAAONILFCAA|2056-06-08|1877|8163|627|2056|4|6|8|2|2056|627|8163|Thursday|2056Q2|N|N|N|2472151|2472302|2471792|2472067|N|N|N|N|N| +2472159|AAAAAAAAPNILFCAA|2056-06-09|1877|8163|627|2056|5|6|9|2|2056|627|8163|Friday|2056Q2|N|Y|N|2472151|2472302|2471793|2472068|N|N|N|N|N| +2472160|AAAAAAAAAOILFCAA|2056-06-10|1877|8163|627|2056|6|6|10|2|2056|627|8163|Saturday|2056Q2|N|Y|N|2472151|2472302|2471794|2472069|N|N|N|N|N| +2472161|AAAAAAAABOILFCAA|2056-06-11|1877|8163|627|2056|0|6|11|2|2056|627|8163|Sunday|2056Q2|N|N|N|2472151|2472302|2471795|2472070|N|N|N|N|N| +2472162|AAAAAAAACOILFCAA|2056-06-12|1877|8163|627|2056|1|6|12|2|2056|627|8163|Monday|2056Q2|N|N|N|2472151|2472302|2471796|2472071|N|N|N|N|N| +2472163|AAAAAAAADOILFCAA|2056-06-13|1877|8164|627|2056|2|6|13|2|2056|627|8164|Tuesday|2056Q2|N|N|N|2472151|2472302|2471797|2472072|N|N|N|N|N| +2472164|AAAAAAAAEOILFCAA|2056-06-14|1877|8164|627|2056|3|6|14|2|2056|627|8164|Wednesday|2056Q2|N|N|N|2472151|2472302|2471798|2472073|N|N|N|N|N| +2472165|AAAAAAAAFOILFCAA|2056-06-15|1877|8164|627|2056|4|6|15|2|2056|627|8164|Thursday|2056Q2|N|N|N|2472151|2472302|2471799|2472074|N|N|N|N|N| +2472166|AAAAAAAAGOILFCAA|2056-06-16|1877|8164|627|2056|5|6|16|2|2056|627|8164|Friday|2056Q2|N|Y|N|2472151|2472302|2471800|2472075|N|N|N|N|N| +2472167|AAAAAAAAHOILFCAA|2056-06-17|1877|8164|627|2056|6|6|17|2|2056|627|8164|Saturday|2056Q2|N|Y|N|2472151|2472302|2471801|2472076|N|N|N|N|N| +2472168|AAAAAAAAIOILFCAA|2056-06-18|1877|8164|627|2056|0|6|18|2|2056|627|8164|Sunday|2056Q2|N|N|N|2472151|2472302|2471802|2472077|N|N|N|N|N| +2472169|AAAAAAAAJOILFCAA|2056-06-19|1877|8164|627|2056|1|6|19|2|2056|627|8164|Monday|2056Q2|N|N|N|2472151|2472302|2471803|2472078|N|N|N|N|N| +2472170|AAAAAAAAKOILFCAA|2056-06-20|1877|8165|627|2056|2|6|20|2|2056|627|8165|Tuesday|2056Q2|N|N|N|2472151|2472302|2471804|2472079|N|N|N|N|N| +2472171|AAAAAAAALOILFCAA|2056-06-21|1877|8165|627|2056|3|6|21|2|2056|627|8165|Wednesday|2056Q2|N|N|N|2472151|2472302|2471805|2472080|N|N|N|N|N| +2472172|AAAAAAAAMOILFCAA|2056-06-22|1877|8165|627|2056|4|6|22|2|2056|627|8165|Thursday|2056Q2|N|N|N|2472151|2472302|2471806|2472081|N|N|N|N|N| +2472173|AAAAAAAANOILFCAA|2056-06-23|1877|8165|627|2056|5|6|23|2|2056|627|8165|Friday|2056Q2|N|Y|N|2472151|2472302|2471807|2472082|N|N|N|N|N| +2472174|AAAAAAAAOOILFCAA|2056-06-24|1877|8165|627|2056|6|6|24|2|2056|627|8165|Saturday|2056Q2|N|Y|N|2472151|2472302|2471808|2472083|N|N|N|N|N| +2472175|AAAAAAAAPOILFCAA|2056-06-25|1877|8165|627|2056|0|6|25|2|2056|627|8165|Sunday|2056Q2|N|N|N|2472151|2472302|2471809|2472084|N|N|N|N|N| +2472176|AAAAAAAAAPILFCAA|2056-06-26|1877|8165|627|2056|1|6|26|2|2056|627|8165|Monday|2056Q2|N|N|N|2472151|2472302|2471810|2472085|N|N|N|N|N| +2472177|AAAAAAAABPILFCAA|2056-06-27|1877|8166|627|2056|2|6|27|2|2056|627|8166|Tuesday|2056Q2|N|N|N|2472151|2472302|2471811|2472086|N|N|N|N|N| +2472178|AAAAAAAACPILFCAA|2056-06-28|1877|8166|627|2056|3|6|28|2|2056|627|8166|Wednesday|2056Q2|N|N|N|2472151|2472302|2471812|2472087|N|N|N|N|N| +2472179|AAAAAAAADPILFCAA|2056-06-29|1877|8166|627|2056|4|6|29|2|2056|627|8166|Thursday|2056Q2|N|N|N|2472151|2472302|2471813|2472088|N|N|N|N|N| +2472180|AAAAAAAAEPILFCAA|2056-06-30|1877|8166|627|2056|5|6|30|2|2056|627|8166|Friday|2056Q2|N|Y|N|2472151|2472302|2471814|2472089|N|N|N|N|N| +2472181|AAAAAAAAFPILFCAA|2056-07-01|1878|8166|627|2056|6|7|1|3|2056|627|8166|Saturday|2056Q3|N|Y|N|2472181|2472362|2471815|2472090|N|N|N|N|N| +2472182|AAAAAAAAGPILFCAA|2056-07-02|1878|8166|627|2056|0|7|2|3|2056|627|8166|Sunday|2056Q3|N|N|N|2472181|2472362|2471816|2472091|N|N|N|N|N| +2472183|AAAAAAAAHPILFCAA|2056-07-03|1878|8166|627|2056|1|7|3|3|2056|627|8166|Monday|2056Q3|N|N|N|2472181|2472362|2471817|2472092|N|N|N|N|N| +2472184|AAAAAAAAIPILFCAA|2056-07-04|1878|8167|627|2056|2|7|4|3|2056|627|8167|Tuesday|2056Q3|Y|N|N|2472181|2472362|2471818|2472093|N|N|N|N|N| +2472185|AAAAAAAAJPILFCAA|2056-07-05|1878|8167|627|2056|3|7|5|3|2056|627|8167|Wednesday|2056Q3|N|N|Y|2472181|2472362|2471819|2472094|N|N|N|N|N| +2472186|AAAAAAAAKPILFCAA|2056-07-06|1878|8167|627|2056|4|7|6|3|2056|627|8167|Thursday|2056Q3|N|N|N|2472181|2472362|2471820|2472095|N|N|N|N|N| +2472187|AAAAAAAALPILFCAA|2056-07-07|1878|8167|627|2056|5|7|7|3|2056|627|8167|Friday|2056Q3|N|Y|N|2472181|2472362|2471821|2472096|N|N|N|N|N| +2472188|AAAAAAAAMPILFCAA|2056-07-08|1878|8167|627|2056|6|7|8|3|2056|627|8167|Saturday|2056Q3|N|Y|N|2472181|2472362|2471822|2472097|N|N|N|N|N| +2472189|AAAAAAAANPILFCAA|2056-07-09|1878|8167|627|2056|0|7|9|3|2056|627|8167|Sunday|2056Q3|N|N|N|2472181|2472362|2471823|2472098|N|N|N|N|N| +2472190|AAAAAAAAOPILFCAA|2056-07-10|1878|8167|627|2056|1|7|10|3|2056|627|8167|Monday|2056Q3|N|N|N|2472181|2472362|2471824|2472099|N|N|N|N|N| +2472191|AAAAAAAAPPILFCAA|2056-07-11|1878|8168|627|2056|2|7|11|3|2056|627|8168|Tuesday|2056Q3|N|N|N|2472181|2472362|2471825|2472100|N|N|N|N|N| +2472192|AAAAAAAAAAJLFCAA|2056-07-12|1878|8168|627|2056|3|7|12|3|2056|627|8168|Wednesday|2056Q3|N|N|N|2472181|2472362|2471826|2472101|N|N|N|N|N| +2472193|AAAAAAAABAJLFCAA|2056-07-13|1878|8168|627|2056|4|7|13|3|2056|627|8168|Thursday|2056Q3|N|N|N|2472181|2472362|2471827|2472102|N|N|N|N|N| +2472194|AAAAAAAACAJLFCAA|2056-07-14|1878|8168|627|2056|5|7|14|3|2056|627|8168|Friday|2056Q3|N|Y|N|2472181|2472362|2471828|2472103|N|N|N|N|N| +2472195|AAAAAAAADAJLFCAA|2056-07-15|1878|8168|627|2056|6|7|15|3|2056|627|8168|Saturday|2056Q3|N|Y|N|2472181|2472362|2471829|2472104|N|N|N|N|N| +2472196|AAAAAAAAEAJLFCAA|2056-07-16|1878|8168|627|2056|0|7|16|3|2056|627|8168|Sunday|2056Q3|N|N|N|2472181|2472362|2471830|2472105|N|N|N|N|N| +2472197|AAAAAAAAFAJLFCAA|2056-07-17|1878|8168|627|2056|1|7|17|3|2056|627|8168|Monday|2056Q3|N|N|N|2472181|2472362|2471831|2472106|N|N|N|N|N| +2472198|AAAAAAAAGAJLFCAA|2056-07-18|1878|8169|627|2056|2|7|18|3|2056|627|8169|Tuesday|2056Q3|N|N|N|2472181|2472362|2471832|2472107|N|N|N|N|N| +2472199|AAAAAAAAHAJLFCAA|2056-07-19|1878|8169|627|2056|3|7|19|3|2056|627|8169|Wednesday|2056Q3|N|N|N|2472181|2472362|2471833|2472108|N|N|N|N|N| +2472200|AAAAAAAAIAJLFCAA|2056-07-20|1878|8169|627|2056|4|7|20|3|2056|627|8169|Thursday|2056Q3|N|N|N|2472181|2472362|2471834|2472109|N|N|N|N|N| +2472201|AAAAAAAAJAJLFCAA|2056-07-21|1878|8169|627|2056|5|7|21|3|2056|627|8169|Friday|2056Q3|N|Y|N|2472181|2472362|2471835|2472110|N|N|N|N|N| +2472202|AAAAAAAAKAJLFCAA|2056-07-22|1878|8169|627|2056|6|7|22|3|2056|627|8169|Saturday|2056Q3|N|Y|N|2472181|2472362|2471836|2472111|N|N|N|N|N| +2472203|AAAAAAAALAJLFCAA|2056-07-23|1878|8169|627|2056|0|7|23|3|2056|627|8169|Sunday|2056Q3|N|N|N|2472181|2472362|2471837|2472112|N|N|N|N|N| +2472204|AAAAAAAAMAJLFCAA|2056-07-24|1878|8169|627|2056|1|7|24|3|2056|627|8169|Monday|2056Q3|N|N|N|2472181|2472362|2471838|2472113|N|N|N|N|N| +2472205|AAAAAAAANAJLFCAA|2056-07-25|1878|8170|627|2056|2|7|25|3|2056|627|8170|Tuesday|2056Q3|N|N|N|2472181|2472362|2471839|2472114|N|N|N|N|N| +2472206|AAAAAAAAOAJLFCAA|2056-07-26|1878|8170|627|2056|3|7|26|3|2056|627|8170|Wednesday|2056Q3|N|N|N|2472181|2472362|2471840|2472115|N|N|N|N|N| +2472207|AAAAAAAAPAJLFCAA|2056-07-27|1878|8170|627|2056|4|7|27|3|2056|627|8170|Thursday|2056Q3|N|N|N|2472181|2472362|2471841|2472116|N|N|N|N|N| +2472208|AAAAAAAAABJLFCAA|2056-07-28|1878|8170|627|2056|5|7|28|3|2056|627|8170|Friday|2056Q3|N|Y|N|2472181|2472362|2471842|2472117|N|N|N|N|N| +2472209|AAAAAAAABBJLFCAA|2056-07-29|1878|8170|627|2056|6|7|29|3|2056|627|8170|Saturday|2056Q3|N|Y|N|2472181|2472362|2471843|2472118|N|N|N|N|N| +2472210|AAAAAAAACBJLFCAA|2056-07-30|1878|8170|627|2056|0|7|30|3|2056|627|8170|Sunday|2056Q3|N|N|N|2472181|2472362|2471844|2472119|N|N|N|N|N| +2472211|AAAAAAAADBJLFCAA|2056-07-31|1878|8170|627|2056|1|7|31|3|2056|627|8170|Monday|2056Q3|N|N|N|2472181|2472362|2471845|2472120|N|N|N|N|N| +2472212|AAAAAAAAEBJLFCAA|2056-08-01|1879|8171|627|2056|2|8|1|3|2056|627|8171|Tuesday|2056Q3|N|N|N|2472212|2472424|2471846|2472121|N|N|N|N|N| +2472213|AAAAAAAAFBJLFCAA|2056-08-02|1879|8171|627|2056|3|8|2|3|2056|627|8171|Wednesday|2056Q3|N|N|N|2472212|2472424|2471847|2472122|N|N|N|N|N| +2472214|AAAAAAAAGBJLFCAA|2056-08-03|1879|8171|627|2056|4|8|3|3|2056|627|8171|Thursday|2056Q3|N|N|N|2472212|2472424|2471848|2472123|N|N|N|N|N| +2472215|AAAAAAAAHBJLFCAA|2056-08-04|1879|8171|627|2056|5|8|4|3|2056|627|8171|Friday|2056Q3|N|Y|N|2472212|2472424|2471849|2472124|N|N|N|N|N| +2472216|AAAAAAAAIBJLFCAA|2056-08-05|1879|8171|627|2056|6|8|5|3|2056|627|8171|Saturday|2056Q3|N|Y|N|2472212|2472424|2471850|2472125|N|N|N|N|N| +2472217|AAAAAAAAJBJLFCAA|2056-08-06|1879|8171|627|2056|0|8|6|3|2056|627|8171|Sunday|2056Q3|N|N|N|2472212|2472424|2471851|2472126|N|N|N|N|N| +2472218|AAAAAAAAKBJLFCAA|2056-08-07|1879|8171|627|2056|1|8|7|3|2056|627|8171|Monday|2056Q3|N|N|N|2472212|2472424|2471852|2472127|N|N|N|N|N| +2472219|AAAAAAAALBJLFCAA|2056-08-08|1879|8172|627|2056|2|8|8|3|2056|627|8172|Tuesday|2056Q3|N|N|N|2472212|2472424|2471853|2472128|N|N|N|N|N| +2472220|AAAAAAAAMBJLFCAA|2056-08-09|1879|8172|627|2056|3|8|9|3|2056|627|8172|Wednesday|2056Q3|N|N|N|2472212|2472424|2471854|2472129|N|N|N|N|N| +2472221|AAAAAAAANBJLFCAA|2056-08-10|1879|8172|627|2056|4|8|10|3|2056|627|8172|Thursday|2056Q3|N|N|N|2472212|2472424|2471855|2472130|N|N|N|N|N| +2472222|AAAAAAAAOBJLFCAA|2056-08-11|1879|8172|627|2056|5|8|11|3|2056|627|8172|Friday|2056Q3|N|Y|N|2472212|2472424|2471856|2472131|N|N|N|N|N| +2472223|AAAAAAAAPBJLFCAA|2056-08-12|1879|8172|627|2056|6|8|12|3|2056|627|8172|Saturday|2056Q3|N|Y|N|2472212|2472424|2471857|2472132|N|N|N|N|N| +2472224|AAAAAAAAACJLFCAA|2056-08-13|1879|8172|627|2056|0|8|13|3|2056|627|8172|Sunday|2056Q3|N|N|N|2472212|2472424|2471858|2472133|N|N|N|N|N| +2472225|AAAAAAAABCJLFCAA|2056-08-14|1879|8172|627|2056|1|8|14|3|2056|627|8172|Monday|2056Q3|N|N|N|2472212|2472424|2471859|2472134|N|N|N|N|N| +2472226|AAAAAAAACCJLFCAA|2056-08-15|1879|8173|627|2056|2|8|15|3|2056|627|8173|Tuesday|2056Q3|N|N|N|2472212|2472424|2471860|2472135|N|N|N|N|N| +2472227|AAAAAAAADCJLFCAA|2056-08-16|1879|8173|627|2056|3|8|16|3|2056|627|8173|Wednesday|2056Q3|N|N|N|2472212|2472424|2471861|2472136|N|N|N|N|N| +2472228|AAAAAAAAECJLFCAA|2056-08-17|1879|8173|627|2056|4|8|17|3|2056|627|8173|Thursday|2056Q3|N|N|N|2472212|2472424|2471862|2472137|N|N|N|N|N| +2472229|AAAAAAAAFCJLFCAA|2056-08-18|1879|8173|627|2056|5|8|18|3|2056|627|8173|Friday|2056Q3|N|Y|N|2472212|2472424|2471863|2472138|N|N|N|N|N| +2472230|AAAAAAAAGCJLFCAA|2056-08-19|1879|8173|627|2056|6|8|19|3|2056|627|8173|Saturday|2056Q3|N|Y|N|2472212|2472424|2471864|2472139|N|N|N|N|N| +2472231|AAAAAAAAHCJLFCAA|2056-08-20|1879|8173|627|2056|0|8|20|3|2056|627|8173|Sunday|2056Q3|N|N|N|2472212|2472424|2471865|2472140|N|N|N|N|N| +2472232|AAAAAAAAICJLFCAA|2056-08-21|1879|8173|627|2056|1|8|21|3|2056|627|8173|Monday|2056Q3|N|N|N|2472212|2472424|2471866|2472141|N|N|N|N|N| +2472233|AAAAAAAAJCJLFCAA|2056-08-22|1879|8174|627|2056|2|8|22|3|2056|627|8174|Tuesday|2056Q3|N|N|N|2472212|2472424|2471867|2472142|N|N|N|N|N| +2472234|AAAAAAAAKCJLFCAA|2056-08-23|1879|8174|627|2056|3|8|23|3|2056|627|8174|Wednesday|2056Q3|N|N|N|2472212|2472424|2471868|2472143|N|N|N|N|N| +2472235|AAAAAAAALCJLFCAA|2056-08-24|1879|8174|627|2056|4|8|24|3|2056|627|8174|Thursday|2056Q3|N|N|N|2472212|2472424|2471869|2472144|N|N|N|N|N| +2472236|AAAAAAAAMCJLFCAA|2056-08-25|1879|8174|627|2056|5|8|25|3|2056|627|8174|Friday|2056Q3|N|Y|N|2472212|2472424|2471870|2472145|N|N|N|N|N| +2472237|AAAAAAAANCJLFCAA|2056-08-26|1879|8174|627|2056|6|8|26|3|2056|627|8174|Saturday|2056Q3|N|Y|N|2472212|2472424|2471871|2472146|N|N|N|N|N| +2472238|AAAAAAAAOCJLFCAA|2056-08-27|1879|8174|627|2056|0|8|27|3|2056|627|8174|Sunday|2056Q3|N|N|N|2472212|2472424|2471872|2472147|N|N|N|N|N| +2472239|AAAAAAAAPCJLFCAA|2056-08-28|1879|8174|627|2056|1|8|28|3|2056|627|8174|Monday|2056Q3|N|N|N|2472212|2472424|2471873|2472148|N|N|N|N|N| +2472240|AAAAAAAAADJLFCAA|2056-08-29|1879|8175|627|2056|2|8|29|3|2056|627|8175|Tuesday|2056Q3|N|N|N|2472212|2472424|2471874|2472149|N|N|N|N|N| +2472241|AAAAAAAABDJLFCAA|2056-08-30|1879|8175|627|2056|3|8|30|3|2056|627|8175|Wednesday|2056Q3|N|N|N|2472212|2472424|2471875|2472150|N|N|N|N|N| +2472242|AAAAAAAACDJLFCAA|2056-08-31|1879|8175|627|2056|4|8|31|3|2056|627|8175|Thursday|2056Q3|N|N|N|2472212|2472424|2471876|2472151|N|N|N|N|N| +2472243|AAAAAAAADDJLFCAA|2056-09-01|1880|8175|628|2056|5|9|1|3|2056|628|8175|Friday|2056Q3|N|Y|N|2472243|2472486|2471877|2472152|N|N|N|N|N| +2472244|AAAAAAAAEDJLFCAA|2056-09-02|1880|8175|628|2056|6|9|2|3|2056|628|8175|Saturday|2056Q3|N|Y|N|2472243|2472486|2471878|2472153|N|N|N|N|N| +2472245|AAAAAAAAFDJLFCAA|2056-09-03|1880|8175|628|2056|0|9|3|3|2056|628|8175|Sunday|2056Q3|N|N|N|2472243|2472486|2471879|2472154|N|N|N|N|N| +2472246|AAAAAAAAGDJLFCAA|2056-09-04|1880|8175|628|2056|1|9|4|3|2056|628|8175|Monday|2056Q3|N|N|N|2472243|2472486|2471880|2472155|N|N|N|N|N| +2472247|AAAAAAAAHDJLFCAA|2056-09-05|1880|8176|628|2056|2|9|5|3|2056|628|8176|Tuesday|2056Q3|N|N|N|2472243|2472486|2471881|2472156|N|N|N|N|N| +2472248|AAAAAAAAIDJLFCAA|2056-09-06|1880|8176|628|2056|3|9|6|3|2056|628|8176|Wednesday|2056Q3|N|N|N|2472243|2472486|2471882|2472157|N|N|N|N|N| +2472249|AAAAAAAAJDJLFCAA|2056-09-07|1880|8176|628|2056|4|9|7|3|2056|628|8176|Thursday|2056Q3|N|N|N|2472243|2472486|2471883|2472158|N|N|N|N|N| +2472250|AAAAAAAAKDJLFCAA|2056-09-08|1880|8176|628|2056|5|9|8|3|2056|628|8176|Friday|2056Q3|N|Y|N|2472243|2472486|2471884|2472159|N|N|N|N|N| +2472251|AAAAAAAALDJLFCAA|2056-09-09|1880|8176|628|2056|6|9|9|3|2056|628|8176|Saturday|2056Q3|N|Y|N|2472243|2472486|2471885|2472160|N|N|N|N|N| +2472252|AAAAAAAAMDJLFCAA|2056-09-10|1880|8176|628|2056|0|9|10|3|2056|628|8176|Sunday|2056Q3|N|N|N|2472243|2472486|2471886|2472161|N|N|N|N|N| +2472253|AAAAAAAANDJLFCAA|2056-09-11|1880|8176|628|2056|1|9|11|3|2056|628|8176|Monday|2056Q3|N|N|N|2472243|2472486|2471887|2472162|N|N|N|N|N| +2472254|AAAAAAAAODJLFCAA|2056-09-12|1880|8177|628|2056|2|9|12|3|2056|628|8177|Tuesday|2056Q3|N|N|N|2472243|2472486|2471888|2472163|N|N|N|N|N| +2472255|AAAAAAAAPDJLFCAA|2056-09-13|1880|8177|628|2056|3|9|13|3|2056|628|8177|Wednesday|2056Q3|N|N|N|2472243|2472486|2471889|2472164|N|N|N|N|N| +2472256|AAAAAAAAAEJLFCAA|2056-09-14|1880|8177|628|2056|4|9|14|3|2056|628|8177|Thursday|2056Q3|N|N|N|2472243|2472486|2471890|2472165|N|N|N|N|N| +2472257|AAAAAAAABEJLFCAA|2056-09-15|1880|8177|628|2056|5|9|15|3|2056|628|8177|Friday|2056Q3|N|Y|N|2472243|2472486|2471891|2472166|N|N|N|N|N| +2472258|AAAAAAAACEJLFCAA|2056-09-16|1880|8177|628|2056|6|9|16|3|2056|628|8177|Saturday|2056Q3|N|Y|N|2472243|2472486|2471892|2472167|N|N|N|N|N| +2472259|AAAAAAAADEJLFCAA|2056-09-17|1880|8177|628|2056|0|9|17|3|2056|628|8177|Sunday|2056Q3|N|N|N|2472243|2472486|2471893|2472168|N|N|N|N|N| +2472260|AAAAAAAAEEJLFCAA|2056-09-18|1880|8177|628|2056|1|9|18|3|2056|628|8177|Monday|2056Q3|N|N|N|2472243|2472486|2471894|2472169|N|N|N|N|N| +2472261|AAAAAAAAFEJLFCAA|2056-09-19|1880|8178|628|2056|2|9|19|3|2056|628|8178|Tuesday|2056Q3|N|N|N|2472243|2472486|2471895|2472170|N|N|N|N|N| +2472262|AAAAAAAAGEJLFCAA|2056-09-20|1880|8178|628|2056|3|9|20|3|2056|628|8178|Wednesday|2056Q3|N|N|N|2472243|2472486|2471896|2472171|N|N|N|N|N| +2472263|AAAAAAAAHEJLFCAA|2056-09-21|1880|8178|628|2056|4|9|21|3|2056|628|8178|Thursday|2056Q3|N|N|N|2472243|2472486|2471897|2472172|N|N|N|N|N| +2472264|AAAAAAAAIEJLFCAA|2056-09-22|1880|8178|628|2056|5|9|22|3|2056|628|8178|Friday|2056Q3|N|Y|N|2472243|2472486|2471898|2472173|N|N|N|N|N| +2472265|AAAAAAAAJEJLFCAA|2056-09-23|1880|8178|628|2056|6|9|23|3|2056|628|8178|Saturday|2056Q3|N|Y|N|2472243|2472486|2471899|2472174|N|N|N|N|N| +2472266|AAAAAAAAKEJLFCAA|2056-09-24|1880|8178|628|2056|0|9|24|3|2056|628|8178|Sunday|2056Q3|N|N|N|2472243|2472486|2471900|2472175|N|N|N|N|N| +2472267|AAAAAAAALEJLFCAA|2056-09-25|1880|8178|628|2056|1|9|25|3|2056|628|8178|Monday|2056Q3|N|N|N|2472243|2472486|2471901|2472176|N|N|N|N|N| +2472268|AAAAAAAAMEJLFCAA|2056-09-26|1880|8179|628|2056|2|9|26|3|2056|628|8179|Tuesday|2056Q3|N|N|N|2472243|2472486|2471902|2472177|N|N|N|N|N| +2472269|AAAAAAAANEJLFCAA|2056-09-27|1880|8179|628|2056|3|9|27|3|2056|628|8179|Wednesday|2056Q3|N|N|N|2472243|2472486|2471903|2472178|N|N|N|N|N| +2472270|AAAAAAAAOEJLFCAA|2056-09-28|1880|8179|628|2056|4|9|28|3|2056|628|8179|Thursday|2056Q3|N|N|N|2472243|2472486|2471904|2472179|N|N|N|N|N| +2472271|AAAAAAAAPEJLFCAA|2056-09-29|1880|8179|628|2056|5|9|29|3|2056|628|8179|Friday|2056Q3|N|Y|N|2472243|2472486|2471905|2472180|N|N|N|N|N| +2472272|AAAAAAAAAFJLFCAA|2056-09-30|1880|8179|628|2056|6|9|30|3|2056|628|8179|Saturday|2056Q3|N|Y|N|2472243|2472486|2471906|2472181|N|N|N|N|N| +2472273|AAAAAAAABFJLFCAA|2056-10-01|1881|8179|628|2056|0|10|1|4|2056|628|8179|Sunday|2056Q4|N|N|N|2472273|2472546|2471907|2472181|N|N|N|N|N| +2472274|AAAAAAAACFJLFCAA|2056-10-02|1881|8179|628|2056|1|10|2|4|2056|628|8179|Monday|2056Q4|N|N|N|2472273|2472546|2471908|2472182|N|N|N|N|N| +2472275|AAAAAAAADFJLFCAA|2056-10-03|1881|8180|628|2056|2|10|3|4|2056|628|8180|Tuesday|2056Q4|N|N|N|2472273|2472546|2471909|2472183|N|N|N|N|N| +2472276|AAAAAAAAEFJLFCAA|2056-10-04|1881|8180|628|2056|3|10|4|4|2056|628|8180|Wednesday|2056Q4|N|N|N|2472273|2472546|2471910|2472184|N|N|N|N|N| +2472277|AAAAAAAAFFJLFCAA|2056-10-05|1881|8180|628|2056|4|10|5|4|2056|628|8180|Thursday|2056Q4|N|N|N|2472273|2472546|2471911|2472185|N|N|N|N|N| +2472278|AAAAAAAAGFJLFCAA|2056-10-06|1881|8180|628|2056|5|10|6|4|2056|628|8180|Friday|2056Q4|N|Y|N|2472273|2472546|2471912|2472186|N|N|N|N|N| +2472279|AAAAAAAAHFJLFCAA|2056-10-07|1881|8180|628|2056|6|10|7|4|2056|628|8180|Saturday|2056Q4|N|Y|N|2472273|2472546|2471913|2472187|N|N|N|N|N| +2472280|AAAAAAAAIFJLFCAA|2056-10-08|1881|8180|628|2056|0|10|8|4|2056|628|8180|Sunday|2056Q4|N|N|N|2472273|2472546|2471914|2472188|N|N|N|N|N| +2472281|AAAAAAAAJFJLFCAA|2056-10-09|1881|8180|628|2056|1|10|9|4|2056|628|8180|Monday|2056Q4|N|N|N|2472273|2472546|2471915|2472189|N|N|N|N|N| +2472282|AAAAAAAAKFJLFCAA|2056-10-10|1881|8181|628|2056|2|10|10|4|2056|628|8181|Tuesday|2056Q4|N|N|N|2472273|2472546|2471916|2472190|N|N|N|N|N| +2472283|AAAAAAAALFJLFCAA|2056-10-11|1881|8181|628|2056|3|10|11|4|2056|628|8181|Wednesday|2056Q4|N|N|N|2472273|2472546|2471917|2472191|N|N|N|N|N| +2472284|AAAAAAAAMFJLFCAA|2056-10-12|1881|8181|628|2056|4|10|12|4|2056|628|8181|Thursday|2056Q4|N|N|N|2472273|2472546|2471918|2472192|N|N|N|N|N| +2472285|AAAAAAAANFJLFCAA|2056-10-13|1881|8181|628|2056|5|10|13|4|2056|628|8181|Friday|2056Q4|N|Y|N|2472273|2472546|2471919|2472193|N|N|N|N|N| +2472286|AAAAAAAAOFJLFCAA|2056-10-14|1881|8181|628|2056|6|10|14|4|2056|628|8181|Saturday|2056Q4|N|Y|N|2472273|2472546|2471920|2472194|N|N|N|N|N| +2472287|AAAAAAAAPFJLFCAA|2056-10-15|1881|8181|628|2056|0|10|15|4|2056|628|8181|Sunday|2056Q4|N|N|N|2472273|2472546|2471921|2472195|N|N|N|N|N| +2472288|AAAAAAAAAGJLFCAA|2056-10-16|1881|8181|628|2056|1|10|16|4|2056|628|8181|Monday|2056Q4|N|N|N|2472273|2472546|2471922|2472196|N|N|N|N|N| +2472289|AAAAAAAABGJLFCAA|2056-10-17|1881|8182|628|2056|2|10|17|4|2056|628|8182|Tuesday|2056Q4|N|N|N|2472273|2472546|2471923|2472197|N|N|N|N|N| +2472290|AAAAAAAACGJLFCAA|2056-10-18|1881|8182|628|2056|3|10|18|4|2056|628|8182|Wednesday|2056Q4|N|N|N|2472273|2472546|2471924|2472198|N|N|N|N|N| +2472291|AAAAAAAADGJLFCAA|2056-10-19|1881|8182|628|2056|4|10|19|4|2056|628|8182|Thursday|2056Q4|N|N|N|2472273|2472546|2471925|2472199|N|N|N|N|N| +2472292|AAAAAAAAEGJLFCAA|2056-10-20|1881|8182|628|2056|5|10|20|4|2056|628|8182|Friday|2056Q4|N|Y|N|2472273|2472546|2471926|2472200|N|N|N|N|N| +2472293|AAAAAAAAFGJLFCAA|2056-10-21|1881|8182|628|2056|6|10|21|4|2056|628|8182|Saturday|2056Q4|N|Y|N|2472273|2472546|2471927|2472201|N|N|N|N|N| +2472294|AAAAAAAAGGJLFCAA|2056-10-22|1881|8182|628|2056|0|10|22|4|2056|628|8182|Sunday|2056Q4|N|N|N|2472273|2472546|2471928|2472202|N|N|N|N|N| +2472295|AAAAAAAAHGJLFCAA|2056-10-23|1881|8182|628|2056|1|10|23|4|2056|628|8182|Monday|2056Q4|N|N|N|2472273|2472546|2471929|2472203|N|N|N|N|N| +2472296|AAAAAAAAIGJLFCAA|2056-10-24|1881|8183|628|2056|2|10|24|4|2056|628|8183|Tuesday|2056Q4|N|N|N|2472273|2472546|2471930|2472204|N|N|N|N|N| +2472297|AAAAAAAAJGJLFCAA|2056-10-25|1881|8183|628|2056|3|10|25|4|2056|628|8183|Wednesday|2056Q4|N|N|N|2472273|2472546|2471931|2472205|N|N|N|N|N| +2472298|AAAAAAAAKGJLFCAA|2056-10-26|1881|8183|628|2056|4|10|26|4|2056|628|8183|Thursday|2056Q4|N|N|N|2472273|2472546|2471932|2472206|N|N|N|N|N| +2472299|AAAAAAAALGJLFCAA|2056-10-27|1881|8183|628|2056|5|10|27|4|2056|628|8183|Friday|2056Q4|N|Y|N|2472273|2472546|2471933|2472207|N|N|N|N|N| +2472300|AAAAAAAAMGJLFCAA|2056-10-28|1881|8183|628|2056|6|10|28|4|2056|628|8183|Saturday|2056Q4|N|Y|N|2472273|2472546|2471934|2472208|N|N|N|N|N| +2472301|AAAAAAAANGJLFCAA|2056-10-29|1881|8183|628|2056|0|10|29|4|2056|628|8183|Sunday|2056Q4|N|N|N|2472273|2472546|2471935|2472209|N|N|N|N|N| +2472302|AAAAAAAAOGJLFCAA|2056-10-30|1881|8183|628|2056|1|10|30|4|2056|628|8183|Monday|2056Q4|N|N|N|2472273|2472546|2471936|2472210|N|N|N|N|N| +2472303|AAAAAAAAPGJLFCAA|2056-10-31|1881|8184|628|2056|2|10|31|4|2056|628|8184|Tuesday|2056Q4|N|N|N|2472273|2472546|2471937|2472211|N|N|N|N|N| +2472304|AAAAAAAAAHJLFCAA|2056-11-01|1882|8184|628|2056|3|11|1|4|2056|628|8184|Wednesday|2056Q4|N|N|N|2472304|2472608|2471938|2472212|N|N|N|N|N| +2472305|AAAAAAAABHJLFCAA|2056-11-02|1882|8184|628|2056|4|11|2|4|2056|628|8184|Thursday|2056Q4|N|N|N|2472304|2472608|2471939|2472213|N|N|N|N|N| +2472306|AAAAAAAACHJLFCAA|2056-11-03|1882|8184|628|2056|5|11|3|4|2056|628|8184|Friday|2056Q4|N|Y|N|2472304|2472608|2471940|2472214|N|N|N|N|N| +2472307|AAAAAAAADHJLFCAA|2056-11-04|1882|8184|628|2056|6|11|4|4|2056|628|8184|Saturday|2056Q4|N|Y|N|2472304|2472608|2471941|2472215|N|N|N|N|N| +2472308|AAAAAAAAEHJLFCAA|2056-11-05|1882|8184|628|2056|0|11|5|4|2056|628|8184|Sunday|2056Q4|N|N|N|2472304|2472608|2471942|2472216|N|N|N|N|N| +2472309|AAAAAAAAFHJLFCAA|2056-11-06|1882|8184|628|2056|1|11|6|4|2056|628|8184|Monday|2056Q4|N|N|N|2472304|2472608|2471943|2472217|N|N|N|N|N| +2472310|AAAAAAAAGHJLFCAA|2056-11-07|1882|8185|628|2056|2|11|7|4|2056|628|8185|Tuesday|2056Q4|N|N|N|2472304|2472608|2471944|2472218|N|N|N|N|N| +2472311|AAAAAAAAHHJLFCAA|2056-11-08|1882|8185|628|2056|3|11|8|4|2056|628|8185|Wednesday|2056Q4|N|N|N|2472304|2472608|2471945|2472219|N|N|N|N|N| +2472312|AAAAAAAAIHJLFCAA|2056-11-09|1882|8185|628|2056|4|11|9|4|2056|628|8185|Thursday|2056Q4|N|N|N|2472304|2472608|2471946|2472220|N|N|N|N|N| +2472313|AAAAAAAAJHJLFCAA|2056-11-10|1882|8185|628|2056|5|11|10|4|2056|628|8185|Friday|2056Q4|N|Y|N|2472304|2472608|2471947|2472221|N|N|N|N|N| +2472314|AAAAAAAAKHJLFCAA|2056-11-11|1882|8185|628|2056|6|11|11|4|2056|628|8185|Saturday|2056Q4|N|Y|N|2472304|2472608|2471948|2472222|N|N|N|N|N| +2472315|AAAAAAAALHJLFCAA|2056-11-12|1882|8185|628|2056|0|11|12|4|2056|628|8185|Sunday|2056Q4|N|N|N|2472304|2472608|2471949|2472223|N|N|N|N|N| +2472316|AAAAAAAAMHJLFCAA|2056-11-13|1882|8185|628|2056|1|11|13|4|2056|628|8185|Monday|2056Q4|N|N|N|2472304|2472608|2471950|2472224|N|N|N|N|N| +2472317|AAAAAAAANHJLFCAA|2056-11-14|1882|8186|628|2056|2|11|14|4|2056|628|8186|Tuesday|2056Q4|N|N|N|2472304|2472608|2471951|2472225|N|N|N|N|N| +2472318|AAAAAAAAOHJLFCAA|2056-11-15|1882|8186|628|2056|3|11|15|4|2056|628|8186|Wednesday|2056Q4|N|N|N|2472304|2472608|2471952|2472226|N|N|N|N|N| +2472319|AAAAAAAAPHJLFCAA|2056-11-16|1882|8186|628|2056|4|11|16|4|2056|628|8186|Thursday|2056Q4|N|N|N|2472304|2472608|2471953|2472227|N|N|N|N|N| +2472320|AAAAAAAAAIJLFCAA|2056-11-17|1882|8186|628|2056|5|11|17|4|2056|628|8186|Friday|2056Q4|N|Y|N|2472304|2472608|2471954|2472228|N|N|N|N|N| +2472321|AAAAAAAABIJLFCAA|2056-11-18|1882|8186|628|2056|6|11|18|4|2056|628|8186|Saturday|2056Q4|N|Y|N|2472304|2472608|2471955|2472229|N|N|N|N|N| +2472322|AAAAAAAACIJLFCAA|2056-11-19|1882|8186|628|2056|0|11|19|4|2056|628|8186|Sunday|2056Q4|N|N|N|2472304|2472608|2471956|2472230|N|N|N|N|N| +2472323|AAAAAAAADIJLFCAA|2056-11-20|1882|8186|628|2056|1|11|20|4|2056|628|8186|Monday|2056Q4|N|N|N|2472304|2472608|2471957|2472231|N|N|N|N|N| +2472324|AAAAAAAAEIJLFCAA|2056-11-21|1882|8187|628|2056|2|11|21|4|2056|628|8187|Tuesday|2056Q4|N|N|N|2472304|2472608|2471958|2472232|N|N|N|N|N| +2472325|AAAAAAAAFIJLFCAA|2056-11-22|1882|8187|628|2056|3|11|22|4|2056|628|8187|Wednesday|2056Q4|N|N|N|2472304|2472608|2471959|2472233|N|N|N|N|N| +2472326|AAAAAAAAGIJLFCAA|2056-11-23|1882|8187|628|2056|4|11|23|4|2056|628|8187|Thursday|2056Q4|N|N|N|2472304|2472608|2471960|2472234|N|N|N|N|N| +2472327|AAAAAAAAHIJLFCAA|2056-11-24|1882|8187|628|2056|5|11|24|4|2056|628|8187|Friday|2056Q4|N|Y|N|2472304|2472608|2471961|2472235|N|N|N|N|N| +2472328|AAAAAAAAIIJLFCAA|2056-11-25|1882|8187|628|2056|6|11|25|4|2056|628|8187|Saturday|2056Q4|N|Y|N|2472304|2472608|2471962|2472236|N|N|N|N|N| +2472329|AAAAAAAAJIJLFCAA|2056-11-26|1882|8187|628|2056|0|11|26|4|2056|628|8187|Sunday|2056Q4|N|N|N|2472304|2472608|2471963|2472237|N|N|N|N|N| +2472330|AAAAAAAAKIJLFCAA|2056-11-27|1882|8187|628|2056|1|11|27|4|2056|628|8187|Monday|2056Q4|N|N|N|2472304|2472608|2471964|2472238|N|N|N|N|N| +2472331|AAAAAAAALIJLFCAA|2056-11-28|1882|8188|628|2056|2|11|28|4|2056|628|8188|Tuesday|2056Q4|N|N|N|2472304|2472608|2471965|2472239|N|N|N|N|N| +2472332|AAAAAAAAMIJLFCAA|2056-11-29|1882|8188|628|2056|3|11|29|4|2056|628|8188|Wednesday|2056Q4|N|N|N|2472304|2472608|2471966|2472240|N|N|N|N|N| +2472333|AAAAAAAANIJLFCAA|2056-11-30|1882|8188|628|2056|4|11|30|4|2056|628|8188|Thursday|2056Q4|N|N|N|2472304|2472608|2471967|2472241|N|N|N|N|N| +2472334|AAAAAAAAOIJLFCAA|2056-12-01|1883|8188|629|2056|5|12|1|4|2056|629|8188|Friday|2056Q4|N|Y|N|2472334|2472668|2471968|2472242|N|N|N|N|N| +2472335|AAAAAAAAPIJLFCAA|2056-12-02|1883|8188|629|2056|6|12|2|4|2056|629|8188|Saturday|2056Q4|N|Y|N|2472334|2472668|2471969|2472243|N|N|N|N|N| +2472336|AAAAAAAAAJJLFCAA|2056-12-03|1883|8188|629|2056|0|12|3|4|2056|629|8188|Sunday|2056Q4|N|N|N|2472334|2472668|2471970|2472244|N|N|N|N|N| +2472337|AAAAAAAABJJLFCAA|2056-12-04|1883|8188|629|2056|1|12|4|4|2056|629|8188|Monday|2056Q4|N|N|N|2472334|2472668|2471971|2472245|N|N|N|N|N| +2472338|AAAAAAAACJJLFCAA|2056-12-05|1883|8189|629|2056|2|12|5|4|2056|629|8189|Tuesday|2056Q4|N|N|N|2472334|2472668|2471972|2472246|N|N|N|N|N| +2472339|AAAAAAAADJJLFCAA|2056-12-06|1883|8189|629|2056|3|12|6|4|2056|629|8189|Wednesday|2056Q4|N|N|N|2472334|2472668|2471973|2472247|N|N|N|N|N| +2472340|AAAAAAAAEJJLFCAA|2056-12-07|1883|8189|629|2056|4|12|7|4|2056|629|8189|Thursday|2056Q4|N|N|N|2472334|2472668|2471974|2472248|N|N|N|N|N| +2472341|AAAAAAAAFJJLFCAA|2056-12-08|1883|8189|629|2056|5|12|8|4|2056|629|8189|Friday|2056Q4|N|Y|N|2472334|2472668|2471975|2472249|N|N|N|N|N| +2472342|AAAAAAAAGJJLFCAA|2056-12-09|1883|8189|629|2056|6|12|9|4|2056|629|8189|Saturday|2056Q4|N|Y|N|2472334|2472668|2471976|2472250|N|N|N|N|N| +2472343|AAAAAAAAHJJLFCAA|2056-12-10|1883|8189|629|2056|0|12|10|4|2056|629|8189|Sunday|2056Q4|N|N|N|2472334|2472668|2471977|2472251|N|N|N|N|N| +2472344|AAAAAAAAIJJLFCAA|2056-12-11|1883|8189|629|2056|1|12|11|4|2056|629|8189|Monday|2056Q4|N|N|N|2472334|2472668|2471978|2472252|N|N|N|N|N| +2472345|AAAAAAAAJJJLFCAA|2056-12-12|1883|8190|629|2056|2|12|12|4|2056|629|8190|Tuesday|2056Q4|N|N|N|2472334|2472668|2471979|2472253|N|N|N|N|N| +2472346|AAAAAAAAKJJLFCAA|2056-12-13|1883|8190|629|2056|3|12|13|4|2056|629|8190|Wednesday|2056Q4|N|N|N|2472334|2472668|2471980|2472254|N|N|N|N|N| +2472347|AAAAAAAALJJLFCAA|2056-12-14|1883|8190|629|2056|4|12|14|4|2056|629|8190|Thursday|2056Q4|N|N|N|2472334|2472668|2471981|2472255|N|N|N|N|N| +2472348|AAAAAAAAMJJLFCAA|2056-12-15|1883|8190|629|2056|5|12|15|4|2056|629|8190|Friday|2056Q4|N|Y|N|2472334|2472668|2471982|2472256|N|N|N|N|N| +2472349|AAAAAAAANJJLFCAA|2056-12-16|1883|8190|629|2056|6|12|16|4|2056|629|8190|Saturday|2056Q4|N|Y|N|2472334|2472668|2471983|2472257|N|N|N|N|N| +2472350|AAAAAAAAOJJLFCAA|2056-12-17|1883|8190|629|2056|0|12|17|4|2056|629|8190|Sunday|2056Q4|N|N|N|2472334|2472668|2471984|2472258|N|N|N|N|N| +2472351|AAAAAAAAPJJLFCAA|2056-12-18|1883|8190|629|2056|1|12|18|4|2056|629|8190|Monday|2056Q4|N|N|N|2472334|2472668|2471985|2472259|N|N|N|N|N| +2472352|AAAAAAAAAKJLFCAA|2056-12-19|1883|8191|629|2056|2|12|19|4|2056|629|8191|Tuesday|2056Q4|N|N|N|2472334|2472668|2471986|2472260|N|N|N|N|N| +2472353|AAAAAAAABKJLFCAA|2056-12-20|1883|8191|629|2056|3|12|20|4|2056|629|8191|Wednesday|2056Q4|N|N|N|2472334|2472668|2471987|2472261|N|N|N|N|N| +2472354|AAAAAAAACKJLFCAA|2056-12-21|1883|8191|629|2056|4|12|21|4|2056|629|8191|Thursday|2056Q4|N|N|N|2472334|2472668|2471988|2472262|N|N|N|N|N| +2472355|AAAAAAAADKJLFCAA|2056-12-22|1883|8191|629|2056|5|12|22|4|2056|629|8191|Friday|2056Q4|N|Y|N|2472334|2472668|2471989|2472263|N|N|N|N|N| +2472356|AAAAAAAAEKJLFCAA|2056-12-23|1883|8191|629|2056|6|12|23|4|2056|629|8191|Saturday|2056Q4|N|Y|N|2472334|2472668|2471990|2472264|N|N|N|N|N| +2472357|AAAAAAAAFKJLFCAA|2056-12-24|1883|8191|629|2056|0|12|24|4|2056|629|8191|Sunday|2056Q4|N|N|N|2472334|2472668|2471991|2472265|N|N|N|N|N| +2472358|AAAAAAAAGKJLFCAA|2056-12-25|1883|8191|629|2056|1|12|25|4|2056|629|8191|Monday|2056Q4|Y|N|N|2472334|2472668|2471992|2472266|N|N|N|N|N| +2472359|AAAAAAAAHKJLFCAA|2056-12-26|1883|8192|629|2056|2|12|26|4|2056|629|8192|Tuesday|2056Q4|N|N|Y|2472334|2472668|2471993|2472267|N|N|N|N|N| +2472360|AAAAAAAAIKJLFCAA|2056-12-27|1883|8192|629|2056|3|12|27|4|2056|629|8192|Wednesday|2056Q4|N|N|N|2472334|2472668|2471994|2472268|N|N|N|N|N| +2472361|AAAAAAAAJKJLFCAA|2056-12-28|1883|8192|629|2056|4|12|28|4|2056|629|8192|Thursday|2056Q4|N|N|N|2472334|2472668|2471995|2472269|N|N|N|N|N| +2472362|AAAAAAAAKKJLFCAA|2056-12-29|1883|8192|629|2056|5|12|29|4|2056|629|8192|Friday|2056Q4|N|Y|N|2472334|2472668|2471996|2472270|N|N|N|N|N| +2472363|AAAAAAAALKJLFCAA|2056-12-30|1883|8192|629|2056|6|12|30|4|2056|629|8192|Saturday|2056Q4|N|Y|N|2472334|2472668|2471997|2472271|N|N|N|N|N| +2472364|AAAAAAAAMKJLFCAA|2056-12-31|1883|8192|629|2056|0|12|31|4|2056|629|8192|Sunday|2056Q4|Y|N|N|2472334|2472668|2471998|2472272|N|N|N|N|N| +2472365|AAAAAAAANKJLFCAA|2057-01-01|1884|8192|629|2057|1|1|1|1|2057|629|8192|Monday|2057Q1|Y|N|Y|2472365|2472364|2471999|2472273|N|N|N|N|N| +2472366|AAAAAAAAOKJLFCAA|2057-01-02|1884|8193|629|2057|2|1|2|1|2057|629|8193|Tuesday|2057Q1|N|N|Y|2472365|2472364|2472000|2472274|N|N|N|N|N| +2472367|AAAAAAAAPKJLFCAA|2057-01-03|1884|8193|629|2057|3|1|3|1|2057|629|8193|Wednesday|2057Q1|N|N|N|2472365|2472364|2472001|2472275|N|N|N|N|N| +2472368|AAAAAAAAALJLFCAA|2057-01-04|1884|8193|629|2057|4|1|4|1|2057|629|8193|Thursday|2057Q1|N|N|N|2472365|2472364|2472002|2472276|N|N|N|N|N| +2472369|AAAAAAAABLJLFCAA|2057-01-05|1884|8193|629|2057|5|1|5|1|2057|629|8193|Friday|2057Q1|N|Y|N|2472365|2472364|2472003|2472277|N|N|N|N|N| +2472370|AAAAAAAACLJLFCAA|2057-01-06|1884|8193|629|2057|6|1|6|1|2057|629|8193|Saturday|2057Q1|N|Y|N|2472365|2472364|2472004|2472278|N|N|N|N|N| +2472371|AAAAAAAADLJLFCAA|2057-01-07|1884|8193|629|2057|0|1|7|1|2057|629|8193|Sunday|2057Q1|N|N|N|2472365|2472364|2472005|2472279|N|N|N|N|N| +2472372|AAAAAAAAELJLFCAA|2057-01-08|1884|8193|629|2057|1|1|8|1|2057|629|8193|Monday|2057Q1|N|N|N|2472365|2472364|2472006|2472280|N|N|N|N|N| +2472373|AAAAAAAAFLJLFCAA|2057-01-09|1884|8194|629|2057|2|1|9|1|2057|629|8194|Tuesday|2057Q1|N|N|N|2472365|2472364|2472007|2472281|N|N|N|N|N| +2472374|AAAAAAAAGLJLFCAA|2057-01-10|1884|8194|629|2057|3|1|10|1|2057|629|8194|Wednesday|2057Q1|N|N|N|2472365|2472364|2472008|2472282|N|N|N|N|N| +2472375|AAAAAAAAHLJLFCAA|2057-01-11|1884|8194|629|2057|4|1|11|1|2057|629|8194|Thursday|2057Q1|N|N|N|2472365|2472364|2472009|2472283|N|N|N|N|N| +2472376|AAAAAAAAILJLFCAA|2057-01-12|1884|8194|629|2057|5|1|12|1|2057|629|8194|Friday|2057Q1|N|Y|N|2472365|2472364|2472010|2472284|N|N|N|N|N| +2472377|AAAAAAAAJLJLFCAA|2057-01-13|1884|8194|629|2057|6|1|13|1|2057|629|8194|Saturday|2057Q1|N|Y|N|2472365|2472364|2472011|2472285|N|N|N|N|N| +2472378|AAAAAAAAKLJLFCAA|2057-01-14|1884|8194|629|2057|0|1|14|1|2057|629|8194|Sunday|2057Q1|N|N|N|2472365|2472364|2472012|2472286|N|N|N|N|N| +2472379|AAAAAAAALLJLFCAA|2057-01-15|1884|8194|629|2057|1|1|15|1|2057|629|8194|Monday|2057Q1|N|N|N|2472365|2472364|2472013|2472287|N|N|N|N|N| +2472380|AAAAAAAAMLJLFCAA|2057-01-16|1884|8195|629|2057|2|1|16|1|2057|629|8195|Tuesday|2057Q1|N|N|N|2472365|2472364|2472014|2472288|N|N|N|N|N| +2472381|AAAAAAAANLJLFCAA|2057-01-17|1884|8195|629|2057|3|1|17|1|2057|629|8195|Wednesday|2057Q1|N|N|N|2472365|2472364|2472015|2472289|N|N|N|N|N| +2472382|AAAAAAAAOLJLFCAA|2057-01-18|1884|8195|629|2057|4|1|18|1|2057|629|8195|Thursday|2057Q1|N|N|N|2472365|2472364|2472016|2472290|N|N|N|N|N| +2472383|AAAAAAAAPLJLFCAA|2057-01-19|1884|8195|629|2057|5|1|19|1|2057|629|8195|Friday|2057Q1|N|Y|N|2472365|2472364|2472017|2472291|N|N|N|N|N| +2472384|AAAAAAAAAMJLFCAA|2057-01-20|1884|8195|629|2057|6|1|20|1|2057|629|8195|Saturday|2057Q1|N|Y|N|2472365|2472364|2472018|2472292|N|N|N|N|N| +2472385|AAAAAAAABMJLFCAA|2057-01-21|1884|8195|629|2057|0|1|21|1|2057|629|8195|Sunday|2057Q1|N|N|N|2472365|2472364|2472019|2472293|N|N|N|N|N| +2472386|AAAAAAAACMJLFCAA|2057-01-22|1884|8195|629|2057|1|1|22|1|2057|629|8195|Monday|2057Q1|N|N|N|2472365|2472364|2472020|2472294|N|N|N|N|N| +2472387|AAAAAAAADMJLFCAA|2057-01-23|1884|8196|629|2057|2|1|23|1|2057|629|8196|Tuesday|2057Q1|N|N|N|2472365|2472364|2472021|2472295|N|N|N|N|N| +2472388|AAAAAAAAEMJLFCAA|2057-01-24|1884|8196|629|2057|3|1|24|1|2057|629|8196|Wednesday|2057Q1|N|N|N|2472365|2472364|2472022|2472296|N|N|N|N|N| +2472389|AAAAAAAAFMJLFCAA|2057-01-25|1884|8196|629|2057|4|1|25|1|2057|629|8196|Thursday|2057Q1|N|N|N|2472365|2472364|2472023|2472297|N|N|N|N|N| +2472390|AAAAAAAAGMJLFCAA|2057-01-26|1884|8196|629|2057|5|1|26|1|2057|629|8196|Friday|2057Q1|N|Y|N|2472365|2472364|2472024|2472298|N|N|N|N|N| +2472391|AAAAAAAAHMJLFCAA|2057-01-27|1884|8196|629|2057|6|1|27|1|2057|629|8196|Saturday|2057Q1|N|Y|N|2472365|2472364|2472025|2472299|N|N|N|N|N| +2472392|AAAAAAAAIMJLFCAA|2057-01-28|1884|8196|629|2057|0|1|28|1|2057|629|8196|Sunday|2057Q1|N|N|N|2472365|2472364|2472026|2472300|N|N|N|N|N| +2472393|AAAAAAAAJMJLFCAA|2057-01-29|1884|8196|629|2057|1|1|29|1|2057|629|8196|Monday|2057Q1|N|N|N|2472365|2472364|2472027|2472301|N|N|N|N|N| +2472394|AAAAAAAAKMJLFCAA|2057-01-30|1884|8197|629|2057|2|1|30|1|2057|629|8197|Tuesday|2057Q1|N|N|N|2472365|2472364|2472028|2472302|N|N|N|N|N| +2472395|AAAAAAAALMJLFCAA|2057-01-31|1884|8197|629|2057|3|1|31|1|2057|629|8197|Wednesday|2057Q1|N|N|N|2472365|2472364|2472029|2472303|N|N|N|N|N| +2472396|AAAAAAAAMMJLFCAA|2057-02-01|1885|8197|629|2057|4|2|1|1|2057|629|8197|Thursday|2057Q1|N|N|N|2472396|2472426|2472030|2472304|N|N|N|N|N| +2472397|AAAAAAAANMJLFCAA|2057-02-02|1885|8197|629|2057|5|2|2|1|2057|629|8197|Friday|2057Q1|N|Y|N|2472396|2472426|2472031|2472305|N|N|N|N|N| +2472398|AAAAAAAAOMJLFCAA|2057-02-03|1885|8197|629|2057|6|2|3|1|2057|629|8197|Saturday|2057Q1|N|Y|N|2472396|2472426|2472032|2472306|N|N|N|N|N| +2472399|AAAAAAAAPMJLFCAA|2057-02-04|1885|8197|629|2057|0|2|4|1|2057|629|8197|Sunday|2057Q1|N|N|N|2472396|2472426|2472033|2472307|N|N|N|N|N| +2472400|AAAAAAAAANJLFCAA|2057-02-05|1885|8197|629|2057|1|2|5|1|2057|629|8197|Monday|2057Q1|N|N|N|2472396|2472426|2472034|2472308|N|N|N|N|N| +2472401|AAAAAAAABNJLFCAA|2057-02-06|1885|8198|629|2057|2|2|6|1|2057|629|8198|Tuesday|2057Q1|N|N|N|2472396|2472426|2472035|2472309|N|N|N|N|N| +2472402|AAAAAAAACNJLFCAA|2057-02-07|1885|8198|629|2057|3|2|7|1|2057|629|8198|Wednesday|2057Q1|N|N|N|2472396|2472426|2472036|2472310|N|N|N|N|N| +2472403|AAAAAAAADNJLFCAA|2057-02-08|1885|8198|629|2057|4|2|8|1|2057|629|8198|Thursday|2057Q1|N|N|N|2472396|2472426|2472037|2472311|N|N|N|N|N| +2472404|AAAAAAAAENJLFCAA|2057-02-09|1885|8198|629|2057|5|2|9|1|2057|629|8198|Friday|2057Q1|N|Y|N|2472396|2472426|2472038|2472312|N|N|N|N|N| +2472405|AAAAAAAAFNJLFCAA|2057-02-10|1885|8198|629|2057|6|2|10|1|2057|629|8198|Saturday|2057Q1|N|Y|N|2472396|2472426|2472039|2472313|N|N|N|N|N| +2472406|AAAAAAAAGNJLFCAA|2057-02-11|1885|8198|629|2057|0|2|11|1|2057|629|8198|Sunday|2057Q1|N|N|N|2472396|2472426|2472040|2472314|N|N|N|N|N| +2472407|AAAAAAAAHNJLFCAA|2057-02-12|1885|8198|629|2057|1|2|12|1|2057|629|8198|Monday|2057Q1|N|N|N|2472396|2472426|2472041|2472315|N|N|N|N|N| +2472408|AAAAAAAAINJLFCAA|2057-02-13|1885|8199|629|2057|2|2|13|1|2057|629|8199|Tuesday|2057Q1|N|N|N|2472396|2472426|2472042|2472316|N|N|N|N|N| +2472409|AAAAAAAAJNJLFCAA|2057-02-14|1885|8199|629|2057|3|2|14|1|2057|629|8199|Wednesday|2057Q1|N|N|N|2472396|2472426|2472043|2472317|N|N|N|N|N| +2472410|AAAAAAAAKNJLFCAA|2057-02-15|1885|8199|629|2057|4|2|15|1|2057|629|8199|Thursday|2057Q1|N|N|N|2472396|2472426|2472044|2472318|N|N|N|N|N| +2472411|AAAAAAAALNJLFCAA|2057-02-16|1885|8199|629|2057|5|2|16|1|2057|629|8199|Friday|2057Q1|N|Y|N|2472396|2472426|2472045|2472319|N|N|N|N|N| +2472412|AAAAAAAAMNJLFCAA|2057-02-17|1885|8199|629|2057|6|2|17|1|2057|629|8199|Saturday|2057Q1|N|Y|N|2472396|2472426|2472046|2472320|N|N|N|N|N| +2472413|AAAAAAAANNJLFCAA|2057-02-18|1885|8199|629|2057|0|2|18|1|2057|629|8199|Sunday|2057Q1|N|N|N|2472396|2472426|2472047|2472321|N|N|N|N|N| +2472414|AAAAAAAAONJLFCAA|2057-02-19|1885|8199|629|2057|1|2|19|1|2057|629|8199|Monday|2057Q1|N|N|N|2472396|2472426|2472048|2472322|N|N|N|N|N| +2472415|AAAAAAAAPNJLFCAA|2057-02-20|1885|8200|629|2057|2|2|20|1|2057|629|8200|Tuesday|2057Q1|N|N|N|2472396|2472426|2472049|2472323|N|N|N|N|N| +2472416|AAAAAAAAAOJLFCAA|2057-02-21|1885|8200|629|2057|3|2|21|1|2057|629|8200|Wednesday|2057Q1|N|N|N|2472396|2472426|2472050|2472324|N|N|N|N|N| +2472417|AAAAAAAABOJLFCAA|2057-02-22|1885|8200|629|2057|4|2|22|1|2057|629|8200|Thursday|2057Q1|N|N|N|2472396|2472426|2472051|2472325|N|N|N|N|N| +2472418|AAAAAAAACOJLFCAA|2057-02-23|1885|8200|629|2057|5|2|23|1|2057|629|8200|Friday|2057Q1|N|Y|N|2472396|2472426|2472052|2472326|N|N|N|N|N| +2472419|AAAAAAAADOJLFCAA|2057-02-24|1885|8200|629|2057|6|2|24|1|2057|629|8200|Saturday|2057Q1|N|Y|N|2472396|2472426|2472053|2472327|N|N|N|N|N| +2472420|AAAAAAAAEOJLFCAA|2057-02-25|1885|8200|629|2057|0|2|25|1|2057|629|8200|Sunday|2057Q1|N|N|N|2472396|2472426|2472054|2472328|N|N|N|N|N| +2472421|AAAAAAAAFOJLFCAA|2057-02-26|1885|8200|629|2057|1|2|26|1|2057|629|8200|Monday|2057Q1|N|N|N|2472396|2472426|2472055|2472329|N|N|N|N|N| +2472422|AAAAAAAAGOJLFCAA|2057-02-27|1885|8201|629|2057|2|2|27|1|2057|629|8201|Tuesday|2057Q1|N|N|N|2472396|2472426|2472056|2472330|N|N|N|N|N| +2472423|AAAAAAAAHOJLFCAA|2057-02-28|1885|8201|629|2057|3|2|28|1|2057|629|8201|Wednesday|2057Q1|N|N|N|2472396|2472426|2472057|2472331|N|N|N|N|N| +2472424|AAAAAAAAIOJLFCAA|2057-03-01|1886|8201|630|2057|4|3|1|1|2057|630|8201|Thursday|2057Q1|N|N|N|2472424|2472482|2472059|2472332|N|N|N|N|N| +2472425|AAAAAAAAJOJLFCAA|2057-03-02|1886|8201|630|2057|5|3|2|1|2057|630|8201|Friday|2057Q1|N|Y|N|2472424|2472482|2472060|2472333|N|N|N|N|N| +2472426|AAAAAAAAKOJLFCAA|2057-03-03|1886|8201|630|2057|6|3|3|1|2057|630|8201|Saturday|2057Q1|N|Y|N|2472424|2472482|2472061|2472334|N|N|N|N|N| +2472427|AAAAAAAALOJLFCAA|2057-03-04|1886|8201|630|2057|0|3|4|1|2057|630|8201|Sunday|2057Q1|N|N|N|2472424|2472482|2472062|2472335|N|N|N|N|N| +2472428|AAAAAAAAMOJLFCAA|2057-03-05|1886|8201|630|2057|1|3|5|1|2057|630|8201|Monday|2057Q1|N|N|N|2472424|2472482|2472063|2472336|N|N|N|N|N| +2472429|AAAAAAAANOJLFCAA|2057-03-06|1886|8202|630|2057|2|3|6|1|2057|630|8202|Tuesday|2057Q1|N|N|N|2472424|2472482|2472064|2472337|N|N|N|N|N| +2472430|AAAAAAAAOOJLFCAA|2057-03-07|1886|8202|630|2057|3|3|7|1|2057|630|8202|Wednesday|2057Q1|N|N|N|2472424|2472482|2472065|2472338|N|N|N|N|N| +2472431|AAAAAAAAPOJLFCAA|2057-03-08|1886|8202|630|2057|4|3|8|1|2057|630|8202|Thursday|2057Q1|N|N|N|2472424|2472482|2472066|2472339|N|N|N|N|N| +2472432|AAAAAAAAAPJLFCAA|2057-03-09|1886|8202|630|2057|5|3|9|1|2057|630|8202|Friday|2057Q1|N|Y|N|2472424|2472482|2472067|2472340|N|N|N|N|N| +2472433|AAAAAAAABPJLFCAA|2057-03-10|1886|8202|630|2057|6|3|10|1|2057|630|8202|Saturday|2057Q1|N|Y|N|2472424|2472482|2472068|2472341|N|N|N|N|N| +2472434|AAAAAAAACPJLFCAA|2057-03-11|1886|8202|630|2057|0|3|11|1|2057|630|8202|Sunday|2057Q1|N|N|N|2472424|2472482|2472069|2472342|N|N|N|N|N| +2472435|AAAAAAAADPJLFCAA|2057-03-12|1886|8202|630|2057|1|3|12|1|2057|630|8202|Monday|2057Q1|N|N|N|2472424|2472482|2472070|2472343|N|N|N|N|N| +2472436|AAAAAAAAEPJLFCAA|2057-03-13|1886|8203|630|2057|2|3|13|1|2057|630|8203|Tuesday|2057Q1|N|N|N|2472424|2472482|2472071|2472344|N|N|N|N|N| +2472437|AAAAAAAAFPJLFCAA|2057-03-14|1886|8203|630|2057|3|3|14|1|2057|630|8203|Wednesday|2057Q1|N|N|N|2472424|2472482|2472072|2472345|N|N|N|N|N| +2472438|AAAAAAAAGPJLFCAA|2057-03-15|1886|8203|630|2057|4|3|15|1|2057|630|8203|Thursday|2057Q1|N|N|N|2472424|2472482|2472073|2472346|N|N|N|N|N| +2472439|AAAAAAAAHPJLFCAA|2057-03-16|1886|8203|630|2057|5|3|16|1|2057|630|8203|Friday|2057Q1|N|Y|N|2472424|2472482|2472074|2472347|N|N|N|N|N| +2472440|AAAAAAAAIPJLFCAA|2057-03-17|1886|8203|630|2057|6|3|17|1|2057|630|8203|Saturday|2057Q1|N|Y|N|2472424|2472482|2472075|2472348|N|N|N|N|N| +2472441|AAAAAAAAJPJLFCAA|2057-03-18|1886|8203|630|2057|0|3|18|1|2057|630|8203|Sunday|2057Q1|N|N|N|2472424|2472482|2472076|2472349|N|N|N|N|N| +2472442|AAAAAAAAKPJLFCAA|2057-03-19|1886|8203|630|2057|1|3|19|1|2057|630|8203|Monday|2057Q1|N|N|N|2472424|2472482|2472077|2472350|N|N|N|N|N| +2472443|AAAAAAAALPJLFCAA|2057-03-20|1886|8204|630|2057|2|3|20|1|2057|630|8204|Tuesday|2057Q1|N|N|N|2472424|2472482|2472078|2472351|N|N|N|N|N| +2472444|AAAAAAAAMPJLFCAA|2057-03-21|1886|8204|630|2057|3|3|21|1|2057|630|8204|Wednesday|2057Q1|N|N|N|2472424|2472482|2472079|2472352|N|N|N|N|N| +2472445|AAAAAAAANPJLFCAA|2057-03-22|1886|8204|630|2057|4|3|22|1|2057|630|8204|Thursday|2057Q1|N|N|N|2472424|2472482|2472080|2472353|N|N|N|N|N| +2472446|AAAAAAAAOPJLFCAA|2057-03-23|1886|8204|630|2057|5|3|23|1|2057|630|8204|Friday|2057Q1|N|Y|N|2472424|2472482|2472081|2472354|N|N|N|N|N| +2472447|AAAAAAAAPPJLFCAA|2057-03-24|1886|8204|630|2057|6|3|24|1|2057|630|8204|Saturday|2057Q1|N|Y|N|2472424|2472482|2472082|2472355|N|N|N|N|N| +2472448|AAAAAAAAAAKLFCAA|2057-03-25|1886|8204|630|2057|0|3|25|1|2057|630|8204|Sunday|2057Q1|N|N|N|2472424|2472482|2472083|2472356|N|N|N|N|N| +2472449|AAAAAAAABAKLFCAA|2057-03-26|1886|8204|630|2057|1|3|26|1|2057|630|8204|Monday|2057Q1|N|N|N|2472424|2472482|2472084|2472357|N|N|N|N|N| +2472450|AAAAAAAACAKLFCAA|2057-03-27|1886|8205|630|2057|2|3|27|1|2057|630|8205|Tuesday|2057Q1|N|N|N|2472424|2472482|2472085|2472358|N|N|N|N|N| +2472451|AAAAAAAADAKLFCAA|2057-03-28|1886|8205|630|2057|3|3|28|1|2057|630|8205|Wednesday|2057Q1|N|N|N|2472424|2472482|2472086|2472359|N|N|N|N|N| +2472452|AAAAAAAAEAKLFCAA|2057-03-29|1886|8205|630|2057|4|3|29|1|2057|630|8205|Thursday|2057Q1|N|N|N|2472424|2472482|2472087|2472360|N|N|N|N|N| +2472453|AAAAAAAAFAKLFCAA|2057-03-30|1886|8205|630|2057|5|3|30|1|2057|630|8205|Friday|2057Q1|N|Y|N|2472424|2472482|2472088|2472361|N|N|N|N|N| +2472454|AAAAAAAAGAKLFCAA|2057-03-31|1886|8205|630|2057|6|3|31|1|2057|630|8205|Saturday|2057Q1|N|Y|N|2472424|2472482|2472089|2472362|N|N|N|N|N| +2472455|AAAAAAAAHAKLFCAA|2057-04-01|1887|8205|630|2057|0|4|1|1|2057|630|8205|Sunday|2057Q1|N|N|N|2472455|2472544|2472090|2472365|N|N|N|N|N| +2472456|AAAAAAAAIAKLFCAA|2057-04-02|1887|8205|630|2057|1|4|2|2|2057|630|8205|Monday|2057Q2|N|N|N|2472455|2472544|2472091|2472366|N|N|N|N|N| +2472457|AAAAAAAAJAKLFCAA|2057-04-03|1887|8206|630|2057|2|4|3|2|2057|630|8206|Tuesday|2057Q2|N|N|N|2472455|2472544|2472092|2472367|N|N|N|N|N| +2472458|AAAAAAAAKAKLFCAA|2057-04-04|1887|8206|630|2057|3|4|4|2|2057|630|8206|Wednesday|2057Q2|N|N|N|2472455|2472544|2472093|2472368|N|N|N|N|N| +2472459|AAAAAAAALAKLFCAA|2057-04-05|1887|8206|630|2057|4|4|5|2|2057|630|8206|Thursday|2057Q2|N|N|N|2472455|2472544|2472094|2472369|N|N|N|N|N| +2472460|AAAAAAAAMAKLFCAA|2057-04-06|1887|8206|630|2057|5|4|6|2|2057|630|8206|Friday|2057Q2|N|Y|N|2472455|2472544|2472095|2472370|N|N|N|N|N| +2472461|AAAAAAAANAKLFCAA|2057-04-07|1887|8206|630|2057|6|4|7|2|2057|630|8206|Saturday|2057Q2|N|Y|N|2472455|2472544|2472096|2472371|N|N|N|N|N| +2472462|AAAAAAAAOAKLFCAA|2057-04-08|1887|8206|630|2057|0|4|8|2|2057|630|8206|Sunday|2057Q2|N|N|N|2472455|2472544|2472097|2472372|N|N|N|N|N| +2472463|AAAAAAAAPAKLFCAA|2057-04-09|1887|8206|630|2057|1|4|9|2|2057|630|8206|Monday|2057Q2|N|N|N|2472455|2472544|2472098|2472373|N|N|N|N|N| +2472464|AAAAAAAAABKLFCAA|2057-04-10|1887|8207|630|2057|2|4|10|2|2057|630|8207|Tuesday|2057Q2|N|N|N|2472455|2472544|2472099|2472374|N|N|N|N|N| +2472465|AAAAAAAABBKLFCAA|2057-04-11|1887|8207|630|2057|3|4|11|2|2057|630|8207|Wednesday|2057Q2|N|N|N|2472455|2472544|2472100|2472375|N|N|N|N|N| +2472466|AAAAAAAACBKLFCAA|2057-04-12|1887|8207|630|2057|4|4|12|2|2057|630|8207|Thursday|2057Q2|N|N|N|2472455|2472544|2472101|2472376|N|N|N|N|N| +2472467|AAAAAAAADBKLFCAA|2057-04-13|1887|8207|630|2057|5|4|13|2|2057|630|8207|Friday|2057Q2|N|Y|N|2472455|2472544|2472102|2472377|N|N|N|N|N| +2472468|AAAAAAAAEBKLFCAA|2057-04-14|1887|8207|630|2057|6|4|14|2|2057|630|8207|Saturday|2057Q2|N|Y|N|2472455|2472544|2472103|2472378|N|N|N|N|N| +2472469|AAAAAAAAFBKLFCAA|2057-04-15|1887|8207|630|2057|0|4|15|2|2057|630|8207|Sunday|2057Q2|N|N|N|2472455|2472544|2472104|2472379|N|N|N|N|N| +2472470|AAAAAAAAGBKLFCAA|2057-04-16|1887|8207|630|2057|1|4|16|2|2057|630|8207|Monday|2057Q2|N|N|N|2472455|2472544|2472105|2472380|N|N|N|N|N| +2472471|AAAAAAAAHBKLFCAA|2057-04-17|1887|8208|630|2057|2|4|17|2|2057|630|8208|Tuesday|2057Q2|N|N|N|2472455|2472544|2472106|2472381|N|N|N|N|N| +2472472|AAAAAAAAIBKLFCAA|2057-04-18|1887|8208|630|2057|3|4|18|2|2057|630|8208|Wednesday|2057Q2|N|N|N|2472455|2472544|2472107|2472382|N|N|N|N|N| +2472473|AAAAAAAAJBKLFCAA|2057-04-19|1887|8208|630|2057|4|4|19|2|2057|630|8208|Thursday|2057Q2|N|N|N|2472455|2472544|2472108|2472383|N|N|N|N|N| +2472474|AAAAAAAAKBKLFCAA|2057-04-20|1887|8208|630|2057|5|4|20|2|2057|630|8208|Friday|2057Q2|N|Y|N|2472455|2472544|2472109|2472384|N|N|N|N|N| +2472475|AAAAAAAALBKLFCAA|2057-04-21|1887|8208|630|2057|6|4|21|2|2057|630|8208|Saturday|2057Q2|N|Y|N|2472455|2472544|2472110|2472385|N|N|N|N|N| +2472476|AAAAAAAAMBKLFCAA|2057-04-22|1887|8208|630|2057|0|4|22|2|2057|630|8208|Sunday|2057Q2|N|N|N|2472455|2472544|2472111|2472386|N|N|N|N|N| +2472477|AAAAAAAANBKLFCAA|2057-04-23|1887|8208|630|2057|1|4|23|2|2057|630|8208|Monday|2057Q2|N|N|N|2472455|2472544|2472112|2472387|N|N|N|N|N| +2472478|AAAAAAAAOBKLFCAA|2057-04-24|1887|8209|630|2057|2|4|24|2|2057|630|8209|Tuesday|2057Q2|N|N|N|2472455|2472544|2472113|2472388|N|N|N|N|N| +2472479|AAAAAAAAPBKLFCAA|2057-04-25|1887|8209|630|2057|3|4|25|2|2057|630|8209|Wednesday|2057Q2|N|N|N|2472455|2472544|2472114|2472389|N|N|N|N|N| +2472480|AAAAAAAAACKLFCAA|2057-04-26|1887|8209|630|2057|4|4|26|2|2057|630|8209|Thursday|2057Q2|N|N|N|2472455|2472544|2472115|2472390|N|N|N|N|N| +2472481|AAAAAAAABCKLFCAA|2057-04-27|1887|8209|630|2057|5|4|27|2|2057|630|8209|Friday|2057Q2|N|Y|N|2472455|2472544|2472116|2472391|N|N|N|N|N| +2472482|AAAAAAAACCKLFCAA|2057-04-28|1887|8209|630|2057|6|4|28|2|2057|630|8209|Saturday|2057Q2|N|Y|N|2472455|2472544|2472117|2472392|N|N|N|N|N| +2472483|AAAAAAAADCKLFCAA|2057-04-29|1887|8209|630|2057|0|4|29|2|2057|630|8209|Sunday|2057Q2|N|N|N|2472455|2472544|2472118|2472393|N|N|N|N|N| +2472484|AAAAAAAAECKLFCAA|2057-04-30|1887|8209|630|2057|1|4|30|2|2057|630|8209|Monday|2057Q2|N|N|N|2472455|2472544|2472119|2472394|N|N|N|N|N| +2472485|AAAAAAAAFCKLFCAA|2057-05-01|1888|8210|630|2057|2|5|1|2|2057|630|8210|Tuesday|2057Q2|N|N|N|2472485|2472604|2472120|2472395|N|N|N|N|N| +2472486|AAAAAAAAGCKLFCAA|2057-05-02|1888|8210|630|2057|3|5|2|2|2057|630|8210|Wednesday|2057Q2|N|N|N|2472485|2472604|2472121|2472396|N|N|N|N|N| +2472487|AAAAAAAAHCKLFCAA|2057-05-03|1888|8210|630|2057|4|5|3|2|2057|630|8210|Thursday|2057Q2|N|N|N|2472485|2472604|2472122|2472397|N|N|N|N|N| +2472488|AAAAAAAAICKLFCAA|2057-05-04|1888|8210|630|2057|5|5|4|2|2057|630|8210|Friday|2057Q2|N|Y|N|2472485|2472604|2472123|2472398|N|N|N|N|N| +2472489|AAAAAAAAJCKLFCAA|2057-05-05|1888|8210|630|2057|6|5|5|2|2057|630|8210|Saturday|2057Q2|N|Y|N|2472485|2472604|2472124|2472399|N|N|N|N|N| +2472490|AAAAAAAAKCKLFCAA|2057-05-06|1888|8210|630|2057|0|5|6|2|2057|630|8210|Sunday|2057Q2|N|N|N|2472485|2472604|2472125|2472400|N|N|N|N|N| +2472491|AAAAAAAALCKLFCAA|2057-05-07|1888|8210|630|2057|1|5|7|2|2057|630|8210|Monday|2057Q2|N|N|N|2472485|2472604|2472126|2472401|N|N|N|N|N| +2472492|AAAAAAAAMCKLFCAA|2057-05-08|1888|8211|630|2057|2|5|8|2|2057|630|8211|Tuesday|2057Q2|N|N|N|2472485|2472604|2472127|2472402|N|N|N|N|N| +2472493|AAAAAAAANCKLFCAA|2057-05-09|1888|8211|630|2057|3|5|9|2|2057|630|8211|Wednesday|2057Q2|N|N|N|2472485|2472604|2472128|2472403|N|N|N|N|N| +2472494|AAAAAAAAOCKLFCAA|2057-05-10|1888|8211|630|2057|4|5|10|2|2057|630|8211|Thursday|2057Q2|N|N|N|2472485|2472604|2472129|2472404|N|N|N|N|N| +2472495|AAAAAAAAPCKLFCAA|2057-05-11|1888|8211|630|2057|5|5|11|2|2057|630|8211|Friday|2057Q2|N|Y|N|2472485|2472604|2472130|2472405|N|N|N|N|N| +2472496|AAAAAAAAADKLFCAA|2057-05-12|1888|8211|630|2057|6|5|12|2|2057|630|8211|Saturday|2057Q2|N|Y|N|2472485|2472604|2472131|2472406|N|N|N|N|N| +2472497|AAAAAAAABDKLFCAA|2057-05-13|1888|8211|630|2057|0|5|13|2|2057|630|8211|Sunday|2057Q2|N|N|N|2472485|2472604|2472132|2472407|N|N|N|N|N| +2472498|AAAAAAAACDKLFCAA|2057-05-14|1888|8211|630|2057|1|5|14|2|2057|630|8211|Monday|2057Q2|N|N|N|2472485|2472604|2472133|2472408|N|N|N|N|N| +2472499|AAAAAAAADDKLFCAA|2057-05-15|1888|8212|630|2057|2|5|15|2|2057|630|8212|Tuesday|2057Q2|N|N|N|2472485|2472604|2472134|2472409|N|N|N|N|N| +2472500|AAAAAAAAEDKLFCAA|2057-05-16|1888|8212|630|2057|3|5|16|2|2057|630|8212|Wednesday|2057Q2|N|N|N|2472485|2472604|2472135|2472410|N|N|N|N|N| +2472501|AAAAAAAAFDKLFCAA|2057-05-17|1888|8212|630|2057|4|5|17|2|2057|630|8212|Thursday|2057Q2|N|N|N|2472485|2472604|2472136|2472411|N|N|N|N|N| +2472502|AAAAAAAAGDKLFCAA|2057-05-18|1888|8212|630|2057|5|5|18|2|2057|630|8212|Friday|2057Q2|N|Y|N|2472485|2472604|2472137|2472412|N|N|N|N|N| +2472503|AAAAAAAAHDKLFCAA|2057-05-19|1888|8212|630|2057|6|5|19|2|2057|630|8212|Saturday|2057Q2|N|Y|N|2472485|2472604|2472138|2472413|N|N|N|N|N| +2472504|AAAAAAAAIDKLFCAA|2057-05-20|1888|8212|630|2057|0|5|20|2|2057|630|8212|Sunday|2057Q2|N|N|N|2472485|2472604|2472139|2472414|N|N|N|N|N| +2472505|AAAAAAAAJDKLFCAA|2057-05-21|1888|8212|630|2057|1|5|21|2|2057|630|8212|Monday|2057Q2|N|N|N|2472485|2472604|2472140|2472415|N|N|N|N|N| +2472506|AAAAAAAAKDKLFCAA|2057-05-22|1888|8213|630|2057|2|5|22|2|2057|630|8213|Tuesday|2057Q2|N|N|N|2472485|2472604|2472141|2472416|N|N|N|N|N| +2472507|AAAAAAAALDKLFCAA|2057-05-23|1888|8213|630|2057|3|5|23|2|2057|630|8213|Wednesday|2057Q2|N|N|N|2472485|2472604|2472142|2472417|N|N|N|N|N| +2472508|AAAAAAAAMDKLFCAA|2057-05-24|1888|8213|630|2057|4|5|24|2|2057|630|8213|Thursday|2057Q2|N|N|N|2472485|2472604|2472143|2472418|N|N|N|N|N| +2472509|AAAAAAAANDKLFCAA|2057-05-25|1888|8213|630|2057|5|5|25|2|2057|630|8213|Friday|2057Q2|N|Y|N|2472485|2472604|2472144|2472419|N|N|N|N|N| +2472510|AAAAAAAAODKLFCAA|2057-05-26|1888|8213|630|2057|6|5|26|2|2057|630|8213|Saturday|2057Q2|N|Y|N|2472485|2472604|2472145|2472420|N|N|N|N|N| +2472511|AAAAAAAAPDKLFCAA|2057-05-27|1888|8213|630|2057|0|5|27|2|2057|630|8213|Sunday|2057Q2|N|N|N|2472485|2472604|2472146|2472421|N|N|N|N|N| +2472512|AAAAAAAAAEKLFCAA|2057-05-28|1888|8213|630|2057|1|5|28|2|2057|630|8213|Monday|2057Q2|N|N|N|2472485|2472604|2472147|2472422|N|N|N|N|N| +2472513|AAAAAAAABEKLFCAA|2057-05-29|1888|8214|630|2057|2|5|29|2|2057|630|8214|Tuesday|2057Q2|N|N|N|2472485|2472604|2472148|2472423|N|N|N|N|N| +2472514|AAAAAAAACEKLFCAA|2057-05-30|1888|8214|630|2057|3|5|30|2|2057|630|8214|Wednesday|2057Q2|N|N|N|2472485|2472604|2472149|2472424|N|N|N|N|N| +2472515|AAAAAAAADEKLFCAA|2057-05-31|1888|8214|630|2057|4|5|31|2|2057|630|8214|Thursday|2057Q2|N|N|N|2472485|2472604|2472150|2472425|N|N|N|N|N| +2472516|AAAAAAAAEEKLFCAA|2057-06-01|1889|8214|631|2057|5|6|1|2|2057|631|8214|Friday|2057Q2|N|Y|N|2472516|2472666|2472151|2472426|N|N|N|N|N| +2472517|AAAAAAAAFEKLFCAA|2057-06-02|1889|8214|631|2057|6|6|2|2|2057|631|8214|Saturday|2057Q2|N|Y|N|2472516|2472666|2472152|2472427|N|N|N|N|N| +2472518|AAAAAAAAGEKLFCAA|2057-06-03|1889|8214|631|2057|0|6|3|2|2057|631|8214|Sunday|2057Q2|N|N|N|2472516|2472666|2472153|2472428|N|N|N|N|N| +2472519|AAAAAAAAHEKLFCAA|2057-06-04|1889|8214|631|2057|1|6|4|2|2057|631|8214|Monday|2057Q2|N|N|N|2472516|2472666|2472154|2472429|N|N|N|N|N| +2472520|AAAAAAAAIEKLFCAA|2057-06-05|1889|8215|631|2057|2|6|5|2|2057|631|8215|Tuesday|2057Q2|N|N|N|2472516|2472666|2472155|2472430|N|N|N|N|N| +2472521|AAAAAAAAJEKLFCAA|2057-06-06|1889|8215|631|2057|3|6|6|2|2057|631|8215|Wednesday|2057Q2|N|N|N|2472516|2472666|2472156|2472431|N|N|N|N|N| +2472522|AAAAAAAAKEKLFCAA|2057-06-07|1889|8215|631|2057|4|6|7|2|2057|631|8215|Thursday|2057Q2|N|N|N|2472516|2472666|2472157|2472432|N|N|N|N|N| +2472523|AAAAAAAALEKLFCAA|2057-06-08|1889|8215|631|2057|5|6|8|2|2057|631|8215|Friday|2057Q2|N|Y|N|2472516|2472666|2472158|2472433|N|N|N|N|N| +2472524|AAAAAAAAMEKLFCAA|2057-06-09|1889|8215|631|2057|6|6|9|2|2057|631|8215|Saturday|2057Q2|N|Y|N|2472516|2472666|2472159|2472434|N|N|N|N|N| +2472525|AAAAAAAANEKLFCAA|2057-06-10|1889|8215|631|2057|0|6|10|2|2057|631|8215|Sunday|2057Q2|N|N|N|2472516|2472666|2472160|2472435|N|N|N|N|N| +2472526|AAAAAAAAOEKLFCAA|2057-06-11|1889|8215|631|2057|1|6|11|2|2057|631|8215|Monday|2057Q2|N|N|N|2472516|2472666|2472161|2472436|N|N|N|N|N| +2472527|AAAAAAAAPEKLFCAA|2057-06-12|1889|8216|631|2057|2|6|12|2|2057|631|8216|Tuesday|2057Q2|N|N|N|2472516|2472666|2472162|2472437|N|N|N|N|N| +2472528|AAAAAAAAAFKLFCAA|2057-06-13|1889|8216|631|2057|3|6|13|2|2057|631|8216|Wednesday|2057Q2|N|N|N|2472516|2472666|2472163|2472438|N|N|N|N|N| +2472529|AAAAAAAABFKLFCAA|2057-06-14|1889|8216|631|2057|4|6|14|2|2057|631|8216|Thursday|2057Q2|N|N|N|2472516|2472666|2472164|2472439|N|N|N|N|N| +2472530|AAAAAAAACFKLFCAA|2057-06-15|1889|8216|631|2057|5|6|15|2|2057|631|8216|Friday|2057Q2|N|Y|N|2472516|2472666|2472165|2472440|N|N|N|N|N| +2472531|AAAAAAAADFKLFCAA|2057-06-16|1889|8216|631|2057|6|6|16|2|2057|631|8216|Saturday|2057Q2|N|Y|N|2472516|2472666|2472166|2472441|N|N|N|N|N| +2472532|AAAAAAAAEFKLFCAA|2057-06-17|1889|8216|631|2057|0|6|17|2|2057|631|8216|Sunday|2057Q2|N|N|N|2472516|2472666|2472167|2472442|N|N|N|N|N| +2472533|AAAAAAAAFFKLFCAA|2057-06-18|1889|8216|631|2057|1|6|18|2|2057|631|8216|Monday|2057Q2|N|N|N|2472516|2472666|2472168|2472443|N|N|N|N|N| +2472534|AAAAAAAAGFKLFCAA|2057-06-19|1889|8217|631|2057|2|6|19|2|2057|631|8217|Tuesday|2057Q2|N|N|N|2472516|2472666|2472169|2472444|N|N|N|N|N| +2472535|AAAAAAAAHFKLFCAA|2057-06-20|1889|8217|631|2057|3|6|20|2|2057|631|8217|Wednesday|2057Q2|N|N|N|2472516|2472666|2472170|2472445|N|N|N|N|N| +2472536|AAAAAAAAIFKLFCAA|2057-06-21|1889|8217|631|2057|4|6|21|2|2057|631|8217|Thursday|2057Q2|N|N|N|2472516|2472666|2472171|2472446|N|N|N|N|N| +2472537|AAAAAAAAJFKLFCAA|2057-06-22|1889|8217|631|2057|5|6|22|2|2057|631|8217|Friday|2057Q2|N|Y|N|2472516|2472666|2472172|2472447|N|N|N|N|N| +2472538|AAAAAAAAKFKLFCAA|2057-06-23|1889|8217|631|2057|6|6|23|2|2057|631|8217|Saturday|2057Q2|N|Y|N|2472516|2472666|2472173|2472448|N|N|N|N|N| +2472539|AAAAAAAALFKLFCAA|2057-06-24|1889|8217|631|2057|0|6|24|2|2057|631|8217|Sunday|2057Q2|N|N|N|2472516|2472666|2472174|2472449|N|N|N|N|N| +2472540|AAAAAAAAMFKLFCAA|2057-06-25|1889|8217|631|2057|1|6|25|2|2057|631|8217|Monday|2057Q2|N|N|N|2472516|2472666|2472175|2472450|N|N|N|N|N| +2472541|AAAAAAAANFKLFCAA|2057-06-26|1889|8218|631|2057|2|6|26|2|2057|631|8218|Tuesday|2057Q2|N|N|N|2472516|2472666|2472176|2472451|N|N|N|N|N| +2472542|AAAAAAAAOFKLFCAA|2057-06-27|1889|8218|631|2057|3|6|27|2|2057|631|8218|Wednesday|2057Q2|N|N|N|2472516|2472666|2472177|2472452|N|N|N|N|N| +2472543|AAAAAAAAPFKLFCAA|2057-06-28|1889|8218|631|2057|4|6|28|2|2057|631|8218|Thursday|2057Q2|N|N|N|2472516|2472666|2472178|2472453|N|N|N|N|N| +2472544|AAAAAAAAAGKLFCAA|2057-06-29|1889|8218|631|2057|5|6|29|2|2057|631|8218|Friday|2057Q2|N|Y|N|2472516|2472666|2472179|2472454|N|N|N|N|N| +2472545|AAAAAAAABGKLFCAA|2057-06-30|1889|8218|631|2057|6|6|30|2|2057|631|8218|Saturday|2057Q2|N|Y|N|2472516|2472666|2472180|2472455|N|N|N|N|N| +2472546|AAAAAAAACGKLFCAA|2057-07-01|1890|8218|631|2057|0|7|1|2|2057|631|8218|Sunday|2057Q2|N|N|N|2472546|2472726|2472181|2472455|N|N|N|N|N| +2472547|AAAAAAAADGKLFCAA|2057-07-02|1890|8218|631|2057|1|7|2|3|2057|631|8218|Monday|2057Q3|N|N|N|2472546|2472726|2472182|2472456|N|N|N|N|N| +2472548|AAAAAAAAEGKLFCAA|2057-07-03|1890|8219|631|2057|2|7|3|3|2057|631|8219|Tuesday|2057Q3|N|N|N|2472546|2472726|2472183|2472457|N|N|N|N|N| +2472549|AAAAAAAAFGKLFCAA|2057-07-04|1890|8219|631|2057|3|7|4|3|2057|631|8219|Wednesday|2057Q3|N|N|N|2472546|2472726|2472184|2472458|N|N|N|N|N| +2472550|AAAAAAAAGGKLFCAA|2057-07-05|1890|8219|631|2057|4|7|5|3|2057|631|8219|Thursday|2057Q3|Y|N|N|2472546|2472726|2472185|2472459|N|N|N|N|N| +2472551|AAAAAAAAHGKLFCAA|2057-07-06|1890|8219|631|2057|5|7|6|3|2057|631|8219|Friday|2057Q3|N|Y|Y|2472546|2472726|2472186|2472460|N|N|N|N|N| +2472552|AAAAAAAAIGKLFCAA|2057-07-07|1890|8219|631|2057|6|7|7|3|2057|631|8219|Saturday|2057Q3|N|Y|N|2472546|2472726|2472187|2472461|N|N|N|N|N| +2472553|AAAAAAAAJGKLFCAA|2057-07-08|1890|8219|631|2057|0|7|8|3|2057|631|8219|Sunday|2057Q3|N|N|N|2472546|2472726|2472188|2472462|N|N|N|N|N| +2472554|AAAAAAAAKGKLFCAA|2057-07-09|1890|8219|631|2057|1|7|9|3|2057|631|8219|Monday|2057Q3|N|N|N|2472546|2472726|2472189|2472463|N|N|N|N|N| +2472555|AAAAAAAALGKLFCAA|2057-07-10|1890|8220|631|2057|2|7|10|3|2057|631|8220|Tuesday|2057Q3|N|N|N|2472546|2472726|2472190|2472464|N|N|N|N|N| +2472556|AAAAAAAAMGKLFCAA|2057-07-11|1890|8220|631|2057|3|7|11|3|2057|631|8220|Wednesday|2057Q3|N|N|N|2472546|2472726|2472191|2472465|N|N|N|N|N| +2472557|AAAAAAAANGKLFCAA|2057-07-12|1890|8220|631|2057|4|7|12|3|2057|631|8220|Thursday|2057Q3|N|N|N|2472546|2472726|2472192|2472466|N|N|N|N|N| +2472558|AAAAAAAAOGKLFCAA|2057-07-13|1890|8220|631|2057|5|7|13|3|2057|631|8220|Friday|2057Q3|N|Y|N|2472546|2472726|2472193|2472467|N|N|N|N|N| +2472559|AAAAAAAAPGKLFCAA|2057-07-14|1890|8220|631|2057|6|7|14|3|2057|631|8220|Saturday|2057Q3|N|Y|N|2472546|2472726|2472194|2472468|N|N|N|N|N| +2472560|AAAAAAAAAHKLFCAA|2057-07-15|1890|8220|631|2057|0|7|15|3|2057|631|8220|Sunday|2057Q3|N|N|N|2472546|2472726|2472195|2472469|N|N|N|N|N| +2472561|AAAAAAAABHKLFCAA|2057-07-16|1890|8220|631|2057|1|7|16|3|2057|631|8220|Monday|2057Q3|N|N|N|2472546|2472726|2472196|2472470|N|N|N|N|N| +2472562|AAAAAAAACHKLFCAA|2057-07-17|1890|8221|631|2057|2|7|17|3|2057|631|8221|Tuesday|2057Q3|N|N|N|2472546|2472726|2472197|2472471|N|N|N|N|N| +2472563|AAAAAAAADHKLFCAA|2057-07-18|1890|8221|631|2057|3|7|18|3|2057|631|8221|Wednesday|2057Q3|N|N|N|2472546|2472726|2472198|2472472|N|N|N|N|N| +2472564|AAAAAAAAEHKLFCAA|2057-07-19|1890|8221|631|2057|4|7|19|3|2057|631|8221|Thursday|2057Q3|N|N|N|2472546|2472726|2472199|2472473|N|N|N|N|N| +2472565|AAAAAAAAFHKLFCAA|2057-07-20|1890|8221|631|2057|5|7|20|3|2057|631|8221|Friday|2057Q3|N|Y|N|2472546|2472726|2472200|2472474|N|N|N|N|N| +2472566|AAAAAAAAGHKLFCAA|2057-07-21|1890|8221|631|2057|6|7|21|3|2057|631|8221|Saturday|2057Q3|N|Y|N|2472546|2472726|2472201|2472475|N|N|N|N|N| +2472567|AAAAAAAAHHKLFCAA|2057-07-22|1890|8221|631|2057|0|7|22|3|2057|631|8221|Sunday|2057Q3|N|N|N|2472546|2472726|2472202|2472476|N|N|N|N|N| +2472568|AAAAAAAAIHKLFCAA|2057-07-23|1890|8221|631|2057|1|7|23|3|2057|631|8221|Monday|2057Q3|N|N|N|2472546|2472726|2472203|2472477|N|N|N|N|N| +2472569|AAAAAAAAJHKLFCAA|2057-07-24|1890|8222|631|2057|2|7|24|3|2057|631|8222|Tuesday|2057Q3|N|N|N|2472546|2472726|2472204|2472478|N|N|N|N|N| +2472570|AAAAAAAAKHKLFCAA|2057-07-25|1890|8222|631|2057|3|7|25|3|2057|631|8222|Wednesday|2057Q3|N|N|N|2472546|2472726|2472205|2472479|N|N|N|N|N| +2472571|AAAAAAAALHKLFCAA|2057-07-26|1890|8222|631|2057|4|7|26|3|2057|631|8222|Thursday|2057Q3|N|N|N|2472546|2472726|2472206|2472480|N|N|N|N|N| +2472572|AAAAAAAAMHKLFCAA|2057-07-27|1890|8222|631|2057|5|7|27|3|2057|631|8222|Friday|2057Q3|N|Y|N|2472546|2472726|2472207|2472481|N|N|N|N|N| +2472573|AAAAAAAANHKLFCAA|2057-07-28|1890|8222|631|2057|6|7|28|3|2057|631|8222|Saturday|2057Q3|N|Y|N|2472546|2472726|2472208|2472482|N|N|N|N|N| +2472574|AAAAAAAAOHKLFCAA|2057-07-29|1890|8222|631|2057|0|7|29|3|2057|631|8222|Sunday|2057Q3|N|N|N|2472546|2472726|2472209|2472483|N|N|N|N|N| +2472575|AAAAAAAAPHKLFCAA|2057-07-30|1890|8222|631|2057|1|7|30|3|2057|631|8222|Monday|2057Q3|N|N|N|2472546|2472726|2472210|2472484|N|N|N|N|N| +2472576|AAAAAAAAAIKLFCAA|2057-07-31|1890|8223|631|2057|2|7|31|3|2057|631|8223|Tuesday|2057Q3|N|N|N|2472546|2472726|2472211|2472485|N|N|N|N|N| +2472577|AAAAAAAABIKLFCAA|2057-08-01|1891|8223|631|2057|3|8|1|3|2057|631|8223|Wednesday|2057Q3|N|N|N|2472577|2472788|2472212|2472486|N|N|N|N|N| +2472578|AAAAAAAACIKLFCAA|2057-08-02|1891|8223|631|2057|4|8|2|3|2057|631|8223|Thursday|2057Q3|N|N|N|2472577|2472788|2472213|2472487|N|N|N|N|N| +2472579|AAAAAAAADIKLFCAA|2057-08-03|1891|8223|631|2057|5|8|3|3|2057|631|8223|Friday|2057Q3|N|Y|N|2472577|2472788|2472214|2472488|N|N|N|N|N| +2472580|AAAAAAAAEIKLFCAA|2057-08-04|1891|8223|631|2057|6|8|4|3|2057|631|8223|Saturday|2057Q3|N|Y|N|2472577|2472788|2472215|2472489|N|N|N|N|N| +2472581|AAAAAAAAFIKLFCAA|2057-08-05|1891|8223|631|2057|0|8|5|3|2057|631|8223|Sunday|2057Q3|N|N|N|2472577|2472788|2472216|2472490|N|N|N|N|N| +2472582|AAAAAAAAGIKLFCAA|2057-08-06|1891|8223|631|2057|1|8|6|3|2057|631|8223|Monday|2057Q3|N|N|N|2472577|2472788|2472217|2472491|N|N|N|N|N| +2472583|AAAAAAAAHIKLFCAA|2057-08-07|1891|8224|631|2057|2|8|7|3|2057|631|8224|Tuesday|2057Q3|N|N|N|2472577|2472788|2472218|2472492|N|N|N|N|N| +2472584|AAAAAAAAIIKLFCAA|2057-08-08|1891|8224|631|2057|3|8|8|3|2057|631|8224|Wednesday|2057Q3|N|N|N|2472577|2472788|2472219|2472493|N|N|N|N|N| +2472585|AAAAAAAAJIKLFCAA|2057-08-09|1891|8224|631|2057|4|8|9|3|2057|631|8224|Thursday|2057Q3|N|N|N|2472577|2472788|2472220|2472494|N|N|N|N|N| +2472586|AAAAAAAAKIKLFCAA|2057-08-10|1891|8224|631|2057|5|8|10|3|2057|631|8224|Friday|2057Q3|N|Y|N|2472577|2472788|2472221|2472495|N|N|N|N|N| +2472587|AAAAAAAALIKLFCAA|2057-08-11|1891|8224|631|2057|6|8|11|3|2057|631|8224|Saturday|2057Q3|N|Y|N|2472577|2472788|2472222|2472496|N|N|N|N|N| +2472588|AAAAAAAAMIKLFCAA|2057-08-12|1891|8224|631|2057|0|8|12|3|2057|631|8224|Sunday|2057Q3|N|N|N|2472577|2472788|2472223|2472497|N|N|N|N|N| +2472589|AAAAAAAANIKLFCAA|2057-08-13|1891|8224|631|2057|1|8|13|3|2057|631|8224|Monday|2057Q3|N|N|N|2472577|2472788|2472224|2472498|N|N|N|N|N| +2472590|AAAAAAAAOIKLFCAA|2057-08-14|1891|8225|631|2057|2|8|14|3|2057|631|8225|Tuesday|2057Q3|N|N|N|2472577|2472788|2472225|2472499|N|N|N|N|N| +2472591|AAAAAAAAPIKLFCAA|2057-08-15|1891|8225|631|2057|3|8|15|3|2057|631|8225|Wednesday|2057Q3|N|N|N|2472577|2472788|2472226|2472500|N|N|N|N|N| +2472592|AAAAAAAAAJKLFCAA|2057-08-16|1891|8225|631|2057|4|8|16|3|2057|631|8225|Thursday|2057Q3|N|N|N|2472577|2472788|2472227|2472501|N|N|N|N|N| +2472593|AAAAAAAABJKLFCAA|2057-08-17|1891|8225|631|2057|5|8|17|3|2057|631|8225|Friday|2057Q3|N|Y|N|2472577|2472788|2472228|2472502|N|N|N|N|N| +2472594|AAAAAAAACJKLFCAA|2057-08-18|1891|8225|631|2057|6|8|18|3|2057|631|8225|Saturday|2057Q3|N|Y|N|2472577|2472788|2472229|2472503|N|N|N|N|N| +2472595|AAAAAAAADJKLFCAA|2057-08-19|1891|8225|631|2057|0|8|19|3|2057|631|8225|Sunday|2057Q3|N|N|N|2472577|2472788|2472230|2472504|N|N|N|N|N| +2472596|AAAAAAAAEJKLFCAA|2057-08-20|1891|8225|631|2057|1|8|20|3|2057|631|8225|Monday|2057Q3|N|N|N|2472577|2472788|2472231|2472505|N|N|N|N|N| +2472597|AAAAAAAAFJKLFCAA|2057-08-21|1891|8226|631|2057|2|8|21|3|2057|631|8226|Tuesday|2057Q3|N|N|N|2472577|2472788|2472232|2472506|N|N|N|N|N| +2472598|AAAAAAAAGJKLFCAA|2057-08-22|1891|8226|631|2057|3|8|22|3|2057|631|8226|Wednesday|2057Q3|N|N|N|2472577|2472788|2472233|2472507|N|N|N|N|N| +2472599|AAAAAAAAHJKLFCAA|2057-08-23|1891|8226|631|2057|4|8|23|3|2057|631|8226|Thursday|2057Q3|N|N|N|2472577|2472788|2472234|2472508|N|N|N|N|N| +2472600|AAAAAAAAIJKLFCAA|2057-08-24|1891|8226|631|2057|5|8|24|3|2057|631|8226|Friday|2057Q3|N|Y|N|2472577|2472788|2472235|2472509|N|N|N|N|N| +2472601|AAAAAAAAJJKLFCAA|2057-08-25|1891|8226|631|2057|6|8|25|3|2057|631|8226|Saturday|2057Q3|N|Y|N|2472577|2472788|2472236|2472510|N|N|N|N|N| +2472602|AAAAAAAAKJKLFCAA|2057-08-26|1891|8226|631|2057|0|8|26|3|2057|631|8226|Sunday|2057Q3|N|N|N|2472577|2472788|2472237|2472511|N|N|N|N|N| +2472603|AAAAAAAALJKLFCAA|2057-08-27|1891|8226|631|2057|1|8|27|3|2057|631|8226|Monday|2057Q3|N|N|N|2472577|2472788|2472238|2472512|N|N|N|N|N| +2472604|AAAAAAAAMJKLFCAA|2057-08-28|1891|8227|631|2057|2|8|28|3|2057|631|8227|Tuesday|2057Q3|N|N|N|2472577|2472788|2472239|2472513|N|N|N|N|N| +2472605|AAAAAAAANJKLFCAA|2057-08-29|1891|8227|631|2057|3|8|29|3|2057|631|8227|Wednesday|2057Q3|N|N|N|2472577|2472788|2472240|2472514|N|N|N|N|N| +2472606|AAAAAAAAOJKLFCAA|2057-08-30|1891|8227|631|2057|4|8|30|3|2057|631|8227|Thursday|2057Q3|N|N|N|2472577|2472788|2472241|2472515|N|N|N|N|N| +2472607|AAAAAAAAPJKLFCAA|2057-08-31|1891|8227|631|2057|5|8|31|3|2057|631|8227|Friday|2057Q3|N|Y|N|2472577|2472788|2472242|2472516|N|N|N|N|N| +2472608|AAAAAAAAAKKLFCAA|2057-09-01|1892|8227|632|2057|6|9|1|3|2057|632|8227|Saturday|2057Q3|N|Y|N|2472608|2472850|2472243|2472517|N|N|N|N|N| +2472609|AAAAAAAABKKLFCAA|2057-09-02|1892|8227|632|2057|0|9|2|3|2057|632|8227|Sunday|2057Q3|N|N|N|2472608|2472850|2472244|2472518|N|N|N|N|N| +2472610|AAAAAAAACKKLFCAA|2057-09-03|1892|8227|632|2057|1|9|3|3|2057|632|8227|Monday|2057Q3|N|N|N|2472608|2472850|2472245|2472519|N|N|N|N|N| +2472611|AAAAAAAADKKLFCAA|2057-09-04|1892|8228|632|2057|2|9|4|3|2057|632|8228|Tuesday|2057Q3|N|N|N|2472608|2472850|2472246|2472520|N|N|N|N|N| +2472612|AAAAAAAAEKKLFCAA|2057-09-05|1892|8228|632|2057|3|9|5|3|2057|632|8228|Wednesday|2057Q3|N|N|N|2472608|2472850|2472247|2472521|N|N|N|N|N| +2472613|AAAAAAAAFKKLFCAA|2057-09-06|1892|8228|632|2057|4|9|6|3|2057|632|8228|Thursday|2057Q3|N|N|N|2472608|2472850|2472248|2472522|N|N|N|N|N| +2472614|AAAAAAAAGKKLFCAA|2057-09-07|1892|8228|632|2057|5|9|7|3|2057|632|8228|Friday|2057Q3|N|Y|N|2472608|2472850|2472249|2472523|N|N|N|N|N| +2472615|AAAAAAAAHKKLFCAA|2057-09-08|1892|8228|632|2057|6|9|8|3|2057|632|8228|Saturday|2057Q3|N|Y|N|2472608|2472850|2472250|2472524|N|N|N|N|N| +2472616|AAAAAAAAIKKLFCAA|2057-09-09|1892|8228|632|2057|0|9|9|3|2057|632|8228|Sunday|2057Q3|N|N|N|2472608|2472850|2472251|2472525|N|N|N|N|N| +2472617|AAAAAAAAJKKLFCAA|2057-09-10|1892|8228|632|2057|1|9|10|3|2057|632|8228|Monday|2057Q3|N|N|N|2472608|2472850|2472252|2472526|N|N|N|N|N| +2472618|AAAAAAAAKKKLFCAA|2057-09-11|1892|8229|632|2057|2|9|11|3|2057|632|8229|Tuesday|2057Q3|N|N|N|2472608|2472850|2472253|2472527|N|N|N|N|N| +2472619|AAAAAAAALKKLFCAA|2057-09-12|1892|8229|632|2057|3|9|12|3|2057|632|8229|Wednesday|2057Q3|N|N|N|2472608|2472850|2472254|2472528|N|N|N|N|N| +2472620|AAAAAAAAMKKLFCAA|2057-09-13|1892|8229|632|2057|4|9|13|3|2057|632|8229|Thursday|2057Q3|N|N|N|2472608|2472850|2472255|2472529|N|N|N|N|N| +2472621|AAAAAAAANKKLFCAA|2057-09-14|1892|8229|632|2057|5|9|14|3|2057|632|8229|Friday|2057Q3|N|Y|N|2472608|2472850|2472256|2472530|N|N|N|N|N| +2472622|AAAAAAAAOKKLFCAA|2057-09-15|1892|8229|632|2057|6|9|15|3|2057|632|8229|Saturday|2057Q3|N|Y|N|2472608|2472850|2472257|2472531|N|N|N|N|N| +2472623|AAAAAAAAPKKLFCAA|2057-09-16|1892|8229|632|2057|0|9|16|3|2057|632|8229|Sunday|2057Q3|N|N|N|2472608|2472850|2472258|2472532|N|N|N|N|N| +2472624|AAAAAAAAALKLFCAA|2057-09-17|1892|8229|632|2057|1|9|17|3|2057|632|8229|Monday|2057Q3|N|N|N|2472608|2472850|2472259|2472533|N|N|N|N|N| +2472625|AAAAAAAABLKLFCAA|2057-09-18|1892|8230|632|2057|2|9|18|3|2057|632|8230|Tuesday|2057Q3|N|N|N|2472608|2472850|2472260|2472534|N|N|N|N|N| +2472626|AAAAAAAACLKLFCAA|2057-09-19|1892|8230|632|2057|3|9|19|3|2057|632|8230|Wednesday|2057Q3|N|N|N|2472608|2472850|2472261|2472535|N|N|N|N|N| +2472627|AAAAAAAADLKLFCAA|2057-09-20|1892|8230|632|2057|4|9|20|3|2057|632|8230|Thursday|2057Q3|N|N|N|2472608|2472850|2472262|2472536|N|N|N|N|N| +2472628|AAAAAAAAELKLFCAA|2057-09-21|1892|8230|632|2057|5|9|21|3|2057|632|8230|Friday|2057Q3|N|Y|N|2472608|2472850|2472263|2472537|N|N|N|N|N| +2472629|AAAAAAAAFLKLFCAA|2057-09-22|1892|8230|632|2057|6|9|22|3|2057|632|8230|Saturday|2057Q3|N|Y|N|2472608|2472850|2472264|2472538|N|N|N|N|N| +2472630|AAAAAAAAGLKLFCAA|2057-09-23|1892|8230|632|2057|0|9|23|3|2057|632|8230|Sunday|2057Q3|N|N|N|2472608|2472850|2472265|2472539|N|N|N|N|N| +2472631|AAAAAAAAHLKLFCAA|2057-09-24|1892|8230|632|2057|1|9|24|3|2057|632|8230|Monday|2057Q3|N|N|N|2472608|2472850|2472266|2472540|N|N|N|N|N| +2472632|AAAAAAAAILKLFCAA|2057-09-25|1892|8231|632|2057|2|9|25|3|2057|632|8231|Tuesday|2057Q3|N|N|N|2472608|2472850|2472267|2472541|N|N|N|N|N| +2472633|AAAAAAAAJLKLFCAA|2057-09-26|1892|8231|632|2057|3|9|26|3|2057|632|8231|Wednesday|2057Q3|N|N|N|2472608|2472850|2472268|2472542|N|N|N|N|N| +2472634|AAAAAAAAKLKLFCAA|2057-09-27|1892|8231|632|2057|4|9|27|3|2057|632|8231|Thursday|2057Q3|N|N|N|2472608|2472850|2472269|2472543|N|N|N|N|N| +2472635|AAAAAAAALLKLFCAA|2057-09-28|1892|8231|632|2057|5|9|28|3|2057|632|8231|Friday|2057Q3|N|Y|N|2472608|2472850|2472270|2472544|N|N|N|N|N| +2472636|AAAAAAAAMLKLFCAA|2057-09-29|1892|8231|632|2057|6|9|29|3|2057|632|8231|Saturday|2057Q3|N|Y|N|2472608|2472850|2472271|2472545|N|N|N|N|N| +2472637|AAAAAAAANLKLFCAA|2057-09-30|1892|8231|632|2057|0|9|30|3|2057|632|8231|Sunday|2057Q3|N|N|N|2472608|2472850|2472272|2472546|N|N|N|N|N| +2472638|AAAAAAAAOLKLFCAA|2057-10-01|1893|8231|632|2057|1|10|1|3|2057|632|8231|Monday|2057Q3|N|N|N|2472638|2472910|2472273|2472546|N|N|N|N|N| +2472639|AAAAAAAAPLKLFCAA|2057-10-02|1893|8232|632|2057|2|10|2|4|2057|632|8232|Tuesday|2057Q4|N|N|N|2472638|2472910|2472274|2472547|N|N|N|N|N| +2472640|AAAAAAAAAMKLFCAA|2057-10-03|1893|8232|632|2057|3|10|3|4|2057|632|8232|Wednesday|2057Q4|N|N|N|2472638|2472910|2472275|2472548|N|N|N|N|N| +2472641|AAAAAAAABMKLFCAA|2057-10-04|1893|8232|632|2057|4|10|4|4|2057|632|8232|Thursday|2057Q4|N|N|N|2472638|2472910|2472276|2472549|N|N|N|N|N| +2472642|AAAAAAAACMKLFCAA|2057-10-05|1893|8232|632|2057|5|10|5|4|2057|632|8232|Friday|2057Q4|N|Y|N|2472638|2472910|2472277|2472550|N|N|N|N|N| +2472643|AAAAAAAADMKLFCAA|2057-10-06|1893|8232|632|2057|6|10|6|4|2057|632|8232|Saturday|2057Q4|N|Y|N|2472638|2472910|2472278|2472551|N|N|N|N|N| +2472644|AAAAAAAAEMKLFCAA|2057-10-07|1893|8232|632|2057|0|10|7|4|2057|632|8232|Sunday|2057Q4|N|N|N|2472638|2472910|2472279|2472552|N|N|N|N|N| +2472645|AAAAAAAAFMKLFCAA|2057-10-08|1893|8232|632|2057|1|10|8|4|2057|632|8232|Monday|2057Q4|N|N|N|2472638|2472910|2472280|2472553|N|N|N|N|N| +2472646|AAAAAAAAGMKLFCAA|2057-10-09|1893|8233|632|2057|2|10|9|4|2057|632|8233|Tuesday|2057Q4|N|N|N|2472638|2472910|2472281|2472554|N|N|N|N|N| +2472647|AAAAAAAAHMKLFCAA|2057-10-10|1893|8233|632|2057|3|10|10|4|2057|632|8233|Wednesday|2057Q4|N|N|N|2472638|2472910|2472282|2472555|N|N|N|N|N| +2472648|AAAAAAAAIMKLFCAA|2057-10-11|1893|8233|632|2057|4|10|11|4|2057|632|8233|Thursday|2057Q4|N|N|N|2472638|2472910|2472283|2472556|N|N|N|N|N| +2472649|AAAAAAAAJMKLFCAA|2057-10-12|1893|8233|632|2057|5|10|12|4|2057|632|8233|Friday|2057Q4|N|Y|N|2472638|2472910|2472284|2472557|N|N|N|N|N| +2472650|AAAAAAAAKMKLFCAA|2057-10-13|1893|8233|632|2057|6|10|13|4|2057|632|8233|Saturday|2057Q4|N|Y|N|2472638|2472910|2472285|2472558|N|N|N|N|N| +2472651|AAAAAAAALMKLFCAA|2057-10-14|1893|8233|632|2057|0|10|14|4|2057|632|8233|Sunday|2057Q4|N|N|N|2472638|2472910|2472286|2472559|N|N|N|N|N| +2472652|AAAAAAAAMMKLFCAA|2057-10-15|1893|8233|632|2057|1|10|15|4|2057|632|8233|Monday|2057Q4|N|N|N|2472638|2472910|2472287|2472560|N|N|N|N|N| +2472653|AAAAAAAANMKLFCAA|2057-10-16|1893|8234|632|2057|2|10|16|4|2057|632|8234|Tuesday|2057Q4|N|N|N|2472638|2472910|2472288|2472561|N|N|N|N|N| +2472654|AAAAAAAAOMKLFCAA|2057-10-17|1893|8234|632|2057|3|10|17|4|2057|632|8234|Wednesday|2057Q4|N|N|N|2472638|2472910|2472289|2472562|N|N|N|N|N| +2472655|AAAAAAAAPMKLFCAA|2057-10-18|1893|8234|632|2057|4|10|18|4|2057|632|8234|Thursday|2057Q4|N|N|N|2472638|2472910|2472290|2472563|N|N|N|N|N| +2472656|AAAAAAAAANKLFCAA|2057-10-19|1893|8234|632|2057|5|10|19|4|2057|632|8234|Friday|2057Q4|N|Y|N|2472638|2472910|2472291|2472564|N|N|N|N|N| +2472657|AAAAAAAABNKLFCAA|2057-10-20|1893|8234|632|2057|6|10|20|4|2057|632|8234|Saturday|2057Q4|N|Y|N|2472638|2472910|2472292|2472565|N|N|N|N|N| +2472658|AAAAAAAACNKLFCAA|2057-10-21|1893|8234|632|2057|0|10|21|4|2057|632|8234|Sunday|2057Q4|N|N|N|2472638|2472910|2472293|2472566|N|N|N|N|N| +2472659|AAAAAAAADNKLFCAA|2057-10-22|1893|8234|632|2057|1|10|22|4|2057|632|8234|Monday|2057Q4|N|N|N|2472638|2472910|2472294|2472567|N|N|N|N|N| +2472660|AAAAAAAAENKLFCAA|2057-10-23|1893|8235|632|2057|2|10|23|4|2057|632|8235|Tuesday|2057Q4|N|N|N|2472638|2472910|2472295|2472568|N|N|N|N|N| +2472661|AAAAAAAAFNKLFCAA|2057-10-24|1893|8235|632|2057|3|10|24|4|2057|632|8235|Wednesday|2057Q4|N|N|N|2472638|2472910|2472296|2472569|N|N|N|N|N| +2472662|AAAAAAAAGNKLFCAA|2057-10-25|1893|8235|632|2057|4|10|25|4|2057|632|8235|Thursday|2057Q4|N|N|N|2472638|2472910|2472297|2472570|N|N|N|N|N| +2472663|AAAAAAAAHNKLFCAA|2057-10-26|1893|8235|632|2057|5|10|26|4|2057|632|8235|Friday|2057Q4|N|Y|N|2472638|2472910|2472298|2472571|N|N|N|N|N| +2472664|AAAAAAAAINKLFCAA|2057-10-27|1893|8235|632|2057|6|10|27|4|2057|632|8235|Saturday|2057Q4|N|Y|N|2472638|2472910|2472299|2472572|N|N|N|N|N| +2472665|AAAAAAAAJNKLFCAA|2057-10-28|1893|8235|632|2057|0|10|28|4|2057|632|8235|Sunday|2057Q4|N|N|N|2472638|2472910|2472300|2472573|N|N|N|N|N| +2472666|AAAAAAAAKNKLFCAA|2057-10-29|1893|8235|632|2057|1|10|29|4|2057|632|8235|Monday|2057Q4|N|N|N|2472638|2472910|2472301|2472574|N|N|N|N|N| +2472667|AAAAAAAALNKLFCAA|2057-10-30|1893|8236|632|2057|2|10|30|4|2057|632|8236|Tuesday|2057Q4|N|N|N|2472638|2472910|2472302|2472575|N|N|N|N|N| +2472668|AAAAAAAAMNKLFCAA|2057-10-31|1893|8236|632|2057|3|10|31|4|2057|632|8236|Wednesday|2057Q4|N|N|N|2472638|2472910|2472303|2472576|N|N|N|N|N| +2472669|AAAAAAAANNKLFCAA|2057-11-01|1894|8236|632|2057|4|11|1|4|2057|632|8236|Thursday|2057Q4|N|N|N|2472669|2472972|2472304|2472577|N|N|N|N|N| +2472670|AAAAAAAAONKLFCAA|2057-11-02|1894|8236|632|2057|5|11|2|4|2057|632|8236|Friday|2057Q4|N|Y|N|2472669|2472972|2472305|2472578|N|N|N|N|N| +2472671|AAAAAAAAPNKLFCAA|2057-11-03|1894|8236|632|2057|6|11|3|4|2057|632|8236|Saturday|2057Q4|N|Y|N|2472669|2472972|2472306|2472579|N|N|N|N|N| +2472672|AAAAAAAAAOKLFCAA|2057-11-04|1894|8236|632|2057|0|11|4|4|2057|632|8236|Sunday|2057Q4|N|N|N|2472669|2472972|2472307|2472580|N|N|N|N|N| +2472673|AAAAAAAABOKLFCAA|2057-11-05|1894|8236|632|2057|1|11|5|4|2057|632|8236|Monday|2057Q4|N|N|N|2472669|2472972|2472308|2472581|N|N|N|N|N| +2472674|AAAAAAAACOKLFCAA|2057-11-06|1894|8237|632|2057|2|11|6|4|2057|632|8237|Tuesday|2057Q4|N|N|N|2472669|2472972|2472309|2472582|N|N|N|N|N| +2472675|AAAAAAAADOKLFCAA|2057-11-07|1894|8237|632|2057|3|11|7|4|2057|632|8237|Wednesday|2057Q4|N|N|N|2472669|2472972|2472310|2472583|N|N|N|N|N| +2472676|AAAAAAAAEOKLFCAA|2057-11-08|1894|8237|632|2057|4|11|8|4|2057|632|8237|Thursday|2057Q4|N|N|N|2472669|2472972|2472311|2472584|N|N|N|N|N| +2472677|AAAAAAAAFOKLFCAA|2057-11-09|1894|8237|632|2057|5|11|9|4|2057|632|8237|Friday|2057Q4|N|Y|N|2472669|2472972|2472312|2472585|N|N|N|N|N| +2472678|AAAAAAAAGOKLFCAA|2057-11-10|1894|8237|632|2057|6|11|10|4|2057|632|8237|Saturday|2057Q4|N|Y|N|2472669|2472972|2472313|2472586|N|N|N|N|N| +2472679|AAAAAAAAHOKLFCAA|2057-11-11|1894|8237|632|2057|0|11|11|4|2057|632|8237|Sunday|2057Q4|N|N|N|2472669|2472972|2472314|2472587|N|N|N|N|N| +2472680|AAAAAAAAIOKLFCAA|2057-11-12|1894|8237|632|2057|1|11|12|4|2057|632|8237|Monday|2057Q4|N|N|N|2472669|2472972|2472315|2472588|N|N|N|N|N| +2472681|AAAAAAAAJOKLFCAA|2057-11-13|1894|8238|632|2057|2|11|13|4|2057|632|8238|Tuesday|2057Q4|N|N|N|2472669|2472972|2472316|2472589|N|N|N|N|N| +2472682|AAAAAAAAKOKLFCAA|2057-11-14|1894|8238|632|2057|3|11|14|4|2057|632|8238|Wednesday|2057Q4|N|N|N|2472669|2472972|2472317|2472590|N|N|N|N|N| +2472683|AAAAAAAALOKLFCAA|2057-11-15|1894|8238|632|2057|4|11|15|4|2057|632|8238|Thursday|2057Q4|N|N|N|2472669|2472972|2472318|2472591|N|N|N|N|N| +2472684|AAAAAAAAMOKLFCAA|2057-11-16|1894|8238|632|2057|5|11|16|4|2057|632|8238|Friday|2057Q4|N|Y|N|2472669|2472972|2472319|2472592|N|N|N|N|N| +2472685|AAAAAAAANOKLFCAA|2057-11-17|1894|8238|632|2057|6|11|17|4|2057|632|8238|Saturday|2057Q4|N|Y|N|2472669|2472972|2472320|2472593|N|N|N|N|N| +2472686|AAAAAAAAOOKLFCAA|2057-11-18|1894|8238|632|2057|0|11|18|4|2057|632|8238|Sunday|2057Q4|N|N|N|2472669|2472972|2472321|2472594|N|N|N|N|N| +2472687|AAAAAAAAPOKLFCAA|2057-11-19|1894|8238|632|2057|1|11|19|4|2057|632|8238|Monday|2057Q4|N|N|N|2472669|2472972|2472322|2472595|N|N|N|N|N| +2472688|AAAAAAAAAPKLFCAA|2057-11-20|1894|8239|632|2057|2|11|20|4|2057|632|8239|Tuesday|2057Q4|N|N|N|2472669|2472972|2472323|2472596|N|N|N|N|N| +2472689|AAAAAAAABPKLFCAA|2057-11-21|1894|8239|632|2057|3|11|21|4|2057|632|8239|Wednesday|2057Q4|N|N|N|2472669|2472972|2472324|2472597|N|N|N|N|N| +2472690|AAAAAAAACPKLFCAA|2057-11-22|1894|8239|632|2057|4|11|22|4|2057|632|8239|Thursday|2057Q4|N|N|N|2472669|2472972|2472325|2472598|N|N|N|N|N| +2472691|AAAAAAAADPKLFCAA|2057-11-23|1894|8239|632|2057|5|11|23|4|2057|632|8239|Friday|2057Q4|N|Y|N|2472669|2472972|2472326|2472599|N|N|N|N|N| +2472692|AAAAAAAAEPKLFCAA|2057-11-24|1894|8239|632|2057|6|11|24|4|2057|632|8239|Saturday|2057Q4|N|Y|N|2472669|2472972|2472327|2472600|N|N|N|N|N| +2472693|AAAAAAAAFPKLFCAA|2057-11-25|1894|8239|632|2057|0|11|25|4|2057|632|8239|Sunday|2057Q4|N|N|N|2472669|2472972|2472328|2472601|N|N|N|N|N| +2472694|AAAAAAAAGPKLFCAA|2057-11-26|1894|8239|632|2057|1|11|26|4|2057|632|8239|Monday|2057Q4|N|N|N|2472669|2472972|2472329|2472602|N|N|N|N|N| +2472695|AAAAAAAAHPKLFCAA|2057-11-27|1894|8240|632|2057|2|11|27|4|2057|632|8240|Tuesday|2057Q4|N|N|N|2472669|2472972|2472330|2472603|N|N|N|N|N| +2472696|AAAAAAAAIPKLFCAA|2057-11-28|1894|8240|632|2057|3|11|28|4|2057|632|8240|Wednesday|2057Q4|N|N|N|2472669|2472972|2472331|2472604|N|N|N|N|N| +2472697|AAAAAAAAJPKLFCAA|2057-11-29|1894|8240|632|2057|4|11|29|4|2057|632|8240|Thursday|2057Q4|N|N|N|2472669|2472972|2472332|2472605|N|N|N|N|N| +2472698|AAAAAAAAKPKLFCAA|2057-11-30|1894|8240|632|2057|5|11|30|4|2057|632|8240|Friday|2057Q4|N|Y|N|2472669|2472972|2472333|2472606|N|N|N|N|N| +2472699|AAAAAAAALPKLFCAA|2057-12-01|1895|8240|633|2057|6|12|1|4|2057|633|8240|Saturday|2057Q4|N|Y|N|2472699|2473032|2472334|2472607|N|N|N|N|N| +2472700|AAAAAAAAMPKLFCAA|2057-12-02|1895|8240|633|2057|0|12|2|4|2057|633|8240|Sunday|2057Q4|N|N|N|2472699|2473032|2472335|2472608|N|N|N|N|N| +2472701|AAAAAAAANPKLFCAA|2057-12-03|1895|8240|633|2057|1|12|3|4|2057|633|8240|Monday|2057Q4|N|N|N|2472699|2473032|2472336|2472609|N|N|N|N|N| +2472702|AAAAAAAAOPKLFCAA|2057-12-04|1895|8241|633|2057|2|12|4|4|2057|633|8241|Tuesday|2057Q4|N|N|N|2472699|2473032|2472337|2472610|N|N|N|N|N| +2472703|AAAAAAAAPPKLFCAA|2057-12-05|1895|8241|633|2057|3|12|5|4|2057|633|8241|Wednesday|2057Q4|N|N|N|2472699|2473032|2472338|2472611|N|N|N|N|N| +2472704|AAAAAAAAAALLFCAA|2057-12-06|1895|8241|633|2057|4|12|6|4|2057|633|8241|Thursday|2057Q4|N|N|N|2472699|2473032|2472339|2472612|N|N|N|N|N| +2472705|AAAAAAAABALLFCAA|2057-12-07|1895|8241|633|2057|5|12|7|4|2057|633|8241|Friday|2057Q4|N|Y|N|2472699|2473032|2472340|2472613|N|N|N|N|N| +2472706|AAAAAAAACALLFCAA|2057-12-08|1895|8241|633|2057|6|12|8|4|2057|633|8241|Saturday|2057Q4|N|Y|N|2472699|2473032|2472341|2472614|N|N|N|N|N| +2472707|AAAAAAAADALLFCAA|2057-12-09|1895|8241|633|2057|0|12|9|4|2057|633|8241|Sunday|2057Q4|N|N|N|2472699|2473032|2472342|2472615|N|N|N|N|N| +2472708|AAAAAAAAEALLFCAA|2057-12-10|1895|8241|633|2057|1|12|10|4|2057|633|8241|Monday|2057Q4|N|N|N|2472699|2473032|2472343|2472616|N|N|N|N|N| +2472709|AAAAAAAAFALLFCAA|2057-12-11|1895|8242|633|2057|2|12|11|4|2057|633|8242|Tuesday|2057Q4|N|N|N|2472699|2473032|2472344|2472617|N|N|N|N|N| +2472710|AAAAAAAAGALLFCAA|2057-12-12|1895|8242|633|2057|3|12|12|4|2057|633|8242|Wednesday|2057Q4|N|N|N|2472699|2473032|2472345|2472618|N|N|N|N|N| +2472711|AAAAAAAAHALLFCAA|2057-12-13|1895|8242|633|2057|4|12|13|4|2057|633|8242|Thursday|2057Q4|N|N|N|2472699|2473032|2472346|2472619|N|N|N|N|N| +2472712|AAAAAAAAIALLFCAA|2057-12-14|1895|8242|633|2057|5|12|14|4|2057|633|8242|Friday|2057Q4|N|Y|N|2472699|2473032|2472347|2472620|N|N|N|N|N| +2472713|AAAAAAAAJALLFCAA|2057-12-15|1895|8242|633|2057|6|12|15|4|2057|633|8242|Saturday|2057Q4|N|Y|N|2472699|2473032|2472348|2472621|N|N|N|N|N| +2472714|AAAAAAAAKALLFCAA|2057-12-16|1895|8242|633|2057|0|12|16|4|2057|633|8242|Sunday|2057Q4|N|N|N|2472699|2473032|2472349|2472622|N|N|N|N|N| +2472715|AAAAAAAALALLFCAA|2057-12-17|1895|8242|633|2057|1|12|17|4|2057|633|8242|Monday|2057Q4|N|N|N|2472699|2473032|2472350|2472623|N|N|N|N|N| +2472716|AAAAAAAAMALLFCAA|2057-12-18|1895|8243|633|2057|2|12|18|4|2057|633|8243|Tuesday|2057Q4|N|N|N|2472699|2473032|2472351|2472624|N|N|N|N|N| +2472717|AAAAAAAANALLFCAA|2057-12-19|1895|8243|633|2057|3|12|19|4|2057|633|8243|Wednesday|2057Q4|N|N|N|2472699|2473032|2472352|2472625|N|N|N|N|N| +2472718|AAAAAAAAOALLFCAA|2057-12-20|1895|8243|633|2057|4|12|20|4|2057|633|8243|Thursday|2057Q4|N|N|N|2472699|2473032|2472353|2472626|N|N|N|N|N| +2472719|AAAAAAAAPALLFCAA|2057-12-21|1895|8243|633|2057|5|12|21|4|2057|633|8243|Friday|2057Q4|N|Y|N|2472699|2473032|2472354|2472627|N|N|N|N|N| +2472720|AAAAAAAAABLLFCAA|2057-12-22|1895|8243|633|2057|6|12|22|4|2057|633|8243|Saturday|2057Q4|N|Y|N|2472699|2473032|2472355|2472628|N|N|N|N|N| +2472721|AAAAAAAABBLLFCAA|2057-12-23|1895|8243|633|2057|0|12|23|4|2057|633|8243|Sunday|2057Q4|N|N|N|2472699|2473032|2472356|2472629|N|N|N|N|N| +2472722|AAAAAAAACBLLFCAA|2057-12-24|1895|8243|633|2057|1|12|24|4|2057|633|8243|Monday|2057Q4|N|N|N|2472699|2473032|2472357|2472630|N|N|N|N|N| +2472723|AAAAAAAADBLLFCAA|2057-12-25|1895|8244|633|2057|2|12|25|4|2057|633|8244|Tuesday|2057Q4|N|N|N|2472699|2473032|2472358|2472631|N|N|N|N|N| +2472724|AAAAAAAAEBLLFCAA|2057-12-26|1895|8244|633|2057|3|12|26|4|2057|633|8244|Wednesday|2057Q4|Y|N|N|2472699|2473032|2472359|2472632|N|N|N|N|N| +2472725|AAAAAAAAFBLLFCAA|2057-12-27|1895|8244|633|2057|4|12|27|4|2057|633|8244|Thursday|2057Q4|N|N|Y|2472699|2473032|2472360|2472633|N|N|N|N|N| +2472726|AAAAAAAAGBLLFCAA|2057-12-28|1895|8244|633|2057|5|12|28|4|2057|633|8244|Friday|2057Q4|N|Y|N|2472699|2473032|2472361|2472634|N|N|N|N|N| +2472727|AAAAAAAAHBLLFCAA|2057-12-29|1895|8244|633|2057|6|12|29|4|2057|633|8244|Saturday|2057Q4|N|Y|N|2472699|2473032|2472362|2472635|N|N|N|N|N| +2472728|AAAAAAAAIBLLFCAA|2057-12-30|1895|8244|633|2057|0|12|30|4|2057|633|8244|Sunday|2057Q4|N|N|N|2472699|2473032|2472363|2472636|N|N|N|N|N| +2472729|AAAAAAAAJBLLFCAA|2057-12-31|1895|8244|633|2057|1|12|31|4|2057|633|8244|Monday|2057Q4|N|N|N|2472699|2473032|2472364|2472637|N|N|N|N|N| +2472730|AAAAAAAAKBLLFCAA|2058-01-01|1896|8245|633|2058|2|1|1|1|2058|633|8245|Tuesday|2058Q1|Y|N|N|2472730|2472729|2472365|2472638|N|N|N|N|N| +2472731|AAAAAAAALBLLFCAA|2058-01-02|1896|8245|633|2058|3|1|2|1|2058|633|8245|Wednesday|2058Q1|N|N|Y|2472730|2472729|2472366|2472639|N|N|N|N|N| +2472732|AAAAAAAAMBLLFCAA|2058-01-03|1896|8245|633|2058|4|1|3|1|2058|633|8245|Thursday|2058Q1|N|N|N|2472730|2472729|2472367|2472640|N|N|N|N|N| +2472733|AAAAAAAANBLLFCAA|2058-01-04|1896|8245|633|2058|5|1|4|1|2058|633|8245|Friday|2058Q1|N|Y|N|2472730|2472729|2472368|2472641|N|N|N|N|N| +2472734|AAAAAAAAOBLLFCAA|2058-01-05|1896|8245|633|2058|6|1|5|1|2058|633|8245|Saturday|2058Q1|N|Y|N|2472730|2472729|2472369|2472642|N|N|N|N|N| +2472735|AAAAAAAAPBLLFCAA|2058-01-06|1896|8245|633|2058|0|1|6|1|2058|633|8245|Sunday|2058Q1|N|N|N|2472730|2472729|2472370|2472643|N|N|N|N|N| +2472736|AAAAAAAAACLLFCAA|2058-01-07|1896|8245|633|2058|1|1|7|1|2058|633|8245|Monday|2058Q1|N|N|N|2472730|2472729|2472371|2472644|N|N|N|N|N| +2472737|AAAAAAAABCLLFCAA|2058-01-08|1896|8246|633|2058|2|1|8|1|2058|633|8246|Tuesday|2058Q1|N|N|N|2472730|2472729|2472372|2472645|N|N|N|N|N| +2472738|AAAAAAAACCLLFCAA|2058-01-09|1896|8246|633|2058|3|1|9|1|2058|633|8246|Wednesday|2058Q1|N|N|N|2472730|2472729|2472373|2472646|N|N|N|N|N| +2472739|AAAAAAAADCLLFCAA|2058-01-10|1896|8246|633|2058|4|1|10|1|2058|633|8246|Thursday|2058Q1|N|N|N|2472730|2472729|2472374|2472647|N|N|N|N|N| +2472740|AAAAAAAAECLLFCAA|2058-01-11|1896|8246|633|2058|5|1|11|1|2058|633|8246|Friday|2058Q1|N|Y|N|2472730|2472729|2472375|2472648|N|N|N|N|N| +2472741|AAAAAAAAFCLLFCAA|2058-01-12|1896|8246|633|2058|6|1|12|1|2058|633|8246|Saturday|2058Q1|N|Y|N|2472730|2472729|2472376|2472649|N|N|N|N|N| +2472742|AAAAAAAAGCLLFCAA|2058-01-13|1896|8246|633|2058|0|1|13|1|2058|633|8246|Sunday|2058Q1|N|N|N|2472730|2472729|2472377|2472650|N|N|N|N|N| +2472743|AAAAAAAAHCLLFCAA|2058-01-14|1896|8246|633|2058|1|1|14|1|2058|633|8246|Monday|2058Q1|N|N|N|2472730|2472729|2472378|2472651|N|N|N|N|N| +2472744|AAAAAAAAICLLFCAA|2058-01-15|1896|8247|633|2058|2|1|15|1|2058|633|8247|Tuesday|2058Q1|N|N|N|2472730|2472729|2472379|2472652|N|N|N|N|N| +2472745|AAAAAAAAJCLLFCAA|2058-01-16|1896|8247|633|2058|3|1|16|1|2058|633|8247|Wednesday|2058Q1|N|N|N|2472730|2472729|2472380|2472653|N|N|N|N|N| +2472746|AAAAAAAAKCLLFCAA|2058-01-17|1896|8247|633|2058|4|1|17|1|2058|633|8247|Thursday|2058Q1|N|N|N|2472730|2472729|2472381|2472654|N|N|N|N|N| +2472747|AAAAAAAALCLLFCAA|2058-01-18|1896|8247|633|2058|5|1|18|1|2058|633|8247|Friday|2058Q1|N|Y|N|2472730|2472729|2472382|2472655|N|N|N|N|N| +2472748|AAAAAAAAMCLLFCAA|2058-01-19|1896|8247|633|2058|6|1|19|1|2058|633|8247|Saturday|2058Q1|N|Y|N|2472730|2472729|2472383|2472656|N|N|N|N|N| +2472749|AAAAAAAANCLLFCAA|2058-01-20|1896|8247|633|2058|0|1|20|1|2058|633|8247|Sunday|2058Q1|N|N|N|2472730|2472729|2472384|2472657|N|N|N|N|N| +2472750|AAAAAAAAOCLLFCAA|2058-01-21|1896|8247|633|2058|1|1|21|1|2058|633|8247|Monday|2058Q1|N|N|N|2472730|2472729|2472385|2472658|N|N|N|N|N| +2472751|AAAAAAAAPCLLFCAA|2058-01-22|1896|8248|633|2058|2|1|22|1|2058|633|8248|Tuesday|2058Q1|N|N|N|2472730|2472729|2472386|2472659|N|N|N|N|N| +2472752|AAAAAAAAADLLFCAA|2058-01-23|1896|8248|633|2058|3|1|23|1|2058|633|8248|Wednesday|2058Q1|N|N|N|2472730|2472729|2472387|2472660|N|N|N|N|N| +2472753|AAAAAAAABDLLFCAA|2058-01-24|1896|8248|633|2058|4|1|24|1|2058|633|8248|Thursday|2058Q1|N|N|N|2472730|2472729|2472388|2472661|N|N|N|N|N| +2472754|AAAAAAAACDLLFCAA|2058-01-25|1896|8248|633|2058|5|1|25|1|2058|633|8248|Friday|2058Q1|N|Y|N|2472730|2472729|2472389|2472662|N|N|N|N|N| +2472755|AAAAAAAADDLLFCAA|2058-01-26|1896|8248|633|2058|6|1|26|1|2058|633|8248|Saturday|2058Q1|N|Y|N|2472730|2472729|2472390|2472663|N|N|N|N|N| +2472756|AAAAAAAAEDLLFCAA|2058-01-27|1896|8248|633|2058|0|1|27|1|2058|633|8248|Sunday|2058Q1|N|N|N|2472730|2472729|2472391|2472664|N|N|N|N|N| +2472757|AAAAAAAAFDLLFCAA|2058-01-28|1896|8248|633|2058|1|1|28|1|2058|633|8248|Monday|2058Q1|N|N|N|2472730|2472729|2472392|2472665|N|N|N|N|N| +2472758|AAAAAAAAGDLLFCAA|2058-01-29|1896|8249|633|2058|2|1|29|1|2058|633|8249|Tuesday|2058Q1|N|N|N|2472730|2472729|2472393|2472666|N|N|N|N|N| +2472759|AAAAAAAAHDLLFCAA|2058-01-30|1896|8249|633|2058|3|1|30|1|2058|633|8249|Wednesday|2058Q1|N|N|N|2472730|2472729|2472394|2472667|N|N|N|N|N| +2472760|AAAAAAAAIDLLFCAA|2058-01-31|1896|8249|633|2058|4|1|31|1|2058|633|8249|Thursday|2058Q1|N|N|N|2472730|2472729|2472395|2472668|N|N|N|N|N| +2472761|AAAAAAAAJDLLFCAA|2058-02-01|1897|8249|633|2058|5|2|1|1|2058|633|8249|Friday|2058Q1|N|Y|N|2472761|2472791|2472396|2472669|N|N|N|N|N| +2472762|AAAAAAAAKDLLFCAA|2058-02-02|1897|8249|633|2058|6|2|2|1|2058|633|8249|Saturday|2058Q1|N|Y|N|2472761|2472791|2472397|2472670|N|N|N|N|N| +2472763|AAAAAAAALDLLFCAA|2058-02-03|1897|8249|633|2058|0|2|3|1|2058|633|8249|Sunday|2058Q1|N|N|N|2472761|2472791|2472398|2472671|N|N|N|N|N| +2472764|AAAAAAAAMDLLFCAA|2058-02-04|1897|8249|633|2058|1|2|4|1|2058|633|8249|Monday|2058Q1|N|N|N|2472761|2472791|2472399|2472672|N|N|N|N|N| +2472765|AAAAAAAANDLLFCAA|2058-02-05|1897|8250|633|2058|2|2|5|1|2058|633|8250|Tuesday|2058Q1|N|N|N|2472761|2472791|2472400|2472673|N|N|N|N|N| +2472766|AAAAAAAAODLLFCAA|2058-02-06|1897|8250|633|2058|3|2|6|1|2058|633|8250|Wednesday|2058Q1|N|N|N|2472761|2472791|2472401|2472674|N|N|N|N|N| +2472767|AAAAAAAAPDLLFCAA|2058-02-07|1897|8250|633|2058|4|2|7|1|2058|633|8250|Thursday|2058Q1|N|N|N|2472761|2472791|2472402|2472675|N|N|N|N|N| +2472768|AAAAAAAAAELLFCAA|2058-02-08|1897|8250|633|2058|5|2|8|1|2058|633|8250|Friday|2058Q1|N|Y|N|2472761|2472791|2472403|2472676|N|N|N|N|N| +2472769|AAAAAAAABELLFCAA|2058-02-09|1897|8250|633|2058|6|2|9|1|2058|633|8250|Saturday|2058Q1|N|Y|N|2472761|2472791|2472404|2472677|N|N|N|N|N| +2472770|AAAAAAAACELLFCAA|2058-02-10|1897|8250|633|2058|0|2|10|1|2058|633|8250|Sunday|2058Q1|N|N|N|2472761|2472791|2472405|2472678|N|N|N|N|N| +2472771|AAAAAAAADELLFCAA|2058-02-11|1897|8250|633|2058|1|2|11|1|2058|633|8250|Monday|2058Q1|N|N|N|2472761|2472791|2472406|2472679|N|N|N|N|N| +2472772|AAAAAAAAEELLFCAA|2058-02-12|1897|8251|633|2058|2|2|12|1|2058|633|8251|Tuesday|2058Q1|N|N|N|2472761|2472791|2472407|2472680|N|N|N|N|N| +2472773|AAAAAAAAFELLFCAA|2058-02-13|1897|8251|633|2058|3|2|13|1|2058|633|8251|Wednesday|2058Q1|N|N|N|2472761|2472791|2472408|2472681|N|N|N|N|N| +2472774|AAAAAAAAGELLFCAA|2058-02-14|1897|8251|633|2058|4|2|14|1|2058|633|8251|Thursday|2058Q1|N|N|N|2472761|2472791|2472409|2472682|N|N|N|N|N| +2472775|AAAAAAAAHELLFCAA|2058-02-15|1897|8251|633|2058|5|2|15|1|2058|633|8251|Friday|2058Q1|N|Y|N|2472761|2472791|2472410|2472683|N|N|N|N|N| +2472776|AAAAAAAAIELLFCAA|2058-02-16|1897|8251|633|2058|6|2|16|1|2058|633|8251|Saturday|2058Q1|N|Y|N|2472761|2472791|2472411|2472684|N|N|N|N|N| +2472777|AAAAAAAAJELLFCAA|2058-02-17|1897|8251|633|2058|0|2|17|1|2058|633|8251|Sunday|2058Q1|N|N|N|2472761|2472791|2472412|2472685|N|N|N|N|N| +2472778|AAAAAAAAKELLFCAA|2058-02-18|1897|8251|633|2058|1|2|18|1|2058|633|8251|Monday|2058Q1|N|N|N|2472761|2472791|2472413|2472686|N|N|N|N|N| +2472779|AAAAAAAALELLFCAA|2058-02-19|1897|8252|633|2058|2|2|19|1|2058|633|8252|Tuesday|2058Q1|N|N|N|2472761|2472791|2472414|2472687|N|N|N|N|N| +2472780|AAAAAAAAMELLFCAA|2058-02-20|1897|8252|633|2058|3|2|20|1|2058|633|8252|Wednesday|2058Q1|N|N|N|2472761|2472791|2472415|2472688|N|N|N|N|N| +2472781|AAAAAAAANELLFCAA|2058-02-21|1897|8252|633|2058|4|2|21|1|2058|633|8252|Thursday|2058Q1|N|N|N|2472761|2472791|2472416|2472689|N|N|N|N|N| +2472782|AAAAAAAAOELLFCAA|2058-02-22|1897|8252|633|2058|5|2|22|1|2058|633|8252|Friday|2058Q1|N|Y|N|2472761|2472791|2472417|2472690|N|N|N|N|N| +2472783|AAAAAAAAPELLFCAA|2058-02-23|1897|8252|633|2058|6|2|23|1|2058|633|8252|Saturday|2058Q1|N|Y|N|2472761|2472791|2472418|2472691|N|N|N|N|N| +2472784|AAAAAAAAAFLLFCAA|2058-02-24|1897|8252|633|2058|0|2|24|1|2058|633|8252|Sunday|2058Q1|N|N|N|2472761|2472791|2472419|2472692|N|N|N|N|N| +2472785|AAAAAAAABFLLFCAA|2058-02-25|1897|8252|633|2058|1|2|25|1|2058|633|8252|Monday|2058Q1|N|N|N|2472761|2472791|2472420|2472693|N|N|N|N|N| +2472786|AAAAAAAACFLLFCAA|2058-02-26|1897|8253|633|2058|2|2|26|1|2058|633|8253|Tuesday|2058Q1|N|N|N|2472761|2472791|2472421|2472694|N|N|N|N|N| +2472787|AAAAAAAADFLLFCAA|2058-02-27|1897|8253|633|2058|3|2|27|1|2058|633|8253|Wednesday|2058Q1|N|N|N|2472761|2472791|2472422|2472695|N|N|N|N|N| +2472788|AAAAAAAAEFLLFCAA|2058-02-28|1897|8253|633|2058|4|2|28|1|2058|633|8253|Thursday|2058Q1|N|N|N|2472761|2472791|2472423|2472696|N|N|N|N|N| +2472789|AAAAAAAAFFLLFCAA|2058-03-01|1898|8253|634|2058|5|3|1|1|2058|634|8253|Friday|2058Q1|N|Y|N|2472789|2472847|2472424|2472697|N|N|N|N|N| +2472790|AAAAAAAAGFLLFCAA|2058-03-02|1898|8253|634|2058|6|3|2|1|2058|634|8253|Saturday|2058Q1|N|Y|N|2472789|2472847|2472425|2472698|N|N|N|N|N| +2472791|AAAAAAAAHFLLFCAA|2058-03-03|1898|8253|634|2058|0|3|3|1|2058|634|8253|Sunday|2058Q1|N|N|N|2472789|2472847|2472426|2472699|N|N|N|N|N| +2472792|AAAAAAAAIFLLFCAA|2058-03-04|1898|8253|634|2058|1|3|4|1|2058|634|8253|Monday|2058Q1|N|N|N|2472789|2472847|2472427|2472700|N|N|N|N|N| +2472793|AAAAAAAAJFLLFCAA|2058-03-05|1898|8254|634|2058|2|3|5|1|2058|634|8254|Tuesday|2058Q1|N|N|N|2472789|2472847|2472428|2472701|N|N|N|N|N| +2472794|AAAAAAAAKFLLFCAA|2058-03-06|1898|8254|634|2058|3|3|6|1|2058|634|8254|Wednesday|2058Q1|N|N|N|2472789|2472847|2472429|2472702|N|N|N|N|N| +2472795|AAAAAAAALFLLFCAA|2058-03-07|1898|8254|634|2058|4|3|7|1|2058|634|8254|Thursday|2058Q1|N|N|N|2472789|2472847|2472430|2472703|N|N|N|N|N| +2472796|AAAAAAAAMFLLFCAA|2058-03-08|1898|8254|634|2058|5|3|8|1|2058|634|8254|Friday|2058Q1|N|Y|N|2472789|2472847|2472431|2472704|N|N|N|N|N| +2472797|AAAAAAAANFLLFCAA|2058-03-09|1898|8254|634|2058|6|3|9|1|2058|634|8254|Saturday|2058Q1|N|Y|N|2472789|2472847|2472432|2472705|N|N|N|N|N| +2472798|AAAAAAAAOFLLFCAA|2058-03-10|1898|8254|634|2058|0|3|10|1|2058|634|8254|Sunday|2058Q1|N|N|N|2472789|2472847|2472433|2472706|N|N|N|N|N| +2472799|AAAAAAAAPFLLFCAA|2058-03-11|1898|8254|634|2058|1|3|11|1|2058|634|8254|Monday|2058Q1|N|N|N|2472789|2472847|2472434|2472707|N|N|N|N|N| +2472800|AAAAAAAAAGLLFCAA|2058-03-12|1898|8255|634|2058|2|3|12|1|2058|634|8255|Tuesday|2058Q1|N|N|N|2472789|2472847|2472435|2472708|N|N|N|N|N| +2472801|AAAAAAAABGLLFCAA|2058-03-13|1898|8255|634|2058|3|3|13|1|2058|634|8255|Wednesday|2058Q1|N|N|N|2472789|2472847|2472436|2472709|N|N|N|N|N| +2472802|AAAAAAAACGLLFCAA|2058-03-14|1898|8255|634|2058|4|3|14|1|2058|634|8255|Thursday|2058Q1|N|N|N|2472789|2472847|2472437|2472710|N|N|N|N|N| +2472803|AAAAAAAADGLLFCAA|2058-03-15|1898|8255|634|2058|5|3|15|1|2058|634|8255|Friday|2058Q1|N|Y|N|2472789|2472847|2472438|2472711|N|N|N|N|N| +2472804|AAAAAAAAEGLLFCAA|2058-03-16|1898|8255|634|2058|6|3|16|1|2058|634|8255|Saturday|2058Q1|N|Y|N|2472789|2472847|2472439|2472712|N|N|N|N|N| +2472805|AAAAAAAAFGLLFCAA|2058-03-17|1898|8255|634|2058|0|3|17|1|2058|634|8255|Sunday|2058Q1|N|N|N|2472789|2472847|2472440|2472713|N|N|N|N|N| +2472806|AAAAAAAAGGLLFCAA|2058-03-18|1898|8255|634|2058|1|3|18|1|2058|634|8255|Monday|2058Q1|N|N|N|2472789|2472847|2472441|2472714|N|N|N|N|N| +2472807|AAAAAAAAHGLLFCAA|2058-03-19|1898|8256|634|2058|2|3|19|1|2058|634|8256|Tuesday|2058Q1|N|N|N|2472789|2472847|2472442|2472715|N|N|N|N|N| +2472808|AAAAAAAAIGLLFCAA|2058-03-20|1898|8256|634|2058|3|3|20|1|2058|634|8256|Wednesday|2058Q1|N|N|N|2472789|2472847|2472443|2472716|N|N|N|N|N| +2472809|AAAAAAAAJGLLFCAA|2058-03-21|1898|8256|634|2058|4|3|21|1|2058|634|8256|Thursday|2058Q1|N|N|N|2472789|2472847|2472444|2472717|N|N|N|N|N| +2472810|AAAAAAAAKGLLFCAA|2058-03-22|1898|8256|634|2058|5|3|22|1|2058|634|8256|Friday|2058Q1|N|Y|N|2472789|2472847|2472445|2472718|N|N|N|N|N| +2472811|AAAAAAAALGLLFCAA|2058-03-23|1898|8256|634|2058|6|3|23|1|2058|634|8256|Saturday|2058Q1|N|Y|N|2472789|2472847|2472446|2472719|N|N|N|N|N| +2472812|AAAAAAAAMGLLFCAA|2058-03-24|1898|8256|634|2058|0|3|24|1|2058|634|8256|Sunday|2058Q1|N|N|N|2472789|2472847|2472447|2472720|N|N|N|N|N| +2472813|AAAAAAAANGLLFCAA|2058-03-25|1898|8256|634|2058|1|3|25|1|2058|634|8256|Monday|2058Q1|N|N|N|2472789|2472847|2472448|2472721|N|N|N|N|N| +2472814|AAAAAAAAOGLLFCAA|2058-03-26|1898|8257|634|2058|2|3|26|1|2058|634|8257|Tuesday|2058Q1|N|N|N|2472789|2472847|2472449|2472722|N|N|N|N|N| +2472815|AAAAAAAAPGLLFCAA|2058-03-27|1898|8257|634|2058|3|3|27|1|2058|634|8257|Wednesday|2058Q1|N|N|N|2472789|2472847|2472450|2472723|N|N|N|N|N| +2472816|AAAAAAAAAHLLFCAA|2058-03-28|1898|8257|634|2058|4|3|28|1|2058|634|8257|Thursday|2058Q1|N|N|N|2472789|2472847|2472451|2472724|N|N|N|N|N| +2472817|AAAAAAAABHLLFCAA|2058-03-29|1898|8257|634|2058|5|3|29|1|2058|634|8257|Friday|2058Q1|N|Y|N|2472789|2472847|2472452|2472725|N|N|N|N|N| +2472818|AAAAAAAACHLLFCAA|2058-03-30|1898|8257|634|2058|6|3|30|1|2058|634|8257|Saturday|2058Q1|N|Y|N|2472789|2472847|2472453|2472726|N|N|N|N|N| +2472819|AAAAAAAADHLLFCAA|2058-03-31|1898|8257|634|2058|0|3|31|1|2058|634|8257|Sunday|2058Q1|N|N|N|2472789|2472847|2472454|2472727|N|N|N|N|N| +2472820|AAAAAAAAEHLLFCAA|2058-04-01|1899|8257|634|2058|1|4|1|1|2058|634|8257|Monday|2058Q1|N|N|N|2472820|2472909|2472455|2472730|N|N|N|N|N| +2472821|AAAAAAAAFHLLFCAA|2058-04-02|1899|8258|634|2058|2|4|2|2|2058|634|8258|Tuesday|2058Q2|N|N|N|2472820|2472909|2472456|2472731|N|N|N|N|N| +2472822|AAAAAAAAGHLLFCAA|2058-04-03|1899|8258|634|2058|3|4|3|2|2058|634|8258|Wednesday|2058Q2|N|N|N|2472820|2472909|2472457|2472732|N|N|N|N|N| +2472823|AAAAAAAAHHLLFCAA|2058-04-04|1899|8258|634|2058|4|4|4|2|2058|634|8258|Thursday|2058Q2|N|N|N|2472820|2472909|2472458|2472733|N|N|N|N|N| +2472824|AAAAAAAAIHLLFCAA|2058-04-05|1899|8258|634|2058|5|4|5|2|2058|634|8258|Friday|2058Q2|N|Y|N|2472820|2472909|2472459|2472734|N|N|N|N|N| +2472825|AAAAAAAAJHLLFCAA|2058-04-06|1899|8258|634|2058|6|4|6|2|2058|634|8258|Saturday|2058Q2|N|Y|N|2472820|2472909|2472460|2472735|N|N|N|N|N| +2472826|AAAAAAAAKHLLFCAA|2058-04-07|1899|8258|634|2058|0|4|7|2|2058|634|8258|Sunday|2058Q2|N|N|N|2472820|2472909|2472461|2472736|N|N|N|N|N| +2472827|AAAAAAAALHLLFCAA|2058-04-08|1899|8258|634|2058|1|4|8|2|2058|634|8258|Monday|2058Q2|N|N|N|2472820|2472909|2472462|2472737|N|N|N|N|N| +2472828|AAAAAAAAMHLLFCAA|2058-04-09|1899|8259|634|2058|2|4|9|2|2058|634|8259|Tuesday|2058Q2|N|N|N|2472820|2472909|2472463|2472738|N|N|N|N|N| +2472829|AAAAAAAANHLLFCAA|2058-04-10|1899|8259|634|2058|3|4|10|2|2058|634|8259|Wednesday|2058Q2|N|N|N|2472820|2472909|2472464|2472739|N|N|N|N|N| +2472830|AAAAAAAAOHLLFCAA|2058-04-11|1899|8259|634|2058|4|4|11|2|2058|634|8259|Thursday|2058Q2|N|N|N|2472820|2472909|2472465|2472740|N|N|N|N|N| +2472831|AAAAAAAAPHLLFCAA|2058-04-12|1899|8259|634|2058|5|4|12|2|2058|634|8259|Friday|2058Q2|N|Y|N|2472820|2472909|2472466|2472741|N|N|N|N|N| +2472832|AAAAAAAAAILLFCAA|2058-04-13|1899|8259|634|2058|6|4|13|2|2058|634|8259|Saturday|2058Q2|N|Y|N|2472820|2472909|2472467|2472742|N|N|N|N|N| +2472833|AAAAAAAABILLFCAA|2058-04-14|1899|8259|634|2058|0|4|14|2|2058|634|8259|Sunday|2058Q2|N|N|N|2472820|2472909|2472468|2472743|N|N|N|N|N| +2472834|AAAAAAAACILLFCAA|2058-04-15|1899|8259|634|2058|1|4|15|2|2058|634|8259|Monday|2058Q2|N|N|N|2472820|2472909|2472469|2472744|N|N|N|N|N| +2472835|AAAAAAAADILLFCAA|2058-04-16|1899|8260|634|2058|2|4|16|2|2058|634|8260|Tuesday|2058Q2|N|N|N|2472820|2472909|2472470|2472745|N|N|N|N|N| +2472836|AAAAAAAAEILLFCAA|2058-04-17|1899|8260|634|2058|3|4|17|2|2058|634|8260|Wednesday|2058Q2|N|N|N|2472820|2472909|2472471|2472746|N|N|N|N|N| +2472837|AAAAAAAAFILLFCAA|2058-04-18|1899|8260|634|2058|4|4|18|2|2058|634|8260|Thursday|2058Q2|N|N|N|2472820|2472909|2472472|2472747|N|N|N|N|N| +2472838|AAAAAAAAGILLFCAA|2058-04-19|1899|8260|634|2058|5|4|19|2|2058|634|8260|Friday|2058Q2|N|Y|N|2472820|2472909|2472473|2472748|N|N|N|N|N| +2472839|AAAAAAAAHILLFCAA|2058-04-20|1899|8260|634|2058|6|4|20|2|2058|634|8260|Saturday|2058Q2|N|Y|N|2472820|2472909|2472474|2472749|N|N|N|N|N| +2472840|AAAAAAAAIILLFCAA|2058-04-21|1899|8260|634|2058|0|4|21|2|2058|634|8260|Sunday|2058Q2|N|N|N|2472820|2472909|2472475|2472750|N|N|N|N|N| +2472841|AAAAAAAAJILLFCAA|2058-04-22|1899|8260|634|2058|1|4|22|2|2058|634|8260|Monday|2058Q2|N|N|N|2472820|2472909|2472476|2472751|N|N|N|N|N| +2472842|AAAAAAAAKILLFCAA|2058-04-23|1899|8261|634|2058|2|4|23|2|2058|634|8261|Tuesday|2058Q2|N|N|N|2472820|2472909|2472477|2472752|N|N|N|N|N| +2472843|AAAAAAAALILLFCAA|2058-04-24|1899|8261|634|2058|3|4|24|2|2058|634|8261|Wednesday|2058Q2|N|N|N|2472820|2472909|2472478|2472753|N|N|N|N|N| +2472844|AAAAAAAAMILLFCAA|2058-04-25|1899|8261|634|2058|4|4|25|2|2058|634|8261|Thursday|2058Q2|N|N|N|2472820|2472909|2472479|2472754|N|N|N|N|N| +2472845|AAAAAAAANILLFCAA|2058-04-26|1899|8261|634|2058|5|4|26|2|2058|634|8261|Friday|2058Q2|N|Y|N|2472820|2472909|2472480|2472755|N|N|N|N|N| +2472846|AAAAAAAAOILLFCAA|2058-04-27|1899|8261|634|2058|6|4|27|2|2058|634|8261|Saturday|2058Q2|N|Y|N|2472820|2472909|2472481|2472756|N|N|N|N|N| +2472847|AAAAAAAAPILLFCAA|2058-04-28|1899|8261|634|2058|0|4|28|2|2058|634|8261|Sunday|2058Q2|N|N|N|2472820|2472909|2472482|2472757|N|N|N|N|N| +2472848|AAAAAAAAAJLLFCAA|2058-04-29|1899|8261|634|2058|1|4|29|2|2058|634|8261|Monday|2058Q2|N|N|N|2472820|2472909|2472483|2472758|N|N|N|N|N| +2472849|AAAAAAAABJLLFCAA|2058-04-30|1899|8262|634|2058|2|4|30|2|2058|634|8262|Tuesday|2058Q2|N|N|N|2472820|2472909|2472484|2472759|N|N|N|N|N| +2472850|AAAAAAAACJLLFCAA|2058-05-01|1900|8262|634|2058|3|5|1|2|2058|634|8262|Wednesday|2058Q2|N|N|N|2472850|2472969|2472485|2472760|N|N|N|N|N| +2472851|AAAAAAAADJLLFCAA|2058-05-02|1900|8262|634|2058|4|5|2|2|2058|634|8262|Thursday|2058Q2|N|N|N|2472850|2472969|2472486|2472761|N|N|N|N|N| +2472852|AAAAAAAAEJLLFCAA|2058-05-03|1900|8262|634|2058|5|5|3|2|2058|634|8262|Friday|2058Q2|N|Y|N|2472850|2472969|2472487|2472762|N|N|N|N|N| +2472853|AAAAAAAAFJLLFCAA|2058-05-04|1900|8262|634|2058|6|5|4|2|2058|634|8262|Saturday|2058Q2|N|Y|N|2472850|2472969|2472488|2472763|N|N|N|N|N| +2472854|AAAAAAAAGJLLFCAA|2058-05-05|1900|8262|634|2058|0|5|5|2|2058|634|8262|Sunday|2058Q2|N|N|N|2472850|2472969|2472489|2472764|N|N|N|N|N| +2472855|AAAAAAAAHJLLFCAA|2058-05-06|1900|8262|634|2058|1|5|6|2|2058|634|8262|Monday|2058Q2|N|N|N|2472850|2472969|2472490|2472765|N|N|N|N|N| +2472856|AAAAAAAAIJLLFCAA|2058-05-07|1900|8263|634|2058|2|5|7|2|2058|634|8263|Tuesday|2058Q2|N|N|N|2472850|2472969|2472491|2472766|N|N|N|N|N| +2472857|AAAAAAAAJJLLFCAA|2058-05-08|1900|8263|634|2058|3|5|8|2|2058|634|8263|Wednesday|2058Q2|N|N|N|2472850|2472969|2472492|2472767|N|N|N|N|N| +2472858|AAAAAAAAKJLLFCAA|2058-05-09|1900|8263|634|2058|4|5|9|2|2058|634|8263|Thursday|2058Q2|N|N|N|2472850|2472969|2472493|2472768|N|N|N|N|N| +2472859|AAAAAAAALJLLFCAA|2058-05-10|1900|8263|634|2058|5|5|10|2|2058|634|8263|Friday|2058Q2|N|Y|N|2472850|2472969|2472494|2472769|N|N|N|N|N| +2472860|AAAAAAAAMJLLFCAA|2058-05-11|1900|8263|634|2058|6|5|11|2|2058|634|8263|Saturday|2058Q2|N|Y|N|2472850|2472969|2472495|2472770|N|N|N|N|N| +2472861|AAAAAAAANJLLFCAA|2058-05-12|1900|8263|634|2058|0|5|12|2|2058|634|8263|Sunday|2058Q2|N|N|N|2472850|2472969|2472496|2472771|N|N|N|N|N| +2472862|AAAAAAAAOJLLFCAA|2058-05-13|1900|8263|634|2058|1|5|13|2|2058|634|8263|Monday|2058Q2|N|N|N|2472850|2472969|2472497|2472772|N|N|N|N|N| +2472863|AAAAAAAAPJLLFCAA|2058-05-14|1900|8264|634|2058|2|5|14|2|2058|634|8264|Tuesday|2058Q2|N|N|N|2472850|2472969|2472498|2472773|N|N|N|N|N| +2472864|AAAAAAAAAKLLFCAA|2058-05-15|1900|8264|634|2058|3|5|15|2|2058|634|8264|Wednesday|2058Q2|N|N|N|2472850|2472969|2472499|2472774|N|N|N|N|N| +2472865|AAAAAAAABKLLFCAA|2058-05-16|1900|8264|634|2058|4|5|16|2|2058|634|8264|Thursday|2058Q2|N|N|N|2472850|2472969|2472500|2472775|N|N|N|N|N| +2472866|AAAAAAAACKLLFCAA|2058-05-17|1900|8264|634|2058|5|5|17|2|2058|634|8264|Friday|2058Q2|N|Y|N|2472850|2472969|2472501|2472776|N|N|N|N|N| +2472867|AAAAAAAADKLLFCAA|2058-05-18|1900|8264|634|2058|6|5|18|2|2058|634|8264|Saturday|2058Q2|N|Y|N|2472850|2472969|2472502|2472777|N|N|N|N|N| +2472868|AAAAAAAAEKLLFCAA|2058-05-19|1900|8264|634|2058|0|5|19|2|2058|634|8264|Sunday|2058Q2|N|N|N|2472850|2472969|2472503|2472778|N|N|N|N|N| +2472869|AAAAAAAAFKLLFCAA|2058-05-20|1900|8264|634|2058|1|5|20|2|2058|634|8264|Monday|2058Q2|N|N|N|2472850|2472969|2472504|2472779|N|N|N|N|N| +2472870|AAAAAAAAGKLLFCAA|2058-05-21|1900|8265|634|2058|2|5|21|2|2058|634|8265|Tuesday|2058Q2|N|N|N|2472850|2472969|2472505|2472780|N|N|N|N|N| +2472871|AAAAAAAAHKLLFCAA|2058-05-22|1900|8265|634|2058|3|5|22|2|2058|634|8265|Wednesday|2058Q2|N|N|N|2472850|2472969|2472506|2472781|N|N|N|N|N| +2472872|AAAAAAAAIKLLFCAA|2058-05-23|1900|8265|634|2058|4|5|23|2|2058|634|8265|Thursday|2058Q2|N|N|N|2472850|2472969|2472507|2472782|N|N|N|N|N| +2472873|AAAAAAAAJKLLFCAA|2058-05-24|1900|8265|634|2058|5|5|24|2|2058|634|8265|Friday|2058Q2|N|Y|N|2472850|2472969|2472508|2472783|N|N|N|N|N| +2472874|AAAAAAAAKKLLFCAA|2058-05-25|1900|8265|634|2058|6|5|25|2|2058|634|8265|Saturday|2058Q2|N|Y|N|2472850|2472969|2472509|2472784|N|N|N|N|N| +2472875|AAAAAAAALKLLFCAA|2058-05-26|1900|8265|634|2058|0|5|26|2|2058|634|8265|Sunday|2058Q2|N|N|N|2472850|2472969|2472510|2472785|N|N|N|N|N| +2472876|AAAAAAAAMKLLFCAA|2058-05-27|1900|8265|634|2058|1|5|27|2|2058|634|8265|Monday|2058Q2|N|N|N|2472850|2472969|2472511|2472786|N|N|N|N|N| +2472877|AAAAAAAANKLLFCAA|2058-05-28|1900|8266|634|2058|2|5|28|2|2058|634|8266|Tuesday|2058Q2|N|N|N|2472850|2472969|2472512|2472787|N|N|N|N|N| +2472878|AAAAAAAAOKLLFCAA|2058-05-29|1900|8266|634|2058|3|5|29|2|2058|634|8266|Wednesday|2058Q2|N|N|N|2472850|2472969|2472513|2472788|N|N|N|N|N| +2472879|AAAAAAAAPKLLFCAA|2058-05-30|1900|8266|634|2058|4|5|30|2|2058|634|8266|Thursday|2058Q2|N|N|N|2472850|2472969|2472514|2472789|N|N|N|N|N| +2472880|AAAAAAAAALLLFCAA|2058-05-31|1900|8266|634|2058|5|5|31|2|2058|634|8266|Friday|2058Q2|N|Y|N|2472850|2472969|2472515|2472790|N|N|N|N|N| +2472881|AAAAAAAABLLLFCAA|2058-06-01|1901|8266|635|2058|6|6|1|2|2058|635|8266|Saturday|2058Q2|N|Y|N|2472881|2473031|2472516|2472791|N|N|N|N|N| +2472882|AAAAAAAACLLLFCAA|2058-06-02|1901|8266|635|2058|0|6|2|2|2058|635|8266|Sunday|2058Q2|N|N|N|2472881|2473031|2472517|2472792|N|N|N|N|N| +2472883|AAAAAAAADLLLFCAA|2058-06-03|1901|8266|635|2058|1|6|3|2|2058|635|8266|Monday|2058Q2|N|N|N|2472881|2473031|2472518|2472793|N|N|N|N|N| +2472884|AAAAAAAAELLLFCAA|2058-06-04|1901|8267|635|2058|2|6|4|2|2058|635|8267|Tuesday|2058Q2|N|N|N|2472881|2473031|2472519|2472794|N|N|N|N|N| +2472885|AAAAAAAAFLLLFCAA|2058-06-05|1901|8267|635|2058|3|6|5|2|2058|635|8267|Wednesday|2058Q2|N|N|N|2472881|2473031|2472520|2472795|N|N|N|N|N| +2472886|AAAAAAAAGLLLFCAA|2058-06-06|1901|8267|635|2058|4|6|6|2|2058|635|8267|Thursday|2058Q2|N|N|N|2472881|2473031|2472521|2472796|N|N|N|N|N| +2472887|AAAAAAAAHLLLFCAA|2058-06-07|1901|8267|635|2058|5|6|7|2|2058|635|8267|Friday|2058Q2|N|Y|N|2472881|2473031|2472522|2472797|N|N|N|N|N| +2472888|AAAAAAAAILLLFCAA|2058-06-08|1901|8267|635|2058|6|6|8|2|2058|635|8267|Saturday|2058Q2|N|Y|N|2472881|2473031|2472523|2472798|N|N|N|N|N| +2472889|AAAAAAAAJLLLFCAA|2058-06-09|1901|8267|635|2058|0|6|9|2|2058|635|8267|Sunday|2058Q2|N|N|N|2472881|2473031|2472524|2472799|N|N|N|N|N| +2472890|AAAAAAAAKLLLFCAA|2058-06-10|1901|8267|635|2058|1|6|10|2|2058|635|8267|Monday|2058Q2|N|N|N|2472881|2473031|2472525|2472800|N|N|N|N|N| +2472891|AAAAAAAALLLLFCAA|2058-06-11|1901|8268|635|2058|2|6|11|2|2058|635|8268|Tuesday|2058Q2|N|N|N|2472881|2473031|2472526|2472801|N|N|N|N|N| +2472892|AAAAAAAAMLLLFCAA|2058-06-12|1901|8268|635|2058|3|6|12|2|2058|635|8268|Wednesday|2058Q2|N|N|N|2472881|2473031|2472527|2472802|N|N|N|N|N| +2472893|AAAAAAAANLLLFCAA|2058-06-13|1901|8268|635|2058|4|6|13|2|2058|635|8268|Thursday|2058Q2|N|N|N|2472881|2473031|2472528|2472803|N|N|N|N|N| +2472894|AAAAAAAAOLLLFCAA|2058-06-14|1901|8268|635|2058|5|6|14|2|2058|635|8268|Friday|2058Q2|N|Y|N|2472881|2473031|2472529|2472804|N|N|N|N|N| +2472895|AAAAAAAAPLLLFCAA|2058-06-15|1901|8268|635|2058|6|6|15|2|2058|635|8268|Saturday|2058Q2|N|Y|N|2472881|2473031|2472530|2472805|N|N|N|N|N| +2472896|AAAAAAAAAMLLFCAA|2058-06-16|1901|8268|635|2058|0|6|16|2|2058|635|8268|Sunday|2058Q2|N|N|N|2472881|2473031|2472531|2472806|N|N|N|N|N| +2472897|AAAAAAAABMLLFCAA|2058-06-17|1901|8268|635|2058|1|6|17|2|2058|635|8268|Monday|2058Q2|N|N|N|2472881|2473031|2472532|2472807|N|N|N|N|N| +2472898|AAAAAAAACMLLFCAA|2058-06-18|1901|8269|635|2058|2|6|18|2|2058|635|8269|Tuesday|2058Q2|N|N|N|2472881|2473031|2472533|2472808|N|N|N|N|N| +2472899|AAAAAAAADMLLFCAA|2058-06-19|1901|8269|635|2058|3|6|19|2|2058|635|8269|Wednesday|2058Q2|N|N|N|2472881|2473031|2472534|2472809|N|N|N|N|N| +2472900|AAAAAAAAEMLLFCAA|2058-06-20|1901|8269|635|2058|4|6|20|2|2058|635|8269|Thursday|2058Q2|N|N|N|2472881|2473031|2472535|2472810|N|N|N|N|N| +2472901|AAAAAAAAFMLLFCAA|2058-06-21|1901|8269|635|2058|5|6|21|2|2058|635|8269|Friday|2058Q2|N|Y|N|2472881|2473031|2472536|2472811|N|N|N|N|N| +2472902|AAAAAAAAGMLLFCAA|2058-06-22|1901|8269|635|2058|6|6|22|2|2058|635|8269|Saturday|2058Q2|N|Y|N|2472881|2473031|2472537|2472812|N|N|N|N|N| +2472903|AAAAAAAAHMLLFCAA|2058-06-23|1901|8269|635|2058|0|6|23|2|2058|635|8269|Sunday|2058Q2|N|N|N|2472881|2473031|2472538|2472813|N|N|N|N|N| +2472904|AAAAAAAAIMLLFCAA|2058-06-24|1901|8269|635|2058|1|6|24|2|2058|635|8269|Monday|2058Q2|N|N|N|2472881|2473031|2472539|2472814|N|N|N|N|N| +2472905|AAAAAAAAJMLLFCAA|2058-06-25|1901|8270|635|2058|2|6|25|2|2058|635|8270|Tuesday|2058Q2|N|N|N|2472881|2473031|2472540|2472815|N|N|N|N|N| +2472906|AAAAAAAAKMLLFCAA|2058-06-26|1901|8270|635|2058|3|6|26|2|2058|635|8270|Wednesday|2058Q2|N|N|N|2472881|2473031|2472541|2472816|N|N|N|N|N| +2472907|AAAAAAAALMLLFCAA|2058-06-27|1901|8270|635|2058|4|6|27|2|2058|635|8270|Thursday|2058Q2|N|N|N|2472881|2473031|2472542|2472817|N|N|N|N|N| +2472908|AAAAAAAAMMLLFCAA|2058-06-28|1901|8270|635|2058|5|6|28|2|2058|635|8270|Friday|2058Q2|N|Y|N|2472881|2473031|2472543|2472818|N|N|N|N|N| +2472909|AAAAAAAANMLLFCAA|2058-06-29|1901|8270|635|2058|6|6|29|2|2058|635|8270|Saturday|2058Q2|N|Y|N|2472881|2473031|2472544|2472819|N|N|N|N|N| +2472910|AAAAAAAAOMLLFCAA|2058-06-30|1901|8270|635|2058|0|6|30|2|2058|635|8270|Sunday|2058Q2|N|N|N|2472881|2473031|2472545|2472820|N|N|N|N|N| +2472911|AAAAAAAAPMLLFCAA|2058-07-01|1902|8270|635|2058|1|7|1|2|2058|635|8270|Monday|2058Q2|N|N|N|2472911|2473091|2472546|2472820|N|N|N|N|N| +2472912|AAAAAAAAANLLFCAA|2058-07-02|1902|8271|635|2058|2|7|2|3|2058|635|8271|Tuesday|2058Q3|N|N|N|2472911|2473091|2472547|2472821|N|N|N|N|N| +2472913|AAAAAAAABNLLFCAA|2058-07-03|1902|8271|635|2058|3|7|3|3|2058|635|8271|Wednesday|2058Q3|N|N|N|2472911|2473091|2472548|2472822|N|N|N|N|N| +2472914|AAAAAAAACNLLFCAA|2058-07-04|1902|8271|635|2058|4|7|4|3|2058|635|8271|Thursday|2058Q3|N|N|N|2472911|2473091|2472549|2472823|N|N|N|N|N| +2472915|AAAAAAAADNLLFCAA|2058-07-05|1902|8271|635|2058|5|7|5|3|2058|635|8271|Friday|2058Q3|Y|Y|N|2472911|2473091|2472550|2472824|N|N|N|N|N| +2472916|AAAAAAAAENLLFCAA|2058-07-06|1902|8271|635|2058|6|7|6|3|2058|635|8271|Saturday|2058Q3|N|Y|Y|2472911|2473091|2472551|2472825|N|N|N|N|N| +2472917|AAAAAAAAFNLLFCAA|2058-07-07|1902|8271|635|2058|0|7|7|3|2058|635|8271|Sunday|2058Q3|N|N|N|2472911|2473091|2472552|2472826|N|N|N|N|N| +2472918|AAAAAAAAGNLLFCAA|2058-07-08|1902|8271|635|2058|1|7|8|3|2058|635|8271|Monday|2058Q3|N|N|N|2472911|2473091|2472553|2472827|N|N|N|N|N| +2472919|AAAAAAAAHNLLFCAA|2058-07-09|1902|8272|635|2058|2|7|9|3|2058|635|8272|Tuesday|2058Q3|N|N|N|2472911|2473091|2472554|2472828|N|N|N|N|N| +2472920|AAAAAAAAINLLFCAA|2058-07-10|1902|8272|635|2058|3|7|10|3|2058|635|8272|Wednesday|2058Q3|N|N|N|2472911|2473091|2472555|2472829|N|N|N|N|N| +2472921|AAAAAAAAJNLLFCAA|2058-07-11|1902|8272|635|2058|4|7|11|3|2058|635|8272|Thursday|2058Q3|N|N|N|2472911|2473091|2472556|2472830|N|N|N|N|N| +2472922|AAAAAAAAKNLLFCAA|2058-07-12|1902|8272|635|2058|5|7|12|3|2058|635|8272|Friday|2058Q3|N|Y|N|2472911|2473091|2472557|2472831|N|N|N|N|N| +2472923|AAAAAAAALNLLFCAA|2058-07-13|1902|8272|635|2058|6|7|13|3|2058|635|8272|Saturday|2058Q3|N|Y|N|2472911|2473091|2472558|2472832|N|N|N|N|N| +2472924|AAAAAAAAMNLLFCAA|2058-07-14|1902|8272|635|2058|0|7|14|3|2058|635|8272|Sunday|2058Q3|N|N|N|2472911|2473091|2472559|2472833|N|N|N|N|N| +2472925|AAAAAAAANNLLFCAA|2058-07-15|1902|8272|635|2058|1|7|15|3|2058|635|8272|Monday|2058Q3|N|N|N|2472911|2473091|2472560|2472834|N|N|N|N|N| +2472926|AAAAAAAAONLLFCAA|2058-07-16|1902|8273|635|2058|2|7|16|3|2058|635|8273|Tuesday|2058Q3|N|N|N|2472911|2473091|2472561|2472835|N|N|N|N|N| +2472927|AAAAAAAAPNLLFCAA|2058-07-17|1902|8273|635|2058|3|7|17|3|2058|635|8273|Wednesday|2058Q3|N|N|N|2472911|2473091|2472562|2472836|N|N|N|N|N| +2472928|AAAAAAAAAOLLFCAA|2058-07-18|1902|8273|635|2058|4|7|18|3|2058|635|8273|Thursday|2058Q3|N|N|N|2472911|2473091|2472563|2472837|N|N|N|N|N| +2472929|AAAAAAAABOLLFCAA|2058-07-19|1902|8273|635|2058|5|7|19|3|2058|635|8273|Friday|2058Q3|N|Y|N|2472911|2473091|2472564|2472838|N|N|N|N|N| +2472930|AAAAAAAACOLLFCAA|2058-07-20|1902|8273|635|2058|6|7|20|3|2058|635|8273|Saturday|2058Q3|N|Y|N|2472911|2473091|2472565|2472839|N|N|N|N|N| +2472931|AAAAAAAADOLLFCAA|2058-07-21|1902|8273|635|2058|0|7|21|3|2058|635|8273|Sunday|2058Q3|N|N|N|2472911|2473091|2472566|2472840|N|N|N|N|N| +2472932|AAAAAAAAEOLLFCAA|2058-07-22|1902|8273|635|2058|1|7|22|3|2058|635|8273|Monday|2058Q3|N|N|N|2472911|2473091|2472567|2472841|N|N|N|N|N| +2472933|AAAAAAAAFOLLFCAA|2058-07-23|1902|8274|635|2058|2|7|23|3|2058|635|8274|Tuesday|2058Q3|N|N|N|2472911|2473091|2472568|2472842|N|N|N|N|N| +2472934|AAAAAAAAGOLLFCAA|2058-07-24|1902|8274|635|2058|3|7|24|3|2058|635|8274|Wednesday|2058Q3|N|N|N|2472911|2473091|2472569|2472843|N|N|N|N|N| +2472935|AAAAAAAAHOLLFCAA|2058-07-25|1902|8274|635|2058|4|7|25|3|2058|635|8274|Thursday|2058Q3|N|N|N|2472911|2473091|2472570|2472844|N|N|N|N|N| +2472936|AAAAAAAAIOLLFCAA|2058-07-26|1902|8274|635|2058|5|7|26|3|2058|635|8274|Friday|2058Q3|N|Y|N|2472911|2473091|2472571|2472845|N|N|N|N|N| +2472937|AAAAAAAAJOLLFCAA|2058-07-27|1902|8274|635|2058|6|7|27|3|2058|635|8274|Saturday|2058Q3|N|Y|N|2472911|2473091|2472572|2472846|N|N|N|N|N| +2472938|AAAAAAAAKOLLFCAA|2058-07-28|1902|8274|635|2058|0|7|28|3|2058|635|8274|Sunday|2058Q3|N|N|N|2472911|2473091|2472573|2472847|N|N|N|N|N| +2472939|AAAAAAAALOLLFCAA|2058-07-29|1902|8274|635|2058|1|7|29|3|2058|635|8274|Monday|2058Q3|N|N|N|2472911|2473091|2472574|2472848|N|N|N|N|N| +2472940|AAAAAAAAMOLLFCAA|2058-07-30|1902|8275|635|2058|2|7|30|3|2058|635|8275|Tuesday|2058Q3|N|N|N|2472911|2473091|2472575|2472849|N|N|N|N|N| +2472941|AAAAAAAANOLLFCAA|2058-07-31|1902|8275|635|2058|3|7|31|3|2058|635|8275|Wednesday|2058Q3|N|N|N|2472911|2473091|2472576|2472850|N|N|N|N|N| +2472942|AAAAAAAAOOLLFCAA|2058-08-01|1903|8275|635|2058|4|8|1|3|2058|635|8275|Thursday|2058Q3|N|N|N|2472942|2473153|2472577|2472851|N|N|N|N|N| +2472943|AAAAAAAAPOLLFCAA|2058-08-02|1903|8275|635|2058|5|8|2|3|2058|635|8275|Friday|2058Q3|N|Y|N|2472942|2473153|2472578|2472852|N|N|N|N|N| +2472944|AAAAAAAAAPLLFCAA|2058-08-03|1903|8275|635|2058|6|8|3|3|2058|635|8275|Saturday|2058Q3|N|Y|N|2472942|2473153|2472579|2472853|N|N|N|N|N| +2472945|AAAAAAAABPLLFCAA|2058-08-04|1903|8275|635|2058|0|8|4|3|2058|635|8275|Sunday|2058Q3|N|N|N|2472942|2473153|2472580|2472854|N|N|N|N|N| +2472946|AAAAAAAACPLLFCAA|2058-08-05|1903|8275|635|2058|1|8|5|3|2058|635|8275|Monday|2058Q3|N|N|N|2472942|2473153|2472581|2472855|N|N|N|N|N| +2472947|AAAAAAAADPLLFCAA|2058-08-06|1903|8276|635|2058|2|8|6|3|2058|635|8276|Tuesday|2058Q3|N|N|N|2472942|2473153|2472582|2472856|N|N|N|N|N| +2472948|AAAAAAAAEPLLFCAA|2058-08-07|1903|8276|635|2058|3|8|7|3|2058|635|8276|Wednesday|2058Q3|N|N|N|2472942|2473153|2472583|2472857|N|N|N|N|N| +2472949|AAAAAAAAFPLLFCAA|2058-08-08|1903|8276|635|2058|4|8|8|3|2058|635|8276|Thursday|2058Q3|N|N|N|2472942|2473153|2472584|2472858|N|N|N|N|N| +2472950|AAAAAAAAGPLLFCAA|2058-08-09|1903|8276|635|2058|5|8|9|3|2058|635|8276|Friday|2058Q3|N|Y|N|2472942|2473153|2472585|2472859|N|N|N|N|N| +2472951|AAAAAAAAHPLLFCAA|2058-08-10|1903|8276|635|2058|6|8|10|3|2058|635|8276|Saturday|2058Q3|N|Y|N|2472942|2473153|2472586|2472860|N|N|N|N|N| +2472952|AAAAAAAAIPLLFCAA|2058-08-11|1903|8276|635|2058|0|8|11|3|2058|635|8276|Sunday|2058Q3|N|N|N|2472942|2473153|2472587|2472861|N|N|N|N|N| +2472953|AAAAAAAAJPLLFCAA|2058-08-12|1903|8276|635|2058|1|8|12|3|2058|635|8276|Monday|2058Q3|N|N|N|2472942|2473153|2472588|2472862|N|N|N|N|N| +2472954|AAAAAAAAKPLLFCAA|2058-08-13|1903|8277|635|2058|2|8|13|3|2058|635|8277|Tuesday|2058Q3|N|N|N|2472942|2473153|2472589|2472863|N|N|N|N|N| +2472955|AAAAAAAALPLLFCAA|2058-08-14|1903|8277|635|2058|3|8|14|3|2058|635|8277|Wednesday|2058Q3|N|N|N|2472942|2473153|2472590|2472864|N|N|N|N|N| +2472956|AAAAAAAAMPLLFCAA|2058-08-15|1903|8277|635|2058|4|8|15|3|2058|635|8277|Thursday|2058Q3|N|N|N|2472942|2473153|2472591|2472865|N|N|N|N|N| +2472957|AAAAAAAANPLLFCAA|2058-08-16|1903|8277|635|2058|5|8|16|3|2058|635|8277|Friday|2058Q3|N|Y|N|2472942|2473153|2472592|2472866|N|N|N|N|N| +2472958|AAAAAAAAOPLLFCAA|2058-08-17|1903|8277|635|2058|6|8|17|3|2058|635|8277|Saturday|2058Q3|N|Y|N|2472942|2473153|2472593|2472867|N|N|N|N|N| +2472959|AAAAAAAAPPLLFCAA|2058-08-18|1903|8277|635|2058|0|8|18|3|2058|635|8277|Sunday|2058Q3|N|N|N|2472942|2473153|2472594|2472868|N|N|N|N|N| +2472960|AAAAAAAAAAMLFCAA|2058-08-19|1903|8277|635|2058|1|8|19|3|2058|635|8277|Monday|2058Q3|N|N|N|2472942|2473153|2472595|2472869|N|N|N|N|N| +2472961|AAAAAAAABAMLFCAA|2058-08-20|1903|8278|635|2058|2|8|20|3|2058|635|8278|Tuesday|2058Q3|N|N|N|2472942|2473153|2472596|2472870|N|N|N|N|N| +2472962|AAAAAAAACAMLFCAA|2058-08-21|1903|8278|635|2058|3|8|21|3|2058|635|8278|Wednesday|2058Q3|N|N|N|2472942|2473153|2472597|2472871|N|N|N|N|N| +2472963|AAAAAAAADAMLFCAA|2058-08-22|1903|8278|635|2058|4|8|22|3|2058|635|8278|Thursday|2058Q3|N|N|N|2472942|2473153|2472598|2472872|N|N|N|N|N| +2472964|AAAAAAAAEAMLFCAA|2058-08-23|1903|8278|635|2058|5|8|23|3|2058|635|8278|Friday|2058Q3|N|Y|N|2472942|2473153|2472599|2472873|N|N|N|N|N| +2472965|AAAAAAAAFAMLFCAA|2058-08-24|1903|8278|635|2058|6|8|24|3|2058|635|8278|Saturday|2058Q3|N|Y|N|2472942|2473153|2472600|2472874|N|N|N|N|N| +2472966|AAAAAAAAGAMLFCAA|2058-08-25|1903|8278|635|2058|0|8|25|3|2058|635|8278|Sunday|2058Q3|N|N|N|2472942|2473153|2472601|2472875|N|N|N|N|N| +2472967|AAAAAAAAHAMLFCAA|2058-08-26|1903|8278|635|2058|1|8|26|3|2058|635|8278|Monday|2058Q3|N|N|N|2472942|2473153|2472602|2472876|N|N|N|N|N| +2472968|AAAAAAAAIAMLFCAA|2058-08-27|1903|8279|635|2058|2|8|27|3|2058|635|8279|Tuesday|2058Q3|N|N|N|2472942|2473153|2472603|2472877|N|N|N|N|N| +2472969|AAAAAAAAJAMLFCAA|2058-08-28|1903|8279|635|2058|3|8|28|3|2058|635|8279|Wednesday|2058Q3|N|N|N|2472942|2473153|2472604|2472878|N|N|N|N|N| +2472970|AAAAAAAAKAMLFCAA|2058-08-29|1903|8279|635|2058|4|8|29|3|2058|635|8279|Thursday|2058Q3|N|N|N|2472942|2473153|2472605|2472879|N|N|N|N|N| +2472971|AAAAAAAALAMLFCAA|2058-08-30|1903|8279|635|2058|5|8|30|3|2058|635|8279|Friday|2058Q3|N|Y|N|2472942|2473153|2472606|2472880|N|N|N|N|N| +2472972|AAAAAAAAMAMLFCAA|2058-08-31|1903|8279|635|2058|6|8|31|3|2058|635|8279|Saturday|2058Q3|N|Y|N|2472942|2473153|2472607|2472881|N|N|N|N|N| +2472973|AAAAAAAANAMLFCAA|2058-09-01|1904|8279|636|2058|0|9|1|3|2058|636|8279|Sunday|2058Q3|N|N|N|2472973|2473215|2472608|2472882|N|N|N|N|N| +2472974|AAAAAAAAOAMLFCAA|2058-09-02|1904|8279|636|2058|1|9|2|3|2058|636|8279|Monday|2058Q3|N|N|N|2472973|2473215|2472609|2472883|N|N|N|N|N| +2472975|AAAAAAAAPAMLFCAA|2058-09-03|1904|8280|636|2058|2|9|3|3|2058|636|8280|Tuesday|2058Q3|N|N|N|2472973|2473215|2472610|2472884|N|N|N|N|N| +2472976|AAAAAAAAABMLFCAA|2058-09-04|1904|8280|636|2058|3|9|4|3|2058|636|8280|Wednesday|2058Q3|N|N|N|2472973|2473215|2472611|2472885|N|N|N|N|N| +2472977|AAAAAAAABBMLFCAA|2058-09-05|1904|8280|636|2058|4|9|5|3|2058|636|8280|Thursday|2058Q3|N|N|N|2472973|2473215|2472612|2472886|N|N|N|N|N| +2472978|AAAAAAAACBMLFCAA|2058-09-06|1904|8280|636|2058|5|9|6|3|2058|636|8280|Friday|2058Q3|N|Y|N|2472973|2473215|2472613|2472887|N|N|N|N|N| +2472979|AAAAAAAADBMLFCAA|2058-09-07|1904|8280|636|2058|6|9|7|3|2058|636|8280|Saturday|2058Q3|N|Y|N|2472973|2473215|2472614|2472888|N|N|N|N|N| +2472980|AAAAAAAAEBMLFCAA|2058-09-08|1904|8280|636|2058|0|9|8|3|2058|636|8280|Sunday|2058Q3|N|N|N|2472973|2473215|2472615|2472889|N|N|N|N|N| +2472981|AAAAAAAAFBMLFCAA|2058-09-09|1904|8280|636|2058|1|9|9|3|2058|636|8280|Monday|2058Q3|N|N|N|2472973|2473215|2472616|2472890|N|N|N|N|N| +2472982|AAAAAAAAGBMLFCAA|2058-09-10|1904|8281|636|2058|2|9|10|3|2058|636|8281|Tuesday|2058Q3|N|N|N|2472973|2473215|2472617|2472891|N|N|N|N|N| +2472983|AAAAAAAAHBMLFCAA|2058-09-11|1904|8281|636|2058|3|9|11|3|2058|636|8281|Wednesday|2058Q3|N|N|N|2472973|2473215|2472618|2472892|N|N|N|N|N| +2472984|AAAAAAAAIBMLFCAA|2058-09-12|1904|8281|636|2058|4|9|12|3|2058|636|8281|Thursday|2058Q3|N|N|N|2472973|2473215|2472619|2472893|N|N|N|N|N| +2472985|AAAAAAAAJBMLFCAA|2058-09-13|1904|8281|636|2058|5|9|13|3|2058|636|8281|Friday|2058Q3|N|Y|N|2472973|2473215|2472620|2472894|N|N|N|N|N| +2472986|AAAAAAAAKBMLFCAA|2058-09-14|1904|8281|636|2058|6|9|14|3|2058|636|8281|Saturday|2058Q3|N|Y|N|2472973|2473215|2472621|2472895|N|N|N|N|N| +2472987|AAAAAAAALBMLFCAA|2058-09-15|1904|8281|636|2058|0|9|15|3|2058|636|8281|Sunday|2058Q3|N|N|N|2472973|2473215|2472622|2472896|N|N|N|N|N| +2472988|AAAAAAAAMBMLFCAA|2058-09-16|1904|8281|636|2058|1|9|16|3|2058|636|8281|Monday|2058Q3|N|N|N|2472973|2473215|2472623|2472897|N|N|N|N|N| +2472989|AAAAAAAANBMLFCAA|2058-09-17|1904|8282|636|2058|2|9|17|3|2058|636|8282|Tuesday|2058Q3|N|N|N|2472973|2473215|2472624|2472898|N|N|N|N|N| +2472990|AAAAAAAAOBMLFCAA|2058-09-18|1904|8282|636|2058|3|9|18|3|2058|636|8282|Wednesday|2058Q3|N|N|N|2472973|2473215|2472625|2472899|N|N|N|N|N| +2472991|AAAAAAAAPBMLFCAA|2058-09-19|1904|8282|636|2058|4|9|19|3|2058|636|8282|Thursday|2058Q3|N|N|N|2472973|2473215|2472626|2472900|N|N|N|N|N| +2472992|AAAAAAAAACMLFCAA|2058-09-20|1904|8282|636|2058|5|9|20|3|2058|636|8282|Friday|2058Q3|N|Y|N|2472973|2473215|2472627|2472901|N|N|N|N|N| +2472993|AAAAAAAABCMLFCAA|2058-09-21|1904|8282|636|2058|6|9|21|3|2058|636|8282|Saturday|2058Q3|N|Y|N|2472973|2473215|2472628|2472902|N|N|N|N|N| +2472994|AAAAAAAACCMLFCAA|2058-09-22|1904|8282|636|2058|0|9|22|3|2058|636|8282|Sunday|2058Q3|N|N|N|2472973|2473215|2472629|2472903|N|N|N|N|N| +2472995|AAAAAAAADCMLFCAA|2058-09-23|1904|8282|636|2058|1|9|23|3|2058|636|8282|Monday|2058Q3|N|N|N|2472973|2473215|2472630|2472904|N|N|N|N|N| +2472996|AAAAAAAAECMLFCAA|2058-09-24|1904|8283|636|2058|2|9|24|3|2058|636|8283|Tuesday|2058Q3|N|N|N|2472973|2473215|2472631|2472905|N|N|N|N|N| +2472997|AAAAAAAAFCMLFCAA|2058-09-25|1904|8283|636|2058|3|9|25|3|2058|636|8283|Wednesday|2058Q3|N|N|N|2472973|2473215|2472632|2472906|N|N|N|N|N| +2472998|AAAAAAAAGCMLFCAA|2058-09-26|1904|8283|636|2058|4|9|26|3|2058|636|8283|Thursday|2058Q3|N|N|N|2472973|2473215|2472633|2472907|N|N|N|N|N| +2472999|AAAAAAAAHCMLFCAA|2058-09-27|1904|8283|636|2058|5|9|27|3|2058|636|8283|Friday|2058Q3|N|Y|N|2472973|2473215|2472634|2472908|N|N|N|N|N| +2473000|AAAAAAAAICMLFCAA|2058-09-28|1904|8283|636|2058|6|9|28|3|2058|636|8283|Saturday|2058Q3|N|Y|N|2472973|2473215|2472635|2472909|N|N|N|N|N| +2473001|AAAAAAAAJCMLFCAA|2058-09-29|1904|8283|636|2058|0|9|29|3|2058|636|8283|Sunday|2058Q3|N|N|N|2472973|2473215|2472636|2472910|N|N|N|N|N| +2473002|AAAAAAAAKCMLFCAA|2058-09-30|1904|8283|636|2058|1|9|30|3|2058|636|8283|Monday|2058Q3|N|N|N|2472973|2473215|2472637|2472911|N|N|N|N|N| +2473003|AAAAAAAALCMLFCAA|2058-10-01|1905|8284|636|2058|2|10|1|3|2058|636|8284|Tuesday|2058Q3|N|N|N|2473003|2473275|2472638|2472911|N|N|N|N|N| +2473004|AAAAAAAAMCMLFCAA|2058-10-02|1905|8284|636|2058|3|10|2|4|2058|636|8284|Wednesday|2058Q4|N|N|N|2473003|2473275|2472639|2472912|N|N|N|N|N| +2473005|AAAAAAAANCMLFCAA|2058-10-03|1905|8284|636|2058|4|10|3|4|2058|636|8284|Thursday|2058Q4|N|N|N|2473003|2473275|2472640|2472913|N|N|N|N|N| +2473006|AAAAAAAAOCMLFCAA|2058-10-04|1905|8284|636|2058|5|10|4|4|2058|636|8284|Friday|2058Q4|N|Y|N|2473003|2473275|2472641|2472914|N|N|N|N|N| +2473007|AAAAAAAAPCMLFCAA|2058-10-05|1905|8284|636|2058|6|10|5|4|2058|636|8284|Saturday|2058Q4|N|Y|N|2473003|2473275|2472642|2472915|N|N|N|N|N| +2473008|AAAAAAAAADMLFCAA|2058-10-06|1905|8284|636|2058|0|10|6|4|2058|636|8284|Sunday|2058Q4|N|N|N|2473003|2473275|2472643|2472916|N|N|N|N|N| +2473009|AAAAAAAABDMLFCAA|2058-10-07|1905|8284|636|2058|1|10|7|4|2058|636|8284|Monday|2058Q4|N|N|N|2473003|2473275|2472644|2472917|N|N|N|N|N| +2473010|AAAAAAAACDMLFCAA|2058-10-08|1905|8285|636|2058|2|10|8|4|2058|636|8285|Tuesday|2058Q4|N|N|N|2473003|2473275|2472645|2472918|N|N|N|N|N| +2473011|AAAAAAAADDMLFCAA|2058-10-09|1905|8285|636|2058|3|10|9|4|2058|636|8285|Wednesday|2058Q4|N|N|N|2473003|2473275|2472646|2472919|N|N|N|N|N| +2473012|AAAAAAAAEDMLFCAA|2058-10-10|1905|8285|636|2058|4|10|10|4|2058|636|8285|Thursday|2058Q4|N|N|N|2473003|2473275|2472647|2472920|N|N|N|N|N| +2473013|AAAAAAAAFDMLFCAA|2058-10-11|1905|8285|636|2058|5|10|11|4|2058|636|8285|Friday|2058Q4|N|Y|N|2473003|2473275|2472648|2472921|N|N|N|N|N| +2473014|AAAAAAAAGDMLFCAA|2058-10-12|1905|8285|636|2058|6|10|12|4|2058|636|8285|Saturday|2058Q4|N|Y|N|2473003|2473275|2472649|2472922|N|N|N|N|N| +2473015|AAAAAAAAHDMLFCAA|2058-10-13|1905|8285|636|2058|0|10|13|4|2058|636|8285|Sunday|2058Q4|N|N|N|2473003|2473275|2472650|2472923|N|N|N|N|N| +2473016|AAAAAAAAIDMLFCAA|2058-10-14|1905|8285|636|2058|1|10|14|4|2058|636|8285|Monday|2058Q4|N|N|N|2473003|2473275|2472651|2472924|N|N|N|N|N| +2473017|AAAAAAAAJDMLFCAA|2058-10-15|1905|8286|636|2058|2|10|15|4|2058|636|8286|Tuesday|2058Q4|N|N|N|2473003|2473275|2472652|2472925|N|N|N|N|N| +2473018|AAAAAAAAKDMLFCAA|2058-10-16|1905|8286|636|2058|3|10|16|4|2058|636|8286|Wednesday|2058Q4|N|N|N|2473003|2473275|2472653|2472926|N|N|N|N|N| +2473019|AAAAAAAALDMLFCAA|2058-10-17|1905|8286|636|2058|4|10|17|4|2058|636|8286|Thursday|2058Q4|N|N|N|2473003|2473275|2472654|2472927|N|N|N|N|N| +2473020|AAAAAAAAMDMLFCAA|2058-10-18|1905|8286|636|2058|5|10|18|4|2058|636|8286|Friday|2058Q4|N|Y|N|2473003|2473275|2472655|2472928|N|N|N|N|N| +2473021|AAAAAAAANDMLFCAA|2058-10-19|1905|8286|636|2058|6|10|19|4|2058|636|8286|Saturday|2058Q4|N|Y|N|2473003|2473275|2472656|2472929|N|N|N|N|N| +2473022|AAAAAAAAODMLFCAA|2058-10-20|1905|8286|636|2058|0|10|20|4|2058|636|8286|Sunday|2058Q4|N|N|N|2473003|2473275|2472657|2472930|N|N|N|N|N| +2473023|AAAAAAAAPDMLFCAA|2058-10-21|1905|8286|636|2058|1|10|21|4|2058|636|8286|Monday|2058Q4|N|N|N|2473003|2473275|2472658|2472931|N|N|N|N|N| +2473024|AAAAAAAAAEMLFCAA|2058-10-22|1905|8287|636|2058|2|10|22|4|2058|636|8287|Tuesday|2058Q4|N|N|N|2473003|2473275|2472659|2472932|N|N|N|N|N| +2473025|AAAAAAAABEMLFCAA|2058-10-23|1905|8287|636|2058|3|10|23|4|2058|636|8287|Wednesday|2058Q4|N|N|N|2473003|2473275|2472660|2472933|N|N|N|N|N| +2473026|AAAAAAAACEMLFCAA|2058-10-24|1905|8287|636|2058|4|10|24|4|2058|636|8287|Thursday|2058Q4|N|N|N|2473003|2473275|2472661|2472934|N|N|N|N|N| +2473027|AAAAAAAADEMLFCAA|2058-10-25|1905|8287|636|2058|5|10|25|4|2058|636|8287|Friday|2058Q4|N|Y|N|2473003|2473275|2472662|2472935|N|N|N|N|N| +2473028|AAAAAAAAEEMLFCAA|2058-10-26|1905|8287|636|2058|6|10|26|4|2058|636|8287|Saturday|2058Q4|N|Y|N|2473003|2473275|2472663|2472936|N|N|N|N|N| +2473029|AAAAAAAAFEMLFCAA|2058-10-27|1905|8287|636|2058|0|10|27|4|2058|636|8287|Sunday|2058Q4|N|N|N|2473003|2473275|2472664|2472937|N|N|N|N|N| +2473030|AAAAAAAAGEMLFCAA|2058-10-28|1905|8287|636|2058|1|10|28|4|2058|636|8287|Monday|2058Q4|N|N|N|2473003|2473275|2472665|2472938|N|N|N|N|N| +2473031|AAAAAAAAHEMLFCAA|2058-10-29|1905|8288|636|2058|2|10|29|4|2058|636|8288|Tuesday|2058Q4|N|N|N|2473003|2473275|2472666|2472939|N|N|N|N|N| +2473032|AAAAAAAAIEMLFCAA|2058-10-30|1905|8288|636|2058|3|10|30|4|2058|636|8288|Wednesday|2058Q4|N|N|N|2473003|2473275|2472667|2472940|N|N|N|N|N| +2473033|AAAAAAAAJEMLFCAA|2058-10-31|1905|8288|636|2058|4|10|31|4|2058|636|8288|Thursday|2058Q4|N|N|N|2473003|2473275|2472668|2472941|N|N|N|N|N| +2473034|AAAAAAAAKEMLFCAA|2058-11-01|1906|8288|636|2058|5|11|1|4|2058|636|8288|Friday|2058Q4|N|Y|N|2473034|2473337|2472669|2472942|N|N|N|N|N| +2473035|AAAAAAAALEMLFCAA|2058-11-02|1906|8288|636|2058|6|11|2|4|2058|636|8288|Saturday|2058Q4|N|Y|N|2473034|2473337|2472670|2472943|N|N|N|N|N| +2473036|AAAAAAAAMEMLFCAA|2058-11-03|1906|8288|636|2058|0|11|3|4|2058|636|8288|Sunday|2058Q4|N|N|N|2473034|2473337|2472671|2472944|N|N|N|N|N| +2473037|AAAAAAAANEMLFCAA|2058-11-04|1906|8288|636|2058|1|11|4|4|2058|636|8288|Monday|2058Q4|N|N|N|2473034|2473337|2472672|2472945|N|N|N|N|N| +2473038|AAAAAAAAOEMLFCAA|2058-11-05|1906|8289|636|2058|2|11|5|4|2058|636|8289|Tuesday|2058Q4|N|N|N|2473034|2473337|2472673|2472946|N|N|N|N|N| +2473039|AAAAAAAAPEMLFCAA|2058-11-06|1906|8289|636|2058|3|11|6|4|2058|636|8289|Wednesday|2058Q4|N|N|N|2473034|2473337|2472674|2472947|N|N|N|N|N| +2473040|AAAAAAAAAFMLFCAA|2058-11-07|1906|8289|636|2058|4|11|7|4|2058|636|8289|Thursday|2058Q4|N|N|N|2473034|2473337|2472675|2472948|N|N|N|N|N| +2473041|AAAAAAAABFMLFCAA|2058-11-08|1906|8289|636|2058|5|11|8|4|2058|636|8289|Friday|2058Q4|N|Y|N|2473034|2473337|2472676|2472949|N|N|N|N|N| +2473042|AAAAAAAACFMLFCAA|2058-11-09|1906|8289|636|2058|6|11|9|4|2058|636|8289|Saturday|2058Q4|N|Y|N|2473034|2473337|2472677|2472950|N|N|N|N|N| +2473043|AAAAAAAADFMLFCAA|2058-11-10|1906|8289|636|2058|0|11|10|4|2058|636|8289|Sunday|2058Q4|N|N|N|2473034|2473337|2472678|2472951|N|N|N|N|N| +2473044|AAAAAAAAEFMLFCAA|2058-11-11|1906|8289|636|2058|1|11|11|4|2058|636|8289|Monday|2058Q4|N|N|N|2473034|2473337|2472679|2472952|N|N|N|N|N| +2473045|AAAAAAAAFFMLFCAA|2058-11-12|1906|8290|636|2058|2|11|12|4|2058|636|8290|Tuesday|2058Q4|N|N|N|2473034|2473337|2472680|2472953|N|N|N|N|N| +2473046|AAAAAAAAGFMLFCAA|2058-11-13|1906|8290|636|2058|3|11|13|4|2058|636|8290|Wednesday|2058Q4|N|N|N|2473034|2473337|2472681|2472954|N|N|N|N|N| +2473047|AAAAAAAAHFMLFCAA|2058-11-14|1906|8290|636|2058|4|11|14|4|2058|636|8290|Thursday|2058Q4|N|N|N|2473034|2473337|2472682|2472955|N|N|N|N|N| +2473048|AAAAAAAAIFMLFCAA|2058-11-15|1906|8290|636|2058|5|11|15|4|2058|636|8290|Friday|2058Q4|N|Y|N|2473034|2473337|2472683|2472956|N|N|N|N|N| +2473049|AAAAAAAAJFMLFCAA|2058-11-16|1906|8290|636|2058|6|11|16|4|2058|636|8290|Saturday|2058Q4|N|Y|N|2473034|2473337|2472684|2472957|N|N|N|N|N| +2473050|AAAAAAAAKFMLFCAA|2058-11-17|1906|8290|636|2058|0|11|17|4|2058|636|8290|Sunday|2058Q4|N|N|N|2473034|2473337|2472685|2472958|N|N|N|N|N| +2473051|AAAAAAAALFMLFCAA|2058-11-18|1906|8290|636|2058|1|11|18|4|2058|636|8290|Monday|2058Q4|N|N|N|2473034|2473337|2472686|2472959|N|N|N|N|N| +2473052|AAAAAAAAMFMLFCAA|2058-11-19|1906|8291|636|2058|2|11|19|4|2058|636|8291|Tuesday|2058Q4|N|N|N|2473034|2473337|2472687|2472960|N|N|N|N|N| +2473053|AAAAAAAANFMLFCAA|2058-11-20|1906|8291|636|2058|3|11|20|4|2058|636|8291|Wednesday|2058Q4|N|N|N|2473034|2473337|2472688|2472961|N|N|N|N|N| +2473054|AAAAAAAAOFMLFCAA|2058-11-21|1906|8291|636|2058|4|11|21|4|2058|636|8291|Thursday|2058Q4|N|N|N|2473034|2473337|2472689|2472962|N|N|N|N|N| +2473055|AAAAAAAAPFMLFCAA|2058-11-22|1906|8291|636|2058|5|11|22|4|2058|636|8291|Friday|2058Q4|N|Y|N|2473034|2473337|2472690|2472963|N|N|N|N|N| +2473056|AAAAAAAAAGMLFCAA|2058-11-23|1906|8291|636|2058|6|11|23|4|2058|636|8291|Saturday|2058Q4|N|Y|N|2473034|2473337|2472691|2472964|N|N|N|N|N| +2473057|AAAAAAAABGMLFCAA|2058-11-24|1906|8291|636|2058|0|11|24|4|2058|636|8291|Sunday|2058Q4|N|N|N|2473034|2473337|2472692|2472965|N|N|N|N|N| +2473058|AAAAAAAACGMLFCAA|2058-11-25|1906|8291|636|2058|1|11|25|4|2058|636|8291|Monday|2058Q4|N|N|N|2473034|2473337|2472693|2472966|N|N|N|N|N| +2473059|AAAAAAAADGMLFCAA|2058-11-26|1906|8292|636|2058|2|11|26|4|2058|636|8292|Tuesday|2058Q4|N|N|N|2473034|2473337|2472694|2472967|N|N|N|N|N| +2473060|AAAAAAAAEGMLFCAA|2058-11-27|1906|8292|636|2058|3|11|27|4|2058|636|8292|Wednesday|2058Q4|N|N|N|2473034|2473337|2472695|2472968|N|N|N|N|N| +2473061|AAAAAAAAFGMLFCAA|2058-11-28|1906|8292|636|2058|4|11|28|4|2058|636|8292|Thursday|2058Q4|N|N|N|2473034|2473337|2472696|2472969|N|N|N|N|N| +2473062|AAAAAAAAGGMLFCAA|2058-11-29|1906|8292|636|2058|5|11|29|4|2058|636|8292|Friday|2058Q4|N|Y|N|2473034|2473337|2472697|2472970|N|N|N|N|N| +2473063|AAAAAAAAHGMLFCAA|2058-11-30|1906|8292|636|2058|6|11|30|4|2058|636|8292|Saturday|2058Q4|N|Y|N|2473034|2473337|2472698|2472971|N|N|N|N|N| +2473064|AAAAAAAAIGMLFCAA|2058-12-01|1907|8292|637|2058|0|12|1|4|2058|637|8292|Sunday|2058Q4|N|N|N|2473064|2473397|2472699|2472972|N|N|N|N|N| +2473065|AAAAAAAAJGMLFCAA|2058-12-02|1907|8292|637|2058|1|12|2|4|2058|637|8292|Monday|2058Q4|N|N|N|2473064|2473397|2472700|2472973|N|N|N|N|N| +2473066|AAAAAAAAKGMLFCAA|2058-12-03|1907|8293|637|2058|2|12|3|4|2058|637|8293|Tuesday|2058Q4|N|N|N|2473064|2473397|2472701|2472974|N|N|N|N|N| +2473067|AAAAAAAALGMLFCAA|2058-12-04|1907|8293|637|2058|3|12|4|4|2058|637|8293|Wednesday|2058Q4|N|N|N|2473064|2473397|2472702|2472975|N|N|N|N|N| +2473068|AAAAAAAAMGMLFCAA|2058-12-05|1907|8293|637|2058|4|12|5|4|2058|637|8293|Thursday|2058Q4|N|N|N|2473064|2473397|2472703|2472976|N|N|N|N|N| +2473069|AAAAAAAANGMLFCAA|2058-12-06|1907|8293|637|2058|5|12|6|4|2058|637|8293|Friday|2058Q4|N|Y|N|2473064|2473397|2472704|2472977|N|N|N|N|N| +2473070|AAAAAAAAOGMLFCAA|2058-12-07|1907|8293|637|2058|6|12|7|4|2058|637|8293|Saturday|2058Q4|N|Y|N|2473064|2473397|2472705|2472978|N|N|N|N|N| +2473071|AAAAAAAAPGMLFCAA|2058-12-08|1907|8293|637|2058|0|12|8|4|2058|637|8293|Sunday|2058Q4|N|N|N|2473064|2473397|2472706|2472979|N|N|N|N|N| +2473072|AAAAAAAAAHMLFCAA|2058-12-09|1907|8293|637|2058|1|12|9|4|2058|637|8293|Monday|2058Q4|N|N|N|2473064|2473397|2472707|2472980|N|N|N|N|N| +2473073|AAAAAAAABHMLFCAA|2058-12-10|1907|8294|637|2058|2|12|10|4|2058|637|8294|Tuesday|2058Q4|N|N|N|2473064|2473397|2472708|2472981|N|N|N|N|N| +2473074|AAAAAAAACHMLFCAA|2058-12-11|1907|8294|637|2058|3|12|11|4|2058|637|8294|Wednesday|2058Q4|N|N|N|2473064|2473397|2472709|2472982|N|N|N|N|N| +2473075|AAAAAAAADHMLFCAA|2058-12-12|1907|8294|637|2058|4|12|12|4|2058|637|8294|Thursday|2058Q4|N|N|N|2473064|2473397|2472710|2472983|N|N|N|N|N| +2473076|AAAAAAAAEHMLFCAA|2058-12-13|1907|8294|637|2058|5|12|13|4|2058|637|8294|Friday|2058Q4|N|Y|N|2473064|2473397|2472711|2472984|N|N|N|N|N| +2473077|AAAAAAAAFHMLFCAA|2058-12-14|1907|8294|637|2058|6|12|14|4|2058|637|8294|Saturday|2058Q4|N|Y|N|2473064|2473397|2472712|2472985|N|N|N|N|N| +2473078|AAAAAAAAGHMLFCAA|2058-12-15|1907|8294|637|2058|0|12|15|4|2058|637|8294|Sunday|2058Q4|N|N|N|2473064|2473397|2472713|2472986|N|N|N|N|N| +2473079|AAAAAAAAHHMLFCAA|2058-12-16|1907|8294|637|2058|1|12|16|4|2058|637|8294|Monday|2058Q4|N|N|N|2473064|2473397|2472714|2472987|N|N|N|N|N| +2473080|AAAAAAAAIHMLFCAA|2058-12-17|1907|8295|637|2058|2|12|17|4|2058|637|8295|Tuesday|2058Q4|N|N|N|2473064|2473397|2472715|2472988|N|N|N|N|N| +2473081|AAAAAAAAJHMLFCAA|2058-12-18|1907|8295|637|2058|3|12|18|4|2058|637|8295|Wednesday|2058Q4|N|N|N|2473064|2473397|2472716|2472989|N|N|N|N|N| +2473082|AAAAAAAAKHMLFCAA|2058-12-19|1907|8295|637|2058|4|12|19|4|2058|637|8295|Thursday|2058Q4|N|N|N|2473064|2473397|2472717|2472990|N|N|N|N|N| +2473083|AAAAAAAALHMLFCAA|2058-12-20|1907|8295|637|2058|5|12|20|4|2058|637|8295|Friday|2058Q4|N|Y|N|2473064|2473397|2472718|2472991|N|N|N|N|N| +2473084|AAAAAAAAMHMLFCAA|2058-12-21|1907|8295|637|2058|6|12|21|4|2058|637|8295|Saturday|2058Q4|N|Y|N|2473064|2473397|2472719|2472992|N|N|N|N|N| +2473085|AAAAAAAANHMLFCAA|2058-12-22|1907|8295|637|2058|0|12|22|4|2058|637|8295|Sunday|2058Q4|N|N|N|2473064|2473397|2472720|2472993|N|N|N|N|N| +2473086|AAAAAAAAOHMLFCAA|2058-12-23|1907|8295|637|2058|1|12|23|4|2058|637|8295|Monday|2058Q4|N|N|N|2473064|2473397|2472721|2472994|N|N|N|N|N| +2473087|AAAAAAAAPHMLFCAA|2058-12-24|1907|8296|637|2058|2|12|24|4|2058|637|8296|Tuesday|2058Q4|N|N|N|2473064|2473397|2472722|2472995|N|N|N|N|N| +2473088|AAAAAAAAAIMLFCAA|2058-12-25|1907|8296|637|2058|3|12|25|4|2058|637|8296|Wednesday|2058Q4|N|N|N|2473064|2473397|2472723|2472996|N|N|N|N|N| +2473089|AAAAAAAABIMLFCAA|2058-12-26|1907|8296|637|2058|4|12|26|4|2058|637|8296|Thursday|2058Q4|Y|N|N|2473064|2473397|2472724|2472997|N|N|N|N|N| +2473090|AAAAAAAACIMLFCAA|2058-12-27|1907|8296|637|2058|5|12|27|4|2058|637|8296|Friday|2058Q4|N|Y|Y|2473064|2473397|2472725|2472998|N|N|N|N|N| +2473091|AAAAAAAADIMLFCAA|2058-12-28|1907|8296|637|2058|6|12|28|4|2058|637|8296|Saturday|2058Q4|N|Y|N|2473064|2473397|2472726|2472999|N|N|N|N|N| +2473092|AAAAAAAAEIMLFCAA|2058-12-29|1907|8296|637|2058|0|12|29|4|2058|637|8296|Sunday|2058Q4|N|N|N|2473064|2473397|2472727|2473000|N|N|N|N|N| +2473093|AAAAAAAAFIMLFCAA|2058-12-30|1907|8296|637|2058|1|12|30|4|2058|637|8296|Monday|2058Q4|N|N|N|2473064|2473397|2472728|2473001|N|N|N|N|N| +2473094|AAAAAAAAGIMLFCAA|2058-12-31|1907|8297|637|2058|2|12|31|4|2058|637|8297|Tuesday|2058Q4|N|N|N|2473064|2473397|2472729|2473002|N|N|N|N|N| +2473095|AAAAAAAAHIMLFCAA|2059-01-01|1908|8297|637|2059|3|1|1|1|2059|637|8297|Wednesday|2059Q1|Y|N|N|2473095|2473094|2472730|2473003|N|N|N|N|N| +2473096|AAAAAAAAIIMLFCAA|2059-01-02|1908|8297|637|2059|4|1|2|1|2059|637|8297|Thursday|2059Q1|N|N|Y|2473095|2473094|2472731|2473004|N|N|N|N|N| +2473097|AAAAAAAAJIMLFCAA|2059-01-03|1908|8297|637|2059|5|1|3|1|2059|637|8297|Friday|2059Q1|N|Y|N|2473095|2473094|2472732|2473005|N|N|N|N|N| +2473098|AAAAAAAAKIMLFCAA|2059-01-04|1908|8297|637|2059|6|1|4|1|2059|637|8297|Saturday|2059Q1|N|Y|N|2473095|2473094|2472733|2473006|N|N|N|N|N| +2473099|AAAAAAAALIMLFCAA|2059-01-05|1908|8297|637|2059|0|1|5|1|2059|637|8297|Sunday|2059Q1|N|N|N|2473095|2473094|2472734|2473007|N|N|N|N|N| +2473100|AAAAAAAAMIMLFCAA|2059-01-06|1908|8297|637|2059|1|1|6|1|2059|637|8297|Monday|2059Q1|N|N|N|2473095|2473094|2472735|2473008|N|N|N|N|N| +2473101|AAAAAAAANIMLFCAA|2059-01-07|1908|8298|637|2059|2|1|7|1|2059|637|8298|Tuesday|2059Q1|N|N|N|2473095|2473094|2472736|2473009|N|N|N|N|N| +2473102|AAAAAAAAOIMLFCAA|2059-01-08|1908|8298|637|2059|3|1|8|1|2059|637|8298|Wednesday|2059Q1|N|N|N|2473095|2473094|2472737|2473010|N|N|N|N|N| +2473103|AAAAAAAAPIMLFCAA|2059-01-09|1908|8298|637|2059|4|1|9|1|2059|637|8298|Thursday|2059Q1|N|N|N|2473095|2473094|2472738|2473011|N|N|N|N|N| +2473104|AAAAAAAAAJMLFCAA|2059-01-10|1908|8298|637|2059|5|1|10|1|2059|637|8298|Friday|2059Q1|N|Y|N|2473095|2473094|2472739|2473012|N|N|N|N|N| +2473105|AAAAAAAABJMLFCAA|2059-01-11|1908|8298|637|2059|6|1|11|1|2059|637|8298|Saturday|2059Q1|N|Y|N|2473095|2473094|2472740|2473013|N|N|N|N|N| +2473106|AAAAAAAACJMLFCAA|2059-01-12|1908|8298|637|2059|0|1|12|1|2059|637|8298|Sunday|2059Q1|N|N|N|2473095|2473094|2472741|2473014|N|N|N|N|N| +2473107|AAAAAAAADJMLFCAA|2059-01-13|1908|8298|637|2059|1|1|13|1|2059|637|8298|Monday|2059Q1|N|N|N|2473095|2473094|2472742|2473015|N|N|N|N|N| +2473108|AAAAAAAAEJMLFCAA|2059-01-14|1908|8299|637|2059|2|1|14|1|2059|637|8299|Tuesday|2059Q1|N|N|N|2473095|2473094|2472743|2473016|N|N|N|N|N| +2473109|AAAAAAAAFJMLFCAA|2059-01-15|1908|8299|637|2059|3|1|15|1|2059|637|8299|Wednesday|2059Q1|N|N|N|2473095|2473094|2472744|2473017|N|N|N|N|N| +2473110|AAAAAAAAGJMLFCAA|2059-01-16|1908|8299|637|2059|4|1|16|1|2059|637|8299|Thursday|2059Q1|N|N|N|2473095|2473094|2472745|2473018|N|N|N|N|N| +2473111|AAAAAAAAHJMLFCAA|2059-01-17|1908|8299|637|2059|5|1|17|1|2059|637|8299|Friday|2059Q1|N|Y|N|2473095|2473094|2472746|2473019|N|N|N|N|N| +2473112|AAAAAAAAIJMLFCAA|2059-01-18|1908|8299|637|2059|6|1|18|1|2059|637|8299|Saturday|2059Q1|N|Y|N|2473095|2473094|2472747|2473020|N|N|N|N|N| +2473113|AAAAAAAAJJMLFCAA|2059-01-19|1908|8299|637|2059|0|1|19|1|2059|637|8299|Sunday|2059Q1|N|N|N|2473095|2473094|2472748|2473021|N|N|N|N|N| +2473114|AAAAAAAAKJMLFCAA|2059-01-20|1908|8299|637|2059|1|1|20|1|2059|637|8299|Monday|2059Q1|N|N|N|2473095|2473094|2472749|2473022|N|N|N|N|N| +2473115|AAAAAAAALJMLFCAA|2059-01-21|1908|8300|637|2059|2|1|21|1|2059|637|8300|Tuesday|2059Q1|N|N|N|2473095|2473094|2472750|2473023|N|N|N|N|N| +2473116|AAAAAAAAMJMLFCAA|2059-01-22|1908|8300|637|2059|3|1|22|1|2059|637|8300|Wednesday|2059Q1|N|N|N|2473095|2473094|2472751|2473024|N|N|N|N|N| +2473117|AAAAAAAANJMLFCAA|2059-01-23|1908|8300|637|2059|4|1|23|1|2059|637|8300|Thursday|2059Q1|N|N|N|2473095|2473094|2472752|2473025|N|N|N|N|N| +2473118|AAAAAAAAOJMLFCAA|2059-01-24|1908|8300|637|2059|5|1|24|1|2059|637|8300|Friday|2059Q1|N|Y|N|2473095|2473094|2472753|2473026|N|N|N|N|N| +2473119|AAAAAAAAPJMLFCAA|2059-01-25|1908|8300|637|2059|6|1|25|1|2059|637|8300|Saturday|2059Q1|N|Y|N|2473095|2473094|2472754|2473027|N|N|N|N|N| +2473120|AAAAAAAAAKMLFCAA|2059-01-26|1908|8300|637|2059|0|1|26|1|2059|637|8300|Sunday|2059Q1|N|N|N|2473095|2473094|2472755|2473028|N|N|N|N|N| +2473121|AAAAAAAABKMLFCAA|2059-01-27|1908|8300|637|2059|1|1|27|1|2059|637|8300|Monday|2059Q1|N|N|N|2473095|2473094|2472756|2473029|N|N|N|N|N| +2473122|AAAAAAAACKMLFCAA|2059-01-28|1908|8301|637|2059|2|1|28|1|2059|637|8301|Tuesday|2059Q1|N|N|N|2473095|2473094|2472757|2473030|N|N|N|N|N| +2473123|AAAAAAAADKMLFCAA|2059-01-29|1908|8301|637|2059|3|1|29|1|2059|637|8301|Wednesday|2059Q1|N|N|N|2473095|2473094|2472758|2473031|N|N|N|N|N| +2473124|AAAAAAAAEKMLFCAA|2059-01-30|1908|8301|637|2059|4|1|30|1|2059|637|8301|Thursday|2059Q1|N|N|N|2473095|2473094|2472759|2473032|N|N|N|N|N| +2473125|AAAAAAAAFKMLFCAA|2059-01-31|1908|8301|637|2059|5|1|31|1|2059|637|8301|Friday|2059Q1|N|Y|N|2473095|2473094|2472760|2473033|N|N|N|N|N| +2473126|AAAAAAAAGKMLFCAA|2059-02-01|1909|8301|637|2059|6|2|1|1|2059|637|8301|Saturday|2059Q1|N|Y|N|2473126|2473156|2472761|2473034|N|N|N|N|N| +2473127|AAAAAAAAHKMLFCAA|2059-02-02|1909|8301|637|2059|0|2|2|1|2059|637|8301|Sunday|2059Q1|N|N|N|2473126|2473156|2472762|2473035|N|N|N|N|N| +2473128|AAAAAAAAIKMLFCAA|2059-02-03|1909|8301|637|2059|1|2|3|1|2059|637|8301|Monday|2059Q1|N|N|N|2473126|2473156|2472763|2473036|N|N|N|N|N| +2473129|AAAAAAAAJKMLFCAA|2059-02-04|1909|8302|637|2059|2|2|4|1|2059|637|8302|Tuesday|2059Q1|N|N|N|2473126|2473156|2472764|2473037|N|N|N|N|N| +2473130|AAAAAAAAKKMLFCAA|2059-02-05|1909|8302|637|2059|3|2|5|1|2059|637|8302|Wednesday|2059Q1|N|N|N|2473126|2473156|2472765|2473038|N|N|N|N|N| +2473131|AAAAAAAALKMLFCAA|2059-02-06|1909|8302|637|2059|4|2|6|1|2059|637|8302|Thursday|2059Q1|N|N|N|2473126|2473156|2472766|2473039|N|N|N|N|N| +2473132|AAAAAAAAMKMLFCAA|2059-02-07|1909|8302|637|2059|5|2|7|1|2059|637|8302|Friday|2059Q1|N|Y|N|2473126|2473156|2472767|2473040|N|N|N|N|N| +2473133|AAAAAAAANKMLFCAA|2059-02-08|1909|8302|637|2059|6|2|8|1|2059|637|8302|Saturday|2059Q1|N|Y|N|2473126|2473156|2472768|2473041|N|N|N|N|N| +2473134|AAAAAAAAOKMLFCAA|2059-02-09|1909|8302|637|2059|0|2|9|1|2059|637|8302|Sunday|2059Q1|N|N|N|2473126|2473156|2472769|2473042|N|N|N|N|N| +2473135|AAAAAAAAPKMLFCAA|2059-02-10|1909|8302|637|2059|1|2|10|1|2059|637|8302|Monday|2059Q1|N|N|N|2473126|2473156|2472770|2473043|N|N|N|N|N| +2473136|AAAAAAAAALMLFCAA|2059-02-11|1909|8303|637|2059|2|2|11|1|2059|637|8303|Tuesday|2059Q1|N|N|N|2473126|2473156|2472771|2473044|N|N|N|N|N| +2473137|AAAAAAAABLMLFCAA|2059-02-12|1909|8303|637|2059|3|2|12|1|2059|637|8303|Wednesday|2059Q1|N|N|N|2473126|2473156|2472772|2473045|N|N|N|N|N| +2473138|AAAAAAAACLMLFCAA|2059-02-13|1909|8303|637|2059|4|2|13|1|2059|637|8303|Thursday|2059Q1|N|N|N|2473126|2473156|2472773|2473046|N|N|N|N|N| +2473139|AAAAAAAADLMLFCAA|2059-02-14|1909|8303|637|2059|5|2|14|1|2059|637|8303|Friday|2059Q1|N|Y|N|2473126|2473156|2472774|2473047|N|N|N|N|N| +2473140|AAAAAAAAELMLFCAA|2059-02-15|1909|8303|637|2059|6|2|15|1|2059|637|8303|Saturday|2059Q1|N|Y|N|2473126|2473156|2472775|2473048|N|N|N|N|N| +2473141|AAAAAAAAFLMLFCAA|2059-02-16|1909|8303|637|2059|0|2|16|1|2059|637|8303|Sunday|2059Q1|N|N|N|2473126|2473156|2472776|2473049|N|N|N|N|N| +2473142|AAAAAAAAGLMLFCAA|2059-02-17|1909|8303|637|2059|1|2|17|1|2059|637|8303|Monday|2059Q1|N|N|N|2473126|2473156|2472777|2473050|N|N|N|N|N| +2473143|AAAAAAAAHLMLFCAA|2059-02-18|1909|8304|637|2059|2|2|18|1|2059|637|8304|Tuesday|2059Q1|N|N|N|2473126|2473156|2472778|2473051|N|N|N|N|N| +2473144|AAAAAAAAILMLFCAA|2059-02-19|1909|8304|637|2059|3|2|19|1|2059|637|8304|Wednesday|2059Q1|N|N|N|2473126|2473156|2472779|2473052|N|N|N|N|N| +2473145|AAAAAAAAJLMLFCAA|2059-02-20|1909|8304|637|2059|4|2|20|1|2059|637|8304|Thursday|2059Q1|N|N|N|2473126|2473156|2472780|2473053|N|N|N|N|N| +2473146|AAAAAAAAKLMLFCAA|2059-02-21|1909|8304|637|2059|5|2|21|1|2059|637|8304|Friday|2059Q1|N|Y|N|2473126|2473156|2472781|2473054|N|N|N|N|N| +2473147|AAAAAAAALLMLFCAA|2059-02-22|1909|8304|637|2059|6|2|22|1|2059|637|8304|Saturday|2059Q1|N|Y|N|2473126|2473156|2472782|2473055|N|N|N|N|N| +2473148|AAAAAAAAMLMLFCAA|2059-02-23|1909|8304|637|2059|0|2|23|1|2059|637|8304|Sunday|2059Q1|N|N|N|2473126|2473156|2472783|2473056|N|N|N|N|N| +2473149|AAAAAAAANLMLFCAA|2059-02-24|1909|8304|637|2059|1|2|24|1|2059|637|8304|Monday|2059Q1|N|N|N|2473126|2473156|2472784|2473057|N|N|N|N|N| +2473150|AAAAAAAAOLMLFCAA|2059-02-25|1909|8305|637|2059|2|2|25|1|2059|637|8305|Tuesday|2059Q1|N|N|N|2473126|2473156|2472785|2473058|N|N|N|N|N| +2473151|AAAAAAAAPLMLFCAA|2059-02-26|1909|8305|637|2059|3|2|26|1|2059|637|8305|Wednesday|2059Q1|N|N|N|2473126|2473156|2472786|2473059|N|N|N|N|N| +2473152|AAAAAAAAAMMLFCAA|2059-02-27|1909|8305|637|2059|4|2|27|1|2059|637|8305|Thursday|2059Q1|N|N|N|2473126|2473156|2472787|2473060|N|N|N|N|N| +2473153|AAAAAAAABMMLFCAA|2059-02-28|1909|8305|637|2059|5|2|28|1|2059|637|8305|Friday|2059Q1|N|Y|N|2473126|2473156|2472788|2473061|N|N|N|N|N| +2473154|AAAAAAAACMMLFCAA|2059-03-01|1910|8305|638|2059|6|3|1|1|2059|638|8305|Saturday|2059Q1|N|Y|N|2473154|2473212|2472789|2473062|N|N|N|N|N| +2473155|AAAAAAAADMMLFCAA|2059-03-02|1910|8305|638|2059|0|3|2|1|2059|638|8305|Sunday|2059Q1|N|N|N|2473154|2473212|2472790|2473063|N|N|N|N|N| +2473156|AAAAAAAAEMMLFCAA|2059-03-03|1910|8305|638|2059|1|3|3|1|2059|638|8305|Monday|2059Q1|N|N|N|2473154|2473212|2472791|2473064|N|N|N|N|N| +2473157|AAAAAAAAFMMLFCAA|2059-03-04|1910|8306|638|2059|2|3|4|1|2059|638|8306|Tuesday|2059Q1|N|N|N|2473154|2473212|2472792|2473065|N|N|N|N|N| +2473158|AAAAAAAAGMMLFCAA|2059-03-05|1910|8306|638|2059|3|3|5|1|2059|638|8306|Wednesday|2059Q1|N|N|N|2473154|2473212|2472793|2473066|N|N|N|N|N| +2473159|AAAAAAAAHMMLFCAA|2059-03-06|1910|8306|638|2059|4|3|6|1|2059|638|8306|Thursday|2059Q1|N|N|N|2473154|2473212|2472794|2473067|N|N|N|N|N| +2473160|AAAAAAAAIMMLFCAA|2059-03-07|1910|8306|638|2059|5|3|7|1|2059|638|8306|Friday|2059Q1|N|Y|N|2473154|2473212|2472795|2473068|N|N|N|N|N| +2473161|AAAAAAAAJMMLFCAA|2059-03-08|1910|8306|638|2059|6|3|8|1|2059|638|8306|Saturday|2059Q1|N|Y|N|2473154|2473212|2472796|2473069|N|N|N|N|N| +2473162|AAAAAAAAKMMLFCAA|2059-03-09|1910|8306|638|2059|0|3|9|1|2059|638|8306|Sunday|2059Q1|N|N|N|2473154|2473212|2472797|2473070|N|N|N|N|N| +2473163|AAAAAAAALMMLFCAA|2059-03-10|1910|8306|638|2059|1|3|10|1|2059|638|8306|Monday|2059Q1|N|N|N|2473154|2473212|2472798|2473071|N|N|N|N|N| +2473164|AAAAAAAAMMMLFCAA|2059-03-11|1910|8307|638|2059|2|3|11|1|2059|638|8307|Tuesday|2059Q1|N|N|N|2473154|2473212|2472799|2473072|N|N|N|N|N| +2473165|AAAAAAAANMMLFCAA|2059-03-12|1910|8307|638|2059|3|3|12|1|2059|638|8307|Wednesday|2059Q1|N|N|N|2473154|2473212|2472800|2473073|N|N|N|N|N| +2473166|AAAAAAAAOMMLFCAA|2059-03-13|1910|8307|638|2059|4|3|13|1|2059|638|8307|Thursday|2059Q1|N|N|N|2473154|2473212|2472801|2473074|N|N|N|N|N| +2473167|AAAAAAAAPMMLFCAA|2059-03-14|1910|8307|638|2059|5|3|14|1|2059|638|8307|Friday|2059Q1|N|Y|N|2473154|2473212|2472802|2473075|N|N|N|N|N| +2473168|AAAAAAAAANMLFCAA|2059-03-15|1910|8307|638|2059|6|3|15|1|2059|638|8307|Saturday|2059Q1|N|Y|N|2473154|2473212|2472803|2473076|N|N|N|N|N| +2473169|AAAAAAAABNMLFCAA|2059-03-16|1910|8307|638|2059|0|3|16|1|2059|638|8307|Sunday|2059Q1|N|N|N|2473154|2473212|2472804|2473077|N|N|N|N|N| +2473170|AAAAAAAACNMLFCAA|2059-03-17|1910|8307|638|2059|1|3|17|1|2059|638|8307|Monday|2059Q1|N|N|N|2473154|2473212|2472805|2473078|N|N|N|N|N| +2473171|AAAAAAAADNMLFCAA|2059-03-18|1910|8308|638|2059|2|3|18|1|2059|638|8308|Tuesday|2059Q1|N|N|N|2473154|2473212|2472806|2473079|N|N|N|N|N| +2473172|AAAAAAAAENMLFCAA|2059-03-19|1910|8308|638|2059|3|3|19|1|2059|638|8308|Wednesday|2059Q1|N|N|N|2473154|2473212|2472807|2473080|N|N|N|N|N| +2473173|AAAAAAAAFNMLFCAA|2059-03-20|1910|8308|638|2059|4|3|20|1|2059|638|8308|Thursday|2059Q1|N|N|N|2473154|2473212|2472808|2473081|N|N|N|N|N| +2473174|AAAAAAAAGNMLFCAA|2059-03-21|1910|8308|638|2059|5|3|21|1|2059|638|8308|Friday|2059Q1|N|Y|N|2473154|2473212|2472809|2473082|N|N|N|N|N| +2473175|AAAAAAAAHNMLFCAA|2059-03-22|1910|8308|638|2059|6|3|22|1|2059|638|8308|Saturday|2059Q1|N|Y|N|2473154|2473212|2472810|2473083|N|N|N|N|N| +2473176|AAAAAAAAINMLFCAA|2059-03-23|1910|8308|638|2059|0|3|23|1|2059|638|8308|Sunday|2059Q1|N|N|N|2473154|2473212|2472811|2473084|N|N|N|N|N| +2473177|AAAAAAAAJNMLFCAA|2059-03-24|1910|8308|638|2059|1|3|24|1|2059|638|8308|Monday|2059Q1|N|N|N|2473154|2473212|2472812|2473085|N|N|N|N|N| +2473178|AAAAAAAAKNMLFCAA|2059-03-25|1910|8309|638|2059|2|3|25|1|2059|638|8309|Tuesday|2059Q1|N|N|N|2473154|2473212|2472813|2473086|N|N|N|N|N| +2473179|AAAAAAAALNMLFCAA|2059-03-26|1910|8309|638|2059|3|3|26|1|2059|638|8309|Wednesday|2059Q1|N|N|N|2473154|2473212|2472814|2473087|N|N|N|N|N| +2473180|AAAAAAAAMNMLFCAA|2059-03-27|1910|8309|638|2059|4|3|27|1|2059|638|8309|Thursday|2059Q1|N|N|N|2473154|2473212|2472815|2473088|N|N|N|N|N| +2473181|AAAAAAAANNMLFCAA|2059-03-28|1910|8309|638|2059|5|3|28|1|2059|638|8309|Friday|2059Q1|N|Y|N|2473154|2473212|2472816|2473089|N|N|N|N|N| +2473182|AAAAAAAAONMLFCAA|2059-03-29|1910|8309|638|2059|6|3|29|1|2059|638|8309|Saturday|2059Q1|N|Y|N|2473154|2473212|2472817|2473090|N|N|N|N|N| +2473183|AAAAAAAAPNMLFCAA|2059-03-30|1910|8309|638|2059|0|3|30|1|2059|638|8309|Sunday|2059Q1|N|N|N|2473154|2473212|2472818|2473091|N|N|N|N|N| +2473184|AAAAAAAAAOMLFCAA|2059-03-31|1910|8309|638|2059|1|3|31|1|2059|638|8309|Monday|2059Q1|N|N|N|2473154|2473212|2472819|2473092|N|N|N|N|N| +2473185|AAAAAAAABOMLFCAA|2059-04-01|1911|8310|638|2059|2|4|1|1|2059|638|8310|Tuesday|2059Q1|N|N|N|2473185|2473274|2472820|2473095|N|N|N|N|N| +2473186|AAAAAAAACOMLFCAA|2059-04-02|1911|8310|638|2059|3|4|2|2|2059|638|8310|Wednesday|2059Q2|N|N|N|2473185|2473274|2472821|2473096|N|N|N|N|N| +2473187|AAAAAAAADOMLFCAA|2059-04-03|1911|8310|638|2059|4|4|3|2|2059|638|8310|Thursday|2059Q2|N|N|N|2473185|2473274|2472822|2473097|N|N|N|N|N| +2473188|AAAAAAAAEOMLFCAA|2059-04-04|1911|8310|638|2059|5|4|4|2|2059|638|8310|Friday|2059Q2|N|Y|N|2473185|2473274|2472823|2473098|N|N|N|N|N| +2473189|AAAAAAAAFOMLFCAA|2059-04-05|1911|8310|638|2059|6|4|5|2|2059|638|8310|Saturday|2059Q2|N|Y|N|2473185|2473274|2472824|2473099|N|N|N|N|N| +2473190|AAAAAAAAGOMLFCAA|2059-04-06|1911|8310|638|2059|0|4|6|2|2059|638|8310|Sunday|2059Q2|N|N|N|2473185|2473274|2472825|2473100|N|N|N|N|N| +2473191|AAAAAAAAHOMLFCAA|2059-04-07|1911|8310|638|2059|1|4|7|2|2059|638|8310|Monday|2059Q2|N|N|N|2473185|2473274|2472826|2473101|N|N|N|N|N| +2473192|AAAAAAAAIOMLFCAA|2059-04-08|1911|8311|638|2059|2|4|8|2|2059|638|8311|Tuesday|2059Q2|N|N|N|2473185|2473274|2472827|2473102|N|N|N|N|N| +2473193|AAAAAAAAJOMLFCAA|2059-04-09|1911|8311|638|2059|3|4|9|2|2059|638|8311|Wednesday|2059Q2|N|N|N|2473185|2473274|2472828|2473103|N|N|N|N|N| +2473194|AAAAAAAAKOMLFCAA|2059-04-10|1911|8311|638|2059|4|4|10|2|2059|638|8311|Thursday|2059Q2|N|N|N|2473185|2473274|2472829|2473104|N|N|N|N|N| +2473195|AAAAAAAALOMLFCAA|2059-04-11|1911|8311|638|2059|5|4|11|2|2059|638|8311|Friday|2059Q2|N|Y|N|2473185|2473274|2472830|2473105|N|N|N|N|N| +2473196|AAAAAAAAMOMLFCAA|2059-04-12|1911|8311|638|2059|6|4|12|2|2059|638|8311|Saturday|2059Q2|N|Y|N|2473185|2473274|2472831|2473106|N|N|N|N|N| +2473197|AAAAAAAANOMLFCAA|2059-04-13|1911|8311|638|2059|0|4|13|2|2059|638|8311|Sunday|2059Q2|N|N|N|2473185|2473274|2472832|2473107|N|N|N|N|N| +2473198|AAAAAAAAOOMLFCAA|2059-04-14|1911|8311|638|2059|1|4|14|2|2059|638|8311|Monday|2059Q2|N|N|N|2473185|2473274|2472833|2473108|N|N|N|N|N| +2473199|AAAAAAAAPOMLFCAA|2059-04-15|1911|8312|638|2059|2|4|15|2|2059|638|8312|Tuesday|2059Q2|N|N|N|2473185|2473274|2472834|2473109|N|N|N|N|N| +2473200|AAAAAAAAAPMLFCAA|2059-04-16|1911|8312|638|2059|3|4|16|2|2059|638|8312|Wednesday|2059Q2|N|N|N|2473185|2473274|2472835|2473110|N|N|N|N|N| +2473201|AAAAAAAABPMLFCAA|2059-04-17|1911|8312|638|2059|4|4|17|2|2059|638|8312|Thursday|2059Q2|N|N|N|2473185|2473274|2472836|2473111|N|N|N|N|N| +2473202|AAAAAAAACPMLFCAA|2059-04-18|1911|8312|638|2059|5|4|18|2|2059|638|8312|Friday|2059Q2|N|Y|N|2473185|2473274|2472837|2473112|N|N|N|N|N| +2473203|AAAAAAAADPMLFCAA|2059-04-19|1911|8312|638|2059|6|4|19|2|2059|638|8312|Saturday|2059Q2|N|Y|N|2473185|2473274|2472838|2473113|N|N|N|N|N| +2473204|AAAAAAAAEPMLFCAA|2059-04-20|1911|8312|638|2059|0|4|20|2|2059|638|8312|Sunday|2059Q2|N|N|N|2473185|2473274|2472839|2473114|N|N|N|N|N| +2473205|AAAAAAAAFPMLFCAA|2059-04-21|1911|8312|638|2059|1|4|21|2|2059|638|8312|Monday|2059Q2|N|N|N|2473185|2473274|2472840|2473115|N|N|N|N|N| +2473206|AAAAAAAAGPMLFCAA|2059-04-22|1911|8313|638|2059|2|4|22|2|2059|638|8313|Tuesday|2059Q2|N|N|N|2473185|2473274|2472841|2473116|N|N|N|N|N| +2473207|AAAAAAAAHPMLFCAA|2059-04-23|1911|8313|638|2059|3|4|23|2|2059|638|8313|Wednesday|2059Q2|N|N|N|2473185|2473274|2472842|2473117|N|N|N|N|N| +2473208|AAAAAAAAIPMLFCAA|2059-04-24|1911|8313|638|2059|4|4|24|2|2059|638|8313|Thursday|2059Q2|N|N|N|2473185|2473274|2472843|2473118|N|N|N|N|N| +2473209|AAAAAAAAJPMLFCAA|2059-04-25|1911|8313|638|2059|5|4|25|2|2059|638|8313|Friday|2059Q2|N|Y|N|2473185|2473274|2472844|2473119|N|N|N|N|N| +2473210|AAAAAAAAKPMLFCAA|2059-04-26|1911|8313|638|2059|6|4|26|2|2059|638|8313|Saturday|2059Q2|N|Y|N|2473185|2473274|2472845|2473120|N|N|N|N|N| +2473211|AAAAAAAALPMLFCAA|2059-04-27|1911|8313|638|2059|0|4|27|2|2059|638|8313|Sunday|2059Q2|N|N|N|2473185|2473274|2472846|2473121|N|N|N|N|N| +2473212|AAAAAAAAMPMLFCAA|2059-04-28|1911|8313|638|2059|1|4|28|2|2059|638|8313|Monday|2059Q2|N|N|N|2473185|2473274|2472847|2473122|N|N|N|N|N| +2473213|AAAAAAAANPMLFCAA|2059-04-29|1911|8314|638|2059|2|4|29|2|2059|638|8314|Tuesday|2059Q2|N|N|N|2473185|2473274|2472848|2473123|N|N|N|N|N| +2473214|AAAAAAAAOPMLFCAA|2059-04-30|1911|8314|638|2059|3|4|30|2|2059|638|8314|Wednesday|2059Q2|N|N|N|2473185|2473274|2472849|2473124|N|N|N|N|N| +2473215|AAAAAAAAPPMLFCAA|2059-05-01|1912|8314|638|2059|4|5|1|2|2059|638|8314|Thursday|2059Q2|N|N|N|2473215|2473334|2472850|2473125|N|N|N|N|N| +2473216|AAAAAAAAAANLFCAA|2059-05-02|1912|8314|638|2059|5|5|2|2|2059|638|8314|Friday|2059Q2|N|Y|N|2473215|2473334|2472851|2473126|N|N|N|N|N| +2473217|AAAAAAAABANLFCAA|2059-05-03|1912|8314|638|2059|6|5|3|2|2059|638|8314|Saturday|2059Q2|N|Y|N|2473215|2473334|2472852|2473127|N|N|N|N|N| +2473218|AAAAAAAACANLFCAA|2059-05-04|1912|8314|638|2059|0|5|4|2|2059|638|8314|Sunday|2059Q2|N|N|N|2473215|2473334|2472853|2473128|N|N|N|N|N| +2473219|AAAAAAAADANLFCAA|2059-05-05|1912|8314|638|2059|1|5|5|2|2059|638|8314|Monday|2059Q2|N|N|N|2473215|2473334|2472854|2473129|N|N|N|N|N| +2473220|AAAAAAAAEANLFCAA|2059-05-06|1912|8315|638|2059|2|5|6|2|2059|638|8315|Tuesday|2059Q2|N|N|N|2473215|2473334|2472855|2473130|N|N|N|N|N| +2473221|AAAAAAAAFANLFCAA|2059-05-07|1912|8315|638|2059|3|5|7|2|2059|638|8315|Wednesday|2059Q2|N|N|N|2473215|2473334|2472856|2473131|N|N|N|N|N| +2473222|AAAAAAAAGANLFCAA|2059-05-08|1912|8315|638|2059|4|5|8|2|2059|638|8315|Thursday|2059Q2|N|N|N|2473215|2473334|2472857|2473132|N|N|N|N|N| +2473223|AAAAAAAAHANLFCAA|2059-05-09|1912|8315|638|2059|5|5|9|2|2059|638|8315|Friday|2059Q2|N|Y|N|2473215|2473334|2472858|2473133|N|N|N|N|N| +2473224|AAAAAAAAIANLFCAA|2059-05-10|1912|8315|638|2059|6|5|10|2|2059|638|8315|Saturday|2059Q2|N|Y|N|2473215|2473334|2472859|2473134|N|N|N|N|N| +2473225|AAAAAAAAJANLFCAA|2059-05-11|1912|8315|638|2059|0|5|11|2|2059|638|8315|Sunday|2059Q2|N|N|N|2473215|2473334|2472860|2473135|N|N|N|N|N| +2473226|AAAAAAAAKANLFCAA|2059-05-12|1912|8315|638|2059|1|5|12|2|2059|638|8315|Monday|2059Q2|N|N|N|2473215|2473334|2472861|2473136|N|N|N|N|N| +2473227|AAAAAAAALANLFCAA|2059-05-13|1912|8316|638|2059|2|5|13|2|2059|638|8316|Tuesday|2059Q2|N|N|N|2473215|2473334|2472862|2473137|N|N|N|N|N| +2473228|AAAAAAAAMANLFCAA|2059-05-14|1912|8316|638|2059|3|5|14|2|2059|638|8316|Wednesday|2059Q2|N|N|N|2473215|2473334|2472863|2473138|N|N|N|N|N| +2473229|AAAAAAAANANLFCAA|2059-05-15|1912|8316|638|2059|4|5|15|2|2059|638|8316|Thursday|2059Q2|N|N|N|2473215|2473334|2472864|2473139|N|N|N|N|N| +2473230|AAAAAAAAOANLFCAA|2059-05-16|1912|8316|638|2059|5|5|16|2|2059|638|8316|Friday|2059Q2|N|Y|N|2473215|2473334|2472865|2473140|N|N|N|N|N| +2473231|AAAAAAAAPANLFCAA|2059-05-17|1912|8316|638|2059|6|5|17|2|2059|638|8316|Saturday|2059Q2|N|Y|N|2473215|2473334|2472866|2473141|N|N|N|N|N| +2473232|AAAAAAAAABNLFCAA|2059-05-18|1912|8316|638|2059|0|5|18|2|2059|638|8316|Sunday|2059Q2|N|N|N|2473215|2473334|2472867|2473142|N|N|N|N|N| +2473233|AAAAAAAABBNLFCAA|2059-05-19|1912|8316|638|2059|1|5|19|2|2059|638|8316|Monday|2059Q2|N|N|N|2473215|2473334|2472868|2473143|N|N|N|N|N| +2473234|AAAAAAAACBNLFCAA|2059-05-20|1912|8317|638|2059|2|5|20|2|2059|638|8317|Tuesday|2059Q2|N|N|N|2473215|2473334|2472869|2473144|N|N|N|N|N| +2473235|AAAAAAAADBNLFCAA|2059-05-21|1912|8317|638|2059|3|5|21|2|2059|638|8317|Wednesday|2059Q2|N|N|N|2473215|2473334|2472870|2473145|N|N|N|N|N| +2473236|AAAAAAAAEBNLFCAA|2059-05-22|1912|8317|638|2059|4|5|22|2|2059|638|8317|Thursday|2059Q2|N|N|N|2473215|2473334|2472871|2473146|N|N|N|N|N| +2473237|AAAAAAAAFBNLFCAA|2059-05-23|1912|8317|638|2059|5|5|23|2|2059|638|8317|Friday|2059Q2|N|Y|N|2473215|2473334|2472872|2473147|N|N|N|N|N| +2473238|AAAAAAAAGBNLFCAA|2059-05-24|1912|8317|638|2059|6|5|24|2|2059|638|8317|Saturday|2059Q2|N|Y|N|2473215|2473334|2472873|2473148|N|N|N|N|N| +2473239|AAAAAAAAHBNLFCAA|2059-05-25|1912|8317|638|2059|0|5|25|2|2059|638|8317|Sunday|2059Q2|N|N|N|2473215|2473334|2472874|2473149|N|N|N|N|N| +2473240|AAAAAAAAIBNLFCAA|2059-05-26|1912|8317|638|2059|1|5|26|2|2059|638|8317|Monday|2059Q2|N|N|N|2473215|2473334|2472875|2473150|N|N|N|N|N| +2473241|AAAAAAAAJBNLFCAA|2059-05-27|1912|8318|638|2059|2|5|27|2|2059|638|8318|Tuesday|2059Q2|N|N|N|2473215|2473334|2472876|2473151|N|N|N|N|N| +2473242|AAAAAAAAKBNLFCAA|2059-05-28|1912|8318|638|2059|3|5|28|2|2059|638|8318|Wednesday|2059Q2|N|N|N|2473215|2473334|2472877|2473152|N|N|N|N|N| +2473243|AAAAAAAALBNLFCAA|2059-05-29|1912|8318|638|2059|4|5|29|2|2059|638|8318|Thursday|2059Q2|N|N|N|2473215|2473334|2472878|2473153|N|N|N|N|N| +2473244|AAAAAAAAMBNLFCAA|2059-05-30|1912|8318|638|2059|5|5|30|2|2059|638|8318|Friday|2059Q2|N|Y|N|2473215|2473334|2472879|2473154|N|N|N|N|N| +2473245|AAAAAAAANBNLFCAA|2059-05-31|1912|8318|638|2059|6|5|31|2|2059|638|8318|Saturday|2059Q2|N|Y|N|2473215|2473334|2472880|2473155|N|N|N|N|N| +2473246|AAAAAAAAOBNLFCAA|2059-06-01|1913|8318|639|2059|0|6|1|2|2059|639|8318|Sunday|2059Q2|N|N|N|2473246|2473396|2472881|2473156|N|N|N|N|N| +2473247|AAAAAAAAPBNLFCAA|2059-06-02|1913|8318|639|2059|1|6|2|2|2059|639|8318|Monday|2059Q2|N|N|N|2473246|2473396|2472882|2473157|N|N|N|N|N| +2473248|AAAAAAAAACNLFCAA|2059-06-03|1913|8319|639|2059|2|6|3|2|2059|639|8319|Tuesday|2059Q2|N|N|N|2473246|2473396|2472883|2473158|N|N|N|N|N| +2473249|AAAAAAAABCNLFCAA|2059-06-04|1913|8319|639|2059|3|6|4|2|2059|639|8319|Wednesday|2059Q2|N|N|N|2473246|2473396|2472884|2473159|N|N|N|N|N| +2473250|AAAAAAAACCNLFCAA|2059-06-05|1913|8319|639|2059|4|6|5|2|2059|639|8319|Thursday|2059Q2|N|N|N|2473246|2473396|2472885|2473160|N|N|N|N|N| +2473251|AAAAAAAADCNLFCAA|2059-06-06|1913|8319|639|2059|5|6|6|2|2059|639|8319|Friday|2059Q2|N|Y|N|2473246|2473396|2472886|2473161|N|N|N|N|N| +2473252|AAAAAAAAECNLFCAA|2059-06-07|1913|8319|639|2059|6|6|7|2|2059|639|8319|Saturday|2059Q2|N|Y|N|2473246|2473396|2472887|2473162|N|N|N|N|N| +2473253|AAAAAAAAFCNLFCAA|2059-06-08|1913|8319|639|2059|0|6|8|2|2059|639|8319|Sunday|2059Q2|N|N|N|2473246|2473396|2472888|2473163|N|N|N|N|N| +2473254|AAAAAAAAGCNLFCAA|2059-06-09|1913|8319|639|2059|1|6|9|2|2059|639|8319|Monday|2059Q2|N|N|N|2473246|2473396|2472889|2473164|N|N|N|N|N| +2473255|AAAAAAAAHCNLFCAA|2059-06-10|1913|8320|639|2059|2|6|10|2|2059|639|8320|Tuesday|2059Q2|N|N|N|2473246|2473396|2472890|2473165|N|N|N|N|N| +2473256|AAAAAAAAICNLFCAA|2059-06-11|1913|8320|639|2059|3|6|11|2|2059|639|8320|Wednesday|2059Q2|N|N|N|2473246|2473396|2472891|2473166|N|N|N|N|N| +2473257|AAAAAAAAJCNLFCAA|2059-06-12|1913|8320|639|2059|4|6|12|2|2059|639|8320|Thursday|2059Q2|N|N|N|2473246|2473396|2472892|2473167|N|N|N|N|N| +2473258|AAAAAAAAKCNLFCAA|2059-06-13|1913|8320|639|2059|5|6|13|2|2059|639|8320|Friday|2059Q2|N|Y|N|2473246|2473396|2472893|2473168|N|N|N|N|N| +2473259|AAAAAAAALCNLFCAA|2059-06-14|1913|8320|639|2059|6|6|14|2|2059|639|8320|Saturday|2059Q2|N|Y|N|2473246|2473396|2472894|2473169|N|N|N|N|N| +2473260|AAAAAAAAMCNLFCAA|2059-06-15|1913|8320|639|2059|0|6|15|2|2059|639|8320|Sunday|2059Q2|N|N|N|2473246|2473396|2472895|2473170|N|N|N|N|N| +2473261|AAAAAAAANCNLFCAA|2059-06-16|1913|8320|639|2059|1|6|16|2|2059|639|8320|Monday|2059Q2|N|N|N|2473246|2473396|2472896|2473171|N|N|N|N|N| +2473262|AAAAAAAAOCNLFCAA|2059-06-17|1913|8321|639|2059|2|6|17|2|2059|639|8321|Tuesday|2059Q2|N|N|N|2473246|2473396|2472897|2473172|N|N|N|N|N| +2473263|AAAAAAAAPCNLFCAA|2059-06-18|1913|8321|639|2059|3|6|18|2|2059|639|8321|Wednesday|2059Q2|N|N|N|2473246|2473396|2472898|2473173|N|N|N|N|N| +2473264|AAAAAAAAADNLFCAA|2059-06-19|1913|8321|639|2059|4|6|19|2|2059|639|8321|Thursday|2059Q2|N|N|N|2473246|2473396|2472899|2473174|N|N|N|N|N| +2473265|AAAAAAAABDNLFCAA|2059-06-20|1913|8321|639|2059|5|6|20|2|2059|639|8321|Friday|2059Q2|N|Y|N|2473246|2473396|2472900|2473175|N|N|N|N|N| +2473266|AAAAAAAACDNLFCAA|2059-06-21|1913|8321|639|2059|6|6|21|2|2059|639|8321|Saturday|2059Q2|N|Y|N|2473246|2473396|2472901|2473176|N|N|N|N|N| +2473267|AAAAAAAADDNLFCAA|2059-06-22|1913|8321|639|2059|0|6|22|2|2059|639|8321|Sunday|2059Q2|N|N|N|2473246|2473396|2472902|2473177|N|N|N|N|N| +2473268|AAAAAAAAEDNLFCAA|2059-06-23|1913|8321|639|2059|1|6|23|2|2059|639|8321|Monday|2059Q2|N|N|N|2473246|2473396|2472903|2473178|N|N|N|N|N| +2473269|AAAAAAAAFDNLFCAA|2059-06-24|1913|8322|639|2059|2|6|24|2|2059|639|8322|Tuesday|2059Q2|N|N|N|2473246|2473396|2472904|2473179|N|N|N|N|N| +2473270|AAAAAAAAGDNLFCAA|2059-06-25|1913|8322|639|2059|3|6|25|2|2059|639|8322|Wednesday|2059Q2|N|N|N|2473246|2473396|2472905|2473180|N|N|N|N|N| +2473271|AAAAAAAAHDNLFCAA|2059-06-26|1913|8322|639|2059|4|6|26|2|2059|639|8322|Thursday|2059Q2|N|N|N|2473246|2473396|2472906|2473181|N|N|N|N|N| +2473272|AAAAAAAAIDNLFCAA|2059-06-27|1913|8322|639|2059|5|6|27|2|2059|639|8322|Friday|2059Q2|N|Y|N|2473246|2473396|2472907|2473182|N|N|N|N|N| +2473273|AAAAAAAAJDNLFCAA|2059-06-28|1913|8322|639|2059|6|6|28|2|2059|639|8322|Saturday|2059Q2|N|Y|N|2473246|2473396|2472908|2473183|N|N|N|N|N| +2473274|AAAAAAAAKDNLFCAA|2059-06-29|1913|8322|639|2059|0|6|29|2|2059|639|8322|Sunday|2059Q2|N|N|N|2473246|2473396|2472909|2473184|N|N|N|N|N| +2473275|AAAAAAAALDNLFCAA|2059-06-30|1913|8322|639|2059|1|6|30|2|2059|639|8322|Monday|2059Q2|N|N|N|2473246|2473396|2472910|2473185|N|N|N|N|N| +2473276|AAAAAAAAMDNLFCAA|2059-07-01|1914|8323|639|2059|2|7|1|2|2059|639|8323|Tuesday|2059Q2|N|N|N|2473276|2473456|2472911|2473185|N|N|N|N|N| +2473277|AAAAAAAANDNLFCAA|2059-07-02|1914|8323|639|2059|3|7|2|3|2059|639|8323|Wednesday|2059Q3|N|N|N|2473276|2473456|2472912|2473186|N|N|N|N|N| +2473278|AAAAAAAAODNLFCAA|2059-07-03|1914|8323|639|2059|4|7|3|3|2059|639|8323|Thursday|2059Q3|N|N|N|2473276|2473456|2472913|2473187|N|N|N|N|N| +2473279|AAAAAAAAPDNLFCAA|2059-07-04|1914|8323|639|2059|5|7|4|3|2059|639|8323|Friday|2059Q3|N|Y|N|2473276|2473456|2472914|2473188|N|N|N|N|N| +2473280|AAAAAAAAAENLFCAA|2059-07-05|1914|8323|639|2059|6|7|5|3|2059|639|8323|Saturday|2059Q3|Y|Y|N|2473276|2473456|2472915|2473189|N|N|N|N|N| +2473281|AAAAAAAABENLFCAA|2059-07-06|1914|8323|639|2059|0|7|6|3|2059|639|8323|Sunday|2059Q3|N|N|Y|2473276|2473456|2472916|2473190|N|N|N|N|N| +2473282|AAAAAAAACENLFCAA|2059-07-07|1914|8323|639|2059|1|7|7|3|2059|639|8323|Monday|2059Q3|N|N|N|2473276|2473456|2472917|2473191|N|N|N|N|N| +2473283|AAAAAAAADENLFCAA|2059-07-08|1914|8324|639|2059|2|7|8|3|2059|639|8324|Tuesday|2059Q3|N|N|N|2473276|2473456|2472918|2473192|N|N|N|N|N| +2473284|AAAAAAAAEENLFCAA|2059-07-09|1914|8324|639|2059|3|7|9|3|2059|639|8324|Wednesday|2059Q3|N|N|N|2473276|2473456|2472919|2473193|N|N|N|N|N| +2473285|AAAAAAAAFENLFCAA|2059-07-10|1914|8324|639|2059|4|7|10|3|2059|639|8324|Thursday|2059Q3|N|N|N|2473276|2473456|2472920|2473194|N|N|N|N|N| +2473286|AAAAAAAAGENLFCAA|2059-07-11|1914|8324|639|2059|5|7|11|3|2059|639|8324|Friday|2059Q3|N|Y|N|2473276|2473456|2472921|2473195|N|N|N|N|N| +2473287|AAAAAAAAHENLFCAA|2059-07-12|1914|8324|639|2059|6|7|12|3|2059|639|8324|Saturday|2059Q3|N|Y|N|2473276|2473456|2472922|2473196|N|N|N|N|N| +2473288|AAAAAAAAIENLFCAA|2059-07-13|1914|8324|639|2059|0|7|13|3|2059|639|8324|Sunday|2059Q3|N|N|N|2473276|2473456|2472923|2473197|N|N|N|N|N| +2473289|AAAAAAAAJENLFCAA|2059-07-14|1914|8324|639|2059|1|7|14|3|2059|639|8324|Monday|2059Q3|N|N|N|2473276|2473456|2472924|2473198|N|N|N|N|N| +2473290|AAAAAAAAKENLFCAA|2059-07-15|1914|8325|639|2059|2|7|15|3|2059|639|8325|Tuesday|2059Q3|N|N|N|2473276|2473456|2472925|2473199|N|N|N|N|N| +2473291|AAAAAAAALENLFCAA|2059-07-16|1914|8325|639|2059|3|7|16|3|2059|639|8325|Wednesday|2059Q3|N|N|N|2473276|2473456|2472926|2473200|N|N|N|N|N| +2473292|AAAAAAAAMENLFCAA|2059-07-17|1914|8325|639|2059|4|7|17|3|2059|639|8325|Thursday|2059Q3|N|N|N|2473276|2473456|2472927|2473201|N|N|N|N|N| +2473293|AAAAAAAANENLFCAA|2059-07-18|1914|8325|639|2059|5|7|18|3|2059|639|8325|Friday|2059Q3|N|Y|N|2473276|2473456|2472928|2473202|N|N|N|N|N| +2473294|AAAAAAAAOENLFCAA|2059-07-19|1914|8325|639|2059|6|7|19|3|2059|639|8325|Saturday|2059Q3|N|Y|N|2473276|2473456|2472929|2473203|N|N|N|N|N| +2473295|AAAAAAAAPENLFCAA|2059-07-20|1914|8325|639|2059|0|7|20|3|2059|639|8325|Sunday|2059Q3|N|N|N|2473276|2473456|2472930|2473204|N|N|N|N|N| +2473296|AAAAAAAAAFNLFCAA|2059-07-21|1914|8325|639|2059|1|7|21|3|2059|639|8325|Monday|2059Q3|N|N|N|2473276|2473456|2472931|2473205|N|N|N|N|N| +2473297|AAAAAAAABFNLFCAA|2059-07-22|1914|8326|639|2059|2|7|22|3|2059|639|8326|Tuesday|2059Q3|N|N|N|2473276|2473456|2472932|2473206|N|N|N|N|N| +2473298|AAAAAAAACFNLFCAA|2059-07-23|1914|8326|639|2059|3|7|23|3|2059|639|8326|Wednesday|2059Q3|N|N|N|2473276|2473456|2472933|2473207|N|N|N|N|N| +2473299|AAAAAAAADFNLFCAA|2059-07-24|1914|8326|639|2059|4|7|24|3|2059|639|8326|Thursday|2059Q3|N|N|N|2473276|2473456|2472934|2473208|N|N|N|N|N| +2473300|AAAAAAAAEFNLFCAA|2059-07-25|1914|8326|639|2059|5|7|25|3|2059|639|8326|Friday|2059Q3|N|Y|N|2473276|2473456|2472935|2473209|N|N|N|N|N| +2473301|AAAAAAAAFFNLFCAA|2059-07-26|1914|8326|639|2059|6|7|26|3|2059|639|8326|Saturday|2059Q3|N|Y|N|2473276|2473456|2472936|2473210|N|N|N|N|N| +2473302|AAAAAAAAGFNLFCAA|2059-07-27|1914|8326|639|2059|0|7|27|3|2059|639|8326|Sunday|2059Q3|N|N|N|2473276|2473456|2472937|2473211|N|N|N|N|N| +2473303|AAAAAAAAHFNLFCAA|2059-07-28|1914|8326|639|2059|1|7|28|3|2059|639|8326|Monday|2059Q3|N|N|N|2473276|2473456|2472938|2473212|N|N|N|N|N| +2473304|AAAAAAAAIFNLFCAA|2059-07-29|1914|8327|639|2059|2|7|29|3|2059|639|8327|Tuesday|2059Q3|N|N|N|2473276|2473456|2472939|2473213|N|N|N|N|N| +2473305|AAAAAAAAJFNLFCAA|2059-07-30|1914|8327|639|2059|3|7|30|3|2059|639|8327|Wednesday|2059Q3|N|N|N|2473276|2473456|2472940|2473214|N|N|N|N|N| +2473306|AAAAAAAAKFNLFCAA|2059-07-31|1914|8327|639|2059|4|7|31|3|2059|639|8327|Thursday|2059Q3|N|N|N|2473276|2473456|2472941|2473215|N|N|N|N|N| +2473307|AAAAAAAALFNLFCAA|2059-08-01|1915|8327|639|2059|5|8|1|3|2059|639|8327|Friday|2059Q3|N|Y|N|2473307|2473518|2472942|2473216|N|N|N|N|N| +2473308|AAAAAAAAMFNLFCAA|2059-08-02|1915|8327|639|2059|6|8|2|3|2059|639|8327|Saturday|2059Q3|N|Y|N|2473307|2473518|2472943|2473217|N|N|N|N|N| +2473309|AAAAAAAANFNLFCAA|2059-08-03|1915|8327|639|2059|0|8|3|3|2059|639|8327|Sunday|2059Q3|N|N|N|2473307|2473518|2472944|2473218|N|N|N|N|N| +2473310|AAAAAAAAOFNLFCAA|2059-08-04|1915|8327|639|2059|1|8|4|3|2059|639|8327|Monday|2059Q3|N|N|N|2473307|2473518|2472945|2473219|N|N|N|N|N| +2473311|AAAAAAAAPFNLFCAA|2059-08-05|1915|8328|639|2059|2|8|5|3|2059|639|8328|Tuesday|2059Q3|N|N|N|2473307|2473518|2472946|2473220|N|N|N|N|N| +2473312|AAAAAAAAAGNLFCAA|2059-08-06|1915|8328|639|2059|3|8|6|3|2059|639|8328|Wednesday|2059Q3|N|N|N|2473307|2473518|2472947|2473221|N|N|N|N|N| +2473313|AAAAAAAABGNLFCAA|2059-08-07|1915|8328|639|2059|4|8|7|3|2059|639|8328|Thursday|2059Q3|N|N|N|2473307|2473518|2472948|2473222|N|N|N|N|N| +2473314|AAAAAAAACGNLFCAA|2059-08-08|1915|8328|639|2059|5|8|8|3|2059|639|8328|Friday|2059Q3|N|Y|N|2473307|2473518|2472949|2473223|N|N|N|N|N| +2473315|AAAAAAAADGNLFCAA|2059-08-09|1915|8328|639|2059|6|8|9|3|2059|639|8328|Saturday|2059Q3|N|Y|N|2473307|2473518|2472950|2473224|N|N|N|N|N| +2473316|AAAAAAAAEGNLFCAA|2059-08-10|1915|8328|639|2059|0|8|10|3|2059|639|8328|Sunday|2059Q3|N|N|N|2473307|2473518|2472951|2473225|N|N|N|N|N| +2473317|AAAAAAAAFGNLFCAA|2059-08-11|1915|8328|639|2059|1|8|11|3|2059|639|8328|Monday|2059Q3|N|N|N|2473307|2473518|2472952|2473226|N|N|N|N|N| +2473318|AAAAAAAAGGNLFCAA|2059-08-12|1915|8329|639|2059|2|8|12|3|2059|639|8329|Tuesday|2059Q3|N|N|N|2473307|2473518|2472953|2473227|N|N|N|N|N| +2473319|AAAAAAAAHGNLFCAA|2059-08-13|1915|8329|639|2059|3|8|13|3|2059|639|8329|Wednesday|2059Q3|N|N|N|2473307|2473518|2472954|2473228|N|N|N|N|N| +2473320|AAAAAAAAIGNLFCAA|2059-08-14|1915|8329|639|2059|4|8|14|3|2059|639|8329|Thursday|2059Q3|N|N|N|2473307|2473518|2472955|2473229|N|N|N|N|N| +2473321|AAAAAAAAJGNLFCAA|2059-08-15|1915|8329|639|2059|5|8|15|3|2059|639|8329|Friday|2059Q3|N|Y|N|2473307|2473518|2472956|2473230|N|N|N|N|N| +2473322|AAAAAAAAKGNLFCAA|2059-08-16|1915|8329|639|2059|6|8|16|3|2059|639|8329|Saturday|2059Q3|N|Y|N|2473307|2473518|2472957|2473231|N|N|N|N|N| +2473323|AAAAAAAALGNLFCAA|2059-08-17|1915|8329|639|2059|0|8|17|3|2059|639|8329|Sunday|2059Q3|N|N|N|2473307|2473518|2472958|2473232|N|N|N|N|N| +2473324|AAAAAAAAMGNLFCAA|2059-08-18|1915|8329|639|2059|1|8|18|3|2059|639|8329|Monday|2059Q3|N|N|N|2473307|2473518|2472959|2473233|N|N|N|N|N| +2473325|AAAAAAAANGNLFCAA|2059-08-19|1915|8330|639|2059|2|8|19|3|2059|639|8330|Tuesday|2059Q3|N|N|N|2473307|2473518|2472960|2473234|N|N|N|N|N| +2473326|AAAAAAAAOGNLFCAA|2059-08-20|1915|8330|639|2059|3|8|20|3|2059|639|8330|Wednesday|2059Q3|N|N|N|2473307|2473518|2472961|2473235|N|N|N|N|N| +2473327|AAAAAAAAPGNLFCAA|2059-08-21|1915|8330|639|2059|4|8|21|3|2059|639|8330|Thursday|2059Q3|N|N|N|2473307|2473518|2472962|2473236|N|N|N|N|N| +2473328|AAAAAAAAAHNLFCAA|2059-08-22|1915|8330|639|2059|5|8|22|3|2059|639|8330|Friday|2059Q3|N|Y|N|2473307|2473518|2472963|2473237|N|N|N|N|N| +2473329|AAAAAAAABHNLFCAA|2059-08-23|1915|8330|639|2059|6|8|23|3|2059|639|8330|Saturday|2059Q3|N|Y|N|2473307|2473518|2472964|2473238|N|N|N|N|N| +2473330|AAAAAAAACHNLFCAA|2059-08-24|1915|8330|639|2059|0|8|24|3|2059|639|8330|Sunday|2059Q3|N|N|N|2473307|2473518|2472965|2473239|N|N|N|N|N| +2473331|AAAAAAAADHNLFCAA|2059-08-25|1915|8330|639|2059|1|8|25|3|2059|639|8330|Monday|2059Q3|N|N|N|2473307|2473518|2472966|2473240|N|N|N|N|N| +2473332|AAAAAAAAEHNLFCAA|2059-08-26|1915|8331|639|2059|2|8|26|3|2059|639|8331|Tuesday|2059Q3|N|N|N|2473307|2473518|2472967|2473241|N|N|N|N|N| +2473333|AAAAAAAAFHNLFCAA|2059-08-27|1915|8331|639|2059|3|8|27|3|2059|639|8331|Wednesday|2059Q3|N|N|N|2473307|2473518|2472968|2473242|N|N|N|N|N| +2473334|AAAAAAAAGHNLFCAA|2059-08-28|1915|8331|639|2059|4|8|28|3|2059|639|8331|Thursday|2059Q3|N|N|N|2473307|2473518|2472969|2473243|N|N|N|N|N| +2473335|AAAAAAAAHHNLFCAA|2059-08-29|1915|8331|639|2059|5|8|29|3|2059|639|8331|Friday|2059Q3|N|Y|N|2473307|2473518|2472970|2473244|N|N|N|N|N| +2473336|AAAAAAAAIHNLFCAA|2059-08-30|1915|8331|639|2059|6|8|30|3|2059|639|8331|Saturday|2059Q3|N|Y|N|2473307|2473518|2472971|2473245|N|N|N|N|N| +2473337|AAAAAAAAJHNLFCAA|2059-08-31|1915|8331|639|2059|0|8|31|3|2059|639|8331|Sunday|2059Q3|N|N|N|2473307|2473518|2472972|2473246|N|N|N|N|N| +2473338|AAAAAAAAKHNLFCAA|2059-09-01|1916|8331|640|2059|1|9|1|3|2059|640|8331|Monday|2059Q3|N|N|N|2473338|2473580|2472973|2473247|N|N|N|N|N| +2473339|AAAAAAAALHNLFCAA|2059-09-02|1916|8332|640|2059|2|9|2|3|2059|640|8332|Tuesday|2059Q3|N|N|N|2473338|2473580|2472974|2473248|N|N|N|N|N| +2473340|AAAAAAAAMHNLFCAA|2059-09-03|1916|8332|640|2059|3|9|3|3|2059|640|8332|Wednesday|2059Q3|N|N|N|2473338|2473580|2472975|2473249|N|N|N|N|N| +2473341|AAAAAAAANHNLFCAA|2059-09-04|1916|8332|640|2059|4|9|4|3|2059|640|8332|Thursday|2059Q3|N|N|N|2473338|2473580|2472976|2473250|N|N|N|N|N| +2473342|AAAAAAAAOHNLFCAA|2059-09-05|1916|8332|640|2059|5|9|5|3|2059|640|8332|Friday|2059Q3|N|Y|N|2473338|2473580|2472977|2473251|N|N|N|N|N| +2473343|AAAAAAAAPHNLFCAA|2059-09-06|1916|8332|640|2059|6|9|6|3|2059|640|8332|Saturday|2059Q3|N|Y|N|2473338|2473580|2472978|2473252|N|N|N|N|N| +2473344|AAAAAAAAAINLFCAA|2059-09-07|1916|8332|640|2059|0|9|7|3|2059|640|8332|Sunday|2059Q3|N|N|N|2473338|2473580|2472979|2473253|N|N|N|N|N| +2473345|AAAAAAAABINLFCAA|2059-09-08|1916|8332|640|2059|1|9|8|3|2059|640|8332|Monday|2059Q3|N|N|N|2473338|2473580|2472980|2473254|N|N|N|N|N| +2473346|AAAAAAAACINLFCAA|2059-09-09|1916|8333|640|2059|2|9|9|3|2059|640|8333|Tuesday|2059Q3|N|N|N|2473338|2473580|2472981|2473255|N|N|N|N|N| +2473347|AAAAAAAADINLFCAA|2059-09-10|1916|8333|640|2059|3|9|10|3|2059|640|8333|Wednesday|2059Q3|N|N|N|2473338|2473580|2472982|2473256|N|N|N|N|N| +2473348|AAAAAAAAEINLFCAA|2059-09-11|1916|8333|640|2059|4|9|11|3|2059|640|8333|Thursday|2059Q3|N|N|N|2473338|2473580|2472983|2473257|N|N|N|N|N| +2473349|AAAAAAAAFINLFCAA|2059-09-12|1916|8333|640|2059|5|9|12|3|2059|640|8333|Friday|2059Q3|N|Y|N|2473338|2473580|2472984|2473258|N|N|N|N|N| +2473350|AAAAAAAAGINLFCAA|2059-09-13|1916|8333|640|2059|6|9|13|3|2059|640|8333|Saturday|2059Q3|N|Y|N|2473338|2473580|2472985|2473259|N|N|N|N|N| +2473351|AAAAAAAAHINLFCAA|2059-09-14|1916|8333|640|2059|0|9|14|3|2059|640|8333|Sunday|2059Q3|N|N|N|2473338|2473580|2472986|2473260|N|N|N|N|N| +2473352|AAAAAAAAIINLFCAA|2059-09-15|1916|8333|640|2059|1|9|15|3|2059|640|8333|Monday|2059Q3|N|N|N|2473338|2473580|2472987|2473261|N|N|N|N|N| +2473353|AAAAAAAAJINLFCAA|2059-09-16|1916|8334|640|2059|2|9|16|3|2059|640|8334|Tuesday|2059Q3|N|N|N|2473338|2473580|2472988|2473262|N|N|N|N|N| +2473354|AAAAAAAAKINLFCAA|2059-09-17|1916|8334|640|2059|3|9|17|3|2059|640|8334|Wednesday|2059Q3|N|N|N|2473338|2473580|2472989|2473263|N|N|N|N|N| +2473355|AAAAAAAALINLFCAA|2059-09-18|1916|8334|640|2059|4|9|18|3|2059|640|8334|Thursday|2059Q3|N|N|N|2473338|2473580|2472990|2473264|N|N|N|N|N| +2473356|AAAAAAAAMINLFCAA|2059-09-19|1916|8334|640|2059|5|9|19|3|2059|640|8334|Friday|2059Q3|N|Y|N|2473338|2473580|2472991|2473265|N|N|N|N|N| +2473357|AAAAAAAANINLFCAA|2059-09-20|1916|8334|640|2059|6|9|20|3|2059|640|8334|Saturday|2059Q3|N|Y|N|2473338|2473580|2472992|2473266|N|N|N|N|N| +2473358|AAAAAAAAOINLFCAA|2059-09-21|1916|8334|640|2059|0|9|21|3|2059|640|8334|Sunday|2059Q3|N|N|N|2473338|2473580|2472993|2473267|N|N|N|N|N| +2473359|AAAAAAAAPINLFCAA|2059-09-22|1916|8334|640|2059|1|9|22|3|2059|640|8334|Monday|2059Q3|N|N|N|2473338|2473580|2472994|2473268|N|N|N|N|N| +2473360|AAAAAAAAAJNLFCAA|2059-09-23|1916|8335|640|2059|2|9|23|3|2059|640|8335|Tuesday|2059Q3|N|N|N|2473338|2473580|2472995|2473269|N|N|N|N|N| +2473361|AAAAAAAABJNLFCAA|2059-09-24|1916|8335|640|2059|3|9|24|3|2059|640|8335|Wednesday|2059Q3|N|N|N|2473338|2473580|2472996|2473270|N|N|N|N|N| +2473362|AAAAAAAACJNLFCAA|2059-09-25|1916|8335|640|2059|4|9|25|3|2059|640|8335|Thursday|2059Q3|N|N|N|2473338|2473580|2472997|2473271|N|N|N|N|N| +2473363|AAAAAAAADJNLFCAA|2059-09-26|1916|8335|640|2059|5|9|26|3|2059|640|8335|Friday|2059Q3|N|Y|N|2473338|2473580|2472998|2473272|N|N|N|N|N| +2473364|AAAAAAAAEJNLFCAA|2059-09-27|1916|8335|640|2059|6|9|27|3|2059|640|8335|Saturday|2059Q3|N|Y|N|2473338|2473580|2472999|2473273|N|N|N|N|N| +2473365|AAAAAAAAFJNLFCAA|2059-09-28|1916|8335|640|2059|0|9|28|3|2059|640|8335|Sunday|2059Q3|N|N|N|2473338|2473580|2473000|2473274|N|N|N|N|N| +2473366|AAAAAAAAGJNLFCAA|2059-09-29|1916|8335|640|2059|1|9|29|3|2059|640|8335|Monday|2059Q3|N|N|N|2473338|2473580|2473001|2473275|N|N|N|N|N| +2473367|AAAAAAAAHJNLFCAA|2059-09-30|1916|8336|640|2059|2|9|30|3|2059|640|8336|Tuesday|2059Q3|N|N|N|2473338|2473580|2473002|2473276|N|N|N|N|N| +2473368|AAAAAAAAIJNLFCAA|2059-10-01|1917|8336|640|2059|3|10|1|3|2059|640|8336|Wednesday|2059Q3|N|N|N|2473368|2473640|2473003|2473276|N|N|N|N|N| +2473369|AAAAAAAAJJNLFCAA|2059-10-02|1917|8336|640|2059|4|10|2|4|2059|640|8336|Thursday|2059Q4|N|N|N|2473368|2473640|2473004|2473277|N|N|N|N|N| +2473370|AAAAAAAAKJNLFCAA|2059-10-03|1917|8336|640|2059|5|10|3|4|2059|640|8336|Friday|2059Q4|N|Y|N|2473368|2473640|2473005|2473278|N|N|N|N|N| +2473371|AAAAAAAALJNLFCAA|2059-10-04|1917|8336|640|2059|6|10|4|4|2059|640|8336|Saturday|2059Q4|N|Y|N|2473368|2473640|2473006|2473279|N|N|N|N|N| +2473372|AAAAAAAAMJNLFCAA|2059-10-05|1917|8336|640|2059|0|10|5|4|2059|640|8336|Sunday|2059Q4|N|N|N|2473368|2473640|2473007|2473280|N|N|N|N|N| +2473373|AAAAAAAANJNLFCAA|2059-10-06|1917|8336|640|2059|1|10|6|4|2059|640|8336|Monday|2059Q4|N|N|N|2473368|2473640|2473008|2473281|N|N|N|N|N| +2473374|AAAAAAAAOJNLFCAA|2059-10-07|1917|8337|640|2059|2|10|7|4|2059|640|8337|Tuesday|2059Q4|N|N|N|2473368|2473640|2473009|2473282|N|N|N|N|N| +2473375|AAAAAAAAPJNLFCAA|2059-10-08|1917|8337|640|2059|3|10|8|4|2059|640|8337|Wednesday|2059Q4|N|N|N|2473368|2473640|2473010|2473283|N|N|N|N|N| +2473376|AAAAAAAAAKNLFCAA|2059-10-09|1917|8337|640|2059|4|10|9|4|2059|640|8337|Thursday|2059Q4|N|N|N|2473368|2473640|2473011|2473284|N|N|N|N|N| +2473377|AAAAAAAABKNLFCAA|2059-10-10|1917|8337|640|2059|5|10|10|4|2059|640|8337|Friday|2059Q4|N|Y|N|2473368|2473640|2473012|2473285|N|N|N|N|N| +2473378|AAAAAAAACKNLFCAA|2059-10-11|1917|8337|640|2059|6|10|11|4|2059|640|8337|Saturday|2059Q4|N|Y|N|2473368|2473640|2473013|2473286|N|N|N|N|N| +2473379|AAAAAAAADKNLFCAA|2059-10-12|1917|8337|640|2059|0|10|12|4|2059|640|8337|Sunday|2059Q4|N|N|N|2473368|2473640|2473014|2473287|N|N|N|N|N| +2473380|AAAAAAAAEKNLFCAA|2059-10-13|1917|8337|640|2059|1|10|13|4|2059|640|8337|Monday|2059Q4|N|N|N|2473368|2473640|2473015|2473288|N|N|N|N|N| +2473381|AAAAAAAAFKNLFCAA|2059-10-14|1917|8338|640|2059|2|10|14|4|2059|640|8338|Tuesday|2059Q4|N|N|N|2473368|2473640|2473016|2473289|N|N|N|N|N| +2473382|AAAAAAAAGKNLFCAA|2059-10-15|1917|8338|640|2059|3|10|15|4|2059|640|8338|Wednesday|2059Q4|N|N|N|2473368|2473640|2473017|2473290|N|N|N|N|N| +2473383|AAAAAAAAHKNLFCAA|2059-10-16|1917|8338|640|2059|4|10|16|4|2059|640|8338|Thursday|2059Q4|N|N|N|2473368|2473640|2473018|2473291|N|N|N|N|N| +2473384|AAAAAAAAIKNLFCAA|2059-10-17|1917|8338|640|2059|5|10|17|4|2059|640|8338|Friday|2059Q4|N|Y|N|2473368|2473640|2473019|2473292|N|N|N|N|N| +2473385|AAAAAAAAJKNLFCAA|2059-10-18|1917|8338|640|2059|6|10|18|4|2059|640|8338|Saturday|2059Q4|N|Y|N|2473368|2473640|2473020|2473293|N|N|N|N|N| +2473386|AAAAAAAAKKNLFCAA|2059-10-19|1917|8338|640|2059|0|10|19|4|2059|640|8338|Sunday|2059Q4|N|N|N|2473368|2473640|2473021|2473294|N|N|N|N|N| +2473387|AAAAAAAALKNLFCAA|2059-10-20|1917|8338|640|2059|1|10|20|4|2059|640|8338|Monday|2059Q4|N|N|N|2473368|2473640|2473022|2473295|N|N|N|N|N| +2473388|AAAAAAAAMKNLFCAA|2059-10-21|1917|8339|640|2059|2|10|21|4|2059|640|8339|Tuesday|2059Q4|N|N|N|2473368|2473640|2473023|2473296|N|N|N|N|N| +2473389|AAAAAAAANKNLFCAA|2059-10-22|1917|8339|640|2059|3|10|22|4|2059|640|8339|Wednesday|2059Q4|N|N|N|2473368|2473640|2473024|2473297|N|N|N|N|N| +2473390|AAAAAAAAOKNLFCAA|2059-10-23|1917|8339|640|2059|4|10|23|4|2059|640|8339|Thursday|2059Q4|N|N|N|2473368|2473640|2473025|2473298|N|N|N|N|N| +2473391|AAAAAAAAPKNLFCAA|2059-10-24|1917|8339|640|2059|5|10|24|4|2059|640|8339|Friday|2059Q4|N|Y|N|2473368|2473640|2473026|2473299|N|N|N|N|N| +2473392|AAAAAAAAALNLFCAA|2059-10-25|1917|8339|640|2059|6|10|25|4|2059|640|8339|Saturday|2059Q4|N|Y|N|2473368|2473640|2473027|2473300|N|N|N|N|N| +2473393|AAAAAAAABLNLFCAA|2059-10-26|1917|8339|640|2059|0|10|26|4|2059|640|8339|Sunday|2059Q4|N|N|N|2473368|2473640|2473028|2473301|N|N|N|N|N| +2473394|AAAAAAAACLNLFCAA|2059-10-27|1917|8339|640|2059|1|10|27|4|2059|640|8339|Monday|2059Q4|N|N|N|2473368|2473640|2473029|2473302|N|N|N|N|N| +2473395|AAAAAAAADLNLFCAA|2059-10-28|1917|8340|640|2059|2|10|28|4|2059|640|8340|Tuesday|2059Q4|N|N|N|2473368|2473640|2473030|2473303|N|N|N|N|N| +2473396|AAAAAAAAELNLFCAA|2059-10-29|1917|8340|640|2059|3|10|29|4|2059|640|8340|Wednesday|2059Q4|N|N|N|2473368|2473640|2473031|2473304|N|N|N|N|N| +2473397|AAAAAAAAFLNLFCAA|2059-10-30|1917|8340|640|2059|4|10|30|4|2059|640|8340|Thursday|2059Q4|N|N|N|2473368|2473640|2473032|2473305|N|N|N|N|N| +2473398|AAAAAAAAGLNLFCAA|2059-10-31|1917|8340|640|2059|5|10|31|4|2059|640|8340|Friday|2059Q4|N|Y|N|2473368|2473640|2473033|2473306|N|N|N|N|N| +2473399|AAAAAAAAHLNLFCAA|2059-11-01|1918|8340|640|2059|6|11|1|4|2059|640|8340|Saturday|2059Q4|N|Y|N|2473399|2473702|2473034|2473307|N|N|N|N|N| +2473400|AAAAAAAAILNLFCAA|2059-11-02|1918|8340|640|2059|0|11|2|4|2059|640|8340|Sunday|2059Q4|N|N|N|2473399|2473702|2473035|2473308|N|N|N|N|N| +2473401|AAAAAAAAJLNLFCAA|2059-11-03|1918|8340|640|2059|1|11|3|4|2059|640|8340|Monday|2059Q4|N|N|N|2473399|2473702|2473036|2473309|N|N|N|N|N| +2473402|AAAAAAAAKLNLFCAA|2059-11-04|1918|8341|640|2059|2|11|4|4|2059|640|8341|Tuesday|2059Q4|N|N|N|2473399|2473702|2473037|2473310|N|N|N|N|N| +2473403|AAAAAAAALLNLFCAA|2059-11-05|1918|8341|640|2059|3|11|5|4|2059|640|8341|Wednesday|2059Q4|N|N|N|2473399|2473702|2473038|2473311|N|N|N|N|N| +2473404|AAAAAAAAMLNLFCAA|2059-11-06|1918|8341|640|2059|4|11|6|4|2059|640|8341|Thursday|2059Q4|N|N|N|2473399|2473702|2473039|2473312|N|N|N|N|N| +2473405|AAAAAAAANLNLFCAA|2059-11-07|1918|8341|640|2059|5|11|7|4|2059|640|8341|Friday|2059Q4|N|Y|N|2473399|2473702|2473040|2473313|N|N|N|N|N| +2473406|AAAAAAAAOLNLFCAA|2059-11-08|1918|8341|640|2059|6|11|8|4|2059|640|8341|Saturday|2059Q4|N|Y|N|2473399|2473702|2473041|2473314|N|N|N|N|N| +2473407|AAAAAAAAPLNLFCAA|2059-11-09|1918|8341|640|2059|0|11|9|4|2059|640|8341|Sunday|2059Q4|N|N|N|2473399|2473702|2473042|2473315|N|N|N|N|N| +2473408|AAAAAAAAAMNLFCAA|2059-11-10|1918|8341|640|2059|1|11|10|4|2059|640|8341|Monday|2059Q4|N|N|N|2473399|2473702|2473043|2473316|N|N|N|N|N| +2473409|AAAAAAAABMNLFCAA|2059-11-11|1918|8342|640|2059|2|11|11|4|2059|640|8342|Tuesday|2059Q4|N|N|N|2473399|2473702|2473044|2473317|N|N|N|N|N| +2473410|AAAAAAAACMNLFCAA|2059-11-12|1918|8342|640|2059|3|11|12|4|2059|640|8342|Wednesday|2059Q4|N|N|N|2473399|2473702|2473045|2473318|N|N|N|N|N| +2473411|AAAAAAAADMNLFCAA|2059-11-13|1918|8342|640|2059|4|11|13|4|2059|640|8342|Thursday|2059Q4|N|N|N|2473399|2473702|2473046|2473319|N|N|N|N|N| +2473412|AAAAAAAAEMNLFCAA|2059-11-14|1918|8342|640|2059|5|11|14|4|2059|640|8342|Friday|2059Q4|N|Y|N|2473399|2473702|2473047|2473320|N|N|N|N|N| +2473413|AAAAAAAAFMNLFCAA|2059-11-15|1918|8342|640|2059|6|11|15|4|2059|640|8342|Saturday|2059Q4|N|Y|N|2473399|2473702|2473048|2473321|N|N|N|N|N| +2473414|AAAAAAAAGMNLFCAA|2059-11-16|1918|8342|640|2059|0|11|16|4|2059|640|8342|Sunday|2059Q4|N|N|N|2473399|2473702|2473049|2473322|N|N|N|N|N| +2473415|AAAAAAAAHMNLFCAA|2059-11-17|1918|8342|640|2059|1|11|17|4|2059|640|8342|Monday|2059Q4|N|N|N|2473399|2473702|2473050|2473323|N|N|N|N|N| +2473416|AAAAAAAAIMNLFCAA|2059-11-18|1918|8343|640|2059|2|11|18|4|2059|640|8343|Tuesday|2059Q4|N|N|N|2473399|2473702|2473051|2473324|N|N|N|N|N| +2473417|AAAAAAAAJMNLFCAA|2059-11-19|1918|8343|640|2059|3|11|19|4|2059|640|8343|Wednesday|2059Q4|N|N|N|2473399|2473702|2473052|2473325|N|N|N|N|N| +2473418|AAAAAAAAKMNLFCAA|2059-11-20|1918|8343|640|2059|4|11|20|4|2059|640|8343|Thursday|2059Q4|N|N|N|2473399|2473702|2473053|2473326|N|N|N|N|N| +2473419|AAAAAAAALMNLFCAA|2059-11-21|1918|8343|640|2059|5|11|21|4|2059|640|8343|Friday|2059Q4|N|Y|N|2473399|2473702|2473054|2473327|N|N|N|N|N| +2473420|AAAAAAAAMMNLFCAA|2059-11-22|1918|8343|640|2059|6|11|22|4|2059|640|8343|Saturday|2059Q4|N|Y|N|2473399|2473702|2473055|2473328|N|N|N|N|N| +2473421|AAAAAAAANMNLFCAA|2059-11-23|1918|8343|640|2059|0|11|23|4|2059|640|8343|Sunday|2059Q4|N|N|N|2473399|2473702|2473056|2473329|N|N|N|N|N| +2473422|AAAAAAAAOMNLFCAA|2059-11-24|1918|8343|640|2059|1|11|24|4|2059|640|8343|Monday|2059Q4|N|N|N|2473399|2473702|2473057|2473330|N|N|N|N|N| +2473423|AAAAAAAAPMNLFCAA|2059-11-25|1918|8344|640|2059|2|11|25|4|2059|640|8344|Tuesday|2059Q4|N|N|N|2473399|2473702|2473058|2473331|N|N|N|N|N| +2473424|AAAAAAAAANNLFCAA|2059-11-26|1918|8344|640|2059|3|11|26|4|2059|640|8344|Wednesday|2059Q4|N|N|N|2473399|2473702|2473059|2473332|N|N|N|N|N| +2473425|AAAAAAAABNNLFCAA|2059-11-27|1918|8344|640|2059|4|11|27|4|2059|640|8344|Thursday|2059Q4|N|N|N|2473399|2473702|2473060|2473333|N|N|N|N|N| +2473426|AAAAAAAACNNLFCAA|2059-11-28|1918|8344|640|2059|5|11|28|4|2059|640|8344|Friday|2059Q4|N|Y|N|2473399|2473702|2473061|2473334|N|N|N|N|N| +2473427|AAAAAAAADNNLFCAA|2059-11-29|1918|8344|640|2059|6|11|29|4|2059|640|8344|Saturday|2059Q4|N|Y|N|2473399|2473702|2473062|2473335|N|N|N|N|N| +2473428|AAAAAAAAENNLFCAA|2059-11-30|1918|8344|640|2059|0|11|30|4|2059|640|8344|Sunday|2059Q4|N|N|N|2473399|2473702|2473063|2473336|N|N|N|N|N| +2473429|AAAAAAAAFNNLFCAA|2059-12-01|1919|8344|641|2059|1|12|1|4|2059|641|8344|Monday|2059Q4|N|N|N|2473429|2473762|2473064|2473337|N|N|N|N|N| +2473430|AAAAAAAAGNNLFCAA|2059-12-02|1919|8345|641|2059|2|12|2|4|2059|641|8345|Tuesday|2059Q4|N|N|N|2473429|2473762|2473065|2473338|N|N|N|N|N| +2473431|AAAAAAAAHNNLFCAA|2059-12-03|1919|8345|641|2059|3|12|3|4|2059|641|8345|Wednesday|2059Q4|N|N|N|2473429|2473762|2473066|2473339|N|N|N|N|N| +2473432|AAAAAAAAINNLFCAA|2059-12-04|1919|8345|641|2059|4|12|4|4|2059|641|8345|Thursday|2059Q4|N|N|N|2473429|2473762|2473067|2473340|N|N|N|N|N| +2473433|AAAAAAAAJNNLFCAA|2059-12-05|1919|8345|641|2059|5|12|5|4|2059|641|8345|Friday|2059Q4|N|Y|N|2473429|2473762|2473068|2473341|N|N|N|N|N| +2473434|AAAAAAAAKNNLFCAA|2059-12-06|1919|8345|641|2059|6|12|6|4|2059|641|8345|Saturday|2059Q4|N|Y|N|2473429|2473762|2473069|2473342|N|N|N|N|N| +2473435|AAAAAAAALNNLFCAA|2059-12-07|1919|8345|641|2059|0|12|7|4|2059|641|8345|Sunday|2059Q4|N|N|N|2473429|2473762|2473070|2473343|N|N|N|N|N| +2473436|AAAAAAAAMNNLFCAA|2059-12-08|1919|8345|641|2059|1|12|8|4|2059|641|8345|Monday|2059Q4|N|N|N|2473429|2473762|2473071|2473344|N|N|N|N|N| +2473437|AAAAAAAANNNLFCAA|2059-12-09|1919|8346|641|2059|2|12|9|4|2059|641|8346|Tuesday|2059Q4|N|N|N|2473429|2473762|2473072|2473345|N|N|N|N|N| +2473438|AAAAAAAAONNLFCAA|2059-12-10|1919|8346|641|2059|3|12|10|4|2059|641|8346|Wednesday|2059Q4|N|N|N|2473429|2473762|2473073|2473346|N|N|N|N|N| +2473439|AAAAAAAAPNNLFCAA|2059-12-11|1919|8346|641|2059|4|12|11|4|2059|641|8346|Thursday|2059Q4|N|N|N|2473429|2473762|2473074|2473347|N|N|N|N|N| +2473440|AAAAAAAAAONLFCAA|2059-12-12|1919|8346|641|2059|5|12|12|4|2059|641|8346|Friday|2059Q4|N|Y|N|2473429|2473762|2473075|2473348|N|N|N|N|N| +2473441|AAAAAAAABONLFCAA|2059-12-13|1919|8346|641|2059|6|12|13|4|2059|641|8346|Saturday|2059Q4|N|Y|N|2473429|2473762|2473076|2473349|N|N|N|N|N| +2473442|AAAAAAAACONLFCAA|2059-12-14|1919|8346|641|2059|0|12|14|4|2059|641|8346|Sunday|2059Q4|N|N|N|2473429|2473762|2473077|2473350|N|N|N|N|N| +2473443|AAAAAAAADONLFCAA|2059-12-15|1919|8346|641|2059|1|12|15|4|2059|641|8346|Monday|2059Q4|N|N|N|2473429|2473762|2473078|2473351|N|N|N|N|N| +2473444|AAAAAAAAEONLFCAA|2059-12-16|1919|8347|641|2059|2|12|16|4|2059|641|8347|Tuesday|2059Q4|N|N|N|2473429|2473762|2473079|2473352|N|N|N|N|N| +2473445|AAAAAAAAFONLFCAA|2059-12-17|1919|8347|641|2059|3|12|17|4|2059|641|8347|Wednesday|2059Q4|N|N|N|2473429|2473762|2473080|2473353|N|N|N|N|N| +2473446|AAAAAAAAGONLFCAA|2059-12-18|1919|8347|641|2059|4|12|18|4|2059|641|8347|Thursday|2059Q4|N|N|N|2473429|2473762|2473081|2473354|N|N|N|N|N| +2473447|AAAAAAAAHONLFCAA|2059-12-19|1919|8347|641|2059|5|12|19|4|2059|641|8347|Friday|2059Q4|N|Y|N|2473429|2473762|2473082|2473355|N|N|N|N|N| +2473448|AAAAAAAAIONLFCAA|2059-12-20|1919|8347|641|2059|6|12|20|4|2059|641|8347|Saturday|2059Q4|N|Y|N|2473429|2473762|2473083|2473356|N|N|N|N|N| +2473449|AAAAAAAAJONLFCAA|2059-12-21|1919|8347|641|2059|0|12|21|4|2059|641|8347|Sunday|2059Q4|N|N|N|2473429|2473762|2473084|2473357|N|N|N|N|N| +2473450|AAAAAAAAKONLFCAA|2059-12-22|1919|8347|641|2059|1|12|22|4|2059|641|8347|Monday|2059Q4|N|N|N|2473429|2473762|2473085|2473358|N|N|N|N|N| +2473451|AAAAAAAALONLFCAA|2059-12-23|1919|8348|641|2059|2|12|23|4|2059|641|8348|Tuesday|2059Q4|N|N|N|2473429|2473762|2473086|2473359|N|N|N|N|N| +2473452|AAAAAAAAMONLFCAA|2059-12-24|1919|8348|641|2059|3|12|24|4|2059|641|8348|Wednesday|2059Q4|N|N|N|2473429|2473762|2473087|2473360|N|N|N|N|N| +2473453|AAAAAAAANONLFCAA|2059-12-25|1919|8348|641|2059|4|12|25|4|2059|641|8348|Thursday|2059Q4|N|N|N|2473429|2473762|2473088|2473361|N|N|N|N|N| +2473454|AAAAAAAAOONLFCAA|2059-12-26|1919|8348|641|2059|5|12|26|4|2059|641|8348|Friday|2059Q4|Y|Y|N|2473429|2473762|2473089|2473362|N|N|N|N|N| +2473455|AAAAAAAAPONLFCAA|2059-12-27|1919|8348|641|2059|6|12|27|4|2059|641|8348|Saturday|2059Q4|N|Y|Y|2473429|2473762|2473090|2473363|N|N|N|N|N| +2473456|AAAAAAAAAPNLFCAA|2059-12-28|1919|8348|641|2059|0|12|28|4|2059|641|8348|Sunday|2059Q4|N|N|N|2473429|2473762|2473091|2473364|N|N|N|N|N| +2473457|AAAAAAAABPNLFCAA|2059-12-29|1919|8348|641|2059|1|12|29|4|2059|641|8348|Monday|2059Q4|N|N|N|2473429|2473762|2473092|2473365|N|N|N|N|N| +2473458|AAAAAAAACPNLFCAA|2059-12-30|1919|8349|641|2059|2|12|30|4|2059|641|8349|Tuesday|2059Q4|N|N|N|2473429|2473762|2473093|2473366|N|N|N|N|N| +2473459|AAAAAAAADPNLFCAA|2059-12-31|1919|8349|641|2059|3|12|31|4|2059|641|8349|Wednesday|2059Q4|N|N|N|2473429|2473762|2473094|2473367|N|N|N|N|N| +2473460|AAAAAAAAEPNLFCAA|2060-01-01|1920|8349|641|2060|4|1|1|1|2060|641|8349|Thursday|2060Q1|Y|N|N|2473460|2473459|2473095|2473368|N|N|N|N|N| +2473461|AAAAAAAAFPNLFCAA|2060-01-02|1920|8349|641|2060|5|1|2|1|2060|641|8349|Friday|2060Q1|N|Y|Y|2473460|2473459|2473096|2473369|N|N|N|N|N| +2473462|AAAAAAAAGPNLFCAA|2060-01-03|1920|8349|641|2060|6|1|3|1|2060|641|8349|Saturday|2060Q1|N|Y|N|2473460|2473459|2473097|2473370|N|N|N|N|N| +2473463|AAAAAAAAHPNLFCAA|2060-01-04|1920|8349|641|2060|0|1|4|1|2060|641|8349|Sunday|2060Q1|N|N|N|2473460|2473459|2473098|2473371|N|N|N|N|N| +2473464|AAAAAAAAIPNLFCAA|2060-01-05|1920|8349|641|2060|1|1|5|1|2060|641|8349|Monday|2060Q1|N|N|N|2473460|2473459|2473099|2473372|N|N|N|N|N| +2473465|AAAAAAAAJPNLFCAA|2060-01-06|1920|8350|641|2060|2|1|6|1|2060|641|8350|Tuesday|2060Q1|N|N|N|2473460|2473459|2473100|2473373|N|N|N|N|N| +2473466|AAAAAAAAKPNLFCAA|2060-01-07|1920|8350|641|2060|3|1|7|1|2060|641|8350|Wednesday|2060Q1|N|N|N|2473460|2473459|2473101|2473374|N|N|N|N|N| +2473467|AAAAAAAALPNLFCAA|2060-01-08|1920|8350|641|2060|4|1|8|1|2060|641|8350|Thursday|2060Q1|N|N|N|2473460|2473459|2473102|2473375|N|N|N|N|N| +2473468|AAAAAAAAMPNLFCAA|2060-01-09|1920|8350|641|2060|5|1|9|1|2060|641|8350|Friday|2060Q1|N|Y|N|2473460|2473459|2473103|2473376|N|N|N|N|N| +2473469|AAAAAAAANPNLFCAA|2060-01-10|1920|8350|641|2060|6|1|10|1|2060|641|8350|Saturday|2060Q1|N|Y|N|2473460|2473459|2473104|2473377|N|N|N|N|N| +2473470|AAAAAAAAOPNLFCAA|2060-01-11|1920|8350|641|2060|0|1|11|1|2060|641|8350|Sunday|2060Q1|N|N|N|2473460|2473459|2473105|2473378|N|N|N|N|N| +2473471|AAAAAAAAPPNLFCAA|2060-01-12|1920|8350|641|2060|1|1|12|1|2060|641|8350|Monday|2060Q1|N|N|N|2473460|2473459|2473106|2473379|N|N|N|N|N| +2473472|AAAAAAAAAAOLFCAA|2060-01-13|1920|8351|641|2060|2|1|13|1|2060|641|8351|Tuesday|2060Q1|N|N|N|2473460|2473459|2473107|2473380|N|N|N|N|N| +2473473|AAAAAAAABAOLFCAA|2060-01-14|1920|8351|641|2060|3|1|14|1|2060|641|8351|Wednesday|2060Q1|N|N|N|2473460|2473459|2473108|2473381|N|N|N|N|N| +2473474|AAAAAAAACAOLFCAA|2060-01-15|1920|8351|641|2060|4|1|15|1|2060|641|8351|Thursday|2060Q1|N|N|N|2473460|2473459|2473109|2473382|N|N|N|N|N| +2473475|AAAAAAAADAOLFCAA|2060-01-16|1920|8351|641|2060|5|1|16|1|2060|641|8351|Friday|2060Q1|N|Y|N|2473460|2473459|2473110|2473383|N|N|N|N|N| +2473476|AAAAAAAAEAOLFCAA|2060-01-17|1920|8351|641|2060|6|1|17|1|2060|641|8351|Saturday|2060Q1|N|Y|N|2473460|2473459|2473111|2473384|N|N|N|N|N| +2473477|AAAAAAAAFAOLFCAA|2060-01-18|1920|8351|641|2060|0|1|18|1|2060|641|8351|Sunday|2060Q1|N|N|N|2473460|2473459|2473112|2473385|N|N|N|N|N| +2473478|AAAAAAAAGAOLFCAA|2060-01-19|1920|8351|641|2060|1|1|19|1|2060|641|8351|Monday|2060Q1|N|N|N|2473460|2473459|2473113|2473386|N|N|N|N|N| +2473479|AAAAAAAAHAOLFCAA|2060-01-20|1920|8352|641|2060|2|1|20|1|2060|641|8352|Tuesday|2060Q1|N|N|N|2473460|2473459|2473114|2473387|N|N|N|N|N| +2473480|AAAAAAAAIAOLFCAA|2060-01-21|1920|8352|641|2060|3|1|21|1|2060|641|8352|Wednesday|2060Q1|N|N|N|2473460|2473459|2473115|2473388|N|N|N|N|N| +2473481|AAAAAAAAJAOLFCAA|2060-01-22|1920|8352|641|2060|4|1|22|1|2060|641|8352|Thursday|2060Q1|N|N|N|2473460|2473459|2473116|2473389|N|N|N|N|N| +2473482|AAAAAAAAKAOLFCAA|2060-01-23|1920|8352|641|2060|5|1|23|1|2060|641|8352|Friday|2060Q1|N|Y|N|2473460|2473459|2473117|2473390|N|N|N|N|N| +2473483|AAAAAAAALAOLFCAA|2060-01-24|1920|8352|641|2060|6|1|24|1|2060|641|8352|Saturday|2060Q1|N|Y|N|2473460|2473459|2473118|2473391|N|N|N|N|N| +2473484|AAAAAAAAMAOLFCAA|2060-01-25|1920|8352|641|2060|0|1|25|1|2060|641|8352|Sunday|2060Q1|N|N|N|2473460|2473459|2473119|2473392|N|N|N|N|N| +2473485|AAAAAAAANAOLFCAA|2060-01-26|1920|8352|641|2060|1|1|26|1|2060|641|8352|Monday|2060Q1|N|N|N|2473460|2473459|2473120|2473393|N|N|N|N|N| +2473486|AAAAAAAAOAOLFCAA|2060-01-27|1920|8353|641|2060|2|1|27|1|2060|641|8353|Tuesday|2060Q1|N|N|N|2473460|2473459|2473121|2473394|N|N|N|N|N| +2473487|AAAAAAAAPAOLFCAA|2060-01-28|1920|8353|641|2060|3|1|28|1|2060|641|8353|Wednesday|2060Q1|N|N|N|2473460|2473459|2473122|2473395|N|N|N|N|N| +2473488|AAAAAAAAABOLFCAA|2060-01-29|1920|8353|641|2060|4|1|29|1|2060|641|8353|Thursday|2060Q1|N|N|N|2473460|2473459|2473123|2473396|N|N|N|N|N| +2473489|AAAAAAAABBOLFCAA|2060-01-30|1920|8353|641|2060|5|1|30|1|2060|641|8353|Friday|2060Q1|N|Y|N|2473460|2473459|2473124|2473397|N|N|N|N|N| +2473490|AAAAAAAACBOLFCAA|2060-01-31|1920|8353|641|2060|6|1|31|1|2060|641|8353|Saturday|2060Q1|N|Y|N|2473460|2473459|2473125|2473398|N|N|N|N|N| +2473491|AAAAAAAADBOLFCAA|2060-02-01|1921|8353|641|2060|0|2|1|1|2060|641|8353|Sunday|2060Q1|N|N|N|2473491|2473521|2473126|2473399|N|N|N|N|N| +2473492|AAAAAAAAEBOLFCAA|2060-02-02|1921|8353|641|2060|1|2|2|1|2060|641|8353|Monday|2060Q1|N|N|N|2473491|2473521|2473127|2473400|N|N|N|N|N| +2473493|AAAAAAAAFBOLFCAA|2060-02-03|1921|8354|641|2060|2|2|3|1|2060|641|8354|Tuesday|2060Q1|N|N|N|2473491|2473521|2473128|2473401|N|N|N|N|N| +2473494|AAAAAAAAGBOLFCAA|2060-02-04|1921|8354|641|2060|3|2|4|1|2060|641|8354|Wednesday|2060Q1|N|N|N|2473491|2473521|2473129|2473402|N|N|N|N|N| +2473495|AAAAAAAAHBOLFCAA|2060-02-05|1921|8354|641|2060|4|2|5|1|2060|641|8354|Thursday|2060Q1|N|N|N|2473491|2473521|2473130|2473403|N|N|N|N|N| +2473496|AAAAAAAAIBOLFCAA|2060-02-06|1921|8354|641|2060|5|2|6|1|2060|641|8354|Friday|2060Q1|N|Y|N|2473491|2473521|2473131|2473404|N|N|N|N|N| +2473497|AAAAAAAAJBOLFCAA|2060-02-07|1921|8354|641|2060|6|2|7|1|2060|641|8354|Saturday|2060Q1|N|Y|N|2473491|2473521|2473132|2473405|N|N|N|N|N| +2473498|AAAAAAAAKBOLFCAA|2060-02-08|1921|8354|641|2060|0|2|8|1|2060|641|8354|Sunday|2060Q1|N|N|N|2473491|2473521|2473133|2473406|N|N|N|N|N| +2473499|AAAAAAAALBOLFCAA|2060-02-09|1921|8354|641|2060|1|2|9|1|2060|641|8354|Monday|2060Q1|N|N|N|2473491|2473521|2473134|2473407|N|N|N|N|N| +2473500|AAAAAAAAMBOLFCAA|2060-02-10|1921|8355|641|2060|2|2|10|1|2060|641|8355|Tuesday|2060Q1|N|N|N|2473491|2473521|2473135|2473408|N|N|N|N|N| +2473501|AAAAAAAANBOLFCAA|2060-02-11|1921|8355|641|2060|3|2|11|1|2060|641|8355|Wednesday|2060Q1|N|N|N|2473491|2473521|2473136|2473409|N|N|N|N|N| +2473502|AAAAAAAAOBOLFCAA|2060-02-12|1921|8355|641|2060|4|2|12|1|2060|641|8355|Thursday|2060Q1|N|N|N|2473491|2473521|2473137|2473410|N|N|N|N|N| +2473503|AAAAAAAAPBOLFCAA|2060-02-13|1921|8355|641|2060|5|2|13|1|2060|641|8355|Friday|2060Q1|N|Y|N|2473491|2473521|2473138|2473411|N|N|N|N|N| +2473504|AAAAAAAAACOLFCAA|2060-02-14|1921|8355|641|2060|6|2|14|1|2060|641|8355|Saturday|2060Q1|N|Y|N|2473491|2473521|2473139|2473412|N|N|N|N|N| +2473505|AAAAAAAABCOLFCAA|2060-02-15|1921|8355|641|2060|0|2|15|1|2060|641|8355|Sunday|2060Q1|N|N|N|2473491|2473521|2473140|2473413|N|N|N|N|N| +2473506|AAAAAAAACCOLFCAA|2060-02-16|1921|8355|641|2060|1|2|16|1|2060|641|8355|Monday|2060Q1|N|N|N|2473491|2473521|2473141|2473414|N|N|N|N|N| +2473507|AAAAAAAADCOLFCAA|2060-02-17|1921|8356|641|2060|2|2|17|1|2060|641|8356|Tuesday|2060Q1|N|N|N|2473491|2473521|2473142|2473415|N|N|N|N|N| +2473508|AAAAAAAAECOLFCAA|2060-02-18|1921|8356|641|2060|3|2|18|1|2060|641|8356|Wednesday|2060Q1|N|N|N|2473491|2473521|2473143|2473416|N|N|N|N|N| +2473509|AAAAAAAAFCOLFCAA|2060-02-19|1921|8356|641|2060|4|2|19|1|2060|641|8356|Thursday|2060Q1|N|N|N|2473491|2473521|2473144|2473417|N|N|N|N|N| +2473510|AAAAAAAAGCOLFCAA|2060-02-20|1921|8356|641|2060|5|2|20|1|2060|641|8356|Friday|2060Q1|N|Y|N|2473491|2473521|2473145|2473418|N|N|N|N|N| +2473511|AAAAAAAAHCOLFCAA|2060-02-21|1921|8356|641|2060|6|2|21|1|2060|641|8356|Saturday|2060Q1|N|Y|N|2473491|2473521|2473146|2473419|N|N|N|N|N| +2473512|AAAAAAAAICOLFCAA|2060-02-22|1921|8356|641|2060|0|2|22|1|2060|641|8356|Sunday|2060Q1|N|N|N|2473491|2473521|2473147|2473420|N|N|N|N|N| +2473513|AAAAAAAAJCOLFCAA|2060-02-23|1921|8356|641|2060|1|2|23|1|2060|641|8356|Monday|2060Q1|N|N|N|2473491|2473521|2473148|2473421|N|N|N|N|N| +2473514|AAAAAAAAKCOLFCAA|2060-02-24|1921|8357|641|2060|2|2|24|1|2060|641|8357|Tuesday|2060Q1|N|N|N|2473491|2473521|2473149|2473422|N|N|N|N|N| +2473515|AAAAAAAALCOLFCAA|2060-02-25|1921|8357|641|2060|3|2|25|1|2060|641|8357|Wednesday|2060Q1|N|N|N|2473491|2473521|2473150|2473423|N|N|N|N|N| +2473516|AAAAAAAAMCOLFCAA|2060-02-26|1921|8357|641|2060|4|2|26|1|2060|641|8357|Thursday|2060Q1|N|N|N|2473491|2473521|2473151|2473424|N|N|N|N|N| +2473517|AAAAAAAANCOLFCAA|2060-02-27|1921|8357|641|2060|5|2|27|1|2060|641|8357|Friday|2060Q1|N|Y|N|2473491|2473521|2473152|2473425|N|N|N|N|N| +2473518|AAAAAAAAOCOLFCAA|2060-02-28|1921|8357|641|2060|6|2|28|1|2060|641|8357|Saturday|2060Q1|N|Y|N|2473491|2473521|2473153|2473426|N|N|N|N|N| +2473519|AAAAAAAAPCOLFCAA|2060-02-29|1921|8357|641|2060|0|2|29|1|2060|641|8357|Sunday|2060Q1|N|N|N|2473491|2473521|2473153|2473427|N|N|N|N|N| +2473520|AAAAAAAAADOLFCAA|2060-03-01|1922|8357|642|2060|1|3|1|1|2060|642|8357|Monday|2060Q1|N|N|N|2473520|2473579|2473154|2473428|N|N|N|N|N| +2473521|AAAAAAAABDOLFCAA|2060-03-02|1922|8358|642|2060|2|3|2|1|2060|642|8358|Tuesday|2060Q1|N|N|N|2473520|2473579|2473155|2473429|N|N|N|N|N| +2473522|AAAAAAAACDOLFCAA|2060-03-03|1922|8358|642|2060|3|3|3|1|2060|642|8358|Wednesday|2060Q1|N|N|N|2473520|2473579|2473156|2473430|N|N|N|N|N| +2473523|AAAAAAAADDOLFCAA|2060-03-04|1922|8358|642|2060|4|3|4|1|2060|642|8358|Thursday|2060Q1|N|N|N|2473520|2473579|2473157|2473431|N|N|N|N|N| +2473524|AAAAAAAAEDOLFCAA|2060-03-05|1922|8358|642|2060|5|3|5|1|2060|642|8358|Friday|2060Q1|N|Y|N|2473520|2473579|2473158|2473432|N|N|N|N|N| +2473525|AAAAAAAAFDOLFCAA|2060-03-06|1922|8358|642|2060|6|3|6|1|2060|642|8358|Saturday|2060Q1|N|Y|N|2473520|2473579|2473159|2473433|N|N|N|N|N| +2473526|AAAAAAAAGDOLFCAA|2060-03-07|1922|8358|642|2060|0|3|7|1|2060|642|8358|Sunday|2060Q1|N|N|N|2473520|2473579|2473160|2473434|N|N|N|N|N| +2473527|AAAAAAAAHDOLFCAA|2060-03-08|1922|8358|642|2060|1|3|8|1|2060|642|8358|Monday|2060Q1|N|N|N|2473520|2473579|2473161|2473435|N|N|N|N|N| +2473528|AAAAAAAAIDOLFCAA|2060-03-09|1922|8359|642|2060|2|3|9|1|2060|642|8359|Tuesday|2060Q1|N|N|N|2473520|2473579|2473162|2473436|N|N|N|N|N| +2473529|AAAAAAAAJDOLFCAA|2060-03-10|1922|8359|642|2060|3|3|10|1|2060|642|8359|Wednesday|2060Q1|N|N|N|2473520|2473579|2473163|2473437|N|N|N|N|N| +2473530|AAAAAAAAKDOLFCAA|2060-03-11|1922|8359|642|2060|4|3|11|1|2060|642|8359|Thursday|2060Q1|N|N|N|2473520|2473579|2473164|2473438|N|N|N|N|N| +2473531|AAAAAAAALDOLFCAA|2060-03-12|1922|8359|642|2060|5|3|12|1|2060|642|8359|Friday|2060Q1|N|Y|N|2473520|2473579|2473165|2473439|N|N|N|N|N| +2473532|AAAAAAAAMDOLFCAA|2060-03-13|1922|8359|642|2060|6|3|13|1|2060|642|8359|Saturday|2060Q1|N|Y|N|2473520|2473579|2473166|2473440|N|N|N|N|N| +2473533|AAAAAAAANDOLFCAA|2060-03-14|1922|8359|642|2060|0|3|14|1|2060|642|8359|Sunday|2060Q1|N|N|N|2473520|2473579|2473167|2473441|N|N|N|N|N| +2473534|AAAAAAAAODOLFCAA|2060-03-15|1922|8359|642|2060|1|3|15|1|2060|642|8359|Monday|2060Q1|N|N|N|2473520|2473579|2473168|2473442|N|N|N|N|N| +2473535|AAAAAAAAPDOLFCAA|2060-03-16|1922|8360|642|2060|2|3|16|1|2060|642|8360|Tuesday|2060Q1|N|N|N|2473520|2473579|2473169|2473443|N|N|N|N|N| +2473536|AAAAAAAAAEOLFCAA|2060-03-17|1922|8360|642|2060|3|3|17|1|2060|642|8360|Wednesday|2060Q1|N|N|N|2473520|2473579|2473170|2473444|N|N|N|N|N| +2473537|AAAAAAAABEOLFCAA|2060-03-18|1922|8360|642|2060|4|3|18|1|2060|642|8360|Thursday|2060Q1|N|N|N|2473520|2473579|2473171|2473445|N|N|N|N|N| +2473538|AAAAAAAACEOLFCAA|2060-03-19|1922|8360|642|2060|5|3|19|1|2060|642|8360|Friday|2060Q1|N|Y|N|2473520|2473579|2473172|2473446|N|N|N|N|N| +2473539|AAAAAAAADEOLFCAA|2060-03-20|1922|8360|642|2060|6|3|20|1|2060|642|8360|Saturday|2060Q1|N|Y|N|2473520|2473579|2473173|2473447|N|N|N|N|N| +2473540|AAAAAAAAEEOLFCAA|2060-03-21|1922|8360|642|2060|0|3|21|1|2060|642|8360|Sunday|2060Q1|N|N|N|2473520|2473579|2473174|2473448|N|N|N|N|N| +2473541|AAAAAAAAFEOLFCAA|2060-03-22|1922|8360|642|2060|1|3|22|1|2060|642|8360|Monday|2060Q1|N|N|N|2473520|2473579|2473175|2473449|N|N|N|N|N| +2473542|AAAAAAAAGEOLFCAA|2060-03-23|1922|8361|642|2060|2|3|23|1|2060|642|8361|Tuesday|2060Q1|N|N|N|2473520|2473579|2473176|2473450|N|N|N|N|N| +2473543|AAAAAAAAHEOLFCAA|2060-03-24|1922|8361|642|2060|3|3|24|1|2060|642|8361|Wednesday|2060Q1|N|N|N|2473520|2473579|2473177|2473451|N|N|N|N|N| +2473544|AAAAAAAAIEOLFCAA|2060-03-25|1922|8361|642|2060|4|3|25|1|2060|642|8361|Thursday|2060Q1|N|N|N|2473520|2473579|2473178|2473452|N|N|N|N|N| +2473545|AAAAAAAAJEOLFCAA|2060-03-26|1922|8361|642|2060|5|3|26|1|2060|642|8361|Friday|2060Q1|N|Y|N|2473520|2473579|2473179|2473453|N|N|N|N|N| +2473546|AAAAAAAAKEOLFCAA|2060-03-27|1922|8361|642|2060|6|3|27|1|2060|642|8361|Saturday|2060Q1|N|Y|N|2473520|2473579|2473180|2473454|N|N|N|N|N| +2473547|AAAAAAAALEOLFCAA|2060-03-28|1922|8361|642|2060|0|3|28|1|2060|642|8361|Sunday|2060Q1|N|N|N|2473520|2473579|2473181|2473455|N|N|N|N|N| +2473548|AAAAAAAAMEOLFCAA|2060-03-29|1922|8361|642|2060|1|3|29|1|2060|642|8361|Monday|2060Q1|N|N|N|2473520|2473579|2473182|2473456|N|N|N|N|N| +2473549|AAAAAAAANEOLFCAA|2060-03-30|1922|8362|642|2060|2|3|30|1|2060|642|8362|Tuesday|2060Q1|N|N|N|2473520|2473579|2473183|2473457|N|N|N|N|N| +2473550|AAAAAAAAOEOLFCAA|2060-03-31|1922|8362|642|2060|3|3|31|1|2060|642|8362|Wednesday|2060Q1|N|N|N|2473520|2473579|2473184|2473458|N|N|N|N|N| +2473551|AAAAAAAAPEOLFCAA|2060-04-01|1923|8362|642|2060|4|4|1|2|2060|642|8362|Thursday|2060Q2|N|N|N|2473551|2473641|2473185|2473460|N|N|N|N|N| +2473552|AAAAAAAAAFOLFCAA|2060-04-02|1923|8362|642|2060|5|4|2|2|2060|642|8362|Friday|2060Q2|N|Y|N|2473551|2473641|2473186|2473461|N|N|N|N|N| +2473553|AAAAAAAABFOLFCAA|2060-04-03|1923|8362|642|2060|6|4|3|2|2060|642|8362|Saturday|2060Q2|N|Y|N|2473551|2473641|2473187|2473462|N|N|N|N|N| +2473554|AAAAAAAACFOLFCAA|2060-04-04|1923|8362|642|2060|0|4|4|2|2060|642|8362|Sunday|2060Q2|N|N|N|2473551|2473641|2473188|2473463|N|N|N|N|N| +2473555|AAAAAAAADFOLFCAA|2060-04-05|1923|8362|642|2060|1|4|5|2|2060|642|8362|Monday|2060Q2|N|N|N|2473551|2473641|2473189|2473464|N|N|N|N|N| +2473556|AAAAAAAAEFOLFCAA|2060-04-06|1923|8363|642|2060|2|4|6|2|2060|642|8363|Tuesday|2060Q2|N|N|N|2473551|2473641|2473190|2473465|N|N|N|N|N| +2473557|AAAAAAAAFFOLFCAA|2060-04-07|1923|8363|642|2060|3|4|7|2|2060|642|8363|Wednesday|2060Q2|N|N|N|2473551|2473641|2473191|2473466|N|N|N|N|N| +2473558|AAAAAAAAGFOLFCAA|2060-04-08|1923|8363|642|2060|4|4|8|2|2060|642|8363|Thursday|2060Q2|N|N|N|2473551|2473641|2473192|2473467|N|N|N|N|N| +2473559|AAAAAAAAHFOLFCAA|2060-04-09|1923|8363|642|2060|5|4|9|2|2060|642|8363|Friday|2060Q2|N|Y|N|2473551|2473641|2473193|2473468|N|N|N|N|N| +2473560|AAAAAAAAIFOLFCAA|2060-04-10|1923|8363|642|2060|6|4|10|2|2060|642|8363|Saturday|2060Q2|N|Y|N|2473551|2473641|2473194|2473469|N|N|N|N|N| +2473561|AAAAAAAAJFOLFCAA|2060-04-11|1923|8363|642|2060|0|4|11|2|2060|642|8363|Sunday|2060Q2|N|N|N|2473551|2473641|2473195|2473470|N|N|N|N|N| +2473562|AAAAAAAAKFOLFCAA|2060-04-12|1923|8363|642|2060|1|4|12|2|2060|642|8363|Monday|2060Q2|N|N|N|2473551|2473641|2473196|2473471|N|N|N|N|N| +2473563|AAAAAAAALFOLFCAA|2060-04-13|1923|8364|642|2060|2|4|13|2|2060|642|8364|Tuesday|2060Q2|N|N|N|2473551|2473641|2473197|2473472|N|N|N|N|N| +2473564|AAAAAAAAMFOLFCAA|2060-04-14|1923|8364|642|2060|3|4|14|2|2060|642|8364|Wednesday|2060Q2|N|N|N|2473551|2473641|2473198|2473473|N|N|N|N|N| +2473565|AAAAAAAANFOLFCAA|2060-04-15|1923|8364|642|2060|4|4|15|2|2060|642|8364|Thursday|2060Q2|N|N|N|2473551|2473641|2473199|2473474|N|N|N|N|N| +2473566|AAAAAAAAOFOLFCAA|2060-04-16|1923|8364|642|2060|5|4|16|2|2060|642|8364|Friday|2060Q2|N|Y|N|2473551|2473641|2473200|2473475|N|N|N|N|N| +2473567|AAAAAAAAPFOLFCAA|2060-04-17|1923|8364|642|2060|6|4|17|2|2060|642|8364|Saturday|2060Q2|N|Y|N|2473551|2473641|2473201|2473476|N|N|N|N|N| +2473568|AAAAAAAAAGOLFCAA|2060-04-18|1923|8364|642|2060|0|4|18|2|2060|642|8364|Sunday|2060Q2|N|N|N|2473551|2473641|2473202|2473477|N|N|N|N|N| +2473569|AAAAAAAABGOLFCAA|2060-04-19|1923|8364|642|2060|1|4|19|2|2060|642|8364|Monday|2060Q2|N|N|N|2473551|2473641|2473203|2473478|N|N|N|N|N| +2473570|AAAAAAAACGOLFCAA|2060-04-20|1923|8365|642|2060|2|4|20|2|2060|642|8365|Tuesday|2060Q2|N|N|N|2473551|2473641|2473204|2473479|N|N|N|N|N| +2473571|AAAAAAAADGOLFCAA|2060-04-21|1923|8365|642|2060|3|4|21|2|2060|642|8365|Wednesday|2060Q2|N|N|N|2473551|2473641|2473205|2473480|N|N|N|N|N| +2473572|AAAAAAAAEGOLFCAA|2060-04-22|1923|8365|642|2060|4|4|22|2|2060|642|8365|Thursday|2060Q2|N|N|N|2473551|2473641|2473206|2473481|N|N|N|N|N| +2473573|AAAAAAAAFGOLFCAA|2060-04-23|1923|8365|642|2060|5|4|23|2|2060|642|8365|Friday|2060Q2|N|Y|N|2473551|2473641|2473207|2473482|N|N|N|N|N| +2473574|AAAAAAAAGGOLFCAA|2060-04-24|1923|8365|642|2060|6|4|24|2|2060|642|8365|Saturday|2060Q2|N|Y|N|2473551|2473641|2473208|2473483|N|N|N|N|N| +2473575|AAAAAAAAHGOLFCAA|2060-04-25|1923|8365|642|2060|0|4|25|2|2060|642|8365|Sunday|2060Q2|N|N|N|2473551|2473641|2473209|2473484|N|N|N|N|N| +2473576|AAAAAAAAIGOLFCAA|2060-04-26|1923|8365|642|2060|1|4|26|2|2060|642|8365|Monday|2060Q2|N|N|N|2473551|2473641|2473210|2473485|N|N|N|N|N| +2473577|AAAAAAAAJGOLFCAA|2060-04-27|1923|8366|642|2060|2|4|27|2|2060|642|8366|Tuesday|2060Q2|N|N|N|2473551|2473641|2473211|2473486|N|N|N|N|N| +2473578|AAAAAAAAKGOLFCAA|2060-04-28|1923|8366|642|2060|3|4|28|2|2060|642|8366|Wednesday|2060Q2|N|N|N|2473551|2473641|2473212|2473487|N|N|N|N|N| +2473579|AAAAAAAALGOLFCAA|2060-04-29|1923|8366|642|2060|4|4|29|2|2060|642|8366|Thursday|2060Q2|N|N|N|2473551|2473641|2473213|2473488|N|N|N|N|N| +2473580|AAAAAAAAMGOLFCAA|2060-04-30|1923|8366|642|2060|5|4|30|2|2060|642|8366|Friday|2060Q2|N|Y|N|2473551|2473641|2473214|2473489|N|N|N|N|N| +2473581|AAAAAAAANGOLFCAA|2060-05-01|1924|8366|642|2060|6|5|1|2|2060|642|8366|Saturday|2060Q2|N|Y|N|2473581|2473701|2473215|2473490|N|N|N|N|N| +2473582|AAAAAAAAOGOLFCAA|2060-05-02|1924|8366|642|2060|0|5|2|2|2060|642|8366|Sunday|2060Q2|N|N|N|2473581|2473701|2473216|2473491|N|N|N|N|N| +2473583|AAAAAAAAPGOLFCAA|2060-05-03|1924|8366|642|2060|1|5|3|2|2060|642|8366|Monday|2060Q2|N|N|N|2473581|2473701|2473217|2473492|N|N|N|N|N| +2473584|AAAAAAAAAHOLFCAA|2060-05-04|1924|8367|642|2060|2|5|4|2|2060|642|8367|Tuesday|2060Q2|N|N|N|2473581|2473701|2473218|2473493|N|N|N|N|N| +2473585|AAAAAAAABHOLFCAA|2060-05-05|1924|8367|642|2060|3|5|5|2|2060|642|8367|Wednesday|2060Q2|N|N|N|2473581|2473701|2473219|2473494|N|N|N|N|N| +2473586|AAAAAAAACHOLFCAA|2060-05-06|1924|8367|642|2060|4|5|6|2|2060|642|8367|Thursday|2060Q2|N|N|N|2473581|2473701|2473220|2473495|N|N|N|N|N| +2473587|AAAAAAAADHOLFCAA|2060-05-07|1924|8367|642|2060|5|5|7|2|2060|642|8367|Friday|2060Q2|N|Y|N|2473581|2473701|2473221|2473496|N|N|N|N|N| +2473588|AAAAAAAAEHOLFCAA|2060-05-08|1924|8367|642|2060|6|5|8|2|2060|642|8367|Saturday|2060Q2|N|Y|N|2473581|2473701|2473222|2473497|N|N|N|N|N| +2473589|AAAAAAAAFHOLFCAA|2060-05-09|1924|8367|642|2060|0|5|9|2|2060|642|8367|Sunday|2060Q2|N|N|N|2473581|2473701|2473223|2473498|N|N|N|N|N| +2473590|AAAAAAAAGHOLFCAA|2060-05-10|1924|8367|642|2060|1|5|10|2|2060|642|8367|Monday|2060Q2|N|N|N|2473581|2473701|2473224|2473499|N|N|N|N|N| +2473591|AAAAAAAAHHOLFCAA|2060-05-11|1924|8368|642|2060|2|5|11|2|2060|642|8368|Tuesday|2060Q2|N|N|N|2473581|2473701|2473225|2473500|N|N|N|N|N| +2473592|AAAAAAAAIHOLFCAA|2060-05-12|1924|8368|642|2060|3|5|12|2|2060|642|8368|Wednesday|2060Q2|N|N|N|2473581|2473701|2473226|2473501|N|N|N|N|N| +2473593|AAAAAAAAJHOLFCAA|2060-05-13|1924|8368|642|2060|4|5|13|2|2060|642|8368|Thursday|2060Q2|N|N|N|2473581|2473701|2473227|2473502|N|N|N|N|N| +2473594|AAAAAAAAKHOLFCAA|2060-05-14|1924|8368|642|2060|5|5|14|2|2060|642|8368|Friday|2060Q2|N|Y|N|2473581|2473701|2473228|2473503|N|N|N|N|N| +2473595|AAAAAAAALHOLFCAA|2060-05-15|1924|8368|642|2060|6|5|15|2|2060|642|8368|Saturday|2060Q2|N|Y|N|2473581|2473701|2473229|2473504|N|N|N|N|N| +2473596|AAAAAAAAMHOLFCAA|2060-05-16|1924|8368|642|2060|0|5|16|2|2060|642|8368|Sunday|2060Q2|N|N|N|2473581|2473701|2473230|2473505|N|N|N|N|N| +2473597|AAAAAAAANHOLFCAA|2060-05-17|1924|8368|642|2060|1|5|17|2|2060|642|8368|Monday|2060Q2|N|N|N|2473581|2473701|2473231|2473506|N|N|N|N|N| +2473598|AAAAAAAAOHOLFCAA|2060-05-18|1924|8369|642|2060|2|5|18|2|2060|642|8369|Tuesday|2060Q2|N|N|N|2473581|2473701|2473232|2473507|N|N|N|N|N| +2473599|AAAAAAAAPHOLFCAA|2060-05-19|1924|8369|642|2060|3|5|19|2|2060|642|8369|Wednesday|2060Q2|N|N|N|2473581|2473701|2473233|2473508|N|N|N|N|N| +2473600|AAAAAAAAAIOLFCAA|2060-05-20|1924|8369|642|2060|4|5|20|2|2060|642|8369|Thursday|2060Q2|N|N|N|2473581|2473701|2473234|2473509|N|N|N|N|N| +2473601|AAAAAAAABIOLFCAA|2060-05-21|1924|8369|642|2060|5|5|21|2|2060|642|8369|Friday|2060Q2|N|Y|N|2473581|2473701|2473235|2473510|N|N|N|N|N| +2473602|AAAAAAAACIOLFCAA|2060-05-22|1924|8369|642|2060|6|5|22|2|2060|642|8369|Saturday|2060Q2|N|Y|N|2473581|2473701|2473236|2473511|N|N|N|N|N| +2473603|AAAAAAAADIOLFCAA|2060-05-23|1924|8369|642|2060|0|5|23|2|2060|642|8369|Sunday|2060Q2|N|N|N|2473581|2473701|2473237|2473512|N|N|N|N|N| +2473604|AAAAAAAAEIOLFCAA|2060-05-24|1924|8369|642|2060|1|5|24|2|2060|642|8369|Monday|2060Q2|N|N|N|2473581|2473701|2473238|2473513|N|N|N|N|N| +2473605|AAAAAAAAFIOLFCAA|2060-05-25|1924|8370|642|2060|2|5|25|2|2060|642|8370|Tuesday|2060Q2|N|N|N|2473581|2473701|2473239|2473514|N|N|N|N|N| +2473606|AAAAAAAAGIOLFCAA|2060-05-26|1924|8370|642|2060|3|5|26|2|2060|642|8370|Wednesday|2060Q2|N|N|N|2473581|2473701|2473240|2473515|N|N|N|N|N| +2473607|AAAAAAAAHIOLFCAA|2060-05-27|1924|8370|642|2060|4|5|27|2|2060|642|8370|Thursday|2060Q2|N|N|N|2473581|2473701|2473241|2473516|N|N|N|N|N| +2473608|AAAAAAAAIIOLFCAA|2060-05-28|1924|8370|642|2060|5|5|28|2|2060|642|8370|Friday|2060Q2|N|Y|N|2473581|2473701|2473242|2473517|N|N|N|N|N| +2473609|AAAAAAAAJIOLFCAA|2060-05-29|1924|8370|642|2060|6|5|29|2|2060|642|8370|Saturday|2060Q2|N|Y|N|2473581|2473701|2473243|2473518|N|N|N|N|N| +2473610|AAAAAAAAKIOLFCAA|2060-05-30|1924|8370|642|2060|0|5|30|2|2060|642|8370|Sunday|2060Q2|N|N|N|2473581|2473701|2473244|2473519|N|N|N|N|N| +2473611|AAAAAAAALIOLFCAA|2060-05-31|1924|8370|642|2060|1|5|31|2|2060|642|8370|Monday|2060Q2|N|N|N|2473581|2473701|2473245|2473520|N|N|N|N|N| +2473612|AAAAAAAAMIOLFCAA|2060-06-01|1925|8371|643|2060|2|6|1|2|2060|643|8371|Tuesday|2060Q2|N|N|N|2473612|2473763|2473246|2473521|N|N|N|N|N| +2473613|AAAAAAAANIOLFCAA|2060-06-02|1925|8371|643|2060|3|6|2|2|2060|643|8371|Wednesday|2060Q2|N|N|N|2473612|2473763|2473247|2473522|N|N|N|N|N| +2473614|AAAAAAAAOIOLFCAA|2060-06-03|1925|8371|643|2060|4|6|3|2|2060|643|8371|Thursday|2060Q2|N|N|N|2473612|2473763|2473248|2473523|N|N|N|N|N| +2473615|AAAAAAAAPIOLFCAA|2060-06-04|1925|8371|643|2060|5|6|4|2|2060|643|8371|Friday|2060Q2|N|Y|N|2473612|2473763|2473249|2473524|N|N|N|N|N| +2473616|AAAAAAAAAJOLFCAA|2060-06-05|1925|8371|643|2060|6|6|5|2|2060|643|8371|Saturday|2060Q2|N|Y|N|2473612|2473763|2473250|2473525|N|N|N|N|N| +2473617|AAAAAAAABJOLFCAA|2060-06-06|1925|8371|643|2060|0|6|6|2|2060|643|8371|Sunday|2060Q2|N|N|N|2473612|2473763|2473251|2473526|N|N|N|N|N| +2473618|AAAAAAAACJOLFCAA|2060-06-07|1925|8371|643|2060|1|6|7|2|2060|643|8371|Monday|2060Q2|N|N|N|2473612|2473763|2473252|2473527|N|N|N|N|N| +2473619|AAAAAAAADJOLFCAA|2060-06-08|1925|8372|643|2060|2|6|8|2|2060|643|8372|Tuesday|2060Q2|N|N|N|2473612|2473763|2473253|2473528|N|N|N|N|N| +2473620|AAAAAAAAEJOLFCAA|2060-06-09|1925|8372|643|2060|3|6|9|2|2060|643|8372|Wednesday|2060Q2|N|N|N|2473612|2473763|2473254|2473529|N|N|N|N|N| +2473621|AAAAAAAAFJOLFCAA|2060-06-10|1925|8372|643|2060|4|6|10|2|2060|643|8372|Thursday|2060Q2|N|N|N|2473612|2473763|2473255|2473530|N|N|N|N|N| +2473622|AAAAAAAAGJOLFCAA|2060-06-11|1925|8372|643|2060|5|6|11|2|2060|643|8372|Friday|2060Q2|N|Y|N|2473612|2473763|2473256|2473531|N|N|N|N|N| +2473623|AAAAAAAAHJOLFCAA|2060-06-12|1925|8372|643|2060|6|6|12|2|2060|643|8372|Saturday|2060Q2|N|Y|N|2473612|2473763|2473257|2473532|N|N|N|N|N| +2473624|AAAAAAAAIJOLFCAA|2060-06-13|1925|8372|643|2060|0|6|13|2|2060|643|8372|Sunday|2060Q2|N|N|N|2473612|2473763|2473258|2473533|N|N|N|N|N| +2473625|AAAAAAAAJJOLFCAA|2060-06-14|1925|8372|643|2060|1|6|14|2|2060|643|8372|Monday|2060Q2|N|N|N|2473612|2473763|2473259|2473534|N|N|N|N|N| +2473626|AAAAAAAAKJOLFCAA|2060-06-15|1925|8373|643|2060|2|6|15|2|2060|643|8373|Tuesday|2060Q2|N|N|N|2473612|2473763|2473260|2473535|N|N|N|N|N| +2473627|AAAAAAAALJOLFCAA|2060-06-16|1925|8373|643|2060|3|6|16|2|2060|643|8373|Wednesday|2060Q2|N|N|N|2473612|2473763|2473261|2473536|N|N|N|N|N| +2473628|AAAAAAAAMJOLFCAA|2060-06-17|1925|8373|643|2060|4|6|17|2|2060|643|8373|Thursday|2060Q2|N|N|N|2473612|2473763|2473262|2473537|N|N|N|N|N| +2473629|AAAAAAAANJOLFCAA|2060-06-18|1925|8373|643|2060|5|6|18|2|2060|643|8373|Friday|2060Q2|N|Y|N|2473612|2473763|2473263|2473538|N|N|N|N|N| +2473630|AAAAAAAAOJOLFCAA|2060-06-19|1925|8373|643|2060|6|6|19|2|2060|643|8373|Saturday|2060Q2|N|Y|N|2473612|2473763|2473264|2473539|N|N|N|N|N| +2473631|AAAAAAAAPJOLFCAA|2060-06-20|1925|8373|643|2060|0|6|20|2|2060|643|8373|Sunday|2060Q2|N|N|N|2473612|2473763|2473265|2473540|N|N|N|N|N| +2473632|AAAAAAAAAKOLFCAA|2060-06-21|1925|8373|643|2060|1|6|21|2|2060|643|8373|Monday|2060Q2|N|N|N|2473612|2473763|2473266|2473541|N|N|N|N|N| +2473633|AAAAAAAABKOLFCAA|2060-06-22|1925|8374|643|2060|2|6|22|2|2060|643|8374|Tuesday|2060Q2|N|N|N|2473612|2473763|2473267|2473542|N|N|N|N|N| +2473634|AAAAAAAACKOLFCAA|2060-06-23|1925|8374|643|2060|3|6|23|2|2060|643|8374|Wednesday|2060Q2|N|N|N|2473612|2473763|2473268|2473543|N|N|N|N|N| +2473635|AAAAAAAADKOLFCAA|2060-06-24|1925|8374|643|2060|4|6|24|2|2060|643|8374|Thursday|2060Q2|N|N|N|2473612|2473763|2473269|2473544|N|N|N|N|N| +2473636|AAAAAAAAEKOLFCAA|2060-06-25|1925|8374|643|2060|5|6|25|2|2060|643|8374|Friday|2060Q2|N|Y|N|2473612|2473763|2473270|2473545|N|N|N|N|N| +2473637|AAAAAAAAFKOLFCAA|2060-06-26|1925|8374|643|2060|6|6|26|2|2060|643|8374|Saturday|2060Q2|N|Y|N|2473612|2473763|2473271|2473546|N|N|N|N|N| +2473638|AAAAAAAAGKOLFCAA|2060-06-27|1925|8374|643|2060|0|6|27|2|2060|643|8374|Sunday|2060Q2|N|N|N|2473612|2473763|2473272|2473547|N|N|N|N|N| +2473639|AAAAAAAAHKOLFCAA|2060-06-28|1925|8374|643|2060|1|6|28|2|2060|643|8374|Monday|2060Q2|N|N|N|2473612|2473763|2473273|2473548|N|N|N|N|N| +2473640|AAAAAAAAIKOLFCAA|2060-06-29|1925|8375|643|2060|2|6|29|2|2060|643|8375|Tuesday|2060Q2|N|N|N|2473612|2473763|2473274|2473549|N|N|N|N|N| +2473641|AAAAAAAAJKOLFCAA|2060-06-30|1925|8375|643|2060|3|6|30|2|2060|643|8375|Wednesday|2060Q2|N|N|N|2473612|2473763|2473275|2473550|N|N|N|N|N| +2473642|AAAAAAAAKKOLFCAA|2060-07-01|1926|8375|643|2060|4|7|1|3|2060|643|8375|Thursday|2060Q3|N|N|N|2473642|2473823|2473276|2473551|N|N|N|N|N| +2473643|AAAAAAAALKOLFCAA|2060-07-02|1926|8375|643|2060|5|7|2|3|2060|643|8375|Friday|2060Q3|N|Y|N|2473642|2473823|2473277|2473552|N|N|N|N|N| +2473644|AAAAAAAAMKOLFCAA|2060-07-03|1926|8375|643|2060|6|7|3|3|2060|643|8375|Saturday|2060Q3|N|Y|N|2473642|2473823|2473278|2473553|N|N|N|N|N| +2473645|AAAAAAAANKOLFCAA|2060-07-04|1926|8375|643|2060|0|7|4|3|2060|643|8375|Sunday|2060Q3|Y|N|N|2473642|2473823|2473279|2473554|N|N|N|N|N| +2473646|AAAAAAAAOKOLFCAA|2060-07-05|1926|8375|643|2060|1|7|5|3|2060|643|8375|Monday|2060Q3|N|N|Y|2473642|2473823|2473280|2473555|N|N|N|N|N| +2473647|AAAAAAAAPKOLFCAA|2060-07-06|1926|8376|643|2060|2|7|6|3|2060|643|8376|Tuesday|2060Q3|N|N|N|2473642|2473823|2473281|2473556|N|N|N|N|N| +2473648|AAAAAAAAALOLFCAA|2060-07-07|1926|8376|643|2060|3|7|7|3|2060|643|8376|Wednesday|2060Q3|N|N|N|2473642|2473823|2473282|2473557|N|N|N|N|N| +2473649|AAAAAAAABLOLFCAA|2060-07-08|1926|8376|643|2060|4|7|8|3|2060|643|8376|Thursday|2060Q3|N|N|N|2473642|2473823|2473283|2473558|N|N|N|N|N| +2473650|AAAAAAAACLOLFCAA|2060-07-09|1926|8376|643|2060|5|7|9|3|2060|643|8376|Friday|2060Q3|N|Y|N|2473642|2473823|2473284|2473559|N|N|N|N|N| +2473651|AAAAAAAADLOLFCAA|2060-07-10|1926|8376|643|2060|6|7|10|3|2060|643|8376|Saturday|2060Q3|N|Y|N|2473642|2473823|2473285|2473560|N|N|N|N|N| +2473652|AAAAAAAAELOLFCAA|2060-07-11|1926|8376|643|2060|0|7|11|3|2060|643|8376|Sunday|2060Q3|N|N|N|2473642|2473823|2473286|2473561|N|N|N|N|N| +2473653|AAAAAAAAFLOLFCAA|2060-07-12|1926|8376|643|2060|1|7|12|3|2060|643|8376|Monday|2060Q3|N|N|N|2473642|2473823|2473287|2473562|N|N|N|N|N| +2473654|AAAAAAAAGLOLFCAA|2060-07-13|1926|8377|643|2060|2|7|13|3|2060|643|8377|Tuesday|2060Q3|N|N|N|2473642|2473823|2473288|2473563|N|N|N|N|N| +2473655|AAAAAAAAHLOLFCAA|2060-07-14|1926|8377|643|2060|3|7|14|3|2060|643|8377|Wednesday|2060Q3|N|N|N|2473642|2473823|2473289|2473564|N|N|N|N|N| +2473656|AAAAAAAAILOLFCAA|2060-07-15|1926|8377|643|2060|4|7|15|3|2060|643|8377|Thursday|2060Q3|N|N|N|2473642|2473823|2473290|2473565|N|N|N|N|N| +2473657|AAAAAAAAJLOLFCAA|2060-07-16|1926|8377|643|2060|5|7|16|3|2060|643|8377|Friday|2060Q3|N|Y|N|2473642|2473823|2473291|2473566|N|N|N|N|N| +2473658|AAAAAAAAKLOLFCAA|2060-07-17|1926|8377|643|2060|6|7|17|3|2060|643|8377|Saturday|2060Q3|N|Y|N|2473642|2473823|2473292|2473567|N|N|N|N|N| +2473659|AAAAAAAALLOLFCAA|2060-07-18|1926|8377|643|2060|0|7|18|3|2060|643|8377|Sunday|2060Q3|N|N|N|2473642|2473823|2473293|2473568|N|N|N|N|N| +2473660|AAAAAAAAMLOLFCAA|2060-07-19|1926|8377|643|2060|1|7|19|3|2060|643|8377|Monday|2060Q3|N|N|N|2473642|2473823|2473294|2473569|N|N|N|N|N| +2473661|AAAAAAAANLOLFCAA|2060-07-20|1926|8378|643|2060|2|7|20|3|2060|643|8378|Tuesday|2060Q3|N|N|N|2473642|2473823|2473295|2473570|N|N|N|N|N| +2473662|AAAAAAAAOLOLFCAA|2060-07-21|1926|8378|643|2060|3|7|21|3|2060|643|8378|Wednesday|2060Q3|N|N|N|2473642|2473823|2473296|2473571|N|N|N|N|N| +2473663|AAAAAAAAPLOLFCAA|2060-07-22|1926|8378|643|2060|4|7|22|3|2060|643|8378|Thursday|2060Q3|N|N|N|2473642|2473823|2473297|2473572|N|N|N|N|N| +2473664|AAAAAAAAAMOLFCAA|2060-07-23|1926|8378|643|2060|5|7|23|3|2060|643|8378|Friday|2060Q3|N|Y|N|2473642|2473823|2473298|2473573|N|N|N|N|N| +2473665|AAAAAAAABMOLFCAA|2060-07-24|1926|8378|643|2060|6|7|24|3|2060|643|8378|Saturday|2060Q3|N|Y|N|2473642|2473823|2473299|2473574|N|N|N|N|N| +2473666|AAAAAAAACMOLFCAA|2060-07-25|1926|8378|643|2060|0|7|25|3|2060|643|8378|Sunday|2060Q3|N|N|N|2473642|2473823|2473300|2473575|N|N|N|N|N| +2473667|AAAAAAAADMOLFCAA|2060-07-26|1926|8378|643|2060|1|7|26|3|2060|643|8378|Monday|2060Q3|N|N|N|2473642|2473823|2473301|2473576|N|N|N|N|N| +2473668|AAAAAAAAEMOLFCAA|2060-07-27|1926|8379|643|2060|2|7|27|3|2060|643|8379|Tuesday|2060Q3|N|N|N|2473642|2473823|2473302|2473577|N|N|N|N|N| +2473669|AAAAAAAAFMOLFCAA|2060-07-28|1926|8379|643|2060|3|7|28|3|2060|643|8379|Wednesday|2060Q3|N|N|N|2473642|2473823|2473303|2473578|N|N|N|N|N| +2473670|AAAAAAAAGMOLFCAA|2060-07-29|1926|8379|643|2060|4|7|29|3|2060|643|8379|Thursday|2060Q3|N|N|N|2473642|2473823|2473304|2473579|N|N|N|N|N| +2473671|AAAAAAAAHMOLFCAA|2060-07-30|1926|8379|643|2060|5|7|30|3|2060|643|8379|Friday|2060Q3|N|Y|N|2473642|2473823|2473305|2473580|N|N|N|N|N| +2473672|AAAAAAAAIMOLFCAA|2060-07-31|1926|8379|643|2060|6|7|31|3|2060|643|8379|Saturday|2060Q3|N|Y|N|2473642|2473823|2473306|2473581|N|N|N|N|N| +2473673|AAAAAAAAJMOLFCAA|2060-08-01|1927|8379|643|2060|0|8|1|3|2060|643|8379|Sunday|2060Q3|N|N|N|2473673|2473885|2473307|2473582|N|N|N|N|N| +2473674|AAAAAAAAKMOLFCAA|2060-08-02|1927|8379|643|2060|1|8|2|3|2060|643|8379|Monday|2060Q3|N|N|N|2473673|2473885|2473308|2473583|N|N|N|N|N| +2473675|AAAAAAAALMOLFCAA|2060-08-03|1927|8380|643|2060|2|8|3|3|2060|643|8380|Tuesday|2060Q3|N|N|N|2473673|2473885|2473309|2473584|N|N|N|N|N| +2473676|AAAAAAAAMMOLFCAA|2060-08-04|1927|8380|643|2060|3|8|4|3|2060|643|8380|Wednesday|2060Q3|N|N|N|2473673|2473885|2473310|2473585|N|N|N|N|N| +2473677|AAAAAAAANMOLFCAA|2060-08-05|1927|8380|643|2060|4|8|5|3|2060|643|8380|Thursday|2060Q3|N|N|N|2473673|2473885|2473311|2473586|N|N|N|N|N| +2473678|AAAAAAAAOMOLFCAA|2060-08-06|1927|8380|643|2060|5|8|6|3|2060|643|8380|Friday|2060Q3|N|Y|N|2473673|2473885|2473312|2473587|N|N|N|N|N| +2473679|AAAAAAAAPMOLFCAA|2060-08-07|1927|8380|643|2060|6|8|7|3|2060|643|8380|Saturday|2060Q3|N|Y|N|2473673|2473885|2473313|2473588|N|N|N|N|N| +2473680|AAAAAAAAANOLFCAA|2060-08-08|1927|8380|643|2060|0|8|8|3|2060|643|8380|Sunday|2060Q3|N|N|N|2473673|2473885|2473314|2473589|N|N|N|N|N| +2473681|AAAAAAAABNOLFCAA|2060-08-09|1927|8380|643|2060|1|8|9|3|2060|643|8380|Monday|2060Q3|N|N|N|2473673|2473885|2473315|2473590|N|N|N|N|N| +2473682|AAAAAAAACNOLFCAA|2060-08-10|1927|8381|643|2060|2|8|10|3|2060|643|8381|Tuesday|2060Q3|N|N|N|2473673|2473885|2473316|2473591|N|N|N|N|N| +2473683|AAAAAAAADNOLFCAA|2060-08-11|1927|8381|643|2060|3|8|11|3|2060|643|8381|Wednesday|2060Q3|N|N|N|2473673|2473885|2473317|2473592|N|N|N|N|N| +2473684|AAAAAAAAENOLFCAA|2060-08-12|1927|8381|643|2060|4|8|12|3|2060|643|8381|Thursday|2060Q3|N|N|N|2473673|2473885|2473318|2473593|N|N|N|N|N| +2473685|AAAAAAAAFNOLFCAA|2060-08-13|1927|8381|643|2060|5|8|13|3|2060|643|8381|Friday|2060Q3|N|Y|N|2473673|2473885|2473319|2473594|N|N|N|N|N| +2473686|AAAAAAAAGNOLFCAA|2060-08-14|1927|8381|643|2060|6|8|14|3|2060|643|8381|Saturday|2060Q3|N|Y|N|2473673|2473885|2473320|2473595|N|N|N|N|N| +2473687|AAAAAAAAHNOLFCAA|2060-08-15|1927|8381|643|2060|0|8|15|3|2060|643|8381|Sunday|2060Q3|N|N|N|2473673|2473885|2473321|2473596|N|N|N|N|N| +2473688|AAAAAAAAINOLFCAA|2060-08-16|1927|8381|643|2060|1|8|16|3|2060|643|8381|Monday|2060Q3|N|N|N|2473673|2473885|2473322|2473597|N|N|N|N|N| +2473689|AAAAAAAAJNOLFCAA|2060-08-17|1927|8382|643|2060|2|8|17|3|2060|643|8382|Tuesday|2060Q3|N|N|N|2473673|2473885|2473323|2473598|N|N|N|N|N| +2473690|AAAAAAAAKNOLFCAA|2060-08-18|1927|8382|643|2060|3|8|18|3|2060|643|8382|Wednesday|2060Q3|N|N|N|2473673|2473885|2473324|2473599|N|N|N|N|N| +2473691|AAAAAAAALNOLFCAA|2060-08-19|1927|8382|643|2060|4|8|19|3|2060|643|8382|Thursday|2060Q3|N|N|N|2473673|2473885|2473325|2473600|N|N|N|N|N| +2473692|AAAAAAAAMNOLFCAA|2060-08-20|1927|8382|643|2060|5|8|20|3|2060|643|8382|Friday|2060Q3|N|Y|N|2473673|2473885|2473326|2473601|N|N|N|N|N| +2473693|AAAAAAAANNOLFCAA|2060-08-21|1927|8382|643|2060|6|8|21|3|2060|643|8382|Saturday|2060Q3|N|Y|N|2473673|2473885|2473327|2473602|N|N|N|N|N| +2473694|AAAAAAAAONOLFCAA|2060-08-22|1927|8382|643|2060|0|8|22|3|2060|643|8382|Sunday|2060Q3|N|N|N|2473673|2473885|2473328|2473603|N|N|N|N|N| +2473695|AAAAAAAAPNOLFCAA|2060-08-23|1927|8382|643|2060|1|8|23|3|2060|643|8382|Monday|2060Q3|N|N|N|2473673|2473885|2473329|2473604|N|N|N|N|N| +2473696|AAAAAAAAAOOLFCAA|2060-08-24|1927|8383|643|2060|2|8|24|3|2060|643|8383|Tuesday|2060Q3|N|N|N|2473673|2473885|2473330|2473605|N|N|N|N|N| +2473697|AAAAAAAABOOLFCAA|2060-08-25|1927|8383|643|2060|3|8|25|3|2060|643|8383|Wednesday|2060Q3|N|N|N|2473673|2473885|2473331|2473606|N|N|N|N|N| +2473698|AAAAAAAACOOLFCAA|2060-08-26|1927|8383|643|2060|4|8|26|3|2060|643|8383|Thursday|2060Q3|N|N|N|2473673|2473885|2473332|2473607|N|N|N|N|N| +2473699|AAAAAAAADOOLFCAA|2060-08-27|1927|8383|643|2060|5|8|27|3|2060|643|8383|Friday|2060Q3|N|Y|N|2473673|2473885|2473333|2473608|N|N|N|N|N| +2473700|AAAAAAAAEOOLFCAA|2060-08-28|1927|8383|643|2060|6|8|28|3|2060|643|8383|Saturday|2060Q3|N|Y|N|2473673|2473885|2473334|2473609|N|N|N|N|N| +2473701|AAAAAAAAFOOLFCAA|2060-08-29|1927|8383|643|2060|0|8|29|3|2060|643|8383|Sunday|2060Q3|N|N|N|2473673|2473885|2473335|2473610|N|N|N|N|N| +2473702|AAAAAAAAGOOLFCAA|2060-08-30|1927|8383|643|2060|1|8|30|3|2060|643|8383|Monday|2060Q3|N|N|N|2473673|2473885|2473336|2473611|N|N|N|N|N| +2473703|AAAAAAAAHOOLFCAA|2060-08-31|1927|8384|643|2060|2|8|31|3|2060|643|8384|Tuesday|2060Q3|N|N|N|2473673|2473885|2473337|2473612|N|N|N|N|N| +2473704|AAAAAAAAIOOLFCAA|2060-09-01|1928|8384|644|2060|3|9|1|3|2060|644|8384|Wednesday|2060Q3|N|N|N|2473704|2473947|2473338|2473613|N|N|N|N|N| +2473705|AAAAAAAAJOOLFCAA|2060-09-02|1928|8384|644|2060|4|9|2|3|2060|644|8384|Thursday|2060Q3|N|N|N|2473704|2473947|2473339|2473614|N|N|N|N|N| +2473706|AAAAAAAAKOOLFCAA|2060-09-03|1928|8384|644|2060|5|9|3|3|2060|644|8384|Friday|2060Q3|N|Y|N|2473704|2473947|2473340|2473615|N|N|N|N|N| +2473707|AAAAAAAALOOLFCAA|2060-09-04|1928|8384|644|2060|6|9|4|3|2060|644|8384|Saturday|2060Q3|N|Y|N|2473704|2473947|2473341|2473616|N|N|N|N|N| +2473708|AAAAAAAAMOOLFCAA|2060-09-05|1928|8384|644|2060|0|9|5|3|2060|644|8384|Sunday|2060Q3|N|N|N|2473704|2473947|2473342|2473617|N|N|N|N|N| +2473709|AAAAAAAANOOLFCAA|2060-09-06|1928|8384|644|2060|1|9|6|3|2060|644|8384|Monday|2060Q3|N|N|N|2473704|2473947|2473343|2473618|N|N|N|N|N| +2473710|AAAAAAAAOOOLFCAA|2060-09-07|1928|8385|644|2060|2|9|7|3|2060|644|8385|Tuesday|2060Q3|N|N|N|2473704|2473947|2473344|2473619|N|N|N|N|N| +2473711|AAAAAAAAPOOLFCAA|2060-09-08|1928|8385|644|2060|3|9|8|3|2060|644|8385|Wednesday|2060Q3|N|N|N|2473704|2473947|2473345|2473620|N|N|N|N|N| +2473712|AAAAAAAAAPOLFCAA|2060-09-09|1928|8385|644|2060|4|9|9|3|2060|644|8385|Thursday|2060Q3|N|N|N|2473704|2473947|2473346|2473621|N|N|N|N|N| +2473713|AAAAAAAABPOLFCAA|2060-09-10|1928|8385|644|2060|5|9|10|3|2060|644|8385|Friday|2060Q3|N|Y|N|2473704|2473947|2473347|2473622|N|N|N|N|N| +2473714|AAAAAAAACPOLFCAA|2060-09-11|1928|8385|644|2060|6|9|11|3|2060|644|8385|Saturday|2060Q3|N|Y|N|2473704|2473947|2473348|2473623|N|N|N|N|N| +2473715|AAAAAAAADPOLFCAA|2060-09-12|1928|8385|644|2060|0|9|12|3|2060|644|8385|Sunday|2060Q3|N|N|N|2473704|2473947|2473349|2473624|N|N|N|N|N| +2473716|AAAAAAAAEPOLFCAA|2060-09-13|1928|8385|644|2060|1|9|13|3|2060|644|8385|Monday|2060Q3|N|N|N|2473704|2473947|2473350|2473625|N|N|N|N|N| +2473717|AAAAAAAAFPOLFCAA|2060-09-14|1928|8386|644|2060|2|9|14|3|2060|644|8386|Tuesday|2060Q3|N|N|N|2473704|2473947|2473351|2473626|N|N|N|N|N| +2473718|AAAAAAAAGPOLFCAA|2060-09-15|1928|8386|644|2060|3|9|15|3|2060|644|8386|Wednesday|2060Q3|N|N|N|2473704|2473947|2473352|2473627|N|N|N|N|N| +2473719|AAAAAAAAHPOLFCAA|2060-09-16|1928|8386|644|2060|4|9|16|3|2060|644|8386|Thursday|2060Q3|N|N|N|2473704|2473947|2473353|2473628|N|N|N|N|N| +2473720|AAAAAAAAIPOLFCAA|2060-09-17|1928|8386|644|2060|5|9|17|3|2060|644|8386|Friday|2060Q3|N|Y|N|2473704|2473947|2473354|2473629|N|N|N|N|N| +2473721|AAAAAAAAJPOLFCAA|2060-09-18|1928|8386|644|2060|6|9|18|3|2060|644|8386|Saturday|2060Q3|N|Y|N|2473704|2473947|2473355|2473630|N|N|N|N|N| +2473722|AAAAAAAAKPOLFCAA|2060-09-19|1928|8386|644|2060|0|9|19|3|2060|644|8386|Sunday|2060Q3|N|N|N|2473704|2473947|2473356|2473631|N|N|N|N|N| +2473723|AAAAAAAALPOLFCAA|2060-09-20|1928|8386|644|2060|1|9|20|3|2060|644|8386|Monday|2060Q3|N|N|N|2473704|2473947|2473357|2473632|N|N|N|N|N| +2473724|AAAAAAAAMPOLFCAA|2060-09-21|1928|8387|644|2060|2|9|21|3|2060|644|8387|Tuesday|2060Q3|N|N|N|2473704|2473947|2473358|2473633|N|N|N|N|N| +2473725|AAAAAAAANPOLFCAA|2060-09-22|1928|8387|644|2060|3|9|22|3|2060|644|8387|Wednesday|2060Q3|N|N|N|2473704|2473947|2473359|2473634|N|N|N|N|N| +2473726|AAAAAAAAOPOLFCAA|2060-09-23|1928|8387|644|2060|4|9|23|3|2060|644|8387|Thursday|2060Q3|N|N|N|2473704|2473947|2473360|2473635|N|N|N|N|N| +2473727|AAAAAAAAPPOLFCAA|2060-09-24|1928|8387|644|2060|5|9|24|3|2060|644|8387|Friday|2060Q3|N|Y|N|2473704|2473947|2473361|2473636|N|N|N|N|N| +2473728|AAAAAAAAAAPLFCAA|2060-09-25|1928|8387|644|2060|6|9|25|3|2060|644|8387|Saturday|2060Q3|N|Y|N|2473704|2473947|2473362|2473637|N|N|N|N|N| +2473729|AAAAAAAABAPLFCAA|2060-09-26|1928|8387|644|2060|0|9|26|3|2060|644|8387|Sunday|2060Q3|N|N|N|2473704|2473947|2473363|2473638|N|N|N|N|N| +2473730|AAAAAAAACAPLFCAA|2060-09-27|1928|8387|644|2060|1|9|27|3|2060|644|8387|Monday|2060Q3|N|N|N|2473704|2473947|2473364|2473639|N|N|N|N|N| +2473731|AAAAAAAADAPLFCAA|2060-09-28|1928|8388|644|2060|2|9|28|3|2060|644|8388|Tuesday|2060Q3|N|N|N|2473704|2473947|2473365|2473640|N|N|N|N|N| +2473732|AAAAAAAAEAPLFCAA|2060-09-29|1928|8388|644|2060|3|9|29|3|2060|644|8388|Wednesday|2060Q3|N|N|N|2473704|2473947|2473366|2473641|N|N|N|N|N| +2473733|AAAAAAAAFAPLFCAA|2060-09-30|1928|8388|644|2060|4|9|30|3|2060|644|8388|Thursday|2060Q3|N|N|N|2473704|2473947|2473367|2473642|N|N|N|N|N| +2473734|AAAAAAAAGAPLFCAA|2060-10-01|1929|8388|644|2060|5|10|1|4|2060|644|8388|Friday|2060Q4|N|Y|N|2473734|2474007|2473368|2473642|N|N|N|N|N| +2473735|AAAAAAAAHAPLFCAA|2060-10-02|1929|8388|644|2060|6|10|2|4|2060|644|8388|Saturday|2060Q4|N|Y|N|2473734|2474007|2473369|2473643|N|N|N|N|N| +2473736|AAAAAAAAIAPLFCAA|2060-10-03|1929|8388|644|2060|0|10|3|4|2060|644|8388|Sunday|2060Q4|N|N|N|2473734|2474007|2473370|2473644|N|N|N|N|N| +2473737|AAAAAAAAJAPLFCAA|2060-10-04|1929|8388|644|2060|1|10|4|4|2060|644|8388|Monday|2060Q4|N|N|N|2473734|2474007|2473371|2473645|N|N|N|N|N| +2473738|AAAAAAAAKAPLFCAA|2060-10-05|1929|8389|644|2060|2|10|5|4|2060|644|8389|Tuesday|2060Q4|N|N|N|2473734|2474007|2473372|2473646|N|N|N|N|N| +2473739|AAAAAAAALAPLFCAA|2060-10-06|1929|8389|644|2060|3|10|6|4|2060|644|8389|Wednesday|2060Q4|N|N|N|2473734|2474007|2473373|2473647|N|N|N|N|N| +2473740|AAAAAAAAMAPLFCAA|2060-10-07|1929|8389|644|2060|4|10|7|4|2060|644|8389|Thursday|2060Q4|N|N|N|2473734|2474007|2473374|2473648|N|N|N|N|N| +2473741|AAAAAAAANAPLFCAA|2060-10-08|1929|8389|644|2060|5|10|8|4|2060|644|8389|Friday|2060Q4|N|Y|N|2473734|2474007|2473375|2473649|N|N|N|N|N| +2473742|AAAAAAAAOAPLFCAA|2060-10-09|1929|8389|644|2060|6|10|9|4|2060|644|8389|Saturday|2060Q4|N|Y|N|2473734|2474007|2473376|2473650|N|N|N|N|N| +2473743|AAAAAAAAPAPLFCAA|2060-10-10|1929|8389|644|2060|0|10|10|4|2060|644|8389|Sunday|2060Q4|N|N|N|2473734|2474007|2473377|2473651|N|N|N|N|N| +2473744|AAAAAAAAABPLFCAA|2060-10-11|1929|8389|644|2060|1|10|11|4|2060|644|8389|Monday|2060Q4|N|N|N|2473734|2474007|2473378|2473652|N|N|N|N|N| +2473745|AAAAAAAABBPLFCAA|2060-10-12|1929|8390|644|2060|2|10|12|4|2060|644|8390|Tuesday|2060Q4|N|N|N|2473734|2474007|2473379|2473653|N|N|N|N|N| +2473746|AAAAAAAACBPLFCAA|2060-10-13|1929|8390|644|2060|3|10|13|4|2060|644|8390|Wednesday|2060Q4|N|N|N|2473734|2474007|2473380|2473654|N|N|N|N|N| +2473747|AAAAAAAADBPLFCAA|2060-10-14|1929|8390|644|2060|4|10|14|4|2060|644|8390|Thursday|2060Q4|N|N|N|2473734|2474007|2473381|2473655|N|N|N|N|N| +2473748|AAAAAAAAEBPLFCAA|2060-10-15|1929|8390|644|2060|5|10|15|4|2060|644|8390|Friday|2060Q4|N|Y|N|2473734|2474007|2473382|2473656|N|N|N|N|N| +2473749|AAAAAAAAFBPLFCAA|2060-10-16|1929|8390|644|2060|6|10|16|4|2060|644|8390|Saturday|2060Q4|N|Y|N|2473734|2474007|2473383|2473657|N|N|N|N|N| +2473750|AAAAAAAAGBPLFCAA|2060-10-17|1929|8390|644|2060|0|10|17|4|2060|644|8390|Sunday|2060Q4|N|N|N|2473734|2474007|2473384|2473658|N|N|N|N|N| +2473751|AAAAAAAAHBPLFCAA|2060-10-18|1929|8390|644|2060|1|10|18|4|2060|644|8390|Monday|2060Q4|N|N|N|2473734|2474007|2473385|2473659|N|N|N|N|N| +2473752|AAAAAAAAIBPLFCAA|2060-10-19|1929|8391|644|2060|2|10|19|4|2060|644|8391|Tuesday|2060Q4|N|N|N|2473734|2474007|2473386|2473660|N|N|N|N|N| +2473753|AAAAAAAAJBPLFCAA|2060-10-20|1929|8391|644|2060|3|10|20|4|2060|644|8391|Wednesday|2060Q4|N|N|N|2473734|2474007|2473387|2473661|N|N|N|N|N| +2473754|AAAAAAAAKBPLFCAA|2060-10-21|1929|8391|644|2060|4|10|21|4|2060|644|8391|Thursday|2060Q4|N|N|N|2473734|2474007|2473388|2473662|N|N|N|N|N| +2473755|AAAAAAAALBPLFCAA|2060-10-22|1929|8391|644|2060|5|10|22|4|2060|644|8391|Friday|2060Q4|N|Y|N|2473734|2474007|2473389|2473663|N|N|N|N|N| +2473756|AAAAAAAAMBPLFCAA|2060-10-23|1929|8391|644|2060|6|10|23|4|2060|644|8391|Saturday|2060Q4|N|Y|N|2473734|2474007|2473390|2473664|N|N|N|N|N| +2473757|AAAAAAAANBPLFCAA|2060-10-24|1929|8391|644|2060|0|10|24|4|2060|644|8391|Sunday|2060Q4|N|N|N|2473734|2474007|2473391|2473665|N|N|N|N|N| +2473758|AAAAAAAAOBPLFCAA|2060-10-25|1929|8391|644|2060|1|10|25|4|2060|644|8391|Monday|2060Q4|N|N|N|2473734|2474007|2473392|2473666|N|N|N|N|N| +2473759|AAAAAAAAPBPLFCAA|2060-10-26|1929|8392|644|2060|2|10|26|4|2060|644|8392|Tuesday|2060Q4|N|N|N|2473734|2474007|2473393|2473667|N|N|N|N|N| +2473760|AAAAAAAAACPLFCAA|2060-10-27|1929|8392|644|2060|3|10|27|4|2060|644|8392|Wednesday|2060Q4|N|N|N|2473734|2474007|2473394|2473668|N|N|N|N|N| +2473761|AAAAAAAABCPLFCAA|2060-10-28|1929|8392|644|2060|4|10|28|4|2060|644|8392|Thursday|2060Q4|N|N|N|2473734|2474007|2473395|2473669|N|N|N|N|N| +2473762|AAAAAAAACCPLFCAA|2060-10-29|1929|8392|644|2060|5|10|29|4|2060|644|8392|Friday|2060Q4|N|Y|N|2473734|2474007|2473396|2473670|N|N|N|N|N| +2473763|AAAAAAAADCPLFCAA|2060-10-30|1929|8392|644|2060|6|10|30|4|2060|644|8392|Saturday|2060Q4|N|Y|N|2473734|2474007|2473397|2473671|N|N|N|N|N| +2473764|AAAAAAAAECPLFCAA|2060-10-31|1929|8392|644|2060|0|10|31|4|2060|644|8392|Sunday|2060Q4|N|N|N|2473734|2474007|2473398|2473672|N|N|N|N|N| +2473765|AAAAAAAAFCPLFCAA|2060-11-01|1930|8392|644|2060|1|11|1|4|2060|644|8392|Monday|2060Q4|N|N|N|2473765|2474069|2473399|2473673|N|N|N|N|N| +2473766|AAAAAAAAGCPLFCAA|2060-11-02|1930|8393|644|2060|2|11|2|4|2060|644|8393|Tuesday|2060Q4|N|N|N|2473765|2474069|2473400|2473674|N|N|N|N|N| +2473767|AAAAAAAAHCPLFCAA|2060-11-03|1930|8393|644|2060|3|11|3|4|2060|644|8393|Wednesday|2060Q4|N|N|N|2473765|2474069|2473401|2473675|N|N|N|N|N| +2473768|AAAAAAAAICPLFCAA|2060-11-04|1930|8393|644|2060|4|11|4|4|2060|644|8393|Thursday|2060Q4|N|N|N|2473765|2474069|2473402|2473676|N|N|N|N|N| +2473769|AAAAAAAAJCPLFCAA|2060-11-05|1930|8393|644|2060|5|11|5|4|2060|644|8393|Friday|2060Q4|N|Y|N|2473765|2474069|2473403|2473677|N|N|N|N|N| +2473770|AAAAAAAAKCPLFCAA|2060-11-06|1930|8393|644|2060|6|11|6|4|2060|644|8393|Saturday|2060Q4|N|Y|N|2473765|2474069|2473404|2473678|N|N|N|N|N| +2473771|AAAAAAAALCPLFCAA|2060-11-07|1930|8393|644|2060|0|11|7|4|2060|644|8393|Sunday|2060Q4|N|N|N|2473765|2474069|2473405|2473679|N|N|N|N|N| +2473772|AAAAAAAAMCPLFCAA|2060-11-08|1930|8393|644|2060|1|11|8|4|2060|644|8393|Monday|2060Q4|N|N|N|2473765|2474069|2473406|2473680|N|N|N|N|N| +2473773|AAAAAAAANCPLFCAA|2060-11-09|1930|8394|644|2060|2|11|9|4|2060|644|8394|Tuesday|2060Q4|N|N|N|2473765|2474069|2473407|2473681|N|N|N|N|N| +2473774|AAAAAAAAOCPLFCAA|2060-11-10|1930|8394|644|2060|3|11|10|4|2060|644|8394|Wednesday|2060Q4|N|N|N|2473765|2474069|2473408|2473682|N|N|N|N|N| +2473775|AAAAAAAAPCPLFCAA|2060-11-11|1930|8394|644|2060|4|11|11|4|2060|644|8394|Thursday|2060Q4|N|N|N|2473765|2474069|2473409|2473683|N|N|N|N|N| +2473776|AAAAAAAAADPLFCAA|2060-11-12|1930|8394|644|2060|5|11|12|4|2060|644|8394|Friday|2060Q4|N|Y|N|2473765|2474069|2473410|2473684|N|N|N|N|N| +2473777|AAAAAAAABDPLFCAA|2060-11-13|1930|8394|644|2060|6|11|13|4|2060|644|8394|Saturday|2060Q4|N|Y|N|2473765|2474069|2473411|2473685|N|N|N|N|N| +2473778|AAAAAAAACDPLFCAA|2060-11-14|1930|8394|644|2060|0|11|14|4|2060|644|8394|Sunday|2060Q4|N|N|N|2473765|2474069|2473412|2473686|N|N|N|N|N| +2473779|AAAAAAAADDPLFCAA|2060-11-15|1930|8394|644|2060|1|11|15|4|2060|644|8394|Monday|2060Q4|N|N|N|2473765|2474069|2473413|2473687|N|N|N|N|N| +2473780|AAAAAAAAEDPLFCAA|2060-11-16|1930|8395|644|2060|2|11|16|4|2060|644|8395|Tuesday|2060Q4|N|N|N|2473765|2474069|2473414|2473688|N|N|N|N|N| +2473781|AAAAAAAAFDPLFCAA|2060-11-17|1930|8395|644|2060|3|11|17|4|2060|644|8395|Wednesday|2060Q4|N|N|N|2473765|2474069|2473415|2473689|N|N|N|N|N| +2473782|AAAAAAAAGDPLFCAA|2060-11-18|1930|8395|644|2060|4|11|18|4|2060|644|8395|Thursday|2060Q4|N|N|N|2473765|2474069|2473416|2473690|N|N|N|N|N| +2473783|AAAAAAAAHDPLFCAA|2060-11-19|1930|8395|644|2060|5|11|19|4|2060|644|8395|Friday|2060Q4|N|Y|N|2473765|2474069|2473417|2473691|N|N|N|N|N| +2473784|AAAAAAAAIDPLFCAA|2060-11-20|1930|8395|644|2060|6|11|20|4|2060|644|8395|Saturday|2060Q4|N|Y|N|2473765|2474069|2473418|2473692|N|N|N|N|N| +2473785|AAAAAAAAJDPLFCAA|2060-11-21|1930|8395|644|2060|0|11|21|4|2060|644|8395|Sunday|2060Q4|N|N|N|2473765|2474069|2473419|2473693|N|N|N|N|N| +2473786|AAAAAAAAKDPLFCAA|2060-11-22|1930|8395|644|2060|1|11|22|4|2060|644|8395|Monday|2060Q4|N|N|N|2473765|2474069|2473420|2473694|N|N|N|N|N| +2473787|AAAAAAAALDPLFCAA|2060-11-23|1930|8396|644|2060|2|11|23|4|2060|644|8396|Tuesday|2060Q4|N|N|N|2473765|2474069|2473421|2473695|N|N|N|N|N| +2473788|AAAAAAAAMDPLFCAA|2060-11-24|1930|8396|644|2060|3|11|24|4|2060|644|8396|Wednesday|2060Q4|N|N|N|2473765|2474069|2473422|2473696|N|N|N|N|N| +2473789|AAAAAAAANDPLFCAA|2060-11-25|1930|8396|644|2060|4|11|25|4|2060|644|8396|Thursday|2060Q4|N|N|N|2473765|2474069|2473423|2473697|N|N|N|N|N| +2473790|AAAAAAAAODPLFCAA|2060-11-26|1930|8396|644|2060|5|11|26|4|2060|644|8396|Friday|2060Q4|N|Y|N|2473765|2474069|2473424|2473698|N|N|N|N|N| +2473791|AAAAAAAAPDPLFCAA|2060-11-27|1930|8396|644|2060|6|11|27|4|2060|644|8396|Saturday|2060Q4|N|Y|N|2473765|2474069|2473425|2473699|N|N|N|N|N| +2473792|AAAAAAAAAEPLFCAA|2060-11-28|1930|8396|644|2060|0|11|28|4|2060|644|8396|Sunday|2060Q4|N|N|N|2473765|2474069|2473426|2473700|N|N|N|N|N| +2473793|AAAAAAAABEPLFCAA|2060-11-29|1930|8396|644|2060|1|11|29|4|2060|644|8396|Monday|2060Q4|N|N|N|2473765|2474069|2473427|2473701|N|N|N|N|N| +2473794|AAAAAAAACEPLFCAA|2060-11-30|1930|8397|644|2060|2|11|30|4|2060|644|8397|Tuesday|2060Q4|N|N|N|2473765|2474069|2473428|2473702|N|N|N|N|N| +2473795|AAAAAAAADEPLFCAA|2060-12-01|1931|8397|645|2060|3|12|1|4|2060|645|8397|Wednesday|2060Q4|N|N|N|2473795|2474129|2473429|2473703|N|N|N|N|N| +2473796|AAAAAAAAEEPLFCAA|2060-12-02|1931|8397|645|2060|4|12|2|4|2060|645|8397|Thursday|2060Q4|N|N|N|2473795|2474129|2473430|2473704|N|N|N|N|N| +2473797|AAAAAAAAFEPLFCAA|2060-12-03|1931|8397|645|2060|5|12|3|4|2060|645|8397|Friday|2060Q4|N|Y|N|2473795|2474129|2473431|2473705|N|N|N|N|N| +2473798|AAAAAAAAGEPLFCAA|2060-12-04|1931|8397|645|2060|6|12|4|4|2060|645|8397|Saturday|2060Q4|N|Y|N|2473795|2474129|2473432|2473706|N|N|N|N|N| +2473799|AAAAAAAAHEPLFCAA|2060-12-05|1931|8397|645|2060|0|12|5|4|2060|645|8397|Sunday|2060Q4|N|N|N|2473795|2474129|2473433|2473707|N|N|N|N|N| +2473800|AAAAAAAAIEPLFCAA|2060-12-06|1931|8397|645|2060|1|12|6|4|2060|645|8397|Monday|2060Q4|N|N|N|2473795|2474129|2473434|2473708|N|N|N|N|N| +2473801|AAAAAAAAJEPLFCAA|2060-12-07|1931|8398|645|2060|2|12|7|4|2060|645|8398|Tuesday|2060Q4|N|N|N|2473795|2474129|2473435|2473709|N|N|N|N|N| +2473802|AAAAAAAAKEPLFCAA|2060-12-08|1931|8398|645|2060|3|12|8|4|2060|645|8398|Wednesday|2060Q4|N|N|N|2473795|2474129|2473436|2473710|N|N|N|N|N| +2473803|AAAAAAAALEPLFCAA|2060-12-09|1931|8398|645|2060|4|12|9|4|2060|645|8398|Thursday|2060Q4|N|N|N|2473795|2474129|2473437|2473711|N|N|N|N|N| +2473804|AAAAAAAAMEPLFCAA|2060-12-10|1931|8398|645|2060|5|12|10|4|2060|645|8398|Friday|2060Q4|N|Y|N|2473795|2474129|2473438|2473712|N|N|N|N|N| +2473805|AAAAAAAANEPLFCAA|2060-12-11|1931|8398|645|2060|6|12|11|4|2060|645|8398|Saturday|2060Q4|N|Y|N|2473795|2474129|2473439|2473713|N|N|N|N|N| +2473806|AAAAAAAAOEPLFCAA|2060-12-12|1931|8398|645|2060|0|12|12|4|2060|645|8398|Sunday|2060Q4|N|N|N|2473795|2474129|2473440|2473714|N|N|N|N|N| +2473807|AAAAAAAAPEPLFCAA|2060-12-13|1931|8398|645|2060|1|12|13|4|2060|645|8398|Monday|2060Q4|N|N|N|2473795|2474129|2473441|2473715|N|N|N|N|N| +2473808|AAAAAAAAAFPLFCAA|2060-12-14|1931|8399|645|2060|2|12|14|4|2060|645|8399|Tuesday|2060Q4|N|N|N|2473795|2474129|2473442|2473716|N|N|N|N|N| +2473809|AAAAAAAABFPLFCAA|2060-12-15|1931|8399|645|2060|3|12|15|4|2060|645|8399|Wednesday|2060Q4|N|N|N|2473795|2474129|2473443|2473717|N|N|N|N|N| +2473810|AAAAAAAACFPLFCAA|2060-12-16|1931|8399|645|2060|4|12|16|4|2060|645|8399|Thursday|2060Q4|N|N|N|2473795|2474129|2473444|2473718|N|N|N|N|N| +2473811|AAAAAAAADFPLFCAA|2060-12-17|1931|8399|645|2060|5|12|17|4|2060|645|8399|Friday|2060Q4|N|Y|N|2473795|2474129|2473445|2473719|N|N|N|N|N| +2473812|AAAAAAAAEFPLFCAA|2060-12-18|1931|8399|645|2060|6|12|18|4|2060|645|8399|Saturday|2060Q4|N|Y|N|2473795|2474129|2473446|2473720|N|N|N|N|N| +2473813|AAAAAAAAFFPLFCAA|2060-12-19|1931|8399|645|2060|0|12|19|4|2060|645|8399|Sunday|2060Q4|N|N|N|2473795|2474129|2473447|2473721|N|N|N|N|N| +2473814|AAAAAAAAGFPLFCAA|2060-12-20|1931|8399|645|2060|1|12|20|4|2060|645|8399|Monday|2060Q4|N|N|N|2473795|2474129|2473448|2473722|N|N|N|N|N| +2473815|AAAAAAAAHFPLFCAA|2060-12-21|1931|8400|645|2060|2|12|21|4|2060|645|8400|Tuesday|2060Q4|N|N|N|2473795|2474129|2473449|2473723|N|N|N|N|N| +2473816|AAAAAAAAIFPLFCAA|2060-12-22|1931|8400|645|2060|3|12|22|4|2060|645|8400|Wednesday|2060Q4|N|N|N|2473795|2474129|2473450|2473724|N|N|N|N|N| +2473817|AAAAAAAAJFPLFCAA|2060-12-23|1931|8400|645|2060|4|12|23|4|2060|645|8400|Thursday|2060Q4|N|N|N|2473795|2474129|2473451|2473725|N|N|N|N|N| +2473818|AAAAAAAAKFPLFCAA|2060-12-24|1931|8400|645|2060|5|12|24|4|2060|645|8400|Friday|2060Q4|N|Y|N|2473795|2474129|2473452|2473726|N|N|N|N|N| +2473819|AAAAAAAALFPLFCAA|2060-12-25|1931|8400|645|2060|6|12|25|4|2060|645|8400|Saturday|2060Q4|Y|Y|N|2473795|2474129|2473453|2473727|N|N|N|N|N| +2473820|AAAAAAAAMFPLFCAA|2060-12-26|1931|8400|645|2060|0|12|26|4|2060|645|8400|Sunday|2060Q4|N|N|Y|2473795|2474129|2473454|2473728|N|N|N|N|N| +2473821|AAAAAAAANFPLFCAA|2060-12-27|1931|8400|645|2060|1|12|27|4|2060|645|8400|Monday|2060Q4|N|N|N|2473795|2474129|2473455|2473729|N|N|N|N|N| +2473822|AAAAAAAAOFPLFCAA|2060-12-28|1931|8401|645|2060|2|12|28|4|2060|645|8401|Tuesday|2060Q4|N|N|N|2473795|2474129|2473456|2473730|N|N|N|N|N| +2473823|AAAAAAAAPFPLFCAA|2060-12-29|1931|8401|645|2060|3|12|29|4|2060|645|8401|Wednesday|2060Q4|N|N|N|2473795|2474129|2473457|2473731|N|N|N|N|N| +2473824|AAAAAAAAAGPLFCAA|2060-12-30|1931|8401|645|2060|4|12|30|4|2060|645|8401|Thursday|2060Q4|N|N|N|2473795|2474129|2473458|2473732|N|N|N|N|N| +2473825|AAAAAAAABGPLFCAA|2060-12-31|1931|8401|645|2060|5|12|31|4|2060|645|8401|Friday|2060Q4|Y|Y|N|2473795|2474129|2473459|2473733|N|N|N|N|N| +2473826|AAAAAAAACGPLFCAA|2061-01-01|1932|8401|645|2061|6|1|1|1|2061|645|8401|Saturday|2061Q1|Y|Y|Y|2473826|2473825|2473460|2473734|N|N|N|N|N| +2473827|AAAAAAAADGPLFCAA|2061-01-02|1932|8401|645|2061|0|1|2|1|2061|645|8401|Sunday|2061Q1|N|N|Y|2473826|2473825|2473461|2473735|N|N|N|N|N| +2473828|AAAAAAAAEGPLFCAA|2061-01-03|1932|8401|645|2061|1|1|3|1|2061|645|8401|Monday|2061Q1|N|N|N|2473826|2473825|2473462|2473736|N|N|N|N|N| +2473829|AAAAAAAAFGPLFCAA|2061-01-04|1932|8402|645|2061|2|1|4|1|2061|645|8402|Tuesday|2061Q1|N|N|N|2473826|2473825|2473463|2473737|N|N|N|N|N| +2473830|AAAAAAAAGGPLFCAA|2061-01-05|1932|8402|645|2061|3|1|5|1|2061|645|8402|Wednesday|2061Q1|N|N|N|2473826|2473825|2473464|2473738|N|N|N|N|N| +2473831|AAAAAAAAHGPLFCAA|2061-01-06|1932|8402|645|2061|4|1|6|1|2061|645|8402|Thursday|2061Q1|N|N|N|2473826|2473825|2473465|2473739|N|N|N|N|N| +2473832|AAAAAAAAIGPLFCAA|2061-01-07|1932|8402|645|2061|5|1|7|1|2061|645|8402|Friday|2061Q1|N|Y|N|2473826|2473825|2473466|2473740|N|N|N|N|N| +2473833|AAAAAAAAJGPLFCAA|2061-01-08|1932|8402|645|2061|6|1|8|1|2061|645|8402|Saturday|2061Q1|N|Y|N|2473826|2473825|2473467|2473741|N|N|N|N|N| +2473834|AAAAAAAAKGPLFCAA|2061-01-09|1932|8402|645|2061|0|1|9|1|2061|645|8402|Sunday|2061Q1|N|N|N|2473826|2473825|2473468|2473742|N|N|N|N|N| +2473835|AAAAAAAALGPLFCAA|2061-01-10|1932|8402|645|2061|1|1|10|1|2061|645|8402|Monday|2061Q1|N|N|N|2473826|2473825|2473469|2473743|N|N|N|N|N| +2473836|AAAAAAAAMGPLFCAA|2061-01-11|1932|8403|645|2061|2|1|11|1|2061|645|8403|Tuesday|2061Q1|N|N|N|2473826|2473825|2473470|2473744|N|N|N|N|N| +2473837|AAAAAAAANGPLFCAA|2061-01-12|1932|8403|645|2061|3|1|12|1|2061|645|8403|Wednesday|2061Q1|N|N|N|2473826|2473825|2473471|2473745|N|N|N|N|N| +2473838|AAAAAAAAOGPLFCAA|2061-01-13|1932|8403|645|2061|4|1|13|1|2061|645|8403|Thursday|2061Q1|N|N|N|2473826|2473825|2473472|2473746|N|N|N|N|N| +2473839|AAAAAAAAPGPLFCAA|2061-01-14|1932|8403|645|2061|5|1|14|1|2061|645|8403|Friday|2061Q1|N|Y|N|2473826|2473825|2473473|2473747|N|N|N|N|N| +2473840|AAAAAAAAAHPLFCAA|2061-01-15|1932|8403|645|2061|6|1|15|1|2061|645|8403|Saturday|2061Q1|N|Y|N|2473826|2473825|2473474|2473748|N|N|N|N|N| +2473841|AAAAAAAABHPLFCAA|2061-01-16|1932|8403|645|2061|0|1|16|1|2061|645|8403|Sunday|2061Q1|N|N|N|2473826|2473825|2473475|2473749|N|N|N|N|N| +2473842|AAAAAAAACHPLFCAA|2061-01-17|1932|8403|645|2061|1|1|17|1|2061|645|8403|Monday|2061Q1|N|N|N|2473826|2473825|2473476|2473750|N|N|N|N|N| +2473843|AAAAAAAADHPLFCAA|2061-01-18|1932|8404|645|2061|2|1|18|1|2061|645|8404|Tuesday|2061Q1|N|N|N|2473826|2473825|2473477|2473751|N|N|N|N|N| +2473844|AAAAAAAAEHPLFCAA|2061-01-19|1932|8404|645|2061|3|1|19|1|2061|645|8404|Wednesday|2061Q1|N|N|N|2473826|2473825|2473478|2473752|N|N|N|N|N| +2473845|AAAAAAAAFHPLFCAA|2061-01-20|1932|8404|645|2061|4|1|20|1|2061|645|8404|Thursday|2061Q1|N|N|N|2473826|2473825|2473479|2473753|N|N|N|N|N| +2473846|AAAAAAAAGHPLFCAA|2061-01-21|1932|8404|645|2061|5|1|21|1|2061|645|8404|Friday|2061Q1|N|Y|N|2473826|2473825|2473480|2473754|N|N|N|N|N| +2473847|AAAAAAAAHHPLFCAA|2061-01-22|1932|8404|645|2061|6|1|22|1|2061|645|8404|Saturday|2061Q1|N|Y|N|2473826|2473825|2473481|2473755|N|N|N|N|N| +2473848|AAAAAAAAIHPLFCAA|2061-01-23|1932|8404|645|2061|0|1|23|1|2061|645|8404|Sunday|2061Q1|N|N|N|2473826|2473825|2473482|2473756|N|N|N|N|N| +2473849|AAAAAAAAJHPLFCAA|2061-01-24|1932|8404|645|2061|1|1|24|1|2061|645|8404|Monday|2061Q1|N|N|N|2473826|2473825|2473483|2473757|N|N|N|N|N| +2473850|AAAAAAAAKHPLFCAA|2061-01-25|1932|8405|645|2061|2|1|25|1|2061|645|8405|Tuesday|2061Q1|N|N|N|2473826|2473825|2473484|2473758|N|N|N|N|N| +2473851|AAAAAAAALHPLFCAA|2061-01-26|1932|8405|645|2061|3|1|26|1|2061|645|8405|Wednesday|2061Q1|N|N|N|2473826|2473825|2473485|2473759|N|N|N|N|N| +2473852|AAAAAAAAMHPLFCAA|2061-01-27|1932|8405|645|2061|4|1|27|1|2061|645|8405|Thursday|2061Q1|N|N|N|2473826|2473825|2473486|2473760|N|N|N|N|N| +2473853|AAAAAAAANHPLFCAA|2061-01-28|1932|8405|645|2061|5|1|28|1|2061|645|8405|Friday|2061Q1|N|Y|N|2473826|2473825|2473487|2473761|N|N|N|N|N| +2473854|AAAAAAAAOHPLFCAA|2061-01-29|1932|8405|645|2061|6|1|29|1|2061|645|8405|Saturday|2061Q1|N|Y|N|2473826|2473825|2473488|2473762|N|N|N|N|N| +2473855|AAAAAAAAPHPLFCAA|2061-01-30|1932|8405|645|2061|0|1|30|1|2061|645|8405|Sunday|2061Q1|N|N|N|2473826|2473825|2473489|2473763|N|N|N|N|N| +2473856|AAAAAAAAAIPLFCAA|2061-01-31|1932|8405|645|2061|1|1|31|1|2061|645|8405|Monday|2061Q1|N|N|N|2473826|2473825|2473490|2473764|N|N|N|N|N| +2473857|AAAAAAAABIPLFCAA|2061-02-01|1933|8406|645|2061|2|2|1|1|2061|645|8406|Tuesday|2061Q1|N|N|N|2473857|2473887|2473491|2473765|N|N|N|N|N| +2473858|AAAAAAAACIPLFCAA|2061-02-02|1933|8406|645|2061|3|2|2|1|2061|645|8406|Wednesday|2061Q1|N|N|N|2473857|2473887|2473492|2473766|N|N|N|N|N| +2473859|AAAAAAAADIPLFCAA|2061-02-03|1933|8406|645|2061|4|2|3|1|2061|645|8406|Thursday|2061Q1|N|N|N|2473857|2473887|2473493|2473767|N|N|N|N|N| +2473860|AAAAAAAAEIPLFCAA|2061-02-04|1933|8406|645|2061|5|2|4|1|2061|645|8406|Friday|2061Q1|N|Y|N|2473857|2473887|2473494|2473768|N|N|N|N|N| +2473861|AAAAAAAAFIPLFCAA|2061-02-05|1933|8406|645|2061|6|2|5|1|2061|645|8406|Saturday|2061Q1|N|Y|N|2473857|2473887|2473495|2473769|N|N|N|N|N| +2473862|AAAAAAAAGIPLFCAA|2061-02-06|1933|8406|645|2061|0|2|6|1|2061|645|8406|Sunday|2061Q1|N|N|N|2473857|2473887|2473496|2473770|N|N|N|N|N| +2473863|AAAAAAAAHIPLFCAA|2061-02-07|1933|8406|645|2061|1|2|7|1|2061|645|8406|Monday|2061Q1|N|N|N|2473857|2473887|2473497|2473771|N|N|N|N|N| +2473864|AAAAAAAAIIPLFCAA|2061-02-08|1933|8407|645|2061|2|2|8|1|2061|645|8407|Tuesday|2061Q1|N|N|N|2473857|2473887|2473498|2473772|N|N|N|N|N| +2473865|AAAAAAAAJIPLFCAA|2061-02-09|1933|8407|645|2061|3|2|9|1|2061|645|8407|Wednesday|2061Q1|N|N|N|2473857|2473887|2473499|2473773|N|N|N|N|N| +2473866|AAAAAAAAKIPLFCAA|2061-02-10|1933|8407|645|2061|4|2|10|1|2061|645|8407|Thursday|2061Q1|N|N|N|2473857|2473887|2473500|2473774|N|N|N|N|N| +2473867|AAAAAAAALIPLFCAA|2061-02-11|1933|8407|645|2061|5|2|11|1|2061|645|8407|Friday|2061Q1|N|Y|N|2473857|2473887|2473501|2473775|N|N|N|N|N| +2473868|AAAAAAAAMIPLFCAA|2061-02-12|1933|8407|645|2061|6|2|12|1|2061|645|8407|Saturday|2061Q1|N|Y|N|2473857|2473887|2473502|2473776|N|N|N|N|N| +2473869|AAAAAAAANIPLFCAA|2061-02-13|1933|8407|645|2061|0|2|13|1|2061|645|8407|Sunday|2061Q1|N|N|N|2473857|2473887|2473503|2473777|N|N|N|N|N| +2473870|AAAAAAAAOIPLFCAA|2061-02-14|1933|8407|645|2061|1|2|14|1|2061|645|8407|Monday|2061Q1|N|N|N|2473857|2473887|2473504|2473778|N|N|N|N|N| +2473871|AAAAAAAAPIPLFCAA|2061-02-15|1933|8408|645|2061|2|2|15|1|2061|645|8408|Tuesday|2061Q1|N|N|N|2473857|2473887|2473505|2473779|N|N|N|N|N| +2473872|AAAAAAAAAJPLFCAA|2061-02-16|1933|8408|645|2061|3|2|16|1|2061|645|8408|Wednesday|2061Q1|N|N|N|2473857|2473887|2473506|2473780|N|N|N|N|N| +2473873|AAAAAAAABJPLFCAA|2061-02-17|1933|8408|645|2061|4|2|17|1|2061|645|8408|Thursday|2061Q1|N|N|N|2473857|2473887|2473507|2473781|N|N|N|N|N| +2473874|AAAAAAAACJPLFCAA|2061-02-18|1933|8408|645|2061|5|2|18|1|2061|645|8408|Friday|2061Q1|N|Y|N|2473857|2473887|2473508|2473782|N|N|N|N|N| +2473875|AAAAAAAADJPLFCAA|2061-02-19|1933|8408|645|2061|6|2|19|1|2061|645|8408|Saturday|2061Q1|N|Y|N|2473857|2473887|2473509|2473783|N|N|N|N|N| +2473876|AAAAAAAAEJPLFCAA|2061-02-20|1933|8408|645|2061|0|2|20|1|2061|645|8408|Sunday|2061Q1|N|N|N|2473857|2473887|2473510|2473784|N|N|N|N|N| +2473877|AAAAAAAAFJPLFCAA|2061-02-21|1933|8408|645|2061|1|2|21|1|2061|645|8408|Monday|2061Q1|N|N|N|2473857|2473887|2473511|2473785|N|N|N|N|N| +2473878|AAAAAAAAGJPLFCAA|2061-02-22|1933|8409|645|2061|2|2|22|1|2061|645|8409|Tuesday|2061Q1|N|N|N|2473857|2473887|2473512|2473786|N|N|N|N|N| +2473879|AAAAAAAAHJPLFCAA|2061-02-23|1933|8409|645|2061|3|2|23|1|2061|645|8409|Wednesday|2061Q1|N|N|N|2473857|2473887|2473513|2473787|N|N|N|N|N| +2473880|AAAAAAAAIJPLFCAA|2061-02-24|1933|8409|645|2061|4|2|24|1|2061|645|8409|Thursday|2061Q1|N|N|N|2473857|2473887|2473514|2473788|N|N|N|N|N| +2473881|AAAAAAAAJJPLFCAA|2061-02-25|1933|8409|645|2061|5|2|25|1|2061|645|8409|Friday|2061Q1|N|Y|N|2473857|2473887|2473515|2473789|N|N|N|N|N| +2473882|AAAAAAAAKJPLFCAA|2061-02-26|1933|8409|645|2061|6|2|26|1|2061|645|8409|Saturday|2061Q1|N|Y|N|2473857|2473887|2473516|2473790|N|N|N|N|N| +2473883|AAAAAAAALJPLFCAA|2061-02-27|1933|8409|645|2061|0|2|27|1|2061|645|8409|Sunday|2061Q1|N|N|N|2473857|2473887|2473517|2473791|N|N|N|N|N| +2473884|AAAAAAAAMJPLFCAA|2061-02-28|1933|8409|645|2061|1|2|28|1|2061|645|8409|Monday|2061Q1|N|N|N|2473857|2473887|2473518|2473792|N|N|N|N|N| +2473885|AAAAAAAANJPLFCAA|2061-03-01|1934|8410|646|2061|2|3|1|1|2061|646|8410|Tuesday|2061Q1|N|N|N|2473885|2473943|2473520|2473793|N|N|N|N|N| +2473886|AAAAAAAAOJPLFCAA|2061-03-02|1934|8410|646|2061|3|3|2|1|2061|646|8410|Wednesday|2061Q1|N|N|N|2473885|2473943|2473521|2473794|N|N|N|N|N| +2473887|AAAAAAAAPJPLFCAA|2061-03-03|1934|8410|646|2061|4|3|3|1|2061|646|8410|Thursday|2061Q1|N|N|N|2473885|2473943|2473522|2473795|N|N|N|N|N| +2473888|AAAAAAAAAKPLFCAA|2061-03-04|1934|8410|646|2061|5|3|4|1|2061|646|8410|Friday|2061Q1|N|Y|N|2473885|2473943|2473523|2473796|N|N|N|N|N| +2473889|AAAAAAAABKPLFCAA|2061-03-05|1934|8410|646|2061|6|3|5|1|2061|646|8410|Saturday|2061Q1|N|Y|N|2473885|2473943|2473524|2473797|N|N|N|N|N| +2473890|AAAAAAAACKPLFCAA|2061-03-06|1934|8410|646|2061|0|3|6|1|2061|646|8410|Sunday|2061Q1|N|N|N|2473885|2473943|2473525|2473798|N|N|N|N|N| +2473891|AAAAAAAADKPLFCAA|2061-03-07|1934|8410|646|2061|1|3|7|1|2061|646|8410|Monday|2061Q1|N|N|N|2473885|2473943|2473526|2473799|N|N|N|N|N| +2473892|AAAAAAAAEKPLFCAA|2061-03-08|1934|8411|646|2061|2|3|8|1|2061|646|8411|Tuesday|2061Q1|N|N|N|2473885|2473943|2473527|2473800|N|N|N|N|N| +2473893|AAAAAAAAFKPLFCAA|2061-03-09|1934|8411|646|2061|3|3|9|1|2061|646|8411|Wednesday|2061Q1|N|N|N|2473885|2473943|2473528|2473801|N|N|N|N|N| +2473894|AAAAAAAAGKPLFCAA|2061-03-10|1934|8411|646|2061|4|3|10|1|2061|646|8411|Thursday|2061Q1|N|N|N|2473885|2473943|2473529|2473802|N|N|N|N|N| +2473895|AAAAAAAAHKPLFCAA|2061-03-11|1934|8411|646|2061|5|3|11|1|2061|646|8411|Friday|2061Q1|N|Y|N|2473885|2473943|2473530|2473803|N|N|N|N|N| +2473896|AAAAAAAAIKPLFCAA|2061-03-12|1934|8411|646|2061|6|3|12|1|2061|646|8411|Saturday|2061Q1|N|Y|N|2473885|2473943|2473531|2473804|N|N|N|N|N| +2473897|AAAAAAAAJKPLFCAA|2061-03-13|1934|8411|646|2061|0|3|13|1|2061|646|8411|Sunday|2061Q1|N|N|N|2473885|2473943|2473532|2473805|N|N|N|N|N| +2473898|AAAAAAAAKKPLFCAA|2061-03-14|1934|8411|646|2061|1|3|14|1|2061|646|8411|Monday|2061Q1|N|N|N|2473885|2473943|2473533|2473806|N|N|N|N|N| +2473899|AAAAAAAALKPLFCAA|2061-03-15|1934|8412|646|2061|2|3|15|1|2061|646|8412|Tuesday|2061Q1|N|N|N|2473885|2473943|2473534|2473807|N|N|N|N|N| +2473900|AAAAAAAAMKPLFCAA|2061-03-16|1934|8412|646|2061|3|3|16|1|2061|646|8412|Wednesday|2061Q1|N|N|N|2473885|2473943|2473535|2473808|N|N|N|N|N| +2473901|AAAAAAAANKPLFCAA|2061-03-17|1934|8412|646|2061|4|3|17|1|2061|646|8412|Thursday|2061Q1|N|N|N|2473885|2473943|2473536|2473809|N|N|N|N|N| +2473902|AAAAAAAAOKPLFCAA|2061-03-18|1934|8412|646|2061|5|3|18|1|2061|646|8412|Friday|2061Q1|N|Y|N|2473885|2473943|2473537|2473810|N|N|N|N|N| +2473903|AAAAAAAAPKPLFCAA|2061-03-19|1934|8412|646|2061|6|3|19|1|2061|646|8412|Saturday|2061Q1|N|Y|N|2473885|2473943|2473538|2473811|N|N|N|N|N| +2473904|AAAAAAAAALPLFCAA|2061-03-20|1934|8412|646|2061|0|3|20|1|2061|646|8412|Sunday|2061Q1|N|N|N|2473885|2473943|2473539|2473812|N|N|N|N|N| +2473905|AAAAAAAABLPLFCAA|2061-03-21|1934|8412|646|2061|1|3|21|1|2061|646|8412|Monday|2061Q1|N|N|N|2473885|2473943|2473540|2473813|N|N|N|N|N| +2473906|AAAAAAAACLPLFCAA|2061-03-22|1934|8413|646|2061|2|3|22|1|2061|646|8413|Tuesday|2061Q1|N|N|N|2473885|2473943|2473541|2473814|N|N|N|N|N| +2473907|AAAAAAAADLPLFCAA|2061-03-23|1934|8413|646|2061|3|3|23|1|2061|646|8413|Wednesday|2061Q1|N|N|N|2473885|2473943|2473542|2473815|N|N|N|N|N| +2473908|AAAAAAAAELPLFCAA|2061-03-24|1934|8413|646|2061|4|3|24|1|2061|646|8413|Thursday|2061Q1|N|N|N|2473885|2473943|2473543|2473816|N|N|N|N|N| +2473909|AAAAAAAAFLPLFCAA|2061-03-25|1934|8413|646|2061|5|3|25|1|2061|646|8413|Friday|2061Q1|N|Y|N|2473885|2473943|2473544|2473817|N|N|N|N|N| +2473910|AAAAAAAAGLPLFCAA|2061-03-26|1934|8413|646|2061|6|3|26|1|2061|646|8413|Saturday|2061Q1|N|Y|N|2473885|2473943|2473545|2473818|N|N|N|N|N| +2473911|AAAAAAAAHLPLFCAA|2061-03-27|1934|8413|646|2061|0|3|27|1|2061|646|8413|Sunday|2061Q1|N|N|N|2473885|2473943|2473546|2473819|N|N|N|N|N| +2473912|AAAAAAAAILPLFCAA|2061-03-28|1934|8413|646|2061|1|3|28|1|2061|646|8413|Monday|2061Q1|N|N|N|2473885|2473943|2473547|2473820|N|N|N|N|N| +2473913|AAAAAAAAJLPLFCAA|2061-03-29|1934|8414|646|2061|2|3|29|1|2061|646|8414|Tuesday|2061Q1|N|N|N|2473885|2473943|2473548|2473821|N|N|N|N|N| +2473914|AAAAAAAAKLPLFCAA|2061-03-30|1934|8414|646|2061|3|3|30|1|2061|646|8414|Wednesday|2061Q1|N|N|N|2473885|2473943|2473549|2473822|N|N|N|N|N| +2473915|AAAAAAAALLPLFCAA|2061-03-31|1934|8414|646|2061|4|3|31|1|2061|646|8414|Thursday|2061Q1|N|N|N|2473885|2473943|2473550|2473823|N|N|N|N|N| +2473916|AAAAAAAAMLPLFCAA|2061-04-01|1935|8414|646|2061|5|4|1|1|2061|646|8414|Friday|2061Q1|N|Y|N|2473916|2474005|2473551|2473826|N|N|N|N|N| +2473917|AAAAAAAANLPLFCAA|2061-04-02|1935|8414|646|2061|6|4|2|2|2061|646|8414|Saturday|2061Q2|N|Y|N|2473916|2474005|2473552|2473827|N|N|N|N|N| +2473918|AAAAAAAAOLPLFCAA|2061-04-03|1935|8414|646|2061|0|4|3|2|2061|646|8414|Sunday|2061Q2|N|N|N|2473916|2474005|2473553|2473828|N|N|N|N|N| +2473919|AAAAAAAAPLPLFCAA|2061-04-04|1935|8414|646|2061|1|4|4|2|2061|646|8414|Monday|2061Q2|N|N|N|2473916|2474005|2473554|2473829|N|N|N|N|N| +2473920|AAAAAAAAAMPLFCAA|2061-04-05|1935|8415|646|2061|2|4|5|2|2061|646|8415|Tuesday|2061Q2|N|N|N|2473916|2474005|2473555|2473830|N|N|N|N|N| +2473921|AAAAAAAABMPLFCAA|2061-04-06|1935|8415|646|2061|3|4|6|2|2061|646|8415|Wednesday|2061Q2|N|N|N|2473916|2474005|2473556|2473831|N|N|N|N|N| +2473922|AAAAAAAACMPLFCAA|2061-04-07|1935|8415|646|2061|4|4|7|2|2061|646|8415|Thursday|2061Q2|N|N|N|2473916|2474005|2473557|2473832|N|N|N|N|N| +2473923|AAAAAAAADMPLFCAA|2061-04-08|1935|8415|646|2061|5|4|8|2|2061|646|8415|Friday|2061Q2|N|Y|N|2473916|2474005|2473558|2473833|N|N|N|N|N| +2473924|AAAAAAAAEMPLFCAA|2061-04-09|1935|8415|646|2061|6|4|9|2|2061|646|8415|Saturday|2061Q2|N|Y|N|2473916|2474005|2473559|2473834|N|N|N|N|N| +2473925|AAAAAAAAFMPLFCAA|2061-04-10|1935|8415|646|2061|0|4|10|2|2061|646|8415|Sunday|2061Q2|N|N|N|2473916|2474005|2473560|2473835|N|N|N|N|N| +2473926|AAAAAAAAGMPLFCAA|2061-04-11|1935|8415|646|2061|1|4|11|2|2061|646|8415|Monday|2061Q2|N|N|N|2473916|2474005|2473561|2473836|N|N|N|N|N| +2473927|AAAAAAAAHMPLFCAA|2061-04-12|1935|8416|646|2061|2|4|12|2|2061|646|8416|Tuesday|2061Q2|N|N|N|2473916|2474005|2473562|2473837|N|N|N|N|N| +2473928|AAAAAAAAIMPLFCAA|2061-04-13|1935|8416|646|2061|3|4|13|2|2061|646|8416|Wednesday|2061Q2|N|N|N|2473916|2474005|2473563|2473838|N|N|N|N|N| +2473929|AAAAAAAAJMPLFCAA|2061-04-14|1935|8416|646|2061|4|4|14|2|2061|646|8416|Thursday|2061Q2|N|N|N|2473916|2474005|2473564|2473839|N|N|N|N|N| +2473930|AAAAAAAAKMPLFCAA|2061-04-15|1935|8416|646|2061|5|4|15|2|2061|646|8416|Friday|2061Q2|N|Y|N|2473916|2474005|2473565|2473840|N|N|N|N|N| +2473931|AAAAAAAALMPLFCAA|2061-04-16|1935|8416|646|2061|6|4|16|2|2061|646|8416|Saturday|2061Q2|N|Y|N|2473916|2474005|2473566|2473841|N|N|N|N|N| +2473932|AAAAAAAAMMPLFCAA|2061-04-17|1935|8416|646|2061|0|4|17|2|2061|646|8416|Sunday|2061Q2|N|N|N|2473916|2474005|2473567|2473842|N|N|N|N|N| +2473933|AAAAAAAANMPLFCAA|2061-04-18|1935|8416|646|2061|1|4|18|2|2061|646|8416|Monday|2061Q2|N|N|N|2473916|2474005|2473568|2473843|N|N|N|N|N| +2473934|AAAAAAAAOMPLFCAA|2061-04-19|1935|8417|646|2061|2|4|19|2|2061|646|8417|Tuesday|2061Q2|N|N|N|2473916|2474005|2473569|2473844|N|N|N|N|N| +2473935|AAAAAAAAPMPLFCAA|2061-04-20|1935|8417|646|2061|3|4|20|2|2061|646|8417|Wednesday|2061Q2|N|N|N|2473916|2474005|2473570|2473845|N|N|N|N|N| +2473936|AAAAAAAAANPLFCAA|2061-04-21|1935|8417|646|2061|4|4|21|2|2061|646|8417|Thursday|2061Q2|N|N|N|2473916|2474005|2473571|2473846|N|N|N|N|N| +2473937|AAAAAAAABNPLFCAA|2061-04-22|1935|8417|646|2061|5|4|22|2|2061|646|8417|Friday|2061Q2|N|Y|N|2473916|2474005|2473572|2473847|N|N|N|N|N| +2473938|AAAAAAAACNPLFCAA|2061-04-23|1935|8417|646|2061|6|4|23|2|2061|646|8417|Saturday|2061Q2|N|Y|N|2473916|2474005|2473573|2473848|N|N|N|N|N| +2473939|AAAAAAAADNPLFCAA|2061-04-24|1935|8417|646|2061|0|4|24|2|2061|646|8417|Sunday|2061Q2|N|N|N|2473916|2474005|2473574|2473849|N|N|N|N|N| +2473940|AAAAAAAAENPLFCAA|2061-04-25|1935|8417|646|2061|1|4|25|2|2061|646|8417|Monday|2061Q2|N|N|N|2473916|2474005|2473575|2473850|N|N|N|N|N| +2473941|AAAAAAAAFNPLFCAA|2061-04-26|1935|8418|646|2061|2|4|26|2|2061|646|8418|Tuesday|2061Q2|N|N|N|2473916|2474005|2473576|2473851|N|N|N|N|N| +2473942|AAAAAAAAGNPLFCAA|2061-04-27|1935|8418|646|2061|3|4|27|2|2061|646|8418|Wednesday|2061Q2|N|N|N|2473916|2474005|2473577|2473852|N|N|N|N|N| +2473943|AAAAAAAAHNPLFCAA|2061-04-28|1935|8418|646|2061|4|4|28|2|2061|646|8418|Thursday|2061Q2|N|N|N|2473916|2474005|2473578|2473853|N|N|N|N|N| +2473944|AAAAAAAAINPLFCAA|2061-04-29|1935|8418|646|2061|5|4|29|2|2061|646|8418|Friday|2061Q2|N|Y|N|2473916|2474005|2473579|2473854|N|N|N|N|N| +2473945|AAAAAAAAJNPLFCAA|2061-04-30|1935|8418|646|2061|6|4|30|2|2061|646|8418|Saturday|2061Q2|N|Y|N|2473916|2474005|2473580|2473855|N|N|N|N|N| +2473946|AAAAAAAAKNPLFCAA|2061-05-01|1936|8418|646|2061|0|5|1|2|2061|646|8418|Sunday|2061Q2|N|N|N|2473946|2474065|2473581|2473856|N|N|N|N|N| +2473947|AAAAAAAALNPLFCAA|2061-05-02|1936|8418|646|2061|1|5|2|2|2061|646|8418|Monday|2061Q2|N|N|N|2473946|2474065|2473582|2473857|N|N|N|N|N| +2473948|AAAAAAAAMNPLFCAA|2061-05-03|1936|8419|646|2061|2|5|3|2|2061|646|8419|Tuesday|2061Q2|N|N|N|2473946|2474065|2473583|2473858|N|N|N|N|N| +2473949|AAAAAAAANNPLFCAA|2061-05-04|1936|8419|646|2061|3|5|4|2|2061|646|8419|Wednesday|2061Q2|N|N|N|2473946|2474065|2473584|2473859|N|N|N|N|N| +2473950|AAAAAAAAONPLFCAA|2061-05-05|1936|8419|646|2061|4|5|5|2|2061|646|8419|Thursday|2061Q2|N|N|N|2473946|2474065|2473585|2473860|N|N|N|N|N| +2473951|AAAAAAAAPNPLFCAA|2061-05-06|1936|8419|646|2061|5|5|6|2|2061|646|8419|Friday|2061Q2|N|Y|N|2473946|2474065|2473586|2473861|N|N|N|N|N| +2473952|AAAAAAAAAOPLFCAA|2061-05-07|1936|8419|646|2061|6|5|7|2|2061|646|8419|Saturday|2061Q2|N|Y|N|2473946|2474065|2473587|2473862|N|N|N|N|N| +2473953|AAAAAAAABOPLFCAA|2061-05-08|1936|8419|646|2061|0|5|8|2|2061|646|8419|Sunday|2061Q2|N|N|N|2473946|2474065|2473588|2473863|N|N|N|N|N| +2473954|AAAAAAAACOPLFCAA|2061-05-09|1936|8419|646|2061|1|5|9|2|2061|646|8419|Monday|2061Q2|N|N|N|2473946|2474065|2473589|2473864|N|N|N|N|N| +2473955|AAAAAAAADOPLFCAA|2061-05-10|1936|8420|646|2061|2|5|10|2|2061|646|8420|Tuesday|2061Q2|N|N|N|2473946|2474065|2473590|2473865|N|N|N|N|N| +2473956|AAAAAAAAEOPLFCAA|2061-05-11|1936|8420|646|2061|3|5|11|2|2061|646|8420|Wednesday|2061Q2|N|N|N|2473946|2474065|2473591|2473866|N|N|N|N|N| +2473957|AAAAAAAAFOPLFCAA|2061-05-12|1936|8420|646|2061|4|5|12|2|2061|646|8420|Thursday|2061Q2|N|N|N|2473946|2474065|2473592|2473867|N|N|N|N|N| +2473958|AAAAAAAAGOPLFCAA|2061-05-13|1936|8420|646|2061|5|5|13|2|2061|646|8420|Friday|2061Q2|N|Y|N|2473946|2474065|2473593|2473868|N|N|N|N|N| +2473959|AAAAAAAAHOPLFCAA|2061-05-14|1936|8420|646|2061|6|5|14|2|2061|646|8420|Saturday|2061Q2|N|Y|N|2473946|2474065|2473594|2473869|N|N|N|N|N| +2473960|AAAAAAAAIOPLFCAA|2061-05-15|1936|8420|646|2061|0|5|15|2|2061|646|8420|Sunday|2061Q2|N|N|N|2473946|2474065|2473595|2473870|N|N|N|N|N| +2473961|AAAAAAAAJOPLFCAA|2061-05-16|1936|8420|646|2061|1|5|16|2|2061|646|8420|Monday|2061Q2|N|N|N|2473946|2474065|2473596|2473871|N|N|N|N|N| +2473962|AAAAAAAAKOPLFCAA|2061-05-17|1936|8421|646|2061|2|5|17|2|2061|646|8421|Tuesday|2061Q2|N|N|N|2473946|2474065|2473597|2473872|N|N|N|N|N| +2473963|AAAAAAAALOPLFCAA|2061-05-18|1936|8421|646|2061|3|5|18|2|2061|646|8421|Wednesday|2061Q2|N|N|N|2473946|2474065|2473598|2473873|N|N|N|N|N| +2473964|AAAAAAAAMOPLFCAA|2061-05-19|1936|8421|646|2061|4|5|19|2|2061|646|8421|Thursday|2061Q2|N|N|N|2473946|2474065|2473599|2473874|N|N|N|N|N| +2473965|AAAAAAAANOPLFCAA|2061-05-20|1936|8421|646|2061|5|5|20|2|2061|646|8421|Friday|2061Q2|N|Y|N|2473946|2474065|2473600|2473875|N|N|N|N|N| +2473966|AAAAAAAAOOPLFCAA|2061-05-21|1936|8421|646|2061|6|5|21|2|2061|646|8421|Saturday|2061Q2|N|Y|N|2473946|2474065|2473601|2473876|N|N|N|N|N| +2473967|AAAAAAAAPOPLFCAA|2061-05-22|1936|8421|646|2061|0|5|22|2|2061|646|8421|Sunday|2061Q2|N|N|N|2473946|2474065|2473602|2473877|N|N|N|N|N| +2473968|AAAAAAAAAPPLFCAA|2061-05-23|1936|8421|646|2061|1|5|23|2|2061|646|8421|Monday|2061Q2|N|N|N|2473946|2474065|2473603|2473878|N|N|N|N|N| +2473969|AAAAAAAABPPLFCAA|2061-05-24|1936|8422|646|2061|2|5|24|2|2061|646|8422|Tuesday|2061Q2|N|N|N|2473946|2474065|2473604|2473879|N|N|N|N|N| +2473970|AAAAAAAACPPLFCAA|2061-05-25|1936|8422|646|2061|3|5|25|2|2061|646|8422|Wednesday|2061Q2|N|N|N|2473946|2474065|2473605|2473880|N|N|N|N|N| +2473971|AAAAAAAADPPLFCAA|2061-05-26|1936|8422|646|2061|4|5|26|2|2061|646|8422|Thursday|2061Q2|N|N|N|2473946|2474065|2473606|2473881|N|N|N|N|N| +2473972|AAAAAAAAEPPLFCAA|2061-05-27|1936|8422|646|2061|5|5|27|2|2061|646|8422|Friday|2061Q2|N|Y|N|2473946|2474065|2473607|2473882|N|N|N|N|N| +2473973|AAAAAAAAFPPLFCAA|2061-05-28|1936|8422|646|2061|6|5|28|2|2061|646|8422|Saturday|2061Q2|N|Y|N|2473946|2474065|2473608|2473883|N|N|N|N|N| +2473974|AAAAAAAAGPPLFCAA|2061-05-29|1936|8422|646|2061|0|5|29|2|2061|646|8422|Sunday|2061Q2|N|N|N|2473946|2474065|2473609|2473884|N|N|N|N|N| +2473975|AAAAAAAAHPPLFCAA|2061-05-30|1936|8422|646|2061|1|5|30|2|2061|646|8422|Monday|2061Q2|N|N|N|2473946|2474065|2473610|2473885|N|N|N|N|N| +2473976|AAAAAAAAIPPLFCAA|2061-05-31|1936|8423|646|2061|2|5|31|2|2061|646|8423|Tuesday|2061Q2|N|N|N|2473946|2474065|2473611|2473886|N|N|N|N|N| +2473977|AAAAAAAAJPPLFCAA|2061-06-01|1937|8423|647|2061|3|6|1|2|2061|647|8423|Wednesday|2061Q2|N|N|N|2473977|2474127|2473612|2473887|N|N|N|N|N| +2473978|AAAAAAAAKPPLFCAA|2061-06-02|1937|8423|647|2061|4|6|2|2|2061|647|8423|Thursday|2061Q2|N|N|N|2473977|2474127|2473613|2473888|N|N|N|N|N| +2473979|AAAAAAAALPPLFCAA|2061-06-03|1937|8423|647|2061|5|6|3|2|2061|647|8423|Friday|2061Q2|N|Y|N|2473977|2474127|2473614|2473889|N|N|N|N|N| +2473980|AAAAAAAAMPPLFCAA|2061-06-04|1937|8423|647|2061|6|6|4|2|2061|647|8423|Saturday|2061Q2|N|Y|N|2473977|2474127|2473615|2473890|N|N|N|N|N| +2473981|AAAAAAAANPPLFCAA|2061-06-05|1937|8423|647|2061|0|6|5|2|2061|647|8423|Sunday|2061Q2|N|N|N|2473977|2474127|2473616|2473891|N|N|N|N|N| +2473982|AAAAAAAAOPPLFCAA|2061-06-06|1937|8423|647|2061|1|6|6|2|2061|647|8423|Monday|2061Q2|N|N|N|2473977|2474127|2473617|2473892|N|N|N|N|N| +2473983|AAAAAAAAPPPLFCAA|2061-06-07|1937|8424|647|2061|2|6|7|2|2061|647|8424|Tuesday|2061Q2|N|N|N|2473977|2474127|2473618|2473893|N|N|N|N|N| +2473984|AAAAAAAAAAAMFCAA|2061-06-08|1937|8424|647|2061|3|6|8|2|2061|647|8424|Wednesday|2061Q2|N|N|N|2473977|2474127|2473619|2473894|N|N|N|N|N| +2473985|AAAAAAAABAAMFCAA|2061-06-09|1937|8424|647|2061|4|6|9|2|2061|647|8424|Thursday|2061Q2|N|N|N|2473977|2474127|2473620|2473895|N|N|N|N|N| +2473986|AAAAAAAACAAMFCAA|2061-06-10|1937|8424|647|2061|5|6|10|2|2061|647|8424|Friday|2061Q2|N|Y|N|2473977|2474127|2473621|2473896|N|N|N|N|N| +2473987|AAAAAAAADAAMFCAA|2061-06-11|1937|8424|647|2061|6|6|11|2|2061|647|8424|Saturday|2061Q2|N|Y|N|2473977|2474127|2473622|2473897|N|N|N|N|N| +2473988|AAAAAAAAEAAMFCAA|2061-06-12|1937|8424|647|2061|0|6|12|2|2061|647|8424|Sunday|2061Q2|N|N|N|2473977|2474127|2473623|2473898|N|N|N|N|N| +2473989|AAAAAAAAFAAMFCAA|2061-06-13|1937|8424|647|2061|1|6|13|2|2061|647|8424|Monday|2061Q2|N|N|N|2473977|2474127|2473624|2473899|N|N|N|N|N| +2473990|AAAAAAAAGAAMFCAA|2061-06-14|1937|8425|647|2061|2|6|14|2|2061|647|8425|Tuesday|2061Q2|N|N|N|2473977|2474127|2473625|2473900|N|N|N|N|N| +2473991|AAAAAAAAHAAMFCAA|2061-06-15|1937|8425|647|2061|3|6|15|2|2061|647|8425|Wednesday|2061Q2|N|N|N|2473977|2474127|2473626|2473901|N|N|N|N|N| +2473992|AAAAAAAAIAAMFCAA|2061-06-16|1937|8425|647|2061|4|6|16|2|2061|647|8425|Thursday|2061Q2|N|N|N|2473977|2474127|2473627|2473902|N|N|N|N|N| +2473993|AAAAAAAAJAAMFCAA|2061-06-17|1937|8425|647|2061|5|6|17|2|2061|647|8425|Friday|2061Q2|N|Y|N|2473977|2474127|2473628|2473903|N|N|N|N|N| +2473994|AAAAAAAAKAAMFCAA|2061-06-18|1937|8425|647|2061|6|6|18|2|2061|647|8425|Saturday|2061Q2|N|Y|N|2473977|2474127|2473629|2473904|N|N|N|N|N| +2473995|AAAAAAAALAAMFCAA|2061-06-19|1937|8425|647|2061|0|6|19|2|2061|647|8425|Sunday|2061Q2|N|N|N|2473977|2474127|2473630|2473905|N|N|N|N|N| +2473996|AAAAAAAAMAAMFCAA|2061-06-20|1937|8425|647|2061|1|6|20|2|2061|647|8425|Monday|2061Q2|N|N|N|2473977|2474127|2473631|2473906|N|N|N|N|N| +2473997|AAAAAAAANAAMFCAA|2061-06-21|1937|8426|647|2061|2|6|21|2|2061|647|8426|Tuesday|2061Q2|N|N|N|2473977|2474127|2473632|2473907|N|N|N|N|N| +2473998|AAAAAAAAOAAMFCAA|2061-06-22|1937|8426|647|2061|3|6|22|2|2061|647|8426|Wednesday|2061Q2|N|N|N|2473977|2474127|2473633|2473908|N|N|N|N|N| +2473999|AAAAAAAAPAAMFCAA|2061-06-23|1937|8426|647|2061|4|6|23|2|2061|647|8426|Thursday|2061Q2|N|N|N|2473977|2474127|2473634|2473909|N|N|N|N|N| +2474000|AAAAAAAAABAMFCAA|2061-06-24|1937|8426|647|2061|5|6|24|2|2061|647|8426|Friday|2061Q2|N|Y|N|2473977|2474127|2473635|2473910|N|N|N|N|N| +2474001|AAAAAAAABBAMFCAA|2061-06-25|1937|8426|647|2061|6|6|25|2|2061|647|8426|Saturday|2061Q2|N|Y|N|2473977|2474127|2473636|2473911|N|N|N|N|N| +2474002|AAAAAAAACBAMFCAA|2061-06-26|1937|8426|647|2061|0|6|26|2|2061|647|8426|Sunday|2061Q2|N|N|N|2473977|2474127|2473637|2473912|N|N|N|N|N| +2474003|AAAAAAAADBAMFCAA|2061-06-27|1937|8426|647|2061|1|6|27|2|2061|647|8426|Monday|2061Q2|N|N|N|2473977|2474127|2473638|2473913|N|N|N|N|N| +2474004|AAAAAAAAEBAMFCAA|2061-06-28|1937|8427|647|2061|2|6|28|2|2061|647|8427|Tuesday|2061Q2|N|N|N|2473977|2474127|2473639|2473914|N|N|N|N|N| +2474005|AAAAAAAAFBAMFCAA|2061-06-29|1937|8427|647|2061|3|6|29|2|2061|647|8427|Wednesday|2061Q2|N|N|N|2473977|2474127|2473640|2473915|N|N|N|N|N| +2474006|AAAAAAAAGBAMFCAA|2061-06-30|1937|8427|647|2061|4|6|30|2|2061|647|8427|Thursday|2061Q2|N|N|N|2473977|2474127|2473641|2473916|N|N|N|N|N| +2474007|AAAAAAAAHBAMFCAA|2061-07-01|1938|8427|647|2061|5|7|1|2|2061|647|8427|Friday|2061Q2|N|Y|N|2474007|2474187|2473642|2473916|N|N|N|N|N| +2474008|AAAAAAAAIBAMFCAA|2061-07-02|1938|8427|647|2061|6|7|2|3|2061|647|8427|Saturday|2061Q3|N|Y|N|2474007|2474187|2473643|2473917|N|N|N|N|N| +2474009|AAAAAAAAJBAMFCAA|2061-07-03|1938|8427|647|2061|0|7|3|3|2061|647|8427|Sunday|2061Q3|N|N|N|2474007|2474187|2473644|2473918|N|N|N|N|N| +2474010|AAAAAAAAKBAMFCAA|2061-07-04|1938|8427|647|2061|1|7|4|3|2061|647|8427|Monday|2061Q3|N|N|N|2474007|2474187|2473645|2473919|N|N|N|N|N| +2474011|AAAAAAAALBAMFCAA|2061-07-05|1938|8428|647|2061|2|7|5|3|2061|647|8428|Tuesday|2061Q3|Y|N|N|2474007|2474187|2473646|2473920|N|N|N|N|N| +2474012|AAAAAAAAMBAMFCAA|2061-07-06|1938|8428|647|2061|3|7|6|3|2061|647|8428|Wednesday|2061Q3|N|N|Y|2474007|2474187|2473647|2473921|N|N|N|N|N| +2474013|AAAAAAAANBAMFCAA|2061-07-07|1938|8428|647|2061|4|7|7|3|2061|647|8428|Thursday|2061Q3|N|N|N|2474007|2474187|2473648|2473922|N|N|N|N|N| +2474014|AAAAAAAAOBAMFCAA|2061-07-08|1938|8428|647|2061|5|7|8|3|2061|647|8428|Friday|2061Q3|N|Y|N|2474007|2474187|2473649|2473923|N|N|N|N|N| +2474015|AAAAAAAAPBAMFCAA|2061-07-09|1938|8428|647|2061|6|7|9|3|2061|647|8428|Saturday|2061Q3|N|Y|N|2474007|2474187|2473650|2473924|N|N|N|N|N| +2474016|AAAAAAAAACAMFCAA|2061-07-10|1938|8428|647|2061|0|7|10|3|2061|647|8428|Sunday|2061Q3|N|N|N|2474007|2474187|2473651|2473925|N|N|N|N|N| +2474017|AAAAAAAABCAMFCAA|2061-07-11|1938|8428|647|2061|1|7|11|3|2061|647|8428|Monday|2061Q3|N|N|N|2474007|2474187|2473652|2473926|N|N|N|N|N| +2474018|AAAAAAAACCAMFCAA|2061-07-12|1938|8429|647|2061|2|7|12|3|2061|647|8429|Tuesday|2061Q3|N|N|N|2474007|2474187|2473653|2473927|N|N|N|N|N| +2474019|AAAAAAAADCAMFCAA|2061-07-13|1938|8429|647|2061|3|7|13|3|2061|647|8429|Wednesday|2061Q3|N|N|N|2474007|2474187|2473654|2473928|N|N|N|N|N| +2474020|AAAAAAAAECAMFCAA|2061-07-14|1938|8429|647|2061|4|7|14|3|2061|647|8429|Thursday|2061Q3|N|N|N|2474007|2474187|2473655|2473929|N|N|N|N|N| +2474021|AAAAAAAAFCAMFCAA|2061-07-15|1938|8429|647|2061|5|7|15|3|2061|647|8429|Friday|2061Q3|N|Y|N|2474007|2474187|2473656|2473930|N|N|N|N|N| +2474022|AAAAAAAAGCAMFCAA|2061-07-16|1938|8429|647|2061|6|7|16|3|2061|647|8429|Saturday|2061Q3|N|Y|N|2474007|2474187|2473657|2473931|N|N|N|N|N| +2474023|AAAAAAAAHCAMFCAA|2061-07-17|1938|8429|647|2061|0|7|17|3|2061|647|8429|Sunday|2061Q3|N|N|N|2474007|2474187|2473658|2473932|N|N|N|N|N| +2474024|AAAAAAAAICAMFCAA|2061-07-18|1938|8429|647|2061|1|7|18|3|2061|647|8429|Monday|2061Q3|N|N|N|2474007|2474187|2473659|2473933|N|N|N|N|N| +2474025|AAAAAAAAJCAMFCAA|2061-07-19|1938|8430|647|2061|2|7|19|3|2061|647|8430|Tuesday|2061Q3|N|N|N|2474007|2474187|2473660|2473934|N|N|N|N|N| +2474026|AAAAAAAAKCAMFCAA|2061-07-20|1938|8430|647|2061|3|7|20|3|2061|647|8430|Wednesday|2061Q3|N|N|N|2474007|2474187|2473661|2473935|N|N|N|N|N| +2474027|AAAAAAAALCAMFCAA|2061-07-21|1938|8430|647|2061|4|7|21|3|2061|647|8430|Thursday|2061Q3|N|N|N|2474007|2474187|2473662|2473936|N|N|N|N|N| +2474028|AAAAAAAAMCAMFCAA|2061-07-22|1938|8430|647|2061|5|7|22|3|2061|647|8430|Friday|2061Q3|N|Y|N|2474007|2474187|2473663|2473937|N|N|N|N|N| +2474029|AAAAAAAANCAMFCAA|2061-07-23|1938|8430|647|2061|6|7|23|3|2061|647|8430|Saturday|2061Q3|N|Y|N|2474007|2474187|2473664|2473938|N|N|N|N|N| +2474030|AAAAAAAAOCAMFCAA|2061-07-24|1938|8430|647|2061|0|7|24|3|2061|647|8430|Sunday|2061Q3|N|N|N|2474007|2474187|2473665|2473939|N|N|N|N|N| +2474031|AAAAAAAAPCAMFCAA|2061-07-25|1938|8430|647|2061|1|7|25|3|2061|647|8430|Monday|2061Q3|N|N|N|2474007|2474187|2473666|2473940|N|N|N|N|N| +2474032|AAAAAAAAADAMFCAA|2061-07-26|1938|8431|647|2061|2|7|26|3|2061|647|8431|Tuesday|2061Q3|N|N|N|2474007|2474187|2473667|2473941|N|N|N|N|N| +2474033|AAAAAAAABDAMFCAA|2061-07-27|1938|8431|647|2061|3|7|27|3|2061|647|8431|Wednesday|2061Q3|N|N|N|2474007|2474187|2473668|2473942|N|N|N|N|N| +2474034|AAAAAAAACDAMFCAA|2061-07-28|1938|8431|647|2061|4|7|28|3|2061|647|8431|Thursday|2061Q3|N|N|N|2474007|2474187|2473669|2473943|N|N|N|N|N| +2474035|AAAAAAAADDAMFCAA|2061-07-29|1938|8431|647|2061|5|7|29|3|2061|647|8431|Friday|2061Q3|N|Y|N|2474007|2474187|2473670|2473944|N|N|N|N|N| +2474036|AAAAAAAAEDAMFCAA|2061-07-30|1938|8431|647|2061|6|7|30|3|2061|647|8431|Saturday|2061Q3|N|Y|N|2474007|2474187|2473671|2473945|N|N|N|N|N| +2474037|AAAAAAAAFDAMFCAA|2061-07-31|1938|8431|647|2061|0|7|31|3|2061|647|8431|Sunday|2061Q3|N|N|N|2474007|2474187|2473672|2473946|N|N|N|N|N| +2474038|AAAAAAAAGDAMFCAA|2061-08-01|1939|8431|647|2061|1|8|1|3|2061|647|8431|Monday|2061Q3|N|N|N|2474038|2474249|2473673|2473947|N|N|N|N|N| +2474039|AAAAAAAAHDAMFCAA|2061-08-02|1939|8432|647|2061|2|8|2|3|2061|647|8432|Tuesday|2061Q3|N|N|N|2474038|2474249|2473674|2473948|N|N|N|N|N| +2474040|AAAAAAAAIDAMFCAA|2061-08-03|1939|8432|647|2061|3|8|3|3|2061|647|8432|Wednesday|2061Q3|N|N|N|2474038|2474249|2473675|2473949|N|N|N|N|N| +2474041|AAAAAAAAJDAMFCAA|2061-08-04|1939|8432|647|2061|4|8|4|3|2061|647|8432|Thursday|2061Q3|N|N|N|2474038|2474249|2473676|2473950|N|N|N|N|N| +2474042|AAAAAAAAKDAMFCAA|2061-08-05|1939|8432|647|2061|5|8|5|3|2061|647|8432|Friday|2061Q3|N|Y|N|2474038|2474249|2473677|2473951|N|N|N|N|N| +2474043|AAAAAAAALDAMFCAA|2061-08-06|1939|8432|647|2061|6|8|6|3|2061|647|8432|Saturday|2061Q3|N|Y|N|2474038|2474249|2473678|2473952|N|N|N|N|N| +2474044|AAAAAAAAMDAMFCAA|2061-08-07|1939|8432|647|2061|0|8|7|3|2061|647|8432|Sunday|2061Q3|N|N|N|2474038|2474249|2473679|2473953|N|N|N|N|N| +2474045|AAAAAAAANDAMFCAA|2061-08-08|1939|8432|647|2061|1|8|8|3|2061|647|8432|Monday|2061Q3|N|N|N|2474038|2474249|2473680|2473954|N|N|N|N|N| +2474046|AAAAAAAAODAMFCAA|2061-08-09|1939|8433|647|2061|2|8|9|3|2061|647|8433|Tuesday|2061Q3|N|N|N|2474038|2474249|2473681|2473955|N|N|N|N|N| +2474047|AAAAAAAAPDAMFCAA|2061-08-10|1939|8433|647|2061|3|8|10|3|2061|647|8433|Wednesday|2061Q3|N|N|N|2474038|2474249|2473682|2473956|N|N|N|N|N| +2474048|AAAAAAAAAEAMFCAA|2061-08-11|1939|8433|647|2061|4|8|11|3|2061|647|8433|Thursday|2061Q3|N|N|N|2474038|2474249|2473683|2473957|N|N|N|N|N| +2474049|AAAAAAAABEAMFCAA|2061-08-12|1939|8433|647|2061|5|8|12|3|2061|647|8433|Friday|2061Q3|N|Y|N|2474038|2474249|2473684|2473958|N|N|N|N|N| +2474050|AAAAAAAACEAMFCAA|2061-08-13|1939|8433|647|2061|6|8|13|3|2061|647|8433|Saturday|2061Q3|N|Y|N|2474038|2474249|2473685|2473959|N|N|N|N|N| +2474051|AAAAAAAADEAMFCAA|2061-08-14|1939|8433|647|2061|0|8|14|3|2061|647|8433|Sunday|2061Q3|N|N|N|2474038|2474249|2473686|2473960|N|N|N|N|N| +2474052|AAAAAAAAEEAMFCAA|2061-08-15|1939|8433|647|2061|1|8|15|3|2061|647|8433|Monday|2061Q3|N|N|N|2474038|2474249|2473687|2473961|N|N|N|N|N| +2474053|AAAAAAAAFEAMFCAA|2061-08-16|1939|8434|647|2061|2|8|16|3|2061|647|8434|Tuesday|2061Q3|N|N|N|2474038|2474249|2473688|2473962|N|N|N|N|N| +2474054|AAAAAAAAGEAMFCAA|2061-08-17|1939|8434|647|2061|3|8|17|3|2061|647|8434|Wednesday|2061Q3|N|N|N|2474038|2474249|2473689|2473963|N|N|N|N|N| +2474055|AAAAAAAAHEAMFCAA|2061-08-18|1939|8434|647|2061|4|8|18|3|2061|647|8434|Thursday|2061Q3|N|N|N|2474038|2474249|2473690|2473964|N|N|N|N|N| +2474056|AAAAAAAAIEAMFCAA|2061-08-19|1939|8434|647|2061|5|8|19|3|2061|647|8434|Friday|2061Q3|N|Y|N|2474038|2474249|2473691|2473965|N|N|N|N|N| +2474057|AAAAAAAAJEAMFCAA|2061-08-20|1939|8434|647|2061|6|8|20|3|2061|647|8434|Saturday|2061Q3|N|Y|N|2474038|2474249|2473692|2473966|N|N|N|N|N| +2474058|AAAAAAAAKEAMFCAA|2061-08-21|1939|8434|647|2061|0|8|21|3|2061|647|8434|Sunday|2061Q3|N|N|N|2474038|2474249|2473693|2473967|N|N|N|N|N| +2474059|AAAAAAAALEAMFCAA|2061-08-22|1939|8434|647|2061|1|8|22|3|2061|647|8434|Monday|2061Q3|N|N|N|2474038|2474249|2473694|2473968|N|N|N|N|N| +2474060|AAAAAAAAMEAMFCAA|2061-08-23|1939|8435|647|2061|2|8|23|3|2061|647|8435|Tuesday|2061Q3|N|N|N|2474038|2474249|2473695|2473969|N|N|N|N|N| +2474061|AAAAAAAANEAMFCAA|2061-08-24|1939|8435|647|2061|3|8|24|3|2061|647|8435|Wednesday|2061Q3|N|N|N|2474038|2474249|2473696|2473970|N|N|N|N|N| +2474062|AAAAAAAAOEAMFCAA|2061-08-25|1939|8435|647|2061|4|8|25|3|2061|647|8435|Thursday|2061Q3|N|N|N|2474038|2474249|2473697|2473971|N|N|N|N|N| +2474063|AAAAAAAAPEAMFCAA|2061-08-26|1939|8435|647|2061|5|8|26|3|2061|647|8435|Friday|2061Q3|N|Y|N|2474038|2474249|2473698|2473972|N|N|N|N|N| +2474064|AAAAAAAAAFAMFCAA|2061-08-27|1939|8435|647|2061|6|8|27|3|2061|647|8435|Saturday|2061Q3|N|Y|N|2474038|2474249|2473699|2473973|N|N|N|N|N| +2474065|AAAAAAAABFAMFCAA|2061-08-28|1939|8435|647|2061|0|8|28|3|2061|647|8435|Sunday|2061Q3|N|N|N|2474038|2474249|2473700|2473974|N|N|N|N|N| +2474066|AAAAAAAACFAMFCAA|2061-08-29|1939|8435|647|2061|1|8|29|3|2061|647|8435|Monday|2061Q3|N|N|N|2474038|2474249|2473701|2473975|N|N|N|N|N| +2474067|AAAAAAAADFAMFCAA|2061-08-30|1939|8436|647|2061|2|8|30|3|2061|647|8436|Tuesday|2061Q3|N|N|N|2474038|2474249|2473702|2473976|N|N|N|N|N| +2474068|AAAAAAAAEFAMFCAA|2061-08-31|1939|8436|647|2061|3|8|31|3|2061|647|8436|Wednesday|2061Q3|N|N|N|2474038|2474249|2473703|2473977|N|N|N|N|N| +2474069|AAAAAAAAFFAMFCAA|2061-09-01|1940|8436|648|2061|4|9|1|3|2061|648|8436|Thursday|2061Q3|N|N|N|2474069|2474311|2473704|2473978|N|N|N|N|N| +2474070|AAAAAAAAGFAMFCAA|2061-09-02|1940|8436|648|2061|5|9|2|3|2061|648|8436|Friday|2061Q3|N|Y|N|2474069|2474311|2473705|2473979|N|N|N|N|N| +2474071|AAAAAAAAHFAMFCAA|2061-09-03|1940|8436|648|2061|6|9|3|3|2061|648|8436|Saturday|2061Q3|N|Y|N|2474069|2474311|2473706|2473980|N|N|N|N|N| +2474072|AAAAAAAAIFAMFCAA|2061-09-04|1940|8436|648|2061|0|9|4|3|2061|648|8436|Sunday|2061Q3|N|N|N|2474069|2474311|2473707|2473981|N|N|N|N|N| +2474073|AAAAAAAAJFAMFCAA|2061-09-05|1940|8436|648|2061|1|9|5|3|2061|648|8436|Monday|2061Q3|N|N|N|2474069|2474311|2473708|2473982|N|N|N|N|N| +2474074|AAAAAAAAKFAMFCAA|2061-09-06|1940|8437|648|2061|2|9|6|3|2061|648|8437|Tuesday|2061Q3|N|N|N|2474069|2474311|2473709|2473983|N|N|N|N|N| +2474075|AAAAAAAALFAMFCAA|2061-09-07|1940|8437|648|2061|3|9|7|3|2061|648|8437|Wednesday|2061Q3|N|N|N|2474069|2474311|2473710|2473984|N|N|N|N|N| +2474076|AAAAAAAAMFAMFCAA|2061-09-08|1940|8437|648|2061|4|9|8|3|2061|648|8437|Thursday|2061Q3|N|N|N|2474069|2474311|2473711|2473985|N|N|N|N|N| +2474077|AAAAAAAANFAMFCAA|2061-09-09|1940|8437|648|2061|5|9|9|3|2061|648|8437|Friday|2061Q3|N|Y|N|2474069|2474311|2473712|2473986|N|N|N|N|N| +2474078|AAAAAAAAOFAMFCAA|2061-09-10|1940|8437|648|2061|6|9|10|3|2061|648|8437|Saturday|2061Q3|N|Y|N|2474069|2474311|2473713|2473987|N|N|N|N|N| +2474079|AAAAAAAAPFAMFCAA|2061-09-11|1940|8437|648|2061|0|9|11|3|2061|648|8437|Sunday|2061Q3|N|N|N|2474069|2474311|2473714|2473988|N|N|N|N|N| +2474080|AAAAAAAAAGAMFCAA|2061-09-12|1940|8437|648|2061|1|9|12|3|2061|648|8437|Monday|2061Q3|N|N|N|2474069|2474311|2473715|2473989|N|N|N|N|N| +2474081|AAAAAAAABGAMFCAA|2061-09-13|1940|8438|648|2061|2|9|13|3|2061|648|8438|Tuesday|2061Q3|N|N|N|2474069|2474311|2473716|2473990|N|N|N|N|N| +2474082|AAAAAAAACGAMFCAA|2061-09-14|1940|8438|648|2061|3|9|14|3|2061|648|8438|Wednesday|2061Q3|N|N|N|2474069|2474311|2473717|2473991|N|N|N|N|N| +2474083|AAAAAAAADGAMFCAA|2061-09-15|1940|8438|648|2061|4|9|15|3|2061|648|8438|Thursday|2061Q3|N|N|N|2474069|2474311|2473718|2473992|N|N|N|N|N| +2474084|AAAAAAAAEGAMFCAA|2061-09-16|1940|8438|648|2061|5|9|16|3|2061|648|8438|Friday|2061Q3|N|Y|N|2474069|2474311|2473719|2473993|N|N|N|N|N| +2474085|AAAAAAAAFGAMFCAA|2061-09-17|1940|8438|648|2061|6|9|17|3|2061|648|8438|Saturday|2061Q3|N|Y|N|2474069|2474311|2473720|2473994|N|N|N|N|N| +2474086|AAAAAAAAGGAMFCAA|2061-09-18|1940|8438|648|2061|0|9|18|3|2061|648|8438|Sunday|2061Q3|N|N|N|2474069|2474311|2473721|2473995|N|N|N|N|N| +2474087|AAAAAAAAHGAMFCAA|2061-09-19|1940|8438|648|2061|1|9|19|3|2061|648|8438|Monday|2061Q3|N|N|N|2474069|2474311|2473722|2473996|N|N|N|N|N| +2474088|AAAAAAAAIGAMFCAA|2061-09-20|1940|8439|648|2061|2|9|20|3|2061|648|8439|Tuesday|2061Q3|N|N|N|2474069|2474311|2473723|2473997|N|N|N|N|N| +2474089|AAAAAAAAJGAMFCAA|2061-09-21|1940|8439|648|2061|3|9|21|3|2061|648|8439|Wednesday|2061Q3|N|N|N|2474069|2474311|2473724|2473998|N|N|N|N|N| +2474090|AAAAAAAAKGAMFCAA|2061-09-22|1940|8439|648|2061|4|9|22|3|2061|648|8439|Thursday|2061Q3|N|N|N|2474069|2474311|2473725|2473999|N|N|N|N|N| +2474091|AAAAAAAALGAMFCAA|2061-09-23|1940|8439|648|2061|5|9|23|3|2061|648|8439|Friday|2061Q3|N|Y|N|2474069|2474311|2473726|2474000|N|N|N|N|N| +2474092|AAAAAAAAMGAMFCAA|2061-09-24|1940|8439|648|2061|6|9|24|3|2061|648|8439|Saturday|2061Q3|N|Y|N|2474069|2474311|2473727|2474001|N|N|N|N|N| +2474093|AAAAAAAANGAMFCAA|2061-09-25|1940|8439|648|2061|0|9|25|3|2061|648|8439|Sunday|2061Q3|N|N|N|2474069|2474311|2473728|2474002|N|N|N|N|N| +2474094|AAAAAAAAOGAMFCAA|2061-09-26|1940|8439|648|2061|1|9|26|3|2061|648|8439|Monday|2061Q3|N|N|N|2474069|2474311|2473729|2474003|N|N|N|N|N| +2474095|AAAAAAAAPGAMFCAA|2061-09-27|1940|8440|648|2061|2|9|27|3|2061|648|8440|Tuesday|2061Q3|N|N|N|2474069|2474311|2473730|2474004|N|N|N|N|N| +2474096|AAAAAAAAAHAMFCAA|2061-09-28|1940|8440|648|2061|3|9|28|3|2061|648|8440|Wednesday|2061Q3|N|N|N|2474069|2474311|2473731|2474005|N|N|N|N|N| +2474097|AAAAAAAABHAMFCAA|2061-09-29|1940|8440|648|2061|4|9|29|3|2061|648|8440|Thursday|2061Q3|N|N|N|2474069|2474311|2473732|2474006|N|N|N|N|N| +2474098|AAAAAAAACHAMFCAA|2061-09-30|1940|8440|648|2061|5|9|30|3|2061|648|8440|Friday|2061Q3|N|Y|N|2474069|2474311|2473733|2474007|N|N|N|N|N| +2474099|AAAAAAAADHAMFCAA|2061-10-01|1941|8440|648|2061|6|10|1|3|2061|648|8440|Saturday|2061Q3|N|Y|N|2474099|2474371|2473734|2474007|N|N|N|N|N| +2474100|AAAAAAAAEHAMFCAA|2061-10-02|1941|8440|648|2061|0|10|2|4|2061|648|8440|Sunday|2061Q4|N|N|N|2474099|2474371|2473735|2474008|N|N|N|N|N| +2474101|AAAAAAAAFHAMFCAA|2061-10-03|1941|8440|648|2061|1|10|3|4|2061|648|8440|Monday|2061Q4|N|N|N|2474099|2474371|2473736|2474009|N|N|N|N|N| +2474102|AAAAAAAAGHAMFCAA|2061-10-04|1941|8441|648|2061|2|10|4|4|2061|648|8441|Tuesday|2061Q4|N|N|N|2474099|2474371|2473737|2474010|N|N|N|N|N| +2474103|AAAAAAAAHHAMFCAA|2061-10-05|1941|8441|648|2061|3|10|5|4|2061|648|8441|Wednesday|2061Q4|N|N|N|2474099|2474371|2473738|2474011|N|N|N|N|N| +2474104|AAAAAAAAIHAMFCAA|2061-10-06|1941|8441|648|2061|4|10|6|4|2061|648|8441|Thursday|2061Q4|N|N|N|2474099|2474371|2473739|2474012|N|N|N|N|N| +2474105|AAAAAAAAJHAMFCAA|2061-10-07|1941|8441|648|2061|5|10|7|4|2061|648|8441|Friday|2061Q4|N|Y|N|2474099|2474371|2473740|2474013|N|N|N|N|N| +2474106|AAAAAAAAKHAMFCAA|2061-10-08|1941|8441|648|2061|6|10|8|4|2061|648|8441|Saturday|2061Q4|N|Y|N|2474099|2474371|2473741|2474014|N|N|N|N|N| +2474107|AAAAAAAALHAMFCAA|2061-10-09|1941|8441|648|2061|0|10|9|4|2061|648|8441|Sunday|2061Q4|N|N|N|2474099|2474371|2473742|2474015|N|N|N|N|N| +2474108|AAAAAAAAMHAMFCAA|2061-10-10|1941|8441|648|2061|1|10|10|4|2061|648|8441|Monday|2061Q4|N|N|N|2474099|2474371|2473743|2474016|N|N|N|N|N| +2474109|AAAAAAAANHAMFCAA|2061-10-11|1941|8442|648|2061|2|10|11|4|2061|648|8442|Tuesday|2061Q4|N|N|N|2474099|2474371|2473744|2474017|N|N|N|N|N| +2474110|AAAAAAAAOHAMFCAA|2061-10-12|1941|8442|648|2061|3|10|12|4|2061|648|8442|Wednesday|2061Q4|N|N|N|2474099|2474371|2473745|2474018|N|N|N|N|N| +2474111|AAAAAAAAPHAMFCAA|2061-10-13|1941|8442|648|2061|4|10|13|4|2061|648|8442|Thursday|2061Q4|N|N|N|2474099|2474371|2473746|2474019|N|N|N|N|N| +2474112|AAAAAAAAAIAMFCAA|2061-10-14|1941|8442|648|2061|5|10|14|4|2061|648|8442|Friday|2061Q4|N|Y|N|2474099|2474371|2473747|2474020|N|N|N|N|N| +2474113|AAAAAAAABIAMFCAA|2061-10-15|1941|8442|648|2061|6|10|15|4|2061|648|8442|Saturday|2061Q4|N|Y|N|2474099|2474371|2473748|2474021|N|N|N|N|N| +2474114|AAAAAAAACIAMFCAA|2061-10-16|1941|8442|648|2061|0|10|16|4|2061|648|8442|Sunday|2061Q4|N|N|N|2474099|2474371|2473749|2474022|N|N|N|N|N| +2474115|AAAAAAAADIAMFCAA|2061-10-17|1941|8442|648|2061|1|10|17|4|2061|648|8442|Monday|2061Q4|N|N|N|2474099|2474371|2473750|2474023|N|N|N|N|N| +2474116|AAAAAAAAEIAMFCAA|2061-10-18|1941|8443|648|2061|2|10|18|4|2061|648|8443|Tuesday|2061Q4|N|N|N|2474099|2474371|2473751|2474024|N|N|N|N|N| +2474117|AAAAAAAAFIAMFCAA|2061-10-19|1941|8443|648|2061|3|10|19|4|2061|648|8443|Wednesday|2061Q4|N|N|N|2474099|2474371|2473752|2474025|N|N|N|N|N| +2474118|AAAAAAAAGIAMFCAA|2061-10-20|1941|8443|648|2061|4|10|20|4|2061|648|8443|Thursday|2061Q4|N|N|N|2474099|2474371|2473753|2474026|N|N|N|N|N| +2474119|AAAAAAAAHIAMFCAA|2061-10-21|1941|8443|648|2061|5|10|21|4|2061|648|8443|Friday|2061Q4|N|Y|N|2474099|2474371|2473754|2474027|N|N|N|N|N| +2474120|AAAAAAAAIIAMFCAA|2061-10-22|1941|8443|648|2061|6|10|22|4|2061|648|8443|Saturday|2061Q4|N|Y|N|2474099|2474371|2473755|2474028|N|N|N|N|N| +2474121|AAAAAAAAJIAMFCAA|2061-10-23|1941|8443|648|2061|0|10|23|4|2061|648|8443|Sunday|2061Q4|N|N|N|2474099|2474371|2473756|2474029|N|N|N|N|N| +2474122|AAAAAAAAKIAMFCAA|2061-10-24|1941|8443|648|2061|1|10|24|4|2061|648|8443|Monday|2061Q4|N|N|N|2474099|2474371|2473757|2474030|N|N|N|N|N| +2474123|AAAAAAAALIAMFCAA|2061-10-25|1941|8444|648|2061|2|10|25|4|2061|648|8444|Tuesday|2061Q4|N|N|N|2474099|2474371|2473758|2474031|N|N|N|N|N| +2474124|AAAAAAAAMIAMFCAA|2061-10-26|1941|8444|648|2061|3|10|26|4|2061|648|8444|Wednesday|2061Q4|N|N|N|2474099|2474371|2473759|2474032|N|N|N|N|N| +2474125|AAAAAAAANIAMFCAA|2061-10-27|1941|8444|648|2061|4|10|27|4|2061|648|8444|Thursday|2061Q4|N|N|N|2474099|2474371|2473760|2474033|N|N|N|N|N| +2474126|AAAAAAAAOIAMFCAA|2061-10-28|1941|8444|648|2061|5|10|28|4|2061|648|8444|Friday|2061Q4|N|Y|N|2474099|2474371|2473761|2474034|N|N|N|N|N| +2474127|AAAAAAAAPIAMFCAA|2061-10-29|1941|8444|648|2061|6|10|29|4|2061|648|8444|Saturday|2061Q4|N|Y|N|2474099|2474371|2473762|2474035|N|N|N|N|N| +2474128|AAAAAAAAAJAMFCAA|2061-10-30|1941|8444|648|2061|0|10|30|4|2061|648|8444|Sunday|2061Q4|N|N|N|2474099|2474371|2473763|2474036|N|N|N|N|N| +2474129|AAAAAAAABJAMFCAA|2061-10-31|1941|8444|648|2061|1|10|31|4|2061|648|8444|Monday|2061Q4|N|N|N|2474099|2474371|2473764|2474037|N|N|N|N|N| +2474130|AAAAAAAACJAMFCAA|2061-11-01|1942|8445|648|2061|2|11|1|4|2061|648|8445|Tuesday|2061Q4|N|N|N|2474130|2474433|2473765|2474038|N|N|N|N|N| +2474131|AAAAAAAADJAMFCAA|2061-11-02|1942|8445|648|2061|3|11|2|4|2061|648|8445|Wednesday|2061Q4|N|N|N|2474130|2474433|2473766|2474039|N|N|N|N|N| +2474132|AAAAAAAAEJAMFCAA|2061-11-03|1942|8445|648|2061|4|11|3|4|2061|648|8445|Thursday|2061Q4|N|N|N|2474130|2474433|2473767|2474040|N|N|N|N|N| +2474133|AAAAAAAAFJAMFCAA|2061-11-04|1942|8445|648|2061|5|11|4|4|2061|648|8445|Friday|2061Q4|N|Y|N|2474130|2474433|2473768|2474041|N|N|N|N|N| +2474134|AAAAAAAAGJAMFCAA|2061-11-05|1942|8445|648|2061|6|11|5|4|2061|648|8445|Saturday|2061Q4|N|Y|N|2474130|2474433|2473769|2474042|N|N|N|N|N| +2474135|AAAAAAAAHJAMFCAA|2061-11-06|1942|8445|648|2061|0|11|6|4|2061|648|8445|Sunday|2061Q4|N|N|N|2474130|2474433|2473770|2474043|N|N|N|N|N| +2474136|AAAAAAAAIJAMFCAA|2061-11-07|1942|8445|648|2061|1|11|7|4|2061|648|8445|Monday|2061Q4|N|N|N|2474130|2474433|2473771|2474044|N|N|N|N|N| +2474137|AAAAAAAAJJAMFCAA|2061-11-08|1942|8446|648|2061|2|11|8|4|2061|648|8446|Tuesday|2061Q4|N|N|N|2474130|2474433|2473772|2474045|N|N|N|N|N| +2474138|AAAAAAAAKJAMFCAA|2061-11-09|1942|8446|648|2061|3|11|9|4|2061|648|8446|Wednesday|2061Q4|N|N|N|2474130|2474433|2473773|2474046|N|N|N|N|N| +2474139|AAAAAAAALJAMFCAA|2061-11-10|1942|8446|648|2061|4|11|10|4|2061|648|8446|Thursday|2061Q4|N|N|N|2474130|2474433|2473774|2474047|N|N|N|N|N| +2474140|AAAAAAAAMJAMFCAA|2061-11-11|1942|8446|648|2061|5|11|11|4|2061|648|8446|Friday|2061Q4|N|Y|N|2474130|2474433|2473775|2474048|N|N|N|N|N| +2474141|AAAAAAAANJAMFCAA|2061-11-12|1942|8446|648|2061|6|11|12|4|2061|648|8446|Saturday|2061Q4|N|Y|N|2474130|2474433|2473776|2474049|N|N|N|N|N| +2474142|AAAAAAAAOJAMFCAA|2061-11-13|1942|8446|648|2061|0|11|13|4|2061|648|8446|Sunday|2061Q4|N|N|N|2474130|2474433|2473777|2474050|N|N|N|N|N| +2474143|AAAAAAAAPJAMFCAA|2061-11-14|1942|8446|648|2061|1|11|14|4|2061|648|8446|Monday|2061Q4|N|N|N|2474130|2474433|2473778|2474051|N|N|N|N|N| +2474144|AAAAAAAAAKAMFCAA|2061-11-15|1942|8447|648|2061|2|11|15|4|2061|648|8447|Tuesday|2061Q4|N|N|N|2474130|2474433|2473779|2474052|N|N|N|N|N| +2474145|AAAAAAAABKAMFCAA|2061-11-16|1942|8447|648|2061|3|11|16|4|2061|648|8447|Wednesday|2061Q4|N|N|N|2474130|2474433|2473780|2474053|N|N|N|N|N| +2474146|AAAAAAAACKAMFCAA|2061-11-17|1942|8447|648|2061|4|11|17|4|2061|648|8447|Thursday|2061Q4|N|N|N|2474130|2474433|2473781|2474054|N|N|N|N|N| +2474147|AAAAAAAADKAMFCAA|2061-11-18|1942|8447|648|2061|5|11|18|4|2061|648|8447|Friday|2061Q4|N|Y|N|2474130|2474433|2473782|2474055|N|N|N|N|N| +2474148|AAAAAAAAEKAMFCAA|2061-11-19|1942|8447|648|2061|6|11|19|4|2061|648|8447|Saturday|2061Q4|N|Y|N|2474130|2474433|2473783|2474056|N|N|N|N|N| +2474149|AAAAAAAAFKAMFCAA|2061-11-20|1942|8447|648|2061|0|11|20|4|2061|648|8447|Sunday|2061Q4|N|N|N|2474130|2474433|2473784|2474057|N|N|N|N|N| +2474150|AAAAAAAAGKAMFCAA|2061-11-21|1942|8447|648|2061|1|11|21|4|2061|648|8447|Monday|2061Q4|N|N|N|2474130|2474433|2473785|2474058|N|N|N|N|N| +2474151|AAAAAAAAHKAMFCAA|2061-11-22|1942|8448|648|2061|2|11|22|4|2061|648|8448|Tuesday|2061Q4|N|N|N|2474130|2474433|2473786|2474059|N|N|N|N|N| +2474152|AAAAAAAAIKAMFCAA|2061-11-23|1942|8448|648|2061|3|11|23|4|2061|648|8448|Wednesday|2061Q4|N|N|N|2474130|2474433|2473787|2474060|N|N|N|N|N| +2474153|AAAAAAAAJKAMFCAA|2061-11-24|1942|8448|648|2061|4|11|24|4|2061|648|8448|Thursday|2061Q4|N|N|N|2474130|2474433|2473788|2474061|N|N|N|N|N| +2474154|AAAAAAAAKKAMFCAA|2061-11-25|1942|8448|648|2061|5|11|25|4|2061|648|8448|Friday|2061Q4|N|Y|N|2474130|2474433|2473789|2474062|N|N|N|N|N| +2474155|AAAAAAAALKAMFCAA|2061-11-26|1942|8448|648|2061|6|11|26|4|2061|648|8448|Saturday|2061Q4|N|Y|N|2474130|2474433|2473790|2474063|N|N|N|N|N| +2474156|AAAAAAAAMKAMFCAA|2061-11-27|1942|8448|648|2061|0|11|27|4|2061|648|8448|Sunday|2061Q4|N|N|N|2474130|2474433|2473791|2474064|N|N|N|N|N| +2474157|AAAAAAAANKAMFCAA|2061-11-28|1942|8448|648|2061|1|11|28|4|2061|648|8448|Monday|2061Q4|N|N|N|2474130|2474433|2473792|2474065|N|N|N|N|N| +2474158|AAAAAAAAOKAMFCAA|2061-11-29|1942|8449|648|2061|2|11|29|4|2061|648|8449|Tuesday|2061Q4|N|N|N|2474130|2474433|2473793|2474066|N|N|N|N|N| +2474159|AAAAAAAAPKAMFCAA|2061-11-30|1942|8449|648|2061|3|11|30|4|2061|648|8449|Wednesday|2061Q4|N|N|N|2474130|2474433|2473794|2474067|N|N|N|N|N| +2474160|AAAAAAAAALAMFCAA|2061-12-01|1943|8449|649|2061|4|12|1|4|2061|649|8449|Thursday|2061Q4|N|N|N|2474160|2474493|2473795|2474068|N|N|N|N|N| +2474161|AAAAAAAABLAMFCAA|2061-12-02|1943|8449|649|2061|5|12|2|4|2061|649|8449|Friday|2061Q4|N|Y|N|2474160|2474493|2473796|2474069|N|N|N|N|N| +2474162|AAAAAAAACLAMFCAA|2061-12-03|1943|8449|649|2061|6|12|3|4|2061|649|8449|Saturday|2061Q4|N|Y|N|2474160|2474493|2473797|2474070|N|N|N|N|N| +2474163|AAAAAAAADLAMFCAA|2061-12-04|1943|8449|649|2061|0|12|4|4|2061|649|8449|Sunday|2061Q4|N|N|N|2474160|2474493|2473798|2474071|N|N|N|N|N| +2474164|AAAAAAAAELAMFCAA|2061-12-05|1943|8449|649|2061|1|12|5|4|2061|649|8449|Monday|2061Q4|N|N|N|2474160|2474493|2473799|2474072|N|N|N|N|N| +2474165|AAAAAAAAFLAMFCAA|2061-12-06|1943|8450|649|2061|2|12|6|4|2061|649|8450|Tuesday|2061Q4|N|N|N|2474160|2474493|2473800|2474073|N|N|N|N|N| +2474166|AAAAAAAAGLAMFCAA|2061-12-07|1943|8450|649|2061|3|12|7|4|2061|649|8450|Wednesday|2061Q4|N|N|N|2474160|2474493|2473801|2474074|N|N|N|N|N| +2474167|AAAAAAAAHLAMFCAA|2061-12-08|1943|8450|649|2061|4|12|8|4|2061|649|8450|Thursday|2061Q4|N|N|N|2474160|2474493|2473802|2474075|N|N|N|N|N| +2474168|AAAAAAAAILAMFCAA|2061-12-09|1943|8450|649|2061|5|12|9|4|2061|649|8450|Friday|2061Q4|N|Y|N|2474160|2474493|2473803|2474076|N|N|N|N|N| +2474169|AAAAAAAAJLAMFCAA|2061-12-10|1943|8450|649|2061|6|12|10|4|2061|649|8450|Saturday|2061Q4|N|Y|N|2474160|2474493|2473804|2474077|N|N|N|N|N| +2474170|AAAAAAAAKLAMFCAA|2061-12-11|1943|8450|649|2061|0|12|11|4|2061|649|8450|Sunday|2061Q4|N|N|N|2474160|2474493|2473805|2474078|N|N|N|N|N| +2474171|AAAAAAAALLAMFCAA|2061-12-12|1943|8450|649|2061|1|12|12|4|2061|649|8450|Monday|2061Q4|N|N|N|2474160|2474493|2473806|2474079|N|N|N|N|N| +2474172|AAAAAAAAMLAMFCAA|2061-12-13|1943|8451|649|2061|2|12|13|4|2061|649|8451|Tuesday|2061Q4|N|N|N|2474160|2474493|2473807|2474080|N|N|N|N|N| +2474173|AAAAAAAANLAMFCAA|2061-12-14|1943|8451|649|2061|3|12|14|4|2061|649|8451|Wednesday|2061Q4|N|N|N|2474160|2474493|2473808|2474081|N|N|N|N|N| +2474174|AAAAAAAAOLAMFCAA|2061-12-15|1943|8451|649|2061|4|12|15|4|2061|649|8451|Thursday|2061Q4|N|N|N|2474160|2474493|2473809|2474082|N|N|N|N|N| +2474175|AAAAAAAAPLAMFCAA|2061-12-16|1943|8451|649|2061|5|12|16|4|2061|649|8451|Friday|2061Q4|N|Y|N|2474160|2474493|2473810|2474083|N|N|N|N|N| +2474176|AAAAAAAAAMAMFCAA|2061-12-17|1943|8451|649|2061|6|12|17|4|2061|649|8451|Saturday|2061Q4|N|Y|N|2474160|2474493|2473811|2474084|N|N|N|N|N| +2474177|AAAAAAAABMAMFCAA|2061-12-18|1943|8451|649|2061|0|12|18|4|2061|649|8451|Sunday|2061Q4|N|N|N|2474160|2474493|2473812|2474085|N|N|N|N|N| +2474178|AAAAAAAACMAMFCAA|2061-12-19|1943|8451|649|2061|1|12|19|4|2061|649|8451|Monday|2061Q4|N|N|N|2474160|2474493|2473813|2474086|N|N|N|N|N| +2474179|AAAAAAAADMAMFCAA|2061-12-20|1943|8452|649|2061|2|12|20|4|2061|649|8452|Tuesday|2061Q4|N|N|N|2474160|2474493|2473814|2474087|N|N|N|N|N| +2474180|AAAAAAAAEMAMFCAA|2061-12-21|1943|8452|649|2061|3|12|21|4|2061|649|8452|Wednesday|2061Q4|N|N|N|2474160|2474493|2473815|2474088|N|N|N|N|N| +2474181|AAAAAAAAFMAMFCAA|2061-12-22|1943|8452|649|2061|4|12|22|4|2061|649|8452|Thursday|2061Q4|N|N|N|2474160|2474493|2473816|2474089|N|N|N|N|N| +2474182|AAAAAAAAGMAMFCAA|2061-12-23|1943|8452|649|2061|5|12|23|4|2061|649|8452|Friday|2061Q4|N|Y|N|2474160|2474493|2473817|2474090|N|N|N|N|N| +2474183|AAAAAAAAHMAMFCAA|2061-12-24|1943|8452|649|2061|6|12|24|4|2061|649|8452|Saturday|2061Q4|N|Y|N|2474160|2474493|2473818|2474091|N|N|N|N|N| +2474184|AAAAAAAAIMAMFCAA|2061-12-25|1943|8452|649|2061|0|12|25|4|2061|649|8452|Sunday|2061Q4|N|N|N|2474160|2474493|2473819|2474092|N|N|N|N|N| +2474185|AAAAAAAAJMAMFCAA|2061-12-26|1943|8452|649|2061|1|12|26|4|2061|649|8452|Monday|2061Q4|Y|N|N|2474160|2474493|2473820|2474093|N|N|N|N|N| +2474186|AAAAAAAAKMAMFCAA|2061-12-27|1943|8453|649|2061|2|12|27|4|2061|649|8453|Tuesday|2061Q4|N|N|Y|2474160|2474493|2473821|2474094|N|N|N|N|N| +2474187|AAAAAAAALMAMFCAA|2061-12-28|1943|8453|649|2061|3|12|28|4|2061|649|8453|Wednesday|2061Q4|N|N|N|2474160|2474493|2473822|2474095|N|N|N|N|N| +2474188|AAAAAAAAMMAMFCAA|2061-12-29|1943|8453|649|2061|4|12|29|4|2061|649|8453|Thursday|2061Q4|N|N|N|2474160|2474493|2473823|2474096|N|N|N|N|N| +2474189|AAAAAAAANMAMFCAA|2061-12-30|1943|8453|649|2061|5|12|30|4|2061|649|8453|Friday|2061Q4|N|Y|N|2474160|2474493|2473824|2474097|N|N|N|N|N| +2474190|AAAAAAAAOMAMFCAA|2061-12-31|1943|8453|649|2061|6|12|31|4|2061|649|8453|Saturday|2061Q4|N|Y|N|2474160|2474493|2473825|2474098|N|N|N|N|N| +2474191|AAAAAAAAPMAMFCAA|2062-01-01|1944|8453|649|2062|0|1|1|1|2062|649|8453|Sunday|2062Q1|Y|N|N|2474191|2474190|2473826|2474099|N|N|N|N|N| +2474192|AAAAAAAAANAMFCAA|2062-01-02|1944|8453|649|2062|1|1|2|1|2062|649|8453|Monday|2062Q1|N|N|Y|2474191|2474190|2473827|2474100|N|N|N|N|N| +2474193|AAAAAAAABNAMFCAA|2062-01-03|1944|8454|649|2062|2|1|3|1|2062|649|8454|Tuesday|2062Q1|N|N|N|2474191|2474190|2473828|2474101|N|N|N|N|N| +2474194|AAAAAAAACNAMFCAA|2062-01-04|1944|8454|649|2062|3|1|4|1|2062|649|8454|Wednesday|2062Q1|N|N|N|2474191|2474190|2473829|2474102|N|N|N|N|N| +2474195|AAAAAAAADNAMFCAA|2062-01-05|1944|8454|649|2062|4|1|5|1|2062|649|8454|Thursday|2062Q1|N|N|N|2474191|2474190|2473830|2474103|N|N|N|N|N| +2474196|AAAAAAAAENAMFCAA|2062-01-06|1944|8454|649|2062|5|1|6|1|2062|649|8454|Friday|2062Q1|N|Y|N|2474191|2474190|2473831|2474104|N|N|N|N|N| +2474197|AAAAAAAAFNAMFCAA|2062-01-07|1944|8454|649|2062|6|1|7|1|2062|649|8454|Saturday|2062Q1|N|Y|N|2474191|2474190|2473832|2474105|N|N|N|N|N| +2474198|AAAAAAAAGNAMFCAA|2062-01-08|1944|8454|649|2062|0|1|8|1|2062|649|8454|Sunday|2062Q1|N|N|N|2474191|2474190|2473833|2474106|N|N|N|N|N| +2474199|AAAAAAAAHNAMFCAA|2062-01-09|1944|8454|649|2062|1|1|9|1|2062|649|8454|Monday|2062Q1|N|N|N|2474191|2474190|2473834|2474107|N|N|N|N|N| +2474200|AAAAAAAAINAMFCAA|2062-01-10|1944|8455|649|2062|2|1|10|1|2062|649|8455|Tuesday|2062Q1|N|N|N|2474191|2474190|2473835|2474108|N|N|N|N|N| +2474201|AAAAAAAAJNAMFCAA|2062-01-11|1944|8455|649|2062|3|1|11|1|2062|649|8455|Wednesday|2062Q1|N|N|N|2474191|2474190|2473836|2474109|N|N|N|N|N| +2474202|AAAAAAAAKNAMFCAA|2062-01-12|1944|8455|649|2062|4|1|12|1|2062|649|8455|Thursday|2062Q1|N|N|N|2474191|2474190|2473837|2474110|N|N|N|N|N| +2474203|AAAAAAAALNAMFCAA|2062-01-13|1944|8455|649|2062|5|1|13|1|2062|649|8455|Friday|2062Q1|N|Y|N|2474191|2474190|2473838|2474111|N|N|N|N|N| +2474204|AAAAAAAAMNAMFCAA|2062-01-14|1944|8455|649|2062|6|1|14|1|2062|649|8455|Saturday|2062Q1|N|Y|N|2474191|2474190|2473839|2474112|N|N|N|N|N| +2474205|AAAAAAAANNAMFCAA|2062-01-15|1944|8455|649|2062|0|1|15|1|2062|649|8455|Sunday|2062Q1|N|N|N|2474191|2474190|2473840|2474113|N|N|N|N|N| +2474206|AAAAAAAAONAMFCAA|2062-01-16|1944|8455|649|2062|1|1|16|1|2062|649|8455|Monday|2062Q1|N|N|N|2474191|2474190|2473841|2474114|N|N|N|N|N| +2474207|AAAAAAAAPNAMFCAA|2062-01-17|1944|8456|649|2062|2|1|17|1|2062|649|8456|Tuesday|2062Q1|N|N|N|2474191|2474190|2473842|2474115|N|N|N|N|N| +2474208|AAAAAAAAAOAMFCAA|2062-01-18|1944|8456|649|2062|3|1|18|1|2062|649|8456|Wednesday|2062Q1|N|N|N|2474191|2474190|2473843|2474116|N|N|N|N|N| +2474209|AAAAAAAABOAMFCAA|2062-01-19|1944|8456|649|2062|4|1|19|1|2062|649|8456|Thursday|2062Q1|N|N|N|2474191|2474190|2473844|2474117|N|N|N|N|N| +2474210|AAAAAAAACOAMFCAA|2062-01-20|1944|8456|649|2062|5|1|20|1|2062|649|8456|Friday|2062Q1|N|Y|N|2474191|2474190|2473845|2474118|N|N|N|N|N| +2474211|AAAAAAAADOAMFCAA|2062-01-21|1944|8456|649|2062|6|1|21|1|2062|649|8456|Saturday|2062Q1|N|Y|N|2474191|2474190|2473846|2474119|N|N|N|N|N| +2474212|AAAAAAAAEOAMFCAA|2062-01-22|1944|8456|649|2062|0|1|22|1|2062|649|8456|Sunday|2062Q1|N|N|N|2474191|2474190|2473847|2474120|N|N|N|N|N| +2474213|AAAAAAAAFOAMFCAA|2062-01-23|1944|8456|649|2062|1|1|23|1|2062|649|8456|Monday|2062Q1|N|N|N|2474191|2474190|2473848|2474121|N|N|N|N|N| +2474214|AAAAAAAAGOAMFCAA|2062-01-24|1944|8457|649|2062|2|1|24|1|2062|649|8457|Tuesday|2062Q1|N|N|N|2474191|2474190|2473849|2474122|N|N|N|N|N| +2474215|AAAAAAAAHOAMFCAA|2062-01-25|1944|8457|649|2062|3|1|25|1|2062|649|8457|Wednesday|2062Q1|N|N|N|2474191|2474190|2473850|2474123|N|N|N|N|N| +2474216|AAAAAAAAIOAMFCAA|2062-01-26|1944|8457|649|2062|4|1|26|1|2062|649|8457|Thursday|2062Q1|N|N|N|2474191|2474190|2473851|2474124|N|N|N|N|N| +2474217|AAAAAAAAJOAMFCAA|2062-01-27|1944|8457|649|2062|5|1|27|1|2062|649|8457|Friday|2062Q1|N|Y|N|2474191|2474190|2473852|2474125|N|N|N|N|N| +2474218|AAAAAAAAKOAMFCAA|2062-01-28|1944|8457|649|2062|6|1|28|1|2062|649|8457|Saturday|2062Q1|N|Y|N|2474191|2474190|2473853|2474126|N|N|N|N|N| +2474219|AAAAAAAALOAMFCAA|2062-01-29|1944|8457|649|2062|0|1|29|1|2062|649|8457|Sunday|2062Q1|N|N|N|2474191|2474190|2473854|2474127|N|N|N|N|N| +2474220|AAAAAAAAMOAMFCAA|2062-01-30|1944|8457|649|2062|1|1|30|1|2062|649|8457|Monday|2062Q1|N|N|N|2474191|2474190|2473855|2474128|N|N|N|N|N| +2474221|AAAAAAAANOAMFCAA|2062-01-31|1944|8458|649|2062|2|1|31|1|2062|649|8458|Tuesday|2062Q1|N|N|N|2474191|2474190|2473856|2474129|N|N|N|N|N| +2474222|AAAAAAAAOOAMFCAA|2062-02-01|1945|8458|649|2062|3|2|1|1|2062|649|8458|Wednesday|2062Q1|N|N|N|2474222|2474252|2473857|2474130|N|N|N|N|N| +2474223|AAAAAAAAPOAMFCAA|2062-02-02|1945|8458|649|2062|4|2|2|1|2062|649|8458|Thursday|2062Q1|N|N|N|2474222|2474252|2473858|2474131|N|N|N|N|N| +2474224|AAAAAAAAAPAMFCAA|2062-02-03|1945|8458|649|2062|5|2|3|1|2062|649|8458|Friday|2062Q1|N|Y|N|2474222|2474252|2473859|2474132|N|N|N|N|N| +2474225|AAAAAAAABPAMFCAA|2062-02-04|1945|8458|649|2062|6|2|4|1|2062|649|8458|Saturday|2062Q1|N|Y|N|2474222|2474252|2473860|2474133|N|N|N|N|N| +2474226|AAAAAAAACPAMFCAA|2062-02-05|1945|8458|649|2062|0|2|5|1|2062|649|8458|Sunday|2062Q1|N|N|N|2474222|2474252|2473861|2474134|N|N|N|N|N| +2474227|AAAAAAAADPAMFCAA|2062-02-06|1945|8458|649|2062|1|2|6|1|2062|649|8458|Monday|2062Q1|N|N|N|2474222|2474252|2473862|2474135|N|N|N|N|N| +2474228|AAAAAAAAEPAMFCAA|2062-02-07|1945|8459|649|2062|2|2|7|1|2062|649|8459|Tuesday|2062Q1|N|N|N|2474222|2474252|2473863|2474136|N|N|N|N|N| +2474229|AAAAAAAAFPAMFCAA|2062-02-08|1945|8459|649|2062|3|2|8|1|2062|649|8459|Wednesday|2062Q1|N|N|N|2474222|2474252|2473864|2474137|N|N|N|N|N| +2474230|AAAAAAAAGPAMFCAA|2062-02-09|1945|8459|649|2062|4|2|9|1|2062|649|8459|Thursday|2062Q1|N|N|N|2474222|2474252|2473865|2474138|N|N|N|N|N| +2474231|AAAAAAAAHPAMFCAA|2062-02-10|1945|8459|649|2062|5|2|10|1|2062|649|8459|Friday|2062Q1|N|Y|N|2474222|2474252|2473866|2474139|N|N|N|N|N| +2474232|AAAAAAAAIPAMFCAA|2062-02-11|1945|8459|649|2062|6|2|11|1|2062|649|8459|Saturday|2062Q1|N|Y|N|2474222|2474252|2473867|2474140|N|N|N|N|N| +2474233|AAAAAAAAJPAMFCAA|2062-02-12|1945|8459|649|2062|0|2|12|1|2062|649|8459|Sunday|2062Q1|N|N|N|2474222|2474252|2473868|2474141|N|N|N|N|N| +2474234|AAAAAAAAKPAMFCAA|2062-02-13|1945|8459|649|2062|1|2|13|1|2062|649|8459|Monday|2062Q1|N|N|N|2474222|2474252|2473869|2474142|N|N|N|N|N| +2474235|AAAAAAAALPAMFCAA|2062-02-14|1945|8460|649|2062|2|2|14|1|2062|649|8460|Tuesday|2062Q1|N|N|N|2474222|2474252|2473870|2474143|N|N|N|N|N| +2474236|AAAAAAAAMPAMFCAA|2062-02-15|1945|8460|649|2062|3|2|15|1|2062|649|8460|Wednesday|2062Q1|N|N|N|2474222|2474252|2473871|2474144|N|N|N|N|N| +2474237|AAAAAAAANPAMFCAA|2062-02-16|1945|8460|649|2062|4|2|16|1|2062|649|8460|Thursday|2062Q1|N|N|N|2474222|2474252|2473872|2474145|N|N|N|N|N| +2474238|AAAAAAAAOPAMFCAA|2062-02-17|1945|8460|649|2062|5|2|17|1|2062|649|8460|Friday|2062Q1|N|Y|N|2474222|2474252|2473873|2474146|N|N|N|N|N| +2474239|AAAAAAAAPPAMFCAA|2062-02-18|1945|8460|649|2062|6|2|18|1|2062|649|8460|Saturday|2062Q1|N|Y|N|2474222|2474252|2473874|2474147|N|N|N|N|N| +2474240|AAAAAAAAAABMFCAA|2062-02-19|1945|8460|649|2062|0|2|19|1|2062|649|8460|Sunday|2062Q1|N|N|N|2474222|2474252|2473875|2474148|N|N|N|N|N| +2474241|AAAAAAAABABMFCAA|2062-02-20|1945|8460|649|2062|1|2|20|1|2062|649|8460|Monday|2062Q1|N|N|N|2474222|2474252|2473876|2474149|N|N|N|N|N| +2474242|AAAAAAAACABMFCAA|2062-02-21|1945|8461|649|2062|2|2|21|1|2062|649|8461|Tuesday|2062Q1|N|N|N|2474222|2474252|2473877|2474150|N|N|N|N|N| +2474243|AAAAAAAADABMFCAA|2062-02-22|1945|8461|649|2062|3|2|22|1|2062|649|8461|Wednesday|2062Q1|N|N|N|2474222|2474252|2473878|2474151|N|N|N|N|N| +2474244|AAAAAAAAEABMFCAA|2062-02-23|1945|8461|649|2062|4|2|23|1|2062|649|8461|Thursday|2062Q1|N|N|N|2474222|2474252|2473879|2474152|N|N|N|N|N| +2474245|AAAAAAAAFABMFCAA|2062-02-24|1945|8461|649|2062|5|2|24|1|2062|649|8461|Friday|2062Q1|N|Y|N|2474222|2474252|2473880|2474153|N|N|N|N|N| +2474246|AAAAAAAAGABMFCAA|2062-02-25|1945|8461|649|2062|6|2|25|1|2062|649|8461|Saturday|2062Q1|N|Y|N|2474222|2474252|2473881|2474154|N|N|N|N|N| +2474247|AAAAAAAAHABMFCAA|2062-02-26|1945|8461|649|2062|0|2|26|1|2062|649|8461|Sunday|2062Q1|N|N|N|2474222|2474252|2473882|2474155|N|N|N|N|N| +2474248|AAAAAAAAIABMFCAA|2062-02-27|1945|8461|649|2062|1|2|27|1|2062|649|8461|Monday|2062Q1|N|N|N|2474222|2474252|2473883|2474156|N|N|N|N|N| +2474249|AAAAAAAAJABMFCAA|2062-02-28|1945|8462|649|2062|2|2|28|1|2062|649|8462|Tuesday|2062Q1|N|N|N|2474222|2474252|2473884|2474157|N|N|N|N|N| +2474250|AAAAAAAAKABMFCAA|2062-03-01|1946|8462|650|2062|3|3|1|1|2062|650|8462|Wednesday|2062Q1|N|N|N|2474250|2474308|2473885|2474158|N|N|N|N|N| +2474251|AAAAAAAALABMFCAA|2062-03-02|1946|8462|650|2062|4|3|2|1|2062|650|8462|Thursday|2062Q1|N|N|N|2474250|2474308|2473886|2474159|N|N|N|N|N| +2474252|AAAAAAAAMABMFCAA|2062-03-03|1946|8462|650|2062|5|3|3|1|2062|650|8462|Friday|2062Q1|N|Y|N|2474250|2474308|2473887|2474160|N|N|N|N|N| +2474253|AAAAAAAANABMFCAA|2062-03-04|1946|8462|650|2062|6|3|4|1|2062|650|8462|Saturday|2062Q1|N|Y|N|2474250|2474308|2473888|2474161|N|N|N|N|N| +2474254|AAAAAAAAOABMFCAA|2062-03-05|1946|8462|650|2062|0|3|5|1|2062|650|8462|Sunday|2062Q1|N|N|N|2474250|2474308|2473889|2474162|N|N|N|N|N| +2474255|AAAAAAAAPABMFCAA|2062-03-06|1946|8462|650|2062|1|3|6|1|2062|650|8462|Monday|2062Q1|N|N|N|2474250|2474308|2473890|2474163|N|N|N|N|N| +2474256|AAAAAAAAABBMFCAA|2062-03-07|1946|8463|650|2062|2|3|7|1|2062|650|8463|Tuesday|2062Q1|N|N|N|2474250|2474308|2473891|2474164|N|N|N|N|N| +2474257|AAAAAAAABBBMFCAA|2062-03-08|1946|8463|650|2062|3|3|8|1|2062|650|8463|Wednesday|2062Q1|N|N|N|2474250|2474308|2473892|2474165|N|N|N|N|N| +2474258|AAAAAAAACBBMFCAA|2062-03-09|1946|8463|650|2062|4|3|9|1|2062|650|8463|Thursday|2062Q1|N|N|N|2474250|2474308|2473893|2474166|N|N|N|N|N| +2474259|AAAAAAAADBBMFCAA|2062-03-10|1946|8463|650|2062|5|3|10|1|2062|650|8463|Friday|2062Q1|N|Y|N|2474250|2474308|2473894|2474167|N|N|N|N|N| +2474260|AAAAAAAAEBBMFCAA|2062-03-11|1946|8463|650|2062|6|3|11|1|2062|650|8463|Saturday|2062Q1|N|Y|N|2474250|2474308|2473895|2474168|N|N|N|N|N| +2474261|AAAAAAAAFBBMFCAA|2062-03-12|1946|8463|650|2062|0|3|12|1|2062|650|8463|Sunday|2062Q1|N|N|N|2474250|2474308|2473896|2474169|N|N|N|N|N| +2474262|AAAAAAAAGBBMFCAA|2062-03-13|1946|8463|650|2062|1|3|13|1|2062|650|8463|Monday|2062Q1|N|N|N|2474250|2474308|2473897|2474170|N|N|N|N|N| +2474263|AAAAAAAAHBBMFCAA|2062-03-14|1946|8464|650|2062|2|3|14|1|2062|650|8464|Tuesday|2062Q1|N|N|N|2474250|2474308|2473898|2474171|N|N|N|N|N| +2474264|AAAAAAAAIBBMFCAA|2062-03-15|1946|8464|650|2062|3|3|15|1|2062|650|8464|Wednesday|2062Q1|N|N|N|2474250|2474308|2473899|2474172|N|N|N|N|N| +2474265|AAAAAAAAJBBMFCAA|2062-03-16|1946|8464|650|2062|4|3|16|1|2062|650|8464|Thursday|2062Q1|N|N|N|2474250|2474308|2473900|2474173|N|N|N|N|N| +2474266|AAAAAAAAKBBMFCAA|2062-03-17|1946|8464|650|2062|5|3|17|1|2062|650|8464|Friday|2062Q1|N|Y|N|2474250|2474308|2473901|2474174|N|N|N|N|N| +2474267|AAAAAAAALBBMFCAA|2062-03-18|1946|8464|650|2062|6|3|18|1|2062|650|8464|Saturday|2062Q1|N|Y|N|2474250|2474308|2473902|2474175|N|N|N|N|N| +2474268|AAAAAAAAMBBMFCAA|2062-03-19|1946|8464|650|2062|0|3|19|1|2062|650|8464|Sunday|2062Q1|N|N|N|2474250|2474308|2473903|2474176|N|N|N|N|N| +2474269|AAAAAAAANBBMFCAA|2062-03-20|1946|8464|650|2062|1|3|20|1|2062|650|8464|Monday|2062Q1|N|N|N|2474250|2474308|2473904|2474177|N|N|N|N|N| +2474270|AAAAAAAAOBBMFCAA|2062-03-21|1946|8465|650|2062|2|3|21|1|2062|650|8465|Tuesday|2062Q1|N|N|N|2474250|2474308|2473905|2474178|N|N|N|N|N| +2474271|AAAAAAAAPBBMFCAA|2062-03-22|1946|8465|650|2062|3|3|22|1|2062|650|8465|Wednesday|2062Q1|N|N|N|2474250|2474308|2473906|2474179|N|N|N|N|N| +2474272|AAAAAAAAACBMFCAA|2062-03-23|1946|8465|650|2062|4|3|23|1|2062|650|8465|Thursday|2062Q1|N|N|N|2474250|2474308|2473907|2474180|N|N|N|N|N| +2474273|AAAAAAAABCBMFCAA|2062-03-24|1946|8465|650|2062|5|3|24|1|2062|650|8465|Friday|2062Q1|N|Y|N|2474250|2474308|2473908|2474181|N|N|N|N|N| +2474274|AAAAAAAACCBMFCAA|2062-03-25|1946|8465|650|2062|6|3|25|1|2062|650|8465|Saturday|2062Q1|N|Y|N|2474250|2474308|2473909|2474182|N|N|N|N|N| +2474275|AAAAAAAADCBMFCAA|2062-03-26|1946|8465|650|2062|0|3|26|1|2062|650|8465|Sunday|2062Q1|N|N|N|2474250|2474308|2473910|2474183|N|N|N|N|N| +2474276|AAAAAAAAECBMFCAA|2062-03-27|1946|8465|650|2062|1|3|27|1|2062|650|8465|Monday|2062Q1|N|N|N|2474250|2474308|2473911|2474184|N|N|N|N|N| +2474277|AAAAAAAAFCBMFCAA|2062-03-28|1946|8466|650|2062|2|3|28|1|2062|650|8466|Tuesday|2062Q1|N|N|N|2474250|2474308|2473912|2474185|N|N|N|N|N| +2474278|AAAAAAAAGCBMFCAA|2062-03-29|1946|8466|650|2062|3|3|29|1|2062|650|8466|Wednesday|2062Q1|N|N|N|2474250|2474308|2473913|2474186|N|N|N|N|N| +2474279|AAAAAAAAHCBMFCAA|2062-03-30|1946|8466|650|2062|4|3|30|1|2062|650|8466|Thursday|2062Q1|N|N|N|2474250|2474308|2473914|2474187|N|N|N|N|N| +2474280|AAAAAAAAICBMFCAA|2062-03-31|1946|8466|650|2062|5|3|31|1|2062|650|8466|Friday|2062Q1|N|Y|N|2474250|2474308|2473915|2474188|N|N|N|N|N| +2474281|AAAAAAAAJCBMFCAA|2062-04-01|1947|8466|650|2062|6|4|1|1|2062|650|8466|Saturday|2062Q1|N|Y|N|2474281|2474370|2473916|2474191|N|N|N|N|N| +2474282|AAAAAAAAKCBMFCAA|2062-04-02|1947|8466|650|2062|0|4|2|2|2062|650|8466|Sunday|2062Q2|N|N|N|2474281|2474370|2473917|2474192|N|N|N|N|N| +2474283|AAAAAAAALCBMFCAA|2062-04-03|1947|8466|650|2062|1|4|3|2|2062|650|8466|Monday|2062Q2|N|N|N|2474281|2474370|2473918|2474193|N|N|N|N|N| +2474284|AAAAAAAAMCBMFCAA|2062-04-04|1947|8467|650|2062|2|4|4|2|2062|650|8467|Tuesday|2062Q2|N|N|N|2474281|2474370|2473919|2474194|N|N|N|N|N| +2474285|AAAAAAAANCBMFCAA|2062-04-05|1947|8467|650|2062|3|4|5|2|2062|650|8467|Wednesday|2062Q2|N|N|N|2474281|2474370|2473920|2474195|N|N|N|N|N| +2474286|AAAAAAAAOCBMFCAA|2062-04-06|1947|8467|650|2062|4|4|6|2|2062|650|8467|Thursday|2062Q2|N|N|N|2474281|2474370|2473921|2474196|N|N|N|N|N| +2474287|AAAAAAAAPCBMFCAA|2062-04-07|1947|8467|650|2062|5|4|7|2|2062|650|8467|Friday|2062Q2|N|Y|N|2474281|2474370|2473922|2474197|N|N|N|N|N| +2474288|AAAAAAAAADBMFCAA|2062-04-08|1947|8467|650|2062|6|4|8|2|2062|650|8467|Saturday|2062Q2|N|Y|N|2474281|2474370|2473923|2474198|N|N|N|N|N| +2474289|AAAAAAAABDBMFCAA|2062-04-09|1947|8467|650|2062|0|4|9|2|2062|650|8467|Sunday|2062Q2|N|N|N|2474281|2474370|2473924|2474199|N|N|N|N|N| +2474290|AAAAAAAACDBMFCAA|2062-04-10|1947|8467|650|2062|1|4|10|2|2062|650|8467|Monday|2062Q2|N|N|N|2474281|2474370|2473925|2474200|N|N|N|N|N| +2474291|AAAAAAAADDBMFCAA|2062-04-11|1947|8468|650|2062|2|4|11|2|2062|650|8468|Tuesday|2062Q2|N|N|N|2474281|2474370|2473926|2474201|N|N|N|N|N| +2474292|AAAAAAAAEDBMFCAA|2062-04-12|1947|8468|650|2062|3|4|12|2|2062|650|8468|Wednesday|2062Q2|N|N|N|2474281|2474370|2473927|2474202|N|N|N|N|N| +2474293|AAAAAAAAFDBMFCAA|2062-04-13|1947|8468|650|2062|4|4|13|2|2062|650|8468|Thursday|2062Q2|N|N|N|2474281|2474370|2473928|2474203|N|N|N|N|N| +2474294|AAAAAAAAGDBMFCAA|2062-04-14|1947|8468|650|2062|5|4|14|2|2062|650|8468|Friday|2062Q2|N|Y|N|2474281|2474370|2473929|2474204|N|N|N|N|N| +2474295|AAAAAAAAHDBMFCAA|2062-04-15|1947|8468|650|2062|6|4|15|2|2062|650|8468|Saturday|2062Q2|N|Y|N|2474281|2474370|2473930|2474205|N|N|N|N|N| +2474296|AAAAAAAAIDBMFCAA|2062-04-16|1947|8468|650|2062|0|4|16|2|2062|650|8468|Sunday|2062Q2|N|N|N|2474281|2474370|2473931|2474206|N|N|N|N|N| +2474297|AAAAAAAAJDBMFCAA|2062-04-17|1947|8468|650|2062|1|4|17|2|2062|650|8468|Monday|2062Q2|N|N|N|2474281|2474370|2473932|2474207|N|N|N|N|N| +2474298|AAAAAAAAKDBMFCAA|2062-04-18|1947|8469|650|2062|2|4|18|2|2062|650|8469|Tuesday|2062Q2|N|N|N|2474281|2474370|2473933|2474208|N|N|N|N|N| +2474299|AAAAAAAALDBMFCAA|2062-04-19|1947|8469|650|2062|3|4|19|2|2062|650|8469|Wednesday|2062Q2|N|N|N|2474281|2474370|2473934|2474209|N|N|N|N|N| +2474300|AAAAAAAAMDBMFCAA|2062-04-20|1947|8469|650|2062|4|4|20|2|2062|650|8469|Thursday|2062Q2|N|N|N|2474281|2474370|2473935|2474210|N|N|N|N|N| +2474301|AAAAAAAANDBMFCAA|2062-04-21|1947|8469|650|2062|5|4|21|2|2062|650|8469|Friday|2062Q2|N|Y|N|2474281|2474370|2473936|2474211|N|N|N|N|N| +2474302|AAAAAAAAODBMFCAA|2062-04-22|1947|8469|650|2062|6|4|22|2|2062|650|8469|Saturday|2062Q2|N|Y|N|2474281|2474370|2473937|2474212|N|N|N|N|N| +2474303|AAAAAAAAPDBMFCAA|2062-04-23|1947|8469|650|2062|0|4|23|2|2062|650|8469|Sunday|2062Q2|N|N|N|2474281|2474370|2473938|2474213|N|N|N|N|N| +2474304|AAAAAAAAAEBMFCAA|2062-04-24|1947|8469|650|2062|1|4|24|2|2062|650|8469|Monday|2062Q2|N|N|N|2474281|2474370|2473939|2474214|N|N|N|N|N| +2474305|AAAAAAAABEBMFCAA|2062-04-25|1947|8470|650|2062|2|4|25|2|2062|650|8470|Tuesday|2062Q2|N|N|N|2474281|2474370|2473940|2474215|N|N|N|N|N| +2474306|AAAAAAAACEBMFCAA|2062-04-26|1947|8470|650|2062|3|4|26|2|2062|650|8470|Wednesday|2062Q2|N|N|N|2474281|2474370|2473941|2474216|N|N|N|N|N| +2474307|AAAAAAAADEBMFCAA|2062-04-27|1947|8470|650|2062|4|4|27|2|2062|650|8470|Thursday|2062Q2|N|N|N|2474281|2474370|2473942|2474217|N|N|N|N|N| +2474308|AAAAAAAAEEBMFCAA|2062-04-28|1947|8470|650|2062|5|4|28|2|2062|650|8470|Friday|2062Q2|N|Y|N|2474281|2474370|2473943|2474218|N|N|N|N|N| +2474309|AAAAAAAAFEBMFCAA|2062-04-29|1947|8470|650|2062|6|4|29|2|2062|650|8470|Saturday|2062Q2|N|Y|N|2474281|2474370|2473944|2474219|N|N|N|N|N| +2474310|AAAAAAAAGEBMFCAA|2062-04-30|1947|8470|650|2062|0|4|30|2|2062|650|8470|Sunday|2062Q2|N|N|N|2474281|2474370|2473945|2474220|N|N|N|N|N| +2474311|AAAAAAAAHEBMFCAA|2062-05-01|1948|8470|650|2062|1|5|1|2|2062|650|8470|Monday|2062Q2|N|N|N|2474311|2474430|2473946|2474221|N|N|N|N|N| +2474312|AAAAAAAAIEBMFCAA|2062-05-02|1948|8471|650|2062|2|5|2|2|2062|650|8471|Tuesday|2062Q2|N|N|N|2474311|2474430|2473947|2474222|N|N|N|N|N| +2474313|AAAAAAAAJEBMFCAA|2062-05-03|1948|8471|650|2062|3|5|3|2|2062|650|8471|Wednesday|2062Q2|N|N|N|2474311|2474430|2473948|2474223|N|N|N|N|N| +2474314|AAAAAAAAKEBMFCAA|2062-05-04|1948|8471|650|2062|4|5|4|2|2062|650|8471|Thursday|2062Q2|N|N|N|2474311|2474430|2473949|2474224|N|N|N|N|N| +2474315|AAAAAAAALEBMFCAA|2062-05-05|1948|8471|650|2062|5|5|5|2|2062|650|8471|Friday|2062Q2|N|Y|N|2474311|2474430|2473950|2474225|N|N|N|N|N| +2474316|AAAAAAAAMEBMFCAA|2062-05-06|1948|8471|650|2062|6|5|6|2|2062|650|8471|Saturday|2062Q2|N|Y|N|2474311|2474430|2473951|2474226|N|N|N|N|N| +2474317|AAAAAAAANEBMFCAA|2062-05-07|1948|8471|650|2062|0|5|7|2|2062|650|8471|Sunday|2062Q2|N|N|N|2474311|2474430|2473952|2474227|N|N|N|N|N| +2474318|AAAAAAAAOEBMFCAA|2062-05-08|1948|8471|650|2062|1|5|8|2|2062|650|8471|Monday|2062Q2|N|N|N|2474311|2474430|2473953|2474228|N|N|N|N|N| +2474319|AAAAAAAAPEBMFCAA|2062-05-09|1948|8472|650|2062|2|5|9|2|2062|650|8472|Tuesday|2062Q2|N|N|N|2474311|2474430|2473954|2474229|N|N|N|N|N| +2474320|AAAAAAAAAFBMFCAA|2062-05-10|1948|8472|650|2062|3|5|10|2|2062|650|8472|Wednesday|2062Q2|N|N|N|2474311|2474430|2473955|2474230|N|N|N|N|N| +2474321|AAAAAAAABFBMFCAA|2062-05-11|1948|8472|650|2062|4|5|11|2|2062|650|8472|Thursday|2062Q2|N|N|N|2474311|2474430|2473956|2474231|N|N|N|N|N| +2474322|AAAAAAAACFBMFCAA|2062-05-12|1948|8472|650|2062|5|5|12|2|2062|650|8472|Friday|2062Q2|N|Y|N|2474311|2474430|2473957|2474232|N|N|N|N|N| +2474323|AAAAAAAADFBMFCAA|2062-05-13|1948|8472|650|2062|6|5|13|2|2062|650|8472|Saturday|2062Q2|N|Y|N|2474311|2474430|2473958|2474233|N|N|N|N|N| +2474324|AAAAAAAAEFBMFCAA|2062-05-14|1948|8472|650|2062|0|5|14|2|2062|650|8472|Sunday|2062Q2|N|N|N|2474311|2474430|2473959|2474234|N|N|N|N|N| +2474325|AAAAAAAAFFBMFCAA|2062-05-15|1948|8472|650|2062|1|5|15|2|2062|650|8472|Monday|2062Q2|N|N|N|2474311|2474430|2473960|2474235|N|N|N|N|N| +2474326|AAAAAAAAGFBMFCAA|2062-05-16|1948|8473|650|2062|2|5|16|2|2062|650|8473|Tuesday|2062Q2|N|N|N|2474311|2474430|2473961|2474236|N|N|N|N|N| +2474327|AAAAAAAAHFBMFCAA|2062-05-17|1948|8473|650|2062|3|5|17|2|2062|650|8473|Wednesday|2062Q2|N|N|N|2474311|2474430|2473962|2474237|N|N|N|N|N| +2474328|AAAAAAAAIFBMFCAA|2062-05-18|1948|8473|650|2062|4|5|18|2|2062|650|8473|Thursday|2062Q2|N|N|N|2474311|2474430|2473963|2474238|N|N|N|N|N| +2474329|AAAAAAAAJFBMFCAA|2062-05-19|1948|8473|650|2062|5|5|19|2|2062|650|8473|Friday|2062Q2|N|Y|N|2474311|2474430|2473964|2474239|N|N|N|N|N| +2474330|AAAAAAAAKFBMFCAA|2062-05-20|1948|8473|650|2062|6|5|20|2|2062|650|8473|Saturday|2062Q2|N|Y|N|2474311|2474430|2473965|2474240|N|N|N|N|N| +2474331|AAAAAAAALFBMFCAA|2062-05-21|1948|8473|650|2062|0|5|21|2|2062|650|8473|Sunday|2062Q2|N|N|N|2474311|2474430|2473966|2474241|N|N|N|N|N| +2474332|AAAAAAAAMFBMFCAA|2062-05-22|1948|8473|650|2062|1|5|22|2|2062|650|8473|Monday|2062Q2|N|N|N|2474311|2474430|2473967|2474242|N|N|N|N|N| +2474333|AAAAAAAANFBMFCAA|2062-05-23|1948|8474|650|2062|2|5|23|2|2062|650|8474|Tuesday|2062Q2|N|N|N|2474311|2474430|2473968|2474243|N|N|N|N|N| +2474334|AAAAAAAAOFBMFCAA|2062-05-24|1948|8474|650|2062|3|5|24|2|2062|650|8474|Wednesday|2062Q2|N|N|N|2474311|2474430|2473969|2474244|N|N|N|N|N| +2474335|AAAAAAAAPFBMFCAA|2062-05-25|1948|8474|650|2062|4|5|25|2|2062|650|8474|Thursday|2062Q2|N|N|N|2474311|2474430|2473970|2474245|N|N|N|N|N| +2474336|AAAAAAAAAGBMFCAA|2062-05-26|1948|8474|650|2062|5|5|26|2|2062|650|8474|Friday|2062Q2|N|Y|N|2474311|2474430|2473971|2474246|N|N|N|N|N| +2474337|AAAAAAAABGBMFCAA|2062-05-27|1948|8474|650|2062|6|5|27|2|2062|650|8474|Saturday|2062Q2|N|Y|N|2474311|2474430|2473972|2474247|N|N|N|N|N| +2474338|AAAAAAAACGBMFCAA|2062-05-28|1948|8474|650|2062|0|5|28|2|2062|650|8474|Sunday|2062Q2|N|N|N|2474311|2474430|2473973|2474248|N|N|N|N|N| +2474339|AAAAAAAADGBMFCAA|2062-05-29|1948|8474|650|2062|1|5|29|2|2062|650|8474|Monday|2062Q2|N|N|N|2474311|2474430|2473974|2474249|N|N|N|N|N| +2474340|AAAAAAAAEGBMFCAA|2062-05-30|1948|8475|650|2062|2|5|30|2|2062|650|8475|Tuesday|2062Q2|N|N|N|2474311|2474430|2473975|2474250|N|N|N|N|N| +2474341|AAAAAAAAFGBMFCAA|2062-05-31|1948|8475|650|2062|3|5|31|2|2062|650|8475|Wednesday|2062Q2|N|N|N|2474311|2474430|2473976|2474251|N|N|N|N|N| +2474342|AAAAAAAAGGBMFCAA|2062-06-01|1949|8475|651|2062|4|6|1|2|2062|651|8475|Thursday|2062Q2|N|N|N|2474342|2474492|2473977|2474252|N|N|N|N|N| +2474343|AAAAAAAAHGBMFCAA|2062-06-02|1949|8475|651|2062|5|6|2|2|2062|651|8475|Friday|2062Q2|N|Y|N|2474342|2474492|2473978|2474253|N|N|N|N|N| +2474344|AAAAAAAAIGBMFCAA|2062-06-03|1949|8475|651|2062|6|6|3|2|2062|651|8475|Saturday|2062Q2|N|Y|N|2474342|2474492|2473979|2474254|N|N|N|N|N| +2474345|AAAAAAAAJGBMFCAA|2062-06-04|1949|8475|651|2062|0|6|4|2|2062|651|8475|Sunday|2062Q2|N|N|N|2474342|2474492|2473980|2474255|N|N|N|N|N| +2474346|AAAAAAAAKGBMFCAA|2062-06-05|1949|8475|651|2062|1|6|5|2|2062|651|8475|Monday|2062Q2|N|N|N|2474342|2474492|2473981|2474256|N|N|N|N|N| +2474347|AAAAAAAALGBMFCAA|2062-06-06|1949|8476|651|2062|2|6|6|2|2062|651|8476|Tuesday|2062Q2|N|N|N|2474342|2474492|2473982|2474257|N|N|N|N|N| +2474348|AAAAAAAAMGBMFCAA|2062-06-07|1949|8476|651|2062|3|6|7|2|2062|651|8476|Wednesday|2062Q2|N|N|N|2474342|2474492|2473983|2474258|N|N|N|N|N| +2474349|AAAAAAAANGBMFCAA|2062-06-08|1949|8476|651|2062|4|6|8|2|2062|651|8476|Thursday|2062Q2|N|N|N|2474342|2474492|2473984|2474259|N|N|N|N|N| +2474350|AAAAAAAAOGBMFCAA|2062-06-09|1949|8476|651|2062|5|6|9|2|2062|651|8476|Friday|2062Q2|N|Y|N|2474342|2474492|2473985|2474260|N|N|N|N|N| +2474351|AAAAAAAAPGBMFCAA|2062-06-10|1949|8476|651|2062|6|6|10|2|2062|651|8476|Saturday|2062Q2|N|Y|N|2474342|2474492|2473986|2474261|N|N|N|N|N| +2474352|AAAAAAAAAHBMFCAA|2062-06-11|1949|8476|651|2062|0|6|11|2|2062|651|8476|Sunday|2062Q2|N|N|N|2474342|2474492|2473987|2474262|N|N|N|N|N| +2474353|AAAAAAAABHBMFCAA|2062-06-12|1949|8476|651|2062|1|6|12|2|2062|651|8476|Monday|2062Q2|N|N|N|2474342|2474492|2473988|2474263|N|N|N|N|N| +2474354|AAAAAAAACHBMFCAA|2062-06-13|1949|8477|651|2062|2|6|13|2|2062|651|8477|Tuesday|2062Q2|N|N|N|2474342|2474492|2473989|2474264|N|N|N|N|N| +2474355|AAAAAAAADHBMFCAA|2062-06-14|1949|8477|651|2062|3|6|14|2|2062|651|8477|Wednesday|2062Q2|N|N|N|2474342|2474492|2473990|2474265|N|N|N|N|N| +2474356|AAAAAAAAEHBMFCAA|2062-06-15|1949|8477|651|2062|4|6|15|2|2062|651|8477|Thursday|2062Q2|N|N|N|2474342|2474492|2473991|2474266|N|N|N|N|N| +2474357|AAAAAAAAFHBMFCAA|2062-06-16|1949|8477|651|2062|5|6|16|2|2062|651|8477|Friday|2062Q2|N|Y|N|2474342|2474492|2473992|2474267|N|N|N|N|N| +2474358|AAAAAAAAGHBMFCAA|2062-06-17|1949|8477|651|2062|6|6|17|2|2062|651|8477|Saturday|2062Q2|N|Y|N|2474342|2474492|2473993|2474268|N|N|N|N|N| +2474359|AAAAAAAAHHBMFCAA|2062-06-18|1949|8477|651|2062|0|6|18|2|2062|651|8477|Sunday|2062Q2|N|N|N|2474342|2474492|2473994|2474269|N|N|N|N|N| +2474360|AAAAAAAAIHBMFCAA|2062-06-19|1949|8477|651|2062|1|6|19|2|2062|651|8477|Monday|2062Q2|N|N|N|2474342|2474492|2473995|2474270|N|N|N|N|N| +2474361|AAAAAAAAJHBMFCAA|2062-06-20|1949|8478|651|2062|2|6|20|2|2062|651|8478|Tuesday|2062Q2|N|N|N|2474342|2474492|2473996|2474271|N|N|N|N|N| +2474362|AAAAAAAAKHBMFCAA|2062-06-21|1949|8478|651|2062|3|6|21|2|2062|651|8478|Wednesday|2062Q2|N|N|N|2474342|2474492|2473997|2474272|N|N|N|N|N| +2474363|AAAAAAAALHBMFCAA|2062-06-22|1949|8478|651|2062|4|6|22|2|2062|651|8478|Thursday|2062Q2|N|N|N|2474342|2474492|2473998|2474273|N|N|N|N|N| +2474364|AAAAAAAAMHBMFCAA|2062-06-23|1949|8478|651|2062|5|6|23|2|2062|651|8478|Friday|2062Q2|N|Y|N|2474342|2474492|2473999|2474274|N|N|N|N|N| +2474365|AAAAAAAANHBMFCAA|2062-06-24|1949|8478|651|2062|6|6|24|2|2062|651|8478|Saturday|2062Q2|N|Y|N|2474342|2474492|2474000|2474275|N|N|N|N|N| +2474366|AAAAAAAAOHBMFCAA|2062-06-25|1949|8478|651|2062|0|6|25|2|2062|651|8478|Sunday|2062Q2|N|N|N|2474342|2474492|2474001|2474276|N|N|N|N|N| +2474367|AAAAAAAAPHBMFCAA|2062-06-26|1949|8478|651|2062|1|6|26|2|2062|651|8478|Monday|2062Q2|N|N|N|2474342|2474492|2474002|2474277|N|N|N|N|N| +2474368|AAAAAAAAAIBMFCAA|2062-06-27|1949|8479|651|2062|2|6|27|2|2062|651|8479|Tuesday|2062Q2|N|N|N|2474342|2474492|2474003|2474278|N|N|N|N|N| +2474369|AAAAAAAABIBMFCAA|2062-06-28|1949|8479|651|2062|3|6|28|2|2062|651|8479|Wednesday|2062Q2|N|N|N|2474342|2474492|2474004|2474279|N|N|N|N|N| +2474370|AAAAAAAACIBMFCAA|2062-06-29|1949|8479|651|2062|4|6|29|2|2062|651|8479|Thursday|2062Q2|N|N|N|2474342|2474492|2474005|2474280|N|N|N|N|N| +2474371|AAAAAAAADIBMFCAA|2062-06-30|1949|8479|651|2062|5|6|30|2|2062|651|8479|Friday|2062Q2|N|Y|N|2474342|2474492|2474006|2474281|N|N|N|N|N| +2474372|AAAAAAAAEIBMFCAA|2062-07-01|1950|8479|651|2062|6|7|1|2|2062|651|8479|Saturday|2062Q2|N|Y|N|2474372|2474552|2474007|2474281|N|N|N|N|N| +2474373|AAAAAAAAFIBMFCAA|2062-07-02|1950|8479|651|2062|0|7|2|3|2062|651|8479|Sunday|2062Q3|N|N|N|2474372|2474552|2474008|2474282|N|N|N|N|N| +2474374|AAAAAAAAGIBMFCAA|2062-07-03|1950|8479|651|2062|1|7|3|3|2062|651|8479|Monday|2062Q3|N|N|N|2474372|2474552|2474009|2474283|N|N|N|N|N| +2474375|AAAAAAAAHIBMFCAA|2062-07-04|1950|8480|651|2062|2|7|4|3|2062|651|8480|Tuesday|2062Q3|N|N|N|2474372|2474552|2474010|2474284|N|N|N|N|N| +2474376|AAAAAAAAIIBMFCAA|2062-07-05|1950|8480|651|2062|3|7|5|3|2062|651|8480|Wednesday|2062Q3|Y|N|N|2474372|2474552|2474011|2474285|N|N|N|N|N| +2474377|AAAAAAAAJIBMFCAA|2062-07-06|1950|8480|651|2062|4|7|6|3|2062|651|8480|Thursday|2062Q3|N|N|Y|2474372|2474552|2474012|2474286|N|N|N|N|N| +2474378|AAAAAAAAKIBMFCAA|2062-07-07|1950|8480|651|2062|5|7|7|3|2062|651|8480|Friday|2062Q3|N|Y|N|2474372|2474552|2474013|2474287|N|N|N|N|N| +2474379|AAAAAAAALIBMFCAA|2062-07-08|1950|8480|651|2062|6|7|8|3|2062|651|8480|Saturday|2062Q3|N|Y|N|2474372|2474552|2474014|2474288|N|N|N|N|N| +2474380|AAAAAAAAMIBMFCAA|2062-07-09|1950|8480|651|2062|0|7|9|3|2062|651|8480|Sunday|2062Q3|N|N|N|2474372|2474552|2474015|2474289|N|N|N|N|N| +2474381|AAAAAAAANIBMFCAA|2062-07-10|1950|8480|651|2062|1|7|10|3|2062|651|8480|Monday|2062Q3|N|N|N|2474372|2474552|2474016|2474290|N|N|N|N|N| +2474382|AAAAAAAAOIBMFCAA|2062-07-11|1950|8481|651|2062|2|7|11|3|2062|651|8481|Tuesday|2062Q3|N|N|N|2474372|2474552|2474017|2474291|N|N|N|N|N| +2474383|AAAAAAAAPIBMFCAA|2062-07-12|1950|8481|651|2062|3|7|12|3|2062|651|8481|Wednesday|2062Q3|N|N|N|2474372|2474552|2474018|2474292|N|N|N|N|N| +2474384|AAAAAAAAAJBMFCAA|2062-07-13|1950|8481|651|2062|4|7|13|3|2062|651|8481|Thursday|2062Q3|N|N|N|2474372|2474552|2474019|2474293|N|N|N|N|N| +2474385|AAAAAAAABJBMFCAA|2062-07-14|1950|8481|651|2062|5|7|14|3|2062|651|8481|Friday|2062Q3|N|Y|N|2474372|2474552|2474020|2474294|N|N|N|N|N| +2474386|AAAAAAAACJBMFCAA|2062-07-15|1950|8481|651|2062|6|7|15|3|2062|651|8481|Saturday|2062Q3|N|Y|N|2474372|2474552|2474021|2474295|N|N|N|N|N| +2474387|AAAAAAAADJBMFCAA|2062-07-16|1950|8481|651|2062|0|7|16|3|2062|651|8481|Sunday|2062Q3|N|N|N|2474372|2474552|2474022|2474296|N|N|N|N|N| +2474388|AAAAAAAAEJBMFCAA|2062-07-17|1950|8481|651|2062|1|7|17|3|2062|651|8481|Monday|2062Q3|N|N|N|2474372|2474552|2474023|2474297|N|N|N|N|N| +2474389|AAAAAAAAFJBMFCAA|2062-07-18|1950|8482|651|2062|2|7|18|3|2062|651|8482|Tuesday|2062Q3|N|N|N|2474372|2474552|2474024|2474298|N|N|N|N|N| +2474390|AAAAAAAAGJBMFCAA|2062-07-19|1950|8482|651|2062|3|7|19|3|2062|651|8482|Wednesday|2062Q3|N|N|N|2474372|2474552|2474025|2474299|N|N|N|N|N| +2474391|AAAAAAAAHJBMFCAA|2062-07-20|1950|8482|651|2062|4|7|20|3|2062|651|8482|Thursday|2062Q3|N|N|N|2474372|2474552|2474026|2474300|N|N|N|N|N| +2474392|AAAAAAAAIJBMFCAA|2062-07-21|1950|8482|651|2062|5|7|21|3|2062|651|8482|Friday|2062Q3|N|Y|N|2474372|2474552|2474027|2474301|N|N|N|N|N| +2474393|AAAAAAAAJJBMFCAA|2062-07-22|1950|8482|651|2062|6|7|22|3|2062|651|8482|Saturday|2062Q3|N|Y|N|2474372|2474552|2474028|2474302|N|N|N|N|N| +2474394|AAAAAAAAKJBMFCAA|2062-07-23|1950|8482|651|2062|0|7|23|3|2062|651|8482|Sunday|2062Q3|N|N|N|2474372|2474552|2474029|2474303|N|N|N|N|N| +2474395|AAAAAAAALJBMFCAA|2062-07-24|1950|8482|651|2062|1|7|24|3|2062|651|8482|Monday|2062Q3|N|N|N|2474372|2474552|2474030|2474304|N|N|N|N|N| +2474396|AAAAAAAAMJBMFCAA|2062-07-25|1950|8483|651|2062|2|7|25|3|2062|651|8483|Tuesday|2062Q3|N|N|N|2474372|2474552|2474031|2474305|N|N|N|N|N| +2474397|AAAAAAAANJBMFCAA|2062-07-26|1950|8483|651|2062|3|7|26|3|2062|651|8483|Wednesday|2062Q3|N|N|N|2474372|2474552|2474032|2474306|N|N|N|N|N| +2474398|AAAAAAAAOJBMFCAA|2062-07-27|1950|8483|651|2062|4|7|27|3|2062|651|8483|Thursday|2062Q3|N|N|N|2474372|2474552|2474033|2474307|N|N|N|N|N| +2474399|AAAAAAAAPJBMFCAA|2062-07-28|1950|8483|651|2062|5|7|28|3|2062|651|8483|Friday|2062Q3|N|Y|N|2474372|2474552|2474034|2474308|N|N|N|N|N| +2474400|AAAAAAAAAKBMFCAA|2062-07-29|1950|8483|651|2062|6|7|29|3|2062|651|8483|Saturday|2062Q3|N|Y|N|2474372|2474552|2474035|2474309|N|N|N|N|N| +2474401|AAAAAAAABKBMFCAA|2062-07-30|1950|8483|651|2062|0|7|30|3|2062|651|8483|Sunday|2062Q3|N|N|N|2474372|2474552|2474036|2474310|N|N|N|N|N| +2474402|AAAAAAAACKBMFCAA|2062-07-31|1950|8483|651|2062|1|7|31|3|2062|651|8483|Monday|2062Q3|N|N|N|2474372|2474552|2474037|2474311|N|N|N|N|N| +2474403|AAAAAAAADKBMFCAA|2062-08-01|1951|8484|651|2062|2|8|1|3|2062|651|8484|Tuesday|2062Q3|N|N|N|2474403|2474614|2474038|2474312|N|N|N|N|N| +2474404|AAAAAAAAEKBMFCAA|2062-08-02|1951|8484|651|2062|3|8|2|3|2062|651|8484|Wednesday|2062Q3|N|N|N|2474403|2474614|2474039|2474313|N|N|N|N|N| +2474405|AAAAAAAAFKBMFCAA|2062-08-03|1951|8484|651|2062|4|8|3|3|2062|651|8484|Thursday|2062Q3|N|N|N|2474403|2474614|2474040|2474314|N|N|N|N|N| +2474406|AAAAAAAAGKBMFCAA|2062-08-04|1951|8484|651|2062|5|8|4|3|2062|651|8484|Friday|2062Q3|N|Y|N|2474403|2474614|2474041|2474315|N|N|N|N|N| +2474407|AAAAAAAAHKBMFCAA|2062-08-05|1951|8484|651|2062|6|8|5|3|2062|651|8484|Saturday|2062Q3|N|Y|N|2474403|2474614|2474042|2474316|N|N|N|N|N| +2474408|AAAAAAAAIKBMFCAA|2062-08-06|1951|8484|651|2062|0|8|6|3|2062|651|8484|Sunday|2062Q3|N|N|N|2474403|2474614|2474043|2474317|N|N|N|N|N| +2474409|AAAAAAAAJKBMFCAA|2062-08-07|1951|8484|651|2062|1|8|7|3|2062|651|8484|Monday|2062Q3|N|N|N|2474403|2474614|2474044|2474318|N|N|N|N|N| +2474410|AAAAAAAAKKBMFCAA|2062-08-08|1951|8485|651|2062|2|8|8|3|2062|651|8485|Tuesday|2062Q3|N|N|N|2474403|2474614|2474045|2474319|N|N|N|N|N| +2474411|AAAAAAAALKBMFCAA|2062-08-09|1951|8485|651|2062|3|8|9|3|2062|651|8485|Wednesday|2062Q3|N|N|N|2474403|2474614|2474046|2474320|N|N|N|N|N| +2474412|AAAAAAAAMKBMFCAA|2062-08-10|1951|8485|651|2062|4|8|10|3|2062|651|8485|Thursday|2062Q3|N|N|N|2474403|2474614|2474047|2474321|N|N|N|N|N| +2474413|AAAAAAAANKBMFCAA|2062-08-11|1951|8485|651|2062|5|8|11|3|2062|651|8485|Friday|2062Q3|N|Y|N|2474403|2474614|2474048|2474322|N|N|N|N|N| +2474414|AAAAAAAAOKBMFCAA|2062-08-12|1951|8485|651|2062|6|8|12|3|2062|651|8485|Saturday|2062Q3|N|Y|N|2474403|2474614|2474049|2474323|N|N|N|N|N| +2474415|AAAAAAAAPKBMFCAA|2062-08-13|1951|8485|651|2062|0|8|13|3|2062|651|8485|Sunday|2062Q3|N|N|N|2474403|2474614|2474050|2474324|N|N|N|N|N| +2474416|AAAAAAAAALBMFCAA|2062-08-14|1951|8485|651|2062|1|8|14|3|2062|651|8485|Monday|2062Q3|N|N|N|2474403|2474614|2474051|2474325|N|N|N|N|N| +2474417|AAAAAAAABLBMFCAA|2062-08-15|1951|8486|651|2062|2|8|15|3|2062|651|8486|Tuesday|2062Q3|N|N|N|2474403|2474614|2474052|2474326|N|N|N|N|N| +2474418|AAAAAAAACLBMFCAA|2062-08-16|1951|8486|651|2062|3|8|16|3|2062|651|8486|Wednesday|2062Q3|N|N|N|2474403|2474614|2474053|2474327|N|N|N|N|N| +2474419|AAAAAAAADLBMFCAA|2062-08-17|1951|8486|651|2062|4|8|17|3|2062|651|8486|Thursday|2062Q3|N|N|N|2474403|2474614|2474054|2474328|N|N|N|N|N| +2474420|AAAAAAAAELBMFCAA|2062-08-18|1951|8486|651|2062|5|8|18|3|2062|651|8486|Friday|2062Q3|N|Y|N|2474403|2474614|2474055|2474329|N|N|N|N|N| +2474421|AAAAAAAAFLBMFCAA|2062-08-19|1951|8486|651|2062|6|8|19|3|2062|651|8486|Saturday|2062Q3|N|Y|N|2474403|2474614|2474056|2474330|N|N|N|N|N| +2474422|AAAAAAAAGLBMFCAA|2062-08-20|1951|8486|651|2062|0|8|20|3|2062|651|8486|Sunday|2062Q3|N|N|N|2474403|2474614|2474057|2474331|N|N|N|N|N| +2474423|AAAAAAAAHLBMFCAA|2062-08-21|1951|8486|651|2062|1|8|21|3|2062|651|8486|Monday|2062Q3|N|N|N|2474403|2474614|2474058|2474332|N|N|N|N|N| +2474424|AAAAAAAAILBMFCAA|2062-08-22|1951|8487|651|2062|2|8|22|3|2062|651|8487|Tuesday|2062Q3|N|N|N|2474403|2474614|2474059|2474333|N|N|N|N|N| +2474425|AAAAAAAAJLBMFCAA|2062-08-23|1951|8487|651|2062|3|8|23|3|2062|651|8487|Wednesday|2062Q3|N|N|N|2474403|2474614|2474060|2474334|N|N|N|N|N| +2474426|AAAAAAAAKLBMFCAA|2062-08-24|1951|8487|651|2062|4|8|24|3|2062|651|8487|Thursday|2062Q3|N|N|N|2474403|2474614|2474061|2474335|N|N|N|N|N| +2474427|AAAAAAAALLBMFCAA|2062-08-25|1951|8487|651|2062|5|8|25|3|2062|651|8487|Friday|2062Q3|N|Y|N|2474403|2474614|2474062|2474336|N|N|N|N|N| +2474428|AAAAAAAAMLBMFCAA|2062-08-26|1951|8487|651|2062|6|8|26|3|2062|651|8487|Saturday|2062Q3|N|Y|N|2474403|2474614|2474063|2474337|N|N|N|N|N| +2474429|AAAAAAAANLBMFCAA|2062-08-27|1951|8487|651|2062|0|8|27|3|2062|651|8487|Sunday|2062Q3|N|N|N|2474403|2474614|2474064|2474338|N|N|N|N|N| +2474430|AAAAAAAAOLBMFCAA|2062-08-28|1951|8487|651|2062|1|8|28|3|2062|651|8487|Monday|2062Q3|N|N|N|2474403|2474614|2474065|2474339|N|N|N|N|N| +2474431|AAAAAAAAPLBMFCAA|2062-08-29|1951|8488|651|2062|2|8|29|3|2062|651|8488|Tuesday|2062Q3|N|N|N|2474403|2474614|2474066|2474340|N|N|N|N|N| +2474432|AAAAAAAAAMBMFCAA|2062-08-30|1951|8488|651|2062|3|8|30|3|2062|651|8488|Wednesday|2062Q3|N|N|N|2474403|2474614|2474067|2474341|N|N|N|N|N| +2474433|AAAAAAAABMBMFCAA|2062-08-31|1951|8488|651|2062|4|8|31|3|2062|651|8488|Thursday|2062Q3|N|N|N|2474403|2474614|2474068|2474342|N|N|N|N|N| +2474434|AAAAAAAACMBMFCAA|2062-09-01|1952|8488|652|2062|5|9|1|3|2062|652|8488|Friday|2062Q3|N|Y|N|2474434|2474676|2474069|2474343|N|N|N|N|N| +2474435|AAAAAAAADMBMFCAA|2062-09-02|1952|8488|652|2062|6|9|2|3|2062|652|8488|Saturday|2062Q3|N|Y|N|2474434|2474676|2474070|2474344|N|N|N|N|N| +2474436|AAAAAAAAEMBMFCAA|2062-09-03|1952|8488|652|2062|0|9|3|3|2062|652|8488|Sunday|2062Q3|N|N|N|2474434|2474676|2474071|2474345|N|N|N|N|N| +2474437|AAAAAAAAFMBMFCAA|2062-09-04|1952|8488|652|2062|1|9|4|3|2062|652|8488|Monday|2062Q3|N|N|N|2474434|2474676|2474072|2474346|N|N|N|N|N| +2474438|AAAAAAAAGMBMFCAA|2062-09-05|1952|8489|652|2062|2|9|5|3|2062|652|8489|Tuesday|2062Q3|N|N|N|2474434|2474676|2474073|2474347|N|N|N|N|N| +2474439|AAAAAAAAHMBMFCAA|2062-09-06|1952|8489|652|2062|3|9|6|3|2062|652|8489|Wednesday|2062Q3|N|N|N|2474434|2474676|2474074|2474348|N|N|N|N|N| +2474440|AAAAAAAAIMBMFCAA|2062-09-07|1952|8489|652|2062|4|9|7|3|2062|652|8489|Thursday|2062Q3|N|N|N|2474434|2474676|2474075|2474349|N|N|N|N|N| +2474441|AAAAAAAAJMBMFCAA|2062-09-08|1952|8489|652|2062|5|9|8|3|2062|652|8489|Friday|2062Q3|N|Y|N|2474434|2474676|2474076|2474350|N|N|N|N|N| +2474442|AAAAAAAAKMBMFCAA|2062-09-09|1952|8489|652|2062|6|9|9|3|2062|652|8489|Saturday|2062Q3|N|Y|N|2474434|2474676|2474077|2474351|N|N|N|N|N| +2474443|AAAAAAAALMBMFCAA|2062-09-10|1952|8489|652|2062|0|9|10|3|2062|652|8489|Sunday|2062Q3|N|N|N|2474434|2474676|2474078|2474352|N|N|N|N|N| +2474444|AAAAAAAAMMBMFCAA|2062-09-11|1952|8489|652|2062|1|9|11|3|2062|652|8489|Monday|2062Q3|N|N|N|2474434|2474676|2474079|2474353|N|N|N|N|N| +2474445|AAAAAAAANMBMFCAA|2062-09-12|1952|8490|652|2062|2|9|12|3|2062|652|8490|Tuesday|2062Q3|N|N|N|2474434|2474676|2474080|2474354|N|N|N|N|N| +2474446|AAAAAAAAOMBMFCAA|2062-09-13|1952|8490|652|2062|3|9|13|3|2062|652|8490|Wednesday|2062Q3|N|N|N|2474434|2474676|2474081|2474355|N|N|N|N|N| +2474447|AAAAAAAAPMBMFCAA|2062-09-14|1952|8490|652|2062|4|9|14|3|2062|652|8490|Thursday|2062Q3|N|N|N|2474434|2474676|2474082|2474356|N|N|N|N|N| +2474448|AAAAAAAAANBMFCAA|2062-09-15|1952|8490|652|2062|5|9|15|3|2062|652|8490|Friday|2062Q3|N|Y|N|2474434|2474676|2474083|2474357|N|N|N|N|N| +2474449|AAAAAAAABNBMFCAA|2062-09-16|1952|8490|652|2062|6|9|16|3|2062|652|8490|Saturday|2062Q3|N|Y|N|2474434|2474676|2474084|2474358|N|N|N|N|N| +2474450|AAAAAAAACNBMFCAA|2062-09-17|1952|8490|652|2062|0|9|17|3|2062|652|8490|Sunday|2062Q3|N|N|N|2474434|2474676|2474085|2474359|N|N|N|N|N| +2474451|AAAAAAAADNBMFCAA|2062-09-18|1952|8490|652|2062|1|9|18|3|2062|652|8490|Monday|2062Q3|N|N|N|2474434|2474676|2474086|2474360|N|N|N|N|N| +2474452|AAAAAAAAENBMFCAA|2062-09-19|1952|8491|652|2062|2|9|19|3|2062|652|8491|Tuesday|2062Q3|N|N|N|2474434|2474676|2474087|2474361|N|N|N|N|N| +2474453|AAAAAAAAFNBMFCAA|2062-09-20|1952|8491|652|2062|3|9|20|3|2062|652|8491|Wednesday|2062Q3|N|N|N|2474434|2474676|2474088|2474362|N|N|N|N|N| +2474454|AAAAAAAAGNBMFCAA|2062-09-21|1952|8491|652|2062|4|9|21|3|2062|652|8491|Thursday|2062Q3|N|N|N|2474434|2474676|2474089|2474363|N|N|N|N|N| +2474455|AAAAAAAAHNBMFCAA|2062-09-22|1952|8491|652|2062|5|9|22|3|2062|652|8491|Friday|2062Q3|N|Y|N|2474434|2474676|2474090|2474364|N|N|N|N|N| +2474456|AAAAAAAAINBMFCAA|2062-09-23|1952|8491|652|2062|6|9|23|3|2062|652|8491|Saturday|2062Q3|N|Y|N|2474434|2474676|2474091|2474365|N|N|N|N|N| +2474457|AAAAAAAAJNBMFCAA|2062-09-24|1952|8491|652|2062|0|9|24|3|2062|652|8491|Sunday|2062Q3|N|N|N|2474434|2474676|2474092|2474366|N|N|N|N|N| +2474458|AAAAAAAAKNBMFCAA|2062-09-25|1952|8491|652|2062|1|9|25|3|2062|652|8491|Monday|2062Q3|N|N|N|2474434|2474676|2474093|2474367|N|N|N|N|N| +2474459|AAAAAAAALNBMFCAA|2062-09-26|1952|8492|652|2062|2|9|26|3|2062|652|8492|Tuesday|2062Q3|N|N|N|2474434|2474676|2474094|2474368|N|N|N|N|N| +2474460|AAAAAAAAMNBMFCAA|2062-09-27|1952|8492|652|2062|3|9|27|3|2062|652|8492|Wednesday|2062Q3|N|N|N|2474434|2474676|2474095|2474369|N|N|N|N|N| +2474461|AAAAAAAANNBMFCAA|2062-09-28|1952|8492|652|2062|4|9|28|3|2062|652|8492|Thursday|2062Q3|N|N|N|2474434|2474676|2474096|2474370|N|N|N|N|N| +2474462|AAAAAAAAONBMFCAA|2062-09-29|1952|8492|652|2062|5|9|29|3|2062|652|8492|Friday|2062Q3|N|Y|N|2474434|2474676|2474097|2474371|N|N|N|N|N| +2474463|AAAAAAAAPNBMFCAA|2062-09-30|1952|8492|652|2062|6|9|30|3|2062|652|8492|Saturday|2062Q3|N|Y|N|2474434|2474676|2474098|2474372|N|N|N|N|N| +2474464|AAAAAAAAAOBMFCAA|2062-10-01|1953|8492|652|2062|0|10|1|3|2062|652|8492|Sunday|2062Q3|N|N|N|2474464|2474736|2474099|2474372|N|N|N|N|N| +2474465|AAAAAAAABOBMFCAA|2062-10-02|1953|8492|652|2062|1|10|2|4|2062|652|8492|Monday|2062Q4|N|N|N|2474464|2474736|2474100|2474373|N|N|N|N|N| +2474466|AAAAAAAACOBMFCAA|2062-10-03|1953|8493|652|2062|2|10|3|4|2062|652|8493|Tuesday|2062Q4|N|N|N|2474464|2474736|2474101|2474374|N|N|N|N|N| +2474467|AAAAAAAADOBMFCAA|2062-10-04|1953|8493|652|2062|3|10|4|4|2062|652|8493|Wednesday|2062Q4|N|N|N|2474464|2474736|2474102|2474375|N|N|N|N|N| +2474468|AAAAAAAAEOBMFCAA|2062-10-05|1953|8493|652|2062|4|10|5|4|2062|652|8493|Thursday|2062Q4|N|N|N|2474464|2474736|2474103|2474376|N|N|N|N|N| +2474469|AAAAAAAAFOBMFCAA|2062-10-06|1953|8493|652|2062|5|10|6|4|2062|652|8493|Friday|2062Q4|N|Y|N|2474464|2474736|2474104|2474377|N|N|N|N|N| +2474470|AAAAAAAAGOBMFCAA|2062-10-07|1953|8493|652|2062|6|10|7|4|2062|652|8493|Saturday|2062Q4|N|Y|N|2474464|2474736|2474105|2474378|N|N|N|N|N| +2474471|AAAAAAAAHOBMFCAA|2062-10-08|1953|8493|652|2062|0|10|8|4|2062|652|8493|Sunday|2062Q4|N|N|N|2474464|2474736|2474106|2474379|N|N|N|N|N| +2474472|AAAAAAAAIOBMFCAA|2062-10-09|1953|8493|652|2062|1|10|9|4|2062|652|8493|Monday|2062Q4|N|N|N|2474464|2474736|2474107|2474380|N|N|N|N|N| +2474473|AAAAAAAAJOBMFCAA|2062-10-10|1953|8494|652|2062|2|10|10|4|2062|652|8494|Tuesday|2062Q4|N|N|N|2474464|2474736|2474108|2474381|N|N|N|N|N| +2474474|AAAAAAAAKOBMFCAA|2062-10-11|1953|8494|652|2062|3|10|11|4|2062|652|8494|Wednesday|2062Q4|N|N|N|2474464|2474736|2474109|2474382|N|N|N|N|N| +2474475|AAAAAAAALOBMFCAA|2062-10-12|1953|8494|652|2062|4|10|12|4|2062|652|8494|Thursday|2062Q4|N|N|N|2474464|2474736|2474110|2474383|N|N|N|N|N| +2474476|AAAAAAAAMOBMFCAA|2062-10-13|1953|8494|652|2062|5|10|13|4|2062|652|8494|Friday|2062Q4|N|Y|N|2474464|2474736|2474111|2474384|N|N|N|N|N| +2474477|AAAAAAAANOBMFCAA|2062-10-14|1953|8494|652|2062|6|10|14|4|2062|652|8494|Saturday|2062Q4|N|Y|N|2474464|2474736|2474112|2474385|N|N|N|N|N| +2474478|AAAAAAAAOOBMFCAA|2062-10-15|1953|8494|652|2062|0|10|15|4|2062|652|8494|Sunday|2062Q4|N|N|N|2474464|2474736|2474113|2474386|N|N|N|N|N| +2474479|AAAAAAAAPOBMFCAA|2062-10-16|1953|8494|652|2062|1|10|16|4|2062|652|8494|Monday|2062Q4|N|N|N|2474464|2474736|2474114|2474387|N|N|N|N|N| +2474480|AAAAAAAAAPBMFCAA|2062-10-17|1953|8495|652|2062|2|10|17|4|2062|652|8495|Tuesday|2062Q4|N|N|N|2474464|2474736|2474115|2474388|N|N|N|N|N| +2474481|AAAAAAAABPBMFCAA|2062-10-18|1953|8495|652|2062|3|10|18|4|2062|652|8495|Wednesday|2062Q4|N|N|N|2474464|2474736|2474116|2474389|N|N|N|N|N| +2474482|AAAAAAAACPBMFCAA|2062-10-19|1953|8495|652|2062|4|10|19|4|2062|652|8495|Thursday|2062Q4|N|N|N|2474464|2474736|2474117|2474390|N|N|N|N|N| +2474483|AAAAAAAADPBMFCAA|2062-10-20|1953|8495|652|2062|5|10|20|4|2062|652|8495|Friday|2062Q4|N|Y|N|2474464|2474736|2474118|2474391|N|N|N|N|N| +2474484|AAAAAAAAEPBMFCAA|2062-10-21|1953|8495|652|2062|6|10|21|4|2062|652|8495|Saturday|2062Q4|N|Y|N|2474464|2474736|2474119|2474392|N|N|N|N|N| +2474485|AAAAAAAAFPBMFCAA|2062-10-22|1953|8495|652|2062|0|10|22|4|2062|652|8495|Sunday|2062Q4|N|N|N|2474464|2474736|2474120|2474393|N|N|N|N|N| +2474486|AAAAAAAAGPBMFCAA|2062-10-23|1953|8495|652|2062|1|10|23|4|2062|652|8495|Monday|2062Q4|N|N|N|2474464|2474736|2474121|2474394|N|N|N|N|N| +2474487|AAAAAAAAHPBMFCAA|2062-10-24|1953|8496|652|2062|2|10|24|4|2062|652|8496|Tuesday|2062Q4|N|N|N|2474464|2474736|2474122|2474395|N|N|N|N|N| +2474488|AAAAAAAAIPBMFCAA|2062-10-25|1953|8496|652|2062|3|10|25|4|2062|652|8496|Wednesday|2062Q4|N|N|N|2474464|2474736|2474123|2474396|N|N|N|N|N| +2474489|AAAAAAAAJPBMFCAA|2062-10-26|1953|8496|652|2062|4|10|26|4|2062|652|8496|Thursday|2062Q4|N|N|N|2474464|2474736|2474124|2474397|N|N|N|N|N| +2474490|AAAAAAAAKPBMFCAA|2062-10-27|1953|8496|652|2062|5|10|27|4|2062|652|8496|Friday|2062Q4|N|Y|N|2474464|2474736|2474125|2474398|N|N|N|N|N| +2474491|AAAAAAAALPBMFCAA|2062-10-28|1953|8496|652|2062|6|10|28|4|2062|652|8496|Saturday|2062Q4|N|Y|N|2474464|2474736|2474126|2474399|N|N|N|N|N| +2474492|AAAAAAAAMPBMFCAA|2062-10-29|1953|8496|652|2062|0|10|29|4|2062|652|8496|Sunday|2062Q4|N|N|N|2474464|2474736|2474127|2474400|N|N|N|N|N| +2474493|AAAAAAAANPBMFCAA|2062-10-30|1953|8496|652|2062|1|10|30|4|2062|652|8496|Monday|2062Q4|N|N|N|2474464|2474736|2474128|2474401|N|N|N|N|N| +2474494|AAAAAAAAOPBMFCAA|2062-10-31|1953|8497|652|2062|2|10|31|4|2062|652|8497|Tuesday|2062Q4|N|N|N|2474464|2474736|2474129|2474402|N|N|N|N|N| +2474495|AAAAAAAAPPBMFCAA|2062-11-01|1954|8497|652|2062|3|11|1|4|2062|652|8497|Wednesday|2062Q4|N|N|N|2474495|2474798|2474130|2474403|N|N|N|N|N| +2474496|AAAAAAAAAACMFCAA|2062-11-02|1954|8497|652|2062|4|11|2|4|2062|652|8497|Thursday|2062Q4|N|N|N|2474495|2474798|2474131|2474404|N|N|N|N|N| +2474497|AAAAAAAABACMFCAA|2062-11-03|1954|8497|652|2062|5|11|3|4|2062|652|8497|Friday|2062Q4|N|Y|N|2474495|2474798|2474132|2474405|N|N|N|N|N| +2474498|AAAAAAAACACMFCAA|2062-11-04|1954|8497|652|2062|6|11|4|4|2062|652|8497|Saturday|2062Q4|N|Y|N|2474495|2474798|2474133|2474406|N|N|N|N|N| +2474499|AAAAAAAADACMFCAA|2062-11-05|1954|8497|652|2062|0|11|5|4|2062|652|8497|Sunday|2062Q4|N|N|N|2474495|2474798|2474134|2474407|N|N|N|N|N| +2474500|AAAAAAAAEACMFCAA|2062-11-06|1954|8497|652|2062|1|11|6|4|2062|652|8497|Monday|2062Q4|N|N|N|2474495|2474798|2474135|2474408|N|N|N|N|N| +2474501|AAAAAAAAFACMFCAA|2062-11-07|1954|8498|652|2062|2|11|7|4|2062|652|8498|Tuesday|2062Q4|N|N|N|2474495|2474798|2474136|2474409|N|N|N|N|N| +2474502|AAAAAAAAGACMFCAA|2062-11-08|1954|8498|652|2062|3|11|8|4|2062|652|8498|Wednesday|2062Q4|N|N|N|2474495|2474798|2474137|2474410|N|N|N|N|N| +2474503|AAAAAAAAHACMFCAA|2062-11-09|1954|8498|652|2062|4|11|9|4|2062|652|8498|Thursday|2062Q4|N|N|N|2474495|2474798|2474138|2474411|N|N|N|N|N| +2474504|AAAAAAAAIACMFCAA|2062-11-10|1954|8498|652|2062|5|11|10|4|2062|652|8498|Friday|2062Q4|N|Y|N|2474495|2474798|2474139|2474412|N|N|N|N|N| +2474505|AAAAAAAAJACMFCAA|2062-11-11|1954|8498|652|2062|6|11|11|4|2062|652|8498|Saturday|2062Q4|N|Y|N|2474495|2474798|2474140|2474413|N|N|N|N|N| +2474506|AAAAAAAAKACMFCAA|2062-11-12|1954|8498|652|2062|0|11|12|4|2062|652|8498|Sunday|2062Q4|N|N|N|2474495|2474798|2474141|2474414|N|N|N|N|N| +2474507|AAAAAAAALACMFCAA|2062-11-13|1954|8498|652|2062|1|11|13|4|2062|652|8498|Monday|2062Q4|N|N|N|2474495|2474798|2474142|2474415|N|N|N|N|N| +2474508|AAAAAAAAMACMFCAA|2062-11-14|1954|8499|652|2062|2|11|14|4|2062|652|8499|Tuesday|2062Q4|N|N|N|2474495|2474798|2474143|2474416|N|N|N|N|N| +2474509|AAAAAAAANACMFCAA|2062-11-15|1954|8499|652|2062|3|11|15|4|2062|652|8499|Wednesday|2062Q4|N|N|N|2474495|2474798|2474144|2474417|N|N|N|N|N| +2474510|AAAAAAAAOACMFCAA|2062-11-16|1954|8499|652|2062|4|11|16|4|2062|652|8499|Thursday|2062Q4|N|N|N|2474495|2474798|2474145|2474418|N|N|N|N|N| +2474511|AAAAAAAAPACMFCAA|2062-11-17|1954|8499|652|2062|5|11|17|4|2062|652|8499|Friday|2062Q4|N|Y|N|2474495|2474798|2474146|2474419|N|N|N|N|N| +2474512|AAAAAAAAABCMFCAA|2062-11-18|1954|8499|652|2062|6|11|18|4|2062|652|8499|Saturday|2062Q4|N|Y|N|2474495|2474798|2474147|2474420|N|N|N|N|N| +2474513|AAAAAAAABBCMFCAA|2062-11-19|1954|8499|652|2062|0|11|19|4|2062|652|8499|Sunday|2062Q4|N|N|N|2474495|2474798|2474148|2474421|N|N|N|N|N| +2474514|AAAAAAAACBCMFCAA|2062-11-20|1954|8499|652|2062|1|11|20|4|2062|652|8499|Monday|2062Q4|N|N|N|2474495|2474798|2474149|2474422|N|N|N|N|N| +2474515|AAAAAAAADBCMFCAA|2062-11-21|1954|8500|652|2062|2|11|21|4|2062|652|8500|Tuesday|2062Q4|N|N|N|2474495|2474798|2474150|2474423|N|N|N|N|N| +2474516|AAAAAAAAEBCMFCAA|2062-11-22|1954|8500|652|2062|3|11|22|4|2062|652|8500|Wednesday|2062Q4|N|N|N|2474495|2474798|2474151|2474424|N|N|N|N|N| +2474517|AAAAAAAAFBCMFCAA|2062-11-23|1954|8500|652|2062|4|11|23|4|2062|652|8500|Thursday|2062Q4|N|N|N|2474495|2474798|2474152|2474425|N|N|N|N|N| +2474518|AAAAAAAAGBCMFCAA|2062-11-24|1954|8500|652|2062|5|11|24|4|2062|652|8500|Friday|2062Q4|N|Y|N|2474495|2474798|2474153|2474426|N|N|N|N|N| +2474519|AAAAAAAAHBCMFCAA|2062-11-25|1954|8500|652|2062|6|11|25|4|2062|652|8500|Saturday|2062Q4|N|Y|N|2474495|2474798|2474154|2474427|N|N|N|N|N| +2474520|AAAAAAAAIBCMFCAA|2062-11-26|1954|8500|652|2062|0|11|26|4|2062|652|8500|Sunday|2062Q4|N|N|N|2474495|2474798|2474155|2474428|N|N|N|N|N| +2474521|AAAAAAAAJBCMFCAA|2062-11-27|1954|8500|652|2062|1|11|27|4|2062|652|8500|Monday|2062Q4|N|N|N|2474495|2474798|2474156|2474429|N|N|N|N|N| +2474522|AAAAAAAAKBCMFCAA|2062-11-28|1954|8501|652|2062|2|11|28|4|2062|652|8501|Tuesday|2062Q4|N|N|N|2474495|2474798|2474157|2474430|N|N|N|N|N| +2474523|AAAAAAAALBCMFCAA|2062-11-29|1954|8501|652|2062|3|11|29|4|2062|652|8501|Wednesday|2062Q4|N|N|N|2474495|2474798|2474158|2474431|N|N|N|N|N| +2474524|AAAAAAAAMBCMFCAA|2062-11-30|1954|8501|652|2062|4|11|30|4|2062|652|8501|Thursday|2062Q4|N|N|N|2474495|2474798|2474159|2474432|N|N|N|N|N| +2474525|AAAAAAAANBCMFCAA|2062-12-01|1955|8501|653|2062|5|12|1|4|2062|653|8501|Friday|2062Q4|N|Y|N|2474525|2474858|2474160|2474433|N|N|N|N|N| +2474526|AAAAAAAAOBCMFCAA|2062-12-02|1955|8501|653|2062|6|12|2|4|2062|653|8501|Saturday|2062Q4|N|Y|N|2474525|2474858|2474161|2474434|N|N|N|N|N| +2474527|AAAAAAAAPBCMFCAA|2062-12-03|1955|8501|653|2062|0|12|3|4|2062|653|8501|Sunday|2062Q4|N|N|N|2474525|2474858|2474162|2474435|N|N|N|N|N| +2474528|AAAAAAAAACCMFCAA|2062-12-04|1955|8501|653|2062|1|12|4|4|2062|653|8501|Monday|2062Q4|N|N|N|2474525|2474858|2474163|2474436|N|N|N|N|N| +2474529|AAAAAAAABCCMFCAA|2062-12-05|1955|8502|653|2062|2|12|5|4|2062|653|8502|Tuesday|2062Q4|N|N|N|2474525|2474858|2474164|2474437|N|N|N|N|N| +2474530|AAAAAAAACCCMFCAA|2062-12-06|1955|8502|653|2062|3|12|6|4|2062|653|8502|Wednesday|2062Q4|N|N|N|2474525|2474858|2474165|2474438|N|N|N|N|N| +2474531|AAAAAAAADCCMFCAA|2062-12-07|1955|8502|653|2062|4|12|7|4|2062|653|8502|Thursday|2062Q4|N|N|N|2474525|2474858|2474166|2474439|N|N|N|N|N| +2474532|AAAAAAAAECCMFCAA|2062-12-08|1955|8502|653|2062|5|12|8|4|2062|653|8502|Friday|2062Q4|N|Y|N|2474525|2474858|2474167|2474440|N|N|N|N|N| +2474533|AAAAAAAAFCCMFCAA|2062-12-09|1955|8502|653|2062|6|12|9|4|2062|653|8502|Saturday|2062Q4|N|Y|N|2474525|2474858|2474168|2474441|N|N|N|N|N| +2474534|AAAAAAAAGCCMFCAA|2062-12-10|1955|8502|653|2062|0|12|10|4|2062|653|8502|Sunday|2062Q4|N|N|N|2474525|2474858|2474169|2474442|N|N|N|N|N| +2474535|AAAAAAAAHCCMFCAA|2062-12-11|1955|8502|653|2062|1|12|11|4|2062|653|8502|Monday|2062Q4|N|N|N|2474525|2474858|2474170|2474443|N|N|N|N|N| +2474536|AAAAAAAAICCMFCAA|2062-12-12|1955|8503|653|2062|2|12|12|4|2062|653|8503|Tuesday|2062Q4|N|N|N|2474525|2474858|2474171|2474444|N|N|N|N|N| +2474537|AAAAAAAAJCCMFCAA|2062-12-13|1955|8503|653|2062|3|12|13|4|2062|653|8503|Wednesday|2062Q4|N|N|N|2474525|2474858|2474172|2474445|N|N|N|N|N| +2474538|AAAAAAAAKCCMFCAA|2062-12-14|1955|8503|653|2062|4|12|14|4|2062|653|8503|Thursday|2062Q4|N|N|N|2474525|2474858|2474173|2474446|N|N|N|N|N| +2474539|AAAAAAAALCCMFCAA|2062-12-15|1955|8503|653|2062|5|12|15|4|2062|653|8503|Friday|2062Q4|N|Y|N|2474525|2474858|2474174|2474447|N|N|N|N|N| +2474540|AAAAAAAAMCCMFCAA|2062-12-16|1955|8503|653|2062|6|12|16|4|2062|653|8503|Saturday|2062Q4|N|Y|N|2474525|2474858|2474175|2474448|N|N|N|N|N| +2474541|AAAAAAAANCCMFCAA|2062-12-17|1955|8503|653|2062|0|12|17|4|2062|653|8503|Sunday|2062Q4|N|N|N|2474525|2474858|2474176|2474449|N|N|N|N|N| +2474542|AAAAAAAAOCCMFCAA|2062-12-18|1955|8503|653|2062|1|12|18|4|2062|653|8503|Monday|2062Q4|N|N|N|2474525|2474858|2474177|2474450|N|N|N|N|N| +2474543|AAAAAAAAPCCMFCAA|2062-12-19|1955|8504|653|2062|2|12|19|4|2062|653|8504|Tuesday|2062Q4|N|N|N|2474525|2474858|2474178|2474451|N|N|N|N|N| +2474544|AAAAAAAAADCMFCAA|2062-12-20|1955|8504|653|2062|3|12|20|4|2062|653|8504|Wednesday|2062Q4|N|N|N|2474525|2474858|2474179|2474452|N|N|N|N|N| +2474545|AAAAAAAABDCMFCAA|2062-12-21|1955|8504|653|2062|4|12|21|4|2062|653|8504|Thursday|2062Q4|N|N|N|2474525|2474858|2474180|2474453|N|N|N|N|N| +2474546|AAAAAAAACDCMFCAA|2062-12-22|1955|8504|653|2062|5|12|22|4|2062|653|8504|Friday|2062Q4|N|Y|N|2474525|2474858|2474181|2474454|N|N|N|N|N| +2474547|AAAAAAAADDCMFCAA|2062-12-23|1955|8504|653|2062|6|12|23|4|2062|653|8504|Saturday|2062Q4|N|Y|N|2474525|2474858|2474182|2474455|N|N|N|N|N| +2474548|AAAAAAAAEDCMFCAA|2062-12-24|1955|8504|653|2062|0|12|24|4|2062|653|8504|Sunday|2062Q4|N|N|N|2474525|2474858|2474183|2474456|N|N|N|N|N| +2474549|AAAAAAAAFDCMFCAA|2062-12-25|1955|8504|653|2062|1|12|25|4|2062|653|8504|Monday|2062Q4|N|N|N|2474525|2474858|2474184|2474457|N|N|N|N|N| +2474550|AAAAAAAAGDCMFCAA|2062-12-26|1955|8505|653|2062|2|12|26|4|2062|653|8505|Tuesday|2062Q4|Y|N|N|2474525|2474858|2474185|2474458|N|N|N|N|N| +2474551|AAAAAAAAHDCMFCAA|2062-12-27|1955|8505|653|2062|3|12|27|4|2062|653|8505|Wednesday|2062Q4|N|N|Y|2474525|2474858|2474186|2474459|N|N|N|N|N| +2474552|AAAAAAAAIDCMFCAA|2062-12-28|1955|8505|653|2062|4|12|28|4|2062|653|8505|Thursday|2062Q4|N|N|N|2474525|2474858|2474187|2474460|N|N|N|N|N| +2474553|AAAAAAAAJDCMFCAA|2062-12-29|1955|8505|653|2062|5|12|29|4|2062|653|8505|Friday|2062Q4|N|Y|N|2474525|2474858|2474188|2474461|N|N|N|N|N| +2474554|AAAAAAAAKDCMFCAA|2062-12-30|1955|8505|653|2062|6|12|30|4|2062|653|8505|Saturday|2062Q4|N|Y|N|2474525|2474858|2474189|2474462|N|N|N|N|N| +2474555|AAAAAAAALDCMFCAA|2062-12-31|1955|8505|653|2062|0|12|31|4|2062|653|8505|Sunday|2062Q4|N|N|N|2474525|2474858|2474190|2474463|N|N|N|N|N| +2474556|AAAAAAAAMDCMFCAA|2063-01-01|1956|8505|653|2063|1|1|1|1|2063|653|8505|Monday|2063Q1|Y|N|N|2474556|2474555|2474191|2474464|N|N|N|N|N| +2474557|AAAAAAAANDCMFCAA|2063-01-02|1956|8506|653|2063|2|1|2|1|2063|653|8506|Tuesday|2063Q1|N|N|Y|2474556|2474555|2474192|2474465|N|N|N|N|N| +2474558|AAAAAAAAODCMFCAA|2063-01-03|1956|8506|653|2063|3|1|3|1|2063|653|8506|Wednesday|2063Q1|N|N|N|2474556|2474555|2474193|2474466|N|N|N|N|N| +2474559|AAAAAAAAPDCMFCAA|2063-01-04|1956|8506|653|2063|4|1|4|1|2063|653|8506|Thursday|2063Q1|N|N|N|2474556|2474555|2474194|2474467|N|N|N|N|N| +2474560|AAAAAAAAAECMFCAA|2063-01-05|1956|8506|653|2063|5|1|5|1|2063|653|8506|Friday|2063Q1|N|Y|N|2474556|2474555|2474195|2474468|N|N|N|N|N| +2474561|AAAAAAAABECMFCAA|2063-01-06|1956|8506|653|2063|6|1|6|1|2063|653|8506|Saturday|2063Q1|N|Y|N|2474556|2474555|2474196|2474469|N|N|N|N|N| +2474562|AAAAAAAACECMFCAA|2063-01-07|1956|8506|653|2063|0|1|7|1|2063|653|8506|Sunday|2063Q1|N|N|N|2474556|2474555|2474197|2474470|N|N|N|N|N| +2474563|AAAAAAAADECMFCAA|2063-01-08|1956|8506|653|2063|1|1|8|1|2063|653|8506|Monday|2063Q1|N|N|N|2474556|2474555|2474198|2474471|N|N|N|N|N| +2474564|AAAAAAAAEECMFCAA|2063-01-09|1956|8507|653|2063|2|1|9|1|2063|653|8507|Tuesday|2063Q1|N|N|N|2474556|2474555|2474199|2474472|N|N|N|N|N| +2474565|AAAAAAAAFECMFCAA|2063-01-10|1956|8507|653|2063|3|1|10|1|2063|653|8507|Wednesday|2063Q1|N|N|N|2474556|2474555|2474200|2474473|N|N|N|N|N| +2474566|AAAAAAAAGECMFCAA|2063-01-11|1956|8507|653|2063|4|1|11|1|2063|653|8507|Thursday|2063Q1|N|N|N|2474556|2474555|2474201|2474474|N|N|N|N|N| +2474567|AAAAAAAAHECMFCAA|2063-01-12|1956|8507|653|2063|5|1|12|1|2063|653|8507|Friday|2063Q1|N|Y|N|2474556|2474555|2474202|2474475|N|N|N|N|N| +2474568|AAAAAAAAIECMFCAA|2063-01-13|1956|8507|653|2063|6|1|13|1|2063|653|8507|Saturday|2063Q1|N|Y|N|2474556|2474555|2474203|2474476|N|N|N|N|N| +2474569|AAAAAAAAJECMFCAA|2063-01-14|1956|8507|653|2063|0|1|14|1|2063|653|8507|Sunday|2063Q1|N|N|N|2474556|2474555|2474204|2474477|N|N|N|N|N| +2474570|AAAAAAAAKECMFCAA|2063-01-15|1956|8507|653|2063|1|1|15|1|2063|653|8507|Monday|2063Q1|N|N|N|2474556|2474555|2474205|2474478|N|N|N|N|N| +2474571|AAAAAAAALECMFCAA|2063-01-16|1956|8508|653|2063|2|1|16|1|2063|653|8508|Tuesday|2063Q1|N|N|N|2474556|2474555|2474206|2474479|N|N|N|N|N| +2474572|AAAAAAAAMECMFCAA|2063-01-17|1956|8508|653|2063|3|1|17|1|2063|653|8508|Wednesday|2063Q1|N|N|N|2474556|2474555|2474207|2474480|N|N|N|N|N| +2474573|AAAAAAAANECMFCAA|2063-01-18|1956|8508|653|2063|4|1|18|1|2063|653|8508|Thursday|2063Q1|N|N|N|2474556|2474555|2474208|2474481|N|N|N|N|N| +2474574|AAAAAAAAOECMFCAA|2063-01-19|1956|8508|653|2063|5|1|19|1|2063|653|8508|Friday|2063Q1|N|Y|N|2474556|2474555|2474209|2474482|N|N|N|N|N| +2474575|AAAAAAAAPECMFCAA|2063-01-20|1956|8508|653|2063|6|1|20|1|2063|653|8508|Saturday|2063Q1|N|Y|N|2474556|2474555|2474210|2474483|N|N|N|N|N| +2474576|AAAAAAAAAFCMFCAA|2063-01-21|1956|8508|653|2063|0|1|21|1|2063|653|8508|Sunday|2063Q1|N|N|N|2474556|2474555|2474211|2474484|N|N|N|N|N| +2474577|AAAAAAAABFCMFCAA|2063-01-22|1956|8508|653|2063|1|1|22|1|2063|653|8508|Monday|2063Q1|N|N|N|2474556|2474555|2474212|2474485|N|N|N|N|N| +2474578|AAAAAAAACFCMFCAA|2063-01-23|1956|8509|653|2063|2|1|23|1|2063|653|8509|Tuesday|2063Q1|N|N|N|2474556|2474555|2474213|2474486|N|N|N|N|N| +2474579|AAAAAAAADFCMFCAA|2063-01-24|1956|8509|653|2063|3|1|24|1|2063|653|8509|Wednesday|2063Q1|N|N|N|2474556|2474555|2474214|2474487|N|N|N|N|N| +2474580|AAAAAAAAEFCMFCAA|2063-01-25|1956|8509|653|2063|4|1|25|1|2063|653|8509|Thursday|2063Q1|N|N|N|2474556|2474555|2474215|2474488|N|N|N|N|N| +2474581|AAAAAAAAFFCMFCAA|2063-01-26|1956|8509|653|2063|5|1|26|1|2063|653|8509|Friday|2063Q1|N|Y|N|2474556|2474555|2474216|2474489|N|N|N|N|N| +2474582|AAAAAAAAGFCMFCAA|2063-01-27|1956|8509|653|2063|6|1|27|1|2063|653|8509|Saturday|2063Q1|N|Y|N|2474556|2474555|2474217|2474490|N|N|N|N|N| +2474583|AAAAAAAAHFCMFCAA|2063-01-28|1956|8509|653|2063|0|1|28|1|2063|653|8509|Sunday|2063Q1|N|N|N|2474556|2474555|2474218|2474491|N|N|N|N|N| +2474584|AAAAAAAAIFCMFCAA|2063-01-29|1956|8509|653|2063|1|1|29|1|2063|653|8509|Monday|2063Q1|N|N|N|2474556|2474555|2474219|2474492|N|N|N|N|N| +2474585|AAAAAAAAJFCMFCAA|2063-01-30|1956|8510|653|2063|2|1|30|1|2063|653|8510|Tuesday|2063Q1|N|N|N|2474556|2474555|2474220|2474493|N|N|N|N|N| +2474586|AAAAAAAAKFCMFCAA|2063-01-31|1956|8510|653|2063|3|1|31|1|2063|653|8510|Wednesday|2063Q1|N|N|N|2474556|2474555|2474221|2474494|N|N|N|N|N| +2474587|AAAAAAAALFCMFCAA|2063-02-01|1957|8510|653|2063|4|2|1|1|2063|653|8510|Thursday|2063Q1|N|N|N|2474587|2474617|2474222|2474495|N|N|N|N|N| +2474588|AAAAAAAAMFCMFCAA|2063-02-02|1957|8510|653|2063|5|2|2|1|2063|653|8510|Friday|2063Q1|N|Y|N|2474587|2474617|2474223|2474496|N|N|N|N|N| +2474589|AAAAAAAANFCMFCAA|2063-02-03|1957|8510|653|2063|6|2|3|1|2063|653|8510|Saturday|2063Q1|N|Y|N|2474587|2474617|2474224|2474497|N|N|N|N|N| +2474590|AAAAAAAAOFCMFCAA|2063-02-04|1957|8510|653|2063|0|2|4|1|2063|653|8510|Sunday|2063Q1|N|N|N|2474587|2474617|2474225|2474498|N|N|N|N|N| +2474591|AAAAAAAAPFCMFCAA|2063-02-05|1957|8510|653|2063|1|2|5|1|2063|653|8510|Monday|2063Q1|N|N|N|2474587|2474617|2474226|2474499|N|N|N|N|N| +2474592|AAAAAAAAAGCMFCAA|2063-02-06|1957|8511|653|2063|2|2|6|1|2063|653|8511|Tuesday|2063Q1|N|N|N|2474587|2474617|2474227|2474500|N|N|N|N|N| +2474593|AAAAAAAABGCMFCAA|2063-02-07|1957|8511|653|2063|3|2|7|1|2063|653|8511|Wednesday|2063Q1|N|N|N|2474587|2474617|2474228|2474501|N|N|N|N|N| +2474594|AAAAAAAACGCMFCAA|2063-02-08|1957|8511|653|2063|4|2|8|1|2063|653|8511|Thursday|2063Q1|N|N|N|2474587|2474617|2474229|2474502|N|N|N|N|N| +2474595|AAAAAAAADGCMFCAA|2063-02-09|1957|8511|653|2063|5|2|9|1|2063|653|8511|Friday|2063Q1|N|Y|N|2474587|2474617|2474230|2474503|N|N|N|N|N| +2474596|AAAAAAAAEGCMFCAA|2063-02-10|1957|8511|653|2063|6|2|10|1|2063|653|8511|Saturday|2063Q1|N|Y|N|2474587|2474617|2474231|2474504|N|N|N|N|N| +2474597|AAAAAAAAFGCMFCAA|2063-02-11|1957|8511|653|2063|0|2|11|1|2063|653|8511|Sunday|2063Q1|N|N|N|2474587|2474617|2474232|2474505|N|N|N|N|N| +2474598|AAAAAAAAGGCMFCAA|2063-02-12|1957|8511|653|2063|1|2|12|1|2063|653|8511|Monday|2063Q1|N|N|N|2474587|2474617|2474233|2474506|N|N|N|N|N| +2474599|AAAAAAAAHGCMFCAA|2063-02-13|1957|8512|653|2063|2|2|13|1|2063|653|8512|Tuesday|2063Q1|N|N|N|2474587|2474617|2474234|2474507|N|N|N|N|N| +2474600|AAAAAAAAIGCMFCAA|2063-02-14|1957|8512|653|2063|3|2|14|1|2063|653|8512|Wednesday|2063Q1|N|N|N|2474587|2474617|2474235|2474508|N|N|N|N|N| +2474601|AAAAAAAAJGCMFCAA|2063-02-15|1957|8512|653|2063|4|2|15|1|2063|653|8512|Thursday|2063Q1|N|N|N|2474587|2474617|2474236|2474509|N|N|N|N|N| +2474602|AAAAAAAAKGCMFCAA|2063-02-16|1957|8512|653|2063|5|2|16|1|2063|653|8512|Friday|2063Q1|N|Y|N|2474587|2474617|2474237|2474510|N|N|N|N|N| +2474603|AAAAAAAALGCMFCAA|2063-02-17|1957|8512|653|2063|6|2|17|1|2063|653|8512|Saturday|2063Q1|N|Y|N|2474587|2474617|2474238|2474511|N|N|N|N|N| +2474604|AAAAAAAAMGCMFCAA|2063-02-18|1957|8512|653|2063|0|2|18|1|2063|653|8512|Sunday|2063Q1|N|N|N|2474587|2474617|2474239|2474512|N|N|N|N|N| +2474605|AAAAAAAANGCMFCAA|2063-02-19|1957|8512|653|2063|1|2|19|1|2063|653|8512|Monday|2063Q1|N|N|N|2474587|2474617|2474240|2474513|N|N|N|N|N| +2474606|AAAAAAAAOGCMFCAA|2063-02-20|1957|8513|653|2063|2|2|20|1|2063|653|8513|Tuesday|2063Q1|N|N|N|2474587|2474617|2474241|2474514|N|N|N|N|N| +2474607|AAAAAAAAPGCMFCAA|2063-02-21|1957|8513|653|2063|3|2|21|1|2063|653|8513|Wednesday|2063Q1|N|N|N|2474587|2474617|2474242|2474515|N|N|N|N|N| +2474608|AAAAAAAAAHCMFCAA|2063-02-22|1957|8513|653|2063|4|2|22|1|2063|653|8513|Thursday|2063Q1|N|N|N|2474587|2474617|2474243|2474516|N|N|N|N|N| +2474609|AAAAAAAABHCMFCAA|2063-02-23|1957|8513|653|2063|5|2|23|1|2063|653|8513|Friday|2063Q1|N|Y|N|2474587|2474617|2474244|2474517|N|N|N|N|N| +2474610|AAAAAAAACHCMFCAA|2063-02-24|1957|8513|653|2063|6|2|24|1|2063|653|8513|Saturday|2063Q1|N|Y|N|2474587|2474617|2474245|2474518|N|N|N|N|N| +2474611|AAAAAAAADHCMFCAA|2063-02-25|1957|8513|653|2063|0|2|25|1|2063|653|8513|Sunday|2063Q1|N|N|N|2474587|2474617|2474246|2474519|N|N|N|N|N| +2474612|AAAAAAAAEHCMFCAA|2063-02-26|1957|8513|653|2063|1|2|26|1|2063|653|8513|Monday|2063Q1|N|N|N|2474587|2474617|2474247|2474520|N|N|N|N|N| +2474613|AAAAAAAAFHCMFCAA|2063-02-27|1957|8514|653|2063|2|2|27|1|2063|653|8514|Tuesday|2063Q1|N|N|N|2474587|2474617|2474248|2474521|N|N|N|N|N| +2474614|AAAAAAAAGHCMFCAA|2063-02-28|1957|8514|653|2063|3|2|28|1|2063|653|8514|Wednesday|2063Q1|N|N|N|2474587|2474617|2474249|2474522|N|N|N|N|N| +2474615|AAAAAAAAHHCMFCAA|2063-03-01|1958|8514|654|2063|4|3|1|1|2063|654|8514|Thursday|2063Q1|N|N|N|2474615|2474673|2474250|2474523|N|N|N|N|N| +2474616|AAAAAAAAIHCMFCAA|2063-03-02|1958|8514|654|2063|5|3|2|1|2063|654|8514|Friday|2063Q1|N|Y|N|2474615|2474673|2474251|2474524|N|N|N|N|N| +2474617|AAAAAAAAJHCMFCAA|2063-03-03|1958|8514|654|2063|6|3|3|1|2063|654|8514|Saturday|2063Q1|N|Y|N|2474615|2474673|2474252|2474525|N|N|N|N|N| +2474618|AAAAAAAAKHCMFCAA|2063-03-04|1958|8514|654|2063|0|3|4|1|2063|654|8514|Sunday|2063Q1|N|N|N|2474615|2474673|2474253|2474526|N|N|N|N|N| +2474619|AAAAAAAALHCMFCAA|2063-03-05|1958|8514|654|2063|1|3|5|1|2063|654|8514|Monday|2063Q1|N|N|N|2474615|2474673|2474254|2474527|N|N|N|N|N| +2474620|AAAAAAAAMHCMFCAA|2063-03-06|1958|8515|654|2063|2|3|6|1|2063|654|8515|Tuesday|2063Q1|N|N|N|2474615|2474673|2474255|2474528|N|N|N|N|N| +2474621|AAAAAAAANHCMFCAA|2063-03-07|1958|8515|654|2063|3|3|7|1|2063|654|8515|Wednesday|2063Q1|N|N|N|2474615|2474673|2474256|2474529|N|N|N|N|N| +2474622|AAAAAAAAOHCMFCAA|2063-03-08|1958|8515|654|2063|4|3|8|1|2063|654|8515|Thursday|2063Q1|N|N|N|2474615|2474673|2474257|2474530|N|N|N|N|N| +2474623|AAAAAAAAPHCMFCAA|2063-03-09|1958|8515|654|2063|5|3|9|1|2063|654|8515|Friday|2063Q1|N|Y|N|2474615|2474673|2474258|2474531|N|N|N|N|N| +2474624|AAAAAAAAAICMFCAA|2063-03-10|1958|8515|654|2063|6|3|10|1|2063|654|8515|Saturday|2063Q1|N|Y|N|2474615|2474673|2474259|2474532|N|N|N|N|N| +2474625|AAAAAAAABICMFCAA|2063-03-11|1958|8515|654|2063|0|3|11|1|2063|654|8515|Sunday|2063Q1|N|N|N|2474615|2474673|2474260|2474533|N|N|N|N|N| +2474626|AAAAAAAACICMFCAA|2063-03-12|1958|8515|654|2063|1|3|12|1|2063|654|8515|Monday|2063Q1|N|N|N|2474615|2474673|2474261|2474534|N|N|N|N|N| +2474627|AAAAAAAADICMFCAA|2063-03-13|1958|8516|654|2063|2|3|13|1|2063|654|8516|Tuesday|2063Q1|N|N|N|2474615|2474673|2474262|2474535|N|N|N|N|N| +2474628|AAAAAAAAEICMFCAA|2063-03-14|1958|8516|654|2063|3|3|14|1|2063|654|8516|Wednesday|2063Q1|N|N|N|2474615|2474673|2474263|2474536|N|N|N|N|N| +2474629|AAAAAAAAFICMFCAA|2063-03-15|1958|8516|654|2063|4|3|15|1|2063|654|8516|Thursday|2063Q1|N|N|N|2474615|2474673|2474264|2474537|N|N|N|N|N| +2474630|AAAAAAAAGICMFCAA|2063-03-16|1958|8516|654|2063|5|3|16|1|2063|654|8516|Friday|2063Q1|N|Y|N|2474615|2474673|2474265|2474538|N|N|N|N|N| +2474631|AAAAAAAAHICMFCAA|2063-03-17|1958|8516|654|2063|6|3|17|1|2063|654|8516|Saturday|2063Q1|N|Y|N|2474615|2474673|2474266|2474539|N|N|N|N|N| +2474632|AAAAAAAAIICMFCAA|2063-03-18|1958|8516|654|2063|0|3|18|1|2063|654|8516|Sunday|2063Q1|N|N|N|2474615|2474673|2474267|2474540|N|N|N|N|N| +2474633|AAAAAAAAJICMFCAA|2063-03-19|1958|8516|654|2063|1|3|19|1|2063|654|8516|Monday|2063Q1|N|N|N|2474615|2474673|2474268|2474541|N|N|N|N|N| +2474634|AAAAAAAAKICMFCAA|2063-03-20|1958|8517|654|2063|2|3|20|1|2063|654|8517|Tuesday|2063Q1|N|N|N|2474615|2474673|2474269|2474542|N|N|N|N|N| +2474635|AAAAAAAALICMFCAA|2063-03-21|1958|8517|654|2063|3|3|21|1|2063|654|8517|Wednesday|2063Q1|N|N|N|2474615|2474673|2474270|2474543|N|N|N|N|N| +2474636|AAAAAAAAMICMFCAA|2063-03-22|1958|8517|654|2063|4|3|22|1|2063|654|8517|Thursday|2063Q1|N|N|N|2474615|2474673|2474271|2474544|N|N|N|N|N| +2474637|AAAAAAAANICMFCAA|2063-03-23|1958|8517|654|2063|5|3|23|1|2063|654|8517|Friday|2063Q1|N|Y|N|2474615|2474673|2474272|2474545|N|N|N|N|N| +2474638|AAAAAAAAOICMFCAA|2063-03-24|1958|8517|654|2063|6|3|24|1|2063|654|8517|Saturday|2063Q1|N|Y|N|2474615|2474673|2474273|2474546|N|N|N|N|N| +2474639|AAAAAAAAPICMFCAA|2063-03-25|1958|8517|654|2063|0|3|25|1|2063|654|8517|Sunday|2063Q1|N|N|N|2474615|2474673|2474274|2474547|N|N|N|N|N| +2474640|AAAAAAAAAJCMFCAA|2063-03-26|1958|8517|654|2063|1|3|26|1|2063|654|8517|Monday|2063Q1|N|N|N|2474615|2474673|2474275|2474548|N|N|N|N|N| +2474641|AAAAAAAABJCMFCAA|2063-03-27|1958|8518|654|2063|2|3|27|1|2063|654|8518|Tuesday|2063Q1|N|N|N|2474615|2474673|2474276|2474549|N|N|N|N|N| +2474642|AAAAAAAACJCMFCAA|2063-03-28|1958|8518|654|2063|3|3|28|1|2063|654|8518|Wednesday|2063Q1|N|N|N|2474615|2474673|2474277|2474550|N|N|N|N|N| +2474643|AAAAAAAADJCMFCAA|2063-03-29|1958|8518|654|2063|4|3|29|1|2063|654|8518|Thursday|2063Q1|N|N|N|2474615|2474673|2474278|2474551|N|N|N|N|N| +2474644|AAAAAAAAEJCMFCAA|2063-03-30|1958|8518|654|2063|5|3|30|1|2063|654|8518|Friday|2063Q1|N|Y|N|2474615|2474673|2474279|2474552|N|N|N|N|N| +2474645|AAAAAAAAFJCMFCAA|2063-03-31|1958|8518|654|2063|6|3|31|1|2063|654|8518|Saturday|2063Q1|N|Y|N|2474615|2474673|2474280|2474553|N|N|N|N|N| +2474646|AAAAAAAAGJCMFCAA|2063-04-01|1959|8518|654|2063|0|4|1|1|2063|654|8518|Sunday|2063Q1|N|N|N|2474646|2474735|2474281|2474556|N|N|N|N|N| +2474647|AAAAAAAAHJCMFCAA|2063-04-02|1959|8518|654|2063|1|4|2|2|2063|654|8518|Monday|2063Q2|N|N|N|2474646|2474735|2474282|2474557|N|N|N|N|N| +2474648|AAAAAAAAIJCMFCAA|2063-04-03|1959|8519|654|2063|2|4|3|2|2063|654|8519|Tuesday|2063Q2|N|N|N|2474646|2474735|2474283|2474558|N|N|N|N|N| +2474649|AAAAAAAAJJCMFCAA|2063-04-04|1959|8519|654|2063|3|4|4|2|2063|654|8519|Wednesday|2063Q2|N|N|N|2474646|2474735|2474284|2474559|N|N|N|N|N| +2474650|AAAAAAAAKJCMFCAA|2063-04-05|1959|8519|654|2063|4|4|5|2|2063|654|8519|Thursday|2063Q2|N|N|N|2474646|2474735|2474285|2474560|N|N|N|N|N| +2474651|AAAAAAAALJCMFCAA|2063-04-06|1959|8519|654|2063|5|4|6|2|2063|654|8519|Friday|2063Q2|N|Y|N|2474646|2474735|2474286|2474561|N|N|N|N|N| +2474652|AAAAAAAAMJCMFCAA|2063-04-07|1959|8519|654|2063|6|4|7|2|2063|654|8519|Saturday|2063Q2|N|Y|N|2474646|2474735|2474287|2474562|N|N|N|N|N| +2474653|AAAAAAAANJCMFCAA|2063-04-08|1959|8519|654|2063|0|4|8|2|2063|654|8519|Sunday|2063Q2|N|N|N|2474646|2474735|2474288|2474563|N|N|N|N|N| +2474654|AAAAAAAAOJCMFCAA|2063-04-09|1959|8519|654|2063|1|4|9|2|2063|654|8519|Monday|2063Q2|N|N|N|2474646|2474735|2474289|2474564|N|N|N|N|N| +2474655|AAAAAAAAPJCMFCAA|2063-04-10|1959|8520|654|2063|2|4|10|2|2063|654|8520|Tuesday|2063Q2|N|N|N|2474646|2474735|2474290|2474565|N|N|N|N|N| +2474656|AAAAAAAAAKCMFCAA|2063-04-11|1959|8520|654|2063|3|4|11|2|2063|654|8520|Wednesday|2063Q2|N|N|N|2474646|2474735|2474291|2474566|N|N|N|N|N| +2474657|AAAAAAAABKCMFCAA|2063-04-12|1959|8520|654|2063|4|4|12|2|2063|654|8520|Thursday|2063Q2|N|N|N|2474646|2474735|2474292|2474567|N|N|N|N|N| +2474658|AAAAAAAACKCMFCAA|2063-04-13|1959|8520|654|2063|5|4|13|2|2063|654|8520|Friday|2063Q2|N|Y|N|2474646|2474735|2474293|2474568|N|N|N|N|N| +2474659|AAAAAAAADKCMFCAA|2063-04-14|1959|8520|654|2063|6|4|14|2|2063|654|8520|Saturday|2063Q2|N|Y|N|2474646|2474735|2474294|2474569|N|N|N|N|N| +2474660|AAAAAAAAEKCMFCAA|2063-04-15|1959|8520|654|2063|0|4|15|2|2063|654|8520|Sunday|2063Q2|N|N|N|2474646|2474735|2474295|2474570|N|N|N|N|N| +2474661|AAAAAAAAFKCMFCAA|2063-04-16|1959|8520|654|2063|1|4|16|2|2063|654|8520|Monday|2063Q2|N|N|N|2474646|2474735|2474296|2474571|N|N|N|N|N| +2474662|AAAAAAAAGKCMFCAA|2063-04-17|1959|8521|654|2063|2|4|17|2|2063|654|8521|Tuesday|2063Q2|N|N|N|2474646|2474735|2474297|2474572|N|N|N|N|N| +2474663|AAAAAAAAHKCMFCAA|2063-04-18|1959|8521|654|2063|3|4|18|2|2063|654|8521|Wednesday|2063Q2|N|N|N|2474646|2474735|2474298|2474573|N|N|N|N|N| +2474664|AAAAAAAAIKCMFCAA|2063-04-19|1959|8521|654|2063|4|4|19|2|2063|654|8521|Thursday|2063Q2|N|N|N|2474646|2474735|2474299|2474574|N|N|N|N|N| +2474665|AAAAAAAAJKCMFCAA|2063-04-20|1959|8521|654|2063|5|4|20|2|2063|654|8521|Friday|2063Q2|N|Y|N|2474646|2474735|2474300|2474575|N|N|N|N|N| +2474666|AAAAAAAAKKCMFCAA|2063-04-21|1959|8521|654|2063|6|4|21|2|2063|654|8521|Saturday|2063Q2|N|Y|N|2474646|2474735|2474301|2474576|N|N|N|N|N| +2474667|AAAAAAAALKCMFCAA|2063-04-22|1959|8521|654|2063|0|4|22|2|2063|654|8521|Sunday|2063Q2|N|N|N|2474646|2474735|2474302|2474577|N|N|N|N|N| +2474668|AAAAAAAAMKCMFCAA|2063-04-23|1959|8521|654|2063|1|4|23|2|2063|654|8521|Monday|2063Q2|N|N|N|2474646|2474735|2474303|2474578|N|N|N|N|N| +2474669|AAAAAAAANKCMFCAA|2063-04-24|1959|8522|654|2063|2|4|24|2|2063|654|8522|Tuesday|2063Q2|N|N|N|2474646|2474735|2474304|2474579|N|N|N|N|N| +2474670|AAAAAAAAOKCMFCAA|2063-04-25|1959|8522|654|2063|3|4|25|2|2063|654|8522|Wednesday|2063Q2|N|N|N|2474646|2474735|2474305|2474580|N|N|N|N|N| +2474671|AAAAAAAAPKCMFCAA|2063-04-26|1959|8522|654|2063|4|4|26|2|2063|654|8522|Thursday|2063Q2|N|N|N|2474646|2474735|2474306|2474581|N|N|N|N|N| +2474672|AAAAAAAAALCMFCAA|2063-04-27|1959|8522|654|2063|5|4|27|2|2063|654|8522|Friday|2063Q2|N|Y|N|2474646|2474735|2474307|2474582|N|N|N|N|N| +2474673|AAAAAAAABLCMFCAA|2063-04-28|1959|8522|654|2063|6|4|28|2|2063|654|8522|Saturday|2063Q2|N|Y|N|2474646|2474735|2474308|2474583|N|N|N|N|N| +2474674|AAAAAAAACLCMFCAA|2063-04-29|1959|8522|654|2063|0|4|29|2|2063|654|8522|Sunday|2063Q2|N|N|N|2474646|2474735|2474309|2474584|N|N|N|N|N| +2474675|AAAAAAAADLCMFCAA|2063-04-30|1959|8522|654|2063|1|4|30|2|2063|654|8522|Monday|2063Q2|N|N|N|2474646|2474735|2474310|2474585|N|N|N|N|N| +2474676|AAAAAAAAELCMFCAA|2063-05-01|1960|8523|654|2063|2|5|1|2|2063|654|8523|Tuesday|2063Q2|N|N|N|2474676|2474795|2474311|2474586|N|N|N|N|N| +2474677|AAAAAAAAFLCMFCAA|2063-05-02|1960|8523|654|2063|3|5|2|2|2063|654|8523|Wednesday|2063Q2|N|N|N|2474676|2474795|2474312|2474587|N|N|N|N|N| +2474678|AAAAAAAAGLCMFCAA|2063-05-03|1960|8523|654|2063|4|5|3|2|2063|654|8523|Thursday|2063Q2|N|N|N|2474676|2474795|2474313|2474588|N|N|N|N|N| +2474679|AAAAAAAAHLCMFCAA|2063-05-04|1960|8523|654|2063|5|5|4|2|2063|654|8523|Friday|2063Q2|N|Y|N|2474676|2474795|2474314|2474589|N|N|N|N|N| +2474680|AAAAAAAAILCMFCAA|2063-05-05|1960|8523|654|2063|6|5|5|2|2063|654|8523|Saturday|2063Q2|N|Y|N|2474676|2474795|2474315|2474590|N|N|N|N|N| +2474681|AAAAAAAAJLCMFCAA|2063-05-06|1960|8523|654|2063|0|5|6|2|2063|654|8523|Sunday|2063Q2|N|N|N|2474676|2474795|2474316|2474591|N|N|N|N|N| +2474682|AAAAAAAAKLCMFCAA|2063-05-07|1960|8523|654|2063|1|5|7|2|2063|654|8523|Monday|2063Q2|N|N|N|2474676|2474795|2474317|2474592|N|N|N|N|N| +2474683|AAAAAAAALLCMFCAA|2063-05-08|1960|8524|654|2063|2|5|8|2|2063|654|8524|Tuesday|2063Q2|N|N|N|2474676|2474795|2474318|2474593|N|N|N|N|N| +2474684|AAAAAAAAMLCMFCAA|2063-05-09|1960|8524|654|2063|3|5|9|2|2063|654|8524|Wednesday|2063Q2|N|N|N|2474676|2474795|2474319|2474594|N|N|N|N|N| +2474685|AAAAAAAANLCMFCAA|2063-05-10|1960|8524|654|2063|4|5|10|2|2063|654|8524|Thursday|2063Q2|N|N|N|2474676|2474795|2474320|2474595|N|N|N|N|N| +2474686|AAAAAAAAOLCMFCAA|2063-05-11|1960|8524|654|2063|5|5|11|2|2063|654|8524|Friday|2063Q2|N|Y|N|2474676|2474795|2474321|2474596|N|N|N|N|N| +2474687|AAAAAAAAPLCMFCAA|2063-05-12|1960|8524|654|2063|6|5|12|2|2063|654|8524|Saturday|2063Q2|N|Y|N|2474676|2474795|2474322|2474597|N|N|N|N|N| +2474688|AAAAAAAAAMCMFCAA|2063-05-13|1960|8524|654|2063|0|5|13|2|2063|654|8524|Sunday|2063Q2|N|N|N|2474676|2474795|2474323|2474598|N|N|N|N|N| +2474689|AAAAAAAABMCMFCAA|2063-05-14|1960|8524|654|2063|1|5|14|2|2063|654|8524|Monday|2063Q2|N|N|N|2474676|2474795|2474324|2474599|N|N|N|N|N| +2474690|AAAAAAAACMCMFCAA|2063-05-15|1960|8525|654|2063|2|5|15|2|2063|654|8525|Tuesday|2063Q2|N|N|N|2474676|2474795|2474325|2474600|N|N|N|N|N| +2474691|AAAAAAAADMCMFCAA|2063-05-16|1960|8525|654|2063|3|5|16|2|2063|654|8525|Wednesday|2063Q2|N|N|N|2474676|2474795|2474326|2474601|N|N|N|N|N| +2474692|AAAAAAAAEMCMFCAA|2063-05-17|1960|8525|654|2063|4|5|17|2|2063|654|8525|Thursday|2063Q2|N|N|N|2474676|2474795|2474327|2474602|N|N|N|N|N| +2474693|AAAAAAAAFMCMFCAA|2063-05-18|1960|8525|654|2063|5|5|18|2|2063|654|8525|Friday|2063Q2|N|Y|N|2474676|2474795|2474328|2474603|N|N|N|N|N| +2474694|AAAAAAAAGMCMFCAA|2063-05-19|1960|8525|654|2063|6|5|19|2|2063|654|8525|Saturday|2063Q2|N|Y|N|2474676|2474795|2474329|2474604|N|N|N|N|N| +2474695|AAAAAAAAHMCMFCAA|2063-05-20|1960|8525|654|2063|0|5|20|2|2063|654|8525|Sunday|2063Q2|N|N|N|2474676|2474795|2474330|2474605|N|N|N|N|N| +2474696|AAAAAAAAIMCMFCAA|2063-05-21|1960|8525|654|2063|1|5|21|2|2063|654|8525|Monday|2063Q2|N|N|N|2474676|2474795|2474331|2474606|N|N|N|N|N| +2474697|AAAAAAAAJMCMFCAA|2063-05-22|1960|8526|654|2063|2|5|22|2|2063|654|8526|Tuesday|2063Q2|N|N|N|2474676|2474795|2474332|2474607|N|N|N|N|N| +2474698|AAAAAAAAKMCMFCAA|2063-05-23|1960|8526|654|2063|3|5|23|2|2063|654|8526|Wednesday|2063Q2|N|N|N|2474676|2474795|2474333|2474608|N|N|N|N|N| +2474699|AAAAAAAALMCMFCAA|2063-05-24|1960|8526|654|2063|4|5|24|2|2063|654|8526|Thursday|2063Q2|N|N|N|2474676|2474795|2474334|2474609|N|N|N|N|N| +2474700|AAAAAAAAMMCMFCAA|2063-05-25|1960|8526|654|2063|5|5|25|2|2063|654|8526|Friday|2063Q2|N|Y|N|2474676|2474795|2474335|2474610|N|N|N|N|N| +2474701|AAAAAAAANMCMFCAA|2063-05-26|1960|8526|654|2063|6|5|26|2|2063|654|8526|Saturday|2063Q2|N|Y|N|2474676|2474795|2474336|2474611|N|N|N|N|N| +2474702|AAAAAAAAOMCMFCAA|2063-05-27|1960|8526|654|2063|0|5|27|2|2063|654|8526|Sunday|2063Q2|N|N|N|2474676|2474795|2474337|2474612|N|N|N|N|N| +2474703|AAAAAAAAPMCMFCAA|2063-05-28|1960|8526|654|2063|1|5|28|2|2063|654|8526|Monday|2063Q2|N|N|N|2474676|2474795|2474338|2474613|N|N|N|N|N| +2474704|AAAAAAAAANCMFCAA|2063-05-29|1960|8527|654|2063|2|5|29|2|2063|654|8527|Tuesday|2063Q2|N|N|N|2474676|2474795|2474339|2474614|N|N|N|N|N| +2474705|AAAAAAAABNCMFCAA|2063-05-30|1960|8527|654|2063|3|5|30|2|2063|654|8527|Wednesday|2063Q2|N|N|N|2474676|2474795|2474340|2474615|N|N|N|N|N| +2474706|AAAAAAAACNCMFCAA|2063-05-31|1960|8527|654|2063|4|5|31|2|2063|654|8527|Thursday|2063Q2|N|N|N|2474676|2474795|2474341|2474616|N|N|N|N|N| +2474707|AAAAAAAADNCMFCAA|2063-06-01|1961|8527|655|2063|5|6|1|2|2063|655|8527|Friday|2063Q2|N|Y|N|2474707|2474857|2474342|2474617|N|N|N|N|N| +2474708|AAAAAAAAENCMFCAA|2063-06-02|1961|8527|655|2063|6|6|2|2|2063|655|8527|Saturday|2063Q2|N|Y|N|2474707|2474857|2474343|2474618|N|N|N|N|N| +2474709|AAAAAAAAFNCMFCAA|2063-06-03|1961|8527|655|2063|0|6|3|2|2063|655|8527|Sunday|2063Q2|N|N|N|2474707|2474857|2474344|2474619|N|N|N|N|N| +2474710|AAAAAAAAGNCMFCAA|2063-06-04|1961|8527|655|2063|1|6|4|2|2063|655|8527|Monday|2063Q2|N|N|N|2474707|2474857|2474345|2474620|N|N|N|N|N| +2474711|AAAAAAAAHNCMFCAA|2063-06-05|1961|8528|655|2063|2|6|5|2|2063|655|8528|Tuesday|2063Q2|N|N|N|2474707|2474857|2474346|2474621|N|N|N|N|N| +2474712|AAAAAAAAINCMFCAA|2063-06-06|1961|8528|655|2063|3|6|6|2|2063|655|8528|Wednesday|2063Q2|N|N|N|2474707|2474857|2474347|2474622|N|N|N|N|N| +2474713|AAAAAAAAJNCMFCAA|2063-06-07|1961|8528|655|2063|4|6|7|2|2063|655|8528|Thursday|2063Q2|N|N|N|2474707|2474857|2474348|2474623|N|N|N|N|N| +2474714|AAAAAAAAKNCMFCAA|2063-06-08|1961|8528|655|2063|5|6|8|2|2063|655|8528|Friday|2063Q2|N|Y|N|2474707|2474857|2474349|2474624|N|N|N|N|N| +2474715|AAAAAAAALNCMFCAA|2063-06-09|1961|8528|655|2063|6|6|9|2|2063|655|8528|Saturday|2063Q2|N|Y|N|2474707|2474857|2474350|2474625|N|N|N|N|N| +2474716|AAAAAAAAMNCMFCAA|2063-06-10|1961|8528|655|2063|0|6|10|2|2063|655|8528|Sunday|2063Q2|N|N|N|2474707|2474857|2474351|2474626|N|N|N|N|N| +2474717|AAAAAAAANNCMFCAA|2063-06-11|1961|8528|655|2063|1|6|11|2|2063|655|8528|Monday|2063Q2|N|N|N|2474707|2474857|2474352|2474627|N|N|N|N|N| +2474718|AAAAAAAAONCMFCAA|2063-06-12|1961|8529|655|2063|2|6|12|2|2063|655|8529|Tuesday|2063Q2|N|N|N|2474707|2474857|2474353|2474628|N|N|N|N|N| +2474719|AAAAAAAAPNCMFCAA|2063-06-13|1961|8529|655|2063|3|6|13|2|2063|655|8529|Wednesday|2063Q2|N|N|N|2474707|2474857|2474354|2474629|N|N|N|N|N| +2474720|AAAAAAAAAOCMFCAA|2063-06-14|1961|8529|655|2063|4|6|14|2|2063|655|8529|Thursday|2063Q2|N|N|N|2474707|2474857|2474355|2474630|N|N|N|N|N| +2474721|AAAAAAAABOCMFCAA|2063-06-15|1961|8529|655|2063|5|6|15|2|2063|655|8529|Friday|2063Q2|N|Y|N|2474707|2474857|2474356|2474631|N|N|N|N|N| +2474722|AAAAAAAACOCMFCAA|2063-06-16|1961|8529|655|2063|6|6|16|2|2063|655|8529|Saturday|2063Q2|N|Y|N|2474707|2474857|2474357|2474632|N|N|N|N|N| +2474723|AAAAAAAADOCMFCAA|2063-06-17|1961|8529|655|2063|0|6|17|2|2063|655|8529|Sunday|2063Q2|N|N|N|2474707|2474857|2474358|2474633|N|N|N|N|N| +2474724|AAAAAAAAEOCMFCAA|2063-06-18|1961|8529|655|2063|1|6|18|2|2063|655|8529|Monday|2063Q2|N|N|N|2474707|2474857|2474359|2474634|N|N|N|N|N| +2474725|AAAAAAAAFOCMFCAA|2063-06-19|1961|8530|655|2063|2|6|19|2|2063|655|8530|Tuesday|2063Q2|N|N|N|2474707|2474857|2474360|2474635|N|N|N|N|N| +2474726|AAAAAAAAGOCMFCAA|2063-06-20|1961|8530|655|2063|3|6|20|2|2063|655|8530|Wednesday|2063Q2|N|N|N|2474707|2474857|2474361|2474636|N|N|N|N|N| +2474727|AAAAAAAAHOCMFCAA|2063-06-21|1961|8530|655|2063|4|6|21|2|2063|655|8530|Thursday|2063Q2|N|N|N|2474707|2474857|2474362|2474637|N|N|N|N|N| +2474728|AAAAAAAAIOCMFCAA|2063-06-22|1961|8530|655|2063|5|6|22|2|2063|655|8530|Friday|2063Q2|N|Y|N|2474707|2474857|2474363|2474638|N|N|N|N|N| +2474729|AAAAAAAAJOCMFCAA|2063-06-23|1961|8530|655|2063|6|6|23|2|2063|655|8530|Saturday|2063Q2|N|Y|N|2474707|2474857|2474364|2474639|N|N|N|N|N| +2474730|AAAAAAAAKOCMFCAA|2063-06-24|1961|8530|655|2063|0|6|24|2|2063|655|8530|Sunday|2063Q2|N|N|N|2474707|2474857|2474365|2474640|N|N|N|N|N| +2474731|AAAAAAAALOCMFCAA|2063-06-25|1961|8530|655|2063|1|6|25|2|2063|655|8530|Monday|2063Q2|N|N|N|2474707|2474857|2474366|2474641|N|N|N|N|N| +2474732|AAAAAAAAMOCMFCAA|2063-06-26|1961|8531|655|2063|2|6|26|2|2063|655|8531|Tuesday|2063Q2|N|N|N|2474707|2474857|2474367|2474642|N|N|N|N|N| +2474733|AAAAAAAANOCMFCAA|2063-06-27|1961|8531|655|2063|3|6|27|2|2063|655|8531|Wednesday|2063Q2|N|N|N|2474707|2474857|2474368|2474643|N|N|N|N|N| +2474734|AAAAAAAAOOCMFCAA|2063-06-28|1961|8531|655|2063|4|6|28|2|2063|655|8531|Thursday|2063Q2|N|N|N|2474707|2474857|2474369|2474644|N|N|N|N|N| +2474735|AAAAAAAAPOCMFCAA|2063-06-29|1961|8531|655|2063|5|6|29|2|2063|655|8531|Friday|2063Q2|N|Y|N|2474707|2474857|2474370|2474645|N|N|N|N|N| +2474736|AAAAAAAAAPCMFCAA|2063-06-30|1961|8531|655|2063|6|6|30|2|2063|655|8531|Saturday|2063Q2|N|Y|N|2474707|2474857|2474371|2474646|N|N|N|N|N| +2474737|AAAAAAAABPCMFCAA|2063-07-01|1962|8531|655|2063|0|7|1|2|2063|655|8531|Sunday|2063Q2|N|N|N|2474737|2474917|2474372|2474646|N|N|N|N|N| +2474738|AAAAAAAACPCMFCAA|2063-07-02|1962|8531|655|2063|1|7|2|3|2063|655|8531|Monday|2063Q3|N|N|N|2474737|2474917|2474373|2474647|N|N|N|N|N| +2474739|AAAAAAAADPCMFCAA|2063-07-03|1962|8532|655|2063|2|7|3|3|2063|655|8532|Tuesday|2063Q3|N|N|N|2474737|2474917|2474374|2474648|N|N|N|N|N| +2474740|AAAAAAAAEPCMFCAA|2063-07-04|1962|8532|655|2063|3|7|4|3|2063|655|8532|Wednesday|2063Q3|N|N|N|2474737|2474917|2474375|2474649|N|N|N|N|N| +2474741|AAAAAAAAFPCMFCAA|2063-07-05|1962|8532|655|2063|4|7|5|3|2063|655|8532|Thursday|2063Q3|Y|N|N|2474737|2474917|2474376|2474650|N|N|N|N|N| +2474742|AAAAAAAAGPCMFCAA|2063-07-06|1962|8532|655|2063|5|7|6|3|2063|655|8532|Friday|2063Q3|N|Y|Y|2474737|2474917|2474377|2474651|N|N|N|N|N| +2474743|AAAAAAAAHPCMFCAA|2063-07-07|1962|8532|655|2063|6|7|7|3|2063|655|8532|Saturday|2063Q3|N|Y|N|2474737|2474917|2474378|2474652|N|N|N|N|N| +2474744|AAAAAAAAIPCMFCAA|2063-07-08|1962|8532|655|2063|0|7|8|3|2063|655|8532|Sunday|2063Q3|N|N|N|2474737|2474917|2474379|2474653|N|N|N|N|N| +2474745|AAAAAAAAJPCMFCAA|2063-07-09|1962|8532|655|2063|1|7|9|3|2063|655|8532|Monday|2063Q3|N|N|N|2474737|2474917|2474380|2474654|N|N|N|N|N| +2474746|AAAAAAAAKPCMFCAA|2063-07-10|1962|8533|655|2063|2|7|10|3|2063|655|8533|Tuesday|2063Q3|N|N|N|2474737|2474917|2474381|2474655|N|N|N|N|N| +2474747|AAAAAAAALPCMFCAA|2063-07-11|1962|8533|655|2063|3|7|11|3|2063|655|8533|Wednesday|2063Q3|N|N|N|2474737|2474917|2474382|2474656|N|N|N|N|N| +2474748|AAAAAAAAMPCMFCAA|2063-07-12|1962|8533|655|2063|4|7|12|3|2063|655|8533|Thursday|2063Q3|N|N|N|2474737|2474917|2474383|2474657|N|N|N|N|N| +2474749|AAAAAAAANPCMFCAA|2063-07-13|1962|8533|655|2063|5|7|13|3|2063|655|8533|Friday|2063Q3|N|Y|N|2474737|2474917|2474384|2474658|N|N|N|N|N| +2474750|AAAAAAAAOPCMFCAA|2063-07-14|1962|8533|655|2063|6|7|14|3|2063|655|8533|Saturday|2063Q3|N|Y|N|2474737|2474917|2474385|2474659|N|N|N|N|N| +2474751|AAAAAAAAPPCMFCAA|2063-07-15|1962|8533|655|2063|0|7|15|3|2063|655|8533|Sunday|2063Q3|N|N|N|2474737|2474917|2474386|2474660|N|N|N|N|N| +2474752|AAAAAAAAAADMFCAA|2063-07-16|1962|8533|655|2063|1|7|16|3|2063|655|8533|Monday|2063Q3|N|N|N|2474737|2474917|2474387|2474661|N|N|N|N|N| +2474753|AAAAAAAABADMFCAA|2063-07-17|1962|8534|655|2063|2|7|17|3|2063|655|8534|Tuesday|2063Q3|N|N|N|2474737|2474917|2474388|2474662|N|N|N|N|N| +2474754|AAAAAAAACADMFCAA|2063-07-18|1962|8534|655|2063|3|7|18|3|2063|655|8534|Wednesday|2063Q3|N|N|N|2474737|2474917|2474389|2474663|N|N|N|N|N| +2474755|AAAAAAAADADMFCAA|2063-07-19|1962|8534|655|2063|4|7|19|3|2063|655|8534|Thursday|2063Q3|N|N|N|2474737|2474917|2474390|2474664|N|N|N|N|N| +2474756|AAAAAAAAEADMFCAA|2063-07-20|1962|8534|655|2063|5|7|20|3|2063|655|8534|Friday|2063Q3|N|Y|N|2474737|2474917|2474391|2474665|N|N|N|N|N| +2474757|AAAAAAAAFADMFCAA|2063-07-21|1962|8534|655|2063|6|7|21|3|2063|655|8534|Saturday|2063Q3|N|Y|N|2474737|2474917|2474392|2474666|N|N|N|N|N| +2474758|AAAAAAAAGADMFCAA|2063-07-22|1962|8534|655|2063|0|7|22|3|2063|655|8534|Sunday|2063Q3|N|N|N|2474737|2474917|2474393|2474667|N|N|N|N|N| +2474759|AAAAAAAAHADMFCAA|2063-07-23|1962|8534|655|2063|1|7|23|3|2063|655|8534|Monday|2063Q3|N|N|N|2474737|2474917|2474394|2474668|N|N|N|N|N| +2474760|AAAAAAAAIADMFCAA|2063-07-24|1962|8535|655|2063|2|7|24|3|2063|655|8535|Tuesday|2063Q3|N|N|N|2474737|2474917|2474395|2474669|N|N|N|N|N| +2474761|AAAAAAAAJADMFCAA|2063-07-25|1962|8535|655|2063|3|7|25|3|2063|655|8535|Wednesday|2063Q3|N|N|N|2474737|2474917|2474396|2474670|N|N|N|N|N| +2474762|AAAAAAAAKADMFCAA|2063-07-26|1962|8535|655|2063|4|7|26|3|2063|655|8535|Thursday|2063Q3|N|N|N|2474737|2474917|2474397|2474671|N|N|N|N|N| +2474763|AAAAAAAALADMFCAA|2063-07-27|1962|8535|655|2063|5|7|27|3|2063|655|8535|Friday|2063Q3|N|Y|N|2474737|2474917|2474398|2474672|N|N|N|N|N| +2474764|AAAAAAAAMADMFCAA|2063-07-28|1962|8535|655|2063|6|7|28|3|2063|655|8535|Saturday|2063Q3|N|Y|N|2474737|2474917|2474399|2474673|N|N|N|N|N| +2474765|AAAAAAAANADMFCAA|2063-07-29|1962|8535|655|2063|0|7|29|3|2063|655|8535|Sunday|2063Q3|N|N|N|2474737|2474917|2474400|2474674|N|N|N|N|N| +2474766|AAAAAAAAOADMFCAA|2063-07-30|1962|8535|655|2063|1|7|30|3|2063|655|8535|Monday|2063Q3|N|N|N|2474737|2474917|2474401|2474675|N|N|N|N|N| +2474767|AAAAAAAAPADMFCAA|2063-07-31|1962|8536|655|2063|2|7|31|3|2063|655|8536|Tuesday|2063Q3|N|N|N|2474737|2474917|2474402|2474676|N|N|N|N|N| +2474768|AAAAAAAAABDMFCAA|2063-08-01|1963|8536|655|2063|3|8|1|3|2063|655|8536|Wednesday|2063Q3|N|N|N|2474768|2474979|2474403|2474677|N|N|N|N|N| +2474769|AAAAAAAABBDMFCAA|2063-08-02|1963|8536|655|2063|4|8|2|3|2063|655|8536|Thursday|2063Q3|N|N|N|2474768|2474979|2474404|2474678|N|N|N|N|N| +2474770|AAAAAAAACBDMFCAA|2063-08-03|1963|8536|655|2063|5|8|3|3|2063|655|8536|Friday|2063Q3|N|Y|N|2474768|2474979|2474405|2474679|N|N|N|N|N| +2474771|AAAAAAAADBDMFCAA|2063-08-04|1963|8536|655|2063|6|8|4|3|2063|655|8536|Saturday|2063Q3|N|Y|N|2474768|2474979|2474406|2474680|N|N|N|N|N| +2474772|AAAAAAAAEBDMFCAA|2063-08-05|1963|8536|655|2063|0|8|5|3|2063|655|8536|Sunday|2063Q3|N|N|N|2474768|2474979|2474407|2474681|N|N|N|N|N| +2474773|AAAAAAAAFBDMFCAA|2063-08-06|1963|8536|655|2063|1|8|6|3|2063|655|8536|Monday|2063Q3|N|N|N|2474768|2474979|2474408|2474682|N|N|N|N|N| +2474774|AAAAAAAAGBDMFCAA|2063-08-07|1963|8537|655|2063|2|8|7|3|2063|655|8537|Tuesday|2063Q3|N|N|N|2474768|2474979|2474409|2474683|N|N|N|N|N| +2474775|AAAAAAAAHBDMFCAA|2063-08-08|1963|8537|655|2063|3|8|8|3|2063|655|8537|Wednesday|2063Q3|N|N|N|2474768|2474979|2474410|2474684|N|N|N|N|N| +2474776|AAAAAAAAIBDMFCAA|2063-08-09|1963|8537|655|2063|4|8|9|3|2063|655|8537|Thursday|2063Q3|N|N|N|2474768|2474979|2474411|2474685|N|N|N|N|N| +2474777|AAAAAAAAJBDMFCAA|2063-08-10|1963|8537|655|2063|5|8|10|3|2063|655|8537|Friday|2063Q3|N|Y|N|2474768|2474979|2474412|2474686|N|N|N|N|N| +2474778|AAAAAAAAKBDMFCAA|2063-08-11|1963|8537|655|2063|6|8|11|3|2063|655|8537|Saturday|2063Q3|N|Y|N|2474768|2474979|2474413|2474687|N|N|N|N|N| +2474779|AAAAAAAALBDMFCAA|2063-08-12|1963|8537|655|2063|0|8|12|3|2063|655|8537|Sunday|2063Q3|N|N|N|2474768|2474979|2474414|2474688|N|N|N|N|N| +2474780|AAAAAAAAMBDMFCAA|2063-08-13|1963|8537|655|2063|1|8|13|3|2063|655|8537|Monday|2063Q3|N|N|N|2474768|2474979|2474415|2474689|N|N|N|N|N| +2474781|AAAAAAAANBDMFCAA|2063-08-14|1963|8538|655|2063|2|8|14|3|2063|655|8538|Tuesday|2063Q3|N|N|N|2474768|2474979|2474416|2474690|N|N|N|N|N| +2474782|AAAAAAAAOBDMFCAA|2063-08-15|1963|8538|655|2063|3|8|15|3|2063|655|8538|Wednesday|2063Q3|N|N|N|2474768|2474979|2474417|2474691|N|N|N|N|N| +2474783|AAAAAAAAPBDMFCAA|2063-08-16|1963|8538|655|2063|4|8|16|3|2063|655|8538|Thursday|2063Q3|N|N|N|2474768|2474979|2474418|2474692|N|N|N|N|N| +2474784|AAAAAAAAACDMFCAA|2063-08-17|1963|8538|655|2063|5|8|17|3|2063|655|8538|Friday|2063Q3|N|Y|N|2474768|2474979|2474419|2474693|N|N|N|N|N| +2474785|AAAAAAAABCDMFCAA|2063-08-18|1963|8538|655|2063|6|8|18|3|2063|655|8538|Saturday|2063Q3|N|Y|N|2474768|2474979|2474420|2474694|N|N|N|N|N| +2474786|AAAAAAAACCDMFCAA|2063-08-19|1963|8538|655|2063|0|8|19|3|2063|655|8538|Sunday|2063Q3|N|N|N|2474768|2474979|2474421|2474695|N|N|N|N|N| +2474787|AAAAAAAADCDMFCAA|2063-08-20|1963|8538|655|2063|1|8|20|3|2063|655|8538|Monday|2063Q3|N|N|N|2474768|2474979|2474422|2474696|N|N|N|N|N| +2474788|AAAAAAAAECDMFCAA|2063-08-21|1963|8539|655|2063|2|8|21|3|2063|655|8539|Tuesday|2063Q3|N|N|N|2474768|2474979|2474423|2474697|N|N|N|N|N| +2474789|AAAAAAAAFCDMFCAA|2063-08-22|1963|8539|655|2063|3|8|22|3|2063|655|8539|Wednesday|2063Q3|N|N|N|2474768|2474979|2474424|2474698|N|N|N|N|N| +2474790|AAAAAAAAGCDMFCAA|2063-08-23|1963|8539|655|2063|4|8|23|3|2063|655|8539|Thursday|2063Q3|N|N|N|2474768|2474979|2474425|2474699|N|N|N|N|N| +2474791|AAAAAAAAHCDMFCAA|2063-08-24|1963|8539|655|2063|5|8|24|3|2063|655|8539|Friday|2063Q3|N|Y|N|2474768|2474979|2474426|2474700|N|N|N|N|N| +2474792|AAAAAAAAICDMFCAA|2063-08-25|1963|8539|655|2063|6|8|25|3|2063|655|8539|Saturday|2063Q3|N|Y|N|2474768|2474979|2474427|2474701|N|N|N|N|N| +2474793|AAAAAAAAJCDMFCAA|2063-08-26|1963|8539|655|2063|0|8|26|3|2063|655|8539|Sunday|2063Q3|N|N|N|2474768|2474979|2474428|2474702|N|N|N|N|N| +2474794|AAAAAAAAKCDMFCAA|2063-08-27|1963|8539|655|2063|1|8|27|3|2063|655|8539|Monday|2063Q3|N|N|N|2474768|2474979|2474429|2474703|N|N|N|N|N| +2474795|AAAAAAAALCDMFCAA|2063-08-28|1963|8540|655|2063|2|8|28|3|2063|655|8540|Tuesday|2063Q3|N|N|N|2474768|2474979|2474430|2474704|N|N|N|N|N| +2474796|AAAAAAAAMCDMFCAA|2063-08-29|1963|8540|655|2063|3|8|29|3|2063|655|8540|Wednesday|2063Q3|N|N|N|2474768|2474979|2474431|2474705|N|N|N|N|N| +2474797|AAAAAAAANCDMFCAA|2063-08-30|1963|8540|655|2063|4|8|30|3|2063|655|8540|Thursday|2063Q3|N|N|N|2474768|2474979|2474432|2474706|N|N|N|N|N| +2474798|AAAAAAAAOCDMFCAA|2063-08-31|1963|8540|655|2063|5|8|31|3|2063|655|8540|Friday|2063Q3|N|Y|N|2474768|2474979|2474433|2474707|N|N|N|N|N| +2474799|AAAAAAAAPCDMFCAA|2063-09-01|1964|8540|656|2063|6|9|1|3|2063|656|8540|Saturday|2063Q3|N|Y|N|2474799|2475041|2474434|2474708|N|N|N|N|N| +2474800|AAAAAAAAADDMFCAA|2063-09-02|1964|8540|656|2063|0|9|2|3|2063|656|8540|Sunday|2063Q3|N|N|N|2474799|2475041|2474435|2474709|N|N|N|N|N| +2474801|AAAAAAAABDDMFCAA|2063-09-03|1964|8540|656|2063|1|9|3|3|2063|656|8540|Monday|2063Q3|N|N|N|2474799|2475041|2474436|2474710|N|N|N|N|N| +2474802|AAAAAAAACDDMFCAA|2063-09-04|1964|8541|656|2063|2|9|4|3|2063|656|8541|Tuesday|2063Q3|N|N|N|2474799|2475041|2474437|2474711|N|N|N|N|N| +2474803|AAAAAAAADDDMFCAA|2063-09-05|1964|8541|656|2063|3|9|5|3|2063|656|8541|Wednesday|2063Q3|N|N|N|2474799|2475041|2474438|2474712|N|N|N|N|N| +2474804|AAAAAAAAEDDMFCAA|2063-09-06|1964|8541|656|2063|4|9|6|3|2063|656|8541|Thursday|2063Q3|N|N|N|2474799|2475041|2474439|2474713|N|N|N|N|N| +2474805|AAAAAAAAFDDMFCAA|2063-09-07|1964|8541|656|2063|5|9|7|3|2063|656|8541|Friday|2063Q3|N|Y|N|2474799|2475041|2474440|2474714|N|N|N|N|N| +2474806|AAAAAAAAGDDMFCAA|2063-09-08|1964|8541|656|2063|6|9|8|3|2063|656|8541|Saturday|2063Q3|N|Y|N|2474799|2475041|2474441|2474715|N|N|N|N|N| +2474807|AAAAAAAAHDDMFCAA|2063-09-09|1964|8541|656|2063|0|9|9|3|2063|656|8541|Sunday|2063Q3|N|N|N|2474799|2475041|2474442|2474716|N|N|N|N|N| +2474808|AAAAAAAAIDDMFCAA|2063-09-10|1964|8541|656|2063|1|9|10|3|2063|656|8541|Monday|2063Q3|N|N|N|2474799|2475041|2474443|2474717|N|N|N|N|N| +2474809|AAAAAAAAJDDMFCAA|2063-09-11|1964|8542|656|2063|2|9|11|3|2063|656|8542|Tuesday|2063Q3|N|N|N|2474799|2475041|2474444|2474718|N|N|N|N|N| +2474810|AAAAAAAAKDDMFCAA|2063-09-12|1964|8542|656|2063|3|9|12|3|2063|656|8542|Wednesday|2063Q3|N|N|N|2474799|2475041|2474445|2474719|N|N|N|N|N| +2474811|AAAAAAAALDDMFCAA|2063-09-13|1964|8542|656|2063|4|9|13|3|2063|656|8542|Thursday|2063Q3|N|N|N|2474799|2475041|2474446|2474720|N|N|N|N|N| +2474812|AAAAAAAAMDDMFCAA|2063-09-14|1964|8542|656|2063|5|9|14|3|2063|656|8542|Friday|2063Q3|N|Y|N|2474799|2475041|2474447|2474721|N|N|N|N|N| +2474813|AAAAAAAANDDMFCAA|2063-09-15|1964|8542|656|2063|6|9|15|3|2063|656|8542|Saturday|2063Q3|N|Y|N|2474799|2475041|2474448|2474722|N|N|N|N|N| +2474814|AAAAAAAAODDMFCAA|2063-09-16|1964|8542|656|2063|0|9|16|3|2063|656|8542|Sunday|2063Q3|N|N|N|2474799|2475041|2474449|2474723|N|N|N|N|N| +2474815|AAAAAAAAPDDMFCAA|2063-09-17|1964|8542|656|2063|1|9|17|3|2063|656|8542|Monday|2063Q3|N|N|N|2474799|2475041|2474450|2474724|N|N|N|N|N| +2474816|AAAAAAAAAEDMFCAA|2063-09-18|1964|8543|656|2063|2|9|18|3|2063|656|8543|Tuesday|2063Q3|N|N|N|2474799|2475041|2474451|2474725|N|N|N|N|N| +2474817|AAAAAAAABEDMFCAA|2063-09-19|1964|8543|656|2063|3|9|19|3|2063|656|8543|Wednesday|2063Q3|N|N|N|2474799|2475041|2474452|2474726|N|N|N|N|N| +2474818|AAAAAAAACEDMFCAA|2063-09-20|1964|8543|656|2063|4|9|20|3|2063|656|8543|Thursday|2063Q3|N|N|N|2474799|2475041|2474453|2474727|N|N|N|N|N| +2474819|AAAAAAAADEDMFCAA|2063-09-21|1964|8543|656|2063|5|9|21|3|2063|656|8543|Friday|2063Q3|N|Y|N|2474799|2475041|2474454|2474728|N|N|N|N|N| +2474820|AAAAAAAAEEDMFCAA|2063-09-22|1964|8543|656|2063|6|9|22|3|2063|656|8543|Saturday|2063Q3|N|Y|N|2474799|2475041|2474455|2474729|N|N|N|N|N| +2474821|AAAAAAAAFEDMFCAA|2063-09-23|1964|8543|656|2063|0|9|23|3|2063|656|8543|Sunday|2063Q3|N|N|N|2474799|2475041|2474456|2474730|N|N|N|N|N| +2474822|AAAAAAAAGEDMFCAA|2063-09-24|1964|8543|656|2063|1|9|24|3|2063|656|8543|Monday|2063Q3|N|N|N|2474799|2475041|2474457|2474731|N|N|N|N|N| +2474823|AAAAAAAAHEDMFCAA|2063-09-25|1964|8544|656|2063|2|9|25|3|2063|656|8544|Tuesday|2063Q3|N|N|N|2474799|2475041|2474458|2474732|N|N|N|N|N| +2474824|AAAAAAAAIEDMFCAA|2063-09-26|1964|8544|656|2063|3|9|26|3|2063|656|8544|Wednesday|2063Q3|N|N|N|2474799|2475041|2474459|2474733|N|N|N|N|N| +2474825|AAAAAAAAJEDMFCAA|2063-09-27|1964|8544|656|2063|4|9|27|3|2063|656|8544|Thursday|2063Q3|N|N|N|2474799|2475041|2474460|2474734|N|N|N|N|N| +2474826|AAAAAAAAKEDMFCAA|2063-09-28|1964|8544|656|2063|5|9|28|3|2063|656|8544|Friday|2063Q3|N|Y|N|2474799|2475041|2474461|2474735|N|N|N|N|N| +2474827|AAAAAAAALEDMFCAA|2063-09-29|1964|8544|656|2063|6|9|29|3|2063|656|8544|Saturday|2063Q3|N|Y|N|2474799|2475041|2474462|2474736|N|N|N|N|N| +2474828|AAAAAAAAMEDMFCAA|2063-09-30|1964|8544|656|2063|0|9|30|3|2063|656|8544|Sunday|2063Q3|N|N|N|2474799|2475041|2474463|2474737|N|N|N|N|N| +2474829|AAAAAAAANEDMFCAA|2063-10-01|1965|8544|656|2063|1|10|1|3|2063|656|8544|Monday|2063Q3|N|N|N|2474829|2475101|2474464|2474737|N|N|N|N|N| +2474830|AAAAAAAAOEDMFCAA|2063-10-02|1965|8545|656|2063|2|10|2|4|2063|656|8545|Tuesday|2063Q4|N|N|N|2474829|2475101|2474465|2474738|N|N|N|N|N| +2474831|AAAAAAAAPEDMFCAA|2063-10-03|1965|8545|656|2063|3|10|3|4|2063|656|8545|Wednesday|2063Q4|N|N|N|2474829|2475101|2474466|2474739|N|N|N|N|N| +2474832|AAAAAAAAAFDMFCAA|2063-10-04|1965|8545|656|2063|4|10|4|4|2063|656|8545|Thursday|2063Q4|N|N|N|2474829|2475101|2474467|2474740|N|N|N|N|N| +2474833|AAAAAAAABFDMFCAA|2063-10-05|1965|8545|656|2063|5|10|5|4|2063|656|8545|Friday|2063Q4|N|Y|N|2474829|2475101|2474468|2474741|N|N|N|N|N| +2474834|AAAAAAAACFDMFCAA|2063-10-06|1965|8545|656|2063|6|10|6|4|2063|656|8545|Saturday|2063Q4|N|Y|N|2474829|2475101|2474469|2474742|N|N|N|N|N| +2474835|AAAAAAAADFDMFCAA|2063-10-07|1965|8545|656|2063|0|10|7|4|2063|656|8545|Sunday|2063Q4|N|N|N|2474829|2475101|2474470|2474743|N|N|N|N|N| +2474836|AAAAAAAAEFDMFCAA|2063-10-08|1965|8545|656|2063|1|10|8|4|2063|656|8545|Monday|2063Q4|N|N|N|2474829|2475101|2474471|2474744|N|N|N|N|N| +2474837|AAAAAAAAFFDMFCAA|2063-10-09|1965|8546|656|2063|2|10|9|4|2063|656|8546|Tuesday|2063Q4|N|N|N|2474829|2475101|2474472|2474745|N|N|N|N|N| +2474838|AAAAAAAAGFDMFCAA|2063-10-10|1965|8546|656|2063|3|10|10|4|2063|656|8546|Wednesday|2063Q4|N|N|N|2474829|2475101|2474473|2474746|N|N|N|N|N| +2474839|AAAAAAAAHFDMFCAA|2063-10-11|1965|8546|656|2063|4|10|11|4|2063|656|8546|Thursday|2063Q4|N|N|N|2474829|2475101|2474474|2474747|N|N|N|N|N| +2474840|AAAAAAAAIFDMFCAA|2063-10-12|1965|8546|656|2063|5|10|12|4|2063|656|8546|Friday|2063Q4|N|Y|N|2474829|2475101|2474475|2474748|N|N|N|N|N| +2474841|AAAAAAAAJFDMFCAA|2063-10-13|1965|8546|656|2063|6|10|13|4|2063|656|8546|Saturday|2063Q4|N|Y|N|2474829|2475101|2474476|2474749|N|N|N|N|N| +2474842|AAAAAAAAKFDMFCAA|2063-10-14|1965|8546|656|2063|0|10|14|4|2063|656|8546|Sunday|2063Q4|N|N|N|2474829|2475101|2474477|2474750|N|N|N|N|N| +2474843|AAAAAAAALFDMFCAA|2063-10-15|1965|8546|656|2063|1|10|15|4|2063|656|8546|Monday|2063Q4|N|N|N|2474829|2475101|2474478|2474751|N|N|N|N|N| +2474844|AAAAAAAAMFDMFCAA|2063-10-16|1965|8547|656|2063|2|10|16|4|2063|656|8547|Tuesday|2063Q4|N|N|N|2474829|2475101|2474479|2474752|N|N|N|N|N| +2474845|AAAAAAAANFDMFCAA|2063-10-17|1965|8547|656|2063|3|10|17|4|2063|656|8547|Wednesday|2063Q4|N|N|N|2474829|2475101|2474480|2474753|N|N|N|N|N| +2474846|AAAAAAAAOFDMFCAA|2063-10-18|1965|8547|656|2063|4|10|18|4|2063|656|8547|Thursday|2063Q4|N|N|N|2474829|2475101|2474481|2474754|N|N|N|N|N| +2474847|AAAAAAAAPFDMFCAA|2063-10-19|1965|8547|656|2063|5|10|19|4|2063|656|8547|Friday|2063Q4|N|Y|N|2474829|2475101|2474482|2474755|N|N|N|N|N| +2474848|AAAAAAAAAGDMFCAA|2063-10-20|1965|8547|656|2063|6|10|20|4|2063|656|8547|Saturday|2063Q4|N|Y|N|2474829|2475101|2474483|2474756|N|N|N|N|N| +2474849|AAAAAAAABGDMFCAA|2063-10-21|1965|8547|656|2063|0|10|21|4|2063|656|8547|Sunday|2063Q4|N|N|N|2474829|2475101|2474484|2474757|N|N|N|N|N| +2474850|AAAAAAAACGDMFCAA|2063-10-22|1965|8547|656|2063|1|10|22|4|2063|656|8547|Monday|2063Q4|N|N|N|2474829|2475101|2474485|2474758|N|N|N|N|N| +2474851|AAAAAAAADGDMFCAA|2063-10-23|1965|8548|656|2063|2|10|23|4|2063|656|8548|Tuesday|2063Q4|N|N|N|2474829|2475101|2474486|2474759|N|N|N|N|N| +2474852|AAAAAAAAEGDMFCAA|2063-10-24|1965|8548|656|2063|3|10|24|4|2063|656|8548|Wednesday|2063Q4|N|N|N|2474829|2475101|2474487|2474760|N|N|N|N|N| +2474853|AAAAAAAAFGDMFCAA|2063-10-25|1965|8548|656|2063|4|10|25|4|2063|656|8548|Thursday|2063Q4|N|N|N|2474829|2475101|2474488|2474761|N|N|N|N|N| +2474854|AAAAAAAAGGDMFCAA|2063-10-26|1965|8548|656|2063|5|10|26|4|2063|656|8548|Friday|2063Q4|N|Y|N|2474829|2475101|2474489|2474762|N|N|N|N|N| +2474855|AAAAAAAAHGDMFCAA|2063-10-27|1965|8548|656|2063|6|10|27|4|2063|656|8548|Saturday|2063Q4|N|Y|N|2474829|2475101|2474490|2474763|N|N|N|N|N| +2474856|AAAAAAAAIGDMFCAA|2063-10-28|1965|8548|656|2063|0|10|28|4|2063|656|8548|Sunday|2063Q4|N|N|N|2474829|2475101|2474491|2474764|N|N|N|N|N| +2474857|AAAAAAAAJGDMFCAA|2063-10-29|1965|8548|656|2063|1|10|29|4|2063|656|8548|Monday|2063Q4|N|N|N|2474829|2475101|2474492|2474765|N|N|N|N|N| +2474858|AAAAAAAAKGDMFCAA|2063-10-30|1965|8549|656|2063|2|10|30|4|2063|656|8549|Tuesday|2063Q4|N|N|N|2474829|2475101|2474493|2474766|N|N|N|N|N| +2474859|AAAAAAAALGDMFCAA|2063-10-31|1965|8549|656|2063|3|10|31|4|2063|656|8549|Wednesday|2063Q4|N|N|N|2474829|2475101|2474494|2474767|N|N|N|N|N| +2474860|AAAAAAAAMGDMFCAA|2063-11-01|1966|8549|656|2063|4|11|1|4|2063|656|8549|Thursday|2063Q4|N|N|N|2474860|2475163|2474495|2474768|N|N|N|N|N| +2474861|AAAAAAAANGDMFCAA|2063-11-02|1966|8549|656|2063|5|11|2|4|2063|656|8549|Friday|2063Q4|N|Y|N|2474860|2475163|2474496|2474769|N|N|N|N|N| +2474862|AAAAAAAAOGDMFCAA|2063-11-03|1966|8549|656|2063|6|11|3|4|2063|656|8549|Saturday|2063Q4|N|Y|N|2474860|2475163|2474497|2474770|N|N|N|N|N| +2474863|AAAAAAAAPGDMFCAA|2063-11-04|1966|8549|656|2063|0|11|4|4|2063|656|8549|Sunday|2063Q4|N|N|N|2474860|2475163|2474498|2474771|N|N|N|N|N| +2474864|AAAAAAAAAHDMFCAA|2063-11-05|1966|8549|656|2063|1|11|5|4|2063|656|8549|Monday|2063Q4|N|N|N|2474860|2475163|2474499|2474772|N|N|N|N|N| +2474865|AAAAAAAABHDMFCAA|2063-11-06|1966|8550|656|2063|2|11|6|4|2063|656|8550|Tuesday|2063Q4|N|N|N|2474860|2475163|2474500|2474773|N|N|N|N|N| +2474866|AAAAAAAACHDMFCAA|2063-11-07|1966|8550|656|2063|3|11|7|4|2063|656|8550|Wednesday|2063Q4|N|N|N|2474860|2475163|2474501|2474774|N|N|N|N|N| +2474867|AAAAAAAADHDMFCAA|2063-11-08|1966|8550|656|2063|4|11|8|4|2063|656|8550|Thursday|2063Q4|N|N|N|2474860|2475163|2474502|2474775|N|N|N|N|N| +2474868|AAAAAAAAEHDMFCAA|2063-11-09|1966|8550|656|2063|5|11|9|4|2063|656|8550|Friday|2063Q4|N|Y|N|2474860|2475163|2474503|2474776|N|N|N|N|N| +2474869|AAAAAAAAFHDMFCAA|2063-11-10|1966|8550|656|2063|6|11|10|4|2063|656|8550|Saturday|2063Q4|N|Y|N|2474860|2475163|2474504|2474777|N|N|N|N|N| +2474870|AAAAAAAAGHDMFCAA|2063-11-11|1966|8550|656|2063|0|11|11|4|2063|656|8550|Sunday|2063Q4|N|N|N|2474860|2475163|2474505|2474778|N|N|N|N|N| +2474871|AAAAAAAAHHDMFCAA|2063-11-12|1966|8550|656|2063|1|11|12|4|2063|656|8550|Monday|2063Q4|N|N|N|2474860|2475163|2474506|2474779|N|N|N|N|N| +2474872|AAAAAAAAIHDMFCAA|2063-11-13|1966|8551|656|2063|2|11|13|4|2063|656|8551|Tuesday|2063Q4|N|N|N|2474860|2475163|2474507|2474780|N|N|N|N|N| +2474873|AAAAAAAAJHDMFCAA|2063-11-14|1966|8551|656|2063|3|11|14|4|2063|656|8551|Wednesday|2063Q4|N|N|N|2474860|2475163|2474508|2474781|N|N|N|N|N| +2474874|AAAAAAAAKHDMFCAA|2063-11-15|1966|8551|656|2063|4|11|15|4|2063|656|8551|Thursday|2063Q4|N|N|N|2474860|2475163|2474509|2474782|N|N|N|N|N| +2474875|AAAAAAAALHDMFCAA|2063-11-16|1966|8551|656|2063|5|11|16|4|2063|656|8551|Friday|2063Q4|N|Y|N|2474860|2475163|2474510|2474783|N|N|N|N|N| +2474876|AAAAAAAAMHDMFCAA|2063-11-17|1966|8551|656|2063|6|11|17|4|2063|656|8551|Saturday|2063Q4|N|Y|N|2474860|2475163|2474511|2474784|N|N|N|N|N| +2474877|AAAAAAAANHDMFCAA|2063-11-18|1966|8551|656|2063|0|11|18|4|2063|656|8551|Sunday|2063Q4|N|N|N|2474860|2475163|2474512|2474785|N|N|N|N|N| +2474878|AAAAAAAAOHDMFCAA|2063-11-19|1966|8551|656|2063|1|11|19|4|2063|656|8551|Monday|2063Q4|N|N|N|2474860|2475163|2474513|2474786|N|N|N|N|N| +2474879|AAAAAAAAPHDMFCAA|2063-11-20|1966|8552|656|2063|2|11|20|4|2063|656|8552|Tuesday|2063Q4|N|N|N|2474860|2475163|2474514|2474787|N|N|N|N|N| +2474880|AAAAAAAAAIDMFCAA|2063-11-21|1966|8552|656|2063|3|11|21|4|2063|656|8552|Wednesday|2063Q4|N|N|N|2474860|2475163|2474515|2474788|N|N|N|N|N| +2474881|AAAAAAAABIDMFCAA|2063-11-22|1966|8552|656|2063|4|11|22|4|2063|656|8552|Thursday|2063Q4|N|N|N|2474860|2475163|2474516|2474789|N|N|N|N|N| +2474882|AAAAAAAACIDMFCAA|2063-11-23|1966|8552|656|2063|5|11|23|4|2063|656|8552|Friday|2063Q4|N|Y|N|2474860|2475163|2474517|2474790|N|N|N|N|N| +2474883|AAAAAAAADIDMFCAA|2063-11-24|1966|8552|656|2063|6|11|24|4|2063|656|8552|Saturday|2063Q4|N|Y|N|2474860|2475163|2474518|2474791|N|N|N|N|N| +2474884|AAAAAAAAEIDMFCAA|2063-11-25|1966|8552|656|2063|0|11|25|4|2063|656|8552|Sunday|2063Q4|N|N|N|2474860|2475163|2474519|2474792|N|N|N|N|N| +2474885|AAAAAAAAFIDMFCAA|2063-11-26|1966|8552|656|2063|1|11|26|4|2063|656|8552|Monday|2063Q4|N|N|N|2474860|2475163|2474520|2474793|N|N|N|N|N| +2474886|AAAAAAAAGIDMFCAA|2063-11-27|1966|8553|656|2063|2|11|27|4|2063|656|8553|Tuesday|2063Q4|N|N|N|2474860|2475163|2474521|2474794|N|N|N|N|N| +2474887|AAAAAAAAHIDMFCAA|2063-11-28|1966|8553|656|2063|3|11|28|4|2063|656|8553|Wednesday|2063Q4|N|N|N|2474860|2475163|2474522|2474795|N|N|N|N|N| +2474888|AAAAAAAAIIDMFCAA|2063-11-29|1966|8553|656|2063|4|11|29|4|2063|656|8553|Thursday|2063Q4|N|N|N|2474860|2475163|2474523|2474796|N|N|N|N|N| +2474889|AAAAAAAAJIDMFCAA|2063-11-30|1966|8553|656|2063|5|11|30|4|2063|656|8553|Friday|2063Q4|N|Y|N|2474860|2475163|2474524|2474797|N|N|N|N|N| +2474890|AAAAAAAAKIDMFCAA|2063-12-01|1967|8553|657|2063|6|12|1|4|2063|657|8553|Saturday|2063Q4|N|Y|N|2474890|2475223|2474525|2474798|N|N|N|N|N| +2474891|AAAAAAAALIDMFCAA|2063-12-02|1967|8553|657|2063|0|12|2|4|2063|657|8553|Sunday|2063Q4|N|N|N|2474890|2475223|2474526|2474799|N|N|N|N|N| +2474892|AAAAAAAAMIDMFCAA|2063-12-03|1967|8553|657|2063|1|12|3|4|2063|657|8553|Monday|2063Q4|N|N|N|2474890|2475223|2474527|2474800|N|N|N|N|N| +2474893|AAAAAAAANIDMFCAA|2063-12-04|1967|8554|657|2063|2|12|4|4|2063|657|8554|Tuesday|2063Q4|N|N|N|2474890|2475223|2474528|2474801|N|N|N|N|N| +2474894|AAAAAAAAOIDMFCAA|2063-12-05|1967|8554|657|2063|3|12|5|4|2063|657|8554|Wednesday|2063Q4|N|N|N|2474890|2475223|2474529|2474802|N|N|N|N|N| +2474895|AAAAAAAAPIDMFCAA|2063-12-06|1967|8554|657|2063|4|12|6|4|2063|657|8554|Thursday|2063Q4|N|N|N|2474890|2475223|2474530|2474803|N|N|N|N|N| +2474896|AAAAAAAAAJDMFCAA|2063-12-07|1967|8554|657|2063|5|12|7|4|2063|657|8554|Friday|2063Q4|N|Y|N|2474890|2475223|2474531|2474804|N|N|N|N|N| +2474897|AAAAAAAABJDMFCAA|2063-12-08|1967|8554|657|2063|6|12|8|4|2063|657|8554|Saturday|2063Q4|N|Y|N|2474890|2475223|2474532|2474805|N|N|N|N|N| +2474898|AAAAAAAACJDMFCAA|2063-12-09|1967|8554|657|2063|0|12|9|4|2063|657|8554|Sunday|2063Q4|N|N|N|2474890|2475223|2474533|2474806|N|N|N|N|N| +2474899|AAAAAAAADJDMFCAA|2063-12-10|1967|8554|657|2063|1|12|10|4|2063|657|8554|Monday|2063Q4|N|N|N|2474890|2475223|2474534|2474807|N|N|N|N|N| +2474900|AAAAAAAAEJDMFCAA|2063-12-11|1967|8555|657|2063|2|12|11|4|2063|657|8555|Tuesday|2063Q4|N|N|N|2474890|2475223|2474535|2474808|N|N|N|N|N| +2474901|AAAAAAAAFJDMFCAA|2063-12-12|1967|8555|657|2063|3|12|12|4|2063|657|8555|Wednesday|2063Q4|N|N|N|2474890|2475223|2474536|2474809|N|N|N|N|N| +2474902|AAAAAAAAGJDMFCAA|2063-12-13|1967|8555|657|2063|4|12|13|4|2063|657|8555|Thursday|2063Q4|N|N|N|2474890|2475223|2474537|2474810|N|N|N|N|N| +2474903|AAAAAAAAHJDMFCAA|2063-12-14|1967|8555|657|2063|5|12|14|4|2063|657|8555|Friday|2063Q4|N|Y|N|2474890|2475223|2474538|2474811|N|N|N|N|N| +2474904|AAAAAAAAIJDMFCAA|2063-12-15|1967|8555|657|2063|6|12|15|4|2063|657|8555|Saturday|2063Q4|N|Y|N|2474890|2475223|2474539|2474812|N|N|N|N|N| +2474905|AAAAAAAAJJDMFCAA|2063-12-16|1967|8555|657|2063|0|12|16|4|2063|657|8555|Sunday|2063Q4|N|N|N|2474890|2475223|2474540|2474813|N|N|N|N|N| +2474906|AAAAAAAAKJDMFCAA|2063-12-17|1967|8555|657|2063|1|12|17|4|2063|657|8555|Monday|2063Q4|N|N|N|2474890|2475223|2474541|2474814|N|N|N|N|N| +2474907|AAAAAAAALJDMFCAA|2063-12-18|1967|8556|657|2063|2|12|18|4|2063|657|8556|Tuesday|2063Q4|N|N|N|2474890|2475223|2474542|2474815|N|N|N|N|N| +2474908|AAAAAAAAMJDMFCAA|2063-12-19|1967|8556|657|2063|3|12|19|4|2063|657|8556|Wednesday|2063Q4|N|N|N|2474890|2475223|2474543|2474816|N|N|N|N|N| +2474909|AAAAAAAANJDMFCAA|2063-12-20|1967|8556|657|2063|4|12|20|4|2063|657|8556|Thursday|2063Q4|N|N|N|2474890|2475223|2474544|2474817|N|N|N|N|N| +2474910|AAAAAAAAOJDMFCAA|2063-12-21|1967|8556|657|2063|5|12|21|4|2063|657|8556|Friday|2063Q4|N|Y|N|2474890|2475223|2474545|2474818|N|N|N|N|N| +2474911|AAAAAAAAPJDMFCAA|2063-12-22|1967|8556|657|2063|6|12|22|4|2063|657|8556|Saturday|2063Q4|N|Y|N|2474890|2475223|2474546|2474819|N|N|N|N|N| +2474912|AAAAAAAAAKDMFCAA|2063-12-23|1967|8556|657|2063|0|12|23|4|2063|657|8556|Sunday|2063Q4|N|N|N|2474890|2475223|2474547|2474820|N|N|N|N|N| +2474913|AAAAAAAABKDMFCAA|2063-12-24|1967|8556|657|2063|1|12|24|4|2063|657|8556|Monday|2063Q4|N|N|N|2474890|2475223|2474548|2474821|N|N|N|N|N| +2474914|AAAAAAAACKDMFCAA|2063-12-25|1967|8557|657|2063|2|12|25|4|2063|657|8557|Tuesday|2063Q4|N|N|N|2474890|2475223|2474549|2474822|N|N|N|N|N| +2474915|AAAAAAAADKDMFCAA|2063-12-26|1967|8557|657|2063|3|12|26|4|2063|657|8557|Wednesday|2063Q4|Y|N|N|2474890|2475223|2474550|2474823|N|N|N|N|N| +2474916|AAAAAAAAEKDMFCAA|2063-12-27|1967|8557|657|2063|4|12|27|4|2063|657|8557|Thursday|2063Q4|N|N|Y|2474890|2475223|2474551|2474824|N|N|N|N|N| +2474917|AAAAAAAAFKDMFCAA|2063-12-28|1967|8557|657|2063|5|12|28|4|2063|657|8557|Friday|2063Q4|N|Y|N|2474890|2475223|2474552|2474825|N|N|N|N|N| +2474918|AAAAAAAAGKDMFCAA|2063-12-29|1967|8557|657|2063|6|12|29|4|2063|657|8557|Saturday|2063Q4|N|Y|N|2474890|2475223|2474553|2474826|N|N|N|N|N| +2474919|AAAAAAAAHKDMFCAA|2063-12-30|1967|8557|657|2063|0|12|30|4|2063|657|8557|Sunday|2063Q4|N|N|N|2474890|2475223|2474554|2474827|N|N|N|N|N| +2474920|AAAAAAAAIKDMFCAA|2063-12-31|1967|8557|657|2063|1|12|31|4|2063|657|8557|Monday|2063Q4|N|N|N|2474890|2475223|2474555|2474828|N|N|N|N|N| +2474921|AAAAAAAAJKDMFCAA|2064-01-01|1968|8558|657|2064|2|1|1|1|2064|657|8558|Tuesday|2064Q1|Y|N|N|2474921|2474920|2474556|2474829|N|N|N|N|N| +2474922|AAAAAAAAKKDMFCAA|2064-01-02|1968|8558|657|2064|3|1|2|1|2064|657|8558|Wednesday|2064Q1|N|N|Y|2474921|2474920|2474557|2474830|N|N|N|N|N| +2474923|AAAAAAAALKDMFCAA|2064-01-03|1968|8558|657|2064|4|1|3|1|2064|657|8558|Thursday|2064Q1|N|N|N|2474921|2474920|2474558|2474831|N|N|N|N|N| +2474924|AAAAAAAAMKDMFCAA|2064-01-04|1968|8558|657|2064|5|1|4|1|2064|657|8558|Friday|2064Q1|N|Y|N|2474921|2474920|2474559|2474832|N|N|N|N|N| +2474925|AAAAAAAANKDMFCAA|2064-01-05|1968|8558|657|2064|6|1|5|1|2064|657|8558|Saturday|2064Q1|N|Y|N|2474921|2474920|2474560|2474833|N|N|N|N|N| +2474926|AAAAAAAAOKDMFCAA|2064-01-06|1968|8558|657|2064|0|1|6|1|2064|657|8558|Sunday|2064Q1|N|N|N|2474921|2474920|2474561|2474834|N|N|N|N|N| +2474927|AAAAAAAAPKDMFCAA|2064-01-07|1968|8558|657|2064|1|1|7|1|2064|657|8558|Monday|2064Q1|N|N|N|2474921|2474920|2474562|2474835|N|N|N|N|N| +2474928|AAAAAAAAALDMFCAA|2064-01-08|1968|8559|657|2064|2|1|8|1|2064|657|8559|Tuesday|2064Q1|N|N|N|2474921|2474920|2474563|2474836|N|N|N|N|N| +2474929|AAAAAAAABLDMFCAA|2064-01-09|1968|8559|657|2064|3|1|9|1|2064|657|8559|Wednesday|2064Q1|N|N|N|2474921|2474920|2474564|2474837|N|N|N|N|N| +2474930|AAAAAAAACLDMFCAA|2064-01-10|1968|8559|657|2064|4|1|10|1|2064|657|8559|Thursday|2064Q1|N|N|N|2474921|2474920|2474565|2474838|N|N|N|N|N| +2474931|AAAAAAAADLDMFCAA|2064-01-11|1968|8559|657|2064|5|1|11|1|2064|657|8559|Friday|2064Q1|N|Y|N|2474921|2474920|2474566|2474839|N|N|N|N|N| +2474932|AAAAAAAAELDMFCAA|2064-01-12|1968|8559|657|2064|6|1|12|1|2064|657|8559|Saturday|2064Q1|N|Y|N|2474921|2474920|2474567|2474840|N|N|N|N|N| +2474933|AAAAAAAAFLDMFCAA|2064-01-13|1968|8559|657|2064|0|1|13|1|2064|657|8559|Sunday|2064Q1|N|N|N|2474921|2474920|2474568|2474841|N|N|N|N|N| +2474934|AAAAAAAAGLDMFCAA|2064-01-14|1968|8559|657|2064|1|1|14|1|2064|657|8559|Monday|2064Q1|N|N|N|2474921|2474920|2474569|2474842|N|N|N|N|N| +2474935|AAAAAAAAHLDMFCAA|2064-01-15|1968|8560|657|2064|2|1|15|1|2064|657|8560|Tuesday|2064Q1|N|N|N|2474921|2474920|2474570|2474843|N|N|N|N|N| +2474936|AAAAAAAAILDMFCAA|2064-01-16|1968|8560|657|2064|3|1|16|1|2064|657|8560|Wednesday|2064Q1|N|N|N|2474921|2474920|2474571|2474844|N|N|N|N|N| +2474937|AAAAAAAAJLDMFCAA|2064-01-17|1968|8560|657|2064|4|1|17|1|2064|657|8560|Thursday|2064Q1|N|N|N|2474921|2474920|2474572|2474845|N|N|N|N|N| +2474938|AAAAAAAAKLDMFCAA|2064-01-18|1968|8560|657|2064|5|1|18|1|2064|657|8560|Friday|2064Q1|N|Y|N|2474921|2474920|2474573|2474846|N|N|N|N|N| +2474939|AAAAAAAALLDMFCAA|2064-01-19|1968|8560|657|2064|6|1|19|1|2064|657|8560|Saturday|2064Q1|N|Y|N|2474921|2474920|2474574|2474847|N|N|N|N|N| +2474940|AAAAAAAAMLDMFCAA|2064-01-20|1968|8560|657|2064|0|1|20|1|2064|657|8560|Sunday|2064Q1|N|N|N|2474921|2474920|2474575|2474848|N|N|N|N|N| +2474941|AAAAAAAANLDMFCAA|2064-01-21|1968|8560|657|2064|1|1|21|1|2064|657|8560|Monday|2064Q1|N|N|N|2474921|2474920|2474576|2474849|N|N|N|N|N| +2474942|AAAAAAAAOLDMFCAA|2064-01-22|1968|8561|657|2064|2|1|22|1|2064|657|8561|Tuesday|2064Q1|N|N|N|2474921|2474920|2474577|2474850|N|N|N|N|N| +2474943|AAAAAAAAPLDMFCAA|2064-01-23|1968|8561|657|2064|3|1|23|1|2064|657|8561|Wednesday|2064Q1|N|N|N|2474921|2474920|2474578|2474851|N|N|N|N|N| +2474944|AAAAAAAAAMDMFCAA|2064-01-24|1968|8561|657|2064|4|1|24|1|2064|657|8561|Thursday|2064Q1|N|N|N|2474921|2474920|2474579|2474852|N|N|N|N|N| +2474945|AAAAAAAABMDMFCAA|2064-01-25|1968|8561|657|2064|5|1|25|1|2064|657|8561|Friday|2064Q1|N|Y|N|2474921|2474920|2474580|2474853|N|N|N|N|N| +2474946|AAAAAAAACMDMFCAA|2064-01-26|1968|8561|657|2064|6|1|26|1|2064|657|8561|Saturday|2064Q1|N|Y|N|2474921|2474920|2474581|2474854|N|N|N|N|N| +2474947|AAAAAAAADMDMFCAA|2064-01-27|1968|8561|657|2064|0|1|27|1|2064|657|8561|Sunday|2064Q1|N|N|N|2474921|2474920|2474582|2474855|N|N|N|N|N| +2474948|AAAAAAAAEMDMFCAA|2064-01-28|1968|8561|657|2064|1|1|28|1|2064|657|8561|Monday|2064Q1|N|N|N|2474921|2474920|2474583|2474856|N|N|N|N|N| +2474949|AAAAAAAAFMDMFCAA|2064-01-29|1968|8562|657|2064|2|1|29|1|2064|657|8562|Tuesday|2064Q1|N|N|N|2474921|2474920|2474584|2474857|N|N|N|N|N| +2474950|AAAAAAAAGMDMFCAA|2064-01-30|1968|8562|657|2064|3|1|30|1|2064|657|8562|Wednesday|2064Q1|N|N|N|2474921|2474920|2474585|2474858|N|N|N|N|N| +2474951|AAAAAAAAHMDMFCAA|2064-01-31|1968|8562|657|2064|4|1|31|1|2064|657|8562|Thursday|2064Q1|N|N|N|2474921|2474920|2474586|2474859|N|N|N|N|N| +2474952|AAAAAAAAIMDMFCAA|2064-02-01|1969|8562|657|2064|5|2|1|1|2064|657|8562|Friday|2064Q1|N|Y|N|2474952|2474982|2474587|2474860|N|N|N|N|N| +2474953|AAAAAAAAJMDMFCAA|2064-02-02|1969|8562|657|2064|6|2|2|1|2064|657|8562|Saturday|2064Q1|N|Y|N|2474952|2474982|2474588|2474861|N|N|N|N|N| +2474954|AAAAAAAAKMDMFCAA|2064-02-03|1969|8562|657|2064|0|2|3|1|2064|657|8562|Sunday|2064Q1|N|N|N|2474952|2474982|2474589|2474862|N|N|N|N|N| +2474955|AAAAAAAALMDMFCAA|2064-02-04|1969|8562|657|2064|1|2|4|1|2064|657|8562|Monday|2064Q1|N|N|N|2474952|2474982|2474590|2474863|N|N|N|N|N| +2474956|AAAAAAAAMMDMFCAA|2064-02-05|1969|8563|657|2064|2|2|5|1|2064|657|8563|Tuesday|2064Q1|N|N|N|2474952|2474982|2474591|2474864|N|N|N|N|N| +2474957|AAAAAAAANMDMFCAA|2064-02-06|1969|8563|657|2064|3|2|6|1|2064|657|8563|Wednesday|2064Q1|N|N|N|2474952|2474982|2474592|2474865|N|N|N|N|N| +2474958|AAAAAAAAOMDMFCAA|2064-02-07|1969|8563|657|2064|4|2|7|1|2064|657|8563|Thursday|2064Q1|N|N|N|2474952|2474982|2474593|2474866|N|N|N|N|N| +2474959|AAAAAAAAPMDMFCAA|2064-02-08|1969|8563|657|2064|5|2|8|1|2064|657|8563|Friday|2064Q1|N|Y|N|2474952|2474982|2474594|2474867|N|N|N|N|N| +2474960|AAAAAAAAANDMFCAA|2064-02-09|1969|8563|657|2064|6|2|9|1|2064|657|8563|Saturday|2064Q1|N|Y|N|2474952|2474982|2474595|2474868|N|N|N|N|N| +2474961|AAAAAAAABNDMFCAA|2064-02-10|1969|8563|657|2064|0|2|10|1|2064|657|8563|Sunday|2064Q1|N|N|N|2474952|2474982|2474596|2474869|N|N|N|N|N| +2474962|AAAAAAAACNDMFCAA|2064-02-11|1969|8563|657|2064|1|2|11|1|2064|657|8563|Monday|2064Q1|N|N|N|2474952|2474982|2474597|2474870|N|N|N|N|N| +2474963|AAAAAAAADNDMFCAA|2064-02-12|1969|8564|657|2064|2|2|12|1|2064|657|8564|Tuesday|2064Q1|N|N|N|2474952|2474982|2474598|2474871|N|N|N|N|N| +2474964|AAAAAAAAENDMFCAA|2064-02-13|1969|8564|657|2064|3|2|13|1|2064|657|8564|Wednesday|2064Q1|N|N|N|2474952|2474982|2474599|2474872|N|N|N|N|N| +2474965|AAAAAAAAFNDMFCAA|2064-02-14|1969|8564|657|2064|4|2|14|1|2064|657|8564|Thursday|2064Q1|N|N|N|2474952|2474982|2474600|2474873|N|N|N|N|N| +2474966|AAAAAAAAGNDMFCAA|2064-02-15|1969|8564|657|2064|5|2|15|1|2064|657|8564|Friday|2064Q1|N|Y|N|2474952|2474982|2474601|2474874|N|N|N|N|N| +2474967|AAAAAAAAHNDMFCAA|2064-02-16|1969|8564|657|2064|6|2|16|1|2064|657|8564|Saturday|2064Q1|N|Y|N|2474952|2474982|2474602|2474875|N|N|N|N|N| +2474968|AAAAAAAAINDMFCAA|2064-02-17|1969|8564|657|2064|0|2|17|1|2064|657|8564|Sunday|2064Q1|N|N|N|2474952|2474982|2474603|2474876|N|N|N|N|N| +2474969|AAAAAAAAJNDMFCAA|2064-02-18|1969|8564|657|2064|1|2|18|1|2064|657|8564|Monday|2064Q1|N|N|N|2474952|2474982|2474604|2474877|N|N|N|N|N| +2474970|AAAAAAAAKNDMFCAA|2064-02-19|1969|8565|657|2064|2|2|19|1|2064|657|8565|Tuesday|2064Q1|N|N|N|2474952|2474982|2474605|2474878|N|N|N|N|N| +2474971|AAAAAAAALNDMFCAA|2064-02-20|1969|8565|657|2064|3|2|20|1|2064|657|8565|Wednesday|2064Q1|N|N|N|2474952|2474982|2474606|2474879|N|N|N|N|N| +2474972|AAAAAAAAMNDMFCAA|2064-02-21|1969|8565|657|2064|4|2|21|1|2064|657|8565|Thursday|2064Q1|N|N|N|2474952|2474982|2474607|2474880|N|N|N|N|N| +2474973|AAAAAAAANNDMFCAA|2064-02-22|1969|8565|657|2064|5|2|22|1|2064|657|8565|Friday|2064Q1|N|Y|N|2474952|2474982|2474608|2474881|N|N|N|N|N| +2474974|AAAAAAAAONDMFCAA|2064-02-23|1969|8565|657|2064|6|2|23|1|2064|657|8565|Saturday|2064Q1|N|Y|N|2474952|2474982|2474609|2474882|N|N|N|N|N| +2474975|AAAAAAAAPNDMFCAA|2064-02-24|1969|8565|657|2064|0|2|24|1|2064|657|8565|Sunday|2064Q1|N|N|N|2474952|2474982|2474610|2474883|N|N|N|N|N| +2474976|AAAAAAAAAODMFCAA|2064-02-25|1969|8565|657|2064|1|2|25|1|2064|657|8565|Monday|2064Q1|N|N|N|2474952|2474982|2474611|2474884|N|N|N|N|N| +2474977|AAAAAAAABODMFCAA|2064-02-26|1969|8566|657|2064|2|2|26|1|2064|657|8566|Tuesday|2064Q1|N|N|N|2474952|2474982|2474612|2474885|N|N|N|N|N| +2474978|AAAAAAAACODMFCAA|2064-02-27|1969|8566|657|2064|3|2|27|1|2064|657|8566|Wednesday|2064Q1|N|N|N|2474952|2474982|2474613|2474886|N|N|N|N|N| +2474979|AAAAAAAADODMFCAA|2064-02-28|1969|8566|657|2064|4|2|28|1|2064|657|8566|Thursday|2064Q1|N|N|N|2474952|2474982|2474614|2474887|N|N|N|N|N| +2474980|AAAAAAAAEODMFCAA|2064-02-29|1969|8566|657|2064|5|2|29|1|2064|657|8566|Friday|2064Q1|N|Y|N|2474952|2474982|2474614|2474888|N|N|N|N|N| +2474981|AAAAAAAAFODMFCAA|2064-03-01|1970|8566|658|2064|6|3|1|1|2064|658|8566|Saturday|2064Q1|N|Y|N|2474981|2475040|2474615|2474889|N|N|N|N|N| +2474982|AAAAAAAAGODMFCAA|2064-03-02|1970|8566|658|2064|0|3|2|1|2064|658|8566|Sunday|2064Q1|N|N|N|2474981|2475040|2474616|2474890|N|N|N|N|N| +2474983|AAAAAAAAHODMFCAA|2064-03-03|1970|8566|658|2064|1|3|3|1|2064|658|8566|Monday|2064Q1|N|N|N|2474981|2475040|2474617|2474891|N|N|N|N|N| +2474984|AAAAAAAAIODMFCAA|2064-03-04|1970|8567|658|2064|2|3|4|1|2064|658|8567|Tuesday|2064Q1|N|N|N|2474981|2475040|2474618|2474892|N|N|N|N|N| +2474985|AAAAAAAAJODMFCAA|2064-03-05|1970|8567|658|2064|3|3|5|1|2064|658|8567|Wednesday|2064Q1|N|N|N|2474981|2475040|2474619|2474893|N|N|N|N|N| +2474986|AAAAAAAAKODMFCAA|2064-03-06|1970|8567|658|2064|4|3|6|1|2064|658|8567|Thursday|2064Q1|N|N|N|2474981|2475040|2474620|2474894|N|N|N|N|N| +2474987|AAAAAAAALODMFCAA|2064-03-07|1970|8567|658|2064|5|3|7|1|2064|658|8567|Friday|2064Q1|N|Y|N|2474981|2475040|2474621|2474895|N|N|N|N|N| +2474988|AAAAAAAAMODMFCAA|2064-03-08|1970|8567|658|2064|6|3|8|1|2064|658|8567|Saturday|2064Q1|N|Y|N|2474981|2475040|2474622|2474896|N|N|N|N|N| +2474989|AAAAAAAANODMFCAA|2064-03-09|1970|8567|658|2064|0|3|9|1|2064|658|8567|Sunday|2064Q1|N|N|N|2474981|2475040|2474623|2474897|N|N|N|N|N| +2474990|AAAAAAAAOODMFCAA|2064-03-10|1970|8567|658|2064|1|3|10|1|2064|658|8567|Monday|2064Q1|N|N|N|2474981|2475040|2474624|2474898|N|N|N|N|N| +2474991|AAAAAAAAPODMFCAA|2064-03-11|1970|8568|658|2064|2|3|11|1|2064|658|8568|Tuesday|2064Q1|N|N|N|2474981|2475040|2474625|2474899|N|N|N|N|N| +2474992|AAAAAAAAAPDMFCAA|2064-03-12|1970|8568|658|2064|3|3|12|1|2064|658|8568|Wednesday|2064Q1|N|N|N|2474981|2475040|2474626|2474900|N|N|N|N|N| +2474993|AAAAAAAABPDMFCAA|2064-03-13|1970|8568|658|2064|4|3|13|1|2064|658|8568|Thursday|2064Q1|N|N|N|2474981|2475040|2474627|2474901|N|N|N|N|N| +2474994|AAAAAAAACPDMFCAA|2064-03-14|1970|8568|658|2064|5|3|14|1|2064|658|8568|Friday|2064Q1|N|Y|N|2474981|2475040|2474628|2474902|N|N|N|N|N| +2474995|AAAAAAAADPDMFCAA|2064-03-15|1970|8568|658|2064|6|3|15|1|2064|658|8568|Saturday|2064Q1|N|Y|N|2474981|2475040|2474629|2474903|N|N|N|N|N| +2474996|AAAAAAAAEPDMFCAA|2064-03-16|1970|8568|658|2064|0|3|16|1|2064|658|8568|Sunday|2064Q1|N|N|N|2474981|2475040|2474630|2474904|N|N|N|N|N| +2474997|AAAAAAAAFPDMFCAA|2064-03-17|1970|8568|658|2064|1|3|17|1|2064|658|8568|Monday|2064Q1|N|N|N|2474981|2475040|2474631|2474905|N|N|N|N|N| +2474998|AAAAAAAAGPDMFCAA|2064-03-18|1970|8569|658|2064|2|3|18|1|2064|658|8569|Tuesday|2064Q1|N|N|N|2474981|2475040|2474632|2474906|N|N|N|N|N| +2474999|AAAAAAAAHPDMFCAA|2064-03-19|1970|8569|658|2064|3|3|19|1|2064|658|8569|Wednesday|2064Q1|N|N|N|2474981|2475040|2474633|2474907|N|N|N|N|N| +2475000|AAAAAAAAIPDMFCAA|2064-03-20|1970|8569|658|2064|4|3|20|1|2064|658|8569|Thursday|2064Q1|N|N|N|2474981|2475040|2474634|2474908|N|N|N|N|N| +2475001|AAAAAAAAJPDMFCAA|2064-03-21|1970|8569|658|2064|5|3|21|1|2064|658|8569|Friday|2064Q1|N|Y|N|2474981|2475040|2474635|2474909|N|N|N|N|N| +2475002|AAAAAAAAKPDMFCAA|2064-03-22|1970|8569|658|2064|6|3|22|1|2064|658|8569|Saturday|2064Q1|N|Y|N|2474981|2475040|2474636|2474910|N|N|N|N|N| +2475003|AAAAAAAALPDMFCAA|2064-03-23|1970|8569|658|2064|0|3|23|1|2064|658|8569|Sunday|2064Q1|N|N|N|2474981|2475040|2474637|2474911|N|N|N|N|N| +2475004|AAAAAAAAMPDMFCAA|2064-03-24|1970|8569|658|2064|1|3|24|1|2064|658|8569|Monday|2064Q1|N|N|N|2474981|2475040|2474638|2474912|N|N|N|N|N| +2475005|AAAAAAAANPDMFCAA|2064-03-25|1970|8570|658|2064|2|3|25|1|2064|658|8570|Tuesday|2064Q1|N|N|N|2474981|2475040|2474639|2474913|N|N|N|N|N| +2475006|AAAAAAAAOPDMFCAA|2064-03-26|1970|8570|658|2064|3|3|26|1|2064|658|8570|Wednesday|2064Q1|N|N|N|2474981|2475040|2474640|2474914|N|N|N|N|N| +2475007|AAAAAAAAPPDMFCAA|2064-03-27|1970|8570|658|2064|4|3|27|1|2064|658|8570|Thursday|2064Q1|N|N|N|2474981|2475040|2474641|2474915|N|N|N|N|N| +2475008|AAAAAAAAAAEMFCAA|2064-03-28|1970|8570|658|2064|5|3|28|1|2064|658|8570|Friday|2064Q1|N|Y|N|2474981|2475040|2474642|2474916|N|N|N|N|N| +2475009|AAAAAAAABAEMFCAA|2064-03-29|1970|8570|658|2064|6|3|29|1|2064|658|8570|Saturday|2064Q1|N|Y|N|2474981|2475040|2474643|2474917|N|N|N|N|N| +2475010|AAAAAAAACAEMFCAA|2064-03-30|1970|8570|658|2064|0|3|30|1|2064|658|8570|Sunday|2064Q1|N|N|N|2474981|2475040|2474644|2474918|N|N|N|N|N| +2475011|AAAAAAAADAEMFCAA|2064-03-31|1970|8570|658|2064|1|3|31|1|2064|658|8570|Monday|2064Q1|N|N|N|2474981|2475040|2474645|2474919|N|N|N|N|N| +2475012|AAAAAAAAEAEMFCAA|2064-04-01|1971|8571|658|2064|2|4|1|2|2064|658|8571|Tuesday|2064Q2|N|N|N|2475012|2475102|2474646|2474921|N|N|N|N|N| +2475013|AAAAAAAAFAEMFCAA|2064-04-02|1971|8571|658|2064|3|4|2|2|2064|658|8571|Wednesday|2064Q2|N|N|N|2475012|2475102|2474647|2474922|N|N|N|N|N| +2475014|AAAAAAAAGAEMFCAA|2064-04-03|1971|8571|658|2064|4|4|3|2|2064|658|8571|Thursday|2064Q2|N|N|N|2475012|2475102|2474648|2474923|N|N|N|N|N| +2475015|AAAAAAAAHAEMFCAA|2064-04-04|1971|8571|658|2064|5|4|4|2|2064|658|8571|Friday|2064Q2|N|Y|N|2475012|2475102|2474649|2474924|N|N|N|N|N| +2475016|AAAAAAAAIAEMFCAA|2064-04-05|1971|8571|658|2064|6|4|5|2|2064|658|8571|Saturday|2064Q2|N|Y|N|2475012|2475102|2474650|2474925|N|N|N|N|N| +2475017|AAAAAAAAJAEMFCAA|2064-04-06|1971|8571|658|2064|0|4|6|2|2064|658|8571|Sunday|2064Q2|N|N|N|2475012|2475102|2474651|2474926|N|N|N|N|N| +2475018|AAAAAAAAKAEMFCAA|2064-04-07|1971|8571|658|2064|1|4|7|2|2064|658|8571|Monday|2064Q2|N|N|N|2475012|2475102|2474652|2474927|N|N|N|N|N| +2475019|AAAAAAAALAEMFCAA|2064-04-08|1971|8572|658|2064|2|4|8|2|2064|658|8572|Tuesday|2064Q2|N|N|N|2475012|2475102|2474653|2474928|N|N|N|N|N| +2475020|AAAAAAAAMAEMFCAA|2064-04-09|1971|8572|658|2064|3|4|9|2|2064|658|8572|Wednesday|2064Q2|N|N|N|2475012|2475102|2474654|2474929|N|N|N|N|N| +2475021|AAAAAAAANAEMFCAA|2064-04-10|1971|8572|658|2064|4|4|10|2|2064|658|8572|Thursday|2064Q2|N|N|N|2475012|2475102|2474655|2474930|N|N|N|N|N| +2475022|AAAAAAAAOAEMFCAA|2064-04-11|1971|8572|658|2064|5|4|11|2|2064|658|8572|Friday|2064Q2|N|Y|N|2475012|2475102|2474656|2474931|N|N|N|N|N| +2475023|AAAAAAAAPAEMFCAA|2064-04-12|1971|8572|658|2064|6|4|12|2|2064|658|8572|Saturday|2064Q2|N|Y|N|2475012|2475102|2474657|2474932|N|N|N|N|N| +2475024|AAAAAAAAABEMFCAA|2064-04-13|1971|8572|658|2064|0|4|13|2|2064|658|8572|Sunday|2064Q2|N|N|N|2475012|2475102|2474658|2474933|N|N|N|N|N| +2475025|AAAAAAAABBEMFCAA|2064-04-14|1971|8572|658|2064|1|4|14|2|2064|658|8572|Monday|2064Q2|N|N|N|2475012|2475102|2474659|2474934|N|N|N|N|N| +2475026|AAAAAAAACBEMFCAA|2064-04-15|1971|8573|658|2064|2|4|15|2|2064|658|8573|Tuesday|2064Q2|N|N|N|2475012|2475102|2474660|2474935|N|N|N|N|N| +2475027|AAAAAAAADBEMFCAA|2064-04-16|1971|8573|658|2064|3|4|16|2|2064|658|8573|Wednesday|2064Q2|N|N|N|2475012|2475102|2474661|2474936|N|N|N|N|N| +2475028|AAAAAAAAEBEMFCAA|2064-04-17|1971|8573|658|2064|4|4|17|2|2064|658|8573|Thursday|2064Q2|N|N|N|2475012|2475102|2474662|2474937|N|N|N|N|N| +2475029|AAAAAAAAFBEMFCAA|2064-04-18|1971|8573|658|2064|5|4|18|2|2064|658|8573|Friday|2064Q2|N|Y|N|2475012|2475102|2474663|2474938|N|N|N|N|N| +2475030|AAAAAAAAGBEMFCAA|2064-04-19|1971|8573|658|2064|6|4|19|2|2064|658|8573|Saturday|2064Q2|N|Y|N|2475012|2475102|2474664|2474939|N|N|N|N|N| +2475031|AAAAAAAAHBEMFCAA|2064-04-20|1971|8573|658|2064|0|4|20|2|2064|658|8573|Sunday|2064Q2|N|N|N|2475012|2475102|2474665|2474940|N|N|N|N|N| +2475032|AAAAAAAAIBEMFCAA|2064-04-21|1971|8573|658|2064|1|4|21|2|2064|658|8573|Monday|2064Q2|N|N|N|2475012|2475102|2474666|2474941|N|N|N|N|N| +2475033|AAAAAAAAJBEMFCAA|2064-04-22|1971|8574|658|2064|2|4|22|2|2064|658|8574|Tuesday|2064Q2|N|N|N|2475012|2475102|2474667|2474942|N|N|N|N|N| +2475034|AAAAAAAAKBEMFCAA|2064-04-23|1971|8574|658|2064|3|4|23|2|2064|658|8574|Wednesday|2064Q2|N|N|N|2475012|2475102|2474668|2474943|N|N|N|N|N| +2475035|AAAAAAAALBEMFCAA|2064-04-24|1971|8574|658|2064|4|4|24|2|2064|658|8574|Thursday|2064Q2|N|N|N|2475012|2475102|2474669|2474944|N|N|N|N|N| +2475036|AAAAAAAAMBEMFCAA|2064-04-25|1971|8574|658|2064|5|4|25|2|2064|658|8574|Friday|2064Q2|N|Y|N|2475012|2475102|2474670|2474945|N|N|N|N|N| +2475037|AAAAAAAANBEMFCAA|2064-04-26|1971|8574|658|2064|6|4|26|2|2064|658|8574|Saturday|2064Q2|N|Y|N|2475012|2475102|2474671|2474946|N|N|N|N|N| +2475038|AAAAAAAAOBEMFCAA|2064-04-27|1971|8574|658|2064|0|4|27|2|2064|658|8574|Sunday|2064Q2|N|N|N|2475012|2475102|2474672|2474947|N|N|N|N|N| +2475039|AAAAAAAAPBEMFCAA|2064-04-28|1971|8574|658|2064|1|4|28|2|2064|658|8574|Monday|2064Q2|N|N|N|2475012|2475102|2474673|2474948|N|N|N|N|N| +2475040|AAAAAAAAACEMFCAA|2064-04-29|1971|8575|658|2064|2|4|29|2|2064|658|8575|Tuesday|2064Q2|N|N|N|2475012|2475102|2474674|2474949|N|N|N|N|N| +2475041|AAAAAAAABCEMFCAA|2064-04-30|1971|8575|658|2064|3|4|30|2|2064|658|8575|Wednesday|2064Q2|N|N|N|2475012|2475102|2474675|2474950|N|N|N|N|N| +2475042|AAAAAAAACCEMFCAA|2064-05-01|1972|8575|658|2064|4|5|1|2|2064|658|8575|Thursday|2064Q2|N|N|N|2475042|2475162|2474676|2474951|N|N|N|N|N| +2475043|AAAAAAAADCEMFCAA|2064-05-02|1972|8575|658|2064|5|5|2|2|2064|658|8575|Friday|2064Q2|N|Y|N|2475042|2475162|2474677|2474952|N|N|N|N|N| +2475044|AAAAAAAAECEMFCAA|2064-05-03|1972|8575|658|2064|6|5|3|2|2064|658|8575|Saturday|2064Q2|N|Y|N|2475042|2475162|2474678|2474953|N|N|N|N|N| +2475045|AAAAAAAAFCEMFCAA|2064-05-04|1972|8575|658|2064|0|5|4|2|2064|658|8575|Sunday|2064Q2|N|N|N|2475042|2475162|2474679|2474954|N|N|N|N|N| +2475046|AAAAAAAAGCEMFCAA|2064-05-05|1972|8575|658|2064|1|5|5|2|2064|658|8575|Monday|2064Q2|N|N|N|2475042|2475162|2474680|2474955|N|N|N|N|N| +2475047|AAAAAAAAHCEMFCAA|2064-05-06|1972|8576|658|2064|2|5|6|2|2064|658|8576|Tuesday|2064Q2|N|N|N|2475042|2475162|2474681|2474956|N|N|N|N|N| +2475048|AAAAAAAAICEMFCAA|2064-05-07|1972|8576|658|2064|3|5|7|2|2064|658|8576|Wednesday|2064Q2|N|N|N|2475042|2475162|2474682|2474957|N|N|N|N|N| +2475049|AAAAAAAAJCEMFCAA|2064-05-08|1972|8576|658|2064|4|5|8|2|2064|658|8576|Thursday|2064Q2|N|N|N|2475042|2475162|2474683|2474958|N|N|N|N|N| +2475050|AAAAAAAAKCEMFCAA|2064-05-09|1972|8576|658|2064|5|5|9|2|2064|658|8576|Friday|2064Q2|N|Y|N|2475042|2475162|2474684|2474959|N|N|N|N|N| +2475051|AAAAAAAALCEMFCAA|2064-05-10|1972|8576|658|2064|6|5|10|2|2064|658|8576|Saturday|2064Q2|N|Y|N|2475042|2475162|2474685|2474960|N|N|N|N|N| +2475052|AAAAAAAAMCEMFCAA|2064-05-11|1972|8576|658|2064|0|5|11|2|2064|658|8576|Sunday|2064Q2|N|N|N|2475042|2475162|2474686|2474961|N|N|N|N|N| +2475053|AAAAAAAANCEMFCAA|2064-05-12|1972|8576|658|2064|1|5|12|2|2064|658|8576|Monday|2064Q2|N|N|N|2475042|2475162|2474687|2474962|N|N|N|N|N| +2475054|AAAAAAAAOCEMFCAA|2064-05-13|1972|8577|658|2064|2|5|13|2|2064|658|8577|Tuesday|2064Q2|N|N|N|2475042|2475162|2474688|2474963|N|N|N|N|N| +2475055|AAAAAAAAPCEMFCAA|2064-05-14|1972|8577|658|2064|3|5|14|2|2064|658|8577|Wednesday|2064Q2|N|N|N|2475042|2475162|2474689|2474964|N|N|N|N|N| +2475056|AAAAAAAAADEMFCAA|2064-05-15|1972|8577|658|2064|4|5|15|2|2064|658|8577|Thursday|2064Q2|N|N|N|2475042|2475162|2474690|2474965|N|N|N|N|N| +2475057|AAAAAAAABDEMFCAA|2064-05-16|1972|8577|658|2064|5|5|16|2|2064|658|8577|Friday|2064Q2|N|Y|N|2475042|2475162|2474691|2474966|N|N|N|N|N| +2475058|AAAAAAAACDEMFCAA|2064-05-17|1972|8577|658|2064|6|5|17|2|2064|658|8577|Saturday|2064Q2|N|Y|N|2475042|2475162|2474692|2474967|N|N|N|N|N| +2475059|AAAAAAAADDEMFCAA|2064-05-18|1972|8577|658|2064|0|5|18|2|2064|658|8577|Sunday|2064Q2|N|N|N|2475042|2475162|2474693|2474968|N|N|N|N|N| +2475060|AAAAAAAAEDEMFCAA|2064-05-19|1972|8577|658|2064|1|5|19|2|2064|658|8577|Monday|2064Q2|N|N|N|2475042|2475162|2474694|2474969|N|N|N|N|N| +2475061|AAAAAAAAFDEMFCAA|2064-05-20|1972|8578|658|2064|2|5|20|2|2064|658|8578|Tuesday|2064Q2|N|N|N|2475042|2475162|2474695|2474970|N|N|N|N|N| +2475062|AAAAAAAAGDEMFCAA|2064-05-21|1972|8578|658|2064|3|5|21|2|2064|658|8578|Wednesday|2064Q2|N|N|N|2475042|2475162|2474696|2474971|N|N|N|N|N| +2475063|AAAAAAAAHDEMFCAA|2064-05-22|1972|8578|658|2064|4|5|22|2|2064|658|8578|Thursday|2064Q2|N|N|N|2475042|2475162|2474697|2474972|N|N|N|N|N| +2475064|AAAAAAAAIDEMFCAA|2064-05-23|1972|8578|658|2064|5|5|23|2|2064|658|8578|Friday|2064Q2|N|Y|N|2475042|2475162|2474698|2474973|N|N|N|N|N| +2475065|AAAAAAAAJDEMFCAA|2064-05-24|1972|8578|658|2064|6|5|24|2|2064|658|8578|Saturday|2064Q2|N|Y|N|2475042|2475162|2474699|2474974|N|N|N|N|N| +2475066|AAAAAAAAKDEMFCAA|2064-05-25|1972|8578|658|2064|0|5|25|2|2064|658|8578|Sunday|2064Q2|N|N|N|2475042|2475162|2474700|2474975|N|N|N|N|N| +2475067|AAAAAAAALDEMFCAA|2064-05-26|1972|8578|658|2064|1|5|26|2|2064|658|8578|Monday|2064Q2|N|N|N|2475042|2475162|2474701|2474976|N|N|N|N|N| +2475068|AAAAAAAAMDEMFCAA|2064-05-27|1972|8579|658|2064|2|5|27|2|2064|658|8579|Tuesday|2064Q2|N|N|N|2475042|2475162|2474702|2474977|N|N|N|N|N| +2475069|AAAAAAAANDEMFCAA|2064-05-28|1972|8579|658|2064|3|5|28|2|2064|658|8579|Wednesday|2064Q2|N|N|N|2475042|2475162|2474703|2474978|N|N|N|N|N| +2475070|AAAAAAAAODEMFCAA|2064-05-29|1972|8579|658|2064|4|5|29|2|2064|658|8579|Thursday|2064Q2|N|N|N|2475042|2475162|2474704|2474979|N|N|N|N|N| +2475071|AAAAAAAAPDEMFCAA|2064-05-30|1972|8579|658|2064|5|5|30|2|2064|658|8579|Friday|2064Q2|N|Y|N|2475042|2475162|2474705|2474980|N|N|N|N|N| +2475072|AAAAAAAAAEEMFCAA|2064-05-31|1972|8579|658|2064|6|5|31|2|2064|658|8579|Saturday|2064Q2|N|Y|N|2475042|2475162|2474706|2474981|N|N|N|N|N| +2475073|AAAAAAAABEEMFCAA|2064-06-01|1973|8579|659|2064|0|6|1|2|2064|659|8579|Sunday|2064Q2|N|N|N|2475073|2475224|2474707|2474982|N|N|N|N|N| +2475074|AAAAAAAACEEMFCAA|2064-06-02|1973|8579|659|2064|1|6|2|2|2064|659|8579|Monday|2064Q2|N|N|N|2475073|2475224|2474708|2474983|N|N|N|N|N| +2475075|AAAAAAAADEEMFCAA|2064-06-03|1973|8580|659|2064|2|6|3|2|2064|659|8580|Tuesday|2064Q2|N|N|N|2475073|2475224|2474709|2474984|N|N|N|N|N| +2475076|AAAAAAAAEEEMFCAA|2064-06-04|1973|8580|659|2064|3|6|4|2|2064|659|8580|Wednesday|2064Q2|N|N|N|2475073|2475224|2474710|2474985|N|N|N|N|N| +2475077|AAAAAAAAFEEMFCAA|2064-06-05|1973|8580|659|2064|4|6|5|2|2064|659|8580|Thursday|2064Q2|N|N|N|2475073|2475224|2474711|2474986|N|N|N|N|N| +2475078|AAAAAAAAGEEMFCAA|2064-06-06|1973|8580|659|2064|5|6|6|2|2064|659|8580|Friday|2064Q2|N|Y|N|2475073|2475224|2474712|2474987|N|N|N|N|N| +2475079|AAAAAAAAHEEMFCAA|2064-06-07|1973|8580|659|2064|6|6|7|2|2064|659|8580|Saturday|2064Q2|N|Y|N|2475073|2475224|2474713|2474988|N|N|N|N|N| +2475080|AAAAAAAAIEEMFCAA|2064-06-08|1973|8580|659|2064|0|6|8|2|2064|659|8580|Sunday|2064Q2|N|N|N|2475073|2475224|2474714|2474989|N|N|N|N|N| +2475081|AAAAAAAAJEEMFCAA|2064-06-09|1973|8580|659|2064|1|6|9|2|2064|659|8580|Monday|2064Q2|N|N|N|2475073|2475224|2474715|2474990|N|N|N|N|N| +2475082|AAAAAAAAKEEMFCAA|2064-06-10|1973|8581|659|2064|2|6|10|2|2064|659|8581|Tuesday|2064Q2|N|N|N|2475073|2475224|2474716|2474991|N|N|N|N|N| +2475083|AAAAAAAALEEMFCAA|2064-06-11|1973|8581|659|2064|3|6|11|2|2064|659|8581|Wednesday|2064Q2|N|N|N|2475073|2475224|2474717|2474992|N|N|N|N|N| +2475084|AAAAAAAAMEEMFCAA|2064-06-12|1973|8581|659|2064|4|6|12|2|2064|659|8581|Thursday|2064Q2|N|N|N|2475073|2475224|2474718|2474993|N|N|N|N|N| +2475085|AAAAAAAANEEMFCAA|2064-06-13|1973|8581|659|2064|5|6|13|2|2064|659|8581|Friday|2064Q2|N|Y|N|2475073|2475224|2474719|2474994|N|N|N|N|N| +2475086|AAAAAAAAOEEMFCAA|2064-06-14|1973|8581|659|2064|6|6|14|2|2064|659|8581|Saturday|2064Q2|N|Y|N|2475073|2475224|2474720|2474995|N|N|N|N|N| +2475087|AAAAAAAAPEEMFCAA|2064-06-15|1973|8581|659|2064|0|6|15|2|2064|659|8581|Sunday|2064Q2|N|N|N|2475073|2475224|2474721|2474996|N|N|N|N|N| +2475088|AAAAAAAAAFEMFCAA|2064-06-16|1973|8581|659|2064|1|6|16|2|2064|659|8581|Monday|2064Q2|N|N|N|2475073|2475224|2474722|2474997|N|N|N|N|N| +2475089|AAAAAAAABFEMFCAA|2064-06-17|1973|8582|659|2064|2|6|17|2|2064|659|8582|Tuesday|2064Q2|N|N|N|2475073|2475224|2474723|2474998|N|N|N|N|N| +2475090|AAAAAAAACFEMFCAA|2064-06-18|1973|8582|659|2064|3|6|18|2|2064|659|8582|Wednesday|2064Q2|N|N|N|2475073|2475224|2474724|2474999|N|N|N|N|N| +2475091|AAAAAAAADFEMFCAA|2064-06-19|1973|8582|659|2064|4|6|19|2|2064|659|8582|Thursday|2064Q2|N|N|N|2475073|2475224|2474725|2475000|N|N|N|N|N| +2475092|AAAAAAAAEFEMFCAA|2064-06-20|1973|8582|659|2064|5|6|20|2|2064|659|8582|Friday|2064Q2|N|Y|N|2475073|2475224|2474726|2475001|N|N|N|N|N| +2475093|AAAAAAAAFFEMFCAA|2064-06-21|1973|8582|659|2064|6|6|21|2|2064|659|8582|Saturday|2064Q2|N|Y|N|2475073|2475224|2474727|2475002|N|N|N|N|N| +2475094|AAAAAAAAGFEMFCAA|2064-06-22|1973|8582|659|2064|0|6|22|2|2064|659|8582|Sunday|2064Q2|N|N|N|2475073|2475224|2474728|2475003|N|N|N|N|N| +2475095|AAAAAAAAHFEMFCAA|2064-06-23|1973|8582|659|2064|1|6|23|2|2064|659|8582|Monday|2064Q2|N|N|N|2475073|2475224|2474729|2475004|N|N|N|N|N| +2475096|AAAAAAAAIFEMFCAA|2064-06-24|1973|8583|659|2064|2|6|24|2|2064|659|8583|Tuesday|2064Q2|N|N|N|2475073|2475224|2474730|2475005|N|N|N|N|N| +2475097|AAAAAAAAJFEMFCAA|2064-06-25|1973|8583|659|2064|3|6|25|2|2064|659|8583|Wednesday|2064Q2|N|N|N|2475073|2475224|2474731|2475006|N|N|N|N|N| +2475098|AAAAAAAAKFEMFCAA|2064-06-26|1973|8583|659|2064|4|6|26|2|2064|659|8583|Thursday|2064Q2|N|N|N|2475073|2475224|2474732|2475007|N|N|N|N|N| +2475099|AAAAAAAALFEMFCAA|2064-06-27|1973|8583|659|2064|5|6|27|2|2064|659|8583|Friday|2064Q2|N|Y|N|2475073|2475224|2474733|2475008|N|N|N|N|N| +2475100|AAAAAAAAMFEMFCAA|2064-06-28|1973|8583|659|2064|6|6|28|2|2064|659|8583|Saturday|2064Q2|N|Y|N|2475073|2475224|2474734|2475009|N|N|N|N|N| +2475101|AAAAAAAANFEMFCAA|2064-06-29|1973|8583|659|2064|0|6|29|2|2064|659|8583|Sunday|2064Q2|N|N|N|2475073|2475224|2474735|2475010|N|N|N|N|N| +2475102|AAAAAAAAOFEMFCAA|2064-06-30|1973|8583|659|2064|1|6|30|2|2064|659|8583|Monday|2064Q2|N|N|N|2475073|2475224|2474736|2475011|N|N|N|N|N| +2475103|AAAAAAAAPFEMFCAA|2064-07-01|1974|8584|659|2064|2|7|1|3|2064|659|8584|Tuesday|2064Q3|N|N|N|2475103|2475284|2474737|2475012|N|N|N|N|N| +2475104|AAAAAAAAAGEMFCAA|2064-07-02|1974|8584|659|2064|3|7|2|3|2064|659|8584|Wednesday|2064Q3|N|N|N|2475103|2475284|2474738|2475013|N|N|N|N|N| +2475105|AAAAAAAABGEMFCAA|2064-07-03|1974|8584|659|2064|4|7|3|3|2064|659|8584|Thursday|2064Q3|N|N|N|2475103|2475284|2474739|2475014|N|N|N|N|N| +2475106|AAAAAAAACGEMFCAA|2064-07-04|1974|8584|659|2064|5|7|4|3|2064|659|8584|Friday|2064Q3|Y|Y|N|2475103|2475284|2474740|2475015|N|N|N|N|N| +2475107|AAAAAAAADGEMFCAA|2064-07-05|1974|8584|659|2064|6|7|5|3|2064|659|8584|Saturday|2064Q3|N|Y|Y|2475103|2475284|2474741|2475016|N|N|N|N|N| +2475108|AAAAAAAAEGEMFCAA|2064-07-06|1974|8584|659|2064|0|7|6|3|2064|659|8584|Sunday|2064Q3|N|N|N|2475103|2475284|2474742|2475017|N|N|N|N|N| +2475109|AAAAAAAAFGEMFCAA|2064-07-07|1974|8584|659|2064|1|7|7|3|2064|659|8584|Monday|2064Q3|N|N|N|2475103|2475284|2474743|2475018|N|N|N|N|N| +2475110|AAAAAAAAGGEMFCAA|2064-07-08|1974|8585|659|2064|2|7|8|3|2064|659|8585|Tuesday|2064Q3|N|N|N|2475103|2475284|2474744|2475019|N|N|N|N|N| +2475111|AAAAAAAAHGEMFCAA|2064-07-09|1974|8585|659|2064|3|7|9|3|2064|659|8585|Wednesday|2064Q3|N|N|N|2475103|2475284|2474745|2475020|N|N|N|N|N| +2475112|AAAAAAAAIGEMFCAA|2064-07-10|1974|8585|659|2064|4|7|10|3|2064|659|8585|Thursday|2064Q3|N|N|N|2475103|2475284|2474746|2475021|N|N|N|N|N| +2475113|AAAAAAAAJGEMFCAA|2064-07-11|1974|8585|659|2064|5|7|11|3|2064|659|8585|Friday|2064Q3|N|Y|N|2475103|2475284|2474747|2475022|N|N|N|N|N| +2475114|AAAAAAAAKGEMFCAA|2064-07-12|1974|8585|659|2064|6|7|12|3|2064|659|8585|Saturday|2064Q3|N|Y|N|2475103|2475284|2474748|2475023|N|N|N|N|N| +2475115|AAAAAAAALGEMFCAA|2064-07-13|1974|8585|659|2064|0|7|13|3|2064|659|8585|Sunday|2064Q3|N|N|N|2475103|2475284|2474749|2475024|N|N|N|N|N| +2475116|AAAAAAAAMGEMFCAA|2064-07-14|1974|8585|659|2064|1|7|14|3|2064|659|8585|Monday|2064Q3|N|N|N|2475103|2475284|2474750|2475025|N|N|N|N|N| +2475117|AAAAAAAANGEMFCAA|2064-07-15|1974|8586|659|2064|2|7|15|3|2064|659|8586|Tuesday|2064Q3|N|N|N|2475103|2475284|2474751|2475026|N|N|N|N|N| +2475118|AAAAAAAAOGEMFCAA|2064-07-16|1974|8586|659|2064|3|7|16|3|2064|659|8586|Wednesday|2064Q3|N|N|N|2475103|2475284|2474752|2475027|N|N|N|N|N| +2475119|AAAAAAAAPGEMFCAA|2064-07-17|1974|8586|659|2064|4|7|17|3|2064|659|8586|Thursday|2064Q3|N|N|N|2475103|2475284|2474753|2475028|N|N|N|N|N| +2475120|AAAAAAAAAHEMFCAA|2064-07-18|1974|8586|659|2064|5|7|18|3|2064|659|8586|Friday|2064Q3|N|Y|N|2475103|2475284|2474754|2475029|N|N|N|N|N| +2475121|AAAAAAAABHEMFCAA|2064-07-19|1974|8586|659|2064|6|7|19|3|2064|659|8586|Saturday|2064Q3|N|Y|N|2475103|2475284|2474755|2475030|N|N|N|N|N| +2475122|AAAAAAAACHEMFCAA|2064-07-20|1974|8586|659|2064|0|7|20|3|2064|659|8586|Sunday|2064Q3|N|N|N|2475103|2475284|2474756|2475031|N|N|N|N|N| +2475123|AAAAAAAADHEMFCAA|2064-07-21|1974|8586|659|2064|1|7|21|3|2064|659|8586|Monday|2064Q3|N|N|N|2475103|2475284|2474757|2475032|N|N|N|N|N| +2475124|AAAAAAAAEHEMFCAA|2064-07-22|1974|8587|659|2064|2|7|22|3|2064|659|8587|Tuesday|2064Q3|N|N|N|2475103|2475284|2474758|2475033|N|N|N|N|N| +2475125|AAAAAAAAFHEMFCAA|2064-07-23|1974|8587|659|2064|3|7|23|3|2064|659|8587|Wednesday|2064Q3|N|N|N|2475103|2475284|2474759|2475034|N|N|N|N|N| +2475126|AAAAAAAAGHEMFCAA|2064-07-24|1974|8587|659|2064|4|7|24|3|2064|659|8587|Thursday|2064Q3|N|N|N|2475103|2475284|2474760|2475035|N|N|N|N|N| +2475127|AAAAAAAAHHEMFCAA|2064-07-25|1974|8587|659|2064|5|7|25|3|2064|659|8587|Friday|2064Q3|N|Y|N|2475103|2475284|2474761|2475036|N|N|N|N|N| +2475128|AAAAAAAAIHEMFCAA|2064-07-26|1974|8587|659|2064|6|7|26|3|2064|659|8587|Saturday|2064Q3|N|Y|N|2475103|2475284|2474762|2475037|N|N|N|N|N| +2475129|AAAAAAAAJHEMFCAA|2064-07-27|1974|8587|659|2064|0|7|27|3|2064|659|8587|Sunday|2064Q3|N|N|N|2475103|2475284|2474763|2475038|N|N|N|N|N| +2475130|AAAAAAAAKHEMFCAA|2064-07-28|1974|8587|659|2064|1|7|28|3|2064|659|8587|Monday|2064Q3|N|N|N|2475103|2475284|2474764|2475039|N|N|N|N|N| +2475131|AAAAAAAALHEMFCAA|2064-07-29|1974|8588|659|2064|2|7|29|3|2064|659|8588|Tuesday|2064Q3|N|N|N|2475103|2475284|2474765|2475040|N|N|N|N|N| +2475132|AAAAAAAAMHEMFCAA|2064-07-30|1974|8588|659|2064|3|7|30|3|2064|659|8588|Wednesday|2064Q3|N|N|N|2475103|2475284|2474766|2475041|N|N|N|N|N| +2475133|AAAAAAAANHEMFCAA|2064-07-31|1974|8588|659|2064|4|7|31|3|2064|659|8588|Thursday|2064Q3|N|N|N|2475103|2475284|2474767|2475042|N|N|N|N|N| +2475134|AAAAAAAAOHEMFCAA|2064-08-01|1975|8588|659|2064|5|8|1|3|2064|659|8588|Friday|2064Q3|N|Y|N|2475134|2475346|2474768|2475043|N|N|N|N|N| +2475135|AAAAAAAAPHEMFCAA|2064-08-02|1975|8588|659|2064|6|8|2|3|2064|659|8588|Saturday|2064Q3|N|Y|N|2475134|2475346|2474769|2475044|N|N|N|N|N| +2475136|AAAAAAAAAIEMFCAA|2064-08-03|1975|8588|659|2064|0|8|3|3|2064|659|8588|Sunday|2064Q3|N|N|N|2475134|2475346|2474770|2475045|N|N|N|N|N| +2475137|AAAAAAAABIEMFCAA|2064-08-04|1975|8588|659|2064|1|8|4|3|2064|659|8588|Monday|2064Q3|N|N|N|2475134|2475346|2474771|2475046|N|N|N|N|N| +2475138|AAAAAAAACIEMFCAA|2064-08-05|1975|8589|659|2064|2|8|5|3|2064|659|8589|Tuesday|2064Q3|N|N|N|2475134|2475346|2474772|2475047|N|N|N|N|N| +2475139|AAAAAAAADIEMFCAA|2064-08-06|1975|8589|659|2064|3|8|6|3|2064|659|8589|Wednesday|2064Q3|N|N|N|2475134|2475346|2474773|2475048|N|N|N|N|N| +2475140|AAAAAAAAEIEMFCAA|2064-08-07|1975|8589|659|2064|4|8|7|3|2064|659|8589|Thursday|2064Q3|N|N|N|2475134|2475346|2474774|2475049|N|N|N|N|N| +2475141|AAAAAAAAFIEMFCAA|2064-08-08|1975|8589|659|2064|5|8|8|3|2064|659|8589|Friday|2064Q3|N|Y|N|2475134|2475346|2474775|2475050|N|N|N|N|N| +2475142|AAAAAAAAGIEMFCAA|2064-08-09|1975|8589|659|2064|6|8|9|3|2064|659|8589|Saturday|2064Q3|N|Y|N|2475134|2475346|2474776|2475051|N|N|N|N|N| +2475143|AAAAAAAAHIEMFCAA|2064-08-10|1975|8589|659|2064|0|8|10|3|2064|659|8589|Sunday|2064Q3|N|N|N|2475134|2475346|2474777|2475052|N|N|N|N|N| +2475144|AAAAAAAAIIEMFCAA|2064-08-11|1975|8589|659|2064|1|8|11|3|2064|659|8589|Monday|2064Q3|N|N|N|2475134|2475346|2474778|2475053|N|N|N|N|N| +2475145|AAAAAAAAJIEMFCAA|2064-08-12|1975|8590|659|2064|2|8|12|3|2064|659|8590|Tuesday|2064Q3|N|N|N|2475134|2475346|2474779|2475054|N|N|N|N|N| +2475146|AAAAAAAAKIEMFCAA|2064-08-13|1975|8590|659|2064|3|8|13|3|2064|659|8590|Wednesday|2064Q3|N|N|N|2475134|2475346|2474780|2475055|N|N|N|N|N| +2475147|AAAAAAAALIEMFCAA|2064-08-14|1975|8590|659|2064|4|8|14|3|2064|659|8590|Thursday|2064Q3|N|N|N|2475134|2475346|2474781|2475056|N|N|N|N|N| +2475148|AAAAAAAAMIEMFCAA|2064-08-15|1975|8590|659|2064|5|8|15|3|2064|659|8590|Friday|2064Q3|N|Y|N|2475134|2475346|2474782|2475057|N|N|N|N|N| +2475149|AAAAAAAANIEMFCAA|2064-08-16|1975|8590|659|2064|6|8|16|3|2064|659|8590|Saturday|2064Q3|N|Y|N|2475134|2475346|2474783|2475058|N|N|N|N|N| +2475150|AAAAAAAAOIEMFCAA|2064-08-17|1975|8590|659|2064|0|8|17|3|2064|659|8590|Sunday|2064Q3|N|N|N|2475134|2475346|2474784|2475059|N|N|N|N|N| +2475151|AAAAAAAAPIEMFCAA|2064-08-18|1975|8590|659|2064|1|8|18|3|2064|659|8590|Monday|2064Q3|N|N|N|2475134|2475346|2474785|2475060|N|N|N|N|N| +2475152|AAAAAAAAAJEMFCAA|2064-08-19|1975|8591|659|2064|2|8|19|3|2064|659|8591|Tuesday|2064Q3|N|N|N|2475134|2475346|2474786|2475061|N|N|N|N|N| +2475153|AAAAAAAABJEMFCAA|2064-08-20|1975|8591|659|2064|3|8|20|3|2064|659|8591|Wednesday|2064Q3|N|N|N|2475134|2475346|2474787|2475062|N|N|N|N|N| +2475154|AAAAAAAACJEMFCAA|2064-08-21|1975|8591|659|2064|4|8|21|3|2064|659|8591|Thursday|2064Q3|N|N|N|2475134|2475346|2474788|2475063|N|N|N|N|N| +2475155|AAAAAAAADJEMFCAA|2064-08-22|1975|8591|659|2064|5|8|22|3|2064|659|8591|Friday|2064Q3|N|Y|N|2475134|2475346|2474789|2475064|N|N|N|N|N| +2475156|AAAAAAAAEJEMFCAA|2064-08-23|1975|8591|659|2064|6|8|23|3|2064|659|8591|Saturday|2064Q3|N|Y|N|2475134|2475346|2474790|2475065|N|N|N|N|N| +2475157|AAAAAAAAFJEMFCAA|2064-08-24|1975|8591|659|2064|0|8|24|3|2064|659|8591|Sunday|2064Q3|N|N|N|2475134|2475346|2474791|2475066|N|N|N|N|N| +2475158|AAAAAAAAGJEMFCAA|2064-08-25|1975|8591|659|2064|1|8|25|3|2064|659|8591|Monday|2064Q3|N|N|N|2475134|2475346|2474792|2475067|N|N|N|N|N| +2475159|AAAAAAAAHJEMFCAA|2064-08-26|1975|8592|659|2064|2|8|26|3|2064|659|8592|Tuesday|2064Q3|N|N|N|2475134|2475346|2474793|2475068|N|N|N|N|N| +2475160|AAAAAAAAIJEMFCAA|2064-08-27|1975|8592|659|2064|3|8|27|3|2064|659|8592|Wednesday|2064Q3|N|N|N|2475134|2475346|2474794|2475069|N|N|N|N|N| +2475161|AAAAAAAAJJEMFCAA|2064-08-28|1975|8592|659|2064|4|8|28|3|2064|659|8592|Thursday|2064Q3|N|N|N|2475134|2475346|2474795|2475070|N|N|N|N|N| +2475162|AAAAAAAAKJEMFCAA|2064-08-29|1975|8592|659|2064|5|8|29|3|2064|659|8592|Friday|2064Q3|N|Y|N|2475134|2475346|2474796|2475071|N|N|N|N|N| +2475163|AAAAAAAALJEMFCAA|2064-08-30|1975|8592|659|2064|6|8|30|3|2064|659|8592|Saturday|2064Q3|N|Y|N|2475134|2475346|2474797|2475072|N|N|N|N|N| +2475164|AAAAAAAAMJEMFCAA|2064-08-31|1975|8592|659|2064|0|8|31|3|2064|659|8592|Sunday|2064Q3|N|N|N|2475134|2475346|2474798|2475073|N|N|N|N|N| +2475165|AAAAAAAANJEMFCAA|2064-09-01|1976|8592|660|2064|1|9|1|3|2064|660|8592|Monday|2064Q3|N|N|N|2475165|2475408|2474799|2475074|N|N|N|N|N| +2475166|AAAAAAAAOJEMFCAA|2064-09-02|1976|8593|660|2064|2|9|2|3|2064|660|8593|Tuesday|2064Q3|N|N|N|2475165|2475408|2474800|2475075|N|N|N|N|N| +2475167|AAAAAAAAPJEMFCAA|2064-09-03|1976|8593|660|2064|3|9|3|3|2064|660|8593|Wednesday|2064Q3|N|N|N|2475165|2475408|2474801|2475076|N|N|N|N|N| +2475168|AAAAAAAAAKEMFCAA|2064-09-04|1976|8593|660|2064|4|9|4|3|2064|660|8593|Thursday|2064Q3|N|N|N|2475165|2475408|2474802|2475077|N|N|N|N|N| +2475169|AAAAAAAABKEMFCAA|2064-09-05|1976|8593|660|2064|5|9|5|3|2064|660|8593|Friday|2064Q3|N|Y|N|2475165|2475408|2474803|2475078|N|N|N|N|N| +2475170|AAAAAAAACKEMFCAA|2064-09-06|1976|8593|660|2064|6|9|6|3|2064|660|8593|Saturday|2064Q3|N|Y|N|2475165|2475408|2474804|2475079|N|N|N|N|N| +2475171|AAAAAAAADKEMFCAA|2064-09-07|1976|8593|660|2064|0|9|7|3|2064|660|8593|Sunday|2064Q3|N|N|N|2475165|2475408|2474805|2475080|N|N|N|N|N| +2475172|AAAAAAAAEKEMFCAA|2064-09-08|1976|8593|660|2064|1|9|8|3|2064|660|8593|Monday|2064Q3|N|N|N|2475165|2475408|2474806|2475081|N|N|N|N|N| +2475173|AAAAAAAAFKEMFCAA|2064-09-09|1976|8594|660|2064|2|9|9|3|2064|660|8594|Tuesday|2064Q3|N|N|N|2475165|2475408|2474807|2475082|N|N|N|N|N| +2475174|AAAAAAAAGKEMFCAA|2064-09-10|1976|8594|660|2064|3|9|10|3|2064|660|8594|Wednesday|2064Q3|N|N|N|2475165|2475408|2474808|2475083|N|N|N|N|N| +2475175|AAAAAAAAHKEMFCAA|2064-09-11|1976|8594|660|2064|4|9|11|3|2064|660|8594|Thursday|2064Q3|N|N|N|2475165|2475408|2474809|2475084|N|N|N|N|N| +2475176|AAAAAAAAIKEMFCAA|2064-09-12|1976|8594|660|2064|5|9|12|3|2064|660|8594|Friday|2064Q3|N|Y|N|2475165|2475408|2474810|2475085|N|N|N|N|N| +2475177|AAAAAAAAJKEMFCAA|2064-09-13|1976|8594|660|2064|6|9|13|3|2064|660|8594|Saturday|2064Q3|N|Y|N|2475165|2475408|2474811|2475086|N|N|N|N|N| +2475178|AAAAAAAAKKEMFCAA|2064-09-14|1976|8594|660|2064|0|9|14|3|2064|660|8594|Sunday|2064Q3|N|N|N|2475165|2475408|2474812|2475087|N|N|N|N|N| +2475179|AAAAAAAALKEMFCAA|2064-09-15|1976|8594|660|2064|1|9|15|3|2064|660|8594|Monday|2064Q3|N|N|N|2475165|2475408|2474813|2475088|N|N|N|N|N| +2475180|AAAAAAAAMKEMFCAA|2064-09-16|1976|8595|660|2064|2|9|16|3|2064|660|8595|Tuesday|2064Q3|N|N|N|2475165|2475408|2474814|2475089|N|N|N|N|N| +2475181|AAAAAAAANKEMFCAA|2064-09-17|1976|8595|660|2064|3|9|17|3|2064|660|8595|Wednesday|2064Q3|N|N|N|2475165|2475408|2474815|2475090|N|N|N|N|N| +2475182|AAAAAAAAOKEMFCAA|2064-09-18|1976|8595|660|2064|4|9|18|3|2064|660|8595|Thursday|2064Q3|N|N|N|2475165|2475408|2474816|2475091|N|N|N|N|N| +2475183|AAAAAAAAPKEMFCAA|2064-09-19|1976|8595|660|2064|5|9|19|3|2064|660|8595|Friday|2064Q3|N|Y|N|2475165|2475408|2474817|2475092|N|N|N|N|N| +2475184|AAAAAAAAALEMFCAA|2064-09-20|1976|8595|660|2064|6|9|20|3|2064|660|8595|Saturday|2064Q3|N|Y|N|2475165|2475408|2474818|2475093|N|N|N|N|N| +2475185|AAAAAAAABLEMFCAA|2064-09-21|1976|8595|660|2064|0|9|21|3|2064|660|8595|Sunday|2064Q3|N|N|N|2475165|2475408|2474819|2475094|N|N|N|N|N| +2475186|AAAAAAAACLEMFCAA|2064-09-22|1976|8595|660|2064|1|9|22|3|2064|660|8595|Monday|2064Q3|N|N|N|2475165|2475408|2474820|2475095|N|N|N|N|N| +2475187|AAAAAAAADLEMFCAA|2064-09-23|1976|8596|660|2064|2|9|23|3|2064|660|8596|Tuesday|2064Q3|N|N|N|2475165|2475408|2474821|2475096|N|N|N|N|N| +2475188|AAAAAAAAELEMFCAA|2064-09-24|1976|8596|660|2064|3|9|24|3|2064|660|8596|Wednesday|2064Q3|N|N|N|2475165|2475408|2474822|2475097|N|N|N|N|N| +2475189|AAAAAAAAFLEMFCAA|2064-09-25|1976|8596|660|2064|4|9|25|3|2064|660|8596|Thursday|2064Q3|N|N|N|2475165|2475408|2474823|2475098|N|N|N|N|N| +2475190|AAAAAAAAGLEMFCAA|2064-09-26|1976|8596|660|2064|5|9|26|3|2064|660|8596|Friday|2064Q3|N|Y|N|2475165|2475408|2474824|2475099|N|N|N|N|N| +2475191|AAAAAAAAHLEMFCAA|2064-09-27|1976|8596|660|2064|6|9|27|3|2064|660|8596|Saturday|2064Q3|N|Y|N|2475165|2475408|2474825|2475100|N|N|N|N|N| +2475192|AAAAAAAAILEMFCAA|2064-09-28|1976|8596|660|2064|0|9|28|3|2064|660|8596|Sunday|2064Q3|N|N|N|2475165|2475408|2474826|2475101|N|N|N|N|N| +2475193|AAAAAAAAJLEMFCAA|2064-09-29|1976|8596|660|2064|1|9|29|3|2064|660|8596|Monday|2064Q3|N|N|N|2475165|2475408|2474827|2475102|N|N|N|N|N| +2475194|AAAAAAAAKLEMFCAA|2064-09-30|1976|8597|660|2064|2|9|30|3|2064|660|8597|Tuesday|2064Q3|N|N|N|2475165|2475408|2474828|2475103|N|N|N|N|N| +2475195|AAAAAAAALLEMFCAA|2064-10-01|1977|8597|660|2064|3|10|1|4|2064|660|8597|Wednesday|2064Q4|N|N|N|2475195|2475468|2474829|2475103|N|N|N|N|N| +2475196|AAAAAAAAMLEMFCAA|2064-10-02|1977|8597|660|2064|4|10|2|4|2064|660|8597|Thursday|2064Q4|N|N|N|2475195|2475468|2474830|2475104|N|N|N|N|N| +2475197|AAAAAAAANLEMFCAA|2064-10-03|1977|8597|660|2064|5|10|3|4|2064|660|8597|Friday|2064Q4|N|Y|N|2475195|2475468|2474831|2475105|N|N|N|N|N| +2475198|AAAAAAAAOLEMFCAA|2064-10-04|1977|8597|660|2064|6|10|4|4|2064|660|8597|Saturday|2064Q4|N|Y|N|2475195|2475468|2474832|2475106|N|N|N|N|N| +2475199|AAAAAAAAPLEMFCAA|2064-10-05|1977|8597|660|2064|0|10|5|4|2064|660|8597|Sunday|2064Q4|N|N|N|2475195|2475468|2474833|2475107|N|N|N|N|N| +2475200|AAAAAAAAAMEMFCAA|2064-10-06|1977|8597|660|2064|1|10|6|4|2064|660|8597|Monday|2064Q4|N|N|N|2475195|2475468|2474834|2475108|N|N|N|N|N| +2475201|AAAAAAAABMEMFCAA|2064-10-07|1977|8598|660|2064|2|10|7|4|2064|660|8598|Tuesday|2064Q4|N|N|N|2475195|2475468|2474835|2475109|N|N|N|N|N| +2475202|AAAAAAAACMEMFCAA|2064-10-08|1977|8598|660|2064|3|10|8|4|2064|660|8598|Wednesday|2064Q4|N|N|N|2475195|2475468|2474836|2475110|N|N|N|N|N| +2475203|AAAAAAAADMEMFCAA|2064-10-09|1977|8598|660|2064|4|10|9|4|2064|660|8598|Thursday|2064Q4|N|N|N|2475195|2475468|2474837|2475111|N|N|N|N|N| +2475204|AAAAAAAAEMEMFCAA|2064-10-10|1977|8598|660|2064|5|10|10|4|2064|660|8598|Friday|2064Q4|N|Y|N|2475195|2475468|2474838|2475112|N|N|N|N|N| +2475205|AAAAAAAAFMEMFCAA|2064-10-11|1977|8598|660|2064|6|10|11|4|2064|660|8598|Saturday|2064Q4|N|Y|N|2475195|2475468|2474839|2475113|N|N|N|N|N| +2475206|AAAAAAAAGMEMFCAA|2064-10-12|1977|8598|660|2064|0|10|12|4|2064|660|8598|Sunday|2064Q4|N|N|N|2475195|2475468|2474840|2475114|N|N|N|N|N| +2475207|AAAAAAAAHMEMFCAA|2064-10-13|1977|8598|660|2064|1|10|13|4|2064|660|8598|Monday|2064Q4|N|N|N|2475195|2475468|2474841|2475115|N|N|N|N|N| +2475208|AAAAAAAAIMEMFCAA|2064-10-14|1977|8599|660|2064|2|10|14|4|2064|660|8599|Tuesday|2064Q4|N|N|N|2475195|2475468|2474842|2475116|N|N|N|N|N| +2475209|AAAAAAAAJMEMFCAA|2064-10-15|1977|8599|660|2064|3|10|15|4|2064|660|8599|Wednesday|2064Q4|N|N|N|2475195|2475468|2474843|2475117|N|N|N|N|N| +2475210|AAAAAAAAKMEMFCAA|2064-10-16|1977|8599|660|2064|4|10|16|4|2064|660|8599|Thursday|2064Q4|N|N|N|2475195|2475468|2474844|2475118|N|N|N|N|N| +2475211|AAAAAAAALMEMFCAA|2064-10-17|1977|8599|660|2064|5|10|17|4|2064|660|8599|Friday|2064Q4|N|Y|N|2475195|2475468|2474845|2475119|N|N|N|N|N| +2475212|AAAAAAAAMMEMFCAA|2064-10-18|1977|8599|660|2064|6|10|18|4|2064|660|8599|Saturday|2064Q4|N|Y|N|2475195|2475468|2474846|2475120|N|N|N|N|N| +2475213|AAAAAAAANMEMFCAA|2064-10-19|1977|8599|660|2064|0|10|19|4|2064|660|8599|Sunday|2064Q4|N|N|N|2475195|2475468|2474847|2475121|N|N|N|N|N| +2475214|AAAAAAAAOMEMFCAA|2064-10-20|1977|8599|660|2064|1|10|20|4|2064|660|8599|Monday|2064Q4|N|N|N|2475195|2475468|2474848|2475122|N|N|N|N|N| +2475215|AAAAAAAAPMEMFCAA|2064-10-21|1977|8600|660|2064|2|10|21|4|2064|660|8600|Tuesday|2064Q4|N|N|N|2475195|2475468|2474849|2475123|N|N|N|N|N| +2475216|AAAAAAAAANEMFCAA|2064-10-22|1977|8600|660|2064|3|10|22|4|2064|660|8600|Wednesday|2064Q4|N|N|N|2475195|2475468|2474850|2475124|N|N|N|N|N| +2475217|AAAAAAAABNEMFCAA|2064-10-23|1977|8600|660|2064|4|10|23|4|2064|660|8600|Thursday|2064Q4|N|N|N|2475195|2475468|2474851|2475125|N|N|N|N|N| +2475218|AAAAAAAACNEMFCAA|2064-10-24|1977|8600|660|2064|5|10|24|4|2064|660|8600|Friday|2064Q4|N|Y|N|2475195|2475468|2474852|2475126|N|N|N|N|N| +2475219|AAAAAAAADNEMFCAA|2064-10-25|1977|8600|660|2064|6|10|25|4|2064|660|8600|Saturday|2064Q4|N|Y|N|2475195|2475468|2474853|2475127|N|N|N|N|N| +2475220|AAAAAAAAENEMFCAA|2064-10-26|1977|8600|660|2064|0|10|26|4|2064|660|8600|Sunday|2064Q4|N|N|N|2475195|2475468|2474854|2475128|N|N|N|N|N| +2475221|AAAAAAAAFNEMFCAA|2064-10-27|1977|8600|660|2064|1|10|27|4|2064|660|8600|Monday|2064Q4|N|N|N|2475195|2475468|2474855|2475129|N|N|N|N|N| +2475222|AAAAAAAAGNEMFCAA|2064-10-28|1977|8601|660|2064|2|10|28|4|2064|660|8601|Tuesday|2064Q4|N|N|N|2475195|2475468|2474856|2475130|N|N|N|N|N| +2475223|AAAAAAAAHNEMFCAA|2064-10-29|1977|8601|660|2064|3|10|29|4|2064|660|8601|Wednesday|2064Q4|N|N|N|2475195|2475468|2474857|2475131|N|N|N|N|N| +2475224|AAAAAAAAINEMFCAA|2064-10-30|1977|8601|660|2064|4|10|30|4|2064|660|8601|Thursday|2064Q4|N|N|N|2475195|2475468|2474858|2475132|N|N|N|N|N| +2475225|AAAAAAAAJNEMFCAA|2064-10-31|1977|8601|660|2064|5|10|31|4|2064|660|8601|Friday|2064Q4|N|Y|N|2475195|2475468|2474859|2475133|N|N|N|N|N| +2475226|AAAAAAAAKNEMFCAA|2064-11-01|1978|8601|660|2064|6|11|1|4|2064|660|8601|Saturday|2064Q4|N|Y|N|2475226|2475530|2474860|2475134|N|N|N|N|N| +2475227|AAAAAAAALNEMFCAA|2064-11-02|1978|8601|660|2064|0|11|2|4|2064|660|8601|Sunday|2064Q4|N|N|N|2475226|2475530|2474861|2475135|N|N|N|N|N| +2475228|AAAAAAAAMNEMFCAA|2064-11-03|1978|8601|660|2064|1|11|3|4|2064|660|8601|Monday|2064Q4|N|N|N|2475226|2475530|2474862|2475136|N|N|N|N|N| +2475229|AAAAAAAANNEMFCAA|2064-11-04|1978|8602|660|2064|2|11|4|4|2064|660|8602|Tuesday|2064Q4|N|N|N|2475226|2475530|2474863|2475137|N|N|N|N|N| +2475230|AAAAAAAAONEMFCAA|2064-11-05|1978|8602|660|2064|3|11|5|4|2064|660|8602|Wednesday|2064Q4|N|N|N|2475226|2475530|2474864|2475138|N|N|N|N|N| +2475231|AAAAAAAAPNEMFCAA|2064-11-06|1978|8602|660|2064|4|11|6|4|2064|660|8602|Thursday|2064Q4|N|N|N|2475226|2475530|2474865|2475139|N|N|N|N|N| +2475232|AAAAAAAAAOEMFCAA|2064-11-07|1978|8602|660|2064|5|11|7|4|2064|660|8602|Friday|2064Q4|N|Y|N|2475226|2475530|2474866|2475140|N|N|N|N|N| +2475233|AAAAAAAABOEMFCAA|2064-11-08|1978|8602|660|2064|6|11|8|4|2064|660|8602|Saturday|2064Q4|N|Y|N|2475226|2475530|2474867|2475141|N|N|N|N|N| +2475234|AAAAAAAACOEMFCAA|2064-11-09|1978|8602|660|2064|0|11|9|4|2064|660|8602|Sunday|2064Q4|N|N|N|2475226|2475530|2474868|2475142|N|N|N|N|N| +2475235|AAAAAAAADOEMFCAA|2064-11-10|1978|8602|660|2064|1|11|10|4|2064|660|8602|Monday|2064Q4|N|N|N|2475226|2475530|2474869|2475143|N|N|N|N|N| +2475236|AAAAAAAAEOEMFCAA|2064-11-11|1978|8603|660|2064|2|11|11|4|2064|660|8603|Tuesday|2064Q4|N|N|N|2475226|2475530|2474870|2475144|N|N|N|N|N| +2475237|AAAAAAAAFOEMFCAA|2064-11-12|1978|8603|660|2064|3|11|12|4|2064|660|8603|Wednesday|2064Q4|N|N|N|2475226|2475530|2474871|2475145|N|N|N|N|N| +2475238|AAAAAAAAGOEMFCAA|2064-11-13|1978|8603|660|2064|4|11|13|4|2064|660|8603|Thursday|2064Q4|N|N|N|2475226|2475530|2474872|2475146|N|N|N|N|N| +2475239|AAAAAAAAHOEMFCAA|2064-11-14|1978|8603|660|2064|5|11|14|4|2064|660|8603|Friday|2064Q4|N|Y|N|2475226|2475530|2474873|2475147|N|N|N|N|N| +2475240|AAAAAAAAIOEMFCAA|2064-11-15|1978|8603|660|2064|6|11|15|4|2064|660|8603|Saturday|2064Q4|N|Y|N|2475226|2475530|2474874|2475148|N|N|N|N|N| +2475241|AAAAAAAAJOEMFCAA|2064-11-16|1978|8603|660|2064|0|11|16|4|2064|660|8603|Sunday|2064Q4|N|N|N|2475226|2475530|2474875|2475149|N|N|N|N|N| +2475242|AAAAAAAAKOEMFCAA|2064-11-17|1978|8603|660|2064|1|11|17|4|2064|660|8603|Monday|2064Q4|N|N|N|2475226|2475530|2474876|2475150|N|N|N|N|N| +2475243|AAAAAAAALOEMFCAA|2064-11-18|1978|8604|660|2064|2|11|18|4|2064|660|8604|Tuesday|2064Q4|N|N|N|2475226|2475530|2474877|2475151|N|N|N|N|N| +2475244|AAAAAAAAMOEMFCAA|2064-11-19|1978|8604|660|2064|3|11|19|4|2064|660|8604|Wednesday|2064Q4|N|N|N|2475226|2475530|2474878|2475152|N|N|N|N|N| +2475245|AAAAAAAANOEMFCAA|2064-11-20|1978|8604|660|2064|4|11|20|4|2064|660|8604|Thursday|2064Q4|N|N|N|2475226|2475530|2474879|2475153|N|N|N|N|N| +2475246|AAAAAAAAOOEMFCAA|2064-11-21|1978|8604|660|2064|5|11|21|4|2064|660|8604|Friday|2064Q4|N|Y|N|2475226|2475530|2474880|2475154|N|N|N|N|N| +2475247|AAAAAAAAPOEMFCAA|2064-11-22|1978|8604|660|2064|6|11|22|4|2064|660|8604|Saturday|2064Q4|N|Y|N|2475226|2475530|2474881|2475155|N|N|N|N|N| +2475248|AAAAAAAAAPEMFCAA|2064-11-23|1978|8604|660|2064|0|11|23|4|2064|660|8604|Sunday|2064Q4|N|N|N|2475226|2475530|2474882|2475156|N|N|N|N|N| +2475249|AAAAAAAABPEMFCAA|2064-11-24|1978|8604|660|2064|1|11|24|4|2064|660|8604|Monday|2064Q4|N|N|N|2475226|2475530|2474883|2475157|N|N|N|N|N| +2475250|AAAAAAAACPEMFCAA|2064-11-25|1978|8605|660|2064|2|11|25|4|2064|660|8605|Tuesday|2064Q4|N|N|N|2475226|2475530|2474884|2475158|N|N|N|N|N| +2475251|AAAAAAAADPEMFCAA|2064-11-26|1978|8605|660|2064|3|11|26|4|2064|660|8605|Wednesday|2064Q4|N|N|N|2475226|2475530|2474885|2475159|N|N|N|N|N| +2475252|AAAAAAAAEPEMFCAA|2064-11-27|1978|8605|660|2064|4|11|27|4|2064|660|8605|Thursday|2064Q4|N|N|N|2475226|2475530|2474886|2475160|N|N|N|N|N| +2475253|AAAAAAAAFPEMFCAA|2064-11-28|1978|8605|660|2064|5|11|28|4|2064|660|8605|Friday|2064Q4|N|Y|N|2475226|2475530|2474887|2475161|N|N|N|N|N| +2475254|AAAAAAAAGPEMFCAA|2064-11-29|1978|8605|660|2064|6|11|29|4|2064|660|8605|Saturday|2064Q4|N|Y|N|2475226|2475530|2474888|2475162|N|N|N|N|N| +2475255|AAAAAAAAHPEMFCAA|2064-11-30|1978|8605|660|2064|0|11|30|4|2064|660|8605|Sunday|2064Q4|N|N|N|2475226|2475530|2474889|2475163|N|N|N|N|N| +2475256|AAAAAAAAIPEMFCAA|2064-12-01|1979|8605|661|2064|1|12|1|4|2064|661|8605|Monday|2064Q4|N|N|N|2475256|2475590|2474890|2475164|N|N|N|N|N| +2475257|AAAAAAAAJPEMFCAA|2064-12-02|1979|8606|661|2064|2|12|2|4|2064|661|8606|Tuesday|2064Q4|N|N|N|2475256|2475590|2474891|2475165|N|N|N|N|N| +2475258|AAAAAAAAKPEMFCAA|2064-12-03|1979|8606|661|2064|3|12|3|4|2064|661|8606|Wednesday|2064Q4|N|N|N|2475256|2475590|2474892|2475166|N|N|N|N|N| +2475259|AAAAAAAALPEMFCAA|2064-12-04|1979|8606|661|2064|4|12|4|4|2064|661|8606|Thursday|2064Q4|N|N|N|2475256|2475590|2474893|2475167|N|N|N|N|N| +2475260|AAAAAAAAMPEMFCAA|2064-12-05|1979|8606|661|2064|5|12|5|4|2064|661|8606|Friday|2064Q4|N|Y|N|2475256|2475590|2474894|2475168|N|N|N|N|N| +2475261|AAAAAAAANPEMFCAA|2064-12-06|1979|8606|661|2064|6|12|6|4|2064|661|8606|Saturday|2064Q4|N|Y|N|2475256|2475590|2474895|2475169|N|N|N|N|N| +2475262|AAAAAAAAOPEMFCAA|2064-12-07|1979|8606|661|2064|0|12|7|4|2064|661|8606|Sunday|2064Q4|N|N|N|2475256|2475590|2474896|2475170|N|N|N|N|N| +2475263|AAAAAAAAPPEMFCAA|2064-12-08|1979|8606|661|2064|1|12|8|4|2064|661|8606|Monday|2064Q4|N|N|N|2475256|2475590|2474897|2475171|N|N|N|N|N| +2475264|AAAAAAAAAAFMFCAA|2064-12-09|1979|8607|661|2064|2|12|9|4|2064|661|8607|Tuesday|2064Q4|N|N|N|2475256|2475590|2474898|2475172|N|N|N|N|N| +2475265|AAAAAAAABAFMFCAA|2064-12-10|1979|8607|661|2064|3|12|10|4|2064|661|8607|Wednesday|2064Q4|N|N|N|2475256|2475590|2474899|2475173|N|N|N|N|N| +2475266|AAAAAAAACAFMFCAA|2064-12-11|1979|8607|661|2064|4|12|11|4|2064|661|8607|Thursday|2064Q4|N|N|N|2475256|2475590|2474900|2475174|N|N|N|N|N| +2475267|AAAAAAAADAFMFCAA|2064-12-12|1979|8607|661|2064|5|12|12|4|2064|661|8607|Friday|2064Q4|N|Y|N|2475256|2475590|2474901|2475175|N|N|N|N|N| +2475268|AAAAAAAAEAFMFCAA|2064-12-13|1979|8607|661|2064|6|12|13|4|2064|661|8607|Saturday|2064Q4|N|Y|N|2475256|2475590|2474902|2475176|N|N|N|N|N| +2475269|AAAAAAAAFAFMFCAA|2064-12-14|1979|8607|661|2064|0|12|14|4|2064|661|8607|Sunday|2064Q4|N|N|N|2475256|2475590|2474903|2475177|N|N|N|N|N| +2475270|AAAAAAAAGAFMFCAA|2064-12-15|1979|8607|661|2064|1|12|15|4|2064|661|8607|Monday|2064Q4|N|N|N|2475256|2475590|2474904|2475178|N|N|N|N|N| +2475271|AAAAAAAAHAFMFCAA|2064-12-16|1979|8608|661|2064|2|12|16|4|2064|661|8608|Tuesday|2064Q4|N|N|N|2475256|2475590|2474905|2475179|N|N|N|N|N| +2475272|AAAAAAAAIAFMFCAA|2064-12-17|1979|8608|661|2064|3|12|17|4|2064|661|8608|Wednesday|2064Q4|N|N|N|2475256|2475590|2474906|2475180|N|N|N|N|N| +2475273|AAAAAAAAJAFMFCAA|2064-12-18|1979|8608|661|2064|4|12|18|4|2064|661|8608|Thursday|2064Q4|N|N|N|2475256|2475590|2474907|2475181|N|N|N|N|N| +2475274|AAAAAAAAKAFMFCAA|2064-12-19|1979|8608|661|2064|5|12|19|4|2064|661|8608|Friday|2064Q4|N|Y|N|2475256|2475590|2474908|2475182|N|N|N|N|N| +2475275|AAAAAAAALAFMFCAA|2064-12-20|1979|8608|661|2064|6|12|20|4|2064|661|8608|Saturday|2064Q4|N|Y|N|2475256|2475590|2474909|2475183|N|N|N|N|N| +2475276|AAAAAAAAMAFMFCAA|2064-12-21|1979|8608|661|2064|0|12|21|4|2064|661|8608|Sunday|2064Q4|N|N|N|2475256|2475590|2474910|2475184|N|N|N|N|N| +2475277|AAAAAAAANAFMFCAA|2064-12-22|1979|8608|661|2064|1|12|22|4|2064|661|8608|Monday|2064Q4|N|N|N|2475256|2475590|2474911|2475185|N|N|N|N|N| +2475278|AAAAAAAAOAFMFCAA|2064-12-23|1979|8609|661|2064|2|12|23|4|2064|661|8609|Tuesday|2064Q4|N|N|N|2475256|2475590|2474912|2475186|N|N|N|N|N| +2475279|AAAAAAAAPAFMFCAA|2064-12-24|1979|8609|661|2064|3|12|24|4|2064|661|8609|Wednesday|2064Q4|N|N|N|2475256|2475590|2474913|2475187|N|N|N|N|N| +2475280|AAAAAAAAABFMFCAA|2064-12-25|1979|8609|661|2064|4|12|25|4|2064|661|8609|Thursday|2064Q4|Y|N|N|2475256|2475590|2474914|2475188|N|N|N|N|N| +2475281|AAAAAAAABBFMFCAA|2064-12-26|1979|8609|661|2064|5|12|26|4|2064|661|8609|Friday|2064Q4|N|Y|Y|2475256|2475590|2474915|2475189|N|N|N|N|N| +2475282|AAAAAAAACBFMFCAA|2064-12-27|1979|8609|661|2064|6|12|27|4|2064|661|8609|Saturday|2064Q4|N|Y|N|2475256|2475590|2474916|2475190|N|N|N|N|N| +2475283|AAAAAAAADBFMFCAA|2064-12-28|1979|8609|661|2064|0|12|28|4|2064|661|8609|Sunday|2064Q4|N|N|N|2475256|2475590|2474917|2475191|N|N|N|N|N| +2475284|AAAAAAAAEBFMFCAA|2064-12-29|1979|8609|661|2064|1|12|29|4|2064|661|8609|Monday|2064Q4|N|N|N|2475256|2475590|2474918|2475192|N|N|N|N|N| +2475285|AAAAAAAAFBFMFCAA|2064-12-30|1979|8610|661|2064|2|12|30|4|2064|661|8610|Tuesday|2064Q4|N|N|N|2475256|2475590|2474919|2475193|N|N|N|N|N| +2475286|AAAAAAAAGBFMFCAA|2064-12-31|1979|8610|661|2064|3|12|31|4|2064|661|8610|Wednesday|2064Q4|Y|N|N|2475256|2475590|2474920|2475194|N|N|N|N|N| +2475287|AAAAAAAAHBFMFCAA|2065-01-01|1980|8610|661|2065|4|1|1|1|2065|661|8610|Thursday|2065Q1|Y|N|Y|2475287|2475286|2474921|2475195|N|N|N|N|N| +2475288|AAAAAAAAIBFMFCAA|2065-01-02|1980|8610|661|2065|5|1|2|1|2065|661|8610|Friday|2065Q1|N|Y|Y|2475287|2475286|2474922|2475196|N|N|N|N|N| +2475289|AAAAAAAAJBFMFCAA|2065-01-03|1980|8610|661|2065|6|1|3|1|2065|661|8610|Saturday|2065Q1|N|Y|N|2475287|2475286|2474923|2475197|N|N|N|N|N| +2475290|AAAAAAAAKBFMFCAA|2065-01-04|1980|8610|661|2065|0|1|4|1|2065|661|8610|Sunday|2065Q1|N|N|N|2475287|2475286|2474924|2475198|N|N|N|N|N| +2475291|AAAAAAAALBFMFCAA|2065-01-05|1980|8610|661|2065|1|1|5|1|2065|661|8610|Monday|2065Q1|N|N|N|2475287|2475286|2474925|2475199|N|N|N|N|N| +2475292|AAAAAAAAMBFMFCAA|2065-01-06|1980|8611|661|2065|2|1|6|1|2065|661|8611|Tuesday|2065Q1|N|N|N|2475287|2475286|2474926|2475200|N|N|N|N|N| +2475293|AAAAAAAANBFMFCAA|2065-01-07|1980|8611|661|2065|3|1|7|1|2065|661|8611|Wednesday|2065Q1|N|N|N|2475287|2475286|2474927|2475201|N|N|N|N|N| +2475294|AAAAAAAAOBFMFCAA|2065-01-08|1980|8611|661|2065|4|1|8|1|2065|661|8611|Thursday|2065Q1|N|N|N|2475287|2475286|2474928|2475202|N|N|N|N|N| +2475295|AAAAAAAAPBFMFCAA|2065-01-09|1980|8611|661|2065|5|1|9|1|2065|661|8611|Friday|2065Q1|N|Y|N|2475287|2475286|2474929|2475203|N|N|N|N|N| +2475296|AAAAAAAAACFMFCAA|2065-01-10|1980|8611|661|2065|6|1|10|1|2065|661|8611|Saturday|2065Q1|N|Y|N|2475287|2475286|2474930|2475204|N|N|N|N|N| +2475297|AAAAAAAABCFMFCAA|2065-01-11|1980|8611|661|2065|0|1|11|1|2065|661|8611|Sunday|2065Q1|N|N|N|2475287|2475286|2474931|2475205|N|N|N|N|N| +2475298|AAAAAAAACCFMFCAA|2065-01-12|1980|8611|661|2065|1|1|12|1|2065|661|8611|Monday|2065Q1|N|N|N|2475287|2475286|2474932|2475206|N|N|N|N|N| +2475299|AAAAAAAADCFMFCAA|2065-01-13|1980|8612|661|2065|2|1|13|1|2065|661|8612|Tuesday|2065Q1|N|N|N|2475287|2475286|2474933|2475207|N|N|N|N|N| +2475300|AAAAAAAAECFMFCAA|2065-01-14|1980|8612|661|2065|3|1|14|1|2065|661|8612|Wednesday|2065Q1|N|N|N|2475287|2475286|2474934|2475208|N|N|N|N|N| +2475301|AAAAAAAAFCFMFCAA|2065-01-15|1980|8612|661|2065|4|1|15|1|2065|661|8612|Thursday|2065Q1|N|N|N|2475287|2475286|2474935|2475209|N|N|N|N|N| +2475302|AAAAAAAAGCFMFCAA|2065-01-16|1980|8612|661|2065|5|1|16|1|2065|661|8612|Friday|2065Q1|N|Y|N|2475287|2475286|2474936|2475210|N|N|N|N|N| +2475303|AAAAAAAAHCFMFCAA|2065-01-17|1980|8612|661|2065|6|1|17|1|2065|661|8612|Saturday|2065Q1|N|Y|N|2475287|2475286|2474937|2475211|N|N|N|N|N| +2475304|AAAAAAAAICFMFCAA|2065-01-18|1980|8612|661|2065|0|1|18|1|2065|661|8612|Sunday|2065Q1|N|N|N|2475287|2475286|2474938|2475212|N|N|N|N|N| +2475305|AAAAAAAAJCFMFCAA|2065-01-19|1980|8612|661|2065|1|1|19|1|2065|661|8612|Monday|2065Q1|N|N|N|2475287|2475286|2474939|2475213|N|N|N|N|N| +2475306|AAAAAAAAKCFMFCAA|2065-01-20|1980|8613|661|2065|2|1|20|1|2065|661|8613|Tuesday|2065Q1|N|N|N|2475287|2475286|2474940|2475214|N|N|N|N|N| +2475307|AAAAAAAALCFMFCAA|2065-01-21|1980|8613|661|2065|3|1|21|1|2065|661|8613|Wednesday|2065Q1|N|N|N|2475287|2475286|2474941|2475215|N|N|N|N|N| +2475308|AAAAAAAAMCFMFCAA|2065-01-22|1980|8613|661|2065|4|1|22|1|2065|661|8613|Thursday|2065Q1|N|N|N|2475287|2475286|2474942|2475216|N|N|N|N|N| +2475309|AAAAAAAANCFMFCAA|2065-01-23|1980|8613|661|2065|5|1|23|1|2065|661|8613|Friday|2065Q1|N|Y|N|2475287|2475286|2474943|2475217|N|N|N|N|N| +2475310|AAAAAAAAOCFMFCAA|2065-01-24|1980|8613|661|2065|6|1|24|1|2065|661|8613|Saturday|2065Q1|N|Y|N|2475287|2475286|2474944|2475218|N|N|N|N|N| +2475311|AAAAAAAAPCFMFCAA|2065-01-25|1980|8613|661|2065|0|1|25|1|2065|661|8613|Sunday|2065Q1|N|N|N|2475287|2475286|2474945|2475219|N|N|N|N|N| +2475312|AAAAAAAAADFMFCAA|2065-01-26|1980|8613|661|2065|1|1|26|1|2065|661|8613|Monday|2065Q1|N|N|N|2475287|2475286|2474946|2475220|N|N|N|N|N| +2475313|AAAAAAAABDFMFCAA|2065-01-27|1980|8614|661|2065|2|1|27|1|2065|661|8614|Tuesday|2065Q1|N|N|N|2475287|2475286|2474947|2475221|N|N|N|N|N| +2475314|AAAAAAAACDFMFCAA|2065-01-28|1980|8614|661|2065|3|1|28|1|2065|661|8614|Wednesday|2065Q1|N|N|N|2475287|2475286|2474948|2475222|N|N|N|N|N| +2475315|AAAAAAAADDFMFCAA|2065-01-29|1980|8614|661|2065|4|1|29|1|2065|661|8614|Thursday|2065Q1|N|N|N|2475287|2475286|2474949|2475223|N|N|N|N|N| +2475316|AAAAAAAAEDFMFCAA|2065-01-30|1980|8614|661|2065|5|1|30|1|2065|661|8614|Friday|2065Q1|N|Y|N|2475287|2475286|2474950|2475224|N|N|N|N|N| +2475317|AAAAAAAAFDFMFCAA|2065-01-31|1980|8614|661|2065|6|1|31|1|2065|661|8614|Saturday|2065Q1|N|Y|N|2475287|2475286|2474951|2475225|N|N|N|N|N| +2475318|AAAAAAAAGDFMFCAA|2065-02-01|1981|8614|661|2065|0|2|1|1|2065|661|8614|Sunday|2065Q1|N|N|N|2475318|2475348|2474952|2475226|N|N|N|N|N| +2475319|AAAAAAAAHDFMFCAA|2065-02-02|1981|8614|661|2065|1|2|2|1|2065|661|8614|Monday|2065Q1|N|N|N|2475318|2475348|2474953|2475227|N|N|N|N|N| +2475320|AAAAAAAAIDFMFCAA|2065-02-03|1981|8615|661|2065|2|2|3|1|2065|661|8615|Tuesday|2065Q1|N|N|N|2475318|2475348|2474954|2475228|N|N|N|N|N| +2475321|AAAAAAAAJDFMFCAA|2065-02-04|1981|8615|661|2065|3|2|4|1|2065|661|8615|Wednesday|2065Q1|N|N|N|2475318|2475348|2474955|2475229|N|N|N|N|N| +2475322|AAAAAAAAKDFMFCAA|2065-02-05|1981|8615|661|2065|4|2|5|1|2065|661|8615|Thursday|2065Q1|N|N|N|2475318|2475348|2474956|2475230|N|N|N|N|N| +2475323|AAAAAAAALDFMFCAA|2065-02-06|1981|8615|661|2065|5|2|6|1|2065|661|8615|Friday|2065Q1|N|Y|N|2475318|2475348|2474957|2475231|N|N|N|N|N| +2475324|AAAAAAAAMDFMFCAA|2065-02-07|1981|8615|661|2065|6|2|7|1|2065|661|8615|Saturday|2065Q1|N|Y|N|2475318|2475348|2474958|2475232|N|N|N|N|N| +2475325|AAAAAAAANDFMFCAA|2065-02-08|1981|8615|661|2065|0|2|8|1|2065|661|8615|Sunday|2065Q1|N|N|N|2475318|2475348|2474959|2475233|N|N|N|N|N| +2475326|AAAAAAAAODFMFCAA|2065-02-09|1981|8615|661|2065|1|2|9|1|2065|661|8615|Monday|2065Q1|N|N|N|2475318|2475348|2474960|2475234|N|N|N|N|N| +2475327|AAAAAAAAPDFMFCAA|2065-02-10|1981|8616|661|2065|2|2|10|1|2065|661|8616|Tuesday|2065Q1|N|N|N|2475318|2475348|2474961|2475235|N|N|N|N|N| +2475328|AAAAAAAAAEFMFCAA|2065-02-11|1981|8616|661|2065|3|2|11|1|2065|661|8616|Wednesday|2065Q1|N|N|N|2475318|2475348|2474962|2475236|N|N|N|N|N| +2475329|AAAAAAAABEFMFCAA|2065-02-12|1981|8616|661|2065|4|2|12|1|2065|661|8616|Thursday|2065Q1|N|N|N|2475318|2475348|2474963|2475237|N|N|N|N|N| +2475330|AAAAAAAACEFMFCAA|2065-02-13|1981|8616|661|2065|5|2|13|1|2065|661|8616|Friday|2065Q1|N|Y|N|2475318|2475348|2474964|2475238|N|N|N|N|N| +2475331|AAAAAAAADEFMFCAA|2065-02-14|1981|8616|661|2065|6|2|14|1|2065|661|8616|Saturday|2065Q1|N|Y|N|2475318|2475348|2474965|2475239|N|N|N|N|N| +2475332|AAAAAAAAEEFMFCAA|2065-02-15|1981|8616|661|2065|0|2|15|1|2065|661|8616|Sunday|2065Q1|N|N|N|2475318|2475348|2474966|2475240|N|N|N|N|N| +2475333|AAAAAAAAFEFMFCAA|2065-02-16|1981|8616|661|2065|1|2|16|1|2065|661|8616|Monday|2065Q1|N|N|N|2475318|2475348|2474967|2475241|N|N|N|N|N| +2475334|AAAAAAAAGEFMFCAA|2065-02-17|1981|8617|661|2065|2|2|17|1|2065|661|8617|Tuesday|2065Q1|N|N|N|2475318|2475348|2474968|2475242|N|N|N|N|N| +2475335|AAAAAAAAHEFMFCAA|2065-02-18|1981|8617|661|2065|3|2|18|1|2065|661|8617|Wednesday|2065Q1|N|N|N|2475318|2475348|2474969|2475243|N|N|N|N|N| +2475336|AAAAAAAAIEFMFCAA|2065-02-19|1981|8617|661|2065|4|2|19|1|2065|661|8617|Thursday|2065Q1|N|N|N|2475318|2475348|2474970|2475244|N|N|N|N|N| +2475337|AAAAAAAAJEFMFCAA|2065-02-20|1981|8617|661|2065|5|2|20|1|2065|661|8617|Friday|2065Q1|N|Y|N|2475318|2475348|2474971|2475245|N|N|N|N|N| +2475338|AAAAAAAAKEFMFCAA|2065-02-21|1981|8617|661|2065|6|2|21|1|2065|661|8617|Saturday|2065Q1|N|Y|N|2475318|2475348|2474972|2475246|N|N|N|N|N| +2475339|AAAAAAAALEFMFCAA|2065-02-22|1981|8617|661|2065|0|2|22|1|2065|661|8617|Sunday|2065Q1|N|N|N|2475318|2475348|2474973|2475247|N|N|N|N|N| +2475340|AAAAAAAAMEFMFCAA|2065-02-23|1981|8617|661|2065|1|2|23|1|2065|661|8617|Monday|2065Q1|N|N|N|2475318|2475348|2474974|2475248|N|N|N|N|N| +2475341|AAAAAAAANEFMFCAA|2065-02-24|1981|8618|661|2065|2|2|24|1|2065|661|8618|Tuesday|2065Q1|N|N|N|2475318|2475348|2474975|2475249|N|N|N|N|N| +2475342|AAAAAAAAOEFMFCAA|2065-02-25|1981|8618|661|2065|3|2|25|1|2065|661|8618|Wednesday|2065Q1|N|N|N|2475318|2475348|2474976|2475250|N|N|N|N|N| +2475343|AAAAAAAAPEFMFCAA|2065-02-26|1981|8618|661|2065|4|2|26|1|2065|661|8618|Thursday|2065Q1|N|N|N|2475318|2475348|2474977|2475251|N|N|N|N|N| +2475344|AAAAAAAAAFFMFCAA|2065-02-27|1981|8618|661|2065|5|2|27|1|2065|661|8618|Friday|2065Q1|N|Y|N|2475318|2475348|2474978|2475252|N|N|N|N|N| +2475345|AAAAAAAABFFMFCAA|2065-02-28|1981|8618|661|2065|6|2|28|1|2065|661|8618|Saturday|2065Q1|N|Y|N|2475318|2475348|2474979|2475253|N|N|N|N|N| +2475346|AAAAAAAACFFMFCAA|2065-03-01|1982|8618|662|2065|0|3|1|1|2065|662|8618|Sunday|2065Q1|N|N|N|2475346|2475404|2474981|2475254|N|N|N|N|N| +2475347|AAAAAAAADFFMFCAA|2065-03-02|1982|8618|662|2065|1|3|2|1|2065|662|8618|Monday|2065Q1|N|N|N|2475346|2475404|2474982|2475255|N|N|N|N|N| +2475348|AAAAAAAAEFFMFCAA|2065-03-03|1982|8619|662|2065|2|3|3|1|2065|662|8619|Tuesday|2065Q1|N|N|N|2475346|2475404|2474983|2475256|N|N|N|N|N| +2475349|AAAAAAAAFFFMFCAA|2065-03-04|1982|8619|662|2065|3|3|4|1|2065|662|8619|Wednesday|2065Q1|N|N|N|2475346|2475404|2474984|2475257|N|N|N|N|N| +2475350|AAAAAAAAGFFMFCAA|2065-03-05|1982|8619|662|2065|4|3|5|1|2065|662|8619|Thursday|2065Q1|N|N|N|2475346|2475404|2474985|2475258|N|N|N|N|N| +2475351|AAAAAAAAHFFMFCAA|2065-03-06|1982|8619|662|2065|5|3|6|1|2065|662|8619|Friday|2065Q1|N|Y|N|2475346|2475404|2474986|2475259|N|N|N|N|N| +2475352|AAAAAAAAIFFMFCAA|2065-03-07|1982|8619|662|2065|6|3|7|1|2065|662|8619|Saturday|2065Q1|N|Y|N|2475346|2475404|2474987|2475260|N|N|N|N|N| +2475353|AAAAAAAAJFFMFCAA|2065-03-08|1982|8619|662|2065|0|3|8|1|2065|662|8619|Sunday|2065Q1|N|N|N|2475346|2475404|2474988|2475261|N|N|N|N|N| +2475354|AAAAAAAAKFFMFCAA|2065-03-09|1982|8619|662|2065|1|3|9|1|2065|662|8619|Monday|2065Q1|N|N|N|2475346|2475404|2474989|2475262|N|N|N|N|N| +2475355|AAAAAAAALFFMFCAA|2065-03-10|1982|8620|662|2065|2|3|10|1|2065|662|8620|Tuesday|2065Q1|N|N|N|2475346|2475404|2474990|2475263|N|N|N|N|N| +2475356|AAAAAAAAMFFMFCAA|2065-03-11|1982|8620|662|2065|3|3|11|1|2065|662|8620|Wednesday|2065Q1|N|N|N|2475346|2475404|2474991|2475264|N|N|N|N|N| +2475357|AAAAAAAANFFMFCAA|2065-03-12|1982|8620|662|2065|4|3|12|1|2065|662|8620|Thursday|2065Q1|N|N|N|2475346|2475404|2474992|2475265|N|N|N|N|N| +2475358|AAAAAAAAOFFMFCAA|2065-03-13|1982|8620|662|2065|5|3|13|1|2065|662|8620|Friday|2065Q1|N|Y|N|2475346|2475404|2474993|2475266|N|N|N|N|N| +2475359|AAAAAAAAPFFMFCAA|2065-03-14|1982|8620|662|2065|6|3|14|1|2065|662|8620|Saturday|2065Q1|N|Y|N|2475346|2475404|2474994|2475267|N|N|N|N|N| +2475360|AAAAAAAAAGFMFCAA|2065-03-15|1982|8620|662|2065|0|3|15|1|2065|662|8620|Sunday|2065Q1|N|N|N|2475346|2475404|2474995|2475268|N|N|N|N|N| +2475361|AAAAAAAABGFMFCAA|2065-03-16|1982|8620|662|2065|1|3|16|1|2065|662|8620|Monday|2065Q1|N|N|N|2475346|2475404|2474996|2475269|N|N|N|N|N| +2475362|AAAAAAAACGFMFCAA|2065-03-17|1982|8621|662|2065|2|3|17|1|2065|662|8621|Tuesday|2065Q1|N|N|N|2475346|2475404|2474997|2475270|N|N|N|N|N| +2475363|AAAAAAAADGFMFCAA|2065-03-18|1982|8621|662|2065|3|3|18|1|2065|662|8621|Wednesday|2065Q1|N|N|N|2475346|2475404|2474998|2475271|N|N|N|N|N| +2475364|AAAAAAAAEGFMFCAA|2065-03-19|1982|8621|662|2065|4|3|19|1|2065|662|8621|Thursday|2065Q1|N|N|N|2475346|2475404|2474999|2475272|N|N|N|N|N| +2475365|AAAAAAAAFGFMFCAA|2065-03-20|1982|8621|662|2065|5|3|20|1|2065|662|8621|Friday|2065Q1|N|Y|N|2475346|2475404|2475000|2475273|N|N|N|N|N| +2475366|AAAAAAAAGGFMFCAA|2065-03-21|1982|8621|662|2065|6|3|21|1|2065|662|8621|Saturday|2065Q1|N|Y|N|2475346|2475404|2475001|2475274|N|N|N|N|N| +2475367|AAAAAAAAHGFMFCAA|2065-03-22|1982|8621|662|2065|0|3|22|1|2065|662|8621|Sunday|2065Q1|N|N|N|2475346|2475404|2475002|2475275|N|N|N|N|N| +2475368|AAAAAAAAIGFMFCAA|2065-03-23|1982|8621|662|2065|1|3|23|1|2065|662|8621|Monday|2065Q1|N|N|N|2475346|2475404|2475003|2475276|N|N|N|N|N| +2475369|AAAAAAAAJGFMFCAA|2065-03-24|1982|8622|662|2065|2|3|24|1|2065|662|8622|Tuesday|2065Q1|N|N|N|2475346|2475404|2475004|2475277|N|N|N|N|N| +2475370|AAAAAAAAKGFMFCAA|2065-03-25|1982|8622|662|2065|3|3|25|1|2065|662|8622|Wednesday|2065Q1|N|N|N|2475346|2475404|2475005|2475278|N|N|N|N|N| +2475371|AAAAAAAALGFMFCAA|2065-03-26|1982|8622|662|2065|4|3|26|1|2065|662|8622|Thursday|2065Q1|N|N|N|2475346|2475404|2475006|2475279|N|N|N|N|N| +2475372|AAAAAAAAMGFMFCAA|2065-03-27|1982|8622|662|2065|5|3|27|1|2065|662|8622|Friday|2065Q1|N|Y|N|2475346|2475404|2475007|2475280|N|N|N|N|N| +2475373|AAAAAAAANGFMFCAA|2065-03-28|1982|8622|662|2065|6|3|28|1|2065|662|8622|Saturday|2065Q1|N|Y|N|2475346|2475404|2475008|2475281|N|N|N|N|N| +2475374|AAAAAAAAOGFMFCAA|2065-03-29|1982|8622|662|2065|0|3|29|1|2065|662|8622|Sunday|2065Q1|N|N|N|2475346|2475404|2475009|2475282|N|N|N|N|N| +2475375|AAAAAAAAPGFMFCAA|2065-03-30|1982|8622|662|2065|1|3|30|1|2065|662|8622|Monday|2065Q1|N|N|N|2475346|2475404|2475010|2475283|N|N|N|N|N| +2475376|AAAAAAAAAHFMFCAA|2065-03-31|1982|8623|662|2065|2|3|31|1|2065|662|8623|Tuesday|2065Q1|N|N|N|2475346|2475404|2475011|2475284|N|N|N|N|N| +2475377|AAAAAAAABHFMFCAA|2065-04-01|1983|8623|662|2065|3|4|1|1|2065|662|8623|Wednesday|2065Q1|N|N|N|2475377|2475466|2475012|2475287|N|N|N|N|N| +2475378|AAAAAAAACHFMFCAA|2065-04-02|1983|8623|662|2065|4|4|2|2|2065|662|8623|Thursday|2065Q2|N|N|N|2475377|2475466|2475013|2475288|N|N|N|N|N| +2475379|AAAAAAAADHFMFCAA|2065-04-03|1983|8623|662|2065|5|4|3|2|2065|662|8623|Friday|2065Q2|N|Y|N|2475377|2475466|2475014|2475289|N|N|N|N|N| +2475380|AAAAAAAAEHFMFCAA|2065-04-04|1983|8623|662|2065|6|4|4|2|2065|662|8623|Saturday|2065Q2|N|Y|N|2475377|2475466|2475015|2475290|N|N|N|N|N| +2475381|AAAAAAAAFHFMFCAA|2065-04-05|1983|8623|662|2065|0|4|5|2|2065|662|8623|Sunday|2065Q2|N|N|N|2475377|2475466|2475016|2475291|N|N|N|N|N| +2475382|AAAAAAAAGHFMFCAA|2065-04-06|1983|8623|662|2065|1|4|6|2|2065|662|8623|Monday|2065Q2|N|N|N|2475377|2475466|2475017|2475292|N|N|N|N|N| +2475383|AAAAAAAAHHFMFCAA|2065-04-07|1983|8624|662|2065|2|4|7|2|2065|662|8624|Tuesday|2065Q2|N|N|N|2475377|2475466|2475018|2475293|N|N|N|N|N| +2475384|AAAAAAAAIHFMFCAA|2065-04-08|1983|8624|662|2065|3|4|8|2|2065|662|8624|Wednesday|2065Q2|N|N|N|2475377|2475466|2475019|2475294|N|N|N|N|N| +2475385|AAAAAAAAJHFMFCAA|2065-04-09|1983|8624|662|2065|4|4|9|2|2065|662|8624|Thursday|2065Q2|N|N|N|2475377|2475466|2475020|2475295|N|N|N|N|N| +2475386|AAAAAAAAKHFMFCAA|2065-04-10|1983|8624|662|2065|5|4|10|2|2065|662|8624|Friday|2065Q2|N|Y|N|2475377|2475466|2475021|2475296|N|N|N|N|N| +2475387|AAAAAAAALHFMFCAA|2065-04-11|1983|8624|662|2065|6|4|11|2|2065|662|8624|Saturday|2065Q2|N|Y|N|2475377|2475466|2475022|2475297|N|N|N|N|N| +2475388|AAAAAAAAMHFMFCAA|2065-04-12|1983|8624|662|2065|0|4|12|2|2065|662|8624|Sunday|2065Q2|N|N|N|2475377|2475466|2475023|2475298|N|N|N|N|N| +2475389|AAAAAAAANHFMFCAA|2065-04-13|1983|8624|662|2065|1|4|13|2|2065|662|8624|Monday|2065Q2|N|N|N|2475377|2475466|2475024|2475299|N|N|N|N|N| +2475390|AAAAAAAAOHFMFCAA|2065-04-14|1983|8625|662|2065|2|4|14|2|2065|662|8625|Tuesday|2065Q2|N|N|N|2475377|2475466|2475025|2475300|N|N|N|N|N| +2475391|AAAAAAAAPHFMFCAA|2065-04-15|1983|8625|662|2065|3|4|15|2|2065|662|8625|Wednesday|2065Q2|N|N|N|2475377|2475466|2475026|2475301|N|N|N|N|N| +2475392|AAAAAAAAAIFMFCAA|2065-04-16|1983|8625|662|2065|4|4|16|2|2065|662|8625|Thursday|2065Q2|N|N|N|2475377|2475466|2475027|2475302|N|N|N|N|N| +2475393|AAAAAAAABIFMFCAA|2065-04-17|1983|8625|662|2065|5|4|17|2|2065|662|8625|Friday|2065Q2|N|Y|N|2475377|2475466|2475028|2475303|N|N|N|N|N| +2475394|AAAAAAAACIFMFCAA|2065-04-18|1983|8625|662|2065|6|4|18|2|2065|662|8625|Saturday|2065Q2|N|Y|N|2475377|2475466|2475029|2475304|N|N|N|N|N| +2475395|AAAAAAAADIFMFCAA|2065-04-19|1983|8625|662|2065|0|4|19|2|2065|662|8625|Sunday|2065Q2|N|N|N|2475377|2475466|2475030|2475305|N|N|N|N|N| +2475396|AAAAAAAAEIFMFCAA|2065-04-20|1983|8625|662|2065|1|4|20|2|2065|662|8625|Monday|2065Q2|N|N|N|2475377|2475466|2475031|2475306|N|N|N|N|N| +2475397|AAAAAAAAFIFMFCAA|2065-04-21|1983|8626|662|2065|2|4|21|2|2065|662|8626|Tuesday|2065Q2|N|N|N|2475377|2475466|2475032|2475307|N|N|N|N|N| +2475398|AAAAAAAAGIFMFCAA|2065-04-22|1983|8626|662|2065|3|4|22|2|2065|662|8626|Wednesday|2065Q2|N|N|N|2475377|2475466|2475033|2475308|N|N|N|N|N| +2475399|AAAAAAAAHIFMFCAA|2065-04-23|1983|8626|662|2065|4|4|23|2|2065|662|8626|Thursday|2065Q2|N|N|N|2475377|2475466|2475034|2475309|N|N|N|N|N| +2475400|AAAAAAAAIIFMFCAA|2065-04-24|1983|8626|662|2065|5|4|24|2|2065|662|8626|Friday|2065Q2|N|Y|N|2475377|2475466|2475035|2475310|N|N|N|N|N| +2475401|AAAAAAAAJIFMFCAA|2065-04-25|1983|8626|662|2065|6|4|25|2|2065|662|8626|Saturday|2065Q2|N|Y|N|2475377|2475466|2475036|2475311|N|N|N|N|N| +2475402|AAAAAAAAKIFMFCAA|2065-04-26|1983|8626|662|2065|0|4|26|2|2065|662|8626|Sunday|2065Q2|N|N|N|2475377|2475466|2475037|2475312|N|N|N|N|N| +2475403|AAAAAAAALIFMFCAA|2065-04-27|1983|8626|662|2065|1|4|27|2|2065|662|8626|Monday|2065Q2|N|N|N|2475377|2475466|2475038|2475313|N|N|N|N|N| +2475404|AAAAAAAAMIFMFCAA|2065-04-28|1983|8627|662|2065|2|4|28|2|2065|662|8627|Tuesday|2065Q2|N|N|N|2475377|2475466|2475039|2475314|N|N|N|N|N| +2475405|AAAAAAAANIFMFCAA|2065-04-29|1983|8627|662|2065|3|4|29|2|2065|662|8627|Wednesday|2065Q2|N|N|N|2475377|2475466|2475040|2475315|N|N|N|N|N| +2475406|AAAAAAAAOIFMFCAA|2065-04-30|1983|8627|662|2065|4|4|30|2|2065|662|8627|Thursday|2065Q2|N|N|N|2475377|2475466|2475041|2475316|N|N|N|N|N| +2475407|AAAAAAAAPIFMFCAA|2065-05-01|1984|8627|662|2065|5|5|1|2|2065|662|8627|Friday|2065Q2|N|Y|N|2475407|2475526|2475042|2475317|N|N|N|N|N| +2475408|AAAAAAAAAJFMFCAA|2065-05-02|1984|8627|662|2065|6|5|2|2|2065|662|8627|Saturday|2065Q2|N|Y|N|2475407|2475526|2475043|2475318|N|N|N|N|N| +2475409|AAAAAAAABJFMFCAA|2065-05-03|1984|8627|662|2065|0|5|3|2|2065|662|8627|Sunday|2065Q2|N|N|N|2475407|2475526|2475044|2475319|N|N|N|N|N| +2475410|AAAAAAAACJFMFCAA|2065-05-04|1984|8627|662|2065|1|5|4|2|2065|662|8627|Monday|2065Q2|N|N|N|2475407|2475526|2475045|2475320|N|N|N|N|N| +2475411|AAAAAAAADJFMFCAA|2065-05-05|1984|8628|662|2065|2|5|5|2|2065|662|8628|Tuesday|2065Q2|N|N|N|2475407|2475526|2475046|2475321|N|N|N|N|N| +2475412|AAAAAAAAEJFMFCAA|2065-05-06|1984|8628|662|2065|3|5|6|2|2065|662|8628|Wednesday|2065Q2|N|N|N|2475407|2475526|2475047|2475322|N|N|N|N|N| +2475413|AAAAAAAAFJFMFCAA|2065-05-07|1984|8628|662|2065|4|5|7|2|2065|662|8628|Thursday|2065Q2|N|N|N|2475407|2475526|2475048|2475323|N|N|N|N|N| +2475414|AAAAAAAAGJFMFCAA|2065-05-08|1984|8628|662|2065|5|5|8|2|2065|662|8628|Friday|2065Q2|N|Y|N|2475407|2475526|2475049|2475324|N|N|N|N|N| +2475415|AAAAAAAAHJFMFCAA|2065-05-09|1984|8628|662|2065|6|5|9|2|2065|662|8628|Saturday|2065Q2|N|Y|N|2475407|2475526|2475050|2475325|N|N|N|N|N| +2475416|AAAAAAAAIJFMFCAA|2065-05-10|1984|8628|662|2065|0|5|10|2|2065|662|8628|Sunday|2065Q2|N|N|N|2475407|2475526|2475051|2475326|N|N|N|N|N| +2475417|AAAAAAAAJJFMFCAA|2065-05-11|1984|8628|662|2065|1|5|11|2|2065|662|8628|Monday|2065Q2|N|N|N|2475407|2475526|2475052|2475327|N|N|N|N|N| +2475418|AAAAAAAAKJFMFCAA|2065-05-12|1984|8629|662|2065|2|5|12|2|2065|662|8629|Tuesday|2065Q2|N|N|N|2475407|2475526|2475053|2475328|N|N|N|N|N| +2475419|AAAAAAAALJFMFCAA|2065-05-13|1984|8629|662|2065|3|5|13|2|2065|662|8629|Wednesday|2065Q2|N|N|N|2475407|2475526|2475054|2475329|N|N|N|N|N| +2475420|AAAAAAAAMJFMFCAA|2065-05-14|1984|8629|662|2065|4|5|14|2|2065|662|8629|Thursday|2065Q2|N|N|N|2475407|2475526|2475055|2475330|N|N|N|N|N| +2475421|AAAAAAAANJFMFCAA|2065-05-15|1984|8629|662|2065|5|5|15|2|2065|662|8629|Friday|2065Q2|N|Y|N|2475407|2475526|2475056|2475331|N|N|N|N|N| +2475422|AAAAAAAAOJFMFCAA|2065-05-16|1984|8629|662|2065|6|5|16|2|2065|662|8629|Saturday|2065Q2|N|Y|N|2475407|2475526|2475057|2475332|N|N|N|N|N| +2475423|AAAAAAAAPJFMFCAA|2065-05-17|1984|8629|662|2065|0|5|17|2|2065|662|8629|Sunday|2065Q2|N|N|N|2475407|2475526|2475058|2475333|N|N|N|N|N| +2475424|AAAAAAAAAKFMFCAA|2065-05-18|1984|8629|662|2065|1|5|18|2|2065|662|8629|Monday|2065Q2|N|N|N|2475407|2475526|2475059|2475334|N|N|N|N|N| +2475425|AAAAAAAABKFMFCAA|2065-05-19|1984|8630|662|2065|2|5|19|2|2065|662|8630|Tuesday|2065Q2|N|N|N|2475407|2475526|2475060|2475335|N|N|N|N|N| +2475426|AAAAAAAACKFMFCAA|2065-05-20|1984|8630|662|2065|3|5|20|2|2065|662|8630|Wednesday|2065Q2|N|N|N|2475407|2475526|2475061|2475336|N|N|N|N|N| +2475427|AAAAAAAADKFMFCAA|2065-05-21|1984|8630|662|2065|4|5|21|2|2065|662|8630|Thursday|2065Q2|N|N|N|2475407|2475526|2475062|2475337|N|N|N|N|N| +2475428|AAAAAAAAEKFMFCAA|2065-05-22|1984|8630|662|2065|5|5|22|2|2065|662|8630|Friday|2065Q2|N|Y|N|2475407|2475526|2475063|2475338|N|N|N|N|N| +2475429|AAAAAAAAFKFMFCAA|2065-05-23|1984|8630|662|2065|6|5|23|2|2065|662|8630|Saturday|2065Q2|N|Y|N|2475407|2475526|2475064|2475339|N|N|N|N|N| +2475430|AAAAAAAAGKFMFCAA|2065-05-24|1984|8630|662|2065|0|5|24|2|2065|662|8630|Sunday|2065Q2|N|N|N|2475407|2475526|2475065|2475340|N|N|N|N|N| +2475431|AAAAAAAAHKFMFCAA|2065-05-25|1984|8630|662|2065|1|5|25|2|2065|662|8630|Monday|2065Q2|N|N|N|2475407|2475526|2475066|2475341|N|N|N|N|N| +2475432|AAAAAAAAIKFMFCAA|2065-05-26|1984|8631|662|2065|2|5|26|2|2065|662|8631|Tuesday|2065Q2|N|N|N|2475407|2475526|2475067|2475342|N|N|N|N|N| +2475433|AAAAAAAAJKFMFCAA|2065-05-27|1984|8631|662|2065|3|5|27|2|2065|662|8631|Wednesday|2065Q2|N|N|N|2475407|2475526|2475068|2475343|N|N|N|N|N| +2475434|AAAAAAAAKKFMFCAA|2065-05-28|1984|8631|662|2065|4|5|28|2|2065|662|8631|Thursday|2065Q2|N|N|N|2475407|2475526|2475069|2475344|N|N|N|N|N| +2475435|AAAAAAAALKFMFCAA|2065-05-29|1984|8631|662|2065|5|5|29|2|2065|662|8631|Friday|2065Q2|N|Y|N|2475407|2475526|2475070|2475345|N|N|N|N|N| +2475436|AAAAAAAAMKFMFCAA|2065-05-30|1984|8631|662|2065|6|5|30|2|2065|662|8631|Saturday|2065Q2|N|Y|N|2475407|2475526|2475071|2475346|N|N|N|N|N| +2475437|AAAAAAAANKFMFCAA|2065-05-31|1984|8631|662|2065|0|5|31|2|2065|662|8631|Sunday|2065Q2|N|N|N|2475407|2475526|2475072|2475347|N|N|N|N|N| +2475438|AAAAAAAAOKFMFCAA|2065-06-01|1985|8631|663|2065|1|6|1|2|2065|663|8631|Monday|2065Q2|N|N|N|2475438|2475588|2475073|2475348|N|N|N|N|N| +2475439|AAAAAAAAPKFMFCAA|2065-06-02|1985|8632|663|2065|2|6|2|2|2065|663|8632|Tuesday|2065Q2|N|N|N|2475438|2475588|2475074|2475349|N|N|N|N|N| +2475440|AAAAAAAAALFMFCAA|2065-06-03|1985|8632|663|2065|3|6|3|2|2065|663|8632|Wednesday|2065Q2|N|N|N|2475438|2475588|2475075|2475350|N|N|N|N|N| +2475441|AAAAAAAABLFMFCAA|2065-06-04|1985|8632|663|2065|4|6|4|2|2065|663|8632|Thursday|2065Q2|N|N|N|2475438|2475588|2475076|2475351|N|N|N|N|N| +2475442|AAAAAAAACLFMFCAA|2065-06-05|1985|8632|663|2065|5|6|5|2|2065|663|8632|Friday|2065Q2|N|Y|N|2475438|2475588|2475077|2475352|N|N|N|N|N| +2475443|AAAAAAAADLFMFCAA|2065-06-06|1985|8632|663|2065|6|6|6|2|2065|663|8632|Saturday|2065Q2|N|Y|N|2475438|2475588|2475078|2475353|N|N|N|N|N| +2475444|AAAAAAAAELFMFCAA|2065-06-07|1985|8632|663|2065|0|6|7|2|2065|663|8632|Sunday|2065Q2|N|N|N|2475438|2475588|2475079|2475354|N|N|N|N|N| +2475445|AAAAAAAAFLFMFCAA|2065-06-08|1985|8632|663|2065|1|6|8|2|2065|663|8632|Monday|2065Q2|N|N|N|2475438|2475588|2475080|2475355|N|N|N|N|N| +2475446|AAAAAAAAGLFMFCAA|2065-06-09|1985|8633|663|2065|2|6|9|2|2065|663|8633|Tuesday|2065Q2|N|N|N|2475438|2475588|2475081|2475356|N|N|N|N|N| +2475447|AAAAAAAAHLFMFCAA|2065-06-10|1985|8633|663|2065|3|6|10|2|2065|663|8633|Wednesday|2065Q2|N|N|N|2475438|2475588|2475082|2475357|N|N|N|N|N| +2475448|AAAAAAAAILFMFCAA|2065-06-11|1985|8633|663|2065|4|6|11|2|2065|663|8633|Thursday|2065Q2|N|N|N|2475438|2475588|2475083|2475358|N|N|N|N|N| +2475449|AAAAAAAAJLFMFCAA|2065-06-12|1985|8633|663|2065|5|6|12|2|2065|663|8633|Friday|2065Q2|N|Y|N|2475438|2475588|2475084|2475359|N|N|N|N|N| +2475450|AAAAAAAAKLFMFCAA|2065-06-13|1985|8633|663|2065|6|6|13|2|2065|663|8633|Saturday|2065Q2|N|Y|N|2475438|2475588|2475085|2475360|N|N|N|N|N| +2475451|AAAAAAAALLFMFCAA|2065-06-14|1985|8633|663|2065|0|6|14|2|2065|663|8633|Sunday|2065Q2|N|N|N|2475438|2475588|2475086|2475361|N|N|N|N|N| +2475452|AAAAAAAAMLFMFCAA|2065-06-15|1985|8633|663|2065|1|6|15|2|2065|663|8633|Monday|2065Q2|N|N|N|2475438|2475588|2475087|2475362|N|N|N|N|N| +2475453|AAAAAAAANLFMFCAA|2065-06-16|1985|8634|663|2065|2|6|16|2|2065|663|8634|Tuesday|2065Q2|N|N|N|2475438|2475588|2475088|2475363|N|N|N|N|N| +2475454|AAAAAAAAOLFMFCAA|2065-06-17|1985|8634|663|2065|3|6|17|2|2065|663|8634|Wednesday|2065Q2|N|N|N|2475438|2475588|2475089|2475364|N|N|N|N|N| +2475455|AAAAAAAAPLFMFCAA|2065-06-18|1985|8634|663|2065|4|6|18|2|2065|663|8634|Thursday|2065Q2|N|N|N|2475438|2475588|2475090|2475365|N|N|N|N|N| +2475456|AAAAAAAAAMFMFCAA|2065-06-19|1985|8634|663|2065|5|6|19|2|2065|663|8634|Friday|2065Q2|N|Y|N|2475438|2475588|2475091|2475366|N|N|N|N|N| +2475457|AAAAAAAABMFMFCAA|2065-06-20|1985|8634|663|2065|6|6|20|2|2065|663|8634|Saturday|2065Q2|N|Y|N|2475438|2475588|2475092|2475367|N|N|N|N|N| +2475458|AAAAAAAACMFMFCAA|2065-06-21|1985|8634|663|2065|0|6|21|2|2065|663|8634|Sunday|2065Q2|N|N|N|2475438|2475588|2475093|2475368|N|N|N|N|N| +2475459|AAAAAAAADMFMFCAA|2065-06-22|1985|8634|663|2065|1|6|22|2|2065|663|8634|Monday|2065Q2|N|N|N|2475438|2475588|2475094|2475369|N|N|N|N|N| +2475460|AAAAAAAAEMFMFCAA|2065-06-23|1985|8635|663|2065|2|6|23|2|2065|663|8635|Tuesday|2065Q2|N|N|N|2475438|2475588|2475095|2475370|N|N|N|N|N| +2475461|AAAAAAAAFMFMFCAA|2065-06-24|1985|8635|663|2065|3|6|24|2|2065|663|8635|Wednesday|2065Q2|N|N|N|2475438|2475588|2475096|2475371|N|N|N|N|N| +2475462|AAAAAAAAGMFMFCAA|2065-06-25|1985|8635|663|2065|4|6|25|2|2065|663|8635|Thursday|2065Q2|N|N|N|2475438|2475588|2475097|2475372|N|N|N|N|N| +2475463|AAAAAAAAHMFMFCAA|2065-06-26|1985|8635|663|2065|5|6|26|2|2065|663|8635|Friday|2065Q2|N|Y|N|2475438|2475588|2475098|2475373|N|N|N|N|N| +2475464|AAAAAAAAIMFMFCAA|2065-06-27|1985|8635|663|2065|6|6|27|2|2065|663|8635|Saturday|2065Q2|N|Y|N|2475438|2475588|2475099|2475374|N|N|N|N|N| +2475465|AAAAAAAAJMFMFCAA|2065-06-28|1985|8635|663|2065|0|6|28|2|2065|663|8635|Sunday|2065Q2|N|N|N|2475438|2475588|2475100|2475375|N|N|N|N|N| +2475466|AAAAAAAAKMFMFCAA|2065-06-29|1985|8635|663|2065|1|6|29|2|2065|663|8635|Monday|2065Q2|N|N|N|2475438|2475588|2475101|2475376|N|N|N|N|N| +2475467|AAAAAAAALMFMFCAA|2065-06-30|1985|8636|663|2065|2|6|30|2|2065|663|8636|Tuesday|2065Q2|N|N|N|2475438|2475588|2475102|2475377|N|N|N|N|N| +2475468|AAAAAAAAMMFMFCAA|2065-07-01|1986|8636|663|2065|3|7|1|2|2065|663|8636|Wednesday|2065Q2|N|N|N|2475468|2475648|2475103|2475377|N|N|N|N|N| +2475469|AAAAAAAANMFMFCAA|2065-07-02|1986|8636|663|2065|4|7|2|3|2065|663|8636|Thursday|2065Q3|N|N|N|2475468|2475648|2475104|2475378|N|N|N|N|N| +2475470|AAAAAAAAOMFMFCAA|2065-07-03|1986|8636|663|2065|5|7|3|3|2065|663|8636|Friday|2065Q3|N|Y|N|2475468|2475648|2475105|2475379|N|N|N|N|N| +2475471|AAAAAAAAPMFMFCAA|2065-07-04|1986|8636|663|2065|6|7|4|3|2065|663|8636|Saturday|2065Q3|N|Y|N|2475468|2475648|2475106|2475380|N|N|N|N|N| +2475472|AAAAAAAAANFMFCAA|2065-07-05|1986|8636|663|2065|0|7|5|3|2065|663|8636|Sunday|2065Q3|Y|N|N|2475468|2475648|2475107|2475381|N|N|N|N|N| +2475473|AAAAAAAABNFMFCAA|2065-07-06|1986|8636|663|2065|1|7|6|3|2065|663|8636|Monday|2065Q3|N|N|Y|2475468|2475648|2475108|2475382|N|N|N|N|N| +2475474|AAAAAAAACNFMFCAA|2065-07-07|1986|8637|663|2065|2|7|7|3|2065|663|8637|Tuesday|2065Q3|N|N|N|2475468|2475648|2475109|2475383|N|N|N|N|N| +2475475|AAAAAAAADNFMFCAA|2065-07-08|1986|8637|663|2065|3|7|8|3|2065|663|8637|Wednesday|2065Q3|N|N|N|2475468|2475648|2475110|2475384|N|N|N|N|N| +2475476|AAAAAAAAENFMFCAA|2065-07-09|1986|8637|663|2065|4|7|9|3|2065|663|8637|Thursday|2065Q3|N|N|N|2475468|2475648|2475111|2475385|N|N|N|N|N| +2475477|AAAAAAAAFNFMFCAA|2065-07-10|1986|8637|663|2065|5|7|10|3|2065|663|8637|Friday|2065Q3|N|Y|N|2475468|2475648|2475112|2475386|N|N|N|N|N| +2475478|AAAAAAAAGNFMFCAA|2065-07-11|1986|8637|663|2065|6|7|11|3|2065|663|8637|Saturday|2065Q3|N|Y|N|2475468|2475648|2475113|2475387|N|N|N|N|N| +2475479|AAAAAAAAHNFMFCAA|2065-07-12|1986|8637|663|2065|0|7|12|3|2065|663|8637|Sunday|2065Q3|N|N|N|2475468|2475648|2475114|2475388|N|N|N|N|N| +2475480|AAAAAAAAINFMFCAA|2065-07-13|1986|8637|663|2065|1|7|13|3|2065|663|8637|Monday|2065Q3|N|N|N|2475468|2475648|2475115|2475389|N|N|N|N|N| +2475481|AAAAAAAAJNFMFCAA|2065-07-14|1986|8638|663|2065|2|7|14|3|2065|663|8638|Tuesday|2065Q3|N|N|N|2475468|2475648|2475116|2475390|N|N|N|N|N| +2475482|AAAAAAAAKNFMFCAA|2065-07-15|1986|8638|663|2065|3|7|15|3|2065|663|8638|Wednesday|2065Q3|N|N|N|2475468|2475648|2475117|2475391|N|N|N|N|N| +2475483|AAAAAAAALNFMFCAA|2065-07-16|1986|8638|663|2065|4|7|16|3|2065|663|8638|Thursday|2065Q3|N|N|N|2475468|2475648|2475118|2475392|N|N|N|N|N| +2475484|AAAAAAAAMNFMFCAA|2065-07-17|1986|8638|663|2065|5|7|17|3|2065|663|8638|Friday|2065Q3|N|Y|N|2475468|2475648|2475119|2475393|N|N|N|N|N| +2475485|AAAAAAAANNFMFCAA|2065-07-18|1986|8638|663|2065|6|7|18|3|2065|663|8638|Saturday|2065Q3|N|Y|N|2475468|2475648|2475120|2475394|N|N|N|N|N| +2475486|AAAAAAAAONFMFCAA|2065-07-19|1986|8638|663|2065|0|7|19|3|2065|663|8638|Sunday|2065Q3|N|N|N|2475468|2475648|2475121|2475395|N|N|N|N|N| +2475487|AAAAAAAAPNFMFCAA|2065-07-20|1986|8638|663|2065|1|7|20|3|2065|663|8638|Monday|2065Q3|N|N|N|2475468|2475648|2475122|2475396|N|N|N|N|N| +2475488|AAAAAAAAAOFMFCAA|2065-07-21|1986|8639|663|2065|2|7|21|3|2065|663|8639|Tuesday|2065Q3|N|N|N|2475468|2475648|2475123|2475397|N|N|N|N|N| +2475489|AAAAAAAABOFMFCAA|2065-07-22|1986|8639|663|2065|3|7|22|3|2065|663|8639|Wednesday|2065Q3|N|N|N|2475468|2475648|2475124|2475398|N|N|N|N|N| +2475490|AAAAAAAACOFMFCAA|2065-07-23|1986|8639|663|2065|4|7|23|3|2065|663|8639|Thursday|2065Q3|N|N|N|2475468|2475648|2475125|2475399|N|N|N|N|N| +2475491|AAAAAAAADOFMFCAA|2065-07-24|1986|8639|663|2065|5|7|24|3|2065|663|8639|Friday|2065Q3|N|Y|N|2475468|2475648|2475126|2475400|N|N|N|N|N| +2475492|AAAAAAAAEOFMFCAA|2065-07-25|1986|8639|663|2065|6|7|25|3|2065|663|8639|Saturday|2065Q3|N|Y|N|2475468|2475648|2475127|2475401|N|N|N|N|N| +2475493|AAAAAAAAFOFMFCAA|2065-07-26|1986|8639|663|2065|0|7|26|3|2065|663|8639|Sunday|2065Q3|N|N|N|2475468|2475648|2475128|2475402|N|N|N|N|N| +2475494|AAAAAAAAGOFMFCAA|2065-07-27|1986|8639|663|2065|1|7|27|3|2065|663|8639|Monday|2065Q3|N|N|N|2475468|2475648|2475129|2475403|N|N|N|N|N| +2475495|AAAAAAAAHOFMFCAA|2065-07-28|1986|8640|663|2065|2|7|28|3|2065|663|8640|Tuesday|2065Q3|N|N|N|2475468|2475648|2475130|2475404|N|N|N|N|N| +2475496|AAAAAAAAIOFMFCAA|2065-07-29|1986|8640|663|2065|3|7|29|3|2065|663|8640|Wednesday|2065Q3|N|N|N|2475468|2475648|2475131|2475405|N|N|N|N|N| +2475497|AAAAAAAAJOFMFCAA|2065-07-30|1986|8640|663|2065|4|7|30|3|2065|663|8640|Thursday|2065Q3|N|N|N|2475468|2475648|2475132|2475406|N|N|N|N|N| +2475498|AAAAAAAAKOFMFCAA|2065-07-31|1986|8640|663|2065|5|7|31|3|2065|663|8640|Friday|2065Q3|N|Y|N|2475468|2475648|2475133|2475407|N|N|N|N|N| +2475499|AAAAAAAALOFMFCAA|2065-08-01|1987|8640|663|2065|6|8|1|3|2065|663|8640|Saturday|2065Q3|N|Y|N|2475499|2475710|2475134|2475408|N|N|N|N|N| +2475500|AAAAAAAAMOFMFCAA|2065-08-02|1987|8640|663|2065|0|8|2|3|2065|663|8640|Sunday|2065Q3|N|N|N|2475499|2475710|2475135|2475409|N|N|N|N|N| +2475501|AAAAAAAANOFMFCAA|2065-08-03|1987|8640|663|2065|1|8|3|3|2065|663|8640|Monday|2065Q3|N|N|N|2475499|2475710|2475136|2475410|N|N|N|N|N| +2475502|AAAAAAAAOOFMFCAA|2065-08-04|1987|8641|663|2065|2|8|4|3|2065|663|8641|Tuesday|2065Q3|N|N|N|2475499|2475710|2475137|2475411|N|N|N|N|N| +2475503|AAAAAAAAPOFMFCAA|2065-08-05|1987|8641|663|2065|3|8|5|3|2065|663|8641|Wednesday|2065Q3|N|N|N|2475499|2475710|2475138|2475412|N|N|N|N|N| +2475504|AAAAAAAAAPFMFCAA|2065-08-06|1987|8641|663|2065|4|8|6|3|2065|663|8641|Thursday|2065Q3|N|N|N|2475499|2475710|2475139|2475413|N|N|N|N|N| +2475505|AAAAAAAABPFMFCAA|2065-08-07|1987|8641|663|2065|5|8|7|3|2065|663|8641|Friday|2065Q3|N|Y|N|2475499|2475710|2475140|2475414|N|N|N|N|N| +2475506|AAAAAAAACPFMFCAA|2065-08-08|1987|8641|663|2065|6|8|8|3|2065|663|8641|Saturday|2065Q3|N|Y|N|2475499|2475710|2475141|2475415|N|N|N|N|N| +2475507|AAAAAAAADPFMFCAA|2065-08-09|1987|8641|663|2065|0|8|9|3|2065|663|8641|Sunday|2065Q3|N|N|N|2475499|2475710|2475142|2475416|N|N|N|N|N| +2475508|AAAAAAAAEPFMFCAA|2065-08-10|1987|8641|663|2065|1|8|10|3|2065|663|8641|Monday|2065Q3|N|N|N|2475499|2475710|2475143|2475417|N|N|N|N|N| +2475509|AAAAAAAAFPFMFCAA|2065-08-11|1987|8642|663|2065|2|8|11|3|2065|663|8642|Tuesday|2065Q3|N|N|N|2475499|2475710|2475144|2475418|N|N|N|N|N| +2475510|AAAAAAAAGPFMFCAA|2065-08-12|1987|8642|663|2065|3|8|12|3|2065|663|8642|Wednesday|2065Q3|N|N|N|2475499|2475710|2475145|2475419|N|N|N|N|N| +2475511|AAAAAAAAHPFMFCAA|2065-08-13|1987|8642|663|2065|4|8|13|3|2065|663|8642|Thursday|2065Q3|N|N|N|2475499|2475710|2475146|2475420|N|N|N|N|N| +2475512|AAAAAAAAIPFMFCAA|2065-08-14|1987|8642|663|2065|5|8|14|3|2065|663|8642|Friday|2065Q3|N|Y|N|2475499|2475710|2475147|2475421|N|N|N|N|N| +2475513|AAAAAAAAJPFMFCAA|2065-08-15|1987|8642|663|2065|6|8|15|3|2065|663|8642|Saturday|2065Q3|N|Y|N|2475499|2475710|2475148|2475422|N|N|N|N|N| +2475514|AAAAAAAAKPFMFCAA|2065-08-16|1987|8642|663|2065|0|8|16|3|2065|663|8642|Sunday|2065Q3|N|N|N|2475499|2475710|2475149|2475423|N|N|N|N|N| +2475515|AAAAAAAALPFMFCAA|2065-08-17|1987|8642|663|2065|1|8|17|3|2065|663|8642|Monday|2065Q3|N|N|N|2475499|2475710|2475150|2475424|N|N|N|N|N| +2475516|AAAAAAAAMPFMFCAA|2065-08-18|1987|8643|663|2065|2|8|18|3|2065|663|8643|Tuesday|2065Q3|N|N|N|2475499|2475710|2475151|2475425|N|N|N|N|N| +2475517|AAAAAAAANPFMFCAA|2065-08-19|1987|8643|663|2065|3|8|19|3|2065|663|8643|Wednesday|2065Q3|N|N|N|2475499|2475710|2475152|2475426|N|N|N|N|N| +2475518|AAAAAAAAOPFMFCAA|2065-08-20|1987|8643|663|2065|4|8|20|3|2065|663|8643|Thursday|2065Q3|N|N|N|2475499|2475710|2475153|2475427|N|N|N|N|N| +2475519|AAAAAAAAPPFMFCAA|2065-08-21|1987|8643|663|2065|5|8|21|3|2065|663|8643|Friday|2065Q3|N|Y|N|2475499|2475710|2475154|2475428|N|N|N|N|N| +2475520|AAAAAAAAAAGMFCAA|2065-08-22|1987|8643|663|2065|6|8|22|3|2065|663|8643|Saturday|2065Q3|N|Y|N|2475499|2475710|2475155|2475429|N|N|N|N|N| +2475521|AAAAAAAABAGMFCAA|2065-08-23|1987|8643|663|2065|0|8|23|3|2065|663|8643|Sunday|2065Q3|N|N|N|2475499|2475710|2475156|2475430|N|N|N|N|N| +2475522|AAAAAAAACAGMFCAA|2065-08-24|1987|8643|663|2065|1|8|24|3|2065|663|8643|Monday|2065Q3|N|N|N|2475499|2475710|2475157|2475431|N|N|N|N|N| +2475523|AAAAAAAADAGMFCAA|2065-08-25|1987|8644|663|2065|2|8|25|3|2065|663|8644|Tuesday|2065Q3|N|N|N|2475499|2475710|2475158|2475432|N|N|N|N|N| +2475524|AAAAAAAAEAGMFCAA|2065-08-26|1987|8644|663|2065|3|8|26|3|2065|663|8644|Wednesday|2065Q3|N|N|N|2475499|2475710|2475159|2475433|N|N|N|N|N| +2475525|AAAAAAAAFAGMFCAA|2065-08-27|1987|8644|663|2065|4|8|27|3|2065|663|8644|Thursday|2065Q3|N|N|N|2475499|2475710|2475160|2475434|N|N|N|N|N| +2475526|AAAAAAAAGAGMFCAA|2065-08-28|1987|8644|663|2065|5|8|28|3|2065|663|8644|Friday|2065Q3|N|Y|N|2475499|2475710|2475161|2475435|N|N|N|N|N| +2475527|AAAAAAAAHAGMFCAA|2065-08-29|1987|8644|663|2065|6|8|29|3|2065|663|8644|Saturday|2065Q3|N|Y|N|2475499|2475710|2475162|2475436|N|N|N|N|N| +2475528|AAAAAAAAIAGMFCAA|2065-08-30|1987|8644|663|2065|0|8|30|3|2065|663|8644|Sunday|2065Q3|N|N|N|2475499|2475710|2475163|2475437|N|N|N|N|N| +2475529|AAAAAAAAJAGMFCAA|2065-08-31|1987|8644|663|2065|1|8|31|3|2065|663|8644|Monday|2065Q3|N|N|N|2475499|2475710|2475164|2475438|N|N|N|N|N| +2475530|AAAAAAAAKAGMFCAA|2065-09-01|1988|8645|664|2065|2|9|1|3|2065|664|8645|Tuesday|2065Q3|N|N|N|2475530|2475772|2475165|2475439|N|N|N|N|N| +2475531|AAAAAAAALAGMFCAA|2065-09-02|1988|8645|664|2065|3|9|2|3|2065|664|8645|Wednesday|2065Q3|N|N|N|2475530|2475772|2475166|2475440|N|N|N|N|N| +2475532|AAAAAAAAMAGMFCAA|2065-09-03|1988|8645|664|2065|4|9|3|3|2065|664|8645|Thursday|2065Q3|N|N|N|2475530|2475772|2475167|2475441|N|N|N|N|N| +2475533|AAAAAAAANAGMFCAA|2065-09-04|1988|8645|664|2065|5|9|4|3|2065|664|8645|Friday|2065Q3|N|Y|N|2475530|2475772|2475168|2475442|N|N|N|N|N| +2475534|AAAAAAAAOAGMFCAA|2065-09-05|1988|8645|664|2065|6|9|5|3|2065|664|8645|Saturday|2065Q3|N|Y|N|2475530|2475772|2475169|2475443|N|N|N|N|N| +2475535|AAAAAAAAPAGMFCAA|2065-09-06|1988|8645|664|2065|0|9|6|3|2065|664|8645|Sunday|2065Q3|N|N|N|2475530|2475772|2475170|2475444|N|N|N|N|N| +2475536|AAAAAAAAABGMFCAA|2065-09-07|1988|8645|664|2065|1|9|7|3|2065|664|8645|Monday|2065Q3|N|N|N|2475530|2475772|2475171|2475445|N|N|N|N|N| +2475537|AAAAAAAABBGMFCAA|2065-09-08|1988|8646|664|2065|2|9|8|3|2065|664|8646|Tuesday|2065Q3|N|N|N|2475530|2475772|2475172|2475446|N|N|N|N|N| +2475538|AAAAAAAACBGMFCAA|2065-09-09|1988|8646|664|2065|3|9|9|3|2065|664|8646|Wednesday|2065Q3|N|N|N|2475530|2475772|2475173|2475447|N|N|N|N|N| +2475539|AAAAAAAADBGMFCAA|2065-09-10|1988|8646|664|2065|4|9|10|3|2065|664|8646|Thursday|2065Q3|N|N|N|2475530|2475772|2475174|2475448|N|N|N|N|N| +2475540|AAAAAAAAEBGMFCAA|2065-09-11|1988|8646|664|2065|5|9|11|3|2065|664|8646|Friday|2065Q3|N|Y|N|2475530|2475772|2475175|2475449|N|N|N|N|N| +2475541|AAAAAAAAFBGMFCAA|2065-09-12|1988|8646|664|2065|6|9|12|3|2065|664|8646|Saturday|2065Q3|N|Y|N|2475530|2475772|2475176|2475450|N|N|N|N|N| +2475542|AAAAAAAAGBGMFCAA|2065-09-13|1988|8646|664|2065|0|9|13|3|2065|664|8646|Sunday|2065Q3|N|N|N|2475530|2475772|2475177|2475451|N|N|N|N|N| +2475543|AAAAAAAAHBGMFCAA|2065-09-14|1988|8646|664|2065|1|9|14|3|2065|664|8646|Monday|2065Q3|N|N|N|2475530|2475772|2475178|2475452|N|N|N|N|N| +2475544|AAAAAAAAIBGMFCAA|2065-09-15|1988|8647|664|2065|2|9|15|3|2065|664|8647|Tuesday|2065Q3|N|N|N|2475530|2475772|2475179|2475453|N|N|N|N|N| +2475545|AAAAAAAAJBGMFCAA|2065-09-16|1988|8647|664|2065|3|9|16|3|2065|664|8647|Wednesday|2065Q3|N|N|N|2475530|2475772|2475180|2475454|N|N|N|N|N| +2475546|AAAAAAAAKBGMFCAA|2065-09-17|1988|8647|664|2065|4|9|17|3|2065|664|8647|Thursday|2065Q3|N|N|N|2475530|2475772|2475181|2475455|N|N|N|N|N| +2475547|AAAAAAAALBGMFCAA|2065-09-18|1988|8647|664|2065|5|9|18|3|2065|664|8647|Friday|2065Q3|N|Y|N|2475530|2475772|2475182|2475456|N|N|N|N|N| +2475548|AAAAAAAAMBGMFCAA|2065-09-19|1988|8647|664|2065|6|9|19|3|2065|664|8647|Saturday|2065Q3|N|Y|N|2475530|2475772|2475183|2475457|N|N|N|N|N| +2475549|AAAAAAAANBGMFCAA|2065-09-20|1988|8647|664|2065|0|9|20|3|2065|664|8647|Sunday|2065Q3|N|N|N|2475530|2475772|2475184|2475458|N|N|N|N|N| +2475550|AAAAAAAAOBGMFCAA|2065-09-21|1988|8647|664|2065|1|9|21|3|2065|664|8647|Monday|2065Q3|N|N|N|2475530|2475772|2475185|2475459|N|N|N|N|N| +2475551|AAAAAAAAPBGMFCAA|2065-09-22|1988|8648|664|2065|2|9|22|3|2065|664|8648|Tuesday|2065Q3|N|N|N|2475530|2475772|2475186|2475460|N|N|N|N|N| +2475552|AAAAAAAAACGMFCAA|2065-09-23|1988|8648|664|2065|3|9|23|3|2065|664|8648|Wednesday|2065Q3|N|N|N|2475530|2475772|2475187|2475461|N|N|N|N|N| +2475553|AAAAAAAABCGMFCAA|2065-09-24|1988|8648|664|2065|4|9|24|3|2065|664|8648|Thursday|2065Q3|N|N|N|2475530|2475772|2475188|2475462|N|N|N|N|N| +2475554|AAAAAAAACCGMFCAA|2065-09-25|1988|8648|664|2065|5|9|25|3|2065|664|8648|Friday|2065Q3|N|Y|N|2475530|2475772|2475189|2475463|N|N|N|N|N| +2475555|AAAAAAAADCGMFCAA|2065-09-26|1988|8648|664|2065|6|9|26|3|2065|664|8648|Saturday|2065Q3|N|Y|N|2475530|2475772|2475190|2475464|N|N|N|N|N| +2475556|AAAAAAAAECGMFCAA|2065-09-27|1988|8648|664|2065|0|9|27|3|2065|664|8648|Sunday|2065Q3|N|N|N|2475530|2475772|2475191|2475465|N|N|N|N|N| +2475557|AAAAAAAAFCGMFCAA|2065-09-28|1988|8648|664|2065|1|9|28|3|2065|664|8648|Monday|2065Q3|N|N|N|2475530|2475772|2475192|2475466|N|N|N|N|N| +2475558|AAAAAAAAGCGMFCAA|2065-09-29|1988|8649|664|2065|2|9|29|3|2065|664|8649|Tuesday|2065Q3|N|N|N|2475530|2475772|2475193|2475467|N|N|N|N|N| +2475559|AAAAAAAAHCGMFCAA|2065-09-30|1988|8649|664|2065|3|9|30|3|2065|664|8649|Wednesday|2065Q3|N|N|N|2475530|2475772|2475194|2475468|N|N|N|N|N| +2475560|AAAAAAAAICGMFCAA|2065-10-01|1989|8649|664|2065|4|10|1|3|2065|664|8649|Thursday|2065Q3|N|N|N|2475560|2475832|2475195|2475468|N|N|N|N|N| +2475561|AAAAAAAAJCGMFCAA|2065-10-02|1989|8649|664|2065|5|10|2|4|2065|664|8649|Friday|2065Q4|N|Y|N|2475560|2475832|2475196|2475469|N|N|N|N|N| +2475562|AAAAAAAAKCGMFCAA|2065-10-03|1989|8649|664|2065|6|10|3|4|2065|664|8649|Saturday|2065Q4|N|Y|N|2475560|2475832|2475197|2475470|N|N|N|N|N| +2475563|AAAAAAAALCGMFCAA|2065-10-04|1989|8649|664|2065|0|10|4|4|2065|664|8649|Sunday|2065Q4|N|N|N|2475560|2475832|2475198|2475471|N|N|N|N|N| +2475564|AAAAAAAAMCGMFCAA|2065-10-05|1989|8649|664|2065|1|10|5|4|2065|664|8649|Monday|2065Q4|N|N|N|2475560|2475832|2475199|2475472|N|N|N|N|N| +2475565|AAAAAAAANCGMFCAA|2065-10-06|1989|8650|664|2065|2|10|6|4|2065|664|8650|Tuesday|2065Q4|N|N|N|2475560|2475832|2475200|2475473|N|N|N|N|N| +2475566|AAAAAAAAOCGMFCAA|2065-10-07|1989|8650|664|2065|3|10|7|4|2065|664|8650|Wednesday|2065Q4|N|N|N|2475560|2475832|2475201|2475474|N|N|N|N|N| +2475567|AAAAAAAAPCGMFCAA|2065-10-08|1989|8650|664|2065|4|10|8|4|2065|664|8650|Thursday|2065Q4|N|N|N|2475560|2475832|2475202|2475475|N|N|N|N|N| +2475568|AAAAAAAAADGMFCAA|2065-10-09|1989|8650|664|2065|5|10|9|4|2065|664|8650|Friday|2065Q4|N|Y|N|2475560|2475832|2475203|2475476|N|N|N|N|N| +2475569|AAAAAAAABDGMFCAA|2065-10-10|1989|8650|664|2065|6|10|10|4|2065|664|8650|Saturday|2065Q4|N|Y|N|2475560|2475832|2475204|2475477|N|N|N|N|N| +2475570|AAAAAAAACDGMFCAA|2065-10-11|1989|8650|664|2065|0|10|11|4|2065|664|8650|Sunday|2065Q4|N|N|N|2475560|2475832|2475205|2475478|N|N|N|N|N| +2475571|AAAAAAAADDGMFCAA|2065-10-12|1989|8650|664|2065|1|10|12|4|2065|664|8650|Monday|2065Q4|N|N|N|2475560|2475832|2475206|2475479|N|N|N|N|N| +2475572|AAAAAAAAEDGMFCAA|2065-10-13|1989|8651|664|2065|2|10|13|4|2065|664|8651|Tuesday|2065Q4|N|N|N|2475560|2475832|2475207|2475480|N|N|N|N|N| +2475573|AAAAAAAAFDGMFCAA|2065-10-14|1989|8651|664|2065|3|10|14|4|2065|664|8651|Wednesday|2065Q4|N|N|N|2475560|2475832|2475208|2475481|N|N|N|N|N| +2475574|AAAAAAAAGDGMFCAA|2065-10-15|1989|8651|664|2065|4|10|15|4|2065|664|8651|Thursday|2065Q4|N|N|N|2475560|2475832|2475209|2475482|N|N|N|N|N| +2475575|AAAAAAAAHDGMFCAA|2065-10-16|1989|8651|664|2065|5|10|16|4|2065|664|8651|Friday|2065Q4|N|Y|N|2475560|2475832|2475210|2475483|N|N|N|N|N| +2475576|AAAAAAAAIDGMFCAA|2065-10-17|1989|8651|664|2065|6|10|17|4|2065|664|8651|Saturday|2065Q4|N|Y|N|2475560|2475832|2475211|2475484|N|N|N|N|N| +2475577|AAAAAAAAJDGMFCAA|2065-10-18|1989|8651|664|2065|0|10|18|4|2065|664|8651|Sunday|2065Q4|N|N|N|2475560|2475832|2475212|2475485|N|N|N|N|N| +2475578|AAAAAAAAKDGMFCAA|2065-10-19|1989|8651|664|2065|1|10|19|4|2065|664|8651|Monday|2065Q4|N|N|N|2475560|2475832|2475213|2475486|N|N|N|N|N| +2475579|AAAAAAAALDGMFCAA|2065-10-20|1989|8652|664|2065|2|10|20|4|2065|664|8652|Tuesday|2065Q4|N|N|N|2475560|2475832|2475214|2475487|N|N|N|N|N| +2475580|AAAAAAAAMDGMFCAA|2065-10-21|1989|8652|664|2065|3|10|21|4|2065|664|8652|Wednesday|2065Q4|N|N|N|2475560|2475832|2475215|2475488|N|N|N|N|N| +2475581|AAAAAAAANDGMFCAA|2065-10-22|1989|8652|664|2065|4|10|22|4|2065|664|8652|Thursday|2065Q4|N|N|N|2475560|2475832|2475216|2475489|N|N|N|N|N| +2475582|AAAAAAAAODGMFCAA|2065-10-23|1989|8652|664|2065|5|10|23|4|2065|664|8652|Friday|2065Q4|N|Y|N|2475560|2475832|2475217|2475490|N|N|N|N|N| +2475583|AAAAAAAAPDGMFCAA|2065-10-24|1989|8652|664|2065|6|10|24|4|2065|664|8652|Saturday|2065Q4|N|Y|N|2475560|2475832|2475218|2475491|N|N|N|N|N| +2475584|AAAAAAAAAEGMFCAA|2065-10-25|1989|8652|664|2065|0|10|25|4|2065|664|8652|Sunday|2065Q4|N|N|N|2475560|2475832|2475219|2475492|N|N|N|N|N| +2475585|AAAAAAAABEGMFCAA|2065-10-26|1989|8652|664|2065|1|10|26|4|2065|664|8652|Monday|2065Q4|N|N|N|2475560|2475832|2475220|2475493|N|N|N|N|N| +2475586|AAAAAAAACEGMFCAA|2065-10-27|1989|8653|664|2065|2|10|27|4|2065|664|8653|Tuesday|2065Q4|N|N|N|2475560|2475832|2475221|2475494|N|N|N|N|N| +2475587|AAAAAAAADEGMFCAA|2065-10-28|1989|8653|664|2065|3|10|28|4|2065|664|8653|Wednesday|2065Q4|N|N|N|2475560|2475832|2475222|2475495|N|N|N|N|N| +2475588|AAAAAAAAEEGMFCAA|2065-10-29|1989|8653|664|2065|4|10|29|4|2065|664|8653|Thursday|2065Q4|N|N|N|2475560|2475832|2475223|2475496|N|N|N|N|N| +2475589|AAAAAAAAFEGMFCAA|2065-10-30|1989|8653|664|2065|5|10|30|4|2065|664|8653|Friday|2065Q4|N|Y|N|2475560|2475832|2475224|2475497|N|N|N|N|N| +2475590|AAAAAAAAGEGMFCAA|2065-10-31|1989|8653|664|2065|6|10|31|4|2065|664|8653|Saturday|2065Q4|N|Y|N|2475560|2475832|2475225|2475498|N|N|N|N|N| +2475591|AAAAAAAAHEGMFCAA|2065-11-01|1990|8653|664|2065|0|11|1|4|2065|664|8653|Sunday|2065Q4|N|N|N|2475591|2475894|2475226|2475499|N|N|N|N|N| +2475592|AAAAAAAAIEGMFCAA|2065-11-02|1990|8653|664|2065|1|11|2|4|2065|664|8653|Monday|2065Q4|N|N|N|2475591|2475894|2475227|2475500|N|N|N|N|N| +2475593|AAAAAAAAJEGMFCAA|2065-11-03|1990|8654|664|2065|2|11|3|4|2065|664|8654|Tuesday|2065Q4|N|N|N|2475591|2475894|2475228|2475501|N|N|N|N|N| +2475594|AAAAAAAAKEGMFCAA|2065-11-04|1990|8654|664|2065|3|11|4|4|2065|664|8654|Wednesday|2065Q4|N|N|N|2475591|2475894|2475229|2475502|N|N|N|N|N| +2475595|AAAAAAAALEGMFCAA|2065-11-05|1990|8654|664|2065|4|11|5|4|2065|664|8654|Thursday|2065Q4|N|N|N|2475591|2475894|2475230|2475503|N|N|N|N|N| +2475596|AAAAAAAAMEGMFCAA|2065-11-06|1990|8654|664|2065|5|11|6|4|2065|664|8654|Friday|2065Q4|N|Y|N|2475591|2475894|2475231|2475504|N|N|N|N|N| +2475597|AAAAAAAANEGMFCAA|2065-11-07|1990|8654|664|2065|6|11|7|4|2065|664|8654|Saturday|2065Q4|N|Y|N|2475591|2475894|2475232|2475505|N|N|N|N|N| +2475598|AAAAAAAAOEGMFCAA|2065-11-08|1990|8654|664|2065|0|11|8|4|2065|664|8654|Sunday|2065Q4|N|N|N|2475591|2475894|2475233|2475506|N|N|N|N|N| +2475599|AAAAAAAAPEGMFCAA|2065-11-09|1990|8654|664|2065|1|11|9|4|2065|664|8654|Monday|2065Q4|N|N|N|2475591|2475894|2475234|2475507|N|N|N|N|N| +2475600|AAAAAAAAAFGMFCAA|2065-11-10|1990|8655|664|2065|2|11|10|4|2065|664|8655|Tuesday|2065Q4|N|N|N|2475591|2475894|2475235|2475508|N|N|N|N|N| +2475601|AAAAAAAABFGMFCAA|2065-11-11|1990|8655|664|2065|3|11|11|4|2065|664|8655|Wednesday|2065Q4|N|N|N|2475591|2475894|2475236|2475509|N|N|N|N|N| +2475602|AAAAAAAACFGMFCAA|2065-11-12|1990|8655|664|2065|4|11|12|4|2065|664|8655|Thursday|2065Q4|N|N|N|2475591|2475894|2475237|2475510|N|N|N|N|N| +2475603|AAAAAAAADFGMFCAA|2065-11-13|1990|8655|664|2065|5|11|13|4|2065|664|8655|Friday|2065Q4|N|Y|N|2475591|2475894|2475238|2475511|N|N|N|N|N| +2475604|AAAAAAAAEFGMFCAA|2065-11-14|1990|8655|664|2065|6|11|14|4|2065|664|8655|Saturday|2065Q4|N|Y|N|2475591|2475894|2475239|2475512|N|N|N|N|N| +2475605|AAAAAAAAFFGMFCAA|2065-11-15|1990|8655|664|2065|0|11|15|4|2065|664|8655|Sunday|2065Q4|N|N|N|2475591|2475894|2475240|2475513|N|N|N|N|N| +2475606|AAAAAAAAGFGMFCAA|2065-11-16|1990|8655|664|2065|1|11|16|4|2065|664|8655|Monday|2065Q4|N|N|N|2475591|2475894|2475241|2475514|N|N|N|N|N| +2475607|AAAAAAAAHFGMFCAA|2065-11-17|1990|8656|664|2065|2|11|17|4|2065|664|8656|Tuesday|2065Q4|N|N|N|2475591|2475894|2475242|2475515|N|N|N|N|N| +2475608|AAAAAAAAIFGMFCAA|2065-11-18|1990|8656|664|2065|3|11|18|4|2065|664|8656|Wednesday|2065Q4|N|N|N|2475591|2475894|2475243|2475516|N|N|N|N|N| +2475609|AAAAAAAAJFGMFCAA|2065-11-19|1990|8656|664|2065|4|11|19|4|2065|664|8656|Thursday|2065Q4|N|N|N|2475591|2475894|2475244|2475517|N|N|N|N|N| +2475610|AAAAAAAAKFGMFCAA|2065-11-20|1990|8656|664|2065|5|11|20|4|2065|664|8656|Friday|2065Q4|N|Y|N|2475591|2475894|2475245|2475518|N|N|N|N|N| +2475611|AAAAAAAALFGMFCAA|2065-11-21|1990|8656|664|2065|6|11|21|4|2065|664|8656|Saturday|2065Q4|N|Y|N|2475591|2475894|2475246|2475519|N|N|N|N|N| +2475612|AAAAAAAAMFGMFCAA|2065-11-22|1990|8656|664|2065|0|11|22|4|2065|664|8656|Sunday|2065Q4|N|N|N|2475591|2475894|2475247|2475520|N|N|N|N|N| +2475613|AAAAAAAANFGMFCAA|2065-11-23|1990|8656|664|2065|1|11|23|4|2065|664|8656|Monday|2065Q4|N|N|N|2475591|2475894|2475248|2475521|N|N|N|N|N| +2475614|AAAAAAAAOFGMFCAA|2065-11-24|1990|8657|664|2065|2|11|24|4|2065|664|8657|Tuesday|2065Q4|N|N|N|2475591|2475894|2475249|2475522|N|N|N|N|N| +2475615|AAAAAAAAPFGMFCAA|2065-11-25|1990|8657|664|2065|3|11|25|4|2065|664|8657|Wednesday|2065Q4|N|N|N|2475591|2475894|2475250|2475523|N|N|N|N|N| +2475616|AAAAAAAAAGGMFCAA|2065-11-26|1990|8657|664|2065|4|11|26|4|2065|664|8657|Thursday|2065Q4|N|N|N|2475591|2475894|2475251|2475524|N|N|N|N|N| +2475617|AAAAAAAABGGMFCAA|2065-11-27|1990|8657|664|2065|5|11|27|4|2065|664|8657|Friday|2065Q4|N|Y|N|2475591|2475894|2475252|2475525|N|N|N|N|N| +2475618|AAAAAAAACGGMFCAA|2065-11-28|1990|8657|664|2065|6|11|28|4|2065|664|8657|Saturday|2065Q4|N|Y|N|2475591|2475894|2475253|2475526|N|N|N|N|N| +2475619|AAAAAAAADGGMFCAA|2065-11-29|1990|8657|664|2065|0|11|29|4|2065|664|8657|Sunday|2065Q4|N|N|N|2475591|2475894|2475254|2475527|N|N|N|N|N| +2475620|AAAAAAAAEGGMFCAA|2065-11-30|1990|8657|664|2065|1|11|30|4|2065|664|8657|Monday|2065Q4|N|N|N|2475591|2475894|2475255|2475528|N|N|N|N|N| +2475621|AAAAAAAAFGGMFCAA|2065-12-01|1991|8658|665|2065|2|12|1|4|2065|665|8658|Tuesday|2065Q4|N|N|N|2475621|2475954|2475256|2475529|N|N|N|N|N| +2475622|AAAAAAAAGGGMFCAA|2065-12-02|1991|8658|665|2065|3|12|2|4|2065|665|8658|Wednesday|2065Q4|N|N|N|2475621|2475954|2475257|2475530|N|N|N|N|N| +2475623|AAAAAAAAHGGMFCAA|2065-12-03|1991|8658|665|2065|4|12|3|4|2065|665|8658|Thursday|2065Q4|N|N|N|2475621|2475954|2475258|2475531|N|N|N|N|N| +2475624|AAAAAAAAIGGMFCAA|2065-12-04|1991|8658|665|2065|5|12|4|4|2065|665|8658|Friday|2065Q4|N|Y|N|2475621|2475954|2475259|2475532|N|N|N|N|N| +2475625|AAAAAAAAJGGMFCAA|2065-12-05|1991|8658|665|2065|6|12|5|4|2065|665|8658|Saturday|2065Q4|N|Y|N|2475621|2475954|2475260|2475533|N|N|N|N|N| +2475626|AAAAAAAAKGGMFCAA|2065-12-06|1991|8658|665|2065|0|12|6|4|2065|665|8658|Sunday|2065Q4|N|N|N|2475621|2475954|2475261|2475534|N|N|N|N|N| +2475627|AAAAAAAALGGMFCAA|2065-12-07|1991|8658|665|2065|1|12|7|4|2065|665|8658|Monday|2065Q4|N|N|N|2475621|2475954|2475262|2475535|N|N|N|N|N| +2475628|AAAAAAAAMGGMFCAA|2065-12-08|1991|8659|665|2065|2|12|8|4|2065|665|8659|Tuesday|2065Q4|N|N|N|2475621|2475954|2475263|2475536|N|N|N|N|N| +2475629|AAAAAAAANGGMFCAA|2065-12-09|1991|8659|665|2065|3|12|9|4|2065|665|8659|Wednesday|2065Q4|N|N|N|2475621|2475954|2475264|2475537|N|N|N|N|N| +2475630|AAAAAAAAOGGMFCAA|2065-12-10|1991|8659|665|2065|4|12|10|4|2065|665|8659|Thursday|2065Q4|N|N|N|2475621|2475954|2475265|2475538|N|N|N|N|N| +2475631|AAAAAAAAPGGMFCAA|2065-12-11|1991|8659|665|2065|5|12|11|4|2065|665|8659|Friday|2065Q4|N|Y|N|2475621|2475954|2475266|2475539|N|N|N|N|N| +2475632|AAAAAAAAAHGMFCAA|2065-12-12|1991|8659|665|2065|6|12|12|4|2065|665|8659|Saturday|2065Q4|N|Y|N|2475621|2475954|2475267|2475540|N|N|N|N|N| +2475633|AAAAAAAABHGMFCAA|2065-12-13|1991|8659|665|2065|0|12|13|4|2065|665|8659|Sunday|2065Q4|N|N|N|2475621|2475954|2475268|2475541|N|N|N|N|N| +2475634|AAAAAAAACHGMFCAA|2065-12-14|1991|8659|665|2065|1|12|14|4|2065|665|8659|Monday|2065Q4|N|N|N|2475621|2475954|2475269|2475542|N|N|N|N|N| +2475635|AAAAAAAADHGMFCAA|2065-12-15|1991|8660|665|2065|2|12|15|4|2065|665|8660|Tuesday|2065Q4|N|N|N|2475621|2475954|2475270|2475543|N|N|N|N|N| +2475636|AAAAAAAAEHGMFCAA|2065-12-16|1991|8660|665|2065|3|12|16|4|2065|665|8660|Wednesday|2065Q4|N|N|N|2475621|2475954|2475271|2475544|N|N|N|N|N| +2475637|AAAAAAAAFHGMFCAA|2065-12-17|1991|8660|665|2065|4|12|17|4|2065|665|8660|Thursday|2065Q4|N|N|N|2475621|2475954|2475272|2475545|N|N|N|N|N| +2475638|AAAAAAAAGHGMFCAA|2065-12-18|1991|8660|665|2065|5|12|18|4|2065|665|8660|Friday|2065Q4|N|Y|N|2475621|2475954|2475273|2475546|N|N|N|N|N| +2475639|AAAAAAAAHHGMFCAA|2065-12-19|1991|8660|665|2065|6|12|19|4|2065|665|8660|Saturday|2065Q4|N|Y|N|2475621|2475954|2475274|2475547|N|N|N|N|N| +2475640|AAAAAAAAIHGMFCAA|2065-12-20|1991|8660|665|2065|0|12|20|4|2065|665|8660|Sunday|2065Q4|N|N|N|2475621|2475954|2475275|2475548|N|N|N|N|N| +2475641|AAAAAAAAJHGMFCAA|2065-12-21|1991|8660|665|2065|1|12|21|4|2065|665|8660|Monday|2065Q4|N|N|N|2475621|2475954|2475276|2475549|N|N|N|N|N| +2475642|AAAAAAAAKHGMFCAA|2065-12-22|1991|8661|665|2065|2|12|22|4|2065|665|8661|Tuesday|2065Q4|N|N|N|2475621|2475954|2475277|2475550|N|N|N|N|N| +2475643|AAAAAAAALHGMFCAA|2065-12-23|1991|8661|665|2065|3|12|23|4|2065|665|8661|Wednesday|2065Q4|N|N|N|2475621|2475954|2475278|2475551|N|N|N|N|N| +2475644|AAAAAAAAMHGMFCAA|2065-12-24|1991|8661|665|2065|4|12|24|4|2065|665|8661|Thursday|2065Q4|N|N|N|2475621|2475954|2475279|2475552|N|N|N|N|N| +2475645|AAAAAAAANHGMFCAA|2065-12-25|1991|8661|665|2065|5|12|25|4|2065|665|8661|Friday|2065Q4|N|Y|N|2475621|2475954|2475280|2475553|N|N|N|N|N| +2475646|AAAAAAAAOHGMFCAA|2065-12-26|1991|8661|665|2065|6|12|26|4|2065|665|8661|Saturday|2065Q4|Y|Y|N|2475621|2475954|2475281|2475554|N|N|N|N|N| +2475647|AAAAAAAAPHGMFCAA|2065-12-27|1991|8661|665|2065|0|12|27|4|2065|665|8661|Sunday|2065Q4|N|N|Y|2475621|2475954|2475282|2475555|N|N|N|N|N| +2475648|AAAAAAAAAIGMFCAA|2065-12-28|1991|8661|665|2065|1|12|28|4|2065|665|8661|Monday|2065Q4|N|N|N|2475621|2475954|2475283|2475556|N|N|N|N|N| +2475649|AAAAAAAABIGMFCAA|2065-12-29|1991|8662|665|2065|2|12|29|4|2065|665|8662|Tuesday|2065Q4|N|N|N|2475621|2475954|2475284|2475557|N|N|N|N|N| +2475650|AAAAAAAACIGMFCAA|2065-12-30|1991|8662|665|2065|3|12|30|4|2065|665|8662|Wednesday|2065Q4|N|N|N|2475621|2475954|2475285|2475558|N|N|N|N|N| +2475651|AAAAAAAADIGMFCAA|2065-12-31|1991|8662|665|2065|4|12|31|4|2065|665|8662|Thursday|2065Q4|N|N|N|2475621|2475954|2475286|2475559|N|N|N|N|N| +2475652|AAAAAAAAEIGMFCAA|2066-01-01|1992|8662|665|2066|5|1|1|1|2066|665|8662|Friday|2066Q1|Y|Y|N|2475652|2475651|2475287|2475560|N|N|N|N|N| +2475653|AAAAAAAAFIGMFCAA|2066-01-02|1992|8662|665|2066|6|1|2|1|2066|665|8662|Saturday|2066Q1|N|Y|Y|2475652|2475651|2475288|2475561|N|N|N|N|N| +2475654|AAAAAAAAGIGMFCAA|2066-01-03|1992|8662|665|2066|0|1|3|1|2066|665|8662|Sunday|2066Q1|N|N|N|2475652|2475651|2475289|2475562|N|N|N|N|N| +2475655|AAAAAAAAHIGMFCAA|2066-01-04|1992|8662|665|2066|1|1|4|1|2066|665|8662|Monday|2066Q1|N|N|N|2475652|2475651|2475290|2475563|N|N|N|N|N| +2475656|AAAAAAAAIIGMFCAA|2066-01-05|1992|8663|665|2066|2|1|5|1|2066|665|8663|Tuesday|2066Q1|N|N|N|2475652|2475651|2475291|2475564|N|N|N|N|N| +2475657|AAAAAAAAJIGMFCAA|2066-01-06|1992|8663|665|2066|3|1|6|1|2066|665|8663|Wednesday|2066Q1|N|N|N|2475652|2475651|2475292|2475565|N|N|N|N|N| +2475658|AAAAAAAAKIGMFCAA|2066-01-07|1992|8663|665|2066|4|1|7|1|2066|665|8663|Thursday|2066Q1|N|N|N|2475652|2475651|2475293|2475566|N|N|N|N|N| +2475659|AAAAAAAALIGMFCAA|2066-01-08|1992|8663|665|2066|5|1|8|1|2066|665|8663|Friday|2066Q1|N|Y|N|2475652|2475651|2475294|2475567|N|N|N|N|N| +2475660|AAAAAAAAMIGMFCAA|2066-01-09|1992|8663|665|2066|6|1|9|1|2066|665|8663|Saturday|2066Q1|N|Y|N|2475652|2475651|2475295|2475568|N|N|N|N|N| +2475661|AAAAAAAANIGMFCAA|2066-01-10|1992|8663|665|2066|0|1|10|1|2066|665|8663|Sunday|2066Q1|N|N|N|2475652|2475651|2475296|2475569|N|N|N|N|N| +2475662|AAAAAAAAOIGMFCAA|2066-01-11|1992|8663|665|2066|1|1|11|1|2066|665|8663|Monday|2066Q1|N|N|N|2475652|2475651|2475297|2475570|N|N|N|N|N| +2475663|AAAAAAAAPIGMFCAA|2066-01-12|1992|8664|665|2066|2|1|12|1|2066|665|8664|Tuesday|2066Q1|N|N|N|2475652|2475651|2475298|2475571|N|N|N|N|N| +2475664|AAAAAAAAAJGMFCAA|2066-01-13|1992|8664|665|2066|3|1|13|1|2066|665|8664|Wednesday|2066Q1|N|N|N|2475652|2475651|2475299|2475572|N|N|N|N|N| +2475665|AAAAAAAABJGMFCAA|2066-01-14|1992|8664|665|2066|4|1|14|1|2066|665|8664|Thursday|2066Q1|N|N|N|2475652|2475651|2475300|2475573|N|N|N|N|N| +2475666|AAAAAAAACJGMFCAA|2066-01-15|1992|8664|665|2066|5|1|15|1|2066|665|8664|Friday|2066Q1|N|Y|N|2475652|2475651|2475301|2475574|N|N|N|N|N| +2475667|AAAAAAAADJGMFCAA|2066-01-16|1992|8664|665|2066|6|1|16|1|2066|665|8664|Saturday|2066Q1|N|Y|N|2475652|2475651|2475302|2475575|N|N|N|N|N| +2475668|AAAAAAAAEJGMFCAA|2066-01-17|1992|8664|665|2066|0|1|17|1|2066|665|8664|Sunday|2066Q1|N|N|N|2475652|2475651|2475303|2475576|N|N|N|N|N| +2475669|AAAAAAAAFJGMFCAA|2066-01-18|1992|8664|665|2066|1|1|18|1|2066|665|8664|Monday|2066Q1|N|N|N|2475652|2475651|2475304|2475577|N|N|N|N|N| +2475670|AAAAAAAAGJGMFCAA|2066-01-19|1992|8665|665|2066|2|1|19|1|2066|665|8665|Tuesday|2066Q1|N|N|N|2475652|2475651|2475305|2475578|N|N|N|N|N| +2475671|AAAAAAAAHJGMFCAA|2066-01-20|1992|8665|665|2066|3|1|20|1|2066|665|8665|Wednesday|2066Q1|N|N|N|2475652|2475651|2475306|2475579|N|N|N|N|N| +2475672|AAAAAAAAIJGMFCAA|2066-01-21|1992|8665|665|2066|4|1|21|1|2066|665|8665|Thursday|2066Q1|N|N|N|2475652|2475651|2475307|2475580|N|N|N|N|N| +2475673|AAAAAAAAJJGMFCAA|2066-01-22|1992|8665|665|2066|5|1|22|1|2066|665|8665|Friday|2066Q1|N|Y|N|2475652|2475651|2475308|2475581|N|N|N|N|N| +2475674|AAAAAAAAKJGMFCAA|2066-01-23|1992|8665|665|2066|6|1|23|1|2066|665|8665|Saturday|2066Q1|N|Y|N|2475652|2475651|2475309|2475582|N|N|N|N|N| +2475675|AAAAAAAALJGMFCAA|2066-01-24|1992|8665|665|2066|0|1|24|1|2066|665|8665|Sunday|2066Q1|N|N|N|2475652|2475651|2475310|2475583|N|N|N|N|N| +2475676|AAAAAAAAMJGMFCAA|2066-01-25|1992|8665|665|2066|1|1|25|1|2066|665|8665|Monday|2066Q1|N|N|N|2475652|2475651|2475311|2475584|N|N|N|N|N| +2475677|AAAAAAAANJGMFCAA|2066-01-26|1992|8666|665|2066|2|1|26|1|2066|665|8666|Tuesday|2066Q1|N|N|N|2475652|2475651|2475312|2475585|N|N|N|N|N| +2475678|AAAAAAAAOJGMFCAA|2066-01-27|1992|8666|665|2066|3|1|27|1|2066|665|8666|Wednesday|2066Q1|N|N|N|2475652|2475651|2475313|2475586|N|N|N|N|N| +2475679|AAAAAAAAPJGMFCAA|2066-01-28|1992|8666|665|2066|4|1|28|1|2066|665|8666|Thursday|2066Q1|N|N|N|2475652|2475651|2475314|2475587|N|N|N|N|N| +2475680|AAAAAAAAAKGMFCAA|2066-01-29|1992|8666|665|2066|5|1|29|1|2066|665|8666|Friday|2066Q1|N|Y|N|2475652|2475651|2475315|2475588|N|N|N|N|N| +2475681|AAAAAAAABKGMFCAA|2066-01-30|1992|8666|665|2066|6|1|30|1|2066|665|8666|Saturday|2066Q1|N|Y|N|2475652|2475651|2475316|2475589|N|N|N|N|N| +2475682|AAAAAAAACKGMFCAA|2066-01-31|1992|8666|665|2066|0|1|31|1|2066|665|8666|Sunday|2066Q1|N|N|N|2475652|2475651|2475317|2475590|N|N|N|N|N| +2475683|AAAAAAAADKGMFCAA|2066-02-01|1993|8666|665|2066|1|2|1|1|2066|665|8666|Monday|2066Q1|N|N|N|2475683|2475713|2475318|2475591|N|N|N|N|N| +2475684|AAAAAAAAEKGMFCAA|2066-02-02|1993|8667|665|2066|2|2|2|1|2066|665|8667|Tuesday|2066Q1|N|N|N|2475683|2475713|2475319|2475592|N|N|N|N|N| +2475685|AAAAAAAAFKGMFCAA|2066-02-03|1993|8667|665|2066|3|2|3|1|2066|665|8667|Wednesday|2066Q1|N|N|N|2475683|2475713|2475320|2475593|N|N|N|N|N| +2475686|AAAAAAAAGKGMFCAA|2066-02-04|1993|8667|665|2066|4|2|4|1|2066|665|8667|Thursday|2066Q1|N|N|N|2475683|2475713|2475321|2475594|N|N|N|N|N| +2475687|AAAAAAAAHKGMFCAA|2066-02-05|1993|8667|665|2066|5|2|5|1|2066|665|8667|Friday|2066Q1|N|Y|N|2475683|2475713|2475322|2475595|N|N|N|N|N| +2475688|AAAAAAAAIKGMFCAA|2066-02-06|1993|8667|665|2066|6|2|6|1|2066|665|8667|Saturday|2066Q1|N|Y|N|2475683|2475713|2475323|2475596|N|N|N|N|N| +2475689|AAAAAAAAJKGMFCAA|2066-02-07|1993|8667|665|2066|0|2|7|1|2066|665|8667|Sunday|2066Q1|N|N|N|2475683|2475713|2475324|2475597|N|N|N|N|N| +2475690|AAAAAAAAKKGMFCAA|2066-02-08|1993|8667|665|2066|1|2|8|1|2066|665|8667|Monday|2066Q1|N|N|N|2475683|2475713|2475325|2475598|N|N|N|N|N| +2475691|AAAAAAAALKGMFCAA|2066-02-09|1993|8668|665|2066|2|2|9|1|2066|665|8668|Tuesday|2066Q1|N|N|N|2475683|2475713|2475326|2475599|N|N|N|N|N| +2475692|AAAAAAAAMKGMFCAA|2066-02-10|1993|8668|665|2066|3|2|10|1|2066|665|8668|Wednesday|2066Q1|N|N|N|2475683|2475713|2475327|2475600|N|N|N|N|N| +2475693|AAAAAAAANKGMFCAA|2066-02-11|1993|8668|665|2066|4|2|11|1|2066|665|8668|Thursday|2066Q1|N|N|N|2475683|2475713|2475328|2475601|N|N|N|N|N| +2475694|AAAAAAAAOKGMFCAA|2066-02-12|1993|8668|665|2066|5|2|12|1|2066|665|8668|Friday|2066Q1|N|Y|N|2475683|2475713|2475329|2475602|N|N|N|N|N| +2475695|AAAAAAAAPKGMFCAA|2066-02-13|1993|8668|665|2066|6|2|13|1|2066|665|8668|Saturday|2066Q1|N|Y|N|2475683|2475713|2475330|2475603|N|N|N|N|N| +2475696|AAAAAAAAALGMFCAA|2066-02-14|1993|8668|665|2066|0|2|14|1|2066|665|8668|Sunday|2066Q1|N|N|N|2475683|2475713|2475331|2475604|N|N|N|N|N| +2475697|AAAAAAAABLGMFCAA|2066-02-15|1993|8668|665|2066|1|2|15|1|2066|665|8668|Monday|2066Q1|N|N|N|2475683|2475713|2475332|2475605|N|N|N|N|N| +2475698|AAAAAAAACLGMFCAA|2066-02-16|1993|8669|665|2066|2|2|16|1|2066|665|8669|Tuesday|2066Q1|N|N|N|2475683|2475713|2475333|2475606|N|N|N|N|N| +2475699|AAAAAAAADLGMFCAA|2066-02-17|1993|8669|665|2066|3|2|17|1|2066|665|8669|Wednesday|2066Q1|N|N|N|2475683|2475713|2475334|2475607|N|N|N|N|N| +2475700|AAAAAAAAELGMFCAA|2066-02-18|1993|8669|665|2066|4|2|18|1|2066|665|8669|Thursday|2066Q1|N|N|N|2475683|2475713|2475335|2475608|N|N|N|N|N| +2475701|AAAAAAAAFLGMFCAA|2066-02-19|1993|8669|665|2066|5|2|19|1|2066|665|8669|Friday|2066Q1|N|Y|N|2475683|2475713|2475336|2475609|N|N|N|N|N| +2475702|AAAAAAAAGLGMFCAA|2066-02-20|1993|8669|665|2066|6|2|20|1|2066|665|8669|Saturday|2066Q1|N|Y|N|2475683|2475713|2475337|2475610|N|N|N|N|N| +2475703|AAAAAAAAHLGMFCAA|2066-02-21|1993|8669|665|2066|0|2|21|1|2066|665|8669|Sunday|2066Q1|N|N|N|2475683|2475713|2475338|2475611|N|N|N|N|N| +2475704|AAAAAAAAILGMFCAA|2066-02-22|1993|8669|665|2066|1|2|22|1|2066|665|8669|Monday|2066Q1|N|N|N|2475683|2475713|2475339|2475612|N|N|N|N|N| +2475705|AAAAAAAAJLGMFCAA|2066-02-23|1993|8670|665|2066|2|2|23|1|2066|665|8670|Tuesday|2066Q1|N|N|N|2475683|2475713|2475340|2475613|N|N|N|N|N| +2475706|AAAAAAAAKLGMFCAA|2066-02-24|1993|8670|665|2066|3|2|24|1|2066|665|8670|Wednesday|2066Q1|N|N|N|2475683|2475713|2475341|2475614|N|N|N|N|N| +2475707|AAAAAAAALLGMFCAA|2066-02-25|1993|8670|665|2066|4|2|25|1|2066|665|8670|Thursday|2066Q1|N|N|N|2475683|2475713|2475342|2475615|N|N|N|N|N| +2475708|AAAAAAAAMLGMFCAA|2066-02-26|1993|8670|665|2066|5|2|26|1|2066|665|8670|Friday|2066Q1|N|Y|N|2475683|2475713|2475343|2475616|N|N|N|N|N| +2475709|AAAAAAAANLGMFCAA|2066-02-27|1993|8670|665|2066|6|2|27|1|2066|665|8670|Saturday|2066Q1|N|Y|N|2475683|2475713|2475344|2475617|N|N|N|N|N| +2475710|AAAAAAAAOLGMFCAA|2066-02-28|1993|8670|665|2066|0|2|28|1|2066|665|8670|Sunday|2066Q1|N|N|N|2475683|2475713|2475345|2475618|N|N|N|N|N| +2475711|AAAAAAAAPLGMFCAA|2066-03-01|1994|8670|666|2066|1|3|1|1|2066|666|8670|Monday|2066Q1|N|N|N|2475711|2475769|2475346|2475619|N|N|N|N|N| +2475712|AAAAAAAAAMGMFCAA|2066-03-02|1994|8671|666|2066|2|3|2|1|2066|666|8671|Tuesday|2066Q1|N|N|N|2475711|2475769|2475347|2475620|N|N|N|N|N| +2475713|AAAAAAAABMGMFCAA|2066-03-03|1994|8671|666|2066|3|3|3|1|2066|666|8671|Wednesday|2066Q1|N|N|N|2475711|2475769|2475348|2475621|N|N|N|N|N| +2475714|AAAAAAAACMGMFCAA|2066-03-04|1994|8671|666|2066|4|3|4|1|2066|666|8671|Thursday|2066Q1|N|N|N|2475711|2475769|2475349|2475622|N|N|N|N|N| +2475715|AAAAAAAADMGMFCAA|2066-03-05|1994|8671|666|2066|5|3|5|1|2066|666|8671|Friday|2066Q1|N|Y|N|2475711|2475769|2475350|2475623|N|N|N|N|N| +2475716|AAAAAAAAEMGMFCAA|2066-03-06|1994|8671|666|2066|6|3|6|1|2066|666|8671|Saturday|2066Q1|N|Y|N|2475711|2475769|2475351|2475624|N|N|N|N|N| +2475717|AAAAAAAAFMGMFCAA|2066-03-07|1994|8671|666|2066|0|3|7|1|2066|666|8671|Sunday|2066Q1|N|N|N|2475711|2475769|2475352|2475625|N|N|N|N|N| +2475718|AAAAAAAAGMGMFCAA|2066-03-08|1994|8671|666|2066|1|3|8|1|2066|666|8671|Monday|2066Q1|N|N|N|2475711|2475769|2475353|2475626|N|N|N|N|N| +2475719|AAAAAAAAHMGMFCAA|2066-03-09|1994|8672|666|2066|2|3|9|1|2066|666|8672|Tuesday|2066Q1|N|N|N|2475711|2475769|2475354|2475627|N|N|N|N|N| +2475720|AAAAAAAAIMGMFCAA|2066-03-10|1994|8672|666|2066|3|3|10|1|2066|666|8672|Wednesday|2066Q1|N|N|N|2475711|2475769|2475355|2475628|N|N|N|N|N| +2475721|AAAAAAAAJMGMFCAA|2066-03-11|1994|8672|666|2066|4|3|11|1|2066|666|8672|Thursday|2066Q1|N|N|N|2475711|2475769|2475356|2475629|N|N|N|N|N| +2475722|AAAAAAAAKMGMFCAA|2066-03-12|1994|8672|666|2066|5|3|12|1|2066|666|8672|Friday|2066Q1|N|Y|N|2475711|2475769|2475357|2475630|N|N|N|N|N| +2475723|AAAAAAAALMGMFCAA|2066-03-13|1994|8672|666|2066|6|3|13|1|2066|666|8672|Saturday|2066Q1|N|Y|N|2475711|2475769|2475358|2475631|N|N|N|N|N| +2475724|AAAAAAAAMMGMFCAA|2066-03-14|1994|8672|666|2066|0|3|14|1|2066|666|8672|Sunday|2066Q1|N|N|N|2475711|2475769|2475359|2475632|N|N|N|N|N| +2475725|AAAAAAAANMGMFCAA|2066-03-15|1994|8672|666|2066|1|3|15|1|2066|666|8672|Monday|2066Q1|N|N|N|2475711|2475769|2475360|2475633|N|N|N|N|N| +2475726|AAAAAAAAOMGMFCAA|2066-03-16|1994|8673|666|2066|2|3|16|1|2066|666|8673|Tuesday|2066Q1|N|N|N|2475711|2475769|2475361|2475634|N|N|N|N|N| +2475727|AAAAAAAAPMGMFCAA|2066-03-17|1994|8673|666|2066|3|3|17|1|2066|666|8673|Wednesday|2066Q1|N|N|N|2475711|2475769|2475362|2475635|N|N|N|N|N| +2475728|AAAAAAAAANGMFCAA|2066-03-18|1994|8673|666|2066|4|3|18|1|2066|666|8673|Thursday|2066Q1|N|N|N|2475711|2475769|2475363|2475636|N|N|N|N|N| +2475729|AAAAAAAABNGMFCAA|2066-03-19|1994|8673|666|2066|5|3|19|1|2066|666|8673|Friday|2066Q1|N|Y|N|2475711|2475769|2475364|2475637|N|N|N|N|N| +2475730|AAAAAAAACNGMFCAA|2066-03-20|1994|8673|666|2066|6|3|20|1|2066|666|8673|Saturday|2066Q1|N|Y|N|2475711|2475769|2475365|2475638|N|N|N|N|N| +2475731|AAAAAAAADNGMFCAA|2066-03-21|1994|8673|666|2066|0|3|21|1|2066|666|8673|Sunday|2066Q1|N|N|N|2475711|2475769|2475366|2475639|N|N|N|N|N| +2475732|AAAAAAAAENGMFCAA|2066-03-22|1994|8673|666|2066|1|3|22|1|2066|666|8673|Monday|2066Q1|N|N|N|2475711|2475769|2475367|2475640|N|N|N|N|N| +2475733|AAAAAAAAFNGMFCAA|2066-03-23|1994|8674|666|2066|2|3|23|1|2066|666|8674|Tuesday|2066Q1|N|N|N|2475711|2475769|2475368|2475641|N|N|N|N|N| +2475734|AAAAAAAAGNGMFCAA|2066-03-24|1994|8674|666|2066|3|3|24|1|2066|666|8674|Wednesday|2066Q1|N|N|N|2475711|2475769|2475369|2475642|N|N|N|N|N| +2475735|AAAAAAAAHNGMFCAA|2066-03-25|1994|8674|666|2066|4|3|25|1|2066|666|8674|Thursday|2066Q1|N|N|N|2475711|2475769|2475370|2475643|N|N|N|N|N| +2475736|AAAAAAAAINGMFCAA|2066-03-26|1994|8674|666|2066|5|3|26|1|2066|666|8674|Friday|2066Q1|N|Y|N|2475711|2475769|2475371|2475644|N|N|N|N|N| +2475737|AAAAAAAAJNGMFCAA|2066-03-27|1994|8674|666|2066|6|3|27|1|2066|666|8674|Saturday|2066Q1|N|Y|N|2475711|2475769|2475372|2475645|N|N|N|N|N| +2475738|AAAAAAAAKNGMFCAA|2066-03-28|1994|8674|666|2066|0|3|28|1|2066|666|8674|Sunday|2066Q1|N|N|N|2475711|2475769|2475373|2475646|N|N|N|N|N| +2475739|AAAAAAAALNGMFCAA|2066-03-29|1994|8674|666|2066|1|3|29|1|2066|666|8674|Monday|2066Q1|N|N|N|2475711|2475769|2475374|2475647|N|N|N|N|N| +2475740|AAAAAAAAMNGMFCAA|2066-03-30|1994|8675|666|2066|2|3|30|1|2066|666|8675|Tuesday|2066Q1|N|N|N|2475711|2475769|2475375|2475648|N|N|N|N|N| +2475741|AAAAAAAANNGMFCAA|2066-03-31|1994|8675|666|2066|3|3|31|1|2066|666|8675|Wednesday|2066Q1|N|N|N|2475711|2475769|2475376|2475649|N|N|N|N|N| +2475742|AAAAAAAAONGMFCAA|2066-04-01|1995|8675|666|2066|4|4|1|1|2066|666|8675|Thursday|2066Q1|N|N|N|2475742|2475831|2475377|2475652|N|N|N|N|N| +2475743|AAAAAAAAPNGMFCAA|2066-04-02|1995|8675|666|2066|5|4|2|2|2066|666|8675|Friday|2066Q2|N|Y|N|2475742|2475831|2475378|2475653|N|N|N|N|N| +2475744|AAAAAAAAAOGMFCAA|2066-04-03|1995|8675|666|2066|6|4|3|2|2066|666|8675|Saturday|2066Q2|N|Y|N|2475742|2475831|2475379|2475654|N|N|N|N|N| +2475745|AAAAAAAABOGMFCAA|2066-04-04|1995|8675|666|2066|0|4|4|2|2066|666|8675|Sunday|2066Q2|N|N|N|2475742|2475831|2475380|2475655|N|N|N|N|N| +2475746|AAAAAAAACOGMFCAA|2066-04-05|1995|8675|666|2066|1|4|5|2|2066|666|8675|Monday|2066Q2|N|N|N|2475742|2475831|2475381|2475656|N|N|N|N|N| +2475747|AAAAAAAADOGMFCAA|2066-04-06|1995|8676|666|2066|2|4|6|2|2066|666|8676|Tuesday|2066Q2|N|N|N|2475742|2475831|2475382|2475657|N|N|N|N|N| +2475748|AAAAAAAAEOGMFCAA|2066-04-07|1995|8676|666|2066|3|4|7|2|2066|666|8676|Wednesday|2066Q2|N|N|N|2475742|2475831|2475383|2475658|N|N|N|N|N| +2475749|AAAAAAAAFOGMFCAA|2066-04-08|1995|8676|666|2066|4|4|8|2|2066|666|8676|Thursday|2066Q2|N|N|N|2475742|2475831|2475384|2475659|N|N|N|N|N| +2475750|AAAAAAAAGOGMFCAA|2066-04-09|1995|8676|666|2066|5|4|9|2|2066|666|8676|Friday|2066Q2|N|Y|N|2475742|2475831|2475385|2475660|N|N|N|N|N| +2475751|AAAAAAAAHOGMFCAA|2066-04-10|1995|8676|666|2066|6|4|10|2|2066|666|8676|Saturday|2066Q2|N|Y|N|2475742|2475831|2475386|2475661|N|N|N|N|N| +2475752|AAAAAAAAIOGMFCAA|2066-04-11|1995|8676|666|2066|0|4|11|2|2066|666|8676|Sunday|2066Q2|N|N|N|2475742|2475831|2475387|2475662|N|N|N|N|N| +2475753|AAAAAAAAJOGMFCAA|2066-04-12|1995|8676|666|2066|1|4|12|2|2066|666|8676|Monday|2066Q2|N|N|N|2475742|2475831|2475388|2475663|N|N|N|N|N| +2475754|AAAAAAAAKOGMFCAA|2066-04-13|1995|8677|666|2066|2|4|13|2|2066|666|8677|Tuesday|2066Q2|N|N|N|2475742|2475831|2475389|2475664|N|N|N|N|N| +2475755|AAAAAAAALOGMFCAA|2066-04-14|1995|8677|666|2066|3|4|14|2|2066|666|8677|Wednesday|2066Q2|N|N|N|2475742|2475831|2475390|2475665|N|N|N|N|N| +2475756|AAAAAAAAMOGMFCAA|2066-04-15|1995|8677|666|2066|4|4|15|2|2066|666|8677|Thursday|2066Q2|N|N|N|2475742|2475831|2475391|2475666|N|N|N|N|N| +2475757|AAAAAAAANOGMFCAA|2066-04-16|1995|8677|666|2066|5|4|16|2|2066|666|8677|Friday|2066Q2|N|Y|N|2475742|2475831|2475392|2475667|N|N|N|N|N| +2475758|AAAAAAAAOOGMFCAA|2066-04-17|1995|8677|666|2066|6|4|17|2|2066|666|8677|Saturday|2066Q2|N|Y|N|2475742|2475831|2475393|2475668|N|N|N|N|N| +2475759|AAAAAAAAPOGMFCAA|2066-04-18|1995|8677|666|2066|0|4|18|2|2066|666|8677|Sunday|2066Q2|N|N|N|2475742|2475831|2475394|2475669|N|N|N|N|N| +2475760|AAAAAAAAAPGMFCAA|2066-04-19|1995|8677|666|2066|1|4|19|2|2066|666|8677|Monday|2066Q2|N|N|N|2475742|2475831|2475395|2475670|N|N|N|N|N| +2475761|AAAAAAAABPGMFCAA|2066-04-20|1995|8678|666|2066|2|4|20|2|2066|666|8678|Tuesday|2066Q2|N|N|N|2475742|2475831|2475396|2475671|N|N|N|N|N| +2475762|AAAAAAAACPGMFCAA|2066-04-21|1995|8678|666|2066|3|4|21|2|2066|666|8678|Wednesday|2066Q2|N|N|N|2475742|2475831|2475397|2475672|N|N|N|N|N| +2475763|AAAAAAAADPGMFCAA|2066-04-22|1995|8678|666|2066|4|4|22|2|2066|666|8678|Thursday|2066Q2|N|N|N|2475742|2475831|2475398|2475673|N|N|N|N|N| +2475764|AAAAAAAAEPGMFCAA|2066-04-23|1995|8678|666|2066|5|4|23|2|2066|666|8678|Friday|2066Q2|N|Y|N|2475742|2475831|2475399|2475674|N|N|N|N|N| +2475765|AAAAAAAAFPGMFCAA|2066-04-24|1995|8678|666|2066|6|4|24|2|2066|666|8678|Saturday|2066Q2|N|Y|N|2475742|2475831|2475400|2475675|N|N|N|N|N| +2475766|AAAAAAAAGPGMFCAA|2066-04-25|1995|8678|666|2066|0|4|25|2|2066|666|8678|Sunday|2066Q2|N|N|N|2475742|2475831|2475401|2475676|N|N|N|N|N| +2475767|AAAAAAAAHPGMFCAA|2066-04-26|1995|8678|666|2066|1|4|26|2|2066|666|8678|Monday|2066Q2|N|N|N|2475742|2475831|2475402|2475677|N|N|N|N|N| +2475768|AAAAAAAAIPGMFCAA|2066-04-27|1995|8679|666|2066|2|4|27|2|2066|666|8679|Tuesday|2066Q2|N|N|N|2475742|2475831|2475403|2475678|N|N|N|N|N| +2475769|AAAAAAAAJPGMFCAA|2066-04-28|1995|8679|666|2066|3|4|28|2|2066|666|8679|Wednesday|2066Q2|N|N|N|2475742|2475831|2475404|2475679|N|N|N|N|N| +2475770|AAAAAAAAKPGMFCAA|2066-04-29|1995|8679|666|2066|4|4|29|2|2066|666|8679|Thursday|2066Q2|N|N|N|2475742|2475831|2475405|2475680|N|N|N|N|N| +2475771|AAAAAAAALPGMFCAA|2066-04-30|1995|8679|666|2066|5|4|30|2|2066|666|8679|Friday|2066Q2|N|Y|N|2475742|2475831|2475406|2475681|N|N|N|N|N| +2475772|AAAAAAAAMPGMFCAA|2066-05-01|1996|8679|666|2066|6|5|1|2|2066|666|8679|Saturday|2066Q2|N|Y|N|2475772|2475891|2475407|2475682|N|N|N|N|N| +2475773|AAAAAAAANPGMFCAA|2066-05-02|1996|8679|666|2066|0|5|2|2|2066|666|8679|Sunday|2066Q2|N|N|N|2475772|2475891|2475408|2475683|N|N|N|N|N| +2475774|AAAAAAAAOPGMFCAA|2066-05-03|1996|8679|666|2066|1|5|3|2|2066|666|8679|Monday|2066Q2|N|N|N|2475772|2475891|2475409|2475684|N|N|N|N|N| +2475775|AAAAAAAAPPGMFCAA|2066-05-04|1996|8680|666|2066|2|5|4|2|2066|666|8680|Tuesday|2066Q2|N|N|N|2475772|2475891|2475410|2475685|N|N|N|N|N| +2475776|AAAAAAAAAAHMFCAA|2066-05-05|1996|8680|666|2066|3|5|5|2|2066|666|8680|Wednesday|2066Q2|N|N|N|2475772|2475891|2475411|2475686|N|N|N|N|N| +2475777|AAAAAAAABAHMFCAA|2066-05-06|1996|8680|666|2066|4|5|6|2|2066|666|8680|Thursday|2066Q2|N|N|N|2475772|2475891|2475412|2475687|N|N|N|N|N| +2475778|AAAAAAAACAHMFCAA|2066-05-07|1996|8680|666|2066|5|5|7|2|2066|666|8680|Friday|2066Q2|N|Y|N|2475772|2475891|2475413|2475688|N|N|N|N|N| +2475779|AAAAAAAADAHMFCAA|2066-05-08|1996|8680|666|2066|6|5|8|2|2066|666|8680|Saturday|2066Q2|N|Y|N|2475772|2475891|2475414|2475689|N|N|N|N|N| +2475780|AAAAAAAAEAHMFCAA|2066-05-09|1996|8680|666|2066|0|5|9|2|2066|666|8680|Sunday|2066Q2|N|N|N|2475772|2475891|2475415|2475690|N|N|N|N|N| +2475781|AAAAAAAAFAHMFCAA|2066-05-10|1996|8680|666|2066|1|5|10|2|2066|666|8680|Monday|2066Q2|N|N|N|2475772|2475891|2475416|2475691|N|N|N|N|N| +2475782|AAAAAAAAGAHMFCAA|2066-05-11|1996|8681|666|2066|2|5|11|2|2066|666|8681|Tuesday|2066Q2|N|N|N|2475772|2475891|2475417|2475692|N|N|N|N|N| +2475783|AAAAAAAAHAHMFCAA|2066-05-12|1996|8681|666|2066|3|5|12|2|2066|666|8681|Wednesday|2066Q2|N|N|N|2475772|2475891|2475418|2475693|N|N|N|N|N| +2475784|AAAAAAAAIAHMFCAA|2066-05-13|1996|8681|666|2066|4|5|13|2|2066|666|8681|Thursday|2066Q2|N|N|N|2475772|2475891|2475419|2475694|N|N|N|N|N| +2475785|AAAAAAAAJAHMFCAA|2066-05-14|1996|8681|666|2066|5|5|14|2|2066|666|8681|Friday|2066Q2|N|Y|N|2475772|2475891|2475420|2475695|N|N|N|N|N| +2475786|AAAAAAAAKAHMFCAA|2066-05-15|1996|8681|666|2066|6|5|15|2|2066|666|8681|Saturday|2066Q2|N|Y|N|2475772|2475891|2475421|2475696|N|N|N|N|N| +2475787|AAAAAAAALAHMFCAA|2066-05-16|1996|8681|666|2066|0|5|16|2|2066|666|8681|Sunday|2066Q2|N|N|N|2475772|2475891|2475422|2475697|N|N|N|N|N| +2475788|AAAAAAAAMAHMFCAA|2066-05-17|1996|8681|666|2066|1|5|17|2|2066|666|8681|Monday|2066Q2|N|N|N|2475772|2475891|2475423|2475698|N|N|N|N|N| +2475789|AAAAAAAANAHMFCAA|2066-05-18|1996|8682|666|2066|2|5|18|2|2066|666|8682|Tuesday|2066Q2|N|N|N|2475772|2475891|2475424|2475699|N|N|N|N|N| +2475790|AAAAAAAAOAHMFCAA|2066-05-19|1996|8682|666|2066|3|5|19|2|2066|666|8682|Wednesday|2066Q2|N|N|N|2475772|2475891|2475425|2475700|N|N|N|N|N| +2475791|AAAAAAAAPAHMFCAA|2066-05-20|1996|8682|666|2066|4|5|20|2|2066|666|8682|Thursday|2066Q2|N|N|N|2475772|2475891|2475426|2475701|N|N|N|N|N| +2475792|AAAAAAAAABHMFCAA|2066-05-21|1996|8682|666|2066|5|5|21|2|2066|666|8682|Friday|2066Q2|N|Y|N|2475772|2475891|2475427|2475702|N|N|N|N|N| +2475793|AAAAAAAABBHMFCAA|2066-05-22|1996|8682|666|2066|6|5|22|2|2066|666|8682|Saturday|2066Q2|N|Y|N|2475772|2475891|2475428|2475703|N|N|N|N|N| +2475794|AAAAAAAACBHMFCAA|2066-05-23|1996|8682|666|2066|0|5|23|2|2066|666|8682|Sunday|2066Q2|N|N|N|2475772|2475891|2475429|2475704|N|N|N|N|N| +2475795|AAAAAAAADBHMFCAA|2066-05-24|1996|8682|666|2066|1|5|24|2|2066|666|8682|Monday|2066Q2|N|N|N|2475772|2475891|2475430|2475705|N|N|N|N|N| +2475796|AAAAAAAAEBHMFCAA|2066-05-25|1996|8683|666|2066|2|5|25|2|2066|666|8683|Tuesday|2066Q2|N|N|N|2475772|2475891|2475431|2475706|N|N|N|N|N| +2475797|AAAAAAAAFBHMFCAA|2066-05-26|1996|8683|666|2066|3|5|26|2|2066|666|8683|Wednesday|2066Q2|N|N|N|2475772|2475891|2475432|2475707|N|N|N|N|N| +2475798|AAAAAAAAGBHMFCAA|2066-05-27|1996|8683|666|2066|4|5|27|2|2066|666|8683|Thursday|2066Q2|N|N|N|2475772|2475891|2475433|2475708|N|N|N|N|N| +2475799|AAAAAAAAHBHMFCAA|2066-05-28|1996|8683|666|2066|5|5|28|2|2066|666|8683|Friday|2066Q2|N|Y|N|2475772|2475891|2475434|2475709|N|N|N|N|N| +2475800|AAAAAAAAIBHMFCAA|2066-05-29|1996|8683|666|2066|6|5|29|2|2066|666|8683|Saturday|2066Q2|N|Y|N|2475772|2475891|2475435|2475710|N|N|N|N|N| +2475801|AAAAAAAAJBHMFCAA|2066-05-30|1996|8683|666|2066|0|5|30|2|2066|666|8683|Sunday|2066Q2|N|N|N|2475772|2475891|2475436|2475711|N|N|N|N|N| +2475802|AAAAAAAAKBHMFCAA|2066-05-31|1996|8683|666|2066|1|5|31|2|2066|666|8683|Monday|2066Q2|N|N|N|2475772|2475891|2475437|2475712|N|N|N|N|N| +2475803|AAAAAAAALBHMFCAA|2066-06-01|1997|8684|667|2066|2|6|1|2|2066|667|8684|Tuesday|2066Q2|N|N|N|2475803|2475953|2475438|2475713|N|N|N|N|N| +2475804|AAAAAAAAMBHMFCAA|2066-06-02|1997|8684|667|2066|3|6|2|2|2066|667|8684|Wednesday|2066Q2|N|N|N|2475803|2475953|2475439|2475714|N|N|N|N|N| +2475805|AAAAAAAANBHMFCAA|2066-06-03|1997|8684|667|2066|4|6|3|2|2066|667|8684|Thursday|2066Q2|N|N|N|2475803|2475953|2475440|2475715|N|N|N|N|N| +2475806|AAAAAAAAOBHMFCAA|2066-06-04|1997|8684|667|2066|5|6|4|2|2066|667|8684|Friday|2066Q2|N|Y|N|2475803|2475953|2475441|2475716|N|N|N|N|N| +2475807|AAAAAAAAPBHMFCAA|2066-06-05|1997|8684|667|2066|6|6|5|2|2066|667|8684|Saturday|2066Q2|N|Y|N|2475803|2475953|2475442|2475717|N|N|N|N|N| +2475808|AAAAAAAAACHMFCAA|2066-06-06|1997|8684|667|2066|0|6|6|2|2066|667|8684|Sunday|2066Q2|N|N|N|2475803|2475953|2475443|2475718|N|N|N|N|N| +2475809|AAAAAAAABCHMFCAA|2066-06-07|1997|8684|667|2066|1|6|7|2|2066|667|8684|Monday|2066Q2|N|N|N|2475803|2475953|2475444|2475719|N|N|N|N|N| +2475810|AAAAAAAACCHMFCAA|2066-06-08|1997|8685|667|2066|2|6|8|2|2066|667|8685|Tuesday|2066Q2|N|N|N|2475803|2475953|2475445|2475720|N|N|N|N|N| +2475811|AAAAAAAADCHMFCAA|2066-06-09|1997|8685|667|2066|3|6|9|2|2066|667|8685|Wednesday|2066Q2|N|N|N|2475803|2475953|2475446|2475721|N|N|N|N|N| +2475812|AAAAAAAAECHMFCAA|2066-06-10|1997|8685|667|2066|4|6|10|2|2066|667|8685|Thursday|2066Q2|N|N|N|2475803|2475953|2475447|2475722|N|N|N|N|N| +2475813|AAAAAAAAFCHMFCAA|2066-06-11|1997|8685|667|2066|5|6|11|2|2066|667|8685|Friday|2066Q2|N|Y|N|2475803|2475953|2475448|2475723|N|N|N|N|N| +2475814|AAAAAAAAGCHMFCAA|2066-06-12|1997|8685|667|2066|6|6|12|2|2066|667|8685|Saturday|2066Q2|N|Y|N|2475803|2475953|2475449|2475724|N|N|N|N|N| +2475815|AAAAAAAAHCHMFCAA|2066-06-13|1997|8685|667|2066|0|6|13|2|2066|667|8685|Sunday|2066Q2|N|N|N|2475803|2475953|2475450|2475725|N|N|N|N|N| +2475816|AAAAAAAAICHMFCAA|2066-06-14|1997|8685|667|2066|1|6|14|2|2066|667|8685|Monday|2066Q2|N|N|N|2475803|2475953|2475451|2475726|N|N|N|N|N| +2475817|AAAAAAAAJCHMFCAA|2066-06-15|1997|8686|667|2066|2|6|15|2|2066|667|8686|Tuesday|2066Q2|N|N|N|2475803|2475953|2475452|2475727|N|N|N|N|N| +2475818|AAAAAAAAKCHMFCAA|2066-06-16|1997|8686|667|2066|3|6|16|2|2066|667|8686|Wednesday|2066Q2|N|N|N|2475803|2475953|2475453|2475728|N|N|N|N|N| +2475819|AAAAAAAALCHMFCAA|2066-06-17|1997|8686|667|2066|4|6|17|2|2066|667|8686|Thursday|2066Q2|N|N|N|2475803|2475953|2475454|2475729|N|N|N|N|N| +2475820|AAAAAAAAMCHMFCAA|2066-06-18|1997|8686|667|2066|5|6|18|2|2066|667|8686|Friday|2066Q2|N|Y|N|2475803|2475953|2475455|2475730|N|N|N|N|N| +2475821|AAAAAAAANCHMFCAA|2066-06-19|1997|8686|667|2066|6|6|19|2|2066|667|8686|Saturday|2066Q2|N|Y|N|2475803|2475953|2475456|2475731|N|N|N|N|N| +2475822|AAAAAAAAOCHMFCAA|2066-06-20|1997|8686|667|2066|0|6|20|2|2066|667|8686|Sunday|2066Q2|N|N|N|2475803|2475953|2475457|2475732|N|N|N|N|N| +2475823|AAAAAAAAPCHMFCAA|2066-06-21|1997|8686|667|2066|1|6|21|2|2066|667|8686|Monday|2066Q2|N|N|N|2475803|2475953|2475458|2475733|N|N|N|N|N| +2475824|AAAAAAAAADHMFCAA|2066-06-22|1997|8687|667|2066|2|6|22|2|2066|667|8687|Tuesday|2066Q2|N|N|N|2475803|2475953|2475459|2475734|N|N|N|N|N| +2475825|AAAAAAAABDHMFCAA|2066-06-23|1997|8687|667|2066|3|6|23|2|2066|667|8687|Wednesday|2066Q2|N|N|N|2475803|2475953|2475460|2475735|N|N|N|N|N| +2475826|AAAAAAAACDHMFCAA|2066-06-24|1997|8687|667|2066|4|6|24|2|2066|667|8687|Thursday|2066Q2|N|N|N|2475803|2475953|2475461|2475736|N|N|N|N|N| +2475827|AAAAAAAADDHMFCAA|2066-06-25|1997|8687|667|2066|5|6|25|2|2066|667|8687|Friday|2066Q2|N|Y|N|2475803|2475953|2475462|2475737|N|N|N|N|N| +2475828|AAAAAAAAEDHMFCAA|2066-06-26|1997|8687|667|2066|6|6|26|2|2066|667|8687|Saturday|2066Q2|N|Y|N|2475803|2475953|2475463|2475738|N|N|N|N|N| +2475829|AAAAAAAAFDHMFCAA|2066-06-27|1997|8687|667|2066|0|6|27|2|2066|667|8687|Sunday|2066Q2|N|N|N|2475803|2475953|2475464|2475739|N|N|N|N|N| +2475830|AAAAAAAAGDHMFCAA|2066-06-28|1997|8687|667|2066|1|6|28|2|2066|667|8687|Monday|2066Q2|N|N|N|2475803|2475953|2475465|2475740|N|N|N|N|N| +2475831|AAAAAAAAHDHMFCAA|2066-06-29|1997|8688|667|2066|2|6|29|2|2066|667|8688|Tuesday|2066Q2|N|N|N|2475803|2475953|2475466|2475741|N|N|N|N|N| +2475832|AAAAAAAAIDHMFCAA|2066-06-30|1997|8688|667|2066|3|6|30|2|2066|667|8688|Wednesday|2066Q2|N|N|N|2475803|2475953|2475467|2475742|N|N|N|N|N| +2475833|AAAAAAAAJDHMFCAA|2066-07-01|1998|8688|667|2066|4|7|1|2|2066|667|8688|Thursday|2066Q2|N|N|N|2475833|2476013|2475468|2475742|N|N|N|N|N| +2475834|AAAAAAAAKDHMFCAA|2066-07-02|1998|8688|667|2066|5|7|2|3|2066|667|8688|Friday|2066Q3|N|Y|N|2475833|2476013|2475469|2475743|N|N|N|N|N| +2475835|AAAAAAAALDHMFCAA|2066-07-03|1998|8688|667|2066|6|7|3|3|2066|667|8688|Saturday|2066Q3|N|Y|N|2475833|2476013|2475470|2475744|N|N|N|N|N| +2475836|AAAAAAAAMDHMFCAA|2066-07-04|1998|8688|667|2066|0|7|4|3|2066|667|8688|Sunday|2066Q3|N|N|N|2475833|2476013|2475471|2475745|N|N|N|N|N| +2475837|AAAAAAAANDHMFCAA|2066-07-05|1998|8688|667|2066|1|7|5|3|2066|667|8688|Monday|2066Q3|Y|N|N|2475833|2476013|2475472|2475746|N|N|N|N|N| +2475838|AAAAAAAAODHMFCAA|2066-07-06|1998|8689|667|2066|2|7|6|3|2066|667|8689|Tuesday|2066Q3|N|N|Y|2475833|2476013|2475473|2475747|N|N|N|N|N| +2475839|AAAAAAAAPDHMFCAA|2066-07-07|1998|8689|667|2066|3|7|7|3|2066|667|8689|Wednesday|2066Q3|N|N|N|2475833|2476013|2475474|2475748|N|N|N|N|N| +2475840|AAAAAAAAAEHMFCAA|2066-07-08|1998|8689|667|2066|4|7|8|3|2066|667|8689|Thursday|2066Q3|N|N|N|2475833|2476013|2475475|2475749|N|N|N|N|N| +2475841|AAAAAAAABEHMFCAA|2066-07-09|1998|8689|667|2066|5|7|9|3|2066|667|8689|Friday|2066Q3|N|Y|N|2475833|2476013|2475476|2475750|N|N|N|N|N| +2475842|AAAAAAAACEHMFCAA|2066-07-10|1998|8689|667|2066|6|7|10|3|2066|667|8689|Saturday|2066Q3|N|Y|N|2475833|2476013|2475477|2475751|N|N|N|N|N| +2475843|AAAAAAAADEHMFCAA|2066-07-11|1998|8689|667|2066|0|7|11|3|2066|667|8689|Sunday|2066Q3|N|N|N|2475833|2476013|2475478|2475752|N|N|N|N|N| +2475844|AAAAAAAAEEHMFCAA|2066-07-12|1998|8689|667|2066|1|7|12|3|2066|667|8689|Monday|2066Q3|N|N|N|2475833|2476013|2475479|2475753|N|N|N|N|N| +2475845|AAAAAAAAFEHMFCAA|2066-07-13|1998|8690|667|2066|2|7|13|3|2066|667|8690|Tuesday|2066Q3|N|N|N|2475833|2476013|2475480|2475754|N|N|N|N|N| +2475846|AAAAAAAAGEHMFCAA|2066-07-14|1998|8690|667|2066|3|7|14|3|2066|667|8690|Wednesday|2066Q3|N|N|N|2475833|2476013|2475481|2475755|N|N|N|N|N| +2475847|AAAAAAAAHEHMFCAA|2066-07-15|1998|8690|667|2066|4|7|15|3|2066|667|8690|Thursday|2066Q3|N|N|N|2475833|2476013|2475482|2475756|N|N|N|N|N| +2475848|AAAAAAAAIEHMFCAA|2066-07-16|1998|8690|667|2066|5|7|16|3|2066|667|8690|Friday|2066Q3|N|Y|N|2475833|2476013|2475483|2475757|N|N|N|N|N| +2475849|AAAAAAAAJEHMFCAA|2066-07-17|1998|8690|667|2066|6|7|17|3|2066|667|8690|Saturday|2066Q3|N|Y|N|2475833|2476013|2475484|2475758|N|N|N|N|N| +2475850|AAAAAAAAKEHMFCAA|2066-07-18|1998|8690|667|2066|0|7|18|3|2066|667|8690|Sunday|2066Q3|N|N|N|2475833|2476013|2475485|2475759|N|N|N|N|N| +2475851|AAAAAAAALEHMFCAA|2066-07-19|1998|8690|667|2066|1|7|19|3|2066|667|8690|Monday|2066Q3|N|N|N|2475833|2476013|2475486|2475760|N|N|N|N|N| +2475852|AAAAAAAAMEHMFCAA|2066-07-20|1998|8691|667|2066|2|7|20|3|2066|667|8691|Tuesday|2066Q3|N|N|N|2475833|2476013|2475487|2475761|N|N|N|N|N| +2475853|AAAAAAAANEHMFCAA|2066-07-21|1998|8691|667|2066|3|7|21|3|2066|667|8691|Wednesday|2066Q3|N|N|N|2475833|2476013|2475488|2475762|N|N|N|N|N| +2475854|AAAAAAAAOEHMFCAA|2066-07-22|1998|8691|667|2066|4|7|22|3|2066|667|8691|Thursday|2066Q3|N|N|N|2475833|2476013|2475489|2475763|N|N|N|N|N| +2475855|AAAAAAAAPEHMFCAA|2066-07-23|1998|8691|667|2066|5|7|23|3|2066|667|8691|Friday|2066Q3|N|Y|N|2475833|2476013|2475490|2475764|N|N|N|N|N| +2475856|AAAAAAAAAFHMFCAA|2066-07-24|1998|8691|667|2066|6|7|24|3|2066|667|8691|Saturday|2066Q3|N|Y|N|2475833|2476013|2475491|2475765|N|N|N|N|N| +2475857|AAAAAAAABFHMFCAA|2066-07-25|1998|8691|667|2066|0|7|25|3|2066|667|8691|Sunday|2066Q3|N|N|N|2475833|2476013|2475492|2475766|N|N|N|N|N| +2475858|AAAAAAAACFHMFCAA|2066-07-26|1998|8691|667|2066|1|7|26|3|2066|667|8691|Monday|2066Q3|N|N|N|2475833|2476013|2475493|2475767|N|N|N|N|N| +2475859|AAAAAAAADFHMFCAA|2066-07-27|1998|8692|667|2066|2|7|27|3|2066|667|8692|Tuesday|2066Q3|N|N|N|2475833|2476013|2475494|2475768|N|N|N|N|N| +2475860|AAAAAAAAEFHMFCAA|2066-07-28|1998|8692|667|2066|3|7|28|3|2066|667|8692|Wednesday|2066Q3|N|N|N|2475833|2476013|2475495|2475769|N|N|N|N|N| +2475861|AAAAAAAAFFHMFCAA|2066-07-29|1998|8692|667|2066|4|7|29|3|2066|667|8692|Thursday|2066Q3|N|N|N|2475833|2476013|2475496|2475770|N|N|N|N|N| +2475862|AAAAAAAAGFHMFCAA|2066-07-30|1998|8692|667|2066|5|7|30|3|2066|667|8692|Friday|2066Q3|N|Y|N|2475833|2476013|2475497|2475771|N|N|N|N|N| +2475863|AAAAAAAAHFHMFCAA|2066-07-31|1998|8692|667|2066|6|7|31|3|2066|667|8692|Saturday|2066Q3|N|Y|N|2475833|2476013|2475498|2475772|N|N|N|N|N| +2475864|AAAAAAAAIFHMFCAA|2066-08-01|1999|8692|667|2066|0|8|1|3|2066|667|8692|Sunday|2066Q3|N|N|N|2475864|2476075|2475499|2475773|N|N|N|N|N| +2475865|AAAAAAAAJFHMFCAA|2066-08-02|1999|8692|667|2066|1|8|2|3|2066|667|8692|Monday|2066Q3|N|N|N|2475864|2476075|2475500|2475774|N|N|N|N|N| +2475866|AAAAAAAAKFHMFCAA|2066-08-03|1999|8693|667|2066|2|8|3|3|2066|667|8693|Tuesday|2066Q3|N|N|N|2475864|2476075|2475501|2475775|N|N|N|N|N| +2475867|AAAAAAAALFHMFCAA|2066-08-04|1999|8693|667|2066|3|8|4|3|2066|667|8693|Wednesday|2066Q3|N|N|N|2475864|2476075|2475502|2475776|N|N|N|N|N| +2475868|AAAAAAAAMFHMFCAA|2066-08-05|1999|8693|667|2066|4|8|5|3|2066|667|8693|Thursday|2066Q3|N|N|N|2475864|2476075|2475503|2475777|N|N|N|N|N| +2475869|AAAAAAAANFHMFCAA|2066-08-06|1999|8693|667|2066|5|8|6|3|2066|667|8693|Friday|2066Q3|N|Y|N|2475864|2476075|2475504|2475778|N|N|N|N|N| +2475870|AAAAAAAAOFHMFCAA|2066-08-07|1999|8693|667|2066|6|8|7|3|2066|667|8693|Saturday|2066Q3|N|Y|N|2475864|2476075|2475505|2475779|N|N|N|N|N| +2475871|AAAAAAAAPFHMFCAA|2066-08-08|1999|8693|667|2066|0|8|8|3|2066|667|8693|Sunday|2066Q3|N|N|N|2475864|2476075|2475506|2475780|N|N|N|N|N| +2475872|AAAAAAAAAGHMFCAA|2066-08-09|1999|8693|667|2066|1|8|9|3|2066|667|8693|Monday|2066Q3|N|N|N|2475864|2476075|2475507|2475781|N|N|N|N|N| +2475873|AAAAAAAABGHMFCAA|2066-08-10|1999|8694|667|2066|2|8|10|3|2066|667|8694|Tuesday|2066Q3|N|N|N|2475864|2476075|2475508|2475782|N|N|N|N|N| +2475874|AAAAAAAACGHMFCAA|2066-08-11|1999|8694|667|2066|3|8|11|3|2066|667|8694|Wednesday|2066Q3|N|N|N|2475864|2476075|2475509|2475783|N|N|N|N|N| +2475875|AAAAAAAADGHMFCAA|2066-08-12|1999|8694|667|2066|4|8|12|3|2066|667|8694|Thursday|2066Q3|N|N|N|2475864|2476075|2475510|2475784|N|N|N|N|N| +2475876|AAAAAAAAEGHMFCAA|2066-08-13|1999|8694|667|2066|5|8|13|3|2066|667|8694|Friday|2066Q3|N|Y|N|2475864|2476075|2475511|2475785|N|N|N|N|N| +2475877|AAAAAAAAFGHMFCAA|2066-08-14|1999|8694|667|2066|6|8|14|3|2066|667|8694|Saturday|2066Q3|N|Y|N|2475864|2476075|2475512|2475786|N|N|N|N|N| +2475878|AAAAAAAAGGHMFCAA|2066-08-15|1999|8694|667|2066|0|8|15|3|2066|667|8694|Sunday|2066Q3|N|N|N|2475864|2476075|2475513|2475787|N|N|N|N|N| +2475879|AAAAAAAAHGHMFCAA|2066-08-16|1999|8694|667|2066|1|8|16|3|2066|667|8694|Monday|2066Q3|N|N|N|2475864|2476075|2475514|2475788|N|N|N|N|N| +2475880|AAAAAAAAIGHMFCAA|2066-08-17|1999|8695|667|2066|2|8|17|3|2066|667|8695|Tuesday|2066Q3|N|N|N|2475864|2476075|2475515|2475789|N|N|N|N|N| +2475881|AAAAAAAAJGHMFCAA|2066-08-18|1999|8695|667|2066|3|8|18|3|2066|667|8695|Wednesday|2066Q3|N|N|N|2475864|2476075|2475516|2475790|N|N|N|N|N| +2475882|AAAAAAAAKGHMFCAA|2066-08-19|1999|8695|667|2066|4|8|19|3|2066|667|8695|Thursday|2066Q3|N|N|N|2475864|2476075|2475517|2475791|N|N|N|N|N| +2475883|AAAAAAAALGHMFCAA|2066-08-20|1999|8695|667|2066|5|8|20|3|2066|667|8695|Friday|2066Q3|N|Y|N|2475864|2476075|2475518|2475792|N|N|N|N|N| +2475884|AAAAAAAAMGHMFCAA|2066-08-21|1999|8695|667|2066|6|8|21|3|2066|667|8695|Saturday|2066Q3|N|Y|N|2475864|2476075|2475519|2475793|N|N|N|N|N| +2475885|AAAAAAAANGHMFCAA|2066-08-22|1999|8695|667|2066|0|8|22|3|2066|667|8695|Sunday|2066Q3|N|N|N|2475864|2476075|2475520|2475794|N|N|N|N|N| +2475886|AAAAAAAAOGHMFCAA|2066-08-23|1999|8695|667|2066|1|8|23|3|2066|667|8695|Monday|2066Q3|N|N|N|2475864|2476075|2475521|2475795|N|N|N|N|N| +2475887|AAAAAAAAPGHMFCAA|2066-08-24|1999|8696|667|2066|2|8|24|3|2066|667|8696|Tuesday|2066Q3|N|N|N|2475864|2476075|2475522|2475796|N|N|N|N|N| +2475888|AAAAAAAAAHHMFCAA|2066-08-25|1999|8696|667|2066|3|8|25|3|2066|667|8696|Wednesday|2066Q3|N|N|N|2475864|2476075|2475523|2475797|N|N|N|N|N| +2475889|AAAAAAAABHHMFCAA|2066-08-26|1999|8696|667|2066|4|8|26|3|2066|667|8696|Thursday|2066Q3|N|N|N|2475864|2476075|2475524|2475798|N|N|N|N|N| +2475890|AAAAAAAACHHMFCAA|2066-08-27|1999|8696|667|2066|5|8|27|3|2066|667|8696|Friday|2066Q3|N|Y|N|2475864|2476075|2475525|2475799|N|N|N|N|N| +2475891|AAAAAAAADHHMFCAA|2066-08-28|1999|8696|667|2066|6|8|28|3|2066|667|8696|Saturday|2066Q3|N|Y|N|2475864|2476075|2475526|2475800|N|N|N|N|N| +2475892|AAAAAAAAEHHMFCAA|2066-08-29|1999|8696|667|2066|0|8|29|3|2066|667|8696|Sunday|2066Q3|N|N|N|2475864|2476075|2475527|2475801|N|N|N|N|N| +2475893|AAAAAAAAFHHMFCAA|2066-08-30|1999|8696|667|2066|1|8|30|3|2066|667|8696|Monday|2066Q3|N|N|N|2475864|2476075|2475528|2475802|N|N|N|N|N| +2475894|AAAAAAAAGHHMFCAA|2066-08-31|1999|8697|667|2066|2|8|31|3|2066|667|8697|Tuesday|2066Q3|N|N|N|2475864|2476075|2475529|2475803|N|N|N|N|N| +2475895|AAAAAAAAHHHMFCAA|2066-09-01|2000|8697|668|2066|3|9|1|3|2066|668|8697|Wednesday|2066Q3|N|N|N|2475895|2476137|2475530|2475804|N|N|N|N|N| +2475896|AAAAAAAAIHHMFCAA|2066-09-02|2000|8697|668|2066|4|9|2|3|2066|668|8697|Thursday|2066Q3|N|N|N|2475895|2476137|2475531|2475805|N|N|N|N|N| +2475897|AAAAAAAAJHHMFCAA|2066-09-03|2000|8697|668|2066|5|9|3|3|2066|668|8697|Friday|2066Q3|N|Y|N|2475895|2476137|2475532|2475806|N|N|N|N|N| +2475898|AAAAAAAAKHHMFCAA|2066-09-04|2000|8697|668|2066|6|9|4|3|2066|668|8697|Saturday|2066Q3|N|Y|N|2475895|2476137|2475533|2475807|N|N|N|N|N| +2475899|AAAAAAAALHHMFCAA|2066-09-05|2000|8697|668|2066|0|9|5|3|2066|668|8697|Sunday|2066Q3|N|N|N|2475895|2476137|2475534|2475808|N|N|N|N|N| +2475900|AAAAAAAAMHHMFCAA|2066-09-06|2000|8697|668|2066|1|9|6|3|2066|668|8697|Monday|2066Q3|N|N|N|2475895|2476137|2475535|2475809|N|N|N|N|N| +2475901|AAAAAAAANHHMFCAA|2066-09-07|2000|8698|668|2066|2|9|7|3|2066|668|8698|Tuesday|2066Q3|N|N|N|2475895|2476137|2475536|2475810|N|N|N|N|N| +2475902|AAAAAAAAOHHMFCAA|2066-09-08|2000|8698|668|2066|3|9|8|3|2066|668|8698|Wednesday|2066Q3|N|N|N|2475895|2476137|2475537|2475811|N|N|N|N|N| +2475903|AAAAAAAAPHHMFCAA|2066-09-09|2000|8698|668|2066|4|9|9|3|2066|668|8698|Thursday|2066Q3|N|N|N|2475895|2476137|2475538|2475812|N|N|N|N|N| +2475904|AAAAAAAAAIHMFCAA|2066-09-10|2000|8698|668|2066|5|9|10|3|2066|668|8698|Friday|2066Q3|N|Y|N|2475895|2476137|2475539|2475813|N|N|N|N|N| +2475905|AAAAAAAABIHMFCAA|2066-09-11|2000|8698|668|2066|6|9|11|3|2066|668|8698|Saturday|2066Q3|N|Y|N|2475895|2476137|2475540|2475814|N|N|N|N|N| +2475906|AAAAAAAACIHMFCAA|2066-09-12|2000|8698|668|2066|0|9|12|3|2066|668|8698|Sunday|2066Q3|N|N|N|2475895|2476137|2475541|2475815|N|N|N|N|N| +2475907|AAAAAAAADIHMFCAA|2066-09-13|2000|8698|668|2066|1|9|13|3|2066|668|8698|Monday|2066Q3|N|N|N|2475895|2476137|2475542|2475816|N|N|N|N|N| +2475908|AAAAAAAAEIHMFCAA|2066-09-14|2000|8699|668|2066|2|9|14|3|2066|668|8699|Tuesday|2066Q3|N|N|N|2475895|2476137|2475543|2475817|N|N|N|N|N| +2475909|AAAAAAAAFIHMFCAA|2066-09-15|2000|8699|668|2066|3|9|15|3|2066|668|8699|Wednesday|2066Q3|N|N|N|2475895|2476137|2475544|2475818|N|N|N|N|N| +2475910|AAAAAAAAGIHMFCAA|2066-09-16|2000|8699|668|2066|4|9|16|3|2066|668|8699|Thursday|2066Q3|N|N|N|2475895|2476137|2475545|2475819|N|N|N|N|N| +2475911|AAAAAAAAHIHMFCAA|2066-09-17|2000|8699|668|2066|5|9|17|3|2066|668|8699|Friday|2066Q3|N|Y|N|2475895|2476137|2475546|2475820|N|N|N|N|N| +2475912|AAAAAAAAIIHMFCAA|2066-09-18|2000|8699|668|2066|6|9|18|3|2066|668|8699|Saturday|2066Q3|N|Y|N|2475895|2476137|2475547|2475821|N|N|N|N|N| +2475913|AAAAAAAAJIHMFCAA|2066-09-19|2000|8699|668|2066|0|9|19|3|2066|668|8699|Sunday|2066Q3|N|N|N|2475895|2476137|2475548|2475822|N|N|N|N|N| +2475914|AAAAAAAAKIHMFCAA|2066-09-20|2000|8699|668|2066|1|9|20|3|2066|668|8699|Monday|2066Q3|N|N|N|2475895|2476137|2475549|2475823|N|N|N|N|N| +2475915|AAAAAAAALIHMFCAA|2066-09-21|2000|8700|668|2066|2|9|21|3|2066|668|8700|Tuesday|2066Q3|N|N|N|2475895|2476137|2475550|2475824|N|N|N|N|N| +2475916|AAAAAAAAMIHMFCAA|2066-09-22|2000|8700|668|2066|3|9|22|3|2066|668|8700|Wednesday|2066Q3|N|N|N|2475895|2476137|2475551|2475825|N|N|N|N|N| +2475917|AAAAAAAANIHMFCAA|2066-09-23|2000|8700|668|2066|4|9|23|3|2066|668|8700|Thursday|2066Q3|N|N|N|2475895|2476137|2475552|2475826|N|N|N|N|N| +2475918|AAAAAAAAOIHMFCAA|2066-09-24|2000|8700|668|2066|5|9|24|3|2066|668|8700|Friday|2066Q3|N|Y|N|2475895|2476137|2475553|2475827|N|N|N|N|N| +2475919|AAAAAAAAPIHMFCAA|2066-09-25|2000|8700|668|2066|6|9|25|3|2066|668|8700|Saturday|2066Q3|N|Y|N|2475895|2476137|2475554|2475828|N|N|N|N|N| +2475920|AAAAAAAAAJHMFCAA|2066-09-26|2000|8700|668|2066|0|9|26|3|2066|668|8700|Sunday|2066Q3|N|N|N|2475895|2476137|2475555|2475829|N|N|N|N|N| +2475921|AAAAAAAABJHMFCAA|2066-09-27|2000|8700|668|2066|1|9|27|3|2066|668|8700|Monday|2066Q3|N|N|N|2475895|2476137|2475556|2475830|N|N|N|N|N| +2475922|AAAAAAAACJHMFCAA|2066-09-28|2000|8701|668|2066|2|9|28|3|2066|668|8701|Tuesday|2066Q3|N|N|N|2475895|2476137|2475557|2475831|N|N|N|N|N| +2475923|AAAAAAAADJHMFCAA|2066-09-29|2000|8701|668|2066|3|9|29|3|2066|668|8701|Wednesday|2066Q3|N|N|N|2475895|2476137|2475558|2475832|N|N|N|N|N| +2475924|AAAAAAAAEJHMFCAA|2066-09-30|2000|8701|668|2066|4|9|30|3|2066|668|8701|Thursday|2066Q3|N|N|N|2475895|2476137|2475559|2475833|N|N|N|N|N| +2475925|AAAAAAAAFJHMFCAA|2066-10-01|2001|8701|668|2066|5|10|1|3|2066|668|8701|Friday|2066Q3|N|Y|N|2475925|2476197|2475560|2475833|N|N|N|N|N| +2475926|AAAAAAAAGJHMFCAA|2066-10-02|2001|8701|668|2066|6|10|2|4|2066|668|8701|Saturday|2066Q4|N|Y|N|2475925|2476197|2475561|2475834|N|N|N|N|N| +2475927|AAAAAAAAHJHMFCAA|2066-10-03|2001|8701|668|2066|0|10|3|4|2066|668|8701|Sunday|2066Q4|N|N|N|2475925|2476197|2475562|2475835|N|N|N|N|N| +2475928|AAAAAAAAIJHMFCAA|2066-10-04|2001|8701|668|2066|1|10|4|4|2066|668|8701|Monday|2066Q4|N|N|N|2475925|2476197|2475563|2475836|N|N|N|N|N| +2475929|AAAAAAAAJJHMFCAA|2066-10-05|2001|8702|668|2066|2|10|5|4|2066|668|8702|Tuesday|2066Q4|N|N|N|2475925|2476197|2475564|2475837|N|N|N|N|N| +2475930|AAAAAAAAKJHMFCAA|2066-10-06|2001|8702|668|2066|3|10|6|4|2066|668|8702|Wednesday|2066Q4|N|N|N|2475925|2476197|2475565|2475838|N|N|N|N|N| +2475931|AAAAAAAALJHMFCAA|2066-10-07|2001|8702|668|2066|4|10|7|4|2066|668|8702|Thursday|2066Q4|N|N|N|2475925|2476197|2475566|2475839|N|N|N|N|N| +2475932|AAAAAAAAMJHMFCAA|2066-10-08|2001|8702|668|2066|5|10|8|4|2066|668|8702|Friday|2066Q4|N|Y|N|2475925|2476197|2475567|2475840|N|N|N|N|N| +2475933|AAAAAAAANJHMFCAA|2066-10-09|2001|8702|668|2066|6|10|9|4|2066|668|8702|Saturday|2066Q4|N|Y|N|2475925|2476197|2475568|2475841|N|N|N|N|N| +2475934|AAAAAAAAOJHMFCAA|2066-10-10|2001|8702|668|2066|0|10|10|4|2066|668|8702|Sunday|2066Q4|N|N|N|2475925|2476197|2475569|2475842|N|N|N|N|N| +2475935|AAAAAAAAPJHMFCAA|2066-10-11|2001|8702|668|2066|1|10|11|4|2066|668|8702|Monday|2066Q4|N|N|N|2475925|2476197|2475570|2475843|N|N|N|N|N| +2475936|AAAAAAAAAKHMFCAA|2066-10-12|2001|8703|668|2066|2|10|12|4|2066|668|8703|Tuesday|2066Q4|N|N|N|2475925|2476197|2475571|2475844|N|N|N|N|N| +2475937|AAAAAAAABKHMFCAA|2066-10-13|2001|8703|668|2066|3|10|13|4|2066|668|8703|Wednesday|2066Q4|N|N|N|2475925|2476197|2475572|2475845|N|N|N|N|N| +2475938|AAAAAAAACKHMFCAA|2066-10-14|2001|8703|668|2066|4|10|14|4|2066|668|8703|Thursday|2066Q4|N|N|N|2475925|2476197|2475573|2475846|N|N|N|N|N| +2475939|AAAAAAAADKHMFCAA|2066-10-15|2001|8703|668|2066|5|10|15|4|2066|668|8703|Friday|2066Q4|N|Y|N|2475925|2476197|2475574|2475847|N|N|N|N|N| +2475940|AAAAAAAAEKHMFCAA|2066-10-16|2001|8703|668|2066|6|10|16|4|2066|668|8703|Saturday|2066Q4|N|Y|N|2475925|2476197|2475575|2475848|N|N|N|N|N| +2475941|AAAAAAAAFKHMFCAA|2066-10-17|2001|8703|668|2066|0|10|17|4|2066|668|8703|Sunday|2066Q4|N|N|N|2475925|2476197|2475576|2475849|N|N|N|N|N| +2475942|AAAAAAAAGKHMFCAA|2066-10-18|2001|8703|668|2066|1|10|18|4|2066|668|8703|Monday|2066Q4|N|N|N|2475925|2476197|2475577|2475850|N|N|N|N|N| +2475943|AAAAAAAAHKHMFCAA|2066-10-19|2001|8704|668|2066|2|10|19|4|2066|668|8704|Tuesday|2066Q4|N|N|N|2475925|2476197|2475578|2475851|N|N|N|N|N| +2475944|AAAAAAAAIKHMFCAA|2066-10-20|2001|8704|668|2066|3|10|20|4|2066|668|8704|Wednesday|2066Q4|N|N|N|2475925|2476197|2475579|2475852|N|N|N|N|N| +2475945|AAAAAAAAJKHMFCAA|2066-10-21|2001|8704|668|2066|4|10|21|4|2066|668|8704|Thursday|2066Q4|N|N|N|2475925|2476197|2475580|2475853|N|N|N|N|N| +2475946|AAAAAAAAKKHMFCAA|2066-10-22|2001|8704|668|2066|5|10|22|4|2066|668|8704|Friday|2066Q4|N|Y|N|2475925|2476197|2475581|2475854|N|N|N|N|N| +2475947|AAAAAAAALKHMFCAA|2066-10-23|2001|8704|668|2066|6|10|23|4|2066|668|8704|Saturday|2066Q4|N|Y|N|2475925|2476197|2475582|2475855|N|N|N|N|N| +2475948|AAAAAAAAMKHMFCAA|2066-10-24|2001|8704|668|2066|0|10|24|4|2066|668|8704|Sunday|2066Q4|N|N|N|2475925|2476197|2475583|2475856|N|N|N|N|N| +2475949|AAAAAAAANKHMFCAA|2066-10-25|2001|8704|668|2066|1|10|25|4|2066|668|8704|Monday|2066Q4|N|N|N|2475925|2476197|2475584|2475857|N|N|N|N|N| +2475950|AAAAAAAAOKHMFCAA|2066-10-26|2001|8705|668|2066|2|10|26|4|2066|668|8705|Tuesday|2066Q4|N|N|N|2475925|2476197|2475585|2475858|N|N|N|N|N| +2475951|AAAAAAAAPKHMFCAA|2066-10-27|2001|8705|668|2066|3|10|27|4|2066|668|8705|Wednesday|2066Q4|N|N|N|2475925|2476197|2475586|2475859|N|N|N|N|N| +2475952|AAAAAAAAALHMFCAA|2066-10-28|2001|8705|668|2066|4|10|28|4|2066|668|8705|Thursday|2066Q4|N|N|N|2475925|2476197|2475587|2475860|N|N|N|N|N| +2475953|AAAAAAAABLHMFCAA|2066-10-29|2001|8705|668|2066|5|10|29|4|2066|668|8705|Friday|2066Q4|N|Y|N|2475925|2476197|2475588|2475861|N|N|N|N|N| +2475954|AAAAAAAACLHMFCAA|2066-10-30|2001|8705|668|2066|6|10|30|4|2066|668|8705|Saturday|2066Q4|N|Y|N|2475925|2476197|2475589|2475862|N|N|N|N|N| +2475955|AAAAAAAADLHMFCAA|2066-10-31|2001|8705|668|2066|0|10|31|4|2066|668|8705|Sunday|2066Q4|N|N|N|2475925|2476197|2475590|2475863|N|N|N|N|N| +2475956|AAAAAAAAELHMFCAA|2066-11-01|2002|8705|668|2066|1|11|1|4|2066|668|8705|Monday|2066Q4|N|N|N|2475956|2476259|2475591|2475864|N|N|N|N|N| +2475957|AAAAAAAAFLHMFCAA|2066-11-02|2002|8706|668|2066|2|11|2|4|2066|668|8706|Tuesday|2066Q4|N|N|N|2475956|2476259|2475592|2475865|N|N|N|N|N| +2475958|AAAAAAAAGLHMFCAA|2066-11-03|2002|8706|668|2066|3|11|3|4|2066|668|8706|Wednesday|2066Q4|N|N|N|2475956|2476259|2475593|2475866|N|N|N|N|N| +2475959|AAAAAAAAHLHMFCAA|2066-11-04|2002|8706|668|2066|4|11|4|4|2066|668|8706|Thursday|2066Q4|N|N|N|2475956|2476259|2475594|2475867|N|N|N|N|N| +2475960|AAAAAAAAILHMFCAA|2066-11-05|2002|8706|668|2066|5|11|5|4|2066|668|8706|Friday|2066Q4|N|Y|N|2475956|2476259|2475595|2475868|N|N|N|N|N| +2475961|AAAAAAAAJLHMFCAA|2066-11-06|2002|8706|668|2066|6|11|6|4|2066|668|8706|Saturday|2066Q4|N|Y|N|2475956|2476259|2475596|2475869|N|N|N|N|N| +2475962|AAAAAAAAKLHMFCAA|2066-11-07|2002|8706|668|2066|0|11|7|4|2066|668|8706|Sunday|2066Q4|N|N|N|2475956|2476259|2475597|2475870|N|N|N|N|N| +2475963|AAAAAAAALLHMFCAA|2066-11-08|2002|8706|668|2066|1|11|8|4|2066|668|8706|Monday|2066Q4|N|N|N|2475956|2476259|2475598|2475871|N|N|N|N|N| +2475964|AAAAAAAAMLHMFCAA|2066-11-09|2002|8707|668|2066|2|11|9|4|2066|668|8707|Tuesday|2066Q4|N|N|N|2475956|2476259|2475599|2475872|N|N|N|N|N| +2475965|AAAAAAAANLHMFCAA|2066-11-10|2002|8707|668|2066|3|11|10|4|2066|668|8707|Wednesday|2066Q4|N|N|N|2475956|2476259|2475600|2475873|N|N|N|N|N| +2475966|AAAAAAAAOLHMFCAA|2066-11-11|2002|8707|668|2066|4|11|11|4|2066|668|8707|Thursday|2066Q4|N|N|N|2475956|2476259|2475601|2475874|N|N|N|N|N| +2475967|AAAAAAAAPLHMFCAA|2066-11-12|2002|8707|668|2066|5|11|12|4|2066|668|8707|Friday|2066Q4|N|Y|N|2475956|2476259|2475602|2475875|N|N|N|N|N| +2475968|AAAAAAAAAMHMFCAA|2066-11-13|2002|8707|668|2066|6|11|13|4|2066|668|8707|Saturday|2066Q4|N|Y|N|2475956|2476259|2475603|2475876|N|N|N|N|N| +2475969|AAAAAAAABMHMFCAA|2066-11-14|2002|8707|668|2066|0|11|14|4|2066|668|8707|Sunday|2066Q4|N|N|N|2475956|2476259|2475604|2475877|N|N|N|N|N| +2475970|AAAAAAAACMHMFCAA|2066-11-15|2002|8707|668|2066|1|11|15|4|2066|668|8707|Monday|2066Q4|N|N|N|2475956|2476259|2475605|2475878|N|N|N|N|N| +2475971|AAAAAAAADMHMFCAA|2066-11-16|2002|8708|668|2066|2|11|16|4|2066|668|8708|Tuesday|2066Q4|N|N|N|2475956|2476259|2475606|2475879|N|N|N|N|N| +2475972|AAAAAAAAEMHMFCAA|2066-11-17|2002|8708|668|2066|3|11|17|4|2066|668|8708|Wednesday|2066Q4|N|N|N|2475956|2476259|2475607|2475880|N|N|N|N|N| +2475973|AAAAAAAAFMHMFCAA|2066-11-18|2002|8708|668|2066|4|11|18|4|2066|668|8708|Thursday|2066Q4|N|N|N|2475956|2476259|2475608|2475881|N|N|N|N|N| +2475974|AAAAAAAAGMHMFCAA|2066-11-19|2002|8708|668|2066|5|11|19|4|2066|668|8708|Friday|2066Q4|N|Y|N|2475956|2476259|2475609|2475882|N|N|N|N|N| +2475975|AAAAAAAAHMHMFCAA|2066-11-20|2002|8708|668|2066|6|11|20|4|2066|668|8708|Saturday|2066Q4|N|Y|N|2475956|2476259|2475610|2475883|N|N|N|N|N| +2475976|AAAAAAAAIMHMFCAA|2066-11-21|2002|8708|668|2066|0|11|21|4|2066|668|8708|Sunday|2066Q4|N|N|N|2475956|2476259|2475611|2475884|N|N|N|N|N| +2475977|AAAAAAAAJMHMFCAA|2066-11-22|2002|8708|668|2066|1|11|22|4|2066|668|8708|Monday|2066Q4|N|N|N|2475956|2476259|2475612|2475885|N|N|N|N|N| +2475978|AAAAAAAAKMHMFCAA|2066-11-23|2002|8709|668|2066|2|11|23|4|2066|668|8709|Tuesday|2066Q4|N|N|N|2475956|2476259|2475613|2475886|N|N|N|N|N| +2475979|AAAAAAAALMHMFCAA|2066-11-24|2002|8709|668|2066|3|11|24|4|2066|668|8709|Wednesday|2066Q4|N|N|N|2475956|2476259|2475614|2475887|N|N|N|N|N| +2475980|AAAAAAAAMMHMFCAA|2066-11-25|2002|8709|668|2066|4|11|25|4|2066|668|8709|Thursday|2066Q4|N|N|N|2475956|2476259|2475615|2475888|N|N|N|N|N| +2475981|AAAAAAAANMHMFCAA|2066-11-26|2002|8709|668|2066|5|11|26|4|2066|668|8709|Friday|2066Q4|N|Y|N|2475956|2476259|2475616|2475889|N|N|N|N|N| +2475982|AAAAAAAAOMHMFCAA|2066-11-27|2002|8709|668|2066|6|11|27|4|2066|668|8709|Saturday|2066Q4|N|Y|N|2475956|2476259|2475617|2475890|N|N|N|N|N| +2475983|AAAAAAAAPMHMFCAA|2066-11-28|2002|8709|668|2066|0|11|28|4|2066|668|8709|Sunday|2066Q4|N|N|N|2475956|2476259|2475618|2475891|N|N|N|N|N| +2475984|AAAAAAAAANHMFCAA|2066-11-29|2002|8709|668|2066|1|11|29|4|2066|668|8709|Monday|2066Q4|N|N|N|2475956|2476259|2475619|2475892|N|N|N|N|N| +2475985|AAAAAAAABNHMFCAA|2066-11-30|2002|8710|668|2066|2|11|30|4|2066|668|8710|Tuesday|2066Q4|N|N|N|2475956|2476259|2475620|2475893|N|N|N|N|N| +2475986|AAAAAAAACNHMFCAA|2066-12-01|2003|8710|669|2066|3|12|1|4|2066|669|8710|Wednesday|2066Q4|N|N|N|2475986|2476319|2475621|2475894|N|N|N|N|N| +2475987|AAAAAAAADNHMFCAA|2066-12-02|2003|8710|669|2066|4|12|2|4|2066|669|8710|Thursday|2066Q4|N|N|N|2475986|2476319|2475622|2475895|N|N|N|N|N| +2475988|AAAAAAAAENHMFCAA|2066-12-03|2003|8710|669|2066|5|12|3|4|2066|669|8710|Friday|2066Q4|N|Y|N|2475986|2476319|2475623|2475896|N|N|N|N|N| +2475989|AAAAAAAAFNHMFCAA|2066-12-04|2003|8710|669|2066|6|12|4|4|2066|669|8710|Saturday|2066Q4|N|Y|N|2475986|2476319|2475624|2475897|N|N|N|N|N| +2475990|AAAAAAAAGNHMFCAA|2066-12-05|2003|8710|669|2066|0|12|5|4|2066|669|8710|Sunday|2066Q4|N|N|N|2475986|2476319|2475625|2475898|N|N|N|N|N| +2475991|AAAAAAAAHNHMFCAA|2066-12-06|2003|8710|669|2066|1|12|6|4|2066|669|8710|Monday|2066Q4|N|N|N|2475986|2476319|2475626|2475899|N|N|N|N|N| +2475992|AAAAAAAAINHMFCAA|2066-12-07|2003|8711|669|2066|2|12|7|4|2066|669|8711|Tuesday|2066Q4|N|N|N|2475986|2476319|2475627|2475900|N|N|N|N|N| +2475993|AAAAAAAAJNHMFCAA|2066-12-08|2003|8711|669|2066|3|12|8|4|2066|669|8711|Wednesday|2066Q4|N|N|N|2475986|2476319|2475628|2475901|N|N|N|N|N| +2475994|AAAAAAAAKNHMFCAA|2066-12-09|2003|8711|669|2066|4|12|9|4|2066|669|8711|Thursday|2066Q4|N|N|N|2475986|2476319|2475629|2475902|N|N|N|N|N| +2475995|AAAAAAAALNHMFCAA|2066-12-10|2003|8711|669|2066|5|12|10|4|2066|669|8711|Friday|2066Q4|N|Y|N|2475986|2476319|2475630|2475903|N|N|N|N|N| +2475996|AAAAAAAAMNHMFCAA|2066-12-11|2003|8711|669|2066|6|12|11|4|2066|669|8711|Saturday|2066Q4|N|Y|N|2475986|2476319|2475631|2475904|N|N|N|N|N| +2475997|AAAAAAAANNHMFCAA|2066-12-12|2003|8711|669|2066|0|12|12|4|2066|669|8711|Sunday|2066Q4|N|N|N|2475986|2476319|2475632|2475905|N|N|N|N|N| +2475998|AAAAAAAAONHMFCAA|2066-12-13|2003|8711|669|2066|1|12|13|4|2066|669|8711|Monday|2066Q4|N|N|N|2475986|2476319|2475633|2475906|N|N|N|N|N| +2475999|AAAAAAAAPNHMFCAA|2066-12-14|2003|8712|669|2066|2|12|14|4|2066|669|8712|Tuesday|2066Q4|N|N|N|2475986|2476319|2475634|2475907|N|N|N|N|N| +2476000|AAAAAAAAAOHMFCAA|2066-12-15|2003|8712|669|2066|3|12|15|4|2066|669|8712|Wednesday|2066Q4|N|N|N|2475986|2476319|2475635|2475908|N|N|N|N|N| +2476001|AAAAAAAABOHMFCAA|2066-12-16|2003|8712|669|2066|4|12|16|4|2066|669|8712|Thursday|2066Q4|N|N|N|2475986|2476319|2475636|2475909|N|N|N|N|N| +2476002|AAAAAAAACOHMFCAA|2066-12-17|2003|8712|669|2066|5|12|17|4|2066|669|8712|Friday|2066Q4|N|Y|N|2475986|2476319|2475637|2475910|N|N|N|N|N| +2476003|AAAAAAAADOHMFCAA|2066-12-18|2003|8712|669|2066|6|12|18|4|2066|669|8712|Saturday|2066Q4|N|Y|N|2475986|2476319|2475638|2475911|N|N|N|N|N| +2476004|AAAAAAAAEOHMFCAA|2066-12-19|2003|8712|669|2066|0|12|19|4|2066|669|8712|Sunday|2066Q4|N|N|N|2475986|2476319|2475639|2475912|N|N|N|N|N| +2476005|AAAAAAAAFOHMFCAA|2066-12-20|2003|8712|669|2066|1|12|20|4|2066|669|8712|Monday|2066Q4|N|N|N|2475986|2476319|2475640|2475913|N|N|N|N|N| +2476006|AAAAAAAAGOHMFCAA|2066-12-21|2003|8713|669|2066|2|12|21|4|2066|669|8713|Tuesday|2066Q4|N|N|N|2475986|2476319|2475641|2475914|N|N|N|N|N| +2476007|AAAAAAAAHOHMFCAA|2066-12-22|2003|8713|669|2066|3|12|22|4|2066|669|8713|Wednesday|2066Q4|N|N|N|2475986|2476319|2475642|2475915|N|N|N|N|N| +2476008|AAAAAAAAIOHMFCAA|2066-12-23|2003|8713|669|2066|4|12|23|4|2066|669|8713|Thursday|2066Q4|N|N|N|2475986|2476319|2475643|2475916|N|N|N|N|N| +2476009|AAAAAAAAJOHMFCAA|2066-12-24|2003|8713|669|2066|5|12|24|4|2066|669|8713|Friday|2066Q4|N|Y|N|2475986|2476319|2475644|2475917|N|N|N|N|N| +2476010|AAAAAAAAKOHMFCAA|2066-12-25|2003|8713|669|2066|6|12|25|4|2066|669|8713|Saturday|2066Q4|N|Y|N|2475986|2476319|2475645|2475918|N|N|N|N|N| +2476011|AAAAAAAALOHMFCAA|2066-12-26|2003|8713|669|2066|0|12|26|4|2066|669|8713|Sunday|2066Q4|Y|N|N|2475986|2476319|2475646|2475919|N|N|N|N|N| +2476012|AAAAAAAAMOHMFCAA|2066-12-27|2003|8713|669|2066|1|12|27|4|2066|669|8713|Monday|2066Q4|N|N|Y|2475986|2476319|2475647|2475920|N|N|N|N|N| +2476013|AAAAAAAANOHMFCAA|2066-12-28|2003|8714|669|2066|2|12|28|4|2066|669|8714|Tuesday|2066Q4|N|N|N|2475986|2476319|2475648|2475921|N|N|N|N|N| +2476014|AAAAAAAAOOHMFCAA|2066-12-29|2003|8714|669|2066|3|12|29|4|2066|669|8714|Wednesday|2066Q4|N|N|N|2475986|2476319|2475649|2475922|N|N|N|N|N| +2476015|AAAAAAAAPOHMFCAA|2066-12-30|2003|8714|669|2066|4|12|30|4|2066|669|8714|Thursday|2066Q4|N|N|N|2475986|2476319|2475650|2475923|N|N|N|N|N| +2476016|AAAAAAAAAPHMFCAA|2066-12-31|2003|8714|669|2066|5|12|31|4|2066|669|8714|Friday|2066Q4|N|Y|N|2475986|2476319|2475651|2475924|N|N|N|N|N| +2476017|AAAAAAAABPHMFCAA|2067-01-01|2004|8714|669|2067|6|1|1|1|2067|669|8714|Saturday|2067Q1|Y|Y|N|2476017|2476016|2475652|2475925|N|N|N|N|N| +2476018|AAAAAAAACPHMFCAA|2067-01-02|2004|8714|669|2067|0|1|2|1|2067|669|8714|Sunday|2067Q1|N|N|Y|2476017|2476016|2475653|2475926|N|N|N|N|N| +2476019|AAAAAAAADPHMFCAA|2067-01-03|2004|8714|669|2067|1|1|3|1|2067|669|8714|Monday|2067Q1|N|N|N|2476017|2476016|2475654|2475927|N|N|N|N|N| +2476020|AAAAAAAAEPHMFCAA|2067-01-04|2004|8715|669|2067|2|1|4|1|2067|669|8715|Tuesday|2067Q1|N|N|N|2476017|2476016|2475655|2475928|N|N|N|N|N| +2476021|AAAAAAAAFPHMFCAA|2067-01-05|2004|8715|669|2067|3|1|5|1|2067|669|8715|Wednesday|2067Q1|N|N|N|2476017|2476016|2475656|2475929|N|N|N|N|N| +2476022|AAAAAAAAGPHMFCAA|2067-01-06|2004|8715|669|2067|4|1|6|1|2067|669|8715|Thursday|2067Q1|N|N|N|2476017|2476016|2475657|2475930|N|N|N|N|N| +2476023|AAAAAAAAHPHMFCAA|2067-01-07|2004|8715|669|2067|5|1|7|1|2067|669|8715|Friday|2067Q1|N|Y|N|2476017|2476016|2475658|2475931|N|N|N|N|N| +2476024|AAAAAAAAIPHMFCAA|2067-01-08|2004|8715|669|2067|6|1|8|1|2067|669|8715|Saturday|2067Q1|N|Y|N|2476017|2476016|2475659|2475932|N|N|N|N|N| +2476025|AAAAAAAAJPHMFCAA|2067-01-09|2004|8715|669|2067|0|1|9|1|2067|669|8715|Sunday|2067Q1|N|N|N|2476017|2476016|2475660|2475933|N|N|N|N|N| +2476026|AAAAAAAAKPHMFCAA|2067-01-10|2004|8715|669|2067|1|1|10|1|2067|669|8715|Monday|2067Q1|N|N|N|2476017|2476016|2475661|2475934|N|N|N|N|N| +2476027|AAAAAAAALPHMFCAA|2067-01-11|2004|8716|669|2067|2|1|11|1|2067|669|8716|Tuesday|2067Q1|N|N|N|2476017|2476016|2475662|2475935|N|N|N|N|N| +2476028|AAAAAAAAMPHMFCAA|2067-01-12|2004|8716|669|2067|3|1|12|1|2067|669|8716|Wednesday|2067Q1|N|N|N|2476017|2476016|2475663|2475936|N|N|N|N|N| +2476029|AAAAAAAANPHMFCAA|2067-01-13|2004|8716|669|2067|4|1|13|1|2067|669|8716|Thursday|2067Q1|N|N|N|2476017|2476016|2475664|2475937|N|N|N|N|N| +2476030|AAAAAAAAOPHMFCAA|2067-01-14|2004|8716|669|2067|5|1|14|1|2067|669|8716|Friday|2067Q1|N|Y|N|2476017|2476016|2475665|2475938|N|N|N|N|N| +2476031|AAAAAAAAPPHMFCAA|2067-01-15|2004|8716|669|2067|6|1|15|1|2067|669|8716|Saturday|2067Q1|N|Y|N|2476017|2476016|2475666|2475939|N|N|N|N|N| +2476032|AAAAAAAAAAIMFCAA|2067-01-16|2004|8716|669|2067|0|1|16|1|2067|669|8716|Sunday|2067Q1|N|N|N|2476017|2476016|2475667|2475940|N|N|N|N|N| +2476033|AAAAAAAABAIMFCAA|2067-01-17|2004|8716|669|2067|1|1|17|1|2067|669|8716|Monday|2067Q1|N|N|N|2476017|2476016|2475668|2475941|N|N|N|N|N| +2476034|AAAAAAAACAIMFCAA|2067-01-18|2004|8717|669|2067|2|1|18|1|2067|669|8717|Tuesday|2067Q1|N|N|N|2476017|2476016|2475669|2475942|N|N|N|N|N| +2476035|AAAAAAAADAIMFCAA|2067-01-19|2004|8717|669|2067|3|1|19|1|2067|669|8717|Wednesday|2067Q1|N|N|N|2476017|2476016|2475670|2475943|N|N|N|N|N| +2476036|AAAAAAAAEAIMFCAA|2067-01-20|2004|8717|669|2067|4|1|20|1|2067|669|8717|Thursday|2067Q1|N|N|N|2476017|2476016|2475671|2475944|N|N|N|N|N| +2476037|AAAAAAAAFAIMFCAA|2067-01-21|2004|8717|669|2067|5|1|21|1|2067|669|8717|Friday|2067Q1|N|Y|N|2476017|2476016|2475672|2475945|N|N|N|N|N| +2476038|AAAAAAAAGAIMFCAA|2067-01-22|2004|8717|669|2067|6|1|22|1|2067|669|8717|Saturday|2067Q1|N|Y|N|2476017|2476016|2475673|2475946|N|N|N|N|N| +2476039|AAAAAAAAHAIMFCAA|2067-01-23|2004|8717|669|2067|0|1|23|1|2067|669|8717|Sunday|2067Q1|N|N|N|2476017|2476016|2475674|2475947|N|N|N|N|N| +2476040|AAAAAAAAIAIMFCAA|2067-01-24|2004|8717|669|2067|1|1|24|1|2067|669|8717|Monday|2067Q1|N|N|N|2476017|2476016|2475675|2475948|N|N|N|N|N| +2476041|AAAAAAAAJAIMFCAA|2067-01-25|2004|8718|669|2067|2|1|25|1|2067|669|8718|Tuesday|2067Q1|N|N|N|2476017|2476016|2475676|2475949|N|N|N|N|N| +2476042|AAAAAAAAKAIMFCAA|2067-01-26|2004|8718|669|2067|3|1|26|1|2067|669|8718|Wednesday|2067Q1|N|N|N|2476017|2476016|2475677|2475950|N|N|N|N|N| +2476043|AAAAAAAALAIMFCAA|2067-01-27|2004|8718|669|2067|4|1|27|1|2067|669|8718|Thursday|2067Q1|N|N|N|2476017|2476016|2475678|2475951|N|N|N|N|N| +2476044|AAAAAAAAMAIMFCAA|2067-01-28|2004|8718|669|2067|5|1|28|1|2067|669|8718|Friday|2067Q1|N|Y|N|2476017|2476016|2475679|2475952|N|N|N|N|N| +2476045|AAAAAAAANAIMFCAA|2067-01-29|2004|8718|669|2067|6|1|29|1|2067|669|8718|Saturday|2067Q1|N|Y|N|2476017|2476016|2475680|2475953|N|N|N|N|N| +2476046|AAAAAAAAOAIMFCAA|2067-01-30|2004|8718|669|2067|0|1|30|1|2067|669|8718|Sunday|2067Q1|N|N|N|2476017|2476016|2475681|2475954|N|N|N|N|N| +2476047|AAAAAAAAPAIMFCAA|2067-01-31|2004|8718|669|2067|1|1|31|1|2067|669|8718|Monday|2067Q1|N|N|N|2476017|2476016|2475682|2475955|N|N|N|N|N| +2476048|AAAAAAAAABIMFCAA|2067-02-01|2005|8719|669|2067|2|2|1|1|2067|669|8719|Tuesday|2067Q1|N|N|N|2476048|2476078|2475683|2475956|N|N|N|N|N| +2476049|AAAAAAAABBIMFCAA|2067-02-02|2005|8719|669|2067|3|2|2|1|2067|669|8719|Wednesday|2067Q1|N|N|N|2476048|2476078|2475684|2475957|N|N|N|N|N| +2476050|AAAAAAAACBIMFCAA|2067-02-03|2005|8719|669|2067|4|2|3|1|2067|669|8719|Thursday|2067Q1|N|N|N|2476048|2476078|2475685|2475958|N|N|N|N|N| +2476051|AAAAAAAADBIMFCAA|2067-02-04|2005|8719|669|2067|5|2|4|1|2067|669|8719|Friday|2067Q1|N|Y|N|2476048|2476078|2475686|2475959|N|N|N|N|N| +2476052|AAAAAAAAEBIMFCAA|2067-02-05|2005|8719|669|2067|6|2|5|1|2067|669|8719|Saturday|2067Q1|N|Y|N|2476048|2476078|2475687|2475960|N|N|N|N|N| +2476053|AAAAAAAAFBIMFCAA|2067-02-06|2005|8719|669|2067|0|2|6|1|2067|669|8719|Sunday|2067Q1|N|N|N|2476048|2476078|2475688|2475961|N|N|N|N|N| +2476054|AAAAAAAAGBIMFCAA|2067-02-07|2005|8719|669|2067|1|2|7|1|2067|669|8719|Monday|2067Q1|N|N|N|2476048|2476078|2475689|2475962|N|N|N|N|N| +2476055|AAAAAAAAHBIMFCAA|2067-02-08|2005|8720|669|2067|2|2|8|1|2067|669|8720|Tuesday|2067Q1|N|N|N|2476048|2476078|2475690|2475963|N|N|N|N|N| +2476056|AAAAAAAAIBIMFCAA|2067-02-09|2005|8720|669|2067|3|2|9|1|2067|669|8720|Wednesday|2067Q1|N|N|N|2476048|2476078|2475691|2475964|N|N|N|N|N| +2476057|AAAAAAAAJBIMFCAA|2067-02-10|2005|8720|669|2067|4|2|10|1|2067|669|8720|Thursday|2067Q1|N|N|N|2476048|2476078|2475692|2475965|N|N|N|N|N| +2476058|AAAAAAAAKBIMFCAA|2067-02-11|2005|8720|669|2067|5|2|11|1|2067|669|8720|Friday|2067Q1|N|Y|N|2476048|2476078|2475693|2475966|N|N|N|N|N| +2476059|AAAAAAAALBIMFCAA|2067-02-12|2005|8720|669|2067|6|2|12|1|2067|669|8720|Saturday|2067Q1|N|Y|N|2476048|2476078|2475694|2475967|N|N|N|N|N| +2476060|AAAAAAAAMBIMFCAA|2067-02-13|2005|8720|669|2067|0|2|13|1|2067|669|8720|Sunday|2067Q1|N|N|N|2476048|2476078|2475695|2475968|N|N|N|N|N| +2476061|AAAAAAAANBIMFCAA|2067-02-14|2005|8720|669|2067|1|2|14|1|2067|669|8720|Monday|2067Q1|N|N|N|2476048|2476078|2475696|2475969|N|N|N|N|N| +2476062|AAAAAAAAOBIMFCAA|2067-02-15|2005|8721|669|2067|2|2|15|1|2067|669|8721|Tuesday|2067Q1|N|N|N|2476048|2476078|2475697|2475970|N|N|N|N|N| +2476063|AAAAAAAAPBIMFCAA|2067-02-16|2005|8721|669|2067|3|2|16|1|2067|669|8721|Wednesday|2067Q1|N|N|N|2476048|2476078|2475698|2475971|N|N|N|N|N| +2476064|AAAAAAAAACIMFCAA|2067-02-17|2005|8721|669|2067|4|2|17|1|2067|669|8721|Thursday|2067Q1|N|N|N|2476048|2476078|2475699|2475972|N|N|N|N|N| +2476065|AAAAAAAABCIMFCAA|2067-02-18|2005|8721|669|2067|5|2|18|1|2067|669|8721|Friday|2067Q1|N|Y|N|2476048|2476078|2475700|2475973|N|N|N|N|N| +2476066|AAAAAAAACCIMFCAA|2067-02-19|2005|8721|669|2067|6|2|19|1|2067|669|8721|Saturday|2067Q1|N|Y|N|2476048|2476078|2475701|2475974|N|N|N|N|N| +2476067|AAAAAAAADCIMFCAA|2067-02-20|2005|8721|669|2067|0|2|20|1|2067|669|8721|Sunday|2067Q1|N|N|N|2476048|2476078|2475702|2475975|N|N|N|N|N| +2476068|AAAAAAAAECIMFCAA|2067-02-21|2005|8721|669|2067|1|2|21|1|2067|669|8721|Monday|2067Q1|N|N|N|2476048|2476078|2475703|2475976|N|N|N|N|N| +2476069|AAAAAAAAFCIMFCAA|2067-02-22|2005|8722|669|2067|2|2|22|1|2067|669|8722|Tuesday|2067Q1|N|N|N|2476048|2476078|2475704|2475977|N|N|N|N|N| +2476070|AAAAAAAAGCIMFCAA|2067-02-23|2005|8722|669|2067|3|2|23|1|2067|669|8722|Wednesday|2067Q1|N|N|N|2476048|2476078|2475705|2475978|N|N|N|N|N| +2476071|AAAAAAAAHCIMFCAA|2067-02-24|2005|8722|669|2067|4|2|24|1|2067|669|8722|Thursday|2067Q1|N|N|N|2476048|2476078|2475706|2475979|N|N|N|N|N| +2476072|AAAAAAAAICIMFCAA|2067-02-25|2005|8722|669|2067|5|2|25|1|2067|669|8722|Friday|2067Q1|N|Y|N|2476048|2476078|2475707|2475980|N|N|N|N|N| +2476073|AAAAAAAAJCIMFCAA|2067-02-26|2005|8722|669|2067|6|2|26|1|2067|669|8722|Saturday|2067Q1|N|Y|N|2476048|2476078|2475708|2475981|N|N|N|N|N| +2476074|AAAAAAAAKCIMFCAA|2067-02-27|2005|8722|669|2067|0|2|27|1|2067|669|8722|Sunday|2067Q1|N|N|N|2476048|2476078|2475709|2475982|N|N|N|N|N| +2476075|AAAAAAAALCIMFCAA|2067-02-28|2005|8722|669|2067|1|2|28|1|2067|669|8722|Monday|2067Q1|N|N|N|2476048|2476078|2475710|2475983|N|N|N|N|N| +2476076|AAAAAAAAMCIMFCAA|2067-03-01|2006|8723|670|2067|2|3|1|1|2067|670|8723|Tuesday|2067Q1|N|N|N|2476076|2476134|2475711|2475984|N|N|N|N|N| +2476077|AAAAAAAANCIMFCAA|2067-03-02|2006|8723|670|2067|3|3|2|1|2067|670|8723|Wednesday|2067Q1|N|N|N|2476076|2476134|2475712|2475985|N|N|N|N|N| +2476078|AAAAAAAAOCIMFCAA|2067-03-03|2006|8723|670|2067|4|3|3|1|2067|670|8723|Thursday|2067Q1|N|N|N|2476076|2476134|2475713|2475986|N|N|N|N|N| +2476079|AAAAAAAAPCIMFCAA|2067-03-04|2006|8723|670|2067|5|3|4|1|2067|670|8723|Friday|2067Q1|N|Y|N|2476076|2476134|2475714|2475987|N|N|N|N|N| +2476080|AAAAAAAAADIMFCAA|2067-03-05|2006|8723|670|2067|6|3|5|1|2067|670|8723|Saturday|2067Q1|N|Y|N|2476076|2476134|2475715|2475988|N|N|N|N|N| +2476081|AAAAAAAABDIMFCAA|2067-03-06|2006|8723|670|2067|0|3|6|1|2067|670|8723|Sunday|2067Q1|N|N|N|2476076|2476134|2475716|2475989|N|N|N|N|N| +2476082|AAAAAAAACDIMFCAA|2067-03-07|2006|8723|670|2067|1|3|7|1|2067|670|8723|Monday|2067Q1|N|N|N|2476076|2476134|2475717|2475990|N|N|N|N|N| +2476083|AAAAAAAADDIMFCAA|2067-03-08|2006|8724|670|2067|2|3|8|1|2067|670|8724|Tuesday|2067Q1|N|N|N|2476076|2476134|2475718|2475991|N|N|N|N|N| +2476084|AAAAAAAAEDIMFCAA|2067-03-09|2006|8724|670|2067|3|3|9|1|2067|670|8724|Wednesday|2067Q1|N|N|N|2476076|2476134|2475719|2475992|N|N|N|N|N| +2476085|AAAAAAAAFDIMFCAA|2067-03-10|2006|8724|670|2067|4|3|10|1|2067|670|8724|Thursday|2067Q1|N|N|N|2476076|2476134|2475720|2475993|N|N|N|N|N| +2476086|AAAAAAAAGDIMFCAA|2067-03-11|2006|8724|670|2067|5|3|11|1|2067|670|8724|Friday|2067Q1|N|Y|N|2476076|2476134|2475721|2475994|N|N|N|N|N| +2476087|AAAAAAAAHDIMFCAA|2067-03-12|2006|8724|670|2067|6|3|12|1|2067|670|8724|Saturday|2067Q1|N|Y|N|2476076|2476134|2475722|2475995|N|N|N|N|N| +2476088|AAAAAAAAIDIMFCAA|2067-03-13|2006|8724|670|2067|0|3|13|1|2067|670|8724|Sunday|2067Q1|N|N|N|2476076|2476134|2475723|2475996|N|N|N|N|N| +2476089|AAAAAAAAJDIMFCAA|2067-03-14|2006|8724|670|2067|1|3|14|1|2067|670|8724|Monday|2067Q1|N|N|N|2476076|2476134|2475724|2475997|N|N|N|N|N| +2476090|AAAAAAAAKDIMFCAA|2067-03-15|2006|8725|670|2067|2|3|15|1|2067|670|8725|Tuesday|2067Q1|N|N|N|2476076|2476134|2475725|2475998|N|N|N|N|N| +2476091|AAAAAAAALDIMFCAA|2067-03-16|2006|8725|670|2067|3|3|16|1|2067|670|8725|Wednesday|2067Q1|N|N|N|2476076|2476134|2475726|2475999|N|N|N|N|N| +2476092|AAAAAAAAMDIMFCAA|2067-03-17|2006|8725|670|2067|4|3|17|1|2067|670|8725|Thursday|2067Q1|N|N|N|2476076|2476134|2475727|2476000|N|N|N|N|N| +2476093|AAAAAAAANDIMFCAA|2067-03-18|2006|8725|670|2067|5|3|18|1|2067|670|8725|Friday|2067Q1|N|Y|N|2476076|2476134|2475728|2476001|N|N|N|N|N| +2476094|AAAAAAAAODIMFCAA|2067-03-19|2006|8725|670|2067|6|3|19|1|2067|670|8725|Saturday|2067Q1|N|Y|N|2476076|2476134|2475729|2476002|N|N|N|N|N| +2476095|AAAAAAAAPDIMFCAA|2067-03-20|2006|8725|670|2067|0|3|20|1|2067|670|8725|Sunday|2067Q1|N|N|N|2476076|2476134|2475730|2476003|N|N|N|N|N| +2476096|AAAAAAAAAEIMFCAA|2067-03-21|2006|8725|670|2067|1|3|21|1|2067|670|8725|Monday|2067Q1|N|N|N|2476076|2476134|2475731|2476004|N|N|N|N|N| +2476097|AAAAAAAABEIMFCAA|2067-03-22|2006|8726|670|2067|2|3|22|1|2067|670|8726|Tuesday|2067Q1|N|N|N|2476076|2476134|2475732|2476005|N|N|N|N|N| +2476098|AAAAAAAACEIMFCAA|2067-03-23|2006|8726|670|2067|3|3|23|1|2067|670|8726|Wednesday|2067Q1|N|N|N|2476076|2476134|2475733|2476006|N|N|N|N|N| +2476099|AAAAAAAADEIMFCAA|2067-03-24|2006|8726|670|2067|4|3|24|1|2067|670|8726|Thursday|2067Q1|N|N|N|2476076|2476134|2475734|2476007|N|N|N|N|N| +2476100|AAAAAAAAEEIMFCAA|2067-03-25|2006|8726|670|2067|5|3|25|1|2067|670|8726|Friday|2067Q1|N|Y|N|2476076|2476134|2475735|2476008|N|N|N|N|N| +2476101|AAAAAAAAFEIMFCAA|2067-03-26|2006|8726|670|2067|6|3|26|1|2067|670|8726|Saturday|2067Q1|N|Y|N|2476076|2476134|2475736|2476009|N|N|N|N|N| +2476102|AAAAAAAAGEIMFCAA|2067-03-27|2006|8726|670|2067|0|3|27|1|2067|670|8726|Sunday|2067Q1|N|N|N|2476076|2476134|2475737|2476010|N|N|N|N|N| +2476103|AAAAAAAAHEIMFCAA|2067-03-28|2006|8726|670|2067|1|3|28|1|2067|670|8726|Monday|2067Q1|N|N|N|2476076|2476134|2475738|2476011|N|N|N|N|N| +2476104|AAAAAAAAIEIMFCAA|2067-03-29|2006|8727|670|2067|2|3|29|1|2067|670|8727|Tuesday|2067Q1|N|N|N|2476076|2476134|2475739|2476012|N|N|N|N|N| +2476105|AAAAAAAAJEIMFCAA|2067-03-30|2006|8727|670|2067|3|3|30|1|2067|670|8727|Wednesday|2067Q1|N|N|N|2476076|2476134|2475740|2476013|N|N|N|N|N| +2476106|AAAAAAAAKEIMFCAA|2067-03-31|2006|8727|670|2067|4|3|31|1|2067|670|8727|Thursday|2067Q1|N|N|N|2476076|2476134|2475741|2476014|N|N|N|N|N| +2476107|AAAAAAAALEIMFCAA|2067-04-01|2007|8727|670|2067|5|4|1|1|2067|670|8727|Friday|2067Q1|N|Y|N|2476107|2476196|2475742|2476017|N|N|N|N|N| +2476108|AAAAAAAAMEIMFCAA|2067-04-02|2007|8727|670|2067|6|4|2|2|2067|670|8727|Saturday|2067Q2|N|Y|N|2476107|2476196|2475743|2476018|N|N|N|N|N| +2476109|AAAAAAAANEIMFCAA|2067-04-03|2007|8727|670|2067|0|4|3|2|2067|670|8727|Sunday|2067Q2|N|N|N|2476107|2476196|2475744|2476019|N|N|N|N|N| +2476110|AAAAAAAAOEIMFCAA|2067-04-04|2007|8727|670|2067|1|4|4|2|2067|670|8727|Monday|2067Q2|N|N|N|2476107|2476196|2475745|2476020|N|N|N|N|N| +2476111|AAAAAAAAPEIMFCAA|2067-04-05|2007|8728|670|2067|2|4|5|2|2067|670|8728|Tuesday|2067Q2|N|N|N|2476107|2476196|2475746|2476021|N|N|N|N|N| +2476112|AAAAAAAAAFIMFCAA|2067-04-06|2007|8728|670|2067|3|4|6|2|2067|670|8728|Wednesday|2067Q2|N|N|N|2476107|2476196|2475747|2476022|N|N|N|N|N| +2476113|AAAAAAAABFIMFCAA|2067-04-07|2007|8728|670|2067|4|4|7|2|2067|670|8728|Thursday|2067Q2|N|N|N|2476107|2476196|2475748|2476023|N|N|N|N|N| +2476114|AAAAAAAACFIMFCAA|2067-04-08|2007|8728|670|2067|5|4|8|2|2067|670|8728|Friday|2067Q2|N|Y|N|2476107|2476196|2475749|2476024|N|N|N|N|N| +2476115|AAAAAAAADFIMFCAA|2067-04-09|2007|8728|670|2067|6|4|9|2|2067|670|8728|Saturday|2067Q2|N|Y|N|2476107|2476196|2475750|2476025|N|N|N|N|N| +2476116|AAAAAAAAEFIMFCAA|2067-04-10|2007|8728|670|2067|0|4|10|2|2067|670|8728|Sunday|2067Q2|N|N|N|2476107|2476196|2475751|2476026|N|N|N|N|N| +2476117|AAAAAAAAFFIMFCAA|2067-04-11|2007|8728|670|2067|1|4|11|2|2067|670|8728|Monday|2067Q2|N|N|N|2476107|2476196|2475752|2476027|N|N|N|N|N| +2476118|AAAAAAAAGFIMFCAA|2067-04-12|2007|8729|670|2067|2|4|12|2|2067|670|8729|Tuesday|2067Q2|N|N|N|2476107|2476196|2475753|2476028|N|N|N|N|N| +2476119|AAAAAAAAHFIMFCAA|2067-04-13|2007|8729|670|2067|3|4|13|2|2067|670|8729|Wednesday|2067Q2|N|N|N|2476107|2476196|2475754|2476029|N|N|N|N|N| +2476120|AAAAAAAAIFIMFCAA|2067-04-14|2007|8729|670|2067|4|4|14|2|2067|670|8729|Thursday|2067Q2|N|N|N|2476107|2476196|2475755|2476030|N|N|N|N|N| +2476121|AAAAAAAAJFIMFCAA|2067-04-15|2007|8729|670|2067|5|4|15|2|2067|670|8729|Friday|2067Q2|N|Y|N|2476107|2476196|2475756|2476031|N|N|N|N|N| +2476122|AAAAAAAAKFIMFCAA|2067-04-16|2007|8729|670|2067|6|4|16|2|2067|670|8729|Saturday|2067Q2|N|Y|N|2476107|2476196|2475757|2476032|N|N|N|N|N| +2476123|AAAAAAAALFIMFCAA|2067-04-17|2007|8729|670|2067|0|4|17|2|2067|670|8729|Sunday|2067Q2|N|N|N|2476107|2476196|2475758|2476033|N|N|N|N|N| +2476124|AAAAAAAAMFIMFCAA|2067-04-18|2007|8729|670|2067|1|4|18|2|2067|670|8729|Monday|2067Q2|N|N|N|2476107|2476196|2475759|2476034|N|N|N|N|N| +2476125|AAAAAAAANFIMFCAA|2067-04-19|2007|8730|670|2067|2|4|19|2|2067|670|8730|Tuesday|2067Q2|N|N|N|2476107|2476196|2475760|2476035|N|N|N|N|N| +2476126|AAAAAAAAOFIMFCAA|2067-04-20|2007|8730|670|2067|3|4|20|2|2067|670|8730|Wednesday|2067Q2|N|N|N|2476107|2476196|2475761|2476036|N|N|N|N|N| +2476127|AAAAAAAAPFIMFCAA|2067-04-21|2007|8730|670|2067|4|4|21|2|2067|670|8730|Thursday|2067Q2|N|N|N|2476107|2476196|2475762|2476037|N|N|N|N|N| +2476128|AAAAAAAAAGIMFCAA|2067-04-22|2007|8730|670|2067|5|4|22|2|2067|670|8730|Friday|2067Q2|N|Y|N|2476107|2476196|2475763|2476038|N|N|N|N|N| +2476129|AAAAAAAABGIMFCAA|2067-04-23|2007|8730|670|2067|6|4|23|2|2067|670|8730|Saturday|2067Q2|N|Y|N|2476107|2476196|2475764|2476039|N|N|N|N|N| +2476130|AAAAAAAACGIMFCAA|2067-04-24|2007|8730|670|2067|0|4|24|2|2067|670|8730|Sunday|2067Q2|N|N|N|2476107|2476196|2475765|2476040|N|N|N|N|N| +2476131|AAAAAAAADGIMFCAA|2067-04-25|2007|8730|670|2067|1|4|25|2|2067|670|8730|Monday|2067Q2|N|N|N|2476107|2476196|2475766|2476041|N|N|N|N|N| +2476132|AAAAAAAAEGIMFCAA|2067-04-26|2007|8731|670|2067|2|4|26|2|2067|670|8731|Tuesday|2067Q2|N|N|N|2476107|2476196|2475767|2476042|N|N|N|N|N| +2476133|AAAAAAAAFGIMFCAA|2067-04-27|2007|8731|670|2067|3|4|27|2|2067|670|8731|Wednesday|2067Q2|N|N|N|2476107|2476196|2475768|2476043|N|N|N|N|N| +2476134|AAAAAAAAGGIMFCAA|2067-04-28|2007|8731|670|2067|4|4|28|2|2067|670|8731|Thursday|2067Q2|N|N|N|2476107|2476196|2475769|2476044|N|N|N|N|N| +2476135|AAAAAAAAHGIMFCAA|2067-04-29|2007|8731|670|2067|5|4|29|2|2067|670|8731|Friday|2067Q2|N|Y|N|2476107|2476196|2475770|2476045|N|N|N|N|N| +2476136|AAAAAAAAIGIMFCAA|2067-04-30|2007|8731|670|2067|6|4|30|2|2067|670|8731|Saturday|2067Q2|N|Y|N|2476107|2476196|2475771|2476046|N|N|N|N|N| +2476137|AAAAAAAAJGIMFCAA|2067-05-01|2008|8731|670|2067|0|5|1|2|2067|670|8731|Sunday|2067Q2|N|N|N|2476137|2476256|2475772|2476047|N|N|N|N|N| +2476138|AAAAAAAAKGIMFCAA|2067-05-02|2008|8731|670|2067|1|5|2|2|2067|670|8731|Monday|2067Q2|N|N|N|2476137|2476256|2475773|2476048|N|N|N|N|N| +2476139|AAAAAAAALGIMFCAA|2067-05-03|2008|8732|670|2067|2|5|3|2|2067|670|8732|Tuesday|2067Q2|N|N|N|2476137|2476256|2475774|2476049|N|N|N|N|N| +2476140|AAAAAAAAMGIMFCAA|2067-05-04|2008|8732|670|2067|3|5|4|2|2067|670|8732|Wednesday|2067Q2|N|N|N|2476137|2476256|2475775|2476050|N|N|N|N|N| +2476141|AAAAAAAANGIMFCAA|2067-05-05|2008|8732|670|2067|4|5|5|2|2067|670|8732|Thursday|2067Q2|N|N|N|2476137|2476256|2475776|2476051|N|N|N|N|N| +2476142|AAAAAAAAOGIMFCAA|2067-05-06|2008|8732|670|2067|5|5|6|2|2067|670|8732|Friday|2067Q2|N|Y|N|2476137|2476256|2475777|2476052|N|N|N|N|N| +2476143|AAAAAAAAPGIMFCAA|2067-05-07|2008|8732|670|2067|6|5|7|2|2067|670|8732|Saturday|2067Q2|N|Y|N|2476137|2476256|2475778|2476053|N|N|N|N|N| +2476144|AAAAAAAAAHIMFCAA|2067-05-08|2008|8732|670|2067|0|5|8|2|2067|670|8732|Sunday|2067Q2|N|N|N|2476137|2476256|2475779|2476054|N|N|N|N|N| +2476145|AAAAAAAABHIMFCAA|2067-05-09|2008|8732|670|2067|1|5|9|2|2067|670|8732|Monday|2067Q2|N|N|N|2476137|2476256|2475780|2476055|N|N|N|N|N| +2476146|AAAAAAAACHIMFCAA|2067-05-10|2008|8733|670|2067|2|5|10|2|2067|670|8733|Tuesday|2067Q2|N|N|N|2476137|2476256|2475781|2476056|N|N|N|N|N| +2476147|AAAAAAAADHIMFCAA|2067-05-11|2008|8733|670|2067|3|5|11|2|2067|670|8733|Wednesday|2067Q2|N|N|N|2476137|2476256|2475782|2476057|N|N|N|N|N| +2476148|AAAAAAAAEHIMFCAA|2067-05-12|2008|8733|670|2067|4|5|12|2|2067|670|8733|Thursday|2067Q2|N|N|N|2476137|2476256|2475783|2476058|N|N|N|N|N| +2476149|AAAAAAAAFHIMFCAA|2067-05-13|2008|8733|670|2067|5|5|13|2|2067|670|8733|Friday|2067Q2|N|Y|N|2476137|2476256|2475784|2476059|N|N|N|N|N| +2476150|AAAAAAAAGHIMFCAA|2067-05-14|2008|8733|670|2067|6|5|14|2|2067|670|8733|Saturday|2067Q2|N|Y|N|2476137|2476256|2475785|2476060|N|N|N|N|N| +2476151|AAAAAAAAHHIMFCAA|2067-05-15|2008|8733|670|2067|0|5|15|2|2067|670|8733|Sunday|2067Q2|N|N|N|2476137|2476256|2475786|2476061|N|N|N|N|N| +2476152|AAAAAAAAIHIMFCAA|2067-05-16|2008|8733|670|2067|1|5|16|2|2067|670|8733|Monday|2067Q2|N|N|N|2476137|2476256|2475787|2476062|N|N|N|N|N| +2476153|AAAAAAAAJHIMFCAA|2067-05-17|2008|8734|670|2067|2|5|17|2|2067|670|8734|Tuesday|2067Q2|N|N|N|2476137|2476256|2475788|2476063|N|N|N|N|N| +2476154|AAAAAAAAKHIMFCAA|2067-05-18|2008|8734|670|2067|3|5|18|2|2067|670|8734|Wednesday|2067Q2|N|N|N|2476137|2476256|2475789|2476064|N|N|N|N|N| +2476155|AAAAAAAALHIMFCAA|2067-05-19|2008|8734|670|2067|4|5|19|2|2067|670|8734|Thursday|2067Q2|N|N|N|2476137|2476256|2475790|2476065|N|N|N|N|N| +2476156|AAAAAAAAMHIMFCAA|2067-05-20|2008|8734|670|2067|5|5|20|2|2067|670|8734|Friday|2067Q2|N|Y|N|2476137|2476256|2475791|2476066|N|N|N|N|N| +2476157|AAAAAAAANHIMFCAA|2067-05-21|2008|8734|670|2067|6|5|21|2|2067|670|8734|Saturday|2067Q2|N|Y|N|2476137|2476256|2475792|2476067|N|N|N|N|N| +2476158|AAAAAAAAOHIMFCAA|2067-05-22|2008|8734|670|2067|0|5|22|2|2067|670|8734|Sunday|2067Q2|N|N|N|2476137|2476256|2475793|2476068|N|N|N|N|N| +2476159|AAAAAAAAPHIMFCAA|2067-05-23|2008|8734|670|2067|1|5|23|2|2067|670|8734|Monday|2067Q2|N|N|N|2476137|2476256|2475794|2476069|N|N|N|N|N| +2476160|AAAAAAAAAIIMFCAA|2067-05-24|2008|8735|670|2067|2|5|24|2|2067|670|8735|Tuesday|2067Q2|N|N|N|2476137|2476256|2475795|2476070|N|N|N|N|N| +2476161|AAAAAAAABIIMFCAA|2067-05-25|2008|8735|670|2067|3|5|25|2|2067|670|8735|Wednesday|2067Q2|N|N|N|2476137|2476256|2475796|2476071|N|N|N|N|N| +2476162|AAAAAAAACIIMFCAA|2067-05-26|2008|8735|670|2067|4|5|26|2|2067|670|8735|Thursday|2067Q2|N|N|N|2476137|2476256|2475797|2476072|N|N|N|N|N| +2476163|AAAAAAAADIIMFCAA|2067-05-27|2008|8735|670|2067|5|5|27|2|2067|670|8735|Friday|2067Q2|N|Y|N|2476137|2476256|2475798|2476073|N|N|N|N|N| +2476164|AAAAAAAAEIIMFCAA|2067-05-28|2008|8735|670|2067|6|5|28|2|2067|670|8735|Saturday|2067Q2|N|Y|N|2476137|2476256|2475799|2476074|N|N|N|N|N| +2476165|AAAAAAAAFIIMFCAA|2067-05-29|2008|8735|670|2067|0|5|29|2|2067|670|8735|Sunday|2067Q2|N|N|N|2476137|2476256|2475800|2476075|N|N|N|N|N| +2476166|AAAAAAAAGIIMFCAA|2067-05-30|2008|8735|670|2067|1|5|30|2|2067|670|8735|Monday|2067Q2|N|N|N|2476137|2476256|2475801|2476076|N|N|N|N|N| +2476167|AAAAAAAAHIIMFCAA|2067-05-31|2008|8736|670|2067|2|5|31|2|2067|670|8736|Tuesday|2067Q2|N|N|N|2476137|2476256|2475802|2476077|N|N|N|N|N| +2476168|AAAAAAAAIIIMFCAA|2067-06-01|2009|8736|671|2067|3|6|1|2|2067|671|8736|Wednesday|2067Q2|N|N|N|2476168|2476318|2475803|2476078|N|N|N|N|N| +2476169|AAAAAAAAJIIMFCAA|2067-06-02|2009|8736|671|2067|4|6|2|2|2067|671|8736|Thursday|2067Q2|N|N|N|2476168|2476318|2475804|2476079|N|N|N|N|N| +2476170|AAAAAAAAKIIMFCAA|2067-06-03|2009|8736|671|2067|5|6|3|2|2067|671|8736|Friday|2067Q2|N|Y|N|2476168|2476318|2475805|2476080|N|N|N|N|N| +2476171|AAAAAAAALIIMFCAA|2067-06-04|2009|8736|671|2067|6|6|4|2|2067|671|8736|Saturday|2067Q2|N|Y|N|2476168|2476318|2475806|2476081|N|N|N|N|N| +2476172|AAAAAAAAMIIMFCAA|2067-06-05|2009|8736|671|2067|0|6|5|2|2067|671|8736|Sunday|2067Q2|N|N|N|2476168|2476318|2475807|2476082|N|N|N|N|N| +2476173|AAAAAAAANIIMFCAA|2067-06-06|2009|8736|671|2067|1|6|6|2|2067|671|8736|Monday|2067Q2|N|N|N|2476168|2476318|2475808|2476083|N|N|N|N|N| +2476174|AAAAAAAAOIIMFCAA|2067-06-07|2009|8737|671|2067|2|6|7|2|2067|671|8737|Tuesday|2067Q2|N|N|N|2476168|2476318|2475809|2476084|N|N|N|N|N| +2476175|AAAAAAAAPIIMFCAA|2067-06-08|2009|8737|671|2067|3|6|8|2|2067|671|8737|Wednesday|2067Q2|N|N|N|2476168|2476318|2475810|2476085|N|N|N|N|N| +2476176|AAAAAAAAAJIMFCAA|2067-06-09|2009|8737|671|2067|4|6|9|2|2067|671|8737|Thursday|2067Q2|N|N|N|2476168|2476318|2475811|2476086|N|N|N|N|N| +2476177|AAAAAAAABJIMFCAA|2067-06-10|2009|8737|671|2067|5|6|10|2|2067|671|8737|Friday|2067Q2|N|Y|N|2476168|2476318|2475812|2476087|N|N|N|N|N| +2476178|AAAAAAAACJIMFCAA|2067-06-11|2009|8737|671|2067|6|6|11|2|2067|671|8737|Saturday|2067Q2|N|Y|N|2476168|2476318|2475813|2476088|N|N|N|N|N| +2476179|AAAAAAAADJIMFCAA|2067-06-12|2009|8737|671|2067|0|6|12|2|2067|671|8737|Sunday|2067Q2|N|N|N|2476168|2476318|2475814|2476089|N|N|N|N|N| +2476180|AAAAAAAAEJIMFCAA|2067-06-13|2009|8737|671|2067|1|6|13|2|2067|671|8737|Monday|2067Q2|N|N|N|2476168|2476318|2475815|2476090|N|N|N|N|N| +2476181|AAAAAAAAFJIMFCAA|2067-06-14|2009|8738|671|2067|2|6|14|2|2067|671|8738|Tuesday|2067Q2|N|N|N|2476168|2476318|2475816|2476091|N|N|N|N|N| +2476182|AAAAAAAAGJIMFCAA|2067-06-15|2009|8738|671|2067|3|6|15|2|2067|671|8738|Wednesday|2067Q2|N|N|N|2476168|2476318|2475817|2476092|N|N|N|N|N| +2476183|AAAAAAAAHJIMFCAA|2067-06-16|2009|8738|671|2067|4|6|16|2|2067|671|8738|Thursday|2067Q2|N|N|N|2476168|2476318|2475818|2476093|N|N|N|N|N| +2476184|AAAAAAAAIJIMFCAA|2067-06-17|2009|8738|671|2067|5|6|17|2|2067|671|8738|Friday|2067Q2|N|Y|N|2476168|2476318|2475819|2476094|N|N|N|N|N| +2476185|AAAAAAAAJJIMFCAA|2067-06-18|2009|8738|671|2067|6|6|18|2|2067|671|8738|Saturday|2067Q2|N|Y|N|2476168|2476318|2475820|2476095|N|N|N|N|N| +2476186|AAAAAAAAKJIMFCAA|2067-06-19|2009|8738|671|2067|0|6|19|2|2067|671|8738|Sunday|2067Q2|N|N|N|2476168|2476318|2475821|2476096|N|N|N|N|N| +2476187|AAAAAAAALJIMFCAA|2067-06-20|2009|8738|671|2067|1|6|20|2|2067|671|8738|Monday|2067Q2|N|N|N|2476168|2476318|2475822|2476097|N|N|N|N|N| +2476188|AAAAAAAAMJIMFCAA|2067-06-21|2009|8739|671|2067|2|6|21|2|2067|671|8739|Tuesday|2067Q2|N|N|N|2476168|2476318|2475823|2476098|N|N|N|N|N| +2476189|AAAAAAAANJIMFCAA|2067-06-22|2009|8739|671|2067|3|6|22|2|2067|671|8739|Wednesday|2067Q2|N|N|N|2476168|2476318|2475824|2476099|N|N|N|N|N| +2476190|AAAAAAAAOJIMFCAA|2067-06-23|2009|8739|671|2067|4|6|23|2|2067|671|8739|Thursday|2067Q2|N|N|N|2476168|2476318|2475825|2476100|N|N|N|N|N| +2476191|AAAAAAAAPJIMFCAA|2067-06-24|2009|8739|671|2067|5|6|24|2|2067|671|8739|Friday|2067Q2|N|Y|N|2476168|2476318|2475826|2476101|N|N|N|N|N| +2476192|AAAAAAAAAKIMFCAA|2067-06-25|2009|8739|671|2067|6|6|25|2|2067|671|8739|Saturday|2067Q2|N|Y|N|2476168|2476318|2475827|2476102|N|N|N|N|N| +2476193|AAAAAAAABKIMFCAA|2067-06-26|2009|8739|671|2067|0|6|26|2|2067|671|8739|Sunday|2067Q2|N|N|N|2476168|2476318|2475828|2476103|N|N|N|N|N| +2476194|AAAAAAAACKIMFCAA|2067-06-27|2009|8739|671|2067|1|6|27|2|2067|671|8739|Monday|2067Q2|N|N|N|2476168|2476318|2475829|2476104|N|N|N|N|N| +2476195|AAAAAAAADKIMFCAA|2067-06-28|2009|8740|671|2067|2|6|28|2|2067|671|8740|Tuesday|2067Q2|N|N|N|2476168|2476318|2475830|2476105|N|N|N|N|N| +2476196|AAAAAAAAEKIMFCAA|2067-06-29|2009|8740|671|2067|3|6|29|2|2067|671|8740|Wednesday|2067Q2|N|N|N|2476168|2476318|2475831|2476106|N|N|N|N|N| +2476197|AAAAAAAAFKIMFCAA|2067-06-30|2009|8740|671|2067|4|6|30|2|2067|671|8740|Thursday|2067Q2|N|N|N|2476168|2476318|2475832|2476107|N|N|N|N|N| +2476198|AAAAAAAAGKIMFCAA|2067-07-01|2010|8740|671|2067|5|7|1|2|2067|671|8740|Friday|2067Q2|N|Y|N|2476198|2476378|2475833|2476107|N|N|N|N|N| +2476199|AAAAAAAAHKIMFCAA|2067-07-02|2010|8740|671|2067|6|7|2|3|2067|671|8740|Saturday|2067Q3|N|Y|N|2476198|2476378|2475834|2476108|N|N|N|N|N| +2476200|AAAAAAAAIKIMFCAA|2067-07-03|2010|8740|671|2067|0|7|3|3|2067|671|8740|Sunday|2067Q3|N|N|N|2476198|2476378|2475835|2476109|N|N|N|N|N| +2476201|AAAAAAAAJKIMFCAA|2067-07-04|2010|8740|671|2067|1|7|4|3|2067|671|8740|Monday|2067Q3|N|N|N|2476198|2476378|2475836|2476110|N|N|N|N|N| +2476202|AAAAAAAAKKIMFCAA|2067-07-05|2010|8741|671|2067|2|7|5|3|2067|671|8741|Tuesday|2067Q3|Y|N|N|2476198|2476378|2475837|2476111|N|N|N|N|N| +2476203|AAAAAAAALKIMFCAA|2067-07-06|2010|8741|671|2067|3|7|6|3|2067|671|8741|Wednesday|2067Q3|N|N|Y|2476198|2476378|2475838|2476112|N|N|N|N|N| +2476204|AAAAAAAAMKIMFCAA|2067-07-07|2010|8741|671|2067|4|7|7|3|2067|671|8741|Thursday|2067Q3|N|N|N|2476198|2476378|2475839|2476113|N|N|N|N|N| +2476205|AAAAAAAANKIMFCAA|2067-07-08|2010|8741|671|2067|5|7|8|3|2067|671|8741|Friday|2067Q3|N|Y|N|2476198|2476378|2475840|2476114|N|N|N|N|N| +2476206|AAAAAAAAOKIMFCAA|2067-07-09|2010|8741|671|2067|6|7|9|3|2067|671|8741|Saturday|2067Q3|N|Y|N|2476198|2476378|2475841|2476115|N|N|N|N|N| +2476207|AAAAAAAAPKIMFCAA|2067-07-10|2010|8741|671|2067|0|7|10|3|2067|671|8741|Sunday|2067Q3|N|N|N|2476198|2476378|2475842|2476116|N|N|N|N|N| +2476208|AAAAAAAAALIMFCAA|2067-07-11|2010|8741|671|2067|1|7|11|3|2067|671|8741|Monday|2067Q3|N|N|N|2476198|2476378|2475843|2476117|N|N|N|N|N| +2476209|AAAAAAAABLIMFCAA|2067-07-12|2010|8742|671|2067|2|7|12|3|2067|671|8742|Tuesday|2067Q3|N|N|N|2476198|2476378|2475844|2476118|N|N|N|N|N| +2476210|AAAAAAAACLIMFCAA|2067-07-13|2010|8742|671|2067|3|7|13|3|2067|671|8742|Wednesday|2067Q3|N|N|N|2476198|2476378|2475845|2476119|N|N|N|N|N| +2476211|AAAAAAAADLIMFCAA|2067-07-14|2010|8742|671|2067|4|7|14|3|2067|671|8742|Thursday|2067Q3|N|N|N|2476198|2476378|2475846|2476120|N|N|N|N|N| +2476212|AAAAAAAAELIMFCAA|2067-07-15|2010|8742|671|2067|5|7|15|3|2067|671|8742|Friday|2067Q3|N|Y|N|2476198|2476378|2475847|2476121|N|N|N|N|N| +2476213|AAAAAAAAFLIMFCAA|2067-07-16|2010|8742|671|2067|6|7|16|3|2067|671|8742|Saturday|2067Q3|N|Y|N|2476198|2476378|2475848|2476122|N|N|N|N|N| +2476214|AAAAAAAAGLIMFCAA|2067-07-17|2010|8742|671|2067|0|7|17|3|2067|671|8742|Sunday|2067Q3|N|N|N|2476198|2476378|2475849|2476123|N|N|N|N|N| +2476215|AAAAAAAAHLIMFCAA|2067-07-18|2010|8742|671|2067|1|7|18|3|2067|671|8742|Monday|2067Q3|N|N|N|2476198|2476378|2475850|2476124|N|N|N|N|N| +2476216|AAAAAAAAILIMFCAA|2067-07-19|2010|8743|671|2067|2|7|19|3|2067|671|8743|Tuesday|2067Q3|N|N|N|2476198|2476378|2475851|2476125|N|N|N|N|N| +2476217|AAAAAAAAJLIMFCAA|2067-07-20|2010|8743|671|2067|3|7|20|3|2067|671|8743|Wednesday|2067Q3|N|N|N|2476198|2476378|2475852|2476126|N|N|N|N|N| +2476218|AAAAAAAAKLIMFCAA|2067-07-21|2010|8743|671|2067|4|7|21|3|2067|671|8743|Thursday|2067Q3|N|N|N|2476198|2476378|2475853|2476127|N|N|N|N|N| +2476219|AAAAAAAALLIMFCAA|2067-07-22|2010|8743|671|2067|5|7|22|3|2067|671|8743|Friday|2067Q3|N|Y|N|2476198|2476378|2475854|2476128|N|N|N|N|N| +2476220|AAAAAAAAMLIMFCAA|2067-07-23|2010|8743|671|2067|6|7|23|3|2067|671|8743|Saturday|2067Q3|N|Y|N|2476198|2476378|2475855|2476129|N|N|N|N|N| +2476221|AAAAAAAANLIMFCAA|2067-07-24|2010|8743|671|2067|0|7|24|3|2067|671|8743|Sunday|2067Q3|N|N|N|2476198|2476378|2475856|2476130|N|N|N|N|N| +2476222|AAAAAAAAOLIMFCAA|2067-07-25|2010|8743|671|2067|1|7|25|3|2067|671|8743|Monday|2067Q3|N|N|N|2476198|2476378|2475857|2476131|N|N|N|N|N| +2476223|AAAAAAAAPLIMFCAA|2067-07-26|2010|8744|671|2067|2|7|26|3|2067|671|8744|Tuesday|2067Q3|N|N|N|2476198|2476378|2475858|2476132|N|N|N|N|N| +2476224|AAAAAAAAAMIMFCAA|2067-07-27|2010|8744|671|2067|3|7|27|3|2067|671|8744|Wednesday|2067Q3|N|N|N|2476198|2476378|2475859|2476133|N|N|N|N|N| +2476225|AAAAAAAABMIMFCAA|2067-07-28|2010|8744|671|2067|4|7|28|3|2067|671|8744|Thursday|2067Q3|N|N|N|2476198|2476378|2475860|2476134|N|N|N|N|N| +2476226|AAAAAAAACMIMFCAA|2067-07-29|2010|8744|671|2067|5|7|29|3|2067|671|8744|Friday|2067Q3|N|Y|N|2476198|2476378|2475861|2476135|N|N|N|N|N| +2476227|AAAAAAAADMIMFCAA|2067-07-30|2010|8744|671|2067|6|7|30|3|2067|671|8744|Saturday|2067Q3|N|Y|N|2476198|2476378|2475862|2476136|N|N|N|N|N| +2476228|AAAAAAAAEMIMFCAA|2067-07-31|2010|8744|671|2067|0|7|31|3|2067|671|8744|Sunday|2067Q3|N|N|N|2476198|2476378|2475863|2476137|N|N|N|N|N| +2476229|AAAAAAAAFMIMFCAA|2067-08-01|2011|8744|671|2067|1|8|1|3|2067|671|8744|Monday|2067Q3|N|N|N|2476229|2476440|2475864|2476138|N|N|N|N|N| +2476230|AAAAAAAAGMIMFCAA|2067-08-02|2011|8745|671|2067|2|8|2|3|2067|671|8745|Tuesday|2067Q3|N|N|N|2476229|2476440|2475865|2476139|N|N|N|N|N| +2476231|AAAAAAAAHMIMFCAA|2067-08-03|2011|8745|671|2067|3|8|3|3|2067|671|8745|Wednesday|2067Q3|N|N|N|2476229|2476440|2475866|2476140|N|N|N|N|N| +2476232|AAAAAAAAIMIMFCAA|2067-08-04|2011|8745|671|2067|4|8|4|3|2067|671|8745|Thursday|2067Q3|N|N|N|2476229|2476440|2475867|2476141|N|N|N|N|N| +2476233|AAAAAAAAJMIMFCAA|2067-08-05|2011|8745|671|2067|5|8|5|3|2067|671|8745|Friday|2067Q3|N|Y|N|2476229|2476440|2475868|2476142|N|N|N|N|N| +2476234|AAAAAAAAKMIMFCAA|2067-08-06|2011|8745|671|2067|6|8|6|3|2067|671|8745|Saturday|2067Q3|N|Y|N|2476229|2476440|2475869|2476143|N|N|N|N|N| +2476235|AAAAAAAALMIMFCAA|2067-08-07|2011|8745|671|2067|0|8|7|3|2067|671|8745|Sunday|2067Q3|N|N|N|2476229|2476440|2475870|2476144|N|N|N|N|N| +2476236|AAAAAAAAMMIMFCAA|2067-08-08|2011|8745|671|2067|1|8|8|3|2067|671|8745|Monday|2067Q3|N|N|N|2476229|2476440|2475871|2476145|N|N|N|N|N| +2476237|AAAAAAAANMIMFCAA|2067-08-09|2011|8746|671|2067|2|8|9|3|2067|671|8746|Tuesday|2067Q3|N|N|N|2476229|2476440|2475872|2476146|N|N|N|N|N| +2476238|AAAAAAAAOMIMFCAA|2067-08-10|2011|8746|671|2067|3|8|10|3|2067|671|8746|Wednesday|2067Q3|N|N|N|2476229|2476440|2475873|2476147|N|N|N|N|N| +2476239|AAAAAAAAPMIMFCAA|2067-08-11|2011|8746|671|2067|4|8|11|3|2067|671|8746|Thursday|2067Q3|N|N|N|2476229|2476440|2475874|2476148|N|N|N|N|N| +2476240|AAAAAAAAANIMFCAA|2067-08-12|2011|8746|671|2067|5|8|12|3|2067|671|8746|Friday|2067Q3|N|Y|N|2476229|2476440|2475875|2476149|N|N|N|N|N| +2476241|AAAAAAAABNIMFCAA|2067-08-13|2011|8746|671|2067|6|8|13|3|2067|671|8746|Saturday|2067Q3|N|Y|N|2476229|2476440|2475876|2476150|N|N|N|N|N| +2476242|AAAAAAAACNIMFCAA|2067-08-14|2011|8746|671|2067|0|8|14|3|2067|671|8746|Sunday|2067Q3|N|N|N|2476229|2476440|2475877|2476151|N|N|N|N|N| +2476243|AAAAAAAADNIMFCAA|2067-08-15|2011|8746|671|2067|1|8|15|3|2067|671|8746|Monday|2067Q3|N|N|N|2476229|2476440|2475878|2476152|N|N|N|N|N| +2476244|AAAAAAAAENIMFCAA|2067-08-16|2011|8747|671|2067|2|8|16|3|2067|671|8747|Tuesday|2067Q3|N|N|N|2476229|2476440|2475879|2476153|N|N|N|N|N| +2476245|AAAAAAAAFNIMFCAA|2067-08-17|2011|8747|671|2067|3|8|17|3|2067|671|8747|Wednesday|2067Q3|N|N|N|2476229|2476440|2475880|2476154|N|N|N|N|N| +2476246|AAAAAAAAGNIMFCAA|2067-08-18|2011|8747|671|2067|4|8|18|3|2067|671|8747|Thursday|2067Q3|N|N|N|2476229|2476440|2475881|2476155|N|N|N|N|N| +2476247|AAAAAAAAHNIMFCAA|2067-08-19|2011|8747|671|2067|5|8|19|3|2067|671|8747|Friday|2067Q3|N|Y|N|2476229|2476440|2475882|2476156|N|N|N|N|N| +2476248|AAAAAAAAINIMFCAA|2067-08-20|2011|8747|671|2067|6|8|20|3|2067|671|8747|Saturday|2067Q3|N|Y|N|2476229|2476440|2475883|2476157|N|N|N|N|N| +2476249|AAAAAAAAJNIMFCAA|2067-08-21|2011|8747|671|2067|0|8|21|3|2067|671|8747|Sunday|2067Q3|N|N|N|2476229|2476440|2475884|2476158|N|N|N|N|N| +2476250|AAAAAAAAKNIMFCAA|2067-08-22|2011|8747|671|2067|1|8|22|3|2067|671|8747|Monday|2067Q3|N|N|N|2476229|2476440|2475885|2476159|N|N|N|N|N| +2476251|AAAAAAAALNIMFCAA|2067-08-23|2011|8748|671|2067|2|8|23|3|2067|671|8748|Tuesday|2067Q3|N|N|N|2476229|2476440|2475886|2476160|N|N|N|N|N| +2476252|AAAAAAAAMNIMFCAA|2067-08-24|2011|8748|671|2067|3|8|24|3|2067|671|8748|Wednesday|2067Q3|N|N|N|2476229|2476440|2475887|2476161|N|N|N|N|N| +2476253|AAAAAAAANNIMFCAA|2067-08-25|2011|8748|671|2067|4|8|25|3|2067|671|8748|Thursday|2067Q3|N|N|N|2476229|2476440|2475888|2476162|N|N|N|N|N| +2476254|AAAAAAAAONIMFCAA|2067-08-26|2011|8748|671|2067|5|8|26|3|2067|671|8748|Friday|2067Q3|N|Y|N|2476229|2476440|2475889|2476163|N|N|N|N|N| +2476255|AAAAAAAAPNIMFCAA|2067-08-27|2011|8748|671|2067|6|8|27|3|2067|671|8748|Saturday|2067Q3|N|Y|N|2476229|2476440|2475890|2476164|N|N|N|N|N| +2476256|AAAAAAAAAOIMFCAA|2067-08-28|2011|8748|671|2067|0|8|28|3|2067|671|8748|Sunday|2067Q3|N|N|N|2476229|2476440|2475891|2476165|N|N|N|N|N| +2476257|AAAAAAAABOIMFCAA|2067-08-29|2011|8748|671|2067|1|8|29|3|2067|671|8748|Monday|2067Q3|N|N|N|2476229|2476440|2475892|2476166|N|N|N|N|N| +2476258|AAAAAAAACOIMFCAA|2067-08-30|2011|8749|671|2067|2|8|30|3|2067|671|8749|Tuesday|2067Q3|N|N|N|2476229|2476440|2475893|2476167|N|N|N|N|N| +2476259|AAAAAAAADOIMFCAA|2067-08-31|2011|8749|671|2067|3|8|31|3|2067|671|8749|Wednesday|2067Q3|N|N|N|2476229|2476440|2475894|2476168|N|N|N|N|N| +2476260|AAAAAAAAEOIMFCAA|2067-09-01|2012|8749|672|2067|4|9|1|3|2067|672|8749|Thursday|2067Q3|N|N|N|2476260|2476502|2475895|2476169|N|N|N|N|N| +2476261|AAAAAAAAFOIMFCAA|2067-09-02|2012|8749|672|2067|5|9|2|3|2067|672|8749|Friday|2067Q3|N|Y|N|2476260|2476502|2475896|2476170|N|N|N|N|N| +2476262|AAAAAAAAGOIMFCAA|2067-09-03|2012|8749|672|2067|6|9|3|3|2067|672|8749|Saturday|2067Q3|N|Y|N|2476260|2476502|2475897|2476171|N|N|N|N|N| +2476263|AAAAAAAAHOIMFCAA|2067-09-04|2012|8749|672|2067|0|9|4|3|2067|672|8749|Sunday|2067Q3|N|N|N|2476260|2476502|2475898|2476172|N|N|N|N|N| +2476264|AAAAAAAAIOIMFCAA|2067-09-05|2012|8749|672|2067|1|9|5|3|2067|672|8749|Monday|2067Q3|N|N|N|2476260|2476502|2475899|2476173|N|N|N|N|N| +2476265|AAAAAAAAJOIMFCAA|2067-09-06|2012|8750|672|2067|2|9|6|3|2067|672|8750|Tuesday|2067Q3|N|N|N|2476260|2476502|2475900|2476174|N|N|N|N|N| +2476266|AAAAAAAAKOIMFCAA|2067-09-07|2012|8750|672|2067|3|9|7|3|2067|672|8750|Wednesday|2067Q3|N|N|N|2476260|2476502|2475901|2476175|N|N|N|N|N| +2476267|AAAAAAAALOIMFCAA|2067-09-08|2012|8750|672|2067|4|9|8|3|2067|672|8750|Thursday|2067Q3|N|N|N|2476260|2476502|2475902|2476176|N|N|N|N|N| +2476268|AAAAAAAAMOIMFCAA|2067-09-09|2012|8750|672|2067|5|9|9|3|2067|672|8750|Friday|2067Q3|N|Y|N|2476260|2476502|2475903|2476177|N|N|N|N|N| +2476269|AAAAAAAANOIMFCAA|2067-09-10|2012|8750|672|2067|6|9|10|3|2067|672|8750|Saturday|2067Q3|N|Y|N|2476260|2476502|2475904|2476178|N|N|N|N|N| +2476270|AAAAAAAAOOIMFCAA|2067-09-11|2012|8750|672|2067|0|9|11|3|2067|672|8750|Sunday|2067Q3|N|N|N|2476260|2476502|2475905|2476179|N|N|N|N|N| +2476271|AAAAAAAAPOIMFCAA|2067-09-12|2012|8750|672|2067|1|9|12|3|2067|672|8750|Monday|2067Q3|N|N|N|2476260|2476502|2475906|2476180|N|N|N|N|N| +2476272|AAAAAAAAAPIMFCAA|2067-09-13|2012|8751|672|2067|2|9|13|3|2067|672|8751|Tuesday|2067Q3|N|N|N|2476260|2476502|2475907|2476181|N|N|N|N|N| +2476273|AAAAAAAABPIMFCAA|2067-09-14|2012|8751|672|2067|3|9|14|3|2067|672|8751|Wednesday|2067Q3|N|N|N|2476260|2476502|2475908|2476182|N|N|N|N|N| +2476274|AAAAAAAACPIMFCAA|2067-09-15|2012|8751|672|2067|4|9|15|3|2067|672|8751|Thursday|2067Q3|N|N|N|2476260|2476502|2475909|2476183|N|N|N|N|N| +2476275|AAAAAAAADPIMFCAA|2067-09-16|2012|8751|672|2067|5|9|16|3|2067|672|8751|Friday|2067Q3|N|Y|N|2476260|2476502|2475910|2476184|N|N|N|N|N| +2476276|AAAAAAAAEPIMFCAA|2067-09-17|2012|8751|672|2067|6|9|17|3|2067|672|8751|Saturday|2067Q3|N|Y|N|2476260|2476502|2475911|2476185|N|N|N|N|N| +2476277|AAAAAAAAFPIMFCAA|2067-09-18|2012|8751|672|2067|0|9|18|3|2067|672|8751|Sunday|2067Q3|N|N|N|2476260|2476502|2475912|2476186|N|N|N|N|N| +2476278|AAAAAAAAGPIMFCAA|2067-09-19|2012|8751|672|2067|1|9|19|3|2067|672|8751|Monday|2067Q3|N|N|N|2476260|2476502|2475913|2476187|N|N|N|N|N| +2476279|AAAAAAAAHPIMFCAA|2067-09-20|2012|8752|672|2067|2|9|20|3|2067|672|8752|Tuesday|2067Q3|N|N|N|2476260|2476502|2475914|2476188|N|N|N|N|N| +2476280|AAAAAAAAIPIMFCAA|2067-09-21|2012|8752|672|2067|3|9|21|3|2067|672|8752|Wednesday|2067Q3|N|N|N|2476260|2476502|2475915|2476189|N|N|N|N|N| +2476281|AAAAAAAAJPIMFCAA|2067-09-22|2012|8752|672|2067|4|9|22|3|2067|672|8752|Thursday|2067Q3|N|N|N|2476260|2476502|2475916|2476190|N|N|N|N|N| +2476282|AAAAAAAAKPIMFCAA|2067-09-23|2012|8752|672|2067|5|9|23|3|2067|672|8752|Friday|2067Q3|N|Y|N|2476260|2476502|2475917|2476191|N|N|N|N|N| +2476283|AAAAAAAALPIMFCAA|2067-09-24|2012|8752|672|2067|6|9|24|3|2067|672|8752|Saturday|2067Q3|N|Y|N|2476260|2476502|2475918|2476192|N|N|N|N|N| +2476284|AAAAAAAAMPIMFCAA|2067-09-25|2012|8752|672|2067|0|9|25|3|2067|672|8752|Sunday|2067Q3|N|N|N|2476260|2476502|2475919|2476193|N|N|N|N|N| +2476285|AAAAAAAANPIMFCAA|2067-09-26|2012|8752|672|2067|1|9|26|3|2067|672|8752|Monday|2067Q3|N|N|N|2476260|2476502|2475920|2476194|N|N|N|N|N| +2476286|AAAAAAAAOPIMFCAA|2067-09-27|2012|8753|672|2067|2|9|27|3|2067|672|8753|Tuesday|2067Q3|N|N|N|2476260|2476502|2475921|2476195|N|N|N|N|N| +2476287|AAAAAAAAPPIMFCAA|2067-09-28|2012|8753|672|2067|3|9|28|3|2067|672|8753|Wednesday|2067Q3|N|N|N|2476260|2476502|2475922|2476196|N|N|N|N|N| +2476288|AAAAAAAAAAJMFCAA|2067-09-29|2012|8753|672|2067|4|9|29|3|2067|672|8753|Thursday|2067Q3|N|N|N|2476260|2476502|2475923|2476197|N|N|N|N|N| +2476289|AAAAAAAABAJMFCAA|2067-09-30|2012|8753|672|2067|5|9|30|3|2067|672|8753|Friday|2067Q3|N|Y|N|2476260|2476502|2475924|2476198|N|N|N|N|N| +2476290|AAAAAAAACAJMFCAA|2067-10-01|2013|8753|672|2067|6|10|1|3|2067|672|8753|Saturday|2067Q3|N|Y|N|2476290|2476562|2475925|2476198|N|N|N|N|N| +2476291|AAAAAAAADAJMFCAA|2067-10-02|2013|8753|672|2067|0|10|2|4|2067|672|8753|Sunday|2067Q4|N|N|N|2476290|2476562|2475926|2476199|N|N|N|N|N| +2476292|AAAAAAAAEAJMFCAA|2067-10-03|2013|8753|672|2067|1|10|3|4|2067|672|8753|Monday|2067Q4|N|N|N|2476290|2476562|2475927|2476200|N|N|N|N|N| +2476293|AAAAAAAAFAJMFCAA|2067-10-04|2013|8754|672|2067|2|10|4|4|2067|672|8754|Tuesday|2067Q4|N|N|N|2476290|2476562|2475928|2476201|N|N|N|N|N| +2476294|AAAAAAAAGAJMFCAA|2067-10-05|2013|8754|672|2067|3|10|5|4|2067|672|8754|Wednesday|2067Q4|N|N|N|2476290|2476562|2475929|2476202|N|N|N|N|N| +2476295|AAAAAAAAHAJMFCAA|2067-10-06|2013|8754|672|2067|4|10|6|4|2067|672|8754|Thursday|2067Q4|N|N|N|2476290|2476562|2475930|2476203|N|N|N|N|N| +2476296|AAAAAAAAIAJMFCAA|2067-10-07|2013|8754|672|2067|5|10|7|4|2067|672|8754|Friday|2067Q4|N|Y|N|2476290|2476562|2475931|2476204|N|N|N|N|N| +2476297|AAAAAAAAJAJMFCAA|2067-10-08|2013|8754|672|2067|6|10|8|4|2067|672|8754|Saturday|2067Q4|N|Y|N|2476290|2476562|2475932|2476205|N|N|N|N|N| +2476298|AAAAAAAAKAJMFCAA|2067-10-09|2013|8754|672|2067|0|10|9|4|2067|672|8754|Sunday|2067Q4|N|N|N|2476290|2476562|2475933|2476206|N|N|N|N|N| +2476299|AAAAAAAALAJMFCAA|2067-10-10|2013|8754|672|2067|1|10|10|4|2067|672|8754|Monday|2067Q4|N|N|N|2476290|2476562|2475934|2476207|N|N|N|N|N| +2476300|AAAAAAAAMAJMFCAA|2067-10-11|2013|8755|672|2067|2|10|11|4|2067|672|8755|Tuesday|2067Q4|N|N|N|2476290|2476562|2475935|2476208|N|N|N|N|N| +2476301|AAAAAAAANAJMFCAA|2067-10-12|2013|8755|672|2067|3|10|12|4|2067|672|8755|Wednesday|2067Q4|N|N|N|2476290|2476562|2475936|2476209|N|N|N|N|N| +2476302|AAAAAAAAOAJMFCAA|2067-10-13|2013|8755|672|2067|4|10|13|4|2067|672|8755|Thursday|2067Q4|N|N|N|2476290|2476562|2475937|2476210|N|N|N|N|N| +2476303|AAAAAAAAPAJMFCAA|2067-10-14|2013|8755|672|2067|5|10|14|4|2067|672|8755|Friday|2067Q4|N|Y|N|2476290|2476562|2475938|2476211|N|N|N|N|N| +2476304|AAAAAAAAABJMFCAA|2067-10-15|2013|8755|672|2067|6|10|15|4|2067|672|8755|Saturday|2067Q4|N|Y|N|2476290|2476562|2475939|2476212|N|N|N|N|N| +2476305|AAAAAAAABBJMFCAA|2067-10-16|2013|8755|672|2067|0|10|16|4|2067|672|8755|Sunday|2067Q4|N|N|N|2476290|2476562|2475940|2476213|N|N|N|N|N| +2476306|AAAAAAAACBJMFCAA|2067-10-17|2013|8755|672|2067|1|10|17|4|2067|672|8755|Monday|2067Q4|N|N|N|2476290|2476562|2475941|2476214|N|N|N|N|N| +2476307|AAAAAAAADBJMFCAA|2067-10-18|2013|8756|672|2067|2|10|18|4|2067|672|8756|Tuesday|2067Q4|N|N|N|2476290|2476562|2475942|2476215|N|N|N|N|N| +2476308|AAAAAAAAEBJMFCAA|2067-10-19|2013|8756|672|2067|3|10|19|4|2067|672|8756|Wednesday|2067Q4|N|N|N|2476290|2476562|2475943|2476216|N|N|N|N|N| +2476309|AAAAAAAAFBJMFCAA|2067-10-20|2013|8756|672|2067|4|10|20|4|2067|672|8756|Thursday|2067Q4|N|N|N|2476290|2476562|2475944|2476217|N|N|N|N|N| +2476310|AAAAAAAAGBJMFCAA|2067-10-21|2013|8756|672|2067|5|10|21|4|2067|672|8756|Friday|2067Q4|N|Y|N|2476290|2476562|2475945|2476218|N|N|N|N|N| +2476311|AAAAAAAAHBJMFCAA|2067-10-22|2013|8756|672|2067|6|10|22|4|2067|672|8756|Saturday|2067Q4|N|Y|N|2476290|2476562|2475946|2476219|N|N|N|N|N| +2476312|AAAAAAAAIBJMFCAA|2067-10-23|2013|8756|672|2067|0|10|23|4|2067|672|8756|Sunday|2067Q4|N|N|N|2476290|2476562|2475947|2476220|N|N|N|N|N| +2476313|AAAAAAAAJBJMFCAA|2067-10-24|2013|8756|672|2067|1|10|24|4|2067|672|8756|Monday|2067Q4|N|N|N|2476290|2476562|2475948|2476221|N|N|N|N|N| +2476314|AAAAAAAAKBJMFCAA|2067-10-25|2013|8757|672|2067|2|10|25|4|2067|672|8757|Tuesday|2067Q4|N|N|N|2476290|2476562|2475949|2476222|N|N|N|N|N| +2476315|AAAAAAAALBJMFCAA|2067-10-26|2013|8757|672|2067|3|10|26|4|2067|672|8757|Wednesday|2067Q4|N|N|N|2476290|2476562|2475950|2476223|N|N|N|N|N| +2476316|AAAAAAAAMBJMFCAA|2067-10-27|2013|8757|672|2067|4|10|27|4|2067|672|8757|Thursday|2067Q4|N|N|N|2476290|2476562|2475951|2476224|N|N|N|N|N| +2476317|AAAAAAAANBJMFCAA|2067-10-28|2013|8757|672|2067|5|10|28|4|2067|672|8757|Friday|2067Q4|N|Y|N|2476290|2476562|2475952|2476225|N|N|N|N|N| +2476318|AAAAAAAAOBJMFCAA|2067-10-29|2013|8757|672|2067|6|10|29|4|2067|672|8757|Saturday|2067Q4|N|Y|N|2476290|2476562|2475953|2476226|N|N|N|N|N| +2476319|AAAAAAAAPBJMFCAA|2067-10-30|2013|8757|672|2067|0|10|30|4|2067|672|8757|Sunday|2067Q4|N|N|N|2476290|2476562|2475954|2476227|N|N|N|N|N| +2476320|AAAAAAAAACJMFCAA|2067-10-31|2013|8757|672|2067|1|10|31|4|2067|672|8757|Monday|2067Q4|N|N|N|2476290|2476562|2475955|2476228|N|N|N|N|N| +2476321|AAAAAAAABCJMFCAA|2067-11-01|2014|8758|672|2067|2|11|1|4|2067|672|8758|Tuesday|2067Q4|N|N|N|2476321|2476624|2475956|2476229|N|N|N|N|N| +2476322|AAAAAAAACCJMFCAA|2067-11-02|2014|8758|672|2067|3|11|2|4|2067|672|8758|Wednesday|2067Q4|N|N|N|2476321|2476624|2475957|2476230|N|N|N|N|N| +2476323|AAAAAAAADCJMFCAA|2067-11-03|2014|8758|672|2067|4|11|3|4|2067|672|8758|Thursday|2067Q4|N|N|N|2476321|2476624|2475958|2476231|N|N|N|N|N| +2476324|AAAAAAAAECJMFCAA|2067-11-04|2014|8758|672|2067|5|11|4|4|2067|672|8758|Friday|2067Q4|N|Y|N|2476321|2476624|2475959|2476232|N|N|N|N|N| +2476325|AAAAAAAAFCJMFCAA|2067-11-05|2014|8758|672|2067|6|11|5|4|2067|672|8758|Saturday|2067Q4|N|Y|N|2476321|2476624|2475960|2476233|N|N|N|N|N| +2476326|AAAAAAAAGCJMFCAA|2067-11-06|2014|8758|672|2067|0|11|6|4|2067|672|8758|Sunday|2067Q4|N|N|N|2476321|2476624|2475961|2476234|N|N|N|N|N| +2476327|AAAAAAAAHCJMFCAA|2067-11-07|2014|8758|672|2067|1|11|7|4|2067|672|8758|Monday|2067Q4|N|N|N|2476321|2476624|2475962|2476235|N|N|N|N|N| +2476328|AAAAAAAAICJMFCAA|2067-11-08|2014|8759|672|2067|2|11|8|4|2067|672|8759|Tuesday|2067Q4|N|N|N|2476321|2476624|2475963|2476236|N|N|N|N|N| +2476329|AAAAAAAAJCJMFCAA|2067-11-09|2014|8759|672|2067|3|11|9|4|2067|672|8759|Wednesday|2067Q4|N|N|N|2476321|2476624|2475964|2476237|N|N|N|N|N| +2476330|AAAAAAAAKCJMFCAA|2067-11-10|2014|8759|672|2067|4|11|10|4|2067|672|8759|Thursday|2067Q4|N|N|N|2476321|2476624|2475965|2476238|N|N|N|N|N| +2476331|AAAAAAAALCJMFCAA|2067-11-11|2014|8759|672|2067|5|11|11|4|2067|672|8759|Friday|2067Q4|N|Y|N|2476321|2476624|2475966|2476239|N|N|N|N|N| +2476332|AAAAAAAAMCJMFCAA|2067-11-12|2014|8759|672|2067|6|11|12|4|2067|672|8759|Saturday|2067Q4|N|Y|N|2476321|2476624|2475967|2476240|N|N|N|N|N| +2476333|AAAAAAAANCJMFCAA|2067-11-13|2014|8759|672|2067|0|11|13|4|2067|672|8759|Sunday|2067Q4|N|N|N|2476321|2476624|2475968|2476241|N|N|N|N|N| +2476334|AAAAAAAAOCJMFCAA|2067-11-14|2014|8759|672|2067|1|11|14|4|2067|672|8759|Monday|2067Q4|N|N|N|2476321|2476624|2475969|2476242|N|N|N|N|N| +2476335|AAAAAAAAPCJMFCAA|2067-11-15|2014|8760|672|2067|2|11|15|4|2067|672|8760|Tuesday|2067Q4|N|N|N|2476321|2476624|2475970|2476243|N|N|N|N|N| +2476336|AAAAAAAAADJMFCAA|2067-11-16|2014|8760|672|2067|3|11|16|4|2067|672|8760|Wednesday|2067Q4|N|N|N|2476321|2476624|2475971|2476244|N|N|N|N|N| +2476337|AAAAAAAABDJMFCAA|2067-11-17|2014|8760|672|2067|4|11|17|4|2067|672|8760|Thursday|2067Q4|N|N|N|2476321|2476624|2475972|2476245|N|N|N|N|N| +2476338|AAAAAAAACDJMFCAA|2067-11-18|2014|8760|672|2067|5|11|18|4|2067|672|8760|Friday|2067Q4|N|Y|N|2476321|2476624|2475973|2476246|N|N|N|N|N| +2476339|AAAAAAAADDJMFCAA|2067-11-19|2014|8760|672|2067|6|11|19|4|2067|672|8760|Saturday|2067Q4|N|Y|N|2476321|2476624|2475974|2476247|N|N|N|N|N| +2476340|AAAAAAAAEDJMFCAA|2067-11-20|2014|8760|672|2067|0|11|20|4|2067|672|8760|Sunday|2067Q4|N|N|N|2476321|2476624|2475975|2476248|N|N|N|N|N| +2476341|AAAAAAAAFDJMFCAA|2067-11-21|2014|8760|672|2067|1|11|21|4|2067|672|8760|Monday|2067Q4|N|N|N|2476321|2476624|2475976|2476249|N|N|N|N|N| +2476342|AAAAAAAAGDJMFCAA|2067-11-22|2014|8761|672|2067|2|11|22|4|2067|672|8761|Tuesday|2067Q4|N|N|N|2476321|2476624|2475977|2476250|N|N|N|N|N| +2476343|AAAAAAAAHDJMFCAA|2067-11-23|2014|8761|672|2067|3|11|23|4|2067|672|8761|Wednesday|2067Q4|N|N|N|2476321|2476624|2475978|2476251|N|N|N|N|N| +2476344|AAAAAAAAIDJMFCAA|2067-11-24|2014|8761|672|2067|4|11|24|4|2067|672|8761|Thursday|2067Q4|N|N|N|2476321|2476624|2475979|2476252|N|N|N|N|N| +2476345|AAAAAAAAJDJMFCAA|2067-11-25|2014|8761|672|2067|5|11|25|4|2067|672|8761|Friday|2067Q4|N|Y|N|2476321|2476624|2475980|2476253|N|N|N|N|N| +2476346|AAAAAAAAKDJMFCAA|2067-11-26|2014|8761|672|2067|6|11|26|4|2067|672|8761|Saturday|2067Q4|N|Y|N|2476321|2476624|2475981|2476254|N|N|N|N|N| +2476347|AAAAAAAALDJMFCAA|2067-11-27|2014|8761|672|2067|0|11|27|4|2067|672|8761|Sunday|2067Q4|N|N|N|2476321|2476624|2475982|2476255|N|N|N|N|N| +2476348|AAAAAAAAMDJMFCAA|2067-11-28|2014|8761|672|2067|1|11|28|4|2067|672|8761|Monday|2067Q4|N|N|N|2476321|2476624|2475983|2476256|N|N|N|N|N| +2476349|AAAAAAAANDJMFCAA|2067-11-29|2014|8762|672|2067|2|11|29|4|2067|672|8762|Tuesday|2067Q4|N|N|N|2476321|2476624|2475984|2476257|N|N|N|N|N| +2476350|AAAAAAAAODJMFCAA|2067-11-30|2014|8762|672|2067|3|11|30|4|2067|672|8762|Wednesday|2067Q4|N|N|N|2476321|2476624|2475985|2476258|N|N|N|N|N| +2476351|AAAAAAAAPDJMFCAA|2067-12-01|2015|8762|673|2067|4|12|1|4|2067|673|8762|Thursday|2067Q4|N|N|N|2476351|2476684|2475986|2476259|N|N|N|N|N| +2476352|AAAAAAAAAEJMFCAA|2067-12-02|2015|8762|673|2067|5|12|2|4|2067|673|8762|Friday|2067Q4|N|Y|N|2476351|2476684|2475987|2476260|N|N|N|N|N| +2476353|AAAAAAAABEJMFCAA|2067-12-03|2015|8762|673|2067|6|12|3|4|2067|673|8762|Saturday|2067Q4|N|Y|N|2476351|2476684|2475988|2476261|N|N|N|N|N| +2476354|AAAAAAAACEJMFCAA|2067-12-04|2015|8762|673|2067|0|12|4|4|2067|673|8762|Sunday|2067Q4|N|N|N|2476351|2476684|2475989|2476262|N|N|N|N|N| +2476355|AAAAAAAADEJMFCAA|2067-12-05|2015|8762|673|2067|1|12|5|4|2067|673|8762|Monday|2067Q4|N|N|N|2476351|2476684|2475990|2476263|N|N|N|N|N| +2476356|AAAAAAAAEEJMFCAA|2067-12-06|2015|8763|673|2067|2|12|6|4|2067|673|8763|Tuesday|2067Q4|N|N|N|2476351|2476684|2475991|2476264|N|N|N|N|N| +2476357|AAAAAAAAFEJMFCAA|2067-12-07|2015|8763|673|2067|3|12|7|4|2067|673|8763|Wednesday|2067Q4|N|N|N|2476351|2476684|2475992|2476265|N|N|N|N|N| +2476358|AAAAAAAAGEJMFCAA|2067-12-08|2015|8763|673|2067|4|12|8|4|2067|673|8763|Thursday|2067Q4|N|N|N|2476351|2476684|2475993|2476266|N|N|N|N|N| +2476359|AAAAAAAAHEJMFCAA|2067-12-09|2015|8763|673|2067|5|12|9|4|2067|673|8763|Friday|2067Q4|N|Y|N|2476351|2476684|2475994|2476267|N|N|N|N|N| +2476360|AAAAAAAAIEJMFCAA|2067-12-10|2015|8763|673|2067|6|12|10|4|2067|673|8763|Saturday|2067Q4|N|Y|N|2476351|2476684|2475995|2476268|N|N|N|N|N| +2476361|AAAAAAAAJEJMFCAA|2067-12-11|2015|8763|673|2067|0|12|11|4|2067|673|8763|Sunday|2067Q4|N|N|N|2476351|2476684|2475996|2476269|N|N|N|N|N| +2476362|AAAAAAAAKEJMFCAA|2067-12-12|2015|8763|673|2067|1|12|12|4|2067|673|8763|Monday|2067Q4|N|N|N|2476351|2476684|2475997|2476270|N|N|N|N|N| +2476363|AAAAAAAALEJMFCAA|2067-12-13|2015|8764|673|2067|2|12|13|4|2067|673|8764|Tuesday|2067Q4|N|N|N|2476351|2476684|2475998|2476271|N|N|N|N|N| +2476364|AAAAAAAAMEJMFCAA|2067-12-14|2015|8764|673|2067|3|12|14|4|2067|673|8764|Wednesday|2067Q4|N|N|N|2476351|2476684|2475999|2476272|N|N|N|N|N| +2476365|AAAAAAAANEJMFCAA|2067-12-15|2015|8764|673|2067|4|12|15|4|2067|673|8764|Thursday|2067Q4|N|N|N|2476351|2476684|2476000|2476273|N|N|N|N|N| +2476366|AAAAAAAAOEJMFCAA|2067-12-16|2015|8764|673|2067|5|12|16|4|2067|673|8764|Friday|2067Q4|N|Y|N|2476351|2476684|2476001|2476274|N|N|N|N|N| +2476367|AAAAAAAAPEJMFCAA|2067-12-17|2015|8764|673|2067|6|12|17|4|2067|673|8764|Saturday|2067Q4|N|Y|N|2476351|2476684|2476002|2476275|N|N|N|N|N| +2476368|AAAAAAAAAFJMFCAA|2067-12-18|2015|8764|673|2067|0|12|18|4|2067|673|8764|Sunday|2067Q4|N|N|N|2476351|2476684|2476003|2476276|N|N|N|N|N| +2476369|AAAAAAAABFJMFCAA|2067-12-19|2015|8764|673|2067|1|12|19|4|2067|673|8764|Monday|2067Q4|N|N|N|2476351|2476684|2476004|2476277|N|N|N|N|N| +2476370|AAAAAAAACFJMFCAA|2067-12-20|2015|8765|673|2067|2|12|20|4|2067|673|8765|Tuesday|2067Q4|N|N|N|2476351|2476684|2476005|2476278|N|N|N|N|N| +2476371|AAAAAAAADFJMFCAA|2067-12-21|2015|8765|673|2067|3|12|21|4|2067|673|8765|Wednesday|2067Q4|N|N|N|2476351|2476684|2476006|2476279|N|N|N|N|N| +2476372|AAAAAAAAEFJMFCAA|2067-12-22|2015|8765|673|2067|4|12|22|4|2067|673|8765|Thursday|2067Q4|N|N|N|2476351|2476684|2476007|2476280|N|N|N|N|N| +2476373|AAAAAAAAFFJMFCAA|2067-12-23|2015|8765|673|2067|5|12|23|4|2067|673|8765|Friday|2067Q4|N|Y|N|2476351|2476684|2476008|2476281|N|N|N|N|N| +2476374|AAAAAAAAGFJMFCAA|2067-12-24|2015|8765|673|2067|6|12|24|4|2067|673|8765|Saturday|2067Q4|N|Y|N|2476351|2476684|2476009|2476282|N|N|N|N|N| +2476375|AAAAAAAAHFJMFCAA|2067-12-25|2015|8765|673|2067|0|12|25|4|2067|673|8765|Sunday|2067Q4|N|N|N|2476351|2476684|2476010|2476283|N|N|N|N|N| +2476376|AAAAAAAAIFJMFCAA|2067-12-26|2015|8765|673|2067|1|12|26|4|2067|673|8765|Monday|2067Q4|Y|N|N|2476351|2476684|2476011|2476284|N|N|N|N|N| +2476377|AAAAAAAAJFJMFCAA|2067-12-27|2015|8766|673|2067|2|12|27|4|2067|673|8766|Tuesday|2067Q4|N|N|Y|2476351|2476684|2476012|2476285|N|N|N|N|N| +2476378|AAAAAAAAKFJMFCAA|2067-12-28|2015|8766|673|2067|3|12|28|4|2067|673|8766|Wednesday|2067Q4|N|N|N|2476351|2476684|2476013|2476286|N|N|N|N|N| +2476379|AAAAAAAALFJMFCAA|2067-12-29|2015|8766|673|2067|4|12|29|4|2067|673|8766|Thursday|2067Q4|N|N|N|2476351|2476684|2476014|2476287|N|N|N|N|N| +2476380|AAAAAAAAMFJMFCAA|2067-12-30|2015|8766|673|2067|5|12|30|4|2067|673|8766|Friday|2067Q4|N|Y|N|2476351|2476684|2476015|2476288|N|N|N|N|N| +2476381|AAAAAAAANFJMFCAA|2067-12-31|2015|8766|673|2067|6|12|31|4|2067|673|8766|Saturday|2067Q4|N|Y|N|2476351|2476684|2476016|2476289|N|N|N|N|N| +2476382|AAAAAAAAOFJMFCAA|2068-01-01|2016|8766|673|2068|0|1|1|1|2068|673|8766|Sunday|2068Q1|Y|N|N|2476382|2476381|2476017|2476290|N|N|N|N|N| +2476383|AAAAAAAAPFJMFCAA|2068-01-02|2016|8766|673|2068|1|1|2|1|2068|673|8766|Monday|2068Q1|N|N|Y|2476382|2476381|2476018|2476291|N|N|N|N|N| +2476384|AAAAAAAAAGJMFCAA|2068-01-03|2016|8767|673|2068|2|1|3|1|2068|673|8767|Tuesday|2068Q1|N|N|N|2476382|2476381|2476019|2476292|N|N|N|N|N| +2476385|AAAAAAAABGJMFCAA|2068-01-04|2016|8767|673|2068|3|1|4|1|2068|673|8767|Wednesday|2068Q1|N|N|N|2476382|2476381|2476020|2476293|N|N|N|N|N| +2476386|AAAAAAAACGJMFCAA|2068-01-05|2016|8767|673|2068|4|1|5|1|2068|673|8767|Thursday|2068Q1|N|N|N|2476382|2476381|2476021|2476294|N|N|N|N|N| +2476387|AAAAAAAADGJMFCAA|2068-01-06|2016|8767|673|2068|5|1|6|1|2068|673|8767|Friday|2068Q1|N|Y|N|2476382|2476381|2476022|2476295|N|N|N|N|N| +2476388|AAAAAAAAEGJMFCAA|2068-01-07|2016|8767|673|2068|6|1|7|1|2068|673|8767|Saturday|2068Q1|N|Y|N|2476382|2476381|2476023|2476296|N|N|N|N|N| +2476389|AAAAAAAAFGJMFCAA|2068-01-08|2016|8767|673|2068|0|1|8|1|2068|673|8767|Sunday|2068Q1|N|N|N|2476382|2476381|2476024|2476297|N|N|N|N|N| +2476390|AAAAAAAAGGJMFCAA|2068-01-09|2016|8767|673|2068|1|1|9|1|2068|673|8767|Monday|2068Q1|N|N|N|2476382|2476381|2476025|2476298|N|N|N|N|N| +2476391|AAAAAAAAHGJMFCAA|2068-01-10|2016|8768|673|2068|2|1|10|1|2068|673|8768|Tuesday|2068Q1|N|N|N|2476382|2476381|2476026|2476299|N|N|N|N|N| +2476392|AAAAAAAAIGJMFCAA|2068-01-11|2016|8768|673|2068|3|1|11|1|2068|673|8768|Wednesday|2068Q1|N|N|N|2476382|2476381|2476027|2476300|N|N|N|N|N| +2476393|AAAAAAAAJGJMFCAA|2068-01-12|2016|8768|673|2068|4|1|12|1|2068|673|8768|Thursday|2068Q1|N|N|N|2476382|2476381|2476028|2476301|N|N|N|N|N| +2476394|AAAAAAAAKGJMFCAA|2068-01-13|2016|8768|673|2068|5|1|13|1|2068|673|8768|Friday|2068Q1|N|Y|N|2476382|2476381|2476029|2476302|N|N|N|N|N| +2476395|AAAAAAAALGJMFCAA|2068-01-14|2016|8768|673|2068|6|1|14|1|2068|673|8768|Saturday|2068Q1|N|Y|N|2476382|2476381|2476030|2476303|N|N|N|N|N| +2476396|AAAAAAAAMGJMFCAA|2068-01-15|2016|8768|673|2068|0|1|15|1|2068|673|8768|Sunday|2068Q1|N|N|N|2476382|2476381|2476031|2476304|N|N|N|N|N| +2476397|AAAAAAAANGJMFCAA|2068-01-16|2016|8768|673|2068|1|1|16|1|2068|673|8768|Monday|2068Q1|N|N|N|2476382|2476381|2476032|2476305|N|N|N|N|N| +2476398|AAAAAAAAOGJMFCAA|2068-01-17|2016|8769|673|2068|2|1|17|1|2068|673|8769|Tuesday|2068Q1|N|N|N|2476382|2476381|2476033|2476306|N|N|N|N|N| +2476399|AAAAAAAAPGJMFCAA|2068-01-18|2016|8769|673|2068|3|1|18|1|2068|673|8769|Wednesday|2068Q1|N|N|N|2476382|2476381|2476034|2476307|N|N|N|N|N| +2476400|AAAAAAAAAHJMFCAA|2068-01-19|2016|8769|673|2068|4|1|19|1|2068|673|8769|Thursday|2068Q1|N|N|N|2476382|2476381|2476035|2476308|N|N|N|N|N| +2476401|AAAAAAAABHJMFCAA|2068-01-20|2016|8769|673|2068|5|1|20|1|2068|673|8769|Friday|2068Q1|N|Y|N|2476382|2476381|2476036|2476309|N|N|N|N|N| +2476402|AAAAAAAACHJMFCAA|2068-01-21|2016|8769|673|2068|6|1|21|1|2068|673|8769|Saturday|2068Q1|N|Y|N|2476382|2476381|2476037|2476310|N|N|N|N|N| +2476403|AAAAAAAADHJMFCAA|2068-01-22|2016|8769|673|2068|0|1|22|1|2068|673|8769|Sunday|2068Q1|N|N|N|2476382|2476381|2476038|2476311|N|N|N|N|N| +2476404|AAAAAAAAEHJMFCAA|2068-01-23|2016|8769|673|2068|1|1|23|1|2068|673|8769|Monday|2068Q1|N|N|N|2476382|2476381|2476039|2476312|N|N|N|N|N| +2476405|AAAAAAAAFHJMFCAA|2068-01-24|2016|8770|673|2068|2|1|24|1|2068|673|8770|Tuesday|2068Q1|N|N|N|2476382|2476381|2476040|2476313|N|N|N|N|N| +2476406|AAAAAAAAGHJMFCAA|2068-01-25|2016|8770|673|2068|3|1|25|1|2068|673|8770|Wednesday|2068Q1|N|N|N|2476382|2476381|2476041|2476314|N|N|N|N|N| +2476407|AAAAAAAAHHJMFCAA|2068-01-26|2016|8770|673|2068|4|1|26|1|2068|673|8770|Thursday|2068Q1|N|N|N|2476382|2476381|2476042|2476315|N|N|N|N|N| +2476408|AAAAAAAAIHJMFCAA|2068-01-27|2016|8770|673|2068|5|1|27|1|2068|673|8770|Friday|2068Q1|N|Y|N|2476382|2476381|2476043|2476316|N|N|N|N|N| +2476409|AAAAAAAAJHJMFCAA|2068-01-28|2016|8770|673|2068|6|1|28|1|2068|673|8770|Saturday|2068Q1|N|Y|N|2476382|2476381|2476044|2476317|N|N|N|N|N| +2476410|AAAAAAAAKHJMFCAA|2068-01-29|2016|8770|673|2068|0|1|29|1|2068|673|8770|Sunday|2068Q1|N|N|N|2476382|2476381|2476045|2476318|N|N|N|N|N| +2476411|AAAAAAAALHJMFCAA|2068-01-30|2016|8770|673|2068|1|1|30|1|2068|673|8770|Monday|2068Q1|N|N|N|2476382|2476381|2476046|2476319|N|N|N|N|N| +2476412|AAAAAAAAMHJMFCAA|2068-01-31|2016|8771|673|2068|2|1|31|1|2068|673|8771|Tuesday|2068Q1|N|N|N|2476382|2476381|2476047|2476320|N|N|N|N|N| +2476413|AAAAAAAANHJMFCAA|2068-02-01|2017|8771|673|2068|3|2|1|1|2068|673|8771|Wednesday|2068Q1|N|N|N|2476413|2476443|2476048|2476321|N|N|N|N|N| +2476414|AAAAAAAAOHJMFCAA|2068-02-02|2017|8771|673|2068|4|2|2|1|2068|673|8771|Thursday|2068Q1|N|N|N|2476413|2476443|2476049|2476322|N|N|N|N|N| +2476415|AAAAAAAAPHJMFCAA|2068-02-03|2017|8771|673|2068|5|2|3|1|2068|673|8771|Friday|2068Q1|N|Y|N|2476413|2476443|2476050|2476323|N|N|N|N|N| +2476416|AAAAAAAAAIJMFCAA|2068-02-04|2017|8771|673|2068|6|2|4|1|2068|673|8771|Saturday|2068Q1|N|Y|N|2476413|2476443|2476051|2476324|N|N|N|N|N| +2476417|AAAAAAAABIJMFCAA|2068-02-05|2017|8771|673|2068|0|2|5|1|2068|673|8771|Sunday|2068Q1|N|N|N|2476413|2476443|2476052|2476325|N|N|N|N|N| +2476418|AAAAAAAACIJMFCAA|2068-02-06|2017|8771|673|2068|1|2|6|1|2068|673|8771|Monday|2068Q1|N|N|N|2476413|2476443|2476053|2476326|N|N|N|N|N| +2476419|AAAAAAAADIJMFCAA|2068-02-07|2017|8772|673|2068|2|2|7|1|2068|673|8772|Tuesday|2068Q1|N|N|N|2476413|2476443|2476054|2476327|N|N|N|N|N| +2476420|AAAAAAAAEIJMFCAA|2068-02-08|2017|8772|673|2068|3|2|8|1|2068|673|8772|Wednesday|2068Q1|N|N|N|2476413|2476443|2476055|2476328|N|N|N|N|N| +2476421|AAAAAAAAFIJMFCAA|2068-02-09|2017|8772|673|2068|4|2|9|1|2068|673|8772|Thursday|2068Q1|N|N|N|2476413|2476443|2476056|2476329|N|N|N|N|N| +2476422|AAAAAAAAGIJMFCAA|2068-02-10|2017|8772|673|2068|5|2|10|1|2068|673|8772|Friday|2068Q1|N|Y|N|2476413|2476443|2476057|2476330|N|N|N|N|N| +2476423|AAAAAAAAHIJMFCAA|2068-02-11|2017|8772|673|2068|6|2|11|1|2068|673|8772|Saturday|2068Q1|N|Y|N|2476413|2476443|2476058|2476331|N|N|N|N|N| +2476424|AAAAAAAAIIJMFCAA|2068-02-12|2017|8772|673|2068|0|2|12|1|2068|673|8772|Sunday|2068Q1|N|N|N|2476413|2476443|2476059|2476332|N|N|N|N|N| +2476425|AAAAAAAAJIJMFCAA|2068-02-13|2017|8772|673|2068|1|2|13|1|2068|673|8772|Monday|2068Q1|N|N|N|2476413|2476443|2476060|2476333|N|N|N|N|N| +2476426|AAAAAAAAKIJMFCAA|2068-02-14|2017|8773|673|2068|2|2|14|1|2068|673|8773|Tuesday|2068Q1|N|N|N|2476413|2476443|2476061|2476334|N|N|N|N|N| +2476427|AAAAAAAALIJMFCAA|2068-02-15|2017|8773|673|2068|3|2|15|1|2068|673|8773|Wednesday|2068Q1|N|N|N|2476413|2476443|2476062|2476335|N|N|N|N|N| +2476428|AAAAAAAAMIJMFCAA|2068-02-16|2017|8773|673|2068|4|2|16|1|2068|673|8773|Thursday|2068Q1|N|N|N|2476413|2476443|2476063|2476336|N|N|N|N|N| +2476429|AAAAAAAANIJMFCAA|2068-02-17|2017|8773|673|2068|5|2|17|1|2068|673|8773|Friday|2068Q1|N|Y|N|2476413|2476443|2476064|2476337|N|N|N|N|N| +2476430|AAAAAAAAOIJMFCAA|2068-02-18|2017|8773|673|2068|6|2|18|1|2068|673|8773|Saturday|2068Q1|N|Y|N|2476413|2476443|2476065|2476338|N|N|N|N|N| +2476431|AAAAAAAAPIJMFCAA|2068-02-19|2017|8773|673|2068|0|2|19|1|2068|673|8773|Sunday|2068Q1|N|N|N|2476413|2476443|2476066|2476339|N|N|N|N|N| +2476432|AAAAAAAAAJJMFCAA|2068-02-20|2017|8773|673|2068|1|2|20|1|2068|673|8773|Monday|2068Q1|N|N|N|2476413|2476443|2476067|2476340|N|N|N|N|N| +2476433|AAAAAAAABJJMFCAA|2068-02-21|2017|8774|673|2068|2|2|21|1|2068|673|8774|Tuesday|2068Q1|N|N|N|2476413|2476443|2476068|2476341|N|N|N|N|N| +2476434|AAAAAAAACJJMFCAA|2068-02-22|2017|8774|673|2068|3|2|22|1|2068|673|8774|Wednesday|2068Q1|N|N|N|2476413|2476443|2476069|2476342|N|N|N|N|N| +2476435|AAAAAAAADJJMFCAA|2068-02-23|2017|8774|673|2068|4|2|23|1|2068|673|8774|Thursday|2068Q1|N|N|N|2476413|2476443|2476070|2476343|N|N|N|N|N| +2476436|AAAAAAAAEJJMFCAA|2068-02-24|2017|8774|673|2068|5|2|24|1|2068|673|8774|Friday|2068Q1|N|Y|N|2476413|2476443|2476071|2476344|N|N|N|N|N| +2476437|AAAAAAAAFJJMFCAA|2068-02-25|2017|8774|673|2068|6|2|25|1|2068|673|8774|Saturday|2068Q1|N|Y|N|2476413|2476443|2476072|2476345|N|N|N|N|N| +2476438|AAAAAAAAGJJMFCAA|2068-02-26|2017|8774|673|2068|0|2|26|1|2068|673|8774|Sunday|2068Q1|N|N|N|2476413|2476443|2476073|2476346|N|N|N|N|N| +2476439|AAAAAAAAHJJMFCAA|2068-02-27|2017|8774|673|2068|1|2|27|1|2068|673|8774|Monday|2068Q1|N|N|N|2476413|2476443|2476074|2476347|N|N|N|N|N| +2476440|AAAAAAAAIJJMFCAA|2068-02-28|2017|8775|673|2068|2|2|28|1|2068|673|8775|Tuesday|2068Q1|N|N|N|2476413|2476443|2476075|2476348|N|N|N|N|N| +2476441|AAAAAAAAJJJMFCAA|2068-02-29|2017|8775|673|2068|3|2|29|1|2068|673|8775|Wednesday|2068Q1|N|N|N|2476413|2476443|2476075|2476349|N|N|N|N|N| +2476442|AAAAAAAAKJJMFCAA|2068-03-01|2018|8775|674|2068|4|3|1|1|2068|674|8775|Thursday|2068Q1|N|N|N|2476442|2476501|2476076|2476350|N|N|N|N|N| +2476443|AAAAAAAALJJMFCAA|2068-03-02|2018|8775|674|2068|5|3|2|1|2068|674|8775|Friday|2068Q1|N|Y|N|2476442|2476501|2476077|2476351|N|N|N|N|N| +2476444|AAAAAAAAMJJMFCAA|2068-03-03|2018|8775|674|2068|6|3|3|1|2068|674|8775|Saturday|2068Q1|N|Y|N|2476442|2476501|2476078|2476352|N|N|N|N|N| +2476445|AAAAAAAANJJMFCAA|2068-03-04|2018|8775|674|2068|0|3|4|1|2068|674|8775|Sunday|2068Q1|N|N|N|2476442|2476501|2476079|2476353|N|N|N|N|N| +2476446|AAAAAAAAOJJMFCAA|2068-03-05|2018|8775|674|2068|1|3|5|1|2068|674|8775|Monday|2068Q1|N|N|N|2476442|2476501|2476080|2476354|N|N|N|N|N| +2476447|AAAAAAAAPJJMFCAA|2068-03-06|2018|8776|674|2068|2|3|6|1|2068|674|8776|Tuesday|2068Q1|N|N|N|2476442|2476501|2476081|2476355|N|N|N|N|N| +2476448|AAAAAAAAAKJMFCAA|2068-03-07|2018|8776|674|2068|3|3|7|1|2068|674|8776|Wednesday|2068Q1|N|N|N|2476442|2476501|2476082|2476356|N|N|N|N|N| +2476449|AAAAAAAABKJMFCAA|2068-03-08|2018|8776|674|2068|4|3|8|1|2068|674|8776|Thursday|2068Q1|N|N|N|2476442|2476501|2476083|2476357|N|N|N|N|N| +2476450|AAAAAAAACKJMFCAA|2068-03-09|2018|8776|674|2068|5|3|9|1|2068|674|8776|Friday|2068Q1|N|Y|N|2476442|2476501|2476084|2476358|N|N|N|N|N| +2476451|AAAAAAAADKJMFCAA|2068-03-10|2018|8776|674|2068|6|3|10|1|2068|674|8776|Saturday|2068Q1|N|Y|N|2476442|2476501|2476085|2476359|N|N|N|N|N| +2476452|AAAAAAAAEKJMFCAA|2068-03-11|2018|8776|674|2068|0|3|11|1|2068|674|8776|Sunday|2068Q1|N|N|N|2476442|2476501|2476086|2476360|N|N|N|N|N| +2476453|AAAAAAAAFKJMFCAA|2068-03-12|2018|8776|674|2068|1|3|12|1|2068|674|8776|Monday|2068Q1|N|N|N|2476442|2476501|2476087|2476361|N|N|N|N|N| +2476454|AAAAAAAAGKJMFCAA|2068-03-13|2018|8777|674|2068|2|3|13|1|2068|674|8777|Tuesday|2068Q1|N|N|N|2476442|2476501|2476088|2476362|N|N|N|N|N| +2476455|AAAAAAAAHKJMFCAA|2068-03-14|2018|8777|674|2068|3|3|14|1|2068|674|8777|Wednesday|2068Q1|N|N|N|2476442|2476501|2476089|2476363|N|N|N|N|N| +2476456|AAAAAAAAIKJMFCAA|2068-03-15|2018|8777|674|2068|4|3|15|1|2068|674|8777|Thursday|2068Q1|N|N|N|2476442|2476501|2476090|2476364|N|N|N|N|N| +2476457|AAAAAAAAJKJMFCAA|2068-03-16|2018|8777|674|2068|5|3|16|1|2068|674|8777|Friday|2068Q1|N|Y|N|2476442|2476501|2476091|2476365|N|N|N|N|N| +2476458|AAAAAAAAKKJMFCAA|2068-03-17|2018|8777|674|2068|6|3|17|1|2068|674|8777|Saturday|2068Q1|N|Y|N|2476442|2476501|2476092|2476366|N|N|N|N|N| +2476459|AAAAAAAALKJMFCAA|2068-03-18|2018|8777|674|2068|0|3|18|1|2068|674|8777|Sunday|2068Q1|N|N|N|2476442|2476501|2476093|2476367|N|N|N|N|N| +2476460|AAAAAAAAMKJMFCAA|2068-03-19|2018|8777|674|2068|1|3|19|1|2068|674|8777|Monday|2068Q1|N|N|N|2476442|2476501|2476094|2476368|N|N|N|N|N| +2476461|AAAAAAAANKJMFCAA|2068-03-20|2018|8778|674|2068|2|3|20|1|2068|674|8778|Tuesday|2068Q1|N|N|N|2476442|2476501|2476095|2476369|N|N|N|N|N| +2476462|AAAAAAAAOKJMFCAA|2068-03-21|2018|8778|674|2068|3|3|21|1|2068|674|8778|Wednesday|2068Q1|N|N|N|2476442|2476501|2476096|2476370|N|N|N|N|N| +2476463|AAAAAAAAPKJMFCAA|2068-03-22|2018|8778|674|2068|4|3|22|1|2068|674|8778|Thursday|2068Q1|N|N|N|2476442|2476501|2476097|2476371|N|N|N|N|N| +2476464|AAAAAAAAALJMFCAA|2068-03-23|2018|8778|674|2068|5|3|23|1|2068|674|8778|Friday|2068Q1|N|Y|N|2476442|2476501|2476098|2476372|N|N|N|N|N| +2476465|AAAAAAAABLJMFCAA|2068-03-24|2018|8778|674|2068|6|3|24|1|2068|674|8778|Saturday|2068Q1|N|Y|N|2476442|2476501|2476099|2476373|N|N|N|N|N| +2476466|AAAAAAAACLJMFCAA|2068-03-25|2018|8778|674|2068|0|3|25|1|2068|674|8778|Sunday|2068Q1|N|N|N|2476442|2476501|2476100|2476374|N|N|N|N|N| +2476467|AAAAAAAADLJMFCAA|2068-03-26|2018|8778|674|2068|1|3|26|1|2068|674|8778|Monday|2068Q1|N|N|N|2476442|2476501|2476101|2476375|N|N|N|N|N| +2476468|AAAAAAAAELJMFCAA|2068-03-27|2018|8779|674|2068|2|3|27|1|2068|674|8779|Tuesday|2068Q1|N|N|N|2476442|2476501|2476102|2476376|N|N|N|N|N| +2476469|AAAAAAAAFLJMFCAA|2068-03-28|2018|8779|674|2068|3|3|28|1|2068|674|8779|Wednesday|2068Q1|N|N|N|2476442|2476501|2476103|2476377|N|N|N|N|N| +2476470|AAAAAAAAGLJMFCAA|2068-03-29|2018|8779|674|2068|4|3|29|1|2068|674|8779|Thursday|2068Q1|N|N|N|2476442|2476501|2476104|2476378|N|N|N|N|N| +2476471|AAAAAAAAHLJMFCAA|2068-03-30|2018|8779|674|2068|5|3|30|1|2068|674|8779|Friday|2068Q1|N|Y|N|2476442|2476501|2476105|2476379|N|N|N|N|N| +2476472|AAAAAAAAILJMFCAA|2068-03-31|2018|8779|674|2068|6|3|31|1|2068|674|8779|Saturday|2068Q1|N|Y|N|2476442|2476501|2476106|2476380|N|N|N|N|N| +2476473|AAAAAAAAJLJMFCAA|2068-04-01|2019|8779|674|2068|0|4|1|2|2068|674|8779|Sunday|2068Q2|N|N|N|2476473|2476563|2476107|2476382|N|N|N|N|N| +2476474|AAAAAAAAKLJMFCAA|2068-04-02|2019|8779|674|2068|1|4|2|2|2068|674|8779|Monday|2068Q2|N|N|N|2476473|2476563|2476108|2476383|N|N|N|N|N| +2476475|AAAAAAAALLJMFCAA|2068-04-03|2019|8780|674|2068|2|4|3|2|2068|674|8780|Tuesday|2068Q2|N|N|N|2476473|2476563|2476109|2476384|N|N|N|N|N| +2476476|AAAAAAAAMLJMFCAA|2068-04-04|2019|8780|674|2068|3|4|4|2|2068|674|8780|Wednesday|2068Q2|N|N|N|2476473|2476563|2476110|2476385|N|N|N|N|N| +2476477|AAAAAAAANLJMFCAA|2068-04-05|2019|8780|674|2068|4|4|5|2|2068|674|8780|Thursday|2068Q2|N|N|N|2476473|2476563|2476111|2476386|N|N|N|N|N| +2476478|AAAAAAAAOLJMFCAA|2068-04-06|2019|8780|674|2068|5|4|6|2|2068|674|8780|Friday|2068Q2|N|Y|N|2476473|2476563|2476112|2476387|N|N|N|N|N| +2476479|AAAAAAAAPLJMFCAA|2068-04-07|2019|8780|674|2068|6|4|7|2|2068|674|8780|Saturday|2068Q2|N|Y|N|2476473|2476563|2476113|2476388|N|N|N|N|N| +2476480|AAAAAAAAAMJMFCAA|2068-04-08|2019|8780|674|2068|0|4|8|2|2068|674|8780|Sunday|2068Q2|N|N|N|2476473|2476563|2476114|2476389|N|N|N|N|N| +2476481|AAAAAAAABMJMFCAA|2068-04-09|2019|8780|674|2068|1|4|9|2|2068|674|8780|Monday|2068Q2|N|N|N|2476473|2476563|2476115|2476390|N|N|N|N|N| +2476482|AAAAAAAACMJMFCAA|2068-04-10|2019|8781|674|2068|2|4|10|2|2068|674|8781|Tuesday|2068Q2|N|N|N|2476473|2476563|2476116|2476391|N|N|N|N|N| +2476483|AAAAAAAADMJMFCAA|2068-04-11|2019|8781|674|2068|3|4|11|2|2068|674|8781|Wednesday|2068Q2|N|N|N|2476473|2476563|2476117|2476392|N|N|N|N|N| +2476484|AAAAAAAAEMJMFCAA|2068-04-12|2019|8781|674|2068|4|4|12|2|2068|674|8781|Thursday|2068Q2|N|N|N|2476473|2476563|2476118|2476393|N|N|N|N|N| +2476485|AAAAAAAAFMJMFCAA|2068-04-13|2019|8781|674|2068|5|4|13|2|2068|674|8781|Friday|2068Q2|N|Y|N|2476473|2476563|2476119|2476394|N|N|N|N|N| +2476486|AAAAAAAAGMJMFCAA|2068-04-14|2019|8781|674|2068|6|4|14|2|2068|674|8781|Saturday|2068Q2|N|Y|N|2476473|2476563|2476120|2476395|N|N|N|N|N| +2476487|AAAAAAAAHMJMFCAA|2068-04-15|2019|8781|674|2068|0|4|15|2|2068|674|8781|Sunday|2068Q2|N|N|N|2476473|2476563|2476121|2476396|N|N|N|N|N| +2476488|AAAAAAAAIMJMFCAA|2068-04-16|2019|8781|674|2068|1|4|16|2|2068|674|8781|Monday|2068Q2|N|N|N|2476473|2476563|2476122|2476397|N|N|N|N|N| +2476489|AAAAAAAAJMJMFCAA|2068-04-17|2019|8782|674|2068|2|4|17|2|2068|674|8782|Tuesday|2068Q2|N|N|N|2476473|2476563|2476123|2476398|N|N|N|N|N| +2476490|AAAAAAAAKMJMFCAA|2068-04-18|2019|8782|674|2068|3|4|18|2|2068|674|8782|Wednesday|2068Q2|N|N|N|2476473|2476563|2476124|2476399|N|N|N|N|N| +2476491|AAAAAAAALMJMFCAA|2068-04-19|2019|8782|674|2068|4|4|19|2|2068|674|8782|Thursday|2068Q2|N|N|N|2476473|2476563|2476125|2476400|N|N|N|N|N| +2476492|AAAAAAAAMMJMFCAA|2068-04-20|2019|8782|674|2068|5|4|20|2|2068|674|8782|Friday|2068Q2|N|Y|N|2476473|2476563|2476126|2476401|N|N|N|N|N| +2476493|AAAAAAAANMJMFCAA|2068-04-21|2019|8782|674|2068|6|4|21|2|2068|674|8782|Saturday|2068Q2|N|Y|N|2476473|2476563|2476127|2476402|N|N|N|N|N| +2476494|AAAAAAAAOMJMFCAA|2068-04-22|2019|8782|674|2068|0|4|22|2|2068|674|8782|Sunday|2068Q2|N|N|N|2476473|2476563|2476128|2476403|N|N|N|N|N| +2476495|AAAAAAAAPMJMFCAA|2068-04-23|2019|8782|674|2068|1|4|23|2|2068|674|8782|Monday|2068Q2|N|N|N|2476473|2476563|2476129|2476404|N|N|N|N|N| +2476496|AAAAAAAAANJMFCAA|2068-04-24|2019|8783|674|2068|2|4|24|2|2068|674|8783|Tuesday|2068Q2|N|N|N|2476473|2476563|2476130|2476405|N|N|N|N|N| +2476497|AAAAAAAABNJMFCAA|2068-04-25|2019|8783|674|2068|3|4|25|2|2068|674|8783|Wednesday|2068Q2|N|N|N|2476473|2476563|2476131|2476406|N|N|N|N|N| +2476498|AAAAAAAACNJMFCAA|2068-04-26|2019|8783|674|2068|4|4|26|2|2068|674|8783|Thursday|2068Q2|N|N|N|2476473|2476563|2476132|2476407|N|N|N|N|N| +2476499|AAAAAAAADNJMFCAA|2068-04-27|2019|8783|674|2068|5|4|27|2|2068|674|8783|Friday|2068Q2|N|Y|N|2476473|2476563|2476133|2476408|N|N|N|N|N| +2476500|AAAAAAAAENJMFCAA|2068-04-28|2019|8783|674|2068|6|4|28|2|2068|674|8783|Saturday|2068Q2|N|Y|N|2476473|2476563|2476134|2476409|N|N|N|N|N| +2476501|AAAAAAAAFNJMFCAA|2068-04-29|2019|8783|674|2068|0|4|29|2|2068|674|8783|Sunday|2068Q2|N|N|N|2476473|2476563|2476135|2476410|N|N|N|N|N| +2476502|AAAAAAAAGNJMFCAA|2068-04-30|2019|8783|674|2068|1|4|30|2|2068|674|8783|Monday|2068Q2|N|N|N|2476473|2476563|2476136|2476411|N|N|N|N|N| +2476503|AAAAAAAAHNJMFCAA|2068-05-01|2020|8784|674|2068|2|5|1|2|2068|674|8784|Tuesday|2068Q2|N|N|N|2476503|2476623|2476137|2476412|N|N|N|N|N| +2476504|AAAAAAAAINJMFCAA|2068-05-02|2020|8784|674|2068|3|5|2|2|2068|674|8784|Wednesday|2068Q2|N|N|N|2476503|2476623|2476138|2476413|N|N|N|N|N| +2476505|AAAAAAAAJNJMFCAA|2068-05-03|2020|8784|674|2068|4|5|3|2|2068|674|8784|Thursday|2068Q2|N|N|N|2476503|2476623|2476139|2476414|N|N|N|N|N| +2476506|AAAAAAAAKNJMFCAA|2068-05-04|2020|8784|674|2068|5|5|4|2|2068|674|8784|Friday|2068Q2|N|Y|N|2476503|2476623|2476140|2476415|N|N|N|N|N| +2476507|AAAAAAAALNJMFCAA|2068-05-05|2020|8784|674|2068|6|5|5|2|2068|674|8784|Saturday|2068Q2|N|Y|N|2476503|2476623|2476141|2476416|N|N|N|N|N| +2476508|AAAAAAAAMNJMFCAA|2068-05-06|2020|8784|674|2068|0|5|6|2|2068|674|8784|Sunday|2068Q2|N|N|N|2476503|2476623|2476142|2476417|N|N|N|N|N| +2476509|AAAAAAAANNJMFCAA|2068-05-07|2020|8784|674|2068|1|5|7|2|2068|674|8784|Monday|2068Q2|N|N|N|2476503|2476623|2476143|2476418|N|N|N|N|N| +2476510|AAAAAAAAONJMFCAA|2068-05-08|2020|8785|674|2068|2|5|8|2|2068|674|8785|Tuesday|2068Q2|N|N|N|2476503|2476623|2476144|2476419|N|N|N|N|N| +2476511|AAAAAAAAPNJMFCAA|2068-05-09|2020|8785|674|2068|3|5|9|2|2068|674|8785|Wednesday|2068Q2|N|N|N|2476503|2476623|2476145|2476420|N|N|N|N|N| +2476512|AAAAAAAAAOJMFCAA|2068-05-10|2020|8785|674|2068|4|5|10|2|2068|674|8785|Thursday|2068Q2|N|N|N|2476503|2476623|2476146|2476421|N|N|N|N|N| +2476513|AAAAAAAABOJMFCAA|2068-05-11|2020|8785|674|2068|5|5|11|2|2068|674|8785|Friday|2068Q2|N|Y|N|2476503|2476623|2476147|2476422|N|N|N|N|N| +2476514|AAAAAAAACOJMFCAA|2068-05-12|2020|8785|674|2068|6|5|12|2|2068|674|8785|Saturday|2068Q2|N|Y|N|2476503|2476623|2476148|2476423|N|N|N|N|N| +2476515|AAAAAAAADOJMFCAA|2068-05-13|2020|8785|674|2068|0|5|13|2|2068|674|8785|Sunday|2068Q2|N|N|N|2476503|2476623|2476149|2476424|N|N|N|N|N| +2476516|AAAAAAAAEOJMFCAA|2068-05-14|2020|8785|674|2068|1|5|14|2|2068|674|8785|Monday|2068Q2|N|N|N|2476503|2476623|2476150|2476425|N|N|N|N|N| +2476517|AAAAAAAAFOJMFCAA|2068-05-15|2020|8786|674|2068|2|5|15|2|2068|674|8786|Tuesday|2068Q2|N|N|N|2476503|2476623|2476151|2476426|N|N|N|N|N| +2476518|AAAAAAAAGOJMFCAA|2068-05-16|2020|8786|674|2068|3|5|16|2|2068|674|8786|Wednesday|2068Q2|N|N|N|2476503|2476623|2476152|2476427|N|N|N|N|N| +2476519|AAAAAAAAHOJMFCAA|2068-05-17|2020|8786|674|2068|4|5|17|2|2068|674|8786|Thursday|2068Q2|N|N|N|2476503|2476623|2476153|2476428|N|N|N|N|N| +2476520|AAAAAAAAIOJMFCAA|2068-05-18|2020|8786|674|2068|5|5|18|2|2068|674|8786|Friday|2068Q2|N|Y|N|2476503|2476623|2476154|2476429|N|N|N|N|N| +2476521|AAAAAAAAJOJMFCAA|2068-05-19|2020|8786|674|2068|6|5|19|2|2068|674|8786|Saturday|2068Q2|N|Y|N|2476503|2476623|2476155|2476430|N|N|N|N|N| +2476522|AAAAAAAAKOJMFCAA|2068-05-20|2020|8786|674|2068|0|5|20|2|2068|674|8786|Sunday|2068Q2|N|N|N|2476503|2476623|2476156|2476431|N|N|N|N|N| +2476523|AAAAAAAALOJMFCAA|2068-05-21|2020|8786|674|2068|1|5|21|2|2068|674|8786|Monday|2068Q2|N|N|N|2476503|2476623|2476157|2476432|N|N|N|N|N| +2476524|AAAAAAAAMOJMFCAA|2068-05-22|2020|8787|674|2068|2|5|22|2|2068|674|8787|Tuesday|2068Q2|N|N|N|2476503|2476623|2476158|2476433|N|N|N|N|N| +2476525|AAAAAAAANOJMFCAA|2068-05-23|2020|8787|674|2068|3|5|23|2|2068|674|8787|Wednesday|2068Q2|N|N|N|2476503|2476623|2476159|2476434|N|N|N|N|N| +2476526|AAAAAAAAOOJMFCAA|2068-05-24|2020|8787|674|2068|4|5|24|2|2068|674|8787|Thursday|2068Q2|N|N|N|2476503|2476623|2476160|2476435|N|N|N|N|N| +2476527|AAAAAAAAPOJMFCAA|2068-05-25|2020|8787|674|2068|5|5|25|2|2068|674|8787|Friday|2068Q2|N|Y|N|2476503|2476623|2476161|2476436|N|N|N|N|N| +2476528|AAAAAAAAAPJMFCAA|2068-05-26|2020|8787|674|2068|6|5|26|2|2068|674|8787|Saturday|2068Q2|N|Y|N|2476503|2476623|2476162|2476437|N|N|N|N|N| +2476529|AAAAAAAABPJMFCAA|2068-05-27|2020|8787|674|2068|0|5|27|2|2068|674|8787|Sunday|2068Q2|N|N|N|2476503|2476623|2476163|2476438|N|N|N|N|N| +2476530|AAAAAAAACPJMFCAA|2068-05-28|2020|8787|674|2068|1|5|28|2|2068|674|8787|Monday|2068Q2|N|N|N|2476503|2476623|2476164|2476439|N|N|N|N|N| +2476531|AAAAAAAADPJMFCAA|2068-05-29|2020|8788|674|2068|2|5|29|2|2068|674|8788|Tuesday|2068Q2|N|N|N|2476503|2476623|2476165|2476440|N|N|N|N|N| +2476532|AAAAAAAAEPJMFCAA|2068-05-30|2020|8788|674|2068|3|5|30|2|2068|674|8788|Wednesday|2068Q2|N|N|N|2476503|2476623|2476166|2476441|N|N|N|N|N| +2476533|AAAAAAAAFPJMFCAA|2068-05-31|2020|8788|674|2068|4|5|31|2|2068|674|8788|Thursday|2068Q2|N|N|N|2476503|2476623|2476167|2476442|N|N|N|N|N| +2476534|AAAAAAAAGPJMFCAA|2068-06-01|2021|8788|675|2068|5|6|1|2|2068|675|8788|Friday|2068Q2|N|Y|N|2476534|2476685|2476168|2476443|N|N|N|N|N| +2476535|AAAAAAAAHPJMFCAA|2068-06-02|2021|8788|675|2068|6|6|2|2|2068|675|8788|Saturday|2068Q2|N|Y|N|2476534|2476685|2476169|2476444|N|N|N|N|N| +2476536|AAAAAAAAIPJMFCAA|2068-06-03|2021|8788|675|2068|0|6|3|2|2068|675|8788|Sunday|2068Q2|N|N|N|2476534|2476685|2476170|2476445|N|N|N|N|N| +2476537|AAAAAAAAJPJMFCAA|2068-06-04|2021|8788|675|2068|1|6|4|2|2068|675|8788|Monday|2068Q2|N|N|N|2476534|2476685|2476171|2476446|N|N|N|N|N| +2476538|AAAAAAAAKPJMFCAA|2068-06-05|2021|8789|675|2068|2|6|5|2|2068|675|8789|Tuesday|2068Q2|N|N|N|2476534|2476685|2476172|2476447|N|N|N|N|N| +2476539|AAAAAAAALPJMFCAA|2068-06-06|2021|8789|675|2068|3|6|6|2|2068|675|8789|Wednesday|2068Q2|N|N|N|2476534|2476685|2476173|2476448|N|N|N|N|N| +2476540|AAAAAAAAMPJMFCAA|2068-06-07|2021|8789|675|2068|4|6|7|2|2068|675|8789|Thursday|2068Q2|N|N|N|2476534|2476685|2476174|2476449|N|N|N|N|N| +2476541|AAAAAAAANPJMFCAA|2068-06-08|2021|8789|675|2068|5|6|8|2|2068|675|8789|Friday|2068Q2|N|Y|N|2476534|2476685|2476175|2476450|N|N|N|N|N| +2476542|AAAAAAAAOPJMFCAA|2068-06-09|2021|8789|675|2068|6|6|9|2|2068|675|8789|Saturday|2068Q2|N|Y|N|2476534|2476685|2476176|2476451|N|N|N|N|N| +2476543|AAAAAAAAPPJMFCAA|2068-06-10|2021|8789|675|2068|0|6|10|2|2068|675|8789|Sunday|2068Q2|N|N|N|2476534|2476685|2476177|2476452|N|N|N|N|N| +2476544|AAAAAAAAAAKMFCAA|2068-06-11|2021|8789|675|2068|1|6|11|2|2068|675|8789|Monday|2068Q2|N|N|N|2476534|2476685|2476178|2476453|N|N|N|N|N| +2476545|AAAAAAAABAKMFCAA|2068-06-12|2021|8790|675|2068|2|6|12|2|2068|675|8790|Tuesday|2068Q2|N|N|N|2476534|2476685|2476179|2476454|N|N|N|N|N| +2476546|AAAAAAAACAKMFCAA|2068-06-13|2021|8790|675|2068|3|6|13|2|2068|675|8790|Wednesday|2068Q2|N|N|N|2476534|2476685|2476180|2476455|N|N|N|N|N| +2476547|AAAAAAAADAKMFCAA|2068-06-14|2021|8790|675|2068|4|6|14|2|2068|675|8790|Thursday|2068Q2|N|N|N|2476534|2476685|2476181|2476456|N|N|N|N|N| +2476548|AAAAAAAAEAKMFCAA|2068-06-15|2021|8790|675|2068|5|6|15|2|2068|675|8790|Friday|2068Q2|N|Y|N|2476534|2476685|2476182|2476457|N|N|N|N|N| +2476549|AAAAAAAAFAKMFCAA|2068-06-16|2021|8790|675|2068|6|6|16|2|2068|675|8790|Saturday|2068Q2|N|Y|N|2476534|2476685|2476183|2476458|N|N|N|N|N| +2476550|AAAAAAAAGAKMFCAA|2068-06-17|2021|8790|675|2068|0|6|17|2|2068|675|8790|Sunday|2068Q2|N|N|N|2476534|2476685|2476184|2476459|N|N|N|N|N| +2476551|AAAAAAAAHAKMFCAA|2068-06-18|2021|8790|675|2068|1|6|18|2|2068|675|8790|Monday|2068Q2|N|N|N|2476534|2476685|2476185|2476460|N|N|N|N|N| +2476552|AAAAAAAAIAKMFCAA|2068-06-19|2021|8791|675|2068|2|6|19|2|2068|675|8791|Tuesday|2068Q2|N|N|N|2476534|2476685|2476186|2476461|N|N|N|N|N| +2476553|AAAAAAAAJAKMFCAA|2068-06-20|2021|8791|675|2068|3|6|20|2|2068|675|8791|Wednesday|2068Q2|N|N|N|2476534|2476685|2476187|2476462|N|N|N|N|N| +2476554|AAAAAAAAKAKMFCAA|2068-06-21|2021|8791|675|2068|4|6|21|2|2068|675|8791|Thursday|2068Q2|N|N|N|2476534|2476685|2476188|2476463|N|N|N|N|N| +2476555|AAAAAAAALAKMFCAA|2068-06-22|2021|8791|675|2068|5|6|22|2|2068|675|8791|Friday|2068Q2|N|Y|N|2476534|2476685|2476189|2476464|N|N|N|N|N| +2476556|AAAAAAAAMAKMFCAA|2068-06-23|2021|8791|675|2068|6|6|23|2|2068|675|8791|Saturday|2068Q2|N|Y|N|2476534|2476685|2476190|2476465|N|N|N|N|N| +2476557|AAAAAAAANAKMFCAA|2068-06-24|2021|8791|675|2068|0|6|24|2|2068|675|8791|Sunday|2068Q2|N|N|N|2476534|2476685|2476191|2476466|N|N|N|N|N| +2476558|AAAAAAAAOAKMFCAA|2068-06-25|2021|8791|675|2068|1|6|25|2|2068|675|8791|Monday|2068Q2|N|N|N|2476534|2476685|2476192|2476467|N|N|N|N|N| +2476559|AAAAAAAAPAKMFCAA|2068-06-26|2021|8792|675|2068|2|6|26|2|2068|675|8792|Tuesday|2068Q2|N|N|N|2476534|2476685|2476193|2476468|N|N|N|N|N| +2476560|AAAAAAAAABKMFCAA|2068-06-27|2021|8792|675|2068|3|6|27|2|2068|675|8792|Wednesday|2068Q2|N|N|N|2476534|2476685|2476194|2476469|N|N|N|N|N| +2476561|AAAAAAAABBKMFCAA|2068-06-28|2021|8792|675|2068|4|6|28|2|2068|675|8792|Thursday|2068Q2|N|N|N|2476534|2476685|2476195|2476470|N|N|N|N|N| +2476562|AAAAAAAACBKMFCAA|2068-06-29|2021|8792|675|2068|5|6|29|2|2068|675|8792|Friday|2068Q2|N|Y|N|2476534|2476685|2476196|2476471|N|N|N|N|N| +2476563|AAAAAAAADBKMFCAA|2068-06-30|2021|8792|675|2068|6|6|30|2|2068|675|8792|Saturday|2068Q2|N|Y|N|2476534|2476685|2476197|2476472|N|N|N|N|N| +2476564|AAAAAAAAEBKMFCAA|2068-07-01|2022|8792|675|2068|0|7|1|3|2068|675|8792|Sunday|2068Q3|N|N|N|2476564|2476745|2476198|2476473|N|N|N|N|N| +2476565|AAAAAAAAFBKMFCAA|2068-07-02|2022|8792|675|2068|1|7|2|3|2068|675|8792|Monday|2068Q3|N|N|N|2476564|2476745|2476199|2476474|N|N|N|N|N| +2476566|AAAAAAAAGBKMFCAA|2068-07-03|2022|8793|675|2068|2|7|3|3|2068|675|8793|Tuesday|2068Q3|N|N|N|2476564|2476745|2476200|2476475|N|N|N|N|N| +2476567|AAAAAAAAHBKMFCAA|2068-07-04|2022|8793|675|2068|3|7|4|3|2068|675|8793|Wednesday|2068Q3|Y|N|N|2476564|2476745|2476201|2476476|N|N|N|N|N| +2476568|AAAAAAAAIBKMFCAA|2068-07-05|2022|8793|675|2068|4|7|5|3|2068|675|8793|Thursday|2068Q3|N|N|Y|2476564|2476745|2476202|2476477|N|N|N|N|N| +2476569|AAAAAAAAJBKMFCAA|2068-07-06|2022|8793|675|2068|5|7|6|3|2068|675|8793|Friday|2068Q3|N|Y|N|2476564|2476745|2476203|2476478|N|N|N|N|N| +2476570|AAAAAAAAKBKMFCAA|2068-07-07|2022|8793|675|2068|6|7|7|3|2068|675|8793|Saturday|2068Q3|N|Y|N|2476564|2476745|2476204|2476479|N|N|N|N|N| +2476571|AAAAAAAALBKMFCAA|2068-07-08|2022|8793|675|2068|0|7|8|3|2068|675|8793|Sunday|2068Q3|N|N|N|2476564|2476745|2476205|2476480|N|N|N|N|N| +2476572|AAAAAAAAMBKMFCAA|2068-07-09|2022|8793|675|2068|1|7|9|3|2068|675|8793|Monday|2068Q3|N|N|N|2476564|2476745|2476206|2476481|N|N|N|N|N| +2476573|AAAAAAAANBKMFCAA|2068-07-10|2022|8794|675|2068|2|7|10|3|2068|675|8794|Tuesday|2068Q3|N|N|N|2476564|2476745|2476207|2476482|N|N|N|N|N| +2476574|AAAAAAAAOBKMFCAA|2068-07-11|2022|8794|675|2068|3|7|11|3|2068|675|8794|Wednesday|2068Q3|N|N|N|2476564|2476745|2476208|2476483|N|N|N|N|N| +2476575|AAAAAAAAPBKMFCAA|2068-07-12|2022|8794|675|2068|4|7|12|3|2068|675|8794|Thursday|2068Q3|N|N|N|2476564|2476745|2476209|2476484|N|N|N|N|N| +2476576|AAAAAAAAACKMFCAA|2068-07-13|2022|8794|675|2068|5|7|13|3|2068|675|8794|Friday|2068Q3|N|Y|N|2476564|2476745|2476210|2476485|N|N|N|N|N| +2476577|AAAAAAAABCKMFCAA|2068-07-14|2022|8794|675|2068|6|7|14|3|2068|675|8794|Saturday|2068Q3|N|Y|N|2476564|2476745|2476211|2476486|N|N|N|N|N| +2476578|AAAAAAAACCKMFCAA|2068-07-15|2022|8794|675|2068|0|7|15|3|2068|675|8794|Sunday|2068Q3|N|N|N|2476564|2476745|2476212|2476487|N|N|N|N|N| +2476579|AAAAAAAADCKMFCAA|2068-07-16|2022|8794|675|2068|1|7|16|3|2068|675|8794|Monday|2068Q3|N|N|N|2476564|2476745|2476213|2476488|N|N|N|N|N| +2476580|AAAAAAAAECKMFCAA|2068-07-17|2022|8795|675|2068|2|7|17|3|2068|675|8795|Tuesday|2068Q3|N|N|N|2476564|2476745|2476214|2476489|N|N|N|N|N| +2476581|AAAAAAAAFCKMFCAA|2068-07-18|2022|8795|675|2068|3|7|18|3|2068|675|8795|Wednesday|2068Q3|N|N|N|2476564|2476745|2476215|2476490|N|N|N|N|N| +2476582|AAAAAAAAGCKMFCAA|2068-07-19|2022|8795|675|2068|4|7|19|3|2068|675|8795|Thursday|2068Q3|N|N|N|2476564|2476745|2476216|2476491|N|N|N|N|N| +2476583|AAAAAAAAHCKMFCAA|2068-07-20|2022|8795|675|2068|5|7|20|3|2068|675|8795|Friday|2068Q3|N|Y|N|2476564|2476745|2476217|2476492|N|N|N|N|N| +2476584|AAAAAAAAICKMFCAA|2068-07-21|2022|8795|675|2068|6|7|21|3|2068|675|8795|Saturday|2068Q3|N|Y|N|2476564|2476745|2476218|2476493|N|N|N|N|N| +2476585|AAAAAAAAJCKMFCAA|2068-07-22|2022|8795|675|2068|0|7|22|3|2068|675|8795|Sunday|2068Q3|N|N|N|2476564|2476745|2476219|2476494|N|N|N|N|N| +2476586|AAAAAAAAKCKMFCAA|2068-07-23|2022|8795|675|2068|1|7|23|3|2068|675|8795|Monday|2068Q3|N|N|N|2476564|2476745|2476220|2476495|N|N|N|N|N| +2476587|AAAAAAAALCKMFCAA|2068-07-24|2022|8796|675|2068|2|7|24|3|2068|675|8796|Tuesday|2068Q3|N|N|N|2476564|2476745|2476221|2476496|N|N|N|N|N| +2476588|AAAAAAAAMCKMFCAA|2068-07-25|2022|8796|675|2068|3|7|25|3|2068|675|8796|Wednesday|2068Q3|N|N|N|2476564|2476745|2476222|2476497|N|N|N|N|N| +2476589|AAAAAAAANCKMFCAA|2068-07-26|2022|8796|675|2068|4|7|26|3|2068|675|8796|Thursday|2068Q3|N|N|N|2476564|2476745|2476223|2476498|N|N|N|N|N| +2476590|AAAAAAAAOCKMFCAA|2068-07-27|2022|8796|675|2068|5|7|27|3|2068|675|8796|Friday|2068Q3|N|Y|N|2476564|2476745|2476224|2476499|N|N|N|N|N| +2476591|AAAAAAAAPCKMFCAA|2068-07-28|2022|8796|675|2068|6|7|28|3|2068|675|8796|Saturday|2068Q3|N|Y|N|2476564|2476745|2476225|2476500|N|N|N|N|N| +2476592|AAAAAAAAADKMFCAA|2068-07-29|2022|8796|675|2068|0|7|29|3|2068|675|8796|Sunday|2068Q3|N|N|N|2476564|2476745|2476226|2476501|N|N|N|N|N| +2476593|AAAAAAAABDKMFCAA|2068-07-30|2022|8796|675|2068|1|7|30|3|2068|675|8796|Monday|2068Q3|N|N|N|2476564|2476745|2476227|2476502|N|N|N|N|N| +2476594|AAAAAAAACDKMFCAA|2068-07-31|2022|8797|675|2068|2|7|31|3|2068|675|8797|Tuesday|2068Q3|N|N|N|2476564|2476745|2476228|2476503|N|N|N|N|N| +2476595|AAAAAAAADDKMFCAA|2068-08-01|2023|8797|675|2068|3|8|1|3|2068|675|8797|Wednesday|2068Q3|N|N|N|2476595|2476807|2476229|2476504|N|N|N|N|N| +2476596|AAAAAAAAEDKMFCAA|2068-08-02|2023|8797|675|2068|4|8|2|3|2068|675|8797|Thursday|2068Q3|N|N|N|2476595|2476807|2476230|2476505|N|N|N|N|N| +2476597|AAAAAAAAFDKMFCAA|2068-08-03|2023|8797|675|2068|5|8|3|3|2068|675|8797|Friday|2068Q3|N|Y|N|2476595|2476807|2476231|2476506|N|N|N|N|N| +2476598|AAAAAAAAGDKMFCAA|2068-08-04|2023|8797|675|2068|6|8|4|3|2068|675|8797|Saturday|2068Q3|N|Y|N|2476595|2476807|2476232|2476507|N|N|N|N|N| +2476599|AAAAAAAAHDKMFCAA|2068-08-05|2023|8797|675|2068|0|8|5|3|2068|675|8797|Sunday|2068Q3|N|N|N|2476595|2476807|2476233|2476508|N|N|N|N|N| +2476600|AAAAAAAAIDKMFCAA|2068-08-06|2023|8797|675|2068|1|8|6|3|2068|675|8797|Monday|2068Q3|N|N|N|2476595|2476807|2476234|2476509|N|N|N|N|N| +2476601|AAAAAAAAJDKMFCAA|2068-08-07|2023|8798|675|2068|2|8|7|3|2068|675|8798|Tuesday|2068Q3|N|N|N|2476595|2476807|2476235|2476510|N|N|N|N|N| +2476602|AAAAAAAAKDKMFCAA|2068-08-08|2023|8798|675|2068|3|8|8|3|2068|675|8798|Wednesday|2068Q3|N|N|N|2476595|2476807|2476236|2476511|N|N|N|N|N| +2476603|AAAAAAAALDKMFCAA|2068-08-09|2023|8798|675|2068|4|8|9|3|2068|675|8798|Thursday|2068Q3|N|N|N|2476595|2476807|2476237|2476512|N|N|N|N|N| +2476604|AAAAAAAAMDKMFCAA|2068-08-10|2023|8798|675|2068|5|8|10|3|2068|675|8798|Friday|2068Q3|N|Y|N|2476595|2476807|2476238|2476513|N|N|N|N|N| +2476605|AAAAAAAANDKMFCAA|2068-08-11|2023|8798|675|2068|6|8|11|3|2068|675|8798|Saturday|2068Q3|N|Y|N|2476595|2476807|2476239|2476514|N|N|N|N|N| +2476606|AAAAAAAAODKMFCAA|2068-08-12|2023|8798|675|2068|0|8|12|3|2068|675|8798|Sunday|2068Q3|N|N|N|2476595|2476807|2476240|2476515|N|N|N|N|N| +2476607|AAAAAAAAPDKMFCAA|2068-08-13|2023|8798|675|2068|1|8|13|3|2068|675|8798|Monday|2068Q3|N|N|N|2476595|2476807|2476241|2476516|N|N|N|N|N| +2476608|AAAAAAAAAEKMFCAA|2068-08-14|2023|8799|675|2068|2|8|14|3|2068|675|8799|Tuesday|2068Q3|N|N|N|2476595|2476807|2476242|2476517|N|N|N|N|N| +2476609|AAAAAAAABEKMFCAA|2068-08-15|2023|8799|675|2068|3|8|15|3|2068|675|8799|Wednesday|2068Q3|N|N|N|2476595|2476807|2476243|2476518|N|N|N|N|N| +2476610|AAAAAAAACEKMFCAA|2068-08-16|2023|8799|675|2068|4|8|16|3|2068|675|8799|Thursday|2068Q3|N|N|N|2476595|2476807|2476244|2476519|N|N|N|N|N| +2476611|AAAAAAAADEKMFCAA|2068-08-17|2023|8799|675|2068|5|8|17|3|2068|675|8799|Friday|2068Q3|N|Y|N|2476595|2476807|2476245|2476520|N|N|N|N|N| +2476612|AAAAAAAAEEKMFCAA|2068-08-18|2023|8799|675|2068|6|8|18|3|2068|675|8799|Saturday|2068Q3|N|Y|N|2476595|2476807|2476246|2476521|N|N|N|N|N| +2476613|AAAAAAAAFEKMFCAA|2068-08-19|2023|8799|675|2068|0|8|19|3|2068|675|8799|Sunday|2068Q3|N|N|N|2476595|2476807|2476247|2476522|N|N|N|N|N| +2476614|AAAAAAAAGEKMFCAA|2068-08-20|2023|8799|675|2068|1|8|20|3|2068|675|8799|Monday|2068Q3|N|N|N|2476595|2476807|2476248|2476523|N|N|N|N|N| +2476615|AAAAAAAAHEKMFCAA|2068-08-21|2023|8800|675|2068|2|8|21|3|2068|675|8800|Tuesday|2068Q3|N|N|N|2476595|2476807|2476249|2476524|N|N|N|N|N| +2476616|AAAAAAAAIEKMFCAA|2068-08-22|2023|8800|675|2068|3|8|22|3|2068|675|8800|Wednesday|2068Q3|N|N|N|2476595|2476807|2476250|2476525|N|N|N|N|N| +2476617|AAAAAAAAJEKMFCAA|2068-08-23|2023|8800|675|2068|4|8|23|3|2068|675|8800|Thursday|2068Q3|N|N|N|2476595|2476807|2476251|2476526|N|N|N|N|N| +2476618|AAAAAAAAKEKMFCAA|2068-08-24|2023|8800|675|2068|5|8|24|3|2068|675|8800|Friday|2068Q3|N|Y|N|2476595|2476807|2476252|2476527|N|N|N|N|N| +2476619|AAAAAAAALEKMFCAA|2068-08-25|2023|8800|675|2068|6|8|25|3|2068|675|8800|Saturday|2068Q3|N|Y|N|2476595|2476807|2476253|2476528|N|N|N|N|N| +2476620|AAAAAAAAMEKMFCAA|2068-08-26|2023|8800|675|2068|0|8|26|3|2068|675|8800|Sunday|2068Q3|N|N|N|2476595|2476807|2476254|2476529|N|N|N|N|N| +2476621|AAAAAAAANEKMFCAA|2068-08-27|2023|8800|675|2068|1|8|27|3|2068|675|8800|Monday|2068Q3|N|N|N|2476595|2476807|2476255|2476530|N|N|N|N|N| +2476622|AAAAAAAAOEKMFCAA|2068-08-28|2023|8801|675|2068|2|8|28|3|2068|675|8801|Tuesday|2068Q3|N|N|N|2476595|2476807|2476256|2476531|N|N|N|N|N| +2476623|AAAAAAAAPEKMFCAA|2068-08-29|2023|8801|675|2068|3|8|29|3|2068|675|8801|Wednesday|2068Q3|N|N|N|2476595|2476807|2476257|2476532|N|N|N|N|N| +2476624|AAAAAAAAAFKMFCAA|2068-08-30|2023|8801|675|2068|4|8|30|3|2068|675|8801|Thursday|2068Q3|N|N|N|2476595|2476807|2476258|2476533|N|N|N|N|N| +2476625|AAAAAAAABFKMFCAA|2068-08-31|2023|8801|675|2068|5|8|31|3|2068|675|8801|Friday|2068Q3|N|Y|N|2476595|2476807|2476259|2476534|N|N|N|N|N| +2476626|AAAAAAAACFKMFCAA|2068-09-01|2024|8801|676|2068|6|9|1|3|2068|676|8801|Saturday|2068Q3|N|Y|N|2476626|2476869|2476260|2476535|N|N|N|N|N| +2476627|AAAAAAAADFKMFCAA|2068-09-02|2024|8801|676|2068|0|9|2|3|2068|676|8801|Sunday|2068Q3|N|N|N|2476626|2476869|2476261|2476536|N|N|N|N|N| +2476628|AAAAAAAAEFKMFCAA|2068-09-03|2024|8801|676|2068|1|9|3|3|2068|676|8801|Monday|2068Q3|N|N|N|2476626|2476869|2476262|2476537|N|N|N|N|N| +2476629|AAAAAAAAFFKMFCAA|2068-09-04|2024|8802|676|2068|2|9|4|3|2068|676|8802|Tuesday|2068Q3|N|N|N|2476626|2476869|2476263|2476538|N|N|N|N|N| +2476630|AAAAAAAAGFKMFCAA|2068-09-05|2024|8802|676|2068|3|9|5|3|2068|676|8802|Wednesday|2068Q3|N|N|N|2476626|2476869|2476264|2476539|N|N|N|N|N| +2476631|AAAAAAAAHFKMFCAA|2068-09-06|2024|8802|676|2068|4|9|6|3|2068|676|8802|Thursday|2068Q3|N|N|N|2476626|2476869|2476265|2476540|N|N|N|N|N| +2476632|AAAAAAAAIFKMFCAA|2068-09-07|2024|8802|676|2068|5|9|7|3|2068|676|8802|Friday|2068Q3|N|Y|N|2476626|2476869|2476266|2476541|N|N|N|N|N| +2476633|AAAAAAAAJFKMFCAA|2068-09-08|2024|8802|676|2068|6|9|8|3|2068|676|8802|Saturday|2068Q3|N|Y|N|2476626|2476869|2476267|2476542|N|N|N|N|N| +2476634|AAAAAAAAKFKMFCAA|2068-09-09|2024|8802|676|2068|0|9|9|3|2068|676|8802|Sunday|2068Q3|N|N|N|2476626|2476869|2476268|2476543|N|N|N|N|N| +2476635|AAAAAAAALFKMFCAA|2068-09-10|2024|8802|676|2068|1|9|10|3|2068|676|8802|Monday|2068Q3|N|N|N|2476626|2476869|2476269|2476544|N|N|N|N|N| +2476636|AAAAAAAAMFKMFCAA|2068-09-11|2024|8803|676|2068|2|9|11|3|2068|676|8803|Tuesday|2068Q3|N|N|N|2476626|2476869|2476270|2476545|N|N|N|N|N| +2476637|AAAAAAAANFKMFCAA|2068-09-12|2024|8803|676|2068|3|9|12|3|2068|676|8803|Wednesday|2068Q3|N|N|N|2476626|2476869|2476271|2476546|N|N|N|N|N| +2476638|AAAAAAAAOFKMFCAA|2068-09-13|2024|8803|676|2068|4|9|13|3|2068|676|8803|Thursday|2068Q3|N|N|N|2476626|2476869|2476272|2476547|N|N|N|N|N| +2476639|AAAAAAAAPFKMFCAA|2068-09-14|2024|8803|676|2068|5|9|14|3|2068|676|8803|Friday|2068Q3|N|Y|N|2476626|2476869|2476273|2476548|N|N|N|N|N| +2476640|AAAAAAAAAGKMFCAA|2068-09-15|2024|8803|676|2068|6|9|15|3|2068|676|8803|Saturday|2068Q3|N|Y|N|2476626|2476869|2476274|2476549|N|N|N|N|N| +2476641|AAAAAAAABGKMFCAA|2068-09-16|2024|8803|676|2068|0|9|16|3|2068|676|8803|Sunday|2068Q3|N|N|N|2476626|2476869|2476275|2476550|N|N|N|N|N| +2476642|AAAAAAAACGKMFCAA|2068-09-17|2024|8803|676|2068|1|9|17|3|2068|676|8803|Monday|2068Q3|N|N|N|2476626|2476869|2476276|2476551|N|N|N|N|N| +2476643|AAAAAAAADGKMFCAA|2068-09-18|2024|8804|676|2068|2|9|18|3|2068|676|8804|Tuesday|2068Q3|N|N|N|2476626|2476869|2476277|2476552|N|N|N|N|N| +2476644|AAAAAAAAEGKMFCAA|2068-09-19|2024|8804|676|2068|3|9|19|3|2068|676|8804|Wednesday|2068Q3|N|N|N|2476626|2476869|2476278|2476553|N|N|N|N|N| +2476645|AAAAAAAAFGKMFCAA|2068-09-20|2024|8804|676|2068|4|9|20|3|2068|676|8804|Thursday|2068Q3|N|N|N|2476626|2476869|2476279|2476554|N|N|N|N|N| +2476646|AAAAAAAAGGKMFCAA|2068-09-21|2024|8804|676|2068|5|9|21|3|2068|676|8804|Friday|2068Q3|N|Y|N|2476626|2476869|2476280|2476555|N|N|N|N|N| +2476647|AAAAAAAAHGKMFCAA|2068-09-22|2024|8804|676|2068|6|9|22|3|2068|676|8804|Saturday|2068Q3|N|Y|N|2476626|2476869|2476281|2476556|N|N|N|N|N| +2476648|AAAAAAAAIGKMFCAA|2068-09-23|2024|8804|676|2068|0|9|23|3|2068|676|8804|Sunday|2068Q3|N|N|N|2476626|2476869|2476282|2476557|N|N|N|N|N| +2476649|AAAAAAAAJGKMFCAA|2068-09-24|2024|8804|676|2068|1|9|24|3|2068|676|8804|Monday|2068Q3|N|N|N|2476626|2476869|2476283|2476558|N|N|N|N|N| +2476650|AAAAAAAAKGKMFCAA|2068-09-25|2024|8805|676|2068|2|9|25|3|2068|676|8805|Tuesday|2068Q3|N|N|N|2476626|2476869|2476284|2476559|N|N|N|N|N| +2476651|AAAAAAAALGKMFCAA|2068-09-26|2024|8805|676|2068|3|9|26|3|2068|676|8805|Wednesday|2068Q3|N|N|N|2476626|2476869|2476285|2476560|N|N|N|N|N| +2476652|AAAAAAAAMGKMFCAA|2068-09-27|2024|8805|676|2068|4|9|27|3|2068|676|8805|Thursday|2068Q3|N|N|N|2476626|2476869|2476286|2476561|N|N|N|N|N| +2476653|AAAAAAAANGKMFCAA|2068-09-28|2024|8805|676|2068|5|9|28|3|2068|676|8805|Friday|2068Q3|N|Y|N|2476626|2476869|2476287|2476562|N|N|N|N|N| +2476654|AAAAAAAAOGKMFCAA|2068-09-29|2024|8805|676|2068|6|9|29|3|2068|676|8805|Saturday|2068Q3|N|Y|N|2476626|2476869|2476288|2476563|N|N|N|N|N| +2476655|AAAAAAAAPGKMFCAA|2068-09-30|2024|8805|676|2068|0|9|30|3|2068|676|8805|Sunday|2068Q3|N|N|N|2476626|2476869|2476289|2476564|N|N|N|N|N| +2476656|AAAAAAAAAHKMFCAA|2068-10-01|2025|8805|676|2068|1|10|1|4|2068|676|8805|Monday|2068Q4|N|N|N|2476656|2476929|2476290|2476564|N|N|N|N|N| +2476657|AAAAAAAABHKMFCAA|2068-10-02|2025|8806|676|2068|2|10|2|4|2068|676|8806|Tuesday|2068Q4|N|N|N|2476656|2476929|2476291|2476565|N|N|N|N|N| +2476658|AAAAAAAACHKMFCAA|2068-10-03|2025|8806|676|2068|3|10|3|4|2068|676|8806|Wednesday|2068Q4|N|N|N|2476656|2476929|2476292|2476566|N|N|N|N|N| +2476659|AAAAAAAADHKMFCAA|2068-10-04|2025|8806|676|2068|4|10|4|4|2068|676|8806|Thursday|2068Q4|N|N|N|2476656|2476929|2476293|2476567|N|N|N|N|N| +2476660|AAAAAAAAEHKMFCAA|2068-10-05|2025|8806|676|2068|5|10|5|4|2068|676|8806|Friday|2068Q4|N|Y|N|2476656|2476929|2476294|2476568|N|N|N|N|N| +2476661|AAAAAAAAFHKMFCAA|2068-10-06|2025|8806|676|2068|6|10|6|4|2068|676|8806|Saturday|2068Q4|N|Y|N|2476656|2476929|2476295|2476569|N|N|N|N|N| +2476662|AAAAAAAAGHKMFCAA|2068-10-07|2025|8806|676|2068|0|10|7|4|2068|676|8806|Sunday|2068Q4|N|N|N|2476656|2476929|2476296|2476570|N|N|N|N|N| +2476663|AAAAAAAAHHKMFCAA|2068-10-08|2025|8806|676|2068|1|10|8|4|2068|676|8806|Monday|2068Q4|N|N|N|2476656|2476929|2476297|2476571|N|N|N|N|N| +2476664|AAAAAAAAIHKMFCAA|2068-10-09|2025|8807|676|2068|2|10|9|4|2068|676|8807|Tuesday|2068Q4|N|N|N|2476656|2476929|2476298|2476572|N|N|N|N|N| +2476665|AAAAAAAAJHKMFCAA|2068-10-10|2025|8807|676|2068|3|10|10|4|2068|676|8807|Wednesday|2068Q4|N|N|N|2476656|2476929|2476299|2476573|N|N|N|N|N| +2476666|AAAAAAAAKHKMFCAA|2068-10-11|2025|8807|676|2068|4|10|11|4|2068|676|8807|Thursday|2068Q4|N|N|N|2476656|2476929|2476300|2476574|N|N|N|N|N| +2476667|AAAAAAAALHKMFCAA|2068-10-12|2025|8807|676|2068|5|10|12|4|2068|676|8807|Friday|2068Q4|N|Y|N|2476656|2476929|2476301|2476575|N|N|N|N|N| +2476668|AAAAAAAAMHKMFCAA|2068-10-13|2025|8807|676|2068|6|10|13|4|2068|676|8807|Saturday|2068Q4|N|Y|N|2476656|2476929|2476302|2476576|N|N|N|N|N| +2476669|AAAAAAAANHKMFCAA|2068-10-14|2025|8807|676|2068|0|10|14|4|2068|676|8807|Sunday|2068Q4|N|N|N|2476656|2476929|2476303|2476577|N|N|N|N|N| +2476670|AAAAAAAAOHKMFCAA|2068-10-15|2025|8807|676|2068|1|10|15|4|2068|676|8807|Monday|2068Q4|N|N|N|2476656|2476929|2476304|2476578|N|N|N|N|N| +2476671|AAAAAAAAPHKMFCAA|2068-10-16|2025|8808|676|2068|2|10|16|4|2068|676|8808|Tuesday|2068Q4|N|N|N|2476656|2476929|2476305|2476579|N|N|N|N|N| +2476672|AAAAAAAAAIKMFCAA|2068-10-17|2025|8808|676|2068|3|10|17|4|2068|676|8808|Wednesday|2068Q4|N|N|N|2476656|2476929|2476306|2476580|N|N|N|N|N| +2476673|AAAAAAAABIKMFCAA|2068-10-18|2025|8808|676|2068|4|10|18|4|2068|676|8808|Thursday|2068Q4|N|N|N|2476656|2476929|2476307|2476581|N|N|N|N|N| +2476674|AAAAAAAACIKMFCAA|2068-10-19|2025|8808|676|2068|5|10|19|4|2068|676|8808|Friday|2068Q4|N|Y|N|2476656|2476929|2476308|2476582|N|N|N|N|N| +2476675|AAAAAAAADIKMFCAA|2068-10-20|2025|8808|676|2068|6|10|20|4|2068|676|8808|Saturday|2068Q4|N|Y|N|2476656|2476929|2476309|2476583|N|N|N|N|N| +2476676|AAAAAAAAEIKMFCAA|2068-10-21|2025|8808|676|2068|0|10|21|4|2068|676|8808|Sunday|2068Q4|N|N|N|2476656|2476929|2476310|2476584|N|N|N|N|N| +2476677|AAAAAAAAFIKMFCAA|2068-10-22|2025|8808|676|2068|1|10|22|4|2068|676|8808|Monday|2068Q4|N|N|N|2476656|2476929|2476311|2476585|N|N|N|N|N| +2476678|AAAAAAAAGIKMFCAA|2068-10-23|2025|8809|676|2068|2|10|23|4|2068|676|8809|Tuesday|2068Q4|N|N|N|2476656|2476929|2476312|2476586|N|N|N|N|N| +2476679|AAAAAAAAHIKMFCAA|2068-10-24|2025|8809|676|2068|3|10|24|4|2068|676|8809|Wednesday|2068Q4|N|N|N|2476656|2476929|2476313|2476587|N|N|N|N|N| +2476680|AAAAAAAAIIKMFCAA|2068-10-25|2025|8809|676|2068|4|10|25|4|2068|676|8809|Thursday|2068Q4|N|N|N|2476656|2476929|2476314|2476588|N|N|N|N|N| +2476681|AAAAAAAAJIKMFCAA|2068-10-26|2025|8809|676|2068|5|10|26|4|2068|676|8809|Friday|2068Q4|N|Y|N|2476656|2476929|2476315|2476589|N|N|N|N|N| +2476682|AAAAAAAAKIKMFCAA|2068-10-27|2025|8809|676|2068|6|10|27|4|2068|676|8809|Saturday|2068Q4|N|Y|N|2476656|2476929|2476316|2476590|N|N|N|N|N| +2476683|AAAAAAAALIKMFCAA|2068-10-28|2025|8809|676|2068|0|10|28|4|2068|676|8809|Sunday|2068Q4|N|N|N|2476656|2476929|2476317|2476591|N|N|N|N|N| +2476684|AAAAAAAAMIKMFCAA|2068-10-29|2025|8809|676|2068|1|10|29|4|2068|676|8809|Monday|2068Q4|N|N|N|2476656|2476929|2476318|2476592|N|N|N|N|N| +2476685|AAAAAAAANIKMFCAA|2068-10-30|2025|8810|676|2068|2|10|30|4|2068|676|8810|Tuesday|2068Q4|N|N|N|2476656|2476929|2476319|2476593|N|N|N|N|N| +2476686|AAAAAAAAOIKMFCAA|2068-10-31|2025|8810|676|2068|3|10|31|4|2068|676|8810|Wednesday|2068Q4|N|N|N|2476656|2476929|2476320|2476594|N|N|N|N|N| +2476687|AAAAAAAAPIKMFCAA|2068-11-01|2026|8810|676|2068|4|11|1|4|2068|676|8810|Thursday|2068Q4|N|N|N|2476687|2476991|2476321|2476595|N|N|N|N|N| +2476688|AAAAAAAAAJKMFCAA|2068-11-02|2026|8810|676|2068|5|11|2|4|2068|676|8810|Friday|2068Q4|N|Y|N|2476687|2476991|2476322|2476596|N|N|N|N|N| +2476689|AAAAAAAABJKMFCAA|2068-11-03|2026|8810|676|2068|6|11|3|4|2068|676|8810|Saturday|2068Q4|N|Y|N|2476687|2476991|2476323|2476597|N|N|N|N|N| +2476690|AAAAAAAACJKMFCAA|2068-11-04|2026|8810|676|2068|0|11|4|4|2068|676|8810|Sunday|2068Q4|N|N|N|2476687|2476991|2476324|2476598|N|N|N|N|N| +2476691|AAAAAAAADJKMFCAA|2068-11-05|2026|8810|676|2068|1|11|5|4|2068|676|8810|Monday|2068Q4|N|N|N|2476687|2476991|2476325|2476599|N|N|N|N|N| +2476692|AAAAAAAAEJKMFCAA|2068-11-06|2026|8811|676|2068|2|11|6|4|2068|676|8811|Tuesday|2068Q4|N|N|N|2476687|2476991|2476326|2476600|N|N|N|N|N| +2476693|AAAAAAAAFJKMFCAA|2068-11-07|2026|8811|676|2068|3|11|7|4|2068|676|8811|Wednesday|2068Q4|N|N|N|2476687|2476991|2476327|2476601|N|N|N|N|N| +2476694|AAAAAAAAGJKMFCAA|2068-11-08|2026|8811|676|2068|4|11|8|4|2068|676|8811|Thursday|2068Q4|N|N|N|2476687|2476991|2476328|2476602|N|N|N|N|N| +2476695|AAAAAAAAHJKMFCAA|2068-11-09|2026|8811|676|2068|5|11|9|4|2068|676|8811|Friday|2068Q4|N|Y|N|2476687|2476991|2476329|2476603|N|N|N|N|N| +2476696|AAAAAAAAIJKMFCAA|2068-11-10|2026|8811|676|2068|6|11|10|4|2068|676|8811|Saturday|2068Q4|N|Y|N|2476687|2476991|2476330|2476604|N|N|N|N|N| +2476697|AAAAAAAAJJKMFCAA|2068-11-11|2026|8811|676|2068|0|11|11|4|2068|676|8811|Sunday|2068Q4|N|N|N|2476687|2476991|2476331|2476605|N|N|N|N|N| +2476698|AAAAAAAAKJKMFCAA|2068-11-12|2026|8811|676|2068|1|11|12|4|2068|676|8811|Monday|2068Q4|N|N|N|2476687|2476991|2476332|2476606|N|N|N|N|N| +2476699|AAAAAAAALJKMFCAA|2068-11-13|2026|8812|676|2068|2|11|13|4|2068|676|8812|Tuesday|2068Q4|N|N|N|2476687|2476991|2476333|2476607|N|N|N|N|N| +2476700|AAAAAAAAMJKMFCAA|2068-11-14|2026|8812|676|2068|3|11|14|4|2068|676|8812|Wednesday|2068Q4|N|N|N|2476687|2476991|2476334|2476608|N|N|N|N|N| +2476701|AAAAAAAANJKMFCAA|2068-11-15|2026|8812|676|2068|4|11|15|4|2068|676|8812|Thursday|2068Q4|N|N|N|2476687|2476991|2476335|2476609|N|N|N|N|N| +2476702|AAAAAAAAOJKMFCAA|2068-11-16|2026|8812|676|2068|5|11|16|4|2068|676|8812|Friday|2068Q4|N|Y|N|2476687|2476991|2476336|2476610|N|N|N|N|N| +2476703|AAAAAAAAPJKMFCAA|2068-11-17|2026|8812|676|2068|6|11|17|4|2068|676|8812|Saturday|2068Q4|N|Y|N|2476687|2476991|2476337|2476611|N|N|N|N|N| +2476704|AAAAAAAAAKKMFCAA|2068-11-18|2026|8812|676|2068|0|11|18|4|2068|676|8812|Sunday|2068Q4|N|N|N|2476687|2476991|2476338|2476612|N|N|N|N|N| +2476705|AAAAAAAABKKMFCAA|2068-11-19|2026|8812|676|2068|1|11|19|4|2068|676|8812|Monday|2068Q4|N|N|N|2476687|2476991|2476339|2476613|N|N|N|N|N| +2476706|AAAAAAAACKKMFCAA|2068-11-20|2026|8813|676|2068|2|11|20|4|2068|676|8813|Tuesday|2068Q4|N|N|N|2476687|2476991|2476340|2476614|N|N|N|N|N| +2476707|AAAAAAAADKKMFCAA|2068-11-21|2026|8813|676|2068|3|11|21|4|2068|676|8813|Wednesday|2068Q4|N|N|N|2476687|2476991|2476341|2476615|N|N|N|N|N| +2476708|AAAAAAAAEKKMFCAA|2068-11-22|2026|8813|676|2068|4|11|22|4|2068|676|8813|Thursday|2068Q4|N|N|N|2476687|2476991|2476342|2476616|N|N|N|N|N| +2476709|AAAAAAAAFKKMFCAA|2068-11-23|2026|8813|676|2068|5|11|23|4|2068|676|8813|Friday|2068Q4|N|Y|N|2476687|2476991|2476343|2476617|N|N|N|N|N| +2476710|AAAAAAAAGKKMFCAA|2068-11-24|2026|8813|676|2068|6|11|24|4|2068|676|8813|Saturday|2068Q4|N|Y|N|2476687|2476991|2476344|2476618|N|N|N|N|N| +2476711|AAAAAAAAHKKMFCAA|2068-11-25|2026|8813|676|2068|0|11|25|4|2068|676|8813|Sunday|2068Q4|N|N|N|2476687|2476991|2476345|2476619|N|N|N|N|N| +2476712|AAAAAAAAIKKMFCAA|2068-11-26|2026|8813|676|2068|1|11|26|4|2068|676|8813|Monday|2068Q4|N|N|N|2476687|2476991|2476346|2476620|N|N|N|N|N| +2476713|AAAAAAAAJKKMFCAA|2068-11-27|2026|8814|676|2068|2|11|27|4|2068|676|8814|Tuesday|2068Q4|N|N|N|2476687|2476991|2476347|2476621|N|N|N|N|N| +2476714|AAAAAAAAKKKMFCAA|2068-11-28|2026|8814|676|2068|3|11|28|4|2068|676|8814|Wednesday|2068Q4|N|N|N|2476687|2476991|2476348|2476622|N|N|N|N|N| +2476715|AAAAAAAALKKMFCAA|2068-11-29|2026|8814|676|2068|4|11|29|4|2068|676|8814|Thursday|2068Q4|N|N|N|2476687|2476991|2476349|2476623|N|N|N|N|N| +2476716|AAAAAAAAMKKMFCAA|2068-11-30|2026|8814|676|2068|5|11|30|4|2068|676|8814|Friday|2068Q4|N|Y|N|2476687|2476991|2476350|2476624|N|N|N|N|N| +2476717|AAAAAAAANKKMFCAA|2068-12-01|2027|8814|677|2068|6|12|1|4|2068|677|8814|Saturday|2068Q4|N|Y|N|2476717|2477051|2476351|2476625|N|N|N|N|N| +2476718|AAAAAAAAOKKMFCAA|2068-12-02|2027|8814|677|2068|0|12|2|4|2068|677|8814|Sunday|2068Q4|N|N|N|2476717|2477051|2476352|2476626|N|N|N|N|N| +2476719|AAAAAAAAPKKMFCAA|2068-12-03|2027|8814|677|2068|1|12|3|4|2068|677|8814|Monday|2068Q4|N|N|N|2476717|2477051|2476353|2476627|N|N|N|N|N| +2476720|AAAAAAAAALKMFCAA|2068-12-04|2027|8815|677|2068|2|12|4|4|2068|677|8815|Tuesday|2068Q4|N|N|N|2476717|2477051|2476354|2476628|N|N|N|N|N| +2476721|AAAAAAAABLKMFCAA|2068-12-05|2027|8815|677|2068|3|12|5|4|2068|677|8815|Wednesday|2068Q4|N|N|N|2476717|2477051|2476355|2476629|N|N|N|N|N| +2476722|AAAAAAAACLKMFCAA|2068-12-06|2027|8815|677|2068|4|12|6|4|2068|677|8815|Thursday|2068Q4|N|N|N|2476717|2477051|2476356|2476630|N|N|N|N|N| +2476723|AAAAAAAADLKMFCAA|2068-12-07|2027|8815|677|2068|5|12|7|4|2068|677|8815|Friday|2068Q4|N|Y|N|2476717|2477051|2476357|2476631|N|N|N|N|N| +2476724|AAAAAAAAELKMFCAA|2068-12-08|2027|8815|677|2068|6|12|8|4|2068|677|8815|Saturday|2068Q4|N|Y|N|2476717|2477051|2476358|2476632|N|N|N|N|N| +2476725|AAAAAAAAFLKMFCAA|2068-12-09|2027|8815|677|2068|0|12|9|4|2068|677|8815|Sunday|2068Q4|N|N|N|2476717|2477051|2476359|2476633|N|N|N|N|N| +2476726|AAAAAAAAGLKMFCAA|2068-12-10|2027|8815|677|2068|1|12|10|4|2068|677|8815|Monday|2068Q4|N|N|N|2476717|2477051|2476360|2476634|N|N|N|N|N| +2476727|AAAAAAAAHLKMFCAA|2068-12-11|2027|8816|677|2068|2|12|11|4|2068|677|8816|Tuesday|2068Q4|N|N|N|2476717|2477051|2476361|2476635|N|N|N|N|N| +2476728|AAAAAAAAILKMFCAA|2068-12-12|2027|8816|677|2068|3|12|12|4|2068|677|8816|Wednesday|2068Q4|N|N|N|2476717|2477051|2476362|2476636|N|N|N|N|N| +2476729|AAAAAAAAJLKMFCAA|2068-12-13|2027|8816|677|2068|4|12|13|4|2068|677|8816|Thursday|2068Q4|N|N|N|2476717|2477051|2476363|2476637|N|N|N|N|N| +2476730|AAAAAAAAKLKMFCAA|2068-12-14|2027|8816|677|2068|5|12|14|4|2068|677|8816|Friday|2068Q4|N|Y|N|2476717|2477051|2476364|2476638|N|N|N|N|N| +2476731|AAAAAAAALLKMFCAA|2068-12-15|2027|8816|677|2068|6|12|15|4|2068|677|8816|Saturday|2068Q4|N|Y|N|2476717|2477051|2476365|2476639|N|N|N|N|N| +2476732|AAAAAAAAMLKMFCAA|2068-12-16|2027|8816|677|2068|0|12|16|4|2068|677|8816|Sunday|2068Q4|N|N|N|2476717|2477051|2476366|2476640|N|N|N|N|N| +2476733|AAAAAAAANLKMFCAA|2068-12-17|2027|8816|677|2068|1|12|17|4|2068|677|8816|Monday|2068Q4|N|N|N|2476717|2477051|2476367|2476641|N|N|N|N|N| +2476734|AAAAAAAAOLKMFCAA|2068-12-18|2027|8817|677|2068|2|12|18|4|2068|677|8817|Tuesday|2068Q4|N|N|N|2476717|2477051|2476368|2476642|N|N|N|N|N| +2476735|AAAAAAAAPLKMFCAA|2068-12-19|2027|8817|677|2068|3|12|19|4|2068|677|8817|Wednesday|2068Q4|N|N|N|2476717|2477051|2476369|2476643|N|N|N|N|N| +2476736|AAAAAAAAAMKMFCAA|2068-12-20|2027|8817|677|2068|4|12|20|4|2068|677|8817|Thursday|2068Q4|N|N|N|2476717|2477051|2476370|2476644|N|N|N|N|N| +2476737|AAAAAAAABMKMFCAA|2068-12-21|2027|8817|677|2068|5|12|21|4|2068|677|8817|Friday|2068Q4|N|Y|N|2476717|2477051|2476371|2476645|N|N|N|N|N| +2476738|AAAAAAAACMKMFCAA|2068-12-22|2027|8817|677|2068|6|12|22|4|2068|677|8817|Saturday|2068Q4|N|Y|N|2476717|2477051|2476372|2476646|N|N|N|N|N| +2476739|AAAAAAAADMKMFCAA|2068-12-23|2027|8817|677|2068|0|12|23|4|2068|677|8817|Sunday|2068Q4|N|N|N|2476717|2477051|2476373|2476647|N|N|N|N|N| +2476740|AAAAAAAAEMKMFCAA|2068-12-24|2027|8817|677|2068|1|12|24|4|2068|677|8817|Monday|2068Q4|N|N|N|2476717|2477051|2476374|2476648|N|N|N|N|N| +2476741|AAAAAAAAFMKMFCAA|2068-12-25|2027|8818|677|2068|2|12|25|4|2068|677|8818|Tuesday|2068Q4|Y|N|N|2476717|2477051|2476375|2476649|N|N|N|N|N| +2476742|AAAAAAAAGMKMFCAA|2068-12-26|2027|8818|677|2068|3|12|26|4|2068|677|8818|Wednesday|2068Q4|N|N|Y|2476717|2477051|2476376|2476650|N|N|N|N|N| +2476743|AAAAAAAAHMKMFCAA|2068-12-27|2027|8818|677|2068|4|12|27|4|2068|677|8818|Thursday|2068Q4|N|N|N|2476717|2477051|2476377|2476651|N|N|N|N|N| +2476744|AAAAAAAAIMKMFCAA|2068-12-28|2027|8818|677|2068|5|12|28|4|2068|677|8818|Friday|2068Q4|N|Y|N|2476717|2477051|2476378|2476652|N|N|N|N|N| +2476745|AAAAAAAAJMKMFCAA|2068-12-29|2027|8818|677|2068|6|12|29|4|2068|677|8818|Saturday|2068Q4|N|Y|N|2476717|2477051|2476379|2476653|N|N|N|N|N| +2476746|AAAAAAAAKMKMFCAA|2068-12-30|2027|8818|677|2068|0|12|30|4|2068|677|8818|Sunday|2068Q4|N|N|N|2476717|2477051|2476380|2476654|N|N|N|N|N| +2476747|AAAAAAAALMKMFCAA|2068-12-31|2027|8818|677|2068|1|12|31|4|2068|677|8818|Monday|2068Q4|Y|N|N|2476717|2477051|2476381|2476655|N|N|N|N|N| +2476748|AAAAAAAAMMKMFCAA|2069-01-01|2028|8819|677|2069|2|1|1|1|2069|677|8819|Tuesday|2069Q1|Y|N|Y|2476748|2476747|2476382|2476656|N|N|N|N|N| +2476749|AAAAAAAANMKMFCAA|2069-01-02|2028|8819|677|2069|3|1|2|1|2069|677|8819|Wednesday|2069Q1|N|N|Y|2476748|2476747|2476383|2476657|N|N|N|N|N| +2476750|AAAAAAAAOMKMFCAA|2069-01-03|2028|8819|677|2069|4|1|3|1|2069|677|8819|Thursday|2069Q1|N|N|N|2476748|2476747|2476384|2476658|N|N|N|N|N| +2476751|AAAAAAAAPMKMFCAA|2069-01-04|2028|8819|677|2069|5|1|4|1|2069|677|8819|Friday|2069Q1|N|Y|N|2476748|2476747|2476385|2476659|N|N|N|N|N| +2476752|AAAAAAAAANKMFCAA|2069-01-05|2028|8819|677|2069|6|1|5|1|2069|677|8819|Saturday|2069Q1|N|Y|N|2476748|2476747|2476386|2476660|N|N|N|N|N| +2476753|AAAAAAAABNKMFCAA|2069-01-06|2028|8819|677|2069|0|1|6|1|2069|677|8819|Sunday|2069Q1|N|N|N|2476748|2476747|2476387|2476661|N|N|N|N|N| +2476754|AAAAAAAACNKMFCAA|2069-01-07|2028|8819|677|2069|1|1|7|1|2069|677|8819|Monday|2069Q1|N|N|N|2476748|2476747|2476388|2476662|N|N|N|N|N| +2476755|AAAAAAAADNKMFCAA|2069-01-08|2028|8820|677|2069|2|1|8|1|2069|677|8820|Tuesday|2069Q1|N|N|N|2476748|2476747|2476389|2476663|N|N|N|N|N| +2476756|AAAAAAAAENKMFCAA|2069-01-09|2028|8820|677|2069|3|1|9|1|2069|677|8820|Wednesday|2069Q1|N|N|N|2476748|2476747|2476390|2476664|N|N|N|N|N| +2476757|AAAAAAAAFNKMFCAA|2069-01-10|2028|8820|677|2069|4|1|10|1|2069|677|8820|Thursday|2069Q1|N|N|N|2476748|2476747|2476391|2476665|N|N|N|N|N| +2476758|AAAAAAAAGNKMFCAA|2069-01-11|2028|8820|677|2069|5|1|11|1|2069|677|8820|Friday|2069Q1|N|Y|N|2476748|2476747|2476392|2476666|N|N|N|N|N| +2476759|AAAAAAAAHNKMFCAA|2069-01-12|2028|8820|677|2069|6|1|12|1|2069|677|8820|Saturday|2069Q1|N|Y|N|2476748|2476747|2476393|2476667|N|N|N|N|N| +2476760|AAAAAAAAINKMFCAA|2069-01-13|2028|8820|677|2069|0|1|13|1|2069|677|8820|Sunday|2069Q1|N|N|N|2476748|2476747|2476394|2476668|N|N|N|N|N| +2476761|AAAAAAAAJNKMFCAA|2069-01-14|2028|8820|677|2069|1|1|14|1|2069|677|8820|Monday|2069Q1|N|N|N|2476748|2476747|2476395|2476669|N|N|N|N|N| +2476762|AAAAAAAAKNKMFCAA|2069-01-15|2028|8821|677|2069|2|1|15|1|2069|677|8821|Tuesday|2069Q1|N|N|N|2476748|2476747|2476396|2476670|N|N|N|N|N| +2476763|AAAAAAAALNKMFCAA|2069-01-16|2028|8821|677|2069|3|1|16|1|2069|677|8821|Wednesday|2069Q1|N|N|N|2476748|2476747|2476397|2476671|N|N|N|N|N| +2476764|AAAAAAAAMNKMFCAA|2069-01-17|2028|8821|677|2069|4|1|17|1|2069|677|8821|Thursday|2069Q1|N|N|N|2476748|2476747|2476398|2476672|N|N|N|N|N| +2476765|AAAAAAAANNKMFCAA|2069-01-18|2028|8821|677|2069|5|1|18|1|2069|677|8821|Friday|2069Q1|N|Y|N|2476748|2476747|2476399|2476673|N|N|N|N|N| +2476766|AAAAAAAAONKMFCAA|2069-01-19|2028|8821|677|2069|6|1|19|1|2069|677|8821|Saturday|2069Q1|N|Y|N|2476748|2476747|2476400|2476674|N|N|N|N|N| +2476767|AAAAAAAAPNKMFCAA|2069-01-20|2028|8821|677|2069|0|1|20|1|2069|677|8821|Sunday|2069Q1|N|N|N|2476748|2476747|2476401|2476675|N|N|N|N|N| +2476768|AAAAAAAAAOKMFCAA|2069-01-21|2028|8821|677|2069|1|1|21|1|2069|677|8821|Monday|2069Q1|N|N|N|2476748|2476747|2476402|2476676|N|N|N|N|N| +2476769|AAAAAAAABOKMFCAA|2069-01-22|2028|8822|677|2069|2|1|22|1|2069|677|8822|Tuesday|2069Q1|N|N|N|2476748|2476747|2476403|2476677|N|N|N|N|N| +2476770|AAAAAAAACOKMFCAA|2069-01-23|2028|8822|677|2069|3|1|23|1|2069|677|8822|Wednesday|2069Q1|N|N|N|2476748|2476747|2476404|2476678|N|N|N|N|N| +2476771|AAAAAAAADOKMFCAA|2069-01-24|2028|8822|677|2069|4|1|24|1|2069|677|8822|Thursday|2069Q1|N|N|N|2476748|2476747|2476405|2476679|N|N|N|N|N| +2476772|AAAAAAAAEOKMFCAA|2069-01-25|2028|8822|677|2069|5|1|25|1|2069|677|8822|Friday|2069Q1|N|Y|N|2476748|2476747|2476406|2476680|N|N|N|N|N| +2476773|AAAAAAAAFOKMFCAA|2069-01-26|2028|8822|677|2069|6|1|26|1|2069|677|8822|Saturday|2069Q1|N|Y|N|2476748|2476747|2476407|2476681|N|N|N|N|N| +2476774|AAAAAAAAGOKMFCAA|2069-01-27|2028|8822|677|2069|0|1|27|1|2069|677|8822|Sunday|2069Q1|N|N|N|2476748|2476747|2476408|2476682|N|N|N|N|N| +2476775|AAAAAAAAHOKMFCAA|2069-01-28|2028|8822|677|2069|1|1|28|1|2069|677|8822|Monday|2069Q1|N|N|N|2476748|2476747|2476409|2476683|N|N|N|N|N| +2476776|AAAAAAAAIOKMFCAA|2069-01-29|2028|8823|677|2069|2|1|29|1|2069|677|8823|Tuesday|2069Q1|N|N|N|2476748|2476747|2476410|2476684|N|N|N|N|N| +2476777|AAAAAAAAJOKMFCAA|2069-01-30|2028|8823|677|2069|3|1|30|1|2069|677|8823|Wednesday|2069Q1|N|N|N|2476748|2476747|2476411|2476685|N|N|N|N|N| +2476778|AAAAAAAAKOKMFCAA|2069-01-31|2028|8823|677|2069|4|1|31|1|2069|677|8823|Thursday|2069Q1|N|N|N|2476748|2476747|2476412|2476686|N|N|N|N|N| +2476779|AAAAAAAALOKMFCAA|2069-02-01|2029|8823|677|2069|5|2|1|1|2069|677|8823|Friday|2069Q1|N|Y|N|2476779|2476809|2476413|2476687|N|N|N|N|N| +2476780|AAAAAAAAMOKMFCAA|2069-02-02|2029|8823|677|2069|6|2|2|1|2069|677|8823|Saturday|2069Q1|N|Y|N|2476779|2476809|2476414|2476688|N|N|N|N|N| +2476781|AAAAAAAANOKMFCAA|2069-02-03|2029|8823|677|2069|0|2|3|1|2069|677|8823|Sunday|2069Q1|N|N|N|2476779|2476809|2476415|2476689|N|N|N|N|N| +2476782|AAAAAAAAOOKMFCAA|2069-02-04|2029|8823|677|2069|1|2|4|1|2069|677|8823|Monday|2069Q1|N|N|N|2476779|2476809|2476416|2476690|N|N|N|N|N| +2476783|AAAAAAAAPOKMFCAA|2069-02-05|2029|8824|677|2069|2|2|5|1|2069|677|8824|Tuesday|2069Q1|N|N|N|2476779|2476809|2476417|2476691|N|N|N|N|N| +2476784|AAAAAAAAAPKMFCAA|2069-02-06|2029|8824|677|2069|3|2|6|1|2069|677|8824|Wednesday|2069Q1|N|N|N|2476779|2476809|2476418|2476692|N|N|N|N|N| +2476785|AAAAAAAABPKMFCAA|2069-02-07|2029|8824|677|2069|4|2|7|1|2069|677|8824|Thursday|2069Q1|N|N|N|2476779|2476809|2476419|2476693|N|N|N|N|N| +2476786|AAAAAAAACPKMFCAA|2069-02-08|2029|8824|677|2069|5|2|8|1|2069|677|8824|Friday|2069Q1|N|Y|N|2476779|2476809|2476420|2476694|N|N|N|N|N| +2476787|AAAAAAAADPKMFCAA|2069-02-09|2029|8824|677|2069|6|2|9|1|2069|677|8824|Saturday|2069Q1|N|Y|N|2476779|2476809|2476421|2476695|N|N|N|N|N| +2476788|AAAAAAAAEPKMFCAA|2069-02-10|2029|8824|677|2069|0|2|10|1|2069|677|8824|Sunday|2069Q1|N|N|N|2476779|2476809|2476422|2476696|N|N|N|N|N| +2476789|AAAAAAAAFPKMFCAA|2069-02-11|2029|8824|677|2069|1|2|11|1|2069|677|8824|Monday|2069Q1|N|N|N|2476779|2476809|2476423|2476697|N|N|N|N|N| +2476790|AAAAAAAAGPKMFCAA|2069-02-12|2029|8825|677|2069|2|2|12|1|2069|677|8825|Tuesday|2069Q1|N|N|N|2476779|2476809|2476424|2476698|N|N|N|N|N| +2476791|AAAAAAAAHPKMFCAA|2069-02-13|2029|8825|677|2069|3|2|13|1|2069|677|8825|Wednesday|2069Q1|N|N|N|2476779|2476809|2476425|2476699|N|N|N|N|N| +2476792|AAAAAAAAIPKMFCAA|2069-02-14|2029|8825|677|2069|4|2|14|1|2069|677|8825|Thursday|2069Q1|N|N|N|2476779|2476809|2476426|2476700|N|N|N|N|N| +2476793|AAAAAAAAJPKMFCAA|2069-02-15|2029|8825|677|2069|5|2|15|1|2069|677|8825|Friday|2069Q1|N|Y|N|2476779|2476809|2476427|2476701|N|N|N|N|N| +2476794|AAAAAAAAKPKMFCAA|2069-02-16|2029|8825|677|2069|6|2|16|1|2069|677|8825|Saturday|2069Q1|N|Y|N|2476779|2476809|2476428|2476702|N|N|N|N|N| +2476795|AAAAAAAALPKMFCAA|2069-02-17|2029|8825|677|2069|0|2|17|1|2069|677|8825|Sunday|2069Q1|N|N|N|2476779|2476809|2476429|2476703|N|N|N|N|N| +2476796|AAAAAAAAMPKMFCAA|2069-02-18|2029|8825|677|2069|1|2|18|1|2069|677|8825|Monday|2069Q1|N|N|N|2476779|2476809|2476430|2476704|N|N|N|N|N| +2476797|AAAAAAAANPKMFCAA|2069-02-19|2029|8826|677|2069|2|2|19|1|2069|677|8826|Tuesday|2069Q1|N|N|N|2476779|2476809|2476431|2476705|N|N|N|N|N| +2476798|AAAAAAAAOPKMFCAA|2069-02-20|2029|8826|677|2069|3|2|20|1|2069|677|8826|Wednesday|2069Q1|N|N|N|2476779|2476809|2476432|2476706|N|N|N|N|N| +2476799|AAAAAAAAPPKMFCAA|2069-02-21|2029|8826|677|2069|4|2|21|1|2069|677|8826|Thursday|2069Q1|N|N|N|2476779|2476809|2476433|2476707|N|N|N|N|N| +2476800|AAAAAAAAAALMFCAA|2069-02-22|2029|8826|677|2069|5|2|22|1|2069|677|8826|Friday|2069Q1|N|Y|N|2476779|2476809|2476434|2476708|N|N|N|N|N| +2476801|AAAAAAAABALMFCAA|2069-02-23|2029|8826|677|2069|6|2|23|1|2069|677|8826|Saturday|2069Q1|N|Y|N|2476779|2476809|2476435|2476709|N|N|N|N|N| +2476802|AAAAAAAACALMFCAA|2069-02-24|2029|8826|677|2069|0|2|24|1|2069|677|8826|Sunday|2069Q1|N|N|N|2476779|2476809|2476436|2476710|N|N|N|N|N| +2476803|AAAAAAAADALMFCAA|2069-02-25|2029|8826|677|2069|1|2|25|1|2069|677|8826|Monday|2069Q1|N|N|N|2476779|2476809|2476437|2476711|N|N|N|N|N| +2476804|AAAAAAAAEALMFCAA|2069-02-26|2029|8827|677|2069|2|2|26|1|2069|677|8827|Tuesday|2069Q1|N|N|N|2476779|2476809|2476438|2476712|N|N|N|N|N| +2476805|AAAAAAAAFALMFCAA|2069-02-27|2029|8827|677|2069|3|2|27|1|2069|677|8827|Wednesday|2069Q1|N|N|N|2476779|2476809|2476439|2476713|N|N|N|N|N| +2476806|AAAAAAAAGALMFCAA|2069-02-28|2029|8827|677|2069|4|2|28|1|2069|677|8827|Thursday|2069Q1|N|N|N|2476779|2476809|2476440|2476714|N|N|N|N|N| +2476807|AAAAAAAAHALMFCAA|2069-03-01|2030|8827|678|2069|5|3|1|1|2069|678|8827|Friday|2069Q1|N|Y|N|2476807|2476865|2476442|2476715|N|N|N|N|N| +2476808|AAAAAAAAIALMFCAA|2069-03-02|2030|8827|678|2069|6|3|2|1|2069|678|8827|Saturday|2069Q1|N|Y|N|2476807|2476865|2476443|2476716|N|N|N|N|N| +2476809|AAAAAAAAJALMFCAA|2069-03-03|2030|8827|678|2069|0|3|3|1|2069|678|8827|Sunday|2069Q1|N|N|N|2476807|2476865|2476444|2476717|N|N|N|N|N| +2476810|AAAAAAAAKALMFCAA|2069-03-04|2030|8827|678|2069|1|3|4|1|2069|678|8827|Monday|2069Q1|N|N|N|2476807|2476865|2476445|2476718|N|N|N|N|N| +2476811|AAAAAAAALALMFCAA|2069-03-05|2030|8828|678|2069|2|3|5|1|2069|678|8828|Tuesday|2069Q1|N|N|N|2476807|2476865|2476446|2476719|N|N|N|N|N| +2476812|AAAAAAAAMALMFCAA|2069-03-06|2030|8828|678|2069|3|3|6|1|2069|678|8828|Wednesday|2069Q1|N|N|N|2476807|2476865|2476447|2476720|N|N|N|N|N| +2476813|AAAAAAAANALMFCAA|2069-03-07|2030|8828|678|2069|4|3|7|1|2069|678|8828|Thursday|2069Q1|N|N|N|2476807|2476865|2476448|2476721|N|N|N|N|N| +2476814|AAAAAAAAOALMFCAA|2069-03-08|2030|8828|678|2069|5|3|8|1|2069|678|8828|Friday|2069Q1|N|Y|N|2476807|2476865|2476449|2476722|N|N|N|N|N| +2476815|AAAAAAAAPALMFCAA|2069-03-09|2030|8828|678|2069|6|3|9|1|2069|678|8828|Saturday|2069Q1|N|Y|N|2476807|2476865|2476450|2476723|N|N|N|N|N| +2476816|AAAAAAAAABLMFCAA|2069-03-10|2030|8828|678|2069|0|3|10|1|2069|678|8828|Sunday|2069Q1|N|N|N|2476807|2476865|2476451|2476724|N|N|N|N|N| +2476817|AAAAAAAABBLMFCAA|2069-03-11|2030|8828|678|2069|1|3|11|1|2069|678|8828|Monday|2069Q1|N|N|N|2476807|2476865|2476452|2476725|N|N|N|N|N| +2476818|AAAAAAAACBLMFCAA|2069-03-12|2030|8829|678|2069|2|3|12|1|2069|678|8829|Tuesday|2069Q1|N|N|N|2476807|2476865|2476453|2476726|N|N|N|N|N| +2476819|AAAAAAAADBLMFCAA|2069-03-13|2030|8829|678|2069|3|3|13|1|2069|678|8829|Wednesday|2069Q1|N|N|N|2476807|2476865|2476454|2476727|N|N|N|N|N| +2476820|AAAAAAAAEBLMFCAA|2069-03-14|2030|8829|678|2069|4|3|14|1|2069|678|8829|Thursday|2069Q1|N|N|N|2476807|2476865|2476455|2476728|N|N|N|N|N| +2476821|AAAAAAAAFBLMFCAA|2069-03-15|2030|8829|678|2069|5|3|15|1|2069|678|8829|Friday|2069Q1|N|Y|N|2476807|2476865|2476456|2476729|N|N|N|N|N| +2476822|AAAAAAAAGBLMFCAA|2069-03-16|2030|8829|678|2069|6|3|16|1|2069|678|8829|Saturday|2069Q1|N|Y|N|2476807|2476865|2476457|2476730|N|N|N|N|N| +2476823|AAAAAAAAHBLMFCAA|2069-03-17|2030|8829|678|2069|0|3|17|1|2069|678|8829|Sunday|2069Q1|N|N|N|2476807|2476865|2476458|2476731|N|N|N|N|N| +2476824|AAAAAAAAIBLMFCAA|2069-03-18|2030|8829|678|2069|1|3|18|1|2069|678|8829|Monday|2069Q1|N|N|N|2476807|2476865|2476459|2476732|N|N|N|N|N| +2476825|AAAAAAAAJBLMFCAA|2069-03-19|2030|8830|678|2069|2|3|19|1|2069|678|8830|Tuesday|2069Q1|N|N|N|2476807|2476865|2476460|2476733|N|N|N|N|N| +2476826|AAAAAAAAKBLMFCAA|2069-03-20|2030|8830|678|2069|3|3|20|1|2069|678|8830|Wednesday|2069Q1|N|N|N|2476807|2476865|2476461|2476734|N|N|N|N|N| +2476827|AAAAAAAALBLMFCAA|2069-03-21|2030|8830|678|2069|4|3|21|1|2069|678|8830|Thursday|2069Q1|N|N|N|2476807|2476865|2476462|2476735|N|N|N|N|N| +2476828|AAAAAAAAMBLMFCAA|2069-03-22|2030|8830|678|2069|5|3|22|1|2069|678|8830|Friday|2069Q1|N|Y|N|2476807|2476865|2476463|2476736|N|N|N|N|N| +2476829|AAAAAAAANBLMFCAA|2069-03-23|2030|8830|678|2069|6|3|23|1|2069|678|8830|Saturday|2069Q1|N|Y|N|2476807|2476865|2476464|2476737|N|N|N|N|N| +2476830|AAAAAAAAOBLMFCAA|2069-03-24|2030|8830|678|2069|0|3|24|1|2069|678|8830|Sunday|2069Q1|N|N|N|2476807|2476865|2476465|2476738|N|N|N|N|N| +2476831|AAAAAAAAPBLMFCAA|2069-03-25|2030|8830|678|2069|1|3|25|1|2069|678|8830|Monday|2069Q1|N|N|N|2476807|2476865|2476466|2476739|N|N|N|N|N| +2476832|AAAAAAAAACLMFCAA|2069-03-26|2030|8831|678|2069|2|3|26|1|2069|678|8831|Tuesday|2069Q1|N|N|N|2476807|2476865|2476467|2476740|N|N|N|N|N| +2476833|AAAAAAAABCLMFCAA|2069-03-27|2030|8831|678|2069|3|3|27|1|2069|678|8831|Wednesday|2069Q1|N|N|N|2476807|2476865|2476468|2476741|N|N|N|N|N| +2476834|AAAAAAAACCLMFCAA|2069-03-28|2030|8831|678|2069|4|3|28|1|2069|678|8831|Thursday|2069Q1|N|N|N|2476807|2476865|2476469|2476742|N|N|N|N|N| +2476835|AAAAAAAADCLMFCAA|2069-03-29|2030|8831|678|2069|5|3|29|1|2069|678|8831|Friday|2069Q1|N|Y|N|2476807|2476865|2476470|2476743|N|N|N|N|N| +2476836|AAAAAAAAECLMFCAA|2069-03-30|2030|8831|678|2069|6|3|30|1|2069|678|8831|Saturday|2069Q1|N|Y|N|2476807|2476865|2476471|2476744|N|N|N|N|N| +2476837|AAAAAAAAFCLMFCAA|2069-03-31|2030|8831|678|2069|0|3|31|1|2069|678|8831|Sunday|2069Q1|N|N|N|2476807|2476865|2476472|2476745|N|N|N|N|N| +2476838|AAAAAAAAGCLMFCAA|2069-04-01|2031|8831|678|2069|1|4|1|1|2069|678|8831|Monday|2069Q1|N|N|N|2476838|2476927|2476473|2476748|N|N|N|N|N| +2476839|AAAAAAAAHCLMFCAA|2069-04-02|2031|8832|678|2069|2|4|2|2|2069|678|8832|Tuesday|2069Q2|N|N|N|2476838|2476927|2476474|2476749|N|N|N|N|N| +2476840|AAAAAAAAICLMFCAA|2069-04-03|2031|8832|678|2069|3|4|3|2|2069|678|8832|Wednesday|2069Q2|N|N|N|2476838|2476927|2476475|2476750|N|N|N|N|N| +2476841|AAAAAAAAJCLMFCAA|2069-04-04|2031|8832|678|2069|4|4|4|2|2069|678|8832|Thursday|2069Q2|N|N|N|2476838|2476927|2476476|2476751|N|N|N|N|N| +2476842|AAAAAAAAKCLMFCAA|2069-04-05|2031|8832|678|2069|5|4|5|2|2069|678|8832|Friday|2069Q2|N|Y|N|2476838|2476927|2476477|2476752|N|N|N|N|N| +2476843|AAAAAAAALCLMFCAA|2069-04-06|2031|8832|678|2069|6|4|6|2|2069|678|8832|Saturday|2069Q2|N|Y|N|2476838|2476927|2476478|2476753|N|N|N|N|N| +2476844|AAAAAAAAMCLMFCAA|2069-04-07|2031|8832|678|2069|0|4|7|2|2069|678|8832|Sunday|2069Q2|N|N|N|2476838|2476927|2476479|2476754|N|N|N|N|N| +2476845|AAAAAAAANCLMFCAA|2069-04-08|2031|8832|678|2069|1|4|8|2|2069|678|8832|Monday|2069Q2|N|N|N|2476838|2476927|2476480|2476755|N|N|N|N|N| +2476846|AAAAAAAAOCLMFCAA|2069-04-09|2031|8833|678|2069|2|4|9|2|2069|678|8833|Tuesday|2069Q2|N|N|N|2476838|2476927|2476481|2476756|N|N|N|N|N| +2476847|AAAAAAAAPCLMFCAA|2069-04-10|2031|8833|678|2069|3|4|10|2|2069|678|8833|Wednesday|2069Q2|N|N|N|2476838|2476927|2476482|2476757|N|N|N|N|N| +2476848|AAAAAAAAADLMFCAA|2069-04-11|2031|8833|678|2069|4|4|11|2|2069|678|8833|Thursday|2069Q2|N|N|N|2476838|2476927|2476483|2476758|N|N|N|N|N| +2476849|AAAAAAAABDLMFCAA|2069-04-12|2031|8833|678|2069|5|4|12|2|2069|678|8833|Friday|2069Q2|N|Y|N|2476838|2476927|2476484|2476759|N|N|N|N|N| +2476850|AAAAAAAACDLMFCAA|2069-04-13|2031|8833|678|2069|6|4|13|2|2069|678|8833|Saturday|2069Q2|N|Y|N|2476838|2476927|2476485|2476760|N|N|N|N|N| +2476851|AAAAAAAADDLMFCAA|2069-04-14|2031|8833|678|2069|0|4|14|2|2069|678|8833|Sunday|2069Q2|N|N|N|2476838|2476927|2476486|2476761|N|N|N|N|N| +2476852|AAAAAAAAEDLMFCAA|2069-04-15|2031|8833|678|2069|1|4|15|2|2069|678|8833|Monday|2069Q2|N|N|N|2476838|2476927|2476487|2476762|N|N|N|N|N| +2476853|AAAAAAAAFDLMFCAA|2069-04-16|2031|8834|678|2069|2|4|16|2|2069|678|8834|Tuesday|2069Q2|N|N|N|2476838|2476927|2476488|2476763|N|N|N|N|N| +2476854|AAAAAAAAGDLMFCAA|2069-04-17|2031|8834|678|2069|3|4|17|2|2069|678|8834|Wednesday|2069Q2|N|N|N|2476838|2476927|2476489|2476764|N|N|N|N|N| +2476855|AAAAAAAAHDLMFCAA|2069-04-18|2031|8834|678|2069|4|4|18|2|2069|678|8834|Thursday|2069Q2|N|N|N|2476838|2476927|2476490|2476765|N|N|N|N|N| +2476856|AAAAAAAAIDLMFCAA|2069-04-19|2031|8834|678|2069|5|4|19|2|2069|678|8834|Friday|2069Q2|N|Y|N|2476838|2476927|2476491|2476766|N|N|N|N|N| +2476857|AAAAAAAAJDLMFCAA|2069-04-20|2031|8834|678|2069|6|4|20|2|2069|678|8834|Saturday|2069Q2|N|Y|N|2476838|2476927|2476492|2476767|N|N|N|N|N| +2476858|AAAAAAAAKDLMFCAA|2069-04-21|2031|8834|678|2069|0|4|21|2|2069|678|8834|Sunday|2069Q2|N|N|N|2476838|2476927|2476493|2476768|N|N|N|N|N| +2476859|AAAAAAAALDLMFCAA|2069-04-22|2031|8834|678|2069|1|4|22|2|2069|678|8834|Monday|2069Q2|N|N|N|2476838|2476927|2476494|2476769|N|N|N|N|N| +2476860|AAAAAAAAMDLMFCAA|2069-04-23|2031|8835|678|2069|2|4|23|2|2069|678|8835|Tuesday|2069Q2|N|N|N|2476838|2476927|2476495|2476770|N|N|N|N|N| +2476861|AAAAAAAANDLMFCAA|2069-04-24|2031|8835|678|2069|3|4|24|2|2069|678|8835|Wednesday|2069Q2|N|N|N|2476838|2476927|2476496|2476771|N|N|N|N|N| +2476862|AAAAAAAAODLMFCAA|2069-04-25|2031|8835|678|2069|4|4|25|2|2069|678|8835|Thursday|2069Q2|N|N|N|2476838|2476927|2476497|2476772|N|N|N|N|N| +2476863|AAAAAAAAPDLMFCAA|2069-04-26|2031|8835|678|2069|5|4|26|2|2069|678|8835|Friday|2069Q2|N|Y|N|2476838|2476927|2476498|2476773|N|N|N|N|N| +2476864|AAAAAAAAAELMFCAA|2069-04-27|2031|8835|678|2069|6|4|27|2|2069|678|8835|Saturday|2069Q2|N|Y|N|2476838|2476927|2476499|2476774|N|N|N|N|N| +2476865|AAAAAAAABELMFCAA|2069-04-28|2031|8835|678|2069|0|4|28|2|2069|678|8835|Sunday|2069Q2|N|N|N|2476838|2476927|2476500|2476775|N|N|N|N|N| +2476866|AAAAAAAACELMFCAA|2069-04-29|2031|8835|678|2069|1|4|29|2|2069|678|8835|Monday|2069Q2|N|N|N|2476838|2476927|2476501|2476776|N|N|N|N|N| +2476867|AAAAAAAADELMFCAA|2069-04-30|2031|8836|678|2069|2|4|30|2|2069|678|8836|Tuesday|2069Q2|N|N|N|2476838|2476927|2476502|2476777|N|N|N|N|N| +2476868|AAAAAAAAEELMFCAA|2069-05-01|2032|8836|678|2069|3|5|1|2|2069|678|8836|Wednesday|2069Q2|N|N|N|2476868|2476987|2476503|2476778|N|N|N|N|N| +2476869|AAAAAAAAFELMFCAA|2069-05-02|2032|8836|678|2069|4|5|2|2|2069|678|8836|Thursday|2069Q2|N|N|N|2476868|2476987|2476504|2476779|N|N|N|N|N| +2476870|AAAAAAAAGELMFCAA|2069-05-03|2032|8836|678|2069|5|5|3|2|2069|678|8836|Friday|2069Q2|N|Y|N|2476868|2476987|2476505|2476780|N|N|N|N|N| +2476871|AAAAAAAAHELMFCAA|2069-05-04|2032|8836|678|2069|6|5|4|2|2069|678|8836|Saturday|2069Q2|N|Y|N|2476868|2476987|2476506|2476781|N|N|N|N|N| +2476872|AAAAAAAAIELMFCAA|2069-05-05|2032|8836|678|2069|0|5|5|2|2069|678|8836|Sunday|2069Q2|N|N|N|2476868|2476987|2476507|2476782|N|N|N|N|N| +2476873|AAAAAAAAJELMFCAA|2069-05-06|2032|8836|678|2069|1|5|6|2|2069|678|8836|Monday|2069Q2|N|N|N|2476868|2476987|2476508|2476783|N|N|N|N|N| +2476874|AAAAAAAAKELMFCAA|2069-05-07|2032|8837|678|2069|2|5|7|2|2069|678|8837|Tuesday|2069Q2|N|N|N|2476868|2476987|2476509|2476784|N|N|N|N|N| +2476875|AAAAAAAALELMFCAA|2069-05-08|2032|8837|678|2069|3|5|8|2|2069|678|8837|Wednesday|2069Q2|N|N|N|2476868|2476987|2476510|2476785|N|N|N|N|N| +2476876|AAAAAAAAMELMFCAA|2069-05-09|2032|8837|678|2069|4|5|9|2|2069|678|8837|Thursday|2069Q2|N|N|N|2476868|2476987|2476511|2476786|N|N|N|N|N| +2476877|AAAAAAAANELMFCAA|2069-05-10|2032|8837|678|2069|5|5|10|2|2069|678|8837|Friday|2069Q2|N|Y|N|2476868|2476987|2476512|2476787|N|N|N|N|N| +2476878|AAAAAAAAOELMFCAA|2069-05-11|2032|8837|678|2069|6|5|11|2|2069|678|8837|Saturday|2069Q2|N|Y|N|2476868|2476987|2476513|2476788|N|N|N|N|N| +2476879|AAAAAAAAPELMFCAA|2069-05-12|2032|8837|678|2069|0|5|12|2|2069|678|8837|Sunday|2069Q2|N|N|N|2476868|2476987|2476514|2476789|N|N|N|N|N| +2476880|AAAAAAAAAFLMFCAA|2069-05-13|2032|8837|678|2069|1|5|13|2|2069|678|8837|Monday|2069Q2|N|N|N|2476868|2476987|2476515|2476790|N|N|N|N|N| +2476881|AAAAAAAABFLMFCAA|2069-05-14|2032|8838|678|2069|2|5|14|2|2069|678|8838|Tuesday|2069Q2|N|N|N|2476868|2476987|2476516|2476791|N|N|N|N|N| +2476882|AAAAAAAACFLMFCAA|2069-05-15|2032|8838|678|2069|3|5|15|2|2069|678|8838|Wednesday|2069Q2|N|N|N|2476868|2476987|2476517|2476792|N|N|N|N|N| +2476883|AAAAAAAADFLMFCAA|2069-05-16|2032|8838|678|2069|4|5|16|2|2069|678|8838|Thursday|2069Q2|N|N|N|2476868|2476987|2476518|2476793|N|N|N|N|N| +2476884|AAAAAAAAEFLMFCAA|2069-05-17|2032|8838|678|2069|5|5|17|2|2069|678|8838|Friday|2069Q2|N|Y|N|2476868|2476987|2476519|2476794|N|N|N|N|N| +2476885|AAAAAAAAFFLMFCAA|2069-05-18|2032|8838|678|2069|6|5|18|2|2069|678|8838|Saturday|2069Q2|N|Y|N|2476868|2476987|2476520|2476795|N|N|N|N|N| +2476886|AAAAAAAAGFLMFCAA|2069-05-19|2032|8838|678|2069|0|5|19|2|2069|678|8838|Sunday|2069Q2|N|N|N|2476868|2476987|2476521|2476796|N|N|N|N|N| +2476887|AAAAAAAAHFLMFCAA|2069-05-20|2032|8838|678|2069|1|5|20|2|2069|678|8838|Monday|2069Q2|N|N|N|2476868|2476987|2476522|2476797|N|N|N|N|N| +2476888|AAAAAAAAIFLMFCAA|2069-05-21|2032|8839|678|2069|2|5|21|2|2069|678|8839|Tuesday|2069Q2|N|N|N|2476868|2476987|2476523|2476798|N|N|N|N|N| +2476889|AAAAAAAAJFLMFCAA|2069-05-22|2032|8839|678|2069|3|5|22|2|2069|678|8839|Wednesday|2069Q2|N|N|N|2476868|2476987|2476524|2476799|N|N|N|N|N| +2476890|AAAAAAAAKFLMFCAA|2069-05-23|2032|8839|678|2069|4|5|23|2|2069|678|8839|Thursday|2069Q2|N|N|N|2476868|2476987|2476525|2476800|N|N|N|N|N| +2476891|AAAAAAAALFLMFCAA|2069-05-24|2032|8839|678|2069|5|5|24|2|2069|678|8839|Friday|2069Q2|N|Y|N|2476868|2476987|2476526|2476801|N|N|N|N|N| +2476892|AAAAAAAAMFLMFCAA|2069-05-25|2032|8839|678|2069|6|5|25|2|2069|678|8839|Saturday|2069Q2|N|Y|N|2476868|2476987|2476527|2476802|N|N|N|N|N| +2476893|AAAAAAAANFLMFCAA|2069-05-26|2032|8839|678|2069|0|5|26|2|2069|678|8839|Sunday|2069Q2|N|N|N|2476868|2476987|2476528|2476803|N|N|N|N|N| +2476894|AAAAAAAAOFLMFCAA|2069-05-27|2032|8839|678|2069|1|5|27|2|2069|678|8839|Monday|2069Q2|N|N|N|2476868|2476987|2476529|2476804|N|N|N|N|N| +2476895|AAAAAAAAPFLMFCAA|2069-05-28|2032|8840|678|2069|2|5|28|2|2069|678|8840|Tuesday|2069Q2|N|N|N|2476868|2476987|2476530|2476805|N|N|N|N|N| +2476896|AAAAAAAAAGLMFCAA|2069-05-29|2032|8840|678|2069|3|5|29|2|2069|678|8840|Wednesday|2069Q2|N|N|N|2476868|2476987|2476531|2476806|N|N|N|N|N| +2476897|AAAAAAAABGLMFCAA|2069-05-30|2032|8840|678|2069|4|5|30|2|2069|678|8840|Thursday|2069Q2|N|N|N|2476868|2476987|2476532|2476807|N|N|N|N|N| +2476898|AAAAAAAACGLMFCAA|2069-05-31|2032|8840|678|2069|5|5|31|2|2069|678|8840|Friday|2069Q2|N|Y|N|2476868|2476987|2476533|2476808|N|N|N|N|N| +2476899|AAAAAAAADGLMFCAA|2069-06-01|2033|8840|679|2069|6|6|1|2|2069|679|8840|Saturday|2069Q2|N|Y|N|2476899|2477049|2476534|2476809|N|N|N|N|N| +2476900|AAAAAAAAEGLMFCAA|2069-06-02|2033|8840|679|2069|0|6|2|2|2069|679|8840|Sunday|2069Q2|N|N|N|2476899|2477049|2476535|2476810|N|N|N|N|N| +2476901|AAAAAAAAFGLMFCAA|2069-06-03|2033|8840|679|2069|1|6|3|2|2069|679|8840|Monday|2069Q2|N|N|N|2476899|2477049|2476536|2476811|N|N|N|N|N| +2476902|AAAAAAAAGGLMFCAA|2069-06-04|2033|8841|679|2069|2|6|4|2|2069|679|8841|Tuesday|2069Q2|N|N|N|2476899|2477049|2476537|2476812|N|N|N|N|N| +2476903|AAAAAAAAHGLMFCAA|2069-06-05|2033|8841|679|2069|3|6|5|2|2069|679|8841|Wednesday|2069Q2|N|N|N|2476899|2477049|2476538|2476813|N|N|N|N|N| +2476904|AAAAAAAAIGLMFCAA|2069-06-06|2033|8841|679|2069|4|6|6|2|2069|679|8841|Thursday|2069Q2|N|N|N|2476899|2477049|2476539|2476814|N|N|N|N|N| +2476905|AAAAAAAAJGLMFCAA|2069-06-07|2033|8841|679|2069|5|6|7|2|2069|679|8841|Friday|2069Q2|N|Y|N|2476899|2477049|2476540|2476815|N|N|N|N|N| +2476906|AAAAAAAAKGLMFCAA|2069-06-08|2033|8841|679|2069|6|6|8|2|2069|679|8841|Saturday|2069Q2|N|Y|N|2476899|2477049|2476541|2476816|N|N|N|N|N| +2476907|AAAAAAAALGLMFCAA|2069-06-09|2033|8841|679|2069|0|6|9|2|2069|679|8841|Sunday|2069Q2|N|N|N|2476899|2477049|2476542|2476817|N|N|N|N|N| +2476908|AAAAAAAAMGLMFCAA|2069-06-10|2033|8841|679|2069|1|6|10|2|2069|679|8841|Monday|2069Q2|N|N|N|2476899|2477049|2476543|2476818|N|N|N|N|N| +2476909|AAAAAAAANGLMFCAA|2069-06-11|2033|8842|679|2069|2|6|11|2|2069|679|8842|Tuesday|2069Q2|N|N|N|2476899|2477049|2476544|2476819|N|N|N|N|N| +2476910|AAAAAAAAOGLMFCAA|2069-06-12|2033|8842|679|2069|3|6|12|2|2069|679|8842|Wednesday|2069Q2|N|N|N|2476899|2477049|2476545|2476820|N|N|N|N|N| +2476911|AAAAAAAAPGLMFCAA|2069-06-13|2033|8842|679|2069|4|6|13|2|2069|679|8842|Thursday|2069Q2|N|N|N|2476899|2477049|2476546|2476821|N|N|N|N|N| +2476912|AAAAAAAAAHLMFCAA|2069-06-14|2033|8842|679|2069|5|6|14|2|2069|679|8842|Friday|2069Q2|N|Y|N|2476899|2477049|2476547|2476822|N|N|N|N|N| +2476913|AAAAAAAABHLMFCAA|2069-06-15|2033|8842|679|2069|6|6|15|2|2069|679|8842|Saturday|2069Q2|N|Y|N|2476899|2477049|2476548|2476823|N|N|N|N|N| +2476914|AAAAAAAACHLMFCAA|2069-06-16|2033|8842|679|2069|0|6|16|2|2069|679|8842|Sunday|2069Q2|N|N|N|2476899|2477049|2476549|2476824|N|N|N|N|N| +2476915|AAAAAAAADHLMFCAA|2069-06-17|2033|8842|679|2069|1|6|17|2|2069|679|8842|Monday|2069Q2|N|N|N|2476899|2477049|2476550|2476825|N|N|N|N|N| +2476916|AAAAAAAAEHLMFCAA|2069-06-18|2033|8843|679|2069|2|6|18|2|2069|679|8843|Tuesday|2069Q2|N|N|N|2476899|2477049|2476551|2476826|N|N|N|N|N| +2476917|AAAAAAAAFHLMFCAA|2069-06-19|2033|8843|679|2069|3|6|19|2|2069|679|8843|Wednesday|2069Q2|N|N|N|2476899|2477049|2476552|2476827|N|N|N|N|N| +2476918|AAAAAAAAGHLMFCAA|2069-06-20|2033|8843|679|2069|4|6|20|2|2069|679|8843|Thursday|2069Q2|N|N|N|2476899|2477049|2476553|2476828|N|N|N|N|N| +2476919|AAAAAAAAHHLMFCAA|2069-06-21|2033|8843|679|2069|5|6|21|2|2069|679|8843|Friday|2069Q2|N|Y|N|2476899|2477049|2476554|2476829|N|N|N|N|N| +2476920|AAAAAAAAIHLMFCAA|2069-06-22|2033|8843|679|2069|6|6|22|2|2069|679|8843|Saturday|2069Q2|N|Y|N|2476899|2477049|2476555|2476830|N|N|N|N|N| +2476921|AAAAAAAAJHLMFCAA|2069-06-23|2033|8843|679|2069|0|6|23|2|2069|679|8843|Sunday|2069Q2|N|N|N|2476899|2477049|2476556|2476831|N|N|N|N|N| +2476922|AAAAAAAAKHLMFCAA|2069-06-24|2033|8843|679|2069|1|6|24|2|2069|679|8843|Monday|2069Q2|N|N|N|2476899|2477049|2476557|2476832|N|N|N|N|N| +2476923|AAAAAAAALHLMFCAA|2069-06-25|2033|8844|679|2069|2|6|25|2|2069|679|8844|Tuesday|2069Q2|N|N|N|2476899|2477049|2476558|2476833|N|N|N|N|N| +2476924|AAAAAAAAMHLMFCAA|2069-06-26|2033|8844|679|2069|3|6|26|2|2069|679|8844|Wednesday|2069Q2|N|N|N|2476899|2477049|2476559|2476834|N|N|N|N|N| +2476925|AAAAAAAANHLMFCAA|2069-06-27|2033|8844|679|2069|4|6|27|2|2069|679|8844|Thursday|2069Q2|N|N|N|2476899|2477049|2476560|2476835|N|N|N|N|N| +2476926|AAAAAAAAOHLMFCAA|2069-06-28|2033|8844|679|2069|5|6|28|2|2069|679|8844|Friday|2069Q2|N|Y|N|2476899|2477049|2476561|2476836|N|N|N|N|N| +2476927|AAAAAAAAPHLMFCAA|2069-06-29|2033|8844|679|2069|6|6|29|2|2069|679|8844|Saturday|2069Q2|N|Y|N|2476899|2477049|2476562|2476837|N|N|N|N|N| +2476928|AAAAAAAAAILMFCAA|2069-06-30|2033|8844|679|2069|0|6|30|2|2069|679|8844|Sunday|2069Q2|N|N|N|2476899|2477049|2476563|2476838|N|N|N|N|N| +2476929|AAAAAAAABILMFCAA|2069-07-01|2034|8844|679|2069|1|7|1|2|2069|679|8844|Monday|2069Q2|N|N|N|2476929|2477109|2476564|2476838|N|N|N|N|N| +2476930|AAAAAAAACILMFCAA|2069-07-02|2034|8845|679|2069|2|7|2|3|2069|679|8845|Tuesday|2069Q3|N|N|N|2476929|2477109|2476565|2476839|N|N|N|N|N| +2476931|AAAAAAAADILMFCAA|2069-07-03|2034|8845|679|2069|3|7|3|3|2069|679|8845|Wednesday|2069Q3|N|N|N|2476929|2477109|2476566|2476840|N|N|N|N|N| +2476932|AAAAAAAAEILMFCAA|2069-07-04|2034|8845|679|2069|4|7|4|3|2069|679|8845|Thursday|2069Q3|N|N|N|2476929|2477109|2476567|2476841|N|N|N|N|N| +2476933|AAAAAAAAFILMFCAA|2069-07-05|2034|8845|679|2069|5|7|5|3|2069|679|8845|Friday|2069Q3|Y|Y|N|2476929|2477109|2476568|2476842|N|N|N|N|N| +2476934|AAAAAAAAGILMFCAA|2069-07-06|2034|8845|679|2069|6|7|6|3|2069|679|8845|Saturday|2069Q3|N|Y|Y|2476929|2477109|2476569|2476843|N|N|N|N|N| +2476935|AAAAAAAAHILMFCAA|2069-07-07|2034|8845|679|2069|0|7|7|3|2069|679|8845|Sunday|2069Q3|N|N|N|2476929|2477109|2476570|2476844|N|N|N|N|N| +2476936|AAAAAAAAIILMFCAA|2069-07-08|2034|8845|679|2069|1|7|8|3|2069|679|8845|Monday|2069Q3|N|N|N|2476929|2477109|2476571|2476845|N|N|N|N|N| +2476937|AAAAAAAAJILMFCAA|2069-07-09|2034|8846|679|2069|2|7|9|3|2069|679|8846|Tuesday|2069Q3|N|N|N|2476929|2477109|2476572|2476846|N|N|N|N|N| +2476938|AAAAAAAAKILMFCAA|2069-07-10|2034|8846|679|2069|3|7|10|3|2069|679|8846|Wednesday|2069Q3|N|N|N|2476929|2477109|2476573|2476847|N|N|N|N|N| +2476939|AAAAAAAALILMFCAA|2069-07-11|2034|8846|679|2069|4|7|11|3|2069|679|8846|Thursday|2069Q3|N|N|N|2476929|2477109|2476574|2476848|N|N|N|N|N| +2476940|AAAAAAAAMILMFCAA|2069-07-12|2034|8846|679|2069|5|7|12|3|2069|679|8846|Friday|2069Q3|N|Y|N|2476929|2477109|2476575|2476849|N|N|N|N|N| +2476941|AAAAAAAANILMFCAA|2069-07-13|2034|8846|679|2069|6|7|13|3|2069|679|8846|Saturday|2069Q3|N|Y|N|2476929|2477109|2476576|2476850|N|N|N|N|N| +2476942|AAAAAAAAOILMFCAA|2069-07-14|2034|8846|679|2069|0|7|14|3|2069|679|8846|Sunday|2069Q3|N|N|N|2476929|2477109|2476577|2476851|N|N|N|N|N| +2476943|AAAAAAAAPILMFCAA|2069-07-15|2034|8846|679|2069|1|7|15|3|2069|679|8846|Monday|2069Q3|N|N|N|2476929|2477109|2476578|2476852|N|N|N|N|N| +2476944|AAAAAAAAAJLMFCAA|2069-07-16|2034|8847|679|2069|2|7|16|3|2069|679|8847|Tuesday|2069Q3|N|N|N|2476929|2477109|2476579|2476853|N|N|N|N|N| +2476945|AAAAAAAABJLMFCAA|2069-07-17|2034|8847|679|2069|3|7|17|3|2069|679|8847|Wednesday|2069Q3|N|N|N|2476929|2477109|2476580|2476854|N|N|N|N|N| +2476946|AAAAAAAACJLMFCAA|2069-07-18|2034|8847|679|2069|4|7|18|3|2069|679|8847|Thursday|2069Q3|N|N|N|2476929|2477109|2476581|2476855|N|N|N|N|N| +2476947|AAAAAAAADJLMFCAA|2069-07-19|2034|8847|679|2069|5|7|19|3|2069|679|8847|Friday|2069Q3|N|Y|N|2476929|2477109|2476582|2476856|N|N|N|N|N| +2476948|AAAAAAAAEJLMFCAA|2069-07-20|2034|8847|679|2069|6|7|20|3|2069|679|8847|Saturday|2069Q3|N|Y|N|2476929|2477109|2476583|2476857|N|N|N|N|N| +2476949|AAAAAAAAFJLMFCAA|2069-07-21|2034|8847|679|2069|0|7|21|3|2069|679|8847|Sunday|2069Q3|N|N|N|2476929|2477109|2476584|2476858|N|N|N|N|N| +2476950|AAAAAAAAGJLMFCAA|2069-07-22|2034|8847|679|2069|1|7|22|3|2069|679|8847|Monday|2069Q3|N|N|N|2476929|2477109|2476585|2476859|N|N|N|N|N| +2476951|AAAAAAAAHJLMFCAA|2069-07-23|2034|8848|679|2069|2|7|23|3|2069|679|8848|Tuesday|2069Q3|N|N|N|2476929|2477109|2476586|2476860|N|N|N|N|N| +2476952|AAAAAAAAIJLMFCAA|2069-07-24|2034|8848|679|2069|3|7|24|3|2069|679|8848|Wednesday|2069Q3|N|N|N|2476929|2477109|2476587|2476861|N|N|N|N|N| +2476953|AAAAAAAAJJLMFCAA|2069-07-25|2034|8848|679|2069|4|7|25|3|2069|679|8848|Thursday|2069Q3|N|N|N|2476929|2477109|2476588|2476862|N|N|N|N|N| +2476954|AAAAAAAAKJLMFCAA|2069-07-26|2034|8848|679|2069|5|7|26|3|2069|679|8848|Friday|2069Q3|N|Y|N|2476929|2477109|2476589|2476863|N|N|N|N|N| +2476955|AAAAAAAALJLMFCAA|2069-07-27|2034|8848|679|2069|6|7|27|3|2069|679|8848|Saturday|2069Q3|N|Y|N|2476929|2477109|2476590|2476864|N|N|N|N|N| +2476956|AAAAAAAAMJLMFCAA|2069-07-28|2034|8848|679|2069|0|7|28|3|2069|679|8848|Sunday|2069Q3|N|N|N|2476929|2477109|2476591|2476865|N|N|N|N|N| +2476957|AAAAAAAANJLMFCAA|2069-07-29|2034|8848|679|2069|1|7|29|3|2069|679|8848|Monday|2069Q3|N|N|N|2476929|2477109|2476592|2476866|N|N|N|N|N| +2476958|AAAAAAAAOJLMFCAA|2069-07-30|2034|8849|679|2069|2|7|30|3|2069|679|8849|Tuesday|2069Q3|N|N|N|2476929|2477109|2476593|2476867|N|N|N|N|N| +2476959|AAAAAAAAPJLMFCAA|2069-07-31|2034|8849|679|2069|3|7|31|3|2069|679|8849|Wednesday|2069Q3|N|N|N|2476929|2477109|2476594|2476868|N|N|N|N|N| +2476960|AAAAAAAAAKLMFCAA|2069-08-01|2035|8849|679|2069|4|8|1|3|2069|679|8849|Thursday|2069Q3|N|N|N|2476960|2477171|2476595|2476869|N|N|N|N|N| +2476961|AAAAAAAABKLMFCAA|2069-08-02|2035|8849|679|2069|5|8|2|3|2069|679|8849|Friday|2069Q3|N|Y|N|2476960|2477171|2476596|2476870|N|N|N|N|N| +2476962|AAAAAAAACKLMFCAA|2069-08-03|2035|8849|679|2069|6|8|3|3|2069|679|8849|Saturday|2069Q3|N|Y|N|2476960|2477171|2476597|2476871|N|N|N|N|N| +2476963|AAAAAAAADKLMFCAA|2069-08-04|2035|8849|679|2069|0|8|4|3|2069|679|8849|Sunday|2069Q3|N|N|N|2476960|2477171|2476598|2476872|N|N|N|N|N| +2476964|AAAAAAAAEKLMFCAA|2069-08-05|2035|8849|679|2069|1|8|5|3|2069|679|8849|Monday|2069Q3|N|N|N|2476960|2477171|2476599|2476873|N|N|N|N|N| +2476965|AAAAAAAAFKLMFCAA|2069-08-06|2035|8850|679|2069|2|8|6|3|2069|679|8850|Tuesday|2069Q3|N|N|N|2476960|2477171|2476600|2476874|N|N|N|N|N| +2476966|AAAAAAAAGKLMFCAA|2069-08-07|2035|8850|679|2069|3|8|7|3|2069|679|8850|Wednesday|2069Q3|N|N|N|2476960|2477171|2476601|2476875|N|N|N|N|N| +2476967|AAAAAAAAHKLMFCAA|2069-08-08|2035|8850|679|2069|4|8|8|3|2069|679|8850|Thursday|2069Q3|N|N|N|2476960|2477171|2476602|2476876|N|N|N|N|N| +2476968|AAAAAAAAIKLMFCAA|2069-08-09|2035|8850|679|2069|5|8|9|3|2069|679|8850|Friday|2069Q3|N|Y|N|2476960|2477171|2476603|2476877|N|N|N|N|N| +2476969|AAAAAAAAJKLMFCAA|2069-08-10|2035|8850|679|2069|6|8|10|3|2069|679|8850|Saturday|2069Q3|N|Y|N|2476960|2477171|2476604|2476878|N|N|N|N|N| +2476970|AAAAAAAAKKLMFCAA|2069-08-11|2035|8850|679|2069|0|8|11|3|2069|679|8850|Sunday|2069Q3|N|N|N|2476960|2477171|2476605|2476879|N|N|N|N|N| +2476971|AAAAAAAALKLMFCAA|2069-08-12|2035|8850|679|2069|1|8|12|3|2069|679|8850|Monday|2069Q3|N|N|N|2476960|2477171|2476606|2476880|N|N|N|N|N| +2476972|AAAAAAAAMKLMFCAA|2069-08-13|2035|8851|679|2069|2|8|13|3|2069|679|8851|Tuesday|2069Q3|N|N|N|2476960|2477171|2476607|2476881|N|N|N|N|N| +2476973|AAAAAAAANKLMFCAA|2069-08-14|2035|8851|679|2069|3|8|14|3|2069|679|8851|Wednesday|2069Q3|N|N|N|2476960|2477171|2476608|2476882|N|N|N|N|N| +2476974|AAAAAAAAOKLMFCAA|2069-08-15|2035|8851|679|2069|4|8|15|3|2069|679|8851|Thursday|2069Q3|N|N|N|2476960|2477171|2476609|2476883|N|N|N|N|N| +2476975|AAAAAAAAPKLMFCAA|2069-08-16|2035|8851|679|2069|5|8|16|3|2069|679|8851|Friday|2069Q3|N|Y|N|2476960|2477171|2476610|2476884|N|N|N|N|N| +2476976|AAAAAAAAALLMFCAA|2069-08-17|2035|8851|679|2069|6|8|17|3|2069|679|8851|Saturday|2069Q3|N|Y|N|2476960|2477171|2476611|2476885|N|N|N|N|N| +2476977|AAAAAAAABLLMFCAA|2069-08-18|2035|8851|679|2069|0|8|18|3|2069|679|8851|Sunday|2069Q3|N|N|N|2476960|2477171|2476612|2476886|N|N|N|N|N| +2476978|AAAAAAAACLLMFCAA|2069-08-19|2035|8851|679|2069|1|8|19|3|2069|679|8851|Monday|2069Q3|N|N|N|2476960|2477171|2476613|2476887|N|N|N|N|N| +2476979|AAAAAAAADLLMFCAA|2069-08-20|2035|8852|679|2069|2|8|20|3|2069|679|8852|Tuesday|2069Q3|N|N|N|2476960|2477171|2476614|2476888|N|N|N|N|N| +2476980|AAAAAAAAELLMFCAA|2069-08-21|2035|8852|679|2069|3|8|21|3|2069|679|8852|Wednesday|2069Q3|N|N|N|2476960|2477171|2476615|2476889|N|N|N|N|N| +2476981|AAAAAAAAFLLMFCAA|2069-08-22|2035|8852|679|2069|4|8|22|3|2069|679|8852|Thursday|2069Q3|N|N|N|2476960|2477171|2476616|2476890|N|N|N|N|N| +2476982|AAAAAAAAGLLMFCAA|2069-08-23|2035|8852|679|2069|5|8|23|3|2069|679|8852|Friday|2069Q3|N|Y|N|2476960|2477171|2476617|2476891|N|N|N|N|N| +2476983|AAAAAAAAHLLMFCAA|2069-08-24|2035|8852|679|2069|6|8|24|3|2069|679|8852|Saturday|2069Q3|N|Y|N|2476960|2477171|2476618|2476892|N|N|N|N|N| +2476984|AAAAAAAAILLMFCAA|2069-08-25|2035|8852|679|2069|0|8|25|3|2069|679|8852|Sunday|2069Q3|N|N|N|2476960|2477171|2476619|2476893|N|N|N|N|N| +2476985|AAAAAAAAJLLMFCAA|2069-08-26|2035|8852|679|2069|1|8|26|3|2069|679|8852|Monday|2069Q3|N|N|N|2476960|2477171|2476620|2476894|N|N|N|N|N| +2476986|AAAAAAAAKLLMFCAA|2069-08-27|2035|8853|679|2069|2|8|27|3|2069|679|8853|Tuesday|2069Q3|N|N|N|2476960|2477171|2476621|2476895|N|N|N|N|N| +2476987|AAAAAAAALLLMFCAA|2069-08-28|2035|8853|679|2069|3|8|28|3|2069|679|8853|Wednesday|2069Q3|N|N|N|2476960|2477171|2476622|2476896|N|N|N|N|N| +2476988|AAAAAAAAMLLMFCAA|2069-08-29|2035|8853|679|2069|4|8|29|3|2069|679|8853|Thursday|2069Q3|N|N|N|2476960|2477171|2476623|2476897|N|N|N|N|N| +2476989|AAAAAAAANLLMFCAA|2069-08-30|2035|8853|679|2069|5|8|30|3|2069|679|8853|Friday|2069Q3|N|Y|N|2476960|2477171|2476624|2476898|N|N|N|N|N| +2476990|AAAAAAAAOLLMFCAA|2069-08-31|2035|8853|679|2069|6|8|31|3|2069|679|8853|Saturday|2069Q3|N|Y|N|2476960|2477171|2476625|2476899|N|N|N|N|N| +2476991|AAAAAAAAPLLMFCAA|2069-09-01|2036|8853|680|2069|0|9|1|3|2069|680|8853|Sunday|2069Q3|N|N|N|2476991|2477233|2476626|2476900|N|N|N|N|N| +2476992|AAAAAAAAAMLMFCAA|2069-09-02|2036|8853|680|2069|1|9|2|3|2069|680|8853|Monday|2069Q3|N|N|N|2476991|2477233|2476627|2476901|N|N|N|N|N| +2476993|AAAAAAAABMLMFCAA|2069-09-03|2036|8854|680|2069|2|9|3|3|2069|680|8854|Tuesday|2069Q3|N|N|N|2476991|2477233|2476628|2476902|N|N|N|N|N| +2476994|AAAAAAAACMLMFCAA|2069-09-04|2036|8854|680|2069|3|9|4|3|2069|680|8854|Wednesday|2069Q3|N|N|N|2476991|2477233|2476629|2476903|N|N|N|N|N| +2476995|AAAAAAAADMLMFCAA|2069-09-05|2036|8854|680|2069|4|9|5|3|2069|680|8854|Thursday|2069Q3|N|N|N|2476991|2477233|2476630|2476904|N|N|N|N|N| +2476996|AAAAAAAAEMLMFCAA|2069-09-06|2036|8854|680|2069|5|9|6|3|2069|680|8854|Friday|2069Q3|N|Y|N|2476991|2477233|2476631|2476905|N|N|N|N|N| +2476997|AAAAAAAAFMLMFCAA|2069-09-07|2036|8854|680|2069|6|9|7|3|2069|680|8854|Saturday|2069Q3|N|Y|N|2476991|2477233|2476632|2476906|N|N|N|N|N| +2476998|AAAAAAAAGMLMFCAA|2069-09-08|2036|8854|680|2069|0|9|8|3|2069|680|8854|Sunday|2069Q3|N|N|N|2476991|2477233|2476633|2476907|N|N|N|N|N| +2476999|AAAAAAAAHMLMFCAA|2069-09-09|2036|8854|680|2069|1|9|9|3|2069|680|8854|Monday|2069Q3|N|N|N|2476991|2477233|2476634|2476908|N|N|N|N|N| +2477000|AAAAAAAAIMLMFCAA|2069-09-10|2036|8855|680|2069|2|9|10|3|2069|680|8855|Tuesday|2069Q3|N|N|N|2476991|2477233|2476635|2476909|N|N|N|N|N| +2477001|AAAAAAAAJMLMFCAA|2069-09-11|2036|8855|680|2069|3|9|11|3|2069|680|8855|Wednesday|2069Q3|N|N|N|2476991|2477233|2476636|2476910|N|N|N|N|N| +2477002|AAAAAAAAKMLMFCAA|2069-09-12|2036|8855|680|2069|4|9|12|3|2069|680|8855|Thursday|2069Q3|N|N|N|2476991|2477233|2476637|2476911|N|N|N|N|N| +2477003|AAAAAAAALMLMFCAA|2069-09-13|2036|8855|680|2069|5|9|13|3|2069|680|8855|Friday|2069Q3|N|Y|N|2476991|2477233|2476638|2476912|N|N|N|N|N| +2477004|AAAAAAAAMMLMFCAA|2069-09-14|2036|8855|680|2069|6|9|14|3|2069|680|8855|Saturday|2069Q3|N|Y|N|2476991|2477233|2476639|2476913|N|N|N|N|N| +2477005|AAAAAAAANMLMFCAA|2069-09-15|2036|8855|680|2069|0|9|15|3|2069|680|8855|Sunday|2069Q3|N|N|N|2476991|2477233|2476640|2476914|N|N|N|N|N| +2477006|AAAAAAAAOMLMFCAA|2069-09-16|2036|8855|680|2069|1|9|16|3|2069|680|8855|Monday|2069Q3|N|N|N|2476991|2477233|2476641|2476915|N|N|N|N|N| +2477007|AAAAAAAAPMLMFCAA|2069-09-17|2036|8856|680|2069|2|9|17|3|2069|680|8856|Tuesday|2069Q3|N|N|N|2476991|2477233|2476642|2476916|N|N|N|N|N| +2477008|AAAAAAAAANLMFCAA|2069-09-18|2036|8856|680|2069|3|9|18|3|2069|680|8856|Wednesday|2069Q3|N|N|N|2476991|2477233|2476643|2476917|N|N|N|N|N| +2477009|AAAAAAAABNLMFCAA|2069-09-19|2036|8856|680|2069|4|9|19|3|2069|680|8856|Thursday|2069Q3|N|N|N|2476991|2477233|2476644|2476918|N|N|N|N|N| +2477010|AAAAAAAACNLMFCAA|2069-09-20|2036|8856|680|2069|5|9|20|3|2069|680|8856|Friday|2069Q3|N|Y|N|2476991|2477233|2476645|2476919|N|N|N|N|N| +2477011|AAAAAAAADNLMFCAA|2069-09-21|2036|8856|680|2069|6|9|21|3|2069|680|8856|Saturday|2069Q3|N|Y|N|2476991|2477233|2476646|2476920|N|N|N|N|N| +2477012|AAAAAAAAENLMFCAA|2069-09-22|2036|8856|680|2069|0|9|22|3|2069|680|8856|Sunday|2069Q3|N|N|N|2476991|2477233|2476647|2476921|N|N|N|N|N| +2477013|AAAAAAAAFNLMFCAA|2069-09-23|2036|8856|680|2069|1|9|23|3|2069|680|8856|Monday|2069Q3|N|N|N|2476991|2477233|2476648|2476922|N|N|N|N|N| +2477014|AAAAAAAAGNLMFCAA|2069-09-24|2036|8857|680|2069|2|9|24|3|2069|680|8857|Tuesday|2069Q3|N|N|N|2476991|2477233|2476649|2476923|N|N|N|N|N| +2477015|AAAAAAAAHNLMFCAA|2069-09-25|2036|8857|680|2069|3|9|25|3|2069|680|8857|Wednesday|2069Q3|N|N|N|2476991|2477233|2476650|2476924|N|N|N|N|N| +2477016|AAAAAAAAINLMFCAA|2069-09-26|2036|8857|680|2069|4|9|26|3|2069|680|8857|Thursday|2069Q3|N|N|N|2476991|2477233|2476651|2476925|N|N|N|N|N| +2477017|AAAAAAAAJNLMFCAA|2069-09-27|2036|8857|680|2069|5|9|27|3|2069|680|8857|Friday|2069Q3|N|Y|N|2476991|2477233|2476652|2476926|N|N|N|N|N| +2477018|AAAAAAAAKNLMFCAA|2069-09-28|2036|8857|680|2069|6|9|28|3|2069|680|8857|Saturday|2069Q3|N|Y|N|2476991|2477233|2476653|2476927|N|N|N|N|N| +2477019|AAAAAAAALNLMFCAA|2069-09-29|2036|8857|680|2069|0|9|29|3|2069|680|8857|Sunday|2069Q3|N|N|N|2476991|2477233|2476654|2476928|N|N|N|N|N| +2477020|AAAAAAAAMNLMFCAA|2069-09-30|2036|8857|680|2069|1|9|30|3|2069|680|8857|Monday|2069Q3|N|N|N|2476991|2477233|2476655|2476929|N|N|N|N|N| +2477021|AAAAAAAANNLMFCAA|2069-10-01|2037|8858|680|2069|2|10|1|3|2069|680|8858|Tuesday|2069Q3|N|N|N|2477021|2477293|2476656|2476929|N|N|N|N|N| +2477022|AAAAAAAAONLMFCAA|2069-10-02|2037|8858|680|2069|3|10|2|4|2069|680|8858|Wednesday|2069Q4|N|N|N|2477021|2477293|2476657|2476930|N|N|N|N|N| +2477023|AAAAAAAAPNLMFCAA|2069-10-03|2037|8858|680|2069|4|10|3|4|2069|680|8858|Thursday|2069Q4|N|N|N|2477021|2477293|2476658|2476931|N|N|N|N|N| +2477024|AAAAAAAAAOLMFCAA|2069-10-04|2037|8858|680|2069|5|10|4|4|2069|680|8858|Friday|2069Q4|N|Y|N|2477021|2477293|2476659|2476932|N|N|N|N|N| +2477025|AAAAAAAABOLMFCAA|2069-10-05|2037|8858|680|2069|6|10|5|4|2069|680|8858|Saturday|2069Q4|N|Y|N|2477021|2477293|2476660|2476933|N|N|N|N|N| +2477026|AAAAAAAACOLMFCAA|2069-10-06|2037|8858|680|2069|0|10|6|4|2069|680|8858|Sunday|2069Q4|N|N|N|2477021|2477293|2476661|2476934|N|N|N|N|N| +2477027|AAAAAAAADOLMFCAA|2069-10-07|2037|8858|680|2069|1|10|7|4|2069|680|8858|Monday|2069Q4|N|N|N|2477021|2477293|2476662|2476935|N|N|N|N|N| +2477028|AAAAAAAAEOLMFCAA|2069-10-08|2037|8859|680|2069|2|10|8|4|2069|680|8859|Tuesday|2069Q4|N|N|N|2477021|2477293|2476663|2476936|N|N|N|N|N| +2477029|AAAAAAAAFOLMFCAA|2069-10-09|2037|8859|680|2069|3|10|9|4|2069|680|8859|Wednesday|2069Q4|N|N|N|2477021|2477293|2476664|2476937|N|N|N|N|N| +2477030|AAAAAAAAGOLMFCAA|2069-10-10|2037|8859|680|2069|4|10|10|4|2069|680|8859|Thursday|2069Q4|N|N|N|2477021|2477293|2476665|2476938|N|N|N|N|N| +2477031|AAAAAAAAHOLMFCAA|2069-10-11|2037|8859|680|2069|5|10|11|4|2069|680|8859|Friday|2069Q4|N|Y|N|2477021|2477293|2476666|2476939|N|N|N|N|N| +2477032|AAAAAAAAIOLMFCAA|2069-10-12|2037|8859|680|2069|6|10|12|4|2069|680|8859|Saturday|2069Q4|N|Y|N|2477021|2477293|2476667|2476940|N|N|N|N|N| +2477033|AAAAAAAAJOLMFCAA|2069-10-13|2037|8859|680|2069|0|10|13|4|2069|680|8859|Sunday|2069Q4|N|N|N|2477021|2477293|2476668|2476941|N|N|N|N|N| +2477034|AAAAAAAAKOLMFCAA|2069-10-14|2037|8859|680|2069|1|10|14|4|2069|680|8859|Monday|2069Q4|N|N|N|2477021|2477293|2476669|2476942|N|N|N|N|N| +2477035|AAAAAAAALOLMFCAA|2069-10-15|2037|8860|680|2069|2|10|15|4|2069|680|8860|Tuesday|2069Q4|N|N|N|2477021|2477293|2476670|2476943|N|N|N|N|N| +2477036|AAAAAAAAMOLMFCAA|2069-10-16|2037|8860|680|2069|3|10|16|4|2069|680|8860|Wednesday|2069Q4|N|N|N|2477021|2477293|2476671|2476944|N|N|N|N|N| +2477037|AAAAAAAANOLMFCAA|2069-10-17|2037|8860|680|2069|4|10|17|4|2069|680|8860|Thursday|2069Q4|N|N|N|2477021|2477293|2476672|2476945|N|N|N|N|N| +2477038|AAAAAAAAOOLMFCAA|2069-10-18|2037|8860|680|2069|5|10|18|4|2069|680|8860|Friday|2069Q4|N|Y|N|2477021|2477293|2476673|2476946|N|N|N|N|N| +2477039|AAAAAAAAPOLMFCAA|2069-10-19|2037|8860|680|2069|6|10|19|4|2069|680|8860|Saturday|2069Q4|N|Y|N|2477021|2477293|2476674|2476947|N|N|N|N|N| +2477040|AAAAAAAAAPLMFCAA|2069-10-20|2037|8860|680|2069|0|10|20|4|2069|680|8860|Sunday|2069Q4|N|N|N|2477021|2477293|2476675|2476948|N|N|N|N|N| +2477041|AAAAAAAABPLMFCAA|2069-10-21|2037|8860|680|2069|1|10|21|4|2069|680|8860|Monday|2069Q4|N|N|N|2477021|2477293|2476676|2476949|N|N|N|N|N| +2477042|AAAAAAAACPLMFCAA|2069-10-22|2037|8861|680|2069|2|10|22|4|2069|680|8861|Tuesday|2069Q4|N|N|N|2477021|2477293|2476677|2476950|N|N|N|N|N| +2477043|AAAAAAAADPLMFCAA|2069-10-23|2037|8861|680|2069|3|10|23|4|2069|680|8861|Wednesday|2069Q4|N|N|N|2477021|2477293|2476678|2476951|N|N|N|N|N| +2477044|AAAAAAAAEPLMFCAA|2069-10-24|2037|8861|680|2069|4|10|24|4|2069|680|8861|Thursday|2069Q4|N|N|N|2477021|2477293|2476679|2476952|N|N|N|N|N| +2477045|AAAAAAAAFPLMFCAA|2069-10-25|2037|8861|680|2069|5|10|25|4|2069|680|8861|Friday|2069Q4|N|Y|N|2477021|2477293|2476680|2476953|N|N|N|N|N| +2477046|AAAAAAAAGPLMFCAA|2069-10-26|2037|8861|680|2069|6|10|26|4|2069|680|8861|Saturday|2069Q4|N|Y|N|2477021|2477293|2476681|2476954|N|N|N|N|N| +2477047|AAAAAAAAHPLMFCAA|2069-10-27|2037|8861|680|2069|0|10|27|4|2069|680|8861|Sunday|2069Q4|N|N|N|2477021|2477293|2476682|2476955|N|N|N|N|N| +2477048|AAAAAAAAIPLMFCAA|2069-10-28|2037|8861|680|2069|1|10|28|4|2069|680|8861|Monday|2069Q4|N|N|N|2477021|2477293|2476683|2476956|N|N|N|N|N| +2477049|AAAAAAAAJPLMFCAA|2069-10-29|2037|8862|680|2069|2|10|29|4|2069|680|8862|Tuesday|2069Q4|N|N|N|2477021|2477293|2476684|2476957|N|N|N|N|N| +2477050|AAAAAAAAKPLMFCAA|2069-10-30|2037|8862|680|2069|3|10|30|4|2069|680|8862|Wednesday|2069Q4|N|N|N|2477021|2477293|2476685|2476958|N|N|N|N|N| +2477051|AAAAAAAALPLMFCAA|2069-10-31|2037|8862|680|2069|4|10|31|4|2069|680|8862|Thursday|2069Q4|N|N|N|2477021|2477293|2476686|2476959|N|N|N|N|N| +2477052|AAAAAAAAMPLMFCAA|2069-11-01|2038|8862|680|2069|5|11|1|4|2069|680|8862|Friday|2069Q4|N|Y|N|2477052|2477355|2476687|2476960|N|N|N|N|N| +2477053|AAAAAAAANPLMFCAA|2069-11-02|2038|8862|680|2069|6|11|2|4|2069|680|8862|Saturday|2069Q4|N|Y|N|2477052|2477355|2476688|2476961|N|N|N|N|N| +2477054|AAAAAAAAOPLMFCAA|2069-11-03|2038|8862|680|2069|0|11|3|4|2069|680|8862|Sunday|2069Q4|N|N|N|2477052|2477355|2476689|2476962|N|N|N|N|N| +2477055|AAAAAAAAPPLMFCAA|2069-11-04|2038|8862|680|2069|1|11|4|4|2069|680|8862|Monday|2069Q4|N|N|N|2477052|2477355|2476690|2476963|N|N|N|N|N| +2477056|AAAAAAAAAAMMFCAA|2069-11-05|2038|8863|680|2069|2|11|5|4|2069|680|8863|Tuesday|2069Q4|N|N|N|2477052|2477355|2476691|2476964|N|N|N|N|N| +2477057|AAAAAAAABAMMFCAA|2069-11-06|2038|8863|680|2069|3|11|6|4|2069|680|8863|Wednesday|2069Q4|N|N|N|2477052|2477355|2476692|2476965|N|N|N|N|N| +2477058|AAAAAAAACAMMFCAA|2069-11-07|2038|8863|680|2069|4|11|7|4|2069|680|8863|Thursday|2069Q4|N|N|N|2477052|2477355|2476693|2476966|N|N|N|N|N| +2477059|AAAAAAAADAMMFCAA|2069-11-08|2038|8863|680|2069|5|11|8|4|2069|680|8863|Friday|2069Q4|N|Y|N|2477052|2477355|2476694|2476967|N|N|N|N|N| +2477060|AAAAAAAAEAMMFCAA|2069-11-09|2038|8863|680|2069|6|11|9|4|2069|680|8863|Saturday|2069Q4|N|Y|N|2477052|2477355|2476695|2476968|N|N|N|N|N| +2477061|AAAAAAAAFAMMFCAA|2069-11-10|2038|8863|680|2069|0|11|10|4|2069|680|8863|Sunday|2069Q4|N|N|N|2477052|2477355|2476696|2476969|N|N|N|N|N| +2477062|AAAAAAAAGAMMFCAA|2069-11-11|2038|8863|680|2069|1|11|11|4|2069|680|8863|Monday|2069Q4|N|N|N|2477052|2477355|2476697|2476970|N|N|N|N|N| +2477063|AAAAAAAAHAMMFCAA|2069-11-12|2038|8864|680|2069|2|11|12|4|2069|680|8864|Tuesday|2069Q4|N|N|N|2477052|2477355|2476698|2476971|N|N|N|N|N| +2477064|AAAAAAAAIAMMFCAA|2069-11-13|2038|8864|680|2069|3|11|13|4|2069|680|8864|Wednesday|2069Q4|N|N|N|2477052|2477355|2476699|2476972|N|N|N|N|N| +2477065|AAAAAAAAJAMMFCAA|2069-11-14|2038|8864|680|2069|4|11|14|4|2069|680|8864|Thursday|2069Q4|N|N|N|2477052|2477355|2476700|2476973|N|N|N|N|N| +2477066|AAAAAAAAKAMMFCAA|2069-11-15|2038|8864|680|2069|5|11|15|4|2069|680|8864|Friday|2069Q4|N|Y|N|2477052|2477355|2476701|2476974|N|N|N|N|N| +2477067|AAAAAAAALAMMFCAA|2069-11-16|2038|8864|680|2069|6|11|16|4|2069|680|8864|Saturday|2069Q4|N|Y|N|2477052|2477355|2476702|2476975|N|N|N|N|N| +2477068|AAAAAAAAMAMMFCAA|2069-11-17|2038|8864|680|2069|0|11|17|4|2069|680|8864|Sunday|2069Q4|N|N|N|2477052|2477355|2476703|2476976|N|N|N|N|N| +2477069|AAAAAAAANAMMFCAA|2069-11-18|2038|8864|680|2069|1|11|18|4|2069|680|8864|Monday|2069Q4|N|N|N|2477052|2477355|2476704|2476977|N|N|N|N|N| +2477070|AAAAAAAAOAMMFCAA|2069-11-19|2038|8865|680|2069|2|11|19|4|2069|680|8865|Tuesday|2069Q4|N|N|N|2477052|2477355|2476705|2476978|N|N|N|N|N| +2477071|AAAAAAAAPAMMFCAA|2069-11-20|2038|8865|680|2069|3|11|20|4|2069|680|8865|Wednesday|2069Q4|N|N|N|2477052|2477355|2476706|2476979|N|N|N|N|N| +2477072|AAAAAAAAABMMFCAA|2069-11-21|2038|8865|680|2069|4|11|21|4|2069|680|8865|Thursday|2069Q4|N|N|N|2477052|2477355|2476707|2476980|N|N|N|N|N| +2477073|AAAAAAAABBMMFCAA|2069-11-22|2038|8865|680|2069|5|11|22|4|2069|680|8865|Friday|2069Q4|N|Y|N|2477052|2477355|2476708|2476981|N|N|N|N|N| +2477074|AAAAAAAACBMMFCAA|2069-11-23|2038|8865|680|2069|6|11|23|4|2069|680|8865|Saturday|2069Q4|N|Y|N|2477052|2477355|2476709|2476982|N|N|N|N|N| +2477075|AAAAAAAADBMMFCAA|2069-11-24|2038|8865|680|2069|0|11|24|4|2069|680|8865|Sunday|2069Q4|N|N|N|2477052|2477355|2476710|2476983|N|N|N|N|N| +2477076|AAAAAAAAEBMMFCAA|2069-11-25|2038|8865|680|2069|1|11|25|4|2069|680|8865|Monday|2069Q4|N|N|N|2477052|2477355|2476711|2476984|N|N|N|N|N| +2477077|AAAAAAAAFBMMFCAA|2069-11-26|2038|8866|680|2069|2|11|26|4|2069|680|8866|Tuesday|2069Q4|N|N|N|2477052|2477355|2476712|2476985|N|N|N|N|N| +2477078|AAAAAAAAGBMMFCAA|2069-11-27|2038|8866|680|2069|3|11|27|4|2069|680|8866|Wednesday|2069Q4|N|N|N|2477052|2477355|2476713|2476986|N|N|N|N|N| +2477079|AAAAAAAAHBMMFCAA|2069-11-28|2038|8866|680|2069|4|11|28|4|2069|680|8866|Thursday|2069Q4|N|N|N|2477052|2477355|2476714|2476987|N|N|N|N|N| +2477080|AAAAAAAAIBMMFCAA|2069-11-29|2038|8866|680|2069|5|11|29|4|2069|680|8866|Friday|2069Q4|N|Y|N|2477052|2477355|2476715|2476988|N|N|N|N|N| +2477081|AAAAAAAAJBMMFCAA|2069-11-30|2038|8866|680|2069|6|11|30|4|2069|680|8866|Saturday|2069Q4|N|Y|N|2477052|2477355|2476716|2476989|N|N|N|N|N| +2477082|AAAAAAAAKBMMFCAA|2069-12-01|2039|8866|681|2069|0|12|1|4|2069|681|8866|Sunday|2069Q4|N|N|N|2477082|2477415|2476717|2476990|N|N|N|N|N| +2477083|AAAAAAAALBMMFCAA|2069-12-02|2039|8866|681|2069|1|12|2|4|2069|681|8866|Monday|2069Q4|N|N|N|2477082|2477415|2476718|2476991|N|N|N|N|N| +2477084|AAAAAAAAMBMMFCAA|2069-12-03|2039|8867|681|2069|2|12|3|4|2069|681|8867|Tuesday|2069Q4|N|N|N|2477082|2477415|2476719|2476992|N|N|N|N|N| +2477085|AAAAAAAANBMMFCAA|2069-12-04|2039|8867|681|2069|3|12|4|4|2069|681|8867|Wednesday|2069Q4|N|N|N|2477082|2477415|2476720|2476993|N|N|N|N|N| +2477086|AAAAAAAAOBMMFCAA|2069-12-05|2039|8867|681|2069|4|12|5|4|2069|681|8867|Thursday|2069Q4|N|N|N|2477082|2477415|2476721|2476994|N|N|N|N|N| +2477087|AAAAAAAAPBMMFCAA|2069-12-06|2039|8867|681|2069|5|12|6|4|2069|681|8867|Friday|2069Q4|N|Y|N|2477082|2477415|2476722|2476995|N|N|N|N|N| +2477088|AAAAAAAAACMMFCAA|2069-12-07|2039|8867|681|2069|6|12|7|4|2069|681|8867|Saturday|2069Q4|N|Y|N|2477082|2477415|2476723|2476996|N|N|N|N|N| +2477089|AAAAAAAABCMMFCAA|2069-12-08|2039|8867|681|2069|0|12|8|4|2069|681|8867|Sunday|2069Q4|N|N|N|2477082|2477415|2476724|2476997|N|N|N|N|N| +2477090|AAAAAAAACCMMFCAA|2069-12-09|2039|8867|681|2069|1|12|9|4|2069|681|8867|Monday|2069Q4|N|N|N|2477082|2477415|2476725|2476998|N|N|N|N|N| +2477091|AAAAAAAADCMMFCAA|2069-12-10|2039|8868|681|2069|2|12|10|4|2069|681|8868|Tuesday|2069Q4|N|N|N|2477082|2477415|2476726|2476999|N|N|N|N|N| +2477092|AAAAAAAAECMMFCAA|2069-12-11|2039|8868|681|2069|3|12|11|4|2069|681|8868|Wednesday|2069Q4|N|N|N|2477082|2477415|2476727|2477000|N|N|N|N|N| +2477093|AAAAAAAAFCMMFCAA|2069-12-12|2039|8868|681|2069|4|12|12|4|2069|681|8868|Thursday|2069Q4|N|N|N|2477082|2477415|2476728|2477001|N|N|N|N|N| +2477094|AAAAAAAAGCMMFCAA|2069-12-13|2039|8868|681|2069|5|12|13|4|2069|681|8868|Friday|2069Q4|N|Y|N|2477082|2477415|2476729|2477002|N|N|N|N|N| +2477095|AAAAAAAAHCMMFCAA|2069-12-14|2039|8868|681|2069|6|12|14|4|2069|681|8868|Saturday|2069Q4|N|Y|N|2477082|2477415|2476730|2477003|N|N|N|N|N| +2477096|AAAAAAAAICMMFCAA|2069-12-15|2039|8868|681|2069|0|12|15|4|2069|681|8868|Sunday|2069Q4|N|N|N|2477082|2477415|2476731|2477004|N|N|N|N|N| +2477097|AAAAAAAAJCMMFCAA|2069-12-16|2039|8868|681|2069|1|12|16|4|2069|681|8868|Monday|2069Q4|N|N|N|2477082|2477415|2476732|2477005|N|N|N|N|N| +2477098|AAAAAAAAKCMMFCAA|2069-12-17|2039|8869|681|2069|2|12|17|4|2069|681|8869|Tuesday|2069Q4|N|N|N|2477082|2477415|2476733|2477006|N|N|N|N|N| +2477099|AAAAAAAALCMMFCAA|2069-12-18|2039|8869|681|2069|3|12|18|4|2069|681|8869|Wednesday|2069Q4|N|N|N|2477082|2477415|2476734|2477007|N|N|N|N|N| +2477100|AAAAAAAAMCMMFCAA|2069-12-19|2039|8869|681|2069|4|12|19|4|2069|681|8869|Thursday|2069Q4|N|N|N|2477082|2477415|2476735|2477008|N|N|N|N|N| +2477101|AAAAAAAANCMMFCAA|2069-12-20|2039|8869|681|2069|5|12|20|4|2069|681|8869|Friday|2069Q4|N|Y|N|2477082|2477415|2476736|2477009|N|N|N|N|N| +2477102|AAAAAAAAOCMMFCAA|2069-12-21|2039|8869|681|2069|6|12|21|4|2069|681|8869|Saturday|2069Q4|N|Y|N|2477082|2477415|2476737|2477010|N|N|N|N|N| +2477103|AAAAAAAAPCMMFCAA|2069-12-22|2039|8869|681|2069|0|12|22|4|2069|681|8869|Sunday|2069Q4|N|N|N|2477082|2477415|2476738|2477011|N|N|N|N|N| +2477104|AAAAAAAAADMMFCAA|2069-12-23|2039|8869|681|2069|1|12|23|4|2069|681|8869|Monday|2069Q4|N|N|N|2477082|2477415|2476739|2477012|N|N|N|N|N| +2477105|AAAAAAAABDMMFCAA|2069-12-24|2039|8870|681|2069|2|12|24|4|2069|681|8870|Tuesday|2069Q4|N|N|N|2477082|2477415|2476740|2477013|N|N|N|N|N| +2477106|AAAAAAAACDMMFCAA|2069-12-25|2039|8870|681|2069|3|12|25|4|2069|681|8870|Wednesday|2069Q4|N|N|N|2477082|2477415|2476741|2477014|N|N|N|N|N| +2477107|AAAAAAAADDMMFCAA|2069-12-26|2039|8870|681|2069|4|12|26|4|2069|681|8870|Thursday|2069Q4|Y|N|N|2477082|2477415|2476742|2477015|N|N|N|N|N| +2477108|AAAAAAAAEDMMFCAA|2069-12-27|2039|8870|681|2069|5|12|27|4|2069|681|8870|Friday|2069Q4|N|Y|Y|2477082|2477415|2476743|2477016|N|N|N|N|N| +2477109|AAAAAAAAFDMMFCAA|2069-12-28|2039|8870|681|2069|6|12|28|4|2069|681|8870|Saturday|2069Q4|N|Y|N|2477082|2477415|2476744|2477017|N|N|N|N|N| +2477110|AAAAAAAAGDMMFCAA|2069-12-29|2039|8870|681|2069|0|12|29|4|2069|681|8870|Sunday|2069Q4|N|N|N|2477082|2477415|2476745|2477018|N|N|N|N|N| +2477111|AAAAAAAAHDMMFCAA|2069-12-30|2039|8870|681|2069|1|12|30|4|2069|681|8870|Monday|2069Q4|N|N|N|2477082|2477415|2476746|2477019|N|N|N|N|N| +2477112|AAAAAAAAIDMMFCAA|2069-12-31|2039|8871|681|2069|2|12|31|4|2069|681|8871|Tuesday|2069Q4|N|N|N|2477082|2477415|2476747|2477020|N|N|N|N|N| +2477113|AAAAAAAAJDMMFCAA|2070-01-01|2040|8871|681|2070|3|1|1|1|2070|681|8871|Wednesday|2070Q1|Y|N|N|2477113|2477112|2476748|2477021|N|N|N|N|N| +2477114|AAAAAAAAKDMMFCAA|2070-01-02|2040|8871|681|2070|4|1|2|1|2070|681|8871|Thursday|2070Q1|N|N|Y|2477113|2477112|2476749|2477022|N|N|N|N|N| +2477115|AAAAAAAALDMMFCAA|2070-01-03|2040|8871|681|2070|5|1|3|1|2070|681|8871|Friday|2070Q1|N|Y|N|2477113|2477112|2476750|2477023|N|N|N|N|N| +2477116|AAAAAAAAMDMMFCAA|2070-01-04|2040|8871|681|2070|6|1|4|1|2070|681|8871|Saturday|2070Q1|N|Y|N|2477113|2477112|2476751|2477024|N|N|N|N|N| +2477117|AAAAAAAANDMMFCAA|2070-01-05|2040|8871|681|2070|0|1|5|1|2070|681|8871|Sunday|2070Q1|N|N|N|2477113|2477112|2476752|2477025|N|N|N|N|N| +2477118|AAAAAAAAODMMFCAA|2070-01-06|2040|8871|681|2070|1|1|6|1|2070|681|8871|Monday|2070Q1|N|N|N|2477113|2477112|2476753|2477026|N|N|N|N|N| +2477119|AAAAAAAAPDMMFCAA|2070-01-07|2040|8872|681|2070|2|1|7|1|2070|681|8872|Tuesday|2070Q1|N|N|N|2477113|2477112|2476754|2477027|N|N|N|N|N| +2477120|AAAAAAAAAEMMFCAA|2070-01-08|2040|8872|681|2070|3|1|8|1|2070|681|8872|Wednesday|2070Q1|N|N|N|2477113|2477112|2476755|2477028|N|N|N|N|N| +2477121|AAAAAAAABEMMFCAA|2070-01-09|2040|8872|681|2070|4|1|9|1|2070|681|8872|Thursday|2070Q1|N|N|N|2477113|2477112|2476756|2477029|N|N|N|N|N| +2477122|AAAAAAAACEMMFCAA|2070-01-10|2040|8872|681|2070|5|1|10|1|2070|681|8872|Friday|2070Q1|N|Y|N|2477113|2477112|2476757|2477030|N|N|N|N|N| +2477123|AAAAAAAADEMMFCAA|2070-01-11|2040|8872|681|2070|6|1|11|1|2070|681|8872|Saturday|2070Q1|N|Y|N|2477113|2477112|2476758|2477031|N|N|N|N|N| +2477124|AAAAAAAAEEMMFCAA|2070-01-12|2040|8872|681|2070|0|1|12|1|2070|681|8872|Sunday|2070Q1|N|N|N|2477113|2477112|2476759|2477032|N|N|N|N|N| +2477125|AAAAAAAAFEMMFCAA|2070-01-13|2040|8872|681|2070|1|1|13|1|2070|681|8872|Monday|2070Q1|N|N|N|2477113|2477112|2476760|2477033|N|N|N|N|N| +2477126|AAAAAAAAGEMMFCAA|2070-01-14|2040|8873|681|2070|2|1|14|1|2070|681|8873|Tuesday|2070Q1|N|N|N|2477113|2477112|2476761|2477034|N|N|N|N|N| +2477127|AAAAAAAAHEMMFCAA|2070-01-15|2040|8873|681|2070|3|1|15|1|2070|681|8873|Wednesday|2070Q1|N|N|N|2477113|2477112|2476762|2477035|N|N|N|N|N| +2477128|AAAAAAAAIEMMFCAA|2070-01-16|2040|8873|681|2070|4|1|16|1|2070|681|8873|Thursday|2070Q1|N|N|N|2477113|2477112|2476763|2477036|N|N|N|N|N| +2477129|AAAAAAAAJEMMFCAA|2070-01-17|2040|8873|681|2070|5|1|17|1|2070|681|8873|Friday|2070Q1|N|Y|N|2477113|2477112|2476764|2477037|N|N|N|N|N| +2477130|AAAAAAAAKEMMFCAA|2070-01-18|2040|8873|681|2070|6|1|18|1|2070|681|8873|Saturday|2070Q1|N|Y|N|2477113|2477112|2476765|2477038|N|N|N|N|N| +2477131|AAAAAAAALEMMFCAA|2070-01-19|2040|8873|681|2070|0|1|19|1|2070|681|8873|Sunday|2070Q1|N|N|N|2477113|2477112|2476766|2477039|N|N|N|N|N| +2477132|AAAAAAAAMEMMFCAA|2070-01-20|2040|8873|681|2070|1|1|20|1|2070|681|8873|Monday|2070Q1|N|N|N|2477113|2477112|2476767|2477040|N|N|N|N|N| +2477133|AAAAAAAANEMMFCAA|2070-01-21|2040|8874|681|2070|2|1|21|1|2070|681|8874|Tuesday|2070Q1|N|N|N|2477113|2477112|2476768|2477041|N|N|N|N|N| +2477134|AAAAAAAAOEMMFCAA|2070-01-22|2040|8874|681|2070|3|1|22|1|2070|681|8874|Wednesday|2070Q1|N|N|N|2477113|2477112|2476769|2477042|N|N|N|N|N| +2477135|AAAAAAAAPEMMFCAA|2070-01-23|2040|8874|681|2070|4|1|23|1|2070|681|8874|Thursday|2070Q1|N|N|N|2477113|2477112|2476770|2477043|N|N|N|N|N| +2477136|AAAAAAAAAFMMFCAA|2070-01-24|2040|8874|681|2070|5|1|24|1|2070|681|8874|Friday|2070Q1|N|Y|N|2477113|2477112|2476771|2477044|N|N|N|N|N| +2477137|AAAAAAAABFMMFCAA|2070-01-25|2040|8874|681|2070|6|1|25|1|2070|681|8874|Saturday|2070Q1|N|Y|N|2477113|2477112|2476772|2477045|N|N|N|N|N| +2477138|AAAAAAAACFMMFCAA|2070-01-26|2040|8874|681|2070|0|1|26|1|2070|681|8874|Sunday|2070Q1|N|N|N|2477113|2477112|2476773|2477046|N|N|N|N|N| +2477139|AAAAAAAADFMMFCAA|2070-01-27|2040|8874|681|2070|1|1|27|1|2070|681|8874|Monday|2070Q1|N|N|N|2477113|2477112|2476774|2477047|N|N|N|N|N| +2477140|AAAAAAAAEFMMFCAA|2070-01-28|2040|8875|681|2070|2|1|28|1|2070|681|8875|Tuesday|2070Q1|N|N|N|2477113|2477112|2476775|2477048|N|N|N|N|N| +2477141|AAAAAAAAFFMMFCAA|2070-01-29|2040|8875|681|2070|3|1|29|1|2070|681|8875|Wednesday|2070Q1|N|N|N|2477113|2477112|2476776|2477049|N|N|N|N|N| +2477142|AAAAAAAAGFMMFCAA|2070-01-30|2040|8875|681|2070|4|1|30|1|2070|681|8875|Thursday|2070Q1|N|N|N|2477113|2477112|2476777|2477050|N|N|N|N|N| +2477143|AAAAAAAAHFMMFCAA|2070-01-31|2040|8875|681|2070|5|1|31|1|2070|681|8875|Friday|2070Q1|N|Y|N|2477113|2477112|2476778|2477051|N|N|N|N|N| +2477144|AAAAAAAAIFMMFCAA|2070-02-01|2041|8875|681|2070|6|2|1|1|2070|681|8875|Saturday|2070Q1|N|Y|N|2477144|2477174|2476779|2477052|N|N|N|N|N| +2477145|AAAAAAAAJFMMFCAA|2070-02-02|2041|8875|681|2070|0|2|2|1|2070|681|8875|Sunday|2070Q1|N|N|N|2477144|2477174|2476780|2477053|N|N|N|N|N| +2477146|AAAAAAAAKFMMFCAA|2070-02-03|2041|8875|681|2070|1|2|3|1|2070|681|8875|Monday|2070Q1|N|N|N|2477144|2477174|2476781|2477054|N|N|N|N|N| +2477147|AAAAAAAALFMMFCAA|2070-02-04|2041|8876|681|2070|2|2|4|1|2070|681|8876|Tuesday|2070Q1|N|N|N|2477144|2477174|2476782|2477055|N|N|N|N|N| +2477148|AAAAAAAAMFMMFCAA|2070-02-05|2041|8876|681|2070|3|2|5|1|2070|681|8876|Wednesday|2070Q1|N|N|N|2477144|2477174|2476783|2477056|N|N|N|N|N| +2477149|AAAAAAAANFMMFCAA|2070-02-06|2041|8876|681|2070|4|2|6|1|2070|681|8876|Thursday|2070Q1|N|N|N|2477144|2477174|2476784|2477057|N|N|N|N|N| +2477150|AAAAAAAAOFMMFCAA|2070-02-07|2041|8876|681|2070|5|2|7|1|2070|681|8876|Friday|2070Q1|N|Y|N|2477144|2477174|2476785|2477058|N|N|N|N|N| +2477151|AAAAAAAAPFMMFCAA|2070-02-08|2041|8876|681|2070|6|2|8|1|2070|681|8876|Saturday|2070Q1|N|Y|N|2477144|2477174|2476786|2477059|N|N|N|N|N| +2477152|AAAAAAAAAGMMFCAA|2070-02-09|2041|8876|681|2070|0|2|9|1|2070|681|8876|Sunday|2070Q1|N|N|N|2477144|2477174|2476787|2477060|N|N|N|N|N| +2477153|AAAAAAAABGMMFCAA|2070-02-10|2041|8876|681|2070|1|2|10|1|2070|681|8876|Monday|2070Q1|N|N|N|2477144|2477174|2476788|2477061|N|N|N|N|N| +2477154|AAAAAAAACGMMFCAA|2070-02-11|2041|8877|681|2070|2|2|11|1|2070|681|8877|Tuesday|2070Q1|N|N|N|2477144|2477174|2476789|2477062|N|N|N|N|N| +2477155|AAAAAAAADGMMFCAA|2070-02-12|2041|8877|681|2070|3|2|12|1|2070|681|8877|Wednesday|2070Q1|N|N|N|2477144|2477174|2476790|2477063|N|N|N|N|N| +2477156|AAAAAAAAEGMMFCAA|2070-02-13|2041|8877|681|2070|4|2|13|1|2070|681|8877|Thursday|2070Q1|N|N|N|2477144|2477174|2476791|2477064|N|N|N|N|N| +2477157|AAAAAAAAFGMMFCAA|2070-02-14|2041|8877|681|2070|5|2|14|1|2070|681|8877|Friday|2070Q1|N|Y|N|2477144|2477174|2476792|2477065|N|N|N|N|N| +2477158|AAAAAAAAGGMMFCAA|2070-02-15|2041|8877|681|2070|6|2|15|1|2070|681|8877|Saturday|2070Q1|N|Y|N|2477144|2477174|2476793|2477066|N|N|N|N|N| +2477159|AAAAAAAAHGMMFCAA|2070-02-16|2041|8877|681|2070|0|2|16|1|2070|681|8877|Sunday|2070Q1|N|N|N|2477144|2477174|2476794|2477067|N|N|N|N|N| +2477160|AAAAAAAAIGMMFCAA|2070-02-17|2041|8877|681|2070|1|2|17|1|2070|681|8877|Monday|2070Q1|N|N|N|2477144|2477174|2476795|2477068|N|N|N|N|N| +2477161|AAAAAAAAJGMMFCAA|2070-02-18|2041|8878|681|2070|2|2|18|1|2070|681|8878|Tuesday|2070Q1|N|N|N|2477144|2477174|2476796|2477069|N|N|N|N|N| +2477162|AAAAAAAAKGMMFCAA|2070-02-19|2041|8878|681|2070|3|2|19|1|2070|681|8878|Wednesday|2070Q1|N|N|N|2477144|2477174|2476797|2477070|N|N|N|N|N| +2477163|AAAAAAAALGMMFCAA|2070-02-20|2041|8878|681|2070|4|2|20|1|2070|681|8878|Thursday|2070Q1|N|N|N|2477144|2477174|2476798|2477071|N|N|N|N|N| +2477164|AAAAAAAAMGMMFCAA|2070-02-21|2041|8878|681|2070|5|2|21|1|2070|681|8878|Friday|2070Q1|N|Y|N|2477144|2477174|2476799|2477072|N|N|N|N|N| +2477165|AAAAAAAANGMMFCAA|2070-02-22|2041|8878|681|2070|6|2|22|1|2070|681|8878|Saturday|2070Q1|N|Y|N|2477144|2477174|2476800|2477073|N|N|N|N|N| +2477166|AAAAAAAAOGMMFCAA|2070-02-23|2041|8878|681|2070|0|2|23|1|2070|681|8878|Sunday|2070Q1|N|N|N|2477144|2477174|2476801|2477074|N|N|N|N|N| +2477167|AAAAAAAAPGMMFCAA|2070-02-24|2041|8878|681|2070|1|2|24|1|2070|681|8878|Monday|2070Q1|N|N|N|2477144|2477174|2476802|2477075|N|N|N|N|N| +2477168|AAAAAAAAAHMMFCAA|2070-02-25|2041|8879|681|2070|2|2|25|1|2070|681|8879|Tuesday|2070Q1|N|N|N|2477144|2477174|2476803|2477076|N|N|N|N|N| +2477169|AAAAAAAABHMMFCAA|2070-02-26|2041|8879|681|2070|3|2|26|1|2070|681|8879|Wednesday|2070Q1|N|N|N|2477144|2477174|2476804|2477077|N|N|N|N|N| +2477170|AAAAAAAACHMMFCAA|2070-02-27|2041|8879|681|2070|4|2|27|1|2070|681|8879|Thursday|2070Q1|N|N|N|2477144|2477174|2476805|2477078|N|N|N|N|N| +2477171|AAAAAAAADHMMFCAA|2070-02-28|2041|8879|681|2070|5|2|28|1|2070|681|8879|Friday|2070Q1|N|Y|N|2477144|2477174|2476806|2477079|N|N|N|N|N| +2477172|AAAAAAAAEHMMFCAA|2070-03-01|2042|8879|682|2070|6|3|1|1|2070|682|8879|Saturday|2070Q1|N|Y|N|2477172|2477230|2476807|2477080|N|N|N|N|N| +2477173|AAAAAAAAFHMMFCAA|2070-03-02|2042|8879|682|2070|0|3|2|1|2070|682|8879|Sunday|2070Q1|N|N|N|2477172|2477230|2476808|2477081|N|N|N|N|N| +2477174|AAAAAAAAGHMMFCAA|2070-03-03|2042|8879|682|2070|1|3|3|1|2070|682|8879|Monday|2070Q1|N|N|N|2477172|2477230|2476809|2477082|N|N|N|N|N| +2477175|AAAAAAAAHHMMFCAA|2070-03-04|2042|8880|682|2070|2|3|4|1|2070|682|8880|Tuesday|2070Q1|N|N|N|2477172|2477230|2476810|2477083|N|N|N|N|N| +2477176|AAAAAAAAIHMMFCAA|2070-03-05|2042|8880|682|2070|3|3|5|1|2070|682|8880|Wednesday|2070Q1|N|N|N|2477172|2477230|2476811|2477084|N|N|N|N|N| +2477177|AAAAAAAAJHMMFCAA|2070-03-06|2042|8880|682|2070|4|3|6|1|2070|682|8880|Thursday|2070Q1|N|N|N|2477172|2477230|2476812|2477085|N|N|N|N|N| +2477178|AAAAAAAAKHMMFCAA|2070-03-07|2042|8880|682|2070|5|3|7|1|2070|682|8880|Friday|2070Q1|N|Y|N|2477172|2477230|2476813|2477086|N|N|N|N|N| +2477179|AAAAAAAALHMMFCAA|2070-03-08|2042|8880|682|2070|6|3|8|1|2070|682|8880|Saturday|2070Q1|N|Y|N|2477172|2477230|2476814|2477087|N|N|N|N|N| +2477180|AAAAAAAAMHMMFCAA|2070-03-09|2042|8880|682|2070|0|3|9|1|2070|682|8880|Sunday|2070Q1|N|N|N|2477172|2477230|2476815|2477088|N|N|N|N|N| +2477181|AAAAAAAANHMMFCAA|2070-03-10|2042|8880|682|2070|1|3|10|1|2070|682|8880|Monday|2070Q1|N|N|N|2477172|2477230|2476816|2477089|N|N|N|N|N| +2477182|AAAAAAAAOHMMFCAA|2070-03-11|2042|8881|682|2070|2|3|11|1|2070|682|8881|Tuesday|2070Q1|N|N|N|2477172|2477230|2476817|2477090|N|N|N|N|N| +2477183|AAAAAAAAPHMMFCAA|2070-03-12|2042|8881|682|2070|3|3|12|1|2070|682|8881|Wednesday|2070Q1|N|N|N|2477172|2477230|2476818|2477091|N|N|N|N|N| +2477184|AAAAAAAAAIMMFCAA|2070-03-13|2042|8881|682|2070|4|3|13|1|2070|682|8881|Thursday|2070Q1|N|N|N|2477172|2477230|2476819|2477092|N|N|N|N|N| +2477185|AAAAAAAABIMMFCAA|2070-03-14|2042|8881|682|2070|5|3|14|1|2070|682|8881|Friday|2070Q1|N|Y|N|2477172|2477230|2476820|2477093|N|N|N|N|N| +2477186|AAAAAAAACIMMFCAA|2070-03-15|2042|8881|682|2070|6|3|15|1|2070|682|8881|Saturday|2070Q1|N|Y|N|2477172|2477230|2476821|2477094|N|N|N|N|N| +2477187|AAAAAAAADIMMFCAA|2070-03-16|2042|8881|682|2070|0|3|16|1|2070|682|8881|Sunday|2070Q1|N|N|N|2477172|2477230|2476822|2477095|N|N|N|N|N| +2477188|AAAAAAAAEIMMFCAA|2070-03-17|2042|8881|682|2070|1|3|17|1|2070|682|8881|Monday|2070Q1|N|N|N|2477172|2477230|2476823|2477096|N|N|N|N|N| +2477189|AAAAAAAAFIMMFCAA|2070-03-18|2042|8882|682|2070|2|3|18|1|2070|682|8882|Tuesday|2070Q1|N|N|N|2477172|2477230|2476824|2477097|N|N|N|N|N| +2477190|AAAAAAAAGIMMFCAA|2070-03-19|2042|8882|682|2070|3|3|19|1|2070|682|8882|Wednesday|2070Q1|N|N|N|2477172|2477230|2476825|2477098|N|N|N|N|N| +2477191|AAAAAAAAHIMMFCAA|2070-03-20|2042|8882|682|2070|4|3|20|1|2070|682|8882|Thursday|2070Q1|N|N|N|2477172|2477230|2476826|2477099|N|N|N|N|N| +2477192|AAAAAAAAIIMMFCAA|2070-03-21|2042|8882|682|2070|5|3|21|1|2070|682|8882|Friday|2070Q1|N|Y|N|2477172|2477230|2476827|2477100|N|N|N|N|N| +2477193|AAAAAAAAJIMMFCAA|2070-03-22|2042|8882|682|2070|6|3|22|1|2070|682|8882|Saturday|2070Q1|N|Y|N|2477172|2477230|2476828|2477101|N|N|N|N|N| +2477194|AAAAAAAAKIMMFCAA|2070-03-23|2042|8882|682|2070|0|3|23|1|2070|682|8882|Sunday|2070Q1|N|N|N|2477172|2477230|2476829|2477102|N|N|N|N|N| +2477195|AAAAAAAALIMMFCAA|2070-03-24|2042|8882|682|2070|1|3|24|1|2070|682|8882|Monday|2070Q1|N|N|N|2477172|2477230|2476830|2477103|N|N|N|N|N| +2477196|AAAAAAAAMIMMFCAA|2070-03-25|2042|8883|682|2070|2|3|25|1|2070|682|8883|Tuesday|2070Q1|N|N|N|2477172|2477230|2476831|2477104|N|N|N|N|N| +2477197|AAAAAAAANIMMFCAA|2070-03-26|2042|8883|682|2070|3|3|26|1|2070|682|8883|Wednesday|2070Q1|N|N|N|2477172|2477230|2476832|2477105|N|N|N|N|N| +2477198|AAAAAAAAOIMMFCAA|2070-03-27|2042|8883|682|2070|4|3|27|1|2070|682|8883|Thursday|2070Q1|N|N|N|2477172|2477230|2476833|2477106|N|N|N|N|N| +2477199|AAAAAAAAPIMMFCAA|2070-03-28|2042|8883|682|2070|5|3|28|1|2070|682|8883|Friday|2070Q1|N|Y|N|2477172|2477230|2476834|2477107|N|N|N|N|N| +2477200|AAAAAAAAAJMMFCAA|2070-03-29|2042|8883|682|2070|6|3|29|1|2070|682|8883|Saturday|2070Q1|N|Y|N|2477172|2477230|2476835|2477108|N|N|N|N|N| +2477201|AAAAAAAABJMMFCAA|2070-03-30|2042|8883|682|2070|0|3|30|1|2070|682|8883|Sunday|2070Q1|N|N|N|2477172|2477230|2476836|2477109|N|N|N|N|N| +2477202|AAAAAAAACJMMFCAA|2070-03-31|2042|8883|682|2070|1|3|31|1|2070|682|8883|Monday|2070Q1|N|N|N|2477172|2477230|2476837|2477110|N|N|N|N|N| +2477203|AAAAAAAADJMMFCAA|2070-04-01|2043|8884|682|2070|2|4|1|1|2070|682|8884|Tuesday|2070Q1|N|N|N|2477203|2477292|2476838|2477113|N|N|N|N|N| +2477204|AAAAAAAAEJMMFCAA|2070-04-02|2043|8884|682|2070|3|4|2|2|2070|682|8884|Wednesday|2070Q2|N|N|N|2477203|2477292|2476839|2477114|N|N|N|N|N| +2477205|AAAAAAAAFJMMFCAA|2070-04-03|2043|8884|682|2070|4|4|3|2|2070|682|8884|Thursday|2070Q2|N|N|N|2477203|2477292|2476840|2477115|N|N|N|N|N| +2477206|AAAAAAAAGJMMFCAA|2070-04-04|2043|8884|682|2070|5|4|4|2|2070|682|8884|Friday|2070Q2|N|Y|N|2477203|2477292|2476841|2477116|N|N|N|N|N| +2477207|AAAAAAAAHJMMFCAA|2070-04-05|2043|8884|682|2070|6|4|5|2|2070|682|8884|Saturday|2070Q2|N|Y|N|2477203|2477292|2476842|2477117|N|N|N|N|N| +2477208|AAAAAAAAIJMMFCAA|2070-04-06|2043|8884|682|2070|0|4|6|2|2070|682|8884|Sunday|2070Q2|N|N|N|2477203|2477292|2476843|2477118|N|N|N|N|N| +2477209|AAAAAAAAJJMMFCAA|2070-04-07|2043|8884|682|2070|1|4|7|2|2070|682|8884|Monday|2070Q2|N|N|N|2477203|2477292|2476844|2477119|N|N|N|N|N| +2477210|AAAAAAAAKJMMFCAA|2070-04-08|2043|8885|682|2070|2|4|8|2|2070|682|8885|Tuesday|2070Q2|N|N|N|2477203|2477292|2476845|2477120|N|N|N|N|N| +2477211|AAAAAAAALJMMFCAA|2070-04-09|2043|8885|682|2070|3|4|9|2|2070|682|8885|Wednesday|2070Q2|N|N|N|2477203|2477292|2476846|2477121|N|N|N|N|N| +2477212|AAAAAAAAMJMMFCAA|2070-04-10|2043|8885|682|2070|4|4|10|2|2070|682|8885|Thursday|2070Q2|N|N|N|2477203|2477292|2476847|2477122|N|N|N|N|N| +2477213|AAAAAAAANJMMFCAA|2070-04-11|2043|8885|682|2070|5|4|11|2|2070|682|8885|Friday|2070Q2|N|Y|N|2477203|2477292|2476848|2477123|N|N|N|N|N| +2477214|AAAAAAAAOJMMFCAA|2070-04-12|2043|8885|682|2070|6|4|12|2|2070|682|8885|Saturday|2070Q2|N|Y|N|2477203|2477292|2476849|2477124|N|N|N|N|N| +2477215|AAAAAAAAPJMMFCAA|2070-04-13|2043|8885|682|2070|0|4|13|2|2070|682|8885|Sunday|2070Q2|N|N|N|2477203|2477292|2476850|2477125|N|N|N|N|N| +2477216|AAAAAAAAAKMMFCAA|2070-04-14|2043|8885|682|2070|1|4|14|2|2070|682|8885|Monday|2070Q2|N|N|N|2477203|2477292|2476851|2477126|N|N|N|N|N| +2477217|AAAAAAAABKMMFCAA|2070-04-15|2043|8886|682|2070|2|4|15|2|2070|682|8886|Tuesday|2070Q2|N|N|N|2477203|2477292|2476852|2477127|N|N|N|N|N| +2477218|AAAAAAAACKMMFCAA|2070-04-16|2043|8886|682|2070|3|4|16|2|2070|682|8886|Wednesday|2070Q2|N|N|N|2477203|2477292|2476853|2477128|N|N|N|N|N| +2477219|AAAAAAAADKMMFCAA|2070-04-17|2043|8886|682|2070|4|4|17|2|2070|682|8886|Thursday|2070Q2|N|N|N|2477203|2477292|2476854|2477129|N|N|N|N|N| +2477220|AAAAAAAAEKMMFCAA|2070-04-18|2043|8886|682|2070|5|4|18|2|2070|682|8886|Friday|2070Q2|N|Y|N|2477203|2477292|2476855|2477130|N|N|N|N|N| +2477221|AAAAAAAAFKMMFCAA|2070-04-19|2043|8886|682|2070|6|4|19|2|2070|682|8886|Saturday|2070Q2|N|Y|N|2477203|2477292|2476856|2477131|N|N|N|N|N| +2477222|AAAAAAAAGKMMFCAA|2070-04-20|2043|8886|682|2070|0|4|20|2|2070|682|8886|Sunday|2070Q2|N|N|N|2477203|2477292|2476857|2477132|N|N|N|N|N| +2477223|AAAAAAAAHKMMFCAA|2070-04-21|2043|8886|682|2070|1|4|21|2|2070|682|8886|Monday|2070Q2|N|N|N|2477203|2477292|2476858|2477133|N|N|N|N|N| +2477224|AAAAAAAAIKMMFCAA|2070-04-22|2043|8887|682|2070|2|4|22|2|2070|682|8887|Tuesday|2070Q2|N|N|N|2477203|2477292|2476859|2477134|N|N|N|N|N| +2477225|AAAAAAAAJKMMFCAA|2070-04-23|2043|8887|682|2070|3|4|23|2|2070|682|8887|Wednesday|2070Q2|N|N|N|2477203|2477292|2476860|2477135|N|N|N|N|N| +2477226|AAAAAAAAKKMMFCAA|2070-04-24|2043|8887|682|2070|4|4|24|2|2070|682|8887|Thursday|2070Q2|N|N|N|2477203|2477292|2476861|2477136|N|N|N|N|N| +2477227|AAAAAAAALKMMFCAA|2070-04-25|2043|8887|682|2070|5|4|25|2|2070|682|8887|Friday|2070Q2|N|Y|N|2477203|2477292|2476862|2477137|N|N|N|N|N| +2477228|AAAAAAAAMKMMFCAA|2070-04-26|2043|8887|682|2070|6|4|26|2|2070|682|8887|Saturday|2070Q2|N|Y|N|2477203|2477292|2476863|2477138|N|N|N|N|N| +2477229|AAAAAAAANKMMFCAA|2070-04-27|2043|8887|682|2070|0|4|27|2|2070|682|8887|Sunday|2070Q2|N|N|N|2477203|2477292|2476864|2477139|N|N|N|N|N| +2477230|AAAAAAAAOKMMFCAA|2070-04-28|2043|8887|682|2070|1|4|28|2|2070|682|8887|Monday|2070Q2|N|N|N|2477203|2477292|2476865|2477140|N|N|N|N|N| +2477231|AAAAAAAAPKMMFCAA|2070-04-29|2043|8888|682|2070|2|4|29|2|2070|682|8888|Tuesday|2070Q2|N|N|N|2477203|2477292|2476866|2477141|N|N|N|N|N| +2477232|AAAAAAAAALMMFCAA|2070-04-30|2043|8888|682|2070|3|4|30|2|2070|682|8888|Wednesday|2070Q2|N|N|N|2477203|2477292|2476867|2477142|N|N|N|N|N| +2477233|AAAAAAAABLMMFCAA|2070-05-01|2044|8888|682|2070|4|5|1|2|2070|682|8888|Thursday|2070Q2|N|N|N|2477233|2477352|2476868|2477143|N|N|N|N|N| +2477234|AAAAAAAACLMMFCAA|2070-05-02|2044|8888|682|2070|5|5|2|2|2070|682|8888|Friday|2070Q2|N|Y|N|2477233|2477352|2476869|2477144|N|N|N|N|N| +2477235|AAAAAAAADLMMFCAA|2070-05-03|2044|8888|682|2070|6|5|3|2|2070|682|8888|Saturday|2070Q2|N|Y|N|2477233|2477352|2476870|2477145|N|N|N|N|N| +2477236|AAAAAAAAELMMFCAA|2070-05-04|2044|8888|682|2070|0|5|4|2|2070|682|8888|Sunday|2070Q2|N|N|N|2477233|2477352|2476871|2477146|N|N|N|N|N| +2477237|AAAAAAAAFLMMFCAA|2070-05-05|2044|8888|682|2070|1|5|5|2|2070|682|8888|Monday|2070Q2|N|N|N|2477233|2477352|2476872|2477147|N|N|N|N|N| +2477238|AAAAAAAAGLMMFCAA|2070-05-06|2044|8889|682|2070|2|5|6|2|2070|682|8889|Tuesday|2070Q2|N|N|N|2477233|2477352|2476873|2477148|N|N|N|N|N| +2477239|AAAAAAAAHLMMFCAA|2070-05-07|2044|8889|682|2070|3|5|7|2|2070|682|8889|Wednesday|2070Q2|N|N|N|2477233|2477352|2476874|2477149|N|N|N|N|N| +2477240|AAAAAAAAILMMFCAA|2070-05-08|2044|8889|682|2070|4|5|8|2|2070|682|8889|Thursday|2070Q2|N|N|N|2477233|2477352|2476875|2477150|N|N|N|N|N| +2477241|AAAAAAAAJLMMFCAA|2070-05-09|2044|8889|682|2070|5|5|9|2|2070|682|8889|Friday|2070Q2|N|Y|N|2477233|2477352|2476876|2477151|N|N|N|N|N| +2477242|AAAAAAAAKLMMFCAA|2070-05-10|2044|8889|682|2070|6|5|10|2|2070|682|8889|Saturday|2070Q2|N|Y|N|2477233|2477352|2476877|2477152|N|N|N|N|N| +2477243|AAAAAAAALLMMFCAA|2070-05-11|2044|8889|682|2070|0|5|11|2|2070|682|8889|Sunday|2070Q2|N|N|N|2477233|2477352|2476878|2477153|N|N|N|N|N| +2477244|AAAAAAAAMLMMFCAA|2070-05-12|2044|8889|682|2070|1|5|12|2|2070|682|8889|Monday|2070Q2|N|N|N|2477233|2477352|2476879|2477154|N|N|N|N|N| +2477245|AAAAAAAANLMMFCAA|2070-05-13|2044|8890|682|2070|2|5|13|2|2070|682|8890|Tuesday|2070Q2|N|N|N|2477233|2477352|2476880|2477155|N|N|N|N|N| +2477246|AAAAAAAAOLMMFCAA|2070-05-14|2044|8890|682|2070|3|5|14|2|2070|682|8890|Wednesday|2070Q2|N|N|N|2477233|2477352|2476881|2477156|N|N|N|N|N| +2477247|AAAAAAAAPLMMFCAA|2070-05-15|2044|8890|682|2070|4|5|15|2|2070|682|8890|Thursday|2070Q2|N|N|N|2477233|2477352|2476882|2477157|N|N|N|N|N| +2477248|AAAAAAAAAMMMFCAA|2070-05-16|2044|8890|682|2070|5|5|16|2|2070|682|8890|Friday|2070Q2|N|Y|N|2477233|2477352|2476883|2477158|N|N|N|N|N| +2477249|AAAAAAAABMMMFCAA|2070-05-17|2044|8890|682|2070|6|5|17|2|2070|682|8890|Saturday|2070Q2|N|Y|N|2477233|2477352|2476884|2477159|N|N|N|N|N| +2477250|AAAAAAAACMMMFCAA|2070-05-18|2044|8890|682|2070|0|5|18|2|2070|682|8890|Sunday|2070Q2|N|N|N|2477233|2477352|2476885|2477160|N|N|N|N|N| +2477251|AAAAAAAADMMMFCAA|2070-05-19|2044|8890|682|2070|1|5|19|2|2070|682|8890|Monday|2070Q2|N|N|N|2477233|2477352|2476886|2477161|N|N|N|N|N| +2477252|AAAAAAAAEMMMFCAA|2070-05-20|2044|8891|682|2070|2|5|20|2|2070|682|8891|Tuesday|2070Q2|N|N|N|2477233|2477352|2476887|2477162|N|N|N|N|N| +2477253|AAAAAAAAFMMMFCAA|2070-05-21|2044|8891|682|2070|3|5|21|2|2070|682|8891|Wednesday|2070Q2|N|N|N|2477233|2477352|2476888|2477163|N|N|N|N|N| +2477254|AAAAAAAAGMMMFCAA|2070-05-22|2044|8891|682|2070|4|5|22|2|2070|682|8891|Thursday|2070Q2|N|N|N|2477233|2477352|2476889|2477164|N|N|N|N|N| +2477255|AAAAAAAAHMMMFCAA|2070-05-23|2044|8891|682|2070|5|5|23|2|2070|682|8891|Friday|2070Q2|N|Y|N|2477233|2477352|2476890|2477165|N|N|N|N|N| +2477256|AAAAAAAAIMMMFCAA|2070-05-24|2044|8891|682|2070|6|5|24|2|2070|682|8891|Saturday|2070Q2|N|Y|N|2477233|2477352|2476891|2477166|N|N|N|N|N| +2477257|AAAAAAAAJMMMFCAA|2070-05-25|2044|8891|682|2070|0|5|25|2|2070|682|8891|Sunday|2070Q2|N|N|N|2477233|2477352|2476892|2477167|N|N|N|N|N| +2477258|AAAAAAAAKMMMFCAA|2070-05-26|2044|8891|682|2070|1|5|26|2|2070|682|8891|Monday|2070Q2|N|N|N|2477233|2477352|2476893|2477168|N|N|N|N|N| +2477259|AAAAAAAALMMMFCAA|2070-05-27|2044|8892|682|2070|2|5|27|2|2070|682|8892|Tuesday|2070Q2|N|N|N|2477233|2477352|2476894|2477169|N|N|N|N|N| +2477260|AAAAAAAAMMMMFCAA|2070-05-28|2044|8892|682|2070|3|5|28|2|2070|682|8892|Wednesday|2070Q2|N|N|N|2477233|2477352|2476895|2477170|N|N|N|N|N| +2477261|AAAAAAAANMMMFCAA|2070-05-29|2044|8892|682|2070|4|5|29|2|2070|682|8892|Thursday|2070Q2|N|N|N|2477233|2477352|2476896|2477171|N|N|N|N|N| +2477262|AAAAAAAAOMMMFCAA|2070-05-30|2044|8892|682|2070|5|5|30|2|2070|682|8892|Friday|2070Q2|N|Y|N|2477233|2477352|2476897|2477172|N|N|N|N|N| +2477263|AAAAAAAAPMMMFCAA|2070-05-31|2044|8892|682|2070|6|5|31|2|2070|682|8892|Saturday|2070Q2|N|Y|N|2477233|2477352|2476898|2477173|N|N|N|N|N| +2477264|AAAAAAAAANMMFCAA|2070-06-01|2045|8892|683|2070|0|6|1|2|2070|683|8892|Sunday|2070Q2|N|N|N|2477264|2477414|2476899|2477174|N|N|N|N|N| +2477265|AAAAAAAABNMMFCAA|2070-06-02|2045|8892|683|2070|1|6|2|2|2070|683|8892|Monday|2070Q2|N|N|N|2477264|2477414|2476900|2477175|N|N|N|N|N| +2477266|AAAAAAAACNMMFCAA|2070-06-03|2045|8893|683|2070|2|6|3|2|2070|683|8893|Tuesday|2070Q2|N|N|N|2477264|2477414|2476901|2477176|N|N|N|N|N| +2477267|AAAAAAAADNMMFCAA|2070-06-04|2045|8893|683|2070|3|6|4|2|2070|683|8893|Wednesday|2070Q2|N|N|N|2477264|2477414|2476902|2477177|N|N|N|N|N| +2477268|AAAAAAAAENMMFCAA|2070-06-05|2045|8893|683|2070|4|6|5|2|2070|683|8893|Thursday|2070Q2|N|N|N|2477264|2477414|2476903|2477178|N|N|N|N|N| +2477269|AAAAAAAAFNMMFCAA|2070-06-06|2045|8893|683|2070|5|6|6|2|2070|683|8893|Friday|2070Q2|N|Y|N|2477264|2477414|2476904|2477179|N|N|N|N|N| +2477270|AAAAAAAAGNMMFCAA|2070-06-07|2045|8893|683|2070|6|6|7|2|2070|683|8893|Saturday|2070Q2|N|Y|N|2477264|2477414|2476905|2477180|N|N|N|N|N| +2477271|AAAAAAAAHNMMFCAA|2070-06-08|2045|8893|683|2070|0|6|8|2|2070|683|8893|Sunday|2070Q2|N|N|N|2477264|2477414|2476906|2477181|N|N|N|N|N| +2477272|AAAAAAAAINMMFCAA|2070-06-09|2045|8893|683|2070|1|6|9|2|2070|683|8893|Monday|2070Q2|N|N|N|2477264|2477414|2476907|2477182|N|N|N|N|N| +2477273|AAAAAAAAJNMMFCAA|2070-06-10|2045|8894|683|2070|2|6|10|2|2070|683|8894|Tuesday|2070Q2|N|N|N|2477264|2477414|2476908|2477183|N|N|N|N|N| +2477274|AAAAAAAAKNMMFCAA|2070-06-11|2045|8894|683|2070|3|6|11|2|2070|683|8894|Wednesday|2070Q2|N|N|N|2477264|2477414|2476909|2477184|N|N|N|N|N| +2477275|AAAAAAAALNMMFCAA|2070-06-12|2045|8894|683|2070|4|6|12|2|2070|683|8894|Thursday|2070Q2|N|N|N|2477264|2477414|2476910|2477185|N|N|N|N|N| +2477276|AAAAAAAAMNMMFCAA|2070-06-13|2045|8894|683|2070|5|6|13|2|2070|683|8894|Friday|2070Q2|N|Y|N|2477264|2477414|2476911|2477186|N|N|N|N|N| +2477277|AAAAAAAANNMMFCAA|2070-06-14|2045|8894|683|2070|6|6|14|2|2070|683|8894|Saturday|2070Q2|N|Y|N|2477264|2477414|2476912|2477187|N|N|N|N|N| +2477278|AAAAAAAAONMMFCAA|2070-06-15|2045|8894|683|2070|0|6|15|2|2070|683|8894|Sunday|2070Q2|N|N|N|2477264|2477414|2476913|2477188|N|N|N|N|N| +2477279|AAAAAAAAPNMMFCAA|2070-06-16|2045|8894|683|2070|1|6|16|2|2070|683|8894|Monday|2070Q2|N|N|N|2477264|2477414|2476914|2477189|N|N|N|N|N| +2477280|AAAAAAAAAOMMFCAA|2070-06-17|2045|8895|683|2070|2|6|17|2|2070|683|8895|Tuesday|2070Q2|N|N|N|2477264|2477414|2476915|2477190|N|N|N|N|N| +2477281|AAAAAAAABOMMFCAA|2070-06-18|2045|8895|683|2070|3|6|18|2|2070|683|8895|Wednesday|2070Q2|N|N|N|2477264|2477414|2476916|2477191|N|N|N|N|N| +2477282|AAAAAAAACOMMFCAA|2070-06-19|2045|8895|683|2070|4|6|19|2|2070|683|8895|Thursday|2070Q2|N|N|N|2477264|2477414|2476917|2477192|N|N|N|N|N| +2477283|AAAAAAAADOMMFCAA|2070-06-20|2045|8895|683|2070|5|6|20|2|2070|683|8895|Friday|2070Q2|N|Y|N|2477264|2477414|2476918|2477193|N|N|N|N|N| +2477284|AAAAAAAAEOMMFCAA|2070-06-21|2045|8895|683|2070|6|6|21|2|2070|683|8895|Saturday|2070Q2|N|Y|N|2477264|2477414|2476919|2477194|N|N|N|N|N| +2477285|AAAAAAAAFOMMFCAA|2070-06-22|2045|8895|683|2070|0|6|22|2|2070|683|8895|Sunday|2070Q2|N|N|N|2477264|2477414|2476920|2477195|N|N|N|N|N| +2477286|AAAAAAAAGOMMFCAA|2070-06-23|2045|8895|683|2070|1|6|23|2|2070|683|8895|Monday|2070Q2|N|N|N|2477264|2477414|2476921|2477196|N|N|N|N|N| +2477287|AAAAAAAAHOMMFCAA|2070-06-24|2045|8896|683|2070|2|6|24|2|2070|683|8896|Tuesday|2070Q2|N|N|N|2477264|2477414|2476922|2477197|N|N|N|N|N| +2477288|AAAAAAAAIOMMFCAA|2070-06-25|2045|8896|683|2070|3|6|25|2|2070|683|8896|Wednesday|2070Q2|N|N|N|2477264|2477414|2476923|2477198|N|N|N|N|N| +2477289|AAAAAAAAJOMMFCAA|2070-06-26|2045|8896|683|2070|4|6|26|2|2070|683|8896|Thursday|2070Q2|N|N|N|2477264|2477414|2476924|2477199|N|N|N|N|N| +2477290|AAAAAAAAKOMMFCAA|2070-06-27|2045|8896|683|2070|5|6|27|2|2070|683|8896|Friday|2070Q2|N|Y|N|2477264|2477414|2476925|2477200|N|N|N|N|N| +2477291|AAAAAAAALOMMFCAA|2070-06-28|2045|8896|683|2070|6|6|28|2|2070|683|8896|Saturday|2070Q2|N|Y|N|2477264|2477414|2476926|2477201|N|N|N|N|N| +2477292|AAAAAAAAMOMMFCAA|2070-06-29|2045|8896|683|2070|0|6|29|2|2070|683|8896|Sunday|2070Q2|N|N|N|2477264|2477414|2476927|2477202|N|N|N|N|N| +2477293|AAAAAAAANOMMFCAA|2070-06-30|2045|8896|683|2070|1|6|30|2|2070|683|8896|Monday|2070Q2|N|N|N|2477264|2477414|2476928|2477203|N|N|N|N|N| +2477294|AAAAAAAAOOMMFCAA|2070-07-01|2046|8897|683|2070|2|7|1|2|2070|683|8897|Tuesday|2070Q2|N|N|N|2477294|2477474|2476929|2477203|N|N|N|N|N| +2477295|AAAAAAAAPOMMFCAA|2070-07-02|2046|8897|683|2070|3|7|2|3|2070|683|8897|Wednesday|2070Q3|N|N|N|2477294|2477474|2476930|2477204|N|N|N|N|N| +2477296|AAAAAAAAAPMMFCAA|2070-07-03|2046|8897|683|2070|4|7|3|3|2070|683|8897|Thursday|2070Q3|N|N|N|2477294|2477474|2476931|2477205|N|N|N|N|N| +2477297|AAAAAAAABPMMFCAA|2070-07-04|2046|8897|683|2070|5|7|4|3|2070|683|8897|Friday|2070Q3|N|Y|N|2477294|2477474|2476932|2477206|N|N|N|N|N| +2477298|AAAAAAAACPMMFCAA|2070-07-05|2046|8897|683|2070|6|7|5|3|2070|683|8897|Saturday|2070Q3|Y|Y|N|2477294|2477474|2476933|2477207|N|N|N|N|N| +2477299|AAAAAAAADPMMFCAA|2070-07-06|2046|8897|683|2070|0|7|6|3|2070|683|8897|Sunday|2070Q3|N|N|Y|2477294|2477474|2476934|2477208|N|N|N|N|N| +2477300|AAAAAAAAEPMMFCAA|2070-07-07|2046|8897|683|2070|1|7|7|3|2070|683|8897|Monday|2070Q3|N|N|N|2477294|2477474|2476935|2477209|N|N|N|N|N| +2477301|AAAAAAAAFPMMFCAA|2070-07-08|2046|8898|683|2070|2|7|8|3|2070|683|8898|Tuesday|2070Q3|N|N|N|2477294|2477474|2476936|2477210|N|N|N|N|N| +2477302|AAAAAAAAGPMMFCAA|2070-07-09|2046|8898|683|2070|3|7|9|3|2070|683|8898|Wednesday|2070Q3|N|N|N|2477294|2477474|2476937|2477211|N|N|N|N|N| +2477303|AAAAAAAAHPMMFCAA|2070-07-10|2046|8898|683|2070|4|7|10|3|2070|683|8898|Thursday|2070Q3|N|N|N|2477294|2477474|2476938|2477212|N|N|N|N|N| +2477304|AAAAAAAAIPMMFCAA|2070-07-11|2046|8898|683|2070|5|7|11|3|2070|683|8898|Friday|2070Q3|N|Y|N|2477294|2477474|2476939|2477213|N|N|N|N|N| +2477305|AAAAAAAAJPMMFCAA|2070-07-12|2046|8898|683|2070|6|7|12|3|2070|683|8898|Saturday|2070Q3|N|Y|N|2477294|2477474|2476940|2477214|N|N|N|N|N| +2477306|AAAAAAAAKPMMFCAA|2070-07-13|2046|8898|683|2070|0|7|13|3|2070|683|8898|Sunday|2070Q3|N|N|N|2477294|2477474|2476941|2477215|N|N|N|N|N| +2477307|AAAAAAAALPMMFCAA|2070-07-14|2046|8898|683|2070|1|7|14|3|2070|683|8898|Monday|2070Q3|N|N|N|2477294|2477474|2476942|2477216|N|N|N|N|N| +2477308|AAAAAAAAMPMMFCAA|2070-07-15|2046|8899|683|2070|2|7|15|3|2070|683|8899|Tuesday|2070Q3|N|N|N|2477294|2477474|2476943|2477217|N|N|N|N|N| +2477309|AAAAAAAANPMMFCAA|2070-07-16|2046|8899|683|2070|3|7|16|3|2070|683|8899|Wednesday|2070Q3|N|N|N|2477294|2477474|2476944|2477218|N|N|N|N|N| +2477310|AAAAAAAAOPMMFCAA|2070-07-17|2046|8899|683|2070|4|7|17|3|2070|683|8899|Thursday|2070Q3|N|N|N|2477294|2477474|2476945|2477219|N|N|N|N|N| +2477311|AAAAAAAAPPMMFCAA|2070-07-18|2046|8899|683|2070|5|7|18|3|2070|683|8899|Friday|2070Q3|N|Y|N|2477294|2477474|2476946|2477220|N|N|N|N|N| +2477312|AAAAAAAAAANMFCAA|2070-07-19|2046|8899|683|2070|6|7|19|3|2070|683|8899|Saturday|2070Q3|N|Y|N|2477294|2477474|2476947|2477221|N|N|N|N|N| +2477313|AAAAAAAABANMFCAA|2070-07-20|2046|8899|683|2070|0|7|20|3|2070|683|8899|Sunday|2070Q3|N|N|N|2477294|2477474|2476948|2477222|N|N|N|N|N| +2477314|AAAAAAAACANMFCAA|2070-07-21|2046|8899|683|2070|1|7|21|3|2070|683|8899|Monday|2070Q3|N|N|N|2477294|2477474|2476949|2477223|N|N|N|N|N| +2477315|AAAAAAAADANMFCAA|2070-07-22|2046|8900|683|2070|2|7|22|3|2070|683|8900|Tuesday|2070Q3|N|N|N|2477294|2477474|2476950|2477224|N|N|N|N|N| +2477316|AAAAAAAAEANMFCAA|2070-07-23|2046|8900|683|2070|3|7|23|3|2070|683|8900|Wednesday|2070Q3|N|N|N|2477294|2477474|2476951|2477225|N|N|N|N|N| +2477317|AAAAAAAAFANMFCAA|2070-07-24|2046|8900|683|2070|4|7|24|3|2070|683|8900|Thursday|2070Q3|N|N|N|2477294|2477474|2476952|2477226|N|N|N|N|N| +2477318|AAAAAAAAGANMFCAA|2070-07-25|2046|8900|683|2070|5|7|25|3|2070|683|8900|Friday|2070Q3|N|Y|N|2477294|2477474|2476953|2477227|N|N|N|N|N| +2477319|AAAAAAAAHANMFCAA|2070-07-26|2046|8900|683|2070|6|7|26|3|2070|683|8900|Saturday|2070Q3|N|Y|N|2477294|2477474|2476954|2477228|N|N|N|N|N| +2477320|AAAAAAAAIANMFCAA|2070-07-27|2046|8900|683|2070|0|7|27|3|2070|683|8900|Sunday|2070Q3|N|N|N|2477294|2477474|2476955|2477229|N|N|N|N|N| +2477321|AAAAAAAAJANMFCAA|2070-07-28|2046|8900|683|2070|1|7|28|3|2070|683|8900|Monday|2070Q3|N|N|N|2477294|2477474|2476956|2477230|N|N|N|N|N| +2477322|AAAAAAAAKANMFCAA|2070-07-29|2046|8901|683|2070|2|7|29|3|2070|683|8901|Tuesday|2070Q3|N|N|N|2477294|2477474|2476957|2477231|N|N|N|N|N| +2477323|AAAAAAAALANMFCAA|2070-07-30|2046|8901|683|2070|3|7|30|3|2070|683|8901|Wednesday|2070Q3|N|N|N|2477294|2477474|2476958|2477232|N|N|N|N|N| +2477324|AAAAAAAAMANMFCAA|2070-07-31|2046|8901|683|2070|4|7|31|3|2070|683|8901|Thursday|2070Q3|N|N|N|2477294|2477474|2476959|2477233|N|N|N|N|N| +2477325|AAAAAAAANANMFCAA|2070-08-01|2047|8901|683|2070|5|8|1|3|2070|683|8901|Friday|2070Q3|N|Y|N|2477325|2477536|2476960|2477234|N|N|N|N|N| +2477326|AAAAAAAAOANMFCAA|2070-08-02|2047|8901|683|2070|6|8|2|3|2070|683|8901|Saturday|2070Q3|N|Y|N|2477325|2477536|2476961|2477235|N|N|N|N|N| +2477327|AAAAAAAAPANMFCAA|2070-08-03|2047|8901|683|2070|0|8|3|3|2070|683|8901|Sunday|2070Q3|N|N|N|2477325|2477536|2476962|2477236|N|N|N|N|N| +2477328|AAAAAAAAABNMFCAA|2070-08-04|2047|8901|683|2070|1|8|4|3|2070|683|8901|Monday|2070Q3|N|N|N|2477325|2477536|2476963|2477237|N|N|N|N|N| +2477329|AAAAAAAABBNMFCAA|2070-08-05|2047|8902|683|2070|2|8|5|3|2070|683|8902|Tuesday|2070Q3|N|N|N|2477325|2477536|2476964|2477238|N|N|N|N|N| +2477330|AAAAAAAACBNMFCAA|2070-08-06|2047|8902|683|2070|3|8|6|3|2070|683|8902|Wednesday|2070Q3|N|N|N|2477325|2477536|2476965|2477239|N|N|N|N|N| +2477331|AAAAAAAADBNMFCAA|2070-08-07|2047|8902|683|2070|4|8|7|3|2070|683|8902|Thursday|2070Q3|N|N|N|2477325|2477536|2476966|2477240|N|N|N|N|N| +2477332|AAAAAAAAEBNMFCAA|2070-08-08|2047|8902|683|2070|5|8|8|3|2070|683|8902|Friday|2070Q3|N|Y|N|2477325|2477536|2476967|2477241|N|N|N|N|N| +2477333|AAAAAAAAFBNMFCAA|2070-08-09|2047|8902|683|2070|6|8|9|3|2070|683|8902|Saturday|2070Q3|N|Y|N|2477325|2477536|2476968|2477242|N|N|N|N|N| +2477334|AAAAAAAAGBNMFCAA|2070-08-10|2047|8902|683|2070|0|8|10|3|2070|683|8902|Sunday|2070Q3|N|N|N|2477325|2477536|2476969|2477243|N|N|N|N|N| +2477335|AAAAAAAAHBNMFCAA|2070-08-11|2047|8902|683|2070|1|8|11|3|2070|683|8902|Monday|2070Q3|N|N|N|2477325|2477536|2476970|2477244|N|N|N|N|N| +2477336|AAAAAAAAIBNMFCAA|2070-08-12|2047|8903|683|2070|2|8|12|3|2070|683|8903|Tuesday|2070Q3|N|N|N|2477325|2477536|2476971|2477245|N|N|N|N|N| +2477337|AAAAAAAAJBNMFCAA|2070-08-13|2047|8903|683|2070|3|8|13|3|2070|683|8903|Wednesday|2070Q3|N|N|N|2477325|2477536|2476972|2477246|N|N|N|N|N| +2477338|AAAAAAAAKBNMFCAA|2070-08-14|2047|8903|683|2070|4|8|14|3|2070|683|8903|Thursday|2070Q3|N|N|N|2477325|2477536|2476973|2477247|N|N|N|N|N| +2477339|AAAAAAAALBNMFCAA|2070-08-15|2047|8903|683|2070|5|8|15|3|2070|683|8903|Friday|2070Q3|N|Y|N|2477325|2477536|2476974|2477248|N|N|N|N|N| +2477340|AAAAAAAAMBNMFCAA|2070-08-16|2047|8903|683|2070|6|8|16|3|2070|683|8903|Saturday|2070Q3|N|Y|N|2477325|2477536|2476975|2477249|N|N|N|N|N| +2477341|AAAAAAAANBNMFCAA|2070-08-17|2047|8903|683|2070|0|8|17|3|2070|683|8903|Sunday|2070Q3|N|N|N|2477325|2477536|2476976|2477250|N|N|N|N|N| +2477342|AAAAAAAAOBNMFCAA|2070-08-18|2047|8903|683|2070|1|8|18|3|2070|683|8903|Monday|2070Q3|N|N|N|2477325|2477536|2476977|2477251|N|N|N|N|N| +2477343|AAAAAAAAPBNMFCAA|2070-08-19|2047|8904|683|2070|2|8|19|3|2070|683|8904|Tuesday|2070Q3|N|N|N|2477325|2477536|2476978|2477252|N|N|N|N|N| +2477344|AAAAAAAAACNMFCAA|2070-08-20|2047|8904|683|2070|3|8|20|3|2070|683|8904|Wednesday|2070Q3|N|N|N|2477325|2477536|2476979|2477253|N|N|N|N|N| +2477345|AAAAAAAABCNMFCAA|2070-08-21|2047|8904|683|2070|4|8|21|3|2070|683|8904|Thursday|2070Q3|N|N|N|2477325|2477536|2476980|2477254|N|N|N|N|N| +2477346|AAAAAAAACCNMFCAA|2070-08-22|2047|8904|683|2070|5|8|22|3|2070|683|8904|Friday|2070Q3|N|Y|N|2477325|2477536|2476981|2477255|N|N|N|N|N| +2477347|AAAAAAAADCNMFCAA|2070-08-23|2047|8904|683|2070|6|8|23|3|2070|683|8904|Saturday|2070Q3|N|Y|N|2477325|2477536|2476982|2477256|N|N|N|N|N| +2477348|AAAAAAAAECNMFCAA|2070-08-24|2047|8904|683|2070|0|8|24|3|2070|683|8904|Sunday|2070Q3|N|N|N|2477325|2477536|2476983|2477257|N|N|N|N|N| +2477349|AAAAAAAAFCNMFCAA|2070-08-25|2047|8904|683|2070|1|8|25|3|2070|683|8904|Monday|2070Q3|N|N|N|2477325|2477536|2476984|2477258|N|N|N|N|N| +2477350|AAAAAAAAGCNMFCAA|2070-08-26|2047|8905|683|2070|2|8|26|3|2070|683|8905|Tuesday|2070Q3|N|N|N|2477325|2477536|2476985|2477259|N|N|N|N|N| +2477351|AAAAAAAAHCNMFCAA|2070-08-27|2047|8905|683|2070|3|8|27|3|2070|683|8905|Wednesday|2070Q3|N|N|N|2477325|2477536|2476986|2477260|N|N|N|N|N| +2477352|AAAAAAAAICNMFCAA|2070-08-28|2047|8905|683|2070|4|8|28|3|2070|683|8905|Thursday|2070Q3|N|N|N|2477325|2477536|2476987|2477261|N|N|N|N|N| +2477353|AAAAAAAAJCNMFCAA|2070-08-29|2047|8905|683|2070|5|8|29|3|2070|683|8905|Friday|2070Q3|N|Y|N|2477325|2477536|2476988|2477262|N|N|N|N|N| +2477354|AAAAAAAAKCNMFCAA|2070-08-30|2047|8905|683|2070|6|8|30|3|2070|683|8905|Saturday|2070Q3|N|Y|N|2477325|2477536|2476989|2477263|N|N|N|N|N| +2477355|AAAAAAAALCNMFCAA|2070-08-31|2047|8905|683|2070|0|8|31|3|2070|683|8905|Sunday|2070Q3|N|N|N|2477325|2477536|2476990|2477264|N|N|N|N|N| +2477356|AAAAAAAAMCNMFCAA|2070-09-01|2048|8905|684|2070|1|9|1|3|2070|684|8905|Monday|2070Q3|N|N|N|2477356|2477598|2476991|2477265|N|N|N|N|N| +2477357|AAAAAAAANCNMFCAA|2070-09-02|2048|8906|684|2070|2|9|2|3|2070|684|8906|Tuesday|2070Q3|N|N|N|2477356|2477598|2476992|2477266|N|N|N|N|N| +2477358|AAAAAAAAOCNMFCAA|2070-09-03|2048|8906|684|2070|3|9|3|3|2070|684|8906|Wednesday|2070Q3|N|N|N|2477356|2477598|2476993|2477267|N|N|N|N|N| +2477359|AAAAAAAAPCNMFCAA|2070-09-04|2048|8906|684|2070|4|9|4|3|2070|684|8906|Thursday|2070Q3|N|N|N|2477356|2477598|2476994|2477268|N|N|N|N|N| +2477360|AAAAAAAAADNMFCAA|2070-09-05|2048|8906|684|2070|5|9|5|3|2070|684|8906|Friday|2070Q3|N|Y|N|2477356|2477598|2476995|2477269|N|N|N|N|N| +2477361|AAAAAAAABDNMFCAA|2070-09-06|2048|8906|684|2070|6|9|6|3|2070|684|8906|Saturday|2070Q3|N|Y|N|2477356|2477598|2476996|2477270|N|N|N|N|N| +2477362|AAAAAAAACDNMFCAA|2070-09-07|2048|8906|684|2070|0|9|7|3|2070|684|8906|Sunday|2070Q3|N|N|N|2477356|2477598|2476997|2477271|N|N|N|N|N| +2477363|AAAAAAAADDNMFCAA|2070-09-08|2048|8906|684|2070|1|9|8|3|2070|684|8906|Monday|2070Q3|N|N|N|2477356|2477598|2476998|2477272|N|N|N|N|N| +2477364|AAAAAAAAEDNMFCAA|2070-09-09|2048|8907|684|2070|2|9|9|3|2070|684|8907|Tuesday|2070Q3|N|N|N|2477356|2477598|2476999|2477273|N|N|N|N|N| +2477365|AAAAAAAAFDNMFCAA|2070-09-10|2048|8907|684|2070|3|9|10|3|2070|684|8907|Wednesday|2070Q3|N|N|N|2477356|2477598|2477000|2477274|N|N|N|N|N| +2477366|AAAAAAAAGDNMFCAA|2070-09-11|2048|8907|684|2070|4|9|11|3|2070|684|8907|Thursday|2070Q3|N|N|N|2477356|2477598|2477001|2477275|N|N|N|N|N| +2477367|AAAAAAAAHDNMFCAA|2070-09-12|2048|8907|684|2070|5|9|12|3|2070|684|8907|Friday|2070Q3|N|Y|N|2477356|2477598|2477002|2477276|N|N|N|N|N| +2477368|AAAAAAAAIDNMFCAA|2070-09-13|2048|8907|684|2070|6|9|13|3|2070|684|8907|Saturday|2070Q3|N|Y|N|2477356|2477598|2477003|2477277|N|N|N|N|N| +2477369|AAAAAAAAJDNMFCAA|2070-09-14|2048|8907|684|2070|0|9|14|3|2070|684|8907|Sunday|2070Q3|N|N|N|2477356|2477598|2477004|2477278|N|N|N|N|N| +2477370|AAAAAAAAKDNMFCAA|2070-09-15|2048|8907|684|2070|1|9|15|3|2070|684|8907|Monday|2070Q3|N|N|N|2477356|2477598|2477005|2477279|N|N|N|N|N| +2477371|AAAAAAAALDNMFCAA|2070-09-16|2048|8908|684|2070|2|9|16|3|2070|684|8908|Tuesday|2070Q3|N|N|N|2477356|2477598|2477006|2477280|N|N|N|N|N| +2477372|AAAAAAAAMDNMFCAA|2070-09-17|2048|8908|684|2070|3|9|17|3|2070|684|8908|Wednesday|2070Q3|N|N|N|2477356|2477598|2477007|2477281|N|N|N|N|N| +2477373|AAAAAAAANDNMFCAA|2070-09-18|2048|8908|684|2070|4|9|18|3|2070|684|8908|Thursday|2070Q3|N|N|N|2477356|2477598|2477008|2477282|N|N|N|N|N| +2477374|AAAAAAAAODNMFCAA|2070-09-19|2048|8908|684|2070|5|9|19|3|2070|684|8908|Friday|2070Q3|N|Y|N|2477356|2477598|2477009|2477283|N|N|N|N|N| +2477375|AAAAAAAAPDNMFCAA|2070-09-20|2048|8908|684|2070|6|9|20|3|2070|684|8908|Saturday|2070Q3|N|Y|N|2477356|2477598|2477010|2477284|N|N|N|N|N| +2477376|AAAAAAAAAENMFCAA|2070-09-21|2048|8908|684|2070|0|9|21|3|2070|684|8908|Sunday|2070Q3|N|N|N|2477356|2477598|2477011|2477285|N|N|N|N|N| +2477377|AAAAAAAABENMFCAA|2070-09-22|2048|8908|684|2070|1|9|22|3|2070|684|8908|Monday|2070Q3|N|N|N|2477356|2477598|2477012|2477286|N|N|N|N|N| +2477378|AAAAAAAACENMFCAA|2070-09-23|2048|8909|684|2070|2|9|23|3|2070|684|8909|Tuesday|2070Q3|N|N|N|2477356|2477598|2477013|2477287|N|N|N|N|N| +2477379|AAAAAAAADENMFCAA|2070-09-24|2048|8909|684|2070|3|9|24|3|2070|684|8909|Wednesday|2070Q3|N|N|N|2477356|2477598|2477014|2477288|N|N|N|N|N| +2477380|AAAAAAAAEENMFCAA|2070-09-25|2048|8909|684|2070|4|9|25|3|2070|684|8909|Thursday|2070Q3|N|N|N|2477356|2477598|2477015|2477289|N|N|N|N|N| +2477381|AAAAAAAAFENMFCAA|2070-09-26|2048|8909|684|2070|5|9|26|3|2070|684|8909|Friday|2070Q3|N|Y|N|2477356|2477598|2477016|2477290|N|N|N|N|N| +2477382|AAAAAAAAGENMFCAA|2070-09-27|2048|8909|684|2070|6|9|27|3|2070|684|8909|Saturday|2070Q3|N|Y|N|2477356|2477598|2477017|2477291|N|N|N|N|N| +2477383|AAAAAAAAHENMFCAA|2070-09-28|2048|8909|684|2070|0|9|28|3|2070|684|8909|Sunday|2070Q3|N|N|N|2477356|2477598|2477018|2477292|N|N|N|N|N| +2477384|AAAAAAAAIENMFCAA|2070-09-29|2048|8909|684|2070|1|9|29|3|2070|684|8909|Monday|2070Q3|N|N|N|2477356|2477598|2477019|2477293|N|N|N|N|N| +2477385|AAAAAAAAJENMFCAA|2070-09-30|2048|8910|684|2070|2|9|30|3|2070|684|8910|Tuesday|2070Q3|N|N|N|2477356|2477598|2477020|2477294|N|N|N|N|N| +2477386|AAAAAAAAKENMFCAA|2070-10-01|2049|8910|684|2070|3|10|1|3|2070|684|8910|Wednesday|2070Q3|N|N|N|2477386|2477658|2477021|2477294|N|N|N|N|N| +2477387|AAAAAAAALENMFCAA|2070-10-02|2049|8910|684|2070|4|10|2|4|2070|684|8910|Thursday|2070Q4|N|N|N|2477386|2477658|2477022|2477295|N|N|N|N|N| +2477388|AAAAAAAAMENMFCAA|2070-10-03|2049|8910|684|2070|5|10|3|4|2070|684|8910|Friday|2070Q4|N|Y|N|2477386|2477658|2477023|2477296|N|N|N|N|N| +2477389|AAAAAAAANENMFCAA|2070-10-04|2049|8910|684|2070|6|10|4|4|2070|684|8910|Saturday|2070Q4|N|Y|N|2477386|2477658|2477024|2477297|N|N|N|N|N| +2477390|AAAAAAAAOENMFCAA|2070-10-05|2049|8910|684|2070|0|10|5|4|2070|684|8910|Sunday|2070Q4|N|N|N|2477386|2477658|2477025|2477298|N|N|N|N|N| +2477391|AAAAAAAAPENMFCAA|2070-10-06|2049|8910|684|2070|1|10|6|4|2070|684|8910|Monday|2070Q4|N|N|N|2477386|2477658|2477026|2477299|N|N|N|N|N| +2477392|AAAAAAAAAFNMFCAA|2070-10-07|2049|8911|684|2070|2|10|7|4|2070|684|8911|Tuesday|2070Q4|N|N|N|2477386|2477658|2477027|2477300|N|N|N|N|N| +2477393|AAAAAAAABFNMFCAA|2070-10-08|2049|8911|684|2070|3|10|8|4|2070|684|8911|Wednesday|2070Q4|N|N|N|2477386|2477658|2477028|2477301|N|N|N|N|N| +2477394|AAAAAAAACFNMFCAA|2070-10-09|2049|8911|684|2070|4|10|9|4|2070|684|8911|Thursday|2070Q4|N|N|N|2477386|2477658|2477029|2477302|N|N|N|N|N| +2477395|AAAAAAAADFNMFCAA|2070-10-10|2049|8911|684|2070|5|10|10|4|2070|684|8911|Friday|2070Q4|N|Y|N|2477386|2477658|2477030|2477303|N|N|N|N|N| +2477396|AAAAAAAAEFNMFCAA|2070-10-11|2049|8911|684|2070|6|10|11|4|2070|684|8911|Saturday|2070Q4|N|Y|N|2477386|2477658|2477031|2477304|N|N|N|N|N| +2477397|AAAAAAAAFFNMFCAA|2070-10-12|2049|8911|684|2070|0|10|12|4|2070|684|8911|Sunday|2070Q4|N|N|N|2477386|2477658|2477032|2477305|N|N|N|N|N| +2477398|AAAAAAAAGFNMFCAA|2070-10-13|2049|8911|684|2070|1|10|13|4|2070|684|8911|Monday|2070Q4|N|N|N|2477386|2477658|2477033|2477306|N|N|N|N|N| +2477399|AAAAAAAAHFNMFCAA|2070-10-14|2049|8912|684|2070|2|10|14|4|2070|684|8912|Tuesday|2070Q4|N|N|N|2477386|2477658|2477034|2477307|N|N|N|N|N| +2477400|AAAAAAAAIFNMFCAA|2070-10-15|2049|8912|684|2070|3|10|15|4|2070|684|8912|Wednesday|2070Q4|N|N|N|2477386|2477658|2477035|2477308|N|N|N|N|N| +2477401|AAAAAAAAJFNMFCAA|2070-10-16|2049|8912|684|2070|4|10|16|4|2070|684|8912|Thursday|2070Q4|N|N|N|2477386|2477658|2477036|2477309|N|N|N|N|N| +2477402|AAAAAAAAKFNMFCAA|2070-10-17|2049|8912|684|2070|5|10|17|4|2070|684|8912|Friday|2070Q4|N|Y|N|2477386|2477658|2477037|2477310|N|N|N|N|N| +2477403|AAAAAAAALFNMFCAA|2070-10-18|2049|8912|684|2070|6|10|18|4|2070|684|8912|Saturday|2070Q4|N|Y|N|2477386|2477658|2477038|2477311|N|N|N|N|N| +2477404|AAAAAAAAMFNMFCAA|2070-10-19|2049|8912|684|2070|0|10|19|4|2070|684|8912|Sunday|2070Q4|N|N|N|2477386|2477658|2477039|2477312|N|N|N|N|N| +2477405|AAAAAAAANFNMFCAA|2070-10-20|2049|8912|684|2070|1|10|20|4|2070|684|8912|Monday|2070Q4|N|N|N|2477386|2477658|2477040|2477313|N|N|N|N|N| +2477406|AAAAAAAAOFNMFCAA|2070-10-21|2049|8913|684|2070|2|10|21|4|2070|684|8913|Tuesday|2070Q4|N|N|N|2477386|2477658|2477041|2477314|N|N|N|N|N| +2477407|AAAAAAAAPFNMFCAA|2070-10-22|2049|8913|684|2070|3|10|22|4|2070|684|8913|Wednesday|2070Q4|N|N|N|2477386|2477658|2477042|2477315|N|N|N|N|N| +2477408|AAAAAAAAAGNMFCAA|2070-10-23|2049|8913|684|2070|4|10|23|4|2070|684|8913|Thursday|2070Q4|N|N|N|2477386|2477658|2477043|2477316|N|N|N|N|N| +2477409|AAAAAAAABGNMFCAA|2070-10-24|2049|8913|684|2070|5|10|24|4|2070|684|8913|Friday|2070Q4|N|Y|N|2477386|2477658|2477044|2477317|N|N|N|N|N| +2477410|AAAAAAAACGNMFCAA|2070-10-25|2049|8913|684|2070|6|10|25|4|2070|684|8913|Saturday|2070Q4|N|Y|N|2477386|2477658|2477045|2477318|N|N|N|N|N| +2477411|AAAAAAAADGNMFCAA|2070-10-26|2049|8913|684|2070|0|10|26|4|2070|684|8913|Sunday|2070Q4|N|N|N|2477386|2477658|2477046|2477319|N|N|N|N|N| +2477412|AAAAAAAAEGNMFCAA|2070-10-27|2049|8913|684|2070|1|10|27|4|2070|684|8913|Monday|2070Q4|N|N|N|2477386|2477658|2477047|2477320|N|N|N|N|N| +2477413|AAAAAAAAFGNMFCAA|2070-10-28|2049|8914|684|2070|2|10|28|4|2070|684|8914|Tuesday|2070Q4|N|N|N|2477386|2477658|2477048|2477321|N|N|N|N|N| +2477414|AAAAAAAAGGNMFCAA|2070-10-29|2049|8914|684|2070|3|10|29|4|2070|684|8914|Wednesday|2070Q4|N|N|N|2477386|2477658|2477049|2477322|N|N|N|N|N| +2477415|AAAAAAAAHGNMFCAA|2070-10-30|2049|8914|684|2070|4|10|30|4|2070|684|8914|Thursday|2070Q4|N|N|N|2477386|2477658|2477050|2477323|N|N|N|N|N| +2477416|AAAAAAAAIGNMFCAA|2070-10-31|2049|8914|684|2070|5|10|31|4|2070|684|8914|Friday|2070Q4|N|Y|N|2477386|2477658|2477051|2477324|N|N|N|N|N| +2477417|AAAAAAAAJGNMFCAA|2070-11-01|2050|8914|684|2070|6|11|1|4|2070|684|8914|Saturday|2070Q4|N|Y|N|2477417|2477720|2477052|2477325|N|N|N|N|N| +2477418|AAAAAAAAKGNMFCAA|2070-11-02|2050|8914|684|2070|0|11|2|4|2070|684|8914|Sunday|2070Q4|N|N|N|2477417|2477720|2477053|2477326|N|N|N|N|N| +2477419|AAAAAAAALGNMFCAA|2070-11-03|2050|8914|684|2070|1|11|3|4|2070|684|8914|Monday|2070Q4|N|N|N|2477417|2477720|2477054|2477327|N|N|N|N|N| +2477420|AAAAAAAAMGNMFCAA|2070-11-04|2050|8915|684|2070|2|11|4|4|2070|684|8915|Tuesday|2070Q4|N|N|N|2477417|2477720|2477055|2477328|N|N|N|N|N| +2477421|AAAAAAAANGNMFCAA|2070-11-05|2050|8915|684|2070|3|11|5|4|2070|684|8915|Wednesday|2070Q4|N|N|N|2477417|2477720|2477056|2477329|N|N|N|N|N| +2477422|AAAAAAAAOGNMFCAA|2070-11-06|2050|8915|684|2070|4|11|6|4|2070|684|8915|Thursday|2070Q4|N|N|N|2477417|2477720|2477057|2477330|N|N|N|N|N| +2477423|AAAAAAAAPGNMFCAA|2070-11-07|2050|8915|684|2070|5|11|7|4|2070|684|8915|Friday|2070Q4|N|Y|N|2477417|2477720|2477058|2477331|N|N|N|N|N| +2477424|AAAAAAAAAHNMFCAA|2070-11-08|2050|8915|684|2070|6|11|8|4|2070|684|8915|Saturday|2070Q4|N|Y|N|2477417|2477720|2477059|2477332|N|N|N|N|N| +2477425|AAAAAAAABHNMFCAA|2070-11-09|2050|8915|684|2070|0|11|9|4|2070|684|8915|Sunday|2070Q4|N|N|N|2477417|2477720|2477060|2477333|N|N|N|N|N| +2477426|AAAAAAAACHNMFCAA|2070-11-10|2050|8915|684|2070|1|11|10|4|2070|684|8915|Monday|2070Q4|N|N|N|2477417|2477720|2477061|2477334|N|N|N|N|N| +2477427|AAAAAAAADHNMFCAA|2070-11-11|2050|8916|684|2070|2|11|11|4|2070|684|8916|Tuesday|2070Q4|N|N|N|2477417|2477720|2477062|2477335|N|N|N|N|N| +2477428|AAAAAAAAEHNMFCAA|2070-11-12|2050|8916|684|2070|3|11|12|4|2070|684|8916|Wednesday|2070Q4|N|N|N|2477417|2477720|2477063|2477336|N|N|N|N|N| +2477429|AAAAAAAAFHNMFCAA|2070-11-13|2050|8916|684|2070|4|11|13|4|2070|684|8916|Thursday|2070Q4|N|N|N|2477417|2477720|2477064|2477337|N|N|N|N|N| +2477430|AAAAAAAAGHNMFCAA|2070-11-14|2050|8916|684|2070|5|11|14|4|2070|684|8916|Friday|2070Q4|N|Y|N|2477417|2477720|2477065|2477338|N|N|N|N|N| +2477431|AAAAAAAAHHNMFCAA|2070-11-15|2050|8916|684|2070|6|11|15|4|2070|684|8916|Saturday|2070Q4|N|Y|N|2477417|2477720|2477066|2477339|N|N|N|N|N| +2477432|AAAAAAAAIHNMFCAA|2070-11-16|2050|8916|684|2070|0|11|16|4|2070|684|8916|Sunday|2070Q4|N|N|N|2477417|2477720|2477067|2477340|N|N|N|N|N| +2477433|AAAAAAAAJHNMFCAA|2070-11-17|2050|8916|684|2070|1|11|17|4|2070|684|8916|Monday|2070Q4|N|N|N|2477417|2477720|2477068|2477341|N|N|N|N|N| +2477434|AAAAAAAAKHNMFCAA|2070-11-18|2050|8917|684|2070|2|11|18|4|2070|684|8917|Tuesday|2070Q4|N|N|N|2477417|2477720|2477069|2477342|N|N|N|N|N| +2477435|AAAAAAAALHNMFCAA|2070-11-19|2050|8917|684|2070|3|11|19|4|2070|684|8917|Wednesday|2070Q4|N|N|N|2477417|2477720|2477070|2477343|N|N|N|N|N| +2477436|AAAAAAAAMHNMFCAA|2070-11-20|2050|8917|684|2070|4|11|20|4|2070|684|8917|Thursday|2070Q4|N|N|N|2477417|2477720|2477071|2477344|N|N|N|N|N| +2477437|AAAAAAAANHNMFCAA|2070-11-21|2050|8917|684|2070|5|11|21|4|2070|684|8917|Friday|2070Q4|N|Y|N|2477417|2477720|2477072|2477345|N|N|N|N|N| +2477438|AAAAAAAAOHNMFCAA|2070-11-22|2050|8917|684|2070|6|11|22|4|2070|684|8917|Saturday|2070Q4|N|Y|N|2477417|2477720|2477073|2477346|N|N|N|N|N| +2477439|AAAAAAAAPHNMFCAA|2070-11-23|2050|8917|684|2070|0|11|23|4|2070|684|8917|Sunday|2070Q4|N|N|N|2477417|2477720|2477074|2477347|N|N|N|N|N| +2477440|AAAAAAAAAINMFCAA|2070-11-24|2050|8917|684|2070|1|11|24|4|2070|684|8917|Monday|2070Q4|N|N|N|2477417|2477720|2477075|2477348|N|N|N|N|N| +2477441|AAAAAAAABINMFCAA|2070-11-25|2050|8918|684|2070|2|11|25|4|2070|684|8918|Tuesday|2070Q4|N|N|N|2477417|2477720|2477076|2477349|N|N|N|N|N| +2477442|AAAAAAAACINMFCAA|2070-11-26|2050|8918|684|2070|3|11|26|4|2070|684|8918|Wednesday|2070Q4|N|N|N|2477417|2477720|2477077|2477350|N|N|N|N|N| +2477443|AAAAAAAADINMFCAA|2070-11-27|2050|8918|684|2070|4|11|27|4|2070|684|8918|Thursday|2070Q4|N|N|N|2477417|2477720|2477078|2477351|N|N|N|N|N| +2477444|AAAAAAAAEINMFCAA|2070-11-28|2050|8918|684|2070|5|11|28|4|2070|684|8918|Friday|2070Q4|N|Y|N|2477417|2477720|2477079|2477352|N|N|N|N|N| +2477445|AAAAAAAAFINMFCAA|2070-11-29|2050|8918|684|2070|6|11|29|4|2070|684|8918|Saturday|2070Q4|N|Y|N|2477417|2477720|2477080|2477353|N|N|N|N|N| +2477446|AAAAAAAAGINMFCAA|2070-11-30|2050|8918|684|2070|0|11|30|4|2070|684|8918|Sunday|2070Q4|N|N|N|2477417|2477720|2477081|2477354|N|N|N|N|N| +2477447|AAAAAAAAHINMFCAA|2070-12-01|2051|8918|685|2070|1|12|1|4|2070|685|8918|Monday|2070Q4|N|N|N|2477447|2477780|2477082|2477355|N|N|N|N|N| +2477448|AAAAAAAAIINMFCAA|2070-12-02|2051|8919|685|2070|2|12|2|4|2070|685|8919|Tuesday|2070Q4|N|N|N|2477447|2477780|2477083|2477356|N|N|N|N|N| +2477449|AAAAAAAAJINMFCAA|2070-12-03|2051|8919|685|2070|3|12|3|4|2070|685|8919|Wednesday|2070Q4|N|N|N|2477447|2477780|2477084|2477357|N|N|N|N|N| +2477450|AAAAAAAAKINMFCAA|2070-12-04|2051|8919|685|2070|4|12|4|4|2070|685|8919|Thursday|2070Q4|N|N|N|2477447|2477780|2477085|2477358|N|N|N|N|N| +2477451|AAAAAAAALINMFCAA|2070-12-05|2051|8919|685|2070|5|12|5|4|2070|685|8919|Friday|2070Q4|N|Y|N|2477447|2477780|2477086|2477359|N|N|N|N|N| +2477452|AAAAAAAAMINMFCAA|2070-12-06|2051|8919|685|2070|6|12|6|4|2070|685|8919|Saturday|2070Q4|N|Y|N|2477447|2477780|2477087|2477360|N|N|N|N|N| +2477453|AAAAAAAANINMFCAA|2070-12-07|2051|8919|685|2070|0|12|7|4|2070|685|8919|Sunday|2070Q4|N|N|N|2477447|2477780|2477088|2477361|N|N|N|N|N| +2477454|AAAAAAAAOINMFCAA|2070-12-08|2051|8919|685|2070|1|12|8|4|2070|685|8919|Monday|2070Q4|N|N|N|2477447|2477780|2477089|2477362|N|N|N|N|N| +2477455|AAAAAAAAPINMFCAA|2070-12-09|2051|8920|685|2070|2|12|9|4|2070|685|8920|Tuesday|2070Q4|N|N|N|2477447|2477780|2477090|2477363|N|N|N|N|N| +2477456|AAAAAAAAAJNMFCAA|2070-12-10|2051|8920|685|2070|3|12|10|4|2070|685|8920|Wednesday|2070Q4|N|N|N|2477447|2477780|2477091|2477364|N|N|N|N|N| +2477457|AAAAAAAABJNMFCAA|2070-12-11|2051|8920|685|2070|4|12|11|4|2070|685|8920|Thursday|2070Q4|N|N|N|2477447|2477780|2477092|2477365|N|N|N|N|N| +2477458|AAAAAAAACJNMFCAA|2070-12-12|2051|8920|685|2070|5|12|12|4|2070|685|8920|Friday|2070Q4|N|Y|N|2477447|2477780|2477093|2477366|N|N|N|N|N| +2477459|AAAAAAAADJNMFCAA|2070-12-13|2051|8920|685|2070|6|12|13|4|2070|685|8920|Saturday|2070Q4|N|Y|N|2477447|2477780|2477094|2477367|N|N|N|N|N| +2477460|AAAAAAAAEJNMFCAA|2070-12-14|2051|8920|685|2070|0|12|14|4|2070|685|8920|Sunday|2070Q4|N|N|N|2477447|2477780|2477095|2477368|N|N|N|N|N| +2477461|AAAAAAAAFJNMFCAA|2070-12-15|2051|8920|685|2070|1|12|15|4|2070|685|8920|Monday|2070Q4|N|N|N|2477447|2477780|2477096|2477369|N|N|N|N|N| +2477462|AAAAAAAAGJNMFCAA|2070-12-16|2051|8921|685|2070|2|12|16|4|2070|685|8921|Tuesday|2070Q4|N|N|N|2477447|2477780|2477097|2477370|N|N|N|N|N| +2477463|AAAAAAAAHJNMFCAA|2070-12-17|2051|8921|685|2070|3|12|17|4|2070|685|8921|Wednesday|2070Q4|N|N|N|2477447|2477780|2477098|2477371|N|N|N|N|N| +2477464|AAAAAAAAIJNMFCAA|2070-12-18|2051|8921|685|2070|4|12|18|4|2070|685|8921|Thursday|2070Q4|N|N|N|2477447|2477780|2477099|2477372|N|N|N|N|N| +2477465|AAAAAAAAJJNMFCAA|2070-12-19|2051|8921|685|2070|5|12|19|4|2070|685|8921|Friday|2070Q4|N|Y|N|2477447|2477780|2477100|2477373|N|N|N|N|N| +2477466|AAAAAAAAKJNMFCAA|2070-12-20|2051|8921|685|2070|6|12|20|4|2070|685|8921|Saturday|2070Q4|N|Y|N|2477447|2477780|2477101|2477374|N|N|N|N|N| +2477467|AAAAAAAALJNMFCAA|2070-12-21|2051|8921|685|2070|0|12|21|4|2070|685|8921|Sunday|2070Q4|N|N|N|2477447|2477780|2477102|2477375|N|N|N|N|N| +2477468|AAAAAAAAMJNMFCAA|2070-12-22|2051|8921|685|2070|1|12|22|4|2070|685|8921|Monday|2070Q4|N|N|N|2477447|2477780|2477103|2477376|N|N|N|N|N| +2477469|AAAAAAAANJNMFCAA|2070-12-23|2051|8922|685|2070|2|12|23|4|2070|685|8922|Tuesday|2070Q4|N|N|N|2477447|2477780|2477104|2477377|N|N|N|N|N| +2477470|AAAAAAAAOJNMFCAA|2070-12-24|2051|8922|685|2070|3|12|24|4|2070|685|8922|Wednesday|2070Q4|N|N|N|2477447|2477780|2477105|2477378|N|N|N|N|N| +2477471|AAAAAAAAPJNMFCAA|2070-12-25|2051|8922|685|2070|4|12|25|4|2070|685|8922|Thursday|2070Q4|N|N|N|2477447|2477780|2477106|2477379|N|N|N|N|N| +2477472|AAAAAAAAAKNMFCAA|2070-12-26|2051|8922|685|2070|5|12|26|4|2070|685|8922|Friday|2070Q4|Y|Y|N|2477447|2477780|2477107|2477380|N|N|N|N|N| +2477473|AAAAAAAABKNMFCAA|2070-12-27|2051|8922|685|2070|6|12|27|4|2070|685|8922|Saturday|2070Q4|N|Y|Y|2477447|2477780|2477108|2477381|N|N|N|N|N| +2477474|AAAAAAAACKNMFCAA|2070-12-28|2051|8922|685|2070|0|12|28|4|2070|685|8922|Sunday|2070Q4|N|N|N|2477447|2477780|2477109|2477382|N|N|N|N|N| +2477475|AAAAAAAADKNMFCAA|2070-12-29|2051|8922|685|2070|1|12|29|4|2070|685|8922|Monday|2070Q4|N|N|N|2477447|2477780|2477110|2477383|N|N|N|N|N| +2477476|AAAAAAAAEKNMFCAA|2070-12-30|2051|8923|685|2070|2|12|30|4|2070|685|8923|Tuesday|2070Q4|N|N|N|2477447|2477780|2477111|2477384|N|N|N|N|N| +2477477|AAAAAAAAFKNMFCAA|2070-12-31|2051|8923|685|2070|3|12|31|4|2070|685|8923|Wednesday|2070Q4|N|N|N|2477447|2477780|2477112|2477385|N|N|N|N|N| +2477478|AAAAAAAAGKNMFCAA|2071-01-01|2052|8923|685|2071|4|1|1|1|2071|685|8923|Thursday|2071Q1|Y|N|N|2477478|2477477|2477113|2477386|N|N|N|N|N| +2477479|AAAAAAAAHKNMFCAA|2071-01-02|2052|8923|685|2071|5|1|2|1|2071|685|8923|Friday|2071Q1|N|Y|Y|2477478|2477477|2477114|2477387|N|N|N|N|N| +2477480|AAAAAAAAIKNMFCAA|2071-01-03|2052|8923|685|2071|6|1|3|1|2071|685|8923|Saturday|2071Q1|N|Y|N|2477478|2477477|2477115|2477388|N|N|N|N|N| +2477481|AAAAAAAAJKNMFCAA|2071-01-04|2052|8923|685|2071|0|1|4|1|2071|685|8923|Sunday|2071Q1|N|N|N|2477478|2477477|2477116|2477389|N|N|N|N|N| +2477482|AAAAAAAAKKNMFCAA|2071-01-05|2052|8923|685|2071|1|1|5|1|2071|685|8923|Monday|2071Q1|N|N|N|2477478|2477477|2477117|2477390|N|N|N|N|N| +2477483|AAAAAAAALKNMFCAA|2071-01-06|2052|8924|685|2071|2|1|6|1|2071|685|8924|Tuesday|2071Q1|N|N|N|2477478|2477477|2477118|2477391|N|N|N|N|N| +2477484|AAAAAAAAMKNMFCAA|2071-01-07|2052|8924|685|2071|3|1|7|1|2071|685|8924|Wednesday|2071Q1|N|N|N|2477478|2477477|2477119|2477392|N|N|N|N|N| +2477485|AAAAAAAANKNMFCAA|2071-01-08|2052|8924|685|2071|4|1|8|1|2071|685|8924|Thursday|2071Q1|N|N|N|2477478|2477477|2477120|2477393|N|N|N|N|N| +2477486|AAAAAAAAOKNMFCAA|2071-01-09|2052|8924|685|2071|5|1|9|1|2071|685|8924|Friday|2071Q1|N|Y|N|2477478|2477477|2477121|2477394|N|N|N|N|N| +2477487|AAAAAAAAPKNMFCAA|2071-01-10|2052|8924|685|2071|6|1|10|1|2071|685|8924|Saturday|2071Q1|N|Y|N|2477478|2477477|2477122|2477395|N|N|N|N|N| +2477488|AAAAAAAAALNMFCAA|2071-01-11|2052|8924|685|2071|0|1|11|1|2071|685|8924|Sunday|2071Q1|N|N|N|2477478|2477477|2477123|2477396|N|N|N|N|N| +2477489|AAAAAAAABLNMFCAA|2071-01-12|2052|8924|685|2071|1|1|12|1|2071|685|8924|Monday|2071Q1|N|N|N|2477478|2477477|2477124|2477397|N|N|N|N|N| +2477490|AAAAAAAACLNMFCAA|2071-01-13|2052|8925|685|2071|2|1|13|1|2071|685|8925|Tuesday|2071Q1|N|N|N|2477478|2477477|2477125|2477398|N|N|N|N|N| +2477491|AAAAAAAADLNMFCAA|2071-01-14|2052|8925|685|2071|3|1|14|1|2071|685|8925|Wednesday|2071Q1|N|N|N|2477478|2477477|2477126|2477399|N|N|N|N|N| +2477492|AAAAAAAAELNMFCAA|2071-01-15|2052|8925|685|2071|4|1|15|1|2071|685|8925|Thursday|2071Q1|N|N|N|2477478|2477477|2477127|2477400|N|N|N|N|N| +2477493|AAAAAAAAFLNMFCAA|2071-01-16|2052|8925|685|2071|5|1|16|1|2071|685|8925|Friday|2071Q1|N|Y|N|2477478|2477477|2477128|2477401|N|N|N|N|N| +2477494|AAAAAAAAGLNMFCAA|2071-01-17|2052|8925|685|2071|6|1|17|1|2071|685|8925|Saturday|2071Q1|N|Y|N|2477478|2477477|2477129|2477402|N|N|N|N|N| +2477495|AAAAAAAAHLNMFCAA|2071-01-18|2052|8925|685|2071|0|1|18|1|2071|685|8925|Sunday|2071Q1|N|N|N|2477478|2477477|2477130|2477403|N|N|N|N|N| +2477496|AAAAAAAAILNMFCAA|2071-01-19|2052|8925|685|2071|1|1|19|1|2071|685|8925|Monday|2071Q1|N|N|N|2477478|2477477|2477131|2477404|N|N|N|N|N| +2477497|AAAAAAAAJLNMFCAA|2071-01-20|2052|8926|685|2071|2|1|20|1|2071|685|8926|Tuesday|2071Q1|N|N|N|2477478|2477477|2477132|2477405|N|N|N|N|N| +2477498|AAAAAAAAKLNMFCAA|2071-01-21|2052|8926|685|2071|3|1|21|1|2071|685|8926|Wednesday|2071Q1|N|N|N|2477478|2477477|2477133|2477406|N|N|N|N|N| +2477499|AAAAAAAALLNMFCAA|2071-01-22|2052|8926|685|2071|4|1|22|1|2071|685|8926|Thursday|2071Q1|N|N|N|2477478|2477477|2477134|2477407|N|N|N|N|N| +2477500|AAAAAAAAMLNMFCAA|2071-01-23|2052|8926|685|2071|5|1|23|1|2071|685|8926|Friday|2071Q1|N|Y|N|2477478|2477477|2477135|2477408|N|N|N|N|N| +2477501|AAAAAAAANLNMFCAA|2071-01-24|2052|8926|685|2071|6|1|24|1|2071|685|8926|Saturday|2071Q1|N|Y|N|2477478|2477477|2477136|2477409|N|N|N|N|N| +2477502|AAAAAAAAOLNMFCAA|2071-01-25|2052|8926|685|2071|0|1|25|1|2071|685|8926|Sunday|2071Q1|N|N|N|2477478|2477477|2477137|2477410|N|N|N|N|N| +2477503|AAAAAAAAPLNMFCAA|2071-01-26|2052|8926|685|2071|1|1|26|1|2071|685|8926|Monday|2071Q1|N|N|N|2477478|2477477|2477138|2477411|N|N|N|N|N| +2477504|AAAAAAAAAMNMFCAA|2071-01-27|2052|8927|685|2071|2|1|27|1|2071|685|8927|Tuesday|2071Q1|N|N|N|2477478|2477477|2477139|2477412|N|N|N|N|N| +2477505|AAAAAAAABMNMFCAA|2071-01-28|2052|8927|685|2071|3|1|28|1|2071|685|8927|Wednesday|2071Q1|N|N|N|2477478|2477477|2477140|2477413|N|N|N|N|N| +2477506|AAAAAAAACMNMFCAA|2071-01-29|2052|8927|685|2071|4|1|29|1|2071|685|8927|Thursday|2071Q1|N|N|N|2477478|2477477|2477141|2477414|N|N|N|N|N| +2477507|AAAAAAAADMNMFCAA|2071-01-30|2052|8927|685|2071|5|1|30|1|2071|685|8927|Friday|2071Q1|N|Y|N|2477478|2477477|2477142|2477415|N|N|N|N|N| +2477508|AAAAAAAAEMNMFCAA|2071-01-31|2052|8927|685|2071|6|1|31|1|2071|685|8927|Saturday|2071Q1|N|Y|N|2477478|2477477|2477143|2477416|N|N|N|N|N| +2477509|AAAAAAAAFMNMFCAA|2071-02-01|2053|8927|685|2071|0|2|1|1|2071|685|8927|Sunday|2071Q1|N|N|N|2477509|2477539|2477144|2477417|N|N|N|N|N| +2477510|AAAAAAAAGMNMFCAA|2071-02-02|2053|8927|685|2071|1|2|2|1|2071|685|8927|Monday|2071Q1|N|N|N|2477509|2477539|2477145|2477418|N|N|N|N|N| +2477511|AAAAAAAAHMNMFCAA|2071-02-03|2053|8928|685|2071|2|2|3|1|2071|685|8928|Tuesday|2071Q1|N|N|N|2477509|2477539|2477146|2477419|N|N|N|N|N| +2477512|AAAAAAAAIMNMFCAA|2071-02-04|2053|8928|685|2071|3|2|4|1|2071|685|8928|Wednesday|2071Q1|N|N|N|2477509|2477539|2477147|2477420|N|N|N|N|N| +2477513|AAAAAAAAJMNMFCAA|2071-02-05|2053|8928|685|2071|4|2|5|1|2071|685|8928|Thursday|2071Q1|N|N|N|2477509|2477539|2477148|2477421|N|N|N|N|N| +2477514|AAAAAAAAKMNMFCAA|2071-02-06|2053|8928|685|2071|5|2|6|1|2071|685|8928|Friday|2071Q1|N|Y|N|2477509|2477539|2477149|2477422|N|N|N|N|N| +2477515|AAAAAAAALMNMFCAA|2071-02-07|2053|8928|685|2071|6|2|7|1|2071|685|8928|Saturday|2071Q1|N|Y|N|2477509|2477539|2477150|2477423|N|N|N|N|N| +2477516|AAAAAAAAMMNMFCAA|2071-02-08|2053|8928|685|2071|0|2|8|1|2071|685|8928|Sunday|2071Q1|N|N|N|2477509|2477539|2477151|2477424|N|N|N|N|N| +2477517|AAAAAAAANMNMFCAA|2071-02-09|2053|8928|685|2071|1|2|9|1|2071|685|8928|Monday|2071Q1|N|N|N|2477509|2477539|2477152|2477425|N|N|N|N|N| +2477518|AAAAAAAAOMNMFCAA|2071-02-10|2053|8929|685|2071|2|2|10|1|2071|685|8929|Tuesday|2071Q1|N|N|N|2477509|2477539|2477153|2477426|N|N|N|N|N| +2477519|AAAAAAAAPMNMFCAA|2071-02-11|2053|8929|685|2071|3|2|11|1|2071|685|8929|Wednesday|2071Q1|N|N|N|2477509|2477539|2477154|2477427|N|N|N|N|N| +2477520|AAAAAAAAANNMFCAA|2071-02-12|2053|8929|685|2071|4|2|12|1|2071|685|8929|Thursday|2071Q1|N|N|N|2477509|2477539|2477155|2477428|N|N|N|N|N| +2477521|AAAAAAAABNNMFCAA|2071-02-13|2053|8929|685|2071|5|2|13|1|2071|685|8929|Friday|2071Q1|N|Y|N|2477509|2477539|2477156|2477429|N|N|N|N|N| +2477522|AAAAAAAACNNMFCAA|2071-02-14|2053|8929|685|2071|6|2|14|1|2071|685|8929|Saturday|2071Q1|N|Y|N|2477509|2477539|2477157|2477430|N|N|N|N|N| +2477523|AAAAAAAADNNMFCAA|2071-02-15|2053|8929|685|2071|0|2|15|1|2071|685|8929|Sunday|2071Q1|N|N|N|2477509|2477539|2477158|2477431|N|N|N|N|N| +2477524|AAAAAAAAENNMFCAA|2071-02-16|2053|8929|685|2071|1|2|16|1|2071|685|8929|Monday|2071Q1|N|N|N|2477509|2477539|2477159|2477432|N|N|N|N|N| +2477525|AAAAAAAAFNNMFCAA|2071-02-17|2053|8930|685|2071|2|2|17|1|2071|685|8930|Tuesday|2071Q1|N|N|N|2477509|2477539|2477160|2477433|N|N|N|N|N| +2477526|AAAAAAAAGNNMFCAA|2071-02-18|2053|8930|685|2071|3|2|18|1|2071|685|8930|Wednesday|2071Q1|N|N|N|2477509|2477539|2477161|2477434|N|N|N|N|N| +2477527|AAAAAAAAHNNMFCAA|2071-02-19|2053|8930|685|2071|4|2|19|1|2071|685|8930|Thursday|2071Q1|N|N|N|2477509|2477539|2477162|2477435|N|N|N|N|N| +2477528|AAAAAAAAINNMFCAA|2071-02-20|2053|8930|685|2071|5|2|20|1|2071|685|8930|Friday|2071Q1|N|Y|N|2477509|2477539|2477163|2477436|N|N|N|N|N| +2477529|AAAAAAAAJNNMFCAA|2071-02-21|2053|8930|685|2071|6|2|21|1|2071|685|8930|Saturday|2071Q1|N|Y|N|2477509|2477539|2477164|2477437|N|N|N|N|N| +2477530|AAAAAAAAKNNMFCAA|2071-02-22|2053|8930|685|2071|0|2|22|1|2071|685|8930|Sunday|2071Q1|N|N|N|2477509|2477539|2477165|2477438|N|N|N|N|N| +2477531|AAAAAAAALNNMFCAA|2071-02-23|2053|8930|685|2071|1|2|23|1|2071|685|8930|Monday|2071Q1|N|N|N|2477509|2477539|2477166|2477439|N|N|N|N|N| +2477532|AAAAAAAAMNNMFCAA|2071-02-24|2053|8931|685|2071|2|2|24|1|2071|685|8931|Tuesday|2071Q1|N|N|N|2477509|2477539|2477167|2477440|N|N|N|N|N| +2477533|AAAAAAAANNNMFCAA|2071-02-25|2053|8931|685|2071|3|2|25|1|2071|685|8931|Wednesday|2071Q1|N|N|N|2477509|2477539|2477168|2477441|N|N|N|N|N| +2477534|AAAAAAAAONNMFCAA|2071-02-26|2053|8931|685|2071|4|2|26|1|2071|685|8931|Thursday|2071Q1|N|N|N|2477509|2477539|2477169|2477442|N|N|N|N|N| +2477535|AAAAAAAAPNNMFCAA|2071-02-27|2053|8931|685|2071|5|2|27|1|2071|685|8931|Friday|2071Q1|N|Y|N|2477509|2477539|2477170|2477443|N|N|N|N|N| +2477536|AAAAAAAAAONMFCAA|2071-02-28|2053|8931|685|2071|6|2|28|1|2071|685|8931|Saturday|2071Q1|N|Y|N|2477509|2477539|2477171|2477444|N|N|N|N|N| +2477537|AAAAAAAABONMFCAA|2071-03-01|2054|8931|686|2071|0|3|1|1|2071|686|8931|Sunday|2071Q1|N|N|N|2477537|2477595|2477172|2477445|N|N|N|N|N| +2477538|AAAAAAAACONMFCAA|2071-03-02|2054|8931|686|2071|1|3|2|1|2071|686|8931|Monday|2071Q1|N|N|N|2477537|2477595|2477173|2477446|N|N|N|N|N| +2477539|AAAAAAAADONMFCAA|2071-03-03|2054|8932|686|2071|2|3|3|1|2071|686|8932|Tuesday|2071Q1|N|N|N|2477537|2477595|2477174|2477447|N|N|N|N|N| +2477540|AAAAAAAAEONMFCAA|2071-03-04|2054|8932|686|2071|3|3|4|1|2071|686|8932|Wednesday|2071Q1|N|N|N|2477537|2477595|2477175|2477448|N|N|N|N|N| +2477541|AAAAAAAAFONMFCAA|2071-03-05|2054|8932|686|2071|4|3|5|1|2071|686|8932|Thursday|2071Q1|N|N|N|2477537|2477595|2477176|2477449|N|N|N|N|N| +2477542|AAAAAAAAGONMFCAA|2071-03-06|2054|8932|686|2071|5|3|6|1|2071|686|8932|Friday|2071Q1|N|Y|N|2477537|2477595|2477177|2477450|N|N|N|N|N| +2477543|AAAAAAAAHONMFCAA|2071-03-07|2054|8932|686|2071|6|3|7|1|2071|686|8932|Saturday|2071Q1|N|Y|N|2477537|2477595|2477178|2477451|N|N|N|N|N| +2477544|AAAAAAAAIONMFCAA|2071-03-08|2054|8932|686|2071|0|3|8|1|2071|686|8932|Sunday|2071Q1|N|N|N|2477537|2477595|2477179|2477452|N|N|N|N|N| +2477545|AAAAAAAAJONMFCAA|2071-03-09|2054|8932|686|2071|1|3|9|1|2071|686|8932|Monday|2071Q1|N|N|N|2477537|2477595|2477180|2477453|N|N|N|N|N| +2477546|AAAAAAAAKONMFCAA|2071-03-10|2054|8933|686|2071|2|3|10|1|2071|686|8933|Tuesday|2071Q1|N|N|N|2477537|2477595|2477181|2477454|N|N|N|N|N| +2477547|AAAAAAAALONMFCAA|2071-03-11|2054|8933|686|2071|3|3|11|1|2071|686|8933|Wednesday|2071Q1|N|N|N|2477537|2477595|2477182|2477455|N|N|N|N|N| +2477548|AAAAAAAAMONMFCAA|2071-03-12|2054|8933|686|2071|4|3|12|1|2071|686|8933|Thursday|2071Q1|N|N|N|2477537|2477595|2477183|2477456|N|N|N|N|N| +2477549|AAAAAAAANONMFCAA|2071-03-13|2054|8933|686|2071|5|3|13|1|2071|686|8933|Friday|2071Q1|N|Y|N|2477537|2477595|2477184|2477457|N|N|N|N|N| +2477550|AAAAAAAAOONMFCAA|2071-03-14|2054|8933|686|2071|6|3|14|1|2071|686|8933|Saturday|2071Q1|N|Y|N|2477537|2477595|2477185|2477458|N|N|N|N|N| +2477551|AAAAAAAAPONMFCAA|2071-03-15|2054|8933|686|2071|0|3|15|1|2071|686|8933|Sunday|2071Q1|N|N|N|2477537|2477595|2477186|2477459|N|N|N|N|N| +2477552|AAAAAAAAAPNMFCAA|2071-03-16|2054|8933|686|2071|1|3|16|1|2071|686|8933|Monday|2071Q1|N|N|N|2477537|2477595|2477187|2477460|N|N|N|N|N| +2477553|AAAAAAAABPNMFCAA|2071-03-17|2054|8934|686|2071|2|3|17|1|2071|686|8934|Tuesday|2071Q1|N|N|N|2477537|2477595|2477188|2477461|N|N|N|N|N| +2477554|AAAAAAAACPNMFCAA|2071-03-18|2054|8934|686|2071|3|3|18|1|2071|686|8934|Wednesday|2071Q1|N|N|N|2477537|2477595|2477189|2477462|N|N|N|N|N| +2477555|AAAAAAAADPNMFCAA|2071-03-19|2054|8934|686|2071|4|3|19|1|2071|686|8934|Thursday|2071Q1|N|N|N|2477537|2477595|2477190|2477463|N|N|N|N|N| +2477556|AAAAAAAAEPNMFCAA|2071-03-20|2054|8934|686|2071|5|3|20|1|2071|686|8934|Friday|2071Q1|N|Y|N|2477537|2477595|2477191|2477464|N|N|N|N|N| +2477557|AAAAAAAAFPNMFCAA|2071-03-21|2054|8934|686|2071|6|3|21|1|2071|686|8934|Saturday|2071Q1|N|Y|N|2477537|2477595|2477192|2477465|N|N|N|N|N| +2477558|AAAAAAAAGPNMFCAA|2071-03-22|2054|8934|686|2071|0|3|22|1|2071|686|8934|Sunday|2071Q1|N|N|N|2477537|2477595|2477193|2477466|N|N|N|N|N| +2477559|AAAAAAAAHPNMFCAA|2071-03-23|2054|8934|686|2071|1|3|23|1|2071|686|8934|Monday|2071Q1|N|N|N|2477537|2477595|2477194|2477467|N|N|N|N|N| +2477560|AAAAAAAAIPNMFCAA|2071-03-24|2054|8935|686|2071|2|3|24|1|2071|686|8935|Tuesday|2071Q1|N|N|N|2477537|2477595|2477195|2477468|N|N|N|N|N| +2477561|AAAAAAAAJPNMFCAA|2071-03-25|2054|8935|686|2071|3|3|25|1|2071|686|8935|Wednesday|2071Q1|N|N|N|2477537|2477595|2477196|2477469|N|N|N|N|N| +2477562|AAAAAAAAKPNMFCAA|2071-03-26|2054|8935|686|2071|4|3|26|1|2071|686|8935|Thursday|2071Q1|N|N|N|2477537|2477595|2477197|2477470|N|N|N|N|N| +2477563|AAAAAAAALPNMFCAA|2071-03-27|2054|8935|686|2071|5|3|27|1|2071|686|8935|Friday|2071Q1|N|Y|N|2477537|2477595|2477198|2477471|N|N|N|N|N| +2477564|AAAAAAAAMPNMFCAA|2071-03-28|2054|8935|686|2071|6|3|28|1|2071|686|8935|Saturday|2071Q1|N|Y|N|2477537|2477595|2477199|2477472|N|N|N|N|N| +2477565|AAAAAAAANPNMFCAA|2071-03-29|2054|8935|686|2071|0|3|29|1|2071|686|8935|Sunday|2071Q1|N|N|N|2477537|2477595|2477200|2477473|N|N|N|N|N| +2477566|AAAAAAAAOPNMFCAA|2071-03-30|2054|8935|686|2071|1|3|30|1|2071|686|8935|Monday|2071Q1|N|N|N|2477537|2477595|2477201|2477474|N|N|N|N|N| +2477567|AAAAAAAAPPNMFCAA|2071-03-31|2054|8936|686|2071|2|3|31|1|2071|686|8936|Tuesday|2071Q1|N|N|N|2477537|2477595|2477202|2477475|N|N|N|N|N| +2477568|AAAAAAAAAAOMFCAA|2071-04-01|2055|8936|686|2071|3|4|1|1|2071|686|8936|Wednesday|2071Q1|N|N|N|2477568|2477657|2477203|2477478|N|N|N|N|N| +2477569|AAAAAAAABAOMFCAA|2071-04-02|2055|8936|686|2071|4|4|2|2|2071|686|8936|Thursday|2071Q2|N|N|N|2477568|2477657|2477204|2477479|N|N|N|N|N| +2477570|AAAAAAAACAOMFCAA|2071-04-03|2055|8936|686|2071|5|4|3|2|2071|686|8936|Friday|2071Q2|N|Y|N|2477568|2477657|2477205|2477480|N|N|N|N|N| +2477571|AAAAAAAADAOMFCAA|2071-04-04|2055|8936|686|2071|6|4|4|2|2071|686|8936|Saturday|2071Q2|N|Y|N|2477568|2477657|2477206|2477481|N|N|N|N|N| +2477572|AAAAAAAAEAOMFCAA|2071-04-05|2055|8936|686|2071|0|4|5|2|2071|686|8936|Sunday|2071Q2|N|N|N|2477568|2477657|2477207|2477482|N|N|N|N|N| +2477573|AAAAAAAAFAOMFCAA|2071-04-06|2055|8936|686|2071|1|4|6|2|2071|686|8936|Monday|2071Q2|N|N|N|2477568|2477657|2477208|2477483|N|N|N|N|N| +2477574|AAAAAAAAGAOMFCAA|2071-04-07|2055|8937|686|2071|2|4|7|2|2071|686|8937|Tuesday|2071Q2|N|N|N|2477568|2477657|2477209|2477484|N|N|N|N|N| +2477575|AAAAAAAAHAOMFCAA|2071-04-08|2055|8937|686|2071|3|4|8|2|2071|686|8937|Wednesday|2071Q2|N|N|N|2477568|2477657|2477210|2477485|N|N|N|N|N| +2477576|AAAAAAAAIAOMFCAA|2071-04-09|2055|8937|686|2071|4|4|9|2|2071|686|8937|Thursday|2071Q2|N|N|N|2477568|2477657|2477211|2477486|N|N|N|N|N| +2477577|AAAAAAAAJAOMFCAA|2071-04-10|2055|8937|686|2071|5|4|10|2|2071|686|8937|Friday|2071Q2|N|Y|N|2477568|2477657|2477212|2477487|N|N|N|N|N| +2477578|AAAAAAAAKAOMFCAA|2071-04-11|2055|8937|686|2071|6|4|11|2|2071|686|8937|Saturday|2071Q2|N|Y|N|2477568|2477657|2477213|2477488|N|N|N|N|N| +2477579|AAAAAAAALAOMFCAA|2071-04-12|2055|8937|686|2071|0|4|12|2|2071|686|8937|Sunday|2071Q2|N|N|N|2477568|2477657|2477214|2477489|N|N|N|N|N| +2477580|AAAAAAAAMAOMFCAA|2071-04-13|2055|8937|686|2071|1|4|13|2|2071|686|8937|Monday|2071Q2|N|N|N|2477568|2477657|2477215|2477490|N|N|N|N|N| +2477581|AAAAAAAANAOMFCAA|2071-04-14|2055|8938|686|2071|2|4|14|2|2071|686|8938|Tuesday|2071Q2|N|N|N|2477568|2477657|2477216|2477491|N|N|N|N|N| +2477582|AAAAAAAAOAOMFCAA|2071-04-15|2055|8938|686|2071|3|4|15|2|2071|686|8938|Wednesday|2071Q2|N|N|N|2477568|2477657|2477217|2477492|N|N|N|N|N| +2477583|AAAAAAAAPAOMFCAA|2071-04-16|2055|8938|686|2071|4|4|16|2|2071|686|8938|Thursday|2071Q2|N|N|N|2477568|2477657|2477218|2477493|N|N|N|N|N| +2477584|AAAAAAAAABOMFCAA|2071-04-17|2055|8938|686|2071|5|4|17|2|2071|686|8938|Friday|2071Q2|N|Y|N|2477568|2477657|2477219|2477494|N|N|N|N|N| +2477585|AAAAAAAABBOMFCAA|2071-04-18|2055|8938|686|2071|6|4|18|2|2071|686|8938|Saturday|2071Q2|N|Y|N|2477568|2477657|2477220|2477495|N|N|N|N|N| +2477586|AAAAAAAACBOMFCAA|2071-04-19|2055|8938|686|2071|0|4|19|2|2071|686|8938|Sunday|2071Q2|N|N|N|2477568|2477657|2477221|2477496|N|N|N|N|N| +2477587|AAAAAAAADBOMFCAA|2071-04-20|2055|8938|686|2071|1|4|20|2|2071|686|8938|Monday|2071Q2|N|N|N|2477568|2477657|2477222|2477497|N|N|N|N|N| +2477588|AAAAAAAAEBOMFCAA|2071-04-21|2055|8939|686|2071|2|4|21|2|2071|686|8939|Tuesday|2071Q2|N|N|N|2477568|2477657|2477223|2477498|N|N|N|N|N| +2477589|AAAAAAAAFBOMFCAA|2071-04-22|2055|8939|686|2071|3|4|22|2|2071|686|8939|Wednesday|2071Q2|N|N|N|2477568|2477657|2477224|2477499|N|N|N|N|N| +2477590|AAAAAAAAGBOMFCAA|2071-04-23|2055|8939|686|2071|4|4|23|2|2071|686|8939|Thursday|2071Q2|N|N|N|2477568|2477657|2477225|2477500|N|N|N|N|N| +2477591|AAAAAAAAHBOMFCAA|2071-04-24|2055|8939|686|2071|5|4|24|2|2071|686|8939|Friday|2071Q2|N|Y|N|2477568|2477657|2477226|2477501|N|N|N|N|N| +2477592|AAAAAAAAIBOMFCAA|2071-04-25|2055|8939|686|2071|6|4|25|2|2071|686|8939|Saturday|2071Q2|N|Y|N|2477568|2477657|2477227|2477502|N|N|N|N|N| +2477593|AAAAAAAAJBOMFCAA|2071-04-26|2055|8939|686|2071|0|4|26|2|2071|686|8939|Sunday|2071Q2|N|N|N|2477568|2477657|2477228|2477503|N|N|N|N|N| +2477594|AAAAAAAAKBOMFCAA|2071-04-27|2055|8939|686|2071|1|4|27|2|2071|686|8939|Monday|2071Q2|N|N|N|2477568|2477657|2477229|2477504|N|N|N|N|N| +2477595|AAAAAAAALBOMFCAA|2071-04-28|2055|8940|686|2071|2|4|28|2|2071|686|8940|Tuesday|2071Q2|N|N|N|2477568|2477657|2477230|2477505|N|N|N|N|N| +2477596|AAAAAAAAMBOMFCAA|2071-04-29|2055|8940|686|2071|3|4|29|2|2071|686|8940|Wednesday|2071Q2|N|N|N|2477568|2477657|2477231|2477506|N|N|N|N|N| +2477597|AAAAAAAANBOMFCAA|2071-04-30|2055|8940|686|2071|4|4|30|2|2071|686|8940|Thursday|2071Q2|N|N|N|2477568|2477657|2477232|2477507|N|N|N|N|N| +2477598|AAAAAAAAOBOMFCAA|2071-05-01|2056|8940|686|2071|5|5|1|2|2071|686|8940|Friday|2071Q2|N|Y|N|2477598|2477717|2477233|2477508|N|N|N|N|N| +2477599|AAAAAAAAPBOMFCAA|2071-05-02|2056|8940|686|2071|6|5|2|2|2071|686|8940|Saturday|2071Q2|N|Y|N|2477598|2477717|2477234|2477509|N|N|N|N|N| +2477600|AAAAAAAAACOMFCAA|2071-05-03|2056|8940|686|2071|0|5|3|2|2071|686|8940|Sunday|2071Q2|N|N|N|2477598|2477717|2477235|2477510|N|N|N|N|N| +2477601|AAAAAAAABCOMFCAA|2071-05-04|2056|8940|686|2071|1|5|4|2|2071|686|8940|Monday|2071Q2|N|N|N|2477598|2477717|2477236|2477511|N|N|N|N|N| +2477602|AAAAAAAACCOMFCAA|2071-05-05|2056|8941|686|2071|2|5|5|2|2071|686|8941|Tuesday|2071Q2|N|N|N|2477598|2477717|2477237|2477512|N|N|N|N|N| +2477603|AAAAAAAADCOMFCAA|2071-05-06|2056|8941|686|2071|3|5|6|2|2071|686|8941|Wednesday|2071Q2|N|N|N|2477598|2477717|2477238|2477513|N|N|N|N|N| +2477604|AAAAAAAAECOMFCAA|2071-05-07|2056|8941|686|2071|4|5|7|2|2071|686|8941|Thursday|2071Q2|N|N|N|2477598|2477717|2477239|2477514|N|N|N|N|N| +2477605|AAAAAAAAFCOMFCAA|2071-05-08|2056|8941|686|2071|5|5|8|2|2071|686|8941|Friday|2071Q2|N|Y|N|2477598|2477717|2477240|2477515|N|N|N|N|N| +2477606|AAAAAAAAGCOMFCAA|2071-05-09|2056|8941|686|2071|6|5|9|2|2071|686|8941|Saturday|2071Q2|N|Y|N|2477598|2477717|2477241|2477516|N|N|N|N|N| +2477607|AAAAAAAAHCOMFCAA|2071-05-10|2056|8941|686|2071|0|5|10|2|2071|686|8941|Sunday|2071Q2|N|N|N|2477598|2477717|2477242|2477517|N|N|N|N|N| +2477608|AAAAAAAAICOMFCAA|2071-05-11|2056|8941|686|2071|1|5|11|2|2071|686|8941|Monday|2071Q2|N|N|N|2477598|2477717|2477243|2477518|N|N|N|N|N| +2477609|AAAAAAAAJCOMFCAA|2071-05-12|2056|8942|686|2071|2|5|12|2|2071|686|8942|Tuesday|2071Q2|N|N|N|2477598|2477717|2477244|2477519|N|N|N|N|N| +2477610|AAAAAAAAKCOMFCAA|2071-05-13|2056|8942|686|2071|3|5|13|2|2071|686|8942|Wednesday|2071Q2|N|N|N|2477598|2477717|2477245|2477520|N|N|N|N|N| +2477611|AAAAAAAALCOMFCAA|2071-05-14|2056|8942|686|2071|4|5|14|2|2071|686|8942|Thursday|2071Q2|N|N|N|2477598|2477717|2477246|2477521|N|N|N|N|N| +2477612|AAAAAAAAMCOMFCAA|2071-05-15|2056|8942|686|2071|5|5|15|2|2071|686|8942|Friday|2071Q2|N|Y|N|2477598|2477717|2477247|2477522|N|N|N|N|N| +2477613|AAAAAAAANCOMFCAA|2071-05-16|2056|8942|686|2071|6|5|16|2|2071|686|8942|Saturday|2071Q2|N|Y|N|2477598|2477717|2477248|2477523|N|N|N|N|N| +2477614|AAAAAAAAOCOMFCAA|2071-05-17|2056|8942|686|2071|0|5|17|2|2071|686|8942|Sunday|2071Q2|N|N|N|2477598|2477717|2477249|2477524|N|N|N|N|N| +2477615|AAAAAAAAPCOMFCAA|2071-05-18|2056|8942|686|2071|1|5|18|2|2071|686|8942|Monday|2071Q2|N|N|N|2477598|2477717|2477250|2477525|N|N|N|N|N| +2477616|AAAAAAAAADOMFCAA|2071-05-19|2056|8943|686|2071|2|5|19|2|2071|686|8943|Tuesday|2071Q2|N|N|N|2477598|2477717|2477251|2477526|N|N|N|N|N| +2477617|AAAAAAAABDOMFCAA|2071-05-20|2056|8943|686|2071|3|5|20|2|2071|686|8943|Wednesday|2071Q2|N|N|N|2477598|2477717|2477252|2477527|N|N|N|N|N| +2477618|AAAAAAAACDOMFCAA|2071-05-21|2056|8943|686|2071|4|5|21|2|2071|686|8943|Thursday|2071Q2|N|N|N|2477598|2477717|2477253|2477528|N|N|N|N|N| +2477619|AAAAAAAADDOMFCAA|2071-05-22|2056|8943|686|2071|5|5|22|2|2071|686|8943|Friday|2071Q2|N|Y|N|2477598|2477717|2477254|2477529|N|N|N|N|N| +2477620|AAAAAAAAEDOMFCAA|2071-05-23|2056|8943|686|2071|6|5|23|2|2071|686|8943|Saturday|2071Q2|N|Y|N|2477598|2477717|2477255|2477530|N|N|N|N|N| +2477621|AAAAAAAAFDOMFCAA|2071-05-24|2056|8943|686|2071|0|5|24|2|2071|686|8943|Sunday|2071Q2|N|N|N|2477598|2477717|2477256|2477531|N|N|N|N|N| +2477622|AAAAAAAAGDOMFCAA|2071-05-25|2056|8943|686|2071|1|5|25|2|2071|686|8943|Monday|2071Q2|N|N|N|2477598|2477717|2477257|2477532|N|N|N|N|N| +2477623|AAAAAAAAHDOMFCAA|2071-05-26|2056|8944|686|2071|2|5|26|2|2071|686|8944|Tuesday|2071Q2|N|N|N|2477598|2477717|2477258|2477533|N|N|N|N|N| +2477624|AAAAAAAAIDOMFCAA|2071-05-27|2056|8944|686|2071|3|5|27|2|2071|686|8944|Wednesday|2071Q2|N|N|N|2477598|2477717|2477259|2477534|N|N|N|N|N| +2477625|AAAAAAAAJDOMFCAA|2071-05-28|2056|8944|686|2071|4|5|28|2|2071|686|8944|Thursday|2071Q2|N|N|N|2477598|2477717|2477260|2477535|N|N|N|N|N| +2477626|AAAAAAAAKDOMFCAA|2071-05-29|2056|8944|686|2071|5|5|29|2|2071|686|8944|Friday|2071Q2|N|Y|N|2477598|2477717|2477261|2477536|N|N|N|N|N| +2477627|AAAAAAAALDOMFCAA|2071-05-30|2056|8944|686|2071|6|5|30|2|2071|686|8944|Saturday|2071Q2|N|Y|N|2477598|2477717|2477262|2477537|N|N|N|N|N| +2477628|AAAAAAAAMDOMFCAA|2071-05-31|2056|8944|686|2071|0|5|31|2|2071|686|8944|Sunday|2071Q2|N|N|N|2477598|2477717|2477263|2477538|N|N|N|N|N| +2477629|AAAAAAAANDOMFCAA|2071-06-01|2057|8944|687|2071|1|6|1|2|2071|687|8944|Monday|2071Q2|N|N|N|2477629|2477779|2477264|2477539|N|N|N|N|N| +2477630|AAAAAAAAODOMFCAA|2071-06-02|2057|8945|687|2071|2|6|2|2|2071|687|8945|Tuesday|2071Q2|N|N|N|2477629|2477779|2477265|2477540|N|N|N|N|N| +2477631|AAAAAAAAPDOMFCAA|2071-06-03|2057|8945|687|2071|3|6|3|2|2071|687|8945|Wednesday|2071Q2|N|N|N|2477629|2477779|2477266|2477541|N|N|N|N|N| +2477632|AAAAAAAAAEOMFCAA|2071-06-04|2057|8945|687|2071|4|6|4|2|2071|687|8945|Thursday|2071Q2|N|N|N|2477629|2477779|2477267|2477542|N|N|N|N|N| +2477633|AAAAAAAABEOMFCAA|2071-06-05|2057|8945|687|2071|5|6|5|2|2071|687|8945|Friday|2071Q2|N|Y|N|2477629|2477779|2477268|2477543|N|N|N|N|N| +2477634|AAAAAAAACEOMFCAA|2071-06-06|2057|8945|687|2071|6|6|6|2|2071|687|8945|Saturday|2071Q2|N|Y|N|2477629|2477779|2477269|2477544|N|N|N|N|N| +2477635|AAAAAAAADEOMFCAA|2071-06-07|2057|8945|687|2071|0|6|7|2|2071|687|8945|Sunday|2071Q2|N|N|N|2477629|2477779|2477270|2477545|N|N|N|N|N| +2477636|AAAAAAAAEEOMFCAA|2071-06-08|2057|8945|687|2071|1|6|8|2|2071|687|8945|Monday|2071Q2|N|N|N|2477629|2477779|2477271|2477546|N|N|N|N|N| +2477637|AAAAAAAAFEOMFCAA|2071-06-09|2057|8946|687|2071|2|6|9|2|2071|687|8946|Tuesday|2071Q2|N|N|N|2477629|2477779|2477272|2477547|N|N|N|N|N| +2477638|AAAAAAAAGEOMFCAA|2071-06-10|2057|8946|687|2071|3|6|10|2|2071|687|8946|Wednesday|2071Q2|N|N|N|2477629|2477779|2477273|2477548|N|N|N|N|N| +2477639|AAAAAAAAHEOMFCAA|2071-06-11|2057|8946|687|2071|4|6|11|2|2071|687|8946|Thursday|2071Q2|N|N|N|2477629|2477779|2477274|2477549|N|N|N|N|N| +2477640|AAAAAAAAIEOMFCAA|2071-06-12|2057|8946|687|2071|5|6|12|2|2071|687|8946|Friday|2071Q2|N|Y|N|2477629|2477779|2477275|2477550|N|N|N|N|N| +2477641|AAAAAAAAJEOMFCAA|2071-06-13|2057|8946|687|2071|6|6|13|2|2071|687|8946|Saturday|2071Q2|N|Y|N|2477629|2477779|2477276|2477551|N|N|N|N|N| +2477642|AAAAAAAAKEOMFCAA|2071-06-14|2057|8946|687|2071|0|6|14|2|2071|687|8946|Sunday|2071Q2|N|N|N|2477629|2477779|2477277|2477552|N|N|N|N|N| +2477643|AAAAAAAALEOMFCAA|2071-06-15|2057|8946|687|2071|1|6|15|2|2071|687|8946|Monday|2071Q2|N|N|N|2477629|2477779|2477278|2477553|N|N|N|N|N| +2477644|AAAAAAAAMEOMFCAA|2071-06-16|2057|8947|687|2071|2|6|16|2|2071|687|8947|Tuesday|2071Q2|N|N|N|2477629|2477779|2477279|2477554|N|N|N|N|N| +2477645|AAAAAAAANEOMFCAA|2071-06-17|2057|8947|687|2071|3|6|17|2|2071|687|8947|Wednesday|2071Q2|N|N|N|2477629|2477779|2477280|2477555|N|N|N|N|N| +2477646|AAAAAAAAOEOMFCAA|2071-06-18|2057|8947|687|2071|4|6|18|2|2071|687|8947|Thursday|2071Q2|N|N|N|2477629|2477779|2477281|2477556|N|N|N|N|N| +2477647|AAAAAAAAPEOMFCAA|2071-06-19|2057|8947|687|2071|5|6|19|2|2071|687|8947|Friday|2071Q2|N|Y|N|2477629|2477779|2477282|2477557|N|N|N|N|N| +2477648|AAAAAAAAAFOMFCAA|2071-06-20|2057|8947|687|2071|6|6|20|2|2071|687|8947|Saturday|2071Q2|N|Y|N|2477629|2477779|2477283|2477558|N|N|N|N|N| +2477649|AAAAAAAABFOMFCAA|2071-06-21|2057|8947|687|2071|0|6|21|2|2071|687|8947|Sunday|2071Q2|N|N|N|2477629|2477779|2477284|2477559|N|N|N|N|N| +2477650|AAAAAAAACFOMFCAA|2071-06-22|2057|8947|687|2071|1|6|22|2|2071|687|8947|Monday|2071Q2|N|N|N|2477629|2477779|2477285|2477560|N|N|N|N|N| +2477651|AAAAAAAADFOMFCAA|2071-06-23|2057|8948|687|2071|2|6|23|2|2071|687|8948|Tuesday|2071Q2|N|N|N|2477629|2477779|2477286|2477561|N|N|N|N|N| +2477652|AAAAAAAAEFOMFCAA|2071-06-24|2057|8948|687|2071|3|6|24|2|2071|687|8948|Wednesday|2071Q2|N|N|N|2477629|2477779|2477287|2477562|N|N|N|N|N| +2477653|AAAAAAAAFFOMFCAA|2071-06-25|2057|8948|687|2071|4|6|25|2|2071|687|8948|Thursday|2071Q2|N|N|N|2477629|2477779|2477288|2477563|N|N|N|N|N| +2477654|AAAAAAAAGFOMFCAA|2071-06-26|2057|8948|687|2071|5|6|26|2|2071|687|8948|Friday|2071Q2|N|Y|N|2477629|2477779|2477289|2477564|N|N|N|N|N| +2477655|AAAAAAAAHFOMFCAA|2071-06-27|2057|8948|687|2071|6|6|27|2|2071|687|8948|Saturday|2071Q2|N|Y|N|2477629|2477779|2477290|2477565|N|N|N|N|N| +2477656|AAAAAAAAIFOMFCAA|2071-06-28|2057|8948|687|2071|0|6|28|2|2071|687|8948|Sunday|2071Q2|N|N|N|2477629|2477779|2477291|2477566|N|N|N|N|N| +2477657|AAAAAAAAJFOMFCAA|2071-06-29|2057|8948|687|2071|1|6|29|2|2071|687|8948|Monday|2071Q2|N|N|N|2477629|2477779|2477292|2477567|N|N|N|N|N| +2477658|AAAAAAAAKFOMFCAA|2071-06-30|2057|8949|687|2071|2|6|30|2|2071|687|8949|Tuesday|2071Q2|N|N|N|2477629|2477779|2477293|2477568|N|N|N|N|N| +2477659|AAAAAAAALFOMFCAA|2071-07-01|2058|8949|687|2071|3|7|1|2|2071|687|8949|Wednesday|2071Q2|N|N|N|2477659|2477839|2477294|2477568|N|N|N|N|N| +2477660|AAAAAAAAMFOMFCAA|2071-07-02|2058|8949|687|2071|4|7|2|3|2071|687|8949|Thursday|2071Q3|N|N|N|2477659|2477839|2477295|2477569|N|N|N|N|N| +2477661|AAAAAAAANFOMFCAA|2071-07-03|2058|8949|687|2071|5|7|3|3|2071|687|8949|Friday|2071Q3|N|Y|N|2477659|2477839|2477296|2477570|N|N|N|N|N| +2477662|AAAAAAAAOFOMFCAA|2071-07-04|2058|8949|687|2071|6|7|4|3|2071|687|8949|Saturday|2071Q3|N|Y|N|2477659|2477839|2477297|2477571|N|N|N|N|N| +2477663|AAAAAAAAPFOMFCAA|2071-07-05|2058|8949|687|2071|0|7|5|3|2071|687|8949|Sunday|2071Q3|Y|N|N|2477659|2477839|2477298|2477572|N|N|N|N|N| +2477664|AAAAAAAAAGOMFCAA|2071-07-06|2058|8949|687|2071|1|7|6|3|2071|687|8949|Monday|2071Q3|N|N|Y|2477659|2477839|2477299|2477573|N|N|N|N|N| +2477665|AAAAAAAABGOMFCAA|2071-07-07|2058|8950|687|2071|2|7|7|3|2071|687|8950|Tuesday|2071Q3|N|N|N|2477659|2477839|2477300|2477574|N|N|N|N|N| +2477666|AAAAAAAACGOMFCAA|2071-07-08|2058|8950|687|2071|3|7|8|3|2071|687|8950|Wednesday|2071Q3|N|N|N|2477659|2477839|2477301|2477575|N|N|N|N|N| +2477667|AAAAAAAADGOMFCAA|2071-07-09|2058|8950|687|2071|4|7|9|3|2071|687|8950|Thursday|2071Q3|N|N|N|2477659|2477839|2477302|2477576|N|N|N|N|N| +2477668|AAAAAAAAEGOMFCAA|2071-07-10|2058|8950|687|2071|5|7|10|3|2071|687|8950|Friday|2071Q3|N|Y|N|2477659|2477839|2477303|2477577|N|N|N|N|N| +2477669|AAAAAAAAFGOMFCAA|2071-07-11|2058|8950|687|2071|6|7|11|3|2071|687|8950|Saturday|2071Q3|N|Y|N|2477659|2477839|2477304|2477578|N|N|N|N|N| +2477670|AAAAAAAAGGOMFCAA|2071-07-12|2058|8950|687|2071|0|7|12|3|2071|687|8950|Sunday|2071Q3|N|N|N|2477659|2477839|2477305|2477579|N|N|N|N|N| +2477671|AAAAAAAAHGOMFCAA|2071-07-13|2058|8950|687|2071|1|7|13|3|2071|687|8950|Monday|2071Q3|N|N|N|2477659|2477839|2477306|2477580|N|N|N|N|N| +2477672|AAAAAAAAIGOMFCAA|2071-07-14|2058|8951|687|2071|2|7|14|3|2071|687|8951|Tuesday|2071Q3|N|N|N|2477659|2477839|2477307|2477581|N|N|N|N|N| +2477673|AAAAAAAAJGOMFCAA|2071-07-15|2058|8951|687|2071|3|7|15|3|2071|687|8951|Wednesday|2071Q3|N|N|N|2477659|2477839|2477308|2477582|N|N|N|N|N| +2477674|AAAAAAAAKGOMFCAA|2071-07-16|2058|8951|687|2071|4|7|16|3|2071|687|8951|Thursday|2071Q3|N|N|N|2477659|2477839|2477309|2477583|N|N|N|N|N| +2477675|AAAAAAAALGOMFCAA|2071-07-17|2058|8951|687|2071|5|7|17|3|2071|687|8951|Friday|2071Q3|N|Y|N|2477659|2477839|2477310|2477584|N|N|N|N|N| +2477676|AAAAAAAAMGOMFCAA|2071-07-18|2058|8951|687|2071|6|7|18|3|2071|687|8951|Saturday|2071Q3|N|Y|N|2477659|2477839|2477311|2477585|N|N|N|N|N| +2477677|AAAAAAAANGOMFCAA|2071-07-19|2058|8951|687|2071|0|7|19|3|2071|687|8951|Sunday|2071Q3|N|N|N|2477659|2477839|2477312|2477586|N|N|N|N|N| +2477678|AAAAAAAAOGOMFCAA|2071-07-20|2058|8951|687|2071|1|7|20|3|2071|687|8951|Monday|2071Q3|N|N|N|2477659|2477839|2477313|2477587|N|N|N|N|N| +2477679|AAAAAAAAPGOMFCAA|2071-07-21|2058|8952|687|2071|2|7|21|3|2071|687|8952|Tuesday|2071Q3|N|N|N|2477659|2477839|2477314|2477588|N|N|N|N|N| +2477680|AAAAAAAAAHOMFCAA|2071-07-22|2058|8952|687|2071|3|7|22|3|2071|687|8952|Wednesday|2071Q3|N|N|N|2477659|2477839|2477315|2477589|N|N|N|N|N| +2477681|AAAAAAAABHOMFCAA|2071-07-23|2058|8952|687|2071|4|7|23|3|2071|687|8952|Thursday|2071Q3|N|N|N|2477659|2477839|2477316|2477590|N|N|N|N|N| +2477682|AAAAAAAACHOMFCAA|2071-07-24|2058|8952|687|2071|5|7|24|3|2071|687|8952|Friday|2071Q3|N|Y|N|2477659|2477839|2477317|2477591|N|N|N|N|N| +2477683|AAAAAAAADHOMFCAA|2071-07-25|2058|8952|687|2071|6|7|25|3|2071|687|8952|Saturday|2071Q3|N|Y|N|2477659|2477839|2477318|2477592|N|N|N|N|N| +2477684|AAAAAAAAEHOMFCAA|2071-07-26|2058|8952|687|2071|0|7|26|3|2071|687|8952|Sunday|2071Q3|N|N|N|2477659|2477839|2477319|2477593|N|N|N|N|N| +2477685|AAAAAAAAFHOMFCAA|2071-07-27|2058|8952|687|2071|1|7|27|3|2071|687|8952|Monday|2071Q3|N|N|N|2477659|2477839|2477320|2477594|N|N|N|N|N| +2477686|AAAAAAAAGHOMFCAA|2071-07-28|2058|8953|687|2071|2|7|28|3|2071|687|8953|Tuesday|2071Q3|N|N|N|2477659|2477839|2477321|2477595|N|N|N|N|N| +2477687|AAAAAAAAHHOMFCAA|2071-07-29|2058|8953|687|2071|3|7|29|3|2071|687|8953|Wednesday|2071Q3|N|N|N|2477659|2477839|2477322|2477596|N|N|N|N|N| +2477688|AAAAAAAAIHOMFCAA|2071-07-30|2058|8953|687|2071|4|7|30|3|2071|687|8953|Thursday|2071Q3|N|N|N|2477659|2477839|2477323|2477597|N|N|N|N|N| +2477689|AAAAAAAAJHOMFCAA|2071-07-31|2058|8953|687|2071|5|7|31|3|2071|687|8953|Friday|2071Q3|N|Y|N|2477659|2477839|2477324|2477598|N|N|N|N|N| +2477690|AAAAAAAAKHOMFCAA|2071-08-01|2059|8953|687|2071|6|8|1|3|2071|687|8953|Saturday|2071Q3|N|Y|N|2477690|2477901|2477325|2477599|N|N|N|N|N| +2477691|AAAAAAAALHOMFCAA|2071-08-02|2059|8953|687|2071|0|8|2|3|2071|687|8953|Sunday|2071Q3|N|N|N|2477690|2477901|2477326|2477600|N|N|N|N|N| +2477692|AAAAAAAAMHOMFCAA|2071-08-03|2059|8953|687|2071|1|8|3|3|2071|687|8953|Monday|2071Q3|N|N|N|2477690|2477901|2477327|2477601|N|N|N|N|N| +2477693|AAAAAAAANHOMFCAA|2071-08-04|2059|8954|687|2071|2|8|4|3|2071|687|8954|Tuesday|2071Q3|N|N|N|2477690|2477901|2477328|2477602|N|N|N|N|N| +2477694|AAAAAAAAOHOMFCAA|2071-08-05|2059|8954|687|2071|3|8|5|3|2071|687|8954|Wednesday|2071Q3|N|N|N|2477690|2477901|2477329|2477603|N|N|N|N|N| +2477695|AAAAAAAAPHOMFCAA|2071-08-06|2059|8954|687|2071|4|8|6|3|2071|687|8954|Thursday|2071Q3|N|N|N|2477690|2477901|2477330|2477604|N|N|N|N|N| +2477696|AAAAAAAAAIOMFCAA|2071-08-07|2059|8954|687|2071|5|8|7|3|2071|687|8954|Friday|2071Q3|N|Y|N|2477690|2477901|2477331|2477605|N|N|N|N|N| +2477697|AAAAAAAABIOMFCAA|2071-08-08|2059|8954|687|2071|6|8|8|3|2071|687|8954|Saturday|2071Q3|N|Y|N|2477690|2477901|2477332|2477606|N|N|N|N|N| +2477698|AAAAAAAACIOMFCAA|2071-08-09|2059|8954|687|2071|0|8|9|3|2071|687|8954|Sunday|2071Q3|N|N|N|2477690|2477901|2477333|2477607|N|N|N|N|N| +2477699|AAAAAAAADIOMFCAA|2071-08-10|2059|8954|687|2071|1|8|10|3|2071|687|8954|Monday|2071Q3|N|N|N|2477690|2477901|2477334|2477608|N|N|N|N|N| +2477700|AAAAAAAAEIOMFCAA|2071-08-11|2059|8955|687|2071|2|8|11|3|2071|687|8955|Tuesday|2071Q3|N|N|N|2477690|2477901|2477335|2477609|N|N|N|N|N| +2477701|AAAAAAAAFIOMFCAA|2071-08-12|2059|8955|687|2071|3|8|12|3|2071|687|8955|Wednesday|2071Q3|N|N|N|2477690|2477901|2477336|2477610|N|N|N|N|N| +2477702|AAAAAAAAGIOMFCAA|2071-08-13|2059|8955|687|2071|4|8|13|3|2071|687|8955|Thursday|2071Q3|N|N|N|2477690|2477901|2477337|2477611|N|N|N|N|N| +2477703|AAAAAAAAHIOMFCAA|2071-08-14|2059|8955|687|2071|5|8|14|3|2071|687|8955|Friday|2071Q3|N|Y|N|2477690|2477901|2477338|2477612|N|N|N|N|N| +2477704|AAAAAAAAIIOMFCAA|2071-08-15|2059|8955|687|2071|6|8|15|3|2071|687|8955|Saturday|2071Q3|N|Y|N|2477690|2477901|2477339|2477613|N|N|N|N|N| +2477705|AAAAAAAAJIOMFCAA|2071-08-16|2059|8955|687|2071|0|8|16|3|2071|687|8955|Sunday|2071Q3|N|N|N|2477690|2477901|2477340|2477614|N|N|N|N|N| +2477706|AAAAAAAAKIOMFCAA|2071-08-17|2059|8955|687|2071|1|8|17|3|2071|687|8955|Monday|2071Q3|N|N|N|2477690|2477901|2477341|2477615|N|N|N|N|N| +2477707|AAAAAAAALIOMFCAA|2071-08-18|2059|8956|687|2071|2|8|18|3|2071|687|8956|Tuesday|2071Q3|N|N|N|2477690|2477901|2477342|2477616|N|N|N|N|N| +2477708|AAAAAAAAMIOMFCAA|2071-08-19|2059|8956|687|2071|3|8|19|3|2071|687|8956|Wednesday|2071Q3|N|N|N|2477690|2477901|2477343|2477617|N|N|N|N|N| +2477709|AAAAAAAANIOMFCAA|2071-08-20|2059|8956|687|2071|4|8|20|3|2071|687|8956|Thursday|2071Q3|N|N|N|2477690|2477901|2477344|2477618|N|N|N|N|N| +2477710|AAAAAAAAOIOMFCAA|2071-08-21|2059|8956|687|2071|5|8|21|3|2071|687|8956|Friday|2071Q3|N|Y|N|2477690|2477901|2477345|2477619|N|N|N|N|N| +2477711|AAAAAAAAPIOMFCAA|2071-08-22|2059|8956|687|2071|6|8|22|3|2071|687|8956|Saturday|2071Q3|N|Y|N|2477690|2477901|2477346|2477620|N|N|N|N|N| +2477712|AAAAAAAAAJOMFCAA|2071-08-23|2059|8956|687|2071|0|8|23|3|2071|687|8956|Sunday|2071Q3|N|N|N|2477690|2477901|2477347|2477621|N|N|N|N|N| +2477713|AAAAAAAABJOMFCAA|2071-08-24|2059|8956|687|2071|1|8|24|3|2071|687|8956|Monday|2071Q3|N|N|N|2477690|2477901|2477348|2477622|N|N|N|N|N| +2477714|AAAAAAAACJOMFCAA|2071-08-25|2059|8957|687|2071|2|8|25|3|2071|687|8957|Tuesday|2071Q3|N|N|N|2477690|2477901|2477349|2477623|N|N|N|N|N| +2477715|AAAAAAAADJOMFCAA|2071-08-26|2059|8957|687|2071|3|8|26|3|2071|687|8957|Wednesday|2071Q3|N|N|N|2477690|2477901|2477350|2477624|N|N|N|N|N| +2477716|AAAAAAAAEJOMFCAA|2071-08-27|2059|8957|687|2071|4|8|27|3|2071|687|8957|Thursday|2071Q3|N|N|N|2477690|2477901|2477351|2477625|N|N|N|N|N| +2477717|AAAAAAAAFJOMFCAA|2071-08-28|2059|8957|687|2071|5|8|28|3|2071|687|8957|Friday|2071Q3|N|Y|N|2477690|2477901|2477352|2477626|N|N|N|N|N| +2477718|AAAAAAAAGJOMFCAA|2071-08-29|2059|8957|687|2071|6|8|29|3|2071|687|8957|Saturday|2071Q3|N|Y|N|2477690|2477901|2477353|2477627|N|N|N|N|N| +2477719|AAAAAAAAHJOMFCAA|2071-08-30|2059|8957|687|2071|0|8|30|3|2071|687|8957|Sunday|2071Q3|N|N|N|2477690|2477901|2477354|2477628|N|N|N|N|N| +2477720|AAAAAAAAIJOMFCAA|2071-08-31|2059|8957|687|2071|1|8|31|3|2071|687|8957|Monday|2071Q3|N|N|N|2477690|2477901|2477355|2477629|N|N|N|N|N| +2477721|AAAAAAAAJJOMFCAA|2071-09-01|2060|8958|688|2071|2|9|1|3|2071|688|8958|Tuesday|2071Q3|N|N|N|2477721|2477963|2477356|2477630|N|N|N|N|N| +2477722|AAAAAAAAKJOMFCAA|2071-09-02|2060|8958|688|2071|3|9|2|3|2071|688|8958|Wednesday|2071Q3|N|N|N|2477721|2477963|2477357|2477631|N|N|N|N|N| +2477723|AAAAAAAALJOMFCAA|2071-09-03|2060|8958|688|2071|4|9|3|3|2071|688|8958|Thursday|2071Q3|N|N|N|2477721|2477963|2477358|2477632|N|N|N|N|N| +2477724|AAAAAAAAMJOMFCAA|2071-09-04|2060|8958|688|2071|5|9|4|3|2071|688|8958|Friday|2071Q3|N|Y|N|2477721|2477963|2477359|2477633|N|N|N|N|N| +2477725|AAAAAAAANJOMFCAA|2071-09-05|2060|8958|688|2071|6|9|5|3|2071|688|8958|Saturday|2071Q3|N|Y|N|2477721|2477963|2477360|2477634|N|N|N|N|N| +2477726|AAAAAAAAOJOMFCAA|2071-09-06|2060|8958|688|2071|0|9|6|3|2071|688|8958|Sunday|2071Q3|N|N|N|2477721|2477963|2477361|2477635|N|N|N|N|N| +2477727|AAAAAAAAPJOMFCAA|2071-09-07|2060|8958|688|2071|1|9|7|3|2071|688|8958|Monday|2071Q3|N|N|N|2477721|2477963|2477362|2477636|N|N|N|N|N| +2477728|AAAAAAAAAKOMFCAA|2071-09-08|2060|8959|688|2071|2|9|8|3|2071|688|8959|Tuesday|2071Q3|N|N|N|2477721|2477963|2477363|2477637|N|N|N|N|N| +2477729|AAAAAAAABKOMFCAA|2071-09-09|2060|8959|688|2071|3|9|9|3|2071|688|8959|Wednesday|2071Q3|N|N|N|2477721|2477963|2477364|2477638|N|N|N|N|N| +2477730|AAAAAAAACKOMFCAA|2071-09-10|2060|8959|688|2071|4|9|10|3|2071|688|8959|Thursday|2071Q3|N|N|N|2477721|2477963|2477365|2477639|N|N|N|N|N| +2477731|AAAAAAAADKOMFCAA|2071-09-11|2060|8959|688|2071|5|9|11|3|2071|688|8959|Friday|2071Q3|N|Y|N|2477721|2477963|2477366|2477640|N|N|N|N|N| +2477732|AAAAAAAAEKOMFCAA|2071-09-12|2060|8959|688|2071|6|9|12|3|2071|688|8959|Saturday|2071Q3|N|Y|N|2477721|2477963|2477367|2477641|N|N|N|N|N| +2477733|AAAAAAAAFKOMFCAA|2071-09-13|2060|8959|688|2071|0|9|13|3|2071|688|8959|Sunday|2071Q3|N|N|N|2477721|2477963|2477368|2477642|N|N|N|N|N| +2477734|AAAAAAAAGKOMFCAA|2071-09-14|2060|8959|688|2071|1|9|14|3|2071|688|8959|Monday|2071Q3|N|N|N|2477721|2477963|2477369|2477643|N|N|N|N|N| +2477735|AAAAAAAAHKOMFCAA|2071-09-15|2060|8960|688|2071|2|9|15|3|2071|688|8960|Tuesday|2071Q3|N|N|N|2477721|2477963|2477370|2477644|N|N|N|N|N| +2477736|AAAAAAAAIKOMFCAA|2071-09-16|2060|8960|688|2071|3|9|16|3|2071|688|8960|Wednesday|2071Q3|N|N|N|2477721|2477963|2477371|2477645|N|N|N|N|N| +2477737|AAAAAAAAJKOMFCAA|2071-09-17|2060|8960|688|2071|4|9|17|3|2071|688|8960|Thursday|2071Q3|N|N|N|2477721|2477963|2477372|2477646|N|N|N|N|N| +2477738|AAAAAAAAKKOMFCAA|2071-09-18|2060|8960|688|2071|5|9|18|3|2071|688|8960|Friday|2071Q3|N|Y|N|2477721|2477963|2477373|2477647|N|N|N|N|N| +2477739|AAAAAAAALKOMFCAA|2071-09-19|2060|8960|688|2071|6|9|19|3|2071|688|8960|Saturday|2071Q3|N|Y|N|2477721|2477963|2477374|2477648|N|N|N|N|N| +2477740|AAAAAAAAMKOMFCAA|2071-09-20|2060|8960|688|2071|0|9|20|3|2071|688|8960|Sunday|2071Q3|N|N|N|2477721|2477963|2477375|2477649|N|N|N|N|N| +2477741|AAAAAAAANKOMFCAA|2071-09-21|2060|8960|688|2071|1|9|21|3|2071|688|8960|Monday|2071Q3|N|N|N|2477721|2477963|2477376|2477650|N|N|N|N|N| +2477742|AAAAAAAAOKOMFCAA|2071-09-22|2060|8961|688|2071|2|9|22|3|2071|688|8961|Tuesday|2071Q3|N|N|N|2477721|2477963|2477377|2477651|N|N|N|N|N| +2477743|AAAAAAAAPKOMFCAA|2071-09-23|2060|8961|688|2071|3|9|23|3|2071|688|8961|Wednesday|2071Q3|N|N|N|2477721|2477963|2477378|2477652|N|N|N|N|N| +2477744|AAAAAAAAALOMFCAA|2071-09-24|2060|8961|688|2071|4|9|24|3|2071|688|8961|Thursday|2071Q3|N|N|N|2477721|2477963|2477379|2477653|N|N|N|N|N| +2477745|AAAAAAAABLOMFCAA|2071-09-25|2060|8961|688|2071|5|9|25|3|2071|688|8961|Friday|2071Q3|N|Y|N|2477721|2477963|2477380|2477654|N|N|N|N|N| +2477746|AAAAAAAACLOMFCAA|2071-09-26|2060|8961|688|2071|6|9|26|3|2071|688|8961|Saturday|2071Q3|N|Y|N|2477721|2477963|2477381|2477655|N|N|N|N|N| +2477747|AAAAAAAADLOMFCAA|2071-09-27|2060|8961|688|2071|0|9|27|3|2071|688|8961|Sunday|2071Q3|N|N|N|2477721|2477963|2477382|2477656|N|N|N|N|N| +2477748|AAAAAAAAELOMFCAA|2071-09-28|2060|8961|688|2071|1|9|28|3|2071|688|8961|Monday|2071Q3|N|N|N|2477721|2477963|2477383|2477657|N|N|N|N|N| +2477749|AAAAAAAAFLOMFCAA|2071-09-29|2060|8962|688|2071|2|9|29|3|2071|688|8962|Tuesday|2071Q3|N|N|N|2477721|2477963|2477384|2477658|N|N|N|N|N| +2477750|AAAAAAAAGLOMFCAA|2071-09-30|2060|8962|688|2071|3|9|30|3|2071|688|8962|Wednesday|2071Q3|N|N|N|2477721|2477963|2477385|2477659|N|N|N|N|N| +2477751|AAAAAAAAHLOMFCAA|2071-10-01|2061|8962|688|2071|4|10|1|3|2071|688|8962|Thursday|2071Q3|N|N|N|2477751|2478023|2477386|2477659|N|N|N|N|N| +2477752|AAAAAAAAILOMFCAA|2071-10-02|2061|8962|688|2071|5|10|2|4|2071|688|8962|Friday|2071Q4|N|Y|N|2477751|2478023|2477387|2477660|N|N|N|N|N| +2477753|AAAAAAAAJLOMFCAA|2071-10-03|2061|8962|688|2071|6|10|3|4|2071|688|8962|Saturday|2071Q4|N|Y|N|2477751|2478023|2477388|2477661|N|N|N|N|N| +2477754|AAAAAAAAKLOMFCAA|2071-10-04|2061|8962|688|2071|0|10|4|4|2071|688|8962|Sunday|2071Q4|N|N|N|2477751|2478023|2477389|2477662|N|N|N|N|N| +2477755|AAAAAAAALLOMFCAA|2071-10-05|2061|8962|688|2071|1|10|5|4|2071|688|8962|Monday|2071Q4|N|N|N|2477751|2478023|2477390|2477663|N|N|N|N|N| +2477756|AAAAAAAAMLOMFCAA|2071-10-06|2061|8963|688|2071|2|10|6|4|2071|688|8963|Tuesday|2071Q4|N|N|N|2477751|2478023|2477391|2477664|N|N|N|N|N| +2477757|AAAAAAAANLOMFCAA|2071-10-07|2061|8963|688|2071|3|10|7|4|2071|688|8963|Wednesday|2071Q4|N|N|N|2477751|2478023|2477392|2477665|N|N|N|N|N| +2477758|AAAAAAAAOLOMFCAA|2071-10-08|2061|8963|688|2071|4|10|8|4|2071|688|8963|Thursday|2071Q4|N|N|N|2477751|2478023|2477393|2477666|N|N|N|N|N| +2477759|AAAAAAAAPLOMFCAA|2071-10-09|2061|8963|688|2071|5|10|9|4|2071|688|8963|Friday|2071Q4|N|Y|N|2477751|2478023|2477394|2477667|N|N|N|N|N| +2477760|AAAAAAAAAMOMFCAA|2071-10-10|2061|8963|688|2071|6|10|10|4|2071|688|8963|Saturday|2071Q4|N|Y|N|2477751|2478023|2477395|2477668|N|N|N|N|N| +2477761|AAAAAAAABMOMFCAA|2071-10-11|2061|8963|688|2071|0|10|11|4|2071|688|8963|Sunday|2071Q4|N|N|N|2477751|2478023|2477396|2477669|N|N|N|N|N| +2477762|AAAAAAAACMOMFCAA|2071-10-12|2061|8963|688|2071|1|10|12|4|2071|688|8963|Monday|2071Q4|N|N|N|2477751|2478023|2477397|2477670|N|N|N|N|N| +2477763|AAAAAAAADMOMFCAA|2071-10-13|2061|8964|688|2071|2|10|13|4|2071|688|8964|Tuesday|2071Q4|N|N|N|2477751|2478023|2477398|2477671|N|N|N|N|N| +2477764|AAAAAAAAEMOMFCAA|2071-10-14|2061|8964|688|2071|3|10|14|4|2071|688|8964|Wednesday|2071Q4|N|N|N|2477751|2478023|2477399|2477672|N|N|N|N|N| +2477765|AAAAAAAAFMOMFCAA|2071-10-15|2061|8964|688|2071|4|10|15|4|2071|688|8964|Thursday|2071Q4|N|N|N|2477751|2478023|2477400|2477673|N|N|N|N|N| +2477766|AAAAAAAAGMOMFCAA|2071-10-16|2061|8964|688|2071|5|10|16|4|2071|688|8964|Friday|2071Q4|N|Y|N|2477751|2478023|2477401|2477674|N|N|N|N|N| +2477767|AAAAAAAAHMOMFCAA|2071-10-17|2061|8964|688|2071|6|10|17|4|2071|688|8964|Saturday|2071Q4|N|Y|N|2477751|2478023|2477402|2477675|N|N|N|N|N| +2477768|AAAAAAAAIMOMFCAA|2071-10-18|2061|8964|688|2071|0|10|18|4|2071|688|8964|Sunday|2071Q4|N|N|N|2477751|2478023|2477403|2477676|N|N|N|N|N| +2477769|AAAAAAAAJMOMFCAA|2071-10-19|2061|8964|688|2071|1|10|19|4|2071|688|8964|Monday|2071Q4|N|N|N|2477751|2478023|2477404|2477677|N|N|N|N|N| +2477770|AAAAAAAAKMOMFCAA|2071-10-20|2061|8965|688|2071|2|10|20|4|2071|688|8965|Tuesday|2071Q4|N|N|N|2477751|2478023|2477405|2477678|N|N|N|N|N| +2477771|AAAAAAAALMOMFCAA|2071-10-21|2061|8965|688|2071|3|10|21|4|2071|688|8965|Wednesday|2071Q4|N|N|N|2477751|2478023|2477406|2477679|N|N|N|N|N| +2477772|AAAAAAAAMMOMFCAA|2071-10-22|2061|8965|688|2071|4|10|22|4|2071|688|8965|Thursday|2071Q4|N|N|N|2477751|2478023|2477407|2477680|N|N|N|N|N| +2477773|AAAAAAAANMOMFCAA|2071-10-23|2061|8965|688|2071|5|10|23|4|2071|688|8965|Friday|2071Q4|N|Y|N|2477751|2478023|2477408|2477681|N|N|N|N|N| +2477774|AAAAAAAAOMOMFCAA|2071-10-24|2061|8965|688|2071|6|10|24|4|2071|688|8965|Saturday|2071Q4|N|Y|N|2477751|2478023|2477409|2477682|N|N|N|N|N| +2477775|AAAAAAAAPMOMFCAA|2071-10-25|2061|8965|688|2071|0|10|25|4|2071|688|8965|Sunday|2071Q4|N|N|N|2477751|2478023|2477410|2477683|N|N|N|N|N| +2477776|AAAAAAAAANOMFCAA|2071-10-26|2061|8965|688|2071|1|10|26|4|2071|688|8965|Monday|2071Q4|N|N|N|2477751|2478023|2477411|2477684|N|N|N|N|N| +2477777|AAAAAAAABNOMFCAA|2071-10-27|2061|8966|688|2071|2|10|27|4|2071|688|8966|Tuesday|2071Q4|N|N|N|2477751|2478023|2477412|2477685|N|N|N|N|N| +2477778|AAAAAAAACNOMFCAA|2071-10-28|2061|8966|688|2071|3|10|28|4|2071|688|8966|Wednesday|2071Q4|N|N|N|2477751|2478023|2477413|2477686|N|N|N|N|N| +2477779|AAAAAAAADNOMFCAA|2071-10-29|2061|8966|688|2071|4|10|29|4|2071|688|8966|Thursday|2071Q4|N|N|N|2477751|2478023|2477414|2477687|N|N|N|N|N| +2477780|AAAAAAAAENOMFCAA|2071-10-30|2061|8966|688|2071|5|10|30|4|2071|688|8966|Friday|2071Q4|N|Y|N|2477751|2478023|2477415|2477688|N|N|N|N|N| +2477781|AAAAAAAAFNOMFCAA|2071-10-31|2061|8966|688|2071|6|10|31|4|2071|688|8966|Saturday|2071Q4|N|Y|N|2477751|2478023|2477416|2477689|N|N|N|N|N| +2477782|AAAAAAAAGNOMFCAA|2071-11-01|2062|8966|688|2071|0|11|1|4|2071|688|8966|Sunday|2071Q4|N|N|N|2477782|2478085|2477417|2477690|N|N|N|N|N| +2477783|AAAAAAAAHNOMFCAA|2071-11-02|2062|8966|688|2071|1|11|2|4|2071|688|8966|Monday|2071Q4|N|N|N|2477782|2478085|2477418|2477691|N|N|N|N|N| +2477784|AAAAAAAAINOMFCAA|2071-11-03|2062|8967|688|2071|2|11|3|4|2071|688|8967|Tuesday|2071Q4|N|N|N|2477782|2478085|2477419|2477692|N|N|N|N|N| +2477785|AAAAAAAAJNOMFCAA|2071-11-04|2062|8967|688|2071|3|11|4|4|2071|688|8967|Wednesday|2071Q4|N|N|N|2477782|2478085|2477420|2477693|N|N|N|N|N| +2477786|AAAAAAAAKNOMFCAA|2071-11-05|2062|8967|688|2071|4|11|5|4|2071|688|8967|Thursday|2071Q4|N|N|N|2477782|2478085|2477421|2477694|N|N|N|N|N| +2477787|AAAAAAAALNOMFCAA|2071-11-06|2062|8967|688|2071|5|11|6|4|2071|688|8967|Friday|2071Q4|N|Y|N|2477782|2478085|2477422|2477695|N|N|N|N|N| +2477788|AAAAAAAAMNOMFCAA|2071-11-07|2062|8967|688|2071|6|11|7|4|2071|688|8967|Saturday|2071Q4|N|Y|N|2477782|2478085|2477423|2477696|N|N|N|N|N| +2477789|AAAAAAAANNOMFCAA|2071-11-08|2062|8967|688|2071|0|11|8|4|2071|688|8967|Sunday|2071Q4|N|N|N|2477782|2478085|2477424|2477697|N|N|N|N|N| +2477790|AAAAAAAAONOMFCAA|2071-11-09|2062|8967|688|2071|1|11|9|4|2071|688|8967|Monday|2071Q4|N|N|N|2477782|2478085|2477425|2477698|N|N|N|N|N| +2477791|AAAAAAAAPNOMFCAA|2071-11-10|2062|8968|688|2071|2|11|10|4|2071|688|8968|Tuesday|2071Q4|N|N|N|2477782|2478085|2477426|2477699|N|N|N|N|N| +2477792|AAAAAAAAAOOMFCAA|2071-11-11|2062|8968|688|2071|3|11|11|4|2071|688|8968|Wednesday|2071Q4|N|N|N|2477782|2478085|2477427|2477700|N|N|N|N|N| +2477793|AAAAAAAABOOMFCAA|2071-11-12|2062|8968|688|2071|4|11|12|4|2071|688|8968|Thursday|2071Q4|N|N|N|2477782|2478085|2477428|2477701|N|N|N|N|N| +2477794|AAAAAAAACOOMFCAA|2071-11-13|2062|8968|688|2071|5|11|13|4|2071|688|8968|Friday|2071Q4|N|Y|N|2477782|2478085|2477429|2477702|N|N|N|N|N| +2477795|AAAAAAAADOOMFCAA|2071-11-14|2062|8968|688|2071|6|11|14|4|2071|688|8968|Saturday|2071Q4|N|Y|N|2477782|2478085|2477430|2477703|N|N|N|N|N| +2477796|AAAAAAAAEOOMFCAA|2071-11-15|2062|8968|688|2071|0|11|15|4|2071|688|8968|Sunday|2071Q4|N|N|N|2477782|2478085|2477431|2477704|N|N|N|N|N| +2477797|AAAAAAAAFOOMFCAA|2071-11-16|2062|8968|688|2071|1|11|16|4|2071|688|8968|Monday|2071Q4|N|N|N|2477782|2478085|2477432|2477705|N|N|N|N|N| +2477798|AAAAAAAAGOOMFCAA|2071-11-17|2062|8969|688|2071|2|11|17|4|2071|688|8969|Tuesday|2071Q4|N|N|N|2477782|2478085|2477433|2477706|N|N|N|N|N| +2477799|AAAAAAAAHOOMFCAA|2071-11-18|2062|8969|688|2071|3|11|18|4|2071|688|8969|Wednesday|2071Q4|N|N|N|2477782|2478085|2477434|2477707|N|N|N|N|N| +2477800|AAAAAAAAIOOMFCAA|2071-11-19|2062|8969|688|2071|4|11|19|4|2071|688|8969|Thursday|2071Q4|N|N|N|2477782|2478085|2477435|2477708|N|N|N|N|N| +2477801|AAAAAAAAJOOMFCAA|2071-11-20|2062|8969|688|2071|5|11|20|4|2071|688|8969|Friday|2071Q4|N|Y|N|2477782|2478085|2477436|2477709|N|N|N|N|N| +2477802|AAAAAAAAKOOMFCAA|2071-11-21|2062|8969|688|2071|6|11|21|4|2071|688|8969|Saturday|2071Q4|N|Y|N|2477782|2478085|2477437|2477710|N|N|N|N|N| +2477803|AAAAAAAALOOMFCAA|2071-11-22|2062|8969|688|2071|0|11|22|4|2071|688|8969|Sunday|2071Q4|N|N|N|2477782|2478085|2477438|2477711|N|N|N|N|N| +2477804|AAAAAAAAMOOMFCAA|2071-11-23|2062|8969|688|2071|1|11|23|4|2071|688|8969|Monday|2071Q4|N|N|N|2477782|2478085|2477439|2477712|N|N|N|N|N| +2477805|AAAAAAAANOOMFCAA|2071-11-24|2062|8970|688|2071|2|11|24|4|2071|688|8970|Tuesday|2071Q4|N|N|N|2477782|2478085|2477440|2477713|N|N|N|N|N| +2477806|AAAAAAAAOOOMFCAA|2071-11-25|2062|8970|688|2071|3|11|25|4|2071|688|8970|Wednesday|2071Q4|N|N|N|2477782|2478085|2477441|2477714|N|N|N|N|N| +2477807|AAAAAAAAPOOMFCAA|2071-11-26|2062|8970|688|2071|4|11|26|4|2071|688|8970|Thursday|2071Q4|N|N|N|2477782|2478085|2477442|2477715|N|N|N|N|N| +2477808|AAAAAAAAAPOMFCAA|2071-11-27|2062|8970|688|2071|5|11|27|4|2071|688|8970|Friday|2071Q4|N|Y|N|2477782|2478085|2477443|2477716|N|N|N|N|N| +2477809|AAAAAAAABPOMFCAA|2071-11-28|2062|8970|688|2071|6|11|28|4|2071|688|8970|Saturday|2071Q4|N|Y|N|2477782|2478085|2477444|2477717|N|N|N|N|N| +2477810|AAAAAAAACPOMFCAA|2071-11-29|2062|8970|688|2071|0|11|29|4|2071|688|8970|Sunday|2071Q4|N|N|N|2477782|2478085|2477445|2477718|N|N|N|N|N| +2477811|AAAAAAAADPOMFCAA|2071-11-30|2062|8970|688|2071|1|11|30|4|2071|688|8970|Monday|2071Q4|N|N|N|2477782|2478085|2477446|2477719|N|N|N|N|N| +2477812|AAAAAAAAEPOMFCAA|2071-12-01|2063|8971|689|2071|2|12|1|4|2071|689|8971|Tuesday|2071Q4|N|N|N|2477812|2478145|2477447|2477720|N|N|N|N|N| +2477813|AAAAAAAAFPOMFCAA|2071-12-02|2063|8971|689|2071|3|12|2|4|2071|689|8971|Wednesday|2071Q4|N|N|N|2477812|2478145|2477448|2477721|N|N|N|N|N| +2477814|AAAAAAAAGPOMFCAA|2071-12-03|2063|8971|689|2071|4|12|3|4|2071|689|8971|Thursday|2071Q4|N|N|N|2477812|2478145|2477449|2477722|N|N|N|N|N| +2477815|AAAAAAAAHPOMFCAA|2071-12-04|2063|8971|689|2071|5|12|4|4|2071|689|8971|Friday|2071Q4|N|Y|N|2477812|2478145|2477450|2477723|N|N|N|N|N| +2477816|AAAAAAAAIPOMFCAA|2071-12-05|2063|8971|689|2071|6|12|5|4|2071|689|8971|Saturday|2071Q4|N|Y|N|2477812|2478145|2477451|2477724|N|N|N|N|N| +2477817|AAAAAAAAJPOMFCAA|2071-12-06|2063|8971|689|2071|0|12|6|4|2071|689|8971|Sunday|2071Q4|N|N|N|2477812|2478145|2477452|2477725|N|N|N|N|N| +2477818|AAAAAAAAKPOMFCAA|2071-12-07|2063|8971|689|2071|1|12|7|4|2071|689|8971|Monday|2071Q4|N|N|N|2477812|2478145|2477453|2477726|N|N|N|N|N| +2477819|AAAAAAAALPOMFCAA|2071-12-08|2063|8972|689|2071|2|12|8|4|2071|689|8972|Tuesday|2071Q4|N|N|N|2477812|2478145|2477454|2477727|N|N|N|N|N| +2477820|AAAAAAAAMPOMFCAA|2071-12-09|2063|8972|689|2071|3|12|9|4|2071|689|8972|Wednesday|2071Q4|N|N|N|2477812|2478145|2477455|2477728|N|N|N|N|N| +2477821|AAAAAAAANPOMFCAA|2071-12-10|2063|8972|689|2071|4|12|10|4|2071|689|8972|Thursday|2071Q4|N|N|N|2477812|2478145|2477456|2477729|N|N|N|N|N| +2477822|AAAAAAAAOPOMFCAA|2071-12-11|2063|8972|689|2071|5|12|11|4|2071|689|8972|Friday|2071Q4|N|Y|N|2477812|2478145|2477457|2477730|N|N|N|N|N| +2477823|AAAAAAAAPPOMFCAA|2071-12-12|2063|8972|689|2071|6|12|12|4|2071|689|8972|Saturday|2071Q4|N|Y|N|2477812|2478145|2477458|2477731|N|N|N|N|N| +2477824|AAAAAAAAAAPMFCAA|2071-12-13|2063|8972|689|2071|0|12|13|4|2071|689|8972|Sunday|2071Q4|N|N|N|2477812|2478145|2477459|2477732|N|N|N|N|N| +2477825|AAAAAAAABAPMFCAA|2071-12-14|2063|8972|689|2071|1|12|14|4|2071|689|8972|Monday|2071Q4|N|N|N|2477812|2478145|2477460|2477733|N|N|N|N|N| +2477826|AAAAAAAACAPMFCAA|2071-12-15|2063|8973|689|2071|2|12|15|4|2071|689|8973|Tuesday|2071Q4|N|N|N|2477812|2478145|2477461|2477734|N|N|N|N|N| +2477827|AAAAAAAADAPMFCAA|2071-12-16|2063|8973|689|2071|3|12|16|4|2071|689|8973|Wednesday|2071Q4|N|N|N|2477812|2478145|2477462|2477735|N|N|N|N|N| +2477828|AAAAAAAAEAPMFCAA|2071-12-17|2063|8973|689|2071|4|12|17|4|2071|689|8973|Thursday|2071Q4|N|N|N|2477812|2478145|2477463|2477736|N|N|N|N|N| +2477829|AAAAAAAAFAPMFCAA|2071-12-18|2063|8973|689|2071|5|12|18|4|2071|689|8973|Friday|2071Q4|N|Y|N|2477812|2478145|2477464|2477737|N|N|N|N|N| +2477830|AAAAAAAAGAPMFCAA|2071-12-19|2063|8973|689|2071|6|12|19|4|2071|689|8973|Saturday|2071Q4|N|Y|N|2477812|2478145|2477465|2477738|N|N|N|N|N| +2477831|AAAAAAAAHAPMFCAA|2071-12-20|2063|8973|689|2071|0|12|20|4|2071|689|8973|Sunday|2071Q4|N|N|N|2477812|2478145|2477466|2477739|N|N|N|N|N| +2477832|AAAAAAAAIAPMFCAA|2071-12-21|2063|8973|689|2071|1|12|21|4|2071|689|8973|Monday|2071Q4|N|N|N|2477812|2478145|2477467|2477740|N|N|N|N|N| +2477833|AAAAAAAAJAPMFCAA|2071-12-22|2063|8974|689|2071|2|12|22|4|2071|689|8974|Tuesday|2071Q4|N|N|N|2477812|2478145|2477468|2477741|N|N|N|N|N| +2477834|AAAAAAAAKAPMFCAA|2071-12-23|2063|8974|689|2071|3|12|23|4|2071|689|8974|Wednesday|2071Q4|N|N|N|2477812|2478145|2477469|2477742|N|N|N|N|N| +2477835|AAAAAAAALAPMFCAA|2071-12-24|2063|8974|689|2071|4|12|24|4|2071|689|8974|Thursday|2071Q4|N|N|N|2477812|2478145|2477470|2477743|N|N|N|N|N| +2477836|AAAAAAAAMAPMFCAA|2071-12-25|2063|8974|689|2071|5|12|25|4|2071|689|8974|Friday|2071Q4|N|Y|N|2477812|2478145|2477471|2477744|N|N|N|N|N| +2477837|AAAAAAAANAPMFCAA|2071-12-26|2063|8974|689|2071|6|12|26|4|2071|689|8974|Saturday|2071Q4|Y|Y|N|2477812|2478145|2477472|2477745|N|N|N|N|N| +2477838|AAAAAAAAOAPMFCAA|2071-12-27|2063|8974|689|2071|0|12|27|4|2071|689|8974|Sunday|2071Q4|N|N|Y|2477812|2478145|2477473|2477746|N|N|N|N|N| +2477839|AAAAAAAAPAPMFCAA|2071-12-28|2063|8974|689|2071|1|12|28|4|2071|689|8974|Monday|2071Q4|N|N|N|2477812|2478145|2477474|2477747|N|N|N|N|N| +2477840|AAAAAAAAABPMFCAA|2071-12-29|2063|8975|689|2071|2|12|29|4|2071|689|8975|Tuesday|2071Q4|N|N|N|2477812|2478145|2477475|2477748|N|N|N|N|N| +2477841|AAAAAAAABBPMFCAA|2071-12-30|2063|8975|689|2071|3|12|30|4|2071|689|8975|Wednesday|2071Q4|N|N|N|2477812|2478145|2477476|2477749|N|N|N|N|N| +2477842|AAAAAAAACBPMFCAA|2071-12-31|2063|8975|689|2071|4|12|31|4|2071|689|8975|Thursday|2071Q4|N|N|N|2477812|2478145|2477477|2477750|N|N|N|N|N| +2477843|AAAAAAAADBPMFCAA|2072-01-01|2064|8975|689|2072|5|1|1|1|2072|689|8975|Friday|2072Q1|Y|Y|N|2477843|2477842|2477478|2477751|N|N|N|N|N| +2477844|AAAAAAAAEBPMFCAA|2072-01-02|2064|8975|689|2072|6|1|2|1|2072|689|8975|Saturday|2072Q1|N|Y|Y|2477843|2477842|2477479|2477752|N|N|N|N|N| +2477845|AAAAAAAAFBPMFCAA|2072-01-03|2064|8975|689|2072|0|1|3|1|2072|689|8975|Sunday|2072Q1|N|N|N|2477843|2477842|2477480|2477753|N|N|N|N|N| +2477846|AAAAAAAAGBPMFCAA|2072-01-04|2064|8975|689|2072|1|1|4|1|2072|689|8975|Monday|2072Q1|N|N|N|2477843|2477842|2477481|2477754|N|N|N|N|N| +2477847|AAAAAAAAHBPMFCAA|2072-01-05|2064|8976|689|2072|2|1|5|1|2072|689|8976|Tuesday|2072Q1|N|N|N|2477843|2477842|2477482|2477755|N|N|N|N|N| +2477848|AAAAAAAAIBPMFCAA|2072-01-06|2064|8976|689|2072|3|1|6|1|2072|689|8976|Wednesday|2072Q1|N|N|N|2477843|2477842|2477483|2477756|N|N|N|N|N| +2477849|AAAAAAAAJBPMFCAA|2072-01-07|2064|8976|689|2072|4|1|7|1|2072|689|8976|Thursday|2072Q1|N|N|N|2477843|2477842|2477484|2477757|N|N|N|N|N| +2477850|AAAAAAAAKBPMFCAA|2072-01-08|2064|8976|689|2072|5|1|8|1|2072|689|8976|Friday|2072Q1|N|Y|N|2477843|2477842|2477485|2477758|N|N|N|N|N| +2477851|AAAAAAAALBPMFCAA|2072-01-09|2064|8976|689|2072|6|1|9|1|2072|689|8976|Saturday|2072Q1|N|Y|N|2477843|2477842|2477486|2477759|N|N|N|N|N| +2477852|AAAAAAAAMBPMFCAA|2072-01-10|2064|8976|689|2072|0|1|10|1|2072|689|8976|Sunday|2072Q1|N|N|N|2477843|2477842|2477487|2477760|N|N|N|N|N| +2477853|AAAAAAAANBPMFCAA|2072-01-11|2064|8976|689|2072|1|1|11|1|2072|689|8976|Monday|2072Q1|N|N|N|2477843|2477842|2477488|2477761|N|N|N|N|N| +2477854|AAAAAAAAOBPMFCAA|2072-01-12|2064|8977|689|2072|2|1|12|1|2072|689|8977|Tuesday|2072Q1|N|N|N|2477843|2477842|2477489|2477762|N|N|N|N|N| +2477855|AAAAAAAAPBPMFCAA|2072-01-13|2064|8977|689|2072|3|1|13|1|2072|689|8977|Wednesday|2072Q1|N|N|N|2477843|2477842|2477490|2477763|N|N|N|N|N| +2477856|AAAAAAAAACPMFCAA|2072-01-14|2064|8977|689|2072|4|1|14|1|2072|689|8977|Thursday|2072Q1|N|N|N|2477843|2477842|2477491|2477764|N|N|N|N|N| +2477857|AAAAAAAABCPMFCAA|2072-01-15|2064|8977|689|2072|5|1|15|1|2072|689|8977|Friday|2072Q1|N|Y|N|2477843|2477842|2477492|2477765|N|N|N|N|N| +2477858|AAAAAAAACCPMFCAA|2072-01-16|2064|8977|689|2072|6|1|16|1|2072|689|8977|Saturday|2072Q1|N|Y|N|2477843|2477842|2477493|2477766|N|N|N|N|N| +2477859|AAAAAAAADCPMFCAA|2072-01-17|2064|8977|689|2072|0|1|17|1|2072|689|8977|Sunday|2072Q1|N|N|N|2477843|2477842|2477494|2477767|N|N|N|N|N| +2477860|AAAAAAAAECPMFCAA|2072-01-18|2064|8977|689|2072|1|1|18|1|2072|689|8977|Monday|2072Q1|N|N|N|2477843|2477842|2477495|2477768|N|N|N|N|N| +2477861|AAAAAAAAFCPMFCAA|2072-01-19|2064|8978|689|2072|2|1|19|1|2072|689|8978|Tuesday|2072Q1|N|N|N|2477843|2477842|2477496|2477769|N|N|N|N|N| +2477862|AAAAAAAAGCPMFCAA|2072-01-20|2064|8978|689|2072|3|1|20|1|2072|689|8978|Wednesday|2072Q1|N|N|N|2477843|2477842|2477497|2477770|N|N|N|N|N| +2477863|AAAAAAAAHCPMFCAA|2072-01-21|2064|8978|689|2072|4|1|21|1|2072|689|8978|Thursday|2072Q1|N|N|N|2477843|2477842|2477498|2477771|N|N|N|N|N| +2477864|AAAAAAAAICPMFCAA|2072-01-22|2064|8978|689|2072|5|1|22|1|2072|689|8978|Friday|2072Q1|N|Y|N|2477843|2477842|2477499|2477772|N|N|N|N|N| +2477865|AAAAAAAAJCPMFCAA|2072-01-23|2064|8978|689|2072|6|1|23|1|2072|689|8978|Saturday|2072Q1|N|Y|N|2477843|2477842|2477500|2477773|N|N|N|N|N| +2477866|AAAAAAAAKCPMFCAA|2072-01-24|2064|8978|689|2072|0|1|24|1|2072|689|8978|Sunday|2072Q1|N|N|N|2477843|2477842|2477501|2477774|N|N|N|N|N| +2477867|AAAAAAAALCPMFCAA|2072-01-25|2064|8978|689|2072|1|1|25|1|2072|689|8978|Monday|2072Q1|N|N|N|2477843|2477842|2477502|2477775|N|N|N|N|N| +2477868|AAAAAAAAMCPMFCAA|2072-01-26|2064|8979|689|2072|2|1|26|1|2072|689|8979|Tuesday|2072Q1|N|N|N|2477843|2477842|2477503|2477776|N|N|N|N|N| +2477869|AAAAAAAANCPMFCAA|2072-01-27|2064|8979|689|2072|3|1|27|1|2072|689|8979|Wednesday|2072Q1|N|N|N|2477843|2477842|2477504|2477777|N|N|N|N|N| +2477870|AAAAAAAAOCPMFCAA|2072-01-28|2064|8979|689|2072|4|1|28|1|2072|689|8979|Thursday|2072Q1|N|N|N|2477843|2477842|2477505|2477778|N|N|N|N|N| +2477871|AAAAAAAAPCPMFCAA|2072-01-29|2064|8979|689|2072|5|1|29|1|2072|689|8979|Friday|2072Q1|N|Y|N|2477843|2477842|2477506|2477779|N|N|N|N|N| +2477872|AAAAAAAAADPMFCAA|2072-01-30|2064|8979|689|2072|6|1|30|1|2072|689|8979|Saturday|2072Q1|N|Y|N|2477843|2477842|2477507|2477780|N|N|N|N|N| +2477873|AAAAAAAABDPMFCAA|2072-01-31|2064|8979|689|2072|0|1|31|1|2072|689|8979|Sunday|2072Q1|N|N|N|2477843|2477842|2477508|2477781|N|N|N|N|N| +2477874|AAAAAAAACDPMFCAA|2072-02-01|2065|8979|689|2072|1|2|1|1|2072|689|8979|Monday|2072Q1|N|N|N|2477874|2477904|2477509|2477782|N|N|N|N|N| +2477875|AAAAAAAADDPMFCAA|2072-02-02|2065|8980|689|2072|2|2|2|1|2072|689|8980|Tuesday|2072Q1|N|N|N|2477874|2477904|2477510|2477783|N|N|N|N|N| +2477876|AAAAAAAAEDPMFCAA|2072-02-03|2065|8980|689|2072|3|2|3|1|2072|689|8980|Wednesday|2072Q1|N|N|N|2477874|2477904|2477511|2477784|N|N|N|N|N| +2477877|AAAAAAAAFDPMFCAA|2072-02-04|2065|8980|689|2072|4|2|4|1|2072|689|8980|Thursday|2072Q1|N|N|N|2477874|2477904|2477512|2477785|N|N|N|N|N| +2477878|AAAAAAAAGDPMFCAA|2072-02-05|2065|8980|689|2072|5|2|5|1|2072|689|8980|Friday|2072Q1|N|Y|N|2477874|2477904|2477513|2477786|N|N|N|N|N| +2477879|AAAAAAAAHDPMFCAA|2072-02-06|2065|8980|689|2072|6|2|6|1|2072|689|8980|Saturday|2072Q1|N|Y|N|2477874|2477904|2477514|2477787|N|N|N|N|N| +2477880|AAAAAAAAIDPMFCAA|2072-02-07|2065|8980|689|2072|0|2|7|1|2072|689|8980|Sunday|2072Q1|N|N|N|2477874|2477904|2477515|2477788|N|N|N|N|N| +2477881|AAAAAAAAJDPMFCAA|2072-02-08|2065|8980|689|2072|1|2|8|1|2072|689|8980|Monday|2072Q1|N|N|N|2477874|2477904|2477516|2477789|N|N|N|N|N| +2477882|AAAAAAAAKDPMFCAA|2072-02-09|2065|8981|689|2072|2|2|9|1|2072|689|8981|Tuesday|2072Q1|N|N|N|2477874|2477904|2477517|2477790|N|N|N|N|N| +2477883|AAAAAAAALDPMFCAA|2072-02-10|2065|8981|689|2072|3|2|10|1|2072|689|8981|Wednesday|2072Q1|N|N|N|2477874|2477904|2477518|2477791|N|N|N|N|N| +2477884|AAAAAAAAMDPMFCAA|2072-02-11|2065|8981|689|2072|4|2|11|1|2072|689|8981|Thursday|2072Q1|N|N|N|2477874|2477904|2477519|2477792|N|N|N|N|N| +2477885|AAAAAAAANDPMFCAA|2072-02-12|2065|8981|689|2072|5|2|12|1|2072|689|8981|Friday|2072Q1|N|Y|N|2477874|2477904|2477520|2477793|N|N|N|N|N| +2477886|AAAAAAAAODPMFCAA|2072-02-13|2065|8981|689|2072|6|2|13|1|2072|689|8981|Saturday|2072Q1|N|Y|N|2477874|2477904|2477521|2477794|N|N|N|N|N| +2477887|AAAAAAAAPDPMFCAA|2072-02-14|2065|8981|689|2072|0|2|14|1|2072|689|8981|Sunday|2072Q1|N|N|N|2477874|2477904|2477522|2477795|N|N|N|N|N| +2477888|AAAAAAAAAEPMFCAA|2072-02-15|2065|8981|689|2072|1|2|15|1|2072|689|8981|Monday|2072Q1|N|N|N|2477874|2477904|2477523|2477796|N|N|N|N|N| +2477889|AAAAAAAABEPMFCAA|2072-02-16|2065|8982|689|2072|2|2|16|1|2072|689|8982|Tuesday|2072Q1|N|N|N|2477874|2477904|2477524|2477797|N|N|N|N|N| +2477890|AAAAAAAACEPMFCAA|2072-02-17|2065|8982|689|2072|3|2|17|1|2072|689|8982|Wednesday|2072Q1|N|N|N|2477874|2477904|2477525|2477798|N|N|N|N|N| +2477891|AAAAAAAADEPMFCAA|2072-02-18|2065|8982|689|2072|4|2|18|1|2072|689|8982|Thursday|2072Q1|N|N|N|2477874|2477904|2477526|2477799|N|N|N|N|N| +2477892|AAAAAAAAEEPMFCAA|2072-02-19|2065|8982|689|2072|5|2|19|1|2072|689|8982|Friday|2072Q1|N|Y|N|2477874|2477904|2477527|2477800|N|N|N|N|N| +2477893|AAAAAAAAFEPMFCAA|2072-02-20|2065|8982|689|2072|6|2|20|1|2072|689|8982|Saturday|2072Q1|N|Y|N|2477874|2477904|2477528|2477801|N|N|N|N|N| +2477894|AAAAAAAAGEPMFCAA|2072-02-21|2065|8982|689|2072|0|2|21|1|2072|689|8982|Sunday|2072Q1|N|N|N|2477874|2477904|2477529|2477802|N|N|N|N|N| +2477895|AAAAAAAAHEPMFCAA|2072-02-22|2065|8982|689|2072|1|2|22|1|2072|689|8982|Monday|2072Q1|N|N|N|2477874|2477904|2477530|2477803|N|N|N|N|N| +2477896|AAAAAAAAIEPMFCAA|2072-02-23|2065|8983|689|2072|2|2|23|1|2072|689|8983|Tuesday|2072Q1|N|N|N|2477874|2477904|2477531|2477804|N|N|N|N|N| +2477897|AAAAAAAAJEPMFCAA|2072-02-24|2065|8983|689|2072|3|2|24|1|2072|689|8983|Wednesday|2072Q1|N|N|N|2477874|2477904|2477532|2477805|N|N|N|N|N| +2477898|AAAAAAAAKEPMFCAA|2072-02-25|2065|8983|689|2072|4|2|25|1|2072|689|8983|Thursday|2072Q1|N|N|N|2477874|2477904|2477533|2477806|N|N|N|N|N| +2477899|AAAAAAAALEPMFCAA|2072-02-26|2065|8983|689|2072|5|2|26|1|2072|689|8983|Friday|2072Q1|N|Y|N|2477874|2477904|2477534|2477807|N|N|N|N|N| +2477900|AAAAAAAAMEPMFCAA|2072-02-27|2065|8983|689|2072|6|2|27|1|2072|689|8983|Saturday|2072Q1|N|Y|N|2477874|2477904|2477535|2477808|N|N|N|N|N| +2477901|AAAAAAAANEPMFCAA|2072-02-28|2065|8983|689|2072|0|2|28|1|2072|689|8983|Sunday|2072Q1|N|N|N|2477874|2477904|2477536|2477809|N|N|N|N|N| +2477902|AAAAAAAAOEPMFCAA|2072-02-29|2065|8983|689|2072|1|2|29|1|2072|689|8983|Monday|2072Q1|N|N|N|2477874|2477904|2477536|2477810|N|N|N|N|N| +2477903|AAAAAAAAPEPMFCAA|2072-03-01|2066|8984|690|2072|2|3|1|1|2072|690|8984|Tuesday|2072Q1|N|N|N|2477903|2477962|2477537|2477811|N|N|N|N|N| +2477904|AAAAAAAAAFPMFCAA|2072-03-02|2066|8984|690|2072|3|3|2|1|2072|690|8984|Wednesday|2072Q1|N|N|N|2477903|2477962|2477538|2477812|N|N|N|N|N| +2477905|AAAAAAAABFPMFCAA|2072-03-03|2066|8984|690|2072|4|3|3|1|2072|690|8984|Thursday|2072Q1|N|N|N|2477903|2477962|2477539|2477813|N|N|N|N|N| +2477906|AAAAAAAACFPMFCAA|2072-03-04|2066|8984|690|2072|5|3|4|1|2072|690|8984|Friday|2072Q1|N|Y|N|2477903|2477962|2477540|2477814|N|N|N|N|N| +2477907|AAAAAAAADFPMFCAA|2072-03-05|2066|8984|690|2072|6|3|5|1|2072|690|8984|Saturday|2072Q1|N|Y|N|2477903|2477962|2477541|2477815|N|N|N|N|N| +2477908|AAAAAAAAEFPMFCAA|2072-03-06|2066|8984|690|2072|0|3|6|1|2072|690|8984|Sunday|2072Q1|N|N|N|2477903|2477962|2477542|2477816|N|N|N|N|N| +2477909|AAAAAAAAFFPMFCAA|2072-03-07|2066|8984|690|2072|1|3|7|1|2072|690|8984|Monday|2072Q1|N|N|N|2477903|2477962|2477543|2477817|N|N|N|N|N| +2477910|AAAAAAAAGFPMFCAA|2072-03-08|2066|8985|690|2072|2|3|8|1|2072|690|8985|Tuesday|2072Q1|N|N|N|2477903|2477962|2477544|2477818|N|N|N|N|N| +2477911|AAAAAAAAHFPMFCAA|2072-03-09|2066|8985|690|2072|3|3|9|1|2072|690|8985|Wednesday|2072Q1|N|N|N|2477903|2477962|2477545|2477819|N|N|N|N|N| +2477912|AAAAAAAAIFPMFCAA|2072-03-10|2066|8985|690|2072|4|3|10|1|2072|690|8985|Thursday|2072Q1|N|N|N|2477903|2477962|2477546|2477820|N|N|N|N|N| +2477913|AAAAAAAAJFPMFCAA|2072-03-11|2066|8985|690|2072|5|3|11|1|2072|690|8985|Friday|2072Q1|N|Y|N|2477903|2477962|2477547|2477821|N|N|N|N|N| +2477914|AAAAAAAAKFPMFCAA|2072-03-12|2066|8985|690|2072|6|3|12|1|2072|690|8985|Saturday|2072Q1|N|Y|N|2477903|2477962|2477548|2477822|N|N|N|N|N| +2477915|AAAAAAAALFPMFCAA|2072-03-13|2066|8985|690|2072|0|3|13|1|2072|690|8985|Sunday|2072Q1|N|N|N|2477903|2477962|2477549|2477823|N|N|N|N|N| +2477916|AAAAAAAAMFPMFCAA|2072-03-14|2066|8985|690|2072|1|3|14|1|2072|690|8985|Monday|2072Q1|N|N|N|2477903|2477962|2477550|2477824|N|N|N|N|N| +2477917|AAAAAAAANFPMFCAA|2072-03-15|2066|8986|690|2072|2|3|15|1|2072|690|8986|Tuesday|2072Q1|N|N|N|2477903|2477962|2477551|2477825|N|N|N|N|N| +2477918|AAAAAAAAOFPMFCAA|2072-03-16|2066|8986|690|2072|3|3|16|1|2072|690|8986|Wednesday|2072Q1|N|N|N|2477903|2477962|2477552|2477826|N|N|N|N|N| +2477919|AAAAAAAAPFPMFCAA|2072-03-17|2066|8986|690|2072|4|3|17|1|2072|690|8986|Thursday|2072Q1|N|N|N|2477903|2477962|2477553|2477827|N|N|N|N|N| +2477920|AAAAAAAAAGPMFCAA|2072-03-18|2066|8986|690|2072|5|3|18|1|2072|690|8986|Friday|2072Q1|N|Y|N|2477903|2477962|2477554|2477828|N|N|N|N|N| +2477921|AAAAAAAABGPMFCAA|2072-03-19|2066|8986|690|2072|6|3|19|1|2072|690|8986|Saturday|2072Q1|N|Y|N|2477903|2477962|2477555|2477829|N|N|N|N|N| +2477922|AAAAAAAACGPMFCAA|2072-03-20|2066|8986|690|2072|0|3|20|1|2072|690|8986|Sunday|2072Q1|N|N|N|2477903|2477962|2477556|2477830|N|N|N|N|N| +2477923|AAAAAAAADGPMFCAA|2072-03-21|2066|8986|690|2072|1|3|21|1|2072|690|8986|Monday|2072Q1|N|N|N|2477903|2477962|2477557|2477831|N|N|N|N|N| +2477924|AAAAAAAAEGPMFCAA|2072-03-22|2066|8987|690|2072|2|3|22|1|2072|690|8987|Tuesday|2072Q1|N|N|N|2477903|2477962|2477558|2477832|N|N|N|N|N| +2477925|AAAAAAAAFGPMFCAA|2072-03-23|2066|8987|690|2072|3|3|23|1|2072|690|8987|Wednesday|2072Q1|N|N|N|2477903|2477962|2477559|2477833|N|N|N|N|N| +2477926|AAAAAAAAGGPMFCAA|2072-03-24|2066|8987|690|2072|4|3|24|1|2072|690|8987|Thursday|2072Q1|N|N|N|2477903|2477962|2477560|2477834|N|N|N|N|N| +2477927|AAAAAAAAHGPMFCAA|2072-03-25|2066|8987|690|2072|5|3|25|1|2072|690|8987|Friday|2072Q1|N|Y|N|2477903|2477962|2477561|2477835|N|N|N|N|N| +2477928|AAAAAAAAIGPMFCAA|2072-03-26|2066|8987|690|2072|6|3|26|1|2072|690|8987|Saturday|2072Q1|N|Y|N|2477903|2477962|2477562|2477836|N|N|N|N|N| +2477929|AAAAAAAAJGPMFCAA|2072-03-27|2066|8987|690|2072|0|3|27|1|2072|690|8987|Sunday|2072Q1|N|N|N|2477903|2477962|2477563|2477837|N|N|N|N|N| +2477930|AAAAAAAAKGPMFCAA|2072-03-28|2066|8987|690|2072|1|3|28|1|2072|690|8987|Monday|2072Q1|N|N|N|2477903|2477962|2477564|2477838|N|N|N|N|N| +2477931|AAAAAAAALGPMFCAA|2072-03-29|2066|8988|690|2072|2|3|29|1|2072|690|8988|Tuesday|2072Q1|N|N|N|2477903|2477962|2477565|2477839|N|N|N|N|N| +2477932|AAAAAAAAMGPMFCAA|2072-03-30|2066|8988|690|2072|3|3|30|1|2072|690|8988|Wednesday|2072Q1|N|N|N|2477903|2477962|2477566|2477840|N|N|N|N|N| +2477933|AAAAAAAANGPMFCAA|2072-03-31|2066|8988|690|2072|4|3|31|1|2072|690|8988|Thursday|2072Q1|N|N|N|2477903|2477962|2477567|2477841|N|N|N|N|N| +2477934|AAAAAAAAOGPMFCAA|2072-04-01|2067|8988|690|2072|5|4|1|2|2072|690|8988|Friday|2072Q2|N|Y|N|2477934|2478024|2477568|2477843|N|N|N|N|N| +2477935|AAAAAAAAPGPMFCAA|2072-04-02|2067|8988|690|2072|6|4|2|2|2072|690|8988|Saturday|2072Q2|N|Y|N|2477934|2478024|2477569|2477844|N|N|N|N|N| +2477936|AAAAAAAAAHPMFCAA|2072-04-03|2067|8988|690|2072|0|4|3|2|2072|690|8988|Sunday|2072Q2|N|N|N|2477934|2478024|2477570|2477845|N|N|N|N|N| +2477937|AAAAAAAABHPMFCAA|2072-04-04|2067|8988|690|2072|1|4|4|2|2072|690|8988|Monday|2072Q2|N|N|N|2477934|2478024|2477571|2477846|N|N|N|N|N| +2477938|AAAAAAAACHPMFCAA|2072-04-05|2067|8989|690|2072|2|4|5|2|2072|690|8989|Tuesday|2072Q2|N|N|N|2477934|2478024|2477572|2477847|N|N|N|N|N| +2477939|AAAAAAAADHPMFCAA|2072-04-06|2067|8989|690|2072|3|4|6|2|2072|690|8989|Wednesday|2072Q2|N|N|N|2477934|2478024|2477573|2477848|N|N|N|N|N| +2477940|AAAAAAAAEHPMFCAA|2072-04-07|2067|8989|690|2072|4|4|7|2|2072|690|8989|Thursday|2072Q2|N|N|N|2477934|2478024|2477574|2477849|N|N|N|N|N| +2477941|AAAAAAAAFHPMFCAA|2072-04-08|2067|8989|690|2072|5|4|8|2|2072|690|8989|Friday|2072Q2|N|Y|N|2477934|2478024|2477575|2477850|N|N|N|N|N| +2477942|AAAAAAAAGHPMFCAA|2072-04-09|2067|8989|690|2072|6|4|9|2|2072|690|8989|Saturday|2072Q2|N|Y|N|2477934|2478024|2477576|2477851|N|N|N|N|N| +2477943|AAAAAAAAHHPMFCAA|2072-04-10|2067|8989|690|2072|0|4|10|2|2072|690|8989|Sunday|2072Q2|N|N|N|2477934|2478024|2477577|2477852|N|N|N|N|N| +2477944|AAAAAAAAIHPMFCAA|2072-04-11|2067|8989|690|2072|1|4|11|2|2072|690|8989|Monday|2072Q2|N|N|N|2477934|2478024|2477578|2477853|N|N|N|N|N| +2477945|AAAAAAAAJHPMFCAA|2072-04-12|2067|8990|690|2072|2|4|12|2|2072|690|8990|Tuesday|2072Q2|N|N|N|2477934|2478024|2477579|2477854|N|N|N|N|N| +2477946|AAAAAAAAKHPMFCAA|2072-04-13|2067|8990|690|2072|3|4|13|2|2072|690|8990|Wednesday|2072Q2|N|N|N|2477934|2478024|2477580|2477855|N|N|N|N|N| +2477947|AAAAAAAALHPMFCAA|2072-04-14|2067|8990|690|2072|4|4|14|2|2072|690|8990|Thursday|2072Q2|N|N|N|2477934|2478024|2477581|2477856|N|N|N|N|N| +2477948|AAAAAAAAMHPMFCAA|2072-04-15|2067|8990|690|2072|5|4|15|2|2072|690|8990|Friday|2072Q2|N|Y|N|2477934|2478024|2477582|2477857|N|N|N|N|N| +2477949|AAAAAAAANHPMFCAA|2072-04-16|2067|8990|690|2072|6|4|16|2|2072|690|8990|Saturday|2072Q2|N|Y|N|2477934|2478024|2477583|2477858|N|N|N|N|N| +2477950|AAAAAAAAOHPMFCAA|2072-04-17|2067|8990|690|2072|0|4|17|2|2072|690|8990|Sunday|2072Q2|N|N|N|2477934|2478024|2477584|2477859|N|N|N|N|N| +2477951|AAAAAAAAPHPMFCAA|2072-04-18|2067|8990|690|2072|1|4|18|2|2072|690|8990|Monday|2072Q2|N|N|N|2477934|2478024|2477585|2477860|N|N|N|N|N| +2477952|AAAAAAAAAIPMFCAA|2072-04-19|2067|8991|690|2072|2|4|19|2|2072|690|8991|Tuesday|2072Q2|N|N|N|2477934|2478024|2477586|2477861|N|N|N|N|N| +2477953|AAAAAAAABIPMFCAA|2072-04-20|2067|8991|690|2072|3|4|20|2|2072|690|8991|Wednesday|2072Q2|N|N|N|2477934|2478024|2477587|2477862|N|N|N|N|N| +2477954|AAAAAAAACIPMFCAA|2072-04-21|2067|8991|690|2072|4|4|21|2|2072|690|8991|Thursday|2072Q2|N|N|N|2477934|2478024|2477588|2477863|N|N|N|N|N| +2477955|AAAAAAAADIPMFCAA|2072-04-22|2067|8991|690|2072|5|4|22|2|2072|690|8991|Friday|2072Q2|N|Y|N|2477934|2478024|2477589|2477864|N|N|N|N|N| +2477956|AAAAAAAAEIPMFCAA|2072-04-23|2067|8991|690|2072|6|4|23|2|2072|690|8991|Saturday|2072Q2|N|Y|N|2477934|2478024|2477590|2477865|N|N|N|N|N| +2477957|AAAAAAAAFIPMFCAA|2072-04-24|2067|8991|690|2072|0|4|24|2|2072|690|8991|Sunday|2072Q2|N|N|N|2477934|2478024|2477591|2477866|N|N|N|N|N| +2477958|AAAAAAAAGIPMFCAA|2072-04-25|2067|8991|690|2072|1|4|25|2|2072|690|8991|Monday|2072Q2|N|N|N|2477934|2478024|2477592|2477867|N|N|N|N|N| +2477959|AAAAAAAAHIPMFCAA|2072-04-26|2067|8992|690|2072|2|4|26|2|2072|690|8992|Tuesday|2072Q2|N|N|N|2477934|2478024|2477593|2477868|N|N|N|N|N| +2477960|AAAAAAAAIIPMFCAA|2072-04-27|2067|8992|690|2072|3|4|27|2|2072|690|8992|Wednesday|2072Q2|N|N|N|2477934|2478024|2477594|2477869|N|N|N|N|N| +2477961|AAAAAAAAJIPMFCAA|2072-04-28|2067|8992|690|2072|4|4|28|2|2072|690|8992|Thursday|2072Q2|N|N|N|2477934|2478024|2477595|2477870|N|N|N|N|N| +2477962|AAAAAAAAKIPMFCAA|2072-04-29|2067|8992|690|2072|5|4|29|2|2072|690|8992|Friday|2072Q2|N|Y|N|2477934|2478024|2477596|2477871|N|N|N|N|N| +2477963|AAAAAAAALIPMFCAA|2072-04-30|2067|8992|690|2072|6|4|30|2|2072|690|8992|Saturday|2072Q2|N|Y|N|2477934|2478024|2477597|2477872|N|N|N|N|N| +2477964|AAAAAAAAMIPMFCAA|2072-05-01|2068|8992|690|2072|0|5|1|2|2072|690|8992|Sunday|2072Q2|N|N|N|2477964|2478084|2477598|2477873|N|N|N|N|N| +2477965|AAAAAAAANIPMFCAA|2072-05-02|2068|8992|690|2072|1|5|2|2|2072|690|8992|Monday|2072Q2|N|N|N|2477964|2478084|2477599|2477874|N|N|N|N|N| +2477966|AAAAAAAAOIPMFCAA|2072-05-03|2068|8993|690|2072|2|5|3|2|2072|690|8993|Tuesday|2072Q2|N|N|N|2477964|2478084|2477600|2477875|N|N|N|N|N| +2477967|AAAAAAAAPIPMFCAA|2072-05-04|2068|8993|690|2072|3|5|4|2|2072|690|8993|Wednesday|2072Q2|N|N|N|2477964|2478084|2477601|2477876|N|N|N|N|N| +2477968|AAAAAAAAAJPMFCAA|2072-05-05|2068|8993|690|2072|4|5|5|2|2072|690|8993|Thursday|2072Q2|N|N|N|2477964|2478084|2477602|2477877|N|N|N|N|N| +2477969|AAAAAAAABJPMFCAA|2072-05-06|2068|8993|690|2072|5|5|6|2|2072|690|8993|Friday|2072Q2|N|Y|N|2477964|2478084|2477603|2477878|N|N|N|N|N| +2477970|AAAAAAAACJPMFCAA|2072-05-07|2068|8993|690|2072|6|5|7|2|2072|690|8993|Saturday|2072Q2|N|Y|N|2477964|2478084|2477604|2477879|N|N|N|N|N| +2477971|AAAAAAAADJPMFCAA|2072-05-08|2068|8993|690|2072|0|5|8|2|2072|690|8993|Sunday|2072Q2|N|N|N|2477964|2478084|2477605|2477880|N|N|N|N|N| +2477972|AAAAAAAAEJPMFCAA|2072-05-09|2068|8993|690|2072|1|5|9|2|2072|690|8993|Monday|2072Q2|N|N|N|2477964|2478084|2477606|2477881|N|N|N|N|N| +2477973|AAAAAAAAFJPMFCAA|2072-05-10|2068|8994|690|2072|2|5|10|2|2072|690|8994|Tuesday|2072Q2|N|N|N|2477964|2478084|2477607|2477882|N|N|N|N|N| +2477974|AAAAAAAAGJPMFCAA|2072-05-11|2068|8994|690|2072|3|5|11|2|2072|690|8994|Wednesday|2072Q2|N|N|N|2477964|2478084|2477608|2477883|N|N|N|N|N| +2477975|AAAAAAAAHJPMFCAA|2072-05-12|2068|8994|690|2072|4|5|12|2|2072|690|8994|Thursday|2072Q2|N|N|N|2477964|2478084|2477609|2477884|N|N|N|N|N| +2477976|AAAAAAAAIJPMFCAA|2072-05-13|2068|8994|690|2072|5|5|13|2|2072|690|8994|Friday|2072Q2|N|Y|N|2477964|2478084|2477610|2477885|N|N|N|N|N| +2477977|AAAAAAAAJJPMFCAA|2072-05-14|2068|8994|690|2072|6|5|14|2|2072|690|8994|Saturday|2072Q2|N|Y|N|2477964|2478084|2477611|2477886|N|N|N|N|N| +2477978|AAAAAAAAKJPMFCAA|2072-05-15|2068|8994|690|2072|0|5|15|2|2072|690|8994|Sunday|2072Q2|N|N|N|2477964|2478084|2477612|2477887|N|N|N|N|N| +2477979|AAAAAAAALJPMFCAA|2072-05-16|2068|8994|690|2072|1|5|16|2|2072|690|8994|Monday|2072Q2|N|N|N|2477964|2478084|2477613|2477888|N|N|N|N|N| +2477980|AAAAAAAAMJPMFCAA|2072-05-17|2068|8995|690|2072|2|5|17|2|2072|690|8995|Tuesday|2072Q2|N|N|N|2477964|2478084|2477614|2477889|N|N|N|N|N| +2477981|AAAAAAAANJPMFCAA|2072-05-18|2068|8995|690|2072|3|5|18|2|2072|690|8995|Wednesday|2072Q2|N|N|N|2477964|2478084|2477615|2477890|N|N|N|N|N| +2477982|AAAAAAAAOJPMFCAA|2072-05-19|2068|8995|690|2072|4|5|19|2|2072|690|8995|Thursday|2072Q2|N|N|N|2477964|2478084|2477616|2477891|N|N|N|N|N| +2477983|AAAAAAAAPJPMFCAA|2072-05-20|2068|8995|690|2072|5|5|20|2|2072|690|8995|Friday|2072Q2|N|Y|N|2477964|2478084|2477617|2477892|N|N|N|N|N| +2477984|AAAAAAAAAKPMFCAA|2072-05-21|2068|8995|690|2072|6|5|21|2|2072|690|8995|Saturday|2072Q2|N|Y|N|2477964|2478084|2477618|2477893|N|N|N|N|N| +2477985|AAAAAAAABKPMFCAA|2072-05-22|2068|8995|690|2072|0|5|22|2|2072|690|8995|Sunday|2072Q2|N|N|N|2477964|2478084|2477619|2477894|N|N|N|N|N| +2477986|AAAAAAAACKPMFCAA|2072-05-23|2068|8995|690|2072|1|5|23|2|2072|690|8995|Monday|2072Q2|N|N|N|2477964|2478084|2477620|2477895|N|N|N|N|N| +2477987|AAAAAAAADKPMFCAA|2072-05-24|2068|8996|690|2072|2|5|24|2|2072|690|8996|Tuesday|2072Q2|N|N|N|2477964|2478084|2477621|2477896|N|N|N|N|N| +2477988|AAAAAAAAEKPMFCAA|2072-05-25|2068|8996|690|2072|3|5|25|2|2072|690|8996|Wednesday|2072Q2|N|N|N|2477964|2478084|2477622|2477897|N|N|N|N|N| +2477989|AAAAAAAAFKPMFCAA|2072-05-26|2068|8996|690|2072|4|5|26|2|2072|690|8996|Thursday|2072Q2|N|N|N|2477964|2478084|2477623|2477898|N|N|N|N|N| +2477990|AAAAAAAAGKPMFCAA|2072-05-27|2068|8996|690|2072|5|5|27|2|2072|690|8996|Friday|2072Q2|N|Y|N|2477964|2478084|2477624|2477899|N|N|N|N|N| +2477991|AAAAAAAAHKPMFCAA|2072-05-28|2068|8996|690|2072|6|5|28|2|2072|690|8996|Saturday|2072Q2|N|Y|N|2477964|2478084|2477625|2477900|N|N|N|N|N| +2477992|AAAAAAAAIKPMFCAA|2072-05-29|2068|8996|690|2072|0|5|29|2|2072|690|8996|Sunday|2072Q2|N|N|N|2477964|2478084|2477626|2477901|N|N|N|N|N| +2477993|AAAAAAAAJKPMFCAA|2072-05-30|2068|8996|690|2072|1|5|30|2|2072|690|8996|Monday|2072Q2|N|N|N|2477964|2478084|2477627|2477902|N|N|N|N|N| +2477994|AAAAAAAAKKPMFCAA|2072-05-31|2068|8997|690|2072|2|5|31|2|2072|690|8997|Tuesday|2072Q2|N|N|N|2477964|2478084|2477628|2477903|N|N|N|N|N| +2477995|AAAAAAAALKPMFCAA|2072-06-01|2069|8997|691|2072|3|6|1|2|2072|691|8997|Wednesday|2072Q2|N|N|N|2477995|2478146|2477629|2477904|N|N|N|N|N| +2477996|AAAAAAAAMKPMFCAA|2072-06-02|2069|8997|691|2072|4|6|2|2|2072|691|8997|Thursday|2072Q2|N|N|N|2477995|2478146|2477630|2477905|N|N|N|N|N| +2477997|AAAAAAAANKPMFCAA|2072-06-03|2069|8997|691|2072|5|6|3|2|2072|691|8997|Friday|2072Q2|N|Y|N|2477995|2478146|2477631|2477906|N|N|N|N|N| +2477998|AAAAAAAAOKPMFCAA|2072-06-04|2069|8997|691|2072|6|6|4|2|2072|691|8997|Saturday|2072Q2|N|Y|N|2477995|2478146|2477632|2477907|N|N|N|N|N| +2477999|AAAAAAAAPKPMFCAA|2072-06-05|2069|8997|691|2072|0|6|5|2|2072|691|8997|Sunday|2072Q2|N|N|N|2477995|2478146|2477633|2477908|N|N|N|N|N| +2478000|AAAAAAAAALPMFCAA|2072-06-06|2069|8997|691|2072|1|6|6|2|2072|691|8997|Monday|2072Q2|N|N|N|2477995|2478146|2477634|2477909|N|N|N|N|N| +2478001|AAAAAAAABLPMFCAA|2072-06-07|2069|8998|691|2072|2|6|7|2|2072|691|8998|Tuesday|2072Q2|N|N|N|2477995|2478146|2477635|2477910|N|N|N|N|N| +2478002|AAAAAAAACLPMFCAA|2072-06-08|2069|8998|691|2072|3|6|8|2|2072|691|8998|Wednesday|2072Q2|N|N|N|2477995|2478146|2477636|2477911|N|N|N|N|N| +2478003|AAAAAAAADLPMFCAA|2072-06-09|2069|8998|691|2072|4|6|9|2|2072|691|8998|Thursday|2072Q2|N|N|N|2477995|2478146|2477637|2477912|N|N|N|N|N| +2478004|AAAAAAAAELPMFCAA|2072-06-10|2069|8998|691|2072|5|6|10|2|2072|691|8998|Friday|2072Q2|N|Y|N|2477995|2478146|2477638|2477913|N|N|N|N|N| +2478005|AAAAAAAAFLPMFCAA|2072-06-11|2069|8998|691|2072|6|6|11|2|2072|691|8998|Saturday|2072Q2|N|Y|N|2477995|2478146|2477639|2477914|N|N|N|N|N| +2478006|AAAAAAAAGLPMFCAA|2072-06-12|2069|8998|691|2072|0|6|12|2|2072|691|8998|Sunday|2072Q2|N|N|N|2477995|2478146|2477640|2477915|N|N|N|N|N| +2478007|AAAAAAAAHLPMFCAA|2072-06-13|2069|8998|691|2072|1|6|13|2|2072|691|8998|Monday|2072Q2|N|N|N|2477995|2478146|2477641|2477916|N|N|N|N|N| +2478008|AAAAAAAAILPMFCAA|2072-06-14|2069|8999|691|2072|2|6|14|2|2072|691|8999|Tuesday|2072Q2|N|N|N|2477995|2478146|2477642|2477917|N|N|N|N|N| +2478009|AAAAAAAAJLPMFCAA|2072-06-15|2069|8999|691|2072|3|6|15|2|2072|691|8999|Wednesday|2072Q2|N|N|N|2477995|2478146|2477643|2477918|N|N|N|N|N| +2478010|AAAAAAAAKLPMFCAA|2072-06-16|2069|8999|691|2072|4|6|16|2|2072|691|8999|Thursday|2072Q2|N|N|N|2477995|2478146|2477644|2477919|N|N|N|N|N| +2478011|AAAAAAAALLPMFCAA|2072-06-17|2069|8999|691|2072|5|6|17|2|2072|691|8999|Friday|2072Q2|N|Y|N|2477995|2478146|2477645|2477920|N|N|N|N|N| +2478012|AAAAAAAAMLPMFCAA|2072-06-18|2069|8999|691|2072|6|6|18|2|2072|691|8999|Saturday|2072Q2|N|Y|N|2477995|2478146|2477646|2477921|N|N|N|N|N| +2478013|AAAAAAAANLPMFCAA|2072-06-19|2069|8999|691|2072|0|6|19|2|2072|691|8999|Sunday|2072Q2|N|N|N|2477995|2478146|2477647|2477922|N|N|N|N|N| +2478014|AAAAAAAAOLPMFCAA|2072-06-20|2069|8999|691|2072|1|6|20|2|2072|691|8999|Monday|2072Q2|N|N|N|2477995|2478146|2477648|2477923|N|N|N|N|N| +2478015|AAAAAAAAPLPMFCAA|2072-06-21|2069|9000|691|2072|2|6|21|2|2072|691|9000|Tuesday|2072Q2|N|N|N|2477995|2478146|2477649|2477924|N|N|N|N|N| +2478016|AAAAAAAAAMPMFCAA|2072-06-22|2069|9000|691|2072|3|6|22|2|2072|691|9000|Wednesday|2072Q2|N|N|N|2477995|2478146|2477650|2477925|N|N|N|N|N| +2478017|AAAAAAAABMPMFCAA|2072-06-23|2069|9000|691|2072|4|6|23|2|2072|691|9000|Thursday|2072Q2|N|N|N|2477995|2478146|2477651|2477926|N|N|N|N|N| +2478018|AAAAAAAACMPMFCAA|2072-06-24|2069|9000|691|2072|5|6|24|2|2072|691|9000|Friday|2072Q2|N|Y|N|2477995|2478146|2477652|2477927|N|N|N|N|N| +2478019|AAAAAAAADMPMFCAA|2072-06-25|2069|9000|691|2072|6|6|25|2|2072|691|9000|Saturday|2072Q2|N|Y|N|2477995|2478146|2477653|2477928|N|N|N|N|N| +2478020|AAAAAAAAEMPMFCAA|2072-06-26|2069|9000|691|2072|0|6|26|2|2072|691|9000|Sunday|2072Q2|N|N|N|2477995|2478146|2477654|2477929|N|N|N|N|N| +2478021|AAAAAAAAFMPMFCAA|2072-06-27|2069|9000|691|2072|1|6|27|2|2072|691|9000|Monday|2072Q2|N|N|N|2477995|2478146|2477655|2477930|N|N|N|N|N| +2478022|AAAAAAAAGMPMFCAA|2072-06-28|2069|9001|691|2072|2|6|28|2|2072|691|9001|Tuesday|2072Q2|N|N|N|2477995|2478146|2477656|2477931|N|N|N|N|N| +2478023|AAAAAAAAHMPMFCAA|2072-06-29|2069|9001|691|2072|3|6|29|2|2072|691|9001|Wednesday|2072Q2|N|N|N|2477995|2478146|2477657|2477932|N|N|N|N|N| +2478024|AAAAAAAAIMPMFCAA|2072-06-30|2069|9001|691|2072|4|6|30|2|2072|691|9001|Thursday|2072Q2|N|N|N|2477995|2478146|2477658|2477933|N|N|N|N|N| +2478025|AAAAAAAAJMPMFCAA|2072-07-01|2070|9001|691|2072|5|7|1|3|2072|691|9001|Friday|2072Q3|N|Y|N|2478025|2478206|2477659|2477934|N|N|N|N|N| +2478026|AAAAAAAAKMPMFCAA|2072-07-02|2070|9001|691|2072|6|7|2|3|2072|691|9001|Saturday|2072Q3|N|Y|N|2478025|2478206|2477660|2477935|N|N|N|N|N| +2478027|AAAAAAAALMPMFCAA|2072-07-03|2070|9001|691|2072|0|7|3|3|2072|691|9001|Sunday|2072Q3|N|N|N|2478025|2478206|2477661|2477936|N|N|N|N|N| +2478028|AAAAAAAAMMPMFCAA|2072-07-04|2070|9001|691|2072|1|7|4|3|2072|691|9001|Monday|2072Q3|Y|N|N|2478025|2478206|2477662|2477937|N|N|N|N|N| +2478029|AAAAAAAANMPMFCAA|2072-07-05|2070|9002|691|2072|2|7|5|3|2072|691|9002|Tuesday|2072Q3|N|N|Y|2478025|2478206|2477663|2477938|N|N|N|N|N| +2478030|AAAAAAAAOMPMFCAA|2072-07-06|2070|9002|691|2072|3|7|6|3|2072|691|9002|Wednesday|2072Q3|N|N|N|2478025|2478206|2477664|2477939|N|N|N|N|N| +2478031|AAAAAAAAPMPMFCAA|2072-07-07|2070|9002|691|2072|4|7|7|3|2072|691|9002|Thursday|2072Q3|N|N|N|2478025|2478206|2477665|2477940|N|N|N|N|N| +2478032|AAAAAAAAANPMFCAA|2072-07-08|2070|9002|691|2072|5|7|8|3|2072|691|9002|Friday|2072Q3|N|Y|N|2478025|2478206|2477666|2477941|N|N|N|N|N| +2478033|AAAAAAAABNPMFCAA|2072-07-09|2070|9002|691|2072|6|7|9|3|2072|691|9002|Saturday|2072Q3|N|Y|N|2478025|2478206|2477667|2477942|N|N|N|N|N| +2478034|AAAAAAAACNPMFCAA|2072-07-10|2070|9002|691|2072|0|7|10|3|2072|691|9002|Sunday|2072Q3|N|N|N|2478025|2478206|2477668|2477943|N|N|N|N|N| +2478035|AAAAAAAADNPMFCAA|2072-07-11|2070|9002|691|2072|1|7|11|3|2072|691|9002|Monday|2072Q3|N|N|N|2478025|2478206|2477669|2477944|N|N|N|N|N| +2478036|AAAAAAAAENPMFCAA|2072-07-12|2070|9003|691|2072|2|7|12|3|2072|691|9003|Tuesday|2072Q3|N|N|N|2478025|2478206|2477670|2477945|N|N|N|N|N| +2478037|AAAAAAAAFNPMFCAA|2072-07-13|2070|9003|691|2072|3|7|13|3|2072|691|9003|Wednesday|2072Q3|N|N|N|2478025|2478206|2477671|2477946|N|N|N|N|N| +2478038|AAAAAAAAGNPMFCAA|2072-07-14|2070|9003|691|2072|4|7|14|3|2072|691|9003|Thursday|2072Q3|N|N|N|2478025|2478206|2477672|2477947|N|N|N|N|N| +2478039|AAAAAAAAHNPMFCAA|2072-07-15|2070|9003|691|2072|5|7|15|3|2072|691|9003|Friday|2072Q3|N|Y|N|2478025|2478206|2477673|2477948|N|N|N|N|N| +2478040|AAAAAAAAINPMFCAA|2072-07-16|2070|9003|691|2072|6|7|16|3|2072|691|9003|Saturday|2072Q3|N|Y|N|2478025|2478206|2477674|2477949|N|N|N|N|N| +2478041|AAAAAAAAJNPMFCAA|2072-07-17|2070|9003|691|2072|0|7|17|3|2072|691|9003|Sunday|2072Q3|N|N|N|2478025|2478206|2477675|2477950|N|N|N|N|N| +2478042|AAAAAAAAKNPMFCAA|2072-07-18|2070|9003|691|2072|1|7|18|3|2072|691|9003|Monday|2072Q3|N|N|N|2478025|2478206|2477676|2477951|N|N|N|N|N| +2478043|AAAAAAAALNPMFCAA|2072-07-19|2070|9004|691|2072|2|7|19|3|2072|691|9004|Tuesday|2072Q3|N|N|N|2478025|2478206|2477677|2477952|N|N|N|N|N| +2478044|AAAAAAAAMNPMFCAA|2072-07-20|2070|9004|691|2072|3|7|20|3|2072|691|9004|Wednesday|2072Q3|N|N|N|2478025|2478206|2477678|2477953|N|N|N|N|N| +2478045|AAAAAAAANNPMFCAA|2072-07-21|2070|9004|691|2072|4|7|21|3|2072|691|9004|Thursday|2072Q3|N|N|N|2478025|2478206|2477679|2477954|N|N|N|N|N| +2478046|AAAAAAAAONPMFCAA|2072-07-22|2070|9004|691|2072|5|7|22|3|2072|691|9004|Friday|2072Q3|N|Y|N|2478025|2478206|2477680|2477955|N|N|N|N|N| +2478047|AAAAAAAAPNPMFCAA|2072-07-23|2070|9004|691|2072|6|7|23|3|2072|691|9004|Saturday|2072Q3|N|Y|N|2478025|2478206|2477681|2477956|N|N|N|N|N| +2478048|AAAAAAAAAOPMFCAA|2072-07-24|2070|9004|691|2072|0|7|24|3|2072|691|9004|Sunday|2072Q3|N|N|N|2478025|2478206|2477682|2477957|N|N|N|N|N| +2478049|AAAAAAAABOPMFCAA|2072-07-25|2070|9004|691|2072|1|7|25|3|2072|691|9004|Monday|2072Q3|N|N|N|2478025|2478206|2477683|2477958|N|N|N|N|N| +2478050|AAAAAAAACOPMFCAA|2072-07-26|2070|9005|691|2072|2|7|26|3|2072|691|9005|Tuesday|2072Q3|N|N|N|2478025|2478206|2477684|2477959|N|N|N|N|N| +2478051|AAAAAAAADOPMFCAA|2072-07-27|2070|9005|691|2072|3|7|27|3|2072|691|9005|Wednesday|2072Q3|N|N|N|2478025|2478206|2477685|2477960|N|N|N|N|N| +2478052|AAAAAAAAEOPMFCAA|2072-07-28|2070|9005|691|2072|4|7|28|3|2072|691|9005|Thursday|2072Q3|N|N|N|2478025|2478206|2477686|2477961|N|N|N|N|N| +2478053|AAAAAAAAFOPMFCAA|2072-07-29|2070|9005|691|2072|5|7|29|3|2072|691|9005|Friday|2072Q3|N|Y|N|2478025|2478206|2477687|2477962|N|N|N|N|N| +2478054|AAAAAAAAGOPMFCAA|2072-07-30|2070|9005|691|2072|6|7|30|3|2072|691|9005|Saturday|2072Q3|N|Y|N|2478025|2478206|2477688|2477963|N|N|N|N|N| +2478055|AAAAAAAAHOPMFCAA|2072-07-31|2070|9005|691|2072|0|7|31|3|2072|691|9005|Sunday|2072Q3|N|N|N|2478025|2478206|2477689|2477964|N|N|N|N|N| +2478056|AAAAAAAAIOPMFCAA|2072-08-01|2071|9005|691|2072|1|8|1|3|2072|691|9005|Monday|2072Q3|N|N|N|2478056|2478268|2477690|2477965|N|N|N|N|N| +2478057|AAAAAAAAJOPMFCAA|2072-08-02|2071|9006|691|2072|2|8|2|3|2072|691|9006|Tuesday|2072Q3|N|N|N|2478056|2478268|2477691|2477966|N|N|N|N|N| +2478058|AAAAAAAAKOPMFCAA|2072-08-03|2071|9006|691|2072|3|8|3|3|2072|691|9006|Wednesday|2072Q3|N|N|N|2478056|2478268|2477692|2477967|N|N|N|N|N| +2478059|AAAAAAAALOPMFCAA|2072-08-04|2071|9006|691|2072|4|8|4|3|2072|691|9006|Thursday|2072Q3|N|N|N|2478056|2478268|2477693|2477968|N|N|N|N|N| +2478060|AAAAAAAAMOPMFCAA|2072-08-05|2071|9006|691|2072|5|8|5|3|2072|691|9006|Friday|2072Q3|N|Y|N|2478056|2478268|2477694|2477969|N|N|N|N|N| +2478061|AAAAAAAANOPMFCAA|2072-08-06|2071|9006|691|2072|6|8|6|3|2072|691|9006|Saturday|2072Q3|N|Y|N|2478056|2478268|2477695|2477970|N|N|N|N|N| +2478062|AAAAAAAAOOPMFCAA|2072-08-07|2071|9006|691|2072|0|8|7|3|2072|691|9006|Sunday|2072Q3|N|N|N|2478056|2478268|2477696|2477971|N|N|N|N|N| +2478063|AAAAAAAAPOPMFCAA|2072-08-08|2071|9006|691|2072|1|8|8|3|2072|691|9006|Monday|2072Q3|N|N|N|2478056|2478268|2477697|2477972|N|N|N|N|N| +2478064|AAAAAAAAAPPMFCAA|2072-08-09|2071|9007|691|2072|2|8|9|3|2072|691|9007|Tuesday|2072Q3|N|N|N|2478056|2478268|2477698|2477973|N|N|N|N|N| +2478065|AAAAAAAABPPMFCAA|2072-08-10|2071|9007|691|2072|3|8|10|3|2072|691|9007|Wednesday|2072Q3|N|N|N|2478056|2478268|2477699|2477974|N|N|N|N|N| +2478066|AAAAAAAACPPMFCAA|2072-08-11|2071|9007|691|2072|4|8|11|3|2072|691|9007|Thursday|2072Q3|N|N|N|2478056|2478268|2477700|2477975|N|N|N|N|N| +2478067|AAAAAAAADPPMFCAA|2072-08-12|2071|9007|691|2072|5|8|12|3|2072|691|9007|Friday|2072Q3|N|Y|N|2478056|2478268|2477701|2477976|N|N|N|N|N| +2478068|AAAAAAAAEPPMFCAA|2072-08-13|2071|9007|691|2072|6|8|13|3|2072|691|9007|Saturday|2072Q3|N|Y|N|2478056|2478268|2477702|2477977|N|N|N|N|N| +2478069|AAAAAAAAFPPMFCAA|2072-08-14|2071|9007|691|2072|0|8|14|3|2072|691|9007|Sunday|2072Q3|N|N|N|2478056|2478268|2477703|2477978|N|N|N|N|N| +2478070|AAAAAAAAGPPMFCAA|2072-08-15|2071|9007|691|2072|1|8|15|3|2072|691|9007|Monday|2072Q3|N|N|N|2478056|2478268|2477704|2477979|N|N|N|N|N| +2478071|AAAAAAAAHPPMFCAA|2072-08-16|2071|9008|691|2072|2|8|16|3|2072|691|9008|Tuesday|2072Q3|N|N|N|2478056|2478268|2477705|2477980|N|N|N|N|N| +2478072|AAAAAAAAIPPMFCAA|2072-08-17|2071|9008|691|2072|3|8|17|3|2072|691|9008|Wednesday|2072Q3|N|N|N|2478056|2478268|2477706|2477981|N|N|N|N|N| +2478073|AAAAAAAAJPPMFCAA|2072-08-18|2071|9008|691|2072|4|8|18|3|2072|691|9008|Thursday|2072Q3|N|N|N|2478056|2478268|2477707|2477982|N|N|N|N|N| +2478074|AAAAAAAAKPPMFCAA|2072-08-19|2071|9008|691|2072|5|8|19|3|2072|691|9008|Friday|2072Q3|N|Y|N|2478056|2478268|2477708|2477983|N|N|N|N|N| +2478075|AAAAAAAALPPMFCAA|2072-08-20|2071|9008|691|2072|6|8|20|3|2072|691|9008|Saturday|2072Q3|N|Y|N|2478056|2478268|2477709|2477984|N|N|N|N|N| +2478076|AAAAAAAAMPPMFCAA|2072-08-21|2071|9008|691|2072|0|8|21|3|2072|691|9008|Sunday|2072Q3|N|N|N|2478056|2478268|2477710|2477985|N|N|N|N|N| +2478077|AAAAAAAANPPMFCAA|2072-08-22|2071|9008|691|2072|1|8|22|3|2072|691|9008|Monday|2072Q3|N|N|N|2478056|2478268|2477711|2477986|N|N|N|N|N| +2478078|AAAAAAAAOPPMFCAA|2072-08-23|2071|9009|691|2072|2|8|23|3|2072|691|9009|Tuesday|2072Q3|N|N|N|2478056|2478268|2477712|2477987|N|N|N|N|N| +2478079|AAAAAAAAPPPMFCAA|2072-08-24|2071|9009|691|2072|3|8|24|3|2072|691|9009|Wednesday|2072Q3|N|N|N|2478056|2478268|2477713|2477988|N|N|N|N|N| +2478080|AAAAAAAAAAANFCAA|2072-08-25|2071|9009|691|2072|4|8|25|3|2072|691|9009|Thursday|2072Q3|N|N|N|2478056|2478268|2477714|2477989|N|N|N|N|N| +2478081|AAAAAAAABAANFCAA|2072-08-26|2071|9009|691|2072|5|8|26|3|2072|691|9009|Friday|2072Q3|N|Y|N|2478056|2478268|2477715|2477990|N|N|N|N|N| +2478082|AAAAAAAACAANFCAA|2072-08-27|2071|9009|691|2072|6|8|27|3|2072|691|9009|Saturday|2072Q3|N|Y|N|2478056|2478268|2477716|2477991|N|N|N|N|N| +2478083|AAAAAAAADAANFCAA|2072-08-28|2071|9009|691|2072|0|8|28|3|2072|691|9009|Sunday|2072Q3|N|N|N|2478056|2478268|2477717|2477992|N|N|N|N|N| +2478084|AAAAAAAAEAANFCAA|2072-08-29|2071|9009|691|2072|1|8|29|3|2072|691|9009|Monday|2072Q3|N|N|N|2478056|2478268|2477718|2477993|N|N|N|N|N| +2478085|AAAAAAAAFAANFCAA|2072-08-30|2071|9010|691|2072|2|8|30|3|2072|691|9010|Tuesday|2072Q3|N|N|N|2478056|2478268|2477719|2477994|N|N|N|N|N| +2478086|AAAAAAAAGAANFCAA|2072-08-31|2071|9010|691|2072|3|8|31|3|2072|691|9010|Wednesday|2072Q3|N|N|N|2478056|2478268|2477720|2477995|N|N|N|N|N| +2478087|AAAAAAAAHAANFCAA|2072-09-01|2072|9010|692|2072|4|9|1|3|2072|692|9010|Thursday|2072Q3|N|N|N|2478087|2478330|2477721|2477996|N|N|N|N|N| +2478088|AAAAAAAAIAANFCAA|2072-09-02|2072|9010|692|2072|5|9|2|3|2072|692|9010|Friday|2072Q3|N|Y|N|2478087|2478330|2477722|2477997|N|N|N|N|N| +2478089|AAAAAAAAJAANFCAA|2072-09-03|2072|9010|692|2072|6|9|3|3|2072|692|9010|Saturday|2072Q3|N|Y|N|2478087|2478330|2477723|2477998|N|N|N|N|N| +2478090|AAAAAAAAKAANFCAA|2072-09-04|2072|9010|692|2072|0|9|4|3|2072|692|9010|Sunday|2072Q3|N|N|N|2478087|2478330|2477724|2477999|N|N|N|N|N| +2478091|AAAAAAAALAANFCAA|2072-09-05|2072|9010|692|2072|1|9|5|3|2072|692|9010|Monday|2072Q3|N|N|N|2478087|2478330|2477725|2478000|N|N|N|N|N| +2478092|AAAAAAAAMAANFCAA|2072-09-06|2072|9011|692|2072|2|9|6|3|2072|692|9011|Tuesday|2072Q3|N|N|N|2478087|2478330|2477726|2478001|N|N|N|N|N| +2478093|AAAAAAAANAANFCAA|2072-09-07|2072|9011|692|2072|3|9|7|3|2072|692|9011|Wednesday|2072Q3|N|N|N|2478087|2478330|2477727|2478002|N|N|N|N|N| +2478094|AAAAAAAAOAANFCAA|2072-09-08|2072|9011|692|2072|4|9|8|3|2072|692|9011|Thursday|2072Q3|N|N|N|2478087|2478330|2477728|2478003|N|N|N|N|N| +2478095|AAAAAAAAPAANFCAA|2072-09-09|2072|9011|692|2072|5|9|9|3|2072|692|9011|Friday|2072Q3|N|Y|N|2478087|2478330|2477729|2478004|N|N|N|N|N| +2478096|AAAAAAAAABANFCAA|2072-09-10|2072|9011|692|2072|6|9|10|3|2072|692|9011|Saturday|2072Q3|N|Y|N|2478087|2478330|2477730|2478005|N|N|N|N|N| +2478097|AAAAAAAABBANFCAA|2072-09-11|2072|9011|692|2072|0|9|11|3|2072|692|9011|Sunday|2072Q3|N|N|N|2478087|2478330|2477731|2478006|N|N|N|N|N| +2478098|AAAAAAAACBANFCAA|2072-09-12|2072|9011|692|2072|1|9|12|3|2072|692|9011|Monday|2072Q3|N|N|N|2478087|2478330|2477732|2478007|N|N|N|N|N| +2478099|AAAAAAAADBANFCAA|2072-09-13|2072|9012|692|2072|2|9|13|3|2072|692|9012|Tuesday|2072Q3|N|N|N|2478087|2478330|2477733|2478008|N|N|N|N|N| +2478100|AAAAAAAAEBANFCAA|2072-09-14|2072|9012|692|2072|3|9|14|3|2072|692|9012|Wednesday|2072Q3|N|N|N|2478087|2478330|2477734|2478009|N|N|N|N|N| +2478101|AAAAAAAAFBANFCAA|2072-09-15|2072|9012|692|2072|4|9|15|3|2072|692|9012|Thursday|2072Q3|N|N|N|2478087|2478330|2477735|2478010|N|N|N|N|N| +2478102|AAAAAAAAGBANFCAA|2072-09-16|2072|9012|692|2072|5|9|16|3|2072|692|9012|Friday|2072Q3|N|Y|N|2478087|2478330|2477736|2478011|N|N|N|N|N| +2478103|AAAAAAAAHBANFCAA|2072-09-17|2072|9012|692|2072|6|9|17|3|2072|692|9012|Saturday|2072Q3|N|Y|N|2478087|2478330|2477737|2478012|N|N|N|N|N| +2478104|AAAAAAAAIBANFCAA|2072-09-18|2072|9012|692|2072|0|9|18|3|2072|692|9012|Sunday|2072Q3|N|N|N|2478087|2478330|2477738|2478013|N|N|N|N|N| +2478105|AAAAAAAAJBANFCAA|2072-09-19|2072|9012|692|2072|1|9|19|3|2072|692|9012|Monday|2072Q3|N|N|N|2478087|2478330|2477739|2478014|N|N|N|N|N| +2478106|AAAAAAAAKBANFCAA|2072-09-20|2072|9013|692|2072|2|9|20|3|2072|692|9013|Tuesday|2072Q3|N|N|N|2478087|2478330|2477740|2478015|N|N|N|N|N| +2478107|AAAAAAAALBANFCAA|2072-09-21|2072|9013|692|2072|3|9|21|3|2072|692|9013|Wednesday|2072Q3|N|N|N|2478087|2478330|2477741|2478016|N|N|N|N|N| +2478108|AAAAAAAAMBANFCAA|2072-09-22|2072|9013|692|2072|4|9|22|3|2072|692|9013|Thursday|2072Q3|N|N|N|2478087|2478330|2477742|2478017|N|N|N|N|N| +2478109|AAAAAAAANBANFCAA|2072-09-23|2072|9013|692|2072|5|9|23|3|2072|692|9013|Friday|2072Q3|N|Y|N|2478087|2478330|2477743|2478018|N|N|N|N|N| +2478110|AAAAAAAAOBANFCAA|2072-09-24|2072|9013|692|2072|6|9|24|3|2072|692|9013|Saturday|2072Q3|N|Y|N|2478087|2478330|2477744|2478019|N|N|N|N|N| +2478111|AAAAAAAAPBANFCAA|2072-09-25|2072|9013|692|2072|0|9|25|3|2072|692|9013|Sunday|2072Q3|N|N|N|2478087|2478330|2477745|2478020|N|N|N|N|N| +2478112|AAAAAAAAACANFCAA|2072-09-26|2072|9013|692|2072|1|9|26|3|2072|692|9013|Monday|2072Q3|N|N|N|2478087|2478330|2477746|2478021|N|N|N|N|N| +2478113|AAAAAAAABCANFCAA|2072-09-27|2072|9014|692|2072|2|9|27|3|2072|692|9014|Tuesday|2072Q3|N|N|N|2478087|2478330|2477747|2478022|N|N|N|N|N| +2478114|AAAAAAAACCANFCAA|2072-09-28|2072|9014|692|2072|3|9|28|3|2072|692|9014|Wednesday|2072Q3|N|N|N|2478087|2478330|2477748|2478023|N|N|N|N|N| +2478115|AAAAAAAADCANFCAA|2072-09-29|2072|9014|692|2072|4|9|29|3|2072|692|9014|Thursday|2072Q3|N|N|N|2478087|2478330|2477749|2478024|N|N|N|N|N| +2478116|AAAAAAAAECANFCAA|2072-09-30|2072|9014|692|2072|5|9|30|3|2072|692|9014|Friday|2072Q3|N|Y|N|2478087|2478330|2477750|2478025|N|N|N|N|N| +2478117|AAAAAAAAFCANFCAA|2072-10-01|2073|9014|692|2072|6|10|1|4|2072|692|9014|Saturday|2072Q4|N|Y|N|2478117|2478390|2477751|2478025|N|N|N|N|N| +2478118|AAAAAAAAGCANFCAA|2072-10-02|2073|9014|692|2072|0|10|2|4|2072|692|9014|Sunday|2072Q4|N|N|N|2478117|2478390|2477752|2478026|N|N|N|N|N| +2478119|AAAAAAAAHCANFCAA|2072-10-03|2073|9014|692|2072|1|10|3|4|2072|692|9014|Monday|2072Q4|N|N|N|2478117|2478390|2477753|2478027|N|N|N|N|N| +2478120|AAAAAAAAICANFCAA|2072-10-04|2073|9015|692|2072|2|10|4|4|2072|692|9015|Tuesday|2072Q4|N|N|N|2478117|2478390|2477754|2478028|N|N|N|N|N| +2478121|AAAAAAAAJCANFCAA|2072-10-05|2073|9015|692|2072|3|10|5|4|2072|692|9015|Wednesday|2072Q4|N|N|N|2478117|2478390|2477755|2478029|N|N|N|N|N| +2478122|AAAAAAAAKCANFCAA|2072-10-06|2073|9015|692|2072|4|10|6|4|2072|692|9015|Thursday|2072Q4|N|N|N|2478117|2478390|2477756|2478030|N|N|N|N|N| +2478123|AAAAAAAALCANFCAA|2072-10-07|2073|9015|692|2072|5|10|7|4|2072|692|9015|Friday|2072Q4|N|Y|N|2478117|2478390|2477757|2478031|N|N|N|N|N| +2478124|AAAAAAAAMCANFCAA|2072-10-08|2073|9015|692|2072|6|10|8|4|2072|692|9015|Saturday|2072Q4|N|Y|N|2478117|2478390|2477758|2478032|N|N|N|N|N| +2478125|AAAAAAAANCANFCAA|2072-10-09|2073|9015|692|2072|0|10|9|4|2072|692|9015|Sunday|2072Q4|N|N|N|2478117|2478390|2477759|2478033|N|N|N|N|N| +2478126|AAAAAAAAOCANFCAA|2072-10-10|2073|9015|692|2072|1|10|10|4|2072|692|9015|Monday|2072Q4|N|N|N|2478117|2478390|2477760|2478034|N|N|N|N|N| +2478127|AAAAAAAAPCANFCAA|2072-10-11|2073|9016|692|2072|2|10|11|4|2072|692|9016|Tuesday|2072Q4|N|N|N|2478117|2478390|2477761|2478035|N|N|N|N|N| +2478128|AAAAAAAAADANFCAA|2072-10-12|2073|9016|692|2072|3|10|12|4|2072|692|9016|Wednesday|2072Q4|N|N|N|2478117|2478390|2477762|2478036|N|N|N|N|N| +2478129|AAAAAAAABDANFCAA|2072-10-13|2073|9016|692|2072|4|10|13|4|2072|692|9016|Thursday|2072Q4|N|N|N|2478117|2478390|2477763|2478037|N|N|N|N|N| +2478130|AAAAAAAACDANFCAA|2072-10-14|2073|9016|692|2072|5|10|14|4|2072|692|9016|Friday|2072Q4|N|Y|N|2478117|2478390|2477764|2478038|N|N|N|N|N| +2478131|AAAAAAAADDANFCAA|2072-10-15|2073|9016|692|2072|6|10|15|4|2072|692|9016|Saturday|2072Q4|N|Y|N|2478117|2478390|2477765|2478039|N|N|N|N|N| +2478132|AAAAAAAAEDANFCAA|2072-10-16|2073|9016|692|2072|0|10|16|4|2072|692|9016|Sunday|2072Q4|N|N|N|2478117|2478390|2477766|2478040|N|N|N|N|N| +2478133|AAAAAAAAFDANFCAA|2072-10-17|2073|9016|692|2072|1|10|17|4|2072|692|9016|Monday|2072Q4|N|N|N|2478117|2478390|2477767|2478041|N|N|N|N|N| +2478134|AAAAAAAAGDANFCAA|2072-10-18|2073|9017|692|2072|2|10|18|4|2072|692|9017|Tuesday|2072Q4|N|N|N|2478117|2478390|2477768|2478042|N|N|N|N|N| +2478135|AAAAAAAAHDANFCAA|2072-10-19|2073|9017|692|2072|3|10|19|4|2072|692|9017|Wednesday|2072Q4|N|N|N|2478117|2478390|2477769|2478043|N|N|N|N|N| +2478136|AAAAAAAAIDANFCAA|2072-10-20|2073|9017|692|2072|4|10|20|4|2072|692|9017|Thursday|2072Q4|N|N|N|2478117|2478390|2477770|2478044|N|N|N|N|N| +2478137|AAAAAAAAJDANFCAA|2072-10-21|2073|9017|692|2072|5|10|21|4|2072|692|9017|Friday|2072Q4|N|Y|N|2478117|2478390|2477771|2478045|N|N|N|N|N| +2478138|AAAAAAAAKDANFCAA|2072-10-22|2073|9017|692|2072|6|10|22|4|2072|692|9017|Saturday|2072Q4|N|Y|N|2478117|2478390|2477772|2478046|N|N|N|N|N| +2478139|AAAAAAAALDANFCAA|2072-10-23|2073|9017|692|2072|0|10|23|4|2072|692|9017|Sunday|2072Q4|N|N|N|2478117|2478390|2477773|2478047|N|N|N|N|N| +2478140|AAAAAAAAMDANFCAA|2072-10-24|2073|9017|692|2072|1|10|24|4|2072|692|9017|Monday|2072Q4|N|N|N|2478117|2478390|2477774|2478048|N|N|N|N|N| +2478141|AAAAAAAANDANFCAA|2072-10-25|2073|9018|692|2072|2|10|25|4|2072|692|9018|Tuesday|2072Q4|N|N|N|2478117|2478390|2477775|2478049|N|N|N|N|N| +2478142|AAAAAAAAODANFCAA|2072-10-26|2073|9018|692|2072|3|10|26|4|2072|692|9018|Wednesday|2072Q4|N|N|N|2478117|2478390|2477776|2478050|N|N|N|N|N| +2478143|AAAAAAAAPDANFCAA|2072-10-27|2073|9018|692|2072|4|10|27|4|2072|692|9018|Thursday|2072Q4|N|N|N|2478117|2478390|2477777|2478051|N|N|N|N|N| +2478144|AAAAAAAAAEANFCAA|2072-10-28|2073|9018|692|2072|5|10|28|4|2072|692|9018|Friday|2072Q4|N|Y|N|2478117|2478390|2477778|2478052|N|N|N|N|N| +2478145|AAAAAAAABEANFCAA|2072-10-29|2073|9018|692|2072|6|10|29|4|2072|692|9018|Saturday|2072Q4|N|Y|N|2478117|2478390|2477779|2478053|N|N|N|N|N| +2478146|AAAAAAAACEANFCAA|2072-10-30|2073|9018|692|2072|0|10|30|4|2072|692|9018|Sunday|2072Q4|N|N|N|2478117|2478390|2477780|2478054|N|N|N|N|N| +2478147|AAAAAAAADEANFCAA|2072-10-31|2073|9018|692|2072|1|10|31|4|2072|692|9018|Monday|2072Q4|N|N|N|2478117|2478390|2477781|2478055|N|N|N|N|N| +2478148|AAAAAAAAEEANFCAA|2072-11-01|2074|9019|692|2072|2|11|1|4|2072|692|9019|Tuesday|2072Q4|N|N|N|2478148|2478452|2477782|2478056|N|N|N|N|N| +2478149|AAAAAAAAFEANFCAA|2072-11-02|2074|9019|692|2072|3|11|2|4|2072|692|9019|Wednesday|2072Q4|N|N|N|2478148|2478452|2477783|2478057|N|N|N|N|N| +2478150|AAAAAAAAGEANFCAA|2072-11-03|2074|9019|692|2072|4|11|3|4|2072|692|9019|Thursday|2072Q4|N|N|N|2478148|2478452|2477784|2478058|N|N|N|N|N| +2478151|AAAAAAAAHEANFCAA|2072-11-04|2074|9019|692|2072|5|11|4|4|2072|692|9019|Friday|2072Q4|N|Y|N|2478148|2478452|2477785|2478059|N|N|N|N|N| +2478152|AAAAAAAAIEANFCAA|2072-11-05|2074|9019|692|2072|6|11|5|4|2072|692|9019|Saturday|2072Q4|N|Y|N|2478148|2478452|2477786|2478060|N|N|N|N|N| +2478153|AAAAAAAAJEANFCAA|2072-11-06|2074|9019|692|2072|0|11|6|4|2072|692|9019|Sunday|2072Q4|N|N|N|2478148|2478452|2477787|2478061|N|N|N|N|N| +2478154|AAAAAAAAKEANFCAA|2072-11-07|2074|9019|692|2072|1|11|7|4|2072|692|9019|Monday|2072Q4|N|N|N|2478148|2478452|2477788|2478062|N|N|N|N|N| +2478155|AAAAAAAALEANFCAA|2072-11-08|2074|9020|692|2072|2|11|8|4|2072|692|9020|Tuesday|2072Q4|N|N|N|2478148|2478452|2477789|2478063|N|N|N|N|N| +2478156|AAAAAAAAMEANFCAA|2072-11-09|2074|9020|692|2072|3|11|9|4|2072|692|9020|Wednesday|2072Q4|N|N|N|2478148|2478452|2477790|2478064|N|N|N|N|N| +2478157|AAAAAAAANEANFCAA|2072-11-10|2074|9020|692|2072|4|11|10|4|2072|692|9020|Thursday|2072Q4|N|N|N|2478148|2478452|2477791|2478065|N|N|N|N|N| +2478158|AAAAAAAAOEANFCAA|2072-11-11|2074|9020|692|2072|5|11|11|4|2072|692|9020|Friday|2072Q4|N|Y|N|2478148|2478452|2477792|2478066|N|N|N|N|N| +2478159|AAAAAAAAPEANFCAA|2072-11-12|2074|9020|692|2072|6|11|12|4|2072|692|9020|Saturday|2072Q4|N|Y|N|2478148|2478452|2477793|2478067|N|N|N|N|N| +2478160|AAAAAAAAAFANFCAA|2072-11-13|2074|9020|692|2072|0|11|13|4|2072|692|9020|Sunday|2072Q4|N|N|N|2478148|2478452|2477794|2478068|N|N|N|N|N| +2478161|AAAAAAAABFANFCAA|2072-11-14|2074|9020|692|2072|1|11|14|4|2072|692|9020|Monday|2072Q4|N|N|N|2478148|2478452|2477795|2478069|N|N|N|N|N| +2478162|AAAAAAAACFANFCAA|2072-11-15|2074|9021|692|2072|2|11|15|4|2072|692|9021|Tuesday|2072Q4|N|N|N|2478148|2478452|2477796|2478070|N|N|N|N|N| +2478163|AAAAAAAADFANFCAA|2072-11-16|2074|9021|692|2072|3|11|16|4|2072|692|9021|Wednesday|2072Q4|N|N|N|2478148|2478452|2477797|2478071|N|N|N|N|N| +2478164|AAAAAAAAEFANFCAA|2072-11-17|2074|9021|692|2072|4|11|17|4|2072|692|9021|Thursday|2072Q4|N|N|N|2478148|2478452|2477798|2478072|N|N|N|N|N| +2478165|AAAAAAAAFFANFCAA|2072-11-18|2074|9021|692|2072|5|11|18|4|2072|692|9021|Friday|2072Q4|N|Y|N|2478148|2478452|2477799|2478073|N|N|N|N|N| +2478166|AAAAAAAAGFANFCAA|2072-11-19|2074|9021|692|2072|6|11|19|4|2072|692|9021|Saturday|2072Q4|N|Y|N|2478148|2478452|2477800|2478074|N|N|N|N|N| +2478167|AAAAAAAAHFANFCAA|2072-11-20|2074|9021|692|2072|0|11|20|4|2072|692|9021|Sunday|2072Q4|N|N|N|2478148|2478452|2477801|2478075|N|N|N|N|N| +2478168|AAAAAAAAIFANFCAA|2072-11-21|2074|9021|692|2072|1|11|21|4|2072|692|9021|Monday|2072Q4|N|N|N|2478148|2478452|2477802|2478076|N|N|N|N|N| +2478169|AAAAAAAAJFANFCAA|2072-11-22|2074|9022|692|2072|2|11|22|4|2072|692|9022|Tuesday|2072Q4|N|N|N|2478148|2478452|2477803|2478077|N|N|N|N|N| +2478170|AAAAAAAAKFANFCAA|2072-11-23|2074|9022|692|2072|3|11|23|4|2072|692|9022|Wednesday|2072Q4|N|N|N|2478148|2478452|2477804|2478078|N|N|N|N|N| +2478171|AAAAAAAALFANFCAA|2072-11-24|2074|9022|692|2072|4|11|24|4|2072|692|9022|Thursday|2072Q4|N|N|N|2478148|2478452|2477805|2478079|N|N|N|N|N| +2478172|AAAAAAAAMFANFCAA|2072-11-25|2074|9022|692|2072|5|11|25|4|2072|692|9022|Friday|2072Q4|N|Y|N|2478148|2478452|2477806|2478080|N|N|N|N|N| +2478173|AAAAAAAANFANFCAA|2072-11-26|2074|9022|692|2072|6|11|26|4|2072|692|9022|Saturday|2072Q4|N|Y|N|2478148|2478452|2477807|2478081|N|N|N|N|N| +2478174|AAAAAAAAOFANFCAA|2072-11-27|2074|9022|692|2072|0|11|27|4|2072|692|9022|Sunday|2072Q4|N|N|N|2478148|2478452|2477808|2478082|N|N|N|N|N| +2478175|AAAAAAAAPFANFCAA|2072-11-28|2074|9022|692|2072|1|11|28|4|2072|692|9022|Monday|2072Q4|N|N|N|2478148|2478452|2477809|2478083|N|N|N|N|N| +2478176|AAAAAAAAAGANFCAA|2072-11-29|2074|9023|692|2072|2|11|29|4|2072|692|9023|Tuesday|2072Q4|N|N|N|2478148|2478452|2477810|2478084|N|N|N|N|N| +2478177|AAAAAAAABGANFCAA|2072-11-30|2074|9023|692|2072|3|11|30|4|2072|692|9023|Wednesday|2072Q4|N|N|N|2478148|2478452|2477811|2478085|N|N|N|N|N| +2478178|AAAAAAAACGANFCAA|2072-12-01|2075|9023|693|2072|4|12|1|4|2072|693|9023|Thursday|2072Q4|N|N|N|2478178|2478512|2477812|2478086|N|N|N|N|N| +2478179|AAAAAAAADGANFCAA|2072-12-02|2075|9023|693|2072|5|12|2|4|2072|693|9023|Friday|2072Q4|N|Y|N|2478178|2478512|2477813|2478087|N|N|N|N|N| +2478180|AAAAAAAAEGANFCAA|2072-12-03|2075|9023|693|2072|6|12|3|4|2072|693|9023|Saturday|2072Q4|N|Y|N|2478178|2478512|2477814|2478088|N|N|N|N|N| +2478181|AAAAAAAAFGANFCAA|2072-12-04|2075|9023|693|2072|0|12|4|4|2072|693|9023|Sunday|2072Q4|N|N|N|2478178|2478512|2477815|2478089|N|N|N|N|N| +2478182|AAAAAAAAGGANFCAA|2072-12-05|2075|9023|693|2072|1|12|5|4|2072|693|9023|Monday|2072Q4|N|N|N|2478178|2478512|2477816|2478090|N|N|N|N|N| +2478183|AAAAAAAAHGANFCAA|2072-12-06|2075|9024|693|2072|2|12|6|4|2072|693|9024|Tuesday|2072Q4|N|N|N|2478178|2478512|2477817|2478091|N|N|N|N|N| +2478184|AAAAAAAAIGANFCAA|2072-12-07|2075|9024|693|2072|3|12|7|4|2072|693|9024|Wednesday|2072Q4|N|N|N|2478178|2478512|2477818|2478092|N|N|N|N|N| +2478185|AAAAAAAAJGANFCAA|2072-12-08|2075|9024|693|2072|4|12|8|4|2072|693|9024|Thursday|2072Q4|N|N|N|2478178|2478512|2477819|2478093|N|N|N|N|N| +2478186|AAAAAAAAKGANFCAA|2072-12-09|2075|9024|693|2072|5|12|9|4|2072|693|9024|Friday|2072Q4|N|Y|N|2478178|2478512|2477820|2478094|N|N|N|N|N| +2478187|AAAAAAAALGANFCAA|2072-12-10|2075|9024|693|2072|6|12|10|4|2072|693|9024|Saturday|2072Q4|N|Y|N|2478178|2478512|2477821|2478095|N|N|N|N|N| +2478188|AAAAAAAAMGANFCAA|2072-12-11|2075|9024|693|2072|0|12|11|4|2072|693|9024|Sunday|2072Q4|N|N|N|2478178|2478512|2477822|2478096|N|N|N|N|N| +2478189|AAAAAAAANGANFCAA|2072-12-12|2075|9024|693|2072|1|12|12|4|2072|693|9024|Monday|2072Q4|N|N|N|2478178|2478512|2477823|2478097|N|N|N|N|N| +2478190|AAAAAAAAOGANFCAA|2072-12-13|2075|9025|693|2072|2|12|13|4|2072|693|9025|Tuesday|2072Q4|N|N|N|2478178|2478512|2477824|2478098|N|N|N|N|N| +2478191|AAAAAAAAPGANFCAA|2072-12-14|2075|9025|693|2072|3|12|14|4|2072|693|9025|Wednesday|2072Q4|N|N|N|2478178|2478512|2477825|2478099|N|N|N|N|N| +2478192|AAAAAAAAAHANFCAA|2072-12-15|2075|9025|693|2072|4|12|15|4|2072|693|9025|Thursday|2072Q4|N|N|N|2478178|2478512|2477826|2478100|N|N|N|N|N| +2478193|AAAAAAAABHANFCAA|2072-12-16|2075|9025|693|2072|5|12|16|4|2072|693|9025|Friday|2072Q4|N|Y|N|2478178|2478512|2477827|2478101|N|N|N|N|N| +2478194|AAAAAAAACHANFCAA|2072-12-17|2075|9025|693|2072|6|12|17|4|2072|693|9025|Saturday|2072Q4|N|Y|N|2478178|2478512|2477828|2478102|N|N|N|N|N| +2478195|AAAAAAAADHANFCAA|2072-12-18|2075|9025|693|2072|0|12|18|4|2072|693|9025|Sunday|2072Q4|N|N|N|2478178|2478512|2477829|2478103|N|N|N|N|N| +2478196|AAAAAAAAEHANFCAA|2072-12-19|2075|9025|693|2072|1|12|19|4|2072|693|9025|Monday|2072Q4|N|N|N|2478178|2478512|2477830|2478104|N|N|N|N|N| +2478197|AAAAAAAAFHANFCAA|2072-12-20|2075|9026|693|2072|2|12|20|4|2072|693|9026|Tuesday|2072Q4|N|N|N|2478178|2478512|2477831|2478105|N|N|N|N|N| +2478198|AAAAAAAAGHANFCAA|2072-12-21|2075|9026|693|2072|3|12|21|4|2072|693|9026|Wednesday|2072Q4|N|N|N|2478178|2478512|2477832|2478106|N|N|N|N|N| +2478199|AAAAAAAAHHANFCAA|2072-12-22|2075|9026|693|2072|4|12|22|4|2072|693|9026|Thursday|2072Q4|N|N|N|2478178|2478512|2477833|2478107|N|N|N|N|N| +2478200|AAAAAAAAIHANFCAA|2072-12-23|2075|9026|693|2072|5|12|23|4|2072|693|9026|Friday|2072Q4|N|Y|N|2478178|2478512|2477834|2478108|N|N|N|N|N| +2478201|AAAAAAAAJHANFCAA|2072-12-24|2075|9026|693|2072|6|12|24|4|2072|693|9026|Saturday|2072Q4|N|Y|N|2478178|2478512|2477835|2478109|N|N|N|N|N| +2478202|AAAAAAAAKHANFCAA|2072-12-25|2075|9026|693|2072|0|12|25|4|2072|693|9026|Sunday|2072Q4|Y|N|N|2478178|2478512|2477836|2478110|N|N|N|N|N| +2478203|AAAAAAAALHANFCAA|2072-12-26|2075|9026|693|2072|1|12|26|4|2072|693|9026|Monday|2072Q4|N|N|Y|2478178|2478512|2477837|2478111|N|N|N|N|N| +2478204|AAAAAAAAMHANFCAA|2072-12-27|2075|9027|693|2072|2|12|27|4|2072|693|9027|Tuesday|2072Q4|N|N|N|2478178|2478512|2477838|2478112|N|N|N|N|N| +2478205|AAAAAAAANHANFCAA|2072-12-28|2075|9027|693|2072|3|12|28|4|2072|693|9027|Wednesday|2072Q4|N|N|N|2478178|2478512|2477839|2478113|N|N|N|N|N| +2478206|AAAAAAAAOHANFCAA|2072-12-29|2075|9027|693|2072|4|12|29|4|2072|693|9027|Thursday|2072Q4|N|N|N|2478178|2478512|2477840|2478114|N|N|N|N|N| +2478207|AAAAAAAAPHANFCAA|2072-12-30|2075|9027|693|2072|5|12|30|4|2072|693|9027|Friday|2072Q4|N|Y|N|2478178|2478512|2477841|2478115|N|N|N|N|N| +2478208|AAAAAAAAAIANFCAA|2072-12-31|2075|9027|693|2072|6|12|31|4|2072|693|9027|Saturday|2072Q4|Y|Y|N|2478178|2478512|2477842|2478116|N|N|N|N|N| +2478209|AAAAAAAABIANFCAA|2073-01-01|2076|9027|693|2073|0|1|1|1|2073|693|9027|Sunday|2073Q1|Y|N|Y|2478209|2478208|2477843|2478117|N|N|N|N|N| +2478210|AAAAAAAACIANFCAA|2073-01-02|2076|9027|693|2073|1|1|2|1|2073|693|9027|Monday|2073Q1|N|N|Y|2478209|2478208|2477844|2478118|N|N|N|N|N| +2478211|AAAAAAAADIANFCAA|2073-01-03|2076|9028|693|2073|2|1|3|1|2073|693|9028|Tuesday|2073Q1|N|N|N|2478209|2478208|2477845|2478119|N|N|N|N|N| +2478212|AAAAAAAAEIANFCAA|2073-01-04|2076|9028|693|2073|3|1|4|1|2073|693|9028|Wednesday|2073Q1|N|N|N|2478209|2478208|2477846|2478120|N|N|N|N|N| +2478213|AAAAAAAAFIANFCAA|2073-01-05|2076|9028|693|2073|4|1|5|1|2073|693|9028|Thursday|2073Q1|N|N|N|2478209|2478208|2477847|2478121|N|N|N|N|N| +2478214|AAAAAAAAGIANFCAA|2073-01-06|2076|9028|693|2073|5|1|6|1|2073|693|9028|Friday|2073Q1|N|Y|N|2478209|2478208|2477848|2478122|N|N|N|N|N| +2478215|AAAAAAAAHIANFCAA|2073-01-07|2076|9028|693|2073|6|1|7|1|2073|693|9028|Saturday|2073Q1|N|Y|N|2478209|2478208|2477849|2478123|N|N|N|N|N| +2478216|AAAAAAAAIIANFCAA|2073-01-08|2076|9028|693|2073|0|1|8|1|2073|693|9028|Sunday|2073Q1|N|N|N|2478209|2478208|2477850|2478124|N|N|N|N|N| +2478217|AAAAAAAAJIANFCAA|2073-01-09|2076|9028|693|2073|1|1|9|1|2073|693|9028|Monday|2073Q1|N|N|N|2478209|2478208|2477851|2478125|N|N|N|N|N| +2478218|AAAAAAAAKIANFCAA|2073-01-10|2076|9029|693|2073|2|1|10|1|2073|693|9029|Tuesday|2073Q1|N|N|N|2478209|2478208|2477852|2478126|N|N|N|N|N| +2478219|AAAAAAAALIANFCAA|2073-01-11|2076|9029|693|2073|3|1|11|1|2073|693|9029|Wednesday|2073Q1|N|N|N|2478209|2478208|2477853|2478127|N|N|N|N|N| +2478220|AAAAAAAAMIANFCAA|2073-01-12|2076|9029|693|2073|4|1|12|1|2073|693|9029|Thursday|2073Q1|N|N|N|2478209|2478208|2477854|2478128|N|N|N|N|N| +2478221|AAAAAAAANIANFCAA|2073-01-13|2076|9029|693|2073|5|1|13|1|2073|693|9029|Friday|2073Q1|N|Y|N|2478209|2478208|2477855|2478129|N|N|N|N|N| +2478222|AAAAAAAAOIANFCAA|2073-01-14|2076|9029|693|2073|6|1|14|1|2073|693|9029|Saturday|2073Q1|N|Y|N|2478209|2478208|2477856|2478130|N|N|N|N|N| +2478223|AAAAAAAAPIANFCAA|2073-01-15|2076|9029|693|2073|0|1|15|1|2073|693|9029|Sunday|2073Q1|N|N|N|2478209|2478208|2477857|2478131|N|N|N|N|N| +2478224|AAAAAAAAAJANFCAA|2073-01-16|2076|9029|693|2073|1|1|16|1|2073|693|9029|Monday|2073Q1|N|N|N|2478209|2478208|2477858|2478132|N|N|N|N|N| +2478225|AAAAAAAABJANFCAA|2073-01-17|2076|9030|693|2073|2|1|17|1|2073|693|9030|Tuesday|2073Q1|N|N|N|2478209|2478208|2477859|2478133|N|N|N|N|N| +2478226|AAAAAAAACJANFCAA|2073-01-18|2076|9030|693|2073|3|1|18|1|2073|693|9030|Wednesday|2073Q1|N|N|N|2478209|2478208|2477860|2478134|N|N|N|N|N| +2478227|AAAAAAAADJANFCAA|2073-01-19|2076|9030|693|2073|4|1|19|1|2073|693|9030|Thursday|2073Q1|N|N|N|2478209|2478208|2477861|2478135|N|N|N|N|N| +2478228|AAAAAAAAEJANFCAA|2073-01-20|2076|9030|693|2073|5|1|20|1|2073|693|9030|Friday|2073Q1|N|Y|N|2478209|2478208|2477862|2478136|N|N|N|N|N| +2478229|AAAAAAAAFJANFCAA|2073-01-21|2076|9030|693|2073|6|1|21|1|2073|693|9030|Saturday|2073Q1|N|Y|N|2478209|2478208|2477863|2478137|N|N|N|N|N| +2478230|AAAAAAAAGJANFCAA|2073-01-22|2076|9030|693|2073|0|1|22|1|2073|693|9030|Sunday|2073Q1|N|N|N|2478209|2478208|2477864|2478138|N|N|N|N|N| +2478231|AAAAAAAAHJANFCAA|2073-01-23|2076|9030|693|2073|1|1|23|1|2073|693|9030|Monday|2073Q1|N|N|N|2478209|2478208|2477865|2478139|N|N|N|N|N| +2478232|AAAAAAAAIJANFCAA|2073-01-24|2076|9031|693|2073|2|1|24|1|2073|693|9031|Tuesday|2073Q1|N|N|N|2478209|2478208|2477866|2478140|N|N|N|N|N| +2478233|AAAAAAAAJJANFCAA|2073-01-25|2076|9031|693|2073|3|1|25|1|2073|693|9031|Wednesday|2073Q1|N|N|N|2478209|2478208|2477867|2478141|N|N|N|N|N| +2478234|AAAAAAAAKJANFCAA|2073-01-26|2076|9031|693|2073|4|1|26|1|2073|693|9031|Thursday|2073Q1|N|N|N|2478209|2478208|2477868|2478142|N|N|N|N|N| +2478235|AAAAAAAALJANFCAA|2073-01-27|2076|9031|693|2073|5|1|27|1|2073|693|9031|Friday|2073Q1|N|Y|N|2478209|2478208|2477869|2478143|N|N|N|N|N| +2478236|AAAAAAAAMJANFCAA|2073-01-28|2076|9031|693|2073|6|1|28|1|2073|693|9031|Saturday|2073Q1|N|Y|N|2478209|2478208|2477870|2478144|N|N|N|N|N| +2478237|AAAAAAAANJANFCAA|2073-01-29|2076|9031|693|2073|0|1|29|1|2073|693|9031|Sunday|2073Q1|N|N|N|2478209|2478208|2477871|2478145|N|N|N|N|N| +2478238|AAAAAAAAOJANFCAA|2073-01-30|2076|9031|693|2073|1|1|30|1|2073|693|9031|Monday|2073Q1|N|N|N|2478209|2478208|2477872|2478146|N|N|N|N|N| +2478239|AAAAAAAAPJANFCAA|2073-01-31|2076|9032|693|2073|2|1|31|1|2073|693|9032|Tuesday|2073Q1|N|N|N|2478209|2478208|2477873|2478147|N|N|N|N|N| +2478240|AAAAAAAAAKANFCAA|2073-02-01|2077|9032|693|2073|3|2|1|1|2073|693|9032|Wednesday|2073Q1|N|N|N|2478240|2478270|2477874|2478148|N|N|N|N|N| +2478241|AAAAAAAABKANFCAA|2073-02-02|2077|9032|693|2073|4|2|2|1|2073|693|9032|Thursday|2073Q1|N|N|N|2478240|2478270|2477875|2478149|N|N|N|N|N| +2478242|AAAAAAAACKANFCAA|2073-02-03|2077|9032|693|2073|5|2|3|1|2073|693|9032|Friday|2073Q1|N|Y|N|2478240|2478270|2477876|2478150|N|N|N|N|N| +2478243|AAAAAAAADKANFCAA|2073-02-04|2077|9032|693|2073|6|2|4|1|2073|693|9032|Saturday|2073Q1|N|Y|N|2478240|2478270|2477877|2478151|N|N|N|N|N| +2478244|AAAAAAAAEKANFCAA|2073-02-05|2077|9032|693|2073|0|2|5|1|2073|693|9032|Sunday|2073Q1|N|N|N|2478240|2478270|2477878|2478152|N|N|N|N|N| +2478245|AAAAAAAAFKANFCAA|2073-02-06|2077|9032|693|2073|1|2|6|1|2073|693|9032|Monday|2073Q1|N|N|N|2478240|2478270|2477879|2478153|N|N|N|N|N| +2478246|AAAAAAAAGKANFCAA|2073-02-07|2077|9033|693|2073|2|2|7|1|2073|693|9033|Tuesday|2073Q1|N|N|N|2478240|2478270|2477880|2478154|N|N|N|N|N| +2478247|AAAAAAAAHKANFCAA|2073-02-08|2077|9033|693|2073|3|2|8|1|2073|693|9033|Wednesday|2073Q1|N|N|N|2478240|2478270|2477881|2478155|N|N|N|N|N| +2478248|AAAAAAAAIKANFCAA|2073-02-09|2077|9033|693|2073|4|2|9|1|2073|693|9033|Thursday|2073Q1|N|N|N|2478240|2478270|2477882|2478156|N|N|N|N|N| +2478249|AAAAAAAAJKANFCAA|2073-02-10|2077|9033|693|2073|5|2|10|1|2073|693|9033|Friday|2073Q1|N|Y|N|2478240|2478270|2477883|2478157|N|N|N|N|N| +2478250|AAAAAAAAKKANFCAA|2073-02-11|2077|9033|693|2073|6|2|11|1|2073|693|9033|Saturday|2073Q1|N|Y|N|2478240|2478270|2477884|2478158|N|N|N|N|N| +2478251|AAAAAAAALKANFCAA|2073-02-12|2077|9033|693|2073|0|2|12|1|2073|693|9033|Sunday|2073Q1|N|N|N|2478240|2478270|2477885|2478159|N|N|N|N|N| +2478252|AAAAAAAAMKANFCAA|2073-02-13|2077|9033|693|2073|1|2|13|1|2073|693|9033|Monday|2073Q1|N|N|N|2478240|2478270|2477886|2478160|N|N|N|N|N| +2478253|AAAAAAAANKANFCAA|2073-02-14|2077|9034|693|2073|2|2|14|1|2073|693|9034|Tuesday|2073Q1|N|N|N|2478240|2478270|2477887|2478161|N|N|N|N|N| +2478254|AAAAAAAAOKANFCAA|2073-02-15|2077|9034|693|2073|3|2|15|1|2073|693|9034|Wednesday|2073Q1|N|N|N|2478240|2478270|2477888|2478162|N|N|N|N|N| +2478255|AAAAAAAAPKANFCAA|2073-02-16|2077|9034|693|2073|4|2|16|1|2073|693|9034|Thursday|2073Q1|N|N|N|2478240|2478270|2477889|2478163|N|N|N|N|N| +2478256|AAAAAAAAALANFCAA|2073-02-17|2077|9034|693|2073|5|2|17|1|2073|693|9034|Friday|2073Q1|N|Y|N|2478240|2478270|2477890|2478164|N|N|N|N|N| +2478257|AAAAAAAABLANFCAA|2073-02-18|2077|9034|693|2073|6|2|18|1|2073|693|9034|Saturday|2073Q1|N|Y|N|2478240|2478270|2477891|2478165|N|N|N|N|N| +2478258|AAAAAAAACLANFCAA|2073-02-19|2077|9034|693|2073|0|2|19|1|2073|693|9034|Sunday|2073Q1|N|N|N|2478240|2478270|2477892|2478166|N|N|N|N|N| +2478259|AAAAAAAADLANFCAA|2073-02-20|2077|9034|693|2073|1|2|20|1|2073|693|9034|Monday|2073Q1|N|N|N|2478240|2478270|2477893|2478167|N|N|N|N|N| +2478260|AAAAAAAAELANFCAA|2073-02-21|2077|9035|693|2073|2|2|21|1|2073|693|9035|Tuesday|2073Q1|N|N|N|2478240|2478270|2477894|2478168|N|N|N|N|N| +2478261|AAAAAAAAFLANFCAA|2073-02-22|2077|9035|693|2073|3|2|22|1|2073|693|9035|Wednesday|2073Q1|N|N|N|2478240|2478270|2477895|2478169|N|N|N|N|N| +2478262|AAAAAAAAGLANFCAA|2073-02-23|2077|9035|693|2073|4|2|23|1|2073|693|9035|Thursday|2073Q1|N|N|N|2478240|2478270|2477896|2478170|N|N|N|N|N| +2478263|AAAAAAAAHLANFCAA|2073-02-24|2077|9035|693|2073|5|2|24|1|2073|693|9035|Friday|2073Q1|N|Y|N|2478240|2478270|2477897|2478171|N|N|N|N|N| +2478264|AAAAAAAAILANFCAA|2073-02-25|2077|9035|693|2073|6|2|25|1|2073|693|9035|Saturday|2073Q1|N|Y|N|2478240|2478270|2477898|2478172|N|N|N|N|N| +2478265|AAAAAAAAJLANFCAA|2073-02-26|2077|9035|693|2073|0|2|26|1|2073|693|9035|Sunday|2073Q1|N|N|N|2478240|2478270|2477899|2478173|N|N|N|N|N| +2478266|AAAAAAAAKLANFCAA|2073-02-27|2077|9035|693|2073|1|2|27|1|2073|693|9035|Monday|2073Q1|N|N|N|2478240|2478270|2477900|2478174|N|N|N|N|N| +2478267|AAAAAAAALLANFCAA|2073-02-28|2077|9036|693|2073|2|2|28|1|2073|693|9036|Tuesday|2073Q1|N|N|N|2478240|2478270|2477901|2478175|N|N|N|N|N| +2478268|AAAAAAAAMLANFCAA|2073-03-01|2078|9036|694|2073|3|3|1|1|2073|694|9036|Wednesday|2073Q1|N|N|N|2478268|2478326|2477903|2478176|N|N|N|N|N| +2478269|AAAAAAAANLANFCAA|2073-03-02|2078|9036|694|2073|4|3|2|1|2073|694|9036|Thursday|2073Q1|N|N|N|2478268|2478326|2477904|2478177|N|N|N|N|N| +2478270|AAAAAAAAOLANFCAA|2073-03-03|2078|9036|694|2073|5|3|3|1|2073|694|9036|Friday|2073Q1|N|Y|N|2478268|2478326|2477905|2478178|N|N|N|N|N| +2478271|AAAAAAAAPLANFCAA|2073-03-04|2078|9036|694|2073|6|3|4|1|2073|694|9036|Saturday|2073Q1|N|Y|N|2478268|2478326|2477906|2478179|N|N|N|N|N| +2478272|AAAAAAAAAMANFCAA|2073-03-05|2078|9036|694|2073|0|3|5|1|2073|694|9036|Sunday|2073Q1|N|N|N|2478268|2478326|2477907|2478180|N|N|N|N|N| +2478273|AAAAAAAABMANFCAA|2073-03-06|2078|9036|694|2073|1|3|6|1|2073|694|9036|Monday|2073Q1|N|N|N|2478268|2478326|2477908|2478181|N|N|N|N|N| +2478274|AAAAAAAACMANFCAA|2073-03-07|2078|9037|694|2073|2|3|7|1|2073|694|9037|Tuesday|2073Q1|N|N|N|2478268|2478326|2477909|2478182|N|N|N|N|N| +2478275|AAAAAAAADMANFCAA|2073-03-08|2078|9037|694|2073|3|3|8|1|2073|694|9037|Wednesday|2073Q1|N|N|N|2478268|2478326|2477910|2478183|N|N|N|N|N| +2478276|AAAAAAAAEMANFCAA|2073-03-09|2078|9037|694|2073|4|3|9|1|2073|694|9037|Thursday|2073Q1|N|N|N|2478268|2478326|2477911|2478184|N|N|N|N|N| +2478277|AAAAAAAAFMANFCAA|2073-03-10|2078|9037|694|2073|5|3|10|1|2073|694|9037|Friday|2073Q1|N|Y|N|2478268|2478326|2477912|2478185|N|N|N|N|N| +2478278|AAAAAAAAGMANFCAA|2073-03-11|2078|9037|694|2073|6|3|11|1|2073|694|9037|Saturday|2073Q1|N|Y|N|2478268|2478326|2477913|2478186|N|N|N|N|N| +2478279|AAAAAAAAHMANFCAA|2073-03-12|2078|9037|694|2073|0|3|12|1|2073|694|9037|Sunday|2073Q1|N|N|N|2478268|2478326|2477914|2478187|N|N|N|N|N| +2478280|AAAAAAAAIMANFCAA|2073-03-13|2078|9037|694|2073|1|3|13|1|2073|694|9037|Monday|2073Q1|N|N|N|2478268|2478326|2477915|2478188|N|N|N|N|N| +2478281|AAAAAAAAJMANFCAA|2073-03-14|2078|9038|694|2073|2|3|14|1|2073|694|9038|Tuesday|2073Q1|N|N|N|2478268|2478326|2477916|2478189|N|N|N|N|N| +2478282|AAAAAAAAKMANFCAA|2073-03-15|2078|9038|694|2073|3|3|15|1|2073|694|9038|Wednesday|2073Q1|N|N|N|2478268|2478326|2477917|2478190|N|N|N|N|N| +2478283|AAAAAAAALMANFCAA|2073-03-16|2078|9038|694|2073|4|3|16|1|2073|694|9038|Thursday|2073Q1|N|N|N|2478268|2478326|2477918|2478191|N|N|N|N|N| +2478284|AAAAAAAAMMANFCAA|2073-03-17|2078|9038|694|2073|5|3|17|1|2073|694|9038|Friday|2073Q1|N|Y|N|2478268|2478326|2477919|2478192|N|N|N|N|N| +2478285|AAAAAAAANMANFCAA|2073-03-18|2078|9038|694|2073|6|3|18|1|2073|694|9038|Saturday|2073Q1|N|Y|N|2478268|2478326|2477920|2478193|N|N|N|N|N| +2478286|AAAAAAAAOMANFCAA|2073-03-19|2078|9038|694|2073|0|3|19|1|2073|694|9038|Sunday|2073Q1|N|N|N|2478268|2478326|2477921|2478194|N|N|N|N|N| +2478287|AAAAAAAAPMANFCAA|2073-03-20|2078|9038|694|2073|1|3|20|1|2073|694|9038|Monday|2073Q1|N|N|N|2478268|2478326|2477922|2478195|N|N|N|N|N| +2478288|AAAAAAAAANANFCAA|2073-03-21|2078|9039|694|2073|2|3|21|1|2073|694|9039|Tuesday|2073Q1|N|N|N|2478268|2478326|2477923|2478196|N|N|N|N|N| +2478289|AAAAAAAABNANFCAA|2073-03-22|2078|9039|694|2073|3|3|22|1|2073|694|9039|Wednesday|2073Q1|N|N|N|2478268|2478326|2477924|2478197|N|N|N|N|N| +2478290|AAAAAAAACNANFCAA|2073-03-23|2078|9039|694|2073|4|3|23|1|2073|694|9039|Thursday|2073Q1|N|N|N|2478268|2478326|2477925|2478198|N|N|N|N|N| +2478291|AAAAAAAADNANFCAA|2073-03-24|2078|9039|694|2073|5|3|24|1|2073|694|9039|Friday|2073Q1|N|Y|N|2478268|2478326|2477926|2478199|N|N|N|N|N| +2478292|AAAAAAAAENANFCAA|2073-03-25|2078|9039|694|2073|6|3|25|1|2073|694|9039|Saturday|2073Q1|N|Y|N|2478268|2478326|2477927|2478200|N|N|N|N|N| +2478293|AAAAAAAAFNANFCAA|2073-03-26|2078|9039|694|2073|0|3|26|1|2073|694|9039|Sunday|2073Q1|N|N|N|2478268|2478326|2477928|2478201|N|N|N|N|N| +2478294|AAAAAAAAGNANFCAA|2073-03-27|2078|9039|694|2073|1|3|27|1|2073|694|9039|Monday|2073Q1|N|N|N|2478268|2478326|2477929|2478202|N|N|N|N|N| +2478295|AAAAAAAAHNANFCAA|2073-03-28|2078|9040|694|2073|2|3|28|1|2073|694|9040|Tuesday|2073Q1|N|N|N|2478268|2478326|2477930|2478203|N|N|N|N|N| +2478296|AAAAAAAAINANFCAA|2073-03-29|2078|9040|694|2073|3|3|29|1|2073|694|9040|Wednesday|2073Q1|N|N|N|2478268|2478326|2477931|2478204|N|N|N|N|N| +2478297|AAAAAAAAJNANFCAA|2073-03-30|2078|9040|694|2073|4|3|30|1|2073|694|9040|Thursday|2073Q1|N|N|N|2478268|2478326|2477932|2478205|N|N|N|N|N| +2478298|AAAAAAAAKNANFCAA|2073-03-31|2078|9040|694|2073|5|3|31|1|2073|694|9040|Friday|2073Q1|N|Y|N|2478268|2478326|2477933|2478206|N|N|N|N|N| +2478299|AAAAAAAALNANFCAA|2073-04-01|2079|9040|694|2073|6|4|1|1|2073|694|9040|Saturday|2073Q1|N|Y|N|2478299|2478388|2477934|2478209|N|N|N|N|N| +2478300|AAAAAAAAMNANFCAA|2073-04-02|2079|9040|694|2073|0|4|2|2|2073|694|9040|Sunday|2073Q2|N|N|N|2478299|2478388|2477935|2478210|N|N|N|N|N| +2478301|AAAAAAAANNANFCAA|2073-04-03|2079|9040|694|2073|1|4|3|2|2073|694|9040|Monday|2073Q2|N|N|N|2478299|2478388|2477936|2478211|N|N|N|N|N| +2478302|AAAAAAAAONANFCAA|2073-04-04|2079|9041|694|2073|2|4|4|2|2073|694|9041|Tuesday|2073Q2|N|N|N|2478299|2478388|2477937|2478212|N|N|N|N|N| +2478303|AAAAAAAAPNANFCAA|2073-04-05|2079|9041|694|2073|3|4|5|2|2073|694|9041|Wednesday|2073Q2|N|N|N|2478299|2478388|2477938|2478213|N|N|N|N|N| +2478304|AAAAAAAAAOANFCAA|2073-04-06|2079|9041|694|2073|4|4|6|2|2073|694|9041|Thursday|2073Q2|N|N|N|2478299|2478388|2477939|2478214|N|N|N|N|N| +2478305|AAAAAAAABOANFCAA|2073-04-07|2079|9041|694|2073|5|4|7|2|2073|694|9041|Friday|2073Q2|N|Y|N|2478299|2478388|2477940|2478215|N|N|N|N|N| +2478306|AAAAAAAACOANFCAA|2073-04-08|2079|9041|694|2073|6|4|8|2|2073|694|9041|Saturday|2073Q2|N|Y|N|2478299|2478388|2477941|2478216|N|N|N|N|N| +2478307|AAAAAAAADOANFCAA|2073-04-09|2079|9041|694|2073|0|4|9|2|2073|694|9041|Sunday|2073Q2|N|N|N|2478299|2478388|2477942|2478217|N|N|N|N|N| +2478308|AAAAAAAAEOANFCAA|2073-04-10|2079|9041|694|2073|1|4|10|2|2073|694|9041|Monday|2073Q2|N|N|N|2478299|2478388|2477943|2478218|N|N|N|N|N| +2478309|AAAAAAAAFOANFCAA|2073-04-11|2079|9042|694|2073|2|4|11|2|2073|694|9042|Tuesday|2073Q2|N|N|N|2478299|2478388|2477944|2478219|N|N|N|N|N| +2478310|AAAAAAAAGOANFCAA|2073-04-12|2079|9042|694|2073|3|4|12|2|2073|694|9042|Wednesday|2073Q2|N|N|N|2478299|2478388|2477945|2478220|N|N|N|N|N| +2478311|AAAAAAAAHOANFCAA|2073-04-13|2079|9042|694|2073|4|4|13|2|2073|694|9042|Thursday|2073Q2|N|N|N|2478299|2478388|2477946|2478221|N|N|N|N|N| +2478312|AAAAAAAAIOANFCAA|2073-04-14|2079|9042|694|2073|5|4|14|2|2073|694|9042|Friday|2073Q2|N|Y|N|2478299|2478388|2477947|2478222|N|N|N|N|N| +2478313|AAAAAAAAJOANFCAA|2073-04-15|2079|9042|694|2073|6|4|15|2|2073|694|9042|Saturday|2073Q2|N|Y|N|2478299|2478388|2477948|2478223|N|N|N|N|N| +2478314|AAAAAAAAKOANFCAA|2073-04-16|2079|9042|694|2073|0|4|16|2|2073|694|9042|Sunday|2073Q2|N|N|N|2478299|2478388|2477949|2478224|N|N|N|N|N| +2478315|AAAAAAAALOANFCAA|2073-04-17|2079|9042|694|2073|1|4|17|2|2073|694|9042|Monday|2073Q2|N|N|N|2478299|2478388|2477950|2478225|N|N|N|N|N| +2478316|AAAAAAAAMOANFCAA|2073-04-18|2079|9043|694|2073|2|4|18|2|2073|694|9043|Tuesday|2073Q2|N|N|N|2478299|2478388|2477951|2478226|N|N|N|N|N| +2478317|AAAAAAAANOANFCAA|2073-04-19|2079|9043|694|2073|3|4|19|2|2073|694|9043|Wednesday|2073Q2|N|N|N|2478299|2478388|2477952|2478227|N|N|N|N|N| +2478318|AAAAAAAAOOANFCAA|2073-04-20|2079|9043|694|2073|4|4|20|2|2073|694|9043|Thursday|2073Q2|N|N|N|2478299|2478388|2477953|2478228|N|N|N|N|N| +2478319|AAAAAAAAPOANFCAA|2073-04-21|2079|9043|694|2073|5|4|21|2|2073|694|9043|Friday|2073Q2|N|Y|N|2478299|2478388|2477954|2478229|N|N|N|N|N| +2478320|AAAAAAAAAPANFCAA|2073-04-22|2079|9043|694|2073|6|4|22|2|2073|694|9043|Saturday|2073Q2|N|Y|N|2478299|2478388|2477955|2478230|N|N|N|N|N| +2478321|AAAAAAAABPANFCAA|2073-04-23|2079|9043|694|2073|0|4|23|2|2073|694|9043|Sunday|2073Q2|N|N|N|2478299|2478388|2477956|2478231|N|N|N|N|N| +2478322|AAAAAAAACPANFCAA|2073-04-24|2079|9043|694|2073|1|4|24|2|2073|694|9043|Monday|2073Q2|N|N|N|2478299|2478388|2477957|2478232|N|N|N|N|N| +2478323|AAAAAAAADPANFCAA|2073-04-25|2079|9044|694|2073|2|4|25|2|2073|694|9044|Tuesday|2073Q2|N|N|N|2478299|2478388|2477958|2478233|N|N|N|N|N| +2478324|AAAAAAAAEPANFCAA|2073-04-26|2079|9044|694|2073|3|4|26|2|2073|694|9044|Wednesday|2073Q2|N|N|N|2478299|2478388|2477959|2478234|N|N|N|N|N| +2478325|AAAAAAAAFPANFCAA|2073-04-27|2079|9044|694|2073|4|4|27|2|2073|694|9044|Thursday|2073Q2|N|N|N|2478299|2478388|2477960|2478235|N|N|N|N|N| +2478326|AAAAAAAAGPANFCAA|2073-04-28|2079|9044|694|2073|5|4|28|2|2073|694|9044|Friday|2073Q2|N|Y|N|2478299|2478388|2477961|2478236|N|N|N|N|N| +2478327|AAAAAAAAHPANFCAA|2073-04-29|2079|9044|694|2073|6|4|29|2|2073|694|9044|Saturday|2073Q2|N|Y|N|2478299|2478388|2477962|2478237|N|N|N|N|N| +2478328|AAAAAAAAIPANFCAA|2073-04-30|2079|9044|694|2073|0|4|30|2|2073|694|9044|Sunday|2073Q2|N|N|N|2478299|2478388|2477963|2478238|N|N|N|N|N| +2478329|AAAAAAAAJPANFCAA|2073-05-01|2080|9044|694|2073|1|5|1|2|2073|694|9044|Monday|2073Q2|N|N|N|2478329|2478448|2477964|2478239|N|N|N|N|N| +2478330|AAAAAAAAKPANFCAA|2073-05-02|2080|9045|694|2073|2|5|2|2|2073|694|9045|Tuesday|2073Q2|N|N|N|2478329|2478448|2477965|2478240|N|N|N|N|N| +2478331|AAAAAAAALPANFCAA|2073-05-03|2080|9045|694|2073|3|5|3|2|2073|694|9045|Wednesday|2073Q2|N|N|N|2478329|2478448|2477966|2478241|N|N|N|N|N| +2478332|AAAAAAAAMPANFCAA|2073-05-04|2080|9045|694|2073|4|5|4|2|2073|694|9045|Thursday|2073Q2|N|N|N|2478329|2478448|2477967|2478242|N|N|N|N|N| +2478333|AAAAAAAANPANFCAA|2073-05-05|2080|9045|694|2073|5|5|5|2|2073|694|9045|Friday|2073Q2|N|Y|N|2478329|2478448|2477968|2478243|N|N|N|N|N| +2478334|AAAAAAAAOPANFCAA|2073-05-06|2080|9045|694|2073|6|5|6|2|2073|694|9045|Saturday|2073Q2|N|Y|N|2478329|2478448|2477969|2478244|N|N|N|N|N| +2478335|AAAAAAAAPPANFCAA|2073-05-07|2080|9045|694|2073|0|5|7|2|2073|694|9045|Sunday|2073Q2|N|N|N|2478329|2478448|2477970|2478245|N|N|N|N|N| +2478336|AAAAAAAAAABNFCAA|2073-05-08|2080|9045|694|2073|1|5|8|2|2073|694|9045|Monday|2073Q2|N|N|N|2478329|2478448|2477971|2478246|N|N|N|N|N| +2478337|AAAAAAAABABNFCAA|2073-05-09|2080|9046|694|2073|2|5|9|2|2073|694|9046|Tuesday|2073Q2|N|N|N|2478329|2478448|2477972|2478247|N|N|N|N|N| +2478338|AAAAAAAACABNFCAA|2073-05-10|2080|9046|694|2073|3|5|10|2|2073|694|9046|Wednesday|2073Q2|N|N|N|2478329|2478448|2477973|2478248|N|N|N|N|N| +2478339|AAAAAAAADABNFCAA|2073-05-11|2080|9046|694|2073|4|5|11|2|2073|694|9046|Thursday|2073Q2|N|N|N|2478329|2478448|2477974|2478249|N|N|N|N|N| +2478340|AAAAAAAAEABNFCAA|2073-05-12|2080|9046|694|2073|5|5|12|2|2073|694|9046|Friday|2073Q2|N|Y|N|2478329|2478448|2477975|2478250|N|N|N|N|N| +2478341|AAAAAAAAFABNFCAA|2073-05-13|2080|9046|694|2073|6|5|13|2|2073|694|9046|Saturday|2073Q2|N|Y|N|2478329|2478448|2477976|2478251|N|N|N|N|N| +2478342|AAAAAAAAGABNFCAA|2073-05-14|2080|9046|694|2073|0|5|14|2|2073|694|9046|Sunday|2073Q2|N|N|N|2478329|2478448|2477977|2478252|N|N|N|N|N| +2478343|AAAAAAAAHABNFCAA|2073-05-15|2080|9046|694|2073|1|5|15|2|2073|694|9046|Monday|2073Q2|N|N|N|2478329|2478448|2477978|2478253|N|N|N|N|N| +2478344|AAAAAAAAIABNFCAA|2073-05-16|2080|9047|694|2073|2|5|16|2|2073|694|9047|Tuesday|2073Q2|N|N|N|2478329|2478448|2477979|2478254|N|N|N|N|N| +2478345|AAAAAAAAJABNFCAA|2073-05-17|2080|9047|694|2073|3|5|17|2|2073|694|9047|Wednesday|2073Q2|N|N|N|2478329|2478448|2477980|2478255|N|N|N|N|N| +2478346|AAAAAAAAKABNFCAA|2073-05-18|2080|9047|694|2073|4|5|18|2|2073|694|9047|Thursday|2073Q2|N|N|N|2478329|2478448|2477981|2478256|N|N|N|N|N| +2478347|AAAAAAAALABNFCAA|2073-05-19|2080|9047|694|2073|5|5|19|2|2073|694|9047|Friday|2073Q2|N|Y|N|2478329|2478448|2477982|2478257|N|N|N|N|N| +2478348|AAAAAAAAMABNFCAA|2073-05-20|2080|9047|694|2073|6|5|20|2|2073|694|9047|Saturday|2073Q2|N|Y|N|2478329|2478448|2477983|2478258|N|N|N|N|N| +2478349|AAAAAAAANABNFCAA|2073-05-21|2080|9047|694|2073|0|5|21|2|2073|694|9047|Sunday|2073Q2|N|N|N|2478329|2478448|2477984|2478259|N|N|N|N|N| +2478350|AAAAAAAAOABNFCAA|2073-05-22|2080|9047|694|2073|1|5|22|2|2073|694|9047|Monday|2073Q2|N|N|N|2478329|2478448|2477985|2478260|N|N|N|N|N| +2478351|AAAAAAAAPABNFCAA|2073-05-23|2080|9048|694|2073|2|5|23|2|2073|694|9048|Tuesday|2073Q2|N|N|N|2478329|2478448|2477986|2478261|N|N|N|N|N| +2478352|AAAAAAAAABBNFCAA|2073-05-24|2080|9048|694|2073|3|5|24|2|2073|694|9048|Wednesday|2073Q2|N|N|N|2478329|2478448|2477987|2478262|N|N|N|N|N| +2478353|AAAAAAAABBBNFCAA|2073-05-25|2080|9048|694|2073|4|5|25|2|2073|694|9048|Thursday|2073Q2|N|N|N|2478329|2478448|2477988|2478263|N|N|N|N|N| +2478354|AAAAAAAACBBNFCAA|2073-05-26|2080|9048|694|2073|5|5|26|2|2073|694|9048|Friday|2073Q2|N|Y|N|2478329|2478448|2477989|2478264|N|N|N|N|N| +2478355|AAAAAAAADBBNFCAA|2073-05-27|2080|9048|694|2073|6|5|27|2|2073|694|9048|Saturday|2073Q2|N|Y|N|2478329|2478448|2477990|2478265|N|N|N|N|N| +2478356|AAAAAAAAEBBNFCAA|2073-05-28|2080|9048|694|2073|0|5|28|2|2073|694|9048|Sunday|2073Q2|N|N|N|2478329|2478448|2477991|2478266|N|N|N|N|N| +2478357|AAAAAAAAFBBNFCAA|2073-05-29|2080|9048|694|2073|1|5|29|2|2073|694|9048|Monday|2073Q2|N|N|N|2478329|2478448|2477992|2478267|N|N|N|N|N| +2478358|AAAAAAAAGBBNFCAA|2073-05-30|2080|9049|694|2073|2|5|30|2|2073|694|9049|Tuesday|2073Q2|N|N|N|2478329|2478448|2477993|2478268|N|N|N|N|N| +2478359|AAAAAAAAHBBNFCAA|2073-05-31|2080|9049|694|2073|3|5|31|2|2073|694|9049|Wednesday|2073Q2|N|N|N|2478329|2478448|2477994|2478269|N|N|N|N|N| +2478360|AAAAAAAAIBBNFCAA|2073-06-01|2081|9049|695|2073|4|6|1|2|2073|695|9049|Thursday|2073Q2|N|N|N|2478360|2478510|2477995|2478270|N|N|N|N|N| +2478361|AAAAAAAAJBBNFCAA|2073-06-02|2081|9049|695|2073|5|6|2|2|2073|695|9049|Friday|2073Q2|N|Y|N|2478360|2478510|2477996|2478271|N|N|N|N|N| +2478362|AAAAAAAAKBBNFCAA|2073-06-03|2081|9049|695|2073|6|6|3|2|2073|695|9049|Saturday|2073Q2|N|Y|N|2478360|2478510|2477997|2478272|N|N|N|N|N| +2478363|AAAAAAAALBBNFCAA|2073-06-04|2081|9049|695|2073|0|6|4|2|2073|695|9049|Sunday|2073Q2|N|N|N|2478360|2478510|2477998|2478273|N|N|N|N|N| +2478364|AAAAAAAAMBBNFCAA|2073-06-05|2081|9049|695|2073|1|6|5|2|2073|695|9049|Monday|2073Q2|N|N|N|2478360|2478510|2477999|2478274|N|N|N|N|N| +2478365|AAAAAAAANBBNFCAA|2073-06-06|2081|9050|695|2073|2|6|6|2|2073|695|9050|Tuesday|2073Q2|N|N|N|2478360|2478510|2478000|2478275|N|N|N|N|N| +2478366|AAAAAAAAOBBNFCAA|2073-06-07|2081|9050|695|2073|3|6|7|2|2073|695|9050|Wednesday|2073Q2|N|N|N|2478360|2478510|2478001|2478276|N|N|N|N|N| +2478367|AAAAAAAAPBBNFCAA|2073-06-08|2081|9050|695|2073|4|6|8|2|2073|695|9050|Thursday|2073Q2|N|N|N|2478360|2478510|2478002|2478277|N|N|N|N|N| +2478368|AAAAAAAAACBNFCAA|2073-06-09|2081|9050|695|2073|5|6|9|2|2073|695|9050|Friday|2073Q2|N|Y|N|2478360|2478510|2478003|2478278|N|N|N|N|N| +2478369|AAAAAAAABCBNFCAA|2073-06-10|2081|9050|695|2073|6|6|10|2|2073|695|9050|Saturday|2073Q2|N|Y|N|2478360|2478510|2478004|2478279|N|N|N|N|N| +2478370|AAAAAAAACCBNFCAA|2073-06-11|2081|9050|695|2073|0|6|11|2|2073|695|9050|Sunday|2073Q2|N|N|N|2478360|2478510|2478005|2478280|N|N|N|N|N| +2478371|AAAAAAAADCBNFCAA|2073-06-12|2081|9050|695|2073|1|6|12|2|2073|695|9050|Monday|2073Q2|N|N|N|2478360|2478510|2478006|2478281|N|N|N|N|N| +2478372|AAAAAAAAECBNFCAA|2073-06-13|2081|9051|695|2073|2|6|13|2|2073|695|9051|Tuesday|2073Q2|N|N|N|2478360|2478510|2478007|2478282|N|N|N|N|N| +2478373|AAAAAAAAFCBNFCAA|2073-06-14|2081|9051|695|2073|3|6|14|2|2073|695|9051|Wednesday|2073Q2|N|N|N|2478360|2478510|2478008|2478283|N|N|N|N|N| +2478374|AAAAAAAAGCBNFCAA|2073-06-15|2081|9051|695|2073|4|6|15|2|2073|695|9051|Thursday|2073Q2|N|N|N|2478360|2478510|2478009|2478284|N|N|N|N|N| +2478375|AAAAAAAAHCBNFCAA|2073-06-16|2081|9051|695|2073|5|6|16|2|2073|695|9051|Friday|2073Q2|N|Y|N|2478360|2478510|2478010|2478285|N|N|N|N|N| +2478376|AAAAAAAAICBNFCAA|2073-06-17|2081|9051|695|2073|6|6|17|2|2073|695|9051|Saturday|2073Q2|N|Y|N|2478360|2478510|2478011|2478286|N|N|N|N|N| +2478377|AAAAAAAAJCBNFCAA|2073-06-18|2081|9051|695|2073|0|6|18|2|2073|695|9051|Sunday|2073Q2|N|N|N|2478360|2478510|2478012|2478287|N|N|N|N|N| +2478378|AAAAAAAAKCBNFCAA|2073-06-19|2081|9051|695|2073|1|6|19|2|2073|695|9051|Monday|2073Q2|N|N|N|2478360|2478510|2478013|2478288|N|N|N|N|N| +2478379|AAAAAAAALCBNFCAA|2073-06-20|2081|9052|695|2073|2|6|20|2|2073|695|9052|Tuesday|2073Q2|N|N|N|2478360|2478510|2478014|2478289|N|N|N|N|N| +2478380|AAAAAAAAMCBNFCAA|2073-06-21|2081|9052|695|2073|3|6|21|2|2073|695|9052|Wednesday|2073Q2|N|N|N|2478360|2478510|2478015|2478290|N|N|N|N|N| +2478381|AAAAAAAANCBNFCAA|2073-06-22|2081|9052|695|2073|4|6|22|2|2073|695|9052|Thursday|2073Q2|N|N|N|2478360|2478510|2478016|2478291|N|N|N|N|N| +2478382|AAAAAAAAOCBNFCAA|2073-06-23|2081|9052|695|2073|5|6|23|2|2073|695|9052|Friday|2073Q2|N|Y|N|2478360|2478510|2478017|2478292|N|N|N|N|N| +2478383|AAAAAAAAPCBNFCAA|2073-06-24|2081|9052|695|2073|6|6|24|2|2073|695|9052|Saturday|2073Q2|N|Y|N|2478360|2478510|2478018|2478293|N|N|N|N|N| +2478384|AAAAAAAAADBNFCAA|2073-06-25|2081|9052|695|2073|0|6|25|2|2073|695|9052|Sunday|2073Q2|N|N|N|2478360|2478510|2478019|2478294|N|N|N|N|N| +2478385|AAAAAAAABDBNFCAA|2073-06-26|2081|9052|695|2073|1|6|26|2|2073|695|9052|Monday|2073Q2|N|N|N|2478360|2478510|2478020|2478295|N|N|N|N|N| +2478386|AAAAAAAACDBNFCAA|2073-06-27|2081|9053|695|2073|2|6|27|2|2073|695|9053|Tuesday|2073Q2|N|N|N|2478360|2478510|2478021|2478296|N|N|N|N|N| +2478387|AAAAAAAADDBNFCAA|2073-06-28|2081|9053|695|2073|3|6|28|2|2073|695|9053|Wednesday|2073Q2|N|N|N|2478360|2478510|2478022|2478297|N|N|N|N|N| +2478388|AAAAAAAAEDBNFCAA|2073-06-29|2081|9053|695|2073|4|6|29|2|2073|695|9053|Thursday|2073Q2|N|N|N|2478360|2478510|2478023|2478298|N|N|N|N|N| +2478389|AAAAAAAAFDBNFCAA|2073-06-30|2081|9053|695|2073|5|6|30|2|2073|695|9053|Friday|2073Q2|N|Y|N|2478360|2478510|2478024|2478299|N|N|N|N|N| +2478390|AAAAAAAAGDBNFCAA|2073-07-01|2082|9053|695|2073|6|7|1|2|2073|695|9053|Saturday|2073Q2|N|Y|N|2478390|2478570|2478025|2478299|N|N|N|N|N| +2478391|AAAAAAAAHDBNFCAA|2073-07-02|2082|9053|695|2073|0|7|2|3|2073|695|9053|Sunday|2073Q3|N|N|N|2478390|2478570|2478026|2478300|N|N|N|N|N| +2478392|AAAAAAAAIDBNFCAA|2073-07-03|2082|9053|695|2073|1|7|3|3|2073|695|9053|Monday|2073Q3|N|N|N|2478390|2478570|2478027|2478301|N|N|N|N|N| +2478393|AAAAAAAAJDBNFCAA|2073-07-04|2082|9054|695|2073|2|7|4|3|2073|695|9054|Tuesday|2073Q3|N|N|N|2478390|2478570|2478028|2478302|N|N|N|N|N| +2478394|AAAAAAAAKDBNFCAA|2073-07-05|2082|9054|695|2073|3|7|5|3|2073|695|9054|Wednesday|2073Q3|Y|N|N|2478390|2478570|2478029|2478303|N|N|N|N|N| +2478395|AAAAAAAALDBNFCAA|2073-07-06|2082|9054|695|2073|4|7|6|3|2073|695|9054|Thursday|2073Q3|N|N|Y|2478390|2478570|2478030|2478304|N|N|N|N|N| +2478396|AAAAAAAAMDBNFCAA|2073-07-07|2082|9054|695|2073|5|7|7|3|2073|695|9054|Friday|2073Q3|N|Y|N|2478390|2478570|2478031|2478305|N|N|N|N|N| +2478397|AAAAAAAANDBNFCAA|2073-07-08|2082|9054|695|2073|6|7|8|3|2073|695|9054|Saturday|2073Q3|N|Y|N|2478390|2478570|2478032|2478306|N|N|N|N|N| +2478398|AAAAAAAAODBNFCAA|2073-07-09|2082|9054|695|2073|0|7|9|3|2073|695|9054|Sunday|2073Q3|N|N|N|2478390|2478570|2478033|2478307|N|N|N|N|N| +2478399|AAAAAAAAPDBNFCAA|2073-07-10|2082|9054|695|2073|1|7|10|3|2073|695|9054|Monday|2073Q3|N|N|N|2478390|2478570|2478034|2478308|N|N|N|N|N| +2478400|AAAAAAAAAEBNFCAA|2073-07-11|2082|9055|695|2073|2|7|11|3|2073|695|9055|Tuesday|2073Q3|N|N|N|2478390|2478570|2478035|2478309|N|N|N|N|N| +2478401|AAAAAAAABEBNFCAA|2073-07-12|2082|9055|695|2073|3|7|12|3|2073|695|9055|Wednesday|2073Q3|N|N|N|2478390|2478570|2478036|2478310|N|N|N|N|N| +2478402|AAAAAAAACEBNFCAA|2073-07-13|2082|9055|695|2073|4|7|13|3|2073|695|9055|Thursday|2073Q3|N|N|N|2478390|2478570|2478037|2478311|N|N|N|N|N| +2478403|AAAAAAAADEBNFCAA|2073-07-14|2082|9055|695|2073|5|7|14|3|2073|695|9055|Friday|2073Q3|N|Y|N|2478390|2478570|2478038|2478312|N|N|N|N|N| +2478404|AAAAAAAAEEBNFCAA|2073-07-15|2082|9055|695|2073|6|7|15|3|2073|695|9055|Saturday|2073Q3|N|Y|N|2478390|2478570|2478039|2478313|N|N|N|N|N| +2478405|AAAAAAAAFEBNFCAA|2073-07-16|2082|9055|695|2073|0|7|16|3|2073|695|9055|Sunday|2073Q3|N|N|N|2478390|2478570|2478040|2478314|N|N|N|N|N| +2478406|AAAAAAAAGEBNFCAA|2073-07-17|2082|9055|695|2073|1|7|17|3|2073|695|9055|Monday|2073Q3|N|N|N|2478390|2478570|2478041|2478315|N|N|N|N|N| +2478407|AAAAAAAAHEBNFCAA|2073-07-18|2082|9056|695|2073|2|7|18|3|2073|695|9056|Tuesday|2073Q3|N|N|N|2478390|2478570|2478042|2478316|N|N|N|N|N| +2478408|AAAAAAAAIEBNFCAA|2073-07-19|2082|9056|695|2073|3|7|19|3|2073|695|9056|Wednesday|2073Q3|N|N|N|2478390|2478570|2478043|2478317|N|N|N|N|N| +2478409|AAAAAAAAJEBNFCAA|2073-07-20|2082|9056|695|2073|4|7|20|3|2073|695|9056|Thursday|2073Q3|N|N|N|2478390|2478570|2478044|2478318|N|N|N|N|N| +2478410|AAAAAAAAKEBNFCAA|2073-07-21|2082|9056|695|2073|5|7|21|3|2073|695|9056|Friday|2073Q3|N|Y|N|2478390|2478570|2478045|2478319|N|N|N|N|N| +2478411|AAAAAAAALEBNFCAA|2073-07-22|2082|9056|695|2073|6|7|22|3|2073|695|9056|Saturday|2073Q3|N|Y|N|2478390|2478570|2478046|2478320|N|N|N|N|N| +2478412|AAAAAAAAMEBNFCAA|2073-07-23|2082|9056|695|2073|0|7|23|3|2073|695|9056|Sunday|2073Q3|N|N|N|2478390|2478570|2478047|2478321|N|N|N|N|N| +2478413|AAAAAAAANEBNFCAA|2073-07-24|2082|9056|695|2073|1|7|24|3|2073|695|9056|Monday|2073Q3|N|N|N|2478390|2478570|2478048|2478322|N|N|N|N|N| +2478414|AAAAAAAAOEBNFCAA|2073-07-25|2082|9057|695|2073|2|7|25|3|2073|695|9057|Tuesday|2073Q3|N|N|N|2478390|2478570|2478049|2478323|N|N|N|N|N| +2478415|AAAAAAAAPEBNFCAA|2073-07-26|2082|9057|695|2073|3|7|26|3|2073|695|9057|Wednesday|2073Q3|N|N|N|2478390|2478570|2478050|2478324|N|N|N|N|N| +2478416|AAAAAAAAAFBNFCAA|2073-07-27|2082|9057|695|2073|4|7|27|3|2073|695|9057|Thursday|2073Q3|N|N|N|2478390|2478570|2478051|2478325|N|N|N|N|N| +2478417|AAAAAAAABFBNFCAA|2073-07-28|2082|9057|695|2073|5|7|28|3|2073|695|9057|Friday|2073Q3|N|Y|N|2478390|2478570|2478052|2478326|N|N|N|N|N| +2478418|AAAAAAAACFBNFCAA|2073-07-29|2082|9057|695|2073|6|7|29|3|2073|695|9057|Saturday|2073Q3|N|Y|N|2478390|2478570|2478053|2478327|N|N|N|N|N| +2478419|AAAAAAAADFBNFCAA|2073-07-30|2082|9057|695|2073|0|7|30|3|2073|695|9057|Sunday|2073Q3|N|N|N|2478390|2478570|2478054|2478328|N|N|N|N|N| +2478420|AAAAAAAAEFBNFCAA|2073-07-31|2082|9057|695|2073|1|7|31|3|2073|695|9057|Monday|2073Q3|N|N|N|2478390|2478570|2478055|2478329|N|N|N|N|N| +2478421|AAAAAAAAFFBNFCAA|2073-08-01|2083|9058|695|2073|2|8|1|3|2073|695|9058|Tuesday|2073Q3|N|N|N|2478421|2478632|2478056|2478330|N|N|N|N|N| +2478422|AAAAAAAAGFBNFCAA|2073-08-02|2083|9058|695|2073|3|8|2|3|2073|695|9058|Wednesday|2073Q3|N|N|N|2478421|2478632|2478057|2478331|N|N|N|N|N| +2478423|AAAAAAAAHFBNFCAA|2073-08-03|2083|9058|695|2073|4|8|3|3|2073|695|9058|Thursday|2073Q3|N|N|N|2478421|2478632|2478058|2478332|N|N|N|N|N| +2478424|AAAAAAAAIFBNFCAA|2073-08-04|2083|9058|695|2073|5|8|4|3|2073|695|9058|Friday|2073Q3|N|Y|N|2478421|2478632|2478059|2478333|N|N|N|N|N| +2478425|AAAAAAAAJFBNFCAA|2073-08-05|2083|9058|695|2073|6|8|5|3|2073|695|9058|Saturday|2073Q3|N|Y|N|2478421|2478632|2478060|2478334|N|N|N|N|N| +2478426|AAAAAAAAKFBNFCAA|2073-08-06|2083|9058|695|2073|0|8|6|3|2073|695|9058|Sunday|2073Q3|N|N|N|2478421|2478632|2478061|2478335|N|N|N|N|N| +2478427|AAAAAAAALFBNFCAA|2073-08-07|2083|9058|695|2073|1|8|7|3|2073|695|9058|Monday|2073Q3|N|N|N|2478421|2478632|2478062|2478336|N|N|N|N|N| +2478428|AAAAAAAAMFBNFCAA|2073-08-08|2083|9059|695|2073|2|8|8|3|2073|695|9059|Tuesday|2073Q3|N|N|N|2478421|2478632|2478063|2478337|N|N|N|N|N| +2478429|AAAAAAAANFBNFCAA|2073-08-09|2083|9059|695|2073|3|8|9|3|2073|695|9059|Wednesday|2073Q3|N|N|N|2478421|2478632|2478064|2478338|N|N|N|N|N| +2478430|AAAAAAAAOFBNFCAA|2073-08-10|2083|9059|695|2073|4|8|10|3|2073|695|9059|Thursday|2073Q3|N|N|N|2478421|2478632|2478065|2478339|N|N|N|N|N| +2478431|AAAAAAAAPFBNFCAA|2073-08-11|2083|9059|695|2073|5|8|11|3|2073|695|9059|Friday|2073Q3|N|Y|N|2478421|2478632|2478066|2478340|N|N|N|N|N| +2478432|AAAAAAAAAGBNFCAA|2073-08-12|2083|9059|695|2073|6|8|12|3|2073|695|9059|Saturday|2073Q3|N|Y|N|2478421|2478632|2478067|2478341|N|N|N|N|N| +2478433|AAAAAAAABGBNFCAA|2073-08-13|2083|9059|695|2073|0|8|13|3|2073|695|9059|Sunday|2073Q3|N|N|N|2478421|2478632|2478068|2478342|N|N|N|N|N| +2478434|AAAAAAAACGBNFCAA|2073-08-14|2083|9059|695|2073|1|8|14|3|2073|695|9059|Monday|2073Q3|N|N|N|2478421|2478632|2478069|2478343|N|N|N|N|N| +2478435|AAAAAAAADGBNFCAA|2073-08-15|2083|9060|695|2073|2|8|15|3|2073|695|9060|Tuesday|2073Q3|N|N|N|2478421|2478632|2478070|2478344|N|N|N|N|N| +2478436|AAAAAAAAEGBNFCAA|2073-08-16|2083|9060|695|2073|3|8|16|3|2073|695|9060|Wednesday|2073Q3|N|N|N|2478421|2478632|2478071|2478345|N|N|N|N|N| +2478437|AAAAAAAAFGBNFCAA|2073-08-17|2083|9060|695|2073|4|8|17|3|2073|695|9060|Thursday|2073Q3|N|N|N|2478421|2478632|2478072|2478346|N|N|N|N|N| +2478438|AAAAAAAAGGBNFCAA|2073-08-18|2083|9060|695|2073|5|8|18|3|2073|695|9060|Friday|2073Q3|N|Y|N|2478421|2478632|2478073|2478347|N|N|N|N|N| +2478439|AAAAAAAAHGBNFCAA|2073-08-19|2083|9060|695|2073|6|8|19|3|2073|695|9060|Saturday|2073Q3|N|Y|N|2478421|2478632|2478074|2478348|N|N|N|N|N| +2478440|AAAAAAAAIGBNFCAA|2073-08-20|2083|9060|695|2073|0|8|20|3|2073|695|9060|Sunday|2073Q3|N|N|N|2478421|2478632|2478075|2478349|N|N|N|N|N| +2478441|AAAAAAAAJGBNFCAA|2073-08-21|2083|9060|695|2073|1|8|21|3|2073|695|9060|Monday|2073Q3|N|N|N|2478421|2478632|2478076|2478350|N|N|N|N|N| +2478442|AAAAAAAAKGBNFCAA|2073-08-22|2083|9061|695|2073|2|8|22|3|2073|695|9061|Tuesday|2073Q3|N|N|N|2478421|2478632|2478077|2478351|N|N|N|N|N| +2478443|AAAAAAAALGBNFCAA|2073-08-23|2083|9061|695|2073|3|8|23|3|2073|695|9061|Wednesday|2073Q3|N|N|N|2478421|2478632|2478078|2478352|N|N|N|N|N| +2478444|AAAAAAAAMGBNFCAA|2073-08-24|2083|9061|695|2073|4|8|24|3|2073|695|9061|Thursday|2073Q3|N|N|N|2478421|2478632|2478079|2478353|N|N|N|N|N| +2478445|AAAAAAAANGBNFCAA|2073-08-25|2083|9061|695|2073|5|8|25|3|2073|695|9061|Friday|2073Q3|N|Y|N|2478421|2478632|2478080|2478354|N|N|N|N|N| +2478446|AAAAAAAAOGBNFCAA|2073-08-26|2083|9061|695|2073|6|8|26|3|2073|695|9061|Saturday|2073Q3|N|Y|N|2478421|2478632|2478081|2478355|N|N|N|N|N| +2478447|AAAAAAAAPGBNFCAA|2073-08-27|2083|9061|695|2073|0|8|27|3|2073|695|9061|Sunday|2073Q3|N|N|N|2478421|2478632|2478082|2478356|N|N|N|N|N| +2478448|AAAAAAAAAHBNFCAA|2073-08-28|2083|9061|695|2073|1|8|28|3|2073|695|9061|Monday|2073Q3|N|N|N|2478421|2478632|2478083|2478357|N|N|N|N|N| +2478449|AAAAAAAABHBNFCAA|2073-08-29|2083|9062|695|2073|2|8|29|3|2073|695|9062|Tuesday|2073Q3|N|N|N|2478421|2478632|2478084|2478358|N|N|N|N|N| +2478450|AAAAAAAACHBNFCAA|2073-08-30|2083|9062|695|2073|3|8|30|3|2073|695|9062|Wednesday|2073Q3|N|N|N|2478421|2478632|2478085|2478359|N|N|N|N|N| +2478451|AAAAAAAADHBNFCAA|2073-08-31|2083|9062|695|2073|4|8|31|3|2073|695|9062|Thursday|2073Q3|N|N|N|2478421|2478632|2478086|2478360|N|N|N|N|N| +2478452|AAAAAAAAEHBNFCAA|2073-09-01|2084|9062|696|2073|5|9|1|3|2073|696|9062|Friday|2073Q3|N|Y|N|2478452|2478694|2478087|2478361|N|N|N|N|N| +2478453|AAAAAAAAFHBNFCAA|2073-09-02|2084|9062|696|2073|6|9|2|3|2073|696|9062|Saturday|2073Q3|N|Y|N|2478452|2478694|2478088|2478362|N|N|N|N|N| +2478454|AAAAAAAAGHBNFCAA|2073-09-03|2084|9062|696|2073|0|9|3|3|2073|696|9062|Sunday|2073Q3|N|N|N|2478452|2478694|2478089|2478363|N|N|N|N|N| +2478455|AAAAAAAAHHBNFCAA|2073-09-04|2084|9062|696|2073|1|9|4|3|2073|696|9062|Monday|2073Q3|N|N|N|2478452|2478694|2478090|2478364|N|N|N|N|N| +2478456|AAAAAAAAIHBNFCAA|2073-09-05|2084|9063|696|2073|2|9|5|3|2073|696|9063|Tuesday|2073Q3|N|N|N|2478452|2478694|2478091|2478365|N|N|N|N|N| +2478457|AAAAAAAAJHBNFCAA|2073-09-06|2084|9063|696|2073|3|9|6|3|2073|696|9063|Wednesday|2073Q3|N|N|N|2478452|2478694|2478092|2478366|N|N|N|N|N| +2478458|AAAAAAAAKHBNFCAA|2073-09-07|2084|9063|696|2073|4|9|7|3|2073|696|9063|Thursday|2073Q3|N|N|N|2478452|2478694|2478093|2478367|N|N|N|N|N| +2478459|AAAAAAAALHBNFCAA|2073-09-08|2084|9063|696|2073|5|9|8|3|2073|696|9063|Friday|2073Q3|N|Y|N|2478452|2478694|2478094|2478368|N|N|N|N|N| +2478460|AAAAAAAAMHBNFCAA|2073-09-09|2084|9063|696|2073|6|9|9|3|2073|696|9063|Saturday|2073Q3|N|Y|N|2478452|2478694|2478095|2478369|N|N|N|N|N| +2478461|AAAAAAAANHBNFCAA|2073-09-10|2084|9063|696|2073|0|9|10|3|2073|696|9063|Sunday|2073Q3|N|N|N|2478452|2478694|2478096|2478370|N|N|N|N|N| +2478462|AAAAAAAAOHBNFCAA|2073-09-11|2084|9063|696|2073|1|9|11|3|2073|696|9063|Monday|2073Q3|N|N|N|2478452|2478694|2478097|2478371|N|N|N|N|N| +2478463|AAAAAAAAPHBNFCAA|2073-09-12|2084|9064|696|2073|2|9|12|3|2073|696|9064|Tuesday|2073Q3|N|N|N|2478452|2478694|2478098|2478372|N|N|N|N|N| +2478464|AAAAAAAAAIBNFCAA|2073-09-13|2084|9064|696|2073|3|9|13|3|2073|696|9064|Wednesday|2073Q3|N|N|N|2478452|2478694|2478099|2478373|N|N|N|N|N| +2478465|AAAAAAAABIBNFCAA|2073-09-14|2084|9064|696|2073|4|9|14|3|2073|696|9064|Thursday|2073Q3|N|N|N|2478452|2478694|2478100|2478374|N|N|N|N|N| +2478466|AAAAAAAACIBNFCAA|2073-09-15|2084|9064|696|2073|5|9|15|3|2073|696|9064|Friday|2073Q3|N|Y|N|2478452|2478694|2478101|2478375|N|N|N|N|N| +2478467|AAAAAAAADIBNFCAA|2073-09-16|2084|9064|696|2073|6|9|16|3|2073|696|9064|Saturday|2073Q3|N|Y|N|2478452|2478694|2478102|2478376|N|N|N|N|N| +2478468|AAAAAAAAEIBNFCAA|2073-09-17|2084|9064|696|2073|0|9|17|3|2073|696|9064|Sunday|2073Q3|N|N|N|2478452|2478694|2478103|2478377|N|N|N|N|N| +2478469|AAAAAAAAFIBNFCAA|2073-09-18|2084|9064|696|2073|1|9|18|3|2073|696|9064|Monday|2073Q3|N|N|N|2478452|2478694|2478104|2478378|N|N|N|N|N| +2478470|AAAAAAAAGIBNFCAA|2073-09-19|2084|9065|696|2073|2|9|19|3|2073|696|9065|Tuesday|2073Q3|N|N|N|2478452|2478694|2478105|2478379|N|N|N|N|N| +2478471|AAAAAAAAHIBNFCAA|2073-09-20|2084|9065|696|2073|3|9|20|3|2073|696|9065|Wednesday|2073Q3|N|N|N|2478452|2478694|2478106|2478380|N|N|N|N|N| +2478472|AAAAAAAAIIBNFCAA|2073-09-21|2084|9065|696|2073|4|9|21|3|2073|696|9065|Thursday|2073Q3|N|N|N|2478452|2478694|2478107|2478381|N|N|N|N|N| +2478473|AAAAAAAAJIBNFCAA|2073-09-22|2084|9065|696|2073|5|9|22|3|2073|696|9065|Friday|2073Q3|N|Y|N|2478452|2478694|2478108|2478382|N|N|N|N|N| +2478474|AAAAAAAAKIBNFCAA|2073-09-23|2084|9065|696|2073|6|9|23|3|2073|696|9065|Saturday|2073Q3|N|Y|N|2478452|2478694|2478109|2478383|N|N|N|N|N| +2478475|AAAAAAAALIBNFCAA|2073-09-24|2084|9065|696|2073|0|9|24|3|2073|696|9065|Sunday|2073Q3|N|N|N|2478452|2478694|2478110|2478384|N|N|N|N|N| +2478476|AAAAAAAAMIBNFCAA|2073-09-25|2084|9065|696|2073|1|9|25|3|2073|696|9065|Monday|2073Q3|N|N|N|2478452|2478694|2478111|2478385|N|N|N|N|N| +2478477|AAAAAAAANIBNFCAA|2073-09-26|2084|9066|696|2073|2|9|26|3|2073|696|9066|Tuesday|2073Q3|N|N|N|2478452|2478694|2478112|2478386|N|N|N|N|N| +2478478|AAAAAAAAOIBNFCAA|2073-09-27|2084|9066|696|2073|3|9|27|3|2073|696|9066|Wednesday|2073Q3|N|N|N|2478452|2478694|2478113|2478387|N|N|N|N|N| +2478479|AAAAAAAAPIBNFCAA|2073-09-28|2084|9066|696|2073|4|9|28|3|2073|696|9066|Thursday|2073Q3|N|N|N|2478452|2478694|2478114|2478388|N|N|N|N|N| +2478480|AAAAAAAAAJBNFCAA|2073-09-29|2084|9066|696|2073|5|9|29|3|2073|696|9066|Friday|2073Q3|N|Y|N|2478452|2478694|2478115|2478389|N|N|N|N|N| +2478481|AAAAAAAABJBNFCAA|2073-09-30|2084|9066|696|2073|6|9|30|3|2073|696|9066|Saturday|2073Q3|N|Y|N|2478452|2478694|2478116|2478390|N|N|N|N|N| +2478482|AAAAAAAACJBNFCAA|2073-10-01|2085|9066|696|2073|0|10|1|3|2073|696|9066|Sunday|2073Q3|N|N|N|2478482|2478754|2478117|2478390|N|N|N|N|N| +2478483|AAAAAAAADJBNFCAA|2073-10-02|2085|9066|696|2073|1|10|2|4|2073|696|9066|Monday|2073Q4|N|N|N|2478482|2478754|2478118|2478391|N|N|N|N|N| +2478484|AAAAAAAAEJBNFCAA|2073-10-03|2085|9067|696|2073|2|10|3|4|2073|696|9067|Tuesday|2073Q4|N|N|N|2478482|2478754|2478119|2478392|N|N|N|N|N| +2478485|AAAAAAAAFJBNFCAA|2073-10-04|2085|9067|696|2073|3|10|4|4|2073|696|9067|Wednesday|2073Q4|N|N|N|2478482|2478754|2478120|2478393|N|N|N|N|N| +2478486|AAAAAAAAGJBNFCAA|2073-10-05|2085|9067|696|2073|4|10|5|4|2073|696|9067|Thursday|2073Q4|N|N|N|2478482|2478754|2478121|2478394|N|N|N|N|N| +2478487|AAAAAAAAHJBNFCAA|2073-10-06|2085|9067|696|2073|5|10|6|4|2073|696|9067|Friday|2073Q4|N|Y|N|2478482|2478754|2478122|2478395|N|N|N|N|N| +2478488|AAAAAAAAIJBNFCAA|2073-10-07|2085|9067|696|2073|6|10|7|4|2073|696|9067|Saturday|2073Q4|N|Y|N|2478482|2478754|2478123|2478396|N|N|N|N|N| +2478489|AAAAAAAAJJBNFCAA|2073-10-08|2085|9067|696|2073|0|10|8|4|2073|696|9067|Sunday|2073Q4|N|N|N|2478482|2478754|2478124|2478397|N|N|N|N|N| +2478490|AAAAAAAAKJBNFCAA|2073-10-09|2085|9067|696|2073|1|10|9|4|2073|696|9067|Monday|2073Q4|N|N|N|2478482|2478754|2478125|2478398|N|N|N|N|N| +2478491|AAAAAAAALJBNFCAA|2073-10-10|2085|9068|696|2073|2|10|10|4|2073|696|9068|Tuesday|2073Q4|N|N|N|2478482|2478754|2478126|2478399|N|N|N|N|N| +2478492|AAAAAAAAMJBNFCAA|2073-10-11|2085|9068|696|2073|3|10|11|4|2073|696|9068|Wednesday|2073Q4|N|N|N|2478482|2478754|2478127|2478400|N|N|N|N|N| +2478493|AAAAAAAANJBNFCAA|2073-10-12|2085|9068|696|2073|4|10|12|4|2073|696|9068|Thursday|2073Q4|N|N|N|2478482|2478754|2478128|2478401|N|N|N|N|N| +2478494|AAAAAAAAOJBNFCAA|2073-10-13|2085|9068|696|2073|5|10|13|4|2073|696|9068|Friday|2073Q4|N|Y|N|2478482|2478754|2478129|2478402|N|N|N|N|N| +2478495|AAAAAAAAPJBNFCAA|2073-10-14|2085|9068|696|2073|6|10|14|4|2073|696|9068|Saturday|2073Q4|N|Y|N|2478482|2478754|2478130|2478403|N|N|N|N|N| +2478496|AAAAAAAAAKBNFCAA|2073-10-15|2085|9068|696|2073|0|10|15|4|2073|696|9068|Sunday|2073Q4|N|N|N|2478482|2478754|2478131|2478404|N|N|N|N|N| +2478497|AAAAAAAABKBNFCAA|2073-10-16|2085|9068|696|2073|1|10|16|4|2073|696|9068|Monday|2073Q4|N|N|N|2478482|2478754|2478132|2478405|N|N|N|N|N| +2478498|AAAAAAAACKBNFCAA|2073-10-17|2085|9069|696|2073|2|10|17|4|2073|696|9069|Tuesday|2073Q4|N|N|N|2478482|2478754|2478133|2478406|N|N|N|N|N| +2478499|AAAAAAAADKBNFCAA|2073-10-18|2085|9069|696|2073|3|10|18|4|2073|696|9069|Wednesday|2073Q4|N|N|N|2478482|2478754|2478134|2478407|N|N|N|N|N| +2478500|AAAAAAAAEKBNFCAA|2073-10-19|2085|9069|696|2073|4|10|19|4|2073|696|9069|Thursday|2073Q4|N|N|N|2478482|2478754|2478135|2478408|N|N|N|N|N| +2478501|AAAAAAAAFKBNFCAA|2073-10-20|2085|9069|696|2073|5|10|20|4|2073|696|9069|Friday|2073Q4|N|Y|N|2478482|2478754|2478136|2478409|N|N|N|N|N| +2478502|AAAAAAAAGKBNFCAA|2073-10-21|2085|9069|696|2073|6|10|21|4|2073|696|9069|Saturday|2073Q4|N|Y|N|2478482|2478754|2478137|2478410|N|N|N|N|N| +2478503|AAAAAAAAHKBNFCAA|2073-10-22|2085|9069|696|2073|0|10|22|4|2073|696|9069|Sunday|2073Q4|N|N|N|2478482|2478754|2478138|2478411|N|N|N|N|N| +2478504|AAAAAAAAIKBNFCAA|2073-10-23|2085|9069|696|2073|1|10|23|4|2073|696|9069|Monday|2073Q4|N|N|N|2478482|2478754|2478139|2478412|N|N|N|N|N| +2478505|AAAAAAAAJKBNFCAA|2073-10-24|2085|9070|696|2073|2|10|24|4|2073|696|9070|Tuesday|2073Q4|N|N|N|2478482|2478754|2478140|2478413|N|N|N|N|N| +2478506|AAAAAAAAKKBNFCAA|2073-10-25|2085|9070|696|2073|3|10|25|4|2073|696|9070|Wednesday|2073Q4|N|N|N|2478482|2478754|2478141|2478414|N|N|N|N|N| +2478507|AAAAAAAALKBNFCAA|2073-10-26|2085|9070|696|2073|4|10|26|4|2073|696|9070|Thursday|2073Q4|N|N|N|2478482|2478754|2478142|2478415|N|N|N|N|N| +2478508|AAAAAAAAMKBNFCAA|2073-10-27|2085|9070|696|2073|5|10|27|4|2073|696|9070|Friday|2073Q4|N|Y|N|2478482|2478754|2478143|2478416|N|N|N|N|N| +2478509|AAAAAAAANKBNFCAA|2073-10-28|2085|9070|696|2073|6|10|28|4|2073|696|9070|Saturday|2073Q4|N|Y|N|2478482|2478754|2478144|2478417|N|N|N|N|N| +2478510|AAAAAAAAOKBNFCAA|2073-10-29|2085|9070|696|2073|0|10|29|4|2073|696|9070|Sunday|2073Q4|N|N|N|2478482|2478754|2478145|2478418|N|N|N|N|N| +2478511|AAAAAAAAPKBNFCAA|2073-10-30|2085|9070|696|2073|1|10|30|4|2073|696|9070|Monday|2073Q4|N|N|N|2478482|2478754|2478146|2478419|N|N|N|N|N| +2478512|AAAAAAAAALBNFCAA|2073-10-31|2085|9071|696|2073|2|10|31|4|2073|696|9071|Tuesday|2073Q4|N|N|N|2478482|2478754|2478147|2478420|N|N|N|N|N| +2478513|AAAAAAAABLBNFCAA|2073-11-01|2086|9071|696|2073|3|11|1|4|2073|696|9071|Wednesday|2073Q4|N|N|N|2478513|2478816|2478148|2478421|N|N|N|N|N| +2478514|AAAAAAAACLBNFCAA|2073-11-02|2086|9071|696|2073|4|11|2|4|2073|696|9071|Thursday|2073Q4|N|N|N|2478513|2478816|2478149|2478422|N|N|N|N|N| +2478515|AAAAAAAADLBNFCAA|2073-11-03|2086|9071|696|2073|5|11|3|4|2073|696|9071|Friday|2073Q4|N|Y|N|2478513|2478816|2478150|2478423|N|N|N|N|N| +2478516|AAAAAAAAELBNFCAA|2073-11-04|2086|9071|696|2073|6|11|4|4|2073|696|9071|Saturday|2073Q4|N|Y|N|2478513|2478816|2478151|2478424|N|N|N|N|N| +2478517|AAAAAAAAFLBNFCAA|2073-11-05|2086|9071|696|2073|0|11|5|4|2073|696|9071|Sunday|2073Q4|N|N|N|2478513|2478816|2478152|2478425|N|N|N|N|N| +2478518|AAAAAAAAGLBNFCAA|2073-11-06|2086|9071|696|2073|1|11|6|4|2073|696|9071|Monday|2073Q4|N|N|N|2478513|2478816|2478153|2478426|N|N|N|N|N| +2478519|AAAAAAAAHLBNFCAA|2073-11-07|2086|9072|696|2073|2|11|7|4|2073|696|9072|Tuesday|2073Q4|N|N|N|2478513|2478816|2478154|2478427|N|N|N|N|N| +2478520|AAAAAAAAILBNFCAA|2073-11-08|2086|9072|696|2073|3|11|8|4|2073|696|9072|Wednesday|2073Q4|N|N|N|2478513|2478816|2478155|2478428|N|N|N|N|N| +2478521|AAAAAAAAJLBNFCAA|2073-11-09|2086|9072|696|2073|4|11|9|4|2073|696|9072|Thursday|2073Q4|N|N|N|2478513|2478816|2478156|2478429|N|N|N|N|N| +2478522|AAAAAAAAKLBNFCAA|2073-11-10|2086|9072|696|2073|5|11|10|4|2073|696|9072|Friday|2073Q4|N|Y|N|2478513|2478816|2478157|2478430|N|N|N|N|N| +2478523|AAAAAAAALLBNFCAA|2073-11-11|2086|9072|696|2073|6|11|11|4|2073|696|9072|Saturday|2073Q4|N|Y|N|2478513|2478816|2478158|2478431|N|N|N|N|N| +2478524|AAAAAAAAMLBNFCAA|2073-11-12|2086|9072|696|2073|0|11|12|4|2073|696|9072|Sunday|2073Q4|N|N|N|2478513|2478816|2478159|2478432|N|N|N|N|N| +2478525|AAAAAAAANLBNFCAA|2073-11-13|2086|9072|696|2073|1|11|13|4|2073|696|9072|Monday|2073Q4|N|N|N|2478513|2478816|2478160|2478433|N|N|N|N|N| +2478526|AAAAAAAAOLBNFCAA|2073-11-14|2086|9073|696|2073|2|11|14|4|2073|696|9073|Tuesday|2073Q4|N|N|N|2478513|2478816|2478161|2478434|N|N|N|N|N| +2478527|AAAAAAAAPLBNFCAA|2073-11-15|2086|9073|696|2073|3|11|15|4|2073|696|9073|Wednesday|2073Q4|N|N|N|2478513|2478816|2478162|2478435|N|N|N|N|N| +2478528|AAAAAAAAAMBNFCAA|2073-11-16|2086|9073|696|2073|4|11|16|4|2073|696|9073|Thursday|2073Q4|N|N|N|2478513|2478816|2478163|2478436|N|N|N|N|N| +2478529|AAAAAAAABMBNFCAA|2073-11-17|2086|9073|696|2073|5|11|17|4|2073|696|9073|Friday|2073Q4|N|Y|N|2478513|2478816|2478164|2478437|N|N|N|N|N| +2478530|AAAAAAAACMBNFCAA|2073-11-18|2086|9073|696|2073|6|11|18|4|2073|696|9073|Saturday|2073Q4|N|Y|N|2478513|2478816|2478165|2478438|N|N|N|N|N| +2478531|AAAAAAAADMBNFCAA|2073-11-19|2086|9073|696|2073|0|11|19|4|2073|696|9073|Sunday|2073Q4|N|N|N|2478513|2478816|2478166|2478439|N|N|N|N|N| +2478532|AAAAAAAAEMBNFCAA|2073-11-20|2086|9073|696|2073|1|11|20|4|2073|696|9073|Monday|2073Q4|N|N|N|2478513|2478816|2478167|2478440|N|N|N|N|N| +2478533|AAAAAAAAFMBNFCAA|2073-11-21|2086|9074|696|2073|2|11|21|4|2073|696|9074|Tuesday|2073Q4|N|N|N|2478513|2478816|2478168|2478441|N|N|N|N|N| +2478534|AAAAAAAAGMBNFCAA|2073-11-22|2086|9074|696|2073|3|11|22|4|2073|696|9074|Wednesday|2073Q4|N|N|N|2478513|2478816|2478169|2478442|N|N|N|N|N| +2478535|AAAAAAAAHMBNFCAA|2073-11-23|2086|9074|696|2073|4|11|23|4|2073|696|9074|Thursday|2073Q4|N|N|N|2478513|2478816|2478170|2478443|N|N|N|N|N| +2478536|AAAAAAAAIMBNFCAA|2073-11-24|2086|9074|696|2073|5|11|24|4|2073|696|9074|Friday|2073Q4|N|Y|N|2478513|2478816|2478171|2478444|N|N|N|N|N| +2478537|AAAAAAAAJMBNFCAA|2073-11-25|2086|9074|696|2073|6|11|25|4|2073|696|9074|Saturday|2073Q4|N|Y|N|2478513|2478816|2478172|2478445|N|N|N|N|N| +2478538|AAAAAAAAKMBNFCAA|2073-11-26|2086|9074|696|2073|0|11|26|4|2073|696|9074|Sunday|2073Q4|N|N|N|2478513|2478816|2478173|2478446|N|N|N|N|N| +2478539|AAAAAAAALMBNFCAA|2073-11-27|2086|9074|696|2073|1|11|27|4|2073|696|9074|Monday|2073Q4|N|N|N|2478513|2478816|2478174|2478447|N|N|N|N|N| +2478540|AAAAAAAAMMBNFCAA|2073-11-28|2086|9075|696|2073|2|11|28|4|2073|696|9075|Tuesday|2073Q4|N|N|N|2478513|2478816|2478175|2478448|N|N|N|N|N| +2478541|AAAAAAAANMBNFCAA|2073-11-29|2086|9075|696|2073|3|11|29|4|2073|696|9075|Wednesday|2073Q4|N|N|N|2478513|2478816|2478176|2478449|N|N|N|N|N| +2478542|AAAAAAAAOMBNFCAA|2073-11-30|2086|9075|696|2073|4|11|30|4|2073|696|9075|Thursday|2073Q4|N|N|N|2478513|2478816|2478177|2478450|N|N|N|N|N| +2478543|AAAAAAAAPMBNFCAA|2073-12-01|2087|9075|697|2073|5|12|1|4|2073|697|9075|Friday|2073Q4|N|Y|N|2478543|2478876|2478178|2478451|N|N|N|N|N| +2478544|AAAAAAAAANBNFCAA|2073-12-02|2087|9075|697|2073|6|12|2|4|2073|697|9075|Saturday|2073Q4|N|Y|N|2478543|2478876|2478179|2478452|N|N|N|N|N| +2478545|AAAAAAAABNBNFCAA|2073-12-03|2087|9075|697|2073|0|12|3|4|2073|697|9075|Sunday|2073Q4|N|N|N|2478543|2478876|2478180|2478453|N|N|N|N|N| +2478546|AAAAAAAACNBNFCAA|2073-12-04|2087|9075|697|2073|1|12|4|4|2073|697|9075|Monday|2073Q4|N|N|N|2478543|2478876|2478181|2478454|N|N|N|N|N| +2478547|AAAAAAAADNBNFCAA|2073-12-05|2087|9076|697|2073|2|12|5|4|2073|697|9076|Tuesday|2073Q4|N|N|N|2478543|2478876|2478182|2478455|N|N|N|N|N| +2478548|AAAAAAAAENBNFCAA|2073-12-06|2087|9076|697|2073|3|12|6|4|2073|697|9076|Wednesday|2073Q4|N|N|N|2478543|2478876|2478183|2478456|N|N|N|N|N| +2478549|AAAAAAAAFNBNFCAA|2073-12-07|2087|9076|697|2073|4|12|7|4|2073|697|9076|Thursday|2073Q4|N|N|N|2478543|2478876|2478184|2478457|N|N|N|N|N| +2478550|AAAAAAAAGNBNFCAA|2073-12-08|2087|9076|697|2073|5|12|8|4|2073|697|9076|Friday|2073Q4|N|Y|N|2478543|2478876|2478185|2478458|N|N|N|N|N| +2478551|AAAAAAAAHNBNFCAA|2073-12-09|2087|9076|697|2073|6|12|9|4|2073|697|9076|Saturday|2073Q4|N|Y|N|2478543|2478876|2478186|2478459|N|N|N|N|N| +2478552|AAAAAAAAINBNFCAA|2073-12-10|2087|9076|697|2073|0|12|10|4|2073|697|9076|Sunday|2073Q4|N|N|N|2478543|2478876|2478187|2478460|N|N|N|N|N| +2478553|AAAAAAAAJNBNFCAA|2073-12-11|2087|9076|697|2073|1|12|11|4|2073|697|9076|Monday|2073Q4|N|N|N|2478543|2478876|2478188|2478461|N|N|N|N|N| +2478554|AAAAAAAAKNBNFCAA|2073-12-12|2087|9077|697|2073|2|12|12|4|2073|697|9077|Tuesday|2073Q4|N|N|N|2478543|2478876|2478189|2478462|N|N|N|N|N| +2478555|AAAAAAAALNBNFCAA|2073-12-13|2087|9077|697|2073|3|12|13|4|2073|697|9077|Wednesday|2073Q4|N|N|N|2478543|2478876|2478190|2478463|N|N|N|N|N| +2478556|AAAAAAAAMNBNFCAA|2073-12-14|2087|9077|697|2073|4|12|14|4|2073|697|9077|Thursday|2073Q4|N|N|N|2478543|2478876|2478191|2478464|N|N|N|N|N| +2478557|AAAAAAAANNBNFCAA|2073-12-15|2087|9077|697|2073|5|12|15|4|2073|697|9077|Friday|2073Q4|N|Y|N|2478543|2478876|2478192|2478465|N|N|N|N|N| +2478558|AAAAAAAAONBNFCAA|2073-12-16|2087|9077|697|2073|6|12|16|4|2073|697|9077|Saturday|2073Q4|N|Y|N|2478543|2478876|2478193|2478466|N|N|N|N|N| +2478559|AAAAAAAAPNBNFCAA|2073-12-17|2087|9077|697|2073|0|12|17|4|2073|697|9077|Sunday|2073Q4|N|N|N|2478543|2478876|2478194|2478467|N|N|N|N|N| +2478560|AAAAAAAAAOBNFCAA|2073-12-18|2087|9077|697|2073|1|12|18|4|2073|697|9077|Monday|2073Q4|N|N|N|2478543|2478876|2478195|2478468|N|N|N|N|N| +2478561|AAAAAAAABOBNFCAA|2073-12-19|2087|9078|697|2073|2|12|19|4|2073|697|9078|Tuesday|2073Q4|N|N|N|2478543|2478876|2478196|2478469|N|N|N|N|N| +2478562|AAAAAAAACOBNFCAA|2073-12-20|2087|9078|697|2073|3|12|20|4|2073|697|9078|Wednesday|2073Q4|N|N|N|2478543|2478876|2478197|2478470|N|N|N|N|N| +2478563|AAAAAAAADOBNFCAA|2073-12-21|2087|9078|697|2073|4|12|21|4|2073|697|9078|Thursday|2073Q4|N|N|N|2478543|2478876|2478198|2478471|N|N|N|N|N| +2478564|AAAAAAAAEOBNFCAA|2073-12-22|2087|9078|697|2073|5|12|22|4|2073|697|9078|Friday|2073Q4|N|Y|N|2478543|2478876|2478199|2478472|N|N|N|N|N| +2478565|AAAAAAAAFOBNFCAA|2073-12-23|2087|9078|697|2073|6|12|23|4|2073|697|9078|Saturday|2073Q4|N|Y|N|2478543|2478876|2478200|2478473|N|N|N|N|N| +2478566|AAAAAAAAGOBNFCAA|2073-12-24|2087|9078|697|2073|0|12|24|4|2073|697|9078|Sunday|2073Q4|N|N|N|2478543|2478876|2478201|2478474|N|N|N|N|N| +2478567|AAAAAAAAHOBNFCAA|2073-12-25|2087|9078|697|2073|1|12|25|4|2073|697|9078|Monday|2073Q4|N|N|N|2478543|2478876|2478202|2478475|N|N|N|N|N| +2478568|AAAAAAAAIOBNFCAA|2073-12-26|2087|9079|697|2073|2|12|26|4|2073|697|9079|Tuesday|2073Q4|Y|N|N|2478543|2478876|2478203|2478476|N|N|N|N|N| +2478569|AAAAAAAAJOBNFCAA|2073-12-27|2087|9079|697|2073|3|12|27|4|2073|697|9079|Wednesday|2073Q4|N|N|Y|2478543|2478876|2478204|2478477|N|N|N|N|N| +2478570|AAAAAAAAKOBNFCAA|2073-12-28|2087|9079|697|2073|4|12|28|4|2073|697|9079|Thursday|2073Q4|N|N|N|2478543|2478876|2478205|2478478|N|N|N|N|N| +2478571|AAAAAAAALOBNFCAA|2073-12-29|2087|9079|697|2073|5|12|29|4|2073|697|9079|Friday|2073Q4|N|Y|N|2478543|2478876|2478206|2478479|N|N|N|N|N| +2478572|AAAAAAAAMOBNFCAA|2073-12-30|2087|9079|697|2073|6|12|30|4|2073|697|9079|Saturday|2073Q4|N|Y|N|2478543|2478876|2478207|2478480|N|N|N|N|N| +2478573|AAAAAAAANOBNFCAA|2073-12-31|2087|9079|697|2073|0|12|31|4|2073|697|9079|Sunday|2073Q4|N|N|N|2478543|2478876|2478208|2478481|N|N|N|N|N| +2478574|AAAAAAAAOOBNFCAA|2074-01-01|2088|9079|697|2074|1|1|1|1|2074|697|9079|Monday|2074Q1|Y|N|N|2478574|2478573|2478209|2478482|N|N|N|N|N| +2478575|AAAAAAAAPOBNFCAA|2074-01-02|2088|9080|697|2074|2|1|2|1|2074|697|9080|Tuesday|2074Q1|N|N|Y|2478574|2478573|2478210|2478483|N|N|N|N|N| +2478576|AAAAAAAAAPBNFCAA|2074-01-03|2088|9080|697|2074|3|1|3|1|2074|697|9080|Wednesday|2074Q1|N|N|N|2478574|2478573|2478211|2478484|N|N|N|N|N| +2478577|AAAAAAAABPBNFCAA|2074-01-04|2088|9080|697|2074|4|1|4|1|2074|697|9080|Thursday|2074Q1|N|N|N|2478574|2478573|2478212|2478485|N|N|N|N|N| +2478578|AAAAAAAACPBNFCAA|2074-01-05|2088|9080|697|2074|5|1|5|1|2074|697|9080|Friday|2074Q1|N|Y|N|2478574|2478573|2478213|2478486|N|N|N|N|N| +2478579|AAAAAAAADPBNFCAA|2074-01-06|2088|9080|697|2074|6|1|6|1|2074|697|9080|Saturday|2074Q1|N|Y|N|2478574|2478573|2478214|2478487|N|N|N|N|N| +2478580|AAAAAAAAEPBNFCAA|2074-01-07|2088|9080|697|2074|0|1|7|1|2074|697|9080|Sunday|2074Q1|N|N|N|2478574|2478573|2478215|2478488|N|N|N|N|N| +2478581|AAAAAAAAFPBNFCAA|2074-01-08|2088|9080|697|2074|1|1|8|1|2074|697|9080|Monday|2074Q1|N|N|N|2478574|2478573|2478216|2478489|N|N|N|N|N| +2478582|AAAAAAAAGPBNFCAA|2074-01-09|2088|9081|697|2074|2|1|9|1|2074|697|9081|Tuesday|2074Q1|N|N|N|2478574|2478573|2478217|2478490|N|N|N|N|N| +2478583|AAAAAAAAHPBNFCAA|2074-01-10|2088|9081|697|2074|3|1|10|1|2074|697|9081|Wednesday|2074Q1|N|N|N|2478574|2478573|2478218|2478491|N|N|N|N|N| +2478584|AAAAAAAAIPBNFCAA|2074-01-11|2088|9081|697|2074|4|1|11|1|2074|697|9081|Thursday|2074Q1|N|N|N|2478574|2478573|2478219|2478492|N|N|N|N|N| +2478585|AAAAAAAAJPBNFCAA|2074-01-12|2088|9081|697|2074|5|1|12|1|2074|697|9081|Friday|2074Q1|N|Y|N|2478574|2478573|2478220|2478493|N|N|N|N|N| +2478586|AAAAAAAAKPBNFCAA|2074-01-13|2088|9081|697|2074|6|1|13|1|2074|697|9081|Saturday|2074Q1|N|Y|N|2478574|2478573|2478221|2478494|N|N|N|N|N| +2478587|AAAAAAAALPBNFCAA|2074-01-14|2088|9081|697|2074|0|1|14|1|2074|697|9081|Sunday|2074Q1|N|N|N|2478574|2478573|2478222|2478495|N|N|N|N|N| +2478588|AAAAAAAAMPBNFCAA|2074-01-15|2088|9081|697|2074|1|1|15|1|2074|697|9081|Monday|2074Q1|N|N|N|2478574|2478573|2478223|2478496|N|N|N|N|N| +2478589|AAAAAAAANPBNFCAA|2074-01-16|2088|9082|697|2074|2|1|16|1|2074|697|9082|Tuesday|2074Q1|N|N|N|2478574|2478573|2478224|2478497|N|N|N|N|N| +2478590|AAAAAAAAOPBNFCAA|2074-01-17|2088|9082|697|2074|3|1|17|1|2074|697|9082|Wednesday|2074Q1|N|N|N|2478574|2478573|2478225|2478498|N|N|N|N|N| +2478591|AAAAAAAAPPBNFCAA|2074-01-18|2088|9082|697|2074|4|1|18|1|2074|697|9082|Thursday|2074Q1|N|N|N|2478574|2478573|2478226|2478499|N|N|N|N|N| +2478592|AAAAAAAAAACNFCAA|2074-01-19|2088|9082|697|2074|5|1|19|1|2074|697|9082|Friday|2074Q1|N|Y|N|2478574|2478573|2478227|2478500|N|N|N|N|N| +2478593|AAAAAAAABACNFCAA|2074-01-20|2088|9082|697|2074|6|1|20|1|2074|697|9082|Saturday|2074Q1|N|Y|N|2478574|2478573|2478228|2478501|N|N|N|N|N| +2478594|AAAAAAAACACNFCAA|2074-01-21|2088|9082|697|2074|0|1|21|1|2074|697|9082|Sunday|2074Q1|N|N|N|2478574|2478573|2478229|2478502|N|N|N|N|N| +2478595|AAAAAAAADACNFCAA|2074-01-22|2088|9082|697|2074|1|1|22|1|2074|697|9082|Monday|2074Q1|N|N|N|2478574|2478573|2478230|2478503|N|N|N|N|N| +2478596|AAAAAAAAEACNFCAA|2074-01-23|2088|9083|697|2074|2|1|23|1|2074|697|9083|Tuesday|2074Q1|N|N|N|2478574|2478573|2478231|2478504|N|N|N|N|N| +2478597|AAAAAAAAFACNFCAA|2074-01-24|2088|9083|697|2074|3|1|24|1|2074|697|9083|Wednesday|2074Q1|N|N|N|2478574|2478573|2478232|2478505|N|N|N|N|N| +2478598|AAAAAAAAGACNFCAA|2074-01-25|2088|9083|697|2074|4|1|25|1|2074|697|9083|Thursday|2074Q1|N|N|N|2478574|2478573|2478233|2478506|N|N|N|N|N| +2478599|AAAAAAAAHACNFCAA|2074-01-26|2088|9083|697|2074|5|1|26|1|2074|697|9083|Friday|2074Q1|N|Y|N|2478574|2478573|2478234|2478507|N|N|N|N|N| +2478600|AAAAAAAAIACNFCAA|2074-01-27|2088|9083|697|2074|6|1|27|1|2074|697|9083|Saturday|2074Q1|N|Y|N|2478574|2478573|2478235|2478508|N|N|N|N|N| +2478601|AAAAAAAAJACNFCAA|2074-01-28|2088|9083|697|2074|0|1|28|1|2074|697|9083|Sunday|2074Q1|N|N|N|2478574|2478573|2478236|2478509|N|N|N|N|N| +2478602|AAAAAAAAKACNFCAA|2074-01-29|2088|9083|697|2074|1|1|29|1|2074|697|9083|Monday|2074Q1|N|N|N|2478574|2478573|2478237|2478510|N|N|N|N|N| +2478603|AAAAAAAALACNFCAA|2074-01-30|2088|9084|697|2074|2|1|30|1|2074|697|9084|Tuesday|2074Q1|N|N|N|2478574|2478573|2478238|2478511|N|N|N|N|N| +2478604|AAAAAAAAMACNFCAA|2074-01-31|2088|9084|697|2074|3|1|31|1|2074|697|9084|Wednesday|2074Q1|N|N|N|2478574|2478573|2478239|2478512|N|N|N|N|N| +2478605|AAAAAAAANACNFCAA|2074-02-01|2089|9084|697|2074|4|2|1|1|2074|697|9084|Thursday|2074Q1|N|N|N|2478605|2478635|2478240|2478513|N|N|N|N|N| +2478606|AAAAAAAAOACNFCAA|2074-02-02|2089|9084|697|2074|5|2|2|1|2074|697|9084|Friday|2074Q1|N|Y|N|2478605|2478635|2478241|2478514|N|N|N|N|N| +2478607|AAAAAAAAPACNFCAA|2074-02-03|2089|9084|697|2074|6|2|3|1|2074|697|9084|Saturday|2074Q1|N|Y|N|2478605|2478635|2478242|2478515|N|N|N|N|N| +2478608|AAAAAAAAABCNFCAA|2074-02-04|2089|9084|697|2074|0|2|4|1|2074|697|9084|Sunday|2074Q1|N|N|N|2478605|2478635|2478243|2478516|N|N|N|N|N| +2478609|AAAAAAAABBCNFCAA|2074-02-05|2089|9084|697|2074|1|2|5|1|2074|697|9084|Monday|2074Q1|N|N|N|2478605|2478635|2478244|2478517|N|N|N|N|N| +2478610|AAAAAAAACBCNFCAA|2074-02-06|2089|9085|697|2074|2|2|6|1|2074|697|9085|Tuesday|2074Q1|N|N|N|2478605|2478635|2478245|2478518|N|N|N|N|N| +2478611|AAAAAAAADBCNFCAA|2074-02-07|2089|9085|697|2074|3|2|7|1|2074|697|9085|Wednesday|2074Q1|N|N|N|2478605|2478635|2478246|2478519|N|N|N|N|N| +2478612|AAAAAAAAEBCNFCAA|2074-02-08|2089|9085|697|2074|4|2|8|1|2074|697|9085|Thursday|2074Q1|N|N|N|2478605|2478635|2478247|2478520|N|N|N|N|N| +2478613|AAAAAAAAFBCNFCAA|2074-02-09|2089|9085|697|2074|5|2|9|1|2074|697|9085|Friday|2074Q1|N|Y|N|2478605|2478635|2478248|2478521|N|N|N|N|N| +2478614|AAAAAAAAGBCNFCAA|2074-02-10|2089|9085|697|2074|6|2|10|1|2074|697|9085|Saturday|2074Q1|N|Y|N|2478605|2478635|2478249|2478522|N|N|N|N|N| +2478615|AAAAAAAAHBCNFCAA|2074-02-11|2089|9085|697|2074|0|2|11|1|2074|697|9085|Sunday|2074Q1|N|N|N|2478605|2478635|2478250|2478523|N|N|N|N|N| +2478616|AAAAAAAAIBCNFCAA|2074-02-12|2089|9085|697|2074|1|2|12|1|2074|697|9085|Monday|2074Q1|N|N|N|2478605|2478635|2478251|2478524|N|N|N|N|N| +2478617|AAAAAAAAJBCNFCAA|2074-02-13|2089|9086|697|2074|2|2|13|1|2074|697|9086|Tuesday|2074Q1|N|N|N|2478605|2478635|2478252|2478525|N|N|N|N|N| +2478618|AAAAAAAAKBCNFCAA|2074-02-14|2089|9086|697|2074|3|2|14|1|2074|697|9086|Wednesday|2074Q1|N|N|N|2478605|2478635|2478253|2478526|N|N|N|N|N| +2478619|AAAAAAAALBCNFCAA|2074-02-15|2089|9086|697|2074|4|2|15|1|2074|697|9086|Thursday|2074Q1|N|N|N|2478605|2478635|2478254|2478527|N|N|N|N|N| +2478620|AAAAAAAAMBCNFCAA|2074-02-16|2089|9086|697|2074|5|2|16|1|2074|697|9086|Friday|2074Q1|N|Y|N|2478605|2478635|2478255|2478528|N|N|N|N|N| +2478621|AAAAAAAANBCNFCAA|2074-02-17|2089|9086|697|2074|6|2|17|1|2074|697|9086|Saturday|2074Q1|N|Y|N|2478605|2478635|2478256|2478529|N|N|N|N|N| +2478622|AAAAAAAAOBCNFCAA|2074-02-18|2089|9086|697|2074|0|2|18|1|2074|697|9086|Sunday|2074Q1|N|N|N|2478605|2478635|2478257|2478530|N|N|N|N|N| +2478623|AAAAAAAAPBCNFCAA|2074-02-19|2089|9086|697|2074|1|2|19|1|2074|697|9086|Monday|2074Q1|N|N|N|2478605|2478635|2478258|2478531|N|N|N|N|N| +2478624|AAAAAAAAACCNFCAA|2074-02-20|2089|9087|697|2074|2|2|20|1|2074|697|9087|Tuesday|2074Q1|N|N|N|2478605|2478635|2478259|2478532|N|N|N|N|N| +2478625|AAAAAAAABCCNFCAA|2074-02-21|2089|9087|697|2074|3|2|21|1|2074|697|9087|Wednesday|2074Q1|N|N|N|2478605|2478635|2478260|2478533|N|N|N|N|N| +2478626|AAAAAAAACCCNFCAA|2074-02-22|2089|9087|697|2074|4|2|22|1|2074|697|9087|Thursday|2074Q1|N|N|N|2478605|2478635|2478261|2478534|N|N|N|N|N| +2478627|AAAAAAAADCCNFCAA|2074-02-23|2089|9087|697|2074|5|2|23|1|2074|697|9087|Friday|2074Q1|N|Y|N|2478605|2478635|2478262|2478535|N|N|N|N|N| +2478628|AAAAAAAAECCNFCAA|2074-02-24|2089|9087|697|2074|6|2|24|1|2074|697|9087|Saturday|2074Q1|N|Y|N|2478605|2478635|2478263|2478536|N|N|N|N|N| +2478629|AAAAAAAAFCCNFCAA|2074-02-25|2089|9087|697|2074|0|2|25|1|2074|697|9087|Sunday|2074Q1|N|N|N|2478605|2478635|2478264|2478537|N|N|N|N|N| +2478630|AAAAAAAAGCCNFCAA|2074-02-26|2089|9087|697|2074|1|2|26|1|2074|697|9087|Monday|2074Q1|N|N|N|2478605|2478635|2478265|2478538|N|N|N|N|N| +2478631|AAAAAAAAHCCNFCAA|2074-02-27|2089|9088|697|2074|2|2|27|1|2074|697|9088|Tuesday|2074Q1|N|N|N|2478605|2478635|2478266|2478539|N|N|N|N|N| +2478632|AAAAAAAAICCNFCAA|2074-02-28|2089|9088|697|2074|3|2|28|1|2074|697|9088|Wednesday|2074Q1|N|N|N|2478605|2478635|2478267|2478540|N|N|N|N|N| +2478633|AAAAAAAAJCCNFCAA|2074-03-01|2090|9088|698|2074|4|3|1|1|2074|698|9088|Thursday|2074Q1|N|N|N|2478633|2478691|2478268|2478541|N|N|N|N|N| +2478634|AAAAAAAAKCCNFCAA|2074-03-02|2090|9088|698|2074|5|3|2|1|2074|698|9088|Friday|2074Q1|N|Y|N|2478633|2478691|2478269|2478542|N|N|N|N|N| +2478635|AAAAAAAALCCNFCAA|2074-03-03|2090|9088|698|2074|6|3|3|1|2074|698|9088|Saturday|2074Q1|N|Y|N|2478633|2478691|2478270|2478543|N|N|N|N|N| +2478636|AAAAAAAAMCCNFCAA|2074-03-04|2090|9088|698|2074|0|3|4|1|2074|698|9088|Sunday|2074Q1|N|N|N|2478633|2478691|2478271|2478544|N|N|N|N|N| +2478637|AAAAAAAANCCNFCAA|2074-03-05|2090|9088|698|2074|1|3|5|1|2074|698|9088|Monday|2074Q1|N|N|N|2478633|2478691|2478272|2478545|N|N|N|N|N| +2478638|AAAAAAAAOCCNFCAA|2074-03-06|2090|9089|698|2074|2|3|6|1|2074|698|9089|Tuesday|2074Q1|N|N|N|2478633|2478691|2478273|2478546|N|N|N|N|N| +2478639|AAAAAAAAPCCNFCAA|2074-03-07|2090|9089|698|2074|3|3|7|1|2074|698|9089|Wednesday|2074Q1|N|N|N|2478633|2478691|2478274|2478547|N|N|N|N|N| +2478640|AAAAAAAAADCNFCAA|2074-03-08|2090|9089|698|2074|4|3|8|1|2074|698|9089|Thursday|2074Q1|N|N|N|2478633|2478691|2478275|2478548|N|N|N|N|N| +2478641|AAAAAAAABDCNFCAA|2074-03-09|2090|9089|698|2074|5|3|9|1|2074|698|9089|Friday|2074Q1|N|Y|N|2478633|2478691|2478276|2478549|N|N|N|N|N| +2478642|AAAAAAAACDCNFCAA|2074-03-10|2090|9089|698|2074|6|3|10|1|2074|698|9089|Saturday|2074Q1|N|Y|N|2478633|2478691|2478277|2478550|N|N|N|N|N| +2478643|AAAAAAAADDCNFCAA|2074-03-11|2090|9089|698|2074|0|3|11|1|2074|698|9089|Sunday|2074Q1|N|N|N|2478633|2478691|2478278|2478551|N|N|N|N|N| +2478644|AAAAAAAAEDCNFCAA|2074-03-12|2090|9089|698|2074|1|3|12|1|2074|698|9089|Monday|2074Q1|N|N|N|2478633|2478691|2478279|2478552|N|N|N|N|N| +2478645|AAAAAAAAFDCNFCAA|2074-03-13|2090|9090|698|2074|2|3|13|1|2074|698|9090|Tuesday|2074Q1|N|N|N|2478633|2478691|2478280|2478553|N|N|N|N|N| +2478646|AAAAAAAAGDCNFCAA|2074-03-14|2090|9090|698|2074|3|3|14|1|2074|698|9090|Wednesday|2074Q1|N|N|N|2478633|2478691|2478281|2478554|N|N|N|N|N| +2478647|AAAAAAAAHDCNFCAA|2074-03-15|2090|9090|698|2074|4|3|15|1|2074|698|9090|Thursday|2074Q1|N|N|N|2478633|2478691|2478282|2478555|N|N|N|N|N| +2478648|AAAAAAAAIDCNFCAA|2074-03-16|2090|9090|698|2074|5|3|16|1|2074|698|9090|Friday|2074Q1|N|Y|N|2478633|2478691|2478283|2478556|N|N|N|N|N| +2478649|AAAAAAAAJDCNFCAA|2074-03-17|2090|9090|698|2074|6|3|17|1|2074|698|9090|Saturday|2074Q1|N|Y|N|2478633|2478691|2478284|2478557|N|N|N|N|N| +2478650|AAAAAAAAKDCNFCAA|2074-03-18|2090|9090|698|2074|0|3|18|1|2074|698|9090|Sunday|2074Q1|N|N|N|2478633|2478691|2478285|2478558|N|N|N|N|N| +2478651|AAAAAAAALDCNFCAA|2074-03-19|2090|9090|698|2074|1|3|19|1|2074|698|9090|Monday|2074Q1|N|N|N|2478633|2478691|2478286|2478559|N|N|N|N|N| +2478652|AAAAAAAAMDCNFCAA|2074-03-20|2090|9091|698|2074|2|3|20|1|2074|698|9091|Tuesday|2074Q1|N|N|N|2478633|2478691|2478287|2478560|N|N|N|N|N| +2478653|AAAAAAAANDCNFCAA|2074-03-21|2090|9091|698|2074|3|3|21|1|2074|698|9091|Wednesday|2074Q1|N|N|N|2478633|2478691|2478288|2478561|N|N|N|N|N| +2478654|AAAAAAAAODCNFCAA|2074-03-22|2090|9091|698|2074|4|3|22|1|2074|698|9091|Thursday|2074Q1|N|N|N|2478633|2478691|2478289|2478562|N|N|N|N|N| +2478655|AAAAAAAAPDCNFCAA|2074-03-23|2090|9091|698|2074|5|3|23|1|2074|698|9091|Friday|2074Q1|N|Y|N|2478633|2478691|2478290|2478563|N|N|N|N|N| +2478656|AAAAAAAAAECNFCAA|2074-03-24|2090|9091|698|2074|6|3|24|1|2074|698|9091|Saturday|2074Q1|N|Y|N|2478633|2478691|2478291|2478564|N|N|N|N|N| +2478657|AAAAAAAABECNFCAA|2074-03-25|2090|9091|698|2074|0|3|25|1|2074|698|9091|Sunday|2074Q1|N|N|N|2478633|2478691|2478292|2478565|N|N|N|N|N| +2478658|AAAAAAAACECNFCAA|2074-03-26|2090|9091|698|2074|1|3|26|1|2074|698|9091|Monday|2074Q1|N|N|N|2478633|2478691|2478293|2478566|N|N|N|N|N| +2478659|AAAAAAAADECNFCAA|2074-03-27|2090|9092|698|2074|2|3|27|1|2074|698|9092|Tuesday|2074Q1|N|N|N|2478633|2478691|2478294|2478567|N|N|N|N|N| +2478660|AAAAAAAAEECNFCAA|2074-03-28|2090|9092|698|2074|3|3|28|1|2074|698|9092|Wednesday|2074Q1|N|N|N|2478633|2478691|2478295|2478568|N|N|N|N|N| +2478661|AAAAAAAAFECNFCAA|2074-03-29|2090|9092|698|2074|4|3|29|1|2074|698|9092|Thursday|2074Q1|N|N|N|2478633|2478691|2478296|2478569|N|N|N|N|N| +2478662|AAAAAAAAGECNFCAA|2074-03-30|2090|9092|698|2074|5|3|30|1|2074|698|9092|Friday|2074Q1|N|Y|N|2478633|2478691|2478297|2478570|N|N|N|N|N| +2478663|AAAAAAAAHECNFCAA|2074-03-31|2090|9092|698|2074|6|3|31|1|2074|698|9092|Saturday|2074Q1|N|Y|N|2478633|2478691|2478298|2478571|N|N|N|N|N| +2478664|AAAAAAAAIECNFCAA|2074-04-01|2091|9092|698|2074|0|4|1|1|2074|698|9092|Sunday|2074Q1|N|N|N|2478664|2478753|2478299|2478574|N|N|N|N|N| +2478665|AAAAAAAAJECNFCAA|2074-04-02|2091|9092|698|2074|1|4|2|2|2074|698|9092|Monday|2074Q2|N|N|N|2478664|2478753|2478300|2478575|N|N|N|N|N| +2478666|AAAAAAAAKECNFCAA|2074-04-03|2091|9093|698|2074|2|4|3|2|2074|698|9093|Tuesday|2074Q2|N|N|N|2478664|2478753|2478301|2478576|N|N|N|N|N| +2478667|AAAAAAAALECNFCAA|2074-04-04|2091|9093|698|2074|3|4|4|2|2074|698|9093|Wednesday|2074Q2|N|N|N|2478664|2478753|2478302|2478577|N|N|N|N|N| +2478668|AAAAAAAAMECNFCAA|2074-04-05|2091|9093|698|2074|4|4|5|2|2074|698|9093|Thursday|2074Q2|N|N|N|2478664|2478753|2478303|2478578|N|N|N|N|N| +2478669|AAAAAAAANECNFCAA|2074-04-06|2091|9093|698|2074|5|4|6|2|2074|698|9093|Friday|2074Q2|N|Y|N|2478664|2478753|2478304|2478579|N|N|N|N|N| +2478670|AAAAAAAAOECNFCAA|2074-04-07|2091|9093|698|2074|6|4|7|2|2074|698|9093|Saturday|2074Q2|N|Y|N|2478664|2478753|2478305|2478580|N|N|N|N|N| +2478671|AAAAAAAAPECNFCAA|2074-04-08|2091|9093|698|2074|0|4|8|2|2074|698|9093|Sunday|2074Q2|N|N|N|2478664|2478753|2478306|2478581|N|N|N|N|N| +2478672|AAAAAAAAAFCNFCAA|2074-04-09|2091|9093|698|2074|1|4|9|2|2074|698|9093|Monday|2074Q2|N|N|N|2478664|2478753|2478307|2478582|N|N|N|N|N| +2478673|AAAAAAAABFCNFCAA|2074-04-10|2091|9094|698|2074|2|4|10|2|2074|698|9094|Tuesday|2074Q2|N|N|N|2478664|2478753|2478308|2478583|N|N|N|N|N| +2478674|AAAAAAAACFCNFCAA|2074-04-11|2091|9094|698|2074|3|4|11|2|2074|698|9094|Wednesday|2074Q2|N|N|N|2478664|2478753|2478309|2478584|N|N|N|N|N| +2478675|AAAAAAAADFCNFCAA|2074-04-12|2091|9094|698|2074|4|4|12|2|2074|698|9094|Thursday|2074Q2|N|N|N|2478664|2478753|2478310|2478585|N|N|N|N|N| +2478676|AAAAAAAAEFCNFCAA|2074-04-13|2091|9094|698|2074|5|4|13|2|2074|698|9094|Friday|2074Q2|N|Y|N|2478664|2478753|2478311|2478586|N|N|N|N|N| +2478677|AAAAAAAAFFCNFCAA|2074-04-14|2091|9094|698|2074|6|4|14|2|2074|698|9094|Saturday|2074Q2|N|Y|N|2478664|2478753|2478312|2478587|N|N|N|N|N| +2478678|AAAAAAAAGFCNFCAA|2074-04-15|2091|9094|698|2074|0|4|15|2|2074|698|9094|Sunday|2074Q2|N|N|N|2478664|2478753|2478313|2478588|N|N|N|N|N| +2478679|AAAAAAAAHFCNFCAA|2074-04-16|2091|9094|698|2074|1|4|16|2|2074|698|9094|Monday|2074Q2|N|N|N|2478664|2478753|2478314|2478589|N|N|N|N|N| +2478680|AAAAAAAAIFCNFCAA|2074-04-17|2091|9095|698|2074|2|4|17|2|2074|698|9095|Tuesday|2074Q2|N|N|N|2478664|2478753|2478315|2478590|N|N|N|N|N| +2478681|AAAAAAAAJFCNFCAA|2074-04-18|2091|9095|698|2074|3|4|18|2|2074|698|9095|Wednesday|2074Q2|N|N|N|2478664|2478753|2478316|2478591|N|N|N|N|N| +2478682|AAAAAAAAKFCNFCAA|2074-04-19|2091|9095|698|2074|4|4|19|2|2074|698|9095|Thursday|2074Q2|N|N|N|2478664|2478753|2478317|2478592|N|N|N|N|N| +2478683|AAAAAAAALFCNFCAA|2074-04-20|2091|9095|698|2074|5|4|20|2|2074|698|9095|Friday|2074Q2|N|Y|N|2478664|2478753|2478318|2478593|N|N|N|N|N| +2478684|AAAAAAAAMFCNFCAA|2074-04-21|2091|9095|698|2074|6|4|21|2|2074|698|9095|Saturday|2074Q2|N|Y|N|2478664|2478753|2478319|2478594|N|N|N|N|N| +2478685|AAAAAAAANFCNFCAA|2074-04-22|2091|9095|698|2074|0|4|22|2|2074|698|9095|Sunday|2074Q2|N|N|N|2478664|2478753|2478320|2478595|N|N|N|N|N| +2478686|AAAAAAAAOFCNFCAA|2074-04-23|2091|9095|698|2074|1|4|23|2|2074|698|9095|Monday|2074Q2|N|N|N|2478664|2478753|2478321|2478596|N|N|N|N|N| +2478687|AAAAAAAAPFCNFCAA|2074-04-24|2091|9096|698|2074|2|4|24|2|2074|698|9096|Tuesday|2074Q2|N|N|N|2478664|2478753|2478322|2478597|N|N|N|N|N| +2478688|AAAAAAAAAGCNFCAA|2074-04-25|2091|9096|698|2074|3|4|25|2|2074|698|9096|Wednesday|2074Q2|N|N|N|2478664|2478753|2478323|2478598|N|N|N|N|N| +2478689|AAAAAAAABGCNFCAA|2074-04-26|2091|9096|698|2074|4|4|26|2|2074|698|9096|Thursday|2074Q2|N|N|N|2478664|2478753|2478324|2478599|N|N|N|N|N| +2478690|AAAAAAAACGCNFCAA|2074-04-27|2091|9096|698|2074|5|4|27|2|2074|698|9096|Friday|2074Q2|N|Y|N|2478664|2478753|2478325|2478600|N|N|N|N|N| +2478691|AAAAAAAADGCNFCAA|2074-04-28|2091|9096|698|2074|6|4|28|2|2074|698|9096|Saturday|2074Q2|N|Y|N|2478664|2478753|2478326|2478601|N|N|N|N|N| +2478692|AAAAAAAAEGCNFCAA|2074-04-29|2091|9096|698|2074|0|4|29|2|2074|698|9096|Sunday|2074Q2|N|N|N|2478664|2478753|2478327|2478602|N|N|N|N|N| +2478693|AAAAAAAAFGCNFCAA|2074-04-30|2091|9096|698|2074|1|4|30|2|2074|698|9096|Monday|2074Q2|N|N|N|2478664|2478753|2478328|2478603|N|N|N|N|N| +2478694|AAAAAAAAGGCNFCAA|2074-05-01|2092|9097|698|2074|2|5|1|2|2074|698|9097|Tuesday|2074Q2|N|N|N|2478694|2478813|2478329|2478604|N|N|N|N|N| +2478695|AAAAAAAAHGCNFCAA|2074-05-02|2092|9097|698|2074|3|5|2|2|2074|698|9097|Wednesday|2074Q2|N|N|N|2478694|2478813|2478330|2478605|N|N|N|N|N| +2478696|AAAAAAAAIGCNFCAA|2074-05-03|2092|9097|698|2074|4|5|3|2|2074|698|9097|Thursday|2074Q2|N|N|N|2478694|2478813|2478331|2478606|N|N|N|N|N| +2478697|AAAAAAAAJGCNFCAA|2074-05-04|2092|9097|698|2074|5|5|4|2|2074|698|9097|Friday|2074Q2|N|Y|N|2478694|2478813|2478332|2478607|N|N|N|N|N| +2478698|AAAAAAAAKGCNFCAA|2074-05-05|2092|9097|698|2074|6|5|5|2|2074|698|9097|Saturday|2074Q2|N|Y|N|2478694|2478813|2478333|2478608|N|N|N|N|N| +2478699|AAAAAAAALGCNFCAA|2074-05-06|2092|9097|698|2074|0|5|6|2|2074|698|9097|Sunday|2074Q2|N|N|N|2478694|2478813|2478334|2478609|N|N|N|N|N| +2478700|AAAAAAAAMGCNFCAA|2074-05-07|2092|9097|698|2074|1|5|7|2|2074|698|9097|Monday|2074Q2|N|N|N|2478694|2478813|2478335|2478610|N|N|N|N|N| +2478701|AAAAAAAANGCNFCAA|2074-05-08|2092|9098|698|2074|2|5|8|2|2074|698|9098|Tuesday|2074Q2|N|N|N|2478694|2478813|2478336|2478611|N|N|N|N|N| +2478702|AAAAAAAAOGCNFCAA|2074-05-09|2092|9098|698|2074|3|5|9|2|2074|698|9098|Wednesday|2074Q2|N|N|N|2478694|2478813|2478337|2478612|N|N|N|N|N| +2478703|AAAAAAAAPGCNFCAA|2074-05-10|2092|9098|698|2074|4|5|10|2|2074|698|9098|Thursday|2074Q2|N|N|N|2478694|2478813|2478338|2478613|N|N|N|N|N| +2478704|AAAAAAAAAHCNFCAA|2074-05-11|2092|9098|698|2074|5|5|11|2|2074|698|9098|Friday|2074Q2|N|Y|N|2478694|2478813|2478339|2478614|N|N|N|N|N| +2478705|AAAAAAAABHCNFCAA|2074-05-12|2092|9098|698|2074|6|5|12|2|2074|698|9098|Saturday|2074Q2|N|Y|N|2478694|2478813|2478340|2478615|N|N|N|N|N| +2478706|AAAAAAAACHCNFCAA|2074-05-13|2092|9098|698|2074|0|5|13|2|2074|698|9098|Sunday|2074Q2|N|N|N|2478694|2478813|2478341|2478616|N|N|N|N|N| +2478707|AAAAAAAADHCNFCAA|2074-05-14|2092|9098|698|2074|1|5|14|2|2074|698|9098|Monday|2074Q2|N|N|N|2478694|2478813|2478342|2478617|N|N|N|N|N| +2478708|AAAAAAAAEHCNFCAA|2074-05-15|2092|9099|698|2074|2|5|15|2|2074|698|9099|Tuesday|2074Q2|N|N|N|2478694|2478813|2478343|2478618|N|N|N|N|N| +2478709|AAAAAAAAFHCNFCAA|2074-05-16|2092|9099|698|2074|3|5|16|2|2074|698|9099|Wednesday|2074Q2|N|N|N|2478694|2478813|2478344|2478619|N|N|N|N|N| +2478710|AAAAAAAAGHCNFCAA|2074-05-17|2092|9099|698|2074|4|5|17|2|2074|698|9099|Thursday|2074Q2|N|N|N|2478694|2478813|2478345|2478620|N|N|N|N|N| +2478711|AAAAAAAAHHCNFCAA|2074-05-18|2092|9099|698|2074|5|5|18|2|2074|698|9099|Friday|2074Q2|N|Y|N|2478694|2478813|2478346|2478621|N|N|N|N|N| +2478712|AAAAAAAAIHCNFCAA|2074-05-19|2092|9099|698|2074|6|5|19|2|2074|698|9099|Saturday|2074Q2|N|Y|N|2478694|2478813|2478347|2478622|N|N|N|N|N| +2478713|AAAAAAAAJHCNFCAA|2074-05-20|2092|9099|698|2074|0|5|20|2|2074|698|9099|Sunday|2074Q2|N|N|N|2478694|2478813|2478348|2478623|N|N|N|N|N| +2478714|AAAAAAAAKHCNFCAA|2074-05-21|2092|9099|698|2074|1|5|21|2|2074|698|9099|Monday|2074Q2|N|N|N|2478694|2478813|2478349|2478624|N|N|N|N|N| +2478715|AAAAAAAALHCNFCAA|2074-05-22|2092|9100|698|2074|2|5|22|2|2074|698|9100|Tuesday|2074Q2|N|N|N|2478694|2478813|2478350|2478625|N|N|N|N|N| +2478716|AAAAAAAAMHCNFCAA|2074-05-23|2092|9100|698|2074|3|5|23|2|2074|698|9100|Wednesday|2074Q2|N|N|N|2478694|2478813|2478351|2478626|N|N|N|N|N| +2478717|AAAAAAAANHCNFCAA|2074-05-24|2092|9100|698|2074|4|5|24|2|2074|698|9100|Thursday|2074Q2|N|N|N|2478694|2478813|2478352|2478627|N|N|N|N|N| +2478718|AAAAAAAAOHCNFCAA|2074-05-25|2092|9100|698|2074|5|5|25|2|2074|698|9100|Friday|2074Q2|N|Y|N|2478694|2478813|2478353|2478628|N|N|N|N|N| +2478719|AAAAAAAAPHCNFCAA|2074-05-26|2092|9100|698|2074|6|5|26|2|2074|698|9100|Saturday|2074Q2|N|Y|N|2478694|2478813|2478354|2478629|N|N|N|N|N| +2478720|AAAAAAAAAICNFCAA|2074-05-27|2092|9100|698|2074|0|5|27|2|2074|698|9100|Sunday|2074Q2|N|N|N|2478694|2478813|2478355|2478630|N|N|N|N|N| +2478721|AAAAAAAABICNFCAA|2074-05-28|2092|9100|698|2074|1|5|28|2|2074|698|9100|Monday|2074Q2|N|N|N|2478694|2478813|2478356|2478631|N|N|N|N|N| +2478722|AAAAAAAACICNFCAA|2074-05-29|2092|9101|698|2074|2|5|29|2|2074|698|9101|Tuesday|2074Q2|N|N|N|2478694|2478813|2478357|2478632|N|N|N|N|N| +2478723|AAAAAAAADICNFCAA|2074-05-30|2092|9101|698|2074|3|5|30|2|2074|698|9101|Wednesday|2074Q2|N|N|N|2478694|2478813|2478358|2478633|N|N|N|N|N| +2478724|AAAAAAAAEICNFCAA|2074-05-31|2092|9101|698|2074|4|5|31|2|2074|698|9101|Thursday|2074Q2|N|N|N|2478694|2478813|2478359|2478634|N|N|N|N|N| +2478725|AAAAAAAAFICNFCAA|2074-06-01|2093|9101|699|2074|5|6|1|2|2074|699|9101|Friday|2074Q2|N|Y|N|2478725|2478875|2478360|2478635|N|N|N|N|N| +2478726|AAAAAAAAGICNFCAA|2074-06-02|2093|9101|699|2074|6|6|2|2|2074|699|9101|Saturday|2074Q2|N|Y|N|2478725|2478875|2478361|2478636|N|N|N|N|N| +2478727|AAAAAAAAHICNFCAA|2074-06-03|2093|9101|699|2074|0|6|3|2|2074|699|9101|Sunday|2074Q2|N|N|N|2478725|2478875|2478362|2478637|N|N|N|N|N| +2478728|AAAAAAAAIICNFCAA|2074-06-04|2093|9101|699|2074|1|6|4|2|2074|699|9101|Monday|2074Q2|N|N|N|2478725|2478875|2478363|2478638|N|N|N|N|N| +2478729|AAAAAAAAJICNFCAA|2074-06-05|2093|9102|699|2074|2|6|5|2|2074|699|9102|Tuesday|2074Q2|N|N|N|2478725|2478875|2478364|2478639|N|N|N|N|N| +2478730|AAAAAAAAKICNFCAA|2074-06-06|2093|9102|699|2074|3|6|6|2|2074|699|9102|Wednesday|2074Q2|N|N|N|2478725|2478875|2478365|2478640|N|N|N|N|N| +2478731|AAAAAAAALICNFCAA|2074-06-07|2093|9102|699|2074|4|6|7|2|2074|699|9102|Thursday|2074Q2|N|N|N|2478725|2478875|2478366|2478641|N|N|N|N|N| +2478732|AAAAAAAAMICNFCAA|2074-06-08|2093|9102|699|2074|5|6|8|2|2074|699|9102|Friday|2074Q2|N|Y|N|2478725|2478875|2478367|2478642|N|N|N|N|N| +2478733|AAAAAAAANICNFCAA|2074-06-09|2093|9102|699|2074|6|6|9|2|2074|699|9102|Saturday|2074Q2|N|Y|N|2478725|2478875|2478368|2478643|N|N|N|N|N| +2478734|AAAAAAAAOICNFCAA|2074-06-10|2093|9102|699|2074|0|6|10|2|2074|699|9102|Sunday|2074Q2|N|N|N|2478725|2478875|2478369|2478644|N|N|N|N|N| +2478735|AAAAAAAAPICNFCAA|2074-06-11|2093|9102|699|2074|1|6|11|2|2074|699|9102|Monday|2074Q2|N|N|N|2478725|2478875|2478370|2478645|N|N|N|N|N| +2478736|AAAAAAAAAJCNFCAA|2074-06-12|2093|9103|699|2074|2|6|12|2|2074|699|9103|Tuesday|2074Q2|N|N|N|2478725|2478875|2478371|2478646|N|N|N|N|N| +2478737|AAAAAAAABJCNFCAA|2074-06-13|2093|9103|699|2074|3|6|13|2|2074|699|9103|Wednesday|2074Q2|N|N|N|2478725|2478875|2478372|2478647|N|N|N|N|N| +2478738|AAAAAAAACJCNFCAA|2074-06-14|2093|9103|699|2074|4|6|14|2|2074|699|9103|Thursday|2074Q2|N|N|N|2478725|2478875|2478373|2478648|N|N|N|N|N| +2478739|AAAAAAAADJCNFCAA|2074-06-15|2093|9103|699|2074|5|6|15|2|2074|699|9103|Friday|2074Q2|N|Y|N|2478725|2478875|2478374|2478649|N|N|N|N|N| +2478740|AAAAAAAAEJCNFCAA|2074-06-16|2093|9103|699|2074|6|6|16|2|2074|699|9103|Saturday|2074Q2|N|Y|N|2478725|2478875|2478375|2478650|N|N|N|N|N| +2478741|AAAAAAAAFJCNFCAA|2074-06-17|2093|9103|699|2074|0|6|17|2|2074|699|9103|Sunday|2074Q2|N|N|N|2478725|2478875|2478376|2478651|N|N|N|N|N| +2478742|AAAAAAAAGJCNFCAA|2074-06-18|2093|9103|699|2074|1|6|18|2|2074|699|9103|Monday|2074Q2|N|N|N|2478725|2478875|2478377|2478652|N|N|N|N|N| +2478743|AAAAAAAAHJCNFCAA|2074-06-19|2093|9104|699|2074|2|6|19|2|2074|699|9104|Tuesday|2074Q2|N|N|N|2478725|2478875|2478378|2478653|N|N|N|N|N| +2478744|AAAAAAAAIJCNFCAA|2074-06-20|2093|9104|699|2074|3|6|20|2|2074|699|9104|Wednesday|2074Q2|N|N|N|2478725|2478875|2478379|2478654|N|N|N|N|N| +2478745|AAAAAAAAJJCNFCAA|2074-06-21|2093|9104|699|2074|4|6|21|2|2074|699|9104|Thursday|2074Q2|N|N|N|2478725|2478875|2478380|2478655|N|N|N|N|N| +2478746|AAAAAAAAKJCNFCAA|2074-06-22|2093|9104|699|2074|5|6|22|2|2074|699|9104|Friday|2074Q2|N|Y|N|2478725|2478875|2478381|2478656|N|N|N|N|N| +2478747|AAAAAAAALJCNFCAA|2074-06-23|2093|9104|699|2074|6|6|23|2|2074|699|9104|Saturday|2074Q2|N|Y|N|2478725|2478875|2478382|2478657|N|N|N|N|N| +2478748|AAAAAAAAMJCNFCAA|2074-06-24|2093|9104|699|2074|0|6|24|2|2074|699|9104|Sunday|2074Q2|N|N|N|2478725|2478875|2478383|2478658|N|N|N|N|N| +2478749|AAAAAAAANJCNFCAA|2074-06-25|2093|9104|699|2074|1|6|25|2|2074|699|9104|Monday|2074Q2|N|N|N|2478725|2478875|2478384|2478659|N|N|N|N|N| +2478750|AAAAAAAAOJCNFCAA|2074-06-26|2093|9105|699|2074|2|6|26|2|2074|699|9105|Tuesday|2074Q2|N|N|N|2478725|2478875|2478385|2478660|N|N|N|N|N| +2478751|AAAAAAAAPJCNFCAA|2074-06-27|2093|9105|699|2074|3|6|27|2|2074|699|9105|Wednesday|2074Q2|N|N|N|2478725|2478875|2478386|2478661|N|N|N|N|N| +2478752|AAAAAAAAAKCNFCAA|2074-06-28|2093|9105|699|2074|4|6|28|2|2074|699|9105|Thursday|2074Q2|N|N|N|2478725|2478875|2478387|2478662|N|N|N|N|N| +2478753|AAAAAAAABKCNFCAA|2074-06-29|2093|9105|699|2074|5|6|29|2|2074|699|9105|Friday|2074Q2|N|Y|N|2478725|2478875|2478388|2478663|N|N|N|N|N| +2478754|AAAAAAAACKCNFCAA|2074-06-30|2093|9105|699|2074|6|6|30|2|2074|699|9105|Saturday|2074Q2|N|Y|N|2478725|2478875|2478389|2478664|N|N|N|N|N| +2478755|AAAAAAAADKCNFCAA|2074-07-01|2094|9105|699|2074|0|7|1|2|2074|699|9105|Sunday|2074Q2|N|N|N|2478755|2478935|2478390|2478664|N|N|N|N|N| +2478756|AAAAAAAAEKCNFCAA|2074-07-02|2094|9105|699|2074|1|7|2|3|2074|699|9105|Monday|2074Q3|N|N|N|2478755|2478935|2478391|2478665|N|N|N|N|N| +2478757|AAAAAAAAFKCNFCAA|2074-07-03|2094|9106|699|2074|2|7|3|3|2074|699|9106|Tuesday|2074Q3|N|N|N|2478755|2478935|2478392|2478666|N|N|N|N|N| +2478758|AAAAAAAAGKCNFCAA|2074-07-04|2094|9106|699|2074|3|7|4|3|2074|699|9106|Wednesday|2074Q3|N|N|N|2478755|2478935|2478393|2478667|N|N|N|N|N| +2478759|AAAAAAAAHKCNFCAA|2074-07-05|2094|9106|699|2074|4|7|5|3|2074|699|9106|Thursday|2074Q3|Y|N|N|2478755|2478935|2478394|2478668|N|N|N|N|N| +2478760|AAAAAAAAIKCNFCAA|2074-07-06|2094|9106|699|2074|5|7|6|3|2074|699|9106|Friday|2074Q3|N|Y|Y|2478755|2478935|2478395|2478669|N|N|N|N|N| +2478761|AAAAAAAAJKCNFCAA|2074-07-07|2094|9106|699|2074|6|7|7|3|2074|699|9106|Saturday|2074Q3|N|Y|N|2478755|2478935|2478396|2478670|N|N|N|N|N| +2478762|AAAAAAAAKKCNFCAA|2074-07-08|2094|9106|699|2074|0|7|8|3|2074|699|9106|Sunday|2074Q3|N|N|N|2478755|2478935|2478397|2478671|N|N|N|N|N| +2478763|AAAAAAAALKCNFCAA|2074-07-09|2094|9106|699|2074|1|7|9|3|2074|699|9106|Monday|2074Q3|N|N|N|2478755|2478935|2478398|2478672|N|N|N|N|N| +2478764|AAAAAAAAMKCNFCAA|2074-07-10|2094|9107|699|2074|2|7|10|3|2074|699|9107|Tuesday|2074Q3|N|N|N|2478755|2478935|2478399|2478673|N|N|N|N|N| +2478765|AAAAAAAANKCNFCAA|2074-07-11|2094|9107|699|2074|3|7|11|3|2074|699|9107|Wednesday|2074Q3|N|N|N|2478755|2478935|2478400|2478674|N|N|N|N|N| +2478766|AAAAAAAAOKCNFCAA|2074-07-12|2094|9107|699|2074|4|7|12|3|2074|699|9107|Thursday|2074Q3|N|N|N|2478755|2478935|2478401|2478675|N|N|N|N|N| +2478767|AAAAAAAAPKCNFCAA|2074-07-13|2094|9107|699|2074|5|7|13|3|2074|699|9107|Friday|2074Q3|N|Y|N|2478755|2478935|2478402|2478676|N|N|N|N|N| +2478768|AAAAAAAAALCNFCAA|2074-07-14|2094|9107|699|2074|6|7|14|3|2074|699|9107|Saturday|2074Q3|N|Y|N|2478755|2478935|2478403|2478677|N|N|N|N|N| +2478769|AAAAAAAABLCNFCAA|2074-07-15|2094|9107|699|2074|0|7|15|3|2074|699|9107|Sunday|2074Q3|N|N|N|2478755|2478935|2478404|2478678|N|N|N|N|N| +2478770|AAAAAAAACLCNFCAA|2074-07-16|2094|9107|699|2074|1|7|16|3|2074|699|9107|Monday|2074Q3|N|N|N|2478755|2478935|2478405|2478679|N|N|N|N|N| +2478771|AAAAAAAADLCNFCAA|2074-07-17|2094|9108|699|2074|2|7|17|3|2074|699|9108|Tuesday|2074Q3|N|N|N|2478755|2478935|2478406|2478680|N|N|N|N|N| +2478772|AAAAAAAAELCNFCAA|2074-07-18|2094|9108|699|2074|3|7|18|3|2074|699|9108|Wednesday|2074Q3|N|N|N|2478755|2478935|2478407|2478681|N|N|N|N|N| +2478773|AAAAAAAAFLCNFCAA|2074-07-19|2094|9108|699|2074|4|7|19|3|2074|699|9108|Thursday|2074Q3|N|N|N|2478755|2478935|2478408|2478682|N|N|N|N|N| +2478774|AAAAAAAAGLCNFCAA|2074-07-20|2094|9108|699|2074|5|7|20|3|2074|699|9108|Friday|2074Q3|N|Y|N|2478755|2478935|2478409|2478683|N|N|N|N|N| +2478775|AAAAAAAAHLCNFCAA|2074-07-21|2094|9108|699|2074|6|7|21|3|2074|699|9108|Saturday|2074Q3|N|Y|N|2478755|2478935|2478410|2478684|N|N|N|N|N| +2478776|AAAAAAAAILCNFCAA|2074-07-22|2094|9108|699|2074|0|7|22|3|2074|699|9108|Sunday|2074Q3|N|N|N|2478755|2478935|2478411|2478685|N|N|N|N|N| +2478777|AAAAAAAAJLCNFCAA|2074-07-23|2094|9108|699|2074|1|7|23|3|2074|699|9108|Monday|2074Q3|N|N|N|2478755|2478935|2478412|2478686|N|N|N|N|N| +2478778|AAAAAAAAKLCNFCAA|2074-07-24|2094|9109|699|2074|2|7|24|3|2074|699|9109|Tuesday|2074Q3|N|N|N|2478755|2478935|2478413|2478687|N|N|N|N|N| +2478779|AAAAAAAALLCNFCAA|2074-07-25|2094|9109|699|2074|3|7|25|3|2074|699|9109|Wednesday|2074Q3|N|N|N|2478755|2478935|2478414|2478688|N|N|N|N|N| +2478780|AAAAAAAAMLCNFCAA|2074-07-26|2094|9109|699|2074|4|7|26|3|2074|699|9109|Thursday|2074Q3|N|N|N|2478755|2478935|2478415|2478689|N|N|N|N|N| +2478781|AAAAAAAANLCNFCAA|2074-07-27|2094|9109|699|2074|5|7|27|3|2074|699|9109|Friday|2074Q3|N|Y|N|2478755|2478935|2478416|2478690|N|N|N|N|N| +2478782|AAAAAAAAOLCNFCAA|2074-07-28|2094|9109|699|2074|6|7|28|3|2074|699|9109|Saturday|2074Q3|N|Y|N|2478755|2478935|2478417|2478691|N|N|N|N|N| +2478783|AAAAAAAAPLCNFCAA|2074-07-29|2094|9109|699|2074|0|7|29|3|2074|699|9109|Sunday|2074Q3|N|N|N|2478755|2478935|2478418|2478692|N|N|N|N|N| +2478784|AAAAAAAAAMCNFCAA|2074-07-30|2094|9109|699|2074|1|7|30|3|2074|699|9109|Monday|2074Q3|N|N|N|2478755|2478935|2478419|2478693|N|N|N|N|N| +2478785|AAAAAAAABMCNFCAA|2074-07-31|2094|9110|699|2074|2|7|31|3|2074|699|9110|Tuesday|2074Q3|N|N|N|2478755|2478935|2478420|2478694|N|N|N|N|N| +2478786|AAAAAAAACMCNFCAA|2074-08-01|2095|9110|699|2074|3|8|1|3|2074|699|9110|Wednesday|2074Q3|N|N|N|2478786|2478997|2478421|2478695|N|N|N|N|N| +2478787|AAAAAAAADMCNFCAA|2074-08-02|2095|9110|699|2074|4|8|2|3|2074|699|9110|Thursday|2074Q3|N|N|N|2478786|2478997|2478422|2478696|N|N|N|N|N| +2478788|AAAAAAAAEMCNFCAA|2074-08-03|2095|9110|699|2074|5|8|3|3|2074|699|9110|Friday|2074Q3|N|Y|N|2478786|2478997|2478423|2478697|N|N|N|N|N| +2478789|AAAAAAAAFMCNFCAA|2074-08-04|2095|9110|699|2074|6|8|4|3|2074|699|9110|Saturday|2074Q3|N|Y|N|2478786|2478997|2478424|2478698|N|N|N|N|N| +2478790|AAAAAAAAGMCNFCAA|2074-08-05|2095|9110|699|2074|0|8|5|3|2074|699|9110|Sunday|2074Q3|N|N|N|2478786|2478997|2478425|2478699|N|N|N|N|N| +2478791|AAAAAAAAHMCNFCAA|2074-08-06|2095|9110|699|2074|1|8|6|3|2074|699|9110|Monday|2074Q3|N|N|N|2478786|2478997|2478426|2478700|N|N|N|N|N| +2478792|AAAAAAAAIMCNFCAA|2074-08-07|2095|9111|699|2074|2|8|7|3|2074|699|9111|Tuesday|2074Q3|N|N|N|2478786|2478997|2478427|2478701|N|N|N|N|N| +2478793|AAAAAAAAJMCNFCAA|2074-08-08|2095|9111|699|2074|3|8|8|3|2074|699|9111|Wednesday|2074Q3|N|N|N|2478786|2478997|2478428|2478702|N|N|N|N|N| +2478794|AAAAAAAAKMCNFCAA|2074-08-09|2095|9111|699|2074|4|8|9|3|2074|699|9111|Thursday|2074Q3|N|N|N|2478786|2478997|2478429|2478703|N|N|N|N|N| +2478795|AAAAAAAALMCNFCAA|2074-08-10|2095|9111|699|2074|5|8|10|3|2074|699|9111|Friday|2074Q3|N|Y|N|2478786|2478997|2478430|2478704|N|N|N|N|N| +2478796|AAAAAAAAMMCNFCAA|2074-08-11|2095|9111|699|2074|6|8|11|3|2074|699|9111|Saturday|2074Q3|N|Y|N|2478786|2478997|2478431|2478705|N|N|N|N|N| +2478797|AAAAAAAANMCNFCAA|2074-08-12|2095|9111|699|2074|0|8|12|3|2074|699|9111|Sunday|2074Q3|N|N|N|2478786|2478997|2478432|2478706|N|N|N|N|N| +2478798|AAAAAAAAOMCNFCAA|2074-08-13|2095|9111|699|2074|1|8|13|3|2074|699|9111|Monday|2074Q3|N|N|N|2478786|2478997|2478433|2478707|N|N|N|N|N| +2478799|AAAAAAAAPMCNFCAA|2074-08-14|2095|9112|699|2074|2|8|14|3|2074|699|9112|Tuesday|2074Q3|N|N|N|2478786|2478997|2478434|2478708|N|N|N|N|N| +2478800|AAAAAAAAANCNFCAA|2074-08-15|2095|9112|699|2074|3|8|15|3|2074|699|9112|Wednesday|2074Q3|N|N|N|2478786|2478997|2478435|2478709|N|N|N|N|N| +2478801|AAAAAAAABNCNFCAA|2074-08-16|2095|9112|699|2074|4|8|16|3|2074|699|9112|Thursday|2074Q3|N|N|N|2478786|2478997|2478436|2478710|N|N|N|N|N| +2478802|AAAAAAAACNCNFCAA|2074-08-17|2095|9112|699|2074|5|8|17|3|2074|699|9112|Friday|2074Q3|N|Y|N|2478786|2478997|2478437|2478711|N|N|N|N|N| +2478803|AAAAAAAADNCNFCAA|2074-08-18|2095|9112|699|2074|6|8|18|3|2074|699|9112|Saturday|2074Q3|N|Y|N|2478786|2478997|2478438|2478712|N|N|N|N|N| +2478804|AAAAAAAAENCNFCAA|2074-08-19|2095|9112|699|2074|0|8|19|3|2074|699|9112|Sunday|2074Q3|N|N|N|2478786|2478997|2478439|2478713|N|N|N|N|N| +2478805|AAAAAAAAFNCNFCAA|2074-08-20|2095|9112|699|2074|1|8|20|3|2074|699|9112|Monday|2074Q3|N|N|N|2478786|2478997|2478440|2478714|N|N|N|N|N| +2478806|AAAAAAAAGNCNFCAA|2074-08-21|2095|9113|699|2074|2|8|21|3|2074|699|9113|Tuesday|2074Q3|N|N|N|2478786|2478997|2478441|2478715|N|N|N|N|N| +2478807|AAAAAAAAHNCNFCAA|2074-08-22|2095|9113|699|2074|3|8|22|3|2074|699|9113|Wednesday|2074Q3|N|N|N|2478786|2478997|2478442|2478716|N|N|N|N|N| +2478808|AAAAAAAAINCNFCAA|2074-08-23|2095|9113|699|2074|4|8|23|3|2074|699|9113|Thursday|2074Q3|N|N|N|2478786|2478997|2478443|2478717|N|N|N|N|N| +2478809|AAAAAAAAJNCNFCAA|2074-08-24|2095|9113|699|2074|5|8|24|3|2074|699|9113|Friday|2074Q3|N|Y|N|2478786|2478997|2478444|2478718|N|N|N|N|N| +2478810|AAAAAAAAKNCNFCAA|2074-08-25|2095|9113|699|2074|6|8|25|3|2074|699|9113|Saturday|2074Q3|N|Y|N|2478786|2478997|2478445|2478719|N|N|N|N|N| +2478811|AAAAAAAALNCNFCAA|2074-08-26|2095|9113|699|2074|0|8|26|3|2074|699|9113|Sunday|2074Q3|N|N|N|2478786|2478997|2478446|2478720|N|N|N|N|N| +2478812|AAAAAAAAMNCNFCAA|2074-08-27|2095|9113|699|2074|1|8|27|3|2074|699|9113|Monday|2074Q3|N|N|N|2478786|2478997|2478447|2478721|N|N|N|N|N| +2478813|AAAAAAAANNCNFCAA|2074-08-28|2095|9114|699|2074|2|8|28|3|2074|699|9114|Tuesday|2074Q3|N|N|N|2478786|2478997|2478448|2478722|N|N|N|N|N| +2478814|AAAAAAAAONCNFCAA|2074-08-29|2095|9114|699|2074|3|8|29|3|2074|699|9114|Wednesday|2074Q3|N|N|N|2478786|2478997|2478449|2478723|N|N|N|N|N| +2478815|AAAAAAAAPNCNFCAA|2074-08-30|2095|9114|699|2074|4|8|30|3|2074|699|9114|Thursday|2074Q3|N|N|N|2478786|2478997|2478450|2478724|N|N|N|N|N| +2478816|AAAAAAAAAOCNFCAA|2074-08-31|2095|9114|699|2074|5|8|31|3|2074|699|9114|Friday|2074Q3|N|Y|N|2478786|2478997|2478451|2478725|N|N|N|N|N| +2478817|AAAAAAAABOCNFCAA|2074-09-01|2096|9114|700|2074|6|9|1|3|2074|700|9114|Saturday|2074Q3|N|Y|N|2478817|2479059|2478452|2478726|N|N|N|N|N| +2478818|AAAAAAAACOCNFCAA|2074-09-02|2096|9114|700|2074|0|9|2|3|2074|700|9114|Sunday|2074Q3|N|N|N|2478817|2479059|2478453|2478727|N|N|N|N|N| +2478819|AAAAAAAADOCNFCAA|2074-09-03|2096|9114|700|2074|1|9|3|3|2074|700|9114|Monday|2074Q3|N|N|N|2478817|2479059|2478454|2478728|N|N|N|N|N| +2478820|AAAAAAAAEOCNFCAA|2074-09-04|2096|9115|700|2074|2|9|4|3|2074|700|9115|Tuesday|2074Q3|N|N|N|2478817|2479059|2478455|2478729|N|N|N|N|N| +2478821|AAAAAAAAFOCNFCAA|2074-09-05|2096|9115|700|2074|3|9|5|3|2074|700|9115|Wednesday|2074Q3|N|N|N|2478817|2479059|2478456|2478730|N|N|N|N|N| +2478822|AAAAAAAAGOCNFCAA|2074-09-06|2096|9115|700|2074|4|9|6|3|2074|700|9115|Thursday|2074Q3|N|N|N|2478817|2479059|2478457|2478731|N|N|N|N|N| +2478823|AAAAAAAAHOCNFCAA|2074-09-07|2096|9115|700|2074|5|9|7|3|2074|700|9115|Friday|2074Q3|N|Y|N|2478817|2479059|2478458|2478732|N|N|N|N|N| +2478824|AAAAAAAAIOCNFCAA|2074-09-08|2096|9115|700|2074|6|9|8|3|2074|700|9115|Saturday|2074Q3|N|Y|N|2478817|2479059|2478459|2478733|N|N|N|N|N| +2478825|AAAAAAAAJOCNFCAA|2074-09-09|2096|9115|700|2074|0|9|9|3|2074|700|9115|Sunday|2074Q3|N|N|N|2478817|2479059|2478460|2478734|N|N|N|N|N| +2478826|AAAAAAAAKOCNFCAA|2074-09-10|2096|9115|700|2074|1|9|10|3|2074|700|9115|Monday|2074Q3|N|N|N|2478817|2479059|2478461|2478735|N|N|N|N|N| +2478827|AAAAAAAALOCNFCAA|2074-09-11|2096|9116|700|2074|2|9|11|3|2074|700|9116|Tuesday|2074Q3|N|N|N|2478817|2479059|2478462|2478736|N|N|N|N|N| +2478828|AAAAAAAAMOCNFCAA|2074-09-12|2096|9116|700|2074|3|9|12|3|2074|700|9116|Wednesday|2074Q3|N|N|N|2478817|2479059|2478463|2478737|N|N|N|N|N| +2478829|AAAAAAAANOCNFCAA|2074-09-13|2096|9116|700|2074|4|9|13|3|2074|700|9116|Thursday|2074Q3|N|N|N|2478817|2479059|2478464|2478738|N|N|N|N|N| +2478830|AAAAAAAAOOCNFCAA|2074-09-14|2096|9116|700|2074|5|9|14|3|2074|700|9116|Friday|2074Q3|N|Y|N|2478817|2479059|2478465|2478739|N|N|N|N|N| +2478831|AAAAAAAAPOCNFCAA|2074-09-15|2096|9116|700|2074|6|9|15|3|2074|700|9116|Saturday|2074Q3|N|Y|N|2478817|2479059|2478466|2478740|N|N|N|N|N| +2478832|AAAAAAAAAPCNFCAA|2074-09-16|2096|9116|700|2074|0|9|16|3|2074|700|9116|Sunday|2074Q3|N|N|N|2478817|2479059|2478467|2478741|N|N|N|N|N| +2478833|AAAAAAAABPCNFCAA|2074-09-17|2096|9116|700|2074|1|9|17|3|2074|700|9116|Monday|2074Q3|N|N|N|2478817|2479059|2478468|2478742|N|N|N|N|N| +2478834|AAAAAAAACPCNFCAA|2074-09-18|2096|9117|700|2074|2|9|18|3|2074|700|9117|Tuesday|2074Q3|N|N|N|2478817|2479059|2478469|2478743|N|N|N|N|N| +2478835|AAAAAAAADPCNFCAA|2074-09-19|2096|9117|700|2074|3|9|19|3|2074|700|9117|Wednesday|2074Q3|N|N|N|2478817|2479059|2478470|2478744|N|N|N|N|N| +2478836|AAAAAAAAEPCNFCAA|2074-09-20|2096|9117|700|2074|4|9|20|3|2074|700|9117|Thursday|2074Q3|N|N|N|2478817|2479059|2478471|2478745|N|N|N|N|N| +2478837|AAAAAAAAFPCNFCAA|2074-09-21|2096|9117|700|2074|5|9|21|3|2074|700|9117|Friday|2074Q3|N|Y|N|2478817|2479059|2478472|2478746|N|N|N|N|N| +2478838|AAAAAAAAGPCNFCAA|2074-09-22|2096|9117|700|2074|6|9|22|3|2074|700|9117|Saturday|2074Q3|N|Y|N|2478817|2479059|2478473|2478747|N|N|N|N|N| +2478839|AAAAAAAAHPCNFCAA|2074-09-23|2096|9117|700|2074|0|9|23|3|2074|700|9117|Sunday|2074Q3|N|N|N|2478817|2479059|2478474|2478748|N|N|N|N|N| +2478840|AAAAAAAAIPCNFCAA|2074-09-24|2096|9117|700|2074|1|9|24|3|2074|700|9117|Monday|2074Q3|N|N|N|2478817|2479059|2478475|2478749|N|N|N|N|N| +2478841|AAAAAAAAJPCNFCAA|2074-09-25|2096|9118|700|2074|2|9|25|3|2074|700|9118|Tuesday|2074Q3|N|N|N|2478817|2479059|2478476|2478750|N|N|N|N|N| +2478842|AAAAAAAAKPCNFCAA|2074-09-26|2096|9118|700|2074|3|9|26|3|2074|700|9118|Wednesday|2074Q3|N|N|N|2478817|2479059|2478477|2478751|N|N|N|N|N| +2478843|AAAAAAAALPCNFCAA|2074-09-27|2096|9118|700|2074|4|9|27|3|2074|700|9118|Thursday|2074Q3|N|N|N|2478817|2479059|2478478|2478752|N|N|N|N|N| +2478844|AAAAAAAAMPCNFCAA|2074-09-28|2096|9118|700|2074|5|9|28|3|2074|700|9118|Friday|2074Q3|N|Y|N|2478817|2479059|2478479|2478753|N|N|N|N|N| +2478845|AAAAAAAANPCNFCAA|2074-09-29|2096|9118|700|2074|6|9|29|3|2074|700|9118|Saturday|2074Q3|N|Y|N|2478817|2479059|2478480|2478754|N|N|N|N|N| +2478846|AAAAAAAAOPCNFCAA|2074-09-30|2096|9118|700|2074|0|9|30|3|2074|700|9118|Sunday|2074Q3|N|N|N|2478817|2479059|2478481|2478755|N|N|N|N|N| +2478847|AAAAAAAAPPCNFCAA|2074-10-01|2097|9118|700|2074|1|10|1|3|2074|700|9118|Monday|2074Q3|N|N|N|2478847|2479119|2478482|2478755|N|N|N|N|N| +2478848|AAAAAAAAAADNFCAA|2074-10-02|2097|9119|700|2074|2|10|2|4|2074|700|9119|Tuesday|2074Q4|N|N|N|2478847|2479119|2478483|2478756|N|N|N|N|N| +2478849|AAAAAAAABADNFCAA|2074-10-03|2097|9119|700|2074|3|10|3|4|2074|700|9119|Wednesday|2074Q4|N|N|N|2478847|2479119|2478484|2478757|N|N|N|N|N| +2478850|AAAAAAAACADNFCAA|2074-10-04|2097|9119|700|2074|4|10|4|4|2074|700|9119|Thursday|2074Q4|N|N|N|2478847|2479119|2478485|2478758|N|N|N|N|N| +2478851|AAAAAAAADADNFCAA|2074-10-05|2097|9119|700|2074|5|10|5|4|2074|700|9119|Friday|2074Q4|N|Y|N|2478847|2479119|2478486|2478759|N|N|N|N|N| +2478852|AAAAAAAAEADNFCAA|2074-10-06|2097|9119|700|2074|6|10|6|4|2074|700|9119|Saturday|2074Q4|N|Y|N|2478847|2479119|2478487|2478760|N|N|N|N|N| +2478853|AAAAAAAAFADNFCAA|2074-10-07|2097|9119|700|2074|0|10|7|4|2074|700|9119|Sunday|2074Q4|N|N|N|2478847|2479119|2478488|2478761|N|N|N|N|N| +2478854|AAAAAAAAGADNFCAA|2074-10-08|2097|9119|700|2074|1|10|8|4|2074|700|9119|Monday|2074Q4|N|N|N|2478847|2479119|2478489|2478762|N|N|N|N|N| +2478855|AAAAAAAAHADNFCAA|2074-10-09|2097|9120|700|2074|2|10|9|4|2074|700|9120|Tuesday|2074Q4|N|N|N|2478847|2479119|2478490|2478763|N|N|N|N|N| +2478856|AAAAAAAAIADNFCAA|2074-10-10|2097|9120|700|2074|3|10|10|4|2074|700|9120|Wednesday|2074Q4|N|N|N|2478847|2479119|2478491|2478764|N|N|N|N|N| +2478857|AAAAAAAAJADNFCAA|2074-10-11|2097|9120|700|2074|4|10|11|4|2074|700|9120|Thursday|2074Q4|N|N|N|2478847|2479119|2478492|2478765|N|N|N|N|N| +2478858|AAAAAAAAKADNFCAA|2074-10-12|2097|9120|700|2074|5|10|12|4|2074|700|9120|Friday|2074Q4|N|Y|N|2478847|2479119|2478493|2478766|N|N|N|N|N| +2478859|AAAAAAAALADNFCAA|2074-10-13|2097|9120|700|2074|6|10|13|4|2074|700|9120|Saturday|2074Q4|N|Y|N|2478847|2479119|2478494|2478767|N|N|N|N|N| +2478860|AAAAAAAAMADNFCAA|2074-10-14|2097|9120|700|2074|0|10|14|4|2074|700|9120|Sunday|2074Q4|N|N|N|2478847|2479119|2478495|2478768|N|N|N|N|N| +2478861|AAAAAAAANADNFCAA|2074-10-15|2097|9120|700|2074|1|10|15|4|2074|700|9120|Monday|2074Q4|N|N|N|2478847|2479119|2478496|2478769|N|N|N|N|N| +2478862|AAAAAAAAOADNFCAA|2074-10-16|2097|9121|700|2074|2|10|16|4|2074|700|9121|Tuesday|2074Q4|N|N|N|2478847|2479119|2478497|2478770|N|N|N|N|N| +2478863|AAAAAAAAPADNFCAA|2074-10-17|2097|9121|700|2074|3|10|17|4|2074|700|9121|Wednesday|2074Q4|N|N|N|2478847|2479119|2478498|2478771|N|N|N|N|N| +2478864|AAAAAAAAABDNFCAA|2074-10-18|2097|9121|700|2074|4|10|18|4|2074|700|9121|Thursday|2074Q4|N|N|N|2478847|2479119|2478499|2478772|N|N|N|N|N| +2478865|AAAAAAAABBDNFCAA|2074-10-19|2097|9121|700|2074|5|10|19|4|2074|700|9121|Friday|2074Q4|N|Y|N|2478847|2479119|2478500|2478773|N|N|N|N|N| +2478866|AAAAAAAACBDNFCAA|2074-10-20|2097|9121|700|2074|6|10|20|4|2074|700|9121|Saturday|2074Q4|N|Y|N|2478847|2479119|2478501|2478774|N|N|N|N|N| +2478867|AAAAAAAADBDNFCAA|2074-10-21|2097|9121|700|2074|0|10|21|4|2074|700|9121|Sunday|2074Q4|N|N|N|2478847|2479119|2478502|2478775|N|N|N|N|N| +2478868|AAAAAAAAEBDNFCAA|2074-10-22|2097|9121|700|2074|1|10|22|4|2074|700|9121|Monday|2074Q4|N|N|N|2478847|2479119|2478503|2478776|N|N|N|N|N| +2478869|AAAAAAAAFBDNFCAA|2074-10-23|2097|9122|700|2074|2|10|23|4|2074|700|9122|Tuesday|2074Q4|N|N|N|2478847|2479119|2478504|2478777|N|N|N|N|N| +2478870|AAAAAAAAGBDNFCAA|2074-10-24|2097|9122|700|2074|3|10|24|4|2074|700|9122|Wednesday|2074Q4|N|N|N|2478847|2479119|2478505|2478778|N|N|N|N|N| +2478871|AAAAAAAAHBDNFCAA|2074-10-25|2097|9122|700|2074|4|10|25|4|2074|700|9122|Thursday|2074Q4|N|N|N|2478847|2479119|2478506|2478779|N|N|N|N|N| +2478872|AAAAAAAAIBDNFCAA|2074-10-26|2097|9122|700|2074|5|10|26|4|2074|700|9122|Friday|2074Q4|N|Y|N|2478847|2479119|2478507|2478780|N|N|N|N|N| +2478873|AAAAAAAAJBDNFCAA|2074-10-27|2097|9122|700|2074|6|10|27|4|2074|700|9122|Saturday|2074Q4|N|Y|N|2478847|2479119|2478508|2478781|N|N|N|N|N| +2478874|AAAAAAAAKBDNFCAA|2074-10-28|2097|9122|700|2074|0|10|28|4|2074|700|9122|Sunday|2074Q4|N|N|N|2478847|2479119|2478509|2478782|N|N|N|N|N| +2478875|AAAAAAAALBDNFCAA|2074-10-29|2097|9122|700|2074|1|10|29|4|2074|700|9122|Monday|2074Q4|N|N|N|2478847|2479119|2478510|2478783|N|N|N|N|N| +2478876|AAAAAAAAMBDNFCAA|2074-10-30|2097|9123|700|2074|2|10|30|4|2074|700|9123|Tuesday|2074Q4|N|N|N|2478847|2479119|2478511|2478784|N|N|N|N|N| +2478877|AAAAAAAANBDNFCAA|2074-10-31|2097|9123|700|2074|3|10|31|4|2074|700|9123|Wednesday|2074Q4|N|N|N|2478847|2479119|2478512|2478785|N|N|N|N|N| +2478878|AAAAAAAAOBDNFCAA|2074-11-01|2098|9123|700|2074|4|11|1|4|2074|700|9123|Thursday|2074Q4|N|N|N|2478878|2479181|2478513|2478786|N|N|N|N|N| +2478879|AAAAAAAAPBDNFCAA|2074-11-02|2098|9123|700|2074|5|11|2|4|2074|700|9123|Friday|2074Q4|N|Y|N|2478878|2479181|2478514|2478787|N|N|N|N|N| +2478880|AAAAAAAAACDNFCAA|2074-11-03|2098|9123|700|2074|6|11|3|4|2074|700|9123|Saturday|2074Q4|N|Y|N|2478878|2479181|2478515|2478788|N|N|N|N|N| +2478881|AAAAAAAABCDNFCAA|2074-11-04|2098|9123|700|2074|0|11|4|4|2074|700|9123|Sunday|2074Q4|N|N|N|2478878|2479181|2478516|2478789|N|N|N|N|N| +2478882|AAAAAAAACCDNFCAA|2074-11-05|2098|9123|700|2074|1|11|5|4|2074|700|9123|Monday|2074Q4|N|N|N|2478878|2479181|2478517|2478790|N|N|N|N|N| +2478883|AAAAAAAADCDNFCAA|2074-11-06|2098|9124|700|2074|2|11|6|4|2074|700|9124|Tuesday|2074Q4|N|N|N|2478878|2479181|2478518|2478791|N|N|N|N|N| +2478884|AAAAAAAAECDNFCAA|2074-11-07|2098|9124|700|2074|3|11|7|4|2074|700|9124|Wednesday|2074Q4|N|N|N|2478878|2479181|2478519|2478792|N|N|N|N|N| +2478885|AAAAAAAAFCDNFCAA|2074-11-08|2098|9124|700|2074|4|11|8|4|2074|700|9124|Thursday|2074Q4|N|N|N|2478878|2479181|2478520|2478793|N|N|N|N|N| +2478886|AAAAAAAAGCDNFCAA|2074-11-09|2098|9124|700|2074|5|11|9|4|2074|700|9124|Friday|2074Q4|N|Y|N|2478878|2479181|2478521|2478794|N|N|N|N|N| +2478887|AAAAAAAAHCDNFCAA|2074-11-10|2098|9124|700|2074|6|11|10|4|2074|700|9124|Saturday|2074Q4|N|Y|N|2478878|2479181|2478522|2478795|N|N|N|N|N| +2478888|AAAAAAAAICDNFCAA|2074-11-11|2098|9124|700|2074|0|11|11|4|2074|700|9124|Sunday|2074Q4|N|N|N|2478878|2479181|2478523|2478796|N|N|N|N|N| +2478889|AAAAAAAAJCDNFCAA|2074-11-12|2098|9124|700|2074|1|11|12|4|2074|700|9124|Monday|2074Q4|N|N|N|2478878|2479181|2478524|2478797|N|N|N|N|N| +2478890|AAAAAAAAKCDNFCAA|2074-11-13|2098|9125|700|2074|2|11|13|4|2074|700|9125|Tuesday|2074Q4|N|N|N|2478878|2479181|2478525|2478798|N|N|N|N|N| +2478891|AAAAAAAALCDNFCAA|2074-11-14|2098|9125|700|2074|3|11|14|4|2074|700|9125|Wednesday|2074Q4|N|N|N|2478878|2479181|2478526|2478799|N|N|N|N|N| +2478892|AAAAAAAAMCDNFCAA|2074-11-15|2098|9125|700|2074|4|11|15|4|2074|700|9125|Thursday|2074Q4|N|N|N|2478878|2479181|2478527|2478800|N|N|N|N|N| +2478893|AAAAAAAANCDNFCAA|2074-11-16|2098|9125|700|2074|5|11|16|4|2074|700|9125|Friday|2074Q4|N|Y|N|2478878|2479181|2478528|2478801|N|N|N|N|N| +2478894|AAAAAAAAOCDNFCAA|2074-11-17|2098|9125|700|2074|6|11|17|4|2074|700|9125|Saturday|2074Q4|N|Y|N|2478878|2479181|2478529|2478802|N|N|N|N|N| +2478895|AAAAAAAAPCDNFCAA|2074-11-18|2098|9125|700|2074|0|11|18|4|2074|700|9125|Sunday|2074Q4|N|N|N|2478878|2479181|2478530|2478803|N|N|N|N|N| +2478896|AAAAAAAAADDNFCAA|2074-11-19|2098|9125|700|2074|1|11|19|4|2074|700|9125|Monday|2074Q4|N|N|N|2478878|2479181|2478531|2478804|N|N|N|N|N| +2478897|AAAAAAAABDDNFCAA|2074-11-20|2098|9126|700|2074|2|11|20|4|2074|700|9126|Tuesday|2074Q4|N|N|N|2478878|2479181|2478532|2478805|N|N|N|N|N| +2478898|AAAAAAAACDDNFCAA|2074-11-21|2098|9126|700|2074|3|11|21|4|2074|700|9126|Wednesday|2074Q4|N|N|N|2478878|2479181|2478533|2478806|N|N|N|N|N| +2478899|AAAAAAAADDDNFCAA|2074-11-22|2098|9126|700|2074|4|11|22|4|2074|700|9126|Thursday|2074Q4|N|N|N|2478878|2479181|2478534|2478807|N|N|N|N|N| +2478900|AAAAAAAAEDDNFCAA|2074-11-23|2098|9126|700|2074|5|11|23|4|2074|700|9126|Friday|2074Q4|N|Y|N|2478878|2479181|2478535|2478808|N|N|N|N|N| +2478901|AAAAAAAAFDDNFCAA|2074-11-24|2098|9126|700|2074|6|11|24|4|2074|700|9126|Saturday|2074Q4|N|Y|N|2478878|2479181|2478536|2478809|N|N|N|N|N| +2478902|AAAAAAAAGDDNFCAA|2074-11-25|2098|9126|700|2074|0|11|25|4|2074|700|9126|Sunday|2074Q4|N|N|N|2478878|2479181|2478537|2478810|N|N|N|N|N| +2478903|AAAAAAAAHDDNFCAA|2074-11-26|2098|9126|700|2074|1|11|26|4|2074|700|9126|Monday|2074Q4|N|N|N|2478878|2479181|2478538|2478811|N|N|N|N|N| +2478904|AAAAAAAAIDDNFCAA|2074-11-27|2098|9127|700|2074|2|11|27|4|2074|700|9127|Tuesday|2074Q4|N|N|N|2478878|2479181|2478539|2478812|N|N|N|N|N| +2478905|AAAAAAAAJDDNFCAA|2074-11-28|2098|9127|700|2074|3|11|28|4|2074|700|9127|Wednesday|2074Q4|N|N|N|2478878|2479181|2478540|2478813|N|N|N|N|N| +2478906|AAAAAAAAKDDNFCAA|2074-11-29|2098|9127|700|2074|4|11|29|4|2074|700|9127|Thursday|2074Q4|N|N|N|2478878|2479181|2478541|2478814|N|N|N|N|N| +2478907|AAAAAAAALDDNFCAA|2074-11-30|2098|9127|700|2074|5|11|30|4|2074|700|9127|Friday|2074Q4|N|Y|N|2478878|2479181|2478542|2478815|N|N|N|N|N| +2478908|AAAAAAAAMDDNFCAA|2074-12-01|2099|9127|701|2074|6|12|1|4|2074|701|9127|Saturday|2074Q4|N|Y|N|2478908|2479241|2478543|2478816|N|N|N|N|N| +2478909|AAAAAAAANDDNFCAA|2074-12-02|2099|9127|701|2074|0|12|2|4|2074|701|9127|Sunday|2074Q4|N|N|N|2478908|2479241|2478544|2478817|N|N|N|N|N| +2478910|AAAAAAAAODDNFCAA|2074-12-03|2099|9127|701|2074|1|12|3|4|2074|701|9127|Monday|2074Q4|N|N|N|2478908|2479241|2478545|2478818|N|N|N|N|N| +2478911|AAAAAAAAPDDNFCAA|2074-12-04|2099|9128|701|2074|2|12|4|4|2074|701|9128|Tuesday|2074Q4|N|N|N|2478908|2479241|2478546|2478819|N|N|N|N|N| +2478912|AAAAAAAAAEDNFCAA|2074-12-05|2099|9128|701|2074|3|12|5|4|2074|701|9128|Wednesday|2074Q4|N|N|N|2478908|2479241|2478547|2478820|N|N|N|N|N| +2478913|AAAAAAAABEDNFCAA|2074-12-06|2099|9128|701|2074|4|12|6|4|2074|701|9128|Thursday|2074Q4|N|N|N|2478908|2479241|2478548|2478821|N|N|N|N|N| +2478914|AAAAAAAACEDNFCAA|2074-12-07|2099|9128|701|2074|5|12|7|4|2074|701|9128|Friday|2074Q4|N|Y|N|2478908|2479241|2478549|2478822|N|N|N|N|N| +2478915|AAAAAAAADEDNFCAA|2074-12-08|2099|9128|701|2074|6|12|8|4|2074|701|9128|Saturday|2074Q4|N|Y|N|2478908|2479241|2478550|2478823|N|N|N|N|N| +2478916|AAAAAAAAEEDNFCAA|2074-12-09|2099|9128|701|2074|0|12|9|4|2074|701|9128|Sunday|2074Q4|N|N|N|2478908|2479241|2478551|2478824|N|N|N|N|N| +2478917|AAAAAAAAFEDNFCAA|2074-12-10|2099|9128|701|2074|1|12|10|4|2074|701|9128|Monday|2074Q4|N|N|N|2478908|2479241|2478552|2478825|N|N|N|N|N| +2478918|AAAAAAAAGEDNFCAA|2074-12-11|2099|9129|701|2074|2|12|11|4|2074|701|9129|Tuesday|2074Q4|N|N|N|2478908|2479241|2478553|2478826|N|N|N|N|N| +2478919|AAAAAAAAHEDNFCAA|2074-12-12|2099|9129|701|2074|3|12|12|4|2074|701|9129|Wednesday|2074Q4|N|N|N|2478908|2479241|2478554|2478827|N|N|N|N|N| +2478920|AAAAAAAAIEDNFCAA|2074-12-13|2099|9129|701|2074|4|12|13|4|2074|701|9129|Thursday|2074Q4|N|N|N|2478908|2479241|2478555|2478828|N|N|N|N|N| +2478921|AAAAAAAAJEDNFCAA|2074-12-14|2099|9129|701|2074|5|12|14|4|2074|701|9129|Friday|2074Q4|N|Y|N|2478908|2479241|2478556|2478829|N|N|N|N|N| +2478922|AAAAAAAAKEDNFCAA|2074-12-15|2099|9129|701|2074|6|12|15|4|2074|701|9129|Saturday|2074Q4|N|Y|N|2478908|2479241|2478557|2478830|N|N|N|N|N| +2478923|AAAAAAAALEDNFCAA|2074-12-16|2099|9129|701|2074|0|12|16|4|2074|701|9129|Sunday|2074Q4|N|N|N|2478908|2479241|2478558|2478831|N|N|N|N|N| +2478924|AAAAAAAAMEDNFCAA|2074-12-17|2099|9129|701|2074|1|12|17|4|2074|701|9129|Monday|2074Q4|N|N|N|2478908|2479241|2478559|2478832|N|N|N|N|N| +2478925|AAAAAAAANEDNFCAA|2074-12-18|2099|9130|701|2074|2|12|18|4|2074|701|9130|Tuesday|2074Q4|N|N|N|2478908|2479241|2478560|2478833|N|N|N|N|N| +2478926|AAAAAAAAOEDNFCAA|2074-12-19|2099|9130|701|2074|3|12|19|4|2074|701|9130|Wednesday|2074Q4|N|N|N|2478908|2479241|2478561|2478834|N|N|N|N|N| +2478927|AAAAAAAAPEDNFCAA|2074-12-20|2099|9130|701|2074|4|12|20|4|2074|701|9130|Thursday|2074Q4|N|N|N|2478908|2479241|2478562|2478835|N|N|N|N|N| +2478928|AAAAAAAAAFDNFCAA|2074-12-21|2099|9130|701|2074|5|12|21|4|2074|701|9130|Friday|2074Q4|N|Y|N|2478908|2479241|2478563|2478836|N|N|N|N|N| +2478929|AAAAAAAABFDNFCAA|2074-12-22|2099|9130|701|2074|6|12|22|4|2074|701|9130|Saturday|2074Q4|N|Y|N|2478908|2479241|2478564|2478837|N|N|N|N|N| +2478930|AAAAAAAACFDNFCAA|2074-12-23|2099|9130|701|2074|0|12|23|4|2074|701|9130|Sunday|2074Q4|N|N|N|2478908|2479241|2478565|2478838|N|N|N|N|N| +2478931|AAAAAAAADFDNFCAA|2074-12-24|2099|9130|701|2074|1|12|24|4|2074|701|9130|Monday|2074Q4|N|N|N|2478908|2479241|2478566|2478839|N|N|N|N|N| +2478932|AAAAAAAAEFDNFCAA|2074-12-25|2099|9131|701|2074|2|12|25|4|2074|701|9131|Tuesday|2074Q4|N|N|N|2478908|2479241|2478567|2478840|N|N|N|N|N| +2478933|AAAAAAAAFFDNFCAA|2074-12-26|2099|9131|701|2074|3|12|26|4|2074|701|9131|Wednesday|2074Q4|Y|N|N|2478908|2479241|2478568|2478841|N|N|N|N|N| +2478934|AAAAAAAAGFDNFCAA|2074-12-27|2099|9131|701|2074|4|12|27|4|2074|701|9131|Thursday|2074Q4|N|N|Y|2478908|2479241|2478569|2478842|N|N|N|N|N| +2478935|AAAAAAAAHFDNFCAA|2074-12-28|2099|9131|701|2074|5|12|28|4|2074|701|9131|Friday|2074Q4|N|Y|N|2478908|2479241|2478570|2478843|N|N|N|N|N| +2478936|AAAAAAAAIFDNFCAA|2074-12-29|2099|9131|701|2074|6|12|29|4|2074|701|9131|Saturday|2074Q4|N|Y|N|2478908|2479241|2478571|2478844|N|N|N|N|N| +2478937|AAAAAAAAJFDNFCAA|2074-12-30|2099|9131|701|2074|0|12|30|4|2074|701|9131|Sunday|2074Q4|N|N|N|2478908|2479241|2478572|2478845|N|N|N|N|N| +2478938|AAAAAAAAKFDNFCAA|2074-12-31|2099|9131|701|2074|1|12|31|4|2074|701|9131|Monday|2074Q4|N|N|N|2478908|2479241|2478573|2478846|N|N|N|N|N| +2478939|AAAAAAAALFDNFCAA|2075-01-01|2100|9132|701|2075|2|1|1|1|2075|701|9132|Tuesday|2075Q1|Y|N|N|2478939|2478938|2478574|2478847|N|N|N|N|N| +2478940|AAAAAAAAMFDNFCAA|2075-01-02|2100|9132|701|2075|3|1|2|1|2075|701|9132|Wednesday|2075Q1|N|N|Y|2478939|2478938|2478575|2478848|N|N|N|N|N| +2478941|AAAAAAAANFDNFCAA|2075-01-03|2100|9132|701|2075|4|1|3|1|2075|701|9132|Thursday|2075Q1|N|N|N|2478939|2478938|2478576|2478849|N|N|N|N|N| +2478942|AAAAAAAAOFDNFCAA|2075-01-04|2100|9132|701|2075|5|1|4|1|2075|701|9132|Friday|2075Q1|N|Y|N|2478939|2478938|2478577|2478850|N|N|N|N|N| +2478943|AAAAAAAAPFDNFCAA|2075-01-05|2100|9132|701|2075|6|1|5|1|2075|701|9132|Saturday|2075Q1|N|Y|N|2478939|2478938|2478578|2478851|N|N|N|N|N| +2478944|AAAAAAAAAGDNFCAA|2075-01-06|2100|9132|701|2075|0|1|6|1|2075|701|9132|Sunday|2075Q1|N|N|N|2478939|2478938|2478579|2478852|N|N|N|N|N| +2478945|AAAAAAAABGDNFCAA|2075-01-07|2100|9132|701|2075|1|1|7|1|2075|701|9132|Monday|2075Q1|N|N|N|2478939|2478938|2478580|2478853|N|N|N|N|N| +2478946|AAAAAAAACGDNFCAA|2075-01-08|2100|9133|701|2075|2|1|8|1|2075|701|9133|Tuesday|2075Q1|N|N|N|2478939|2478938|2478581|2478854|N|N|N|N|N| +2478947|AAAAAAAADGDNFCAA|2075-01-09|2100|9133|701|2075|3|1|9|1|2075|701|9133|Wednesday|2075Q1|N|N|N|2478939|2478938|2478582|2478855|N|N|N|N|N| +2478948|AAAAAAAAEGDNFCAA|2075-01-10|2100|9133|701|2075|4|1|10|1|2075|701|9133|Thursday|2075Q1|N|N|N|2478939|2478938|2478583|2478856|N|N|N|N|N| +2478949|AAAAAAAAFGDNFCAA|2075-01-11|2100|9133|701|2075|5|1|11|1|2075|701|9133|Friday|2075Q1|N|Y|N|2478939|2478938|2478584|2478857|N|N|N|N|N| +2478950|AAAAAAAAGGDNFCAA|2075-01-12|2100|9133|701|2075|6|1|12|1|2075|701|9133|Saturday|2075Q1|N|Y|N|2478939|2478938|2478585|2478858|N|N|N|N|N| +2478951|AAAAAAAAHGDNFCAA|2075-01-13|2100|9133|701|2075|0|1|13|1|2075|701|9133|Sunday|2075Q1|N|N|N|2478939|2478938|2478586|2478859|N|N|N|N|N| +2478952|AAAAAAAAIGDNFCAA|2075-01-14|2100|9133|701|2075|1|1|14|1|2075|701|9133|Monday|2075Q1|N|N|N|2478939|2478938|2478587|2478860|N|N|N|N|N| +2478953|AAAAAAAAJGDNFCAA|2075-01-15|2100|9134|701|2075|2|1|15|1|2075|701|9134|Tuesday|2075Q1|N|N|N|2478939|2478938|2478588|2478861|N|N|N|N|N| +2478954|AAAAAAAAKGDNFCAA|2075-01-16|2100|9134|701|2075|3|1|16|1|2075|701|9134|Wednesday|2075Q1|N|N|N|2478939|2478938|2478589|2478862|N|N|N|N|N| +2478955|AAAAAAAALGDNFCAA|2075-01-17|2100|9134|701|2075|4|1|17|1|2075|701|9134|Thursday|2075Q1|N|N|N|2478939|2478938|2478590|2478863|N|N|N|N|N| +2478956|AAAAAAAAMGDNFCAA|2075-01-18|2100|9134|701|2075|5|1|18|1|2075|701|9134|Friday|2075Q1|N|Y|N|2478939|2478938|2478591|2478864|N|N|N|N|N| +2478957|AAAAAAAANGDNFCAA|2075-01-19|2100|9134|701|2075|6|1|19|1|2075|701|9134|Saturday|2075Q1|N|Y|N|2478939|2478938|2478592|2478865|N|N|N|N|N| +2478958|AAAAAAAAOGDNFCAA|2075-01-20|2100|9134|701|2075|0|1|20|1|2075|701|9134|Sunday|2075Q1|N|N|N|2478939|2478938|2478593|2478866|N|N|N|N|N| +2478959|AAAAAAAAPGDNFCAA|2075-01-21|2100|9134|701|2075|1|1|21|1|2075|701|9134|Monday|2075Q1|N|N|N|2478939|2478938|2478594|2478867|N|N|N|N|N| +2478960|AAAAAAAAAHDNFCAA|2075-01-22|2100|9135|701|2075|2|1|22|1|2075|701|9135|Tuesday|2075Q1|N|N|N|2478939|2478938|2478595|2478868|N|N|N|N|N| +2478961|AAAAAAAABHDNFCAA|2075-01-23|2100|9135|701|2075|3|1|23|1|2075|701|9135|Wednesday|2075Q1|N|N|N|2478939|2478938|2478596|2478869|N|N|N|N|N| +2478962|AAAAAAAACHDNFCAA|2075-01-24|2100|9135|701|2075|4|1|24|1|2075|701|9135|Thursday|2075Q1|N|N|N|2478939|2478938|2478597|2478870|N|N|N|N|N| +2478963|AAAAAAAADHDNFCAA|2075-01-25|2100|9135|701|2075|5|1|25|1|2075|701|9135|Friday|2075Q1|N|Y|N|2478939|2478938|2478598|2478871|N|N|N|N|N| +2478964|AAAAAAAAEHDNFCAA|2075-01-26|2100|9135|701|2075|6|1|26|1|2075|701|9135|Saturday|2075Q1|N|Y|N|2478939|2478938|2478599|2478872|N|N|N|N|N| +2478965|AAAAAAAAFHDNFCAA|2075-01-27|2100|9135|701|2075|0|1|27|1|2075|701|9135|Sunday|2075Q1|N|N|N|2478939|2478938|2478600|2478873|N|N|N|N|N| +2478966|AAAAAAAAGHDNFCAA|2075-01-28|2100|9135|701|2075|1|1|28|1|2075|701|9135|Monday|2075Q1|N|N|N|2478939|2478938|2478601|2478874|N|N|N|N|N| +2478967|AAAAAAAAHHDNFCAA|2075-01-29|2100|9136|701|2075|2|1|29|1|2075|701|9136|Tuesday|2075Q1|N|N|N|2478939|2478938|2478602|2478875|N|N|N|N|N| +2478968|AAAAAAAAIHDNFCAA|2075-01-30|2100|9136|701|2075|3|1|30|1|2075|701|9136|Wednesday|2075Q1|N|N|N|2478939|2478938|2478603|2478876|N|N|N|N|N| +2478969|AAAAAAAAJHDNFCAA|2075-01-31|2100|9136|701|2075|4|1|31|1|2075|701|9136|Thursday|2075Q1|N|N|N|2478939|2478938|2478604|2478877|N|N|N|N|N| +2478970|AAAAAAAAKHDNFCAA|2075-02-01|2101|9136|701|2075|5|2|1|1|2075|701|9136|Friday|2075Q1|N|Y|N|2478970|2479000|2478605|2478878|N|N|N|N|N| +2478971|AAAAAAAALHDNFCAA|2075-02-02|2101|9136|701|2075|6|2|2|1|2075|701|9136|Saturday|2075Q1|N|Y|N|2478970|2479000|2478606|2478879|N|N|N|N|N| +2478972|AAAAAAAAMHDNFCAA|2075-02-03|2101|9136|701|2075|0|2|3|1|2075|701|9136|Sunday|2075Q1|N|N|N|2478970|2479000|2478607|2478880|N|N|N|N|N| +2478973|AAAAAAAANHDNFCAA|2075-02-04|2101|9136|701|2075|1|2|4|1|2075|701|9136|Monday|2075Q1|N|N|N|2478970|2479000|2478608|2478881|N|N|N|N|N| +2478974|AAAAAAAAOHDNFCAA|2075-02-05|2101|9137|701|2075|2|2|5|1|2075|701|9137|Tuesday|2075Q1|N|N|N|2478970|2479000|2478609|2478882|N|N|N|N|N| +2478975|AAAAAAAAPHDNFCAA|2075-02-06|2101|9137|701|2075|3|2|6|1|2075|701|9137|Wednesday|2075Q1|N|N|N|2478970|2479000|2478610|2478883|N|N|N|N|N| +2478976|AAAAAAAAAIDNFCAA|2075-02-07|2101|9137|701|2075|4|2|7|1|2075|701|9137|Thursday|2075Q1|N|N|N|2478970|2479000|2478611|2478884|N|N|N|N|N| +2478977|AAAAAAAABIDNFCAA|2075-02-08|2101|9137|701|2075|5|2|8|1|2075|701|9137|Friday|2075Q1|N|Y|N|2478970|2479000|2478612|2478885|N|N|N|N|N| +2478978|AAAAAAAACIDNFCAA|2075-02-09|2101|9137|701|2075|6|2|9|1|2075|701|9137|Saturday|2075Q1|N|Y|N|2478970|2479000|2478613|2478886|N|N|N|N|N| +2478979|AAAAAAAADIDNFCAA|2075-02-10|2101|9137|701|2075|0|2|10|1|2075|701|9137|Sunday|2075Q1|N|N|N|2478970|2479000|2478614|2478887|N|N|N|N|N| +2478980|AAAAAAAAEIDNFCAA|2075-02-11|2101|9137|701|2075|1|2|11|1|2075|701|9137|Monday|2075Q1|N|N|N|2478970|2479000|2478615|2478888|N|N|N|N|N| +2478981|AAAAAAAAFIDNFCAA|2075-02-12|2101|9138|701|2075|2|2|12|1|2075|701|9138|Tuesday|2075Q1|N|N|N|2478970|2479000|2478616|2478889|N|N|N|N|N| +2478982|AAAAAAAAGIDNFCAA|2075-02-13|2101|9138|701|2075|3|2|13|1|2075|701|9138|Wednesday|2075Q1|N|N|N|2478970|2479000|2478617|2478890|N|N|N|N|N| +2478983|AAAAAAAAHIDNFCAA|2075-02-14|2101|9138|701|2075|4|2|14|1|2075|701|9138|Thursday|2075Q1|N|N|N|2478970|2479000|2478618|2478891|N|N|N|N|N| +2478984|AAAAAAAAIIDNFCAA|2075-02-15|2101|9138|701|2075|5|2|15|1|2075|701|9138|Friday|2075Q1|N|Y|N|2478970|2479000|2478619|2478892|N|N|N|N|N| +2478985|AAAAAAAAJIDNFCAA|2075-02-16|2101|9138|701|2075|6|2|16|1|2075|701|9138|Saturday|2075Q1|N|Y|N|2478970|2479000|2478620|2478893|N|N|N|N|N| +2478986|AAAAAAAAKIDNFCAA|2075-02-17|2101|9138|701|2075|0|2|17|1|2075|701|9138|Sunday|2075Q1|N|N|N|2478970|2479000|2478621|2478894|N|N|N|N|N| +2478987|AAAAAAAALIDNFCAA|2075-02-18|2101|9138|701|2075|1|2|18|1|2075|701|9138|Monday|2075Q1|N|N|N|2478970|2479000|2478622|2478895|N|N|N|N|N| +2478988|AAAAAAAAMIDNFCAA|2075-02-19|2101|9139|701|2075|2|2|19|1|2075|701|9139|Tuesday|2075Q1|N|N|N|2478970|2479000|2478623|2478896|N|N|N|N|N| +2478989|AAAAAAAANIDNFCAA|2075-02-20|2101|9139|701|2075|3|2|20|1|2075|701|9139|Wednesday|2075Q1|N|N|N|2478970|2479000|2478624|2478897|N|N|N|N|N| +2478990|AAAAAAAAOIDNFCAA|2075-02-21|2101|9139|701|2075|4|2|21|1|2075|701|9139|Thursday|2075Q1|N|N|N|2478970|2479000|2478625|2478898|N|N|N|N|N| +2478991|AAAAAAAAPIDNFCAA|2075-02-22|2101|9139|701|2075|5|2|22|1|2075|701|9139|Friday|2075Q1|N|Y|N|2478970|2479000|2478626|2478899|N|N|N|N|N| +2478992|AAAAAAAAAJDNFCAA|2075-02-23|2101|9139|701|2075|6|2|23|1|2075|701|9139|Saturday|2075Q1|N|Y|N|2478970|2479000|2478627|2478900|N|N|N|N|N| +2478993|AAAAAAAABJDNFCAA|2075-02-24|2101|9139|701|2075|0|2|24|1|2075|701|9139|Sunday|2075Q1|N|N|N|2478970|2479000|2478628|2478901|N|N|N|N|N| +2478994|AAAAAAAACJDNFCAA|2075-02-25|2101|9139|701|2075|1|2|25|1|2075|701|9139|Monday|2075Q1|N|N|N|2478970|2479000|2478629|2478902|N|N|N|N|N| +2478995|AAAAAAAADJDNFCAA|2075-02-26|2101|9140|701|2075|2|2|26|1|2075|701|9140|Tuesday|2075Q1|N|N|N|2478970|2479000|2478630|2478903|N|N|N|N|N| +2478996|AAAAAAAAEJDNFCAA|2075-02-27|2101|9140|701|2075|3|2|27|1|2075|701|9140|Wednesday|2075Q1|N|N|N|2478970|2479000|2478631|2478904|N|N|N|N|N| +2478997|AAAAAAAAFJDNFCAA|2075-02-28|2101|9140|701|2075|4|2|28|1|2075|701|9140|Thursday|2075Q1|N|N|N|2478970|2479000|2478632|2478905|N|N|N|N|N| +2478998|AAAAAAAAGJDNFCAA|2075-03-01|2102|9140|702|2075|5|3|1|1|2075|702|9140|Friday|2075Q1|N|Y|N|2478998|2479056|2478633|2478906|N|N|N|N|N| +2478999|AAAAAAAAHJDNFCAA|2075-03-02|2102|9140|702|2075|6|3|2|1|2075|702|9140|Saturday|2075Q1|N|Y|N|2478998|2479056|2478634|2478907|N|N|N|N|N| +2479000|AAAAAAAAIJDNFCAA|2075-03-03|2102|9140|702|2075|0|3|3|1|2075|702|9140|Sunday|2075Q1|N|N|N|2478998|2479056|2478635|2478908|N|N|N|N|N| +2479001|AAAAAAAAJJDNFCAA|2075-03-04|2102|9140|702|2075|1|3|4|1|2075|702|9140|Monday|2075Q1|N|N|N|2478998|2479056|2478636|2478909|N|N|N|N|N| +2479002|AAAAAAAAKJDNFCAA|2075-03-05|2102|9141|702|2075|2|3|5|1|2075|702|9141|Tuesday|2075Q1|N|N|N|2478998|2479056|2478637|2478910|N|N|N|N|N| +2479003|AAAAAAAALJDNFCAA|2075-03-06|2102|9141|702|2075|3|3|6|1|2075|702|9141|Wednesday|2075Q1|N|N|N|2478998|2479056|2478638|2478911|N|N|N|N|N| +2479004|AAAAAAAAMJDNFCAA|2075-03-07|2102|9141|702|2075|4|3|7|1|2075|702|9141|Thursday|2075Q1|N|N|N|2478998|2479056|2478639|2478912|N|N|N|N|N| +2479005|AAAAAAAANJDNFCAA|2075-03-08|2102|9141|702|2075|5|3|8|1|2075|702|9141|Friday|2075Q1|N|Y|N|2478998|2479056|2478640|2478913|N|N|N|N|N| +2479006|AAAAAAAAOJDNFCAA|2075-03-09|2102|9141|702|2075|6|3|9|1|2075|702|9141|Saturday|2075Q1|N|Y|N|2478998|2479056|2478641|2478914|N|N|N|N|N| +2479007|AAAAAAAAPJDNFCAA|2075-03-10|2102|9141|702|2075|0|3|10|1|2075|702|9141|Sunday|2075Q1|N|N|N|2478998|2479056|2478642|2478915|N|N|N|N|N| +2479008|AAAAAAAAAKDNFCAA|2075-03-11|2102|9141|702|2075|1|3|11|1|2075|702|9141|Monday|2075Q1|N|N|N|2478998|2479056|2478643|2478916|N|N|N|N|N| +2479009|AAAAAAAABKDNFCAA|2075-03-12|2102|9142|702|2075|2|3|12|1|2075|702|9142|Tuesday|2075Q1|N|N|N|2478998|2479056|2478644|2478917|N|N|N|N|N| +2479010|AAAAAAAACKDNFCAA|2075-03-13|2102|9142|702|2075|3|3|13|1|2075|702|9142|Wednesday|2075Q1|N|N|N|2478998|2479056|2478645|2478918|N|N|N|N|N| +2479011|AAAAAAAADKDNFCAA|2075-03-14|2102|9142|702|2075|4|3|14|1|2075|702|9142|Thursday|2075Q1|N|N|N|2478998|2479056|2478646|2478919|N|N|N|N|N| +2479012|AAAAAAAAEKDNFCAA|2075-03-15|2102|9142|702|2075|5|3|15|1|2075|702|9142|Friday|2075Q1|N|Y|N|2478998|2479056|2478647|2478920|N|N|N|N|N| +2479013|AAAAAAAAFKDNFCAA|2075-03-16|2102|9142|702|2075|6|3|16|1|2075|702|9142|Saturday|2075Q1|N|Y|N|2478998|2479056|2478648|2478921|N|N|N|N|N| +2479014|AAAAAAAAGKDNFCAA|2075-03-17|2102|9142|702|2075|0|3|17|1|2075|702|9142|Sunday|2075Q1|N|N|N|2478998|2479056|2478649|2478922|N|N|N|N|N| +2479015|AAAAAAAAHKDNFCAA|2075-03-18|2102|9142|702|2075|1|3|18|1|2075|702|9142|Monday|2075Q1|N|N|N|2478998|2479056|2478650|2478923|N|N|N|N|N| +2479016|AAAAAAAAIKDNFCAA|2075-03-19|2102|9143|702|2075|2|3|19|1|2075|702|9143|Tuesday|2075Q1|N|N|N|2478998|2479056|2478651|2478924|N|N|N|N|N| +2479017|AAAAAAAAJKDNFCAA|2075-03-20|2102|9143|702|2075|3|3|20|1|2075|702|9143|Wednesday|2075Q1|N|N|N|2478998|2479056|2478652|2478925|N|N|N|N|N| +2479018|AAAAAAAAKKDNFCAA|2075-03-21|2102|9143|702|2075|4|3|21|1|2075|702|9143|Thursday|2075Q1|N|N|N|2478998|2479056|2478653|2478926|N|N|N|N|N| +2479019|AAAAAAAALKDNFCAA|2075-03-22|2102|9143|702|2075|5|3|22|1|2075|702|9143|Friday|2075Q1|N|Y|N|2478998|2479056|2478654|2478927|N|N|N|N|N| +2479020|AAAAAAAAMKDNFCAA|2075-03-23|2102|9143|702|2075|6|3|23|1|2075|702|9143|Saturday|2075Q1|N|Y|N|2478998|2479056|2478655|2478928|N|N|N|N|N| +2479021|AAAAAAAANKDNFCAA|2075-03-24|2102|9143|702|2075|0|3|24|1|2075|702|9143|Sunday|2075Q1|N|N|N|2478998|2479056|2478656|2478929|N|N|N|N|N| +2479022|AAAAAAAAOKDNFCAA|2075-03-25|2102|9143|702|2075|1|3|25|1|2075|702|9143|Monday|2075Q1|N|N|N|2478998|2479056|2478657|2478930|N|N|N|N|N| +2479023|AAAAAAAAPKDNFCAA|2075-03-26|2102|9144|702|2075|2|3|26|1|2075|702|9144|Tuesday|2075Q1|N|N|N|2478998|2479056|2478658|2478931|N|N|N|N|N| +2479024|AAAAAAAAALDNFCAA|2075-03-27|2102|9144|702|2075|3|3|27|1|2075|702|9144|Wednesday|2075Q1|N|N|N|2478998|2479056|2478659|2478932|N|N|N|N|N| +2479025|AAAAAAAABLDNFCAA|2075-03-28|2102|9144|702|2075|4|3|28|1|2075|702|9144|Thursday|2075Q1|N|N|N|2478998|2479056|2478660|2478933|N|N|N|N|N| +2479026|AAAAAAAACLDNFCAA|2075-03-29|2102|9144|702|2075|5|3|29|1|2075|702|9144|Friday|2075Q1|N|Y|N|2478998|2479056|2478661|2478934|N|N|N|N|N| +2479027|AAAAAAAADLDNFCAA|2075-03-30|2102|9144|702|2075|6|3|30|1|2075|702|9144|Saturday|2075Q1|N|Y|N|2478998|2479056|2478662|2478935|N|N|N|N|N| +2479028|AAAAAAAAELDNFCAA|2075-03-31|2102|9144|702|2075|0|3|31|1|2075|702|9144|Sunday|2075Q1|N|N|N|2478998|2479056|2478663|2478936|N|N|N|N|N| +2479029|AAAAAAAAFLDNFCAA|2075-04-01|2103|9144|702|2075|1|4|1|1|2075|702|9144|Monday|2075Q1|N|N|N|2479029|2479118|2478664|2478939|N|N|N|N|N| +2479030|AAAAAAAAGLDNFCAA|2075-04-02|2103|9145|702|2075|2|4|2|2|2075|702|9145|Tuesday|2075Q2|N|N|N|2479029|2479118|2478665|2478940|N|N|N|N|N| +2479031|AAAAAAAAHLDNFCAA|2075-04-03|2103|9145|702|2075|3|4|3|2|2075|702|9145|Wednesday|2075Q2|N|N|N|2479029|2479118|2478666|2478941|N|N|N|N|N| +2479032|AAAAAAAAILDNFCAA|2075-04-04|2103|9145|702|2075|4|4|4|2|2075|702|9145|Thursday|2075Q2|N|N|N|2479029|2479118|2478667|2478942|N|N|N|N|N| +2479033|AAAAAAAAJLDNFCAA|2075-04-05|2103|9145|702|2075|5|4|5|2|2075|702|9145|Friday|2075Q2|N|Y|N|2479029|2479118|2478668|2478943|N|N|N|N|N| +2479034|AAAAAAAAKLDNFCAA|2075-04-06|2103|9145|702|2075|6|4|6|2|2075|702|9145|Saturday|2075Q2|N|Y|N|2479029|2479118|2478669|2478944|N|N|N|N|N| +2479035|AAAAAAAALLDNFCAA|2075-04-07|2103|9145|702|2075|0|4|7|2|2075|702|9145|Sunday|2075Q2|N|N|N|2479029|2479118|2478670|2478945|N|N|N|N|N| +2479036|AAAAAAAAMLDNFCAA|2075-04-08|2103|9145|702|2075|1|4|8|2|2075|702|9145|Monday|2075Q2|N|N|N|2479029|2479118|2478671|2478946|N|N|N|N|N| +2479037|AAAAAAAANLDNFCAA|2075-04-09|2103|9146|702|2075|2|4|9|2|2075|702|9146|Tuesday|2075Q2|N|N|N|2479029|2479118|2478672|2478947|N|N|N|N|N| +2479038|AAAAAAAAOLDNFCAA|2075-04-10|2103|9146|702|2075|3|4|10|2|2075|702|9146|Wednesday|2075Q2|N|N|N|2479029|2479118|2478673|2478948|N|N|N|N|N| +2479039|AAAAAAAAPLDNFCAA|2075-04-11|2103|9146|702|2075|4|4|11|2|2075|702|9146|Thursday|2075Q2|N|N|N|2479029|2479118|2478674|2478949|N|N|N|N|N| +2479040|AAAAAAAAAMDNFCAA|2075-04-12|2103|9146|702|2075|5|4|12|2|2075|702|9146|Friday|2075Q2|N|Y|N|2479029|2479118|2478675|2478950|N|N|N|N|N| +2479041|AAAAAAAABMDNFCAA|2075-04-13|2103|9146|702|2075|6|4|13|2|2075|702|9146|Saturday|2075Q2|N|Y|N|2479029|2479118|2478676|2478951|N|N|N|N|N| +2479042|AAAAAAAACMDNFCAA|2075-04-14|2103|9146|702|2075|0|4|14|2|2075|702|9146|Sunday|2075Q2|N|N|N|2479029|2479118|2478677|2478952|N|N|N|N|N| +2479043|AAAAAAAADMDNFCAA|2075-04-15|2103|9146|702|2075|1|4|15|2|2075|702|9146|Monday|2075Q2|N|N|N|2479029|2479118|2478678|2478953|N|N|N|N|N| +2479044|AAAAAAAAEMDNFCAA|2075-04-16|2103|9147|702|2075|2|4|16|2|2075|702|9147|Tuesday|2075Q2|N|N|N|2479029|2479118|2478679|2478954|N|N|N|N|N| +2479045|AAAAAAAAFMDNFCAA|2075-04-17|2103|9147|702|2075|3|4|17|2|2075|702|9147|Wednesday|2075Q2|N|N|N|2479029|2479118|2478680|2478955|N|N|N|N|N| +2479046|AAAAAAAAGMDNFCAA|2075-04-18|2103|9147|702|2075|4|4|18|2|2075|702|9147|Thursday|2075Q2|N|N|N|2479029|2479118|2478681|2478956|N|N|N|N|N| +2479047|AAAAAAAAHMDNFCAA|2075-04-19|2103|9147|702|2075|5|4|19|2|2075|702|9147|Friday|2075Q2|N|Y|N|2479029|2479118|2478682|2478957|N|N|N|N|N| +2479048|AAAAAAAAIMDNFCAA|2075-04-20|2103|9147|702|2075|6|4|20|2|2075|702|9147|Saturday|2075Q2|N|Y|N|2479029|2479118|2478683|2478958|N|N|N|N|N| +2479049|AAAAAAAAJMDNFCAA|2075-04-21|2103|9147|702|2075|0|4|21|2|2075|702|9147|Sunday|2075Q2|N|N|N|2479029|2479118|2478684|2478959|N|N|N|N|N| +2479050|AAAAAAAAKMDNFCAA|2075-04-22|2103|9147|702|2075|1|4|22|2|2075|702|9147|Monday|2075Q2|N|N|N|2479029|2479118|2478685|2478960|N|N|N|N|N| +2479051|AAAAAAAALMDNFCAA|2075-04-23|2103|9148|702|2075|2|4|23|2|2075|702|9148|Tuesday|2075Q2|N|N|N|2479029|2479118|2478686|2478961|N|N|N|N|N| +2479052|AAAAAAAAMMDNFCAA|2075-04-24|2103|9148|702|2075|3|4|24|2|2075|702|9148|Wednesday|2075Q2|N|N|N|2479029|2479118|2478687|2478962|N|N|N|N|N| +2479053|AAAAAAAANMDNFCAA|2075-04-25|2103|9148|702|2075|4|4|25|2|2075|702|9148|Thursday|2075Q2|N|N|N|2479029|2479118|2478688|2478963|N|N|N|N|N| +2479054|AAAAAAAAOMDNFCAA|2075-04-26|2103|9148|702|2075|5|4|26|2|2075|702|9148|Friday|2075Q2|N|Y|N|2479029|2479118|2478689|2478964|N|N|N|N|N| +2479055|AAAAAAAAPMDNFCAA|2075-04-27|2103|9148|702|2075|6|4|27|2|2075|702|9148|Saturday|2075Q2|N|Y|N|2479029|2479118|2478690|2478965|N|N|N|N|N| +2479056|AAAAAAAAANDNFCAA|2075-04-28|2103|9148|702|2075|0|4|28|2|2075|702|9148|Sunday|2075Q2|N|N|N|2479029|2479118|2478691|2478966|N|N|N|N|N| +2479057|AAAAAAAABNDNFCAA|2075-04-29|2103|9148|702|2075|1|4|29|2|2075|702|9148|Monday|2075Q2|N|N|N|2479029|2479118|2478692|2478967|N|N|N|N|N| +2479058|AAAAAAAACNDNFCAA|2075-04-30|2103|9149|702|2075|2|4|30|2|2075|702|9149|Tuesday|2075Q2|N|N|N|2479029|2479118|2478693|2478968|N|N|N|N|N| +2479059|AAAAAAAADNDNFCAA|2075-05-01|2104|9149|702|2075|3|5|1|2|2075|702|9149|Wednesday|2075Q2|N|N|N|2479059|2479178|2478694|2478969|N|N|N|N|N| +2479060|AAAAAAAAENDNFCAA|2075-05-02|2104|9149|702|2075|4|5|2|2|2075|702|9149|Thursday|2075Q2|N|N|N|2479059|2479178|2478695|2478970|N|N|N|N|N| +2479061|AAAAAAAAFNDNFCAA|2075-05-03|2104|9149|702|2075|5|5|3|2|2075|702|9149|Friday|2075Q2|N|Y|N|2479059|2479178|2478696|2478971|N|N|N|N|N| +2479062|AAAAAAAAGNDNFCAA|2075-05-04|2104|9149|702|2075|6|5|4|2|2075|702|9149|Saturday|2075Q2|N|Y|N|2479059|2479178|2478697|2478972|N|N|N|N|N| +2479063|AAAAAAAAHNDNFCAA|2075-05-05|2104|9149|702|2075|0|5|5|2|2075|702|9149|Sunday|2075Q2|N|N|N|2479059|2479178|2478698|2478973|N|N|N|N|N| +2479064|AAAAAAAAINDNFCAA|2075-05-06|2104|9149|702|2075|1|5|6|2|2075|702|9149|Monday|2075Q2|N|N|N|2479059|2479178|2478699|2478974|N|N|N|N|N| +2479065|AAAAAAAAJNDNFCAA|2075-05-07|2104|9150|702|2075|2|5|7|2|2075|702|9150|Tuesday|2075Q2|N|N|N|2479059|2479178|2478700|2478975|N|N|N|N|N| +2479066|AAAAAAAAKNDNFCAA|2075-05-08|2104|9150|702|2075|3|5|8|2|2075|702|9150|Wednesday|2075Q2|N|N|N|2479059|2479178|2478701|2478976|N|N|N|N|N| +2479067|AAAAAAAALNDNFCAA|2075-05-09|2104|9150|702|2075|4|5|9|2|2075|702|9150|Thursday|2075Q2|N|N|N|2479059|2479178|2478702|2478977|N|N|N|N|N| +2479068|AAAAAAAAMNDNFCAA|2075-05-10|2104|9150|702|2075|5|5|10|2|2075|702|9150|Friday|2075Q2|N|Y|N|2479059|2479178|2478703|2478978|N|N|N|N|N| +2479069|AAAAAAAANNDNFCAA|2075-05-11|2104|9150|702|2075|6|5|11|2|2075|702|9150|Saturday|2075Q2|N|Y|N|2479059|2479178|2478704|2478979|N|N|N|N|N| +2479070|AAAAAAAAONDNFCAA|2075-05-12|2104|9150|702|2075|0|5|12|2|2075|702|9150|Sunday|2075Q2|N|N|N|2479059|2479178|2478705|2478980|N|N|N|N|N| +2479071|AAAAAAAAPNDNFCAA|2075-05-13|2104|9150|702|2075|1|5|13|2|2075|702|9150|Monday|2075Q2|N|N|N|2479059|2479178|2478706|2478981|N|N|N|N|N| +2479072|AAAAAAAAAODNFCAA|2075-05-14|2104|9151|702|2075|2|5|14|2|2075|702|9151|Tuesday|2075Q2|N|N|N|2479059|2479178|2478707|2478982|N|N|N|N|N| +2479073|AAAAAAAABODNFCAA|2075-05-15|2104|9151|702|2075|3|5|15|2|2075|702|9151|Wednesday|2075Q2|N|N|N|2479059|2479178|2478708|2478983|N|N|N|N|N| +2479074|AAAAAAAACODNFCAA|2075-05-16|2104|9151|702|2075|4|5|16|2|2075|702|9151|Thursday|2075Q2|N|N|N|2479059|2479178|2478709|2478984|N|N|N|N|N| +2479075|AAAAAAAADODNFCAA|2075-05-17|2104|9151|702|2075|5|5|17|2|2075|702|9151|Friday|2075Q2|N|Y|N|2479059|2479178|2478710|2478985|N|N|N|N|N| +2479076|AAAAAAAAEODNFCAA|2075-05-18|2104|9151|702|2075|6|5|18|2|2075|702|9151|Saturday|2075Q2|N|Y|N|2479059|2479178|2478711|2478986|N|N|N|N|N| +2479077|AAAAAAAAFODNFCAA|2075-05-19|2104|9151|702|2075|0|5|19|2|2075|702|9151|Sunday|2075Q2|N|N|N|2479059|2479178|2478712|2478987|N|N|N|N|N| +2479078|AAAAAAAAGODNFCAA|2075-05-20|2104|9151|702|2075|1|5|20|2|2075|702|9151|Monday|2075Q2|N|N|N|2479059|2479178|2478713|2478988|N|N|N|N|N| +2479079|AAAAAAAAHODNFCAA|2075-05-21|2104|9152|702|2075|2|5|21|2|2075|702|9152|Tuesday|2075Q2|N|N|N|2479059|2479178|2478714|2478989|N|N|N|N|N| +2479080|AAAAAAAAIODNFCAA|2075-05-22|2104|9152|702|2075|3|5|22|2|2075|702|9152|Wednesday|2075Q2|N|N|N|2479059|2479178|2478715|2478990|N|N|N|N|N| +2479081|AAAAAAAAJODNFCAA|2075-05-23|2104|9152|702|2075|4|5|23|2|2075|702|9152|Thursday|2075Q2|N|N|N|2479059|2479178|2478716|2478991|N|N|N|N|N| +2479082|AAAAAAAAKODNFCAA|2075-05-24|2104|9152|702|2075|5|5|24|2|2075|702|9152|Friday|2075Q2|N|Y|N|2479059|2479178|2478717|2478992|N|N|N|N|N| +2479083|AAAAAAAALODNFCAA|2075-05-25|2104|9152|702|2075|6|5|25|2|2075|702|9152|Saturday|2075Q2|N|Y|N|2479059|2479178|2478718|2478993|N|N|N|N|N| +2479084|AAAAAAAAMODNFCAA|2075-05-26|2104|9152|702|2075|0|5|26|2|2075|702|9152|Sunday|2075Q2|N|N|N|2479059|2479178|2478719|2478994|N|N|N|N|N| +2479085|AAAAAAAANODNFCAA|2075-05-27|2104|9152|702|2075|1|5|27|2|2075|702|9152|Monday|2075Q2|N|N|N|2479059|2479178|2478720|2478995|N|N|N|N|N| +2479086|AAAAAAAAOODNFCAA|2075-05-28|2104|9153|702|2075|2|5|28|2|2075|702|9153|Tuesday|2075Q2|N|N|N|2479059|2479178|2478721|2478996|N|N|N|N|N| +2479087|AAAAAAAAPODNFCAA|2075-05-29|2104|9153|702|2075|3|5|29|2|2075|702|9153|Wednesday|2075Q2|N|N|N|2479059|2479178|2478722|2478997|N|N|N|N|N| +2479088|AAAAAAAAAPDNFCAA|2075-05-30|2104|9153|702|2075|4|5|30|2|2075|702|9153|Thursday|2075Q2|N|N|N|2479059|2479178|2478723|2478998|N|N|N|N|N| +2479089|AAAAAAAABPDNFCAA|2075-05-31|2104|9153|702|2075|5|5|31|2|2075|702|9153|Friday|2075Q2|N|Y|N|2479059|2479178|2478724|2478999|N|N|N|N|N| +2479090|AAAAAAAACPDNFCAA|2075-06-01|2105|9153|703|2075|6|6|1|2|2075|703|9153|Saturday|2075Q2|N|Y|N|2479090|2479240|2478725|2479000|N|N|N|N|N| +2479091|AAAAAAAADPDNFCAA|2075-06-02|2105|9153|703|2075|0|6|2|2|2075|703|9153|Sunday|2075Q2|N|N|N|2479090|2479240|2478726|2479001|N|N|N|N|N| +2479092|AAAAAAAAEPDNFCAA|2075-06-03|2105|9153|703|2075|1|6|3|2|2075|703|9153|Monday|2075Q2|N|N|N|2479090|2479240|2478727|2479002|N|N|N|N|N| +2479093|AAAAAAAAFPDNFCAA|2075-06-04|2105|9154|703|2075|2|6|4|2|2075|703|9154|Tuesday|2075Q2|N|N|N|2479090|2479240|2478728|2479003|N|N|N|N|N| +2479094|AAAAAAAAGPDNFCAA|2075-06-05|2105|9154|703|2075|3|6|5|2|2075|703|9154|Wednesday|2075Q2|N|N|N|2479090|2479240|2478729|2479004|N|N|N|N|N| +2479095|AAAAAAAAHPDNFCAA|2075-06-06|2105|9154|703|2075|4|6|6|2|2075|703|9154|Thursday|2075Q2|N|N|N|2479090|2479240|2478730|2479005|N|N|N|N|N| +2479096|AAAAAAAAIPDNFCAA|2075-06-07|2105|9154|703|2075|5|6|7|2|2075|703|9154|Friday|2075Q2|N|Y|N|2479090|2479240|2478731|2479006|N|N|N|N|N| +2479097|AAAAAAAAJPDNFCAA|2075-06-08|2105|9154|703|2075|6|6|8|2|2075|703|9154|Saturday|2075Q2|N|Y|N|2479090|2479240|2478732|2479007|N|N|N|N|N| +2479098|AAAAAAAAKPDNFCAA|2075-06-09|2105|9154|703|2075|0|6|9|2|2075|703|9154|Sunday|2075Q2|N|N|N|2479090|2479240|2478733|2479008|N|N|N|N|N| +2479099|AAAAAAAALPDNFCAA|2075-06-10|2105|9154|703|2075|1|6|10|2|2075|703|9154|Monday|2075Q2|N|N|N|2479090|2479240|2478734|2479009|N|N|N|N|N| +2479100|AAAAAAAAMPDNFCAA|2075-06-11|2105|9155|703|2075|2|6|11|2|2075|703|9155|Tuesday|2075Q2|N|N|N|2479090|2479240|2478735|2479010|N|N|N|N|N| +2479101|AAAAAAAANPDNFCAA|2075-06-12|2105|9155|703|2075|3|6|12|2|2075|703|9155|Wednesday|2075Q2|N|N|N|2479090|2479240|2478736|2479011|N|N|N|N|N| +2479102|AAAAAAAAOPDNFCAA|2075-06-13|2105|9155|703|2075|4|6|13|2|2075|703|9155|Thursday|2075Q2|N|N|N|2479090|2479240|2478737|2479012|N|N|N|N|N| +2479103|AAAAAAAAPPDNFCAA|2075-06-14|2105|9155|703|2075|5|6|14|2|2075|703|9155|Friday|2075Q2|N|Y|N|2479090|2479240|2478738|2479013|N|N|N|N|N| +2479104|AAAAAAAAAAENFCAA|2075-06-15|2105|9155|703|2075|6|6|15|2|2075|703|9155|Saturday|2075Q2|N|Y|N|2479090|2479240|2478739|2479014|N|N|N|N|N| +2479105|AAAAAAAABAENFCAA|2075-06-16|2105|9155|703|2075|0|6|16|2|2075|703|9155|Sunday|2075Q2|N|N|N|2479090|2479240|2478740|2479015|N|N|N|N|N| +2479106|AAAAAAAACAENFCAA|2075-06-17|2105|9155|703|2075|1|6|17|2|2075|703|9155|Monday|2075Q2|N|N|N|2479090|2479240|2478741|2479016|N|N|N|N|N| +2479107|AAAAAAAADAENFCAA|2075-06-18|2105|9156|703|2075|2|6|18|2|2075|703|9156|Tuesday|2075Q2|N|N|N|2479090|2479240|2478742|2479017|N|N|N|N|N| +2479108|AAAAAAAAEAENFCAA|2075-06-19|2105|9156|703|2075|3|6|19|2|2075|703|9156|Wednesday|2075Q2|N|N|N|2479090|2479240|2478743|2479018|N|N|N|N|N| +2479109|AAAAAAAAFAENFCAA|2075-06-20|2105|9156|703|2075|4|6|20|2|2075|703|9156|Thursday|2075Q2|N|N|N|2479090|2479240|2478744|2479019|N|N|N|N|N| +2479110|AAAAAAAAGAENFCAA|2075-06-21|2105|9156|703|2075|5|6|21|2|2075|703|9156|Friday|2075Q2|N|Y|N|2479090|2479240|2478745|2479020|N|N|N|N|N| +2479111|AAAAAAAAHAENFCAA|2075-06-22|2105|9156|703|2075|6|6|22|2|2075|703|9156|Saturday|2075Q2|N|Y|N|2479090|2479240|2478746|2479021|N|N|N|N|N| +2479112|AAAAAAAAIAENFCAA|2075-06-23|2105|9156|703|2075|0|6|23|2|2075|703|9156|Sunday|2075Q2|N|N|N|2479090|2479240|2478747|2479022|N|N|N|N|N| +2479113|AAAAAAAAJAENFCAA|2075-06-24|2105|9156|703|2075|1|6|24|2|2075|703|9156|Monday|2075Q2|N|N|N|2479090|2479240|2478748|2479023|N|N|N|N|N| +2479114|AAAAAAAAKAENFCAA|2075-06-25|2105|9157|703|2075|2|6|25|2|2075|703|9157|Tuesday|2075Q2|N|N|N|2479090|2479240|2478749|2479024|N|N|N|N|N| +2479115|AAAAAAAALAENFCAA|2075-06-26|2105|9157|703|2075|3|6|26|2|2075|703|9157|Wednesday|2075Q2|N|N|N|2479090|2479240|2478750|2479025|N|N|N|N|N| +2479116|AAAAAAAAMAENFCAA|2075-06-27|2105|9157|703|2075|4|6|27|2|2075|703|9157|Thursday|2075Q2|N|N|N|2479090|2479240|2478751|2479026|N|N|N|N|N| +2479117|AAAAAAAANAENFCAA|2075-06-28|2105|9157|703|2075|5|6|28|2|2075|703|9157|Friday|2075Q2|N|Y|N|2479090|2479240|2478752|2479027|N|N|N|N|N| +2479118|AAAAAAAAOAENFCAA|2075-06-29|2105|9157|703|2075|6|6|29|2|2075|703|9157|Saturday|2075Q2|N|Y|N|2479090|2479240|2478753|2479028|N|N|N|N|N| +2479119|AAAAAAAAPAENFCAA|2075-06-30|2105|9157|703|2075|0|6|30|2|2075|703|9157|Sunday|2075Q2|N|N|N|2479090|2479240|2478754|2479029|N|N|N|N|N| +2479120|AAAAAAAAABENFCAA|2075-07-01|2106|9157|703|2075|1|7|1|2|2075|703|9157|Monday|2075Q2|N|N|N|2479120|2479300|2478755|2479029|N|N|N|N|N| +2479121|AAAAAAAABBENFCAA|2075-07-02|2106|9158|703|2075|2|7|2|3|2075|703|9158|Tuesday|2075Q3|N|N|N|2479120|2479300|2478756|2479030|N|N|N|N|N| +2479122|AAAAAAAACBENFCAA|2075-07-03|2106|9158|703|2075|3|7|3|3|2075|703|9158|Wednesday|2075Q3|N|N|N|2479120|2479300|2478757|2479031|N|N|N|N|N| +2479123|AAAAAAAADBENFCAA|2075-07-04|2106|9158|703|2075|4|7|4|3|2075|703|9158|Thursday|2075Q3|N|N|N|2479120|2479300|2478758|2479032|N|N|N|N|N| +2479124|AAAAAAAAEBENFCAA|2075-07-05|2106|9158|703|2075|5|7|5|3|2075|703|9158|Friday|2075Q3|Y|Y|N|2479120|2479300|2478759|2479033|N|N|N|N|N| +2479125|AAAAAAAAFBENFCAA|2075-07-06|2106|9158|703|2075|6|7|6|3|2075|703|9158|Saturday|2075Q3|N|Y|Y|2479120|2479300|2478760|2479034|N|N|N|N|N| +2479126|AAAAAAAAGBENFCAA|2075-07-07|2106|9158|703|2075|0|7|7|3|2075|703|9158|Sunday|2075Q3|N|N|N|2479120|2479300|2478761|2479035|N|N|N|N|N| +2479127|AAAAAAAAHBENFCAA|2075-07-08|2106|9158|703|2075|1|7|8|3|2075|703|9158|Monday|2075Q3|N|N|N|2479120|2479300|2478762|2479036|N|N|N|N|N| +2479128|AAAAAAAAIBENFCAA|2075-07-09|2106|9159|703|2075|2|7|9|3|2075|703|9159|Tuesday|2075Q3|N|N|N|2479120|2479300|2478763|2479037|N|N|N|N|N| +2479129|AAAAAAAAJBENFCAA|2075-07-10|2106|9159|703|2075|3|7|10|3|2075|703|9159|Wednesday|2075Q3|N|N|N|2479120|2479300|2478764|2479038|N|N|N|N|N| +2479130|AAAAAAAAKBENFCAA|2075-07-11|2106|9159|703|2075|4|7|11|3|2075|703|9159|Thursday|2075Q3|N|N|N|2479120|2479300|2478765|2479039|N|N|N|N|N| +2479131|AAAAAAAALBENFCAA|2075-07-12|2106|9159|703|2075|5|7|12|3|2075|703|9159|Friday|2075Q3|N|Y|N|2479120|2479300|2478766|2479040|N|N|N|N|N| +2479132|AAAAAAAAMBENFCAA|2075-07-13|2106|9159|703|2075|6|7|13|3|2075|703|9159|Saturday|2075Q3|N|Y|N|2479120|2479300|2478767|2479041|N|N|N|N|N| +2479133|AAAAAAAANBENFCAA|2075-07-14|2106|9159|703|2075|0|7|14|3|2075|703|9159|Sunday|2075Q3|N|N|N|2479120|2479300|2478768|2479042|N|N|N|N|N| +2479134|AAAAAAAAOBENFCAA|2075-07-15|2106|9159|703|2075|1|7|15|3|2075|703|9159|Monday|2075Q3|N|N|N|2479120|2479300|2478769|2479043|N|N|N|N|N| +2479135|AAAAAAAAPBENFCAA|2075-07-16|2106|9160|703|2075|2|7|16|3|2075|703|9160|Tuesday|2075Q3|N|N|N|2479120|2479300|2478770|2479044|N|N|N|N|N| +2479136|AAAAAAAAACENFCAA|2075-07-17|2106|9160|703|2075|3|7|17|3|2075|703|9160|Wednesday|2075Q3|N|N|N|2479120|2479300|2478771|2479045|N|N|N|N|N| +2479137|AAAAAAAABCENFCAA|2075-07-18|2106|9160|703|2075|4|7|18|3|2075|703|9160|Thursday|2075Q3|N|N|N|2479120|2479300|2478772|2479046|N|N|N|N|N| +2479138|AAAAAAAACCENFCAA|2075-07-19|2106|9160|703|2075|5|7|19|3|2075|703|9160|Friday|2075Q3|N|Y|N|2479120|2479300|2478773|2479047|N|N|N|N|N| +2479139|AAAAAAAADCENFCAA|2075-07-20|2106|9160|703|2075|6|7|20|3|2075|703|9160|Saturday|2075Q3|N|Y|N|2479120|2479300|2478774|2479048|N|N|N|N|N| +2479140|AAAAAAAAECENFCAA|2075-07-21|2106|9160|703|2075|0|7|21|3|2075|703|9160|Sunday|2075Q3|N|N|N|2479120|2479300|2478775|2479049|N|N|N|N|N| +2479141|AAAAAAAAFCENFCAA|2075-07-22|2106|9160|703|2075|1|7|22|3|2075|703|9160|Monday|2075Q3|N|N|N|2479120|2479300|2478776|2479050|N|N|N|N|N| +2479142|AAAAAAAAGCENFCAA|2075-07-23|2106|9161|703|2075|2|7|23|3|2075|703|9161|Tuesday|2075Q3|N|N|N|2479120|2479300|2478777|2479051|N|N|N|N|N| +2479143|AAAAAAAAHCENFCAA|2075-07-24|2106|9161|703|2075|3|7|24|3|2075|703|9161|Wednesday|2075Q3|N|N|N|2479120|2479300|2478778|2479052|N|N|N|N|N| +2479144|AAAAAAAAICENFCAA|2075-07-25|2106|9161|703|2075|4|7|25|3|2075|703|9161|Thursday|2075Q3|N|N|N|2479120|2479300|2478779|2479053|N|N|N|N|N| +2479145|AAAAAAAAJCENFCAA|2075-07-26|2106|9161|703|2075|5|7|26|3|2075|703|9161|Friday|2075Q3|N|Y|N|2479120|2479300|2478780|2479054|N|N|N|N|N| +2479146|AAAAAAAAKCENFCAA|2075-07-27|2106|9161|703|2075|6|7|27|3|2075|703|9161|Saturday|2075Q3|N|Y|N|2479120|2479300|2478781|2479055|N|N|N|N|N| +2479147|AAAAAAAALCENFCAA|2075-07-28|2106|9161|703|2075|0|7|28|3|2075|703|9161|Sunday|2075Q3|N|N|N|2479120|2479300|2478782|2479056|N|N|N|N|N| +2479148|AAAAAAAAMCENFCAA|2075-07-29|2106|9161|703|2075|1|7|29|3|2075|703|9161|Monday|2075Q3|N|N|N|2479120|2479300|2478783|2479057|N|N|N|N|N| +2479149|AAAAAAAANCENFCAA|2075-07-30|2106|9162|703|2075|2|7|30|3|2075|703|9162|Tuesday|2075Q3|N|N|N|2479120|2479300|2478784|2479058|N|N|N|N|N| +2479150|AAAAAAAAOCENFCAA|2075-07-31|2106|9162|703|2075|3|7|31|3|2075|703|9162|Wednesday|2075Q3|N|N|N|2479120|2479300|2478785|2479059|N|N|N|N|N| +2479151|AAAAAAAAPCENFCAA|2075-08-01|2107|9162|703|2075|4|8|1|3|2075|703|9162|Thursday|2075Q3|N|N|N|2479151|2479362|2478786|2479060|N|N|N|N|N| +2479152|AAAAAAAAADENFCAA|2075-08-02|2107|9162|703|2075|5|8|2|3|2075|703|9162|Friday|2075Q3|N|Y|N|2479151|2479362|2478787|2479061|N|N|N|N|N| +2479153|AAAAAAAABDENFCAA|2075-08-03|2107|9162|703|2075|6|8|3|3|2075|703|9162|Saturday|2075Q3|N|Y|N|2479151|2479362|2478788|2479062|N|N|N|N|N| +2479154|AAAAAAAACDENFCAA|2075-08-04|2107|9162|703|2075|0|8|4|3|2075|703|9162|Sunday|2075Q3|N|N|N|2479151|2479362|2478789|2479063|N|N|N|N|N| +2479155|AAAAAAAADDENFCAA|2075-08-05|2107|9162|703|2075|1|8|5|3|2075|703|9162|Monday|2075Q3|N|N|N|2479151|2479362|2478790|2479064|N|N|N|N|N| +2479156|AAAAAAAAEDENFCAA|2075-08-06|2107|9163|703|2075|2|8|6|3|2075|703|9163|Tuesday|2075Q3|N|N|N|2479151|2479362|2478791|2479065|N|N|N|N|N| +2479157|AAAAAAAAFDENFCAA|2075-08-07|2107|9163|703|2075|3|8|7|3|2075|703|9163|Wednesday|2075Q3|N|N|N|2479151|2479362|2478792|2479066|N|N|N|N|N| +2479158|AAAAAAAAGDENFCAA|2075-08-08|2107|9163|703|2075|4|8|8|3|2075|703|9163|Thursday|2075Q3|N|N|N|2479151|2479362|2478793|2479067|N|N|N|N|N| +2479159|AAAAAAAAHDENFCAA|2075-08-09|2107|9163|703|2075|5|8|9|3|2075|703|9163|Friday|2075Q3|N|Y|N|2479151|2479362|2478794|2479068|N|N|N|N|N| +2479160|AAAAAAAAIDENFCAA|2075-08-10|2107|9163|703|2075|6|8|10|3|2075|703|9163|Saturday|2075Q3|N|Y|N|2479151|2479362|2478795|2479069|N|N|N|N|N| +2479161|AAAAAAAAJDENFCAA|2075-08-11|2107|9163|703|2075|0|8|11|3|2075|703|9163|Sunday|2075Q3|N|N|N|2479151|2479362|2478796|2479070|N|N|N|N|N| +2479162|AAAAAAAAKDENFCAA|2075-08-12|2107|9163|703|2075|1|8|12|3|2075|703|9163|Monday|2075Q3|N|N|N|2479151|2479362|2478797|2479071|N|N|N|N|N| +2479163|AAAAAAAALDENFCAA|2075-08-13|2107|9164|703|2075|2|8|13|3|2075|703|9164|Tuesday|2075Q3|N|N|N|2479151|2479362|2478798|2479072|N|N|N|N|N| +2479164|AAAAAAAAMDENFCAA|2075-08-14|2107|9164|703|2075|3|8|14|3|2075|703|9164|Wednesday|2075Q3|N|N|N|2479151|2479362|2478799|2479073|N|N|N|N|N| +2479165|AAAAAAAANDENFCAA|2075-08-15|2107|9164|703|2075|4|8|15|3|2075|703|9164|Thursday|2075Q3|N|N|N|2479151|2479362|2478800|2479074|N|N|N|N|N| +2479166|AAAAAAAAODENFCAA|2075-08-16|2107|9164|703|2075|5|8|16|3|2075|703|9164|Friday|2075Q3|N|Y|N|2479151|2479362|2478801|2479075|N|N|N|N|N| +2479167|AAAAAAAAPDENFCAA|2075-08-17|2107|9164|703|2075|6|8|17|3|2075|703|9164|Saturday|2075Q3|N|Y|N|2479151|2479362|2478802|2479076|N|N|N|N|N| +2479168|AAAAAAAAAEENFCAA|2075-08-18|2107|9164|703|2075|0|8|18|3|2075|703|9164|Sunday|2075Q3|N|N|N|2479151|2479362|2478803|2479077|N|N|N|N|N| +2479169|AAAAAAAABEENFCAA|2075-08-19|2107|9164|703|2075|1|8|19|3|2075|703|9164|Monday|2075Q3|N|N|N|2479151|2479362|2478804|2479078|N|N|N|N|N| +2479170|AAAAAAAACEENFCAA|2075-08-20|2107|9165|703|2075|2|8|20|3|2075|703|9165|Tuesday|2075Q3|N|N|N|2479151|2479362|2478805|2479079|N|N|N|N|N| +2479171|AAAAAAAADEENFCAA|2075-08-21|2107|9165|703|2075|3|8|21|3|2075|703|9165|Wednesday|2075Q3|N|N|N|2479151|2479362|2478806|2479080|N|N|N|N|N| +2479172|AAAAAAAAEEENFCAA|2075-08-22|2107|9165|703|2075|4|8|22|3|2075|703|9165|Thursday|2075Q3|N|N|N|2479151|2479362|2478807|2479081|N|N|N|N|N| +2479173|AAAAAAAAFEENFCAA|2075-08-23|2107|9165|703|2075|5|8|23|3|2075|703|9165|Friday|2075Q3|N|Y|N|2479151|2479362|2478808|2479082|N|N|N|N|N| +2479174|AAAAAAAAGEENFCAA|2075-08-24|2107|9165|703|2075|6|8|24|3|2075|703|9165|Saturday|2075Q3|N|Y|N|2479151|2479362|2478809|2479083|N|N|N|N|N| +2479175|AAAAAAAAHEENFCAA|2075-08-25|2107|9165|703|2075|0|8|25|3|2075|703|9165|Sunday|2075Q3|N|N|N|2479151|2479362|2478810|2479084|N|N|N|N|N| +2479176|AAAAAAAAIEENFCAA|2075-08-26|2107|9165|703|2075|1|8|26|3|2075|703|9165|Monday|2075Q3|N|N|N|2479151|2479362|2478811|2479085|N|N|N|N|N| +2479177|AAAAAAAAJEENFCAA|2075-08-27|2107|9166|703|2075|2|8|27|3|2075|703|9166|Tuesday|2075Q3|N|N|N|2479151|2479362|2478812|2479086|N|N|N|N|N| +2479178|AAAAAAAAKEENFCAA|2075-08-28|2107|9166|703|2075|3|8|28|3|2075|703|9166|Wednesday|2075Q3|N|N|N|2479151|2479362|2478813|2479087|N|N|N|N|N| +2479179|AAAAAAAALEENFCAA|2075-08-29|2107|9166|703|2075|4|8|29|3|2075|703|9166|Thursday|2075Q3|N|N|N|2479151|2479362|2478814|2479088|N|N|N|N|N| +2479180|AAAAAAAAMEENFCAA|2075-08-30|2107|9166|703|2075|5|8|30|3|2075|703|9166|Friday|2075Q3|N|Y|N|2479151|2479362|2478815|2479089|N|N|N|N|N| +2479181|AAAAAAAANEENFCAA|2075-08-31|2107|9166|703|2075|6|8|31|3|2075|703|9166|Saturday|2075Q3|N|Y|N|2479151|2479362|2478816|2479090|N|N|N|N|N| +2479182|AAAAAAAAOEENFCAA|2075-09-01|2108|9166|704|2075|0|9|1|3|2075|704|9166|Sunday|2075Q3|N|N|N|2479182|2479424|2478817|2479091|N|N|N|N|N| +2479183|AAAAAAAAPEENFCAA|2075-09-02|2108|9166|704|2075|1|9|2|3|2075|704|9166|Monday|2075Q3|N|N|N|2479182|2479424|2478818|2479092|N|N|N|N|N| +2479184|AAAAAAAAAFENFCAA|2075-09-03|2108|9167|704|2075|2|9|3|3|2075|704|9167|Tuesday|2075Q3|N|N|N|2479182|2479424|2478819|2479093|N|N|N|N|N| +2479185|AAAAAAAABFENFCAA|2075-09-04|2108|9167|704|2075|3|9|4|3|2075|704|9167|Wednesday|2075Q3|N|N|N|2479182|2479424|2478820|2479094|N|N|N|N|N| +2479186|AAAAAAAACFENFCAA|2075-09-05|2108|9167|704|2075|4|9|5|3|2075|704|9167|Thursday|2075Q3|N|N|N|2479182|2479424|2478821|2479095|N|N|N|N|N| +2479187|AAAAAAAADFENFCAA|2075-09-06|2108|9167|704|2075|5|9|6|3|2075|704|9167|Friday|2075Q3|N|Y|N|2479182|2479424|2478822|2479096|N|N|N|N|N| +2479188|AAAAAAAAEFENFCAA|2075-09-07|2108|9167|704|2075|6|9|7|3|2075|704|9167|Saturday|2075Q3|N|Y|N|2479182|2479424|2478823|2479097|N|N|N|N|N| +2479189|AAAAAAAAFFENFCAA|2075-09-08|2108|9167|704|2075|0|9|8|3|2075|704|9167|Sunday|2075Q3|N|N|N|2479182|2479424|2478824|2479098|N|N|N|N|N| +2479190|AAAAAAAAGFENFCAA|2075-09-09|2108|9167|704|2075|1|9|9|3|2075|704|9167|Monday|2075Q3|N|N|N|2479182|2479424|2478825|2479099|N|N|N|N|N| +2479191|AAAAAAAAHFENFCAA|2075-09-10|2108|9168|704|2075|2|9|10|3|2075|704|9168|Tuesday|2075Q3|N|N|N|2479182|2479424|2478826|2479100|N|N|N|N|N| +2479192|AAAAAAAAIFENFCAA|2075-09-11|2108|9168|704|2075|3|9|11|3|2075|704|9168|Wednesday|2075Q3|N|N|N|2479182|2479424|2478827|2479101|N|N|N|N|N| +2479193|AAAAAAAAJFENFCAA|2075-09-12|2108|9168|704|2075|4|9|12|3|2075|704|9168|Thursday|2075Q3|N|N|N|2479182|2479424|2478828|2479102|N|N|N|N|N| +2479194|AAAAAAAAKFENFCAA|2075-09-13|2108|9168|704|2075|5|9|13|3|2075|704|9168|Friday|2075Q3|N|Y|N|2479182|2479424|2478829|2479103|N|N|N|N|N| +2479195|AAAAAAAALFENFCAA|2075-09-14|2108|9168|704|2075|6|9|14|3|2075|704|9168|Saturday|2075Q3|N|Y|N|2479182|2479424|2478830|2479104|N|N|N|N|N| +2479196|AAAAAAAAMFENFCAA|2075-09-15|2108|9168|704|2075|0|9|15|3|2075|704|9168|Sunday|2075Q3|N|N|N|2479182|2479424|2478831|2479105|N|N|N|N|N| +2479197|AAAAAAAANFENFCAA|2075-09-16|2108|9168|704|2075|1|9|16|3|2075|704|9168|Monday|2075Q3|N|N|N|2479182|2479424|2478832|2479106|N|N|N|N|N| +2479198|AAAAAAAAOFENFCAA|2075-09-17|2108|9169|704|2075|2|9|17|3|2075|704|9169|Tuesday|2075Q3|N|N|N|2479182|2479424|2478833|2479107|N|N|N|N|N| +2479199|AAAAAAAAPFENFCAA|2075-09-18|2108|9169|704|2075|3|9|18|3|2075|704|9169|Wednesday|2075Q3|N|N|N|2479182|2479424|2478834|2479108|N|N|N|N|N| +2479200|AAAAAAAAAGENFCAA|2075-09-19|2108|9169|704|2075|4|9|19|3|2075|704|9169|Thursday|2075Q3|N|N|N|2479182|2479424|2478835|2479109|N|N|N|N|N| +2479201|AAAAAAAABGENFCAA|2075-09-20|2108|9169|704|2075|5|9|20|3|2075|704|9169|Friday|2075Q3|N|Y|N|2479182|2479424|2478836|2479110|N|N|N|N|N| +2479202|AAAAAAAACGENFCAA|2075-09-21|2108|9169|704|2075|6|9|21|3|2075|704|9169|Saturday|2075Q3|N|Y|N|2479182|2479424|2478837|2479111|N|N|N|N|N| +2479203|AAAAAAAADGENFCAA|2075-09-22|2108|9169|704|2075|0|9|22|3|2075|704|9169|Sunday|2075Q3|N|N|N|2479182|2479424|2478838|2479112|N|N|N|N|N| +2479204|AAAAAAAAEGENFCAA|2075-09-23|2108|9169|704|2075|1|9|23|3|2075|704|9169|Monday|2075Q3|N|N|N|2479182|2479424|2478839|2479113|N|N|N|N|N| +2479205|AAAAAAAAFGENFCAA|2075-09-24|2108|9170|704|2075|2|9|24|3|2075|704|9170|Tuesday|2075Q3|N|N|N|2479182|2479424|2478840|2479114|N|N|N|N|N| +2479206|AAAAAAAAGGENFCAA|2075-09-25|2108|9170|704|2075|3|9|25|3|2075|704|9170|Wednesday|2075Q3|N|N|N|2479182|2479424|2478841|2479115|N|N|N|N|N| +2479207|AAAAAAAAHGENFCAA|2075-09-26|2108|9170|704|2075|4|9|26|3|2075|704|9170|Thursday|2075Q3|N|N|N|2479182|2479424|2478842|2479116|N|N|N|N|N| +2479208|AAAAAAAAIGENFCAA|2075-09-27|2108|9170|704|2075|5|9|27|3|2075|704|9170|Friday|2075Q3|N|Y|N|2479182|2479424|2478843|2479117|N|N|N|N|N| +2479209|AAAAAAAAJGENFCAA|2075-09-28|2108|9170|704|2075|6|9|28|3|2075|704|9170|Saturday|2075Q3|N|Y|N|2479182|2479424|2478844|2479118|N|N|N|N|N| +2479210|AAAAAAAAKGENFCAA|2075-09-29|2108|9170|704|2075|0|9|29|3|2075|704|9170|Sunday|2075Q3|N|N|N|2479182|2479424|2478845|2479119|N|N|N|N|N| +2479211|AAAAAAAALGENFCAA|2075-09-30|2108|9170|704|2075|1|9|30|3|2075|704|9170|Monday|2075Q3|N|N|N|2479182|2479424|2478846|2479120|N|N|N|N|N| +2479212|AAAAAAAAMGENFCAA|2075-10-01|2109|9171|704|2075|2|10|1|3|2075|704|9171|Tuesday|2075Q3|N|N|N|2479212|2479484|2478847|2479120|N|N|N|N|N| +2479213|AAAAAAAANGENFCAA|2075-10-02|2109|9171|704|2075|3|10|2|4|2075|704|9171|Wednesday|2075Q4|N|N|N|2479212|2479484|2478848|2479121|N|N|N|N|N| +2479214|AAAAAAAAOGENFCAA|2075-10-03|2109|9171|704|2075|4|10|3|4|2075|704|9171|Thursday|2075Q4|N|N|N|2479212|2479484|2478849|2479122|N|N|N|N|N| +2479215|AAAAAAAAPGENFCAA|2075-10-04|2109|9171|704|2075|5|10|4|4|2075|704|9171|Friday|2075Q4|N|Y|N|2479212|2479484|2478850|2479123|N|N|N|N|N| +2479216|AAAAAAAAAHENFCAA|2075-10-05|2109|9171|704|2075|6|10|5|4|2075|704|9171|Saturday|2075Q4|N|Y|N|2479212|2479484|2478851|2479124|N|N|N|N|N| +2479217|AAAAAAAABHENFCAA|2075-10-06|2109|9171|704|2075|0|10|6|4|2075|704|9171|Sunday|2075Q4|N|N|N|2479212|2479484|2478852|2479125|N|N|N|N|N| +2479218|AAAAAAAACHENFCAA|2075-10-07|2109|9171|704|2075|1|10|7|4|2075|704|9171|Monday|2075Q4|N|N|N|2479212|2479484|2478853|2479126|N|N|N|N|N| +2479219|AAAAAAAADHENFCAA|2075-10-08|2109|9172|704|2075|2|10|8|4|2075|704|9172|Tuesday|2075Q4|N|N|N|2479212|2479484|2478854|2479127|N|N|N|N|N| +2479220|AAAAAAAAEHENFCAA|2075-10-09|2109|9172|704|2075|3|10|9|4|2075|704|9172|Wednesday|2075Q4|N|N|N|2479212|2479484|2478855|2479128|N|N|N|N|N| +2479221|AAAAAAAAFHENFCAA|2075-10-10|2109|9172|704|2075|4|10|10|4|2075|704|9172|Thursday|2075Q4|N|N|N|2479212|2479484|2478856|2479129|N|N|N|N|N| +2479222|AAAAAAAAGHENFCAA|2075-10-11|2109|9172|704|2075|5|10|11|4|2075|704|9172|Friday|2075Q4|N|Y|N|2479212|2479484|2478857|2479130|N|N|N|N|N| +2479223|AAAAAAAAHHENFCAA|2075-10-12|2109|9172|704|2075|6|10|12|4|2075|704|9172|Saturday|2075Q4|N|Y|N|2479212|2479484|2478858|2479131|N|N|N|N|N| +2479224|AAAAAAAAIHENFCAA|2075-10-13|2109|9172|704|2075|0|10|13|4|2075|704|9172|Sunday|2075Q4|N|N|N|2479212|2479484|2478859|2479132|N|N|N|N|N| +2479225|AAAAAAAAJHENFCAA|2075-10-14|2109|9172|704|2075|1|10|14|4|2075|704|9172|Monday|2075Q4|N|N|N|2479212|2479484|2478860|2479133|N|N|N|N|N| +2479226|AAAAAAAAKHENFCAA|2075-10-15|2109|9173|704|2075|2|10|15|4|2075|704|9173|Tuesday|2075Q4|N|N|N|2479212|2479484|2478861|2479134|N|N|N|N|N| +2479227|AAAAAAAALHENFCAA|2075-10-16|2109|9173|704|2075|3|10|16|4|2075|704|9173|Wednesday|2075Q4|N|N|N|2479212|2479484|2478862|2479135|N|N|N|N|N| +2479228|AAAAAAAAMHENFCAA|2075-10-17|2109|9173|704|2075|4|10|17|4|2075|704|9173|Thursday|2075Q4|N|N|N|2479212|2479484|2478863|2479136|N|N|N|N|N| +2479229|AAAAAAAANHENFCAA|2075-10-18|2109|9173|704|2075|5|10|18|4|2075|704|9173|Friday|2075Q4|N|Y|N|2479212|2479484|2478864|2479137|N|N|N|N|N| +2479230|AAAAAAAAOHENFCAA|2075-10-19|2109|9173|704|2075|6|10|19|4|2075|704|9173|Saturday|2075Q4|N|Y|N|2479212|2479484|2478865|2479138|N|N|N|N|N| +2479231|AAAAAAAAPHENFCAA|2075-10-20|2109|9173|704|2075|0|10|20|4|2075|704|9173|Sunday|2075Q4|N|N|N|2479212|2479484|2478866|2479139|N|N|N|N|N| +2479232|AAAAAAAAAIENFCAA|2075-10-21|2109|9173|704|2075|1|10|21|4|2075|704|9173|Monday|2075Q4|N|N|N|2479212|2479484|2478867|2479140|N|N|N|N|N| +2479233|AAAAAAAABIENFCAA|2075-10-22|2109|9174|704|2075|2|10|22|4|2075|704|9174|Tuesday|2075Q4|N|N|N|2479212|2479484|2478868|2479141|N|N|N|N|N| +2479234|AAAAAAAACIENFCAA|2075-10-23|2109|9174|704|2075|3|10|23|4|2075|704|9174|Wednesday|2075Q4|N|N|N|2479212|2479484|2478869|2479142|N|N|N|N|N| +2479235|AAAAAAAADIENFCAA|2075-10-24|2109|9174|704|2075|4|10|24|4|2075|704|9174|Thursday|2075Q4|N|N|N|2479212|2479484|2478870|2479143|N|N|N|N|N| +2479236|AAAAAAAAEIENFCAA|2075-10-25|2109|9174|704|2075|5|10|25|4|2075|704|9174|Friday|2075Q4|N|Y|N|2479212|2479484|2478871|2479144|N|N|N|N|N| +2479237|AAAAAAAAFIENFCAA|2075-10-26|2109|9174|704|2075|6|10|26|4|2075|704|9174|Saturday|2075Q4|N|Y|N|2479212|2479484|2478872|2479145|N|N|N|N|N| +2479238|AAAAAAAAGIENFCAA|2075-10-27|2109|9174|704|2075|0|10|27|4|2075|704|9174|Sunday|2075Q4|N|N|N|2479212|2479484|2478873|2479146|N|N|N|N|N| +2479239|AAAAAAAAHIENFCAA|2075-10-28|2109|9174|704|2075|1|10|28|4|2075|704|9174|Monday|2075Q4|N|N|N|2479212|2479484|2478874|2479147|N|N|N|N|N| +2479240|AAAAAAAAIIENFCAA|2075-10-29|2109|9175|704|2075|2|10|29|4|2075|704|9175|Tuesday|2075Q4|N|N|N|2479212|2479484|2478875|2479148|N|N|N|N|N| +2479241|AAAAAAAAJIENFCAA|2075-10-30|2109|9175|704|2075|3|10|30|4|2075|704|9175|Wednesday|2075Q4|N|N|N|2479212|2479484|2478876|2479149|N|N|N|N|N| +2479242|AAAAAAAAKIENFCAA|2075-10-31|2109|9175|704|2075|4|10|31|4|2075|704|9175|Thursday|2075Q4|N|N|N|2479212|2479484|2478877|2479150|N|N|N|N|N| +2479243|AAAAAAAALIENFCAA|2075-11-01|2110|9175|704|2075|5|11|1|4|2075|704|9175|Friday|2075Q4|N|Y|N|2479243|2479546|2478878|2479151|N|N|N|N|N| +2479244|AAAAAAAAMIENFCAA|2075-11-02|2110|9175|704|2075|6|11|2|4|2075|704|9175|Saturday|2075Q4|N|Y|N|2479243|2479546|2478879|2479152|N|N|N|N|N| +2479245|AAAAAAAANIENFCAA|2075-11-03|2110|9175|704|2075|0|11|3|4|2075|704|9175|Sunday|2075Q4|N|N|N|2479243|2479546|2478880|2479153|N|N|N|N|N| +2479246|AAAAAAAAOIENFCAA|2075-11-04|2110|9175|704|2075|1|11|4|4|2075|704|9175|Monday|2075Q4|N|N|N|2479243|2479546|2478881|2479154|N|N|N|N|N| +2479247|AAAAAAAAPIENFCAA|2075-11-05|2110|9176|704|2075|2|11|5|4|2075|704|9176|Tuesday|2075Q4|N|N|N|2479243|2479546|2478882|2479155|N|N|N|N|N| +2479248|AAAAAAAAAJENFCAA|2075-11-06|2110|9176|704|2075|3|11|6|4|2075|704|9176|Wednesday|2075Q4|N|N|N|2479243|2479546|2478883|2479156|N|N|N|N|N| +2479249|AAAAAAAABJENFCAA|2075-11-07|2110|9176|704|2075|4|11|7|4|2075|704|9176|Thursday|2075Q4|N|N|N|2479243|2479546|2478884|2479157|N|N|N|N|N| +2479250|AAAAAAAACJENFCAA|2075-11-08|2110|9176|704|2075|5|11|8|4|2075|704|9176|Friday|2075Q4|N|Y|N|2479243|2479546|2478885|2479158|N|N|N|N|N| +2479251|AAAAAAAADJENFCAA|2075-11-09|2110|9176|704|2075|6|11|9|4|2075|704|9176|Saturday|2075Q4|N|Y|N|2479243|2479546|2478886|2479159|N|N|N|N|N| +2479252|AAAAAAAAEJENFCAA|2075-11-10|2110|9176|704|2075|0|11|10|4|2075|704|9176|Sunday|2075Q4|N|N|N|2479243|2479546|2478887|2479160|N|N|N|N|N| +2479253|AAAAAAAAFJENFCAA|2075-11-11|2110|9176|704|2075|1|11|11|4|2075|704|9176|Monday|2075Q4|N|N|N|2479243|2479546|2478888|2479161|N|N|N|N|N| +2479254|AAAAAAAAGJENFCAA|2075-11-12|2110|9177|704|2075|2|11|12|4|2075|704|9177|Tuesday|2075Q4|N|N|N|2479243|2479546|2478889|2479162|N|N|N|N|N| +2479255|AAAAAAAAHJENFCAA|2075-11-13|2110|9177|704|2075|3|11|13|4|2075|704|9177|Wednesday|2075Q4|N|N|N|2479243|2479546|2478890|2479163|N|N|N|N|N| +2479256|AAAAAAAAIJENFCAA|2075-11-14|2110|9177|704|2075|4|11|14|4|2075|704|9177|Thursday|2075Q4|N|N|N|2479243|2479546|2478891|2479164|N|N|N|N|N| +2479257|AAAAAAAAJJENFCAA|2075-11-15|2110|9177|704|2075|5|11|15|4|2075|704|9177|Friday|2075Q4|N|Y|N|2479243|2479546|2478892|2479165|N|N|N|N|N| +2479258|AAAAAAAAKJENFCAA|2075-11-16|2110|9177|704|2075|6|11|16|4|2075|704|9177|Saturday|2075Q4|N|Y|N|2479243|2479546|2478893|2479166|N|N|N|N|N| +2479259|AAAAAAAALJENFCAA|2075-11-17|2110|9177|704|2075|0|11|17|4|2075|704|9177|Sunday|2075Q4|N|N|N|2479243|2479546|2478894|2479167|N|N|N|N|N| +2479260|AAAAAAAAMJENFCAA|2075-11-18|2110|9177|704|2075|1|11|18|4|2075|704|9177|Monday|2075Q4|N|N|N|2479243|2479546|2478895|2479168|N|N|N|N|N| +2479261|AAAAAAAANJENFCAA|2075-11-19|2110|9178|704|2075|2|11|19|4|2075|704|9178|Tuesday|2075Q4|N|N|N|2479243|2479546|2478896|2479169|N|N|N|N|N| +2479262|AAAAAAAAOJENFCAA|2075-11-20|2110|9178|704|2075|3|11|20|4|2075|704|9178|Wednesday|2075Q4|N|N|N|2479243|2479546|2478897|2479170|N|N|N|N|N| +2479263|AAAAAAAAPJENFCAA|2075-11-21|2110|9178|704|2075|4|11|21|4|2075|704|9178|Thursday|2075Q4|N|N|N|2479243|2479546|2478898|2479171|N|N|N|N|N| +2479264|AAAAAAAAAKENFCAA|2075-11-22|2110|9178|704|2075|5|11|22|4|2075|704|9178|Friday|2075Q4|N|Y|N|2479243|2479546|2478899|2479172|N|N|N|N|N| +2479265|AAAAAAAABKENFCAA|2075-11-23|2110|9178|704|2075|6|11|23|4|2075|704|9178|Saturday|2075Q4|N|Y|N|2479243|2479546|2478900|2479173|N|N|N|N|N| +2479266|AAAAAAAACKENFCAA|2075-11-24|2110|9178|704|2075|0|11|24|4|2075|704|9178|Sunday|2075Q4|N|N|N|2479243|2479546|2478901|2479174|N|N|N|N|N| +2479267|AAAAAAAADKENFCAA|2075-11-25|2110|9178|704|2075|1|11|25|4|2075|704|9178|Monday|2075Q4|N|N|N|2479243|2479546|2478902|2479175|N|N|N|N|N| +2479268|AAAAAAAAEKENFCAA|2075-11-26|2110|9179|704|2075|2|11|26|4|2075|704|9179|Tuesday|2075Q4|N|N|N|2479243|2479546|2478903|2479176|N|N|N|N|N| +2479269|AAAAAAAAFKENFCAA|2075-11-27|2110|9179|704|2075|3|11|27|4|2075|704|9179|Wednesday|2075Q4|N|N|N|2479243|2479546|2478904|2479177|N|N|N|N|N| +2479270|AAAAAAAAGKENFCAA|2075-11-28|2110|9179|704|2075|4|11|28|4|2075|704|9179|Thursday|2075Q4|N|N|N|2479243|2479546|2478905|2479178|N|N|N|N|N| +2479271|AAAAAAAAHKENFCAA|2075-11-29|2110|9179|704|2075|5|11|29|4|2075|704|9179|Friday|2075Q4|N|Y|N|2479243|2479546|2478906|2479179|N|N|N|N|N| +2479272|AAAAAAAAIKENFCAA|2075-11-30|2110|9179|704|2075|6|11|30|4|2075|704|9179|Saturday|2075Q4|N|Y|N|2479243|2479546|2478907|2479180|N|N|N|N|N| +2479273|AAAAAAAAJKENFCAA|2075-12-01|2111|9179|705|2075|0|12|1|4|2075|705|9179|Sunday|2075Q4|N|N|N|2479273|2479606|2478908|2479181|N|N|N|N|N| +2479274|AAAAAAAAKKENFCAA|2075-12-02|2111|9179|705|2075|1|12|2|4|2075|705|9179|Monday|2075Q4|N|N|N|2479273|2479606|2478909|2479182|N|N|N|N|N| +2479275|AAAAAAAALKENFCAA|2075-12-03|2111|9180|705|2075|2|12|3|4|2075|705|9180|Tuesday|2075Q4|N|N|N|2479273|2479606|2478910|2479183|N|N|N|N|N| +2479276|AAAAAAAAMKENFCAA|2075-12-04|2111|9180|705|2075|3|12|4|4|2075|705|9180|Wednesday|2075Q4|N|N|N|2479273|2479606|2478911|2479184|N|N|N|N|N| +2479277|AAAAAAAANKENFCAA|2075-12-05|2111|9180|705|2075|4|12|5|4|2075|705|9180|Thursday|2075Q4|N|N|N|2479273|2479606|2478912|2479185|N|N|N|N|N| +2479278|AAAAAAAAOKENFCAA|2075-12-06|2111|9180|705|2075|5|12|6|4|2075|705|9180|Friday|2075Q4|N|Y|N|2479273|2479606|2478913|2479186|N|N|N|N|N| +2479279|AAAAAAAAPKENFCAA|2075-12-07|2111|9180|705|2075|6|12|7|4|2075|705|9180|Saturday|2075Q4|N|Y|N|2479273|2479606|2478914|2479187|N|N|N|N|N| +2479280|AAAAAAAAALENFCAA|2075-12-08|2111|9180|705|2075|0|12|8|4|2075|705|9180|Sunday|2075Q4|N|N|N|2479273|2479606|2478915|2479188|N|N|N|N|N| +2479281|AAAAAAAABLENFCAA|2075-12-09|2111|9180|705|2075|1|12|9|4|2075|705|9180|Monday|2075Q4|N|N|N|2479273|2479606|2478916|2479189|N|N|N|N|N| +2479282|AAAAAAAACLENFCAA|2075-12-10|2111|9181|705|2075|2|12|10|4|2075|705|9181|Tuesday|2075Q4|N|N|N|2479273|2479606|2478917|2479190|N|N|N|N|N| +2479283|AAAAAAAADLENFCAA|2075-12-11|2111|9181|705|2075|3|12|11|4|2075|705|9181|Wednesday|2075Q4|N|N|N|2479273|2479606|2478918|2479191|N|N|N|N|N| +2479284|AAAAAAAAELENFCAA|2075-12-12|2111|9181|705|2075|4|12|12|4|2075|705|9181|Thursday|2075Q4|N|N|N|2479273|2479606|2478919|2479192|N|N|N|N|N| +2479285|AAAAAAAAFLENFCAA|2075-12-13|2111|9181|705|2075|5|12|13|4|2075|705|9181|Friday|2075Q4|N|Y|N|2479273|2479606|2478920|2479193|N|N|N|N|N| +2479286|AAAAAAAAGLENFCAA|2075-12-14|2111|9181|705|2075|6|12|14|4|2075|705|9181|Saturday|2075Q4|N|Y|N|2479273|2479606|2478921|2479194|N|N|N|N|N| +2479287|AAAAAAAAHLENFCAA|2075-12-15|2111|9181|705|2075|0|12|15|4|2075|705|9181|Sunday|2075Q4|N|N|N|2479273|2479606|2478922|2479195|N|N|N|N|N| +2479288|AAAAAAAAILENFCAA|2075-12-16|2111|9181|705|2075|1|12|16|4|2075|705|9181|Monday|2075Q4|N|N|N|2479273|2479606|2478923|2479196|N|N|N|N|N| +2479289|AAAAAAAAJLENFCAA|2075-12-17|2111|9182|705|2075|2|12|17|4|2075|705|9182|Tuesday|2075Q4|N|N|N|2479273|2479606|2478924|2479197|N|N|N|N|N| +2479290|AAAAAAAAKLENFCAA|2075-12-18|2111|9182|705|2075|3|12|18|4|2075|705|9182|Wednesday|2075Q4|N|N|N|2479273|2479606|2478925|2479198|N|N|N|N|N| +2479291|AAAAAAAALLENFCAA|2075-12-19|2111|9182|705|2075|4|12|19|4|2075|705|9182|Thursday|2075Q4|N|N|N|2479273|2479606|2478926|2479199|N|N|N|N|N| +2479292|AAAAAAAAMLENFCAA|2075-12-20|2111|9182|705|2075|5|12|20|4|2075|705|9182|Friday|2075Q4|N|Y|N|2479273|2479606|2478927|2479200|N|N|N|N|N| +2479293|AAAAAAAANLENFCAA|2075-12-21|2111|9182|705|2075|6|12|21|4|2075|705|9182|Saturday|2075Q4|N|Y|N|2479273|2479606|2478928|2479201|N|N|N|N|N| +2479294|AAAAAAAAOLENFCAA|2075-12-22|2111|9182|705|2075|0|12|22|4|2075|705|9182|Sunday|2075Q4|N|N|N|2479273|2479606|2478929|2479202|N|N|N|N|N| +2479295|AAAAAAAAPLENFCAA|2075-12-23|2111|9182|705|2075|1|12|23|4|2075|705|9182|Monday|2075Q4|N|N|N|2479273|2479606|2478930|2479203|N|N|N|N|N| +2479296|AAAAAAAAAMENFCAA|2075-12-24|2111|9183|705|2075|2|12|24|4|2075|705|9183|Tuesday|2075Q4|N|N|N|2479273|2479606|2478931|2479204|N|N|N|N|N| +2479297|AAAAAAAABMENFCAA|2075-12-25|2111|9183|705|2075|3|12|25|4|2075|705|9183|Wednesday|2075Q4|N|N|N|2479273|2479606|2478932|2479205|N|N|N|N|N| +2479298|AAAAAAAACMENFCAA|2075-12-26|2111|9183|705|2075|4|12|26|4|2075|705|9183|Thursday|2075Q4|Y|N|N|2479273|2479606|2478933|2479206|N|N|N|N|N| +2479299|AAAAAAAADMENFCAA|2075-12-27|2111|9183|705|2075|5|12|27|4|2075|705|9183|Friday|2075Q4|N|Y|Y|2479273|2479606|2478934|2479207|N|N|N|N|N| +2479300|AAAAAAAAEMENFCAA|2075-12-28|2111|9183|705|2075|6|12|28|4|2075|705|9183|Saturday|2075Q4|N|Y|N|2479273|2479606|2478935|2479208|N|N|N|N|N| +2479301|AAAAAAAAFMENFCAA|2075-12-29|2111|9183|705|2075|0|12|29|4|2075|705|9183|Sunday|2075Q4|N|N|N|2479273|2479606|2478936|2479209|N|N|N|N|N| +2479302|AAAAAAAAGMENFCAA|2075-12-30|2111|9183|705|2075|1|12|30|4|2075|705|9183|Monday|2075Q4|N|N|N|2479273|2479606|2478937|2479210|N|N|N|N|N| +2479303|AAAAAAAAHMENFCAA|2075-12-31|2111|9184|705|2075|2|12|31|4|2075|705|9184|Tuesday|2075Q4|N|N|N|2479273|2479606|2478938|2479211|N|N|N|N|N| +2479304|AAAAAAAAIMENFCAA|2076-01-01|2112|9184|705|2076|3|1|1|1|2076|705|9184|Wednesday|2076Q1|Y|N|N|2479304|2479303|2478939|2479212|N|N|N|N|N| +2479305|AAAAAAAAJMENFCAA|2076-01-02|2112|9184|705|2076|4|1|2|1|2076|705|9184|Thursday|2076Q1|N|N|Y|2479304|2479303|2478940|2479213|N|N|N|N|N| +2479306|AAAAAAAAKMENFCAA|2076-01-03|2112|9184|705|2076|5|1|3|1|2076|705|9184|Friday|2076Q1|N|Y|N|2479304|2479303|2478941|2479214|N|N|N|N|N| +2479307|AAAAAAAALMENFCAA|2076-01-04|2112|9184|705|2076|6|1|4|1|2076|705|9184|Saturday|2076Q1|N|Y|N|2479304|2479303|2478942|2479215|N|N|N|N|N| +2479308|AAAAAAAAMMENFCAA|2076-01-05|2112|9184|705|2076|0|1|5|1|2076|705|9184|Sunday|2076Q1|N|N|N|2479304|2479303|2478943|2479216|N|N|N|N|N| +2479309|AAAAAAAANMENFCAA|2076-01-06|2112|9184|705|2076|1|1|6|1|2076|705|9184|Monday|2076Q1|N|N|N|2479304|2479303|2478944|2479217|N|N|N|N|N| +2479310|AAAAAAAAOMENFCAA|2076-01-07|2112|9185|705|2076|2|1|7|1|2076|705|9185|Tuesday|2076Q1|N|N|N|2479304|2479303|2478945|2479218|N|N|N|N|N| +2479311|AAAAAAAAPMENFCAA|2076-01-08|2112|9185|705|2076|3|1|8|1|2076|705|9185|Wednesday|2076Q1|N|N|N|2479304|2479303|2478946|2479219|N|N|N|N|N| +2479312|AAAAAAAAANENFCAA|2076-01-09|2112|9185|705|2076|4|1|9|1|2076|705|9185|Thursday|2076Q1|N|N|N|2479304|2479303|2478947|2479220|N|N|N|N|N| +2479313|AAAAAAAABNENFCAA|2076-01-10|2112|9185|705|2076|5|1|10|1|2076|705|9185|Friday|2076Q1|N|Y|N|2479304|2479303|2478948|2479221|N|N|N|N|N| +2479314|AAAAAAAACNENFCAA|2076-01-11|2112|9185|705|2076|6|1|11|1|2076|705|9185|Saturday|2076Q1|N|Y|N|2479304|2479303|2478949|2479222|N|N|N|N|N| +2479315|AAAAAAAADNENFCAA|2076-01-12|2112|9185|705|2076|0|1|12|1|2076|705|9185|Sunday|2076Q1|N|N|N|2479304|2479303|2478950|2479223|N|N|N|N|N| +2479316|AAAAAAAAENENFCAA|2076-01-13|2112|9185|705|2076|1|1|13|1|2076|705|9185|Monday|2076Q1|N|N|N|2479304|2479303|2478951|2479224|N|N|N|N|N| +2479317|AAAAAAAAFNENFCAA|2076-01-14|2112|9186|705|2076|2|1|14|1|2076|705|9186|Tuesday|2076Q1|N|N|N|2479304|2479303|2478952|2479225|N|N|N|N|N| +2479318|AAAAAAAAGNENFCAA|2076-01-15|2112|9186|705|2076|3|1|15|1|2076|705|9186|Wednesday|2076Q1|N|N|N|2479304|2479303|2478953|2479226|N|N|N|N|N| +2479319|AAAAAAAAHNENFCAA|2076-01-16|2112|9186|705|2076|4|1|16|1|2076|705|9186|Thursday|2076Q1|N|N|N|2479304|2479303|2478954|2479227|N|N|N|N|N| +2479320|AAAAAAAAINENFCAA|2076-01-17|2112|9186|705|2076|5|1|17|1|2076|705|9186|Friday|2076Q1|N|Y|N|2479304|2479303|2478955|2479228|N|N|N|N|N| +2479321|AAAAAAAAJNENFCAA|2076-01-18|2112|9186|705|2076|6|1|18|1|2076|705|9186|Saturday|2076Q1|N|Y|N|2479304|2479303|2478956|2479229|N|N|N|N|N| +2479322|AAAAAAAAKNENFCAA|2076-01-19|2112|9186|705|2076|0|1|19|1|2076|705|9186|Sunday|2076Q1|N|N|N|2479304|2479303|2478957|2479230|N|N|N|N|N| +2479323|AAAAAAAALNENFCAA|2076-01-20|2112|9186|705|2076|1|1|20|1|2076|705|9186|Monday|2076Q1|N|N|N|2479304|2479303|2478958|2479231|N|N|N|N|N| +2479324|AAAAAAAAMNENFCAA|2076-01-21|2112|9187|705|2076|2|1|21|1|2076|705|9187|Tuesday|2076Q1|N|N|N|2479304|2479303|2478959|2479232|N|N|N|N|N| +2479325|AAAAAAAANNENFCAA|2076-01-22|2112|9187|705|2076|3|1|22|1|2076|705|9187|Wednesday|2076Q1|N|N|N|2479304|2479303|2478960|2479233|N|N|N|N|N| +2479326|AAAAAAAAONENFCAA|2076-01-23|2112|9187|705|2076|4|1|23|1|2076|705|9187|Thursday|2076Q1|N|N|N|2479304|2479303|2478961|2479234|N|N|N|N|N| +2479327|AAAAAAAAPNENFCAA|2076-01-24|2112|9187|705|2076|5|1|24|1|2076|705|9187|Friday|2076Q1|N|Y|N|2479304|2479303|2478962|2479235|N|N|N|N|N| +2479328|AAAAAAAAAOENFCAA|2076-01-25|2112|9187|705|2076|6|1|25|1|2076|705|9187|Saturday|2076Q1|N|Y|N|2479304|2479303|2478963|2479236|N|N|N|N|N| +2479329|AAAAAAAABOENFCAA|2076-01-26|2112|9187|705|2076|0|1|26|1|2076|705|9187|Sunday|2076Q1|N|N|N|2479304|2479303|2478964|2479237|N|N|N|N|N| +2479330|AAAAAAAACOENFCAA|2076-01-27|2112|9187|705|2076|1|1|27|1|2076|705|9187|Monday|2076Q1|N|N|N|2479304|2479303|2478965|2479238|N|N|N|N|N| +2479331|AAAAAAAADOENFCAA|2076-01-28|2112|9188|705|2076|2|1|28|1|2076|705|9188|Tuesday|2076Q1|N|N|N|2479304|2479303|2478966|2479239|N|N|N|N|N| +2479332|AAAAAAAAEOENFCAA|2076-01-29|2112|9188|705|2076|3|1|29|1|2076|705|9188|Wednesday|2076Q1|N|N|N|2479304|2479303|2478967|2479240|N|N|N|N|N| +2479333|AAAAAAAAFOENFCAA|2076-01-30|2112|9188|705|2076|4|1|30|1|2076|705|9188|Thursday|2076Q1|N|N|N|2479304|2479303|2478968|2479241|N|N|N|N|N| +2479334|AAAAAAAAGOENFCAA|2076-01-31|2112|9188|705|2076|5|1|31|1|2076|705|9188|Friday|2076Q1|N|Y|N|2479304|2479303|2478969|2479242|N|N|N|N|N| +2479335|AAAAAAAAHOENFCAA|2076-02-01|2113|9188|705|2076|6|2|1|1|2076|705|9188|Saturday|2076Q1|N|Y|N|2479335|2479365|2478970|2479243|N|N|N|N|N| +2479336|AAAAAAAAIOENFCAA|2076-02-02|2113|9188|705|2076|0|2|2|1|2076|705|9188|Sunday|2076Q1|N|N|N|2479335|2479365|2478971|2479244|N|N|N|N|N| +2479337|AAAAAAAAJOENFCAA|2076-02-03|2113|9188|705|2076|1|2|3|1|2076|705|9188|Monday|2076Q1|N|N|N|2479335|2479365|2478972|2479245|N|N|N|N|N| +2479338|AAAAAAAAKOENFCAA|2076-02-04|2113|9189|705|2076|2|2|4|1|2076|705|9189|Tuesday|2076Q1|N|N|N|2479335|2479365|2478973|2479246|N|N|N|N|N| +2479339|AAAAAAAALOENFCAA|2076-02-05|2113|9189|705|2076|3|2|5|1|2076|705|9189|Wednesday|2076Q1|N|N|N|2479335|2479365|2478974|2479247|N|N|N|N|N| +2479340|AAAAAAAAMOENFCAA|2076-02-06|2113|9189|705|2076|4|2|6|1|2076|705|9189|Thursday|2076Q1|N|N|N|2479335|2479365|2478975|2479248|N|N|N|N|N| +2479341|AAAAAAAANOENFCAA|2076-02-07|2113|9189|705|2076|5|2|7|1|2076|705|9189|Friday|2076Q1|N|Y|N|2479335|2479365|2478976|2479249|N|N|N|N|N| +2479342|AAAAAAAAOOENFCAA|2076-02-08|2113|9189|705|2076|6|2|8|1|2076|705|9189|Saturday|2076Q1|N|Y|N|2479335|2479365|2478977|2479250|N|N|N|N|N| +2479343|AAAAAAAAPOENFCAA|2076-02-09|2113|9189|705|2076|0|2|9|1|2076|705|9189|Sunday|2076Q1|N|N|N|2479335|2479365|2478978|2479251|N|N|N|N|N| +2479344|AAAAAAAAAPENFCAA|2076-02-10|2113|9189|705|2076|1|2|10|1|2076|705|9189|Monday|2076Q1|N|N|N|2479335|2479365|2478979|2479252|N|N|N|N|N| +2479345|AAAAAAAABPENFCAA|2076-02-11|2113|9190|705|2076|2|2|11|1|2076|705|9190|Tuesday|2076Q1|N|N|N|2479335|2479365|2478980|2479253|N|N|N|N|N| +2479346|AAAAAAAACPENFCAA|2076-02-12|2113|9190|705|2076|3|2|12|1|2076|705|9190|Wednesday|2076Q1|N|N|N|2479335|2479365|2478981|2479254|N|N|N|N|N| +2479347|AAAAAAAADPENFCAA|2076-02-13|2113|9190|705|2076|4|2|13|1|2076|705|9190|Thursday|2076Q1|N|N|N|2479335|2479365|2478982|2479255|N|N|N|N|N| +2479348|AAAAAAAAEPENFCAA|2076-02-14|2113|9190|705|2076|5|2|14|1|2076|705|9190|Friday|2076Q1|N|Y|N|2479335|2479365|2478983|2479256|N|N|N|N|N| +2479349|AAAAAAAAFPENFCAA|2076-02-15|2113|9190|705|2076|6|2|15|1|2076|705|9190|Saturday|2076Q1|N|Y|N|2479335|2479365|2478984|2479257|N|N|N|N|N| +2479350|AAAAAAAAGPENFCAA|2076-02-16|2113|9190|705|2076|0|2|16|1|2076|705|9190|Sunday|2076Q1|N|N|N|2479335|2479365|2478985|2479258|N|N|N|N|N| +2479351|AAAAAAAAHPENFCAA|2076-02-17|2113|9190|705|2076|1|2|17|1|2076|705|9190|Monday|2076Q1|N|N|N|2479335|2479365|2478986|2479259|N|N|N|N|N| +2479352|AAAAAAAAIPENFCAA|2076-02-18|2113|9191|705|2076|2|2|18|1|2076|705|9191|Tuesday|2076Q1|N|N|N|2479335|2479365|2478987|2479260|N|N|N|N|N| +2479353|AAAAAAAAJPENFCAA|2076-02-19|2113|9191|705|2076|3|2|19|1|2076|705|9191|Wednesday|2076Q1|N|N|N|2479335|2479365|2478988|2479261|N|N|N|N|N| +2479354|AAAAAAAAKPENFCAA|2076-02-20|2113|9191|705|2076|4|2|20|1|2076|705|9191|Thursday|2076Q1|N|N|N|2479335|2479365|2478989|2479262|N|N|N|N|N| +2479355|AAAAAAAALPENFCAA|2076-02-21|2113|9191|705|2076|5|2|21|1|2076|705|9191|Friday|2076Q1|N|Y|N|2479335|2479365|2478990|2479263|N|N|N|N|N| +2479356|AAAAAAAAMPENFCAA|2076-02-22|2113|9191|705|2076|6|2|22|1|2076|705|9191|Saturday|2076Q1|N|Y|N|2479335|2479365|2478991|2479264|N|N|N|N|N| +2479357|AAAAAAAANPENFCAA|2076-02-23|2113|9191|705|2076|0|2|23|1|2076|705|9191|Sunday|2076Q1|N|N|N|2479335|2479365|2478992|2479265|N|N|N|N|N| +2479358|AAAAAAAAOPENFCAA|2076-02-24|2113|9191|705|2076|1|2|24|1|2076|705|9191|Monday|2076Q1|N|N|N|2479335|2479365|2478993|2479266|N|N|N|N|N| +2479359|AAAAAAAAPPENFCAA|2076-02-25|2113|9192|705|2076|2|2|25|1|2076|705|9192|Tuesday|2076Q1|N|N|N|2479335|2479365|2478994|2479267|N|N|N|N|N| +2479360|AAAAAAAAAAFNFCAA|2076-02-26|2113|9192|705|2076|3|2|26|1|2076|705|9192|Wednesday|2076Q1|N|N|N|2479335|2479365|2478995|2479268|N|N|N|N|N| +2479361|AAAAAAAABAFNFCAA|2076-02-27|2113|9192|705|2076|4|2|27|1|2076|705|9192|Thursday|2076Q1|N|N|N|2479335|2479365|2478996|2479269|N|N|N|N|N| +2479362|AAAAAAAACAFNFCAA|2076-02-28|2113|9192|705|2076|5|2|28|1|2076|705|9192|Friday|2076Q1|N|Y|N|2479335|2479365|2478997|2479270|N|N|N|N|N| +2479363|AAAAAAAADAFNFCAA|2076-02-29|2113|9192|705|2076|6|2|29|1|2076|705|9192|Saturday|2076Q1|N|Y|N|2479335|2479365|2478997|2479271|N|N|N|N|N| +2479364|AAAAAAAAEAFNFCAA|2076-03-01|2114|9192|706|2076|0|3|1|1|2076|706|9192|Sunday|2076Q1|N|N|N|2479364|2479423|2478998|2479272|N|N|N|N|N| +2479365|AAAAAAAAFAFNFCAA|2076-03-02|2114|9192|706|2076|1|3|2|1|2076|706|9192|Monday|2076Q1|N|N|N|2479364|2479423|2478999|2479273|N|N|N|N|N| +2479366|AAAAAAAAGAFNFCAA|2076-03-03|2114|9193|706|2076|2|3|3|1|2076|706|9193|Tuesday|2076Q1|N|N|N|2479364|2479423|2479000|2479274|N|N|N|N|N| +2479367|AAAAAAAAHAFNFCAA|2076-03-04|2114|9193|706|2076|3|3|4|1|2076|706|9193|Wednesday|2076Q1|N|N|N|2479364|2479423|2479001|2479275|N|N|N|N|N| +2479368|AAAAAAAAIAFNFCAA|2076-03-05|2114|9193|706|2076|4|3|5|1|2076|706|9193|Thursday|2076Q1|N|N|N|2479364|2479423|2479002|2479276|N|N|N|N|N| +2479369|AAAAAAAAJAFNFCAA|2076-03-06|2114|9193|706|2076|5|3|6|1|2076|706|9193|Friday|2076Q1|N|Y|N|2479364|2479423|2479003|2479277|N|N|N|N|N| +2479370|AAAAAAAAKAFNFCAA|2076-03-07|2114|9193|706|2076|6|3|7|1|2076|706|9193|Saturday|2076Q1|N|Y|N|2479364|2479423|2479004|2479278|N|N|N|N|N| +2479371|AAAAAAAALAFNFCAA|2076-03-08|2114|9193|706|2076|0|3|8|1|2076|706|9193|Sunday|2076Q1|N|N|N|2479364|2479423|2479005|2479279|N|N|N|N|N| +2479372|AAAAAAAAMAFNFCAA|2076-03-09|2114|9193|706|2076|1|3|9|1|2076|706|9193|Monday|2076Q1|N|N|N|2479364|2479423|2479006|2479280|N|N|N|N|N| +2479373|AAAAAAAANAFNFCAA|2076-03-10|2114|9194|706|2076|2|3|10|1|2076|706|9194|Tuesday|2076Q1|N|N|N|2479364|2479423|2479007|2479281|N|N|N|N|N| +2479374|AAAAAAAAOAFNFCAA|2076-03-11|2114|9194|706|2076|3|3|11|1|2076|706|9194|Wednesday|2076Q1|N|N|N|2479364|2479423|2479008|2479282|N|N|N|N|N| +2479375|AAAAAAAAPAFNFCAA|2076-03-12|2114|9194|706|2076|4|3|12|1|2076|706|9194|Thursday|2076Q1|N|N|N|2479364|2479423|2479009|2479283|N|N|N|N|N| +2479376|AAAAAAAAABFNFCAA|2076-03-13|2114|9194|706|2076|5|3|13|1|2076|706|9194|Friday|2076Q1|N|Y|N|2479364|2479423|2479010|2479284|N|N|N|N|N| +2479377|AAAAAAAABBFNFCAA|2076-03-14|2114|9194|706|2076|6|3|14|1|2076|706|9194|Saturday|2076Q1|N|Y|N|2479364|2479423|2479011|2479285|N|N|N|N|N| +2479378|AAAAAAAACBFNFCAA|2076-03-15|2114|9194|706|2076|0|3|15|1|2076|706|9194|Sunday|2076Q1|N|N|N|2479364|2479423|2479012|2479286|N|N|N|N|N| +2479379|AAAAAAAADBFNFCAA|2076-03-16|2114|9194|706|2076|1|3|16|1|2076|706|9194|Monday|2076Q1|N|N|N|2479364|2479423|2479013|2479287|N|N|N|N|N| +2479380|AAAAAAAAEBFNFCAA|2076-03-17|2114|9195|706|2076|2|3|17|1|2076|706|9195|Tuesday|2076Q1|N|N|N|2479364|2479423|2479014|2479288|N|N|N|N|N| +2479381|AAAAAAAAFBFNFCAA|2076-03-18|2114|9195|706|2076|3|3|18|1|2076|706|9195|Wednesday|2076Q1|N|N|N|2479364|2479423|2479015|2479289|N|N|N|N|N| +2479382|AAAAAAAAGBFNFCAA|2076-03-19|2114|9195|706|2076|4|3|19|1|2076|706|9195|Thursday|2076Q1|N|N|N|2479364|2479423|2479016|2479290|N|N|N|N|N| +2479383|AAAAAAAAHBFNFCAA|2076-03-20|2114|9195|706|2076|5|3|20|1|2076|706|9195|Friday|2076Q1|N|Y|N|2479364|2479423|2479017|2479291|N|N|N|N|N| +2479384|AAAAAAAAIBFNFCAA|2076-03-21|2114|9195|706|2076|6|3|21|1|2076|706|9195|Saturday|2076Q1|N|Y|N|2479364|2479423|2479018|2479292|N|N|N|N|N| +2479385|AAAAAAAAJBFNFCAA|2076-03-22|2114|9195|706|2076|0|3|22|1|2076|706|9195|Sunday|2076Q1|N|N|N|2479364|2479423|2479019|2479293|N|N|N|N|N| +2479386|AAAAAAAAKBFNFCAA|2076-03-23|2114|9195|706|2076|1|3|23|1|2076|706|9195|Monday|2076Q1|N|N|N|2479364|2479423|2479020|2479294|N|N|N|N|N| +2479387|AAAAAAAALBFNFCAA|2076-03-24|2114|9196|706|2076|2|3|24|1|2076|706|9196|Tuesday|2076Q1|N|N|N|2479364|2479423|2479021|2479295|N|N|N|N|N| +2479388|AAAAAAAAMBFNFCAA|2076-03-25|2114|9196|706|2076|3|3|25|1|2076|706|9196|Wednesday|2076Q1|N|N|N|2479364|2479423|2479022|2479296|N|N|N|N|N| +2479389|AAAAAAAANBFNFCAA|2076-03-26|2114|9196|706|2076|4|3|26|1|2076|706|9196|Thursday|2076Q1|N|N|N|2479364|2479423|2479023|2479297|N|N|N|N|N| +2479390|AAAAAAAAOBFNFCAA|2076-03-27|2114|9196|706|2076|5|3|27|1|2076|706|9196|Friday|2076Q1|N|Y|N|2479364|2479423|2479024|2479298|N|N|N|N|N| +2479391|AAAAAAAAPBFNFCAA|2076-03-28|2114|9196|706|2076|6|3|28|1|2076|706|9196|Saturday|2076Q1|N|Y|N|2479364|2479423|2479025|2479299|N|N|N|N|N| +2479392|AAAAAAAAACFNFCAA|2076-03-29|2114|9196|706|2076|0|3|29|1|2076|706|9196|Sunday|2076Q1|N|N|N|2479364|2479423|2479026|2479300|N|N|N|N|N| +2479393|AAAAAAAABCFNFCAA|2076-03-30|2114|9196|706|2076|1|3|30|1|2076|706|9196|Monday|2076Q1|N|N|N|2479364|2479423|2479027|2479301|N|N|N|N|N| +2479394|AAAAAAAACCFNFCAA|2076-03-31|2114|9197|706|2076|2|3|31|1|2076|706|9197|Tuesday|2076Q1|N|N|N|2479364|2479423|2479028|2479302|N|N|N|N|N| +2479395|AAAAAAAADCFNFCAA|2076-04-01|2115|9197|706|2076|3|4|1|2|2076|706|9197|Wednesday|2076Q2|N|N|N|2479395|2479485|2479029|2479304|N|N|N|N|N| +2479396|AAAAAAAAECFNFCAA|2076-04-02|2115|9197|706|2076|4|4|2|2|2076|706|9197|Thursday|2076Q2|N|N|N|2479395|2479485|2479030|2479305|N|N|N|N|N| +2479397|AAAAAAAAFCFNFCAA|2076-04-03|2115|9197|706|2076|5|4|3|2|2076|706|9197|Friday|2076Q2|N|Y|N|2479395|2479485|2479031|2479306|N|N|N|N|N| +2479398|AAAAAAAAGCFNFCAA|2076-04-04|2115|9197|706|2076|6|4|4|2|2076|706|9197|Saturday|2076Q2|N|Y|N|2479395|2479485|2479032|2479307|N|N|N|N|N| +2479399|AAAAAAAAHCFNFCAA|2076-04-05|2115|9197|706|2076|0|4|5|2|2076|706|9197|Sunday|2076Q2|N|N|N|2479395|2479485|2479033|2479308|N|N|N|N|N| +2479400|AAAAAAAAICFNFCAA|2076-04-06|2115|9197|706|2076|1|4|6|2|2076|706|9197|Monday|2076Q2|N|N|N|2479395|2479485|2479034|2479309|N|N|N|N|N| +2479401|AAAAAAAAJCFNFCAA|2076-04-07|2115|9198|706|2076|2|4|7|2|2076|706|9198|Tuesday|2076Q2|N|N|N|2479395|2479485|2479035|2479310|N|N|N|N|N| +2479402|AAAAAAAAKCFNFCAA|2076-04-08|2115|9198|706|2076|3|4|8|2|2076|706|9198|Wednesday|2076Q2|N|N|N|2479395|2479485|2479036|2479311|N|N|N|N|N| +2479403|AAAAAAAALCFNFCAA|2076-04-09|2115|9198|706|2076|4|4|9|2|2076|706|9198|Thursday|2076Q2|N|N|N|2479395|2479485|2479037|2479312|N|N|N|N|N| +2479404|AAAAAAAAMCFNFCAA|2076-04-10|2115|9198|706|2076|5|4|10|2|2076|706|9198|Friday|2076Q2|N|Y|N|2479395|2479485|2479038|2479313|N|N|N|N|N| +2479405|AAAAAAAANCFNFCAA|2076-04-11|2115|9198|706|2076|6|4|11|2|2076|706|9198|Saturday|2076Q2|N|Y|N|2479395|2479485|2479039|2479314|N|N|N|N|N| +2479406|AAAAAAAAOCFNFCAA|2076-04-12|2115|9198|706|2076|0|4|12|2|2076|706|9198|Sunday|2076Q2|N|N|N|2479395|2479485|2479040|2479315|N|N|N|N|N| +2479407|AAAAAAAAPCFNFCAA|2076-04-13|2115|9198|706|2076|1|4|13|2|2076|706|9198|Monday|2076Q2|N|N|N|2479395|2479485|2479041|2479316|N|N|N|N|N| +2479408|AAAAAAAAADFNFCAA|2076-04-14|2115|9199|706|2076|2|4|14|2|2076|706|9199|Tuesday|2076Q2|N|N|N|2479395|2479485|2479042|2479317|N|N|N|N|N| +2479409|AAAAAAAABDFNFCAA|2076-04-15|2115|9199|706|2076|3|4|15|2|2076|706|9199|Wednesday|2076Q2|N|N|N|2479395|2479485|2479043|2479318|N|N|N|N|N| +2479410|AAAAAAAACDFNFCAA|2076-04-16|2115|9199|706|2076|4|4|16|2|2076|706|9199|Thursday|2076Q2|N|N|N|2479395|2479485|2479044|2479319|N|N|N|N|N| +2479411|AAAAAAAADDFNFCAA|2076-04-17|2115|9199|706|2076|5|4|17|2|2076|706|9199|Friday|2076Q2|N|Y|N|2479395|2479485|2479045|2479320|N|N|N|N|N| +2479412|AAAAAAAAEDFNFCAA|2076-04-18|2115|9199|706|2076|6|4|18|2|2076|706|9199|Saturday|2076Q2|N|Y|N|2479395|2479485|2479046|2479321|N|N|N|N|N| +2479413|AAAAAAAAFDFNFCAA|2076-04-19|2115|9199|706|2076|0|4|19|2|2076|706|9199|Sunday|2076Q2|N|N|N|2479395|2479485|2479047|2479322|N|N|N|N|N| +2479414|AAAAAAAAGDFNFCAA|2076-04-20|2115|9199|706|2076|1|4|20|2|2076|706|9199|Monday|2076Q2|N|N|N|2479395|2479485|2479048|2479323|N|N|N|N|N| +2479415|AAAAAAAAHDFNFCAA|2076-04-21|2115|9200|706|2076|2|4|21|2|2076|706|9200|Tuesday|2076Q2|N|N|N|2479395|2479485|2479049|2479324|N|N|N|N|N| +2479416|AAAAAAAAIDFNFCAA|2076-04-22|2115|9200|706|2076|3|4|22|2|2076|706|9200|Wednesday|2076Q2|N|N|N|2479395|2479485|2479050|2479325|N|N|N|N|N| +2479417|AAAAAAAAJDFNFCAA|2076-04-23|2115|9200|706|2076|4|4|23|2|2076|706|9200|Thursday|2076Q2|N|N|N|2479395|2479485|2479051|2479326|N|N|N|N|N| +2479418|AAAAAAAAKDFNFCAA|2076-04-24|2115|9200|706|2076|5|4|24|2|2076|706|9200|Friday|2076Q2|N|Y|N|2479395|2479485|2479052|2479327|N|N|N|N|N| +2479419|AAAAAAAALDFNFCAA|2076-04-25|2115|9200|706|2076|6|4|25|2|2076|706|9200|Saturday|2076Q2|N|Y|N|2479395|2479485|2479053|2479328|N|N|N|N|N| +2479420|AAAAAAAAMDFNFCAA|2076-04-26|2115|9200|706|2076|0|4|26|2|2076|706|9200|Sunday|2076Q2|N|N|N|2479395|2479485|2479054|2479329|N|N|N|N|N| +2479421|AAAAAAAANDFNFCAA|2076-04-27|2115|9200|706|2076|1|4|27|2|2076|706|9200|Monday|2076Q2|N|N|N|2479395|2479485|2479055|2479330|N|N|N|N|N| +2479422|AAAAAAAAODFNFCAA|2076-04-28|2115|9201|706|2076|2|4|28|2|2076|706|9201|Tuesday|2076Q2|N|N|N|2479395|2479485|2479056|2479331|N|N|N|N|N| +2479423|AAAAAAAAPDFNFCAA|2076-04-29|2115|9201|706|2076|3|4|29|2|2076|706|9201|Wednesday|2076Q2|N|N|N|2479395|2479485|2479057|2479332|N|N|N|N|N| +2479424|AAAAAAAAAEFNFCAA|2076-04-30|2115|9201|706|2076|4|4|30|2|2076|706|9201|Thursday|2076Q2|N|N|N|2479395|2479485|2479058|2479333|N|N|N|N|N| +2479425|AAAAAAAABEFNFCAA|2076-05-01|2116|9201|706|2076|5|5|1|2|2076|706|9201|Friday|2076Q2|N|Y|N|2479425|2479545|2479059|2479334|N|N|N|N|N| +2479426|AAAAAAAACEFNFCAA|2076-05-02|2116|9201|706|2076|6|5|2|2|2076|706|9201|Saturday|2076Q2|N|Y|N|2479425|2479545|2479060|2479335|N|N|N|N|N| +2479427|AAAAAAAADEFNFCAA|2076-05-03|2116|9201|706|2076|0|5|3|2|2076|706|9201|Sunday|2076Q2|N|N|N|2479425|2479545|2479061|2479336|N|N|N|N|N| +2479428|AAAAAAAAEEFNFCAA|2076-05-04|2116|9201|706|2076|1|5|4|2|2076|706|9201|Monday|2076Q2|N|N|N|2479425|2479545|2479062|2479337|N|N|N|N|N| +2479429|AAAAAAAAFEFNFCAA|2076-05-05|2116|9202|706|2076|2|5|5|2|2076|706|9202|Tuesday|2076Q2|N|N|N|2479425|2479545|2479063|2479338|N|N|N|N|N| +2479430|AAAAAAAAGEFNFCAA|2076-05-06|2116|9202|706|2076|3|5|6|2|2076|706|9202|Wednesday|2076Q2|N|N|N|2479425|2479545|2479064|2479339|N|N|N|N|N| +2479431|AAAAAAAAHEFNFCAA|2076-05-07|2116|9202|706|2076|4|5|7|2|2076|706|9202|Thursday|2076Q2|N|N|N|2479425|2479545|2479065|2479340|N|N|N|N|N| +2479432|AAAAAAAAIEFNFCAA|2076-05-08|2116|9202|706|2076|5|5|8|2|2076|706|9202|Friday|2076Q2|N|Y|N|2479425|2479545|2479066|2479341|N|N|N|N|N| +2479433|AAAAAAAAJEFNFCAA|2076-05-09|2116|9202|706|2076|6|5|9|2|2076|706|9202|Saturday|2076Q2|N|Y|N|2479425|2479545|2479067|2479342|N|N|N|N|N| +2479434|AAAAAAAAKEFNFCAA|2076-05-10|2116|9202|706|2076|0|5|10|2|2076|706|9202|Sunday|2076Q2|N|N|N|2479425|2479545|2479068|2479343|N|N|N|N|N| +2479435|AAAAAAAALEFNFCAA|2076-05-11|2116|9202|706|2076|1|5|11|2|2076|706|9202|Monday|2076Q2|N|N|N|2479425|2479545|2479069|2479344|N|N|N|N|N| +2479436|AAAAAAAAMEFNFCAA|2076-05-12|2116|9203|706|2076|2|5|12|2|2076|706|9203|Tuesday|2076Q2|N|N|N|2479425|2479545|2479070|2479345|N|N|N|N|N| +2479437|AAAAAAAANEFNFCAA|2076-05-13|2116|9203|706|2076|3|5|13|2|2076|706|9203|Wednesday|2076Q2|N|N|N|2479425|2479545|2479071|2479346|N|N|N|N|N| +2479438|AAAAAAAAOEFNFCAA|2076-05-14|2116|9203|706|2076|4|5|14|2|2076|706|9203|Thursday|2076Q2|N|N|N|2479425|2479545|2479072|2479347|N|N|N|N|N| +2479439|AAAAAAAAPEFNFCAA|2076-05-15|2116|9203|706|2076|5|5|15|2|2076|706|9203|Friday|2076Q2|N|Y|N|2479425|2479545|2479073|2479348|N|N|N|N|N| +2479440|AAAAAAAAAFFNFCAA|2076-05-16|2116|9203|706|2076|6|5|16|2|2076|706|9203|Saturday|2076Q2|N|Y|N|2479425|2479545|2479074|2479349|N|N|N|N|N| +2479441|AAAAAAAABFFNFCAA|2076-05-17|2116|9203|706|2076|0|5|17|2|2076|706|9203|Sunday|2076Q2|N|N|N|2479425|2479545|2479075|2479350|N|N|N|N|N| +2479442|AAAAAAAACFFNFCAA|2076-05-18|2116|9203|706|2076|1|5|18|2|2076|706|9203|Monday|2076Q2|N|N|N|2479425|2479545|2479076|2479351|N|N|N|N|N| +2479443|AAAAAAAADFFNFCAA|2076-05-19|2116|9204|706|2076|2|5|19|2|2076|706|9204|Tuesday|2076Q2|N|N|N|2479425|2479545|2479077|2479352|N|N|N|N|N| +2479444|AAAAAAAAEFFNFCAA|2076-05-20|2116|9204|706|2076|3|5|20|2|2076|706|9204|Wednesday|2076Q2|N|N|N|2479425|2479545|2479078|2479353|N|N|N|N|N| +2479445|AAAAAAAAFFFNFCAA|2076-05-21|2116|9204|706|2076|4|5|21|2|2076|706|9204|Thursday|2076Q2|N|N|N|2479425|2479545|2479079|2479354|N|N|N|N|N| +2479446|AAAAAAAAGFFNFCAA|2076-05-22|2116|9204|706|2076|5|5|22|2|2076|706|9204|Friday|2076Q2|N|Y|N|2479425|2479545|2479080|2479355|N|N|N|N|N| +2479447|AAAAAAAAHFFNFCAA|2076-05-23|2116|9204|706|2076|6|5|23|2|2076|706|9204|Saturday|2076Q2|N|Y|N|2479425|2479545|2479081|2479356|N|N|N|N|N| +2479448|AAAAAAAAIFFNFCAA|2076-05-24|2116|9204|706|2076|0|5|24|2|2076|706|9204|Sunday|2076Q2|N|N|N|2479425|2479545|2479082|2479357|N|N|N|N|N| +2479449|AAAAAAAAJFFNFCAA|2076-05-25|2116|9204|706|2076|1|5|25|2|2076|706|9204|Monday|2076Q2|N|N|N|2479425|2479545|2479083|2479358|N|N|N|N|N| +2479450|AAAAAAAAKFFNFCAA|2076-05-26|2116|9205|706|2076|2|5|26|2|2076|706|9205|Tuesday|2076Q2|N|N|N|2479425|2479545|2479084|2479359|N|N|N|N|N| +2479451|AAAAAAAALFFNFCAA|2076-05-27|2116|9205|706|2076|3|5|27|2|2076|706|9205|Wednesday|2076Q2|N|N|N|2479425|2479545|2479085|2479360|N|N|N|N|N| +2479452|AAAAAAAAMFFNFCAA|2076-05-28|2116|9205|706|2076|4|5|28|2|2076|706|9205|Thursday|2076Q2|N|N|N|2479425|2479545|2479086|2479361|N|N|N|N|N| +2479453|AAAAAAAANFFNFCAA|2076-05-29|2116|9205|706|2076|5|5|29|2|2076|706|9205|Friday|2076Q2|N|Y|N|2479425|2479545|2479087|2479362|N|N|N|N|N| +2479454|AAAAAAAAOFFNFCAA|2076-05-30|2116|9205|706|2076|6|5|30|2|2076|706|9205|Saturday|2076Q2|N|Y|N|2479425|2479545|2479088|2479363|N|N|N|N|N| +2479455|AAAAAAAAPFFNFCAA|2076-05-31|2116|9205|706|2076|0|5|31|2|2076|706|9205|Sunday|2076Q2|N|N|N|2479425|2479545|2479089|2479364|N|N|N|N|N| +2479456|AAAAAAAAAGFNFCAA|2076-06-01|2117|9205|707|2076|1|6|1|2|2076|707|9205|Monday|2076Q2|N|N|N|2479456|2479607|2479090|2479365|N|N|N|N|N| +2479457|AAAAAAAABGFNFCAA|2076-06-02|2117|9206|707|2076|2|6|2|2|2076|707|9206|Tuesday|2076Q2|N|N|N|2479456|2479607|2479091|2479366|N|N|N|N|N| +2479458|AAAAAAAACGFNFCAA|2076-06-03|2117|9206|707|2076|3|6|3|2|2076|707|9206|Wednesday|2076Q2|N|N|N|2479456|2479607|2479092|2479367|N|N|N|N|N| +2479459|AAAAAAAADGFNFCAA|2076-06-04|2117|9206|707|2076|4|6|4|2|2076|707|9206|Thursday|2076Q2|N|N|N|2479456|2479607|2479093|2479368|N|N|N|N|N| +2479460|AAAAAAAAEGFNFCAA|2076-06-05|2117|9206|707|2076|5|6|5|2|2076|707|9206|Friday|2076Q2|N|Y|N|2479456|2479607|2479094|2479369|N|N|N|N|N| +2479461|AAAAAAAAFGFNFCAA|2076-06-06|2117|9206|707|2076|6|6|6|2|2076|707|9206|Saturday|2076Q2|N|Y|N|2479456|2479607|2479095|2479370|N|N|N|N|N| +2479462|AAAAAAAAGGFNFCAA|2076-06-07|2117|9206|707|2076|0|6|7|2|2076|707|9206|Sunday|2076Q2|N|N|N|2479456|2479607|2479096|2479371|N|N|N|N|N| +2479463|AAAAAAAAHGFNFCAA|2076-06-08|2117|9206|707|2076|1|6|8|2|2076|707|9206|Monday|2076Q2|N|N|N|2479456|2479607|2479097|2479372|N|N|N|N|N| +2479464|AAAAAAAAIGFNFCAA|2076-06-09|2117|9207|707|2076|2|6|9|2|2076|707|9207|Tuesday|2076Q2|N|N|N|2479456|2479607|2479098|2479373|N|N|N|N|N| +2479465|AAAAAAAAJGFNFCAA|2076-06-10|2117|9207|707|2076|3|6|10|2|2076|707|9207|Wednesday|2076Q2|N|N|N|2479456|2479607|2479099|2479374|N|N|N|N|N| +2479466|AAAAAAAAKGFNFCAA|2076-06-11|2117|9207|707|2076|4|6|11|2|2076|707|9207|Thursday|2076Q2|N|N|N|2479456|2479607|2479100|2479375|N|N|N|N|N| +2479467|AAAAAAAALGFNFCAA|2076-06-12|2117|9207|707|2076|5|6|12|2|2076|707|9207|Friday|2076Q2|N|Y|N|2479456|2479607|2479101|2479376|N|N|N|N|N| +2479468|AAAAAAAAMGFNFCAA|2076-06-13|2117|9207|707|2076|6|6|13|2|2076|707|9207|Saturday|2076Q2|N|Y|N|2479456|2479607|2479102|2479377|N|N|N|N|N| +2479469|AAAAAAAANGFNFCAA|2076-06-14|2117|9207|707|2076|0|6|14|2|2076|707|9207|Sunday|2076Q2|N|N|N|2479456|2479607|2479103|2479378|N|N|N|N|N| +2479470|AAAAAAAAOGFNFCAA|2076-06-15|2117|9207|707|2076|1|6|15|2|2076|707|9207|Monday|2076Q2|N|N|N|2479456|2479607|2479104|2479379|N|N|N|N|N| +2479471|AAAAAAAAPGFNFCAA|2076-06-16|2117|9208|707|2076|2|6|16|2|2076|707|9208|Tuesday|2076Q2|N|N|N|2479456|2479607|2479105|2479380|N|N|N|N|N| +2479472|AAAAAAAAAHFNFCAA|2076-06-17|2117|9208|707|2076|3|6|17|2|2076|707|9208|Wednesday|2076Q2|N|N|N|2479456|2479607|2479106|2479381|N|N|N|N|N| +2479473|AAAAAAAABHFNFCAA|2076-06-18|2117|9208|707|2076|4|6|18|2|2076|707|9208|Thursday|2076Q2|N|N|N|2479456|2479607|2479107|2479382|N|N|N|N|N| +2479474|AAAAAAAACHFNFCAA|2076-06-19|2117|9208|707|2076|5|6|19|2|2076|707|9208|Friday|2076Q2|N|Y|N|2479456|2479607|2479108|2479383|N|N|N|N|N| +2479475|AAAAAAAADHFNFCAA|2076-06-20|2117|9208|707|2076|6|6|20|2|2076|707|9208|Saturday|2076Q2|N|Y|N|2479456|2479607|2479109|2479384|N|N|N|N|N| +2479476|AAAAAAAAEHFNFCAA|2076-06-21|2117|9208|707|2076|0|6|21|2|2076|707|9208|Sunday|2076Q2|N|N|N|2479456|2479607|2479110|2479385|N|N|N|N|N| +2479477|AAAAAAAAFHFNFCAA|2076-06-22|2117|9208|707|2076|1|6|22|2|2076|707|9208|Monday|2076Q2|N|N|N|2479456|2479607|2479111|2479386|N|N|N|N|N| +2479478|AAAAAAAAGHFNFCAA|2076-06-23|2117|9209|707|2076|2|6|23|2|2076|707|9209|Tuesday|2076Q2|N|N|N|2479456|2479607|2479112|2479387|N|N|N|N|N| +2479479|AAAAAAAAHHFNFCAA|2076-06-24|2117|9209|707|2076|3|6|24|2|2076|707|9209|Wednesday|2076Q2|N|N|N|2479456|2479607|2479113|2479388|N|N|N|N|N| +2479480|AAAAAAAAIHFNFCAA|2076-06-25|2117|9209|707|2076|4|6|25|2|2076|707|9209|Thursday|2076Q2|N|N|N|2479456|2479607|2479114|2479389|N|N|N|N|N| +2479481|AAAAAAAAJHFNFCAA|2076-06-26|2117|9209|707|2076|5|6|26|2|2076|707|9209|Friday|2076Q2|N|Y|N|2479456|2479607|2479115|2479390|N|N|N|N|N| +2479482|AAAAAAAAKHFNFCAA|2076-06-27|2117|9209|707|2076|6|6|27|2|2076|707|9209|Saturday|2076Q2|N|Y|N|2479456|2479607|2479116|2479391|N|N|N|N|N| +2479483|AAAAAAAALHFNFCAA|2076-06-28|2117|9209|707|2076|0|6|28|2|2076|707|9209|Sunday|2076Q2|N|N|N|2479456|2479607|2479117|2479392|N|N|N|N|N| +2479484|AAAAAAAAMHFNFCAA|2076-06-29|2117|9209|707|2076|1|6|29|2|2076|707|9209|Monday|2076Q2|N|N|N|2479456|2479607|2479118|2479393|N|N|N|N|N| +2479485|AAAAAAAANHFNFCAA|2076-06-30|2117|9210|707|2076|2|6|30|2|2076|707|9210|Tuesday|2076Q2|N|N|N|2479456|2479607|2479119|2479394|N|N|N|N|N| +2479486|AAAAAAAAOHFNFCAA|2076-07-01|2118|9210|707|2076|3|7|1|3|2076|707|9210|Wednesday|2076Q3|N|N|N|2479486|2479667|2479120|2479395|N|N|N|N|N| +2479487|AAAAAAAAPHFNFCAA|2076-07-02|2118|9210|707|2076|4|7|2|3|2076|707|9210|Thursday|2076Q3|N|N|N|2479486|2479667|2479121|2479396|N|N|N|N|N| +2479488|AAAAAAAAAIFNFCAA|2076-07-03|2118|9210|707|2076|5|7|3|3|2076|707|9210|Friday|2076Q3|N|Y|N|2479486|2479667|2479122|2479397|N|N|N|N|N| +2479489|AAAAAAAABIFNFCAA|2076-07-04|2118|9210|707|2076|6|7|4|3|2076|707|9210|Saturday|2076Q3|Y|Y|N|2479486|2479667|2479123|2479398|N|N|N|N|N| +2479490|AAAAAAAACIFNFCAA|2076-07-05|2118|9210|707|2076|0|7|5|3|2076|707|9210|Sunday|2076Q3|N|N|Y|2479486|2479667|2479124|2479399|N|N|N|N|N| +2479491|AAAAAAAADIFNFCAA|2076-07-06|2118|9210|707|2076|1|7|6|3|2076|707|9210|Monday|2076Q3|N|N|N|2479486|2479667|2479125|2479400|N|N|N|N|N| +2479492|AAAAAAAAEIFNFCAA|2076-07-07|2118|9211|707|2076|2|7|7|3|2076|707|9211|Tuesday|2076Q3|N|N|N|2479486|2479667|2479126|2479401|N|N|N|N|N| +2479493|AAAAAAAAFIFNFCAA|2076-07-08|2118|9211|707|2076|3|7|8|3|2076|707|9211|Wednesday|2076Q3|N|N|N|2479486|2479667|2479127|2479402|N|N|N|N|N| +2479494|AAAAAAAAGIFNFCAA|2076-07-09|2118|9211|707|2076|4|7|9|3|2076|707|9211|Thursday|2076Q3|N|N|N|2479486|2479667|2479128|2479403|N|N|N|N|N| +2479495|AAAAAAAAHIFNFCAA|2076-07-10|2118|9211|707|2076|5|7|10|3|2076|707|9211|Friday|2076Q3|N|Y|N|2479486|2479667|2479129|2479404|N|N|N|N|N| +2479496|AAAAAAAAIIFNFCAA|2076-07-11|2118|9211|707|2076|6|7|11|3|2076|707|9211|Saturday|2076Q3|N|Y|N|2479486|2479667|2479130|2479405|N|N|N|N|N| +2479497|AAAAAAAAJIFNFCAA|2076-07-12|2118|9211|707|2076|0|7|12|3|2076|707|9211|Sunday|2076Q3|N|N|N|2479486|2479667|2479131|2479406|N|N|N|N|N| +2479498|AAAAAAAAKIFNFCAA|2076-07-13|2118|9211|707|2076|1|7|13|3|2076|707|9211|Monday|2076Q3|N|N|N|2479486|2479667|2479132|2479407|N|N|N|N|N| +2479499|AAAAAAAALIFNFCAA|2076-07-14|2118|9212|707|2076|2|7|14|3|2076|707|9212|Tuesday|2076Q3|N|N|N|2479486|2479667|2479133|2479408|N|N|N|N|N| +2479500|AAAAAAAAMIFNFCAA|2076-07-15|2118|9212|707|2076|3|7|15|3|2076|707|9212|Wednesday|2076Q3|N|N|N|2479486|2479667|2479134|2479409|N|N|N|N|N| +2479501|AAAAAAAANIFNFCAA|2076-07-16|2118|9212|707|2076|4|7|16|3|2076|707|9212|Thursday|2076Q3|N|N|N|2479486|2479667|2479135|2479410|N|N|N|N|N| +2479502|AAAAAAAAOIFNFCAA|2076-07-17|2118|9212|707|2076|5|7|17|3|2076|707|9212|Friday|2076Q3|N|Y|N|2479486|2479667|2479136|2479411|N|N|N|N|N| +2479503|AAAAAAAAPIFNFCAA|2076-07-18|2118|9212|707|2076|6|7|18|3|2076|707|9212|Saturday|2076Q3|N|Y|N|2479486|2479667|2479137|2479412|N|N|N|N|N| +2479504|AAAAAAAAAJFNFCAA|2076-07-19|2118|9212|707|2076|0|7|19|3|2076|707|9212|Sunday|2076Q3|N|N|N|2479486|2479667|2479138|2479413|N|N|N|N|N| +2479505|AAAAAAAABJFNFCAA|2076-07-20|2118|9212|707|2076|1|7|20|3|2076|707|9212|Monday|2076Q3|N|N|N|2479486|2479667|2479139|2479414|N|N|N|N|N| +2479506|AAAAAAAACJFNFCAA|2076-07-21|2118|9213|707|2076|2|7|21|3|2076|707|9213|Tuesday|2076Q3|N|N|N|2479486|2479667|2479140|2479415|N|N|N|N|N| +2479507|AAAAAAAADJFNFCAA|2076-07-22|2118|9213|707|2076|3|7|22|3|2076|707|9213|Wednesday|2076Q3|N|N|N|2479486|2479667|2479141|2479416|N|N|N|N|N| +2479508|AAAAAAAAEJFNFCAA|2076-07-23|2118|9213|707|2076|4|7|23|3|2076|707|9213|Thursday|2076Q3|N|N|N|2479486|2479667|2479142|2479417|N|N|N|N|N| +2479509|AAAAAAAAFJFNFCAA|2076-07-24|2118|9213|707|2076|5|7|24|3|2076|707|9213|Friday|2076Q3|N|Y|N|2479486|2479667|2479143|2479418|N|N|N|N|N| +2479510|AAAAAAAAGJFNFCAA|2076-07-25|2118|9213|707|2076|6|7|25|3|2076|707|9213|Saturday|2076Q3|N|Y|N|2479486|2479667|2479144|2479419|N|N|N|N|N| +2479511|AAAAAAAAHJFNFCAA|2076-07-26|2118|9213|707|2076|0|7|26|3|2076|707|9213|Sunday|2076Q3|N|N|N|2479486|2479667|2479145|2479420|N|N|N|N|N| +2479512|AAAAAAAAIJFNFCAA|2076-07-27|2118|9213|707|2076|1|7|27|3|2076|707|9213|Monday|2076Q3|N|N|N|2479486|2479667|2479146|2479421|N|N|N|N|N| +2479513|AAAAAAAAJJFNFCAA|2076-07-28|2118|9214|707|2076|2|7|28|3|2076|707|9214|Tuesday|2076Q3|N|N|N|2479486|2479667|2479147|2479422|N|N|N|N|N| +2479514|AAAAAAAAKJFNFCAA|2076-07-29|2118|9214|707|2076|3|7|29|3|2076|707|9214|Wednesday|2076Q3|N|N|N|2479486|2479667|2479148|2479423|N|N|N|N|N| +2479515|AAAAAAAALJFNFCAA|2076-07-30|2118|9214|707|2076|4|7|30|3|2076|707|9214|Thursday|2076Q3|N|N|N|2479486|2479667|2479149|2479424|N|N|N|N|N| +2479516|AAAAAAAAMJFNFCAA|2076-07-31|2118|9214|707|2076|5|7|31|3|2076|707|9214|Friday|2076Q3|N|Y|N|2479486|2479667|2479150|2479425|N|N|N|N|N| +2479517|AAAAAAAANJFNFCAA|2076-08-01|2119|9214|707|2076|6|8|1|3|2076|707|9214|Saturday|2076Q3|N|Y|N|2479517|2479729|2479151|2479426|N|N|N|N|N| +2479518|AAAAAAAAOJFNFCAA|2076-08-02|2119|9214|707|2076|0|8|2|3|2076|707|9214|Sunday|2076Q3|N|N|N|2479517|2479729|2479152|2479427|N|N|N|N|N| +2479519|AAAAAAAAPJFNFCAA|2076-08-03|2119|9214|707|2076|1|8|3|3|2076|707|9214|Monday|2076Q3|N|N|N|2479517|2479729|2479153|2479428|N|N|N|N|N| +2479520|AAAAAAAAAKFNFCAA|2076-08-04|2119|9215|707|2076|2|8|4|3|2076|707|9215|Tuesday|2076Q3|N|N|N|2479517|2479729|2479154|2479429|N|N|N|N|N| +2479521|AAAAAAAABKFNFCAA|2076-08-05|2119|9215|707|2076|3|8|5|3|2076|707|9215|Wednesday|2076Q3|N|N|N|2479517|2479729|2479155|2479430|N|N|N|N|N| +2479522|AAAAAAAACKFNFCAA|2076-08-06|2119|9215|707|2076|4|8|6|3|2076|707|9215|Thursday|2076Q3|N|N|N|2479517|2479729|2479156|2479431|N|N|N|N|N| +2479523|AAAAAAAADKFNFCAA|2076-08-07|2119|9215|707|2076|5|8|7|3|2076|707|9215|Friday|2076Q3|N|Y|N|2479517|2479729|2479157|2479432|N|N|N|N|N| +2479524|AAAAAAAAEKFNFCAA|2076-08-08|2119|9215|707|2076|6|8|8|3|2076|707|9215|Saturday|2076Q3|N|Y|N|2479517|2479729|2479158|2479433|N|N|N|N|N| +2479525|AAAAAAAAFKFNFCAA|2076-08-09|2119|9215|707|2076|0|8|9|3|2076|707|9215|Sunday|2076Q3|N|N|N|2479517|2479729|2479159|2479434|N|N|N|N|N| +2479526|AAAAAAAAGKFNFCAA|2076-08-10|2119|9215|707|2076|1|8|10|3|2076|707|9215|Monday|2076Q3|N|N|N|2479517|2479729|2479160|2479435|N|N|N|N|N| +2479527|AAAAAAAAHKFNFCAA|2076-08-11|2119|9216|707|2076|2|8|11|3|2076|707|9216|Tuesday|2076Q3|N|N|N|2479517|2479729|2479161|2479436|N|N|N|N|N| +2479528|AAAAAAAAIKFNFCAA|2076-08-12|2119|9216|707|2076|3|8|12|3|2076|707|9216|Wednesday|2076Q3|N|N|N|2479517|2479729|2479162|2479437|N|N|N|N|N| +2479529|AAAAAAAAJKFNFCAA|2076-08-13|2119|9216|707|2076|4|8|13|3|2076|707|9216|Thursday|2076Q3|N|N|N|2479517|2479729|2479163|2479438|N|N|N|N|N| +2479530|AAAAAAAAKKFNFCAA|2076-08-14|2119|9216|707|2076|5|8|14|3|2076|707|9216|Friday|2076Q3|N|Y|N|2479517|2479729|2479164|2479439|N|N|N|N|N| +2479531|AAAAAAAALKFNFCAA|2076-08-15|2119|9216|707|2076|6|8|15|3|2076|707|9216|Saturday|2076Q3|N|Y|N|2479517|2479729|2479165|2479440|N|N|N|N|N| +2479532|AAAAAAAAMKFNFCAA|2076-08-16|2119|9216|707|2076|0|8|16|3|2076|707|9216|Sunday|2076Q3|N|N|N|2479517|2479729|2479166|2479441|N|N|N|N|N| +2479533|AAAAAAAANKFNFCAA|2076-08-17|2119|9216|707|2076|1|8|17|3|2076|707|9216|Monday|2076Q3|N|N|N|2479517|2479729|2479167|2479442|N|N|N|N|N| +2479534|AAAAAAAAOKFNFCAA|2076-08-18|2119|9217|707|2076|2|8|18|3|2076|707|9217|Tuesday|2076Q3|N|N|N|2479517|2479729|2479168|2479443|N|N|N|N|N| +2479535|AAAAAAAAPKFNFCAA|2076-08-19|2119|9217|707|2076|3|8|19|3|2076|707|9217|Wednesday|2076Q3|N|N|N|2479517|2479729|2479169|2479444|N|N|N|N|N| +2479536|AAAAAAAAALFNFCAA|2076-08-20|2119|9217|707|2076|4|8|20|3|2076|707|9217|Thursday|2076Q3|N|N|N|2479517|2479729|2479170|2479445|N|N|N|N|N| +2479537|AAAAAAAABLFNFCAA|2076-08-21|2119|9217|707|2076|5|8|21|3|2076|707|9217|Friday|2076Q3|N|Y|N|2479517|2479729|2479171|2479446|N|N|N|N|N| +2479538|AAAAAAAACLFNFCAA|2076-08-22|2119|9217|707|2076|6|8|22|3|2076|707|9217|Saturday|2076Q3|N|Y|N|2479517|2479729|2479172|2479447|N|N|N|N|N| +2479539|AAAAAAAADLFNFCAA|2076-08-23|2119|9217|707|2076|0|8|23|3|2076|707|9217|Sunday|2076Q3|N|N|N|2479517|2479729|2479173|2479448|N|N|N|N|N| +2479540|AAAAAAAAELFNFCAA|2076-08-24|2119|9217|707|2076|1|8|24|3|2076|707|9217|Monday|2076Q3|N|N|N|2479517|2479729|2479174|2479449|N|N|N|N|N| +2479541|AAAAAAAAFLFNFCAA|2076-08-25|2119|9218|707|2076|2|8|25|3|2076|707|9218|Tuesday|2076Q3|N|N|N|2479517|2479729|2479175|2479450|N|N|N|N|N| +2479542|AAAAAAAAGLFNFCAA|2076-08-26|2119|9218|707|2076|3|8|26|3|2076|707|9218|Wednesday|2076Q3|N|N|N|2479517|2479729|2479176|2479451|N|N|N|N|N| +2479543|AAAAAAAAHLFNFCAA|2076-08-27|2119|9218|707|2076|4|8|27|3|2076|707|9218|Thursday|2076Q3|N|N|N|2479517|2479729|2479177|2479452|N|N|N|N|N| +2479544|AAAAAAAAILFNFCAA|2076-08-28|2119|9218|707|2076|5|8|28|3|2076|707|9218|Friday|2076Q3|N|Y|N|2479517|2479729|2479178|2479453|N|N|N|N|N| +2479545|AAAAAAAAJLFNFCAA|2076-08-29|2119|9218|707|2076|6|8|29|3|2076|707|9218|Saturday|2076Q3|N|Y|N|2479517|2479729|2479179|2479454|N|N|N|N|N| +2479546|AAAAAAAAKLFNFCAA|2076-08-30|2119|9218|707|2076|0|8|30|3|2076|707|9218|Sunday|2076Q3|N|N|N|2479517|2479729|2479180|2479455|N|N|N|N|N| +2479547|AAAAAAAALLFNFCAA|2076-08-31|2119|9218|707|2076|1|8|31|3|2076|707|9218|Monday|2076Q3|N|N|N|2479517|2479729|2479181|2479456|N|N|N|N|N| +2479548|AAAAAAAAMLFNFCAA|2076-09-01|2120|9219|708|2076|2|9|1|3|2076|708|9219|Tuesday|2076Q3|N|N|N|2479548|2479791|2479182|2479457|N|N|N|N|N| +2479549|AAAAAAAANLFNFCAA|2076-09-02|2120|9219|708|2076|3|9|2|3|2076|708|9219|Wednesday|2076Q3|N|N|N|2479548|2479791|2479183|2479458|N|N|N|N|N| +2479550|AAAAAAAAOLFNFCAA|2076-09-03|2120|9219|708|2076|4|9|3|3|2076|708|9219|Thursday|2076Q3|N|N|N|2479548|2479791|2479184|2479459|N|N|N|N|N| +2479551|AAAAAAAAPLFNFCAA|2076-09-04|2120|9219|708|2076|5|9|4|3|2076|708|9219|Friday|2076Q3|N|Y|N|2479548|2479791|2479185|2479460|N|N|N|N|N| +2479552|AAAAAAAAAMFNFCAA|2076-09-05|2120|9219|708|2076|6|9|5|3|2076|708|9219|Saturday|2076Q3|N|Y|N|2479548|2479791|2479186|2479461|N|N|N|N|N| +2479553|AAAAAAAABMFNFCAA|2076-09-06|2120|9219|708|2076|0|9|6|3|2076|708|9219|Sunday|2076Q3|N|N|N|2479548|2479791|2479187|2479462|N|N|N|N|N| +2479554|AAAAAAAACMFNFCAA|2076-09-07|2120|9219|708|2076|1|9|7|3|2076|708|9219|Monday|2076Q3|N|N|N|2479548|2479791|2479188|2479463|N|N|N|N|N| +2479555|AAAAAAAADMFNFCAA|2076-09-08|2120|9220|708|2076|2|9|8|3|2076|708|9220|Tuesday|2076Q3|N|N|N|2479548|2479791|2479189|2479464|N|N|N|N|N| +2479556|AAAAAAAAEMFNFCAA|2076-09-09|2120|9220|708|2076|3|9|9|3|2076|708|9220|Wednesday|2076Q3|N|N|N|2479548|2479791|2479190|2479465|N|N|N|N|N| +2479557|AAAAAAAAFMFNFCAA|2076-09-10|2120|9220|708|2076|4|9|10|3|2076|708|9220|Thursday|2076Q3|N|N|N|2479548|2479791|2479191|2479466|N|N|N|N|N| +2479558|AAAAAAAAGMFNFCAA|2076-09-11|2120|9220|708|2076|5|9|11|3|2076|708|9220|Friday|2076Q3|N|Y|N|2479548|2479791|2479192|2479467|N|N|N|N|N| +2479559|AAAAAAAAHMFNFCAA|2076-09-12|2120|9220|708|2076|6|9|12|3|2076|708|9220|Saturday|2076Q3|N|Y|N|2479548|2479791|2479193|2479468|N|N|N|N|N| +2479560|AAAAAAAAIMFNFCAA|2076-09-13|2120|9220|708|2076|0|9|13|3|2076|708|9220|Sunday|2076Q3|N|N|N|2479548|2479791|2479194|2479469|N|N|N|N|N| +2479561|AAAAAAAAJMFNFCAA|2076-09-14|2120|9220|708|2076|1|9|14|3|2076|708|9220|Monday|2076Q3|N|N|N|2479548|2479791|2479195|2479470|N|N|N|N|N| +2479562|AAAAAAAAKMFNFCAA|2076-09-15|2120|9221|708|2076|2|9|15|3|2076|708|9221|Tuesday|2076Q3|N|N|N|2479548|2479791|2479196|2479471|N|N|N|N|N| +2479563|AAAAAAAALMFNFCAA|2076-09-16|2120|9221|708|2076|3|9|16|3|2076|708|9221|Wednesday|2076Q3|N|N|N|2479548|2479791|2479197|2479472|N|N|N|N|N| +2479564|AAAAAAAAMMFNFCAA|2076-09-17|2120|9221|708|2076|4|9|17|3|2076|708|9221|Thursday|2076Q3|N|N|N|2479548|2479791|2479198|2479473|N|N|N|N|N| +2479565|AAAAAAAANMFNFCAA|2076-09-18|2120|9221|708|2076|5|9|18|3|2076|708|9221|Friday|2076Q3|N|Y|N|2479548|2479791|2479199|2479474|N|N|N|N|N| +2479566|AAAAAAAAOMFNFCAA|2076-09-19|2120|9221|708|2076|6|9|19|3|2076|708|9221|Saturday|2076Q3|N|Y|N|2479548|2479791|2479200|2479475|N|N|N|N|N| +2479567|AAAAAAAAPMFNFCAA|2076-09-20|2120|9221|708|2076|0|9|20|3|2076|708|9221|Sunday|2076Q3|N|N|N|2479548|2479791|2479201|2479476|N|N|N|N|N| +2479568|AAAAAAAAANFNFCAA|2076-09-21|2120|9221|708|2076|1|9|21|3|2076|708|9221|Monday|2076Q3|N|N|N|2479548|2479791|2479202|2479477|N|N|N|N|N| +2479569|AAAAAAAABNFNFCAA|2076-09-22|2120|9222|708|2076|2|9|22|3|2076|708|9222|Tuesday|2076Q3|N|N|N|2479548|2479791|2479203|2479478|N|N|N|N|N| +2479570|AAAAAAAACNFNFCAA|2076-09-23|2120|9222|708|2076|3|9|23|3|2076|708|9222|Wednesday|2076Q3|N|N|N|2479548|2479791|2479204|2479479|N|N|N|N|N| +2479571|AAAAAAAADNFNFCAA|2076-09-24|2120|9222|708|2076|4|9|24|3|2076|708|9222|Thursday|2076Q3|N|N|N|2479548|2479791|2479205|2479480|N|N|N|N|N| +2479572|AAAAAAAAENFNFCAA|2076-09-25|2120|9222|708|2076|5|9|25|3|2076|708|9222|Friday|2076Q3|N|Y|N|2479548|2479791|2479206|2479481|N|N|N|N|N| +2479573|AAAAAAAAFNFNFCAA|2076-09-26|2120|9222|708|2076|6|9|26|3|2076|708|9222|Saturday|2076Q3|N|Y|N|2479548|2479791|2479207|2479482|N|N|N|N|N| +2479574|AAAAAAAAGNFNFCAA|2076-09-27|2120|9222|708|2076|0|9|27|3|2076|708|9222|Sunday|2076Q3|N|N|N|2479548|2479791|2479208|2479483|N|N|N|N|N| +2479575|AAAAAAAAHNFNFCAA|2076-09-28|2120|9222|708|2076|1|9|28|3|2076|708|9222|Monday|2076Q3|N|N|N|2479548|2479791|2479209|2479484|N|N|N|N|N| +2479576|AAAAAAAAINFNFCAA|2076-09-29|2120|9223|708|2076|2|9|29|3|2076|708|9223|Tuesday|2076Q3|N|N|N|2479548|2479791|2479210|2479485|N|N|N|N|N| +2479577|AAAAAAAAJNFNFCAA|2076-09-30|2120|9223|708|2076|3|9|30|3|2076|708|9223|Wednesday|2076Q3|N|N|N|2479548|2479791|2479211|2479486|N|N|N|N|N| +2479578|AAAAAAAAKNFNFCAA|2076-10-01|2121|9223|708|2076|4|10|1|4|2076|708|9223|Thursday|2076Q4|N|N|N|2479578|2479851|2479212|2479486|N|N|N|N|N| +2479579|AAAAAAAALNFNFCAA|2076-10-02|2121|9223|708|2076|5|10|2|4|2076|708|9223|Friday|2076Q4|N|Y|N|2479578|2479851|2479213|2479487|N|N|N|N|N| +2479580|AAAAAAAAMNFNFCAA|2076-10-03|2121|9223|708|2076|6|10|3|4|2076|708|9223|Saturday|2076Q4|N|Y|N|2479578|2479851|2479214|2479488|N|N|N|N|N| +2479581|AAAAAAAANNFNFCAA|2076-10-04|2121|9223|708|2076|0|10|4|4|2076|708|9223|Sunday|2076Q4|N|N|N|2479578|2479851|2479215|2479489|N|N|N|N|N| +2479582|AAAAAAAAONFNFCAA|2076-10-05|2121|9223|708|2076|1|10|5|4|2076|708|9223|Monday|2076Q4|N|N|N|2479578|2479851|2479216|2479490|N|N|N|N|N| +2479583|AAAAAAAAPNFNFCAA|2076-10-06|2121|9224|708|2076|2|10|6|4|2076|708|9224|Tuesday|2076Q4|N|N|N|2479578|2479851|2479217|2479491|N|N|N|N|N| +2479584|AAAAAAAAAOFNFCAA|2076-10-07|2121|9224|708|2076|3|10|7|4|2076|708|9224|Wednesday|2076Q4|N|N|N|2479578|2479851|2479218|2479492|N|N|N|N|N| +2479585|AAAAAAAABOFNFCAA|2076-10-08|2121|9224|708|2076|4|10|8|4|2076|708|9224|Thursday|2076Q4|N|N|N|2479578|2479851|2479219|2479493|N|N|N|N|N| +2479586|AAAAAAAACOFNFCAA|2076-10-09|2121|9224|708|2076|5|10|9|4|2076|708|9224|Friday|2076Q4|N|Y|N|2479578|2479851|2479220|2479494|N|N|N|N|N| +2479587|AAAAAAAADOFNFCAA|2076-10-10|2121|9224|708|2076|6|10|10|4|2076|708|9224|Saturday|2076Q4|N|Y|N|2479578|2479851|2479221|2479495|N|N|N|N|N| +2479588|AAAAAAAAEOFNFCAA|2076-10-11|2121|9224|708|2076|0|10|11|4|2076|708|9224|Sunday|2076Q4|N|N|N|2479578|2479851|2479222|2479496|N|N|N|N|N| +2479589|AAAAAAAAFOFNFCAA|2076-10-12|2121|9224|708|2076|1|10|12|4|2076|708|9224|Monday|2076Q4|N|N|N|2479578|2479851|2479223|2479497|N|N|N|N|N| +2479590|AAAAAAAAGOFNFCAA|2076-10-13|2121|9225|708|2076|2|10|13|4|2076|708|9225|Tuesday|2076Q4|N|N|N|2479578|2479851|2479224|2479498|N|N|N|N|N| +2479591|AAAAAAAAHOFNFCAA|2076-10-14|2121|9225|708|2076|3|10|14|4|2076|708|9225|Wednesday|2076Q4|N|N|N|2479578|2479851|2479225|2479499|N|N|N|N|N| +2479592|AAAAAAAAIOFNFCAA|2076-10-15|2121|9225|708|2076|4|10|15|4|2076|708|9225|Thursday|2076Q4|N|N|N|2479578|2479851|2479226|2479500|N|N|N|N|N| +2479593|AAAAAAAAJOFNFCAA|2076-10-16|2121|9225|708|2076|5|10|16|4|2076|708|9225|Friday|2076Q4|N|Y|N|2479578|2479851|2479227|2479501|N|N|N|N|N| +2479594|AAAAAAAAKOFNFCAA|2076-10-17|2121|9225|708|2076|6|10|17|4|2076|708|9225|Saturday|2076Q4|N|Y|N|2479578|2479851|2479228|2479502|N|N|N|N|N| +2479595|AAAAAAAALOFNFCAA|2076-10-18|2121|9225|708|2076|0|10|18|4|2076|708|9225|Sunday|2076Q4|N|N|N|2479578|2479851|2479229|2479503|N|N|N|N|N| +2479596|AAAAAAAAMOFNFCAA|2076-10-19|2121|9225|708|2076|1|10|19|4|2076|708|9225|Monday|2076Q4|N|N|N|2479578|2479851|2479230|2479504|N|N|N|N|N| +2479597|AAAAAAAANOFNFCAA|2076-10-20|2121|9226|708|2076|2|10|20|4|2076|708|9226|Tuesday|2076Q4|N|N|N|2479578|2479851|2479231|2479505|N|N|N|N|N| +2479598|AAAAAAAAOOFNFCAA|2076-10-21|2121|9226|708|2076|3|10|21|4|2076|708|9226|Wednesday|2076Q4|N|N|N|2479578|2479851|2479232|2479506|N|N|N|N|N| +2479599|AAAAAAAAPOFNFCAA|2076-10-22|2121|9226|708|2076|4|10|22|4|2076|708|9226|Thursday|2076Q4|N|N|N|2479578|2479851|2479233|2479507|N|N|N|N|N| +2479600|AAAAAAAAAPFNFCAA|2076-10-23|2121|9226|708|2076|5|10|23|4|2076|708|9226|Friday|2076Q4|N|Y|N|2479578|2479851|2479234|2479508|N|N|N|N|N| +2479601|AAAAAAAABPFNFCAA|2076-10-24|2121|9226|708|2076|6|10|24|4|2076|708|9226|Saturday|2076Q4|N|Y|N|2479578|2479851|2479235|2479509|N|N|N|N|N| +2479602|AAAAAAAACPFNFCAA|2076-10-25|2121|9226|708|2076|0|10|25|4|2076|708|9226|Sunday|2076Q4|N|N|N|2479578|2479851|2479236|2479510|N|N|N|N|N| +2479603|AAAAAAAADPFNFCAA|2076-10-26|2121|9226|708|2076|1|10|26|4|2076|708|9226|Monday|2076Q4|N|N|N|2479578|2479851|2479237|2479511|N|N|N|N|N| +2479604|AAAAAAAAEPFNFCAA|2076-10-27|2121|9227|708|2076|2|10|27|4|2076|708|9227|Tuesday|2076Q4|N|N|N|2479578|2479851|2479238|2479512|N|N|N|N|N| +2479605|AAAAAAAAFPFNFCAA|2076-10-28|2121|9227|708|2076|3|10|28|4|2076|708|9227|Wednesday|2076Q4|N|N|N|2479578|2479851|2479239|2479513|N|N|N|N|N| +2479606|AAAAAAAAGPFNFCAA|2076-10-29|2121|9227|708|2076|4|10|29|4|2076|708|9227|Thursday|2076Q4|N|N|N|2479578|2479851|2479240|2479514|N|N|N|N|N| +2479607|AAAAAAAAHPFNFCAA|2076-10-30|2121|9227|708|2076|5|10|30|4|2076|708|9227|Friday|2076Q4|N|Y|N|2479578|2479851|2479241|2479515|N|N|N|N|N| +2479608|AAAAAAAAIPFNFCAA|2076-10-31|2121|9227|708|2076|6|10|31|4|2076|708|9227|Saturday|2076Q4|N|Y|N|2479578|2479851|2479242|2479516|N|N|N|N|N| +2479609|AAAAAAAAJPFNFCAA|2076-11-01|2122|9227|708|2076|0|11|1|4|2076|708|9227|Sunday|2076Q4|N|N|N|2479609|2479913|2479243|2479517|N|N|N|N|N| +2479610|AAAAAAAAKPFNFCAA|2076-11-02|2122|9227|708|2076|1|11|2|4|2076|708|9227|Monday|2076Q4|N|N|N|2479609|2479913|2479244|2479518|N|N|N|N|N| +2479611|AAAAAAAALPFNFCAA|2076-11-03|2122|9228|708|2076|2|11|3|4|2076|708|9228|Tuesday|2076Q4|N|N|N|2479609|2479913|2479245|2479519|N|N|N|N|N| +2479612|AAAAAAAAMPFNFCAA|2076-11-04|2122|9228|708|2076|3|11|4|4|2076|708|9228|Wednesday|2076Q4|N|N|N|2479609|2479913|2479246|2479520|N|N|N|N|N| +2479613|AAAAAAAANPFNFCAA|2076-11-05|2122|9228|708|2076|4|11|5|4|2076|708|9228|Thursday|2076Q4|N|N|N|2479609|2479913|2479247|2479521|N|N|N|N|N| +2479614|AAAAAAAAOPFNFCAA|2076-11-06|2122|9228|708|2076|5|11|6|4|2076|708|9228|Friday|2076Q4|N|Y|N|2479609|2479913|2479248|2479522|N|N|N|N|N| +2479615|AAAAAAAAPPFNFCAA|2076-11-07|2122|9228|708|2076|6|11|7|4|2076|708|9228|Saturday|2076Q4|N|Y|N|2479609|2479913|2479249|2479523|N|N|N|N|N| +2479616|AAAAAAAAAAGNFCAA|2076-11-08|2122|9228|708|2076|0|11|8|4|2076|708|9228|Sunday|2076Q4|N|N|N|2479609|2479913|2479250|2479524|N|N|N|N|N| +2479617|AAAAAAAABAGNFCAA|2076-11-09|2122|9228|708|2076|1|11|9|4|2076|708|9228|Monday|2076Q4|N|N|N|2479609|2479913|2479251|2479525|N|N|N|N|N| +2479618|AAAAAAAACAGNFCAA|2076-11-10|2122|9229|708|2076|2|11|10|4|2076|708|9229|Tuesday|2076Q4|N|N|N|2479609|2479913|2479252|2479526|N|N|N|N|N| +2479619|AAAAAAAADAGNFCAA|2076-11-11|2122|9229|708|2076|3|11|11|4|2076|708|9229|Wednesday|2076Q4|N|N|N|2479609|2479913|2479253|2479527|N|N|N|N|N| +2479620|AAAAAAAAEAGNFCAA|2076-11-12|2122|9229|708|2076|4|11|12|4|2076|708|9229|Thursday|2076Q4|N|N|N|2479609|2479913|2479254|2479528|N|N|N|N|N| +2479621|AAAAAAAAFAGNFCAA|2076-11-13|2122|9229|708|2076|5|11|13|4|2076|708|9229|Friday|2076Q4|N|Y|N|2479609|2479913|2479255|2479529|N|N|N|N|N| +2479622|AAAAAAAAGAGNFCAA|2076-11-14|2122|9229|708|2076|6|11|14|4|2076|708|9229|Saturday|2076Q4|N|Y|N|2479609|2479913|2479256|2479530|N|N|N|N|N| +2479623|AAAAAAAAHAGNFCAA|2076-11-15|2122|9229|708|2076|0|11|15|4|2076|708|9229|Sunday|2076Q4|N|N|N|2479609|2479913|2479257|2479531|N|N|N|N|N| +2479624|AAAAAAAAIAGNFCAA|2076-11-16|2122|9229|708|2076|1|11|16|4|2076|708|9229|Monday|2076Q4|N|N|N|2479609|2479913|2479258|2479532|N|N|N|N|N| +2479625|AAAAAAAAJAGNFCAA|2076-11-17|2122|9230|708|2076|2|11|17|4|2076|708|9230|Tuesday|2076Q4|N|N|N|2479609|2479913|2479259|2479533|N|N|N|N|N| +2479626|AAAAAAAAKAGNFCAA|2076-11-18|2122|9230|708|2076|3|11|18|4|2076|708|9230|Wednesday|2076Q4|N|N|N|2479609|2479913|2479260|2479534|N|N|N|N|N| +2479627|AAAAAAAALAGNFCAA|2076-11-19|2122|9230|708|2076|4|11|19|4|2076|708|9230|Thursday|2076Q4|N|N|N|2479609|2479913|2479261|2479535|N|N|N|N|N| +2479628|AAAAAAAAMAGNFCAA|2076-11-20|2122|9230|708|2076|5|11|20|4|2076|708|9230|Friday|2076Q4|N|Y|N|2479609|2479913|2479262|2479536|N|N|N|N|N| +2479629|AAAAAAAANAGNFCAA|2076-11-21|2122|9230|708|2076|6|11|21|4|2076|708|9230|Saturday|2076Q4|N|Y|N|2479609|2479913|2479263|2479537|N|N|N|N|N| +2479630|AAAAAAAAOAGNFCAA|2076-11-22|2122|9230|708|2076|0|11|22|4|2076|708|9230|Sunday|2076Q4|N|N|N|2479609|2479913|2479264|2479538|N|N|N|N|N| +2479631|AAAAAAAAPAGNFCAA|2076-11-23|2122|9230|708|2076|1|11|23|4|2076|708|9230|Monday|2076Q4|N|N|N|2479609|2479913|2479265|2479539|N|N|N|N|N| +2479632|AAAAAAAAABGNFCAA|2076-11-24|2122|9231|708|2076|2|11|24|4|2076|708|9231|Tuesday|2076Q4|N|N|N|2479609|2479913|2479266|2479540|N|N|N|N|N| +2479633|AAAAAAAABBGNFCAA|2076-11-25|2122|9231|708|2076|3|11|25|4|2076|708|9231|Wednesday|2076Q4|N|N|N|2479609|2479913|2479267|2479541|N|N|N|N|N| +2479634|AAAAAAAACBGNFCAA|2076-11-26|2122|9231|708|2076|4|11|26|4|2076|708|9231|Thursday|2076Q4|N|N|N|2479609|2479913|2479268|2479542|N|N|N|N|N| +2479635|AAAAAAAADBGNFCAA|2076-11-27|2122|9231|708|2076|5|11|27|4|2076|708|9231|Friday|2076Q4|N|Y|N|2479609|2479913|2479269|2479543|N|N|N|N|N| +2479636|AAAAAAAAEBGNFCAA|2076-11-28|2122|9231|708|2076|6|11|28|4|2076|708|9231|Saturday|2076Q4|N|Y|N|2479609|2479913|2479270|2479544|N|N|N|N|N| +2479637|AAAAAAAAFBGNFCAA|2076-11-29|2122|9231|708|2076|0|11|29|4|2076|708|9231|Sunday|2076Q4|N|N|N|2479609|2479913|2479271|2479545|N|N|N|N|N| +2479638|AAAAAAAAGBGNFCAA|2076-11-30|2122|9231|708|2076|1|11|30|4|2076|708|9231|Monday|2076Q4|N|N|N|2479609|2479913|2479272|2479546|N|N|N|N|N| +2479639|AAAAAAAAHBGNFCAA|2076-12-01|2123|9232|709|2076|2|12|1|4|2076|709|9232|Tuesday|2076Q4|N|N|N|2479639|2479973|2479273|2479547|N|N|N|N|N| +2479640|AAAAAAAAIBGNFCAA|2076-12-02|2123|9232|709|2076|3|12|2|4|2076|709|9232|Wednesday|2076Q4|N|N|N|2479639|2479973|2479274|2479548|N|N|N|N|N| +2479641|AAAAAAAAJBGNFCAA|2076-12-03|2123|9232|709|2076|4|12|3|4|2076|709|9232|Thursday|2076Q4|N|N|N|2479639|2479973|2479275|2479549|N|N|N|N|N| +2479642|AAAAAAAAKBGNFCAA|2076-12-04|2123|9232|709|2076|5|12|4|4|2076|709|9232|Friday|2076Q4|N|Y|N|2479639|2479973|2479276|2479550|N|N|N|N|N| +2479643|AAAAAAAALBGNFCAA|2076-12-05|2123|9232|709|2076|6|12|5|4|2076|709|9232|Saturday|2076Q4|N|Y|N|2479639|2479973|2479277|2479551|N|N|N|N|N| +2479644|AAAAAAAAMBGNFCAA|2076-12-06|2123|9232|709|2076|0|12|6|4|2076|709|9232|Sunday|2076Q4|N|N|N|2479639|2479973|2479278|2479552|N|N|N|N|N| +2479645|AAAAAAAANBGNFCAA|2076-12-07|2123|9232|709|2076|1|12|7|4|2076|709|9232|Monday|2076Q4|N|N|N|2479639|2479973|2479279|2479553|N|N|N|N|N| +2479646|AAAAAAAAOBGNFCAA|2076-12-08|2123|9233|709|2076|2|12|8|4|2076|709|9233|Tuesday|2076Q4|N|N|N|2479639|2479973|2479280|2479554|N|N|N|N|N| +2479647|AAAAAAAAPBGNFCAA|2076-12-09|2123|9233|709|2076|3|12|9|4|2076|709|9233|Wednesday|2076Q4|N|N|N|2479639|2479973|2479281|2479555|N|N|N|N|N| +2479648|AAAAAAAAACGNFCAA|2076-12-10|2123|9233|709|2076|4|12|10|4|2076|709|9233|Thursday|2076Q4|N|N|N|2479639|2479973|2479282|2479556|N|N|N|N|N| +2479649|AAAAAAAABCGNFCAA|2076-12-11|2123|9233|709|2076|5|12|11|4|2076|709|9233|Friday|2076Q4|N|Y|N|2479639|2479973|2479283|2479557|N|N|N|N|N| +2479650|AAAAAAAACCGNFCAA|2076-12-12|2123|9233|709|2076|6|12|12|4|2076|709|9233|Saturday|2076Q4|N|Y|N|2479639|2479973|2479284|2479558|N|N|N|N|N| +2479651|AAAAAAAADCGNFCAA|2076-12-13|2123|9233|709|2076|0|12|13|4|2076|709|9233|Sunday|2076Q4|N|N|N|2479639|2479973|2479285|2479559|N|N|N|N|N| +2479652|AAAAAAAAECGNFCAA|2076-12-14|2123|9233|709|2076|1|12|14|4|2076|709|9233|Monday|2076Q4|N|N|N|2479639|2479973|2479286|2479560|N|N|N|N|N| +2479653|AAAAAAAAFCGNFCAA|2076-12-15|2123|9234|709|2076|2|12|15|4|2076|709|9234|Tuesday|2076Q4|N|N|N|2479639|2479973|2479287|2479561|N|N|N|N|N| +2479654|AAAAAAAAGCGNFCAA|2076-12-16|2123|9234|709|2076|3|12|16|4|2076|709|9234|Wednesday|2076Q4|N|N|N|2479639|2479973|2479288|2479562|N|N|N|N|N| +2479655|AAAAAAAAHCGNFCAA|2076-12-17|2123|9234|709|2076|4|12|17|4|2076|709|9234|Thursday|2076Q4|N|N|N|2479639|2479973|2479289|2479563|N|N|N|N|N| +2479656|AAAAAAAAICGNFCAA|2076-12-18|2123|9234|709|2076|5|12|18|4|2076|709|9234|Friday|2076Q4|N|Y|N|2479639|2479973|2479290|2479564|N|N|N|N|N| +2479657|AAAAAAAAJCGNFCAA|2076-12-19|2123|9234|709|2076|6|12|19|4|2076|709|9234|Saturday|2076Q4|N|Y|N|2479639|2479973|2479291|2479565|N|N|N|N|N| +2479658|AAAAAAAAKCGNFCAA|2076-12-20|2123|9234|709|2076|0|12|20|4|2076|709|9234|Sunday|2076Q4|N|N|N|2479639|2479973|2479292|2479566|N|N|N|N|N| +2479659|AAAAAAAALCGNFCAA|2076-12-21|2123|9234|709|2076|1|12|21|4|2076|709|9234|Monday|2076Q4|N|N|N|2479639|2479973|2479293|2479567|N|N|N|N|N| +2479660|AAAAAAAAMCGNFCAA|2076-12-22|2123|9235|709|2076|2|12|22|4|2076|709|9235|Tuesday|2076Q4|N|N|N|2479639|2479973|2479294|2479568|N|N|N|N|N| +2479661|AAAAAAAANCGNFCAA|2076-12-23|2123|9235|709|2076|3|12|23|4|2076|709|9235|Wednesday|2076Q4|N|N|N|2479639|2479973|2479295|2479569|N|N|N|N|N| +2479662|AAAAAAAAOCGNFCAA|2076-12-24|2123|9235|709|2076|4|12|24|4|2076|709|9235|Thursday|2076Q4|N|N|N|2479639|2479973|2479296|2479570|N|N|N|N|N| +2479663|AAAAAAAAPCGNFCAA|2076-12-25|2123|9235|709|2076|5|12|25|4|2076|709|9235|Friday|2076Q4|Y|Y|N|2479639|2479973|2479297|2479571|N|N|N|N|N| +2479664|AAAAAAAAADGNFCAA|2076-12-26|2123|9235|709|2076|6|12|26|4|2076|709|9235|Saturday|2076Q4|N|Y|Y|2479639|2479973|2479298|2479572|N|N|N|N|N| +2479665|AAAAAAAABDGNFCAA|2076-12-27|2123|9235|709|2076|0|12|27|4|2076|709|9235|Sunday|2076Q4|N|N|N|2479639|2479973|2479299|2479573|N|N|N|N|N| +2479666|AAAAAAAACDGNFCAA|2076-12-28|2123|9235|709|2076|1|12|28|4|2076|709|9235|Monday|2076Q4|N|N|N|2479639|2479973|2479300|2479574|N|N|N|N|N| +2479667|AAAAAAAADDGNFCAA|2076-12-29|2123|9236|709|2076|2|12|29|4|2076|709|9236|Tuesday|2076Q4|N|N|N|2479639|2479973|2479301|2479575|N|N|N|N|N| +2479668|AAAAAAAAEDGNFCAA|2076-12-30|2123|9236|709|2076|3|12|30|4|2076|709|9236|Wednesday|2076Q4|N|N|N|2479639|2479973|2479302|2479576|N|N|N|N|N| +2479669|AAAAAAAAFDGNFCAA|2076-12-31|2123|9236|709|2076|4|12|31|4|2076|709|9236|Thursday|2076Q4|Y|N|N|2479639|2479973|2479303|2479577|N|N|N|N|N| +2479670|AAAAAAAAGDGNFCAA|2077-01-01|2124|9236|709|2077|5|1|1|1|2077|709|9236|Friday|2077Q1|Y|Y|Y|2479670|2479669|2479304|2479578|N|N|N|N|N| +2479671|AAAAAAAAHDGNFCAA|2077-01-02|2124|9236|709|2077|6|1|2|1|2077|709|9236|Saturday|2077Q1|N|Y|Y|2479670|2479669|2479305|2479579|N|N|N|N|N| +2479672|AAAAAAAAIDGNFCAA|2077-01-03|2124|9236|709|2077|0|1|3|1|2077|709|9236|Sunday|2077Q1|N|N|N|2479670|2479669|2479306|2479580|N|N|N|N|N| +2479673|AAAAAAAAJDGNFCAA|2077-01-04|2124|9236|709|2077|1|1|4|1|2077|709|9236|Monday|2077Q1|N|N|N|2479670|2479669|2479307|2479581|N|N|N|N|N| +2479674|AAAAAAAAKDGNFCAA|2077-01-05|2124|9237|709|2077|2|1|5|1|2077|709|9237|Tuesday|2077Q1|N|N|N|2479670|2479669|2479308|2479582|N|N|N|N|N| +2479675|AAAAAAAALDGNFCAA|2077-01-06|2124|9237|709|2077|3|1|6|1|2077|709|9237|Wednesday|2077Q1|N|N|N|2479670|2479669|2479309|2479583|N|N|N|N|N| +2479676|AAAAAAAAMDGNFCAA|2077-01-07|2124|9237|709|2077|4|1|7|1|2077|709|9237|Thursday|2077Q1|N|N|N|2479670|2479669|2479310|2479584|N|N|N|N|N| +2479677|AAAAAAAANDGNFCAA|2077-01-08|2124|9237|709|2077|5|1|8|1|2077|709|9237|Friday|2077Q1|N|Y|N|2479670|2479669|2479311|2479585|N|N|N|N|N| +2479678|AAAAAAAAODGNFCAA|2077-01-09|2124|9237|709|2077|6|1|9|1|2077|709|9237|Saturday|2077Q1|N|Y|N|2479670|2479669|2479312|2479586|N|N|N|N|N| +2479679|AAAAAAAAPDGNFCAA|2077-01-10|2124|9237|709|2077|0|1|10|1|2077|709|9237|Sunday|2077Q1|N|N|N|2479670|2479669|2479313|2479587|N|N|N|N|N| +2479680|AAAAAAAAAEGNFCAA|2077-01-11|2124|9237|709|2077|1|1|11|1|2077|709|9237|Monday|2077Q1|N|N|N|2479670|2479669|2479314|2479588|N|N|N|N|N| +2479681|AAAAAAAABEGNFCAA|2077-01-12|2124|9238|709|2077|2|1|12|1|2077|709|9238|Tuesday|2077Q1|N|N|N|2479670|2479669|2479315|2479589|N|N|N|N|N| +2479682|AAAAAAAACEGNFCAA|2077-01-13|2124|9238|709|2077|3|1|13|1|2077|709|9238|Wednesday|2077Q1|N|N|N|2479670|2479669|2479316|2479590|N|N|N|N|N| +2479683|AAAAAAAADEGNFCAA|2077-01-14|2124|9238|709|2077|4|1|14|1|2077|709|9238|Thursday|2077Q1|N|N|N|2479670|2479669|2479317|2479591|N|N|N|N|N| +2479684|AAAAAAAAEEGNFCAA|2077-01-15|2124|9238|709|2077|5|1|15|1|2077|709|9238|Friday|2077Q1|N|Y|N|2479670|2479669|2479318|2479592|N|N|N|N|N| +2479685|AAAAAAAAFEGNFCAA|2077-01-16|2124|9238|709|2077|6|1|16|1|2077|709|9238|Saturday|2077Q1|N|Y|N|2479670|2479669|2479319|2479593|N|N|N|N|N| +2479686|AAAAAAAAGEGNFCAA|2077-01-17|2124|9238|709|2077|0|1|17|1|2077|709|9238|Sunday|2077Q1|N|N|N|2479670|2479669|2479320|2479594|N|N|N|N|N| +2479687|AAAAAAAAHEGNFCAA|2077-01-18|2124|9238|709|2077|1|1|18|1|2077|709|9238|Monday|2077Q1|N|N|N|2479670|2479669|2479321|2479595|N|N|N|N|N| +2479688|AAAAAAAAIEGNFCAA|2077-01-19|2124|9239|709|2077|2|1|19|1|2077|709|9239|Tuesday|2077Q1|N|N|N|2479670|2479669|2479322|2479596|N|N|N|N|N| +2479689|AAAAAAAAJEGNFCAA|2077-01-20|2124|9239|709|2077|3|1|20|1|2077|709|9239|Wednesday|2077Q1|N|N|N|2479670|2479669|2479323|2479597|N|N|N|N|N| +2479690|AAAAAAAAKEGNFCAA|2077-01-21|2124|9239|709|2077|4|1|21|1|2077|709|9239|Thursday|2077Q1|N|N|N|2479670|2479669|2479324|2479598|N|N|N|N|N| +2479691|AAAAAAAALEGNFCAA|2077-01-22|2124|9239|709|2077|5|1|22|1|2077|709|9239|Friday|2077Q1|N|Y|N|2479670|2479669|2479325|2479599|N|N|N|N|N| +2479692|AAAAAAAAMEGNFCAA|2077-01-23|2124|9239|709|2077|6|1|23|1|2077|709|9239|Saturday|2077Q1|N|Y|N|2479670|2479669|2479326|2479600|N|N|N|N|N| +2479693|AAAAAAAANEGNFCAA|2077-01-24|2124|9239|709|2077|0|1|24|1|2077|709|9239|Sunday|2077Q1|N|N|N|2479670|2479669|2479327|2479601|N|N|N|N|N| +2479694|AAAAAAAAOEGNFCAA|2077-01-25|2124|9239|709|2077|1|1|25|1|2077|709|9239|Monday|2077Q1|N|N|N|2479670|2479669|2479328|2479602|N|N|N|N|N| +2479695|AAAAAAAAPEGNFCAA|2077-01-26|2124|9240|709|2077|2|1|26|1|2077|709|9240|Tuesday|2077Q1|N|N|N|2479670|2479669|2479329|2479603|N|N|N|N|N| +2479696|AAAAAAAAAFGNFCAA|2077-01-27|2124|9240|709|2077|3|1|27|1|2077|709|9240|Wednesday|2077Q1|N|N|N|2479670|2479669|2479330|2479604|N|N|N|N|N| +2479697|AAAAAAAABFGNFCAA|2077-01-28|2124|9240|709|2077|4|1|28|1|2077|709|9240|Thursday|2077Q1|N|N|N|2479670|2479669|2479331|2479605|N|N|N|N|N| +2479698|AAAAAAAACFGNFCAA|2077-01-29|2124|9240|709|2077|5|1|29|1|2077|709|9240|Friday|2077Q1|N|Y|N|2479670|2479669|2479332|2479606|N|N|N|N|N| +2479699|AAAAAAAADFGNFCAA|2077-01-30|2124|9240|709|2077|6|1|30|1|2077|709|9240|Saturday|2077Q1|N|Y|N|2479670|2479669|2479333|2479607|N|N|N|N|N| +2479700|AAAAAAAAEFGNFCAA|2077-01-31|2124|9240|709|2077|0|1|31|1|2077|709|9240|Sunday|2077Q1|N|N|N|2479670|2479669|2479334|2479608|N|N|N|N|N| +2479701|AAAAAAAAFFGNFCAA|2077-02-01|2125|9240|709|2077|1|2|1|1|2077|709|9240|Monday|2077Q1|N|N|N|2479701|2479731|2479335|2479609|N|N|N|N|N| +2479702|AAAAAAAAGFGNFCAA|2077-02-02|2125|9241|709|2077|2|2|2|1|2077|709|9241|Tuesday|2077Q1|N|N|N|2479701|2479731|2479336|2479610|N|N|N|N|N| +2479703|AAAAAAAAHFGNFCAA|2077-02-03|2125|9241|709|2077|3|2|3|1|2077|709|9241|Wednesday|2077Q1|N|N|N|2479701|2479731|2479337|2479611|N|N|N|N|N| +2479704|AAAAAAAAIFGNFCAA|2077-02-04|2125|9241|709|2077|4|2|4|1|2077|709|9241|Thursday|2077Q1|N|N|N|2479701|2479731|2479338|2479612|N|N|N|N|N| +2479705|AAAAAAAAJFGNFCAA|2077-02-05|2125|9241|709|2077|5|2|5|1|2077|709|9241|Friday|2077Q1|N|Y|N|2479701|2479731|2479339|2479613|N|N|N|N|N| +2479706|AAAAAAAAKFGNFCAA|2077-02-06|2125|9241|709|2077|6|2|6|1|2077|709|9241|Saturday|2077Q1|N|Y|N|2479701|2479731|2479340|2479614|N|N|N|N|N| +2479707|AAAAAAAALFGNFCAA|2077-02-07|2125|9241|709|2077|0|2|7|1|2077|709|9241|Sunday|2077Q1|N|N|N|2479701|2479731|2479341|2479615|N|N|N|N|N| +2479708|AAAAAAAAMFGNFCAA|2077-02-08|2125|9241|709|2077|1|2|8|1|2077|709|9241|Monday|2077Q1|N|N|N|2479701|2479731|2479342|2479616|N|N|N|N|N| +2479709|AAAAAAAANFGNFCAA|2077-02-09|2125|9242|709|2077|2|2|9|1|2077|709|9242|Tuesday|2077Q1|N|N|N|2479701|2479731|2479343|2479617|N|N|N|N|N| +2479710|AAAAAAAAOFGNFCAA|2077-02-10|2125|9242|709|2077|3|2|10|1|2077|709|9242|Wednesday|2077Q1|N|N|N|2479701|2479731|2479344|2479618|N|N|N|N|N| +2479711|AAAAAAAAPFGNFCAA|2077-02-11|2125|9242|709|2077|4|2|11|1|2077|709|9242|Thursday|2077Q1|N|N|N|2479701|2479731|2479345|2479619|N|N|N|N|N| +2479712|AAAAAAAAAGGNFCAA|2077-02-12|2125|9242|709|2077|5|2|12|1|2077|709|9242|Friday|2077Q1|N|Y|N|2479701|2479731|2479346|2479620|N|N|N|N|N| +2479713|AAAAAAAABGGNFCAA|2077-02-13|2125|9242|709|2077|6|2|13|1|2077|709|9242|Saturday|2077Q1|N|Y|N|2479701|2479731|2479347|2479621|N|N|N|N|N| +2479714|AAAAAAAACGGNFCAA|2077-02-14|2125|9242|709|2077|0|2|14|1|2077|709|9242|Sunday|2077Q1|N|N|N|2479701|2479731|2479348|2479622|N|N|N|N|N| +2479715|AAAAAAAADGGNFCAA|2077-02-15|2125|9242|709|2077|1|2|15|1|2077|709|9242|Monday|2077Q1|N|N|N|2479701|2479731|2479349|2479623|N|N|N|N|N| +2479716|AAAAAAAAEGGNFCAA|2077-02-16|2125|9243|709|2077|2|2|16|1|2077|709|9243|Tuesday|2077Q1|N|N|N|2479701|2479731|2479350|2479624|N|N|N|N|N| +2479717|AAAAAAAAFGGNFCAA|2077-02-17|2125|9243|709|2077|3|2|17|1|2077|709|9243|Wednesday|2077Q1|N|N|N|2479701|2479731|2479351|2479625|N|N|N|N|N| +2479718|AAAAAAAAGGGNFCAA|2077-02-18|2125|9243|709|2077|4|2|18|1|2077|709|9243|Thursday|2077Q1|N|N|N|2479701|2479731|2479352|2479626|N|N|N|N|N| +2479719|AAAAAAAAHGGNFCAA|2077-02-19|2125|9243|709|2077|5|2|19|1|2077|709|9243|Friday|2077Q1|N|Y|N|2479701|2479731|2479353|2479627|N|N|N|N|N| +2479720|AAAAAAAAIGGNFCAA|2077-02-20|2125|9243|709|2077|6|2|20|1|2077|709|9243|Saturday|2077Q1|N|Y|N|2479701|2479731|2479354|2479628|N|N|N|N|N| +2479721|AAAAAAAAJGGNFCAA|2077-02-21|2125|9243|709|2077|0|2|21|1|2077|709|9243|Sunday|2077Q1|N|N|N|2479701|2479731|2479355|2479629|N|N|N|N|N| +2479722|AAAAAAAAKGGNFCAA|2077-02-22|2125|9243|709|2077|1|2|22|1|2077|709|9243|Monday|2077Q1|N|N|N|2479701|2479731|2479356|2479630|N|N|N|N|N| +2479723|AAAAAAAALGGNFCAA|2077-02-23|2125|9244|709|2077|2|2|23|1|2077|709|9244|Tuesday|2077Q1|N|N|N|2479701|2479731|2479357|2479631|N|N|N|N|N| +2479724|AAAAAAAAMGGNFCAA|2077-02-24|2125|9244|709|2077|3|2|24|1|2077|709|9244|Wednesday|2077Q1|N|N|N|2479701|2479731|2479358|2479632|N|N|N|N|N| +2479725|AAAAAAAANGGNFCAA|2077-02-25|2125|9244|709|2077|4|2|25|1|2077|709|9244|Thursday|2077Q1|N|N|N|2479701|2479731|2479359|2479633|N|N|N|N|N| +2479726|AAAAAAAAOGGNFCAA|2077-02-26|2125|9244|709|2077|5|2|26|1|2077|709|9244|Friday|2077Q1|N|Y|N|2479701|2479731|2479360|2479634|N|N|N|N|N| +2479727|AAAAAAAAPGGNFCAA|2077-02-27|2125|9244|709|2077|6|2|27|1|2077|709|9244|Saturday|2077Q1|N|Y|N|2479701|2479731|2479361|2479635|N|N|N|N|N| +2479728|AAAAAAAAAHGNFCAA|2077-02-28|2125|9244|709|2077|0|2|28|1|2077|709|9244|Sunday|2077Q1|N|N|N|2479701|2479731|2479362|2479636|N|N|N|N|N| +2479729|AAAAAAAABHGNFCAA|2077-03-01|2126|9244|710|2077|1|3|1|1|2077|710|9244|Monday|2077Q1|N|N|N|2479729|2479787|2479364|2479637|N|N|N|N|N| +2479730|AAAAAAAACHGNFCAA|2077-03-02|2126|9245|710|2077|2|3|2|1|2077|710|9245|Tuesday|2077Q1|N|N|N|2479729|2479787|2479365|2479638|N|N|N|N|N| +2479731|AAAAAAAADHGNFCAA|2077-03-03|2126|9245|710|2077|3|3|3|1|2077|710|9245|Wednesday|2077Q1|N|N|N|2479729|2479787|2479366|2479639|N|N|N|N|N| +2479732|AAAAAAAAEHGNFCAA|2077-03-04|2126|9245|710|2077|4|3|4|1|2077|710|9245|Thursday|2077Q1|N|N|N|2479729|2479787|2479367|2479640|N|N|N|N|N| +2479733|AAAAAAAAFHGNFCAA|2077-03-05|2126|9245|710|2077|5|3|5|1|2077|710|9245|Friday|2077Q1|N|Y|N|2479729|2479787|2479368|2479641|N|N|N|N|N| +2479734|AAAAAAAAGHGNFCAA|2077-03-06|2126|9245|710|2077|6|3|6|1|2077|710|9245|Saturday|2077Q1|N|Y|N|2479729|2479787|2479369|2479642|N|N|N|N|N| +2479735|AAAAAAAAHHGNFCAA|2077-03-07|2126|9245|710|2077|0|3|7|1|2077|710|9245|Sunday|2077Q1|N|N|N|2479729|2479787|2479370|2479643|N|N|N|N|N| +2479736|AAAAAAAAIHGNFCAA|2077-03-08|2126|9245|710|2077|1|3|8|1|2077|710|9245|Monday|2077Q1|N|N|N|2479729|2479787|2479371|2479644|N|N|N|N|N| +2479737|AAAAAAAAJHGNFCAA|2077-03-09|2126|9246|710|2077|2|3|9|1|2077|710|9246|Tuesday|2077Q1|N|N|N|2479729|2479787|2479372|2479645|N|N|N|N|N| +2479738|AAAAAAAAKHGNFCAA|2077-03-10|2126|9246|710|2077|3|3|10|1|2077|710|9246|Wednesday|2077Q1|N|N|N|2479729|2479787|2479373|2479646|N|N|N|N|N| +2479739|AAAAAAAALHGNFCAA|2077-03-11|2126|9246|710|2077|4|3|11|1|2077|710|9246|Thursday|2077Q1|N|N|N|2479729|2479787|2479374|2479647|N|N|N|N|N| +2479740|AAAAAAAAMHGNFCAA|2077-03-12|2126|9246|710|2077|5|3|12|1|2077|710|9246|Friday|2077Q1|N|Y|N|2479729|2479787|2479375|2479648|N|N|N|N|N| +2479741|AAAAAAAANHGNFCAA|2077-03-13|2126|9246|710|2077|6|3|13|1|2077|710|9246|Saturday|2077Q1|N|Y|N|2479729|2479787|2479376|2479649|N|N|N|N|N| +2479742|AAAAAAAAOHGNFCAA|2077-03-14|2126|9246|710|2077|0|3|14|1|2077|710|9246|Sunday|2077Q1|N|N|N|2479729|2479787|2479377|2479650|N|N|N|N|N| +2479743|AAAAAAAAPHGNFCAA|2077-03-15|2126|9246|710|2077|1|3|15|1|2077|710|9246|Monday|2077Q1|N|N|N|2479729|2479787|2479378|2479651|N|N|N|N|N| +2479744|AAAAAAAAAIGNFCAA|2077-03-16|2126|9247|710|2077|2|3|16|1|2077|710|9247|Tuesday|2077Q1|N|N|N|2479729|2479787|2479379|2479652|N|N|N|N|N| +2479745|AAAAAAAABIGNFCAA|2077-03-17|2126|9247|710|2077|3|3|17|1|2077|710|9247|Wednesday|2077Q1|N|N|N|2479729|2479787|2479380|2479653|N|N|N|N|N| +2479746|AAAAAAAACIGNFCAA|2077-03-18|2126|9247|710|2077|4|3|18|1|2077|710|9247|Thursday|2077Q1|N|N|N|2479729|2479787|2479381|2479654|N|N|N|N|N| +2479747|AAAAAAAADIGNFCAA|2077-03-19|2126|9247|710|2077|5|3|19|1|2077|710|9247|Friday|2077Q1|N|Y|N|2479729|2479787|2479382|2479655|N|N|N|N|N| +2479748|AAAAAAAAEIGNFCAA|2077-03-20|2126|9247|710|2077|6|3|20|1|2077|710|9247|Saturday|2077Q1|N|Y|N|2479729|2479787|2479383|2479656|N|N|N|N|N| +2479749|AAAAAAAAFIGNFCAA|2077-03-21|2126|9247|710|2077|0|3|21|1|2077|710|9247|Sunday|2077Q1|N|N|N|2479729|2479787|2479384|2479657|N|N|N|N|N| +2479750|AAAAAAAAGIGNFCAA|2077-03-22|2126|9247|710|2077|1|3|22|1|2077|710|9247|Monday|2077Q1|N|N|N|2479729|2479787|2479385|2479658|N|N|N|N|N| +2479751|AAAAAAAAHIGNFCAA|2077-03-23|2126|9248|710|2077|2|3|23|1|2077|710|9248|Tuesday|2077Q1|N|N|N|2479729|2479787|2479386|2479659|N|N|N|N|N| +2479752|AAAAAAAAIIGNFCAA|2077-03-24|2126|9248|710|2077|3|3|24|1|2077|710|9248|Wednesday|2077Q1|N|N|N|2479729|2479787|2479387|2479660|N|N|N|N|N| +2479753|AAAAAAAAJIGNFCAA|2077-03-25|2126|9248|710|2077|4|3|25|1|2077|710|9248|Thursday|2077Q1|N|N|N|2479729|2479787|2479388|2479661|N|N|N|N|N| +2479754|AAAAAAAAKIGNFCAA|2077-03-26|2126|9248|710|2077|5|3|26|1|2077|710|9248|Friday|2077Q1|N|Y|N|2479729|2479787|2479389|2479662|N|N|N|N|N| +2479755|AAAAAAAALIGNFCAA|2077-03-27|2126|9248|710|2077|6|3|27|1|2077|710|9248|Saturday|2077Q1|N|Y|N|2479729|2479787|2479390|2479663|N|N|N|N|N| +2479756|AAAAAAAAMIGNFCAA|2077-03-28|2126|9248|710|2077|0|3|28|1|2077|710|9248|Sunday|2077Q1|N|N|N|2479729|2479787|2479391|2479664|N|N|N|N|N| +2479757|AAAAAAAANIGNFCAA|2077-03-29|2126|9248|710|2077|1|3|29|1|2077|710|9248|Monday|2077Q1|N|N|N|2479729|2479787|2479392|2479665|N|N|N|N|N| +2479758|AAAAAAAAOIGNFCAA|2077-03-30|2126|9249|710|2077|2|3|30|1|2077|710|9249|Tuesday|2077Q1|N|N|N|2479729|2479787|2479393|2479666|N|N|N|N|N| +2479759|AAAAAAAAPIGNFCAA|2077-03-31|2126|9249|710|2077|3|3|31|1|2077|710|9249|Wednesday|2077Q1|N|N|N|2479729|2479787|2479394|2479667|N|N|N|N|N| +2479760|AAAAAAAAAJGNFCAA|2077-04-01|2127|9249|710|2077|4|4|1|1|2077|710|9249|Thursday|2077Q1|N|N|N|2479760|2479849|2479395|2479670|N|N|N|N|N| +2479761|AAAAAAAABJGNFCAA|2077-04-02|2127|9249|710|2077|5|4|2|2|2077|710|9249|Friday|2077Q2|N|Y|N|2479760|2479849|2479396|2479671|N|N|N|N|N| +2479762|AAAAAAAACJGNFCAA|2077-04-03|2127|9249|710|2077|6|4|3|2|2077|710|9249|Saturday|2077Q2|N|Y|N|2479760|2479849|2479397|2479672|N|N|N|N|N| +2479763|AAAAAAAADJGNFCAA|2077-04-04|2127|9249|710|2077|0|4|4|2|2077|710|9249|Sunday|2077Q2|N|N|N|2479760|2479849|2479398|2479673|N|N|N|N|N| +2479764|AAAAAAAAEJGNFCAA|2077-04-05|2127|9249|710|2077|1|4|5|2|2077|710|9249|Monday|2077Q2|N|N|N|2479760|2479849|2479399|2479674|N|N|N|N|N| +2479765|AAAAAAAAFJGNFCAA|2077-04-06|2127|9250|710|2077|2|4|6|2|2077|710|9250|Tuesday|2077Q2|N|N|N|2479760|2479849|2479400|2479675|N|N|N|N|N| +2479766|AAAAAAAAGJGNFCAA|2077-04-07|2127|9250|710|2077|3|4|7|2|2077|710|9250|Wednesday|2077Q2|N|N|N|2479760|2479849|2479401|2479676|N|N|N|N|N| +2479767|AAAAAAAAHJGNFCAA|2077-04-08|2127|9250|710|2077|4|4|8|2|2077|710|9250|Thursday|2077Q2|N|N|N|2479760|2479849|2479402|2479677|N|N|N|N|N| +2479768|AAAAAAAAIJGNFCAA|2077-04-09|2127|9250|710|2077|5|4|9|2|2077|710|9250|Friday|2077Q2|N|Y|N|2479760|2479849|2479403|2479678|N|N|N|N|N| +2479769|AAAAAAAAJJGNFCAA|2077-04-10|2127|9250|710|2077|6|4|10|2|2077|710|9250|Saturday|2077Q2|N|Y|N|2479760|2479849|2479404|2479679|N|N|N|N|N| +2479770|AAAAAAAAKJGNFCAA|2077-04-11|2127|9250|710|2077|0|4|11|2|2077|710|9250|Sunday|2077Q2|N|N|N|2479760|2479849|2479405|2479680|N|N|N|N|N| +2479771|AAAAAAAALJGNFCAA|2077-04-12|2127|9250|710|2077|1|4|12|2|2077|710|9250|Monday|2077Q2|N|N|N|2479760|2479849|2479406|2479681|N|N|N|N|N| +2479772|AAAAAAAAMJGNFCAA|2077-04-13|2127|9251|710|2077|2|4|13|2|2077|710|9251|Tuesday|2077Q2|N|N|N|2479760|2479849|2479407|2479682|N|N|N|N|N| +2479773|AAAAAAAANJGNFCAA|2077-04-14|2127|9251|710|2077|3|4|14|2|2077|710|9251|Wednesday|2077Q2|N|N|N|2479760|2479849|2479408|2479683|N|N|N|N|N| +2479774|AAAAAAAAOJGNFCAA|2077-04-15|2127|9251|710|2077|4|4|15|2|2077|710|9251|Thursday|2077Q2|N|N|N|2479760|2479849|2479409|2479684|N|N|N|N|N| +2479775|AAAAAAAAPJGNFCAA|2077-04-16|2127|9251|710|2077|5|4|16|2|2077|710|9251|Friday|2077Q2|N|Y|N|2479760|2479849|2479410|2479685|N|N|N|N|N| +2479776|AAAAAAAAAKGNFCAA|2077-04-17|2127|9251|710|2077|6|4|17|2|2077|710|9251|Saturday|2077Q2|N|Y|N|2479760|2479849|2479411|2479686|N|N|N|N|N| +2479777|AAAAAAAABKGNFCAA|2077-04-18|2127|9251|710|2077|0|4|18|2|2077|710|9251|Sunday|2077Q2|N|N|N|2479760|2479849|2479412|2479687|N|N|N|N|N| +2479778|AAAAAAAACKGNFCAA|2077-04-19|2127|9251|710|2077|1|4|19|2|2077|710|9251|Monday|2077Q2|N|N|N|2479760|2479849|2479413|2479688|N|N|N|N|N| +2479779|AAAAAAAADKGNFCAA|2077-04-20|2127|9252|710|2077|2|4|20|2|2077|710|9252|Tuesday|2077Q2|N|N|N|2479760|2479849|2479414|2479689|N|N|N|N|N| +2479780|AAAAAAAAEKGNFCAA|2077-04-21|2127|9252|710|2077|3|4|21|2|2077|710|9252|Wednesday|2077Q2|N|N|N|2479760|2479849|2479415|2479690|N|N|N|N|N| +2479781|AAAAAAAAFKGNFCAA|2077-04-22|2127|9252|710|2077|4|4|22|2|2077|710|9252|Thursday|2077Q2|N|N|N|2479760|2479849|2479416|2479691|N|N|N|N|N| +2479782|AAAAAAAAGKGNFCAA|2077-04-23|2127|9252|710|2077|5|4|23|2|2077|710|9252|Friday|2077Q2|N|Y|N|2479760|2479849|2479417|2479692|N|N|N|N|N| +2479783|AAAAAAAAHKGNFCAA|2077-04-24|2127|9252|710|2077|6|4|24|2|2077|710|9252|Saturday|2077Q2|N|Y|N|2479760|2479849|2479418|2479693|N|N|N|N|N| +2479784|AAAAAAAAIKGNFCAA|2077-04-25|2127|9252|710|2077|0|4|25|2|2077|710|9252|Sunday|2077Q2|N|N|N|2479760|2479849|2479419|2479694|N|N|N|N|N| +2479785|AAAAAAAAJKGNFCAA|2077-04-26|2127|9252|710|2077|1|4|26|2|2077|710|9252|Monday|2077Q2|N|N|N|2479760|2479849|2479420|2479695|N|N|N|N|N| +2479786|AAAAAAAAKKGNFCAA|2077-04-27|2127|9253|710|2077|2|4|27|2|2077|710|9253|Tuesday|2077Q2|N|N|N|2479760|2479849|2479421|2479696|N|N|N|N|N| +2479787|AAAAAAAALKGNFCAA|2077-04-28|2127|9253|710|2077|3|4|28|2|2077|710|9253|Wednesday|2077Q2|N|N|N|2479760|2479849|2479422|2479697|N|N|N|N|N| +2479788|AAAAAAAAMKGNFCAA|2077-04-29|2127|9253|710|2077|4|4|29|2|2077|710|9253|Thursday|2077Q2|N|N|N|2479760|2479849|2479423|2479698|N|N|N|N|N| +2479789|AAAAAAAANKGNFCAA|2077-04-30|2127|9253|710|2077|5|4|30|2|2077|710|9253|Friday|2077Q2|N|Y|N|2479760|2479849|2479424|2479699|N|N|N|N|N| +2479790|AAAAAAAAOKGNFCAA|2077-05-01|2128|9253|710|2077|6|5|1|2|2077|710|9253|Saturday|2077Q2|N|Y|N|2479790|2479909|2479425|2479700|N|N|N|N|N| +2479791|AAAAAAAAPKGNFCAA|2077-05-02|2128|9253|710|2077|0|5|2|2|2077|710|9253|Sunday|2077Q2|N|N|N|2479790|2479909|2479426|2479701|N|N|N|N|N| +2479792|AAAAAAAAALGNFCAA|2077-05-03|2128|9253|710|2077|1|5|3|2|2077|710|9253|Monday|2077Q2|N|N|N|2479790|2479909|2479427|2479702|N|N|N|N|N| +2479793|AAAAAAAABLGNFCAA|2077-05-04|2128|9254|710|2077|2|5|4|2|2077|710|9254|Tuesday|2077Q2|N|N|N|2479790|2479909|2479428|2479703|N|N|N|N|N| +2479794|AAAAAAAACLGNFCAA|2077-05-05|2128|9254|710|2077|3|5|5|2|2077|710|9254|Wednesday|2077Q2|N|N|N|2479790|2479909|2479429|2479704|N|N|N|N|N| +2479795|AAAAAAAADLGNFCAA|2077-05-06|2128|9254|710|2077|4|5|6|2|2077|710|9254|Thursday|2077Q2|N|N|N|2479790|2479909|2479430|2479705|N|N|N|N|N| +2479796|AAAAAAAAELGNFCAA|2077-05-07|2128|9254|710|2077|5|5|7|2|2077|710|9254|Friday|2077Q2|N|Y|N|2479790|2479909|2479431|2479706|N|N|N|N|N| +2479797|AAAAAAAAFLGNFCAA|2077-05-08|2128|9254|710|2077|6|5|8|2|2077|710|9254|Saturday|2077Q2|N|Y|N|2479790|2479909|2479432|2479707|N|N|N|N|N| +2479798|AAAAAAAAGLGNFCAA|2077-05-09|2128|9254|710|2077|0|5|9|2|2077|710|9254|Sunday|2077Q2|N|N|N|2479790|2479909|2479433|2479708|N|N|N|N|N| +2479799|AAAAAAAAHLGNFCAA|2077-05-10|2128|9254|710|2077|1|5|10|2|2077|710|9254|Monday|2077Q2|N|N|N|2479790|2479909|2479434|2479709|N|N|N|N|N| +2479800|AAAAAAAAILGNFCAA|2077-05-11|2128|9255|710|2077|2|5|11|2|2077|710|9255|Tuesday|2077Q2|N|N|N|2479790|2479909|2479435|2479710|N|N|N|N|N| +2479801|AAAAAAAAJLGNFCAA|2077-05-12|2128|9255|710|2077|3|5|12|2|2077|710|9255|Wednesday|2077Q2|N|N|N|2479790|2479909|2479436|2479711|N|N|N|N|N| +2479802|AAAAAAAAKLGNFCAA|2077-05-13|2128|9255|710|2077|4|5|13|2|2077|710|9255|Thursday|2077Q2|N|N|N|2479790|2479909|2479437|2479712|N|N|N|N|N| +2479803|AAAAAAAALLGNFCAA|2077-05-14|2128|9255|710|2077|5|5|14|2|2077|710|9255|Friday|2077Q2|N|Y|N|2479790|2479909|2479438|2479713|N|N|N|N|N| +2479804|AAAAAAAAMLGNFCAA|2077-05-15|2128|9255|710|2077|6|5|15|2|2077|710|9255|Saturday|2077Q2|N|Y|N|2479790|2479909|2479439|2479714|N|N|N|N|N| +2479805|AAAAAAAANLGNFCAA|2077-05-16|2128|9255|710|2077|0|5|16|2|2077|710|9255|Sunday|2077Q2|N|N|N|2479790|2479909|2479440|2479715|N|N|N|N|N| +2479806|AAAAAAAAOLGNFCAA|2077-05-17|2128|9255|710|2077|1|5|17|2|2077|710|9255|Monday|2077Q2|N|N|N|2479790|2479909|2479441|2479716|N|N|N|N|N| +2479807|AAAAAAAAPLGNFCAA|2077-05-18|2128|9256|710|2077|2|5|18|2|2077|710|9256|Tuesday|2077Q2|N|N|N|2479790|2479909|2479442|2479717|N|N|N|N|N| +2479808|AAAAAAAAAMGNFCAA|2077-05-19|2128|9256|710|2077|3|5|19|2|2077|710|9256|Wednesday|2077Q2|N|N|N|2479790|2479909|2479443|2479718|N|N|N|N|N| +2479809|AAAAAAAABMGNFCAA|2077-05-20|2128|9256|710|2077|4|5|20|2|2077|710|9256|Thursday|2077Q2|N|N|N|2479790|2479909|2479444|2479719|N|N|N|N|N| +2479810|AAAAAAAACMGNFCAA|2077-05-21|2128|9256|710|2077|5|5|21|2|2077|710|9256|Friday|2077Q2|N|Y|N|2479790|2479909|2479445|2479720|N|N|N|N|N| +2479811|AAAAAAAADMGNFCAA|2077-05-22|2128|9256|710|2077|6|5|22|2|2077|710|9256|Saturday|2077Q2|N|Y|N|2479790|2479909|2479446|2479721|N|N|N|N|N| +2479812|AAAAAAAAEMGNFCAA|2077-05-23|2128|9256|710|2077|0|5|23|2|2077|710|9256|Sunday|2077Q2|N|N|N|2479790|2479909|2479447|2479722|N|N|N|N|N| +2479813|AAAAAAAAFMGNFCAA|2077-05-24|2128|9256|710|2077|1|5|24|2|2077|710|9256|Monday|2077Q2|N|N|N|2479790|2479909|2479448|2479723|N|N|N|N|N| +2479814|AAAAAAAAGMGNFCAA|2077-05-25|2128|9257|710|2077|2|5|25|2|2077|710|9257|Tuesday|2077Q2|N|N|N|2479790|2479909|2479449|2479724|N|N|N|N|N| +2479815|AAAAAAAAHMGNFCAA|2077-05-26|2128|9257|710|2077|3|5|26|2|2077|710|9257|Wednesday|2077Q2|N|N|N|2479790|2479909|2479450|2479725|N|N|N|N|N| +2479816|AAAAAAAAIMGNFCAA|2077-05-27|2128|9257|710|2077|4|5|27|2|2077|710|9257|Thursday|2077Q2|N|N|N|2479790|2479909|2479451|2479726|N|N|N|N|N| +2479817|AAAAAAAAJMGNFCAA|2077-05-28|2128|9257|710|2077|5|5|28|2|2077|710|9257|Friday|2077Q2|N|Y|N|2479790|2479909|2479452|2479727|N|N|N|N|N| +2479818|AAAAAAAAKMGNFCAA|2077-05-29|2128|9257|710|2077|6|5|29|2|2077|710|9257|Saturday|2077Q2|N|Y|N|2479790|2479909|2479453|2479728|N|N|N|N|N| +2479819|AAAAAAAALMGNFCAA|2077-05-30|2128|9257|710|2077|0|5|30|2|2077|710|9257|Sunday|2077Q2|N|N|N|2479790|2479909|2479454|2479729|N|N|N|N|N| +2479820|AAAAAAAAMMGNFCAA|2077-05-31|2128|9257|710|2077|1|5|31|2|2077|710|9257|Monday|2077Q2|N|N|N|2479790|2479909|2479455|2479730|N|N|N|N|N| +2479821|AAAAAAAANMGNFCAA|2077-06-01|2129|9258|711|2077|2|6|1|2|2077|711|9258|Tuesday|2077Q2|N|N|N|2479821|2479971|2479456|2479731|N|N|N|N|N| +2479822|AAAAAAAAOMGNFCAA|2077-06-02|2129|9258|711|2077|3|6|2|2|2077|711|9258|Wednesday|2077Q2|N|N|N|2479821|2479971|2479457|2479732|N|N|N|N|N| +2479823|AAAAAAAAPMGNFCAA|2077-06-03|2129|9258|711|2077|4|6|3|2|2077|711|9258|Thursday|2077Q2|N|N|N|2479821|2479971|2479458|2479733|N|N|N|N|N| +2479824|AAAAAAAAANGNFCAA|2077-06-04|2129|9258|711|2077|5|6|4|2|2077|711|9258|Friday|2077Q2|N|Y|N|2479821|2479971|2479459|2479734|N|N|N|N|N| +2479825|AAAAAAAABNGNFCAA|2077-06-05|2129|9258|711|2077|6|6|5|2|2077|711|9258|Saturday|2077Q2|N|Y|N|2479821|2479971|2479460|2479735|N|N|N|N|N| +2479826|AAAAAAAACNGNFCAA|2077-06-06|2129|9258|711|2077|0|6|6|2|2077|711|9258|Sunday|2077Q2|N|N|N|2479821|2479971|2479461|2479736|N|N|N|N|N| +2479827|AAAAAAAADNGNFCAA|2077-06-07|2129|9258|711|2077|1|6|7|2|2077|711|9258|Monday|2077Q2|N|N|N|2479821|2479971|2479462|2479737|N|N|N|N|N| +2479828|AAAAAAAAENGNFCAA|2077-06-08|2129|9259|711|2077|2|6|8|2|2077|711|9259|Tuesday|2077Q2|N|N|N|2479821|2479971|2479463|2479738|N|N|N|N|N| +2479829|AAAAAAAAFNGNFCAA|2077-06-09|2129|9259|711|2077|3|6|9|2|2077|711|9259|Wednesday|2077Q2|N|N|N|2479821|2479971|2479464|2479739|N|N|N|N|N| +2479830|AAAAAAAAGNGNFCAA|2077-06-10|2129|9259|711|2077|4|6|10|2|2077|711|9259|Thursday|2077Q2|N|N|N|2479821|2479971|2479465|2479740|N|N|N|N|N| +2479831|AAAAAAAAHNGNFCAA|2077-06-11|2129|9259|711|2077|5|6|11|2|2077|711|9259|Friday|2077Q2|N|Y|N|2479821|2479971|2479466|2479741|N|N|N|N|N| +2479832|AAAAAAAAINGNFCAA|2077-06-12|2129|9259|711|2077|6|6|12|2|2077|711|9259|Saturday|2077Q2|N|Y|N|2479821|2479971|2479467|2479742|N|N|N|N|N| +2479833|AAAAAAAAJNGNFCAA|2077-06-13|2129|9259|711|2077|0|6|13|2|2077|711|9259|Sunday|2077Q2|N|N|N|2479821|2479971|2479468|2479743|N|N|N|N|N| +2479834|AAAAAAAAKNGNFCAA|2077-06-14|2129|9259|711|2077|1|6|14|2|2077|711|9259|Monday|2077Q2|N|N|N|2479821|2479971|2479469|2479744|N|N|N|N|N| +2479835|AAAAAAAALNGNFCAA|2077-06-15|2129|9260|711|2077|2|6|15|2|2077|711|9260|Tuesday|2077Q2|N|N|N|2479821|2479971|2479470|2479745|N|N|N|N|N| +2479836|AAAAAAAAMNGNFCAA|2077-06-16|2129|9260|711|2077|3|6|16|2|2077|711|9260|Wednesday|2077Q2|N|N|N|2479821|2479971|2479471|2479746|N|N|N|N|N| +2479837|AAAAAAAANNGNFCAA|2077-06-17|2129|9260|711|2077|4|6|17|2|2077|711|9260|Thursday|2077Q2|N|N|N|2479821|2479971|2479472|2479747|N|N|N|N|N| +2479838|AAAAAAAAONGNFCAA|2077-06-18|2129|9260|711|2077|5|6|18|2|2077|711|9260|Friday|2077Q2|N|Y|N|2479821|2479971|2479473|2479748|N|N|N|N|N| +2479839|AAAAAAAAPNGNFCAA|2077-06-19|2129|9260|711|2077|6|6|19|2|2077|711|9260|Saturday|2077Q2|N|Y|N|2479821|2479971|2479474|2479749|N|N|N|N|N| +2479840|AAAAAAAAAOGNFCAA|2077-06-20|2129|9260|711|2077|0|6|20|2|2077|711|9260|Sunday|2077Q2|N|N|N|2479821|2479971|2479475|2479750|N|N|N|N|N| +2479841|AAAAAAAABOGNFCAA|2077-06-21|2129|9260|711|2077|1|6|21|2|2077|711|9260|Monday|2077Q2|N|N|N|2479821|2479971|2479476|2479751|N|N|N|N|N| +2479842|AAAAAAAACOGNFCAA|2077-06-22|2129|9261|711|2077|2|6|22|2|2077|711|9261|Tuesday|2077Q2|N|N|N|2479821|2479971|2479477|2479752|N|N|N|N|N| +2479843|AAAAAAAADOGNFCAA|2077-06-23|2129|9261|711|2077|3|6|23|2|2077|711|9261|Wednesday|2077Q2|N|N|N|2479821|2479971|2479478|2479753|N|N|N|N|N| +2479844|AAAAAAAAEOGNFCAA|2077-06-24|2129|9261|711|2077|4|6|24|2|2077|711|9261|Thursday|2077Q2|N|N|N|2479821|2479971|2479479|2479754|N|N|N|N|N| +2479845|AAAAAAAAFOGNFCAA|2077-06-25|2129|9261|711|2077|5|6|25|2|2077|711|9261|Friday|2077Q2|N|Y|N|2479821|2479971|2479480|2479755|N|N|N|N|N| +2479846|AAAAAAAAGOGNFCAA|2077-06-26|2129|9261|711|2077|6|6|26|2|2077|711|9261|Saturday|2077Q2|N|Y|N|2479821|2479971|2479481|2479756|N|N|N|N|N| +2479847|AAAAAAAAHOGNFCAA|2077-06-27|2129|9261|711|2077|0|6|27|2|2077|711|9261|Sunday|2077Q2|N|N|N|2479821|2479971|2479482|2479757|N|N|N|N|N| +2479848|AAAAAAAAIOGNFCAA|2077-06-28|2129|9261|711|2077|1|6|28|2|2077|711|9261|Monday|2077Q2|N|N|N|2479821|2479971|2479483|2479758|N|N|N|N|N| +2479849|AAAAAAAAJOGNFCAA|2077-06-29|2129|9262|711|2077|2|6|29|2|2077|711|9262|Tuesday|2077Q2|N|N|N|2479821|2479971|2479484|2479759|N|N|N|N|N| +2479850|AAAAAAAAKOGNFCAA|2077-06-30|2129|9262|711|2077|3|6|30|2|2077|711|9262|Wednesday|2077Q2|N|N|N|2479821|2479971|2479485|2479760|N|N|N|N|N| +2479851|AAAAAAAALOGNFCAA|2077-07-01|2130|9262|711|2077|4|7|1|2|2077|711|9262|Thursday|2077Q2|N|N|N|2479851|2480031|2479486|2479760|N|N|N|N|N| +2479852|AAAAAAAAMOGNFCAA|2077-07-02|2130|9262|711|2077|5|7|2|3|2077|711|9262|Friday|2077Q3|N|Y|N|2479851|2480031|2479487|2479761|N|N|N|N|N| +2479853|AAAAAAAANOGNFCAA|2077-07-03|2130|9262|711|2077|6|7|3|3|2077|711|9262|Saturday|2077Q3|N|Y|N|2479851|2480031|2479488|2479762|N|N|N|N|N| +2479854|AAAAAAAAOOGNFCAA|2077-07-04|2130|9262|711|2077|0|7|4|3|2077|711|9262|Sunday|2077Q3|N|N|N|2479851|2480031|2479489|2479763|N|N|N|N|N| +2479855|AAAAAAAAPOGNFCAA|2077-07-05|2130|9262|711|2077|1|7|5|3|2077|711|9262|Monday|2077Q3|Y|N|N|2479851|2480031|2479490|2479764|N|N|N|N|N| +2479856|AAAAAAAAAPGNFCAA|2077-07-06|2130|9263|711|2077|2|7|6|3|2077|711|9263|Tuesday|2077Q3|N|N|Y|2479851|2480031|2479491|2479765|N|N|N|N|N| +2479857|AAAAAAAABPGNFCAA|2077-07-07|2130|9263|711|2077|3|7|7|3|2077|711|9263|Wednesday|2077Q3|N|N|N|2479851|2480031|2479492|2479766|N|N|N|N|N| +2479858|AAAAAAAACPGNFCAA|2077-07-08|2130|9263|711|2077|4|7|8|3|2077|711|9263|Thursday|2077Q3|N|N|N|2479851|2480031|2479493|2479767|N|N|N|N|N| +2479859|AAAAAAAADPGNFCAA|2077-07-09|2130|9263|711|2077|5|7|9|3|2077|711|9263|Friday|2077Q3|N|Y|N|2479851|2480031|2479494|2479768|N|N|N|N|N| +2479860|AAAAAAAAEPGNFCAA|2077-07-10|2130|9263|711|2077|6|7|10|3|2077|711|9263|Saturday|2077Q3|N|Y|N|2479851|2480031|2479495|2479769|N|N|N|N|N| +2479861|AAAAAAAAFPGNFCAA|2077-07-11|2130|9263|711|2077|0|7|11|3|2077|711|9263|Sunday|2077Q3|N|N|N|2479851|2480031|2479496|2479770|N|N|N|N|N| +2479862|AAAAAAAAGPGNFCAA|2077-07-12|2130|9263|711|2077|1|7|12|3|2077|711|9263|Monday|2077Q3|N|N|N|2479851|2480031|2479497|2479771|N|N|N|N|N| +2479863|AAAAAAAAHPGNFCAA|2077-07-13|2130|9264|711|2077|2|7|13|3|2077|711|9264|Tuesday|2077Q3|N|N|N|2479851|2480031|2479498|2479772|N|N|N|N|N| +2479864|AAAAAAAAIPGNFCAA|2077-07-14|2130|9264|711|2077|3|7|14|3|2077|711|9264|Wednesday|2077Q3|N|N|N|2479851|2480031|2479499|2479773|N|N|N|N|N| +2479865|AAAAAAAAJPGNFCAA|2077-07-15|2130|9264|711|2077|4|7|15|3|2077|711|9264|Thursday|2077Q3|N|N|N|2479851|2480031|2479500|2479774|N|N|N|N|N| +2479866|AAAAAAAAKPGNFCAA|2077-07-16|2130|9264|711|2077|5|7|16|3|2077|711|9264|Friday|2077Q3|N|Y|N|2479851|2480031|2479501|2479775|N|N|N|N|N| +2479867|AAAAAAAALPGNFCAA|2077-07-17|2130|9264|711|2077|6|7|17|3|2077|711|9264|Saturday|2077Q3|N|Y|N|2479851|2480031|2479502|2479776|N|N|N|N|N| +2479868|AAAAAAAAMPGNFCAA|2077-07-18|2130|9264|711|2077|0|7|18|3|2077|711|9264|Sunday|2077Q3|N|N|N|2479851|2480031|2479503|2479777|N|N|N|N|N| +2479869|AAAAAAAANPGNFCAA|2077-07-19|2130|9264|711|2077|1|7|19|3|2077|711|9264|Monday|2077Q3|N|N|N|2479851|2480031|2479504|2479778|N|N|N|N|N| +2479870|AAAAAAAAOPGNFCAA|2077-07-20|2130|9265|711|2077|2|7|20|3|2077|711|9265|Tuesday|2077Q3|N|N|N|2479851|2480031|2479505|2479779|N|N|N|N|N| +2479871|AAAAAAAAPPGNFCAA|2077-07-21|2130|9265|711|2077|3|7|21|3|2077|711|9265|Wednesday|2077Q3|N|N|N|2479851|2480031|2479506|2479780|N|N|N|N|N| +2479872|AAAAAAAAAAHNFCAA|2077-07-22|2130|9265|711|2077|4|7|22|3|2077|711|9265|Thursday|2077Q3|N|N|N|2479851|2480031|2479507|2479781|N|N|N|N|N| +2479873|AAAAAAAABAHNFCAA|2077-07-23|2130|9265|711|2077|5|7|23|3|2077|711|9265|Friday|2077Q3|N|Y|N|2479851|2480031|2479508|2479782|N|N|N|N|N| +2479874|AAAAAAAACAHNFCAA|2077-07-24|2130|9265|711|2077|6|7|24|3|2077|711|9265|Saturday|2077Q3|N|Y|N|2479851|2480031|2479509|2479783|N|N|N|N|N| +2479875|AAAAAAAADAHNFCAA|2077-07-25|2130|9265|711|2077|0|7|25|3|2077|711|9265|Sunday|2077Q3|N|N|N|2479851|2480031|2479510|2479784|N|N|N|N|N| +2479876|AAAAAAAAEAHNFCAA|2077-07-26|2130|9265|711|2077|1|7|26|3|2077|711|9265|Monday|2077Q3|N|N|N|2479851|2480031|2479511|2479785|N|N|N|N|N| +2479877|AAAAAAAAFAHNFCAA|2077-07-27|2130|9266|711|2077|2|7|27|3|2077|711|9266|Tuesday|2077Q3|N|N|N|2479851|2480031|2479512|2479786|N|N|N|N|N| +2479878|AAAAAAAAGAHNFCAA|2077-07-28|2130|9266|711|2077|3|7|28|3|2077|711|9266|Wednesday|2077Q3|N|N|N|2479851|2480031|2479513|2479787|N|N|N|N|N| +2479879|AAAAAAAAHAHNFCAA|2077-07-29|2130|9266|711|2077|4|7|29|3|2077|711|9266|Thursday|2077Q3|N|N|N|2479851|2480031|2479514|2479788|N|N|N|N|N| +2479880|AAAAAAAAIAHNFCAA|2077-07-30|2130|9266|711|2077|5|7|30|3|2077|711|9266|Friday|2077Q3|N|Y|N|2479851|2480031|2479515|2479789|N|N|N|N|N| +2479881|AAAAAAAAJAHNFCAA|2077-07-31|2130|9266|711|2077|6|7|31|3|2077|711|9266|Saturday|2077Q3|N|Y|N|2479851|2480031|2479516|2479790|N|N|N|N|N| +2479882|AAAAAAAAKAHNFCAA|2077-08-01|2131|9266|711|2077|0|8|1|3|2077|711|9266|Sunday|2077Q3|N|N|N|2479882|2480093|2479517|2479791|N|N|N|N|N| +2479883|AAAAAAAALAHNFCAA|2077-08-02|2131|9266|711|2077|1|8|2|3|2077|711|9266|Monday|2077Q3|N|N|N|2479882|2480093|2479518|2479792|N|N|N|N|N| +2479884|AAAAAAAAMAHNFCAA|2077-08-03|2131|9267|711|2077|2|8|3|3|2077|711|9267|Tuesday|2077Q3|N|N|N|2479882|2480093|2479519|2479793|N|N|N|N|N| +2479885|AAAAAAAANAHNFCAA|2077-08-04|2131|9267|711|2077|3|8|4|3|2077|711|9267|Wednesday|2077Q3|N|N|N|2479882|2480093|2479520|2479794|N|N|N|N|N| +2479886|AAAAAAAAOAHNFCAA|2077-08-05|2131|9267|711|2077|4|8|5|3|2077|711|9267|Thursday|2077Q3|N|N|N|2479882|2480093|2479521|2479795|N|N|N|N|N| +2479887|AAAAAAAAPAHNFCAA|2077-08-06|2131|9267|711|2077|5|8|6|3|2077|711|9267|Friday|2077Q3|N|Y|N|2479882|2480093|2479522|2479796|N|N|N|N|N| +2479888|AAAAAAAAABHNFCAA|2077-08-07|2131|9267|711|2077|6|8|7|3|2077|711|9267|Saturday|2077Q3|N|Y|N|2479882|2480093|2479523|2479797|N|N|N|N|N| +2479889|AAAAAAAABBHNFCAA|2077-08-08|2131|9267|711|2077|0|8|8|3|2077|711|9267|Sunday|2077Q3|N|N|N|2479882|2480093|2479524|2479798|N|N|N|N|N| +2479890|AAAAAAAACBHNFCAA|2077-08-09|2131|9267|711|2077|1|8|9|3|2077|711|9267|Monday|2077Q3|N|N|N|2479882|2480093|2479525|2479799|N|N|N|N|N| +2479891|AAAAAAAADBHNFCAA|2077-08-10|2131|9268|711|2077|2|8|10|3|2077|711|9268|Tuesday|2077Q3|N|N|N|2479882|2480093|2479526|2479800|N|N|N|N|N| +2479892|AAAAAAAAEBHNFCAA|2077-08-11|2131|9268|711|2077|3|8|11|3|2077|711|9268|Wednesday|2077Q3|N|N|N|2479882|2480093|2479527|2479801|N|N|N|N|N| +2479893|AAAAAAAAFBHNFCAA|2077-08-12|2131|9268|711|2077|4|8|12|3|2077|711|9268|Thursday|2077Q3|N|N|N|2479882|2480093|2479528|2479802|N|N|N|N|N| +2479894|AAAAAAAAGBHNFCAA|2077-08-13|2131|9268|711|2077|5|8|13|3|2077|711|9268|Friday|2077Q3|N|Y|N|2479882|2480093|2479529|2479803|N|N|N|N|N| +2479895|AAAAAAAAHBHNFCAA|2077-08-14|2131|9268|711|2077|6|8|14|3|2077|711|9268|Saturday|2077Q3|N|Y|N|2479882|2480093|2479530|2479804|N|N|N|N|N| +2479896|AAAAAAAAIBHNFCAA|2077-08-15|2131|9268|711|2077|0|8|15|3|2077|711|9268|Sunday|2077Q3|N|N|N|2479882|2480093|2479531|2479805|N|N|N|N|N| +2479897|AAAAAAAAJBHNFCAA|2077-08-16|2131|9268|711|2077|1|8|16|3|2077|711|9268|Monday|2077Q3|N|N|N|2479882|2480093|2479532|2479806|N|N|N|N|N| +2479898|AAAAAAAAKBHNFCAA|2077-08-17|2131|9269|711|2077|2|8|17|3|2077|711|9269|Tuesday|2077Q3|N|N|N|2479882|2480093|2479533|2479807|N|N|N|N|N| +2479899|AAAAAAAALBHNFCAA|2077-08-18|2131|9269|711|2077|3|8|18|3|2077|711|9269|Wednesday|2077Q3|N|N|N|2479882|2480093|2479534|2479808|N|N|N|N|N| +2479900|AAAAAAAAMBHNFCAA|2077-08-19|2131|9269|711|2077|4|8|19|3|2077|711|9269|Thursday|2077Q3|N|N|N|2479882|2480093|2479535|2479809|N|N|N|N|N| +2479901|AAAAAAAANBHNFCAA|2077-08-20|2131|9269|711|2077|5|8|20|3|2077|711|9269|Friday|2077Q3|N|Y|N|2479882|2480093|2479536|2479810|N|N|N|N|N| +2479902|AAAAAAAAOBHNFCAA|2077-08-21|2131|9269|711|2077|6|8|21|3|2077|711|9269|Saturday|2077Q3|N|Y|N|2479882|2480093|2479537|2479811|N|N|N|N|N| +2479903|AAAAAAAAPBHNFCAA|2077-08-22|2131|9269|711|2077|0|8|22|3|2077|711|9269|Sunday|2077Q3|N|N|N|2479882|2480093|2479538|2479812|N|N|N|N|N| +2479904|AAAAAAAAACHNFCAA|2077-08-23|2131|9269|711|2077|1|8|23|3|2077|711|9269|Monday|2077Q3|N|N|N|2479882|2480093|2479539|2479813|N|N|N|N|N| +2479905|AAAAAAAABCHNFCAA|2077-08-24|2131|9270|711|2077|2|8|24|3|2077|711|9270|Tuesday|2077Q3|N|N|N|2479882|2480093|2479540|2479814|N|N|N|N|N| +2479906|AAAAAAAACCHNFCAA|2077-08-25|2131|9270|711|2077|3|8|25|3|2077|711|9270|Wednesday|2077Q3|N|N|N|2479882|2480093|2479541|2479815|N|N|N|N|N| +2479907|AAAAAAAADCHNFCAA|2077-08-26|2131|9270|711|2077|4|8|26|3|2077|711|9270|Thursday|2077Q3|N|N|N|2479882|2480093|2479542|2479816|N|N|N|N|N| +2479908|AAAAAAAAECHNFCAA|2077-08-27|2131|9270|711|2077|5|8|27|3|2077|711|9270|Friday|2077Q3|N|Y|N|2479882|2480093|2479543|2479817|N|N|N|N|N| +2479909|AAAAAAAAFCHNFCAA|2077-08-28|2131|9270|711|2077|6|8|28|3|2077|711|9270|Saturday|2077Q3|N|Y|N|2479882|2480093|2479544|2479818|N|N|N|N|N| +2479910|AAAAAAAAGCHNFCAA|2077-08-29|2131|9270|711|2077|0|8|29|3|2077|711|9270|Sunday|2077Q3|N|N|N|2479882|2480093|2479545|2479819|N|N|N|N|N| +2479911|AAAAAAAAHCHNFCAA|2077-08-30|2131|9270|711|2077|1|8|30|3|2077|711|9270|Monday|2077Q3|N|N|N|2479882|2480093|2479546|2479820|N|N|N|N|N| +2479912|AAAAAAAAICHNFCAA|2077-08-31|2131|9271|711|2077|2|8|31|3|2077|711|9271|Tuesday|2077Q3|N|N|N|2479882|2480093|2479547|2479821|N|N|N|N|N| +2479913|AAAAAAAAJCHNFCAA|2077-09-01|2132|9271|712|2077|3|9|1|3|2077|712|9271|Wednesday|2077Q3|N|N|N|2479913|2480155|2479548|2479822|N|N|N|N|N| +2479914|AAAAAAAAKCHNFCAA|2077-09-02|2132|9271|712|2077|4|9|2|3|2077|712|9271|Thursday|2077Q3|N|N|N|2479913|2480155|2479549|2479823|N|N|N|N|N| +2479915|AAAAAAAALCHNFCAA|2077-09-03|2132|9271|712|2077|5|9|3|3|2077|712|9271|Friday|2077Q3|N|Y|N|2479913|2480155|2479550|2479824|N|N|N|N|N| +2479916|AAAAAAAAMCHNFCAA|2077-09-04|2132|9271|712|2077|6|9|4|3|2077|712|9271|Saturday|2077Q3|N|Y|N|2479913|2480155|2479551|2479825|N|N|N|N|N| +2479917|AAAAAAAANCHNFCAA|2077-09-05|2132|9271|712|2077|0|9|5|3|2077|712|9271|Sunday|2077Q3|N|N|N|2479913|2480155|2479552|2479826|N|N|N|N|N| +2479918|AAAAAAAAOCHNFCAA|2077-09-06|2132|9271|712|2077|1|9|6|3|2077|712|9271|Monday|2077Q3|N|N|N|2479913|2480155|2479553|2479827|N|N|N|N|N| +2479919|AAAAAAAAPCHNFCAA|2077-09-07|2132|9272|712|2077|2|9|7|3|2077|712|9272|Tuesday|2077Q3|N|N|N|2479913|2480155|2479554|2479828|N|N|N|N|N| +2479920|AAAAAAAAADHNFCAA|2077-09-08|2132|9272|712|2077|3|9|8|3|2077|712|9272|Wednesday|2077Q3|N|N|N|2479913|2480155|2479555|2479829|N|N|N|N|N| +2479921|AAAAAAAABDHNFCAA|2077-09-09|2132|9272|712|2077|4|9|9|3|2077|712|9272|Thursday|2077Q3|N|N|N|2479913|2480155|2479556|2479830|N|N|N|N|N| +2479922|AAAAAAAACDHNFCAA|2077-09-10|2132|9272|712|2077|5|9|10|3|2077|712|9272|Friday|2077Q3|N|Y|N|2479913|2480155|2479557|2479831|N|N|N|N|N| +2479923|AAAAAAAADDHNFCAA|2077-09-11|2132|9272|712|2077|6|9|11|3|2077|712|9272|Saturday|2077Q3|N|Y|N|2479913|2480155|2479558|2479832|N|N|N|N|N| +2479924|AAAAAAAAEDHNFCAA|2077-09-12|2132|9272|712|2077|0|9|12|3|2077|712|9272|Sunday|2077Q3|N|N|N|2479913|2480155|2479559|2479833|N|N|N|N|N| +2479925|AAAAAAAAFDHNFCAA|2077-09-13|2132|9272|712|2077|1|9|13|3|2077|712|9272|Monday|2077Q3|N|N|N|2479913|2480155|2479560|2479834|N|N|N|N|N| +2479926|AAAAAAAAGDHNFCAA|2077-09-14|2132|9273|712|2077|2|9|14|3|2077|712|9273|Tuesday|2077Q3|N|N|N|2479913|2480155|2479561|2479835|N|N|N|N|N| +2479927|AAAAAAAAHDHNFCAA|2077-09-15|2132|9273|712|2077|3|9|15|3|2077|712|9273|Wednesday|2077Q3|N|N|N|2479913|2480155|2479562|2479836|N|N|N|N|N| +2479928|AAAAAAAAIDHNFCAA|2077-09-16|2132|9273|712|2077|4|9|16|3|2077|712|9273|Thursday|2077Q3|N|N|N|2479913|2480155|2479563|2479837|N|N|N|N|N| +2479929|AAAAAAAAJDHNFCAA|2077-09-17|2132|9273|712|2077|5|9|17|3|2077|712|9273|Friday|2077Q3|N|Y|N|2479913|2480155|2479564|2479838|N|N|N|N|N| +2479930|AAAAAAAAKDHNFCAA|2077-09-18|2132|9273|712|2077|6|9|18|3|2077|712|9273|Saturday|2077Q3|N|Y|N|2479913|2480155|2479565|2479839|N|N|N|N|N| +2479931|AAAAAAAALDHNFCAA|2077-09-19|2132|9273|712|2077|0|9|19|3|2077|712|9273|Sunday|2077Q3|N|N|N|2479913|2480155|2479566|2479840|N|N|N|N|N| +2479932|AAAAAAAAMDHNFCAA|2077-09-20|2132|9273|712|2077|1|9|20|3|2077|712|9273|Monday|2077Q3|N|N|N|2479913|2480155|2479567|2479841|N|N|N|N|N| +2479933|AAAAAAAANDHNFCAA|2077-09-21|2132|9274|712|2077|2|9|21|3|2077|712|9274|Tuesday|2077Q3|N|N|N|2479913|2480155|2479568|2479842|N|N|N|N|N| +2479934|AAAAAAAAODHNFCAA|2077-09-22|2132|9274|712|2077|3|9|22|3|2077|712|9274|Wednesday|2077Q3|N|N|N|2479913|2480155|2479569|2479843|N|N|N|N|N| +2479935|AAAAAAAAPDHNFCAA|2077-09-23|2132|9274|712|2077|4|9|23|3|2077|712|9274|Thursday|2077Q3|N|N|N|2479913|2480155|2479570|2479844|N|N|N|N|N| +2479936|AAAAAAAAAEHNFCAA|2077-09-24|2132|9274|712|2077|5|9|24|3|2077|712|9274|Friday|2077Q3|N|Y|N|2479913|2480155|2479571|2479845|N|N|N|N|N| +2479937|AAAAAAAABEHNFCAA|2077-09-25|2132|9274|712|2077|6|9|25|3|2077|712|9274|Saturday|2077Q3|N|Y|N|2479913|2480155|2479572|2479846|N|N|N|N|N| +2479938|AAAAAAAACEHNFCAA|2077-09-26|2132|9274|712|2077|0|9|26|3|2077|712|9274|Sunday|2077Q3|N|N|N|2479913|2480155|2479573|2479847|N|N|N|N|N| +2479939|AAAAAAAADEHNFCAA|2077-09-27|2132|9274|712|2077|1|9|27|3|2077|712|9274|Monday|2077Q3|N|N|N|2479913|2480155|2479574|2479848|N|N|N|N|N| +2479940|AAAAAAAAEEHNFCAA|2077-09-28|2132|9275|712|2077|2|9|28|3|2077|712|9275|Tuesday|2077Q3|N|N|N|2479913|2480155|2479575|2479849|N|N|N|N|N| +2479941|AAAAAAAAFEHNFCAA|2077-09-29|2132|9275|712|2077|3|9|29|3|2077|712|9275|Wednesday|2077Q3|N|N|N|2479913|2480155|2479576|2479850|N|N|N|N|N| +2479942|AAAAAAAAGEHNFCAA|2077-09-30|2132|9275|712|2077|4|9|30|3|2077|712|9275|Thursday|2077Q3|N|N|N|2479913|2480155|2479577|2479851|N|N|N|N|N| +2479943|AAAAAAAAHEHNFCAA|2077-10-01|2133|9275|712|2077|5|10|1|3|2077|712|9275|Friday|2077Q3|N|Y|N|2479943|2480215|2479578|2479851|N|N|N|N|N| +2479944|AAAAAAAAIEHNFCAA|2077-10-02|2133|9275|712|2077|6|10|2|4|2077|712|9275|Saturday|2077Q4|N|Y|N|2479943|2480215|2479579|2479852|N|N|N|N|N| +2479945|AAAAAAAAJEHNFCAA|2077-10-03|2133|9275|712|2077|0|10|3|4|2077|712|9275|Sunday|2077Q4|N|N|N|2479943|2480215|2479580|2479853|N|N|N|N|N| +2479946|AAAAAAAAKEHNFCAA|2077-10-04|2133|9275|712|2077|1|10|4|4|2077|712|9275|Monday|2077Q4|N|N|N|2479943|2480215|2479581|2479854|N|N|N|N|N| +2479947|AAAAAAAALEHNFCAA|2077-10-05|2133|9276|712|2077|2|10|5|4|2077|712|9276|Tuesday|2077Q4|N|N|N|2479943|2480215|2479582|2479855|N|N|N|N|N| +2479948|AAAAAAAAMEHNFCAA|2077-10-06|2133|9276|712|2077|3|10|6|4|2077|712|9276|Wednesday|2077Q4|N|N|N|2479943|2480215|2479583|2479856|N|N|N|N|N| +2479949|AAAAAAAANEHNFCAA|2077-10-07|2133|9276|712|2077|4|10|7|4|2077|712|9276|Thursday|2077Q4|N|N|N|2479943|2480215|2479584|2479857|N|N|N|N|N| +2479950|AAAAAAAAOEHNFCAA|2077-10-08|2133|9276|712|2077|5|10|8|4|2077|712|9276|Friday|2077Q4|N|Y|N|2479943|2480215|2479585|2479858|N|N|N|N|N| +2479951|AAAAAAAAPEHNFCAA|2077-10-09|2133|9276|712|2077|6|10|9|4|2077|712|9276|Saturday|2077Q4|N|Y|N|2479943|2480215|2479586|2479859|N|N|N|N|N| +2479952|AAAAAAAAAFHNFCAA|2077-10-10|2133|9276|712|2077|0|10|10|4|2077|712|9276|Sunday|2077Q4|N|N|N|2479943|2480215|2479587|2479860|N|N|N|N|N| +2479953|AAAAAAAABFHNFCAA|2077-10-11|2133|9276|712|2077|1|10|11|4|2077|712|9276|Monday|2077Q4|N|N|N|2479943|2480215|2479588|2479861|N|N|N|N|N| +2479954|AAAAAAAACFHNFCAA|2077-10-12|2133|9277|712|2077|2|10|12|4|2077|712|9277|Tuesday|2077Q4|N|N|N|2479943|2480215|2479589|2479862|N|N|N|N|N| +2479955|AAAAAAAADFHNFCAA|2077-10-13|2133|9277|712|2077|3|10|13|4|2077|712|9277|Wednesday|2077Q4|N|N|N|2479943|2480215|2479590|2479863|N|N|N|N|N| +2479956|AAAAAAAAEFHNFCAA|2077-10-14|2133|9277|712|2077|4|10|14|4|2077|712|9277|Thursday|2077Q4|N|N|N|2479943|2480215|2479591|2479864|N|N|N|N|N| +2479957|AAAAAAAAFFHNFCAA|2077-10-15|2133|9277|712|2077|5|10|15|4|2077|712|9277|Friday|2077Q4|N|Y|N|2479943|2480215|2479592|2479865|N|N|N|N|N| +2479958|AAAAAAAAGFHNFCAA|2077-10-16|2133|9277|712|2077|6|10|16|4|2077|712|9277|Saturday|2077Q4|N|Y|N|2479943|2480215|2479593|2479866|N|N|N|N|N| +2479959|AAAAAAAAHFHNFCAA|2077-10-17|2133|9277|712|2077|0|10|17|4|2077|712|9277|Sunday|2077Q4|N|N|N|2479943|2480215|2479594|2479867|N|N|N|N|N| +2479960|AAAAAAAAIFHNFCAA|2077-10-18|2133|9277|712|2077|1|10|18|4|2077|712|9277|Monday|2077Q4|N|N|N|2479943|2480215|2479595|2479868|N|N|N|N|N| +2479961|AAAAAAAAJFHNFCAA|2077-10-19|2133|9278|712|2077|2|10|19|4|2077|712|9278|Tuesday|2077Q4|N|N|N|2479943|2480215|2479596|2479869|N|N|N|N|N| +2479962|AAAAAAAAKFHNFCAA|2077-10-20|2133|9278|712|2077|3|10|20|4|2077|712|9278|Wednesday|2077Q4|N|N|N|2479943|2480215|2479597|2479870|N|N|N|N|N| +2479963|AAAAAAAALFHNFCAA|2077-10-21|2133|9278|712|2077|4|10|21|4|2077|712|9278|Thursday|2077Q4|N|N|N|2479943|2480215|2479598|2479871|N|N|N|N|N| +2479964|AAAAAAAAMFHNFCAA|2077-10-22|2133|9278|712|2077|5|10|22|4|2077|712|9278|Friday|2077Q4|N|Y|N|2479943|2480215|2479599|2479872|N|N|N|N|N| +2479965|AAAAAAAANFHNFCAA|2077-10-23|2133|9278|712|2077|6|10|23|4|2077|712|9278|Saturday|2077Q4|N|Y|N|2479943|2480215|2479600|2479873|N|N|N|N|N| +2479966|AAAAAAAAOFHNFCAA|2077-10-24|2133|9278|712|2077|0|10|24|4|2077|712|9278|Sunday|2077Q4|N|N|N|2479943|2480215|2479601|2479874|N|N|N|N|N| +2479967|AAAAAAAAPFHNFCAA|2077-10-25|2133|9278|712|2077|1|10|25|4|2077|712|9278|Monday|2077Q4|N|N|N|2479943|2480215|2479602|2479875|N|N|N|N|N| +2479968|AAAAAAAAAGHNFCAA|2077-10-26|2133|9279|712|2077|2|10|26|4|2077|712|9279|Tuesday|2077Q4|N|N|N|2479943|2480215|2479603|2479876|N|N|N|N|N| +2479969|AAAAAAAABGHNFCAA|2077-10-27|2133|9279|712|2077|3|10|27|4|2077|712|9279|Wednesday|2077Q4|N|N|N|2479943|2480215|2479604|2479877|N|N|N|N|N| +2479970|AAAAAAAACGHNFCAA|2077-10-28|2133|9279|712|2077|4|10|28|4|2077|712|9279|Thursday|2077Q4|N|N|N|2479943|2480215|2479605|2479878|N|N|N|N|N| +2479971|AAAAAAAADGHNFCAA|2077-10-29|2133|9279|712|2077|5|10|29|4|2077|712|9279|Friday|2077Q4|N|Y|N|2479943|2480215|2479606|2479879|N|N|N|N|N| +2479972|AAAAAAAAEGHNFCAA|2077-10-30|2133|9279|712|2077|6|10|30|4|2077|712|9279|Saturday|2077Q4|N|Y|N|2479943|2480215|2479607|2479880|N|N|N|N|N| +2479973|AAAAAAAAFGHNFCAA|2077-10-31|2133|9279|712|2077|0|10|31|4|2077|712|9279|Sunday|2077Q4|N|N|N|2479943|2480215|2479608|2479881|N|N|N|N|N| +2479974|AAAAAAAAGGHNFCAA|2077-11-01|2134|9279|712|2077|1|11|1|4|2077|712|9279|Monday|2077Q4|N|N|N|2479974|2480277|2479609|2479882|N|N|N|N|N| +2479975|AAAAAAAAHGHNFCAA|2077-11-02|2134|9280|712|2077|2|11|2|4|2077|712|9280|Tuesday|2077Q4|N|N|N|2479974|2480277|2479610|2479883|N|N|N|N|N| +2479976|AAAAAAAAIGHNFCAA|2077-11-03|2134|9280|712|2077|3|11|3|4|2077|712|9280|Wednesday|2077Q4|N|N|N|2479974|2480277|2479611|2479884|N|N|N|N|N| +2479977|AAAAAAAAJGHNFCAA|2077-11-04|2134|9280|712|2077|4|11|4|4|2077|712|9280|Thursday|2077Q4|N|N|N|2479974|2480277|2479612|2479885|N|N|N|N|N| +2479978|AAAAAAAAKGHNFCAA|2077-11-05|2134|9280|712|2077|5|11|5|4|2077|712|9280|Friday|2077Q4|N|Y|N|2479974|2480277|2479613|2479886|N|N|N|N|N| +2479979|AAAAAAAALGHNFCAA|2077-11-06|2134|9280|712|2077|6|11|6|4|2077|712|9280|Saturday|2077Q4|N|Y|N|2479974|2480277|2479614|2479887|N|N|N|N|N| +2479980|AAAAAAAAMGHNFCAA|2077-11-07|2134|9280|712|2077|0|11|7|4|2077|712|9280|Sunday|2077Q4|N|N|N|2479974|2480277|2479615|2479888|N|N|N|N|N| +2479981|AAAAAAAANGHNFCAA|2077-11-08|2134|9280|712|2077|1|11|8|4|2077|712|9280|Monday|2077Q4|N|N|N|2479974|2480277|2479616|2479889|N|N|N|N|N| +2479982|AAAAAAAAOGHNFCAA|2077-11-09|2134|9281|712|2077|2|11|9|4|2077|712|9281|Tuesday|2077Q4|N|N|N|2479974|2480277|2479617|2479890|N|N|N|N|N| +2479983|AAAAAAAAPGHNFCAA|2077-11-10|2134|9281|712|2077|3|11|10|4|2077|712|9281|Wednesday|2077Q4|N|N|N|2479974|2480277|2479618|2479891|N|N|N|N|N| +2479984|AAAAAAAAAHHNFCAA|2077-11-11|2134|9281|712|2077|4|11|11|4|2077|712|9281|Thursday|2077Q4|N|N|N|2479974|2480277|2479619|2479892|N|N|N|N|N| +2479985|AAAAAAAABHHNFCAA|2077-11-12|2134|9281|712|2077|5|11|12|4|2077|712|9281|Friday|2077Q4|N|Y|N|2479974|2480277|2479620|2479893|N|N|N|N|N| +2479986|AAAAAAAACHHNFCAA|2077-11-13|2134|9281|712|2077|6|11|13|4|2077|712|9281|Saturday|2077Q4|N|Y|N|2479974|2480277|2479621|2479894|N|N|N|N|N| +2479987|AAAAAAAADHHNFCAA|2077-11-14|2134|9281|712|2077|0|11|14|4|2077|712|9281|Sunday|2077Q4|N|N|N|2479974|2480277|2479622|2479895|N|N|N|N|N| +2479988|AAAAAAAAEHHNFCAA|2077-11-15|2134|9281|712|2077|1|11|15|4|2077|712|9281|Monday|2077Q4|N|N|N|2479974|2480277|2479623|2479896|N|N|N|N|N| +2479989|AAAAAAAAFHHNFCAA|2077-11-16|2134|9282|712|2077|2|11|16|4|2077|712|9282|Tuesday|2077Q4|N|N|N|2479974|2480277|2479624|2479897|N|N|N|N|N| +2479990|AAAAAAAAGHHNFCAA|2077-11-17|2134|9282|712|2077|3|11|17|4|2077|712|9282|Wednesday|2077Q4|N|N|N|2479974|2480277|2479625|2479898|N|N|N|N|N| +2479991|AAAAAAAAHHHNFCAA|2077-11-18|2134|9282|712|2077|4|11|18|4|2077|712|9282|Thursday|2077Q4|N|N|N|2479974|2480277|2479626|2479899|N|N|N|N|N| +2479992|AAAAAAAAIHHNFCAA|2077-11-19|2134|9282|712|2077|5|11|19|4|2077|712|9282|Friday|2077Q4|N|Y|N|2479974|2480277|2479627|2479900|N|N|N|N|N| +2479993|AAAAAAAAJHHNFCAA|2077-11-20|2134|9282|712|2077|6|11|20|4|2077|712|9282|Saturday|2077Q4|N|Y|N|2479974|2480277|2479628|2479901|N|N|N|N|N| +2479994|AAAAAAAAKHHNFCAA|2077-11-21|2134|9282|712|2077|0|11|21|4|2077|712|9282|Sunday|2077Q4|N|N|N|2479974|2480277|2479629|2479902|N|N|N|N|N| +2479995|AAAAAAAALHHNFCAA|2077-11-22|2134|9282|712|2077|1|11|22|4|2077|712|9282|Monday|2077Q4|N|N|N|2479974|2480277|2479630|2479903|N|N|N|N|N| +2479996|AAAAAAAAMHHNFCAA|2077-11-23|2134|9283|712|2077|2|11|23|4|2077|712|9283|Tuesday|2077Q4|N|N|N|2479974|2480277|2479631|2479904|N|N|N|N|N| +2479997|AAAAAAAANHHNFCAA|2077-11-24|2134|9283|712|2077|3|11|24|4|2077|712|9283|Wednesday|2077Q4|N|N|N|2479974|2480277|2479632|2479905|N|N|N|N|N| +2479998|AAAAAAAAOHHNFCAA|2077-11-25|2134|9283|712|2077|4|11|25|4|2077|712|9283|Thursday|2077Q4|N|N|N|2479974|2480277|2479633|2479906|N|N|N|N|N| +2479999|AAAAAAAAPHHNFCAA|2077-11-26|2134|9283|712|2077|5|11|26|4|2077|712|9283|Friday|2077Q4|N|Y|N|2479974|2480277|2479634|2479907|N|N|N|N|N| +2480000|AAAAAAAAAIHNFCAA|2077-11-27|2134|9283|712|2077|6|11|27|4|2077|712|9283|Saturday|2077Q4|N|Y|N|2479974|2480277|2479635|2479908|N|N|N|N|N| +2480001|AAAAAAAABIHNFCAA|2077-11-28|2134|9283|712|2077|0|11|28|4|2077|712|9283|Sunday|2077Q4|N|N|N|2479974|2480277|2479636|2479909|N|N|N|N|N| +2480002|AAAAAAAACIHNFCAA|2077-11-29|2134|9283|712|2077|1|11|29|4|2077|712|9283|Monday|2077Q4|N|N|N|2479974|2480277|2479637|2479910|N|N|N|N|N| +2480003|AAAAAAAADIHNFCAA|2077-11-30|2134|9284|712|2077|2|11|30|4|2077|712|9284|Tuesday|2077Q4|N|N|N|2479974|2480277|2479638|2479911|N|N|N|N|N| +2480004|AAAAAAAAEIHNFCAA|2077-12-01|2135|9284|713|2077|3|12|1|4|2077|713|9284|Wednesday|2077Q4|N|N|N|2480004|2480337|2479639|2479912|N|N|N|N|N| +2480005|AAAAAAAAFIHNFCAA|2077-12-02|2135|9284|713|2077|4|12|2|4|2077|713|9284|Thursday|2077Q4|N|N|N|2480004|2480337|2479640|2479913|N|N|N|N|N| +2480006|AAAAAAAAGIHNFCAA|2077-12-03|2135|9284|713|2077|5|12|3|4|2077|713|9284|Friday|2077Q4|N|Y|N|2480004|2480337|2479641|2479914|N|N|N|N|N| +2480007|AAAAAAAAHIHNFCAA|2077-12-04|2135|9284|713|2077|6|12|4|4|2077|713|9284|Saturday|2077Q4|N|Y|N|2480004|2480337|2479642|2479915|N|N|N|N|N| +2480008|AAAAAAAAIIHNFCAA|2077-12-05|2135|9284|713|2077|0|12|5|4|2077|713|9284|Sunday|2077Q4|N|N|N|2480004|2480337|2479643|2479916|N|N|N|N|N| +2480009|AAAAAAAAJIHNFCAA|2077-12-06|2135|9284|713|2077|1|12|6|4|2077|713|9284|Monday|2077Q4|N|N|N|2480004|2480337|2479644|2479917|N|N|N|N|N| +2480010|AAAAAAAAKIHNFCAA|2077-12-07|2135|9285|713|2077|2|12|7|4|2077|713|9285|Tuesday|2077Q4|N|N|N|2480004|2480337|2479645|2479918|N|N|N|N|N| +2480011|AAAAAAAALIHNFCAA|2077-12-08|2135|9285|713|2077|3|12|8|4|2077|713|9285|Wednesday|2077Q4|N|N|N|2480004|2480337|2479646|2479919|N|N|N|N|N| +2480012|AAAAAAAAMIHNFCAA|2077-12-09|2135|9285|713|2077|4|12|9|4|2077|713|9285|Thursday|2077Q4|N|N|N|2480004|2480337|2479647|2479920|N|N|N|N|N| +2480013|AAAAAAAANIHNFCAA|2077-12-10|2135|9285|713|2077|5|12|10|4|2077|713|9285|Friday|2077Q4|N|Y|N|2480004|2480337|2479648|2479921|N|N|N|N|N| +2480014|AAAAAAAAOIHNFCAA|2077-12-11|2135|9285|713|2077|6|12|11|4|2077|713|9285|Saturday|2077Q4|N|Y|N|2480004|2480337|2479649|2479922|N|N|N|N|N| +2480015|AAAAAAAAPIHNFCAA|2077-12-12|2135|9285|713|2077|0|12|12|4|2077|713|9285|Sunday|2077Q4|N|N|N|2480004|2480337|2479650|2479923|N|N|N|N|N| +2480016|AAAAAAAAAJHNFCAA|2077-12-13|2135|9285|713|2077|1|12|13|4|2077|713|9285|Monday|2077Q4|N|N|N|2480004|2480337|2479651|2479924|N|N|N|N|N| +2480017|AAAAAAAABJHNFCAA|2077-12-14|2135|9286|713|2077|2|12|14|4|2077|713|9286|Tuesday|2077Q4|N|N|N|2480004|2480337|2479652|2479925|N|N|N|N|N| +2480018|AAAAAAAACJHNFCAA|2077-12-15|2135|9286|713|2077|3|12|15|4|2077|713|9286|Wednesday|2077Q4|N|N|N|2480004|2480337|2479653|2479926|N|N|N|N|N| +2480019|AAAAAAAADJHNFCAA|2077-12-16|2135|9286|713|2077|4|12|16|4|2077|713|9286|Thursday|2077Q4|N|N|N|2480004|2480337|2479654|2479927|N|N|N|N|N| +2480020|AAAAAAAAEJHNFCAA|2077-12-17|2135|9286|713|2077|5|12|17|4|2077|713|9286|Friday|2077Q4|N|Y|N|2480004|2480337|2479655|2479928|N|N|N|N|N| +2480021|AAAAAAAAFJHNFCAA|2077-12-18|2135|9286|713|2077|6|12|18|4|2077|713|9286|Saturday|2077Q4|N|Y|N|2480004|2480337|2479656|2479929|N|N|N|N|N| +2480022|AAAAAAAAGJHNFCAA|2077-12-19|2135|9286|713|2077|0|12|19|4|2077|713|9286|Sunday|2077Q4|N|N|N|2480004|2480337|2479657|2479930|N|N|N|N|N| +2480023|AAAAAAAAHJHNFCAA|2077-12-20|2135|9286|713|2077|1|12|20|4|2077|713|9286|Monday|2077Q4|N|N|N|2480004|2480337|2479658|2479931|N|N|N|N|N| +2480024|AAAAAAAAIJHNFCAA|2077-12-21|2135|9287|713|2077|2|12|21|4|2077|713|9287|Tuesday|2077Q4|N|N|N|2480004|2480337|2479659|2479932|N|N|N|N|N| +2480025|AAAAAAAAJJHNFCAA|2077-12-22|2135|9287|713|2077|3|12|22|4|2077|713|9287|Wednesday|2077Q4|N|N|N|2480004|2480337|2479660|2479933|N|N|N|N|N| +2480026|AAAAAAAAKJHNFCAA|2077-12-23|2135|9287|713|2077|4|12|23|4|2077|713|9287|Thursday|2077Q4|N|N|N|2480004|2480337|2479661|2479934|N|N|N|N|N| +2480027|AAAAAAAALJHNFCAA|2077-12-24|2135|9287|713|2077|5|12|24|4|2077|713|9287|Friday|2077Q4|N|Y|N|2480004|2480337|2479662|2479935|N|N|N|N|N| +2480028|AAAAAAAAMJHNFCAA|2077-12-25|2135|9287|713|2077|6|12|25|4|2077|713|9287|Saturday|2077Q4|N|Y|N|2480004|2480337|2479663|2479936|N|N|N|N|N| +2480029|AAAAAAAANJHNFCAA|2077-12-26|2135|9287|713|2077|0|12|26|4|2077|713|9287|Sunday|2077Q4|Y|N|N|2480004|2480337|2479664|2479937|N|N|N|N|N| +2480030|AAAAAAAAOJHNFCAA|2077-12-27|2135|9287|713|2077|1|12|27|4|2077|713|9287|Monday|2077Q4|N|N|Y|2480004|2480337|2479665|2479938|N|N|N|N|N| +2480031|AAAAAAAAPJHNFCAA|2077-12-28|2135|9288|713|2077|2|12|28|4|2077|713|9288|Tuesday|2077Q4|N|N|N|2480004|2480337|2479666|2479939|N|N|N|N|N| +2480032|AAAAAAAAAKHNFCAA|2077-12-29|2135|9288|713|2077|3|12|29|4|2077|713|9288|Wednesday|2077Q4|N|N|N|2480004|2480337|2479667|2479940|N|N|N|N|N| +2480033|AAAAAAAABKHNFCAA|2077-12-30|2135|9288|713|2077|4|12|30|4|2077|713|9288|Thursday|2077Q4|N|N|N|2480004|2480337|2479668|2479941|N|N|N|N|N| +2480034|AAAAAAAACKHNFCAA|2077-12-31|2135|9288|713|2077|5|12|31|4|2077|713|9288|Friday|2077Q4|N|Y|N|2480004|2480337|2479669|2479942|N|N|N|N|N| +2480035|AAAAAAAADKHNFCAA|2078-01-01|2136|9288|713|2078|6|1|1|1|2078|713|9288|Saturday|2078Q1|Y|Y|N|2480035|2480034|2479670|2479943|N|N|N|N|N| +2480036|AAAAAAAAEKHNFCAA|2078-01-02|2136|9288|713|2078|0|1|2|1|2078|713|9288|Sunday|2078Q1|N|N|Y|2480035|2480034|2479671|2479944|N|N|N|N|N| +2480037|AAAAAAAAFKHNFCAA|2078-01-03|2136|9288|713|2078|1|1|3|1|2078|713|9288|Monday|2078Q1|N|N|N|2480035|2480034|2479672|2479945|N|N|N|N|N| +2480038|AAAAAAAAGKHNFCAA|2078-01-04|2136|9289|713|2078|2|1|4|1|2078|713|9289|Tuesday|2078Q1|N|N|N|2480035|2480034|2479673|2479946|N|N|N|N|N| +2480039|AAAAAAAAHKHNFCAA|2078-01-05|2136|9289|713|2078|3|1|5|1|2078|713|9289|Wednesday|2078Q1|N|N|N|2480035|2480034|2479674|2479947|N|N|N|N|N| +2480040|AAAAAAAAIKHNFCAA|2078-01-06|2136|9289|713|2078|4|1|6|1|2078|713|9289|Thursday|2078Q1|N|N|N|2480035|2480034|2479675|2479948|N|N|N|N|N| +2480041|AAAAAAAAJKHNFCAA|2078-01-07|2136|9289|713|2078|5|1|7|1|2078|713|9289|Friday|2078Q1|N|Y|N|2480035|2480034|2479676|2479949|N|N|N|N|N| +2480042|AAAAAAAAKKHNFCAA|2078-01-08|2136|9289|713|2078|6|1|8|1|2078|713|9289|Saturday|2078Q1|N|Y|N|2480035|2480034|2479677|2479950|N|N|N|N|N| +2480043|AAAAAAAALKHNFCAA|2078-01-09|2136|9289|713|2078|0|1|9|1|2078|713|9289|Sunday|2078Q1|N|N|N|2480035|2480034|2479678|2479951|N|N|N|N|N| +2480044|AAAAAAAAMKHNFCAA|2078-01-10|2136|9289|713|2078|1|1|10|1|2078|713|9289|Monday|2078Q1|N|N|N|2480035|2480034|2479679|2479952|N|N|N|N|N| +2480045|AAAAAAAANKHNFCAA|2078-01-11|2136|9290|713|2078|2|1|11|1|2078|713|9290|Tuesday|2078Q1|N|N|N|2480035|2480034|2479680|2479953|N|N|N|N|N| +2480046|AAAAAAAAOKHNFCAA|2078-01-12|2136|9290|713|2078|3|1|12|1|2078|713|9290|Wednesday|2078Q1|N|N|N|2480035|2480034|2479681|2479954|N|N|N|N|N| +2480047|AAAAAAAAPKHNFCAA|2078-01-13|2136|9290|713|2078|4|1|13|1|2078|713|9290|Thursday|2078Q1|N|N|N|2480035|2480034|2479682|2479955|N|N|N|N|N| +2480048|AAAAAAAAALHNFCAA|2078-01-14|2136|9290|713|2078|5|1|14|1|2078|713|9290|Friday|2078Q1|N|Y|N|2480035|2480034|2479683|2479956|N|N|N|N|N| +2480049|AAAAAAAABLHNFCAA|2078-01-15|2136|9290|713|2078|6|1|15|1|2078|713|9290|Saturday|2078Q1|N|Y|N|2480035|2480034|2479684|2479957|N|N|N|N|N| +2480050|AAAAAAAACLHNFCAA|2078-01-16|2136|9290|713|2078|0|1|16|1|2078|713|9290|Sunday|2078Q1|N|N|N|2480035|2480034|2479685|2479958|N|N|N|N|N| +2480051|AAAAAAAADLHNFCAA|2078-01-17|2136|9290|713|2078|1|1|17|1|2078|713|9290|Monday|2078Q1|N|N|N|2480035|2480034|2479686|2479959|N|N|N|N|N| +2480052|AAAAAAAAELHNFCAA|2078-01-18|2136|9291|713|2078|2|1|18|1|2078|713|9291|Tuesday|2078Q1|N|N|N|2480035|2480034|2479687|2479960|N|N|N|N|N| +2480053|AAAAAAAAFLHNFCAA|2078-01-19|2136|9291|713|2078|3|1|19|1|2078|713|9291|Wednesday|2078Q1|N|N|N|2480035|2480034|2479688|2479961|N|N|N|N|N| +2480054|AAAAAAAAGLHNFCAA|2078-01-20|2136|9291|713|2078|4|1|20|1|2078|713|9291|Thursday|2078Q1|N|N|N|2480035|2480034|2479689|2479962|N|N|N|N|N| +2480055|AAAAAAAAHLHNFCAA|2078-01-21|2136|9291|713|2078|5|1|21|1|2078|713|9291|Friday|2078Q1|N|Y|N|2480035|2480034|2479690|2479963|N|N|N|N|N| +2480056|AAAAAAAAILHNFCAA|2078-01-22|2136|9291|713|2078|6|1|22|1|2078|713|9291|Saturday|2078Q1|N|Y|N|2480035|2480034|2479691|2479964|N|N|N|N|N| +2480057|AAAAAAAAJLHNFCAA|2078-01-23|2136|9291|713|2078|0|1|23|1|2078|713|9291|Sunday|2078Q1|N|N|N|2480035|2480034|2479692|2479965|N|N|N|N|N| +2480058|AAAAAAAAKLHNFCAA|2078-01-24|2136|9291|713|2078|1|1|24|1|2078|713|9291|Monday|2078Q1|N|N|N|2480035|2480034|2479693|2479966|N|N|N|N|N| +2480059|AAAAAAAALLHNFCAA|2078-01-25|2136|9292|713|2078|2|1|25|1|2078|713|9292|Tuesday|2078Q1|N|N|N|2480035|2480034|2479694|2479967|N|N|N|N|N| +2480060|AAAAAAAAMLHNFCAA|2078-01-26|2136|9292|713|2078|3|1|26|1|2078|713|9292|Wednesday|2078Q1|N|N|N|2480035|2480034|2479695|2479968|N|N|N|N|N| +2480061|AAAAAAAANLHNFCAA|2078-01-27|2136|9292|713|2078|4|1|27|1|2078|713|9292|Thursday|2078Q1|N|N|N|2480035|2480034|2479696|2479969|N|N|N|N|N| +2480062|AAAAAAAAOLHNFCAA|2078-01-28|2136|9292|713|2078|5|1|28|1|2078|713|9292|Friday|2078Q1|N|Y|N|2480035|2480034|2479697|2479970|N|N|N|N|N| +2480063|AAAAAAAAPLHNFCAA|2078-01-29|2136|9292|713|2078|6|1|29|1|2078|713|9292|Saturday|2078Q1|N|Y|N|2480035|2480034|2479698|2479971|N|N|N|N|N| +2480064|AAAAAAAAAMHNFCAA|2078-01-30|2136|9292|713|2078|0|1|30|1|2078|713|9292|Sunday|2078Q1|N|N|N|2480035|2480034|2479699|2479972|N|N|N|N|N| +2480065|AAAAAAAABMHNFCAA|2078-01-31|2136|9292|713|2078|1|1|31|1|2078|713|9292|Monday|2078Q1|N|N|N|2480035|2480034|2479700|2479973|N|N|N|N|N| +2480066|AAAAAAAACMHNFCAA|2078-02-01|2137|9293|713|2078|2|2|1|1|2078|713|9293|Tuesday|2078Q1|N|N|N|2480066|2480096|2479701|2479974|N|N|N|N|N| +2480067|AAAAAAAADMHNFCAA|2078-02-02|2137|9293|713|2078|3|2|2|1|2078|713|9293|Wednesday|2078Q1|N|N|N|2480066|2480096|2479702|2479975|N|N|N|N|N| +2480068|AAAAAAAAEMHNFCAA|2078-02-03|2137|9293|713|2078|4|2|3|1|2078|713|9293|Thursday|2078Q1|N|N|N|2480066|2480096|2479703|2479976|N|N|N|N|N| +2480069|AAAAAAAAFMHNFCAA|2078-02-04|2137|9293|713|2078|5|2|4|1|2078|713|9293|Friday|2078Q1|N|Y|N|2480066|2480096|2479704|2479977|N|N|N|N|N| +2480070|AAAAAAAAGMHNFCAA|2078-02-05|2137|9293|713|2078|6|2|5|1|2078|713|9293|Saturday|2078Q1|N|Y|N|2480066|2480096|2479705|2479978|N|N|N|N|N| +2480071|AAAAAAAAHMHNFCAA|2078-02-06|2137|9293|713|2078|0|2|6|1|2078|713|9293|Sunday|2078Q1|N|N|N|2480066|2480096|2479706|2479979|N|N|N|N|N| +2480072|AAAAAAAAIMHNFCAA|2078-02-07|2137|9293|713|2078|1|2|7|1|2078|713|9293|Monday|2078Q1|N|N|N|2480066|2480096|2479707|2479980|N|N|N|N|N| +2480073|AAAAAAAAJMHNFCAA|2078-02-08|2137|9294|713|2078|2|2|8|1|2078|713|9294|Tuesday|2078Q1|N|N|N|2480066|2480096|2479708|2479981|N|N|N|N|N| +2480074|AAAAAAAAKMHNFCAA|2078-02-09|2137|9294|713|2078|3|2|9|1|2078|713|9294|Wednesday|2078Q1|N|N|N|2480066|2480096|2479709|2479982|N|N|N|N|N| +2480075|AAAAAAAALMHNFCAA|2078-02-10|2137|9294|713|2078|4|2|10|1|2078|713|9294|Thursday|2078Q1|N|N|N|2480066|2480096|2479710|2479983|N|N|N|N|N| +2480076|AAAAAAAAMMHNFCAA|2078-02-11|2137|9294|713|2078|5|2|11|1|2078|713|9294|Friday|2078Q1|N|Y|N|2480066|2480096|2479711|2479984|N|N|N|N|N| +2480077|AAAAAAAANMHNFCAA|2078-02-12|2137|9294|713|2078|6|2|12|1|2078|713|9294|Saturday|2078Q1|N|Y|N|2480066|2480096|2479712|2479985|N|N|N|N|N| +2480078|AAAAAAAAOMHNFCAA|2078-02-13|2137|9294|713|2078|0|2|13|1|2078|713|9294|Sunday|2078Q1|N|N|N|2480066|2480096|2479713|2479986|N|N|N|N|N| +2480079|AAAAAAAAPMHNFCAA|2078-02-14|2137|9294|713|2078|1|2|14|1|2078|713|9294|Monday|2078Q1|N|N|N|2480066|2480096|2479714|2479987|N|N|N|N|N| +2480080|AAAAAAAAANHNFCAA|2078-02-15|2137|9295|713|2078|2|2|15|1|2078|713|9295|Tuesday|2078Q1|N|N|N|2480066|2480096|2479715|2479988|N|N|N|N|N| +2480081|AAAAAAAABNHNFCAA|2078-02-16|2137|9295|713|2078|3|2|16|1|2078|713|9295|Wednesday|2078Q1|N|N|N|2480066|2480096|2479716|2479989|N|N|N|N|N| +2480082|AAAAAAAACNHNFCAA|2078-02-17|2137|9295|713|2078|4|2|17|1|2078|713|9295|Thursday|2078Q1|N|N|N|2480066|2480096|2479717|2479990|N|N|N|N|N| +2480083|AAAAAAAADNHNFCAA|2078-02-18|2137|9295|713|2078|5|2|18|1|2078|713|9295|Friday|2078Q1|N|Y|N|2480066|2480096|2479718|2479991|N|N|N|N|N| +2480084|AAAAAAAAENHNFCAA|2078-02-19|2137|9295|713|2078|6|2|19|1|2078|713|9295|Saturday|2078Q1|N|Y|N|2480066|2480096|2479719|2479992|N|N|N|N|N| +2480085|AAAAAAAAFNHNFCAA|2078-02-20|2137|9295|713|2078|0|2|20|1|2078|713|9295|Sunday|2078Q1|N|N|N|2480066|2480096|2479720|2479993|N|N|N|N|N| +2480086|AAAAAAAAGNHNFCAA|2078-02-21|2137|9295|713|2078|1|2|21|1|2078|713|9295|Monday|2078Q1|N|N|N|2480066|2480096|2479721|2479994|N|N|N|N|N| +2480087|AAAAAAAAHNHNFCAA|2078-02-22|2137|9296|713|2078|2|2|22|1|2078|713|9296|Tuesday|2078Q1|N|N|N|2480066|2480096|2479722|2479995|N|N|N|N|N| +2480088|AAAAAAAAINHNFCAA|2078-02-23|2137|9296|713|2078|3|2|23|1|2078|713|9296|Wednesday|2078Q1|N|N|N|2480066|2480096|2479723|2479996|N|N|N|N|N| +2480089|AAAAAAAAJNHNFCAA|2078-02-24|2137|9296|713|2078|4|2|24|1|2078|713|9296|Thursday|2078Q1|N|N|N|2480066|2480096|2479724|2479997|N|N|N|N|N| +2480090|AAAAAAAAKNHNFCAA|2078-02-25|2137|9296|713|2078|5|2|25|1|2078|713|9296|Friday|2078Q1|N|Y|N|2480066|2480096|2479725|2479998|N|N|N|N|N| +2480091|AAAAAAAALNHNFCAA|2078-02-26|2137|9296|713|2078|6|2|26|1|2078|713|9296|Saturday|2078Q1|N|Y|N|2480066|2480096|2479726|2479999|N|N|N|N|N| +2480092|AAAAAAAAMNHNFCAA|2078-02-27|2137|9296|713|2078|0|2|27|1|2078|713|9296|Sunday|2078Q1|N|N|N|2480066|2480096|2479727|2480000|N|N|N|N|N| +2480093|AAAAAAAANNHNFCAA|2078-02-28|2137|9296|713|2078|1|2|28|1|2078|713|9296|Monday|2078Q1|N|N|N|2480066|2480096|2479728|2480001|N|N|N|N|N| +2480094|AAAAAAAAONHNFCAA|2078-03-01|2138|9297|714|2078|2|3|1|1|2078|714|9297|Tuesday|2078Q1|N|N|N|2480094|2480152|2479729|2480002|N|N|N|N|N| +2480095|AAAAAAAAPNHNFCAA|2078-03-02|2138|9297|714|2078|3|3|2|1|2078|714|9297|Wednesday|2078Q1|N|N|N|2480094|2480152|2479730|2480003|N|N|N|N|N| +2480096|AAAAAAAAAOHNFCAA|2078-03-03|2138|9297|714|2078|4|3|3|1|2078|714|9297|Thursday|2078Q1|N|N|N|2480094|2480152|2479731|2480004|N|N|N|N|N| +2480097|AAAAAAAABOHNFCAA|2078-03-04|2138|9297|714|2078|5|3|4|1|2078|714|9297|Friday|2078Q1|N|Y|N|2480094|2480152|2479732|2480005|N|N|N|N|N| +2480098|AAAAAAAACOHNFCAA|2078-03-05|2138|9297|714|2078|6|3|5|1|2078|714|9297|Saturday|2078Q1|N|Y|N|2480094|2480152|2479733|2480006|N|N|N|N|N| +2480099|AAAAAAAADOHNFCAA|2078-03-06|2138|9297|714|2078|0|3|6|1|2078|714|9297|Sunday|2078Q1|N|N|N|2480094|2480152|2479734|2480007|N|N|N|N|N| +2480100|AAAAAAAAEOHNFCAA|2078-03-07|2138|9297|714|2078|1|3|7|1|2078|714|9297|Monday|2078Q1|N|N|N|2480094|2480152|2479735|2480008|N|N|N|N|N| +2480101|AAAAAAAAFOHNFCAA|2078-03-08|2138|9298|714|2078|2|3|8|1|2078|714|9298|Tuesday|2078Q1|N|N|N|2480094|2480152|2479736|2480009|N|N|N|N|N| +2480102|AAAAAAAAGOHNFCAA|2078-03-09|2138|9298|714|2078|3|3|9|1|2078|714|9298|Wednesday|2078Q1|N|N|N|2480094|2480152|2479737|2480010|N|N|N|N|N| +2480103|AAAAAAAAHOHNFCAA|2078-03-10|2138|9298|714|2078|4|3|10|1|2078|714|9298|Thursday|2078Q1|N|N|N|2480094|2480152|2479738|2480011|N|N|N|N|N| +2480104|AAAAAAAAIOHNFCAA|2078-03-11|2138|9298|714|2078|5|3|11|1|2078|714|9298|Friday|2078Q1|N|Y|N|2480094|2480152|2479739|2480012|N|N|N|N|N| +2480105|AAAAAAAAJOHNFCAA|2078-03-12|2138|9298|714|2078|6|3|12|1|2078|714|9298|Saturday|2078Q1|N|Y|N|2480094|2480152|2479740|2480013|N|N|N|N|N| +2480106|AAAAAAAAKOHNFCAA|2078-03-13|2138|9298|714|2078|0|3|13|1|2078|714|9298|Sunday|2078Q1|N|N|N|2480094|2480152|2479741|2480014|N|N|N|N|N| +2480107|AAAAAAAALOHNFCAA|2078-03-14|2138|9298|714|2078|1|3|14|1|2078|714|9298|Monday|2078Q1|N|N|N|2480094|2480152|2479742|2480015|N|N|N|N|N| +2480108|AAAAAAAAMOHNFCAA|2078-03-15|2138|9299|714|2078|2|3|15|1|2078|714|9299|Tuesday|2078Q1|N|N|N|2480094|2480152|2479743|2480016|N|N|N|N|N| +2480109|AAAAAAAANOHNFCAA|2078-03-16|2138|9299|714|2078|3|3|16|1|2078|714|9299|Wednesday|2078Q1|N|N|N|2480094|2480152|2479744|2480017|N|N|N|N|N| +2480110|AAAAAAAAOOHNFCAA|2078-03-17|2138|9299|714|2078|4|3|17|1|2078|714|9299|Thursday|2078Q1|N|N|N|2480094|2480152|2479745|2480018|N|N|N|N|N| +2480111|AAAAAAAAPOHNFCAA|2078-03-18|2138|9299|714|2078|5|3|18|1|2078|714|9299|Friday|2078Q1|N|Y|N|2480094|2480152|2479746|2480019|N|N|N|N|N| +2480112|AAAAAAAAAPHNFCAA|2078-03-19|2138|9299|714|2078|6|3|19|1|2078|714|9299|Saturday|2078Q1|N|Y|N|2480094|2480152|2479747|2480020|N|N|N|N|N| +2480113|AAAAAAAABPHNFCAA|2078-03-20|2138|9299|714|2078|0|3|20|1|2078|714|9299|Sunday|2078Q1|N|N|N|2480094|2480152|2479748|2480021|N|N|N|N|N| +2480114|AAAAAAAACPHNFCAA|2078-03-21|2138|9299|714|2078|1|3|21|1|2078|714|9299|Monday|2078Q1|N|N|N|2480094|2480152|2479749|2480022|N|N|N|N|N| +2480115|AAAAAAAADPHNFCAA|2078-03-22|2138|9300|714|2078|2|3|22|1|2078|714|9300|Tuesday|2078Q1|N|N|N|2480094|2480152|2479750|2480023|N|N|N|N|N| +2480116|AAAAAAAAEPHNFCAA|2078-03-23|2138|9300|714|2078|3|3|23|1|2078|714|9300|Wednesday|2078Q1|N|N|N|2480094|2480152|2479751|2480024|N|N|N|N|N| +2480117|AAAAAAAAFPHNFCAA|2078-03-24|2138|9300|714|2078|4|3|24|1|2078|714|9300|Thursday|2078Q1|N|N|N|2480094|2480152|2479752|2480025|N|N|N|N|N| +2480118|AAAAAAAAGPHNFCAA|2078-03-25|2138|9300|714|2078|5|3|25|1|2078|714|9300|Friday|2078Q1|N|Y|N|2480094|2480152|2479753|2480026|N|N|N|N|N| +2480119|AAAAAAAAHPHNFCAA|2078-03-26|2138|9300|714|2078|6|3|26|1|2078|714|9300|Saturday|2078Q1|N|Y|N|2480094|2480152|2479754|2480027|N|N|N|N|N| +2480120|AAAAAAAAIPHNFCAA|2078-03-27|2138|9300|714|2078|0|3|27|1|2078|714|9300|Sunday|2078Q1|N|N|N|2480094|2480152|2479755|2480028|N|N|N|N|N| +2480121|AAAAAAAAJPHNFCAA|2078-03-28|2138|9300|714|2078|1|3|28|1|2078|714|9300|Monday|2078Q1|N|N|N|2480094|2480152|2479756|2480029|N|N|N|N|N| +2480122|AAAAAAAAKPHNFCAA|2078-03-29|2138|9301|714|2078|2|3|29|1|2078|714|9301|Tuesday|2078Q1|N|N|N|2480094|2480152|2479757|2480030|N|N|N|N|N| +2480123|AAAAAAAALPHNFCAA|2078-03-30|2138|9301|714|2078|3|3|30|1|2078|714|9301|Wednesday|2078Q1|N|N|N|2480094|2480152|2479758|2480031|N|N|N|N|N| +2480124|AAAAAAAAMPHNFCAA|2078-03-31|2138|9301|714|2078|4|3|31|1|2078|714|9301|Thursday|2078Q1|N|N|N|2480094|2480152|2479759|2480032|N|N|N|N|N| +2480125|AAAAAAAANPHNFCAA|2078-04-01|2139|9301|714|2078|5|4|1|1|2078|714|9301|Friday|2078Q1|N|Y|N|2480125|2480214|2479760|2480035|N|N|N|N|N| +2480126|AAAAAAAAOPHNFCAA|2078-04-02|2139|9301|714|2078|6|4|2|2|2078|714|9301|Saturday|2078Q2|N|Y|N|2480125|2480214|2479761|2480036|N|N|N|N|N| +2480127|AAAAAAAAPPHNFCAA|2078-04-03|2139|9301|714|2078|0|4|3|2|2078|714|9301|Sunday|2078Q2|N|N|N|2480125|2480214|2479762|2480037|N|N|N|N|N| +2480128|AAAAAAAAAAINFCAA|2078-04-04|2139|9301|714|2078|1|4|4|2|2078|714|9301|Monday|2078Q2|N|N|N|2480125|2480214|2479763|2480038|N|N|N|N|N| +2480129|AAAAAAAABAINFCAA|2078-04-05|2139|9302|714|2078|2|4|5|2|2078|714|9302|Tuesday|2078Q2|N|N|N|2480125|2480214|2479764|2480039|N|N|N|N|N| +2480130|AAAAAAAACAINFCAA|2078-04-06|2139|9302|714|2078|3|4|6|2|2078|714|9302|Wednesday|2078Q2|N|N|N|2480125|2480214|2479765|2480040|N|N|N|N|N| +2480131|AAAAAAAADAINFCAA|2078-04-07|2139|9302|714|2078|4|4|7|2|2078|714|9302|Thursday|2078Q2|N|N|N|2480125|2480214|2479766|2480041|N|N|N|N|N| +2480132|AAAAAAAAEAINFCAA|2078-04-08|2139|9302|714|2078|5|4|8|2|2078|714|9302|Friday|2078Q2|N|Y|N|2480125|2480214|2479767|2480042|N|N|N|N|N| +2480133|AAAAAAAAFAINFCAA|2078-04-09|2139|9302|714|2078|6|4|9|2|2078|714|9302|Saturday|2078Q2|N|Y|N|2480125|2480214|2479768|2480043|N|N|N|N|N| +2480134|AAAAAAAAGAINFCAA|2078-04-10|2139|9302|714|2078|0|4|10|2|2078|714|9302|Sunday|2078Q2|N|N|N|2480125|2480214|2479769|2480044|N|N|N|N|N| +2480135|AAAAAAAAHAINFCAA|2078-04-11|2139|9302|714|2078|1|4|11|2|2078|714|9302|Monday|2078Q2|N|N|N|2480125|2480214|2479770|2480045|N|N|N|N|N| +2480136|AAAAAAAAIAINFCAA|2078-04-12|2139|9303|714|2078|2|4|12|2|2078|714|9303|Tuesday|2078Q2|N|N|N|2480125|2480214|2479771|2480046|N|N|N|N|N| +2480137|AAAAAAAAJAINFCAA|2078-04-13|2139|9303|714|2078|3|4|13|2|2078|714|9303|Wednesday|2078Q2|N|N|N|2480125|2480214|2479772|2480047|N|N|N|N|N| +2480138|AAAAAAAAKAINFCAA|2078-04-14|2139|9303|714|2078|4|4|14|2|2078|714|9303|Thursday|2078Q2|N|N|N|2480125|2480214|2479773|2480048|N|N|N|N|N| +2480139|AAAAAAAALAINFCAA|2078-04-15|2139|9303|714|2078|5|4|15|2|2078|714|9303|Friday|2078Q2|N|Y|N|2480125|2480214|2479774|2480049|N|N|N|N|N| +2480140|AAAAAAAAMAINFCAA|2078-04-16|2139|9303|714|2078|6|4|16|2|2078|714|9303|Saturday|2078Q2|N|Y|N|2480125|2480214|2479775|2480050|N|N|N|N|N| +2480141|AAAAAAAANAINFCAA|2078-04-17|2139|9303|714|2078|0|4|17|2|2078|714|9303|Sunday|2078Q2|N|N|N|2480125|2480214|2479776|2480051|N|N|N|N|N| +2480142|AAAAAAAAOAINFCAA|2078-04-18|2139|9303|714|2078|1|4|18|2|2078|714|9303|Monday|2078Q2|N|N|N|2480125|2480214|2479777|2480052|N|N|N|N|N| +2480143|AAAAAAAAPAINFCAA|2078-04-19|2139|9304|714|2078|2|4|19|2|2078|714|9304|Tuesday|2078Q2|N|N|N|2480125|2480214|2479778|2480053|N|N|N|N|N| +2480144|AAAAAAAAABINFCAA|2078-04-20|2139|9304|714|2078|3|4|20|2|2078|714|9304|Wednesday|2078Q2|N|N|N|2480125|2480214|2479779|2480054|N|N|N|N|N| +2480145|AAAAAAAABBINFCAA|2078-04-21|2139|9304|714|2078|4|4|21|2|2078|714|9304|Thursday|2078Q2|N|N|N|2480125|2480214|2479780|2480055|N|N|N|N|N| +2480146|AAAAAAAACBINFCAA|2078-04-22|2139|9304|714|2078|5|4|22|2|2078|714|9304|Friday|2078Q2|N|Y|N|2480125|2480214|2479781|2480056|N|N|N|N|N| +2480147|AAAAAAAADBINFCAA|2078-04-23|2139|9304|714|2078|6|4|23|2|2078|714|9304|Saturday|2078Q2|N|Y|N|2480125|2480214|2479782|2480057|N|N|N|N|N| +2480148|AAAAAAAAEBINFCAA|2078-04-24|2139|9304|714|2078|0|4|24|2|2078|714|9304|Sunday|2078Q2|N|N|N|2480125|2480214|2479783|2480058|N|N|N|N|N| +2480149|AAAAAAAAFBINFCAA|2078-04-25|2139|9304|714|2078|1|4|25|2|2078|714|9304|Monday|2078Q2|N|N|N|2480125|2480214|2479784|2480059|N|N|N|N|N| +2480150|AAAAAAAAGBINFCAA|2078-04-26|2139|9305|714|2078|2|4|26|2|2078|714|9305|Tuesday|2078Q2|N|N|N|2480125|2480214|2479785|2480060|N|N|N|N|N| +2480151|AAAAAAAAHBINFCAA|2078-04-27|2139|9305|714|2078|3|4|27|2|2078|714|9305|Wednesday|2078Q2|N|N|N|2480125|2480214|2479786|2480061|N|N|N|N|N| +2480152|AAAAAAAAIBINFCAA|2078-04-28|2139|9305|714|2078|4|4|28|2|2078|714|9305|Thursday|2078Q2|N|N|N|2480125|2480214|2479787|2480062|N|N|N|N|N| +2480153|AAAAAAAAJBINFCAA|2078-04-29|2139|9305|714|2078|5|4|29|2|2078|714|9305|Friday|2078Q2|N|Y|N|2480125|2480214|2479788|2480063|N|N|N|N|N| +2480154|AAAAAAAAKBINFCAA|2078-04-30|2139|9305|714|2078|6|4|30|2|2078|714|9305|Saturday|2078Q2|N|Y|N|2480125|2480214|2479789|2480064|N|N|N|N|N| +2480155|AAAAAAAALBINFCAA|2078-05-01|2140|9305|714|2078|0|5|1|2|2078|714|9305|Sunday|2078Q2|N|N|N|2480155|2480274|2479790|2480065|N|N|N|N|N| +2480156|AAAAAAAAMBINFCAA|2078-05-02|2140|9305|714|2078|1|5|2|2|2078|714|9305|Monday|2078Q2|N|N|N|2480155|2480274|2479791|2480066|N|N|N|N|N| +2480157|AAAAAAAANBINFCAA|2078-05-03|2140|9306|714|2078|2|5|3|2|2078|714|9306|Tuesday|2078Q2|N|N|N|2480155|2480274|2479792|2480067|N|N|N|N|N| +2480158|AAAAAAAAOBINFCAA|2078-05-04|2140|9306|714|2078|3|5|4|2|2078|714|9306|Wednesday|2078Q2|N|N|N|2480155|2480274|2479793|2480068|N|N|N|N|N| +2480159|AAAAAAAAPBINFCAA|2078-05-05|2140|9306|714|2078|4|5|5|2|2078|714|9306|Thursday|2078Q2|N|N|N|2480155|2480274|2479794|2480069|N|N|N|N|N| +2480160|AAAAAAAAACINFCAA|2078-05-06|2140|9306|714|2078|5|5|6|2|2078|714|9306|Friday|2078Q2|N|Y|N|2480155|2480274|2479795|2480070|N|N|N|N|N| +2480161|AAAAAAAABCINFCAA|2078-05-07|2140|9306|714|2078|6|5|7|2|2078|714|9306|Saturday|2078Q2|N|Y|N|2480155|2480274|2479796|2480071|N|N|N|N|N| +2480162|AAAAAAAACCINFCAA|2078-05-08|2140|9306|714|2078|0|5|8|2|2078|714|9306|Sunday|2078Q2|N|N|N|2480155|2480274|2479797|2480072|N|N|N|N|N| +2480163|AAAAAAAADCINFCAA|2078-05-09|2140|9306|714|2078|1|5|9|2|2078|714|9306|Monday|2078Q2|N|N|N|2480155|2480274|2479798|2480073|N|N|N|N|N| +2480164|AAAAAAAAECINFCAA|2078-05-10|2140|9307|714|2078|2|5|10|2|2078|714|9307|Tuesday|2078Q2|N|N|N|2480155|2480274|2479799|2480074|N|N|N|N|N| +2480165|AAAAAAAAFCINFCAA|2078-05-11|2140|9307|714|2078|3|5|11|2|2078|714|9307|Wednesday|2078Q2|N|N|N|2480155|2480274|2479800|2480075|N|N|N|N|N| +2480166|AAAAAAAAGCINFCAA|2078-05-12|2140|9307|714|2078|4|5|12|2|2078|714|9307|Thursday|2078Q2|N|N|N|2480155|2480274|2479801|2480076|N|N|N|N|N| +2480167|AAAAAAAAHCINFCAA|2078-05-13|2140|9307|714|2078|5|5|13|2|2078|714|9307|Friday|2078Q2|N|Y|N|2480155|2480274|2479802|2480077|N|N|N|N|N| +2480168|AAAAAAAAICINFCAA|2078-05-14|2140|9307|714|2078|6|5|14|2|2078|714|9307|Saturday|2078Q2|N|Y|N|2480155|2480274|2479803|2480078|N|N|N|N|N| +2480169|AAAAAAAAJCINFCAA|2078-05-15|2140|9307|714|2078|0|5|15|2|2078|714|9307|Sunday|2078Q2|N|N|N|2480155|2480274|2479804|2480079|N|N|N|N|N| +2480170|AAAAAAAAKCINFCAA|2078-05-16|2140|9307|714|2078|1|5|16|2|2078|714|9307|Monday|2078Q2|N|N|N|2480155|2480274|2479805|2480080|N|N|N|N|N| +2480171|AAAAAAAALCINFCAA|2078-05-17|2140|9308|714|2078|2|5|17|2|2078|714|9308|Tuesday|2078Q2|N|N|N|2480155|2480274|2479806|2480081|N|N|N|N|N| +2480172|AAAAAAAAMCINFCAA|2078-05-18|2140|9308|714|2078|3|5|18|2|2078|714|9308|Wednesday|2078Q2|N|N|N|2480155|2480274|2479807|2480082|N|N|N|N|N| +2480173|AAAAAAAANCINFCAA|2078-05-19|2140|9308|714|2078|4|5|19|2|2078|714|9308|Thursday|2078Q2|N|N|N|2480155|2480274|2479808|2480083|N|N|N|N|N| +2480174|AAAAAAAAOCINFCAA|2078-05-20|2140|9308|714|2078|5|5|20|2|2078|714|9308|Friday|2078Q2|N|Y|N|2480155|2480274|2479809|2480084|N|N|N|N|N| +2480175|AAAAAAAAPCINFCAA|2078-05-21|2140|9308|714|2078|6|5|21|2|2078|714|9308|Saturday|2078Q2|N|Y|N|2480155|2480274|2479810|2480085|N|N|N|N|N| +2480176|AAAAAAAAADINFCAA|2078-05-22|2140|9308|714|2078|0|5|22|2|2078|714|9308|Sunday|2078Q2|N|N|N|2480155|2480274|2479811|2480086|N|N|N|N|N| +2480177|AAAAAAAABDINFCAA|2078-05-23|2140|9308|714|2078|1|5|23|2|2078|714|9308|Monday|2078Q2|N|N|N|2480155|2480274|2479812|2480087|N|N|N|N|N| +2480178|AAAAAAAACDINFCAA|2078-05-24|2140|9309|714|2078|2|5|24|2|2078|714|9309|Tuesday|2078Q2|N|N|N|2480155|2480274|2479813|2480088|N|N|N|N|N| +2480179|AAAAAAAADDINFCAA|2078-05-25|2140|9309|714|2078|3|5|25|2|2078|714|9309|Wednesday|2078Q2|N|N|N|2480155|2480274|2479814|2480089|N|N|N|N|N| +2480180|AAAAAAAAEDINFCAA|2078-05-26|2140|9309|714|2078|4|5|26|2|2078|714|9309|Thursday|2078Q2|N|N|N|2480155|2480274|2479815|2480090|N|N|N|N|N| +2480181|AAAAAAAAFDINFCAA|2078-05-27|2140|9309|714|2078|5|5|27|2|2078|714|9309|Friday|2078Q2|N|Y|N|2480155|2480274|2479816|2480091|N|N|N|N|N| +2480182|AAAAAAAAGDINFCAA|2078-05-28|2140|9309|714|2078|6|5|28|2|2078|714|9309|Saturday|2078Q2|N|Y|N|2480155|2480274|2479817|2480092|N|N|N|N|N| +2480183|AAAAAAAAHDINFCAA|2078-05-29|2140|9309|714|2078|0|5|29|2|2078|714|9309|Sunday|2078Q2|N|N|N|2480155|2480274|2479818|2480093|N|N|N|N|N| +2480184|AAAAAAAAIDINFCAA|2078-05-30|2140|9309|714|2078|1|5|30|2|2078|714|9309|Monday|2078Q2|N|N|N|2480155|2480274|2479819|2480094|N|N|N|N|N| +2480185|AAAAAAAAJDINFCAA|2078-05-31|2140|9310|714|2078|2|5|31|2|2078|714|9310|Tuesday|2078Q2|N|N|N|2480155|2480274|2479820|2480095|N|N|N|N|N| +2480186|AAAAAAAAKDINFCAA|2078-06-01|2141|9310|715|2078|3|6|1|2|2078|715|9310|Wednesday|2078Q2|N|N|N|2480186|2480336|2479821|2480096|N|N|N|N|N| +2480187|AAAAAAAALDINFCAA|2078-06-02|2141|9310|715|2078|4|6|2|2|2078|715|9310|Thursday|2078Q2|N|N|N|2480186|2480336|2479822|2480097|N|N|N|N|N| +2480188|AAAAAAAAMDINFCAA|2078-06-03|2141|9310|715|2078|5|6|3|2|2078|715|9310|Friday|2078Q2|N|Y|N|2480186|2480336|2479823|2480098|N|N|N|N|N| +2480189|AAAAAAAANDINFCAA|2078-06-04|2141|9310|715|2078|6|6|4|2|2078|715|9310|Saturday|2078Q2|N|Y|N|2480186|2480336|2479824|2480099|N|N|N|N|N| +2480190|AAAAAAAAODINFCAA|2078-06-05|2141|9310|715|2078|0|6|5|2|2078|715|9310|Sunday|2078Q2|N|N|N|2480186|2480336|2479825|2480100|N|N|N|N|N| +2480191|AAAAAAAAPDINFCAA|2078-06-06|2141|9310|715|2078|1|6|6|2|2078|715|9310|Monday|2078Q2|N|N|N|2480186|2480336|2479826|2480101|N|N|N|N|N| +2480192|AAAAAAAAAEINFCAA|2078-06-07|2141|9311|715|2078|2|6|7|2|2078|715|9311|Tuesday|2078Q2|N|N|N|2480186|2480336|2479827|2480102|N|N|N|N|N| +2480193|AAAAAAAABEINFCAA|2078-06-08|2141|9311|715|2078|3|6|8|2|2078|715|9311|Wednesday|2078Q2|N|N|N|2480186|2480336|2479828|2480103|N|N|N|N|N| +2480194|AAAAAAAACEINFCAA|2078-06-09|2141|9311|715|2078|4|6|9|2|2078|715|9311|Thursday|2078Q2|N|N|N|2480186|2480336|2479829|2480104|N|N|N|N|N| +2480195|AAAAAAAADEINFCAA|2078-06-10|2141|9311|715|2078|5|6|10|2|2078|715|9311|Friday|2078Q2|N|Y|N|2480186|2480336|2479830|2480105|N|N|N|N|N| +2480196|AAAAAAAAEEINFCAA|2078-06-11|2141|9311|715|2078|6|6|11|2|2078|715|9311|Saturday|2078Q2|N|Y|N|2480186|2480336|2479831|2480106|N|N|N|N|N| +2480197|AAAAAAAAFEINFCAA|2078-06-12|2141|9311|715|2078|0|6|12|2|2078|715|9311|Sunday|2078Q2|N|N|N|2480186|2480336|2479832|2480107|N|N|N|N|N| +2480198|AAAAAAAAGEINFCAA|2078-06-13|2141|9311|715|2078|1|6|13|2|2078|715|9311|Monday|2078Q2|N|N|N|2480186|2480336|2479833|2480108|N|N|N|N|N| +2480199|AAAAAAAAHEINFCAA|2078-06-14|2141|9312|715|2078|2|6|14|2|2078|715|9312|Tuesday|2078Q2|N|N|N|2480186|2480336|2479834|2480109|N|N|N|N|N| +2480200|AAAAAAAAIEINFCAA|2078-06-15|2141|9312|715|2078|3|6|15|2|2078|715|9312|Wednesday|2078Q2|N|N|N|2480186|2480336|2479835|2480110|N|N|N|N|N| +2480201|AAAAAAAAJEINFCAA|2078-06-16|2141|9312|715|2078|4|6|16|2|2078|715|9312|Thursday|2078Q2|N|N|N|2480186|2480336|2479836|2480111|N|N|N|N|N| +2480202|AAAAAAAAKEINFCAA|2078-06-17|2141|9312|715|2078|5|6|17|2|2078|715|9312|Friday|2078Q2|N|Y|N|2480186|2480336|2479837|2480112|N|N|N|N|N| +2480203|AAAAAAAALEINFCAA|2078-06-18|2141|9312|715|2078|6|6|18|2|2078|715|9312|Saturday|2078Q2|N|Y|N|2480186|2480336|2479838|2480113|N|N|N|N|N| +2480204|AAAAAAAAMEINFCAA|2078-06-19|2141|9312|715|2078|0|6|19|2|2078|715|9312|Sunday|2078Q2|N|N|N|2480186|2480336|2479839|2480114|N|N|N|N|N| +2480205|AAAAAAAANEINFCAA|2078-06-20|2141|9312|715|2078|1|6|20|2|2078|715|9312|Monday|2078Q2|N|N|N|2480186|2480336|2479840|2480115|N|N|N|N|N| +2480206|AAAAAAAAOEINFCAA|2078-06-21|2141|9313|715|2078|2|6|21|2|2078|715|9313|Tuesday|2078Q2|N|N|N|2480186|2480336|2479841|2480116|N|N|N|N|N| +2480207|AAAAAAAAPEINFCAA|2078-06-22|2141|9313|715|2078|3|6|22|2|2078|715|9313|Wednesday|2078Q2|N|N|N|2480186|2480336|2479842|2480117|N|N|N|N|N| +2480208|AAAAAAAAAFINFCAA|2078-06-23|2141|9313|715|2078|4|6|23|2|2078|715|9313|Thursday|2078Q2|N|N|N|2480186|2480336|2479843|2480118|N|N|N|N|N| +2480209|AAAAAAAABFINFCAA|2078-06-24|2141|9313|715|2078|5|6|24|2|2078|715|9313|Friday|2078Q2|N|Y|N|2480186|2480336|2479844|2480119|N|N|N|N|N| +2480210|AAAAAAAACFINFCAA|2078-06-25|2141|9313|715|2078|6|6|25|2|2078|715|9313|Saturday|2078Q2|N|Y|N|2480186|2480336|2479845|2480120|N|N|N|N|N| +2480211|AAAAAAAADFINFCAA|2078-06-26|2141|9313|715|2078|0|6|26|2|2078|715|9313|Sunday|2078Q2|N|N|N|2480186|2480336|2479846|2480121|N|N|N|N|N| +2480212|AAAAAAAAEFINFCAA|2078-06-27|2141|9313|715|2078|1|6|27|2|2078|715|9313|Monday|2078Q2|N|N|N|2480186|2480336|2479847|2480122|N|N|N|N|N| +2480213|AAAAAAAAFFINFCAA|2078-06-28|2141|9314|715|2078|2|6|28|2|2078|715|9314|Tuesday|2078Q2|N|N|N|2480186|2480336|2479848|2480123|N|N|N|N|N| +2480214|AAAAAAAAGFINFCAA|2078-06-29|2141|9314|715|2078|3|6|29|2|2078|715|9314|Wednesday|2078Q2|N|N|N|2480186|2480336|2479849|2480124|N|N|N|N|N| +2480215|AAAAAAAAHFINFCAA|2078-06-30|2141|9314|715|2078|4|6|30|2|2078|715|9314|Thursday|2078Q2|N|N|N|2480186|2480336|2479850|2480125|N|N|N|N|N| +2480216|AAAAAAAAIFINFCAA|2078-07-01|2142|9314|715|2078|5|7|1|2|2078|715|9314|Friday|2078Q2|N|Y|N|2480216|2480396|2479851|2480125|N|N|N|N|N| +2480217|AAAAAAAAJFINFCAA|2078-07-02|2142|9314|715|2078|6|7|2|3|2078|715|9314|Saturday|2078Q3|N|Y|N|2480216|2480396|2479852|2480126|N|N|N|N|N| +2480218|AAAAAAAAKFINFCAA|2078-07-03|2142|9314|715|2078|0|7|3|3|2078|715|9314|Sunday|2078Q3|N|N|N|2480216|2480396|2479853|2480127|N|N|N|N|N| +2480219|AAAAAAAALFINFCAA|2078-07-04|2142|9314|715|2078|1|7|4|3|2078|715|9314|Monday|2078Q3|N|N|N|2480216|2480396|2479854|2480128|N|N|N|N|N| +2480220|AAAAAAAAMFINFCAA|2078-07-05|2142|9315|715|2078|2|7|5|3|2078|715|9315|Tuesday|2078Q3|Y|N|N|2480216|2480396|2479855|2480129|N|N|N|N|N| +2480221|AAAAAAAANFINFCAA|2078-07-06|2142|9315|715|2078|3|7|6|3|2078|715|9315|Wednesday|2078Q3|N|N|Y|2480216|2480396|2479856|2480130|N|N|N|N|N| +2480222|AAAAAAAAOFINFCAA|2078-07-07|2142|9315|715|2078|4|7|7|3|2078|715|9315|Thursday|2078Q3|N|N|N|2480216|2480396|2479857|2480131|N|N|N|N|N| +2480223|AAAAAAAAPFINFCAA|2078-07-08|2142|9315|715|2078|5|7|8|3|2078|715|9315|Friday|2078Q3|N|Y|N|2480216|2480396|2479858|2480132|N|N|N|N|N| +2480224|AAAAAAAAAGINFCAA|2078-07-09|2142|9315|715|2078|6|7|9|3|2078|715|9315|Saturday|2078Q3|N|Y|N|2480216|2480396|2479859|2480133|N|N|N|N|N| +2480225|AAAAAAAABGINFCAA|2078-07-10|2142|9315|715|2078|0|7|10|3|2078|715|9315|Sunday|2078Q3|N|N|N|2480216|2480396|2479860|2480134|N|N|N|N|N| +2480226|AAAAAAAACGINFCAA|2078-07-11|2142|9315|715|2078|1|7|11|3|2078|715|9315|Monday|2078Q3|N|N|N|2480216|2480396|2479861|2480135|N|N|N|N|N| +2480227|AAAAAAAADGINFCAA|2078-07-12|2142|9316|715|2078|2|7|12|3|2078|715|9316|Tuesday|2078Q3|N|N|N|2480216|2480396|2479862|2480136|N|N|N|N|N| +2480228|AAAAAAAAEGINFCAA|2078-07-13|2142|9316|715|2078|3|7|13|3|2078|715|9316|Wednesday|2078Q3|N|N|N|2480216|2480396|2479863|2480137|N|N|N|N|N| +2480229|AAAAAAAAFGINFCAA|2078-07-14|2142|9316|715|2078|4|7|14|3|2078|715|9316|Thursday|2078Q3|N|N|N|2480216|2480396|2479864|2480138|N|N|N|N|N| +2480230|AAAAAAAAGGINFCAA|2078-07-15|2142|9316|715|2078|5|7|15|3|2078|715|9316|Friday|2078Q3|N|Y|N|2480216|2480396|2479865|2480139|N|N|N|N|N| +2480231|AAAAAAAAHGINFCAA|2078-07-16|2142|9316|715|2078|6|7|16|3|2078|715|9316|Saturday|2078Q3|N|Y|N|2480216|2480396|2479866|2480140|N|N|N|N|N| +2480232|AAAAAAAAIGINFCAA|2078-07-17|2142|9316|715|2078|0|7|17|3|2078|715|9316|Sunday|2078Q3|N|N|N|2480216|2480396|2479867|2480141|N|N|N|N|N| +2480233|AAAAAAAAJGINFCAA|2078-07-18|2142|9316|715|2078|1|7|18|3|2078|715|9316|Monday|2078Q3|N|N|N|2480216|2480396|2479868|2480142|N|N|N|N|N| +2480234|AAAAAAAAKGINFCAA|2078-07-19|2142|9317|715|2078|2|7|19|3|2078|715|9317|Tuesday|2078Q3|N|N|N|2480216|2480396|2479869|2480143|N|N|N|N|N| +2480235|AAAAAAAALGINFCAA|2078-07-20|2142|9317|715|2078|3|7|20|3|2078|715|9317|Wednesday|2078Q3|N|N|N|2480216|2480396|2479870|2480144|N|N|N|N|N| +2480236|AAAAAAAAMGINFCAA|2078-07-21|2142|9317|715|2078|4|7|21|3|2078|715|9317|Thursday|2078Q3|N|N|N|2480216|2480396|2479871|2480145|N|N|N|N|N| +2480237|AAAAAAAANGINFCAA|2078-07-22|2142|9317|715|2078|5|7|22|3|2078|715|9317|Friday|2078Q3|N|Y|N|2480216|2480396|2479872|2480146|N|N|N|N|N| +2480238|AAAAAAAAOGINFCAA|2078-07-23|2142|9317|715|2078|6|7|23|3|2078|715|9317|Saturday|2078Q3|N|Y|N|2480216|2480396|2479873|2480147|N|N|N|N|N| +2480239|AAAAAAAAPGINFCAA|2078-07-24|2142|9317|715|2078|0|7|24|3|2078|715|9317|Sunday|2078Q3|N|N|N|2480216|2480396|2479874|2480148|N|N|N|N|N| +2480240|AAAAAAAAAHINFCAA|2078-07-25|2142|9317|715|2078|1|7|25|3|2078|715|9317|Monday|2078Q3|N|N|N|2480216|2480396|2479875|2480149|N|N|N|N|N| +2480241|AAAAAAAABHINFCAA|2078-07-26|2142|9318|715|2078|2|7|26|3|2078|715|9318|Tuesday|2078Q3|N|N|N|2480216|2480396|2479876|2480150|N|N|N|N|N| +2480242|AAAAAAAACHINFCAA|2078-07-27|2142|9318|715|2078|3|7|27|3|2078|715|9318|Wednesday|2078Q3|N|N|N|2480216|2480396|2479877|2480151|N|N|N|N|N| +2480243|AAAAAAAADHINFCAA|2078-07-28|2142|9318|715|2078|4|7|28|3|2078|715|9318|Thursday|2078Q3|N|N|N|2480216|2480396|2479878|2480152|N|N|N|N|N| +2480244|AAAAAAAAEHINFCAA|2078-07-29|2142|9318|715|2078|5|7|29|3|2078|715|9318|Friday|2078Q3|N|Y|N|2480216|2480396|2479879|2480153|N|N|N|N|N| +2480245|AAAAAAAAFHINFCAA|2078-07-30|2142|9318|715|2078|6|7|30|3|2078|715|9318|Saturday|2078Q3|N|Y|N|2480216|2480396|2479880|2480154|N|N|N|N|N| +2480246|AAAAAAAAGHINFCAA|2078-07-31|2142|9318|715|2078|0|7|31|3|2078|715|9318|Sunday|2078Q3|N|N|N|2480216|2480396|2479881|2480155|N|N|N|N|N| +2480247|AAAAAAAAHHINFCAA|2078-08-01|2143|9318|715|2078|1|8|1|3|2078|715|9318|Monday|2078Q3|N|N|N|2480247|2480458|2479882|2480156|N|N|N|N|N| +2480248|AAAAAAAAIHINFCAA|2078-08-02|2143|9319|715|2078|2|8|2|3|2078|715|9319|Tuesday|2078Q3|N|N|N|2480247|2480458|2479883|2480157|N|N|N|N|N| +2480249|AAAAAAAAJHINFCAA|2078-08-03|2143|9319|715|2078|3|8|3|3|2078|715|9319|Wednesday|2078Q3|N|N|N|2480247|2480458|2479884|2480158|N|N|N|N|N| +2480250|AAAAAAAAKHINFCAA|2078-08-04|2143|9319|715|2078|4|8|4|3|2078|715|9319|Thursday|2078Q3|N|N|N|2480247|2480458|2479885|2480159|N|N|N|N|N| +2480251|AAAAAAAALHINFCAA|2078-08-05|2143|9319|715|2078|5|8|5|3|2078|715|9319|Friday|2078Q3|N|Y|N|2480247|2480458|2479886|2480160|N|N|N|N|N| +2480252|AAAAAAAAMHINFCAA|2078-08-06|2143|9319|715|2078|6|8|6|3|2078|715|9319|Saturday|2078Q3|N|Y|N|2480247|2480458|2479887|2480161|N|N|N|N|N| +2480253|AAAAAAAANHINFCAA|2078-08-07|2143|9319|715|2078|0|8|7|3|2078|715|9319|Sunday|2078Q3|N|N|N|2480247|2480458|2479888|2480162|N|N|N|N|N| +2480254|AAAAAAAAOHINFCAA|2078-08-08|2143|9319|715|2078|1|8|8|3|2078|715|9319|Monday|2078Q3|N|N|N|2480247|2480458|2479889|2480163|N|N|N|N|N| +2480255|AAAAAAAAPHINFCAA|2078-08-09|2143|9320|715|2078|2|8|9|3|2078|715|9320|Tuesday|2078Q3|N|N|N|2480247|2480458|2479890|2480164|N|N|N|N|N| +2480256|AAAAAAAAAIINFCAA|2078-08-10|2143|9320|715|2078|3|8|10|3|2078|715|9320|Wednesday|2078Q3|N|N|N|2480247|2480458|2479891|2480165|N|N|N|N|N| +2480257|AAAAAAAABIINFCAA|2078-08-11|2143|9320|715|2078|4|8|11|3|2078|715|9320|Thursday|2078Q3|N|N|N|2480247|2480458|2479892|2480166|N|N|N|N|N| +2480258|AAAAAAAACIINFCAA|2078-08-12|2143|9320|715|2078|5|8|12|3|2078|715|9320|Friday|2078Q3|N|Y|N|2480247|2480458|2479893|2480167|N|N|N|N|N| +2480259|AAAAAAAADIINFCAA|2078-08-13|2143|9320|715|2078|6|8|13|3|2078|715|9320|Saturday|2078Q3|N|Y|N|2480247|2480458|2479894|2480168|N|N|N|N|N| +2480260|AAAAAAAAEIINFCAA|2078-08-14|2143|9320|715|2078|0|8|14|3|2078|715|9320|Sunday|2078Q3|N|N|N|2480247|2480458|2479895|2480169|N|N|N|N|N| +2480261|AAAAAAAAFIINFCAA|2078-08-15|2143|9320|715|2078|1|8|15|3|2078|715|9320|Monday|2078Q3|N|N|N|2480247|2480458|2479896|2480170|N|N|N|N|N| +2480262|AAAAAAAAGIINFCAA|2078-08-16|2143|9321|715|2078|2|8|16|3|2078|715|9321|Tuesday|2078Q3|N|N|N|2480247|2480458|2479897|2480171|N|N|N|N|N| +2480263|AAAAAAAAHIINFCAA|2078-08-17|2143|9321|715|2078|3|8|17|3|2078|715|9321|Wednesday|2078Q3|N|N|N|2480247|2480458|2479898|2480172|N|N|N|N|N| +2480264|AAAAAAAAIIINFCAA|2078-08-18|2143|9321|715|2078|4|8|18|3|2078|715|9321|Thursday|2078Q3|N|N|N|2480247|2480458|2479899|2480173|N|N|N|N|N| +2480265|AAAAAAAAJIINFCAA|2078-08-19|2143|9321|715|2078|5|8|19|3|2078|715|9321|Friday|2078Q3|N|Y|N|2480247|2480458|2479900|2480174|N|N|N|N|N| +2480266|AAAAAAAAKIINFCAA|2078-08-20|2143|9321|715|2078|6|8|20|3|2078|715|9321|Saturday|2078Q3|N|Y|N|2480247|2480458|2479901|2480175|N|N|N|N|N| +2480267|AAAAAAAALIINFCAA|2078-08-21|2143|9321|715|2078|0|8|21|3|2078|715|9321|Sunday|2078Q3|N|N|N|2480247|2480458|2479902|2480176|N|N|N|N|N| +2480268|AAAAAAAAMIINFCAA|2078-08-22|2143|9321|715|2078|1|8|22|3|2078|715|9321|Monday|2078Q3|N|N|N|2480247|2480458|2479903|2480177|N|N|N|N|N| +2480269|AAAAAAAANIINFCAA|2078-08-23|2143|9322|715|2078|2|8|23|3|2078|715|9322|Tuesday|2078Q3|N|N|N|2480247|2480458|2479904|2480178|N|N|N|N|N| +2480270|AAAAAAAAOIINFCAA|2078-08-24|2143|9322|715|2078|3|8|24|3|2078|715|9322|Wednesday|2078Q3|N|N|N|2480247|2480458|2479905|2480179|N|N|N|N|N| +2480271|AAAAAAAAPIINFCAA|2078-08-25|2143|9322|715|2078|4|8|25|3|2078|715|9322|Thursday|2078Q3|N|N|N|2480247|2480458|2479906|2480180|N|N|N|N|N| +2480272|AAAAAAAAAJINFCAA|2078-08-26|2143|9322|715|2078|5|8|26|3|2078|715|9322|Friday|2078Q3|N|Y|N|2480247|2480458|2479907|2480181|N|N|N|N|N| +2480273|AAAAAAAABJINFCAA|2078-08-27|2143|9322|715|2078|6|8|27|3|2078|715|9322|Saturday|2078Q3|N|Y|N|2480247|2480458|2479908|2480182|N|N|N|N|N| +2480274|AAAAAAAACJINFCAA|2078-08-28|2143|9322|715|2078|0|8|28|3|2078|715|9322|Sunday|2078Q3|N|N|N|2480247|2480458|2479909|2480183|N|N|N|N|N| +2480275|AAAAAAAADJINFCAA|2078-08-29|2143|9322|715|2078|1|8|29|3|2078|715|9322|Monday|2078Q3|N|N|N|2480247|2480458|2479910|2480184|N|N|N|N|N| +2480276|AAAAAAAAEJINFCAA|2078-08-30|2143|9323|715|2078|2|8|30|3|2078|715|9323|Tuesday|2078Q3|N|N|N|2480247|2480458|2479911|2480185|N|N|N|N|N| +2480277|AAAAAAAAFJINFCAA|2078-08-31|2143|9323|715|2078|3|8|31|3|2078|715|9323|Wednesday|2078Q3|N|N|N|2480247|2480458|2479912|2480186|N|N|N|N|N| +2480278|AAAAAAAAGJINFCAA|2078-09-01|2144|9323|716|2078|4|9|1|3|2078|716|9323|Thursday|2078Q3|N|N|N|2480278|2480520|2479913|2480187|N|N|N|N|N| +2480279|AAAAAAAAHJINFCAA|2078-09-02|2144|9323|716|2078|5|9|2|3|2078|716|9323|Friday|2078Q3|N|Y|N|2480278|2480520|2479914|2480188|N|N|N|N|N| +2480280|AAAAAAAAIJINFCAA|2078-09-03|2144|9323|716|2078|6|9|3|3|2078|716|9323|Saturday|2078Q3|N|Y|N|2480278|2480520|2479915|2480189|N|N|N|N|N| +2480281|AAAAAAAAJJINFCAA|2078-09-04|2144|9323|716|2078|0|9|4|3|2078|716|9323|Sunday|2078Q3|N|N|N|2480278|2480520|2479916|2480190|N|N|N|N|N| +2480282|AAAAAAAAKJINFCAA|2078-09-05|2144|9323|716|2078|1|9|5|3|2078|716|9323|Monday|2078Q3|N|N|N|2480278|2480520|2479917|2480191|N|N|N|N|N| +2480283|AAAAAAAALJINFCAA|2078-09-06|2144|9324|716|2078|2|9|6|3|2078|716|9324|Tuesday|2078Q3|N|N|N|2480278|2480520|2479918|2480192|N|N|N|N|N| +2480284|AAAAAAAAMJINFCAA|2078-09-07|2144|9324|716|2078|3|9|7|3|2078|716|9324|Wednesday|2078Q3|N|N|N|2480278|2480520|2479919|2480193|N|N|N|N|N| +2480285|AAAAAAAANJINFCAA|2078-09-08|2144|9324|716|2078|4|9|8|3|2078|716|9324|Thursday|2078Q3|N|N|N|2480278|2480520|2479920|2480194|N|N|N|N|N| +2480286|AAAAAAAAOJINFCAA|2078-09-09|2144|9324|716|2078|5|9|9|3|2078|716|9324|Friday|2078Q3|N|Y|N|2480278|2480520|2479921|2480195|N|N|N|N|N| +2480287|AAAAAAAAPJINFCAA|2078-09-10|2144|9324|716|2078|6|9|10|3|2078|716|9324|Saturday|2078Q3|N|Y|N|2480278|2480520|2479922|2480196|N|N|N|N|N| +2480288|AAAAAAAAAKINFCAA|2078-09-11|2144|9324|716|2078|0|9|11|3|2078|716|9324|Sunday|2078Q3|N|N|N|2480278|2480520|2479923|2480197|N|N|N|N|N| +2480289|AAAAAAAABKINFCAA|2078-09-12|2144|9324|716|2078|1|9|12|3|2078|716|9324|Monday|2078Q3|N|N|N|2480278|2480520|2479924|2480198|N|N|N|N|N| +2480290|AAAAAAAACKINFCAA|2078-09-13|2144|9325|716|2078|2|9|13|3|2078|716|9325|Tuesday|2078Q3|N|N|N|2480278|2480520|2479925|2480199|N|N|N|N|N| +2480291|AAAAAAAADKINFCAA|2078-09-14|2144|9325|716|2078|3|9|14|3|2078|716|9325|Wednesday|2078Q3|N|N|N|2480278|2480520|2479926|2480200|N|N|N|N|N| +2480292|AAAAAAAAEKINFCAA|2078-09-15|2144|9325|716|2078|4|9|15|3|2078|716|9325|Thursday|2078Q3|N|N|N|2480278|2480520|2479927|2480201|N|N|N|N|N| +2480293|AAAAAAAAFKINFCAA|2078-09-16|2144|9325|716|2078|5|9|16|3|2078|716|9325|Friday|2078Q3|N|Y|N|2480278|2480520|2479928|2480202|N|N|N|N|N| +2480294|AAAAAAAAGKINFCAA|2078-09-17|2144|9325|716|2078|6|9|17|3|2078|716|9325|Saturday|2078Q3|N|Y|N|2480278|2480520|2479929|2480203|N|N|N|N|N| +2480295|AAAAAAAAHKINFCAA|2078-09-18|2144|9325|716|2078|0|9|18|3|2078|716|9325|Sunday|2078Q3|N|N|N|2480278|2480520|2479930|2480204|N|N|N|N|N| +2480296|AAAAAAAAIKINFCAA|2078-09-19|2144|9325|716|2078|1|9|19|3|2078|716|9325|Monday|2078Q3|N|N|N|2480278|2480520|2479931|2480205|N|N|N|N|N| +2480297|AAAAAAAAJKINFCAA|2078-09-20|2144|9326|716|2078|2|9|20|3|2078|716|9326|Tuesday|2078Q3|N|N|N|2480278|2480520|2479932|2480206|N|N|N|N|N| +2480298|AAAAAAAAKKINFCAA|2078-09-21|2144|9326|716|2078|3|9|21|3|2078|716|9326|Wednesday|2078Q3|N|N|N|2480278|2480520|2479933|2480207|N|N|N|N|N| +2480299|AAAAAAAALKINFCAA|2078-09-22|2144|9326|716|2078|4|9|22|3|2078|716|9326|Thursday|2078Q3|N|N|N|2480278|2480520|2479934|2480208|N|N|N|N|N| +2480300|AAAAAAAAMKINFCAA|2078-09-23|2144|9326|716|2078|5|9|23|3|2078|716|9326|Friday|2078Q3|N|Y|N|2480278|2480520|2479935|2480209|N|N|N|N|N| +2480301|AAAAAAAANKINFCAA|2078-09-24|2144|9326|716|2078|6|9|24|3|2078|716|9326|Saturday|2078Q3|N|Y|N|2480278|2480520|2479936|2480210|N|N|N|N|N| +2480302|AAAAAAAAOKINFCAA|2078-09-25|2144|9326|716|2078|0|9|25|3|2078|716|9326|Sunday|2078Q3|N|N|N|2480278|2480520|2479937|2480211|N|N|N|N|N| +2480303|AAAAAAAAPKINFCAA|2078-09-26|2144|9326|716|2078|1|9|26|3|2078|716|9326|Monday|2078Q3|N|N|N|2480278|2480520|2479938|2480212|N|N|N|N|N| +2480304|AAAAAAAAALINFCAA|2078-09-27|2144|9327|716|2078|2|9|27|3|2078|716|9327|Tuesday|2078Q3|N|N|N|2480278|2480520|2479939|2480213|N|N|N|N|N| +2480305|AAAAAAAABLINFCAA|2078-09-28|2144|9327|716|2078|3|9|28|3|2078|716|9327|Wednesday|2078Q3|N|N|N|2480278|2480520|2479940|2480214|N|N|N|N|N| +2480306|AAAAAAAACLINFCAA|2078-09-29|2144|9327|716|2078|4|9|29|3|2078|716|9327|Thursday|2078Q3|N|N|N|2480278|2480520|2479941|2480215|N|N|N|N|N| +2480307|AAAAAAAADLINFCAA|2078-09-30|2144|9327|716|2078|5|9|30|3|2078|716|9327|Friday|2078Q3|N|Y|N|2480278|2480520|2479942|2480216|N|N|N|N|N| +2480308|AAAAAAAAELINFCAA|2078-10-01|2145|9327|716|2078|6|10|1|3|2078|716|9327|Saturday|2078Q3|N|Y|N|2480308|2480580|2479943|2480216|N|N|N|N|N| +2480309|AAAAAAAAFLINFCAA|2078-10-02|2145|9327|716|2078|0|10|2|4|2078|716|9327|Sunday|2078Q4|N|N|N|2480308|2480580|2479944|2480217|N|N|N|N|N| +2480310|AAAAAAAAGLINFCAA|2078-10-03|2145|9327|716|2078|1|10|3|4|2078|716|9327|Monday|2078Q4|N|N|N|2480308|2480580|2479945|2480218|N|N|N|N|N| +2480311|AAAAAAAAHLINFCAA|2078-10-04|2145|9328|716|2078|2|10|4|4|2078|716|9328|Tuesday|2078Q4|N|N|N|2480308|2480580|2479946|2480219|N|N|N|N|N| +2480312|AAAAAAAAILINFCAA|2078-10-05|2145|9328|716|2078|3|10|5|4|2078|716|9328|Wednesday|2078Q4|N|N|N|2480308|2480580|2479947|2480220|N|N|N|N|N| +2480313|AAAAAAAAJLINFCAA|2078-10-06|2145|9328|716|2078|4|10|6|4|2078|716|9328|Thursday|2078Q4|N|N|N|2480308|2480580|2479948|2480221|N|N|N|N|N| +2480314|AAAAAAAAKLINFCAA|2078-10-07|2145|9328|716|2078|5|10|7|4|2078|716|9328|Friday|2078Q4|N|Y|N|2480308|2480580|2479949|2480222|N|N|N|N|N| +2480315|AAAAAAAALLINFCAA|2078-10-08|2145|9328|716|2078|6|10|8|4|2078|716|9328|Saturday|2078Q4|N|Y|N|2480308|2480580|2479950|2480223|N|N|N|N|N| +2480316|AAAAAAAAMLINFCAA|2078-10-09|2145|9328|716|2078|0|10|9|4|2078|716|9328|Sunday|2078Q4|N|N|N|2480308|2480580|2479951|2480224|N|N|N|N|N| +2480317|AAAAAAAANLINFCAA|2078-10-10|2145|9328|716|2078|1|10|10|4|2078|716|9328|Monday|2078Q4|N|N|N|2480308|2480580|2479952|2480225|N|N|N|N|N| +2480318|AAAAAAAAOLINFCAA|2078-10-11|2145|9329|716|2078|2|10|11|4|2078|716|9329|Tuesday|2078Q4|N|N|N|2480308|2480580|2479953|2480226|N|N|N|N|N| +2480319|AAAAAAAAPLINFCAA|2078-10-12|2145|9329|716|2078|3|10|12|4|2078|716|9329|Wednesday|2078Q4|N|N|N|2480308|2480580|2479954|2480227|N|N|N|N|N| +2480320|AAAAAAAAAMINFCAA|2078-10-13|2145|9329|716|2078|4|10|13|4|2078|716|9329|Thursday|2078Q4|N|N|N|2480308|2480580|2479955|2480228|N|N|N|N|N| +2480321|AAAAAAAABMINFCAA|2078-10-14|2145|9329|716|2078|5|10|14|4|2078|716|9329|Friday|2078Q4|N|Y|N|2480308|2480580|2479956|2480229|N|N|N|N|N| +2480322|AAAAAAAACMINFCAA|2078-10-15|2145|9329|716|2078|6|10|15|4|2078|716|9329|Saturday|2078Q4|N|Y|N|2480308|2480580|2479957|2480230|N|N|N|N|N| +2480323|AAAAAAAADMINFCAA|2078-10-16|2145|9329|716|2078|0|10|16|4|2078|716|9329|Sunday|2078Q4|N|N|N|2480308|2480580|2479958|2480231|N|N|N|N|N| +2480324|AAAAAAAAEMINFCAA|2078-10-17|2145|9329|716|2078|1|10|17|4|2078|716|9329|Monday|2078Q4|N|N|N|2480308|2480580|2479959|2480232|N|N|N|N|N| +2480325|AAAAAAAAFMINFCAA|2078-10-18|2145|9330|716|2078|2|10|18|4|2078|716|9330|Tuesday|2078Q4|N|N|N|2480308|2480580|2479960|2480233|N|N|N|N|N| +2480326|AAAAAAAAGMINFCAA|2078-10-19|2145|9330|716|2078|3|10|19|4|2078|716|9330|Wednesday|2078Q4|N|N|N|2480308|2480580|2479961|2480234|N|N|N|N|N| +2480327|AAAAAAAAHMINFCAA|2078-10-20|2145|9330|716|2078|4|10|20|4|2078|716|9330|Thursday|2078Q4|N|N|N|2480308|2480580|2479962|2480235|N|N|N|N|N| +2480328|AAAAAAAAIMINFCAA|2078-10-21|2145|9330|716|2078|5|10|21|4|2078|716|9330|Friday|2078Q4|N|Y|N|2480308|2480580|2479963|2480236|N|N|N|N|N| +2480329|AAAAAAAAJMINFCAA|2078-10-22|2145|9330|716|2078|6|10|22|4|2078|716|9330|Saturday|2078Q4|N|Y|N|2480308|2480580|2479964|2480237|N|N|N|N|N| +2480330|AAAAAAAAKMINFCAA|2078-10-23|2145|9330|716|2078|0|10|23|4|2078|716|9330|Sunday|2078Q4|N|N|N|2480308|2480580|2479965|2480238|N|N|N|N|N| +2480331|AAAAAAAALMINFCAA|2078-10-24|2145|9330|716|2078|1|10|24|4|2078|716|9330|Monday|2078Q4|N|N|N|2480308|2480580|2479966|2480239|N|N|N|N|N| +2480332|AAAAAAAAMMINFCAA|2078-10-25|2145|9331|716|2078|2|10|25|4|2078|716|9331|Tuesday|2078Q4|N|N|N|2480308|2480580|2479967|2480240|N|N|N|N|N| +2480333|AAAAAAAANMINFCAA|2078-10-26|2145|9331|716|2078|3|10|26|4|2078|716|9331|Wednesday|2078Q4|N|N|N|2480308|2480580|2479968|2480241|N|N|N|N|N| +2480334|AAAAAAAAOMINFCAA|2078-10-27|2145|9331|716|2078|4|10|27|4|2078|716|9331|Thursday|2078Q4|N|N|N|2480308|2480580|2479969|2480242|N|N|N|N|N| +2480335|AAAAAAAAPMINFCAA|2078-10-28|2145|9331|716|2078|5|10|28|4|2078|716|9331|Friday|2078Q4|N|Y|N|2480308|2480580|2479970|2480243|N|N|N|N|N| +2480336|AAAAAAAAANINFCAA|2078-10-29|2145|9331|716|2078|6|10|29|4|2078|716|9331|Saturday|2078Q4|N|Y|N|2480308|2480580|2479971|2480244|N|N|N|N|N| +2480337|AAAAAAAABNINFCAA|2078-10-30|2145|9331|716|2078|0|10|30|4|2078|716|9331|Sunday|2078Q4|N|N|N|2480308|2480580|2479972|2480245|N|N|N|N|N| +2480338|AAAAAAAACNINFCAA|2078-10-31|2145|9331|716|2078|1|10|31|4|2078|716|9331|Monday|2078Q4|N|N|N|2480308|2480580|2479973|2480246|N|N|N|N|N| +2480339|AAAAAAAADNINFCAA|2078-11-01|2146|9332|716|2078|2|11|1|4|2078|716|9332|Tuesday|2078Q4|N|N|N|2480339|2480642|2479974|2480247|N|N|N|N|N| +2480340|AAAAAAAAENINFCAA|2078-11-02|2146|9332|716|2078|3|11|2|4|2078|716|9332|Wednesday|2078Q4|N|N|N|2480339|2480642|2479975|2480248|N|N|N|N|N| +2480341|AAAAAAAAFNINFCAA|2078-11-03|2146|9332|716|2078|4|11|3|4|2078|716|9332|Thursday|2078Q4|N|N|N|2480339|2480642|2479976|2480249|N|N|N|N|N| +2480342|AAAAAAAAGNINFCAA|2078-11-04|2146|9332|716|2078|5|11|4|4|2078|716|9332|Friday|2078Q4|N|Y|N|2480339|2480642|2479977|2480250|N|N|N|N|N| +2480343|AAAAAAAAHNINFCAA|2078-11-05|2146|9332|716|2078|6|11|5|4|2078|716|9332|Saturday|2078Q4|N|Y|N|2480339|2480642|2479978|2480251|N|N|N|N|N| +2480344|AAAAAAAAININFCAA|2078-11-06|2146|9332|716|2078|0|11|6|4|2078|716|9332|Sunday|2078Q4|N|N|N|2480339|2480642|2479979|2480252|N|N|N|N|N| +2480345|AAAAAAAAJNINFCAA|2078-11-07|2146|9332|716|2078|1|11|7|4|2078|716|9332|Monday|2078Q4|N|N|N|2480339|2480642|2479980|2480253|N|N|N|N|N| +2480346|AAAAAAAAKNINFCAA|2078-11-08|2146|9333|716|2078|2|11|8|4|2078|716|9333|Tuesday|2078Q4|N|N|N|2480339|2480642|2479981|2480254|N|N|N|N|N| +2480347|AAAAAAAALNINFCAA|2078-11-09|2146|9333|716|2078|3|11|9|4|2078|716|9333|Wednesday|2078Q4|N|N|N|2480339|2480642|2479982|2480255|N|N|N|N|N| +2480348|AAAAAAAAMNINFCAA|2078-11-10|2146|9333|716|2078|4|11|10|4|2078|716|9333|Thursday|2078Q4|N|N|N|2480339|2480642|2479983|2480256|N|N|N|N|N| +2480349|AAAAAAAANNINFCAA|2078-11-11|2146|9333|716|2078|5|11|11|4|2078|716|9333|Friday|2078Q4|N|Y|N|2480339|2480642|2479984|2480257|N|N|N|N|N| +2480350|AAAAAAAAONINFCAA|2078-11-12|2146|9333|716|2078|6|11|12|4|2078|716|9333|Saturday|2078Q4|N|Y|N|2480339|2480642|2479985|2480258|N|N|N|N|N| +2480351|AAAAAAAAPNINFCAA|2078-11-13|2146|9333|716|2078|0|11|13|4|2078|716|9333|Sunday|2078Q4|N|N|N|2480339|2480642|2479986|2480259|N|N|N|N|N| +2480352|AAAAAAAAAOINFCAA|2078-11-14|2146|9333|716|2078|1|11|14|4|2078|716|9333|Monday|2078Q4|N|N|N|2480339|2480642|2479987|2480260|N|N|N|N|N| +2480353|AAAAAAAABOINFCAA|2078-11-15|2146|9334|716|2078|2|11|15|4|2078|716|9334|Tuesday|2078Q4|N|N|N|2480339|2480642|2479988|2480261|N|N|N|N|N| +2480354|AAAAAAAACOINFCAA|2078-11-16|2146|9334|716|2078|3|11|16|4|2078|716|9334|Wednesday|2078Q4|N|N|N|2480339|2480642|2479989|2480262|N|N|N|N|N| +2480355|AAAAAAAADOINFCAA|2078-11-17|2146|9334|716|2078|4|11|17|4|2078|716|9334|Thursday|2078Q4|N|N|N|2480339|2480642|2479990|2480263|N|N|N|N|N| +2480356|AAAAAAAAEOINFCAA|2078-11-18|2146|9334|716|2078|5|11|18|4|2078|716|9334|Friday|2078Q4|N|Y|N|2480339|2480642|2479991|2480264|N|N|N|N|N| +2480357|AAAAAAAAFOINFCAA|2078-11-19|2146|9334|716|2078|6|11|19|4|2078|716|9334|Saturday|2078Q4|N|Y|N|2480339|2480642|2479992|2480265|N|N|N|N|N| +2480358|AAAAAAAAGOINFCAA|2078-11-20|2146|9334|716|2078|0|11|20|4|2078|716|9334|Sunday|2078Q4|N|N|N|2480339|2480642|2479993|2480266|N|N|N|N|N| +2480359|AAAAAAAAHOINFCAA|2078-11-21|2146|9334|716|2078|1|11|21|4|2078|716|9334|Monday|2078Q4|N|N|N|2480339|2480642|2479994|2480267|N|N|N|N|N| +2480360|AAAAAAAAIOINFCAA|2078-11-22|2146|9335|716|2078|2|11|22|4|2078|716|9335|Tuesday|2078Q4|N|N|N|2480339|2480642|2479995|2480268|N|N|N|N|N| +2480361|AAAAAAAAJOINFCAA|2078-11-23|2146|9335|716|2078|3|11|23|4|2078|716|9335|Wednesday|2078Q4|N|N|N|2480339|2480642|2479996|2480269|N|N|N|N|N| +2480362|AAAAAAAAKOINFCAA|2078-11-24|2146|9335|716|2078|4|11|24|4|2078|716|9335|Thursday|2078Q4|N|N|N|2480339|2480642|2479997|2480270|N|N|N|N|N| +2480363|AAAAAAAALOINFCAA|2078-11-25|2146|9335|716|2078|5|11|25|4|2078|716|9335|Friday|2078Q4|N|Y|N|2480339|2480642|2479998|2480271|N|N|N|N|N| +2480364|AAAAAAAAMOINFCAA|2078-11-26|2146|9335|716|2078|6|11|26|4|2078|716|9335|Saturday|2078Q4|N|Y|N|2480339|2480642|2479999|2480272|N|N|N|N|N| +2480365|AAAAAAAANOINFCAA|2078-11-27|2146|9335|716|2078|0|11|27|4|2078|716|9335|Sunday|2078Q4|N|N|N|2480339|2480642|2480000|2480273|N|N|N|N|N| +2480366|AAAAAAAAOOINFCAA|2078-11-28|2146|9335|716|2078|1|11|28|4|2078|716|9335|Monday|2078Q4|N|N|N|2480339|2480642|2480001|2480274|N|N|N|N|N| +2480367|AAAAAAAAPOINFCAA|2078-11-29|2146|9336|716|2078|2|11|29|4|2078|716|9336|Tuesday|2078Q4|N|N|N|2480339|2480642|2480002|2480275|N|N|N|N|N| +2480368|AAAAAAAAAPINFCAA|2078-11-30|2146|9336|716|2078|3|11|30|4|2078|716|9336|Wednesday|2078Q4|N|N|N|2480339|2480642|2480003|2480276|N|N|N|N|N| +2480369|AAAAAAAABPINFCAA|2078-12-01|2147|9336|717|2078|4|12|1|4|2078|717|9336|Thursday|2078Q4|N|N|N|2480369|2480702|2480004|2480277|N|N|N|N|N| +2480370|AAAAAAAACPINFCAA|2078-12-02|2147|9336|717|2078|5|12|2|4|2078|717|9336|Friday|2078Q4|N|Y|N|2480369|2480702|2480005|2480278|N|N|N|N|N| +2480371|AAAAAAAADPINFCAA|2078-12-03|2147|9336|717|2078|6|12|3|4|2078|717|9336|Saturday|2078Q4|N|Y|N|2480369|2480702|2480006|2480279|N|N|N|N|N| +2480372|AAAAAAAAEPINFCAA|2078-12-04|2147|9336|717|2078|0|12|4|4|2078|717|9336|Sunday|2078Q4|N|N|N|2480369|2480702|2480007|2480280|N|N|N|N|N| +2480373|AAAAAAAAFPINFCAA|2078-12-05|2147|9336|717|2078|1|12|5|4|2078|717|9336|Monday|2078Q4|N|N|N|2480369|2480702|2480008|2480281|N|N|N|N|N| +2480374|AAAAAAAAGPINFCAA|2078-12-06|2147|9337|717|2078|2|12|6|4|2078|717|9337|Tuesday|2078Q4|N|N|N|2480369|2480702|2480009|2480282|N|N|N|N|N| +2480375|AAAAAAAAHPINFCAA|2078-12-07|2147|9337|717|2078|3|12|7|4|2078|717|9337|Wednesday|2078Q4|N|N|N|2480369|2480702|2480010|2480283|N|N|N|N|N| +2480376|AAAAAAAAIPINFCAA|2078-12-08|2147|9337|717|2078|4|12|8|4|2078|717|9337|Thursday|2078Q4|N|N|N|2480369|2480702|2480011|2480284|N|N|N|N|N| +2480377|AAAAAAAAJPINFCAA|2078-12-09|2147|9337|717|2078|5|12|9|4|2078|717|9337|Friday|2078Q4|N|Y|N|2480369|2480702|2480012|2480285|N|N|N|N|N| +2480378|AAAAAAAAKPINFCAA|2078-12-10|2147|9337|717|2078|6|12|10|4|2078|717|9337|Saturday|2078Q4|N|Y|N|2480369|2480702|2480013|2480286|N|N|N|N|N| +2480379|AAAAAAAALPINFCAA|2078-12-11|2147|9337|717|2078|0|12|11|4|2078|717|9337|Sunday|2078Q4|N|N|N|2480369|2480702|2480014|2480287|N|N|N|N|N| +2480380|AAAAAAAAMPINFCAA|2078-12-12|2147|9337|717|2078|1|12|12|4|2078|717|9337|Monday|2078Q4|N|N|N|2480369|2480702|2480015|2480288|N|N|N|N|N| +2480381|AAAAAAAANPINFCAA|2078-12-13|2147|9338|717|2078|2|12|13|4|2078|717|9338|Tuesday|2078Q4|N|N|N|2480369|2480702|2480016|2480289|N|N|N|N|N| +2480382|AAAAAAAAOPINFCAA|2078-12-14|2147|9338|717|2078|3|12|14|4|2078|717|9338|Wednesday|2078Q4|N|N|N|2480369|2480702|2480017|2480290|N|N|N|N|N| +2480383|AAAAAAAAPPINFCAA|2078-12-15|2147|9338|717|2078|4|12|15|4|2078|717|9338|Thursday|2078Q4|N|N|N|2480369|2480702|2480018|2480291|N|N|N|N|N| +2480384|AAAAAAAAAAJNFCAA|2078-12-16|2147|9338|717|2078|5|12|16|4|2078|717|9338|Friday|2078Q4|N|Y|N|2480369|2480702|2480019|2480292|N|N|N|N|N| +2480385|AAAAAAAABAJNFCAA|2078-12-17|2147|9338|717|2078|6|12|17|4|2078|717|9338|Saturday|2078Q4|N|Y|N|2480369|2480702|2480020|2480293|N|N|N|N|N| +2480386|AAAAAAAACAJNFCAA|2078-12-18|2147|9338|717|2078|0|12|18|4|2078|717|9338|Sunday|2078Q4|N|N|N|2480369|2480702|2480021|2480294|N|N|N|N|N| +2480387|AAAAAAAADAJNFCAA|2078-12-19|2147|9338|717|2078|1|12|19|4|2078|717|9338|Monday|2078Q4|N|N|N|2480369|2480702|2480022|2480295|N|N|N|N|N| +2480388|AAAAAAAAEAJNFCAA|2078-12-20|2147|9339|717|2078|2|12|20|4|2078|717|9339|Tuesday|2078Q4|N|N|N|2480369|2480702|2480023|2480296|N|N|N|N|N| +2480389|AAAAAAAAFAJNFCAA|2078-12-21|2147|9339|717|2078|3|12|21|4|2078|717|9339|Wednesday|2078Q4|N|N|N|2480369|2480702|2480024|2480297|N|N|N|N|N| +2480390|AAAAAAAAGAJNFCAA|2078-12-22|2147|9339|717|2078|4|12|22|4|2078|717|9339|Thursday|2078Q4|N|N|N|2480369|2480702|2480025|2480298|N|N|N|N|N| +2480391|AAAAAAAAHAJNFCAA|2078-12-23|2147|9339|717|2078|5|12|23|4|2078|717|9339|Friday|2078Q4|N|Y|N|2480369|2480702|2480026|2480299|N|N|N|N|N| +2480392|AAAAAAAAIAJNFCAA|2078-12-24|2147|9339|717|2078|6|12|24|4|2078|717|9339|Saturday|2078Q4|N|Y|N|2480369|2480702|2480027|2480300|N|N|N|N|N| +2480393|AAAAAAAAJAJNFCAA|2078-12-25|2147|9339|717|2078|0|12|25|4|2078|717|9339|Sunday|2078Q4|N|N|N|2480369|2480702|2480028|2480301|N|N|N|N|N| +2480394|AAAAAAAAKAJNFCAA|2078-12-26|2147|9339|717|2078|1|12|26|4|2078|717|9339|Monday|2078Q4|Y|N|N|2480369|2480702|2480029|2480302|N|N|N|N|N| +2480395|AAAAAAAALAJNFCAA|2078-12-27|2147|9340|717|2078|2|12|27|4|2078|717|9340|Tuesday|2078Q4|N|N|Y|2480369|2480702|2480030|2480303|N|N|N|N|N| +2480396|AAAAAAAAMAJNFCAA|2078-12-28|2147|9340|717|2078|3|12|28|4|2078|717|9340|Wednesday|2078Q4|N|N|N|2480369|2480702|2480031|2480304|N|N|N|N|N| +2480397|AAAAAAAANAJNFCAA|2078-12-29|2147|9340|717|2078|4|12|29|4|2078|717|9340|Thursday|2078Q4|N|N|N|2480369|2480702|2480032|2480305|N|N|N|N|N| +2480398|AAAAAAAAOAJNFCAA|2078-12-30|2147|9340|717|2078|5|12|30|4|2078|717|9340|Friday|2078Q4|N|Y|N|2480369|2480702|2480033|2480306|N|N|N|N|N| +2480399|AAAAAAAAPAJNFCAA|2078-12-31|2147|9340|717|2078|6|12|31|4|2078|717|9340|Saturday|2078Q4|N|Y|N|2480369|2480702|2480034|2480307|N|N|N|N|N| +2480400|AAAAAAAAABJNFCAA|2079-01-01|2148|9340|717|2079|0|1|1|1|2079|717|9340|Sunday|2079Q1|Y|N|N|2480400|2480399|2480035|2480308|N|N|N|N|N| +2480401|AAAAAAAABBJNFCAA|2079-01-02|2148|9340|717|2079|1|1|2|1|2079|717|9340|Monday|2079Q1|N|N|Y|2480400|2480399|2480036|2480309|N|N|N|N|N| +2480402|AAAAAAAACBJNFCAA|2079-01-03|2148|9341|717|2079|2|1|3|1|2079|717|9341|Tuesday|2079Q1|N|N|N|2480400|2480399|2480037|2480310|N|N|N|N|N| +2480403|AAAAAAAADBJNFCAA|2079-01-04|2148|9341|717|2079|3|1|4|1|2079|717|9341|Wednesday|2079Q1|N|N|N|2480400|2480399|2480038|2480311|N|N|N|N|N| +2480404|AAAAAAAAEBJNFCAA|2079-01-05|2148|9341|717|2079|4|1|5|1|2079|717|9341|Thursday|2079Q1|N|N|N|2480400|2480399|2480039|2480312|N|N|N|N|N| +2480405|AAAAAAAAFBJNFCAA|2079-01-06|2148|9341|717|2079|5|1|6|1|2079|717|9341|Friday|2079Q1|N|Y|N|2480400|2480399|2480040|2480313|N|N|N|N|N| +2480406|AAAAAAAAGBJNFCAA|2079-01-07|2148|9341|717|2079|6|1|7|1|2079|717|9341|Saturday|2079Q1|N|Y|N|2480400|2480399|2480041|2480314|N|N|N|N|N| +2480407|AAAAAAAAHBJNFCAA|2079-01-08|2148|9341|717|2079|0|1|8|1|2079|717|9341|Sunday|2079Q1|N|N|N|2480400|2480399|2480042|2480315|N|N|N|N|N| +2480408|AAAAAAAAIBJNFCAA|2079-01-09|2148|9341|717|2079|1|1|9|1|2079|717|9341|Monday|2079Q1|N|N|N|2480400|2480399|2480043|2480316|N|N|N|N|N| +2480409|AAAAAAAAJBJNFCAA|2079-01-10|2148|9342|717|2079|2|1|10|1|2079|717|9342|Tuesday|2079Q1|N|N|N|2480400|2480399|2480044|2480317|N|N|N|N|N| +2480410|AAAAAAAAKBJNFCAA|2079-01-11|2148|9342|717|2079|3|1|11|1|2079|717|9342|Wednesday|2079Q1|N|N|N|2480400|2480399|2480045|2480318|N|N|N|N|N| +2480411|AAAAAAAALBJNFCAA|2079-01-12|2148|9342|717|2079|4|1|12|1|2079|717|9342|Thursday|2079Q1|N|N|N|2480400|2480399|2480046|2480319|N|N|N|N|N| +2480412|AAAAAAAAMBJNFCAA|2079-01-13|2148|9342|717|2079|5|1|13|1|2079|717|9342|Friday|2079Q1|N|Y|N|2480400|2480399|2480047|2480320|N|N|N|N|N| +2480413|AAAAAAAANBJNFCAA|2079-01-14|2148|9342|717|2079|6|1|14|1|2079|717|9342|Saturday|2079Q1|N|Y|N|2480400|2480399|2480048|2480321|N|N|N|N|N| +2480414|AAAAAAAAOBJNFCAA|2079-01-15|2148|9342|717|2079|0|1|15|1|2079|717|9342|Sunday|2079Q1|N|N|N|2480400|2480399|2480049|2480322|N|N|N|N|N| +2480415|AAAAAAAAPBJNFCAA|2079-01-16|2148|9342|717|2079|1|1|16|1|2079|717|9342|Monday|2079Q1|N|N|N|2480400|2480399|2480050|2480323|N|N|N|N|N| +2480416|AAAAAAAAACJNFCAA|2079-01-17|2148|9343|717|2079|2|1|17|1|2079|717|9343|Tuesday|2079Q1|N|N|N|2480400|2480399|2480051|2480324|N|N|N|N|N| +2480417|AAAAAAAABCJNFCAA|2079-01-18|2148|9343|717|2079|3|1|18|1|2079|717|9343|Wednesday|2079Q1|N|N|N|2480400|2480399|2480052|2480325|N|N|N|N|N| +2480418|AAAAAAAACCJNFCAA|2079-01-19|2148|9343|717|2079|4|1|19|1|2079|717|9343|Thursday|2079Q1|N|N|N|2480400|2480399|2480053|2480326|N|N|N|N|N| +2480419|AAAAAAAADCJNFCAA|2079-01-20|2148|9343|717|2079|5|1|20|1|2079|717|9343|Friday|2079Q1|N|Y|N|2480400|2480399|2480054|2480327|N|N|N|N|N| +2480420|AAAAAAAAECJNFCAA|2079-01-21|2148|9343|717|2079|6|1|21|1|2079|717|9343|Saturday|2079Q1|N|Y|N|2480400|2480399|2480055|2480328|N|N|N|N|N| +2480421|AAAAAAAAFCJNFCAA|2079-01-22|2148|9343|717|2079|0|1|22|1|2079|717|9343|Sunday|2079Q1|N|N|N|2480400|2480399|2480056|2480329|N|N|N|N|N| +2480422|AAAAAAAAGCJNFCAA|2079-01-23|2148|9343|717|2079|1|1|23|1|2079|717|9343|Monday|2079Q1|N|N|N|2480400|2480399|2480057|2480330|N|N|N|N|N| +2480423|AAAAAAAAHCJNFCAA|2079-01-24|2148|9344|717|2079|2|1|24|1|2079|717|9344|Tuesday|2079Q1|N|N|N|2480400|2480399|2480058|2480331|N|N|N|N|N| +2480424|AAAAAAAAICJNFCAA|2079-01-25|2148|9344|717|2079|3|1|25|1|2079|717|9344|Wednesday|2079Q1|N|N|N|2480400|2480399|2480059|2480332|N|N|N|N|N| +2480425|AAAAAAAAJCJNFCAA|2079-01-26|2148|9344|717|2079|4|1|26|1|2079|717|9344|Thursday|2079Q1|N|N|N|2480400|2480399|2480060|2480333|N|N|N|N|N| +2480426|AAAAAAAAKCJNFCAA|2079-01-27|2148|9344|717|2079|5|1|27|1|2079|717|9344|Friday|2079Q1|N|Y|N|2480400|2480399|2480061|2480334|N|N|N|N|N| +2480427|AAAAAAAALCJNFCAA|2079-01-28|2148|9344|717|2079|6|1|28|1|2079|717|9344|Saturday|2079Q1|N|Y|N|2480400|2480399|2480062|2480335|N|N|N|N|N| +2480428|AAAAAAAAMCJNFCAA|2079-01-29|2148|9344|717|2079|0|1|29|1|2079|717|9344|Sunday|2079Q1|N|N|N|2480400|2480399|2480063|2480336|N|N|N|N|N| +2480429|AAAAAAAANCJNFCAA|2079-01-30|2148|9344|717|2079|1|1|30|1|2079|717|9344|Monday|2079Q1|N|N|N|2480400|2480399|2480064|2480337|N|N|N|N|N| +2480430|AAAAAAAAOCJNFCAA|2079-01-31|2148|9345|717|2079|2|1|31|1|2079|717|9345|Tuesday|2079Q1|N|N|N|2480400|2480399|2480065|2480338|N|N|N|N|N| +2480431|AAAAAAAAPCJNFCAA|2079-02-01|2149|9345|717|2079|3|2|1|1|2079|717|9345|Wednesday|2079Q1|N|N|N|2480431|2480461|2480066|2480339|N|N|N|N|N| +2480432|AAAAAAAAADJNFCAA|2079-02-02|2149|9345|717|2079|4|2|2|1|2079|717|9345|Thursday|2079Q1|N|N|N|2480431|2480461|2480067|2480340|N|N|N|N|N| +2480433|AAAAAAAABDJNFCAA|2079-02-03|2149|9345|717|2079|5|2|3|1|2079|717|9345|Friday|2079Q1|N|Y|N|2480431|2480461|2480068|2480341|N|N|N|N|N| +2480434|AAAAAAAACDJNFCAA|2079-02-04|2149|9345|717|2079|6|2|4|1|2079|717|9345|Saturday|2079Q1|N|Y|N|2480431|2480461|2480069|2480342|N|N|N|N|N| +2480435|AAAAAAAADDJNFCAA|2079-02-05|2149|9345|717|2079|0|2|5|1|2079|717|9345|Sunday|2079Q1|N|N|N|2480431|2480461|2480070|2480343|N|N|N|N|N| +2480436|AAAAAAAAEDJNFCAA|2079-02-06|2149|9345|717|2079|1|2|6|1|2079|717|9345|Monday|2079Q1|N|N|N|2480431|2480461|2480071|2480344|N|N|N|N|N| +2480437|AAAAAAAAFDJNFCAA|2079-02-07|2149|9346|717|2079|2|2|7|1|2079|717|9346|Tuesday|2079Q1|N|N|N|2480431|2480461|2480072|2480345|N|N|N|N|N| +2480438|AAAAAAAAGDJNFCAA|2079-02-08|2149|9346|717|2079|3|2|8|1|2079|717|9346|Wednesday|2079Q1|N|N|N|2480431|2480461|2480073|2480346|N|N|N|N|N| +2480439|AAAAAAAAHDJNFCAA|2079-02-09|2149|9346|717|2079|4|2|9|1|2079|717|9346|Thursday|2079Q1|N|N|N|2480431|2480461|2480074|2480347|N|N|N|N|N| +2480440|AAAAAAAAIDJNFCAA|2079-02-10|2149|9346|717|2079|5|2|10|1|2079|717|9346|Friday|2079Q1|N|Y|N|2480431|2480461|2480075|2480348|N|N|N|N|N| +2480441|AAAAAAAAJDJNFCAA|2079-02-11|2149|9346|717|2079|6|2|11|1|2079|717|9346|Saturday|2079Q1|N|Y|N|2480431|2480461|2480076|2480349|N|N|N|N|N| +2480442|AAAAAAAAKDJNFCAA|2079-02-12|2149|9346|717|2079|0|2|12|1|2079|717|9346|Sunday|2079Q1|N|N|N|2480431|2480461|2480077|2480350|N|N|N|N|N| +2480443|AAAAAAAALDJNFCAA|2079-02-13|2149|9346|717|2079|1|2|13|1|2079|717|9346|Monday|2079Q1|N|N|N|2480431|2480461|2480078|2480351|N|N|N|N|N| +2480444|AAAAAAAAMDJNFCAA|2079-02-14|2149|9347|717|2079|2|2|14|1|2079|717|9347|Tuesday|2079Q1|N|N|N|2480431|2480461|2480079|2480352|N|N|N|N|N| +2480445|AAAAAAAANDJNFCAA|2079-02-15|2149|9347|717|2079|3|2|15|1|2079|717|9347|Wednesday|2079Q1|N|N|N|2480431|2480461|2480080|2480353|N|N|N|N|N| +2480446|AAAAAAAAODJNFCAA|2079-02-16|2149|9347|717|2079|4|2|16|1|2079|717|9347|Thursday|2079Q1|N|N|N|2480431|2480461|2480081|2480354|N|N|N|N|N| +2480447|AAAAAAAAPDJNFCAA|2079-02-17|2149|9347|717|2079|5|2|17|1|2079|717|9347|Friday|2079Q1|N|Y|N|2480431|2480461|2480082|2480355|N|N|N|N|N| +2480448|AAAAAAAAAEJNFCAA|2079-02-18|2149|9347|717|2079|6|2|18|1|2079|717|9347|Saturday|2079Q1|N|Y|N|2480431|2480461|2480083|2480356|N|N|N|N|N| +2480449|AAAAAAAABEJNFCAA|2079-02-19|2149|9347|717|2079|0|2|19|1|2079|717|9347|Sunday|2079Q1|N|N|N|2480431|2480461|2480084|2480357|N|N|N|N|N| +2480450|AAAAAAAACEJNFCAA|2079-02-20|2149|9347|717|2079|1|2|20|1|2079|717|9347|Monday|2079Q1|N|N|N|2480431|2480461|2480085|2480358|N|N|N|N|N| +2480451|AAAAAAAADEJNFCAA|2079-02-21|2149|9348|717|2079|2|2|21|1|2079|717|9348|Tuesday|2079Q1|N|N|N|2480431|2480461|2480086|2480359|N|N|N|N|N| +2480452|AAAAAAAAEEJNFCAA|2079-02-22|2149|9348|717|2079|3|2|22|1|2079|717|9348|Wednesday|2079Q1|N|N|N|2480431|2480461|2480087|2480360|N|N|N|N|N| +2480453|AAAAAAAAFEJNFCAA|2079-02-23|2149|9348|717|2079|4|2|23|1|2079|717|9348|Thursday|2079Q1|N|N|N|2480431|2480461|2480088|2480361|N|N|N|N|N| +2480454|AAAAAAAAGEJNFCAA|2079-02-24|2149|9348|717|2079|5|2|24|1|2079|717|9348|Friday|2079Q1|N|Y|N|2480431|2480461|2480089|2480362|N|N|N|N|N| +2480455|AAAAAAAAHEJNFCAA|2079-02-25|2149|9348|717|2079|6|2|25|1|2079|717|9348|Saturday|2079Q1|N|Y|N|2480431|2480461|2480090|2480363|N|N|N|N|N| +2480456|AAAAAAAAIEJNFCAA|2079-02-26|2149|9348|717|2079|0|2|26|1|2079|717|9348|Sunday|2079Q1|N|N|N|2480431|2480461|2480091|2480364|N|N|N|N|N| +2480457|AAAAAAAAJEJNFCAA|2079-02-27|2149|9348|717|2079|1|2|27|1|2079|717|9348|Monday|2079Q1|N|N|N|2480431|2480461|2480092|2480365|N|N|N|N|N| +2480458|AAAAAAAAKEJNFCAA|2079-02-28|2149|9349|717|2079|2|2|28|1|2079|717|9349|Tuesday|2079Q1|N|N|N|2480431|2480461|2480093|2480366|N|N|N|N|N| +2480459|AAAAAAAALEJNFCAA|2079-03-01|2150|9349|718|2079|3|3|1|1|2079|718|9349|Wednesday|2079Q1|N|N|N|2480459|2480517|2480094|2480367|N|N|N|N|N| +2480460|AAAAAAAAMEJNFCAA|2079-03-02|2150|9349|718|2079|4|3|2|1|2079|718|9349|Thursday|2079Q1|N|N|N|2480459|2480517|2480095|2480368|N|N|N|N|N| +2480461|AAAAAAAANEJNFCAA|2079-03-03|2150|9349|718|2079|5|3|3|1|2079|718|9349|Friday|2079Q1|N|Y|N|2480459|2480517|2480096|2480369|N|N|N|N|N| +2480462|AAAAAAAAOEJNFCAA|2079-03-04|2150|9349|718|2079|6|3|4|1|2079|718|9349|Saturday|2079Q1|N|Y|N|2480459|2480517|2480097|2480370|N|N|N|N|N| +2480463|AAAAAAAAPEJNFCAA|2079-03-05|2150|9349|718|2079|0|3|5|1|2079|718|9349|Sunday|2079Q1|N|N|N|2480459|2480517|2480098|2480371|N|N|N|N|N| +2480464|AAAAAAAAAFJNFCAA|2079-03-06|2150|9349|718|2079|1|3|6|1|2079|718|9349|Monday|2079Q1|N|N|N|2480459|2480517|2480099|2480372|N|N|N|N|N| +2480465|AAAAAAAABFJNFCAA|2079-03-07|2150|9350|718|2079|2|3|7|1|2079|718|9350|Tuesday|2079Q1|N|N|N|2480459|2480517|2480100|2480373|N|N|N|N|N| +2480466|AAAAAAAACFJNFCAA|2079-03-08|2150|9350|718|2079|3|3|8|1|2079|718|9350|Wednesday|2079Q1|N|N|N|2480459|2480517|2480101|2480374|N|N|N|N|N| +2480467|AAAAAAAADFJNFCAA|2079-03-09|2150|9350|718|2079|4|3|9|1|2079|718|9350|Thursday|2079Q1|N|N|N|2480459|2480517|2480102|2480375|N|N|N|N|N| +2480468|AAAAAAAAEFJNFCAA|2079-03-10|2150|9350|718|2079|5|3|10|1|2079|718|9350|Friday|2079Q1|N|Y|N|2480459|2480517|2480103|2480376|N|N|N|N|N| +2480469|AAAAAAAAFFJNFCAA|2079-03-11|2150|9350|718|2079|6|3|11|1|2079|718|9350|Saturday|2079Q1|N|Y|N|2480459|2480517|2480104|2480377|N|N|N|N|N| +2480470|AAAAAAAAGFJNFCAA|2079-03-12|2150|9350|718|2079|0|3|12|1|2079|718|9350|Sunday|2079Q1|N|N|N|2480459|2480517|2480105|2480378|N|N|N|N|N| +2480471|AAAAAAAAHFJNFCAA|2079-03-13|2150|9350|718|2079|1|3|13|1|2079|718|9350|Monday|2079Q1|N|N|N|2480459|2480517|2480106|2480379|N|N|N|N|N| +2480472|AAAAAAAAIFJNFCAA|2079-03-14|2150|9351|718|2079|2|3|14|1|2079|718|9351|Tuesday|2079Q1|N|N|N|2480459|2480517|2480107|2480380|N|N|N|N|N| +2480473|AAAAAAAAJFJNFCAA|2079-03-15|2150|9351|718|2079|3|3|15|1|2079|718|9351|Wednesday|2079Q1|N|N|N|2480459|2480517|2480108|2480381|N|N|N|N|N| +2480474|AAAAAAAAKFJNFCAA|2079-03-16|2150|9351|718|2079|4|3|16|1|2079|718|9351|Thursday|2079Q1|N|N|N|2480459|2480517|2480109|2480382|N|N|N|N|N| +2480475|AAAAAAAALFJNFCAA|2079-03-17|2150|9351|718|2079|5|3|17|1|2079|718|9351|Friday|2079Q1|N|Y|N|2480459|2480517|2480110|2480383|N|N|N|N|N| +2480476|AAAAAAAAMFJNFCAA|2079-03-18|2150|9351|718|2079|6|3|18|1|2079|718|9351|Saturday|2079Q1|N|Y|N|2480459|2480517|2480111|2480384|N|N|N|N|N| +2480477|AAAAAAAANFJNFCAA|2079-03-19|2150|9351|718|2079|0|3|19|1|2079|718|9351|Sunday|2079Q1|N|N|N|2480459|2480517|2480112|2480385|N|N|N|N|N| +2480478|AAAAAAAAOFJNFCAA|2079-03-20|2150|9351|718|2079|1|3|20|1|2079|718|9351|Monday|2079Q1|N|N|N|2480459|2480517|2480113|2480386|N|N|N|N|N| +2480479|AAAAAAAAPFJNFCAA|2079-03-21|2150|9352|718|2079|2|3|21|1|2079|718|9352|Tuesday|2079Q1|N|N|N|2480459|2480517|2480114|2480387|N|N|N|N|N| +2480480|AAAAAAAAAGJNFCAA|2079-03-22|2150|9352|718|2079|3|3|22|1|2079|718|9352|Wednesday|2079Q1|N|N|N|2480459|2480517|2480115|2480388|N|N|N|N|N| +2480481|AAAAAAAABGJNFCAA|2079-03-23|2150|9352|718|2079|4|3|23|1|2079|718|9352|Thursday|2079Q1|N|N|N|2480459|2480517|2480116|2480389|N|N|N|N|N| +2480482|AAAAAAAACGJNFCAA|2079-03-24|2150|9352|718|2079|5|3|24|1|2079|718|9352|Friday|2079Q1|N|Y|N|2480459|2480517|2480117|2480390|N|N|N|N|N| +2480483|AAAAAAAADGJNFCAA|2079-03-25|2150|9352|718|2079|6|3|25|1|2079|718|9352|Saturday|2079Q1|N|Y|N|2480459|2480517|2480118|2480391|N|N|N|N|N| +2480484|AAAAAAAAEGJNFCAA|2079-03-26|2150|9352|718|2079|0|3|26|1|2079|718|9352|Sunday|2079Q1|N|N|N|2480459|2480517|2480119|2480392|N|N|N|N|N| +2480485|AAAAAAAAFGJNFCAA|2079-03-27|2150|9352|718|2079|1|3|27|1|2079|718|9352|Monday|2079Q1|N|N|N|2480459|2480517|2480120|2480393|N|N|N|N|N| +2480486|AAAAAAAAGGJNFCAA|2079-03-28|2150|9353|718|2079|2|3|28|1|2079|718|9353|Tuesday|2079Q1|N|N|N|2480459|2480517|2480121|2480394|N|N|N|N|N| +2480487|AAAAAAAAHGJNFCAA|2079-03-29|2150|9353|718|2079|3|3|29|1|2079|718|9353|Wednesday|2079Q1|N|N|N|2480459|2480517|2480122|2480395|N|N|N|N|N| +2480488|AAAAAAAAIGJNFCAA|2079-03-30|2150|9353|718|2079|4|3|30|1|2079|718|9353|Thursday|2079Q1|N|N|N|2480459|2480517|2480123|2480396|N|N|N|N|N| +2480489|AAAAAAAAJGJNFCAA|2079-03-31|2150|9353|718|2079|5|3|31|1|2079|718|9353|Friday|2079Q1|N|Y|N|2480459|2480517|2480124|2480397|N|N|N|N|N| +2480490|AAAAAAAAKGJNFCAA|2079-04-01|2151|9353|718|2079|6|4|1|1|2079|718|9353|Saturday|2079Q1|N|Y|N|2480490|2480579|2480125|2480400|N|N|N|N|N| +2480491|AAAAAAAALGJNFCAA|2079-04-02|2151|9353|718|2079|0|4|2|2|2079|718|9353|Sunday|2079Q2|N|N|N|2480490|2480579|2480126|2480401|N|N|N|N|N| +2480492|AAAAAAAAMGJNFCAA|2079-04-03|2151|9353|718|2079|1|4|3|2|2079|718|9353|Monday|2079Q2|N|N|N|2480490|2480579|2480127|2480402|N|N|N|N|N| +2480493|AAAAAAAANGJNFCAA|2079-04-04|2151|9354|718|2079|2|4|4|2|2079|718|9354|Tuesday|2079Q2|N|N|N|2480490|2480579|2480128|2480403|N|N|N|N|N| +2480494|AAAAAAAAOGJNFCAA|2079-04-05|2151|9354|718|2079|3|4|5|2|2079|718|9354|Wednesday|2079Q2|N|N|N|2480490|2480579|2480129|2480404|N|N|N|N|N| +2480495|AAAAAAAAPGJNFCAA|2079-04-06|2151|9354|718|2079|4|4|6|2|2079|718|9354|Thursday|2079Q2|N|N|N|2480490|2480579|2480130|2480405|N|N|N|N|N| +2480496|AAAAAAAAAHJNFCAA|2079-04-07|2151|9354|718|2079|5|4|7|2|2079|718|9354|Friday|2079Q2|N|Y|N|2480490|2480579|2480131|2480406|N|N|N|N|N| +2480497|AAAAAAAABHJNFCAA|2079-04-08|2151|9354|718|2079|6|4|8|2|2079|718|9354|Saturday|2079Q2|N|Y|N|2480490|2480579|2480132|2480407|N|N|N|N|N| +2480498|AAAAAAAACHJNFCAA|2079-04-09|2151|9354|718|2079|0|4|9|2|2079|718|9354|Sunday|2079Q2|N|N|N|2480490|2480579|2480133|2480408|N|N|N|N|N| +2480499|AAAAAAAADHJNFCAA|2079-04-10|2151|9354|718|2079|1|4|10|2|2079|718|9354|Monday|2079Q2|N|N|N|2480490|2480579|2480134|2480409|N|N|N|N|N| +2480500|AAAAAAAAEHJNFCAA|2079-04-11|2151|9355|718|2079|2|4|11|2|2079|718|9355|Tuesday|2079Q2|N|N|N|2480490|2480579|2480135|2480410|N|N|N|N|N| +2480501|AAAAAAAAFHJNFCAA|2079-04-12|2151|9355|718|2079|3|4|12|2|2079|718|9355|Wednesday|2079Q2|N|N|N|2480490|2480579|2480136|2480411|N|N|N|N|N| +2480502|AAAAAAAAGHJNFCAA|2079-04-13|2151|9355|718|2079|4|4|13|2|2079|718|9355|Thursday|2079Q2|N|N|N|2480490|2480579|2480137|2480412|N|N|N|N|N| +2480503|AAAAAAAAHHJNFCAA|2079-04-14|2151|9355|718|2079|5|4|14|2|2079|718|9355|Friday|2079Q2|N|Y|N|2480490|2480579|2480138|2480413|N|N|N|N|N| +2480504|AAAAAAAAIHJNFCAA|2079-04-15|2151|9355|718|2079|6|4|15|2|2079|718|9355|Saturday|2079Q2|N|Y|N|2480490|2480579|2480139|2480414|N|N|N|N|N| +2480505|AAAAAAAAJHJNFCAA|2079-04-16|2151|9355|718|2079|0|4|16|2|2079|718|9355|Sunday|2079Q2|N|N|N|2480490|2480579|2480140|2480415|N|N|N|N|N| +2480506|AAAAAAAAKHJNFCAA|2079-04-17|2151|9355|718|2079|1|4|17|2|2079|718|9355|Monday|2079Q2|N|N|N|2480490|2480579|2480141|2480416|N|N|N|N|N| +2480507|AAAAAAAALHJNFCAA|2079-04-18|2151|9356|718|2079|2|4|18|2|2079|718|9356|Tuesday|2079Q2|N|N|N|2480490|2480579|2480142|2480417|N|N|N|N|N| +2480508|AAAAAAAAMHJNFCAA|2079-04-19|2151|9356|718|2079|3|4|19|2|2079|718|9356|Wednesday|2079Q2|N|N|N|2480490|2480579|2480143|2480418|N|N|N|N|N| +2480509|AAAAAAAANHJNFCAA|2079-04-20|2151|9356|718|2079|4|4|20|2|2079|718|9356|Thursday|2079Q2|N|N|N|2480490|2480579|2480144|2480419|N|N|N|N|N| +2480510|AAAAAAAAOHJNFCAA|2079-04-21|2151|9356|718|2079|5|4|21|2|2079|718|9356|Friday|2079Q2|N|Y|N|2480490|2480579|2480145|2480420|N|N|N|N|N| +2480511|AAAAAAAAPHJNFCAA|2079-04-22|2151|9356|718|2079|6|4|22|2|2079|718|9356|Saturday|2079Q2|N|Y|N|2480490|2480579|2480146|2480421|N|N|N|N|N| +2480512|AAAAAAAAAIJNFCAA|2079-04-23|2151|9356|718|2079|0|4|23|2|2079|718|9356|Sunday|2079Q2|N|N|N|2480490|2480579|2480147|2480422|N|N|N|N|N| +2480513|AAAAAAAABIJNFCAA|2079-04-24|2151|9356|718|2079|1|4|24|2|2079|718|9356|Monday|2079Q2|N|N|N|2480490|2480579|2480148|2480423|N|N|N|N|N| +2480514|AAAAAAAACIJNFCAA|2079-04-25|2151|9357|718|2079|2|4|25|2|2079|718|9357|Tuesday|2079Q2|N|N|N|2480490|2480579|2480149|2480424|N|N|N|N|N| +2480515|AAAAAAAADIJNFCAA|2079-04-26|2151|9357|718|2079|3|4|26|2|2079|718|9357|Wednesday|2079Q2|N|N|N|2480490|2480579|2480150|2480425|N|N|N|N|N| +2480516|AAAAAAAAEIJNFCAA|2079-04-27|2151|9357|718|2079|4|4|27|2|2079|718|9357|Thursday|2079Q2|N|N|N|2480490|2480579|2480151|2480426|N|N|N|N|N| +2480517|AAAAAAAAFIJNFCAA|2079-04-28|2151|9357|718|2079|5|4|28|2|2079|718|9357|Friday|2079Q2|N|Y|N|2480490|2480579|2480152|2480427|N|N|N|N|N| +2480518|AAAAAAAAGIJNFCAA|2079-04-29|2151|9357|718|2079|6|4|29|2|2079|718|9357|Saturday|2079Q2|N|Y|N|2480490|2480579|2480153|2480428|N|N|N|N|N| +2480519|AAAAAAAAHIJNFCAA|2079-04-30|2151|9357|718|2079|0|4|30|2|2079|718|9357|Sunday|2079Q2|N|N|N|2480490|2480579|2480154|2480429|N|N|N|N|N| +2480520|AAAAAAAAIIJNFCAA|2079-05-01|2152|9357|718|2079|1|5|1|2|2079|718|9357|Monday|2079Q2|N|N|N|2480520|2480639|2480155|2480430|N|N|N|N|N| +2480521|AAAAAAAAJIJNFCAA|2079-05-02|2152|9358|718|2079|2|5|2|2|2079|718|9358|Tuesday|2079Q2|N|N|N|2480520|2480639|2480156|2480431|N|N|N|N|N| +2480522|AAAAAAAAKIJNFCAA|2079-05-03|2152|9358|718|2079|3|5|3|2|2079|718|9358|Wednesday|2079Q2|N|N|N|2480520|2480639|2480157|2480432|N|N|N|N|N| +2480523|AAAAAAAALIJNFCAA|2079-05-04|2152|9358|718|2079|4|5|4|2|2079|718|9358|Thursday|2079Q2|N|N|N|2480520|2480639|2480158|2480433|N|N|N|N|N| +2480524|AAAAAAAAMIJNFCAA|2079-05-05|2152|9358|718|2079|5|5|5|2|2079|718|9358|Friday|2079Q2|N|Y|N|2480520|2480639|2480159|2480434|N|N|N|N|N| +2480525|AAAAAAAANIJNFCAA|2079-05-06|2152|9358|718|2079|6|5|6|2|2079|718|9358|Saturday|2079Q2|N|Y|N|2480520|2480639|2480160|2480435|N|N|N|N|N| +2480526|AAAAAAAAOIJNFCAA|2079-05-07|2152|9358|718|2079|0|5|7|2|2079|718|9358|Sunday|2079Q2|N|N|N|2480520|2480639|2480161|2480436|N|N|N|N|N| +2480527|AAAAAAAAPIJNFCAA|2079-05-08|2152|9358|718|2079|1|5|8|2|2079|718|9358|Monday|2079Q2|N|N|N|2480520|2480639|2480162|2480437|N|N|N|N|N| +2480528|AAAAAAAAAJJNFCAA|2079-05-09|2152|9359|718|2079|2|5|9|2|2079|718|9359|Tuesday|2079Q2|N|N|N|2480520|2480639|2480163|2480438|N|N|N|N|N| +2480529|AAAAAAAABJJNFCAA|2079-05-10|2152|9359|718|2079|3|5|10|2|2079|718|9359|Wednesday|2079Q2|N|N|N|2480520|2480639|2480164|2480439|N|N|N|N|N| +2480530|AAAAAAAACJJNFCAA|2079-05-11|2152|9359|718|2079|4|5|11|2|2079|718|9359|Thursday|2079Q2|N|N|N|2480520|2480639|2480165|2480440|N|N|N|N|N| +2480531|AAAAAAAADJJNFCAA|2079-05-12|2152|9359|718|2079|5|5|12|2|2079|718|9359|Friday|2079Q2|N|Y|N|2480520|2480639|2480166|2480441|N|N|N|N|N| +2480532|AAAAAAAAEJJNFCAA|2079-05-13|2152|9359|718|2079|6|5|13|2|2079|718|9359|Saturday|2079Q2|N|Y|N|2480520|2480639|2480167|2480442|N|N|N|N|N| +2480533|AAAAAAAAFJJNFCAA|2079-05-14|2152|9359|718|2079|0|5|14|2|2079|718|9359|Sunday|2079Q2|N|N|N|2480520|2480639|2480168|2480443|N|N|N|N|N| +2480534|AAAAAAAAGJJNFCAA|2079-05-15|2152|9359|718|2079|1|5|15|2|2079|718|9359|Monday|2079Q2|N|N|N|2480520|2480639|2480169|2480444|N|N|N|N|N| +2480535|AAAAAAAAHJJNFCAA|2079-05-16|2152|9360|718|2079|2|5|16|2|2079|718|9360|Tuesday|2079Q2|N|N|N|2480520|2480639|2480170|2480445|N|N|N|N|N| +2480536|AAAAAAAAIJJNFCAA|2079-05-17|2152|9360|718|2079|3|5|17|2|2079|718|9360|Wednesday|2079Q2|N|N|N|2480520|2480639|2480171|2480446|N|N|N|N|N| +2480537|AAAAAAAAJJJNFCAA|2079-05-18|2152|9360|718|2079|4|5|18|2|2079|718|9360|Thursday|2079Q2|N|N|N|2480520|2480639|2480172|2480447|N|N|N|N|N| +2480538|AAAAAAAAKJJNFCAA|2079-05-19|2152|9360|718|2079|5|5|19|2|2079|718|9360|Friday|2079Q2|N|Y|N|2480520|2480639|2480173|2480448|N|N|N|N|N| +2480539|AAAAAAAALJJNFCAA|2079-05-20|2152|9360|718|2079|6|5|20|2|2079|718|9360|Saturday|2079Q2|N|Y|N|2480520|2480639|2480174|2480449|N|N|N|N|N| +2480540|AAAAAAAAMJJNFCAA|2079-05-21|2152|9360|718|2079|0|5|21|2|2079|718|9360|Sunday|2079Q2|N|N|N|2480520|2480639|2480175|2480450|N|N|N|N|N| +2480541|AAAAAAAANJJNFCAA|2079-05-22|2152|9360|718|2079|1|5|22|2|2079|718|9360|Monday|2079Q2|N|N|N|2480520|2480639|2480176|2480451|N|N|N|N|N| +2480542|AAAAAAAAOJJNFCAA|2079-05-23|2152|9361|718|2079|2|5|23|2|2079|718|9361|Tuesday|2079Q2|N|N|N|2480520|2480639|2480177|2480452|N|N|N|N|N| +2480543|AAAAAAAAPJJNFCAA|2079-05-24|2152|9361|718|2079|3|5|24|2|2079|718|9361|Wednesday|2079Q2|N|N|N|2480520|2480639|2480178|2480453|N|N|N|N|N| +2480544|AAAAAAAAAKJNFCAA|2079-05-25|2152|9361|718|2079|4|5|25|2|2079|718|9361|Thursday|2079Q2|N|N|N|2480520|2480639|2480179|2480454|N|N|N|N|N| +2480545|AAAAAAAABKJNFCAA|2079-05-26|2152|9361|718|2079|5|5|26|2|2079|718|9361|Friday|2079Q2|N|Y|N|2480520|2480639|2480180|2480455|N|N|N|N|N| +2480546|AAAAAAAACKJNFCAA|2079-05-27|2152|9361|718|2079|6|5|27|2|2079|718|9361|Saturday|2079Q2|N|Y|N|2480520|2480639|2480181|2480456|N|N|N|N|N| +2480547|AAAAAAAADKJNFCAA|2079-05-28|2152|9361|718|2079|0|5|28|2|2079|718|9361|Sunday|2079Q2|N|N|N|2480520|2480639|2480182|2480457|N|N|N|N|N| +2480548|AAAAAAAAEKJNFCAA|2079-05-29|2152|9361|718|2079|1|5|29|2|2079|718|9361|Monday|2079Q2|N|N|N|2480520|2480639|2480183|2480458|N|N|N|N|N| +2480549|AAAAAAAAFKJNFCAA|2079-05-30|2152|9362|718|2079|2|5|30|2|2079|718|9362|Tuesday|2079Q2|N|N|N|2480520|2480639|2480184|2480459|N|N|N|N|N| +2480550|AAAAAAAAGKJNFCAA|2079-05-31|2152|9362|718|2079|3|5|31|2|2079|718|9362|Wednesday|2079Q2|N|N|N|2480520|2480639|2480185|2480460|N|N|N|N|N| +2480551|AAAAAAAAHKJNFCAA|2079-06-01|2153|9362|719|2079|4|6|1|2|2079|719|9362|Thursday|2079Q2|N|N|N|2480551|2480701|2480186|2480461|N|N|N|N|N| +2480552|AAAAAAAAIKJNFCAA|2079-06-02|2153|9362|719|2079|5|6|2|2|2079|719|9362|Friday|2079Q2|N|Y|N|2480551|2480701|2480187|2480462|N|N|N|N|N| +2480553|AAAAAAAAJKJNFCAA|2079-06-03|2153|9362|719|2079|6|6|3|2|2079|719|9362|Saturday|2079Q2|N|Y|N|2480551|2480701|2480188|2480463|N|N|N|N|N| +2480554|AAAAAAAAKKJNFCAA|2079-06-04|2153|9362|719|2079|0|6|4|2|2079|719|9362|Sunday|2079Q2|N|N|N|2480551|2480701|2480189|2480464|N|N|N|N|N| +2480555|AAAAAAAALKJNFCAA|2079-06-05|2153|9362|719|2079|1|6|5|2|2079|719|9362|Monday|2079Q2|N|N|N|2480551|2480701|2480190|2480465|N|N|N|N|N| +2480556|AAAAAAAAMKJNFCAA|2079-06-06|2153|9363|719|2079|2|6|6|2|2079|719|9363|Tuesday|2079Q2|N|N|N|2480551|2480701|2480191|2480466|N|N|N|N|N| +2480557|AAAAAAAANKJNFCAA|2079-06-07|2153|9363|719|2079|3|6|7|2|2079|719|9363|Wednesday|2079Q2|N|N|N|2480551|2480701|2480192|2480467|N|N|N|N|N| +2480558|AAAAAAAAOKJNFCAA|2079-06-08|2153|9363|719|2079|4|6|8|2|2079|719|9363|Thursday|2079Q2|N|N|N|2480551|2480701|2480193|2480468|N|N|N|N|N| +2480559|AAAAAAAAPKJNFCAA|2079-06-09|2153|9363|719|2079|5|6|9|2|2079|719|9363|Friday|2079Q2|N|Y|N|2480551|2480701|2480194|2480469|N|N|N|N|N| +2480560|AAAAAAAAALJNFCAA|2079-06-10|2153|9363|719|2079|6|6|10|2|2079|719|9363|Saturday|2079Q2|N|Y|N|2480551|2480701|2480195|2480470|N|N|N|N|N| +2480561|AAAAAAAABLJNFCAA|2079-06-11|2153|9363|719|2079|0|6|11|2|2079|719|9363|Sunday|2079Q2|N|N|N|2480551|2480701|2480196|2480471|N|N|N|N|N| +2480562|AAAAAAAACLJNFCAA|2079-06-12|2153|9363|719|2079|1|6|12|2|2079|719|9363|Monday|2079Q2|N|N|N|2480551|2480701|2480197|2480472|N|N|N|N|N| +2480563|AAAAAAAADLJNFCAA|2079-06-13|2153|9364|719|2079|2|6|13|2|2079|719|9364|Tuesday|2079Q2|N|N|N|2480551|2480701|2480198|2480473|N|N|N|N|N| +2480564|AAAAAAAAELJNFCAA|2079-06-14|2153|9364|719|2079|3|6|14|2|2079|719|9364|Wednesday|2079Q2|N|N|N|2480551|2480701|2480199|2480474|N|N|N|N|N| +2480565|AAAAAAAAFLJNFCAA|2079-06-15|2153|9364|719|2079|4|6|15|2|2079|719|9364|Thursday|2079Q2|N|N|N|2480551|2480701|2480200|2480475|N|N|N|N|N| +2480566|AAAAAAAAGLJNFCAA|2079-06-16|2153|9364|719|2079|5|6|16|2|2079|719|9364|Friday|2079Q2|N|Y|N|2480551|2480701|2480201|2480476|N|N|N|N|N| +2480567|AAAAAAAAHLJNFCAA|2079-06-17|2153|9364|719|2079|6|6|17|2|2079|719|9364|Saturday|2079Q2|N|Y|N|2480551|2480701|2480202|2480477|N|N|N|N|N| +2480568|AAAAAAAAILJNFCAA|2079-06-18|2153|9364|719|2079|0|6|18|2|2079|719|9364|Sunday|2079Q2|N|N|N|2480551|2480701|2480203|2480478|N|N|N|N|N| +2480569|AAAAAAAAJLJNFCAA|2079-06-19|2153|9364|719|2079|1|6|19|2|2079|719|9364|Monday|2079Q2|N|N|N|2480551|2480701|2480204|2480479|N|N|N|N|N| +2480570|AAAAAAAAKLJNFCAA|2079-06-20|2153|9365|719|2079|2|6|20|2|2079|719|9365|Tuesday|2079Q2|N|N|N|2480551|2480701|2480205|2480480|N|N|N|N|N| +2480571|AAAAAAAALLJNFCAA|2079-06-21|2153|9365|719|2079|3|6|21|2|2079|719|9365|Wednesday|2079Q2|N|N|N|2480551|2480701|2480206|2480481|N|N|N|N|N| +2480572|AAAAAAAAMLJNFCAA|2079-06-22|2153|9365|719|2079|4|6|22|2|2079|719|9365|Thursday|2079Q2|N|N|N|2480551|2480701|2480207|2480482|N|N|N|N|N| +2480573|AAAAAAAANLJNFCAA|2079-06-23|2153|9365|719|2079|5|6|23|2|2079|719|9365|Friday|2079Q2|N|Y|N|2480551|2480701|2480208|2480483|N|N|N|N|N| +2480574|AAAAAAAAOLJNFCAA|2079-06-24|2153|9365|719|2079|6|6|24|2|2079|719|9365|Saturday|2079Q2|N|Y|N|2480551|2480701|2480209|2480484|N|N|N|N|N| +2480575|AAAAAAAAPLJNFCAA|2079-06-25|2153|9365|719|2079|0|6|25|2|2079|719|9365|Sunday|2079Q2|N|N|N|2480551|2480701|2480210|2480485|N|N|N|N|N| +2480576|AAAAAAAAAMJNFCAA|2079-06-26|2153|9365|719|2079|1|6|26|2|2079|719|9365|Monday|2079Q2|N|N|N|2480551|2480701|2480211|2480486|N|N|N|N|N| +2480577|AAAAAAAABMJNFCAA|2079-06-27|2153|9366|719|2079|2|6|27|2|2079|719|9366|Tuesday|2079Q2|N|N|N|2480551|2480701|2480212|2480487|N|N|N|N|N| +2480578|AAAAAAAACMJNFCAA|2079-06-28|2153|9366|719|2079|3|6|28|2|2079|719|9366|Wednesday|2079Q2|N|N|N|2480551|2480701|2480213|2480488|N|N|N|N|N| +2480579|AAAAAAAADMJNFCAA|2079-06-29|2153|9366|719|2079|4|6|29|2|2079|719|9366|Thursday|2079Q2|N|N|N|2480551|2480701|2480214|2480489|N|N|N|N|N| +2480580|AAAAAAAAEMJNFCAA|2079-06-30|2153|9366|719|2079|5|6|30|2|2079|719|9366|Friday|2079Q2|N|Y|N|2480551|2480701|2480215|2480490|N|N|N|N|N| +2480581|AAAAAAAAFMJNFCAA|2079-07-01|2154|9366|719|2079|6|7|1|2|2079|719|9366|Saturday|2079Q2|N|Y|N|2480581|2480761|2480216|2480490|N|N|N|N|N| +2480582|AAAAAAAAGMJNFCAA|2079-07-02|2154|9366|719|2079|0|7|2|3|2079|719|9366|Sunday|2079Q3|N|N|N|2480581|2480761|2480217|2480491|N|N|N|N|N| +2480583|AAAAAAAAHMJNFCAA|2079-07-03|2154|9366|719|2079|1|7|3|3|2079|719|9366|Monday|2079Q3|N|N|N|2480581|2480761|2480218|2480492|N|N|N|N|N| +2480584|AAAAAAAAIMJNFCAA|2079-07-04|2154|9367|719|2079|2|7|4|3|2079|719|9367|Tuesday|2079Q3|N|N|N|2480581|2480761|2480219|2480493|N|N|N|N|N| +2480585|AAAAAAAAJMJNFCAA|2079-07-05|2154|9367|719|2079|3|7|5|3|2079|719|9367|Wednesday|2079Q3|Y|N|N|2480581|2480761|2480220|2480494|N|N|N|N|N| +2480586|AAAAAAAAKMJNFCAA|2079-07-06|2154|9367|719|2079|4|7|6|3|2079|719|9367|Thursday|2079Q3|N|N|Y|2480581|2480761|2480221|2480495|N|N|N|N|N| +2480587|AAAAAAAALMJNFCAA|2079-07-07|2154|9367|719|2079|5|7|7|3|2079|719|9367|Friday|2079Q3|N|Y|N|2480581|2480761|2480222|2480496|N|N|N|N|N| +2480588|AAAAAAAAMMJNFCAA|2079-07-08|2154|9367|719|2079|6|7|8|3|2079|719|9367|Saturday|2079Q3|N|Y|N|2480581|2480761|2480223|2480497|N|N|N|N|N| +2480589|AAAAAAAANMJNFCAA|2079-07-09|2154|9367|719|2079|0|7|9|3|2079|719|9367|Sunday|2079Q3|N|N|N|2480581|2480761|2480224|2480498|N|N|N|N|N| +2480590|AAAAAAAAOMJNFCAA|2079-07-10|2154|9367|719|2079|1|7|10|3|2079|719|9367|Monday|2079Q3|N|N|N|2480581|2480761|2480225|2480499|N|N|N|N|N| +2480591|AAAAAAAAPMJNFCAA|2079-07-11|2154|9368|719|2079|2|7|11|3|2079|719|9368|Tuesday|2079Q3|N|N|N|2480581|2480761|2480226|2480500|N|N|N|N|N| +2480592|AAAAAAAAANJNFCAA|2079-07-12|2154|9368|719|2079|3|7|12|3|2079|719|9368|Wednesday|2079Q3|N|N|N|2480581|2480761|2480227|2480501|N|N|N|N|N| +2480593|AAAAAAAABNJNFCAA|2079-07-13|2154|9368|719|2079|4|7|13|3|2079|719|9368|Thursday|2079Q3|N|N|N|2480581|2480761|2480228|2480502|N|N|N|N|N| +2480594|AAAAAAAACNJNFCAA|2079-07-14|2154|9368|719|2079|5|7|14|3|2079|719|9368|Friday|2079Q3|N|Y|N|2480581|2480761|2480229|2480503|N|N|N|N|N| +2480595|AAAAAAAADNJNFCAA|2079-07-15|2154|9368|719|2079|6|7|15|3|2079|719|9368|Saturday|2079Q3|N|Y|N|2480581|2480761|2480230|2480504|N|N|N|N|N| +2480596|AAAAAAAAENJNFCAA|2079-07-16|2154|9368|719|2079|0|7|16|3|2079|719|9368|Sunday|2079Q3|N|N|N|2480581|2480761|2480231|2480505|N|N|N|N|N| +2480597|AAAAAAAAFNJNFCAA|2079-07-17|2154|9368|719|2079|1|7|17|3|2079|719|9368|Monday|2079Q3|N|N|N|2480581|2480761|2480232|2480506|N|N|N|N|N| +2480598|AAAAAAAAGNJNFCAA|2079-07-18|2154|9369|719|2079|2|7|18|3|2079|719|9369|Tuesday|2079Q3|N|N|N|2480581|2480761|2480233|2480507|N|N|N|N|N| +2480599|AAAAAAAAHNJNFCAA|2079-07-19|2154|9369|719|2079|3|7|19|3|2079|719|9369|Wednesday|2079Q3|N|N|N|2480581|2480761|2480234|2480508|N|N|N|N|N| +2480600|AAAAAAAAINJNFCAA|2079-07-20|2154|9369|719|2079|4|7|20|3|2079|719|9369|Thursday|2079Q3|N|N|N|2480581|2480761|2480235|2480509|N|N|N|N|N| +2480601|AAAAAAAAJNJNFCAA|2079-07-21|2154|9369|719|2079|5|7|21|3|2079|719|9369|Friday|2079Q3|N|Y|N|2480581|2480761|2480236|2480510|N|N|N|N|N| +2480602|AAAAAAAAKNJNFCAA|2079-07-22|2154|9369|719|2079|6|7|22|3|2079|719|9369|Saturday|2079Q3|N|Y|N|2480581|2480761|2480237|2480511|N|N|N|N|N| +2480603|AAAAAAAALNJNFCAA|2079-07-23|2154|9369|719|2079|0|7|23|3|2079|719|9369|Sunday|2079Q3|N|N|N|2480581|2480761|2480238|2480512|N|N|N|N|N| +2480604|AAAAAAAAMNJNFCAA|2079-07-24|2154|9369|719|2079|1|7|24|3|2079|719|9369|Monday|2079Q3|N|N|N|2480581|2480761|2480239|2480513|N|N|N|N|N| +2480605|AAAAAAAANNJNFCAA|2079-07-25|2154|9370|719|2079|2|7|25|3|2079|719|9370|Tuesday|2079Q3|N|N|N|2480581|2480761|2480240|2480514|N|N|N|N|N| +2480606|AAAAAAAAONJNFCAA|2079-07-26|2154|9370|719|2079|3|7|26|3|2079|719|9370|Wednesday|2079Q3|N|N|N|2480581|2480761|2480241|2480515|N|N|N|N|N| +2480607|AAAAAAAAPNJNFCAA|2079-07-27|2154|9370|719|2079|4|7|27|3|2079|719|9370|Thursday|2079Q3|N|N|N|2480581|2480761|2480242|2480516|N|N|N|N|N| +2480608|AAAAAAAAAOJNFCAA|2079-07-28|2154|9370|719|2079|5|7|28|3|2079|719|9370|Friday|2079Q3|N|Y|N|2480581|2480761|2480243|2480517|N|N|N|N|N| +2480609|AAAAAAAABOJNFCAA|2079-07-29|2154|9370|719|2079|6|7|29|3|2079|719|9370|Saturday|2079Q3|N|Y|N|2480581|2480761|2480244|2480518|N|N|N|N|N| +2480610|AAAAAAAACOJNFCAA|2079-07-30|2154|9370|719|2079|0|7|30|3|2079|719|9370|Sunday|2079Q3|N|N|N|2480581|2480761|2480245|2480519|N|N|N|N|N| +2480611|AAAAAAAADOJNFCAA|2079-07-31|2154|9370|719|2079|1|7|31|3|2079|719|9370|Monday|2079Q3|N|N|N|2480581|2480761|2480246|2480520|N|N|N|N|N| +2480612|AAAAAAAAEOJNFCAA|2079-08-01|2155|9371|719|2079|2|8|1|3|2079|719|9371|Tuesday|2079Q3|N|N|N|2480612|2480823|2480247|2480521|N|N|N|N|N| +2480613|AAAAAAAAFOJNFCAA|2079-08-02|2155|9371|719|2079|3|8|2|3|2079|719|9371|Wednesday|2079Q3|N|N|N|2480612|2480823|2480248|2480522|N|N|N|N|N| +2480614|AAAAAAAAGOJNFCAA|2079-08-03|2155|9371|719|2079|4|8|3|3|2079|719|9371|Thursday|2079Q3|N|N|N|2480612|2480823|2480249|2480523|N|N|N|N|N| +2480615|AAAAAAAAHOJNFCAA|2079-08-04|2155|9371|719|2079|5|8|4|3|2079|719|9371|Friday|2079Q3|N|Y|N|2480612|2480823|2480250|2480524|N|N|N|N|N| +2480616|AAAAAAAAIOJNFCAA|2079-08-05|2155|9371|719|2079|6|8|5|3|2079|719|9371|Saturday|2079Q3|N|Y|N|2480612|2480823|2480251|2480525|N|N|N|N|N| +2480617|AAAAAAAAJOJNFCAA|2079-08-06|2155|9371|719|2079|0|8|6|3|2079|719|9371|Sunday|2079Q3|N|N|N|2480612|2480823|2480252|2480526|N|N|N|N|N| +2480618|AAAAAAAAKOJNFCAA|2079-08-07|2155|9371|719|2079|1|8|7|3|2079|719|9371|Monday|2079Q3|N|N|N|2480612|2480823|2480253|2480527|N|N|N|N|N| +2480619|AAAAAAAALOJNFCAA|2079-08-08|2155|9372|719|2079|2|8|8|3|2079|719|9372|Tuesday|2079Q3|N|N|N|2480612|2480823|2480254|2480528|N|N|N|N|N| +2480620|AAAAAAAAMOJNFCAA|2079-08-09|2155|9372|719|2079|3|8|9|3|2079|719|9372|Wednesday|2079Q3|N|N|N|2480612|2480823|2480255|2480529|N|N|N|N|N| +2480621|AAAAAAAANOJNFCAA|2079-08-10|2155|9372|719|2079|4|8|10|3|2079|719|9372|Thursday|2079Q3|N|N|N|2480612|2480823|2480256|2480530|N|N|N|N|N| +2480622|AAAAAAAAOOJNFCAA|2079-08-11|2155|9372|719|2079|5|8|11|3|2079|719|9372|Friday|2079Q3|N|Y|N|2480612|2480823|2480257|2480531|N|N|N|N|N| +2480623|AAAAAAAAPOJNFCAA|2079-08-12|2155|9372|719|2079|6|8|12|3|2079|719|9372|Saturday|2079Q3|N|Y|N|2480612|2480823|2480258|2480532|N|N|N|N|N| +2480624|AAAAAAAAAPJNFCAA|2079-08-13|2155|9372|719|2079|0|8|13|3|2079|719|9372|Sunday|2079Q3|N|N|N|2480612|2480823|2480259|2480533|N|N|N|N|N| +2480625|AAAAAAAABPJNFCAA|2079-08-14|2155|9372|719|2079|1|8|14|3|2079|719|9372|Monday|2079Q3|N|N|N|2480612|2480823|2480260|2480534|N|N|N|N|N| +2480626|AAAAAAAACPJNFCAA|2079-08-15|2155|9373|719|2079|2|8|15|3|2079|719|9373|Tuesday|2079Q3|N|N|N|2480612|2480823|2480261|2480535|N|N|N|N|N| +2480627|AAAAAAAADPJNFCAA|2079-08-16|2155|9373|719|2079|3|8|16|3|2079|719|9373|Wednesday|2079Q3|N|N|N|2480612|2480823|2480262|2480536|N|N|N|N|N| +2480628|AAAAAAAAEPJNFCAA|2079-08-17|2155|9373|719|2079|4|8|17|3|2079|719|9373|Thursday|2079Q3|N|N|N|2480612|2480823|2480263|2480537|N|N|N|N|N| +2480629|AAAAAAAAFPJNFCAA|2079-08-18|2155|9373|719|2079|5|8|18|3|2079|719|9373|Friday|2079Q3|N|Y|N|2480612|2480823|2480264|2480538|N|N|N|N|N| +2480630|AAAAAAAAGPJNFCAA|2079-08-19|2155|9373|719|2079|6|8|19|3|2079|719|9373|Saturday|2079Q3|N|Y|N|2480612|2480823|2480265|2480539|N|N|N|N|N| +2480631|AAAAAAAAHPJNFCAA|2079-08-20|2155|9373|719|2079|0|8|20|3|2079|719|9373|Sunday|2079Q3|N|N|N|2480612|2480823|2480266|2480540|N|N|N|N|N| +2480632|AAAAAAAAIPJNFCAA|2079-08-21|2155|9373|719|2079|1|8|21|3|2079|719|9373|Monday|2079Q3|N|N|N|2480612|2480823|2480267|2480541|N|N|N|N|N| +2480633|AAAAAAAAJPJNFCAA|2079-08-22|2155|9374|719|2079|2|8|22|3|2079|719|9374|Tuesday|2079Q3|N|N|N|2480612|2480823|2480268|2480542|N|N|N|N|N| +2480634|AAAAAAAAKPJNFCAA|2079-08-23|2155|9374|719|2079|3|8|23|3|2079|719|9374|Wednesday|2079Q3|N|N|N|2480612|2480823|2480269|2480543|N|N|N|N|N| +2480635|AAAAAAAALPJNFCAA|2079-08-24|2155|9374|719|2079|4|8|24|3|2079|719|9374|Thursday|2079Q3|N|N|N|2480612|2480823|2480270|2480544|N|N|N|N|N| +2480636|AAAAAAAAMPJNFCAA|2079-08-25|2155|9374|719|2079|5|8|25|3|2079|719|9374|Friday|2079Q3|N|Y|N|2480612|2480823|2480271|2480545|N|N|N|N|N| +2480637|AAAAAAAANPJNFCAA|2079-08-26|2155|9374|719|2079|6|8|26|3|2079|719|9374|Saturday|2079Q3|N|Y|N|2480612|2480823|2480272|2480546|N|N|N|N|N| +2480638|AAAAAAAAOPJNFCAA|2079-08-27|2155|9374|719|2079|0|8|27|3|2079|719|9374|Sunday|2079Q3|N|N|N|2480612|2480823|2480273|2480547|N|N|N|N|N| +2480639|AAAAAAAAPPJNFCAA|2079-08-28|2155|9374|719|2079|1|8|28|3|2079|719|9374|Monday|2079Q3|N|N|N|2480612|2480823|2480274|2480548|N|N|N|N|N| +2480640|AAAAAAAAAAKNFCAA|2079-08-29|2155|9375|719|2079|2|8|29|3|2079|719|9375|Tuesday|2079Q3|N|N|N|2480612|2480823|2480275|2480549|N|N|N|N|N| +2480641|AAAAAAAABAKNFCAA|2079-08-30|2155|9375|719|2079|3|8|30|3|2079|719|9375|Wednesday|2079Q3|N|N|N|2480612|2480823|2480276|2480550|N|N|N|N|N| +2480642|AAAAAAAACAKNFCAA|2079-08-31|2155|9375|719|2079|4|8|31|3|2079|719|9375|Thursday|2079Q3|N|N|N|2480612|2480823|2480277|2480551|N|N|N|N|N| +2480643|AAAAAAAADAKNFCAA|2079-09-01|2156|9375|720|2079|5|9|1|3|2079|720|9375|Friday|2079Q3|N|Y|N|2480643|2480885|2480278|2480552|N|N|N|N|N| +2480644|AAAAAAAAEAKNFCAA|2079-09-02|2156|9375|720|2079|6|9|2|3|2079|720|9375|Saturday|2079Q3|N|Y|N|2480643|2480885|2480279|2480553|N|N|N|N|N| +2480645|AAAAAAAAFAKNFCAA|2079-09-03|2156|9375|720|2079|0|9|3|3|2079|720|9375|Sunday|2079Q3|N|N|N|2480643|2480885|2480280|2480554|N|N|N|N|N| +2480646|AAAAAAAAGAKNFCAA|2079-09-04|2156|9375|720|2079|1|9|4|3|2079|720|9375|Monday|2079Q3|N|N|N|2480643|2480885|2480281|2480555|N|N|N|N|N| +2480647|AAAAAAAAHAKNFCAA|2079-09-05|2156|9376|720|2079|2|9|5|3|2079|720|9376|Tuesday|2079Q3|N|N|N|2480643|2480885|2480282|2480556|N|N|N|N|N| +2480648|AAAAAAAAIAKNFCAA|2079-09-06|2156|9376|720|2079|3|9|6|3|2079|720|9376|Wednesday|2079Q3|N|N|N|2480643|2480885|2480283|2480557|N|N|N|N|N| +2480649|AAAAAAAAJAKNFCAA|2079-09-07|2156|9376|720|2079|4|9|7|3|2079|720|9376|Thursday|2079Q3|N|N|N|2480643|2480885|2480284|2480558|N|N|N|N|N| +2480650|AAAAAAAAKAKNFCAA|2079-09-08|2156|9376|720|2079|5|9|8|3|2079|720|9376|Friday|2079Q3|N|Y|N|2480643|2480885|2480285|2480559|N|N|N|N|N| +2480651|AAAAAAAALAKNFCAA|2079-09-09|2156|9376|720|2079|6|9|9|3|2079|720|9376|Saturday|2079Q3|N|Y|N|2480643|2480885|2480286|2480560|N|N|N|N|N| +2480652|AAAAAAAAMAKNFCAA|2079-09-10|2156|9376|720|2079|0|9|10|3|2079|720|9376|Sunday|2079Q3|N|N|N|2480643|2480885|2480287|2480561|N|N|N|N|N| +2480653|AAAAAAAANAKNFCAA|2079-09-11|2156|9376|720|2079|1|9|11|3|2079|720|9376|Monday|2079Q3|N|N|N|2480643|2480885|2480288|2480562|N|N|N|N|N| +2480654|AAAAAAAAOAKNFCAA|2079-09-12|2156|9377|720|2079|2|9|12|3|2079|720|9377|Tuesday|2079Q3|N|N|N|2480643|2480885|2480289|2480563|N|N|N|N|N| +2480655|AAAAAAAAPAKNFCAA|2079-09-13|2156|9377|720|2079|3|9|13|3|2079|720|9377|Wednesday|2079Q3|N|N|N|2480643|2480885|2480290|2480564|N|N|N|N|N| +2480656|AAAAAAAAABKNFCAA|2079-09-14|2156|9377|720|2079|4|9|14|3|2079|720|9377|Thursday|2079Q3|N|N|N|2480643|2480885|2480291|2480565|N|N|N|N|N| +2480657|AAAAAAAABBKNFCAA|2079-09-15|2156|9377|720|2079|5|9|15|3|2079|720|9377|Friday|2079Q3|N|Y|N|2480643|2480885|2480292|2480566|N|N|N|N|N| +2480658|AAAAAAAACBKNFCAA|2079-09-16|2156|9377|720|2079|6|9|16|3|2079|720|9377|Saturday|2079Q3|N|Y|N|2480643|2480885|2480293|2480567|N|N|N|N|N| +2480659|AAAAAAAADBKNFCAA|2079-09-17|2156|9377|720|2079|0|9|17|3|2079|720|9377|Sunday|2079Q3|N|N|N|2480643|2480885|2480294|2480568|N|N|N|N|N| +2480660|AAAAAAAAEBKNFCAA|2079-09-18|2156|9377|720|2079|1|9|18|3|2079|720|9377|Monday|2079Q3|N|N|N|2480643|2480885|2480295|2480569|N|N|N|N|N| +2480661|AAAAAAAAFBKNFCAA|2079-09-19|2156|9378|720|2079|2|9|19|3|2079|720|9378|Tuesday|2079Q3|N|N|N|2480643|2480885|2480296|2480570|N|N|N|N|N| +2480662|AAAAAAAAGBKNFCAA|2079-09-20|2156|9378|720|2079|3|9|20|3|2079|720|9378|Wednesday|2079Q3|N|N|N|2480643|2480885|2480297|2480571|N|N|N|N|N| +2480663|AAAAAAAAHBKNFCAA|2079-09-21|2156|9378|720|2079|4|9|21|3|2079|720|9378|Thursday|2079Q3|N|N|N|2480643|2480885|2480298|2480572|N|N|N|N|N| +2480664|AAAAAAAAIBKNFCAA|2079-09-22|2156|9378|720|2079|5|9|22|3|2079|720|9378|Friday|2079Q3|N|Y|N|2480643|2480885|2480299|2480573|N|N|N|N|N| +2480665|AAAAAAAAJBKNFCAA|2079-09-23|2156|9378|720|2079|6|9|23|3|2079|720|9378|Saturday|2079Q3|N|Y|N|2480643|2480885|2480300|2480574|N|N|N|N|N| +2480666|AAAAAAAAKBKNFCAA|2079-09-24|2156|9378|720|2079|0|9|24|3|2079|720|9378|Sunday|2079Q3|N|N|N|2480643|2480885|2480301|2480575|N|N|N|N|N| +2480667|AAAAAAAALBKNFCAA|2079-09-25|2156|9378|720|2079|1|9|25|3|2079|720|9378|Monday|2079Q3|N|N|N|2480643|2480885|2480302|2480576|N|N|N|N|N| +2480668|AAAAAAAAMBKNFCAA|2079-09-26|2156|9379|720|2079|2|9|26|3|2079|720|9379|Tuesday|2079Q3|N|N|N|2480643|2480885|2480303|2480577|N|N|N|N|N| +2480669|AAAAAAAANBKNFCAA|2079-09-27|2156|9379|720|2079|3|9|27|3|2079|720|9379|Wednesday|2079Q3|N|N|N|2480643|2480885|2480304|2480578|N|N|N|N|N| +2480670|AAAAAAAAOBKNFCAA|2079-09-28|2156|9379|720|2079|4|9|28|3|2079|720|9379|Thursday|2079Q3|N|N|N|2480643|2480885|2480305|2480579|N|N|N|N|N| +2480671|AAAAAAAAPBKNFCAA|2079-09-29|2156|9379|720|2079|5|9|29|3|2079|720|9379|Friday|2079Q3|N|Y|N|2480643|2480885|2480306|2480580|N|N|N|N|N| +2480672|AAAAAAAAACKNFCAA|2079-09-30|2156|9379|720|2079|6|9|30|3|2079|720|9379|Saturday|2079Q3|N|Y|N|2480643|2480885|2480307|2480581|N|N|N|N|N| +2480673|AAAAAAAABCKNFCAA|2079-10-01|2157|9379|720|2079|0|10|1|3|2079|720|9379|Sunday|2079Q3|N|N|N|2480673|2480945|2480308|2480581|N|N|N|N|N| +2480674|AAAAAAAACCKNFCAA|2079-10-02|2157|9379|720|2079|1|10|2|4|2079|720|9379|Monday|2079Q4|N|N|N|2480673|2480945|2480309|2480582|N|N|N|N|N| +2480675|AAAAAAAADCKNFCAA|2079-10-03|2157|9380|720|2079|2|10|3|4|2079|720|9380|Tuesday|2079Q4|N|N|N|2480673|2480945|2480310|2480583|N|N|N|N|N| +2480676|AAAAAAAAECKNFCAA|2079-10-04|2157|9380|720|2079|3|10|4|4|2079|720|9380|Wednesday|2079Q4|N|N|N|2480673|2480945|2480311|2480584|N|N|N|N|N| +2480677|AAAAAAAAFCKNFCAA|2079-10-05|2157|9380|720|2079|4|10|5|4|2079|720|9380|Thursday|2079Q4|N|N|N|2480673|2480945|2480312|2480585|N|N|N|N|N| +2480678|AAAAAAAAGCKNFCAA|2079-10-06|2157|9380|720|2079|5|10|6|4|2079|720|9380|Friday|2079Q4|N|Y|N|2480673|2480945|2480313|2480586|N|N|N|N|N| +2480679|AAAAAAAAHCKNFCAA|2079-10-07|2157|9380|720|2079|6|10|7|4|2079|720|9380|Saturday|2079Q4|N|Y|N|2480673|2480945|2480314|2480587|N|N|N|N|N| +2480680|AAAAAAAAICKNFCAA|2079-10-08|2157|9380|720|2079|0|10|8|4|2079|720|9380|Sunday|2079Q4|N|N|N|2480673|2480945|2480315|2480588|N|N|N|N|N| +2480681|AAAAAAAAJCKNFCAA|2079-10-09|2157|9380|720|2079|1|10|9|4|2079|720|9380|Monday|2079Q4|N|N|N|2480673|2480945|2480316|2480589|N|N|N|N|N| +2480682|AAAAAAAAKCKNFCAA|2079-10-10|2157|9381|720|2079|2|10|10|4|2079|720|9381|Tuesday|2079Q4|N|N|N|2480673|2480945|2480317|2480590|N|N|N|N|N| +2480683|AAAAAAAALCKNFCAA|2079-10-11|2157|9381|720|2079|3|10|11|4|2079|720|9381|Wednesday|2079Q4|N|N|N|2480673|2480945|2480318|2480591|N|N|N|N|N| +2480684|AAAAAAAAMCKNFCAA|2079-10-12|2157|9381|720|2079|4|10|12|4|2079|720|9381|Thursday|2079Q4|N|N|N|2480673|2480945|2480319|2480592|N|N|N|N|N| +2480685|AAAAAAAANCKNFCAA|2079-10-13|2157|9381|720|2079|5|10|13|4|2079|720|9381|Friday|2079Q4|N|Y|N|2480673|2480945|2480320|2480593|N|N|N|N|N| +2480686|AAAAAAAAOCKNFCAA|2079-10-14|2157|9381|720|2079|6|10|14|4|2079|720|9381|Saturday|2079Q4|N|Y|N|2480673|2480945|2480321|2480594|N|N|N|N|N| +2480687|AAAAAAAAPCKNFCAA|2079-10-15|2157|9381|720|2079|0|10|15|4|2079|720|9381|Sunday|2079Q4|N|N|N|2480673|2480945|2480322|2480595|N|N|N|N|N| +2480688|AAAAAAAAADKNFCAA|2079-10-16|2157|9381|720|2079|1|10|16|4|2079|720|9381|Monday|2079Q4|N|N|N|2480673|2480945|2480323|2480596|N|N|N|N|N| +2480689|AAAAAAAABDKNFCAA|2079-10-17|2157|9382|720|2079|2|10|17|4|2079|720|9382|Tuesday|2079Q4|N|N|N|2480673|2480945|2480324|2480597|N|N|N|N|N| +2480690|AAAAAAAACDKNFCAA|2079-10-18|2157|9382|720|2079|3|10|18|4|2079|720|9382|Wednesday|2079Q4|N|N|N|2480673|2480945|2480325|2480598|N|N|N|N|N| +2480691|AAAAAAAADDKNFCAA|2079-10-19|2157|9382|720|2079|4|10|19|4|2079|720|9382|Thursday|2079Q4|N|N|N|2480673|2480945|2480326|2480599|N|N|N|N|N| +2480692|AAAAAAAAEDKNFCAA|2079-10-20|2157|9382|720|2079|5|10|20|4|2079|720|9382|Friday|2079Q4|N|Y|N|2480673|2480945|2480327|2480600|N|N|N|N|N| +2480693|AAAAAAAAFDKNFCAA|2079-10-21|2157|9382|720|2079|6|10|21|4|2079|720|9382|Saturday|2079Q4|N|Y|N|2480673|2480945|2480328|2480601|N|N|N|N|N| +2480694|AAAAAAAAGDKNFCAA|2079-10-22|2157|9382|720|2079|0|10|22|4|2079|720|9382|Sunday|2079Q4|N|N|N|2480673|2480945|2480329|2480602|N|N|N|N|N| +2480695|AAAAAAAAHDKNFCAA|2079-10-23|2157|9382|720|2079|1|10|23|4|2079|720|9382|Monday|2079Q4|N|N|N|2480673|2480945|2480330|2480603|N|N|N|N|N| +2480696|AAAAAAAAIDKNFCAA|2079-10-24|2157|9383|720|2079|2|10|24|4|2079|720|9383|Tuesday|2079Q4|N|N|N|2480673|2480945|2480331|2480604|N|N|N|N|N| +2480697|AAAAAAAAJDKNFCAA|2079-10-25|2157|9383|720|2079|3|10|25|4|2079|720|9383|Wednesday|2079Q4|N|N|N|2480673|2480945|2480332|2480605|N|N|N|N|N| +2480698|AAAAAAAAKDKNFCAA|2079-10-26|2157|9383|720|2079|4|10|26|4|2079|720|9383|Thursday|2079Q4|N|N|N|2480673|2480945|2480333|2480606|N|N|N|N|N| +2480699|AAAAAAAALDKNFCAA|2079-10-27|2157|9383|720|2079|5|10|27|4|2079|720|9383|Friday|2079Q4|N|Y|N|2480673|2480945|2480334|2480607|N|N|N|N|N| +2480700|AAAAAAAAMDKNFCAA|2079-10-28|2157|9383|720|2079|6|10|28|4|2079|720|9383|Saturday|2079Q4|N|Y|N|2480673|2480945|2480335|2480608|N|N|N|N|N| +2480701|AAAAAAAANDKNFCAA|2079-10-29|2157|9383|720|2079|0|10|29|4|2079|720|9383|Sunday|2079Q4|N|N|N|2480673|2480945|2480336|2480609|N|N|N|N|N| +2480702|AAAAAAAAODKNFCAA|2079-10-30|2157|9383|720|2079|1|10|30|4|2079|720|9383|Monday|2079Q4|N|N|N|2480673|2480945|2480337|2480610|N|N|N|N|N| +2480703|AAAAAAAAPDKNFCAA|2079-10-31|2157|9384|720|2079|2|10|31|4|2079|720|9384|Tuesday|2079Q4|N|N|N|2480673|2480945|2480338|2480611|N|N|N|N|N| +2480704|AAAAAAAAAEKNFCAA|2079-11-01|2158|9384|720|2079|3|11|1|4|2079|720|9384|Wednesday|2079Q4|N|N|N|2480704|2481007|2480339|2480612|N|N|N|N|N| +2480705|AAAAAAAABEKNFCAA|2079-11-02|2158|9384|720|2079|4|11|2|4|2079|720|9384|Thursday|2079Q4|N|N|N|2480704|2481007|2480340|2480613|N|N|N|N|N| +2480706|AAAAAAAACEKNFCAA|2079-11-03|2158|9384|720|2079|5|11|3|4|2079|720|9384|Friday|2079Q4|N|Y|N|2480704|2481007|2480341|2480614|N|N|N|N|N| +2480707|AAAAAAAADEKNFCAA|2079-11-04|2158|9384|720|2079|6|11|4|4|2079|720|9384|Saturday|2079Q4|N|Y|N|2480704|2481007|2480342|2480615|N|N|N|N|N| +2480708|AAAAAAAAEEKNFCAA|2079-11-05|2158|9384|720|2079|0|11|5|4|2079|720|9384|Sunday|2079Q4|N|N|N|2480704|2481007|2480343|2480616|N|N|N|N|N| +2480709|AAAAAAAAFEKNFCAA|2079-11-06|2158|9384|720|2079|1|11|6|4|2079|720|9384|Monday|2079Q4|N|N|N|2480704|2481007|2480344|2480617|N|N|N|N|N| +2480710|AAAAAAAAGEKNFCAA|2079-11-07|2158|9385|720|2079|2|11|7|4|2079|720|9385|Tuesday|2079Q4|N|N|N|2480704|2481007|2480345|2480618|N|N|N|N|N| +2480711|AAAAAAAAHEKNFCAA|2079-11-08|2158|9385|720|2079|3|11|8|4|2079|720|9385|Wednesday|2079Q4|N|N|N|2480704|2481007|2480346|2480619|N|N|N|N|N| +2480712|AAAAAAAAIEKNFCAA|2079-11-09|2158|9385|720|2079|4|11|9|4|2079|720|9385|Thursday|2079Q4|N|N|N|2480704|2481007|2480347|2480620|N|N|N|N|N| +2480713|AAAAAAAAJEKNFCAA|2079-11-10|2158|9385|720|2079|5|11|10|4|2079|720|9385|Friday|2079Q4|N|Y|N|2480704|2481007|2480348|2480621|N|N|N|N|N| +2480714|AAAAAAAAKEKNFCAA|2079-11-11|2158|9385|720|2079|6|11|11|4|2079|720|9385|Saturday|2079Q4|N|Y|N|2480704|2481007|2480349|2480622|N|N|N|N|N| +2480715|AAAAAAAALEKNFCAA|2079-11-12|2158|9385|720|2079|0|11|12|4|2079|720|9385|Sunday|2079Q4|N|N|N|2480704|2481007|2480350|2480623|N|N|N|N|N| +2480716|AAAAAAAAMEKNFCAA|2079-11-13|2158|9385|720|2079|1|11|13|4|2079|720|9385|Monday|2079Q4|N|N|N|2480704|2481007|2480351|2480624|N|N|N|N|N| +2480717|AAAAAAAANEKNFCAA|2079-11-14|2158|9386|720|2079|2|11|14|4|2079|720|9386|Tuesday|2079Q4|N|N|N|2480704|2481007|2480352|2480625|N|N|N|N|N| +2480718|AAAAAAAAOEKNFCAA|2079-11-15|2158|9386|720|2079|3|11|15|4|2079|720|9386|Wednesday|2079Q4|N|N|N|2480704|2481007|2480353|2480626|N|N|N|N|N| +2480719|AAAAAAAAPEKNFCAA|2079-11-16|2158|9386|720|2079|4|11|16|4|2079|720|9386|Thursday|2079Q4|N|N|N|2480704|2481007|2480354|2480627|N|N|N|N|N| +2480720|AAAAAAAAAFKNFCAA|2079-11-17|2158|9386|720|2079|5|11|17|4|2079|720|9386|Friday|2079Q4|N|Y|N|2480704|2481007|2480355|2480628|N|N|N|N|N| +2480721|AAAAAAAABFKNFCAA|2079-11-18|2158|9386|720|2079|6|11|18|4|2079|720|9386|Saturday|2079Q4|N|Y|N|2480704|2481007|2480356|2480629|N|N|N|N|N| +2480722|AAAAAAAACFKNFCAA|2079-11-19|2158|9386|720|2079|0|11|19|4|2079|720|9386|Sunday|2079Q4|N|N|N|2480704|2481007|2480357|2480630|N|N|N|N|N| +2480723|AAAAAAAADFKNFCAA|2079-11-20|2158|9386|720|2079|1|11|20|4|2079|720|9386|Monday|2079Q4|N|N|N|2480704|2481007|2480358|2480631|N|N|N|N|N| +2480724|AAAAAAAAEFKNFCAA|2079-11-21|2158|9387|720|2079|2|11|21|4|2079|720|9387|Tuesday|2079Q4|N|N|N|2480704|2481007|2480359|2480632|N|N|N|N|N| +2480725|AAAAAAAAFFKNFCAA|2079-11-22|2158|9387|720|2079|3|11|22|4|2079|720|9387|Wednesday|2079Q4|N|N|N|2480704|2481007|2480360|2480633|N|N|N|N|N| +2480726|AAAAAAAAGFKNFCAA|2079-11-23|2158|9387|720|2079|4|11|23|4|2079|720|9387|Thursday|2079Q4|N|N|N|2480704|2481007|2480361|2480634|N|N|N|N|N| +2480727|AAAAAAAAHFKNFCAA|2079-11-24|2158|9387|720|2079|5|11|24|4|2079|720|9387|Friday|2079Q4|N|Y|N|2480704|2481007|2480362|2480635|N|N|N|N|N| +2480728|AAAAAAAAIFKNFCAA|2079-11-25|2158|9387|720|2079|6|11|25|4|2079|720|9387|Saturday|2079Q4|N|Y|N|2480704|2481007|2480363|2480636|N|N|N|N|N| +2480729|AAAAAAAAJFKNFCAA|2079-11-26|2158|9387|720|2079|0|11|26|4|2079|720|9387|Sunday|2079Q4|N|N|N|2480704|2481007|2480364|2480637|N|N|N|N|N| +2480730|AAAAAAAAKFKNFCAA|2079-11-27|2158|9387|720|2079|1|11|27|4|2079|720|9387|Monday|2079Q4|N|N|N|2480704|2481007|2480365|2480638|N|N|N|N|N| +2480731|AAAAAAAALFKNFCAA|2079-11-28|2158|9388|720|2079|2|11|28|4|2079|720|9388|Tuesday|2079Q4|N|N|N|2480704|2481007|2480366|2480639|N|N|N|N|N| +2480732|AAAAAAAAMFKNFCAA|2079-11-29|2158|9388|720|2079|3|11|29|4|2079|720|9388|Wednesday|2079Q4|N|N|N|2480704|2481007|2480367|2480640|N|N|N|N|N| +2480733|AAAAAAAANFKNFCAA|2079-11-30|2158|9388|720|2079|4|11|30|4|2079|720|9388|Thursday|2079Q4|N|N|N|2480704|2481007|2480368|2480641|N|N|N|N|N| +2480734|AAAAAAAAOFKNFCAA|2079-12-01|2159|9388|721|2079|5|12|1|4|2079|721|9388|Friday|2079Q4|N|Y|N|2480734|2481067|2480369|2480642|N|N|N|N|N| +2480735|AAAAAAAAPFKNFCAA|2079-12-02|2159|9388|721|2079|6|12|2|4|2079|721|9388|Saturday|2079Q4|N|Y|N|2480734|2481067|2480370|2480643|N|N|N|N|N| +2480736|AAAAAAAAAGKNFCAA|2079-12-03|2159|9388|721|2079|0|12|3|4|2079|721|9388|Sunday|2079Q4|N|N|N|2480734|2481067|2480371|2480644|N|N|N|N|N| +2480737|AAAAAAAABGKNFCAA|2079-12-04|2159|9388|721|2079|1|12|4|4|2079|721|9388|Monday|2079Q4|N|N|N|2480734|2481067|2480372|2480645|N|N|N|N|N| +2480738|AAAAAAAACGKNFCAA|2079-12-05|2159|9389|721|2079|2|12|5|4|2079|721|9389|Tuesday|2079Q4|N|N|N|2480734|2481067|2480373|2480646|N|N|N|N|N| +2480739|AAAAAAAADGKNFCAA|2079-12-06|2159|9389|721|2079|3|12|6|4|2079|721|9389|Wednesday|2079Q4|N|N|N|2480734|2481067|2480374|2480647|N|N|N|N|N| +2480740|AAAAAAAAEGKNFCAA|2079-12-07|2159|9389|721|2079|4|12|7|4|2079|721|9389|Thursday|2079Q4|N|N|N|2480734|2481067|2480375|2480648|N|N|N|N|N| +2480741|AAAAAAAAFGKNFCAA|2079-12-08|2159|9389|721|2079|5|12|8|4|2079|721|9389|Friday|2079Q4|N|Y|N|2480734|2481067|2480376|2480649|N|N|N|N|N| +2480742|AAAAAAAAGGKNFCAA|2079-12-09|2159|9389|721|2079|6|12|9|4|2079|721|9389|Saturday|2079Q4|N|Y|N|2480734|2481067|2480377|2480650|N|N|N|N|N| +2480743|AAAAAAAAHGKNFCAA|2079-12-10|2159|9389|721|2079|0|12|10|4|2079|721|9389|Sunday|2079Q4|N|N|N|2480734|2481067|2480378|2480651|N|N|N|N|N| +2480744|AAAAAAAAIGKNFCAA|2079-12-11|2159|9389|721|2079|1|12|11|4|2079|721|9389|Monday|2079Q4|N|N|N|2480734|2481067|2480379|2480652|N|N|N|N|N| +2480745|AAAAAAAAJGKNFCAA|2079-12-12|2159|9390|721|2079|2|12|12|4|2079|721|9390|Tuesday|2079Q4|N|N|N|2480734|2481067|2480380|2480653|N|N|N|N|N| +2480746|AAAAAAAAKGKNFCAA|2079-12-13|2159|9390|721|2079|3|12|13|4|2079|721|9390|Wednesday|2079Q4|N|N|N|2480734|2481067|2480381|2480654|N|N|N|N|N| +2480747|AAAAAAAALGKNFCAA|2079-12-14|2159|9390|721|2079|4|12|14|4|2079|721|9390|Thursday|2079Q4|N|N|N|2480734|2481067|2480382|2480655|N|N|N|N|N| +2480748|AAAAAAAAMGKNFCAA|2079-12-15|2159|9390|721|2079|5|12|15|4|2079|721|9390|Friday|2079Q4|N|Y|N|2480734|2481067|2480383|2480656|N|N|N|N|N| +2480749|AAAAAAAANGKNFCAA|2079-12-16|2159|9390|721|2079|6|12|16|4|2079|721|9390|Saturday|2079Q4|N|Y|N|2480734|2481067|2480384|2480657|N|N|N|N|N| +2480750|AAAAAAAAOGKNFCAA|2079-12-17|2159|9390|721|2079|0|12|17|4|2079|721|9390|Sunday|2079Q4|N|N|N|2480734|2481067|2480385|2480658|N|N|N|N|N| +2480751|AAAAAAAAPGKNFCAA|2079-12-18|2159|9390|721|2079|1|12|18|4|2079|721|9390|Monday|2079Q4|N|N|N|2480734|2481067|2480386|2480659|N|N|N|N|N| +2480752|AAAAAAAAAHKNFCAA|2079-12-19|2159|9391|721|2079|2|12|19|4|2079|721|9391|Tuesday|2079Q4|N|N|N|2480734|2481067|2480387|2480660|N|N|N|N|N| +2480753|AAAAAAAABHKNFCAA|2079-12-20|2159|9391|721|2079|3|12|20|4|2079|721|9391|Wednesday|2079Q4|N|N|N|2480734|2481067|2480388|2480661|N|N|N|N|N| +2480754|AAAAAAAACHKNFCAA|2079-12-21|2159|9391|721|2079|4|12|21|4|2079|721|9391|Thursday|2079Q4|N|N|N|2480734|2481067|2480389|2480662|N|N|N|N|N| +2480755|AAAAAAAADHKNFCAA|2079-12-22|2159|9391|721|2079|5|12|22|4|2079|721|9391|Friday|2079Q4|N|Y|N|2480734|2481067|2480390|2480663|N|N|N|N|N| +2480756|AAAAAAAAEHKNFCAA|2079-12-23|2159|9391|721|2079|6|12|23|4|2079|721|9391|Saturday|2079Q4|N|Y|N|2480734|2481067|2480391|2480664|N|N|N|N|N| +2480757|AAAAAAAAFHKNFCAA|2079-12-24|2159|9391|721|2079|0|12|24|4|2079|721|9391|Sunday|2079Q4|N|N|N|2480734|2481067|2480392|2480665|N|N|N|N|N| +2480758|AAAAAAAAGHKNFCAA|2079-12-25|2159|9391|721|2079|1|12|25|4|2079|721|9391|Monday|2079Q4|N|N|N|2480734|2481067|2480393|2480666|N|N|N|N|N| +2480759|AAAAAAAAHHKNFCAA|2079-12-26|2159|9392|721|2079|2|12|26|4|2079|721|9392|Tuesday|2079Q4|Y|N|N|2480734|2481067|2480394|2480667|N|N|N|N|N| +2480760|AAAAAAAAIHKNFCAA|2079-12-27|2159|9392|721|2079|3|12|27|4|2079|721|9392|Wednesday|2079Q4|N|N|Y|2480734|2481067|2480395|2480668|N|N|N|N|N| +2480761|AAAAAAAAJHKNFCAA|2079-12-28|2159|9392|721|2079|4|12|28|4|2079|721|9392|Thursday|2079Q4|N|N|N|2480734|2481067|2480396|2480669|N|N|N|N|N| +2480762|AAAAAAAAKHKNFCAA|2079-12-29|2159|9392|721|2079|5|12|29|4|2079|721|9392|Friday|2079Q4|N|Y|N|2480734|2481067|2480397|2480670|N|N|N|N|N| +2480763|AAAAAAAALHKNFCAA|2079-12-30|2159|9392|721|2079|6|12|30|4|2079|721|9392|Saturday|2079Q4|N|Y|N|2480734|2481067|2480398|2480671|N|N|N|N|N| +2480764|AAAAAAAAMHKNFCAA|2079-12-31|2159|9392|721|2079|0|12|31|4|2079|721|9392|Sunday|2079Q4|N|N|N|2480734|2481067|2480399|2480672|N|N|N|N|N| +2480765|AAAAAAAANHKNFCAA|2080-01-01|2160|9392|721|2080|1|1|1|1|2080|721|9392|Monday|2080Q1|Y|N|N|2480765|2480764|2480400|2480673|N|N|N|N|N| +2480766|AAAAAAAAOHKNFCAA|2080-01-02|2160|9393|721|2080|2|1|2|1|2080|721|9393|Tuesday|2080Q1|N|N|Y|2480765|2480764|2480401|2480674|N|N|N|N|N| +2480767|AAAAAAAAPHKNFCAA|2080-01-03|2160|9393|721|2080|3|1|3|1|2080|721|9393|Wednesday|2080Q1|N|N|N|2480765|2480764|2480402|2480675|N|N|N|N|N| +2480768|AAAAAAAAAIKNFCAA|2080-01-04|2160|9393|721|2080|4|1|4|1|2080|721|9393|Thursday|2080Q1|N|N|N|2480765|2480764|2480403|2480676|N|N|N|N|N| +2480769|AAAAAAAABIKNFCAA|2080-01-05|2160|9393|721|2080|5|1|5|1|2080|721|9393|Friday|2080Q1|N|Y|N|2480765|2480764|2480404|2480677|N|N|N|N|N| +2480770|AAAAAAAACIKNFCAA|2080-01-06|2160|9393|721|2080|6|1|6|1|2080|721|9393|Saturday|2080Q1|N|Y|N|2480765|2480764|2480405|2480678|N|N|N|N|N| +2480771|AAAAAAAADIKNFCAA|2080-01-07|2160|9393|721|2080|0|1|7|1|2080|721|9393|Sunday|2080Q1|N|N|N|2480765|2480764|2480406|2480679|N|N|N|N|N| +2480772|AAAAAAAAEIKNFCAA|2080-01-08|2160|9393|721|2080|1|1|8|1|2080|721|9393|Monday|2080Q1|N|N|N|2480765|2480764|2480407|2480680|N|N|N|N|N| +2480773|AAAAAAAAFIKNFCAA|2080-01-09|2160|9394|721|2080|2|1|9|1|2080|721|9394|Tuesday|2080Q1|N|N|N|2480765|2480764|2480408|2480681|N|N|N|N|N| +2480774|AAAAAAAAGIKNFCAA|2080-01-10|2160|9394|721|2080|3|1|10|1|2080|721|9394|Wednesday|2080Q1|N|N|N|2480765|2480764|2480409|2480682|N|N|N|N|N| +2480775|AAAAAAAAHIKNFCAA|2080-01-11|2160|9394|721|2080|4|1|11|1|2080|721|9394|Thursday|2080Q1|N|N|N|2480765|2480764|2480410|2480683|N|N|N|N|N| +2480776|AAAAAAAAIIKNFCAA|2080-01-12|2160|9394|721|2080|5|1|12|1|2080|721|9394|Friday|2080Q1|N|Y|N|2480765|2480764|2480411|2480684|N|N|N|N|N| +2480777|AAAAAAAAJIKNFCAA|2080-01-13|2160|9394|721|2080|6|1|13|1|2080|721|9394|Saturday|2080Q1|N|Y|N|2480765|2480764|2480412|2480685|N|N|N|N|N| +2480778|AAAAAAAAKIKNFCAA|2080-01-14|2160|9394|721|2080|0|1|14|1|2080|721|9394|Sunday|2080Q1|N|N|N|2480765|2480764|2480413|2480686|N|N|N|N|N| +2480779|AAAAAAAALIKNFCAA|2080-01-15|2160|9394|721|2080|1|1|15|1|2080|721|9394|Monday|2080Q1|N|N|N|2480765|2480764|2480414|2480687|N|N|N|N|N| +2480780|AAAAAAAAMIKNFCAA|2080-01-16|2160|9395|721|2080|2|1|16|1|2080|721|9395|Tuesday|2080Q1|N|N|N|2480765|2480764|2480415|2480688|N|N|N|N|N| +2480781|AAAAAAAANIKNFCAA|2080-01-17|2160|9395|721|2080|3|1|17|1|2080|721|9395|Wednesday|2080Q1|N|N|N|2480765|2480764|2480416|2480689|N|N|N|N|N| +2480782|AAAAAAAAOIKNFCAA|2080-01-18|2160|9395|721|2080|4|1|18|1|2080|721|9395|Thursday|2080Q1|N|N|N|2480765|2480764|2480417|2480690|N|N|N|N|N| +2480783|AAAAAAAAPIKNFCAA|2080-01-19|2160|9395|721|2080|5|1|19|1|2080|721|9395|Friday|2080Q1|N|Y|N|2480765|2480764|2480418|2480691|N|N|N|N|N| +2480784|AAAAAAAAAJKNFCAA|2080-01-20|2160|9395|721|2080|6|1|20|1|2080|721|9395|Saturday|2080Q1|N|Y|N|2480765|2480764|2480419|2480692|N|N|N|N|N| +2480785|AAAAAAAABJKNFCAA|2080-01-21|2160|9395|721|2080|0|1|21|1|2080|721|9395|Sunday|2080Q1|N|N|N|2480765|2480764|2480420|2480693|N|N|N|N|N| +2480786|AAAAAAAACJKNFCAA|2080-01-22|2160|9395|721|2080|1|1|22|1|2080|721|9395|Monday|2080Q1|N|N|N|2480765|2480764|2480421|2480694|N|N|N|N|N| +2480787|AAAAAAAADJKNFCAA|2080-01-23|2160|9396|721|2080|2|1|23|1|2080|721|9396|Tuesday|2080Q1|N|N|N|2480765|2480764|2480422|2480695|N|N|N|N|N| +2480788|AAAAAAAAEJKNFCAA|2080-01-24|2160|9396|721|2080|3|1|24|1|2080|721|9396|Wednesday|2080Q1|N|N|N|2480765|2480764|2480423|2480696|N|N|N|N|N| +2480789|AAAAAAAAFJKNFCAA|2080-01-25|2160|9396|721|2080|4|1|25|1|2080|721|9396|Thursday|2080Q1|N|N|N|2480765|2480764|2480424|2480697|N|N|N|N|N| +2480790|AAAAAAAAGJKNFCAA|2080-01-26|2160|9396|721|2080|5|1|26|1|2080|721|9396|Friday|2080Q1|N|Y|N|2480765|2480764|2480425|2480698|N|N|N|N|N| +2480791|AAAAAAAAHJKNFCAA|2080-01-27|2160|9396|721|2080|6|1|27|1|2080|721|9396|Saturday|2080Q1|N|Y|N|2480765|2480764|2480426|2480699|N|N|N|N|N| +2480792|AAAAAAAAIJKNFCAA|2080-01-28|2160|9396|721|2080|0|1|28|1|2080|721|9396|Sunday|2080Q1|N|N|N|2480765|2480764|2480427|2480700|N|N|N|N|N| +2480793|AAAAAAAAJJKNFCAA|2080-01-29|2160|9396|721|2080|1|1|29|1|2080|721|9396|Monday|2080Q1|N|N|N|2480765|2480764|2480428|2480701|N|N|N|N|N| +2480794|AAAAAAAAKJKNFCAA|2080-01-30|2160|9397|721|2080|2|1|30|1|2080|721|9397|Tuesday|2080Q1|N|N|N|2480765|2480764|2480429|2480702|N|N|N|N|N| +2480795|AAAAAAAALJKNFCAA|2080-01-31|2160|9397|721|2080|3|1|31|1|2080|721|9397|Wednesday|2080Q1|N|N|N|2480765|2480764|2480430|2480703|N|N|N|N|N| +2480796|AAAAAAAAMJKNFCAA|2080-02-01|2161|9397|721|2080|4|2|1|1|2080|721|9397|Thursday|2080Q1|N|N|N|2480796|2480826|2480431|2480704|N|N|N|N|N| +2480797|AAAAAAAANJKNFCAA|2080-02-02|2161|9397|721|2080|5|2|2|1|2080|721|9397|Friday|2080Q1|N|Y|N|2480796|2480826|2480432|2480705|N|N|N|N|N| +2480798|AAAAAAAAOJKNFCAA|2080-02-03|2161|9397|721|2080|6|2|3|1|2080|721|9397|Saturday|2080Q1|N|Y|N|2480796|2480826|2480433|2480706|N|N|N|N|N| +2480799|AAAAAAAAPJKNFCAA|2080-02-04|2161|9397|721|2080|0|2|4|1|2080|721|9397|Sunday|2080Q1|N|N|N|2480796|2480826|2480434|2480707|N|N|N|N|N| +2480800|AAAAAAAAAKKNFCAA|2080-02-05|2161|9397|721|2080|1|2|5|1|2080|721|9397|Monday|2080Q1|N|N|N|2480796|2480826|2480435|2480708|N|N|N|N|N| +2480801|AAAAAAAABKKNFCAA|2080-02-06|2161|9398|721|2080|2|2|6|1|2080|721|9398|Tuesday|2080Q1|N|N|N|2480796|2480826|2480436|2480709|N|N|N|N|N| +2480802|AAAAAAAACKKNFCAA|2080-02-07|2161|9398|721|2080|3|2|7|1|2080|721|9398|Wednesday|2080Q1|N|N|N|2480796|2480826|2480437|2480710|N|N|N|N|N| +2480803|AAAAAAAADKKNFCAA|2080-02-08|2161|9398|721|2080|4|2|8|1|2080|721|9398|Thursday|2080Q1|N|N|N|2480796|2480826|2480438|2480711|N|N|N|N|N| +2480804|AAAAAAAAEKKNFCAA|2080-02-09|2161|9398|721|2080|5|2|9|1|2080|721|9398|Friday|2080Q1|N|Y|N|2480796|2480826|2480439|2480712|N|N|N|N|N| +2480805|AAAAAAAAFKKNFCAA|2080-02-10|2161|9398|721|2080|6|2|10|1|2080|721|9398|Saturday|2080Q1|N|Y|N|2480796|2480826|2480440|2480713|N|N|N|N|N| +2480806|AAAAAAAAGKKNFCAA|2080-02-11|2161|9398|721|2080|0|2|11|1|2080|721|9398|Sunday|2080Q1|N|N|N|2480796|2480826|2480441|2480714|N|N|N|N|N| +2480807|AAAAAAAAHKKNFCAA|2080-02-12|2161|9398|721|2080|1|2|12|1|2080|721|9398|Monday|2080Q1|N|N|N|2480796|2480826|2480442|2480715|N|N|N|N|N| +2480808|AAAAAAAAIKKNFCAA|2080-02-13|2161|9399|721|2080|2|2|13|1|2080|721|9399|Tuesday|2080Q1|N|N|N|2480796|2480826|2480443|2480716|N|N|N|N|N| +2480809|AAAAAAAAJKKNFCAA|2080-02-14|2161|9399|721|2080|3|2|14|1|2080|721|9399|Wednesday|2080Q1|N|N|N|2480796|2480826|2480444|2480717|N|N|N|N|N| +2480810|AAAAAAAAKKKNFCAA|2080-02-15|2161|9399|721|2080|4|2|15|1|2080|721|9399|Thursday|2080Q1|N|N|N|2480796|2480826|2480445|2480718|N|N|N|N|N| +2480811|AAAAAAAALKKNFCAA|2080-02-16|2161|9399|721|2080|5|2|16|1|2080|721|9399|Friday|2080Q1|N|Y|N|2480796|2480826|2480446|2480719|N|N|N|N|N| +2480812|AAAAAAAAMKKNFCAA|2080-02-17|2161|9399|721|2080|6|2|17|1|2080|721|9399|Saturday|2080Q1|N|Y|N|2480796|2480826|2480447|2480720|N|N|N|N|N| +2480813|AAAAAAAANKKNFCAA|2080-02-18|2161|9399|721|2080|0|2|18|1|2080|721|9399|Sunday|2080Q1|N|N|N|2480796|2480826|2480448|2480721|N|N|N|N|N| +2480814|AAAAAAAAOKKNFCAA|2080-02-19|2161|9399|721|2080|1|2|19|1|2080|721|9399|Monday|2080Q1|N|N|N|2480796|2480826|2480449|2480722|N|N|N|N|N| +2480815|AAAAAAAAPKKNFCAA|2080-02-20|2161|9400|721|2080|2|2|20|1|2080|721|9400|Tuesday|2080Q1|N|N|N|2480796|2480826|2480450|2480723|N|N|N|N|N| +2480816|AAAAAAAAALKNFCAA|2080-02-21|2161|9400|721|2080|3|2|21|1|2080|721|9400|Wednesday|2080Q1|N|N|N|2480796|2480826|2480451|2480724|N|N|N|N|N| +2480817|AAAAAAAABLKNFCAA|2080-02-22|2161|9400|721|2080|4|2|22|1|2080|721|9400|Thursday|2080Q1|N|N|N|2480796|2480826|2480452|2480725|N|N|N|N|N| +2480818|AAAAAAAACLKNFCAA|2080-02-23|2161|9400|721|2080|5|2|23|1|2080|721|9400|Friday|2080Q1|N|Y|N|2480796|2480826|2480453|2480726|N|N|N|N|N| +2480819|AAAAAAAADLKNFCAA|2080-02-24|2161|9400|721|2080|6|2|24|1|2080|721|9400|Saturday|2080Q1|N|Y|N|2480796|2480826|2480454|2480727|N|N|N|N|N| +2480820|AAAAAAAAELKNFCAA|2080-02-25|2161|9400|721|2080|0|2|25|1|2080|721|9400|Sunday|2080Q1|N|N|N|2480796|2480826|2480455|2480728|N|N|N|N|N| +2480821|AAAAAAAAFLKNFCAA|2080-02-26|2161|9400|721|2080|1|2|26|1|2080|721|9400|Monday|2080Q1|N|N|N|2480796|2480826|2480456|2480729|N|N|N|N|N| +2480822|AAAAAAAAGLKNFCAA|2080-02-27|2161|9401|721|2080|2|2|27|1|2080|721|9401|Tuesday|2080Q1|N|N|N|2480796|2480826|2480457|2480730|N|N|N|N|N| +2480823|AAAAAAAAHLKNFCAA|2080-02-28|2161|9401|721|2080|3|2|28|1|2080|721|9401|Wednesday|2080Q1|N|N|N|2480796|2480826|2480458|2480731|N|N|N|N|N| +2480824|AAAAAAAAILKNFCAA|2080-02-29|2161|9401|721|2080|4|2|29|1|2080|721|9401|Thursday|2080Q1|N|N|N|2480796|2480826|2480458|2480732|N|N|N|N|N| +2480825|AAAAAAAAJLKNFCAA|2080-03-01|2162|9401|722|2080|5|3|1|1|2080|722|9401|Friday|2080Q1|N|Y|N|2480825|2480884|2480459|2480733|N|N|N|N|N| +2480826|AAAAAAAAKLKNFCAA|2080-03-02|2162|9401|722|2080|6|3|2|1|2080|722|9401|Saturday|2080Q1|N|Y|N|2480825|2480884|2480460|2480734|N|N|N|N|N| +2480827|AAAAAAAALLKNFCAA|2080-03-03|2162|9401|722|2080|0|3|3|1|2080|722|9401|Sunday|2080Q1|N|N|N|2480825|2480884|2480461|2480735|N|N|N|N|N| +2480828|AAAAAAAAMLKNFCAA|2080-03-04|2162|9401|722|2080|1|3|4|1|2080|722|9401|Monday|2080Q1|N|N|N|2480825|2480884|2480462|2480736|N|N|N|N|N| +2480829|AAAAAAAANLKNFCAA|2080-03-05|2162|9402|722|2080|2|3|5|1|2080|722|9402|Tuesday|2080Q1|N|N|N|2480825|2480884|2480463|2480737|N|N|N|N|N| +2480830|AAAAAAAAOLKNFCAA|2080-03-06|2162|9402|722|2080|3|3|6|1|2080|722|9402|Wednesday|2080Q1|N|N|N|2480825|2480884|2480464|2480738|N|N|N|N|N| +2480831|AAAAAAAAPLKNFCAA|2080-03-07|2162|9402|722|2080|4|3|7|1|2080|722|9402|Thursday|2080Q1|N|N|N|2480825|2480884|2480465|2480739|N|N|N|N|N| +2480832|AAAAAAAAAMKNFCAA|2080-03-08|2162|9402|722|2080|5|3|8|1|2080|722|9402|Friday|2080Q1|N|Y|N|2480825|2480884|2480466|2480740|N|N|N|N|N| +2480833|AAAAAAAABMKNFCAA|2080-03-09|2162|9402|722|2080|6|3|9|1|2080|722|9402|Saturday|2080Q1|N|Y|N|2480825|2480884|2480467|2480741|N|N|N|N|N| +2480834|AAAAAAAACMKNFCAA|2080-03-10|2162|9402|722|2080|0|3|10|1|2080|722|9402|Sunday|2080Q1|N|N|N|2480825|2480884|2480468|2480742|N|N|N|N|N| +2480835|AAAAAAAADMKNFCAA|2080-03-11|2162|9402|722|2080|1|3|11|1|2080|722|9402|Monday|2080Q1|N|N|N|2480825|2480884|2480469|2480743|N|N|N|N|N| +2480836|AAAAAAAAEMKNFCAA|2080-03-12|2162|9403|722|2080|2|3|12|1|2080|722|9403|Tuesday|2080Q1|N|N|N|2480825|2480884|2480470|2480744|N|N|N|N|N| +2480837|AAAAAAAAFMKNFCAA|2080-03-13|2162|9403|722|2080|3|3|13|1|2080|722|9403|Wednesday|2080Q1|N|N|N|2480825|2480884|2480471|2480745|N|N|N|N|N| +2480838|AAAAAAAAGMKNFCAA|2080-03-14|2162|9403|722|2080|4|3|14|1|2080|722|9403|Thursday|2080Q1|N|N|N|2480825|2480884|2480472|2480746|N|N|N|N|N| +2480839|AAAAAAAAHMKNFCAA|2080-03-15|2162|9403|722|2080|5|3|15|1|2080|722|9403|Friday|2080Q1|N|Y|N|2480825|2480884|2480473|2480747|N|N|N|N|N| +2480840|AAAAAAAAIMKNFCAA|2080-03-16|2162|9403|722|2080|6|3|16|1|2080|722|9403|Saturday|2080Q1|N|Y|N|2480825|2480884|2480474|2480748|N|N|N|N|N| +2480841|AAAAAAAAJMKNFCAA|2080-03-17|2162|9403|722|2080|0|3|17|1|2080|722|9403|Sunday|2080Q1|N|N|N|2480825|2480884|2480475|2480749|N|N|N|N|N| +2480842|AAAAAAAAKMKNFCAA|2080-03-18|2162|9403|722|2080|1|3|18|1|2080|722|9403|Monday|2080Q1|N|N|N|2480825|2480884|2480476|2480750|N|N|N|N|N| +2480843|AAAAAAAALMKNFCAA|2080-03-19|2162|9404|722|2080|2|3|19|1|2080|722|9404|Tuesday|2080Q1|N|N|N|2480825|2480884|2480477|2480751|N|N|N|N|N| +2480844|AAAAAAAAMMKNFCAA|2080-03-20|2162|9404|722|2080|3|3|20|1|2080|722|9404|Wednesday|2080Q1|N|N|N|2480825|2480884|2480478|2480752|N|N|N|N|N| +2480845|AAAAAAAANMKNFCAA|2080-03-21|2162|9404|722|2080|4|3|21|1|2080|722|9404|Thursday|2080Q1|N|N|N|2480825|2480884|2480479|2480753|N|N|N|N|N| +2480846|AAAAAAAAOMKNFCAA|2080-03-22|2162|9404|722|2080|5|3|22|1|2080|722|9404|Friday|2080Q1|N|Y|N|2480825|2480884|2480480|2480754|N|N|N|N|N| +2480847|AAAAAAAAPMKNFCAA|2080-03-23|2162|9404|722|2080|6|3|23|1|2080|722|9404|Saturday|2080Q1|N|Y|N|2480825|2480884|2480481|2480755|N|N|N|N|N| +2480848|AAAAAAAAANKNFCAA|2080-03-24|2162|9404|722|2080|0|3|24|1|2080|722|9404|Sunday|2080Q1|N|N|N|2480825|2480884|2480482|2480756|N|N|N|N|N| +2480849|AAAAAAAABNKNFCAA|2080-03-25|2162|9404|722|2080|1|3|25|1|2080|722|9404|Monday|2080Q1|N|N|N|2480825|2480884|2480483|2480757|N|N|N|N|N| +2480850|AAAAAAAACNKNFCAA|2080-03-26|2162|9405|722|2080|2|3|26|1|2080|722|9405|Tuesday|2080Q1|N|N|N|2480825|2480884|2480484|2480758|N|N|N|N|N| +2480851|AAAAAAAADNKNFCAA|2080-03-27|2162|9405|722|2080|3|3|27|1|2080|722|9405|Wednesday|2080Q1|N|N|N|2480825|2480884|2480485|2480759|N|N|N|N|N| +2480852|AAAAAAAAENKNFCAA|2080-03-28|2162|9405|722|2080|4|3|28|1|2080|722|9405|Thursday|2080Q1|N|N|N|2480825|2480884|2480486|2480760|N|N|N|N|N| +2480853|AAAAAAAAFNKNFCAA|2080-03-29|2162|9405|722|2080|5|3|29|1|2080|722|9405|Friday|2080Q1|N|Y|N|2480825|2480884|2480487|2480761|N|N|N|N|N| +2480854|AAAAAAAAGNKNFCAA|2080-03-30|2162|9405|722|2080|6|3|30|1|2080|722|9405|Saturday|2080Q1|N|Y|N|2480825|2480884|2480488|2480762|N|N|N|N|N| +2480855|AAAAAAAAHNKNFCAA|2080-03-31|2162|9405|722|2080|0|3|31|1|2080|722|9405|Sunday|2080Q1|N|N|N|2480825|2480884|2480489|2480763|N|N|N|N|N| +2480856|AAAAAAAAINKNFCAA|2080-04-01|2163|9405|722|2080|1|4|1|2|2080|722|9405|Monday|2080Q2|N|N|N|2480856|2480946|2480490|2480765|N|N|N|N|N| +2480857|AAAAAAAAJNKNFCAA|2080-04-02|2163|9406|722|2080|2|4|2|2|2080|722|9406|Tuesday|2080Q2|N|N|N|2480856|2480946|2480491|2480766|N|N|N|N|N| +2480858|AAAAAAAAKNKNFCAA|2080-04-03|2163|9406|722|2080|3|4|3|2|2080|722|9406|Wednesday|2080Q2|N|N|N|2480856|2480946|2480492|2480767|N|N|N|N|N| +2480859|AAAAAAAALNKNFCAA|2080-04-04|2163|9406|722|2080|4|4|4|2|2080|722|9406|Thursday|2080Q2|N|N|N|2480856|2480946|2480493|2480768|N|N|N|N|N| +2480860|AAAAAAAAMNKNFCAA|2080-04-05|2163|9406|722|2080|5|4|5|2|2080|722|9406|Friday|2080Q2|N|Y|N|2480856|2480946|2480494|2480769|N|N|N|N|N| +2480861|AAAAAAAANNKNFCAA|2080-04-06|2163|9406|722|2080|6|4|6|2|2080|722|9406|Saturday|2080Q2|N|Y|N|2480856|2480946|2480495|2480770|N|N|N|N|N| +2480862|AAAAAAAAONKNFCAA|2080-04-07|2163|9406|722|2080|0|4|7|2|2080|722|9406|Sunday|2080Q2|N|N|N|2480856|2480946|2480496|2480771|N|N|N|N|N| +2480863|AAAAAAAAPNKNFCAA|2080-04-08|2163|9406|722|2080|1|4|8|2|2080|722|9406|Monday|2080Q2|N|N|N|2480856|2480946|2480497|2480772|N|N|N|N|N| +2480864|AAAAAAAAAOKNFCAA|2080-04-09|2163|9407|722|2080|2|4|9|2|2080|722|9407|Tuesday|2080Q2|N|N|N|2480856|2480946|2480498|2480773|N|N|N|N|N| +2480865|AAAAAAAABOKNFCAA|2080-04-10|2163|9407|722|2080|3|4|10|2|2080|722|9407|Wednesday|2080Q2|N|N|N|2480856|2480946|2480499|2480774|N|N|N|N|N| +2480866|AAAAAAAACOKNFCAA|2080-04-11|2163|9407|722|2080|4|4|11|2|2080|722|9407|Thursday|2080Q2|N|N|N|2480856|2480946|2480500|2480775|N|N|N|N|N| +2480867|AAAAAAAADOKNFCAA|2080-04-12|2163|9407|722|2080|5|4|12|2|2080|722|9407|Friday|2080Q2|N|Y|N|2480856|2480946|2480501|2480776|N|N|N|N|N| +2480868|AAAAAAAAEOKNFCAA|2080-04-13|2163|9407|722|2080|6|4|13|2|2080|722|9407|Saturday|2080Q2|N|Y|N|2480856|2480946|2480502|2480777|N|N|N|N|N| +2480869|AAAAAAAAFOKNFCAA|2080-04-14|2163|9407|722|2080|0|4|14|2|2080|722|9407|Sunday|2080Q2|N|N|N|2480856|2480946|2480503|2480778|N|N|N|N|N| +2480870|AAAAAAAAGOKNFCAA|2080-04-15|2163|9407|722|2080|1|4|15|2|2080|722|9407|Monday|2080Q2|N|N|N|2480856|2480946|2480504|2480779|N|N|N|N|N| +2480871|AAAAAAAAHOKNFCAA|2080-04-16|2163|9408|722|2080|2|4|16|2|2080|722|9408|Tuesday|2080Q2|N|N|N|2480856|2480946|2480505|2480780|N|N|N|N|N| +2480872|AAAAAAAAIOKNFCAA|2080-04-17|2163|9408|722|2080|3|4|17|2|2080|722|9408|Wednesday|2080Q2|N|N|N|2480856|2480946|2480506|2480781|N|N|N|N|N| +2480873|AAAAAAAAJOKNFCAA|2080-04-18|2163|9408|722|2080|4|4|18|2|2080|722|9408|Thursday|2080Q2|N|N|N|2480856|2480946|2480507|2480782|N|N|N|N|N| +2480874|AAAAAAAAKOKNFCAA|2080-04-19|2163|9408|722|2080|5|4|19|2|2080|722|9408|Friday|2080Q2|N|Y|N|2480856|2480946|2480508|2480783|N|N|N|N|N| +2480875|AAAAAAAALOKNFCAA|2080-04-20|2163|9408|722|2080|6|4|20|2|2080|722|9408|Saturday|2080Q2|N|Y|N|2480856|2480946|2480509|2480784|N|N|N|N|N| +2480876|AAAAAAAAMOKNFCAA|2080-04-21|2163|9408|722|2080|0|4|21|2|2080|722|9408|Sunday|2080Q2|N|N|N|2480856|2480946|2480510|2480785|N|N|N|N|N| +2480877|AAAAAAAANOKNFCAA|2080-04-22|2163|9408|722|2080|1|4|22|2|2080|722|9408|Monday|2080Q2|N|N|N|2480856|2480946|2480511|2480786|N|N|N|N|N| +2480878|AAAAAAAAOOKNFCAA|2080-04-23|2163|9409|722|2080|2|4|23|2|2080|722|9409|Tuesday|2080Q2|N|N|N|2480856|2480946|2480512|2480787|N|N|N|N|N| +2480879|AAAAAAAAPOKNFCAA|2080-04-24|2163|9409|722|2080|3|4|24|2|2080|722|9409|Wednesday|2080Q2|N|N|N|2480856|2480946|2480513|2480788|N|N|N|N|N| +2480880|AAAAAAAAAPKNFCAA|2080-04-25|2163|9409|722|2080|4|4|25|2|2080|722|9409|Thursday|2080Q2|N|N|N|2480856|2480946|2480514|2480789|N|N|N|N|N| +2480881|AAAAAAAABPKNFCAA|2080-04-26|2163|9409|722|2080|5|4|26|2|2080|722|9409|Friday|2080Q2|N|Y|N|2480856|2480946|2480515|2480790|N|N|N|N|N| +2480882|AAAAAAAACPKNFCAA|2080-04-27|2163|9409|722|2080|6|4|27|2|2080|722|9409|Saturday|2080Q2|N|Y|N|2480856|2480946|2480516|2480791|N|N|N|N|N| +2480883|AAAAAAAADPKNFCAA|2080-04-28|2163|9409|722|2080|0|4|28|2|2080|722|9409|Sunday|2080Q2|N|N|N|2480856|2480946|2480517|2480792|N|N|N|N|N| +2480884|AAAAAAAAEPKNFCAA|2080-04-29|2163|9409|722|2080|1|4|29|2|2080|722|9409|Monday|2080Q2|N|N|N|2480856|2480946|2480518|2480793|N|N|N|N|N| +2480885|AAAAAAAAFPKNFCAA|2080-04-30|2163|9410|722|2080|2|4|30|2|2080|722|9410|Tuesday|2080Q2|N|N|N|2480856|2480946|2480519|2480794|N|N|N|N|N| +2480886|AAAAAAAAGPKNFCAA|2080-05-01|2164|9410|722|2080|3|5|1|2|2080|722|9410|Wednesday|2080Q2|N|N|N|2480886|2481006|2480520|2480795|N|N|N|N|N| +2480887|AAAAAAAAHPKNFCAA|2080-05-02|2164|9410|722|2080|4|5|2|2|2080|722|9410|Thursday|2080Q2|N|N|N|2480886|2481006|2480521|2480796|N|N|N|N|N| +2480888|AAAAAAAAIPKNFCAA|2080-05-03|2164|9410|722|2080|5|5|3|2|2080|722|9410|Friday|2080Q2|N|Y|N|2480886|2481006|2480522|2480797|N|N|N|N|N| +2480889|AAAAAAAAJPKNFCAA|2080-05-04|2164|9410|722|2080|6|5|4|2|2080|722|9410|Saturday|2080Q2|N|Y|N|2480886|2481006|2480523|2480798|N|N|N|N|N| +2480890|AAAAAAAAKPKNFCAA|2080-05-05|2164|9410|722|2080|0|5|5|2|2080|722|9410|Sunday|2080Q2|N|N|N|2480886|2481006|2480524|2480799|N|N|N|N|N| +2480891|AAAAAAAALPKNFCAA|2080-05-06|2164|9410|722|2080|1|5|6|2|2080|722|9410|Monday|2080Q2|N|N|N|2480886|2481006|2480525|2480800|N|N|N|N|N| +2480892|AAAAAAAAMPKNFCAA|2080-05-07|2164|9411|722|2080|2|5|7|2|2080|722|9411|Tuesday|2080Q2|N|N|N|2480886|2481006|2480526|2480801|N|N|N|N|N| +2480893|AAAAAAAANPKNFCAA|2080-05-08|2164|9411|722|2080|3|5|8|2|2080|722|9411|Wednesday|2080Q2|N|N|N|2480886|2481006|2480527|2480802|N|N|N|N|N| +2480894|AAAAAAAAOPKNFCAA|2080-05-09|2164|9411|722|2080|4|5|9|2|2080|722|9411|Thursday|2080Q2|N|N|N|2480886|2481006|2480528|2480803|N|N|N|N|N| +2480895|AAAAAAAAPPKNFCAA|2080-05-10|2164|9411|722|2080|5|5|10|2|2080|722|9411|Friday|2080Q2|N|Y|N|2480886|2481006|2480529|2480804|N|N|N|N|N| +2480896|AAAAAAAAAALNFCAA|2080-05-11|2164|9411|722|2080|6|5|11|2|2080|722|9411|Saturday|2080Q2|N|Y|N|2480886|2481006|2480530|2480805|N|N|N|N|N| +2480897|AAAAAAAABALNFCAA|2080-05-12|2164|9411|722|2080|0|5|12|2|2080|722|9411|Sunday|2080Q2|N|N|N|2480886|2481006|2480531|2480806|N|N|N|N|N| +2480898|AAAAAAAACALNFCAA|2080-05-13|2164|9411|722|2080|1|5|13|2|2080|722|9411|Monday|2080Q2|N|N|N|2480886|2481006|2480532|2480807|N|N|N|N|N| +2480899|AAAAAAAADALNFCAA|2080-05-14|2164|9412|722|2080|2|5|14|2|2080|722|9412|Tuesday|2080Q2|N|N|N|2480886|2481006|2480533|2480808|N|N|N|N|N| +2480900|AAAAAAAAEALNFCAA|2080-05-15|2164|9412|722|2080|3|5|15|2|2080|722|9412|Wednesday|2080Q2|N|N|N|2480886|2481006|2480534|2480809|N|N|N|N|N| +2480901|AAAAAAAAFALNFCAA|2080-05-16|2164|9412|722|2080|4|5|16|2|2080|722|9412|Thursday|2080Q2|N|N|N|2480886|2481006|2480535|2480810|N|N|N|N|N| +2480902|AAAAAAAAGALNFCAA|2080-05-17|2164|9412|722|2080|5|5|17|2|2080|722|9412|Friday|2080Q2|N|Y|N|2480886|2481006|2480536|2480811|N|N|N|N|N| +2480903|AAAAAAAAHALNFCAA|2080-05-18|2164|9412|722|2080|6|5|18|2|2080|722|9412|Saturday|2080Q2|N|Y|N|2480886|2481006|2480537|2480812|N|N|N|N|N| +2480904|AAAAAAAAIALNFCAA|2080-05-19|2164|9412|722|2080|0|5|19|2|2080|722|9412|Sunday|2080Q2|N|N|N|2480886|2481006|2480538|2480813|N|N|N|N|N| +2480905|AAAAAAAAJALNFCAA|2080-05-20|2164|9412|722|2080|1|5|20|2|2080|722|9412|Monday|2080Q2|N|N|N|2480886|2481006|2480539|2480814|N|N|N|N|N| +2480906|AAAAAAAAKALNFCAA|2080-05-21|2164|9413|722|2080|2|5|21|2|2080|722|9413|Tuesday|2080Q2|N|N|N|2480886|2481006|2480540|2480815|N|N|N|N|N| +2480907|AAAAAAAALALNFCAA|2080-05-22|2164|9413|722|2080|3|5|22|2|2080|722|9413|Wednesday|2080Q2|N|N|N|2480886|2481006|2480541|2480816|N|N|N|N|N| +2480908|AAAAAAAAMALNFCAA|2080-05-23|2164|9413|722|2080|4|5|23|2|2080|722|9413|Thursday|2080Q2|N|N|N|2480886|2481006|2480542|2480817|N|N|N|N|N| +2480909|AAAAAAAANALNFCAA|2080-05-24|2164|9413|722|2080|5|5|24|2|2080|722|9413|Friday|2080Q2|N|Y|N|2480886|2481006|2480543|2480818|N|N|N|N|N| +2480910|AAAAAAAAOALNFCAA|2080-05-25|2164|9413|722|2080|6|5|25|2|2080|722|9413|Saturday|2080Q2|N|Y|N|2480886|2481006|2480544|2480819|N|N|N|N|N| +2480911|AAAAAAAAPALNFCAA|2080-05-26|2164|9413|722|2080|0|5|26|2|2080|722|9413|Sunday|2080Q2|N|N|N|2480886|2481006|2480545|2480820|N|N|N|N|N| +2480912|AAAAAAAAABLNFCAA|2080-05-27|2164|9413|722|2080|1|5|27|2|2080|722|9413|Monday|2080Q2|N|N|N|2480886|2481006|2480546|2480821|N|N|N|N|N| +2480913|AAAAAAAABBLNFCAA|2080-05-28|2164|9414|722|2080|2|5|28|2|2080|722|9414|Tuesday|2080Q2|N|N|N|2480886|2481006|2480547|2480822|N|N|N|N|N| +2480914|AAAAAAAACBLNFCAA|2080-05-29|2164|9414|722|2080|3|5|29|2|2080|722|9414|Wednesday|2080Q2|N|N|N|2480886|2481006|2480548|2480823|N|N|N|N|N| +2480915|AAAAAAAADBLNFCAA|2080-05-30|2164|9414|722|2080|4|5|30|2|2080|722|9414|Thursday|2080Q2|N|N|N|2480886|2481006|2480549|2480824|N|N|N|N|N| +2480916|AAAAAAAAEBLNFCAA|2080-05-31|2164|9414|722|2080|5|5|31|2|2080|722|9414|Friday|2080Q2|N|Y|N|2480886|2481006|2480550|2480825|N|N|N|N|N| +2480917|AAAAAAAAFBLNFCAA|2080-06-01|2165|9414|723|2080|6|6|1|2|2080|723|9414|Saturday|2080Q2|N|Y|N|2480917|2481068|2480551|2480826|N|N|N|N|N| +2480918|AAAAAAAAGBLNFCAA|2080-06-02|2165|9414|723|2080|0|6|2|2|2080|723|9414|Sunday|2080Q2|N|N|N|2480917|2481068|2480552|2480827|N|N|N|N|N| +2480919|AAAAAAAAHBLNFCAA|2080-06-03|2165|9414|723|2080|1|6|3|2|2080|723|9414|Monday|2080Q2|N|N|N|2480917|2481068|2480553|2480828|N|N|N|N|N| +2480920|AAAAAAAAIBLNFCAA|2080-06-04|2165|9415|723|2080|2|6|4|2|2080|723|9415|Tuesday|2080Q2|N|N|N|2480917|2481068|2480554|2480829|N|N|N|N|N| +2480921|AAAAAAAAJBLNFCAA|2080-06-05|2165|9415|723|2080|3|6|5|2|2080|723|9415|Wednesday|2080Q2|N|N|N|2480917|2481068|2480555|2480830|N|N|N|N|N| +2480922|AAAAAAAAKBLNFCAA|2080-06-06|2165|9415|723|2080|4|6|6|2|2080|723|9415|Thursday|2080Q2|N|N|N|2480917|2481068|2480556|2480831|N|N|N|N|N| +2480923|AAAAAAAALBLNFCAA|2080-06-07|2165|9415|723|2080|5|6|7|2|2080|723|9415|Friday|2080Q2|N|Y|N|2480917|2481068|2480557|2480832|N|N|N|N|N| +2480924|AAAAAAAAMBLNFCAA|2080-06-08|2165|9415|723|2080|6|6|8|2|2080|723|9415|Saturday|2080Q2|N|Y|N|2480917|2481068|2480558|2480833|N|N|N|N|N| +2480925|AAAAAAAANBLNFCAA|2080-06-09|2165|9415|723|2080|0|6|9|2|2080|723|9415|Sunday|2080Q2|N|N|N|2480917|2481068|2480559|2480834|N|N|N|N|N| +2480926|AAAAAAAAOBLNFCAA|2080-06-10|2165|9415|723|2080|1|6|10|2|2080|723|9415|Monday|2080Q2|N|N|N|2480917|2481068|2480560|2480835|N|N|N|N|N| +2480927|AAAAAAAAPBLNFCAA|2080-06-11|2165|9416|723|2080|2|6|11|2|2080|723|9416|Tuesday|2080Q2|N|N|N|2480917|2481068|2480561|2480836|N|N|N|N|N| +2480928|AAAAAAAAACLNFCAA|2080-06-12|2165|9416|723|2080|3|6|12|2|2080|723|9416|Wednesday|2080Q2|N|N|N|2480917|2481068|2480562|2480837|N|N|N|N|N| +2480929|AAAAAAAABCLNFCAA|2080-06-13|2165|9416|723|2080|4|6|13|2|2080|723|9416|Thursday|2080Q2|N|N|N|2480917|2481068|2480563|2480838|N|N|N|N|N| +2480930|AAAAAAAACCLNFCAA|2080-06-14|2165|9416|723|2080|5|6|14|2|2080|723|9416|Friday|2080Q2|N|Y|N|2480917|2481068|2480564|2480839|N|N|N|N|N| +2480931|AAAAAAAADCLNFCAA|2080-06-15|2165|9416|723|2080|6|6|15|2|2080|723|9416|Saturday|2080Q2|N|Y|N|2480917|2481068|2480565|2480840|N|N|N|N|N| +2480932|AAAAAAAAECLNFCAA|2080-06-16|2165|9416|723|2080|0|6|16|2|2080|723|9416|Sunday|2080Q2|N|N|N|2480917|2481068|2480566|2480841|N|N|N|N|N| +2480933|AAAAAAAAFCLNFCAA|2080-06-17|2165|9416|723|2080|1|6|17|2|2080|723|9416|Monday|2080Q2|N|N|N|2480917|2481068|2480567|2480842|N|N|N|N|N| +2480934|AAAAAAAAGCLNFCAA|2080-06-18|2165|9417|723|2080|2|6|18|2|2080|723|9417|Tuesday|2080Q2|N|N|N|2480917|2481068|2480568|2480843|N|N|N|N|N| +2480935|AAAAAAAAHCLNFCAA|2080-06-19|2165|9417|723|2080|3|6|19|2|2080|723|9417|Wednesday|2080Q2|N|N|N|2480917|2481068|2480569|2480844|N|N|N|N|N| +2480936|AAAAAAAAICLNFCAA|2080-06-20|2165|9417|723|2080|4|6|20|2|2080|723|9417|Thursday|2080Q2|N|N|N|2480917|2481068|2480570|2480845|N|N|N|N|N| +2480937|AAAAAAAAJCLNFCAA|2080-06-21|2165|9417|723|2080|5|6|21|2|2080|723|9417|Friday|2080Q2|N|Y|N|2480917|2481068|2480571|2480846|N|N|N|N|N| +2480938|AAAAAAAAKCLNFCAA|2080-06-22|2165|9417|723|2080|6|6|22|2|2080|723|9417|Saturday|2080Q2|N|Y|N|2480917|2481068|2480572|2480847|N|N|N|N|N| +2480939|AAAAAAAALCLNFCAA|2080-06-23|2165|9417|723|2080|0|6|23|2|2080|723|9417|Sunday|2080Q2|N|N|N|2480917|2481068|2480573|2480848|N|N|N|N|N| +2480940|AAAAAAAAMCLNFCAA|2080-06-24|2165|9417|723|2080|1|6|24|2|2080|723|9417|Monday|2080Q2|N|N|N|2480917|2481068|2480574|2480849|N|N|N|N|N| +2480941|AAAAAAAANCLNFCAA|2080-06-25|2165|9418|723|2080|2|6|25|2|2080|723|9418|Tuesday|2080Q2|N|N|N|2480917|2481068|2480575|2480850|N|N|N|N|N| +2480942|AAAAAAAAOCLNFCAA|2080-06-26|2165|9418|723|2080|3|6|26|2|2080|723|9418|Wednesday|2080Q2|N|N|N|2480917|2481068|2480576|2480851|N|N|N|N|N| +2480943|AAAAAAAAPCLNFCAA|2080-06-27|2165|9418|723|2080|4|6|27|2|2080|723|9418|Thursday|2080Q2|N|N|N|2480917|2481068|2480577|2480852|N|N|N|N|N| +2480944|AAAAAAAAADLNFCAA|2080-06-28|2165|9418|723|2080|5|6|28|2|2080|723|9418|Friday|2080Q2|N|Y|N|2480917|2481068|2480578|2480853|N|N|N|N|N| +2480945|AAAAAAAABDLNFCAA|2080-06-29|2165|9418|723|2080|6|6|29|2|2080|723|9418|Saturday|2080Q2|N|Y|N|2480917|2481068|2480579|2480854|N|N|N|N|N| +2480946|AAAAAAAACDLNFCAA|2080-06-30|2165|9418|723|2080|0|6|30|2|2080|723|9418|Sunday|2080Q2|N|N|N|2480917|2481068|2480580|2480855|N|N|N|N|N| +2480947|AAAAAAAADDLNFCAA|2080-07-01|2166|9418|723|2080|1|7|1|3|2080|723|9418|Monday|2080Q3|N|N|N|2480947|2481128|2480581|2480856|N|N|N|N|N| +2480948|AAAAAAAAEDLNFCAA|2080-07-02|2166|9419|723|2080|2|7|2|3|2080|723|9419|Tuesday|2080Q3|N|N|N|2480947|2481128|2480582|2480857|N|N|N|N|N| +2480949|AAAAAAAAFDLNFCAA|2080-07-03|2166|9419|723|2080|3|7|3|3|2080|723|9419|Wednesday|2080Q3|N|N|N|2480947|2481128|2480583|2480858|N|N|N|N|N| +2480950|AAAAAAAAGDLNFCAA|2080-07-04|2166|9419|723|2080|4|7|4|3|2080|723|9419|Thursday|2080Q3|Y|N|N|2480947|2481128|2480584|2480859|N|N|N|N|N| +2480951|AAAAAAAAHDLNFCAA|2080-07-05|2166|9419|723|2080|5|7|5|3|2080|723|9419|Friday|2080Q3|N|Y|Y|2480947|2481128|2480585|2480860|N|N|N|N|N| +2480952|AAAAAAAAIDLNFCAA|2080-07-06|2166|9419|723|2080|6|7|6|3|2080|723|9419|Saturday|2080Q3|N|Y|N|2480947|2481128|2480586|2480861|N|N|N|N|N| +2480953|AAAAAAAAJDLNFCAA|2080-07-07|2166|9419|723|2080|0|7|7|3|2080|723|9419|Sunday|2080Q3|N|N|N|2480947|2481128|2480587|2480862|N|N|N|N|N| +2480954|AAAAAAAAKDLNFCAA|2080-07-08|2166|9419|723|2080|1|7|8|3|2080|723|9419|Monday|2080Q3|N|N|N|2480947|2481128|2480588|2480863|N|N|N|N|N| +2480955|AAAAAAAALDLNFCAA|2080-07-09|2166|9420|723|2080|2|7|9|3|2080|723|9420|Tuesday|2080Q3|N|N|N|2480947|2481128|2480589|2480864|N|N|N|N|N| +2480956|AAAAAAAAMDLNFCAA|2080-07-10|2166|9420|723|2080|3|7|10|3|2080|723|9420|Wednesday|2080Q3|N|N|N|2480947|2481128|2480590|2480865|N|N|N|N|N| +2480957|AAAAAAAANDLNFCAA|2080-07-11|2166|9420|723|2080|4|7|11|3|2080|723|9420|Thursday|2080Q3|N|N|N|2480947|2481128|2480591|2480866|N|N|N|N|N| +2480958|AAAAAAAAODLNFCAA|2080-07-12|2166|9420|723|2080|5|7|12|3|2080|723|9420|Friday|2080Q3|N|Y|N|2480947|2481128|2480592|2480867|N|N|N|N|N| +2480959|AAAAAAAAPDLNFCAA|2080-07-13|2166|9420|723|2080|6|7|13|3|2080|723|9420|Saturday|2080Q3|N|Y|N|2480947|2481128|2480593|2480868|N|N|N|N|N| +2480960|AAAAAAAAAELNFCAA|2080-07-14|2166|9420|723|2080|0|7|14|3|2080|723|9420|Sunday|2080Q3|N|N|N|2480947|2481128|2480594|2480869|N|N|N|N|N| +2480961|AAAAAAAABELNFCAA|2080-07-15|2166|9420|723|2080|1|7|15|3|2080|723|9420|Monday|2080Q3|N|N|N|2480947|2481128|2480595|2480870|N|N|N|N|N| +2480962|AAAAAAAACELNFCAA|2080-07-16|2166|9421|723|2080|2|7|16|3|2080|723|9421|Tuesday|2080Q3|N|N|N|2480947|2481128|2480596|2480871|N|N|N|N|N| +2480963|AAAAAAAADELNFCAA|2080-07-17|2166|9421|723|2080|3|7|17|3|2080|723|9421|Wednesday|2080Q3|N|N|N|2480947|2481128|2480597|2480872|N|N|N|N|N| +2480964|AAAAAAAAEELNFCAA|2080-07-18|2166|9421|723|2080|4|7|18|3|2080|723|9421|Thursday|2080Q3|N|N|N|2480947|2481128|2480598|2480873|N|N|N|N|N| +2480965|AAAAAAAAFELNFCAA|2080-07-19|2166|9421|723|2080|5|7|19|3|2080|723|9421|Friday|2080Q3|N|Y|N|2480947|2481128|2480599|2480874|N|N|N|N|N| +2480966|AAAAAAAAGELNFCAA|2080-07-20|2166|9421|723|2080|6|7|20|3|2080|723|9421|Saturday|2080Q3|N|Y|N|2480947|2481128|2480600|2480875|N|N|N|N|N| +2480967|AAAAAAAAHELNFCAA|2080-07-21|2166|9421|723|2080|0|7|21|3|2080|723|9421|Sunday|2080Q3|N|N|N|2480947|2481128|2480601|2480876|N|N|N|N|N| +2480968|AAAAAAAAIELNFCAA|2080-07-22|2166|9421|723|2080|1|7|22|3|2080|723|9421|Monday|2080Q3|N|N|N|2480947|2481128|2480602|2480877|N|N|N|N|N| +2480969|AAAAAAAAJELNFCAA|2080-07-23|2166|9422|723|2080|2|7|23|3|2080|723|9422|Tuesday|2080Q3|N|N|N|2480947|2481128|2480603|2480878|N|N|N|N|N| +2480970|AAAAAAAAKELNFCAA|2080-07-24|2166|9422|723|2080|3|7|24|3|2080|723|9422|Wednesday|2080Q3|N|N|N|2480947|2481128|2480604|2480879|N|N|N|N|N| +2480971|AAAAAAAALELNFCAA|2080-07-25|2166|9422|723|2080|4|7|25|3|2080|723|9422|Thursday|2080Q3|N|N|N|2480947|2481128|2480605|2480880|N|N|N|N|N| +2480972|AAAAAAAAMELNFCAA|2080-07-26|2166|9422|723|2080|5|7|26|3|2080|723|9422|Friday|2080Q3|N|Y|N|2480947|2481128|2480606|2480881|N|N|N|N|N| +2480973|AAAAAAAANELNFCAA|2080-07-27|2166|9422|723|2080|6|7|27|3|2080|723|9422|Saturday|2080Q3|N|Y|N|2480947|2481128|2480607|2480882|N|N|N|N|N| +2480974|AAAAAAAAOELNFCAA|2080-07-28|2166|9422|723|2080|0|7|28|3|2080|723|9422|Sunday|2080Q3|N|N|N|2480947|2481128|2480608|2480883|N|N|N|N|N| +2480975|AAAAAAAAPELNFCAA|2080-07-29|2166|9422|723|2080|1|7|29|3|2080|723|9422|Monday|2080Q3|N|N|N|2480947|2481128|2480609|2480884|N|N|N|N|N| +2480976|AAAAAAAAAFLNFCAA|2080-07-30|2166|9423|723|2080|2|7|30|3|2080|723|9423|Tuesday|2080Q3|N|N|N|2480947|2481128|2480610|2480885|N|N|N|N|N| +2480977|AAAAAAAABFLNFCAA|2080-07-31|2166|9423|723|2080|3|7|31|3|2080|723|9423|Wednesday|2080Q3|N|N|N|2480947|2481128|2480611|2480886|N|N|N|N|N| +2480978|AAAAAAAACFLNFCAA|2080-08-01|2167|9423|723|2080|4|8|1|3|2080|723|9423|Thursday|2080Q3|N|N|N|2480978|2481190|2480612|2480887|N|N|N|N|N| +2480979|AAAAAAAADFLNFCAA|2080-08-02|2167|9423|723|2080|5|8|2|3|2080|723|9423|Friday|2080Q3|N|Y|N|2480978|2481190|2480613|2480888|N|N|N|N|N| +2480980|AAAAAAAAEFLNFCAA|2080-08-03|2167|9423|723|2080|6|8|3|3|2080|723|9423|Saturday|2080Q3|N|Y|N|2480978|2481190|2480614|2480889|N|N|N|N|N| +2480981|AAAAAAAAFFLNFCAA|2080-08-04|2167|9423|723|2080|0|8|4|3|2080|723|9423|Sunday|2080Q3|N|N|N|2480978|2481190|2480615|2480890|N|N|N|N|N| +2480982|AAAAAAAAGFLNFCAA|2080-08-05|2167|9423|723|2080|1|8|5|3|2080|723|9423|Monday|2080Q3|N|N|N|2480978|2481190|2480616|2480891|N|N|N|N|N| +2480983|AAAAAAAAHFLNFCAA|2080-08-06|2167|9424|723|2080|2|8|6|3|2080|723|9424|Tuesday|2080Q3|N|N|N|2480978|2481190|2480617|2480892|N|N|N|N|N| +2480984|AAAAAAAAIFLNFCAA|2080-08-07|2167|9424|723|2080|3|8|7|3|2080|723|9424|Wednesday|2080Q3|N|N|N|2480978|2481190|2480618|2480893|N|N|N|N|N| +2480985|AAAAAAAAJFLNFCAA|2080-08-08|2167|9424|723|2080|4|8|8|3|2080|723|9424|Thursday|2080Q3|N|N|N|2480978|2481190|2480619|2480894|N|N|N|N|N| +2480986|AAAAAAAAKFLNFCAA|2080-08-09|2167|9424|723|2080|5|8|9|3|2080|723|9424|Friday|2080Q3|N|Y|N|2480978|2481190|2480620|2480895|N|N|N|N|N| +2480987|AAAAAAAALFLNFCAA|2080-08-10|2167|9424|723|2080|6|8|10|3|2080|723|9424|Saturday|2080Q3|N|Y|N|2480978|2481190|2480621|2480896|N|N|N|N|N| +2480988|AAAAAAAAMFLNFCAA|2080-08-11|2167|9424|723|2080|0|8|11|3|2080|723|9424|Sunday|2080Q3|N|N|N|2480978|2481190|2480622|2480897|N|N|N|N|N| +2480989|AAAAAAAANFLNFCAA|2080-08-12|2167|9424|723|2080|1|8|12|3|2080|723|9424|Monday|2080Q3|N|N|N|2480978|2481190|2480623|2480898|N|N|N|N|N| +2480990|AAAAAAAAOFLNFCAA|2080-08-13|2167|9425|723|2080|2|8|13|3|2080|723|9425|Tuesday|2080Q3|N|N|N|2480978|2481190|2480624|2480899|N|N|N|N|N| +2480991|AAAAAAAAPFLNFCAA|2080-08-14|2167|9425|723|2080|3|8|14|3|2080|723|9425|Wednesday|2080Q3|N|N|N|2480978|2481190|2480625|2480900|N|N|N|N|N| +2480992|AAAAAAAAAGLNFCAA|2080-08-15|2167|9425|723|2080|4|8|15|3|2080|723|9425|Thursday|2080Q3|N|N|N|2480978|2481190|2480626|2480901|N|N|N|N|N| +2480993|AAAAAAAABGLNFCAA|2080-08-16|2167|9425|723|2080|5|8|16|3|2080|723|9425|Friday|2080Q3|N|Y|N|2480978|2481190|2480627|2480902|N|N|N|N|N| +2480994|AAAAAAAACGLNFCAA|2080-08-17|2167|9425|723|2080|6|8|17|3|2080|723|9425|Saturday|2080Q3|N|Y|N|2480978|2481190|2480628|2480903|N|N|N|N|N| +2480995|AAAAAAAADGLNFCAA|2080-08-18|2167|9425|723|2080|0|8|18|3|2080|723|9425|Sunday|2080Q3|N|N|N|2480978|2481190|2480629|2480904|N|N|N|N|N| +2480996|AAAAAAAAEGLNFCAA|2080-08-19|2167|9425|723|2080|1|8|19|3|2080|723|9425|Monday|2080Q3|N|N|N|2480978|2481190|2480630|2480905|N|N|N|N|N| +2480997|AAAAAAAAFGLNFCAA|2080-08-20|2167|9426|723|2080|2|8|20|3|2080|723|9426|Tuesday|2080Q3|N|N|N|2480978|2481190|2480631|2480906|N|N|N|N|N| +2480998|AAAAAAAAGGLNFCAA|2080-08-21|2167|9426|723|2080|3|8|21|3|2080|723|9426|Wednesday|2080Q3|N|N|N|2480978|2481190|2480632|2480907|N|N|N|N|N| +2480999|AAAAAAAAHGLNFCAA|2080-08-22|2167|9426|723|2080|4|8|22|3|2080|723|9426|Thursday|2080Q3|N|N|N|2480978|2481190|2480633|2480908|N|N|N|N|N| +2481000|AAAAAAAAIGLNFCAA|2080-08-23|2167|9426|723|2080|5|8|23|3|2080|723|9426|Friday|2080Q3|N|Y|N|2480978|2481190|2480634|2480909|N|N|N|N|N| +2481001|AAAAAAAAJGLNFCAA|2080-08-24|2167|9426|723|2080|6|8|24|3|2080|723|9426|Saturday|2080Q3|N|Y|N|2480978|2481190|2480635|2480910|N|N|N|N|N| +2481002|AAAAAAAAKGLNFCAA|2080-08-25|2167|9426|723|2080|0|8|25|3|2080|723|9426|Sunday|2080Q3|N|N|N|2480978|2481190|2480636|2480911|N|N|N|N|N| +2481003|AAAAAAAALGLNFCAA|2080-08-26|2167|9426|723|2080|1|8|26|3|2080|723|9426|Monday|2080Q3|N|N|N|2480978|2481190|2480637|2480912|N|N|N|N|N| +2481004|AAAAAAAAMGLNFCAA|2080-08-27|2167|9427|723|2080|2|8|27|3|2080|723|9427|Tuesday|2080Q3|N|N|N|2480978|2481190|2480638|2480913|N|N|N|N|N| +2481005|AAAAAAAANGLNFCAA|2080-08-28|2167|9427|723|2080|3|8|28|3|2080|723|9427|Wednesday|2080Q3|N|N|N|2480978|2481190|2480639|2480914|N|N|N|N|N| +2481006|AAAAAAAAOGLNFCAA|2080-08-29|2167|9427|723|2080|4|8|29|3|2080|723|9427|Thursday|2080Q3|N|N|N|2480978|2481190|2480640|2480915|N|N|N|N|N| +2481007|AAAAAAAAPGLNFCAA|2080-08-30|2167|9427|723|2080|5|8|30|3|2080|723|9427|Friday|2080Q3|N|Y|N|2480978|2481190|2480641|2480916|N|N|N|N|N| +2481008|AAAAAAAAAHLNFCAA|2080-08-31|2167|9427|723|2080|6|8|31|3|2080|723|9427|Saturday|2080Q3|N|Y|N|2480978|2481190|2480642|2480917|N|N|N|N|N| +2481009|AAAAAAAABHLNFCAA|2080-09-01|2168|9427|724|2080|0|9|1|3|2080|724|9427|Sunday|2080Q3|N|N|N|2481009|2481252|2480643|2480918|N|N|N|N|N| +2481010|AAAAAAAACHLNFCAA|2080-09-02|2168|9427|724|2080|1|9|2|3|2080|724|9427|Monday|2080Q3|N|N|N|2481009|2481252|2480644|2480919|N|N|N|N|N| +2481011|AAAAAAAADHLNFCAA|2080-09-03|2168|9428|724|2080|2|9|3|3|2080|724|9428|Tuesday|2080Q3|N|N|N|2481009|2481252|2480645|2480920|N|N|N|N|N| +2481012|AAAAAAAAEHLNFCAA|2080-09-04|2168|9428|724|2080|3|9|4|3|2080|724|9428|Wednesday|2080Q3|N|N|N|2481009|2481252|2480646|2480921|N|N|N|N|N| +2481013|AAAAAAAAFHLNFCAA|2080-09-05|2168|9428|724|2080|4|9|5|3|2080|724|9428|Thursday|2080Q3|N|N|N|2481009|2481252|2480647|2480922|N|N|N|N|N| +2481014|AAAAAAAAGHLNFCAA|2080-09-06|2168|9428|724|2080|5|9|6|3|2080|724|9428|Friday|2080Q3|N|Y|N|2481009|2481252|2480648|2480923|N|N|N|N|N| +2481015|AAAAAAAAHHLNFCAA|2080-09-07|2168|9428|724|2080|6|9|7|3|2080|724|9428|Saturday|2080Q3|N|Y|N|2481009|2481252|2480649|2480924|N|N|N|N|N| +2481016|AAAAAAAAIHLNFCAA|2080-09-08|2168|9428|724|2080|0|9|8|3|2080|724|9428|Sunday|2080Q3|N|N|N|2481009|2481252|2480650|2480925|N|N|N|N|N| +2481017|AAAAAAAAJHLNFCAA|2080-09-09|2168|9428|724|2080|1|9|9|3|2080|724|9428|Monday|2080Q3|N|N|N|2481009|2481252|2480651|2480926|N|N|N|N|N| +2481018|AAAAAAAAKHLNFCAA|2080-09-10|2168|9429|724|2080|2|9|10|3|2080|724|9429|Tuesday|2080Q3|N|N|N|2481009|2481252|2480652|2480927|N|N|N|N|N| +2481019|AAAAAAAALHLNFCAA|2080-09-11|2168|9429|724|2080|3|9|11|3|2080|724|9429|Wednesday|2080Q3|N|N|N|2481009|2481252|2480653|2480928|N|N|N|N|N| +2481020|AAAAAAAAMHLNFCAA|2080-09-12|2168|9429|724|2080|4|9|12|3|2080|724|9429|Thursday|2080Q3|N|N|N|2481009|2481252|2480654|2480929|N|N|N|N|N| +2481021|AAAAAAAANHLNFCAA|2080-09-13|2168|9429|724|2080|5|9|13|3|2080|724|9429|Friday|2080Q3|N|Y|N|2481009|2481252|2480655|2480930|N|N|N|N|N| +2481022|AAAAAAAAOHLNFCAA|2080-09-14|2168|9429|724|2080|6|9|14|3|2080|724|9429|Saturday|2080Q3|N|Y|N|2481009|2481252|2480656|2480931|N|N|N|N|N| +2481023|AAAAAAAAPHLNFCAA|2080-09-15|2168|9429|724|2080|0|9|15|3|2080|724|9429|Sunday|2080Q3|N|N|N|2481009|2481252|2480657|2480932|N|N|N|N|N| +2481024|AAAAAAAAAILNFCAA|2080-09-16|2168|9429|724|2080|1|9|16|3|2080|724|9429|Monday|2080Q3|N|N|N|2481009|2481252|2480658|2480933|N|N|N|N|N| +2481025|AAAAAAAABILNFCAA|2080-09-17|2168|9430|724|2080|2|9|17|3|2080|724|9430|Tuesday|2080Q3|N|N|N|2481009|2481252|2480659|2480934|N|N|N|N|N| +2481026|AAAAAAAACILNFCAA|2080-09-18|2168|9430|724|2080|3|9|18|3|2080|724|9430|Wednesday|2080Q3|N|N|N|2481009|2481252|2480660|2480935|N|N|N|N|N| +2481027|AAAAAAAADILNFCAA|2080-09-19|2168|9430|724|2080|4|9|19|3|2080|724|9430|Thursday|2080Q3|N|N|N|2481009|2481252|2480661|2480936|N|N|N|N|N| +2481028|AAAAAAAAEILNFCAA|2080-09-20|2168|9430|724|2080|5|9|20|3|2080|724|9430|Friday|2080Q3|N|Y|N|2481009|2481252|2480662|2480937|N|N|N|N|N| +2481029|AAAAAAAAFILNFCAA|2080-09-21|2168|9430|724|2080|6|9|21|3|2080|724|9430|Saturday|2080Q3|N|Y|N|2481009|2481252|2480663|2480938|N|N|N|N|N| +2481030|AAAAAAAAGILNFCAA|2080-09-22|2168|9430|724|2080|0|9|22|3|2080|724|9430|Sunday|2080Q3|N|N|N|2481009|2481252|2480664|2480939|N|N|N|N|N| +2481031|AAAAAAAAHILNFCAA|2080-09-23|2168|9430|724|2080|1|9|23|3|2080|724|9430|Monday|2080Q3|N|N|N|2481009|2481252|2480665|2480940|N|N|N|N|N| +2481032|AAAAAAAAIILNFCAA|2080-09-24|2168|9431|724|2080|2|9|24|3|2080|724|9431|Tuesday|2080Q3|N|N|N|2481009|2481252|2480666|2480941|N|N|N|N|N| +2481033|AAAAAAAAJILNFCAA|2080-09-25|2168|9431|724|2080|3|9|25|3|2080|724|9431|Wednesday|2080Q3|N|N|N|2481009|2481252|2480667|2480942|N|N|N|N|N| +2481034|AAAAAAAAKILNFCAA|2080-09-26|2168|9431|724|2080|4|9|26|3|2080|724|9431|Thursday|2080Q3|N|N|N|2481009|2481252|2480668|2480943|N|N|N|N|N| +2481035|AAAAAAAALILNFCAA|2080-09-27|2168|9431|724|2080|5|9|27|3|2080|724|9431|Friday|2080Q3|N|Y|N|2481009|2481252|2480669|2480944|N|N|N|N|N| +2481036|AAAAAAAAMILNFCAA|2080-09-28|2168|9431|724|2080|6|9|28|3|2080|724|9431|Saturday|2080Q3|N|Y|N|2481009|2481252|2480670|2480945|N|N|N|N|N| +2481037|AAAAAAAANILNFCAA|2080-09-29|2168|9431|724|2080|0|9|29|3|2080|724|9431|Sunday|2080Q3|N|N|N|2481009|2481252|2480671|2480946|N|N|N|N|N| +2481038|AAAAAAAAOILNFCAA|2080-09-30|2168|9431|724|2080|1|9|30|3|2080|724|9431|Monday|2080Q3|N|N|N|2481009|2481252|2480672|2480947|N|N|N|N|N| +2481039|AAAAAAAAPILNFCAA|2080-10-01|2169|9432|724|2080|2|10|1|4|2080|724|9432|Tuesday|2080Q4|N|N|N|2481039|2481312|2480673|2480947|N|N|N|N|N| +2481040|AAAAAAAAAJLNFCAA|2080-10-02|2169|9432|724|2080|3|10|2|4|2080|724|9432|Wednesday|2080Q4|N|N|N|2481039|2481312|2480674|2480948|N|N|N|N|N| +2481041|AAAAAAAABJLNFCAA|2080-10-03|2169|9432|724|2080|4|10|3|4|2080|724|9432|Thursday|2080Q4|N|N|N|2481039|2481312|2480675|2480949|N|N|N|N|N| +2481042|AAAAAAAACJLNFCAA|2080-10-04|2169|9432|724|2080|5|10|4|4|2080|724|9432|Friday|2080Q4|N|Y|N|2481039|2481312|2480676|2480950|N|N|N|N|N| +2481043|AAAAAAAADJLNFCAA|2080-10-05|2169|9432|724|2080|6|10|5|4|2080|724|9432|Saturday|2080Q4|N|Y|N|2481039|2481312|2480677|2480951|N|N|N|N|N| +2481044|AAAAAAAAEJLNFCAA|2080-10-06|2169|9432|724|2080|0|10|6|4|2080|724|9432|Sunday|2080Q4|N|N|N|2481039|2481312|2480678|2480952|N|N|N|N|N| +2481045|AAAAAAAAFJLNFCAA|2080-10-07|2169|9432|724|2080|1|10|7|4|2080|724|9432|Monday|2080Q4|N|N|N|2481039|2481312|2480679|2480953|N|N|N|N|N| +2481046|AAAAAAAAGJLNFCAA|2080-10-08|2169|9433|724|2080|2|10|8|4|2080|724|9433|Tuesday|2080Q4|N|N|N|2481039|2481312|2480680|2480954|N|N|N|N|N| +2481047|AAAAAAAAHJLNFCAA|2080-10-09|2169|9433|724|2080|3|10|9|4|2080|724|9433|Wednesday|2080Q4|N|N|N|2481039|2481312|2480681|2480955|N|N|N|N|N| +2481048|AAAAAAAAIJLNFCAA|2080-10-10|2169|9433|724|2080|4|10|10|4|2080|724|9433|Thursday|2080Q4|N|N|N|2481039|2481312|2480682|2480956|N|N|N|N|N| +2481049|AAAAAAAAJJLNFCAA|2080-10-11|2169|9433|724|2080|5|10|11|4|2080|724|9433|Friday|2080Q4|N|Y|N|2481039|2481312|2480683|2480957|N|N|N|N|N| +2481050|AAAAAAAAKJLNFCAA|2080-10-12|2169|9433|724|2080|6|10|12|4|2080|724|9433|Saturday|2080Q4|N|Y|N|2481039|2481312|2480684|2480958|N|N|N|N|N| +2481051|AAAAAAAALJLNFCAA|2080-10-13|2169|9433|724|2080|0|10|13|4|2080|724|9433|Sunday|2080Q4|N|N|N|2481039|2481312|2480685|2480959|N|N|N|N|N| +2481052|AAAAAAAAMJLNFCAA|2080-10-14|2169|9433|724|2080|1|10|14|4|2080|724|9433|Monday|2080Q4|N|N|N|2481039|2481312|2480686|2480960|N|N|N|N|N| +2481053|AAAAAAAANJLNFCAA|2080-10-15|2169|9434|724|2080|2|10|15|4|2080|724|9434|Tuesday|2080Q4|N|N|N|2481039|2481312|2480687|2480961|N|N|N|N|N| +2481054|AAAAAAAAOJLNFCAA|2080-10-16|2169|9434|724|2080|3|10|16|4|2080|724|9434|Wednesday|2080Q4|N|N|N|2481039|2481312|2480688|2480962|N|N|N|N|N| +2481055|AAAAAAAAPJLNFCAA|2080-10-17|2169|9434|724|2080|4|10|17|4|2080|724|9434|Thursday|2080Q4|N|N|N|2481039|2481312|2480689|2480963|N|N|N|N|N| +2481056|AAAAAAAAAKLNFCAA|2080-10-18|2169|9434|724|2080|5|10|18|4|2080|724|9434|Friday|2080Q4|N|Y|N|2481039|2481312|2480690|2480964|N|N|N|N|N| +2481057|AAAAAAAABKLNFCAA|2080-10-19|2169|9434|724|2080|6|10|19|4|2080|724|9434|Saturday|2080Q4|N|Y|N|2481039|2481312|2480691|2480965|N|N|N|N|N| +2481058|AAAAAAAACKLNFCAA|2080-10-20|2169|9434|724|2080|0|10|20|4|2080|724|9434|Sunday|2080Q4|N|N|N|2481039|2481312|2480692|2480966|N|N|N|N|N| +2481059|AAAAAAAADKLNFCAA|2080-10-21|2169|9434|724|2080|1|10|21|4|2080|724|9434|Monday|2080Q4|N|N|N|2481039|2481312|2480693|2480967|N|N|N|N|N| +2481060|AAAAAAAAEKLNFCAA|2080-10-22|2169|9435|724|2080|2|10|22|4|2080|724|9435|Tuesday|2080Q4|N|N|N|2481039|2481312|2480694|2480968|N|N|N|N|N| +2481061|AAAAAAAAFKLNFCAA|2080-10-23|2169|9435|724|2080|3|10|23|4|2080|724|9435|Wednesday|2080Q4|N|N|N|2481039|2481312|2480695|2480969|N|N|N|N|N| +2481062|AAAAAAAAGKLNFCAA|2080-10-24|2169|9435|724|2080|4|10|24|4|2080|724|9435|Thursday|2080Q4|N|N|N|2481039|2481312|2480696|2480970|N|N|N|N|N| +2481063|AAAAAAAAHKLNFCAA|2080-10-25|2169|9435|724|2080|5|10|25|4|2080|724|9435|Friday|2080Q4|N|Y|N|2481039|2481312|2480697|2480971|N|N|N|N|N| +2481064|AAAAAAAAIKLNFCAA|2080-10-26|2169|9435|724|2080|6|10|26|4|2080|724|9435|Saturday|2080Q4|N|Y|N|2481039|2481312|2480698|2480972|N|N|N|N|N| +2481065|AAAAAAAAJKLNFCAA|2080-10-27|2169|9435|724|2080|0|10|27|4|2080|724|9435|Sunday|2080Q4|N|N|N|2481039|2481312|2480699|2480973|N|N|N|N|N| +2481066|AAAAAAAAKKLNFCAA|2080-10-28|2169|9435|724|2080|1|10|28|4|2080|724|9435|Monday|2080Q4|N|N|N|2481039|2481312|2480700|2480974|N|N|N|N|N| +2481067|AAAAAAAALKLNFCAA|2080-10-29|2169|9436|724|2080|2|10|29|4|2080|724|9436|Tuesday|2080Q4|N|N|N|2481039|2481312|2480701|2480975|N|N|N|N|N| +2481068|AAAAAAAAMKLNFCAA|2080-10-30|2169|9436|724|2080|3|10|30|4|2080|724|9436|Wednesday|2080Q4|N|N|N|2481039|2481312|2480702|2480976|N|N|N|N|N| +2481069|AAAAAAAANKLNFCAA|2080-10-31|2169|9436|724|2080|4|10|31|4|2080|724|9436|Thursday|2080Q4|N|N|N|2481039|2481312|2480703|2480977|N|N|N|N|N| +2481070|AAAAAAAAOKLNFCAA|2080-11-01|2170|9436|724|2080|5|11|1|4|2080|724|9436|Friday|2080Q4|N|Y|N|2481070|2481374|2480704|2480978|N|N|N|N|N| +2481071|AAAAAAAAPKLNFCAA|2080-11-02|2170|9436|724|2080|6|11|2|4|2080|724|9436|Saturday|2080Q4|N|Y|N|2481070|2481374|2480705|2480979|N|N|N|N|N| +2481072|AAAAAAAAALLNFCAA|2080-11-03|2170|9436|724|2080|0|11|3|4|2080|724|9436|Sunday|2080Q4|N|N|N|2481070|2481374|2480706|2480980|N|N|N|N|N| +2481073|AAAAAAAABLLNFCAA|2080-11-04|2170|9436|724|2080|1|11|4|4|2080|724|9436|Monday|2080Q4|N|N|N|2481070|2481374|2480707|2480981|N|N|N|N|N| +2481074|AAAAAAAACLLNFCAA|2080-11-05|2170|9437|724|2080|2|11|5|4|2080|724|9437|Tuesday|2080Q4|N|N|N|2481070|2481374|2480708|2480982|N|N|N|N|N| +2481075|AAAAAAAADLLNFCAA|2080-11-06|2170|9437|724|2080|3|11|6|4|2080|724|9437|Wednesday|2080Q4|N|N|N|2481070|2481374|2480709|2480983|N|N|N|N|N| +2481076|AAAAAAAAELLNFCAA|2080-11-07|2170|9437|724|2080|4|11|7|4|2080|724|9437|Thursday|2080Q4|N|N|N|2481070|2481374|2480710|2480984|N|N|N|N|N| +2481077|AAAAAAAAFLLNFCAA|2080-11-08|2170|9437|724|2080|5|11|8|4|2080|724|9437|Friday|2080Q4|N|Y|N|2481070|2481374|2480711|2480985|N|N|N|N|N| +2481078|AAAAAAAAGLLNFCAA|2080-11-09|2170|9437|724|2080|6|11|9|4|2080|724|9437|Saturday|2080Q4|N|Y|N|2481070|2481374|2480712|2480986|N|N|N|N|N| +2481079|AAAAAAAAHLLNFCAA|2080-11-10|2170|9437|724|2080|0|11|10|4|2080|724|9437|Sunday|2080Q4|N|N|N|2481070|2481374|2480713|2480987|N|N|N|N|N| +2481080|AAAAAAAAILLNFCAA|2080-11-11|2170|9437|724|2080|1|11|11|4|2080|724|9437|Monday|2080Q4|N|N|N|2481070|2481374|2480714|2480988|N|N|N|N|N| +2481081|AAAAAAAAJLLNFCAA|2080-11-12|2170|9438|724|2080|2|11|12|4|2080|724|9438|Tuesday|2080Q4|N|N|N|2481070|2481374|2480715|2480989|N|N|N|N|N| +2481082|AAAAAAAAKLLNFCAA|2080-11-13|2170|9438|724|2080|3|11|13|4|2080|724|9438|Wednesday|2080Q4|N|N|N|2481070|2481374|2480716|2480990|N|N|N|N|N| +2481083|AAAAAAAALLLNFCAA|2080-11-14|2170|9438|724|2080|4|11|14|4|2080|724|9438|Thursday|2080Q4|N|N|N|2481070|2481374|2480717|2480991|N|N|N|N|N| +2481084|AAAAAAAAMLLNFCAA|2080-11-15|2170|9438|724|2080|5|11|15|4|2080|724|9438|Friday|2080Q4|N|Y|N|2481070|2481374|2480718|2480992|N|N|N|N|N| +2481085|AAAAAAAANLLNFCAA|2080-11-16|2170|9438|724|2080|6|11|16|4|2080|724|9438|Saturday|2080Q4|N|Y|N|2481070|2481374|2480719|2480993|N|N|N|N|N| +2481086|AAAAAAAAOLLNFCAA|2080-11-17|2170|9438|724|2080|0|11|17|4|2080|724|9438|Sunday|2080Q4|N|N|N|2481070|2481374|2480720|2480994|N|N|N|N|N| +2481087|AAAAAAAAPLLNFCAA|2080-11-18|2170|9438|724|2080|1|11|18|4|2080|724|9438|Monday|2080Q4|N|N|N|2481070|2481374|2480721|2480995|N|N|N|N|N| +2481088|AAAAAAAAAMLNFCAA|2080-11-19|2170|9439|724|2080|2|11|19|4|2080|724|9439|Tuesday|2080Q4|N|N|N|2481070|2481374|2480722|2480996|N|N|N|N|N| +2481089|AAAAAAAABMLNFCAA|2080-11-20|2170|9439|724|2080|3|11|20|4|2080|724|9439|Wednesday|2080Q4|N|N|N|2481070|2481374|2480723|2480997|N|N|N|N|N| +2481090|AAAAAAAACMLNFCAA|2080-11-21|2170|9439|724|2080|4|11|21|4|2080|724|9439|Thursday|2080Q4|N|N|N|2481070|2481374|2480724|2480998|N|N|N|N|N| +2481091|AAAAAAAADMLNFCAA|2080-11-22|2170|9439|724|2080|5|11|22|4|2080|724|9439|Friday|2080Q4|N|Y|N|2481070|2481374|2480725|2480999|N|N|N|N|N| +2481092|AAAAAAAAEMLNFCAA|2080-11-23|2170|9439|724|2080|6|11|23|4|2080|724|9439|Saturday|2080Q4|N|Y|N|2481070|2481374|2480726|2481000|N|N|N|N|N| +2481093|AAAAAAAAFMLNFCAA|2080-11-24|2170|9439|724|2080|0|11|24|4|2080|724|9439|Sunday|2080Q4|N|N|N|2481070|2481374|2480727|2481001|N|N|N|N|N| +2481094|AAAAAAAAGMLNFCAA|2080-11-25|2170|9439|724|2080|1|11|25|4|2080|724|9439|Monday|2080Q4|N|N|N|2481070|2481374|2480728|2481002|N|N|N|N|N| +2481095|AAAAAAAAHMLNFCAA|2080-11-26|2170|9440|724|2080|2|11|26|4|2080|724|9440|Tuesday|2080Q4|N|N|N|2481070|2481374|2480729|2481003|N|N|N|N|N| +2481096|AAAAAAAAIMLNFCAA|2080-11-27|2170|9440|724|2080|3|11|27|4|2080|724|9440|Wednesday|2080Q4|N|N|N|2481070|2481374|2480730|2481004|N|N|N|N|N| +2481097|AAAAAAAAJMLNFCAA|2080-11-28|2170|9440|724|2080|4|11|28|4|2080|724|9440|Thursday|2080Q4|N|N|N|2481070|2481374|2480731|2481005|N|N|N|N|N| +2481098|AAAAAAAAKMLNFCAA|2080-11-29|2170|9440|724|2080|5|11|29|4|2080|724|9440|Friday|2080Q4|N|Y|N|2481070|2481374|2480732|2481006|N|N|N|N|N| +2481099|AAAAAAAALMLNFCAA|2080-11-30|2170|9440|724|2080|6|11|30|4|2080|724|9440|Saturday|2080Q4|N|Y|N|2481070|2481374|2480733|2481007|N|N|N|N|N| +2481100|AAAAAAAAMMLNFCAA|2080-12-01|2171|9440|725|2080|0|12|1|4|2080|725|9440|Sunday|2080Q4|N|N|N|2481100|2481434|2480734|2481008|N|N|N|N|N| +2481101|AAAAAAAANMLNFCAA|2080-12-02|2171|9440|725|2080|1|12|2|4|2080|725|9440|Monday|2080Q4|N|N|N|2481100|2481434|2480735|2481009|N|N|N|N|N| +2481102|AAAAAAAAOMLNFCAA|2080-12-03|2171|9441|725|2080|2|12|3|4|2080|725|9441|Tuesday|2080Q4|N|N|N|2481100|2481434|2480736|2481010|N|N|N|N|N| +2481103|AAAAAAAAPMLNFCAA|2080-12-04|2171|9441|725|2080|3|12|4|4|2080|725|9441|Wednesday|2080Q4|N|N|N|2481100|2481434|2480737|2481011|N|N|N|N|N| +2481104|AAAAAAAAANLNFCAA|2080-12-05|2171|9441|725|2080|4|12|5|4|2080|725|9441|Thursday|2080Q4|N|N|N|2481100|2481434|2480738|2481012|N|N|N|N|N| +2481105|AAAAAAAABNLNFCAA|2080-12-06|2171|9441|725|2080|5|12|6|4|2080|725|9441|Friday|2080Q4|N|Y|N|2481100|2481434|2480739|2481013|N|N|N|N|N| +2481106|AAAAAAAACNLNFCAA|2080-12-07|2171|9441|725|2080|6|12|7|4|2080|725|9441|Saturday|2080Q4|N|Y|N|2481100|2481434|2480740|2481014|N|N|N|N|N| +2481107|AAAAAAAADNLNFCAA|2080-12-08|2171|9441|725|2080|0|12|8|4|2080|725|9441|Sunday|2080Q4|N|N|N|2481100|2481434|2480741|2481015|N|N|N|N|N| +2481108|AAAAAAAAENLNFCAA|2080-12-09|2171|9441|725|2080|1|12|9|4|2080|725|9441|Monday|2080Q4|N|N|N|2481100|2481434|2480742|2481016|N|N|N|N|N| +2481109|AAAAAAAAFNLNFCAA|2080-12-10|2171|9442|725|2080|2|12|10|4|2080|725|9442|Tuesday|2080Q4|N|N|N|2481100|2481434|2480743|2481017|N|N|N|N|N| +2481110|AAAAAAAAGNLNFCAA|2080-12-11|2171|9442|725|2080|3|12|11|4|2080|725|9442|Wednesday|2080Q4|N|N|N|2481100|2481434|2480744|2481018|N|N|N|N|N| +2481111|AAAAAAAAHNLNFCAA|2080-12-12|2171|9442|725|2080|4|12|12|4|2080|725|9442|Thursday|2080Q4|N|N|N|2481100|2481434|2480745|2481019|N|N|N|N|N| +2481112|AAAAAAAAINLNFCAA|2080-12-13|2171|9442|725|2080|5|12|13|4|2080|725|9442|Friday|2080Q4|N|Y|N|2481100|2481434|2480746|2481020|N|N|N|N|N| +2481113|AAAAAAAAJNLNFCAA|2080-12-14|2171|9442|725|2080|6|12|14|4|2080|725|9442|Saturday|2080Q4|N|Y|N|2481100|2481434|2480747|2481021|N|N|N|N|N| +2481114|AAAAAAAAKNLNFCAA|2080-12-15|2171|9442|725|2080|0|12|15|4|2080|725|9442|Sunday|2080Q4|N|N|N|2481100|2481434|2480748|2481022|N|N|N|N|N| +2481115|AAAAAAAALNLNFCAA|2080-12-16|2171|9442|725|2080|1|12|16|4|2080|725|9442|Monday|2080Q4|N|N|N|2481100|2481434|2480749|2481023|N|N|N|N|N| +2481116|AAAAAAAAMNLNFCAA|2080-12-17|2171|9443|725|2080|2|12|17|4|2080|725|9443|Tuesday|2080Q4|N|N|N|2481100|2481434|2480750|2481024|N|N|N|N|N| +2481117|AAAAAAAANNLNFCAA|2080-12-18|2171|9443|725|2080|3|12|18|4|2080|725|9443|Wednesday|2080Q4|N|N|N|2481100|2481434|2480751|2481025|N|N|N|N|N| +2481118|AAAAAAAAONLNFCAA|2080-12-19|2171|9443|725|2080|4|12|19|4|2080|725|9443|Thursday|2080Q4|N|N|N|2481100|2481434|2480752|2481026|N|N|N|N|N| +2481119|AAAAAAAAPNLNFCAA|2080-12-20|2171|9443|725|2080|5|12|20|4|2080|725|9443|Friday|2080Q4|N|Y|N|2481100|2481434|2480753|2481027|N|N|N|N|N| +2481120|AAAAAAAAAOLNFCAA|2080-12-21|2171|9443|725|2080|6|12|21|4|2080|725|9443|Saturday|2080Q4|N|Y|N|2481100|2481434|2480754|2481028|N|N|N|N|N| +2481121|AAAAAAAABOLNFCAA|2080-12-22|2171|9443|725|2080|0|12|22|4|2080|725|9443|Sunday|2080Q4|N|N|N|2481100|2481434|2480755|2481029|N|N|N|N|N| +2481122|AAAAAAAACOLNFCAA|2080-12-23|2171|9443|725|2080|1|12|23|4|2080|725|9443|Monday|2080Q4|N|N|N|2481100|2481434|2480756|2481030|N|N|N|N|N| +2481123|AAAAAAAADOLNFCAA|2080-12-24|2171|9444|725|2080|2|12|24|4|2080|725|9444|Tuesday|2080Q4|N|N|N|2481100|2481434|2480757|2481031|N|N|N|N|N| +2481124|AAAAAAAAEOLNFCAA|2080-12-25|2171|9444|725|2080|3|12|25|4|2080|725|9444|Wednesday|2080Q4|Y|N|N|2481100|2481434|2480758|2481032|N|N|N|N|N| +2481125|AAAAAAAAFOLNFCAA|2080-12-26|2171|9444|725|2080|4|12|26|4|2080|725|9444|Thursday|2080Q4|N|N|Y|2481100|2481434|2480759|2481033|N|N|N|N|N| +2481126|AAAAAAAAGOLNFCAA|2080-12-27|2171|9444|725|2080|5|12|27|4|2080|725|9444|Friday|2080Q4|N|Y|N|2481100|2481434|2480760|2481034|N|N|N|N|N| +2481127|AAAAAAAAHOLNFCAA|2080-12-28|2171|9444|725|2080|6|12|28|4|2080|725|9444|Saturday|2080Q4|N|Y|N|2481100|2481434|2480761|2481035|N|N|N|N|N| +2481128|AAAAAAAAIOLNFCAA|2080-12-29|2171|9444|725|2080|0|12|29|4|2080|725|9444|Sunday|2080Q4|N|N|N|2481100|2481434|2480762|2481036|N|N|N|N|N| +2481129|AAAAAAAAJOLNFCAA|2080-12-30|2171|9444|725|2080|1|12|30|4|2080|725|9444|Monday|2080Q4|N|N|N|2481100|2481434|2480763|2481037|N|N|N|N|N| +2481130|AAAAAAAAKOLNFCAA|2080-12-31|2171|9445|725|2080|2|12|31|4|2080|725|9445|Tuesday|2080Q4|Y|N|N|2481100|2481434|2480764|2481038|N|N|N|N|N| +2481131|AAAAAAAALOLNFCAA|2081-01-01|2172|9445|725|2081|3|1|1|1|2081|725|9445|Wednesday|2081Q1|Y|N|Y|2481131|2481130|2480765|2481039|N|N|N|N|N| +2481132|AAAAAAAAMOLNFCAA|2081-01-02|2172|9445|725|2081|4|1|2|1|2081|725|9445|Thursday|2081Q1|N|N|Y|2481131|2481130|2480766|2481040|N|N|N|N|N| +2481133|AAAAAAAANOLNFCAA|2081-01-03|2172|9445|725|2081|5|1|3|1|2081|725|9445|Friday|2081Q1|N|Y|N|2481131|2481130|2480767|2481041|N|N|N|N|N| +2481134|AAAAAAAAOOLNFCAA|2081-01-04|2172|9445|725|2081|6|1|4|1|2081|725|9445|Saturday|2081Q1|N|Y|N|2481131|2481130|2480768|2481042|N|N|N|N|N| +2481135|AAAAAAAAPOLNFCAA|2081-01-05|2172|9445|725|2081|0|1|5|1|2081|725|9445|Sunday|2081Q1|N|N|N|2481131|2481130|2480769|2481043|N|N|N|N|N| +2481136|AAAAAAAAAPLNFCAA|2081-01-06|2172|9445|725|2081|1|1|6|1|2081|725|9445|Monday|2081Q1|N|N|N|2481131|2481130|2480770|2481044|N|N|N|N|N| +2481137|AAAAAAAABPLNFCAA|2081-01-07|2172|9446|725|2081|2|1|7|1|2081|725|9446|Tuesday|2081Q1|N|N|N|2481131|2481130|2480771|2481045|N|N|N|N|N| +2481138|AAAAAAAACPLNFCAA|2081-01-08|2172|9446|725|2081|3|1|8|1|2081|725|9446|Wednesday|2081Q1|N|N|N|2481131|2481130|2480772|2481046|N|N|N|N|N| +2481139|AAAAAAAADPLNFCAA|2081-01-09|2172|9446|725|2081|4|1|9|1|2081|725|9446|Thursday|2081Q1|N|N|N|2481131|2481130|2480773|2481047|N|N|N|N|N| +2481140|AAAAAAAAEPLNFCAA|2081-01-10|2172|9446|725|2081|5|1|10|1|2081|725|9446|Friday|2081Q1|N|Y|N|2481131|2481130|2480774|2481048|N|N|N|N|N| +2481141|AAAAAAAAFPLNFCAA|2081-01-11|2172|9446|725|2081|6|1|11|1|2081|725|9446|Saturday|2081Q1|N|Y|N|2481131|2481130|2480775|2481049|N|N|N|N|N| +2481142|AAAAAAAAGPLNFCAA|2081-01-12|2172|9446|725|2081|0|1|12|1|2081|725|9446|Sunday|2081Q1|N|N|N|2481131|2481130|2480776|2481050|N|N|N|N|N| +2481143|AAAAAAAAHPLNFCAA|2081-01-13|2172|9446|725|2081|1|1|13|1|2081|725|9446|Monday|2081Q1|N|N|N|2481131|2481130|2480777|2481051|N|N|N|N|N| +2481144|AAAAAAAAIPLNFCAA|2081-01-14|2172|9447|725|2081|2|1|14|1|2081|725|9447|Tuesday|2081Q1|N|N|N|2481131|2481130|2480778|2481052|N|N|N|N|N| +2481145|AAAAAAAAJPLNFCAA|2081-01-15|2172|9447|725|2081|3|1|15|1|2081|725|9447|Wednesday|2081Q1|N|N|N|2481131|2481130|2480779|2481053|N|N|N|N|N| +2481146|AAAAAAAAKPLNFCAA|2081-01-16|2172|9447|725|2081|4|1|16|1|2081|725|9447|Thursday|2081Q1|N|N|N|2481131|2481130|2480780|2481054|N|N|N|N|N| +2481147|AAAAAAAALPLNFCAA|2081-01-17|2172|9447|725|2081|5|1|17|1|2081|725|9447|Friday|2081Q1|N|Y|N|2481131|2481130|2480781|2481055|N|N|N|N|N| +2481148|AAAAAAAAMPLNFCAA|2081-01-18|2172|9447|725|2081|6|1|18|1|2081|725|9447|Saturday|2081Q1|N|Y|N|2481131|2481130|2480782|2481056|N|N|N|N|N| +2481149|AAAAAAAANPLNFCAA|2081-01-19|2172|9447|725|2081|0|1|19|1|2081|725|9447|Sunday|2081Q1|N|N|N|2481131|2481130|2480783|2481057|N|N|N|N|N| +2481150|AAAAAAAAOPLNFCAA|2081-01-20|2172|9447|725|2081|1|1|20|1|2081|725|9447|Monday|2081Q1|N|N|N|2481131|2481130|2480784|2481058|N|N|N|N|N| +2481151|AAAAAAAAPPLNFCAA|2081-01-21|2172|9448|725|2081|2|1|21|1|2081|725|9448|Tuesday|2081Q1|N|N|N|2481131|2481130|2480785|2481059|N|N|N|N|N| +2481152|AAAAAAAAAAMNFCAA|2081-01-22|2172|9448|725|2081|3|1|22|1|2081|725|9448|Wednesday|2081Q1|N|N|N|2481131|2481130|2480786|2481060|N|N|N|N|N| +2481153|AAAAAAAABAMNFCAA|2081-01-23|2172|9448|725|2081|4|1|23|1|2081|725|9448|Thursday|2081Q1|N|N|N|2481131|2481130|2480787|2481061|N|N|N|N|N| +2481154|AAAAAAAACAMNFCAA|2081-01-24|2172|9448|725|2081|5|1|24|1|2081|725|9448|Friday|2081Q1|N|Y|N|2481131|2481130|2480788|2481062|N|N|N|N|N| +2481155|AAAAAAAADAMNFCAA|2081-01-25|2172|9448|725|2081|6|1|25|1|2081|725|9448|Saturday|2081Q1|N|Y|N|2481131|2481130|2480789|2481063|N|N|N|N|N| +2481156|AAAAAAAAEAMNFCAA|2081-01-26|2172|9448|725|2081|0|1|26|1|2081|725|9448|Sunday|2081Q1|N|N|N|2481131|2481130|2480790|2481064|N|N|N|N|N| +2481157|AAAAAAAAFAMNFCAA|2081-01-27|2172|9448|725|2081|1|1|27|1|2081|725|9448|Monday|2081Q1|N|N|N|2481131|2481130|2480791|2481065|N|N|N|N|N| +2481158|AAAAAAAAGAMNFCAA|2081-01-28|2172|9449|725|2081|2|1|28|1|2081|725|9449|Tuesday|2081Q1|N|N|N|2481131|2481130|2480792|2481066|N|N|N|N|N| +2481159|AAAAAAAAHAMNFCAA|2081-01-29|2172|9449|725|2081|3|1|29|1|2081|725|9449|Wednesday|2081Q1|N|N|N|2481131|2481130|2480793|2481067|N|N|N|N|N| +2481160|AAAAAAAAIAMNFCAA|2081-01-30|2172|9449|725|2081|4|1|30|1|2081|725|9449|Thursday|2081Q1|N|N|N|2481131|2481130|2480794|2481068|N|N|N|N|N| +2481161|AAAAAAAAJAMNFCAA|2081-01-31|2172|9449|725|2081|5|1|31|1|2081|725|9449|Friday|2081Q1|N|Y|N|2481131|2481130|2480795|2481069|N|N|N|N|N| +2481162|AAAAAAAAKAMNFCAA|2081-02-01|2173|9449|725|2081|6|2|1|1|2081|725|9449|Saturday|2081Q1|N|Y|N|2481162|2481192|2480796|2481070|N|N|N|N|N| +2481163|AAAAAAAALAMNFCAA|2081-02-02|2173|9449|725|2081|0|2|2|1|2081|725|9449|Sunday|2081Q1|N|N|N|2481162|2481192|2480797|2481071|N|N|N|N|N| +2481164|AAAAAAAAMAMNFCAA|2081-02-03|2173|9449|725|2081|1|2|3|1|2081|725|9449|Monday|2081Q1|N|N|N|2481162|2481192|2480798|2481072|N|N|N|N|N| +2481165|AAAAAAAANAMNFCAA|2081-02-04|2173|9450|725|2081|2|2|4|1|2081|725|9450|Tuesday|2081Q1|N|N|N|2481162|2481192|2480799|2481073|N|N|N|N|N| +2481166|AAAAAAAAOAMNFCAA|2081-02-05|2173|9450|725|2081|3|2|5|1|2081|725|9450|Wednesday|2081Q1|N|N|N|2481162|2481192|2480800|2481074|N|N|N|N|N| +2481167|AAAAAAAAPAMNFCAA|2081-02-06|2173|9450|725|2081|4|2|6|1|2081|725|9450|Thursday|2081Q1|N|N|N|2481162|2481192|2480801|2481075|N|N|N|N|N| +2481168|AAAAAAAAABMNFCAA|2081-02-07|2173|9450|725|2081|5|2|7|1|2081|725|9450|Friday|2081Q1|N|Y|N|2481162|2481192|2480802|2481076|N|N|N|N|N| +2481169|AAAAAAAABBMNFCAA|2081-02-08|2173|9450|725|2081|6|2|8|1|2081|725|9450|Saturday|2081Q1|N|Y|N|2481162|2481192|2480803|2481077|N|N|N|N|N| +2481170|AAAAAAAACBMNFCAA|2081-02-09|2173|9450|725|2081|0|2|9|1|2081|725|9450|Sunday|2081Q1|N|N|N|2481162|2481192|2480804|2481078|N|N|N|N|N| +2481171|AAAAAAAADBMNFCAA|2081-02-10|2173|9450|725|2081|1|2|10|1|2081|725|9450|Monday|2081Q1|N|N|N|2481162|2481192|2480805|2481079|N|N|N|N|N| +2481172|AAAAAAAAEBMNFCAA|2081-02-11|2173|9451|725|2081|2|2|11|1|2081|725|9451|Tuesday|2081Q1|N|N|N|2481162|2481192|2480806|2481080|N|N|N|N|N| +2481173|AAAAAAAAFBMNFCAA|2081-02-12|2173|9451|725|2081|3|2|12|1|2081|725|9451|Wednesday|2081Q1|N|N|N|2481162|2481192|2480807|2481081|N|N|N|N|N| +2481174|AAAAAAAAGBMNFCAA|2081-02-13|2173|9451|725|2081|4|2|13|1|2081|725|9451|Thursday|2081Q1|N|N|N|2481162|2481192|2480808|2481082|N|N|N|N|N| +2481175|AAAAAAAAHBMNFCAA|2081-02-14|2173|9451|725|2081|5|2|14|1|2081|725|9451|Friday|2081Q1|N|Y|N|2481162|2481192|2480809|2481083|N|N|N|N|N| +2481176|AAAAAAAAIBMNFCAA|2081-02-15|2173|9451|725|2081|6|2|15|1|2081|725|9451|Saturday|2081Q1|N|Y|N|2481162|2481192|2480810|2481084|N|N|N|N|N| +2481177|AAAAAAAAJBMNFCAA|2081-02-16|2173|9451|725|2081|0|2|16|1|2081|725|9451|Sunday|2081Q1|N|N|N|2481162|2481192|2480811|2481085|N|N|N|N|N| +2481178|AAAAAAAAKBMNFCAA|2081-02-17|2173|9451|725|2081|1|2|17|1|2081|725|9451|Monday|2081Q1|N|N|N|2481162|2481192|2480812|2481086|N|N|N|N|N| +2481179|AAAAAAAALBMNFCAA|2081-02-18|2173|9452|725|2081|2|2|18|1|2081|725|9452|Tuesday|2081Q1|N|N|N|2481162|2481192|2480813|2481087|N|N|N|N|N| +2481180|AAAAAAAAMBMNFCAA|2081-02-19|2173|9452|725|2081|3|2|19|1|2081|725|9452|Wednesday|2081Q1|N|N|N|2481162|2481192|2480814|2481088|N|N|N|N|N| +2481181|AAAAAAAANBMNFCAA|2081-02-20|2173|9452|725|2081|4|2|20|1|2081|725|9452|Thursday|2081Q1|N|N|N|2481162|2481192|2480815|2481089|N|N|N|N|N| +2481182|AAAAAAAAOBMNFCAA|2081-02-21|2173|9452|725|2081|5|2|21|1|2081|725|9452|Friday|2081Q1|N|Y|N|2481162|2481192|2480816|2481090|N|N|N|N|N| +2481183|AAAAAAAAPBMNFCAA|2081-02-22|2173|9452|725|2081|6|2|22|1|2081|725|9452|Saturday|2081Q1|N|Y|N|2481162|2481192|2480817|2481091|N|N|N|N|N| +2481184|AAAAAAAAACMNFCAA|2081-02-23|2173|9452|725|2081|0|2|23|1|2081|725|9452|Sunday|2081Q1|N|N|N|2481162|2481192|2480818|2481092|N|N|N|N|N| +2481185|AAAAAAAABCMNFCAA|2081-02-24|2173|9452|725|2081|1|2|24|1|2081|725|9452|Monday|2081Q1|N|N|N|2481162|2481192|2480819|2481093|N|N|N|N|N| +2481186|AAAAAAAACCMNFCAA|2081-02-25|2173|9453|725|2081|2|2|25|1|2081|725|9453|Tuesday|2081Q1|N|N|N|2481162|2481192|2480820|2481094|N|N|N|N|N| +2481187|AAAAAAAADCMNFCAA|2081-02-26|2173|9453|725|2081|3|2|26|1|2081|725|9453|Wednesday|2081Q1|N|N|N|2481162|2481192|2480821|2481095|N|N|N|N|N| +2481188|AAAAAAAAECMNFCAA|2081-02-27|2173|9453|725|2081|4|2|27|1|2081|725|9453|Thursday|2081Q1|N|N|N|2481162|2481192|2480822|2481096|N|N|N|N|N| +2481189|AAAAAAAAFCMNFCAA|2081-02-28|2173|9453|725|2081|5|2|28|1|2081|725|9453|Friday|2081Q1|N|Y|N|2481162|2481192|2480823|2481097|N|N|N|N|N| +2481190|AAAAAAAAGCMNFCAA|2081-03-01|2174|9453|726|2081|6|3|1|1|2081|726|9453|Saturday|2081Q1|N|Y|N|2481190|2481248|2480825|2481098|N|N|N|N|N| +2481191|AAAAAAAAHCMNFCAA|2081-03-02|2174|9453|726|2081|0|3|2|1|2081|726|9453|Sunday|2081Q1|N|N|N|2481190|2481248|2480826|2481099|N|N|N|N|N| +2481192|AAAAAAAAICMNFCAA|2081-03-03|2174|9453|726|2081|1|3|3|1|2081|726|9453|Monday|2081Q1|N|N|N|2481190|2481248|2480827|2481100|N|N|N|N|N| +2481193|AAAAAAAAJCMNFCAA|2081-03-04|2174|9454|726|2081|2|3|4|1|2081|726|9454|Tuesday|2081Q1|N|N|N|2481190|2481248|2480828|2481101|N|N|N|N|N| +2481194|AAAAAAAAKCMNFCAA|2081-03-05|2174|9454|726|2081|3|3|5|1|2081|726|9454|Wednesday|2081Q1|N|N|N|2481190|2481248|2480829|2481102|N|N|N|N|N| +2481195|AAAAAAAALCMNFCAA|2081-03-06|2174|9454|726|2081|4|3|6|1|2081|726|9454|Thursday|2081Q1|N|N|N|2481190|2481248|2480830|2481103|N|N|N|N|N| +2481196|AAAAAAAAMCMNFCAA|2081-03-07|2174|9454|726|2081|5|3|7|1|2081|726|9454|Friday|2081Q1|N|Y|N|2481190|2481248|2480831|2481104|N|N|N|N|N| +2481197|AAAAAAAANCMNFCAA|2081-03-08|2174|9454|726|2081|6|3|8|1|2081|726|9454|Saturday|2081Q1|N|Y|N|2481190|2481248|2480832|2481105|N|N|N|N|N| +2481198|AAAAAAAAOCMNFCAA|2081-03-09|2174|9454|726|2081|0|3|9|1|2081|726|9454|Sunday|2081Q1|N|N|N|2481190|2481248|2480833|2481106|N|N|N|N|N| +2481199|AAAAAAAAPCMNFCAA|2081-03-10|2174|9454|726|2081|1|3|10|1|2081|726|9454|Monday|2081Q1|N|N|N|2481190|2481248|2480834|2481107|N|N|N|N|N| +2481200|AAAAAAAAADMNFCAA|2081-03-11|2174|9455|726|2081|2|3|11|1|2081|726|9455|Tuesday|2081Q1|N|N|N|2481190|2481248|2480835|2481108|N|N|N|N|N| +2481201|AAAAAAAABDMNFCAA|2081-03-12|2174|9455|726|2081|3|3|12|1|2081|726|9455|Wednesday|2081Q1|N|N|N|2481190|2481248|2480836|2481109|N|N|N|N|N| +2481202|AAAAAAAACDMNFCAA|2081-03-13|2174|9455|726|2081|4|3|13|1|2081|726|9455|Thursday|2081Q1|N|N|N|2481190|2481248|2480837|2481110|N|N|N|N|N| +2481203|AAAAAAAADDMNFCAA|2081-03-14|2174|9455|726|2081|5|3|14|1|2081|726|9455|Friday|2081Q1|N|Y|N|2481190|2481248|2480838|2481111|N|N|N|N|N| +2481204|AAAAAAAAEDMNFCAA|2081-03-15|2174|9455|726|2081|6|3|15|1|2081|726|9455|Saturday|2081Q1|N|Y|N|2481190|2481248|2480839|2481112|N|N|N|N|N| +2481205|AAAAAAAAFDMNFCAA|2081-03-16|2174|9455|726|2081|0|3|16|1|2081|726|9455|Sunday|2081Q1|N|N|N|2481190|2481248|2480840|2481113|N|N|N|N|N| +2481206|AAAAAAAAGDMNFCAA|2081-03-17|2174|9455|726|2081|1|3|17|1|2081|726|9455|Monday|2081Q1|N|N|N|2481190|2481248|2480841|2481114|N|N|N|N|N| +2481207|AAAAAAAAHDMNFCAA|2081-03-18|2174|9456|726|2081|2|3|18|1|2081|726|9456|Tuesday|2081Q1|N|N|N|2481190|2481248|2480842|2481115|N|N|N|N|N| +2481208|AAAAAAAAIDMNFCAA|2081-03-19|2174|9456|726|2081|3|3|19|1|2081|726|9456|Wednesday|2081Q1|N|N|N|2481190|2481248|2480843|2481116|N|N|N|N|N| +2481209|AAAAAAAAJDMNFCAA|2081-03-20|2174|9456|726|2081|4|3|20|1|2081|726|9456|Thursday|2081Q1|N|N|N|2481190|2481248|2480844|2481117|N|N|N|N|N| +2481210|AAAAAAAAKDMNFCAA|2081-03-21|2174|9456|726|2081|5|3|21|1|2081|726|9456|Friday|2081Q1|N|Y|N|2481190|2481248|2480845|2481118|N|N|N|N|N| +2481211|AAAAAAAALDMNFCAA|2081-03-22|2174|9456|726|2081|6|3|22|1|2081|726|9456|Saturday|2081Q1|N|Y|N|2481190|2481248|2480846|2481119|N|N|N|N|N| +2481212|AAAAAAAAMDMNFCAA|2081-03-23|2174|9456|726|2081|0|3|23|1|2081|726|9456|Sunday|2081Q1|N|N|N|2481190|2481248|2480847|2481120|N|N|N|N|N| +2481213|AAAAAAAANDMNFCAA|2081-03-24|2174|9456|726|2081|1|3|24|1|2081|726|9456|Monday|2081Q1|N|N|N|2481190|2481248|2480848|2481121|N|N|N|N|N| +2481214|AAAAAAAAODMNFCAA|2081-03-25|2174|9457|726|2081|2|3|25|1|2081|726|9457|Tuesday|2081Q1|N|N|N|2481190|2481248|2480849|2481122|N|N|N|N|N| +2481215|AAAAAAAAPDMNFCAA|2081-03-26|2174|9457|726|2081|3|3|26|1|2081|726|9457|Wednesday|2081Q1|N|N|N|2481190|2481248|2480850|2481123|N|N|N|N|N| +2481216|AAAAAAAAAEMNFCAA|2081-03-27|2174|9457|726|2081|4|3|27|1|2081|726|9457|Thursday|2081Q1|N|N|N|2481190|2481248|2480851|2481124|N|N|N|N|N| +2481217|AAAAAAAABEMNFCAA|2081-03-28|2174|9457|726|2081|5|3|28|1|2081|726|9457|Friday|2081Q1|N|Y|N|2481190|2481248|2480852|2481125|N|N|N|N|N| +2481218|AAAAAAAACEMNFCAA|2081-03-29|2174|9457|726|2081|6|3|29|1|2081|726|9457|Saturday|2081Q1|N|Y|N|2481190|2481248|2480853|2481126|N|N|N|N|N| +2481219|AAAAAAAADEMNFCAA|2081-03-30|2174|9457|726|2081|0|3|30|1|2081|726|9457|Sunday|2081Q1|N|N|N|2481190|2481248|2480854|2481127|N|N|N|N|N| +2481220|AAAAAAAAEEMNFCAA|2081-03-31|2174|9457|726|2081|1|3|31|1|2081|726|9457|Monday|2081Q1|N|N|N|2481190|2481248|2480855|2481128|N|N|N|N|N| +2481221|AAAAAAAAFEMNFCAA|2081-04-01|2175|9458|726|2081|2|4|1|1|2081|726|9458|Tuesday|2081Q1|N|N|N|2481221|2481310|2480856|2481131|N|N|N|N|N| +2481222|AAAAAAAAGEMNFCAA|2081-04-02|2175|9458|726|2081|3|4|2|2|2081|726|9458|Wednesday|2081Q2|N|N|N|2481221|2481310|2480857|2481132|N|N|N|N|N| +2481223|AAAAAAAAHEMNFCAA|2081-04-03|2175|9458|726|2081|4|4|3|2|2081|726|9458|Thursday|2081Q2|N|N|N|2481221|2481310|2480858|2481133|N|N|N|N|N| +2481224|AAAAAAAAIEMNFCAA|2081-04-04|2175|9458|726|2081|5|4|4|2|2081|726|9458|Friday|2081Q2|N|Y|N|2481221|2481310|2480859|2481134|N|N|N|N|N| +2481225|AAAAAAAAJEMNFCAA|2081-04-05|2175|9458|726|2081|6|4|5|2|2081|726|9458|Saturday|2081Q2|N|Y|N|2481221|2481310|2480860|2481135|N|N|N|N|N| +2481226|AAAAAAAAKEMNFCAA|2081-04-06|2175|9458|726|2081|0|4|6|2|2081|726|9458|Sunday|2081Q2|N|N|N|2481221|2481310|2480861|2481136|N|N|N|N|N| +2481227|AAAAAAAALEMNFCAA|2081-04-07|2175|9458|726|2081|1|4|7|2|2081|726|9458|Monday|2081Q2|N|N|N|2481221|2481310|2480862|2481137|N|N|N|N|N| +2481228|AAAAAAAAMEMNFCAA|2081-04-08|2175|9459|726|2081|2|4|8|2|2081|726|9459|Tuesday|2081Q2|N|N|N|2481221|2481310|2480863|2481138|N|N|N|N|N| +2481229|AAAAAAAANEMNFCAA|2081-04-09|2175|9459|726|2081|3|4|9|2|2081|726|9459|Wednesday|2081Q2|N|N|N|2481221|2481310|2480864|2481139|N|N|N|N|N| +2481230|AAAAAAAAOEMNFCAA|2081-04-10|2175|9459|726|2081|4|4|10|2|2081|726|9459|Thursday|2081Q2|N|N|N|2481221|2481310|2480865|2481140|N|N|N|N|N| +2481231|AAAAAAAAPEMNFCAA|2081-04-11|2175|9459|726|2081|5|4|11|2|2081|726|9459|Friday|2081Q2|N|Y|N|2481221|2481310|2480866|2481141|N|N|N|N|N| +2481232|AAAAAAAAAFMNFCAA|2081-04-12|2175|9459|726|2081|6|4|12|2|2081|726|9459|Saturday|2081Q2|N|Y|N|2481221|2481310|2480867|2481142|N|N|N|N|N| +2481233|AAAAAAAABFMNFCAA|2081-04-13|2175|9459|726|2081|0|4|13|2|2081|726|9459|Sunday|2081Q2|N|N|N|2481221|2481310|2480868|2481143|N|N|N|N|N| +2481234|AAAAAAAACFMNFCAA|2081-04-14|2175|9459|726|2081|1|4|14|2|2081|726|9459|Monday|2081Q2|N|N|N|2481221|2481310|2480869|2481144|N|N|N|N|N| +2481235|AAAAAAAADFMNFCAA|2081-04-15|2175|9460|726|2081|2|4|15|2|2081|726|9460|Tuesday|2081Q2|N|N|N|2481221|2481310|2480870|2481145|N|N|N|N|N| +2481236|AAAAAAAAEFMNFCAA|2081-04-16|2175|9460|726|2081|3|4|16|2|2081|726|9460|Wednesday|2081Q2|N|N|N|2481221|2481310|2480871|2481146|N|N|N|N|N| +2481237|AAAAAAAAFFMNFCAA|2081-04-17|2175|9460|726|2081|4|4|17|2|2081|726|9460|Thursday|2081Q2|N|N|N|2481221|2481310|2480872|2481147|N|N|N|N|N| +2481238|AAAAAAAAGFMNFCAA|2081-04-18|2175|9460|726|2081|5|4|18|2|2081|726|9460|Friday|2081Q2|N|Y|N|2481221|2481310|2480873|2481148|N|N|N|N|N| +2481239|AAAAAAAAHFMNFCAA|2081-04-19|2175|9460|726|2081|6|4|19|2|2081|726|9460|Saturday|2081Q2|N|Y|N|2481221|2481310|2480874|2481149|N|N|N|N|N| +2481240|AAAAAAAAIFMNFCAA|2081-04-20|2175|9460|726|2081|0|4|20|2|2081|726|9460|Sunday|2081Q2|N|N|N|2481221|2481310|2480875|2481150|N|N|N|N|N| +2481241|AAAAAAAAJFMNFCAA|2081-04-21|2175|9460|726|2081|1|4|21|2|2081|726|9460|Monday|2081Q2|N|N|N|2481221|2481310|2480876|2481151|N|N|N|N|N| +2481242|AAAAAAAAKFMNFCAA|2081-04-22|2175|9461|726|2081|2|4|22|2|2081|726|9461|Tuesday|2081Q2|N|N|N|2481221|2481310|2480877|2481152|N|N|N|N|N| +2481243|AAAAAAAALFMNFCAA|2081-04-23|2175|9461|726|2081|3|4|23|2|2081|726|9461|Wednesday|2081Q2|N|N|N|2481221|2481310|2480878|2481153|N|N|N|N|N| +2481244|AAAAAAAAMFMNFCAA|2081-04-24|2175|9461|726|2081|4|4|24|2|2081|726|9461|Thursday|2081Q2|N|N|N|2481221|2481310|2480879|2481154|N|N|N|N|N| +2481245|AAAAAAAANFMNFCAA|2081-04-25|2175|9461|726|2081|5|4|25|2|2081|726|9461|Friday|2081Q2|N|Y|N|2481221|2481310|2480880|2481155|N|N|N|N|N| +2481246|AAAAAAAAOFMNFCAA|2081-04-26|2175|9461|726|2081|6|4|26|2|2081|726|9461|Saturday|2081Q2|N|Y|N|2481221|2481310|2480881|2481156|N|N|N|N|N| +2481247|AAAAAAAAPFMNFCAA|2081-04-27|2175|9461|726|2081|0|4|27|2|2081|726|9461|Sunday|2081Q2|N|N|N|2481221|2481310|2480882|2481157|N|N|N|N|N| +2481248|AAAAAAAAAGMNFCAA|2081-04-28|2175|9461|726|2081|1|4|28|2|2081|726|9461|Monday|2081Q2|N|N|N|2481221|2481310|2480883|2481158|N|N|N|N|N| +2481249|AAAAAAAABGMNFCAA|2081-04-29|2175|9462|726|2081|2|4|29|2|2081|726|9462|Tuesday|2081Q2|N|N|N|2481221|2481310|2480884|2481159|N|N|N|N|N| +2481250|AAAAAAAACGMNFCAA|2081-04-30|2175|9462|726|2081|3|4|30|2|2081|726|9462|Wednesday|2081Q2|N|N|N|2481221|2481310|2480885|2481160|N|N|N|N|N| +2481251|AAAAAAAADGMNFCAA|2081-05-01|2176|9462|726|2081|4|5|1|2|2081|726|9462|Thursday|2081Q2|N|N|N|2481251|2481370|2480886|2481161|N|N|N|N|N| +2481252|AAAAAAAAEGMNFCAA|2081-05-02|2176|9462|726|2081|5|5|2|2|2081|726|9462|Friday|2081Q2|N|Y|N|2481251|2481370|2480887|2481162|N|N|N|N|N| +2481253|AAAAAAAAFGMNFCAA|2081-05-03|2176|9462|726|2081|6|5|3|2|2081|726|9462|Saturday|2081Q2|N|Y|N|2481251|2481370|2480888|2481163|N|N|N|N|N| +2481254|AAAAAAAAGGMNFCAA|2081-05-04|2176|9462|726|2081|0|5|4|2|2081|726|9462|Sunday|2081Q2|N|N|N|2481251|2481370|2480889|2481164|N|N|N|N|N| +2481255|AAAAAAAAHGMNFCAA|2081-05-05|2176|9462|726|2081|1|5|5|2|2081|726|9462|Monday|2081Q2|N|N|N|2481251|2481370|2480890|2481165|N|N|N|N|N| +2481256|AAAAAAAAIGMNFCAA|2081-05-06|2176|9463|726|2081|2|5|6|2|2081|726|9463|Tuesday|2081Q2|N|N|N|2481251|2481370|2480891|2481166|N|N|N|N|N| +2481257|AAAAAAAAJGMNFCAA|2081-05-07|2176|9463|726|2081|3|5|7|2|2081|726|9463|Wednesday|2081Q2|N|N|N|2481251|2481370|2480892|2481167|N|N|N|N|N| +2481258|AAAAAAAAKGMNFCAA|2081-05-08|2176|9463|726|2081|4|5|8|2|2081|726|9463|Thursday|2081Q2|N|N|N|2481251|2481370|2480893|2481168|N|N|N|N|N| +2481259|AAAAAAAALGMNFCAA|2081-05-09|2176|9463|726|2081|5|5|9|2|2081|726|9463|Friday|2081Q2|N|Y|N|2481251|2481370|2480894|2481169|N|N|N|N|N| +2481260|AAAAAAAAMGMNFCAA|2081-05-10|2176|9463|726|2081|6|5|10|2|2081|726|9463|Saturday|2081Q2|N|Y|N|2481251|2481370|2480895|2481170|N|N|N|N|N| +2481261|AAAAAAAANGMNFCAA|2081-05-11|2176|9463|726|2081|0|5|11|2|2081|726|9463|Sunday|2081Q2|N|N|N|2481251|2481370|2480896|2481171|N|N|N|N|N| +2481262|AAAAAAAAOGMNFCAA|2081-05-12|2176|9463|726|2081|1|5|12|2|2081|726|9463|Monday|2081Q2|N|N|N|2481251|2481370|2480897|2481172|N|N|N|N|N| +2481263|AAAAAAAAPGMNFCAA|2081-05-13|2176|9464|726|2081|2|5|13|2|2081|726|9464|Tuesday|2081Q2|N|N|N|2481251|2481370|2480898|2481173|N|N|N|N|N| +2481264|AAAAAAAAAHMNFCAA|2081-05-14|2176|9464|726|2081|3|5|14|2|2081|726|9464|Wednesday|2081Q2|N|N|N|2481251|2481370|2480899|2481174|N|N|N|N|N| +2481265|AAAAAAAABHMNFCAA|2081-05-15|2176|9464|726|2081|4|5|15|2|2081|726|9464|Thursday|2081Q2|N|N|N|2481251|2481370|2480900|2481175|N|N|N|N|N| +2481266|AAAAAAAACHMNFCAA|2081-05-16|2176|9464|726|2081|5|5|16|2|2081|726|9464|Friday|2081Q2|N|Y|N|2481251|2481370|2480901|2481176|N|N|N|N|N| +2481267|AAAAAAAADHMNFCAA|2081-05-17|2176|9464|726|2081|6|5|17|2|2081|726|9464|Saturday|2081Q2|N|Y|N|2481251|2481370|2480902|2481177|N|N|N|N|N| +2481268|AAAAAAAAEHMNFCAA|2081-05-18|2176|9464|726|2081|0|5|18|2|2081|726|9464|Sunday|2081Q2|N|N|N|2481251|2481370|2480903|2481178|N|N|N|N|N| +2481269|AAAAAAAAFHMNFCAA|2081-05-19|2176|9464|726|2081|1|5|19|2|2081|726|9464|Monday|2081Q2|N|N|N|2481251|2481370|2480904|2481179|N|N|N|N|N| +2481270|AAAAAAAAGHMNFCAA|2081-05-20|2176|9465|726|2081|2|5|20|2|2081|726|9465|Tuesday|2081Q2|N|N|N|2481251|2481370|2480905|2481180|N|N|N|N|N| +2481271|AAAAAAAAHHMNFCAA|2081-05-21|2176|9465|726|2081|3|5|21|2|2081|726|9465|Wednesday|2081Q2|N|N|N|2481251|2481370|2480906|2481181|N|N|N|N|N| +2481272|AAAAAAAAIHMNFCAA|2081-05-22|2176|9465|726|2081|4|5|22|2|2081|726|9465|Thursday|2081Q2|N|N|N|2481251|2481370|2480907|2481182|N|N|N|N|N| +2481273|AAAAAAAAJHMNFCAA|2081-05-23|2176|9465|726|2081|5|5|23|2|2081|726|9465|Friday|2081Q2|N|Y|N|2481251|2481370|2480908|2481183|N|N|N|N|N| +2481274|AAAAAAAAKHMNFCAA|2081-05-24|2176|9465|726|2081|6|5|24|2|2081|726|9465|Saturday|2081Q2|N|Y|N|2481251|2481370|2480909|2481184|N|N|N|N|N| +2481275|AAAAAAAALHMNFCAA|2081-05-25|2176|9465|726|2081|0|5|25|2|2081|726|9465|Sunday|2081Q2|N|N|N|2481251|2481370|2480910|2481185|N|N|N|N|N| +2481276|AAAAAAAAMHMNFCAA|2081-05-26|2176|9465|726|2081|1|5|26|2|2081|726|9465|Monday|2081Q2|N|N|N|2481251|2481370|2480911|2481186|N|N|N|N|N| +2481277|AAAAAAAANHMNFCAA|2081-05-27|2176|9466|726|2081|2|5|27|2|2081|726|9466|Tuesday|2081Q2|N|N|N|2481251|2481370|2480912|2481187|N|N|N|N|N| +2481278|AAAAAAAAOHMNFCAA|2081-05-28|2176|9466|726|2081|3|5|28|2|2081|726|9466|Wednesday|2081Q2|N|N|N|2481251|2481370|2480913|2481188|N|N|N|N|N| +2481279|AAAAAAAAPHMNFCAA|2081-05-29|2176|9466|726|2081|4|5|29|2|2081|726|9466|Thursday|2081Q2|N|N|N|2481251|2481370|2480914|2481189|N|N|N|N|N| +2481280|AAAAAAAAAIMNFCAA|2081-05-30|2176|9466|726|2081|5|5|30|2|2081|726|9466|Friday|2081Q2|N|Y|N|2481251|2481370|2480915|2481190|N|N|N|N|N| +2481281|AAAAAAAABIMNFCAA|2081-05-31|2176|9466|726|2081|6|5|31|2|2081|726|9466|Saturday|2081Q2|N|Y|N|2481251|2481370|2480916|2481191|N|N|N|N|N| +2481282|AAAAAAAACIMNFCAA|2081-06-01|2177|9466|727|2081|0|6|1|2|2081|727|9466|Sunday|2081Q2|N|N|N|2481282|2481432|2480917|2481192|N|N|N|N|N| +2481283|AAAAAAAADIMNFCAA|2081-06-02|2177|9466|727|2081|1|6|2|2|2081|727|9466|Monday|2081Q2|N|N|N|2481282|2481432|2480918|2481193|N|N|N|N|N| +2481284|AAAAAAAAEIMNFCAA|2081-06-03|2177|9467|727|2081|2|6|3|2|2081|727|9467|Tuesday|2081Q2|N|N|N|2481282|2481432|2480919|2481194|N|N|N|N|N| +2481285|AAAAAAAAFIMNFCAA|2081-06-04|2177|9467|727|2081|3|6|4|2|2081|727|9467|Wednesday|2081Q2|N|N|N|2481282|2481432|2480920|2481195|N|N|N|N|N| +2481286|AAAAAAAAGIMNFCAA|2081-06-05|2177|9467|727|2081|4|6|5|2|2081|727|9467|Thursday|2081Q2|N|N|N|2481282|2481432|2480921|2481196|N|N|N|N|N| +2481287|AAAAAAAAHIMNFCAA|2081-06-06|2177|9467|727|2081|5|6|6|2|2081|727|9467|Friday|2081Q2|N|Y|N|2481282|2481432|2480922|2481197|N|N|N|N|N| +2481288|AAAAAAAAIIMNFCAA|2081-06-07|2177|9467|727|2081|6|6|7|2|2081|727|9467|Saturday|2081Q2|N|Y|N|2481282|2481432|2480923|2481198|N|N|N|N|N| +2481289|AAAAAAAAJIMNFCAA|2081-06-08|2177|9467|727|2081|0|6|8|2|2081|727|9467|Sunday|2081Q2|N|N|N|2481282|2481432|2480924|2481199|N|N|N|N|N| +2481290|AAAAAAAAKIMNFCAA|2081-06-09|2177|9467|727|2081|1|6|9|2|2081|727|9467|Monday|2081Q2|N|N|N|2481282|2481432|2480925|2481200|N|N|N|N|N| +2481291|AAAAAAAALIMNFCAA|2081-06-10|2177|9468|727|2081|2|6|10|2|2081|727|9468|Tuesday|2081Q2|N|N|N|2481282|2481432|2480926|2481201|N|N|N|N|N| +2481292|AAAAAAAAMIMNFCAA|2081-06-11|2177|9468|727|2081|3|6|11|2|2081|727|9468|Wednesday|2081Q2|N|N|N|2481282|2481432|2480927|2481202|N|N|N|N|N| +2481293|AAAAAAAANIMNFCAA|2081-06-12|2177|9468|727|2081|4|6|12|2|2081|727|9468|Thursday|2081Q2|N|N|N|2481282|2481432|2480928|2481203|N|N|N|N|N| +2481294|AAAAAAAAOIMNFCAA|2081-06-13|2177|9468|727|2081|5|6|13|2|2081|727|9468|Friday|2081Q2|N|Y|N|2481282|2481432|2480929|2481204|N|N|N|N|N| +2481295|AAAAAAAAPIMNFCAA|2081-06-14|2177|9468|727|2081|6|6|14|2|2081|727|9468|Saturday|2081Q2|N|Y|N|2481282|2481432|2480930|2481205|N|N|N|N|N| +2481296|AAAAAAAAAJMNFCAA|2081-06-15|2177|9468|727|2081|0|6|15|2|2081|727|9468|Sunday|2081Q2|N|N|N|2481282|2481432|2480931|2481206|N|N|N|N|N| +2481297|AAAAAAAABJMNFCAA|2081-06-16|2177|9468|727|2081|1|6|16|2|2081|727|9468|Monday|2081Q2|N|N|N|2481282|2481432|2480932|2481207|N|N|N|N|N| +2481298|AAAAAAAACJMNFCAA|2081-06-17|2177|9469|727|2081|2|6|17|2|2081|727|9469|Tuesday|2081Q2|N|N|N|2481282|2481432|2480933|2481208|N|N|N|N|N| +2481299|AAAAAAAADJMNFCAA|2081-06-18|2177|9469|727|2081|3|6|18|2|2081|727|9469|Wednesday|2081Q2|N|N|N|2481282|2481432|2480934|2481209|N|N|N|N|N| +2481300|AAAAAAAAEJMNFCAA|2081-06-19|2177|9469|727|2081|4|6|19|2|2081|727|9469|Thursday|2081Q2|N|N|N|2481282|2481432|2480935|2481210|N|N|N|N|N| +2481301|AAAAAAAAFJMNFCAA|2081-06-20|2177|9469|727|2081|5|6|20|2|2081|727|9469|Friday|2081Q2|N|Y|N|2481282|2481432|2480936|2481211|N|N|N|N|N| +2481302|AAAAAAAAGJMNFCAA|2081-06-21|2177|9469|727|2081|6|6|21|2|2081|727|9469|Saturday|2081Q2|N|Y|N|2481282|2481432|2480937|2481212|N|N|N|N|N| +2481303|AAAAAAAAHJMNFCAA|2081-06-22|2177|9469|727|2081|0|6|22|2|2081|727|9469|Sunday|2081Q2|N|N|N|2481282|2481432|2480938|2481213|N|N|N|N|N| +2481304|AAAAAAAAIJMNFCAA|2081-06-23|2177|9469|727|2081|1|6|23|2|2081|727|9469|Monday|2081Q2|N|N|N|2481282|2481432|2480939|2481214|N|N|N|N|N| +2481305|AAAAAAAAJJMNFCAA|2081-06-24|2177|9470|727|2081|2|6|24|2|2081|727|9470|Tuesday|2081Q2|N|N|N|2481282|2481432|2480940|2481215|N|N|N|N|N| +2481306|AAAAAAAAKJMNFCAA|2081-06-25|2177|9470|727|2081|3|6|25|2|2081|727|9470|Wednesday|2081Q2|N|N|N|2481282|2481432|2480941|2481216|N|N|N|N|N| +2481307|AAAAAAAALJMNFCAA|2081-06-26|2177|9470|727|2081|4|6|26|2|2081|727|9470|Thursday|2081Q2|N|N|N|2481282|2481432|2480942|2481217|N|N|N|N|N| +2481308|AAAAAAAAMJMNFCAA|2081-06-27|2177|9470|727|2081|5|6|27|2|2081|727|9470|Friday|2081Q2|N|Y|N|2481282|2481432|2480943|2481218|N|N|N|N|N| +2481309|AAAAAAAANJMNFCAA|2081-06-28|2177|9470|727|2081|6|6|28|2|2081|727|9470|Saturday|2081Q2|N|Y|N|2481282|2481432|2480944|2481219|N|N|N|N|N| +2481310|AAAAAAAAOJMNFCAA|2081-06-29|2177|9470|727|2081|0|6|29|2|2081|727|9470|Sunday|2081Q2|N|N|N|2481282|2481432|2480945|2481220|N|N|N|N|N| +2481311|AAAAAAAAPJMNFCAA|2081-06-30|2177|9470|727|2081|1|6|30|2|2081|727|9470|Monday|2081Q2|N|N|N|2481282|2481432|2480946|2481221|N|N|N|N|N| +2481312|AAAAAAAAAKMNFCAA|2081-07-01|2178|9471|727|2081|2|7|1|2|2081|727|9471|Tuesday|2081Q2|N|N|N|2481312|2481492|2480947|2481221|N|N|N|N|N| +2481313|AAAAAAAABKMNFCAA|2081-07-02|2178|9471|727|2081|3|7|2|3|2081|727|9471|Wednesday|2081Q3|N|N|N|2481312|2481492|2480948|2481222|N|N|N|N|N| +2481314|AAAAAAAACKMNFCAA|2081-07-03|2178|9471|727|2081|4|7|3|3|2081|727|9471|Thursday|2081Q3|N|N|N|2481312|2481492|2480949|2481223|N|N|N|N|N| +2481315|AAAAAAAADKMNFCAA|2081-07-04|2178|9471|727|2081|5|7|4|3|2081|727|9471|Friday|2081Q3|N|Y|N|2481312|2481492|2480950|2481224|N|N|N|N|N| +2481316|AAAAAAAAEKMNFCAA|2081-07-05|2178|9471|727|2081|6|7|5|3|2081|727|9471|Saturday|2081Q3|Y|Y|N|2481312|2481492|2480951|2481225|N|N|N|N|N| +2481317|AAAAAAAAFKMNFCAA|2081-07-06|2178|9471|727|2081|0|7|6|3|2081|727|9471|Sunday|2081Q3|N|N|Y|2481312|2481492|2480952|2481226|N|N|N|N|N| +2481318|AAAAAAAAGKMNFCAA|2081-07-07|2178|9471|727|2081|1|7|7|3|2081|727|9471|Monday|2081Q3|N|N|N|2481312|2481492|2480953|2481227|N|N|N|N|N| +2481319|AAAAAAAAHKMNFCAA|2081-07-08|2178|9472|727|2081|2|7|8|3|2081|727|9472|Tuesday|2081Q3|N|N|N|2481312|2481492|2480954|2481228|N|N|N|N|N| +2481320|AAAAAAAAIKMNFCAA|2081-07-09|2178|9472|727|2081|3|7|9|3|2081|727|9472|Wednesday|2081Q3|N|N|N|2481312|2481492|2480955|2481229|N|N|N|N|N| +2481321|AAAAAAAAJKMNFCAA|2081-07-10|2178|9472|727|2081|4|7|10|3|2081|727|9472|Thursday|2081Q3|N|N|N|2481312|2481492|2480956|2481230|N|N|N|N|N| +2481322|AAAAAAAAKKMNFCAA|2081-07-11|2178|9472|727|2081|5|7|11|3|2081|727|9472|Friday|2081Q3|N|Y|N|2481312|2481492|2480957|2481231|N|N|N|N|N| +2481323|AAAAAAAALKMNFCAA|2081-07-12|2178|9472|727|2081|6|7|12|3|2081|727|9472|Saturday|2081Q3|N|Y|N|2481312|2481492|2480958|2481232|N|N|N|N|N| +2481324|AAAAAAAAMKMNFCAA|2081-07-13|2178|9472|727|2081|0|7|13|3|2081|727|9472|Sunday|2081Q3|N|N|N|2481312|2481492|2480959|2481233|N|N|N|N|N| +2481325|AAAAAAAANKMNFCAA|2081-07-14|2178|9472|727|2081|1|7|14|3|2081|727|9472|Monday|2081Q3|N|N|N|2481312|2481492|2480960|2481234|N|N|N|N|N| +2481326|AAAAAAAAOKMNFCAA|2081-07-15|2178|9473|727|2081|2|7|15|3|2081|727|9473|Tuesday|2081Q3|N|N|N|2481312|2481492|2480961|2481235|N|N|N|N|N| +2481327|AAAAAAAAPKMNFCAA|2081-07-16|2178|9473|727|2081|3|7|16|3|2081|727|9473|Wednesday|2081Q3|N|N|N|2481312|2481492|2480962|2481236|N|N|N|N|N| +2481328|AAAAAAAAALMNFCAA|2081-07-17|2178|9473|727|2081|4|7|17|3|2081|727|9473|Thursday|2081Q3|N|N|N|2481312|2481492|2480963|2481237|N|N|N|N|N| +2481329|AAAAAAAABLMNFCAA|2081-07-18|2178|9473|727|2081|5|7|18|3|2081|727|9473|Friday|2081Q3|N|Y|N|2481312|2481492|2480964|2481238|N|N|N|N|N| +2481330|AAAAAAAACLMNFCAA|2081-07-19|2178|9473|727|2081|6|7|19|3|2081|727|9473|Saturday|2081Q3|N|Y|N|2481312|2481492|2480965|2481239|N|N|N|N|N| +2481331|AAAAAAAADLMNFCAA|2081-07-20|2178|9473|727|2081|0|7|20|3|2081|727|9473|Sunday|2081Q3|N|N|N|2481312|2481492|2480966|2481240|N|N|N|N|N| +2481332|AAAAAAAAELMNFCAA|2081-07-21|2178|9473|727|2081|1|7|21|3|2081|727|9473|Monday|2081Q3|N|N|N|2481312|2481492|2480967|2481241|N|N|N|N|N| +2481333|AAAAAAAAFLMNFCAA|2081-07-22|2178|9474|727|2081|2|7|22|3|2081|727|9474|Tuesday|2081Q3|N|N|N|2481312|2481492|2480968|2481242|N|N|N|N|N| +2481334|AAAAAAAAGLMNFCAA|2081-07-23|2178|9474|727|2081|3|7|23|3|2081|727|9474|Wednesday|2081Q3|N|N|N|2481312|2481492|2480969|2481243|N|N|N|N|N| +2481335|AAAAAAAAHLMNFCAA|2081-07-24|2178|9474|727|2081|4|7|24|3|2081|727|9474|Thursday|2081Q3|N|N|N|2481312|2481492|2480970|2481244|N|N|N|N|N| +2481336|AAAAAAAAILMNFCAA|2081-07-25|2178|9474|727|2081|5|7|25|3|2081|727|9474|Friday|2081Q3|N|Y|N|2481312|2481492|2480971|2481245|N|N|N|N|N| +2481337|AAAAAAAAJLMNFCAA|2081-07-26|2178|9474|727|2081|6|7|26|3|2081|727|9474|Saturday|2081Q3|N|Y|N|2481312|2481492|2480972|2481246|N|N|N|N|N| +2481338|AAAAAAAAKLMNFCAA|2081-07-27|2178|9474|727|2081|0|7|27|3|2081|727|9474|Sunday|2081Q3|N|N|N|2481312|2481492|2480973|2481247|N|N|N|N|N| +2481339|AAAAAAAALLMNFCAA|2081-07-28|2178|9474|727|2081|1|7|28|3|2081|727|9474|Monday|2081Q3|N|N|N|2481312|2481492|2480974|2481248|N|N|N|N|N| +2481340|AAAAAAAAMLMNFCAA|2081-07-29|2178|9475|727|2081|2|7|29|3|2081|727|9475|Tuesday|2081Q3|N|N|N|2481312|2481492|2480975|2481249|N|N|N|N|N| +2481341|AAAAAAAANLMNFCAA|2081-07-30|2178|9475|727|2081|3|7|30|3|2081|727|9475|Wednesday|2081Q3|N|N|N|2481312|2481492|2480976|2481250|N|N|N|N|N| +2481342|AAAAAAAAOLMNFCAA|2081-07-31|2178|9475|727|2081|4|7|31|3|2081|727|9475|Thursday|2081Q3|N|N|N|2481312|2481492|2480977|2481251|N|N|N|N|N| +2481343|AAAAAAAAPLMNFCAA|2081-08-01|2179|9475|727|2081|5|8|1|3|2081|727|9475|Friday|2081Q3|N|Y|N|2481343|2481554|2480978|2481252|N|N|N|N|N| +2481344|AAAAAAAAAMMNFCAA|2081-08-02|2179|9475|727|2081|6|8|2|3|2081|727|9475|Saturday|2081Q3|N|Y|N|2481343|2481554|2480979|2481253|N|N|N|N|N| +2481345|AAAAAAAABMMNFCAA|2081-08-03|2179|9475|727|2081|0|8|3|3|2081|727|9475|Sunday|2081Q3|N|N|N|2481343|2481554|2480980|2481254|N|N|N|N|N| +2481346|AAAAAAAACMMNFCAA|2081-08-04|2179|9475|727|2081|1|8|4|3|2081|727|9475|Monday|2081Q3|N|N|N|2481343|2481554|2480981|2481255|N|N|N|N|N| +2481347|AAAAAAAADMMNFCAA|2081-08-05|2179|9476|727|2081|2|8|5|3|2081|727|9476|Tuesday|2081Q3|N|N|N|2481343|2481554|2480982|2481256|N|N|N|N|N| +2481348|AAAAAAAAEMMNFCAA|2081-08-06|2179|9476|727|2081|3|8|6|3|2081|727|9476|Wednesday|2081Q3|N|N|N|2481343|2481554|2480983|2481257|N|N|N|N|N| +2481349|AAAAAAAAFMMNFCAA|2081-08-07|2179|9476|727|2081|4|8|7|3|2081|727|9476|Thursday|2081Q3|N|N|N|2481343|2481554|2480984|2481258|N|N|N|N|N| +2481350|AAAAAAAAGMMNFCAA|2081-08-08|2179|9476|727|2081|5|8|8|3|2081|727|9476|Friday|2081Q3|N|Y|N|2481343|2481554|2480985|2481259|N|N|N|N|N| +2481351|AAAAAAAAHMMNFCAA|2081-08-09|2179|9476|727|2081|6|8|9|3|2081|727|9476|Saturday|2081Q3|N|Y|N|2481343|2481554|2480986|2481260|N|N|N|N|N| +2481352|AAAAAAAAIMMNFCAA|2081-08-10|2179|9476|727|2081|0|8|10|3|2081|727|9476|Sunday|2081Q3|N|N|N|2481343|2481554|2480987|2481261|N|N|N|N|N| +2481353|AAAAAAAAJMMNFCAA|2081-08-11|2179|9476|727|2081|1|8|11|3|2081|727|9476|Monday|2081Q3|N|N|N|2481343|2481554|2480988|2481262|N|N|N|N|N| +2481354|AAAAAAAAKMMNFCAA|2081-08-12|2179|9477|727|2081|2|8|12|3|2081|727|9477|Tuesday|2081Q3|N|N|N|2481343|2481554|2480989|2481263|N|N|N|N|N| +2481355|AAAAAAAALMMNFCAA|2081-08-13|2179|9477|727|2081|3|8|13|3|2081|727|9477|Wednesday|2081Q3|N|N|N|2481343|2481554|2480990|2481264|N|N|N|N|N| +2481356|AAAAAAAAMMMNFCAA|2081-08-14|2179|9477|727|2081|4|8|14|3|2081|727|9477|Thursday|2081Q3|N|N|N|2481343|2481554|2480991|2481265|N|N|N|N|N| +2481357|AAAAAAAANMMNFCAA|2081-08-15|2179|9477|727|2081|5|8|15|3|2081|727|9477|Friday|2081Q3|N|Y|N|2481343|2481554|2480992|2481266|N|N|N|N|N| +2481358|AAAAAAAAOMMNFCAA|2081-08-16|2179|9477|727|2081|6|8|16|3|2081|727|9477|Saturday|2081Q3|N|Y|N|2481343|2481554|2480993|2481267|N|N|N|N|N| +2481359|AAAAAAAAPMMNFCAA|2081-08-17|2179|9477|727|2081|0|8|17|3|2081|727|9477|Sunday|2081Q3|N|N|N|2481343|2481554|2480994|2481268|N|N|N|N|N| +2481360|AAAAAAAAANMNFCAA|2081-08-18|2179|9477|727|2081|1|8|18|3|2081|727|9477|Monday|2081Q3|N|N|N|2481343|2481554|2480995|2481269|N|N|N|N|N| +2481361|AAAAAAAABNMNFCAA|2081-08-19|2179|9478|727|2081|2|8|19|3|2081|727|9478|Tuesday|2081Q3|N|N|N|2481343|2481554|2480996|2481270|N|N|N|N|N| +2481362|AAAAAAAACNMNFCAA|2081-08-20|2179|9478|727|2081|3|8|20|3|2081|727|9478|Wednesday|2081Q3|N|N|N|2481343|2481554|2480997|2481271|N|N|N|N|N| +2481363|AAAAAAAADNMNFCAA|2081-08-21|2179|9478|727|2081|4|8|21|3|2081|727|9478|Thursday|2081Q3|N|N|N|2481343|2481554|2480998|2481272|N|N|N|N|N| +2481364|AAAAAAAAENMNFCAA|2081-08-22|2179|9478|727|2081|5|8|22|3|2081|727|9478|Friday|2081Q3|N|Y|N|2481343|2481554|2480999|2481273|N|N|N|N|N| +2481365|AAAAAAAAFNMNFCAA|2081-08-23|2179|9478|727|2081|6|8|23|3|2081|727|9478|Saturday|2081Q3|N|Y|N|2481343|2481554|2481000|2481274|N|N|N|N|N| +2481366|AAAAAAAAGNMNFCAA|2081-08-24|2179|9478|727|2081|0|8|24|3|2081|727|9478|Sunday|2081Q3|N|N|N|2481343|2481554|2481001|2481275|N|N|N|N|N| +2481367|AAAAAAAAHNMNFCAA|2081-08-25|2179|9478|727|2081|1|8|25|3|2081|727|9478|Monday|2081Q3|N|N|N|2481343|2481554|2481002|2481276|N|N|N|N|N| +2481368|AAAAAAAAINMNFCAA|2081-08-26|2179|9479|727|2081|2|8|26|3|2081|727|9479|Tuesday|2081Q3|N|N|N|2481343|2481554|2481003|2481277|N|N|N|N|N| +2481369|AAAAAAAAJNMNFCAA|2081-08-27|2179|9479|727|2081|3|8|27|3|2081|727|9479|Wednesday|2081Q3|N|N|N|2481343|2481554|2481004|2481278|N|N|N|N|N| +2481370|AAAAAAAAKNMNFCAA|2081-08-28|2179|9479|727|2081|4|8|28|3|2081|727|9479|Thursday|2081Q3|N|N|N|2481343|2481554|2481005|2481279|N|N|N|N|N| +2481371|AAAAAAAALNMNFCAA|2081-08-29|2179|9479|727|2081|5|8|29|3|2081|727|9479|Friday|2081Q3|N|Y|N|2481343|2481554|2481006|2481280|N|N|N|N|N| +2481372|AAAAAAAAMNMNFCAA|2081-08-30|2179|9479|727|2081|6|8|30|3|2081|727|9479|Saturday|2081Q3|N|Y|N|2481343|2481554|2481007|2481281|N|N|N|N|N| +2481373|AAAAAAAANNMNFCAA|2081-08-31|2179|9479|727|2081|0|8|31|3|2081|727|9479|Sunday|2081Q3|N|N|N|2481343|2481554|2481008|2481282|N|N|N|N|N| +2481374|AAAAAAAAONMNFCAA|2081-09-01|2180|9479|728|2081|1|9|1|3|2081|728|9479|Monday|2081Q3|N|N|N|2481374|2481616|2481009|2481283|N|N|N|N|N| +2481375|AAAAAAAAPNMNFCAA|2081-09-02|2180|9480|728|2081|2|9|2|3|2081|728|9480|Tuesday|2081Q3|N|N|N|2481374|2481616|2481010|2481284|N|N|N|N|N| +2481376|AAAAAAAAAOMNFCAA|2081-09-03|2180|9480|728|2081|3|9|3|3|2081|728|9480|Wednesday|2081Q3|N|N|N|2481374|2481616|2481011|2481285|N|N|N|N|N| +2481377|AAAAAAAABOMNFCAA|2081-09-04|2180|9480|728|2081|4|9|4|3|2081|728|9480|Thursday|2081Q3|N|N|N|2481374|2481616|2481012|2481286|N|N|N|N|N| +2481378|AAAAAAAACOMNFCAA|2081-09-05|2180|9480|728|2081|5|9|5|3|2081|728|9480|Friday|2081Q3|N|Y|N|2481374|2481616|2481013|2481287|N|N|N|N|N| +2481379|AAAAAAAADOMNFCAA|2081-09-06|2180|9480|728|2081|6|9|6|3|2081|728|9480|Saturday|2081Q3|N|Y|N|2481374|2481616|2481014|2481288|N|N|N|N|N| +2481380|AAAAAAAAEOMNFCAA|2081-09-07|2180|9480|728|2081|0|9|7|3|2081|728|9480|Sunday|2081Q3|N|N|N|2481374|2481616|2481015|2481289|N|N|N|N|N| +2481381|AAAAAAAAFOMNFCAA|2081-09-08|2180|9480|728|2081|1|9|8|3|2081|728|9480|Monday|2081Q3|N|N|N|2481374|2481616|2481016|2481290|N|N|N|N|N| +2481382|AAAAAAAAGOMNFCAA|2081-09-09|2180|9481|728|2081|2|9|9|3|2081|728|9481|Tuesday|2081Q3|N|N|N|2481374|2481616|2481017|2481291|N|N|N|N|N| +2481383|AAAAAAAAHOMNFCAA|2081-09-10|2180|9481|728|2081|3|9|10|3|2081|728|9481|Wednesday|2081Q3|N|N|N|2481374|2481616|2481018|2481292|N|N|N|N|N| +2481384|AAAAAAAAIOMNFCAA|2081-09-11|2180|9481|728|2081|4|9|11|3|2081|728|9481|Thursday|2081Q3|N|N|N|2481374|2481616|2481019|2481293|N|N|N|N|N| +2481385|AAAAAAAAJOMNFCAA|2081-09-12|2180|9481|728|2081|5|9|12|3|2081|728|9481|Friday|2081Q3|N|Y|N|2481374|2481616|2481020|2481294|N|N|N|N|N| +2481386|AAAAAAAAKOMNFCAA|2081-09-13|2180|9481|728|2081|6|9|13|3|2081|728|9481|Saturday|2081Q3|N|Y|N|2481374|2481616|2481021|2481295|N|N|N|N|N| +2481387|AAAAAAAALOMNFCAA|2081-09-14|2180|9481|728|2081|0|9|14|3|2081|728|9481|Sunday|2081Q3|N|N|N|2481374|2481616|2481022|2481296|N|N|N|N|N| +2481388|AAAAAAAAMOMNFCAA|2081-09-15|2180|9481|728|2081|1|9|15|3|2081|728|9481|Monday|2081Q3|N|N|N|2481374|2481616|2481023|2481297|N|N|N|N|N| +2481389|AAAAAAAANOMNFCAA|2081-09-16|2180|9482|728|2081|2|9|16|3|2081|728|9482|Tuesday|2081Q3|N|N|N|2481374|2481616|2481024|2481298|N|N|N|N|N| +2481390|AAAAAAAAOOMNFCAA|2081-09-17|2180|9482|728|2081|3|9|17|3|2081|728|9482|Wednesday|2081Q3|N|N|N|2481374|2481616|2481025|2481299|N|N|N|N|N| +2481391|AAAAAAAAPOMNFCAA|2081-09-18|2180|9482|728|2081|4|9|18|3|2081|728|9482|Thursday|2081Q3|N|N|N|2481374|2481616|2481026|2481300|N|N|N|N|N| +2481392|AAAAAAAAAPMNFCAA|2081-09-19|2180|9482|728|2081|5|9|19|3|2081|728|9482|Friday|2081Q3|N|Y|N|2481374|2481616|2481027|2481301|N|N|N|N|N| +2481393|AAAAAAAABPMNFCAA|2081-09-20|2180|9482|728|2081|6|9|20|3|2081|728|9482|Saturday|2081Q3|N|Y|N|2481374|2481616|2481028|2481302|N|N|N|N|N| +2481394|AAAAAAAACPMNFCAA|2081-09-21|2180|9482|728|2081|0|9|21|3|2081|728|9482|Sunday|2081Q3|N|N|N|2481374|2481616|2481029|2481303|N|N|N|N|N| +2481395|AAAAAAAADPMNFCAA|2081-09-22|2180|9482|728|2081|1|9|22|3|2081|728|9482|Monday|2081Q3|N|N|N|2481374|2481616|2481030|2481304|N|N|N|N|N| +2481396|AAAAAAAAEPMNFCAA|2081-09-23|2180|9483|728|2081|2|9|23|3|2081|728|9483|Tuesday|2081Q3|N|N|N|2481374|2481616|2481031|2481305|N|N|N|N|N| +2481397|AAAAAAAAFPMNFCAA|2081-09-24|2180|9483|728|2081|3|9|24|3|2081|728|9483|Wednesday|2081Q3|N|N|N|2481374|2481616|2481032|2481306|N|N|N|N|N| +2481398|AAAAAAAAGPMNFCAA|2081-09-25|2180|9483|728|2081|4|9|25|3|2081|728|9483|Thursday|2081Q3|N|N|N|2481374|2481616|2481033|2481307|N|N|N|N|N| +2481399|AAAAAAAAHPMNFCAA|2081-09-26|2180|9483|728|2081|5|9|26|3|2081|728|9483|Friday|2081Q3|N|Y|N|2481374|2481616|2481034|2481308|N|N|N|N|N| +2481400|AAAAAAAAIPMNFCAA|2081-09-27|2180|9483|728|2081|6|9|27|3|2081|728|9483|Saturday|2081Q3|N|Y|N|2481374|2481616|2481035|2481309|N|N|N|N|N| +2481401|AAAAAAAAJPMNFCAA|2081-09-28|2180|9483|728|2081|0|9|28|3|2081|728|9483|Sunday|2081Q3|N|N|N|2481374|2481616|2481036|2481310|N|N|N|N|N| +2481402|AAAAAAAAKPMNFCAA|2081-09-29|2180|9483|728|2081|1|9|29|3|2081|728|9483|Monday|2081Q3|N|N|N|2481374|2481616|2481037|2481311|N|N|N|N|N| +2481403|AAAAAAAALPMNFCAA|2081-09-30|2180|9484|728|2081|2|9|30|3|2081|728|9484|Tuesday|2081Q3|N|N|N|2481374|2481616|2481038|2481312|N|N|N|N|N| +2481404|AAAAAAAAMPMNFCAA|2081-10-01|2181|9484|728|2081|3|10|1|3|2081|728|9484|Wednesday|2081Q3|N|N|N|2481404|2481676|2481039|2481312|N|N|N|N|N| +2481405|AAAAAAAANPMNFCAA|2081-10-02|2181|9484|728|2081|4|10|2|4|2081|728|9484|Thursday|2081Q4|N|N|N|2481404|2481676|2481040|2481313|N|N|N|N|N| +2481406|AAAAAAAAOPMNFCAA|2081-10-03|2181|9484|728|2081|5|10|3|4|2081|728|9484|Friday|2081Q4|N|Y|N|2481404|2481676|2481041|2481314|N|N|N|N|N| +2481407|AAAAAAAAPPMNFCAA|2081-10-04|2181|9484|728|2081|6|10|4|4|2081|728|9484|Saturday|2081Q4|N|Y|N|2481404|2481676|2481042|2481315|N|N|N|N|N| +2481408|AAAAAAAAAANNFCAA|2081-10-05|2181|9484|728|2081|0|10|5|4|2081|728|9484|Sunday|2081Q4|N|N|N|2481404|2481676|2481043|2481316|N|N|N|N|N| +2481409|AAAAAAAABANNFCAA|2081-10-06|2181|9484|728|2081|1|10|6|4|2081|728|9484|Monday|2081Q4|N|N|N|2481404|2481676|2481044|2481317|N|N|N|N|N| +2481410|AAAAAAAACANNFCAA|2081-10-07|2181|9485|728|2081|2|10|7|4|2081|728|9485|Tuesday|2081Q4|N|N|N|2481404|2481676|2481045|2481318|N|N|N|N|N| +2481411|AAAAAAAADANNFCAA|2081-10-08|2181|9485|728|2081|3|10|8|4|2081|728|9485|Wednesday|2081Q4|N|N|N|2481404|2481676|2481046|2481319|N|N|N|N|N| +2481412|AAAAAAAAEANNFCAA|2081-10-09|2181|9485|728|2081|4|10|9|4|2081|728|9485|Thursday|2081Q4|N|N|N|2481404|2481676|2481047|2481320|N|N|N|N|N| +2481413|AAAAAAAAFANNFCAA|2081-10-10|2181|9485|728|2081|5|10|10|4|2081|728|9485|Friday|2081Q4|N|Y|N|2481404|2481676|2481048|2481321|N|N|N|N|N| +2481414|AAAAAAAAGANNFCAA|2081-10-11|2181|9485|728|2081|6|10|11|4|2081|728|9485|Saturday|2081Q4|N|Y|N|2481404|2481676|2481049|2481322|N|N|N|N|N| +2481415|AAAAAAAAHANNFCAA|2081-10-12|2181|9485|728|2081|0|10|12|4|2081|728|9485|Sunday|2081Q4|N|N|N|2481404|2481676|2481050|2481323|N|N|N|N|N| +2481416|AAAAAAAAIANNFCAA|2081-10-13|2181|9485|728|2081|1|10|13|4|2081|728|9485|Monday|2081Q4|N|N|N|2481404|2481676|2481051|2481324|N|N|N|N|N| +2481417|AAAAAAAAJANNFCAA|2081-10-14|2181|9486|728|2081|2|10|14|4|2081|728|9486|Tuesday|2081Q4|N|N|N|2481404|2481676|2481052|2481325|N|N|N|N|N| +2481418|AAAAAAAAKANNFCAA|2081-10-15|2181|9486|728|2081|3|10|15|4|2081|728|9486|Wednesday|2081Q4|N|N|N|2481404|2481676|2481053|2481326|N|N|N|N|N| +2481419|AAAAAAAALANNFCAA|2081-10-16|2181|9486|728|2081|4|10|16|4|2081|728|9486|Thursday|2081Q4|N|N|N|2481404|2481676|2481054|2481327|N|N|N|N|N| +2481420|AAAAAAAAMANNFCAA|2081-10-17|2181|9486|728|2081|5|10|17|4|2081|728|9486|Friday|2081Q4|N|Y|N|2481404|2481676|2481055|2481328|N|N|N|N|N| +2481421|AAAAAAAANANNFCAA|2081-10-18|2181|9486|728|2081|6|10|18|4|2081|728|9486|Saturday|2081Q4|N|Y|N|2481404|2481676|2481056|2481329|N|N|N|N|N| +2481422|AAAAAAAAOANNFCAA|2081-10-19|2181|9486|728|2081|0|10|19|4|2081|728|9486|Sunday|2081Q4|N|N|N|2481404|2481676|2481057|2481330|N|N|N|N|N| +2481423|AAAAAAAAPANNFCAA|2081-10-20|2181|9486|728|2081|1|10|20|4|2081|728|9486|Monday|2081Q4|N|N|N|2481404|2481676|2481058|2481331|N|N|N|N|N| +2481424|AAAAAAAAABNNFCAA|2081-10-21|2181|9487|728|2081|2|10|21|4|2081|728|9487|Tuesday|2081Q4|N|N|N|2481404|2481676|2481059|2481332|N|N|N|N|N| +2481425|AAAAAAAABBNNFCAA|2081-10-22|2181|9487|728|2081|3|10|22|4|2081|728|9487|Wednesday|2081Q4|N|N|N|2481404|2481676|2481060|2481333|N|N|N|N|N| +2481426|AAAAAAAACBNNFCAA|2081-10-23|2181|9487|728|2081|4|10|23|4|2081|728|9487|Thursday|2081Q4|N|N|N|2481404|2481676|2481061|2481334|N|N|N|N|N| +2481427|AAAAAAAADBNNFCAA|2081-10-24|2181|9487|728|2081|5|10|24|4|2081|728|9487|Friday|2081Q4|N|Y|N|2481404|2481676|2481062|2481335|N|N|N|N|N| +2481428|AAAAAAAAEBNNFCAA|2081-10-25|2181|9487|728|2081|6|10|25|4|2081|728|9487|Saturday|2081Q4|N|Y|N|2481404|2481676|2481063|2481336|N|N|N|N|N| +2481429|AAAAAAAAFBNNFCAA|2081-10-26|2181|9487|728|2081|0|10|26|4|2081|728|9487|Sunday|2081Q4|N|N|N|2481404|2481676|2481064|2481337|N|N|N|N|N| +2481430|AAAAAAAAGBNNFCAA|2081-10-27|2181|9487|728|2081|1|10|27|4|2081|728|9487|Monday|2081Q4|N|N|N|2481404|2481676|2481065|2481338|N|N|N|N|N| +2481431|AAAAAAAAHBNNFCAA|2081-10-28|2181|9488|728|2081|2|10|28|4|2081|728|9488|Tuesday|2081Q4|N|N|N|2481404|2481676|2481066|2481339|N|N|N|N|N| +2481432|AAAAAAAAIBNNFCAA|2081-10-29|2181|9488|728|2081|3|10|29|4|2081|728|9488|Wednesday|2081Q4|N|N|N|2481404|2481676|2481067|2481340|N|N|N|N|N| +2481433|AAAAAAAAJBNNFCAA|2081-10-30|2181|9488|728|2081|4|10|30|4|2081|728|9488|Thursday|2081Q4|N|N|N|2481404|2481676|2481068|2481341|N|N|N|N|N| +2481434|AAAAAAAAKBNNFCAA|2081-10-31|2181|9488|728|2081|5|10|31|4|2081|728|9488|Friday|2081Q4|N|Y|N|2481404|2481676|2481069|2481342|N|N|N|N|N| +2481435|AAAAAAAALBNNFCAA|2081-11-01|2182|9488|728|2081|6|11|1|4|2081|728|9488|Saturday|2081Q4|N|Y|N|2481435|2481738|2481070|2481343|N|N|N|N|N| +2481436|AAAAAAAAMBNNFCAA|2081-11-02|2182|9488|728|2081|0|11|2|4|2081|728|9488|Sunday|2081Q4|N|N|N|2481435|2481738|2481071|2481344|N|N|N|N|N| +2481437|AAAAAAAANBNNFCAA|2081-11-03|2182|9488|728|2081|1|11|3|4|2081|728|9488|Monday|2081Q4|N|N|N|2481435|2481738|2481072|2481345|N|N|N|N|N| +2481438|AAAAAAAAOBNNFCAA|2081-11-04|2182|9489|728|2081|2|11|4|4|2081|728|9489|Tuesday|2081Q4|N|N|N|2481435|2481738|2481073|2481346|N|N|N|N|N| +2481439|AAAAAAAAPBNNFCAA|2081-11-05|2182|9489|728|2081|3|11|5|4|2081|728|9489|Wednesday|2081Q4|N|N|N|2481435|2481738|2481074|2481347|N|N|N|N|N| +2481440|AAAAAAAAACNNFCAA|2081-11-06|2182|9489|728|2081|4|11|6|4|2081|728|9489|Thursday|2081Q4|N|N|N|2481435|2481738|2481075|2481348|N|N|N|N|N| +2481441|AAAAAAAABCNNFCAA|2081-11-07|2182|9489|728|2081|5|11|7|4|2081|728|9489|Friday|2081Q4|N|Y|N|2481435|2481738|2481076|2481349|N|N|N|N|N| +2481442|AAAAAAAACCNNFCAA|2081-11-08|2182|9489|728|2081|6|11|8|4|2081|728|9489|Saturday|2081Q4|N|Y|N|2481435|2481738|2481077|2481350|N|N|N|N|N| +2481443|AAAAAAAADCNNFCAA|2081-11-09|2182|9489|728|2081|0|11|9|4|2081|728|9489|Sunday|2081Q4|N|N|N|2481435|2481738|2481078|2481351|N|N|N|N|N| +2481444|AAAAAAAAECNNFCAA|2081-11-10|2182|9489|728|2081|1|11|10|4|2081|728|9489|Monday|2081Q4|N|N|N|2481435|2481738|2481079|2481352|N|N|N|N|N| +2481445|AAAAAAAAFCNNFCAA|2081-11-11|2182|9490|728|2081|2|11|11|4|2081|728|9490|Tuesday|2081Q4|N|N|N|2481435|2481738|2481080|2481353|N|N|N|N|N| +2481446|AAAAAAAAGCNNFCAA|2081-11-12|2182|9490|728|2081|3|11|12|4|2081|728|9490|Wednesday|2081Q4|N|N|N|2481435|2481738|2481081|2481354|N|N|N|N|N| +2481447|AAAAAAAAHCNNFCAA|2081-11-13|2182|9490|728|2081|4|11|13|4|2081|728|9490|Thursday|2081Q4|N|N|N|2481435|2481738|2481082|2481355|N|N|N|N|N| +2481448|AAAAAAAAICNNFCAA|2081-11-14|2182|9490|728|2081|5|11|14|4|2081|728|9490|Friday|2081Q4|N|Y|N|2481435|2481738|2481083|2481356|N|N|N|N|N| +2481449|AAAAAAAAJCNNFCAA|2081-11-15|2182|9490|728|2081|6|11|15|4|2081|728|9490|Saturday|2081Q4|N|Y|N|2481435|2481738|2481084|2481357|N|N|N|N|N| +2481450|AAAAAAAAKCNNFCAA|2081-11-16|2182|9490|728|2081|0|11|16|4|2081|728|9490|Sunday|2081Q4|N|N|N|2481435|2481738|2481085|2481358|N|N|N|N|N| +2481451|AAAAAAAALCNNFCAA|2081-11-17|2182|9490|728|2081|1|11|17|4|2081|728|9490|Monday|2081Q4|N|N|N|2481435|2481738|2481086|2481359|N|N|N|N|N| +2481452|AAAAAAAAMCNNFCAA|2081-11-18|2182|9491|728|2081|2|11|18|4|2081|728|9491|Tuesday|2081Q4|N|N|N|2481435|2481738|2481087|2481360|N|N|N|N|N| +2481453|AAAAAAAANCNNFCAA|2081-11-19|2182|9491|728|2081|3|11|19|4|2081|728|9491|Wednesday|2081Q4|N|N|N|2481435|2481738|2481088|2481361|N|N|N|N|N| +2481454|AAAAAAAAOCNNFCAA|2081-11-20|2182|9491|728|2081|4|11|20|4|2081|728|9491|Thursday|2081Q4|N|N|N|2481435|2481738|2481089|2481362|N|N|N|N|N| +2481455|AAAAAAAAPCNNFCAA|2081-11-21|2182|9491|728|2081|5|11|21|4|2081|728|9491|Friday|2081Q4|N|Y|N|2481435|2481738|2481090|2481363|N|N|N|N|N| +2481456|AAAAAAAAADNNFCAA|2081-11-22|2182|9491|728|2081|6|11|22|4|2081|728|9491|Saturday|2081Q4|N|Y|N|2481435|2481738|2481091|2481364|N|N|N|N|N| +2481457|AAAAAAAABDNNFCAA|2081-11-23|2182|9491|728|2081|0|11|23|4|2081|728|9491|Sunday|2081Q4|N|N|N|2481435|2481738|2481092|2481365|N|N|N|N|N| +2481458|AAAAAAAACDNNFCAA|2081-11-24|2182|9491|728|2081|1|11|24|4|2081|728|9491|Monday|2081Q4|N|N|N|2481435|2481738|2481093|2481366|N|N|N|N|N| +2481459|AAAAAAAADDNNFCAA|2081-11-25|2182|9492|728|2081|2|11|25|4|2081|728|9492|Tuesday|2081Q4|N|N|N|2481435|2481738|2481094|2481367|N|N|N|N|N| +2481460|AAAAAAAAEDNNFCAA|2081-11-26|2182|9492|728|2081|3|11|26|4|2081|728|9492|Wednesday|2081Q4|N|N|N|2481435|2481738|2481095|2481368|N|N|N|N|N| +2481461|AAAAAAAAFDNNFCAA|2081-11-27|2182|9492|728|2081|4|11|27|4|2081|728|9492|Thursday|2081Q4|N|N|N|2481435|2481738|2481096|2481369|N|N|N|N|N| +2481462|AAAAAAAAGDNNFCAA|2081-11-28|2182|9492|728|2081|5|11|28|4|2081|728|9492|Friday|2081Q4|N|Y|N|2481435|2481738|2481097|2481370|N|N|N|N|N| +2481463|AAAAAAAAHDNNFCAA|2081-11-29|2182|9492|728|2081|6|11|29|4|2081|728|9492|Saturday|2081Q4|N|Y|N|2481435|2481738|2481098|2481371|N|N|N|N|N| +2481464|AAAAAAAAIDNNFCAA|2081-11-30|2182|9492|728|2081|0|11|30|4|2081|728|9492|Sunday|2081Q4|N|N|N|2481435|2481738|2481099|2481372|N|N|N|N|N| +2481465|AAAAAAAAJDNNFCAA|2081-12-01|2183|9492|729|2081|1|12|1|4|2081|729|9492|Monday|2081Q4|N|N|N|2481465|2481798|2481100|2481373|N|N|N|N|N| +2481466|AAAAAAAAKDNNFCAA|2081-12-02|2183|9493|729|2081|2|12|2|4|2081|729|9493|Tuesday|2081Q4|N|N|N|2481465|2481798|2481101|2481374|N|N|N|N|N| +2481467|AAAAAAAALDNNFCAA|2081-12-03|2183|9493|729|2081|3|12|3|4|2081|729|9493|Wednesday|2081Q4|N|N|N|2481465|2481798|2481102|2481375|N|N|N|N|N| +2481468|AAAAAAAAMDNNFCAA|2081-12-04|2183|9493|729|2081|4|12|4|4|2081|729|9493|Thursday|2081Q4|N|N|N|2481465|2481798|2481103|2481376|N|N|N|N|N| +2481469|AAAAAAAANDNNFCAA|2081-12-05|2183|9493|729|2081|5|12|5|4|2081|729|9493|Friday|2081Q4|N|Y|N|2481465|2481798|2481104|2481377|N|N|N|N|N| +2481470|AAAAAAAAODNNFCAA|2081-12-06|2183|9493|729|2081|6|12|6|4|2081|729|9493|Saturday|2081Q4|N|Y|N|2481465|2481798|2481105|2481378|N|N|N|N|N| +2481471|AAAAAAAAPDNNFCAA|2081-12-07|2183|9493|729|2081|0|12|7|4|2081|729|9493|Sunday|2081Q4|N|N|N|2481465|2481798|2481106|2481379|N|N|N|N|N| +2481472|AAAAAAAAAENNFCAA|2081-12-08|2183|9493|729|2081|1|12|8|4|2081|729|9493|Monday|2081Q4|N|N|N|2481465|2481798|2481107|2481380|N|N|N|N|N| +2481473|AAAAAAAABENNFCAA|2081-12-09|2183|9494|729|2081|2|12|9|4|2081|729|9494|Tuesday|2081Q4|N|N|N|2481465|2481798|2481108|2481381|N|N|N|N|N| +2481474|AAAAAAAACENNFCAA|2081-12-10|2183|9494|729|2081|3|12|10|4|2081|729|9494|Wednesday|2081Q4|N|N|N|2481465|2481798|2481109|2481382|N|N|N|N|N| +2481475|AAAAAAAADENNFCAA|2081-12-11|2183|9494|729|2081|4|12|11|4|2081|729|9494|Thursday|2081Q4|N|N|N|2481465|2481798|2481110|2481383|N|N|N|N|N| +2481476|AAAAAAAAEENNFCAA|2081-12-12|2183|9494|729|2081|5|12|12|4|2081|729|9494|Friday|2081Q4|N|Y|N|2481465|2481798|2481111|2481384|N|N|N|N|N| +2481477|AAAAAAAAFENNFCAA|2081-12-13|2183|9494|729|2081|6|12|13|4|2081|729|9494|Saturday|2081Q4|N|Y|N|2481465|2481798|2481112|2481385|N|N|N|N|N| +2481478|AAAAAAAAGENNFCAA|2081-12-14|2183|9494|729|2081|0|12|14|4|2081|729|9494|Sunday|2081Q4|N|N|N|2481465|2481798|2481113|2481386|N|N|N|N|N| +2481479|AAAAAAAAHENNFCAA|2081-12-15|2183|9494|729|2081|1|12|15|4|2081|729|9494|Monday|2081Q4|N|N|N|2481465|2481798|2481114|2481387|N|N|N|N|N| +2481480|AAAAAAAAIENNFCAA|2081-12-16|2183|9495|729|2081|2|12|16|4|2081|729|9495|Tuesday|2081Q4|N|N|N|2481465|2481798|2481115|2481388|N|N|N|N|N| +2481481|AAAAAAAAJENNFCAA|2081-12-17|2183|9495|729|2081|3|12|17|4|2081|729|9495|Wednesday|2081Q4|N|N|N|2481465|2481798|2481116|2481389|N|N|N|N|N| +2481482|AAAAAAAAKENNFCAA|2081-12-18|2183|9495|729|2081|4|12|18|4|2081|729|9495|Thursday|2081Q4|N|N|N|2481465|2481798|2481117|2481390|N|N|N|N|N| +2481483|AAAAAAAALENNFCAA|2081-12-19|2183|9495|729|2081|5|12|19|4|2081|729|9495|Friday|2081Q4|N|Y|N|2481465|2481798|2481118|2481391|N|N|N|N|N| +2481484|AAAAAAAAMENNFCAA|2081-12-20|2183|9495|729|2081|6|12|20|4|2081|729|9495|Saturday|2081Q4|N|Y|N|2481465|2481798|2481119|2481392|N|N|N|N|N| +2481485|AAAAAAAANENNFCAA|2081-12-21|2183|9495|729|2081|0|12|21|4|2081|729|9495|Sunday|2081Q4|N|N|N|2481465|2481798|2481120|2481393|N|N|N|N|N| +2481486|AAAAAAAAOENNFCAA|2081-12-22|2183|9495|729|2081|1|12|22|4|2081|729|9495|Monday|2081Q4|N|N|N|2481465|2481798|2481121|2481394|N|N|N|N|N| +2481487|AAAAAAAAPENNFCAA|2081-12-23|2183|9496|729|2081|2|12|23|4|2081|729|9496|Tuesday|2081Q4|N|N|N|2481465|2481798|2481122|2481395|N|N|N|N|N| +2481488|AAAAAAAAAFNNFCAA|2081-12-24|2183|9496|729|2081|3|12|24|4|2081|729|9496|Wednesday|2081Q4|N|N|N|2481465|2481798|2481123|2481396|N|N|N|N|N| +2481489|AAAAAAAABFNNFCAA|2081-12-25|2183|9496|729|2081|4|12|25|4|2081|729|9496|Thursday|2081Q4|N|N|N|2481465|2481798|2481124|2481397|N|N|N|N|N| +2481490|AAAAAAAACFNNFCAA|2081-12-26|2183|9496|729|2081|5|12|26|4|2081|729|9496|Friday|2081Q4|Y|Y|N|2481465|2481798|2481125|2481398|N|N|N|N|N| +2481491|AAAAAAAADFNNFCAA|2081-12-27|2183|9496|729|2081|6|12|27|4|2081|729|9496|Saturday|2081Q4|N|Y|Y|2481465|2481798|2481126|2481399|N|N|N|N|N| +2481492|AAAAAAAAEFNNFCAA|2081-12-28|2183|9496|729|2081|0|12|28|4|2081|729|9496|Sunday|2081Q4|N|N|N|2481465|2481798|2481127|2481400|N|N|N|N|N| +2481493|AAAAAAAAFFNNFCAA|2081-12-29|2183|9496|729|2081|1|12|29|4|2081|729|9496|Monday|2081Q4|N|N|N|2481465|2481798|2481128|2481401|N|N|N|N|N| +2481494|AAAAAAAAGFNNFCAA|2081-12-30|2183|9497|729|2081|2|12|30|4|2081|729|9497|Tuesday|2081Q4|N|N|N|2481465|2481798|2481129|2481402|N|N|N|N|N| +2481495|AAAAAAAAHFNNFCAA|2081-12-31|2183|9497|729|2081|3|12|31|4|2081|729|9497|Wednesday|2081Q4|N|N|N|2481465|2481798|2481130|2481403|N|N|N|N|N| +2481496|AAAAAAAAIFNNFCAA|2082-01-01|2184|9497|729|2082|4|1|1|1|2082|729|9497|Thursday|2082Q1|Y|N|N|2481496|2481495|2481131|2481404|N|N|N|N|N| +2481497|AAAAAAAAJFNNFCAA|2082-01-02|2184|9497|729|2082|5|1|2|1|2082|729|9497|Friday|2082Q1|N|Y|Y|2481496|2481495|2481132|2481405|N|N|N|N|N| +2481498|AAAAAAAAKFNNFCAA|2082-01-03|2184|9497|729|2082|6|1|3|1|2082|729|9497|Saturday|2082Q1|N|Y|N|2481496|2481495|2481133|2481406|N|N|N|N|N| +2481499|AAAAAAAALFNNFCAA|2082-01-04|2184|9497|729|2082|0|1|4|1|2082|729|9497|Sunday|2082Q1|N|N|N|2481496|2481495|2481134|2481407|N|N|N|N|N| +2481500|AAAAAAAAMFNNFCAA|2082-01-05|2184|9497|729|2082|1|1|5|1|2082|729|9497|Monday|2082Q1|N|N|N|2481496|2481495|2481135|2481408|N|N|N|N|N| +2481501|AAAAAAAANFNNFCAA|2082-01-06|2184|9498|729|2082|2|1|6|1|2082|729|9498|Tuesday|2082Q1|N|N|N|2481496|2481495|2481136|2481409|N|N|N|N|N| +2481502|AAAAAAAAOFNNFCAA|2082-01-07|2184|9498|729|2082|3|1|7|1|2082|729|9498|Wednesday|2082Q1|N|N|N|2481496|2481495|2481137|2481410|N|N|N|N|N| +2481503|AAAAAAAAPFNNFCAA|2082-01-08|2184|9498|729|2082|4|1|8|1|2082|729|9498|Thursday|2082Q1|N|N|N|2481496|2481495|2481138|2481411|N|N|N|N|N| +2481504|AAAAAAAAAGNNFCAA|2082-01-09|2184|9498|729|2082|5|1|9|1|2082|729|9498|Friday|2082Q1|N|Y|N|2481496|2481495|2481139|2481412|N|N|N|N|N| +2481505|AAAAAAAABGNNFCAA|2082-01-10|2184|9498|729|2082|6|1|10|1|2082|729|9498|Saturday|2082Q1|N|Y|N|2481496|2481495|2481140|2481413|N|N|N|N|N| +2481506|AAAAAAAACGNNFCAA|2082-01-11|2184|9498|729|2082|0|1|11|1|2082|729|9498|Sunday|2082Q1|N|N|N|2481496|2481495|2481141|2481414|N|N|N|N|N| +2481507|AAAAAAAADGNNFCAA|2082-01-12|2184|9498|729|2082|1|1|12|1|2082|729|9498|Monday|2082Q1|N|N|N|2481496|2481495|2481142|2481415|N|N|N|N|N| +2481508|AAAAAAAAEGNNFCAA|2082-01-13|2184|9499|729|2082|2|1|13|1|2082|729|9499|Tuesday|2082Q1|N|N|N|2481496|2481495|2481143|2481416|N|N|N|N|N| +2481509|AAAAAAAAFGNNFCAA|2082-01-14|2184|9499|729|2082|3|1|14|1|2082|729|9499|Wednesday|2082Q1|N|N|N|2481496|2481495|2481144|2481417|N|N|N|N|N| +2481510|AAAAAAAAGGNNFCAA|2082-01-15|2184|9499|729|2082|4|1|15|1|2082|729|9499|Thursday|2082Q1|N|N|N|2481496|2481495|2481145|2481418|N|N|N|N|N| +2481511|AAAAAAAAHGNNFCAA|2082-01-16|2184|9499|729|2082|5|1|16|1|2082|729|9499|Friday|2082Q1|N|Y|N|2481496|2481495|2481146|2481419|N|N|N|N|N| +2481512|AAAAAAAAIGNNFCAA|2082-01-17|2184|9499|729|2082|6|1|17|1|2082|729|9499|Saturday|2082Q1|N|Y|N|2481496|2481495|2481147|2481420|N|N|N|N|N| +2481513|AAAAAAAAJGNNFCAA|2082-01-18|2184|9499|729|2082|0|1|18|1|2082|729|9499|Sunday|2082Q1|N|N|N|2481496|2481495|2481148|2481421|N|N|N|N|N| +2481514|AAAAAAAAKGNNFCAA|2082-01-19|2184|9499|729|2082|1|1|19|1|2082|729|9499|Monday|2082Q1|N|N|N|2481496|2481495|2481149|2481422|N|N|N|N|N| +2481515|AAAAAAAALGNNFCAA|2082-01-20|2184|9500|729|2082|2|1|20|1|2082|729|9500|Tuesday|2082Q1|N|N|N|2481496|2481495|2481150|2481423|N|N|N|N|N| +2481516|AAAAAAAAMGNNFCAA|2082-01-21|2184|9500|729|2082|3|1|21|1|2082|729|9500|Wednesday|2082Q1|N|N|N|2481496|2481495|2481151|2481424|N|N|N|N|N| +2481517|AAAAAAAANGNNFCAA|2082-01-22|2184|9500|729|2082|4|1|22|1|2082|729|9500|Thursday|2082Q1|N|N|N|2481496|2481495|2481152|2481425|N|N|N|N|N| +2481518|AAAAAAAAOGNNFCAA|2082-01-23|2184|9500|729|2082|5|1|23|1|2082|729|9500|Friday|2082Q1|N|Y|N|2481496|2481495|2481153|2481426|N|N|N|N|N| +2481519|AAAAAAAAPGNNFCAA|2082-01-24|2184|9500|729|2082|6|1|24|1|2082|729|9500|Saturday|2082Q1|N|Y|N|2481496|2481495|2481154|2481427|N|N|N|N|N| +2481520|AAAAAAAAAHNNFCAA|2082-01-25|2184|9500|729|2082|0|1|25|1|2082|729|9500|Sunday|2082Q1|N|N|N|2481496|2481495|2481155|2481428|N|N|N|N|N| +2481521|AAAAAAAABHNNFCAA|2082-01-26|2184|9500|729|2082|1|1|26|1|2082|729|9500|Monday|2082Q1|N|N|N|2481496|2481495|2481156|2481429|N|N|N|N|N| +2481522|AAAAAAAACHNNFCAA|2082-01-27|2184|9501|729|2082|2|1|27|1|2082|729|9501|Tuesday|2082Q1|N|N|N|2481496|2481495|2481157|2481430|N|N|N|N|N| +2481523|AAAAAAAADHNNFCAA|2082-01-28|2184|9501|729|2082|3|1|28|1|2082|729|9501|Wednesday|2082Q1|N|N|N|2481496|2481495|2481158|2481431|N|N|N|N|N| +2481524|AAAAAAAAEHNNFCAA|2082-01-29|2184|9501|729|2082|4|1|29|1|2082|729|9501|Thursday|2082Q1|N|N|N|2481496|2481495|2481159|2481432|N|N|N|N|N| +2481525|AAAAAAAAFHNNFCAA|2082-01-30|2184|9501|729|2082|5|1|30|1|2082|729|9501|Friday|2082Q1|N|Y|N|2481496|2481495|2481160|2481433|N|N|N|N|N| +2481526|AAAAAAAAGHNNFCAA|2082-01-31|2184|9501|729|2082|6|1|31|1|2082|729|9501|Saturday|2082Q1|N|Y|N|2481496|2481495|2481161|2481434|N|N|N|N|N| +2481527|AAAAAAAAHHNNFCAA|2082-02-01|2185|9501|729|2082|0|2|1|1|2082|729|9501|Sunday|2082Q1|N|N|N|2481527|2481557|2481162|2481435|N|N|N|N|N| +2481528|AAAAAAAAIHNNFCAA|2082-02-02|2185|9501|729|2082|1|2|2|1|2082|729|9501|Monday|2082Q1|N|N|N|2481527|2481557|2481163|2481436|N|N|N|N|N| +2481529|AAAAAAAAJHNNFCAA|2082-02-03|2185|9502|729|2082|2|2|3|1|2082|729|9502|Tuesday|2082Q1|N|N|N|2481527|2481557|2481164|2481437|N|N|N|N|N| +2481530|AAAAAAAAKHNNFCAA|2082-02-04|2185|9502|729|2082|3|2|4|1|2082|729|9502|Wednesday|2082Q1|N|N|N|2481527|2481557|2481165|2481438|N|N|N|N|N| +2481531|AAAAAAAALHNNFCAA|2082-02-05|2185|9502|729|2082|4|2|5|1|2082|729|9502|Thursday|2082Q1|N|N|N|2481527|2481557|2481166|2481439|N|N|N|N|N| +2481532|AAAAAAAAMHNNFCAA|2082-02-06|2185|9502|729|2082|5|2|6|1|2082|729|9502|Friday|2082Q1|N|Y|N|2481527|2481557|2481167|2481440|N|N|N|N|N| +2481533|AAAAAAAANHNNFCAA|2082-02-07|2185|9502|729|2082|6|2|7|1|2082|729|9502|Saturday|2082Q1|N|Y|N|2481527|2481557|2481168|2481441|N|N|N|N|N| +2481534|AAAAAAAAOHNNFCAA|2082-02-08|2185|9502|729|2082|0|2|8|1|2082|729|9502|Sunday|2082Q1|N|N|N|2481527|2481557|2481169|2481442|N|N|N|N|N| +2481535|AAAAAAAAPHNNFCAA|2082-02-09|2185|9502|729|2082|1|2|9|1|2082|729|9502|Monday|2082Q1|N|N|N|2481527|2481557|2481170|2481443|N|N|N|N|N| +2481536|AAAAAAAAAINNFCAA|2082-02-10|2185|9503|729|2082|2|2|10|1|2082|729|9503|Tuesday|2082Q1|N|N|N|2481527|2481557|2481171|2481444|N|N|N|N|N| +2481537|AAAAAAAABINNFCAA|2082-02-11|2185|9503|729|2082|3|2|11|1|2082|729|9503|Wednesday|2082Q1|N|N|N|2481527|2481557|2481172|2481445|N|N|N|N|N| +2481538|AAAAAAAACINNFCAA|2082-02-12|2185|9503|729|2082|4|2|12|1|2082|729|9503|Thursday|2082Q1|N|N|N|2481527|2481557|2481173|2481446|N|N|N|N|N| +2481539|AAAAAAAADINNFCAA|2082-02-13|2185|9503|729|2082|5|2|13|1|2082|729|9503|Friday|2082Q1|N|Y|N|2481527|2481557|2481174|2481447|N|N|N|N|N| +2481540|AAAAAAAAEINNFCAA|2082-02-14|2185|9503|729|2082|6|2|14|1|2082|729|9503|Saturday|2082Q1|N|Y|N|2481527|2481557|2481175|2481448|N|N|N|N|N| +2481541|AAAAAAAAFINNFCAA|2082-02-15|2185|9503|729|2082|0|2|15|1|2082|729|9503|Sunday|2082Q1|N|N|N|2481527|2481557|2481176|2481449|N|N|N|N|N| +2481542|AAAAAAAAGINNFCAA|2082-02-16|2185|9503|729|2082|1|2|16|1|2082|729|9503|Monday|2082Q1|N|N|N|2481527|2481557|2481177|2481450|N|N|N|N|N| +2481543|AAAAAAAAHINNFCAA|2082-02-17|2185|9504|729|2082|2|2|17|1|2082|729|9504|Tuesday|2082Q1|N|N|N|2481527|2481557|2481178|2481451|N|N|N|N|N| +2481544|AAAAAAAAIINNFCAA|2082-02-18|2185|9504|729|2082|3|2|18|1|2082|729|9504|Wednesday|2082Q1|N|N|N|2481527|2481557|2481179|2481452|N|N|N|N|N| +2481545|AAAAAAAAJINNFCAA|2082-02-19|2185|9504|729|2082|4|2|19|1|2082|729|9504|Thursday|2082Q1|N|N|N|2481527|2481557|2481180|2481453|N|N|N|N|N| +2481546|AAAAAAAAKINNFCAA|2082-02-20|2185|9504|729|2082|5|2|20|1|2082|729|9504|Friday|2082Q1|N|Y|N|2481527|2481557|2481181|2481454|N|N|N|N|N| +2481547|AAAAAAAALINNFCAA|2082-02-21|2185|9504|729|2082|6|2|21|1|2082|729|9504|Saturday|2082Q1|N|Y|N|2481527|2481557|2481182|2481455|N|N|N|N|N| +2481548|AAAAAAAAMINNFCAA|2082-02-22|2185|9504|729|2082|0|2|22|1|2082|729|9504|Sunday|2082Q1|N|N|N|2481527|2481557|2481183|2481456|N|N|N|N|N| +2481549|AAAAAAAANINNFCAA|2082-02-23|2185|9504|729|2082|1|2|23|1|2082|729|9504|Monday|2082Q1|N|N|N|2481527|2481557|2481184|2481457|N|N|N|N|N| +2481550|AAAAAAAAOINNFCAA|2082-02-24|2185|9505|729|2082|2|2|24|1|2082|729|9505|Tuesday|2082Q1|N|N|N|2481527|2481557|2481185|2481458|N|N|N|N|N| +2481551|AAAAAAAAPINNFCAA|2082-02-25|2185|9505|729|2082|3|2|25|1|2082|729|9505|Wednesday|2082Q1|N|N|N|2481527|2481557|2481186|2481459|N|N|N|N|N| +2481552|AAAAAAAAAJNNFCAA|2082-02-26|2185|9505|729|2082|4|2|26|1|2082|729|9505|Thursday|2082Q1|N|N|N|2481527|2481557|2481187|2481460|N|N|N|N|N| +2481553|AAAAAAAABJNNFCAA|2082-02-27|2185|9505|729|2082|5|2|27|1|2082|729|9505|Friday|2082Q1|N|Y|N|2481527|2481557|2481188|2481461|N|N|N|N|N| +2481554|AAAAAAAACJNNFCAA|2082-02-28|2185|9505|729|2082|6|2|28|1|2082|729|9505|Saturday|2082Q1|N|Y|N|2481527|2481557|2481189|2481462|N|N|N|N|N| +2481555|AAAAAAAADJNNFCAA|2082-03-01|2186|9505|730|2082|0|3|1|1|2082|730|9505|Sunday|2082Q1|N|N|N|2481555|2481613|2481190|2481463|N|N|N|N|N| +2481556|AAAAAAAAEJNNFCAA|2082-03-02|2186|9505|730|2082|1|3|2|1|2082|730|9505|Monday|2082Q1|N|N|N|2481555|2481613|2481191|2481464|N|N|N|N|N| +2481557|AAAAAAAAFJNNFCAA|2082-03-03|2186|9506|730|2082|2|3|3|1|2082|730|9506|Tuesday|2082Q1|N|N|N|2481555|2481613|2481192|2481465|N|N|N|N|N| +2481558|AAAAAAAAGJNNFCAA|2082-03-04|2186|9506|730|2082|3|3|4|1|2082|730|9506|Wednesday|2082Q1|N|N|N|2481555|2481613|2481193|2481466|N|N|N|N|N| +2481559|AAAAAAAAHJNNFCAA|2082-03-05|2186|9506|730|2082|4|3|5|1|2082|730|9506|Thursday|2082Q1|N|N|N|2481555|2481613|2481194|2481467|N|N|N|N|N| +2481560|AAAAAAAAIJNNFCAA|2082-03-06|2186|9506|730|2082|5|3|6|1|2082|730|9506|Friday|2082Q1|N|Y|N|2481555|2481613|2481195|2481468|N|N|N|N|N| +2481561|AAAAAAAAJJNNFCAA|2082-03-07|2186|9506|730|2082|6|3|7|1|2082|730|9506|Saturday|2082Q1|N|Y|N|2481555|2481613|2481196|2481469|N|N|N|N|N| +2481562|AAAAAAAAKJNNFCAA|2082-03-08|2186|9506|730|2082|0|3|8|1|2082|730|9506|Sunday|2082Q1|N|N|N|2481555|2481613|2481197|2481470|N|N|N|N|N| +2481563|AAAAAAAALJNNFCAA|2082-03-09|2186|9506|730|2082|1|3|9|1|2082|730|9506|Monday|2082Q1|N|N|N|2481555|2481613|2481198|2481471|N|N|N|N|N| +2481564|AAAAAAAAMJNNFCAA|2082-03-10|2186|9507|730|2082|2|3|10|1|2082|730|9507|Tuesday|2082Q1|N|N|N|2481555|2481613|2481199|2481472|N|N|N|N|N| +2481565|AAAAAAAANJNNFCAA|2082-03-11|2186|9507|730|2082|3|3|11|1|2082|730|9507|Wednesday|2082Q1|N|N|N|2481555|2481613|2481200|2481473|N|N|N|N|N| +2481566|AAAAAAAAOJNNFCAA|2082-03-12|2186|9507|730|2082|4|3|12|1|2082|730|9507|Thursday|2082Q1|N|N|N|2481555|2481613|2481201|2481474|N|N|N|N|N| +2481567|AAAAAAAAPJNNFCAA|2082-03-13|2186|9507|730|2082|5|3|13|1|2082|730|9507|Friday|2082Q1|N|Y|N|2481555|2481613|2481202|2481475|N|N|N|N|N| +2481568|AAAAAAAAAKNNFCAA|2082-03-14|2186|9507|730|2082|6|3|14|1|2082|730|9507|Saturday|2082Q1|N|Y|N|2481555|2481613|2481203|2481476|N|N|N|N|N| +2481569|AAAAAAAABKNNFCAA|2082-03-15|2186|9507|730|2082|0|3|15|1|2082|730|9507|Sunday|2082Q1|N|N|N|2481555|2481613|2481204|2481477|N|N|N|N|N| +2481570|AAAAAAAACKNNFCAA|2082-03-16|2186|9507|730|2082|1|3|16|1|2082|730|9507|Monday|2082Q1|N|N|N|2481555|2481613|2481205|2481478|N|N|N|N|N| +2481571|AAAAAAAADKNNFCAA|2082-03-17|2186|9508|730|2082|2|3|17|1|2082|730|9508|Tuesday|2082Q1|N|N|N|2481555|2481613|2481206|2481479|N|N|N|N|N| +2481572|AAAAAAAAEKNNFCAA|2082-03-18|2186|9508|730|2082|3|3|18|1|2082|730|9508|Wednesday|2082Q1|N|N|N|2481555|2481613|2481207|2481480|N|N|N|N|N| +2481573|AAAAAAAAFKNNFCAA|2082-03-19|2186|9508|730|2082|4|3|19|1|2082|730|9508|Thursday|2082Q1|N|N|N|2481555|2481613|2481208|2481481|N|N|N|N|N| +2481574|AAAAAAAAGKNNFCAA|2082-03-20|2186|9508|730|2082|5|3|20|1|2082|730|9508|Friday|2082Q1|N|Y|N|2481555|2481613|2481209|2481482|N|N|N|N|N| +2481575|AAAAAAAAHKNNFCAA|2082-03-21|2186|9508|730|2082|6|3|21|1|2082|730|9508|Saturday|2082Q1|N|Y|N|2481555|2481613|2481210|2481483|N|N|N|N|N| +2481576|AAAAAAAAIKNNFCAA|2082-03-22|2186|9508|730|2082|0|3|22|1|2082|730|9508|Sunday|2082Q1|N|N|N|2481555|2481613|2481211|2481484|N|N|N|N|N| +2481577|AAAAAAAAJKNNFCAA|2082-03-23|2186|9508|730|2082|1|3|23|1|2082|730|9508|Monday|2082Q1|N|N|N|2481555|2481613|2481212|2481485|N|N|N|N|N| +2481578|AAAAAAAAKKNNFCAA|2082-03-24|2186|9509|730|2082|2|3|24|1|2082|730|9509|Tuesday|2082Q1|N|N|N|2481555|2481613|2481213|2481486|N|N|N|N|N| +2481579|AAAAAAAALKNNFCAA|2082-03-25|2186|9509|730|2082|3|3|25|1|2082|730|9509|Wednesday|2082Q1|N|N|N|2481555|2481613|2481214|2481487|N|N|N|N|N| +2481580|AAAAAAAAMKNNFCAA|2082-03-26|2186|9509|730|2082|4|3|26|1|2082|730|9509|Thursday|2082Q1|N|N|N|2481555|2481613|2481215|2481488|N|N|N|N|N| +2481581|AAAAAAAANKNNFCAA|2082-03-27|2186|9509|730|2082|5|3|27|1|2082|730|9509|Friday|2082Q1|N|Y|N|2481555|2481613|2481216|2481489|N|N|N|N|N| +2481582|AAAAAAAAOKNNFCAA|2082-03-28|2186|9509|730|2082|6|3|28|1|2082|730|9509|Saturday|2082Q1|N|Y|N|2481555|2481613|2481217|2481490|N|N|N|N|N| +2481583|AAAAAAAAPKNNFCAA|2082-03-29|2186|9509|730|2082|0|3|29|1|2082|730|9509|Sunday|2082Q1|N|N|N|2481555|2481613|2481218|2481491|N|N|N|N|N| +2481584|AAAAAAAAALNNFCAA|2082-03-30|2186|9509|730|2082|1|3|30|1|2082|730|9509|Monday|2082Q1|N|N|N|2481555|2481613|2481219|2481492|N|N|N|N|N| +2481585|AAAAAAAABLNNFCAA|2082-03-31|2186|9510|730|2082|2|3|31|1|2082|730|9510|Tuesday|2082Q1|N|N|N|2481555|2481613|2481220|2481493|N|N|N|N|N| +2481586|AAAAAAAACLNNFCAA|2082-04-01|2187|9510|730|2082|3|4|1|1|2082|730|9510|Wednesday|2082Q1|N|N|N|2481586|2481675|2481221|2481496|N|N|N|N|N| +2481587|AAAAAAAADLNNFCAA|2082-04-02|2187|9510|730|2082|4|4|2|2|2082|730|9510|Thursday|2082Q2|N|N|N|2481586|2481675|2481222|2481497|N|N|N|N|N| +2481588|AAAAAAAAELNNFCAA|2082-04-03|2187|9510|730|2082|5|4|3|2|2082|730|9510|Friday|2082Q2|N|Y|N|2481586|2481675|2481223|2481498|N|N|N|N|N| +2481589|AAAAAAAAFLNNFCAA|2082-04-04|2187|9510|730|2082|6|4|4|2|2082|730|9510|Saturday|2082Q2|N|Y|N|2481586|2481675|2481224|2481499|N|N|N|N|N| +2481590|AAAAAAAAGLNNFCAA|2082-04-05|2187|9510|730|2082|0|4|5|2|2082|730|9510|Sunday|2082Q2|N|N|N|2481586|2481675|2481225|2481500|N|N|N|N|N| +2481591|AAAAAAAAHLNNFCAA|2082-04-06|2187|9510|730|2082|1|4|6|2|2082|730|9510|Monday|2082Q2|N|N|N|2481586|2481675|2481226|2481501|N|N|N|N|N| +2481592|AAAAAAAAILNNFCAA|2082-04-07|2187|9511|730|2082|2|4|7|2|2082|730|9511|Tuesday|2082Q2|N|N|N|2481586|2481675|2481227|2481502|N|N|N|N|N| +2481593|AAAAAAAAJLNNFCAA|2082-04-08|2187|9511|730|2082|3|4|8|2|2082|730|9511|Wednesday|2082Q2|N|N|N|2481586|2481675|2481228|2481503|N|N|N|N|N| +2481594|AAAAAAAAKLNNFCAA|2082-04-09|2187|9511|730|2082|4|4|9|2|2082|730|9511|Thursday|2082Q2|N|N|N|2481586|2481675|2481229|2481504|N|N|N|N|N| +2481595|AAAAAAAALLNNFCAA|2082-04-10|2187|9511|730|2082|5|4|10|2|2082|730|9511|Friday|2082Q2|N|Y|N|2481586|2481675|2481230|2481505|N|N|N|N|N| +2481596|AAAAAAAAMLNNFCAA|2082-04-11|2187|9511|730|2082|6|4|11|2|2082|730|9511|Saturday|2082Q2|N|Y|N|2481586|2481675|2481231|2481506|N|N|N|N|N| +2481597|AAAAAAAANLNNFCAA|2082-04-12|2187|9511|730|2082|0|4|12|2|2082|730|9511|Sunday|2082Q2|N|N|N|2481586|2481675|2481232|2481507|N|N|N|N|N| +2481598|AAAAAAAAOLNNFCAA|2082-04-13|2187|9511|730|2082|1|4|13|2|2082|730|9511|Monday|2082Q2|N|N|N|2481586|2481675|2481233|2481508|N|N|N|N|N| +2481599|AAAAAAAAPLNNFCAA|2082-04-14|2187|9512|730|2082|2|4|14|2|2082|730|9512|Tuesday|2082Q2|N|N|N|2481586|2481675|2481234|2481509|N|N|N|N|N| +2481600|AAAAAAAAAMNNFCAA|2082-04-15|2187|9512|730|2082|3|4|15|2|2082|730|9512|Wednesday|2082Q2|N|N|N|2481586|2481675|2481235|2481510|N|N|N|N|N| +2481601|AAAAAAAABMNNFCAA|2082-04-16|2187|9512|730|2082|4|4|16|2|2082|730|9512|Thursday|2082Q2|N|N|N|2481586|2481675|2481236|2481511|N|N|N|N|N| +2481602|AAAAAAAACMNNFCAA|2082-04-17|2187|9512|730|2082|5|4|17|2|2082|730|9512|Friday|2082Q2|N|Y|N|2481586|2481675|2481237|2481512|N|N|N|N|N| +2481603|AAAAAAAADMNNFCAA|2082-04-18|2187|9512|730|2082|6|4|18|2|2082|730|9512|Saturday|2082Q2|N|Y|N|2481586|2481675|2481238|2481513|N|N|N|N|N| +2481604|AAAAAAAAEMNNFCAA|2082-04-19|2187|9512|730|2082|0|4|19|2|2082|730|9512|Sunday|2082Q2|N|N|N|2481586|2481675|2481239|2481514|N|N|N|N|N| +2481605|AAAAAAAAFMNNFCAA|2082-04-20|2187|9512|730|2082|1|4|20|2|2082|730|9512|Monday|2082Q2|N|N|N|2481586|2481675|2481240|2481515|N|N|N|N|N| +2481606|AAAAAAAAGMNNFCAA|2082-04-21|2187|9513|730|2082|2|4|21|2|2082|730|9513|Tuesday|2082Q2|N|N|N|2481586|2481675|2481241|2481516|N|N|N|N|N| +2481607|AAAAAAAAHMNNFCAA|2082-04-22|2187|9513|730|2082|3|4|22|2|2082|730|9513|Wednesday|2082Q2|N|N|N|2481586|2481675|2481242|2481517|N|N|N|N|N| +2481608|AAAAAAAAIMNNFCAA|2082-04-23|2187|9513|730|2082|4|4|23|2|2082|730|9513|Thursday|2082Q2|N|N|N|2481586|2481675|2481243|2481518|N|N|N|N|N| +2481609|AAAAAAAAJMNNFCAA|2082-04-24|2187|9513|730|2082|5|4|24|2|2082|730|9513|Friday|2082Q2|N|Y|N|2481586|2481675|2481244|2481519|N|N|N|N|N| +2481610|AAAAAAAAKMNNFCAA|2082-04-25|2187|9513|730|2082|6|4|25|2|2082|730|9513|Saturday|2082Q2|N|Y|N|2481586|2481675|2481245|2481520|N|N|N|N|N| +2481611|AAAAAAAALMNNFCAA|2082-04-26|2187|9513|730|2082|0|4|26|2|2082|730|9513|Sunday|2082Q2|N|N|N|2481586|2481675|2481246|2481521|N|N|N|N|N| +2481612|AAAAAAAAMMNNFCAA|2082-04-27|2187|9513|730|2082|1|4|27|2|2082|730|9513|Monday|2082Q2|N|N|N|2481586|2481675|2481247|2481522|N|N|N|N|N| +2481613|AAAAAAAANMNNFCAA|2082-04-28|2187|9514|730|2082|2|4|28|2|2082|730|9514|Tuesday|2082Q2|N|N|N|2481586|2481675|2481248|2481523|N|N|N|N|N| +2481614|AAAAAAAAOMNNFCAA|2082-04-29|2187|9514|730|2082|3|4|29|2|2082|730|9514|Wednesday|2082Q2|N|N|N|2481586|2481675|2481249|2481524|N|N|N|N|N| +2481615|AAAAAAAAPMNNFCAA|2082-04-30|2187|9514|730|2082|4|4|30|2|2082|730|9514|Thursday|2082Q2|N|N|N|2481586|2481675|2481250|2481525|N|N|N|N|N| +2481616|AAAAAAAAANNNFCAA|2082-05-01|2188|9514|730|2082|5|5|1|2|2082|730|9514|Friday|2082Q2|N|Y|N|2481616|2481735|2481251|2481526|N|N|N|N|N| +2481617|AAAAAAAABNNNFCAA|2082-05-02|2188|9514|730|2082|6|5|2|2|2082|730|9514|Saturday|2082Q2|N|Y|N|2481616|2481735|2481252|2481527|N|N|N|N|N| +2481618|AAAAAAAACNNNFCAA|2082-05-03|2188|9514|730|2082|0|5|3|2|2082|730|9514|Sunday|2082Q2|N|N|N|2481616|2481735|2481253|2481528|N|N|N|N|N| +2481619|AAAAAAAADNNNFCAA|2082-05-04|2188|9514|730|2082|1|5|4|2|2082|730|9514|Monday|2082Q2|N|N|N|2481616|2481735|2481254|2481529|N|N|N|N|N| +2481620|AAAAAAAAENNNFCAA|2082-05-05|2188|9515|730|2082|2|5|5|2|2082|730|9515|Tuesday|2082Q2|N|N|N|2481616|2481735|2481255|2481530|N|N|N|N|N| +2481621|AAAAAAAAFNNNFCAA|2082-05-06|2188|9515|730|2082|3|5|6|2|2082|730|9515|Wednesday|2082Q2|N|N|N|2481616|2481735|2481256|2481531|N|N|N|N|N| +2481622|AAAAAAAAGNNNFCAA|2082-05-07|2188|9515|730|2082|4|5|7|2|2082|730|9515|Thursday|2082Q2|N|N|N|2481616|2481735|2481257|2481532|N|N|N|N|N| +2481623|AAAAAAAAHNNNFCAA|2082-05-08|2188|9515|730|2082|5|5|8|2|2082|730|9515|Friday|2082Q2|N|Y|N|2481616|2481735|2481258|2481533|N|N|N|N|N| +2481624|AAAAAAAAINNNFCAA|2082-05-09|2188|9515|730|2082|6|5|9|2|2082|730|9515|Saturday|2082Q2|N|Y|N|2481616|2481735|2481259|2481534|N|N|N|N|N| +2481625|AAAAAAAAJNNNFCAA|2082-05-10|2188|9515|730|2082|0|5|10|2|2082|730|9515|Sunday|2082Q2|N|N|N|2481616|2481735|2481260|2481535|N|N|N|N|N| +2481626|AAAAAAAAKNNNFCAA|2082-05-11|2188|9515|730|2082|1|5|11|2|2082|730|9515|Monday|2082Q2|N|N|N|2481616|2481735|2481261|2481536|N|N|N|N|N| +2481627|AAAAAAAALNNNFCAA|2082-05-12|2188|9516|730|2082|2|5|12|2|2082|730|9516|Tuesday|2082Q2|N|N|N|2481616|2481735|2481262|2481537|N|N|N|N|N| +2481628|AAAAAAAAMNNNFCAA|2082-05-13|2188|9516|730|2082|3|5|13|2|2082|730|9516|Wednesday|2082Q2|N|N|N|2481616|2481735|2481263|2481538|N|N|N|N|N| +2481629|AAAAAAAANNNNFCAA|2082-05-14|2188|9516|730|2082|4|5|14|2|2082|730|9516|Thursday|2082Q2|N|N|N|2481616|2481735|2481264|2481539|N|N|N|N|N| +2481630|AAAAAAAAONNNFCAA|2082-05-15|2188|9516|730|2082|5|5|15|2|2082|730|9516|Friday|2082Q2|N|Y|N|2481616|2481735|2481265|2481540|N|N|N|N|N| +2481631|AAAAAAAAPNNNFCAA|2082-05-16|2188|9516|730|2082|6|5|16|2|2082|730|9516|Saturday|2082Q2|N|Y|N|2481616|2481735|2481266|2481541|N|N|N|N|N| +2481632|AAAAAAAAAONNFCAA|2082-05-17|2188|9516|730|2082|0|5|17|2|2082|730|9516|Sunday|2082Q2|N|N|N|2481616|2481735|2481267|2481542|N|N|N|N|N| +2481633|AAAAAAAABONNFCAA|2082-05-18|2188|9516|730|2082|1|5|18|2|2082|730|9516|Monday|2082Q2|N|N|N|2481616|2481735|2481268|2481543|N|N|N|N|N| +2481634|AAAAAAAACONNFCAA|2082-05-19|2188|9517|730|2082|2|5|19|2|2082|730|9517|Tuesday|2082Q2|N|N|N|2481616|2481735|2481269|2481544|N|N|N|N|N| +2481635|AAAAAAAADONNFCAA|2082-05-20|2188|9517|730|2082|3|5|20|2|2082|730|9517|Wednesday|2082Q2|N|N|N|2481616|2481735|2481270|2481545|N|N|N|N|N| +2481636|AAAAAAAAEONNFCAA|2082-05-21|2188|9517|730|2082|4|5|21|2|2082|730|9517|Thursday|2082Q2|N|N|N|2481616|2481735|2481271|2481546|N|N|N|N|N| +2481637|AAAAAAAAFONNFCAA|2082-05-22|2188|9517|730|2082|5|5|22|2|2082|730|9517|Friday|2082Q2|N|Y|N|2481616|2481735|2481272|2481547|N|N|N|N|N| +2481638|AAAAAAAAGONNFCAA|2082-05-23|2188|9517|730|2082|6|5|23|2|2082|730|9517|Saturday|2082Q2|N|Y|N|2481616|2481735|2481273|2481548|N|N|N|N|N| +2481639|AAAAAAAAHONNFCAA|2082-05-24|2188|9517|730|2082|0|5|24|2|2082|730|9517|Sunday|2082Q2|N|N|N|2481616|2481735|2481274|2481549|N|N|N|N|N| +2481640|AAAAAAAAIONNFCAA|2082-05-25|2188|9517|730|2082|1|5|25|2|2082|730|9517|Monday|2082Q2|N|N|N|2481616|2481735|2481275|2481550|N|N|N|N|N| +2481641|AAAAAAAAJONNFCAA|2082-05-26|2188|9518|730|2082|2|5|26|2|2082|730|9518|Tuesday|2082Q2|N|N|N|2481616|2481735|2481276|2481551|N|N|N|N|N| +2481642|AAAAAAAAKONNFCAA|2082-05-27|2188|9518|730|2082|3|5|27|2|2082|730|9518|Wednesday|2082Q2|N|N|N|2481616|2481735|2481277|2481552|N|N|N|N|N| +2481643|AAAAAAAALONNFCAA|2082-05-28|2188|9518|730|2082|4|5|28|2|2082|730|9518|Thursday|2082Q2|N|N|N|2481616|2481735|2481278|2481553|N|N|N|N|N| +2481644|AAAAAAAAMONNFCAA|2082-05-29|2188|9518|730|2082|5|5|29|2|2082|730|9518|Friday|2082Q2|N|Y|N|2481616|2481735|2481279|2481554|N|N|N|N|N| +2481645|AAAAAAAANONNFCAA|2082-05-30|2188|9518|730|2082|6|5|30|2|2082|730|9518|Saturday|2082Q2|N|Y|N|2481616|2481735|2481280|2481555|N|N|N|N|N| +2481646|AAAAAAAAOONNFCAA|2082-05-31|2188|9518|730|2082|0|5|31|2|2082|730|9518|Sunday|2082Q2|N|N|N|2481616|2481735|2481281|2481556|N|N|N|N|N| +2481647|AAAAAAAAPONNFCAA|2082-06-01|2189|9518|731|2082|1|6|1|2|2082|731|9518|Monday|2082Q2|N|N|N|2481647|2481797|2481282|2481557|N|N|N|N|N| +2481648|AAAAAAAAAPNNFCAA|2082-06-02|2189|9519|731|2082|2|6|2|2|2082|731|9519|Tuesday|2082Q2|N|N|N|2481647|2481797|2481283|2481558|N|N|N|N|N| +2481649|AAAAAAAABPNNFCAA|2082-06-03|2189|9519|731|2082|3|6|3|2|2082|731|9519|Wednesday|2082Q2|N|N|N|2481647|2481797|2481284|2481559|N|N|N|N|N| +2481650|AAAAAAAACPNNFCAA|2082-06-04|2189|9519|731|2082|4|6|4|2|2082|731|9519|Thursday|2082Q2|N|N|N|2481647|2481797|2481285|2481560|N|N|N|N|N| +2481651|AAAAAAAADPNNFCAA|2082-06-05|2189|9519|731|2082|5|6|5|2|2082|731|9519|Friday|2082Q2|N|Y|N|2481647|2481797|2481286|2481561|N|N|N|N|N| +2481652|AAAAAAAAEPNNFCAA|2082-06-06|2189|9519|731|2082|6|6|6|2|2082|731|9519|Saturday|2082Q2|N|Y|N|2481647|2481797|2481287|2481562|N|N|N|N|N| +2481653|AAAAAAAAFPNNFCAA|2082-06-07|2189|9519|731|2082|0|6|7|2|2082|731|9519|Sunday|2082Q2|N|N|N|2481647|2481797|2481288|2481563|N|N|N|N|N| +2481654|AAAAAAAAGPNNFCAA|2082-06-08|2189|9519|731|2082|1|6|8|2|2082|731|9519|Monday|2082Q2|N|N|N|2481647|2481797|2481289|2481564|N|N|N|N|N| +2481655|AAAAAAAAHPNNFCAA|2082-06-09|2189|9520|731|2082|2|6|9|2|2082|731|9520|Tuesday|2082Q2|N|N|N|2481647|2481797|2481290|2481565|N|N|N|N|N| +2481656|AAAAAAAAIPNNFCAA|2082-06-10|2189|9520|731|2082|3|6|10|2|2082|731|9520|Wednesday|2082Q2|N|N|N|2481647|2481797|2481291|2481566|N|N|N|N|N| +2481657|AAAAAAAAJPNNFCAA|2082-06-11|2189|9520|731|2082|4|6|11|2|2082|731|9520|Thursday|2082Q2|N|N|N|2481647|2481797|2481292|2481567|N|N|N|N|N| +2481658|AAAAAAAAKPNNFCAA|2082-06-12|2189|9520|731|2082|5|6|12|2|2082|731|9520|Friday|2082Q2|N|Y|N|2481647|2481797|2481293|2481568|N|N|N|N|N| +2481659|AAAAAAAALPNNFCAA|2082-06-13|2189|9520|731|2082|6|6|13|2|2082|731|9520|Saturday|2082Q2|N|Y|N|2481647|2481797|2481294|2481569|N|N|N|N|N| +2481660|AAAAAAAAMPNNFCAA|2082-06-14|2189|9520|731|2082|0|6|14|2|2082|731|9520|Sunday|2082Q2|N|N|N|2481647|2481797|2481295|2481570|N|N|N|N|N| +2481661|AAAAAAAANPNNFCAA|2082-06-15|2189|9520|731|2082|1|6|15|2|2082|731|9520|Monday|2082Q2|N|N|N|2481647|2481797|2481296|2481571|N|N|N|N|N| +2481662|AAAAAAAAOPNNFCAA|2082-06-16|2189|9521|731|2082|2|6|16|2|2082|731|9521|Tuesday|2082Q2|N|N|N|2481647|2481797|2481297|2481572|N|N|N|N|N| +2481663|AAAAAAAAPPNNFCAA|2082-06-17|2189|9521|731|2082|3|6|17|2|2082|731|9521|Wednesday|2082Q2|N|N|N|2481647|2481797|2481298|2481573|N|N|N|N|N| +2481664|AAAAAAAAAAONFCAA|2082-06-18|2189|9521|731|2082|4|6|18|2|2082|731|9521|Thursday|2082Q2|N|N|N|2481647|2481797|2481299|2481574|N|N|N|N|N| +2481665|AAAAAAAABAONFCAA|2082-06-19|2189|9521|731|2082|5|6|19|2|2082|731|9521|Friday|2082Q2|N|Y|N|2481647|2481797|2481300|2481575|N|N|N|N|N| +2481666|AAAAAAAACAONFCAA|2082-06-20|2189|9521|731|2082|6|6|20|2|2082|731|9521|Saturday|2082Q2|N|Y|N|2481647|2481797|2481301|2481576|N|N|N|N|N| +2481667|AAAAAAAADAONFCAA|2082-06-21|2189|9521|731|2082|0|6|21|2|2082|731|9521|Sunday|2082Q2|N|N|N|2481647|2481797|2481302|2481577|N|N|N|N|N| +2481668|AAAAAAAAEAONFCAA|2082-06-22|2189|9521|731|2082|1|6|22|2|2082|731|9521|Monday|2082Q2|N|N|N|2481647|2481797|2481303|2481578|N|N|N|N|N| +2481669|AAAAAAAAFAONFCAA|2082-06-23|2189|9522|731|2082|2|6|23|2|2082|731|9522|Tuesday|2082Q2|N|N|N|2481647|2481797|2481304|2481579|N|N|N|N|N| +2481670|AAAAAAAAGAONFCAA|2082-06-24|2189|9522|731|2082|3|6|24|2|2082|731|9522|Wednesday|2082Q2|N|N|N|2481647|2481797|2481305|2481580|N|N|N|N|N| +2481671|AAAAAAAAHAONFCAA|2082-06-25|2189|9522|731|2082|4|6|25|2|2082|731|9522|Thursday|2082Q2|N|N|N|2481647|2481797|2481306|2481581|N|N|N|N|N| +2481672|AAAAAAAAIAONFCAA|2082-06-26|2189|9522|731|2082|5|6|26|2|2082|731|9522|Friday|2082Q2|N|Y|N|2481647|2481797|2481307|2481582|N|N|N|N|N| +2481673|AAAAAAAAJAONFCAA|2082-06-27|2189|9522|731|2082|6|6|27|2|2082|731|9522|Saturday|2082Q2|N|Y|N|2481647|2481797|2481308|2481583|N|N|N|N|N| +2481674|AAAAAAAAKAONFCAA|2082-06-28|2189|9522|731|2082|0|6|28|2|2082|731|9522|Sunday|2082Q2|N|N|N|2481647|2481797|2481309|2481584|N|N|N|N|N| +2481675|AAAAAAAALAONFCAA|2082-06-29|2189|9522|731|2082|1|6|29|2|2082|731|9522|Monday|2082Q2|N|N|N|2481647|2481797|2481310|2481585|N|N|N|N|N| +2481676|AAAAAAAAMAONFCAA|2082-06-30|2189|9523|731|2082|2|6|30|2|2082|731|9523|Tuesday|2082Q2|N|N|N|2481647|2481797|2481311|2481586|N|N|N|N|N| +2481677|AAAAAAAANAONFCAA|2082-07-01|2190|9523|731|2082|3|7|1|2|2082|731|9523|Wednesday|2082Q2|N|N|N|2481677|2481857|2481312|2481586|N|N|N|N|N| +2481678|AAAAAAAAOAONFCAA|2082-07-02|2190|9523|731|2082|4|7|2|3|2082|731|9523|Thursday|2082Q3|N|N|N|2481677|2481857|2481313|2481587|N|N|N|N|N| +2481679|AAAAAAAAPAONFCAA|2082-07-03|2190|9523|731|2082|5|7|3|3|2082|731|9523|Friday|2082Q3|N|Y|N|2481677|2481857|2481314|2481588|N|N|N|N|N| +2481680|AAAAAAAAABONFCAA|2082-07-04|2190|9523|731|2082|6|7|4|3|2082|731|9523|Saturday|2082Q3|N|Y|N|2481677|2481857|2481315|2481589|N|N|N|N|N| +2481681|AAAAAAAABBONFCAA|2082-07-05|2190|9523|731|2082|0|7|5|3|2082|731|9523|Sunday|2082Q3|Y|N|N|2481677|2481857|2481316|2481590|N|N|N|N|N| +2481682|AAAAAAAACBONFCAA|2082-07-06|2190|9523|731|2082|1|7|6|3|2082|731|9523|Monday|2082Q3|N|N|Y|2481677|2481857|2481317|2481591|N|N|N|N|N| +2481683|AAAAAAAADBONFCAA|2082-07-07|2190|9524|731|2082|2|7|7|3|2082|731|9524|Tuesday|2082Q3|N|N|N|2481677|2481857|2481318|2481592|N|N|N|N|N| +2481684|AAAAAAAAEBONFCAA|2082-07-08|2190|9524|731|2082|3|7|8|3|2082|731|9524|Wednesday|2082Q3|N|N|N|2481677|2481857|2481319|2481593|N|N|N|N|N| +2481685|AAAAAAAAFBONFCAA|2082-07-09|2190|9524|731|2082|4|7|9|3|2082|731|9524|Thursday|2082Q3|N|N|N|2481677|2481857|2481320|2481594|N|N|N|N|N| +2481686|AAAAAAAAGBONFCAA|2082-07-10|2190|9524|731|2082|5|7|10|3|2082|731|9524|Friday|2082Q3|N|Y|N|2481677|2481857|2481321|2481595|N|N|N|N|N| +2481687|AAAAAAAAHBONFCAA|2082-07-11|2190|9524|731|2082|6|7|11|3|2082|731|9524|Saturday|2082Q3|N|Y|N|2481677|2481857|2481322|2481596|N|N|N|N|N| +2481688|AAAAAAAAIBONFCAA|2082-07-12|2190|9524|731|2082|0|7|12|3|2082|731|9524|Sunday|2082Q3|N|N|N|2481677|2481857|2481323|2481597|N|N|N|N|N| +2481689|AAAAAAAAJBONFCAA|2082-07-13|2190|9524|731|2082|1|7|13|3|2082|731|9524|Monday|2082Q3|N|N|N|2481677|2481857|2481324|2481598|N|N|N|N|N| +2481690|AAAAAAAAKBONFCAA|2082-07-14|2190|9525|731|2082|2|7|14|3|2082|731|9525|Tuesday|2082Q3|N|N|N|2481677|2481857|2481325|2481599|N|N|N|N|N| +2481691|AAAAAAAALBONFCAA|2082-07-15|2190|9525|731|2082|3|7|15|3|2082|731|9525|Wednesday|2082Q3|N|N|N|2481677|2481857|2481326|2481600|N|N|N|N|N| +2481692|AAAAAAAAMBONFCAA|2082-07-16|2190|9525|731|2082|4|7|16|3|2082|731|9525|Thursday|2082Q3|N|N|N|2481677|2481857|2481327|2481601|N|N|N|N|N| +2481693|AAAAAAAANBONFCAA|2082-07-17|2190|9525|731|2082|5|7|17|3|2082|731|9525|Friday|2082Q3|N|Y|N|2481677|2481857|2481328|2481602|N|N|N|N|N| +2481694|AAAAAAAAOBONFCAA|2082-07-18|2190|9525|731|2082|6|7|18|3|2082|731|9525|Saturday|2082Q3|N|Y|N|2481677|2481857|2481329|2481603|N|N|N|N|N| +2481695|AAAAAAAAPBONFCAA|2082-07-19|2190|9525|731|2082|0|7|19|3|2082|731|9525|Sunday|2082Q3|N|N|N|2481677|2481857|2481330|2481604|N|N|N|N|N| +2481696|AAAAAAAAACONFCAA|2082-07-20|2190|9525|731|2082|1|7|20|3|2082|731|9525|Monday|2082Q3|N|N|N|2481677|2481857|2481331|2481605|N|N|N|N|N| +2481697|AAAAAAAABCONFCAA|2082-07-21|2190|9526|731|2082|2|7|21|3|2082|731|9526|Tuesday|2082Q3|N|N|N|2481677|2481857|2481332|2481606|N|N|N|N|N| +2481698|AAAAAAAACCONFCAA|2082-07-22|2190|9526|731|2082|3|7|22|3|2082|731|9526|Wednesday|2082Q3|N|N|N|2481677|2481857|2481333|2481607|N|N|N|N|N| +2481699|AAAAAAAADCONFCAA|2082-07-23|2190|9526|731|2082|4|7|23|3|2082|731|9526|Thursday|2082Q3|N|N|N|2481677|2481857|2481334|2481608|N|N|N|N|N| +2481700|AAAAAAAAECONFCAA|2082-07-24|2190|9526|731|2082|5|7|24|3|2082|731|9526|Friday|2082Q3|N|Y|N|2481677|2481857|2481335|2481609|N|N|N|N|N| +2481701|AAAAAAAAFCONFCAA|2082-07-25|2190|9526|731|2082|6|7|25|3|2082|731|9526|Saturday|2082Q3|N|Y|N|2481677|2481857|2481336|2481610|N|N|N|N|N| +2481702|AAAAAAAAGCONFCAA|2082-07-26|2190|9526|731|2082|0|7|26|3|2082|731|9526|Sunday|2082Q3|N|N|N|2481677|2481857|2481337|2481611|N|N|N|N|N| +2481703|AAAAAAAAHCONFCAA|2082-07-27|2190|9526|731|2082|1|7|27|3|2082|731|9526|Monday|2082Q3|N|N|N|2481677|2481857|2481338|2481612|N|N|N|N|N| +2481704|AAAAAAAAICONFCAA|2082-07-28|2190|9527|731|2082|2|7|28|3|2082|731|9527|Tuesday|2082Q3|N|N|N|2481677|2481857|2481339|2481613|N|N|N|N|N| +2481705|AAAAAAAAJCONFCAA|2082-07-29|2190|9527|731|2082|3|7|29|3|2082|731|9527|Wednesday|2082Q3|N|N|N|2481677|2481857|2481340|2481614|N|N|N|N|N| +2481706|AAAAAAAAKCONFCAA|2082-07-30|2190|9527|731|2082|4|7|30|3|2082|731|9527|Thursday|2082Q3|N|N|N|2481677|2481857|2481341|2481615|N|N|N|N|N| +2481707|AAAAAAAALCONFCAA|2082-07-31|2190|9527|731|2082|5|7|31|3|2082|731|9527|Friday|2082Q3|N|Y|N|2481677|2481857|2481342|2481616|N|N|N|N|N| +2481708|AAAAAAAAMCONFCAA|2082-08-01|2191|9527|731|2082|6|8|1|3|2082|731|9527|Saturday|2082Q3|N|Y|N|2481708|2481919|2481343|2481617|N|N|N|N|N| +2481709|AAAAAAAANCONFCAA|2082-08-02|2191|9527|731|2082|0|8|2|3|2082|731|9527|Sunday|2082Q3|N|N|N|2481708|2481919|2481344|2481618|N|N|N|N|N| +2481710|AAAAAAAAOCONFCAA|2082-08-03|2191|9527|731|2082|1|8|3|3|2082|731|9527|Monday|2082Q3|N|N|N|2481708|2481919|2481345|2481619|N|N|N|N|N| +2481711|AAAAAAAAPCONFCAA|2082-08-04|2191|9528|731|2082|2|8|4|3|2082|731|9528|Tuesday|2082Q3|N|N|N|2481708|2481919|2481346|2481620|N|N|N|N|N| +2481712|AAAAAAAAADONFCAA|2082-08-05|2191|9528|731|2082|3|8|5|3|2082|731|9528|Wednesday|2082Q3|N|N|N|2481708|2481919|2481347|2481621|N|N|N|N|N| +2481713|AAAAAAAABDONFCAA|2082-08-06|2191|9528|731|2082|4|8|6|3|2082|731|9528|Thursday|2082Q3|N|N|N|2481708|2481919|2481348|2481622|N|N|N|N|N| +2481714|AAAAAAAACDONFCAA|2082-08-07|2191|9528|731|2082|5|8|7|3|2082|731|9528|Friday|2082Q3|N|Y|N|2481708|2481919|2481349|2481623|N|N|N|N|N| +2481715|AAAAAAAADDONFCAA|2082-08-08|2191|9528|731|2082|6|8|8|3|2082|731|9528|Saturday|2082Q3|N|Y|N|2481708|2481919|2481350|2481624|N|N|N|N|N| +2481716|AAAAAAAAEDONFCAA|2082-08-09|2191|9528|731|2082|0|8|9|3|2082|731|9528|Sunday|2082Q3|N|N|N|2481708|2481919|2481351|2481625|N|N|N|N|N| +2481717|AAAAAAAAFDONFCAA|2082-08-10|2191|9528|731|2082|1|8|10|3|2082|731|9528|Monday|2082Q3|N|N|N|2481708|2481919|2481352|2481626|N|N|N|N|N| +2481718|AAAAAAAAGDONFCAA|2082-08-11|2191|9529|731|2082|2|8|11|3|2082|731|9529|Tuesday|2082Q3|N|N|N|2481708|2481919|2481353|2481627|N|N|N|N|N| +2481719|AAAAAAAAHDONFCAA|2082-08-12|2191|9529|731|2082|3|8|12|3|2082|731|9529|Wednesday|2082Q3|N|N|N|2481708|2481919|2481354|2481628|N|N|N|N|N| +2481720|AAAAAAAAIDONFCAA|2082-08-13|2191|9529|731|2082|4|8|13|3|2082|731|9529|Thursday|2082Q3|N|N|N|2481708|2481919|2481355|2481629|N|N|N|N|N| +2481721|AAAAAAAAJDONFCAA|2082-08-14|2191|9529|731|2082|5|8|14|3|2082|731|9529|Friday|2082Q3|N|Y|N|2481708|2481919|2481356|2481630|N|N|N|N|N| +2481722|AAAAAAAAKDONFCAA|2082-08-15|2191|9529|731|2082|6|8|15|3|2082|731|9529|Saturday|2082Q3|N|Y|N|2481708|2481919|2481357|2481631|N|N|N|N|N| +2481723|AAAAAAAALDONFCAA|2082-08-16|2191|9529|731|2082|0|8|16|3|2082|731|9529|Sunday|2082Q3|N|N|N|2481708|2481919|2481358|2481632|N|N|N|N|N| +2481724|AAAAAAAAMDONFCAA|2082-08-17|2191|9529|731|2082|1|8|17|3|2082|731|9529|Monday|2082Q3|N|N|N|2481708|2481919|2481359|2481633|N|N|N|N|N| +2481725|AAAAAAAANDONFCAA|2082-08-18|2191|9530|731|2082|2|8|18|3|2082|731|9530|Tuesday|2082Q3|N|N|N|2481708|2481919|2481360|2481634|N|N|N|N|N| +2481726|AAAAAAAAODONFCAA|2082-08-19|2191|9530|731|2082|3|8|19|3|2082|731|9530|Wednesday|2082Q3|N|N|N|2481708|2481919|2481361|2481635|N|N|N|N|N| +2481727|AAAAAAAAPDONFCAA|2082-08-20|2191|9530|731|2082|4|8|20|3|2082|731|9530|Thursday|2082Q3|N|N|N|2481708|2481919|2481362|2481636|N|N|N|N|N| +2481728|AAAAAAAAAEONFCAA|2082-08-21|2191|9530|731|2082|5|8|21|3|2082|731|9530|Friday|2082Q3|N|Y|N|2481708|2481919|2481363|2481637|N|N|N|N|N| +2481729|AAAAAAAABEONFCAA|2082-08-22|2191|9530|731|2082|6|8|22|3|2082|731|9530|Saturday|2082Q3|N|Y|N|2481708|2481919|2481364|2481638|N|N|N|N|N| +2481730|AAAAAAAACEONFCAA|2082-08-23|2191|9530|731|2082|0|8|23|3|2082|731|9530|Sunday|2082Q3|N|N|N|2481708|2481919|2481365|2481639|N|N|N|N|N| +2481731|AAAAAAAADEONFCAA|2082-08-24|2191|9530|731|2082|1|8|24|3|2082|731|9530|Monday|2082Q3|N|N|N|2481708|2481919|2481366|2481640|N|N|N|N|N| +2481732|AAAAAAAAEEONFCAA|2082-08-25|2191|9531|731|2082|2|8|25|3|2082|731|9531|Tuesday|2082Q3|N|N|N|2481708|2481919|2481367|2481641|N|N|N|N|N| +2481733|AAAAAAAAFEONFCAA|2082-08-26|2191|9531|731|2082|3|8|26|3|2082|731|9531|Wednesday|2082Q3|N|N|N|2481708|2481919|2481368|2481642|N|N|N|N|N| +2481734|AAAAAAAAGEONFCAA|2082-08-27|2191|9531|731|2082|4|8|27|3|2082|731|9531|Thursday|2082Q3|N|N|N|2481708|2481919|2481369|2481643|N|N|N|N|N| +2481735|AAAAAAAAHEONFCAA|2082-08-28|2191|9531|731|2082|5|8|28|3|2082|731|9531|Friday|2082Q3|N|Y|N|2481708|2481919|2481370|2481644|N|N|N|N|N| +2481736|AAAAAAAAIEONFCAA|2082-08-29|2191|9531|731|2082|6|8|29|3|2082|731|9531|Saturday|2082Q3|N|Y|N|2481708|2481919|2481371|2481645|N|N|N|N|N| +2481737|AAAAAAAAJEONFCAA|2082-08-30|2191|9531|731|2082|0|8|30|3|2082|731|9531|Sunday|2082Q3|N|N|N|2481708|2481919|2481372|2481646|N|N|N|N|N| +2481738|AAAAAAAAKEONFCAA|2082-08-31|2191|9531|731|2082|1|8|31|3|2082|731|9531|Monday|2082Q3|N|N|N|2481708|2481919|2481373|2481647|N|N|N|N|N| +2481739|AAAAAAAALEONFCAA|2082-09-01|2192|9532|732|2082|2|9|1|3|2082|732|9532|Tuesday|2082Q3|N|N|N|2481739|2481981|2481374|2481648|N|N|N|N|N| +2481740|AAAAAAAAMEONFCAA|2082-09-02|2192|9532|732|2082|3|9|2|3|2082|732|9532|Wednesday|2082Q3|N|N|N|2481739|2481981|2481375|2481649|N|N|N|N|N| +2481741|AAAAAAAANEONFCAA|2082-09-03|2192|9532|732|2082|4|9|3|3|2082|732|9532|Thursday|2082Q3|N|N|N|2481739|2481981|2481376|2481650|N|N|N|N|N| +2481742|AAAAAAAAOEONFCAA|2082-09-04|2192|9532|732|2082|5|9|4|3|2082|732|9532|Friday|2082Q3|N|Y|N|2481739|2481981|2481377|2481651|N|N|N|N|N| +2481743|AAAAAAAAPEONFCAA|2082-09-05|2192|9532|732|2082|6|9|5|3|2082|732|9532|Saturday|2082Q3|N|Y|N|2481739|2481981|2481378|2481652|N|N|N|N|N| +2481744|AAAAAAAAAFONFCAA|2082-09-06|2192|9532|732|2082|0|9|6|3|2082|732|9532|Sunday|2082Q3|N|N|N|2481739|2481981|2481379|2481653|N|N|N|N|N| +2481745|AAAAAAAABFONFCAA|2082-09-07|2192|9532|732|2082|1|9|7|3|2082|732|9532|Monday|2082Q3|N|N|N|2481739|2481981|2481380|2481654|N|N|N|N|N| +2481746|AAAAAAAACFONFCAA|2082-09-08|2192|9533|732|2082|2|9|8|3|2082|732|9533|Tuesday|2082Q3|N|N|N|2481739|2481981|2481381|2481655|N|N|N|N|N| +2481747|AAAAAAAADFONFCAA|2082-09-09|2192|9533|732|2082|3|9|9|3|2082|732|9533|Wednesday|2082Q3|N|N|N|2481739|2481981|2481382|2481656|N|N|N|N|N| +2481748|AAAAAAAAEFONFCAA|2082-09-10|2192|9533|732|2082|4|9|10|3|2082|732|9533|Thursday|2082Q3|N|N|N|2481739|2481981|2481383|2481657|N|N|N|N|N| +2481749|AAAAAAAAFFONFCAA|2082-09-11|2192|9533|732|2082|5|9|11|3|2082|732|9533|Friday|2082Q3|N|Y|N|2481739|2481981|2481384|2481658|N|N|N|N|N| +2481750|AAAAAAAAGFONFCAA|2082-09-12|2192|9533|732|2082|6|9|12|3|2082|732|9533|Saturday|2082Q3|N|Y|N|2481739|2481981|2481385|2481659|N|N|N|N|N| +2481751|AAAAAAAAHFONFCAA|2082-09-13|2192|9533|732|2082|0|9|13|3|2082|732|9533|Sunday|2082Q3|N|N|N|2481739|2481981|2481386|2481660|N|N|N|N|N| +2481752|AAAAAAAAIFONFCAA|2082-09-14|2192|9533|732|2082|1|9|14|3|2082|732|9533|Monday|2082Q3|N|N|N|2481739|2481981|2481387|2481661|N|N|N|N|N| +2481753|AAAAAAAAJFONFCAA|2082-09-15|2192|9534|732|2082|2|9|15|3|2082|732|9534|Tuesday|2082Q3|N|N|N|2481739|2481981|2481388|2481662|N|N|N|N|N| +2481754|AAAAAAAAKFONFCAA|2082-09-16|2192|9534|732|2082|3|9|16|3|2082|732|9534|Wednesday|2082Q3|N|N|N|2481739|2481981|2481389|2481663|N|N|N|N|N| +2481755|AAAAAAAALFONFCAA|2082-09-17|2192|9534|732|2082|4|9|17|3|2082|732|9534|Thursday|2082Q3|N|N|N|2481739|2481981|2481390|2481664|N|N|N|N|N| +2481756|AAAAAAAAMFONFCAA|2082-09-18|2192|9534|732|2082|5|9|18|3|2082|732|9534|Friday|2082Q3|N|Y|N|2481739|2481981|2481391|2481665|N|N|N|N|N| +2481757|AAAAAAAANFONFCAA|2082-09-19|2192|9534|732|2082|6|9|19|3|2082|732|9534|Saturday|2082Q3|N|Y|N|2481739|2481981|2481392|2481666|N|N|N|N|N| +2481758|AAAAAAAAOFONFCAA|2082-09-20|2192|9534|732|2082|0|9|20|3|2082|732|9534|Sunday|2082Q3|N|N|N|2481739|2481981|2481393|2481667|N|N|N|N|N| +2481759|AAAAAAAAPFONFCAA|2082-09-21|2192|9534|732|2082|1|9|21|3|2082|732|9534|Monday|2082Q3|N|N|N|2481739|2481981|2481394|2481668|N|N|N|N|N| +2481760|AAAAAAAAAGONFCAA|2082-09-22|2192|9535|732|2082|2|9|22|3|2082|732|9535|Tuesday|2082Q3|N|N|N|2481739|2481981|2481395|2481669|N|N|N|N|N| +2481761|AAAAAAAABGONFCAA|2082-09-23|2192|9535|732|2082|3|9|23|3|2082|732|9535|Wednesday|2082Q3|N|N|N|2481739|2481981|2481396|2481670|N|N|N|N|N| +2481762|AAAAAAAACGONFCAA|2082-09-24|2192|9535|732|2082|4|9|24|3|2082|732|9535|Thursday|2082Q3|N|N|N|2481739|2481981|2481397|2481671|N|N|N|N|N| +2481763|AAAAAAAADGONFCAA|2082-09-25|2192|9535|732|2082|5|9|25|3|2082|732|9535|Friday|2082Q3|N|Y|N|2481739|2481981|2481398|2481672|N|N|N|N|N| +2481764|AAAAAAAAEGONFCAA|2082-09-26|2192|9535|732|2082|6|9|26|3|2082|732|9535|Saturday|2082Q3|N|Y|N|2481739|2481981|2481399|2481673|N|N|N|N|N| +2481765|AAAAAAAAFGONFCAA|2082-09-27|2192|9535|732|2082|0|9|27|3|2082|732|9535|Sunday|2082Q3|N|N|N|2481739|2481981|2481400|2481674|N|N|N|N|N| +2481766|AAAAAAAAGGONFCAA|2082-09-28|2192|9535|732|2082|1|9|28|3|2082|732|9535|Monday|2082Q3|N|N|N|2481739|2481981|2481401|2481675|N|N|N|N|N| +2481767|AAAAAAAAHGONFCAA|2082-09-29|2192|9536|732|2082|2|9|29|3|2082|732|9536|Tuesday|2082Q3|N|N|N|2481739|2481981|2481402|2481676|N|N|N|N|N| +2481768|AAAAAAAAIGONFCAA|2082-09-30|2192|9536|732|2082|3|9|30|3|2082|732|9536|Wednesday|2082Q3|N|N|N|2481739|2481981|2481403|2481677|N|N|N|N|N| +2481769|AAAAAAAAJGONFCAA|2082-10-01|2193|9536|732|2082|4|10|1|3|2082|732|9536|Thursday|2082Q3|N|N|N|2481769|2482041|2481404|2481677|N|N|N|N|N| +2481770|AAAAAAAAKGONFCAA|2082-10-02|2193|9536|732|2082|5|10|2|4|2082|732|9536|Friday|2082Q4|N|Y|N|2481769|2482041|2481405|2481678|N|N|N|N|N| +2481771|AAAAAAAALGONFCAA|2082-10-03|2193|9536|732|2082|6|10|3|4|2082|732|9536|Saturday|2082Q4|N|Y|N|2481769|2482041|2481406|2481679|N|N|N|N|N| +2481772|AAAAAAAAMGONFCAA|2082-10-04|2193|9536|732|2082|0|10|4|4|2082|732|9536|Sunday|2082Q4|N|N|N|2481769|2482041|2481407|2481680|N|N|N|N|N| +2481773|AAAAAAAANGONFCAA|2082-10-05|2193|9536|732|2082|1|10|5|4|2082|732|9536|Monday|2082Q4|N|N|N|2481769|2482041|2481408|2481681|N|N|N|N|N| +2481774|AAAAAAAAOGONFCAA|2082-10-06|2193|9537|732|2082|2|10|6|4|2082|732|9537|Tuesday|2082Q4|N|N|N|2481769|2482041|2481409|2481682|N|N|N|N|N| +2481775|AAAAAAAAPGONFCAA|2082-10-07|2193|9537|732|2082|3|10|7|4|2082|732|9537|Wednesday|2082Q4|N|N|N|2481769|2482041|2481410|2481683|N|N|N|N|N| +2481776|AAAAAAAAAHONFCAA|2082-10-08|2193|9537|732|2082|4|10|8|4|2082|732|9537|Thursday|2082Q4|N|N|N|2481769|2482041|2481411|2481684|N|N|N|N|N| +2481777|AAAAAAAABHONFCAA|2082-10-09|2193|9537|732|2082|5|10|9|4|2082|732|9537|Friday|2082Q4|N|Y|N|2481769|2482041|2481412|2481685|N|N|N|N|N| +2481778|AAAAAAAACHONFCAA|2082-10-10|2193|9537|732|2082|6|10|10|4|2082|732|9537|Saturday|2082Q4|N|Y|N|2481769|2482041|2481413|2481686|N|N|N|N|N| +2481779|AAAAAAAADHONFCAA|2082-10-11|2193|9537|732|2082|0|10|11|4|2082|732|9537|Sunday|2082Q4|N|N|N|2481769|2482041|2481414|2481687|N|N|N|N|N| +2481780|AAAAAAAAEHONFCAA|2082-10-12|2193|9537|732|2082|1|10|12|4|2082|732|9537|Monday|2082Q4|N|N|N|2481769|2482041|2481415|2481688|N|N|N|N|N| +2481781|AAAAAAAAFHONFCAA|2082-10-13|2193|9538|732|2082|2|10|13|4|2082|732|9538|Tuesday|2082Q4|N|N|N|2481769|2482041|2481416|2481689|N|N|N|N|N| +2481782|AAAAAAAAGHONFCAA|2082-10-14|2193|9538|732|2082|3|10|14|4|2082|732|9538|Wednesday|2082Q4|N|N|N|2481769|2482041|2481417|2481690|N|N|N|N|N| +2481783|AAAAAAAAHHONFCAA|2082-10-15|2193|9538|732|2082|4|10|15|4|2082|732|9538|Thursday|2082Q4|N|N|N|2481769|2482041|2481418|2481691|N|N|N|N|N| +2481784|AAAAAAAAIHONFCAA|2082-10-16|2193|9538|732|2082|5|10|16|4|2082|732|9538|Friday|2082Q4|N|Y|N|2481769|2482041|2481419|2481692|N|N|N|N|N| +2481785|AAAAAAAAJHONFCAA|2082-10-17|2193|9538|732|2082|6|10|17|4|2082|732|9538|Saturday|2082Q4|N|Y|N|2481769|2482041|2481420|2481693|N|N|N|N|N| +2481786|AAAAAAAAKHONFCAA|2082-10-18|2193|9538|732|2082|0|10|18|4|2082|732|9538|Sunday|2082Q4|N|N|N|2481769|2482041|2481421|2481694|N|N|N|N|N| +2481787|AAAAAAAALHONFCAA|2082-10-19|2193|9538|732|2082|1|10|19|4|2082|732|9538|Monday|2082Q4|N|N|N|2481769|2482041|2481422|2481695|N|N|N|N|N| +2481788|AAAAAAAAMHONFCAA|2082-10-20|2193|9539|732|2082|2|10|20|4|2082|732|9539|Tuesday|2082Q4|N|N|N|2481769|2482041|2481423|2481696|N|N|N|N|N| +2481789|AAAAAAAANHONFCAA|2082-10-21|2193|9539|732|2082|3|10|21|4|2082|732|9539|Wednesday|2082Q4|N|N|N|2481769|2482041|2481424|2481697|N|N|N|N|N| +2481790|AAAAAAAAOHONFCAA|2082-10-22|2193|9539|732|2082|4|10|22|4|2082|732|9539|Thursday|2082Q4|N|N|N|2481769|2482041|2481425|2481698|N|N|N|N|N| +2481791|AAAAAAAAPHONFCAA|2082-10-23|2193|9539|732|2082|5|10|23|4|2082|732|9539|Friday|2082Q4|N|Y|N|2481769|2482041|2481426|2481699|N|N|N|N|N| +2481792|AAAAAAAAAIONFCAA|2082-10-24|2193|9539|732|2082|6|10|24|4|2082|732|9539|Saturday|2082Q4|N|Y|N|2481769|2482041|2481427|2481700|N|N|N|N|N| +2481793|AAAAAAAABIONFCAA|2082-10-25|2193|9539|732|2082|0|10|25|4|2082|732|9539|Sunday|2082Q4|N|N|N|2481769|2482041|2481428|2481701|N|N|N|N|N| +2481794|AAAAAAAACIONFCAA|2082-10-26|2193|9539|732|2082|1|10|26|4|2082|732|9539|Monday|2082Q4|N|N|N|2481769|2482041|2481429|2481702|N|N|N|N|N| +2481795|AAAAAAAADIONFCAA|2082-10-27|2193|9540|732|2082|2|10|27|4|2082|732|9540|Tuesday|2082Q4|N|N|N|2481769|2482041|2481430|2481703|N|N|N|N|N| +2481796|AAAAAAAAEIONFCAA|2082-10-28|2193|9540|732|2082|3|10|28|4|2082|732|9540|Wednesday|2082Q4|N|N|N|2481769|2482041|2481431|2481704|N|N|N|N|N| +2481797|AAAAAAAAFIONFCAA|2082-10-29|2193|9540|732|2082|4|10|29|4|2082|732|9540|Thursday|2082Q4|N|N|N|2481769|2482041|2481432|2481705|N|N|N|N|N| +2481798|AAAAAAAAGIONFCAA|2082-10-30|2193|9540|732|2082|5|10|30|4|2082|732|9540|Friday|2082Q4|N|Y|N|2481769|2482041|2481433|2481706|N|N|N|N|N| +2481799|AAAAAAAAHIONFCAA|2082-10-31|2193|9540|732|2082|6|10|31|4|2082|732|9540|Saturday|2082Q4|N|Y|N|2481769|2482041|2481434|2481707|N|N|N|N|N| +2481800|AAAAAAAAIIONFCAA|2082-11-01|2194|9540|732|2082|0|11|1|4|2082|732|9540|Sunday|2082Q4|N|N|N|2481800|2482103|2481435|2481708|N|N|N|N|N| +2481801|AAAAAAAAJIONFCAA|2082-11-02|2194|9540|732|2082|1|11|2|4|2082|732|9540|Monday|2082Q4|N|N|N|2481800|2482103|2481436|2481709|N|N|N|N|N| +2481802|AAAAAAAAKIONFCAA|2082-11-03|2194|9541|732|2082|2|11|3|4|2082|732|9541|Tuesday|2082Q4|N|N|N|2481800|2482103|2481437|2481710|N|N|N|N|N| +2481803|AAAAAAAALIONFCAA|2082-11-04|2194|9541|732|2082|3|11|4|4|2082|732|9541|Wednesday|2082Q4|N|N|N|2481800|2482103|2481438|2481711|N|N|N|N|N| +2481804|AAAAAAAAMIONFCAA|2082-11-05|2194|9541|732|2082|4|11|5|4|2082|732|9541|Thursday|2082Q4|N|N|N|2481800|2482103|2481439|2481712|N|N|N|N|N| +2481805|AAAAAAAANIONFCAA|2082-11-06|2194|9541|732|2082|5|11|6|4|2082|732|9541|Friday|2082Q4|N|Y|N|2481800|2482103|2481440|2481713|N|N|N|N|N| +2481806|AAAAAAAAOIONFCAA|2082-11-07|2194|9541|732|2082|6|11|7|4|2082|732|9541|Saturday|2082Q4|N|Y|N|2481800|2482103|2481441|2481714|N|N|N|N|N| +2481807|AAAAAAAAPIONFCAA|2082-11-08|2194|9541|732|2082|0|11|8|4|2082|732|9541|Sunday|2082Q4|N|N|N|2481800|2482103|2481442|2481715|N|N|N|N|N| +2481808|AAAAAAAAAJONFCAA|2082-11-09|2194|9541|732|2082|1|11|9|4|2082|732|9541|Monday|2082Q4|N|N|N|2481800|2482103|2481443|2481716|N|N|N|N|N| +2481809|AAAAAAAABJONFCAA|2082-11-10|2194|9542|732|2082|2|11|10|4|2082|732|9542|Tuesday|2082Q4|N|N|N|2481800|2482103|2481444|2481717|N|N|N|N|N| +2481810|AAAAAAAACJONFCAA|2082-11-11|2194|9542|732|2082|3|11|11|4|2082|732|9542|Wednesday|2082Q4|N|N|N|2481800|2482103|2481445|2481718|N|N|N|N|N| +2481811|AAAAAAAADJONFCAA|2082-11-12|2194|9542|732|2082|4|11|12|4|2082|732|9542|Thursday|2082Q4|N|N|N|2481800|2482103|2481446|2481719|N|N|N|N|N| +2481812|AAAAAAAAEJONFCAA|2082-11-13|2194|9542|732|2082|5|11|13|4|2082|732|9542|Friday|2082Q4|N|Y|N|2481800|2482103|2481447|2481720|N|N|N|N|N| +2481813|AAAAAAAAFJONFCAA|2082-11-14|2194|9542|732|2082|6|11|14|4|2082|732|9542|Saturday|2082Q4|N|Y|N|2481800|2482103|2481448|2481721|N|N|N|N|N| +2481814|AAAAAAAAGJONFCAA|2082-11-15|2194|9542|732|2082|0|11|15|4|2082|732|9542|Sunday|2082Q4|N|N|N|2481800|2482103|2481449|2481722|N|N|N|N|N| +2481815|AAAAAAAAHJONFCAA|2082-11-16|2194|9542|732|2082|1|11|16|4|2082|732|9542|Monday|2082Q4|N|N|N|2481800|2482103|2481450|2481723|N|N|N|N|N| +2481816|AAAAAAAAIJONFCAA|2082-11-17|2194|9543|732|2082|2|11|17|4|2082|732|9543|Tuesday|2082Q4|N|N|N|2481800|2482103|2481451|2481724|N|N|N|N|N| +2481817|AAAAAAAAJJONFCAA|2082-11-18|2194|9543|732|2082|3|11|18|4|2082|732|9543|Wednesday|2082Q4|N|N|N|2481800|2482103|2481452|2481725|N|N|N|N|N| +2481818|AAAAAAAAKJONFCAA|2082-11-19|2194|9543|732|2082|4|11|19|4|2082|732|9543|Thursday|2082Q4|N|N|N|2481800|2482103|2481453|2481726|N|N|N|N|N| +2481819|AAAAAAAALJONFCAA|2082-11-20|2194|9543|732|2082|5|11|20|4|2082|732|9543|Friday|2082Q4|N|Y|N|2481800|2482103|2481454|2481727|N|N|N|N|N| +2481820|AAAAAAAAMJONFCAA|2082-11-21|2194|9543|732|2082|6|11|21|4|2082|732|9543|Saturday|2082Q4|N|Y|N|2481800|2482103|2481455|2481728|N|N|N|N|N| +2481821|AAAAAAAANJONFCAA|2082-11-22|2194|9543|732|2082|0|11|22|4|2082|732|9543|Sunday|2082Q4|N|N|N|2481800|2482103|2481456|2481729|N|N|N|N|N| +2481822|AAAAAAAAOJONFCAA|2082-11-23|2194|9543|732|2082|1|11|23|4|2082|732|9543|Monday|2082Q4|N|N|N|2481800|2482103|2481457|2481730|N|N|N|N|N| +2481823|AAAAAAAAPJONFCAA|2082-11-24|2194|9544|732|2082|2|11|24|4|2082|732|9544|Tuesday|2082Q4|N|N|N|2481800|2482103|2481458|2481731|N|N|N|N|N| +2481824|AAAAAAAAAKONFCAA|2082-11-25|2194|9544|732|2082|3|11|25|4|2082|732|9544|Wednesday|2082Q4|N|N|N|2481800|2482103|2481459|2481732|N|N|N|N|N| +2481825|AAAAAAAABKONFCAA|2082-11-26|2194|9544|732|2082|4|11|26|4|2082|732|9544|Thursday|2082Q4|N|N|N|2481800|2482103|2481460|2481733|N|N|N|N|N| +2481826|AAAAAAAACKONFCAA|2082-11-27|2194|9544|732|2082|5|11|27|4|2082|732|9544|Friday|2082Q4|N|Y|N|2481800|2482103|2481461|2481734|N|N|N|N|N| +2481827|AAAAAAAADKONFCAA|2082-11-28|2194|9544|732|2082|6|11|28|4|2082|732|9544|Saturday|2082Q4|N|Y|N|2481800|2482103|2481462|2481735|N|N|N|N|N| +2481828|AAAAAAAAEKONFCAA|2082-11-29|2194|9544|732|2082|0|11|29|4|2082|732|9544|Sunday|2082Q4|N|N|N|2481800|2482103|2481463|2481736|N|N|N|N|N| +2481829|AAAAAAAAFKONFCAA|2082-11-30|2194|9544|732|2082|1|11|30|4|2082|732|9544|Monday|2082Q4|N|N|N|2481800|2482103|2481464|2481737|N|N|N|N|N| +2481830|AAAAAAAAGKONFCAA|2082-12-01|2195|9545|733|2082|2|12|1|4|2082|733|9545|Tuesday|2082Q4|N|N|N|2481830|2482163|2481465|2481738|N|N|N|N|N| +2481831|AAAAAAAAHKONFCAA|2082-12-02|2195|9545|733|2082|3|12|2|4|2082|733|9545|Wednesday|2082Q4|N|N|N|2481830|2482163|2481466|2481739|N|N|N|N|N| +2481832|AAAAAAAAIKONFCAA|2082-12-03|2195|9545|733|2082|4|12|3|4|2082|733|9545|Thursday|2082Q4|N|N|N|2481830|2482163|2481467|2481740|N|N|N|N|N| +2481833|AAAAAAAAJKONFCAA|2082-12-04|2195|9545|733|2082|5|12|4|4|2082|733|9545|Friday|2082Q4|N|Y|N|2481830|2482163|2481468|2481741|N|N|N|N|N| +2481834|AAAAAAAAKKONFCAA|2082-12-05|2195|9545|733|2082|6|12|5|4|2082|733|9545|Saturday|2082Q4|N|Y|N|2481830|2482163|2481469|2481742|N|N|N|N|N| +2481835|AAAAAAAALKONFCAA|2082-12-06|2195|9545|733|2082|0|12|6|4|2082|733|9545|Sunday|2082Q4|N|N|N|2481830|2482163|2481470|2481743|N|N|N|N|N| +2481836|AAAAAAAAMKONFCAA|2082-12-07|2195|9545|733|2082|1|12|7|4|2082|733|9545|Monday|2082Q4|N|N|N|2481830|2482163|2481471|2481744|N|N|N|N|N| +2481837|AAAAAAAANKONFCAA|2082-12-08|2195|9546|733|2082|2|12|8|4|2082|733|9546|Tuesday|2082Q4|N|N|N|2481830|2482163|2481472|2481745|N|N|N|N|N| +2481838|AAAAAAAAOKONFCAA|2082-12-09|2195|9546|733|2082|3|12|9|4|2082|733|9546|Wednesday|2082Q4|N|N|N|2481830|2482163|2481473|2481746|N|N|N|N|N| +2481839|AAAAAAAAPKONFCAA|2082-12-10|2195|9546|733|2082|4|12|10|4|2082|733|9546|Thursday|2082Q4|N|N|N|2481830|2482163|2481474|2481747|N|N|N|N|N| +2481840|AAAAAAAAALONFCAA|2082-12-11|2195|9546|733|2082|5|12|11|4|2082|733|9546|Friday|2082Q4|N|Y|N|2481830|2482163|2481475|2481748|N|N|N|N|N| +2481841|AAAAAAAABLONFCAA|2082-12-12|2195|9546|733|2082|6|12|12|4|2082|733|9546|Saturday|2082Q4|N|Y|N|2481830|2482163|2481476|2481749|N|N|N|N|N| +2481842|AAAAAAAACLONFCAA|2082-12-13|2195|9546|733|2082|0|12|13|4|2082|733|9546|Sunday|2082Q4|N|N|N|2481830|2482163|2481477|2481750|N|N|N|N|N| +2481843|AAAAAAAADLONFCAA|2082-12-14|2195|9546|733|2082|1|12|14|4|2082|733|9546|Monday|2082Q4|N|N|N|2481830|2482163|2481478|2481751|N|N|N|N|N| +2481844|AAAAAAAAELONFCAA|2082-12-15|2195|9547|733|2082|2|12|15|4|2082|733|9547|Tuesday|2082Q4|N|N|N|2481830|2482163|2481479|2481752|N|N|N|N|N| +2481845|AAAAAAAAFLONFCAA|2082-12-16|2195|9547|733|2082|3|12|16|4|2082|733|9547|Wednesday|2082Q4|N|N|N|2481830|2482163|2481480|2481753|N|N|N|N|N| +2481846|AAAAAAAAGLONFCAA|2082-12-17|2195|9547|733|2082|4|12|17|4|2082|733|9547|Thursday|2082Q4|N|N|N|2481830|2482163|2481481|2481754|N|N|N|N|N| +2481847|AAAAAAAAHLONFCAA|2082-12-18|2195|9547|733|2082|5|12|18|4|2082|733|9547|Friday|2082Q4|N|Y|N|2481830|2482163|2481482|2481755|N|N|N|N|N| +2481848|AAAAAAAAILONFCAA|2082-12-19|2195|9547|733|2082|6|12|19|4|2082|733|9547|Saturday|2082Q4|N|Y|N|2481830|2482163|2481483|2481756|N|N|N|N|N| +2481849|AAAAAAAAJLONFCAA|2082-12-20|2195|9547|733|2082|0|12|20|4|2082|733|9547|Sunday|2082Q4|N|N|N|2481830|2482163|2481484|2481757|N|N|N|N|N| +2481850|AAAAAAAAKLONFCAA|2082-12-21|2195|9547|733|2082|1|12|21|4|2082|733|9547|Monday|2082Q4|N|N|N|2481830|2482163|2481485|2481758|N|N|N|N|N| +2481851|AAAAAAAALLONFCAA|2082-12-22|2195|9548|733|2082|2|12|22|4|2082|733|9548|Tuesday|2082Q4|N|N|N|2481830|2482163|2481486|2481759|N|N|N|N|N| +2481852|AAAAAAAAMLONFCAA|2082-12-23|2195|9548|733|2082|3|12|23|4|2082|733|9548|Wednesday|2082Q4|N|N|N|2481830|2482163|2481487|2481760|N|N|N|N|N| +2481853|AAAAAAAANLONFCAA|2082-12-24|2195|9548|733|2082|4|12|24|4|2082|733|9548|Thursday|2082Q4|N|N|N|2481830|2482163|2481488|2481761|N|N|N|N|N| +2481854|AAAAAAAAOLONFCAA|2082-12-25|2195|9548|733|2082|5|12|25|4|2082|733|9548|Friday|2082Q4|N|Y|N|2481830|2482163|2481489|2481762|N|N|N|N|N| +2481855|AAAAAAAAPLONFCAA|2082-12-26|2195|9548|733|2082|6|12|26|4|2082|733|9548|Saturday|2082Q4|Y|Y|N|2481830|2482163|2481490|2481763|N|N|N|N|N| +2481856|AAAAAAAAAMONFCAA|2082-12-27|2195|9548|733|2082|0|12|27|4|2082|733|9548|Sunday|2082Q4|N|N|Y|2481830|2482163|2481491|2481764|N|N|N|N|N| +2481857|AAAAAAAABMONFCAA|2082-12-28|2195|9548|733|2082|1|12|28|4|2082|733|9548|Monday|2082Q4|N|N|N|2481830|2482163|2481492|2481765|N|N|N|N|N| +2481858|AAAAAAAACMONFCAA|2082-12-29|2195|9549|733|2082|2|12|29|4|2082|733|9549|Tuesday|2082Q4|N|N|N|2481830|2482163|2481493|2481766|N|N|N|N|N| +2481859|AAAAAAAADMONFCAA|2082-12-30|2195|9549|733|2082|3|12|30|4|2082|733|9549|Wednesday|2082Q4|N|N|N|2481830|2482163|2481494|2481767|N|N|N|N|N| +2481860|AAAAAAAAEMONFCAA|2082-12-31|2195|9549|733|2082|4|12|31|4|2082|733|9549|Thursday|2082Q4|N|N|N|2481830|2482163|2481495|2481768|N|N|N|N|N| +2481861|AAAAAAAAFMONFCAA|2083-01-01|2196|9549|733|2083|5|1|1|1|2083|733|9549|Friday|2083Q1|Y|Y|N|2481861|2481860|2481496|2481769|N|N|N|N|N| +2481862|AAAAAAAAGMONFCAA|2083-01-02|2196|9549|733|2083|6|1|2|1|2083|733|9549|Saturday|2083Q1|N|Y|Y|2481861|2481860|2481497|2481770|N|N|N|N|N| +2481863|AAAAAAAAHMONFCAA|2083-01-03|2196|9549|733|2083|0|1|3|1|2083|733|9549|Sunday|2083Q1|N|N|N|2481861|2481860|2481498|2481771|N|N|N|N|N| +2481864|AAAAAAAAIMONFCAA|2083-01-04|2196|9549|733|2083|1|1|4|1|2083|733|9549|Monday|2083Q1|N|N|N|2481861|2481860|2481499|2481772|N|N|N|N|N| +2481865|AAAAAAAAJMONFCAA|2083-01-05|2196|9550|733|2083|2|1|5|1|2083|733|9550|Tuesday|2083Q1|N|N|N|2481861|2481860|2481500|2481773|N|N|N|N|N| +2481866|AAAAAAAAKMONFCAA|2083-01-06|2196|9550|733|2083|3|1|6|1|2083|733|9550|Wednesday|2083Q1|N|N|N|2481861|2481860|2481501|2481774|N|N|N|N|N| +2481867|AAAAAAAALMONFCAA|2083-01-07|2196|9550|733|2083|4|1|7|1|2083|733|9550|Thursday|2083Q1|N|N|N|2481861|2481860|2481502|2481775|N|N|N|N|N| +2481868|AAAAAAAAMMONFCAA|2083-01-08|2196|9550|733|2083|5|1|8|1|2083|733|9550|Friday|2083Q1|N|Y|N|2481861|2481860|2481503|2481776|N|N|N|N|N| +2481869|AAAAAAAANMONFCAA|2083-01-09|2196|9550|733|2083|6|1|9|1|2083|733|9550|Saturday|2083Q1|N|Y|N|2481861|2481860|2481504|2481777|N|N|N|N|N| +2481870|AAAAAAAAOMONFCAA|2083-01-10|2196|9550|733|2083|0|1|10|1|2083|733|9550|Sunday|2083Q1|N|N|N|2481861|2481860|2481505|2481778|N|N|N|N|N| +2481871|AAAAAAAAPMONFCAA|2083-01-11|2196|9550|733|2083|1|1|11|1|2083|733|9550|Monday|2083Q1|N|N|N|2481861|2481860|2481506|2481779|N|N|N|N|N| +2481872|AAAAAAAAANONFCAA|2083-01-12|2196|9551|733|2083|2|1|12|1|2083|733|9551|Tuesday|2083Q1|N|N|N|2481861|2481860|2481507|2481780|N|N|N|N|N| +2481873|AAAAAAAABNONFCAA|2083-01-13|2196|9551|733|2083|3|1|13|1|2083|733|9551|Wednesday|2083Q1|N|N|N|2481861|2481860|2481508|2481781|N|N|N|N|N| +2481874|AAAAAAAACNONFCAA|2083-01-14|2196|9551|733|2083|4|1|14|1|2083|733|9551|Thursday|2083Q1|N|N|N|2481861|2481860|2481509|2481782|N|N|N|N|N| +2481875|AAAAAAAADNONFCAA|2083-01-15|2196|9551|733|2083|5|1|15|1|2083|733|9551|Friday|2083Q1|N|Y|N|2481861|2481860|2481510|2481783|N|N|N|N|N| +2481876|AAAAAAAAENONFCAA|2083-01-16|2196|9551|733|2083|6|1|16|1|2083|733|9551|Saturday|2083Q1|N|Y|N|2481861|2481860|2481511|2481784|N|N|N|N|N| +2481877|AAAAAAAAFNONFCAA|2083-01-17|2196|9551|733|2083|0|1|17|1|2083|733|9551|Sunday|2083Q1|N|N|N|2481861|2481860|2481512|2481785|N|N|N|N|N| +2481878|AAAAAAAAGNONFCAA|2083-01-18|2196|9551|733|2083|1|1|18|1|2083|733|9551|Monday|2083Q1|N|N|N|2481861|2481860|2481513|2481786|N|N|N|N|N| +2481879|AAAAAAAAHNONFCAA|2083-01-19|2196|9552|733|2083|2|1|19|1|2083|733|9552|Tuesday|2083Q1|N|N|N|2481861|2481860|2481514|2481787|N|N|N|N|N| +2481880|AAAAAAAAINONFCAA|2083-01-20|2196|9552|733|2083|3|1|20|1|2083|733|9552|Wednesday|2083Q1|N|N|N|2481861|2481860|2481515|2481788|N|N|N|N|N| +2481881|AAAAAAAAJNONFCAA|2083-01-21|2196|9552|733|2083|4|1|21|1|2083|733|9552|Thursday|2083Q1|N|N|N|2481861|2481860|2481516|2481789|N|N|N|N|N| +2481882|AAAAAAAAKNONFCAA|2083-01-22|2196|9552|733|2083|5|1|22|1|2083|733|9552|Friday|2083Q1|N|Y|N|2481861|2481860|2481517|2481790|N|N|N|N|N| +2481883|AAAAAAAALNONFCAA|2083-01-23|2196|9552|733|2083|6|1|23|1|2083|733|9552|Saturday|2083Q1|N|Y|N|2481861|2481860|2481518|2481791|N|N|N|N|N| +2481884|AAAAAAAAMNONFCAA|2083-01-24|2196|9552|733|2083|0|1|24|1|2083|733|9552|Sunday|2083Q1|N|N|N|2481861|2481860|2481519|2481792|N|N|N|N|N| +2481885|AAAAAAAANNONFCAA|2083-01-25|2196|9552|733|2083|1|1|25|1|2083|733|9552|Monday|2083Q1|N|N|N|2481861|2481860|2481520|2481793|N|N|N|N|N| +2481886|AAAAAAAAONONFCAA|2083-01-26|2196|9553|733|2083|2|1|26|1|2083|733|9553|Tuesday|2083Q1|N|N|N|2481861|2481860|2481521|2481794|N|N|N|N|N| +2481887|AAAAAAAAPNONFCAA|2083-01-27|2196|9553|733|2083|3|1|27|1|2083|733|9553|Wednesday|2083Q1|N|N|N|2481861|2481860|2481522|2481795|N|N|N|N|N| +2481888|AAAAAAAAAOONFCAA|2083-01-28|2196|9553|733|2083|4|1|28|1|2083|733|9553|Thursday|2083Q1|N|N|N|2481861|2481860|2481523|2481796|N|N|N|N|N| +2481889|AAAAAAAABOONFCAA|2083-01-29|2196|9553|733|2083|5|1|29|1|2083|733|9553|Friday|2083Q1|N|Y|N|2481861|2481860|2481524|2481797|N|N|N|N|N| +2481890|AAAAAAAACOONFCAA|2083-01-30|2196|9553|733|2083|6|1|30|1|2083|733|9553|Saturday|2083Q1|N|Y|N|2481861|2481860|2481525|2481798|N|N|N|N|N| +2481891|AAAAAAAADOONFCAA|2083-01-31|2196|9553|733|2083|0|1|31|1|2083|733|9553|Sunday|2083Q1|N|N|N|2481861|2481860|2481526|2481799|N|N|N|N|N| +2481892|AAAAAAAAEOONFCAA|2083-02-01|2197|9553|733|2083|1|2|1|1|2083|733|9553|Monday|2083Q1|N|N|N|2481892|2481922|2481527|2481800|N|N|N|N|N| +2481893|AAAAAAAAFOONFCAA|2083-02-02|2197|9554|733|2083|2|2|2|1|2083|733|9554|Tuesday|2083Q1|N|N|N|2481892|2481922|2481528|2481801|N|N|N|N|N| +2481894|AAAAAAAAGOONFCAA|2083-02-03|2197|9554|733|2083|3|2|3|1|2083|733|9554|Wednesday|2083Q1|N|N|N|2481892|2481922|2481529|2481802|N|N|N|N|N| +2481895|AAAAAAAAHOONFCAA|2083-02-04|2197|9554|733|2083|4|2|4|1|2083|733|9554|Thursday|2083Q1|N|N|N|2481892|2481922|2481530|2481803|N|N|N|N|N| +2481896|AAAAAAAAIOONFCAA|2083-02-05|2197|9554|733|2083|5|2|5|1|2083|733|9554|Friday|2083Q1|N|Y|N|2481892|2481922|2481531|2481804|N|N|N|N|N| +2481897|AAAAAAAAJOONFCAA|2083-02-06|2197|9554|733|2083|6|2|6|1|2083|733|9554|Saturday|2083Q1|N|Y|N|2481892|2481922|2481532|2481805|N|N|N|N|N| +2481898|AAAAAAAAKOONFCAA|2083-02-07|2197|9554|733|2083|0|2|7|1|2083|733|9554|Sunday|2083Q1|N|N|N|2481892|2481922|2481533|2481806|N|N|N|N|N| +2481899|AAAAAAAALOONFCAA|2083-02-08|2197|9554|733|2083|1|2|8|1|2083|733|9554|Monday|2083Q1|N|N|N|2481892|2481922|2481534|2481807|N|N|N|N|N| +2481900|AAAAAAAAMOONFCAA|2083-02-09|2197|9555|733|2083|2|2|9|1|2083|733|9555|Tuesday|2083Q1|N|N|N|2481892|2481922|2481535|2481808|N|N|N|N|N| +2481901|AAAAAAAANOONFCAA|2083-02-10|2197|9555|733|2083|3|2|10|1|2083|733|9555|Wednesday|2083Q1|N|N|N|2481892|2481922|2481536|2481809|N|N|N|N|N| +2481902|AAAAAAAAOOONFCAA|2083-02-11|2197|9555|733|2083|4|2|11|1|2083|733|9555|Thursday|2083Q1|N|N|N|2481892|2481922|2481537|2481810|N|N|N|N|N| +2481903|AAAAAAAAPOONFCAA|2083-02-12|2197|9555|733|2083|5|2|12|1|2083|733|9555|Friday|2083Q1|N|Y|N|2481892|2481922|2481538|2481811|N|N|N|N|N| +2481904|AAAAAAAAAPONFCAA|2083-02-13|2197|9555|733|2083|6|2|13|1|2083|733|9555|Saturday|2083Q1|N|Y|N|2481892|2481922|2481539|2481812|N|N|N|N|N| +2481905|AAAAAAAABPONFCAA|2083-02-14|2197|9555|733|2083|0|2|14|1|2083|733|9555|Sunday|2083Q1|N|N|N|2481892|2481922|2481540|2481813|N|N|N|N|N| +2481906|AAAAAAAACPONFCAA|2083-02-15|2197|9555|733|2083|1|2|15|1|2083|733|9555|Monday|2083Q1|N|N|N|2481892|2481922|2481541|2481814|N|N|N|N|N| +2481907|AAAAAAAADPONFCAA|2083-02-16|2197|9556|733|2083|2|2|16|1|2083|733|9556|Tuesday|2083Q1|N|N|N|2481892|2481922|2481542|2481815|N|N|N|N|N| +2481908|AAAAAAAAEPONFCAA|2083-02-17|2197|9556|733|2083|3|2|17|1|2083|733|9556|Wednesday|2083Q1|N|N|N|2481892|2481922|2481543|2481816|N|N|N|N|N| +2481909|AAAAAAAAFPONFCAA|2083-02-18|2197|9556|733|2083|4|2|18|1|2083|733|9556|Thursday|2083Q1|N|N|N|2481892|2481922|2481544|2481817|N|N|N|N|N| +2481910|AAAAAAAAGPONFCAA|2083-02-19|2197|9556|733|2083|5|2|19|1|2083|733|9556|Friday|2083Q1|N|Y|N|2481892|2481922|2481545|2481818|N|N|N|N|N| +2481911|AAAAAAAAHPONFCAA|2083-02-20|2197|9556|733|2083|6|2|20|1|2083|733|9556|Saturday|2083Q1|N|Y|N|2481892|2481922|2481546|2481819|N|N|N|N|N| +2481912|AAAAAAAAIPONFCAA|2083-02-21|2197|9556|733|2083|0|2|21|1|2083|733|9556|Sunday|2083Q1|N|N|N|2481892|2481922|2481547|2481820|N|N|N|N|N| +2481913|AAAAAAAAJPONFCAA|2083-02-22|2197|9556|733|2083|1|2|22|1|2083|733|9556|Monday|2083Q1|N|N|N|2481892|2481922|2481548|2481821|N|N|N|N|N| +2481914|AAAAAAAAKPONFCAA|2083-02-23|2197|9557|733|2083|2|2|23|1|2083|733|9557|Tuesday|2083Q1|N|N|N|2481892|2481922|2481549|2481822|N|N|N|N|N| +2481915|AAAAAAAALPONFCAA|2083-02-24|2197|9557|733|2083|3|2|24|1|2083|733|9557|Wednesday|2083Q1|N|N|N|2481892|2481922|2481550|2481823|N|N|N|N|N| +2481916|AAAAAAAAMPONFCAA|2083-02-25|2197|9557|733|2083|4|2|25|1|2083|733|9557|Thursday|2083Q1|N|N|N|2481892|2481922|2481551|2481824|N|N|N|N|N| +2481917|AAAAAAAANPONFCAA|2083-02-26|2197|9557|733|2083|5|2|26|1|2083|733|9557|Friday|2083Q1|N|Y|N|2481892|2481922|2481552|2481825|N|N|N|N|N| +2481918|AAAAAAAAOPONFCAA|2083-02-27|2197|9557|733|2083|6|2|27|1|2083|733|9557|Saturday|2083Q1|N|Y|N|2481892|2481922|2481553|2481826|N|N|N|N|N| +2481919|AAAAAAAAPPONFCAA|2083-02-28|2197|9557|733|2083|0|2|28|1|2083|733|9557|Sunday|2083Q1|N|N|N|2481892|2481922|2481554|2481827|N|N|N|N|N| +2481920|AAAAAAAAAAPNFCAA|2083-03-01|2198|9557|734|2083|1|3|1|1|2083|734|9557|Monday|2083Q1|N|N|N|2481920|2481978|2481555|2481828|N|N|N|N|N| +2481921|AAAAAAAABAPNFCAA|2083-03-02|2198|9558|734|2083|2|3|2|1|2083|734|9558|Tuesday|2083Q1|N|N|N|2481920|2481978|2481556|2481829|N|N|N|N|N| +2481922|AAAAAAAACAPNFCAA|2083-03-03|2198|9558|734|2083|3|3|3|1|2083|734|9558|Wednesday|2083Q1|N|N|N|2481920|2481978|2481557|2481830|N|N|N|N|N| +2481923|AAAAAAAADAPNFCAA|2083-03-04|2198|9558|734|2083|4|3|4|1|2083|734|9558|Thursday|2083Q1|N|N|N|2481920|2481978|2481558|2481831|N|N|N|N|N| +2481924|AAAAAAAAEAPNFCAA|2083-03-05|2198|9558|734|2083|5|3|5|1|2083|734|9558|Friday|2083Q1|N|Y|N|2481920|2481978|2481559|2481832|N|N|N|N|N| +2481925|AAAAAAAAFAPNFCAA|2083-03-06|2198|9558|734|2083|6|3|6|1|2083|734|9558|Saturday|2083Q1|N|Y|N|2481920|2481978|2481560|2481833|N|N|N|N|N| +2481926|AAAAAAAAGAPNFCAA|2083-03-07|2198|9558|734|2083|0|3|7|1|2083|734|9558|Sunday|2083Q1|N|N|N|2481920|2481978|2481561|2481834|N|N|N|N|N| +2481927|AAAAAAAAHAPNFCAA|2083-03-08|2198|9558|734|2083|1|3|8|1|2083|734|9558|Monday|2083Q1|N|N|N|2481920|2481978|2481562|2481835|N|N|N|N|N| +2481928|AAAAAAAAIAPNFCAA|2083-03-09|2198|9559|734|2083|2|3|9|1|2083|734|9559|Tuesday|2083Q1|N|N|N|2481920|2481978|2481563|2481836|N|N|N|N|N| +2481929|AAAAAAAAJAPNFCAA|2083-03-10|2198|9559|734|2083|3|3|10|1|2083|734|9559|Wednesday|2083Q1|N|N|N|2481920|2481978|2481564|2481837|N|N|N|N|N| +2481930|AAAAAAAAKAPNFCAA|2083-03-11|2198|9559|734|2083|4|3|11|1|2083|734|9559|Thursday|2083Q1|N|N|N|2481920|2481978|2481565|2481838|N|N|N|N|N| +2481931|AAAAAAAALAPNFCAA|2083-03-12|2198|9559|734|2083|5|3|12|1|2083|734|9559|Friday|2083Q1|N|Y|N|2481920|2481978|2481566|2481839|N|N|N|N|N| +2481932|AAAAAAAAMAPNFCAA|2083-03-13|2198|9559|734|2083|6|3|13|1|2083|734|9559|Saturday|2083Q1|N|Y|N|2481920|2481978|2481567|2481840|N|N|N|N|N| +2481933|AAAAAAAANAPNFCAA|2083-03-14|2198|9559|734|2083|0|3|14|1|2083|734|9559|Sunday|2083Q1|N|N|N|2481920|2481978|2481568|2481841|N|N|N|N|N| +2481934|AAAAAAAAOAPNFCAA|2083-03-15|2198|9559|734|2083|1|3|15|1|2083|734|9559|Monday|2083Q1|N|N|N|2481920|2481978|2481569|2481842|N|N|N|N|N| +2481935|AAAAAAAAPAPNFCAA|2083-03-16|2198|9560|734|2083|2|3|16|1|2083|734|9560|Tuesday|2083Q1|N|N|N|2481920|2481978|2481570|2481843|N|N|N|N|N| +2481936|AAAAAAAAABPNFCAA|2083-03-17|2198|9560|734|2083|3|3|17|1|2083|734|9560|Wednesday|2083Q1|N|N|N|2481920|2481978|2481571|2481844|N|N|N|N|N| +2481937|AAAAAAAABBPNFCAA|2083-03-18|2198|9560|734|2083|4|3|18|1|2083|734|9560|Thursday|2083Q1|N|N|N|2481920|2481978|2481572|2481845|N|N|N|N|N| +2481938|AAAAAAAACBPNFCAA|2083-03-19|2198|9560|734|2083|5|3|19|1|2083|734|9560|Friday|2083Q1|N|Y|N|2481920|2481978|2481573|2481846|N|N|N|N|N| +2481939|AAAAAAAADBPNFCAA|2083-03-20|2198|9560|734|2083|6|3|20|1|2083|734|9560|Saturday|2083Q1|N|Y|N|2481920|2481978|2481574|2481847|N|N|N|N|N| +2481940|AAAAAAAAEBPNFCAA|2083-03-21|2198|9560|734|2083|0|3|21|1|2083|734|9560|Sunday|2083Q1|N|N|N|2481920|2481978|2481575|2481848|N|N|N|N|N| +2481941|AAAAAAAAFBPNFCAA|2083-03-22|2198|9560|734|2083|1|3|22|1|2083|734|9560|Monday|2083Q1|N|N|N|2481920|2481978|2481576|2481849|N|N|N|N|N| +2481942|AAAAAAAAGBPNFCAA|2083-03-23|2198|9561|734|2083|2|3|23|1|2083|734|9561|Tuesday|2083Q1|N|N|N|2481920|2481978|2481577|2481850|N|N|N|N|N| +2481943|AAAAAAAAHBPNFCAA|2083-03-24|2198|9561|734|2083|3|3|24|1|2083|734|9561|Wednesday|2083Q1|N|N|N|2481920|2481978|2481578|2481851|N|N|N|N|N| +2481944|AAAAAAAAIBPNFCAA|2083-03-25|2198|9561|734|2083|4|3|25|1|2083|734|9561|Thursday|2083Q1|N|N|N|2481920|2481978|2481579|2481852|N|N|N|N|N| +2481945|AAAAAAAAJBPNFCAA|2083-03-26|2198|9561|734|2083|5|3|26|1|2083|734|9561|Friday|2083Q1|N|Y|N|2481920|2481978|2481580|2481853|N|N|N|N|N| +2481946|AAAAAAAAKBPNFCAA|2083-03-27|2198|9561|734|2083|6|3|27|1|2083|734|9561|Saturday|2083Q1|N|Y|N|2481920|2481978|2481581|2481854|N|N|N|N|N| +2481947|AAAAAAAALBPNFCAA|2083-03-28|2198|9561|734|2083|0|3|28|1|2083|734|9561|Sunday|2083Q1|N|N|N|2481920|2481978|2481582|2481855|N|N|N|N|N| +2481948|AAAAAAAAMBPNFCAA|2083-03-29|2198|9561|734|2083|1|3|29|1|2083|734|9561|Monday|2083Q1|N|N|N|2481920|2481978|2481583|2481856|N|N|N|N|N| +2481949|AAAAAAAANBPNFCAA|2083-03-30|2198|9562|734|2083|2|3|30|1|2083|734|9562|Tuesday|2083Q1|N|N|N|2481920|2481978|2481584|2481857|N|N|N|N|N| +2481950|AAAAAAAAOBPNFCAA|2083-03-31|2198|9562|734|2083|3|3|31|1|2083|734|9562|Wednesday|2083Q1|N|N|N|2481920|2481978|2481585|2481858|N|N|N|N|N| +2481951|AAAAAAAAPBPNFCAA|2083-04-01|2199|9562|734|2083|4|4|1|1|2083|734|9562|Thursday|2083Q1|N|N|N|2481951|2482040|2481586|2481861|N|N|N|N|N| +2481952|AAAAAAAAACPNFCAA|2083-04-02|2199|9562|734|2083|5|4|2|2|2083|734|9562|Friday|2083Q2|N|Y|N|2481951|2482040|2481587|2481862|N|N|N|N|N| +2481953|AAAAAAAABCPNFCAA|2083-04-03|2199|9562|734|2083|6|4|3|2|2083|734|9562|Saturday|2083Q2|N|Y|N|2481951|2482040|2481588|2481863|N|N|N|N|N| +2481954|AAAAAAAACCPNFCAA|2083-04-04|2199|9562|734|2083|0|4|4|2|2083|734|9562|Sunday|2083Q2|N|N|N|2481951|2482040|2481589|2481864|N|N|N|N|N| +2481955|AAAAAAAADCPNFCAA|2083-04-05|2199|9562|734|2083|1|4|5|2|2083|734|9562|Monday|2083Q2|N|N|N|2481951|2482040|2481590|2481865|N|N|N|N|N| +2481956|AAAAAAAAECPNFCAA|2083-04-06|2199|9563|734|2083|2|4|6|2|2083|734|9563|Tuesday|2083Q2|N|N|N|2481951|2482040|2481591|2481866|N|N|N|N|N| +2481957|AAAAAAAAFCPNFCAA|2083-04-07|2199|9563|734|2083|3|4|7|2|2083|734|9563|Wednesday|2083Q2|N|N|N|2481951|2482040|2481592|2481867|N|N|N|N|N| +2481958|AAAAAAAAGCPNFCAA|2083-04-08|2199|9563|734|2083|4|4|8|2|2083|734|9563|Thursday|2083Q2|N|N|N|2481951|2482040|2481593|2481868|N|N|N|N|N| +2481959|AAAAAAAAHCPNFCAA|2083-04-09|2199|9563|734|2083|5|4|9|2|2083|734|9563|Friday|2083Q2|N|Y|N|2481951|2482040|2481594|2481869|N|N|N|N|N| +2481960|AAAAAAAAICPNFCAA|2083-04-10|2199|9563|734|2083|6|4|10|2|2083|734|9563|Saturday|2083Q2|N|Y|N|2481951|2482040|2481595|2481870|N|N|N|N|N| +2481961|AAAAAAAAJCPNFCAA|2083-04-11|2199|9563|734|2083|0|4|11|2|2083|734|9563|Sunday|2083Q2|N|N|N|2481951|2482040|2481596|2481871|N|N|N|N|N| +2481962|AAAAAAAAKCPNFCAA|2083-04-12|2199|9563|734|2083|1|4|12|2|2083|734|9563|Monday|2083Q2|N|N|N|2481951|2482040|2481597|2481872|N|N|N|N|N| +2481963|AAAAAAAALCPNFCAA|2083-04-13|2199|9564|734|2083|2|4|13|2|2083|734|9564|Tuesday|2083Q2|N|N|N|2481951|2482040|2481598|2481873|N|N|N|N|N| +2481964|AAAAAAAAMCPNFCAA|2083-04-14|2199|9564|734|2083|3|4|14|2|2083|734|9564|Wednesday|2083Q2|N|N|N|2481951|2482040|2481599|2481874|N|N|N|N|N| +2481965|AAAAAAAANCPNFCAA|2083-04-15|2199|9564|734|2083|4|4|15|2|2083|734|9564|Thursday|2083Q2|N|N|N|2481951|2482040|2481600|2481875|N|N|N|N|N| +2481966|AAAAAAAAOCPNFCAA|2083-04-16|2199|9564|734|2083|5|4|16|2|2083|734|9564|Friday|2083Q2|N|Y|N|2481951|2482040|2481601|2481876|N|N|N|N|N| +2481967|AAAAAAAAPCPNFCAA|2083-04-17|2199|9564|734|2083|6|4|17|2|2083|734|9564|Saturday|2083Q2|N|Y|N|2481951|2482040|2481602|2481877|N|N|N|N|N| +2481968|AAAAAAAAADPNFCAA|2083-04-18|2199|9564|734|2083|0|4|18|2|2083|734|9564|Sunday|2083Q2|N|N|N|2481951|2482040|2481603|2481878|N|N|N|N|N| +2481969|AAAAAAAABDPNFCAA|2083-04-19|2199|9564|734|2083|1|4|19|2|2083|734|9564|Monday|2083Q2|N|N|N|2481951|2482040|2481604|2481879|N|N|N|N|N| +2481970|AAAAAAAACDPNFCAA|2083-04-20|2199|9565|734|2083|2|4|20|2|2083|734|9565|Tuesday|2083Q2|N|N|N|2481951|2482040|2481605|2481880|N|N|N|N|N| +2481971|AAAAAAAADDPNFCAA|2083-04-21|2199|9565|734|2083|3|4|21|2|2083|734|9565|Wednesday|2083Q2|N|N|N|2481951|2482040|2481606|2481881|N|N|N|N|N| +2481972|AAAAAAAAEDPNFCAA|2083-04-22|2199|9565|734|2083|4|4|22|2|2083|734|9565|Thursday|2083Q2|N|N|N|2481951|2482040|2481607|2481882|N|N|N|N|N| +2481973|AAAAAAAAFDPNFCAA|2083-04-23|2199|9565|734|2083|5|4|23|2|2083|734|9565|Friday|2083Q2|N|Y|N|2481951|2482040|2481608|2481883|N|N|N|N|N| +2481974|AAAAAAAAGDPNFCAA|2083-04-24|2199|9565|734|2083|6|4|24|2|2083|734|9565|Saturday|2083Q2|N|Y|N|2481951|2482040|2481609|2481884|N|N|N|N|N| +2481975|AAAAAAAAHDPNFCAA|2083-04-25|2199|9565|734|2083|0|4|25|2|2083|734|9565|Sunday|2083Q2|N|N|N|2481951|2482040|2481610|2481885|N|N|N|N|N| +2481976|AAAAAAAAIDPNFCAA|2083-04-26|2199|9565|734|2083|1|4|26|2|2083|734|9565|Monday|2083Q2|N|N|N|2481951|2482040|2481611|2481886|N|N|N|N|N| +2481977|AAAAAAAAJDPNFCAA|2083-04-27|2199|9566|734|2083|2|4|27|2|2083|734|9566|Tuesday|2083Q2|N|N|N|2481951|2482040|2481612|2481887|N|N|N|N|N| +2481978|AAAAAAAAKDPNFCAA|2083-04-28|2199|9566|734|2083|3|4|28|2|2083|734|9566|Wednesday|2083Q2|N|N|N|2481951|2482040|2481613|2481888|N|N|N|N|N| +2481979|AAAAAAAALDPNFCAA|2083-04-29|2199|9566|734|2083|4|4|29|2|2083|734|9566|Thursday|2083Q2|N|N|N|2481951|2482040|2481614|2481889|N|N|N|N|N| +2481980|AAAAAAAAMDPNFCAA|2083-04-30|2199|9566|734|2083|5|4|30|2|2083|734|9566|Friday|2083Q2|N|Y|N|2481951|2482040|2481615|2481890|N|N|N|N|N| +2481981|AAAAAAAANDPNFCAA|2083-05-01|2200|9566|734|2083|6|5|1|2|2083|734|9566|Saturday|2083Q2|N|Y|N|2481981|2482100|2481616|2481891|N|N|N|N|N| +2481982|AAAAAAAAODPNFCAA|2083-05-02|2200|9566|734|2083|0|5|2|2|2083|734|9566|Sunday|2083Q2|N|N|N|2481981|2482100|2481617|2481892|N|N|N|N|N| +2481983|AAAAAAAAPDPNFCAA|2083-05-03|2200|9566|734|2083|1|5|3|2|2083|734|9566|Monday|2083Q2|N|N|N|2481981|2482100|2481618|2481893|N|N|N|N|N| +2481984|AAAAAAAAAEPNFCAA|2083-05-04|2200|9567|734|2083|2|5|4|2|2083|734|9567|Tuesday|2083Q2|N|N|N|2481981|2482100|2481619|2481894|N|N|N|N|N| +2481985|AAAAAAAABEPNFCAA|2083-05-05|2200|9567|734|2083|3|5|5|2|2083|734|9567|Wednesday|2083Q2|N|N|N|2481981|2482100|2481620|2481895|N|N|N|N|N| +2481986|AAAAAAAACEPNFCAA|2083-05-06|2200|9567|734|2083|4|5|6|2|2083|734|9567|Thursday|2083Q2|N|N|N|2481981|2482100|2481621|2481896|N|N|N|N|N| +2481987|AAAAAAAADEPNFCAA|2083-05-07|2200|9567|734|2083|5|5|7|2|2083|734|9567|Friday|2083Q2|N|Y|N|2481981|2482100|2481622|2481897|N|N|N|N|N| +2481988|AAAAAAAAEEPNFCAA|2083-05-08|2200|9567|734|2083|6|5|8|2|2083|734|9567|Saturday|2083Q2|N|Y|N|2481981|2482100|2481623|2481898|N|N|N|N|N| +2481989|AAAAAAAAFEPNFCAA|2083-05-09|2200|9567|734|2083|0|5|9|2|2083|734|9567|Sunday|2083Q2|N|N|N|2481981|2482100|2481624|2481899|N|N|N|N|N| +2481990|AAAAAAAAGEPNFCAA|2083-05-10|2200|9567|734|2083|1|5|10|2|2083|734|9567|Monday|2083Q2|N|N|N|2481981|2482100|2481625|2481900|N|N|N|N|N| +2481991|AAAAAAAAHEPNFCAA|2083-05-11|2200|9568|734|2083|2|5|11|2|2083|734|9568|Tuesday|2083Q2|N|N|N|2481981|2482100|2481626|2481901|N|N|N|N|N| +2481992|AAAAAAAAIEPNFCAA|2083-05-12|2200|9568|734|2083|3|5|12|2|2083|734|9568|Wednesday|2083Q2|N|N|N|2481981|2482100|2481627|2481902|N|N|N|N|N| +2481993|AAAAAAAAJEPNFCAA|2083-05-13|2200|9568|734|2083|4|5|13|2|2083|734|9568|Thursday|2083Q2|N|N|N|2481981|2482100|2481628|2481903|N|N|N|N|N| +2481994|AAAAAAAAKEPNFCAA|2083-05-14|2200|9568|734|2083|5|5|14|2|2083|734|9568|Friday|2083Q2|N|Y|N|2481981|2482100|2481629|2481904|N|N|N|N|N| +2481995|AAAAAAAALEPNFCAA|2083-05-15|2200|9568|734|2083|6|5|15|2|2083|734|9568|Saturday|2083Q2|N|Y|N|2481981|2482100|2481630|2481905|N|N|N|N|N| +2481996|AAAAAAAAMEPNFCAA|2083-05-16|2200|9568|734|2083|0|5|16|2|2083|734|9568|Sunday|2083Q2|N|N|N|2481981|2482100|2481631|2481906|N|N|N|N|N| +2481997|AAAAAAAANEPNFCAA|2083-05-17|2200|9568|734|2083|1|5|17|2|2083|734|9568|Monday|2083Q2|N|N|N|2481981|2482100|2481632|2481907|N|N|N|N|N| +2481998|AAAAAAAAOEPNFCAA|2083-05-18|2200|9569|734|2083|2|5|18|2|2083|734|9569|Tuesday|2083Q2|N|N|N|2481981|2482100|2481633|2481908|N|N|N|N|N| +2481999|AAAAAAAAPEPNFCAA|2083-05-19|2200|9569|734|2083|3|5|19|2|2083|734|9569|Wednesday|2083Q2|N|N|N|2481981|2482100|2481634|2481909|N|N|N|N|N| +2482000|AAAAAAAAAFPNFCAA|2083-05-20|2200|9569|734|2083|4|5|20|2|2083|734|9569|Thursday|2083Q2|N|N|N|2481981|2482100|2481635|2481910|N|N|N|N|N| +2482001|AAAAAAAABFPNFCAA|2083-05-21|2200|9569|734|2083|5|5|21|2|2083|734|9569|Friday|2083Q2|N|Y|N|2481981|2482100|2481636|2481911|N|N|N|N|N| +2482002|AAAAAAAACFPNFCAA|2083-05-22|2200|9569|734|2083|6|5|22|2|2083|734|9569|Saturday|2083Q2|N|Y|N|2481981|2482100|2481637|2481912|N|N|N|N|N| +2482003|AAAAAAAADFPNFCAA|2083-05-23|2200|9569|734|2083|0|5|23|2|2083|734|9569|Sunday|2083Q2|N|N|N|2481981|2482100|2481638|2481913|N|N|N|N|N| +2482004|AAAAAAAAEFPNFCAA|2083-05-24|2200|9569|734|2083|1|5|24|2|2083|734|9569|Monday|2083Q2|N|N|N|2481981|2482100|2481639|2481914|N|N|N|N|N| +2482005|AAAAAAAAFFPNFCAA|2083-05-25|2200|9570|734|2083|2|5|25|2|2083|734|9570|Tuesday|2083Q2|N|N|N|2481981|2482100|2481640|2481915|N|N|N|N|N| +2482006|AAAAAAAAGFPNFCAA|2083-05-26|2200|9570|734|2083|3|5|26|2|2083|734|9570|Wednesday|2083Q2|N|N|N|2481981|2482100|2481641|2481916|N|N|N|N|N| +2482007|AAAAAAAAHFPNFCAA|2083-05-27|2200|9570|734|2083|4|5|27|2|2083|734|9570|Thursday|2083Q2|N|N|N|2481981|2482100|2481642|2481917|N|N|N|N|N| +2482008|AAAAAAAAIFPNFCAA|2083-05-28|2200|9570|734|2083|5|5|28|2|2083|734|9570|Friday|2083Q2|N|Y|N|2481981|2482100|2481643|2481918|N|N|N|N|N| +2482009|AAAAAAAAJFPNFCAA|2083-05-29|2200|9570|734|2083|6|5|29|2|2083|734|9570|Saturday|2083Q2|N|Y|N|2481981|2482100|2481644|2481919|N|N|N|N|N| +2482010|AAAAAAAAKFPNFCAA|2083-05-30|2200|9570|734|2083|0|5|30|2|2083|734|9570|Sunday|2083Q2|N|N|N|2481981|2482100|2481645|2481920|N|N|N|N|N| +2482011|AAAAAAAALFPNFCAA|2083-05-31|2200|9570|734|2083|1|5|31|2|2083|734|9570|Monday|2083Q2|N|N|N|2481981|2482100|2481646|2481921|N|N|N|N|N| +2482012|AAAAAAAAMFPNFCAA|2083-06-01|2201|9571|735|2083|2|6|1|2|2083|735|9571|Tuesday|2083Q2|N|N|N|2482012|2482162|2481647|2481922|N|N|N|N|N| +2482013|AAAAAAAANFPNFCAA|2083-06-02|2201|9571|735|2083|3|6|2|2|2083|735|9571|Wednesday|2083Q2|N|N|N|2482012|2482162|2481648|2481923|N|N|N|N|N| +2482014|AAAAAAAAOFPNFCAA|2083-06-03|2201|9571|735|2083|4|6|3|2|2083|735|9571|Thursday|2083Q2|N|N|N|2482012|2482162|2481649|2481924|N|N|N|N|N| +2482015|AAAAAAAAPFPNFCAA|2083-06-04|2201|9571|735|2083|5|6|4|2|2083|735|9571|Friday|2083Q2|N|Y|N|2482012|2482162|2481650|2481925|N|N|N|N|N| +2482016|AAAAAAAAAGPNFCAA|2083-06-05|2201|9571|735|2083|6|6|5|2|2083|735|9571|Saturday|2083Q2|N|Y|N|2482012|2482162|2481651|2481926|N|N|N|N|N| +2482017|AAAAAAAABGPNFCAA|2083-06-06|2201|9571|735|2083|0|6|6|2|2083|735|9571|Sunday|2083Q2|N|N|N|2482012|2482162|2481652|2481927|N|N|N|N|N| +2482018|AAAAAAAACGPNFCAA|2083-06-07|2201|9571|735|2083|1|6|7|2|2083|735|9571|Monday|2083Q2|N|N|N|2482012|2482162|2481653|2481928|N|N|N|N|N| +2482019|AAAAAAAADGPNFCAA|2083-06-08|2201|9572|735|2083|2|6|8|2|2083|735|9572|Tuesday|2083Q2|N|N|N|2482012|2482162|2481654|2481929|N|N|N|N|N| +2482020|AAAAAAAAEGPNFCAA|2083-06-09|2201|9572|735|2083|3|6|9|2|2083|735|9572|Wednesday|2083Q2|N|N|N|2482012|2482162|2481655|2481930|N|N|N|N|N| +2482021|AAAAAAAAFGPNFCAA|2083-06-10|2201|9572|735|2083|4|6|10|2|2083|735|9572|Thursday|2083Q2|N|N|N|2482012|2482162|2481656|2481931|N|N|N|N|N| +2482022|AAAAAAAAGGPNFCAA|2083-06-11|2201|9572|735|2083|5|6|11|2|2083|735|9572|Friday|2083Q2|N|Y|N|2482012|2482162|2481657|2481932|N|N|N|N|N| +2482023|AAAAAAAAHGPNFCAA|2083-06-12|2201|9572|735|2083|6|6|12|2|2083|735|9572|Saturday|2083Q2|N|Y|N|2482012|2482162|2481658|2481933|N|N|N|N|N| +2482024|AAAAAAAAIGPNFCAA|2083-06-13|2201|9572|735|2083|0|6|13|2|2083|735|9572|Sunday|2083Q2|N|N|N|2482012|2482162|2481659|2481934|N|N|N|N|N| +2482025|AAAAAAAAJGPNFCAA|2083-06-14|2201|9572|735|2083|1|6|14|2|2083|735|9572|Monday|2083Q2|N|N|N|2482012|2482162|2481660|2481935|N|N|N|N|N| +2482026|AAAAAAAAKGPNFCAA|2083-06-15|2201|9573|735|2083|2|6|15|2|2083|735|9573|Tuesday|2083Q2|N|N|N|2482012|2482162|2481661|2481936|N|N|N|N|N| +2482027|AAAAAAAALGPNFCAA|2083-06-16|2201|9573|735|2083|3|6|16|2|2083|735|9573|Wednesday|2083Q2|N|N|N|2482012|2482162|2481662|2481937|N|N|N|N|N| +2482028|AAAAAAAAMGPNFCAA|2083-06-17|2201|9573|735|2083|4|6|17|2|2083|735|9573|Thursday|2083Q2|N|N|N|2482012|2482162|2481663|2481938|N|N|N|N|N| +2482029|AAAAAAAANGPNFCAA|2083-06-18|2201|9573|735|2083|5|6|18|2|2083|735|9573|Friday|2083Q2|N|Y|N|2482012|2482162|2481664|2481939|N|N|N|N|N| +2482030|AAAAAAAAOGPNFCAA|2083-06-19|2201|9573|735|2083|6|6|19|2|2083|735|9573|Saturday|2083Q2|N|Y|N|2482012|2482162|2481665|2481940|N|N|N|N|N| +2482031|AAAAAAAAPGPNFCAA|2083-06-20|2201|9573|735|2083|0|6|20|2|2083|735|9573|Sunday|2083Q2|N|N|N|2482012|2482162|2481666|2481941|N|N|N|N|N| +2482032|AAAAAAAAAHPNFCAA|2083-06-21|2201|9573|735|2083|1|6|21|2|2083|735|9573|Monday|2083Q2|N|N|N|2482012|2482162|2481667|2481942|N|N|N|N|N| +2482033|AAAAAAAABHPNFCAA|2083-06-22|2201|9574|735|2083|2|6|22|2|2083|735|9574|Tuesday|2083Q2|N|N|N|2482012|2482162|2481668|2481943|N|N|N|N|N| +2482034|AAAAAAAACHPNFCAA|2083-06-23|2201|9574|735|2083|3|6|23|2|2083|735|9574|Wednesday|2083Q2|N|N|N|2482012|2482162|2481669|2481944|N|N|N|N|N| +2482035|AAAAAAAADHPNFCAA|2083-06-24|2201|9574|735|2083|4|6|24|2|2083|735|9574|Thursday|2083Q2|N|N|N|2482012|2482162|2481670|2481945|N|N|N|N|N| +2482036|AAAAAAAAEHPNFCAA|2083-06-25|2201|9574|735|2083|5|6|25|2|2083|735|9574|Friday|2083Q2|N|Y|N|2482012|2482162|2481671|2481946|N|N|N|N|N| +2482037|AAAAAAAAFHPNFCAA|2083-06-26|2201|9574|735|2083|6|6|26|2|2083|735|9574|Saturday|2083Q2|N|Y|N|2482012|2482162|2481672|2481947|N|N|N|N|N| +2482038|AAAAAAAAGHPNFCAA|2083-06-27|2201|9574|735|2083|0|6|27|2|2083|735|9574|Sunday|2083Q2|N|N|N|2482012|2482162|2481673|2481948|N|N|N|N|N| +2482039|AAAAAAAAHHPNFCAA|2083-06-28|2201|9574|735|2083|1|6|28|2|2083|735|9574|Monday|2083Q2|N|N|N|2482012|2482162|2481674|2481949|N|N|N|N|N| +2482040|AAAAAAAAIHPNFCAA|2083-06-29|2201|9575|735|2083|2|6|29|2|2083|735|9575|Tuesday|2083Q2|N|N|N|2482012|2482162|2481675|2481950|N|N|N|N|N| +2482041|AAAAAAAAJHPNFCAA|2083-06-30|2201|9575|735|2083|3|6|30|2|2083|735|9575|Wednesday|2083Q2|N|N|N|2482012|2482162|2481676|2481951|N|N|N|N|N| +2482042|AAAAAAAAKHPNFCAA|2083-07-01|2202|9575|735|2083|4|7|1|2|2083|735|9575|Thursday|2083Q2|N|N|N|2482042|2482222|2481677|2481951|N|N|N|N|N| +2482043|AAAAAAAALHPNFCAA|2083-07-02|2202|9575|735|2083|5|7|2|3|2083|735|9575|Friday|2083Q3|N|Y|N|2482042|2482222|2481678|2481952|N|N|N|N|N| +2482044|AAAAAAAAMHPNFCAA|2083-07-03|2202|9575|735|2083|6|7|3|3|2083|735|9575|Saturday|2083Q3|N|Y|N|2482042|2482222|2481679|2481953|N|N|N|N|N| +2482045|AAAAAAAANHPNFCAA|2083-07-04|2202|9575|735|2083|0|7|4|3|2083|735|9575|Sunday|2083Q3|N|N|N|2482042|2482222|2481680|2481954|N|N|N|N|N| +2482046|AAAAAAAAOHPNFCAA|2083-07-05|2202|9575|735|2083|1|7|5|3|2083|735|9575|Monday|2083Q3|Y|N|N|2482042|2482222|2481681|2481955|N|N|N|N|N| +2482047|AAAAAAAAPHPNFCAA|2083-07-06|2202|9576|735|2083|2|7|6|3|2083|735|9576|Tuesday|2083Q3|N|N|Y|2482042|2482222|2481682|2481956|N|N|N|N|N| +2482048|AAAAAAAAAIPNFCAA|2083-07-07|2202|9576|735|2083|3|7|7|3|2083|735|9576|Wednesday|2083Q3|N|N|N|2482042|2482222|2481683|2481957|N|N|N|N|N| +2482049|AAAAAAAABIPNFCAA|2083-07-08|2202|9576|735|2083|4|7|8|3|2083|735|9576|Thursday|2083Q3|N|N|N|2482042|2482222|2481684|2481958|N|N|N|N|N| +2482050|AAAAAAAACIPNFCAA|2083-07-09|2202|9576|735|2083|5|7|9|3|2083|735|9576|Friday|2083Q3|N|Y|N|2482042|2482222|2481685|2481959|N|N|N|N|N| +2482051|AAAAAAAADIPNFCAA|2083-07-10|2202|9576|735|2083|6|7|10|3|2083|735|9576|Saturday|2083Q3|N|Y|N|2482042|2482222|2481686|2481960|N|N|N|N|N| +2482052|AAAAAAAAEIPNFCAA|2083-07-11|2202|9576|735|2083|0|7|11|3|2083|735|9576|Sunday|2083Q3|N|N|N|2482042|2482222|2481687|2481961|N|N|N|N|N| +2482053|AAAAAAAAFIPNFCAA|2083-07-12|2202|9576|735|2083|1|7|12|3|2083|735|9576|Monday|2083Q3|N|N|N|2482042|2482222|2481688|2481962|N|N|N|N|N| +2482054|AAAAAAAAGIPNFCAA|2083-07-13|2202|9577|735|2083|2|7|13|3|2083|735|9577|Tuesday|2083Q3|N|N|N|2482042|2482222|2481689|2481963|N|N|N|N|N| +2482055|AAAAAAAAHIPNFCAA|2083-07-14|2202|9577|735|2083|3|7|14|3|2083|735|9577|Wednesday|2083Q3|N|N|N|2482042|2482222|2481690|2481964|N|N|N|N|N| +2482056|AAAAAAAAIIPNFCAA|2083-07-15|2202|9577|735|2083|4|7|15|3|2083|735|9577|Thursday|2083Q3|N|N|N|2482042|2482222|2481691|2481965|N|N|N|N|N| +2482057|AAAAAAAAJIPNFCAA|2083-07-16|2202|9577|735|2083|5|7|16|3|2083|735|9577|Friday|2083Q3|N|Y|N|2482042|2482222|2481692|2481966|N|N|N|N|N| +2482058|AAAAAAAAKIPNFCAA|2083-07-17|2202|9577|735|2083|6|7|17|3|2083|735|9577|Saturday|2083Q3|N|Y|N|2482042|2482222|2481693|2481967|N|N|N|N|N| +2482059|AAAAAAAALIPNFCAA|2083-07-18|2202|9577|735|2083|0|7|18|3|2083|735|9577|Sunday|2083Q3|N|N|N|2482042|2482222|2481694|2481968|N|N|N|N|N| +2482060|AAAAAAAAMIPNFCAA|2083-07-19|2202|9577|735|2083|1|7|19|3|2083|735|9577|Monday|2083Q3|N|N|N|2482042|2482222|2481695|2481969|N|N|N|N|N| +2482061|AAAAAAAANIPNFCAA|2083-07-20|2202|9578|735|2083|2|7|20|3|2083|735|9578|Tuesday|2083Q3|N|N|N|2482042|2482222|2481696|2481970|N|N|N|N|N| +2482062|AAAAAAAAOIPNFCAA|2083-07-21|2202|9578|735|2083|3|7|21|3|2083|735|9578|Wednesday|2083Q3|N|N|N|2482042|2482222|2481697|2481971|N|N|N|N|N| +2482063|AAAAAAAAPIPNFCAA|2083-07-22|2202|9578|735|2083|4|7|22|3|2083|735|9578|Thursday|2083Q3|N|N|N|2482042|2482222|2481698|2481972|N|N|N|N|N| +2482064|AAAAAAAAAJPNFCAA|2083-07-23|2202|9578|735|2083|5|7|23|3|2083|735|9578|Friday|2083Q3|N|Y|N|2482042|2482222|2481699|2481973|N|N|N|N|N| +2482065|AAAAAAAABJPNFCAA|2083-07-24|2202|9578|735|2083|6|7|24|3|2083|735|9578|Saturday|2083Q3|N|Y|N|2482042|2482222|2481700|2481974|N|N|N|N|N| +2482066|AAAAAAAACJPNFCAA|2083-07-25|2202|9578|735|2083|0|7|25|3|2083|735|9578|Sunday|2083Q3|N|N|N|2482042|2482222|2481701|2481975|N|N|N|N|N| +2482067|AAAAAAAADJPNFCAA|2083-07-26|2202|9578|735|2083|1|7|26|3|2083|735|9578|Monday|2083Q3|N|N|N|2482042|2482222|2481702|2481976|N|N|N|N|N| +2482068|AAAAAAAAEJPNFCAA|2083-07-27|2202|9579|735|2083|2|7|27|3|2083|735|9579|Tuesday|2083Q3|N|N|N|2482042|2482222|2481703|2481977|N|N|N|N|N| +2482069|AAAAAAAAFJPNFCAA|2083-07-28|2202|9579|735|2083|3|7|28|3|2083|735|9579|Wednesday|2083Q3|N|N|N|2482042|2482222|2481704|2481978|N|N|N|N|N| +2482070|AAAAAAAAGJPNFCAA|2083-07-29|2202|9579|735|2083|4|7|29|3|2083|735|9579|Thursday|2083Q3|N|N|N|2482042|2482222|2481705|2481979|N|N|N|N|N| +2482071|AAAAAAAAHJPNFCAA|2083-07-30|2202|9579|735|2083|5|7|30|3|2083|735|9579|Friday|2083Q3|N|Y|N|2482042|2482222|2481706|2481980|N|N|N|N|N| +2482072|AAAAAAAAIJPNFCAA|2083-07-31|2202|9579|735|2083|6|7|31|3|2083|735|9579|Saturday|2083Q3|N|Y|N|2482042|2482222|2481707|2481981|N|N|N|N|N| +2482073|AAAAAAAAJJPNFCAA|2083-08-01|2203|9579|735|2083|0|8|1|3|2083|735|9579|Sunday|2083Q3|N|N|N|2482073|2482284|2481708|2481982|N|N|N|N|N| +2482074|AAAAAAAAKJPNFCAA|2083-08-02|2203|9579|735|2083|1|8|2|3|2083|735|9579|Monday|2083Q3|N|N|N|2482073|2482284|2481709|2481983|N|N|N|N|N| +2482075|AAAAAAAALJPNFCAA|2083-08-03|2203|9580|735|2083|2|8|3|3|2083|735|9580|Tuesday|2083Q3|N|N|N|2482073|2482284|2481710|2481984|N|N|N|N|N| +2482076|AAAAAAAAMJPNFCAA|2083-08-04|2203|9580|735|2083|3|8|4|3|2083|735|9580|Wednesday|2083Q3|N|N|N|2482073|2482284|2481711|2481985|N|N|N|N|N| +2482077|AAAAAAAANJPNFCAA|2083-08-05|2203|9580|735|2083|4|8|5|3|2083|735|9580|Thursday|2083Q3|N|N|N|2482073|2482284|2481712|2481986|N|N|N|N|N| +2482078|AAAAAAAAOJPNFCAA|2083-08-06|2203|9580|735|2083|5|8|6|3|2083|735|9580|Friday|2083Q3|N|Y|N|2482073|2482284|2481713|2481987|N|N|N|N|N| +2482079|AAAAAAAAPJPNFCAA|2083-08-07|2203|9580|735|2083|6|8|7|3|2083|735|9580|Saturday|2083Q3|N|Y|N|2482073|2482284|2481714|2481988|N|N|N|N|N| +2482080|AAAAAAAAAKPNFCAA|2083-08-08|2203|9580|735|2083|0|8|8|3|2083|735|9580|Sunday|2083Q3|N|N|N|2482073|2482284|2481715|2481989|N|N|N|N|N| +2482081|AAAAAAAABKPNFCAA|2083-08-09|2203|9580|735|2083|1|8|9|3|2083|735|9580|Monday|2083Q3|N|N|N|2482073|2482284|2481716|2481990|N|N|N|N|N| +2482082|AAAAAAAACKPNFCAA|2083-08-10|2203|9581|735|2083|2|8|10|3|2083|735|9581|Tuesday|2083Q3|N|N|N|2482073|2482284|2481717|2481991|N|N|N|N|N| +2482083|AAAAAAAADKPNFCAA|2083-08-11|2203|9581|735|2083|3|8|11|3|2083|735|9581|Wednesday|2083Q3|N|N|N|2482073|2482284|2481718|2481992|N|N|N|N|N| +2482084|AAAAAAAAEKPNFCAA|2083-08-12|2203|9581|735|2083|4|8|12|3|2083|735|9581|Thursday|2083Q3|N|N|N|2482073|2482284|2481719|2481993|N|N|N|N|N| +2482085|AAAAAAAAFKPNFCAA|2083-08-13|2203|9581|735|2083|5|8|13|3|2083|735|9581|Friday|2083Q3|N|Y|N|2482073|2482284|2481720|2481994|N|N|N|N|N| +2482086|AAAAAAAAGKPNFCAA|2083-08-14|2203|9581|735|2083|6|8|14|3|2083|735|9581|Saturday|2083Q3|N|Y|N|2482073|2482284|2481721|2481995|N|N|N|N|N| +2482087|AAAAAAAAHKPNFCAA|2083-08-15|2203|9581|735|2083|0|8|15|3|2083|735|9581|Sunday|2083Q3|N|N|N|2482073|2482284|2481722|2481996|N|N|N|N|N| +2482088|AAAAAAAAIKPNFCAA|2083-08-16|2203|9581|735|2083|1|8|16|3|2083|735|9581|Monday|2083Q3|N|N|N|2482073|2482284|2481723|2481997|N|N|N|N|N| +2482089|AAAAAAAAJKPNFCAA|2083-08-17|2203|9582|735|2083|2|8|17|3|2083|735|9582|Tuesday|2083Q3|N|N|N|2482073|2482284|2481724|2481998|N|N|N|N|N| +2482090|AAAAAAAAKKPNFCAA|2083-08-18|2203|9582|735|2083|3|8|18|3|2083|735|9582|Wednesday|2083Q3|N|N|N|2482073|2482284|2481725|2481999|N|N|N|N|N| +2482091|AAAAAAAALKPNFCAA|2083-08-19|2203|9582|735|2083|4|8|19|3|2083|735|9582|Thursday|2083Q3|N|N|N|2482073|2482284|2481726|2482000|N|N|N|N|N| +2482092|AAAAAAAAMKPNFCAA|2083-08-20|2203|9582|735|2083|5|8|20|3|2083|735|9582|Friday|2083Q3|N|Y|N|2482073|2482284|2481727|2482001|N|N|N|N|N| +2482093|AAAAAAAANKPNFCAA|2083-08-21|2203|9582|735|2083|6|8|21|3|2083|735|9582|Saturday|2083Q3|N|Y|N|2482073|2482284|2481728|2482002|N|N|N|N|N| +2482094|AAAAAAAAOKPNFCAA|2083-08-22|2203|9582|735|2083|0|8|22|3|2083|735|9582|Sunday|2083Q3|N|N|N|2482073|2482284|2481729|2482003|N|N|N|N|N| +2482095|AAAAAAAAPKPNFCAA|2083-08-23|2203|9582|735|2083|1|8|23|3|2083|735|9582|Monday|2083Q3|N|N|N|2482073|2482284|2481730|2482004|N|N|N|N|N| +2482096|AAAAAAAAALPNFCAA|2083-08-24|2203|9583|735|2083|2|8|24|3|2083|735|9583|Tuesday|2083Q3|N|N|N|2482073|2482284|2481731|2482005|N|N|N|N|N| +2482097|AAAAAAAABLPNFCAA|2083-08-25|2203|9583|735|2083|3|8|25|3|2083|735|9583|Wednesday|2083Q3|N|N|N|2482073|2482284|2481732|2482006|N|N|N|N|N| +2482098|AAAAAAAACLPNFCAA|2083-08-26|2203|9583|735|2083|4|8|26|3|2083|735|9583|Thursday|2083Q3|N|N|N|2482073|2482284|2481733|2482007|N|N|N|N|N| +2482099|AAAAAAAADLPNFCAA|2083-08-27|2203|9583|735|2083|5|8|27|3|2083|735|9583|Friday|2083Q3|N|Y|N|2482073|2482284|2481734|2482008|N|N|N|N|N| +2482100|AAAAAAAAELPNFCAA|2083-08-28|2203|9583|735|2083|6|8|28|3|2083|735|9583|Saturday|2083Q3|N|Y|N|2482073|2482284|2481735|2482009|N|N|N|N|N| +2482101|AAAAAAAAFLPNFCAA|2083-08-29|2203|9583|735|2083|0|8|29|3|2083|735|9583|Sunday|2083Q3|N|N|N|2482073|2482284|2481736|2482010|N|N|N|N|N| +2482102|AAAAAAAAGLPNFCAA|2083-08-30|2203|9583|735|2083|1|8|30|3|2083|735|9583|Monday|2083Q3|N|N|N|2482073|2482284|2481737|2482011|N|N|N|N|N| +2482103|AAAAAAAAHLPNFCAA|2083-08-31|2203|9584|735|2083|2|8|31|3|2083|735|9584|Tuesday|2083Q3|N|N|N|2482073|2482284|2481738|2482012|N|N|N|N|N| +2482104|AAAAAAAAILPNFCAA|2083-09-01|2204|9584|736|2083|3|9|1|3|2083|736|9584|Wednesday|2083Q3|N|N|N|2482104|2482346|2481739|2482013|N|N|N|N|N| +2482105|AAAAAAAAJLPNFCAA|2083-09-02|2204|9584|736|2083|4|9|2|3|2083|736|9584|Thursday|2083Q3|N|N|N|2482104|2482346|2481740|2482014|N|N|N|N|N| +2482106|AAAAAAAAKLPNFCAA|2083-09-03|2204|9584|736|2083|5|9|3|3|2083|736|9584|Friday|2083Q3|N|Y|N|2482104|2482346|2481741|2482015|N|N|N|N|N| +2482107|AAAAAAAALLPNFCAA|2083-09-04|2204|9584|736|2083|6|9|4|3|2083|736|9584|Saturday|2083Q3|N|Y|N|2482104|2482346|2481742|2482016|N|N|N|N|N| +2482108|AAAAAAAAMLPNFCAA|2083-09-05|2204|9584|736|2083|0|9|5|3|2083|736|9584|Sunday|2083Q3|N|N|N|2482104|2482346|2481743|2482017|N|N|N|N|N| +2482109|AAAAAAAANLPNFCAA|2083-09-06|2204|9584|736|2083|1|9|6|3|2083|736|9584|Monday|2083Q3|N|N|N|2482104|2482346|2481744|2482018|N|N|N|N|N| +2482110|AAAAAAAAOLPNFCAA|2083-09-07|2204|9585|736|2083|2|9|7|3|2083|736|9585|Tuesday|2083Q3|N|N|N|2482104|2482346|2481745|2482019|N|N|N|N|N| +2482111|AAAAAAAAPLPNFCAA|2083-09-08|2204|9585|736|2083|3|9|8|3|2083|736|9585|Wednesday|2083Q3|N|N|N|2482104|2482346|2481746|2482020|N|N|N|N|N| +2482112|AAAAAAAAAMPNFCAA|2083-09-09|2204|9585|736|2083|4|9|9|3|2083|736|9585|Thursday|2083Q3|N|N|N|2482104|2482346|2481747|2482021|N|N|N|N|N| +2482113|AAAAAAAABMPNFCAA|2083-09-10|2204|9585|736|2083|5|9|10|3|2083|736|9585|Friday|2083Q3|N|Y|N|2482104|2482346|2481748|2482022|N|N|N|N|N| +2482114|AAAAAAAACMPNFCAA|2083-09-11|2204|9585|736|2083|6|9|11|3|2083|736|9585|Saturday|2083Q3|N|Y|N|2482104|2482346|2481749|2482023|N|N|N|N|N| +2482115|AAAAAAAADMPNFCAA|2083-09-12|2204|9585|736|2083|0|9|12|3|2083|736|9585|Sunday|2083Q3|N|N|N|2482104|2482346|2481750|2482024|N|N|N|N|N| +2482116|AAAAAAAAEMPNFCAA|2083-09-13|2204|9585|736|2083|1|9|13|3|2083|736|9585|Monday|2083Q3|N|N|N|2482104|2482346|2481751|2482025|N|N|N|N|N| +2482117|AAAAAAAAFMPNFCAA|2083-09-14|2204|9586|736|2083|2|9|14|3|2083|736|9586|Tuesday|2083Q3|N|N|N|2482104|2482346|2481752|2482026|N|N|N|N|N| +2482118|AAAAAAAAGMPNFCAA|2083-09-15|2204|9586|736|2083|3|9|15|3|2083|736|9586|Wednesday|2083Q3|N|N|N|2482104|2482346|2481753|2482027|N|N|N|N|N| +2482119|AAAAAAAAHMPNFCAA|2083-09-16|2204|9586|736|2083|4|9|16|3|2083|736|9586|Thursday|2083Q3|N|N|N|2482104|2482346|2481754|2482028|N|N|N|N|N| +2482120|AAAAAAAAIMPNFCAA|2083-09-17|2204|9586|736|2083|5|9|17|3|2083|736|9586|Friday|2083Q3|N|Y|N|2482104|2482346|2481755|2482029|N|N|N|N|N| +2482121|AAAAAAAAJMPNFCAA|2083-09-18|2204|9586|736|2083|6|9|18|3|2083|736|9586|Saturday|2083Q3|N|Y|N|2482104|2482346|2481756|2482030|N|N|N|N|N| +2482122|AAAAAAAAKMPNFCAA|2083-09-19|2204|9586|736|2083|0|9|19|3|2083|736|9586|Sunday|2083Q3|N|N|N|2482104|2482346|2481757|2482031|N|N|N|N|N| +2482123|AAAAAAAALMPNFCAA|2083-09-20|2204|9586|736|2083|1|9|20|3|2083|736|9586|Monday|2083Q3|N|N|N|2482104|2482346|2481758|2482032|N|N|N|N|N| +2482124|AAAAAAAAMMPNFCAA|2083-09-21|2204|9587|736|2083|2|9|21|3|2083|736|9587|Tuesday|2083Q3|N|N|N|2482104|2482346|2481759|2482033|N|N|N|N|N| +2482125|AAAAAAAANMPNFCAA|2083-09-22|2204|9587|736|2083|3|9|22|3|2083|736|9587|Wednesday|2083Q3|N|N|N|2482104|2482346|2481760|2482034|N|N|N|N|N| +2482126|AAAAAAAAOMPNFCAA|2083-09-23|2204|9587|736|2083|4|9|23|3|2083|736|9587|Thursday|2083Q3|N|N|N|2482104|2482346|2481761|2482035|N|N|N|N|N| +2482127|AAAAAAAAPMPNFCAA|2083-09-24|2204|9587|736|2083|5|9|24|3|2083|736|9587|Friday|2083Q3|N|Y|N|2482104|2482346|2481762|2482036|N|N|N|N|N| +2482128|AAAAAAAAANPNFCAA|2083-09-25|2204|9587|736|2083|6|9|25|3|2083|736|9587|Saturday|2083Q3|N|Y|N|2482104|2482346|2481763|2482037|N|N|N|N|N| +2482129|AAAAAAAABNPNFCAA|2083-09-26|2204|9587|736|2083|0|9|26|3|2083|736|9587|Sunday|2083Q3|N|N|N|2482104|2482346|2481764|2482038|N|N|N|N|N| +2482130|AAAAAAAACNPNFCAA|2083-09-27|2204|9587|736|2083|1|9|27|3|2083|736|9587|Monday|2083Q3|N|N|N|2482104|2482346|2481765|2482039|N|N|N|N|N| +2482131|AAAAAAAADNPNFCAA|2083-09-28|2204|9588|736|2083|2|9|28|3|2083|736|9588|Tuesday|2083Q3|N|N|N|2482104|2482346|2481766|2482040|N|N|N|N|N| +2482132|AAAAAAAAENPNFCAA|2083-09-29|2204|9588|736|2083|3|9|29|3|2083|736|9588|Wednesday|2083Q3|N|N|N|2482104|2482346|2481767|2482041|N|N|N|N|N| +2482133|AAAAAAAAFNPNFCAA|2083-09-30|2204|9588|736|2083|4|9|30|3|2083|736|9588|Thursday|2083Q3|N|N|N|2482104|2482346|2481768|2482042|N|N|N|N|N| +2482134|AAAAAAAAGNPNFCAA|2083-10-01|2205|9588|736|2083|5|10|1|3|2083|736|9588|Friday|2083Q3|N|Y|N|2482134|2482406|2481769|2482042|N|N|N|N|N| +2482135|AAAAAAAAHNPNFCAA|2083-10-02|2205|9588|736|2083|6|10|2|4|2083|736|9588|Saturday|2083Q4|N|Y|N|2482134|2482406|2481770|2482043|N|N|N|N|N| +2482136|AAAAAAAAINPNFCAA|2083-10-03|2205|9588|736|2083|0|10|3|4|2083|736|9588|Sunday|2083Q4|N|N|N|2482134|2482406|2481771|2482044|N|N|N|N|N| +2482137|AAAAAAAAJNPNFCAA|2083-10-04|2205|9588|736|2083|1|10|4|4|2083|736|9588|Monday|2083Q4|N|N|N|2482134|2482406|2481772|2482045|N|N|N|N|N| +2482138|AAAAAAAAKNPNFCAA|2083-10-05|2205|9589|736|2083|2|10|5|4|2083|736|9589|Tuesday|2083Q4|N|N|N|2482134|2482406|2481773|2482046|N|N|N|N|N| +2482139|AAAAAAAALNPNFCAA|2083-10-06|2205|9589|736|2083|3|10|6|4|2083|736|9589|Wednesday|2083Q4|N|N|N|2482134|2482406|2481774|2482047|N|N|N|N|N| +2482140|AAAAAAAAMNPNFCAA|2083-10-07|2205|9589|736|2083|4|10|7|4|2083|736|9589|Thursday|2083Q4|N|N|N|2482134|2482406|2481775|2482048|N|N|N|N|N| +2482141|AAAAAAAANNPNFCAA|2083-10-08|2205|9589|736|2083|5|10|8|4|2083|736|9589|Friday|2083Q4|N|Y|N|2482134|2482406|2481776|2482049|N|N|N|N|N| +2482142|AAAAAAAAONPNFCAA|2083-10-09|2205|9589|736|2083|6|10|9|4|2083|736|9589|Saturday|2083Q4|N|Y|N|2482134|2482406|2481777|2482050|N|N|N|N|N| +2482143|AAAAAAAAPNPNFCAA|2083-10-10|2205|9589|736|2083|0|10|10|4|2083|736|9589|Sunday|2083Q4|N|N|N|2482134|2482406|2481778|2482051|N|N|N|N|N| +2482144|AAAAAAAAAOPNFCAA|2083-10-11|2205|9589|736|2083|1|10|11|4|2083|736|9589|Monday|2083Q4|N|N|N|2482134|2482406|2481779|2482052|N|N|N|N|N| +2482145|AAAAAAAABOPNFCAA|2083-10-12|2205|9590|736|2083|2|10|12|4|2083|736|9590|Tuesday|2083Q4|N|N|N|2482134|2482406|2481780|2482053|N|N|N|N|N| +2482146|AAAAAAAACOPNFCAA|2083-10-13|2205|9590|736|2083|3|10|13|4|2083|736|9590|Wednesday|2083Q4|N|N|N|2482134|2482406|2481781|2482054|N|N|N|N|N| +2482147|AAAAAAAADOPNFCAA|2083-10-14|2205|9590|736|2083|4|10|14|4|2083|736|9590|Thursday|2083Q4|N|N|N|2482134|2482406|2481782|2482055|N|N|N|N|N| +2482148|AAAAAAAAEOPNFCAA|2083-10-15|2205|9590|736|2083|5|10|15|4|2083|736|9590|Friday|2083Q4|N|Y|N|2482134|2482406|2481783|2482056|N|N|N|N|N| +2482149|AAAAAAAAFOPNFCAA|2083-10-16|2205|9590|736|2083|6|10|16|4|2083|736|9590|Saturday|2083Q4|N|Y|N|2482134|2482406|2481784|2482057|N|N|N|N|N| +2482150|AAAAAAAAGOPNFCAA|2083-10-17|2205|9590|736|2083|0|10|17|4|2083|736|9590|Sunday|2083Q4|N|N|N|2482134|2482406|2481785|2482058|N|N|N|N|N| +2482151|AAAAAAAAHOPNFCAA|2083-10-18|2205|9590|736|2083|1|10|18|4|2083|736|9590|Monday|2083Q4|N|N|N|2482134|2482406|2481786|2482059|N|N|N|N|N| +2482152|AAAAAAAAIOPNFCAA|2083-10-19|2205|9591|736|2083|2|10|19|4|2083|736|9591|Tuesday|2083Q4|N|N|N|2482134|2482406|2481787|2482060|N|N|N|N|N| +2482153|AAAAAAAAJOPNFCAA|2083-10-20|2205|9591|736|2083|3|10|20|4|2083|736|9591|Wednesday|2083Q4|N|N|N|2482134|2482406|2481788|2482061|N|N|N|N|N| +2482154|AAAAAAAAKOPNFCAA|2083-10-21|2205|9591|736|2083|4|10|21|4|2083|736|9591|Thursday|2083Q4|N|N|N|2482134|2482406|2481789|2482062|N|N|N|N|N| +2482155|AAAAAAAALOPNFCAA|2083-10-22|2205|9591|736|2083|5|10|22|4|2083|736|9591|Friday|2083Q4|N|Y|N|2482134|2482406|2481790|2482063|N|N|N|N|N| +2482156|AAAAAAAAMOPNFCAA|2083-10-23|2205|9591|736|2083|6|10|23|4|2083|736|9591|Saturday|2083Q4|N|Y|N|2482134|2482406|2481791|2482064|N|N|N|N|N| +2482157|AAAAAAAANOPNFCAA|2083-10-24|2205|9591|736|2083|0|10|24|4|2083|736|9591|Sunday|2083Q4|N|N|N|2482134|2482406|2481792|2482065|N|N|N|N|N| +2482158|AAAAAAAAOOPNFCAA|2083-10-25|2205|9591|736|2083|1|10|25|4|2083|736|9591|Monday|2083Q4|N|N|N|2482134|2482406|2481793|2482066|N|N|N|N|N| +2482159|AAAAAAAAPOPNFCAA|2083-10-26|2205|9592|736|2083|2|10|26|4|2083|736|9592|Tuesday|2083Q4|N|N|N|2482134|2482406|2481794|2482067|N|N|N|N|N| +2482160|AAAAAAAAAPPNFCAA|2083-10-27|2205|9592|736|2083|3|10|27|4|2083|736|9592|Wednesday|2083Q4|N|N|N|2482134|2482406|2481795|2482068|N|N|N|N|N| +2482161|AAAAAAAABPPNFCAA|2083-10-28|2205|9592|736|2083|4|10|28|4|2083|736|9592|Thursday|2083Q4|N|N|N|2482134|2482406|2481796|2482069|N|N|N|N|N| +2482162|AAAAAAAACPPNFCAA|2083-10-29|2205|9592|736|2083|5|10|29|4|2083|736|9592|Friday|2083Q4|N|Y|N|2482134|2482406|2481797|2482070|N|N|N|N|N| +2482163|AAAAAAAADPPNFCAA|2083-10-30|2205|9592|736|2083|6|10|30|4|2083|736|9592|Saturday|2083Q4|N|Y|N|2482134|2482406|2481798|2482071|N|N|N|N|N| +2482164|AAAAAAAAEPPNFCAA|2083-10-31|2205|9592|736|2083|0|10|31|4|2083|736|9592|Sunday|2083Q4|N|N|N|2482134|2482406|2481799|2482072|N|N|N|N|N| +2482165|AAAAAAAAFPPNFCAA|2083-11-01|2206|9592|736|2083|1|11|1|4|2083|736|9592|Monday|2083Q4|N|N|N|2482165|2482468|2481800|2482073|N|N|N|N|N| +2482166|AAAAAAAAGPPNFCAA|2083-11-02|2206|9593|736|2083|2|11|2|4|2083|736|9593|Tuesday|2083Q4|N|N|N|2482165|2482468|2481801|2482074|N|N|N|N|N| +2482167|AAAAAAAAHPPNFCAA|2083-11-03|2206|9593|736|2083|3|11|3|4|2083|736|9593|Wednesday|2083Q4|N|N|N|2482165|2482468|2481802|2482075|N|N|N|N|N| +2482168|AAAAAAAAIPPNFCAA|2083-11-04|2206|9593|736|2083|4|11|4|4|2083|736|9593|Thursday|2083Q4|N|N|N|2482165|2482468|2481803|2482076|N|N|N|N|N| +2482169|AAAAAAAAJPPNFCAA|2083-11-05|2206|9593|736|2083|5|11|5|4|2083|736|9593|Friday|2083Q4|N|Y|N|2482165|2482468|2481804|2482077|N|N|N|N|N| +2482170|AAAAAAAAKPPNFCAA|2083-11-06|2206|9593|736|2083|6|11|6|4|2083|736|9593|Saturday|2083Q4|N|Y|N|2482165|2482468|2481805|2482078|N|N|N|N|N| +2482171|AAAAAAAALPPNFCAA|2083-11-07|2206|9593|736|2083|0|11|7|4|2083|736|9593|Sunday|2083Q4|N|N|N|2482165|2482468|2481806|2482079|N|N|N|N|N| +2482172|AAAAAAAAMPPNFCAA|2083-11-08|2206|9593|736|2083|1|11|8|4|2083|736|9593|Monday|2083Q4|N|N|N|2482165|2482468|2481807|2482080|N|N|N|N|N| +2482173|AAAAAAAANPPNFCAA|2083-11-09|2206|9594|736|2083|2|11|9|4|2083|736|9594|Tuesday|2083Q4|N|N|N|2482165|2482468|2481808|2482081|N|N|N|N|N| +2482174|AAAAAAAAOPPNFCAA|2083-11-10|2206|9594|736|2083|3|11|10|4|2083|736|9594|Wednesday|2083Q4|N|N|N|2482165|2482468|2481809|2482082|N|N|N|N|N| +2482175|AAAAAAAAPPPNFCAA|2083-11-11|2206|9594|736|2083|4|11|11|4|2083|736|9594|Thursday|2083Q4|N|N|N|2482165|2482468|2481810|2482083|N|N|N|N|N| +2482176|AAAAAAAAAAAOFCAA|2083-11-12|2206|9594|736|2083|5|11|12|4|2083|736|9594|Friday|2083Q4|N|Y|N|2482165|2482468|2481811|2482084|N|N|N|N|N| +2482177|AAAAAAAABAAOFCAA|2083-11-13|2206|9594|736|2083|6|11|13|4|2083|736|9594|Saturday|2083Q4|N|Y|N|2482165|2482468|2481812|2482085|N|N|N|N|N| +2482178|AAAAAAAACAAOFCAA|2083-11-14|2206|9594|736|2083|0|11|14|4|2083|736|9594|Sunday|2083Q4|N|N|N|2482165|2482468|2481813|2482086|N|N|N|N|N| +2482179|AAAAAAAADAAOFCAA|2083-11-15|2206|9594|736|2083|1|11|15|4|2083|736|9594|Monday|2083Q4|N|N|N|2482165|2482468|2481814|2482087|N|N|N|N|N| +2482180|AAAAAAAAEAAOFCAA|2083-11-16|2206|9595|736|2083|2|11|16|4|2083|736|9595|Tuesday|2083Q4|N|N|N|2482165|2482468|2481815|2482088|N|N|N|N|N| +2482181|AAAAAAAAFAAOFCAA|2083-11-17|2206|9595|736|2083|3|11|17|4|2083|736|9595|Wednesday|2083Q4|N|N|N|2482165|2482468|2481816|2482089|N|N|N|N|N| +2482182|AAAAAAAAGAAOFCAA|2083-11-18|2206|9595|736|2083|4|11|18|4|2083|736|9595|Thursday|2083Q4|N|N|N|2482165|2482468|2481817|2482090|N|N|N|N|N| +2482183|AAAAAAAAHAAOFCAA|2083-11-19|2206|9595|736|2083|5|11|19|4|2083|736|9595|Friday|2083Q4|N|Y|N|2482165|2482468|2481818|2482091|N|N|N|N|N| +2482184|AAAAAAAAIAAOFCAA|2083-11-20|2206|9595|736|2083|6|11|20|4|2083|736|9595|Saturday|2083Q4|N|Y|N|2482165|2482468|2481819|2482092|N|N|N|N|N| +2482185|AAAAAAAAJAAOFCAA|2083-11-21|2206|9595|736|2083|0|11|21|4|2083|736|9595|Sunday|2083Q4|N|N|N|2482165|2482468|2481820|2482093|N|N|N|N|N| +2482186|AAAAAAAAKAAOFCAA|2083-11-22|2206|9595|736|2083|1|11|22|4|2083|736|9595|Monday|2083Q4|N|N|N|2482165|2482468|2481821|2482094|N|N|N|N|N| +2482187|AAAAAAAALAAOFCAA|2083-11-23|2206|9596|736|2083|2|11|23|4|2083|736|9596|Tuesday|2083Q4|N|N|N|2482165|2482468|2481822|2482095|N|N|N|N|N| +2482188|AAAAAAAAMAAOFCAA|2083-11-24|2206|9596|736|2083|3|11|24|4|2083|736|9596|Wednesday|2083Q4|N|N|N|2482165|2482468|2481823|2482096|N|N|N|N|N| +2482189|AAAAAAAANAAOFCAA|2083-11-25|2206|9596|736|2083|4|11|25|4|2083|736|9596|Thursday|2083Q4|N|N|N|2482165|2482468|2481824|2482097|N|N|N|N|N| +2482190|AAAAAAAAOAAOFCAA|2083-11-26|2206|9596|736|2083|5|11|26|4|2083|736|9596|Friday|2083Q4|N|Y|N|2482165|2482468|2481825|2482098|N|N|N|N|N| +2482191|AAAAAAAAPAAOFCAA|2083-11-27|2206|9596|736|2083|6|11|27|4|2083|736|9596|Saturday|2083Q4|N|Y|N|2482165|2482468|2481826|2482099|N|N|N|N|N| +2482192|AAAAAAAAABAOFCAA|2083-11-28|2206|9596|736|2083|0|11|28|4|2083|736|9596|Sunday|2083Q4|N|N|N|2482165|2482468|2481827|2482100|N|N|N|N|N| +2482193|AAAAAAAABBAOFCAA|2083-11-29|2206|9596|736|2083|1|11|29|4|2083|736|9596|Monday|2083Q4|N|N|N|2482165|2482468|2481828|2482101|N|N|N|N|N| +2482194|AAAAAAAACBAOFCAA|2083-11-30|2206|9597|736|2083|2|11|30|4|2083|736|9597|Tuesday|2083Q4|N|N|N|2482165|2482468|2481829|2482102|N|N|N|N|N| +2482195|AAAAAAAADBAOFCAA|2083-12-01|2207|9597|737|2083|3|12|1|4|2083|737|9597|Wednesday|2083Q4|N|N|N|2482195|2482528|2481830|2482103|N|N|N|N|N| +2482196|AAAAAAAAEBAOFCAA|2083-12-02|2207|9597|737|2083|4|12|2|4|2083|737|9597|Thursday|2083Q4|N|N|N|2482195|2482528|2481831|2482104|N|N|N|N|N| +2482197|AAAAAAAAFBAOFCAA|2083-12-03|2207|9597|737|2083|5|12|3|4|2083|737|9597|Friday|2083Q4|N|Y|N|2482195|2482528|2481832|2482105|N|N|N|N|N| +2482198|AAAAAAAAGBAOFCAA|2083-12-04|2207|9597|737|2083|6|12|4|4|2083|737|9597|Saturday|2083Q4|N|Y|N|2482195|2482528|2481833|2482106|N|N|N|N|N| +2482199|AAAAAAAAHBAOFCAA|2083-12-05|2207|9597|737|2083|0|12|5|4|2083|737|9597|Sunday|2083Q4|N|N|N|2482195|2482528|2481834|2482107|N|N|N|N|N| +2482200|AAAAAAAAIBAOFCAA|2083-12-06|2207|9597|737|2083|1|12|6|4|2083|737|9597|Monday|2083Q4|N|N|N|2482195|2482528|2481835|2482108|N|N|N|N|N| +2482201|AAAAAAAAJBAOFCAA|2083-12-07|2207|9598|737|2083|2|12|7|4|2083|737|9598|Tuesday|2083Q4|N|N|N|2482195|2482528|2481836|2482109|N|N|N|N|N| +2482202|AAAAAAAAKBAOFCAA|2083-12-08|2207|9598|737|2083|3|12|8|4|2083|737|9598|Wednesday|2083Q4|N|N|N|2482195|2482528|2481837|2482110|N|N|N|N|N| +2482203|AAAAAAAALBAOFCAA|2083-12-09|2207|9598|737|2083|4|12|9|4|2083|737|9598|Thursday|2083Q4|N|N|N|2482195|2482528|2481838|2482111|N|N|N|N|N| +2482204|AAAAAAAAMBAOFCAA|2083-12-10|2207|9598|737|2083|5|12|10|4|2083|737|9598|Friday|2083Q4|N|Y|N|2482195|2482528|2481839|2482112|N|N|N|N|N| +2482205|AAAAAAAANBAOFCAA|2083-12-11|2207|9598|737|2083|6|12|11|4|2083|737|9598|Saturday|2083Q4|N|Y|N|2482195|2482528|2481840|2482113|N|N|N|N|N| +2482206|AAAAAAAAOBAOFCAA|2083-12-12|2207|9598|737|2083|0|12|12|4|2083|737|9598|Sunday|2083Q4|N|N|N|2482195|2482528|2481841|2482114|N|N|N|N|N| +2482207|AAAAAAAAPBAOFCAA|2083-12-13|2207|9598|737|2083|1|12|13|4|2083|737|9598|Monday|2083Q4|N|N|N|2482195|2482528|2481842|2482115|N|N|N|N|N| +2482208|AAAAAAAAACAOFCAA|2083-12-14|2207|9599|737|2083|2|12|14|4|2083|737|9599|Tuesday|2083Q4|N|N|N|2482195|2482528|2481843|2482116|N|N|N|N|N| +2482209|AAAAAAAABCAOFCAA|2083-12-15|2207|9599|737|2083|3|12|15|4|2083|737|9599|Wednesday|2083Q4|N|N|N|2482195|2482528|2481844|2482117|N|N|N|N|N| +2482210|AAAAAAAACCAOFCAA|2083-12-16|2207|9599|737|2083|4|12|16|4|2083|737|9599|Thursday|2083Q4|N|N|N|2482195|2482528|2481845|2482118|N|N|N|N|N| +2482211|AAAAAAAADCAOFCAA|2083-12-17|2207|9599|737|2083|5|12|17|4|2083|737|9599|Friday|2083Q4|N|Y|N|2482195|2482528|2481846|2482119|N|N|N|N|N| +2482212|AAAAAAAAECAOFCAA|2083-12-18|2207|9599|737|2083|6|12|18|4|2083|737|9599|Saturday|2083Q4|N|Y|N|2482195|2482528|2481847|2482120|N|N|N|N|N| +2482213|AAAAAAAAFCAOFCAA|2083-12-19|2207|9599|737|2083|0|12|19|4|2083|737|9599|Sunday|2083Q4|N|N|N|2482195|2482528|2481848|2482121|N|N|N|N|N| +2482214|AAAAAAAAGCAOFCAA|2083-12-20|2207|9599|737|2083|1|12|20|4|2083|737|9599|Monday|2083Q4|N|N|N|2482195|2482528|2481849|2482122|N|N|N|N|N| +2482215|AAAAAAAAHCAOFCAA|2083-12-21|2207|9600|737|2083|2|12|21|4|2083|737|9600|Tuesday|2083Q4|N|N|N|2482195|2482528|2481850|2482123|N|N|N|N|N| +2482216|AAAAAAAAICAOFCAA|2083-12-22|2207|9600|737|2083|3|12|22|4|2083|737|9600|Wednesday|2083Q4|N|N|N|2482195|2482528|2481851|2482124|N|N|N|N|N| +2482217|AAAAAAAAJCAOFCAA|2083-12-23|2207|9600|737|2083|4|12|23|4|2083|737|9600|Thursday|2083Q4|N|N|N|2482195|2482528|2481852|2482125|N|N|N|N|N| +2482218|AAAAAAAAKCAOFCAA|2083-12-24|2207|9600|737|2083|5|12|24|4|2083|737|9600|Friday|2083Q4|N|Y|N|2482195|2482528|2481853|2482126|N|N|N|N|N| +2482219|AAAAAAAALCAOFCAA|2083-12-25|2207|9600|737|2083|6|12|25|4|2083|737|9600|Saturday|2083Q4|N|Y|N|2482195|2482528|2481854|2482127|N|N|N|N|N| +2482220|AAAAAAAAMCAOFCAA|2083-12-26|2207|9600|737|2083|0|12|26|4|2083|737|9600|Sunday|2083Q4|Y|N|N|2482195|2482528|2481855|2482128|N|N|N|N|N| +2482221|AAAAAAAANCAOFCAA|2083-12-27|2207|9600|737|2083|1|12|27|4|2083|737|9600|Monday|2083Q4|N|N|Y|2482195|2482528|2481856|2482129|N|N|N|N|N| +2482222|AAAAAAAAOCAOFCAA|2083-12-28|2207|9601|737|2083|2|12|28|4|2083|737|9601|Tuesday|2083Q4|N|N|N|2482195|2482528|2481857|2482130|N|N|N|N|N| +2482223|AAAAAAAAPCAOFCAA|2083-12-29|2207|9601|737|2083|3|12|29|4|2083|737|9601|Wednesday|2083Q4|N|N|N|2482195|2482528|2481858|2482131|N|N|N|N|N| +2482224|AAAAAAAAADAOFCAA|2083-12-30|2207|9601|737|2083|4|12|30|4|2083|737|9601|Thursday|2083Q4|N|N|N|2482195|2482528|2481859|2482132|N|N|N|N|N| +2482225|AAAAAAAABDAOFCAA|2083-12-31|2207|9601|737|2083|5|12|31|4|2083|737|9601|Friday|2083Q4|N|Y|N|2482195|2482528|2481860|2482133|N|N|N|N|N| +2482226|AAAAAAAACDAOFCAA|2084-01-01|2208|9601|737|2084|6|1|1|1|2084|737|9601|Saturday|2084Q1|Y|Y|N|2482226|2482225|2481861|2482134|N|N|N|N|N| +2482227|AAAAAAAADDAOFCAA|2084-01-02|2208|9601|737|2084|0|1|2|1|2084|737|9601|Sunday|2084Q1|N|N|Y|2482226|2482225|2481862|2482135|N|N|N|N|N| +2482228|AAAAAAAAEDAOFCAA|2084-01-03|2208|9601|737|2084|1|1|3|1|2084|737|9601|Monday|2084Q1|N|N|N|2482226|2482225|2481863|2482136|N|N|N|N|N| +2482229|AAAAAAAAFDAOFCAA|2084-01-04|2208|9602|737|2084|2|1|4|1|2084|737|9602|Tuesday|2084Q1|N|N|N|2482226|2482225|2481864|2482137|N|N|N|N|N| +2482230|AAAAAAAAGDAOFCAA|2084-01-05|2208|9602|737|2084|3|1|5|1|2084|737|9602|Wednesday|2084Q1|N|N|N|2482226|2482225|2481865|2482138|N|N|N|N|N| +2482231|AAAAAAAAHDAOFCAA|2084-01-06|2208|9602|737|2084|4|1|6|1|2084|737|9602|Thursday|2084Q1|N|N|N|2482226|2482225|2481866|2482139|N|N|N|N|N| +2482232|AAAAAAAAIDAOFCAA|2084-01-07|2208|9602|737|2084|5|1|7|1|2084|737|9602|Friday|2084Q1|N|Y|N|2482226|2482225|2481867|2482140|N|N|N|N|N| +2482233|AAAAAAAAJDAOFCAA|2084-01-08|2208|9602|737|2084|6|1|8|1|2084|737|9602|Saturday|2084Q1|N|Y|N|2482226|2482225|2481868|2482141|N|N|N|N|N| +2482234|AAAAAAAAKDAOFCAA|2084-01-09|2208|9602|737|2084|0|1|9|1|2084|737|9602|Sunday|2084Q1|N|N|N|2482226|2482225|2481869|2482142|N|N|N|N|N| +2482235|AAAAAAAALDAOFCAA|2084-01-10|2208|9602|737|2084|1|1|10|1|2084|737|9602|Monday|2084Q1|N|N|N|2482226|2482225|2481870|2482143|N|N|N|N|N| +2482236|AAAAAAAAMDAOFCAA|2084-01-11|2208|9603|737|2084|2|1|11|1|2084|737|9603|Tuesday|2084Q1|N|N|N|2482226|2482225|2481871|2482144|N|N|N|N|N| +2482237|AAAAAAAANDAOFCAA|2084-01-12|2208|9603|737|2084|3|1|12|1|2084|737|9603|Wednesday|2084Q1|N|N|N|2482226|2482225|2481872|2482145|N|N|N|N|N| +2482238|AAAAAAAAODAOFCAA|2084-01-13|2208|9603|737|2084|4|1|13|1|2084|737|9603|Thursday|2084Q1|N|N|N|2482226|2482225|2481873|2482146|N|N|N|N|N| +2482239|AAAAAAAAPDAOFCAA|2084-01-14|2208|9603|737|2084|5|1|14|1|2084|737|9603|Friday|2084Q1|N|Y|N|2482226|2482225|2481874|2482147|N|N|N|N|N| +2482240|AAAAAAAAAEAOFCAA|2084-01-15|2208|9603|737|2084|6|1|15|1|2084|737|9603|Saturday|2084Q1|N|Y|N|2482226|2482225|2481875|2482148|N|N|N|N|N| +2482241|AAAAAAAABEAOFCAA|2084-01-16|2208|9603|737|2084|0|1|16|1|2084|737|9603|Sunday|2084Q1|N|N|N|2482226|2482225|2481876|2482149|N|N|N|N|N| +2482242|AAAAAAAACEAOFCAA|2084-01-17|2208|9603|737|2084|1|1|17|1|2084|737|9603|Monday|2084Q1|N|N|N|2482226|2482225|2481877|2482150|N|N|N|N|N| +2482243|AAAAAAAADEAOFCAA|2084-01-18|2208|9604|737|2084|2|1|18|1|2084|737|9604|Tuesday|2084Q1|N|N|N|2482226|2482225|2481878|2482151|N|N|N|N|N| +2482244|AAAAAAAAEEAOFCAA|2084-01-19|2208|9604|737|2084|3|1|19|1|2084|737|9604|Wednesday|2084Q1|N|N|N|2482226|2482225|2481879|2482152|N|N|N|N|N| +2482245|AAAAAAAAFEAOFCAA|2084-01-20|2208|9604|737|2084|4|1|20|1|2084|737|9604|Thursday|2084Q1|N|N|N|2482226|2482225|2481880|2482153|N|N|N|N|N| +2482246|AAAAAAAAGEAOFCAA|2084-01-21|2208|9604|737|2084|5|1|21|1|2084|737|9604|Friday|2084Q1|N|Y|N|2482226|2482225|2481881|2482154|N|N|N|N|N| +2482247|AAAAAAAAHEAOFCAA|2084-01-22|2208|9604|737|2084|6|1|22|1|2084|737|9604|Saturday|2084Q1|N|Y|N|2482226|2482225|2481882|2482155|N|N|N|N|N| +2482248|AAAAAAAAIEAOFCAA|2084-01-23|2208|9604|737|2084|0|1|23|1|2084|737|9604|Sunday|2084Q1|N|N|N|2482226|2482225|2481883|2482156|N|N|N|N|N| +2482249|AAAAAAAAJEAOFCAA|2084-01-24|2208|9604|737|2084|1|1|24|1|2084|737|9604|Monday|2084Q1|N|N|N|2482226|2482225|2481884|2482157|N|N|N|N|N| +2482250|AAAAAAAAKEAOFCAA|2084-01-25|2208|9605|737|2084|2|1|25|1|2084|737|9605|Tuesday|2084Q1|N|N|N|2482226|2482225|2481885|2482158|N|N|N|N|N| +2482251|AAAAAAAALEAOFCAA|2084-01-26|2208|9605|737|2084|3|1|26|1|2084|737|9605|Wednesday|2084Q1|N|N|N|2482226|2482225|2481886|2482159|N|N|N|N|N| +2482252|AAAAAAAAMEAOFCAA|2084-01-27|2208|9605|737|2084|4|1|27|1|2084|737|9605|Thursday|2084Q1|N|N|N|2482226|2482225|2481887|2482160|N|N|N|N|N| +2482253|AAAAAAAANEAOFCAA|2084-01-28|2208|9605|737|2084|5|1|28|1|2084|737|9605|Friday|2084Q1|N|Y|N|2482226|2482225|2481888|2482161|N|N|N|N|N| +2482254|AAAAAAAAOEAOFCAA|2084-01-29|2208|9605|737|2084|6|1|29|1|2084|737|9605|Saturday|2084Q1|N|Y|N|2482226|2482225|2481889|2482162|N|N|N|N|N| +2482255|AAAAAAAAPEAOFCAA|2084-01-30|2208|9605|737|2084|0|1|30|1|2084|737|9605|Sunday|2084Q1|N|N|N|2482226|2482225|2481890|2482163|N|N|N|N|N| +2482256|AAAAAAAAAFAOFCAA|2084-01-31|2208|9605|737|2084|1|1|31|1|2084|737|9605|Monday|2084Q1|N|N|N|2482226|2482225|2481891|2482164|N|N|N|N|N| +2482257|AAAAAAAABFAOFCAA|2084-02-01|2209|9606|737|2084|2|2|1|1|2084|737|9606|Tuesday|2084Q1|N|N|N|2482257|2482287|2481892|2482165|N|N|N|N|N| +2482258|AAAAAAAACFAOFCAA|2084-02-02|2209|9606|737|2084|3|2|2|1|2084|737|9606|Wednesday|2084Q1|N|N|N|2482257|2482287|2481893|2482166|N|N|N|N|N| +2482259|AAAAAAAADFAOFCAA|2084-02-03|2209|9606|737|2084|4|2|3|1|2084|737|9606|Thursday|2084Q1|N|N|N|2482257|2482287|2481894|2482167|N|N|N|N|N| +2482260|AAAAAAAAEFAOFCAA|2084-02-04|2209|9606|737|2084|5|2|4|1|2084|737|9606|Friday|2084Q1|N|Y|N|2482257|2482287|2481895|2482168|N|N|N|N|N| +2482261|AAAAAAAAFFAOFCAA|2084-02-05|2209|9606|737|2084|6|2|5|1|2084|737|9606|Saturday|2084Q1|N|Y|N|2482257|2482287|2481896|2482169|N|N|N|N|N| +2482262|AAAAAAAAGFAOFCAA|2084-02-06|2209|9606|737|2084|0|2|6|1|2084|737|9606|Sunday|2084Q1|N|N|N|2482257|2482287|2481897|2482170|N|N|N|N|N| +2482263|AAAAAAAAHFAOFCAA|2084-02-07|2209|9606|737|2084|1|2|7|1|2084|737|9606|Monday|2084Q1|N|N|N|2482257|2482287|2481898|2482171|N|N|N|N|N| +2482264|AAAAAAAAIFAOFCAA|2084-02-08|2209|9607|737|2084|2|2|8|1|2084|737|9607|Tuesday|2084Q1|N|N|N|2482257|2482287|2481899|2482172|N|N|N|N|N| +2482265|AAAAAAAAJFAOFCAA|2084-02-09|2209|9607|737|2084|3|2|9|1|2084|737|9607|Wednesday|2084Q1|N|N|N|2482257|2482287|2481900|2482173|N|N|N|N|N| +2482266|AAAAAAAAKFAOFCAA|2084-02-10|2209|9607|737|2084|4|2|10|1|2084|737|9607|Thursday|2084Q1|N|N|N|2482257|2482287|2481901|2482174|N|N|N|N|N| +2482267|AAAAAAAALFAOFCAA|2084-02-11|2209|9607|737|2084|5|2|11|1|2084|737|9607|Friday|2084Q1|N|Y|N|2482257|2482287|2481902|2482175|N|N|N|N|N| +2482268|AAAAAAAAMFAOFCAA|2084-02-12|2209|9607|737|2084|6|2|12|1|2084|737|9607|Saturday|2084Q1|N|Y|N|2482257|2482287|2481903|2482176|N|N|N|N|N| +2482269|AAAAAAAANFAOFCAA|2084-02-13|2209|9607|737|2084|0|2|13|1|2084|737|9607|Sunday|2084Q1|N|N|N|2482257|2482287|2481904|2482177|N|N|N|N|N| +2482270|AAAAAAAAOFAOFCAA|2084-02-14|2209|9607|737|2084|1|2|14|1|2084|737|9607|Monday|2084Q1|N|N|N|2482257|2482287|2481905|2482178|N|N|N|N|N| +2482271|AAAAAAAAPFAOFCAA|2084-02-15|2209|9608|737|2084|2|2|15|1|2084|737|9608|Tuesday|2084Q1|N|N|N|2482257|2482287|2481906|2482179|N|N|N|N|N| +2482272|AAAAAAAAAGAOFCAA|2084-02-16|2209|9608|737|2084|3|2|16|1|2084|737|9608|Wednesday|2084Q1|N|N|N|2482257|2482287|2481907|2482180|N|N|N|N|N| +2482273|AAAAAAAABGAOFCAA|2084-02-17|2209|9608|737|2084|4|2|17|1|2084|737|9608|Thursday|2084Q1|N|N|N|2482257|2482287|2481908|2482181|N|N|N|N|N| +2482274|AAAAAAAACGAOFCAA|2084-02-18|2209|9608|737|2084|5|2|18|1|2084|737|9608|Friday|2084Q1|N|Y|N|2482257|2482287|2481909|2482182|N|N|N|N|N| +2482275|AAAAAAAADGAOFCAA|2084-02-19|2209|9608|737|2084|6|2|19|1|2084|737|9608|Saturday|2084Q1|N|Y|N|2482257|2482287|2481910|2482183|N|N|N|N|N| +2482276|AAAAAAAAEGAOFCAA|2084-02-20|2209|9608|737|2084|0|2|20|1|2084|737|9608|Sunday|2084Q1|N|N|N|2482257|2482287|2481911|2482184|N|N|N|N|N| +2482277|AAAAAAAAFGAOFCAA|2084-02-21|2209|9608|737|2084|1|2|21|1|2084|737|9608|Monday|2084Q1|N|N|N|2482257|2482287|2481912|2482185|N|N|N|N|N| +2482278|AAAAAAAAGGAOFCAA|2084-02-22|2209|9609|737|2084|2|2|22|1|2084|737|9609|Tuesday|2084Q1|N|N|N|2482257|2482287|2481913|2482186|N|N|N|N|N| +2482279|AAAAAAAAHGAOFCAA|2084-02-23|2209|9609|737|2084|3|2|23|1|2084|737|9609|Wednesday|2084Q1|N|N|N|2482257|2482287|2481914|2482187|N|N|N|N|N| +2482280|AAAAAAAAIGAOFCAA|2084-02-24|2209|9609|737|2084|4|2|24|1|2084|737|9609|Thursday|2084Q1|N|N|N|2482257|2482287|2481915|2482188|N|N|N|N|N| +2482281|AAAAAAAAJGAOFCAA|2084-02-25|2209|9609|737|2084|5|2|25|1|2084|737|9609|Friday|2084Q1|N|Y|N|2482257|2482287|2481916|2482189|N|N|N|N|N| +2482282|AAAAAAAAKGAOFCAA|2084-02-26|2209|9609|737|2084|6|2|26|1|2084|737|9609|Saturday|2084Q1|N|Y|N|2482257|2482287|2481917|2482190|N|N|N|N|N| +2482283|AAAAAAAALGAOFCAA|2084-02-27|2209|9609|737|2084|0|2|27|1|2084|737|9609|Sunday|2084Q1|N|N|N|2482257|2482287|2481918|2482191|N|N|N|N|N| +2482284|AAAAAAAAMGAOFCAA|2084-02-28|2209|9609|737|2084|1|2|28|1|2084|737|9609|Monday|2084Q1|N|N|N|2482257|2482287|2481919|2482192|N|N|N|N|N| +2482285|AAAAAAAANGAOFCAA|2084-02-29|2209|9610|737|2084|2|2|29|1|2084|737|9610|Tuesday|2084Q1|N|N|N|2482257|2482287|2481919|2482193|N|N|N|N|N| +2482286|AAAAAAAAOGAOFCAA|2084-03-01|2210|9610|738|2084|3|3|1|1|2084|738|9610|Wednesday|2084Q1|N|N|N|2482286|2482345|2481920|2482194|N|N|N|N|N| +2482287|AAAAAAAAPGAOFCAA|2084-03-02|2210|9610|738|2084|4|3|2|1|2084|738|9610|Thursday|2084Q1|N|N|N|2482286|2482345|2481921|2482195|N|N|N|N|N| +2482288|AAAAAAAAAHAOFCAA|2084-03-03|2210|9610|738|2084|5|3|3|1|2084|738|9610|Friday|2084Q1|N|Y|N|2482286|2482345|2481922|2482196|N|N|N|N|N| +2482289|AAAAAAAABHAOFCAA|2084-03-04|2210|9610|738|2084|6|3|4|1|2084|738|9610|Saturday|2084Q1|N|Y|N|2482286|2482345|2481923|2482197|N|N|N|N|N| +2482290|AAAAAAAACHAOFCAA|2084-03-05|2210|9610|738|2084|0|3|5|1|2084|738|9610|Sunday|2084Q1|N|N|N|2482286|2482345|2481924|2482198|N|N|N|N|N| +2482291|AAAAAAAADHAOFCAA|2084-03-06|2210|9610|738|2084|1|3|6|1|2084|738|9610|Monday|2084Q1|N|N|N|2482286|2482345|2481925|2482199|N|N|N|N|N| +2482292|AAAAAAAAEHAOFCAA|2084-03-07|2210|9611|738|2084|2|3|7|1|2084|738|9611|Tuesday|2084Q1|N|N|N|2482286|2482345|2481926|2482200|N|N|N|N|N| +2482293|AAAAAAAAFHAOFCAA|2084-03-08|2210|9611|738|2084|3|3|8|1|2084|738|9611|Wednesday|2084Q1|N|N|N|2482286|2482345|2481927|2482201|N|N|N|N|N| +2482294|AAAAAAAAGHAOFCAA|2084-03-09|2210|9611|738|2084|4|3|9|1|2084|738|9611|Thursday|2084Q1|N|N|N|2482286|2482345|2481928|2482202|N|N|N|N|N| +2482295|AAAAAAAAHHAOFCAA|2084-03-10|2210|9611|738|2084|5|3|10|1|2084|738|9611|Friday|2084Q1|N|Y|N|2482286|2482345|2481929|2482203|N|N|N|N|N| +2482296|AAAAAAAAIHAOFCAA|2084-03-11|2210|9611|738|2084|6|3|11|1|2084|738|9611|Saturday|2084Q1|N|Y|N|2482286|2482345|2481930|2482204|N|N|N|N|N| +2482297|AAAAAAAAJHAOFCAA|2084-03-12|2210|9611|738|2084|0|3|12|1|2084|738|9611|Sunday|2084Q1|N|N|N|2482286|2482345|2481931|2482205|N|N|N|N|N| +2482298|AAAAAAAAKHAOFCAA|2084-03-13|2210|9611|738|2084|1|3|13|1|2084|738|9611|Monday|2084Q1|N|N|N|2482286|2482345|2481932|2482206|N|N|N|N|N| +2482299|AAAAAAAALHAOFCAA|2084-03-14|2210|9612|738|2084|2|3|14|1|2084|738|9612|Tuesday|2084Q1|N|N|N|2482286|2482345|2481933|2482207|N|N|N|N|N| +2482300|AAAAAAAAMHAOFCAA|2084-03-15|2210|9612|738|2084|3|3|15|1|2084|738|9612|Wednesday|2084Q1|N|N|N|2482286|2482345|2481934|2482208|N|N|N|N|N| +2482301|AAAAAAAANHAOFCAA|2084-03-16|2210|9612|738|2084|4|3|16|1|2084|738|9612|Thursday|2084Q1|N|N|N|2482286|2482345|2481935|2482209|N|N|N|N|N| +2482302|AAAAAAAAOHAOFCAA|2084-03-17|2210|9612|738|2084|5|3|17|1|2084|738|9612|Friday|2084Q1|N|Y|N|2482286|2482345|2481936|2482210|N|N|N|N|N| +2482303|AAAAAAAAPHAOFCAA|2084-03-18|2210|9612|738|2084|6|3|18|1|2084|738|9612|Saturday|2084Q1|N|Y|N|2482286|2482345|2481937|2482211|N|N|N|N|N| +2482304|AAAAAAAAAIAOFCAA|2084-03-19|2210|9612|738|2084|0|3|19|1|2084|738|9612|Sunday|2084Q1|N|N|N|2482286|2482345|2481938|2482212|N|N|N|N|N| +2482305|AAAAAAAABIAOFCAA|2084-03-20|2210|9612|738|2084|1|3|20|1|2084|738|9612|Monday|2084Q1|N|N|N|2482286|2482345|2481939|2482213|N|N|N|N|N| +2482306|AAAAAAAACIAOFCAA|2084-03-21|2210|9613|738|2084|2|3|21|1|2084|738|9613|Tuesday|2084Q1|N|N|N|2482286|2482345|2481940|2482214|N|N|N|N|N| +2482307|AAAAAAAADIAOFCAA|2084-03-22|2210|9613|738|2084|3|3|22|1|2084|738|9613|Wednesday|2084Q1|N|N|N|2482286|2482345|2481941|2482215|N|N|N|N|N| +2482308|AAAAAAAAEIAOFCAA|2084-03-23|2210|9613|738|2084|4|3|23|1|2084|738|9613|Thursday|2084Q1|N|N|N|2482286|2482345|2481942|2482216|N|N|N|N|N| +2482309|AAAAAAAAFIAOFCAA|2084-03-24|2210|9613|738|2084|5|3|24|1|2084|738|9613|Friday|2084Q1|N|Y|N|2482286|2482345|2481943|2482217|N|N|N|N|N| +2482310|AAAAAAAAGIAOFCAA|2084-03-25|2210|9613|738|2084|6|3|25|1|2084|738|9613|Saturday|2084Q1|N|Y|N|2482286|2482345|2481944|2482218|N|N|N|N|N| +2482311|AAAAAAAAHIAOFCAA|2084-03-26|2210|9613|738|2084|0|3|26|1|2084|738|9613|Sunday|2084Q1|N|N|N|2482286|2482345|2481945|2482219|N|N|N|N|N| +2482312|AAAAAAAAIIAOFCAA|2084-03-27|2210|9613|738|2084|1|3|27|1|2084|738|9613|Monday|2084Q1|N|N|N|2482286|2482345|2481946|2482220|N|N|N|N|N| +2482313|AAAAAAAAJIAOFCAA|2084-03-28|2210|9614|738|2084|2|3|28|1|2084|738|9614|Tuesday|2084Q1|N|N|N|2482286|2482345|2481947|2482221|N|N|N|N|N| +2482314|AAAAAAAAKIAOFCAA|2084-03-29|2210|9614|738|2084|3|3|29|1|2084|738|9614|Wednesday|2084Q1|N|N|N|2482286|2482345|2481948|2482222|N|N|N|N|N| +2482315|AAAAAAAALIAOFCAA|2084-03-30|2210|9614|738|2084|4|3|30|1|2084|738|9614|Thursday|2084Q1|N|N|N|2482286|2482345|2481949|2482223|N|N|N|N|N| +2482316|AAAAAAAAMIAOFCAA|2084-03-31|2210|9614|738|2084|5|3|31|1|2084|738|9614|Friday|2084Q1|N|Y|N|2482286|2482345|2481950|2482224|N|N|N|N|N| +2482317|AAAAAAAANIAOFCAA|2084-04-01|2211|9614|738|2084|6|4|1|2|2084|738|9614|Saturday|2084Q2|N|Y|N|2482317|2482407|2481951|2482226|N|N|N|N|N| +2482318|AAAAAAAAOIAOFCAA|2084-04-02|2211|9614|738|2084|0|4|2|2|2084|738|9614|Sunday|2084Q2|N|N|N|2482317|2482407|2481952|2482227|N|N|N|N|N| +2482319|AAAAAAAAPIAOFCAA|2084-04-03|2211|9614|738|2084|1|4|3|2|2084|738|9614|Monday|2084Q2|N|N|N|2482317|2482407|2481953|2482228|N|N|N|N|N| +2482320|AAAAAAAAAJAOFCAA|2084-04-04|2211|9615|738|2084|2|4|4|2|2084|738|9615|Tuesday|2084Q2|N|N|N|2482317|2482407|2481954|2482229|N|N|N|N|N| +2482321|AAAAAAAABJAOFCAA|2084-04-05|2211|9615|738|2084|3|4|5|2|2084|738|9615|Wednesday|2084Q2|N|N|N|2482317|2482407|2481955|2482230|N|N|N|N|N| +2482322|AAAAAAAACJAOFCAA|2084-04-06|2211|9615|738|2084|4|4|6|2|2084|738|9615|Thursday|2084Q2|N|N|N|2482317|2482407|2481956|2482231|N|N|N|N|N| +2482323|AAAAAAAADJAOFCAA|2084-04-07|2211|9615|738|2084|5|4|7|2|2084|738|9615|Friday|2084Q2|N|Y|N|2482317|2482407|2481957|2482232|N|N|N|N|N| +2482324|AAAAAAAAEJAOFCAA|2084-04-08|2211|9615|738|2084|6|4|8|2|2084|738|9615|Saturday|2084Q2|N|Y|N|2482317|2482407|2481958|2482233|N|N|N|N|N| +2482325|AAAAAAAAFJAOFCAA|2084-04-09|2211|9615|738|2084|0|4|9|2|2084|738|9615|Sunday|2084Q2|N|N|N|2482317|2482407|2481959|2482234|N|N|N|N|N| +2482326|AAAAAAAAGJAOFCAA|2084-04-10|2211|9615|738|2084|1|4|10|2|2084|738|9615|Monday|2084Q2|N|N|N|2482317|2482407|2481960|2482235|N|N|N|N|N| +2482327|AAAAAAAAHJAOFCAA|2084-04-11|2211|9616|738|2084|2|4|11|2|2084|738|9616|Tuesday|2084Q2|N|N|N|2482317|2482407|2481961|2482236|N|N|N|N|N| +2482328|AAAAAAAAIJAOFCAA|2084-04-12|2211|9616|738|2084|3|4|12|2|2084|738|9616|Wednesday|2084Q2|N|N|N|2482317|2482407|2481962|2482237|N|N|N|N|N| +2482329|AAAAAAAAJJAOFCAA|2084-04-13|2211|9616|738|2084|4|4|13|2|2084|738|9616|Thursday|2084Q2|N|N|N|2482317|2482407|2481963|2482238|N|N|N|N|N| +2482330|AAAAAAAAKJAOFCAA|2084-04-14|2211|9616|738|2084|5|4|14|2|2084|738|9616|Friday|2084Q2|N|Y|N|2482317|2482407|2481964|2482239|N|N|N|N|N| +2482331|AAAAAAAALJAOFCAA|2084-04-15|2211|9616|738|2084|6|4|15|2|2084|738|9616|Saturday|2084Q2|N|Y|N|2482317|2482407|2481965|2482240|N|N|N|N|N| +2482332|AAAAAAAAMJAOFCAA|2084-04-16|2211|9616|738|2084|0|4|16|2|2084|738|9616|Sunday|2084Q2|N|N|N|2482317|2482407|2481966|2482241|N|N|N|N|N| +2482333|AAAAAAAANJAOFCAA|2084-04-17|2211|9616|738|2084|1|4|17|2|2084|738|9616|Monday|2084Q2|N|N|N|2482317|2482407|2481967|2482242|N|N|N|N|N| +2482334|AAAAAAAAOJAOFCAA|2084-04-18|2211|9617|738|2084|2|4|18|2|2084|738|9617|Tuesday|2084Q2|N|N|N|2482317|2482407|2481968|2482243|N|N|N|N|N| +2482335|AAAAAAAAPJAOFCAA|2084-04-19|2211|9617|738|2084|3|4|19|2|2084|738|9617|Wednesday|2084Q2|N|N|N|2482317|2482407|2481969|2482244|N|N|N|N|N| +2482336|AAAAAAAAAKAOFCAA|2084-04-20|2211|9617|738|2084|4|4|20|2|2084|738|9617|Thursday|2084Q2|N|N|N|2482317|2482407|2481970|2482245|N|N|N|N|N| +2482337|AAAAAAAABKAOFCAA|2084-04-21|2211|9617|738|2084|5|4|21|2|2084|738|9617|Friday|2084Q2|N|Y|N|2482317|2482407|2481971|2482246|N|N|N|N|N| +2482338|AAAAAAAACKAOFCAA|2084-04-22|2211|9617|738|2084|6|4|22|2|2084|738|9617|Saturday|2084Q2|N|Y|N|2482317|2482407|2481972|2482247|N|N|N|N|N| +2482339|AAAAAAAADKAOFCAA|2084-04-23|2211|9617|738|2084|0|4|23|2|2084|738|9617|Sunday|2084Q2|N|N|N|2482317|2482407|2481973|2482248|N|N|N|N|N| +2482340|AAAAAAAAEKAOFCAA|2084-04-24|2211|9617|738|2084|1|4|24|2|2084|738|9617|Monday|2084Q2|N|N|N|2482317|2482407|2481974|2482249|N|N|N|N|N| +2482341|AAAAAAAAFKAOFCAA|2084-04-25|2211|9618|738|2084|2|4|25|2|2084|738|9618|Tuesday|2084Q2|N|N|N|2482317|2482407|2481975|2482250|N|N|N|N|N| +2482342|AAAAAAAAGKAOFCAA|2084-04-26|2211|9618|738|2084|3|4|26|2|2084|738|9618|Wednesday|2084Q2|N|N|N|2482317|2482407|2481976|2482251|N|N|N|N|N| +2482343|AAAAAAAAHKAOFCAA|2084-04-27|2211|9618|738|2084|4|4|27|2|2084|738|9618|Thursday|2084Q2|N|N|N|2482317|2482407|2481977|2482252|N|N|N|N|N| +2482344|AAAAAAAAIKAOFCAA|2084-04-28|2211|9618|738|2084|5|4|28|2|2084|738|9618|Friday|2084Q2|N|Y|N|2482317|2482407|2481978|2482253|N|N|N|N|N| +2482345|AAAAAAAAJKAOFCAA|2084-04-29|2211|9618|738|2084|6|4|29|2|2084|738|9618|Saturday|2084Q2|N|Y|N|2482317|2482407|2481979|2482254|N|N|N|N|N| +2482346|AAAAAAAAKKAOFCAA|2084-04-30|2211|9618|738|2084|0|4|30|2|2084|738|9618|Sunday|2084Q2|N|N|N|2482317|2482407|2481980|2482255|N|N|N|N|N| +2482347|AAAAAAAALKAOFCAA|2084-05-01|2212|9618|738|2084|1|5|1|2|2084|738|9618|Monday|2084Q2|N|N|N|2482347|2482467|2481981|2482256|N|N|N|N|N| +2482348|AAAAAAAAMKAOFCAA|2084-05-02|2212|9619|738|2084|2|5|2|2|2084|738|9619|Tuesday|2084Q2|N|N|N|2482347|2482467|2481982|2482257|N|N|N|N|N| +2482349|AAAAAAAANKAOFCAA|2084-05-03|2212|9619|738|2084|3|5|3|2|2084|738|9619|Wednesday|2084Q2|N|N|N|2482347|2482467|2481983|2482258|N|N|N|N|N| +2482350|AAAAAAAAOKAOFCAA|2084-05-04|2212|9619|738|2084|4|5|4|2|2084|738|9619|Thursday|2084Q2|N|N|N|2482347|2482467|2481984|2482259|N|N|N|N|N| +2482351|AAAAAAAAPKAOFCAA|2084-05-05|2212|9619|738|2084|5|5|5|2|2084|738|9619|Friday|2084Q2|N|Y|N|2482347|2482467|2481985|2482260|N|N|N|N|N| +2482352|AAAAAAAAALAOFCAA|2084-05-06|2212|9619|738|2084|6|5|6|2|2084|738|9619|Saturday|2084Q2|N|Y|N|2482347|2482467|2481986|2482261|N|N|N|N|N| +2482353|AAAAAAAABLAOFCAA|2084-05-07|2212|9619|738|2084|0|5|7|2|2084|738|9619|Sunday|2084Q2|N|N|N|2482347|2482467|2481987|2482262|N|N|N|N|N| +2482354|AAAAAAAACLAOFCAA|2084-05-08|2212|9619|738|2084|1|5|8|2|2084|738|9619|Monday|2084Q2|N|N|N|2482347|2482467|2481988|2482263|N|N|N|N|N| +2482355|AAAAAAAADLAOFCAA|2084-05-09|2212|9620|738|2084|2|5|9|2|2084|738|9620|Tuesday|2084Q2|N|N|N|2482347|2482467|2481989|2482264|N|N|N|N|N| +2482356|AAAAAAAAELAOFCAA|2084-05-10|2212|9620|738|2084|3|5|10|2|2084|738|9620|Wednesday|2084Q2|N|N|N|2482347|2482467|2481990|2482265|N|N|N|N|N| +2482357|AAAAAAAAFLAOFCAA|2084-05-11|2212|9620|738|2084|4|5|11|2|2084|738|9620|Thursday|2084Q2|N|N|N|2482347|2482467|2481991|2482266|N|N|N|N|N| +2482358|AAAAAAAAGLAOFCAA|2084-05-12|2212|9620|738|2084|5|5|12|2|2084|738|9620|Friday|2084Q2|N|Y|N|2482347|2482467|2481992|2482267|N|N|N|N|N| +2482359|AAAAAAAAHLAOFCAA|2084-05-13|2212|9620|738|2084|6|5|13|2|2084|738|9620|Saturday|2084Q2|N|Y|N|2482347|2482467|2481993|2482268|N|N|N|N|N| +2482360|AAAAAAAAILAOFCAA|2084-05-14|2212|9620|738|2084|0|5|14|2|2084|738|9620|Sunday|2084Q2|N|N|N|2482347|2482467|2481994|2482269|N|N|N|N|N| +2482361|AAAAAAAAJLAOFCAA|2084-05-15|2212|9620|738|2084|1|5|15|2|2084|738|9620|Monday|2084Q2|N|N|N|2482347|2482467|2481995|2482270|N|N|N|N|N| +2482362|AAAAAAAAKLAOFCAA|2084-05-16|2212|9621|738|2084|2|5|16|2|2084|738|9621|Tuesday|2084Q2|N|N|N|2482347|2482467|2481996|2482271|N|N|N|N|N| +2482363|AAAAAAAALLAOFCAA|2084-05-17|2212|9621|738|2084|3|5|17|2|2084|738|9621|Wednesday|2084Q2|N|N|N|2482347|2482467|2481997|2482272|N|N|N|N|N| +2482364|AAAAAAAAMLAOFCAA|2084-05-18|2212|9621|738|2084|4|5|18|2|2084|738|9621|Thursday|2084Q2|N|N|N|2482347|2482467|2481998|2482273|N|N|N|N|N| +2482365|AAAAAAAANLAOFCAA|2084-05-19|2212|9621|738|2084|5|5|19|2|2084|738|9621|Friday|2084Q2|N|Y|N|2482347|2482467|2481999|2482274|N|N|N|N|N| +2482366|AAAAAAAAOLAOFCAA|2084-05-20|2212|9621|738|2084|6|5|20|2|2084|738|9621|Saturday|2084Q2|N|Y|N|2482347|2482467|2482000|2482275|N|N|N|N|N| +2482367|AAAAAAAAPLAOFCAA|2084-05-21|2212|9621|738|2084|0|5|21|2|2084|738|9621|Sunday|2084Q2|N|N|N|2482347|2482467|2482001|2482276|N|N|N|N|N| +2482368|AAAAAAAAAMAOFCAA|2084-05-22|2212|9621|738|2084|1|5|22|2|2084|738|9621|Monday|2084Q2|N|N|N|2482347|2482467|2482002|2482277|N|N|N|N|N| +2482369|AAAAAAAABMAOFCAA|2084-05-23|2212|9622|738|2084|2|5|23|2|2084|738|9622|Tuesday|2084Q2|N|N|N|2482347|2482467|2482003|2482278|N|N|N|N|N| +2482370|AAAAAAAACMAOFCAA|2084-05-24|2212|9622|738|2084|3|5|24|2|2084|738|9622|Wednesday|2084Q2|N|N|N|2482347|2482467|2482004|2482279|N|N|N|N|N| +2482371|AAAAAAAADMAOFCAA|2084-05-25|2212|9622|738|2084|4|5|25|2|2084|738|9622|Thursday|2084Q2|N|N|N|2482347|2482467|2482005|2482280|N|N|N|N|N| +2482372|AAAAAAAAEMAOFCAA|2084-05-26|2212|9622|738|2084|5|5|26|2|2084|738|9622|Friday|2084Q2|N|Y|N|2482347|2482467|2482006|2482281|N|N|N|N|N| +2482373|AAAAAAAAFMAOFCAA|2084-05-27|2212|9622|738|2084|6|5|27|2|2084|738|9622|Saturday|2084Q2|N|Y|N|2482347|2482467|2482007|2482282|N|N|N|N|N| +2482374|AAAAAAAAGMAOFCAA|2084-05-28|2212|9622|738|2084|0|5|28|2|2084|738|9622|Sunday|2084Q2|N|N|N|2482347|2482467|2482008|2482283|N|N|N|N|N| +2482375|AAAAAAAAHMAOFCAA|2084-05-29|2212|9622|738|2084|1|5|29|2|2084|738|9622|Monday|2084Q2|N|N|N|2482347|2482467|2482009|2482284|N|N|N|N|N| +2482376|AAAAAAAAIMAOFCAA|2084-05-30|2212|9623|738|2084|2|5|30|2|2084|738|9623|Tuesday|2084Q2|N|N|N|2482347|2482467|2482010|2482285|N|N|N|N|N| +2482377|AAAAAAAAJMAOFCAA|2084-05-31|2212|9623|738|2084|3|5|31|2|2084|738|9623|Wednesday|2084Q2|N|N|N|2482347|2482467|2482011|2482286|N|N|N|N|N| +2482378|AAAAAAAAKMAOFCAA|2084-06-01|2213|9623|739|2084|4|6|1|2|2084|739|9623|Thursday|2084Q2|N|N|N|2482378|2482529|2482012|2482287|N|N|N|N|N| +2482379|AAAAAAAALMAOFCAA|2084-06-02|2213|9623|739|2084|5|6|2|2|2084|739|9623|Friday|2084Q2|N|Y|N|2482378|2482529|2482013|2482288|N|N|N|N|N| +2482380|AAAAAAAAMMAOFCAA|2084-06-03|2213|9623|739|2084|6|6|3|2|2084|739|9623|Saturday|2084Q2|N|Y|N|2482378|2482529|2482014|2482289|N|N|N|N|N| +2482381|AAAAAAAANMAOFCAA|2084-06-04|2213|9623|739|2084|0|6|4|2|2084|739|9623|Sunday|2084Q2|N|N|N|2482378|2482529|2482015|2482290|N|N|N|N|N| +2482382|AAAAAAAAOMAOFCAA|2084-06-05|2213|9623|739|2084|1|6|5|2|2084|739|9623|Monday|2084Q2|N|N|N|2482378|2482529|2482016|2482291|N|N|N|N|N| +2482383|AAAAAAAAPMAOFCAA|2084-06-06|2213|9624|739|2084|2|6|6|2|2084|739|9624|Tuesday|2084Q2|N|N|N|2482378|2482529|2482017|2482292|N|N|N|N|N| +2482384|AAAAAAAAANAOFCAA|2084-06-07|2213|9624|739|2084|3|6|7|2|2084|739|9624|Wednesday|2084Q2|N|N|N|2482378|2482529|2482018|2482293|N|N|N|N|N| +2482385|AAAAAAAABNAOFCAA|2084-06-08|2213|9624|739|2084|4|6|8|2|2084|739|9624|Thursday|2084Q2|N|N|N|2482378|2482529|2482019|2482294|N|N|N|N|N| +2482386|AAAAAAAACNAOFCAA|2084-06-09|2213|9624|739|2084|5|6|9|2|2084|739|9624|Friday|2084Q2|N|Y|N|2482378|2482529|2482020|2482295|N|N|N|N|N| +2482387|AAAAAAAADNAOFCAA|2084-06-10|2213|9624|739|2084|6|6|10|2|2084|739|9624|Saturday|2084Q2|N|Y|N|2482378|2482529|2482021|2482296|N|N|N|N|N| +2482388|AAAAAAAAENAOFCAA|2084-06-11|2213|9624|739|2084|0|6|11|2|2084|739|9624|Sunday|2084Q2|N|N|N|2482378|2482529|2482022|2482297|N|N|N|N|N| +2482389|AAAAAAAAFNAOFCAA|2084-06-12|2213|9624|739|2084|1|6|12|2|2084|739|9624|Monday|2084Q2|N|N|N|2482378|2482529|2482023|2482298|N|N|N|N|N| +2482390|AAAAAAAAGNAOFCAA|2084-06-13|2213|9625|739|2084|2|6|13|2|2084|739|9625|Tuesday|2084Q2|N|N|N|2482378|2482529|2482024|2482299|N|N|N|N|N| +2482391|AAAAAAAAHNAOFCAA|2084-06-14|2213|9625|739|2084|3|6|14|2|2084|739|9625|Wednesday|2084Q2|N|N|N|2482378|2482529|2482025|2482300|N|N|N|N|N| +2482392|AAAAAAAAINAOFCAA|2084-06-15|2213|9625|739|2084|4|6|15|2|2084|739|9625|Thursday|2084Q2|N|N|N|2482378|2482529|2482026|2482301|N|N|N|N|N| +2482393|AAAAAAAAJNAOFCAA|2084-06-16|2213|9625|739|2084|5|6|16|2|2084|739|9625|Friday|2084Q2|N|Y|N|2482378|2482529|2482027|2482302|N|N|N|N|N| +2482394|AAAAAAAAKNAOFCAA|2084-06-17|2213|9625|739|2084|6|6|17|2|2084|739|9625|Saturday|2084Q2|N|Y|N|2482378|2482529|2482028|2482303|N|N|N|N|N| +2482395|AAAAAAAALNAOFCAA|2084-06-18|2213|9625|739|2084|0|6|18|2|2084|739|9625|Sunday|2084Q2|N|N|N|2482378|2482529|2482029|2482304|N|N|N|N|N| +2482396|AAAAAAAAMNAOFCAA|2084-06-19|2213|9625|739|2084|1|6|19|2|2084|739|9625|Monday|2084Q2|N|N|N|2482378|2482529|2482030|2482305|N|N|N|N|N| +2482397|AAAAAAAANNAOFCAA|2084-06-20|2213|9626|739|2084|2|6|20|2|2084|739|9626|Tuesday|2084Q2|N|N|N|2482378|2482529|2482031|2482306|N|N|N|N|N| +2482398|AAAAAAAAONAOFCAA|2084-06-21|2213|9626|739|2084|3|6|21|2|2084|739|9626|Wednesday|2084Q2|N|N|N|2482378|2482529|2482032|2482307|N|N|N|N|N| +2482399|AAAAAAAAPNAOFCAA|2084-06-22|2213|9626|739|2084|4|6|22|2|2084|739|9626|Thursday|2084Q2|N|N|N|2482378|2482529|2482033|2482308|N|N|N|N|N| +2482400|AAAAAAAAAOAOFCAA|2084-06-23|2213|9626|739|2084|5|6|23|2|2084|739|9626|Friday|2084Q2|N|Y|N|2482378|2482529|2482034|2482309|N|N|N|N|N| +2482401|AAAAAAAABOAOFCAA|2084-06-24|2213|9626|739|2084|6|6|24|2|2084|739|9626|Saturday|2084Q2|N|Y|N|2482378|2482529|2482035|2482310|N|N|N|N|N| +2482402|AAAAAAAACOAOFCAA|2084-06-25|2213|9626|739|2084|0|6|25|2|2084|739|9626|Sunday|2084Q2|N|N|N|2482378|2482529|2482036|2482311|N|N|N|N|N| +2482403|AAAAAAAADOAOFCAA|2084-06-26|2213|9626|739|2084|1|6|26|2|2084|739|9626|Monday|2084Q2|N|N|N|2482378|2482529|2482037|2482312|N|N|N|N|N| +2482404|AAAAAAAAEOAOFCAA|2084-06-27|2213|9627|739|2084|2|6|27|2|2084|739|9627|Tuesday|2084Q2|N|N|N|2482378|2482529|2482038|2482313|N|N|N|N|N| +2482405|AAAAAAAAFOAOFCAA|2084-06-28|2213|9627|739|2084|3|6|28|2|2084|739|9627|Wednesday|2084Q2|N|N|N|2482378|2482529|2482039|2482314|N|N|N|N|N| +2482406|AAAAAAAAGOAOFCAA|2084-06-29|2213|9627|739|2084|4|6|29|2|2084|739|9627|Thursday|2084Q2|N|N|N|2482378|2482529|2482040|2482315|N|N|N|N|N| +2482407|AAAAAAAAHOAOFCAA|2084-06-30|2213|9627|739|2084|5|6|30|2|2084|739|9627|Friday|2084Q2|N|Y|N|2482378|2482529|2482041|2482316|N|N|N|N|N| +2482408|AAAAAAAAIOAOFCAA|2084-07-01|2214|9627|739|2084|6|7|1|3|2084|739|9627|Saturday|2084Q3|N|Y|N|2482408|2482589|2482042|2482317|N|N|N|N|N| +2482409|AAAAAAAAJOAOFCAA|2084-07-02|2214|9627|739|2084|0|7|2|3|2084|739|9627|Sunday|2084Q3|N|N|N|2482408|2482589|2482043|2482318|N|N|N|N|N| +2482410|AAAAAAAAKOAOFCAA|2084-07-03|2214|9627|739|2084|1|7|3|3|2084|739|9627|Monday|2084Q3|N|N|N|2482408|2482589|2482044|2482319|N|N|N|N|N| +2482411|AAAAAAAALOAOFCAA|2084-07-04|2214|9628|739|2084|2|7|4|3|2084|739|9628|Tuesday|2084Q3|Y|N|N|2482408|2482589|2482045|2482320|N|N|N|N|N| +2482412|AAAAAAAAMOAOFCAA|2084-07-05|2214|9628|739|2084|3|7|5|3|2084|739|9628|Wednesday|2084Q3|N|N|Y|2482408|2482589|2482046|2482321|N|N|N|N|N| +2482413|AAAAAAAANOAOFCAA|2084-07-06|2214|9628|739|2084|4|7|6|3|2084|739|9628|Thursday|2084Q3|N|N|N|2482408|2482589|2482047|2482322|N|N|N|N|N| +2482414|AAAAAAAAOOAOFCAA|2084-07-07|2214|9628|739|2084|5|7|7|3|2084|739|9628|Friday|2084Q3|N|Y|N|2482408|2482589|2482048|2482323|N|N|N|N|N| +2482415|AAAAAAAAPOAOFCAA|2084-07-08|2214|9628|739|2084|6|7|8|3|2084|739|9628|Saturday|2084Q3|N|Y|N|2482408|2482589|2482049|2482324|N|N|N|N|N| +2482416|AAAAAAAAAPAOFCAA|2084-07-09|2214|9628|739|2084|0|7|9|3|2084|739|9628|Sunday|2084Q3|N|N|N|2482408|2482589|2482050|2482325|N|N|N|N|N| +2482417|AAAAAAAABPAOFCAA|2084-07-10|2214|9628|739|2084|1|7|10|3|2084|739|9628|Monday|2084Q3|N|N|N|2482408|2482589|2482051|2482326|N|N|N|N|N| +2482418|AAAAAAAACPAOFCAA|2084-07-11|2214|9629|739|2084|2|7|11|3|2084|739|9629|Tuesday|2084Q3|N|N|N|2482408|2482589|2482052|2482327|N|N|N|N|N| +2482419|AAAAAAAADPAOFCAA|2084-07-12|2214|9629|739|2084|3|7|12|3|2084|739|9629|Wednesday|2084Q3|N|N|N|2482408|2482589|2482053|2482328|N|N|N|N|N| +2482420|AAAAAAAAEPAOFCAA|2084-07-13|2214|9629|739|2084|4|7|13|3|2084|739|9629|Thursday|2084Q3|N|N|N|2482408|2482589|2482054|2482329|N|N|N|N|N| +2482421|AAAAAAAAFPAOFCAA|2084-07-14|2214|9629|739|2084|5|7|14|3|2084|739|9629|Friday|2084Q3|N|Y|N|2482408|2482589|2482055|2482330|N|N|N|N|N| +2482422|AAAAAAAAGPAOFCAA|2084-07-15|2214|9629|739|2084|6|7|15|3|2084|739|9629|Saturday|2084Q3|N|Y|N|2482408|2482589|2482056|2482331|N|N|N|N|N| +2482423|AAAAAAAAHPAOFCAA|2084-07-16|2214|9629|739|2084|0|7|16|3|2084|739|9629|Sunday|2084Q3|N|N|N|2482408|2482589|2482057|2482332|N|N|N|N|N| +2482424|AAAAAAAAIPAOFCAA|2084-07-17|2214|9629|739|2084|1|7|17|3|2084|739|9629|Monday|2084Q3|N|N|N|2482408|2482589|2482058|2482333|N|N|N|N|N| +2482425|AAAAAAAAJPAOFCAA|2084-07-18|2214|9630|739|2084|2|7|18|3|2084|739|9630|Tuesday|2084Q3|N|N|N|2482408|2482589|2482059|2482334|N|N|N|N|N| +2482426|AAAAAAAAKPAOFCAA|2084-07-19|2214|9630|739|2084|3|7|19|3|2084|739|9630|Wednesday|2084Q3|N|N|N|2482408|2482589|2482060|2482335|N|N|N|N|N| +2482427|AAAAAAAALPAOFCAA|2084-07-20|2214|9630|739|2084|4|7|20|3|2084|739|9630|Thursday|2084Q3|N|N|N|2482408|2482589|2482061|2482336|N|N|N|N|N| +2482428|AAAAAAAAMPAOFCAA|2084-07-21|2214|9630|739|2084|5|7|21|3|2084|739|9630|Friday|2084Q3|N|Y|N|2482408|2482589|2482062|2482337|N|N|N|N|N| +2482429|AAAAAAAANPAOFCAA|2084-07-22|2214|9630|739|2084|6|7|22|3|2084|739|9630|Saturday|2084Q3|N|Y|N|2482408|2482589|2482063|2482338|N|N|N|N|N| +2482430|AAAAAAAAOPAOFCAA|2084-07-23|2214|9630|739|2084|0|7|23|3|2084|739|9630|Sunday|2084Q3|N|N|N|2482408|2482589|2482064|2482339|N|N|N|N|N| +2482431|AAAAAAAAPPAOFCAA|2084-07-24|2214|9630|739|2084|1|7|24|3|2084|739|9630|Monday|2084Q3|N|N|N|2482408|2482589|2482065|2482340|N|N|N|N|N| +2482432|AAAAAAAAAABOFCAA|2084-07-25|2214|9631|739|2084|2|7|25|3|2084|739|9631|Tuesday|2084Q3|N|N|N|2482408|2482589|2482066|2482341|N|N|N|N|N| +2482433|AAAAAAAABABOFCAA|2084-07-26|2214|9631|739|2084|3|7|26|3|2084|739|9631|Wednesday|2084Q3|N|N|N|2482408|2482589|2482067|2482342|N|N|N|N|N| +2482434|AAAAAAAACABOFCAA|2084-07-27|2214|9631|739|2084|4|7|27|3|2084|739|9631|Thursday|2084Q3|N|N|N|2482408|2482589|2482068|2482343|N|N|N|N|N| +2482435|AAAAAAAADABOFCAA|2084-07-28|2214|9631|739|2084|5|7|28|3|2084|739|9631|Friday|2084Q3|N|Y|N|2482408|2482589|2482069|2482344|N|N|N|N|N| +2482436|AAAAAAAAEABOFCAA|2084-07-29|2214|9631|739|2084|6|7|29|3|2084|739|9631|Saturday|2084Q3|N|Y|N|2482408|2482589|2482070|2482345|N|N|N|N|N| +2482437|AAAAAAAAFABOFCAA|2084-07-30|2214|9631|739|2084|0|7|30|3|2084|739|9631|Sunday|2084Q3|N|N|N|2482408|2482589|2482071|2482346|N|N|N|N|N| +2482438|AAAAAAAAGABOFCAA|2084-07-31|2214|9631|739|2084|1|7|31|3|2084|739|9631|Monday|2084Q3|N|N|N|2482408|2482589|2482072|2482347|N|N|N|N|N| +2482439|AAAAAAAAHABOFCAA|2084-08-01|2215|9632|739|2084|2|8|1|3|2084|739|9632|Tuesday|2084Q3|N|N|N|2482439|2482651|2482073|2482348|N|N|N|N|N| +2482440|AAAAAAAAIABOFCAA|2084-08-02|2215|9632|739|2084|3|8|2|3|2084|739|9632|Wednesday|2084Q3|N|N|N|2482439|2482651|2482074|2482349|N|N|N|N|N| +2482441|AAAAAAAAJABOFCAA|2084-08-03|2215|9632|739|2084|4|8|3|3|2084|739|9632|Thursday|2084Q3|N|N|N|2482439|2482651|2482075|2482350|N|N|N|N|N| +2482442|AAAAAAAAKABOFCAA|2084-08-04|2215|9632|739|2084|5|8|4|3|2084|739|9632|Friday|2084Q3|N|Y|N|2482439|2482651|2482076|2482351|N|N|N|N|N| +2482443|AAAAAAAALABOFCAA|2084-08-05|2215|9632|739|2084|6|8|5|3|2084|739|9632|Saturday|2084Q3|N|Y|N|2482439|2482651|2482077|2482352|N|N|N|N|N| +2482444|AAAAAAAAMABOFCAA|2084-08-06|2215|9632|739|2084|0|8|6|3|2084|739|9632|Sunday|2084Q3|N|N|N|2482439|2482651|2482078|2482353|N|N|N|N|N| +2482445|AAAAAAAANABOFCAA|2084-08-07|2215|9632|739|2084|1|8|7|3|2084|739|9632|Monday|2084Q3|N|N|N|2482439|2482651|2482079|2482354|N|N|N|N|N| +2482446|AAAAAAAAOABOFCAA|2084-08-08|2215|9633|739|2084|2|8|8|3|2084|739|9633|Tuesday|2084Q3|N|N|N|2482439|2482651|2482080|2482355|N|N|N|N|N| +2482447|AAAAAAAAPABOFCAA|2084-08-09|2215|9633|739|2084|3|8|9|3|2084|739|9633|Wednesday|2084Q3|N|N|N|2482439|2482651|2482081|2482356|N|N|N|N|N| +2482448|AAAAAAAAABBOFCAA|2084-08-10|2215|9633|739|2084|4|8|10|3|2084|739|9633|Thursday|2084Q3|N|N|N|2482439|2482651|2482082|2482357|N|N|N|N|N| +2482449|AAAAAAAABBBOFCAA|2084-08-11|2215|9633|739|2084|5|8|11|3|2084|739|9633|Friday|2084Q3|N|Y|N|2482439|2482651|2482083|2482358|N|N|N|N|N| +2482450|AAAAAAAACBBOFCAA|2084-08-12|2215|9633|739|2084|6|8|12|3|2084|739|9633|Saturday|2084Q3|N|Y|N|2482439|2482651|2482084|2482359|N|N|N|N|N| +2482451|AAAAAAAADBBOFCAA|2084-08-13|2215|9633|739|2084|0|8|13|3|2084|739|9633|Sunday|2084Q3|N|N|N|2482439|2482651|2482085|2482360|N|N|N|N|N| +2482452|AAAAAAAAEBBOFCAA|2084-08-14|2215|9633|739|2084|1|8|14|3|2084|739|9633|Monday|2084Q3|N|N|N|2482439|2482651|2482086|2482361|N|N|N|N|N| +2482453|AAAAAAAAFBBOFCAA|2084-08-15|2215|9634|739|2084|2|8|15|3|2084|739|9634|Tuesday|2084Q3|N|N|N|2482439|2482651|2482087|2482362|N|N|N|N|N| +2482454|AAAAAAAAGBBOFCAA|2084-08-16|2215|9634|739|2084|3|8|16|3|2084|739|9634|Wednesday|2084Q3|N|N|N|2482439|2482651|2482088|2482363|N|N|N|N|N| +2482455|AAAAAAAAHBBOFCAA|2084-08-17|2215|9634|739|2084|4|8|17|3|2084|739|9634|Thursday|2084Q3|N|N|N|2482439|2482651|2482089|2482364|N|N|N|N|N| +2482456|AAAAAAAAIBBOFCAA|2084-08-18|2215|9634|739|2084|5|8|18|3|2084|739|9634|Friday|2084Q3|N|Y|N|2482439|2482651|2482090|2482365|N|N|N|N|N| +2482457|AAAAAAAAJBBOFCAA|2084-08-19|2215|9634|739|2084|6|8|19|3|2084|739|9634|Saturday|2084Q3|N|Y|N|2482439|2482651|2482091|2482366|N|N|N|N|N| +2482458|AAAAAAAAKBBOFCAA|2084-08-20|2215|9634|739|2084|0|8|20|3|2084|739|9634|Sunday|2084Q3|N|N|N|2482439|2482651|2482092|2482367|N|N|N|N|N| +2482459|AAAAAAAALBBOFCAA|2084-08-21|2215|9634|739|2084|1|8|21|3|2084|739|9634|Monday|2084Q3|N|N|N|2482439|2482651|2482093|2482368|N|N|N|N|N| +2482460|AAAAAAAAMBBOFCAA|2084-08-22|2215|9635|739|2084|2|8|22|3|2084|739|9635|Tuesday|2084Q3|N|N|N|2482439|2482651|2482094|2482369|N|N|N|N|N| +2482461|AAAAAAAANBBOFCAA|2084-08-23|2215|9635|739|2084|3|8|23|3|2084|739|9635|Wednesday|2084Q3|N|N|N|2482439|2482651|2482095|2482370|N|N|N|N|N| +2482462|AAAAAAAAOBBOFCAA|2084-08-24|2215|9635|739|2084|4|8|24|3|2084|739|9635|Thursday|2084Q3|N|N|N|2482439|2482651|2482096|2482371|N|N|N|N|N| +2482463|AAAAAAAAPBBOFCAA|2084-08-25|2215|9635|739|2084|5|8|25|3|2084|739|9635|Friday|2084Q3|N|Y|N|2482439|2482651|2482097|2482372|N|N|N|N|N| +2482464|AAAAAAAAACBOFCAA|2084-08-26|2215|9635|739|2084|6|8|26|3|2084|739|9635|Saturday|2084Q3|N|Y|N|2482439|2482651|2482098|2482373|N|N|N|N|N| +2482465|AAAAAAAABCBOFCAA|2084-08-27|2215|9635|739|2084|0|8|27|3|2084|739|9635|Sunday|2084Q3|N|N|N|2482439|2482651|2482099|2482374|N|N|N|N|N| +2482466|AAAAAAAACCBOFCAA|2084-08-28|2215|9635|739|2084|1|8|28|3|2084|739|9635|Monday|2084Q3|N|N|N|2482439|2482651|2482100|2482375|N|N|N|N|N| +2482467|AAAAAAAADCBOFCAA|2084-08-29|2215|9636|739|2084|2|8|29|3|2084|739|9636|Tuesday|2084Q3|N|N|N|2482439|2482651|2482101|2482376|N|N|N|N|N| +2482468|AAAAAAAAECBOFCAA|2084-08-30|2215|9636|739|2084|3|8|30|3|2084|739|9636|Wednesday|2084Q3|N|N|N|2482439|2482651|2482102|2482377|N|N|N|N|N| +2482469|AAAAAAAAFCBOFCAA|2084-08-31|2215|9636|739|2084|4|8|31|3|2084|739|9636|Thursday|2084Q3|N|N|N|2482439|2482651|2482103|2482378|N|N|N|N|N| +2482470|AAAAAAAAGCBOFCAA|2084-09-01|2216|9636|740|2084|5|9|1|3|2084|740|9636|Friday|2084Q3|N|Y|N|2482470|2482713|2482104|2482379|N|N|N|N|N| +2482471|AAAAAAAAHCBOFCAA|2084-09-02|2216|9636|740|2084|6|9|2|3|2084|740|9636|Saturday|2084Q3|N|Y|N|2482470|2482713|2482105|2482380|N|N|N|N|N| +2482472|AAAAAAAAICBOFCAA|2084-09-03|2216|9636|740|2084|0|9|3|3|2084|740|9636|Sunday|2084Q3|N|N|N|2482470|2482713|2482106|2482381|N|N|N|N|N| +2482473|AAAAAAAAJCBOFCAA|2084-09-04|2216|9636|740|2084|1|9|4|3|2084|740|9636|Monday|2084Q3|N|N|N|2482470|2482713|2482107|2482382|N|N|N|N|N| +2482474|AAAAAAAAKCBOFCAA|2084-09-05|2216|9637|740|2084|2|9|5|3|2084|740|9637|Tuesday|2084Q3|N|N|N|2482470|2482713|2482108|2482383|N|N|N|N|N| +2482475|AAAAAAAALCBOFCAA|2084-09-06|2216|9637|740|2084|3|9|6|3|2084|740|9637|Wednesday|2084Q3|N|N|N|2482470|2482713|2482109|2482384|N|N|N|N|N| +2482476|AAAAAAAAMCBOFCAA|2084-09-07|2216|9637|740|2084|4|9|7|3|2084|740|9637|Thursday|2084Q3|N|N|N|2482470|2482713|2482110|2482385|N|N|N|N|N| +2482477|AAAAAAAANCBOFCAA|2084-09-08|2216|9637|740|2084|5|9|8|3|2084|740|9637|Friday|2084Q3|N|Y|N|2482470|2482713|2482111|2482386|N|N|N|N|N| +2482478|AAAAAAAAOCBOFCAA|2084-09-09|2216|9637|740|2084|6|9|9|3|2084|740|9637|Saturday|2084Q3|N|Y|N|2482470|2482713|2482112|2482387|N|N|N|N|N| +2482479|AAAAAAAAPCBOFCAA|2084-09-10|2216|9637|740|2084|0|9|10|3|2084|740|9637|Sunday|2084Q3|N|N|N|2482470|2482713|2482113|2482388|N|N|N|N|N| +2482480|AAAAAAAAADBOFCAA|2084-09-11|2216|9637|740|2084|1|9|11|3|2084|740|9637|Monday|2084Q3|N|N|N|2482470|2482713|2482114|2482389|N|N|N|N|N| +2482481|AAAAAAAABDBOFCAA|2084-09-12|2216|9638|740|2084|2|9|12|3|2084|740|9638|Tuesday|2084Q3|N|N|N|2482470|2482713|2482115|2482390|N|N|N|N|N| +2482482|AAAAAAAACDBOFCAA|2084-09-13|2216|9638|740|2084|3|9|13|3|2084|740|9638|Wednesday|2084Q3|N|N|N|2482470|2482713|2482116|2482391|N|N|N|N|N| +2482483|AAAAAAAADDBOFCAA|2084-09-14|2216|9638|740|2084|4|9|14|3|2084|740|9638|Thursday|2084Q3|N|N|N|2482470|2482713|2482117|2482392|N|N|N|N|N| +2482484|AAAAAAAAEDBOFCAA|2084-09-15|2216|9638|740|2084|5|9|15|3|2084|740|9638|Friday|2084Q3|N|Y|N|2482470|2482713|2482118|2482393|N|N|N|N|N| +2482485|AAAAAAAAFDBOFCAA|2084-09-16|2216|9638|740|2084|6|9|16|3|2084|740|9638|Saturday|2084Q3|N|Y|N|2482470|2482713|2482119|2482394|N|N|N|N|N| +2482486|AAAAAAAAGDBOFCAA|2084-09-17|2216|9638|740|2084|0|9|17|3|2084|740|9638|Sunday|2084Q3|N|N|N|2482470|2482713|2482120|2482395|N|N|N|N|N| +2482487|AAAAAAAAHDBOFCAA|2084-09-18|2216|9638|740|2084|1|9|18|3|2084|740|9638|Monday|2084Q3|N|N|N|2482470|2482713|2482121|2482396|N|N|N|N|N| +2482488|AAAAAAAAIDBOFCAA|2084-09-19|2216|9639|740|2084|2|9|19|3|2084|740|9639|Tuesday|2084Q3|N|N|N|2482470|2482713|2482122|2482397|N|N|N|N|N| +2482489|AAAAAAAAJDBOFCAA|2084-09-20|2216|9639|740|2084|3|9|20|3|2084|740|9639|Wednesday|2084Q3|N|N|N|2482470|2482713|2482123|2482398|N|N|N|N|N| +2482490|AAAAAAAAKDBOFCAA|2084-09-21|2216|9639|740|2084|4|9|21|3|2084|740|9639|Thursday|2084Q3|N|N|N|2482470|2482713|2482124|2482399|N|N|N|N|N| +2482491|AAAAAAAALDBOFCAA|2084-09-22|2216|9639|740|2084|5|9|22|3|2084|740|9639|Friday|2084Q3|N|Y|N|2482470|2482713|2482125|2482400|N|N|N|N|N| +2482492|AAAAAAAAMDBOFCAA|2084-09-23|2216|9639|740|2084|6|9|23|3|2084|740|9639|Saturday|2084Q3|N|Y|N|2482470|2482713|2482126|2482401|N|N|N|N|N| +2482493|AAAAAAAANDBOFCAA|2084-09-24|2216|9639|740|2084|0|9|24|3|2084|740|9639|Sunday|2084Q3|N|N|N|2482470|2482713|2482127|2482402|N|N|N|N|N| +2482494|AAAAAAAAODBOFCAA|2084-09-25|2216|9639|740|2084|1|9|25|3|2084|740|9639|Monday|2084Q3|N|N|N|2482470|2482713|2482128|2482403|N|N|N|N|N| +2482495|AAAAAAAAPDBOFCAA|2084-09-26|2216|9640|740|2084|2|9|26|3|2084|740|9640|Tuesday|2084Q3|N|N|N|2482470|2482713|2482129|2482404|N|N|N|N|N| +2482496|AAAAAAAAAEBOFCAA|2084-09-27|2216|9640|740|2084|3|9|27|3|2084|740|9640|Wednesday|2084Q3|N|N|N|2482470|2482713|2482130|2482405|N|N|N|N|N| +2482497|AAAAAAAABEBOFCAA|2084-09-28|2216|9640|740|2084|4|9|28|3|2084|740|9640|Thursday|2084Q3|N|N|N|2482470|2482713|2482131|2482406|N|N|N|N|N| +2482498|AAAAAAAACEBOFCAA|2084-09-29|2216|9640|740|2084|5|9|29|3|2084|740|9640|Friday|2084Q3|N|Y|N|2482470|2482713|2482132|2482407|N|N|N|N|N| +2482499|AAAAAAAADEBOFCAA|2084-09-30|2216|9640|740|2084|6|9|30|3|2084|740|9640|Saturday|2084Q3|N|Y|N|2482470|2482713|2482133|2482408|N|N|N|N|N| +2482500|AAAAAAAAEEBOFCAA|2084-10-01|2217|9640|740|2084|0|10|1|4|2084|740|9640|Sunday|2084Q4|N|N|N|2482500|2482773|2482134|2482408|N|N|N|N|N| +2482501|AAAAAAAAFEBOFCAA|2084-10-02|2217|9640|740|2084|1|10|2|4|2084|740|9640|Monday|2084Q4|N|N|N|2482500|2482773|2482135|2482409|N|N|N|N|N| +2482502|AAAAAAAAGEBOFCAA|2084-10-03|2217|9641|740|2084|2|10|3|4|2084|740|9641|Tuesday|2084Q4|N|N|N|2482500|2482773|2482136|2482410|N|N|N|N|N| +2482503|AAAAAAAAHEBOFCAA|2084-10-04|2217|9641|740|2084|3|10|4|4|2084|740|9641|Wednesday|2084Q4|N|N|N|2482500|2482773|2482137|2482411|N|N|N|N|N| +2482504|AAAAAAAAIEBOFCAA|2084-10-05|2217|9641|740|2084|4|10|5|4|2084|740|9641|Thursday|2084Q4|N|N|N|2482500|2482773|2482138|2482412|N|N|N|N|N| +2482505|AAAAAAAAJEBOFCAA|2084-10-06|2217|9641|740|2084|5|10|6|4|2084|740|9641|Friday|2084Q4|N|Y|N|2482500|2482773|2482139|2482413|N|N|N|N|N| +2482506|AAAAAAAAKEBOFCAA|2084-10-07|2217|9641|740|2084|6|10|7|4|2084|740|9641|Saturday|2084Q4|N|Y|N|2482500|2482773|2482140|2482414|N|N|N|N|N| +2482507|AAAAAAAALEBOFCAA|2084-10-08|2217|9641|740|2084|0|10|8|4|2084|740|9641|Sunday|2084Q4|N|N|N|2482500|2482773|2482141|2482415|N|N|N|N|N| +2482508|AAAAAAAAMEBOFCAA|2084-10-09|2217|9641|740|2084|1|10|9|4|2084|740|9641|Monday|2084Q4|N|N|N|2482500|2482773|2482142|2482416|N|N|N|N|N| +2482509|AAAAAAAANEBOFCAA|2084-10-10|2217|9642|740|2084|2|10|10|4|2084|740|9642|Tuesday|2084Q4|N|N|N|2482500|2482773|2482143|2482417|N|N|N|N|N| +2482510|AAAAAAAAOEBOFCAA|2084-10-11|2217|9642|740|2084|3|10|11|4|2084|740|9642|Wednesday|2084Q4|N|N|N|2482500|2482773|2482144|2482418|N|N|N|N|N| +2482511|AAAAAAAAPEBOFCAA|2084-10-12|2217|9642|740|2084|4|10|12|4|2084|740|9642|Thursday|2084Q4|N|N|N|2482500|2482773|2482145|2482419|N|N|N|N|N| +2482512|AAAAAAAAAFBOFCAA|2084-10-13|2217|9642|740|2084|5|10|13|4|2084|740|9642|Friday|2084Q4|N|Y|N|2482500|2482773|2482146|2482420|N|N|N|N|N| +2482513|AAAAAAAABFBOFCAA|2084-10-14|2217|9642|740|2084|6|10|14|4|2084|740|9642|Saturday|2084Q4|N|Y|N|2482500|2482773|2482147|2482421|N|N|N|N|N| +2482514|AAAAAAAACFBOFCAA|2084-10-15|2217|9642|740|2084|0|10|15|4|2084|740|9642|Sunday|2084Q4|N|N|N|2482500|2482773|2482148|2482422|N|N|N|N|N| +2482515|AAAAAAAADFBOFCAA|2084-10-16|2217|9642|740|2084|1|10|16|4|2084|740|9642|Monday|2084Q4|N|N|N|2482500|2482773|2482149|2482423|N|N|N|N|N| +2482516|AAAAAAAAEFBOFCAA|2084-10-17|2217|9643|740|2084|2|10|17|4|2084|740|9643|Tuesday|2084Q4|N|N|N|2482500|2482773|2482150|2482424|N|N|N|N|N| +2482517|AAAAAAAAFFBOFCAA|2084-10-18|2217|9643|740|2084|3|10|18|4|2084|740|9643|Wednesday|2084Q4|N|N|N|2482500|2482773|2482151|2482425|N|N|N|N|N| +2482518|AAAAAAAAGFBOFCAA|2084-10-19|2217|9643|740|2084|4|10|19|4|2084|740|9643|Thursday|2084Q4|N|N|N|2482500|2482773|2482152|2482426|N|N|N|N|N| +2482519|AAAAAAAAHFBOFCAA|2084-10-20|2217|9643|740|2084|5|10|20|4|2084|740|9643|Friday|2084Q4|N|Y|N|2482500|2482773|2482153|2482427|N|N|N|N|N| +2482520|AAAAAAAAIFBOFCAA|2084-10-21|2217|9643|740|2084|6|10|21|4|2084|740|9643|Saturday|2084Q4|N|Y|N|2482500|2482773|2482154|2482428|N|N|N|N|N| +2482521|AAAAAAAAJFBOFCAA|2084-10-22|2217|9643|740|2084|0|10|22|4|2084|740|9643|Sunday|2084Q4|N|N|N|2482500|2482773|2482155|2482429|N|N|N|N|N| +2482522|AAAAAAAAKFBOFCAA|2084-10-23|2217|9643|740|2084|1|10|23|4|2084|740|9643|Monday|2084Q4|N|N|N|2482500|2482773|2482156|2482430|N|N|N|N|N| +2482523|AAAAAAAALFBOFCAA|2084-10-24|2217|9644|740|2084|2|10|24|4|2084|740|9644|Tuesday|2084Q4|N|N|N|2482500|2482773|2482157|2482431|N|N|N|N|N| +2482524|AAAAAAAAMFBOFCAA|2084-10-25|2217|9644|740|2084|3|10|25|4|2084|740|9644|Wednesday|2084Q4|N|N|N|2482500|2482773|2482158|2482432|N|N|N|N|N| +2482525|AAAAAAAANFBOFCAA|2084-10-26|2217|9644|740|2084|4|10|26|4|2084|740|9644|Thursday|2084Q4|N|N|N|2482500|2482773|2482159|2482433|N|N|N|N|N| +2482526|AAAAAAAAOFBOFCAA|2084-10-27|2217|9644|740|2084|5|10|27|4|2084|740|9644|Friday|2084Q4|N|Y|N|2482500|2482773|2482160|2482434|N|N|N|N|N| +2482527|AAAAAAAAPFBOFCAA|2084-10-28|2217|9644|740|2084|6|10|28|4|2084|740|9644|Saturday|2084Q4|N|Y|N|2482500|2482773|2482161|2482435|N|N|N|N|N| +2482528|AAAAAAAAAGBOFCAA|2084-10-29|2217|9644|740|2084|0|10|29|4|2084|740|9644|Sunday|2084Q4|N|N|N|2482500|2482773|2482162|2482436|N|N|N|N|N| +2482529|AAAAAAAABGBOFCAA|2084-10-30|2217|9644|740|2084|1|10|30|4|2084|740|9644|Monday|2084Q4|N|N|N|2482500|2482773|2482163|2482437|N|N|N|N|N| +2482530|AAAAAAAACGBOFCAA|2084-10-31|2217|9645|740|2084|2|10|31|4|2084|740|9645|Tuesday|2084Q4|N|N|N|2482500|2482773|2482164|2482438|N|N|N|N|N| +2482531|AAAAAAAADGBOFCAA|2084-11-01|2218|9645|740|2084|3|11|1|4|2084|740|9645|Wednesday|2084Q4|N|N|N|2482531|2482835|2482165|2482439|N|N|N|N|N| +2482532|AAAAAAAAEGBOFCAA|2084-11-02|2218|9645|740|2084|4|11|2|4|2084|740|9645|Thursday|2084Q4|N|N|N|2482531|2482835|2482166|2482440|N|N|N|N|N| +2482533|AAAAAAAAFGBOFCAA|2084-11-03|2218|9645|740|2084|5|11|3|4|2084|740|9645|Friday|2084Q4|N|Y|N|2482531|2482835|2482167|2482441|N|N|N|N|N| +2482534|AAAAAAAAGGBOFCAA|2084-11-04|2218|9645|740|2084|6|11|4|4|2084|740|9645|Saturday|2084Q4|N|Y|N|2482531|2482835|2482168|2482442|N|N|N|N|N| +2482535|AAAAAAAAHGBOFCAA|2084-11-05|2218|9645|740|2084|0|11|5|4|2084|740|9645|Sunday|2084Q4|N|N|N|2482531|2482835|2482169|2482443|N|N|N|N|N| +2482536|AAAAAAAAIGBOFCAA|2084-11-06|2218|9645|740|2084|1|11|6|4|2084|740|9645|Monday|2084Q4|N|N|N|2482531|2482835|2482170|2482444|N|N|N|N|N| +2482537|AAAAAAAAJGBOFCAA|2084-11-07|2218|9646|740|2084|2|11|7|4|2084|740|9646|Tuesday|2084Q4|N|N|N|2482531|2482835|2482171|2482445|N|N|N|N|N| +2482538|AAAAAAAAKGBOFCAA|2084-11-08|2218|9646|740|2084|3|11|8|4|2084|740|9646|Wednesday|2084Q4|N|N|N|2482531|2482835|2482172|2482446|N|N|N|N|N| +2482539|AAAAAAAALGBOFCAA|2084-11-09|2218|9646|740|2084|4|11|9|4|2084|740|9646|Thursday|2084Q4|N|N|N|2482531|2482835|2482173|2482447|N|N|N|N|N| +2482540|AAAAAAAAMGBOFCAA|2084-11-10|2218|9646|740|2084|5|11|10|4|2084|740|9646|Friday|2084Q4|N|Y|N|2482531|2482835|2482174|2482448|N|N|N|N|N| +2482541|AAAAAAAANGBOFCAA|2084-11-11|2218|9646|740|2084|6|11|11|4|2084|740|9646|Saturday|2084Q4|N|Y|N|2482531|2482835|2482175|2482449|N|N|N|N|N| +2482542|AAAAAAAAOGBOFCAA|2084-11-12|2218|9646|740|2084|0|11|12|4|2084|740|9646|Sunday|2084Q4|N|N|N|2482531|2482835|2482176|2482450|N|N|N|N|N| +2482543|AAAAAAAAPGBOFCAA|2084-11-13|2218|9646|740|2084|1|11|13|4|2084|740|9646|Monday|2084Q4|N|N|N|2482531|2482835|2482177|2482451|N|N|N|N|N| +2482544|AAAAAAAAAHBOFCAA|2084-11-14|2218|9647|740|2084|2|11|14|4|2084|740|9647|Tuesday|2084Q4|N|N|N|2482531|2482835|2482178|2482452|N|N|N|N|N| +2482545|AAAAAAAABHBOFCAA|2084-11-15|2218|9647|740|2084|3|11|15|4|2084|740|9647|Wednesday|2084Q4|N|N|N|2482531|2482835|2482179|2482453|N|N|N|N|N| +2482546|AAAAAAAACHBOFCAA|2084-11-16|2218|9647|740|2084|4|11|16|4|2084|740|9647|Thursday|2084Q4|N|N|N|2482531|2482835|2482180|2482454|N|N|N|N|N| +2482547|AAAAAAAADHBOFCAA|2084-11-17|2218|9647|740|2084|5|11|17|4|2084|740|9647|Friday|2084Q4|N|Y|N|2482531|2482835|2482181|2482455|N|N|N|N|N| +2482548|AAAAAAAAEHBOFCAA|2084-11-18|2218|9647|740|2084|6|11|18|4|2084|740|9647|Saturday|2084Q4|N|Y|N|2482531|2482835|2482182|2482456|N|N|N|N|N| +2482549|AAAAAAAAFHBOFCAA|2084-11-19|2218|9647|740|2084|0|11|19|4|2084|740|9647|Sunday|2084Q4|N|N|N|2482531|2482835|2482183|2482457|N|N|N|N|N| +2482550|AAAAAAAAGHBOFCAA|2084-11-20|2218|9647|740|2084|1|11|20|4|2084|740|9647|Monday|2084Q4|N|N|N|2482531|2482835|2482184|2482458|N|N|N|N|N| +2482551|AAAAAAAAHHBOFCAA|2084-11-21|2218|9648|740|2084|2|11|21|4|2084|740|9648|Tuesday|2084Q4|N|N|N|2482531|2482835|2482185|2482459|N|N|N|N|N| +2482552|AAAAAAAAIHBOFCAA|2084-11-22|2218|9648|740|2084|3|11|22|4|2084|740|9648|Wednesday|2084Q4|N|N|N|2482531|2482835|2482186|2482460|N|N|N|N|N| +2482553|AAAAAAAAJHBOFCAA|2084-11-23|2218|9648|740|2084|4|11|23|4|2084|740|9648|Thursday|2084Q4|N|N|N|2482531|2482835|2482187|2482461|N|N|N|N|N| +2482554|AAAAAAAAKHBOFCAA|2084-11-24|2218|9648|740|2084|5|11|24|4|2084|740|9648|Friday|2084Q4|N|Y|N|2482531|2482835|2482188|2482462|N|N|N|N|N| +2482555|AAAAAAAALHBOFCAA|2084-11-25|2218|9648|740|2084|6|11|25|4|2084|740|9648|Saturday|2084Q4|N|Y|N|2482531|2482835|2482189|2482463|N|N|N|N|N| +2482556|AAAAAAAAMHBOFCAA|2084-11-26|2218|9648|740|2084|0|11|26|4|2084|740|9648|Sunday|2084Q4|N|N|N|2482531|2482835|2482190|2482464|N|N|N|N|N| +2482557|AAAAAAAANHBOFCAA|2084-11-27|2218|9648|740|2084|1|11|27|4|2084|740|9648|Monday|2084Q4|N|N|N|2482531|2482835|2482191|2482465|N|N|N|N|N| +2482558|AAAAAAAAOHBOFCAA|2084-11-28|2218|9649|740|2084|2|11|28|4|2084|740|9649|Tuesday|2084Q4|N|N|N|2482531|2482835|2482192|2482466|N|N|N|N|N| +2482559|AAAAAAAAPHBOFCAA|2084-11-29|2218|9649|740|2084|3|11|29|4|2084|740|9649|Wednesday|2084Q4|N|N|N|2482531|2482835|2482193|2482467|N|N|N|N|N| +2482560|AAAAAAAAAIBOFCAA|2084-11-30|2218|9649|740|2084|4|11|30|4|2084|740|9649|Thursday|2084Q4|N|N|N|2482531|2482835|2482194|2482468|N|N|N|N|N| +2482561|AAAAAAAABIBOFCAA|2084-12-01|2219|9649|741|2084|5|12|1|4|2084|741|9649|Friday|2084Q4|N|Y|N|2482561|2482895|2482195|2482469|N|N|N|N|N| +2482562|AAAAAAAACIBOFCAA|2084-12-02|2219|9649|741|2084|6|12|2|4|2084|741|9649|Saturday|2084Q4|N|Y|N|2482561|2482895|2482196|2482470|N|N|N|N|N| +2482563|AAAAAAAADIBOFCAA|2084-12-03|2219|9649|741|2084|0|12|3|4|2084|741|9649|Sunday|2084Q4|N|N|N|2482561|2482895|2482197|2482471|N|N|N|N|N| +2482564|AAAAAAAAEIBOFCAA|2084-12-04|2219|9649|741|2084|1|12|4|4|2084|741|9649|Monday|2084Q4|N|N|N|2482561|2482895|2482198|2482472|N|N|N|N|N| +2482565|AAAAAAAAFIBOFCAA|2084-12-05|2219|9650|741|2084|2|12|5|4|2084|741|9650|Tuesday|2084Q4|N|N|N|2482561|2482895|2482199|2482473|N|N|N|N|N| +2482566|AAAAAAAAGIBOFCAA|2084-12-06|2219|9650|741|2084|3|12|6|4|2084|741|9650|Wednesday|2084Q4|N|N|N|2482561|2482895|2482200|2482474|N|N|N|N|N| +2482567|AAAAAAAAHIBOFCAA|2084-12-07|2219|9650|741|2084|4|12|7|4|2084|741|9650|Thursday|2084Q4|N|N|N|2482561|2482895|2482201|2482475|N|N|N|N|N| +2482568|AAAAAAAAIIBOFCAA|2084-12-08|2219|9650|741|2084|5|12|8|4|2084|741|9650|Friday|2084Q4|N|Y|N|2482561|2482895|2482202|2482476|N|N|N|N|N| +2482569|AAAAAAAAJIBOFCAA|2084-12-09|2219|9650|741|2084|6|12|9|4|2084|741|9650|Saturday|2084Q4|N|Y|N|2482561|2482895|2482203|2482477|N|N|N|N|N| +2482570|AAAAAAAAKIBOFCAA|2084-12-10|2219|9650|741|2084|0|12|10|4|2084|741|9650|Sunday|2084Q4|N|N|N|2482561|2482895|2482204|2482478|N|N|N|N|N| +2482571|AAAAAAAALIBOFCAA|2084-12-11|2219|9650|741|2084|1|12|11|4|2084|741|9650|Monday|2084Q4|N|N|N|2482561|2482895|2482205|2482479|N|N|N|N|N| +2482572|AAAAAAAAMIBOFCAA|2084-12-12|2219|9651|741|2084|2|12|12|4|2084|741|9651|Tuesday|2084Q4|N|N|N|2482561|2482895|2482206|2482480|N|N|N|N|N| +2482573|AAAAAAAANIBOFCAA|2084-12-13|2219|9651|741|2084|3|12|13|4|2084|741|9651|Wednesday|2084Q4|N|N|N|2482561|2482895|2482207|2482481|N|N|N|N|N| +2482574|AAAAAAAAOIBOFCAA|2084-12-14|2219|9651|741|2084|4|12|14|4|2084|741|9651|Thursday|2084Q4|N|N|N|2482561|2482895|2482208|2482482|N|N|N|N|N| +2482575|AAAAAAAAPIBOFCAA|2084-12-15|2219|9651|741|2084|5|12|15|4|2084|741|9651|Friday|2084Q4|N|Y|N|2482561|2482895|2482209|2482483|N|N|N|N|N| +2482576|AAAAAAAAAJBOFCAA|2084-12-16|2219|9651|741|2084|6|12|16|4|2084|741|9651|Saturday|2084Q4|N|Y|N|2482561|2482895|2482210|2482484|N|N|N|N|N| +2482577|AAAAAAAABJBOFCAA|2084-12-17|2219|9651|741|2084|0|12|17|4|2084|741|9651|Sunday|2084Q4|N|N|N|2482561|2482895|2482211|2482485|N|N|N|N|N| +2482578|AAAAAAAACJBOFCAA|2084-12-18|2219|9651|741|2084|1|12|18|4|2084|741|9651|Monday|2084Q4|N|N|N|2482561|2482895|2482212|2482486|N|N|N|N|N| +2482579|AAAAAAAADJBOFCAA|2084-12-19|2219|9652|741|2084|2|12|19|4|2084|741|9652|Tuesday|2084Q4|N|N|N|2482561|2482895|2482213|2482487|N|N|N|N|N| +2482580|AAAAAAAAEJBOFCAA|2084-12-20|2219|9652|741|2084|3|12|20|4|2084|741|9652|Wednesday|2084Q4|N|N|N|2482561|2482895|2482214|2482488|N|N|N|N|N| +2482581|AAAAAAAAFJBOFCAA|2084-12-21|2219|9652|741|2084|4|12|21|4|2084|741|9652|Thursday|2084Q4|N|N|N|2482561|2482895|2482215|2482489|N|N|N|N|N| +2482582|AAAAAAAAGJBOFCAA|2084-12-22|2219|9652|741|2084|5|12|22|4|2084|741|9652|Friday|2084Q4|N|Y|N|2482561|2482895|2482216|2482490|N|N|N|N|N| +2482583|AAAAAAAAHJBOFCAA|2084-12-23|2219|9652|741|2084|6|12|23|4|2084|741|9652|Saturday|2084Q4|N|Y|N|2482561|2482895|2482217|2482491|N|N|N|N|N| +2482584|AAAAAAAAIJBOFCAA|2084-12-24|2219|9652|741|2084|0|12|24|4|2084|741|9652|Sunday|2084Q4|N|N|N|2482561|2482895|2482218|2482492|N|N|N|N|N| +2482585|AAAAAAAAJJBOFCAA|2084-12-25|2219|9652|741|2084|1|12|25|4|2084|741|9652|Monday|2084Q4|Y|N|N|2482561|2482895|2482219|2482493|N|N|N|N|N| +2482586|AAAAAAAAKJBOFCAA|2084-12-26|2219|9653|741|2084|2|12|26|4|2084|741|9653|Tuesday|2084Q4|N|N|Y|2482561|2482895|2482220|2482494|N|N|N|N|N| +2482587|AAAAAAAALJBOFCAA|2084-12-27|2219|9653|741|2084|3|12|27|4|2084|741|9653|Wednesday|2084Q4|N|N|N|2482561|2482895|2482221|2482495|N|N|N|N|N| +2482588|AAAAAAAAMJBOFCAA|2084-12-28|2219|9653|741|2084|4|12|28|4|2084|741|9653|Thursday|2084Q4|N|N|N|2482561|2482895|2482222|2482496|N|N|N|N|N| +2482589|AAAAAAAANJBOFCAA|2084-12-29|2219|9653|741|2084|5|12|29|4|2084|741|9653|Friday|2084Q4|N|Y|N|2482561|2482895|2482223|2482497|N|N|N|N|N| +2482590|AAAAAAAAOJBOFCAA|2084-12-30|2219|9653|741|2084|6|12|30|4|2084|741|9653|Saturday|2084Q4|N|Y|N|2482561|2482895|2482224|2482498|N|N|N|N|N| +2482591|AAAAAAAAPJBOFCAA|2084-12-31|2219|9653|741|2084|0|12|31|4|2084|741|9653|Sunday|2084Q4|Y|N|N|2482561|2482895|2482225|2482499|N|N|N|N|N| +2482592|AAAAAAAAAKBOFCAA|2085-01-01|2220|9653|741|2085|1|1|1|1|2085|741|9653|Monday|2085Q1|Y|N|Y|2482592|2482591|2482226|2482500|N|N|N|N|N| +2482593|AAAAAAAABKBOFCAA|2085-01-02|2220|9654|741|2085|2|1|2|1|2085|741|9654|Tuesday|2085Q1|N|N|Y|2482592|2482591|2482227|2482501|N|N|N|N|N| +2482594|AAAAAAAACKBOFCAA|2085-01-03|2220|9654|741|2085|3|1|3|1|2085|741|9654|Wednesday|2085Q1|N|N|N|2482592|2482591|2482228|2482502|N|N|N|N|N| +2482595|AAAAAAAADKBOFCAA|2085-01-04|2220|9654|741|2085|4|1|4|1|2085|741|9654|Thursday|2085Q1|N|N|N|2482592|2482591|2482229|2482503|N|N|N|N|N| +2482596|AAAAAAAAEKBOFCAA|2085-01-05|2220|9654|741|2085|5|1|5|1|2085|741|9654|Friday|2085Q1|N|Y|N|2482592|2482591|2482230|2482504|N|N|N|N|N| +2482597|AAAAAAAAFKBOFCAA|2085-01-06|2220|9654|741|2085|6|1|6|1|2085|741|9654|Saturday|2085Q1|N|Y|N|2482592|2482591|2482231|2482505|N|N|N|N|N| +2482598|AAAAAAAAGKBOFCAA|2085-01-07|2220|9654|741|2085|0|1|7|1|2085|741|9654|Sunday|2085Q1|N|N|N|2482592|2482591|2482232|2482506|N|N|N|N|N| +2482599|AAAAAAAAHKBOFCAA|2085-01-08|2220|9654|741|2085|1|1|8|1|2085|741|9654|Monday|2085Q1|N|N|N|2482592|2482591|2482233|2482507|N|N|N|N|N| +2482600|AAAAAAAAIKBOFCAA|2085-01-09|2220|9655|741|2085|2|1|9|1|2085|741|9655|Tuesday|2085Q1|N|N|N|2482592|2482591|2482234|2482508|N|N|N|N|N| +2482601|AAAAAAAAJKBOFCAA|2085-01-10|2220|9655|741|2085|3|1|10|1|2085|741|9655|Wednesday|2085Q1|N|N|N|2482592|2482591|2482235|2482509|N|N|N|N|N| +2482602|AAAAAAAAKKBOFCAA|2085-01-11|2220|9655|741|2085|4|1|11|1|2085|741|9655|Thursday|2085Q1|N|N|N|2482592|2482591|2482236|2482510|N|N|N|N|N| +2482603|AAAAAAAALKBOFCAA|2085-01-12|2220|9655|741|2085|5|1|12|1|2085|741|9655|Friday|2085Q1|N|Y|N|2482592|2482591|2482237|2482511|N|N|N|N|N| +2482604|AAAAAAAAMKBOFCAA|2085-01-13|2220|9655|741|2085|6|1|13|1|2085|741|9655|Saturday|2085Q1|N|Y|N|2482592|2482591|2482238|2482512|N|N|N|N|N| +2482605|AAAAAAAANKBOFCAA|2085-01-14|2220|9655|741|2085|0|1|14|1|2085|741|9655|Sunday|2085Q1|N|N|N|2482592|2482591|2482239|2482513|N|N|N|N|N| +2482606|AAAAAAAAOKBOFCAA|2085-01-15|2220|9655|741|2085|1|1|15|1|2085|741|9655|Monday|2085Q1|N|N|N|2482592|2482591|2482240|2482514|N|N|N|N|N| +2482607|AAAAAAAAPKBOFCAA|2085-01-16|2220|9656|741|2085|2|1|16|1|2085|741|9656|Tuesday|2085Q1|N|N|N|2482592|2482591|2482241|2482515|N|N|N|N|N| +2482608|AAAAAAAAALBOFCAA|2085-01-17|2220|9656|741|2085|3|1|17|1|2085|741|9656|Wednesday|2085Q1|N|N|N|2482592|2482591|2482242|2482516|N|N|N|N|N| +2482609|AAAAAAAABLBOFCAA|2085-01-18|2220|9656|741|2085|4|1|18|1|2085|741|9656|Thursday|2085Q1|N|N|N|2482592|2482591|2482243|2482517|N|N|N|N|N| +2482610|AAAAAAAACLBOFCAA|2085-01-19|2220|9656|741|2085|5|1|19|1|2085|741|9656|Friday|2085Q1|N|Y|N|2482592|2482591|2482244|2482518|N|N|N|N|N| +2482611|AAAAAAAADLBOFCAA|2085-01-20|2220|9656|741|2085|6|1|20|1|2085|741|9656|Saturday|2085Q1|N|Y|N|2482592|2482591|2482245|2482519|N|N|N|N|N| +2482612|AAAAAAAAELBOFCAA|2085-01-21|2220|9656|741|2085|0|1|21|1|2085|741|9656|Sunday|2085Q1|N|N|N|2482592|2482591|2482246|2482520|N|N|N|N|N| +2482613|AAAAAAAAFLBOFCAA|2085-01-22|2220|9656|741|2085|1|1|22|1|2085|741|9656|Monday|2085Q1|N|N|N|2482592|2482591|2482247|2482521|N|N|N|N|N| +2482614|AAAAAAAAGLBOFCAA|2085-01-23|2220|9657|741|2085|2|1|23|1|2085|741|9657|Tuesday|2085Q1|N|N|N|2482592|2482591|2482248|2482522|N|N|N|N|N| +2482615|AAAAAAAAHLBOFCAA|2085-01-24|2220|9657|741|2085|3|1|24|1|2085|741|9657|Wednesday|2085Q1|N|N|N|2482592|2482591|2482249|2482523|N|N|N|N|N| +2482616|AAAAAAAAILBOFCAA|2085-01-25|2220|9657|741|2085|4|1|25|1|2085|741|9657|Thursday|2085Q1|N|N|N|2482592|2482591|2482250|2482524|N|N|N|N|N| +2482617|AAAAAAAAJLBOFCAA|2085-01-26|2220|9657|741|2085|5|1|26|1|2085|741|9657|Friday|2085Q1|N|Y|N|2482592|2482591|2482251|2482525|N|N|N|N|N| +2482618|AAAAAAAAKLBOFCAA|2085-01-27|2220|9657|741|2085|6|1|27|1|2085|741|9657|Saturday|2085Q1|N|Y|N|2482592|2482591|2482252|2482526|N|N|N|N|N| +2482619|AAAAAAAALLBOFCAA|2085-01-28|2220|9657|741|2085|0|1|28|1|2085|741|9657|Sunday|2085Q1|N|N|N|2482592|2482591|2482253|2482527|N|N|N|N|N| +2482620|AAAAAAAAMLBOFCAA|2085-01-29|2220|9657|741|2085|1|1|29|1|2085|741|9657|Monday|2085Q1|N|N|N|2482592|2482591|2482254|2482528|N|N|N|N|N| +2482621|AAAAAAAANLBOFCAA|2085-01-30|2220|9658|741|2085|2|1|30|1|2085|741|9658|Tuesday|2085Q1|N|N|N|2482592|2482591|2482255|2482529|N|N|N|N|N| +2482622|AAAAAAAAOLBOFCAA|2085-01-31|2220|9658|741|2085|3|1|31|1|2085|741|9658|Wednesday|2085Q1|N|N|N|2482592|2482591|2482256|2482530|N|N|N|N|N| +2482623|AAAAAAAAPLBOFCAA|2085-02-01|2221|9658|741|2085|4|2|1|1|2085|741|9658|Thursday|2085Q1|N|N|N|2482623|2482653|2482257|2482531|N|N|N|N|N| +2482624|AAAAAAAAAMBOFCAA|2085-02-02|2221|9658|741|2085|5|2|2|1|2085|741|9658|Friday|2085Q1|N|Y|N|2482623|2482653|2482258|2482532|N|N|N|N|N| +2482625|AAAAAAAABMBOFCAA|2085-02-03|2221|9658|741|2085|6|2|3|1|2085|741|9658|Saturday|2085Q1|N|Y|N|2482623|2482653|2482259|2482533|N|N|N|N|N| +2482626|AAAAAAAACMBOFCAA|2085-02-04|2221|9658|741|2085|0|2|4|1|2085|741|9658|Sunday|2085Q1|N|N|N|2482623|2482653|2482260|2482534|N|N|N|N|N| +2482627|AAAAAAAADMBOFCAA|2085-02-05|2221|9658|741|2085|1|2|5|1|2085|741|9658|Monday|2085Q1|N|N|N|2482623|2482653|2482261|2482535|N|N|N|N|N| +2482628|AAAAAAAAEMBOFCAA|2085-02-06|2221|9659|741|2085|2|2|6|1|2085|741|9659|Tuesday|2085Q1|N|N|N|2482623|2482653|2482262|2482536|N|N|N|N|N| +2482629|AAAAAAAAFMBOFCAA|2085-02-07|2221|9659|741|2085|3|2|7|1|2085|741|9659|Wednesday|2085Q1|N|N|N|2482623|2482653|2482263|2482537|N|N|N|N|N| +2482630|AAAAAAAAGMBOFCAA|2085-02-08|2221|9659|741|2085|4|2|8|1|2085|741|9659|Thursday|2085Q1|N|N|N|2482623|2482653|2482264|2482538|N|N|N|N|N| +2482631|AAAAAAAAHMBOFCAA|2085-02-09|2221|9659|741|2085|5|2|9|1|2085|741|9659|Friday|2085Q1|N|Y|N|2482623|2482653|2482265|2482539|N|N|N|N|N| +2482632|AAAAAAAAIMBOFCAA|2085-02-10|2221|9659|741|2085|6|2|10|1|2085|741|9659|Saturday|2085Q1|N|Y|N|2482623|2482653|2482266|2482540|N|N|N|N|N| +2482633|AAAAAAAAJMBOFCAA|2085-02-11|2221|9659|741|2085|0|2|11|1|2085|741|9659|Sunday|2085Q1|N|N|N|2482623|2482653|2482267|2482541|N|N|N|N|N| +2482634|AAAAAAAAKMBOFCAA|2085-02-12|2221|9659|741|2085|1|2|12|1|2085|741|9659|Monday|2085Q1|N|N|N|2482623|2482653|2482268|2482542|N|N|N|N|N| +2482635|AAAAAAAALMBOFCAA|2085-02-13|2221|9660|741|2085|2|2|13|1|2085|741|9660|Tuesday|2085Q1|N|N|N|2482623|2482653|2482269|2482543|N|N|N|N|N| +2482636|AAAAAAAAMMBOFCAA|2085-02-14|2221|9660|741|2085|3|2|14|1|2085|741|9660|Wednesday|2085Q1|N|N|N|2482623|2482653|2482270|2482544|N|N|N|N|N| +2482637|AAAAAAAANMBOFCAA|2085-02-15|2221|9660|741|2085|4|2|15|1|2085|741|9660|Thursday|2085Q1|N|N|N|2482623|2482653|2482271|2482545|N|N|N|N|N| +2482638|AAAAAAAAOMBOFCAA|2085-02-16|2221|9660|741|2085|5|2|16|1|2085|741|9660|Friday|2085Q1|N|Y|N|2482623|2482653|2482272|2482546|N|N|N|N|N| +2482639|AAAAAAAAPMBOFCAA|2085-02-17|2221|9660|741|2085|6|2|17|1|2085|741|9660|Saturday|2085Q1|N|Y|N|2482623|2482653|2482273|2482547|N|N|N|N|N| +2482640|AAAAAAAAANBOFCAA|2085-02-18|2221|9660|741|2085|0|2|18|1|2085|741|9660|Sunday|2085Q1|N|N|N|2482623|2482653|2482274|2482548|N|N|N|N|N| +2482641|AAAAAAAABNBOFCAA|2085-02-19|2221|9660|741|2085|1|2|19|1|2085|741|9660|Monday|2085Q1|N|N|N|2482623|2482653|2482275|2482549|N|N|N|N|N| +2482642|AAAAAAAACNBOFCAA|2085-02-20|2221|9661|741|2085|2|2|20|1|2085|741|9661|Tuesday|2085Q1|N|N|N|2482623|2482653|2482276|2482550|N|N|N|N|N| +2482643|AAAAAAAADNBOFCAA|2085-02-21|2221|9661|741|2085|3|2|21|1|2085|741|9661|Wednesday|2085Q1|N|N|N|2482623|2482653|2482277|2482551|N|N|N|N|N| +2482644|AAAAAAAAENBOFCAA|2085-02-22|2221|9661|741|2085|4|2|22|1|2085|741|9661|Thursday|2085Q1|N|N|N|2482623|2482653|2482278|2482552|N|N|N|N|N| +2482645|AAAAAAAAFNBOFCAA|2085-02-23|2221|9661|741|2085|5|2|23|1|2085|741|9661|Friday|2085Q1|N|Y|N|2482623|2482653|2482279|2482553|N|N|N|N|N| +2482646|AAAAAAAAGNBOFCAA|2085-02-24|2221|9661|741|2085|6|2|24|1|2085|741|9661|Saturday|2085Q1|N|Y|N|2482623|2482653|2482280|2482554|N|N|N|N|N| +2482647|AAAAAAAAHNBOFCAA|2085-02-25|2221|9661|741|2085|0|2|25|1|2085|741|9661|Sunday|2085Q1|N|N|N|2482623|2482653|2482281|2482555|N|N|N|N|N| +2482648|AAAAAAAAINBOFCAA|2085-02-26|2221|9661|741|2085|1|2|26|1|2085|741|9661|Monday|2085Q1|N|N|N|2482623|2482653|2482282|2482556|N|N|N|N|N| +2482649|AAAAAAAAJNBOFCAA|2085-02-27|2221|9662|741|2085|2|2|27|1|2085|741|9662|Tuesday|2085Q1|N|N|N|2482623|2482653|2482283|2482557|N|N|N|N|N| +2482650|AAAAAAAAKNBOFCAA|2085-02-28|2221|9662|741|2085|3|2|28|1|2085|741|9662|Wednesday|2085Q1|N|N|N|2482623|2482653|2482284|2482558|N|N|N|N|N| +2482651|AAAAAAAALNBOFCAA|2085-03-01|2222|9662|742|2085|4|3|1|1|2085|742|9662|Thursday|2085Q1|N|N|N|2482651|2482709|2482286|2482559|N|N|N|N|N| +2482652|AAAAAAAAMNBOFCAA|2085-03-02|2222|9662|742|2085|5|3|2|1|2085|742|9662|Friday|2085Q1|N|Y|N|2482651|2482709|2482287|2482560|N|N|N|N|N| +2482653|AAAAAAAANNBOFCAA|2085-03-03|2222|9662|742|2085|6|3|3|1|2085|742|9662|Saturday|2085Q1|N|Y|N|2482651|2482709|2482288|2482561|N|N|N|N|N| +2482654|AAAAAAAAONBOFCAA|2085-03-04|2222|9662|742|2085|0|3|4|1|2085|742|9662|Sunday|2085Q1|N|N|N|2482651|2482709|2482289|2482562|N|N|N|N|N| +2482655|AAAAAAAAPNBOFCAA|2085-03-05|2222|9662|742|2085|1|3|5|1|2085|742|9662|Monday|2085Q1|N|N|N|2482651|2482709|2482290|2482563|N|N|N|N|N| +2482656|AAAAAAAAAOBOFCAA|2085-03-06|2222|9663|742|2085|2|3|6|1|2085|742|9663|Tuesday|2085Q1|N|N|N|2482651|2482709|2482291|2482564|N|N|N|N|N| +2482657|AAAAAAAABOBOFCAA|2085-03-07|2222|9663|742|2085|3|3|7|1|2085|742|9663|Wednesday|2085Q1|N|N|N|2482651|2482709|2482292|2482565|N|N|N|N|N| +2482658|AAAAAAAACOBOFCAA|2085-03-08|2222|9663|742|2085|4|3|8|1|2085|742|9663|Thursday|2085Q1|N|N|N|2482651|2482709|2482293|2482566|N|N|N|N|N| +2482659|AAAAAAAADOBOFCAA|2085-03-09|2222|9663|742|2085|5|3|9|1|2085|742|9663|Friday|2085Q1|N|Y|N|2482651|2482709|2482294|2482567|N|N|N|N|N| +2482660|AAAAAAAAEOBOFCAA|2085-03-10|2222|9663|742|2085|6|3|10|1|2085|742|9663|Saturday|2085Q1|N|Y|N|2482651|2482709|2482295|2482568|N|N|N|N|N| +2482661|AAAAAAAAFOBOFCAA|2085-03-11|2222|9663|742|2085|0|3|11|1|2085|742|9663|Sunday|2085Q1|N|N|N|2482651|2482709|2482296|2482569|N|N|N|N|N| +2482662|AAAAAAAAGOBOFCAA|2085-03-12|2222|9663|742|2085|1|3|12|1|2085|742|9663|Monday|2085Q1|N|N|N|2482651|2482709|2482297|2482570|N|N|N|N|N| +2482663|AAAAAAAAHOBOFCAA|2085-03-13|2222|9664|742|2085|2|3|13|1|2085|742|9664|Tuesday|2085Q1|N|N|N|2482651|2482709|2482298|2482571|N|N|N|N|N| +2482664|AAAAAAAAIOBOFCAA|2085-03-14|2222|9664|742|2085|3|3|14|1|2085|742|9664|Wednesday|2085Q1|N|N|N|2482651|2482709|2482299|2482572|N|N|N|N|N| +2482665|AAAAAAAAJOBOFCAA|2085-03-15|2222|9664|742|2085|4|3|15|1|2085|742|9664|Thursday|2085Q1|N|N|N|2482651|2482709|2482300|2482573|N|N|N|N|N| +2482666|AAAAAAAAKOBOFCAA|2085-03-16|2222|9664|742|2085|5|3|16|1|2085|742|9664|Friday|2085Q1|N|Y|N|2482651|2482709|2482301|2482574|N|N|N|N|N| +2482667|AAAAAAAALOBOFCAA|2085-03-17|2222|9664|742|2085|6|3|17|1|2085|742|9664|Saturday|2085Q1|N|Y|N|2482651|2482709|2482302|2482575|N|N|N|N|N| +2482668|AAAAAAAAMOBOFCAA|2085-03-18|2222|9664|742|2085|0|3|18|1|2085|742|9664|Sunday|2085Q1|N|N|N|2482651|2482709|2482303|2482576|N|N|N|N|N| +2482669|AAAAAAAANOBOFCAA|2085-03-19|2222|9664|742|2085|1|3|19|1|2085|742|9664|Monday|2085Q1|N|N|N|2482651|2482709|2482304|2482577|N|N|N|N|N| +2482670|AAAAAAAAOOBOFCAA|2085-03-20|2222|9665|742|2085|2|3|20|1|2085|742|9665|Tuesday|2085Q1|N|N|N|2482651|2482709|2482305|2482578|N|N|N|N|N| +2482671|AAAAAAAAPOBOFCAA|2085-03-21|2222|9665|742|2085|3|3|21|1|2085|742|9665|Wednesday|2085Q1|N|N|N|2482651|2482709|2482306|2482579|N|N|N|N|N| +2482672|AAAAAAAAAPBOFCAA|2085-03-22|2222|9665|742|2085|4|3|22|1|2085|742|9665|Thursday|2085Q1|N|N|N|2482651|2482709|2482307|2482580|N|N|N|N|N| +2482673|AAAAAAAABPBOFCAA|2085-03-23|2222|9665|742|2085|5|3|23|1|2085|742|9665|Friday|2085Q1|N|Y|N|2482651|2482709|2482308|2482581|N|N|N|N|N| +2482674|AAAAAAAACPBOFCAA|2085-03-24|2222|9665|742|2085|6|3|24|1|2085|742|9665|Saturday|2085Q1|N|Y|N|2482651|2482709|2482309|2482582|N|N|N|N|N| +2482675|AAAAAAAADPBOFCAA|2085-03-25|2222|9665|742|2085|0|3|25|1|2085|742|9665|Sunday|2085Q1|N|N|N|2482651|2482709|2482310|2482583|N|N|N|N|N| +2482676|AAAAAAAAEPBOFCAA|2085-03-26|2222|9665|742|2085|1|3|26|1|2085|742|9665|Monday|2085Q1|N|N|N|2482651|2482709|2482311|2482584|N|N|N|N|N| +2482677|AAAAAAAAFPBOFCAA|2085-03-27|2222|9666|742|2085|2|3|27|1|2085|742|9666|Tuesday|2085Q1|N|N|N|2482651|2482709|2482312|2482585|N|N|N|N|N| +2482678|AAAAAAAAGPBOFCAA|2085-03-28|2222|9666|742|2085|3|3|28|1|2085|742|9666|Wednesday|2085Q1|N|N|N|2482651|2482709|2482313|2482586|N|N|N|N|N| +2482679|AAAAAAAAHPBOFCAA|2085-03-29|2222|9666|742|2085|4|3|29|1|2085|742|9666|Thursday|2085Q1|N|N|N|2482651|2482709|2482314|2482587|N|N|N|N|N| +2482680|AAAAAAAAIPBOFCAA|2085-03-30|2222|9666|742|2085|5|3|30|1|2085|742|9666|Friday|2085Q1|N|Y|N|2482651|2482709|2482315|2482588|N|N|N|N|N| +2482681|AAAAAAAAJPBOFCAA|2085-03-31|2222|9666|742|2085|6|3|31|1|2085|742|9666|Saturday|2085Q1|N|Y|N|2482651|2482709|2482316|2482589|N|N|N|N|N| +2482682|AAAAAAAAKPBOFCAA|2085-04-01|2223|9666|742|2085|0|4|1|1|2085|742|9666|Sunday|2085Q1|N|N|N|2482682|2482771|2482317|2482592|N|N|N|N|N| +2482683|AAAAAAAALPBOFCAA|2085-04-02|2223|9666|742|2085|1|4|2|2|2085|742|9666|Monday|2085Q2|N|N|N|2482682|2482771|2482318|2482593|N|N|N|N|N| +2482684|AAAAAAAAMPBOFCAA|2085-04-03|2223|9667|742|2085|2|4|3|2|2085|742|9667|Tuesday|2085Q2|N|N|N|2482682|2482771|2482319|2482594|N|N|N|N|N| +2482685|AAAAAAAANPBOFCAA|2085-04-04|2223|9667|742|2085|3|4|4|2|2085|742|9667|Wednesday|2085Q2|N|N|N|2482682|2482771|2482320|2482595|N|N|N|N|N| +2482686|AAAAAAAAOPBOFCAA|2085-04-05|2223|9667|742|2085|4|4|5|2|2085|742|9667|Thursday|2085Q2|N|N|N|2482682|2482771|2482321|2482596|N|N|N|N|N| +2482687|AAAAAAAAPPBOFCAA|2085-04-06|2223|9667|742|2085|5|4|6|2|2085|742|9667|Friday|2085Q2|N|Y|N|2482682|2482771|2482322|2482597|N|N|N|N|N| +2482688|AAAAAAAAAACOFCAA|2085-04-07|2223|9667|742|2085|6|4|7|2|2085|742|9667|Saturday|2085Q2|N|Y|N|2482682|2482771|2482323|2482598|N|N|N|N|N| +2482689|AAAAAAAABACOFCAA|2085-04-08|2223|9667|742|2085|0|4|8|2|2085|742|9667|Sunday|2085Q2|N|N|N|2482682|2482771|2482324|2482599|N|N|N|N|N| +2482690|AAAAAAAACACOFCAA|2085-04-09|2223|9667|742|2085|1|4|9|2|2085|742|9667|Monday|2085Q2|N|N|N|2482682|2482771|2482325|2482600|N|N|N|N|N| +2482691|AAAAAAAADACOFCAA|2085-04-10|2223|9668|742|2085|2|4|10|2|2085|742|9668|Tuesday|2085Q2|N|N|N|2482682|2482771|2482326|2482601|N|N|N|N|N| +2482692|AAAAAAAAEACOFCAA|2085-04-11|2223|9668|742|2085|3|4|11|2|2085|742|9668|Wednesday|2085Q2|N|N|N|2482682|2482771|2482327|2482602|N|N|N|N|N| +2482693|AAAAAAAAFACOFCAA|2085-04-12|2223|9668|742|2085|4|4|12|2|2085|742|9668|Thursday|2085Q2|N|N|N|2482682|2482771|2482328|2482603|N|N|N|N|N| +2482694|AAAAAAAAGACOFCAA|2085-04-13|2223|9668|742|2085|5|4|13|2|2085|742|9668|Friday|2085Q2|N|Y|N|2482682|2482771|2482329|2482604|N|N|N|N|N| +2482695|AAAAAAAAHACOFCAA|2085-04-14|2223|9668|742|2085|6|4|14|2|2085|742|9668|Saturday|2085Q2|N|Y|N|2482682|2482771|2482330|2482605|N|N|N|N|N| +2482696|AAAAAAAAIACOFCAA|2085-04-15|2223|9668|742|2085|0|4|15|2|2085|742|9668|Sunday|2085Q2|N|N|N|2482682|2482771|2482331|2482606|N|N|N|N|N| +2482697|AAAAAAAAJACOFCAA|2085-04-16|2223|9668|742|2085|1|4|16|2|2085|742|9668|Monday|2085Q2|N|N|N|2482682|2482771|2482332|2482607|N|N|N|N|N| +2482698|AAAAAAAAKACOFCAA|2085-04-17|2223|9669|742|2085|2|4|17|2|2085|742|9669|Tuesday|2085Q2|N|N|N|2482682|2482771|2482333|2482608|N|N|N|N|N| +2482699|AAAAAAAALACOFCAA|2085-04-18|2223|9669|742|2085|3|4|18|2|2085|742|9669|Wednesday|2085Q2|N|N|N|2482682|2482771|2482334|2482609|N|N|N|N|N| +2482700|AAAAAAAAMACOFCAA|2085-04-19|2223|9669|742|2085|4|4|19|2|2085|742|9669|Thursday|2085Q2|N|N|N|2482682|2482771|2482335|2482610|N|N|N|N|N| +2482701|AAAAAAAANACOFCAA|2085-04-20|2223|9669|742|2085|5|4|20|2|2085|742|9669|Friday|2085Q2|N|Y|N|2482682|2482771|2482336|2482611|N|N|N|N|N| +2482702|AAAAAAAAOACOFCAA|2085-04-21|2223|9669|742|2085|6|4|21|2|2085|742|9669|Saturday|2085Q2|N|Y|N|2482682|2482771|2482337|2482612|N|N|N|N|N| +2482703|AAAAAAAAPACOFCAA|2085-04-22|2223|9669|742|2085|0|4|22|2|2085|742|9669|Sunday|2085Q2|N|N|N|2482682|2482771|2482338|2482613|N|N|N|N|N| +2482704|AAAAAAAAABCOFCAA|2085-04-23|2223|9669|742|2085|1|4|23|2|2085|742|9669|Monday|2085Q2|N|N|N|2482682|2482771|2482339|2482614|N|N|N|N|N| +2482705|AAAAAAAABBCOFCAA|2085-04-24|2223|9670|742|2085|2|4|24|2|2085|742|9670|Tuesday|2085Q2|N|N|N|2482682|2482771|2482340|2482615|N|N|N|N|N| +2482706|AAAAAAAACBCOFCAA|2085-04-25|2223|9670|742|2085|3|4|25|2|2085|742|9670|Wednesday|2085Q2|N|N|N|2482682|2482771|2482341|2482616|N|N|N|N|N| +2482707|AAAAAAAADBCOFCAA|2085-04-26|2223|9670|742|2085|4|4|26|2|2085|742|9670|Thursday|2085Q2|N|N|N|2482682|2482771|2482342|2482617|N|N|N|N|N| +2482708|AAAAAAAAEBCOFCAA|2085-04-27|2223|9670|742|2085|5|4|27|2|2085|742|9670|Friday|2085Q2|N|Y|N|2482682|2482771|2482343|2482618|N|N|N|N|N| +2482709|AAAAAAAAFBCOFCAA|2085-04-28|2223|9670|742|2085|6|4|28|2|2085|742|9670|Saturday|2085Q2|N|Y|N|2482682|2482771|2482344|2482619|N|N|N|N|N| +2482710|AAAAAAAAGBCOFCAA|2085-04-29|2223|9670|742|2085|0|4|29|2|2085|742|9670|Sunday|2085Q2|N|N|N|2482682|2482771|2482345|2482620|N|N|N|N|N| +2482711|AAAAAAAAHBCOFCAA|2085-04-30|2223|9670|742|2085|1|4|30|2|2085|742|9670|Monday|2085Q2|N|N|N|2482682|2482771|2482346|2482621|N|N|N|N|N| +2482712|AAAAAAAAIBCOFCAA|2085-05-01|2224|9671|742|2085|2|5|1|2|2085|742|9671|Tuesday|2085Q2|N|N|N|2482712|2482831|2482347|2482622|N|N|N|N|N| +2482713|AAAAAAAAJBCOFCAA|2085-05-02|2224|9671|742|2085|3|5|2|2|2085|742|9671|Wednesday|2085Q2|N|N|N|2482712|2482831|2482348|2482623|N|N|N|N|N| +2482714|AAAAAAAAKBCOFCAA|2085-05-03|2224|9671|742|2085|4|5|3|2|2085|742|9671|Thursday|2085Q2|N|N|N|2482712|2482831|2482349|2482624|N|N|N|N|N| +2482715|AAAAAAAALBCOFCAA|2085-05-04|2224|9671|742|2085|5|5|4|2|2085|742|9671|Friday|2085Q2|N|Y|N|2482712|2482831|2482350|2482625|N|N|N|N|N| +2482716|AAAAAAAAMBCOFCAA|2085-05-05|2224|9671|742|2085|6|5|5|2|2085|742|9671|Saturday|2085Q2|N|Y|N|2482712|2482831|2482351|2482626|N|N|N|N|N| +2482717|AAAAAAAANBCOFCAA|2085-05-06|2224|9671|742|2085|0|5|6|2|2085|742|9671|Sunday|2085Q2|N|N|N|2482712|2482831|2482352|2482627|N|N|N|N|N| +2482718|AAAAAAAAOBCOFCAA|2085-05-07|2224|9671|742|2085|1|5|7|2|2085|742|9671|Monday|2085Q2|N|N|N|2482712|2482831|2482353|2482628|N|N|N|N|N| +2482719|AAAAAAAAPBCOFCAA|2085-05-08|2224|9672|742|2085|2|5|8|2|2085|742|9672|Tuesday|2085Q2|N|N|N|2482712|2482831|2482354|2482629|N|N|N|N|N| +2482720|AAAAAAAAACCOFCAA|2085-05-09|2224|9672|742|2085|3|5|9|2|2085|742|9672|Wednesday|2085Q2|N|N|N|2482712|2482831|2482355|2482630|N|N|N|N|N| +2482721|AAAAAAAABCCOFCAA|2085-05-10|2224|9672|742|2085|4|5|10|2|2085|742|9672|Thursday|2085Q2|N|N|N|2482712|2482831|2482356|2482631|N|N|N|N|N| +2482722|AAAAAAAACCCOFCAA|2085-05-11|2224|9672|742|2085|5|5|11|2|2085|742|9672|Friday|2085Q2|N|Y|N|2482712|2482831|2482357|2482632|N|N|N|N|N| +2482723|AAAAAAAADCCOFCAA|2085-05-12|2224|9672|742|2085|6|5|12|2|2085|742|9672|Saturday|2085Q2|N|Y|N|2482712|2482831|2482358|2482633|N|N|N|N|N| +2482724|AAAAAAAAECCOFCAA|2085-05-13|2224|9672|742|2085|0|5|13|2|2085|742|9672|Sunday|2085Q2|N|N|N|2482712|2482831|2482359|2482634|N|N|N|N|N| +2482725|AAAAAAAAFCCOFCAA|2085-05-14|2224|9672|742|2085|1|5|14|2|2085|742|9672|Monday|2085Q2|N|N|N|2482712|2482831|2482360|2482635|N|N|N|N|N| +2482726|AAAAAAAAGCCOFCAA|2085-05-15|2224|9673|742|2085|2|5|15|2|2085|742|9673|Tuesday|2085Q2|N|N|N|2482712|2482831|2482361|2482636|N|N|N|N|N| +2482727|AAAAAAAAHCCOFCAA|2085-05-16|2224|9673|742|2085|3|5|16|2|2085|742|9673|Wednesday|2085Q2|N|N|N|2482712|2482831|2482362|2482637|N|N|N|N|N| +2482728|AAAAAAAAICCOFCAA|2085-05-17|2224|9673|742|2085|4|5|17|2|2085|742|9673|Thursday|2085Q2|N|N|N|2482712|2482831|2482363|2482638|N|N|N|N|N| +2482729|AAAAAAAAJCCOFCAA|2085-05-18|2224|9673|742|2085|5|5|18|2|2085|742|9673|Friday|2085Q2|N|Y|N|2482712|2482831|2482364|2482639|N|N|N|N|N| +2482730|AAAAAAAAKCCOFCAA|2085-05-19|2224|9673|742|2085|6|5|19|2|2085|742|9673|Saturday|2085Q2|N|Y|N|2482712|2482831|2482365|2482640|N|N|N|N|N| +2482731|AAAAAAAALCCOFCAA|2085-05-20|2224|9673|742|2085|0|5|20|2|2085|742|9673|Sunday|2085Q2|N|N|N|2482712|2482831|2482366|2482641|N|N|N|N|N| +2482732|AAAAAAAAMCCOFCAA|2085-05-21|2224|9673|742|2085|1|5|21|2|2085|742|9673|Monday|2085Q2|N|N|N|2482712|2482831|2482367|2482642|N|N|N|N|N| +2482733|AAAAAAAANCCOFCAA|2085-05-22|2224|9674|742|2085|2|5|22|2|2085|742|9674|Tuesday|2085Q2|N|N|N|2482712|2482831|2482368|2482643|N|N|N|N|N| +2482734|AAAAAAAAOCCOFCAA|2085-05-23|2224|9674|742|2085|3|5|23|2|2085|742|9674|Wednesday|2085Q2|N|N|N|2482712|2482831|2482369|2482644|N|N|N|N|N| +2482735|AAAAAAAAPCCOFCAA|2085-05-24|2224|9674|742|2085|4|5|24|2|2085|742|9674|Thursday|2085Q2|N|N|N|2482712|2482831|2482370|2482645|N|N|N|N|N| +2482736|AAAAAAAAADCOFCAA|2085-05-25|2224|9674|742|2085|5|5|25|2|2085|742|9674|Friday|2085Q2|N|Y|N|2482712|2482831|2482371|2482646|N|N|N|N|N| +2482737|AAAAAAAABDCOFCAA|2085-05-26|2224|9674|742|2085|6|5|26|2|2085|742|9674|Saturday|2085Q2|N|Y|N|2482712|2482831|2482372|2482647|N|N|N|N|N| +2482738|AAAAAAAACDCOFCAA|2085-05-27|2224|9674|742|2085|0|5|27|2|2085|742|9674|Sunday|2085Q2|N|N|N|2482712|2482831|2482373|2482648|N|N|N|N|N| +2482739|AAAAAAAADDCOFCAA|2085-05-28|2224|9674|742|2085|1|5|28|2|2085|742|9674|Monday|2085Q2|N|N|N|2482712|2482831|2482374|2482649|N|N|N|N|N| +2482740|AAAAAAAAEDCOFCAA|2085-05-29|2224|9675|742|2085|2|5|29|2|2085|742|9675|Tuesday|2085Q2|N|N|N|2482712|2482831|2482375|2482650|N|N|N|N|N| +2482741|AAAAAAAAFDCOFCAA|2085-05-30|2224|9675|742|2085|3|5|30|2|2085|742|9675|Wednesday|2085Q2|N|N|N|2482712|2482831|2482376|2482651|N|N|N|N|N| +2482742|AAAAAAAAGDCOFCAA|2085-05-31|2224|9675|742|2085|4|5|31|2|2085|742|9675|Thursday|2085Q2|N|N|N|2482712|2482831|2482377|2482652|N|N|N|N|N| +2482743|AAAAAAAAHDCOFCAA|2085-06-01|2225|9675|743|2085|5|6|1|2|2085|743|9675|Friday|2085Q2|N|Y|N|2482743|2482893|2482378|2482653|N|N|N|N|N| +2482744|AAAAAAAAIDCOFCAA|2085-06-02|2225|9675|743|2085|6|6|2|2|2085|743|9675|Saturday|2085Q2|N|Y|N|2482743|2482893|2482379|2482654|N|N|N|N|N| +2482745|AAAAAAAAJDCOFCAA|2085-06-03|2225|9675|743|2085|0|6|3|2|2085|743|9675|Sunday|2085Q2|N|N|N|2482743|2482893|2482380|2482655|N|N|N|N|N| +2482746|AAAAAAAAKDCOFCAA|2085-06-04|2225|9675|743|2085|1|6|4|2|2085|743|9675|Monday|2085Q2|N|N|N|2482743|2482893|2482381|2482656|N|N|N|N|N| +2482747|AAAAAAAALDCOFCAA|2085-06-05|2225|9676|743|2085|2|6|5|2|2085|743|9676|Tuesday|2085Q2|N|N|N|2482743|2482893|2482382|2482657|N|N|N|N|N| +2482748|AAAAAAAAMDCOFCAA|2085-06-06|2225|9676|743|2085|3|6|6|2|2085|743|9676|Wednesday|2085Q2|N|N|N|2482743|2482893|2482383|2482658|N|N|N|N|N| +2482749|AAAAAAAANDCOFCAA|2085-06-07|2225|9676|743|2085|4|6|7|2|2085|743|9676|Thursday|2085Q2|N|N|N|2482743|2482893|2482384|2482659|N|N|N|N|N| +2482750|AAAAAAAAODCOFCAA|2085-06-08|2225|9676|743|2085|5|6|8|2|2085|743|9676|Friday|2085Q2|N|Y|N|2482743|2482893|2482385|2482660|N|N|N|N|N| +2482751|AAAAAAAAPDCOFCAA|2085-06-09|2225|9676|743|2085|6|6|9|2|2085|743|9676|Saturday|2085Q2|N|Y|N|2482743|2482893|2482386|2482661|N|N|N|N|N| +2482752|AAAAAAAAAECOFCAA|2085-06-10|2225|9676|743|2085|0|6|10|2|2085|743|9676|Sunday|2085Q2|N|N|N|2482743|2482893|2482387|2482662|N|N|N|N|N| +2482753|AAAAAAAABECOFCAA|2085-06-11|2225|9676|743|2085|1|6|11|2|2085|743|9676|Monday|2085Q2|N|N|N|2482743|2482893|2482388|2482663|N|N|N|N|N| +2482754|AAAAAAAACECOFCAA|2085-06-12|2225|9677|743|2085|2|6|12|2|2085|743|9677|Tuesday|2085Q2|N|N|N|2482743|2482893|2482389|2482664|N|N|N|N|N| +2482755|AAAAAAAADECOFCAA|2085-06-13|2225|9677|743|2085|3|6|13|2|2085|743|9677|Wednesday|2085Q2|N|N|N|2482743|2482893|2482390|2482665|N|N|N|N|N| +2482756|AAAAAAAAEECOFCAA|2085-06-14|2225|9677|743|2085|4|6|14|2|2085|743|9677|Thursday|2085Q2|N|N|N|2482743|2482893|2482391|2482666|N|N|N|N|N| +2482757|AAAAAAAAFECOFCAA|2085-06-15|2225|9677|743|2085|5|6|15|2|2085|743|9677|Friday|2085Q2|N|Y|N|2482743|2482893|2482392|2482667|N|N|N|N|N| +2482758|AAAAAAAAGECOFCAA|2085-06-16|2225|9677|743|2085|6|6|16|2|2085|743|9677|Saturday|2085Q2|N|Y|N|2482743|2482893|2482393|2482668|N|N|N|N|N| +2482759|AAAAAAAAHECOFCAA|2085-06-17|2225|9677|743|2085|0|6|17|2|2085|743|9677|Sunday|2085Q2|N|N|N|2482743|2482893|2482394|2482669|N|N|N|N|N| +2482760|AAAAAAAAIECOFCAA|2085-06-18|2225|9677|743|2085|1|6|18|2|2085|743|9677|Monday|2085Q2|N|N|N|2482743|2482893|2482395|2482670|N|N|N|N|N| +2482761|AAAAAAAAJECOFCAA|2085-06-19|2225|9678|743|2085|2|6|19|2|2085|743|9678|Tuesday|2085Q2|N|N|N|2482743|2482893|2482396|2482671|N|N|N|N|N| +2482762|AAAAAAAAKECOFCAA|2085-06-20|2225|9678|743|2085|3|6|20|2|2085|743|9678|Wednesday|2085Q2|N|N|N|2482743|2482893|2482397|2482672|N|N|N|N|N| +2482763|AAAAAAAALECOFCAA|2085-06-21|2225|9678|743|2085|4|6|21|2|2085|743|9678|Thursday|2085Q2|N|N|N|2482743|2482893|2482398|2482673|N|N|N|N|N| +2482764|AAAAAAAAMECOFCAA|2085-06-22|2225|9678|743|2085|5|6|22|2|2085|743|9678|Friday|2085Q2|N|Y|N|2482743|2482893|2482399|2482674|N|N|N|N|N| +2482765|AAAAAAAANECOFCAA|2085-06-23|2225|9678|743|2085|6|6|23|2|2085|743|9678|Saturday|2085Q2|N|Y|N|2482743|2482893|2482400|2482675|N|N|N|N|N| +2482766|AAAAAAAAOECOFCAA|2085-06-24|2225|9678|743|2085|0|6|24|2|2085|743|9678|Sunday|2085Q2|N|N|N|2482743|2482893|2482401|2482676|N|N|N|N|N| +2482767|AAAAAAAAPECOFCAA|2085-06-25|2225|9678|743|2085|1|6|25|2|2085|743|9678|Monday|2085Q2|N|N|N|2482743|2482893|2482402|2482677|N|N|N|N|N| +2482768|AAAAAAAAAFCOFCAA|2085-06-26|2225|9679|743|2085|2|6|26|2|2085|743|9679|Tuesday|2085Q2|N|N|N|2482743|2482893|2482403|2482678|N|N|N|N|N| +2482769|AAAAAAAABFCOFCAA|2085-06-27|2225|9679|743|2085|3|6|27|2|2085|743|9679|Wednesday|2085Q2|N|N|N|2482743|2482893|2482404|2482679|N|N|N|N|N| +2482770|AAAAAAAACFCOFCAA|2085-06-28|2225|9679|743|2085|4|6|28|2|2085|743|9679|Thursday|2085Q2|N|N|N|2482743|2482893|2482405|2482680|N|N|N|N|N| +2482771|AAAAAAAADFCOFCAA|2085-06-29|2225|9679|743|2085|5|6|29|2|2085|743|9679|Friday|2085Q2|N|Y|N|2482743|2482893|2482406|2482681|N|N|N|N|N| +2482772|AAAAAAAAEFCOFCAA|2085-06-30|2225|9679|743|2085|6|6|30|2|2085|743|9679|Saturday|2085Q2|N|Y|N|2482743|2482893|2482407|2482682|N|N|N|N|N| +2482773|AAAAAAAAFFCOFCAA|2085-07-01|2226|9679|743|2085|0|7|1|2|2085|743|9679|Sunday|2085Q2|N|N|N|2482773|2482953|2482408|2482682|N|N|N|N|N| +2482774|AAAAAAAAGFCOFCAA|2085-07-02|2226|9679|743|2085|1|7|2|3|2085|743|9679|Monday|2085Q3|N|N|N|2482773|2482953|2482409|2482683|N|N|N|N|N| +2482775|AAAAAAAAHFCOFCAA|2085-07-03|2226|9680|743|2085|2|7|3|3|2085|743|9680|Tuesday|2085Q3|N|N|N|2482773|2482953|2482410|2482684|N|N|N|N|N| +2482776|AAAAAAAAIFCOFCAA|2085-07-04|2226|9680|743|2085|3|7|4|3|2085|743|9680|Wednesday|2085Q3|N|N|N|2482773|2482953|2482411|2482685|N|N|N|N|N| +2482777|AAAAAAAAJFCOFCAA|2085-07-05|2226|9680|743|2085|4|7|5|3|2085|743|9680|Thursday|2085Q3|Y|N|N|2482773|2482953|2482412|2482686|N|N|N|N|N| +2482778|AAAAAAAAKFCOFCAA|2085-07-06|2226|9680|743|2085|5|7|6|3|2085|743|9680|Friday|2085Q3|N|Y|Y|2482773|2482953|2482413|2482687|N|N|N|N|N| +2482779|AAAAAAAALFCOFCAA|2085-07-07|2226|9680|743|2085|6|7|7|3|2085|743|9680|Saturday|2085Q3|N|Y|N|2482773|2482953|2482414|2482688|N|N|N|N|N| +2482780|AAAAAAAAMFCOFCAA|2085-07-08|2226|9680|743|2085|0|7|8|3|2085|743|9680|Sunday|2085Q3|N|N|N|2482773|2482953|2482415|2482689|N|N|N|N|N| +2482781|AAAAAAAANFCOFCAA|2085-07-09|2226|9680|743|2085|1|7|9|3|2085|743|9680|Monday|2085Q3|N|N|N|2482773|2482953|2482416|2482690|N|N|N|N|N| +2482782|AAAAAAAAOFCOFCAA|2085-07-10|2226|9681|743|2085|2|7|10|3|2085|743|9681|Tuesday|2085Q3|N|N|N|2482773|2482953|2482417|2482691|N|N|N|N|N| +2482783|AAAAAAAAPFCOFCAA|2085-07-11|2226|9681|743|2085|3|7|11|3|2085|743|9681|Wednesday|2085Q3|N|N|N|2482773|2482953|2482418|2482692|N|N|N|N|N| +2482784|AAAAAAAAAGCOFCAA|2085-07-12|2226|9681|743|2085|4|7|12|3|2085|743|9681|Thursday|2085Q3|N|N|N|2482773|2482953|2482419|2482693|N|N|N|N|N| +2482785|AAAAAAAABGCOFCAA|2085-07-13|2226|9681|743|2085|5|7|13|3|2085|743|9681|Friday|2085Q3|N|Y|N|2482773|2482953|2482420|2482694|N|N|N|N|N| +2482786|AAAAAAAACGCOFCAA|2085-07-14|2226|9681|743|2085|6|7|14|3|2085|743|9681|Saturday|2085Q3|N|Y|N|2482773|2482953|2482421|2482695|N|N|N|N|N| +2482787|AAAAAAAADGCOFCAA|2085-07-15|2226|9681|743|2085|0|7|15|3|2085|743|9681|Sunday|2085Q3|N|N|N|2482773|2482953|2482422|2482696|N|N|N|N|N| +2482788|AAAAAAAAEGCOFCAA|2085-07-16|2226|9681|743|2085|1|7|16|3|2085|743|9681|Monday|2085Q3|N|N|N|2482773|2482953|2482423|2482697|N|N|N|N|N| +2482789|AAAAAAAAFGCOFCAA|2085-07-17|2226|9682|743|2085|2|7|17|3|2085|743|9682|Tuesday|2085Q3|N|N|N|2482773|2482953|2482424|2482698|N|N|N|N|N| +2482790|AAAAAAAAGGCOFCAA|2085-07-18|2226|9682|743|2085|3|7|18|3|2085|743|9682|Wednesday|2085Q3|N|N|N|2482773|2482953|2482425|2482699|N|N|N|N|N| +2482791|AAAAAAAAHGCOFCAA|2085-07-19|2226|9682|743|2085|4|7|19|3|2085|743|9682|Thursday|2085Q3|N|N|N|2482773|2482953|2482426|2482700|N|N|N|N|N| +2482792|AAAAAAAAIGCOFCAA|2085-07-20|2226|9682|743|2085|5|7|20|3|2085|743|9682|Friday|2085Q3|N|Y|N|2482773|2482953|2482427|2482701|N|N|N|N|N| +2482793|AAAAAAAAJGCOFCAA|2085-07-21|2226|9682|743|2085|6|7|21|3|2085|743|9682|Saturday|2085Q3|N|Y|N|2482773|2482953|2482428|2482702|N|N|N|N|N| +2482794|AAAAAAAAKGCOFCAA|2085-07-22|2226|9682|743|2085|0|7|22|3|2085|743|9682|Sunday|2085Q3|N|N|N|2482773|2482953|2482429|2482703|N|N|N|N|N| +2482795|AAAAAAAALGCOFCAA|2085-07-23|2226|9682|743|2085|1|7|23|3|2085|743|9682|Monday|2085Q3|N|N|N|2482773|2482953|2482430|2482704|N|N|N|N|N| +2482796|AAAAAAAAMGCOFCAA|2085-07-24|2226|9683|743|2085|2|7|24|3|2085|743|9683|Tuesday|2085Q3|N|N|N|2482773|2482953|2482431|2482705|N|N|N|N|N| +2482797|AAAAAAAANGCOFCAA|2085-07-25|2226|9683|743|2085|3|7|25|3|2085|743|9683|Wednesday|2085Q3|N|N|N|2482773|2482953|2482432|2482706|N|N|N|N|N| +2482798|AAAAAAAAOGCOFCAA|2085-07-26|2226|9683|743|2085|4|7|26|3|2085|743|9683|Thursday|2085Q3|N|N|N|2482773|2482953|2482433|2482707|N|N|N|N|N| +2482799|AAAAAAAAPGCOFCAA|2085-07-27|2226|9683|743|2085|5|7|27|3|2085|743|9683|Friday|2085Q3|N|Y|N|2482773|2482953|2482434|2482708|N|N|N|N|N| +2482800|AAAAAAAAAHCOFCAA|2085-07-28|2226|9683|743|2085|6|7|28|3|2085|743|9683|Saturday|2085Q3|N|Y|N|2482773|2482953|2482435|2482709|N|N|N|N|N| +2482801|AAAAAAAABHCOFCAA|2085-07-29|2226|9683|743|2085|0|7|29|3|2085|743|9683|Sunday|2085Q3|N|N|N|2482773|2482953|2482436|2482710|N|N|N|N|N| +2482802|AAAAAAAACHCOFCAA|2085-07-30|2226|9683|743|2085|1|7|30|3|2085|743|9683|Monday|2085Q3|N|N|N|2482773|2482953|2482437|2482711|N|N|N|N|N| +2482803|AAAAAAAADHCOFCAA|2085-07-31|2226|9684|743|2085|2|7|31|3|2085|743|9684|Tuesday|2085Q3|N|N|N|2482773|2482953|2482438|2482712|N|N|N|N|N| +2482804|AAAAAAAAEHCOFCAA|2085-08-01|2227|9684|743|2085|3|8|1|3|2085|743|9684|Wednesday|2085Q3|N|N|N|2482804|2483015|2482439|2482713|N|N|N|N|N| +2482805|AAAAAAAAFHCOFCAA|2085-08-02|2227|9684|743|2085|4|8|2|3|2085|743|9684|Thursday|2085Q3|N|N|N|2482804|2483015|2482440|2482714|N|N|N|N|N| +2482806|AAAAAAAAGHCOFCAA|2085-08-03|2227|9684|743|2085|5|8|3|3|2085|743|9684|Friday|2085Q3|N|Y|N|2482804|2483015|2482441|2482715|N|N|N|N|N| +2482807|AAAAAAAAHHCOFCAA|2085-08-04|2227|9684|743|2085|6|8|4|3|2085|743|9684|Saturday|2085Q3|N|Y|N|2482804|2483015|2482442|2482716|N|N|N|N|N| +2482808|AAAAAAAAIHCOFCAA|2085-08-05|2227|9684|743|2085|0|8|5|3|2085|743|9684|Sunday|2085Q3|N|N|N|2482804|2483015|2482443|2482717|N|N|N|N|N| +2482809|AAAAAAAAJHCOFCAA|2085-08-06|2227|9684|743|2085|1|8|6|3|2085|743|9684|Monday|2085Q3|N|N|N|2482804|2483015|2482444|2482718|N|N|N|N|N| +2482810|AAAAAAAAKHCOFCAA|2085-08-07|2227|9685|743|2085|2|8|7|3|2085|743|9685|Tuesday|2085Q3|N|N|N|2482804|2483015|2482445|2482719|N|N|N|N|N| +2482811|AAAAAAAALHCOFCAA|2085-08-08|2227|9685|743|2085|3|8|8|3|2085|743|9685|Wednesday|2085Q3|N|N|N|2482804|2483015|2482446|2482720|N|N|N|N|N| +2482812|AAAAAAAAMHCOFCAA|2085-08-09|2227|9685|743|2085|4|8|9|3|2085|743|9685|Thursday|2085Q3|N|N|N|2482804|2483015|2482447|2482721|N|N|N|N|N| +2482813|AAAAAAAANHCOFCAA|2085-08-10|2227|9685|743|2085|5|8|10|3|2085|743|9685|Friday|2085Q3|N|Y|N|2482804|2483015|2482448|2482722|N|N|N|N|N| +2482814|AAAAAAAAOHCOFCAA|2085-08-11|2227|9685|743|2085|6|8|11|3|2085|743|9685|Saturday|2085Q3|N|Y|N|2482804|2483015|2482449|2482723|N|N|N|N|N| +2482815|AAAAAAAAPHCOFCAA|2085-08-12|2227|9685|743|2085|0|8|12|3|2085|743|9685|Sunday|2085Q3|N|N|N|2482804|2483015|2482450|2482724|N|N|N|N|N| +2482816|AAAAAAAAAICOFCAA|2085-08-13|2227|9685|743|2085|1|8|13|3|2085|743|9685|Monday|2085Q3|N|N|N|2482804|2483015|2482451|2482725|N|N|N|N|N| +2482817|AAAAAAAABICOFCAA|2085-08-14|2227|9686|743|2085|2|8|14|3|2085|743|9686|Tuesday|2085Q3|N|N|N|2482804|2483015|2482452|2482726|N|N|N|N|N| +2482818|AAAAAAAACICOFCAA|2085-08-15|2227|9686|743|2085|3|8|15|3|2085|743|9686|Wednesday|2085Q3|N|N|N|2482804|2483015|2482453|2482727|N|N|N|N|N| +2482819|AAAAAAAADICOFCAA|2085-08-16|2227|9686|743|2085|4|8|16|3|2085|743|9686|Thursday|2085Q3|N|N|N|2482804|2483015|2482454|2482728|N|N|N|N|N| +2482820|AAAAAAAAEICOFCAA|2085-08-17|2227|9686|743|2085|5|8|17|3|2085|743|9686|Friday|2085Q3|N|Y|N|2482804|2483015|2482455|2482729|N|N|N|N|N| +2482821|AAAAAAAAFICOFCAA|2085-08-18|2227|9686|743|2085|6|8|18|3|2085|743|9686|Saturday|2085Q3|N|Y|N|2482804|2483015|2482456|2482730|N|N|N|N|N| +2482822|AAAAAAAAGICOFCAA|2085-08-19|2227|9686|743|2085|0|8|19|3|2085|743|9686|Sunday|2085Q3|N|N|N|2482804|2483015|2482457|2482731|N|N|N|N|N| +2482823|AAAAAAAAHICOFCAA|2085-08-20|2227|9686|743|2085|1|8|20|3|2085|743|9686|Monday|2085Q3|N|N|N|2482804|2483015|2482458|2482732|N|N|N|N|N| +2482824|AAAAAAAAIICOFCAA|2085-08-21|2227|9687|743|2085|2|8|21|3|2085|743|9687|Tuesday|2085Q3|N|N|N|2482804|2483015|2482459|2482733|N|N|N|N|N| +2482825|AAAAAAAAJICOFCAA|2085-08-22|2227|9687|743|2085|3|8|22|3|2085|743|9687|Wednesday|2085Q3|N|N|N|2482804|2483015|2482460|2482734|N|N|N|N|N| +2482826|AAAAAAAAKICOFCAA|2085-08-23|2227|9687|743|2085|4|8|23|3|2085|743|9687|Thursday|2085Q3|N|N|N|2482804|2483015|2482461|2482735|N|N|N|N|N| +2482827|AAAAAAAALICOFCAA|2085-08-24|2227|9687|743|2085|5|8|24|3|2085|743|9687|Friday|2085Q3|N|Y|N|2482804|2483015|2482462|2482736|N|N|N|N|N| +2482828|AAAAAAAAMICOFCAA|2085-08-25|2227|9687|743|2085|6|8|25|3|2085|743|9687|Saturday|2085Q3|N|Y|N|2482804|2483015|2482463|2482737|N|N|N|N|N| +2482829|AAAAAAAANICOFCAA|2085-08-26|2227|9687|743|2085|0|8|26|3|2085|743|9687|Sunday|2085Q3|N|N|N|2482804|2483015|2482464|2482738|N|N|N|N|N| +2482830|AAAAAAAAOICOFCAA|2085-08-27|2227|9687|743|2085|1|8|27|3|2085|743|9687|Monday|2085Q3|N|N|N|2482804|2483015|2482465|2482739|N|N|N|N|N| +2482831|AAAAAAAAPICOFCAA|2085-08-28|2227|9688|743|2085|2|8|28|3|2085|743|9688|Tuesday|2085Q3|N|N|N|2482804|2483015|2482466|2482740|N|N|N|N|N| +2482832|AAAAAAAAAJCOFCAA|2085-08-29|2227|9688|743|2085|3|8|29|3|2085|743|9688|Wednesday|2085Q3|N|N|N|2482804|2483015|2482467|2482741|N|N|N|N|N| +2482833|AAAAAAAABJCOFCAA|2085-08-30|2227|9688|743|2085|4|8|30|3|2085|743|9688|Thursday|2085Q3|N|N|N|2482804|2483015|2482468|2482742|N|N|N|N|N| +2482834|AAAAAAAACJCOFCAA|2085-08-31|2227|9688|743|2085|5|8|31|3|2085|743|9688|Friday|2085Q3|N|Y|N|2482804|2483015|2482469|2482743|N|N|N|N|N| +2482835|AAAAAAAADJCOFCAA|2085-09-01|2228|9688|744|2085|6|9|1|3|2085|744|9688|Saturday|2085Q3|N|Y|N|2482835|2483077|2482470|2482744|N|N|N|N|N| +2482836|AAAAAAAAEJCOFCAA|2085-09-02|2228|9688|744|2085|0|9|2|3|2085|744|9688|Sunday|2085Q3|N|N|N|2482835|2483077|2482471|2482745|N|N|N|N|N| +2482837|AAAAAAAAFJCOFCAA|2085-09-03|2228|9688|744|2085|1|9|3|3|2085|744|9688|Monday|2085Q3|N|N|N|2482835|2483077|2482472|2482746|N|N|N|N|N| +2482838|AAAAAAAAGJCOFCAA|2085-09-04|2228|9689|744|2085|2|9|4|3|2085|744|9689|Tuesday|2085Q3|N|N|N|2482835|2483077|2482473|2482747|N|N|N|N|N| +2482839|AAAAAAAAHJCOFCAA|2085-09-05|2228|9689|744|2085|3|9|5|3|2085|744|9689|Wednesday|2085Q3|N|N|N|2482835|2483077|2482474|2482748|N|N|N|N|N| +2482840|AAAAAAAAIJCOFCAA|2085-09-06|2228|9689|744|2085|4|9|6|3|2085|744|9689|Thursday|2085Q3|N|N|N|2482835|2483077|2482475|2482749|N|N|N|N|N| +2482841|AAAAAAAAJJCOFCAA|2085-09-07|2228|9689|744|2085|5|9|7|3|2085|744|9689|Friday|2085Q3|N|Y|N|2482835|2483077|2482476|2482750|N|N|N|N|N| +2482842|AAAAAAAAKJCOFCAA|2085-09-08|2228|9689|744|2085|6|9|8|3|2085|744|9689|Saturday|2085Q3|N|Y|N|2482835|2483077|2482477|2482751|N|N|N|N|N| +2482843|AAAAAAAALJCOFCAA|2085-09-09|2228|9689|744|2085|0|9|9|3|2085|744|9689|Sunday|2085Q3|N|N|N|2482835|2483077|2482478|2482752|N|N|N|N|N| +2482844|AAAAAAAAMJCOFCAA|2085-09-10|2228|9689|744|2085|1|9|10|3|2085|744|9689|Monday|2085Q3|N|N|N|2482835|2483077|2482479|2482753|N|N|N|N|N| +2482845|AAAAAAAANJCOFCAA|2085-09-11|2228|9690|744|2085|2|9|11|3|2085|744|9690|Tuesday|2085Q3|N|N|N|2482835|2483077|2482480|2482754|N|N|N|N|N| +2482846|AAAAAAAAOJCOFCAA|2085-09-12|2228|9690|744|2085|3|9|12|3|2085|744|9690|Wednesday|2085Q3|N|N|N|2482835|2483077|2482481|2482755|N|N|N|N|N| +2482847|AAAAAAAAPJCOFCAA|2085-09-13|2228|9690|744|2085|4|9|13|3|2085|744|9690|Thursday|2085Q3|N|N|N|2482835|2483077|2482482|2482756|N|N|N|N|N| +2482848|AAAAAAAAAKCOFCAA|2085-09-14|2228|9690|744|2085|5|9|14|3|2085|744|9690|Friday|2085Q3|N|Y|N|2482835|2483077|2482483|2482757|N|N|N|N|N| +2482849|AAAAAAAABKCOFCAA|2085-09-15|2228|9690|744|2085|6|9|15|3|2085|744|9690|Saturday|2085Q3|N|Y|N|2482835|2483077|2482484|2482758|N|N|N|N|N| +2482850|AAAAAAAACKCOFCAA|2085-09-16|2228|9690|744|2085|0|9|16|3|2085|744|9690|Sunday|2085Q3|N|N|N|2482835|2483077|2482485|2482759|N|N|N|N|N| +2482851|AAAAAAAADKCOFCAA|2085-09-17|2228|9690|744|2085|1|9|17|3|2085|744|9690|Monday|2085Q3|N|N|N|2482835|2483077|2482486|2482760|N|N|N|N|N| +2482852|AAAAAAAAEKCOFCAA|2085-09-18|2228|9691|744|2085|2|9|18|3|2085|744|9691|Tuesday|2085Q3|N|N|N|2482835|2483077|2482487|2482761|N|N|N|N|N| +2482853|AAAAAAAAFKCOFCAA|2085-09-19|2228|9691|744|2085|3|9|19|3|2085|744|9691|Wednesday|2085Q3|N|N|N|2482835|2483077|2482488|2482762|N|N|N|N|N| +2482854|AAAAAAAAGKCOFCAA|2085-09-20|2228|9691|744|2085|4|9|20|3|2085|744|9691|Thursday|2085Q3|N|N|N|2482835|2483077|2482489|2482763|N|N|N|N|N| +2482855|AAAAAAAAHKCOFCAA|2085-09-21|2228|9691|744|2085|5|9|21|3|2085|744|9691|Friday|2085Q3|N|Y|N|2482835|2483077|2482490|2482764|N|N|N|N|N| +2482856|AAAAAAAAIKCOFCAA|2085-09-22|2228|9691|744|2085|6|9|22|3|2085|744|9691|Saturday|2085Q3|N|Y|N|2482835|2483077|2482491|2482765|N|N|N|N|N| +2482857|AAAAAAAAJKCOFCAA|2085-09-23|2228|9691|744|2085|0|9|23|3|2085|744|9691|Sunday|2085Q3|N|N|N|2482835|2483077|2482492|2482766|N|N|N|N|N| +2482858|AAAAAAAAKKCOFCAA|2085-09-24|2228|9691|744|2085|1|9|24|3|2085|744|9691|Monday|2085Q3|N|N|N|2482835|2483077|2482493|2482767|N|N|N|N|N| +2482859|AAAAAAAALKCOFCAA|2085-09-25|2228|9692|744|2085|2|9|25|3|2085|744|9692|Tuesday|2085Q3|N|N|N|2482835|2483077|2482494|2482768|N|N|N|N|N| +2482860|AAAAAAAAMKCOFCAA|2085-09-26|2228|9692|744|2085|3|9|26|3|2085|744|9692|Wednesday|2085Q3|N|N|N|2482835|2483077|2482495|2482769|N|N|N|N|N| +2482861|AAAAAAAANKCOFCAA|2085-09-27|2228|9692|744|2085|4|9|27|3|2085|744|9692|Thursday|2085Q3|N|N|N|2482835|2483077|2482496|2482770|N|N|N|N|N| +2482862|AAAAAAAAOKCOFCAA|2085-09-28|2228|9692|744|2085|5|9|28|3|2085|744|9692|Friday|2085Q3|N|Y|N|2482835|2483077|2482497|2482771|N|N|N|N|N| +2482863|AAAAAAAAPKCOFCAA|2085-09-29|2228|9692|744|2085|6|9|29|3|2085|744|9692|Saturday|2085Q3|N|Y|N|2482835|2483077|2482498|2482772|N|N|N|N|N| +2482864|AAAAAAAAALCOFCAA|2085-09-30|2228|9692|744|2085|0|9|30|3|2085|744|9692|Sunday|2085Q3|N|N|N|2482835|2483077|2482499|2482773|N|N|N|N|N| +2482865|AAAAAAAABLCOFCAA|2085-10-01|2229|9692|744|2085|1|10|1|3|2085|744|9692|Monday|2085Q3|N|N|N|2482865|2483137|2482500|2482773|N|N|N|N|N| +2482866|AAAAAAAACLCOFCAA|2085-10-02|2229|9693|744|2085|2|10|2|4|2085|744|9693|Tuesday|2085Q4|N|N|N|2482865|2483137|2482501|2482774|N|N|N|N|N| +2482867|AAAAAAAADLCOFCAA|2085-10-03|2229|9693|744|2085|3|10|3|4|2085|744|9693|Wednesday|2085Q4|N|N|N|2482865|2483137|2482502|2482775|N|N|N|N|N| +2482868|AAAAAAAAELCOFCAA|2085-10-04|2229|9693|744|2085|4|10|4|4|2085|744|9693|Thursday|2085Q4|N|N|N|2482865|2483137|2482503|2482776|N|N|N|N|N| +2482869|AAAAAAAAFLCOFCAA|2085-10-05|2229|9693|744|2085|5|10|5|4|2085|744|9693|Friday|2085Q4|N|Y|N|2482865|2483137|2482504|2482777|N|N|N|N|N| +2482870|AAAAAAAAGLCOFCAA|2085-10-06|2229|9693|744|2085|6|10|6|4|2085|744|9693|Saturday|2085Q4|N|Y|N|2482865|2483137|2482505|2482778|N|N|N|N|N| +2482871|AAAAAAAAHLCOFCAA|2085-10-07|2229|9693|744|2085|0|10|7|4|2085|744|9693|Sunday|2085Q4|N|N|N|2482865|2483137|2482506|2482779|N|N|N|N|N| +2482872|AAAAAAAAILCOFCAA|2085-10-08|2229|9693|744|2085|1|10|8|4|2085|744|9693|Monday|2085Q4|N|N|N|2482865|2483137|2482507|2482780|N|N|N|N|N| +2482873|AAAAAAAAJLCOFCAA|2085-10-09|2229|9694|744|2085|2|10|9|4|2085|744|9694|Tuesday|2085Q4|N|N|N|2482865|2483137|2482508|2482781|N|N|N|N|N| +2482874|AAAAAAAAKLCOFCAA|2085-10-10|2229|9694|744|2085|3|10|10|4|2085|744|9694|Wednesday|2085Q4|N|N|N|2482865|2483137|2482509|2482782|N|N|N|N|N| +2482875|AAAAAAAALLCOFCAA|2085-10-11|2229|9694|744|2085|4|10|11|4|2085|744|9694|Thursday|2085Q4|N|N|N|2482865|2483137|2482510|2482783|N|N|N|N|N| +2482876|AAAAAAAAMLCOFCAA|2085-10-12|2229|9694|744|2085|5|10|12|4|2085|744|9694|Friday|2085Q4|N|Y|N|2482865|2483137|2482511|2482784|N|N|N|N|N| +2482877|AAAAAAAANLCOFCAA|2085-10-13|2229|9694|744|2085|6|10|13|4|2085|744|9694|Saturday|2085Q4|N|Y|N|2482865|2483137|2482512|2482785|N|N|N|N|N| +2482878|AAAAAAAAOLCOFCAA|2085-10-14|2229|9694|744|2085|0|10|14|4|2085|744|9694|Sunday|2085Q4|N|N|N|2482865|2483137|2482513|2482786|N|N|N|N|N| +2482879|AAAAAAAAPLCOFCAA|2085-10-15|2229|9694|744|2085|1|10|15|4|2085|744|9694|Monday|2085Q4|N|N|N|2482865|2483137|2482514|2482787|N|N|N|N|N| +2482880|AAAAAAAAAMCOFCAA|2085-10-16|2229|9695|744|2085|2|10|16|4|2085|744|9695|Tuesday|2085Q4|N|N|N|2482865|2483137|2482515|2482788|N|N|N|N|N| +2482881|AAAAAAAABMCOFCAA|2085-10-17|2229|9695|744|2085|3|10|17|4|2085|744|9695|Wednesday|2085Q4|N|N|N|2482865|2483137|2482516|2482789|N|N|N|N|N| +2482882|AAAAAAAACMCOFCAA|2085-10-18|2229|9695|744|2085|4|10|18|4|2085|744|9695|Thursday|2085Q4|N|N|N|2482865|2483137|2482517|2482790|N|N|N|N|N| +2482883|AAAAAAAADMCOFCAA|2085-10-19|2229|9695|744|2085|5|10|19|4|2085|744|9695|Friday|2085Q4|N|Y|N|2482865|2483137|2482518|2482791|N|N|N|N|N| +2482884|AAAAAAAAEMCOFCAA|2085-10-20|2229|9695|744|2085|6|10|20|4|2085|744|9695|Saturday|2085Q4|N|Y|N|2482865|2483137|2482519|2482792|N|N|N|N|N| +2482885|AAAAAAAAFMCOFCAA|2085-10-21|2229|9695|744|2085|0|10|21|4|2085|744|9695|Sunday|2085Q4|N|N|N|2482865|2483137|2482520|2482793|N|N|N|N|N| +2482886|AAAAAAAAGMCOFCAA|2085-10-22|2229|9695|744|2085|1|10|22|4|2085|744|9695|Monday|2085Q4|N|N|N|2482865|2483137|2482521|2482794|N|N|N|N|N| +2482887|AAAAAAAAHMCOFCAA|2085-10-23|2229|9696|744|2085|2|10|23|4|2085|744|9696|Tuesday|2085Q4|N|N|N|2482865|2483137|2482522|2482795|N|N|N|N|N| +2482888|AAAAAAAAIMCOFCAA|2085-10-24|2229|9696|744|2085|3|10|24|4|2085|744|9696|Wednesday|2085Q4|N|N|N|2482865|2483137|2482523|2482796|N|N|N|N|N| +2482889|AAAAAAAAJMCOFCAA|2085-10-25|2229|9696|744|2085|4|10|25|4|2085|744|9696|Thursday|2085Q4|N|N|N|2482865|2483137|2482524|2482797|N|N|N|N|N| +2482890|AAAAAAAAKMCOFCAA|2085-10-26|2229|9696|744|2085|5|10|26|4|2085|744|9696|Friday|2085Q4|N|Y|N|2482865|2483137|2482525|2482798|N|N|N|N|N| +2482891|AAAAAAAALMCOFCAA|2085-10-27|2229|9696|744|2085|6|10|27|4|2085|744|9696|Saturday|2085Q4|N|Y|N|2482865|2483137|2482526|2482799|N|N|N|N|N| +2482892|AAAAAAAAMMCOFCAA|2085-10-28|2229|9696|744|2085|0|10|28|4|2085|744|9696|Sunday|2085Q4|N|N|N|2482865|2483137|2482527|2482800|N|N|N|N|N| +2482893|AAAAAAAANMCOFCAA|2085-10-29|2229|9696|744|2085|1|10|29|4|2085|744|9696|Monday|2085Q4|N|N|N|2482865|2483137|2482528|2482801|N|N|N|N|N| +2482894|AAAAAAAAOMCOFCAA|2085-10-30|2229|9697|744|2085|2|10|30|4|2085|744|9697|Tuesday|2085Q4|N|N|N|2482865|2483137|2482529|2482802|N|N|N|N|N| +2482895|AAAAAAAAPMCOFCAA|2085-10-31|2229|9697|744|2085|3|10|31|4|2085|744|9697|Wednesday|2085Q4|N|N|N|2482865|2483137|2482530|2482803|N|N|N|N|N| +2482896|AAAAAAAAANCOFCAA|2085-11-01|2230|9697|744|2085|4|11|1|4|2085|744|9697|Thursday|2085Q4|N|N|N|2482896|2483199|2482531|2482804|N|N|N|N|N| +2482897|AAAAAAAABNCOFCAA|2085-11-02|2230|9697|744|2085|5|11|2|4|2085|744|9697|Friday|2085Q4|N|Y|N|2482896|2483199|2482532|2482805|N|N|N|N|N| +2482898|AAAAAAAACNCOFCAA|2085-11-03|2230|9697|744|2085|6|11|3|4|2085|744|9697|Saturday|2085Q4|N|Y|N|2482896|2483199|2482533|2482806|N|N|N|N|N| +2482899|AAAAAAAADNCOFCAA|2085-11-04|2230|9697|744|2085|0|11|4|4|2085|744|9697|Sunday|2085Q4|N|N|N|2482896|2483199|2482534|2482807|N|N|N|N|N| +2482900|AAAAAAAAENCOFCAA|2085-11-05|2230|9697|744|2085|1|11|5|4|2085|744|9697|Monday|2085Q4|N|N|N|2482896|2483199|2482535|2482808|N|N|N|N|N| +2482901|AAAAAAAAFNCOFCAA|2085-11-06|2230|9698|744|2085|2|11|6|4|2085|744|9698|Tuesday|2085Q4|N|N|N|2482896|2483199|2482536|2482809|N|N|N|N|N| +2482902|AAAAAAAAGNCOFCAA|2085-11-07|2230|9698|744|2085|3|11|7|4|2085|744|9698|Wednesday|2085Q4|N|N|N|2482896|2483199|2482537|2482810|N|N|N|N|N| +2482903|AAAAAAAAHNCOFCAA|2085-11-08|2230|9698|744|2085|4|11|8|4|2085|744|9698|Thursday|2085Q4|N|N|N|2482896|2483199|2482538|2482811|N|N|N|N|N| +2482904|AAAAAAAAINCOFCAA|2085-11-09|2230|9698|744|2085|5|11|9|4|2085|744|9698|Friday|2085Q4|N|Y|N|2482896|2483199|2482539|2482812|N|N|N|N|N| +2482905|AAAAAAAAJNCOFCAA|2085-11-10|2230|9698|744|2085|6|11|10|4|2085|744|9698|Saturday|2085Q4|N|Y|N|2482896|2483199|2482540|2482813|N|N|N|N|N| +2482906|AAAAAAAAKNCOFCAA|2085-11-11|2230|9698|744|2085|0|11|11|4|2085|744|9698|Sunday|2085Q4|N|N|N|2482896|2483199|2482541|2482814|N|N|N|N|N| +2482907|AAAAAAAALNCOFCAA|2085-11-12|2230|9698|744|2085|1|11|12|4|2085|744|9698|Monday|2085Q4|N|N|N|2482896|2483199|2482542|2482815|N|N|N|N|N| +2482908|AAAAAAAAMNCOFCAA|2085-11-13|2230|9699|744|2085|2|11|13|4|2085|744|9699|Tuesday|2085Q4|N|N|N|2482896|2483199|2482543|2482816|N|N|N|N|N| +2482909|AAAAAAAANNCOFCAA|2085-11-14|2230|9699|744|2085|3|11|14|4|2085|744|9699|Wednesday|2085Q4|N|N|N|2482896|2483199|2482544|2482817|N|N|N|N|N| +2482910|AAAAAAAAONCOFCAA|2085-11-15|2230|9699|744|2085|4|11|15|4|2085|744|9699|Thursday|2085Q4|N|N|N|2482896|2483199|2482545|2482818|N|N|N|N|N| +2482911|AAAAAAAAPNCOFCAA|2085-11-16|2230|9699|744|2085|5|11|16|4|2085|744|9699|Friday|2085Q4|N|Y|N|2482896|2483199|2482546|2482819|N|N|N|N|N| +2482912|AAAAAAAAAOCOFCAA|2085-11-17|2230|9699|744|2085|6|11|17|4|2085|744|9699|Saturday|2085Q4|N|Y|N|2482896|2483199|2482547|2482820|N|N|N|N|N| +2482913|AAAAAAAABOCOFCAA|2085-11-18|2230|9699|744|2085|0|11|18|4|2085|744|9699|Sunday|2085Q4|N|N|N|2482896|2483199|2482548|2482821|N|N|N|N|N| +2482914|AAAAAAAACOCOFCAA|2085-11-19|2230|9699|744|2085|1|11|19|4|2085|744|9699|Monday|2085Q4|N|N|N|2482896|2483199|2482549|2482822|N|N|N|N|N| +2482915|AAAAAAAADOCOFCAA|2085-11-20|2230|9700|744|2085|2|11|20|4|2085|744|9700|Tuesday|2085Q4|N|N|N|2482896|2483199|2482550|2482823|N|N|N|N|N| +2482916|AAAAAAAAEOCOFCAA|2085-11-21|2230|9700|744|2085|3|11|21|4|2085|744|9700|Wednesday|2085Q4|N|N|N|2482896|2483199|2482551|2482824|N|N|N|N|N| +2482917|AAAAAAAAFOCOFCAA|2085-11-22|2230|9700|744|2085|4|11|22|4|2085|744|9700|Thursday|2085Q4|N|N|N|2482896|2483199|2482552|2482825|N|N|N|N|N| +2482918|AAAAAAAAGOCOFCAA|2085-11-23|2230|9700|744|2085|5|11|23|4|2085|744|9700|Friday|2085Q4|N|Y|N|2482896|2483199|2482553|2482826|N|N|N|N|N| +2482919|AAAAAAAAHOCOFCAA|2085-11-24|2230|9700|744|2085|6|11|24|4|2085|744|9700|Saturday|2085Q4|N|Y|N|2482896|2483199|2482554|2482827|N|N|N|N|N| +2482920|AAAAAAAAIOCOFCAA|2085-11-25|2230|9700|744|2085|0|11|25|4|2085|744|9700|Sunday|2085Q4|N|N|N|2482896|2483199|2482555|2482828|N|N|N|N|N| +2482921|AAAAAAAAJOCOFCAA|2085-11-26|2230|9700|744|2085|1|11|26|4|2085|744|9700|Monday|2085Q4|N|N|N|2482896|2483199|2482556|2482829|N|N|N|N|N| +2482922|AAAAAAAAKOCOFCAA|2085-11-27|2230|9701|744|2085|2|11|27|4|2085|744|9701|Tuesday|2085Q4|N|N|N|2482896|2483199|2482557|2482830|N|N|N|N|N| +2482923|AAAAAAAALOCOFCAA|2085-11-28|2230|9701|744|2085|3|11|28|4|2085|744|9701|Wednesday|2085Q4|N|N|N|2482896|2483199|2482558|2482831|N|N|N|N|N| +2482924|AAAAAAAAMOCOFCAA|2085-11-29|2230|9701|744|2085|4|11|29|4|2085|744|9701|Thursday|2085Q4|N|N|N|2482896|2483199|2482559|2482832|N|N|N|N|N| +2482925|AAAAAAAANOCOFCAA|2085-11-30|2230|9701|744|2085|5|11|30|4|2085|744|9701|Friday|2085Q4|N|Y|N|2482896|2483199|2482560|2482833|N|N|N|N|N| +2482926|AAAAAAAAOOCOFCAA|2085-12-01|2231|9701|745|2085|6|12|1|4|2085|745|9701|Saturday|2085Q4|N|Y|N|2482926|2483259|2482561|2482834|N|N|N|N|N| +2482927|AAAAAAAAPOCOFCAA|2085-12-02|2231|9701|745|2085|0|12|2|4|2085|745|9701|Sunday|2085Q4|N|N|N|2482926|2483259|2482562|2482835|N|N|N|N|N| +2482928|AAAAAAAAAPCOFCAA|2085-12-03|2231|9701|745|2085|1|12|3|4|2085|745|9701|Monday|2085Q4|N|N|N|2482926|2483259|2482563|2482836|N|N|N|N|N| +2482929|AAAAAAAABPCOFCAA|2085-12-04|2231|9702|745|2085|2|12|4|4|2085|745|9702|Tuesday|2085Q4|N|N|N|2482926|2483259|2482564|2482837|N|N|N|N|N| +2482930|AAAAAAAACPCOFCAA|2085-12-05|2231|9702|745|2085|3|12|5|4|2085|745|9702|Wednesday|2085Q4|N|N|N|2482926|2483259|2482565|2482838|N|N|N|N|N| +2482931|AAAAAAAADPCOFCAA|2085-12-06|2231|9702|745|2085|4|12|6|4|2085|745|9702|Thursday|2085Q4|N|N|N|2482926|2483259|2482566|2482839|N|N|N|N|N| +2482932|AAAAAAAAEPCOFCAA|2085-12-07|2231|9702|745|2085|5|12|7|4|2085|745|9702|Friday|2085Q4|N|Y|N|2482926|2483259|2482567|2482840|N|N|N|N|N| +2482933|AAAAAAAAFPCOFCAA|2085-12-08|2231|9702|745|2085|6|12|8|4|2085|745|9702|Saturday|2085Q4|N|Y|N|2482926|2483259|2482568|2482841|N|N|N|N|N| +2482934|AAAAAAAAGPCOFCAA|2085-12-09|2231|9702|745|2085|0|12|9|4|2085|745|9702|Sunday|2085Q4|N|N|N|2482926|2483259|2482569|2482842|N|N|N|N|N| +2482935|AAAAAAAAHPCOFCAA|2085-12-10|2231|9702|745|2085|1|12|10|4|2085|745|9702|Monday|2085Q4|N|N|N|2482926|2483259|2482570|2482843|N|N|N|N|N| +2482936|AAAAAAAAIPCOFCAA|2085-12-11|2231|9703|745|2085|2|12|11|4|2085|745|9703|Tuesday|2085Q4|N|N|N|2482926|2483259|2482571|2482844|N|N|N|N|N| +2482937|AAAAAAAAJPCOFCAA|2085-12-12|2231|9703|745|2085|3|12|12|4|2085|745|9703|Wednesday|2085Q4|N|N|N|2482926|2483259|2482572|2482845|N|N|N|N|N| +2482938|AAAAAAAAKPCOFCAA|2085-12-13|2231|9703|745|2085|4|12|13|4|2085|745|9703|Thursday|2085Q4|N|N|N|2482926|2483259|2482573|2482846|N|N|N|N|N| +2482939|AAAAAAAALPCOFCAA|2085-12-14|2231|9703|745|2085|5|12|14|4|2085|745|9703|Friday|2085Q4|N|Y|N|2482926|2483259|2482574|2482847|N|N|N|N|N| +2482940|AAAAAAAAMPCOFCAA|2085-12-15|2231|9703|745|2085|6|12|15|4|2085|745|9703|Saturday|2085Q4|N|Y|N|2482926|2483259|2482575|2482848|N|N|N|N|N| +2482941|AAAAAAAANPCOFCAA|2085-12-16|2231|9703|745|2085|0|12|16|4|2085|745|9703|Sunday|2085Q4|N|N|N|2482926|2483259|2482576|2482849|N|N|N|N|N| +2482942|AAAAAAAAOPCOFCAA|2085-12-17|2231|9703|745|2085|1|12|17|4|2085|745|9703|Monday|2085Q4|N|N|N|2482926|2483259|2482577|2482850|N|N|N|N|N| +2482943|AAAAAAAAPPCOFCAA|2085-12-18|2231|9704|745|2085|2|12|18|4|2085|745|9704|Tuesday|2085Q4|N|N|N|2482926|2483259|2482578|2482851|N|N|N|N|N| +2482944|AAAAAAAAAADOFCAA|2085-12-19|2231|9704|745|2085|3|12|19|4|2085|745|9704|Wednesday|2085Q4|N|N|N|2482926|2483259|2482579|2482852|N|N|N|N|N| +2482945|AAAAAAAABADOFCAA|2085-12-20|2231|9704|745|2085|4|12|20|4|2085|745|9704|Thursday|2085Q4|N|N|N|2482926|2483259|2482580|2482853|N|N|N|N|N| +2482946|AAAAAAAACADOFCAA|2085-12-21|2231|9704|745|2085|5|12|21|4|2085|745|9704|Friday|2085Q4|N|Y|N|2482926|2483259|2482581|2482854|N|N|N|N|N| +2482947|AAAAAAAADADOFCAA|2085-12-22|2231|9704|745|2085|6|12|22|4|2085|745|9704|Saturday|2085Q4|N|Y|N|2482926|2483259|2482582|2482855|N|N|N|N|N| +2482948|AAAAAAAAEADOFCAA|2085-12-23|2231|9704|745|2085|0|12|23|4|2085|745|9704|Sunday|2085Q4|N|N|N|2482926|2483259|2482583|2482856|N|N|N|N|N| +2482949|AAAAAAAAFADOFCAA|2085-12-24|2231|9704|745|2085|1|12|24|4|2085|745|9704|Monday|2085Q4|N|N|N|2482926|2483259|2482584|2482857|N|N|N|N|N| +2482950|AAAAAAAAGADOFCAA|2085-12-25|2231|9705|745|2085|2|12|25|4|2085|745|9705|Tuesday|2085Q4|N|N|N|2482926|2483259|2482585|2482858|N|N|N|N|N| +2482951|AAAAAAAAHADOFCAA|2085-12-26|2231|9705|745|2085|3|12|26|4|2085|745|9705|Wednesday|2085Q4|Y|N|N|2482926|2483259|2482586|2482859|N|N|N|N|N| +2482952|AAAAAAAAIADOFCAA|2085-12-27|2231|9705|745|2085|4|12|27|4|2085|745|9705|Thursday|2085Q4|N|N|Y|2482926|2483259|2482587|2482860|N|N|N|N|N| +2482953|AAAAAAAAJADOFCAA|2085-12-28|2231|9705|745|2085|5|12|28|4|2085|745|9705|Friday|2085Q4|N|Y|N|2482926|2483259|2482588|2482861|N|N|N|N|N| +2482954|AAAAAAAAKADOFCAA|2085-12-29|2231|9705|745|2085|6|12|29|4|2085|745|9705|Saturday|2085Q4|N|Y|N|2482926|2483259|2482589|2482862|N|N|N|N|N| +2482955|AAAAAAAALADOFCAA|2085-12-30|2231|9705|745|2085|0|12|30|4|2085|745|9705|Sunday|2085Q4|N|N|N|2482926|2483259|2482590|2482863|N|N|N|N|N| +2482956|AAAAAAAAMADOFCAA|2085-12-31|2231|9705|745|2085|1|12|31|4|2085|745|9705|Monday|2085Q4|N|N|N|2482926|2483259|2482591|2482864|N|N|N|N|N| +2482957|AAAAAAAANADOFCAA|2086-01-01|2232|9706|745|2086|2|1|1|1|2086|745|9706|Tuesday|2086Q1|Y|N|N|2482957|2482956|2482592|2482865|N|N|N|N|N| +2482958|AAAAAAAAOADOFCAA|2086-01-02|2232|9706|745|2086|3|1|2|1|2086|745|9706|Wednesday|2086Q1|N|N|Y|2482957|2482956|2482593|2482866|N|N|N|N|N| +2482959|AAAAAAAAPADOFCAA|2086-01-03|2232|9706|745|2086|4|1|3|1|2086|745|9706|Thursday|2086Q1|N|N|N|2482957|2482956|2482594|2482867|N|N|N|N|N| +2482960|AAAAAAAAABDOFCAA|2086-01-04|2232|9706|745|2086|5|1|4|1|2086|745|9706|Friday|2086Q1|N|Y|N|2482957|2482956|2482595|2482868|N|N|N|N|N| +2482961|AAAAAAAABBDOFCAA|2086-01-05|2232|9706|745|2086|6|1|5|1|2086|745|9706|Saturday|2086Q1|N|Y|N|2482957|2482956|2482596|2482869|N|N|N|N|N| +2482962|AAAAAAAACBDOFCAA|2086-01-06|2232|9706|745|2086|0|1|6|1|2086|745|9706|Sunday|2086Q1|N|N|N|2482957|2482956|2482597|2482870|N|N|N|N|N| +2482963|AAAAAAAADBDOFCAA|2086-01-07|2232|9706|745|2086|1|1|7|1|2086|745|9706|Monday|2086Q1|N|N|N|2482957|2482956|2482598|2482871|N|N|N|N|N| +2482964|AAAAAAAAEBDOFCAA|2086-01-08|2232|9707|745|2086|2|1|8|1|2086|745|9707|Tuesday|2086Q1|N|N|N|2482957|2482956|2482599|2482872|N|N|N|N|N| +2482965|AAAAAAAAFBDOFCAA|2086-01-09|2232|9707|745|2086|3|1|9|1|2086|745|9707|Wednesday|2086Q1|N|N|N|2482957|2482956|2482600|2482873|N|N|N|N|N| +2482966|AAAAAAAAGBDOFCAA|2086-01-10|2232|9707|745|2086|4|1|10|1|2086|745|9707|Thursday|2086Q1|N|N|N|2482957|2482956|2482601|2482874|N|N|N|N|N| +2482967|AAAAAAAAHBDOFCAA|2086-01-11|2232|9707|745|2086|5|1|11|1|2086|745|9707|Friday|2086Q1|N|Y|N|2482957|2482956|2482602|2482875|N|N|N|N|N| +2482968|AAAAAAAAIBDOFCAA|2086-01-12|2232|9707|745|2086|6|1|12|1|2086|745|9707|Saturday|2086Q1|N|Y|N|2482957|2482956|2482603|2482876|N|N|N|N|N| +2482969|AAAAAAAAJBDOFCAA|2086-01-13|2232|9707|745|2086|0|1|13|1|2086|745|9707|Sunday|2086Q1|N|N|N|2482957|2482956|2482604|2482877|N|N|N|N|N| +2482970|AAAAAAAAKBDOFCAA|2086-01-14|2232|9707|745|2086|1|1|14|1|2086|745|9707|Monday|2086Q1|N|N|N|2482957|2482956|2482605|2482878|N|N|N|N|N| +2482971|AAAAAAAALBDOFCAA|2086-01-15|2232|9708|745|2086|2|1|15|1|2086|745|9708|Tuesday|2086Q1|N|N|N|2482957|2482956|2482606|2482879|N|N|N|N|N| +2482972|AAAAAAAAMBDOFCAA|2086-01-16|2232|9708|745|2086|3|1|16|1|2086|745|9708|Wednesday|2086Q1|N|N|N|2482957|2482956|2482607|2482880|N|N|N|N|N| +2482973|AAAAAAAANBDOFCAA|2086-01-17|2232|9708|745|2086|4|1|17|1|2086|745|9708|Thursday|2086Q1|N|N|N|2482957|2482956|2482608|2482881|N|N|N|N|N| +2482974|AAAAAAAAOBDOFCAA|2086-01-18|2232|9708|745|2086|5|1|18|1|2086|745|9708|Friday|2086Q1|N|Y|N|2482957|2482956|2482609|2482882|N|N|N|N|N| +2482975|AAAAAAAAPBDOFCAA|2086-01-19|2232|9708|745|2086|6|1|19|1|2086|745|9708|Saturday|2086Q1|N|Y|N|2482957|2482956|2482610|2482883|N|N|N|N|N| +2482976|AAAAAAAAACDOFCAA|2086-01-20|2232|9708|745|2086|0|1|20|1|2086|745|9708|Sunday|2086Q1|N|N|N|2482957|2482956|2482611|2482884|N|N|N|N|N| +2482977|AAAAAAAABCDOFCAA|2086-01-21|2232|9708|745|2086|1|1|21|1|2086|745|9708|Monday|2086Q1|N|N|N|2482957|2482956|2482612|2482885|N|N|N|N|N| +2482978|AAAAAAAACCDOFCAA|2086-01-22|2232|9709|745|2086|2|1|22|1|2086|745|9709|Tuesday|2086Q1|N|N|N|2482957|2482956|2482613|2482886|N|N|N|N|N| +2482979|AAAAAAAADCDOFCAA|2086-01-23|2232|9709|745|2086|3|1|23|1|2086|745|9709|Wednesday|2086Q1|N|N|N|2482957|2482956|2482614|2482887|N|N|N|N|N| +2482980|AAAAAAAAECDOFCAA|2086-01-24|2232|9709|745|2086|4|1|24|1|2086|745|9709|Thursday|2086Q1|N|N|N|2482957|2482956|2482615|2482888|N|N|N|N|N| +2482981|AAAAAAAAFCDOFCAA|2086-01-25|2232|9709|745|2086|5|1|25|1|2086|745|9709|Friday|2086Q1|N|Y|N|2482957|2482956|2482616|2482889|N|N|N|N|N| +2482982|AAAAAAAAGCDOFCAA|2086-01-26|2232|9709|745|2086|6|1|26|1|2086|745|9709|Saturday|2086Q1|N|Y|N|2482957|2482956|2482617|2482890|N|N|N|N|N| +2482983|AAAAAAAAHCDOFCAA|2086-01-27|2232|9709|745|2086|0|1|27|1|2086|745|9709|Sunday|2086Q1|N|N|N|2482957|2482956|2482618|2482891|N|N|N|N|N| +2482984|AAAAAAAAICDOFCAA|2086-01-28|2232|9709|745|2086|1|1|28|1|2086|745|9709|Monday|2086Q1|N|N|N|2482957|2482956|2482619|2482892|N|N|N|N|N| +2482985|AAAAAAAAJCDOFCAA|2086-01-29|2232|9710|745|2086|2|1|29|1|2086|745|9710|Tuesday|2086Q1|N|N|N|2482957|2482956|2482620|2482893|N|N|N|N|N| +2482986|AAAAAAAAKCDOFCAA|2086-01-30|2232|9710|745|2086|3|1|30|1|2086|745|9710|Wednesday|2086Q1|N|N|N|2482957|2482956|2482621|2482894|N|N|N|N|N| +2482987|AAAAAAAALCDOFCAA|2086-01-31|2232|9710|745|2086|4|1|31|1|2086|745|9710|Thursday|2086Q1|N|N|N|2482957|2482956|2482622|2482895|N|N|N|N|N| +2482988|AAAAAAAAMCDOFCAA|2086-02-01|2233|9710|745|2086|5|2|1|1|2086|745|9710|Friday|2086Q1|N|Y|N|2482988|2483018|2482623|2482896|N|N|N|N|N| +2482989|AAAAAAAANCDOFCAA|2086-02-02|2233|9710|745|2086|6|2|2|1|2086|745|9710|Saturday|2086Q1|N|Y|N|2482988|2483018|2482624|2482897|N|N|N|N|N| +2482990|AAAAAAAAOCDOFCAA|2086-02-03|2233|9710|745|2086|0|2|3|1|2086|745|9710|Sunday|2086Q1|N|N|N|2482988|2483018|2482625|2482898|N|N|N|N|N| +2482991|AAAAAAAAPCDOFCAA|2086-02-04|2233|9710|745|2086|1|2|4|1|2086|745|9710|Monday|2086Q1|N|N|N|2482988|2483018|2482626|2482899|N|N|N|N|N| +2482992|AAAAAAAAADDOFCAA|2086-02-05|2233|9711|745|2086|2|2|5|1|2086|745|9711|Tuesday|2086Q1|N|N|N|2482988|2483018|2482627|2482900|N|N|N|N|N| +2482993|AAAAAAAABDDOFCAA|2086-02-06|2233|9711|745|2086|3|2|6|1|2086|745|9711|Wednesday|2086Q1|N|N|N|2482988|2483018|2482628|2482901|N|N|N|N|N| +2482994|AAAAAAAACDDOFCAA|2086-02-07|2233|9711|745|2086|4|2|7|1|2086|745|9711|Thursday|2086Q1|N|N|N|2482988|2483018|2482629|2482902|N|N|N|N|N| +2482995|AAAAAAAADDDOFCAA|2086-02-08|2233|9711|745|2086|5|2|8|1|2086|745|9711|Friday|2086Q1|N|Y|N|2482988|2483018|2482630|2482903|N|N|N|N|N| +2482996|AAAAAAAAEDDOFCAA|2086-02-09|2233|9711|745|2086|6|2|9|1|2086|745|9711|Saturday|2086Q1|N|Y|N|2482988|2483018|2482631|2482904|N|N|N|N|N| +2482997|AAAAAAAAFDDOFCAA|2086-02-10|2233|9711|745|2086|0|2|10|1|2086|745|9711|Sunday|2086Q1|N|N|N|2482988|2483018|2482632|2482905|N|N|N|N|N| +2482998|AAAAAAAAGDDOFCAA|2086-02-11|2233|9711|745|2086|1|2|11|1|2086|745|9711|Monday|2086Q1|N|N|N|2482988|2483018|2482633|2482906|N|N|N|N|N| +2482999|AAAAAAAAHDDOFCAA|2086-02-12|2233|9712|745|2086|2|2|12|1|2086|745|9712|Tuesday|2086Q1|N|N|N|2482988|2483018|2482634|2482907|N|N|N|N|N| +2483000|AAAAAAAAIDDOFCAA|2086-02-13|2233|9712|745|2086|3|2|13|1|2086|745|9712|Wednesday|2086Q1|N|N|N|2482988|2483018|2482635|2482908|N|N|N|N|N| +2483001|AAAAAAAAJDDOFCAA|2086-02-14|2233|9712|745|2086|4|2|14|1|2086|745|9712|Thursday|2086Q1|N|N|N|2482988|2483018|2482636|2482909|N|N|N|N|N| +2483002|AAAAAAAAKDDOFCAA|2086-02-15|2233|9712|745|2086|5|2|15|1|2086|745|9712|Friday|2086Q1|N|Y|N|2482988|2483018|2482637|2482910|N|N|N|N|N| +2483003|AAAAAAAALDDOFCAA|2086-02-16|2233|9712|745|2086|6|2|16|1|2086|745|9712|Saturday|2086Q1|N|Y|N|2482988|2483018|2482638|2482911|N|N|N|N|N| +2483004|AAAAAAAAMDDOFCAA|2086-02-17|2233|9712|745|2086|0|2|17|1|2086|745|9712|Sunday|2086Q1|N|N|N|2482988|2483018|2482639|2482912|N|N|N|N|N| +2483005|AAAAAAAANDDOFCAA|2086-02-18|2233|9712|745|2086|1|2|18|1|2086|745|9712|Monday|2086Q1|N|N|N|2482988|2483018|2482640|2482913|N|N|N|N|N| +2483006|AAAAAAAAODDOFCAA|2086-02-19|2233|9713|745|2086|2|2|19|1|2086|745|9713|Tuesday|2086Q1|N|N|N|2482988|2483018|2482641|2482914|N|N|N|N|N| +2483007|AAAAAAAAPDDOFCAA|2086-02-20|2233|9713|745|2086|3|2|20|1|2086|745|9713|Wednesday|2086Q1|N|N|N|2482988|2483018|2482642|2482915|N|N|N|N|N| +2483008|AAAAAAAAAEDOFCAA|2086-02-21|2233|9713|745|2086|4|2|21|1|2086|745|9713|Thursday|2086Q1|N|N|N|2482988|2483018|2482643|2482916|N|N|N|N|N| +2483009|AAAAAAAABEDOFCAA|2086-02-22|2233|9713|745|2086|5|2|22|1|2086|745|9713|Friday|2086Q1|N|Y|N|2482988|2483018|2482644|2482917|N|N|N|N|N| +2483010|AAAAAAAACEDOFCAA|2086-02-23|2233|9713|745|2086|6|2|23|1|2086|745|9713|Saturday|2086Q1|N|Y|N|2482988|2483018|2482645|2482918|N|N|N|N|N| +2483011|AAAAAAAADEDOFCAA|2086-02-24|2233|9713|745|2086|0|2|24|1|2086|745|9713|Sunday|2086Q1|N|N|N|2482988|2483018|2482646|2482919|N|N|N|N|N| +2483012|AAAAAAAAEEDOFCAA|2086-02-25|2233|9713|745|2086|1|2|25|1|2086|745|9713|Monday|2086Q1|N|N|N|2482988|2483018|2482647|2482920|N|N|N|N|N| +2483013|AAAAAAAAFEDOFCAA|2086-02-26|2233|9714|745|2086|2|2|26|1|2086|745|9714|Tuesday|2086Q1|N|N|N|2482988|2483018|2482648|2482921|N|N|N|N|N| +2483014|AAAAAAAAGEDOFCAA|2086-02-27|2233|9714|745|2086|3|2|27|1|2086|745|9714|Wednesday|2086Q1|N|N|N|2482988|2483018|2482649|2482922|N|N|N|N|N| +2483015|AAAAAAAAHEDOFCAA|2086-02-28|2233|9714|745|2086|4|2|28|1|2086|745|9714|Thursday|2086Q1|N|N|N|2482988|2483018|2482650|2482923|N|N|N|N|N| +2483016|AAAAAAAAIEDOFCAA|2086-03-01|2234|9714|746|2086|5|3|1|1|2086|746|9714|Friday|2086Q1|N|Y|N|2483016|2483074|2482651|2482924|N|N|N|N|N| +2483017|AAAAAAAAJEDOFCAA|2086-03-02|2234|9714|746|2086|6|3|2|1|2086|746|9714|Saturday|2086Q1|N|Y|N|2483016|2483074|2482652|2482925|N|N|N|N|N| +2483018|AAAAAAAAKEDOFCAA|2086-03-03|2234|9714|746|2086|0|3|3|1|2086|746|9714|Sunday|2086Q1|N|N|N|2483016|2483074|2482653|2482926|N|N|N|N|N| +2483019|AAAAAAAALEDOFCAA|2086-03-04|2234|9714|746|2086|1|3|4|1|2086|746|9714|Monday|2086Q1|N|N|N|2483016|2483074|2482654|2482927|N|N|N|N|N| +2483020|AAAAAAAAMEDOFCAA|2086-03-05|2234|9715|746|2086|2|3|5|1|2086|746|9715|Tuesday|2086Q1|N|N|N|2483016|2483074|2482655|2482928|N|N|N|N|N| +2483021|AAAAAAAANEDOFCAA|2086-03-06|2234|9715|746|2086|3|3|6|1|2086|746|9715|Wednesday|2086Q1|N|N|N|2483016|2483074|2482656|2482929|N|N|N|N|N| +2483022|AAAAAAAAOEDOFCAA|2086-03-07|2234|9715|746|2086|4|3|7|1|2086|746|9715|Thursday|2086Q1|N|N|N|2483016|2483074|2482657|2482930|N|N|N|N|N| +2483023|AAAAAAAAPEDOFCAA|2086-03-08|2234|9715|746|2086|5|3|8|1|2086|746|9715|Friday|2086Q1|N|Y|N|2483016|2483074|2482658|2482931|N|N|N|N|N| +2483024|AAAAAAAAAFDOFCAA|2086-03-09|2234|9715|746|2086|6|3|9|1|2086|746|9715|Saturday|2086Q1|N|Y|N|2483016|2483074|2482659|2482932|N|N|N|N|N| +2483025|AAAAAAAABFDOFCAA|2086-03-10|2234|9715|746|2086|0|3|10|1|2086|746|9715|Sunday|2086Q1|N|N|N|2483016|2483074|2482660|2482933|N|N|N|N|N| +2483026|AAAAAAAACFDOFCAA|2086-03-11|2234|9715|746|2086|1|3|11|1|2086|746|9715|Monday|2086Q1|N|N|N|2483016|2483074|2482661|2482934|N|N|N|N|N| +2483027|AAAAAAAADFDOFCAA|2086-03-12|2234|9716|746|2086|2|3|12|1|2086|746|9716|Tuesday|2086Q1|N|N|N|2483016|2483074|2482662|2482935|N|N|N|N|N| +2483028|AAAAAAAAEFDOFCAA|2086-03-13|2234|9716|746|2086|3|3|13|1|2086|746|9716|Wednesday|2086Q1|N|N|N|2483016|2483074|2482663|2482936|N|N|N|N|N| +2483029|AAAAAAAAFFDOFCAA|2086-03-14|2234|9716|746|2086|4|3|14|1|2086|746|9716|Thursday|2086Q1|N|N|N|2483016|2483074|2482664|2482937|N|N|N|N|N| +2483030|AAAAAAAAGFDOFCAA|2086-03-15|2234|9716|746|2086|5|3|15|1|2086|746|9716|Friday|2086Q1|N|Y|N|2483016|2483074|2482665|2482938|N|N|N|N|N| +2483031|AAAAAAAAHFDOFCAA|2086-03-16|2234|9716|746|2086|6|3|16|1|2086|746|9716|Saturday|2086Q1|N|Y|N|2483016|2483074|2482666|2482939|N|N|N|N|N| +2483032|AAAAAAAAIFDOFCAA|2086-03-17|2234|9716|746|2086|0|3|17|1|2086|746|9716|Sunday|2086Q1|N|N|N|2483016|2483074|2482667|2482940|N|N|N|N|N| +2483033|AAAAAAAAJFDOFCAA|2086-03-18|2234|9716|746|2086|1|3|18|1|2086|746|9716|Monday|2086Q1|N|N|N|2483016|2483074|2482668|2482941|N|N|N|N|N| +2483034|AAAAAAAAKFDOFCAA|2086-03-19|2234|9717|746|2086|2|3|19|1|2086|746|9717|Tuesday|2086Q1|N|N|N|2483016|2483074|2482669|2482942|N|N|N|N|N| +2483035|AAAAAAAALFDOFCAA|2086-03-20|2234|9717|746|2086|3|3|20|1|2086|746|9717|Wednesday|2086Q1|N|N|N|2483016|2483074|2482670|2482943|N|N|N|N|N| +2483036|AAAAAAAAMFDOFCAA|2086-03-21|2234|9717|746|2086|4|3|21|1|2086|746|9717|Thursday|2086Q1|N|N|N|2483016|2483074|2482671|2482944|N|N|N|N|N| +2483037|AAAAAAAANFDOFCAA|2086-03-22|2234|9717|746|2086|5|3|22|1|2086|746|9717|Friday|2086Q1|N|Y|N|2483016|2483074|2482672|2482945|N|N|N|N|N| +2483038|AAAAAAAAOFDOFCAA|2086-03-23|2234|9717|746|2086|6|3|23|1|2086|746|9717|Saturday|2086Q1|N|Y|N|2483016|2483074|2482673|2482946|N|N|N|N|N| +2483039|AAAAAAAAPFDOFCAA|2086-03-24|2234|9717|746|2086|0|3|24|1|2086|746|9717|Sunday|2086Q1|N|N|N|2483016|2483074|2482674|2482947|N|N|N|N|N| +2483040|AAAAAAAAAGDOFCAA|2086-03-25|2234|9717|746|2086|1|3|25|1|2086|746|9717|Monday|2086Q1|N|N|N|2483016|2483074|2482675|2482948|N|N|N|N|N| +2483041|AAAAAAAABGDOFCAA|2086-03-26|2234|9718|746|2086|2|3|26|1|2086|746|9718|Tuesday|2086Q1|N|N|N|2483016|2483074|2482676|2482949|N|N|N|N|N| +2483042|AAAAAAAACGDOFCAA|2086-03-27|2234|9718|746|2086|3|3|27|1|2086|746|9718|Wednesday|2086Q1|N|N|N|2483016|2483074|2482677|2482950|N|N|N|N|N| +2483043|AAAAAAAADGDOFCAA|2086-03-28|2234|9718|746|2086|4|3|28|1|2086|746|9718|Thursday|2086Q1|N|N|N|2483016|2483074|2482678|2482951|N|N|N|N|N| +2483044|AAAAAAAAEGDOFCAA|2086-03-29|2234|9718|746|2086|5|3|29|1|2086|746|9718|Friday|2086Q1|N|Y|N|2483016|2483074|2482679|2482952|N|N|N|N|N| +2483045|AAAAAAAAFGDOFCAA|2086-03-30|2234|9718|746|2086|6|3|30|1|2086|746|9718|Saturday|2086Q1|N|Y|N|2483016|2483074|2482680|2482953|N|N|N|N|N| +2483046|AAAAAAAAGGDOFCAA|2086-03-31|2234|9718|746|2086|0|3|31|1|2086|746|9718|Sunday|2086Q1|N|N|N|2483016|2483074|2482681|2482954|N|N|N|N|N| +2483047|AAAAAAAAHGDOFCAA|2086-04-01|2235|9718|746|2086|1|4|1|1|2086|746|9718|Monday|2086Q1|N|N|N|2483047|2483136|2482682|2482957|N|N|N|N|N| +2483048|AAAAAAAAIGDOFCAA|2086-04-02|2235|9719|746|2086|2|4|2|2|2086|746|9719|Tuesday|2086Q2|N|N|N|2483047|2483136|2482683|2482958|N|N|N|N|N| +2483049|AAAAAAAAJGDOFCAA|2086-04-03|2235|9719|746|2086|3|4|3|2|2086|746|9719|Wednesday|2086Q2|N|N|N|2483047|2483136|2482684|2482959|N|N|N|N|N| +2483050|AAAAAAAAKGDOFCAA|2086-04-04|2235|9719|746|2086|4|4|4|2|2086|746|9719|Thursday|2086Q2|N|N|N|2483047|2483136|2482685|2482960|N|N|N|N|N| +2483051|AAAAAAAALGDOFCAA|2086-04-05|2235|9719|746|2086|5|4|5|2|2086|746|9719|Friday|2086Q2|N|Y|N|2483047|2483136|2482686|2482961|N|N|N|N|N| +2483052|AAAAAAAAMGDOFCAA|2086-04-06|2235|9719|746|2086|6|4|6|2|2086|746|9719|Saturday|2086Q2|N|Y|N|2483047|2483136|2482687|2482962|N|N|N|N|N| +2483053|AAAAAAAANGDOFCAA|2086-04-07|2235|9719|746|2086|0|4|7|2|2086|746|9719|Sunday|2086Q2|N|N|N|2483047|2483136|2482688|2482963|N|N|N|N|N| +2483054|AAAAAAAAOGDOFCAA|2086-04-08|2235|9719|746|2086|1|4|8|2|2086|746|9719|Monday|2086Q2|N|N|N|2483047|2483136|2482689|2482964|N|N|N|N|N| +2483055|AAAAAAAAPGDOFCAA|2086-04-09|2235|9720|746|2086|2|4|9|2|2086|746|9720|Tuesday|2086Q2|N|N|N|2483047|2483136|2482690|2482965|N|N|N|N|N| +2483056|AAAAAAAAAHDOFCAA|2086-04-10|2235|9720|746|2086|3|4|10|2|2086|746|9720|Wednesday|2086Q2|N|N|N|2483047|2483136|2482691|2482966|N|N|N|N|N| +2483057|AAAAAAAABHDOFCAA|2086-04-11|2235|9720|746|2086|4|4|11|2|2086|746|9720|Thursday|2086Q2|N|N|N|2483047|2483136|2482692|2482967|N|N|N|N|N| +2483058|AAAAAAAACHDOFCAA|2086-04-12|2235|9720|746|2086|5|4|12|2|2086|746|9720|Friday|2086Q2|N|Y|N|2483047|2483136|2482693|2482968|N|N|N|N|N| +2483059|AAAAAAAADHDOFCAA|2086-04-13|2235|9720|746|2086|6|4|13|2|2086|746|9720|Saturday|2086Q2|N|Y|N|2483047|2483136|2482694|2482969|N|N|N|N|N| +2483060|AAAAAAAAEHDOFCAA|2086-04-14|2235|9720|746|2086|0|4|14|2|2086|746|9720|Sunday|2086Q2|N|N|N|2483047|2483136|2482695|2482970|N|N|N|N|N| +2483061|AAAAAAAAFHDOFCAA|2086-04-15|2235|9720|746|2086|1|4|15|2|2086|746|9720|Monday|2086Q2|N|N|N|2483047|2483136|2482696|2482971|N|N|N|N|N| +2483062|AAAAAAAAGHDOFCAA|2086-04-16|2235|9721|746|2086|2|4|16|2|2086|746|9721|Tuesday|2086Q2|N|N|N|2483047|2483136|2482697|2482972|N|N|N|N|N| +2483063|AAAAAAAAHHDOFCAA|2086-04-17|2235|9721|746|2086|3|4|17|2|2086|746|9721|Wednesday|2086Q2|N|N|N|2483047|2483136|2482698|2482973|N|N|N|N|N| +2483064|AAAAAAAAIHDOFCAA|2086-04-18|2235|9721|746|2086|4|4|18|2|2086|746|9721|Thursday|2086Q2|N|N|N|2483047|2483136|2482699|2482974|N|N|N|N|N| +2483065|AAAAAAAAJHDOFCAA|2086-04-19|2235|9721|746|2086|5|4|19|2|2086|746|9721|Friday|2086Q2|N|Y|N|2483047|2483136|2482700|2482975|N|N|N|N|N| +2483066|AAAAAAAAKHDOFCAA|2086-04-20|2235|9721|746|2086|6|4|20|2|2086|746|9721|Saturday|2086Q2|N|Y|N|2483047|2483136|2482701|2482976|N|N|N|N|N| +2483067|AAAAAAAALHDOFCAA|2086-04-21|2235|9721|746|2086|0|4|21|2|2086|746|9721|Sunday|2086Q2|N|N|N|2483047|2483136|2482702|2482977|N|N|N|N|N| +2483068|AAAAAAAAMHDOFCAA|2086-04-22|2235|9721|746|2086|1|4|22|2|2086|746|9721|Monday|2086Q2|N|N|N|2483047|2483136|2482703|2482978|N|N|N|N|N| +2483069|AAAAAAAANHDOFCAA|2086-04-23|2235|9722|746|2086|2|4|23|2|2086|746|9722|Tuesday|2086Q2|N|N|N|2483047|2483136|2482704|2482979|N|N|N|N|N| +2483070|AAAAAAAAOHDOFCAA|2086-04-24|2235|9722|746|2086|3|4|24|2|2086|746|9722|Wednesday|2086Q2|N|N|N|2483047|2483136|2482705|2482980|N|N|N|N|N| +2483071|AAAAAAAAPHDOFCAA|2086-04-25|2235|9722|746|2086|4|4|25|2|2086|746|9722|Thursday|2086Q2|N|N|N|2483047|2483136|2482706|2482981|N|N|N|N|N| +2483072|AAAAAAAAAIDOFCAA|2086-04-26|2235|9722|746|2086|5|4|26|2|2086|746|9722|Friday|2086Q2|N|Y|N|2483047|2483136|2482707|2482982|N|N|N|N|N| +2483073|AAAAAAAABIDOFCAA|2086-04-27|2235|9722|746|2086|6|4|27|2|2086|746|9722|Saturday|2086Q2|N|Y|N|2483047|2483136|2482708|2482983|N|N|N|N|N| +2483074|AAAAAAAACIDOFCAA|2086-04-28|2235|9722|746|2086|0|4|28|2|2086|746|9722|Sunday|2086Q2|N|N|N|2483047|2483136|2482709|2482984|N|N|N|N|N| +2483075|AAAAAAAADIDOFCAA|2086-04-29|2235|9722|746|2086|1|4|29|2|2086|746|9722|Monday|2086Q2|N|N|N|2483047|2483136|2482710|2482985|N|N|N|N|N| +2483076|AAAAAAAAEIDOFCAA|2086-04-30|2235|9723|746|2086|2|4|30|2|2086|746|9723|Tuesday|2086Q2|N|N|N|2483047|2483136|2482711|2482986|N|N|N|N|N| +2483077|AAAAAAAAFIDOFCAA|2086-05-01|2236|9723|746|2086|3|5|1|2|2086|746|9723|Wednesday|2086Q2|N|N|N|2483077|2483196|2482712|2482987|N|N|N|N|N| +2483078|AAAAAAAAGIDOFCAA|2086-05-02|2236|9723|746|2086|4|5|2|2|2086|746|9723|Thursday|2086Q2|N|N|N|2483077|2483196|2482713|2482988|N|N|N|N|N| +2483079|AAAAAAAAHIDOFCAA|2086-05-03|2236|9723|746|2086|5|5|3|2|2086|746|9723|Friday|2086Q2|N|Y|N|2483077|2483196|2482714|2482989|N|N|N|N|N| +2483080|AAAAAAAAIIDOFCAA|2086-05-04|2236|9723|746|2086|6|5|4|2|2086|746|9723|Saturday|2086Q2|N|Y|N|2483077|2483196|2482715|2482990|N|N|N|N|N| +2483081|AAAAAAAAJIDOFCAA|2086-05-05|2236|9723|746|2086|0|5|5|2|2086|746|9723|Sunday|2086Q2|N|N|N|2483077|2483196|2482716|2482991|N|N|N|N|N| +2483082|AAAAAAAAKIDOFCAA|2086-05-06|2236|9723|746|2086|1|5|6|2|2086|746|9723|Monday|2086Q2|N|N|N|2483077|2483196|2482717|2482992|N|N|N|N|N| +2483083|AAAAAAAALIDOFCAA|2086-05-07|2236|9724|746|2086|2|5|7|2|2086|746|9724|Tuesday|2086Q2|N|N|N|2483077|2483196|2482718|2482993|N|N|N|N|N| +2483084|AAAAAAAAMIDOFCAA|2086-05-08|2236|9724|746|2086|3|5|8|2|2086|746|9724|Wednesday|2086Q2|N|N|N|2483077|2483196|2482719|2482994|N|N|N|N|N| +2483085|AAAAAAAANIDOFCAA|2086-05-09|2236|9724|746|2086|4|5|9|2|2086|746|9724|Thursday|2086Q2|N|N|N|2483077|2483196|2482720|2482995|N|N|N|N|N| +2483086|AAAAAAAAOIDOFCAA|2086-05-10|2236|9724|746|2086|5|5|10|2|2086|746|9724|Friday|2086Q2|N|Y|N|2483077|2483196|2482721|2482996|N|N|N|N|N| +2483087|AAAAAAAAPIDOFCAA|2086-05-11|2236|9724|746|2086|6|5|11|2|2086|746|9724|Saturday|2086Q2|N|Y|N|2483077|2483196|2482722|2482997|N|N|N|N|N| +2483088|AAAAAAAAAJDOFCAA|2086-05-12|2236|9724|746|2086|0|5|12|2|2086|746|9724|Sunday|2086Q2|N|N|N|2483077|2483196|2482723|2482998|N|N|N|N|N| +2483089|AAAAAAAABJDOFCAA|2086-05-13|2236|9724|746|2086|1|5|13|2|2086|746|9724|Monday|2086Q2|N|N|N|2483077|2483196|2482724|2482999|N|N|N|N|N| +2483090|AAAAAAAACJDOFCAA|2086-05-14|2236|9725|746|2086|2|5|14|2|2086|746|9725|Tuesday|2086Q2|N|N|N|2483077|2483196|2482725|2483000|N|N|N|N|N| +2483091|AAAAAAAADJDOFCAA|2086-05-15|2236|9725|746|2086|3|5|15|2|2086|746|9725|Wednesday|2086Q2|N|N|N|2483077|2483196|2482726|2483001|N|N|N|N|N| +2483092|AAAAAAAAEJDOFCAA|2086-05-16|2236|9725|746|2086|4|5|16|2|2086|746|9725|Thursday|2086Q2|N|N|N|2483077|2483196|2482727|2483002|N|N|N|N|N| +2483093|AAAAAAAAFJDOFCAA|2086-05-17|2236|9725|746|2086|5|5|17|2|2086|746|9725|Friday|2086Q2|N|Y|N|2483077|2483196|2482728|2483003|N|N|N|N|N| +2483094|AAAAAAAAGJDOFCAA|2086-05-18|2236|9725|746|2086|6|5|18|2|2086|746|9725|Saturday|2086Q2|N|Y|N|2483077|2483196|2482729|2483004|N|N|N|N|N| +2483095|AAAAAAAAHJDOFCAA|2086-05-19|2236|9725|746|2086|0|5|19|2|2086|746|9725|Sunday|2086Q2|N|N|N|2483077|2483196|2482730|2483005|N|N|N|N|N| +2483096|AAAAAAAAIJDOFCAA|2086-05-20|2236|9725|746|2086|1|5|20|2|2086|746|9725|Monday|2086Q2|N|N|N|2483077|2483196|2482731|2483006|N|N|N|N|N| +2483097|AAAAAAAAJJDOFCAA|2086-05-21|2236|9726|746|2086|2|5|21|2|2086|746|9726|Tuesday|2086Q2|N|N|N|2483077|2483196|2482732|2483007|N|N|N|N|N| +2483098|AAAAAAAAKJDOFCAA|2086-05-22|2236|9726|746|2086|3|5|22|2|2086|746|9726|Wednesday|2086Q2|N|N|N|2483077|2483196|2482733|2483008|N|N|N|N|N| +2483099|AAAAAAAALJDOFCAA|2086-05-23|2236|9726|746|2086|4|5|23|2|2086|746|9726|Thursday|2086Q2|N|N|N|2483077|2483196|2482734|2483009|N|N|N|N|N| +2483100|AAAAAAAAMJDOFCAA|2086-05-24|2236|9726|746|2086|5|5|24|2|2086|746|9726|Friday|2086Q2|N|Y|N|2483077|2483196|2482735|2483010|N|N|N|N|N| +2483101|AAAAAAAANJDOFCAA|2086-05-25|2236|9726|746|2086|6|5|25|2|2086|746|9726|Saturday|2086Q2|N|Y|N|2483077|2483196|2482736|2483011|N|N|N|N|N| +2483102|AAAAAAAAOJDOFCAA|2086-05-26|2236|9726|746|2086|0|5|26|2|2086|746|9726|Sunday|2086Q2|N|N|N|2483077|2483196|2482737|2483012|N|N|N|N|N| +2483103|AAAAAAAAPJDOFCAA|2086-05-27|2236|9726|746|2086|1|5|27|2|2086|746|9726|Monday|2086Q2|N|N|N|2483077|2483196|2482738|2483013|N|N|N|N|N| +2483104|AAAAAAAAAKDOFCAA|2086-05-28|2236|9727|746|2086|2|5|28|2|2086|746|9727|Tuesday|2086Q2|N|N|N|2483077|2483196|2482739|2483014|N|N|N|N|N| +2483105|AAAAAAAABKDOFCAA|2086-05-29|2236|9727|746|2086|3|5|29|2|2086|746|9727|Wednesday|2086Q2|N|N|N|2483077|2483196|2482740|2483015|N|N|N|N|N| +2483106|AAAAAAAACKDOFCAA|2086-05-30|2236|9727|746|2086|4|5|30|2|2086|746|9727|Thursday|2086Q2|N|N|N|2483077|2483196|2482741|2483016|N|N|N|N|N| +2483107|AAAAAAAADKDOFCAA|2086-05-31|2236|9727|746|2086|5|5|31|2|2086|746|9727|Friday|2086Q2|N|Y|N|2483077|2483196|2482742|2483017|N|N|N|N|N| +2483108|AAAAAAAAEKDOFCAA|2086-06-01|2237|9727|747|2086|6|6|1|2|2086|747|9727|Saturday|2086Q2|N|Y|N|2483108|2483258|2482743|2483018|N|N|N|N|N| +2483109|AAAAAAAAFKDOFCAA|2086-06-02|2237|9727|747|2086|0|6|2|2|2086|747|9727|Sunday|2086Q2|N|N|N|2483108|2483258|2482744|2483019|N|N|N|N|N| +2483110|AAAAAAAAGKDOFCAA|2086-06-03|2237|9727|747|2086|1|6|3|2|2086|747|9727|Monday|2086Q2|N|N|N|2483108|2483258|2482745|2483020|N|N|N|N|N| +2483111|AAAAAAAAHKDOFCAA|2086-06-04|2237|9728|747|2086|2|6|4|2|2086|747|9728|Tuesday|2086Q2|N|N|N|2483108|2483258|2482746|2483021|N|N|N|N|N| +2483112|AAAAAAAAIKDOFCAA|2086-06-05|2237|9728|747|2086|3|6|5|2|2086|747|9728|Wednesday|2086Q2|N|N|N|2483108|2483258|2482747|2483022|N|N|N|N|N| +2483113|AAAAAAAAJKDOFCAA|2086-06-06|2237|9728|747|2086|4|6|6|2|2086|747|9728|Thursday|2086Q2|N|N|N|2483108|2483258|2482748|2483023|N|N|N|N|N| +2483114|AAAAAAAAKKDOFCAA|2086-06-07|2237|9728|747|2086|5|6|7|2|2086|747|9728|Friday|2086Q2|N|Y|N|2483108|2483258|2482749|2483024|N|N|N|N|N| +2483115|AAAAAAAALKDOFCAA|2086-06-08|2237|9728|747|2086|6|6|8|2|2086|747|9728|Saturday|2086Q2|N|Y|N|2483108|2483258|2482750|2483025|N|N|N|N|N| +2483116|AAAAAAAAMKDOFCAA|2086-06-09|2237|9728|747|2086|0|6|9|2|2086|747|9728|Sunday|2086Q2|N|N|N|2483108|2483258|2482751|2483026|N|N|N|N|N| +2483117|AAAAAAAANKDOFCAA|2086-06-10|2237|9728|747|2086|1|6|10|2|2086|747|9728|Monday|2086Q2|N|N|N|2483108|2483258|2482752|2483027|N|N|N|N|N| +2483118|AAAAAAAAOKDOFCAA|2086-06-11|2237|9729|747|2086|2|6|11|2|2086|747|9729|Tuesday|2086Q2|N|N|N|2483108|2483258|2482753|2483028|N|N|N|N|N| +2483119|AAAAAAAAPKDOFCAA|2086-06-12|2237|9729|747|2086|3|6|12|2|2086|747|9729|Wednesday|2086Q2|N|N|N|2483108|2483258|2482754|2483029|N|N|N|N|N| +2483120|AAAAAAAAALDOFCAA|2086-06-13|2237|9729|747|2086|4|6|13|2|2086|747|9729|Thursday|2086Q2|N|N|N|2483108|2483258|2482755|2483030|N|N|N|N|N| +2483121|AAAAAAAABLDOFCAA|2086-06-14|2237|9729|747|2086|5|6|14|2|2086|747|9729|Friday|2086Q2|N|Y|N|2483108|2483258|2482756|2483031|N|N|N|N|N| +2483122|AAAAAAAACLDOFCAA|2086-06-15|2237|9729|747|2086|6|6|15|2|2086|747|9729|Saturday|2086Q2|N|Y|N|2483108|2483258|2482757|2483032|N|N|N|N|N| +2483123|AAAAAAAADLDOFCAA|2086-06-16|2237|9729|747|2086|0|6|16|2|2086|747|9729|Sunday|2086Q2|N|N|N|2483108|2483258|2482758|2483033|N|N|N|N|N| +2483124|AAAAAAAAELDOFCAA|2086-06-17|2237|9729|747|2086|1|6|17|2|2086|747|9729|Monday|2086Q2|N|N|N|2483108|2483258|2482759|2483034|N|N|N|N|N| +2483125|AAAAAAAAFLDOFCAA|2086-06-18|2237|9730|747|2086|2|6|18|2|2086|747|9730|Tuesday|2086Q2|N|N|N|2483108|2483258|2482760|2483035|N|N|N|N|N| +2483126|AAAAAAAAGLDOFCAA|2086-06-19|2237|9730|747|2086|3|6|19|2|2086|747|9730|Wednesday|2086Q2|N|N|N|2483108|2483258|2482761|2483036|N|N|N|N|N| +2483127|AAAAAAAAHLDOFCAA|2086-06-20|2237|9730|747|2086|4|6|20|2|2086|747|9730|Thursday|2086Q2|N|N|N|2483108|2483258|2482762|2483037|N|N|N|N|N| +2483128|AAAAAAAAILDOFCAA|2086-06-21|2237|9730|747|2086|5|6|21|2|2086|747|9730|Friday|2086Q2|N|Y|N|2483108|2483258|2482763|2483038|N|N|N|N|N| +2483129|AAAAAAAAJLDOFCAA|2086-06-22|2237|9730|747|2086|6|6|22|2|2086|747|9730|Saturday|2086Q2|N|Y|N|2483108|2483258|2482764|2483039|N|N|N|N|N| +2483130|AAAAAAAAKLDOFCAA|2086-06-23|2237|9730|747|2086|0|6|23|2|2086|747|9730|Sunday|2086Q2|N|N|N|2483108|2483258|2482765|2483040|N|N|N|N|N| +2483131|AAAAAAAALLDOFCAA|2086-06-24|2237|9730|747|2086|1|6|24|2|2086|747|9730|Monday|2086Q2|N|N|N|2483108|2483258|2482766|2483041|N|N|N|N|N| +2483132|AAAAAAAAMLDOFCAA|2086-06-25|2237|9731|747|2086|2|6|25|2|2086|747|9731|Tuesday|2086Q2|N|N|N|2483108|2483258|2482767|2483042|N|N|N|N|N| +2483133|AAAAAAAANLDOFCAA|2086-06-26|2237|9731|747|2086|3|6|26|2|2086|747|9731|Wednesday|2086Q2|N|N|N|2483108|2483258|2482768|2483043|N|N|N|N|N| +2483134|AAAAAAAAOLDOFCAA|2086-06-27|2237|9731|747|2086|4|6|27|2|2086|747|9731|Thursday|2086Q2|N|N|N|2483108|2483258|2482769|2483044|N|N|N|N|N| +2483135|AAAAAAAAPLDOFCAA|2086-06-28|2237|9731|747|2086|5|6|28|2|2086|747|9731|Friday|2086Q2|N|Y|N|2483108|2483258|2482770|2483045|N|N|N|N|N| +2483136|AAAAAAAAAMDOFCAA|2086-06-29|2237|9731|747|2086|6|6|29|2|2086|747|9731|Saturday|2086Q2|N|Y|N|2483108|2483258|2482771|2483046|N|N|N|N|N| +2483137|AAAAAAAABMDOFCAA|2086-06-30|2237|9731|747|2086|0|6|30|2|2086|747|9731|Sunday|2086Q2|N|N|N|2483108|2483258|2482772|2483047|N|N|N|N|N| +2483138|AAAAAAAACMDOFCAA|2086-07-01|2238|9731|747|2086|1|7|1|2|2086|747|9731|Monday|2086Q2|N|N|N|2483138|2483318|2482773|2483047|N|N|N|N|N| +2483139|AAAAAAAADMDOFCAA|2086-07-02|2238|9732|747|2086|2|7|2|3|2086|747|9732|Tuesday|2086Q3|N|N|N|2483138|2483318|2482774|2483048|N|N|N|N|N| +2483140|AAAAAAAAEMDOFCAA|2086-07-03|2238|9732|747|2086|3|7|3|3|2086|747|9732|Wednesday|2086Q3|N|N|N|2483138|2483318|2482775|2483049|N|N|N|N|N| +2483141|AAAAAAAAFMDOFCAA|2086-07-04|2238|9732|747|2086|4|7|4|3|2086|747|9732|Thursday|2086Q3|N|N|N|2483138|2483318|2482776|2483050|N|N|N|N|N| +2483142|AAAAAAAAGMDOFCAA|2086-07-05|2238|9732|747|2086|5|7|5|3|2086|747|9732|Friday|2086Q3|Y|Y|N|2483138|2483318|2482777|2483051|N|N|N|N|N| +2483143|AAAAAAAAHMDOFCAA|2086-07-06|2238|9732|747|2086|6|7|6|3|2086|747|9732|Saturday|2086Q3|N|Y|Y|2483138|2483318|2482778|2483052|N|N|N|N|N| +2483144|AAAAAAAAIMDOFCAA|2086-07-07|2238|9732|747|2086|0|7|7|3|2086|747|9732|Sunday|2086Q3|N|N|N|2483138|2483318|2482779|2483053|N|N|N|N|N| +2483145|AAAAAAAAJMDOFCAA|2086-07-08|2238|9732|747|2086|1|7|8|3|2086|747|9732|Monday|2086Q3|N|N|N|2483138|2483318|2482780|2483054|N|N|N|N|N| +2483146|AAAAAAAAKMDOFCAA|2086-07-09|2238|9733|747|2086|2|7|9|3|2086|747|9733|Tuesday|2086Q3|N|N|N|2483138|2483318|2482781|2483055|N|N|N|N|N| +2483147|AAAAAAAALMDOFCAA|2086-07-10|2238|9733|747|2086|3|7|10|3|2086|747|9733|Wednesday|2086Q3|N|N|N|2483138|2483318|2482782|2483056|N|N|N|N|N| +2483148|AAAAAAAAMMDOFCAA|2086-07-11|2238|9733|747|2086|4|7|11|3|2086|747|9733|Thursday|2086Q3|N|N|N|2483138|2483318|2482783|2483057|N|N|N|N|N| +2483149|AAAAAAAANMDOFCAA|2086-07-12|2238|9733|747|2086|5|7|12|3|2086|747|9733|Friday|2086Q3|N|Y|N|2483138|2483318|2482784|2483058|N|N|N|N|N| +2483150|AAAAAAAAOMDOFCAA|2086-07-13|2238|9733|747|2086|6|7|13|3|2086|747|9733|Saturday|2086Q3|N|Y|N|2483138|2483318|2482785|2483059|N|N|N|N|N| +2483151|AAAAAAAAPMDOFCAA|2086-07-14|2238|9733|747|2086|0|7|14|3|2086|747|9733|Sunday|2086Q3|N|N|N|2483138|2483318|2482786|2483060|N|N|N|N|N| +2483152|AAAAAAAAANDOFCAA|2086-07-15|2238|9733|747|2086|1|7|15|3|2086|747|9733|Monday|2086Q3|N|N|N|2483138|2483318|2482787|2483061|N|N|N|N|N| +2483153|AAAAAAAABNDOFCAA|2086-07-16|2238|9734|747|2086|2|7|16|3|2086|747|9734|Tuesday|2086Q3|N|N|N|2483138|2483318|2482788|2483062|N|N|N|N|N| +2483154|AAAAAAAACNDOFCAA|2086-07-17|2238|9734|747|2086|3|7|17|3|2086|747|9734|Wednesday|2086Q3|N|N|N|2483138|2483318|2482789|2483063|N|N|N|N|N| +2483155|AAAAAAAADNDOFCAA|2086-07-18|2238|9734|747|2086|4|7|18|3|2086|747|9734|Thursday|2086Q3|N|N|N|2483138|2483318|2482790|2483064|N|N|N|N|N| +2483156|AAAAAAAAENDOFCAA|2086-07-19|2238|9734|747|2086|5|7|19|3|2086|747|9734|Friday|2086Q3|N|Y|N|2483138|2483318|2482791|2483065|N|N|N|N|N| +2483157|AAAAAAAAFNDOFCAA|2086-07-20|2238|9734|747|2086|6|7|20|3|2086|747|9734|Saturday|2086Q3|N|Y|N|2483138|2483318|2482792|2483066|N|N|N|N|N| +2483158|AAAAAAAAGNDOFCAA|2086-07-21|2238|9734|747|2086|0|7|21|3|2086|747|9734|Sunday|2086Q3|N|N|N|2483138|2483318|2482793|2483067|N|N|N|N|N| +2483159|AAAAAAAAHNDOFCAA|2086-07-22|2238|9734|747|2086|1|7|22|3|2086|747|9734|Monday|2086Q3|N|N|N|2483138|2483318|2482794|2483068|N|N|N|N|N| +2483160|AAAAAAAAINDOFCAA|2086-07-23|2238|9735|747|2086|2|7|23|3|2086|747|9735|Tuesday|2086Q3|N|N|N|2483138|2483318|2482795|2483069|N|N|N|N|N| +2483161|AAAAAAAAJNDOFCAA|2086-07-24|2238|9735|747|2086|3|7|24|3|2086|747|9735|Wednesday|2086Q3|N|N|N|2483138|2483318|2482796|2483070|N|N|N|N|N| +2483162|AAAAAAAAKNDOFCAA|2086-07-25|2238|9735|747|2086|4|7|25|3|2086|747|9735|Thursday|2086Q3|N|N|N|2483138|2483318|2482797|2483071|N|N|N|N|N| +2483163|AAAAAAAALNDOFCAA|2086-07-26|2238|9735|747|2086|5|7|26|3|2086|747|9735|Friday|2086Q3|N|Y|N|2483138|2483318|2482798|2483072|N|N|N|N|N| +2483164|AAAAAAAAMNDOFCAA|2086-07-27|2238|9735|747|2086|6|7|27|3|2086|747|9735|Saturday|2086Q3|N|Y|N|2483138|2483318|2482799|2483073|N|N|N|N|N| +2483165|AAAAAAAANNDOFCAA|2086-07-28|2238|9735|747|2086|0|7|28|3|2086|747|9735|Sunday|2086Q3|N|N|N|2483138|2483318|2482800|2483074|N|N|N|N|N| +2483166|AAAAAAAAONDOFCAA|2086-07-29|2238|9735|747|2086|1|7|29|3|2086|747|9735|Monday|2086Q3|N|N|N|2483138|2483318|2482801|2483075|N|N|N|N|N| +2483167|AAAAAAAAPNDOFCAA|2086-07-30|2238|9736|747|2086|2|7|30|3|2086|747|9736|Tuesday|2086Q3|N|N|N|2483138|2483318|2482802|2483076|N|N|N|N|N| +2483168|AAAAAAAAAODOFCAA|2086-07-31|2238|9736|747|2086|3|7|31|3|2086|747|9736|Wednesday|2086Q3|N|N|N|2483138|2483318|2482803|2483077|N|N|N|N|N| +2483169|AAAAAAAABODOFCAA|2086-08-01|2239|9736|747|2086|4|8|1|3|2086|747|9736|Thursday|2086Q3|N|N|N|2483169|2483380|2482804|2483078|N|N|N|N|N| +2483170|AAAAAAAACODOFCAA|2086-08-02|2239|9736|747|2086|5|8|2|3|2086|747|9736|Friday|2086Q3|N|Y|N|2483169|2483380|2482805|2483079|N|N|N|N|N| +2483171|AAAAAAAADODOFCAA|2086-08-03|2239|9736|747|2086|6|8|3|3|2086|747|9736|Saturday|2086Q3|N|Y|N|2483169|2483380|2482806|2483080|N|N|N|N|N| +2483172|AAAAAAAAEODOFCAA|2086-08-04|2239|9736|747|2086|0|8|4|3|2086|747|9736|Sunday|2086Q3|N|N|N|2483169|2483380|2482807|2483081|N|N|N|N|N| +2483173|AAAAAAAAFODOFCAA|2086-08-05|2239|9736|747|2086|1|8|5|3|2086|747|9736|Monday|2086Q3|N|N|N|2483169|2483380|2482808|2483082|N|N|N|N|N| +2483174|AAAAAAAAGODOFCAA|2086-08-06|2239|9737|747|2086|2|8|6|3|2086|747|9737|Tuesday|2086Q3|N|N|N|2483169|2483380|2482809|2483083|N|N|N|N|N| +2483175|AAAAAAAAHODOFCAA|2086-08-07|2239|9737|747|2086|3|8|7|3|2086|747|9737|Wednesday|2086Q3|N|N|N|2483169|2483380|2482810|2483084|N|N|N|N|N| +2483176|AAAAAAAAIODOFCAA|2086-08-08|2239|9737|747|2086|4|8|8|3|2086|747|9737|Thursday|2086Q3|N|N|N|2483169|2483380|2482811|2483085|N|N|N|N|N| +2483177|AAAAAAAAJODOFCAA|2086-08-09|2239|9737|747|2086|5|8|9|3|2086|747|9737|Friday|2086Q3|N|Y|N|2483169|2483380|2482812|2483086|N|N|N|N|N| +2483178|AAAAAAAAKODOFCAA|2086-08-10|2239|9737|747|2086|6|8|10|3|2086|747|9737|Saturday|2086Q3|N|Y|N|2483169|2483380|2482813|2483087|N|N|N|N|N| +2483179|AAAAAAAALODOFCAA|2086-08-11|2239|9737|747|2086|0|8|11|3|2086|747|9737|Sunday|2086Q3|N|N|N|2483169|2483380|2482814|2483088|N|N|N|N|N| +2483180|AAAAAAAAMODOFCAA|2086-08-12|2239|9737|747|2086|1|8|12|3|2086|747|9737|Monday|2086Q3|N|N|N|2483169|2483380|2482815|2483089|N|N|N|N|N| +2483181|AAAAAAAANODOFCAA|2086-08-13|2239|9738|747|2086|2|8|13|3|2086|747|9738|Tuesday|2086Q3|N|N|N|2483169|2483380|2482816|2483090|N|N|N|N|N| +2483182|AAAAAAAAOODOFCAA|2086-08-14|2239|9738|747|2086|3|8|14|3|2086|747|9738|Wednesday|2086Q3|N|N|N|2483169|2483380|2482817|2483091|N|N|N|N|N| +2483183|AAAAAAAAPODOFCAA|2086-08-15|2239|9738|747|2086|4|8|15|3|2086|747|9738|Thursday|2086Q3|N|N|N|2483169|2483380|2482818|2483092|N|N|N|N|N| +2483184|AAAAAAAAAPDOFCAA|2086-08-16|2239|9738|747|2086|5|8|16|3|2086|747|9738|Friday|2086Q3|N|Y|N|2483169|2483380|2482819|2483093|N|N|N|N|N| +2483185|AAAAAAAABPDOFCAA|2086-08-17|2239|9738|747|2086|6|8|17|3|2086|747|9738|Saturday|2086Q3|N|Y|N|2483169|2483380|2482820|2483094|N|N|N|N|N| +2483186|AAAAAAAACPDOFCAA|2086-08-18|2239|9738|747|2086|0|8|18|3|2086|747|9738|Sunday|2086Q3|N|N|N|2483169|2483380|2482821|2483095|N|N|N|N|N| +2483187|AAAAAAAADPDOFCAA|2086-08-19|2239|9738|747|2086|1|8|19|3|2086|747|9738|Monday|2086Q3|N|N|N|2483169|2483380|2482822|2483096|N|N|N|N|N| +2483188|AAAAAAAAEPDOFCAA|2086-08-20|2239|9739|747|2086|2|8|20|3|2086|747|9739|Tuesday|2086Q3|N|N|N|2483169|2483380|2482823|2483097|N|N|N|N|N| +2483189|AAAAAAAAFPDOFCAA|2086-08-21|2239|9739|747|2086|3|8|21|3|2086|747|9739|Wednesday|2086Q3|N|N|N|2483169|2483380|2482824|2483098|N|N|N|N|N| +2483190|AAAAAAAAGPDOFCAA|2086-08-22|2239|9739|747|2086|4|8|22|3|2086|747|9739|Thursday|2086Q3|N|N|N|2483169|2483380|2482825|2483099|N|N|N|N|N| +2483191|AAAAAAAAHPDOFCAA|2086-08-23|2239|9739|747|2086|5|8|23|3|2086|747|9739|Friday|2086Q3|N|Y|N|2483169|2483380|2482826|2483100|N|N|N|N|N| +2483192|AAAAAAAAIPDOFCAA|2086-08-24|2239|9739|747|2086|6|8|24|3|2086|747|9739|Saturday|2086Q3|N|Y|N|2483169|2483380|2482827|2483101|N|N|N|N|N| +2483193|AAAAAAAAJPDOFCAA|2086-08-25|2239|9739|747|2086|0|8|25|3|2086|747|9739|Sunday|2086Q3|N|N|N|2483169|2483380|2482828|2483102|N|N|N|N|N| +2483194|AAAAAAAAKPDOFCAA|2086-08-26|2239|9739|747|2086|1|8|26|3|2086|747|9739|Monday|2086Q3|N|N|N|2483169|2483380|2482829|2483103|N|N|N|N|N| +2483195|AAAAAAAALPDOFCAA|2086-08-27|2239|9740|747|2086|2|8|27|3|2086|747|9740|Tuesday|2086Q3|N|N|N|2483169|2483380|2482830|2483104|N|N|N|N|N| +2483196|AAAAAAAAMPDOFCAA|2086-08-28|2239|9740|747|2086|3|8|28|3|2086|747|9740|Wednesday|2086Q3|N|N|N|2483169|2483380|2482831|2483105|N|N|N|N|N| +2483197|AAAAAAAANPDOFCAA|2086-08-29|2239|9740|747|2086|4|8|29|3|2086|747|9740|Thursday|2086Q3|N|N|N|2483169|2483380|2482832|2483106|N|N|N|N|N| +2483198|AAAAAAAAOPDOFCAA|2086-08-30|2239|9740|747|2086|5|8|30|3|2086|747|9740|Friday|2086Q3|N|Y|N|2483169|2483380|2482833|2483107|N|N|N|N|N| +2483199|AAAAAAAAPPDOFCAA|2086-08-31|2239|9740|747|2086|6|8|31|3|2086|747|9740|Saturday|2086Q3|N|Y|N|2483169|2483380|2482834|2483108|N|N|N|N|N| +2483200|AAAAAAAAAAEOFCAA|2086-09-01|2240|9740|748|2086|0|9|1|3|2086|748|9740|Sunday|2086Q3|N|N|N|2483200|2483442|2482835|2483109|N|N|N|N|N| +2483201|AAAAAAAABAEOFCAA|2086-09-02|2240|9740|748|2086|1|9|2|3|2086|748|9740|Monday|2086Q3|N|N|N|2483200|2483442|2482836|2483110|N|N|N|N|N| +2483202|AAAAAAAACAEOFCAA|2086-09-03|2240|9741|748|2086|2|9|3|3|2086|748|9741|Tuesday|2086Q3|N|N|N|2483200|2483442|2482837|2483111|N|N|N|N|N| +2483203|AAAAAAAADAEOFCAA|2086-09-04|2240|9741|748|2086|3|9|4|3|2086|748|9741|Wednesday|2086Q3|N|N|N|2483200|2483442|2482838|2483112|N|N|N|N|N| +2483204|AAAAAAAAEAEOFCAA|2086-09-05|2240|9741|748|2086|4|9|5|3|2086|748|9741|Thursday|2086Q3|N|N|N|2483200|2483442|2482839|2483113|N|N|N|N|N| +2483205|AAAAAAAAFAEOFCAA|2086-09-06|2240|9741|748|2086|5|9|6|3|2086|748|9741|Friday|2086Q3|N|Y|N|2483200|2483442|2482840|2483114|N|N|N|N|N| +2483206|AAAAAAAAGAEOFCAA|2086-09-07|2240|9741|748|2086|6|9|7|3|2086|748|9741|Saturday|2086Q3|N|Y|N|2483200|2483442|2482841|2483115|N|N|N|N|N| +2483207|AAAAAAAAHAEOFCAA|2086-09-08|2240|9741|748|2086|0|9|8|3|2086|748|9741|Sunday|2086Q3|N|N|N|2483200|2483442|2482842|2483116|N|N|N|N|N| +2483208|AAAAAAAAIAEOFCAA|2086-09-09|2240|9741|748|2086|1|9|9|3|2086|748|9741|Monday|2086Q3|N|N|N|2483200|2483442|2482843|2483117|N|N|N|N|N| +2483209|AAAAAAAAJAEOFCAA|2086-09-10|2240|9742|748|2086|2|9|10|3|2086|748|9742|Tuesday|2086Q3|N|N|N|2483200|2483442|2482844|2483118|N|N|N|N|N| +2483210|AAAAAAAAKAEOFCAA|2086-09-11|2240|9742|748|2086|3|9|11|3|2086|748|9742|Wednesday|2086Q3|N|N|N|2483200|2483442|2482845|2483119|N|N|N|N|N| +2483211|AAAAAAAALAEOFCAA|2086-09-12|2240|9742|748|2086|4|9|12|3|2086|748|9742|Thursday|2086Q3|N|N|N|2483200|2483442|2482846|2483120|N|N|N|N|N| +2483212|AAAAAAAAMAEOFCAA|2086-09-13|2240|9742|748|2086|5|9|13|3|2086|748|9742|Friday|2086Q3|N|Y|N|2483200|2483442|2482847|2483121|N|N|N|N|N| +2483213|AAAAAAAANAEOFCAA|2086-09-14|2240|9742|748|2086|6|9|14|3|2086|748|9742|Saturday|2086Q3|N|Y|N|2483200|2483442|2482848|2483122|N|N|N|N|N| +2483214|AAAAAAAAOAEOFCAA|2086-09-15|2240|9742|748|2086|0|9|15|3|2086|748|9742|Sunday|2086Q3|N|N|N|2483200|2483442|2482849|2483123|N|N|N|N|N| +2483215|AAAAAAAAPAEOFCAA|2086-09-16|2240|9742|748|2086|1|9|16|3|2086|748|9742|Monday|2086Q3|N|N|N|2483200|2483442|2482850|2483124|N|N|N|N|N| +2483216|AAAAAAAAABEOFCAA|2086-09-17|2240|9743|748|2086|2|9|17|3|2086|748|9743|Tuesday|2086Q3|N|N|N|2483200|2483442|2482851|2483125|N|N|N|N|N| +2483217|AAAAAAAABBEOFCAA|2086-09-18|2240|9743|748|2086|3|9|18|3|2086|748|9743|Wednesday|2086Q3|N|N|N|2483200|2483442|2482852|2483126|N|N|N|N|N| +2483218|AAAAAAAACBEOFCAA|2086-09-19|2240|9743|748|2086|4|9|19|3|2086|748|9743|Thursday|2086Q3|N|N|N|2483200|2483442|2482853|2483127|N|N|N|N|N| +2483219|AAAAAAAADBEOFCAA|2086-09-20|2240|9743|748|2086|5|9|20|3|2086|748|9743|Friday|2086Q3|N|Y|N|2483200|2483442|2482854|2483128|N|N|N|N|N| +2483220|AAAAAAAAEBEOFCAA|2086-09-21|2240|9743|748|2086|6|9|21|3|2086|748|9743|Saturday|2086Q3|N|Y|N|2483200|2483442|2482855|2483129|N|N|N|N|N| +2483221|AAAAAAAAFBEOFCAA|2086-09-22|2240|9743|748|2086|0|9|22|3|2086|748|9743|Sunday|2086Q3|N|N|N|2483200|2483442|2482856|2483130|N|N|N|N|N| +2483222|AAAAAAAAGBEOFCAA|2086-09-23|2240|9743|748|2086|1|9|23|3|2086|748|9743|Monday|2086Q3|N|N|N|2483200|2483442|2482857|2483131|N|N|N|N|N| +2483223|AAAAAAAAHBEOFCAA|2086-09-24|2240|9744|748|2086|2|9|24|3|2086|748|9744|Tuesday|2086Q3|N|N|N|2483200|2483442|2482858|2483132|N|N|N|N|N| +2483224|AAAAAAAAIBEOFCAA|2086-09-25|2240|9744|748|2086|3|9|25|3|2086|748|9744|Wednesday|2086Q3|N|N|N|2483200|2483442|2482859|2483133|N|N|N|N|N| +2483225|AAAAAAAAJBEOFCAA|2086-09-26|2240|9744|748|2086|4|9|26|3|2086|748|9744|Thursday|2086Q3|N|N|N|2483200|2483442|2482860|2483134|N|N|N|N|N| +2483226|AAAAAAAAKBEOFCAA|2086-09-27|2240|9744|748|2086|5|9|27|3|2086|748|9744|Friday|2086Q3|N|Y|N|2483200|2483442|2482861|2483135|N|N|N|N|N| +2483227|AAAAAAAALBEOFCAA|2086-09-28|2240|9744|748|2086|6|9|28|3|2086|748|9744|Saturday|2086Q3|N|Y|N|2483200|2483442|2482862|2483136|N|N|N|N|N| +2483228|AAAAAAAAMBEOFCAA|2086-09-29|2240|9744|748|2086|0|9|29|3|2086|748|9744|Sunday|2086Q3|N|N|N|2483200|2483442|2482863|2483137|N|N|N|N|N| +2483229|AAAAAAAANBEOFCAA|2086-09-30|2240|9744|748|2086|1|9|30|3|2086|748|9744|Monday|2086Q3|N|N|N|2483200|2483442|2482864|2483138|N|N|N|N|N| +2483230|AAAAAAAAOBEOFCAA|2086-10-01|2241|9745|748|2086|2|10|1|3|2086|748|9745|Tuesday|2086Q3|N|N|N|2483230|2483502|2482865|2483138|N|N|N|N|N| +2483231|AAAAAAAAPBEOFCAA|2086-10-02|2241|9745|748|2086|3|10|2|4|2086|748|9745|Wednesday|2086Q4|N|N|N|2483230|2483502|2482866|2483139|N|N|N|N|N| +2483232|AAAAAAAAACEOFCAA|2086-10-03|2241|9745|748|2086|4|10|3|4|2086|748|9745|Thursday|2086Q4|N|N|N|2483230|2483502|2482867|2483140|N|N|N|N|N| +2483233|AAAAAAAABCEOFCAA|2086-10-04|2241|9745|748|2086|5|10|4|4|2086|748|9745|Friday|2086Q4|N|Y|N|2483230|2483502|2482868|2483141|N|N|N|N|N| +2483234|AAAAAAAACCEOFCAA|2086-10-05|2241|9745|748|2086|6|10|5|4|2086|748|9745|Saturday|2086Q4|N|Y|N|2483230|2483502|2482869|2483142|N|N|N|N|N| +2483235|AAAAAAAADCEOFCAA|2086-10-06|2241|9745|748|2086|0|10|6|4|2086|748|9745|Sunday|2086Q4|N|N|N|2483230|2483502|2482870|2483143|N|N|N|N|N| +2483236|AAAAAAAAECEOFCAA|2086-10-07|2241|9745|748|2086|1|10|7|4|2086|748|9745|Monday|2086Q4|N|N|N|2483230|2483502|2482871|2483144|N|N|N|N|N| +2483237|AAAAAAAAFCEOFCAA|2086-10-08|2241|9746|748|2086|2|10|8|4|2086|748|9746|Tuesday|2086Q4|N|N|N|2483230|2483502|2482872|2483145|N|N|N|N|N| +2483238|AAAAAAAAGCEOFCAA|2086-10-09|2241|9746|748|2086|3|10|9|4|2086|748|9746|Wednesday|2086Q4|N|N|N|2483230|2483502|2482873|2483146|N|N|N|N|N| +2483239|AAAAAAAAHCEOFCAA|2086-10-10|2241|9746|748|2086|4|10|10|4|2086|748|9746|Thursday|2086Q4|N|N|N|2483230|2483502|2482874|2483147|N|N|N|N|N| +2483240|AAAAAAAAICEOFCAA|2086-10-11|2241|9746|748|2086|5|10|11|4|2086|748|9746|Friday|2086Q4|N|Y|N|2483230|2483502|2482875|2483148|N|N|N|N|N| +2483241|AAAAAAAAJCEOFCAA|2086-10-12|2241|9746|748|2086|6|10|12|4|2086|748|9746|Saturday|2086Q4|N|Y|N|2483230|2483502|2482876|2483149|N|N|N|N|N| +2483242|AAAAAAAAKCEOFCAA|2086-10-13|2241|9746|748|2086|0|10|13|4|2086|748|9746|Sunday|2086Q4|N|N|N|2483230|2483502|2482877|2483150|N|N|N|N|N| +2483243|AAAAAAAALCEOFCAA|2086-10-14|2241|9746|748|2086|1|10|14|4|2086|748|9746|Monday|2086Q4|N|N|N|2483230|2483502|2482878|2483151|N|N|N|N|N| +2483244|AAAAAAAAMCEOFCAA|2086-10-15|2241|9747|748|2086|2|10|15|4|2086|748|9747|Tuesday|2086Q4|N|N|N|2483230|2483502|2482879|2483152|N|N|N|N|N| +2483245|AAAAAAAANCEOFCAA|2086-10-16|2241|9747|748|2086|3|10|16|4|2086|748|9747|Wednesday|2086Q4|N|N|N|2483230|2483502|2482880|2483153|N|N|N|N|N| +2483246|AAAAAAAAOCEOFCAA|2086-10-17|2241|9747|748|2086|4|10|17|4|2086|748|9747|Thursday|2086Q4|N|N|N|2483230|2483502|2482881|2483154|N|N|N|N|N| +2483247|AAAAAAAAPCEOFCAA|2086-10-18|2241|9747|748|2086|5|10|18|4|2086|748|9747|Friday|2086Q4|N|Y|N|2483230|2483502|2482882|2483155|N|N|N|N|N| +2483248|AAAAAAAAADEOFCAA|2086-10-19|2241|9747|748|2086|6|10|19|4|2086|748|9747|Saturday|2086Q4|N|Y|N|2483230|2483502|2482883|2483156|N|N|N|N|N| +2483249|AAAAAAAABDEOFCAA|2086-10-20|2241|9747|748|2086|0|10|20|4|2086|748|9747|Sunday|2086Q4|N|N|N|2483230|2483502|2482884|2483157|N|N|N|N|N| +2483250|AAAAAAAACDEOFCAA|2086-10-21|2241|9747|748|2086|1|10|21|4|2086|748|9747|Monday|2086Q4|N|N|N|2483230|2483502|2482885|2483158|N|N|N|N|N| +2483251|AAAAAAAADDEOFCAA|2086-10-22|2241|9748|748|2086|2|10|22|4|2086|748|9748|Tuesday|2086Q4|N|N|N|2483230|2483502|2482886|2483159|N|N|N|N|N| +2483252|AAAAAAAAEDEOFCAA|2086-10-23|2241|9748|748|2086|3|10|23|4|2086|748|9748|Wednesday|2086Q4|N|N|N|2483230|2483502|2482887|2483160|N|N|N|N|N| +2483253|AAAAAAAAFDEOFCAA|2086-10-24|2241|9748|748|2086|4|10|24|4|2086|748|9748|Thursday|2086Q4|N|N|N|2483230|2483502|2482888|2483161|N|N|N|N|N| +2483254|AAAAAAAAGDEOFCAA|2086-10-25|2241|9748|748|2086|5|10|25|4|2086|748|9748|Friday|2086Q4|N|Y|N|2483230|2483502|2482889|2483162|N|N|N|N|N| +2483255|AAAAAAAAHDEOFCAA|2086-10-26|2241|9748|748|2086|6|10|26|4|2086|748|9748|Saturday|2086Q4|N|Y|N|2483230|2483502|2482890|2483163|N|N|N|N|N| +2483256|AAAAAAAAIDEOFCAA|2086-10-27|2241|9748|748|2086|0|10|27|4|2086|748|9748|Sunday|2086Q4|N|N|N|2483230|2483502|2482891|2483164|N|N|N|N|N| +2483257|AAAAAAAAJDEOFCAA|2086-10-28|2241|9748|748|2086|1|10|28|4|2086|748|9748|Monday|2086Q4|N|N|N|2483230|2483502|2482892|2483165|N|N|N|N|N| +2483258|AAAAAAAAKDEOFCAA|2086-10-29|2241|9749|748|2086|2|10|29|4|2086|748|9749|Tuesday|2086Q4|N|N|N|2483230|2483502|2482893|2483166|N|N|N|N|N| +2483259|AAAAAAAALDEOFCAA|2086-10-30|2241|9749|748|2086|3|10|30|4|2086|748|9749|Wednesday|2086Q4|N|N|N|2483230|2483502|2482894|2483167|N|N|N|N|N| +2483260|AAAAAAAAMDEOFCAA|2086-10-31|2241|9749|748|2086|4|10|31|4|2086|748|9749|Thursday|2086Q4|N|N|N|2483230|2483502|2482895|2483168|N|N|N|N|N| +2483261|AAAAAAAANDEOFCAA|2086-11-01|2242|9749|748|2086|5|11|1|4|2086|748|9749|Friday|2086Q4|N|Y|N|2483261|2483564|2482896|2483169|N|N|N|N|N| +2483262|AAAAAAAAODEOFCAA|2086-11-02|2242|9749|748|2086|6|11|2|4|2086|748|9749|Saturday|2086Q4|N|Y|N|2483261|2483564|2482897|2483170|N|N|N|N|N| +2483263|AAAAAAAAPDEOFCAA|2086-11-03|2242|9749|748|2086|0|11|3|4|2086|748|9749|Sunday|2086Q4|N|N|N|2483261|2483564|2482898|2483171|N|N|N|N|N| +2483264|AAAAAAAAAEEOFCAA|2086-11-04|2242|9749|748|2086|1|11|4|4|2086|748|9749|Monday|2086Q4|N|N|N|2483261|2483564|2482899|2483172|N|N|N|N|N| +2483265|AAAAAAAABEEOFCAA|2086-11-05|2242|9750|748|2086|2|11|5|4|2086|748|9750|Tuesday|2086Q4|N|N|N|2483261|2483564|2482900|2483173|N|N|N|N|N| +2483266|AAAAAAAACEEOFCAA|2086-11-06|2242|9750|748|2086|3|11|6|4|2086|748|9750|Wednesday|2086Q4|N|N|N|2483261|2483564|2482901|2483174|N|N|N|N|N| +2483267|AAAAAAAADEEOFCAA|2086-11-07|2242|9750|748|2086|4|11|7|4|2086|748|9750|Thursday|2086Q4|N|N|N|2483261|2483564|2482902|2483175|N|N|N|N|N| +2483268|AAAAAAAAEEEOFCAA|2086-11-08|2242|9750|748|2086|5|11|8|4|2086|748|9750|Friday|2086Q4|N|Y|N|2483261|2483564|2482903|2483176|N|N|N|N|N| +2483269|AAAAAAAAFEEOFCAA|2086-11-09|2242|9750|748|2086|6|11|9|4|2086|748|9750|Saturday|2086Q4|N|Y|N|2483261|2483564|2482904|2483177|N|N|N|N|N| +2483270|AAAAAAAAGEEOFCAA|2086-11-10|2242|9750|748|2086|0|11|10|4|2086|748|9750|Sunday|2086Q4|N|N|N|2483261|2483564|2482905|2483178|N|N|N|N|N| +2483271|AAAAAAAAHEEOFCAA|2086-11-11|2242|9750|748|2086|1|11|11|4|2086|748|9750|Monday|2086Q4|N|N|N|2483261|2483564|2482906|2483179|N|N|N|N|N| +2483272|AAAAAAAAIEEOFCAA|2086-11-12|2242|9751|748|2086|2|11|12|4|2086|748|9751|Tuesday|2086Q4|N|N|N|2483261|2483564|2482907|2483180|N|N|N|N|N| +2483273|AAAAAAAAJEEOFCAA|2086-11-13|2242|9751|748|2086|3|11|13|4|2086|748|9751|Wednesday|2086Q4|N|N|N|2483261|2483564|2482908|2483181|N|N|N|N|N| +2483274|AAAAAAAAKEEOFCAA|2086-11-14|2242|9751|748|2086|4|11|14|4|2086|748|9751|Thursday|2086Q4|N|N|N|2483261|2483564|2482909|2483182|N|N|N|N|N| +2483275|AAAAAAAALEEOFCAA|2086-11-15|2242|9751|748|2086|5|11|15|4|2086|748|9751|Friday|2086Q4|N|Y|N|2483261|2483564|2482910|2483183|N|N|N|N|N| +2483276|AAAAAAAAMEEOFCAA|2086-11-16|2242|9751|748|2086|6|11|16|4|2086|748|9751|Saturday|2086Q4|N|Y|N|2483261|2483564|2482911|2483184|N|N|N|N|N| +2483277|AAAAAAAANEEOFCAA|2086-11-17|2242|9751|748|2086|0|11|17|4|2086|748|9751|Sunday|2086Q4|N|N|N|2483261|2483564|2482912|2483185|N|N|N|N|N| +2483278|AAAAAAAAOEEOFCAA|2086-11-18|2242|9751|748|2086|1|11|18|4|2086|748|9751|Monday|2086Q4|N|N|N|2483261|2483564|2482913|2483186|N|N|N|N|N| +2483279|AAAAAAAAPEEOFCAA|2086-11-19|2242|9752|748|2086|2|11|19|4|2086|748|9752|Tuesday|2086Q4|N|N|N|2483261|2483564|2482914|2483187|N|N|N|N|N| +2483280|AAAAAAAAAFEOFCAA|2086-11-20|2242|9752|748|2086|3|11|20|4|2086|748|9752|Wednesday|2086Q4|N|N|N|2483261|2483564|2482915|2483188|N|N|N|N|N| +2483281|AAAAAAAABFEOFCAA|2086-11-21|2242|9752|748|2086|4|11|21|4|2086|748|9752|Thursday|2086Q4|N|N|N|2483261|2483564|2482916|2483189|N|N|N|N|N| +2483282|AAAAAAAACFEOFCAA|2086-11-22|2242|9752|748|2086|5|11|22|4|2086|748|9752|Friday|2086Q4|N|Y|N|2483261|2483564|2482917|2483190|N|N|N|N|N| +2483283|AAAAAAAADFEOFCAA|2086-11-23|2242|9752|748|2086|6|11|23|4|2086|748|9752|Saturday|2086Q4|N|Y|N|2483261|2483564|2482918|2483191|N|N|N|N|N| +2483284|AAAAAAAAEFEOFCAA|2086-11-24|2242|9752|748|2086|0|11|24|4|2086|748|9752|Sunday|2086Q4|N|N|N|2483261|2483564|2482919|2483192|N|N|N|N|N| +2483285|AAAAAAAAFFEOFCAA|2086-11-25|2242|9752|748|2086|1|11|25|4|2086|748|9752|Monday|2086Q4|N|N|N|2483261|2483564|2482920|2483193|N|N|N|N|N| +2483286|AAAAAAAAGFEOFCAA|2086-11-26|2242|9753|748|2086|2|11|26|4|2086|748|9753|Tuesday|2086Q4|N|N|N|2483261|2483564|2482921|2483194|N|N|N|N|N| +2483287|AAAAAAAAHFEOFCAA|2086-11-27|2242|9753|748|2086|3|11|27|4|2086|748|9753|Wednesday|2086Q4|N|N|N|2483261|2483564|2482922|2483195|N|N|N|N|N| +2483288|AAAAAAAAIFEOFCAA|2086-11-28|2242|9753|748|2086|4|11|28|4|2086|748|9753|Thursday|2086Q4|N|N|N|2483261|2483564|2482923|2483196|N|N|N|N|N| +2483289|AAAAAAAAJFEOFCAA|2086-11-29|2242|9753|748|2086|5|11|29|4|2086|748|9753|Friday|2086Q4|N|Y|N|2483261|2483564|2482924|2483197|N|N|N|N|N| +2483290|AAAAAAAAKFEOFCAA|2086-11-30|2242|9753|748|2086|6|11|30|4|2086|748|9753|Saturday|2086Q4|N|Y|N|2483261|2483564|2482925|2483198|N|N|N|N|N| +2483291|AAAAAAAALFEOFCAA|2086-12-01|2243|9753|749|2086|0|12|1|4|2086|749|9753|Sunday|2086Q4|N|N|N|2483291|2483624|2482926|2483199|N|N|N|N|N| +2483292|AAAAAAAAMFEOFCAA|2086-12-02|2243|9753|749|2086|1|12|2|4|2086|749|9753|Monday|2086Q4|N|N|N|2483291|2483624|2482927|2483200|N|N|N|N|N| +2483293|AAAAAAAANFEOFCAA|2086-12-03|2243|9754|749|2086|2|12|3|4|2086|749|9754|Tuesday|2086Q4|N|N|N|2483291|2483624|2482928|2483201|N|N|N|N|N| +2483294|AAAAAAAAOFEOFCAA|2086-12-04|2243|9754|749|2086|3|12|4|4|2086|749|9754|Wednesday|2086Q4|N|N|N|2483291|2483624|2482929|2483202|N|N|N|N|N| +2483295|AAAAAAAAPFEOFCAA|2086-12-05|2243|9754|749|2086|4|12|5|4|2086|749|9754|Thursday|2086Q4|N|N|N|2483291|2483624|2482930|2483203|N|N|N|N|N| +2483296|AAAAAAAAAGEOFCAA|2086-12-06|2243|9754|749|2086|5|12|6|4|2086|749|9754|Friday|2086Q4|N|Y|N|2483291|2483624|2482931|2483204|N|N|N|N|N| +2483297|AAAAAAAABGEOFCAA|2086-12-07|2243|9754|749|2086|6|12|7|4|2086|749|9754|Saturday|2086Q4|N|Y|N|2483291|2483624|2482932|2483205|N|N|N|N|N| +2483298|AAAAAAAACGEOFCAA|2086-12-08|2243|9754|749|2086|0|12|8|4|2086|749|9754|Sunday|2086Q4|N|N|N|2483291|2483624|2482933|2483206|N|N|N|N|N| +2483299|AAAAAAAADGEOFCAA|2086-12-09|2243|9754|749|2086|1|12|9|4|2086|749|9754|Monday|2086Q4|N|N|N|2483291|2483624|2482934|2483207|N|N|N|N|N| +2483300|AAAAAAAAEGEOFCAA|2086-12-10|2243|9755|749|2086|2|12|10|4|2086|749|9755|Tuesday|2086Q4|N|N|N|2483291|2483624|2482935|2483208|N|N|N|N|N| +2483301|AAAAAAAAFGEOFCAA|2086-12-11|2243|9755|749|2086|3|12|11|4|2086|749|9755|Wednesday|2086Q4|N|N|N|2483291|2483624|2482936|2483209|N|N|N|N|N| +2483302|AAAAAAAAGGEOFCAA|2086-12-12|2243|9755|749|2086|4|12|12|4|2086|749|9755|Thursday|2086Q4|N|N|N|2483291|2483624|2482937|2483210|N|N|N|N|N| +2483303|AAAAAAAAHGEOFCAA|2086-12-13|2243|9755|749|2086|5|12|13|4|2086|749|9755|Friday|2086Q4|N|Y|N|2483291|2483624|2482938|2483211|N|N|N|N|N| +2483304|AAAAAAAAIGEOFCAA|2086-12-14|2243|9755|749|2086|6|12|14|4|2086|749|9755|Saturday|2086Q4|N|Y|N|2483291|2483624|2482939|2483212|N|N|N|N|N| +2483305|AAAAAAAAJGEOFCAA|2086-12-15|2243|9755|749|2086|0|12|15|4|2086|749|9755|Sunday|2086Q4|N|N|N|2483291|2483624|2482940|2483213|N|N|N|N|N| +2483306|AAAAAAAAKGEOFCAA|2086-12-16|2243|9755|749|2086|1|12|16|4|2086|749|9755|Monday|2086Q4|N|N|N|2483291|2483624|2482941|2483214|N|N|N|N|N| +2483307|AAAAAAAALGEOFCAA|2086-12-17|2243|9756|749|2086|2|12|17|4|2086|749|9756|Tuesday|2086Q4|N|N|N|2483291|2483624|2482942|2483215|N|N|N|N|N| +2483308|AAAAAAAAMGEOFCAA|2086-12-18|2243|9756|749|2086|3|12|18|4|2086|749|9756|Wednesday|2086Q4|N|N|N|2483291|2483624|2482943|2483216|N|N|N|N|N| +2483309|AAAAAAAANGEOFCAA|2086-12-19|2243|9756|749|2086|4|12|19|4|2086|749|9756|Thursday|2086Q4|N|N|N|2483291|2483624|2482944|2483217|N|N|N|N|N| +2483310|AAAAAAAAOGEOFCAA|2086-12-20|2243|9756|749|2086|5|12|20|4|2086|749|9756|Friday|2086Q4|N|Y|N|2483291|2483624|2482945|2483218|N|N|N|N|N| +2483311|AAAAAAAAPGEOFCAA|2086-12-21|2243|9756|749|2086|6|12|21|4|2086|749|9756|Saturday|2086Q4|N|Y|N|2483291|2483624|2482946|2483219|N|N|N|N|N| +2483312|AAAAAAAAAHEOFCAA|2086-12-22|2243|9756|749|2086|0|12|22|4|2086|749|9756|Sunday|2086Q4|N|N|N|2483291|2483624|2482947|2483220|N|N|N|N|N| +2483313|AAAAAAAABHEOFCAA|2086-12-23|2243|9756|749|2086|1|12|23|4|2086|749|9756|Monday|2086Q4|N|N|N|2483291|2483624|2482948|2483221|N|N|N|N|N| +2483314|AAAAAAAACHEOFCAA|2086-12-24|2243|9757|749|2086|2|12|24|4|2086|749|9757|Tuesday|2086Q4|N|N|N|2483291|2483624|2482949|2483222|N|N|N|N|N| +2483315|AAAAAAAADHEOFCAA|2086-12-25|2243|9757|749|2086|3|12|25|4|2086|749|9757|Wednesday|2086Q4|N|N|N|2483291|2483624|2482950|2483223|N|N|N|N|N| +2483316|AAAAAAAAEHEOFCAA|2086-12-26|2243|9757|749|2086|4|12|26|4|2086|749|9757|Thursday|2086Q4|Y|N|N|2483291|2483624|2482951|2483224|N|N|N|N|N| +2483317|AAAAAAAAFHEOFCAA|2086-12-27|2243|9757|749|2086|5|12|27|4|2086|749|9757|Friday|2086Q4|N|Y|Y|2483291|2483624|2482952|2483225|N|N|N|N|N| +2483318|AAAAAAAAGHEOFCAA|2086-12-28|2243|9757|749|2086|6|12|28|4|2086|749|9757|Saturday|2086Q4|N|Y|N|2483291|2483624|2482953|2483226|N|N|N|N|N| +2483319|AAAAAAAAHHEOFCAA|2086-12-29|2243|9757|749|2086|0|12|29|4|2086|749|9757|Sunday|2086Q4|N|N|N|2483291|2483624|2482954|2483227|N|N|N|N|N| +2483320|AAAAAAAAIHEOFCAA|2086-12-30|2243|9757|749|2086|1|12|30|4|2086|749|9757|Monday|2086Q4|N|N|N|2483291|2483624|2482955|2483228|N|N|N|N|N| +2483321|AAAAAAAAJHEOFCAA|2086-12-31|2243|9758|749|2086|2|12|31|4|2086|749|9758|Tuesday|2086Q4|N|N|N|2483291|2483624|2482956|2483229|N|N|N|N|N| +2483322|AAAAAAAAKHEOFCAA|2087-01-01|2244|9758|749|2087|3|1|1|1|2087|749|9758|Wednesday|2087Q1|Y|N|N|2483322|2483321|2482957|2483230|N|N|N|N|N| +2483323|AAAAAAAALHEOFCAA|2087-01-02|2244|9758|749|2087|4|1|2|1|2087|749|9758|Thursday|2087Q1|N|N|Y|2483322|2483321|2482958|2483231|N|N|N|N|N| +2483324|AAAAAAAAMHEOFCAA|2087-01-03|2244|9758|749|2087|5|1|3|1|2087|749|9758|Friday|2087Q1|N|Y|N|2483322|2483321|2482959|2483232|N|N|N|N|N| +2483325|AAAAAAAANHEOFCAA|2087-01-04|2244|9758|749|2087|6|1|4|1|2087|749|9758|Saturday|2087Q1|N|Y|N|2483322|2483321|2482960|2483233|N|N|N|N|N| +2483326|AAAAAAAAOHEOFCAA|2087-01-05|2244|9758|749|2087|0|1|5|1|2087|749|9758|Sunday|2087Q1|N|N|N|2483322|2483321|2482961|2483234|N|N|N|N|N| +2483327|AAAAAAAAPHEOFCAA|2087-01-06|2244|9758|749|2087|1|1|6|1|2087|749|9758|Monday|2087Q1|N|N|N|2483322|2483321|2482962|2483235|N|N|N|N|N| +2483328|AAAAAAAAAIEOFCAA|2087-01-07|2244|9759|749|2087|2|1|7|1|2087|749|9759|Tuesday|2087Q1|N|N|N|2483322|2483321|2482963|2483236|N|N|N|N|N| +2483329|AAAAAAAABIEOFCAA|2087-01-08|2244|9759|749|2087|3|1|8|1|2087|749|9759|Wednesday|2087Q1|N|N|N|2483322|2483321|2482964|2483237|N|N|N|N|N| +2483330|AAAAAAAACIEOFCAA|2087-01-09|2244|9759|749|2087|4|1|9|1|2087|749|9759|Thursday|2087Q1|N|N|N|2483322|2483321|2482965|2483238|N|N|N|N|N| +2483331|AAAAAAAADIEOFCAA|2087-01-10|2244|9759|749|2087|5|1|10|1|2087|749|9759|Friday|2087Q1|N|Y|N|2483322|2483321|2482966|2483239|N|N|N|N|N| +2483332|AAAAAAAAEIEOFCAA|2087-01-11|2244|9759|749|2087|6|1|11|1|2087|749|9759|Saturday|2087Q1|N|Y|N|2483322|2483321|2482967|2483240|N|N|N|N|N| +2483333|AAAAAAAAFIEOFCAA|2087-01-12|2244|9759|749|2087|0|1|12|1|2087|749|9759|Sunday|2087Q1|N|N|N|2483322|2483321|2482968|2483241|N|N|N|N|N| +2483334|AAAAAAAAGIEOFCAA|2087-01-13|2244|9759|749|2087|1|1|13|1|2087|749|9759|Monday|2087Q1|N|N|N|2483322|2483321|2482969|2483242|N|N|N|N|N| +2483335|AAAAAAAAHIEOFCAA|2087-01-14|2244|9760|749|2087|2|1|14|1|2087|749|9760|Tuesday|2087Q1|N|N|N|2483322|2483321|2482970|2483243|N|N|N|N|N| +2483336|AAAAAAAAIIEOFCAA|2087-01-15|2244|9760|749|2087|3|1|15|1|2087|749|9760|Wednesday|2087Q1|N|N|N|2483322|2483321|2482971|2483244|N|N|N|N|N| +2483337|AAAAAAAAJIEOFCAA|2087-01-16|2244|9760|749|2087|4|1|16|1|2087|749|9760|Thursday|2087Q1|N|N|N|2483322|2483321|2482972|2483245|N|N|N|N|N| +2483338|AAAAAAAAKIEOFCAA|2087-01-17|2244|9760|749|2087|5|1|17|1|2087|749|9760|Friday|2087Q1|N|Y|N|2483322|2483321|2482973|2483246|N|N|N|N|N| +2483339|AAAAAAAALIEOFCAA|2087-01-18|2244|9760|749|2087|6|1|18|1|2087|749|9760|Saturday|2087Q1|N|Y|N|2483322|2483321|2482974|2483247|N|N|N|N|N| +2483340|AAAAAAAAMIEOFCAA|2087-01-19|2244|9760|749|2087|0|1|19|1|2087|749|9760|Sunday|2087Q1|N|N|N|2483322|2483321|2482975|2483248|N|N|N|N|N| +2483341|AAAAAAAANIEOFCAA|2087-01-20|2244|9760|749|2087|1|1|20|1|2087|749|9760|Monday|2087Q1|N|N|N|2483322|2483321|2482976|2483249|N|N|N|N|N| +2483342|AAAAAAAAOIEOFCAA|2087-01-21|2244|9761|749|2087|2|1|21|1|2087|749|9761|Tuesday|2087Q1|N|N|N|2483322|2483321|2482977|2483250|N|N|N|N|N| +2483343|AAAAAAAAPIEOFCAA|2087-01-22|2244|9761|749|2087|3|1|22|1|2087|749|9761|Wednesday|2087Q1|N|N|N|2483322|2483321|2482978|2483251|N|N|N|N|N| +2483344|AAAAAAAAAJEOFCAA|2087-01-23|2244|9761|749|2087|4|1|23|1|2087|749|9761|Thursday|2087Q1|N|N|N|2483322|2483321|2482979|2483252|N|N|N|N|N| +2483345|AAAAAAAABJEOFCAA|2087-01-24|2244|9761|749|2087|5|1|24|1|2087|749|9761|Friday|2087Q1|N|Y|N|2483322|2483321|2482980|2483253|N|N|N|N|N| +2483346|AAAAAAAACJEOFCAA|2087-01-25|2244|9761|749|2087|6|1|25|1|2087|749|9761|Saturday|2087Q1|N|Y|N|2483322|2483321|2482981|2483254|N|N|N|N|N| +2483347|AAAAAAAADJEOFCAA|2087-01-26|2244|9761|749|2087|0|1|26|1|2087|749|9761|Sunday|2087Q1|N|N|N|2483322|2483321|2482982|2483255|N|N|N|N|N| +2483348|AAAAAAAAEJEOFCAA|2087-01-27|2244|9761|749|2087|1|1|27|1|2087|749|9761|Monday|2087Q1|N|N|N|2483322|2483321|2482983|2483256|N|N|N|N|N| +2483349|AAAAAAAAFJEOFCAA|2087-01-28|2244|9762|749|2087|2|1|28|1|2087|749|9762|Tuesday|2087Q1|N|N|N|2483322|2483321|2482984|2483257|N|N|N|N|N| +2483350|AAAAAAAAGJEOFCAA|2087-01-29|2244|9762|749|2087|3|1|29|1|2087|749|9762|Wednesday|2087Q1|N|N|N|2483322|2483321|2482985|2483258|N|N|N|N|N| +2483351|AAAAAAAAHJEOFCAA|2087-01-30|2244|9762|749|2087|4|1|30|1|2087|749|9762|Thursday|2087Q1|N|N|N|2483322|2483321|2482986|2483259|N|N|N|N|N| +2483352|AAAAAAAAIJEOFCAA|2087-01-31|2244|9762|749|2087|5|1|31|1|2087|749|9762|Friday|2087Q1|N|Y|N|2483322|2483321|2482987|2483260|N|N|N|N|N| +2483353|AAAAAAAAJJEOFCAA|2087-02-01|2245|9762|749|2087|6|2|1|1|2087|749|9762|Saturday|2087Q1|N|Y|N|2483353|2483383|2482988|2483261|N|N|N|N|N| +2483354|AAAAAAAAKJEOFCAA|2087-02-02|2245|9762|749|2087|0|2|2|1|2087|749|9762|Sunday|2087Q1|N|N|N|2483353|2483383|2482989|2483262|N|N|N|N|N| +2483355|AAAAAAAALJEOFCAA|2087-02-03|2245|9762|749|2087|1|2|3|1|2087|749|9762|Monday|2087Q1|N|N|N|2483353|2483383|2482990|2483263|N|N|N|N|N| +2483356|AAAAAAAAMJEOFCAA|2087-02-04|2245|9763|749|2087|2|2|4|1|2087|749|9763|Tuesday|2087Q1|N|N|N|2483353|2483383|2482991|2483264|N|N|N|N|N| +2483357|AAAAAAAANJEOFCAA|2087-02-05|2245|9763|749|2087|3|2|5|1|2087|749|9763|Wednesday|2087Q1|N|N|N|2483353|2483383|2482992|2483265|N|N|N|N|N| +2483358|AAAAAAAAOJEOFCAA|2087-02-06|2245|9763|749|2087|4|2|6|1|2087|749|9763|Thursday|2087Q1|N|N|N|2483353|2483383|2482993|2483266|N|N|N|N|N| +2483359|AAAAAAAAPJEOFCAA|2087-02-07|2245|9763|749|2087|5|2|7|1|2087|749|9763|Friday|2087Q1|N|Y|N|2483353|2483383|2482994|2483267|N|N|N|N|N| +2483360|AAAAAAAAAKEOFCAA|2087-02-08|2245|9763|749|2087|6|2|8|1|2087|749|9763|Saturday|2087Q1|N|Y|N|2483353|2483383|2482995|2483268|N|N|N|N|N| +2483361|AAAAAAAABKEOFCAA|2087-02-09|2245|9763|749|2087|0|2|9|1|2087|749|9763|Sunday|2087Q1|N|N|N|2483353|2483383|2482996|2483269|N|N|N|N|N| +2483362|AAAAAAAACKEOFCAA|2087-02-10|2245|9763|749|2087|1|2|10|1|2087|749|9763|Monday|2087Q1|N|N|N|2483353|2483383|2482997|2483270|N|N|N|N|N| +2483363|AAAAAAAADKEOFCAA|2087-02-11|2245|9764|749|2087|2|2|11|1|2087|749|9764|Tuesday|2087Q1|N|N|N|2483353|2483383|2482998|2483271|N|N|N|N|N| +2483364|AAAAAAAAEKEOFCAA|2087-02-12|2245|9764|749|2087|3|2|12|1|2087|749|9764|Wednesday|2087Q1|N|N|N|2483353|2483383|2482999|2483272|N|N|N|N|N| +2483365|AAAAAAAAFKEOFCAA|2087-02-13|2245|9764|749|2087|4|2|13|1|2087|749|9764|Thursday|2087Q1|N|N|N|2483353|2483383|2483000|2483273|N|N|N|N|N| +2483366|AAAAAAAAGKEOFCAA|2087-02-14|2245|9764|749|2087|5|2|14|1|2087|749|9764|Friday|2087Q1|N|Y|N|2483353|2483383|2483001|2483274|N|N|N|N|N| +2483367|AAAAAAAAHKEOFCAA|2087-02-15|2245|9764|749|2087|6|2|15|1|2087|749|9764|Saturday|2087Q1|N|Y|N|2483353|2483383|2483002|2483275|N|N|N|N|N| +2483368|AAAAAAAAIKEOFCAA|2087-02-16|2245|9764|749|2087|0|2|16|1|2087|749|9764|Sunday|2087Q1|N|N|N|2483353|2483383|2483003|2483276|N|N|N|N|N| +2483369|AAAAAAAAJKEOFCAA|2087-02-17|2245|9764|749|2087|1|2|17|1|2087|749|9764|Monday|2087Q1|N|N|N|2483353|2483383|2483004|2483277|N|N|N|N|N| +2483370|AAAAAAAAKKEOFCAA|2087-02-18|2245|9765|749|2087|2|2|18|1|2087|749|9765|Tuesday|2087Q1|N|N|N|2483353|2483383|2483005|2483278|N|N|N|N|N| +2483371|AAAAAAAALKEOFCAA|2087-02-19|2245|9765|749|2087|3|2|19|1|2087|749|9765|Wednesday|2087Q1|N|N|N|2483353|2483383|2483006|2483279|N|N|N|N|N| +2483372|AAAAAAAAMKEOFCAA|2087-02-20|2245|9765|749|2087|4|2|20|1|2087|749|9765|Thursday|2087Q1|N|N|N|2483353|2483383|2483007|2483280|N|N|N|N|N| +2483373|AAAAAAAANKEOFCAA|2087-02-21|2245|9765|749|2087|5|2|21|1|2087|749|9765|Friday|2087Q1|N|Y|N|2483353|2483383|2483008|2483281|N|N|N|N|N| +2483374|AAAAAAAAOKEOFCAA|2087-02-22|2245|9765|749|2087|6|2|22|1|2087|749|9765|Saturday|2087Q1|N|Y|N|2483353|2483383|2483009|2483282|N|N|N|N|N| +2483375|AAAAAAAAPKEOFCAA|2087-02-23|2245|9765|749|2087|0|2|23|1|2087|749|9765|Sunday|2087Q1|N|N|N|2483353|2483383|2483010|2483283|N|N|N|N|N| +2483376|AAAAAAAAALEOFCAA|2087-02-24|2245|9765|749|2087|1|2|24|1|2087|749|9765|Monday|2087Q1|N|N|N|2483353|2483383|2483011|2483284|N|N|N|N|N| +2483377|AAAAAAAABLEOFCAA|2087-02-25|2245|9766|749|2087|2|2|25|1|2087|749|9766|Tuesday|2087Q1|N|N|N|2483353|2483383|2483012|2483285|N|N|N|N|N| +2483378|AAAAAAAACLEOFCAA|2087-02-26|2245|9766|749|2087|3|2|26|1|2087|749|9766|Wednesday|2087Q1|N|N|N|2483353|2483383|2483013|2483286|N|N|N|N|N| +2483379|AAAAAAAADLEOFCAA|2087-02-27|2245|9766|749|2087|4|2|27|1|2087|749|9766|Thursday|2087Q1|N|N|N|2483353|2483383|2483014|2483287|N|N|N|N|N| +2483380|AAAAAAAAELEOFCAA|2087-02-28|2245|9766|749|2087|5|2|28|1|2087|749|9766|Friday|2087Q1|N|Y|N|2483353|2483383|2483015|2483288|N|N|N|N|N| +2483381|AAAAAAAAFLEOFCAA|2087-03-01|2246|9766|750|2087|6|3|1|1|2087|750|9766|Saturday|2087Q1|N|Y|N|2483381|2483439|2483016|2483289|N|N|N|N|N| +2483382|AAAAAAAAGLEOFCAA|2087-03-02|2246|9766|750|2087|0|3|2|1|2087|750|9766|Sunday|2087Q1|N|N|N|2483381|2483439|2483017|2483290|N|N|N|N|N| +2483383|AAAAAAAAHLEOFCAA|2087-03-03|2246|9766|750|2087|1|3|3|1|2087|750|9766|Monday|2087Q1|N|N|N|2483381|2483439|2483018|2483291|N|N|N|N|N| +2483384|AAAAAAAAILEOFCAA|2087-03-04|2246|9767|750|2087|2|3|4|1|2087|750|9767|Tuesday|2087Q1|N|N|N|2483381|2483439|2483019|2483292|N|N|N|N|N| +2483385|AAAAAAAAJLEOFCAA|2087-03-05|2246|9767|750|2087|3|3|5|1|2087|750|9767|Wednesday|2087Q1|N|N|N|2483381|2483439|2483020|2483293|N|N|N|N|N| +2483386|AAAAAAAAKLEOFCAA|2087-03-06|2246|9767|750|2087|4|3|6|1|2087|750|9767|Thursday|2087Q1|N|N|N|2483381|2483439|2483021|2483294|N|N|N|N|N| +2483387|AAAAAAAALLEOFCAA|2087-03-07|2246|9767|750|2087|5|3|7|1|2087|750|9767|Friday|2087Q1|N|Y|N|2483381|2483439|2483022|2483295|N|N|N|N|N| +2483388|AAAAAAAAMLEOFCAA|2087-03-08|2246|9767|750|2087|6|3|8|1|2087|750|9767|Saturday|2087Q1|N|Y|N|2483381|2483439|2483023|2483296|N|N|N|N|N| +2483389|AAAAAAAANLEOFCAA|2087-03-09|2246|9767|750|2087|0|3|9|1|2087|750|9767|Sunday|2087Q1|N|N|N|2483381|2483439|2483024|2483297|N|N|N|N|N| +2483390|AAAAAAAAOLEOFCAA|2087-03-10|2246|9767|750|2087|1|3|10|1|2087|750|9767|Monday|2087Q1|N|N|N|2483381|2483439|2483025|2483298|N|N|N|N|N| +2483391|AAAAAAAAPLEOFCAA|2087-03-11|2246|9768|750|2087|2|3|11|1|2087|750|9768|Tuesday|2087Q1|N|N|N|2483381|2483439|2483026|2483299|N|N|N|N|N| +2483392|AAAAAAAAAMEOFCAA|2087-03-12|2246|9768|750|2087|3|3|12|1|2087|750|9768|Wednesday|2087Q1|N|N|N|2483381|2483439|2483027|2483300|N|N|N|N|N| +2483393|AAAAAAAABMEOFCAA|2087-03-13|2246|9768|750|2087|4|3|13|1|2087|750|9768|Thursday|2087Q1|N|N|N|2483381|2483439|2483028|2483301|N|N|N|N|N| +2483394|AAAAAAAACMEOFCAA|2087-03-14|2246|9768|750|2087|5|3|14|1|2087|750|9768|Friday|2087Q1|N|Y|N|2483381|2483439|2483029|2483302|N|N|N|N|N| +2483395|AAAAAAAADMEOFCAA|2087-03-15|2246|9768|750|2087|6|3|15|1|2087|750|9768|Saturday|2087Q1|N|Y|N|2483381|2483439|2483030|2483303|N|N|N|N|N| +2483396|AAAAAAAAEMEOFCAA|2087-03-16|2246|9768|750|2087|0|3|16|1|2087|750|9768|Sunday|2087Q1|N|N|N|2483381|2483439|2483031|2483304|N|N|N|N|N| +2483397|AAAAAAAAFMEOFCAA|2087-03-17|2246|9768|750|2087|1|3|17|1|2087|750|9768|Monday|2087Q1|N|N|N|2483381|2483439|2483032|2483305|N|N|N|N|N| +2483398|AAAAAAAAGMEOFCAA|2087-03-18|2246|9769|750|2087|2|3|18|1|2087|750|9769|Tuesday|2087Q1|N|N|N|2483381|2483439|2483033|2483306|N|N|N|N|N| +2483399|AAAAAAAAHMEOFCAA|2087-03-19|2246|9769|750|2087|3|3|19|1|2087|750|9769|Wednesday|2087Q1|N|N|N|2483381|2483439|2483034|2483307|N|N|N|N|N| +2483400|AAAAAAAAIMEOFCAA|2087-03-20|2246|9769|750|2087|4|3|20|1|2087|750|9769|Thursday|2087Q1|N|N|N|2483381|2483439|2483035|2483308|N|N|N|N|N| +2483401|AAAAAAAAJMEOFCAA|2087-03-21|2246|9769|750|2087|5|3|21|1|2087|750|9769|Friday|2087Q1|N|Y|N|2483381|2483439|2483036|2483309|N|N|N|N|N| +2483402|AAAAAAAAKMEOFCAA|2087-03-22|2246|9769|750|2087|6|3|22|1|2087|750|9769|Saturday|2087Q1|N|Y|N|2483381|2483439|2483037|2483310|N|N|N|N|N| +2483403|AAAAAAAALMEOFCAA|2087-03-23|2246|9769|750|2087|0|3|23|1|2087|750|9769|Sunday|2087Q1|N|N|N|2483381|2483439|2483038|2483311|N|N|N|N|N| +2483404|AAAAAAAAMMEOFCAA|2087-03-24|2246|9769|750|2087|1|3|24|1|2087|750|9769|Monday|2087Q1|N|N|N|2483381|2483439|2483039|2483312|N|N|N|N|N| +2483405|AAAAAAAANMEOFCAA|2087-03-25|2246|9770|750|2087|2|3|25|1|2087|750|9770|Tuesday|2087Q1|N|N|N|2483381|2483439|2483040|2483313|N|N|N|N|N| +2483406|AAAAAAAAOMEOFCAA|2087-03-26|2246|9770|750|2087|3|3|26|1|2087|750|9770|Wednesday|2087Q1|N|N|N|2483381|2483439|2483041|2483314|N|N|N|N|N| +2483407|AAAAAAAAPMEOFCAA|2087-03-27|2246|9770|750|2087|4|3|27|1|2087|750|9770|Thursday|2087Q1|N|N|N|2483381|2483439|2483042|2483315|N|N|N|N|N| +2483408|AAAAAAAAANEOFCAA|2087-03-28|2246|9770|750|2087|5|3|28|1|2087|750|9770|Friday|2087Q1|N|Y|N|2483381|2483439|2483043|2483316|N|N|N|N|N| +2483409|AAAAAAAABNEOFCAA|2087-03-29|2246|9770|750|2087|6|3|29|1|2087|750|9770|Saturday|2087Q1|N|Y|N|2483381|2483439|2483044|2483317|N|N|N|N|N| +2483410|AAAAAAAACNEOFCAA|2087-03-30|2246|9770|750|2087|0|3|30|1|2087|750|9770|Sunday|2087Q1|N|N|N|2483381|2483439|2483045|2483318|N|N|N|N|N| +2483411|AAAAAAAADNEOFCAA|2087-03-31|2246|9770|750|2087|1|3|31|1|2087|750|9770|Monday|2087Q1|N|N|N|2483381|2483439|2483046|2483319|N|N|N|N|N| +2483412|AAAAAAAAENEOFCAA|2087-04-01|2247|9771|750|2087|2|4|1|1|2087|750|9771|Tuesday|2087Q1|N|N|N|2483412|2483501|2483047|2483322|N|N|N|N|N| +2483413|AAAAAAAAFNEOFCAA|2087-04-02|2247|9771|750|2087|3|4|2|2|2087|750|9771|Wednesday|2087Q2|N|N|N|2483412|2483501|2483048|2483323|N|N|N|N|N| +2483414|AAAAAAAAGNEOFCAA|2087-04-03|2247|9771|750|2087|4|4|3|2|2087|750|9771|Thursday|2087Q2|N|N|N|2483412|2483501|2483049|2483324|N|N|N|N|N| +2483415|AAAAAAAAHNEOFCAA|2087-04-04|2247|9771|750|2087|5|4|4|2|2087|750|9771|Friday|2087Q2|N|Y|N|2483412|2483501|2483050|2483325|N|N|N|N|N| +2483416|AAAAAAAAINEOFCAA|2087-04-05|2247|9771|750|2087|6|4|5|2|2087|750|9771|Saturday|2087Q2|N|Y|N|2483412|2483501|2483051|2483326|N|N|N|N|N| +2483417|AAAAAAAAJNEOFCAA|2087-04-06|2247|9771|750|2087|0|4|6|2|2087|750|9771|Sunday|2087Q2|N|N|N|2483412|2483501|2483052|2483327|N|N|N|N|N| +2483418|AAAAAAAAKNEOFCAA|2087-04-07|2247|9771|750|2087|1|4|7|2|2087|750|9771|Monday|2087Q2|N|N|N|2483412|2483501|2483053|2483328|N|N|N|N|N| +2483419|AAAAAAAALNEOFCAA|2087-04-08|2247|9772|750|2087|2|4|8|2|2087|750|9772|Tuesday|2087Q2|N|N|N|2483412|2483501|2483054|2483329|N|N|N|N|N| +2483420|AAAAAAAAMNEOFCAA|2087-04-09|2247|9772|750|2087|3|4|9|2|2087|750|9772|Wednesday|2087Q2|N|N|N|2483412|2483501|2483055|2483330|N|N|N|N|N| +2483421|AAAAAAAANNEOFCAA|2087-04-10|2247|9772|750|2087|4|4|10|2|2087|750|9772|Thursday|2087Q2|N|N|N|2483412|2483501|2483056|2483331|N|N|N|N|N| +2483422|AAAAAAAAONEOFCAA|2087-04-11|2247|9772|750|2087|5|4|11|2|2087|750|9772|Friday|2087Q2|N|Y|N|2483412|2483501|2483057|2483332|N|N|N|N|N| +2483423|AAAAAAAAPNEOFCAA|2087-04-12|2247|9772|750|2087|6|4|12|2|2087|750|9772|Saturday|2087Q2|N|Y|N|2483412|2483501|2483058|2483333|N|N|N|N|N| +2483424|AAAAAAAAAOEOFCAA|2087-04-13|2247|9772|750|2087|0|4|13|2|2087|750|9772|Sunday|2087Q2|N|N|N|2483412|2483501|2483059|2483334|N|N|N|N|N| +2483425|AAAAAAAABOEOFCAA|2087-04-14|2247|9772|750|2087|1|4|14|2|2087|750|9772|Monday|2087Q2|N|N|N|2483412|2483501|2483060|2483335|N|N|N|N|N| +2483426|AAAAAAAACOEOFCAA|2087-04-15|2247|9773|750|2087|2|4|15|2|2087|750|9773|Tuesday|2087Q2|N|N|N|2483412|2483501|2483061|2483336|N|N|N|N|N| +2483427|AAAAAAAADOEOFCAA|2087-04-16|2247|9773|750|2087|3|4|16|2|2087|750|9773|Wednesday|2087Q2|N|N|N|2483412|2483501|2483062|2483337|N|N|N|N|N| +2483428|AAAAAAAAEOEOFCAA|2087-04-17|2247|9773|750|2087|4|4|17|2|2087|750|9773|Thursday|2087Q2|N|N|N|2483412|2483501|2483063|2483338|N|N|N|N|N| +2483429|AAAAAAAAFOEOFCAA|2087-04-18|2247|9773|750|2087|5|4|18|2|2087|750|9773|Friday|2087Q2|N|Y|N|2483412|2483501|2483064|2483339|N|N|N|N|N| +2483430|AAAAAAAAGOEOFCAA|2087-04-19|2247|9773|750|2087|6|4|19|2|2087|750|9773|Saturday|2087Q2|N|Y|N|2483412|2483501|2483065|2483340|N|N|N|N|N| +2483431|AAAAAAAAHOEOFCAA|2087-04-20|2247|9773|750|2087|0|4|20|2|2087|750|9773|Sunday|2087Q2|N|N|N|2483412|2483501|2483066|2483341|N|N|N|N|N| +2483432|AAAAAAAAIOEOFCAA|2087-04-21|2247|9773|750|2087|1|4|21|2|2087|750|9773|Monday|2087Q2|N|N|N|2483412|2483501|2483067|2483342|N|N|N|N|N| +2483433|AAAAAAAAJOEOFCAA|2087-04-22|2247|9774|750|2087|2|4|22|2|2087|750|9774|Tuesday|2087Q2|N|N|N|2483412|2483501|2483068|2483343|N|N|N|N|N| +2483434|AAAAAAAAKOEOFCAA|2087-04-23|2247|9774|750|2087|3|4|23|2|2087|750|9774|Wednesday|2087Q2|N|N|N|2483412|2483501|2483069|2483344|N|N|N|N|N| +2483435|AAAAAAAALOEOFCAA|2087-04-24|2247|9774|750|2087|4|4|24|2|2087|750|9774|Thursday|2087Q2|N|N|N|2483412|2483501|2483070|2483345|N|N|N|N|N| +2483436|AAAAAAAAMOEOFCAA|2087-04-25|2247|9774|750|2087|5|4|25|2|2087|750|9774|Friday|2087Q2|N|Y|N|2483412|2483501|2483071|2483346|N|N|N|N|N| +2483437|AAAAAAAANOEOFCAA|2087-04-26|2247|9774|750|2087|6|4|26|2|2087|750|9774|Saturday|2087Q2|N|Y|N|2483412|2483501|2483072|2483347|N|N|N|N|N| +2483438|AAAAAAAAOOEOFCAA|2087-04-27|2247|9774|750|2087|0|4|27|2|2087|750|9774|Sunday|2087Q2|N|N|N|2483412|2483501|2483073|2483348|N|N|N|N|N| +2483439|AAAAAAAAPOEOFCAA|2087-04-28|2247|9774|750|2087|1|4|28|2|2087|750|9774|Monday|2087Q2|N|N|N|2483412|2483501|2483074|2483349|N|N|N|N|N| +2483440|AAAAAAAAAPEOFCAA|2087-04-29|2247|9775|750|2087|2|4|29|2|2087|750|9775|Tuesday|2087Q2|N|N|N|2483412|2483501|2483075|2483350|N|N|N|N|N| +2483441|AAAAAAAABPEOFCAA|2087-04-30|2247|9775|750|2087|3|4|30|2|2087|750|9775|Wednesday|2087Q2|N|N|N|2483412|2483501|2483076|2483351|N|N|N|N|N| +2483442|AAAAAAAACPEOFCAA|2087-05-01|2248|9775|750|2087|4|5|1|2|2087|750|9775|Thursday|2087Q2|N|N|N|2483442|2483561|2483077|2483352|N|N|N|N|N| +2483443|AAAAAAAADPEOFCAA|2087-05-02|2248|9775|750|2087|5|5|2|2|2087|750|9775|Friday|2087Q2|N|Y|N|2483442|2483561|2483078|2483353|N|N|N|N|N| +2483444|AAAAAAAAEPEOFCAA|2087-05-03|2248|9775|750|2087|6|5|3|2|2087|750|9775|Saturday|2087Q2|N|Y|N|2483442|2483561|2483079|2483354|N|N|N|N|N| +2483445|AAAAAAAAFPEOFCAA|2087-05-04|2248|9775|750|2087|0|5|4|2|2087|750|9775|Sunday|2087Q2|N|N|N|2483442|2483561|2483080|2483355|N|N|N|N|N| +2483446|AAAAAAAAGPEOFCAA|2087-05-05|2248|9775|750|2087|1|5|5|2|2087|750|9775|Monday|2087Q2|N|N|N|2483442|2483561|2483081|2483356|N|N|N|N|N| +2483447|AAAAAAAAHPEOFCAA|2087-05-06|2248|9776|750|2087|2|5|6|2|2087|750|9776|Tuesday|2087Q2|N|N|N|2483442|2483561|2483082|2483357|N|N|N|N|N| +2483448|AAAAAAAAIPEOFCAA|2087-05-07|2248|9776|750|2087|3|5|7|2|2087|750|9776|Wednesday|2087Q2|N|N|N|2483442|2483561|2483083|2483358|N|N|N|N|N| +2483449|AAAAAAAAJPEOFCAA|2087-05-08|2248|9776|750|2087|4|5|8|2|2087|750|9776|Thursday|2087Q2|N|N|N|2483442|2483561|2483084|2483359|N|N|N|N|N| +2483450|AAAAAAAAKPEOFCAA|2087-05-09|2248|9776|750|2087|5|5|9|2|2087|750|9776|Friday|2087Q2|N|Y|N|2483442|2483561|2483085|2483360|N|N|N|N|N| +2483451|AAAAAAAALPEOFCAA|2087-05-10|2248|9776|750|2087|6|5|10|2|2087|750|9776|Saturday|2087Q2|N|Y|N|2483442|2483561|2483086|2483361|N|N|N|N|N| +2483452|AAAAAAAAMPEOFCAA|2087-05-11|2248|9776|750|2087|0|5|11|2|2087|750|9776|Sunday|2087Q2|N|N|N|2483442|2483561|2483087|2483362|N|N|N|N|N| +2483453|AAAAAAAANPEOFCAA|2087-05-12|2248|9776|750|2087|1|5|12|2|2087|750|9776|Monday|2087Q2|N|N|N|2483442|2483561|2483088|2483363|N|N|N|N|N| +2483454|AAAAAAAAOPEOFCAA|2087-05-13|2248|9777|750|2087|2|5|13|2|2087|750|9777|Tuesday|2087Q2|N|N|N|2483442|2483561|2483089|2483364|N|N|N|N|N| +2483455|AAAAAAAAPPEOFCAA|2087-05-14|2248|9777|750|2087|3|5|14|2|2087|750|9777|Wednesday|2087Q2|N|N|N|2483442|2483561|2483090|2483365|N|N|N|N|N| +2483456|AAAAAAAAAAFOFCAA|2087-05-15|2248|9777|750|2087|4|5|15|2|2087|750|9777|Thursday|2087Q2|N|N|N|2483442|2483561|2483091|2483366|N|N|N|N|N| +2483457|AAAAAAAABAFOFCAA|2087-05-16|2248|9777|750|2087|5|5|16|2|2087|750|9777|Friday|2087Q2|N|Y|N|2483442|2483561|2483092|2483367|N|N|N|N|N| +2483458|AAAAAAAACAFOFCAA|2087-05-17|2248|9777|750|2087|6|5|17|2|2087|750|9777|Saturday|2087Q2|N|Y|N|2483442|2483561|2483093|2483368|N|N|N|N|N| +2483459|AAAAAAAADAFOFCAA|2087-05-18|2248|9777|750|2087|0|5|18|2|2087|750|9777|Sunday|2087Q2|N|N|N|2483442|2483561|2483094|2483369|N|N|N|N|N| +2483460|AAAAAAAAEAFOFCAA|2087-05-19|2248|9777|750|2087|1|5|19|2|2087|750|9777|Monday|2087Q2|N|N|N|2483442|2483561|2483095|2483370|N|N|N|N|N| +2483461|AAAAAAAAFAFOFCAA|2087-05-20|2248|9778|750|2087|2|5|20|2|2087|750|9778|Tuesday|2087Q2|N|N|N|2483442|2483561|2483096|2483371|N|N|N|N|N| +2483462|AAAAAAAAGAFOFCAA|2087-05-21|2248|9778|750|2087|3|5|21|2|2087|750|9778|Wednesday|2087Q2|N|N|N|2483442|2483561|2483097|2483372|N|N|N|N|N| +2483463|AAAAAAAAHAFOFCAA|2087-05-22|2248|9778|750|2087|4|5|22|2|2087|750|9778|Thursday|2087Q2|N|N|N|2483442|2483561|2483098|2483373|N|N|N|N|N| +2483464|AAAAAAAAIAFOFCAA|2087-05-23|2248|9778|750|2087|5|5|23|2|2087|750|9778|Friday|2087Q2|N|Y|N|2483442|2483561|2483099|2483374|N|N|N|N|N| +2483465|AAAAAAAAJAFOFCAA|2087-05-24|2248|9778|750|2087|6|5|24|2|2087|750|9778|Saturday|2087Q2|N|Y|N|2483442|2483561|2483100|2483375|N|N|N|N|N| +2483466|AAAAAAAAKAFOFCAA|2087-05-25|2248|9778|750|2087|0|5|25|2|2087|750|9778|Sunday|2087Q2|N|N|N|2483442|2483561|2483101|2483376|N|N|N|N|N| +2483467|AAAAAAAALAFOFCAA|2087-05-26|2248|9778|750|2087|1|5|26|2|2087|750|9778|Monday|2087Q2|N|N|N|2483442|2483561|2483102|2483377|N|N|N|N|N| +2483468|AAAAAAAAMAFOFCAA|2087-05-27|2248|9779|750|2087|2|5|27|2|2087|750|9779|Tuesday|2087Q2|N|N|N|2483442|2483561|2483103|2483378|N|N|N|N|N| +2483469|AAAAAAAANAFOFCAA|2087-05-28|2248|9779|750|2087|3|5|28|2|2087|750|9779|Wednesday|2087Q2|N|N|N|2483442|2483561|2483104|2483379|N|N|N|N|N| +2483470|AAAAAAAAOAFOFCAA|2087-05-29|2248|9779|750|2087|4|5|29|2|2087|750|9779|Thursday|2087Q2|N|N|N|2483442|2483561|2483105|2483380|N|N|N|N|N| +2483471|AAAAAAAAPAFOFCAA|2087-05-30|2248|9779|750|2087|5|5|30|2|2087|750|9779|Friday|2087Q2|N|Y|N|2483442|2483561|2483106|2483381|N|N|N|N|N| +2483472|AAAAAAAAABFOFCAA|2087-05-31|2248|9779|750|2087|6|5|31|2|2087|750|9779|Saturday|2087Q2|N|Y|N|2483442|2483561|2483107|2483382|N|N|N|N|N| +2483473|AAAAAAAABBFOFCAA|2087-06-01|2249|9779|751|2087|0|6|1|2|2087|751|9779|Sunday|2087Q2|N|N|N|2483473|2483623|2483108|2483383|N|N|N|N|N| +2483474|AAAAAAAACBFOFCAA|2087-06-02|2249|9779|751|2087|1|6|2|2|2087|751|9779|Monday|2087Q2|N|N|N|2483473|2483623|2483109|2483384|N|N|N|N|N| +2483475|AAAAAAAADBFOFCAA|2087-06-03|2249|9780|751|2087|2|6|3|2|2087|751|9780|Tuesday|2087Q2|N|N|N|2483473|2483623|2483110|2483385|N|N|N|N|N| +2483476|AAAAAAAAEBFOFCAA|2087-06-04|2249|9780|751|2087|3|6|4|2|2087|751|9780|Wednesday|2087Q2|N|N|N|2483473|2483623|2483111|2483386|N|N|N|N|N| +2483477|AAAAAAAAFBFOFCAA|2087-06-05|2249|9780|751|2087|4|6|5|2|2087|751|9780|Thursday|2087Q2|N|N|N|2483473|2483623|2483112|2483387|N|N|N|N|N| +2483478|AAAAAAAAGBFOFCAA|2087-06-06|2249|9780|751|2087|5|6|6|2|2087|751|9780|Friday|2087Q2|N|Y|N|2483473|2483623|2483113|2483388|N|N|N|N|N| +2483479|AAAAAAAAHBFOFCAA|2087-06-07|2249|9780|751|2087|6|6|7|2|2087|751|9780|Saturday|2087Q2|N|Y|N|2483473|2483623|2483114|2483389|N|N|N|N|N| +2483480|AAAAAAAAIBFOFCAA|2087-06-08|2249|9780|751|2087|0|6|8|2|2087|751|9780|Sunday|2087Q2|N|N|N|2483473|2483623|2483115|2483390|N|N|N|N|N| +2483481|AAAAAAAAJBFOFCAA|2087-06-09|2249|9780|751|2087|1|6|9|2|2087|751|9780|Monday|2087Q2|N|N|N|2483473|2483623|2483116|2483391|N|N|N|N|N| +2483482|AAAAAAAAKBFOFCAA|2087-06-10|2249|9781|751|2087|2|6|10|2|2087|751|9781|Tuesday|2087Q2|N|N|N|2483473|2483623|2483117|2483392|N|N|N|N|N| +2483483|AAAAAAAALBFOFCAA|2087-06-11|2249|9781|751|2087|3|6|11|2|2087|751|9781|Wednesday|2087Q2|N|N|N|2483473|2483623|2483118|2483393|N|N|N|N|N| +2483484|AAAAAAAAMBFOFCAA|2087-06-12|2249|9781|751|2087|4|6|12|2|2087|751|9781|Thursday|2087Q2|N|N|N|2483473|2483623|2483119|2483394|N|N|N|N|N| +2483485|AAAAAAAANBFOFCAA|2087-06-13|2249|9781|751|2087|5|6|13|2|2087|751|9781|Friday|2087Q2|N|Y|N|2483473|2483623|2483120|2483395|N|N|N|N|N| +2483486|AAAAAAAAOBFOFCAA|2087-06-14|2249|9781|751|2087|6|6|14|2|2087|751|9781|Saturday|2087Q2|N|Y|N|2483473|2483623|2483121|2483396|N|N|N|N|N| +2483487|AAAAAAAAPBFOFCAA|2087-06-15|2249|9781|751|2087|0|6|15|2|2087|751|9781|Sunday|2087Q2|N|N|N|2483473|2483623|2483122|2483397|N|N|N|N|N| +2483488|AAAAAAAAACFOFCAA|2087-06-16|2249|9781|751|2087|1|6|16|2|2087|751|9781|Monday|2087Q2|N|N|N|2483473|2483623|2483123|2483398|N|N|N|N|N| +2483489|AAAAAAAABCFOFCAA|2087-06-17|2249|9782|751|2087|2|6|17|2|2087|751|9782|Tuesday|2087Q2|N|N|N|2483473|2483623|2483124|2483399|N|N|N|N|N| +2483490|AAAAAAAACCFOFCAA|2087-06-18|2249|9782|751|2087|3|6|18|2|2087|751|9782|Wednesday|2087Q2|N|N|N|2483473|2483623|2483125|2483400|N|N|N|N|N| +2483491|AAAAAAAADCFOFCAA|2087-06-19|2249|9782|751|2087|4|6|19|2|2087|751|9782|Thursday|2087Q2|N|N|N|2483473|2483623|2483126|2483401|N|N|N|N|N| +2483492|AAAAAAAAECFOFCAA|2087-06-20|2249|9782|751|2087|5|6|20|2|2087|751|9782|Friday|2087Q2|N|Y|N|2483473|2483623|2483127|2483402|N|N|N|N|N| +2483493|AAAAAAAAFCFOFCAA|2087-06-21|2249|9782|751|2087|6|6|21|2|2087|751|9782|Saturday|2087Q2|N|Y|N|2483473|2483623|2483128|2483403|N|N|N|N|N| +2483494|AAAAAAAAGCFOFCAA|2087-06-22|2249|9782|751|2087|0|6|22|2|2087|751|9782|Sunday|2087Q2|N|N|N|2483473|2483623|2483129|2483404|N|N|N|N|N| +2483495|AAAAAAAAHCFOFCAA|2087-06-23|2249|9782|751|2087|1|6|23|2|2087|751|9782|Monday|2087Q2|N|N|N|2483473|2483623|2483130|2483405|N|N|N|N|N| +2483496|AAAAAAAAICFOFCAA|2087-06-24|2249|9783|751|2087|2|6|24|2|2087|751|9783|Tuesday|2087Q2|N|N|N|2483473|2483623|2483131|2483406|N|N|N|N|N| +2483497|AAAAAAAAJCFOFCAA|2087-06-25|2249|9783|751|2087|3|6|25|2|2087|751|9783|Wednesday|2087Q2|N|N|N|2483473|2483623|2483132|2483407|N|N|N|N|N| +2483498|AAAAAAAAKCFOFCAA|2087-06-26|2249|9783|751|2087|4|6|26|2|2087|751|9783|Thursday|2087Q2|N|N|N|2483473|2483623|2483133|2483408|N|N|N|N|N| +2483499|AAAAAAAALCFOFCAA|2087-06-27|2249|9783|751|2087|5|6|27|2|2087|751|9783|Friday|2087Q2|N|Y|N|2483473|2483623|2483134|2483409|N|N|N|N|N| +2483500|AAAAAAAAMCFOFCAA|2087-06-28|2249|9783|751|2087|6|6|28|2|2087|751|9783|Saturday|2087Q2|N|Y|N|2483473|2483623|2483135|2483410|N|N|N|N|N| +2483501|AAAAAAAANCFOFCAA|2087-06-29|2249|9783|751|2087|0|6|29|2|2087|751|9783|Sunday|2087Q2|N|N|N|2483473|2483623|2483136|2483411|N|N|N|N|N| +2483502|AAAAAAAAOCFOFCAA|2087-06-30|2249|9783|751|2087|1|6|30|2|2087|751|9783|Monday|2087Q2|N|N|N|2483473|2483623|2483137|2483412|N|N|N|N|N| +2483503|AAAAAAAAPCFOFCAA|2087-07-01|2250|9784|751|2087|2|7|1|2|2087|751|9784|Tuesday|2087Q2|N|N|N|2483503|2483683|2483138|2483412|N|N|N|N|N| +2483504|AAAAAAAAADFOFCAA|2087-07-02|2250|9784|751|2087|3|7|2|3|2087|751|9784|Wednesday|2087Q3|N|N|N|2483503|2483683|2483139|2483413|N|N|N|N|N| +2483505|AAAAAAAABDFOFCAA|2087-07-03|2250|9784|751|2087|4|7|3|3|2087|751|9784|Thursday|2087Q3|N|N|N|2483503|2483683|2483140|2483414|N|N|N|N|N| +2483506|AAAAAAAACDFOFCAA|2087-07-04|2250|9784|751|2087|5|7|4|3|2087|751|9784|Friday|2087Q3|N|Y|N|2483503|2483683|2483141|2483415|N|N|N|N|N| +2483507|AAAAAAAADDFOFCAA|2087-07-05|2250|9784|751|2087|6|7|5|3|2087|751|9784|Saturday|2087Q3|Y|Y|N|2483503|2483683|2483142|2483416|N|N|N|N|N| +2483508|AAAAAAAAEDFOFCAA|2087-07-06|2250|9784|751|2087|0|7|6|3|2087|751|9784|Sunday|2087Q3|N|N|Y|2483503|2483683|2483143|2483417|N|N|N|N|N| +2483509|AAAAAAAAFDFOFCAA|2087-07-07|2250|9784|751|2087|1|7|7|3|2087|751|9784|Monday|2087Q3|N|N|N|2483503|2483683|2483144|2483418|N|N|N|N|N| +2483510|AAAAAAAAGDFOFCAA|2087-07-08|2250|9785|751|2087|2|7|8|3|2087|751|9785|Tuesday|2087Q3|N|N|N|2483503|2483683|2483145|2483419|N|N|N|N|N| +2483511|AAAAAAAAHDFOFCAA|2087-07-09|2250|9785|751|2087|3|7|9|3|2087|751|9785|Wednesday|2087Q3|N|N|N|2483503|2483683|2483146|2483420|N|N|N|N|N| +2483512|AAAAAAAAIDFOFCAA|2087-07-10|2250|9785|751|2087|4|7|10|3|2087|751|9785|Thursday|2087Q3|N|N|N|2483503|2483683|2483147|2483421|N|N|N|N|N| +2483513|AAAAAAAAJDFOFCAA|2087-07-11|2250|9785|751|2087|5|7|11|3|2087|751|9785|Friday|2087Q3|N|Y|N|2483503|2483683|2483148|2483422|N|N|N|N|N| +2483514|AAAAAAAAKDFOFCAA|2087-07-12|2250|9785|751|2087|6|7|12|3|2087|751|9785|Saturday|2087Q3|N|Y|N|2483503|2483683|2483149|2483423|N|N|N|N|N| +2483515|AAAAAAAALDFOFCAA|2087-07-13|2250|9785|751|2087|0|7|13|3|2087|751|9785|Sunday|2087Q3|N|N|N|2483503|2483683|2483150|2483424|N|N|N|N|N| +2483516|AAAAAAAAMDFOFCAA|2087-07-14|2250|9785|751|2087|1|7|14|3|2087|751|9785|Monday|2087Q3|N|N|N|2483503|2483683|2483151|2483425|N|N|N|N|N| +2483517|AAAAAAAANDFOFCAA|2087-07-15|2250|9786|751|2087|2|7|15|3|2087|751|9786|Tuesday|2087Q3|N|N|N|2483503|2483683|2483152|2483426|N|N|N|N|N| +2483518|AAAAAAAAODFOFCAA|2087-07-16|2250|9786|751|2087|3|7|16|3|2087|751|9786|Wednesday|2087Q3|N|N|N|2483503|2483683|2483153|2483427|N|N|N|N|N| +2483519|AAAAAAAAPDFOFCAA|2087-07-17|2250|9786|751|2087|4|7|17|3|2087|751|9786|Thursday|2087Q3|N|N|N|2483503|2483683|2483154|2483428|N|N|N|N|N| +2483520|AAAAAAAAAEFOFCAA|2087-07-18|2250|9786|751|2087|5|7|18|3|2087|751|9786|Friday|2087Q3|N|Y|N|2483503|2483683|2483155|2483429|N|N|N|N|N| +2483521|AAAAAAAABEFOFCAA|2087-07-19|2250|9786|751|2087|6|7|19|3|2087|751|9786|Saturday|2087Q3|N|Y|N|2483503|2483683|2483156|2483430|N|N|N|N|N| +2483522|AAAAAAAACEFOFCAA|2087-07-20|2250|9786|751|2087|0|7|20|3|2087|751|9786|Sunday|2087Q3|N|N|N|2483503|2483683|2483157|2483431|N|N|N|N|N| +2483523|AAAAAAAADEFOFCAA|2087-07-21|2250|9786|751|2087|1|7|21|3|2087|751|9786|Monday|2087Q3|N|N|N|2483503|2483683|2483158|2483432|N|N|N|N|N| +2483524|AAAAAAAAEEFOFCAA|2087-07-22|2250|9787|751|2087|2|7|22|3|2087|751|9787|Tuesday|2087Q3|N|N|N|2483503|2483683|2483159|2483433|N|N|N|N|N| +2483525|AAAAAAAAFEFOFCAA|2087-07-23|2250|9787|751|2087|3|7|23|3|2087|751|9787|Wednesday|2087Q3|N|N|N|2483503|2483683|2483160|2483434|N|N|N|N|N| +2483526|AAAAAAAAGEFOFCAA|2087-07-24|2250|9787|751|2087|4|7|24|3|2087|751|9787|Thursday|2087Q3|N|N|N|2483503|2483683|2483161|2483435|N|N|N|N|N| +2483527|AAAAAAAAHEFOFCAA|2087-07-25|2250|9787|751|2087|5|7|25|3|2087|751|9787|Friday|2087Q3|N|Y|N|2483503|2483683|2483162|2483436|N|N|N|N|N| +2483528|AAAAAAAAIEFOFCAA|2087-07-26|2250|9787|751|2087|6|7|26|3|2087|751|9787|Saturday|2087Q3|N|Y|N|2483503|2483683|2483163|2483437|N|N|N|N|N| +2483529|AAAAAAAAJEFOFCAA|2087-07-27|2250|9787|751|2087|0|7|27|3|2087|751|9787|Sunday|2087Q3|N|N|N|2483503|2483683|2483164|2483438|N|N|N|N|N| +2483530|AAAAAAAAKEFOFCAA|2087-07-28|2250|9787|751|2087|1|7|28|3|2087|751|9787|Monday|2087Q3|N|N|N|2483503|2483683|2483165|2483439|N|N|N|N|N| +2483531|AAAAAAAALEFOFCAA|2087-07-29|2250|9788|751|2087|2|7|29|3|2087|751|9788|Tuesday|2087Q3|N|N|N|2483503|2483683|2483166|2483440|N|N|N|N|N| +2483532|AAAAAAAAMEFOFCAA|2087-07-30|2250|9788|751|2087|3|7|30|3|2087|751|9788|Wednesday|2087Q3|N|N|N|2483503|2483683|2483167|2483441|N|N|N|N|N| +2483533|AAAAAAAANEFOFCAA|2087-07-31|2250|9788|751|2087|4|7|31|3|2087|751|9788|Thursday|2087Q3|N|N|N|2483503|2483683|2483168|2483442|N|N|N|N|N| +2483534|AAAAAAAAOEFOFCAA|2087-08-01|2251|9788|751|2087|5|8|1|3|2087|751|9788|Friday|2087Q3|N|Y|N|2483534|2483745|2483169|2483443|N|N|N|N|N| +2483535|AAAAAAAAPEFOFCAA|2087-08-02|2251|9788|751|2087|6|8|2|3|2087|751|9788|Saturday|2087Q3|N|Y|N|2483534|2483745|2483170|2483444|N|N|N|N|N| +2483536|AAAAAAAAAFFOFCAA|2087-08-03|2251|9788|751|2087|0|8|3|3|2087|751|9788|Sunday|2087Q3|N|N|N|2483534|2483745|2483171|2483445|N|N|N|N|N| +2483537|AAAAAAAABFFOFCAA|2087-08-04|2251|9788|751|2087|1|8|4|3|2087|751|9788|Monday|2087Q3|N|N|N|2483534|2483745|2483172|2483446|N|N|N|N|N| +2483538|AAAAAAAACFFOFCAA|2087-08-05|2251|9789|751|2087|2|8|5|3|2087|751|9789|Tuesday|2087Q3|N|N|N|2483534|2483745|2483173|2483447|N|N|N|N|N| +2483539|AAAAAAAADFFOFCAA|2087-08-06|2251|9789|751|2087|3|8|6|3|2087|751|9789|Wednesday|2087Q3|N|N|N|2483534|2483745|2483174|2483448|N|N|N|N|N| +2483540|AAAAAAAAEFFOFCAA|2087-08-07|2251|9789|751|2087|4|8|7|3|2087|751|9789|Thursday|2087Q3|N|N|N|2483534|2483745|2483175|2483449|N|N|N|N|N| +2483541|AAAAAAAAFFFOFCAA|2087-08-08|2251|9789|751|2087|5|8|8|3|2087|751|9789|Friday|2087Q3|N|Y|N|2483534|2483745|2483176|2483450|N|N|N|N|N| +2483542|AAAAAAAAGFFOFCAA|2087-08-09|2251|9789|751|2087|6|8|9|3|2087|751|9789|Saturday|2087Q3|N|Y|N|2483534|2483745|2483177|2483451|N|N|N|N|N| +2483543|AAAAAAAAHFFOFCAA|2087-08-10|2251|9789|751|2087|0|8|10|3|2087|751|9789|Sunday|2087Q3|N|N|N|2483534|2483745|2483178|2483452|N|N|N|N|N| +2483544|AAAAAAAAIFFOFCAA|2087-08-11|2251|9789|751|2087|1|8|11|3|2087|751|9789|Monday|2087Q3|N|N|N|2483534|2483745|2483179|2483453|N|N|N|N|N| +2483545|AAAAAAAAJFFOFCAA|2087-08-12|2251|9790|751|2087|2|8|12|3|2087|751|9790|Tuesday|2087Q3|N|N|N|2483534|2483745|2483180|2483454|N|N|N|N|N| +2483546|AAAAAAAAKFFOFCAA|2087-08-13|2251|9790|751|2087|3|8|13|3|2087|751|9790|Wednesday|2087Q3|N|N|N|2483534|2483745|2483181|2483455|N|N|N|N|N| +2483547|AAAAAAAALFFOFCAA|2087-08-14|2251|9790|751|2087|4|8|14|3|2087|751|9790|Thursday|2087Q3|N|N|N|2483534|2483745|2483182|2483456|N|N|N|N|N| +2483548|AAAAAAAAMFFOFCAA|2087-08-15|2251|9790|751|2087|5|8|15|3|2087|751|9790|Friday|2087Q3|N|Y|N|2483534|2483745|2483183|2483457|N|N|N|N|N| +2483549|AAAAAAAANFFOFCAA|2087-08-16|2251|9790|751|2087|6|8|16|3|2087|751|9790|Saturday|2087Q3|N|Y|N|2483534|2483745|2483184|2483458|N|N|N|N|N| +2483550|AAAAAAAAOFFOFCAA|2087-08-17|2251|9790|751|2087|0|8|17|3|2087|751|9790|Sunday|2087Q3|N|N|N|2483534|2483745|2483185|2483459|N|N|N|N|N| +2483551|AAAAAAAAPFFOFCAA|2087-08-18|2251|9790|751|2087|1|8|18|3|2087|751|9790|Monday|2087Q3|N|N|N|2483534|2483745|2483186|2483460|N|N|N|N|N| +2483552|AAAAAAAAAGFOFCAA|2087-08-19|2251|9791|751|2087|2|8|19|3|2087|751|9791|Tuesday|2087Q3|N|N|N|2483534|2483745|2483187|2483461|N|N|N|N|N| +2483553|AAAAAAAABGFOFCAA|2087-08-20|2251|9791|751|2087|3|8|20|3|2087|751|9791|Wednesday|2087Q3|N|N|N|2483534|2483745|2483188|2483462|N|N|N|N|N| +2483554|AAAAAAAACGFOFCAA|2087-08-21|2251|9791|751|2087|4|8|21|3|2087|751|9791|Thursday|2087Q3|N|N|N|2483534|2483745|2483189|2483463|N|N|N|N|N| +2483555|AAAAAAAADGFOFCAA|2087-08-22|2251|9791|751|2087|5|8|22|3|2087|751|9791|Friday|2087Q3|N|Y|N|2483534|2483745|2483190|2483464|N|N|N|N|N| +2483556|AAAAAAAAEGFOFCAA|2087-08-23|2251|9791|751|2087|6|8|23|3|2087|751|9791|Saturday|2087Q3|N|Y|N|2483534|2483745|2483191|2483465|N|N|N|N|N| +2483557|AAAAAAAAFGFOFCAA|2087-08-24|2251|9791|751|2087|0|8|24|3|2087|751|9791|Sunday|2087Q3|N|N|N|2483534|2483745|2483192|2483466|N|N|N|N|N| +2483558|AAAAAAAAGGFOFCAA|2087-08-25|2251|9791|751|2087|1|8|25|3|2087|751|9791|Monday|2087Q3|N|N|N|2483534|2483745|2483193|2483467|N|N|N|N|N| +2483559|AAAAAAAAHGFOFCAA|2087-08-26|2251|9792|751|2087|2|8|26|3|2087|751|9792|Tuesday|2087Q3|N|N|N|2483534|2483745|2483194|2483468|N|N|N|N|N| +2483560|AAAAAAAAIGFOFCAA|2087-08-27|2251|9792|751|2087|3|8|27|3|2087|751|9792|Wednesday|2087Q3|N|N|N|2483534|2483745|2483195|2483469|N|N|N|N|N| +2483561|AAAAAAAAJGFOFCAA|2087-08-28|2251|9792|751|2087|4|8|28|3|2087|751|9792|Thursday|2087Q3|N|N|N|2483534|2483745|2483196|2483470|N|N|N|N|N| +2483562|AAAAAAAAKGFOFCAA|2087-08-29|2251|9792|751|2087|5|8|29|3|2087|751|9792|Friday|2087Q3|N|Y|N|2483534|2483745|2483197|2483471|N|N|N|N|N| +2483563|AAAAAAAALGFOFCAA|2087-08-30|2251|9792|751|2087|6|8|30|3|2087|751|9792|Saturday|2087Q3|N|Y|N|2483534|2483745|2483198|2483472|N|N|N|N|N| +2483564|AAAAAAAAMGFOFCAA|2087-08-31|2251|9792|751|2087|0|8|31|3|2087|751|9792|Sunday|2087Q3|N|N|N|2483534|2483745|2483199|2483473|N|N|N|N|N| +2483565|AAAAAAAANGFOFCAA|2087-09-01|2252|9792|752|2087|1|9|1|3|2087|752|9792|Monday|2087Q3|N|N|N|2483565|2483807|2483200|2483474|N|N|N|N|N| +2483566|AAAAAAAAOGFOFCAA|2087-09-02|2252|9793|752|2087|2|9|2|3|2087|752|9793|Tuesday|2087Q3|N|N|N|2483565|2483807|2483201|2483475|N|N|N|N|N| +2483567|AAAAAAAAPGFOFCAA|2087-09-03|2252|9793|752|2087|3|9|3|3|2087|752|9793|Wednesday|2087Q3|N|N|N|2483565|2483807|2483202|2483476|N|N|N|N|N| +2483568|AAAAAAAAAHFOFCAA|2087-09-04|2252|9793|752|2087|4|9|4|3|2087|752|9793|Thursday|2087Q3|N|N|N|2483565|2483807|2483203|2483477|N|N|N|N|N| +2483569|AAAAAAAABHFOFCAA|2087-09-05|2252|9793|752|2087|5|9|5|3|2087|752|9793|Friday|2087Q3|N|Y|N|2483565|2483807|2483204|2483478|N|N|N|N|N| +2483570|AAAAAAAACHFOFCAA|2087-09-06|2252|9793|752|2087|6|9|6|3|2087|752|9793|Saturday|2087Q3|N|Y|N|2483565|2483807|2483205|2483479|N|N|N|N|N| +2483571|AAAAAAAADHFOFCAA|2087-09-07|2252|9793|752|2087|0|9|7|3|2087|752|9793|Sunday|2087Q3|N|N|N|2483565|2483807|2483206|2483480|N|N|N|N|N| +2483572|AAAAAAAAEHFOFCAA|2087-09-08|2252|9793|752|2087|1|9|8|3|2087|752|9793|Monday|2087Q3|N|N|N|2483565|2483807|2483207|2483481|N|N|N|N|N| +2483573|AAAAAAAAFHFOFCAA|2087-09-09|2252|9794|752|2087|2|9|9|3|2087|752|9794|Tuesday|2087Q3|N|N|N|2483565|2483807|2483208|2483482|N|N|N|N|N| +2483574|AAAAAAAAGHFOFCAA|2087-09-10|2252|9794|752|2087|3|9|10|3|2087|752|9794|Wednesday|2087Q3|N|N|N|2483565|2483807|2483209|2483483|N|N|N|N|N| +2483575|AAAAAAAAHHFOFCAA|2087-09-11|2252|9794|752|2087|4|9|11|3|2087|752|9794|Thursday|2087Q3|N|N|N|2483565|2483807|2483210|2483484|N|N|N|N|N| +2483576|AAAAAAAAIHFOFCAA|2087-09-12|2252|9794|752|2087|5|9|12|3|2087|752|9794|Friday|2087Q3|N|Y|N|2483565|2483807|2483211|2483485|N|N|N|N|N| +2483577|AAAAAAAAJHFOFCAA|2087-09-13|2252|9794|752|2087|6|9|13|3|2087|752|9794|Saturday|2087Q3|N|Y|N|2483565|2483807|2483212|2483486|N|N|N|N|N| +2483578|AAAAAAAAKHFOFCAA|2087-09-14|2252|9794|752|2087|0|9|14|3|2087|752|9794|Sunday|2087Q3|N|N|N|2483565|2483807|2483213|2483487|N|N|N|N|N| +2483579|AAAAAAAALHFOFCAA|2087-09-15|2252|9794|752|2087|1|9|15|3|2087|752|9794|Monday|2087Q3|N|N|N|2483565|2483807|2483214|2483488|N|N|N|N|N| +2483580|AAAAAAAAMHFOFCAA|2087-09-16|2252|9795|752|2087|2|9|16|3|2087|752|9795|Tuesday|2087Q3|N|N|N|2483565|2483807|2483215|2483489|N|N|N|N|N| +2483581|AAAAAAAANHFOFCAA|2087-09-17|2252|9795|752|2087|3|9|17|3|2087|752|9795|Wednesday|2087Q3|N|N|N|2483565|2483807|2483216|2483490|N|N|N|N|N| +2483582|AAAAAAAAOHFOFCAA|2087-09-18|2252|9795|752|2087|4|9|18|3|2087|752|9795|Thursday|2087Q3|N|N|N|2483565|2483807|2483217|2483491|N|N|N|N|N| +2483583|AAAAAAAAPHFOFCAA|2087-09-19|2252|9795|752|2087|5|9|19|3|2087|752|9795|Friday|2087Q3|N|Y|N|2483565|2483807|2483218|2483492|N|N|N|N|N| +2483584|AAAAAAAAAIFOFCAA|2087-09-20|2252|9795|752|2087|6|9|20|3|2087|752|9795|Saturday|2087Q3|N|Y|N|2483565|2483807|2483219|2483493|N|N|N|N|N| +2483585|AAAAAAAABIFOFCAA|2087-09-21|2252|9795|752|2087|0|9|21|3|2087|752|9795|Sunday|2087Q3|N|N|N|2483565|2483807|2483220|2483494|N|N|N|N|N| +2483586|AAAAAAAACIFOFCAA|2087-09-22|2252|9795|752|2087|1|9|22|3|2087|752|9795|Monday|2087Q3|N|N|N|2483565|2483807|2483221|2483495|N|N|N|N|N| +2483587|AAAAAAAADIFOFCAA|2087-09-23|2252|9796|752|2087|2|9|23|3|2087|752|9796|Tuesday|2087Q3|N|N|N|2483565|2483807|2483222|2483496|N|N|N|N|N| +2483588|AAAAAAAAEIFOFCAA|2087-09-24|2252|9796|752|2087|3|9|24|3|2087|752|9796|Wednesday|2087Q3|N|N|N|2483565|2483807|2483223|2483497|N|N|N|N|N| +2483589|AAAAAAAAFIFOFCAA|2087-09-25|2252|9796|752|2087|4|9|25|3|2087|752|9796|Thursday|2087Q3|N|N|N|2483565|2483807|2483224|2483498|N|N|N|N|N| +2483590|AAAAAAAAGIFOFCAA|2087-09-26|2252|9796|752|2087|5|9|26|3|2087|752|9796|Friday|2087Q3|N|Y|N|2483565|2483807|2483225|2483499|N|N|N|N|N| +2483591|AAAAAAAAHIFOFCAA|2087-09-27|2252|9796|752|2087|6|9|27|3|2087|752|9796|Saturday|2087Q3|N|Y|N|2483565|2483807|2483226|2483500|N|N|N|N|N| +2483592|AAAAAAAAIIFOFCAA|2087-09-28|2252|9796|752|2087|0|9|28|3|2087|752|9796|Sunday|2087Q3|N|N|N|2483565|2483807|2483227|2483501|N|N|N|N|N| +2483593|AAAAAAAAJIFOFCAA|2087-09-29|2252|9796|752|2087|1|9|29|3|2087|752|9796|Monday|2087Q3|N|N|N|2483565|2483807|2483228|2483502|N|N|N|N|N| +2483594|AAAAAAAAKIFOFCAA|2087-09-30|2252|9797|752|2087|2|9|30|3|2087|752|9797|Tuesday|2087Q3|N|N|N|2483565|2483807|2483229|2483503|N|N|N|N|N| +2483595|AAAAAAAALIFOFCAA|2087-10-01|2253|9797|752|2087|3|10|1|3|2087|752|9797|Wednesday|2087Q3|N|N|N|2483595|2483867|2483230|2483503|N|N|N|N|N| +2483596|AAAAAAAAMIFOFCAA|2087-10-02|2253|9797|752|2087|4|10|2|4|2087|752|9797|Thursday|2087Q4|N|N|N|2483595|2483867|2483231|2483504|N|N|N|N|N| +2483597|AAAAAAAANIFOFCAA|2087-10-03|2253|9797|752|2087|5|10|3|4|2087|752|9797|Friday|2087Q4|N|Y|N|2483595|2483867|2483232|2483505|N|N|N|N|N| +2483598|AAAAAAAAOIFOFCAA|2087-10-04|2253|9797|752|2087|6|10|4|4|2087|752|9797|Saturday|2087Q4|N|Y|N|2483595|2483867|2483233|2483506|N|N|N|N|N| +2483599|AAAAAAAAPIFOFCAA|2087-10-05|2253|9797|752|2087|0|10|5|4|2087|752|9797|Sunday|2087Q4|N|N|N|2483595|2483867|2483234|2483507|N|N|N|N|N| +2483600|AAAAAAAAAJFOFCAA|2087-10-06|2253|9797|752|2087|1|10|6|4|2087|752|9797|Monday|2087Q4|N|N|N|2483595|2483867|2483235|2483508|N|N|N|N|N| +2483601|AAAAAAAABJFOFCAA|2087-10-07|2253|9798|752|2087|2|10|7|4|2087|752|9798|Tuesday|2087Q4|N|N|N|2483595|2483867|2483236|2483509|N|N|N|N|N| +2483602|AAAAAAAACJFOFCAA|2087-10-08|2253|9798|752|2087|3|10|8|4|2087|752|9798|Wednesday|2087Q4|N|N|N|2483595|2483867|2483237|2483510|N|N|N|N|N| +2483603|AAAAAAAADJFOFCAA|2087-10-09|2253|9798|752|2087|4|10|9|4|2087|752|9798|Thursday|2087Q4|N|N|N|2483595|2483867|2483238|2483511|N|N|N|N|N| +2483604|AAAAAAAAEJFOFCAA|2087-10-10|2253|9798|752|2087|5|10|10|4|2087|752|9798|Friday|2087Q4|N|Y|N|2483595|2483867|2483239|2483512|N|N|N|N|N| +2483605|AAAAAAAAFJFOFCAA|2087-10-11|2253|9798|752|2087|6|10|11|4|2087|752|9798|Saturday|2087Q4|N|Y|N|2483595|2483867|2483240|2483513|N|N|N|N|N| +2483606|AAAAAAAAGJFOFCAA|2087-10-12|2253|9798|752|2087|0|10|12|4|2087|752|9798|Sunday|2087Q4|N|N|N|2483595|2483867|2483241|2483514|N|N|N|N|N| +2483607|AAAAAAAAHJFOFCAA|2087-10-13|2253|9798|752|2087|1|10|13|4|2087|752|9798|Monday|2087Q4|N|N|N|2483595|2483867|2483242|2483515|N|N|N|N|N| +2483608|AAAAAAAAIJFOFCAA|2087-10-14|2253|9799|752|2087|2|10|14|4|2087|752|9799|Tuesday|2087Q4|N|N|N|2483595|2483867|2483243|2483516|N|N|N|N|N| +2483609|AAAAAAAAJJFOFCAA|2087-10-15|2253|9799|752|2087|3|10|15|4|2087|752|9799|Wednesday|2087Q4|N|N|N|2483595|2483867|2483244|2483517|N|N|N|N|N| +2483610|AAAAAAAAKJFOFCAA|2087-10-16|2253|9799|752|2087|4|10|16|4|2087|752|9799|Thursday|2087Q4|N|N|N|2483595|2483867|2483245|2483518|N|N|N|N|N| +2483611|AAAAAAAALJFOFCAA|2087-10-17|2253|9799|752|2087|5|10|17|4|2087|752|9799|Friday|2087Q4|N|Y|N|2483595|2483867|2483246|2483519|N|N|N|N|N| +2483612|AAAAAAAAMJFOFCAA|2087-10-18|2253|9799|752|2087|6|10|18|4|2087|752|9799|Saturday|2087Q4|N|Y|N|2483595|2483867|2483247|2483520|N|N|N|N|N| +2483613|AAAAAAAANJFOFCAA|2087-10-19|2253|9799|752|2087|0|10|19|4|2087|752|9799|Sunday|2087Q4|N|N|N|2483595|2483867|2483248|2483521|N|N|N|N|N| +2483614|AAAAAAAAOJFOFCAA|2087-10-20|2253|9799|752|2087|1|10|20|4|2087|752|9799|Monday|2087Q4|N|N|N|2483595|2483867|2483249|2483522|N|N|N|N|N| +2483615|AAAAAAAAPJFOFCAA|2087-10-21|2253|9800|752|2087|2|10|21|4|2087|752|9800|Tuesday|2087Q4|N|N|N|2483595|2483867|2483250|2483523|N|N|N|N|N| +2483616|AAAAAAAAAKFOFCAA|2087-10-22|2253|9800|752|2087|3|10|22|4|2087|752|9800|Wednesday|2087Q4|N|N|N|2483595|2483867|2483251|2483524|N|N|N|N|N| +2483617|AAAAAAAABKFOFCAA|2087-10-23|2253|9800|752|2087|4|10|23|4|2087|752|9800|Thursday|2087Q4|N|N|N|2483595|2483867|2483252|2483525|N|N|N|N|N| +2483618|AAAAAAAACKFOFCAA|2087-10-24|2253|9800|752|2087|5|10|24|4|2087|752|9800|Friday|2087Q4|N|Y|N|2483595|2483867|2483253|2483526|N|N|N|N|N| +2483619|AAAAAAAADKFOFCAA|2087-10-25|2253|9800|752|2087|6|10|25|4|2087|752|9800|Saturday|2087Q4|N|Y|N|2483595|2483867|2483254|2483527|N|N|N|N|N| +2483620|AAAAAAAAEKFOFCAA|2087-10-26|2253|9800|752|2087|0|10|26|4|2087|752|9800|Sunday|2087Q4|N|N|N|2483595|2483867|2483255|2483528|N|N|N|N|N| +2483621|AAAAAAAAFKFOFCAA|2087-10-27|2253|9800|752|2087|1|10|27|4|2087|752|9800|Monday|2087Q4|N|N|N|2483595|2483867|2483256|2483529|N|N|N|N|N| +2483622|AAAAAAAAGKFOFCAA|2087-10-28|2253|9801|752|2087|2|10|28|4|2087|752|9801|Tuesday|2087Q4|N|N|N|2483595|2483867|2483257|2483530|N|N|N|N|N| +2483623|AAAAAAAAHKFOFCAA|2087-10-29|2253|9801|752|2087|3|10|29|4|2087|752|9801|Wednesday|2087Q4|N|N|N|2483595|2483867|2483258|2483531|N|N|N|N|N| +2483624|AAAAAAAAIKFOFCAA|2087-10-30|2253|9801|752|2087|4|10|30|4|2087|752|9801|Thursday|2087Q4|N|N|N|2483595|2483867|2483259|2483532|N|N|N|N|N| +2483625|AAAAAAAAJKFOFCAA|2087-10-31|2253|9801|752|2087|5|10|31|4|2087|752|9801|Friday|2087Q4|N|Y|N|2483595|2483867|2483260|2483533|N|N|N|N|N| +2483626|AAAAAAAAKKFOFCAA|2087-11-01|2254|9801|752|2087|6|11|1|4|2087|752|9801|Saturday|2087Q4|N|Y|N|2483626|2483929|2483261|2483534|N|N|N|N|N| +2483627|AAAAAAAALKFOFCAA|2087-11-02|2254|9801|752|2087|0|11|2|4|2087|752|9801|Sunday|2087Q4|N|N|N|2483626|2483929|2483262|2483535|N|N|N|N|N| +2483628|AAAAAAAAMKFOFCAA|2087-11-03|2254|9801|752|2087|1|11|3|4|2087|752|9801|Monday|2087Q4|N|N|N|2483626|2483929|2483263|2483536|N|N|N|N|N| +2483629|AAAAAAAANKFOFCAA|2087-11-04|2254|9802|752|2087|2|11|4|4|2087|752|9802|Tuesday|2087Q4|N|N|N|2483626|2483929|2483264|2483537|N|N|N|N|N| +2483630|AAAAAAAAOKFOFCAA|2087-11-05|2254|9802|752|2087|3|11|5|4|2087|752|9802|Wednesday|2087Q4|N|N|N|2483626|2483929|2483265|2483538|N|N|N|N|N| +2483631|AAAAAAAAPKFOFCAA|2087-11-06|2254|9802|752|2087|4|11|6|4|2087|752|9802|Thursday|2087Q4|N|N|N|2483626|2483929|2483266|2483539|N|N|N|N|N| +2483632|AAAAAAAAALFOFCAA|2087-11-07|2254|9802|752|2087|5|11|7|4|2087|752|9802|Friday|2087Q4|N|Y|N|2483626|2483929|2483267|2483540|N|N|N|N|N| +2483633|AAAAAAAABLFOFCAA|2087-11-08|2254|9802|752|2087|6|11|8|4|2087|752|9802|Saturday|2087Q4|N|Y|N|2483626|2483929|2483268|2483541|N|N|N|N|N| +2483634|AAAAAAAACLFOFCAA|2087-11-09|2254|9802|752|2087|0|11|9|4|2087|752|9802|Sunday|2087Q4|N|N|N|2483626|2483929|2483269|2483542|N|N|N|N|N| +2483635|AAAAAAAADLFOFCAA|2087-11-10|2254|9802|752|2087|1|11|10|4|2087|752|9802|Monday|2087Q4|N|N|N|2483626|2483929|2483270|2483543|N|N|N|N|N| +2483636|AAAAAAAAELFOFCAA|2087-11-11|2254|9803|752|2087|2|11|11|4|2087|752|9803|Tuesday|2087Q4|N|N|N|2483626|2483929|2483271|2483544|N|N|N|N|N| +2483637|AAAAAAAAFLFOFCAA|2087-11-12|2254|9803|752|2087|3|11|12|4|2087|752|9803|Wednesday|2087Q4|N|N|N|2483626|2483929|2483272|2483545|N|N|N|N|N| +2483638|AAAAAAAAGLFOFCAA|2087-11-13|2254|9803|752|2087|4|11|13|4|2087|752|9803|Thursday|2087Q4|N|N|N|2483626|2483929|2483273|2483546|N|N|N|N|N| +2483639|AAAAAAAAHLFOFCAA|2087-11-14|2254|9803|752|2087|5|11|14|4|2087|752|9803|Friday|2087Q4|N|Y|N|2483626|2483929|2483274|2483547|N|N|N|N|N| +2483640|AAAAAAAAILFOFCAA|2087-11-15|2254|9803|752|2087|6|11|15|4|2087|752|9803|Saturday|2087Q4|N|Y|N|2483626|2483929|2483275|2483548|N|N|N|N|N| +2483641|AAAAAAAAJLFOFCAA|2087-11-16|2254|9803|752|2087|0|11|16|4|2087|752|9803|Sunday|2087Q4|N|N|N|2483626|2483929|2483276|2483549|N|N|N|N|N| +2483642|AAAAAAAAKLFOFCAA|2087-11-17|2254|9803|752|2087|1|11|17|4|2087|752|9803|Monday|2087Q4|N|N|N|2483626|2483929|2483277|2483550|N|N|N|N|N| +2483643|AAAAAAAALLFOFCAA|2087-11-18|2254|9804|752|2087|2|11|18|4|2087|752|9804|Tuesday|2087Q4|N|N|N|2483626|2483929|2483278|2483551|N|N|N|N|N| +2483644|AAAAAAAAMLFOFCAA|2087-11-19|2254|9804|752|2087|3|11|19|4|2087|752|9804|Wednesday|2087Q4|N|N|N|2483626|2483929|2483279|2483552|N|N|N|N|N| +2483645|AAAAAAAANLFOFCAA|2087-11-20|2254|9804|752|2087|4|11|20|4|2087|752|9804|Thursday|2087Q4|N|N|N|2483626|2483929|2483280|2483553|N|N|N|N|N| +2483646|AAAAAAAAOLFOFCAA|2087-11-21|2254|9804|752|2087|5|11|21|4|2087|752|9804|Friday|2087Q4|N|Y|N|2483626|2483929|2483281|2483554|N|N|N|N|N| +2483647|AAAAAAAAPLFOFCAA|2087-11-22|2254|9804|752|2087|6|11|22|4|2087|752|9804|Saturday|2087Q4|N|Y|N|2483626|2483929|2483282|2483555|N|N|N|N|N| +2483648|AAAAAAAAAMFOFCAA|2087-11-23|2254|9804|752|2087|0|11|23|4|2087|752|9804|Sunday|2087Q4|N|N|N|2483626|2483929|2483283|2483556|N|N|N|N|N| +2483649|AAAAAAAABMFOFCAA|2087-11-24|2254|9804|752|2087|1|11|24|4|2087|752|9804|Monday|2087Q4|N|N|N|2483626|2483929|2483284|2483557|N|N|N|N|N| +2483650|AAAAAAAACMFOFCAA|2087-11-25|2254|9805|752|2087|2|11|25|4|2087|752|9805|Tuesday|2087Q4|N|N|N|2483626|2483929|2483285|2483558|N|N|N|N|N| +2483651|AAAAAAAADMFOFCAA|2087-11-26|2254|9805|752|2087|3|11|26|4|2087|752|9805|Wednesday|2087Q4|N|N|N|2483626|2483929|2483286|2483559|N|N|N|N|N| +2483652|AAAAAAAAEMFOFCAA|2087-11-27|2254|9805|752|2087|4|11|27|4|2087|752|9805|Thursday|2087Q4|N|N|N|2483626|2483929|2483287|2483560|N|N|N|N|N| +2483653|AAAAAAAAFMFOFCAA|2087-11-28|2254|9805|752|2087|5|11|28|4|2087|752|9805|Friday|2087Q4|N|Y|N|2483626|2483929|2483288|2483561|N|N|N|N|N| +2483654|AAAAAAAAGMFOFCAA|2087-11-29|2254|9805|752|2087|6|11|29|4|2087|752|9805|Saturday|2087Q4|N|Y|N|2483626|2483929|2483289|2483562|N|N|N|N|N| +2483655|AAAAAAAAHMFOFCAA|2087-11-30|2254|9805|752|2087|0|11|30|4|2087|752|9805|Sunday|2087Q4|N|N|N|2483626|2483929|2483290|2483563|N|N|N|N|N| +2483656|AAAAAAAAIMFOFCAA|2087-12-01|2255|9805|753|2087|1|12|1|4|2087|753|9805|Monday|2087Q4|N|N|N|2483656|2483989|2483291|2483564|N|N|N|N|N| +2483657|AAAAAAAAJMFOFCAA|2087-12-02|2255|9806|753|2087|2|12|2|4|2087|753|9806|Tuesday|2087Q4|N|N|N|2483656|2483989|2483292|2483565|N|N|N|N|N| +2483658|AAAAAAAAKMFOFCAA|2087-12-03|2255|9806|753|2087|3|12|3|4|2087|753|9806|Wednesday|2087Q4|N|N|N|2483656|2483989|2483293|2483566|N|N|N|N|N| +2483659|AAAAAAAALMFOFCAA|2087-12-04|2255|9806|753|2087|4|12|4|4|2087|753|9806|Thursday|2087Q4|N|N|N|2483656|2483989|2483294|2483567|N|N|N|N|N| +2483660|AAAAAAAAMMFOFCAA|2087-12-05|2255|9806|753|2087|5|12|5|4|2087|753|9806|Friday|2087Q4|N|Y|N|2483656|2483989|2483295|2483568|N|N|N|N|N| +2483661|AAAAAAAANMFOFCAA|2087-12-06|2255|9806|753|2087|6|12|6|4|2087|753|9806|Saturday|2087Q4|N|Y|N|2483656|2483989|2483296|2483569|N|N|N|N|N| +2483662|AAAAAAAAOMFOFCAA|2087-12-07|2255|9806|753|2087|0|12|7|4|2087|753|9806|Sunday|2087Q4|N|N|N|2483656|2483989|2483297|2483570|N|N|N|N|N| +2483663|AAAAAAAAPMFOFCAA|2087-12-08|2255|9806|753|2087|1|12|8|4|2087|753|9806|Monday|2087Q4|N|N|N|2483656|2483989|2483298|2483571|N|N|N|N|N| +2483664|AAAAAAAAANFOFCAA|2087-12-09|2255|9807|753|2087|2|12|9|4|2087|753|9807|Tuesday|2087Q4|N|N|N|2483656|2483989|2483299|2483572|N|N|N|N|N| +2483665|AAAAAAAABNFOFCAA|2087-12-10|2255|9807|753|2087|3|12|10|4|2087|753|9807|Wednesday|2087Q4|N|N|N|2483656|2483989|2483300|2483573|N|N|N|N|N| +2483666|AAAAAAAACNFOFCAA|2087-12-11|2255|9807|753|2087|4|12|11|4|2087|753|9807|Thursday|2087Q4|N|N|N|2483656|2483989|2483301|2483574|N|N|N|N|N| +2483667|AAAAAAAADNFOFCAA|2087-12-12|2255|9807|753|2087|5|12|12|4|2087|753|9807|Friday|2087Q4|N|Y|N|2483656|2483989|2483302|2483575|N|N|N|N|N| +2483668|AAAAAAAAENFOFCAA|2087-12-13|2255|9807|753|2087|6|12|13|4|2087|753|9807|Saturday|2087Q4|N|Y|N|2483656|2483989|2483303|2483576|N|N|N|N|N| +2483669|AAAAAAAAFNFOFCAA|2087-12-14|2255|9807|753|2087|0|12|14|4|2087|753|9807|Sunday|2087Q4|N|N|N|2483656|2483989|2483304|2483577|N|N|N|N|N| +2483670|AAAAAAAAGNFOFCAA|2087-12-15|2255|9807|753|2087|1|12|15|4|2087|753|9807|Monday|2087Q4|N|N|N|2483656|2483989|2483305|2483578|N|N|N|N|N| +2483671|AAAAAAAAHNFOFCAA|2087-12-16|2255|9808|753|2087|2|12|16|4|2087|753|9808|Tuesday|2087Q4|N|N|N|2483656|2483989|2483306|2483579|N|N|N|N|N| +2483672|AAAAAAAAINFOFCAA|2087-12-17|2255|9808|753|2087|3|12|17|4|2087|753|9808|Wednesday|2087Q4|N|N|N|2483656|2483989|2483307|2483580|N|N|N|N|N| +2483673|AAAAAAAAJNFOFCAA|2087-12-18|2255|9808|753|2087|4|12|18|4|2087|753|9808|Thursday|2087Q4|N|N|N|2483656|2483989|2483308|2483581|N|N|N|N|N| +2483674|AAAAAAAAKNFOFCAA|2087-12-19|2255|9808|753|2087|5|12|19|4|2087|753|9808|Friday|2087Q4|N|Y|N|2483656|2483989|2483309|2483582|N|N|N|N|N| +2483675|AAAAAAAALNFOFCAA|2087-12-20|2255|9808|753|2087|6|12|20|4|2087|753|9808|Saturday|2087Q4|N|Y|N|2483656|2483989|2483310|2483583|N|N|N|N|N| +2483676|AAAAAAAAMNFOFCAA|2087-12-21|2255|9808|753|2087|0|12|21|4|2087|753|9808|Sunday|2087Q4|N|N|N|2483656|2483989|2483311|2483584|N|N|N|N|N| +2483677|AAAAAAAANNFOFCAA|2087-12-22|2255|9808|753|2087|1|12|22|4|2087|753|9808|Monday|2087Q4|N|N|N|2483656|2483989|2483312|2483585|N|N|N|N|N| +2483678|AAAAAAAAONFOFCAA|2087-12-23|2255|9809|753|2087|2|12|23|4|2087|753|9809|Tuesday|2087Q4|N|N|N|2483656|2483989|2483313|2483586|N|N|N|N|N| +2483679|AAAAAAAAPNFOFCAA|2087-12-24|2255|9809|753|2087|3|12|24|4|2087|753|9809|Wednesday|2087Q4|N|N|N|2483656|2483989|2483314|2483587|N|N|N|N|N| +2483680|AAAAAAAAAOFOFCAA|2087-12-25|2255|9809|753|2087|4|12|25|4|2087|753|9809|Thursday|2087Q4|N|N|N|2483656|2483989|2483315|2483588|N|N|N|N|N| +2483681|AAAAAAAABOFOFCAA|2087-12-26|2255|9809|753|2087|5|12|26|4|2087|753|9809|Friday|2087Q4|Y|Y|N|2483656|2483989|2483316|2483589|N|N|N|N|N| +2483682|AAAAAAAACOFOFCAA|2087-12-27|2255|9809|753|2087|6|12|27|4|2087|753|9809|Saturday|2087Q4|N|Y|Y|2483656|2483989|2483317|2483590|N|N|N|N|N| +2483683|AAAAAAAADOFOFCAA|2087-12-28|2255|9809|753|2087|0|12|28|4|2087|753|9809|Sunday|2087Q4|N|N|N|2483656|2483989|2483318|2483591|N|N|N|N|N| +2483684|AAAAAAAAEOFOFCAA|2087-12-29|2255|9809|753|2087|1|12|29|4|2087|753|9809|Monday|2087Q4|N|N|N|2483656|2483989|2483319|2483592|N|N|N|N|N| +2483685|AAAAAAAAFOFOFCAA|2087-12-30|2255|9810|753|2087|2|12|30|4|2087|753|9810|Tuesday|2087Q4|N|N|N|2483656|2483989|2483320|2483593|N|N|N|N|N| +2483686|AAAAAAAAGOFOFCAA|2087-12-31|2255|9810|753|2087|3|12|31|4|2087|753|9810|Wednesday|2087Q4|N|N|N|2483656|2483989|2483321|2483594|N|N|N|N|N| +2483687|AAAAAAAAHOFOFCAA|2088-01-01|2256|9810|753|2088|4|1|1|1|2088|753|9810|Thursday|2088Q1|Y|N|N|2483687|2483686|2483322|2483595|N|N|N|N|N| +2483688|AAAAAAAAIOFOFCAA|2088-01-02|2256|9810|753|2088|5|1|2|1|2088|753|9810|Friday|2088Q1|N|Y|Y|2483687|2483686|2483323|2483596|N|N|N|N|N| +2483689|AAAAAAAAJOFOFCAA|2088-01-03|2256|9810|753|2088|6|1|3|1|2088|753|9810|Saturday|2088Q1|N|Y|N|2483687|2483686|2483324|2483597|N|N|N|N|N| +2483690|AAAAAAAAKOFOFCAA|2088-01-04|2256|9810|753|2088|0|1|4|1|2088|753|9810|Sunday|2088Q1|N|N|N|2483687|2483686|2483325|2483598|N|N|N|N|N| +2483691|AAAAAAAALOFOFCAA|2088-01-05|2256|9810|753|2088|1|1|5|1|2088|753|9810|Monday|2088Q1|N|N|N|2483687|2483686|2483326|2483599|N|N|N|N|N| +2483692|AAAAAAAAMOFOFCAA|2088-01-06|2256|9811|753|2088|2|1|6|1|2088|753|9811|Tuesday|2088Q1|N|N|N|2483687|2483686|2483327|2483600|N|N|N|N|N| +2483693|AAAAAAAANOFOFCAA|2088-01-07|2256|9811|753|2088|3|1|7|1|2088|753|9811|Wednesday|2088Q1|N|N|N|2483687|2483686|2483328|2483601|N|N|N|N|N| +2483694|AAAAAAAAOOFOFCAA|2088-01-08|2256|9811|753|2088|4|1|8|1|2088|753|9811|Thursday|2088Q1|N|N|N|2483687|2483686|2483329|2483602|N|N|N|N|N| +2483695|AAAAAAAAPOFOFCAA|2088-01-09|2256|9811|753|2088|5|1|9|1|2088|753|9811|Friday|2088Q1|N|Y|N|2483687|2483686|2483330|2483603|N|N|N|N|N| +2483696|AAAAAAAAAPFOFCAA|2088-01-10|2256|9811|753|2088|6|1|10|1|2088|753|9811|Saturday|2088Q1|N|Y|N|2483687|2483686|2483331|2483604|N|N|N|N|N| +2483697|AAAAAAAABPFOFCAA|2088-01-11|2256|9811|753|2088|0|1|11|1|2088|753|9811|Sunday|2088Q1|N|N|N|2483687|2483686|2483332|2483605|N|N|N|N|N| +2483698|AAAAAAAACPFOFCAA|2088-01-12|2256|9811|753|2088|1|1|12|1|2088|753|9811|Monday|2088Q1|N|N|N|2483687|2483686|2483333|2483606|N|N|N|N|N| +2483699|AAAAAAAADPFOFCAA|2088-01-13|2256|9812|753|2088|2|1|13|1|2088|753|9812|Tuesday|2088Q1|N|N|N|2483687|2483686|2483334|2483607|N|N|N|N|N| +2483700|AAAAAAAAEPFOFCAA|2088-01-14|2256|9812|753|2088|3|1|14|1|2088|753|9812|Wednesday|2088Q1|N|N|N|2483687|2483686|2483335|2483608|N|N|N|N|N| +2483701|AAAAAAAAFPFOFCAA|2088-01-15|2256|9812|753|2088|4|1|15|1|2088|753|9812|Thursday|2088Q1|N|N|N|2483687|2483686|2483336|2483609|N|N|N|N|N| +2483702|AAAAAAAAGPFOFCAA|2088-01-16|2256|9812|753|2088|5|1|16|1|2088|753|9812|Friday|2088Q1|N|Y|N|2483687|2483686|2483337|2483610|N|N|N|N|N| +2483703|AAAAAAAAHPFOFCAA|2088-01-17|2256|9812|753|2088|6|1|17|1|2088|753|9812|Saturday|2088Q1|N|Y|N|2483687|2483686|2483338|2483611|N|N|N|N|N| +2483704|AAAAAAAAIPFOFCAA|2088-01-18|2256|9812|753|2088|0|1|18|1|2088|753|9812|Sunday|2088Q1|N|N|N|2483687|2483686|2483339|2483612|N|N|N|N|N| +2483705|AAAAAAAAJPFOFCAA|2088-01-19|2256|9812|753|2088|1|1|19|1|2088|753|9812|Monday|2088Q1|N|N|N|2483687|2483686|2483340|2483613|N|N|N|N|N| +2483706|AAAAAAAAKPFOFCAA|2088-01-20|2256|9813|753|2088|2|1|20|1|2088|753|9813|Tuesday|2088Q1|N|N|N|2483687|2483686|2483341|2483614|N|N|N|N|N| +2483707|AAAAAAAALPFOFCAA|2088-01-21|2256|9813|753|2088|3|1|21|1|2088|753|9813|Wednesday|2088Q1|N|N|N|2483687|2483686|2483342|2483615|N|N|N|N|N| +2483708|AAAAAAAAMPFOFCAA|2088-01-22|2256|9813|753|2088|4|1|22|1|2088|753|9813|Thursday|2088Q1|N|N|N|2483687|2483686|2483343|2483616|N|N|N|N|N| +2483709|AAAAAAAANPFOFCAA|2088-01-23|2256|9813|753|2088|5|1|23|1|2088|753|9813|Friday|2088Q1|N|Y|N|2483687|2483686|2483344|2483617|N|N|N|N|N| +2483710|AAAAAAAAOPFOFCAA|2088-01-24|2256|9813|753|2088|6|1|24|1|2088|753|9813|Saturday|2088Q1|N|Y|N|2483687|2483686|2483345|2483618|N|N|N|N|N| +2483711|AAAAAAAAPPFOFCAA|2088-01-25|2256|9813|753|2088|0|1|25|1|2088|753|9813|Sunday|2088Q1|N|N|N|2483687|2483686|2483346|2483619|N|N|N|N|N| +2483712|AAAAAAAAAAGOFCAA|2088-01-26|2256|9813|753|2088|1|1|26|1|2088|753|9813|Monday|2088Q1|N|N|N|2483687|2483686|2483347|2483620|N|N|N|N|N| +2483713|AAAAAAAABAGOFCAA|2088-01-27|2256|9814|753|2088|2|1|27|1|2088|753|9814|Tuesday|2088Q1|N|N|N|2483687|2483686|2483348|2483621|N|N|N|N|N| +2483714|AAAAAAAACAGOFCAA|2088-01-28|2256|9814|753|2088|3|1|28|1|2088|753|9814|Wednesday|2088Q1|N|N|N|2483687|2483686|2483349|2483622|N|N|N|N|N| +2483715|AAAAAAAADAGOFCAA|2088-01-29|2256|9814|753|2088|4|1|29|1|2088|753|9814|Thursday|2088Q1|N|N|N|2483687|2483686|2483350|2483623|N|N|N|N|N| +2483716|AAAAAAAAEAGOFCAA|2088-01-30|2256|9814|753|2088|5|1|30|1|2088|753|9814|Friday|2088Q1|N|Y|N|2483687|2483686|2483351|2483624|N|N|N|N|N| +2483717|AAAAAAAAFAGOFCAA|2088-01-31|2256|9814|753|2088|6|1|31|1|2088|753|9814|Saturday|2088Q1|N|Y|N|2483687|2483686|2483352|2483625|N|N|N|N|N| +2483718|AAAAAAAAGAGOFCAA|2088-02-01|2257|9814|753|2088|0|2|1|1|2088|753|9814|Sunday|2088Q1|N|N|N|2483718|2483748|2483353|2483626|N|N|N|N|N| +2483719|AAAAAAAAHAGOFCAA|2088-02-02|2257|9814|753|2088|1|2|2|1|2088|753|9814|Monday|2088Q1|N|N|N|2483718|2483748|2483354|2483627|N|N|N|N|N| +2483720|AAAAAAAAIAGOFCAA|2088-02-03|2257|9815|753|2088|2|2|3|1|2088|753|9815|Tuesday|2088Q1|N|N|N|2483718|2483748|2483355|2483628|N|N|N|N|N| +2483721|AAAAAAAAJAGOFCAA|2088-02-04|2257|9815|753|2088|3|2|4|1|2088|753|9815|Wednesday|2088Q1|N|N|N|2483718|2483748|2483356|2483629|N|N|N|N|N| +2483722|AAAAAAAAKAGOFCAA|2088-02-05|2257|9815|753|2088|4|2|5|1|2088|753|9815|Thursday|2088Q1|N|N|N|2483718|2483748|2483357|2483630|N|N|N|N|N| +2483723|AAAAAAAALAGOFCAA|2088-02-06|2257|9815|753|2088|5|2|6|1|2088|753|9815|Friday|2088Q1|N|Y|N|2483718|2483748|2483358|2483631|N|N|N|N|N| +2483724|AAAAAAAAMAGOFCAA|2088-02-07|2257|9815|753|2088|6|2|7|1|2088|753|9815|Saturday|2088Q1|N|Y|N|2483718|2483748|2483359|2483632|N|N|N|N|N| +2483725|AAAAAAAANAGOFCAA|2088-02-08|2257|9815|753|2088|0|2|8|1|2088|753|9815|Sunday|2088Q1|N|N|N|2483718|2483748|2483360|2483633|N|N|N|N|N| +2483726|AAAAAAAAOAGOFCAA|2088-02-09|2257|9815|753|2088|1|2|9|1|2088|753|9815|Monday|2088Q1|N|N|N|2483718|2483748|2483361|2483634|N|N|N|N|N| +2483727|AAAAAAAAPAGOFCAA|2088-02-10|2257|9816|753|2088|2|2|10|1|2088|753|9816|Tuesday|2088Q1|N|N|N|2483718|2483748|2483362|2483635|N|N|N|N|N| +2483728|AAAAAAAAABGOFCAA|2088-02-11|2257|9816|753|2088|3|2|11|1|2088|753|9816|Wednesday|2088Q1|N|N|N|2483718|2483748|2483363|2483636|N|N|N|N|N| +2483729|AAAAAAAABBGOFCAA|2088-02-12|2257|9816|753|2088|4|2|12|1|2088|753|9816|Thursday|2088Q1|N|N|N|2483718|2483748|2483364|2483637|N|N|N|N|N| +2483730|AAAAAAAACBGOFCAA|2088-02-13|2257|9816|753|2088|5|2|13|1|2088|753|9816|Friday|2088Q1|N|Y|N|2483718|2483748|2483365|2483638|N|N|N|N|N| +2483731|AAAAAAAADBGOFCAA|2088-02-14|2257|9816|753|2088|6|2|14|1|2088|753|9816|Saturday|2088Q1|N|Y|N|2483718|2483748|2483366|2483639|N|N|N|N|N| +2483732|AAAAAAAAEBGOFCAA|2088-02-15|2257|9816|753|2088|0|2|15|1|2088|753|9816|Sunday|2088Q1|N|N|N|2483718|2483748|2483367|2483640|N|N|N|N|N| +2483733|AAAAAAAAFBGOFCAA|2088-02-16|2257|9816|753|2088|1|2|16|1|2088|753|9816|Monday|2088Q1|N|N|N|2483718|2483748|2483368|2483641|N|N|N|N|N| +2483734|AAAAAAAAGBGOFCAA|2088-02-17|2257|9817|753|2088|2|2|17|1|2088|753|9817|Tuesday|2088Q1|N|N|N|2483718|2483748|2483369|2483642|N|N|N|N|N| +2483735|AAAAAAAAHBGOFCAA|2088-02-18|2257|9817|753|2088|3|2|18|1|2088|753|9817|Wednesday|2088Q1|N|N|N|2483718|2483748|2483370|2483643|N|N|N|N|N| +2483736|AAAAAAAAIBGOFCAA|2088-02-19|2257|9817|753|2088|4|2|19|1|2088|753|9817|Thursday|2088Q1|N|N|N|2483718|2483748|2483371|2483644|N|N|N|N|N| +2483737|AAAAAAAAJBGOFCAA|2088-02-20|2257|9817|753|2088|5|2|20|1|2088|753|9817|Friday|2088Q1|N|Y|N|2483718|2483748|2483372|2483645|N|N|N|N|N| +2483738|AAAAAAAAKBGOFCAA|2088-02-21|2257|9817|753|2088|6|2|21|1|2088|753|9817|Saturday|2088Q1|N|Y|N|2483718|2483748|2483373|2483646|N|N|N|N|N| +2483739|AAAAAAAALBGOFCAA|2088-02-22|2257|9817|753|2088|0|2|22|1|2088|753|9817|Sunday|2088Q1|N|N|N|2483718|2483748|2483374|2483647|N|N|N|N|N| +2483740|AAAAAAAAMBGOFCAA|2088-02-23|2257|9817|753|2088|1|2|23|1|2088|753|9817|Monday|2088Q1|N|N|N|2483718|2483748|2483375|2483648|N|N|N|N|N| +2483741|AAAAAAAANBGOFCAA|2088-02-24|2257|9818|753|2088|2|2|24|1|2088|753|9818|Tuesday|2088Q1|N|N|N|2483718|2483748|2483376|2483649|N|N|N|N|N| +2483742|AAAAAAAAOBGOFCAA|2088-02-25|2257|9818|753|2088|3|2|25|1|2088|753|9818|Wednesday|2088Q1|N|N|N|2483718|2483748|2483377|2483650|N|N|N|N|N| +2483743|AAAAAAAAPBGOFCAA|2088-02-26|2257|9818|753|2088|4|2|26|1|2088|753|9818|Thursday|2088Q1|N|N|N|2483718|2483748|2483378|2483651|N|N|N|N|N| +2483744|AAAAAAAAACGOFCAA|2088-02-27|2257|9818|753|2088|5|2|27|1|2088|753|9818|Friday|2088Q1|N|Y|N|2483718|2483748|2483379|2483652|N|N|N|N|N| +2483745|AAAAAAAABCGOFCAA|2088-02-28|2257|9818|753|2088|6|2|28|1|2088|753|9818|Saturday|2088Q1|N|Y|N|2483718|2483748|2483380|2483653|N|N|N|N|N| +2483746|AAAAAAAACCGOFCAA|2088-02-29|2257|9818|753|2088|0|2|29|1|2088|753|9818|Sunday|2088Q1|N|N|N|2483718|2483748|2483380|2483654|N|N|N|N|N| +2483747|AAAAAAAADCGOFCAA|2088-03-01|2258|9818|754|2088|1|3|1|1|2088|754|9818|Monday|2088Q1|N|N|N|2483747|2483806|2483381|2483655|N|N|N|N|N| +2483748|AAAAAAAAECGOFCAA|2088-03-02|2258|9819|754|2088|2|3|2|1|2088|754|9819|Tuesday|2088Q1|N|N|N|2483747|2483806|2483382|2483656|N|N|N|N|N| +2483749|AAAAAAAAFCGOFCAA|2088-03-03|2258|9819|754|2088|3|3|3|1|2088|754|9819|Wednesday|2088Q1|N|N|N|2483747|2483806|2483383|2483657|N|N|N|N|N| +2483750|AAAAAAAAGCGOFCAA|2088-03-04|2258|9819|754|2088|4|3|4|1|2088|754|9819|Thursday|2088Q1|N|N|N|2483747|2483806|2483384|2483658|N|N|N|N|N| +2483751|AAAAAAAAHCGOFCAA|2088-03-05|2258|9819|754|2088|5|3|5|1|2088|754|9819|Friday|2088Q1|N|Y|N|2483747|2483806|2483385|2483659|N|N|N|N|N| +2483752|AAAAAAAAICGOFCAA|2088-03-06|2258|9819|754|2088|6|3|6|1|2088|754|9819|Saturday|2088Q1|N|Y|N|2483747|2483806|2483386|2483660|N|N|N|N|N| +2483753|AAAAAAAAJCGOFCAA|2088-03-07|2258|9819|754|2088|0|3|7|1|2088|754|9819|Sunday|2088Q1|N|N|N|2483747|2483806|2483387|2483661|N|N|N|N|N| +2483754|AAAAAAAAKCGOFCAA|2088-03-08|2258|9819|754|2088|1|3|8|1|2088|754|9819|Monday|2088Q1|N|N|N|2483747|2483806|2483388|2483662|N|N|N|N|N| +2483755|AAAAAAAALCGOFCAA|2088-03-09|2258|9820|754|2088|2|3|9|1|2088|754|9820|Tuesday|2088Q1|N|N|N|2483747|2483806|2483389|2483663|N|N|N|N|N| +2483756|AAAAAAAAMCGOFCAA|2088-03-10|2258|9820|754|2088|3|3|10|1|2088|754|9820|Wednesday|2088Q1|N|N|N|2483747|2483806|2483390|2483664|N|N|N|N|N| +2483757|AAAAAAAANCGOFCAA|2088-03-11|2258|9820|754|2088|4|3|11|1|2088|754|9820|Thursday|2088Q1|N|N|N|2483747|2483806|2483391|2483665|N|N|N|N|N| +2483758|AAAAAAAAOCGOFCAA|2088-03-12|2258|9820|754|2088|5|3|12|1|2088|754|9820|Friday|2088Q1|N|Y|N|2483747|2483806|2483392|2483666|N|N|N|N|N| +2483759|AAAAAAAAPCGOFCAA|2088-03-13|2258|9820|754|2088|6|3|13|1|2088|754|9820|Saturday|2088Q1|N|Y|N|2483747|2483806|2483393|2483667|N|N|N|N|N| +2483760|AAAAAAAAADGOFCAA|2088-03-14|2258|9820|754|2088|0|3|14|1|2088|754|9820|Sunday|2088Q1|N|N|N|2483747|2483806|2483394|2483668|N|N|N|N|N| +2483761|AAAAAAAABDGOFCAA|2088-03-15|2258|9820|754|2088|1|3|15|1|2088|754|9820|Monday|2088Q1|N|N|N|2483747|2483806|2483395|2483669|N|N|N|N|N| +2483762|AAAAAAAACDGOFCAA|2088-03-16|2258|9821|754|2088|2|3|16|1|2088|754|9821|Tuesday|2088Q1|N|N|N|2483747|2483806|2483396|2483670|N|N|N|N|N| +2483763|AAAAAAAADDGOFCAA|2088-03-17|2258|9821|754|2088|3|3|17|1|2088|754|9821|Wednesday|2088Q1|N|N|N|2483747|2483806|2483397|2483671|N|N|N|N|N| +2483764|AAAAAAAAEDGOFCAA|2088-03-18|2258|9821|754|2088|4|3|18|1|2088|754|9821|Thursday|2088Q1|N|N|N|2483747|2483806|2483398|2483672|N|N|N|N|N| +2483765|AAAAAAAAFDGOFCAA|2088-03-19|2258|9821|754|2088|5|3|19|1|2088|754|9821|Friday|2088Q1|N|Y|N|2483747|2483806|2483399|2483673|N|N|N|N|N| +2483766|AAAAAAAAGDGOFCAA|2088-03-20|2258|9821|754|2088|6|3|20|1|2088|754|9821|Saturday|2088Q1|N|Y|N|2483747|2483806|2483400|2483674|N|N|N|N|N| +2483767|AAAAAAAAHDGOFCAA|2088-03-21|2258|9821|754|2088|0|3|21|1|2088|754|9821|Sunday|2088Q1|N|N|N|2483747|2483806|2483401|2483675|N|N|N|N|N| +2483768|AAAAAAAAIDGOFCAA|2088-03-22|2258|9821|754|2088|1|3|22|1|2088|754|9821|Monday|2088Q1|N|N|N|2483747|2483806|2483402|2483676|N|N|N|N|N| +2483769|AAAAAAAAJDGOFCAA|2088-03-23|2258|9822|754|2088|2|3|23|1|2088|754|9822|Tuesday|2088Q1|N|N|N|2483747|2483806|2483403|2483677|N|N|N|N|N| +2483770|AAAAAAAAKDGOFCAA|2088-03-24|2258|9822|754|2088|3|3|24|1|2088|754|9822|Wednesday|2088Q1|N|N|N|2483747|2483806|2483404|2483678|N|N|N|N|N| +2483771|AAAAAAAALDGOFCAA|2088-03-25|2258|9822|754|2088|4|3|25|1|2088|754|9822|Thursday|2088Q1|N|N|N|2483747|2483806|2483405|2483679|N|N|N|N|N| +2483772|AAAAAAAAMDGOFCAA|2088-03-26|2258|9822|754|2088|5|3|26|1|2088|754|9822|Friday|2088Q1|N|Y|N|2483747|2483806|2483406|2483680|N|N|N|N|N| +2483773|AAAAAAAANDGOFCAA|2088-03-27|2258|9822|754|2088|6|3|27|1|2088|754|9822|Saturday|2088Q1|N|Y|N|2483747|2483806|2483407|2483681|N|N|N|N|N| +2483774|AAAAAAAAODGOFCAA|2088-03-28|2258|9822|754|2088|0|3|28|1|2088|754|9822|Sunday|2088Q1|N|N|N|2483747|2483806|2483408|2483682|N|N|N|N|N| +2483775|AAAAAAAAPDGOFCAA|2088-03-29|2258|9822|754|2088|1|3|29|1|2088|754|9822|Monday|2088Q1|N|N|N|2483747|2483806|2483409|2483683|N|N|N|N|N| +2483776|AAAAAAAAAEGOFCAA|2088-03-30|2258|9823|754|2088|2|3|30|1|2088|754|9823|Tuesday|2088Q1|N|N|N|2483747|2483806|2483410|2483684|N|N|N|N|N| +2483777|AAAAAAAABEGOFCAA|2088-03-31|2258|9823|754|2088|3|3|31|1|2088|754|9823|Wednesday|2088Q1|N|N|N|2483747|2483806|2483411|2483685|N|N|N|N|N| +2483778|AAAAAAAACEGOFCAA|2088-04-01|2259|9823|754|2088|4|4|1|2|2088|754|9823|Thursday|2088Q2|N|N|N|2483778|2483868|2483412|2483687|N|N|N|N|N| +2483779|AAAAAAAADEGOFCAA|2088-04-02|2259|9823|754|2088|5|4|2|2|2088|754|9823|Friday|2088Q2|N|Y|N|2483778|2483868|2483413|2483688|N|N|N|N|N| +2483780|AAAAAAAAEEGOFCAA|2088-04-03|2259|9823|754|2088|6|4|3|2|2088|754|9823|Saturday|2088Q2|N|Y|N|2483778|2483868|2483414|2483689|N|N|N|N|N| +2483781|AAAAAAAAFEGOFCAA|2088-04-04|2259|9823|754|2088|0|4|4|2|2088|754|9823|Sunday|2088Q2|N|N|N|2483778|2483868|2483415|2483690|N|N|N|N|N| +2483782|AAAAAAAAGEGOFCAA|2088-04-05|2259|9823|754|2088|1|4|5|2|2088|754|9823|Monday|2088Q2|N|N|N|2483778|2483868|2483416|2483691|N|N|N|N|N| +2483783|AAAAAAAAHEGOFCAA|2088-04-06|2259|9824|754|2088|2|4|6|2|2088|754|9824|Tuesday|2088Q2|N|N|N|2483778|2483868|2483417|2483692|N|N|N|N|N| +2483784|AAAAAAAAIEGOFCAA|2088-04-07|2259|9824|754|2088|3|4|7|2|2088|754|9824|Wednesday|2088Q2|N|N|N|2483778|2483868|2483418|2483693|N|N|N|N|N| +2483785|AAAAAAAAJEGOFCAA|2088-04-08|2259|9824|754|2088|4|4|8|2|2088|754|9824|Thursday|2088Q2|N|N|N|2483778|2483868|2483419|2483694|N|N|N|N|N| +2483786|AAAAAAAAKEGOFCAA|2088-04-09|2259|9824|754|2088|5|4|9|2|2088|754|9824|Friday|2088Q2|N|Y|N|2483778|2483868|2483420|2483695|N|N|N|N|N| +2483787|AAAAAAAALEGOFCAA|2088-04-10|2259|9824|754|2088|6|4|10|2|2088|754|9824|Saturday|2088Q2|N|Y|N|2483778|2483868|2483421|2483696|N|N|N|N|N| +2483788|AAAAAAAAMEGOFCAA|2088-04-11|2259|9824|754|2088|0|4|11|2|2088|754|9824|Sunday|2088Q2|N|N|N|2483778|2483868|2483422|2483697|N|N|N|N|N| +2483789|AAAAAAAANEGOFCAA|2088-04-12|2259|9824|754|2088|1|4|12|2|2088|754|9824|Monday|2088Q2|N|N|N|2483778|2483868|2483423|2483698|N|N|N|N|N| +2483790|AAAAAAAAOEGOFCAA|2088-04-13|2259|9825|754|2088|2|4|13|2|2088|754|9825|Tuesday|2088Q2|N|N|N|2483778|2483868|2483424|2483699|N|N|N|N|N| +2483791|AAAAAAAAPEGOFCAA|2088-04-14|2259|9825|754|2088|3|4|14|2|2088|754|9825|Wednesday|2088Q2|N|N|N|2483778|2483868|2483425|2483700|N|N|N|N|N| +2483792|AAAAAAAAAFGOFCAA|2088-04-15|2259|9825|754|2088|4|4|15|2|2088|754|9825|Thursday|2088Q2|N|N|N|2483778|2483868|2483426|2483701|N|N|N|N|N| +2483793|AAAAAAAABFGOFCAA|2088-04-16|2259|9825|754|2088|5|4|16|2|2088|754|9825|Friday|2088Q2|N|Y|N|2483778|2483868|2483427|2483702|N|N|N|N|N| +2483794|AAAAAAAACFGOFCAA|2088-04-17|2259|9825|754|2088|6|4|17|2|2088|754|9825|Saturday|2088Q2|N|Y|N|2483778|2483868|2483428|2483703|N|N|N|N|N| +2483795|AAAAAAAADFGOFCAA|2088-04-18|2259|9825|754|2088|0|4|18|2|2088|754|9825|Sunday|2088Q2|N|N|N|2483778|2483868|2483429|2483704|N|N|N|N|N| +2483796|AAAAAAAAEFGOFCAA|2088-04-19|2259|9825|754|2088|1|4|19|2|2088|754|9825|Monday|2088Q2|N|N|N|2483778|2483868|2483430|2483705|N|N|N|N|N| +2483797|AAAAAAAAFFGOFCAA|2088-04-20|2259|9826|754|2088|2|4|20|2|2088|754|9826|Tuesday|2088Q2|N|N|N|2483778|2483868|2483431|2483706|N|N|N|N|N| +2483798|AAAAAAAAGFGOFCAA|2088-04-21|2259|9826|754|2088|3|4|21|2|2088|754|9826|Wednesday|2088Q2|N|N|N|2483778|2483868|2483432|2483707|N|N|N|N|N| +2483799|AAAAAAAAHFGOFCAA|2088-04-22|2259|9826|754|2088|4|4|22|2|2088|754|9826|Thursday|2088Q2|N|N|N|2483778|2483868|2483433|2483708|N|N|N|N|N| +2483800|AAAAAAAAIFGOFCAA|2088-04-23|2259|9826|754|2088|5|4|23|2|2088|754|9826|Friday|2088Q2|N|Y|N|2483778|2483868|2483434|2483709|N|N|N|N|N| +2483801|AAAAAAAAJFGOFCAA|2088-04-24|2259|9826|754|2088|6|4|24|2|2088|754|9826|Saturday|2088Q2|N|Y|N|2483778|2483868|2483435|2483710|N|N|N|N|N| +2483802|AAAAAAAAKFGOFCAA|2088-04-25|2259|9826|754|2088|0|4|25|2|2088|754|9826|Sunday|2088Q2|N|N|N|2483778|2483868|2483436|2483711|N|N|N|N|N| +2483803|AAAAAAAALFGOFCAA|2088-04-26|2259|9826|754|2088|1|4|26|2|2088|754|9826|Monday|2088Q2|N|N|N|2483778|2483868|2483437|2483712|N|N|N|N|N| +2483804|AAAAAAAAMFGOFCAA|2088-04-27|2259|9827|754|2088|2|4|27|2|2088|754|9827|Tuesday|2088Q2|N|N|N|2483778|2483868|2483438|2483713|N|N|N|N|N| +2483805|AAAAAAAANFGOFCAA|2088-04-28|2259|9827|754|2088|3|4|28|2|2088|754|9827|Wednesday|2088Q2|N|N|N|2483778|2483868|2483439|2483714|N|N|N|N|N| +2483806|AAAAAAAAOFGOFCAA|2088-04-29|2259|9827|754|2088|4|4|29|2|2088|754|9827|Thursday|2088Q2|N|N|N|2483778|2483868|2483440|2483715|N|N|N|N|N| +2483807|AAAAAAAAPFGOFCAA|2088-04-30|2259|9827|754|2088|5|4|30|2|2088|754|9827|Friday|2088Q2|N|Y|N|2483778|2483868|2483441|2483716|N|N|N|N|N| +2483808|AAAAAAAAAGGOFCAA|2088-05-01|2260|9827|754|2088|6|5|1|2|2088|754|9827|Saturday|2088Q2|N|Y|N|2483808|2483928|2483442|2483717|N|N|N|N|N| +2483809|AAAAAAAABGGOFCAA|2088-05-02|2260|9827|754|2088|0|5|2|2|2088|754|9827|Sunday|2088Q2|N|N|N|2483808|2483928|2483443|2483718|N|N|N|N|N| +2483810|AAAAAAAACGGOFCAA|2088-05-03|2260|9827|754|2088|1|5|3|2|2088|754|9827|Monday|2088Q2|N|N|N|2483808|2483928|2483444|2483719|N|N|N|N|N| +2483811|AAAAAAAADGGOFCAA|2088-05-04|2260|9828|754|2088|2|5|4|2|2088|754|9828|Tuesday|2088Q2|N|N|N|2483808|2483928|2483445|2483720|N|N|N|N|N| +2483812|AAAAAAAAEGGOFCAA|2088-05-05|2260|9828|754|2088|3|5|5|2|2088|754|9828|Wednesday|2088Q2|N|N|N|2483808|2483928|2483446|2483721|N|N|N|N|N| +2483813|AAAAAAAAFGGOFCAA|2088-05-06|2260|9828|754|2088|4|5|6|2|2088|754|9828|Thursday|2088Q2|N|N|N|2483808|2483928|2483447|2483722|N|N|N|N|N| +2483814|AAAAAAAAGGGOFCAA|2088-05-07|2260|9828|754|2088|5|5|7|2|2088|754|9828|Friday|2088Q2|N|Y|N|2483808|2483928|2483448|2483723|N|N|N|N|N| +2483815|AAAAAAAAHGGOFCAA|2088-05-08|2260|9828|754|2088|6|5|8|2|2088|754|9828|Saturday|2088Q2|N|Y|N|2483808|2483928|2483449|2483724|N|N|N|N|N| +2483816|AAAAAAAAIGGOFCAA|2088-05-09|2260|9828|754|2088|0|5|9|2|2088|754|9828|Sunday|2088Q2|N|N|N|2483808|2483928|2483450|2483725|N|N|N|N|N| +2483817|AAAAAAAAJGGOFCAA|2088-05-10|2260|9828|754|2088|1|5|10|2|2088|754|9828|Monday|2088Q2|N|N|N|2483808|2483928|2483451|2483726|N|N|N|N|N| +2483818|AAAAAAAAKGGOFCAA|2088-05-11|2260|9829|754|2088|2|5|11|2|2088|754|9829|Tuesday|2088Q2|N|N|N|2483808|2483928|2483452|2483727|N|N|N|N|N| +2483819|AAAAAAAALGGOFCAA|2088-05-12|2260|9829|754|2088|3|5|12|2|2088|754|9829|Wednesday|2088Q2|N|N|N|2483808|2483928|2483453|2483728|N|N|N|N|N| +2483820|AAAAAAAAMGGOFCAA|2088-05-13|2260|9829|754|2088|4|5|13|2|2088|754|9829|Thursday|2088Q2|N|N|N|2483808|2483928|2483454|2483729|N|N|N|N|N| +2483821|AAAAAAAANGGOFCAA|2088-05-14|2260|9829|754|2088|5|5|14|2|2088|754|9829|Friday|2088Q2|N|Y|N|2483808|2483928|2483455|2483730|N|N|N|N|N| +2483822|AAAAAAAAOGGOFCAA|2088-05-15|2260|9829|754|2088|6|5|15|2|2088|754|9829|Saturday|2088Q2|N|Y|N|2483808|2483928|2483456|2483731|N|N|N|N|N| +2483823|AAAAAAAAPGGOFCAA|2088-05-16|2260|9829|754|2088|0|5|16|2|2088|754|9829|Sunday|2088Q2|N|N|N|2483808|2483928|2483457|2483732|N|N|N|N|N| +2483824|AAAAAAAAAHGOFCAA|2088-05-17|2260|9829|754|2088|1|5|17|2|2088|754|9829|Monday|2088Q2|N|N|N|2483808|2483928|2483458|2483733|N|N|N|N|N| +2483825|AAAAAAAABHGOFCAA|2088-05-18|2260|9830|754|2088|2|5|18|2|2088|754|9830|Tuesday|2088Q2|N|N|N|2483808|2483928|2483459|2483734|N|N|N|N|N| +2483826|AAAAAAAACHGOFCAA|2088-05-19|2260|9830|754|2088|3|5|19|2|2088|754|9830|Wednesday|2088Q2|N|N|N|2483808|2483928|2483460|2483735|N|N|N|N|N| +2483827|AAAAAAAADHGOFCAA|2088-05-20|2260|9830|754|2088|4|5|20|2|2088|754|9830|Thursday|2088Q2|N|N|N|2483808|2483928|2483461|2483736|N|N|N|N|N| +2483828|AAAAAAAAEHGOFCAA|2088-05-21|2260|9830|754|2088|5|5|21|2|2088|754|9830|Friday|2088Q2|N|Y|N|2483808|2483928|2483462|2483737|N|N|N|N|N| +2483829|AAAAAAAAFHGOFCAA|2088-05-22|2260|9830|754|2088|6|5|22|2|2088|754|9830|Saturday|2088Q2|N|Y|N|2483808|2483928|2483463|2483738|N|N|N|N|N| +2483830|AAAAAAAAGHGOFCAA|2088-05-23|2260|9830|754|2088|0|5|23|2|2088|754|9830|Sunday|2088Q2|N|N|N|2483808|2483928|2483464|2483739|N|N|N|N|N| +2483831|AAAAAAAAHHGOFCAA|2088-05-24|2260|9830|754|2088|1|5|24|2|2088|754|9830|Monday|2088Q2|N|N|N|2483808|2483928|2483465|2483740|N|N|N|N|N| +2483832|AAAAAAAAIHGOFCAA|2088-05-25|2260|9831|754|2088|2|5|25|2|2088|754|9831|Tuesday|2088Q2|N|N|N|2483808|2483928|2483466|2483741|N|N|N|N|N| +2483833|AAAAAAAAJHGOFCAA|2088-05-26|2260|9831|754|2088|3|5|26|2|2088|754|9831|Wednesday|2088Q2|N|N|N|2483808|2483928|2483467|2483742|N|N|N|N|N| +2483834|AAAAAAAAKHGOFCAA|2088-05-27|2260|9831|754|2088|4|5|27|2|2088|754|9831|Thursday|2088Q2|N|N|N|2483808|2483928|2483468|2483743|N|N|N|N|N| +2483835|AAAAAAAALHGOFCAA|2088-05-28|2260|9831|754|2088|5|5|28|2|2088|754|9831|Friday|2088Q2|N|Y|N|2483808|2483928|2483469|2483744|N|N|N|N|N| +2483836|AAAAAAAAMHGOFCAA|2088-05-29|2260|9831|754|2088|6|5|29|2|2088|754|9831|Saturday|2088Q2|N|Y|N|2483808|2483928|2483470|2483745|N|N|N|N|N| +2483837|AAAAAAAANHGOFCAA|2088-05-30|2260|9831|754|2088|0|5|30|2|2088|754|9831|Sunday|2088Q2|N|N|N|2483808|2483928|2483471|2483746|N|N|N|N|N| +2483838|AAAAAAAAOHGOFCAA|2088-05-31|2260|9831|754|2088|1|5|31|2|2088|754|9831|Monday|2088Q2|N|N|N|2483808|2483928|2483472|2483747|N|N|N|N|N| +2483839|AAAAAAAAPHGOFCAA|2088-06-01|2261|9832|755|2088|2|6|1|2|2088|755|9832|Tuesday|2088Q2|N|N|N|2483839|2483990|2483473|2483748|N|N|N|N|N| +2483840|AAAAAAAAAIGOFCAA|2088-06-02|2261|9832|755|2088|3|6|2|2|2088|755|9832|Wednesday|2088Q2|N|N|N|2483839|2483990|2483474|2483749|N|N|N|N|N| +2483841|AAAAAAAABIGOFCAA|2088-06-03|2261|9832|755|2088|4|6|3|2|2088|755|9832|Thursday|2088Q2|N|N|N|2483839|2483990|2483475|2483750|N|N|N|N|N| +2483842|AAAAAAAACIGOFCAA|2088-06-04|2261|9832|755|2088|5|6|4|2|2088|755|9832|Friday|2088Q2|N|Y|N|2483839|2483990|2483476|2483751|N|N|N|N|N| +2483843|AAAAAAAADIGOFCAA|2088-06-05|2261|9832|755|2088|6|6|5|2|2088|755|9832|Saturday|2088Q2|N|Y|N|2483839|2483990|2483477|2483752|N|N|N|N|N| +2483844|AAAAAAAAEIGOFCAA|2088-06-06|2261|9832|755|2088|0|6|6|2|2088|755|9832|Sunday|2088Q2|N|N|N|2483839|2483990|2483478|2483753|N|N|N|N|N| +2483845|AAAAAAAAFIGOFCAA|2088-06-07|2261|9832|755|2088|1|6|7|2|2088|755|9832|Monday|2088Q2|N|N|N|2483839|2483990|2483479|2483754|N|N|N|N|N| +2483846|AAAAAAAAGIGOFCAA|2088-06-08|2261|9833|755|2088|2|6|8|2|2088|755|9833|Tuesday|2088Q2|N|N|N|2483839|2483990|2483480|2483755|N|N|N|N|N| +2483847|AAAAAAAAHIGOFCAA|2088-06-09|2261|9833|755|2088|3|6|9|2|2088|755|9833|Wednesday|2088Q2|N|N|N|2483839|2483990|2483481|2483756|N|N|N|N|N| +2483848|AAAAAAAAIIGOFCAA|2088-06-10|2261|9833|755|2088|4|6|10|2|2088|755|9833|Thursday|2088Q2|N|N|N|2483839|2483990|2483482|2483757|N|N|N|N|N| +2483849|AAAAAAAAJIGOFCAA|2088-06-11|2261|9833|755|2088|5|6|11|2|2088|755|9833|Friday|2088Q2|N|Y|N|2483839|2483990|2483483|2483758|N|N|N|N|N| +2483850|AAAAAAAAKIGOFCAA|2088-06-12|2261|9833|755|2088|6|6|12|2|2088|755|9833|Saturday|2088Q2|N|Y|N|2483839|2483990|2483484|2483759|N|N|N|N|N| +2483851|AAAAAAAALIGOFCAA|2088-06-13|2261|9833|755|2088|0|6|13|2|2088|755|9833|Sunday|2088Q2|N|N|N|2483839|2483990|2483485|2483760|N|N|N|N|N| +2483852|AAAAAAAAMIGOFCAA|2088-06-14|2261|9833|755|2088|1|6|14|2|2088|755|9833|Monday|2088Q2|N|N|N|2483839|2483990|2483486|2483761|N|N|N|N|N| +2483853|AAAAAAAANIGOFCAA|2088-06-15|2261|9834|755|2088|2|6|15|2|2088|755|9834|Tuesday|2088Q2|N|N|N|2483839|2483990|2483487|2483762|N|N|N|N|N| +2483854|AAAAAAAAOIGOFCAA|2088-06-16|2261|9834|755|2088|3|6|16|2|2088|755|9834|Wednesday|2088Q2|N|N|N|2483839|2483990|2483488|2483763|N|N|N|N|N| +2483855|AAAAAAAAPIGOFCAA|2088-06-17|2261|9834|755|2088|4|6|17|2|2088|755|9834|Thursday|2088Q2|N|N|N|2483839|2483990|2483489|2483764|N|N|N|N|N| +2483856|AAAAAAAAAJGOFCAA|2088-06-18|2261|9834|755|2088|5|6|18|2|2088|755|9834|Friday|2088Q2|N|Y|N|2483839|2483990|2483490|2483765|N|N|N|N|N| +2483857|AAAAAAAABJGOFCAA|2088-06-19|2261|9834|755|2088|6|6|19|2|2088|755|9834|Saturday|2088Q2|N|Y|N|2483839|2483990|2483491|2483766|N|N|N|N|N| +2483858|AAAAAAAACJGOFCAA|2088-06-20|2261|9834|755|2088|0|6|20|2|2088|755|9834|Sunday|2088Q2|N|N|N|2483839|2483990|2483492|2483767|N|N|N|N|N| +2483859|AAAAAAAADJGOFCAA|2088-06-21|2261|9834|755|2088|1|6|21|2|2088|755|9834|Monday|2088Q2|N|N|N|2483839|2483990|2483493|2483768|N|N|N|N|N| +2483860|AAAAAAAAEJGOFCAA|2088-06-22|2261|9835|755|2088|2|6|22|2|2088|755|9835|Tuesday|2088Q2|N|N|N|2483839|2483990|2483494|2483769|N|N|N|N|N| +2483861|AAAAAAAAFJGOFCAA|2088-06-23|2261|9835|755|2088|3|6|23|2|2088|755|9835|Wednesday|2088Q2|N|N|N|2483839|2483990|2483495|2483770|N|N|N|N|N| +2483862|AAAAAAAAGJGOFCAA|2088-06-24|2261|9835|755|2088|4|6|24|2|2088|755|9835|Thursday|2088Q2|N|N|N|2483839|2483990|2483496|2483771|N|N|N|N|N| +2483863|AAAAAAAAHJGOFCAA|2088-06-25|2261|9835|755|2088|5|6|25|2|2088|755|9835|Friday|2088Q2|N|Y|N|2483839|2483990|2483497|2483772|N|N|N|N|N| +2483864|AAAAAAAAIJGOFCAA|2088-06-26|2261|9835|755|2088|6|6|26|2|2088|755|9835|Saturday|2088Q2|N|Y|N|2483839|2483990|2483498|2483773|N|N|N|N|N| +2483865|AAAAAAAAJJGOFCAA|2088-06-27|2261|9835|755|2088|0|6|27|2|2088|755|9835|Sunday|2088Q2|N|N|N|2483839|2483990|2483499|2483774|N|N|N|N|N| +2483866|AAAAAAAAKJGOFCAA|2088-06-28|2261|9835|755|2088|1|6|28|2|2088|755|9835|Monday|2088Q2|N|N|N|2483839|2483990|2483500|2483775|N|N|N|N|N| +2483867|AAAAAAAALJGOFCAA|2088-06-29|2261|9836|755|2088|2|6|29|2|2088|755|9836|Tuesday|2088Q2|N|N|N|2483839|2483990|2483501|2483776|N|N|N|N|N| +2483868|AAAAAAAAMJGOFCAA|2088-06-30|2261|9836|755|2088|3|6|30|2|2088|755|9836|Wednesday|2088Q2|N|N|N|2483839|2483990|2483502|2483777|N|N|N|N|N| +2483869|AAAAAAAANJGOFCAA|2088-07-01|2262|9836|755|2088|4|7|1|3|2088|755|9836|Thursday|2088Q3|N|N|N|2483869|2484050|2483503|2483778|N|N|N|N|N| +2483870|AAAAAAAAOJGOFCAA|2088-07-02|2262|9836|755|2088|5|7|2|3|2088|755|9836|Friday|2088Q3|N|Y|N|2483869|2484050|2483504|2483779|N|N|N|N|N| +2483871|AAAAAAAAPJGOFCAA|2088-07-03|2262|9836|755|2088|6|7|3|3|2088|755|9836|Saturday|2088Q3|N|Y|N|2483869|2484050|2483505|2483780|N|N|N|N|N| +2483872|AAAAAAAAAKGOFCAA|2088-07-04|2262|9836|755|2088|0|7|4|3|2088|755|9836|Sunday|2088Q3|Y|N|N|2483869|2484050|2483506|2483781|N|N|N|N|N| +2483873|AAAAAAAABKGOFCAA|2088-07-05|2262|9836|755|2088|1|7|5|3|2088|755|9836|Monday|2088Q3|N|N|Y|2483869|2484050|2483507|2483782|N|N|N|N|N| +2483874|AAAAAAAACKGOFCAA|2088-07-06|2262|9837|755|2088|2|7|6|3|2088|755|9837|Tuesday|2088Q3|N|N|N|2483869|2484050|2483508|2483783|N|N|N|N|N| +2483875|AAAAAAAADKGOFCAA|2088-07-07|2262|9837|755|2088|3|7|7|3|2088|755|9837|Wednesday|2088Q3|N|N|N|2483869|2484050|2483509|2483784|N|N|N|N|N| +2483876|AAAAAAAAEKGOFCAA|2088-07-08|2262|9837|755|2088|4|7|8|3|2088|755|9837|Thursday|2088Q3|N|N|N|2483869|2484050|2483510|2483785|N|N|N|N|N| +2483877|AAAAAAAAFKGOFCAA|2088-07-09|2262|9837|755|2088|5|7|9|3|2088|755|9837|Friday|2088Q3|N|Y|N|2483869|2484050|2483511|2483786|N|N|N|N|N| +2483878|AAAAAAAAGKGOFCAA|2088-07-10|2262|9837|755|2088|6|7|10|3|2088|755|9837|Saturday|2088Q3|N|Y|N|2483869|2484050|2483512|2483787|N|N|N|N|N| +2483879|AAAAAAAAHKGOFCAA|2088-07-11|2262|9837|755|2088|0|7|11|3|2088|755|9837|Sunday|2088Q3|N|N|N|2483869|2484050|2483513|2483788|N|N|N|N|N| +2483880|AAAAAAAAIKGOFCAA|2088-07-12|2262|9837|755|2088|1|7|12|3|2088|755|9837|Monday|2088Q3|N|N|N|2483869|2484050|2483514|2483789|N|N|N|N|N| +2483881|AAAAAAAAJKGOFCAA|2088-07-13|2262|9838|755|2088|2|7|13|3|2088|755|9838|Tuesday|2088Q3|N|N|N|2483869|2484050|2483515|2483790|N|N|N|N|N| +2483882|AAAAAAAAKKGOFCAA|2088-07-14|2262|9838|755|2088|3|7|14|3|2088|755|9838|Wednesday|2088Q3|N|N|N|2483869|2484050|2483516|2483791|N|N|N|N|N| +2483883|AAAAAAAALKGOFCAA|2088-07-15|2262|9838|755|2088|4|7|15|3|2088|755|9838|Thursday|2088Q3|N|N|N|2483869|2484050|2483517|2483792|N|N|N|N|N| +2483884|AAAAAAAAMKGOFCAA|2088-07-16|2262|9838|755|2088|5|7|16|3|2088|755|9838|Friday|2088Q3|N|Y|N|2483869|2484050|2483518|2483793|N|N|N|N|N| +2483885|AAAAAAAANKGOFCAA|2088-07-17|2262|9838|755|2088|6|7|17|3|2088|755|9838|Saturday|2088Q3|N|Y|N|2483869|2484050|2483519|2483794|N|N|N|N|N| +2483886|AAAAAAAAOKGOFCAA|2088-07-18|2262|9838|755|2088|0|7|18|3|2088|755|9838|Sunday|2088Q3|N|N|N|2483869|2484050|2483520|2483795|N|N|N|N|N| +2483887|AAAAAAAAPKGOFCAA|2088-07-19|2262|9838|755|2088|1|7|19|3|2088|755|9838|Monday|2088Q3|N|N|N|2483869|2484050|2483521|2483796|N|N|N|N|N| +2483888|AAAAAAAAALGOFCAA|2088-07-20|2262|9839|755|2088|2|7|20|3|2088|755|9839|Tuesday|2088Q3|N|N|N|2483869|2484050|2483522|2483797|N|N|N|N|N| +2483889|AAAAAAAABLGOFCAA|2088-07-21|2262|9839|755|2088|3|7|21|3|2088|755|9839|Wednesday|2088Q3|N|N|N|2483869|2484050|2483523|2483798|N|N|N|N|N| +2483890|AAAAAAAACLGOFCAA|2088-07-22|2262|9839|755|2088|4|7|22|3|2088|755|9839|Thursday|2088Q3|N|N|N|2483869|2484050|2483524|2483799|N|N|N|N|N| +2483891|AAAAAAAADLGOFCAA|2088-07-23|2262|9839|755|2088|5|7|23|3|2088|755|9839|Friday|2088Q3|N|Y|N|2483869|2484050|2483525|2483800|N|N|N|N|N| +2483892|AAAAAAAAELGOFCAA|2088-07-24|2262|9839|755|2088|6|7|24|3|2088|755|9839|Saturday|2088Q3|N|Y|N|2483869|2484050|2483526|2483801|N|N|N|N|N| +2483893|AAAAAAAAFLGOFCAA|2088-07-25|2262|9839|755|2088|0|7|25|3|2088|755|9839|Sunday|2088Q3|N|N|N|2483869|2484050|2483527|2483802|N|N|N|N|N| +2483894|AAAAAAAAGLGOFCAA|2088-07-26|2262|9839|755|2088|1|7|26|3|2088|755|9839|Monday|2088Q3|N|N|N|2483869|2484050|2483528|2483803|N|N|N|N|N| +2483895|AAAAAAAAHLGOFCAA|2088-07-27|2262|9840|755|2088|2|7|27|3|2088|755|9840|Tuesday|2088Q3|N|N|N|2483869|2484050|2483529|2483804|N|N|N|N|N| +2483896|AAAAAAAAILGOFCAA|2088-07-28|2262|9840|755|2088|3|7|28|3|2088|755|9840|Wednesday|2088Q3|N|N|N|2483869|2484050|2483530|2483805|N|N|N|N|N| +2483897|AAAAAAAAJLGOFCAA|2088-07-29|2262|9840|755|2088|4|7|29|3|2088|755|9840|Thursday|2088Q3|N|N|N|2483869|2484050|2483531|2483806|N|N|N|N|N| +2483898|AAAAAAAAKLGOFCAA|2088-07-30|2262|9840|755|2088|5|7|30|3|2088|755|9840|Friday|2088Q3|N|Y|N|2483869|2484050|2483532|2483807|N|N|N|N|N| +2483899|AAAAAAAALLGOFCAA|2088-07-31|2262|9840|755|2088|6|7|31|3|2088|755|9840|Saturday|2088Q3|N|Y|N|2483869|2484050|2483533|2483808|N|N|N|N|N| +2483900|AAAAAAAAMLGOFCAA|2088-08-01|2263|9840|755|2088|0|8|1|3|2088|755|9840|Sunday|2088Q3|N|N|N|2483900|2484112|2483534|2483809|N|N|N|N|N| +2483901|AAAAAAAANLGOFCAA|2088-08-02|2263|9840|755|2088|1|8|2|3|2088|755|9840|Monday|2088Q3|N|N|N|2483900|2484112|2483535|2483810|N|N|N|N|N| +2483902|AAAAAAAAOLGOFCAA|2088-08-03|2263|9841|755|2088|2|8|3|3|2088|755|9841|Tuesday|2088Q3|N|N|N|2483900|2484112|2483536|2483811|N|N|N|N|N| +2483903|AAAAAAAAPLGOFCAA|2088-08-04|2263|9841|755|2088|3|8|4|3|2088|755|9841|Wednesday|2088Q3|N|N|N|2483900|2484112|2483537|2483812|N|N|N|N|N| +2483904|AAAAAAAAAMGOFCAA|2088-08-05|2263|9841|755|2088|4|8|5|3|2088|755|9841|Thursday|2088Q3|N|N|N|2483900|2484112|2483538|2483813|N|N|N|N|N| +2483905|AAAAAAAABMGOFCAA|2088-08-06|2263|9841|755|2088|5|8|6|3|2088|755|9841|Friday|2088Q3|N|Y|N|2483900|2484112|2483539|2483814|N|N|N|N|N| +2483906|AAAAAAAACMGOFCAA|2088-08-07|2263|9841|755|2088|6|8|7|3|2088|755|9841|Saturday|2088Q3|N|Y|N|2483900|2484112|2483540|2483815|N|N|N|N|N| +2483907|AAAAAAAADMGOFCAA|2088-08-08|2263|9841|755|2088|0|8|8|3|2088|755|9841|Sunday|2088Q3|N|N|N|2483900|2484112|2483541|2483816|N|N|N|N|N| +2483908|AAAAAAAAEMGOFCAA|2088-08-09|2263|9841|755|2088|1|8|9|3|2088|755|9841|Monday|2088Q3|N|N|N|2483900|2484112|2483542|2483817|N|N|N|N|N| +2483909|AAAAAAAAFMGOFCAA|2088-08-10|2263|9842|755|2088|2|8|10|3|2088|755|9842|Tuesday|2088Q3|N|N|N|2483900|2484112|2483543|2483818|N|N|N|N|N| +2483910|AAAAAAAAGMGOFCAA|2088-08-11|2263|9842|755|2088|3|8|11|3|2088|755|9842|Wednesday|2088Q3|N|N|N|2483900|2484112|2483544|2483819|N|N|N|N|N| +2483911|AAAAAAAAHMGOFCAA|2088-08-12|2263|9842|755|2088|4|8|12|3|2088|755|9842|Thursday|2088Q3|N|N|N|2483900|2484112|2483545|2483820|N|N|N|N|N| +2483912|AAAAAAAAIMGOFCAA|2088-08-13|2263|9842|755|2088|5|8|13|3|2088|755|9842|Friday|2088Q3|N|Y|N|2483900|2484112|2483546|2483821|N|N|N|N|N| +2483913|AAAAAAAAJMGOFCAA|2088-08-14|2263|9842|755|2088|6|8|14|3|2088|755|9842|Saturday|2088Q3|N|Y|N|2483900|2484112|2483547|2483822|N|N|N|N|N| +2483914|AAAAAAAAKMGOFCAA|2088-08-15|2263|9842|755|2088|0|8|15|3|2088|755|9842|Sunday|2088Q3|N|N|N|2483900|2484112|2483548|2483823|N|N|N|N|N| +2483915|AAAAAAAALMGOFCAA|2088-08-16|2263|9842|755|2088|1|8|16|3|2088|755|9842|Monday|2088Q3|N|N|N|2483900|2484112|2483549|2483824|N|N|N|N|N| +2483916|AAAAAAAAMMGOFCAA|2088-08-17|2263|9843|755|2088|2|8|17|3|2088|755|9843|Tuesday|2088Q3|N|N|N|2483900|2484112|2483550|2483825|N|N|N|N|N| +2483917|AAAAAAAANMGOFCAA|2088-08-18|2263|9843|755|2088|3|8|18|3|2088|755|9843|Wednesday|2088Q3|N|N|N|2483900|2484112|2483551|2483826|N|N|N|N|N| +2483918|AAAAAAAAOMGOFCAA|2088-08-19|2263|9843|755|2088|4|8|19|3|2088|755|9843|Thursday|2088Q3|N|N|N|2483900|2484112|2483552|2483827|N|N|N|N|N| +2483919|AAAAAAAAPMGOFCAA|2088-08-20|2263|9843|755|2088|5|8|20|3|2088|755|9843|Friday|2088Q3|N|Y|N|2483900|2484112|2483553|2483828|N|N|N|N|N| +2483920|AAAAAAAAANGOFCAA|2088-08-21|2263|9843|755|2088|6|8|21|3|2088|755|9843|Saturday|2088Q3|N|Y|N|2483900|2484112|2483554|2483829|N|N|N|N|N| +2483921|AAAAAAAABNGOFCAA|2088-08-22|2263|9843|755|2088|0|8|22|3|2088|755|9843|Sunday|2088Q3|N|N|N|2483900|2484112|2483555|2483830|N|N|N|N|N| +2483922|AAAAAAAACNGOFCAA|2088-08-23|2263|9843|755|2088|1|8|23|3|2088|755|9843|Monday|2088Q3|N|N|N|2483900|2484112|2483556|2483831|N|N|N|N|N| +2483923|AAAAAAAADNGOFCAA|2088-08-24|2263|9844|755|2088|2|8|24|3|2088|755|9844|Tuesday|2088Q3|N|N|N|2483900|2484112|2483557|2483832|N|N|N|N|N| +2483924|AAAAAAAAENGOFCAA|2088-08-25|2263|9844|755|2088|3|8|25|3|2088|755|9844|Wednesday|2088Q3|N|N|N|2483900|2484112|2483558|2483833|N|N|N|N|N| +2483925|AAAAAAAAFNGOFCAA|2088-08-26|2263|9844|755|2088|4|8|26|3|2088|755|9844|Thursday|2088Q3|N|N|N|2483900|2484112|2483559|2483834|N|N|N|N|N| +2483926|AAAAAAAAGNGOFCAA|2088-08-27|2263|9844|755|2088|5|8|27|3|2088|755|9844|Friday|2088Q3|N|Y|N|2483900|2484112|2483560|2483835|N|N|N|N|N| +2483927|AAAAAAAAHNGOFCAA|2088-08-28|2263|9844|755|2088|6|8|28|3|2088|755|9844|Saturday|2088Q3|N|Y|N|2483900|2484112|2483561|2483836|N|N|N|N|N| +2483928|AAAAAAAAINGOFCAA|2088-08-29|2263|9844|755|2088|0|8|29|3|2088|755|9844|Sunday|2088Q3|N|N|N|2483900|2484112|2483562|2483837|N|N|N|N|N| +2483929|AAAAAAAAJNGOFCAA|2088-08-30|2263|9844|755|2088|1|8|30|3|2088|755|9844|Monday|2088Q3|N|N|N|2483900|2484112|2483563|2483838|N|N|N|N|N| +2483930|AAAAAAAAKNGOFCAA|2088-08-31|2263|9845|755|2088|2|8|31|3|2088|755|9845|Tuesday|2088Q3|N|N|N|2483900|2484112|2483564|2483839|N|N|N|N|N| +2483931|AAAAAAAALNGOFCAA|2088-09-01|2264|9845|756|2088|3|9|1|3|2088|756|9845|Wednesday|2088Q3|N|N|N|2483931|2484174|2483565|2483840|N|N|N|N|N| +2483932|AAAAAAAAMNGOFCAA|2088-09-02|2264|9845|756|2088|4|9|2|3|2088|756|9845|Thursday|2088Q3|N|N|N|2483931|2484174|2483566|2483841|N|N|N|N|N| +2483933|AAAAAAAANNGOFCAA|2088-09-03|2264|9845|756|2088|5|9|3|3|2088|756|9845|Friday|2088Q3|N|Y|N|2483931|2484174|2483567|2483842|N|N|N|N|N| +2483934|AAAAAAAAONGOFCAA|2088-09-04|2264|9845|756|2088|6|9|4|3|2088|756|9845|Saturday|2088Q3|N|Y|N|2483931|2484174|2483568|2483843|N|N|N|N|N| +2483935|AAAAAAAAPNGOFCAA|2088-09-05|2264|9845|756|2088|0|9|5|3|2088|756|9845|Sunday|2088Q3|N|N|N|2483931|2484174|2483569|2483844|N|N|N|N|N| +2483936|AAAAAAAAAOGOFCAA|2088-09-06|2264|9845|756|2088|1|9|6|3|2088|756|9845|Monday|2088Q3|N|N|N|2483931|2484174|2483570|2483845|N|N|N|N|N| +2483937|AAAAAAAABOGOFCAA|2088-09-07|2264|9846|756|2088|2|9|7|3|2088|756|9846|Tuesday|2088Q3|N|N|N|2483931|2484174|2483571|2483846|N|N|N|N|N| +2483938|AAAAAAAACOGOFCAA|2088-09-08|2264|9846|756|2088|3|9|8|3|2088|756|9846|Wednesday|2088Q3|N|N|N|2483931|2484174|2483572|2483847|N|N|N|N|N| +2483939|AAAAAAAADOGOFCAA|2088-09-09|2264|9846|756|2088|4|9|9|3|2088|756|9846|Thursday|2088Q3|N|N|N|2483931|2484174|2483573|2483848|N|N|N|N|N| +2483940|AAAAAAAAEOGOFCAA|2088-09-10|2264|9846|756|2088|5|9|10|3|2088|756|9846|Friday|2088Q3|N|Y|N|2483931|2484174|2483574|2483849|N|N|N|N|N| +2483941|AAAAAAAAFOGOFCAA|2088-09-11|2264|9846|756|2088|6|9|11|3|2088|756|9846|Saturday|2088Q3|N|Y|N|2483931|2484174|2483575|2483850|N|N|N|N|N| +2483942|AAAAAAAAGOGOFCAA|2088-09-12|2264|9846|756|2088|0|9|12|3|2088|756|9846|Sunday|2088Q3|N|N|N|2483931|2484174|2483576|2483851|N|N|N|N|N| +2483943|AAAAAAAAHOGOFCAA|2088-09-13|2264|9846|756|2088|1|9|13|3|2088|756|9846|Monday|2088Q3|N|N|N|2483931|2484174|2483577|2483852|N|N|N|N|N| +2483944|AAAAAAAAIOGOFCAA|2088-09-14|2264|9847|756|2088|2|9|14|3|2088|756|9847|Tuesday|2088Q3|N|N|N|2483931|2484174|2483578|2483853|N|N|N|N|N| +2483945|AAAAAAAAJOGOFCAA|2088-09-15|2264|9847|756|2088|3|9|15|3|2088|756|9847|Wednesday|2088Q3|N|N|N|2483931|2484174|2483579|2483854|N|N|N|N|N| +2483946|AAAAAAAAKOGOFCAA|2088-09-16|2264|9847|756|2088|4|9|16|3|2088|756|9847|Thursday|2088Q3|N|N|N|2483931|2484174|2483580|2483855|N|N|N|N|N| +2483947|AAAAAAAALOGOFCAA|2088-09-17|2264|9847|756|2088|5|9|17|3|2088|756|9847|Friday|2088Q3|N|Y|N|2483931|2484174|2483581|2483856|N|N|N|N|N| +2483948|AAAAAAAAMOGOFCAA|2088-09-18|2264|9847|756|2088|6|9|18|3|2088|756|9847|Saturday|2088Q3|N|Y|N|2483931|2484174|2483582|2483857|N|N|N|N|N| +2483949|AAAAAAAANOGOFCAA|2088-09-19|2264|9847|756|2088|0|9|19|3|2088|756|9847|Sunday|2088Q3|N|N|N|2483931|2484174|2483583|2483858|N|N|N|N|N| +2483950|AAAAAAAAOOGOFCAA|2088-09-20|2264|9847|756|2088|1|9|20|3|2088|756|9847|Monday|2088Q3|N|N|N|2483931|2484174|2483584|2483859|N|N|N|N|N| +2483951|AAAAAAAAPOGOFCAA|2088-09-21|2264|9848|756|2088|2|9|21|3|2088|756|9848|Tuesday|2088Q3|N|N|N|2483931|2484174|2483585|2483860|N|N|N|N|N| +2483952|AAAAAAAAAPGOFCAA|2088-09-22|2264|9848|756|2088|3|9|22|3|2088|756|9848|Wednesday|2088Q3|N|N|N|2483931|2484174|2483586|2483861|N|N|N|N|N| +2483953|AAAAAAAABPGOFCAA|2088-09-23|2264|9848|756|2088|4|9|23|3|2088|756|9848|Thursday|2088Q3|N|N|N|2483931|2484174|2483587|2483862|N|N|N|N|N| +2483954|AAAAAAAACPGOFCAA|2088-09-24|2264|9848|756|2088|5|9|24|3|2088|756|9848|Friday|2088Q3|N|Y|N|2483931|2484174|2483588|2483863|N|N|N|N|N| +2483955|AAAAAAAADPGOFCAA|2088-09-25|2264|9848|756|2088|6|9|25|3|2088|756|9848|Saturday|2088Q3|N|Y|N|2483931|2484174|2483589|2483864|N|N|N|N|N| +2483956|AAAAAAAAEPGOFCAA|2088-09-26|2264|9848|756|2088|0|9|26|3|2088|756|9848|Sunday|2088Q3|N|N|N|2483931|2484174|2483590|2483865|N|N|N|N|N| +2483957|AAAAAAAAFPGOFCAA|2088-09-27|2264|9848|756|2088|1|9|27|3|2088|756|9848|Monday|2088Q3|N|N|N|2483931|2484174|2483591|2483866|N|N|N|N|N| +2483958|AAAAAAAAGPGOFCAA|2088-09-28|2264|9849|756|2088|2|9|28|3|2088|756|9849|Tuesday|2088Q3|N|N|N|2483931|2484174|2483592|2483867|N|N|N|N|N| +2483959|AAAAAAAAHPGOFCAA|2088-09-29|2264|9849|756|2088|3|9|29|3|2088|756|9849|Wednesday|2088Q3|N|N|N|2483931|2484174|2483593|2483868|N|N|N|N|N| +2483960|AAAAAAAAIPGOFCAA|2088-09-30|2264|9849|756|2088|4|9|30|3|2088|756|9849|Thursday|2088Q3|N|N|N|2483931|2484174|2483594|2483869|N|N|N|N|N| +2483961|AAAAAAAAJPGOFCAA|2088-10-01|2265|9849|756|2088|5|10|1|4|2088|756|9849|Friday|2088Q4|N|Y|N|2483961|2484234|2483595|2483869|N|N|N|N|N| +2483962|AAAAAAAAKPGOFCAA|2088-10-02|2265|9849|756|2088|6|10|2|4|2088|756|9849|Saturday|2088Q4|N|Y|N|2483961|2484234|2483596|2483870|N|N|N|N|N| +2483963|AAAAAAAALPGOFCAA|2088-10-03|2265|9849|756|2088|0|10|3|4|2088|756|9849|Sunday|2088Q4|N|N|N|2483961|2484234|2483597|2483871|N|N|N|N|N| +2483964|AAAAAAAAMPGOFCAA|2088-10-04|2265|9849|756|2088|1|10|4|4|2088|756|9849|Monday|2088Q4|N|N|N|2483961|2484234|2483598|2483872|N|N|N|N|N| +2483965|AAAAAAAANPGOFCAA|2088-10-05|2265|9850|756|2088|2|10|5|4|2088|756|9850|Tuesday|2088Q4|N|N|N|2483961|2484234|2483599|2483873|N|N|N|N|N| +2483966|AAAAAAAAOPGOFCAA|2088-10-06|2265|9850|756|2088|3|10|6|4|2088|756|9850|Wednesday|2088Q4|N|N|N|2483961|2484234|2483600|2483874|N|N|N|N|N| +2483967|AAAAAAAAPPGOFCAA|2088-10-07|2265|9850|756|2088|4|10|7|4|2088|756|9850|Thursday|2088Q4|N|N|N|2483961|2484234|2483601|2483875|N|N|N|N|N| +2483968|AAAAAAAAAAHOFCAA|2088-10-08|2265|9850|756|2088|5|10|8|4|2088|756|9850|Friday|2088Q4|N|Y|N|2483961|2484234|2483602|2483876|N|N|N|N|N| +2483969|AAAAAAAABAHOFCAA|2088-10-09|2265|9850|756|2088|6|10|9|4|2088|756|9850|Saturday|2088Q4|N|Y|N|2483961|2484234|2483603|2483877|N|N|N|N|N| +2483970|AAAAAAAACAHOFCAA|2088-10-10|2265|9850|756|2088|0|10|10|4|2088|756|9850|Sunday|2088Q4|N|N|N|2483961|2484234|2483604|2483878|N|N|N|N|N| +2483971|AAAAAAAADAHOFCAA|2088-10-11|2265|9850|756|2088|1|10|11|4|2088|756|9850|Monday|2088Q4|N|N|N|2483961|2484234|2483605|2483879|N|N|N|N|N| +2483972|AAAAAAAAEAHOFCAA|2088-10-12|2265|9851|756|2088|2|10|12|4|2088|756|9851|Tuesday|2088Q4|N|N|N|2483961|2484234|2483606|2483880|N|N|N|N|N| +2483973|AAAAAAAAFAHOFCAA|2088-10-13|2265|9851|756|2088|3|10|13|4|2088|756|9851|Wednesday|2088Q4|N|N|N|2483961|2484234|2483607|2483881|N|N|N|N|N| +2483974|AAAAAAAAGAHOFCAA|2088-10-14|2265|9851|756|2088|4|10|14|4|2088|756|9851|Thursday|2088Q4|N|N|N|2483961|2484234|2483608|2483882|N|N|N|N|N| +2483975|AAAAAAAAHAHOFCAA|2088-10-15|2265|9851|756|2088|5|10|15|4|2088|756|9851|Friday|2088Q4|N|Y|N|2483961|2484234|2483609|2483883|N|N|N|N|N| +2483976|AAAAAAAAIAHOFCAA|2088-10-16|2265|9851|756|2088|6|10|16|4|2088|756|9851|Saturday|2088Q4|N|Y|N|2483961|2484234|2483610|2483884|N|N|N|N|N| +2483977|AAAAAAAAJAHOFCAA|2088-10-17|2265|9851|756|2088|0|10|17|4|2088|756|9851|Sunday|2088Q4|N|N|N|2483961|2484234|2483611|2483885|N|N|N|N|N| +2483978|AAAAAAAAKAHOFCAA|2088-10-18|2265|9851|756|2088|1|10|18|4|2088|756|9851|Monday|2088Q4|N|N|N|2483961|2484234|2483612|2483886|N|N|N|N|N| +2483979|AAAAAAAALAHOFCAA|2088-10-19|2265|9852|756|2088|2|10|19|4|2088|756|9852|Tuesday|2088Q4|N|N|N|2483961|2484234|2483613|2483887|N|N|N|N|N| +2483980|AAAAAAAAMAHOFCAA|2088-10-20|2265|9852|756|2088|3|10|20|4|2088|756|9852|Wednesday|2088Q4|N|N|N|2483961|2484234|2483614|2483888|N|N|N|N|N| +2483981|AAAAAAAANAHOFCAA|2088-10-21|2265|9852|756|2088|4|10|21|4|2088|756|9852|Thursday|2088Q4|N|N|N|2483961|2484234|2483615|2483889|N|N|N|N|N| +2483982|AAAAAAAAOAHOFCAA|2088-10-22|2265|9852|756|2088|5|10|22|4|2088|756|9852|Friday|2088Q4|N|Y|N|2483961|2484234|2483616|2483890|N|N|N|N|N| +2483983|AAAAAAAAPAHOFCAA|2088-10-23|2265|9852|756|2088|6|10|23|4|2088|756|9852|Saturday|2088Q4|N|Y|N|2483961|2484234|2483617|2483891|N|N|N|N|N| +2483984|AAAAAAAAABHOFCAA|2088-10-24|2265|9852|756|2088|0|10|24|4|2088|756|9852|Sunday|2088Q4|N|N|N|2483961|2484234|2483618|2483892|N|N|N|N|N| +2483985|AAAAAAAABBHOFCAA|2088-10-25|2265|9852|756|2088|1|10|25|4|2088|756|9852|Monday|2088Q4|N|N|N|2483961|2484234|2483619|2483893|N|N|N|N|N| +2483986|AAAAAAAACBHOFCAA|2088-10-26|2265|9853|756|2088|2|10|26|4|2088|756|9853|Tuesday|2088Q4|N|N|N|2483961|2484234|2483620|2483894|N|N|N|N|N| +2483987|AAAAAAAADBHOFCAA|2088-10-27|2265|9853|756|2088|3|10|27|4|2088|756|9853|Wednesday|2088Q4|N|N|N|2483961|2484234|2483621|2483895|N|N|N|N|N| +2483988|AAAAAAAAEBHOFCAA|2088-10-28|2265|9853|756|2088|4|10|28|4|2088|756|9853|Thursday|2088Q4|N|N|N|2483961|2484234|2483622|2483896|N|N|N|N|N| +2483989|AAAAAAAAFBHOFCAA|2088-10-29|2265|9853|756|2088|5|10|29|4|2088|756|9853|Friday|2088Q4|N|Y|N|2483961|2484234|2483623|2483897|N|N|N|N|N| +2483990|AAAAAAAAGBHOFCAA|2088-10-30|2265|9853|756|2088|6|10|30|4|2088|756|9853|Saturday|2088Q4|N|Y|N|2483961|2484234|2483624|2483898|N|N|N|N|N| +2483991|AAAAAAAAHBHOFCAA|2088-10-31|2265|9853|756|2088|0|10|31|4|2088|756|9853|Sunday|2088Q4|N|N|N|2483961|2484234|2483625|2483899|N|N|N|N|N| +2483992|AAAAAAAAIBHOFCAA|2088-11-01|2266|9853|756|2088|1|11|1|4|2088|756|9853|Monday|2088Q4|N|N|N|2483992|2484296|2483626|2483900|N|N|N|N|N| +2483993|AAAAAAAAJBHOFCAA|2088-11-02|2266|9854|756|2088|2|11|2|4|2088|756|9854|Tuesday|2088Q4|N|N|N|2483992|2484296|2483627|2483901|N|N|N|N|N| +2483994|AAAAAAAAKBHOFCAA|2088-11-03|2266|9854|756|2088|3|11|3|4|2088|756|9854|Wednesday|2088Q4|N|N|N|2483992|2484296|2483628|2483902|N|N|N|N|N| +2483995|AAAAAAAALBHOFCAA|2088-11-04|2266|9854|756|2088|4|11|4|4|2088|756|9854|Thursday|2088Q4|N|N|N|2483992|2484296|2483629|2483903|N|N|N|N|N| +2483996|AAAAAAAAMBHOFCAA|2088-11-05|2266|9854|756|2088|5|11|5|4|2088|756|9854|Friday|2088Q4|N|Y|N|2483992|2484296|2483630|2483904|N|N|N|N|N| +2483997|AAAAAAAANBHOFCAA|2088-11-06|2266|9854|756|2088|6|11|6|4|2088|756|9854|Saturday|2088Q4|N|Y|N|2483992|2484296|2483631|2483905|N|N|N|N|N| +2483998|AAAAAAAAOBHOFCAA|2088-11-07|2266|9854|756|2088|0|11|7|4|2088|756|9854|Sunday|2088Q4|N|N|N|2483992|2484296|2483632|2483906|N|N|N|N|N| +2483999|AAAAAAAAPBHOFCAA|2088-11-08|2266|9854|756|2088|1|11|8|4|2088|756|9854|Monday|2088Q4|N|N|N|2483992|2484296|2483633|2483907|N|N|N|N|N| +2484000|AAAAAAAAACHOFCAA|2088-11-09|2266|9855|756|2088|2|11|9|4|2088|756|9855|Tuesday|2088Q4|N|N|N|2483992|2484296|2483634|2483908|N|N|N|N|N| +2484001|AAAAAAAABCHOFCAA|2088-11-10|2266|9855|756|2088|3|11|10|4|2088|756|9855|Wednesday|2088Q4|N|N|N|2483992|2484296|2483635|2483909|N|N|N|N|N| +2484002|AAAAAAAACCHOFCAA|2088-11-11|2266|9855|756|2088|4|11|11|4|2088|756|9855|Thursday|2088Q4|N|N|N|2483992|2484296|2483636|2483910|N|N|N|N|N| +2484003|AAAAAAAADCHOFCAA|2088-11-12|2266|9855|756|2088|5|11|12|4|2088|756|9855|Friday|2088Q4|N|Y|N|2483992|2484296|2483637|2483911|N|N|N|N|N| +2484004|AAAAAAAAECHOFCAA|2088-11-13|2266|9855|756|2088|6|11|13|4|2088|756|9855|Saturday|2088Q4|N|Y|N|2483992|2484296|2483638|2483912|N|N|N|N|N| +2484005|AAAAAAAAFCHOFCAA|2088-11-14|2266|9855|756|2088|0|11|14|4|2088|756|9855|Sunday|2088Q4|N|N|N|2483992|2484296|2483639|2483913|N|N|N|N|N| +2484006|AAAAAAAAGCHOFCAA|2088-11-15|2266|9855|756|2088|1|11|15|4|2088|756|9855|Monday|2088Q4|N|N|N|2483992|2484296|2483640|2483914|N|N|N|N|N| +2484007|AAAAAAAAHCHOFCAA|2088-11-16|2266|9856|756|2088|2|11|16|4|2088|756|9856|Tuesday|2088Q4|N|N|N|2483992|2484296|2483641|2483915|N|N|N|N|N| +2484008|AAAAAAAAICHOFCAA|2088-11-17|2266|9856|756|2088|3|11|17|4|2088|756|9856|Wednesday|2088Q4|N|N|N|2483992|2484296|2483642|2483916|N|N|N|N|N| +2484009|AAAAAAAAJCHOFCAA|2088-11-18|2266|9856|756|2088|4|11|18|4|2088|756|9856|Thursday|2088Q4|N|N|N|2483992|2484296|2483643|2483917|N|N|N|N|N| +2484010|AAAAAAAAKCHOFCAA|2088-11-19|2266|9856|756|2088|5|11|19|4|2088|756|9856|Friday|2088Q4|N|Y|N|2483992|2484296|2483644|2483918|N|N|N|N|N| +2484011|AAAAAAAALCHOFCAA|2088-11-20|2266|9856|756|2088|6|11|20|4|2088|756|9856|Saturday|2088Q4|N|Y|N|2483992|2484296|2483645|2483919|N|N|N|N|N| +2484012|AAAAAAAAMCHOFCAA|2088-11-21|2266|9856|756|2088|0|11|21|4|2088|756|9856|Sunday|2088Q4|N|N|N|2483992|2484296|2483646|2483920|N|N|N|N|N| +2484013|AAAAAAAANCHOFCAA|2088-11-22|2266|9856|756|2088|1|11|22|4|2088|756|9856|Monday|2088Q4|N|N|N|2483992|2484296|2483647|2483921|N|N|N|N|N| +2484014|AAAAAAAAOCHOFCAA|2088-11-23|2266|9857|756|2088|2|11|23|4|2088|756|9857|Tuesday|2088Q4|N|N|N|2483992|2484296|2483648|2483922|N|N|N|N|N| +2484015|AAAAAAAAPCHOFCAA|2088-11-24|2266|9857|756|2088|3|11|24|4|2088|756|9857|Wednesday|2088Q4|N|N|N|2483992|2484296|2483649|2483923|N|N|N|N|N| +2484016|AAAAAAAAADHOFCAA|2088-11-25|2266|9857|756|2088|4|11|25|4|2088|756|9857|Thursday|2088Q4|N|N|N|2483992|2484296|2483650|2483924|N|N|N|N|N| +2484017|AAAAAAAABDHOFCAA|2088-11-26|2266|9857|756|2088|5|11|26|4|2088|756|9857|Friday|2088Q4|N|Y|N|2483992|2484296|2483651|2483925|N|N|N|N|N| +2484018|AAAAAAAACDHOFCAA|2088-11-27|2266|9857|756|2088|6|11|27|4|2088|756|9857|Saturday|2088Q4|N|Y|N|2483992|2484296|2483652|2483926|N|N|N|N|N| +2484019|AAAAAAAADDHOFCAA|2088-11-28|2266|9857|756|2088|0|11|28|4|2088|756|9857|Sunday|2088Q4|N|N|N|2483992|2484296|2483653|2483927|N|N|N|N|N| +2484020|AAAAAAAAEDHOFCAA|2088-11-29|2266|9857|756|2088|1|11|29|4|2088|756|9857|Monday|2088Q4|N|N|N|2483992|2484296|2483654|2483928|N|N|N|N|N| +2484021|AAAAAAAAFDHOFCAA|2088-11-30|2266|9858|756|2088|2|11|30|4|2088|756|9858|Tuesday|2088Q4|N|N|N|2483992|2484296|2483655|2483929|N|N|N|N|N| +2484022|AAAAAAAAGDHOFCAA|2088-12-01|2267|9858|757|2088|3|12|1|4|2088|757|9858|Wednesday|2088Q4|N|N|N|2484022|2484356|2483656|2483930|N|N|N|N|N| +2484023|AAAAAAAAHDHOFCAA|2088-12-02|2267|9858|757|2088|4|12|2|4|2088|757|9858|Thursday|2088Q4|N|N|N|2484022|2484356|2483657|2483931|N|N|N|N|N| +2484024|AAAAAAAAIDHOFCAA|2088-12-03|2267|9858|757|2088|5|12|3|4|2088|757|9858|Friday|2088Q4|N|Y|N|2484022|2484356|2483658|2483932|N|N|N|N|N| +2484025|AAAAAAAAJDHOFCAA|2088-12-04|2267|9858|757|2088|6|12|4|4|2088|757|9858|Saturday|2088Q4|N|Y|N|2484022|2484356|2483659|2483933|N|N|N|N|N| +2484026|AAAAAAAAKDHOFCAA|2088-12-05|2267|9858|757|2088|0|12|5|4|2088|757|9858|Sunday|2088Q4|N|N|N|2484022|2484356|2483660|2483934|N|N|N|N|N| +2484027|AAAAAAAALDHOFCAA|2088-12-06|2267|9858|757|2088|1|12|6|4|2088|757|9858|Monday|2088Q4|N|N|N|2484022|2484356|2483661|2483935|N|N|N|N|N| +2484028|AAAAAAAAMDHOFCAA|2088-12-07|2267|9859|757|2088|2|12|7|4|2088|757|9859|Tuesday|2088Q4|N|N|N|2484022|2484356|2483662|2483936|N|N|N|N|N| +2484029|AAAAAAAANDHOFCAA|2088-12-08|2267|9859|757|2088|3|12|8|4|2088|757|9859|Wednesday|2088Q4|N|N|N|2484022|2484356|2483663|2483937|N|N|N|N|N| +2484030|AAAAAAAAODHOFCAA|2088-12-09|2267|9859|757|2088|4|12|9|4|2088|757|9859|Thursday|2088Q4|N|N|N|2484022|2484356|2483664|2483938|N|N|N|N|N| +2484031|AAAAAAAAPDHOFCAA|2088-12-10|2267|9859|757|2088|5|12|10|4|2088|757|9859|Friday|2088Q4|N|Y|N|2484022|2484356|2483665|2483939|N|N|N|N|N| +2484032|AAAAAAAAAEHOFCAA|2088-12-11|2267|9859|757|2088|6|12|11|4|2088|757|9859|Saturday|2088Q4|N|Y|N|2484022|2484356|2483666|2483940|N|N|N|N|N| +2484033|AAAAAAAABEHOFCAA|2088-12-12|2267|9859|757|2088|0|12|12|4|2088|757|9859|Sunday|2088Q4|N|N|N|2484022|2484356|2483667|2483941|N|N|N|N|N| +2484034|AAAAAAAACEHOFCAA|2088-12-13|2267|9859|757|2088|1|12|13|4|2088|757|9859|Monday|2088Q4|N|N|N|2484022|2484356|2483668|2483942|N|N|N|N|N| +2484035|AAAAAAAADEHOFCAA|2088-12-14|2267|9860|757|2088|2|12|14|4|2088|757|9860|Tuesday|2088Q4|N|N|N|2484022|2484356|2483669|2483943|N|N|N|N|N| +2484036|AAAAAAAAEEHOFCAA|2088-12-15|2267|9860|757|2088|3|12|15|4|2088|757|9860|Wednesday|2088Q4|N|N|N|2484022|2484356|2483670|2483944|N|N|N|N|N| +2484037|AAAAAAAAFEHOFCAA|2088-12-16|2267|9860|757|2088|4|12|16|4|2088|757|9860|Thursday|2088Q4|N|N|N|2484022|2484356|2483671|2483945|N|N|N|N|N| +2484038|AAAAAAAAGEHOFCAA|2088-12-17|2267|9860|757|2088|5|12|17|4|2088|757|9860|Friday|2088Q4|N|Y|N|2484022|2484356|2483672|2483946|N|N|N|N|N| +2484039|AAAAAAAAHEHOFCAA|2088-12-18|2267|9860|757|2088|6|12|18|4|2088|757|9860|Saturday|2088Q4|N|Y|N|2484022|2484356|2483673|2483947|N|N|N|N|N| +2484040|AAAAAAAAIEHOFCAA|2088-12-19|2267|9860|757|2088|0|12|19|4|2088|757|9860|Sunday|2088Q4|N|N|N|2484022|2484356|2483674|2483948|N|N|N|N|N| +2484041|AAAAAAAAJEHOFCAA|2088-12-20|2267|9860|757|2088|1|12|20|4|2088|757|9860|Monday|2088Q4|N|N|N|2484022|2484356|2483675|2483949|N|N|N|N|N| +2484042|AAAAAAAAKEHOFCAA|2088-12-21|2267|9861|757|2088|2|12|21|4|2088|757|9861|Tuesday|2088Q4|N|N|N|2484022|2484356|2483676|2483950|N|N|N|N|N| +2484043|AAAAAAAALEHOFCAA|2088-12-22|2267|9861|757|2088|3|12|22|4|2088|757|9861|Wednesday|2088Q4|N|N|N|2484022|2484356|2483677|2483951|N|N|N|N|N| +2484044|AAAAAAAAMEHOFCAA|2088-12-23|2267|9861|757|2088|4|12|23|4|2088|757|9861|Thursday|2088Q4|N|N|N|2484022|2484356|2483678|2483952|N|N|N|N|N| +2484045|AAAAAAAANEHOFCAA|2088-12-24|2267|9861|757|2088|5|12|24|4|2088|757|9861|Friday|2088Q4|N|Y|N|2484022|2484356|2483679|2483953|N|N|N|N|N| +2484046|AAAAAAAAOEHOFCAA|2088-12-25|2267|9861|757|2088|6|12|25|4|2088|757|9861|Saturday|2088Q4|Y|Y|N|2484022|2484356|2483680|2483954|N|N|N|N|N| +2484047|AAAAAAAAPEHOFCAA|2088-12-26|2267|9861|757|2088|0|12|26|4|2088|757|9861|Sunday|2088Q4|N|N|Y|2484022|2484356|2483681|2483955|N|N|N|N|N| +2484048|AAAAAAAAAFHOFCAA|2088-12-27|2267|9861|757|2088|1|12|27|4|2088|757|9861|Monday|2088Q4|N|N|N|2484022|2484356|2483682|2483956|N|N|N|N|N| +2484049|AAAAAAAABFHOFCAA|2088-12-28|2267|9862|757|2088|2|12|28|4|2088|757|9862|Tuesday|2088Q4|N|N|N|2484022|2484356|2483683|2483957|N|N|N|N|N| +2484050|AAAAAAAACFHOFCAA|2088-12-29|2267|9862|757|2088|3|12|29|4|2088|757|9862|Wednesday|2088Q4|N|N|N|2484022|2484356|2483684|2483958|N|N|N|N|N| +2484051|AAAAAAAADFHOFCAA|2088-12-30|2267|9862|757|2088|4|12|30|4|2088|757|9862|Thursday|2088Q4|N|N|N|2484022|2484356|2483685|2483959|N|N|N|N|N| +2484052|AAAAAAAAEFHOFCAA|2088-12-31|2267|9862|757|2088|5|12|31|4|2088|757|9862|Friday|2088Q4|Y|Y|N|2484022|2484356|2483686|2483960|N|N|N|N|N| +2484053|AAAAAAAAFFHOFCAA|2089-01-01|2268|9862|757|2089|6|1|1|1|2089|757|9862|Saturday|2089Q1|Y|Y|Y|2484053|2484052|2483687|2483961|N|N|N|N|N| +2484054|AAAAAAAAGFHOFCAA|2089-01-02|2268|9862|757|2089|0|1|2|1|2089|757|9862|Sunday|2089Q1|N|N|Y|2484053|2484052|2483688|2483962|N|N|N|N|N| +2484055|AAAAAAAAHFHOFCAA|2089-01-03|2268|9862|757|2089|1|1|3|1|2089|757|9862|Monday|2089Q1|N|N|N|2484053|2484052|2483689|2483963|N|N|N|N|N| +2484056|AAAAAAAAIFHOFCAA|2089-01-04|2268|9863|757|2089|2|1|4|1|2089|757|9863|Tuesday|2089Q1|N|N|N|2484053|2484052|2483690|2483964|N|N|N|N|N| +2484057|AAAAAAAAJFHOFCAA|2089-01-05|2268|9863|757|2089|3|1|5|1|2089|757|9863|Wednesday|2089Q1|N|N|N|2484053|2484052|2483691|2483965|N|N|N|N|N| +2484058|AAAAAAAAKFHOFCAA|2089-01-06|2268|9863|757|2089|4|1|6|1|2089|757|9863|Thursday|2089Q1|N|N|N|2484053|2484052|2483692|2483966|N|N|N|N|N| +2484059|AAAAAAAALFHOFCAA|2089-01-07|2268|9863|757|2089|5|1|7|1|2089|757|9863|Friday|2089Q1|N|Y|N|2484053|2484052|2483693|2483967|N|N|N|N|N| +2484060|AAAAAAAAMFHOFCAA|2089-01-08|2268|9863|757|2089|6|1|8|1|2089|757|9863|Saturday|2089Q1|N|Y|N|2484053|2484052|2483694|2483968|N|N|N|N|N| +2484061|AAAAAAAANFHOFCAA|2089-01-09|2268|9863|757|2089|0|1|9|1|2089|757|9863|Sunday|2089Q1|N|N|N|2484053|2484052|2483695|2483969|N|N|N|N|N| +2484062|AAAAAAAAOFHOFCAA|2089-01-10|2268|9863|757|2089|1|1|10|1|2089|757|9863|Monday|2089Q1|N|N|N|2484053|2484052|2483696|2483970|N|N|N|N|N| +2484063|AAAAAAAAPFHOFCAA|2089-01-11|2268|9864|757|2089|2|1|11|1|2089|757|9864|Tuesday|2089Q1|N|N|N|2484053|2484052|2483697|2483971|N|N|N|N|N| +2484064|AAAAAAAAAGHOFCAA|2089-01-12|2268|9864|757|2089|3|1|12|1|2089|757|9864|Wednesday|2089Q1|N|N|N|2484053|2484052|2483698|2483972|N|N|N|N|N| +2484065|AAAAAAAABGHOFCAA|2089-01-13|2268|9864|757|2089|4|1|13|1|2089|757|9864|Thursday|2089Q1|N|N|N|2484053|2484052|2483699|2483973|N|N|N|N|N| +2484066|AAAAAAAACGHOFCAA|2089-01-14|2268|9864|757|2089|5|1|14|1|2089|757|9864|Friday|2089Q1|N|Y|N|2484053|2484052|2483700|2483974|N|N|N|N|N| +2484067|AAAAAAAADGHOFCAA|2089-01-15|2268|9864|757|2089|6|1|15|1|2089|757|9864|Saturday|2089Q1|N|Y|N|2484053|2484052|2483701|2483975|N|N|N|N|N| +2484068|AAAAAAAAEGHOFCAA|2089-01-16|2268|9864|757|2089|0|1|16|1|2089|757|9864|Sunday|2089Q1|N|N|N|2484053|2484052|2483702|2483976|N|N|N|N|N| +2484069|AAAAAAAAFGHOFCAA|2089-01-17|2268|9864|757|2089|1|1|17|1|2089|757|9864|Monday|2089Q1|N|N|N|2484053|2484052|2483703|2483977|N|N|N|N|N| +2484070|AAAAAAAAGGHOFCAA|2089-01-18|2268|9865|757|2089|2|1|18|1|2089|757|9865|Tuesday|2089Q1|N|N|N|2484053|2484052|2483704|2483978|N|N|N|N|N| +2484071|AAAAAAAAHGHOFCAA|2089-01-19|2268|9865|757|2089|3|1|19|1|2089|757|9865|Wednesday|2089Q1|N|N|N|2484053|2484052|2483705|2483979|N|N|N|N|N| +2484072|AAAAAAAAIGHOFCAA|2089-01-20|2268|9865|757|2089|4|1|20|1|2089|757|9865|Thursday|2089Q1|N|N|N|2484053|2484052|2483706|2483980|N|N|N|N|N| +2484073|AAAAAAAAJGHOFCAA|2089-01-21|2268|9865|757|2089|5|1|21|1|2089|757|9865|Friday|2089Q1|N|Y|N|2484053|2484052|2483707|2483981|N|N|N|N|N| +2484074|AAAAAAAAKGHOFCAA|2089-01-22|2268|9865|757|2089|6|1|22|1|2089|757|9865|Saturday|2089Q1|N|Y|N|2484053|2484052|2483708|2483982|N|N|N|N|N| +2484075|AAAAAAAALGHOFCAA|2089-01-23|2268|9865|757|2089|0|1|23|1|2089|757|9865|Sunday|2089Q1|N|N|N|2484053|2484052|2483709|2483983|N|N|N|N|N| +2484076|AAAAAAAAMGHOFCAA|2089-01-24|2268|9865|757|2089|1|1|24|1|2089|757|9865|Monday|2089Q1|N|N|N|2484053|2484052|2483710|2483984|N|N|N|N|N| +2484077|AAAAAAAANGHOFCAA|2089-01-25|2268|9866|757|2089|2|1|25|1|2089|757|9866|Tuesday|2089Q1|N|N|N|2484053|2484052|2483711|2483985|N|N|N|N|N| +2484078|AAAAAAAAOGHOFCAA|2089-01-26|2268|9866|757|2089|3|1|26|1|2089|757|9866|Wednesday|2089Q1|N|N|N|2484053|2484052|2483712|2483986|N|N|N|N|N| +2484079|AAAAAAAAPGHOFCAA|2089-01-27|2268|9866|757|2089|4|1|27|1|2089|757|9866|Thursday|2089Q1|N|N|N|2484053|2484052|2483713|2483987|N|N|N|N|N| +2484080|AAAAAAAAAHHOFCAA|2089-01-28|2268|9866|757|2089|5|1|28|1|2089|757|9866|Friday|2089Q1|N|Y|N|2484053|2484052|2483714|2483988|N|N|N|N|N| +2484081|AAAAAAAABHHOFCAA|2089-01-29|2268|9866|757|2089|6|1|29|1|2089|757|9866|Saturday|2089Q1|N|Y|N|2484053|2484052|2483715|2483989|N|N|N|N|N| +2484082|AAAAAAAACHHOFCAA|2089-01-30|2268|9866|757|2089|0|1|30|1|2089|757|9866|Sunday|2089Q1|N|N|N|2484053|2484052|2483716|2483990|N|N|N|N|N| +2484083|AAAAAAAADHHOFCAA|2089-01-31|2268|9866|757|2089|1|1|31|1|2089|757|9866|Monday|2089Q1|N|N|N|2484053|2484052|2483717|2483991|N|N|N|N|N| +2484084|AAAAAAAAEHHOFCAA|2089-02-01|2269|9867|757|2089|2|2|1|1|2089|757|9867|Tuesday|2089Q1|N|N|N|2484084|2484114|2483718|2483992|N|N|N|N|N| +2484085|AAAAAAAAFHHOFCAA|2089-02-02|2269|9867|757|2089|3|2|2|1|2089|757|9867|Wednesday|2089Q1|N|N|N|2484084|2484114|2483719|2483993|N|N|N|N|N| +2484086|AAAAAAAAGHHOFCAA|2089-02-03|2269|9867|757|2089|4|2|3|1|2089|757|9867|Thursday|2089Q1|N|N|N|2484084|2484114|2483720|2483994|N|N|N|N|N| +2484087|AAAAAAAAHHHOFCAA|2089-02-04|2269|9867|757|2089|5|2|4|1|2089|757|9867|Friday|2089Q1|N|Y|N|2484084|2484114|2483721|2483995|N|N|N|N|N| +2484088|AAAAAAAAIHHOFCAA|2089-02-05|2269|9867|757|2089|6|2|5|1|2089|757|9867|Saturday|2089Q1|N|Y|N|2484084|2484114|2483722|2483996|N|N|N|N|N| +2484089|AAAAAAAAJHHOFCAA|2089-02-06|2269|9867|757|2089|0|2|6|1|2089|757|9867|Sunday|2089Q1|N|N|N|2484084|2484114|2483723|2483997|N|N|N|N|N| +2484090|AAAAAAAAKHHOFCAA|2089-02-07|2269|9867|757|2089|1|2|7|1|2089|757|9867|Monday|2089Q1|N|N|N|2484084|2484114|2483724|2483998|N|N|N|N|N| +2484091|AAAAAAAALHHOFCAA|2089-02-08|2269|9868|757|2089|2|2|8|1|2089|757|9868|Tuesday|2089Q1|N|N|N|2484084|2484114|2483725|2483999|N|N|N|N|N| +2484092|AAAAAAAAMHHOFCAA|2089-02-09|2269|9868|757|2089|3|2|9|1|2089|757|9868|Wednesday|2089Q1|N|N|N|2484084|2484114|2483726|2484000|N|N|N|N|N| +2484093|AAAAAAAANHHOFCAA|2089-02-10|2269|9868|757|2089|4|2|10|1|2089|757|9868|Thursday|2089Q1|N|N|N|2484084|2484114|2483727|2484001|N|N|N|N|N| +2484094|AAAAAAAAOHHOFCAA|2089-02-11|2269|9868|757|2089|5|2|11|1|2089|757|9868|Friday|2089Q1|N|Y|N|2484084|2484114|2483728|2484002|N|N|N|N|N| +2484095|AAAAAAAAPHHOFCAA|2089-02-12|2269|9868|757|2089|6|2|12|1|2089|757|9868|Saturday|2089Q1|N|Y|N|2484084|2484114|2483729|2484003|N|N|N|N|N| +2484096|AAAAAAAAAIHOFCAA|2089-02-13|2269|9868|757|2089|0|2|13|1|2089|757|9868|Sunday|2089Q1|N|N|N|2484084|2484114|2483730|2484004|N|N|N|N|N| +2484097|AAAAAAAABIHOFCAA|2089-02-14|2269|9868|757|2089|1|2|14|1|2089|757|9868|Monday|2089Q1|N|N|N|2484084|2484114|2483731|2484005|N|N|N|N|N| +2484098|AAAAAAAACIHOFCAA|2089-02-15|2269|9869|757|2089|2|2|15|1|2089|757|9869|Tuesday|2089Q1|N|N|N|2484084|2484114|2483732|2484006|N|N|N|N|N| +2484099|AAAAAAAADIHOFCAA|2089-02-16|2269|9869|757|2089|3|2|16|1|2089|757|9869|Wednesday|2089Q1|N|N|N|2484084|2484114|2483733|2484007|N|N|N|N|N| +2484100|AAAAAAAAEIHOFCAA|2089-02-17|2269|9869|757|2089|4|2|17|1|2089|757|9869|Thursday|2089Q1|N|N|N|2484084|2484114|2483734|2484008|N|N|N|N|N| +2484101|AAAAAAAAFIHOFCAA|2089-02-18|2269|9869|757|2089|5|2|18|1|2089|757|9869|Friday|2089Q1|N|Y|N|2484084|2484114|2483735|2484009|N|N|N|N|N| +2484102|AAAAAAAAGIHOFCAA|2089-02-19|2269|9869|757|2089|6|2|19|1|2089|757|9869|Saturday|2089Q1|N|Y|N|2484084|2484114|2483736|2484010|N|N|N|N|N| +2484103|AAAAAAAAHIHOFCAA|2089-02-20|2269|9869|757|2089|0|2|20|1|2089|757|9869|Sunday|2089Q1|N|N|N|2484084|2484114|2483737|2484011|N|N|N|N|N| +2484104|AAAAAAAAIIHOFCAA|2089-02-21|2269|9869|757|2089|1|2|21|1|2089|757|9869|Monday|2089Q1|N|N|N|2484084|2484114|2483738|2484012|N|N|N|N|N| +2484105|AAAAAAAAJIHOFCAA|2089-02-22|2269|9870|757|2089|2|2|22|1|2089|757|9870|Tuesday|2089Q1|N|N|N|2484084|2484114|2483739|2484013|N|N|N|N|N| +2484106|AAAAAAAAKIHOFCAA|2089-02-23|2269|9870|757|2089|3|2|23|1|2089|757|9870|Wednesday|2089Q1|N|N|N|2484084|2484114|2483740|2484014|N|N|N|N|N| +2484107|AAAAAAAALIHOFCAA|2089-02-24|2269|9870|757|2089|4|2|24|1|2089|757|9870|Thursday|2089Q1|N|N|N|2484084|2484114|2483741|2484015|N|N|N|N|N| +2484108|AAAAAAAAMIHOFCAA|2089-02-25|2269|9870|757|2089|5|2|25|1|2089|757|9870|Friday|2089Q1|N|Y|N|2484084|2484114|2483742|2484016|N|N|N|N|N| +2484109|AAAAAAAANIHOFCAA|2089-02-26|2269|9870|757|2089|6|2|26|1|2089|757|9870|Saturday|2089Q1|N|Y|N|2484084|2484114|2483743|2484017|N|N|N|N|N| +2484110|AAAAAAAAOIHOFCAA|2089-02-27|2269|9870|757|2089|0|2|27|1|2089|757|9870|Sunday|2089Q1|N|N|N|2484084|2484114|2483744|2484018|N|N|N|N|N| +2484111|AAAAAAAAPIHOFCAA|2089-02-28|2269|9870|757|2089|1|2|28|1|2089|757|9870|Monday|2089Q1|N|N|N|2484084|2484114|2483745|2484019|N|N|N|N|N| +2484112|AAAAAAAAAJHOFCAA|2089-03-01|2270|9871|758|2089|2|3|1|1|2089|758|9871|Tuesday|2089Q1|N|N|N|2484112|2484170|2483747|2484020|N|N|N|N|N| +2484113|AAAAAAAABJHOFCAA|2089-03-02|2270|9871|758|2089|3|3|2|1|2089|758|9871|Wednesday|2089Q1|N|N|N|2484112|2484170|2483748|2484021|N|N|N|N|N| +2484114|AAAAAAAACJHOFCAA|2089-03-03|2270|9871|758|2089|4|3|3|1|2089|758|9871|Thursday|2089Q1|N|N|N|2484112|2484170|2483749|2484022|N|N|N|N|N| +2484115|AAAAAAAADJHOFCAA|2089-03-04|2270|9871|758|2089|5|3|4|1|2089|758|9871|Friday|2089Q1|N|Y|N|2484112|2484170|2483750|2484023|N|N|N|N|N| +2484116|AAAAAAAAEJHOFCAA|2089-03-05|2270|9871|758|2089|6|3|5|1|2089|758|9871|Saturday|2089Q1|N|Y|N|2484112|2484170|2483751|2484024|N|N|N|N|N| +2484117|AAAAAAAAFJHOFCAA|2089-03-06|2270|9871|758|2089|0|3|6|1|2089|758|9871|Sunday|2089Q1|N|N|N|2484112|2484170|2483752|2484025|N|N|N|N|N| +2484118|AAAAAAAAGJHOFCAA|2089-03-07|2270|9871|758|2089|1|3|7|1|2089|758|9871|Monday|2089Q1|N|N|N|2484112|2484170|2483753|2484026|N|N|N|N|N| +2484119|AAAAAAAAHJHOFCAA|2089-03-08|2270|9872|758|2089|2|3|8|1|2089|758|9872|Tuesday|2089Q1|N|N|N|2484112|2484170|2483754|2484027|N|N|N|N|N| +2484120|AAAAAAAAIJHOFCAA|2089-03-09|2270|9872|758|2089|3|3|9|1|2089|758|9872|Wednesday|2089Q1|N|N|N|2484112|2484170|2483755|2484028|N|N|N|N|N| +2484121|AAAAAAAAJJHOFCAA|2089-03-10|2270|9872|758|2089|4|3|10|1|2089|758|9872|Thursday|2089Q1|N|N|N|2484112|2484170|2483756|2484029|N|N|N|N|N| +2484122|AAAAAAAAKJHOFCAA|2089-03-11|2270|9872|758|2089|5|3|11|1|2089|758|9872|Friday|2089Q1|N|Y|N|2484112|2484170|2483757|2484030|N|N|N|N|N| +2484123|AAAAAAAALJHOFCAA|2089-03-12|2270|9872|758|2089|6|3|12|1|2089|758|9872|Saturday|2089Q1|N|Y|N|2484112|2484170|2483758|2484031|N|N|N|N|N| +2484124|AAAAAAAAMJHOFCAA|2089-03-13|2270|9872|758|2089|0|3|13|1|2089|758|9872|Sunday|2089Q1|N|N|N|2484112|2484170|2483759|2484032|N|N|N|N|N| +2484125|AAAAAAAANJHOFCAA|2089-03-14|2270|9872|758|2089|1|3|14|1|2089|758|9872|Monday|2089Q1|N|N|N|2484112|2484170|2483760|2484033|N|N|N|N|N| +2484126|AAAAAAAAOJHOFCAA|2089-03-15|2270|9873|758|2089|2|3|15|1|2089|758|9873|Tuesday|2089Q1|N|N|N|2484112|2484170|2483761|2484034|N|N|N|N|N| +2484127|AAAAAAAAPJHOFCAA|2089-03-16|2270|9873|758|2089|3|3|16|1|2089|758|9873|Wednesday|2089Q1|N|N|N|2484112|2484170|2483762|2484035|N|N|N|N|N| +2484128|AAAAAAAAAKHOFCAA|2089-03-17|2270|9873|758|2089|4|3|17|1|2089|758|9873|Thursday|2089Q1|N|N|N|2484112|2484170|2483763|2484036|N|N|N|N|N| +2484129|AAAAAAAABKHOFCAA|2089-03-18|2270|9873|758|2089|5|3|18|1|2089|758|9873|Friday|2089Q1|N|Y|N|2484112|2484170|2483764|2484037|N|N|N|N|N| +2484130|AAAAAAAACKHOFCAA|2089-03-19|2270|9873|758|2089|6|3|19|1|2089|758|9873|Saturday|2089Q1|N|Y|N|2484112|2484170|2483765|2484038|N|N|N|N|N| +2484131|AAAAAAAADKHOFCAA|2089-03-20|2270|9873|758|2089|0|3|20|1|2089|758|9873|Sunday|2089Q1|N|N|N|2484112|2484170|2483766|2484039|N|N|N|N|N| +2484132|AAAAAAAAEKHOFCAA|2089-03-21|2270|9873|758|2089|1|3|21|1|2089|758|9873|Monday|2089Q1|N|N|N|2484112|2484170|2483767|2484040|N|N|N|N|N| +2484133|AAAAAAAAFKHOFCAA|2089-03-22|2270|9874|758|2089|2|3|22|1|2089|758|9874|Tuesday|2089Q1|N|N|N|2484112|2484170|2483768|2484041|N|N|N|N|N| +2484134|AAAAAAAAGKHOFCAA|2089-03-23|2270|9874|758|2089|3|3|23|1|2089|758|9874|Wednesday|2089Q1|N|N|N|2484112|2484170|2483769|2484042|N|N|N|N|N| +2484135|AAAAAAAAHKHOFCAA|2089-03-24|2270|9874|758|2089|4|3|24|1|2089|758|9874|Thursday|2089Q1|N|N|N|2484112|2484170|2483770|2484043|N|N|N|N|N| +2484136|AAAAAAAAIKHOFCAA|2089-03-25|2270|9874|758|2089|5|3|25|1|2089|758|9874|Friday|2089Q1|N|Y|N|2484112|2484170|2483771|2484044|N|N|N|N|N| +2484137|AAAAAAAAJKHOFCAA|2089-03-26|2270|9874|758|2089|6|3|26|1|2089|758|9874|Saturday|2089Q1|N|Y|N|2484112|2484170|2483772|2484045|N|N|N|N|N| +2484138|AAAAAAAAKKHOFCAA|2089-03-27|2270|9874|758|2089|0|3|27|1|2089|758|9874|Sunday|2089Q1|N|N|N|2484112|2484170|2483773|2484046|N|N|N|N|N| +2484139|AAAAAAAALKHOFCAA|2089-03-28|2270|9874|758|2089|1|3|28|1|2089|758|9874|Monday|2089Q1|N|N|N|2484112|2484170|2483774|2484047|N|N|N|N|N| +2484140|AAAAAAAAMKHOFCAA|2089-03-29|2270|9875|758|2089|2|3|29|1|2089|758|9875|Tuesday|2089Q1|N|N|N|2484112|2484170|2483775|2484048|N|N|N|N|N| +2484141|AAAAAAAANKHOFCAA|2089-03-30|2270|9875|758|2089|3|3|30|1|2089|758|9875|Wednesday|2089Q1|N|N|N|2484112|2484170|2483776|2484049|N|N|N|N|N| +2484142|AAAAAAAAOKHOFCAA|2089-03-31|2270|9875|758|2089|4|3|31|1|2089|758|9875|Thursday|2089Q1|N|N|N|2484112|2484170|2483777|2484050|N|N|N|N|N| +2484143|AAAAAAAAPKHOFCAA|2089-04-01|2271|9875|758|2089|5|4|1|1|2089|758|9875|Friday|2089Q1|N|Y|N|2484143|2484232|2483778|2484053|N|N|N|N|N| +2484144|AAAAAAAAALHOFCAA|2089-04-02|2271|9875|758|2089|6|4|2|2|2089|758|9875|Saturday|2089Q2|N|Y|N|2484143|2484232|2483779|2484054|N|N|N|N|N| +2484145|AAAAAAAABLHOFCAA|2089-04-03|2271|9875|758|2089|0|4|3|2|2089|758|9875|Sunday|2089Q2|N|N|N|2484143|2484232|2483780|2484055|N|N|N|N|N| +2484146|AAAAAAAACLHOFCAA|2089-04-04|2271|9875|758|2089|1|4|4|2|2089|758|9875|Monday|2089Q2|N|N|N|2484143|2484232|2483781|2484056|N|N|N|N|N| +2484147|AAAAAAAADLHOFCAA|2089-04-05|2271|9876|758|2089|2|4|5|2|2089|758|9876|Tuesday|2089Q2|N|N|N|2484143|2484232|2483782|2484057|N|N|N|N|N| +2484148|AAAAAAAAELHOFCAA|2089-04-06|2271|9876|758|2089|3|4|6|2|2089|758|9876|Wednesday|2089Q2|N|N|N|2484143|2484232|2483783|2484058|N|N|N|N|N| +2484149|AAAAAAAAFLHOFCAA|2089-04-07|2271|9876|758|2089|4|4|7|2|2089|758|9876|Thursday|2089Q2|N|N|N|2484143|2484232|2483784|2484059|N|N|N|N|N| +2484150|AAAAAAAAGLHOFCAA|2089-04-08|2271|9876|758|2089|5|4|8|2|2089|758|9876|Friday|2089Q2|N|Y|N|2484143|2484232|2483785|2484060|N|N|N|N|N| +2484151|AAAAAAAAHLHOFCAA|2089-04-09|2271|9876|758|2089|6|4|9|2|2089|758|9876|Saturday|2089Q2|N|Y|N|2484143|2484232|2483786|2484061|N|N|N|N|N| +2484152|AAAAAAAAILHOFCAA|2089-04-10|2271|9876|758|2089|0|4|10|2|2089|758|9876|Sunday|2089Q2|N|N|N|2484143|2484232|2483787|2484062|N|N|N|N|N| +2484153|AAAAAAAAJLHOFCAA|2089-04-11|2271|9876|758|2089|1|4|11|2|2089|758|9876|Monday|2089Q2|N|N|N|2484143|2484232|2483788|2484063|N|N|N|N|N| +2484154|AAAAAAAAKLHOFCAA|2089-04-12|2271|9877|758|2089|2|4|12|2|2089|758|9877|Tuesday|2089Q2|N|N|N|2484143|2484232|2483789|2484064|N|N|N|N|N| +2484155|AAAAAAAALLHOFCAA|2089-04-13|2271|9877|758|2089|3|4|13|2|2089|758|9877|Wednesday|2089Q2|N|N|N|2484143|2484232|2483790|2484065|N|N|N|N|N| +2484156|AAAAAAAAMLHOFCAA|2089-04-14|2271|9877|758|2089|4|4|14|2|2089|758|9877|Thursday|2089Q2|N|N|N|2484143|2484232|2483791|2484066|N|N|N|N|N| +2484157|AAAAAAAANLHOFCAA|2089-04-15|2271|9877|758|2089|5|4|15|2|2089|758|9877|Friday|2089Q2|N|Y|N|2484143|2484232|2483792|2484067|N|N|N|N|N| +2484158|AAAAAAAAOLHOFCAA|2089-04-16|2271|9877|758|2089|6|4|16|2|2089|758|9877|Saturday|2089Q2|N|Y|N|2484143|2484232|2483793|2484068|N|N|N|N|N| +2484159|AAAAAAAAPLHOFCAA|2089-04-17|2271|9877|758|2089|0|4|17|2|2089|758|9877|Sunday|2089Q2|N|N|N|2484143|2484232|2483794|2484069|N|N|N|N|N| +2484160|AAAAAAAAAMHOFCAA|2089-04-18|2271|9877|758|2089|1|4|18|2|2089|758|9877|Monday|2089Q2|N|N|N|2484143|2484232|2483795|2484070|N|N|N|N|N| +2484161|AAAAAAAABMHOFCAA|2089-04-19|2271|9878|758|2089|2|4|19|2|2089|758|9878|Tuesday|2089Q2|N|N|N|2484143|2484232|2483796|2484071|N|N|N|N|N| +2484162|AAAAAAAACMHOFCAA|2089-04-20|2271|9878|758|2089|3|4|20|2|2089|758|9878|Wednesday|2089Q2|N|N|N|2484143|2484232|2483797|2484072|N|N|N|N|N| +2484163|AAAAAAAADMHOFCAA|2089-04-21|2271|9878|758|2089|4|4|21|2|2089|758|9878|Thursday|2089Q2|N|N|N|2484143|2484232|2483798|2484073|N|N|N|N|N| +2484164|AAAAAAAAEMHOFCAA|2089-04-22|2271|9878|758|2089|5|4|22|2|2089|758|9878|Friday|2089Q2|N|Y|N|2484143|2484232|2483799|2484074|N|N|N|N|N| +2484165|AAAAAAAAFMHOFCAA|2089-04-23|2271|9878|758|2089|6|4|23|2|2089|758|9878|Saturday|2089Q2|N|Y|N|2484143|2484232|2483800|2484075|N|N|N|N|N| +2484166|AAAAAAAAGMHOFCAA|2089-04-24|2271|9878|758|2089|0|4|24|2|2089|758|9878|Sunday|2089Q2|N|N|N|2484143|2484232|2483801|2484076|N|N|N|N|N| +2484167|AAAAAAAAHMHOFCAA|2089-04-25|2271|9878|758|2089|1|4|25|2|2089|758|9878|Monday|2089Q2|N|N|N|2484143|2484232|2483802|2484077|N|N|N|N|N| +2484168|AAAAAAAAIMHOFCAA|2089-04-26|2271|9879|758|2089|2|4|26|2|2089|758|9879|Tuesday|2089Q2|N|N|N|2484143|2484232|2483803|2484078|N|N|N|N|N| +2484169|AAAAAAAAJMHOFCAA|2089-04-27|2271|9879|758|2089|3|4|27|2|2089|758|9879|Wednesday|2089Q2|N|N|N|2484143|2484232|2483804|2484079|N|N|N|N|N| +2484170|AAAAAAAAKMHOFCAA|2089-04-28|2271|9879|758|2089|4|4|28|2|2089|758|9879|Thursday|2089Q2|N|N|N|2484143|2484232|2483805|2484080|N|N|N|N|N| +2484171|AAAAAAAALMHOFCAA|2089-04-29|2271|9879|758|2089|5|4|29|2|2089|758|9879|Friday|2089Q2|N|Y|N|2484143|2484232|2483806|2484081|N|N|N|N|N| +2484172|AAAAAAAAMMHOFCAA|2089-04-30|2271|9879|758|2089|6|4|30|2|2089|758|9879|Saturday|2089Q2|N|Y|N|2484143|2484232|2483807|2484082|N|N|N|N|N| +2484173|AAAAAAAANMHOFCAA|2089-05-01|2272|9879|758|2089|0|5|1|2|2089|758|9879|Sunday|2089Q2|N|N|N|2484173|2484292|2483808|2484083|N|N|N|N|N| +2484174|AAAAAAAAOMHOFCAA|2089-05-02|2272|9879|758|2089|1|5|2|2|2089|758|9879|Monday|2089Q2|N|N|N|2484173|2484292|2483809|2484084|N|N|N|N|N| +2484175|AAAAAAAAPMHOFCAA|2089-05-03|2272|9880|758|2089|2|5|3|2|2089|758|9880|Tuesday|2089Q2|N|N|N|2484173|2484292|2483810|2484085|N|N|N|N|N| +2484176|AAAAAAAAANHOFCAA|2089-05-04|2272|9880|758|2089|3|5|4|2|2089|758|9880|Wednesday|2089Q2|N|N|N|2484173|2484292|2483811|2484086|N|N|N|N|N| +2484177|AAAAAAAABNHOFCAA|2089-05-05|2272|9880|758|2089|4|5|5|2|2089|758|9880|Thursday|2089Q2|N|N|N|2484173|2484292|2483812|2484087|N|N|N|N|N| +2484178|AAAAAAAACNHOFCAA|2089-05-06|2272|9880|758|2089|5|5|6|2|2089|758|9880|Friday|2089Q2|N|Y|N|2484173|2484292|2483813|2484088|N|N|N|N|N| +2484179|AAAAAAAADNHOFCAA|2089-05-07|2272|9880|758|2089|6|5|7|2|2089|758|9880|Saturday|2089Q2|N|Y|N|2484173|2484292|2483814|2484089|N|N|N|N|N| +2484180|AAAAAAAAENHOFCAA|2089-05-08|2272|9880|758|2089|0|5|8|2|2089|758|9880|Sunday|2089Q2|N|N|N|2484173|2484292|2483815|2484090|N|N|N|N|N| +2484181|AAAAAAAAFNHOFCAA|2089-05-09|2272|9880|758|2089|1|5|9|2|2089|758|9880|Monday|2089Q2|N|N|N|2484173|2484292|2483816|2484091|N|N|N|N|N| +2484182|AAAAAAAAGNHOFCAA|2089-05-10|2272|9881|758|2089|2|5|10|2|2089|758|9881|Tuesday|2089Q2|N|N|N|2484173|2484292|2483817|2484092|N|N|N|N|N| +2484183|AAAAAAAAHNHOFCAA|2089-05-11|2272|9881|758|2089|3|5|11|2|2089|758|9881|Wednesday|2089Q2|N|N|N|2484173|2484292|2483818|2484093|N|N|N|N|N| +2484184|AAAAAAAAINHOFCAA|2089-05-12|2272|9881|758|2089|4|5|12|2|2089|758|9881|Thursday|2089Q2|N|N|N|2484173|2484292|2483819|2484094|N|N|N|N|N| +2484185|AAAAAAAAJNHOFCAA|2089-05-13|2272|9881|758|2089|5|5|13|2|2089|758|9881|Friday|2089Q2|N|Y|N|2484173|2484292|2483820|2484095|N|N|N|N|N| +2484186|AAAAAAAAKNHOFCAA|2089-05-14|2272|9881|758|2089|6|5|14|2|2089|758|9881|Saturday|2089Q2|N|Y|N|2484173|2484292|2483821|2484096|N|N|N|N|N| +2484187|AAAAAAAALNHOFCAA|2089-05-15|2272|9881|758|2089|0|5|15|2|2089|758|9881|Sunday|2089Q2|N|N|N|2484173|2484292|2483822|2484097|N|N|N|N|N| +2484188|AAAAAAAAMNHOFCAA|2089-05-16|2272|9881|758|2089|1|5|16|2|2089|758|9881|Monday|2089Q2|N|N|N|2484173|2484292|2483823|2484098|N|N|N|N|N| +2484189|AAAAAAAANNHOFCAA|2089-05-17|2272|9882|758|2089|2|5|17|2|2089|758|9882|Tuesday|2089Q2|N|N|N|2484173|2484292|2483824|2484099|N|N|N|N|N| +2484190|AAAAAAAAONHOFCAA|2089-05-18|2272|9882|758|2089|3|5|18|2|2089|758|9882|Wednesday|2089Q2|N|N|N|2484173|2484292|2483825|2484100|N|N|N|N|N| +2484191|AAAAAAAAPNHOFCAA|2089-05-19|2272|9882|758|2089|4|5|19|2|2089|758|9882|Thursday|2089Q2|N|N|N|2484173|2484292|2483826|2484101|N|N|N|N|N| +2484192|AAAAAAAAAOHOFCAA|2089-05-20|2272|9882|758|2089|5|5|20|2|2089|758|9882|Friday|2089Q2|N|Y|N|2484173|2484292|2483827|2484102|N|N|N|N|N| +2484193|AAAAAAAABOHOFCAA|2089-05-21|2272|9882|758|2089|6|5|21|2|2089|758|9882|Saturday|2089Q2|N|Y|N|2484173|2484292|2483828|2484103|N|N|N|N|N| +2484194|AAAAAAAACOHOFCAA|2089-05-22|2272|9882|758|2089|0|5|22|2|2089|758|9882|Sunday|2089Q2|N|N|N|2484173|2484292|2483829|2484104|N|N|N|N|N| +2484195|AAAAAAAADOHOFCAA|2089-05-23|2272|9882|758|2089|1|5|23|2|2089|758|9882|Monday|2089Q2|N|N|N|2484173|2484292|2483830|2484105|N|N|N|N|N| +2484196|AAAAAAAAEOHOFCAA|2089-05-24|2272|9883|758|2089|2|5|24|2|2089|758|9883|Tuesday|2089Q2|N|N|N|2484173|2484292|2483831|2484106|N|N|N|N|N| +2484197|AAAAAAAAFOHOFCAA|2089-05-25|2272|9883|758|2089|3|5|25|2|2089|758|9883|Wednesday|2089Q2|N|N|N|2484173|2484292|2483832|2484107|N|N|N|N|N| +2484198|AAAAAAAAGOHOFCAA|2089-05-26|2272|9883|758|2089|4|5|26|2|2089|758|9883|Thursday|2089Q2|N|N|N|2484173|2484292|2483833|2484108|N|N|N|N|N| +2484199|AAAAAAAAHOHOFCAA|2089-05-27|2272|9883|758|2089|5|5|27|2|2089|758|9883|Friday|2089Q2|N|Y|N|2484173|2484292|2483834|2484109|N|N|N|N|N| +2484200|AAAAAAAAIOHOFCAA|2089-05-28|2272|9883|758|2089|6|5|28|2|2089|758|9883|Saturday|2089Q2|N|Y|N|2484173|2484292|2483835|2484110|N|N|N|N|N| +2484201|AAAAAAAAJOHOFCAA|2089-05-29|2272|9883|758|2089|0|5|29|2|2089|758|9883|Sunday|2089Q2|N|N|N|2484173|2484292|2483836|2484111|N|N|N|N|N| +2484202|AAAAAAAAKOHOFCAA|2089-05-30|2272|9883|758|2089|1|5|30|2|2089|758|9883|Monday|2089Q2|N|N|N|2484173|2484292|2483837|2484112|N|N|N|N|N| +2484203|AAAAAAAALOHOFCAA|2089-05-31|2272|9884|758|2089|2|5|31|2|2089|758|9884|Tuesday|2089Q2|N|N|N|2484173|2484292|2483838|2484113|N|N|N|N|N| +2484204|AAAAAAAAMOHOFCAA|2089-06-01|2273|9884|759|2089|3|6|1|2|2089|759|9884|Wednesday|2089Q2|N|N|N|2484204|2484354|2483839|2484114|N|N|N|N|N| +2484205|AAAAAAAANOHOFCAA|2089-06-02|2273|9884|759|2089|4|6|2|2|2089|759|9884|Thursday|2089Q2|N|N|N|2484204|2484354|2483840|2484115|N|N|N|N|N| +2484206|AAAAAAAAOOHOFCAA|2089-06-03|2273|9884|759|2089|5|6|3|2|2089|759|9884|Friday|2089Q2|N|Y|N|2484204|2484354|2483841|2484116|N|N|N|N|N| +2484207|AAAAAAAAPOHOFCAA|2089-06-04|2273|9884|759|2089|6|6|4|2|2089|759|9884|Saturday|2089Q2|N|Y|N|2484204|2484354|2483842|2484117|N|N|N|N|N| +2484208|AAAAAAAAAPHOFCAA|2089-06-05|2273|9884|759|2089|0|6|5|2|2089|759|9884|Sunday|2089Q2|N|N|N|2484204|2484354|2483843|2484118|N|N|N|N|N| +2484209|AAAAAAAABPHOFCAA|2089-06-06|2273|9884|759|2089|1|6|6|2|2089|759|9884|Monday|2089Q2|N|N|N|2484204|2484354|2483844|2484119|N|N|N|N|N| +2484210|AAAAAAAACPHOFCAA|2089-06-07|2273|9885|759|2089|2|6|7|2|2089|759|9885|Tuesday|2089Q2|N|N|N|2484204|2484354|2483845|2484120|N|N|N|N|N| +2484211|AAAAAAAADPHOFCAA|2089-06-08|2273|9885|759|2089|3|6|8|2|2089|759|9885|Wednesday|2089Q2|N|N|N|2484204|2484354|2483846|2484121|N|N|N|N|N| +2484212|AAAAAAAAEPHOFCAA|2089-06-09|2273|9885|759|2089|4|6|9|2|2089|759|9885|Thursday|2089Q2|N|N|N|2484204|2484354|2483847|2484122|N|N|N|N|N| +2484213|AAAAAAAAFPHOFCAA|2089-06-10|2273|9885|759|2089|5|6|10|2|2089|759|9885|Friday|2089Q2|N|Y|N|2484204|2484354|2483848|2484123|N|N|N|N|N| +2484214|AAAAAAAAGPHOFCAA|2089-06-11|2273|9885|759|2089|6|6|11|2|2089|759|9885|Saturday|2089Q2|N|Y|N|2484204|2484354|2483849|2484124|N|N|N|N|N| +2484215|AAAAAAAAHPHOFCAA|2089-06-12|2273|9885|759|2089|0|6|12|2|2089|759|9885|Sunday|2089Q2|N|N|N|2484204|2484354|2483850|2484125|N|N|N|N|N| +2484216|AAAAAAAAIPHOFCAA|2089-06-13|2273|9885|759|2089|1|6|13|2|2089|759|9885|Monday|2089Q2|N|N|N|2484204|2484354|2483851|2484126|N|N|N|N|N| +2484217|AAAAAAAAJPHOFCAA|2089-06-14|2273|9886|759|2089|2|6|14|2|2089|759|9886|Tuesday|2089Q2|N|N|N|2484204|2484354|2483852|2484127|N|N|N|N|N| +2484218|AAAAAAAAKPHOFCAA|2089-06-15|2273|9886|759|2089|3|6|15|2|2089|759|9886|Wednesday|2089Q2|N|N|N|2484204|2484354|2483853|2484128|N|N|N|N|N| +2484219|AAAAAAAALPHOFCAA|2089-06-16|2273|9886|759|2089|4|6|16|2|2089|759|9886|Thursday|2089Q2|N|N|N|2484204|2484354|2483854|2484129|N|N|N|N|N| +2484220|AAAAAAAAMPHOFCAA|2089-06-17|2273|9886|759|2089|5|6|17|2|2089|759|9886|Friday|2089Q2|N|Y|N|2484204|2484354|2483855|2484130|N|N|N|N|N| +2484221|AAAAAAAANPHOFCAA|2089-06-18|2273|9886|759|2089|6|6|18|2|2089|759|9886|Saturday|2089Q2|N|Y|N|2484204|2484354|2483856|2484131|N|N|N|N|N| +2484222|AAAAAAAAOPHOFCAA|2089-06-19|2273|9886|759|2089|0|6|19|2|2089|759|9886|Sunday|2089Q2|N|N|N|2484204|2484354|2483857|2484132|N|N|N|N|N| +2484223|AAAAAAAAPPHOFCAA|2089-06-20|2273|9886|759|2089|1|6|20|2|2089|759|9886|Monday|2089Q2|N|N|N|2484204|2484354|2483858|2484133|N|N|N|N|N| +2484224|AAAAAAAAAAIOFCAA|2089-06-21|2273|9887|759|2089|2|6|21|2|2089|759|9887|Tuesday|2089Q2|N|N|N|2484204|2484354|2483859|2484134|N|N|N|N|N| +2484225|AAAAAAAABAIOFCAA|2089-06-22|2273|9887|759|2089|3|6|22|2|2089|759|9887|Wednesday|2089Q2|N|N|N|2484204|2484354|2483860|2484135|N|N|N|N|N| +2484226|AAAAAAAACAIOFCAA|2089-06-23|2273|9887|759|2089|4|6|23|2|2089|759|9887|Thursday|2089Q2|N|N|N|2484204|2484354|2483861|2484136|N|N|N|N|N| +2484227|AAAAAAAADAIOFCAA|2089-06-24|2273|9887|759|2089|5|6|24|2|2089|759|9887|Friday|2089Q2|N|Y|N|2484204|2484354|2483862|2484137|N|N|N|N|N| +2484228|AAAAAAAAEAIOFCAA|2089-06-25|2273|9887|759|2089|6|6|25|2|2089|759|9887|Saturday|2089Q2|N|Y|N|2484204|2484354|2483863|2484138|N|N|N|N|N| +2484229|AAAAAAAAFAIOFCAA|2089-06-26|2273|9887|759|2089|0|6|26|2|2089|759|9887|Sunday|2089Q2|N|N|N|2484204|2484354|2483864|2484139|N|N|N|N|N| +2484230|AAAAAAAAGAIOFCAA|2089-06-27|2273|9887|759|2089|1|6|27|2|2089|759|9887|Monday|2089Q2|N|N|N|2484204|2484354|2483865|2484140|N|N|N|N|N| +2484231|AAAAAAAAHAIOFCAA|2089-06-28|2273|9888|759|2089|2|6|28|2|2089|759|9888|Tuesday|2089Q2|N|N|N|2484204|2484354|2483866|2484141|N|N|N|N|N| +2484232|AAAAAAAAIAIOFCAA|2089-06-29|2273|9888|759|2089|3|6|29|2|2089|759|9888|Wednesday|2089Q2|N|N|N|2484204|2484354|2483867|2484142|N|N|N|N|N| +2484233|AAAAAAAAJAIOFCAA|2089-06-30|2273|9888|759|2089|4|6|30|2|2089|759|9888|Thursday|2089Q2|N|N|N|2484204|2484354|2483868|2484143|N|N|N|N|N| +2484234|AAAAAAAAKAIOFCAA|2089-07-01|2274|9888|759|2089|5|7|1|2|2089|759|9888|Friday|2089Q2|N|Y|N|2484234|2484414|2483869|2484143|N|N|N|N|N| +2484235|AAAAAAAALAIOFCAA|2089-07-02|2274|9888|759|2089|6|7|2|3|2089|759|9888|Saturday|2089Q3|N|Y|N|2484234|2484414|2483870|2484144|N|N|N|N|N| +2484236|AAAAAAAAMAIOFCAA|2089-07-03|2274|9888|759|2089|0|7|3|3|2089|759|9888|Sunday|2089Q3|N|N|N|2484234|2484414|2483871|2484145|N|N|N|N|N| +2484237|AAAAAAAANAIOFCAA|2089-07-04|2274|9888|759|2089|1|7|4|3|2089|759|9888|Monday|2089Q3|N|N|N|2484234|2484414|2483872|2484146|N|N|N|N|N| +2484238|AAAAAAAAOAIOFCAA|2089-07-05|2274|9889|759|2089|2|7|5|3|2089|759|9889|Tuesday|2089Q3|Y|N|N|2484234|2484414|2483873|2484147|N|N|N|N|N| +2484239|AAAAAAAAPAIOFCAA|2089-07-06|2274|9889|759|2089|3|7|6|3|2089|759|9889|Wednesday|2089Q3|N|N|Y|2484234|2484414|2483874|2484148|N|N|N|N|N| +2484240|AAAAAAAAABIOFCAA|2089-07-07|2274|9889|759|2089|4|7|7|3|2089|759|9889|Thursday|2089Q3|N|N|N|2484234|2484414|2483875|2484149|N|N|N|N|N| +2484241|AAAAAAAABBIOFCAA|2089-07-08|2274|9889|759|2089|5|7|8|3|2089|759|9889|Friday|2089Q3|N|Y|N|2484234|2484414|2483876|2484150|N|N|N|N|N| +2484242|AAAAAAAACBIOFCAA|2089-07-09|2274|9889|759|2089|6|7|9|3|2089|759|9889|Saturday|2089Q3|N|Y|N|2484234|2484414|2483877|2484151|N|N|N|N|N| +2484243|AAAAAAAADBIOFCAA|2089-07-10|2274|9889|759|2089|0|7|10|3|2089|759|9889|Sunday|2089Q3|N|N|N|2484234|2484414|2483878|2484152|N|N|N|N|N| +2484244|AAAAAAAAEBIOFCAA|2089-07-11|2274|9889|759|2089|1|7|11|3|2089|759|9889|Monday|2089Q3|N|N|N|2484234|2484414|2483879|2484153|N|N|N|N|N| +2484245|AAAAAAAAFBIOFCAA|2089-07-12|2274|9890|759|2089|2|7|12|3|2089|759|9890|Tuesday|2089Q3|N|N|N|2484234|2484414|2483880|2484154|N|N|N|N|N| +2484246|AAAAAAAAGBIOFCAA|2089-07-13|2274|9890|759|2089|3|7|13|3|2089|759|9890|Wednesday|2089Q3|N|N|N|2484234|2484414|2483881|2484155|N|N|N|N|N| +2484247|AAAAAAAAHBIOFCAA|2089-07-14|2274|9890|759|2089|4|7|14|3|2089|759|9890|Thursday|2089Q3|N|N|N|2484234|2484414|2483882|2484156|N|N|N|N|N| +2484248|AAAAAAAAIBIOFCAA|2089-07-15|2274|9890|759|2089|5|7|15|3|2089|759|9890|Friday|2089Q3|N|Y|N|2484234|2484414|2483883|2484157|N|N|N|N|N| +2484249|AAAAAAAAJBIOFCAA|2089-07-16|2274|9890|759|2089|6|7|16|3|2089|759|9890|Saturday|2089Q3|N|Y|N|2484234|2484414|2483884|2484158|N|N|N|N|N| +2484250|AAAAAAAAKBIOFCAA|2089-07-17|2274|9890|759|2089|0|7|17|3|2089|759|9890|Sunday|2089Q3|N|N|N|2484234|2484414|2483885|2484159|N|N|N|N|N| +2484251|AAAAAAAALBIOFCAA|2089-07-18|2274|9890|759|2089|1|7|18|3|2089|759|9890|Monday|2089Q3|N|N|N|2484234|2484414|2483886|2484160|N|N|N|N|N| +2484252|AAAAAAAAMBIOFCAA|2089-07-19|2274|9891|759|2089|2|7|19|3|2089|759|9891|Tuesday|2089Q3|N|N|N|2484234|2484414|2483887|2484161|N|N|N|N|N| +2484253|AAAAAAAANBIOFCAA|2089-07-20|2274|9891|759|2089|3|7|20|3|2089|759|9891|Wednesday|2089Q3|N|N|N|2484234|2484414|2483888|2484162|N|N|N|N|N| +2484254|AAAAAAAAOBIOFCAA|2089-07-21|2274|9891|759|2089|4|7|21|3|2089|759|9891|Thursday|2089Q3|N|N|N|2484234|2484414|2483889|2484163|N|N|N|N|N| +2484255|AAAAAAAAPBIOFCAA|2089-07-22|2274|9891|759|2089|5|7|22|3|2089|759|9891|Friday|2089Q3|N|Y|N|2484234|2484414|2483890|2484164|N|N|N|N|N| +2484256|AAAAAAAAACIOFCAA|2089-07-23|2274|9891|759|2089|6|7|23|3|2089|759|9891|Saturday|2089Q3|N|Y|N|2484234|2484414|2483891|2484165|N|N|N|N|N| +2484257|AAAAAAAABCIOFCAA|2089-07-24|2274|9891|759|2089|0|7|24|3|2089|759|9891|Sunday|2089Q3|N|N|N|2484234|2484414|2483892|2484166|N|N|N|N|N| +2484258|AAAAAAAACCIOFCAA|2089-07-25|2274|9891|759|2089|1|7|25|3|2089|759|9891|Monday|2089Q3|N|N|N|2484234|2484414|2483893|2484167|N|N|N|N|N| +2484259|AAAAAAAADCIOFCAA|2089-07-26|2274|9892|759|2089|2|7|26|3|2089|759|9892|Tuesday|2089Q3|N|N|N|2484234|2484414|2483894|2484168|N|N|N|N|N| +2484260|AAAAAAAAECIOFCAA|2089-07-27|2274|9892|759|2089|3|7|27|3|2089|759|9892|Wednesday|2089Q3|N|N|N|2484234|2484414|2483895|2484169|N|N|N|N|N| +2484261|AAAAAAAAFCIOFCAA|2089-07-28|2274|9892|759|2089|4|7|28|3|2089|759|9892|Thursday|2089Q3|N|N|N|2484234|2484414|2483896|2484170|N|N|N|N|N| +2484262|AAAAAAAAGCIOFCAA|2089-07-29|2274|9892|759|2089|5|7|29|3|2089|759|9892|Friday|2089Q3|N|Y|N|2484234|2484414|2483897|2484171|N|N|N|N|N| +2484263|AAAAAAAAHCIOFCAA|2089-07-30|2274|9892|759|2089|6|7|30|3|2089|759|9892|Saturday|2089Q3|N|Y|N|2484234|2484414|2483898|2484172|N|N|N|N|N| +2484264|AAAAAAAAICIOFCAA|2089-07-31|2274|9892|759|2089|0|7|31|3|2089|759|9892|Sunday|2089Q3|N|N|N|2484234|2484414|2483899|2484173|N|N|N|N|N| +2484265|AAAAAAAAJCIOFCAA|2089-08-01|2275|9892|759|2089|1|8|1|3|2089|759|9892|Monday|2089Q3|N|N|N|2484265|2484476|2483900|2484174|N|N|N|N|N| +2484266|AAAAAAAAKCIOFCAA|2089-08-02|2275|9893|759|2089|2|8|2|3|2089|759|9893|Tuesday|2089Q3|N|N|N|2484265|2484476|2483901|2484175|N|N|N|N|N| +2484267|AAAAAAAALCIOFCAA|2089-08-03|2275|9893|759|2089|3|8|3|3|2089|759|9893|Wednesday|2089Q3|N|N|N|2484265|2484476|2483902|2484176|N|N|N|N|N| +2484268|AAAAAAAAMCIOFCAA|2089-08-04|2275|9893|759|2089|4|8|4|3|2089|759|9893|Thursday|2089Q3|N|N|N|2484265|2484476|2483903|2484177|N|N|N|N|N| +2484269|AAAAAAAANCIOFCAA|2089-08-05|2275|9893|759|2089|5|8|5|3|2089|759|9893|Friday|2089Q3|N|Y|N|2484265|2484476|2483904|2484178|N|N|N|N|N| +2484270|AAAAAAAAOCIOFCAA|2089-08-06|2275|9893|759|2089|6|8|6|3|2089|759|9893|Saturday|2089Q3|N|Y|N|2484265|2484476|2483905|2484179|N|N|N|N|N| +2484271|AAAAAAAAPCIOFCAA|2089-08-07|2275|9893|759|2089|0|8|7|3|2089|759|9893|Sunday|2089Q3|N|N|N|2484265|2484476|2483906|2484180|N|N|N|N|N| +2484272|AAAAAAAAADIOFCAA|2089-08-08|2275|9893|759|2089|1|8|8|3|2089|759|9893|Monday|2089Q3|N|N|N|2484265|2484476|2483907|2484181|N|N|N|N|N| +2484273|AAAAAAAABDIOFCAA|2089-08-09|2275|9894|759|2089|2|8|9|3|2089|759|9894|Tuesday|2089Q3|N|N|N|2484265|2484476|2483908|2484182|N|N|N|N|N| +2484274|AAAAAAAACDIOFCAA|2089-08-10|2275|9894|759|2089|3|8|10|3|2089|759|9894|Wednesday|2089Q3|N|N|N|2484265|2484476|2483909|2484183|N|N|N|N|N| +2484275|AAAAAAAADDIOFCAA|2089-08-11|2275|9894|759|2089|4|8|11|3|2089|759|9894|Thursday|2089Q3|N|N|N|2484265|2484476|2483910|2484184|N|N|N|N|N| +2484276|AAAAAAAAEDIOFCAA|2089-08-12|2275|9894|759|2089|5|8|12|3|2089|759|9894|Friday|2089Q3|N|Y|N|2484265|2484476|2483911|2484185|N|N|N|N|N| +2484277|AAAAAAAAFDIOFCAA|2089-08-13|2275|9894|759|2089|6|8|13|3|2089|759|9894|Saturday|2089Q3|N|Y|N|2484265|2484476|2483912|2484186|N|N|N|N|N| +2484278|AAAAAAAAGDIOFCAA|2089-08-14|2275|9894|759|2089|0|8|14|3|2089|759|9894|Sunday|2089Q3|N|N|N|2484265|2484476|2483913|2484187|N|N|N|N|N| +2484279|AAAAAAAAHDIOFCAA|2089-08-15|2275|9894|759|2089|1|8|15|3|2089|759|9894|Monday|2089Q3|N|N|N|2484265|2484476|2483914|2484188|N|N|N|N|N| +2484280|AAAAAAAAIDIOFCAA|2089-08-16|2275|9895|759|2089|2|8|16|3|2089|759|9895|Tuesday|2089Q3|N|N|N|2484265|2484476|2483915|2484189|N|N|N|N|N| +2484281|AAAAAAAAJDIOFCAA|2089-08-17|2275|9895|759|2089|3|8|17|3|2089|759|9895|Wednesday|2089Q3|N|N|N|2484265|2484476|2483916|2484190|N|N|N|N|N| +2484282|AAAAAAAAKDIOFCAA|2089-08-18|2275|9895|759|2089|4|8|18|3|2089|759|9895|Thursday|2089Q3|N|N|N|2484265|2484476|2483917|2484191|N|N|N|N|N| +2484283|AAAAAAAALDIOFCAA|2089-08-19|2275|9895|759|2089|5|8|19|3|2089|759|9895|Friday|2089Q3|N|Y|N|2484265|2484476|2483918|2484192|N|N|N|N|N| +2484284|AAAAAAAAMDIOFCAA|2089-08-20|2275|9895|759|2089|6|8|20|3|2089|759|9895|Saturday|2089Q3|N|Y|N|2484265|2484476|2483919|2484193|N|N|N|N|N| +2484285|AAAAAAAANDIOFCAA|2089-08-21|2275|9895|759|2089|0|8|21|3|2089|759|9895|Sunday|2089Q3|N|N|N|2484265|2484476|2483920|2484194|N|N|N|N|N| +2484286|AAAAAAAAODIOFCAA|2089-08-22|2275|9895|759|2089|1|8|22|3|2089|759|9895|Monday|2089Q3|N|N|N|2484265|2484476|2483921|2484195|N|N|N|N|N| +2484287|AAAAAAAAPDIOFCAA|2089-08-23|2275|9896|759|2089|2|8|23|3|2089|759|9896|Tuesday|2089Q3|N|N|N|2484265|2484476|2483922|2484196|N|N|N|N|N| +2484288|AAAAAAAAAEIOFCAA|2089-08-24|2275|9896|759|2089|3|8|24|3|2089|759|9896|Wednesday|2089Q3|N|N|N|2484265|2484476|2483923|2484197|N|N|N|N|N| +2484289|AAAAAAAABEIOFCAA|2089-08-25|2275|9896|759|2089|4|8|25|3|2089|759|9896|Thursday|2089Q3|N|N|N|2484265|2484476|2483924|2484198|N|N|N|N|N| +2484290|AAAAAAAACEIOFCAA|2089-08-26|2275|9896|759|2089|5|8|26|3|2089|759|9896|Friday|2089Q3|N|Y|N|2484265|2484476|2483925|2484199|N|N|N|N|N| +2484291|AAAAAAAADEIOFCAA|2089-08-27|2275|9896|759|2089|6|8|27|3|2089|759|9896|Saturday|2089Q3|N|Y|N|2484265|2484476|2483926|2484200|N|N|N|N|N| +2484292|AAAAAAAAEEIOFCAA|2089-08-28|2275|9896|759|2089|0|8|28|3|2089|759|9896|Sunday|2089Q3|N|N|N|2484265|2484476|2483927|2484201|N|N|N|N|N| +2484293|AAAAAAAAFEIOFCAA|2089-08-29|2275|9896|759|2089|1|8|29|3|2089|759|9896|Monday|2089Q3|N|N|N|2484265|2484476|2483928|2484202|N|N|N|N|N| +2484294|AAAAAAAAGEIOFCAA|2089-08-30|2275|9897|759|2089|2|8|30|3|2089|759|9897|Tuesday|2089Q3|N|N|N|2484265|2484476|2483929|2484203|N|N|N|N|N| +2484295|AAAAAAAAHEIOFCAA|2089-08-31|2275|9897|759|2089|3|8|31|3|2089|759|9897|Wednesday|2089Q3|N|N|N|2484265|2484476|2483930|2484204|N|N|N|N|N| +2484296|AAAAAAAAIEIOFCAA|2089-09-01|2276|9897|760|2089|4|9|1|3|2089|760|9897|Thursday|2089Q3|N|N|N|2484296|2484538|2483931|2484205|N|N|N|N|N| +2484297|AAAAAAAAJEIOFCAA|2089-09-02|2276|9897|760|2089|5|9|2|3|2089|760|9897|Friday|2089Q3|N|Y|N|2484296|2484538|2483932|2484206|N|N|N|N|N| +2484298|AAAAAAAAKEIOFCAA|2089-09-03|2276|9897|760|2089|6|9|3|3|2089|760|9897|Saturday|2089Q3|N|Y|N|2484296|2484538|2483933|2484207|N|N|N|N|N| +2484299|AAAAAAAALEIOFCAA|2089-09-04|2276|9897|760|2089|0|9|4|3|2089|760|9897|Sunday|2089Q3|N|N|N|2484296|2484538|2483934|2484208|N|N|N|N|N| +2484300|AAAAAAAAMEIOFCAA|2089-09-05|2276|9897|760|2089|1|9|5|3|2089|760|9897|Monday|2089Q3|N|N|N|2484296|2484538|2483935|2484209|N|N|N|N|N| +2484301|AAAAAAAANEIOFCAA|2089-09-06|2276|9898|760|2089|2|9|6|3|2089|760|9898|Tuesday|2089Q3|N|N|N|2484296|2484538|2483936|2484210|N|N|N|N|N| +2484302|AAAAAAAAOEIOFCAA|2089-09-07|2276|9898|760|2089|3|9|7|3|2089|760|9898|Wednesday|2089Q3|N|N|N|2484296|2484538|2483937|2484211|N|N|N|N|N| +2484303|AAAAAAAAPEIOFCAA|2089-09-08|2276|9898|760|2089|4|9|8|3|2089|760|9898|Thursday|2089Q3|N|N|N|2484296|2484538|2483938|2484212|N|N|N|N|N| +2484304|AAAAAAAAAFIOFCAA|2089-09-09|2276|9898|760|2089|5|9|9|3|2089|760|9898|Friday|2089Q3|N|Y|N|2484296|2484538|2483939|2484213|N|N|N|N|N| +2484305|AAAAAAAABFIOFCAA|2089-09-10|2276|9898|760|2089|6|9|10|3|2089|760|9898|Saturday|2089Q3|N|Y|N|2484296|2484538|2483940|2484214|N|N|N|N|N| +2484306|AAAAAAAACFIOFCAA|2089-09-11|2276|9898|760|2089|0|9|11|3|2089|760|9898|Sunday|2089Q3|N|N|N|2484296|2484538|2483941|2484215|N|N|N|N|N| +2484307|AAAAAAAADFIOFCAA|2089-09-12|2276|9898|760|2089|1|9|12|3|2089|760|9898|Monday|2089Q3|N|N|N|2484296|2484538|2483942|2484216|N|N|N|N|N| +2484308|AAAAAAAAEFIOFCAA|2089-09-13|2276|9899|760|2089|2|9|13|3|2089|760|9899|Tuesday|2089Q3|N|N|N|2484296|2484538|2483943|2484217|N|N|N|N|N| +2484309|AAAAAAAAFFIOFCAA|2089-09-14|2276|9899|760|2089|3|9|14|3|2089|760|9899|Wednesday|2089Q3|N|N|N|2484296|2484538|2483944|2484218|N|N|N|N|N| +2484310|AAAAAAAAGFIOFCAA|2089-09-15|2276|9899|760|2089|4|9|15|3|2089|760|9899|Thursday|2089Q3|N|N|N|2484296|2484538|2483945|2484219|N|N|N|N|N| +2484311|AAAAAAAAHFIOFCAA|2089-09-16|2276|9899|760|2089|5|9|16|3|2089|760|9899|Friday|2089Q3|N|Y|N|2484296|2484538|2483946|2484220|N|N|N|N|N| +2484312|AAAAAAAAIFIOFCAA|2089-09-17|2276|9899|760|2089|6|9|17|3|2089|760|9899|Saturday|2089Q3|N|Y|N|2484296|2484538|2483947|2484221|N|N|N|N|N| +2484313|AAAAAAAAJFIOFCAA|2089-09-18|2276|9899|760|2089|0|9|18|3|2089|760|9899|Sunday|2089Q3|N|N|N|2484296|2484538|2483948|2484222|N|N|N|N|N| +2484314|AAAAAAAAKFIOFCAA|2089-09-19|2276|9899|760|2089|1|9|19|3|2089|760|9899|Monday|2089Q3|N|N|N|2484296|2484538|2483949|2484223|N|N|N|N|N| +2484315|AAAAAAAALFIOFCAA|2089-09-20|2276|9900|760|2089|2|9|20|3|2089|760|9900|Tuesday|2089Q3|N|N|N|2484296|2484538|2483950|2484224|N|N|N|N|N| +2484316|AAAAAAAAMFIOFCAA|2089-09-21|2276|9900|760|2089|3|9|21|3|2089|760|9900|Wednesday|2089Q3|N|N|N|2484296|2484538|2483951|2484225|N|N|N|N|N| +2484317|AAAAAAAANFIOFCAA|2089-09-22|2276|9900|760|2089|4|9|22|3|2089|760|9900|Thursday|2089Q3|N|N|N|2484296|2484538|2483952|2484226|N|N|N|N|N| +2484318|AAAAAAAAOFIOFCAA|2089-09-23|2276|9900|760|2089|5|9|23|3|2089|760|9900|Friday|2089Q3|N|Y|N|2484296|2484538|2483953|2484227|N|N|N|N|N| +2484319|AAAAAAAAPFIOFCAA|2089-09-24|2276|9900|760|2089|6|9|24|3|2089|760|9900|Saturday|2089Q3|N|Y|N|2484296|2484538|2483954|2484228|N|N|N|N|N| +2484320|AAAAAAAAAGIOFCAA|2089-09-25|2276|9900|760|2089|0|9|25|3|2089|760|9900|Sunday|2089Q3|N|N|N|2484296|2484538|2483955|2484229|N|N|N|N|N| +2484321|AAAAAAAABGIOFCAA|2089-09-26|2276|9900|760|2089|1|9|26|3|2089|760|9900|Monday|2089Q3|N|N|N|2484296|2484538|2483956|2484230|N|N|N|N|N| +2484322|AAAAAAAACGIOFCAA|2089-09-27|2276|9901|760|2089|2|9|27|3|2089|760|9901|Tuesday|2089Q3|N|N|N|2484296|2484538|2483957|2484231|N|N|N|N|N| +2484323|AAAAAAAADGIOFCAA|2089-09-28|2276|9901|760|2089|3|9|28|3|2089|760|9901|Wednesday|2089Q3|N|N|N|2484296|2484538|2483958|2484232|N|N|N|N|N| +2484324|AAAAAAAAEGIOFCAA|2089-09-29|2276|9901|760|2089|4|9|29|3|2089|760|9901|Thursday|2089Q3|N|N|N|2484296|2484538|2483959|2484233|N|N|N|N|N| +2484325|AAAAAAAAFGIOFCAA|2089-09-30|2276|9901|760|2089|5|9|30|3|2089|760|9901|Friday|2089Q3|N|Y|N|2484296|2484538|2483960|2484234|N|N|N|N|N| +2484326|AAAAAAAAGGIOFCAA|2089-10-01|2277|9901|760|2089|6|10|1|3|2089|760|9901|Saturday|2089Q3|N|Y|N|2484326|2484598|2483961|2484234|N|N|N|N|N| +2484327|AAAAAAAAHGIOFCAA|2089-10-02|2277|9901|760|2089|0|10|2|4|2089|760|9901|Sunday|2089Q4|N|N|N|2484326|2484598|2483962|2484235|N|N|N|N|N| +2484328|AAAAAAAAIGIOFCAA|2089-10-03|2277|9901|760|2089|1|10|3|4|2089|760|9901|Monday|2089Q4|N|N|N|2484326|2484598|2483963|2484236|N|N|N|N|N| +2484329|AAAAAAAAJGIOFCAA|2089-10-04|2277|9902|760|2089|2|10|4|4|2089|760|9902|Tuesday|2089Q4|N|N|N|2484326|2484598|2483964|2484237|N|N|N|N|N| +2484330|AAAAAAAAKGIOFCAA|2089-10-05|2277|9902|760|2089|3|10|5|4|2089|760|9902|Wednesday|2089Q4|N|N|N|2484326|2484598|2483965|2484238|N|N|N|N|N| +2484331|AAAAAAAALGIOFCAA|2089-10-06|2277|9902|760|2089|4|10|6|4|2089|760|9902|Thursday|2089Q4|N|N|N|2484326|2484598|2483966|2484239|N|N|N|N|N| +2484332|AAAAAAAAMGIOFCAA|2089-10-07|2277|9902|760|2089|5|10|7|4|2089|760|9902|Friday|2089Q4|N|Y|N|2484326|2484598|2483967|2484240|N|N|N|N|N| +2484333|AAAAAAAANGIOFCAA|2089-10-08|2277|9902|760|2089|6|10|8|4|2089|760|9902|Saturday|2089Q4|N|Y|N|2484326|2484598|2483968|2484241|N|N|N|N|N| +2484334|AAAAAAAAOGIOFCAA|2089-10-09|2277|9902|760|2089|0|10|9|4|2089|760|9902|Sunday|2089Q4|N|N|N|2484326|2484598|2483969|2484242|N|N|N|N|N| +2484335|AAAAAAAAPGIOFCAA|2089-10-10|2277|9902|760|2089|1|10|10|4|2089|760|9902|Monday|2089Q4|N|N|N|2484326|2484598|2483970|2484243|N|N|N|N|N| +2484336|AAAAAAAAAHIOFCAA|2089-10-11|2277|9903|760|2089|2|10|11|4|2089|760|9903|Tuesday|2089Q4|N|N|N|2484326|2484598|2483971|2484244|N|N|N|N|N| +2484337|AAAAAAAABHIOFCAA|2089-10-12|2277|9903|760|2089|3|10|12|4|2089|760|9903|Wednesday|2089Q4|N|N|N|2484326|2484598|2483972|2484245|N|N|N|N|N| +2484338|AAAAAAAACHIOFCAA|2089-10-13|2277|9903|760|2089|4|10|13|4|2089|760|9903|Thursday|2089Q4|N|N|N|2484326|2484598|2483973|2484246|N|N|N|N|N| +2484339|AAAAAAAADHIOFCAA|2089-10-14|2277|9903|760|2089|5|10|14|4|2089|760|9903|Friday|2089Q4|N|Y|N|2484326|2484598|2483974|2484247|N|N|N|N|N| +2484340|AAAAAAAAEHIOFCAA|2089-10-15|2277|9903|760|2089|6|10|15|4|2089|760|9903|Saturday|2089Q4|N|Y|N|2484326|2484598|2483975|2484248|N|N|N|N|N| +2484341|AAAAAAAAFHIOFCAA|2089-10-16|2277|9903|760|2089|0|10|16|4|2089|760|9903|Sunday|2089Q4|N|N|N|2484326|2484598|2483976|2484249|N|N|N|N|N| +2484342|AAAAAAAAGHIOFCAA|2089-10-17|2277|9903|760|2089|1|10|17|4|2089|760|9903|Monday|2089Q4|N|N|N|2484326|2484598|2483977|2484250|N|N|N|N|N| +2484343|AAAAAAAAHHIOFCAA|2089-10-18|2277|9904|760|2089|2|10|18|4|2089|760|9904|Tuesday|2089Q4|N|N|N|2484326|2484598|2483978|2484251|N|N|N|N|N| +2484344|AAAAAAAAIHIOFCAA|2089-10-19|2277|9904|760|2089|3|10|19|4|2089|760|9904|Wednesday|2089Q4|N|N|N|2484326|2484598|2483979|2484252|N|N|N|N|N| +2484345|AAAAAAAAJHIOFCAA|2089-10-20|2277|9904|760|2089|4|10|20|4|2089|760|9904|Thursday|2089Q4|N|N|N|2484326|2484598|2483980|2484253|N|N|N|N|N| +2484346|AAAAAAAAKHIOFCAA|2089-10-21|2277|9904|760|2089|5|10|21|4|2089|760|9904|Friday|2089Q4|N|Y|N|2484326|2484598|2483981|2484254|N|N|N|N|N| +2484347|AAAAAAAALHIOFCAA|2089-10-22|2277|9904|760|2089|6|10|22|4|2089|760|9904|Saturday|2089Q4|N|Y|N|2484326|2484598|2483982|2484255|N|N|N|N|N| +2484348|AAAAAAAAMHIOFCAA|2089-10-23|2277|9904|760|2089|0|10|23|4|2089|760|9904|Sunday|2089Q4|N|N|N|2484326|2484598|2483983|2484256|N|N|N|N|N| +2484349|AAAAAAAANHIOFCAA|2089-10-24|2277|9904|760|2089|1|10|24|4|2089|760|9904|Monday|2089Q4|N|N|N|2484326|2484598|2483984|2484257|N|N|N|N|N| +2484350|AAAAAAAAOHIOFCAA|2089-10-25|2277|9905|760|2089|2|10|25|4|2089|760|9905|Tuesday|2089Q4|N|N|N|2484326|2484598|2483985|2484258|N|N|N|N|N| +2484351|AAAAAAAAPHIOFCAA|2089-10-26|2277|9905|760|2089|3|10|26|4|2089|760|9905|Wednesday|2089Q4|N|N|N|2484326|2484598|2483986|2484259|N|N|N|N|N| +2484352|AAAAAAAAAIIOFCAA|2089-10-27|2277|9905|760|2089|4|10|27|4|2089|760|9905|Thursday|2089Q4|N|N|N|2484326|2484598|2483987|2484260|N|N|N|N|N| +2484353|AAAAAAAABIIOFCAA|2089-10-28|2277|9905|760|2089|5|10|28|4|2089|760|9905|Friday|2089Q4|N|Y|N|2484326|2484598|2483988|2484261|N|N|N|N|N| +2484354|AAAAAAAACIIOFCAA|2089-10-29|2277|9905|760|2089|6|10|29|4|2089|760|9905|Saturday|2089Q4|N|Y|N|2484326|2484598|2483989|2484262|N|N|N|N|N| +2484355|AAAAAAAADIIOFCAA|2089-10-30|2277|9905|760|2089|0|10|30|4|2089|760|9905|Sunday|2089Q4|N|N|N|2484326|2484598|2483990|2484263|N|N|N|N|N| +2484356|AAAAAAAAEIIOFCAA|2089-10-31|2277|9905|760|2089|1|10|31|4|2089|760|9905|Monday|2089Q4|N|N|N|2484326|2484598|2483991|2484264|N|N|N|N|N| +2484357|AAAAAAAAFIIOFCAA|2089-11-01|2278|9906|760|2089|2|11|1|4|2089|760|9906|Tuesday|2089Q4|N|N|N|2484357|2484660|2483992|2484265|N|N|N|N|N| +2484358|AAAAAAAAGIIOFCAA|2089-11-02|2278|9906|760|2089|3|11|2|4|2089|760|9906|Wednesday|2089Q4|N|N|N|2484357|2484660|2483993|2484266|N|N|N|N|N| +2484359|AAAAAAAAHIIOFCAA|2089-11-03|2278|9906|760|2089|4|11|3|4|2089|760|9906|Thursday|2089Q4|N|N|N|2484357|2484660|2483994|2484267|N|N|N|N|N| +2484360|AAAAAAAAIIIOFCAA|2089-11-04|2278|9906|760|2089|5|11|4|4|2089|760|9906|Friday|2089Q4|N|Y|N|2484357|2484660|2483995|2484268|N|N|N|N|N| +2484361|AAAAAAAAJIIOFCAA|2089-11-05|2278|9906|760|2089|6|11|5|4|2089|760|9906|Saturday|2089Q4|N|Y|N|2484357|2484660|2483996|2484269|N|N|N|N|N| +2484362|AAAAAAAAKIIOFCAA|2089-11-06|2278|9906|760|2089|0|11|6|4|2089|760|9906|Sunday|2089Q4|N|N|N|2484357|2484660|2483997|2484270|N|N|N|N|N| +2484363|AAAAAAAALIIOFCAA|2089-11-07|2278|9906|760|2089|1|11|7|4|2089|760|9906|Monday|2089Q4|N|N|N|2484357|2484660|2483998|2484271|N|N|N|N|N| +2484364|AAAAAAAAMIIOFCAA|2089-11-08|2278|9907|760|2089|2|11|8|4|2089|760|9907|Tuesday|2089Q4|N|N|N|2484357|2484660|2483999|2484272|N|N|N|N|N| +2484365|AAAAAAAANIIOFCAA|2089-11-09|2278|9907|760|2089|3|11|9|4|2089|760|9907|Wednesday|2089Q4|N|N|N|2484357|2484660|2484000|2484273|N|N|N|N|N| +2484366|AAAAAAAAOIIOFCAA|2089-11-10|2278|9907|760|2089|4|11|10|4|2089|760|9907|Thursday|2089Q4|N|N|N|2484357|2484660|2484001|2484274|N|N|N|N|N| +2484367|AAAAAAAAPIIOFCAA|2089-11-11|2278|9907|760|2089|5|11|11|4|2089|760|9907|Friday|2089Q4|N|Y|N|2484357|2484660|2484002|2484275|N|N|N|N|N| +2484368|AAAAAAAAAJIOFCAA|2089-11-12|2278|9907|760|2089|6|11|12|4|2089|760|9907|Saturday|2089Q4|N|Y|N|2484357|2484660|2484003|2484276|N|N|N|N|N| +2484369|AAAAAAAABJIOFCAA|2089-11-13|2278|9907|760|2089|0|11|13|4|2089|760|9907|Sunday|2089Q4|N|N|N|2484357|2484660|2484004|2484277|N|N|N|N|N| +2484370|AAAAAAAACJIOFCAA|2089-11-14|2278|9907|760|2089|1|11|14|4|2089|760|9907|Monday|2089Q4|N|N|N|2484357|2484660|2484005|2484278|N|N|N|N|N| +2484371|AAAAAAAADJIOFCAA|2089-11-15|2278|9908|760|2089|2|11|15|4|2089|760|9908|Tuesday|2089Q4|N|N|N|2484357|2484660|2484006|2484279|N|N|N|N|N| +2484372|AAAAAAAAEJIOFCAA|2089-11-16|2278|9908|760|2089|3|11|16|4|2089|760|9908|Wednesday|2089Q4|N|N|N|2484357|2484660|2484007|2484280|N|N|N|N|N| +2484373|AAAAAAAAFJIOFCAA|2089-11-17|2278|9908|760|2089|4|11|17|4|2089|760|9908|Thursday|2089Q4|N|N|N|2484357|2484660|2484008|2484281|N|N|N|N|N| +2484374|AAAAAAAAGJIOFCAA|2089-11-18|2278|9908|760|2089|5|11|18|4|2089|760|9908|Friday|2089Q4|N|Y|N|2484357|2484660|2484009|2484282|N|N|N|N|N| +2484375|AAAAAAAAHJIOFCAA|2089-11-19|2278|9908|760|2089|6|11|19|4|2089|760|9908|Saturday|2089Q4|N|Y|N|2484357|2484660|2484010|2484283|N|N|N|N|N| +2484376|AAAAAAAAIJIOFCAA|2089-11-20|2278|9908|760|2089|0|11|20|4|2089|760|9908|Sunday|2089Q4|N|N|N|2484357|2484660|2484011|2484284|N|N|N|N|N| +2484377|AAAAAAAAJJIOFCAA|2089-11-21|2278|9908|760|2089|1|11|21|4|2089|760|9908|Monday|2089Q4|N|N|N|2484357|2484660|2484012|2484285|N|N|N|N|N| +2484378|AAAAAAAAKJIOFCAA|2089-11-22|2278|9909|760|2089|2|11|22|4|2089|760|9909|Tuesday|2089Q4|N|N|N|2484357|2484660|2484013|2484286|N|N|N|N|N| +2484379|AAAAAAAALJIOFCAA|2089-11-23|2278|9909|760|2089|3|11|23|4|2089|760|9909|Wednesday|2089Q4|N|N|N|2484357|2484660|2484014|2484287|N|N|N|N|N| +2484380|AAAAAAAAMJIOFCAA|2089-11-24|2278|9909|760|2089|4|11|24|4|2089|760|9909|Thursday|2089Q4|N|N|N|2484357|2484660|2484015|2484288|N|N|N|N|N| +2484381|AAAAAAAANJIOFCAA|2089-11-25|2278|9909|760|2089|5|11|25|4|2089|760|9909|Friday|2089Q4|N|Y|N|2484357|2484660|2484016|2484289|N|N|N|N|N| +2484382|AAAAAAAAOJIOFCAA|2089-11-26|2278|9909|760|2089|6|11|26|4|2089|760|9909|Saturday|2089Q4|N|Y|N|2484357|2484660|2484017|2484290|N|N|N|N|N| +2484383|AAAAAAAAPJIOFCAA|2089-11-27|2278|9909|760|2089|0|11|27|4|2089|760|9909|Sunday|2089Q4|N|N|N|2484357|2484660|2484018|2484291|N|N|N|N|N| +2484384|AAAAAAAAAKIOFCAA|2089-11-28|2278|9909|760|2089|1|11|28|4|2089|760|9909|Monday|2089Q4|N|N|N|2484357|2484660|2484019|2484292|N|N|N|N|N| +2484385|AAAAAAAABKIOFCAA|2089-11-29|2278|9910|760|2089|2|11|29|4|2089|760|9910|Tuesday|2089Q4|N|N|N|2484357|2484660|2484020|2484293|N|N|N|N|N| +2484386|AAAAAAAACKIOFCAA|2089-11-30|2278|9910|760|2089|3|11|30|4|2089|760|9910|Wednesday|2089Q4|N|N|N|2484357|2484660|2484021|2484294|N|N|N|N|N| +2484387|AAAAAAAADKIOFCAA|2089-12-01|2279|9910|761|2089|4|12|1|4|2089|761|9910|Thursday|2089Q4|N|N|N|2484387|2484720|2484022|2484295|N|N|N|N|N| +2484388|AAAAAAAAEKIOFCAA|2089-12-02|2279|9910|761|2089|5|12|2|4|2089|761|9910|Friday|2089Q4|N|Y|N|2484387|2484720|2484023|2484296|N|N|N|N|N| +2484389|AAAAAAAAFKIOFCAA|2089-12-03|2279|9910|761|2089|6|12|3|4|2089|761|9910|Saturday|2089Q4|N|Y|N|2484387|2484720|2484024|2484297|N|N|N|N|N| +2484390|AAAAAAAAGKIOFCAA|2089-12-04|2279|9910|761|2089|0|12|4|4|2089|761|9910|Sunday|2089Q4|N|N|N|2484387|2484720|2484025|2484298|N|N|N|N|N| +2484391|AAAAAAAAHKIOFCAA|2089-12-05|2279|9910|761|2089|1|12|5|4|2089|761|9910|Monday|2089Q4|N|N|N|2484387|2484720|2484026|2484299|N|N|N|N|N| +2484392|AAAAAAAAIKIOFCAA|2089-12-06|2279|9911|761|2089|2|12|6|4|2089|761|9911|Tuesday|2089Q4|N|N|N|2484387|2484720|2484027|2484300|N|N|N|N|N| +2484393|AAAAAAAAJKIOFCAA|2089-12-07|2279|9911|761|2089|3|12|7|4|2089|761|9911|Wednesday|2089Q4|N|N|N|2484387|2484720|2484028|2484301|N|N|N|N|N| +2484394|AAAAAAAAKKIOFCAA|2089-12-08|2279|9911|761|2089|4|12|8|4|2089|761|9911|Thursday|2089Q4|N|N|N|2484387|2484720|2484029|2484302|N|N|N|N|N| +2484395|AAAAAAAALKIOFCAA|2089-12-09|2279|9911|761|2089|5|12|9|4|2089|761|9911|Friday|2089Q4|N|Y|N|2484387|2484720|2484030|2484303|N|N|N|N|N| +2484396|AAAAAAAAMKIOFCAA|2089-12-10|2279|9911|761|2089|6|12|10|4|2089|761|9911|Saturday|2089Q4|N|Y|N|2484387|2484720|2484031|2484304|N|N|N|N|N| +2484397|AAAAAAAANKIOFCAA|2089-12-11|2279|9911|761|2089|0|12|11|4|2089|761|9911|Sunday|2089Q4|N|N|N|2484387|2484720|2484032|2484305|N|N|N|N|N| +2484398|AAAAAAAAOKIOFCAA|2089-12-12|2279|9911|761|2089|1|12|12|4|2089|761|9911|Monday|2089Q4|N|N|N|2484387|2484720|2484033|2484306|N|N|N|N|N| +2484399|AAAAAAAAPKIOFCAA|2089-12-13|2279|9912|761|2089|2|12|13|4|2089|761|9912|Tuesday|2089Q4|N|N|N|2484387|2484720|2484034|2484307|N|N|N|N|N| +2484400|AAAAAAAAALIOFCAA|2089-12-14|2279|9912|761|2089|3|12|14|4|2089|761|9912|Wednesday|2089Q4|N|N|N|2484387|2484720|2484035|2484308|N|N|N|N|N| +2484401|AAAAAAAABLIOFCAA|2089-12-15|2279|9912|761|2089|4|12|15|4|2089|761|9912|Thursday|2089Q4|N|N|N|2484387|2484720|2484036|2484309|N|N|N|N|N| +2484402|AAAAAAAACLIOFCAA|2089-12-16|2279|9912|761|2089|5|12|16|4|2089|761|9912|Friday|2089Q4|N|Y|N|2484387|2484720|2484037|2484310|N|N|N|N|N| +2484403|AAAAAAAADLIOFCAA|2089-12-17|2279|9912|761|2089|6|12|17|4|2089|761|9912|Saturday|2089Q4|N|Y|N|2484387|2484720|2484038|2484311|N|N|N|N|N| +2484404|AAAAAAAAELIOFCAA|2089-12-18|2279|9912|761|2089|0|12|18|4|2089|761|9912|Sunday|2089Q4|N|N|N|2484387|2484720|2484039|2484312|N|N|N|N|N| +2484405|AAAAAAAAFLIOFCAA|2089-12-19|2279|9912|761|2089|1|12|19|4|2089|761|9912|Monday|2089Q4|N|N|N|2484387|2484720|2484040|2484313|N|N|N|N|N| +2484406|AAAAAAAAGLIOFCAA|2089-12-20|2279|9913|761|2089|2|12|20|4|2089|761|9913|Tuesday|2089Q4|N|N|N|2484387|2484720|2484041|2484314|N|N|N|N|N| +2484407|AAAAAAAAHLIOFCAA|2089-12-21|2279|9913|761|2089|3|12|21|4|2089|761|9913|Wednesday|2089Q4|N|N|N|2484387|2484720|2484042|2484315|N|N|N|N|N| +2484408|AAAAAAAAILIOFCAA|2089-12-22|2279|9913|761|2089|4|12|22|4|2089|761|9913|Thursday|2089Q4|N|N|N|2484387|2484720|2484043|2484316|N|N|N|N|N| +2484409|AAAAAAAAJLIOFCAA|2089-12-23|2279|9913|761|2089|5|12|23|4|2089|761|9913|Friday|2089Q4|N|Y|N|2484387|2484720|2484044|2484317|N|N|N|N|N| +2484410|AAAAAAAAKLIOFCAA|2089-12-24|2279|9913|761|2089|6|12|24|4|2089|761|9913|Saturday|2089Q4|N|Y|N|2484387|2484720|2484045|2484318|N|N|N|N|N| +2484411|AAAAAAAALLIOFCAA|2089-12-25|2279|9913|761|2089|0|12|25|4|2089|761|9913|Sunday|2089Q4|N|N|N|2484387|2484720|2484046|2484319|N|N|N|N|N| +2484412|AAAAAAAAMLIOFCAA|2089-12-26|2279|9913|761|2089|1|12|26|4|2089|761|9913|Monday|2089Q4|Y|N|N|2484387|2484720|2484047|2484320|N|N|N|N|N| +2484413|AAAAAAAANLIOFCAA|2089-12-27|2279|9914|761|2089|2|12|27|4|2089|761|9914|Tuesday|2089Q4|N|N|Y|2484387|2484720|2484048|2484321|N|N|N|N|N| +2484414|AAAAAAAAOLIOFCAA|2089-12-28|2279|9914|761|2089|3|12|28|4|2089|761|9914|Wednesday|2089Q4|N|N|N|2484387|2484720|2484049|2484322|N|N|N|N|N| +2484415|AAAAAAAAPLIOFCAA|2089-12-29|2279|9914|761|2089|4|12|29|4|2089|761|9914|Thursday|2089Q4|N|N|N|2484387|2484720|2484050|2484323|N|N|N|N|N| +2484416|AAAAAAAAAMIOFCAA|2089-12-30|2279|9914|761|2089|5|12|30|4|2089|761|9914|Friday|2089Q4|N|Y|N|2484387|2484720|2484051|2484324|N|N|N|N|N| +2484417|AAAAAAAABMIOFCAA|2089-12-31|2279|9914|761|2089|6|12|31|4|2089|761|9914|Saturday|2089Q4|N|Y|N|2484387|2484720|2484052|2484325|N|N|N|N|N| +2484418|AAAAAAAACMIOFCAA|2090-01-01|2280|9914|761|2090|0|1|1|1|2090|761|9914|Sunday|2090Q1|Y|N|N|2484418|2484417|2484053|2484326|N|N|N|N|N| +2484419|AAAAAAAADMIOFCAA|2090-01-02|2280|9914|761|2090|1|1|2|1|2090|761|9914|Monday|2090Q1|N|N|Y|2484418|2484417|2484054|2484327|N|N|N|N|N| +2484420|AAAAAAAAEMIOFCAA|2090-01-03|2280|9915|761|2090|2|1|3|1|2090|761|9915|Tuesday|2090Q1|N|N|N|2484418|2484417|2484055|2484328|N|N|N|N|N| +2484421|AAAAAAAAFMIOFCAA|2090-01-04|2280|9915|761|2090|3|1|4|1|2090|761|9915|Wednesday|2090Q1|N|N|N|2484418|2484417|2484056|2484329|N|N|N|N|N| +2484422|AAAAAAAAGMIOFCAA|2090-01-05|2280|9915|761|2090|4|1|5|1|2090|761|9915|Thursday|2090Q1|N|N|N|2484418|2484417|2484057|2484330|N|N|N|N|N| +2484423|AAAAAAAAHMIOFCAA|2090-01-06|2280|9915|761|2090|5|1|6|1|2090|761|9915|Friday|2090Q1|N|Y|N|2484418|2484417|2484058|2484331|N|N|N|N|N| +2484424|AAAAAAAAIMIOFCAA|2090-01-07|2280|9915|761|2090|6|1|7|1|2090|761|9915|Saturday|2090Q1|N|Y|N|2484418|2484417|2484059|2484332|N|N|N|N|N| +2484425|AAAAAAAAJMIOFCAA|2090-01-08|2280|9915|761|2090|0|1|8|1|2090|761|9915|Sunday|2090Q1|N|N|N|2484418|2484417|2484060|2484333|N|N|N|N|N| +2484426|AAAAAAAAKMIOFCAA|2090-01-09|2280|9915|761|2090|1|1|9|1|2090|761|9915|Monday|2090Q1|N|N|N|2484418|2484417|2484061|2484334|N|N|N|N|N| +2484427|AAAAAAAALMIOFCAA|2090-01-10|2280|9916|761|2090|2|1|10|1|2090|761|9916|Tuesday|2090Q1|N|N|N|2484418|2484417|2484062|2484335|N|N|N|N|N| +2484428|AAAAAAAAMMIOFCAA|2090-01-11|2280|9916|761|2090|3|1|11|1|2090|761|9916|Wednesday|2090Q1|N|N|N|2484418|2484417|2484063|2484336|N|N|N|N|N| +2484429|AAAAAAAANMIOFCAA|2090-01-12|2280|9916|761|2090|4|1|12|1|2090|761|9916|Thursday|2090Q1|N|N|N|2484418|2484417|2484064|2484337|N|N|N|N|N| +2484430|AAAAAAAAOMIOFCAA|2090-01-13|2280|9916|761|2090|5|1|13|1|2090|761|9916|Friday|2090Q1|N|Y|N|2484418|2484417|2484065|2484338|N|N|N|N|N| +2484431|AAAAAAAAPMIOFCAA|2090-01-14|2280|9916|761|2090|6|1|14|1|2090|761|9916|Saturday|2090Q1|N|Y|N|2484418|2484417|2484066|2484339|N|N|N|N|N| +2484432|AAAAAAAAANIOFCAA|2090-01-15|2280|9916|761|2090|0|1|15|1|2090|761|9916|Sunday|2090Q1|N|N|N|2484418|2484417|2484067|2484340|N|N|N|N|N| +2484433|AAAAAAAABNIOFCAA|2090-01-16|2280|9916|761|2090|1|1|16|1|2090|761|9916|Monday|2090Q1|N|N|N|2484418|2484417|2484068|2484341|N|N|N|N|N| +2484434|AAAAAAAACNIOFCAA|2090-01-17|2280|9917|761|2090|2|1|17|1|2090|761|9917|Tuesday|2090Q1|N|N|N|2484418|2484417|2484069|2484342|N|N|N|N|N| +2484435|AAAAAAAADNIOFCAA|2090-01-18|2280|9917|761|2090|3|1|18|1|2090|761|9917|Wednesday|2090Q1|N|N|N|2484418|2484417|2484070|2484343|N|N|N|N|N| +2484436|AAAAAAAAENIOFCAA|2090-01-19|2280|9917|761|2090|4|1|19|1|2090|761|9917|Thursday|2090Q1|N|N|N|2484418|2484417|2484071|2484344|N|N|N|N|N| +2484437|AAAAAAAAFNIOFCAA|2090-01-20|2280|9917|761|2090|5|1|20|1|2090|761|9917|Friday|2090Q1|N|Y|N|2484418|2484417|2484072|2484345|N|N|N|N|N| +2484438|AAAAAAAAGNIOFCAA|2090-01-21|2280|9917|761|2090|6|1|21|1|2090|761|9917|Saturday|2090Q1|N|Y|N|2484418|2484417|2484073|2484346|N|N|N|N|N| +2484439|AAAAAAAAHNIOFCAA|2090-01-22|2280|9917|761|2090|0|1|22|1|2090|761|9917|Sunday|2090Q1|N|N|N|2484418|2484417|2484074|2484347|N|N|N|N|N| +2484440|AAAAAAAAINIOFCAA|2090-01-23|2280|9917|761|2090|1|1|23|1|2090|761|9917|Monday|2090Q1|N|N|N|2484418|2484417|2484075|2484348|N|N|N|N|N| +2484441|AAAAAAAAJNIOFCAA|2090-01-24|2280|9918|761|2090|2|1|24|1|2090|761|9918|Tuesday|2090Q1|N|N|N|2484418|2484417|2484076|2484349|N|N|N|N|N| +2484442|AAAAAAAAKNIOFCAA|2090-01-25|2280|9918|761|2090|3|1|25|1|2090|761|9918|Wednesday|2090Q1|N|N|N|2484418|2484417|2484077|2484350|N|N|N|N|N| +2484443|AAAAAAAALNIOFCAA|2090-01-26|2280|9918|761|2090|4|1|26|1|2090|761|9918|Thursday|2090Q1|N|N|N|2484418|2484417|2484078|2484351|N|N|N|N|N| +2484444|AAAAAAAAMNIOFCAA|2090-01-27|2280|9918|761|2090|5|1|27|1|2090|761|9918|Friday|2090Q1|N|Y|N|2484418|2484417|2484079|2484352|N|N|N|N|N| +2484445|AAAAAAAANNIOFCAA|2090-01-28|2280|9918|761|2090|6|1|28|1|2090|761|9918|Saturday|2090Q1|N|Y|N|2484418|2484417|2484080|2484353|N|N|N|N|N| +2484446|AAAAAAAAONIOFCAA|2090-01-29|2280|9918|761|2090|0|1|29|1|2090|761|9918|Sunday|2090Q1|N|N|N|2484418|2484417|2484081|2484354|N|N|N|N|N| +2484447|AAAAAAAAPNIOFCAA|2090-01-30|2280|9918|761|2090|1|1|30|1|2090|761|9918|Monday|2090Q1|N|N|N|2484418|2484417|2484082|2484355|N|N|N|N|N| +2484448|AAAAAAAAAOIOFCAA|2090-01-31|2280|9919|761|2090|2|1|31|1|2090|761|9919|Tuesday|2090Q1|N|N|N|2484418|2484417|2484083|2484356|N|N|N|N|N| +2484449|AAAAAAAABOIOFCAA|2090-02-01|2281|9919|761|2090|3|2|1|1|2090|761|9919|Wednesday|2090Q1|N|N|N|2484449|2484479|2484084|2484357|N|N|N|N|N| +2484450|AAAAAAAACOIOFCAA|2090-02-02|2281|9919|761|2090|4|2|2|1|2090|761|9919|Thursday|2090Q1|N|N|N|2484449|2484479|2484085|2484358|N|N|N|N|N| +2484451|AAAAAAAADOIOFCAA|2090-02-03|2281|9919|761|2090|5|2|3|1|2090|761|9919|Friday|2090Q1|N|Y|N|2484449|2484479|2484086|2484359|N|N|N|N|N| +2484452|AAAAAAAAEOIOFCAA|2090-02-04|2281|9919|761|2090|6|2|4|1|2090|761|9919|Saturday|2090Q1|N|Y|N|2484449|2484479|2484087|2484360|N|N|N|N|N| +2484453|AAAAAAAAFOIOFCAA|2090-02-05|2281|9919|761|2090|0|2|5|1|2090|761|9919|Sunday|2090Q1|N|N|N|2484449|2484479|2484088|2484361|N|N|N|N|N| +2484454|AAAAAAAAGOIOFCAA|2090-02-06|2281|9919|761|2090|1|2|6|1|2090|761|9919|Monday|2090Q1|N|N|N|2484449|2484479|2484089|2484362|N|N|N|N|N| +2484455|AAAAAAAAHOIOFCAA|2090-02-07|2281|9920|761|2090|2|2|7|1|2090|761|9920|Tuesday|2090Q1|N|N|N|2484449|2484479|2484090|2484363|N|N|N|N|N| +2484456|AAAAAAAAIOIOFCAA|2090-02-08|2281|9920|761|2090|3|2|8|1|2090|761|9920|Wednesday|2090Q1|N|N|N|2484449|2484479|2484091|2484364|N|N|N|N|N| +2484457|AAAAAAAAJOIOFCAA|2090-02-09|2281|9920|761|2090|4|2|9|1|2090|761|9920|Thursday|2090Q1|N|N|N|2484449|2484479|2484092|2484365|N|N|N|N|N| +2484458|AAAAAAAAKOIOFCAA|2090-02-10|2281|9920|761|2090|5|2|10|1|2090|761|9920|Friday|2090Q1|N|Y|N|2484449|2484479|2484093|2484366|N|N|N|N|N| +2484459|AAAAAAAALOIOFCAA|2090-02-11|2281|9920|761|2090|6|2|11|1|2090|761|9920|Saturday|2090Q1|N|Y|N|2484449|2484479|2484094|2484367|N|N|N|N|N| +2484460|AAAAAAAAMOIOFCAA|2090-02-12|2281|9920|761|2090|0|2|12|1|2090|761|9920|Sunday|2090Q1|N|N|N|2484449|2484479|2484095|2484368|N|N|N|N|N| +2484461|AAAAAAAANOIOFCAA|2090-02-13|2281|9920|761|2090|1|2|13|1|2090|761|9920|Monday|2090Q1|N|N|N|2484449|2484479|2484096|2484369|N|N|N|N|N| +2484462|AAAAAAAAOOIOFCAA|2090-02-14|2281|9921|761|2090|2|2|14|1|2090|761|9921|Tuesday|2090Q1|N|N|N|2484449|2484479|2484097|2484370|N|N|N|N|N| +2484463|AAAAAAAAPOIOFCAA|2090-02-15|2281|9921|761|2090|3|2|15|1|2090|761|9921|Wednesday|2090Q1|N|N|N|2484449|2484479|2484098|2484371|N|N|N|N|N| +2484464|AAAAAAAAAPIOFCAA|2090-02-16|2281|9921|761|2090|4|2|16|1|2090|761|9921|Thursday|2090Q1|N|N|N|2484449|2484479|2484099|2484372|N|N|N|N|N| +2484465|AAAAAAAABPIOFCAA|2090-02-17|2281|9921|761|2090|5|2|17|1|2090|761|9921|Friday|2090Q1|N|Y|N|2484449|2484479|2484100|2484373|N|N|N|N|N| +2484466|AAAAAAAACPIOFCAA|2090-02-18|2281|9921|761|2090|6|2|18|1|2090|761|9921|Saturday|2090Q1|N|Y|N|2484449|2484479|2484101|2484374|N|N|N|N|N| +2484467|AAAAAAAADPIOFCAA|2090-02-19|2281|9921|761|2090|0|2|19|1|2090|761|9921|Sunday|2090Q1|N|N|N|2484449|2484479|2484102|2484375|N|N|N|N|N| +2484468|AAAAAAAAEPIOFCAA|2090-02-20|2281|9921|761|2090|1|2|20|1|2090|761|9921|Monday|2090Q1|N|N|N|2484449|2484479|2484103|2484376|N|N|N|N|N| +2484469|AAAAAAAAFPIOFCAA|2090-02-21|2281|9922|761|2090|2|2|21|1|2090|761|9922|Tuesday|2090Q1|N|N|N|2484449|2484479|2484104|2484377|N|N|N|N|N| +2484470|AAAAAAAAGPIOFCAA|2090-02-22|2281|9922|761|2090|3|2|22|1|2090|761|9922|Wednesday|2090Q1|N|N|N|2484449|2484479|2484105|2484378|N|N|N|N|N| +2484471|AAAAAAAAHPIOFCAA|2090-02-23|2281|9922|761|2090|4|2|23|1|2090|761|9922|Thursday|2090Q1|N|N|N|2484449|2484479|2484106|2484379|N|N|N|N|N| +2484472|AAAAAAAAIPIOFCAA|2090-02-24|2281|9922|761|2090|5|2|24|1|2090|761|9922|Friday|2090Q1|N|Y|N|2484449|2484479|2484107|2484380|N|N|N|N|N| +2484473|AAAAAAAAJPIOFCAA|2090-02-25|2281|9922|761|2090|6|2|25|1|2090|761|9922|Saturday|2090Q1|N|Y|N|2484449|2484479|2484108|2484381|N|N|N|N|N| +2484474|AAAAAAAAKPIOFCAA|2090-02-26|2281|9922|761|2090|0|2|26|1|2090|761|9922|Sunday|2090Q1|N|N|N|2484449|2484479|2484109|2484382|N|N|N|N|N| +2484475|AAAAAAAALPIOFCAA|2090-02-27|2281|9922|761|2090|1|2|27|1|2090|761|9922|Monday|2090Q1|N|N|N|2484449|2484479|2484110|2484383|N|N|N|N|N| +2484476|AAAAAAAAMPIOFCAA|2090-02-28|2281|9923|761|2090|2|2|28|1|2090|761|9923|Tuesday|2090Q1|N|N|N|2484449|2484479|2484111|2484384|N|N|N|N|N| +2484477|AAAAAAAANPIOFCAA|2090-03-01|2282|9923|762|2090|3|3|1|1|2090|762|9923|Wednesday|2090Q1|N|N|N|2484477|2484535|2484112|2484385|N|N|N|N|N| +2484478|AAAAAAAAOPIOFCAA|2090-03-02|2282|9923|762|2090|4|3|2|1|2090|762|9923|Thursday|2090Q1|N|N|N|2484477|2484535|2484113|2484386|N|N|N|N|N| +2484479|AAAAAAAAPPIOFCAA|2090-03-03|2282|9923|762|2090|5|3|3|1|2090|762|9923|Friday|2090Q1|N|Y|N|2484477|2484535|2484114|2484387|N|N|N|N|N| +2484480|AAAAAAAAAAJOFCAA|2090-03-04|2282|9923|762|2090|6|3|4|1|2090|762|9923|Saturday|2090Q1|N|Y|N|2484477|2484535|2484115|2484388|N|N|N|N|N| +2484481|AAAAAAAABAJOFCAA|2090-03-05|2282|9923|762|2090|0|3|5|1|2090|762|9923|Sunday|2090Q1|N|N|N|2484477|2484535|2484116|2484389|N|N|N|N|N| +2484482|AAAAAAAACAJOFCAA|2090-03-06|2282|9923|762|2090|1|3|6|1|2090|762|9923|Monday|2090Q1|N|N|N|2484477|2484535|2484117|2484390|N|N|N|N|N| +2484483|AAAAAAAADAJOFCAA|2090-03-07|2282|9924|762|2090|2|3|7|1|2090|762|9924|Tuesday|2090Q1|N|N|N|2484477|2484535|2484118|2484391|N|N|N|N|N| +2484484|AAAAAAAAEAJOFCAA|2090-03-08|2282|9924|762|2090|3|3|8|1|2090|762|9924|Wednesday|2090Q1|N|N|N|2484477|2484535|2484119|2484392|N|N|N|N|N| +2484485|AAAAAAAAFAJOFCAA|2090-03-09|2282|9924|762|2090|4|3|9|1|2090|762|9924|Thursday|2090Q1|N|N|N|2484477|2484535|2484120|2484393|N|N|N|N|N| +2484486|AAAAAAAAGAJOFCAA|2090-03-10|2282|9924|762|2090|5|3|10|1|2090|762|9924|Friday|2090Q1|N|Y|N|2484477|2484535|2484121|2484394|N|N|N|N|N| +2484487|AAAAAAAAHAJOFCAA|2090-03-11|2282|9924|762|2090|6|3|11|1|2090|762|9924|Saturday|2090Q1|N|Y|N|2484477|2484535|2484122|2484395|N|N|N|N|N| +2484488|AAAAAAAAIAJOFCAA|2090-03-12|2282|9924|762|2090|0|3|12|1|2090|762|9924|Sunday|2090Q1|N|N|N|2484477|2484535|2484123|2484396|N|N|N|N|N| +2484489|AAAAAAAAJAJOFCAA|2090-03-13|2282|9924|762|2090|1|3|13|1|2090|762|9924|Monday|2090Q1|N|N|N|2484477|2484535|2484124|2484397|N|N|N|N|N| +2484490|AAAAAAAAKAJOFCAA|2090-03-14|2282|9925|762|2090|2|3|14|1|2090|762|9925|Tuesday|2090Q1|N|N|N|2484477|2484535|2484125|2484398|N|N|N|N|N| +2484491|AAAAAAAALAJOFCAA|2090-03-15|2282|9925|762|2090|3|3|15|1|2090|762|9925|Wednesday|2090Q1|N|N|N|2484477|2484535|2484126|2484399|N|N|N|N|N| +2484492|AAAAAAAAMAJOFCAA|2090-03-16|2282|9925|762|2090|4|3|16|1|2090|762|9925|Thursday|2090Q1|N|N|N|2484477|2484535|2484127|2484400|N|N|N|N|N| +2484493|AAAAAAAANAJOFCAA|2090-03-17|2282|9925|762|2090|5|3|17|1|2090|762|9925|Friday|2090Q1|N|Y|N|2484477|2484535|2484128|2484401|N|N|N|N|N| +2484494|AAAAAAAAOAJOFCAA|2090-03-18|2282|9925|762|2090|6|3|18|1|2090|762|9925|Saturday|2090Q1|N|Y|N|2484477|2484535|2484129|2484402|N|N|N|N|N| +2484495|AAAAAAAAPAJOFCAA|2090-03-19|2282|9925|762|2090|0|3|19|1|2090|762|9925|Sunday|2090Q1|N|N|N|2484477|2484535|2484130|2484403|N|N|N|N|N| +2484496|AAAAAAAAABJOFCAA|2090-03-20|2282|9925|762|2090|1|3|20|1|2090|762|9925|Monday|2090Q1|N|N|N|2484477|2484535|2484131|2484404|N|N|N|N|N| +2484497|AAAAAAAABBJOFCAA|2090-03-21|2282|9926|762|2090|2|3|21|1|2090|762|9926|Tuesday|2090Q1|N|N|N|2484477|2484535|2484132|2484405|N|N|N|N|N| +2484498|AAAAAAAACBJOFCAA|2090-03-22|2282|9926|762|2090|3|3|22|1|2090|762|9926|Wednesday|2090Q1|N|N|N|2484477|2484535|2484133|2484406|N|N|N|N|N| +2484499|AAAAAAAADBJOFCAA|2090-03-23|2282|9926|762|2090|4|3|23|1|2090|762|9926|Thursday|2090Q1|N|N|N|2484477|2484535|2484134|2484407|N|N|N|N|N| +2484500|AAAAAAAAEBJOFCAA|2090-03-24|2282|9926|762|2090|5|3|24|1|2090|762|9926|Friday|2090Q1|N|Y|N|2484477|2484535|2484135|2484408|N|N|N|N|N| +2484501|AAAAAAAAFBJOFCAA|2090-03-25|2282|9926|762|2090|6|3|25|1|2090|762|9926|Saturday|2090Q1|N|Y|N|2484477|2484535|2484136|2484409|N|N|N|N|N| +2484502|AAAAAAAAGBJOFCAA|2090-03-26|2282|9926|762|2090|0|3|26|1|2090|762|9926|Sunday|2090Q1|N|N|N|2484477|2484535|2484137|2484410|N|N|N|N|N| +2484503|AAAAAAAAHBJOFCAA|2090-03-27|2282|9926|762|2090|1|3|27|1|2090|762|9926|Monday|2090Q1|N|N|N|2484477|2484535|2484138|2484411|N|N|N|N|N| +2484504|AAAAAAAAIBJOFCAA|2090-03-28|2282|9927|762|2090|2|3|28|1|2090|762|9927|Tuesday|2090Q1|N|N|N|2484477|2484535|2484139|2484412|N|N|N|N|N| +2484505|AAAAAAAAJBJOFCAA|2090-03-29|2282|9927|762|2090|3|3|29|1|2090|762|9927|Wednesday|2090Q1|N|N|N|2484477|2484535|2484140|2484413|N|N|N|N|N| +2484506|AAAAAAAAKBJOFCAA|2090-03-30|2282|9927|762|2090|4|3|30|1|2090|762|9927|Thursday|2090Q1|N|N|N|2484477|2484535|2484141|2484414|N|N|N|N|N| +2484507|AAAAAAAALBJOFCAA|2090-03-31|2282|9927|762|2090|5|3|31|1|2090|762|9927|Friday|2090Q1|N|Y|N|2484477|2484535|2484142|2484415|N|N|N|N|N| +2484508|AAAAAAAAMBJOFCAA|2090-04-01|2283|9927|762|2090|6|4|1|1|2090|762|9927|Saturday|2090Q1|N|Y|N|2484508|2484597|2484143|2484418|N|N|N|N|N| +2484509|AAAAAAAANBJOFCAA|2090-04-02|2283|9927|762|2090|0|4|2|2|2090|762|9927|Sunday|2090Q2|N|N|N|2484508|2484597|2484144|2484419|N|N|N|N|N| +2484510|AAAAAAAAOBJOFCAA|2090-04-03|2283|9927|762|2090|1|4|3|2|2090|762|9927|Monday|2090Q2|N|N|N|2484508|2484597|2484145|2484420|N|N|N|N|N| +2484511|AAAAAAAAPBJOFCAA|2090-04-04|2283|9928|762|2090|2|4|4|2|2090|762|9928|Tuesday|2090Q2|N|N|N|2484508|2484597|2484146|2484421|N|N|N|N|N| +2484512|AAAAAAAAACJOFCAA|2090-04-05|2283|9928|762|2090|3|4|5|2|2090|762|9928|Wednesday|2090Q2|N|N|N|2484508|2484597|2484147|2484422|N|N|N|N|N| +2484513|AAAAAAAABCJOFCAA|2090-04-06|2283|9928|762|2090|4|4|6|2|2090|762|9928|Thursday|2090Q2|N|N|N|2484508|2484597|2484148|2484423|N|N|N|N|N| +2484514|AAAAAAAACCJOFCAA|2090-04-07|2283|9928|762|2090|5|4|7|2|2090|762|9928|Friday|2090Q2|N|Y|N|2484508|2484597|2484149|2484424|N|N|N|N|N| +2484515|AAAAAAAADCJOFCAA|2090-04-08|2283|9928|762|2090|6|4|8|2|2090|762|9928|Saturday|2090Q2|N|Y|N|2484508|2484597|2484150|2484425|N|N|N|N|N| +2484516|AAAAAAAAECJOFCAA|2090-04-09|2283|9928|762|2090|0|4|9|2|2090|762|9928|Sunday|2090Q2|N|N|N|2484508|2484597|2484151|2484426|N|N|N|N|N| +2484517|AAAAAAAAFCJOFCAA|2090-04-10|2283|9928|762|2090|1|4|10|2|2090|762|9928|Monday|2090Q2|N|N|N|2484508|2484597|2484152|2484427|N|N|N|N|N| +2484518|AAAAAAAAGCJOFCAA|2090-04-11|2283|9929|762|2090|2|4|11|2|2090|762|9929|Tuesday|2090Q2|N|N|N|2484508|2484597|2484153|2484428|N|N|N|N|N| +2484519|AAAAAAAAHCJOFCAA|2090-04-12|2283|9929|762|2090|3|4|12|2|2090|762|9929|Wednesday|2090Q2|N|N|N|2484508|2484597|2484154|2484429|N|N|N|N|N| +2484520|AAAAAAAAICJOFCAA|2090-04-13|2283|9929|762|2090|4|4|13|2|2090|762|9929|Thursday|2090Q2|N|N|N|2484508|2484597|2484155|2484430|N|N|N|N|N| +2484521|AAAAAAAAJCJOFCAA|2090-04-14|2283|9929|762|2090|5|4|14|2|2090|762|9929|Friday|2090Q2|N|Y|N|2484508|2484597|2484156|2484431|N|N|N|N|N| +2484522|AAAAAAAAKCJOFCAA|2090-04-15|2283|9929|762|2090|6|4|15|2|2090|762|9929|Saturday|2090Q2|N|Y|N|2484508|2484597|2484157|2484432|N|N|N|N|N| +2484523|AAAAAAAALCJOFCAA|2090-04-16|2283|9929|762|2090|0|4|16|2|2090|762|9929|Sunday|2090Q2|N|N|N|2484508|2484597|2484158|2484433|N|N|N|N|N| +2484524|AAAAAAAAMCJOFCAA|2090-04-17|2283|9929|762|2090|1|4|17|2|2090|762|9929|Monday|2090Q2|N|N|N|2484508|2484597|2484159|2484434|N|N|N|N|N| +2484525|AAAAAAAANCJOFCAA|2090-04-18|2283|9930|762|2090|2|4|18|2|2090|762|9930|Tuesday|2090Q2|N|N|N|2484508|2484597|2484160|2484435|N|N|N|N|N| +2484526|AAAAAAAAOCJOFCAA|2090-04-19|2283|9930|762|2090|3|4|19|2|2090|762|9930|Wednesday|2090Q2|N|N|N|2484508|2484597|2484161|2484436|N|N|N|N|N| +2484527|AAAAAAAAPCJOFCAA|2090-04-20|2283|9930|762|2090|4|4|20|2|2090|762|9930|Thursday|2090Q2|N|N|N|2484508|2484597|2484162|2484437|N|N|N|N|N| +2484528|AAAAAAAAADJOFCAA|2090-04-21|2283|9930|762|2090|5|4|21|2|2090|762|9930|Friday|2090Q2|N|Y|N|2484508|2484597|2484163|2484438|N|N|N|N|N| +2484529|AAAAAAAABDJOFCAA|2090-04-22|2283|9930|762|2090|6|4|22|2|2090|762|9930|Saturday|2090Q2|N|Y|N|2484508|2484597|2484164|2484439|N|N|N|N|N| +2484530|AAAAAAAACDJOFCAA|2090-04-23|2283|9930|762|2090|0|4|23|2|2090|762|9930|Sunday|2090Q2|N|N|N|2484508|2484597|2484165|2484440|N|N|N|N|N| +2484531|AAAAAAAADDJOFCAA|2090-04-24|2283|9930|762|2090|1|4|24|2|2090|762|9930|Monday|2090Q2|N|N|N|2484508|2484597|2484166|2484441|N|N|N|N|N| +2484532|AAAAAAAAEDJOFCAA|2090-04-25|2283|9931|762|2090|2|4|25|2|2090|762|9931|Tuesday|2090Q2|N|N|N|2484508|2484597|2484167|2484442|N|N|N|N|N| +2484533|AAAAAAAAFDJOFCAA|2090-04-26|2283|9931|762|2090|3|4|26|2|2090|762|9931|Wednesday|2090Q2|N|N|N|2484508|2484597|2484168|2484443|N|N|N|N|N| +2484534|AAAAAAAAGDJOFCAA|2090-04-27|2283|9931|762|2090|4|4|27|2|2090|762|9931|Thursday|2090Q2|N|N|N|2484508|2484597|2484169|2484444|N|N|N|N|N| +2484535|AAAAAAAAHDJOFCAA|2090-04-28|2283|9931|762|2090|5|4|28|2|2090|762|9931|Friday|2090Q2|N|Y|N|2484508|2484597|2484170|2484445|N|N|N|N|N| +2484536|AAAAAAAAIDJOFCAA|2090-04-29|2283|9931|762|2090|6|4|29|2|2090|762|9931|Saturday|2090Q2|N|Y|N|2484508|2484597|2484171|2484446|N|N|N|N|N| +2484537|AAAAAAAAJDJOFCAA|2090-04-30|2283|9931|762|2090|0|4|30|2|2090|762|9931|Sunday|2090Q2|N|N|N|2484508|2484597|2484172|2484447|N|N|N|N|N| +2484538|AAAAAAAAKDJOFCAA|2090-05-01|2284|9931|762|2090|1|5|1|2|2090|762|9931|Monday|2090Q2|N|N|N|2484538|2484657|2484173|2484448|N|N|N|N|N| +2484539|AAAAAAAALDJOFCAA|2090-05-02|2284|9932|762|2090|2|5|2|2|2090|762|9932|Tuesday|2090Q2|N|N|N|2484538|2484657|2484174|2484449|N|N|N|N|N| +2484540|AAAAAAAAMDJOFCAA|2090-05-03|2284|9932|762|2090|3|5|3|2|2090|762|9932|Wednesday|2090Q2|N|N|N|2484538|2484657|2484175|2484450|N|N|N|N|N| +2484541|AAAAAAAANDJOFCAA|2090-05-04|2284|9932|762|2090|4|5|4|2|2090|762|9932|Thursday|2090Q2|N|N|N|2484538|2484657|2484176|2484451|N|N|N|N|N| +2484542|AAAAAAAAODJOFCAA|2090-05-05|2284|9932|762|2090|5|5|5|2|2090|762|9932|Friday|2090Q2|N|Y|N|2484538|2484657|2484177|2484452|N|N|N|N|N| +2484543|AAAAAAAAPDJOFCAA|2090-05-06|2284|9932|762|2090|6|5|6|2|2090|762|9932|Saturday|2090Q2|N|Y|N|2484538|2484657|2484178|2484453|N|N|N|N|N| +2484544|AAAAAAAAAEJOFCAA|2090-05-07|2284|9932|762|2090|0|5|7|2|2090|762|9932|Sunday|2090Q2|N|N|N|2484538|2484657|2484179|2484454|N|N|N|N|N| +2484545|AAAAAAAABEJOFCAA|2090-05-08|2284|9932|762|2090|1|5|8|2|2090|762|9932|Monday|2090Q2|N|N|N|2484538|2484657|2484180|2484455|N|N|N|N|N| +2484546|AAAAAAAACEJOFCAA|2090-05-09|2284|9933|762|2090|2|5|9|2|2090|762|9933|Tuesday|2090Q2|N|N|N|2484538|2484657|2484181|2484456|N|N|N|N|N| +2484547|AAAAAAAADEJOFCAA|2090-05-10|2284|9933|762|2090|3|5|10|2|2090|762|9933|Wednesday|2090Q2|N|N|N|2484538|2484657|2484182|2484457|N|N|N|N|N| +2484548|AAAAAAAAEEJOFCAA|2090-05-11|2284|9933|762|2090|4|5|11|2|2090|762|9933|Thursday|2090Q2|N|N|N|2484538|2484657|2484183|2484458|N|N|N|N|N| +2484549|AAAAAAAAFEJOFCAA|2090-05-12|2284|9933|762|2090|5|5|12|2|2090|762|9933|Friday|2090Q2|N|Y|N|2484538|2484657|2484184|2484459|N|N|N|N|N| +2484550|AAAAAAAAGEJOFCAA|2090-05-13|2284|9933|762|2090|6|5|13|2|2090|762|9933|Saturday|2090Q2|N|Y|N|2484538|2484657|2484185|2484460|N|N|N|N|N| +2484551|AAAAAAAAHEJOFCAA|2090-05-14|2284|9933|762|2090|0|5|14|2|2090|762|9933|Sunday|2090Q2|N|N|N|2484538|2484657|2484186|2484461|N|N|N|N|N| +2484552|AAAAAAAAIEJOFCAA|2090-05-15|2284|9933|762|2090|1|5|15|2|2090|762|9933|Monday|2090Q2|N|N|N|2484538|2484657|2484187|2484462|N|N|N|N|N| +2484553|AAAAAAAAJEJOFCAA|2090-05-16|2284|9934|762|2090|2|5|16|2|2090|762|9934|Tuesday|2090Q2|N|N|N|2484538|2484657|2484188|2484463|N|N|N|N|N| +2484554|AAAAAAAAKEJOFCAA|2090-05-17|2284|9934|762|2090|3|5|17|2|2090|762|9934|Wednesday|2090Q2|N|N|N|2484538|2484657|2484189|2484464|N|N|N|N|N| +2484555|AAAAAAAALEJOFCAA|2090-05-18|2284|9934|762|2090|4|5|18|2|2090|762|9934|Thursday|2090Q2|N|N|N|2484538|2484657|2484190|2484465|N|N|N|N|N| +2484556|AAAAAAAAMEJOFCAA|2090-05-19|2284|9934|762|2090|5|5|19|2|2090|762|9934|Friday|2090Q2|N|Y|N|2484538|2484657|2484191|2484466|N|N|N|N|N| +2484557|AAAAAAAANEJOFCAA|2090-05-20|2284|9934|762|2090|6|5|20|2|2090|762|9934|Saturday|2090Q2|N|Y|N|2484538|2484657|2484192|2484467|N|N|N|N|N| +2484558|AAAAAAAAOEJOFCAA|2090-05-21|2284|9934|762|2090|0|5|21|2|2090|762|9934|Sunday|2090Q2|N|N|N|2484538|2484657|2484193|2484468|N|N|N|N|N| +2484559|AAAAAAAAPEJOFCAA|2090-05-22|2284|9934|762|2090|1|5|22|2|2090|762|9934|Monday|2090Q2|N|N|N|2484538|2484657|2484194|2484469|N|N|N|N|N| +2484560|AAAAAAAAAFJOFCAA|2090-05-23|2284|9935|762|2090|2|5|23|2|2090|762|9935|Tuesday|2090Q2|N|N|N|2484538|2484657|2484195|2484470|N|N|N|N|N| +2484561|AAAAAAAABFJOFCAA|2090-05-24|2284|9935|762|2090|3|5|24|2|2090|762|9935|Wednesday|2090Q2|N|N|N|2484538|2484657|2484196|2484471|N|N|N|N|N| +2484562|AAAAAAAACFJOFCAA|2090-05-25|2284|9935|762|2090|4|5|25|2|2090|762|9935|Thursday|2090Q2|N|N|N|2484538|2484657|2484197|2484472|N|N|N|N|N| +2484563|AAAAAAAADFJOFCAA|2090-05-26|2284|9935|762|2090|5|5|26|2|2090|762|9935|Friday|2090Q2|N|Y|N|2484538|2484657|2484198|2484473|N|N|N|N|N| +2484564|AAAAAAAAEFJOFCAA|2090-05-27|2284|9935|762|2090|6|5|27|2|2090|762|9935|Saturday|2090Q2|N|Y|N|2484538|2484657|2484199|2484474|N|N|N|N|N| +2484565|AAAAAAAAFFJOFCAA|2090-05-28|2284|9935|762|2090|0|5|28|2|2090|762|9935|Sunday|2090Q2|N|N|N|2484538|2484657|2484200|2484475|N|N|N|N|N| +2484566|AAAAAAAAGFJOFCAA|2090-05-29|2284|9935|762|2090|1|5|29|2|2090|762|9935|Monday|2090Q2|N|N|N|2484538|2484657|2484201|2484476|N|N|N|N|N| +2484567|AAAAAAAAHFJOFCAA|2090-05-30|2284|9936|762|2090|2|5|30|2|2090|762|9936|Tuesday|2090Q2|N|N|N|2484538|2484657|2484202|2484477|N|N|N|N|N| +2484568|AAAAAAAAIFJOFCAA|2090-05-31|2284|9936|762|2090|3|5|31|2|2090|762|9936|Wednesday|2090Q2|N|N|N|2484538|2484657|2484203|2484478|N|N|N|N|N| +2484569|AAAAAAAAJFJOFCAA|2090-06-01|2285|9936|763|2090|4|6|1|2|2090|763|9936|Thursday|2090Q2|N|N|N|2484569|2484719|2484204|2484479|N|N|N|N|N| +2484570|AAAAAAAAKFJOFCAA|2090-06-02|2285|9936|763|2090|5|6|2|2|2090|763|9936|Friday|2090Q2|N|Y|N|2484569|2484719|2484205|2484480|N|N|N|N|N| +2484571|AAAAAAAALFJOFCAA|2090-06-03|2285|9936|763|2090|6|6|3|2|2090|763|9936|Saturday|2090Q2|N|Y|N|2484569|2484719|2484206|2484481|N|N|N|N|N| +2484572|AAAAAAAAMFJOFCAA|2090-06-04|2285|9936|763|2090|0|6|4|2|2090|763|9936|Sunday|2090Q2|N|N|N|2484569|2484719|2484207|2484482|N|N|N|N|N| +2484573|AAAAAAAANFJOFCAA|2090-06-05|2285|9936|763|2090|1|6|5|2|2090|763|9936|Monday|2090Q2|N|N|N|2484569|2484719|2484208|2484483|N|N|N|N|N| +2484574|AAAAAAAAOFJOFCAA|2090-06-06|2285|9937|763|2090|2|6|6|2|2090|763|9937|Tuesday|2090Q2|N|N|N|2484569|2484719|2484209|2484484|N|N|N|N|N| +2484575|AAAAAAAAPFJOFCAA|2090-06-07|2285|9937|763|2090|3|6|7|2|2090|763|9937|Wednesday|2090Q2|N|N|N|2484569|2484719|2484210|2484485|N|N|N|N|N| +2484576|AAAAAAAAAGJOFCAA|2090-06-08|2285|9937|763|2090|4|6|8|2|2090|763|9937|Thursday|2090Q2|N|N|N|2484569|2484719|2484211|2484486|N|N|N|N|N| +2484577|AAAAAAAABGJOFCAA|2090-06-09|2285|9937|763|2090|5|6|9|2|2090|763|9937|Friday|2090Q2|N|Y|N|2484569|2484719|2484212|2484487|N|N|N|N|N| +2484578|AAAAAAAACGJOFCAA|2090-06-10|2285|9937|763|2090|6|6|10|2|2090|763|9937|Saturday|2090Q2|N|Y|N|2484569|2484719|2484213|2484488|N|N|N|N|N| +2484579|AAAAAAAADGJOFCAA|2090-06-11|2285|9937|763|2090|0|6|11|2|2090|763|9937|Sunday|2090Q2|N|N|N|2484569|2484719|2484214|2484489|N|N|N|N|N| +2484580|AAAAAAAAEGJOFCAA|2090-06-12|2285|9937|763|2090|1|6|12|2|2090|763|9937|Monday|2090Q2|N|N|N|2484569|2484719|2484215|2484490|N|N|N|N|N| +2484581|AAAAAAAAFGJOFCAA|2090-06-13|2285|9938|763|2090|2|6|13|2|2090|763|9938|Tuesday|2090Q2|N|N|N|2484569|2484719|2484216|2484491|N|N|N|N|N| +2484582|AAAAAAAAGGJOFCAA|2090-06-14|2285|9938|763|2090|3|6|14|2|2090|763|9938|Wednesday|2090Q2|N|N|N|2484569|2484719|2484217|2484492|N|N|N|N|N| +2484583|AAAAAAAAHGJOFCAA|2090-06-15|2285|9938|763|2090|4|6|15|2|2090|763|9938|Thursday|2090Q2|N|N|N|2484569|2484719|2484218|2484493|N|N|N|N|N| +2484584|AAAAAAAAIGJOFCAA|2090-06-16|2285|9938|763|2090|5|6|16|2|2090|763|9938|Friday|2090Q2|N|Y|N|2484569|2484719|2484219|2484494|N|N|N|N|N| +2484585|AAAAAAAAJGJOFCAA|2090-06-17|2285|9938|763|2090|6|6|17|2|2090|763|9938|Saturday|2090Q2|N|Y|N|2484569|2484719|2484220|2484495|N|N|N|N|N| +2484586|AAAAAAAAKGJOFCAA|2090-06-18|2285|9938|763|2090|0|6|18|2|2090|763|9938|Sunday|2090Q2|N|N|N|2484569|2484719|2484221|2484496|N|N|N|N|N| +2484587|AAAAAAAALGJOFCAA|2090-06-19|2285|9938|763|2090|1|6|19|2|2090|763|9938|Monday|2090Q2|N|N|N|2484569|2484719|2484222|2484497|N|N|N|N|N| +2484588|AAAAAAAAMGJOFCAA|2090-06-20|2285|9939|763|2090|2|6|20|2|2090|763|9939|Tuesday|2090Q2|N|N|N|2484569|2484719|2484223|2484498|N|N|N|N|N| +2484589|AAAAAAAANGJOFCAA|2090-06-21|2285|9939|763|2090|3|6|21|2|2090|763|9939|Wednesday|2090Q2|N|N|N|2484569|2484719|2484224|2484499|N|N|N|N|N| +2484590|AAAAAAAAOGJOFCAA|2090-06-22|2285|9939|763|2090|4|6|22|2|2090|763|9939|Thursday|2090Q2|N|N|N|2484569|2484719|2484225|2484500|N|N|N|N|N| +2484591|AAAAAAAAPGJOFCAA|2090-06-23|2285|9939|763|2090|5|6|23|2|2090|763|9939|Friday|2090Q2|N|Y|N|2484569|2484719|2484226|2484501|N|N|N|N|N| +2484592|AAAAAAAAAHJOFCAA|2090-06-24|2285|9939|763|2090|6|6|24|2|2090|763|9939|Saturday|2090Q2|N|Y|N|2484569|2484719|2484227|2484502|N|N|N|N|N| +2484593|AAAAAAAABHJOFCAA|2090-06-25|2285|9939|763|2090|0|6|25|2|2090|763|9939|Sunday|2090Q2|N|N|N|2484569|2484719|2484228|2484503|N|N|N|N|N| +2484594|AAAAAAAACHJOFCAA|2090-06-26|2285|9939|763|2090|1|6|26|2|2090|763|9939|Monday|2090Q2|N|N|N|2484569|2484719|2484229|2484504|N|N|N|N|N| +2484595|AAAAAAAADHJOFCAA|2090-06-27|2285|9940|763|2090|2|6|27|2|2090|763|9940|Tuesday|2090Q2|N|N|N|2484569|2484719|2484230|2484505|N|N|N|N|N| +2484596|AAAAAAAAEHJOFCAA|2090-06-28|2285|9940|763|2090|3|6|28|2|2090|763|9940|Wednesday|2090Q2|N|N|N|2484569|2484719|2484231|2484506|N|N|N|N|N| +2484597|AAAAAAAAFHJOFCAA|2090-06-29|2285|9940|763|2090|4|6|29|2|2090|763|9940|Thursday|2090Q2|N|N|N|2484569|2484719|2484232|2484507|N|N|N|N|N| +2484598|AAAAAAAAGHJOFCAA|2090-06-30|2285|9940|763|2090|5|6|30|2|2090|763|9940|Friday|2090Q2|N|Y|N|2484569|2484719|2484233|2484508|N|N|N|N|N| +2484599|AAAAAAAAHHJOFCAA|2090-07-01|2286|9940|763|2090|6|7|1|2|2090|763|9940|Saturday|2090Q2|N|Y|N|2484599|2484779|2484234|2484508|N|N|N|N|N| +2484600|AAAAAAAAIHJOFCAA|2090-07-02|2286|9940|763|2090|0|7|2|3|2090|763|9940|Sunday|2090Q3|N|N|N|2484599|2484779|2484235|2484509|N|N|N|N|N| +2484601|AAAAAAAAJHJOFCAA|2090-07-03|2286|9940|763|2090|1|7|3|3|2090|763|9940|Monday|2090Q3|N|N|N|2484599|2484779|2484236|2484510|N|N|N|N|N| +2484602|AAAAAAAAKHJOFCAA|2090-07-04|2286|9941|763|2090|2|7|4|3|2090|763|9941|Tuesday|2090Q3|N|N|N|2484599|2484779|2484237|2484511|N|N|N|N|N| +2484603|AAAAAAAALHJOFCAA|2090-07-05|2286|9941|763|2090|3|7|5|3|2090|763|9941|Wednesday|2090Q3|Y|N|N|2484599|2484779|2484238|2484512|N|N|N|N|N| +2484604|AAAAAAAAMHJOFCAA|2090-07-06|2286|9941|763|2090|4|7|6|3|2090|763|9941|Thursday|2090Q3|N|N|Y|2484599|2484779|2484239|2484513|N|N|N|N|N| +2484605|AAAAAAAANHJOFCAA|2090-07-07|2286|9941|763|2090|5|7|7|3|2090|763|9941|Friday|2090Q3|N|Y|N|2484599|2484779|2484240|2484514|N|N|N|N|N| +2484606|AAAAAAAAOHJOFCAA|2090-07-08|2286|9941|763|2090|6|7|8|3|2090|763|9941|Saturday|2090Q3|N|Y|N|2484599|2484779|2484241|2484515|N|N|N|N|N| +2484607|AAAAAAAAPHJOFCAA|2090-07-09|2286|9941|763|2090|0|7|9|3|2090|763|9941|Sunday|2090Q3|N|N|N|2484599|2484779|2484242|2484516|N|N|N|N|N| +2484608|AAAAAAAAAIJOFCAA|2090-07-10|2286|9941|763|2090|1|7|10|3|2090|763|9941|Monday|2090Q3|N|N|N|2484599|2484779|2484243|2484517|N|N|N|N|N| +2484609|AAAAAAAABIJOFCAA|2090-07-11|2286|9942|763|2090|2|7|11|3|2090|763|9942|Tuesday|2090Q3|N|N|N|2484599|2484779|2484244|2484518|N|N|N|N|N| +2484610|AAAAAAAACIJOFCAA|2090-07-12|2286|9942|763|2090|3|7|12|3|2090|763|9942|Wednesday|2090Q3|N|N|N|2484599|2484779|2484245|2484519|N|N|N|N|N| +2484611|AAAAAAAADIJOFCAA|2090-07-13|2286|9942|763|2090|4|7|13|3|2090|763|9942|Thursday|2090Q3|N|N|N|2484599|2484779|2484246|2484520|N|N|N|N|N| +2484612|AAAAAAAAEIJOFCAA|2090-07-14|2286|9942|763|2090|5|7|14|3|2090|763|9942|Friday|2090Q3|N|Y|N|2484599|2484779|2484247|2484521|N|N|N|N|N| +2484613|AAAAAAAAFIJOFCAA|2090-07-15|2286|9942|763|2090|6|7|15|3|2090|763|9942|Saturday|2090Q3|N|Y|N|2484599|2484779|2484248|2484522|N|N|N|N|N| +2484614|AAAAAAAAGIJOFCAA|2090-07-16|2286|9942|763|2090|0|7|16|3|2090|763|9942|Sunday|2090Q3|N|N|N|2484599|2484779|2484249|2484523|N|N|N|N|N| +2484615|AAAAAAAAHIJOFCAA|2090-07-17|2286|9942|763|2090|1|7|17|3|2090|763|9942|Monday|2090Q3|N|N|N|2484599|2484779|2484250|2484524|N|N|N|N|N| +2484616|AAAAAAAAIIJOFCAA|2090-07-18|2286|9943|763|2090|2|7|18|3|2090|763|9943|Tuesday|2090Q3|N|N|N|2484599|2484779|2484251|2484525|N|N|N|N|N| +2484617|AAAAAAAAJIJOFCAA|2090-07-19|2286|9943|763|2090|3|7|19|3|2090|763|9943|Wednesday|2090Q3|N|N|N|2484599|2484779|2484252|2484526|N|N|N|N|N| +2484618|AAAAAAAAKIJOFCAA|2090-07-20|2286|9943|763|2090|4|7|20|3|2090|763|9943|Thursday|2090Q3|N|N|N|2484599|2484779|2484253|2484527|N|N|N|N|N| +2484619|AAAAAAAALIJOFCAA|2090-07-21|2286|9943|763|2090|5|7|21|3|2090|763|9943|Friday|2090Q3|N|Y|N|2484599|2484779|2484254|2484528|N|N|N|N|N| +2484620|AAAAAAAAMIJOFCAA|2090-07-22|2286|9943|763|2090|6|7|22|3|2090|763|9943|Saturday|2090Q3|N|Y|N|2484599|2484779|2484255|2484529|N|N|N|N|N| +2484621|AAAAAAAANIJOFCAA|2090-07-23|2286|9943|763|2090|0|7|23|3|2090|763|9943|Sunday|2090Q3|N|N|N|2484599|2484779|2484256|2484530|N|N|N|N|N| +2484622|AAAAAAAAOIJOFCAA|2090-07-24|2286|9943|763|2090|1|7|24|3|2090|763|9943|Monday|2090Q3|N|N|N|2484599|2484779|2484257|2484531|N|N|N|N|N| +2484623|AAAAAAAAPIJOFCAA|2090-07-25|2286|9944|763|2090|2|7|25|3|2090|763|9944|Tuesday|2090Q3|N|N|N|2484599|2484779|2484258|2484532|N|N|N|N|N| +2484624|AAAAAAAAAJJOFCAA|2090-07-26|2286|9944|763|2090|3|7|26|3|2090|763|9944|Wednesday|2090Q3|N|N|N|2484599|2484779|2484259|2484533|N|N|N|N|N| +2484625|AAAAAAAABJJOFCAA|2090-07-27|2286|9944|763|2090|4|7|27|3|2090|763|9944|Thursday|2090Q3|N|N|N|2484599|2484779|2484260|2484534|N|N|N|N|N| +2484626|AAAAAAAACJJOFCAA|2090-07-28|2286|9944|763|2090|5|7|28|3|2090|763|9944|Friday|2090Q3|N|Y|N|2484599|2484779|2484261|2484535|N|N|N|N|N| +2484627|AAAAAAAADJJOFCAA|2090-07-29|2286|9944|763|2090|6|7|29|3|2090|763|9944|Saturday|2090Q3|N|Y|N|2484599|2484779|2484262|2484536|N|N|N|N|N| +2484628|AAAAAAAAEJJOFCAA|2090-07-30|2286|9944|763|2090|0|7|30|3|2090|763|9944|Sunday|2090Q3|N|N|N|2484599|2484779|2484263|2484537|N|N|N|N|N| +2484629|AAAAAAAAFJJOFCAA|2090-07-31|2286|9944|763|2090|1|7|31|3|2090|763|9944|Monday|2090Q3|N|N|N|2484599|2484779|2484264|2484538|N|N|N|N|N| +2484630|AAAAAAAAGJJOFCAA|2090-08-01|2287|9945|763|2090|2|8|1|3|2090|763|9945|Tuesday|2090Q3|N|N|N|2484630|2484841|2484265|2484539|N|N|N|N|N| +2484631|AAAAAAAAHJJOFCAA|2090-08-02|2287|9945|763|2090|3|8|2|3|2090|763|9945|Wednesday|2090Q3|N|N|N|2484630|2484841|2484266|2484540|N|N|N|N|N| +2484632|AAAAAAAAIJJOFCAA|2090-08-03|2287|9945|763|2090|4|8|3|3|2090|763|9945|Thursday|2090Q3|N|N|N|2484630|2484841|2484267|2484541|N|N|N|N|N| +2484633|AAAAAAAAJJJOFCAA|2090-08-04|2287|9945|763|2090|5|8|4|3|2090|763|9945|Friday|2090Q3|N|Y|N|2484630|2484841|2484268|2484542|N|N|N|N|N| +2484634|AAAAAAAAKJJOFCAA|2090-08-05|2287|9945|763|2090|6|8|5|3|2090|763|9945|Saturday|2090Q3|N|Y|N|2484630|2484841|2484269|2484543|N|N|N|N|N| +2484635|AAAAAAAALJJOFCAA|2090-08-06|2287|9945|763|2090|0|8|6|3|2090|763|9945|Sunday|2090Q3|N|N|N|2484630|2484841|2484270|2484544|N|N|N|N|N| +2484636|AAAAAAAAMJJOFCAA|2090-08-07|2287|9945|763|2090|1|8|7|3|2090|763|9945|Monday|2090Q3|N|N|N|2484630|2484841|2484271|2484545|N|N|N|N|N| +2484637|AAAAAAAANJJOFCAA|2090-08-08|2287|9946|763|2090|2|8|8|3|2090|763|9946|Tuesday|2090Q3|N|N|N|2484630|2484841|2484272|2484546|N|N|N|N|N| +2484638|AAAAAAAAOJJOFCAA|2090-08-09|2287|9946|763|2090|3|8|9|3|2090|763|9946|Wednesday|2090Q3|N|N|N|2484630|2484841|2484273|2484547|N|N|N|N|N| +2484639|AAAAAAAAPJJOFCAA|2090-08-10|2287|9946|763|2090|4|8|10|3|2090|763|9946|Thursday|2090Q3|N|N|N|2484630|2484841|2484274|2484548|N|N|N|N|N| +2484640|AAAAAAAAAKJOFCAA|2090-08-11|2287|9946|763|2090|5|8|11|3|2090|763|9946|Friday|2090Q3|N|Y|N|2484630|2484841|2484275|2484549|N|N|N|N|N| +2484641|AAAAAAAABKJOFCAA|2090-08-12|2287|9946|763|2090|6|8|12|3|2090|763|9946|Saturday|2090Q3|N|Y|N|2484630|2484841|2484276|2484550|N|N|N|N|N| +2484642|AAAAAAAACKJOFCAA|2090-08-13|2287|9946|763|2090|0|8|13|3|2090|763|9946|Sunday|2090Q3|N|N|N|2484630|2484841|2484277|2484551|N|N|N|N|N| +2484643|AAAAAAAADKJOFCAA|2090-08-14|2287|9946|763|2090|1|8|14|3|2090|763|9946|Monday|2090Q3|N|N|N|2484630|2484841|2484278|2484552|N|N|N|N|N| +2484644|AAAAAAAAEKJOFCAA|2090-08-15|2287|9947|763|2090|2|8|15|3|2090|763|9947|Tuesday|2090Q3|N|N|N|2484630|2484841|2484279|2484553|N|N|N|N|N| +2484645|AAAAAAAAFKJOFCAA|2090-08-16|2287|9947|763|2090|3|8|16|3|2090|763|9947|Wednesday|2090Q3|N|N|N|2484630|2484841|2484280|2484554|N|N|N|N|N| +2484646|AAAAAAAAGKJOFCAA|2090-08-17|2287|9947|763|2090|4|8|17|3|2090|763|9947|Thursday|2090Q3|N|N|N|2484630|2484841|2484281|2484555|N|N|N|N|N| +2484647|AAAAAAAAHKJOFCAA|2090-08-18|2287|9947|763|2090|5|8|18|3|2090|763|9947|Friday|2090Q3|N|Y|N|2484630|2484841|2484282|2484556|N|N|N|N|N| +2484648|AAAAAAAAIKJOFCAA|2090-08-19|2287|9947|763|2090|6|8|19|3|2090|763|9947|Saturday|2090Q3|N|Y|N|2484630|2484841|2484283|2484557|N|N|N|N|N| +2484649|AAAAAAAAJKJOFCAA|2090-08-20|2287|9947|763|2090|0|8|20|3|2090|763|9947|Sunday|2090Q3|N|N|N|2484630|2484841|2484284|2484558|N|N|N|N|N| +2484650|AAAAAAAAKKJOFCAA|2090-08-21|2287|9947|763|2090|1|8|21|3|2090|763|9947|Monday|2090Q3|N|N|N|2484630|2484841|2484285|2484559|N|N|N|N|N| +2484651|AAAAAAAALKJOFCAA|2090-08-22|2287|9948|763|2090|2|8|22|3|2090|763|9948|Tuesday|2090Q3|N|N|N|2484630|2484841|2484286|2484560|N|N|N|N|N| +2484652|AAAAAAAAMKJOFCAA|2090-08-23|2287|9948|763|2090|3|8|23|3|2090|763|9948|Wednesday|2090Q3|N|N|N|2484630|2484841|2484287|2484561|N|N|N|N|N| +2484653|AAAAAAAANKJOFCAA|2090-08-24|2287|9948|763|2090|4|8|24|3|2090|763|9948|Thursday|2090Q3|N|N|N|2484630|2484841|2484288|2484562|N|N|N|N|N| +2484654|AAAAAAAAOKJOFCAA|2090-08-25|2287|9948|763|2090|5|8|25|3|2090|763|9948|Friday|2090Q3|N|Y|N|2484630|2484841|2484289|2484563|N|N|N|N|N| +2484655|AAAAAAAAPKJOFCAA|2090-08-26|2287|9948|763|2090|6|8|26|3|2090|763|9948|Saturday|2090Q3|N|Y|N|2484630|2484841|2484290|2484564|N|N|N|N|N| +2484656|AAAAAAAAALJOFCAA|2090-08-27|2287|9948|763|2090|0|8|27|3|2090|763|9948|Sunday|2090Q3|N|N|N|2484630|2484841|2484291|2484565|N|N|N|N|N| +2484657|AAAAAAAABLJOFCAA|2090-08-28|2287|9948|763|2090|1|8|28|3|2090|763|9948|Monday|2090Q3|N|N|N|2484630|2484841|2484292|2484566|N|N|N|N|N| +2484658|AAAAAAAACLJOFCAA|2090-08-29|2287|9949|763|2090|2|8|29|3|2090|763|9949|Tuesday|2090Q3|N|N|N|2484630|2484841|2484293|2484567|N|N|N|N|N| +2484659|AAAAAAAADLJOFCAA|2090-08-30|2287|9949|763|2090|3|8|30|3|2090|763|9949|Wednesday|2090Q3|N|N|N|2484630|2484841|2484294|2484568|N|N|N|N|N| +2484660|AAAAAAAAELJOFCAA|2090-08-31|2287|9949|763|2090|4|8|31|3|2090|763|9949|Thursday|2090Q3|N|N|N|2484630|2484841|2484295|2484569|N|N|N|N|N| +2484661|AAAAAAAAFLJOFCAA|2090-09-01|2288|9949|764|2090|5|9|1|3|2090|764|9949|Friday|2090Q3|N|Y|N|2484661|2484903|2484296|2484570|N|N|N|N|N| +2484662|AAAAAAAAGLJOFCAA|2090-09-02|2288|9949|764|2090|6|9|2|3|2090|764|9949|Saturday|2090Q3|N|Y|N|2484661|2484903|2484297|2484571|N|N|N|N|N| +2484663|AAAAAAAAHLJOFCAA|2090-09-03|2288|9949|764|2090|0|9|3|3|2090|764|9949|Sunday|2090Q3|N|N|N|2484661|2484903|2484298|2484572|N|N|N|N|N| +2484664|AAAAAAAAILJOFCAA|2090-09-04|2288|9949|764|2090|1|9|4|3|2090|764|9949|Monday|2090Q3|N|N|N|2484661|2484903|2484299|2484573|N|N|N|N|N| +2484665|AAAAAAAAJLJOFCAA|2090-09-05|2288|9950|764|2090|2|9|5|3|2090|764|9950|Tuesday|2090Q3|N|N|N|2484661|2484903|2484300|2484574|N|N|N|N|N| +2484666|AAAAAAAAKLJOFCAA|2090-09-06|2288|9950|764|2090|3|9|6|3|2090|764|9950|Wednesday|2090Q3|N|N|N|2484661|2484903|2484301|2484575|N|N|N|N|N| +2484667|AAAAAAAALLJOFCAA|2090-09-07|2288|9950|764|2090|4|9|7|3|2090|764|9950|Thursday|2090Q3|N|N|N|2484661|2484903|2484302|2484576|N|N|N|N|N| +2484668|AAAAAAAAMLJOFCAA|2090-09-08|2288|9950|764|2090|5|9|8|3|2090|764|9950|Friday|2090Q3|N|Y|N|2484661|2484903|2484303|2484577|N|N|N|N|N| +2484669|AAAAAAAANLJOFCAA|2090-09-09|2288|9950|764|2090|6|9|9|3|2090|764|9950|Saturday|2090Q3|N|Y|N|2484661|2484903|2484304|2484578|N|N|N|N|N| +2484670|AAAAAAAAOLJOFCAA|2090-09-10|2288|9950|764|2090|0|9|10|3|2090|764|9950|Sunday|2090Q3|N|N|N|2484661|2484903|2484305|2484579|N|N|N|N|N| +2484671|AAAAAAAAPLJOFCAA|2090-09-11|2288|9950|764|2090|1|9|11|3|2090|764|9950|Monday|2090Q3|N|N|N|2484661|2484903|2484306|2484580|N|N|N|N|N| +2484672|AAAAAAAAAMJOFCAA|2090-09-12|2288|9951|764|2090|2|9|12|3|2090|764|9951|Tuesday|2090Q3|N|N|N|2484661|2484903|2484307|2484581|N|N|N|N|N| +2484673|AAAAAAAABMJOFCAA|2090-09-13|2288|9951|764|2090|3|9|13|3|2090|764|9951|Wednesday|2090Q3|N|N|N|2484661|2484903|2484308|2484582|N|N|N|N|N| +2484674|AAAAAAAACMJOFCAA|2090-09-14|2288|9951|764|2090|4|9|14|3|2090|764|9951|Thursday|2090Q3|N|N|N|2484661|2484903|2484309|2484583|N|N|N|N|N| +2484675|AAAAAAAADMJOFCAA|2090-09-15|2288|9951|764|2090|5|9|15|3|2090|764|9951|Friday|2090Q3|N|Y|N|2484661|2484903|2484310|2484584|N|N|N|N|N| +2484676|AAAAAAAAEMJOFCAA|2090-09-16|2288|9951|764|2090|6|9|16|3|2090|764|9951|Saturday|2090Q3|N|Y|N|2484661|2484903|2484311|2484585|N|N|N|N|N| +2484677|AAAAAAAAFMJOFCAA|2090-09-17|2288|9951|764|2090|0|9|17|3|2090|764|9951|Sunday|2090Q3|N|N|N|2484661|2484903|2484312|2484586|N|N|N|N|N| +2484678|AAAAAAAAGMJOFCAA|2090-09-18|2288|9951|764|2090|1|9|18|3|2090|764|9951|Monday|2090Q3|N|N|N|2484661|2484903|2484313|2484587|N|N|N|N|N| +2484679|AAAAAAAAHMJOFCAA|2090-09-19|2288|9952|764|2090|2|9|19|3|2090|764|9952|Tuesday|2090Q3|N|N|N|2484661|2484903|2484314|2484588|N|N|N|N|N| +2484680|AAAAAAAAIMJOFCAA|2090-09-20|2288|9952|764|2090|3|9|20|3|2090|764|9952|Wednesday|2090Q3|N|N|N|2484661|2484903|2484315|2484589|N|N|N|N|N| +2484681|AAAAAAAAJMJOFCAA|2090-09-21|2288|9952|764|2090|4|9|21|3|2090|764|9952|Thursday|2090Q3|N|N|N|2484661|2484903|2484316|2484590|N|N|N|N|N| +2484682|AAAAAAAAKMJOFCAA|2090-09-22|2288|9952|764|2090|5|9|22|3|2090|764|9952|Friday|2090Q3|N|Y|N|2484661|2484903|2484317|2484591|N|N|N|N|N| +2484683|AAAAAAAALMJOFCAA|2090-09-23|2288|9952|764|2090|6|9|23|3|2090|764|9952|Saturday|2090Q3|N|Y|N|2484661|2484903|2484318|2484592|N|N|N|N|N| +2484684|AAAAAAAAMMJOFCAA|2090-09-24|2288|9952|764|2090|0|9|24|3|2090|764|9952|Sunday|2090Q3|N|N|N|2484661|2484903|2484319|2484593|N|N|N|N|N| +2484685|AAAAAAAANMJOFCAA|2090-09-25|2288|9952|764|2090|1|9|25|3|2090|764|9952|Monday|2090Q3|N|N|N|2484661|2484903|2484320|2484594|N|N|N|N|N| +2484686|AAAAAAAAOMJOFCAA|2090-09-26|2288|9953|764|2090|2|9|26|3|2090|764|9953|Tuesday|2090Q3|N|N|N|2484661|2484903|2484321|2484595|N|N|N|N|N| +2484687|AAAAAAAAPMJOFCAA|2090-09-27|2288|9953|764|2090|3|9|27|3|2090|764|9953|Wednesday|2090Q3|N|N|N|2484661|2484903|2484322|2484596|N|N|N|N|N| +2484688|AAAAAAAAANJOFCAA|2090-09-28|2288|9953|764|2090|4|9|28|3|2090|764|9953|Thursday|2090Q3|N|N|N|2484661|2484903|2484323|2484597|N|N|N|N|N| +2484689|AAAAAAAABNJOFCAA|2090-09-29|2288|9953|764|2090|5|9|29|3|2090|764|9953|Friday|2090Q3|N|Y|N|2484661|2484903|2484324|2484598|N|N|N|N|N| +2484690|AAAAAAAACNJOFCAA|2090-09-30|2288|9953|764|2090|6|9|30|3|2090|764|9953|Saturday|2090Q3|N|Y|N|2484661|2484903|2484325|2484599|N|N|N|N|N| +2484691|AAAAAAAADNJOFCAA|2090-10-01|2289|9953|764|2090|0|10|1|3|2090|764|9953|Sunday|2090Q3|N|N|N|2484691|2484963|2484326|2484599|N|N|N|N|N| +2484692|AAAAAAAAENJOFCAA|2090-10-02|2289|9953|764|2090|1|10|2|4|2090|764|9953|Monday|2090Q4|N|N|N|2484691|2484963|2484327|2484600|N|N|N|N|N| +2484693|AAAAAAAAFNJOFCAA|2090-10-03|2289|9954|764|2090|2|10|3|4|2090|764|9954|Tuesday|2090Q4|N|N|N|2484691|2484963|2484328|2484601|N|N|N|N|N| +2484694|AAAAAAAAGNJOFCAA|2090-10-04|2289|9954|764|2090|3|10|4|4|2090|764|9954|Wednesday|2090Q4|N|N|N|2484691|2484963|2484329|2484602|N|N|N|N|N| +2484695|AAAAAAAAHNJOFCAA|2090-10-05|2289|9954|764|2090|4|10|5|4|2090|764|9954|Thursday|2090Q4|N|N|N|2484691|2484963|2484330|2484603|N|N|N|N|N| +2484696|AAAAAAAAINJOFCAA|2090-10-06|2289|9954|764|2090|5|10|6|4|2090|764|9954|Friday|2090Q4|N|Y|N|2484691|2484963|2484331|2484604|N|N|N|N|N| +2484697|AAAAAAAAJNJOFCAA|2090-10-07|2289|9954|764|2090|6|10|7|4|2090|764|9954|Saturday|2090Q4|N|Y|N|2484691|2484963|2484332|2484605|N|N|N|N|N| +2484698|AAAAAAAAKNJOFCAA|2090-10-08|2289|9954|764|2090|0|10|8|4|2090|764|9954|Sunday|2090Q4|N|N|N|2484691|2484963|2484333|2484606|N|N|N|N|N| +2484699|AAAAAAAALNJOFCAA|2090-10-09|2289|9954|764|2090|1|10|9|4|2090|764|9954|Monday|2090Q4|N|N|N|2484691|2484963|2484334|2484607|N|N|N|N|N| +2484700|AAAAAAAAMNJOFCAA|2090-10-10|2289|9955|764|2090|2|10|10|4|2090|764|9955|Tuesday|2090Q4|N|N|N|2484691|2484963|2484335|2484608|N|N|N|N|N| +2484701|AAAAAAAANNJOFCAA|2090-10-11|2289|9955|764|2090|3|10|11|4|2090|764|9955|Wednesday|2090Q4|N|N|N|2484691|2484963|2484336|2484609|N|N|N|N|N| +2484702|AAAAAAAAONJOFCAA|2090-10-12|2289|9955|764|2090|4|10|12|4|2090|764|9955|Thursday|2090Q4|N|N|N|2484691|2484963|2484337|2484610|N|N|N|N|N| +2484703|AAAAAAAAPNJOFCAA|2090-10-13|2289|9955|764|2090|5|10|13|4|2090|764|9955|Friday|2090Q4|N|Y|N|2484691|2484963|2484338|2484611|N|N|N|N|N| +2484704|AAAAAAAAAOJOFCAA|2090-10-14|2289|9955|764|2090|6|10|14|4|2090|764|9955|Saturday|2090Q4|N|Y|N|2484691|2484963|2484339|2484612|N|N|N|N|N| +2484705|AAAAAAAABOJOFCAA|2090-10-15|2289|9955|764|2090|0|10|15|4|2090|764|9955|Sunday|2090Q4|N|N|N|2484691|2484963|2484340|2484613|N|N|N|N|N| +2484706|AAAAAAAACOJOFCAA|2090-10-16|2289|9955|764|2090|1|10|16|4|2090|764|9955|Monday|2090Q4|N|N|N|2484691|2484963|2484341|2484614|N|N|N|N|N| +2484707|AAAAAAAADOJOFCAA|2090-10-17|2289|9956|764|2090|2|10|17|4|2090|764|9956|Tuesday|2090Q4|N|N|N|2484691|2484963|2484342|2484615|N|N|N|N|N| +2484708|AAAAAAAAEOJOFCAA|2090-10-18|2289|9956|764|2090|3|10|18|4|2090|764|9956|Wednesday|2090Q4|N|N|N|2484691|2484963|2484343|2484616|N|N|N|N|N| +2484709|AAAAAAAAFOJOFCAA|2090-10-19|2289|9956|764|2090|4|10|19|4|2090|764|9956|Thursday|2090Q4|N|N|N|2484691|2484963|2484344|2484617|N|N|N|N|N| +2484710|AAAAAAAAGOJOFCAA|2090-10-20|2289|9956|764|2090|5|10|20|4|2090|764|9956|Friday|2090Q4|N|Y|N|2484691|2484963|2484345|2484618|N|N|N|N|N| +2484711|AAAAAAAAHOJOFCAA|2090-10-21|2289|9956|764|2090|6|10|21|4|2090|764|9956|Saturday|2090Q4|N|Y|N|2484691|2484963|2484346|2484619|N|N|N|N|N| +2484712|AAAAAAAAIOJOFCAA|2090-10-22|2289|9956|764|2090|0|10|22|4|2090|764|9956|Sunday|2090Q4|N|N|N|2484691|2484963|2484347|2484620|N|N|N|N|N| +2484713|AAAAAAAAJOJOFCAA|2090-10-23|2289|9956|764|2090|1|10|23|4|2090|764|9956|Monday|2090Q4|N|N|N|2484691|2484963|2484348|2484621|N|N|N|N|N| +2484714|AAAAAAAAKOJOFCAA|2090-10-24|2289|9957|764|2090|2|10|24|4|2090|764|9957|Tuesday|2090Q4|N|N|N|2484691|2484963|2484349|2484622|N|N|N|N|N| +2484715|AAAAAAAALOJOFCAA|2090-10-25|2289|9957|764|2090|3|10|25|4|2090|764|9957|Wednesday|2090Q4|N|N|N|2484691|2484963|2484350|2484623|N|N|N|N|N| +2484716|AAAAAAAAMOJOFCAA|2090-10-26|2289|9957|764|2090|4|10|26|4|2090|764|9957|Thursday|2090Q4|N|N|N|2484691|2484963|2484351|2484624|N|N|N|N|N| +2484717|AAAAAAAANOJOFCAA|2090-10-27|2289|9957|764|2090|5|10|27|4|2090|764|9957|Friday|2090Q4|N|Y|N|2484691|2484963|2484352|2484625|N|N|N|N|N| +2484718|AAAAAAAAOOJOFCAA|2090-10-28|2289|9957|764|2090|6|10|28|4|2090|764|9957|Saturday|2090Q4|N|Y|N|2484691|2484963|2484353|2484626|N|N|N|N|N| +2484719|AAAAAAAAPOJOFCAA|2090-10-29|2289|9957|764|2090|0|10|29|4|2090|764|9957|Sunday|2090Q4|N|N|N|2484691|2484963|2484354|2484627|N|N|N|N|N| +2484720|AAAAAAAAAPJOFCAA|2090-10-30|2289|9957|764|2090|1|10|30|4|2090|764|9957|Monday|2090Q4|N|N|N|2484691|2484963|2484355|2484628|N|N|N|N|N| +2484721|AAAAAAAABPJOFCAA|2090-10-31|2289|9958|764|2090|2|10|31|4|2090|764|9958|Tuesday|2090Q4|N|N|N|2484691|2484963|2484356|2484629|N|N|N|N|N| +2484722|AAAAAAAACPJOFCAA|2090-11-01|2290|9958|764|2090|3|11|1|4|2090|764|9958|Wednesday|2090Q4|N|N|N|2484722|2485025|2484357|2484630|N|N|N|N|N| +2484723|AAAAAAAADPJOFCAA|2090-11-02|2290|9958|764|2090|4|11|2|4|2090|764|9958|Thursday|2090Q4|N|N|N|2484722|2485025|2484358|2484631|N|N|N|N|N| +2484724|AAAAAAAAEPJOFCAA|2090-11-03|2290|9958|764|2090|5|11|3|4|2090|764|9958|Friday|2090Q4|N|Y|N|2484722|2485025|2484359|2484632|N|N|N|N|N| +2484725|AAAAAAAAFPJOFCAA|2090-11-04|2290|9958|764|2090|6|11|4|4|2090|764|9958|Saturday|2090Q4|N|Y|N|2484722|2485025|2484360|2484633|N|N|N|N|N| +2484726|AAAAAAAAGPJOFCAA|2090-11-05|2290|9958|764|2090|0|11|5|4|2090|764|9958|Sunday|2090Q4|N|N|N|2484722|2485025|2484361|2484634|N|N|N|N|N| +2484727|AAAAAAAAHPJOFCAA|2090-11-06|2290|9958|764|2090|1|11|6|4|2090|764|9958|Monday|2090Q4|N|N|N|2484722|2485025|2484362|2484635|N|N|N|N|N| +2484728|AAAAAAAAIPJOFCAA|2090-11-07|2290|9959|764|2090|2|11|7|4|2090|764|9959|Tuesday|2090Q4|N|N|N|2484722|2485025|2484363|2484636|N|N|N|N|N| +2484729|AAAAAAAAJPJOFCAA|2090-11-08|2290|9959|764|2090|3|11|8|4|2090|764|9959|Wednesday|2090Q4|N|N|N|2484722|2485025|2484364|2484637|N|N|N|N|N| +2484730|AAAAAAAAKPJOFCAA|2090-11-09|2290|9959|764|2090|4|11|9|4|2090|764|9959|Thursday|2090Q4|N|N|N|2484722|2485025|2484365|2484638|N|N|N|N|N| +2484731|AAAAAAAALPJOFCAA|2090-11-10|2290|9959|764|2090|5|11|10|4|2090|764|9959|Friday|2090Q4|N|Y|N|2484722|2485025|2484366|2484639|N|N|N|N|N| +2484732|AAAAAAAAMPJOFCAA|2090-11-11|2290|9959|764|2090|6|11|11|4|2090|764|9959|Saturday|2090Q4|N|Y|N|2484722|2485025|2484367|2484640|N|N|N|N|N| +2484733|AAAAAAAANPJOFCAA|2090-11-12|2290|9959|764|2090|0|11|12|4|2090|764|9959|Sunday|2090Q4|N|N|N|2484722|2485025|2484368|2484641|N|N|N|N|N| +2484734|AAAAAAAAOPJOFCAA|2090-11-13|2290|9959|764|2090|1|11|13|4|2090|764|9959|Monday|2090Q4|N|N|N|2484722|2485025|2484369|2484642|N|N|N|N|N| +2484735|AAAAAAAAPPJOFCAA|2090-11-14|2290|9960|764|2090|2|11|14|4|2090|764|9960|Tuesday|2090Q4|N|N|N|2484722|2485025|2484370|2484643|N|N|N|N|N| +2484736|AAAAAAAAAAKOFCAA|2090-11-15|2290|9960|764|2090|3|11|15|4|2090|764|9960|Wednesday|2090Q4|N|N|N|2484722|2485025|2484371|2484644|N|N|N|N|N| +2484737|AAAAAAAABAKOFCAA|2090-11-16|2290|9960|764|2090|4|11|16|4|2090|764|9960|Thursday|2090Q4|N|N|N|2484722|2485025|2484372|2484645|N|N|N|N|N| +2484738|AAAAAAAACAKOFCAA|2090-11-17|2290|9960|764|2090|5|11|17|4|2090|764|9960|Friday|2090Q4|N|Y|N|2484722|2485025|2484373|2484646|N|N|N|N|N| +2484739|AAAAAAAADAKOFCAA|2090-11-18|2290|9960|764|2090|6|11|18|4|2090|764|9960|Saturday|2090Q4|N|Y|N|2484722|2485025|2484374|2484647|N|N|N|N|N| +2484740|AAAAAAAAEAKOFCAA|2090-11-19|2290|9960|764|2090|0|11|19|4|2090|764|9960|Sunday|2090Q4|N|N|N|2484722|2485025|2484375|2484648|N|N|N|N|N| +2484741|AAAAAAAAFAKOFCAA|2090-11-20|2290|9960|764|2090|1|11|20|4|2090|764|9960|Monday|2090Q4|N|N|N|2484722|2485025|2484376|2484649|N|N|N|N|N| +2484742|AAAAAAAAGAKOFCAA|2090-11-21|2290|9961|764|2090|2|11|21|4|2090|764|9961|Tuesday|2090Q4|N|N|N|2484722|2485025|2484377|2484650|N|N|N|N|N| +2484743|AAAAAAAAHAKOFCAA|2090-11-22|2290|9961|764|2090|3|11|22|4|2090|764|9961|Wednesday|2090Q4|N|N|N|2484722|2485025|2484378|2484651|N|N|N|N|N| +2484744|AAAAAAAAIAKOFCAA|2090-11-23|2290|9961|764|2090|4|11|23|4|2090|764|9961|Thursday|2090Q4|N|N|N|2484722|2485025|2484379|2484652|N|N|N|N|N| +2484745|AAAAAAAAJAKOFCAA|2090-11-24|2290|9961|764|2090|5|11|24|4|2090|764|9961|Friday|2090Q4|N|Y|N|2484722|2485025|2484380|2484653|N|N|N|N|N| +2484746|AAAAAAAAKAKOFCAA|2090-11-25|2290|9961|764|2090|6|11|25|4|2090|764|9961|Saturday|2090Q4|N|Y|N|2484722|2485025|2484381|2484654|N|N|N|N|N| +2484747|AAAAAAAALAKOFCAA|2090-11-26|2290|9961|764|2090|0|11|26|4|2090|764|9961|Sunday|2090Q4|N|N|N|2484722|2485025|2484382|2484655|N|N|N|N|N| +2484748|AAAAAAAAMAKOFCAA|2090-11-27|2290|9961|764|2090|1|11|27|4|2090|764|9961|Monday|2090Q4|N|N|N|2484722|2485025|2484383|2484656|N|N|N|N|N| +2484749|AAAAAAAANAKOFCAA|2090-11-28|2290|9962|764|2090|2|11|28|4|2090|764|9962|Tuesday|2090Q4|N|N|N|2484722|2485025|2484384|2484657|N|N|N|N|N| +2484750|AAAAAAAAOAKOFCAA|2090-11-29|2290|9962|764|2090|3|11|29|4|2090|764|9962|Wednesday|2090Q4|N|N|N|2484722|2485025|2484385|2484658|N|N|N|N|N| +2484751|AAAAAAAAPAKOFCAA|2090-11-30|2290|9962|764|2090|4|11|30|4|2090|764|9962|Thursday|2090Q4|N|N|N|2484722|2485025|2484386|2484659|N|N|N|N|N| +2484752|AAAAAAAAABKOFCAA|2090-12-01|2291|9962|765|2090|5|12|1|4|2090|765|9962|Friday|2090Q4|N|Y|N|2484752|2485085|2484387|2484660|N|N|N|N|N| +2484753|AAAAAAAABBKOFCAA|2090-12-02|2291|9962|765|2090|6|12|2|4|2090|765|9962|Saturday|2090Q4|N|Y|N|2484752|2485085|2484388|2484661|N|N|N|N|N| +2484754|AAAAAAAACBKOFCAA|2090-12-03|2291|9962|765|2090|0|12|3|4|2090|765|9962|Sunday|2090Q4|N|N|N|2484752|2485085|2484389|2484662|N|N|N|N|N| +2484755|AAAAAAAADBKOFCAA|2090-12-04|2291|9962|765|2090|1|12|4|4|2090|765|9962|Monday|2090Q4|N|N|N|2484752|2485085|2484390|2484663|N|N|N|N|N| +2484756|AAAAAAAAEBKOFCAA|2090-12-05|2291|9963|765|2090|2|12|5|4|2090|765|9963|Tuesday|2090Q4|N|N|N|2484752|2485085|2484391|2484664|N|N|N|N|N| +2484757|AAAAAAAAFBKOFCAA|2090-12-06|2291|9963|765|2090|3|12|6|4|2090|765|9963|Wednesday|2090Q4|N|N|N|2484752|2485085|2484392|2484665|N|N|N|N|N| +2484758|AAAAAAAAGBKOFCAA|2090-12-07|2291|9963|765|2090|4|12|7|4|2090|765|9963|Thursday|2090Q4|N|N|N|2484752|2485085|2484393|2484666|N|N|N|N|N| +2484759|AAAAAAAAHBKOFCAA|2090-12-08|2291|9963|765|2090|5|12|8|4|2090|765|9963|Friday|2090Q4|N|Y|N|2484752|2485085|2484394|2484667|N|N|N|N|N| +2484760|AAAAAAAAIBKOFCAA|2090-12-09|2291|9963|765|2090|6|12|9|4|2090|765|9963|Saturday|2090Q4|N|Y|N|2484752|2485085|2484395|2484668|N|N|N|N|N| +2484761|AAAAAAAAJBKOFCAA|2090-12-10|2291|9963|765|2090|0|12|10|4|2090|765|9963|Sunday|2090Q4|N|N|N|2484752|2485085|2484396|2484669|N|N|N|N|N| +2484762|AAAAAAAAKBKOFCAA|2090-12-11|2291|9963|765|2090|1|12|11|4|2090|765|9963|Monday|2090Q4|N|N|N|2484752|2485085|2484397|2484670|N|N|N|N|N| +2484763|AAAAAAAALBKOFCAA|2090-12-12|2291|9964|765|2090|2|12|12|4|2090|765|9964|Tuesday|2090Q4|N|N|N|2484752|2485085|2484398|2484671|N|N|N|N|N| +2484764|AAAAAAAAMBKOFCAA|2090-12-13|2291|9964|765|2090|3|12|13|4|2090|765|9964|Wednesday|2090Q4|N|N|N|2484752|2485085|2484399|2484672|N|N|N|N|N| +2484765|AAAAAAAANBKOFCAA|2090-12-14|2291|9964|765|2090|4|12|14|4|2090|765|9964|Thursday|2090Q4|N|N|N|2484752|2485085|2484400|2484673|N|N|N|N|N| +2484766|AAAAAAAAOBKOFCAA|2090-12-15|2291|9964|765|2090|5|12|15|4|2090|765|9964|Friday|2090Q4|N|Y|N|2484752|2485085|2484401|2484674|N|N|N|N|N| +2484767|AAAAAAAAPBKOFCAA|2090-12-16|2291|9964|765|2090|6|12|16|4|2090|765|9964|Saturday|2090Q4|N|Y|N|2484752|2485085|2484402|2484675|N|N|N|N|N| +2484768|AAAAAAAAACKOFCAA|2090-12-17|2291|9964|765|2090|0|12|17|4|2090|765|9964|Sunday|2090Q4|N|N|N|2484752|2485085|2484403|2484676|N|N|N|N|N| +2484769|AAAAAAAABCKOFCAA|2090-12-18|2291|9964|765|2090|1|12|18|4|2090|765|9964|Monday|2090Q4|N|N|N|2484752|2485085|2484404|2484677|N|N|N|N|N| +2484770|AAAAAAAACCKOFCAA|2090-12-19|2291|9965|765|2090|2|12|19|4|2090|765|9965|Tuesday|2090Q4|N|N|N|2484752|2485085|2484405|2484678|N|N|N|N|N| +2484771|AAAAAAAADCKOFCAA|2090-12-20|2291|9965|765|2090|3|12|20|4|2090|765|9965|Wednesday|2090Q4|N|N|N|2484752|2485085|2484406|2484679|N|N|N|N|N| +2484772|AAAAAAAAECKOFCAA|2090-12-21|2291|9965|765|2090|4|12|21|4|2090|765|9965|Thursday|2090Q4|N|N|N|2484752|2485085|2484407|2484680|N|N|N|N|N| +2484773|AAAAAAAAFCKOFCAA|2090-12-22|2291|9965|765|2090|5|12|22|4|2090|765|9965|Friday|2090Q4|N|Y|N|2484752|2485085|2484408|2484681|N|N|N|N|N| +2484774|AAAAAAAAGCKOFCAA|2090-12-23|2291|9965|765|2090|6|12|23|4|2090|765|9965|Saturday|2090Q4|N|Y|N|2484752|2485085|2484409|2484682|N|N|N|N|N| +2484775|AAAAAAAAHCKOFCAA|2090-12-24|2291|9965|765|2090|0|12|24|4|2090|765|9965|Sunday|2090Q4|N|N|N|2484752|2485085|2484410|2484683|N|N|N|N|N| +2484776|AAAAAAAAICKOFCAA|2090-12-25|2291|9965|765|2090|1|12|25|4|2090|765|9965|Monday|2090Q4|N|N|N|2484752|2485085|2484411|2484684|N|N|N|N|N| +2484777|AAAAAAAAJCKOFCAA|2090-12-26|2291|9966|765|2090|2|12|26|4|2090|765|9966|Tuesday|2090Q4|Y|N|N|2484752|2485085|2484412|2484685|N|N|N|N|N| +2484778|AAAAAAAAKCKOFCAA|2090-12-27|2291|9966|765|2090|3|12|27|4|2090|765|9966|Wednesday|2090Q4|N|N|Y|2484752|2485085|2484413|2484686|N|N|N|N|N| +2484779|AAAAAAAALCKOFCAA|2090-12-28|2291|9966|765|2090|4|12|28|4|2090|765|9966|Thursday|2090Q4|N|N|N|2484752|2485085|2484414|2484687|N|N|N|N|N| +2484780|AAAAAAAAMCKOFCAA|2090-12-29|2291|9966|765|2090|5|12|29|4|2090|765|9966|Friday|2090Q4|N|Y|N|2484752|2485085|2484415|2484688|N|N|N|N|N| +2484781|AAAAAAAANCKOFCAA|2090-12-30|2291|9966|765|2090|6|12|30|4|2090|765|9966|Saturday|2090Q4|N|Y|N|2484752|2485085|2484416|2484689|N|N|N|N|N| +2484782|AAAAAAAAOCKOFCAA|2090-12-31|2291|9966|765|2090|0|12|31|4|2090|765|9966|Sunday|2090Q4|N|N|N|2484752|2485085|2484417|2484690|N|N|N|N|N| +2484783|AAAAAAAAPCKOFCAA|2091-01-01|2292|9966|765|2091|1|1|1|1|2091|765|9966|Monday|2091Q1|Y|N|N|2484783|2484782|2484418|2484691|N|N|N|N|N| +2484784|AAAAAAAAADKOFCAA|2091-01-02|2292|9967|765|2091|2|1|2|1|2091|765|9967|Tuesday|2091Q1|N|N|Y|2484783|2484782|2484419|2484692|N|N|N|N|N| +2484785|AAAAAAAABDKOFCAA|2091-01-03|2292|9967|765|2091|3|1|3|1|2091|765|9967|Wednesday|2091Q1|N|N|N|2484783|2484782|2484420|2484693|N|N|N|N|N| +2484786|AAAAAAAACDKOFCAA|2091-01-04|2292|9967|765|2091|4|1|4|1|2091|765|9967|Thursday|2091Q1|N|N|N|2484783|2484782|2484421|2484694|N|N|N|N|N| +2484787|AAAAAAAADDKOFCAA|2091-01-05|2292|9967|765|2091|5|1|5|1|2091|765|9967|Friday|2091Q1|N|Y|N|2484783|2484782|2484422|2484695|N|N|N|N|N| +2484788|AAAAAAAAEDKOFCAA|2091-01-06|2292|9967|765|2091|6|1|6|1|2091|765|9967|Saturday|2091Q1|N|Y|N|2484783|2484782|2484423|2484696|N|N|N|N|N| +2484789|AAAAAAAAFDKOFCAA|2091-01-07|2292|9967|765|2091|0|1|7|1|2091|765|9967|Sunday|2091Q1|N|N|N|2484783|2484782|2484424|2484697|N|N|N|N|N| +2484790|AAAAAAAAGDKOFCAA|2091-01-08|2292|9967|765|2091|1|1|8|1|2091|765|9967|Monday|2091Q1|N|N|N|2484783|2484782|2484425|2484698|N|N|N|N|N| +2484791|AAAAAAAAHDKOFCAA|2091-01-09|2292|9968|765|2091|2|1|9|1|2091|765|9968|Tuesday|2091Q1|N|N|N|2484783|2484782|2484426|2484699|N|N|N|N|N| +2484792|AAAAAAAAIDKOFCAA|2091-01-10|2292|9968|765|2091|3|1|10|1|2091|765|9968|Wednesday|2091Q1|N|N|N|2484783|2484782|2484427|2484700|N|N|N|N|N| +2484793|AAAAAAAAJDKOFCAA|2091-01-11|2292|9968|765|2091|4|1|11|1|2091|765|9968|Thursday|2091Q1|N|N|N|2484783|2484782|2484428|2484701|N|N|N|N|N| +2484794|AAAAAAAAKDKOFCAA|2091-01-12|2292|9968|765|2091|5|1|12|1|2091|765|9968|Friday|2091Q1|N|Y|N|2484783|2484782|2484429|2484702|N|N|N|N|N| +2484795|AAAAAAAALDKOFCAA|2091-01-13|2292|9968|765|2091|6|1|13|1|2091|765|9968|Saturday|2091Q1|N|Y|N|2484783|2484782|2484430|2484703|N|N|N|N|N| +2484796|AAAAAAAAMDKOFCAA|2091-01-14|2292|9968|765|2091|0|1|14|1|2091|765|9968|Sunday|2091Q1|N|N|N|2484783|2484782|2484431|2484704|N|N|N|N|N| +2484797|AAAAAAAANDKOFCAA|2091-01-15|2292|9968|765|2091|1|1|15|1|2091|765|9968|Monday|2091Q1|N|N|N|2484783|2484782|2484432|2484705|N|N|N|N|N| +2484798|AAAAAAAAODKOFCAA|2091-01-16|2292|9969|765|2091|2|1|16|1|2091|765|9969|Tuesday|2091Q1|N|N|N|2484783|2484782|2484433|2484706|N|N|N|N|N| +2484799|AAAAAAAAPDKOFCAA|2091-01-17|2292|9969|765|2091|3|1|17|1|2091|765|9969|Wednesday|2091Q1|N|N|N|2484783|2484782|2484434|2484707|N|N|N|N|N| +2484800|AAAAAAAAAEKOFCAA|2091-01-18|2292|9969|765|2091|4|1|18|1|2091|765|9969|Thursday|2091Q1|N|N|N|2484783|2484782|2484435|2484708|N|N|N|N|N| +2484801|AAAAAAAABEKOFCAA|2091-01-19|2292|9969|765|2091|5|1|19|1|2091|765|9969|Friday|2091Q1|N|Y|N|2484783|2484782|2484436|2484709|N|N|N|N|N| +2484802|AAAAAAAACEKOFCAA|2091-01-20|2292|9969|765|2091|6|1|20|1|2091|765|9969|Saturday|2091Q1|N|Y|N|2484783|2484782|2484437|2484710|N|N|N|N|N| +2484803|AAAAAAAADEKOFCAA|2091-01-21|2292|9969|765|2091|0|1|21|1|2091|765|9969|Sunday|2091Q1|N|N|N|2484783|2484782|2484438|2484711|N|N|N|N|N| +2484804|AAAAAAAAEEKOFCAA|2091-01-22|2292|9969|765|2091|1|1|22|1|2091|765|9969|Monday|2091Q1|N|N|N|2484783|2484782|2484439|2484712|N|N|N|N|N| +2484805|AAAAAAAAFEKOFCAA|2091-01-23|2292|9970|765|2091|2|1|23|1|2091|765|9970|Tuesday|2091Q1|N|N|N|2484783|2484782|2484440|2484713|N|N|N|N|N| +2484806|AAAAAAAAGEKOFCAA|2091-01-24|2292|9970|765|2091|3|1|24|1|2091|765|9970|Wednesday|2091Q1|N|N|N|2484783|2484782|2484441|2484714|N|N|N|N|N| +2484807|AAAAAAAAHEKOFCAA|2091-01-25|2292|9970|765|2091|4|1|25|1|2091|765|9970|Thursday|2091Q1|N|N|N|2484783|2484782|2484442|2484715|N|N|N|N|N| +2484808|AAAAAAAAIEKOFCAA|2091-01-26|2292|9970|765|2091|5|1|26|1|2091|765|9970|Friday|2091Q1|N|Y|N|2484783|2484782|2484443|2484716|N|N|N|N|N| +2484809|AAAAAAAAJEKOFCAA|2091-01-27|2292|9970|765|2091|6|1|27|1|2091|765|9970|Saturday|2091Q1|N|Y|N|2484783|2484782|2484444|2484717|N|N|N|N|N| +2484810|AAAAAAAAKEKOFCAA|2091-01-28|2292|9970|765|2091|0|1|28|1|2091|765|9970|Sunday|2091Q1|N|N|N|2484783|2484782|2484445|2484718|N|N|N|N|N| +2484811|AAAAAAAALEKOFCAA|2091-01-29|2292|9970|765|2091|1|1|29|1|2091|765|9970|Monday|2091Q1|N|N|N|2484783|2484782|2484446|2484719|N|N|N|N|N| +2484812|AAAAAAAAMEKOFCAA|2091-01-30|2292|9971|765|2091|2|1|30|1|2091|765|9971|Tuesday|2091Q1|N|N|N|2484783|2484782|2484447|2484720|N|N|N|N|N| +2484813|AAAAAAAANEKOFCAA|2091-01-31|2292|9971|765|2091|3|1|31|1|2091|765|9971|Wednesday|2091Q1|N|N|N|2484783|2484782|2484448|2484721|N|N|N|N|N| +2484814|AAAAAAAAOEKOFCAA|2091-02-01|2293|9971|765|2091|4|2|1|1|2091|765|9971|Thursday|2091Q1|N|N|N|2484814|2484844|2484449|2484722|N|N|N|N|N| +2484815|AAAAAAAAPEKOFCAA|2091-02-02|2293|9971|765|2091|5|2|2|1|2091|765|9971|Friday|2091Q1|N|Y|N|2484814|2484844|2484450|2484723|N|N|N|N|N| +2484816|AAAAAAAAAFKOFCAA|2091-02-03|2293|9971|765|2091|6|2|3|1|2091|765|9971|Saturday|2091Q1|N|Y|N|2484814|2484844|2484451|2484724|N|N|N|N|N| +2484817|AAAAAAAABFKOFCAA|2091-02-04|2293|9971|765|2091|0|2|4|1|2091|765|9971|Sunday|2091Q1|N|N|N|2484814|2484844|2484452|2484725|N|N|N|N|N| +2484818|AAAAAAAACFKOFCAA|2091-02-05|2293|9971|765|2091|1|2|5|1|2091|765|9971|Monday|2091Q1|N|N|N|2484814|2484844|2484453|2484726|N|N|N|N|N| +2484819|AAAAAAAADFKOFCAA|2091-02-06|2293|9972|765|2091|2|2|6|1|2091|765|9972|Tuesday|2091Q1|N|N|N|2484814|2484844|2484454|2484727|N|N|N|N|N| +2484820|AAAAAAAAEFKOFCAA|2091-02-07|2293|9972|765|2091|3|2|7|1|2091|765|9972|Wednesday|2091Q1|N|N|N|2484814|2484844|2484455|2484728|N|N|N|N|N| +2484821|AAAAAAAAFFKOFCAA|2091-02-08|2293|9972|765|2091|4|2|8|1|2091|765|9972|Thursday|2091Q1|N|N|N|2484814|2484844|2484456|2484729|N|N|N|N|N| +2484822|AAAAAAAAGFKOFCAA|2091-02-09|2293|9972|765|2091|5|2|9|1|2091|765|9972|Friday|2091Q1|N|Y|N|2484814|2484844|2484457|2484730|N|N|N|N|N| +2484823|AAAAAAAAHFKOFCAA|2091-02-10|2293|9972|765|2091|6|2|10|1|2091|765|9972|Saturday|2091Q1|N|Y|N|2484814|2484844|2484458|2484731|N|N|N|N|N| +2484824|AAAAAAAAIFKOFCAA|2091-02-11|2293|9972|765|2091|0|2|11|1|2091|765|9972|Sunday|2091Q1|N|N|N|2484814|2484844|2484459|2484732|N|N|N|N|N| +2484825|AAAAAAAAJFKOFCAA|2091-02-12|2293|9972|765|2091|1|2|12|1|2091|765|9972|Monday|2091Q1|N|N|N|2484814|2484844|2484460|2484733|N|N|N|N|N| +2484826|AAAAAAAAKFKOFCAA|2091-02-13|2293|9973|765|2091|2|2|13|1|2091|765|9973|Tuesday|2091Q1|N|N|N|2484814|2484844|2484461|2484734|N|N|N|N|N| +2484827|AAAAAAAALFKOFCAA|2091-02-14|2293|9973|765|2091|3|2|14|1|2091|765|9973|Wednesday|2091Q1|N|N|N|2484814|2484844|2484462|2484735|N|N|N|N|N| +2484828|AAAAAAAAMFKOFCAA|2091-02-15|2293|9973|765|2091|4|2|15|1|2091|765|9973|Thursday|2091Q1|N|N|N|2484814|2484844|2484463|2484736|N|N|N|N|N| +2484829|AAAAAAAANFKOFCAA|2091-02-16|2293|9973|765|2091|5|2|16|1|2091|765|9973|Friday|2091Q1|N|Y|N|2484814|2484844|2484464|2484737|N|N|N|N|N| +2484830|AAAAAAAAOFKOFCAA|2091-02-17|2293|9973|765|2091|6|2|17|1|2091|765|9973|Saturday|2091Q1|N|Y|N|2484814|2484844|2484465|2484738|N|N|N|N|N| +2484831|AAAAAAAAPFKOFCAA|2091-02-18|2293|9973|765|2091|0|2|18|1|2091|765|9973|Sunday|2091Q1|N|N|N|2484814|2484844|2484466|2484739|N|N|N|N|N| +2484832|AAAAAAAAAGKOFCAA|2091-02-19|2293|9973|765|2091|1|2|19|1|2091|765|9973|Monday|2091Q1|N|N|N|2484814|2484844|2484467|2484740|N|N|N|N|N| +2484833|AAAAAAAABGKOFCAA|2091-02-20|2293|9974|765|2091|2|2|20|1|2091|765|9974|Tuesday|2091Q1|N|N|N|2484814|2484844|2484468|2484741|N|N|N|N|N| +2484834|AAAAAAAACGKOFCAA|2091-02-21|2293|9974|765|2091|3|2|21|1|2091|765|9974|Wednesday|2091Q1|N|N|N|2484814|2484844|2484469|2484742|N|N|N|N|N| +2484835|AAAAAAAADGKOFCAA|2091-02-22|2293|9974|765|2091|4|2|22|1|2091|765|9974|Thursday|2091Q1|N|N|N|2484814|2484844|2484470|2484743|N|N|N|N|N| +2484836|AAAAAAAAEGKOFCAA|2091-02-23|2293|9974|765|2091|5|2|23|1|2091|765|9974|Friday|2091Q1|N|Y|N|2484814|2484844|2484471|2484744|N|N|N|N|N| +2484837|AAAAAAAAFGKOFCAA|2091-02-24|2293|9974|765|2091|6|2|24|1|2091|765|9974|Saturday|2091Q1|N|Y|N|2484814|2484844|2484472|2484745|N|N|N|N|N| +2484838|AAAAAAAAGGKOFCAA|2091-02-25|2293|9974|765|2091|0|2|25|1|2091|765|9974|Sunday|2091Q1|N|N|N|2484814|2484844|2484473|2484746|N|N|N|N|N| +2484839|AAAAAAAAHGKOFCAA|2091-02-26|2293|9974|765|2091|1|2|26|1|2091|765|9974|Monday|2091Q1|N|N|N|2484814|2484844|2484474|2484747|N|N|N|N|N| +2484840|AAAAAAAAIGKOFCAA|2091-02-27|2293|9975|765|2091|2|2|27|1|2091|765|9975|Tuesday|2091Q1|N|N|N|2484814|2484844|2484475|2484748|N|N|N|N|N| +2484841|AAAAAAAAJGKOFCAA|2091-02-28|2293|9975|765|2091|3|2|28|1|2091|765|9975|Wednesday|2091Q1|N|N|N|2484814|2484844|2484476|2484749|N|N|N|N|N| +2484842|AAAAAAAAKGKOFCAA|2091-03-01|2294|9975|766|2091|4|3|1|1|2091|766|9975|Thursday|2091Q1|N|N|N|2484842|2484900|2484477|2484750|N|N|N|N|N| +2484843|AAAAAAAALGKOFCAA|2091-03-02|2294|9975|766|2091|5|3|2|1|2091|766|9975|Friday|2091Q1|N|Y|N|2484842|2484900|2484478|2484751|N|N|N|N|N| +2484844|AAAAAAAAMGKOFCAA|2091-03-03|2294|9975|766|2091|6|3|3|1|2091|766|9975|Saturday|2091Q1|N|Y|N|2484842|2484900|2484479|2484752|N|N|N|N|N| +2484845|AAAAAAAANGKOFCAA|2091-03-04|2294|9975|766|2091|0|3|4|1|2091|766|9975|Sunday|2091Q1|N|N|N|2484842|2484900|2484480|2484753|N|N|N|N|N| +2484846|AAAAAAAAOGKOFCAA|2091-03-05|2294|9975|766|2091|1|3|5|1|2091|766|9975|Monday|2091Q1|N|N|N|2484842|2484900|2484481|2484754|N|N|N|N|N| +2484847|AAAAAAAAPGKOFCAA|2091-03-06|2294|9976|766|2091|2|3|6|1|2091|766|9976|Tuesday|2091Q1|N|N|N|2484842|2484900|2484482|2484755|N|N|N|N|N| +2484848|AAAAAAAAAHKOFCAA|2091-03-07|2294|9976|766|2091|3|3|7|1|2091|766|9976|Wednesday|2091Q1|N|N|N|2484842|2484900|2484483|2484756|N|N|N|N|N| +2484849|AAAAAAAABHKOFCAA|2091-03-08|2294|9976|766|2091|4|3|8|1|2091|766|9976|Thursday|2091Q1|N|N|N|2484842|2484900|2484484|2484757|N|N|N|N|N| +2484850|AAAAAAAACHKOFCAA|2091-03-09|2294|9976|766|2091|5|3|9|1|2091|766|9976|Friday|2091Q1|N|Y|N|2484842|2484900|2484485|2484758|N|N|N|N|N| +2484851|AAAAAAAADHKOFCAA|2091-03-10|2294|9976|766|2091|6|3|10|1|2091|766|9976|Saturday|2091Q1|N|Y|N|2484842|2484900|2484486|2484759|N|N|N|N|N| +2484852|AAAAAAAAEHKOFCAA|2091-03-11|2294|9976|766|2091|0|3|11|1|2091|766|9976|Sunday|2091Q1|N|N|N|2484842|2484900|2484487|2484760|N|N|N|N|N| +2484853|AAAAAAAAFHKOFCAA|2091-03-12|2294|9976|766|2091|1|3|12|1|2091|766|9976|Monday|2091Q1|N|N|N|2484842|2484900|2484488|2484761|N|N|N|N|N| +2484854|AAAAAAAAGHKOFCAA|2091-03-13|2294|9977|766|2091|2|3|13|1|2091|766|9977|Tuesday|2091Q1|N|N|N|2484842|2484900|2484489|2484762|N|N|N|N|N| +2484855|AAAAAAAAHHKOFCAA|2091-03-14|2294|9977|766|2091|3|3|14|1|2091|766|9977|Wednesday|2091Q1|N|N|N|2484842|2484900|2484490|2484763|N|N|N|N|N| +2484856|AAAAAAAAIHKOFCAA|2091-03-15|2294|9977|766|2091|4|3|15|1|2091|766|9977|Thursday|2091Q1|N|N|N|2484842|2484900|2484491|2484764|N|N|N|N|N| +2484857|AAAAAAAAJHKOFCAA|2091-03-16|2294|9977|766|2091|5|3|16|1|2091|766|9977|Friday|2091Q1|N|Y|N|2484842|2484900|2484492|2484765|N|N|N|N|N| +2484858|AAAAAAAAKHKOFCAA|2091-03-17|2294|9977|766|2091|6|3|17|1|2091|766|9977|Saturday|2091Q1|N|Y|N|2484842|2484900|2484493|2484766|N|N|N|N|N| +2484859|AAAAAAAALHKOFCAA|2091-03-18|2294|9977|766|2091|0|3|18|1|2091|766|9977|Sunday|2091Q1|N|N|N|2484842|2484900|2484494|2484767|N|N|N|N|N| +2484860|AAAAAAAAMHKOFCAA|2091-03-19|2294|9977|766|2091|1|3|19|1|2091|766|9977|Monday|2091Q1|N|N|N|2484842|2484900|2484495|2484768|N|N|N|N|N| +2484861|AAAAAAAANHKOFCAA|2091-03-20|2294|9978|766|2091|2|3|20|1|2091|766|9978|Tuesday|2091Q1|N|N|N|2484842|2484900|2484496|2484769|N|N|N|N|N| +2484862|AAAAAAAAOHKOFCAA|2091-03-21|2294|9978|766|2091|3|3|21|1|2091|766|9978|Wednesday|2091Q1|N|N|N|2484842|2484900|2484497|2484770|N|N|N|N|N| +2484863|AAAAAAAAPHKOFCAA|2091-03-22|2294|9978|766|2091|4|3|22|1|2091|766|9978|Thursday|2091Q1|N|N|N|2484842|2484900|2484498|2484771|N|N|N|N|N| +2484864|AAAAAAAAAIKOFCAA|2091-03-23|2294|9978|766|2091|5|3|23|1|2091|766|9978|Friday|2091Q1|N|Y|N|2484842|2484900|2484499|2484772|N|N|N|N|N| +2484865|AAAAAAAABIKOFCAA|2091-03-24|2294|9978|766|2091|6|3|24|1|2091|766|9978|Saturday|2091Q1|N|Y|N|2484842|2484900|2484500|2484773|N|N|N|N|N| +2484866|AAAAAAAACIKOFCAA|2091-03-25|2294|9978|766|2091|0|3|25|1|2091|766|9978|Sunday|2091Q1|N|N|N|2484842|2484900|2484501|2484774|N|N|N|N|N| +2484867|AAAAAAAADIKOFCAA|2091-03-26|2294|9978|766|2091|1|3|26|1|2091|766|9978|Monday|2091Q1|N|N|N|2484842|2484900|2484502|2484775|N|N|N|N|N| +2484868|AAAAAAAAEIKOFCAA|2091-03-27|2294|9979|766|2091|2|3|27|1|2091|766|9979|Tuesday|2091Q1|N|N|N|2484842|2484900|2484503|2484776|N|N|N|N|N| +2484869|AAAAAAAAFIKOFCAA|2091-03-28|2294|9979|766|2091|3|3|28|1|2091|766|9979|Wednesday|2091Q1|N|N|N|2484842|2484900|2484504|2484777|N|N|N|N|N| +2484870|AAAAAAAAGIKOFCAA|2091-03-29|2294|9979|766|2091|4|3|29|1|2091|766|9979|Thursday|2091Q1|N|N|N|2484842|2484900|2484505|2484778|N|N|N|N|N| +2484871|AAAAAAAAHIKOFCAA|2091-03-30|2294|9979|766|2091|5|3|30|1|2091|766|9979|Friday|2091Q1|N|Y|N|2484842|2484900|2484506|2484779|N|N|N|N|N| +2484872|AAAAAAAAIIKOFCAA|2091-03-31|2294|9979|766|2091|6|3|31|1|2091|766|9979|Saturday|2091Q1|N|Y|N|2484842|2484900|2484507|2484780|N|N|N|N|N| +2484873|AAAAAAAAJIKOFCAA|2091-04-01|2295|9979|766|2091|0|4|1|1|2091|766|9979|Sunday|2091Q1|N|N|N|2484873|2484962|2484508|2484783|N|N|N|N|N| +2484874|AAAAAAAAKIKOFCAA|2091-04-02|2295|9979|766|2091|1|4|2|2|2091|766|9979|Monday|2091Q2|N|N|N|2484873|2484962|2484509|2484784|N|N|N|N|N| +2484875|AAAAAAAALIKOFCAA|2091-04-03|2295|9980|766|2091|2|4|3|2|2091|766|9980|Tuesday|2091Q2|N|N|N|2484873|2484962|2484510|2484785|N|N|N|N|N| +2484876|AAAAAAAAMIKOFCAA|2091-04-04|2295|9980|766|2091|3|4|4|2|2091|766|9980|Wednesday|2091Q2|N|N|N|2484873|2484962|2484511|2484786|N|N|N|N|N| +2484877|AAAAAAAANIKOFCAA|2091-04-05|2295|9980|766|2091|4|4|5|2|2091|766|9980|Thursday|2091Q2|N|N|N|2484873|2484962|2484512|2484787|N|N|N|N|N| +2484878|AAAAAAAAOIKOFCAA|2091-04-06|2295|9980|766|2091|5|4|6|2|2091|766|9980|Friday|2091Q2|N|Y|N|2484873|2484962|2484513|2484788|N|N|N|N|N| +2484879|AAAAAAAAPIKOFCAA|2091-04-07|2295|9980|766|2091|6|4|7|2|2091|766|9980|Saturday|2091Q2|N|Y|N|2484873|2484962|2484514|2484789|N|N|N|N|N| +2484880|AAAAAAAAAJKOFCAA|2091-04-08|2295|9980|766|2091|0|4|8|2|2091|766|9980|Sunday|2091Q2|N|N|N|2484873|2484962|2484515|2484790|N|N|N|N|N| +2484881|AAAAAAAABJKOFCAA|2091-04-09|2295|9980|766|2091|1|4|9|2|2091|766|9980|Monday|2091Q2|N|N|N|2484873|2484962|2484516|2484791|N|N|N|N|N| +2484882|AAAAAAAACJKOFCAA|2091-04-10|2295|9981|766|2091|2|4|10|2|2091|766|9981|Tuesday|2091Q2|N|N|N|2484873|2484962|2484517|2484792|N|N|N|N|N| +2484883|AAAAAAAADJKOFCAA|2091-04-11|2295|9981|766|2091|3|4|11|2|2091|766|9981|Wednesday|2091Q2|N|N|N|2484873|2484962|2484518|2484793|N|N|N|N|N| +2484884|AAAAAAAAEJKOFCAA|2091-04-12|2295|9981|766|2091|4|4|12|2|2091|766|9981|Thursday|2091Q2|N|N|N|2484873|2484962|2484519|2484794|N|N|N|N|N| +2484885|AAAAAAAAFJKOFCAA|2091-04-13|2295|9981|766|2091|5|4|13|2|2091|766|9981|Friday|2091Q2|N|Y|N|2484873|2484962|2484520|2484795|N|N|N|N|N| +2484886|AAAAAAAAGJKOFCAA|2091-04-14|2295|9981|766|2091|6|4|14|2|2091|766|9981|Saturday|2091Q2|N|Y|N|2484873|2484962|2484521|2484796|N|N|N|N|N| +2484887|AAAAAAAAHJKOFCAA|2091-04-15|2295|9981|766|2091|0|4|15|2|2091|766|9981|Sunday|2091Q2|N|N|N|2484873|2484962|2484522|2484797|N|N|N|N|N| +2484888|AAAAAAAAIJKOFCAA|2091-04-16|2295|9981|766|2091|1|4|16|2|2091|766|9981|Monday|2091Q2|N|N|N|2484873|2484962|2484523|2484798|N|N|N|N|N| +2484889|AAAAAAAAJJKOFCAA|2091-04-17|2295|9982|766|2091|2|4|17|2|2091|766|9982|Tuesday|2091Q2|N|N|N|2484873|2484962|2484524|2484799|N|N|N|N|N| +2484890|AAAAAAAAKJKOFCAA|2091-04-18|2295|9982|766|2091|3|4|18|2|2091|766|9982|Wednesday|2091Q2|N|N|N|2484873|2484962|2484525|2484800|N|N|N|N|N| +2484891|AAAAAAAALJKOFCAA|2091-04-19|2295|9982|766|2091|4|4|19|2|2091|766|9982|Thursday|2091Q2|N|N|N|2484873|2484962|2484526|2484801|N|N|N|N|N| +2484892|AAAAAAAAMJKOFCAA|2091-04-20|2295|9982|766|2091|5|4|20|2|2091|766|9982|Friday|2091Q2|N|Y|N|2484873|2484962|2484527|2484802|N|N|N|N|N| +2484893|AAAAAAAANJKOFCAA|2091-04-21|2295|9982|766|2091|6|4|21|2|2091|766|9982|Saturday|2091Q2|N|Y|N|2484873|2484962|2484528|2484803|N|N|N|N|N| +2484894|AAAAAAAAOJKOFCAA|2091-04-22|2295|9982|766|2091|0|4|22|2|2091|766|9982|Sunday|2091Q2|N|N|N|2484873|2484962|2484529|2484804|N|N|N|N|N| +2484895|AAAAAAAAPJKOFCAA|2091-04-23|2295|9982|766|2091|1|4|23|2|2091|766|9982|Monday|2091Q2|N|N|N|2484873|2484962|2484530|2484805|N|N|N|N|N| +2484896|AAAAAAAAAKKOFCAA|2091-04-24|2295|9983|766|2091|2|4|24|2|2091|766|9983|Tuesday|2091Q2|N|N|N|2484873|2484962|2484531|2484806|N|N|N|N|N| +2484897|AAAAAAAABKKOFCAA|2091-04-25|2295|9983|766|2091|3|4|25|2|2091|766|9983|Wednesday|2091Q2|N|N|N|2484873|2484962|2484532|2484807|N|N|N|N|N| +2484898|AAAAAAAACKKOFCAA|2091-04-26|2295|9983|766|2091|4|4|26|2|2091|766|9983|Thursday|2091Q2|N|N|N|2484873|2484962|2484533|2484808|N|N|N|N|N| +2484899|AAAAAAAADKKOFCAA|2091-04-27|2295|9983|766|2091|5|4|27|2|2091|766|9983|Friday|2091Q2|N|Y|N|2484873|2484962|2484534|2484809|N|N|N|N|N| +2484900|AAAAAAAAEKKOFCAA|2091-04-28|2295|9983|766|2091|6|4|28|2|2091|766|9983|Saturday|2091Q2|N|Y|N|2484873|2484962|2484535|2484810|N|N|N|N|N| +2484901|AAAAAAAAFKKOFCAA|2091-04-29|2295|9983|766|2091|0|4|29|2|2091|766|9983|Sunday|2091Q2|N|N|N|2484873|2484962|2484536|2484811|N|N|N|N|N| +2484902|AAAAAAAAGKKOFCAA|2091-04-30|2295|9983|766|2091|1|4|30|2|2091|766|9983|Monday|2091Q2|N|N|N|2484873|2484962|2484537|2484812|N|N|N|N|N| +2484903|AAAAAAAAHKKOFCAA|2091-05-01|2296|9984|766|2091|2|5|1|2|2091|766|9984|Tuesday|2091Q2|N|N|N|2484903|2485022|2484538|2484813|N|N|N|N|N| +2484904|AAAAAAAAIKKOFCAA|2091-05-02|2296|9984|766|2091|3|5|2|2|2091|766|9984|Wednesday|2091Q2|N|N|N|2484903|2485022|2484539|2484814|N|N|N|N|N| +2484905|AAAAAAAAJKKOFCAA|2091-05-03|2296|9984|766|2091|4|5|3|2|2091|766|9984|Thursday|2091Q2|N|N|N|2484903|2485022|2484540|2484815|N|N|N|N|N| +2484906|AAAAAAAAKKKOFCAA|2091-05-04|2296|9984|766|2091|5|5|4|2|2091|766|9984|Friday|2091Q2|N|Y|N|2484903|2485022|2484541|2484816|N|N|N|N|N| +2484907|AAAAAAAALKKOFCAA|2091-05-05|2296|9984|766|2091|6|5|5|2|2091|766|9984|Saturday|2091Q2|N|Y|N|2484903|2485022|2484542|2484817|N|N|N|N|N| +2484908|AAAAAAAAMKKOFCAA|2091-05-06|2296|9984|766|2091|0|5|6|2|2091|766|9984|Sunday|2091Q2|N|N|N|2484903|2485022|2484543|2484818|N|N|N|N|N| +2484909|AAAAAAAANKKOFCAA|2091-05-07|2296|9984|766|2091|1|5|7|2|2091|766|9984|Monday|2091Q2|N|N|N|2484903|2485022|2484544|2484819|N|N|N|N|N| +2484910|AAAAAAAAOKKOFCAA|2091-05-08|2296|9985|766|2091|2|5|8|2|2091|766|9985|Tuesday|2091Q2|N|N|N|2484903|2485022|2484545|2484820|N|N|N|N|N| +2484911|AAAAAAAAPKKOFCAA|2091-05-09|2296|9985|766|2091|3|5|9|2|2091|766|9985|Wednesday|2091Q2|N|N|N|2484903|2485022|2484546|2484821|N|N|N|N|N| +2484912|AAAAAAAAALKOFCAA|2091-05-10|2296|9985|766|2091|4|5|10|2|2091|766|9985|Thursday|2091Q2|N|N|N|2484903|2485022|2484547|2484822|N|N|N|N|N| +2484913|AAAAAAAABLKOFCAA|2091-05-11|2296|9985|766|2091|5|5|11|2|2091|766|9985|Friday|2091Q2|N|Y|N|2484903|2485022|2484548|2484823|N|N|N|N|N| +2484914|AAAAAAAACLKOFCAA|2091-05-12|2296|9985|766|2091|6|5|12|2|2091|766|9985|Saturday|2091Q2|N|Y|N|2484903|2485022|2484549|2484824|N|N|N|N|N| +2484915|AAAAAAAADLKOFCAA|2091-05-13|2296|9985|766|2091|0|5|13|2|2091|766|9985|Sunday|2091Q2|N|N|N|2484903|2485022|2484550|2484825|N|N|N|N|N| +2484916|AAAAAAAAELKOFCAA|2091-05-14|2296|9985|766|2091|1|5|14|2|2091|766|9985|Monday|2091Q2|N|N|N|2484903|2485022|2484551|2484826|N|N|N|N|N| +2484917|AAAAAAAAFLKOFCAA|2091-05-15|2296|9986|766|2091|2|5|15|2|2091|766|9986|Tuesday|2091Q2|N|N|N|2484903|2485022|2484552|2484827|N|N|N|N|N| +2484918|AAAAAAAAGLKOFCAA|2091-05-16|2296|9986|766|2091|3|5|16|2|2091|766|9986|Wednesday|2091Q2|N|N|N|2484903|2485022|2484553|2484828|N|N|N|N|N| +2484919|AAAAAAAAHLKOFCAA|2091-05-17|2296|9986|766|2091|4|5|17|2|2091|766|9986|Thursday|2091Q2|N|N|N|2484903|2485022|2484554|2484829|N|N|N|N|N| +2484920|AAAAAAAAILKOFCAA|2091-05-18|2296|9986|766|2091|5|5|18|2|2091|766|9986|Friday|2091Q2|N|Y|N|2484903|2485022|2484555|2484830|N|N|N|N|N| +2484921|AAAAAAAAJLKOFCAA|2091-05-19|2296|9986|766|2091|6|5|19|2|2091|766|9986|Saturday|2091Q2|N|Y|N|2484903|2485022|2484556|2484831|N|N|N|N|N| +2484922|AAAAAAAAKLKOFCAA|2091-05-20|2296|9986|766|2091|0|5|20|2|2091|766|9986|Sunday|2091Q2|N|N|N|2484903|2485022|2484557|2484832|N|N|N|N|N| +2484923|AAAAAAAALLKOFCAA|2091-05-21|2296|9986|766|2091|1|5|21|2|2091|766|9986|Monday|2091Q2|N|N|N|2484903|2485022|2484558|2484833|N|N|N|N|N| +2484924|AAAAAAAAMLKOFCAA|2091-05-22|2296|9987|766|2091|2|5|22|2|2091|766|9987|Tuesday|2091Q2|N|N|N|2484903|2485022|2484559|2484834|N|N|N|N|N| +2484925|AAAAAAAANLKOFCAA|2091-05-23|2296|9987|766|2091|3|5|23|2|2091|766|9987|Wednesday|2091Q2|N|N|N|2484903|2485022|2484560|2484835|N|N|N|N|N| +2484926|AAAAAAAAOLKOFCAA|2091-05-24|2296|9987|766|2091|4|5|24|2|2091|766|9987|Thursday|2091Q2|N|N|N|2484903|2485022|2484561|2484836|N|N|N|N|N| +2484927|AAAAAAAAPLKOFCAA|2091-05-25|2296|9987|766|2091|5|5|25|2|2091|766|9987|Friday|2091Q2|N|Y|N|2484903|2485022|2484562|2484837|N|N|N|N|N| +2484928|AAAAAAAAAMKOFCAA|2091-05-26|2296|9987|766|2091|6|5|26|2|2091|766|9987|Saturday|2091Q2|N|Y|N|2484903|2485022|2484563|2484838|N|N|N|N|N| +2484929|AAAAAAAABMKOFCAA|2091-05-27|2296|9987|766|2091|0|5|27|2|2091|766|9987|Sunday|2091Q2|N|N|N|2484903|2485022|2484564|2484839|N|N|N|N|N| +2484930|AAAAAAAACMKOFCAA|2091-05-28|2296|9987|766|2091|1|5|28|2|2091|766|9987|Monday|2091Q2|N|N|N|2484903|2485022|2484565|2484840|N|N|N|N|N| +2484931|AAAAAAAADMKOFCAA|2091-05-29|2296|9988|766|2091|2|5|29|2|2091|766|9988|Tuesday|2091Q2|N|N|N|2484903|2485022|2484566|2484841|N|N|N|N|N| +2484932|AAAAAAAAEMKOFCAA|2091-05-30|2296|9988|766|2091|3|5|30|2|2091|766|9988|Wednesday|2091Q2|N|N|N|2484903|2485022|2484567|2484842|N|N|N|N|N| +2484933|AAAAAAAAFMKOFCAA|2091-05-31|2296|9988|766|2091|4|5|31|2|2091|766|9988|Thursday|2091Q2|N|N|N|2484903|2485022|2484568|2484843|N|N|N|N|N| +2484934|AAAAAAAAGMKOFCAA|2091-06-01|2297|9988|767|2091|5|6|1|2|2091|767|9988|Friday|2091Q2|N|Y|N|2484934|2485084|2484569|2484844|N|N|N|N|N| +2484935|AAAAAAAAHMKOFCAA|2091-06-02|2297|9988|767|2091|6|6|2|2|2091|767|9988|Saturday|2091Q2|N|Y|N|2484934|2485084|2484570|2484845|N|N|N|N|N| +2484936|AAAAAAAAIMKOFCAA|2091-06-03|2297|9988|767|2091|0|6|3|2|2091|767|9988|Sunday|2091Q2|N|N|N|2484934|2485084|2484571|2484846|N|N|N|N|N| +2484937|AAAAAAAAJMKOFCAA|2091-06-04|2297|9988|767|2091|1|6|4|2|2091|767|9988|Monday|2091Q2|N|N|N|2484934|2485084|2484572|2484847|N|N|N|N|N| +2484938|AAAAAAAAKMKOFCAA|2091-06-05|2297|9989|767|2091|2|6|5|2|2091|767|9989|Tuesday|2091Q2|N|N|N|2484934|2485084|2484573|2484848|N|N|N|N|N| +2484939|AAAAAAAALMKOFCAA|2091-06-06|2297|9989|767|2091|3|6|6|2|2091|767|9989|Wednesday|2091Q2|N|N|N|2484934|2485084|2484574|2484849|N|N|N|N|N| +2484940|AAAAAAAAMMKOFCAA|2091-06-07|2297|9989|767|2091|4|6|7|2|2091|767|9989|Thursday|2091Q2|N|N|N|2484934|2485084|2484575|2484850|N|N|N|N|N| +2484941|AAAAAAAANMKOFCAA|2091-06-08|2297|9989|767|2091|5|6|8|2|2091|767|9989|Friday|2091Q2|N|Y|N|2484934|2485084|2484576|2484851|N|N|N|N|N| +2484942|AAAAAAAAOMKOFCAA|2091-06-09|2297|9989|767|2091|6|6|9|2|2091|767|9989|Saturday|2091Q2|N|Y|N|2484934|2485084|2484577|2484852|N|N|N|N|N| +2484943|AAAAAAAAPMKOFCAA|2091-06-10|2297|9989|767|2091|0|6|10|2|2091|767|9989|Sunday|2091Q2|N|N|N|2484934|2485084|2484578|2484853|N|N|N|N|N| +2484944|AAAAAAAAANKOFCAA|2091-06-11|2297|9989|767|2091|1|6|11|2|2091|767|9989|Monday|2091Q2|N|N|N|2484934|2485084|2484579|2484854|N|N|N|N|N| +2484945|AAAAAAAABNKOFCAA|2091-06-12|2297|9990|767|2091|2|6|12|2|2091|767|9990|Tuesday|2091Q2|N|N|N|2484934|2485084|2484580|2484855|N|N|N|N|N| +2484946|AAAAAAAACNKOFCAA|2091-06-13|2297|9990|767|2091|3|6|13|2|2091|767|9990|Wednesday|2091Q2|N|N|N|2484934|2485084|2484581|2484856|N|N|N|N|N| +2484947|AAAAAAAADNKOFCAA|2091-06-14|2297|9990|767|2091|4|6|14|2|2091|767|9990|Thursday|2091Q2|N|N|N|2484934|2485084|2484582|2484857|N|N|N|N|N| +2484948|AAAAAAAAENKOFCAA|2091-06-15|2297|9990|767|2091|5|6|15|2|2091|767|9990|Friday|2091Q2|N|Y|N|2484934|2485084|2484583|2484858|N|N|N|N|N| +2484949|AAAAAAAAFNKOFCAA|2091-06-16|2297|9990|767|2091|6|6|16|2|2091|767|9990|Saturday|2091Q2|N|Y|N|2484934|2485084|2484584|2484859|N|N|N|N|N| +2484950|AAAAAAAAGNKOFCAA|2091-06-17|2297|9990|767|2091|0|6|17|2|2091|767|9990|Sunday|2091Q2|N|N|N|2484934|2485084|2484585|2484860|N|N|N|N|N| +2484951|AAAAAAAAHNKOFCAA|2091-06-18|2297|9990|767|2091|1|6|18|2|2091|767|9990|Monday|2091Q2|N|N|N|2484934|2485084|2484586|2484861|N|N|N|N|N| +2484952|AAAAAAAAINKOFCAA|2091-06-19|2297|9991|767|2091|2|6|19|2|2091|767|9991|Tuesday|2091Q2|N|N|N|2484934|2485084|2484587|2484862|N|N|N|N|N| +2484953|AAAAAAAAJNKOFCAA|2091-06-20|2297|9991|767|2091|3|6|20|2|2091|767|9991|Wednesday|2091Q2|N|N|N|2484934|2485084|2484588|2484863|N|N|N|N|N| +2484954|AAAAAAAAKNKOFCAA|2091-06-21|2297|9991|767|2091|4|6|21|2|2091|767|9991|Thursday|2091Q2|N|N|N|2484934|2485084|2484589|2484864|N|N|N|N|N| +2484955|AAAAAAAALNKOFCAA|2091-06-22|2297|9991|767|2091|5|6|22|2|2091|767|9991|Friday|2091Q2|N|Y|N|2484934|2485084|2484590|2484865|N|N|N|N|N| +2484956|AAAAAAAAMNKOFCAA|2091-06-23|2297|9991|767|2091|6|6|23|2|2091|767|9991|Saturday|2091Q2|N|Y|N|2484934|2485084|2484591|2484866|N|N|N|N|N| +2484957|AAAAAAAANNKOFCAA|2091-06-24|2297|9991|767|2091|0|6|24|2|2091|767|9991|Sunday|2091Q2|N|N|N|2484934|2485084|2484592|2484867|N|N|N|N|N| +2484958|AAAAAAAAONKOFCAA|2091-06-25|2297|9991|767|2091|1|6|25|2|2091|767|9991|Monday|2091Q2|N|N|N|2484934|2485084|2484593|2484868|N|N|N|N|N| +2484959|AAAAAAAAPNKOFCAA|2091-06-26|2297|9992|767|2091|2|6|26|2|2091|767|9992|Tuesday|2091Q2|N|N|N|2484934|2485084|2484594|2484869|N|N|N|N|N| +2484960|AAAAAAAAAOKOFCAA|2091-06-27|2297|9992|767|2091|3|6|27|2|2091|767|9992|Wednesday|2091Q2|N|N|N|2484934|2485084|2484595|2484870|N|N|N|N|N| +2484961|AAAAAAAABOKOFCAA|2091-06-28|2297|9992|767|2091|4|6|28|2|2091|767|9992|Thursday|2091Q2|N|N|N|2484934|2485084|2484596|2484871|N|N|N|N|N| +2484962|AAAAAAAACOKOFCAA|2091-06-29|2297|9992|767|2091|5|6|29|2|2091|767|9992|Friday|2091Q2|N|Y|N|2484934|2485084|2484597|2484872|N|N|N|N|N| +2484963|AAAAAAAADOKOFCAA|2091-06-30|2297|9992|767|2091|6|6|30|2|2091|767|9992|Saturday|2091Q2|N|Y|N|2484934|2485084|2484598|2484873|N|N|N|N|N| +2484964|AAAAAAAAEOKOFCAA|2091-07-01|2298|9992|767|2091|0|7|1|2|2091|767|9992|Sunday|2091Q2|N|N|N|2484964|2485144|2484599|2484873|N|N|N|N|N| +2484965|AAAAAAAAFOKOFCAA|2091-07-02|2298|9992|767|2091|1|7|2|3|2091|767|9992|Monday|2091Q3|N|N|N|2484964|2485144|2484600|2484874|N|N|N|N|N| +2484966|AAAAAAAAGOKOFCAA|2091-07-03|2298|9993|767|2091|2|7|3|3|2091|767|9993|Tuesday|2091Q3|N|N|N|2484964|2485144|2484601|2484875|N|N|N|N|N| +2484967|AAAAAAAAHOKOFCAA|2091-07-04|2298|9993|767|2091|3|7|4|3|2091|767|9993|Wednesday|2091Q3|N|N|N|2484964|2485144|2484602|2484876|N|N|N|N|N| +2484968|AAAAAAAAIOKOFCAA|2091-07-05|2298|9993|767|2091|4|7|5|3|2091|767|9993|Thursday|2091Q3|Y|N|N|2484964|2485144|2484603|2484877|N|N|N|N|N| +2484969|AAAAAAAAJOKOFCAA|2091-07-06|2298|9993|767|2091|5|7|6|3|2091|767|9993|Friday|2091Q3|N|Y|Y|2484964|2485144|2484604|2484878|N|N|N|N|N| +2484970|AAAAAAAAKOKOFCAA|2091-07-07|2298|9993|767|2091|6|7|7|3|2091|767|9993|Saturday|2091Q3|N|Y|N|2484964|2485144|2484605|2484879|N|N|N|N|N| +2484971|AAAAAAAALOKOFCAA|2091-07-08|2298|9993|767|2091|0|7|8|3|2091|767|9993|Sunday|2091Q3|N|N|N|2484964|2485144|2484606|2484880|N|N|N|N|N| +2484972|AAAAAAAAMOKOFCAA|2091-07-09|2298|9993|767|2091|1|7|9|3|2091|767|9993|Monday|2091Q3|N|N|N|2484964|2485144|2484607|2484881|N|N|N|N|N| +2484973|AAAAAAAANOKOFCAA|2091-07-10|2298|9994|767|2091|2|7|10|3|2091|767|9994|Tuesday|2091Q3|N|N|N|2484964|2485144|2484608|2484882|N|N|N|N|N| +2484974|AAAAAAAAOOKOFCAA|2091-07-11|2298|9994|767|2091|3|7|11|3|2091|767|9994|Wednesday|2091Q3|N|N|N|2484964|2485144|2484609|2484883|N|N|N|N|N| +2484975|AAAAAAAAPOKOFCAA|2091-07-12|2298|9994|767|2091|4|7|12|3|2091|767|9994|Thursday|2091Q3|N|N|N|2484964|2485144|2484610|2484884|N|N|N|N|N| +2484976|AAAAAAAAAPKOFCAA|2091-07-13|2298|9994|767|2091|5|7|13|3|2091|767|9994|Friday|2091Q3|N|Y|N|2484964|2485144|2484611|2484885|N|N|N|N|N| +2484977|AAAAAAAABPKOFCAA|2091-07-14|2298|9994|767|2091|6|7|14|3|2091|767|9994|Saturday|2091Q3|N|Y|N|2484964|2485144|2484612|2484886|N|N|N|N|N| +2484978|AAAAAAAACPKOFCAA|2091-07-15|2298|9994|767|2091|0|7|15|3|2091|767|9994|Sunday|2091Q3|N|N|N|2484964|2485144|2484613|2484887|N|N|N|N|N| +2484979|AAAAAAAADPKOFCAA|2091-07-16|2298|9994|767|2091|1|7|16|3|2091|767|9994|Monday|2091Q3|N|N|N|2484964|2485144|2484614|2484888|N|N|N|N|N| +2484980|AAAAAAAAEPKOFCAA|2091-07-17|2298|9995|767|2091|2|7|17|3|2091|767|9995|Tuesday|2091Q3|N|N|N|2484964|2485144|2484615|2484889|N|N|N|N|N| +2484981|AAAAAAAAFPKOFCAA|2091-07-18|2298|9995|767|2091|3|7|18|3|2091|767|9995|Wednesday|2091Q3|N|N|N|2484964|2485144|2484616|2484890|N|N|N|N|N| +2484982|AAAAAAAAGPKOFCAA|2091-07-19|2298|9995|767|2091|4|7|19|3|2091|767|9995|Thursday|2091Q3|N|N|N|2484964|2485144|2484617|2484891|N|N|N|N|N| +2484983|AAAAAAAAHPKOFCAA|2091-07-20|2298|9995|767|2091|5|7|20|3|2091|767|9995|Friday|2091Q3|N|Y|N|2484964|2485144|2484618|2484892|N|N|N|N|N| +2484984|AAAAAAAAIPKOFCAA|2091-07-21|2298|9995|767|2091|6|7|21|3|2091|767|9995|Saturday|2091Q3|N|Y|N|2484964|2485144|2484619|2484893|N|N|N|N|N| +2484985|AAAAAAAAJPKOFCAA|2091-07-22|2298|9995|767|2091|0|7|22|3|2091|767|9995|Sunday|2091Q3|N|N|N|2484964|2485144|2484620|2484894|N|N|N|N|N| +2484986|AAAAAAAAKPKOFCAA|2091-07-23|2298|9995|767|2091|1|7|23|3|2091|767|9995|Monday|2091Q3|N|N|N|2484964|2485144|2484621|2484895|N|N|N|N|N| +2484987|AAAAAAAALPKOFCAA|2091-07-24|2298|9996|767|2091|2|7|24|3|2091|767|9996|Tuesday|2091Q3|N|N|N|2484964|2485144|2484622|2484896|N|N|N|N|N| +2484988|AAAAAAAAMPKOFCAA|2091-07-25|2298|9996|767|2091|3|7|25|3|2091|767|9996|Wednesday|2091Q3|N|N|N|2484964|2485144|2484623|2484897|N|N|N|N|N| +2484989|AAAAAAAANPKOFCAA|2091-07-26|2298|9996|767|2091|4|7|26|3|2091|767|9996|Thursday|2091Q3|N|N|N|2484964|2485144|2484624|2484898|N|N|N|N|N| +2484990|AAAAAAAAOPKOFCAA|2091-07-27|2298|9996|767|2091|5|7|27|3|2091|767|9996|Friday|2091Q3|N|Y|N|2484964|2485144|2484625|2484899|N|N|N|N|N| +2484991|AAAAAAAAPPKOFCAA|2091-07-28|2298|9996|767|2091|6|7|28|3|2091|767|9996|Saturday|2091Q3|N|Y|N|2484964|2485144|2484626|2484900|N|N|N|N|N| +2484992|AAAAAAAAAALOFCAA|2091-07-29|2298|9996|767|2091|0|7|29|3|2091|767|9996|Sunday|2091Q3|N|N|N|2484964|2485144|2484627|2484901|N|N|N|N|N| +2484993|AAAAAAAABALOFCAA|2091-07-30|2298|9996|767|2091|1|7|30|3|2091|767|9996|Monday|2091Q3|N|N|N|2484964|2485144|2484628|2484902|N|N|N|N|N| +2484994|AAAAAAAACALOFCAA|2091-07-31|2298|9997|767|2091|2|7|31|3|2091|767|9997|Tuesday|2091Q3|N|N|N|2484964|2485144|2484629|2484903|N|N|N|N|N| +2484995|AAAAAAAADALOFCAA|2091-08-01|2299|9997|767|2091|3|8|1|3|2091|767|9997|Wednesday|2091Q3|N|N|N|2484995|2485206|2484630|2484904|N|N|N|N|N| +2484996|AAAAAAAAEALOFCAA|2091-08-02|2299|9997|767|2091|4|8|2|3|2091|767|9997|Thursday|2091Q3|N|N|N|2484995|2485206|2484631|2484905|N|N|N|N|N| +2484997|AAAAAAAAFALOFCAA|2091-08-03|2299|9997|767|2091|5|8|3|3|2091|767|9997|Friday|2091Q3|N|Y|N|2484995|2485206|2484632|2484906|N|N|N|N|N| +2484998|AAAAAAAAGALOFCAA|2091-08-04|2299|9997|767|2091|6|8|4|3|2091|767|9997|Saturday|2091Q3|N|Y|N|2484995|2485206|2484633|2484907|N|N|N|N|N| +2484999|AAAAAAAAHALOFCAA|2091-08-05|2299|9997|767|2091|0|8|5|3|2091|767|9997|Sunday|2091Q3|N|N|N|2484995|2485206|2484634|2484908|N|N|N|N|N| +2485000|AAAAAAAAIALOFCAA|2091-08-06|2299|9997|767|2091|1|8|6|3|2091|767|9997|Monday|2091Q3|N|N|N|2484995|2485206|2484635|2484909|N|N|N|N|N| +2485001|AAAAAAAAJALOFCAA|2091-08-07|2299|9998|767|2091|2|8|7|3|2091|767|9998|Tuesday|2091Q3|N|N|N|2484995|2485206|2484636|2484910|N|N|N|N|N| +2485002|AAAAAAAAKALOFCAA|2091-08-08|2299|9998|767|2091|3|8|8|3|2091|767|9998|Wednesday|2091Q3|N|N|N|2484995|2485206|2484637|2484911|N|N|N|N|N| +2485003|AAAAAAAALALOFCAA|2091-08-09|2299|9998|767|2091|4|8|9|3|2091|767|9998|Thursday|2091Q3|N|N|N|2484995|2485206|2484638|2484912|N|N|N|N|N| +2485004|AAAAAAAAMALOFCAA|2091-08-10|2299|9998|767|2091|5|8|10|3|2091|767|9998|Friday|2091Q3|N|Y|N|2484995|2485206|2484639|2484913|N|N|N|N|N| +2485005|AAAAAAAANALOFCAA|2091-08-11|2299|9998|767|2091|6|8|11|3|2091|767|9998|Saturday|2091Q3|N|Y|N|2484995|2485206|2484640|2484914|N|N|N|N|N| +2485006|AAAAAAAAOALOFCAA|2091-08-12|2299|9998|767|2091|0|8|12|3|2091|767|9998|Sunday|2091Q3|N|N|N|2484995|2485206|2484641|2484915|N|N|N|N|N| +2485007|AAAAAAAAPALOFCAA|2091-08-13|2299|9998|767|2091|1|8|13|3|2091|767|9998|Monday|2091Q3|N|N|N|2484995|2485206|2484642|2484916|N|N|N|N|N| +2485008|AAAAAAAAABLOFCAA|2091-08-14|2299|9999|767|2091|2|8|14|3|2091|767|9999|Tuesday|2091Q3|N|N|N|2484995|2485206|2484643|2484917|N|N|N|N|N| +2485009|AAAAAAAABBLOFCAA|2091-08-15|2299|9999|767|2091|3|8|15|3|2091|767|9999|Wednesday|2091Q3|N|N|N|2484995|2485206|2484644|2484918|N|N|N|N|N| +2485010|AAAAAAAACBLOFCAA|2091-08-16|2299|9999|767|2091|4|8|16|3|2091|767|9999|Thursday|2091Q3|N|N|N|2484995|2485206|2484645|2484919|N|N|N|N|N| +2485011|AAAAAAAADBLOFCAA|2091-08-17|2299|9999|767|2091|5|8|17|3|2091|767|9999|Friday|2091Q3|N|Y|N|2484995|2485206|2484646|2484920|N|N|N|N|N| +2485012|AAAAAAAAEBLOFCAA|2091-08-18|2299|9999|767|2091|6|8|18|3|2091|767|9999|Saturday|2091Q3|N|Y|N|2484995|2485206|2484647|2484921|N|N|N|N|N| +2485013|AAAAAAAAFBLOFCAA|2091-08-19|2299|9999|767|2091|0|8|19|3|2091|767|9999|Sunday|2091Q3|N|N|N|2484995|2485206|2484648|2484922|N|N|N|N|N| +2485014|AAAAAAAAGBLOFCAA|2091-08-20|2299|9999|767|2091|1|8|20|3|2091|767|9999|Monday|2091Q3|N|N|N|2484995|2485206|2484649|2484923|N|N|N|N|N| +2485015|AAAAAAAAHBLOFCAA|2091-08-21|2299|10000|767|2091|2|8|21|3|2091|767|10000|Tuesday|2091Q3|N|N|N|2484995|2485206|2484650|2484924|N|N|N|N|N| +2485016|AAAAAAAAIBLOFCAA|2091-08-22|2299|10000|767|2091|3|8|22|3|2091|767|10000|Wednesday|2091Q3|N|N|N|2484995|2485206|2484651|2484925|N|N|N|N|N| +2485017|AAAAAAAAJBLOFCAA|2091-08-23|2299|10000|767|2091|4|8|23|3|2091|767|10000|Thursday|2091Q3|N|N|N|2484995|2485206|2484652|2484926|N|N|N|N|N| +2485018|AAAAAAAAKBLOFCAA|2091-08-24|2299|10000|767|2091|5|8|24|3|2091|767|10000|Friday|2091Q3|N|Y|N|2484995|2485206|2484653|2484927|N|N|N|N|N| +2485019|AAAAAAAALBLOFCAA|2091-08-25|2299|10000|767|2091|6|8|25|3|2091|767|10000|Saturday|2091Q3|N|Y|N|2484995|2485206|2484654|2484928|N|N|N|N|N| +2485020|AAAAAAAAMBLOFCAA|2091-08-26|2299|10000|767|2091|0|8|26|3|2091|767|10000|Sunday|2091Q3|N|N|N|2484995|2485206|2484655|2484929|N|N|N|N|N| +2485021|AAAAAAAANBLOFCAA|2091-08-27|2299|10000|767|2091|1|8|27|3|2091|767|10000|Monday|2091Q3|N|N|N|2484995|2485206|2484656|2484930|N|N|N|N|N| +2485022|AAAAAAAAOBLOFCAA|2091-08-28|2299|10001|767|2091|2|8|28|3|2091|767|10001|Tuesday|2091Q3|N|N|N|2484995|2485206|2484657|2484931|N|N|N|N|N| +2485023|AAAAAAAAPBLOFCAA|2091-08-29|2299|10001|767|2091|3|8|29|3|2091|767|10001|Wednesday|2091Q3|N|N|N|2484995|2485206|2484658|2484932|N|N|N|N|N| +2485024|AAAAAAAAACLOFCAA|2091-08-30|2299|10001|767|2091|4|8|30|3|2091|767|10001|Thursday|2091Q3|N|N|N|2484995|2485206|2484659|2484933|N|N|N|N|N| +2485025|AAAAAAAABCLOFCAA|2091-08-31|2299|10001|767|2091|5|8|31|3|2091|767|10001|Friday|2091Q3|N|Y|N|2484995|2485206|2484660|2484934|N|N|N|N|N| +2485026|AAAAAAAACCLOFCAA|2091-09-01|2300|10001|768|2091|6|9|1|3|2091|768|10001|Saturday|2091Q3|N|Y|N|2485026|2485268|2484661|2484935|N|N|N|N|N| +2485027|AAAAAAAADCLOFCAA|2091-09-02|2300|10001|768|2091|0|9|2|3|2091|768|10001|Sunday|2091Q3|N|N|N|2485026|2485268|2484662|2484936|N|N|N|N|N| +2485028|AAAAAAAAECLOFCAA|2091-09-03|2300|10001|768|2091|1|9|3|3|2091|768|10001|Monday|2091Q3|N|N|N|2485026|2485268|2484663|2484937|N|N|N|N|N| +2485029|AAAAAAAAFCLOFCAA|2091-09-04|2300|10002|768|2091|2|9|4|3|2091|768|10002|Tuesday|2091Q3|N|N|N|2485026|2485268|2484664|2484938|N|N|N|N|N| +2485030|AAAAAAAAGCLOFCAA|2091-09-05|2300|10002|768|2091|3|9|5|3|2091|768|10002|Wednesday|2091Q3|N|N|N|2485026|2485268|2484665|2484939|N|N|N|N|N| +2485031|AAAAAAAAHCLOFCAA|2091-09-06|2300|10002|768|2091|4|9|6|3|2091|768|10002|Thursday|2091Q3|N|N|N|2485026|2485268|2484666|2484940|N|N|N|N|N| +2485032|AAAAAAAAICLOFCAA|2091-09-07|2300|10002|768|2091|5|9|7|3|2091|768|10002|Friday|2091Q3|N|Y|N|2485026|2485268|2484667|2484941|N|N|N|N|N| +2485033|AAAAAAAAJCLOFCAA|2091-09-08|2300|10002|768|2091|6|9|8|3|2091|768|10002|Saturday|2091Q3|N|Y|N|2485026|2485268|2484668|2484942|N|N|N|N|N| +2485034|AAAAAAAAKCLOFCAA|2091-09-09|2300|10002|768|2091|0|9|9|3|2091|768|10002|Sunday|2091Q3|N|N|N|2485026|2485268|2484669|2484943|N|N|N|N|N| +2485035|AAAAAAAALCLOFCAA|2091-09-10|2300|10002|768|2091|1|9|10|3|2091|768|10002|Monday|2091Q3|N|N|N|2485026|2485268|2484670|2484944|N|N|N|N|N| +2485036|AAAAAAAAMCLOFCAA|2091-09-11|2300|10003|768|2091|2|9|11|3|2091|768|10003|Tuesday|2091Q3|N|N|N|2485026|2485268|2484671|2484945|N|N|N|N|N| +2485037|AAAAAAAANCLOFCAA|2091-09-12|2300|10003|768|2091|3|9|12|3|2091|768|10003|Wednesday|2091Q3|N|N|N|2485026|2485268|2484672|2484946|N|N|N|N|N| +2485038|AAAAAAAAOCLOFCAA|2091-09-13|2300|10003|768|2091|4|9|13|3|2091|768|10003|Thursday|2091Q3|N|N|N|2485026|2485268|2484673|2484947|N|N|N|N|N| +2485039|AAAAAAAAPCLOFCAA|2091-09-14|2300|10003|768|2091|5|9|14|3|2091|768|10003|Friday|2091Q3|N|Y|N|2485026|2485268|2484674|2484948|N|N|N|N|N| +2485040|AAAAAAAAADLOFCAA|2091-09-15|2300|10003|768|2091|6|9|15|3|2091|768|10003|Saturday|2091Q3|N|Y|N|2485026|2485268|2484675|2484949|N|N|N|N|N| +2485041|AAAAAAAABDLOFCAA|2091-09-16|2300|10003|768|2091|0|9|16|3|2091|768|10003|Sunday|2091Q3|N|N|N|2485026|2485268|2484676|2484950|N|N|N|N|N| +2485042|AAAAAAAACDLOFCAA|2091-09-17|2300|10003|768|2091|1|9|17|3|2091|768|10003|Monday|2091Q3|N|N|N|2485026|2485268|2484677|2484951|N|N|N|N|N| +2485043|AAAAAAAADDLOFCAA|2091-09-18|2300|10004|768|2091|2|9|18|3|2091|768|10004|Tuesday|2091Q3|N|N|N|2485026|2485268|2484678|2484952|N|N|N|N|N| +2485044|AAAAAAAAEDLOFCAA|2091-09-19|2300|10004|768|2091|3|9|19|3|2091|768|10004|Wednesday|2091Q3|N|N|N|2485026|2485268|2484679|2484953|N|N|N|N|N| +2485045|AAAAAAAAFDLOFCAA|2091-09-20|2300|10004|768|2091|4|9|20|3|2091|768|10004|Thursday|2091Q3|N|N|N|2485026|2485268|2484680|2484954|N|N|N|N|N| +2485046|AAAAAAAAGDLOFCAA|2091-09-21|2300|10004|768|2091|5|9|21|3|2091|768|10004|Friday|2091Q3|N|Y|N|2485026|2485268|2484681|2484955|N|N|N|N|N| +2485047|AAAAAAAAHDLOFCAA|2091-09-22|2300|10004|768|2091|6|9|22|3|2091|768|10004|Saturday|2091Q3|N|Y|N|2485026|2485268|2484682|2484956|N|N|N|N|N| +2485048|AAAAAAAAIDLOFCAA|2091-09-23|2300|10004|768|2091|0|9|23|3|2091|768|10004|Sunday|2091Q3|N|N|N|2485026|2485268|2484683|2484957|N|N|N|N|N| +2485049|AAAAAAAAJDLOFCAA|2091-09-24|2300|10004|768|2091|1|9|24|3|2091|768|10004|Monday|2091Q3|N|N|N|2485026|2485268|2484684|2484958|N|N|N|N|N| +2485050|AAAAAAAAKDLOFCAA|2091-09-25|2300|10005|768|2091|2|9|25|3|2091|768|10005|Tuesday|2091Q3|N|N|N|2485026|2485268|2484685|2484959|N|N|N|N|N| +2485051|AAAAAAAALDLOFCAA|2091-09-26|2300|10005|768|2091|3|9|26|3|2091|768|10005|Wednesday|2091Q3|N|N|N|2485026|2485268|2484686|2484960|N|N|N|N|N| +2485052|AAAAAAAAMDLOFCAA|2091-09-27|2300|10005|768|2091|4|9|27|3|2091|768|10005|Thursday|2091Q3|N|N|N|2485026|2485268|2484687|2484961|N|N|N|N|N| +2485053|AAAAAAAANDLOFCAA|2091-09-28|2300|10005|768|2091|5|9|28|3|2091|768|10005|Friday|2091Q3|N|Y|N|2485026|2485268|2484688|2484962|N|N|N|N|N| +2485054|AAAAAAAAODLOFCAA|2091-09-29|2300|10005|768|2091|6|9|29|3|2091|768|10005|Saturday|2091Q3|N|Y|N|2485026|2485268|2484689|2484963|N|N|N|N|N| +2485055|AAAAAAAAPDLOFCAA|2091-09-30|2300|10005|768|2091|0|9|30|3|2091|768|10005|Sunday|2091Q3|N|N|N|2485026|2485268|2484690|2484964|N|N|N|N|N| +2485056|AAAAAAAAAELOFCAA|2091-10-01|2301|10005|768|2091|1|10|1|3|2091|768|10005|Monday|2091Q3|N|N|N|2485056|2485328|2484691|2484964|N|N|N|N|N| +2485057|AAAAAAAABELOFCAA|2091-10-02|2301|10006|768|2091|2|10|2|4|2091|768|10006|Tuesday|2091Q4|N|N|N|2485056|2485328|2484692|2484965|N|N|N|N|N| +2485058|AAAAAAAACELOFCAA|2091-10-03|2301|10006|768|2091|3|10|3|4|2091|768|10006|Wednesday|2091Q4|N|N|N|2485056|2485328|2484693|2484966|N|N|N|N|N| +2485059|AAAAAAAADELOFCAA|2091-10-04|2301|10006|768|2091|4|10|4|4|2091|768|10006|Thursday|2091Q4|N|N|N|2485056|2485328|2484694|2484967|N|N|N|N|N| +2485060|AAAAAAAAEELOFCAA|2091-10-05|2301|10006|768|2091|5|10|5|4|2091|768|10006|Friday|2091Q4|N|Y|N|2485056|2485328|2484695|2484968|N|N|N|N|N| +2485061|AAAAAAAAFELOFCAA|2091-10-06|2301|10006|768|2091|6|10|6|4|2091|768|10006|Saturday|2091Q4|N|Y|N|2485056|2485328|2484696|2484969|N|N|N|N|N| +2485062|AAAAAAAAGELOFCAA|2091-10-07|2301|10006|768|2091|0|10|7|4|2091|768|10006|Sunday|2091Q4|N|N|N|2485056|2485328|2484697|2484970|N|N|N|N|N| +2485063|AAAAAAAAHELOFCAA|2091-10-08|2301|10006|768|2091|1|10|8|4|2091|768|10006|Monday|2091Q4|N|N|N|2485056|2485328|2484698|2484971|N|N|N|N|N| +2485064|AAAAAAAAIELOFCAA|2091-10-09|2301|10007|768|2091|2|10|9|4|2091|768|10007|Tuesday|2091Q4|N|N|N|2485056|2485328|2484699|2484972|N|N|N|N|N| +2485065|AAAAAAAAJELOFCAA|2091-10-10|2301|10007|768|2091|3|10|10|4|2091|768|10007|Wednesday|2091Q4|N|N|N|2485056|2485328|2484700|2484973|N|N|N|N|N| +2485066|AAAAAAAAKELOFCAA|2091-10-11|2301|10007|768|2091|4|10|11|4|2091|768|10007|Thursday|2091Q4|N|N|N|2485056|2485328|2484701|2484974|N|N|N|N|N| +2485067|AAAAAAAALELOFCAA|2091-10-12|2301|10007|768|2091|5|10|12|4|2091|768|10007|Friday|2091Q4|N|Y|N|2485056|2485328|2484702|2484975|N|N|N|N|N| +2485068|AAAAAAAAMELOFCAA|2091-10-13|2301|10007|768|2091|6|10|13|4|2091|768|10007|Saturday|2091Q4|N|Y|N|2485056|2485328|2484703|2484976|N|N|N|N|N| +2485069|AAAAAAAANELOFCAA|2091-10-14|2301|10007|768|2091|0|10|14|4|2091|768|10007|Sunday|2091Q4|N|N|N|2485056|2485328|2484704|2484977|N|N|N|N|N| +2485070|AAAAAAAAOELOFCAA|2091-10-15|2301|10007|768|2091|1|10|15|4|2091|768|10007|Monday|2091Q4|N|N|N|2485056|2485328|2484705|2484978|N|N|N|N|N| +2485071|AAAAAAAAPELOFCAA|2091-10-16|2301|10008|768|2091|2|10|16|4|2091|768|10008|Tuesday|2091Q4|N|N|N|2485056|2485328|2484706|2484979|N|N|N|N|N| +2485072|AAAAAAAAAFLOFCAA|2091-10-17|2301|10008|768|2091|3|10|17|4|2091|768|10008|Wednesday|2091Q4|N|N|N|2485056|2485328|2484707|2484980|N|N|N|N|N| +2485073|AAAAAAAABFLOFCAA|2091-10-18|2301|10008|768|2091|4|10|18|4|2091|768|10008|Thursday|2091Q4|N|N|N|2485056|2485328|2484708|2484981|N|N|N|N|N| +2485074|AAAAAAAACFLOFCAA|2091-10-19|2301|10008|768|2091|5|10|19|4|2091|768|10008|Friday|2091Q4|N|Y|N|2485056|2485328|2484709|2484982|N|N|N|N|N| +2485075|AAAAAAAADFLOFCAA|2091-10-20|2301|10008|768|2091|6|10|20|4|2091|768|10008|Saturday|2091Q4|N|Y|N|2485056|2485328|2484710|2484983|N|N|N|N|N| +2485076|AAAAAAAAEFLOFCAA|2091-10-21|2301|10008|768|2091|0|10|21|4|2091|768|10008|Sunday|2091Q4|N|N|N|2485056|2485328|2484711|2484984|N|N|N|N|N| +2485077|AAAAAAAAFFLOFCAA|2091-10-22|2301|10008|768|2091|1|10|22|4|2091|768|10008|Monday|2091Q4|N|N|N|2485056|2485328|2484712|2484985|N|N|N|N|N| +2485078|AAAAAAAAGFLOFCAA|2091-10-23|2301|10009|768|2091|2|10|23|4|2091|768|10009|Tuesday|2091Q4|N|N|N|2485056|2485328|2484713|2484986|N|N|N|N|N| +2485079|AAAAAAAAHFLOFCAA|2091-10-24|2301|10009|768|2091|3|10|24|4|2091|768|10009|Wednesday|2091Q4|N|N|N|2485056|2485328|2484714|2484987|N|N|N|N|N| +2485080|AAAAAAAAIFLOFCAA|2091-10-25|2301|10009|768|2091|4|10|25|4|2091|768|10009|Thursday|2091Q4|N|N|N|2485056|2485328|2484715|2484988|N|N|N|N|N| +2485081|AAAAAAAAJFLOFCAA|2091-10-26|2301|10009|768|2091|5|10|26|4|2091|768|10009|Friday|2091Q4|N|Y|N|2485056|2485328|2484716|2484989|N|N|N|N|N| +2485082|AAAAAAAAKFLOFCAA|2091-10-27|2301|10009|768|2091|6|10|27|4|2091|768|10009|Saturday|2091Q4|N|Y|N|2485056|2485328|2484717|2484990|N|N|N|N|N| +2485083|AAAAAAAALFLOFCAA|2091-10-28|2301|10009|768|2091|0|10|28|4|2091|768|10009|Sunday|2091Q4|N|N|N|2485056|2485328|2484718|2484991|N|N|N|N|N| +2485084|AAAAAAAAMFLOFCAA|2091-10-29|2301|10009|768|2091|1|10|29|4|2091|768|10009|Monday|2091Q4|N|N|N|2485056|2485328|2484719|2484992|N|N|N|N|N| +2485085|AAAAAAAANFLOFCAA|2091-10-30|2301|10010|768|2091|2|10|30|4|2091|768|10010|Tuesday|2091Q4|N|N|N|2485056|2485328|2484720|2484993|N|N|N|N|N| +2485086|AAAAAAAAOFLOFCAA|2091-10-31|2301|10010|768|2091|3|10|31|4|2091|768|10010|Wednesday|2091Q4|N|N|N|2485056|2485328|2484721|2484994|N|N|N|N|N| +2485087|AAAAAAAAPFLOFCAA|2091-11-01|2302|10010|768|2091|4|11|1|4|2091|768|10010|Thursday|2091Q4|N|N|N|2485087|2485390|2484722|2484995|N|N|N|N|N| +2485088|AAAAAAAAAGLOFCAA|2091-11-02|2302|10010|768|2091|5|11|2|4|2091|768|10010|Friday|2091Q4|N|Y|N|2485087|2485390|2484723|2484996|N|N|N|N|N| +2485089|AAAAAAAABGLOFCAA|2091-11-03|2302|10010|768|2091|6|11|3|4|2091|768|10010|Saturday|2091Q4|N|Y|N|2485087|2485390|2484724|2484997|N|N|N|N|N| +2485090|AAAAAAAACGLOFCAA|2091-11-04|2302|10010|768|2091|0|11|4|4|2091|768|10010|Sunday|2091Q4|N|N|N|2485087|2485390|2484725|2484998|N|N|N|N|N| +2485091|AAAAAAAADGLOFCAA|2091-11-05|2302|10010|768|2091|1|11|5|4|2091|768|10010|Monday|2091Q4|N|N|N|2485087|2485390|2484726|2484999|N|N|N|N|N| +2485092|AAAAAAAAEGLOFCAA|2091-11-06|2302|10011|768|2091|2|11|6|4|2091|768|10011|Tuesday|2091Q4|N|N|N|2485087|2485390|2484727|2485000|N|N|N|N|N| +2485093|AAAAAAAAFGLOFCAA|2091-11-07|2302|10011|768|2091|3|11|7|4|2091|768|10011|Wednesday|2091Q4|N|N|N|2485087|2485390|2484728|2485001|N|N|N|N|N| +2485094|AAAAAAAAGGLOFCAA|2091-11-08|2302|10011|768|2091|4|11|8|4|2091|768|10011|Thursday|2091Q4|N|N|N|2485087|2485390|2484729|2485002|N|N|N|N|N| +2485095|AAAAAAAAHGLOFCAA|2091-11-09|2302|10011|768|2091|5|11|9|4|2091|768|10011|Friday|2091Q4|N|Y|N|2485087|2485390|2484730|2485003|N|N|N|N|N| +2485096|AAAAAAAAIGLOFCAA|2091-11-10|2302|10011|768|2091|6|11|10|4|2091|768|10011|Saturday|2091Q4|N|Y|N|2485087|2485390|2484731|2485004|N|N|N|N|N| +2485097|AAAAAAAAJGLOFCAA|2091-11-11|2302|10011|768|2091|0|11|11|4|2091|768|10011|Sunday|2091Q4|N|N|N|2485087|2485390|2484732|2485005|N|N|N|N|N| +2485098|AAAAAAAAKGLOFCAA|2091-11-12|2302|10011|768|2091|1|11|12|4|2091|768|10011|Monday|2091Q4|N|N|N|2485087|2485390|2484733|2485006|N|N|N|N|N| +2485099|AAAAAAAALGLOFCAA|2091-11-13|2302|10012|768|2091|2|11|13|4|2091|768|10012|Tuesday|2091Q4|N|N|N|2485087|2485390|2484734|2485007|N|N|N|N|N| +2485100|AAAAAAAAMGLOFCAA|2091-11-14|2302|10012|768|2091|3|11|14|4|2091|768|10012|Wednesday|2091Q4|N|N|N|2485087|2485390|2484735|2485008|N|N|N|N|N| +2485101|AAAAAAAANGLOFCAA|2091-11-15|2302|10012|768|2091|4|11|15|4|2091|768|10012|Thursday|2091Q4|N|N|N|2485087|2485390|2484736|2485009|N|N|N|N|N| +2485102|AAAAAAAAOGLOFCAA|2091-11-16|2302|10012|768|2091|5|11|16|4|2091|768|10012|Friday|2091Q4|N|Y|N|2485087|2485390|2484737|2485010|N|N|N|N|N| +2485103|AAAAAAAAPGLOFCAA|2091-11-17|2302|10012|768|2091|6|11|17|4|2091|768|10012|Saturday|2091Q4|N|Y|N|2485087|2485390|2484738|2485011|N|N|N|N|N| +2485104|AAAAAAAAAHLOFCAA|2091-11-18|2302|10012|768|2091|0|11|18|4|2091|768|10012|Sunday|2091Q4|N|N|N|2485087|2485390|2484739|2485012|N|N|N|N|N| +2485105|AAAAAAAABHLOFCAA|2091-11-19|2302|10012|768|2091|1|11|19|4|2091|768|10012|Monday|2091Q4|N|N|N|2485087|2485390|2484740|2485013|N|N|N|N|N| +2485106|AAAAAAAACHLOFCAA|2091-11-20|2302|10013|768|2091|2|11|20|4|2091|768|10013|Tuesday|2091Q4|N|N|N|2485087|2485390|2484741|2485014|N|N|N|N|N| +2485107|AAAAAAAADHLOFCAA|2091-11-21|2302|10013|768|2091|3|11|21|4|2091|768|10013|Wednesday|2091Q4|N|N|N|2485087|2485390|2484742|2485015|N|N|N|N|N| +2485108|AAAAAAAAEHLOFCAA|2091-11-22|2302|10013|768|2091|4|11|22|4|2091|768|10013|Thursday|2091Q4|N|N|N|2485087|2485390|2484743|2485016|N|N|N|N|N| +2485109|AAAAAAAAFHLOFCAA|2091-11-23|2302|10013|768|2091|5|11|23|4|2091|768|10013|Friday|2091Q4|N|Y|N|2485087|2485390|2484744|2485017|N|N|N|N|N| +2485110|AAAAAAAAGHLOFCAA|2091-11-24|2302|10013|768|2091|6|11|24|4|2091|768|10013|Saturday|2091Q4|N|Y|N|2485087|2485390|2484745|2485018|N|N|N|N|N| +2485111|AAAAAAAAHHLOFCAA|2091-11-25|2302|10013|768|2091|0|11|25|4|2091|768|10013|Sunday|2091Q4|N|N|N|2485087|2485390|2484746|2485019|N|N|N|N|N| +2485112|AAAAAAAAIHLOFCAA|2091-11-26|2302|10013|768|2091|1|11|26|4|2091|768|10013|Monday|2091Q4|N|N|N|2485087|2485390|2484747|2485020|N|N|N|N|N| +2485113|AAAAAAAAJHLOFCAA|2091-11-27|2302|10014|768|2091|2|11|27|4|2091|768|10014|Tuesday|2091Q4|N|N|N|2485087|2485390|2484748|2485021|N|N|N|N|N| +2485114|AAAAAAAAKHLOFCAA|2091-11-28|2302|10014|768|2091|3|11|28|4|2091|768|10014|Wednesday|2091Q4|N|N|N|2485087|2485390|2484749|2485022|N|N|N|N|N| +2485115|AAAAAAAALHLOFCAA|2091-11-29|2302|10014|768|2091|4|11|29|4|2091|768|10014|Thursday|2091Q4|N|N|N|2485087|2485390|2484750|2485023|N|N|N|N|N| +2485116|AAAAAAAAMHLOFCAA|2091-11-30|2302|10014|768|2091|5|11|30|4|2091|768|10014|Friday|2091Q4|N|Y|N|2485087|2485390|2484751|2485024|N|N|N|N|N| +2485117|AAAAAAAANHLOFCAA|2091-12-01|2303|10014|769|2091|6|12|1|4|2091|769|10014|Saturday|2091Q4|N|Y|N|2485117|2485450|2484752|2485025|N|N|N|N|N| +2485118|AAAAAAAAOHLOFCAA|2091-12-02|2303|10014|769|2091|0|12|2|4|2091|769|10014|Sunday|2091Q4|N|N|N|2485117|2485450|2484753|2485026|N|N|N|N|N| +2485119|AAAAAAAAPHLOFCAA|2091-12-03|2303|10014|769|2091|1|12|3|4|2091|769|10014|Monday|2091Q4|N|N|N|2485117|2485450|2484754|2485027|N|N|N|N|N| +2485120|AAAAAAAAAILOFCAA|2091-12-04|2303|10015|769|2091|2|12|4|4|2091|769|10015|Tuesday|2091Q4|N|N|N|2485117|2485450|2484755|2485028|N|N|N|N|N| +2485121|AAAAAAAABILOFCAA|2091-12-05|2303|10015|769|2091|3|12|5|4|2091|769|10015|Wednesday|2091Q4|N|N|N|2485117|2485450|2484756|2485029|N|N|N|N|N| +2485122|AAAAAAAACILOFCAA|2091-12-06|2303|10015|769|2091|4|12|6|4|2091|769|10015|Thursday|2091Q4|N|N|N|2485117|2485450|2484757|2485030|N|N|N|N|N| +2485123|AAAAAAAADILOFCAA|2091-12-07|2303|10015|769|2091|5|12|7|4|2091|769|10015|Friday|2091Q4|N|Y|N|2485117|2485450|2484758|2485031|N|N|N|N|N| +2485124|AAAAAAAAEILOFCAA|2091-12-08|2303|10015|769|2091|6|12|8|4|2091|769|10015|Saturday|2091Q4|N|Y|N|2485117|2485450|2484759|2485032|N|N|N|N|N| +2485125|AAAAAAAAFILOFCAA|2091-12-09|2303|10015|769|2091|0|12|9|4|2091|769|10015|Sunday|2091Q4|N|N|N|2485117|2485450|2484760|2485033|N|N|N|N|N| +2485126|AAAAAAAAGILOFCAA|2091-12-10|2303|10015|769|2091|1|12|10|4|2091|769|10015|Monday|2091Q4|N|N|N|2485117|2485450|2484761|2485034|N|N|N|N|N| +2485127|AAAAAAAAHILOFCAA|2091-12-11|2303|10016|769|2091|2|12|11|4|2091|769|10016|Tuesday|2091Q4|N|N|N|2485117|2485450|2484762|2485035|N|N|N|N|N| +2485128|AAAAAAAAIILOFCAA|2091-12-12|2303|10016|769|2091|3|12|12|4|2091|769|10016|Wednesday|2091Q4|N|N|N|2485117|2485450|2484763|2485036|N|N|N|N|N| +2485129|AAAAAAAAJILOFCAA|2091-12-13|2303|10016|769|2091|4|12|13|4|2091|769|10016|Thursday|2091Q4|N|N|N|2485117|2485450|2484764|2485037|N|N|N|N|N| +2485130|AAAAAAAAKILOFCAA|2091-12-14|2303|10016|769|2091|5|12|14|4|2091|769|10016|Friday|2091Q4|N|Y|N|2485117|2485450|2484765|2485038|N|N|N|N|N| +2485131|AAAAAAAALILOFCAA|2091-12-15|2303|10016|769|2091|6|12|15|4|2091|769|10016|Saturday|2091Q4|N|Y|N|2485117|2485450|2484766|2485039|N|N|N|N|N| +2485132|AAAAAAAAMILOFCAA|2091-12-16|2303|10016|769|2091|0|12|16|4|2091|769|10016|Sunday|2091Q4|N|N|N|2485117|2485450|2484767|2485040|N|N|N|N|N| +2485133|AAAAAAAANILOFCAA|2091-12-17|2303|10016|769|2091|1|12|17|4|2091|769|10016|Monday|2091Q4|N|N|N|2485117|2485450|2484768|2485041|N|N|N|N|N| +2485134|AAAAAAAAOILOFCAA|2091-12-18|2303|10017|769|2091|2|12|18|4|2091|769|10017|Tuesday|2091Q4|N|N|N|2485117|2485450|2484769|2485042|N|N|N|N|N| +2485135|AAAAAAAAPILOFCAA|2091-12-19|2303|10017|769|2091|3|12|19|4|2091|769|10017|Wednesday|2091Q4|N|N|N|2485117|2485450|2484770|2485043|N|N|N|N|N| +2485136|AAAAAAAAAJLOFCAA|2091-12-20|2303|10017|769|2091|4|12|20|4|2091|769|10017|Thursday|2091Q4|N|N|N|2485117|2485450|2484771|2485044|N|N|N|N|N| +2485137|AAAAAAAABJLOFCAA|2091-12-21|2303|10017|769|2091|5|12|21|4|2091|769|10017|Friday|2091Q4|N|Y|N|2485117|2485450|2484772|2485045|N|N|N|N|N| +2485138|AAAAAAAACJLOFCAA|2091-12-22|2303|10017|769|2091|6|12|22|4|2091|769|10017|Saturday|2091Q4|N|Y|N|2485117|2485450|2484773|2485046|N|N|N|N|N| +2485139|AAAAAAAADJLOFCAA|2091-12-23|2303|10017|769|2091|0|12|23|4|2091|769|10017|Sunday|2091Q4|N|N|N|2485117|2485450|2484774|2485047|N|N|N|N|N| +2485140|AAAAAAAAEJLOFCAA|2091-12-24|2303|10017|769|2091|1|12|24|4|2091|769|10017|Monday|2091Q4|N|N|N|2485117|2485450|2484775|2485048|N|N|N|N|N| +2485141|AAAAAAAAFJLOFCAA|2091-12-25|2303|10018|769|2091|2|12|25|4|2091|769|10018|Tuesday|2091Q4|N|N|N|2485117|2485450|2484776|2485049|N|N|N|N|N| +2485142|AAAAAAAAGJLOFCAA|2091-12-26|2303|10018|769|2091|3|12|26|4|2091|769|10018|Wednesday|2091Q4|Y|N|N|2485117|2485450|2484777|2485050|N|N|N|N|N| +2485143|AAAAAAAAHJLOFCAA|2091-12-27|2303|10018|769|2091|4|12|27|4|2091|769|10018|Thursday|2091Q4|N|N|Y|2485117|2485450|2484778|2485051|N|N|N|N|N| +2485144|AAAAAAAAIJLOFCAA|2091-12-28|2303|10018|769|2091|5|12|28|4|2091|769|10018|Friday|2091Q4|N|Y|N|2485117|2485450|2484779|2485052|N|N|N|N|N| +2485145|AAAAAAAAJJLOFCAA|2091-12-29|2303|10018|769|2091|6|12|29|4|2091|769|10018|Saturday|2091Q4|N|Y|N|2485117|2485450|2484780|2485053|N|N|N|N|N| +2485146|AAAAAAAAKJLOFCAA|2091-12-30|2303|10018|769|2091|0|12|30|4|2091|769|10018|Sunday|2091Q4|N|N|N|2485117|2485450|2484781|2485054|N|N|N|N|N| +2485147|AAAAAAAALJLOFCAA|2091-12-31|2303|10018|769|2091|1|12|31|4|2091|769|10018|Monday|2091Q4|N|N|N|2485117|2485450|2484782|2485055|N|N|N|N|N| +2485148|AAAAAAAAMJLOFCAA|2092-01-01|2304|10019|769|2092|2|1|1|1|2092|769|10019|Tuesday|2092Q1|Y|N|N|2485148|2485147|2484783|2485056|N|N|N|N|N| +2485149|AAAAAAAANJLOFCAA|2092-01-02|2304|10019|769|2092|3|1|2|1|2092|769|10019|Wednesday|2092Q1|N|N|Y|2485148|2485147|2484784|2485057|N|N|N|N|N| +2485150|AAAAAAAAOJLOFCAA|2092-01-03|2304|10019|769|2092|4|1|3|1|2092|769|10019|Thursday|2092Q1|N|N|N|2485148|2485147|2484785|2485058|N|N|N|N|N| +2485151|AAAAAAAAPJLOFCAA|2092-01-04|2304|10019|769|2092|5|1|4|1|2092|769|10019|Friday|2092Q1|N|Y|N|2485148|2485147|2484786|2485059|N|N|N|N|N| +2485152|AAAAAAAAAKLOFCAA|2092-01-05|2304|10019|769|2092|6|1|5|1|2092|769|10019|Saturday|2092Q1|N|Y|N|2485148|2485147|2484787|2485060|N|N|N|N|N| +2485153|AAAAAAAABKLOFCAA|2092-01-06|2304|10019|769|2092|0|1|6|1|2092|769|10019|Sunday|2092Q1|N|N|N|2485148|2485147|2484788|2485061|N|N|N|N|N| +2485154|AAAAAAAACKLOFCAA|2092-01-07|2304|10019|769|2092|1|1|7|1|2092|769|10019|Monday|2092Q1|N|N|N|2485148|2485147|2484789|2485062|N|N|N|N|N| +2485155|AAAAAAAADKLOFCAA|2092-01-08|2304|10020|769|2092|2|1|8|1|2092|769|10020|Tuesday|2092Q1|N|N|N|2485148|2485147|2484790|2485063|N|N|N|N|N| +2485156|AAAAAAAAEKLOFCAA|2092-01-09|2304|10020|769|2092|3|1|9|1|2092|769|10020|Wednesday|2092Q1|N|N|N|2485148|2485147|2484791|2485064|N|N|N|N|N| +2485157|AAAAAAAAFKLOFCAA|2092-01-10|2304|10020|769|2092|4|1|10|1|2092|769|10020|Thursday|2092Q1|N|N|N|2485148|2485147|2484792|2485065|N|N|N|N|N| +2485158|AAAAAAAAGKLOFCAA|2092-01-11|2304|10020|769|2092|5|1|11|1|2092|769|10020|Friday|2092Q1|N|Y|N|2485148|2485147|2484793|2485066|N|N|N|N|N| +2485159|AAAAAAAAHKLOFCAA|2092-01-12|2304|10020|769|2092|6|1|12|1|2092|769|10020|Saturday|2092Q1|N|Y|N|2485148|2485147|2484794|2485067|N|N|N|N|N| +2485160|AAAAAAAAIKLOFCAA|2092-01-13|2304|10020|769|2092|0|1|13|1|2092|769|10020|Sunday|2092Q1|N|N|N|2485148|2485147|2484795|2485068|N|N|N|N|N| +2485161|AAAAAAAAJKLOFCAA|2092-01-14|2304|10020|769|2092|1|1|14|1|2092|769|10020|Monday|2092Q1|N|N|N|2485148|2485147|2484796|2485069|N|N|N|N|N| +2485162|AAAAAAAAKKLOFCAA|2092-01-15|2304|10021|769|2092|2|1|15|1|2092|769|10021|Tuesday|2092Q1|N|N|N|2485148|2485147|2484797|2485070|N|N|N|N|N| +2485163|AAAAAAAALKLOFCAA|2092-01-16|2304|10021|769|2092|3|1|16|1|2092|769|10021|Wednesday|2092Q1|N|N|N|2485148|2485147|2484798|2485071|N|N|N|N|N| +2485164|AAAAAAAAMKLOFCAA|2092-01-17|2304|10021|769|2092|4|1|17|1|2092|769|10021|Thursday|2092Q1|N|N|N|2485148|2485147|2484799|2485072|N|N|N|N|N| +2485165|AAAAAAAANKLOFCAA|2092-01-18|2304|10021|769|2092|5|1|18|1|2092|769|10021|Friday|2092Q1|N|Y|N|2485148|2485147|2484800|2485073|N|N|N|N|N| +2485166|AAAAAAAAOKLOFCAA|2092-01-19|2304|10021|769|2092|6|1|19|1|2092|769|10021|Saturday|2092Q1|N|Y|N|2485148|2485147|2484801|2485074|N|N|N|N|N| +2485167|AAAAAAAAPKLOFCAA|2092-01-20|2304|10021|769|2092|0|1|20|1|2092|769|10021|Sunday|2092Q1|N|N|N|2485148|2485147|2484802|2485075|N|N|N|N|N| +2485168|AAAAAAAAALLOFCAA|2092-01-21|2304|10021|769|2092|1|1|21|1|2092|769|10021|Monday|2092Q1|N|N|N|2485148|2485147|2484803|2485076|N|N|N|N|N| +2485169|AAAAAAAABLLOFCAA|2092-01-22|2304|10022|769|2092|2|1|22|1|2092|769|10022|Tuesday|2092Q1|N|N|N|2485148|2485147|2484804|2485077|N|N|N|N|N| +2485170|AAAAAAAACLLOFCAA|2092-01-23|2304|10022|769|2092|3|1|23|1|2092|769|10022|Wednesday|2092Q1|N|N|N|2485148|2485147|2484805|2485078|N|N|N|N|N| +2485171|AAAAAAAADLLOFCAA|2092-01-24|2304|10022|769|2092|4|1|24|1|2092|769|10022|Thursday|2092Q1|N|N|N|2485148|2485147|2484806|2485079|N|N|N|N|N| +2485172|AAAAAAAAELLOFCAA|2092-01-25|2304|10022|769|2092|5|1|25|1|2092|769|10022|Friday|2092Q1|N|Y|N|2485148|2485147|2484807|2485080|N|N|N|N|N| +2485173|AAAAAAAAFLLOFCAA|2092-01-26|2304|10022|769|2092|6|1|26|1|2092|769|10022|Saturday|2092Q1|N|Y|N|2485148|2485147|2484808|2485081|N|N|N|N|N| +2485174|AAAAAAAAGLLOFCAA|2092-01-27|2304|10022|769|2092|0|1|27|1|2092|769|10022|Sunday|2092Q1|N|N|N|2485148|2485147|2484809|2485082|N|N|N|N|N| +2485175|AAAAAAAAHLLOFCAA|2092-01-28|2304|10022|769|2092|1|1|28|1|2092|769|10022|Monday|2092Q1|N|N|N|2485148|2485147|2484810|2485083|N|N|N|N|N| +2485176|AAAAAAAAILLOFCAA|2092-01-29|2304|10023|769|2092|2|1|29|1|2092|769|10023|Tuesday|2092Q1|N|N|N|2485148|2485147|2484811|2485084|N|N|N|N|N| +2485177|AAAAAAAAJLLOFCAA|2092-01-30|2304|10023|769|2092|3|1|30|1|2092|769|10023|Wednesday|2092Q1|N|N|N|2485148|2485147|2484812|2485085|N|N|N|N|N| +2485178|AAAAAAAAKLLOFCAA|2092-01-31|2304|10023|769|2092|4|1|31|1|2092|769|10023|Thursday|2092Q1|N|N|N|2485148|2485147|2484813|2485086|N|N|N|N|N| +2485179|AAAAAAAALLLOFCAA|2092-02-01|2305|10023|769|2092|5|2|1|1|2092|769|10023|Friday|2092Q1|N|Y|N|2485179|2485209|2484814|2485087|N|N|N|N|N| +2485180|AAAAAAAAMLLOFCAA|2092-02-02|2305|10023|769|2092|6|2|2|1|2092|769|10023|Saturday|2092Q1|N|Y|N|2485179|2485209|2484815|2485088|N|N|N|N|N| +2485181|AAAAAAAANLLOFCAA|2092-02-03|2305|10023|769|2092|0|2|3|1|2092|769|10023|Sunday|2092Q1|N|N|N|2485179|2485209|2484816|2485089|N|N|N|N|N| +2485182|AAAAAAAAOLLOFCAA|2092-02-04|2305|10023|769|2092|1|2|4|1|2092|769|10023|Monday|2092Q1|N|N|N|2485179|2485209|2484817|2485090|N|N|N|N|N| +2485183|AAAAAAAAPLLOFCAA|2092-02-05|2305|10024|769|2092|2|2|5|1|2092|769|10024|Tuesday|2092Q1|N|N|N|2485179|2485209|2484818|2485091|N|N|N|N|N| +2485184|AAAAAAAAAMLOFCAA|2092-02-06|2305|10024|769|2092|3|2|6|1|2092|769|10024|Wednesday|2092Q1|N|N|N|2485179|2485209|2484819|2485092|N|N|N|N|N| +2485185|AAAAAAAABMLOFCAA|2092-02-07|2305|10024|769|2092|4|2|7|1|2092|769|10024|Thursday|2092Q1|N|N|N|2485179|2485209|2484820|2485093|N|N|N|N|N| +2485186|AAAAAAAACMLOFCAA|2092-02-08|2305|10024|769|2092|5|2|8|1|2092|769|10024|Friday|2092Q1|N|Y|N|2485179|2485209|2484821|2485094|N|N|N|N|N| +2485187|AAAAAAAADMLOFCAA|2092-02-09|2305|10024|769|2092|6|2|9|1|2092|769|10024|Saturday|2092Q1|N|Y|N|2485179|2485209|2484822|2485095|N|N|N|N|N| +2485188|AAAAAAAAEMLOFCAA|2092-02-10|2305|10024|769|2092|0|2|10|1|2092|769|10024|Sunday|2092Q1|N|N|N|2485179|2485209|2484823|2485096|N|N|N|N|N| +2485189|AAAAAAAAFMLOFCAA|2092-02-11|2305|10024|769|2092|1|2|11|1|2092|769|10024|Monday|2092Q1|N|N|N|2485179|2485209|2484824|2485097|N|N|N|N|N| +2485190|AAAAAAAAGMLOFCAA|2092-02-12|2305|10025|769|2092|2|2|12|1|2092|769|10025|Tuesday|2092Q1|N|N|N|2485179|2485209|2484825|2485098|N|N|N|N|N| +2485191|AAAAAAAAHMLOFCAA|2092-02-13|2305|10025|769|2092|3|2|13|1|2092|769|10025|Wednesday|2092Q1|N|N|N|2485179|2485209|2484826|2485099|N|N|N|N|N| +2485192|AAAAAAAAIMLOFCAA|2092-02-14|2305|10025|769|2092|4|2|14|1|2092|769|10025|Thursday|2092Q1|N|N|N|2485179|2485209|2484827|2485100|N|N|N|N|N| +2485193|AAAAAAAAJMLOFCAA|2092-02-15|2305|10025|769|2092|5|2|15|1|2092|769|10025|Friday|2092Q1|N|Y|N|2485179|2485209|2484828|2485101|N|N|N|N|N| +2485194|AAAAAAAAKMLOFCAA|2092-02-16|2305|10025|769|2092|6|2|16|1|2092|769|10025|Saturday|2092Q1|N|Y|N|2485179|2485209|2484829|2485102|N|N|N|N|N| +2485195|AAAAAAAALMLOFCAA|2092-02-17|2305|10025|769|2092|0|2|17|1|2092|769|10025|Sunday|2092Q1|N|N|N|2485179|2485209|2484830|2485103|N|N|N|N|N| +2485196|AAAAAAAAMMLOFCAA|2092-02-18|2305|10025|769|2092|1|2|18|1|2092|769|10025|Monday|2092Q1|N|N|N|2485179|2485209|2484831|2485104|N|N|N|N|N| +2485197|AAAAAAAANMLOFCAA|2092-02-19|2305|10026|769|2092|2|2|19|1|2092|769|10026|Tuesday|2092Q1|N|N|N|2485179|2485209|2484832|2485105|N|N|N|N|N| +2485198|AAAAAAAAOMLOFCAA|2092-02-20|2305|10026|769|2092|3|2|20|1|2092|769|10026|Wednesday|2092Q1|N|N|N|2485179|2485209|2484833|2485106|N|N|N|N|N| +2485199|AAAAAAAAPMLOFCAA|2092-02-21|2305|10026|769|2092|4|2|21|1|2092|769|10026|Thursday|2092Q1|N|N|N|2485179|2485209|2484834|2485107|N|N|N|N|N| +2485200|AAAAAAAAANLOFCAA|2092-02-22|2305|10026|769|2092|5|2|22|1|2092|769|10026|Friday|2092Q1|N|Y|N|2485179|2485209|2484835|2485108|N|N|N|N|N| +2485201|AAAAAAAABNLOFCAA|2092-02-23|2305|10026|769|2092|6|2|23|1|2092|769|10026|Saturday|2092Q1|N|Y|N|2485179|2485209|2484836|2485109|N|N|N|N|N| +2485202|AAAAAAAACNLOFCAA|2092-02-24|2305|10026|769|2092|0|2|24|1|2092|769|10026|Sunday|2092Q1|N|N|N|2485179|2485209|2484837|2485110|N|N|N|N|N| +2485203|AAAAAAAADNLOFCAA|2092-02-25|2305|10026|769|2092|1|2|25|1|2092|769|10026|Monday|2092Q1|N|N|N|2485179|2485209|2484838|2485111|N|N|N|N|N| +2485204|AAAAAAAAENLOFCAA|2092-02-26|2305|10027|769|2092|2|2|26|1|2092|769|10027|Tuesday|2092Q1|N|N|N|2485179|2485209|2484839|2485112|N|N|N|N|N| +2485205|AAAAAAAAFNLOFCAA|2092-02-27|2305|10027|769|2092|3|2|27|1|2092|769|10027|Wednesday|2092Q1|N|N|N|2485179|2485209|2484840|2485113|N|N|N|N|N| +2485206|AAAAAAAAGNLOFCAA|2092-02-28|2305|10027|769|2092|4|2|28|1|2092|769|10027|Thursday|2092Q1|N|N|N|2485179|2485209|2484841|2485114|N|N|N|N|N| +2485207|AAAAAAAAHNLOFCAA|2092-02-29|2305|10027|769|2092|5|2|29|1|2092|769|10027|Friday|2092Q1|N|Y|N|2485179|2485209|2484841|2485115|N|N|N|N|N| +2485208|AAAAAAAAINLOFCAA|2092-03-01|2306|10027|770|2092|6|3|1|1|2092|770|10027|Saturday|2092Q1|N|Y|N|2485208|2485267|2484842|2485116|N|N|N|N|N| +2485209|AAAAAAAAJNLOFCAA|2092-03-02|2306|10027|770|2092|0|3|2|1|2092|770|10027|Sunday|2092Q1|N|N|N|2485208|2485267|2484843|2485117|N|N|N|N|N| +2485210|AAAAAAAAKNLOFCAA|2092-03-03|2306|10027|770|2092|1|3|3|1|2092|770|10027|Monday|2092Q1|N|N|N|2485208|2485267|2484844|2485118|N|N|N|N|N| +2485211|AAAAAAAALNLOFCAA|2092-03-04|2306|10028|770|2092|2|3|4|1|2092|770|10028|Tuesday|2092Q1|N|N|N|2485208|2485267|2484845|2485119|N|N|N|N|N| +2485212|AAAAAAAAMNLOFCAA|2092-03-05|2306|10028|770|2092|3|3|5|1|2092|770|10028|Wednesday|2092Q1|N|N|N|2485208|2485267|2484846|2485120|N|N|N|N|N| +2485213|AAAAAAAANNLOFCAA|2092-03-06|2306|10028|770|2092|4|3|6|1|2092|770|10028|Thursday|2092Q1|N|N|N|2485208|2485267|2484847|2485121|N|N|N|N|N| +2485214|AAAAAAAAONLOFCAA|2092-03-07|2306|10028|770|2092|5|3|7|1|2092|770|10028|Friday|2092Q1|N|Y|N|2485208|2485267|2484848|2485122|N|N|N|N|N| +2485215|AAAAAAAAPNLOFCAA|2092-03-08|2306|10028|770|2092|6|3|8|1|2092|770|10028|Saturday|2092Q1|N|Y|N|2485208|2485267|2484849|2485123|N|N|N|N|N| +2485216|AAAAAAAAAOLOFCAA|2092-03-09|2306|10028|770|2092|0|3|9|1|2092|770|10028|Sunday|2092Q1|N|N|N|2485208|2485267|2484850|2485124|N|N|N|N|N| +2485217|AAAAAAAABOLOFCAA|2092-03-10|2306|10028|770|2092|1|3|10|1|2092|770|10028|Monday|2092Q1|N|N|N|2485208|2485267|2484851|2485125|N|N|N|N|N| +2485218|AAAAAAAACOLOFCAA|2092-03-11|2306|10029|770|2092|2|3|11|1|2092|770|10029|Tuesday|2092Q1|N|N|N|2485208|2485267|2484852|2485126|N|N|N|N|N| +2485219|AAAAAAAADOLOFCAA|2092-03-12|2306|10029|770|2092|3|3|12|1|2092|770|10029|Wednesday|2092Q1|N|N|N|2485208|2485267|2484853|2485127|N|N|N|N|N| +2485220|AAAAAAAAEOLOFCAA|2092-03-13|2306|10029|770|2092|4|3|13|1|2092|770|10029|Thursday|2092Q1|N|N|N|2485208|2485267|2484854|2485128|N|N|N|N|N| +2485221|AAAAAAAAFOLOFCAA|2092-03-14|2306|10029|770|2092|5|3|14|1|2092|770|10029|Friday|2092Q1|N|Y|N|2485208|2485267|2484855|2485129|N|N|N|N|N| +2485222|AAAAAAAAGOLOFCAA|2092-03-15|2306|10029|770|2092|6|3|15|1|2092|770|10029|Saturday|2092Q1|N|Y|N|2485208|2485267|2484856|2485130|N|N|N|N|N| +2485223|AAAAAAAAHOLOFCAA|2092-03-16|2306|10029|770|2092|0|3|16|1|2092|770|10029|Sunday|2092Q1|N|N|N|2485208|2485267|2484857|2485131|N|N|N|N|N| +2485224|AAAAAAAAIOLOFCAA|2092-03-17|2306|10029|770|2092|1|3|17|1|2092|770|10029|Monday|2092Q1|N|N|N|2485208|2485267|2484858|2485132|N|N|N|N|N| +2485225|AAAAAAAAJOLOFCAA|2092-03-18|2306|10030|770|2092|2|3|18|1|2092|770|10030|Tuesday|2092Q1|N|N|N|2485208|2485267|2484859|2485133|N|N|N|N|N| +2485226|AAAAAAAAKOLOFCAA|2092-03-19|2306|10030|770|2092|3|3|19|1|2092|770|10030|Wednesday|2092Q1|N|N|N|2485208|2485267|2484860|2485134|N|N|N|N|N| +2485227|AAAAAAAALOLOFCAA|2092-03-20|2306|10030|770|2092|4|3|20|1|2092|770|10030|Thursday|2092Q1|N|N|N|2485208|2485267|2484861|2485135|N|N|N|N|N| +2485228|AAAAAAAAMOLOFCAA|2092-03-21|2306|10030|770|2092|5|3|21|1|2092|770|10030|Friday|2092Q1|N|Y|N|2485208|2485267|2484862|2485136|N|N|N|N|N| +2485229|AAAAAAAANOLOFCAA|2092-03-22|2306|10030|770|2092|6|3|22|1|2092|770|10030|Saturday|2092Q1|N|Y|N|2485208|2485267|2484863|2485137|N|N|N|N|N| +2485230|AAAAAAAAOOLOFCAA|2092-03-23|2306|10030|770|2092|0|3|23|1|2092|770|10030|Sunday|2092Q1|N|N|N|2485208|2485267|2484864|2485138|N|N|N|N|N| +2485231|AAAAAAAAPOLOFCAA|2092-03-24|2306|10030|770|2092|1|3|24|1|2092|770|10030|Monday|2092Q1|N|N|N|2485208|2485267|2484865|2485139|N|N|N|N|N| +2485232|AAAAAAAAAPLOFCAA|2092-03-25|2306|10031|770|2092|2|3|25|1|2092|770|10031|Tuesday|2092Q1|N|N|N|2485208|2485267|2484866|2485140|N|N|N|N|N| +2485233|AAAAAAAABPLOFCAA|2092-03-26|2306|10031|770|2092|3|3|26|1|2092|770|10031|Wednesday|2092Q1|N|N|N|2485208|2485267|2484867|2485141|N|N|N|N|N| +2485234|AAAAAAAACPLOFCAA|2092-03-27|2306|10031|770|2092|4|3|27|1|2092|770|10031|Thursday|2092Q1|N|N|N|2485208|2485267|2484868|2485142|N|N|N|N|N| +2485235|AAAAAAAADPLOFCAA|2092-03-28|2306|10031|770|2092|5|3|28|1|2092|770|10031|Friday|2092Q1|N|Y|N|2485208|2485267|2484869|2485143|N|N|N|N|N| +2485236|AAAAAAAAEPLOFCAA|2092-03-29|2306|10031|770|2092|6|3|29|1|2092|770|10031|Saturday|2092Q1|N|Y|N|2485208|2485267|2484870|2485144|N|N|N|N|N| +2485237|AAAAAAAAFPLOFCAA|2092-03-30|2306|10031|770|2092|0|3|30|1|2092|770|10031|Sunday|2092Q1|N|N|N|2485208|2485267|2484871|2485145|N|N|N|N|N| +2485238|AAAAAAAAGPLOFCAA|2092-03-31|2306|10031|770|2092|1|3|31|1|2092|770|10031|Monday|2092Q1|N|N|N|2485208|2485267|2484872|2485146|N|N|N|N|N| +2485239|AAAAAAAAHPLOFCAA|2092-04-01|2307|10032|770|2092|2|4|1|2|2092|770|10032|Tuesday|2092Q2|N|N|N|2485239|2485329|2484873|2485148|N|N|N|N|N| +2485240|AAAAAAAAIPLOFCAA|2092-04-02|2307|10032|770|2092|3|4|2|2|2092|770|10032|Wednesday|2092Q2|N|N|N|2485239|2485329|2484874|2485149|N|N|N|N|N| +2485241|AAAAAAAAJPLOFCAA|2092-04-03|2307|10032|770|2092|4|4|3|2|2092|770|10032|Thursday|2092Q2|N|N|N|2485239|2485329|2484875|2485150|N|N|N|N|N| +2485242|AAAAAAAAKPLOFCAA|2092-04-04|2307|10032|770|2092|5|4|4|2|2092|770|10032|Friday|2092Q2|N|Y|N|2485239|2485329|2484876|2485151|N|N|N|N|N| +2485243|AAAAAAAALPLOFCAA|2092-04-05|2307|10032|770|2092|6|4|5|2|2092|770|10032|Saturday|2092Q2|N|Y|N|2485239|2485329|2484877|2485152|N|N|N|N|N| +2485244|AAAAAAAAMPLOFCAA|2092-04-06|2307|10032|770|2092|0|4|6|2|2092|770|10032|Sunday|2092Q2|N|N|N|2485239|2485329|2484878|2485153|N|N|N|N|N| +2485245|AAAAAAAANPLOFCAA|2092-04-07|2307|10032|770|2092|1|4|7|2|2092|770|10032|Monday|2092Q2|N|N|N|2485239|2485329|2484879|2485154|N|N|N|N|N| +2485246|AAAAAAAAOPLOFCAA|2092-04-08|2307|10033|770|2092|2|4|8|2|2092|770|10033|Tuesday|2092Q2|N|N|N|2485239|2485329|2484880|2485155|N|N|N|N|N| +2485247|AAAAAAAAPPLOFCAA|2092-04-09|2307|10033|770|2092|3|4|9|2|2092|770|10033|Wednesday|2092Q2|N|N|N|2485239|2485329|2484881|2485156|N|N|N|N|N| +2485248|AAAAAAAAAAMOFCAA|2092-04-10|2307|10033|770|2092|4|4|10|2|2092|770|10033|Thursday|2092Q2|N|N|N|2485239|2485329|2484882|2485157|N|N|N|N|N| +2485249|AAAAAAAABAMOFCAA|2092-04-11|2307|10033|770|2092|5|4|11|2|2092|770|10033|Friday|2092Q2|N|Y|N|2485239|2485329|2484883|2485158|N|N|N|N|N| +2485250|AAAAAAAACAMOFCAA|2092-04-12|2307|10033|770|2092|6|4|12|2|2092|770|10033|Saturday|2092Q2|N|Y|N|2485239|2485329|2484884|2485159|N|N|N|N|N| +2485251|AAAAAAAADAMOFCAA|2092-04-13|2307|10033|770|2092|0|4|13|2|2092|770|10033|Sunday|2092Q2|N|N|N|2485239|2485329|2484885|2485160|N|N|N|N|N| +2485252|AAAAAAAAEAMOFCAA|2092-04-14|2307|10033|770|2092|1|4|14|2|2092|770|10033|Monday|2092Q2|N|N|N|2485239|2485329|2484886|2485161|N|N|N|N|N| +2485253|AAAAAAAAFAMOFCAA|2092-04-15|2307|10034|770|2092|2|4|15|2|2092|770|10034|Tuesday|2092Q2|N|N|N|2485239|2485329|2484887|2485162|N|N|N|N|N| +2485254|AAAAAAAAGAMOFCAA|2092-04-16|2307|10034|770|2092|3|4|16|2|2092|770|10034|Wednesday|2092Q2|N|N|N|2485239|2485329|2484888|2485163|N|N|N|N|N| +2485255|AAAAAAAAHAMOFCAA|2092-04-17|2307|10034|770|2092|4|4|17|2|2092|770|10034|Thursday|2092Q2|N|N|N|2485239|2485329|2484889|2485164|N|N|N|N|N| +2485256|AAAAAAAAIAMOFCAA|2092-04-18|2307|10034|770|2092|5|4|18|2|2092|770|10034|Friday|2092Q2|N|Y|N|2485239|2485329|2484890|2485165|N|N|N|N|N| +2485257|AAAAAAAAJAMOFCAA|2092-04-19|2307|10034|770|2092|6|4|19|2|2092|770|10034|Saturday|2092Q2|N|Y|N|2485239|2485329|2484891|2485166|N|N|N|N|N| +2485258|AAAAAAAAKAMOFCAA|2092-04-20|2307|10034|770|2092|0|4|20|2|2092|770|10034|Sunday|2092Q2|N|N|N|2485239|2485329|2484892|2485167|N|N|N|N|N| +2485259|AAAAAAAALAMOFCAA|2092-04-21|2307|10034|770|2092|1|4|21|2|2092|770|10034|Monday|2092Q2|N|N|N|2485239|2485329|2484893|2485168|N|N|N|N|N| +2485260|AAAAAAAAMAMOFCAA|2092-04-22|2307|10035|770|2092|2|4|22|2|2092|770|10035|Tuesday|2092Q2|N|N|N|2485239|2485329|2484894|2485169|N|N|N|N|N| +2485261|AAAAAAAANAMOFCAA|2092-04-23|2307|10035|770|2092|3|4|23|2|2092|770|10035|Wednesday|2092Q2|N|N|N|2485239|2485329|2484895|2485170|N|N|N|N|N| +2485262|AAAAAAAAOAMOFCAA|2092-04-24|2307|10035|770|2092|4|4|24|2|2092|770|10035|Thursday|2092Q2|N|N|N|2485239|2485329|2484896|2485171|N|N|N|N|N| +2485263|AAAAAAAAPAMOFCAA|2092-04-25|2307|10035|770|2092|5|4|25|2|2092|770|10035|Friday|2092Q2|N|Y|N|2485239|2485329|2484897|2485172|N|N|N|N|N| +2485264|AAAAAAAAABMOFCAA|2092-04-26|2307|10035|770|2092|6|4|26|2|2092|770|10035|Saturday|2092Q2|N|Y|N|2485239|2485329|2484898|2485173|N|N|N|N|N| +2485265|AAAAAAAABBMOFCAA|2092-04-27|2307|10035|770|2092|0|4|27|2|2092|770|10035|Sunday|2092Q2|N|N|N|2485239|2485329|2484899|2485174|N|N|N|N|N| +2485266|AAAAAAAACBMOFCAA|2092-04-28|2307|10035|770|2092|1|4|28|2|2092|770|10035|Monday|2092Q2|N|N|N|2485239|2485329|2484900|2485175|N|N|N|N|N| +2485267|AAAAAAAADBMOFCAA|2092-04-29|2307|10036|770|2092|2|4|29|2|2092|770|10036|Tuesday|2092Q2|N|N|N|2485239|2485329|2484901|2485176|N|N|N|N|N| +2485268|AAAAAAAAEBMOFCAA|2092-04-30|2307|10036|770|2092|3|4|30|2|2092|770|10036|Wednesday|2092Q2|N|N|N|2485239|2485329|2484902|2485177|N|N|N|N|N| +2485269|AAAAAAAAFBMOFCAA|2092-05-01|2308|10036|770|2092|4|5|1|2|2092|770|10036|Thursday|2092Q2|N|N|N|2485269|2485389|2484903|2485178|N|N|N|N|N| +2485270|AAAAAAAAGBMOFCAA|2092-05-02|2308|10036|770|2092|5|5|2|2|2092|770|10036|Friday|2092Q2|N|Y|N|2485269|2485389|2484904|2485179|N|N|N|N|N| +2485271|AAAAAAAAHBMOFCAA|2092-05-03|2308|10036|770|2092|6|5|3|2|2092|770|10036|Saturday|2092Q2|N|Y|N|2485269|2485389|2484905|2485180|N|N|N|N|N| +2485272|AAAAAAAAIBMOFCAA|2092-05-04|2308|10036|770|2092|0|5|4|2|2092|770|10036|Sunday|2092Q2|N|N|N|2485269|2485389|2484906|2485181|N|N|N|N|N| +2485273|AAAAAAAAJBMOFCAA|2092-05-05|2308|10036|770|2092|1|5|5|2|2092|770|10036|Monday|2092Q2|N|N|N|2485269|2485389|2484907|2485182|N|N|N|N|N| +2485274|AAAAAAAAKBMOFCAA|2092-05-06|2308|10037|770|2092|2|5|6|2|2092|770|10037|Tuesday|2092Q2|N|N|N|2485269|2485389|2484908|2485183|N|N|N|N|N| +2485275|AAAAAAAALBMOFCAA|2092-05-07|2308|10037|770|2092|3|5|7|2|2092|770|10037|Wednesday|2092Q2|N|N|N|2485269|2485389|2484909|2485184|N|N|N|N|N| +2485276|AAAAAAAAMBMOFCAA|2092-05-08|2308|10037|770|2092|4|5|8|2|2092|770|10037|Thursday|2092Q2|N|N|N|2485269|2485389|2484910|2485185|N|N|N|N|N| +2485277|AAAAAAAANBMOFCAA|2092-05-09|2308|10037|770|2092|5|5|9|2|2092|770|10037|Friday|2092Q2|N|Y|N|2485269|2485389|2484911|2485186|N|N|N|N|N| +2485278|AAAAAAAAOBMOFCAA|2092-05-10|2308|10037|770|2092|6|5|10|2|2092|770|10037|Saturday|2092Q2|N|Y|N|2485269|2485389|2484912|2485187|N|N|N|N|N| +2485279|AAAAAAAAPBMOFCAA|2092-05-11|2308|10037|770|2092|0|5|11|2|2092|770|10037|Sunday|2092Q2|N|N|N|2485269|2485389|2484913|2485188|N|N|N|N|N| +2485280|AAAAAAAAACMOFCAA|2092-05-12|2308|10037|770|2092|1|5|12|2|2092|770|10037|Monday|2092Q2|N|N|N|2485269|2485389|2484914|2485189|N|N|N|N|N| +2485281|AAAAAAAABCMOFCAA|2092-05-13|2308|10038|770|2092|2|5|13|2|2092|770|10038|Tuesday|2092Q2|N|N|N|2485269|2485389|2484915|2485190|N|N|N|N|N| +2485282|AAAAAAAACCMOFCAA|2092-05-14|2308|10038|770|2092|3|5|14|2|2092|770|10038|Wednesday|2092Q2|N|N|N|2485269|2485389|2484916|2485191|N|N|N|N|N| +2485283|AAAAAAAADCMOFCAA|2092-05-15|2308|10038|770|2092|4|5|15|2|2092|770|10038|Thursday|2092Q2|N|N|N|2485269|2485389|2484917|2485192|N|N|N|N|N| +2485284|AAAAAAAAECMOFCAA|2092-05-16|2308|10038|770|2092|5|5|16|2|2092|770|10038|Friday|2092Q2|N|Y|N|2485269|2485389|2484918|2485193|N|N|N|N|N| +2485285|AAAAAAAAFCMOFCAA|2092-05-17|2308|10038|770|2092|6|5|17|2|2092|770|10038|Saturday|2092Q2|N|Y|N|2485269|2485389|2484919|2485194|N|N|N|N|N| +2485286|AAAAAAAAGCMOFCAA|2092-05-18|2308|10038|770|2092|0|5|18|2|2092|770|10038|Sunday|2092Q2|N|N|N|2485269|2485389|2484920|2485195|N|N|N|N|N| +2485287|AAAAAAAAHCMOFCAA|2092-05-19|2308|10038|770|2092|1|5|19|2|2092|770|10038|Monday|2092Q2|N|N|N|2485269|2485389|2484921|2485196|N|N|N|N|N| +2485288|AAAAAAAAICMOFCAA|2092-05-20|2308|10039|770|2092|2|5|20|2|2092|770|10039|Tuesday|2092Q2|N|N|N|2485269|2485389|2484922|2485197|N|N|N|N|N| +2485289|AAAAAAAAJCMOFCAA|2092-05-21|2308|10039|770|2092|3|5|21|2|2092|770|10039|Wednesday|2092Q2|N|N|N|2485269|2485389|2484923|2485198|N|N|N|N|N| +2485290|AAAAAAAAKCMOFCAA|2092-05-22|2308|10039|770|2092|4|5|22|2|2092|770|10039|Thursday|2092Q2|N|N|N|2485269|2485389|2484924|2485199|N|N|N|N|N| +2485291|AAAAAAAALCMOFCAA|2092-05-23|2308|10039|770|2092|5|5|23|2|2092|770|10039|Friday|2092Q2|N|Y|N|2485269|2485389|2484925|2485200|N|N|N|N|N| +2485292|AAAAAAAAMCMOFCAA|2092-05-24|2308|10039|770|2092|6|5|24|2|2092|770|10039|Saturday|2092Q2|N|Y|N|2485269|2485389|2484926|2485201|N|N|N|N|N| +2485293|AAAAAAAANCMOFCAA|2092-05-25|2308|10039|770|2092|0|5|25|2|2092|770|10039|Sunday|2092Q2|N|N|N|2485269|2485389|2484927|2485202|N|N|N|N|N| +2485294|AAAAAAAAOCMOFCAA|2092-05-26|2308|10039|770|2092|1|5|26|2|2092|770|10039|Monday|2092Q2|N|N|N|2485269|2485389|2484928|2485203|N|N|N|N|N| +2485295|AAAAAAAAPCMOFCAA|2092-05-27|2308|10040|770|2092|2|5|27|2|2092|770|10040|Tuesday|2092Q2|N|N|N|2485269|2485389|2484929|2485204|N|N|N|N|N| +2485296|AAAAAAAAADMOFCAA|2092-05-28|2308|10040|770|2092|3|5|28|2|2092|770|10040|Wednesday|2092Q2|N|N|N|2485269|2485389|2484930|2485205|N|N|N|N|N| +2485297|AAAAAAAABDMOFCAA|2092-05-29|2308|10040|770|2092|4|5|29|2|2092|770|10040|Thursday|2092Q2|N|N|N|2485269|2485389|2484931|2485206|N|N|N|N|N| +2485298|AAAAAAAACDMOFCAA|2092-05-30|2308|10040|770|2092|5|5|30|2|2092|770|10040|Friday|2092Q2|N|Y|N|2485269|2485389|2484932|2485207|N|N|N|N|N| +2485299|AAAAAAAADDMOFCAA|2092-05-31|2308|10040|770|2092|6|5|31|2|2092|770|10040|Saturday|2092Q2|N|Y|N|2485269|2485389|2484933|2485208|N|N|N|N|N| +2485300|AAAAAAAAEDMOFCAA|2092-06-01|2309|10040|771|2092|0|6|1|2|2092|771|10040|Sunday|2092Q2|N|N|N|2485300|2485451|2484934|2485209|N|N|N|N|N| +2485301|AAAAAAAAFDMOFCAA|2092-06-02|2309|10040|771|2092|1|6|2|2|2092|771|10040|Monday|2092Q2|N|N|N|2485300|2485451|2484935|2485210|N|N|N|N|N| +2485302|AAAAAAAAGDMOFCAA|2092-06-03|2309|10041|771|2092|2|6|3|2|2092|771|10041|Tuesday|2092Q2|N|N|N|2485300|2485451|2484936|2485211|N|N|N|N|N| +2485303|AAAAAAAAHDMOFCAA|2092-06-04|2309|10041|771|2092|3|6|4|2|2092|771|10041|Wednesday|2092Q2|N|N|N|2485300|2485451|2484937|2485212|N|N|N|N|N| +2485304|AAAAAAAAIDMOFCAA|2092-06-05|2309|10041|771|2092|4|6|5|2|2092|771|10041|Thursday|2092Q2|N|N|N|2485300|2485451|2484938|2485213|N|N|N|N|N| +2485305|AAAAAAAAJDMOFCAA|2092-06-06|2309|10041|771|2092|5|6|6|2|2092|771|10041|Friday|2092Q2|N|Y|N|2485300|2485451|2484939|2485214|N|N|N|N|N| +2485306|AAAAAAAAKDMOFCAA|2092-06-07|2309|10041|771|2092|6|6|7|2|2092|771|10041|Saturday|2092Q2|N|Y|N|2485300|2485451|2484940|2485215|N|N|N|N|N| +2485307|AAAAAAAALDMOFCAA|2092-06-08|2309|10041|771|2092|0|6|8|2|2092|771|10041|Sunday|2092Q2|N|N|N|2485300|2485451|2484941|2485216|N|N|N|N|N| +2485308|AAAAAAAAMDMOFCAA|2092-06-09|2309|10041|771|2092|1|6|9|2|2092|771|10041|Monday|2092Q2|N|N|N|2485300|2485451|2484942|2485217|N|N|N|N|N| +2485309|AAAAAAAANDMOFCAA|2092-06-10|2309|10042|771|2092|2|6|10|2|2092|771|10042|Tuesday|2092Q2|N|N|N|2485300|2485451|2484943|2485218|N|N|N|N|N| +2485310|AAAAAAAAODMOFCAA|2092-06-11|2309|10042|771|2092|3|6|11|2|2092|771|10042|Wednesday|2092Q2|N|N|N|2485300|2485451|2484944|2485219|N|N|N|N|N| +2485311|AAAAAAAAPDMOFCAA|2092-06-12|2309|10042|771|2092|4|6|12|2|2092|771|10042|Thursday|2092Q2|N|N|N|2485300|2485451|2484945|2485220|N|N|N|N|N| +2485312|AAAAAAAAAEMOFCAA|2092-06-13|2309|10042|771|2092|5|6|13|2|2092|771|10042|Friday|2092Q2|N|Y|N|2485300|2485451|2484946|2485221|N|N|N|N|N| +2485313|AAAAAAAABEMOFCAA|2092-06-14|2309|10042|771|2092|6|6|14|2|2092|771|10042|Saturday|2092Q2|N|Y|N|2485300|2485451|2484947|2485222|N|N|N|N|N| +2485314|AAAAAAAACEMOFCAA|2092-06-15|2309|10042|771|2092|0|6|15|2|2092|771|10042|Sunday|2092Q2|N|N|N|2485300|2485451|2484948|2485223|N|N|N|N|N| +2485315|AAAAAAAADEMOFCAA|2092-06-16|2309|10042|771|2092|1|6|16|2|2092|771|10042|Monday|2092Q2|N|N|N|2485300|2485451|2484949|2485224|N|N|N|N|N| +2485316|AAAAAAAAEEMOFCAA|2092-06-17|2309|10043|771|2092|2|6|17|2|2092|771|10043|Tuesday|2092Q2|N|N|N|2485300|2485451|2484950|2485225|N|N|N|N|N| +2485317|AAAAAAAAFEMOFCAA|2092-06-18|2309|10043|771|2092|3|6|18|2|2092|771|10043|Wednesday|2092Q2|N|N|N|2485300|2485451|2484951|2485226|N|N|N|N|N| +2485318|AAAAAAAAGEMOFCAA|2092-06-19|2309|10043|771|2092|4|6|19|2|2092|771|10043|Thursday|2092Q2|N|N|N|2485300|2485451|2484952|2485227|N|N|N|N|N| +2485319|AAAAAAAAHEMOFCAA|2092-06-20|2309|10043|771|2092|5|6|20|2|2092|771|10043|Friday|2092Q2|N|Y|N|2485300|2485451|2484953|2485228|N|N|N|N|N| +2485320|AAAAAAAAIEMOFCAA|2092-06-21|2309|10043|771|2092|6|6|21|2|2092|771|10043|Saturday|2092Q2|N|Y|N|2485300|2485451|2484954|2485229|N|N|N|N|N| +2485321|AAAAAAAAJEMOFCAA|2092-06-22|2309|10043|771|2092|0|6|22|2|2092|771|10043|Sunday|2092Q2|N|N|N|2485300|2485451|2484955|2485230|N|N|N|N|N| +2485322|AAAAAAAAKEMOFCAA|2092-06-23|2309|10043|771|2092|1|6|23|2|2092|771|10043|Monday|2092Q2|N|N|N|2485300|2485451|2484956|2485231|N|N|N|N|N| +2485323|AAAAAAAALEMOFCAA|2092-06-24|2309|10044|771|2092|2|6|24|2|2092|771|10044|Tuesday|2092Q2|N|N|N|2485300|2485451|2484957|2485232|N|N|N|N|N| +2485324|AAAAAAAAMEMOFCAA|2092-06-25|2309|10044|771|2092|3|6|25|2|2092|771|10044|Wednesday|2092Q2|N|N|N|2485300|2485451|2484958|2485233|N|N|N|N|N| +2485325|AAAAAAAANEMOFCAA|2092-06-26|2309|10044|771|2092|4|6|26|2|2092|771|10044|Thursday|2092Q2|N|N|N|2485300|2485451|2484959|2485234|N|N|N|N|N| +2485326|AAAAAAAAOEMOFCAA|2092-06-27|2309|10044|771|2092|5|6|27|2|2092|771|10044|Friday|2092Q2|N|Y|N|2485300|2485451|2484960|2485235|N|N|N|N|N| +2485327|AAAAAAAAPEMOFCAA|2092-06-28|2309|10044|771|2092|6|6|28|2|2092|771|10044|Saturday|2092Q2|N|Y|N|2485300|2485451|2484961|2485236|N|N|N|N|N| +2485328|AAAAAAAAAFMOFCAA|2092-06-29|2309|10044|771|2092|0|6|29|2|2092|771|10044|Sunday|2092Q2|N|N|N|2485300|2485451|2484962|2485237|N|N|N|N|N| +2485329|AAAAAAAABFMOFCAA|2092-06-30|2309|10044|771|2092|1|6|30|2|2092|771|10044|Monday|2092Q2|N|N|N|2485300|2485451|2484963|2485238|N|N|N|N|N| +2485330|AAAAAAAACFMOFCAA|2092-07-01|2310|10045|771|2092|2|7|1|3|2092|771|10045|Tuesday|2092Q3|N|N|N|2485330|2485511|2484964|2485239|N|N|N|N|N| +2485331|AAAAAAAADFMOFCAA|2092-07-02|2310|10045|771|2092|3|7|2|3|2092|771|10045|Wednesday|2092Q3|N|N|N|2485330|2485511|2484965|2485240|N|N|N|N|N| +2485332|AAAAAAAAEFMOFCAA|2092-07-03|2310|10045|771|2092|4|7|3|3|2092|771|10045|Thursday|2092Q3|N|N|N|2485330|2485511|2484966|2485241|N|N|N|N|N| +2485333|AAAAAAAAFFMOFCAA|2092-07-04|2310|10045|771|2092|5|7|4|3|2092|771|10045|Friday|2092Q3|Y|Y|N|2485330|2485511|2484967|2485242|N|N|N|N|N| +2485334|AAAAAAAAGFMOFCAA|2092-07-05|2310|10045|771|2092|6|7|5|3|2092|771|10045|Saturday|2092Q3|N|Y|Y|2485330|2485511|2484968|2485243|N|N|N|N|N| +2485335|AAAAAAAAHFMOFCAA|2092-07-06|2310|10045|771|2092|0|7|6|3|2092|771|10045|Sunday|2092Q3|N|N|N|2485330|2485511|2484969|2485244|N|N|N|N|N| +2485336|AAAAAAAAIFMOFCAA|2092-07-07|2310|10045|771|2092|1|7|7|3|2092|771|10045|Monday|2092Q3|N|N|N|2485330|2485511|2484970|2485245|N|N|N|N|N| +2485337|AAAAAAAAJFMOFCAA|2092-07-08|2310|10046|771|2092|2|7|8|3|2092|771|10046|Tuesday|2092Q3|N|N|N|2485330|2485511|2484971|2485246|N|N|N|N|N| +2485338|AAAAAAAAKFMOFCAA|2092-07-09|2310|10046|771|2092|3|7|9|3|2092|771|10046|Wednesday|2092Q3|N|N|N|2485330|2485511|2484972|2485247|N|N|N|N|N| +2485339|AAAAAAAALFMOFCAA|2092-07-10|2310|10046|771|2092|4|7|10|3|2092|771|10046|Thursday|2092Q3|N|N|N|2485330|2485511|2484973|2485248|N|N|N|N|N| +2485340|AAAAAAAAMFMOFCAA|2092-07-11|2310|10046|771|2092|5|7|11|3|2092|771|10046|Friday|2092Q3|N|Y|N|2485330|2485511|2484974|2485249|N|N|N|N|N| +2485341|AAAAAAAANFMOFCAA|2092-07-12|2310|10046|771|2092|6|7|12|3|2092|771|10046|Saturday|2092Q3|N|Y|N|2485330|2485511|2484975|2485250|N|N|N|N|N| +2485342|AAAAAAAAOFMOFCAA|2092-07-13|2310|10046|771|2092|0|7|13|3|2092|771|10046|Sunday|2092Q3|N|N|N|2485330|2485511|2484976|2485251|N|N|N|N|N| +2485343|AAAAAAAAPFMOFCAA|2092-07-14|2310|10046|771|2092|1|7|14|3|2092|771|10046|Monday|2092Q3|N|N|N|2485330|2485511|2484977|2485252|N|N|N|N|N| +2485344|AAAAAAAAAGMOFCAA|2092-07-15|2310|10047|771|2092|2|7|15|3|2092|771|10047|Tuesday|2092Q3|N|N|N|2485330|2485511|2484978|2485253|N|N|N|N|N| +2485345|AAAAAAAABGMOFCAA|2092-07-16|2310|10047|771|2092|3|7|16|3|2092|771|10047|Wednesday|2092Q3|N|N|N|2485330|2485511|2484979|2485254|N|N|N|N|N| +2485346|AAAAAAAACGMOFCAA|2092-07-17|2310|10047|771|2092|4|7|17|3|2092|771|10047|Thursday|2092Q3|N|N|N|2485330|2485511|2484980|2485255|N|N|N|N|N| +2485347|AAAAAAAADGMOFCAA|2092-07-18|2310|10047|771|2092|5|7|18|3|2092|771|10047|Friday|2092Q3|N|Y|N|2485330|2485511|2484981|2485256|N|N|N|N|N| +2485348|AAAAAAAAEGMOFCAA|2092-07-19|2310|10047|771|2092|6|7|19|3|2092|771|10047|Saturday|2092Q3|N|Y|N|2485330|2485511|2484982|2485257|N|N|N|N|N| +2485349|AAAAAAAAFGMOFCAA|2092-07-20|2310|10047|771|2092|0|7|20|3|2092|771|10047|Sunday|2092Q3|N|N|N|2485330|2485511|2484983|2485258|N|N|N|N|N| +2485350|AAAAAAAAGGMOFCAA|2092-07-21|2310|10047|771|2092|1|7|21|3|2092|771|10047|Monday|2092Q3|N|N|N|2485330|2485511|2484984|2485259|N|N|N|N|N| +2485351|AAAAAAAAHGMOFCAA|2092-07-22|2310|10048|771|2092|2|7|22|3|2092|771|10048|Tuesday|2092Q3|N|N|N|2485330|2485511|2484985|2485260|N|N|N|N|N| +2485352|AAAAAAAAIGMOFCAA|2092-07-23|2310|10048|771|2092|3|7|23|3|2092|771|10048|Wednesday|2092Q3|N|N|N|2485330|2485511|2484986|2485261|N|N|N|N|N| +2485353|AAAAAAAAJGMOFCAA|2092-07-24|2310|10048|771|2092|4|7|24|3|2092|771|10048|Thursday|2092Q3|N|N|N|2485330|2485511|2484987|2485262|N|N|N|N|N| +2485354|AAAAAAAAKGMOFCAA|2092-07-25|2310|10048|771|2092|5|7|25|3|2092|771|10048|Friday|2092Q3|N|Y|N|2485330|2485511|2484988|2485263|N|N|N|N|N| +2485355|AAAAAAAALGMOFCAA|2092-07-26|2310|10048|771|2092|6|7|26|3|2092|771|10048|Saturday|2092Q3|N|Y|N|2485330|2485511|2484989|2485264|N|N|N|N|N| +2485356|AAAAAAAAMGMOFCAA|2092-07-27|2310|10048|771|2092|0|7|27|3|2092|771|10048|Sunday|2092Q3|N|N|N|2485330|2485511|2484990|2485265|N|N|N|N|N| +2485357|AAAAAAAANGMOFCAA|2092-07-28|2310|10048|771|2092|1|7|28|3|2092|771|10048|Monday|2092Q3|N|N|N|2485330|2485511|2484991|2485266|N|N|N|N|N| +2485358|AAAAAAAAOGMOFCAA|2092-07-29|2310|10049|771|2092|2|7|29|3|2092|771|10049|Tuesday|2092Q3|N|N|N|2485330|2485511|2484992|2485267|N|N|N|N|N| +2485359|AAAAAAAAPGMOFCAA|2092-07-30|2310|10049|771|2092|3|7|30|3|2092|771|10049|Wednesday|2092Q3|N|N|N|2485330|2485511|2484993|2485268|N|N|N|N|N| +2485360|AAAAAAAAAHMOFCAA|2092-07-31|2310|10049|771|2092|4|7|31|3|2092|771|10049|Thursday|2092Q3|N|N|N|2485330|2485511|2484994|2485269|N|N|N|N|N| +2485361|AAAAAAAABHMOFCAA|2092-08-01|2311|10049|771|2092|5|8|1|3|2092|771|10049|Friday|2092Q3|N|Y|N|2485361|2485573|2484995|2485270|N|N|N|N|N| +2485362|AAAAAAAACHMOFCAA|2092-08-02|2311|10049|771|2092|6|8|2|3|2092|771|10049|Saturday|2092Q3|N|Y|N|2485361|2485573|2484996|2485271|N|N|N|N|N| +2485363|AAAAAAAADHMOFCAA|2092-08-03|2311|10049|771|2092|0|8|3|3|2092|771|10049|Sunday|2092Q3|N|N|N|2485361|2485573|2484997|2485272|N|N|N|N|N| +2485364|AAAAAAAAEHMOFCAA|2092-08-04|2311|10049|771|2092|1|8|4|3|2092|771|10049|Monday|2092Q3|N|N|N|2485361|2485573|2484998|2485273|N|N|N|N|N| +2485365|AAAAAAAAFHMOFCAA|2092-08-05|2311|10050|771|2092|2|8|5|3|2092|771|10050|Tuesday|2092Q3|N|N|N|2485361|2485573|2484999|2485274|N|N|N|N|N| +2485366|AAAAAAAAGHMOFCAA|2092-08-06|2311|10050|771|2092|3|8|6|3|2092|771|10050|Wednesday|2092Q3|N|N|N|2485361|2485573|2485000|2485275|N|N|N|N|N| +2485367|AAAAAAAAHHMOFCAA|2092-08-07|2311|10050|771|2092|4|8|7|3|2092|771|10050|Thursday|2092Q3|N|N|N|2485361|2485573|2485001|2485276|N|N|N|N|N| +2485368|AAAAAAAAIHMOFCAA|2092-08-08|2311|10050|771|2092|5|8|8|3|2092|771|10050|Friday|2092Q3|N|Y|N|2485361|2485573|2485002|2485277|N|N|N|N|N| +2485369|AAAAAAAAJHMOFCAA|2092-08-09|2311|10050|771|2092|6|8|9|3|2092|771|10050|Saturday|2092Q3|N|Y|N|2485361|2485573|2485003|2485278|N|N|N|N|N| +2485370|AAAAAAAAKHMOFCAA|2092-08-10|2311|10050|771|2092|0|8|10|3|2092|771|10050|Sunday|2092Q3|N|N|N|2485361|2485573|2485004|2485279|N|N|N|N|N| +2485371|AAAAAAAALHMOFCAA|2092-08-11|2311|10050|771|2092|1|8|11|3|2092|771|10050|Monday|2092Q3|N|N|N|2485361|2485573|2485005|2485280|N|N|N|N|N| +2485372|AAAAAAAAMHMOFCAA|2092-08-12|2311|10051|771|2092|2|8|12|3|2092|771|10051|Tuesday|2092Q3|N|N|N|2485361|2485573|2485006|2485281|N|N|N|N|N| +2485373|AAAAAAAANHMOFCAA|2092-08-13|2311|10051|771|2092|3|8|13|3|2092|771|10051|Wednesday|2092Q3|N|N|N|2485361|2485573|2485007|2485282|N|N|N|N|N| +2485374|AAAAAAAAOHMOFCAA|2092-08-14|2311|10051|771|2092|4|8|14|3|2092|771|10051|Thursday|2092Q3|N|N|N|2485361|2485573|2485008|2485283|N|N|N|N|N| +2485375|AAAAAAAAPHMOFCAA|2092-08-15|2311|10051|771|2092|5|8|15|3|2092|771|10051|Friday|2092Q3|N|Y|N|2485361|2485573|2485009|2485284|N|N|N|N|N| +2485376|AAAAAAAAAIMOFCAA|2092-08-16|2311|10051|771|2092|6|8|16|3|2092|771|10051|Saturday|2092Q3|N|Y|N|2485361|2485573|2485010|2485285|N|N|N|N|N| +2485377|AAAAAAAABIMOFCAA|2092-08-17|2311|10051|771|2092|0|8|17|3|2092|771|10051|Sunday|2092Q3|N|N|N|2485361|2485573|2485011|2485286|N|N|N|N|N| +2485378|AAAAAAAACIMOFCAA|2092-08-18|2311|10051|771|2092|1|8|18|3|2092|771|10051|Monday|2092Q3|N|N|N|2485361|2485573|2485012|2485287|N|N|N|N|N| +2485379|AAAAAAAADIMOFCAA|2092-08-19|2311|10052|771|2092|2|8|19|3|2092|771|10052|Tuesday|2092Q3|N|N|N|2485361|2485573|2485013|2485288|N|N|N|N|N| +2485380|AAAAAAAAEIMOFCAA|2092-08-20|2311|10052|771|2092|3|8|20|3|2092|771|10052|Wednesday|2092Q3|N|N|N|2485361|2485573|2485014|2485289|N|N|N|N|N| +2485381|AAAAAAAAFIMOFCAA|2092-08-21|2311|10052|771|2092|4|8|21|3|2092|771|10052|Thursday|2092Q3|N|N|N|2485361|2485573|2485015|2485290|N|N|N|N|N| +2485382|AAAAAAAAGIMOFCAA|2092-08-22|2311|10052|771|2092|5|8|22|3|2092|771|10052|Friday|2092Q3|N|Y|N|2485361|2485573|2485016|2485291|N|N|N|N|N| +2485383|AAAAAAAAHIMOFCAA|2092-08-23|2311|10052|771|2092|6|8|23|3|2092|771|10052|Saturday|2092Q3|N|Y|N|2485361|2485573|2485017|2485292|N|N|N|N|N| +2485384|AAAAAAAAIIMOFCAA|2092-08-24|2311|10052|771|2092|0|8|24|3|2092|771|10052|Sunday|2092Q3|N|N|N|2485361|2485573|2485018|2485293|N|N|N|N|N| +2485385|AAAAAAAAJIMOFCAA|2092-08-25|2311|10052|771|2092|1|8|25|3|2092|771|10052|Monday|2092Q3|N|N|N|2485361|2485573|2485019|2485294|N|N|N|N|N| +2485386|AAAAAAAAKIMOFCAA|2092-08-26|2311|10053|771|2092|2|8|26|3|2092|771|10053|Tuesday|2092Q3|N|N|N|2485361|2485573|2485020|2485295|N|N|N|N|N| +2485387|AAAAAAAALIMOFCAA|2092-08-27|2311|10053|771|2092|3|8|27|3|2092|771|10053|Wednesday|2092Q3|N|N|N|2485361|2485573|2485021|2485296|N|N|N|N|N| +2485388|AAAAAAAAMIMOFCAA|2092-08-28|2311|10053|771|2092|4|8|28|3|2092|771|10053|Thursday|2092Q3|N|N|N|2485361|2485573|2485022|2485297|N|N|N|N|N| +2485389|AAAAAAAANIMOFCAA|2092-08-29|2311|10053|771|2092|5|8|29|3|2092|771|10053|Friday|2092Q3|N|Y|N|2485361|2485573|2485023|2485298|N|N|N|N|N| +2485390|AAAAAAAAOIMOFCAA|2092-08-30|2311|10053|771|2092|6|8|30|3|2092|771|10053|Saturday|2092Q3|N|Y|N|2485361|2485573|2485024|2485299|N|N|N|N|N| +2485391|AAAAAAAAPIMOFCAA|2092-08-31|2311|10053|771|2092|0|8|31|3|2092|771|10053|Sunday|2092Q3|N|N|N|2485361|2485573|2485025|2485300|N|N|N|N|N| +2485392|AAAAAAAAAJMOFCAA|2092-09-01|2312|10053|772|2092|1|9|1|3|2092|772|10053|Monday|2092Q3|N|N|N|2485392|2485635|2485026|2485301|N|N|N|N|N| +2485393|AAAAAAAABJMOFCAA|2092-09-02|2312|10054|772|2092|2|9|2|3|2092|772|10054|Tuesday|2092Q3|N|N|N|2485392|2485635|2485027|2485302|N|N|N|N|N| +2485394|AAAAAAAACJMOFCAA|2092-09-03|2312|10054|772|2092|3|9|3|3|2092|772|10054|Wednesday|2092Q3|N|N|N|2485392|2485635|2485028|2485303|N|N|N|N|N| +2485395|AAAAAAAADJMOFCAA|2092-09-04|2312|10054|772|2092|4|9|4|3|2092|772|10054|Thursday|2092Q3|N|N|N|2485392|2485635|2485029|2485304|N|N|N|N|N| +2485396|AAAAAAAAEJMOFCAA|2092-09-05|2312|10054|772|2092|5|9|5|3|2092|772|10054|Friday|2092Q3|N|Y|N|2485392|2485635|2485030|2485305|N|N|N|N|N| +2485397|AAAAAAAAFJMOFCAA|2092-09-06|2312|10054|772|2092|6|9|6|3|2092|772|10054|Saturday|2092Q3|N|Y|N|2485392|2485635|2485031|2485306|N|N|N|N|N| +2485398|AAAAAAAAGJMOFCAA|2092-09-07|2312|10054|772|2092|0|9|7|3|2092|772|10054|Sunday|2092Q3|N|N|N|2485392|2485635|2485032|2485307|N|N|N|N|N| +2485399|AAAAAAAAHJMOFCAA|2092-09-08|2312|10054|772|2092|1|9|8|3|2092|772|10054|Monday|2092Q3|N|N|N|2485392|2485635|2485033|2485308|N|N|N|N|N| +2485400|AAAAAAAAIJMOFCAA|2092-09-09|2312|10055|772|2092|2|9|9|3|2092|772|10055|Tuesday|2092Q3|N|N|N|2485392|2485635|2485034|2485309|N|N|N|N|N| +2485401|AAAAAAAAJJMOFCAA|2092-09-10|2312|10055|772|2092|3|9|10|3|2092|772|10055|Wednesday|2092Q3|N|N|N|2485392|2485635|2485035|2485310|N|N|N|N|N| +2485402|AAAAAAAAKJMOFCAA|2092-09-11|2312|10055|772|2092|4|9|11|3|2092|772|10055|Thursday|2092Q3|N|N|N|2485392|2485635|2485036|2485311|N|N|N|N|N| +2485403|AAAAAAAALJMOFCAA|2092-09-12|2312|10055|772|2092|5|9|12|3|2092|772|10055|Friday|2092Q3|N|Y|N|2485392|2485635|2485037|2485312|N|N|N|N|N| +2485404|AAAAAAAAMJMOFCAA|2092-09-13|2312|10055|772|2092|6|9|13|3|2092|772|10055|Saturday|2092Q3|N|Y|N|2485392|2485635|2485038|2485313|N|N|N|N|N| +2485405|AAAAAAAANJMOFCAA|2092-09-14|2312|10055|772|2092|0|9|14|3|2092|772|10055|Sunday|2092Q3|N|N|N|2485392|2485635|2485039|2485314|N|N|N|N|N| +2485406|AAAAAAAAOJMOFCAA|2092-09-15|2312|10055|772|2092|1|9|15|3|2092|772|10055|Monday|2092Q3|N|N|N|2485392|2485635|2485040|2485315|N|N|N|N|N| +2485407|AAAAAAAAPJMOFCAA|2092-09-16|2312|10056|772|2092|2|9|16|3|2092|772|10056|Tuesday|2092Q3|N|N|N|2485392|2485635|2485041|2485316|N|N|N|N|N| +2485408|AAAAAAAAAKMOFCAA|2092-09-17|2312|10056|772|2092|3|9|17|3|2092|772|10056|Wednesday|2092Q3|N|N|N|2485392|2485635|2485042|2485317|N|N|N|N|N| +2485409|AAAAAAAABKMOFCAA|2092-09-18|2312|10056|772|2092|4|9|18|3|2092|772|10056|Thursday|2092Q3|N|N|N|2485392|2485635|2485043|2485318|N|N|N|N|N| +2485410|AAAAAAAACKMOFCAA|2092-09-19|2312|10056|772|2092|5|9|19|3|2092|772|10056|Friday|2092Q3|N|Y|N|2485392|2485635|2485044|2485319|N|N|N|N|N| +2485411|AAAAAAAADKMOFCAA|2092-09-20|2312|10056|772|2092|6|9|20|3|2092|772|10056|Saturday|2092Q3|N|Y|N|2485392|2485635|2485045|2485320|N|N|N|N|N| +2485412|AAAAAAAAEKMOFCAA|2092-09-21|2312|10056|772|2092|0|9|21|3|2092|772|10056|Sunday|2092Q3|N|N|N|2485392|2485635|2485046|2485321|N|N|N|N|N| +2485413|AAAAAAAAFKMOFCAA|2092-09-22|2312|10056|772|2092|1|9|22|3|2092|772|10056|Monday|2092Q3|N|N|N|2485392|2485635|2485047|2485322|N|N|N|N|N| +2485414|AAAAAAAAGKMOFCAA|2092-09-23|2312|10057|772|2092|2|9|23|3|2092|772|10057|Tuesday|2092Q3|N|N|N|2485392|2485635|2485048|2485323|N|N|N|N|N| +2485415|AAAAAAAAHKMOFCAA|2092-09-24|2312|10057|772|2092|3|9|24|3|2092|772|10057|Wednesday|2092Q3|N|N|N|2485392|2485635|2485049|2485324|N|N|N|N|N| +2485416|AAAAAAAAIKMOFCAA|2092-09-25|2312|10057|772|2092|4|9|25|3|2092|772|10057|Thursday|2092Q3|N|N|N|2485392|2485635|2485050|2485325|N|N|N|N|N| +2485417|AAAAAAAAJKMOFCAA|2092-09-26|2312|10057|772|2092|5|9|26|3|2092|772|10057|Friday|2092Q3|N|Y|N|2485392|2485635|2485051|2485326|N|N|N|N|N| +2485418|AAAAAAAAKKMOFCAA|2092-09-27|2312|10057|772|2092|6|9|27|3|2092|772|10057|Saturday|2092Q3|N|Y|N|2485392|2485635|2485052|2485327|N|N|N|N|N| +2485419|AAAAAAAALKMOFCAA|2092-09-28|2312|10057|772|2092|0|9|28|3|2092|772|10057|Sunday|2092Q3|N|N|N|2485392|2485635|2485053|2485328|N|N|N|N|N| +2485420|AAAAAAAAMKMOFCAA|2092-09-29|2312|10057|772|2092|1|9|29|3|2092|772|10057|Monday|2092Q3|N|N|N|2485392|2485635|2485054|2485329|N|N|N|N|N| +2485421|AAAAAAAANKMOFCAA|2092-09-30|2312|10058|772|2092|2|9|30|3|2092|772|10058|Tuesday|2092Q3|N|N|N|2485392|2485635|2485055|2485330|N|N|N|N|N| +2485422|AAAAAAAAOKMOFCAA|2092-10-01|2313|10058|772|2092|3|10|1|4|2092|772|10058|Wednesday|2092Q4|N|N|N|2485422|2485695|2485056|2485330|N|N|N|N|N| +2485423|AAAAAAAAPKMOFCAA|2092-10-02|2313|10058|772|2092|4|10|2|4|2092|772|10058|Thursday|2092Q4|N|N|N|2485422|2485695|2485057|2485331|N|N|N|N|N| +2485424|AAAAAAAAALMOFCAA|2092-10-03|2313|10058|772|2092|5|10|3|4|2092|772|10058|Friday|2092Q4|N|Y|N|2485422|2485695|2485058|2485332|N|N|N|N|N| +2485425|AAAAAAAABLMOFCAA|2092-10-04|2313|10058|772|2092|6|10|4|4|2092|772|10058|Saturday|2092Q4|N|Y|N|2485422|2485695|2485059|2485333|N|N|N|N|N| +2485426|AAAAAAAACLMOFCAA|2092-10-05|2313|10058|772|2092|0|10|5|4|2092|772|10058|Sunday|2092Q4|N|N|N|2485422|2485695|2485060|2485334|N|N|N|N|N| +2485427|AAAAAAAADLMOFCAA|2092-10-06|2313|10058|772|2092|1|10|6|4|2092|772|10058|Monday|2092Q4|N|N|N|2485422|2485695|2485061|2485335|N|N|N|N|N| +2485428|AAAAAAAAELMOFCAA|2092-10-07|2313|10059|772|2092|2|10|7|4|2092|772|10059|Tuesday|2092Q4|N|N|N|2485422|2485695|2485062|2485336|N|N|N|N|N| +2485429|AAAAAAAAFLMOFCAA|2092-10-08|2313|10059|772|2092|3|10|8|4|2092|772|10059|Wednesday|2092Q4|N|N|N|2485422|2485695|2485063|2485337|N|N|N|N|N| +2485430|AAAAAAAAGLMOFCAA|2092-10-09|2313|10059|772|2092|4|10|9|4|2092|772|10059|Thursday|2092Q4|N|N|N|2485422|2485695|2485064|2485338|N|N|N|N|N| +2485431|AAAAAAAAHLMOFCAA|2092-10-10|2313|10059|772|2092|5|10|10|4|2092|772|10059|Friday|2092Q4|N|Y|N|2485422|2485695|2485065|2485339|N|N|N|N|N| +2485432|AAAAAAAAILMOFCAA|2092-10-11|2313|10059|772|2092|6|10|11|4|2092|772|10059|Saturday|2092Q4|N|Y|N|2485422|2485695|2485066|2485340|N|N|N|N|N| +2485433|AAAAAAAAJLMOFCAA|2092-10-12|2313|10059|772|2092|0|10|12|4|2092|772|10059|Sunday|2092Q4|N|N|N|2485422|2485695|2485067|2485341|N|N|N|N|N| +2485434|AAAAAAAAKLMOFCAA|2092-10-13|2313|10059|772|2092|1|10|13|4|2092|772|10059|Monday|2092Q4|N|N|N|2485422|2485695|2485068|2485342|N|N|N|N|N| +2485435|AAAAAAAALLMOFCAA|2092-10-14|2313|10060|772|2092|2|10|14|4|2092|772|10060|Tuesday|2092Q4|N|N|N|2485422|2485695|2485069|2485343|N|N|N|N|N| +2485436|AAAAAAAAMLMOFCAA|2092-10-15|2313|10060|772|2092|3|10|15|4|2092|772|10060|Wednesday|2092Q4|N|N|N|2485422|2485695|2485070|2485344|N|N|N|N|N| +2485437|AAAAAAAANLMOFCAA|2092-10-16|2313|10060|772|2092|4|10|16|4|2092|772|10060|Thursday|2092Q4|N|N|N|2485422|2485695|2485071|2485345|N|N|N|N|N| +2485438|AAAAAAAAOLMOFCAA|2092-10-17|2313|10060|772|2092|5|10|17|4|2092|772|10060|Friday|2092Q4|N|Y|N|2485422|2485695|2485072|2485346|N|N|N|N|N| +2485439|AAAAAAAAPLMOFCAA|2092-10-18|2313|10060|772|2092|6|10|18|4|2092|772|10060|Saturday|2092Q4|N|Y|N|2485422|2485695|2485073|2485347|N|N|N|N|N| +2485440|AAAAAAAAAMMOFCAA|2092-10-19|2313|10060|772|2092|0|10|19|4|2092|772|10060|Sunday|2092Q4|N|N|N|2485422|2485695|2485074|2485348|N|N|N|N|N| +2485441|AAAAAAAABMMOFCAA|2092-10-20|2313|10060|772|2092|1|10|20|4|2092|772|10060|Monday|2092Q4|N|N|N|2485422|2485695|2485075|2485349|N|N|N|N|N| +2485442|AAAAAAAACMMOFCAA|2092-10-21|2313|10061|772|2092|2|10|21|4|2092|772|10061|Tuesday|2092Q4|N|N|N|2485422|2485695|2485076|2485350|N|N|N|N|N| +2485443|AAAAAAAADMMOFCAA|2092-10-22|2313|10061|772|2092|3|10|22|4|2092|772|10061|Wednesday|2092Q4|N|N|N|2485422|2485695|2485077|2485351|N|N|N|N|N| +2485444|AAAAAAAAEMMOFCAA|2092-10-23|2313|10061|772|2092|4|10|23|4|2092|772|10061|Thursday|2092Q4|N|N|N|2485422|2485695|2485078|2485352|N|N|N|N|N| +2485445|AAAAAAAAFMMOFCAA|2092-10-24|2313|10061|772|2092|5|10|24|4|2092|772|10061|Friday|2092Q4|N|Y|N|2485422|2485695|2485079|2485353|N|N|N|N|N| +2485446|AAAAAAAAGMMOFCAA|2092-10-25|2313|10061|772|2092|6|10|25|4|2092|772|10061|Saturday|2092Q4|N|Y|N|2485422|2485695|2485080|2485354|N|N|N|N|N| +2485447|AAAAAAAAHMMOFCAA|2092-10-26|2313|10061|772|2092|0|10|26|4|2092|772|10061|Sunday|2092Q4|N|N|N|2485422|2485695|2485081|2485355|N|N|N|N|N| +2485448|AAAAAAAAIMMOFCAA|2092-10-27|2313|10061|772|2092|1|10|27|4|2092|772|10061|Monday|2092Q4|N|N|N|2485422|2485695|2485082|2485356|N|N|N|N|N| +2485449|AAAAAAAAJMMOFCAA|2092-10-28|2313|10062|772|2092|2|10|28|4|2092|772|10062|Tuesday|2092Q4|N|N|N|2485422|2485695|2485083|2485357|N|N|N|N|N| +2485450|AAAAAAAAKMMOFCAA|2092-10-29|2313|10062|772|2092|3|10|29|4|2092|772|10062|Wednesday|2092Q4|N|N|N|2485422|2485695|2485084|2485358|N|N|N|N|N| +2485451|AAAAAAAALMMOFCAA|2092-10-30|2313|10062|772|2092|4|10|30|4|2092|772|10062|Thursday|2092Q4|N|N|N|2485422|2485695|2485085|2485359|N|N|N|N|N| +2485452|AAAAAAAAMMMOFCAA|2092-10-31|2313|10062|772|2092|5|10|31|4|2092|772|10062|Friday|2092Q4|N|Y|N|2485422|2485695|2485086|2485360|N|N|N|N|N| +2485453|AAAAAAAANMMOFCAA|2092-11-01|2314|10062|772|2092|6|11|1|4|2092|772|10062|Saturday|2092Q4|N|Y|N|2485453|2485757|2485087|2485361|N|N|N|N|N| +2485454|AAAAAAAAOMMOFCAA|2092-11-02|2314|10062|772|2092|0|11|2|4|2092|772|10062|Sunday|2092Q4|N|N|N|2485453|2485757|2485088|2485362|N|N|N|N|N| +2485455|AAAAAAAAPMMOFCAA|2092-11-03|2314|10062|772|2092|1|11|3|4|2092|772|10062|Monday|2092Q4|N|N|N|2485453|2485757|2485089|2485363|N|N|N|N|N| +2485456|AAAAAAAAANMOFCAA|2092-11-04|2314|10063|772|2092|2|11|4|4|2092|772|10063|Tuesday|2092Q4|N|N|N|2485453|2485757|2485090|2485364|N|N|N|N|N| +2485457|AAAAAAAABNMOFCAA|2092-11-05|2314|10063|772|2092|3|11|5|4|2092|772|10063|Wednesday|2092Q4|N|N|N|2485453|2485757|2485091|2485365|N|N|N|N|N| +2485458|AAAAAAAACNMOFCAA|2092-11-06|2314|10063|772|2092|4|11|6|4|2092|772|10063|Thursday|2092Q4|N|N|N|2485453|2485757|2485092|2485366|N|N|N|N|N| +2485459|AAAAAAAADNMOFCAA|2092-11-07|2314|10063|772|2092|5|11|7|4|2092|772|10063|Friday|2092Q4|N|Y|N|2485453|2485757|2485093|2485367|N|N|N|N|N| +2485460|AAAAAAAAENMOFCAA|2092-11-08|2314|10063|772|2092|6|11|8|4|2092|772|10063|Saturday|2092Q4|N|Y|N|2485453|2485757|2485094|2485368|N|N|N|N|N| +2485461|AAAAAAAAFNMOFCAA|2092-11-09|2314|10063|772|2092|0|11|9|4|2092|772|10063|Sunday|2092Q4|N|N|N|2485453|2485757|2485095|2485369|N|N|N|N|N| +2485462|AAAAAAAAGNMOFCAA|2092-11-10|2314|10063|772|2092|1|11|10|4|2092|772|10063|Monday|2092Q4|N|N|N|2485453|2485757|2485096|2485370|N|N|N|N|N| +2485463|AAAAAAAAHNMOFCAA|2092-11-11|2314|10064|772|2092|2|11|11|4|2092|772|10064|Tuesday|2092Q4|N|N|N|2485453|2485757|2485097|2485371|N|N|N|N|N| +2485464|AAAAAAAAINMOFCAA|2092-11-12|2314|10064|772|2092|3|11|12|4|2092|772|10064|Wednesday|2092Q4|N|N|N|2485453|2485757|2485098|2485372|N|N|N|N|N| +2485465|AAAAAAAAJNMOFCAA|2092-11-13|2314|10064|772|2092|4|11|13|4|2092|772|10064|Thursday|2092Q4|N|N|N|2485453|2485757|2485099|2485373|N|N|N|N|N| +2485466|AAAAAAAAKNMOFCAA|2092-11-14|2314|10064|772|2092|5|11|14|4|2092|772|10064|Friday|2092Q4|N|Y|N|2485453|2485757|2485100|2485374|N|N|N|N|N| +2485467|AAAAAAAALNMOFCAA|2092-11-15|2314|10064|772|2092|6|11|15|4|2092|772|10064|Saturday|2092Q4|N|Y|N|2485453|2485757|2485101|2485375|N|N|N|N|N| +2485468|AAAAAAAAMNMOFCAA|2092-11-16|2314|10064|772|2092|0|11|16|4|2092|772|10064|Sunday|2092Q4|N|N|N|2485453|2485757|2485102|2485376|N|N|N|N|N| +2485469|AAAAAAAANNMOFCAA|2092-11-17|2314|10064|772|2092|1|11|17|4|2092|772|10064|Monday|2092Q4|N|N|N|2485453|2485757|2485103|2485377|N|N|N|N|N| +2485470|AAAAAAAAONMOFCAA|2092-11-18|2314|10065|772|2092|2|11|18|4|2092|772|10065|Tuesday|2092Q4|N|N|N|2485453|2485757|2485104|2485378|N|N|N|N|N| +2485471|AAAAAAAAPNMOFCAA|2092-11-19|2314|10065|772|2092|3|11|19|4|2092|772|10065|Wednesday|2092Q4|N|N|N|2485453|2485757|2485105|2485379|N|N|N|N|N| +2485472|AAAAAAAAAOMOFCAA|2092-11-20|2314|10065|772|2092|4|11|20|4|2092|772|10065|Thursday|2092Q4|N|N|N|2485453|2485757|2485106|2485380|N|N|N|N|N| +2485473|AAAAAAAABOMOFCAA|2092-11-21|2314|10065|772|2092|5|11|21|4|2092|772|10065|Friday|2092Q4|N|Y|N|2485453|2485757|2485107|2485381|N|N|N|N|N| +2485474|AAAAAAAACOMOFCAA|2092-11-22|2314|10065|772|2092|6|11|22|4|2092|772|10065|Saturday|2092Q4|N|Y|N|2485453|2485757|2485108|2485382|N|N|N|N|N| +2485475|AAAAAAAADOMOFCAA|2092-11-23|2314|10065|772|2092|0|11|23|4|2092|772|10065|Sunday|2092Q4|N|N|N|2485453|2485757|2485109|2485383|N|N|N|N|N| +2485476|AAAAAAAAEOMOFCAA|2092-11-24|2314|10065|772|2092|1|11|24|4|2092|772|10065|Monday|2092Q4|N|N|N|2485453|2485757|2485110|2485384|N|N|N|N|N| +2485477|AAAAAAAAFOMOFCAA|2092-11-25|2314|10066|772|2092|2|11|25|4|2092|772|10066|Tuesday|2092Q4|N|N|N|2485453|2485757|2485111|2485385|N|N|N|N|N| +2485478|AAAAAAAAGOMOFCAA|2092-11-26|2314|10066|772|2092|3|11|26|4|2092|772|10066|Wednesday|2092Q4|N|N|N|2485453|2485757|2485112|2485386|N|N|N|N|N| +2485479|AAAAAAAAHOMOFCAA|2092-11-27|2314|10066|772|2092|4|11|27|4|2092|772|10066|Thursday|2092Q4|N|N|N|2485453|2485757|2485113|2485387|N|N|N|N|N| +2485480|AAAAAAAAIOMOFCAA|2092-11-28|2314|10066|772|2092|5|11|28|4|2092|772|10066|Friday|2092Q4|N|Y|N|2485453|2485757|2485114|2485388|N|N|N|N|N| +2485481|AAAAAAAAJOMOFCAA|2092-11-29|2314|10066|772|2092|6|11|29|4|2092|772|10066|Saturday|2092Q4|N|Y|N|2485453|2485757|2485115|2485389|N|N|N|N|N| +2485482|AAAAAAAAKOMOFCAA|2092-11-30|2314|10066|772|2092|0|11|30|4|2092|772|10066|Sunday|2092Q4|N|N|N|2485453|2485757|2485116|2485390|N|N|N|N|N| +2485483|AAAAAAAALOMOFCAA|2092-12-01|2315|10066|773|2092|1|12|1|4|2092|773|10066|Monday|2092Q4|N|N|N|2485483|2485817|2485117|2485391|N|N|N|N|N| +2485484|AAAAAAAAMOMOFCAA|2092-12-02|2315|10067|773|2092|2|12|2|4|2092|773|10067|Tuesday|2092Q4|N|N|N|2485483|2485817|2485118|2485392|N|N|N|N|N| +2485485|AAAAAAAANOMOFCAA|2092-12-03|2315|10067|773|2092|3|12|3|4|2092|773|10067|Wednesday|2092Q4|N|N|N|2485483|2485817|2485119|2485393|N|N|N|N|N| +2485486|AAAAAAAAOOMOFCAA|2092-12-04|2315|10067|773|2092|4|12|4|4|2092|773|10067|Thursday|2092Q4|N|N|N|2485483|2485817|2485120|2485394|N|N|N|N|N| +2485487|AAAAAAAAPOMOFCAA|2092-12-05|2315|10067|773|2092|5|12|5|4|2092|773|10067|Friday|2092Q4|N|Y|N|2485483|2485817|2485121|2485395|N|N|N|N|N| +2485488|AAAAAAAAAPMOFCAA|2092-12-06|2315|10067|773|2092|6|12|6|4|2092|773|10067|Saturday|2092Q4|N|Y|N|2485483|2485817|2485122|2485396|N|N|N|N|N| +2485489|AAAAAAAABPMOFCAA|2092-12-07|2315|10067|773|2092|0|12|7|4|2092|773|10067|Sunday|2092Q4|N|N|N|2485483|2485817|2485123|2485397|N|N|N|N|N| +2485490|AAAAAAAACPMOFCAA|2092-12-08|2315|10067|773|2092|1|12|8|4|2092|773|10067|Monday|2092Q4|N|N|N|2485483|2485817|2485124|2485398|N|N|N|N|N| +2485491|AAAAAAAADPMOFCAA|2092-12-09|2315|10068|773|2092|2|12|9|4|2092|773|10068|Tuesday|2092Q4|N|N|N|2485483|2485817|2485125|2485399|N|N|N|N|N| +2485492|AAAAAAAAEPMOFCAA|2092-12-10|2315|10068|773|2092|3|12|10|4|2092|773|10068|Wednesday|2092Q4|N|N|N|2485483|2485817|2485126|2485400|N|N|N|N|N| +2485493|AAAAAAAAFPMOFCAA|2092-12-11|2315|10068|773|2092|4|12|11|4|2092|773|10068|Thursday|2092Q4|N|N|N|2485483|2485817|2485127|2485401|N|N|N|N|N| +2485494|AAAAAAAAGPMOFCAA|2092-12-12|2315|10068|773|2092|5|12|12|4|2092|773|10068|Friday|2092Q4|N|Y|N|2485483|2485817|2485128|2485402|N|N|N|N|N| +2485495|AAAAAAAAHPMOFCAA|2092-12-13|2315|10068|773|2092|6|12|13|4|2092|773|10068|Saturday|2092Q4|N|Y|N|2485483|2485817|2485129|2485403|N|N|N|N|N| +2485496|AAAAAAAAIPMOFCAA|2092-12-14|2315|10068|773|2092|0|12|14|4|2092|773|10068|Sunday|2092Q4|N|N|N|2485483|2485817|2485130|2485404|N|N|N|N|N| +2485497|AAAAAAAAJPMOFCAA|2092-12-15|2315|10068|773|2092|1|12|15|4|2092|773|10068|Monday|2092Q4|N|N|N|2485483|2485817|2485131|2485405|N|N|N|N|N| +2485498|AAAAAAAAKPMOFCAA|2092-12-16|2315|10069|773|2092|2|12|16|4|2092|773|10069|Tuesday|2092Q4|N|N|N|2485483|2485817|2485132|2485406|N|N|N|N|N| +2485499|AAAAAAAALPMOFCAA|2092-12-17|2315|10069|773|2092|3|12|17|4|2092|773|10069|Wednesday|2092Q4|N|N|N|2485483|2485817|2485133|2485407|N|N|N|N|N| +2485500|AAAAAAAAMPMOFCAA|2092-12-18|2315|10069|773|2092|4|12|18|4|2092|773|10069|Thursday|2092Q4|N|N|N|2485483|2485817|2485134|2485408|N|N|N|N|N| +2485501|AAAAAAAANPMOFCAA|2092-12-19|2315|10069|773|2092|5|12|19|4|2092|773|10069|Friday|2092Q4|N|Y|N|2485483|2485817|2485135|2485409|N|N|N|N|N| +2485502|AAAAAAAAOPMOFCAA|2092-12-20|2315|10069|773|2092|6|12|20|4|2092|773|10069|Saturday|2092Q4|N|Y|N|2485483|2485817|2485136|2485410|N|N|N|N|N| +2485503|AAAAAAAAPPMOFCAA|2092-12-21|2315|10069|773|2092|0|12|21|4|2092|773|10069|Sunday|2092Q4|N|N|N|2485483|2485817|2485137|2485411|N|N|N|N|N| +2485504|AAAAAAAAAANOFCAA|2092-12-22|2315|10069|773|2092|1|12|22|4|2092|773|10069|Monday|2092Q4|N|N|N|2485483|2485817|2485138|2485412|N|N|N|N|N| +2485505|AAAAAAAABANOFCAA|2092-12-23|2315|10070|773|2092|2|12|23|4|2092|773|10070|Tuesday|2092Q4|N|N|N|2485483|2485817|2485139|2485413|N|N|N|N|N| +2485506|AAAAAAAACANOFCAA|2092-12-24|2315|10070|773|2092|3|12|24|4|2092|773|10070|Wednesday|2092Q4|N|N|N|2485483|2485817|2485140|2485414|N|N|N|N|N| +2485507|AAAAAAAADANOFCAA|2092-12-25|2315|10070|773|2092|4|12|25|4|2092|773|10070|Thursday|2092Q4|Y|N|N|2485483|2485817|2485141|2485415|N|N|N|N|N| +2485508|AAAAAAAAEANOFCAA|2092-12-26|2315|10070|773|2092|5|12|26|4|2092|773|10070|Friday|2092Q4|N|Y|Y|2485483|2485817|2485142|2485416|N|N|N|N|N| +2485509|AAAAAAAAFANOFCAA|2092-12-27|2315|10070|773|2092|6|12|27|4|2092|773|10070|Saturday|2092Q4|N|Y|N|2485483|2485817|2485143|2485417|N|N|N|N|N| +2485510|AAAAAAAAGANOFCAA|2092-12-28|2315|10070|773|2092|0|12|28|4|2092|773|10070|Sunday|2092Q4|N|N|N|2485483|2485817|2485144|2485418|N|N|N|N|N| +2485511|AAAAAAAAHANOFCAA|2092-12-29|2315|10070|773|2092|1|12|29|4|2092|773|10070|Monday|2092Q4|N|N|N|2485483|2485817|2485145|2485419|N|N|N|N|N| +2485512|AAAAAAAAIANOFCAA|2092-12-30|2315|10071|773|2092|2|12|30|4|2092|773|10071|Tuesday|2092Q4|N|N|N|2485483|2485817|2485146|2485420|N|N|N|N|N| +2485513|AAAAAAAAJANOFCAA|2092-12-31|2315|10071|773|2092|3|12|31|4|2092|773|10071|Wednesday|2092Q4|Y|N|N|2485483|2485817|2485147|2485421|N|N|N|N|N| +2485514|AAAAAAAAKANOFCAA|2093-01-01|2316|10071|773|2093|4|1|1|1|2093|773|10071|Thursday|2093Q1|Y|N|Y|2485514|2485513|2485148|2485422|N|N|N|N|N| +2485515|AAAAAAAALANOFCAA|2093-01-02|2316|10071|773|2093|5|1|2|1|2093|773|10071|Friday|2093Q1|N|Y|Y|2485514|2485513|2485149|2485423|N|N|N|N|N| +2485516|AAAAAAAAMANOFCAA|2093-01-03|2316|10071|773|2093|6|1|3|1|2093|773|10071|Saturday|2093Q1|N|Y|N|2485514|2485513|2485150|2485424|N|N|N|N|N| +2485517|AAAAAAAANANOFCAA|2093-01-04|2316|10071|773|2093|0|1|4|1|2093|773|10071|Sunday|2093Q1|N|N|N|2485514|2485513|2485151|2485425|N|N|N|N|N| +2485518|AAAAAAAAOANOFCAA|2093-01-05|2316|10071|773|2093|1|1|5|1|2093|773|10071|Monday|2093Q1|N|N|N|2485514|2485513|2485152|2485426|N|N|N|N|N| +2485519|AAAAAAAAPANOFCAA|2093-01-06|2316|10072|773|2093|2|1|6|1|2093|773|10072|Tuesday|2093Q1|N|N|N|2485514|2485513|2485153|2485427|N|N|N|N|N| +2485520|AAAAAAAAABNOFCAA|2093-01-07|2316|10072|773|2093|3|1|7|1|2093|773|10072|Wednesday|2093Q1|N|N|N|2485514|2485513|2485154|2485428|N|N|N|N|N| +2485521|AAAAAAAABBNOFCAA|2093-01-08|2316|10072|773|2093|4|1|8|1|2093|773|10072|Thursday|2093Q1|N|N|N|2485514|2485513|2485155|2485429|N|N|N|N|N| +2485522|AAAAAAAACBNOFCAA|2093-01-09|2316|10072|773|2093|5|1|9|1|2093|773|10072|Friday|2093Q1|N|Y|N|2485514|2485513|2485156|2485430|N|N|N|N|N| +2485523|AAAAAAAADBNOFCAA|2093-01-10|2316|10072|773|2093|6|1|10|1|2093|773|10072|Saturday|2093Q1|N|Y|N|2485514|2485513|2485157|2485431|N|N|N|N|N| +2485524|AAAAAAAAEBNOFCAA|2093-01-11|2316|10072|773|2093|0|1|11|1|2093|773|10072|Sunday|2093Q1|N|N|N|2485514|2485513|2485158|2485432|N|N|N|N|N| +2485525|AAAAAAAAFBNOFCAA|2093-01-12|2316|10072|773|2093|1|1|12|1|2093|773|10072|Monday|2093Q1|N|N|N|2485514|2485513|2485159|2485433|N|N|N|N|N| +2485526|AAAAAAAAGBNOFCAA|2093-01-13|2316|10073|773|2093|2|1|13|1|2093|773|10073|Tuesday|2093Q1|N|N|N|2485514|2485513|2485160|2485434|N|N|N|N|N| +2485527|AAAAAAAAHBNOFCAA|2093-01-14|2316|10073|773|2093|3|1|14|1|2093|773|10073|Wednesday|2093Q1|N|N|N|2485514|2485513|2485161|2485435|N|N|N|N|N| +2485528|AAAAAAAAIBNOFCAA|2093-01-15|2316|10073|773|2093|4|1|15|1|2093|773|10073|Thursday|2093Q1|N|N|N|2485514|2485513|2485162|2485436|N|N|N|N|N| +2485529|AAAAAAAAJBNOFCAA|2093-01-16|2316|10073|773|2093|5|1|16|1|2093|773|10073|Friday|2093Q1|N|Y|N|2485514|2485513|2485163|2485437|N|N|N|N|N| +2485530|AAAAAAAAKBNOFCAA|2093-01-17|2316|10073|773|2093|6|1|17|1|2093|773|10073|Saturday|2093Q1|N|Y|N|2485514|2485513|2485164|2485438|N|N|N|N|N| +2485531|AAAAAAAALBNOFCAA|2093-01-18|2316|10073|773|2093|0|1|18|1|2093|773|10073|Sunday|2093Q1|N|N|N|2485514|2485513|2485165|2485439|N|N|N|N|N| +2485532|AAAAAAAAMBNOFCAA|2093-01-19|2316|10073|773|2093|1|1|19|1|2093|773|10073|Monday|2093Q1|N|N|N|2485514|2485513|2485166|2485440|N|N|N|N|N| +2485533|AAAAAAAANBNOFCAA|2093-01-20|2316|10074|773|2093|2|1|20|1|2093|773|10074|Tuesday|2093Q1|N|N|N|2485514|2485513|2485167|2485441|N|N|N|N|N| +2485534|AAAAAAAAOBNOFCAA|2093-01-21|2316|10074|773|2093|3|1|21|1|2093|773|10074|Wednesday|2093Q1|N|N|N|2485514|2485513|2485168|2485442|N|N|N|N|N| +2485535|AAAAAAAAPBNOFCAA|2093-01-22|2316|10074|773|2093|4|1|22|1|2093|773|10074|Thursday|2093Q1|N|N|N|2485514|2485513|2485169|2485443|N|N|N|N|N| +2485536|AAAAAAAAACNOFCAA|2093-01-23|2316|10074|773|2093|5|1|23|1|2093|773|10074|Friday|2093Q1|N|Y|N|2485514|2485513|2485170|2485444|N|N|N|N|N| +2485537|AAAAAAAABCNOFCAA|2093-01-24|2316|10074|773|2093|6|1|24|1|2093|773|10074|Saturday|2093Q1|N|Y|N|2485514|2485513|2485171|2485445|N|N|N|N|N| +2485538|AAAAAAAACCNOFCAA|2093-01-25|2316|10074|773|2093|0|1|25|1|2093|773|10074|Sunday|2093Q1|N|N|N|2485514|2485513|2485172|2485446|N|N|N|N|N| +2485539|AAAAAAAADCNOFCAA|2093-01-26|2316|10074|773|2093|1|1|26|1|2093|773|10074|Monday|2093Q1|N|N|N|2485514|2485513|2485173|2485447|N|N|N|N|N| +2485540|AAAAAAAAECNOFCAA|2093-01-27|2316|10075|773|2093|2|1|27|1|2093|773|10075|Tuesday|2093Q1|N|N|N|2485514|2485513|2485174|2485448|N|N|N|N|N| +2485541|AAAAAAAAFCNOFCAA|2093-01-28|2316|10075|773|2093|3|1|28|1|2093|773|10075|Wednesday|2093Q1|N|N|N|2485514|2485513|2485175|2485449|N|N|N|N|N| +2485542|AAAAAAAAGCNOFCAA|2093-01-29|2316|10075|773|2093|4|1|29|1|2093|773|10075|Thursday|2093Q1|N|N|N|2485514|2485513|2485176|2485450|N|N|N|N|N| +2485543|AAAAAAAAHCNOFCAA|2093-01-30|2316|10075|773|2093|5|1|30|1|2093|773|10075|Friday|2093Q1|N|Y|N|2485514|2485513|2485177|2485451|N|N|N|N|N| +2485544|AAAAAAAAICNOFCAA|2093-01-31|2316|10075|773|2093|6|1|31|1|2093|773|10075|Saturday|2093Q1|N|Y|N|2485514|2485513|2485178|2485452|N|N|N|N|N| +2485545|AAAAAAAAJCNOFCAA|2093-02-01|2317|10075|773|2093|0|2|1|1|2093|773|10075|Sunday|2093Q1|N|N|N|2485545|2485575|2485179|2485453|N|N|N|N|N| +2485546|AAAAAAAAKCNOFCAA|2093-02-02|2317|10075|773|2093|1|2|2|1|2093|773|10075|Monday|2093Q1|N|N|N|2485545|2485575|2485180|2485454|N|N|N|N|N| +2485547|AAAAAAAALCNOFCAA|2093-02-03|2317|10076|773|2093|2|2|3|1|2093|773|10076|Tuesday|2093Q1|N|N|N|2485545|2485575|2485181|2485455|N|N|N|N|N| +2485548|AAAAAAAAMCNOFCAA|2093-02-04|2317|10076|773|2093|3|2|4|1|2093|773|10076|Wednesday|2093Q1|N|N|N|2485545|2485575|2485182|2485456|N|N|N|N|N| +2485549|AAAAAAAANCNOFCAA|2093-02-05|2317|10076|773|2093|4|2|5|1|2093|773|10076|Thursday|2093Q1|N|N|N|2485545|2485575|2485183|2485457|N|N|N|N|N| +2485550|AAAAAAAAOCNOFCAA|2093-02-06|2317|10076|773|2093|5|2|6|1|2093|773|10076|Friday|2093Q1|N|Y|N|2485545|2485575|2485184|2485458|N|N|N|N|N| +2485551|AAAAAAAAPCNOFCAA|2093-02-07|2317|10076|773|2093|6|2|7|1|2093|773|10076|Saturday|2093Q1|N|Y|N|2485545|2485575|2485185|2485459|N|N|N|N|N| +2485552|AAAAAAAAADNOFCAA|2093-02-08|2317|10076|773|2093|0|2|8|1|2093|773|10076|Sunday|2093Q1|N|N|N|2485545|2485575|2485186|2485460|N|N|N|N|N| +2485553|AAAAAAAABDNOFCAA|2093-02-09|2317|10076|773|2093|1|2|9|1|2093|773|10076|Monday|2093Q1|N|N|N|2485545|2485575|2485187|2485461|N|N|N|N|N| +2485554|AAAAAAAACDNOFCAA|2093-02-10|2317|10077|773|2093|2|2|10|1|2093|773|10077|Tuesday|2093Q1|N|N|N|2485545|2485575|2485188|2485462|N|N|N|N|N| +2485555|AAAAAAAADDNOFCAA|2093-02-11|2317|10077|773|2093|3|2|11|1|2093|773|10077|Wednesday|2093Q1|N|N|N|2485545|2485575|2485189|2485463|N|N|N|N|N| +2485556|AAAAAAAAEDNOFCAA|2093-02-12|2317|10077|773|2093|4|2|12|1|2093|773|10077|Thursday|2093Q1|N|N|N|2485545|2485575|2485190|2485464|N|N|N|N|N| +2485557|AAAAAAAAFDNOFCAA|2093-02-13|2317|10077|773|2093|5|2|13|1|2093|773|10077|Friday|2093Q1|N|Y|N|2485545|2485575|2485191|2485465|N|N|N|N|N| +2485558|AAAAAAAAGDNOFCAA|2093-02-14|2317|10077|773|2093|6|2|14|1|2093|773|10077|Saturday|2093Q1|N|Y|N|2485545|2485575|2485192|2485466|N|N|N|N|N| +2485559|AAAAAAAAHDNOFCAA|2093-02-15|2317|10077|773|2093|0|2|15|1|2093|773|10077|Sunday|2093Q1|N|N|N|2485545|2485575|2485193|2485467|N|N|N|N|N| +2485560|AAAAAAAAIDNOFCAA|2093-02-16|2317|10077|773|2093|1|2|16|1|2093|773|10077|Monday|2093Q1|N|N|N|2485545|2485575|2485194|2485468|N|N|N|N|N| +2485561|AAAAAAAAJDNOFCAA|2093-02-17|2317|10078|773|2093|2|2|17|1|2093|773|10078|Tuesday|2093Q1|N|N|N|2485545|2485575|2485195|2485469|N|N|N|N|N| +2485562|AAAAAAAAKDNOFCAA|2093-02-18|2317|10078|773|2093|3|2|18|1|2093|773|10078|Wednesday|2093Q1|N|N|N|2485545|2485575|2485196|2485470|N|N|N|N|N| +2485563|AAAAAAAALDNOFCAA|2093-02-19|2317|10078|773|2093|4|2|19|1|2093|773|10078|Thursday|2093Q1|N|N|N|2485545|2485575|2485197|2485471|N|N|N|N|N| +2485564|AAAAAAAAMDNOFCAA|2093-02-20|2317|10078|773|2093|5|2|20|1|2093|773|10078|Friday|2093Q1|N|Y|N|2485545|2485575|2485198|2485472|N|N|N|N|N| +2485565|AAAAAAAANDNOFCAA|2093-02-21|2317|10078|773|2093|6|2|21|1|2093|773|10078|Saturday|2093Q1|N|Y|N|2485545|2485575|2485199|2485473|N|N|N|N|N| +2485566|AAAAAAAAODNOFCAA|2093-02-22|2317|10078|773|2093|0|2|22|1|2093|773|10078|Sunday|2093Q1|N|N|N|2485545|2485575|2485200|2485474|N|N|N|N|N| +2485567|AAAAAAAAPDNOFCAA|2093-02-23|2317|10078|773|2093|1|2|23|1|2093|773|10078|Monday|2093Q1|N|N|N|2485545|2485575|2485201|2485475|N|N|N|N|N| +2485568|AAAAAAAAAENOFCAA|2093-02-24|2317|10079|773|2093|2|2|24|1|2093|773|10079|Tuesday|2093Q1|N|N|N|2485545|2485575|2485202|2485476|N|N|N|N|N| +2485569|AAAAAAAABENOFCAA|2093-02-25|2317|10079|773|2093|3|2|25|1|2093|773|10079|Wednesday|2093Q1|N|N|N|2485545|2485575|2485203|2485477|N|N|N|N|N| +2485570|AAAAAAAACENOFCAA|2093-02-26|2317|10079|773|2093|4|2|26|1|2093|773|10079|Thursday|2093Q1|N|N|N|2485545|2485575|2485204|2485478|N|N|N|N|N| +2485571|AAAAAAAADENOFCAA|2093-02-27|2317|10079|773|2093|5|2|27|1|2093|773|10079|Friday|2093Q1|N|Y|N|2485545|2485575|2485205|2485479|N|N|N|N|N| +2485572|AAAAAAAAEENOFCAA|2093-02-28|2317|10079|773|2093|6|2|28|1|2093|773|10079|Saturday|2093Q1|N|Y|N|2485545|2485575|2485206|2485480|N|N|N|N|N| +2485573|AAAAAAAAFENOFCAA|2093-03-01|2318|10079|774|2093|0|3|1|1|2093|774|10079|Sunday|2093Q1|N|N|N|2485573|2485631|2485208|2485481|N|N|N|N|N| +2485574|AAAAAAAAGENOFCAA|2093-03-02|2318|10079|774|2093|1|3|2|1|2093|774|10079|Monday|2093Q1|N|N|N|2485573|2485631|2485209|2485482|N|N|N|N|N| +2485575|AAAAAAAAHENOFCAA|2093-03-03|2318|10080|774|2093|2|3|3|1|2093|774|10080|Tuesday|2093Q1|N|N|N|2485573|2485631|2485210|2485483|N|N|N|N|N| +2485576|AAAAAAAAIENOFCAA|2093-03-04|2318|10080|774|2093|3|3|4|1|2093|774|10080|Wednesday|2093Q1|N|N|N|2485573|2485631|2485211|2485484|N|N|N|N|N| +2485577|AAAAAAAAJENOFCAA|2093-03-05|2318|10080|774|2093|4|3|5|1|2093|774|10080|Thursday|2093Q1|N|N|N|2485573|2485631|2485212|2485485|N|N|N|N|N| +2485578|AAAAAAAAKENOFCAA|2093-03-06|2318|10080|774|2093|5|3|6|1|2093|774|10080|Friday|2093Q1|N|Y|N|2485573|2485631|2485213|2485486|N|N|N|N|N| +2485579|AAAAAAAALENOFCAA|2093-03-07|2318|10080|774|2093|6|3|7|1|2093|774|10080|Saturday|2093Q1|N|Y|N|2485573|2485631|2485214|2485487|N|N|N|N|N| +2485580|AAAAAAAAMENOFCAA|2093-03-08|2318|10080|774|2093|0|3|8|1|2093|774|10080|Sunday|2093Q1|N|N|N|2485573|2485631|2485215|2485488|N|N|N|N|N| +2485581|AAAAAAAANENOFCAA|2093-03-09|2318|10080|774|2093|1|3|9|1|2093|774|10080|Monday|2093Q1|N|N|N|2485573|2485631|2485216|2485489|N|N|N|N|N| +2485582|AAAAAAAAOENOFCAA|2093-03-10|2318|10081|774|2093|2|3|10|1|2093|774|10081|Tuesday|2093Q1|N|N|N|2485573|2485631|2485217|2485490|N|N|N|N|N| +2485583|AAAAAAAAPENOFCAA|2093-03-11|2318|10081|774|2093|3|3|11|1|2093|774|10081|Wednesday|2093Q1|N|N|N|2485573|2485631|2485218|2485491|N|N|N|N|N| +2485584|AAAAAAAAAFNOFCAA|2093-03-12|2318|10081|774|2093|4|3|12|1|2093|774|10081|Thursday|2093Q1|N|N|N|2485573|2485631|2485219|2485492|N|N|N|N|N| +2485585|AAAAAAAABFNOFCAA|2093-03-13|2318|10081|774|2093|5|3|13|1|2093|774|10081|Friday|2093Q1|N|Y|N|2485573|2485631|2485220|2485493|N|N|N|N|N| +2485586|AAAAAAAACFNOFCAA|2093-03-14|2318|10081|774|2093|6|3|14|1|2093|774|10081|Saturday|2093Q1|N|Y|N|2485573|2485631|2485221|2485494|N|N|N|N|N| +2485587|AAAAAAAADFNOFCAA|2093-03-15|2318|10081|774|2093|0|3|15|1|2093|774|10081|Sunday|2093Q1|N|N|N|2485573|2485631|2485222|2485495|N|N|N|N|N| +2485588|AAAAAAAAEFNOFCAA|2093-03-16|2318|10081|774|2093|1|3|16|1|2093|774|10081|Monday|2093Q1|N|N|N|2485573|2485631|2485223|2485496|N|N|N|N|N| +2485589|AAAAAAAAFFNOFCAA|2093-03-17|2318|10082|774|2093|2|3|17|1|2093|774|10082|Tuesday|2093Q1|N|N|N|2485573|2485631|2485224|2485497|N|N|N|N|N| +2485590|AAAAAAAAGFNOFCAA|2093-03-18|2318|10082|774|2093|3|3|18|1|2093|774|10082|Wednesday|2093Q1|N|N|N|2485573|2485631|2485225|2485498|N|N|N|N|N| +2485591|AAAAAAAAHFNOFCAA|2093-03-19|2318|10082|774|2093|4|3|19|1|2093|774|10082|Thursday|2093Q1|N|N|N|2485573|2485631|2485226|2485499|N|N|N|N|N| +2485592|AAAAAAAAIFNOFCAA|2093-03-20|2318|10082|774|2093|5|3|20|1|2093|774|10082|Friday|2093Q1|N|Y|N|2485573|2485631|2485227|2485500|N|N|N|N|N| +2485593|AAAAAAAAJFNOFCAA|2093-03-21|2318|10082|774|2093|6|3|21|1|2093|774|10082|Saturday|2093Q1|N|Y|N|2485573|2485631|2485228|2485501|N|N|N|N|N| +2485594|AAAAAAAAKFNOFCAA|2093-03-22|2318|10082|774|2093|0|3|22|1|2093|774|10082|Sunday|2093Q1|N|N|N|2485573|2485631|2485229|2485502|N|N|N|N|N| +2485595|AAAAAAAALFNOFCAA|2093-03-23|2318|10082|774|2093|1|3|23|1|2093|774|10082|Monday|2093Q1|N|N|N|2485573|2485631|2485230|2485503|N|N|N|N|N| +2485596|AAAAAAAAMFNOFCAA|2093-03-24|2318|10083|774|2093|2|3|24|1|2093|774|10083|Tuesday|2093Q1|N|N|N|2485573|2485631|2485231|2485504|N|N|N|N|N| +2485597|AAAAAAAANFNOFCAA|2093-03-25|2318|10083|774|2093|3|3|25|1|2093|774|10083|Wednesday|2093Q1|N|N|N|2485573|2485631|2485232|2485505|N|N|N|N|N| +2485598|AAAAAAAAOFNOFCAA|2093-03-26|2318|10083|774|2093|4|3|26|1|2093|774|10083|Thursday|2093Q1|N|N|N|2485573|2485631|2485233|2485506|N|N|N|N|N| +2485599|AAAAAAAAPFNOFCAA|2093-03-27|2318|10083|774|2093|5|3|27|1|2093|774|10083|Friday|2093Q1|N|Y|N|2485573|2485631|2485234|2485507|N|N|N|N|N| +2485600|AAAAAAAAAGNOFCAA|2093-03-28|2318|10083|774|2093|6|3|28|1|2093|774|10083|Saturday|2093Q1|N|Y|N|2485573|2485631|2485235|2485508|N|N|N|N|N| +2485601|AAAAAAAABGNOFCAA|2093-03-29|2318|10083|774|2093|0|3|29|1|2093|774|10083|Sunday|2093Q1|N|N|N|2485573|2485631|2485236|2485509|N|N|N|N|N| +2485602|AAAAAAAACGNOFCAA|2093-03-30|2318|10083|774|2093|1|3|30|1|2093|774|10083|Monday|2093Q1|N|N|N|2485573|2485631|2485237|2485510|N|N|N|N|N| +2485603|AAAAAAAADGNOFCAA|2093-03-31|2318|10084|774|2093|2|3|31|1|2093|774|10084|Tuesday|2093Q1|N|N|N|2485573|2485631|2485238|2485511|N|N|N|N|N| +2485604|AAAAAAAAEGNOFCAA|2093-04-01|2319|10084|774|2093|3|4|1|1|2093|774|10084|Wednesday|2093Q1|N|N|N|2485604|2485693|2485239|2485514|N|N|N|N|N| +2485605|AAAAAAAAFGNOFCAA|2093-04-02|2319|10084|774|2093|4|4|2|2|2093|774|10084|Thursday|2093Q2|N|N|N|2485604|2485693|2485240|2485515|N|N|N|N|N| +2485606|AAAAAAAAGGNOFCAA|2093-04-03|2319|10084|774|2093|5|4|3|2|2093|774|10084|Friday|2093Q2|N|Y|N|2485604|2485693|2485241|2485516|N|N|N|N|N| +2485607|AAAAAAAAHGNOFCAA|2093-04-04|2319|10084|774|2093|6|4|4|2|2093|774|10084|Saturday|2093Q2|N|Y|N|2485604|2485693|2485242|2485517|N|N|N|N|N| +2485608|AAAAAAAAIGNOFCAA|2093-04-05|2319|10084|774|2093|0|4|5|2|2093|774|10084|Sunday|2093Q2|N|N|N|2485604|2485693|2485243|2485518|N|N|N|N|N| +2485609|AAAAAAAAJGNOFCAA|2093-04-06|2319|10084|774|2093|1|4|6|2|2093|774|10084|Monday|2093Q2|N|N|N|2485604|2485693|2485244|2485519|N|N|N|N|N| +2485610|AAAAAAAAKGNOFCAA|2093-04-07|2319|10085|774|2093|2|4|7|2|2093|774|10085|Tuesday|2093Q2|N|N|N|2485604|2485693|2485245|2485520|N|N|N|N|N| +2485611|AAAAAAAALGNOFCAA|2093-04-08|2319|10085|774|2093|3|4|8|2|2093|774|10085|Wednesday|2093Q2|N|N|N|2485604|2485693|2485246|2485521|N|N|N|N|N| +2485612|AAAAAAAAMGNOFCAA|2093-04-09|2319|10085|774|2093|4|4|9|2|2093|774|10085|Thursday|2093Q2|N|N|N|2485604|2485693|2485247|2485522|N|N|N|N|N| +2485613|AAAAAAAANGNOFCAA|2093-04-10|2319|10085|774|2093|5|4|10|2|2093|774|10085|Friday|2093Q2|N|Y|N|2485604|2485693|2485248|2485523|N|N|N|N|N| +2485614|AAAAAAAAOGNOFCAA|2093-04-11|2319|10085|774|2093|6|4|11|2|2093|774|10085|Saturday|2093Q2|N|Y|N|2485604|2485693|2485249|2485524|N|N|N|N|N| +2485615|AAAAAAAAPGNOFCAA|2093-04-12|2319|10085|774|2093|0|4|12|2|2093|774|10085|Sunday|2093Q2|N|N|N|2485604|2485693|2485250|2485525|N|N|N|N|N| +2485616|AAAAAAAAAHNOFCAA|2093-04-13|2319|10085|774|2093|1|4|13|2|2093|774|10085|Monday|2093Q2|N|N|N|2485604|2485693|2485251|2485526|N|N|N|N|N| +2485617|AAAAAAAABHNOFCAA|2093-04-14|2319|10086|774|2093|2|4|14|2|2093|774|10086|Tuesday|2093Q2|N|N|N|2485604|2485693|2485252|2485527|N|N|N|N|N| +2485618|AAAAAAAACHNOFCAA|2093-04-15|2319|10086|774|2093|3|4|15|2|2093|774|10086|Wednesday|2093Q2|N|N|N|2485604|2485693|2485253|2485528|N|N|N|N|N| +2485619|AAAAAAAADHNOFCAA|2093-04-16|2319|10086|774|2093|4|4|16|2|2093|774|10086|Thursday|2093Q2|N|N|N|2485604|2485693|2485254|2485529|N|N|N|N|N| +2485620|AAAAAAAAEHNOFCAA|2093-04-17|2319|10086|774|2093|5|4|17|2|2093|774|10086|Friday|2093Q2|N|Y|N|2485604|2485693|2485255|2485530|N|N|N|N|N| +2485621|AAAAAAAAFHNOFCAA|2093-04-18|2319|10086|774|2093|6|4|18|2|2093|774|10086|Saturday|2093Q2|N|Y|N|2485604|2485693|2485256|2485531|N|N|N|N|N| +2485622|AAAAAAAAGHNOFCAA|2093-04-19|2319|10086|774|2093|0|4|19|2|2093|774|10086|Sunday|2093Q2|N|N|N|2485604|2485693|2485257|2485532|N|N|N|N|N| +2485623|AAAAAAAAHHNOFCAA|2093-04-20|2319|10086|774|2093|1|4|20|2|2093|774|10086|Monday|2093Q2|N|N|N|2485604|2485693|2485258|2485533|N|N|N|N|N| +2485624|AAAAAAAAIHNOFCAA|2093-04-21|2319|10087|774|2093|2|4|21|2|2093|774|10087|Tuesday|2093Q2|N|N|N|2485604|2485693|2485259|2485534|N|N|N|N|N| +2485625|AAAAAAAAJHNOFCAA|2093-04-22|2319|10087|774|2093|3|4|22|2|2093|774|10087|Wednesday|2093Q2|N|N|N|2485604|2485693|2485260|2485535|N|N|N|N|N| +2485626|AAAAAAAAKHNOFCAA|2093-04-23|2319|10087|774|2093|4|4|23|2|2093|774|10087|Thursday|2093Q2|N|N|N|2485604|2485693|2485261|2485536|N|N|N|N|N| +2485627|AAAAAAAALHNOFCAA|2093-04-24|2319|10087|774|2093|5|4|24|2|2093|774|10087|Friday|2093Q2|N|Y|N|2485604|2485693|2485262|2485537|N|N|N|N|N| +2485628|AAAAAAAAMHNOFCAA|2093-04-25|2319|10087|774|2093|6|4|25|2|2093|774|10087|Saturday|2093Q2|N|Y|N|2485604|2485693|2485263|2485538|N|N|N|N|N| +2485629|AAAAAAAANHNOFCAA|2093-04-26|2319|10087|774|2093|0|4|26|2|2093|774|10087|Sunday|2093Q2|N|N|N|2485604|2485693|2485264|2485539|N|N|N|N|N| +2485630|AAAAAAAAOHNOFCAA|2093-04-27|2319|10087|774|2093|1|4|27|2|2093|774|10087|Monday|2093Q2|N|N|N|2485604|2485693|2485265|2485540|N|N|N|N|N| +2485631|AAAAAAAAPHNOFCAA|2093-04-28|2319|10088|774|2093|2|4|28|2|2093|774|10088|Tuesday|2093Q2|N|N|N|2485604|2485693|2485266|2485541|N|N|N|N|N| +2485632|AAAAAAAAAINOFCAA|2093-04-29|2319|10088|774|2093|3|4|29|2|2093|774|10088|Wednesday|2093Q2|N|N|N|2485604|2485693|2485267|2485542|N|N|N|N|N| +2485633|AAAAAAAABINOFCAA|2093-04-30|2319|10088|774|2093|4|4|30|2|2093|774|10088|Thursday|2093Q2|N|N|N|2485604|2485693|2485268|2485543|N|N|N|N|N| +2485634|AAAAAAAACINOFCAA|2093-05-01|2320|10088|774|2093|5|5|1|2|2093|774|10088|Friday|2093Q2|N|Y|N|2485634|2485753|2485269|2485544|N|N|N|N|N| +2485635|AAAAAAAADINOFCAA|2093-05-02|2320|10088|774|2093|6|5|2|2|2093|774|10088|Saturday|2093Q2|N|Y|N|2485634|2485753|2485270|2485545|N|N|N|N|N| +2485636|AAAAAAAAEINOFCAA|2093-05-03|2320|10088|774|2093|0|5|3|2|2093|774|10088|Sunday|2093Q2|N|N|N|2485634|2485753|2485271|2485546|N|N|N|N|N| +2485637|AAAAAAAAFINOFCAA|2093-05-04|2320|10088|774|2093|1|5|4|2|2093|774|10088|Monday|2093Q2|N|N|N|2485634|2485753|2485272|2485547|N|N|N|N|N| +2485638|AAAAAAAAGINOFCAA|2093-05-05|2320|10089|774|2093|2|5|5|2|2093|774|10089|Tuesday|2093Q2|N|N|N|2485634|2485753|2485273|2485548|N|N|N|N|N| +2485639|AAAAAAAAHINOFCAA|2093-05-06|2320|10089|774|2093|3|5|6|2|2093|774|10089|Wednesday|2093Q2|N|N|N|2485634|2485753|2485274|2485549|N|N|N|N|N| +2485640|AAAAAAAAIINOFCAA|2093-05-07|2320|10089|774|2093|4|5|7|2|2093|774|10089|Thursday|2093Q2|N|N|N|2485634|2485753|2485275|2485550|N|N|N|N|N| +2485641|AAAAAAAAJINOFCAA|2093-05-08|2320|10089|774|2093|5|5|8|2|2093|774|10089|Friday|2093Q2|N|Y|N|2485634|2485753|2485276|2485551|N|N|N|N|N| +2485642|AAAAAAAAKINOFCAA|2093-05-09|2320|10089|774|2093|6|5|9|2|2093|774|10089|Saturday|2093Q2|N|Y|N|2485634|2485753|2485277|2485552|N|N|N|N|N| +2485643|AAAAAAAALINOFCAA|2093-05-10|2320|10089|774|2093|0|5|10|2|2093|774|10089|Sunday|2093Q2|N|N|N|2485634|2485753|2485278|2485553|N|N|N|N|N| +2485644|AAAAAAAAMINOFCAA|2093-05-11|2320|10089|774|2093|1|5|11|2|2093|774|10089|Monday|2093Q2|N|N|N|2485634|2485753|2485279|2485554|N|N|N|N|N| +2485645|AAAAAAAANINOFCAA|2093-05-12|2320|10090|774|2093|2|5|12|2|2093|774|10090|Tuesday|2093Q2|N|N|N|2485634|2485753|2485280|2485555|N|N|N|N|N| +2485646|AAAAAAAAOINOFCAA|2093-05-13|2320|10090|774|2093|3|5|13|2|2093|774|10090|Wednesday|2093Q2|N|N|N|2485634|2485753|2485281|2485556|N|N|N|N|N| +2485647|AAAAAAAAPINOFCAA|2093-05-14|2320|10090|774|2093|4|5|14|2|2093|774|10090|Thursday|2093Q2|N|N|N|2485634|2485753|2485282|2485557|N|N|N|N|N| +2485648|AAAAAAAAAJNOFCAA|2093-05-15|2320|10090|774|2093|5|5|15|2|2093|774|10090|Friday|2093Q2|N|Y|N|2485634|2485753|2485283|2485558|N|N|N|N|N| +2485649|AAAAAAAABJNOFCAA|2093-05-16|2320|10090|774|2093|6|5|16|2|2093|774|10090|Saturday|2093Q2|N|Y|N|2485634|2485753|2485284|2485559|N|N|N|N|N| +2485650|AAAAAAAACJNOFCAA|2093-05-17|2320|10090|774|2093|0|5|17|2|2093|774|10090|Sunday|2093Q2|N|N|N|2485634|2485753|2485285|2485560|N|N|N|N|N| +2485651|AAAAAAAADJNOFCAA|2093-05-18|2320|10090|774|2093|1|5|18|2|2093|774|10090|Monday|2093Q2|N|N|N|2485634|2485753|2485286|2485561|N|N|N|N|N| +2485652|AAAAAAAAEJNOFCAA|2093-05-19|2320|10091|774|2093|2|5|19|2|2093|774|10091|Tuesday|2093Q2|N|N|N|2485634|2485753|2485287|2485562|N|N|N|N|N| +2485653|AAAAAAAAFJNOFCAA|2093-05-20|2320|10091|774|2093|3|5|20|2|2093|774|10091|Wednesday|2093Q2|N|N|N|2485634|2485753|2485288|2485563|N|N|N|N|N| +2485654|AAAAAAAAGJNOFCAA|2093-05-21|2320|10091|774|2093|4|5|21|2|2093|774|10091|Thursday|2093Q2|N|N|N|2485634|2485753|2485289|2485564|N|N|N|N|N| +2485655|AAAAAAAAHJNOFCAA|2093-05-22|2320|10091|774|2093|5|5|22|2|2093|774|10091|Friday|2093Q2|N|Y|N|2485634|2485753|2485290|2485565|N|N|N|N|N| +2485656|AAAAAAAAIJNOFCAA|2093-05-23|2320|10091|774|2093|6|5|23|2|2093|774|10091|Saturday|2093Q2|N|Y|N|2485634|2485753|2485291|2485566|N|N|N|N|N| +2485657|AAAAAAAAJJNOFCAA|2093-05-24|2320|10091|774|2093|0|5|24|2|2093|774|10091|Sunday|2093Q2|N|N|N|2485634|2485753|2485292|2485567|N|N|N|N|N| +2485658|AAAAAAAAKJNOFCAA|2093-05-25|2320|10091|774|2093|1|5|25|2|2093|774|10091|Monday|2093Q2|N|N|N|2485634|2485753|2485293|2485568|N|N|N|N|N| +2485659|AAAAAAAALJNOFCAA|2093-05-26|2320|10092|774|2093|2|5|26|2|2093|774|10092|Tuesday|2093Q2|N|N|N|2485634|2485753|2485294|2485569|N|N|N|N|N| +2485660|AAAAAAAAMJNOFCAA|2093-05-27|2320|10092|774|2093|3|5|27|2|2093|774|10092|Wednesday|2093Q2|N|N|N|2485634|2485753|2485295|2485570|N|N|N|N|N| +2485661|AAAAAAAANJNOFCAA|2093-05-28|2320|10092|774|2093|4|5|28|2|2093|774|10092|Thursday|2093Q2|N|N|N|2485634|2485753|2485296|2485571|N|N|N|N|N| +2485662|AAAAAAAAOJNOFCAA|2093-05-29|2320|10092|774|2093|5|5|29|2|2093|774|10092|Friday|2093Q2|N|Y|N|2485634|2485753|2485297|2485572|N|N|N|N|N| +2485663|AAAAAAAAPJNOFCAA|2093-05-30|2320|10092|774|2093|6|5|30|2|2093|774|10092|Saturday|2093Q2|N|Y|N|2485634|2485753|2485298|2485573|N|N|N|N|N| +2485664|AAAAAAAAAKNOFCAA|2093-05-31|2320|10092|774|2093|0|5|31|2|2093|774|10092|Sunday|2093Q2|N|N|N|2485634|2485753|2485299|2485574|N|N|N|N|N| +2485665|AAAAAAAABKNOFCAA|2093-06-01|2321|10092|775|2093|1|6|1|2|2093|775|10092|Monday|2093Q2|N|N|N|2485665|2485815|2485300|2485575|N|N|N|N|N| +2485666|AAAAAAAACKNOFCAA|2093-06-02|2321|10093|775|2093|2|6|2|2|2093|775|10093|Tuesday|2093Q2|N|N|N|2485665|2485815|2485301|2485576|N|N|N|N|N| +2485667|AAAAAAAADKNOFCAA|2093-06-03|2321|10093|775|2093|3|6|3|2|2093|775|10093|Wednesday|2093Q2|N|N|N|2485665|2485815|2485302|2485577|N|N|N|N|N| +2485668|AAAAAAAAEKNOFCAA|2093-06-04|2321|10093|775|2093|4|6|4|2|2093|775|10093|Thursday|2093Q2|N|N|N|2485665|2485815|2485303|2485578|N|N|N|N|N| +2485669|AAAAAAAAFKNOFCAA|2093-06-05|2321|10093|775|2093|5|6|5|2|2093|775|10093|Friday|2093Q2|N|Y|N|2485665|2485815|2485304|2485579|N|N|N|N|N| +2485670|AAAAAAAAGKNOFCAA|2093-06-06|2321|10093|775|2093|6|6|6|2|2093|775|10093|Saturday|2093Q2|N|Y|N|2485665|2485815|2485305|2485580|N|N|N|N|N| +2485671|AAAAAAAAHKNOFCAA|2093-06-07|2321|10093|775|2093|0|6|7|2|2093|775|10093|Sunday|2093Q2|N|N|N|2485665|2485815|2485306|2485581|N|N|N|N|N| +2485672|AAAAAAAAIKNOFCAA|2093-06-08|2321|10093|775|2093|1|6|8|2|2093|775|10093|Monday|2093Q2|N|N|N|2485665|2485815|2485307|2485582|N|N|N|N|N| +2485673|AAAAAAAAJKNOFCAA|2093-06-09|2321|10094|775|2093|2|6|9|2|2093|775|10094|Tuesday|2093Q2|N|N|N|2485665|2485815|2485308|2485583|N|N|N|N|N| +2485674|AAAAAAAAKKNOFCAA|2093-06-10|2321|10094|775|2093|3|6|10|2|2093|775|10094|Wednesday|2093Q2|N|N|N|2485665|2485815|2485309|2485584|N|N|N|N|N| +2485675|AAAAAAAALKNOFCAA|2093-06-11|2321|10094|775|2093|4|6|11|2|2093|775|10094|Thursday|2093Q2|N|N|N|2485665|2485815|2485310|2485585|N|N|N|N|N| +2485676|AAAAAAAAMKNOFCAA|2093-06-12|2321|10094|775|2093|5|6|12|2|2093|775|10094|Friday|2093Q2|N|Y|N|2485665|2485815|2485311|2485586|N|N|N|N|N| +2485677|AAAAAAAANKNOFCAA|2093-06-13|2321|10094|775|2093|6|6|13|2|2093|775|10094|Saturday|2093Q2|N|Y|N|2485665|2485815|2485312|2485587|N|N|N|N|N| +2485678|AAAAAAAAOKNOFCAA|2093-06-14|2321|10094|775|2093|0|6|14|2|2093|775|10094|Sunday|2093Q2|N|N|N|2485665|2485815|2485313|2485588|N|N|N|N|N| +2485679|AAAAAAAAPKNOFCAA|2093-06-15|2321|10094|775|2093|1|6|15|2|2093|775|10094|Monday|2093Q2|N|N|N|2485665|2485815|2485314|2485589|N|N|N|N|N| +2485680|AAAAAAAAALNOFCAA|2093-06-16|2321|10095|775|2093|2|6|16|2|2093|775|10095|Tuesday|2093Q2|N|N|N|2485665|2485815|2485315|2485590|N|N|N|N|N| +2485681|AAAAAAAABLNOFCAA|2093-06-17|2321|10095|775|2093|3|6|17|2|2093|775|10095|Wednesday|2093Q2|N|N|N|2485665|2485815|2485316|2485591|N|N|N|N|N| +2485682|AAAAAAAACLNOFCAA|2093-06-18|2321|10095|775|2093|4|6|18|2|2093|775|10095|Thursday|2093Q2|N|N|N|2485665|2485815|2485317|2485592|N|N|N|N|N| +2485683|AAAAAAAADLNOFCAA|2093-06-19|2321|10095|775|2093|5|6|19|2|2093|775|10095|Friday|2093Q2|N|Y|N|2485665|2485815|2485318|2485593|N|N|N|N|N| +2485684|AAAAAAAAELNOFCAA|2093-06-20|2321|10095|775|2093|6|6|20|2|2093|775|10095|Saturday|2093Q2|N|Y|N|2485665|2485815|2485319|2485594|N|N|N|N|N| +2485685|AAAAAAAAFLNOFCAA|2093-06-21|2321|10095|775|2093|0|6|21|2|2093|775|10095|Sunday|2093Q2|N|N|N|2485665|2485815|2485320|2485595|N|N|N|N|N| +2485686|AAAAAAAAGLNOFCAA|2093-06-22|2321|10095|775|2093|1|6|22|2|2093|775|10095|Monday|2093Q2|N|N|N|2485665|2485815|2485321|2485596|N|N|N|N|N| +2485687|AAAAAAAAHLNOFCAA|2093-06-23|2321|10096|775|2093|2|6|23|2|2093|775|10096|Tuesday|2093Q2|N|N|N|2485665|2485815|2485322|2485597|N|N|N|N|N| +2485688|AAAAAAAAILNOFCAA|2093-06-24|2321|10096|775|2093|3|6|24|2|2093|775|10096|Wednesday|2093Q2|N|N|N|2485665|2485815|2485323|2485598|N|N|N|N|N| +2485689|AAAAAAAAJLNOFCAA|2093-06-25|2321|10096|775|2093|4|6|25|2|2093|775|10096|Thursday|2093Q2|N|N|N|2485665|2485815|2485324|2485599|N|N|N|N|N| +2485690|AAAAAAAAKLNOFCAA|2093-06-26|2321|10096|775|2093|5|6|26|2|2093|775|10096|Friday|2093Q2|N|Y|N|2485665|2485815|2485325|2485600|N|N|N|N|N| +2485691|AAAAAAAALLNOFCAA|2093-06-27|2321|10096|775|2093|6|6|27|2|2093|775|10096|Saturday|2093Q2|N|Y|N|2485665|2485815|2485326|2485601|N|N|N|N|N| +2485692|AAAAAAAAMLNOFCAA|2093-06-28|2321|10096|775|2093|0|6|28|2|2093|775|10096|Sunday|2093Q2|N|N|N|2485665|2485815|2485327|2485602|N|N|N|N|N| +2485693|AAAAAAAANLNOFCAA|2093-06-29|2321|10096|775|2093|1|6|29|2|2093|775|10096|Monday|2093Q2|N|N|N|2485665|2485815|2485328|2485603|N|N|N|N|N| +2485694|AAAAAAAAOLNOFCAA|2093-06-30|2321|10097|775|2093|2|6|30|2|2093|775|10097|Tuesday|2093Q2|N|N|N|2485665|2485815|2485329|2485604|N|N|N|N|N| +2485695|AAAAAAAAPLNOFCAA|2093-07-01|2322|10097|775|2093|3|7|1|2|2093|775|10097|Wednesday|2093Q2|N|N|N|2485695|2485875|2485330|2485604|N|N|N|N|N| +2485696|AAAAAAAAAMNOFCAA|2093-07-02|2322|10097|775|2093|4|7|2|3|2093|775|10097|Thursday|2093Q3|N|N|N|2485695|2485875|2485331|2485605|N|N|N|N|N| +2485697|AAAAAAAABMNOFCAA|2093-07-03|2322|10097|775|2093|5|7|3|3|2093|775|10097|Friday|2093Q3|N|Y|N|2485695|2485875|2485332|2485606|N|N|N|N|N| +2485698|AAAAAAAACMNOFCAA|2093-07-04|2322|10097|775|2093|6|7|4|3|2093|775|10097|Saturday|2093Q3|N|Y|N|2485695|2485875|2485333|2485607|N|N|N|N|N| +2485699|AAAAAAAADMNOFCAA|2093-07-05|2322|10097|775|2093|0|7|5|3|2093|775|10097|Sunday|2093Q3|Y|N|N|2485695|2485875|2485334|2485608|N|N|N|N|N| +2485700|AAAAAAAAEMNOFCAA|2093-07-06|2322|10097|775|2093|1|7|6|3|2093|775|10097|Monday|2093Q3|N|N|Y|2485695|2485875|2485335|2485609|N|N|N|N|N| +2485701|AAAAAAAAFMNOFCAA|2093-07-07|2322|10098|775|2093|2|7|7|3|2093|775|10098|Tuesday|2093Q3|N|N|N|2485695|2485875|2485336|2485610|N|N|N|N|N| +2485702|AAAAAAAAGMNOFCAA|2093-07-08|2322|10098|775|2093|3|7|8|3|2093|775|10098|Wednesday|2093Q3|N|N|N|2485695|2485875|2485337|2485611|N|N|N|N|N| +2485703|AAAAAAAAHMNOFCAA|2093-07-09|2322|10098|775|2093|4|7|9|3|2093|775|10098|Thursday|2093Q3|N|N|N|2485695|2485875|2485338|2485612|N|N|N|N|N| +2485704|AAAAAAAAIMNOFCAA|2093-07-10|2322|10098|775|2093|5|7|10|3|2093|775|10098|Friday|2093Q3|N|Y|N|2485695|2485875|2485339|2485613|N|N|N|N|N| +2485705|AAAAAAAAJMNOFCAA|2093-07-11|2322|10098|775|2093|6|7|11|3|2093|775|10098|Saturday|2093Q3|N|Y|N|2485695|2485875|2485340|2485614|N|N|N|N|N| +2485706|AAAAAAAAKMNOFCAA|2093-07-12|2322|10098|775|2093|0|7|12|3|2093|775|10098|Sunday|2093Q3|N|N|N|2485695|2485875|2485341|2485615|N|N|N|N|N| +2485707|AAAAAAAALMNOFCAA|2093-07-13|2322|10098|775|2093|1|7|13|3|2093|775|10098|Monday|2093Q3|N|N|N|2485695|2485875|2485342|2485616|N|N|N|N|N| +2485708|AAAAAAAAMMNOFCAA|2093-07-14|2322|10099|775|2093|2|7|14|3|2093|775|10099|Tuesday|2093Q3|N|N|N|2485695|2485875|2485343|2485617|N|N|N|N|N| +2485709|AAAAAAAANMNOFCAA|2093-07-15|2322|10099|775|2093|3|7|15|3|2093|775|10099|Wednesday|2093Q3|N|N|N|2485695|2485875|2485344|2485618|N|N|N|N|N| +2485710|AAAAAAAAOMNOFCAA|2093-07-16|2322|10099|775|2093|4|7|16|3|2093|775|10099|Thursday|2093Q3|N|N|N|2485695|2485875|2485345|2485619|N|N|N|N|N| +2485711|AAAAAAAAPMNOFCAA|2093-07-17|2322|10099|775|2093|5|7|17|3|2093|775|10099|Friday|2093Q3|N|Y|N|2485695|2485875|2485346|2485620|N|N|N|N|N| +2485712|AAAAAAAAANNOFCAA|2093-07-18|2322|10099|775|2093|6|7|18|3|2093|775|10099|Saturday|2093Q3|N|Y|N|2485695|2485875|2485347|2485621|N|N|N|N|N| +2485713|AAAAAAAABNNOFCAA|2093-07-19|2322|10099|775|2093|0|7|19|3|2093|775|10099|Sunday|2093Q3|N|N|N|2485695|2485875|2485348|2485622|N|N|N|N|N| +2485714|AAAAAAAACNNOFCAA|2093-07-20|2322|10099|775|2093|1|7|20|3|2093|775|10099|Monday|2093Q3|N|N|N|2485695|2485875|2485349|2485623|N|N|N|N|N| +2485715|AAAAAAAADNNOFCAA|2093-07-21|2322|10100|775|2093|2|7|21|3|2093|775|10100|Tuesday|2093Q3|N|N|N|2485695|2485875|2485350|2485624|N|N|N|N|N| +2485716|AAAAAAAAENNOFCAA|2093-07-22|2322|10100|775|2093|3|7|22|3|2093|775|10100|Wednesday|2093Q3|N|N|N|2485695|2485875|2485351|2485625|N|N|N|N|N| +2485717|AAAAAAAAFNNOFCAA|2093-07-23|2322|10100|775|2093|4|7|23|3|2093|775|10100|Thursday|2093Q3|N|N|N|2485695|2485875|2485352|2485626|N|N|N|N|N| +2485718|AAAAAAAAGNNOFCAA|2093-07-24|2322|10100|775|2093|5|7|24|3|2093|775|10100|Friday|2093Q3|N|Y|N|2485695|2485875|2485353|2485627|N|N|N|N|N| +2485719|AAAAAAAAHNNOFCAA|2093-07-25|2322|10100|775|2093|6|7|25|3|2093|775|10100|Saturday|2093Q3|N|Y|N|2485695|2485875|2485354|2485628|N|N|N|N|N| +2485720|AAAAAAAAINNOFCAA|2093-07-26|2322|10100|775|2093|0|7|26|3|2093|775|10100|Sunday|2093Q3|N|N|N|2485695|2485875|2485355|2485629|N|N|N|N|N| +2485721|AAAAAAAAJNNOFCAA|2093-07-27|2322|10100|775|2093|1|7|27|3|2093|775|10100|Monday|2093Q3|N|N|N|2485695|2485875|2485356|2485630|N|N|N|N|N| +2485722|AAAAAAAAKNNOFCAA|2093-07-28|2322|10101|775|2093|2|7|28|3|2093|775|10101|Tuesday|2093Q3|N|N|N|2485695|2485875|2485357|2485631|N|N|N|N|N| +2485723|AAAAAAAALNNOFCAA|2093-07-29|2322|10101|775|2093|3|7|29|3|2093|775|10101|Wednesday|2093Q3|N|N|N|2485695|2485875|2485358|2485632|N|N|N|N|N| +2485724|AAAAAAAAMNNOFCAA|2093-07-30|2322|10101|775|2093|4|7|30|3|2093|775|10101|Thursday|2093Q3|N|N|N|2485695|2485875|2485359|2485633|N|N|N|N|N| +2485725|AAAAAAAANNNOFCAA|2093-07-31|2322|10101|775|2093|5|7|31|3|2093|775|10101|Friday|2093Q3|N|Y|N|2485695|2485875|2485360|2485634|N|N|N|N|N| +2485726|AAAAAAAAONNOFCAA|2093-08-01|2323|10101|775|2093|6|8|1|3|2093|775|10101|Saturday|2093Q3|N|Y|N|2485726|2485937|2485361|2485635|N|N|N|N|N| +2485727|AAAAAAAAPNNOFCAA|2093-08-02|2323|10101|775|2093|0|8|2|3|2093|775|10101|Sunday|2093Q3|N|N|N|2485726|2485937|2485362|2485636|N|N|N|N|N| +2485728|AAAAAAAAAONOFCAA|2093-08-03|2323|10101|775|2093|1|8|3|3|2093|775|10101|Monday|2093Q3|N|N|N|2485726|2485937|2485363|2485637|N|N|N|N|N| +2485729|AAAAAAAABONOFCAA|2093-08-04|2323|10102|775|2093|2|8|4|3|2093|775|10102|Tuesday|2093Q3|N|N|N|2485726|2485937|2485364|2485638|N|N|N|N|N| +2485730|AAAAAAAACONOFCAA|2093-08-05|2323|10102|775|2093|3|8|5|3|2093|775|10102|Wednesday|2093Q3|N|N|N|2485726|2485937|2485365|2485639|N|N|N|N|N| +2485731|AAAAAAAADONOFCAA|2093-08-06|2323|10102|775|2093|4|8|6|3|2093|775|10102|Thursday|2093Q3|N|N|N|2485726|2485937|2485366|2485640|N|N|N|N|N| +2485732|AAAAAAAAEONOFCAA|2093-08-07|2323|10102|775|2093|5|8|7|3|2093|775|10102|Friday|2093Q3|N|Y|N|2485726|2485937|2485367|2485641|N|N|N|N|N| +2485733|AAAAAAAAFONOFCAA|2093-08-08|2323|10102|775|2093|6|8|8|3|2093|775|10102|Saturday|2093Q3|N|Y|N|2485726|2485937|2485368|2485642|N|N|N|N|N| +2485734|AAAAAAAAGONOFCAA|2093-08-09|2323|10102|775|2093|0|8|9|3|2093|775|10102|Sunday|2093Q3|N|N|N|2485726|2485937|2485369|2485643|N|N|N|N|N| +2485735|AAAAAAAAHONOFCAA|2093-08-10|2323|10102|775|2093|1|8|10|3|2093|775|10102|Monday|2093Q3|N|N|N|2485726|2485937|2485370|2485644|N|N|N|N|N| +2485736|AAAAAAAAIONOFCAA|2093-08-11|2323|10103|775|2093|2|8|11|3|2093|775|10103|Tuesday|2093Q3|N|N|N|2485726|2485937|2485371|2485645|N|N|N|N|N| +2485737|AAAAAAAAJONOFCAA|2093-08-12|2323|10103|775|2093|3|8|12|3|2093|775|10103|Wednesday|2093Q3|N|N|N|2485726|2485937|2485372|2485646|N|N|N|N|N| +2485738|AAAAAAAAKONOFCAA|2093-08-13|2323|10103|775|2093|4|8|13|3|2093|775|10103|Thursday|2093Q3|N|N|N|2485726|2485937|2485373|2485647|N|N|N|N|N| +2485739|AAAAAAAALONOFCAA|2093-08-14|2323|10103|775|2093|5|8|14|3|2093|775|10103|Friday|2093Q3|N|Y|N|2485726|2485937|2485374|2485648|N|N|N|N|N| +2485740|AAAAAAAAMONOFCAA|2093-08-15|2323|10103|775|2093|6|8|15|3|2093|775|10103|Saturday|2093Q3|N|Y|N|2485726|2485937|2485375|2485649|N|N|N|N|N| +2485741|AAAAAAAANONOFCAA|2093-08-16|2323|10103|775|2093|0|8|16|3|2093|775|10103|Sunday|2093Q3|N|N|N|2485726|2485937|2485376|2485650|N|N|N|N|N| +2485742|AAAAAAAAOONOFCAA|2093-08-17|2323|10103|775|2093|1|8|17|3|2093|775|10103|Monday|2093Q3|N|N|N|2485726|2485937|2485377|2485651|N|N|N|N|N| +2485743|AAAAAAAAPONOFCAA|2093-08-18|2323|10104|775|2093|2|8|18|3|2093|775|10104|Tuesday|2093Q3|N|N|N|2485726|2485937|2485378|2485652|N|N|N|N|N| +2485744|AAAAAAAAAPNOFCAA|2093-08-19|2323|10104|775|2093|3|8|19|3|2093|775|10104|Wednesday|2093Q3|N|N|N|2485726|2485937|2485379|2485653|N|N|N|N|N| +2485745|AAAAAAAABPNOFCAA|2093-08-20|2323|10104|775|2093|4|8|20|3|2093|775|10104|Thursday|2093Q3|N|N|N|2485726|2485937|2485380|2485654|N|N|N|N|N| +2485746|AAAAAAAACPNOFCAA|2093-08-21|2323|10104|775|2093|5|8|21|3|2093|775|10104|Friday|2093Q3|N|Y|N|2485726|2485937|2485381|2485655|N|N|N|N|N| +2485747|AAAAAAAADPNOFCAA|2093-08-22|2323|10104|775|2093|6|8|22|3|2093|775|10104|Saturday|2093Q3|N|Y|N|2485726|2485937|2485382|2485656|N|N|N|N|N| +2485748|AAAAAAAAEPNOFCAA|2093-08-23|2323|10104|775|2093|0|8|23|3|2093|775|10104|Sunday|2093Q3|N|N|N|2485726|2485937|2485383|2485657|N|N|N|N|N| +2485749|AAAAAAAAFPNOFCAA|2093-08-24|2323|10104|775|2093|1|8|24|3|2093|775|10104|Monday|2093Q3|N|N|N|2485726|2485937|2485384|2485658|N|N|N|N|N| +2485750|AAAAAAAAGPNOFCAA|2093-08-25|2323|10105|775|2093|2|8|25|3|2093|775|10105|Tuesday|2093Q3|N|N|N|2485726|2485937|2485385|2485659|N|N|N|N|N| +2485751|AAAAAAAAHPNOFCAA|2093-08-26|2323|10105|775|2093|3|8|26|3|2093|775|10105|Wednesday|2093Q3|N|N|N|2485726|2485937|2485386|2485660|N|N|N|N|N| +2485752|AAAAAAAAIPNOFCAA|2093-08-27|2323|10105|775|2093|4|8|27|3|2093|775|10105|Thursday|2093Q3|N|N|N|2485726|2485937|2485387|2485661|N|N|N|N|N| +2485753|AAAAAAAAJPNOFCAA|2093-08-28|2323|10105|775|2093|5|8|28|3|2093|775|10105|Friday|2093Q3|N|Y|N|2485726|2485937|2485388|2485662|N|N|N|N|N| +2485754|AAAAAAAAKPNOFCAA|2093-08-29|2323|10105|775|2093|6|8|29|3|2093|775|10105|Saturday|2093Q3|N|Y|N|2485726|2485937|2485389|2485663|N|N|N|N|N| +2485755|AAAAAAAALPNOFCAA|2093-08-30|2323|10105|775|2093|0|8|30|3|2093|775|10105|Sunday|2093Q3|N|N|N|2485726|2485937|2485390|2485664|N|N|N|N|N| +2485756|AAAAAAAAMPNOFCAA|2093-08-31|2323|10105|775|2093|1|8|31|3|2093|775|10105|Monday|2093Q3|N|N|N|2485726|2485937|2485391|2485665|N|N|N|N|N| +2485757|AAAAAAAANPNOFCAA|2093-09-01|2324|10106|776|2093|2|9|1|3|2093|776|10106|Tuesday|2093Q3|N|N|N|2485757|2485999|2485392|2485666|N|N|N|N|N| +2485758|AAAAAAAAOPNOFCAA|2093-09-02|2324|10106|776|2093|3|9|2|3|2093|776|10106|Wednesday|2093Q3|N|N|N|2485757|2485999|2485393|2485667|N|N|N|N|N| +2485759|AAAAAAAAPPNOFCAA|2093-09-03|2324|10106|776|2093|4|9|3|3|2093|776|10106|Thursday|2093Q3|N|N|N|2485757|2485999|2485394|2485668|N|N|N|N|N| +2485760|AAAAAAAAAAOOFCAA|2093-09-04|2324|10106|776|2093|5|9|4|3|2093|776|10106|Friday|2093Q3|N|Y|N|2485757|2485999|2485395|2485669|N|N|N|N|N| +2485761|AAAAAAAABAOOFCAA|2093-09-05|2324|10106|776|2093|6|9|5|3|2093|776|10106|Saturday|2093Q3|N|Y|N|2485757|2485999|2485396|2485670|N|N|N|N|N| +2485762|AAAAAAAACAOOFCAA|2093-09-06|2324|10106|776|2093|0|9|6|3|2093|776|10106|Sunday|2093Q3|N|N|N|2485757|2485999|2485397|2485671|N|N|N|N|N| +2485763|AAAAAAAADAOOFCAA|2093-09-07|2324|10106|776|2093|1|9|7|3|2093|776|10106|Monday|2093Q3|N|N|N|2485757|2485999|2485398|2485672|N|N|N|N|N| +2485764|AAAAAAAAEAOOFCAA|2093-09-08|2324|10107|776|2093|2|9|8|3|2093|776|10107|Tuesday|2093Q3|N|N|N|2485757|2485999|2485399|2485673|N|N|N|N|N| +2485765|AAAAAAAAFAOOFCAA|2093-09-09|2324|10107|776|2093|3|9|9|3|2093|776|10107|Wednesday|2093Q3|N|N|N|2485757|2485999|2485400|2485674|N|N|N|N|N| +2485766|AAAAAAAAGAOOFCAA|2093-09-10|2324|10107|776|2093|4|9|10|3|2093|776|10107|Thursday|2093Q3|N|N|N|2485757|2485999|2485401|2485675|N|N|N|N|N| +2485767|AAAAAAAAHAOOFCAA|2093-09-11|2324|10107|776|2093|5|9|11|3|2093|776|10107|Friday|2093Q3|N|Y|N|2485757|2485999|2485402|2485676|N|N|N|N|N| +2485768|AAAAAAAAIAOOFCAA|2093-09-12|2324|10107|776|2093|6|9|12|3|2093|776|10107|Saturday|2093Q3|N|Y|N|2485757|2485999|2485403|2485677|N|N|N|N|N| +2485769|AAAAAAAAJAOOFCAA|2093-09-13|2324|10107|776|2093|0|9|13|3|2093|776|10107|Sunday|2093Q3|N|N|N|2485757|2485999|2485404|2485678|N|N|N|N|N| +2485770|AAAAAAAAKAOOFCAA|2093-09-14|2324|10107|776|2093|1|9|14|3|2093|776|10107|Monday|2093Q3|N|N|N|2485757|2485999|2485405|2485679|N|N|N|N|N| +2485771|AAAAAAAALAOOFCAA|2093-09-15|2324|10108|776|2093|2|9|15|3|2093|776|10108|Tuesday|2093Q3|N|N|N|2485757|2485999|2485406|2485680|N|N|N|N|N| +2485772|AAAAAAAAMAOOFCAA|2093-09-16|2324|10108|776|2093|3|9|16|3|2093|776|10108|Wednesday|2093Q3|N|N|N|2485757|2485999|2485407|2485681|N|N|N|N|N| +2485773|AAAAAAAANAOOFCAA|2093-09-17|2324|10108|776|2093|4|9|17|3|2093|776|10108|Thursday|2093Q3|N|N|N|2485757|2485999|2485408|2485682|N|N|N|N|N| +2485774|AAAAAAAAOAOOFCAA|2093-09-18|2324|10108|776|2093|5|9|18|3|2093|776|10108|Friday|2093Q3|N|Y|N|2485757|2485999|2485409|2485683|N|N|N|N|N| +2485775|AAAAAAAAPAOOFCAA|2093-09-19|2324|10108|776|2093|6|9|19|3|2093|776|10108|Saturday|2093Q3|N|Y|N|2485757|2485999|2485410|2485684|N|N|N|N|N| +2485776|AAAAAAAAABOOFCAA|2093-09-20|2324|10108|776|2093|0|9|20|3|2093|776|10108|Sunday|2093Q3|N|N|N|2485757|2485999|2485411|2485685|N|N|N|N|N| +2485777|AAAAAAAABBOOFCAA|2093-09-21|2324|10108|776|2093|1|9|21|3|2093|776|10108|Monday|2093Q3|N|N|N|2485757|2485999|2485412|2485686|N|N|N|N|N| +2485778|AAAAAAAACBOOFCAA|2093-09-22|2324|10109|776|2093|2|9|22|3|2093|776|10109|Tuesday|2093Q3|N|N|N|2485757|2485999|2485413|2485687|N|N|N|N|N| +2485779|AAAAAAAADBOOFCAA|2093-09-23|2324|10109|776|2093|3|9|23|3|2093|776|10109|Wednesday|2093Q3|N|N|N|2485757|2485999|2485414|2485688|N|N|N|N|N| +2485780|AAAAAAAAEBOOFCAA|2093-09-24|2324|10109|776|2093|4|9|24|3|2093|776|10109|Thursday|2093Q3|N|N|N|2485757|2485999|2485415|2485689|N|N|N|N|N| +2485781|AAAAAAAAFBOOFCAA|2093-09-25|2324|10109|776|2093|5|9|25|3|2093|776|10109|Friday|2093Q3|N|Y|N|2485757|2485999|2485416|2485690|N|N|N|N|N| +2485782|AAAAAAAAGBOOFCAA|2093-09-26|2324|10109|776|2093|6|9|26|3|2093|776|10109|Saturday|2093Q3|N|Y|N|2485757|2485999|2485417|2485691|N|N|N|N|N| +2485783|AAAAAAAAHBOOFCAA|2093-09-27|2324|10109|776|2093|0|9|27|3|2093|776|10109|Sunday|2093Q3|N|N|N|2485757|2485999|2485418|2485692|N|N|N|N|N| +2485784|AAAAAAAAIBOOFCAA|2093-09-28|2324|10109|776|2093|1|9|28|3|2093|776|10109|Monday|2093Q3|N|N|N|2485757|2485999|2485419|2485693|N|N|N|N|N| +2485785|AAAAAAAAJBOOFCAA|2093-09-29|2324|10110|776|2093|2|9|29|3|2093|776|10110|Tuesday|2093Q3|N|N|N|2485757|2485999|2485420|2485694|N|N|N|N|N| +2485786|AAAAAAAAKBOOFCAA|2093-09-30|2324|10110|776|2093|3|9|30|3|2093|776|10110|Wednesday|2093Q3|N|N|N|2485757|2485999|2485421|2485695|N|N|N|N|N| +2485787|AAAAAAAALBOOFCAA|2093-10-01|2325|10110|776|2093|4|10|1|3|2093|776|10110|Thursday|2093Q3|N|N|N|2485787|2486059|2485422|2485695|N|N|N|N|N| +2485788|AAAAAAAAMBOOFCAA|2093-10-02|2325|10110|776|2093|5|10|2|4|2093|776|10110|Friday|2093Q4|N|Y|N|2485787|2486059|2485423|2485696|N|N|N|N|N| +2485789|AAAAAAAANBOOFCAA|2093-10-03|2325|10110|776|2093|6|10|3|4|2093|776|10110|Saturday|2093Q4|N|Y|N|2485787|2486059|2485424|2485697|N|N|N|N|N| +2485790|AAAAAAAAOBOOFCAA|2093-10-04|2325|10110|776|2093|0|10|4|4|2093|776|10110|Sunday|2093Q4|N|N|N|2485787|2486059|2485425|2485698|N|N|N|N|N| +2485791|AAAAAAAAPBOOFCAA|2093-10-05|2325|10110|776|2093|1|10|5|4|2093|776|10110|Monday|2093Q4|N|N|N|2485787|2486059|2485426|2485699|N|N|N|N|N| +2485792|AAAAAAAAACOOFCAA|2093-10-06|2325|10111|776|2093|2|10|6|4|2093|776|10111|Tuesday|2093Q4|N|N|N|2485787|2486059|2485427|2485700|N|N|N|N|N| +2485793|AAAAAAAABCOOFCAA|2093-10-07|2325|10111|776|2093|3|10|7|4|2093|776|10111|Wednesday|2093Q4|N|N|N|2485787|2486059|2485428|2485701|N|N|N|N|N| +2485794|AAAAAAAACCOOFCAA|2093-10-08|2325|10111|776|2093|4|10|8|4|2093|776|10111|Thursday|2093Q4|N|N|N|2485787|2486059|2485429|2485702|N|N|N|N|N| +2485795|AAAAAAAADCOOFCAA|2093-10-09|2325|10111|776|2093|5|10|9|4|2093|776|10111|Friday|2093Q4|N|Y|N|2485787|2486059|2485430|2485703|N|N|N|N|N| +2485796|AAAAAAAAECOOFCAA|2093-10-10|2325|10111|776|2093|6|10|10|4|2093|776|10111|Saturday|2093Q4|N|Y|N|2485787|2486059|2485431|2485704|N|N|N|N|N| +2485797|AAAAAAAAFCOOFCAA|2093-10-11|2325|10111|776|2093|0|10|11|4|2093|776|10111|Sunday|2093Q4|N|N|N|2485787|2486059|2485432|2485705|N|N|N|N|N| +2485798|AAAAAAAAGCOOFCAA|2093-10-12|2325|10111|776|2093|1|10|12|4|2093|776|10111|Monday|2093Q4|N|N|N|2485787|2486059|2485433|2485706|N|N|N|N|N| +2485799|AAAAAAAAHCOOFCAA|2093-10-13|2325|10112|776|2093|2|10|13|4|2093|776|10112|Tuesday|2093Q4|N|N|N|2485787|2486059|2485434|2485707|N|N|N|N|N| +2485800|AAAAAAAAICOOFCAA|2093-10-14|2325|10112|776|2093|3|10|14|4|2093|776|10112|Wednesday|2093Q4|N|N|N|2485787|2486059|2485435|2485708|N|N|N|N|N| +2485801|AAAAAAAAJCOOFCAA|2093-10-15|2325|10112|776|2093|4|10|15|4|2093|776|10112|Thursday|2093Q4|N|N|N|2485787|2486059|2485436|2485709|N|N|N|N|N| +2485802|AAAAAAAAKCOOFCAA|2093-10-16|2325|10112|776|2093|5|10|16|4|2093|776|10112|Friday|2093Q4|N|Y|N|2485787|2486059|2485437|2485710|N|N|N|N|N| +2485803|AAAAAAAALCOOFCAA|2093-10-17|2325|10112|776|2093|6|10|17|4|2093|776|10112|Saturday|2093Q4|N|Y|N|2485787|2486059|2485438|2485711|N|N|N|N|N| +2485804|AAAAAAAAMCOOFCAA|2093-10-18|2325|10112|776|2093|0|10|18|4|2093|776|10112|Sunday|2093Q4|N|N|N|2485787|2486059|2485439|2485712|N|N|N|N|N| +2485805|AAAAAAAANCOOFCAA|2093-10-19|2325|10112|776|2093|1|10|19|4|2093|776|10112|Monday|2093Q4|N|N|N|2485787|2486059|2485440|2485713|N|N|N|N|N| +2485806|AAAAAAAAOCOOFCAA|2093-10-20|2325|10113|776|2093|2|10|20|4|2093|776|10113|Tuesday|2093Q4|N|N|N|2485787|2486059|2485441|2485714|N|N|N|N|N| +2485807|AAAAAAAAPCOOFCAA|2093-10-21|2325|10113|776|2093|3|10|21|4|2093|776|10113|Wednesday|2093Q4|N|N|N|2485787|2486059|2485442|2485715|N|N|N|N|N| +2485808|AAAAAAAAADOOFCAA|2093-10-22|2325|10113|776|2093|4|10|22|4|2093|776|10113|Thursday|2093Q4|N|N|N|2485787|2486059|2485443|2485716|N|N|N|N|N| +2485809|AAAAAAAABDOOFCAA|2093-10-23|2325|10113|776|2093|5|10|23|4|2093|776|10113|Friday|2093Q4|N|Y|N|2485787|2486059|2485444|2485717|N|N|N|N|N| +2485810|AAAAAAAACDOOFCAA|2093-10-24|2325|10113|776|2093|6|10|24|4|2093|776|10113|Saturday|2093Q4|N|Y|N|2485787|2486059|2485445|2485718|N|N|N|N|N| +2485811|AAAAAAAADDOOFCAA|2093-10-25|2325|10113|776|2093|0|10|25|4|2093|776|10113|Sunday|2093Q4|N|N|N|2485787|2486059|2485446|2485719|N|N|N|N|N| +2485812|AAAAAAAAEDOOFCAA|2093-10-26|2325|10113|776|2093|1|10|26|4|2093|776|10113|Monday|2093Q4|N|N|N|2485787|2486059|2485447|2485720|N|N|N|N|N| +2485813|AAAAAAAAFDOOFCAA|2093-10-27|2325|10114|776|2093|2|10|27|4|2093|776|10114|Tuesday|2093Q4|N|N|N|2485787|2486059|2485448|2485721|N|N|N|N|N| +2485814|AAAAAAAAGDOOFCAA|2093-10-28|2325|10114|776|2093|3|10|28|4|2093|776|10114|Wednesday|2093Q4|N|N|N|2485787|2486059|2485449|2485722|N|N|N|N|N| +2485815|AAAAAAAAHDOOFCAA|2093-10-29|2325|10114|776|2093|4|10|29|4|2093|776|10114|Thursday|2093Q4|N|N|N|2485787|2486059|2485450|2485723|N|N|N|N|N| +2485816|AAAAAAAAIDOOFCAA|2093-10-30|2325|10114|776|2093|5|10|30|4|2093|776|10114|Friday|2093Q4|N|Y|N|2485787|2486059|2485451|2485724|N|N|N|N|N| +2485817|AAAAAAAAJDOOFCAA|2093-10-31|2325|10114|776|2093|6|10|31|4|2093|776|10114|Saturday|2093Q4|N|Y|N|2485787|2486059|2485452|2485725|N|N|N|N|N| +2485818|AAAAAAAAKDOOFCAA|2093-11-01|2326|10114|776|2093|0|11|1|4|2093|776|10114|Sunday|2093Q4|N|N|N|2485818|2486121|2485453|2485726|N|N|N|N|N| +2485819|AAAAAAAALDOOFCAA|2093-11-02|2326|10114|776|2093|1|11|2|4|2093|776|10114|Monday|2093Q4|N|N|N|2485818|2486121|2485454|2485727|N|N|N|N|N| +2485820|AAAAAAAAMDOOFCAA|2093-11-03|2326|10115|776|2093|2|11|3|4|2093|776|10115|Tuesday|2093Q4|N|N|N|2485818|2486121|2485455|2485728|N|N|N|N|N| +2485821|AAAAAAAANDOOFCAA|2093-11-04|2326|10115|776|2093|3|11|4|4|2093|776|10115|Wednesday|2093Q4|N|N|N|2485818|2486121|2485456|2485729|N|N|N|N|N| +2485822|AAAAAAAAODOOFCAA|2093-11-05|2326|10115|776|2093|4|11|5|4|2093|776|10115|Thursday|2093Q4|N|N|N|2485818|2486121|2485457|2485730|N|N|N|N|N| +2485823|AAAAAAAAPDOOFCAA|2093-11-06|2326|10115|776|2093|5|11|6|4|2093|776|10115|Friday|2093Q4|N|Y|N|2485818|2486121|2485458|2485731|N|N|N|N|N| +2485824|AAAAAAAAAEOOFCAA|2093-11-07|2326|10115|776|2093|6|11|7|4|2093|776|10115|Saturday|2093Q4|N|Y|N|2485818|2486121|2485459|2485732|N|N|N|N|N| +2485825|AAAAAAAABEOOFCAA|2093-11-08|2326|10115|776|2093|0|11|8|4|2093|776|10115|Sunday|2093Q4|N|N|N|2485818|2486121|2485460|2485733|N|N|N|N|N| +2485826|AAAAAAAACEOOFCAA|2093-11-09|2326|10115|776|2093|1|11|9|4|2093|776|10115|Monday|2093Q4|N|N|N|2485818|2486121|2485461|2485734|N|N|N|N|N| +2485827|AAAAAAAADEOOFCAA|2093-11-10|2326|10116|776|2093|2|11|10|4|2093|776|10116|Tuesday|2093Q4|N|N|N|2485818|2486121|2485462|2485735|N|N|N|N|N| +2485828|AAAAAAAAEEOOFCAA|2093-11-11|2326|10116|776|2093|3|11|11|4|2093|776|10116|Wednesday|2093Q4|N|N|N|2485818|2486121|2485463|2485736|N|N|N|N|N| +2485829|AAAAAAAAFEOOFCAA|2093-11-12|2326|10116|776|2093|4|11|12|4|2093|776|10116|Thursday|2093Q4|N|N|N|2485818|2486121|2485464|2485737|N|N|N|N|N| +2485830|AAAAAAAAGEOOFCAA|2093-11-13|2326|10116|776|2093|5|11|13|4|2093|776|10116|Friday|2093Q4|N|Y|N|2485818|2486121|2485465|2485738|N|N|N|N|N| +2485831|AAAAAAAAHEOOFCAA|2093-11-14|2326|10116|776|2093|6|11|14|4|2093|776|10116|Saturday|2093Q4|N|Y|N|2485818|2486121|2485466|2485739|N|N|N|N|N| +2485832|AAAAAAAAIEOOFCAA|2093-11-15|2326|10116|776|2093|0|11|15|4|2093|776|10116|Sunday|2093Q4|N|N|N|2485818|2486121|2485467|2485740|N|N|N|N|N| +2485833|AAAAAAAAJEOOFCAA|2093-11-16|2326|10116|776|2093|1|11|16|4|2093|776|10116|Monday|2093Q4|N|N|N|2485818|2486121|2485468|2485741|N|N|N|N|N| +2485834|AAAAAAAAKEOOFCAA|2093-11-17|2326|10117|776|2093|2|11|17|4|2093|776|10117|Tuesday|2093Q4|N|N|N|2485818|2486121|2485469|2485742|N|N|N|N|N| +2485835|AAAAAAAALEOOFCAA|2093-11-18|2326|10117|776|2093|3|11|18|4|2093|776|10117|Wednesday|2093Q4|N|N|N|2485818|2486121|2485470|2485743|N|N|N|N|N| +2485836|AAAAAAAAMEOOFCAA|2093-11-19|2326|10117|776|2093|4|11|19|4|2093|776|10117|Thursday|2093Q4|N|N|N|2485818|2486121|2485471|2485744|N|N|N|N|N| +2485837|AAAAAAAANEOOFCAA|2093-11-20|2326|10117|776|2093|5|11|20|4|2093|776|10117|Friday|2093Q4|N|Y|N|2485818|2486121|2485472|2485745|N|N|N|N|N| +2485838|AAAAAAAAOEOOFCAA|2093-11-21|2326|10117|776|2093|6|11|21|4|2093|776|10117|Saturday|2093Q4|N|Y|N|2485818|2486121|2485473|2485746|N|N|N|N|N| +2485839|AAAAAAAAPEOOFCAA|2093-11-22|2326|10117|776|2093|0|11|22|4|2093|776|10117|Sunday|2093Q4|N|N|N|2485818|2486121|2485474|2485747|N|N|N|N|N| +2485840|AAAAAAAAAFOOFCAA|2093-11-23|2326|10117|776|2093|1|11|23|4|2093|776|10117|Monday|2093Q4|N|N|N|2485818|2486121|2485475|2485748|N|N|N|N|N| +2485841|AAAAAAAABFOOFCAA|2093-11-24|2326|10118|776|2093|2|11|24|4|2093|776|10118|Tuesday|2093Q4|N|N|N|2485818|2486121|2485476|2485749|N|N|N|N|N| +2485842|AAAAAAAACFOOFCAA|2093-11-25|2326|10118|776|2093|3|11|25|4|2093|776|10118|Wednesday|2093Q4|N|N|N|2485818|2486121|2485477|2485750|N|N|N|N|N| +2485843|AAAAAAAADFOOFCAA|2093-11-26|2326|10118|776|2093|4|11|26|4|2093|776|10118|Thursday|2093Q4|N|N|N|2485818|2486121|2485478|2485751|N|N|N|N|N| +2485844|AAAAAAAAEFOOFCAA|2093-11-27|2326|10118|776|2093|5|11|27|4|2093|776|10118|Friday|2093Q4|N|Y|N|2485818|2486121|2485479|2485752|N|N|N|N|N| +2485845|AAAAAAAAFFOOFCAA|2093-11-28|2326|10118|776|2093|6|11|28|4|2093|776|10118|Saturday|2093Q4|N|Y|N|2485818|2486121|2485480|2485753|N|N|N|N|N| +2485846|AAAAAAAAGFOOFCAA|2093-11-29|2326|10118|776|2093|0|11|29|4|2093|776|10118|Sunday|2093Q4|N|N|N|2485818|2486121|2485481|2485754|N|N|N|N|N| +2485847|AAAAAAAAHFOOFCAA|2093-11-30|2326|10118|776|2093|1|11|30|4|2093|776|10118|Monday|2093Q4|N|N|N|2485818|2486121|2485482|2485755|N|N|N|N|N| +2485848|AAAAAAAAIFOOFCAA|2093-12-01|2327|10119|777|2093|2|12|1|4|2093|777|10119|Tuesday|2093Q4|N|N|N|2485848|2486181|2485483|2485756|N|N|N|N|N| +2485849|AAAAAAAAJFOOFCAA|2093-12-02|2327|10119|777|2093|3|12|2|4|2093|777|10119|Wednesday|2093Q4|N|N|N|2485848|2486181|2485484|2485757|N|N|N|N|N| +2485850|AAAAAAAAKFOOFCAA|2093-12-03|2327|10119|777|2093|4|12|3|4|2093|777|10119|Thursday|2093Q4|N|N|N|2485848|2486181|2485485|2485758|N|N|N|N|N| +2485851|AAAAAAAALFOOFCAA|2093-12-04|2327|10119|777|2093|5|12|4|4|2093|777|10119|Friday|2093Q4|N|Y|N|2485848|2486181|2485486|2485759|N|N|N|N|N| +2485852|AAAAAAAAMFOOFCAA|2093-12-05|2327|10119|777|2093|6|12|5|4|2093|777|10119|Saturday|2093Q4|N|Y|N|2485848|2486181|2485487|2485760|N|N|N|N|N| +2485853|AAAAAAAANFOOFCAA|2093-12-06|2327|10119|777|2093|0|12|6|4|2093|777|10119|Sunday|2093Q4|N|N|N|2485848|2486181|2485488|2485761|N|N|N|N|N| +2485854|AAAAAAAAOFOOFCAA|2093-12-07|2327|10119|777|2093|1|12|7|4|2093|777|10119|Monday|2093Q4|N|N|N|2485848|2486181|2485489|2485762|N|N|N|N|N| +2485855|AAAAAAAAPFOOFCAA|2093-12-08|2327|10120|777|2093|2|12|8|4|2093|777|10120|Tuesday|2093Q4|N|N|N|2485848|2486181|2485490|2485763|N|N|N|N|N| +2485856|AAAAAAAAAGOOFCAA|2093-12-09|2327|10120|777|2093|3|12|9|4|2093|777|10120|Wednesday|2093Q4|N|N|N|2485848|2486181|2485491|2485764|N|N|N|N|N| +2485857|AAAAAAAABGOOFCAA|2093-12-10|2327|10120|777|2093|4|12|10|4|2093|777|10120|Thursday|2093Q4|N|N|N|2485848|2486181|2485492|2485765|N|N|N|N|N| +2485858|AAAAAAAACGOOFCAA|2093-12-11|2327|10120|777|2093|5|12|11|4|2093|777|10120|Friday|2093Q4|N|Y|N|2485848|2486181|2485493|2485766|N|N|N|N|N| +2485859|AAAAAAAADGOOFCAA|2093-12-12|2327|10120|777|2093|6|12|12|4|2093|777|10120|Saturday|2093Q4|N|Y|N|2485848|2486181|2485494|2485767|N|N|N|N|N| +2485860|AAAAAAAAEGOOFCAA|2093-12-13|2327|10120|777|2093|0|12|13|4|2093|777|10120|Sunday|2093Q4|N|N|N|2485848|2486181|2485495|2485768|N|N|N|N|N| +2485861|AAAAAAAAFGOOFCAA|2093-12-14|2327|10120|777|2093|1|12|14|4|2093|777|10120|Monday|2093Q4|N|N|N|2485848|2486181|2485496|2485769|N|N|N|N|N| +2485862|AAAAAAAAGGOOFCAA|2093-12-15|2327|10121|777|2093|2|12|15|4|2093|777|10121|Tuesday|2093Q4|N|N|N|2485848|2486181|2485497|2485770|N|N|N|N|N| +2485863|AAAAAAAAHGOOFCAA|2093-12-16|2327|10121|777|2093|3|12|16|4|2093|777|10121|Wednesday|2093Q4|N|N|N|2485848|2486181|2485498|2485771|N|N|N|N|N| +2485864|AAAAAAAAIGOOFCAA|2093-12-17|2327|10121|777|2093|4|12|17|4|2093|777|10121|Thursday|2093Q4|N|N|N|2485848|2486181|2485499|2485772|N|N|N|N|N| +2485865|AAAAAAAAJGOOFCAA|2093-12-18|2327|10121|777|2093|5|12|18|4|2093|777|10121|Friday|2093Q4|N|Y|N|2485848|2486181|2485500|2485773|N|N|N|N|N| +2485866|AAAAAAAAKGOOFCAA|2093-12-19|2327|10121|777|2093|6|12|19|4|2093|777|10121|Saturday|2093Q4|N|Y|N|2485848|2486181|2485501|2485774|N|N|N|N|N| +2485867|AAAAAAAALGOOFCAA|2093-12-20|2327|10121|777|2093|0|12|20|4|2093|777|10121|Sunday|2093Q4|N|N|N|2485848|2486181|2485502|2485775|N|N|N|N|N| +2485868|AAAAAAAAMGOOFCAA|2093-12-21|2327|10121|777|2093|1|12|21|4|2093|777|10121|Monday|2093Q4|N|N|N|2485848|2486181|2485503|2485776|N|N|N|N|N| +2485869|AAAAAAAANGOOFCAA|2093-12-22|2327|10122|777|2093|2|12|22|4|2093|777|10122|Tuesday|2093Q4|N|N|N|2485848|2486181|2485504|2485777|N|N|N|N|N| +2485870|AAAAAAAAOGOOFCAA|2093-12-23|2327|10122|777|2093|3|12|23|4|2093|777|10122|Wednesday|2093Q4|N|N|N|2485848|2486181|2485505|2485778|N|N|N|N|N| +2485871|AAAAAAAAPGOOFCAA|2093-12-24|2327|10122|777|2093|4|12|24|4|2093|777|10122|Thursday|2093Q4|N|N|N|2485848|2486181|2485506|2485779|N|N|N|N|N| +2485872|AAAAAAAAAHOOFCAA|2093-12-25|2327|10122|777|2093|5|12|25|4|2093|777|10122|Friday|2093Q4|N|Y|N|2485848|2486181|2485507|2485780|N|N|N|N|N| +2485873|AAAAAAAABHOOFCAA|2093-12-26|2327|10122|777|2093|6|12|26|4|2093|777|10122|Saturday|2093Q4|Y|Y|N|2485848|2486181|2485508|2485781|N|N|N|N|N| +2485874|AAAAAAAACHOOFCAA|2093-12-27|2327|10122|777|2093|0|12|27|4|2093|777|10122|Sunday|2093Q4|N|N|Y|2485848|2486181|2485509|2485782|N|N|N|N|N| +2485875|AAAAAAAADHOOFCAA|2093-12-28|2327|10122|777|2093|1|12|28|4|2093|777|10122|Monday|2093Q4|N|N|N|2485848|2486181|2485510|2485783|N|N|N|N|N| +2485876|AAAAAAAAEHOOFCAA|2093-12-29|2327|10123|777|2093|2|12|29|4|2093|777|10123|Tuesday|2093Q4|N|N|N|2485848|2486181|2485511|2485784|N|N|N|N|N| +2485877|AAAAAAAAFHOOFCAA|2093-12-30|2327|10123|777|2093|3|12|30|4|2093|777|10123|Wednesday|2093Q4|N|N|N|2485848|2486181|2485512|2485785|N|N|N|N|N| +2485878|AAAAAAAAGHOOFCAA|2093-12-31|2327|10123|777|2093|4|12|31|4|2093|777|10123|Thursday|2093Q4|N|N|N|2485848|2486181|2485513|2485786|N|N|N|N|N| +2485879|AAAAAAAAHHOOFCAA|2094-01-01|2328|10123|777|2094|5|1|1|1|2094|777|10123|Friday|2094Q1|Y|Y|N|2485879|2485878|2485514|2485787|N|N|N|N|N| +2485880|AAAAAAAAIHOOFCAA|2094-01-02|2328|10123|777|2094|6|1|2|1|2094|777|10123|Saturday|2094Q1|N|Y|Y|2485879|2485878|2485515|2485788|N|N|N|N|N| +2485881|AAAAAAAAJHOOFCAA|2094-01-03|2328|10123|777|2094|0|1|3|1|2094|777|10123|Sunday|2094Q1|N|N|N|2485879|2485878|2485516|2485789|N|N|N|N|N| +2485882|AAAAAAAAKHOOFCAA|2094-01-04|2328|10123|777|2094|1|1|4|1|2094|777|10123|Monday|2094Q1|N|N|N|2485879|2485878|2485517|2485790|N|N|N|N|N| +2485883|AAAAAAAALHOOFCAA|2094-01-05|2328|10124|777|2094|2|1|5|1|2094|777|10124|Tuesday|2094Q1|N|N|N|2485879|2485878|2485518|2485791|N|N|N|N|N| +2485884|AAAAAAAAMHOOFCAA|2094-01-06|2328|10124|777|2094|3|1|6|1|2094|777|10124|Wednesday|2094Q1|N|N|N|2485879|2485878|2485519|2485792|N|N|N|N|N| +2485885|AAAAAAAANHOOFCAA|2094-01-07|2328|10124|777|2094|4|1|7|1|2094|777|10124|Thursday|2094Q1|N|N|N|2485879|2485878|2485520|2485793|N|N|N|N|N| +2485886|AAAAAAAAOHOOFCAA|2094-01-08|2328|10124|777|2094|5|1|8|1|2094|777|10124|Friday|2094Q1|N|Y|N|2485879|2485878|2485521|2485794|N|N|N|N|N| +2485887|AAAAAAAAPHOOFCAA|2094-01-09|2328|10124|777|2094|6|1|9|1|2094|777|10124|Saturday|2094Q1|N|Y|N|2485879|2485878|2485522|2485795|N|N|N|N|N| +2485888|AAAAAAAAAIOOFCAA|2094-01-10|2328|10124|777|2094|0|1|10|1|2094|777|10124|Sunday|2094Q1|N|N|N|2485879|2485878|2485523|2485796|N|N|N|N|N| +2485889|AAAAAAAABIOOFCAA|2094-01-11|2328|10124|777|2094|1|1|11|1|2094|777|10124|Monday|2094Q1|N|N|N|2485879|2485878|2485524|2485797|N|N|N|N|N| +2485890|AAAAAAAACIOOFCAA|2094-01-12|2328|10125|777|2094|2|1|12|1|2094|777|10125|Tuesday|2094Q1|N|N|N|2485879|2485878|2485525|2485798|N|N|N|N|N| +2485891|AAAAAAAADIOOFCAA|2094-01-13|2328|10125|777|2094|3|1|13|1|2094|777|10125|Wednesday|2094Q1|N|N|N|2485879|2485878|2485526|2485799|N|N|N|N|N| +2485892|AAAAAAAAEIOOFCAA|2094-01-14|2328|10125|777|2094|4|1|14|1|2094|777|10125|Thursday|2094Q1|N|N|N|2485879|2485878|2485527|2485800|N|N|N|N|N| +2485893|AAAAAAAAFIOOFCAA|2094-01-15|2328|10125|777|2094|5|1|15|1|2094|777|10125|Friday|2094Q1|N|Y|N|2485879|2485878|2485528|2485801|N|N|N|N|N| +2485894|AAAAAAAAGIOOFCAA|2094-01-16|2328|10125|777|2094|6|1|16|1|2094|777|10125|Saturday|2094Q1|N|Y|N|2485879|2485878|2485529|2485802|N|N|N|N|N| +2485895|AAAAAAAAHIOOFCAA|2094-01-17|2328|10125|777|2094|0|1|17|1|2094|777|10125|Sunday|2094Q1|N|N|N|2485879|2485878|2485530|2485803|N|N|N|N|N| +2485896|AAAAAAAAIIOOFCAA|2094-01-18|2328|10125|777|2094|1|1|18|1|2094|777|10125|Monday|2094Q1|N|N|N|2485879|2485878|2485531|2485804|N|N|N|N|N| +2485897|AAAAAAAAJIOOFCAA|2094-01-19|2328|10126|777|2094|2|1|19|1|2094|777|10126|Tuesday|2094Q1|N|N|N|2485879|2485878|2485532|2485805|N|N|N|N|N| +2485898|AAAAAAAAKIOOFCAA|2094-01-20|2328|10126|777|2094|3|1|20|1|2094|777|10126|Wednesday|2094Q1|N|N|N|2485879|2485878|2485533|2485806|N|N|N|N|N| +2485899|AAAAAAAALIOOFCAA|2094-01-21|2328|10126|777|2094|4|1|21|1|2094|777|10126|Thursday|2094Q1|N|N|N|2485879|2485878|2485534|2485807|N|N|N|N|N| +2485900|AAAAAAAAMIOOFCAA|2094-01-22|2328|10126|777|2094|5|1|22|1|2094|777|10126|Friday|2094Q1|N|Y|N|2485879|2485878|2485535|2485808|N|N|N|N|N| +2485901|AAAAAAAANIOOFCAA|2094-01-23|2328|10126|777|2094|6|1|23|1|2094|777|10126|Saturday|2094Q1|N|Y|N|2485879|2485878|2485536|2485809|N|N|N|N|N| +2485902|AAAAAAAAOIOOFCAA|2094-01-24|2328|10126|777|2094|0|1|24|1|2094|777|10126|Sunday|2094Q1|N|N|N|2485879|2485878|2485537|2485810|N|N|N|N|N| +2485903|AAAAAAAAPIOOFCAA|2094-01-25|2328|10126|777|2094|1|1|25|1|2094|777|10126|Monday|2094Q1|N|N|N|2485879|2485878|2485538|2485811|N|N|N|N|N| +2485904|AAAAAAAAAJOOFCAA|2094-01-26|2328|10127|777|2094|2|1|26|1|2094|777|10127|Tuesday|2094Q1|N|N|N|2485879|2485878|2485539|2485812|N|N|N|N|N| +2485905|AAAAAAAABJOOFCAA|2094-01-27|2328|10127|777|2094|3|1|27|1|2094|777|10127|Wednesday|2094Q1|N|N|N|2485879|2485878|2485540|2485813|N|N|N|N|N| +2485906|AAAAAAAACJOOFCAA|2094-01-28|2328|10127|777|2094|4|1|28|1|2094|777|10127|Thursday|2094Q1|N|N|N|2485879|2485878|2485541|2485814|N|N|N|N|N| +2485907|AAAAAAAADJOOFCAA|2094-01-29|2328|10127|777|2094|5|1|29|1|2094|777|10127|Friday|2094Q1|N|Y|N|2485879|2485878|2485542|2485815|N|N|N|N|N| +2485908|AAAAAAAAEJOOFCAA|2094-01-30|2328|10127|777|2094|6|1|30|1|2094|777|10127|Saturday|2094Q1|N|Y|N|2485879|2485878|2485543|2485816|N|N|N|N|N| +2485909|AAAAAAAAFJOOFCAA|2094-01-31|2328|10127|777|2094|0|1|31|1|2094|777|10127|Sunday|2094Q1|N|N|N|2485879|2485878|2485544|2485817|N|N|N|N|N| +2485910|AAAAAAAAGJOOFCAA|2094-02-01|2329|10127|777|2094|1|2|1|1|2094|777|10127|Monday|2094Q1|N|N|N|2485910|2485940|2485545|2485818|N|N|N|N|N| +2485911|AAAAAAAAHJOOFCAA|2094-02-02|2329|10128|777|2094|2|2|2|1|2094|777|10128|Tuesday|2094Q1|N|N|N|2485910|2485940|2485546|2485819|N|N|N|N|N| +2485912|AAAAAAAAIJOOFCAA|2094-02-03|2329|10128|777|2094|3|2|3|1|2094|777|10128|Wednesday|2094Q1|N|N|N|2485910|2485940|2485547|2485820|N|N|N|N|N| +2485913|AAAAAAAAJJOOFCAA|2094-02-04|2329|10128|777|2094|4|2|4|1|2094|777|10128|Thursday|2094Q1|N|N|N|2485910|2485940|2485548|2485821|N|N|N|N|N| +2485914|AAAAAAAAKJOOFCAA|2094-02-05|2329|10128|777|2094|5|2|5|1|2094|777|10128|Friday|2094Q1|N|Y|N|2485910|2485940|2485549|2485822|N|N|N|N|N| +2485915|AAAAAAAALJOOFCAA|2094-02-06|2329|10128|777|2094|6|2|6|1|2094|777|10128|Saturday|2094Q1|N|Y|N|2485910|2485940|2485550|2485823|N|N|N|N|N| +2485916|AAAAAAAAMJOOFCAA|2094-02-07|2329|10128|777|2094|0|2|7|1|2094|777|10128|Sunday|2094Q1|N|N|N|2485910|2485940|2485551|2485824|N|N|N|N|N| +2485917|AAAAAAAANJOOFCAA|2094-02-08|2329|10128|777|2094|1|2|8|1|2094|777|10128|Monday|2094Q1|N|N|N|2485910|2485940|2485552|2485825|N|N|N|N|N| +2485918|AAAAAAAAOJOOFCAA|2094-02-09|2329|10129|777|2094|2|2|9|1|2094|777|10129|Tuesday|2094Q1|N|N|N|2485910|2485940|2485553|2485826|N|N|N|N|N| +2485919|AAAAAAAAPJOOFCAA|2094-02-10|2329|10129|777|2094|3|2|10|1|2094|777|10129|Wednesday|2094Q1|N|N|N|2485910|2485940|2485554|2485827|N|N|N|N|N| +2485920|AAAAAAAAAKOOFCAA|2094-02-11|2329|10129|777|2094|4|2|11|1|2094|777|10129|Thursday|2094Q1|N|N|N|2485910|2485940|2485555|2485828|N|N|N|N|N| +2485921|AAAAAAAABKOOFCAA|2094-02-12|2329|10129|777|2094|5|2|12|1|2094|777|10129|Friday|2094Q1|N|Y|N|2485910|2485940|2485556|2485829|N|N|N|N|N| +2485922|AAAAAAAACKOOFCAA|2094-02-13|2329|10129|777|2094|6|2|13|1|2094|777|10129|Saturday|2094Q1|N|Y|N|2485910|2485940|2485557|2485830|N|N|N|N|N| +2485923|AAAAAAAADKOOFCAA|2094-02-14|2329|10129|777|2094|0|2|14|1|2094|777|10129|Sunday|2094Q1|N|N|N|2485910|2485940|2485558|2485831|N|N|N|N|N| +2485924|AAAAAAAAEKOOFCAA|2094-02-15|2329|10129|777|2094|1|2|15|1|2094|777|10129|Monday|2094Q1|N|N|N|2485910|2485940|2485559|2485832|N|N|N|N|N| +2485925|AAAAAAAAFKOOFCAA|2094-02-16|2329|10130|777|2094|2|2|16|1|2094|777|10130|Tuesday|2094Q1|N|N|N|2485910|2485940|2485560|2485833|N|N|N|N|N| +2485926|AAAAAAAAGKOOFCAA|2094-02-17|2329|10130|777|2094|3|2|17|1|2094|777|10130|Wednesday|2094Q1|N|N|N|2485910|2485940|2485561|2485834|N|N|N|N|N| +2485927|AAAAAAAAHKOOFCAA|2094-02-18|2329|10130|777|2094|4|2|18|1|2094|777|10130|Thursday|2094Q1|N|N|N|2485910|2485940|2485562|2485835|N|N|N|N|N| +2485928|AAAAAAAAIKOOFCAA|2094-02-19|2329|10130|777|2094|5|2|19|1|2094|777|10130|Friday|2094Q1|N|Y|N|2485910|2485940|2485563|2485836|N|N|N|N|N| +2485929|AAAAAAAAJKOOFCAA|2094-02-20|2329|10130|777|2094|6|2|20|1|2094|777|10130|Saturday|2094Q1|N|Y|N|2485910|2485940|2485564|2485837|N|N|N|N|N| +2485930|AAAAAAAAKKOOFCAA|2094-02-21|2329|10130|777|2094|0|2|21|1|2094|777|10130|Sunday|2094Q1|N|N|N|2485910|2485940|2485565|2485838|N|N|N|N|N| +2485931|AAAAAAAALKOOFCAA|2094-02-22|2329|10130|777|2094|1|2|22|1|2094|777|10130|Monday|2094Q1|N|N|N|2485910|2485940|2485566|2485839|N|N|N|N|N| +2485932|AAAAAAAAMKOOFCAA|2094-02-23|2329|10131|777|2094|2|2|23|1|2094|777|10131|Tuesday|2094Q1|N|N|N|2485910|2485940|2485567|2485840|N|N|N|N|N| +2485933|AAAAAAAANKOOFCAA|2094-02-24|2329|10131|777|2094|3|2|24|1|2094|777|10131|Wednesday|2094Q1|N|N|N|2485910|2485940|2485568|2485841|N|N|N|N|N| +2485934|AAAAAAAAOKOOFCAA|2094-02-25|2329|10131|777|2094|4|2|25|1|2094|777|10131|Thursday|2094Q1|N|N|N|2485910|2485940|2485569|2485842|N|N|N|N|N| +2485935|AAAAAAAAPKOOFCAA|2094-02-26|2329|10131|777|2094|5|2|26|1|2094|777|10131|Friday|2094Q1|N|Y|N|2485910|2485940|2485570|2485843|N|N|N|N|N| +2485936|AAAAAAAAALOOFCAA|2094-02-27|2329|10131|777|2094|6|2|27|1|2094|777|10131|Saturday|2094Q1|N|Y|N|2485910|2485940|2485571|2485844|N|N|N|N|N| +2485937|AAAAAAAABLOOFCAA|2094-02-28|2329|10131|777|2094|0|2|28|1|2094|777|10131|Sunday|2094Q1|N|N|N|2485910|2485940|2485572|2485845|N|N|N|N|N| +2485938|AAAAAAAACLOOFCAA|2094-03-01|2330|10131|778|2094|1|3|1|1|2094|778|10131|Monday|2094Q1|N|N|N|2485938|2485996|2485573|2485846|N|N|N|N|N| +2485939|AAAAAAAADLOOFCAA|2094-03-02|2330|10132|778|2094|2|3|2|1|2094|778|10132|Tuesday|2094Q1|N|N|N|2485938|2485996|2485574|2485847|N|N|N|N|N| +2485940|AAAAAAAAELOOFCAA|2094-03-03|2330|10132|778|2094|3|3|3|1|2094|778|10132|Wednesday|2094Q1|N|N|N|2485938|2485996|2485575|2485848|N|N|N|N|N| +2485941|AAAAAAAAFLOOFCAA|2094-03-04|2330|10132|778|2094|4|3|4|1|2094|778|10132|Thursday|2094Q1|N|N|N|2485938|2485996|2485576|2485849|N|N|N|N|N| +2485942|AAAAAAAAGLOOFCAA|2094-03-05|2330|10132|778|2094|5|3|5|1|2094|778|10132|Friday|2094Q1|N|Y|N|2485938|2485996|2485577|2485850|N|N|N|N|N| +2485943|AAAAAAAAHLOOFCAA|2094-03-06|2330|10132|778|2094|6|3|6|1|2094|778|10132|Saturday|2094Q1|N|Y|N|2485938|2485996|2485578|2485851|N|N|N|N|N| +2485944|AAAAAAAAILOOFCAA|2094-03-07|2330|10132|778|2094|0|3|7|1|2094|778|10132|Sunday|2094Q1|N|N|N|2485938|2485996|2485579|2485852|N|N|N|N|N| +2485945|AAAAAAAAJLOOFCAA|2094-03-08|2330|10132|778|2094|1|3|8|1|2094|778|10132|Monday|2094Q1|N|N|N|2485938|2485996|2485580|2485853|N|N|N|N|N| +2485946|AAAAAAAAKLOOFCAA|2094-03-09|2330|10133|778|2094|2|3|9|1|2094|778|10133|Tuesday|2094Q1|N|N|N|2485938|2485996|2485581|2485854|N|N|N|N|N| +2485947|AAAAAAAALLOOFCAA|2094-03-10|2330|10133|778|2094|3|3|10|1|2094|778|10133|Wednesday|2094Q1|N|N|N|2485938|2485996|2485582|2485855|N|N|N|N|N| +2485948|AAAAAAAAMLOOFCAA|2094-03-11|2330|10133|778|2094|4|3|11|1|2094|778|10133|Thursday|2094Q1|N|N|N|2485938|2485996|2485583|2485856|N|N|N|N|N| +2485949|AAAAAAAANLOOFCAA|2094-03-12|2330|10133|778|2094|5|3|12|1|2094|778|10133|Friday|2094Q1|N|Y|N|2485938|2485996|2485584|2485857|N|N|N|N|N| +2485950|AAAAAAAAOLOOFCAA|2094-03-13|2330|10133|778|2094|6|3|13|1|2094|778|10133|Saturday|2094Q1|N|Y|N|2485938|2485996|2485585|2485858|N|N|N|N|N| +2485951|AAAAAAAAPLOOFCAA|2094-03-14|2330|10133|778|2094|0|3|14|1|2094|778|10133|Sunday|2094Q1|N|N|N|2485938|2485996|2485586|2485859|N|N|N|N|N| +2485952|AAAAAAAAAMOOFCAA|2094-03-15|2330|10133|778|2094|1|3|15|1|2094|778|10133|Monday|2094Q1|N|N|N|2485938|2485996|2485587|2485860|N|N|N|N|N| +2485953|AAAAAAAABMOOFCAA|2094-03-16|2330|10134|778|2094|2|3|16|1|2094|778|10134|Tuesday|2094Q1|N|N|N|2485938|2485996|2485588|2485861|N|N|N|N|N| +2485954|AAAAAAAACMOOFCAA|2094-03-17|2330|10134|778|2094|3|3|17|1|2094|778|10134|Wednesday|2094Q1|N|N|N|2485938|2485996|2485589|2485862|N|N|N|N|N| +2485955|AAAAAAAADMOOFCAA|2094-03-18|2330|10134|778|2094|4|3|18|1|2094|778|10134|Thursday|2094Q1|N|N|N|2485938|2485996|2485590|2485863|N|N|N|N|N| +2485956|AAAAAAAAEMOOFCAA|2094-03-19|2330|10134|778|2094|5|3|19|1|2094|778|10134|Friday|2094Q1|N|Y|N|2485938|2485996|2485591|2485864|N|N|N|N|N| +2485957|AAAAAAAAFMOOFCAA|2094-03-20|2330|10134|778|2094|6|3|20|1|2094|778|10134|Saturday|2094Q1|N|Y|N|2485938|2485996|2485592|2485865|N|N|N|N|N| +2485958|AAAAAAAAGMOOFCAA|2094-03-21|2330|10134|778|2094|0|3|21|1|2094|778|10134|Sunday|2094Q1|N|N|N|2485938|2485996|2485593|2485866|N|N|N|N|N| +2485959|AAAAAAAAHMOOFCAA|2094-03-22|2330|10134|778|2094|1|3|22|1|2094|778|10134|Monday|2094Q1|N|N|N|2485938|2485996|2485594|2485867|N|N|N|N|N| +2485960|AAAAAAAAIMOOFCAA|2094-03-23|2330|10135|778|2094|2|3|23|1|2094|778|10135|Tuesday|2094Q1|N|N|N|2485938|2485996|2485595|2485868|N|N|N|N|N| +2485961|AAAAAAAAJMOOFCAA|2094-03-24|2330|10135|778|2094|3|3|24|1|2094|778|10135|Wednesday|2094Q1|N|N|N|2485938|2485996|2485596|2485869|N|N|N|N|N| +2485962|AAAAAAAAKMOOFCAA|2094-03-25|2330|10135|778|2094|4|3|25|1|2094|778|10135|Thursday|2094Q1|N|N|N|2485938|2485996|2485597|2485870|N|N|N|N|N| +2485963|AAAAAAAALMOOFCAA|2094-03-26|2330|10135|778|2094|5|3|26|1|2094|778|10135|Friday|2094Q1|N|Y|N|2485938|2485996|2485598|2485871|N|N|N|N|N| +2485964|AAAAAAAAMMOOFCAA|2094-03-27|2330|10135|778|2094|6|3|27|1|2094|778|10135|Saturday|2094Q1|N|Y|N|2485938|2485996|2485599|2485872|N|N|N|N|N| +2485965|AAAAAAAANMOOFCAA|2094-03-28|2330|10135|778|2094|0|3|28|1|2094|778|10135|Sunday|2094Q1|N|N|N|2485938|2485996|2485600|2485873|N|N|N|N|N| +2485966|AAAAAAAAOMOOFCAA|2094-03-29|2330|10135|778|2094|1|3|29|1|2094|778|10135|Monday|2094Q1|N|N|N|2485938|2485996|2485601|2485874|N|N|N|N|N| +2485967|AAAAAAAAPMOOFCAA|2094-03-30|2330|10136|778|2094|2|3|30|1|2094|778|10136|Tuesday|2094Q1|N|N|N|2485938|2485996|2485602|2485875|N|N|N|N|N| +2485968|AAAAAAAAANOOFCAA|2094-03-31|2330|10136|778|2094|3|3|31|1|2094|778|10136|Wednesday|2094Q1|N|N|N|2485938|2485996|2485603|2485876|N|N|N|N|N| +2485969|AAAAAAAABNOOFCAA|2094-04-01|2331|10136|778|2094|4|4|1|1|2094|778|10136|Thursday|2094Q1|N|N|N|2485969|2486058|2485604|2485879|N|N|N|N|N| +2485970|AAAAAAAACNOOFCAA|2094-04-02|2331|10136|778|2094|5|4|2|2|2094|778|10136|Friday|2094Q2|N|Y|N|2485969|2486058|2485605|2485880|N|N|N|N|N| +2485971|AAAAAAAADNOOFCAA|2094-04-03|2331|10136|778|2094|6|4|3|2|2094|778|10136|Saturday|2094Q2|N|Y|N|2485969|2486058|2485606|2485881|N|N|N|N|N| +2485972|AAAAAAAAENOOFCAA|2094-04-04|2331|10136|778|2094|0|4|4|2|2094|778|10136|Sunday|2094Q2|N|N|N|2485969|2486058|2485607|2485882|N|N|N|N|N| +2485973|AAAAAAAAFNOOFCAA|2094-04-05|2331|10136|778|2094|1|4|5|2|2094|778|10136|Monday|2094Q2|N|N|N|2485969|2486058|2485608|2485883|N|N|N|N|N| +2485974|AAAAAAAAGNOOFCAA|2094-04-06|2331|10137|778|2094|2|4|6|2|2094|778|10137|Tuesday|2094Q2|N|N|N|2485969|2486058|2485609|2485884|N|N|N|N|N| +2485975|AAAAAAAAHNOOFCAA|2094-04-07|2331|10137|778|2094|3|4|7|2|2094|778|10137|Wednesday|2094Q2|N|N|N|2485969|2486058|2485610|2485885|N|N|N|N|N| +2485976|AAAAAAAAINOOFCAA|2094-04-08|2331|10137|778|2094|4|4|8|2|2094|778|10137|Thursday|2094Q2|N|N|N|2485969|2486058|2485611|2485886|N|N|N|N|N| +2485977|AAAAAAAAJNOOFCAA|2094-04-09|2331|10137|778|2094|5|4|9|2|2094|778|10137|Friday|2094Q2|N|Y|N|2485969|2486058|2485612|2485887|N|N|N|N|N| +2485978|AAAAAAAAKNOOFCAA|2094-04-10|2331|10137|778|2094|6|4|10|2|2094|778|10137|Saturday|2094Q2|N|Y|N|2485969|2486058|2485613|2485888|N|N|N|N|N| +2485979|AAAAAAAALNOOFCAA|2094-04-11|2331|10137|778|2094|0|4|11|2|2094|778|10137|Sunday|2094Q2|N|N|N|2485969|2486058|2485614|2485889|N|N|N|N|N| +2485980|AAAAAAAAMNOOFCAA|2094-04-12|2331|10137|778|2094|1|4|12|2|2094|778|10137|Monday|2094Q2|N|N|N|2485969|2486058|2485615|2485890|N|N|N|N|N| +2485981|AAAAAAAANNOOFCAA|2094-04-13|2331|10138|778|2094|2|4|13|2|2094|778|10138|Tuesday|2094Q2|N|N|N|2485969|2486058|2485616|2485891|N|N|N|N|N| +2485982|AAAAAAAAONOOFCAA|2094-04-14|2331|10138|778|2094|3|4|14|2|2094|778|10138|Wednesday|2094Q2|N|N|N|2485969|2486058|2485617|2485892|N|N|N|N|N| +2485983|AAAAAAAAPNOOFCAA|2094-04-15|2331|10138|778|2094|4|4|15|2|2094|778|10138|Thursday|2094Q2|N|N|N|2485969|2486058|2485618|2485893|N|N|N|N|N| +2485984|AAAAAAAAAOOOFCAA|2094-04-16|2331|10138|778|2094|5|4|16|2|2094|778|10138|Friday|2094Q2|N|Y|N|2485969|2486058|2485619|2485894|N|N|N|N|N| +2485985|AAAAAAAABOOOFCAA|2094-04-17|2331|10138|778|2094|6|4|17|2|2094|778|10138|Saturday|2094Q2|N|Y|N|2485969|2486058|2485620|2485895|N|N|N|N|N| +2485986|AAAAAAAACOOOFCAA|2094-04-18|2331|10138|778|2094|0|4|18|2|2094|778|10138|Sunday|2094Q2|N|N|N|2485969|2486058|2485621|2485896|N|N|N|N|N| +2485987|AAAAAAAADOOOFCAA|2094-04-19|2331|10138|778|2094|1|4|19|2|2094|778|10138|Monday|2094Q2|N|N|N|2485969|2486058|2485622|2485897|N|N|N|N|N| +2485988|AAAAAAAAEOOOFCAA|2094-04-20|2331|10139|778|2094|2|4|20|2|2094|778|10139|Tuesday|2094Q2|N|N|N|2485969|2486058|2485623|2485898|N|N|N|N|N| +2485989|AAAAAAAAFOOOFCAA|2094-04-21|2331|10139|778|2094|3|4|21|2|2094|778|10139|Wednesday|2094Q2|N|N|N|2485969|2486058|2485624|2485899|N|N|N|N|N| +2485990|AAAAAAAAGOOOFCAA|2094-04-22|2331|10139|778|2094|4|4|22|2|2094|778|10139|Thursday|2094Q2|N|N|N|2485969|2486058|2485625|2485900|N|N|N|N|N| +2485991|AAAAAAAAHOOOFCAA|2094-04-23|2331|10139|778|2094|5|4|23|2|2094|778|10139|Friday|2094Q2|N|Y|N|2485969|2486058|2485626|2485901|N|N|N|N|N| +2485992|AAAAAAAAIOOOFCAA|2094-04-24|2331|10139|778|2094|6|4|24|2|2094|778|10139|Saturday|2094Q2|N|Y|N|2485969|2486058|2485627|2485902|N|N|N|N|N| +2485993|AAAAAAAAJOOOFCAA|2094-04-25|2331|10139|778|2094|0|4|25|2|2094|778|10139|Sunday|2094Q2|N|N|N|2485969|2486058|2485628|2485903|N|N|N|N|N| +2485994|AAAAAAAAKOOOFCAA|2094-04-26|2331|10139|778|2094|1|4|26|2|2094|778|10139|Monday|2094Q2|N|N|N|2485969|2486058|2485629|2485904|N|N|N|N|N| +2485995|AAAAAAAALOOOFCAA|2094-04-27|2331|10140|778|2094|2|4|27|2|2094|778|10140|Tuesday|2094Q2|N|N|N|2485969|2486058|2485630|2485905|N|N|N|N|N| +2485996|AAAAAAAAMOOOFCAA|2094-04-28|2331|10140|778|2094|3|4|28|2|2094|778|10140|Wednesday|2094Q2|N|N|N|2485969|2486058|2485631|2485906|N|N|N|N|N| +2485997|AAAAAAAANOOOFCAA|2094-04-29|2331|10140|778|2094|4|4|29|2|2094|778|10140|Thursday|2094Q2|N|N|N|2485969|2486058|2485632|2485907|N|N|N|N|N| +2485998|AAAAAAAAOOOOFCAA|2094-04-30|2331|10140|778|2094|5|4|30|2|2094|778|10140|Friday|2094Q2|N|Y|N|2485969|2486058|2485633|2485908|N|N|N|N|N| +2485999|AAAAAAAAPOOOFCAA|2094-05-01|2332|10140|778|2094|6|5|1|2|2094|778|10140|Saturday|2094Q2|N|Y|N|2485999|2486118|2485634|2485909|N|N|N|N|N| +2486000|AAAAAAAAAPOOFCAA|2094-05-02|2332|10140|778|2094|0|5|2|2|2094|778|10140|Sunday|2094Q2|N|N|N|2485999|2486118|2485635|2485910|N|N|N|N|N| +2486001|AAAAAAAABPOOFCAA|2094-05-03|2332|10140|778|2094|1|5|3|2|2094|778|10140|Monday|2094Q2|N|N|N|2485999|2486118|2485636|2485911|N|N|N|N|N| +2486002|AAAAAAAACPOOFCAA|2094-05-04|2332|10141|778|2094|2|5|4|2|2094|778|10141|Tuesday|2094Q2|N|N|N|2485999|2486118|2485637|2485912|N|N|N|N|N| +2486003|AAAAAAAADPOOFCAA|2094-05-05|2332|10141|778|2094|3|5|5|2|2094|778|10141|Wednesday|2094Q2|N|N|N|2485999|2486118|2485638|2485913|N|N|N|N|N| +2486004|AAAAAAAAEPOOFCAA|2094-05-06|2332|10141|778|2094|4|5|6|2|2094|778|10141|Thursday|2094Q2|N|N|N|2485999|2486118|2485639|2485914|N|N|N|N|N| +2486005|AAAAAAAAFPOOFCAA|2094-05-07|2332|10141|778|2094|5|5|7|2|2094|778|10141|Friday|2094Q2|N|Y|N|2485999|2486118|2485640|2485915|N|N|N|N|N| +2486006|AAAAAAAAGPOOFCAA|2094-05-08|2332|10141|778|2094|6|5|8|2|2094|778|10141|Saturday|2094Q2|N|Y|N|2485999|2486118|2485641|2485916|N|N|N|N|N| +2486007|AAAAAAAAHPOOFCAA|2094-05-09|2332|10141|778|2094|0|5|9|2|2094|778|10141|Sunday|2094Q2|N|N|N|2485999|2486118|2485642|2485917|N|N|N|N|N| +2486008|AAAAAAAAIPOOFCAA|2094-05-10|2332|10141|778|2094|1|5|10|2|2094|778|10141|Monday|2094Q2|N|N|N|2485999|2486118|2485643|2485918|N|N|N|N|N| +2486009|AAAAAAAAJPOOFCAA|2094-05-11|2332|10142|778|2094|2|5|11|2|2094|778|10142|Tuesday|2094Q2|N|N|N|2485999|2486118|2485644|2485919|N|N|N|N|N| +2486010|AAAAAAAAKPOOFCAA|2094-05-12|2332|10142|778|2094|3|5|12|2|2094|778|10142|Wednesday|2094Q2|N|N|N|2485999|2486118|2485645|2485920|N|N|N|N|N| +2486011|AAAAAAAALPOOFCAA|2094-05-13|2332|10142|778|2094|4|5|13|2|2094|778|10142|Thursday|2094Q2|N|N|N|2485999|2486118|2485646|2485921|N|N|N|N|N| +2486012|AAAAAAAAMPOOFCAA|2094-05-14|2332|10142|778|2094|5|5|14|2|2094|778|10142|Friday|2094Q2|N|Y|N|2485999|2486118|2485647|2485922|N|N|N|N|N| +2486013|AAAAAAAANPOOFCAA|2094-05-15|2332|10142|778|2094|6|5|15|2|2094|778|10142|Saturday|2094Q2|N|Y|N|2485999|2486118|2485648|2485923|N|N|N|N|N| +2486014|AAAAAAAAOPOOFCAA|2094-05-16|2332|10142|778|2094|0|5|16|2|2094|778|10142|Sunday|2094Q2|N|N|N|2485999|2486118|2485649|2485924|N|N|N|N|N| +2486015|AAAAAAAAPPOOFCAA|2094-05-17|2332|10142|778|2094|1|5|17|2|2094|778|10142|Monday|2094Q2|N|N|N|2485999|2486118|2485650|2485925|N|N|N|N|N| +2486016|AAAAAAAAAAPOFCAA|2094-05-18|2332|10143|778|2094|2|5|18|2|2094|778|10143|Tuesday|2094Q2|N|N|N|2485999|2486118|2485651|2485926|N|N|N|N|N| +2486017|AAAAAAAABAPOFCAA|2094-05-19|2332|10143|778|2094|3|5|19|2|2094|778|10143|Wednesday|2094Q2|N|N|N|2485999|2486118|2485652|2485927|N|N|N|N|N| +2486018|AAAAAAAACAPOFCAA|2094-05-20|2332|10143|778|2094|4|5|20|2|2094|778|10143|Thursday|2094Q2|N|N|N|2485999|2486118|2485653|2485928|N|N|N|N|N| +2486019|AAAAAAAADAPOFCAA|2094-05-21|2332|10143|778|2094|5|5|21|2|2094|778|10143|Friday|2094Q2|N|Y|N|2485999|2486118|2485654|2485929|N|N|N|N|N| +2486020|AAAAAAAAEAPOFCAA|2094-05-22|2332|10143|778|2094|6|5|22|2|2094|778|10143|Saturday|2094Q2|N|Y|N|2485999|2486118|2485655|2485930|N|N|N|N|N| +2486021|AAAAAAAAFAPOFCAA|2094-05-23|2332|10143|778|2094|0|5|23|2|2094|778|10143|Sunday|2094Q2|N|N|N|2485999|2486118|2485656|2485931|N|N|N|N|N| +2486022|AAAAAAAAGAPOFCAA|2094-05-24|2332|10143|778|2094|1|5|24|2|2094|778|10143|Monday|2094Q2|N|N|N|2485999|2486118|2485657|2485932|N|N|N|N|N| +2486023|AAAAAAAAHAPOFCAA|2094-05-25|2332|10144|778|2094|2|5|25|2|2094|778|10144|Tuesday|2094Q2|N|N|N|2485999|2486118|2485658|2485933|N|N|N|N|N| +2486024|AAAAAAAAIAPOFCAA|2094-05-26|2332|10144|778|2094|3|5|26|2|2094|778|10144|Wednesday|2094Q2|N|N|N|2485999|2486118|2485659|2485934|N|N|N|N|N| +2486025|AAAAAAAAJAPOFCAA|2094-05-27|2332|10144|778|2094|4|5|27|2|2094|778|10144|Thursday|2094Q2|N|N|N|2485999|2486118|2485660|2485935|N|N|N|N|N| +2486026|AAAAAAAAKAPOFCAA|2094-05-28|2332|10144|778|2094|5|5|28|2|2094|778|10144|Friday|2094Q2|N|Y|N|2485999|2486118|2485661|2485936|N|N|N|N|N| +2486027|AAAAAAAALAPOFCAA|2094-05-29|2332|10144|778|2094|6|5|29|2|2094|778|10144|Saturday|2094Q2|N|Y|N|2485999|2486118|2485662|2485937|N|N|N|N|N| +2486028|AAAAAAAAMAPOFCAA|2094-05-30|2332|10144|778|2094|0|5|30|2|2094|778|10144|Sunday|2094Q2|N|N|N|2485999|2486118|2485663|2485938|N|N|N|N|N| +2486029|AAAAAAAANAPOFCAA|2094-05-31|2332|10144|778|2094|1|5|31|2|2094|778|10144|Monday|2094Q2|N|N|N|2485999|2486118|2485664|2485939|N|N|N|N|N| +2486030|AAAAAAAAOAPOFCAA|2094-06-01|2333|10145|779|2094|2|6|1|2|2094|779|10145|Tuesday|2094Q2|N|N|N|2486030|2486180|2485665|2485940|N|N|N|N|N| +2486031|AAAAAAAAPAPOFCAA|2094-06-02|2333|10145|779|2094|3|6|2|2|2094|779|10145|Wednesday|2094Q2|N|N|N|2486030|2486180|2485666|2485941|N|N|N|N|N| +2486032|AAAAAAAAABPOFCAA|2094-06-03|2333|10145|779|2094|4|6|3|2|2094|779|10145|Thursday|2094Q2|N|N|N|2486030|2486180|2485667|2485942|N|N|N|N|N| +2486033|AAAAAAAABBPOFCAA|2094-06-04|2333|10145|779|2094|5|6|4|2|2094|779|10145|Friday|2094Q2|N|Y|N|2486030|2486180|2485668|2485943|N|N|N|N|N| +2486034|AAAAAAAACBPOFCAA|2094-06-05|2333|10145|779|2094|6|6|5|2|2094|779|10145|Saturday|2094Q2|N|Y|N|2486030|2486180|2485669|2485944|N|N|N|N|N| +2486035|AAAAAAAADBPOFCAA|2094-06-06|2333|10145|779|2094|0|6|6|2|2094|779|10145|Sunday|2094Q2|N|N|N|2486030|2486180|2485670|2485945|N|N|N|N|N| +2486036|AAAAAAAAEBPOFCAA|2094-06-07|2333|10145|779|2094|1|6|7|2|2094|779|10145|Monday|2094Q2|N|N|N|2486030|2486180|2485671|2485946|N|N|N|N|N| +2486037|AAAAAAAAFBPOFCAA|2094-06-08|2333|10146|779|2094|2|6|8|2|2094|779|10146|Tuesday|2094Q2|N|N|N|2486030|2486180|2485672|2485947|N|N|N|N|N| +2486038|AAAAAAAAGBPOFCAA|2094-06-09|2333|10146|779|2094|3|6|9|2|2094|779|10146|Wednesday|2094Q2|N|N|N|2486030|2486180|2485673|2485948|N|N|N|N|N| +2486039|AAAAAAAAHBPOFCAA|2094-06-10|2333|10146|779|2094|4|6|10|2|2094|779|10146|Thursday|2094Q2|N|N|N|2486030|2486180|2485674|2485949|N|N|N|N|N| +2486040|AAAAAAAAIBPOFCAA|2094-06-11|2333|10146|779|2094|5|6|11|2|2094|779|10146|Friday|2094Q2|N|Y|N|2486030|2486180|2485675|2485950|N|N|N|N|N| +2486041|AAAAAAAAJBPOFCAA|2094-06-12|2333|10146|779|2094|6|6|12|2|2094|779|10146|Saturday|2094Q2|N|Y|N|2486030|2486180|2485676|2485951|N|N|N|N|N| +2486042|AAAAAAAAKBPOFCAA|2094-06-13|2333|10146|779|2094|0|6|13|2|2094|779|10146|Sunday|2094Q2|N|N|N|2486030|2486180|2485677|2485952|N|N|N|N|N| +2486043|AAAAAAAALBPOFCAA|2094-06-14|2333|10146|779|2094|1|6|14|2|2094|779|10146|Monday|2094Q2|N|N|N|2486030|2486180|2485678|2485953|N|N|N|N|N| +2486044|AAAAAAAAMBPOFCAA|2094-06-15|2333|10147|779|2094|2|6|15|2|2094|779|10147|Tuesday|2094Q2|N|N|N|2486030|2486180|2485679|2485954|N|N|N|N|N| +2486045|AAAAAAAANBPOFCAA|2094-06-16|2333|10147|779|2094|3|6|16|2|2094|779|10147|Wednesday|2094Q2|N|N|N|2486030|2486180|2485680|2485955|N|N|N|N|N| +2486046|AAAAAAAAOBPOFCAA|2094-06-17|2333|10147|779|2094|4|6|17|2|2094|779|10147|Thursday|2094Q2|N|N|N|2486030|2486180|2485681|2485956|N|N|N|N|N| +2486047|AAAAAAAAPBPOFCAA|2094-06-18|2333|10147|779|2094|5|6|18|2|2094|779|10147|Friday|2094Q2|N|Y|N|2486030|2486180|2485682|2485957|N|N|N|N|N| +2486048|AAAAAAAAACPOFCAA|2094-06-19|2333|10147|779|2094|6|6|19|2|2094|779|10147|Saturday|2094Q2|N|Y|N|2486030|2486180|2485683|2485958|N|N|N|N|N| +2486049|AAAAAAAABCPOFCAA|2094-06-20|2333|10147|779|2094|0|6|20|2|2094|779|10147|Sunday|2094Q2|N|N|N|2486030|2486180|2485684|2485959|N|N|N|N|N| +2486050|AAAAAAAACCPOFCAA|2094-06-21|2333|10147|779|2094|1|6|21|2|2094|779|10147|Monday|2094Q2|N|N|N|2486030|2486180|2485685|2485960|N|N|N|N|N| +2486051|AAAAAAAADCPOFCAA|2094-06-22|2333|10148|779|2094|2|6|22|2|2094|779|10148|Tuesday|2094Q2|N|N|N|2486030|2486180|2485686|2485961|N|N|N|N|N| +2486052|AAAAAAAAECPOFCAA|2094-06-23|2333|10148|779|2094|3|6|23|2|2094|779|10148|Wednesday|2094Q2|N|N|N|2486030|2486180|2485687|2485962|N|N|N|N|N| +2486053|AAAAAAAAFCPOFCAA|2094-06-24|2333|10148|779|2094|4|6|24|2|2094|779|10148|Thursday|2094Q2|N|N|N|2486030|2486180|2485688|2485963|N|N|N|N|N| +2486054|AAAAAAAAGCPOFCAA|2094-06-25|2333|10148|779|2094|5|6|25|2|2094|779|10148|Friday|2094Q2|N|Y|N|2486030|2486180|2485689|2485964|N|N|N|N|N| +2486055|AAAAAAAAHCPOFCAA|2094-06-26|2333|10148|779|2094|6|6|26|2|2094|779|10148|Saturday|2094Q2|N|Y|N|2486030|2486180|2485690|2485965|N|N|N|N|N| +2486056|AAAAAAAAICPOFCAA|2094-06-27|2333|10148|779|2094|0|6|27|2|2094|779|10148|Sunday|2094Q2|N|N|N|2486030|2486180|2485691|2485966|N|N|N|N|N| +2486057|AAAAAAAAJCPOFCAA|2094-06-28|2333|10148|779|2094|1|6|28|2|2094|779|10148|Monday|2094Q2|N|N|N|2486030|2486180|2485692|2485967|N|N|N|N|N| +2486058|AAAAAAAAKCPOFCAA|2094-06-29|2333|10149|779|2094|2|6|29|2|2094|779|10149|Tuesday|2094Q2|N|N|N|2486030|2486180|2485693|2485968|N|N|N|N|N| +2486059|AAAAAAAALCPOFCAA|2094-06-30|2333|10149|779|2094|3|6|30|2|2094|779|10149|Wednesday|2094Q2|N|N|N|2486030|2486180|2485694|2485969|N|N|N|N|N| +2486060|AAAAAAAAMCPOFCAA|2094-07-01|2334|10149|779|2094|4|7|1|2|2094|779|10149|Thursday|2094Q2|N|N|N|2486060|2486240|2485695|2485969|N|N|N|N|N| +2486061|AAAAAAAANCPOFCAA|2094-07-02|2334|10149|779|2094|5|7|2|3|2094|779|10149|Friday|2094Q3|N|Y|N|2486060|2486240|2485696|2485970|N|N|N|N|N| +2486062|AAAAAAAAOCPOFCAA|2094-07-03|2334|10149|779|2094|6|7|3|3|2094|779|10149|Saturday|2094Q3|N|Y|N|2486060|2486240|2485697|2485971|N|N|N|N|N| +2486063|AAAAAAAAPCPOFCAA|2094-07-04|2334|10149|779|2094|0|7|4|3|2094|779|10149|Sunday|2094Q3|N|N|N|2486060|2486240|2485698|2485972|N|N|N|N|N| +2486064|AAAAAAAAADPOFCAA|2094-07-05|2334|10149|779|2094|1|7|5|3|2094|779|10149|Monday|2094Q3|Y|N|N|2486060|2486240|2485699|2485973|N|N|N|N|N| +2486065|AAAAAAAABDPOFCAA|2094-07-06|2334|10150|779|2094|2|7|6|3|2094|779|10150|Tuesday|2094Q3|N|N|Y|2486060|2486240|2485700|2485974|N|N|N|N|N| +2486066|AAAAAAAACDPOFCAA|2094-07-07|2334|10150|779|2094|3|7|7|3|2094|779|10150|Wednesday|2094Q3|N|N|N|2486060|2486240|2485701|2485975|N|N|N|N|N| +2486067|AAAAAAAADDPOFCAA|2094-07-08|2334|10150|779|2094|4|7|8|3|2094|779|10150|Thursday|2094Q3|N|N|N|2486060|2486240|2485702|2485976|N|N|N|N|N| +2486068|AAAAAAAAEDPOFCAA|2094-07-09|2334|10150|779|2094|5|7|9|3|2094|779|10150|Friday|2094Q3|N|Y|N|2486060|2486240|2485703|2485977|N|N|N|N|N| +2486069|AAAAAAAAFDPOFCAA|2094-07-10|2334|10150|779|2094|6|7|10|3|2094|779|10150|Saturday|2094Q3|N|Y|N|2486060|2486240|2485704|2485978|N|N|N|N|N| +2486070|AAAAAAAAGDPOFCAA|2094-07-11|2334|10150|779|2094|0|7|11|3|2094|779|10150|Sunday|2094Q3|N|N|N|2486060|2486240|2485705|2485979|N|N|N|N|N| +2486071|AAAAAAAAHDPOFCAA|2094-07-12|2334|10150|779|2094|1|7|12|3|2094|779|10150|Monday|2094Q3|N|N|N|2486060|2486240|2485706|2485980|N|N|N|N|N| +2486072|AAAAAAAAIDPOFCAA|2094-07-13|2334|10151|779|2094|2|7|13|3|2094|779|10151|Tuesday|2094Q3|N|N|N|2486060|2486240|2485707|2485981|N|N|N|N|N| +2486073|AAAAAAAAJDPOFCAA|2094-07-14|2334|10151|779|2094|3|7|14|3|2094|779|10151|Wednesday|2094Q3|N|N|N|2486060|2486240|2485708|2485982|N|N|N|N|N| +2486074|AAAAAAAAKDPOFCAA|2094-07-15|2334|10151|779|2094|4|7|15|3|2094|779|10151|Thursday|2094Q3|N|N|N|2486060|2486240|2485709|2485983|N|N|N|N|N| +2486075|AAAAAAAALDPOFCAA|2094-07-16|2334|10151|779|2094|5|7|16|3|2094|779|10151|Friday|2094Q3|N|Y|N|2486060|2486240|2485710|2485984|N|N|N|N|N| +2486076|AAAAAAAAMDPOFCAA|2094-07-17|2334|10151|779|2094|6|7|17|3|2094|779|10151|Saturday|2094Q3|N|Y|N|2486060|2486240|2485711|2485985|N|N|N|N|N| +2486077|AAAAAAAANDPOFCAA|2094-07-18|2334|10151|779|2094|0|7|18|3|2094|779|10151|Sunday|2094Q3|N|N|N|2486060|2486240|2485712|2485986|N|N|N|N|N| +2486078|AAAAAAAAODPOFCAA|2094-07-19|2334|10151|779|2094|1|7|19|3|2094|779|10151|Monday|2094Q3|N|N|N|2486060|2486240|2485713|2485987|N|N|N|N|N| +2486079|AAAAAAAAPDPOFCAA|2094-07-20|2334|10152|779|2094|2|7|20|3|2094|779|10152|Tuesday|2094Q3|N|N|N|2486060|2486240|2485714|2485988|N|N|N|N|N| +2486080|AAAAAAAAAEPOFCAA|2094-07-21|2334|10152|779|2094|3|7|21|3|2094|779|10152|Wednesday|2094Q3|N|N|N|2486060|2486240|2485715|2485989|N|N|N|N|N| +2486081|AAAAAAAABEPOFCAA|2094-07-22|2334|10152|779|2094|4|7|22|3|2094|779|10152|Thursday|2094Q3|N|N|N|2486060|2486240|2485716|2485990|N|N|N|N|N| +2486082|AAAAAAAACEPOFCAA|2094-07-23|2334|10152|779|2094|5|7|23|3|2094|779|10152|Friday|2094Q3|N|Y|N|2486060|2486240|2485717|2485991|N|N|N|N|N| +2486083|AAAAAAAADEPOFCAA|2094-07-24|2334|10152|779|2094|6|7|24|3|2094|779|10152|Saturday|2094Q3|N|Y|N|2486060|2486240|2485718|2485992|N|N|N|N|N| +2486084|AAAAAAAAEEPOFCAA|2094-07-25|2334|10152|779|2094|0|7|25|3|2094|779|10152|Sunday|2094Q3|N|N|N|2486060|2486240|2485719|2485993|N|N|N|N|N| +2486085|AAAAAAAAFEPOFCAA|2094-07-26|2334|10152|779|2094|1|7|26|3|2094|779|10152|Monday|2094Q3|N|N|N|2486060|2486240|2485720|2485994|N|N|N|N|N| +2486086|AAAAAAAAGEPOFCAA|2094-07-27|2334|10153|779|2094|2|7|27|3|2094|779|10153|Tuesday|2094Q3|N|N|N|2486060|2486240|2485721|2485995|N|N|N|N|N| +2486087|AAAAAAAAHEPOFCAA|2094-07-28|2334|10153|779|2094|3|7|28|3|2094|779|10153|Wednesday|2094Q3|N|N|N|2486060|2486240|2485722|2485996|N|N|N|N|N| +2486088|AAAAAAAAIEPOFCAA|2094-07-29|2334|10153|779|2094|4|7|29|3|2094|779|10153|Thursday|2094Q3|N|N|N|2486060|2486240|2485723|2485997|N|N|N|N|N| +2486089|AAAAAAAAJEPOFCAA|2094-07-30|2334|10153|779|2094|5|7|30|3|2094|779|10153|Friday|2094Q3|N|Y|N|2486060|2486240|2485724|2485998|N|N|N|N|N| +2486090|AAAAAAAAKEPOFCAA|2094-07-31|2334|10153|779|2094|6|7|31|3|2094|779|10153|Saturday|2094Q3|N|Y|N|2486060|2486240|2485725|2485999|N|N|N|N|N| +2486091|AAAAAAAALEPOFCAA|2094-08-01|2335|10153|779|2094|0|8|1|3|2094|779|10153|Sunday|2094Q3|N|N|N|2486091|2486302|2485726|2486000|N|N|N|N|N| +2486092|AAAAAAAAMEPOFCAA|2094-08-02|2335|10153|779|2094|1|8|2|3|2094|779|10153|Monday|2094Q3|N|N|N|2486091|2486302|2485727|2486001|N|N|N|N|N| +2486093|AAAAAAAANEPOFCAA|2094-08-03|2335|10154|779|2094|2|8|3|3|2094|779|10154|Tuesday|2094Q3|N|N|N|2486091|2486302|2485728|2486002|N|N|N|N|N| +2486094|AAAAAAAAOEPOFCAA|2094-08-04|2335|10154|779|2094|3|8|4|3|2094|779|10154|Wednesday|2094Q3|N|N|N|2486091|2486302|2485729|2486003|N|N|N|N|N| +2486095|AAAAAAAAPEPOFCAA|2094-08-05|2335|10154|779|2094|4|8|5|3|2094|779|10154|Thursday|2094Q3|N|N|N|2486091|2486302|2485730|2486004|N|N|N|N|N| +2486096|AAAAAAAAAFPOFCAA|2094-08-06|2335|10154|779|2094|5|8|6|3|2094|779|10154|Friday|2094Q3|N|Y|N|2486091|2486302|2485731|2486005|N|N|N|N|N| +2486097|AAAAAAAABFPOFCAA|2094-08-07|2335|10154|779|2094|6|8|7|3|2094|779|10154|Saturday|2094Q3|N|Y|N|2486091|2486302|2485732|2486006|N|N|N|N|N| +2486098|AAAAAAAACFPOFCAA|2094-08-08|2335|10154|779|2094|0|8|8|3|2094|779|10154|Sunday|2094Q3|N|N|N|2486091|2486302|2485733|2486007|N|N|N|N|N| +2486099|AAAAAAAADFPOFCAA|2094-08-09|2335|10154|779|2094|1|8|9|3|2094|779|10154|Monday|2094Q3|N|N|N|2486091|2486302|2485734|2486008|N|N|N|N|N| +2486100|AAAAAAAAEFPOFCAA|2094-08-10|2335|10155|779|2094|2|8|10|3|2094|779|10155|Tuesday|2094Q3|N|N|N|2486091|2486302|2485735|2486009|N|N|N|N|N| +2486101|AAAAAAAAFFPOFCAA|2094-08-11|2335|10155|779|2094|3|8|11|3|2094|779|10155|Wednesday|2094Q3|N|N|N|2486091|2486302|2485736|2486010|N|N|N|N|N| +2486102|AAAAAAAAGFPOFCAA|2094-08-12|2335|10155|779|2094|4|8|12|3|2094|779|10155|Thursday|2094Q3|N|N|N|2486091|2486302|2485737|2486011|N|N|N|N|N| +2486103|AAAAAAAAHFPOFCAA|2094-08-13|2335|10155|779|2094|5|8|13|3|2094|779|10155|Friday|2094Q3|N|Y|N|2486091|2486302|2485738|2486012|N|N|N|N|N| +2486104|AAAAAAAAIFPOFCAA|2094-08-14|2335|10155|779|2094|6|8|14|3|2094|779|10155|Saturday|2094Q3|N|Y|N|2486091|2486302|2485739|2486013|N|N|N|N|N| +2486105|AAAAAAAAJFPOFCAA|2094-08-15|2335|10155|779|2094|0|8|15|3|2094|779|10155|Sunday|2094Q3|N|N|N|2486091|2486302|2485740|2486014|N|N|N|N|N| +2486106|AAAAAAAAKFPOFCAA|2094-08-16|2335|10155|779|2094|1|8|16|3|2094|779|10155|Monday|2094Q3|N|N|N|2486091|2486302|2485741|2486015|N|N|N|N|N| +2486107|AAAAAAAALFPOFCAA|2094-08-17|2335|10156|779|2094|2|8|17|3|2094|779|10156|Tuesday|2094Q3|N|N|N|2486091|2486302|2485742|2486016|N|N|N|N|N| +2486108|AAAAAAAAMFPOFCAA|2094-08-18|2335|10156|779|2094|3|8|18|3|2094|779|10156|Wednesday|2094Q3|N|N|N|2486091|2486302|2485743|2486017|N|N|N|N|N| +2486109|AAAAAAAANFPOFCAA|2094-08-19|2335|10156|779|2094|4|8|19|3|2094|779|10156|Thursday|2094Q3|N|N|N|2486091|2486302|2485744|2486018|N|N|N|N|N| +2486110|AAAAAAAAOFPOFCAA|2094-08-20|2335|10156|779|2094|5|8|20|3|2094|779|10156|Friday|2094Q3|N|Y|N|2486091|2486302|2485745|2486019|N|N|N|N|N| +2486111|AAAAAAAAPFPOFCAA|2094-08-21|2335|10156|779|2094|6|8|21|3|2094|779|10156|Saturday|2094Q3|N|Y|N|2486091|2486302|2485746|2486020|N|N|N|N|N| +2486112|AAAAAAAAAGPOFCAA|2094-08-22|2335|10156|779|2094|0|8|22|3|2094|779|10156|Sunday|2094Q3|N|N|N|2486091|2486302|2485747|2486021|N|N|N|N|N| +2486113|AAAAAAAABGPOFCAA|2094-08-23|2335|10156|779|2094|1|8|23|3|2094|779|10156|Monday|2094Q3|N|N|N|2486091|2486302|2485748|2486022|N|N|N|N|N| +2486114|AAAAAAAACGPOFCAA|2094-08-24|2335|10157|779|2094|2|8|24|3|2094|779|10157|Tuesday|2094Q3|N|N|N|2486091|2486302|2485749|2486023|N|N|N|N|N| +2486115|AAAAAAAADGPOFCAA|2094-08-25|2335|10157|779|2094|3|8|25|3|2094|779|10157|Wednesday|2094Q3|N|N|N|2486091|2486302|2485750|2486024|N|N|N|N|N| +2486116|AAAAAAAAEGPOFCAA|2094-08-26|2335|10157|779|2094|4|8|26|3|2094|779|10157|Thursday|2094Q3|N|N|N|2486091|2486302|2485751|2486025|N|N|N|N|N| +2486117|AAAAAAAAFGPOFCAA|2094-08-27|2335|10157|779|2094|5|8|27|3|2094|779|10157|Friday|2094Q3|N|Y|N|2486091|2486302|2485752|2486026|N|N|N|N|N| +2486118|AAAAAAAAGGPOFCAA|2094-08-28|2335|10157|779|2094|6|8|28|3|2094|779|10157|Saturday|2094Q3|N|Y|N|2486091|2486302|2485753|2486027|N|N|N|N|N| +2486119|AAAAAAAAHGPOFCAA|2094-08-29|2335|10157|779|2094|0|8|29|3|2094|779|10157|Sunday|2094Q3|N|N|N|2486091|2486302|2485754|2486028|N|N|N|N|N| +2486120|AAAAAAAAIGPOFCAA|2094-08-30|2335|10157|779|2094|1|8|30|3|2094|779|10157|Monday|2094Q3|N|N|N|2486091|2486302|2485755|2486029|N|N|N|N|N| +2486121|AAAAAAAAJGPOFCAA|2094-08-31|2335|10158|779|2094|2|8|31|3|2094|779|10158|Tuesday|2094Q3|N|N|N|2486091|2486302|2485756|2486030|N|N|N|N|N| +2486122|AAAAAAAAKGPOFCAA|2094-09-01|2336|10158|780|2094|3|9|1|3|2094|780|10158|Wednesday|2094Q3|N|N|N|2486122|2486364|2485757|2486031|N|N|N|N|N| +2486123|AAAAAAAALGPOFCAA|2094-09-02|2336|10158|780|2094|4|9|2|3|2094|780|10158|Thursday|2094Q3|N|N|N|2486122|2486364|2485758|2486032|N|N|N|N|N| +2486124|AAAAAAAAMGPOFCAA|2094-09-03|2336|10158|780|2094|5|9|3|3|2094|780|10158|Friday|2094Q3|N|Y|N|2486122|2486364|2485759|2486033|N|N|N|N|N| +2486125|AAAAAAAANGPOFCAA|2094-09-04|2336|10158|780|2094|6|9|4|3|2094|780|10158|Saturday|2094Q3|N|Y|N|2486122|2486364|2485760|2486034|N|N|N|N|N| +2486126|AAAAAAAAOGPOFCAA|2094-09-05|2336|10158|780|2094|0|9|5|3|2094|780|10158|Sunday|2094Q3|N|N|N|2486122|2486364|2485761|2486035|N|N|N|N|N| +2486127|AAAAAAAAPGPOFCAA|2094-09-06|2336|10158|780|2094|1|9|6|3|2094|780|10158|Monday|2094Q3|N|N|N|2486122|2486364|2485762|2486036|N|N|N|N|N| +2486128|AAAAAAAAAHPOFCAA|2094-09-07|2336|10159|780|2094|2|9|7|3|2094|780|10159|Tuesday|2094Q3|N|N|N|2486122|2486364|2485763|2486037|N|N|N|N|N| +2486129|AAAAAAAABHPOFCAA|2094-09-08|2336|10159|780|2094|3|9|8|3|2094|780|10159|Wednesday|2094Q3|N|N|N|2486122|2486364|2485764|2486038|N|N|N|N|N| +2486130|AAAAAAAACHPOFCAA|2094-09-09|2336|10159|780|2094|4|9|9|3|2094|780|10159|Thursday|2094Q3|N|N|N|2486122|2486364|2485765|2486039|N|N|N|N|N| +2486131|AAAAAAAADHPOFCAA|2094-09-10|2336|10159|780|2094|5|9|10|3|2094|780|10159|Friday|2094Q3|N|Y|N|2486122|2486364|2485766|2486040|N|N|N|N|N| +2486132|AAAAAAAAEHPOFCAA|2094-09-11|2336|10159|780|2094|6|9|11|3|2094|780|10159|Saturday|2094Q3|N|Y|N|2486122|2486364|2485767|2486041|N|N|N|N|N| +2486133|AAAAAAAAFHPOFCAA|2094-09-12|2336|10159|780|2094|0|9|12|3|2094|780|10159|Sunday|2094Q3|N|N|N|2486122|2486364|2485768|2486042|N|N|N|N|N| +2486134|AAAAAAAAGHPOFCAA|2094-09-13|2336|10159|780|2094|1|9|13|3|2094|780|10159|Monday|2094Q3|N|N|N|2486122|2486364|2485769|2486043|N|N|N|N|N| +2486135|AAAAAAAAHHPOFCAA|2094-09-14|2336|10160|780|2094|2|9|14|3|2094|780|10160|Tuesday|2094Q3|N|N|N|2486122|2486364|2485770|2486044|N|N|N|N|N| +2486136|AAAAAAAAIHPOFCAA|2094-09-15|2336|10160|780|2094|3|9|15|3|2094|780|10160|Wednesday|2094Q3|N|N|N|2486122|2486364|2485771|2486045|N|N|N|N|N| +2486137|AAAAAAAAJHPOFCAA|2094-09-16|2336|10160|780|2094|4|9|16|3|2094|780|10160|Thursday|2094Q3|N|N|N|2486122|2486364|2485772|2486046|N|N|N|N|N| +2486138|AAAAAAAAKHPOFCAA|2094-09-17|2336|10160|780|2094|5|9|17|3|2094|780|10160|Friday|2094Q3|N|Y|N|2486122|2486364|2485773|2486047|N|N|N|N|N| +2486139|AAAAAAAALHPOFCAA|2094-09-18|2336|10160|780|2094|6|9|18|3|2094|780|10160|Saturday|2094Q3|N|Y|N|2486122|2486364|2485774|2486048|N|N|N|N|N| +2486140|AAAAAAAAMHPOFCAA|2094-09-19|2336|10160|780|2094|0|9|19|3|2094|780|10160|Sunday|2094Q3|N|N|N|2486122|2486364|2485775|2486049|N|N|N|N|N| +2486141|AAAAAAAANHPOFCAA|2094-09-20|2336|10160|780|2094|1|9|20|3|2094|780|10160|Monday|2094Q3|N|N|N|2486122|2486364|2485776|2486050|N|N|N|N|N| +2486142|AAAAAAAAOHPOFCAA|2094-09-21|2336|10161|780|2094|2|9|21|3|2094|780|10161|Tuesday|2094Q3|N|N|N|2486122|2486364|2485777|2486051|N|N|N|N|N| +2486143|AAAAAAAAPHPOFCAA|2094-09-22|2336|10161|780|2094|3|9|22|3|2094|780|10161|Wednesday|2094Q3|N|N|N|2486122|2486364|2485778|2486052|N|N|N|N|N| +2486144|AAAAAAAAAIPOFCAA|2094-09-23|2336|10161|780|2094|4|9|23|3|2094|780|10161|Thursday|2094Q3|N|N|N|2486122|2486364|2485779|2486053|N|N|N|N|N| +2486145|AAAAAAAABIPOFCAA|2094-09-24|2336|10161|780|2094|5|9|24|3|2094|780|10161|Friday|2094Q3|N|Y|N|2486122|2486364|2485780|2486054|N|N|N|N|N| +2486146|AAAAAAAACIPOFCAA|2094-09-25|2336|10161|780|2094|6|9|25|3|2094|780|10161|Saturday|2094Q3|N|Y|N|2486122|2486364|2485781|2486055|N|N|N|N|N| +2486147|AAAAAAAADIPOFCAA|2094-09-26|2336|10161|780|2094|0|9|26|3|2094|780|10161|Sunday|2094Q3|N|N|N|2486122|2486364|2485782|2486056|N|N|N|N|N| +2486148|AAAAAAAAEIPOFCAA|2094-09-27|2336|10161|780|2094|1|9|27|3|2094|780|10161|Monday|2094Q3|N|N|N|2486122|2486364|2485783|2486057|N|N|N|N|N| +2486149|AAAAAAAAFIPOFCAA|2094-09-28|2336|10162|780|2094|2|9|28|3|2094|780|10162|Tuesday|2094Q3|N|N|N|2486122|2486364|2485784|2486058|N|N|N|N|N| +2486150|AAAAAAAAGIPOFCAA|2094-09-29|2336|10162|780|2094|3|9|29|3|2094|780|10162|Wednesday|2094Q3|N|N|N|2486122|2486364|2485785|2486059|N|N|N|N|N| +2486151|AAAAAAAAHIPOFCAA|2094-09-30|2336|10162|780|2094|4|9|30|3|2094|780|10162|Thursday|2094Q3|N|N|N|2486122|2486364|2485786|2486060|N|N|N|N|N| +2486152|AAAAAAAAIIPOFCAA|2094-10-01|2337|10162|780|2094|5|10|1|3|2094|780|10162|Friday|2094Q3|N|Y|N|2486152|2486424|2485787|2486060|N|N|N|N|N| +2486153|AAAAAAAAJIPOFCAA|2094-10-02|2337|10162|780|2094|6|10|2|4|2094|780|10162|Saturday|2094Q4|N|Y|N|2486152|2486424|2485788|2486061|N|N|N|N|N| +2486154|AAAAAAAAKIPOFCAA|2094-10-03|2337|10162|780|2094|0|10|3|4|2094|780|10162|Sunday|2094Q4|N|N|N|2486152|2486424|2485789|2486062|N|N|N|N|N| +2486155|AAAAAAAALIPOFCAA|2094-10-04|2337|10162|780|2094|1|10|4|4|2094|780|10162|Monday|2094Q4|N|N|N|2486152|2486424|2485790|2486063|N|N|N|N|N| +2486156|AAAAAAAAMIPOFCAA|2094-10-05|2337|10163|780|2094|2|10|5|4|2094|780|10163|Tuesday|2094Q4|N|N|N|2486152|2486424|2485791|2486064|N|N|N|N|N| +2486157|AAAAAAAANIPOFCAA|2094-10-06|2337|10163|780|2094|3|10|6|4|2094|780|10163|Wednesday|2094Q4|N|N|N|2486152|2486424|2485792|2486065|N|N|N|N|N| +2486158|AAAAAAAAOIPOFCAA|2094-10-07|2337|10163|780|2094|4|10|7|4|2094|780|10163|Thursday|2094Q4|N|N|N|2486152|2486424|2485793|2486066|N|N|N|N|N| +2486159|AAAAAAAAPIPOFCAA|2094-10-08|2337|10163|780|2094|5|10|8|4|2094|780|10163|Friday|2094Q4|N|Y|N|2486152|2486424|2485794|2486067|N|N|N|N|N| +2486160|AAAAAAAAAJPOFCAA|2094-10-09|2337|10163|780|2094|6|10|9|4|2094|780|10163|Saturday|2094Q4|N|Y|N|2486152|2486424|2485795|2486068|N|N|N|N|N| +2486161|AAAAAAAABJPOFCAA|2094-10-10|2337|10163|780|2094|0|10|10|4|2094|780|10163|Sunday|2094Q4|N|N|N|2486152|2486424|2485796|2486069|N|N|N|N|N| +2486162|AAAAAAAACJPOFCAA|2094-10-11|2337|10163|780|2094|1|10|11|4|2094|780|10163|Monday|2094Q4|N|N|N|2486152|2486424|2485797|2486070|N|N|N|N|N| +2486163|AAAAAAAADJPOFCAA|2094-10-12|2337|10164|780|2094|2|10|12|4|2094|780|10164|Tuesday|2094Q4|N|N|N|2486152|2486424|2485798|2486071|N|N|N|N|N| +2486164|AAAAAAAAEJPOFCAA|2094-10-13|2337|10164|780|2094|3|10|13|4|2094|780|10164|Wednesday|2094Q4|N|N|N|2486152|2486424|2485799|2486072|N|N|N|N|N| +2486165|AAAAAAAAFJPOFCAA|2094-10-14|2337|10164|780|2094|4|10|14|4|2094|780|10164|Thursday|2094Q4|N|N|N|2486152|2486424|2485800|2486073|N|N|N|N|N| +2486166|AAAAAAAAGJPOFCAA|2094-10-15|2337|10164|780|2094|5|10|15|4|2094|780|10164|Friday|2094Q4|N|Y|N|2486152|2486424|2485801|2486074|N|N|N|N|N| +2486167|AAAAAAAAHJPOFCAA|2094-10-16|2337|10164|780|2094|6|10|16|4|2094|780|10164|Saturday|2094Q4|N|Y|N|2486152|2486424|2485802|2486075|N|N|N|N|N| +2486168|AAAAAAAAIJPOFCAA|2094-10-17|2337|10164|780|2094|0|10|17|4|2094|780|10164|Sunday|2094Q4|N|N|N|2486152|2486424|2485803|2486076|N|N|N|N|N| +2486169|AAAAAAAAJJPOFCAA|2094-10-18|2337|10164|780|2094|1|10|18|4|2094|780|10164|Monday|2094Q4|N|N|N|2486152|2486424|2485804|2486077|N|N|N|N|N| +2486170|AAAAAAAAKJPOFCAA|2094-10-19|2337|10165|780|2094|2|10|19|4|2094|780|10165|Tuesday|2094Q4|N|N|N|2486152|2486424|2485805|2486078|N|N|N|N|N| +2486171|AAAAAAAALJPOFCAA|2094-10-20|2337|10165|780|2094|3|10|20|4|2094|780|10165|Wednesday|2094Q4|N|N|N|2486152|2486424|2485806|2486079|N|N|N|N|N| +2486172|AAAAAAAAMJPOFCAA|2094-10-21|2337|10165|780|2094|4|10|21|4|2094|780|10165|Thursday|2094Q4|N|N|N|2486152|2486424|2485807|2486080|N|N|N|N|N| +2486173|AAAAAAAANJPOFCAA|2094-10-22|2337|10165|780|2094|5|10|22|4|2094|780|10165|Friday|2094Q4|N|Y|N|2486152|2486424|2485808|2486081|N|N|N|N|N| +2486174|AAAAAAAAOJPOFCAA|2094-10-23|2337|10165|780|2094|6|10|23|4|2094|780|10165|Saturday|2094Q4|N|Y|N|2486152|2486424|2485809|2486082|N|N|N|N|N| +2486175|AAAAAAAAPJPOFCAA|2094-10-24|2337|10165|780|2094|0|10|24|4|2094|780|10165|Sunday|2094Q4|N|N|N|2486152|2486424|2485810|2486083|N|N|N|N|N| +2486176|AAAAAAAAAKPOFCAA|2094-10-25|2337|10165|780|2094|1|10|25|4|2094|780|10165|Monday|2094Q4|N|N|N|2486152|2486424|2485811|2486084|N|N|N|N|N| +2486177|AAAAAAAABKPOFCAA|2094-10-26|2337|10166|780|2094|2|10|26|4|2094|780|10166|Tuesday|2094Q4|N|N|N|2486152|2486424|2485812|2486085|N|N|N|N|N| +2486178|AAAAAAAACKPOFCAA|2094-10-27|2337|10166|780|2094|3|10|27|4|2094|780|10166|Wednesday|2094Q4|N|N|N|2486152|2486424|2485813|2486086|N|N|N|N|N| +2486179|AAAAAAAADKPOFCAA|2094-10-28|2337|10166|780|2094|4|10|28|4|2094|780|10166|Thursday|2094Q4|N|N|N|2486152|2486424|2485814|2486087|N|N|N|N|N| +2486180|AAAAAAAAEKPOFCAA|2094-10-29|2337|10166|780|2094|5|10|29|4|2094|780|10166|Friday|2094Q4|N|Y|N|2486152|2486424|2485815|2486088|N|N|N|N|N| +2486181|AAAAAAAAFKPOFCAA|2094-10-30|2337|10166|780|2094|6|10|30|4|2094|780|10166|Saturday|2094Q4|N|Y|N|2486152|2486424|2485816|2486089|N|N|N|N|N| +2486182|AAAAAAAAGKPOFCAA|2094-10-31|2337|10166|780|2094|0|10|31|4|2094|780|10166|Sunday|2094Q4|N|N|N|2486152|2486424|2485817|2486090|N|N|N|N|N| +2486183|AAAAAAAAHKPOFCAA|2094-11-01|2338|10166|780|2094|1|11|1|4|2094|780|10166|Monday|2094Q4|N|N|N|2486183|2486486|2485818|2486091|N|N|N|N|N| +2486184|AAAAAAAAIKPOFCAA|2094-11-02|2338|10167|780|2094|2|11|2|4|2094|780|10167|Tuesday|2094Q4|N|N|N|2486183|2486486|2485819|2486092|N|N|N|N|N| +2486185|AAAAAAAAJKPOFCAA|2094-11-03|2338|10167|780|2094|3|11|3|4|2094|780|10167|Wednesday|2094Q4|N|N|N|2486183|2486486|2485820|2486093|N|N|N|N|N| +2486186|AAAAAAAAKKPOFCAA|2094-11-04|2338|10167|780|2094|4|11|4|4|2094|780|10167|Thursday|2094Q4|N|N|N|2486183|2486486|2485821|2486094|N|N|N|N|N| +2486187|AAAAAAAALKPOFCAA|2094-11-05|2338|10167|780|2094|5|11|5|4|2094|780|10167|Friday|2094Q4|N|Y|N|2486183|2486486|2485822|2486095|N|N|N|N|N| +2486188|AAAAAAAAMKPOFCAA|2094-11-06|2338|10167|780|2094|6|11|6|4|2094|780|10167|Saturday|2094Q4|N|Y|N|2486183|2486486|2485823|2486096|N|N|N|N|N| +2486189|AAAAAAAANKPOFCAA|2094-11-07|2338|10167|780|2094|0|11|7|4|2094|780|10167|Sunday|2094Q4|N|N|N|2486183|2486486|2485824|2486097|N|N|N|N|N| +2486190|AAAAAAAAOKPOFCAA|2094-11-08|2338|10167|780|2094|1|11|8|4|2094|780|10167|Monday|2094Q4|N|N|N|2486183|2486486|2485825|2486098|N|N|N|N|N| +2486191|AAAAAAAAPKPOFCAA|2094-11-09|2338|10168|780|2094|2|11|9|4|2094|780|10168|Tuesday|2094Q4|N|N|N|2486183|2486486|2485826|2486099|N|N|N|N|N| +2486192|AAAAAAAAALPOFCAA|2094-11-10|2338|10168|780|2094|3|11|10|4|2094|780|10168|Wednesday|2094Q4|N|N|N|2486183|2486486|2485827|2486100|N|N|N|N|N| +2486193|AAAAAAAABLPOFCAA|2094-11-11|2338|10168|780|2094|4|11|11|4|2094|780|10168|Thursday|2094Q4|N|N|N|2486183|2486486|2485828|2486101|N|N|N|N|N| +2486194|AAAAAAAACLPOFCAA|2094-11-12|2338|10168|780|2094|5|11|12|4|2094|780|10168|Friday|2094Q4|N|Y|N|2486183|2486486|2485829|2486102|N|N|N|N|N| +2486195|AAAAAAAADLPOFCAA|2094-11-13|2338|10168|780|2094|6|11|13|4|2094|780|10168|Saturday|2094Q4|N|Y|N|2486183|2486486|2485830|2486103|N|N|N|N|N| +2486196|AAAAAAAAELPOFCAA|2094-11-14|2338|10168|780|2094|0|11|14|4|2094|780|10168|Sunday|2094Q4|N|N|N|2486183|2486486|2485831|2486104|N|N|N|N|N| +2486197|AAAAAAAAFLPOFCAA|2094-11-15|2338|10168|780|2094|1|11|15|4|2094|780|10168|Monday|2094Q4|N|N|N|2486183|2486486|2485832|2486105|N|N|N|N|N| +2486198|AAAAAAAAGLPOFCAA|2094-11-16|2338|10169|780|2094|2|11|16|4|2094|780|10169|Tuesday|2094Q4|N|N|N|2486183|2486486|2485833|2486106|N|N|N|N|N| +2486199|AAAAAAAAHLPOFCAA|2094-11-17|2338|10169|780|2094|3|11|17|4|2094|780|10169|Wednesday|2094Q4|N|N|N|2486183|2486486|2485834|2486107|N|N|N|N|N| +2486200|AAAAAAAAILPOFCAA|2094-11-18|2338|10169|780|2094|4|11|18|4|2094|780|10169|Thursday|2094Q4|N|N|N|2486183|2486486|2485835|2486108|N|N|N|N|N| +2486201|AAAAAAAAJLPOFCAA|2094-11-19|2338|10169|780|2094|5|11|19|4|2094|780|10169|Friday|2094Q4|N|Y|N|2486183|2486486|2485836|2486109|N|N|N|N|N| +2486202|AAAAAAAAKLPOFCAA|2094-11-20|2338|10169|780|2094|6|11|20|4|2094|780|10169|Saturday|2094Q4|N|Y|N|2486183|2486486|2485837|2486110|N|N|N|N|N| +2486203|AAAAAAAALLPOFCAA|2094-11-21|2338|10169|780|2094|0|11|21|4|2094|780|10169|Sunday|2094Q4|N|N|N|2486183|2486486|2485838|2486111|N|N|N|N|N| +2486204|AAAAAAAAMLPOFCAA|2094-11-22|2338|10169|780|2094|1|11|22|4|2094|780|10169|Monday|2094Q4|N|N|N|2486183|2486486|2485839|2486112|N|N|N|N|N| +2486205|AAAAAAAANLPOFCAA|2094-11-23|2338|10170|780|2094|2|11|23|4|2094|780|10170|Tuesday|2094Q4|N|N|N|2486183|2486486|2485840|2486113|N|N|N|N|N| +2486206|AAAAAAAAOLPOFCAA|2094-11-24|2338|10170|780|2094|3|11|24|4|2094|780|10170|Wednesday|2094Q4|N|N|N|2486183|2486486|2485841|2486114|N|N|N|N|N| +2486207|AAAAAAAAPLPOFCAA|2094-11-25|2338|10170|780|2094|4|11|25|4|2094|780|10170|Thursday|2094Q4|N|N|N|2486183|2486486|2485842|2486115|N|N|N|N|N| +2486208|AAAAAAAAAMPOFCAA|2094-11-26|2338|10170|780|2094|5|11|26|4|2094|780|10170|Friday|2094Q4|N|Y|N|2486183|2486486|2485843|2486116|N|N|N|N|N| +2486209|AAAAAAAABMPOFCAA|2094-11-27|2338|10170|780|2094|6|11|27|4|2094|780|10170|Saturday|2094Q4|N|Y|N|2486183|2486486|2485844|2486117|N|N|N|N|N| +2486210|AAAAAAAACMPOFCAA|2094-11-28|2338|10170|780|2094|0|11|28|4|2094|780|10170|Sunday|2094Q4|N|N|N|2486183|2486486|2485845|2486118|N|N|N|N|N| +2486211|AAAAAAAADMPOFCAA|2094-11-29|2338|10170|780|2094|1|11|29|4|2094|780|10170|Monday|2094Q4|N|N|N|2486183|2486486|2485846|2486119|N|N|N|N|N| +2486212|AAAAAAAAEMPOFCAA|2094-11-30|2338|10171|780|2094|2|11|30|4|2094|780|10171|Tuesday|2094Q4|N|N|N|2486183|2486486|2485847|2486120|N|N|N|N|N| +2486213|AAAAAAAAFMPOFCAA|2094-12-01|2339|10171|781|2094|3|12|1|4|2094|781|10171|Wednesday|2094Q4|N|N|N|2486213|2486546|2485848|2486121|N|N|N|N|N| +2486214|AAAAAAAAGMPOFCAA|2094-12-02|2339|10171|781|2094|4|12|2|4|2094|781|10171|Thursday|2094Q4|N|N|N|2486213|2486546|2485849|2486122|N|N|N|N|N| +2486215|AAAAAAAAHMPOFCAA|2094-12-03|2339|10171|781|2094|5|12|3|4|2094|781|10171|Friday|2094Q4|N|Y|N|2486213|2486546|2485850|2486123|N|N|N|N|N| +2486216|AAAAAAAAIMPOFCAA|2094-12-04|2339|10171|781|2094|6|12|4|4|2094|781|10171|Saturday|2094Q4|N|Y|N|2486213|2486546|2485851|2486124|N|N|N|N|N| +2486217|AAAAAAAAJMPOFCAA|2094-12-05|2339|10171|781|2094|0|12|5|4|2094|781|10171|Sunday|2094Q4|N|N|N|2486213|2486546|2485852|2486125|N|N|N|N|N| +2486218|AAAAAAAAKMPOFCAA|2094-12-06|2339|10171|781|2094|1|12|6|4|2094|781|10171|Monday|2094Q4|N|N|N|2486213|2486546|2485853|2486126|N|N|N|N|N| +2486219|AAAAAAAALMPOFCAA|2094-12-07|2339|10172|781|2094|2|12|7|4|2094|781|10172|Tuesday|2094Q4|N|N|N|2486213|2486546|2485854|2486127|N|N|N|N|N| +2486220|AAAAAAAAMMPOFCAA|2094-12-08|2339|10172|781|2094|3|12|8|4|2094|781|10172|Wednesday|2094Q4|N|N|N|2486213|2486546|2485855|2486128|N|N|N|N|N| +2486221|AAAAAAAANMPOFCAA|2094-12-09|2339|10172|781|2094|4|12|9|4|2094|781|10172|Thursday|2094Q4|N|N|N|2486213|2486546|2485856|2486129|N|N|N|N|N| +2486222|AAAAAAAAOMPOFCAA|2094-12-10|2339|10172|781|2094|5|12|10|4|2094|781|10172|Friday|2094Q4|N|Y|N|2486213|2486546|2485857|2486130|N|N|N|N|N| +2486223|AAAAAAAAPMPOFCAA|2094-12-11|2339|10172|781|2094|6|12|11|4|2094|781|10172|Saturday|2094Q4|N|Y|N|2486213|2486546|2485858|2486131|N|N|N|N|N| +2486224|AAAAAAAAANPOFCAA|2094-12-12|2339|10172|781|2094|0|12|12|4|2094|781|10172|Sunday|2094Q4|N|N|N|2486213|2486546|2485859|2486132|N|N|N|N|N| +2486225|AAAAAAAABNPOFCAA|2094-12-13|2339|10172|781|2094|1|12|13|4|2094|781|10172|Monday|2094Q4|N|N|N|2486213|2486546|2485860|2486133|N|N|N|N|N| +2486226|AAAAAAAACNPOFCAA|2094-12-14|2339|10173|781|2094|2|12|14|4|2094|781|10173|Tuesday|2094Q4|N|N|N|2486213|2486546|2485861|2486134|N|N|N|N|N| +2486227|AAAAAAAADNPOFCAA|2094-12-15|2339|10173|781|2094|3|12|15|4|2094|781|10173|Wednesday|2094Q4|N|N|N|2486213|2486546|2485862|2486135|N|N|N|N|N| +2486228|AAAAAAAAENPOFCAA|2094-12-16|2339|10173|781|2094|4|12|16|4|2094|781|10173|Thursday|2094Q4|N|N|N|2486213|2486546|2485863|2486136|N|N|N|N|N| +2486229|AAAAAAAAFNPOFCAA|2094-12-17|2339|10173|781|2094|5|12|17|4|2094|781|10173|Friday|2094Q4|N|Y|N|2486213|2486546|2485864|2486137|N|N|N|N|N| +2486230|AAAAAAAAGNPOFCAA|2094-12-18|2339|10173|781|2094|6|12|18|4|2094|781|10173|Saturday|2094Q4|N|Y|N|2486213|2486546|2485865|2486138|N|N|N|N|N| +2486231|AAAAAAAAHNPOFCAA|2094-12-19|2339|10173|781|2094|0|12|19|4|2094|781|10173|Sunday|2094Q4|N|N|N|2486213|2486546|2485866|2486139|N|N|N|N|N| +2486232|AAAAAAAAINPOFCAA|2094-12-20|2339|10173|781|2094|1|12|20|4|2094|781|10173|Monday|2094Q4|N|N|N|2486213|2486546|2485867|2486140|N|N|N|N|N| +2486233|AAAAAAAAJNPOFCAA|2094-12-21|2339|10174|781|2094|2|12|21|4|2094|781|10174|Tuesday|2094Q4|N|N|N|2486213|2486546|2485868|2486141|N|N|N|N|N| +2486234|AAAAAAAAKNPOFCAA|2094-12-22|2339|10174|781|2094|3|12|22|4|2094|781|10174|Wednesday|2094Q4|N|N|N|2486213|2486546|2485869|2486142|N|N|N|N|N| +2486235|AAAAAAAALNPOFCAA|2094-12-23|2339|10174|781|2094|4|12|23|4|2094|781|10174|Thursday|2094Q4|N|N|N|2486213|2486546|2485870|2486143|N|N|N|N|N| +2486236|AAAAAAAAMNPOFCAA|2094-12-24|2339|10174|781|2094|5|12|24|4|2094|781|10174|Friday|2094Q4|N|Y|N|2486213|2486546|2485871|2486144|N|N|N|N|N| +2486237|AAAAAAAANNPOFCAA|2094-12-25|2339|10174|781|2094|6|12|25|4|2094|781|10174|Saturday|2094Q4|N|Y|N|2486213|2486546|2485872|2486145|N|N|N|N|N| +2486238|AAAAAAAAONPOFCAA|2094-12-26|2339|10174|781|2094|0|12|26|4|2094|781|10174|Sunday|2094Q4|Y|N|N|2486213|2486546|2485873|2486146|N|N|N|N|N| +2486239|AAAAAAAAPNPOFCAA|2094-12-27|2339|10174|781|2094|1|12|27|4|2094|781|10174|Monday|2094Q4|N|N|Y|2486213|2486546|2485874|2486147|N|N|N|N|N| +2486240|AAAAAAAAAOPOFCAA|2094-12-28|2339|10175|781|2094|2|12|28|4|2094|781|10175|Tuesday|2094Q4|N|N|N|2486213|2486546|2485875|2486148|N|N|N|N|N| +2486241|AAAAAAAABOPOFCAA|2094-12-29|2339|10175|781|2094|3|12|29|4|2094|781|10175|Wednesday|2094Q4|N|N|N|2486213|2486546|2485876|2486149|N|N|N|N|N| +2486242|AAAAAAAACOPOFCAA|2094-12-30|2339|10175|781|2094|4|12|30|4|2094|781|10175|Thursday|2094Q4|N|N|N|2486213|2486546|2485877|2486150|N|N|N|N|N| +2486243|AAAAAAAADOPOFCAA|2094-12-31|2339|10175|781|2094|5|12|31|4|2094|781|10175|Friday|2094Q4|N|Y|N|2486213|2486546|2485878|2486151|N|N|N|N|N| +2486244|AAAAAAAAEOPOFCAA|2095-01-01|2340|10175|781|2095|6|1|1|1|2095|781|10175|Saturday|2095Q1|Y|Y|N|2486244|2486243|2485879|2486152|N|N|N|N|N| +2486245|AAAAAAAAFOPOFCAA|2095-01-02|2340|10175|781|2095|0|1|2|1|2095|781|10175|Sunday|2095Q1|N|N|Y|2486244|2486243|2485880|2486153|N|N|N|N|N| +2486246|AAAAAAAAGOPOFCAA|2095-01-03|2340|10175|781|2095|1|1|3|1|2095|781|10175|Monday|2095Q1|N|N|N|2486244|2486243|2485881|2486154|N|N|N|N|N| +2486247|AAAAAAAAHOPOFCAA|2095-01-04|2340|10176|781|2095|2|1|4|1|2095|781|10176|Tuesday|2095Q1|N|N|N|2486244|2486243|2485882|2486155|N|N|N|N|N| +2486248|AAAAAAAAIOPOFCAA|2095-01-05|2340|10176|781|2095|3|1|5|1|2095|781|10176|Wednesday|2095Q1|N|N|N|2486244|2486243|2485883|2486156|N|N|N|N|N| +2486249|AAAAAAAAJOPOFCAA|2095-01-06|2340|10176|781|2095|4|1|6|1|2095|781|10176|Thursday|2095Q1|N|N|N|2486244|2486243|2485884|2486157|N|N|N|N|N| +2486250|AAAAAAAAKOPOFCAA|2095-01-07|2340|10176|781|2095|5|1|7|1|2095|781|10176|Friday|2095Q1|N|Y|N|2486244|2486243|2485885|2486158|N|N|N|N|N| +2486251|AAAAAAAALOPOFCAA|2095-01-08|2340|10176|781|2095|6|1|8|1|2095|781|10176|Saturday|2095Q1|N|Y|N|2486244|2486243|2485886|2486159|N|N|N|N|N| +2486252|AAAAAAAAMOPOFCAA|2095-01-09|2340|10176|781|2095|0|1|9|1|2095|781|10176|Sunday|2095Q1|N|N|N|2486244|2486243|2485887|2486160|N|N|N|N|N| +2486253|AAAAAAAANOPOFCAA|2095-01-10|2340|10176|781|2095|1|1|10|1|2095|781|10176|Monday|2095Q1|N|N|N|2486244|2486243|2485888|2486161|N|N|N|N|N| +2486254|AAAAAAAAOOPOFCAA|2095-01-11|2340|10177|781|2095|2|1|11|1|2095|781|10177|Tuesday|2095Q1|N|N|N|2486244|2486243|2485889|2486162|N|N|N|N|N| +2486255|AAAAAAAAPOPOFCAA|2095-01-12|2340|10177|781|2095|3|1|12|1|2095|781|10177|Wednesday|2095Q1|N|N|N|2486244|2486243|2485890|2486163|N|N|N|N|N| +2486256|AAAAAAAAAPPOFCAA|2095-01-13|2340|10177|781|2095|4|1|13|1|2095|781|10177|Thursday|2095Q1|N|N|N|2486244|2486243|2485891|2486164|N|N|N|N|N| +2486257|AAAAAAAABPPOFCAA|2095-01-14|2340|10177|781|2095|5|1|14|1|2095|781|10177|Friday|2095Q1|N|Y|N|2486244|2486243|2485892|2486165|N|N|N|N|N| +2486258|AAAAAAAACPPOFCAA|2095-01-15|2340|10177|781|2095|6|1|15|1|2095|781|10177|Saturday|2095Q1|N|Y|N|2486244|2486243|2485893|2486166|N|N|N|N|N| +2486259|AAAAAAAADPPOFCAA|2095-01-16|2340|10177|781|2095|0|1|16|1|2095|781|10177|Sunday|2095Q1|N|N|N|2486244|2486243|2485894|2486167|N|N|N|N|N| +2486260|AAAAAAAAEPPOFCAA|2095-01-17|2340|10177|781|2095|1|1|17|1|2095|781|10177|Monday|2095Q1|N|N|N|2486244|2486243|2485895|2486168|N|N|N|N|N| +2486261|AAAAAAAAFPPOFCAA|2095-01-18|2340|10178|781|2095|2|1|18|1|2095|781|10178|Tuesday|2095Q1|N|N|N|2486244|2486243|2485896|2486169|N|N|N|N|N| +2486262|AAAAAAAAGPPOFCAA|2095-01-19|2340|10178|781|2095|3|1|19|1|2095|781|10178|Wednesday|2095Q1|N|N|N|2486244|2486243|2485897|2486170|N|N|N|N|N| +2486263|AAAAAAAAHPPOFCAA|2095-01-20|2340|10178|781|2095|4|1|20|1|2095|781|10178|Thursday|2095Q1|N|N|N|2486244|2486243|2485898|2486171|N|N|N|N|N| +2486264|AAAAAAAAIPPOFCAA|2095-01-21|2340|10178|781|2095|5|1|21|1|2095|781|10178|Friday|2095Q1|N|Y|N|2486244|2486243|2485899|2486172|N|N|N|N|N| +2486265|AAAAAAAAJPPOFCAA|2095-01-22|2340|10178|781|2095|6|1|22|1|2095|781|10178|Saturday|2095Q1|N|Y|N|2486244|2486243|2485900|2486173|N|N|N|N|N| +2486266|AAAAAAAAKPPOFCAA|2095-01-23|2340|10178|781|2095|0|1|23|1|2095|781|10178|Sunday|2095Q1|N|N|N|2486244|2486243|2485901|2486174|N|N|N|N|N| +2486267|AAAAAAAALPPOFCAA|2095-01-24|2340|10178|781|2095|1|1|24|1|2095|781|10178|Monday|2095Q1|N|N|N|2486244|2486243|2485902|2486175|N|N|N|N|N| +2486268|AAAAAAAAMPPOFCAA|2095-01-25|2340|10179|781|2095|2|1|25|1|2095|781|10179|Tuesday|2095Q1|N|N|N|2486244|2486243|2485903|2486176|N|N|N|N|N| +2486269|AAAAAAAANPPOFCAA|2095-01-26|2340|10179|781|2095|3|1|26|1|2095|781|10179|Wednesday|2095Q1|N|N|N|2486244|2486243|2485904|2486177|N|N|N|N|N| +2486270|AAAAAAAAOPPOFCAA|2095-01-27|2340|10179|781|2095|4|1|27|1|2095|781|10179|Thursday|2095Q1|N|N|N|2486244|2486243|2485905|2486178|N|N|N|N|N| +2486271|AAAAAAAAPPPOFCAA|2095-01-28|2340|10179|781|2095|5|1|28|1|2095|781|10179|Friday|2095Q1|N|Y|N|2486244|2486243|2485906|2486179|N|N|N|N|N| +2486272|AAAAAAAAAAAPFCAA|2095-01-29|2340|10179|781|2095|6|1|29|1|2095|781|10179|Saturday|2095Q1|N|Y|N|2486244|2486243|2485907|2486180|N|N|N|N|N| +2486273|AAAAAAAABAAPFCAA|2095-01-30|2340|10179|781|2095|0|1|30|1|2095|781|10179|Sunday|2095Q1|N|N|N|2486244|2486243|2485908|2486181|N|N|N|N|N| +2486274|AAAAAAAACAAPFCAA|2095-01-31|2340|10179|781|2095|1|1|31|1|2095|781|10179|Monday|2095Q1|N|N|N|2486244|2486243|2485909|2486182|N|N|N|N|N| +2486275|AAAAAAAADAAPFCAA|2095-02-01|2341|10180|781|2095|2|2|1|1|2095|781|10180|Tuesday|2095Q1|N|N|N|2486275|2486305|2485910|2486183|N|N|N|N|N| +2486276|AAAAAAAAEAAPFCAA|2095-02-02|2341|10180|781|2095|3|2|2|1|2095|781|10180|Wednesday|2095Q1|N|N|N|2486275|2486305|2485911|2486184|N|N|N|N|N| +2486277|AAAAAAAAFAAPFCAA|2095-02-03|2341|10180|781|2095|4|2|3|1|2095|781|10180|Thursday|2095Q1|N|N|N|2486275|2486305|2485912|2486185|N|N|N|N|N| +2486278|AAAAAAAAGAAPFCAA|2095-02-04|2341|10180|781|2095|5|2|4|1|2095|781|10180|Friday|2095Q1|N|Y|N|2486275|2486305|2485913|2486186|N|N|N|N|N| +2486279|AAAAAAAAHAAPFCAA|2095-02-05|2341|10180|781|2095|6|2|5|1|2095|781|10180|Saturday|2095Q1|N|Y|N|2486275|2486305|2485914|2486187|N|N|N|N|N| +2486280|AAAAAAAAIAAPFCAA|2095-02-06|2341|10180|781|2095|0|2|6|1|2095|781|10180|Sunday|2095Q1|N|N|N|2486275|2486305|2485915|2486188|N|N|N|N|N| +2486281|AAAAAAAAJAAPFCAA|2095-02-07|2341|10180|781|2095|1|2|7|1|2095|781|10180|Monday|2095Q1|N|N|N|2486275|2486305|2485916|2486189|N|N|N|N|N| +2486282|AAAAAAAAKAAPFCAA|2095-02-08|2341|10181|781|2095|2|2|8|1|2095|781|10181|Tuesday|2095Q1|N|N|N|2486275|2486305|2485917|2486190|N|N|N|N|N| +2486283|AAAAAAAALAAPFCAA|2095-02-09|2341|10181|781|2095|3|2|9|1|2095|781|10181|Wednesday|2095Q1|N|N|N|2486275|2486305|2485918|2486191|N|N|N|N|N| +2486284|AAAAAAAAMAAPFCAA|2095-02-10|2341|10181|781|2095|4|2|10|1|2095|781|10181|Thursday|2095Q1|N|N|N|2486275|2486305|2485919|2486192|N|N|N|N|N| +2486285|AAAAAAAANAAPFCAA|2095-02-11|2341|10181|781|2095|5|2|11|1|2095|781|10181|Friday|2095Q1|N|Y|N|2486275|2486305|2485920|2486193|N|N|N|N|N| +2486286|AAAAAAAAOAAPFCAA|2095-02-12|2341|10181|781|2095|6|2|12|1|2095|781|10181|Saturday|2095Q1|N|Y|N|2486275|2486305|2485921|2486194|N|N|N|N|N| +2486287|AAAAAAAAPAAPFCAA|2095-02-13|2341|10181|781|2095|0|2|13|1|2095|781|10181|Sunday|2095Q1|N|N|N|2486275|2486305|2485922|2486195|N|N|N|N|N| +2486288|AAAAAAAAABAPFCAA|2095-02-14|2341|10181|781|2095|1|2|14|1|2095|781|10181|Monday|2095Q1|N|N|N|2486275|2486305|2485923|2486196|N|N|N|N|N| +2486289|AAAAAAAABBAPFCAA|2095-02-15|2341|10182|781|2095|2|2|15|1|2095|781|10182|Tuesday|2095Q1|N|N|N|2486275|2486305|2485924|2486197|N|N|N|N|N| +2486290|AAAAAAAACBAPFCAA|2095-02-16|2341|10182|781|2095|3|2|16|1|2095|781|10182|Wednesday|2095Q1|N|N|N|2486275|2486305|2485925|2486198|N|N|N|N|N| +2486291|AAAAAAAADBAPFCAA|2095-02-17|2341|10182|781|2095|4|2|17|1|2095|781|10182|Thursday|2095Q1|N|N|N|2486275|2486305|2485926|2486199|N|N|N|N|N| +2486292|AAAAAAAAEBAPFCAA|2095-02-18|2341|10182|781|2095|5|2|18|1|2095|781|10182|Friday|2095Q1|N|Y|N|2486275|2486305|2485927|2486200|N|N|N|N|N| +2486293|AAAAAAAAFBAPFCAA|2095-02-19|2341|10182|781|2095|6|2|19|1|2095|781|10182|Saturday|2095Q1|N|Y|N|2486275|2486305|2485928|2486201|N|N|N|N|N| +2486294|AAAAAAAAGBAPFCAA|2095-02-20|2341|10182|781|2095|0|2|20|1|2095|781|10182|Sunday|2095Q1|N|N|N|2486275|2486305|2485929|2486202|N|N|N|N|N| +2486295|AAAAAAAAHBAPFCAA|2095-02-21|2341|10182|781|2095|1|2|21|1|2095|781|10182|Monday|2095Q1|N|N|N|2486275|2486305|2485930|2486203|N|N|N|N|N| +2486296|AAAAAAAAIBAPFCAA|2095-02-22|2341|10183|781|2095|2|2|22|1|2095|781|10183|Tuesday|2095Q1|N|N|N|2486275|2486305|2485931|2486204|N|N|N|N|N| +2486297|AAAAAAAAJBAPFCAA|2095-02-23|2341|10183|781|2095|3|2|23|1|2095|781|10183|Wednesday|2095Q1|N|N|N|2486275|2486305|2485932|2486205|N|N|N|N|N| +2486298|AAAAAAAAKBAPFCAA|2095-02-24|2341|10183|781|2095|4|2|24|1|2095|781|10183|Thursday|2095Q1|N|N|N|2486275|2486305|2485933|2486206|N|N|N|N|N| +2486299|AAAAAAAALBAPFCAA|2095-02-25|2341|10183|781|2095|5|2|25|1|2095|781|10183|Friday|2095Q1|N|Y|N|2486275|2486305|2485934|2486207|N|N|N|N|N| +2486300|AAAAAAAAMBAPFCAA|2095-02-26|2341|10183|781|2095|6|2|26|1|2095|781|10183|Saturday|2095Q1|N|Y|N|2486275|2486305|2485935|2486208|N|N|N|N|N| +2486301|AAAAAAAANBAPFCAA|2095-02-27|2341|10183|781|2095|0|2|27|1|2095|781|10183|Sunday|2095Q1|N|N|N|2486275|2486305|2485936|2486209|N|N|N|N|N| +2486302|AAAAAAAAOBAPFCAA|2095-02-28|2341|10183|781|2095|1|2|28|1|2095|781|10183|Monday|2095Q1|N|N|N|2486275|2486305|2485937|2486210|N|N|N|N|N| +2486303|AAAAAAAAPBAPFCAA|2095-03-01|2342|10184|782|2095|2|3|1|1|2095|782|10184|Tuesday|2095Q1|N|N|N|2486303|2486361|2485938|2486211|N|N|N|N|N| +2486304|AAAAAAAAACAPFCAA|2095-03-02|2342|10184|782|2095|3|3|2|1|2095|782|10184|Wednesday|2095Q1|N|N|N|2486303|2486361|2485939|2486212|N|N|N|N|N| +2486305|AAAAAAAABCAPFCAA|2095-03-03|2342|10184|782|2095|4|3|3|1|2095|782|10184|Thursday|2095Q1|N|N|N|2486303|2486361|2485940|2486213|N|N|N|N|N| +2486306|AAAAAAAACCAPFCAA|2095-03-04|2342|10184|782|2095|5|3|4|1|2095|782|10184|Friday|2095Q1|N|Y|N|2486303|2486361|2485941|2486214|N|N|N|N|N| +2486307|AAAAAAAADCAPFCAA|2095-03-05|2342|10184|782|2095|6|3|5|1|2095|782|10184|Saturday|2095Q1|N|Y|N|2486303|2486361|2485942|2486215|N|N|N|N|N| +2486308|AAAAAAAAECAPFCAA|2095-03-06|2342|10184|782|2095|0|3|6|1|2095|782|10184|Sunday|2095Q1|N|N|N|2486303|2486361|2485943|2486216|N|N|N|N|N| +2486309|AAAAAAAAFCAPFCAA|2095-03-07|2342|10184|782|2095|1|3|7|1|2095|782|10184|Monday|2095Q1|N|N|N|2486303|2486361|2485944|2486217|N|N|N|N|N| +2486310|AAAAAAAAGCAPFCAA|2095-03-08|2342|10185|782|2095|2|3|8|1|2095|782|10185|Tuesday|2095Q1|N|N|N|2486303|2486361|2485945|2486218|N|N|N|N|N| +2486311|AAAAAAAAHCAPFCAA|2095-03-09|2342|10185|782|2095|3|3|9|1|2095|782|10185|Wednesday|2095Q1|N|N|N|2486303|2486361|2485946|2486219|N|N|N|N|N| +2486312|AAAAAAAAICAPFCAA|2095-03-10|2342|10185|782|2095|4|3|10|1|2095|782|10185|Thursday|2095Q1|N|N|N|2486303|2486361|2485947|2486220|N|N|N|N|N| +2486313|AAAAAAAAJCAPFCAA|2095-03-11|2342|10185|782|2095|5|3|11|1|2095|782|10185|Friday|2095Q1|N|Y|N|2486303|2486361|2485948|2486221|N|N|N|N|N| +2486314|AAAAAAAAKCAPFCAA|2095-03-12|2342|10185|782|2095|6|3|12|1|2095|782|10185|Saturday|2095Q1|N|Y|N|2486303|2486361|2485949|2486222|N|N|N|N|N| +2486315|AAAAAAAALCAPFCAA|2095-03-13|2342|10185|782|2095|0|3|13|1|2095|782|10185|Sunday|2095Q1|N|N|N|2486303|2486361|2485950|2486223|N|N|N|N|N| +2486316|AAAAAAAAMCAPFCAA|2095-03-14|2342|10185|782|2095|1|3|14|1|2095|782|10185|Monday|2095Q1|N|N|N|2486303|2486361|2485951|2486224|N|N|N|N|N| +2486317|AAAAAAAANCAPFCAA|2095-03-15|2342|10186|782|2095|2|3|15|1|2095|782|10186|Tuesday|2095Q1|N|N|N|2486303|2486361|2485952|2486225|N|N|N|N|N| +2486318|AAAAAAAAOCAPFCAA|2095-03-16|2342|10186|782|2095|3|3|16|1|2095|782|10186|Wednesday|2095Q1|N|N|N|2486303|2486361|2485953|2486226|N|N|N|N|N| +2486319|AAAAAAAAPCAPFCAA|2095-03-17|2342|10186|782|2095|4|3|17|1|2095|782|10186|Thursday|2095Q1|N|N|N|2486303|2486361|2485954|2486227|N|N|N|N|N| +2486320|AAAAAAAAADAPFCAA|2095-03-18|2342|10186|782|2095|5|3|18|1|2095|782|10186|Friday|2095Q1|N|Y|N|2486303|2486361|2485955|2486228|N|N|N|N|N| +2486321|AAAAAAAABDAPFCAA|2095-03-19|2342|10186|782|2095|6|3|19|1|2095|782|10186|Saturday|2095Q1|N|Y|N|2486303|2486361|2485956|2486229|N|N|N|N|N| +2486322|AAAAAAAACDAPFCAA|2095-03-20|2342|10186|782|2095|0|3|20|1|2095|782|10186|Sunday|2095Q1|N|N|N|2486303|2486361|2485957|2486230|N|N|N|N|N| +2486323|AAAAAAAADDAPFCAA|2095-03-21|2342|10186|782|2095|1|3|21|1|2095|782|10186|Monday|2095Q1|N|N|N|2486303|2486361|2485958|2486231|N|N|N|N|N| +2486324|AAAAAAAAEDAPFCAA|2095-03-22|2342|10187|782|2095|2|3|22|1|2095|782|10187|Tuesday|2095Q1|N|N|N|2486303|2486361|2485959|2486232|N|N|N|N|N| +2486325|AAAAAAAAFDAPFCAA|2095-03-23|2342|10187|782|2095|3|3|23|1|2095|782|10187|Wednesday|2095Q1|N|N|N|2486303|2486361|2485960|2486233|N|N|N|N|N| +2486326|AAAAAAAAGDAPFCAA|2095-03-24|2342|10187|782|2095|4|3|24|1|2095|782|10187|Thursday|2095Q1|N|N|N|2486303|2486361|2485961|2486234|N|N|N|N|N| +2486327|AAAAAAAAHDAPFCAA|2095-03-25|2342|10187|782|2095|5|3|25|1|2095|782|10187|Friday|2095Q1|N|Y|N|2486303|2486361|2485962|2486235|N|N|N|N|N| +2486328|AAAAAAAAIDAPFCAA|2095-03-26|2342|10187|782|2095|6|3|26|1|2095|782|10187|Saturday|2095Q1|N|Y|N|2486303|2486361|2485963|2486236|N|N|N|N|N| +2486329|AAAAAAAAJDAPFCAA|2095-03-27|2342|10187|782|2095|0|3|27|1|2095|782|10187|Sunday|2095Q1|N|N|N|2486303|2486361|2485964|2486237|N|N|N|N|N| +2486330|AAAAAAAAKDAPFCAA|2095-03-28|2342|10187|782|2095|1|3|28|1|2095|782|10187|Monday|2095Q1|N|N|N|2486303|2486361|2485965|2486238|N|N|N|N|N| +2486331|AAAAAAAALDAPFCAA|2095-03-29|2342|10188|782|2095|2|3|29|1|2095|782|10188|Tuesday|2095Q1|N|N|N|2486303|2486361|2485966|2486239|N|N|N|N|N| +2486332|AAAAAAAAMDAPFCAA|2095-03-30|2342|10188|782|2095|3|3|30|1|2095|782|10188|Wednesday|2095Q1|N|N|N|2486303|2486361|2485967|2486240|N|N|N|N|N| +2486333|AAAAAAAANDAPFCAA|2095-03-31|2342|10188|782|2095|4|3|31|1|2095|782|10188|Thursday|2095Q1|N|N|N|2486303|2486361|2485968|2486241|N|N|N|N|N| +2486334|AAAAAAAAODAPFCAA|2095-04-01|2343|10188|782|2095|5|4|1|1|2095|782|10188|Friday|2095Q1|N|Y|N|2486334|2486423|2485969|2486244|N|N|N|N|N| +2486335|AAAAAAAAPDAPFCAA|2095-04-02|2343|10188|782|2095|6|4|2|2|2095|782|10188|Saturday|2095Q2|N|Y|N|2486334|2486423|2485970|2486245|N|N|N|N|N| +2486336|AAAAAAAAAEAPFCAA|2095-04-03|2343|10188|782|2095|0|4|3|2|2095|782|10188|Sunday|2095Q2|N|N|N|2486334|2486423|2485971|2486246|N|N|N|N|N| +2486337|AAAAAAAABEAPFCAA|2095-04-04|2343|10188|782|2095|1|4|4|2|2095|782|10188|Monday|2095Q2|N|N|N|2486334|2486423|2485972|2486247|N|N|N|N|N| +2486338|AAAAAAAACEAPFCAA|2095-04-05|2343|10189|782|2095|2|4|5|2|2095|782|10189|Tuesday|2095Q2|N|N|N|2486334|2486423|2485973|2486248|N|N|N|N|N| +2486339|AAAAAAAADEAPFCAA|2095-04-06|2343|10189|782|2095|3|4|6|2|2095|782|10189|Wednesday|2095Q2|N|N|N|2486334|2486423|2485974|2486249|N|N|N|N|N| +2486340|AAAAAAAAEEAPFCAA|2095-04-07|2343|10189|782|2095|4|4|7|2|2095|782|10189|Thursday|2095Q2|N|N|N|2486334|2486423|2485975|2486250|N|N|N|N|N| +2486341|AAAAAAAAFEAPFCAA|2095-04-08|2343|10189|782|2095|5|4|8|2|2095|782|10189|Friday|2095Q2|N|Y|N|2486334|2486423|2485976|2486251|N|N|N|N|N| +2486342|AAAAAAAAGEAPFCAA|2095-04-09|2343|10189|782|2095|6|4|9|2|2095|782|10189|Saturday|2095Q2|N|Y|N|2486334|2486423|2485977|2486252|N|N|N|N|N| +2486343|AAAAAAAAHEAPFCAA|2095-04-10|2343|10189|782|2095|0|4|10|2|2095|782|10189|Sunday|2095Q2|N|N|N|2486334|2486423|2485978|2486253|N|N|N|N|N| +2486344|AAAAAAAAIEAPFCAA|2095-04-11|2343|10189|782|2095|1|4|11|2|2095|782|10189|Monday|2095Q2|N|N|N|2486334|2486423|2485979|2486254|N|N|N|N|N| +2486345|AAAAAAAAJEAPFCAA|2095-04-12|2343|10190|782|2095|2|4|12|2|2095|782|10190|Tuesday|2095Q2|N|N|N|2486334|2486423|2485980|2486255|N|N|N|N|N| +2486346|AAAAAAAAKEAPFCAA|2095-04-13|2343|10190|782|2095|3|4|13|2|2095|782|10190|Wednesday|2095Q2|N|N|N|2486334|2486423|2485981|2486256|N|N|N|N|N| +2486347|AAAAAAAALEAPFCAA|2095-04-14|2343|10190|782|2095|4|4|14|2|2095|782|10190|Thursday|2095Q2|N|N|N|2486334|2486423|2485982|2486257|N|N|N|N|N| +2486348|AAAAAAAAMEAPFCAA|2095-04-15|2343|10190|782|2095|5|4|15|2|2095|782|10190|Friday|2095Q2|N|Y|N|2486334|2486423|2485983|2486258|N|N|N|N|N| +2486349|AAAAAAAANEAPFCAA|2095-04-16|2343|10190|782|2095|6|4|16|2|2095|782|10190|Saturday|2095Q2|N|Y|N|2486334|2486423|2485984|2486259|N|N|N|N|N| +2486350|AAAAAAAAOEAPFCAA|2095-04-17|2343|10190|782|2095|0|4|17|2|2095|782|10190|Sunday|2095Q2|N|N|N|2486334|2486423|2485985|2486260|N|N|N|N|N| +2486351|AAAAAAAAPEAPFCAA|2095-04-18|2343|10190|782|2095|1|4|18|2|2095|782|10190|Monday|2095Q2|N|N|N|2486334|2486423|2485986|2486261|N|N|N|N|N| +2486352|AAAAAAAAAFAPFCAA|2095-04-19|2343|10191|782|2095|2|4|19|2|2095|782|10191|Tuesday|2095Q2|N|N|N|2486334|2486423|2485987|2486262|N|N|N|N|N| +2486353|AAAAAAAABFAPFCAA|2095-04-20|2343|10191|782|2095|3|4|20|2|2095|782|10191|Wednesday|2095Q2|N|N|N|2486334|2486423|2485988|2486263|N|N|N|N|N| +2486354|AAAAAAAACFAPFCAA|2095-04-21|2343|10191|782|2095|4|4|21|2|2095|782|10191|Thursday|2095Q2|N|N|N|2486334|2486423|2485989|2486264|N|N|N|N|N| +2486355|AAAAAAAADFAPFCAA|2095-04-22|2343|10191|782|2095|5|4|22|2|2095|782|10191|Friday|2095Q2|N|Y|N|2486334|2486423|2485990|2486265|N|N|N|N|N| +2486356|AAAAAAAAEFAPFCAA|2095-04-23|2343|10191|782|2095|6|4|23|2|2095|782|10191|Saturday|2095Q2|N|Y|N|2486334|2486423|2485991|2486266|N|N|N|N|N| +2486357|AAAAAAAAFFAPFCAA|2095-04-24|2343|10191|782|2095|0|4|24|2|2095|782|10191|Sunday|2095Q2|N|N|N|2486334|2486423|2485992|2486267|N|N|N|N|N| +2486358|AAAAAAAAGFAPFCAA|2095-04-25|2343|10191|782|2095|1|4|25|2|2095|782|10191|Monday|2095Q2|N|N|N|2486334|2486423|2485993|2486268|N|N|N|N|N| +2486359|AAAAAAAAHFAPFCAA|2095-04-26|2343|10192|782|2095|2|4|26|2|2095|782|10192|Tuesday|2095Q2|N|N|N|2486334|2486423|2485994|2486269|N|N|N|N|N| +2486360|AAAAAAAAIFAPFCAA|2095-04-27|2343|10192|782|2095|3|4|27|2|2095|782|10192|Wednesday|2095Q2|N|N|N|2486334|2486423|2485995|2486270|N|N|N|N|N| +2486361|AAAAAAAAJFAPFCAA|2095-04-28|2343|10192|782|2095|4|4|28|2|2095|782|10192|Thursday|2095Q2|N|N|N|2486334|2486423|2485996|2486271|N|N|N|N|N| +2486362|AAAAAAAAKFAPFCAA|2095-04-29|2343|10192|782|2095|5|4|29|2|2095|782|10192|Friday|2095Q2|N|Y|N|2486334|2486423|2485997|2486272|N|N|N|N|N| +2486363|AAAAAAAALFAPFCAA|2095-04-30|2343|10192|782|2095|6|4|30|2|2095|782|10192|Saturday|2095Q2|N|Y|N|2486334|2486423|2485998|2486273|N|N|N|N|N| +2486364|AAAAAAAAMFAPFCAA|2095-05-01|2344|10192|782|2095|0|5|1|2|2095|782|10192|Sunday|2095Q2|N|N|N|2486364|2486483|2485999|2486274|N|N|N|N|N| +2486365|AAAAAAAANFAPFCAA|2095-05-02|2344|10192|782|2095|1|5|2|2|2095|782|10192|Monday|2095Q2|N|N|N|2486364|2486483|2486000|2486275|N|N|N|N|N| +2486366|AAAAAAAAOFAPFCAA|2095-05-03|2344|10193|782|2095|2|5|3|2|2095|782|10193|Tuesday|2095Q2|N|N|N|2486364|2486483|2486001|2486276|N|N|N|N|N| +2486367|AAAAAAAAPFAPFCAA|2095-05-04|2344|10193|782|2095|3|5|4|2|2095|782|10193|Wednesday|2095Q2|N|N|N|2486364|2486483|2486002|2486277|N|N|N|N|N| +2486368|AAAAAAAAAGAPFCAA|2095-05-05|2344|10193|782|2095|4|5|5|2|2095|782|10193|Thursday|2095Q2|N|N|N|2486364|2486483|2486003|2486278|N|N|N|N|N| +2486369|AAAAAAAABGAPFCAA|2095-05-06|2344|10193|782|2095|5|5|6|2|2095|782|10193|Friday|2095Q2|N|Y|N|2486364|2486483|2486004|2486279|N|N|N|N|N| +2486370|AAAAAAAACGAPFCAA|2095-05-07|2344|10193|782|2095|6|5|7|2|2095|782|10193|Saturday|2095Q2|N|Y|N|2486364|2486483|2486005|2486280|N|N|N|N|N| +2486371|AAAAAAAADGAPFCAA|2095-05-08|2344|10193|782|2095|0|5|8|2|2095|782|10193|Sunday|2095Q2|N|N|N|2486364|2486483|2486006|2486281|N|N|N|N|N| +2486372|AAAAAAAAEGAPFCAA|2095-05-09|2344|10193|782|2095|1|5|9|2|2095|782|10193|Monday|2095Q2|N|N|N|2486364|2486483|2486007|2486282|N|N|N|N|N| +2486373|AAAAAAAAFGAPFCAA|2095-05-10|2344|10194|782|2095|2|5|10|2|2095|782|10194|Tuesday|2095Q2|N|N|N|2486364|2486483|2486008|2486283|N|N|N|N|N| +2486374|AAAAAAAAGGAPFCAA|2095-05-11|2344|10194|782|2095|3|5|11|2|2095|782|10194|Wednesday|2095Q2|N|N|N|2486364|2486483|2486009|2486284|N|N|N|N|N| +2486375|AAAAAAAAHGAPFCAA|2095-05-12|2344|10194|782|2095|4|5|12|2|2095|782|10194|Thursday|2095Q2|N|N|N|2486364|2486483|2486010|2486285|N|N|N|N|N| +2486376|AAAAAAAAIGAPFCAA|2095-05-13|2344|10194|782|2095|5|5|13|2|2095|782|10194|Friday|2095Q2|N|Y|N|2486364|2486483|2486011|2486286|N|N|N|N|N| +2486377|AAAAAAAAJGAPFCAA|2095-05-14|2344|10194|782|2095|6|5|14|2|2095|782|10194|Saturday|2095Q2|N|Y|N|2486364|2486483|2486012|2486287|N|N|N|N|N| +2486378|AAAAAAAAKGAPFCAA|2095-05-15|2344|10194|782|2095|0|5|15|2|2095|782|10194|Sunday|2095Q2|N|N|N|2486364|2486483|2486013|2486288|N|N|N|N|N| +2486379|AAAAAAAALGAPFCAA|2095-05-16|2344|10194|782|2095|1|5|16|2|2095|782|10194|Monday|2095Q2|N|N|N|2486364|2486483|2486014|2486289|N|N|N|N|N| +2486380|AAAAAAAAMGAPFCAA|2095-05-17|2344|10195|782|2095|2|5|17|2|2095|782|10195|Tuesday|2095Q2|N|N|N|2486364|2486483|2486015|2486290|N|N|N|N|N| +2486381|AAAAAAAANGAPFCAA|2095-05-18|2344|10195|782|2095|3|5|18|2|2095|782|10195|Wednesday|2095Q2|N|N|N|2486364|2486483|2486016|2486291|N|N|N|N|N| +2486382|AAAAAAAAOGAPFCAA|2095-05-19|2344|10195|782|2095|4|5|19|2|2095|782|10195|Thursday|2095Q2|N|N|N|2486364|2486483|2486017|2486292|N|N|N|N|N| +2486383|AAAAAAAAPGAPFCAA|2095-05-20|2344|10195|782|2095|5|5|20|2|2095|782|10195|Friday|2095Q2|N|Y|N|2486364|2486483|2486018|2486293|N|N|N|N|N| +2486384|AAAAAAAAAHAPFCAA|2095-05-21|2344|10195|782|2095|6|5|21|2|2095|782|10195|Saturday|2095Q2|N|Y|N|2486364|2486483|2486019|2486294|N|N|N|N|N| +2486385|AAAAAAAABHAPFCAA|2095-05-22|2344|10195|782|2095|0|5|22|2|2095|782|10195|Sunday|2095Q2|N|N|N|2486364|2486483|2486020|2486295|N|N|N|N|N| +2486386|AAAAAAAACHAPFCAA|2095-05-23|2344|10195|782|2095|1|5|23|2|2095|782|10195|Monday|2095Q2|N|N|N|2486364|2486483|2486021|2486296|N|N|N|N|N| +2486387|AAAAAAAADHAPFCAA|2095-05-24|2344|10196|782|2095|2|5|24|2|2095|782|10196|Tuesday|2095Q2|N|N|N|2486364|2486483|2486022|2486297|N|N|N|N|N| +2486388|AAAAAAAAEHAPFCAA|2095-05-25|2344|10196|782|2095|3|5|25|2|2095|782|10196|Wednesday|2095Q2|N|N|N|2486364|2486483|2486023|2486298|N|N|N|N|N| +2486389|AAAAAAAAFHAPFCAA|2095-05-26|2344|10196|782|2095|4|5|26|2|2095|782|10196|Thursday|2095Q2|N|N|N|2486364|2486483|2486024|2486299|N|N|N|N|N| +2486390|AAAAAAAAGHAPFCAA|2095-05-27|2344|10196|782|2095|5|5|27|2|2095|782|10196|Friday|2095Q2|N|Y|N|2486364|2486483|2486025|2486300|N|N|N|N|N| +2486391|AAAAAAAAHHAPFCAA|2095-05-28|2344|10196|782|2095|6|5|28|2|2095|782|10196|Saturday|2095Q2|N|Y|N|2486364|2486483|2486026|2486301|N|N|N|N|N| +2486392|AAAAAAAAIHAPFCAA|2095-05-29|2344|10196|782|2095|0|5|29|2|2095|782|10196|Sunday|2095Q2|N|N|N|2486364|2486483|2486027|2486302|N|N|N|N|N| +2486393|AAAAAAAAJHAPFCAA|2095-05-30|2344|10196|782|2095|1|5|30|2|2095|782|10196|Monday|2095Q2|N|N|N|2486364|2486483|2486028|2486303|N|N|N|N|N| +2486394|AAAAAAAAKHAPFCAA|2095-05-31|2344|10197|782|2095|2|5|31|2|2095|782|10197|Tuesday|2095Q2|N|N|N|2486364|2486483|2486029|2486304|N|N|N|N|N| +2486395|AAAAAAAALHAPFCAA|2095-06-01|2345|10197|783|2095|3|6|1|2|2095|783|10197|Wednesday|2095Q2|N|N|N|2486395|2486545|2486030|2486305|N|N|N|N|N| +2486396|AAAAAAAAMHAPFCAA|2095-06-02|2345|10197|783|2095|4|6|2|2|2095|783|10197|Thursday|2095Q2|N|N|N|2486395|2486545|2486031|2486306|N|N|N|N|N| +2486397|AAAAAAAANHAPFCAA|2095-06-03|2345|10197|783|2095|5|6|3|2|2095|783|10197|Friday|2095Q2|N|Y|N|2486395|2486545|2486032|2486307|N|N|N|N|N| +2486398|AAAAAAAAOHAPFCAA|2095-06-04|2345|10197|783|2095|6|6|4|2|2095|783|10197|Saturday|2095Q2|N|Y|N|2486395|2486545|2486033|2486308|N|N|N|N|N| +2486399|AAAAAAAAPHAPFCAA|2095-06-05|2345|10197|783|2095|0|6|5|2|2095|783|10197|Sunday|2095Q2|N|N|N|2486395|2486545|2486034|2486309|N|N|N|N|N| +2486400|AAAAAAAAAIAPFCAA|2095-06-06|2345|10197|783|2095|1|6|6|2|2095|783|10197|Monday|2095Q2|N|N|N|2486395|2486545|2486035|2486310|N|N|N|N|N| +2486401|AAAAAAAABIAPFCAA|2095-06-07|2345|10198|783|2095|2|6|7|2|2095|783|10198|Tuesday|2095Q2|N|N|N|2486395|2486545|2486036|2486311|N|N|N|N|N| +2486402|AAAAAAAACIAPFCAA|2095-06-08|2345|10198|783|2095|3|6|8|2|2095|783|10198|Wednesday|2095Q2|N|N|N|2486395|2486545|2486037|2486312|N|N|N|N|N| +2486403|AAAAAAAADIAPFCAA|2095-06-09|2345|10198|783|2095|4|6|9|2|2095|783|10198|Thursday|2095Q2|N|N|N|2486395|2486545|2486038|2486313|N|N|N|N|N| +2486404|AAAAAAAAEIAPFCAA|2095-06-10|2345|10198|783|2095|5|6|10|2|2095|783|10198|Friday|2095Q2|N|Y|N|2486395|2486545|2486039|2486314|N|N|N|N|N| +2486405|AAAAAAAAFIAPFCAA|2095-06-11|2345|10198|783|2095|6|6|11|2|2095|783|10198|Saturday|2095Q2|N|Y|N|2486395|2486545|2486040|2486315|N|N|N|N|N| +2486406|AAAAAAAAGIAPFCAA|2095-06-12|2345|10198|783|2095|0|6|12|2|2095|783|10198|Sunday|2095Q2|N|N|N|2486395|2486545|2486041|2486316|N|N|N|N|N| +2486407|AAAAAAAAHIAPFCAA|2095-06-13|2345|10198|783|2095|1|6|13|2|2095|783|10198|Monday|2095Q2|N|N|N|2486395|2486545|2486042|2486317|N|N|N|N|N| +2486408|AAAAAAAAIIAPFCAA|2095-06-14|2345|10199|783|2095|2|6|14|2|2095|783|10199|Tuesday|2095Q2|N|N|N|2486395|2486545|2486043|2486318|N|N|N|N|N| +2486409|AAAAAAAAJIAPFCAA|2095-06-15|2345|10199|783|2095|3|6|15|2|2095|783|10199|Wednesday|2095Q2|N|N|N|2486395|2486545|2486044|2486319|N|N|N|N|N| +2486410|AAAAAAAAKIAPFCAA|2095-06-16|2345|10199|783|2095|4|6|16|2|2095|783|10199|Thursday|2095Q2|N|N|N|2486395|2486545|2486045|2486320|N|N|N|N|N| +2486411|AAAAAAAALIAPFCAA|2095-06-17|2345|10199|783|2095|5|6|17|2|2095|783|10199|Friday|2095Q2|N|Y|N|2486395|2486545|2486046|2486321|N|N|N|N|N| +2486412|AAAAAAAAMIAPFCAA|2095-06-18|2345|10199|783|2095|6|6|18|2|2095|783|10199|Saturday|2095Q2|N|Y|N|2486395|2486545|2486047|2486322|N|N|N|N|N| +2486413|AAAAAAAANIAPFCAA|2095-06-19|2345|10199|783|2095|0|6|19|2|2095|783|10199|Sunday|2095Q2|N|N|N|2486395|2486545|2486048|2486323|N|N|N|N|N| +2486414|AAAAAAAAOIAPFCAA|2095-06-20|2345|10199|783|2095|1|6|20|2|2095|783|10199|Monday|2095Q2|N|N|N|2486395|2486545|2486049|2486324|N|N|N|N|N| +2486415|AAAAAAAAPIAPFCAA|2095-06-21|2345|10200|783|2095|2|6|21|2|2095|783|10200|Tuesday|2095Q2|N|N|N|2486395|2486545|2486050|2486325|N|N|N|N|N| +2486416|AAAAAAAAAJAPFCAA|2095-06-22|2345|10200|783|2095|3|6|22|2|2095|783|10200|Wednesday|2095Q2|N|N|N|2486395|2486545|2486051|2486326|N|N|N|N|N| +2486417|AAAAAAAABJAPFCAA|2095-06-23|2345|10200|783|2095|4|6|23|2|2095|783|10200|Thursday|2095Q2|N|N|N|2486395|2486545|2486052|2486327|N|N|N|N|N| +2486418|AAAAAAAACJAPFCAA|2095-06-24|2345|10200|783|2095|5|6|24|2|2095|783|10200|Friday|2095Q2|N|Y|N|2486395|2486545|2486053|2486328|N|N|N|N|N| +2486419|AAAAAAAADJAPFCAA|2095-06-25|2345|10200|783|2095|6|6|25|2|2095|783|10200|Saturday|2095Q2|N|Y|N|2486395|2486545|2486054|2486329|N|N|N|N|N| +2486420|AAAAAAAAEJAPFCAA|2095-06-26|2345|10200|783|2095|0|6|26|2|2095|783|10200|Sunday|2095Q2|N|N|N|2486395|2486545|2486055|2486330|N|N|N|N|N| +2486421|AAAAAAAAFJAPFCAA|2095-06-27|2345|10200|783|2095|1|6|27|2|2095|783|10200|Monday|2095Q2|N|N|N|2486395|2486545|2486056|2486331|N|N|N|N|N| +2486422|AAAAAAAAGJAPFCAA|2095-06-28|2345|10201|783|2095|2|6|28|2|2095|783|10201|Tuesday|2095Q2|N|N|N|2486395|2486545|2486057|2486332|N|N|N|N|N| +2486423|AAAAAAAAHJAPFCAA|2095-06-29|2345|10201|783|2095|3|6|29|2|2095|783|10201|Wednesday|2095Q2|N|N|N|2486395|2486545|2486058|2486333|N|N|N|N|N| +2486424|AAAAAAAAIJAPFCAA|2095-06-30|2345|10201|783|2095|4|6|30|2|2095|783|10201|Thursday|2095Q2|N|N|N|2486395|2486545|2486059|2486334|N|N|N|N|N| +2486425|AAAAAAAAJJAPFCAA|2095-07-01|2346|10201|783|2095|5|7|1|2|2095|783|10201|Friday|2095Q2|N|Y|N|2486425|2486605|2486060|2486334|N|N|N|N|N| +2486426|AAAAAAAAKJAPFCAA|2095-07-02|2346|10201|783|2095|6|7|2|3|2095|783|10201|Saturday|2095Q3|N|Y|N|2486425|2486605|2486061|2486335|N|N|N|N|N| +2486427|AAAAAAAALJAPFCAA|2095-07-03|2346|10201|783|2095|0|7|3|3|2095|783|10201|Sunday|2095Q3|N|N|N|2486425|2486605|2486062|2486336|N|N|N|N|N| +2486428|AAAAAAAAMJAPFCAA|2095-07-04|2346|10201|783|2095|1|7|4|3|2095|783|10201|Monday|2095Q3|N|N|N|2486425|2486605|2486063|2486337|N|N|N|N|N| +2486429|AAAAAAAANJAPFCAA|2095-07-05|2346|10202|783|2095|2|7|5|3|2095|783|10202|Tuesday|2095Q3|Y|N|N|2486425|2486605|2486064|2486338|N|N|N|N|N| +2486430|AAAAAAAAOJAPFCAA|2095-07-06|2346|10202|783|2095|3|7|6|3|2095|783|10202|Wednesday|2095Q3|N|N|Y|2486425|2486605|2486065|2486339|N|N|N|N|N| +2486431|AAAAAAAAPJAPFCAA|2095-07-07|2346|10202|783|2095|4|7|7|3|2095|783|10202|Thursday|2095Q3|N|N|N|2486425|2486605|2486066|2486340|N|N|N|N|N| +2486432|AAAAAAAAAKAPFCAA|2095-07-08|2346|10202|783|2095|5|7|8|3|2095|783|10202|Friday|2095Q3|N|Y|N|2486425|2486605|2486067|2486341|N|N|N|N|N| +2486433|AAAAAAAABKAPFCAA|2095-07-09|2346|10202|783|2095|6|7|9|3|2095|783|10202|Saturday|2095Q3|N|Y|N|2486425|2486605|2486068|2486342|N|N|N|N|N| +2486434|AAAAAAAACKAPFCAA|2095-07-10|2346|10202|783|2095|0|7|10|3|2095|783|10202|Sunday|2095Q3|N|N|N|2486425|2486605|2486069|2486343|N|N|N|N|N| +2486435|AAAAAAAADKAPFCAA|2095-07-11|2346|10202|783|2095|1|7|11|3|2095|783|10202|Monday|2095Q3|N|N|N|2486425|2486605|2486070|2486344|N|N|N|N|N| +2486436|AAAAAAAAEKAPFCAA|2095-07-12|2346|10203|783|2095|2|7|12|3|2095|783|10203|Tuesday|2095Q3|N|N|N|2486425|2486605|2486071|2486345|N|N|N|N|N| +2486437|AAAAAAAAFKAPFCAA|2095-07-13|2346|10203|783|2095|3|7|13|3|2095|783|10203|Wednesday|2095Q3|N|N|N|2486425|2486605|2486072|2486346|N|N|N|N|N| +2486438|AAAAAAAAGKAPFCAA|2095-07-14|2346|10203|783|2095|4|7|14|3|2095|783|10203|Thursday|2095Q3|N|N|N|2486425|2486605|2486073|2486347|N|N|N|N|N| +2486439|AAAAAAAAHKAPFCAA|2095-07-15|2346|10203|783|2095|5|7|15|3|2095|783|10203|Friday|2095Q3|N|Y|N|2486425|2486605|2486074|2486348|N|N|N|N|N| +2486440|AAAAAAAAIKAPFCAA|2095-07-16|2346|10203|783|2095|6|7|16|3|2095|783|10203|Saturday|2095Q3|N|Y|N|2486425|2486605|2486075|2486349|N|N|N|N|N| +2486441|AAAAAAAAJKAPFCAA|2095-07-17|2346|10203|783|2095|0|7|17|3|2095|783|10203|Sunday|2095Q3|N|N|N|2486425|2486605|2486076|2486350|N|N|N|N|N| +2486442|AAAAAAAAKKAPFCAA|2095-07-18|2346|10203|783|2095|1|7|18|3|2095|783|10203|Monday|2095Q3|N|N|N|2486425|2486605|2486077|2486351|N|N|N|N|N| +2486443|AAAAAAAALKAPFCAA|2095-07-19|2346|10204|783|2095|2|7|19|3|2095|783|10204|Tuesday|2095Q3|N|N|N|2486425|2486605|2486078|2486352|N|N|N|N|N| +2486444|AAAAAAAAMKAPFCAA|2095-07-20|2346|10204|783|2095|3|7|20|3|2095|783|10204|Wednesday|2095Q3|N|N|N|2486425|2486605|2486079|2486353|N|N|N|N|N| +2486445|AAAAAAAANKAPFCAA|2095-07-21|2346|10204|783|2095|4|7|21|3|2095|783|10204|Thursday|2095Q3|N|N|N|2486425|2486605|2486080|2486354|N|N|N|N|N| +2486446|AAAAAAAAOKAPFCAA|2095-07-22|2346|10204|783|2095|5|7|22|3|2095|783|10204|Friday|2095Q3|N|Y|N|2486425|2486605|2486081|2486355|N|N|N|N|N| +2486447|AAAAAAAAPKAPFCAA|2095-07-23|2346|10204|783|2095|6|7|23|3|2095|783|10204|Saturday|2095Q3|N|Y|N|2486425|2486605|2486082|2486356|N|N|N|N|N| +2486448|AAAAAAAAALAPFCAA|2095-07-24|2346|10204|783|2095|0|7|24|3|2095|783|10204|Sunday|2095Q3|N|N|N|2486425|2486605|2486083|2486357|N|N|N|N|N| +2486449|AAAAAAAABLAPFCAA|2095-07-25|2346|10204|783|2095|1|7|25|3|2095|783|10204|Monday|2095Q3|N|N|N|2486425|2486605|2486084|2486358|N|N|N|N|N| +2486450|AAAAAAAACLAPFCAA|2095-07-26|2346|10205|783|2095|2|7|26|3|2095|783|10205|Tuesday|2095Q3|N|N|N|2486425|2486605|2486085|2486359|N|N|N|N|N| +2486451|AAAAAAAADLAPFCAA|2095-07-27|2346|10205|783|2095|3|7|27|3|2095|783|10205|Wednesday|2095Q3|N|N|N|2486425|2486605|2486086|2486360|N|N|N|N|N| +2486452|AAAAAAAAELAPFCAA|2095-07-28|2346|10205|783|2095|4|7|28|3|2095|783|10205|Thursday|2095Q3|N|N|N|2486425|2486605|2486087|2486361|N|N|N|N|N| +2486453|AAAAAAAAFLAPFCAA|2095-07-29|2346|10205|783|2095|5|7|29|3|2095|783|10205|Friday|2095Q3|N|Y|N|2486425|2486605|2486088|2486362|N|N|N|N|N| +2486454|AAAAAAAAGLAPFCAA|2095-07-30|2346|10205|783|2095|6|7|30|3|2095|783|10205|Saturday|2095Q3|N|Y|N|2486425|2486605|2486089|2486363|N|N|N|N|N| +2486455|AAAAAAAAHLAPFCAA|2095-07-31|2346|10205|783|2095|0|7|31|3|2095|783|10205|Sunday|2095Q3|N|N|N|2486425|2486605|2486090|2486364|N|N|N|N|N| +2486456|AAAAAAAAILAPFCAA|2095-08-01|2347|10205|783|2095|1|8|1|3|2095|783|10205|Monday|2095Q3|N|N|N|2486456|2486667|2486091|2486365|N|N|N|N|N| +2486457|AAAAAAAAJLAPFCAA|2095-08-02|2347|10206|783|2095|2|8|2|3|2095|783|10206|Tuesday|2095Q3|N|N|N|2486456|2486667|2486092|2486366|N|N|N|N|N| +2486458|AAAAAAAAKLAPFCAA|2095-08-03|2347|10206|783|2095|3|8|3|3|2095|783|10206|Wednesday|2095Q3|N|N|N|2486456|2486667|2486093|2486367|N|N|N|N|N| +2486459|AAAAAAAALLAPFCAA|2095-08-04|2347|10206|783|2095|4|8|4|3|2095|783|10206|Thursday|2095Q3|N|N|N|2486456|2486667|2486094|2486368|N|N|N|N|N| +2486460|AAAAAAAAMLAPFCAA|2095-08-05|2347|10206|783|2095|5|8|5|3|2095|783|10206|Friday|2095Q3|N|Y|N|2486456|2486667|2486095|2486369|N|N|N|N|N| +2486461|AAAAAAAANLAPFCAA|2095-08-06|2347|10206|783|2095|6|8|6|3|2095|783|10206|Saturday|2095Q3|N|Y|N|2486456|2486667|2486096|2486370|N|N|N|N|N| +2486462|AAAAAAAAOLAPFCAA|2095-08-07|2347|10206|783|2095|0|8|7|3|2095|783|10206|Sunday|2095Q3|N|N|N|2486456|2486667|2486097|2486371|N|N|N|N|N| +2486463|AAAAAAAAPLAPFCAA|2095-08-08|2347|10206|783|2095|1|8|8|3|2095|783|10206|Monday|2095Q3|N|N|N|2486456|2486667|2486098|2486372|N|N|N|N|N| +2486464|AAAAAAAAAMAPFCAA|2095-08-09|2347|10207|783|2095|2|8|9|3|2095|783|10207|Tuesday|2095Q3|N|N|N|2486456|2486667|2486099|2486373|N|N|N|N|N| +2486465|AAAAAAAABMAPFCAA|2095-08-10|2347|10207|783|2095|3|8|10|3|2095|783|10207|Wednesday|2095Q3|N|N|N|2486456|2486667|2486100|2486374|N|N|N|N|N| +2486466|AAAAAAAACMAPFCAA|2095-08-11|2347|10207|783|2095|4|8|11|3|2095|783|10207|Thursday|2095Q3|N|N|N|2486456|2486667|2486101|2486375|N|N|N|N|N| +2486467|AAAAAAAADMAPFCAA|2095-08-12|2347|10207|783|2095|5|8|12|3|2095|783|10207|Friday|2095Q3|N|Y|N|2486456|2486667|2486102|2486376|N|N|N|N|N| +2486468|AAAAAAAAEMAPFCAA|2095-08-13|2347|10207|783|2095|6|8|13|3|2095|783|10207|Saturday|2095Q3|N|Y|N|2486456|2486667|2486103|2486377|N|N|N|N|N| +2486469|AAAAAAAAFMAPFCAA|2095-08-14|2347|10207|783|2095|0|8|14|3|2095|783|10207|Sunday|2095Q3|N|N|N|2486456|2486667|2486104|2486378|N|N|N|N|N| +2486470|AAAAAAAAGMAPFCAA|2095-08-15|2347|10207|783|2095|1|8|15|3|2095|783|10207|Monday|2095Q3|N|N|N|2486456|2486667|2486105|2486379|N|N|N|N|N| +2486471|AAAAAAAAHMAPFCAA|2095-08-16|2347|10208|783|2095|2|8|16|3|2095|783|10208|Tuesday|2095Q3|N|N|N|2486456|2486667|2486106|2486380|N|N|N|N|N| +2486472|AAAAAAAAIMAPFCAA|2095-08-17|2347|10208|783|2095|3|8|17|3|2095|783|10208|Wednesday|2095Q3|N|N|N|2486456|2486667|2486107|2486381|N|N|N|N|N| +2486473|AAAAAAAAJMAPFCAA|2095-08-18|2347|10208|783|2095|4|8|18|3|2095|783|10208|Thursday|2095Q3|N|N|N|2486456|2486667|2486108|2486382|N|N|N|N|N| +2486474|AAAAAAAAKMAPFCAA|2095-08-19|2347|10208|783|2095|5|8|19|3|2095|783|10208|Friday|2095Q3|N|Y|N|2486456|2486667|2486109|2486383|N|N|N|N|N| +2486475|AAAAAAAALMAPFCAA|2095-08-20|2347|10208|783|2095|6|8|20|3|2095|783|10208|Saturday|2095Q3|N|Y|N|2486456|2486667|2486110|2486384|N|N|N|N|N| +2486476|AAAAAAAAMMAPFCAA|2095-08-21|2347|10208|783|2095|0|8|21|3|2095|783|10208|Sunday|2095Q3|N|N|N|2486456|2486667|2486111|2486385|N|N|N|N|N| +2486477|AAAAAAAANMAPFCAA|2095-08-22|2347|10208|783|2095|1|8|22|3|2095|783|10208|Monday|2095Q3|N|N|N|2486456|2486667|2486112|2486386|N|N|N|N|N| +2486478|AAAAAAAAOMAPFCAA|2095-08-23|2347|10209|783|2095|2|8|23|3|2095|783|10209|Tuesday|2095Q3|N|N|N|2486456|2486667|2486113|2486387|N|N|N|N|N| +2486479|AAAAAAAAPMAPFCAA|2095-08-24|2347|10209|783|2095|3|8|24|3|2095|783|10209|Wednesday|2095Q3|N|N|N|2486456|2486667|2486114|2486388|N|N|N|N|N| +2486480|AAAAAAAAANAPFCAA|2095-08-25|2347|10209|783|2095|4|8|25|3|2095|783|10209|Thursday|2095Q3|N|N|N|2486456|2486667|2486115|2486389|N|N|N|N|N| +2486481|AAAAAAAABNAPFCAA|2095-08-26|2347|10209|783|2095|5|8|26|3|2095|783|10209|Friday|2095Q3|N|Y|N|2486456|2486667|2486116|2486390|N|N|N|N|N| +2486482|AAAAAAAACNAPFCAA|2095-08-27|2347|10209|783|2095|6|8|27|3|2095|783|10209|Saturday|2095Q3|N|Y|N|2486456|2486667|2486117|2486391|N|N|N|N|N| +2486483|AAAAAAAADNAPFCAA|2095-08-28|2347|10209|783|2095|0|8|28|3|2095|783|10209|Sunday|2095Q3|N|N|N|2486456|2486667|2486118|2486392|N|N|N|N|N| +2486484|AAAAAAAAENAPFCAA|2095-08-29|2347|10209|783|2095|1|8|29|3|2095|783|10209|Monday|2095Q3|N|N|N|2486456|2486667|2486119|2486393|N|N|N|N|N| +2486485|AAAAAAAAFNAPFCAA|2095-08-30|2347|10210|783|2095|2|8|30|3|2095|783|10210|Tuesday|2095Q3|N|N|N|2486456|2486667|2486120|2486394|N|N|N|N|N| +2486486|AAAAAAAAGNAPFCAA|2095-08-31|2347|10210|783|2095|3|8|31|3|2095|783|10210|Wednesday|2095Q3|N|N|N|2486456|2486667|2486121|2486395|N|N|N|N|N| +2486487|AAAAAAAAHNAPFCAA|2095-09-01|2348|10210|784|2095|4|9|1|3|2095|784|10210|Thursday|2095Q3|N|N|N|2486487|2486729|2486122|2486396|N|N|N|N|N| +2486488|AAAAAAAAINAPFCAA|2095-09-02|2348|10210|784|2095|5|9|2|3|2095|784|10210|Friday|2095Q3|N|Y|N|2486487|2486729|2486123|2486397|N|N|N|N|N| +2486489|AAAAAAAAJNAPFCAA|2095-09-03|2348|10210|784|2095|6|9|3|3|2095|784|10210|Saturday|2095Q3|N|Y|N|2486487|2486729|2486124|2486398|N|N|N|N|N| +2486490|AAAAAAAAKNAPFCAA|2095-09-04|2348|10210|784|2095|0|9|4|3|2095|784|10210|Sunday|2095Q3|N|N|N|2486487|2486729|2486125|2486399|N|N|N|N|N| +2486491|AAAAAAAALNAPFCAA|2095-09-05|2348|10210|784|2095|1|9|5|3|2095|784|10210|Monday|2095Q3|N|N|N|2486487|2486729|2486126|2486400|N|N|N|N|N| +2486492|AAAAAAAAMNAPFCAA|2095-09-06|2348|10211|784|2095|2|9|6|3|2095|784|10211|Tuesday|2095Q3|N|N|N|2486487|2486729|2486127|2486401|N|N|N|N|N| +2486493|AAAAAAAANNAPFCAA|2095-09-07|2348|10211|784|2095|3|9|7|3|2095|784|10211|Wednesday|2095Q3|N|N|N|2486487|2486729|2486128|2486402|N|N|N|N|N| +2486494|AAAAAAAAONAPFCAA|2095-09-08|2348|10211|784|2095|4|9|8|3|2095|784|10211|Thursday|2095Q3|N|N|N|2486487|2486729|2486129|2486403|N|N|N|N|N| +2486495|AAAAAAAAPNAPFCAA|2095-09-09|2348|10211|784|2095|5|9|9|3|2095|784|10211|Friday|2095Q3|N|Y|N|2486487|2486729|2486130|2486404|N|N|N|N|N| +2486496|AAAAAAAAAOAPFCAA|2095-09-10|2348|10211|784|2095|6|9|10|3|2095|784|10211|Saturday|2095Q3|N|Y|N|2486487|2486729|2486131|2486405|N|N|N|N|N| +2486497|AAAAAAAABOAPFCAA|2095-09-11|2348|10211|784|2095|0|9|11|3|2095|784|10211|Sunday|2095Q3|N|N|N|2486487|2486729|2486132|2486406|N|N|N|N|N| +2486498|AAAAAAAACOAPFCAA|2095-09-12|2348|10211|784|2095|1|9|12|3|2095|784|10211|Monday|2095Q3|N|N|N|2486487|2486729|2486133|2486407|N|N|N|N|N| +2486499|AAAAAAAADOAPFCAA|2095-09-13|2348|10212|784|2095|2|9|13|3|2095|784|10212|Tuesday|2095Q3|N|N|N|2486487|2486729|2486134|2486408|N|N|N|N|N| +2486500|AAAAAAAAEOAPFCAA|2095-09-14|2348|10212|784|2095|3|9|14|3|2095|784|10212|Wednesday|2095Q3|N|N|N|2486487|2486729|2486135|2486409|N|N|N|N|N| +2486501|AAAAAAAAFOAPFCAA|2095-09-15|2348|10212|784|2095|4|9|15|3|2095|784|10212|Thursday|2095Q3|N|N|N|2486487|2486729|2486136|2486410|N|N|N|N|N| +2486502|AAAAAAAAGOAPFCAA|2095-09-16|2348|10212|784|2095|5|9|16|3|2095|784|10212|Friday|2095Q3|N|Y|N|2486487|2486729|2486137|2486411|N|N|N|N|N| +2486503|AAAAAAAAHOAPFCAA|2095-09-17|2348|10212|784|2095|6|9|17|3|2095|784|10212|Saturday|2095Q3|N|Y|N|2486487|2486729|2486138|2486412|N|N|N|N|N| +2486504|AAAAAAAAIOAPFCAA|2095-09-18|2348|10212|784|2095|0|9|18|3|2095|784|10212|Sunday|2095Q3|N|N|N|2486487|2486729|2486139|2486413|N|N|N|N|N| +2486505|AAAAAAAAJOAPFCAA|2095-09-19|2348|10212|784|2095|1|9|19|3|2095|784|10212|Monday|2095Q3|N|N|N|2486487|2486729|2486140|2486414|N|N|N|N|N| +2486506|AAAAAAAAKOAPFCAA|2095-09-20|2348|10213|784|2095|2|9|20|3|2095|784|10213|Tuesday|2095Q3|N|N|N|2486487|2486729|2486141|2486415|N|N|N|N|N| +2486507|AAAAAAAALOAPFCAA|2095-09-21|2348|10213|784|2095|3|9|21|3|2095|784|10213|Wednesday|2095Q3|N|N|N|2486487|2486729|2486142|2486416|N|N|N|N|N| +2486508|AAAAAAAAMOAPFCAA|2095-09-22|2348|10213|784|2095|4|9|22|3|2095|784|10213|Thursday|2095Q3|N|N|N|2486487|2486729|2486143|2486417|N|N|N|N|N| +2486509|AAAAAAAANOAPFCAA|2095-09-23|2348|10213|784|2095|5|9|23|3|2095|784|10213|Friday|2095Q3|N|Y|N|2486487|2486729|2486144|2486418|N|N|N|N|N| +2486510|AAAAAAAAOOAPFCAA|2095-09-24|2348|10213|784|2095|6|9|24|3|2095|784|10213|Saturday|2095Q3|N|Y|N|2486487|2486729|2486145|2486419|N|N|N|N|N| +2486511|AAAAAAAAPOAPFCAA|2095-09-25|2348|10213|784|2095|0|9|25|3|2095|784|10213|Sunday|2095Q3|N|N|N|2486487|2486729|2486146|2486420|N|N|N|N|N| +2486512|AAAAAAAAAPAPFCAA|2095-09-26|2348|10213|784|2095|1|9|26|3|2095|784|10213|Monday|2095Q3|N|N|N|2486487|2486729|2486147|2486421|N|N|N|N|N| +2486513|AAAAAAAABPAPFCAA|2095-09-27|2348|10214|784|2095|2|9|27|3|2095|784|10214|Tuesday|2095Q3|N|N|N|2486487|2486729|2486148|2486422|N|N|N|N|N| +2486514|AAAAAAAACPAPFCAA|2095-09-28|2348|10214|784|2095|3|9|28|3|2095|784|10214|Wednesday|2095Q3|N|N|N|2486487|2486729|2486149|2486423|N|N|N|N|N| +2486515|AAAAAAAADPAPFCAA|2095-09-29|2348|10214|784|2095|4|9|29|3|2095|784|10214|Thursday|2095Q3|N|N|N|2486487|2486729|2486150|2486424|N|N|N|N|N| +2486516|AAAAAAAAEPAPFCAA|2095-09-30|2348|10214|784|2095|5|9|30|3|2095|784|10214|Friday|2095Q3|N|Y|N|2486487|2486729|2486151|2486425|N|N|N|N|N| +2486517|AAAAAAAAFPAPFCAA|2095-10-01|2349|10214|784|2095|6|10|1|3|2095|784|10214|Saturday|2095Q3|N|Y|N|2486517|2486789|2486152|2486425|N|N|N|N|N| +2486518|AAAAAAAAGPAPFCAA|2095-10-02|2349|10214|784|2095|0|10|2|4|2095|784|10214|Sunday|2095Q4|N|N|N|2486517|2486789|2486153|2486426|N|N|N|N|N| +2486519|AAAAAAAAHPAPFCAA|2095-10-03|2349|10214|784|2095|1|10|3|4|2095|784|10214|Monday|2095Q4|N|N|N|2486517|2486789|2486154|2486427|N|N|N|N|N| +2486520|AAAAAAAAIPAPFCAA|2095-10-04|2349|10215|784|2095|2|10|4|4|2095|784|10215|Tuesday|2095Q4|N|N|N|2486517|2486789|2486155|2486428|N|N|N|N|N| +2486521|AAAAAAAAJPAPFCAA|2095-10-05|2349|10215|784|2095|3|10|5|4|2095|784|10215|Wednesday|2095Q4|N|N|N|2486517|2486789|2486156|2486429|N|N|N|N|N| +2486522|AAAAAAAAKPAPFCAA|2095-10-06|2349|10215|784|2095|4|10|6|4|2095|784|10215|Thursday|2095Q4|N|N|N|2486517|2486789|2486157|2486430|N|N|N|N|N| +2486523|AAAAAAAALPAPFCAA|2095-10-07|2349|10215|784|2095|5|10|7|4|2095|784|10215|Friday|2095Q4|N|Y|N|2486517|2486789|2486158|2486431|N|N|N|N|N| +2486524|AAAAAAAAMPAPFCAA|2095-10-08|2349|10215|784|2095|6|10|8|4|2095|784|10215|Saturday|2095Q4|N|Y|N|2486517|2486789|2486159|2486432|N|N|N|N|N| +2486525|AAAAAAAANPAPFCAA|2095-10-09|2349|10215|784|2095|0|10|9|4|2095|784|10215|Sunday|2095Q4|N|N|N|2486517|2486789|2486160|2486433|N|N|N|N|N| +2486526|AAAAAAAAOPAPFCAA|2095-10-10|2349|10215|784|2095|1|10|10|4|2095|784|10215|Monday|2095Q4|N|N|N|2486517|2486789|2486161|2486434|N|N|N|N|N| +2486527|AAAAAAAAPPAPFCAA|2095-10-11|2349|10216|784|2095|2|10|11|4|2095|784|10216|Tuesday|2095Q4|N|N|N|2486517|2486789|2486162|2486435|N|N|N|N|N| +2486528|AAAAAAAAAABPFCAA|2095-10-12|2349|10216|784|2095|3|10|12|4|2095|784|10216|Wednesday|2095Q4|N|N|N|2486517|2486789|2486163|2486436|N|N|N|N|N| +2486529|AAAAAAAABABPFCAA|2095-10-13|2349|10216|784|2095|4|10|13|4|2095|784|10216|Thursday|2095Q4|N|N|N|2486517|2486789|2486164|2486437|N|N|N|N|N| +2486530|AAAAAAAACABPFCAA|2095-10-14|2349|10216|784|2095|5|10|14|4|2095|784|10216|Friday|2095Q4|N|Y|N|2486517|2486789|2486165|2486438|N|N|N|N|N| +2486531|AAAAAAAADABPFCAA|2095-10-15|2349|10216|784|2095|6|10|15|4|2095|784|10216|Saturday|2095Q4|N|Y|N|2486517|2486789|2486166|2486439|N|N|N|N|N| +2486532|AAAAAAAAEABPFCAA|2095-10-16|2349|10216|784|2095|0|10|16|4|2095|784|10216|Sunday|2095Q4|N|N|N|2486517|2486789|2486167|2486440|N|N|N|N|N| +2486533|AAAAAAAAFABPFCAA|2095-10-17|2349|10216|784|2095|1|10|17|4|2095|784|10216|Monday|2095Q4|N|N|N|2486517|2486789|2486168|2486441|N|N|N|N|N| +2486534|AAAAAAAAGABPFCAA|2095-10-18|2349|10217|784|2095|2|10|18|4|2095|784|10217|Tuesday|2095Q4|N|N|N|2486517|2486789|2486169|2486442|N|N|N|N|N| +2486535|AAAAAAAAHABPFCAA|2095-10-19|2349|10217|784|2095|3|10|19|4|2095|784|10217|Wednesday|2095Q4|N|N|N|2486517|2486789|2486170|2486443|N|N|N|N|N| +2486536|AAAAAAAAIABPFCAA|2095-10-20|2349|10217|784|2095|4|10|20|4|2095|784|10217|Thursday|2095Q4|N|N|N|2486517|2486789|2486171|2486444|N|N|N|N|N| +2486537|AAAAAAAAJABPFCAA|2095-10-21|2349|10217|784|2095|5|10|21|4|2095|784|10217|Friday|2095Q4|N|Y|N|2486517|2486789|2486172|2486445|N|N|N|N|N| +2486538|AAAAAAAAKABPFCAA|2095-10-22|2349|10217|784|2095|6|10|22|4|2095|784|10217|Saturday|2095Q4|N|Y|N|2486517|2486789|2486173|2486446|N|N|N|N|N| +2486539|AAAAAAAALABPFCAA|2095-10-23|2349|10217|784|2095|0|10|23|4|2095|784|10217|Sunday|2095Q4|N|N|N|2486517|2486789|2486174|2486447|N|N|N|N|N| +2486540|AAAAAAAAMABPFCAA|2095-10-24|2349|10217|784|2095|1|10|24|4|2095|784|10217|Monday|2095Q4|N|N|N|2486517|2486789|2486175|2486448|N|N|N|N|N| +2486541|AAAAAAAANABPFCAA|2095-10-25|2349|10218|784|2095|2|10|25|4|2095|784|10218|Tuesday|2095Q4|N|N|N|2486517|2486789|2486176|2486449|N|N|N|N|N| +2486542|AAAAAAAAOABPFCAA|2095-10-26|2349|10218|784|2095|3|10|26|4|2095|784|10218|Wednesday|2095Q4|N|N|N|2486517|2486789|2486177|2486450|N|N|N|N|N| +2486543|AAAAAAAAPABPFCAA|2095-10-27|2349|10218|784|2095|4|10|27|4|2095|784|10218|Thursday|2095Q4|N|N|N|2486517|2486789|2486178|2486451|N|N|N|N|N| +2486544|AAAAAAAAABBPFCAA|2095-10-28|2349|10218|784|2095|5|10|28|4|2095|784|10218|Friday|2095Q4|N|Y|N|2486517|2486789|2486179|2486452|N|N|N|N|N| +2486545|AAAAAAAABBBPFCAA|2095-10-29|2349|10218|784|2095|6|10|29|4|2095|784|10218|Saturday|2095Q4|N|Y|N|2486517|2486789|2486180|2486453|N|N|N|N|N| +2486546|AAAAAAAACBBPFCAA|2095-10-30|2349|10218|784|2095|0|10|30|4|2095|784|10218|Sunday|2095Q4|N|N|N|2486517|2486789|2486181|2486454|N|N|N|N|N| +2486547|AAAAAAAADBBPFCAA|2095-10-31|2349|10218|784|2095|1|10|31|4|2095|784|10218|Monday|2095Q4|N|N|N|2486517|2486789|2486182|2486455|N|N|N|N|N| +2486548|AAAAAAAAEBBPFCAA|2095-11-01|2350|10219|784|2095|2|11|1|4|2095|784|10219|Tuesday|2095Q4|N|N|N|2486548|2486851|2486183|2486456|N|N|N|N|N| +2486549|AAAAAAAAFBBPFCAA|2095-11-02|2350|10219|784|2095|3|11|2|4|2095|784|10219|Wednesday|2095Q4|N|N|N|2486548|2486851|2486184|2486457|N|N|N|N|N| +2486550|AAAAAAAAGBBPFCAA|2095-11-03|2350|10219|784|2095|4|11|3|4|2095|784|10219|Thursday|2095Q4|N|N|N|2486548|2486851|2486185|2486458|N|N|N|N|N| +2486551|AAAAAAAAHBBPFCAA|2095-11-04|2350|10219|784|2095|5|11|4|4|2095|784|10219|Friday|2095Q4|N|Y|N|2486548|2486851|2486186|2486459|N|N|N|N|N| +2486552|AAAAAAAAIBBPFCAA|2095-11-05|2350|10219|784|2095|6|11|5|4|2095|784|10219|Saturday|2095Q4|N|Y|N|2486548|2486851|2486187|2486460|N|N|N|N|N| +2486553|AAAAAAAAJBBPFCAA|2095-11-06|2350|10219|784|2095|0|11|6|4|2095|784|10219|Sunday|2095Q4|N|N|N|2486548|2486851|2486188|2486461|N|N|N|N|N| +2486554|AAAAAAAAKBBPFCAA|2095-11-07|2350|10219|784|2095|1|11|7|4|2095|784|10219|Monday|2095Q4|N|N|N|2486548|2486851|2486189|2486462|N|N|N|N|N| +2486555|AAAAAAAALBBPFCAA|2095-11-08|2350|10220|784|2095|2|11|8|4|2095|784|10220|Tuesday|2095Q4|N|N|N|2486548|2486851|2486190|2486463|N|N|N|N|N| +2486556|AAAAAAAAMBBPFCAA|2095-11-09|2350|10220|784|2095|3|11|9|4|2095|784|10220|Wednesday|2095Q4|N|N|N|2486548|2486851|2486191|2486464|N|N|N|N|N| +2486557|AAAAAAAANBBPFCAA|2095-11-10|2350|10220|784|2095|4|11|10|4|2095|784|10220|Thursday|2095Q4|N|N|N|2486548|2486851|2486192|2486465|N|N|N|N|N| +2486558|AAAAAAAAOBBPFCAA|2095-11-11|2350|10220|784|2095|5|11|11|4|2095|784|10220|Friday|2095Q4|N|Y|N|2486548|2486851|2486193|2486466|N|N|N|N|N| +2486559|AAAAAAAAPBBPFCAA|2095-11-12|2350|10220|784|2095|6|11|12|4|2095|784|10220|Saturday|2095Q4|N|Y|N|2486548|2486851|2486194|2486467|N|N|N|N|N| +2486560|AAAAAAAAACBPFCAA|2095-11-13|2350|10220|784|2095|0|11|13|4|2095|784|10220|Sunday|2095Q4|N|N|N|2486548|2486851|2486195|2486468|N|N|N|N|N| +2486561|AAAAAAAABCBPFCAA|2095-11-14|2350|10220|784|2095|1|11|14|4|2095|784|10220|Monday|2095Q4|N|N|N|2486548|2486851|2486196|2486469|N|N|N|N|N| +2486562|AAAAAAAACCBPFCAA|2095-11-15|2350|10221|784|2095|2|11|15|4|2095|784|10221|Tuesday|2095Q4|N|N|N|2486548|2486851|2486197|2486470|N|N|N|N|N| +2486563|AAAAAAAADCBPFCAA|2095-11-16|2350|10221|784|2095|3|11|16|4|2095|784|10221|Wednesday|2095Q4|N|N|N|2486548|2486851|2486198|2486471|N|N|N|N|N| +2486564|AAAAAAAAECBPFCAA|2095-11-17|2350|10221|784|2095|4|11|17|4|2095|784|10221|Thursday|2095Q4|N|N|N|2486548|2486851|2486199|2486472|N|N|N|N|N| +2486565|AAAAAAAAFCBPFCAA|2095-11-18|2350|10221|784|2095|5|11|18|4|2095|784|10221|Friday|2095Q4|N|Y|N|2486548|2486851|2486200|2486473|N|N|N|N|N| +2486566|AAAAAAAAGCBPFCAA|2095-11-19|2350|10221|784|2095|6|11|19|4|2095|784|10221|Saturday|2095Q4|N|Y|N|2486548|2486851|2486201|2486474|N|N|N|N|N| +2486567|AAAAAAAAHCBPFCAA|2095-11-20|2350|10221|784|2095|0|11|20|4|2095|784|10221|Sunday|2095Q4|N|N|N|2486548|2486851|2486202|2486475|N|N|N|N|N| +2486568|AAAAAAAAICBPFCAA|2095-11-21|2350|10221|784|2095|1|11|21|4|2095|784|10221|Monday|2095Q4|N|N|N|2486548|2486851|2486203|2486476|N|N|N|N|N| +2486569|AAAAAAAAJCBPFCAA|2095-11-22|2350|10222|784|2095|2|11|22|4|2095|784|10222|Tuesday|2095Q4|N|N|N|2486548|2486851|2486204|2486477|N|N|N|N|N| +2486570|AAAAAAAAKCBPFCAA|2095-11-23|2350|10222|784|2095|3|11|23|4|2095|784|10222|Wednesday|2095Q4|N|N|N|2486548|2486851|2486205|2486478|N|N|N|N|N| +2486571|AAAAAAAALCBPFCAA|2095-11-24|2350|10222|784|2095|4|11|24|4|2095|784|10222|Thursday|2095Q4|N|N|N|2486548|2486851|2486206|2486479|N|N|N|N|N| +2486572|AAAAAAAAMCBPFCAA|2095-11-25|2350|10222|784|2095|5|11|25|4|2095|784|10222|Friday|2095Q4|N|Y|N|2486548|2486851|2486207|2486480|N|N|N|N|N| +2486573|AAAAAAAANCBPFCAA|2095-11-26|2350|10222|784|2095|6|11|26|4|2095|784|10222|Saturday|2095Q4|N|Y|N|2486548|2486851|2486208|2486481|N|N|N|N|N| +2486574|AAAAAAAAOCBPFCAA|2095-11-27|2350|10222|784|2095|0|11|27|4|2095|784|10222|Sunday|2095Q4|N|N|N|2486548|2486851|2486209|2486482|N|N|N|N|N| +2486575|AAAAAAAAPCBPFCAA|2095-11-28|2350|10222|784|2095|1|11|28|4|2095|784|10222|Monday|2095Q4|N|N|N|2486548|2486851|2486210|2486483|N|N|N|N|N| +2486576|AAAAAAAAADBPFCAA|2095-11-29|2350|10223|784|2095|2|11|29|4|2095|784|10223|Tuesday|2095Q4|N|N|N|2486548|2486851|2486211|2486484|N|N|N|N|N| +2486577|AAAAAAAABDBPFCAA|2095-11-30|2350|10223|784|2095|3|11|30|4|2095|784|10223|Wednesday|2095Q4|N|N|N|2486548|2486851|2486212|2486485|N|N|N|N|N| +2486578|AAAAAAAACDBPFCAA|2095-12-01|2351|10223|785|2095|4|12|1|4|2095|785|10223|Thursday|2095Q4|N|N|N|2486578|2486911|2486213|2486486|N|N|N|N|N| +2486579|AAAAAAAADDBPFCAA|2095-12-02|2351|10223|785|2095|5|12|2|4|2095|785|10223|Friday|2095Q4|N|Y|N|2486578|2486911|2486214|2486487|N|N|N|N|N| +2486580|AAAAAAAAEDBPFCAA|2095-12-03|2351|10223|785|2095|6|12|3|4|2095|785|10223|Saturday|2095Q4|N|Y|N|2486578|2486911|2486215|2486488|N|N|N|N|N| +2486581|AAAAAAAAFDBPFCAA|2095-12-04|2351|10223|785|2095|0|12|4|4|2095|785|10223|Sunday|2095Q4|N|N|N|2486578|2486911|2486216|2486489|N|N|N|N|N| +2486582|AAAAAAAAGDBPFCAA|2095-12-05|2351|10223|785|2095|1|12|5|4|2095|785|10223|Monday|2095Q4|N|N|N|2486578|2486911|2486217|2486490|N|N|N|N|N| +2486583|AAAAAAAAHDBPFCAA|2095-12-06|2351|10224|785|2095|2|12|6|4|2095|785|10224|Tuesday|2095Q4|N|N|N|2486578|2486911|2486218|2486491|N|N|N|N|N| +2486584|AAAAAAAAIDBPFCAA|2095-12-07|2351|10224|785|2095|3|12|7|4|2095|785|10224|Wednesday|2095Q4|N|N|N|2486578|2486911|2486219|2486492|N|N|N|N|N| +2486585|AAAAAAAAJDBPFCAA|2095-12-08|2351|10224|785|2095|4|12|8|4|2095|785|10224|Thursday|2095Q4|N|N|N|2486578|2486911|2486220|2486493|N|N|N|N|N| +2486586|AAAAAAAAKDBPFCAA|2095-12-09|2351|10224|785|2095|5|12|9|4|2095|785|10224|Friday|2095Q4|N|Y|N|2486578|2486911|2486221|2486494|N|N|N|N|N| +2486587|AAAAAAAALDBPFCAA|2095-12-10|2351|10224|785|2095|6|12|10|4|2095|785|10224|Saturday|2095Q4|N|Y|N|2486578|2486911|2486222|2486495|N|N|N|N|N| +2486588|AAAAAAAAMDBPFCAA|2095-12-11|2351|10224|785|2095|0|12|11|4|2095|785|10224|Sunday|2095Q4|N|N|N|2486578|2486911|2486223|2486496|N|N|N|N|N| +2486589|AAAAAAAANDBPFCAA|2095-12-12|2351|10224|785|2095|1|12|12|4|2095|785|10224|Monday|2095Q4|N|N|N|2486578|2486911|2486224|2486497|N|N|N|N|N| +2486590|AAAAAAAAODBPFCAA|2095-12-13|2351|10225|785|2095|2|12|13|4|2095|785|10225|Tuesday|2095Q4|N|N|N|2486578|2486911|2486225|2486498|N|N|N|N|N| +2486591|AAAAAAAAPDBPFCAA|2095-12-14|2351|10225|785|2095|3|12|14|4|2095|785|10225|Wednesday|2095Q4|N|N|N|2486578|2486911|2486226|2486499|N|N|N|N|N| +2486592|AAAAAAAAAEBPFCAA|2095-12-15|2351|10225|785|2095|4|12|15|4|2095|785|10225|Thursday|2095Q4|N|N|N|2486578|2486911|2486227|2486500|N|N|N|N|N| +2486593|AAAAAAAABEBPFCAA|2095-12-16|2351|10225|785|2095|5|12|16|4|2095|785|10225|Friday|2095Q4|N|Y|N|2486578|2486911|2486228|2486501|N|N|N|N|N| +2486594|AAAAAAAACEBPFCAA|2095-12-17|2351|10225|785|2095|6|12|17|4|2095|785|10225|Saturday|2095Q4|N|Y|N|2486578|2486911|2486229|2486502|N|N|N|N|N| +2486595|AAAAAAAADEBPFCAA|2095-12-18|2351|10225|785|2095|0|12|18|4|2095|785|10225|Sunday|2095Q4|N|N|N|2486578|2486911|2486230|2486503|N|N|N|N|N| +2486596|AAAAAAAAEEBPFCAA|2095-12-19|2351|10225|785|2095|1|12|19|4|2095|785|10225|Monday|2095Q4|N|N|N|2486578|2486911|2486231|2486504|N|N|N|N|N| +2486597|AAAAAAAAFEBPFCAA|2095-12-20|2351|10226|785|2095|2|12|20|4|2095|785|10226|Tuesday|2095Q4|N|N|N|2486578|2486911|2486232|2486505|N|N|N|N|N| +2486598|AAAAAAAAGEBPFCAA|2095-12-21|2351|10226|785|2095|3|12|21|4|2095|785|10226|Wednesday|2095Q4|N|N|N|2486578|2486911|2486233|2486506|N|N|N|N|N| +2486599|AAAAAAAAHEBPFCAA|2095-12-22|2351|10226|785|2095|4|12|22|4|2095|785|10226|Thursday|2095Q4|N|N|N|2486578|2486911|2486234|2486507|N|N|N|N|N| +2486600|AAAAAAAAIEBPFCAA|2095-12-23|2351|10226|785|2095|5|12|23|4|2095|785|10226|Friday|2095Q4|N|Y|N|2486578|2486911|2486235|2486508|N|N|N|N|N| +2486601|AAAAAAAAJEBPFCAA|2095-12-24|2351|10226|785|2095|6|12|24|4|2095|785|10226|Saturday|2095Q4|N|Y|N|2486578|2486911|2486236|2486509|N|N|N|N|N| +2486602|AAAAAAAAKEBPFCAA|2095-12-25|2351|10226|785|2095|0|12|25|4|2095|785|10226|Sunday|2095Q4|N|N|N|2486578|2486911|2486237|2486510|N|N|N|N|N| +2486603|AAAAAAAALEBPFCAA|2095-12-26|2351|10226|785|2095|1|12|26|4|2095|785|10226|Monday|2095Q4|Y|N|N|2486578|2486911|2486238|2486511|N|N|N|N|N| +2486604|AAAAAAAAMEBPFCAA|2095-12-27|2351|10227|785|2095|2|12|27|4|2095|785|10227|Tuesday|2095Q4|N|N|Y|2486578|2486911|2486239|2486512|N|N|N|N|N| +2486605|AAAAAAAANEBPFCAA|2095-12-28|2351|10227|785|2095|3|12|28|4|2095|785|10227|Wednesday|2095Q4|N|N|N|2486578|2486911|2486240|2486513|N|N|N|N|N| +2486606|AAAAAAAAOEBPFCAA|2095-12-29|2351|10227|785|2095|4|12|29|4|2095|785|10227|Thursday|2095Q4|N|N|N|2486578|2486911|2486241|2486514|N|N|N|N|N| +2486607|AAAAAAAAPEBPFCAA|2095-12-30|2351|10227|785|2095|5|12|30|4|2095|785|10227|Friday|2095Q4|N|Y|N|2486578|2486911|2486242|2486515|N|N|N|N|N| +2486608|AAAAAAAAAFBPFCAA|2095-12-31|2351|10227|785|2095|6|12|31|4|2095|785|10227|Saturday|2095Q4|N|Y|N|2486578|2486911|2486243|2486516|N|N|N|N|N| +2486609|AAAAAAAABFBPFCAA|2096-01-01|2352|10227|785|2096|0|1|1|1|2096|785|10227|Sunday|2096Q1|Y|N|N|2486609|2486608|2486244|2486517|N|N|N|N|N| +2486610|AAAAAAAACFBPFCAA|2096-01-02|2352|10227|785|2096|1|1|2|1|2096|785|10227|Monday|2096Q1|N|N|Y|2486609|2486608|2486245|2486518|N|N|N|N|N| +2486611|AAAAAAAADFBPFCAA|2096-01-03|2352|10228|785|2096|2|1|3|1|2096|785|10228|Tuesday|2096Q1|N|N|N|2486609|2486608|2486246|2486519|N|N|N|N|N| +2486612|AAAAAAAAEFBPFCAA|2096-01-04|2352|10228|785|2096|3|1|4|1|2096|785|10228|Wednesday|2096Q1|N|N|N|2486609|2486608|2486247|2486520|N|N|N|N|N| +2486613|AAAAAAAAFFBPFCAA|2096-01-05|2352|10228|785|2096|4|1|5|1|2096|785|10228|Thursday|2096Q1|N|N|N|2486609|2486608|2486248|2486521|N|N|N|N|N| +2486614|AAAAAAAAGFBPFCAA|2096-01-06|2352|10228|785|2096|5|1|6|1|2096|785|10228|Friday|2096Q1|N|Y|N|2486609|2486608|2486249|2486522|N|N|N|N|N| +2486615|AAAAAAAAHFBPFCAA|2096-01-07|2352|10228|785|2096|6|1|7|1|2096|785|10228|Saturday|2096Q1|N|Y|N|2486609|2486608|2486250|2486523|N|N|N|N|N| +2486616|AAAAAAAAIFBPFCAA|2096-01-08|2352|10228|785|2096|0|1|8|1|2096|785|10228|Sunday|2096Q1|N|N|N|2486609|2486608|2486251|2486524|N|N|N|N|N| +2486617|AAAAAAAAJFBPFCAA|2096-01-09|2352|10228|785|2096|1|1|9|1|2096|785|10228|Monday|2096Q1|N|N|N|2486609|2486608|2486252|2486525|N|N|N|N|N| +2486618|AAAAAAAAKFBPFCAA|2096-01-10|2352|10229|785|2096|2|1|10|1|2096|785|10229|Tuesday|2096Q1|N|N|N|2486609|2486608|2486253|2486526|N|N|N|N|N| +2486619|AAAAAAAALFBPFCAA|2096-01-11|2352|10229|785|2096|3|1|11|1|2096|785|10229|Wednesday|2096Q1|N|N|N|2486609|2486608|2486254|2486527|N|N|N|N|N| +2486620|AAAAAAAAMFBPFCAA|2096-01-12|2352|10229|785|2096|4|1|12|1|2096|785|10229|Thursday|2096Q1|N|N|N|2486609|2486608|2486255|2486528|N|N|N|N|N| +2486621|AAAAAAAANFBPFCAA|2096-01-13|2352|10229|785|2096|5|1|13|1|2096|785|10229|Friday|2096Q1|N|Y|N|2486609|2486608|2486256|2486529|N|N|N|N|N| +2486622|AAAAAAAAOFBPFCAA|2096-01-14|2352|10229|785|2096|6|1|14|1|2096|785|10229|Saturday|2096Q1|N|Y|N|2486609|2486608|2486257|2486530|N|N|N|N|N| +2486623|AAAAAAAAPFBPFCAA|2096-01-15|2352|10229|785|2096|0|1|15|1|2096|785|10229|Sunday|2096Q1|N|N|N|2486609|2486608|2486258|2486531|N|N|N|N|N| +2486624|AAAAAAAAAGBPFCAA|2096-01-16|2352|10229|785|2096|1|1|16|1|2096|785|10229|Monday|2096Q1|N|N|N|2486609|2486608|2486259|2486532|N|N|N|N|N| +2486625|AAAAAAAABGBPFCAA|2096-01-17|2352|10230|785|2096|2|1|17|1|2096|785|10230|Tuesday|2096Q1|N|N|N|2486609|2486608|2486260|2486533|N|N|N|N|N| +2486626|AAAAAAAACGBPFCAA|2096-01-18|2352|10230|785|2096|3|1|18|1|2096|785|10230|Wednesday|2096Q1|N|N|N|2486609|2486608|2486261|2486534|N|N|N|N|N| +2486627|AAAAAAAADGBPFCAA|2096-01-19|2352|10230|785|2096|4|1|19|1|2096|785|10230|Thursday|2096Q1|N|N|N|2486609|2486608|2486262|2486535|N|N|N|N|N| +2486628|AAAAAAAAEGBPFCAA|2096-01-20|2352|10230|785|2096|5|1|20|1|2096|785|10230|Friday|2096Q1|N|Y|N|2486609|2486608|2486263|2486536|N|N|N|N|N| +2486629|AAAAAAAAFGBPFCAA|2096-01-21|2352|10230|785|2096|6|1|21|1|2096|785|10230|Saturday|2096Q1|N|Y|N|2486609|2486608|2486264|2486537|N|N|N|N|N| +2486630|AAAAAAAAGGBPFCAA|2096-01-22|2352|10230|785|2096|0|1|22|1|2096|785|10230|Sunday|2096Q1|N|N|N|2486609|2486608|2486265|2486538|N|N|N|N|N| +2486631|AAAAAAAAHGBPFCAA|2096-01-23|2352|10230|785|2096|1|1|23|1|2096|785|10230|Monday|2096Q1|N|N|N|2486609|2486608|2486266|2486539|N|N|N|N|N| +2486632|AAAAAAAAIGBPFCAA|2096-01-24|2352|10231|785|2096|2|1|24|1|2096|785|10231|Tuesday|2096Q1|N|N|N|2486609|2486608|2486267|2486540|N|N|N|N|N| +2486633|AAAAAAAAJGBPFCAA|2096-01-25|2352|10231|785|2096|3|1|25|1|2096|785|10231|Wednesday|2096Q1|N|N|N|2486609|2486608|2486268|2486541|N|N|N|N|N| +2486634|AAAAAAAAKGBPFCAA|2096-01-26|2352|10231|785|2096|4|1|26|1|2096|785|10231|Thursday|2096Q1|N|N|N|2486609|2486608|2486269|2486542|N|N|N|N|N| +2486635|AAAAAAAALGBPFCAA|2096-01-27|2352|10231|785|2096|5|1|27|1|2096|785|10231|Friday|2096Q1|N|Y|N|2486609|2486608|2486270|2486543|N|N|N|N|N| +2486636|AAAAAAAAMGBPFCAA|2096-01-28|2352|10231|785|2096|6|1|28|1|2096|785|10231|Saturday|2096Q1|N|Y|N|2486609|2486608|2486271|2486544|N|N|N|N|N| +2486637|AAAAAAAANGBPFCAA|2096-01-29|2352|10231|785|2096|0|1|29|1|2096|785|10231|Sunday|2096Q1|N|N|N|2486609|2486608|2486272|2486545|N|N|N|N|N| +2486638|AAAAAAAAOGBPFCAA|2096-01-30|2352|10231|785|2096|1|1|30|1|2096|785|10231|Monday|2096Q1|N|N|N|2486609|2486608|2486273|2486546|N|N|N|N|N| +2486639|AAAAAAAAPGBPFCAA|2096-01-31|2352|10232|785|2096|2|1|31|1|2096|785|10232|Tuesday|2096Q1|N|N|N|2486609|2486608|2486274|2486547|N|N|N|N|N| +2486640|AAAAAAAAAHBPFCAA|2096-02-01|2353|10232|785|2096|3|2|1|1|2096|785|10232|Wednesday|2096Q1|N|N|N|2486640|2486670|2486275|2486548|N|N|N|N|N| +2486641|AAAAAAAABHBPFCAA|2096-02-02|2353|10232|785|2096|4|2|2|1|2096|785|10232|Thursday|2096Q1|N|N|N|2486640|2486670|2486276|2486549|N|N|N|N|N| +2486642|AAAAAAAACHBPFCAA|2096-02-03|2353|10232|785|2096|5|2|3|1|2096|785|10232|Friday|2096Q1|N|Y|N|2486640|2486670|2486277|2486550|N|N|N|N|N| +2486643|AAAAAAAADHBPFCAA|2096-02-04|2353|10232|785|2096|6|2|4|1|2096|785|10232|Saturday|2096Q1|N|Y|N|2486640|2486670|2486278|2486551|N|N|N|N|N| +2486644|AAAAAAAAEHBPFCAA|2096-02-05|2353|10232|785|2096|0|2|5|1|2096|785|10232|Sunday|2096Q1|N|N|N|2486640|2486670|2486279|2486552|N|N|N|N|N| +2486645|AAAAAAAAFHBPFCAA|2096-02-06|2353|10232|785|2096|1|2|6|1|2096|785|10232|Monday|2096Q1|N|N|N|2486640|2486670|2486280|2486553|N|N|N|N|N| +2486646|AAAAAAAAGHBPFCAA|2096-02-07|2353|10233|785|2096|2|2|7|1|2096|785|10233|Tuesday|2096Q1|N|N|N|2486640|2486670|2486281|2486554|N|N|N|N|N| +2486647|AAAAAAAAHHBPFCAA|2096-02-08|2353|10233|785|2096|3|2|8|1|2096|785|10233|Wednesday|2096Q1|N|N|N|2486640|2486670|2486282|2486555|N|N|N|N|N| +2486648|AAAAAAAAIHBPFCAA|2096-02-09|2353|10233|785|2096|4|2|9|1|2096|785|10233|Thursday|2096Q1|N|N|N|2486640|2486670|2486283|2486556|N|N|N|N|N| +2486649|AAAAAAAAJHBPFCAA|2096-02-10|2353|10233|785|2096|5|2|10|1|2096|785|10233|Friday|2096Q1|N|Y|N|2486640|2486670|2486284|2486557|N|N|N|N|N| +2486650|AAAAAAAAKHBPFCAA|2096-02-11|2353|10233|785|2096|6|2|11|1|2096|785|10233|Saturday|2096Q1|N|Y|N|2486640|2486670|2486285|2486558|N|N|N|N|N| +2486651|AAAAAAAALHBPFCAA|2096-02-12|2353|10233|785|2096|0|2|12|1|2096|785|10233|Sunday|2096Q1|N|N|N|2486640|2486670|2486286|2486559|N|N|N|N|N| +2486652|AAAAAAAAMHBPFCAA|2096-02-13|2353|10233|785|2096|1|2|13|1|2096|785|10233|Monday|2096Q1|N|N|N|2486640|2486670|2486287|2486560|N|N|N|N|N| +2486653|AAAAAAAANHBPFCAA|2096-02-14|2353|10234|785|2096|2|2|14|1|2096|785|10234|Tuesday|2096Q1|N|N|N|2486640|2486670|2486288|2486561|N|N|N|N|N| +2486654|AAAAAAAAOHBPFCAA|2096-02-15|2353|10234|785|2096|3|2|15|1|2096|785|10234|Wednesday|2096Q1|N|N|N|2486640|2486670|2486289|2486562|N|N|N|N|N| +2486655|AAAAAAAAPHBPFCAA|2096-02-16|2353|10234|785|2096|4|2|16|1|2096|785|10234|Thursday|2096Q1|N|N|N|2486640|2486670|2486290|2486563|N|N|N|N|N| +2486656|AAAAAAAAAIBPFCAA|2096-02-17|2353|10234|785|2096|5|2|17|1|2096|785|10234|Friday|2096Q1|N|Y|N|2486640|2486670|2486291|2486564|N|N|N|N|N| +2486657|AAAAAAAABIBPFCAA|2096-02-18|2353|10234|785|2096|6|2|18|1|2096|785|10234|Saturday|2096Q1|N|Y|N|2486640|2486670|2486292|2486565|N|N|N|N|N| +2486658|AAAAAAAACIBPFCAA|2096-02-19|2353|10234|785|2096|0|2|19|1|2096|785|10234|Sunday|2096Q1|N|N|N|2486640|2486670|2486293|2486566|N|N|N|N|N| +2486659|AAAAAAAADIBPFCAA|2096-02-20|2353|10234|785|2096|1|2|20|1|2096|785|10234|Monday|2096Q1|N|N|N|2486640|2486670|2486294|2486567|N|N|N|N|N| +2486660|AAAAAAAAEIBPFCAA|2096-02-21|2353|10235|785|2096|2|2|21|1|2096|785|10235|Tuesday|2096Q1|N|N|N|2486640|2486670|2486295|2486568|N|N|N|N|N| +2486661|AAAAAAAAFIBPFCAA|2096-02-22|2353|10235|785|2096|3|2|22|1|2096|785|10235|Wednesday|2096Q1|N|N|N|2486640|2486670|2486296|2486569|N|N|N|N|N| +2486662|AAAAAAAAGIBPFCAA|2096-02-23|2353|10235|785|2096|4|2|23|1|2096|785|10235|Thursday|2096Q1|N|N|N|2486640|2486670|2486297|2486570|N|N|N|N|N| +2486663|AAAAAAAAHIBPFCAA|2096-02-24|2353|10235|785|2096|5|2|24|1|2096|785|10235|Friday|2096Q1|N|Y|N|2486640|2486670|2486298|2486571|N|N|N|N|N| +2486664|AAAAAAAAIIBPFCAA|2096-02-25|2353|10235|785|2096|6|2|25|1|2096|785|10235|Saturday|2096Q1|N|Y|N|2486640|2486670|2486299|2486572|N|N|N|N|N| +2486665|AAAAAAAAJIBPFCAA|2096-02-26|2353|10235|785|2096|0|2|26|1|2096|785|10235|Sunday|2096Q1|N|N|N|2486640|2486670|2486300|2486573|N|N|N|N|N| +2486666|AAAAAAAAKIBPFCAA|2096-02-27|2353|10235|785|2096|1|2|27|1|2096|785|10235|Monday|2096Q1|N|N|N|2486640|2486670|2486301|2486574|N|N|N|N|N| +2486667|AAAAAAAALIBPFCAA|2096-02-28|2353|10236|785|2096|2|2|28|1|2096|785|10236|Tuesday|2096Q1|N|N|N|2486640|2486670|2486302|2486575|N|N|N|N|N| +2486668|AAAAAAAAMIBPFCAA|2096-02-29|2353|10236|785|2096|3|2|29|1|2096|785|10236|Wednesday|2096Q1|N|N|N|2486640|2486670|2486302|2486576|N|N|N|N|N| +2486669|AAAAAAAANIBPFCAA|2096-03-01|2354|10236|786|2096|4|3|1|1|2096|786|10236|Thursday|2096Q1|N|N|N|2486669|2486728|2486303|2486577|N|N|N|N|N| +2486670|AAAAAAAAOIBPFCAA|2096-03-02|2354|10236|786|2096|5|3|2|1|2096|786|10236|Friday|2096Q1|N|Y|N|2486669|2486728|2486304|2486578|N|N|N|N|N| +2486671|AAAAAAAAPIBPFCAA|2096-03-03|2354|10236|786|2096|6|3|3|1|2096|786|10236|Saturday|2096Q1|N|Y|N|2486669|2486728|2486305|2486579|N|N|N|N|N| +2486672|AAAAAAAAAJBPFCAA|2096-03-04|2354|10236|786|2096|0|3|4|1|2096|786|10236|Sunday|2096Q1|N|N|N|2486669|2486728|2486306|2486580|N|N|N|N|N| +2486673|AAAAAAAABJBPFCAA|2096-03-05|2354|10236|786|2096|1|3|5|1|2096|786|10236|Monday|2096Q1|N|N|N|2486669|2486728|2486307|2486581|N|N|N|N|N| +2486674|AAAAAAAACJBPFCAA|2096-03-06|2354|10237|786|2096|2|3|6|1|2096|786|10237|Tuesday|2096Q1|N|N|N|2486669|2486728|2486308|2486582|N|N|N|N|N| +2486675|AAAAAAAADJBPFCAA|2096-03-07|2354|10237|786|2096|3|3|7|1|2096|786|10237|Wednesday|2096Q1|N|N|N|2486669|2486728|2486309|2486583|N|N|N|N|N| +2486676|AAAAAAAAEJBPFCAA|2096-03-08|2354|10237|786|2096|4|3|8|1|2096|786|10237|Thursday|2096Q1|N|N|N|2486669|2486728|2486310|2486584|N|N|N|N|N| +2486677|AAAAAAAAFJBPFCAA|2096-03-09|2354|10237|786|2096|5|3|9|1|2096|786|10237|Friday|2096Q1|N|Y|N|2486669|2486728|2486311|2486585|N|N|N|N|N| +2486678|AAAAAAAAGJBPFCAA|2096-03-10|2354|10237|786|2096|6|3|10|1|2096|786|10237|Saturday|2096Q1|N|Y|N|2486669|2486728|2486312|2486586|N|N|N|N|N| +2486679|AAAAAAAAHJBPFCAA|2096-03-11|2354|10237|786|2096|0|3|11|1|2096|786|10237|Sunday|2096Q1|N|N|N|2486669|2486728|2486313|2486587|N|N|N|N|N| +2486680|AAAAAAAAIJBPFCAA|2096-03-12|2354|10237|786|2096|1|3|12|1|2096|786|10237|Monday|2096Q1|N|N|N|2486669|2486728|2486314|2486588|N|N|N|N|N| +2486681|AAAAAAAAJJBPFCAA|2096-03-13|2354|10238|786|2096|2|3|13|1|2096|786|10238|Tuesday|2096Q1|N|N|N|2486669|2486728|2486315|2486589|N|N|N|N|N| +2486682|AAAAAAAAKJBPFCAA|2096-03-14|2354|10238|786|2096|3|3|14|1|2096|786|10238|Wednesday|2096Q1|N|N|N|2486669|2486728|2486316|2486590|N|N|N|N|N| +2486683|AAAAAAAALJBPFCAA|2096-03-15|2354|10238|786|2096|4|3|15|1|2096|786|10238|Thursday|2096Q1|N|N|N|2486669|2486728|2486317|2486591|N|N|N|N|N| +2486684|AAAAAAAAMJBPFCAA|2096-03-16|2354|10238|786|2096|5|3|16|1|2096|786|10238|Friday|2096Q1|N|Y|N|2486669|2486728|2486318|2486592|N|N|N|N|N| +2486685|AAAAAAAANJBPFCAA|2096-03-17|2354|10238|786|2096|6|3|17|1|2096|786|10238|Saturday|2096Q1|N|Y|N|2486669|2486728|2486319|2486593|N|N|N|N|N| +2486686|AAAAAAAAOJBPFCAA|2096-03-18|2354|10238|786|2096|0|3|18|1|2096|786|10238|Sunday|2096Q1|N|N|N|2486669|2486728|2486320|2486594|N|N|N|N|N| +2486687|AAAAAAAAPJBPFCAA|2096-03-19|2354|10238|786|2096|1|3|19|1|2096|786|10238|Monday|2096Q1|N|N|N|2486669|2486728|2486321|2486595|N|N|N|N|N| +2486688|AAAAAAAAAKBPFCAA|2096-03-20|2354|10239|786|2096|2|3|20|1|2096|786|10239|Tuesday|2096Q1|N|N|N|2486669|2486728|2486322|2486596|N|N|N|N|N| +2486689|AAAAAAAABKBPFCAA|2096-03-21|2354|10239|786|2096|3|3|21|1|2096|786|10239|Wednesday|2096Q1|N|N|N|2486669|2486728|2486323|2486597|N|N|N|N|N| +2486690|AAAAAAAACKBPFCAA|2096-03-22|2354|10239|786|2096|4|3|22|1|2096|786|10239|Thursday|2096Q1|N|N|N|2486669|2486728|2486324|2486598|N|N|N|N|N| +2486691|AAAAAAAADKBPFCAA|2096-03-23|2354|10239|786|2096|5|3|23|1|2096|786|10239|Friday|2096Q1|N|Y|N|2486669|2486728|2486325|2486599|N|N|N|N|N| +2486692|AAAAAAAAEKBPFCAA|2096-03-24|2354|10239|786|2096|6|3|24|1|2096|786|10239|Saturday|2096Q1|N|Y|N|2486669|2486728|2486326|2486600|N|N|N|N|N| +2486693|AAAAAAAAFKBPFCAA|2096-03-25|2354|10239|786|2096|0|3|25|1|2096|786|10239|Sunday|2096Q1|N|N|N|2486669|2486728|2486327|2486601|N|N|N|N|N| +2486694|AAAAAAAAGKBPFCAA|2096-03-26|2354|10239|786|2096|1|3|26|1|2096|786|10239|Monday|2096Q1|N|N|N|2486669|2486728|2486328|2486602|N|N|N|N|N| +2486695|AAAAAAAAHKBPFCAA|2096-03-27|2354|10240|786|2096|2|3|27|1|2096|786|10240|Tuesday|2096Q1|N|N|N|2486669|2486728|2486329|2486603|N|N|N|N|N| +2486696|AAAAAAAAIKBPFCAA|2096-03-28|2354|10240|786|2096|3|3|28|1|2096|786|10240|Wednesday|2096Q1|N|N|N|2486669|2486728|2486330|2486604|N|N|N|N|N| +2486697|AAAAAAAAJKBPFCAA|2096-03-29|2354|10240|786|2096|4|3|29|1|2096|786|10240|Thursday|2096Q1|N|N|N|2486669|2486728|2486331|2486605|N|N|N|N|N| +2486698|AAAAAAAAKKBPFCAA|2096-03-30|2354|10240|786|2096|5|3|30|1|2096|786|10240|Friday|2096Q1|N|Y|N|2486669|2486728|2486332|2486606|N|N|N|N|N| +2486699|AAAAAAAALKBPFCAA|2096-03-31|2354|10240|786|2096|6|3|31|1|2096|786|10240|Saturday|2096Q1|N|Y|N|2486669|2486728|2486333|2486607|N|N|N|N|N| +2486700|AAAAAAAAMKBPFCAA|2096-04-01|2355|10240|786|2096|0|4|1|2|2096|786|10240|Sunday|2096Q2|N|N|N|2486700|2486790|2486334|2486609|N|N|N|N|N| +2486701|AAAAAAAANKBPFCAA|2096-04-02|2355|10240|786|2096|1|4|2|2|2096|786|10240|Monday|2096Q2|N|N|N|2486700|2486790|2486335|2486610|N|N|N|N|N| +2486702|AAAAAAAAOKBPFCAA|2096-04-03|2355|10241|786|2096|2|4|3|2|2096|786|10241|Tuesday|2096Q2|N|N|N|2486700|2486790|2486336|2486611|N|N|N|N|N| +2486703|AAAAAAAAPKBPFCAA|2096-04-04|2355|10241|786|2096|3|4|4|2|2096|786|10241|Wednesday|2096Q2|N|N|N|2486700|2486790|2486337|2486612|N|N|N|N|N| +2486704|AAAAAAAAALBPFCAA|2096-04-05|2355|10241|786|2096|4|4|5|2|2096|786|10241|Thursday|2096Q2|N|N|N|2486700|2486790|2486338|2486613|N|N|N|N|N| +2486705|AAAAAAAABLBPFCAA|2096-04-06|2355|10241|786|2096|5|4|6|2|2096|786|10241|Friday|2096Q2|N|Y|N|2486700|2486790|2486339|2486614|N|N|N|N|N| +2486706|AAAAAAAACLBPFCAA|2096-04-07|2355|10241|786|2096|6|4|7|2|2096|786|10241|Saturday|2096Q2|N|Y|N|2486700|2486790|2486340|2486615|N|N|N|N|N| +2486707|AAAAAAAADLBPFCAA|2096-04-08|2355|10241|786|2096|0|4|8|2|2096|786|10241|Sunday|2096Q2|N|N|N|2486700|2486790|2486341|2486616|N|N|N|N|N| +2486708|AAAAAAAAELBPFCAA|2096-04-09|2355|10241|786|2096|1|4|9|2|2096|786|10241|Monday|2096Q2|N|N|N|2486700|2486790|2486342|2486617|N|N|N|N|N| +2486709|AAAAAAAAFLBPFCAA|2096-04-10|2355|10242|786|2096|2|4|10|2|2096|786|10242|Tuesday|2096Q2|N|N|N|2486700|2486790|2486343|2486618|N|N|N|N|N| +2486710|AAAAAAAAGLBPFCAA|2096-04-11|2355|10242|786|2096|3|4|11|2|2096|786|10242|Wednesday|2096Q2|N|N|N|2486700|2486790|2486344|2486619|N|N|N|N|N| +2486711|AAAAAAAAHLBPFCAA|2096-04-12|2355|10242|786|2096|4|4|12|2|2096|786|10242|Thursday|2096Q2|N|N|N|2486700|2486790|2486345|2486620|N|N|N|N|N| +2486712|AAAAAAAAILBPFCAA|2096-04-13|2355|10242|786|2096|5|4|13|2|2096|786|10242|Friday|2096Q2|N|Y|N|2486700|2486790|2486346|2486621|N|N|N|N|N| +2486713|AAAAAAAAJLBPFCAA|2096-04-14|2355|10242|786|2096|6|4|14|2|2096|786|10242|Saturday|2096Q2|N|Y|N|2486700|2486790|2486347|2486622|N|N|N|N|N| +2486714|AAAAAAAAKLBPFCAA|2096-04-15|2355|10242|786|2096|0|4|15|2|2096|786|10242|Sunday|2096Q2|N|N|N|2486700|2486790|2486348|2486623|N|N|N|N|N| +2486715|AAAAAAAALLBPFCAA|2096-04-16|2355|10242|786|2096|1|4|16|2|2096|786|10242|Monday|2096Q2|N|N|N|2486700|2486790|2486349|2486624|N|N|N|N|N| +2486716|AAAAAAAAMLBPFCAA|2096-04-17|2355|10243|786|2096|2|4|17|2|2096|786|10243|Tuesday|2096Q2|N|N|N|2486700|2486790|2486350|2486625|N|N|N|N|N| +2486717|AAAAAAAANLBPFCAA|2096-04-18|2355|10243|786|2096|3|4|18|2|2096|786|10243|Wednesday|2096Q2|N|N|N|2486700|2486790|2486351|2486626|N|N|N|N|N| +2486718|AAAAAAAAOLBPFCAA|2096-04-19|2355|10243|786|2096|4|4|19|2|2096|786|10243|Thursday|2096Q2|N|N|N|2486700|2486790|2486352|2486627|N|N|N|N|N| +2486719|AAAAAAAAPLBPFCAA|2096-04-20|2355|10243|786|2096|5|4|20|2|2096|786|10243|Friday|2096Q2|N|Y|N|2486700|2486790|2486353|2486628|N|N|N|N|N| +2486720|AAAAAAAAAMBPFCAA|2096-04-21|2355|10243|786|2096|6|4|21|2|2096|786|10243|Saturday|2096Q2|N|Y|N|2486700|2486790|2486354|2486629|N|N|N|N|N| +2486721|AAAAAAAABMBPFCAA|2096-04-22|2355|10243|786|2096|0|4|22|2|2096|786|10243|Sunday|2096Q2|N|N|N|2486700|2486790|2486355|2486630|N|N|N|N|N| +2486722|AAAAAAAACMBPFCAA|2096-04-23|2355|10243|786|2096|1|4|23|2|2096|786|10243|Monday|2096Q2|N|N|N|2486700|2486790|2486356|2486631|N|N|N|N|N| +2486723|AAAAAAAADMBPFCAA|2096-04-24|2355|10244|786|2096|2|4|24|2|2096|786|10244|Tuesday|2096Q2|N|N|N|2486700|2486790|2486357|2486632|N|N|N|N|N| +2486724|AAAAAAAAEMBPFCAA|2096-04-25|2355|10244|786|2096|3|4|25|2|2096|786|10244|Wednesday|2096Q2|N|N|N|2486700|2486790|2486358|2486633|N|N|N|N|N| +2486725|AAAAAAAAFMBPFCAA|2096-04-26|2355|10244|786|2096|4|4|26|2|2096|786|10244|Thursday|2096Q2|N|N|N|2486700|2486790|2486359|2486634|N|N|N|N|N| +2486726|AAAAAAAAGMBPFCAA|2096-04-27|2355|10244|786|2096|5|4|27|2|2096|786|10244|Friday|2096Q2|N|Y|N|2486700|2486790|2486360|2486635|N|N|N|N|N| +2486727|AAAAAAAAHMBPFCAA|2096-04-28|2355|10244|786|2096|6|4|28|2|2096|786|10244|Saturday|2096Q2|N|Y|N|2486700|2486790|2486361|2486636|N|N|N|N|N| +2486728|AAAAAAAAIMBPFCAA|2096-04-29|2355|10244|786|2096|0|4|29|2|2096|786|10244|Sunday|2096Q2|N|N|N|2486700|2486790|2486362|2486637|N|N|N|N|N| +2486729|AAAAAAAAJMBPFCAA|2096-04-30|2355|10244|786|2096|1|4|30|2|2096|786|10244|Monday|2096Q2|N|N|N|2486700|2486790|2486363|2486638|N|N|N|N|N| +2486730|AAAAAAAAKMBPFCAA|2096-05-01|2356|10245|786|2096|2|5|1|2|2096|786|10245|Tuesday|2096Q2|N|N|N|2486730|2486850|2486364|2486639|N|N|N|N|N| +2486731|AAAAAAAALMBPFCAA|2096-05-02|2356|10245|786|2096|3|5|2|2|2096|786|10245|Wednesday|2096Q2|N|N|N|2486730|2486850|2486365|2486640|N|N|N|N|N| +2486732|AAAAAAAAMMBPFCAA|2096-05-03|2356|10245|786|2096|4|5|3|2|2096|786|10245|Thursday|2096Q2|N|N|N|2486730|2486850|2486366|2486641|N|N|N|N|N| +2486733|AAAAAAAANMBPFCAA|2096-05-04|2356|10245|786|2096|5|5|4|2|2096|786|10245|Friday|2096Q2|N|Y|N|2486730|2486850|2486367|2486642|N|N|N|N|N| +2486734|AAAAAAAAOMBPFCAA|2096-05-05|2356|10245|786|2096|6|5|5|2|2096|786|10245|Saturday|2096Q2|N|Y|N|2486730|2486850|2486368|2486643|N|N|N|N|N| +2486735|AAAAAAAAPMBPFCAA|2096-05-06|2356|10245|786|2096|0|5|6|2|2096|786|10245|Sunday|2096Q2|N|N|N|2486730|2486850|2486369|2486644|N|N|N|N|N| +2486736|AAAAAAAAANBPFCAA|2096-05-07|2356|10245|786|2096|1|5|7|2|2096|786|10245|Monday|2096Q2|N|N|N|2486730|2486850|2486370|2486645|N|N|N|N|N| +2486737|AAAAAAAABNBPFCAA|2096-05-08|2356|10246|786|2096|2|5|8|2|2096|786|10246|Tuesday|2096Q2|N|N|N|2486730|2486850|2486371|2486646|N|N|N|N|N| +2486738|AAAAAAAACNBPFCAA|2096-05-09|2356|10246|786|2096|3|5|9|2|2096|786|10246|Wednesday|2096Q2|N|N|N|2486730|2486850|2486372|2486647|N|N|N|N|N| +2486739|AAAAAAAADNBPFCAA|2096-05-10|2356|10246|786|2096|4|5|10|2|2096|786|10246|Thursday|2096Q2|N|N|N|2486730|2486850|2486373|2486648|N|N|N|N|N| +2486740|AAAAAAAAENBPFCAA|2096-05-11|2356|10246|786|2096|5|5|11|2|2096|786|10246|Friday|2096Q2|N|Y|N|2486730|2486850|2486374|2486649|N|N|N|N|N| +2486741|AAAAAAAAFNBPFCAA|2096-05-12|2356|10246|786|2096|6|5|12|2|2096|786|10246|Saturday|2096Q2|N|Y|N|2486730|2486850|2486375|2486650|N|N|N|N|N| +2486742|AAAAAAAAGNBPFCAA|2096-05-13|2356|10246|786|2096|0|5|13|2|2096|786|10246|Sunday|2096Q2|N|N|N|2486730|2486850|2486376|2486651|N|N|N|N|N| +2486743|AAAAAAAAHNBPFCAA|2096-05-14|2356|10246|786|2096|1|5|14|2|2096|786|10246|Monday|2096Q2|N|N|N|2486730|2486850|2486377|2486652|N|N|N|N|N| +2486744|AAAAAAAAINBPFCAA|2096-05-15|2356|10247|786|2096|2|5|15|2|2096|786|10247|Tuesday|2096Q2|N|N|N|2486730|2486850|2486378|2486653|N|N|N|N|N| +2486745|AAAAAAAAJNBPFCAA|2096-05-16|2356|10247|786|2096|3|5|16|2|2096|786|10247|Wednesday|2096Q2|N|N|N|2486730|2486850|2486379|2486654|N|N|N|N|N| +2486746|AAAAAAAAKNBPFCAA|2096-05-17|2356|10247|786|2096|4|5|17|2|2096|786|10247|Thursday|2096Q2|N|N|N|2486730|2486850|2486380|2486655|N|N|N|N|N| +2486747|AAAAAAAALNBPFCAA|2096-05-18|2356|10247|786|2096|5|5|18|2|2096|786|10247|Friday|2096Q2|N|Y|N|2486730|2486850|2486381|2486656|N|N|N|N|N| +2486748|AAAAAAAAMNBPFCAA|2096-05-19|2356|10247|786|2096|6|5|19|2|2096|786|10247|Saturday|2096Q2|N|Y|N|2486730|2486850|2486382|2486657|N|N|N|N|N| +2486749|AAAAAAAANNBPFCAA|2096-05-20|2356|10247|786|2096|0|5|20|2|2096|786|10247|Sunday|2096Q2|N|N|N|2486730|2486850|2486383|2486658|N|N|N|N|N| +2486750|AAAAAAAAONBPFCAA|2096-05-21|2356|10247|786|2096|1|5|21|2|2096|786|10247|Monday|2096Q2|N|N|N|2486730|2486850|2486384|2486659|N|N|N|N|N| +2486751|AAAAAAAAPNBPFCAA|2096-05-22|2356|10248|786|2096|2|5|22|2|2096|786|10248|Tuesday|2096Q2|N|N|N|2486730|2486850|2486385|2486660|N|N|N|N|N| +2486752|AAAAAAAAAOBPFCAA|2096-05-23|2356|10248|786|2096|3|5|23|2|2096|786|10248|Wednesday|2096Q2|N|N|N|2486730|2486850|2486386|2486661|N|N|N|N|N| +2486753|AAAAAAAABOBPFCAA|2096-05-24|2356|10248|786|2096|4|5|24|2|2096|786|10248|Thursday|2096Q2|N|N|N|2486730|2486850|2486387|2486662|N|N|N|N|N| +2486754|AAAAAAAACOBPFCAA|2096-05-25|2356|10248|786|2096|5|5|25|2|2096|786|10248|Friday|2096Q2|N|Y|N|2486730|2486850|2486388|2486663|N|N|N|N|N| +2486755|AAAAAAAADOBPFCAA|2096-05-26|2356|10248|786|2096|6|5|26|2|2096|786|10248|Saturday|2096Q2|N|Y|N|2486730|2486850|2486389|2486664|N|N|N|N|N| +2486756|AAAAAAAAEOBPFCAA|2096-05-27|2356|10248|786|2096|0|5|27|2|2096|786|10248|Sunday|2096Q2|N|N|N|2486730|2486850|2486390|2486665|N|N|N|N|N| +2486757|AAAAAAAAFOBPFCAA|2096-05-28|2356|10248|786|2096|1|5|28|2|2096|786|10248|Monday|2096Q2|N|N|N|2486730|2486850|2486391|2486666|N|N|N|N|N| +2486758|AAAAAAAAGOBPFCAA|2096-05-29|2356|10249|786|2096|2|5|29|2|2096|786|10249|Tuesday|2096Q2|N|N|N|2486730|2486850|2486392|2486667|N|N|N|N|N| +2486759|AAAAAAAAHOBPFCAA|2096-05-30|2356|10249|786|2096|3|5|30|2|2096|786|10249|Wednesday|2096Q2|N|N|N|2486730|2486850|2486393|2486668|N|N|N|N|N| +2486760|AAAAAAAAIOBPFCAA|2096-05-31|2356|10249|786|2096|4|5|31|2|2096|786|10249|Thursday|2096Q2|N|N|N|2486730|2486850|2486394|2486669|N|N|N|N|N| +2486761|AAAAAAAAJOBPFCAA|2096-06-01|2357|10249|787|2096|5|6|1|2|2096|787|10249|Friday|2096Q2|N|Y|N|2486761|2486912|2486395|2486670|N|N|N|N|N| +2486762|AAAAAAAAKOBPFCAA|2096-06-02|2357|10249|787|2096|6|6|2|2|2096|787|10249|Saturday|2096Q2|N|Y|N|2486761|2486912|2486396|2486671|N|N|N|N|N| +2486763|AAAAAAAALOBPFCAA|2096-06-03|2357|10249|787|2096|0|6|3|2|2096|787|10249|Sunday|2096Q2|N|N|N|2486761|2486912|2486397|2486672|N|N|N|N|N| +2486764|AAAAAAAAMOBPFCAA|2096-06-04|2357|10249|787|2096|1|6|4|2|2096|787|10249|Monday|2096Q2|N|N|N|2486761|2486912|2486398|2486673|N|N|N|N|N| +2486765|AAAAAAAANOBPFCAA|2096-06-05|2357|10250|787|2096|2|6|5|2|2096|787|10250|Tuesday|2096Q2|N|N|N|2486761|2486912|2486399|2486674|N|N|N|N|N| +2486766|AAAAAAAAOOBPFCAA|2096-06-06|2357|10250|787|2096|3|6|6|2|2096|787|10250|Wednesday|2096Q2|N|N|N|2486761|2486912|2486400|2486675|N|N|N|N|N| +2486767|AAAAAAAAPOBPFCAA|2096-06-07|2357|10250|787|2096|4|6|7|2|2096|787|10250|Thursday|2096Q2|N|N|N|2486761|2486912|2486401|2486676|N|N|N|N|N| +2486768|AAAAAAAAAPBPFCAA|2096-06-08|2357|10250|787|2096|5|6|8|2|2096|787|10250|Friday|2096Q2|N|Y|N|2486761|2486912|2486402|2486677|N|N|N|N|N| +2486769|AAAAAAAABPBPFCAA|2096-06-09|2357|10250|787|2096|6|6|9|2|2096|787|10250|Saturday|2096Q2|N|Y|N|2486761|2486912|2486403|2486678|N|N|N|N|N| +2486770|AAAAAAAACPBPFCAA|2096-06-10|2357|10250|787|2096|0|6|10|2|2096|787|10250|Sunday|2096Q2|N|N|N|2486761|2486912|2486404|2486679|N|N|N|N|N| +2486771|AAAAAAAADPBPFCAA|2096-06-11|2357|10250|787|2096|1|6|11|2|2096|787|10250|Monday|2096Q2|N|N|N|2486761|2486912|2486405|2486680|N|N|N|N|N| +2486772|AAAAAAAAEPBPFCAA|2096-06-12|2357|10251|787|2096|2|6|12|2|2096|787|10251|Tuesday|2096Q2|N|N|N|2486761|2486912|2486406|2486681|N|N|N|N|N| +2486773|AAAAAAAAFPBPFCAA|2096-06-13|2357|10251|787|2096|3|6|13|2|2096|787|10251|Wednesday|2096Q2|N|N|N|2486761|2486912|2486407|2486682|N|N|N|N|N| +2486774|AAAAAAAAGPBPFCAA|2096-06-14|2357|10251|787|2096|4|6|14|2|2096|787|10251|Thursday|2096Q2|N|N|N|2486761|2486912|2486408|2486683|N|N|N|N|N| +2486775|AAAAAAAAHPBPFCAA|2096-06-15|2357|10251|787|2096|5|6|15|2|2096|787|10251|Friday|2096Q2|N|Y|N|2486761|2486912|2486409|2486684|N|N|N|N|N| +2486776|AAAAAAAAIPBPFCAA|2096-06-16|2357|10251|787|2096|6|6|16|2|2096|787|10251|Saturday|2096Q2|N|Y|N|2486761|2486912|2486410|2486685|N|N|N|N|N| +2486777|AAAAAAAAJPBPFCAA|2096-06-17|2357|10251|787|2096|0|6|17|2|2096|787|10251|Sunday|2096Q2|N|N|N|2486761|2486912|2486411|2486686|N|N|N|N|N| +2486778|AAAAAAAAKPBPFCAA|2096-06-18|2357|10251|787|2096|1|6|18|2|2096|787|10251|Monday|2096Q2|N|N|N|2486761|2486912|2486412|2486687|N|N|N|N|N| +2486779|AAAAAAAALPBPFCAA|2096-06-19|2357|10252|787|2096|2|6|19|2|2096|787|10252|Tuesday|2096Q2|N|N|N|2486761|2486912|2486413|2486688|N|N|N|N|N| +2486780|AAAAAAAAMPBPFCAA|2096-06-20|2357|10252|787|2096|3|6|20|2|2096|787|10252|Wednesday|2096Q2|N|N|N|2486761|2486912|2486414|2486689|N|N|N|N|N| +2486781|AAAAAAAANPBPFCAA|2096-06-21|2357|10252|787|2096|4|6|21|2|2096|787|10252|Thursday|2096Q2|N|N|N|2486761|2486912|2486415|2486690|N|N|N|N|N| +2486782|AAAAAAAAOPBPFCAA|2096-06-22|2357|10252|787|2096|5|6|22|2|2096|787|10252|Friday|2096Q2|N|Y|N|2486761|2486912|2486416|2486691|N|N|N|N|N| +2486783|AAAAAAAAPPBPFCAA|2096-06-23|2357|10252|787|2096|6|6|23|2|2096|787|10252|Saturday|2096Q2|N|Y|N|2486761|2486912|2486417|2486692|N|N|N|N|N| +2486784|AAAAAAAAAACPFCAA|2096-06-24|2357|10252|787|2096|0|6|24|2|2096|787|10252|Sunday|2096Q2|N|N|N|2486761|2486912|2486418|2486693|N|N|N|N|N| +2486785|AAAAAAAABACPFCAA|2096-06-25|2357|10252|787|2096|1|6|25|2|2096|787|10252|Monday|2096Q2|N|N|N|2486761|2486912|2486419|2486694|N|N|N|N|N| +2486786|AAAAAAAACACPFCAA|2096-06-26|2357|10253|787|2096|2|6|26|2|2096|787|10253|Tuesday|2096Q2|N|N|N|2486761|2486912|2486420|2486695|N|N|N|N|N| +2486787|AAAAAAAADACPFCAA|2096-06-27|2357|10253|787|2096|3|6|27|2|2096|787|10253|Wednesday|2096Q2|N|N|N|2486761|2486912|2486421|2486696|N|N|N|N|N| +2486788|AAAAAAAAEACPFCAA|2096-06-28|2357|10253|787|2096|4|6|28|2|2096|787|10253|Thursday|2096Q2|N|N|N|2486761|2486912|2486422|2486697|N|N|N|N|N| +2486789|AAAAAAAAFACPFCAA|2096-06-29|2357|10253|787|2096|5|6|29|2|2096|787|10253|Friday|2096Q2|N|Y|N|2486761|2486912|2486423|2486698|N|N|N|N|N| +2486790|AAAAAAAAGACPFCAA|2096-06-30|2357|10253|787|2096|6|6|30|2|2096|787|10253|Saturday|2096Q2|N|Y|N|2486761|2486912|2486424|2486699|N|N|N|N|N| +2486791|AAAAAAAAHACPFCAA|2096-07-01|2358|10253|787|2096|0|7|1|3|2096|787|10253|Sunday|2096Q3|N|N|N|2486791|2486972|2486425|2486700|N|N|N|N|N| +2486792|AAAAAAAAIACPFCAA|2096-07-02|2358|10253|787|2096|1|7|2|3|2096|787|10253|Monday|2096Q3|N|N|N|2486791|2486972|2486426|2486701|N|N|N|N|N| +2486793|AAAAAAAAJACPFCAA|2096-07-03|2358|10254|787|2096|2|7|3|3|2096|787|10254|Tuesday|2096Q3|N|N|N|2486791|2486972|2486427|2486702|N|N|N|N|N| +2486794|AAAAAAAAKACPFCAA|2096-07-04|2358|10254|787|2096|3|7|4|3|2096|787|10254|Wednesday|2096Q3|Y|N|N|2486791|2486972|2486428|2486703|N|N|N|N|N| +2486795|AAAAAAAALACPFCAA|2096-07-05|2358|10254|787|2096|4|7|5|3|2096|787|10254|Thursday|2096Q3|N|N|Y|2486791|2486972|2486429|2486704|N|N|N|N|N| +2486796|AAAAAAAAMACPFCAA|2096-07-06|2358|10254|787|2096|5|7|6|3|2096|787|10254|Friday|2096Q3|N|Y|N|2486791|2486972|2486430|2486705|N|N|N|N|N| +2486797|AAAAAAAANACPFCAA|2096-07-07|2358|10254|787|2096|6|7|7|3|2096|787|10254|Saturday|2096Q3|N|Y|N|2486791|2486972|2486431|2486706|N|N|N|N|N| +2486798|AAAAAAAAOACPFCAA|2096-07-08|2358|10254|787|2096|0|7|8|3|2096|787|10254|Sunday|2096Q3|N|N|N|2486791|2486972|2486432|2486707|N|N|N|N|N| +2486799|AAAAAAAAPACPFCAA|2096-07-09|2358|10254|787|2096|1|7|9|3|2096|787|10254|Monday|2096Q3|N|N|N|2486791|2486972|2486433|2486708|N|N|N|N|N| +2486800|AAAAAAAAABCPFCAA|2096-07-10|2358|10255|787|2096|2|7|10|3|2096|787|10255|Tuesday|2096Q3|N|N|N|2486791|2486972|2486434|2486709|N|N|N|N|N| +2486801|AAAAAAAABBCPFCAA|2096-07-11|2358|10255|787|2096|3|7|11|3|2096|787|10255|Wednesday|2096Q3|N|N|N|2486791|2486972|2486435|2486710|N|N|N|N|N| +2486802|AAAAAAAACBCPFCAA|2096-07-12|2358|10255|787|2096|4|7|12|3|2096|787|10255|Thursday|2096Q3|N|N|N|2486791|2486972|2486436|2486711|N|N|N|N|N| +2486803|AAAAAAAADBCPFCAA|2096-07-13|2358|10255|787|2096|5|7|13|3|2096|787|10255|Friday|2096Q3|N|Y|N|2486791|2486972|2486437|2486712|N|N|N|N|N| +2486804|AAAAAAAAEBCPFCAA|2096-07-14|2358|10255|787|2096|6|7|14|3|2096|787|10255|Saturday|2096Q3|N|Y|N|2486791|2486972|2486438|2486713|N|N|N|N|N| +2486805|AAAAAAAAFBCPFCAA|2096-07-15|2358|10255|787|2096|0|7|15|3|2096|787|10255|Sunday|2096Q3|N|N|N|2486791|2486972|2486439|2486714|N|N|N|N|N| +2486806|AAAAAAAAGBCPFCAA|2096-07-16|2358|10255|787|2096|1|7|16|3|2096|787|10255|Monday|2096Q3|N|N|N|2486791|2486972|2486440|2486715|N|N|N|N|N| +2486807|AAAAAAAAHBCPFCAA|2096-07-17|2358|10256|787|2096|2|7|17|3|2096|787|10256|Tuesday|2096Q3|N|N|N|2486791|2486972|2486441|2486716|N|N|N|N|N| +2486808|AAAAAAAAIBCPFCAA|2096-07-18|2358|10256|787|2096|3|7|18|3|2096|787|10256|Wednesday|2096Q3|N|N|N|2486791|2486972|2486442|2486717|N|N|N|N|N| +2486809|AAAAAAAAJBCPFCAA|2096-07-19|2358|10256|787|2096|4|7|19|3|2096|787|10256|Thursday|2096Q3|N|N|N|2486791|2486972|2486443|2486718|N|N|N|N|N| +2486810|AAAAAAAAKBCPFCAA|2096-07-20|2358|10256|787|2096|5|7|20|3|2096|787|10256|Friday|2096Q3|N|Y|N|2486791|2486972|2486444|2486719|N|N|N|N|N| +2486811|AAAAAAAALBCPFCAA|2096-07-21|2358|10256|787|2096|6|7|21|3|2096|787|10256|Saturday|2096Q3|N|Y|N|2486791|2486972|2486445|2486720|N|N|N|N|N| +2486812|AAAAAAAAMBCPFCAA|2096-07-22|2358|10256|787|2096|0|7|22|3|2096|787|10256|Sunday|2096Q3|N|N|N|2486791|2486972|2486446|2486721|N|N|N|N|N| +2486813|AAAAAAAANBCPFCAA|2096-07-23|2358|10256|787|2096|1|7|23|3|2096|787|10256|Monday|2096Q3|N|N|N|2486791|2486972|2486447|2486722|N|N|N|N|N| +2486814|AAAAAAAAOBCPFCAA|2096-07-24|2358|10257|787|2096|2|7|24|3|2096|787|10257|Tuesday|2096Q3|N|N|N|2486791|2486972|2486448|2486723|N|N|N|N|N| +2486815|AAAAAAAAPBCPFCAA|2096-07-25|2358|10257|787|2096|3|7|25|3|2096|787|10257|Wednesday|2096Q3|N|N|N|2486791|2486972|2486449|2486724|N|N|N|N|N| +2486816|AAAAAAAAACCPFCAA|2096-07-26|2358|10257|787|2096|4|7|26|3|2096|787|10257|Thursday|2096Q3|N|N|N|2486791|2486972|2486450|2486725|N|N|N|N|N| +2486817|AAAAAAAABCCPFCAA|2096-07-27|2358|10257|787|2096|5|7|27|3|2096|787|10257|Friday|2096Q3|N|Y|N|2486791|2486972|2486451|2486726|N|N|N|N|N| +2486818|AAAAAAAACCCPFCAA|2096-07-28|2358|10257|787|2096|6|7|28|3|2096|787|10257|Saturday|2096Q3|N|Y|N|2486791|2486972|2486452|2486727|N|N|N|N|N| +2486819|AAAAAAAADCCPFCAA|2096-07-29|2358|10257|787|2096|0|7|29|3|2096|787|10257|Sunday|2096Q3|N|N|N|2486791|2486972|2486453|2486728|N|N|N|N|N| +2486820|AAAAAAAAECCPFCAA|2096-07-30|2358|10257|787|2096|1|7|30|3|2096|787|10257|Monday|2096Q3|N|N|N|2486791|2486972|2486454|2486729|N|N|N|N|N| +2486821|AAAAAAAAFCCPFCAA|2096-07-31|2358|10258|787|2096|2|7|31|3|2096|787|10258|Tuesday|2096Q3|N|N|N|2486791|2486972|2486455|2486730|N|N|N|N|N| +2486822|AAAAAAAAGCCPFCAA|2096-08-01|2359|10258|787|2096|3|8|1|3|2096|787|10258|Wednesday|2096Q3|N|N|N|2486822|2487034|2486456|2486731|N|N|N|N|N| +2486823|AAAAAAAAHCCPFCAA|2096-08-02|2359|10258|787|2096|4|8|2|3|2096|787|10258|Thursday|2096Q3|N|N|N|2486822|2487034|2486457|2486732|N|N|N|N|N| +2486824|AAAAAAAAICCPFCAA|2096-08-03|2359|10258|787|2096|5|8|3|3|2096|787|10258|Friday|2096Q3|N|Y|N|2486822|2487034|2486458|2486733|N|N|N|N|N| +2486825|AAAAAAAAJCCPFCAA|2096-08-04|2359|10258|787|2096|6|8|4|3|2096|787|10258|Saturday|2096Q3|N|Y|N|2486822|2487034|2486459|2486734|N|N|N|N|N| +2486826|AAAAAAAAKCCPFCAA|2096-08-05|2359|10258|787|2096|0|8|5|3|2096|787|10258|Sunday|2096Q3|N|N|N|2486822|2487034|2486460|2486735|N|N|N|N|N| +2486827|AAAAAAAALCCPFCAA|2096-08-06|2359|10258|787|2096|1|8|6|3|2096|787|10258|Monday|2096Q3|N|N|N|2486822|2487034|2486461|2486736|N|N|N|N|N| +2486828|AAAAAAAAMCCPFCAA|2096-08-07|2359|10259|787|2096|2|8|7|3|2096|787|10259|Tuesday|2096Q3|N|N|N|2486822|2487034|2486462|2486737|N|N|N|N|N| +2486829|AAAAAAAANCCPFCAA|2096-08-08|2359|10259|787|2096|3|8|8|3|2096|787|10259|Wednesday|2096Q3|N|N|N|2486822|2487034|2486463|2486738|N|N|N|N|N| +2486830|AAAAAAAAOCCPFCAA|2096-08-09|2359|10259|787|2096|4|8|9|3|2096|787|10259|Thursday|2096Q3|N|N|N|2486822|2487034|2486464|2486739|N|N|N|N|N| +2486831|AAAAAAAAPCCPFCAA|2096-08-10|2359|10259|787|2096|5|8|10|3|2096|787|10259|Friday|2096Q3|N|Y|N|2486822|2487034|2486465|2486740|N|N|N|N|N| +2486832|AAAAAAAAADCPFCAA|2096-08-11|2359|10259|787|2096|6|8|11|3|2096|787|10259|Saturday|2096Q3|N|Y|N|2486822|2487034|2486466|2486741|N|N|N|N|N| +2486833|AAAAAAAABDCPFCAA|2096-08-12|2359|10259|787|2096|0|8|12|3|2096|787|10259|Sunday|2096Q3|N|N|N|2486822|2487034|2486467|2486742|N|N|N|N|N| +2486834|AAAAAAAACDCPFCAA|2096-08-13|2359|10259|787|2096|1|8|13|3|2096|787|10259|Monday|2096Q3|N|N|N|2486822|2487034|2486468|2486743|N|N|N|N|N| +2486835|AAAAAAAADDCPFCAA|2096-08-14|2359|10260|787|2096|2|8|14|3|2096|787|10260|Tuesday|2096Q3|N|N|N|2486822|2487034|2486469|2486744|N|N|N|N|N| +2486836|AAAAAAAAEDCPFCAA|2096-08-15|2359|10260|787|2096|3|8|15|3|2096|787|10260|Wednesday|2096Q3|N|N|N|2486822|2487034|2486470|2486745|N|N|N|N|N| +2486837|AAAAAAAAFDCPFCAA|2096-08-16|2359|10260|787|2096|4|8|16|3|2096|787|10260|Thursday|2096Q3|N|N|N|2486822|2487034|2486471|2486746|N|N|N|N|N| +2486838|AAAAAAAAGDCPFCAA|2096-08-17|2359|10260|787|2096|5|8|17|3|2096|787|10260|Friday|2096Q3|N|Y|N|2486822|2487034|2486472|2486747|N|N|N|N|N| +2486839|AAAAAAAAHDCPFCAA|2096-08-18|2359|10260|787|2096|6|8|18|3|2096|787|10260|Saturday|2096Q3|N|Y|N|2486822|2487034|2486473|2486748|N|N|N|N|N| +2486840|AAAAAAAAIDCPFCAA|2096-08-19|2359|10260|787|2096|0|8|19|3|2096|787|10260|Sunday|2096Q3|N|N|N|2486822|2487034|2486474|2486749|N|N|N|N|N| +2486841|AAAAAAAAJDCPFCAA|2096-08-20|2359|10260|787|2096|1|8|20|3|2096|787|10260|Monday|2096Q3|N|N|N|2486822|2487034|2486475|2486750|N|N|N|N|N| +2486842|AAAAAAAAKDCPFCAA|2096-08-21|2359|10261|787|2096|2|8|21|3|2096|787|10261|Tuesday|2096Q3|N|N|N|2486822|2487034|2486476|2486751|N|N|N|N|N| +2486843|AAAAAAAALDCPFCAA|2096-08-22|2359|10261|787|2096|3|8|22|3|2096|787|10261|Wednesday|2096Q3|N|N|N|2486822|2487034|2486477|2486752|N|N|N|N|N| +2486844|AAAAAAAAMDCPFCAA|2096-08-23|2359|10261|787|2096|4|8|23|3|2096|787|10261|Thursday|2096Q3|N|N|N|2486822|2487034|2486478|2486753|N|N|N|N|N| +2486845|AAAAAAAANDCPFCAA|2096-08-24|2359|10261|787|2096|5|8|24|3|2096|787|10261|Friday|2096Q3|N|Y|N|2486822|2487034|2486479|2486754|N|N|N|N|N| +2486846|AAAAAAAAODCPFCAA|2096-08-25|2359|10261|787|2096|6|8|25|3|2096|787|10261|Saturday|2096Q3|N|Y|N|2486822|2487034|2486480|2486755|N|N|N|N|N| +2486847|AAAAAAAAPDCPFCAA|2096-08-26|2359|10261|787|2096|0|8|26|3|2096|787|10261|Sunday|2096Q3|N|N|N|2486822|2487034|2486481|2486756|N|N|N|N|N| +2486848|AAAAAAAAAECPFCAA|2096-08-27|2359|10261|787|2096|1|8|27|3|2096|787|10261|Monday|2096Q3|N|N|N|2486822|2487034|2486482|2486757|N|N|N|N|N| +2486849|AAAAAAAABECPFCAA|2096-08-28|2359|10262|787|2096|2|8|28|3|2096|787|10262|Tuesday|2096Q3|N|N|N|2486822|2487034|2486483|2486758|N|N|N|N|N| +2486850|AAAAAAAACECPFCAA|2096-08-29|2359|10262|787|2096|3|8|29|3|2096|787|10262|Wednesday|2096Q3|N|N|N|2486822|2487034|2486484|2486759|N|N|N|N|N| +2486851|AAAAAAAADECPFCAA|2096-08-30|2359|10262|787|2096|4|8|30|3|2096|787|10262|Thursday|2096Q3|N|N|N|2486822|2487034|2486485|2486760|N|N|N|N|N| +2486852|AAAAAAAAEECPFCAA|2096-08-31|2359|10262|787|2096|5|8|31|3|2096|787|10262|Friday|2096Q3|N|Y|N|2486822|2487034|2486486|2486761|N|N|N|N|N| +2486853|AAAAAAAAFECPFCAA|2096-09-01|2360|10262|788|2096|6|9|1|3|2096|788|10262|Saturday|2096Q3|N|Y|N|2486853|2487096|2486487|2486762|N|N|N|N|N| +2486854|AAAAAAAAGECPFCAA|2096-09-02|2360|10262|788|2096|0|9|2|3|2096|788|10262|Sunday|2096Q3|N|N|N|2486853|2487096|2486488|2486763|N|N|N|N|N| +2486855|AAAAAAAAHECPFCAA|2096-09-03|2360|10262|788|2096|1|9|3|3|2096|788|10262|Monday|2096Q3|N|N|N|2486853|2487096|2486489|2486764|N|N|N|N|N| +2486856|AAAAAAAAIECPFCAA|2096-09-04|2360|10263|788|2096|2|9|4|3|2096|788|10263|Tuesday|2096Q3|N|N|N|2486853|2487096|2486490|2486765|N|N|N|N|N| +2486857|AAAAAAAAJECPFCAA|2096-09-05|2360|10263|788|2096|3|9|5|3|2096|788|10263|Wednesday|2096Q3|N|N|N|2486853|2487096|2486491|2486766|N|N|N|N|N| +2486858|AAAAAAAAKECPFCAA|2096-09-06|2360|10263|788|2096|4|9|6|3|2096|788|10263|Thursday|2096Q3|N|N|N|2486853|2487096|2486492|2486767|N|N|N|N|N| +2486859|AAAAAAAALECPFCAA|2096-09-07|2360|10263|788|2096|5|9|7|3|2096|788|10263|Friday|2096Q3|N|Y|N|2486853|2487096|2486493|2486768|N|N|N|N|N| +2486860|AAAAAAAAMECPFCAA|2096-09-08|2360|10263|788|2096|6|9|8|3|2096|788|10263|Saturday|2096Q3|N|Y|N|2486853|2487096|2486494|2486769|N|N|N|N|N| +2486861|AAAAAAAANECPFCAA|2096-09-09|2360|10263|788|2096|0|9|9|3|2096|788|10263|Sunday|2096Q3|N|N|N|2486853|2487096|2486495|2486770|N|N|N|N|N| +2486862|AAAAAAAAOECPFCAA|2096-09-10|2360|10263|788|2096|1|9|10|3|2096|788|10263|Monday|2096Q3|N|N|N|2486853|2487096|2486496|2486771|N|N|N|N|N| +2486863|AAAAAAAAPECPFCAA|2096-09-11|2360|10264|788|2096|2|9|11|3|2096|788|10264|Tuesday|2096Q3|N|N|N|2486853|2487096|2486497|2486772|N|N|N|N|N| +2486864|AAAAAAAAAFCPFCAA|2096-09-12|2360|10264|788|2096|3|9|12|3|2096|788|10264|Wednesday|2096Q3|N|N|N|2486853|2487096|2486498|2486773|N|N|N|N|N| +2486865|AAAAAAAABFCPFCAA|2096-09-13|2360|10264|788|2096|4|9|13|3|2096|788|10264|Thursday|2096Q3|N|N|N|2486853|2487096|2486499|2486774|N|N|N|N|N| +2486866|AAAAAAAACFCPFCAA|2096-09-14|2360|10264|788|2096|5|9|14|3|2096|788|10264|Friday|2096Q3|N|Y|N|2486853|2487096|2486500|2486775|N|N|N|N|N| +2486867|AAAAAAAADFCPFCAA|2096-09-15|2360|10264|788|2096|6|9|15|3|2096|788|10264|Saturday|2096Q3|N|Y|N|2486853|2487096|2486501|2486776|N|N|N|N|N| +2486868|AAAAAAAAEFCPFCAA|2096-09-16|2360|10264|788|2096|0|9|16|3|2096|788|10264|Sunday|2096Q3|N|N|N|2486853|2487096|2486502|2486777|N|N|N|N|N| +2486869|AAAAAAAAFFCPFCAA|2096-09-17|2360|10264|788|2096|1|9|17|3|2096|788|10264|Monday|2096Q3|N|N|N|2486853|2487096|2486503|2486778|N|N|N|N|N| +2486870|AAAAAAAAGFCPFCAA|2096-09-18|2360|10265|788|2096|2|9|18|3|2096|788|10265|Tuesday|2096Q3|N|N|N|2486853|2487096|2486504|2486779|N|N|N|N|N| +2486871|AAAAAAAAHFCPFCAA|2096-09-19|2360|10265|788|2096|3|9|19|3|2096|788|10265|Wednesday|2096Q3|N|N|N|2486853|2487096|2486505|2486780|N|N|N|N|N| +2486872|AAAAAAAAIFCPFCAA|2096-09-20|2360|10265|788|2096|4|9|20|3|2096|788|10265|Thursday|2096Q3|N|N|N|2486853|2487096|2486506|2486781|N|N|N|N|N| +2486873|AAAAAAAAJFCPFCAA|2096-09-21|2360|10265|788|2096|5|9|21|3|2096|788|10265|Friday|2096Q3|N|Y|N|2486853|2487096|2486507|2486782|N|N|N|N|N| +2486874|AAAAAAAAKFCPFCAA|2096-09-22|2360|10265|788|2096|6|9|22|3|2096|788|10265|Saturday|2096Q3|N|Y|N|2486853|2487096|2486508|2486783|N|N|N|N|N| +2486875|AAAAAAAALFCPFCAA|2096-09-23|2360|10265|788|2096|0|9|23|3|2096|788|10265|Sunday|2096Q3|N|N|N|2486853|2487096|2486509|2486784|N|N|N|N|N| +2486876|AAAAAAAAMFCPFCAA|2096-09-24|2360|10265|788|2096|1|9|24|3|2096|788|10265|Monday|2096Q3|N|N|N|2486853|2487096|2486510|2486785|N|N|N|N|N| +2486877|AAAAAAAANFCPFCAA|2096-09-25|2360|10266|788|2096|2|9|25|3|2096|788|10266|Tuesday|2096Q3|N|N|N|2486853|2487096|2486511|2486786|N|N|N|N|N| +2486878|AAAAAAAAOFCPFCAA|2096-09-26|2360|10266|788|2096|3|9|26|3|2096|788|10266|Wednesday|2096Q3|N|N|N|2486853|2487096|2486512|2486787|N|N|N|N|N| +2486879|AAAAAAAAPFCPFCAA|2096-09-27|2360|10266|788|2096|4|9|27|3|2096|788|10266|Thursday|2096Q3|N|N|N|2486853|2487096|2486513|2486788|N|N|N|N|N| +2486880|AAAAAAAAAGCPFCAA|2096-09-28|2360|10266|788|2096|5|9|28|3|2096|788|10266|Friday|2096Q3|N|Y|N|2486853|2487096|2486514|2486789|N|N|N|N|N| +2486881|AAAAAAAABGCPFCAA|2096-09-29|2360|10266|788|2096|6|9|29|3|2096|788|10266|Saturday|2096Q3|N|Y|N|2486853|2487096|2486515|2486790|N|N|N|N|N| +2486882|AAAAAAAACGCPFCAA|2096-09-30|2360|10266|788|2096|0|9|30|3|2096|788|10266|Sunday|2096Q3|N|N|N|2486853|2487096|2486516|2486791|N|N|N|N|N| +2486883|AAAAAAAADGCPFCAA|2096-10-01|2361|10266|788|2096|1|10|1|4|2096|788|10266|Monday|2096Q4|N|N|N|2486883|2487156|2486517|2486791|N|N|N|N|N| +2486884|AAAAAAAAEGCPFCAA|2096-10-02|2361|10267|788|2096|2|10|2|4|2096|788|10267|Tuesday|2096Q4|N|N|N|2486883|2487156|2486518|2486792|N|N|N|N|N| +2486885|AAAAAAAAFGCPFCAA|2096-10-03|2361|10267|788|2096|3|10|3|4|2096|788|10267|Wednesday|2096Q4|N|N|N|2486883|2487156|2486519|2486793|N|N|N|N|N| +2486886|AAAAAAAAGGCPFCAA|2096-10-04|2361|10267|788|2096|4|10|4|4|2096|788|10267|Thursday|2096Q4|N|N|N|2486883|2487156|2486520|2486794|N|N|N|N|N| +2486887|AAAAAAAAHGCPFCAA|2096-10-05|2361|10267|788|2096|5|10|5|4|2096|788|10267|Friday|2096Q4|N|Y|N|2486883|2487156|2486521|2486795|N|N|N|N|N| +2486888|AAAAAAAAIGCPFCAA|2096-10-06|2361|10267|788|2096|6|10|6|4|2096|788|10267|Saturday|2096Q4|N|Y|N|2486883|2487156|2486522|2486796|N|N|N|N|N| +2486889|AAAAAAAAJGCPFCAA|2096-10-07|2361|10267|788|2096|0|10|7|4|2096|788|10267|Sunday|2096Q4|N|N|N|2486883|2487156|2486523|2486797|N|N|N|N|N| +2486890|AAAAAAAAKGCPFCAA|2096-10-08|2361|10267|788|2096|1|10|8|4|2096|788|10267|Monday|2096Q4|N|N|N|2486883|2487156|2486524|2486798|N|N|N|N|N| +2486891|AAAAAAAALGCPFCAA|2096-10-09|2361|10268|788|2096|2|10|9|4|2096|788|10268|Tuesday|2096Q4|N|N|N|2486883|2487156|2486525|2486799|N|N|N|N|N| +2486892|AAAAAAAAMGCPFCAA|2096-10-10|2361|10268|788|2096|3|10|10|4|2096|788|10268|Wednesday|2096Q4|N|N|N|2486883|2487156|2486526|2486800|N|N|N|N|N| +2486893|AAAAAAAANGCPFCAA|2096-10-11|2361|10268|788|2096|4|10|11|4|2096|788|10268|Thursday|2096Q4|N|N|N|2486883|2487156|2486527|2486801|N|N|N|N|N| +2486894|AAAAAAAAOGCPFCAA|2096-10-12|2361|10268|788|2096|5|10|12|4|2096|788|10268|Friday|2096Q4|N|Y|N|2486883|2487156|2486528|2486802|N|N|N|N|N| +2486895|AAAAAAAAPGCPFCAA|2096-10-13|2361|10268|788|2096|6|10|13|4|2096|788|10268|Saturday|2096Q4|N|Y|N|2486883|2487156|2486529|2486803|N|N|N|N|N| +2486896|AAAAAAAAAHCPFCAA|2096-10-14|2361|10268|788|2096|0|10|14|4|2096|788|10268|Sunday|2096Q4|N|N|N|2486883|2487156|2486530|2486804|N|N|N|N|N| +2486897|AAAAAAAABHCPFCAA|2096-10-15|2361|10268|788|2096|1|10|15|4|2096|788|10268|Monday|2096Q4|N|N|N|2486883|2487156|2486531|2486805|N|N|N|N|N| +2486898|AAAAAAAACHCPFCAA|2096-10-16|2361|10269|788|2096|2|10|16|4|2096|788|10269|Tuesday|2096Q4|N|N|N|2486883|2487156|2486532|2486806|N|N|N|N|N| +2486899|AAAAAAAADHCPFCAA|2096-10-17|2361|10269|788|2096|3|10|17|4|2096|788|10269|Wednesday|2096Q4|N|N|N|2486883|2487156|2486533|2486807|N|N|N|N|N| +2486900|AAAAAAAAEHCPFCAA|2096-10-18|2361|10269|788|2096|4|10|18|4|2096|788|10269|Thursday|2096Q4|N|N|N|2486883|2487156|2486534|2486808|N|N|N|N|N| +2486901|AAAAAAAAFHCPFCAA|2096-10-19|2361|10269|788|2096|5|10|19|4|2096|788|10269|Friday|2096Q4|N|Y|N|2486883|2487156|2486535|2486809|N|N|N|N|N| +2486902|AAAAAAAAGHCPFCAA|2096-10-20|2361|10269|788|2096|6|10|20|4|2096|788|10269|Saturday|2096Q4|N|Y|N|2486883|2487156|2486536|2486810|N|N|N|N|N| +2486903|AAAAAAAAHHCPFCAA|2096-10-21|2361|10269|788|2096|0|10|21|4|2096|788|10269|Sunday|2096Q4|N|N|N|2486883|2487156|2486537|2486811|N|N|N|N|N| +2486904|AAAAAAAAIHCPFCAA|2096-10-22|2361|10269|788|2096|1|10|22|4|2096|788|10269|Monday|2096Q4|N|N|N|2486883|2487156|2486538|2486812|N|N|N|N|N| +2486905|AAAAAAAAJHCPFCAA|2096-10-23|2361|10270|788|2096|2|10|23|4|2096|788|10270|Tuesday|2096Q4|N|N|N|2486883|2487156|2486539|2486813|N|N|N|N|N| +2486906|AAAAAAAAKHCPFCAA|2096-10-24|2361|10270|788|2096|3|10|24|4|2096|788|10270|Wednesday|2096Q4|N|N|N|2486883|2487156|2486540|2486814|N|N|N|N|N| +2486907|AAAAAAAALHCPFCAA|2096-10-25|2361|10270|788|2096|4|10|25|4|2096|788|10270|Thursday|2096Q4|N|N|N|2486883|2487156|2486541|2486815|N|N|N|N|N| +2486908|AAAAAAAAMHCPFCAA|2096-10-26|2361|10270|788|2096|5|10|26|4|2096|788|10270|Friday|2096Q4|N|Y|N|2486883|2487156|2486542|2486816|N|N|N|N|N| +2486909|AAAAAAAANHCPFCAA|2096-10-27|2361|10270|788|2096|6|10|27|4|2096|788|10270|Saturday|2096Q4|N|Y|N|2486883|2487156|2486543|2486817|N|N|N|N|N| +2486910|AAAAAAAAOHCPFCAA|2096-10-28|2361|10270|788|2096|0|10|28|4|2096|788|10270|Sunday|2096Q4|N|N|N|2486883|2487156|2486544|2486818|N|N|N|N|N| +2486911|AAAAAAAAPHCPFCAA|2096-10-29|2361|10270|788|2096|1|10|29|4|2096|788|10270|Monday|2096Q4|N|N|N|2486883|2487156|2486545|2486819|N|N|N|N|N| +2486912|AAAAAAAAAICPFCAA|2096-10-30|2361|10271|788|2096|2|10|30|4|2096|788|10271|Tuesday|2096Q4|N|N|N|2486883|2487156|2486546|2486820|N|N|N|N|N| +2486913|AAAAAAAABICPFCAA|2096-10-31|2361|10271|788|2096|3|10|31|4|2096|788|10271|Wednesday|2096Q4|N|N|N|2486883|2487156|2486547|2486821|N|N|N|N|N| +2486914|AAAAAAAACICPFCAA|2096-11-01|2362|10271|788|2096|4|11|1|4|2096|788|10271|Thursday|2096Q4|N|N|N|2486914|2487218|2486548|2486822|N|N|N|N|N| +2486915|AAAAAAAADICPFCAA|2096-11-02|2362|10271|788|2096|5|11|2|4|2096|788|10271|Friday|2096Q4|N|Y|N|2486914|2487218|2486549|2486823|N|N|N|N|N| +2486916|AAAAAAAAEICPFCAA|2096-11-03|2362|10271|788|2096|6|11|3|4|2096|788|10271|Saturday|2096Q4|N|Y|N|2486914|2487218|2486550|2486824|N|N|N|N|N| +2486917|AAAAAAAAFICPFCAA|2096-11-04|2362|10271|788|2096|0|11|4|4|2096|788|10271|Sunday|2096Q4|N|N|N|2486914|2487218|2486551|2486825|N|N|N|N|N| +2486918|AAAAAAAAGICPFCAA|2096-11-05|2362|10271|788|2096|1|11|5|4|2096|788|10271|Monday|2096Q4|N|N|N|2486914|2487218|2486552|2486826|N|N|N|N|N| +2486919|AAAAAAAAHICPFCAA|2096-11-06|2362|10272|788|2096|2|11|6|4|2096|788|10272|Tuesday|2096Q4|N|N|N|2486914|2487218|2486553|2486827|N|N|N|N|N| +2486920|AAAAAAAAIICPFCAA|2096-11-07|2362|10272|788|2096|3|11|7|4|2096|788|10272|Wednesday|2096Q4|N|N|N|2486914|2487218|2486554|2486828|N|N|N|N|N| +2486921|AAAAAAAAJICPFCAA|2096-11-08|2362|10272|788|2096|4|11|8|4|2096|788|10272|Thursday|2096Q4|N|N|N|2486914|2487218|2486555|2486829|N|N|N|N|N| +2486922|AAAAAAAAKICPFCAA|2096-11-09|2362|10272|788|2096|5|11|9|4|2096|788|10272|Friday|2096Q4|N|Y|N|2486914|2487218|2486556|2486830|N|N|N|N|N| +2486923|AAAAAAAALICPFCAA|2096-11-10|2362|10272|788|2096|6|11|10|4|2096|788|10272|Saturday|2096Q4|N|Y|N|2486914|2487218|2486557|2486831|N|N|N|N|N| +2486924|AAAAAAAAMICPFCAA|2096-11-11|2362|10272|788|2096|0|11|11|4|2096|788|10272|Sunday|2096Q4|N|N|N|2486914|2487218|2486558|2486832|N|N|N|N|N| +2486925|AAAAAAAANICPFCAA|2096-11-12|2362|10272|788|2096|1|11|12|4|2096|788|10272|Monday|2096Q4|N|N|N|2486914|2487218|2486559|2486833|N|N|N|N|N| +2486926|AAAAAAAAOICPFCAA|2096-11-13|2362|10273|788|2096|2|11|13|4|2096|788|10273|Tuesday|2096Q4|N|N|N|2486914|2487218|2486560|2486834|N|N|N|N|N| +2486927|AAAAAAAAPICPFCAA|2096-11-14|2362|10273|788|2096|3|11|14|4|2096|788|10273|Wednesday|2096Q4|N|N|N|2486914|2487218|2486561|2486835|N|N|N|N|N| +2486928|AAAAAAAAAJCPFCAA|2096-11-15|2362|10273|788|2096|4|11|15|4|2096|788|10273|Thursday|2096Q4|N|N|N|2486914|2487218|2486562|2486836|N|N|N|N|N| +2486929|AAAAAAAABJCPFCAA|2096-11-16|2362|10273|788|2096|5|11|16|4|2096|788|10273|Friday|2096Q4|N|Y|N|2486914|2487218|2486563|2486837|N|N|N|N|N| +2486930|AAAAAAAACJCPFCAA|2096-11-17|2362|10273|788|2096|6|11|17|4|2096|788|10273|Saturday|2096Q4|N|Y|N|2486914|2487218|2486564|2486838|N|N|N|N|N| +2486931|AAAAAAAADJCPFCAA|2096-11-18|2362|10273|788|2096|0|11|18|4|2096|788|10273|Sunday|2096Q4|N|N|N|2486914|2487218|2486565|2486839|N|N|N|N|N| +2486932|AAAAAAAAEJCPFCAA|2096-11-19|2362|10273|788|2096|1|11|19|4|2096|788|10273|Monday|2096Q4|N|N|N|2486914|2487218|2486566|2486840|N|N|N|N|N| +2486933|AAAAAAAAFJCPFCAA|2096-11-20|2362|10274|788|2096|2|11|20|4|2096|788|10274|Tuesday|2096Q4|N|N|N|2486914|2487218|2486567|2486841|N|N|N|N|N| +2486934|AAAAAAAAGJCPFCAA|2096-11-21|2362|10274|788|2096|3|11|21|4|2096|788|10274|Wednesday|2096Q4|N|N|N|2486914|2487218|2486568|2486842|N|N|N|N|N| +2486935|AAAAAAAAHJCPFCAA|2096-11-22|2362|10274|788|2096|4|11|22|4|2096|788|10274|Thursday|2096Q4|N|N|N|2486914|2487218|2486569|2486843|N|N|N|N|N| +2486936|AAAAAAAAIJCPFCAA|2096-11-23|2362|10274|788|2096|5|11|23|4|2096|788|10274|Friday|2096Q4|N|Y|N|2486914|2487218|2486570|2486844|N|N|N|N|N| +2486937|AAAAAAAAJJCPFCAA|2096-11-24|2362|10274|788|2096|6|11|24|4|2096|788|10274|Saturday|2096Q4|N|Y|N|2486914|2487218|2486571|2486845|N|N|N|N|N| +2486938|AAAAAAAAKJCPFCAA|2096-11-25|2362|10274|788|2096|0|11|25|4|2096|788|10274|Sunday|2096Q4|N|N|N|2486914|2487218|2486572|2486846|N|N|N|N|N| +2486939|AAAAAAAALJCPFCAA|2096-11-26|2362|10274|788|2096|1|11|26|4|2096|788|10274|Monday|2096Q4|N|N|N|2486914|2487218|2486573|2486847|N|N|N|N|N| +2486940|AAAAAAAAMJCPFCAA|2096-11-27|2362|10275|788|2096|2|11|27|4|2096|788|10275|Tuesday|2096Q4|N|N|N|2486914|2487218|2486574|2486848|N|N|N|N|N| +2486941|AAAAAAAANJCPFCAA|2096-11-28|2362|10275|788|2096|3|11|28|4|2096|788|10275|Wednesday|2096Q4|N|N|N|2486914|2487218|2486575|2486849|N|N|N|N|N| +2486942|AAAAAAAAOJCPFCAA|2096-11-29|2362|10275|788|2096|4|11|29|4|2096|788|10275|Thursday|2096Q4|N|N|N|2486914|2487218|2486576|2486850|N|N|N|N|N| +2486943|AAAAAAAAPJCPFCAA|2096-11-30|2362|10275|788|2096|5|11|30|4|2096|788|10275|Friday|2096Q4|N|Y|N|2486914|2487218|2486577|2486851|N|N|N|N|N| +2486944|AAAAAAAAAKCPFCAA|2096-12-01|2363|10275|789|2096|6|12|1|4|2096|789|10275|Saturday|2096Q4|N|Y|N|2486944|2487278|2486578|2486852|N|N|N|N|N| +2486945|AAAAAAAABKCPFCAA|2096-12-02|2363|10275|789|2096|0|12|2|4|2096|789|10275|Sunday|2096Q4|N|N|N|2486944|2487278|2486579|2486853|N|N|N|N|N| +2486946|AAAAAAAACKCPFCAA|2096-12-03|2363|10275|789|2096|1|12|3|4|2096|789|10275|Monday|2096Q4|N|N|N|2486944|2487278|2486580|2486854|N|N|N|N|N| +2486947|AAAAAAAADKCPFCAA|2096-12-04|2363|10276|789|2096|2|12|4|4|2096|789|10276|Tuesday|2096Q4|N|N|N|2486944|2487278|2486581|2486855|N|N|N|N|N| +2486948|AAAAAAAAEKCPFCAA|2096-12-05|2363|10276|789|2096|3|12|5|4|2096|789|10276|Wednesday|2096Q4|N|N|N|2486944|2487278|2486582|2486856|N|N|N|N|N| +2486949|AAAAAAAAFKCPFCAA|2096-12-06|2363|10276|789|2096|4|12|6|4|2096|789|10276|Thursday|2096Q4|N|N|N|2486944|2487278|2486583|2486857|N|N|N|N|N| +2486950|AAAAAAAAGKCPFCAA|2096-12-07|2363|10276|789|2096|5|12|7|4|2096|789|10276|Friday|2096Q4|N|Y|N|2486944|2487278|2486584|2486858|N|N|N|N|N| +2486951|AAAAAAAAHKCPFCAA|2096-12-08|2363|10276|789|2096|6|12|8|4|2096|789|10276|Saturday|2096Q4|N|Y|N|2486944|2487278|2486585|2486859|N|N|N|N|N| +2486952|AAAAAAAAIKCPFCAA|2096-12-09|2363|10276|789|2096|0|12|9|4|2096|789|10276|Sunday|2096Q4|N|N|N|2486944|2487278|2486586|2486860|N|N|N|N|N| +2486953|AAAAAAAAJKCPFCAA|2096-12-10|2363|10276|789|2096|1|12|10|4|2096|789|10276|Monday|2096Q4|N|N|N|2486944|2487278|2486587|2486861|N|N|N|N|N| +2486954|AAAAAAAAKKCPFCAA|2096-12-11|2363|10277|789|2096|2|12|11|4|2096|789|10277|Tuesday|2096Q4|N|N|N|2486944|2487278|2486588|2486862|N|N|N|N|N| +2486955|AAAAAAAALKCPFCAA|2096-12-12|2363|10277|789|2096|3|12|12|4|2096|789|10277|Wednesday|2096Q4|N|N|N|2486944|2487278|2486589|2486863|N|N|N|N|N| +2486956|AAAAAAAAMKCPFCAA|2096-12-13|2363|10277|789|2096|4|12|13|4|2096|789|10277|Thursday|2096Q4|N|N|N|2486944|2487278|2486590|2486864|N|N|N|N|N| +2486957|AAAAAAAANKCPFCAA|2096-12-14|2363|10277|789|2096|5|12|14|4|2096|789|10277|Friday|2096Q4|N|Y|N|2486944|2487278|2486591|2486865|N|N|N|N|N| +2486958|AAAAAAAAOKCPFCAA|2096-12-15|2363|10277|789|2096|6|12|15|4|2096|789|10277|Saturday|2096Q4|N|Y|N|2486944|2487278|2486592|2486866|N|N|N|N|N| +2486959|AAAAAAAAPKCPFCAA|2096-12-16|2363|10277|789|2096|0|12|16|4|2096|789|10277|Sunday|2096Q4|N|N|N|2486944|2487278|2486593|2486867|N|N|N|N|N| +2486960|AAAAAAAAALCPFCAA|2096-12-17|2363|10277|789|2096|1|12|17|4|2096|789|10277|Monday|2096Q4|N|N|N|2486944|2487278|2486594|2486868|N|N|N|N|N| +2486961|AAAAAAAABLCPFCAA|2096-12-18|2363|10278|789|2096|2|12|18|4|2096|789|10278|Tuesday|2096Q4|N|N|N|2486944|2487278|2486595|2486869|N|N|N|N|N| +2486962|AAAAAAAACLCPFCAA|2096-12-19|2363|10278|789|2096|3|12|19|4|2096|789|10278|Wednesday|2096Q4|N|N|N|2486944|2487278|2486596|2486870|N|N|N|N|N| +2486963|AAAAAAAADLCPFCAA|2096-12-20|2363|10278|789|2096|4|12|20|4|2096|789|10278|Thursday|2096Q4|N|N|N|2486944|2487278|2486597|2486871|N|N|N|N|N| +2486964|AAAAAAAAELCPFCAA|2096-12-21|2363|10278|789|2096|5|12|21|4|2096|789|10278|Friday|2096Q4|N|Y|N|2486944|2487278|2486598|2486872|N|N|N|N|N| +2486965|AAAAAAAAFLCPFCAA|2096-12-22|2363|10278|789|2096|6|12|22|4|2096|789|10278|Saturday|2096Q4|N|Y|N|2486944|2487278|2486599|2486873|N|N|N|N|N| +2486966|AAAAAAAAGLCPFCAA|2096-12-23|2363|10278|789|2096|0|12|23|4|2096|789|10278|Sunday|2096Q4|N|N|N|2486944|2487278|2486600|2486874|N|N|N|N|N| +2486967|AAAAAAAAHLCPFCAA|2096-12-24|2363|10278|789|2096|1|12|24|4|2096|789|10278|Monday|2096Q4|N|N|N|2486944|2487278|2486601|2486875|N|N|N|N|N| +2486968|AAAAAAAAILCPFCAA|2096-12-25|2363|10279|789|2096|2|12|25|4|2096|789|10279|Tuesday|2096Q4|Y|N|N|2486944|2487278|2486602|2486876|N|N|N|N|N| +2486969|AAAAAAAAJLCPFCAA|2096-12-26|2363|10279|789|2096|3|12|26|4|2096|789|10279|Wednesday|2096Q4|N|N|Y|2486944|2487278|2486603|2486877|N|N|N|N|N| +2486970|AAAAAAAAKLCPFCAA|2096-12-27|2363|10279|789|2096|4|12|27|4|2096|789|10279|Thursday|2096Q4|N|N|N|2486944|2487278|2486604|2486878|N|N|N|N|N| +2486971|AAAAAAAALLCPFCAA|2096-12-28|2363|10279|789|2096|5|12|28|4|2096|789|10279|Friday|2096Q4|N|Y|N|2486944|2487278|2486605|2486879|N|N|N|N|N| +2486972|AAAAAAAAMLCPFCAA|2096-12-29|2363|10279|789|2096|6|12|29|4|2096|789|10279|Saturday|2096Q4|N|Y|N|2486944|2487278|2486606|2486880|N|N|N|N|N| +2486973|AAAAAAAANLCPFCAA|2096-12-30|2363|10279|789|2096|0|12|30|4|2096|789|10279|Sunday|2096Q4|N|N|N|2486944|2487278|2486607|2486881|N|N|N|N|N| +2486974|AAAAAAAAOLCPFCAA|2096-12-31|2363|10279|789|2096|1|12|31|4|2096|789|10279|Monday|2096Q4|Y|N|N|2486944|2487278|2486608|2486882|N|N|N|N|N| +2486975|AAAAAAAAPLCPFCAA|2097-01-01|2364|10280|789|2097|2|1|1|1|2097|789|10280|Tuesday|2097Q1|Y|N|Y|2486975|2486974|2486609|2486883|N|N|N|N|N| +2486976|AAAAAAAAAMCPFCAA|2097-01-02|2364|10280|789|2097|3|1|2|1|2097|789|10280|Wednesday|2097Q1|N|N|Y|2486975|2486974|2486610|2486884|N|N|N|N|N| +2486977|AAAAAAAABMCPFCAA|2097-01-03|2364|10280|789|2097|4|1|3|1|2097|789|10280|Thursday|2097Q1|N|N|N|2486975|2486974|2486611|2486885|N|N|N|N|N| +2486978|AAAAAAAACMCPFCAA|2097-01-04|2364|10280|789|2097|5|1|4|1|2097|789|10280|Friday|2097Q1|N|Y|N|2486975|2486974|2486612|2486886|N|N|N|N|N| +2486979|AAAAAAAADMCPFCAA|2097-01-05|2364|10280|789|2097|6|1|5|1|2097|789|10280|Saturday|2097Q1|N|Y|N|2486975|2486974|2486613|2486887|N|N|N|N|N| +2486980|AAAAAAAAEMCPFCAA|2097-01-06|2364|10280|789|2097|0|1|6|1|2097|789|10280|Sunday|2097Q1|N|N|N|2486975|2486974|2486614|2486888|N|N|N|N|N| +2486981|AAAAAAAAFMCPFCAA|2097-01-07|2364|10280|789|2097|1|1|7|1|2097|789|10280|Monday|2097Q1|N|N|N|2486975|2486974|2486615|2486889|N|N|N|N|N| +2486982|AAAAAAAAGMCPFCAA|2097-01-08|2364|10281|789|2097|2|1|8|1|2097|789|10281|Tuesday|2097Q1|N|N|N|2486975|2486974|2486616|2486890|N|N|N|N|N| +2486983|AAAAAAAAHMCPFCAA|2097-01-09|2364|10281|789|2097|3|1|9|1|2097|789|10281|Wednesday|2097Q1|N|N|N|2486975|2486974|2486617|2486891|N|N|N|N|N| +2486984|AAAAAAAAIMCPFCAA|2097-01-10|2364|10281|789|2097|4|1|10|1|2097|789|10281|Thursday|2097Q1|N|N|N|2486975|2486974|2486618|2486892|N|N|N|N|N| +2486985|AAAAAAAAJMCPFCAA|2097-01-11|2364|10281|789|2097|5|1|11|1|2097|789|10281|Friday|2097Q1|N|Y|N|2486975|2486974|2486619|2486893|N|N|N|N|N| +2486986|AAAAAAAAKMCPFCAA|2097-01-12|2364|10281|789|2097|6|1|12|1|2097|789|10281|Saturday|2097Q1|N|Y|N|2486975|2486974|2486620|2486894|N|N|N|N|N| +2486987|AAAAAAAALMCPFCAA|2097-01-13|2364|10281|789|2097|0|1|13|1|2097|789|10281|Sunday|2097Q1|N|N|N|2486975|2486974|2486621|2486895|N|N|N|N|N| +2486988|AAAAAAAAMMCPFCAA|2097-01-14|2364|10281|789|2097|1|1|14|1|2097|789|10281|Monday|2097Q1|N|N|N|2486975|2486974|2486622|2486896|N|N|N|N|N| +2486989|AAAAAAAANMCPFCAA|2097-01-15|2364|10282|789|2097|2|1|15|1|2097|789|10282|Tuesday|2097Q1|N|N|N|2486975|2486974|2486623|2486897|N|N|N|N|N| +2486990|AAAAAAAAOMCPFCAA|2097-01-16|2364|10282|789|2097|3|1|16|1|2097|789|10282|Wednesday|2097Q1|N|N|N|2486975|2486974|2486624|2486898|N|N|N|N|N| +2486991|AAAAAAAAPMCPFCAA|2097-01-17|2364|10282|789|2097|4|1|17|1|2097|789|10282|Thursday|2097Q1|N|N|N|2486975|2486974|2486625|2486899|N|N|N|N|N| +2486992|AAAAAAAAANCPFCAA|2097-01-18|2364|10282|789|2097|5|1|18|1|2097|789|10282|Friday|2097Q1|N|Y|N|2486975|2486974|2486626|2486900|N|N|N|N|N| +2486993|AAAAAAAABNCPFCAA|2097-01-19|2364|10282|789|2097|6|1|19|1|2097|789|10282|Saturday|2097Q1|N|Y|N|2486975|2486974|2486627|2486901|N|N|N|N|N| +2486994|AAAAAAAACNCPFCAA|2097-01-20|2364|10282|789|2097|0|1|20|1|2097|789|10282|Sunday|2097Q1|N|N|N|2486975|2486974|2486628|2486902|N|N|N|N|N| +2486995|AAAAAAAADNCPFCAA|2097-01-21|2364|10282|789|2097|1|1|21|1|2097|789|10282|Monday|2097Q1|N|N|N|2486975|2486974|2486629|2486903|N|N|N|N|N| +2486996|AAAAAAAAENCPFCAA|2097-01-22|2364|10283|789|2097|2|1|22|1|2097|789|10283|Tuesday|2097Q1|N|N|N|2486975|2486974|2486630|2486904|N|N|N|N|N| +2486997|AAAAAAAAFNCPFCAA|2097-01-23|2364|10283|789|2097|3|1|23|1|2097|789|10283|Wednesday|2097Q1|N|N|N|2486975|2486974|2486631|2486905|N|N|N|N|N| +2486998|AAAAAAAAGNCPFCAA|2097-01-24|2364|10283|789|2097|4|1|24|1|2097|789|10283|Thursday|2097Q1|N|N|N|2486975|2486974|2486632|2486906|N|N|N|N|N| +2486999|AAAAAAAAHNCPFCAA|2097-01-25|2364|10283|789|2097|5|1|25|1|2097|789|10283|Friday|2097Q1|N|Y|N|2486975|2486974|2486633|2486907|N|N|N|N|N| +2487000|AAAAAAAAINCPFCAA|2097-01-26|2364|10283|789|2097|6|1|26|1|2097|789|10283|Saturday|2097Q1|N|Y|N|2486975|2486974|2486634|2486908|N|N|N|N|N| +2487001|AAAAAAAAJNCPFCAA|2097-01-27|2364|10283|789|2097|0|1|27|1|2097|789|10283|Sunday|2097Q1|N|N|N|2486975|2486974|2486635|2486909|N|N|N|N|N| +2487002|AAAAAAAAKNCPFCAA|2097-01-28|2364|10283|789|2097|1|1|28|1|2097|789|10283|Monday|2097Q1|N|N|N|2486975|2486974|2486636|2486910|N|N|N|N|N| +2487003|AAAAAAAALNCPFCAA|2097-01-29|2364|10284|789|2097|2|1|29|1|2097|789|10284|Tuesday|2097Q1|N|N|N|2486975|2486974|2486637|2486911|N|N|N|N|N| +2487004|AAAAAAAAMNCPFCAA|2097-01-30|2364|10284|789|2097|3|1|30|1|2097|789|10284|Wednesday|2097Q1|N|N|N|2486975|2486974|2486638|2486912|N|N|N|N|N| +2487005|AAAAAAAANNCPFCAA|2097-01-31|2364|10284|789|2097|4|1|31|1|2097|789|10284|Thursday|2097Q1|N|N|N|2486975|2486974|2486639|2486913|N|N|N|N|N| +2487006|AAAAAAAAONCPFCAA|2097-02-01|2365|10284|789|2097|5|2|1|1|2097|789|10284|Friday|2097Q1|N|Y|N|2487006|2487036|2486640|2486914|N|N|N|N|N| +2487007|AAAAAAAAPNCPFCAA|2097-02-02|2365|10284|789|2097|6|2|2|1|2097|789|10284|Saturday|2097Q1|N|Y|N|2487006|2487036|2486641|2486915|N|N|N|N|N| +2487008|AAAAAAAAAOCPFCAA|2097-02-03|2365|10284|789|2097|0|2|3|1|2097|789|10284|Sunday|2097Q1|N|N|N|2487006|2487036|2486642|2486916|N|N|N|N|N| +2487009|AAAAAAAABOCPFCAA|2097-02-04|2365|10284|789|2097|1|2|4|1|2097|789|10284|Monday|2097Q1|N|N|N|2487006|2487036|2486643|2486917|N|N|N|N|N| +2487010|AAAAAAAACOCPFCAA|2097-02-05|2365|10285|789|2097|2|2|5|1|2097|789|10285|Tuesday|2097Q1|N|N|N|2487006|2487036|2486644|2486918|N|N|N|N|N| +2487011|AAAAAAAADOCPFCAA|2097-02-06|2365|10285|789|2097|3|2|6|1|2097|789|10285|Wednesday|2097Q1|N|N|N|2487006|2487036|2486645|2486919|N|N|N|N|N| +2487012|AAAAAAAAEOCPFCAA|2097-02-07|2365|10285|789|2097|4|2|7|1|2097|789|10285|Thursday|2097Q1|N|N|N|2487006|2487036|2486646|2486920|N|N|N|N|N| +2487013|AAAAAAAAFOCPFCAA|2097-02-08|2365|10285|789|2097|5|2|8|1|2097|789|10285|Friday|2097Q1|N|Y|N|2487006|2487036|2486647|2486921|N|N|N|N|N| +2487014|AAAAAAAAGOCPFCAA|2097-02-09|2365|10285|789|2097|6|2|9|1|2097|789|10285|Saturday|2097Q1|N|Y|N|2487006|2487036|2486648|2486922|N|N|N|N|N| +2487015|AAAAAAAAHOCPFCAA|2097-02-10|2365|10285|789|2097|0|2|10|1|2097|789|10285|Sunday|2097Q1|N|N|N|2487006|2487036|2486649|2486923|N|N|N|N|N| +2487016|AAAAAAAAIOCPFCAA|2097-02-11|2365|10285|789|2097|1|2|11|1|2097|789|10285|Monday|2097Q1|N|N|N|2487006|2487036|2486650|2486924|N|N|N|N|N| +2487017|AAAAAAAAJOCPFCAA|2097-02-12|2365|10286|789|2097|2|2|12|1|2097|789|10286|Tuesday|2097Q1|N|N|N|2487006|2487036|2486651|2486925|N|N|N|N|N| +2487018|AAAAAAAAKOCPFCAA|2097-02-13|2365|10286|789|2097|3|2|13|1|2097|789|10286|Wednesday|2097Q1|N|N|N|2487006|2487036|2486652|2486926|N|N|N|N|N| +2487019|AAAAAAAALOCPFCAA|2097-02-14|2365|10286|789|2097|4|2|14|1|2097|789|10286|Thursday|2097Q1|N|N|N|2487006|2487036|2486653|2486927|N|N|N|N|N| +2487020|AAAAAAAAMOCPFCAA|2097-02-15|2365|10286|789|2097|5|2|15|1|2097|789|10286|Friday|2097Q1|N|Y|N|2487006|2487036|2486654|2486928|N|N|N|N|N| +2487021|AAAAAAAANOCPFCAA|2097-02-16|2365|10286|789|2097|6|2|16|1|2097|789|10286|Saturday|2097Q1|N|Y|N|2487006|2487036|2486655|2486929|N|N|N|N|N| +2487022|AAAAAAAAOOCPFCAA|2097-02-17|2365|10286|789|2097|0|2|17|1|2097|789|10286|Sunday|2097Q1|N|N|N|2487006|2487036|2486656|2486930|N|N|N|N|N| +2487023|AAAAAAAAPOCPFCAA|2097-02-18|2365|10286|789|2097|1|2|18|1|2097|789|10286|Monday|2097Q1|N|N|N|2487006|2487036|2486657|2486931|N|N|N|N|N| +2487024|AAAAAAAAAPCPFCAA|2097-02-19|2365|10287|789|2097|2|2|19|1|2097|789|10287|Tuesday|2097Q1|N|N|N|2487006|2487036|2486658|2486932|N|N|N|N|N| +2487025|AAAAAAAABPCPFCAA|2097-02-20|2365|10287|789|2097|3|2|20|1|2097|789|10287|Wednesday|2097Q1|N|N|N|2487006|2487036|2486659|2486933|N|N|N|N|N| +2487026|AAAAAAAACPCPFCAA|2097-02-21|2365|10287|789|2097|4|2|21|1|2097|789|10287|Thursday|2097Q1|N|N|N|2487006|2487036|2486660|2486934|N|N|N|N|N| +2487027|AAAAAAAADPCPFCAA|2097-02-22|2365|10287|789|2097|5|2|22|1|2097|789|10287|Friday|2097Q1|N|Y|N|2487006|2487036|2486661|2486935|N|N|N|N|N| +2487028|AAAAAAAAEPCPFCAA|2097-02-23|2365|10287|789|2097|6|2|23|1|2097|789|10287|Saturday|2097Q1|N|Y|N|2487006|2487036|2486662|2486936|N|N|N|N|N| +2487029|AAAAAAAAFPCPFCAA|2097-02-24|2365|10287|789|2097|0|2|24|1|2097|789|10287|Sunday|2097Q1|N|N|N|2487006|2487036|2486663|2486937|N|N|N|N|N| +2487030|AAAAAAAAGPCPFCAA|2097-02-25|2365|10287|789|2097|1|2|25|1|2097|789|10287|Monday|2097Q1|N|N|N|2487006|2487036|2486664|2486938|N|N|N|N|N| +2487031|AAAAAAAAHPCPFCAA|2097-02-26|2365|10288|789|2097|2|2|26|1|2097|789|10288|Tuesday|2097Q1|N|N|N|2487006|2487036|2486665|2486939|N|N|N|N|N| +2487032|AAAAAAAAIPCPFCAA|2097-02-27|2365|10288|789|2097|3|2|27|1|2097|789|10288|Wednesday|2097Q1|N|N|N|2487006|2487036|2486666|2486940|N|N|N|N|N| +2487033|AAAAAAAAJPCPFCAA|2097-02-28|2365|10288|789|2097|4|2|28|1|2097|789|10288|Thursday|2097Q1|N|N|N|2487006|2487036|2486667|2486941|N|N|N|N|N| +2487034|AAAAAAAAKPCPFCAA|2097-03-01|2366|10288|790|2097|5|3|1|1|2097|790|10288|Friday|2097Q1|N|Y|N|2487034|2487092|2486669|2486942|N|N|N|N|N| +2487035|AAAAAAAALPCPFCAA|2097-03-02|2366|10288|790|2097|6|3|2|1|2097|790|10288|Saturday|2097Q1|N|Y|N|2487034|2487092|2486670|2486943|N|N|N|N|N| +2487036|AAAAAAAAMPCPFCAA|2097-03-03|2366|10288|790|2097|0|3|3|1|2097|790|10288|Sunday|2097Q1|N|N|N|2487034|2487092|2486671|2486944|N|N|N|N|N| +2487037|AAAAAAAANPCPFCAA|2097-03-04|2366|10288|790|2097|1|3|4|1|2097|790|10288|Monday|2097Q1|N|N|N|2487034|2487092|2486672|2486945|N|N|N|N|N| +2487038|AAAAAAAAOPCPFCAA|2097-03-05|2366|10289|790|2097|2|3|5|1|2097|790|10289|Tuesday|2097Q1|N|N|N|2487034|2487092|2486673|2486946|N|N|N|N|N| +2487039|AAAAAAAAPPCPFCAA|2097-03-06|2366|10289|790|2097|3|3|6|1|2097|790|10289|Wednesday|2097Q1|N|N|N|2487034|2487092|2486674|2486947|N|N|N|N|N| +2487040|AAAAAAAAAADPFCAA|2097-03-07|2366|10289|790|2097|4|3|7|1|2097|790|10289|Thursday|2097Q1|N|N|N|2487034|2487092|2486675|2486948|N|N|N|N|N| +2487041|AAAAAAAABADPFCAA|2097-03-08|2366|10289|790|2097|5|3|8|1|2097|790|10289|Friday|2097Q1|N|Y|N|2487034|2487092|2486676|2486949|N|N|N|N|N| +2487042|AAAAAAAACADPFCAA|2097-03-09|2366|10289|790|2097|6|3|9|1|2097|790|10289|Saturday|2097Q1|N|Y|N|2487034|2487092|2486677|2486950|N|N|N|N|N| +2487043|AAAAAAAADADPFCAA|2097-03-10|2366|10289|790|2097|0|3|10|1|2097|790|10289|Sunday|2097Q1|N|N|N|2487034|2487092|2486678|2486951|N|N|N|N|N| +2487044|AAAAAAAAEADPFCAA|2097-03-11|2366|10289|790|2097|1|3|11|1|2097|790|10289|Monday|2097Q1|N|N|N|2487034|2487092|2486679|2486952|N|N|N|N|N| +2487045|AAAAAAAAFADPFCAA|2097-03-12|2366|10290|790|2097|2|3|12|1|2097|790|10290|Tuesday|2097Q1|N|N|N|2487034|2487092|2486680|2486953|N|N|N|N|N| +2487046|AAAAAAAAGADPFCAA|2097-03-13|2366|10290|790|2097|3|3|13|1|2097|790|10290|Wednesday|2097Q1|N|N|N|2487034|2487092|2486681|2486954|N|N|N|N|N| +2487047|AAAAAAAAHADPFCAA|2097-03-14|2366|10290|790|2097|4|3|14|1|2097|790|10290|Thursday|2097Q1|N|N|N|2487034|2487092|2486682|2486955|N|N|N|N|N| +2487048|AAAAAAAAIADPFCAA|2097-03-15|2366|10290|790|2097|5|3|15|1|2097|790|10290|Friday|2097Q1|N|Y|N|2487034|2487092|2486683|2486956|N|N|N|N|N| +2487049|AAAAAAAAJADPFCAA|2097-03-16|2366|10290|790|2097|6|3|16|1|2097|790|10290|Saturday|2097Q1|N|Y|N|2487034|2487092|2486684|2486957|N|N|N|N|N| +2487050|AAAAAAAAKADPFCAA|2097-03-17|2366|10290|790|2097|0|3|17|1|2097|790|10290|Sunday|2097Q1|N|N|N|2487034|2487092|2486685|2486958|N|N|N|N|N| +2487051|AAAAAAAALADPFCAA|2097-03-18|2366|10290|790|2097|1|3|18|1|2097|790|10290|Monday|2097Q1|N|N|N|2487034|2487092|2486686|2486959|N|N|N|N|N| +2487052|AAAAAAAAMADPFCAA|2097-03-19|2366|10291|790|2097|2|3|19|1|2097|790|10291|Tuesday|2097Q1|N|N|N|2487034|2487092|2486687|2486960|N|N|N|N|N| +2487053|AAAAAAAANADPFCAA|2097-03-20|2366|10291|790|2097|3|3|20|1|2097|790|10291|Wednesday|2097Q1|N|N|N|2487034|2487092|2486688|2486961|N|N|N|N|N| +2487054|AAAAAAAAOADPFCAA|2097-03-21|2366|10291|790|2097|4|3|21|1|2097|790|10291|Thursday|2097Q1|N|N|N|2487034|2487092|2486689|2486962|N|N|N|N|N| +2487055|AAAAAAAAPADPFCAA|2097-03-22|2366|10291|790|2097|5|3|22|1|2097|790|10291|Friday|2097Q1|N|Y|N|2487034|2487092|2486690|2486963|N|N|N|N|N| +2487056|AAAAAAAAABDPFCAA|2097-03-23|2366|10291|790|2097|6|3|23|1|2097|790|10291|Saturday|2097Q1|N|Y|N|2487034|2487092|2486691|2486964|N|N|N|N|N| +2487057|AAAAAAAABBDPFCAA|2097-03-24|2366|10291|790|2097|0|3|24|1|2097|790|10291|Sunday|2097Q1|N|N|N|2487034|2487092|2486692|2486965|N|N|N|N|N| +2487058|AAAAAAAACBDPFCAA|2097-03-25|2366|10291|790|2097|1|3|25|1|2097|790|10291|Monday|2097Q1|N|N|N|2487034|2487092|2486693|2486966|N|N|N|N|N| +2487059|AAAAAAAADBDPFCAA|2097-03-26|2366|10292|790|2097|2|3|26|1|2097|790|10292|Tuesday|2097Q1|N|N|N|2487034|2487092|2486694|2486967|N|N|N|N|N| +2487060|AAAAAAAAEBDPFCAA|2097-03-27|2366|10292|790|2097|3|3|27|1|2097|790|10292|Wednesday|2097Q1|N|N|N|2487034|2487092|2486695|2486968|N|N|N|N|N| +2487061|AAAAAAAAFBDPFCAA|2097-03-28|2366|10292|790|2097|4|3|28|1|2097|790|10292|Thursday|2097Q1|N|N|N|2487034|2487092|2486696|2486969|N|N|N|N|N| +2487062|AAAAAAAAGBDPFCAA|2097-03-29|2366|10292|790|2097|5|3|29|1|2097|790|10292|Friday|2097Q1|N|Y|N|2487034|2487092|2486697|2486970|N|N|N|N|N| +2487063|AAAAAAAAHBDPFCAA|2097-03-30|2366|10292|790|2097|6|3|30|1|2097|790|10292|Saturday|2097Q1|N|Y|N|2487034|2487092|2486698|2486971|N|N|N|N|N| +2487064|AAAAAAAAIBDPFCAA|2097-03-31|2366|10292|790|2097|0|3|31|1|2097|790|10292|Sunday|2097Q1|N|N|N|2487034|2487092|2486699|2486972|N|N|N|N|N| +2487065|AAAAAAAAJBDPFCAA|2097-04-01|2367|10292|790|2097|1|4|1|1|2097|790|10292|Monday|2097Q1|N|N|N|2487065|2487154|2486700|2486975|N|N|N|N|N| +2487066|AAAAAAAAKBDPFCAA|2097-04-02|2367|10293|790|2097|2|4|2|2|2097|790|10293|Tuesday|2097Q2|N|N|N|2487065|2487154|2486701|2486976|N|N|N|N|N| +2487067|AAAAAAAALBDPFCAA|2097-04-03|2367|10293|790|2097|3|4|3|2|2097|790|10293|Wednesday|2097Q2|N|N|N|2487065|2487154|2486702|2486977|N|N|N|N|N| +2487068|AAAAAAAAMBDPFCAA|2097-04-04|2367|10293|790|2097|4|4|4|2|2097|790|10293|Thursday|2097Q2|N|N|N|2487065|2487154|2486703|2486978|N|N|N|N|N| +2487069|AAAAAAAANBDPFCAA|2097-04-05|2367|10293|790|2097|5|4|5|2|2097|790|10293|Friday|2097Q2|N|Y|N|2487065|2487154|2486704|2486979|N|N|N|N|N| +2487070|AAAAAAAAOBDPFCAA|2097-04-06|2367|10293|790|2097|6|4|6|2|2097|790|10293|Saturday|2097Q2|N|Y|N|2487065|2487154|2486705|2486980|N|N|N|N|N| +2487071|AAAAAAAAPBDPFCAA|2097-04-07|2367|10293|790|2097|0|4|7|2|2097|790|10293|Sunday|2097Q2|N|N|N|2487065|2487154|2486706|2486981|N|N|N|N|N| +2487072|AAAAAAAAACDPFCAA|2097-04-08|2367|10293|790|2097|1|4|8|2|2097|790|10293|Monday|2097Q2|N|N|N|2487065|2487154|2486707|2486982|N|N|N|N|N| +2487073|AAAAAAAABCDPFCAA|2097-04-09|2367|10294|790|2097|2|4|9|2|2097|790|10294|Tuesday|2097Q2|N|N|N|2487065|2487154|2486708|2486983|N|N|N|N|N| +2487074|AAAAAAAACCDPFCAA|2097-04-10|2367|10294|790|2097|3|4|10|2|2097|790|10294|Wednesday|2097Q2|N|N|N|2487065|2487154|2486709|2486984|N|N|N|N|N| +2487075|AAAAAAAADCDPFCAA|2097-04-11|2367|10294|790|2097|4|4|11|2|2097|790|10294|Thursday|2097Q2|N|N|N|2487065|2487154|2486710|2486985|N|N|N|N|N| +2487076|AAAAAAAAECDPFCAA|2097-04-12|2367|10294|790|2097|5|4|12|2|2097|790|10294|Friday|2097Q2|N|Y|N|2487065|2487154|2486711|2486986|N|N|N|N|N| +2487077|AAAAAAAAFCDPFCAA|2097-04-13|2367|10294|790|2097|6|4|13|2|2097|790|10294|Saturday|2097Q2|N|Y|N|2487065|2487154|2486712|2486987|N|N|N|N|N| +2487078|AAAAAAAAGCDPFCAA|2097-04-14|2367|10294|790|2097|0|4|14|2|2097|790|10294|Sunday|2097Q2|N|N|N|2487065|2487154|2486713|2486988|N|N|N|N|N| +2487079|AAAAAAAAHCDPFCAA|2097-04-15|2367|10294|790|2097|1|4|15|2|2097|790|10294|Monday|2097Q2|N|N|N|2487065|2487154|2486714|2486989|N|N|N|N|N| +2487080|AAAAAAAAICDPFCAA|2097-04-16|2367|10295|790|2097|2|4|16|2|2097|790|10295|Tuesday|2097Q2|N|N|N|2487065|2487154|2486715|2486990|N|N|N|N|N| +2487081|AAAAAAAAJCDPFCAA|2097-04-17|2367|10295|790|2097|3|4|17|2|2097|790|10295|Wednesday|2097Q2|N|N|N|2487065|2487154|2486716|2486991|N|N|N|N|N| +2487082|AAAAAAAAKCDPFCAA|2097-04-18|2367|10295|790|2097|4|4|18|2|2097|790|10295|Thursday|2097Q2|N|N|N|2487065|2487154|2486717|2486992|N|N|N|N|N| +2487083|AAAAAAAALCDPFCAA|2097-04-19|2367|10295|790|2097|5|4|19|2|2097|790|10295|Friday|2097Q2|N|Y|N|2487065|2487154|2486718|2486993|N|N|N|N|N| +2487084|AAAAAAAAMCDPFCAA|2097-04-20|2367|10295|790|2097|6|4|20|2|2097|790|10295|Saturday|2097Q2|N|Y|N|2487065|2487154|2486719|2486994|N|N|N|N|N| +2487085|AAAAAAAANCDPFCAA|2097-04-21|2367|10295|790|2097|0|4|21|2|2097|790|10295|Sunday|2097Q2|N|N|N|2487065|2487154|2486720|2486995|N|N|N|N|N| +2487086|AAAAAAAAOCDPFCAA|2097-04-22|2367|10295|790|2097|1|4|22|2|2097|790|10295|Monday|2097Q2|N|N|N|2487065|2487154|2486721|2486996|N|N|N|N|N| +2487087|AAAAAAAAPCDPFCAA|2097-04-23|2367|10296|790|2097|2|4|23|2|2097|790|10296|Tuesday|2097Q2|N|N|N|2487065|2487154|2486722|2486997|N|N|N|N|N| +2487088|AAAAAAAAADDPFCAA|2097-04-24|2367|10296|790|2097|3|4|24|2|2097|790|10296|Wednesday|2097Q2|N|N|N|2487065|2487154|2486723|2486998|N|N|N|N|N| +2487089|AAAAAAAABDDPFCAA|2097-04-25|2367|10296|790|2097|4|4|25|2|2097|790|10296|Thursday|2097Q2|N|N|N|2487065|2487154|2486724|2486999|N|N|N|N|N| +2487090|AAAAAAAACDDPFCAA|2097-04-26|2367|10296|790|2097|5|4|26|2|2097|790|10296|Friday|2097Q2|N|Y|N|2487065|2487154|2486725|2487000|N|N|N|N|N| +2487091|AAAAAAAADDDPFCAA|2097-04-27|2367|10296|790|2097|6|4|27|2|2097|790|10296|Saturday|2097Q2|N|Y|N|2487065|2487154|2486726|2487001|N|N|N|N|N| +2487092|AAAAAAAAEDDPFCAA|2097-04-28|2367|10296|790|2097|0|4|28|2|2097|790|10296|Sunday|2097Q2|N|N|N|2487065|2487154|2486727|2487002|N|N|N|N|N| +2487093|AAAAAAAAFDDPFCAA|2097-04-29|2367|10296|790|2097|1|4|29|2|2097|790|10296|Monday|2097Q2|N|N|N|2487065|2487154|2486728|2487003|N|N|N|N|N| +2487094|AAAAAAAAGDDPFCAA|2097-04-30|2367|10297|790|2097|2|4|30|2|2097|790|10297|Tuesday|2097Q2|N|N|N|2487065|2487154|2486729|2487004|N|N|N|N|N| +2487095|AAAAAAAAHDDPFCAA|2097-05-01|2368|10297|790|2097|3|5|1|2|2097|790|10297|Wednesday|2097Q2|N|N|N|2487095|2487214|2486730|2487005|N|N|N|N|N| +2487096|AAAAAAAAIDDPFCAA|2097-05-02|2368|10297|790|2097|4|5|2|2|2097|790|10297|Thursday|2097Q2|N|N|N|2487095|2487214|2486731|2487006|N|N|N|N|N| +2487097|AAAAAAAAJDDPFCAA|2097-05-03|2368|10297|790|2097|5|5|3|2|2097|790|10297|Friday|2097Q2|N|Y|N|2487095|2487214|2486732|2487007|N|N|N|N|N| +2487098|AAAAAAAAKDDPFCAA|2097-05-04|2368|10297|790|2097|6|5|4|2|2097|790|10297|Saturday|2097Q2|N|Y|N|2487095|2487214|2486733|2487008|N|N|N|N|N| +2487099|AAAAAAAALDDPFCAA|2097-05-05|2368|10297|790|2097|0|5|5|2|2097|790|10297|Sunday|2097Q2|N|N|N|2487095|2487214|2486734|2487009|N|N|N|N|N| +2487100|AAAAAAAAMDDPFCAA|2097-05-06|2368|10297|790|2097|1|5|6|2|2097|790|10297|Monday|2097Q2|N|N|N|2487095|2487214|2486735|2487010|N|N|N|N|N| +2487101|AAAAAAAANDDPFCAA|2097-05-07|2368|10298|790|2097|2|5|7|2|2097|790|10298|Tuesday|2097Q2|N|N|N|2487095|2487214|2486736|2487011|N|N|N|N|N| +2487102|AAAAAAAAODDPFCAA|2097-05-08|2368|10298|790|2097|3|5|8|2|2097|790|10298|Wednesday|2097Q2|N|N|N|2487095|2487214|2486737|2487012|N|N|N|N|N| +2487103|AAAAAAAAPDDPFCAA|2097-05-09|2368|10298|790|2097|4|5|9|2|2097|790|10298|Thursday|2097Q2|N|N|N|2487095|2487214|2486738|2487013|N|N|N|N|N| +2487104|AAAAAAAAAEDPFCAA|2097-05-10|2368|10298|790|2097|5|5|10|2|2097|790|10298|Friday|2097Q2|N|Y|N|2487095|2487214|2486739|2487014|N|N|N|N|N| +2487105|AAAAAAAABEDPFCAA|2097-05-11|2368|10298|790|2097|6|5|11|2|2097|790|10298|Saturday|2097Q2|N|Y|N|2487095|2487214|2486740|2487015|N|N|N|N|N| +2487106|AAAAAAAACEDPFCAA|2097-05-12|2368|10298|790|2097|0|5|12|2|2097|790|10298|Sunday|2097Q2|N|N|N|2487095|2487214|2486741|2487016|N|N|N|N|N| +2487107|AAAAAAAADEDPFCAA|2097-05-13|2368|10298|790|2097|1|5|13|2|2097|790|10298|Monday|2097Q2|N|N|N|2487095|2487214|2486742|2487017|N|N|N|N|N| +2487108|AAAAAAAAEEDPFCAA|2097-05-14|2368|10299|790|2097|2|5|14|2|2097|790|10299|Tuesday|2097Q2|N|N|N|2487095|2487214|2486743|2487018|N|N|N|N|N| +2487109|AAAAAAAAFEDPFCAA|2097-05-15|2368|10299|790|2097|3|5|15|2|2097|790|10299|Wednesday|2097Q2|N|N|N|2487095|2487214|2486744|2487019|N|N|N|N|N| +2487110|AAAAAAAAGEDPFCAA|2097-05-16|2368|10299|790|2097|4|5|16|2|2097|790|10299|Thursday|2097Q2|N|N|N|2487095|2487214|2486745|2487020|N|N|N|N|N| +2487111|AAAAAAAAHEDPFCAA|2097-05-17|2368|10299|790|2097|5|5|17|2|2097|790|10299|Friday|2097Q2|N|Y|N|2487095|2487214|2486746|2487021|N|N|N|N|N| +2487112|AAAAAAAAIEDPFCAA|2097-05-18|2368|10299|790|2097|6|5|18|2|2097|790|10299|Saturday|2097Q2|N|Y|N|2487095|2487214|2486747|2487022|N|N|N|N|N| +2487113|AAAAAAAAJEDPFCAA|2097-05-19|2368|10299|790|2097|0|5|19|2|2097|790|10299|Sunday|2097Q2|N|N|N|2487095|2487214|2486748|2487023|N|N|N|N|N| +2487114|AAAAAAAAKEDPFCAA|2097-05-20|2368|10299|790|2097|1|5|20|2|2097|790|10299|Monday|2097Q2|N|N|N|2487095|2487214|2486749|2487024|N|N|N|N|N| +2487115|AAAAAAAALEDPFCAA|2097-05-21|2368|10300|790|2097|2|5|21|2|2097|790|10300|Tuesday|2097Q2|N|N|N|2487095|2487214|2486750|2487025|N|N|N|N|N| +2487116|AAAAAAAAMEDPFCAA|2097-05-22|2368|10300|790|2097|3|5|22|2|2097|790|10300|Wednesday|2097Q2|N|N|N|2487095|2487214|2486751|2487026|N|N|N|N|N| +2487117|AAAAAAAANEDPFCAA|2097-05-23|2368|10300|790|2097|4|5|23|2|2097|790|10300|Thursday|2097Q2|N|N|N|2487095|2487214|2486752|2487027|N|N|N|N|N| +2487118|AAAAAAAAOEDPFCAA|2097-05-24|2368|10300|790|2097|5|5|24|2|2097|790|10300|Friday|2097Q2|N|Y|N|2487095|2487214|2486753|2487028|N|N|N|N|N| +2487119|AAAAAAAAPEDPFCAA|2097-05-25|2368|10300|790|2097|6|5|25|2|2097|790|10300|Saturday|2097Q2|N|Y|N|2487095|2487214|2486754|2487029|N|N|N|N|N| +2487120|AAAAAAAAAFDPFCAA|2097-05-26|2368|10300|790|2097|0|5|26|2|2097|790|10300|Sunday|2097Q2|N|N|N|2487095|2487214|2486755|2487030|N|N|N|N|N| +2487121|AAAAAAAABFDPFCAA|2097-05-27|2368|10300|790|2097|1|5|27|2|2097|790|10300|Monday|2097Q2|N|N|N|2487095|2487214|2486756|2487031|N|N|N|N|N| +2487122|AAAAAAAACFDPFCAA|2097-05-28|2368|10301|790|2097|2|5|28|2|2097|790|10301|Tuesday|2097Q2|N|N|N|2487095|2487214|2486757|2487032|N|N|N|N|N| +2487123|AAAAAAAADFDPFCAA|2097-05-29|2368|10301|790|2097|3|5|29|2|2097|790|10301|Wednesday|2097Q2|N|N|N|2487095|2487214|2486758|2487033|N|N|N|N|N| +2487124|AAAAAAAAEFDPFCAA|2097-05-30|2368|10301|790|2097|4|5|30|2|2097|790|10301|Thursday|2097Q2|N|N|N|2487095|2487214|2486759|2487034|N|N|N|N|N| +2487125|AAAAAAAAFFDPFCAA|2097-05-31|2368|10301|790|2097|5|5|31|2|2097|790|10301|Friday|2097Q2|N|Y|N|2487095|2487214|2486760|2487035|N|N|N|N|N| +2487126|AAAAAAAAGFDPFCAA|2097-06-01|2369|10301|791|2097|6|6|1|2|2097|791|10301|Saturday|2097Q2|N|Y|N|2487126|2487276|2486761|2487036|N|N|N|N|N| +2487127|AAAAAAAAHFDPFCAA|2097-06-02|2369|10301|791|2097|0|6|2|2|2097|791|10301|Sunday|2097Q2|N|N|N|2487126|2487276|2486762|2487037|N|N|N|N|N| +2487128|AAAAAAAAIFDPFCAA|2097-06-03|2369|10301|791|2097|1|6|3|2|2097|791|10301|Monday|2097Q2|N|N|N|2487126|2487276|2486763|2487038|N|N|N|N|N| +2487129|AAAAAAAAJFDPFCAA|2097-06-04|2369|10302|791|2097|2|6|4|2|2097|791|10302|Tuesday|2097Q2|N|N|N|2487126|2487276|2486764|2487039|N|N|N|N|N| +2487130|AAAAAAAAKFDPFCAA|2097-06-05|2369|10302|791|2097|3|6|5|2|2097|791|10302|Wednesday|2097Q2|N|N|N|2487126|2487276|2486765|2487040|N|N|N|N|N| +2487131|AAAAAAAALFDPFCAA|2097-06-06|2369|10302|791|2097|4|6|6|2|2097|791|10302|Thursday|2097Q2|N|N|N|2487126|2487276|2486766|2487041|N|N|N|N|N| +2487132|AAAAAAAAMFDPFCAA|2097-06-07|2369|10302|791|2097|5|6|7|2|2097|791|10302|Friday|2097Q2|N|Y|N|2487126|2487276|2486767|2487042|N|N|N|N|N| +2487133|AAAAAAAANFDPFCAA|2097-06-08|2369|10302|791|2097|6|6|8|2|2097|791|10302|Saturday|2097Q2|N|Y|N|2487126|2487276|2486768|2487043|N|N|N|N|N| +2487134|AAAAAAAAOFDPFCAA|2097-06-09|2369|10302|791|2097|0|6|9|2|2097|791|10302|Sunday|2097Q2|N|N|N|2487126|2487276|2486769|2487044|N|N|N|N|N| +2487135|AAAAAAAAPFDPFCAA|2097-06-10|2369|10302|791|2097|1|6|10|2|2097|791|10302|Monday|2097Q2|N|N|N|2487126|2487276|2486770|2487045|N|N|N|N|N| +2487136|AAAAAAAAAGDPFCAA|2097-06-11|2369|10303|791|2097|2|6|11|2|2097|791|10303|Tuesday|2097Q2|N|N|N|2487126|2487276|2486771|2487046|N|N|N|N|N| +2487137|AAAAAAAABGDPFCAA|2097-06-12|2369|10303|791|2097|3|6|12|2|2097|791|10303|Wednesday|2097Q2|N|N|N|2487126|2487276|2486772|2487047|N|N|N|N|N| +2487138|AAAAAAAACGDPFCAA|2097-06-13|2369|10303|791|2097|4|6|13|2|2097|791|10303|Thursday|2097Q2|N|N|N|2487126|2487276|2486773|2487048|N|N|N|N|N| +2487139|AAAAAAAADGDPFCAA|2097-06-14|2369|10303|791|2097|5|6|14|2|2097|791|10303|Friday|2097Q2|N|Y|N|2487126|2487276|2486774|2487049|N|N|N|N|N| +2487140|AAAAAAAAEGDPFCAA|2097-06-15|2369|10303|791|2097|6|6|15|2|2097|791|10303|Saturday|2097Q2|N|Y|N|2487126|2487276|2486775|2487050|N|N|N|N|N| +2487141|AAAAAAAAFGDPFCAA|2097-06-16|2369|10303|791|2097|0|6|16|2|2097|791|10303|Sunday|2097Q2|N|N|N|2487126|2487276|2486776|2487051|N|N|N|N|N| +2487142|AAAAAAAAGGDPFCAA|2097-06-17|2369|10303|791|2097|1|6|17|2|2097|791|10303|Monday|2097Q2|N|N|N|2487126|2487276|2486777|2487052|N|N|N|N|N| +2487143|AAAAAAAAHGDPFCAA|2097-06-18|2369|10304|791|2097|2|6|18|2|2097|791|10304|Tuesday|2097Q2|N|N|N|2487126|2487276|2486778|2487053|N|N|N|N|N| +2487144|AAAAAAAAIGDPFCAA|2097-06-19|2369|10304|791|2097|3|6|19|2|2097|791|10304|Wednesday|2097Q2|N|N|N|2487126|2487276|2486779|2487054|N|N|N|N|N| +2487145|AAAAAAAAJGDPFCAA|2097-06-20|2369|10304|791|2097|4|6|20|2|2097|791|10304|Thursday|2097Q2|N|N|N|2487126|2487276|2486780|2487055|N|N|N|N|N| +2487146|AAAAAAAAKGDPFCAA|2097-06-21|2369|10304|791|2097|5|6|21|2|2097|791|10304|Friday|2097Q2|N|Y|N|2487126|2487276|2486781|2487056|N|N|N|N|N| +2487147|AAAAAAAALGDPFCAA|2097-06-22|2369|10304|791|2097|6|6|22|2|2097|791|10304|Saturday|2097Q2|N|Y|N|2487126|2487276|2486782|2487057|N|N|N|N|N| +2487148|AAAAAAAAMGDPFCAA|2097-06-23|2369|10304|791|2097|0|6|23|2|2097|791|10304|Sunday|2097Q2|N|N|N|2487126|2487276|2486783|2487058|N|N|N|N|N| +2487149|AAAAAAAANGDPFCAA|2097-06-24|2369|10304|791|2097|1|6|24|2|2097|791|10304|Monday|2097Q2|N|N|N|2487126|2487276|2486784|2487059|N|N|N|N|N| +2487150|AAAAAAAAOGDPFCAA|2097-06-25|2369|10305|791|2097|2|6|25|2|2097|791|10305|Tuesday|2097Q2|N|N|N|2487126|2487276|2486785|2487060|N|N|N|N|N| +2487151|AAAAAAAAPGDPFCAA|2097-06-26|2369|10305|791|2097|3|6|26|2|2097|791|10305|Wednesday|2097Q2|N|N|N|2487126|2487276|2486786|2487061|N|N|N|N|N| +2487152|AAAAAAAAAHDPFCAA|2097-06-27|2369|10305|791|2097|4|6|27|2|2097|791|10305|Thursday|2097Q2|N|N|N|2487126|2487276|2486787|2487062|N|N|N|N|N| +2487153|AAAAAAAABHDPFCAA|2097-06-28|2369|10305|791|2097|5|6|28|2|2097|791|10305|Friday|2097Q2|N|Y|N|2487126|2487276|2486788|2487063|N|N|N|N|N| +2487154|AAAAAAAACHDPFCAA|2097-06-29|2369|10305|791|2097|6|6|29|2|2097|791|10305|Saturday|2097Q2|N|Y|N|2487126|2487276|2486789|2487064|N|N|N|N|N| +2487155|AAAAAAAADHDPFCAA|2097-06-30|2369|10305|791|2097|0|6|30|2|2097|791|10305|Sunday|2097Q2|N|N|N|2487126|2487276|2486790|2487065|N|N|N|N|N| +2487156|AAAAAAAAEHDPFCAA|2097-07-01|2370|10305|791|2097|1|7|1|2|2097|791|10305|Monday|2097Q2|N|N|N|2487156|2487336|2486791|2487065|N|N|N|N|N| +2487157|AAAAAAAAFHDPFCAA|2097-07-02|2370|10306|791|2097|2|7|2|3|2097|791|10306|Tuesday|2097Q3|N|N|N|2487156|2487336|2486792|2487066|N|N|N|N|N| +2487158|AAAAAAAAGHDPFCAA|2097-07-03|2370|10306|791|2097|3|7|3|3|2097|791|10306|Wednesday|2097Q3|N|N|N|2487156|2487336|2486793|2487067|N|N|N|N|N| +2487159|AAAAAAAAHHDPFCAA|2097-07-04|2370|10306|791|2097|4|7|4|3|2097|791|10306|Thursday|2097Q3|N|N|N|2487156|2487336|2486794|2487068|N|N|N|N|N| +2487160|AAAAAAAAIHDPFCAA|2097-07-05|2370|10306|791|2097|5|7|5|3|2097|791|10306|Friday|2097Q3|Y|Y|N|2487156|2487336|2486795|2487069|N|N|N|N|N| +2487161|AAAAAAAAJHDPFCAA|2097-07-06|2370|10306|791|2097|6|7|6|3|2097|791|10306|Saturday|2097Q3|N|Y|Y|2487156|2487336|2486796|2487070|N|N|N|N|N| +2487162|AAAAAAAAKHDPFCAA|2097-07-07|2370|10306|791|2097|0|7|7|3|2097|791|10306|Sunday|2097Q3|N|N|N|2487156|2487336|2486797|2487071|N|N|N|N|N| +2487163|AAAAAAAALHDPFCAA|2097-07-08|2370|10306|791|2097|1|7|8|3|2097|791|10306|Monday|2097Q3|N|N|N|2487156|2487336|2486798|2487072|N|N|N|N|N| +2487164|AAAAAAAAMHDPFCAA|2097-07-09|2370|10307|791|2097|2|7|9|3|2097|791|10307|Tuesday|2097Q3|N|N|N|2487156|2487336|2486799|2487073|N|N|N|N|N| +2487165|AAAAAAAANHDPFCAA|2097-07-10|2370|10307|791|2097|3|7|10|3|2097|791|10307|Wednesday|2097Q3|N|N|N|2487156|2487336|2486800|2487074|N|N|N|N|N| +2487166|AAAAAAAAOHDPFCAA|2097-07-11|2370|10307|791|2097|4|7|11|3|2097|791|10307|Thursday|2097Q3|N|N|N|2487156|2487336|2486801|2487075|N|N|N|N|N| +2487167|AAAAAAAAPHDPFCAA|2097-07-12|2370|10307|791|2097|5|7|12|3|2097|791|10307|Friday|2097Q3|N|Y|N|2487156|2487336|2486802|2487076|N|N|N|N|N| +2487168|AAAAAAAAAIDPFCAA|2097-07-13|2370|10307|791|2097|6|7|13|3|2097|791|10307|Saturday|2097Q3|N|Y|N|2487156|2487336|2486803|2487077|N|N|N|N|N| +2487169|AAAAAAAABIDPFCAA|2097-07-14|2370|10307|791|2097|0|7|14|3|2097|791|10307|Sunday|2097Q3|N|N|N|2487156|2487336|2486804|2487078|N|N|N|N|N| +2487170|AAAAAAAACIDPFCAA|2097-07-15|2370|10307|791|2097|1|7|15|3|2097|791|10307|Monday|2097Q3|N|N|N|2487156|2487336|2486805|2487079|N|N|N|N|N| +2487171|AAAAAAAADIDPFCAA|2097-07-16|2370|10308|791|2097|2|7|16|3|2097|791|10308|Tuesday|2097Q3|N|N|N|2487156|2487336|2486806|2487080|N|N|N|N|N| +2487172|AAAAAAAAEIDPFCAA|2097-07-17|2370|10308|791|2097|3|7|17|3|2097|791|10308|Wednesday|2097Q3|N|N|N|2487156|2487336|2486807|2487081|N|N|N|N|N| +2487173|AAAAAAAAFIDPFCAA|2097-07-18|2370|10308|791|2097|4|7|18|3|2097|791|10308|Thursday|2097Q3|N|N|N|2487156|2487336|2486808|2487082|N|N|N|N|N| +2487174|AAAAAAAAGIDPFCAA|2097-07-19|2370|10308|791|2097|5|7|19|3|2097|791|10308|Friday|2097Q3|N|Y|N|2487156|2487336|2486809|2487083|N|N|N|N|N| +2487175|AAAAAAAAHIDPFCAA|2097-07-20|2370|10308|791|2097|6|7|20|3|2097|791|10308|Saturday|2097Q3|N|Y|N|2487156|2487336|2486810|2487084|N|N|N|N|N| +2487176|AAAAAAAAIIDPFCAA|2097-07-21|2370|10308|791|2097|0|7|21|3|2097|791|10308|Sunday|2097Q3|N|N|N|2487156|2487336|2486811|2487085|N|N|N|N|N| +2487177|AAAAAAAAJIDPFCAA|2097-07-22|2370|10308|791|2097|1|7|22|3|2097|791|10308|Monday|2097Q3|N|N|N|2487156|2487336|2486812|2487086|N|N|N|N|N| +2487178|AAAAAAAAKIDPFCAA|2097-07-23|2370|10309|791|2097|2|7|23|3|2097|791|10309|Tuesday|2097Q3|N|N|N|2487156|2487336|2486813|2487087|N|N|N|N|N| +2487179|AAAAAAAALIDPFCAA|2097-07-24|2370|10309|791|2097|3|7|24|3|2097|791|10309|Wednesday|2097Q3|N|N|N|2487156|2487336|2486814|2487088|N|N|N|N|N| +2487180|AAAAAAAAMIDPFCAA|2097-07-25|2370|10309|791|2097|4|7|25|3|2097|791|10309|Thursday|2097Q3|N|N|N|2487156|2487336|2486815|2487089|N|N|N|N|N| +2487181|AAAAAAAANIDPFCAA|2097-07-26|2370|10309|791|2097|5|7|26|3|2097|791|10309|Friday|2097Q3|N|Y|N|2487156|2487336|2486816|2487090|N|N|N|N|N| +2487182|AAAAAAAAOIDPFCAA|2097-07-27|2370|10309|791|2097|6|7|27|3|2097|791|10309|Saturday|2097Q3|N|Y|N|2487156|2487336|2486817|2487091|N|N|N|N|N| +2487183|AAAAAAAAPIDPFCAA|2097-07-28|2370|10309|791|2097|0|7|28|3|2097|791|10309|Sunday|2097Q3|N|N|N|2487156|2487336|2486818|2487092|N|N|N|N|N| +2487184|AAAAAAAAAJDPFCAA|2097-07-29|2370|10309|791|2097|1|7|29|3|2097|791|10309|Monday|2097Q3|N|N|N|2487156|2487336|2486819|2487093|N|N|N|N|N| +2487185|AAAAAAAABJDPFCAA|2097-07-30|2370|10310|791|2097|2|7|30|3|2097|791|10310|Tuesday|2097Q3|N|N|N|2487156|2487336|2486820|2487094|N|N|N|N|N| +2487186|AAAAAAAACJDPFCAA|2097-07-31|2370|10310|791|2097|3|7|31|3|2097|791|10310|Wednesday|2097Q3|N|N|N|2487156|2487336|2486821|2487095|N|N|N|N|N| +2487187|AAAAAAAADJDPFCAA|2097-08-01|2371|10310|791|2097|4|8|1|3|2097|791|10310|Thursday|2097Q3|N|N|N|2487187|2487398|2486822|2487096|N|N|N|N|N| +2487188|AAAAAAAAEJDPFCAA|2097-08-02|2371|10310|791|2097|5|8|2|3|2097|791|10310|Friday|2097Q3|N|Y|N|2487187|2487398|2486823|2487097|N|N|N|N|N| +2487189|AAAAAAAAFJDPFCAA|2097-08-03|2371|10310|791|2097|6|8|3|3|2097|791|10310|Saturday|2097Q3|N|Y|N|2487187|2487398|2486824|2487098|N|N|N|N|N| +2487190|AAAAAAAAGJDPFCAA|2097-08-04|2371|10310|791|2097|0|8|4|3|2097|791|10310|Sunday|2097Q3|N|N|N|2487187|2487398|2486825|2487099|N|N|N|N|N| +2487191|AAAAAAAAHJDPFCAA|2097-08-05|2371|10310|791|2097|1|8|5|3|2097|791|10310|Monday|2097Q3|N|N|N|2487187|2487398|2486826|2487100|N|N|N|N|N| +2487192|AAAAAAAAIJDPFCAA|2097-08-06|2371|10311|791|2097|2|8|6|3|2097|791|10311|Tuesday|2097Q3|N|N|N|2487187|2487398|2486827|2487101|N|N|N|N|N| +2487193|AAAAAAAAJJDPFCAA|2097-08-07|2371|10311|791|2097|3|8|7|3|2097|791|10311|Wednesday|2097Q3|N|N|N|2487187|2487398|2486828|2487102|N|N|N|N|N| +2487194|AAAAAAAAKJDPFCAA|2097-08-08|2371|10311|791|2097|4|8|8|3|2097|791|10311|Thursday|2097Q3|N|N|N|2487187|2487398|2486829|2487103|N|N|N|N|N| +2487195|AAAAAAAALJDPFCAA|2097-08-09|2371|10311|791|2097|5|8|9|3|2097|791|10311|Friday|2097Q3|N|Y|N|2487187|2487398|2486830|2487104|N|N|N|N|N| +2487196|AAAAAAAAMJDPFCAA|2097-08-10|2371|10311|791|2097|6|8|10|3|2097|791|10311|Saturday|2097Q3|N|Y|N|2487187|2487398|2486831|2487105|N|N|N|N|N| +2487197|AAAAAAAANJDPFCAA|2097-08-11|2371|10311|791|2097|0|8|11|3|2097|791|10311|Sunday|2097Q3|N|N|N|2487187|2487398|2486832|2487106|N|N|N|N|N| +2487198|AAAAAAAAOJDPFCAA|2097-08-12|2371|10311|791|2097|1|8|12|3|2097|791|10311|Monday|2097Q3|N|N|N|2487187|2487398|2486833|2487107|N|N|N|N|N| +2487199|AAAAAAAAPJDPFCAA|2097-08-13|2371|10312|791|2097|2|8|13|3|2097|791|10312|Tuesday|2097Q3|N|N|N|2487187|2487398|2486834|2487108|N|N|N|N|N| +2487200|AAAAAAAAAKDPFCAA|2097-08-14|2371|10312|791|2097|3|8|14|3|2097|791|10312|Wednesday|2097Q3|N|N|N|2487187|2487398|2486835|2487109|N|N|N|N|N| +2487201|AAAAAAAABKDPFCAA|2097-08-15|2371|10312|791|2097|4|8|15|3|2097|791|10312|Thursday|2097Q3|N|N|N|2487187|2487398|2486836|2487110|N|N|N|N|N| +2487202|AAAAAAAACKDPFCAA|2097-08-16|2371|10312|791|2097|5|8|16|3|2097|791|10312|Friday|2097Q3|N|Y|N|2487187|2487398|2486837|2487111|N|N|N|N|N| +2487203|AAAAAAAADKDPFCAA|2097-08-17|2371|10312|791|2097|6|8|17|3|2097|791|10312|Saturday|2097Q3|N|Y|N|2487187|2487398|2486838|2487112|N|N|N|N|N| +2487204|AAAAAAAAEKDPFCAA|2097-08-18|2371|10312|791|2097|0|8|18|3|2097|791|10312|Sunday|2097Q3|N|N|N|2487187|2487398|2486839|2487113|N|N|N|N|N| +2487205|AAAAAAAAFKDPFCAA|2097-08-19|2371|10312|791|2097|1|8|19|3|2097|791|10312|Monday|2097Q3|N|N|N|2487187|2487398|2486840|2487114|N|N|N|N|N| +2487206|AAAAAAAAGKDPFCAA|2097-08-20|2371|10313|791|2097|2|8|20|3|2097|791|10313|Tuesday|2097Q3|N|N|N|2487187|2487398|2486841|2487115|N|N|N|N|N| +2487207|AAAAAAAAHKDPFCAA|2097-08-21|2371|10313|791|2097|3|8|21|3|2097|791|10313|Wednesday|2097Q3|N|N|N|2487187|2487398|2486842|2487116|N|N|N|N|N| +2487208|AAAAAAAAIKDPFCAA|2097-08-22|2371|10313|791|2097|4|8|22|3|2097|791|10313|Thursday|2097Q3|N|N|N|2487187|2487398|2486843|2487117|N|N|N|N|N| +2487209|AAAAAAAAJKDPFCAA|2097-08-23|2371|10313|791|2097|5|8|23|3|2097|791|10313|Friday|2097Q3|N|Y|N|2487187|2487398|2486844|2487118|N|N|N|N|N| +2487210|AAAAAAAAKKDPFCAA|2097-08-24|2371|10313|791|2097|6|8|24|3|2097|791|10313|Saturday|2097Q3|N|Y|N|2487187|2487398|2486845|2487119|N|N|N|N|N| +2487211|AAAAAAAALKDPFCAA|2097-08-25|2371|10313|791|2097|0|8|25|3|2097|791|10313|Sunday|2097Q3|N|N|N|2487187|2487398|2486846|2487120|N|N|N|N|N| +2487212|AAAAAAAAMKDPFCAA|2097-08-26|2371|10313|791|2097|1|8|26|3|2097|791|10313|Monday|2097Q3|N|N|N|2487187|2487398|2486847|2487121|N|N|N|N|N| +2487213|AAAAAAAANKDPFCAA|2097-08-27|2371|10314|791|2097|2|8|27|3|2097|791|10314|Tuesday|2097Q3|N|N|N|2487187|2487398|2486848|2487122|N|N|N|N|N| +2487214|AAAAAAAAOKDPFCAA|2097-08-28|2371|10314|791|2097|3|8|28|3|2097|791|10314|Wednesday|2097Q3|N|N|N|2487187|2487398|2486849|2487123|N|N|N|N|N| +2487215|AAAAAAAAPKDPFCAA|2097-08-29|2371|10314|791|2097|4|8|29|3|2097|791|10314|Thursday|2097Q3|N|N|N|2487187|2487398|2486850|2487124|N|N|N|N|N| +2487216|AAAAAAAAALDPFCAA|2097-08-30|2371|10314|791|2097|5|8|30|3|2097|791|10314|Friday|2097Q3|N|Y|N|2487187|2487398|2486851|2487125|N|N|N|N|N| +2487217|AAAAAAAABLDPFCAA|2097-08-31|2371|10314|791|2097|6|8|31|3|2097|791|10314|Saturday|2097Q3|N|Y|N|2487187|2487398|2486852|2487126|N|N|N|N|N| +2487218|AAAAAAAACLDPFCAA|2097-09-01|2372|10314|792|2097|0|9|1|3|2097|792|10314|Sunday|2097Q3|N|N|N|2487218|2487460|2486853|2487127|N|N|N|N|N| +2487219|AAAAAAAADLDPFCAA|2097-09-02|2372|10314|792|2097|1|9|2|3|2097|792|10314|Monday|2097Q3|N|N|N|2487218|2487460|2486854|2487128|N|N|N|N|N| +2487220|AAAAAAAAELDPFCAA|2097-09-03|2372|10315|792|2097|2|9|3|3|2097|792|10315|Tuesday|2097Q3|N|N|N|2487218|2487460|2486855|2487129|N|N|N|N|N| +2487221|AAAAAAAAFLDPFCAA|2097-09-04|2372|10315|792|2097|3|9|4|3|2097|792|10315|Wednesday|2097Q3|N|N|N|2487218|2487460|2486856|2487130|N|N|N|N|N| +2487222|AAAAAAAAGLDPFCAA|2097-09-05|2372|10315|792|2097|4|9|5|3|2097|792|10315|Thursday|2097Q3|N|N|N|2487218|2487460|2486857|2487131|N|N|N|N|N| +2487223|AAAAAAAAHLDPFCAA|2097-09-06|2372|10315|792|2097|5|9|6|3|2097|792|10315|Friday|2097Q3|N|Y|N|2487218|2487460|2486858|2487132|N|N|N|N|N| +2487224|AAAAAAAAILDPFCAA|2097-09-07|2372|10315|792|2097|6|9|7|3|2097|792|10315|Saturday|2097Q3|N|Y|N|2487218|2487460|2486859|2487133|N|N|N|N|N| +2487225|AAAAAAAAJLDPFCAA|2097-09-08|2372|10315|792|2097|0|9|8|3|2097|792|10315|Sunday|2097Q3|N|N|N|2487218|2487460|2486860|2487134|N|N|N|N|N| +2487226|AAAAAAAAKLDPFCAA|2097-09-09|2372|10315|792|2097|1|9|9|3|2097|792|10315|Monday|2097Q3|N|N|N|2487218|2487460|2486861|2487135|N|N|N|N|N| +2487227|AAAAAAAALLDPFCAA|2097-09-10|2372|10316|792|2097|2|9|10|3|2097|792|10316|Tuesday|2097Q3|N|N|N|2487218|2487460|2486862|2487136|N|N|N|N|N| +2487228|AAAAAAAAMLDPFCAA|2097-09-11|2372|10316|792|2097|3|9|11|3|2097|792|10316|Wednesday|2097Q3|N|N|N|2487218|2487460|2486863|2487137|N|N|N|N|N| +2487229|AAAAAAAANLDPFCAA|2097-09-12|2372|10316|792|2097|4|9|12|3|2097|792|10316|Thursday|2097Q3|N|N|N|2487218|2487460|2486864|2487138|N|N|N|N|N| +2487230|AAAAAAAAOLDPFCAA|2097-09-13|2372|10316|792|2097|5|9|13|3|2097|792|10316|Friday|2097Q3|N|Y|N|2487218|2487460|2486865|2487139|N|N|N|N|N| +2487231|AAAAAAAAPLDPFCAA|2097-09-14|2372|10316|792|2097|6|9|14|3|2097|792|10316|Saturday|2097Q3|N|Y|N|2487218|2487460|2486866|2487140|N|N|N|N|N| +2487232|AAAAAAAAAMDPFCAA|2097-09-15|2372|10316|792|2097|0|9|15|3|2097|792|10316|Sunday|2097Q3|N|N|N|2487218|2487460|2486867|2487141|N|N|N|N|N| +2487233|AAAAAAAABMDPFCAA|2097-09-16|2372|10316|792|2097|1|9|16|3|2097|792|10316|Monday|2097Q3|N|N|N|2487218|2487460|2486868|2487142|N|N|N|N|N| +2487234|AAAAAAAACMDPFCAA|2097-09-17|2372|10317|792|2097|2|9|17|3|2097|792|10317|Tuesday|2097Q3|N|N|N|2487218|2487460|2486869|2487143|N|N|N|N|N| +2487235|AAAAAAAADMDPFCAA|2097-09-18|2372|10317|792|2097|3|9|18|3|2097|792|10317|Wednesday|2097Q3|N|N|N|2487218|2487460|2486870|2487144|N|N|N|N|N| +2487236|AAAAAAAAEMDPFCAA|2097-09-19|2372|10317|792|2097|4|9|19|3|2097|792|10317|Thursday|2097Q3|N|N|N|2487218|2487460|2486871|2487145|N|N|N|N|N| +2487237|AAAAAAAAFMDPFCAA|2097-09-20|2372|10317|792|2097|5|9|20|3|2097|792|10317|Friday|2097Q3|N|Y|N|2487218|2487460|2486872|2487146|N|N|N|N|N| +2487238|AAAAAAAAGMDPFCAA|2097-09-21|2372|10317|792|2097|6|9|21|3|2097|792|10317|Saturday|2097Q3|N|Y|N|2487218|2487460|2486873|2487147|N|N|N|N|N| +2487239|AAAAAAAAHMDPFCAA|2097-09-22|2372|10317|792|2097|0|9|22|3|2097|792|10317|Sunday|2097Q3|N|N|N|2487218|2487460|2486874|2487148|N|N|N|N|N| +2487240|AAAAAAAAIMDPFCAA|2097-09-23|2372|10317|792|2097|1|9|23|3|2097|792|10317|Monday|2097Q3|N|N|N|2487218|2487460|2486875|2487149|N|N|N|N|N| +2487241|AAAAAAAAJMDPFCAA|2097-09-24|2372|10318|792|2097|2|9|24|3|2097|792|10318|Tuesday|2097Q3|N|N|N|2487218|2487460|2486876|2487150|N|N|N|N|N| +2487242|AAAAAAAAKMDPFCAA|2097-09-25|2372|10318|792|2097|3|9|25|3|2097|792|10318|Wednesday|2097Q3|N|N|N|2487218|2487460|2486877|2487151|N|N|N|N|N| +2487243|AAAAAAAALMDPFCAA|2097-09-26|2372|10318|792|2097|4|9|26|3|2097|792|10318|Thursday|2097Q3|N|N|N|2487218|2487460|2486878|2487152|N|N|N|N|N| +2487244|AAAAAAAAMMDPFCAA|2097-09-27|2372|10318|792|2097|5|9|27|3|2097|792|10318|Friday|2097Q3|N|Y|N|2487218|2487460|2486879|2487153|N|N|N|N|N| +2487245|AAAAAAAANMDPFCAA|2097-09-28|2372|10318|792|2097|6|9|28|3|2097|792|10318|Saturday|2097Q3|N|Y|N|2487218|2487460|2486880|2487154|N|N|N|N|N| +2487246|AAAAAAAAOMDPFCAA|2097-09-29|2372|10318|792|2097|0|9|29|3|2097|792|10318|Sunday|2097Q3|N|N|N|2487218|2487460|2486881|2487155|N|N|N|N|N| +2487247|AAAAAAAAPMDPFCAA|2097-09-30|2372|10318|792|2097|1|9|30|3|2097|792|10318|Monday|2097Q3|N|N|N|2487218|2487460|2486882|2487156|N|N|N|N|N| +2487248|AAAAAAAAANDPFCAA|2097-10-01|2373|10319|792|2097|2|10|1|3|2097|792|10319|Tuesday|2097Q3|N|N|N|2487248|2487520|2486883|2487156|N|N|N|N|N| +2487249|AAAAAAAABNDPFCAA|2097-10-02|2373|10319|792|2097|3|10|2|4|2097|792|10319|Wednesday|2097Q4|N|N|N|2487248|2487520|2486884|2487157|N|N|N|N|N| +2487250|AAAAAAAACNDPFCAA|2097-10-03|2373|10319|792|2097|4|10|3|4|2097|792|10319|Thursday|2097Q4|N|N|N|2487248|2487520|2486885|2487158|N|N|N|N|N| +2487251|AAAAAAAADNDPFCAA|2097-10-04|2373|10319|792|2097|5|10|4|4|2097|792|10319|Friday|2097Q4|N|Y|N|2487248|2487520|2486886|2487159|N|N|N|N|N| +2487252|AAAAAAAAENDPFCAA|2097-10-05|2373|10319|792|2097|6|10|5|4|2097|792|10319|Saturday|2097Q4|N|Y|N|2487248|2487520|2486887|2487160|N|N|N|N|N| +2487253|AAAAAAAAFNDPFCAA|2097-10-06|2373|10319|792|2097|0|10|6|4|2097|792|10319|Sunday|2097Q4|N|N|N|2487248|2487520|2486888|2487161|N|N|N|N|N| +2487254|AAAAAAAAGNDPFCAA|2097-10-07|2373|10319|792|2097|1|10|7|4|2097|792|10319|Monday|2097Q4|N|N|N|2487248|2487520|2486889|2487162|N|N|N|N|N| +2487255|AAAAAAAAHNDPFCAA|2097-10-08|2373|10320|792|2097|2|10|8|4|2097|792|10320|Tuesday|2097Q4|N|N|N|2487248|2487520|2486890|2487163|N|N|N|N|N| +2487256|AAAAAAAAINDPFCAA|2097-10-09|2373|10320|792|2097|3|10|9|4|2097|792|10320|Wednesday|2097Q4|N|N|N|2487248|2487520|2486891|2487164|N|N|N|N|N| +2487257|AAAAAAAAJNDPFCAA|2097-10-10|2373|10320|792|2097|4|10|10|4|2097|792|10320|Thursday|2097Q4|N|N|N|2487248|2487520|2486892|2487165|N|N|N|N|N| +2487258|AAAAAAAAKNDPFCAA|2097-10-11|2373|10320|792|2097|5|10|11|4|2097|792|10320|Friday|2097Q4|N|Y|N|2487248|2487520|2486893|2487166|N|N|N|N|N| +2487259|AAAAAAAALNDPFCAA|2097-10-12|2373|10320|792|2097|6|10|12|4|2097|792|10320|Saturday|2097Q4|N|Y|N|2487248|2487520|2486894|2487167|N|N|N|N|N| +2487260|AAAAAAAAMNDPFCAA|2097-10-13|2373|10320|792|2097|0|10|13|4|2097|792|10320|Sunday|2097Q4|N|N|N|2487248|2487520|2486895|2487168|N|N|N|N|N| +2487261|AAAAAAAANNDPFCAA|2097-10-14|2373|10320|792|2097|1|10|14|4|2097|792|10320|Monday|2097Q4|N|N|N|2487248|2487520|2486896|2487169|N|N|N|N|N| +2487262|AAAAAAAAONDPFCAA|2097-10-15|2373|10321|792|2097|2|10|15|4|2097|792|10321|Tuesday|2097Q4|N|N|N|2487248|2487520|2486897|2487170|N|N|N|N|N| +2487263|AAAAAAAAPNDPFCAA|2097-10-16|2373|10321|792|2097|3|10|16|4|2097|792|10321|Wednesday|2097Q4|N|N|N|2487248|2487520|2486898|2487171|N|N|N|N|N| +2487264|AAAAAAAAAODPFCAA|2097-10-17|2373|10321|792|2097|4|10|17|4|2097|792|10321|Thursday|2097Q4|N|N|N|2487248|2487520|2486899|2487172|N|N|N|N|N| +2487265|AAAAAAAABODPFCAA|2097-10-18|2373|10321|792|2097|5|10|18|4|2097|792|10321|Friday|2097Q4|N|Y|N|2487248|2487520|2486900|2487173|N|N|N|N|N| +2487266|AAAAAAAACODPFCAA|2097-10-19|2373|10321|792|2097|6|10|19|4|2097|792|10321|Saturday|2097Q4|N|Y|N|2487248|2487520|2486901|2487174|N|N|N|N|N| +2487267|AAAAAAAADODPFCAA|2097-10-20|2373|10321|792|2097|0|10|20|4|2097|792|10321|Sunday|2097Q4|N|N|N|2487248|2487520|2486902|2487175|N|N|N|N|N| +2487268|AAAAAAAAEODPFCAA|2097-10-21|2373|10321|792|2097|1|10|21|4|2097|792|10321|Monday|2097Q4|N|N|N|2487248|2487520|2486903|2487176|N|N|N|N|N| +2487269|AAAAAAAAFODPFCAA|2097-10-22|2373|10322|792|2097|2|10|22|4|2097|792|10322|Tuesday|2097Q4|N|N|N|2487248|2487520|2486904|2487177|N|N|N|N|N| +2487270|AAAAAAAAGODPFCAA|2097-10-23|2373|10322|792|2097|3|10|23|4|2097|792|10322|Wednesday|2097Q4|N|N|N|2487248|2487520|2486905|2487178|N|N|N|N|N| +2487271|AAAAAAAAHODPFCAA|2097-10-24|2373|10322|792|2097|4|10|24|4|2097|792|10322|Thursday|2097Q4|N|N|N|2487248|2487520|2486906|2487179|N|N|N|N|N| +2487272|AAAAAAAAIODPFCAA|2097-10-25|2373|10322|792|2097|5|10|25|4|2097|792|10322|Friday|2097Q4|N|Y|N|2487248|2487520|2486907|2487180|N|N|N|N|N| +2487273|AAAAAAAAJODPFCAA|2097-10-26|2373|10322|792|2097|6|10|26|4|2097|792|10322|Saturday|2097Q4|N|Y|N|2487248|2487520|2486908|2487181|N|N|N|N|N| +2487274|AAAAAAAAKODPFCAA|2097-10-27|2373|10322|792|2097|0|10|27|4|2097|792|10322|Sunday|2097Q4|N|N|N|2487248|2487520|2486909|2487182|N|N|N|N|N| +2487275|AAAAAAAALODPFCAA|2097-10-28|2373|10322|792|2097|1|10|28|4|2097|792|10322|Monday|2097Q4|N|N|N|2487248|2487520|2486910|2487183|N|N|N|N|N| +2487276|AAAAAAAAMODPFCAA|2097-10-29|2373|10323|792|2097|2|10|29|4|2097|792|10323|Tuesday|2097Q4|N|N|N|2487248|2487520|2486911|2487184|N|N|N|N|N| +2487277|AAAAAAAANODPFCAA|2097-10-30|2373|10323|792|2097|3|10|30|4|2097|792|10323|Wednesday|2097Q4|N|N|N|2487248|2487520|2486912|2487185|N|N|N|N|N| +2487278|AAAAAAAAOODPFCAA|2097-10-31|2373|10323|792|2097|4|10|31|4|2097|792|10323|Thursday|2097Q4|N|N|N|2487248|2487520|2486913|2487186|N|N|N|N|N| +2487279|AAAAAAAAPODPFCAA|2097-11-01|2374|10323|792|2097|5|11|1|4|2097|792|10323|Friday|2097Q4|N|Y|N|2487279|2487582|2486914|2487187|N|N|N|N|N| +2487280|AAAAAAAAAPDPFCAA|2097-11-02|2374|10323|792|2097|6|11|2|4|2097|792|10323|Saturday|2097Q4|N|Y|N|2487279|2487582|2486915|2487188|N|N|N|N|N| +2487281|AAAAAAAABPDPFCAA|2097-11-03|2374|10323|792|2097|0|11|3|4|2097|792|10323|Sunday|2097Q4|N|N|N|2487279|2487582|2486916|2487189|N|N|N|N|N| +2487282|AAAAAAAACPDPFCAA|2097-11-04|2374|10323|792|2097|1|11|4|4|2097|792|10323|Monday|2097Q4|N|N|N|2487279|2487582|2486917|2487190|N|N|N|N|N| +2487283|AAAAAAAADPDPFCAA|2097-11-05|2374|10324|792|2097|2|11|5|4|2097|792|10324|Tuesday|2097Q4|N|N|N|2487279|2487582|2486918|2487191|N|N|N|N|N| +2487284|AAAAAAAAEPDPFCAA|2097-11-06|2374|10324|792|2097|3|11|6|4|2097|792|10324|Wednesday|2097Q4|N|N|N|2487279|2487582|2486919|2487192|N|N|N|N|N| +2487285|AAAAAAAAFPDPFCAA|2097-11-07|2374|10324|792|2097|4|11|7|4|2097|792|10324|Thursday|2097Q4|N|N|N|2487279|2487582|2486920|2487193|N|N|N|N|N| +2487286|AAAAAAAAGPDPFCAA|2097-11-08|2374|10324|792|2097|5|11|8|4|2097|792|10324|Friday|2097Q4|N|Y|N|2487279|2487582|2486921|2487194|N|N|N|N|N| +2487287|AAAAAAAAHPDPFCAA|2097-11-09|2374|10324|792|2097|6|11|9|4|2097|792|10324|Saturday|2097Q4|N|Y|N|2487279|2487582|2486922|2487195|N|N|N|N|N| +2487288|AAAAAAAAIPDPFCAA|2097-11-10|2374|10324|792|2097|0|11|10|4|2097|792|10324|Sunday|2097Q4|N|N|N|2487279|2487582|2486923|2487196|N|N|N|N|N| +2487289|AAAAAAAAJPDPFCAA|2097-11-11|2374|10324|792|2097|1|11|11|4|2097|792|10324|Monday|2097Q4|N|N|N|2487279|2487582|2486924|2487197|N|N|N|N|N| +2487290|AAAAAAAAKPDPFCAA|2097-11-12|2374|10325|792|2097|2|11|12|4|2097|792|10325|Tuesday|2097Q4|N|N|N|2487279|2487582|2486925|2487198|N|N|N|N|N| +2487291|AAAAAAAALPDPFCAA|2097-11-13|2374|10325|792|2097|3|11|13|4|2097|792|10325|Wednesday|2097Q4|N|N|N|2487279|2487582|2486926|2487199|N|N|N|N|N| +2487292|AAAAAAAAMPDPFCAA|2097-11-14|2374|10325|792|2097|4|11|14|4|2097|792|10325|Thursday|2097Q4|N|N|N|2487279|2487582|2486927|2487200|N|N|N|N|N| +2487293|AAAAAAAANPDPFCAA|2097-11-15|2374|10325|792|2097|5|11|15|4|2097|792|10325|Friday|2097Q4|N|Y|N|2487279|2487582|2486928|2487201|N|N|N|N|N| +2487294|AAAAAAAAOPDPFCAA|2097-11-16|2374|10325|792|2097|6|11|16|4|2097|792|10325|Saturday|2097Q4|N|Y|N|2487279|2487582|2486929|2487202|N|N|N|N|N| +2487295|AAAAAAAAPPDPFCAA|2097-11-17|2374|10325|792|2097|0|11|17|4|2097|792|10325|Sunday|2097Q4|N|N|N|2487279|2487582|2486930|2487203|N|N|N|N|N| +2487296|AAAAAAAAAAEPFCAA|2097-11-18|2374|10325|792|2097|1|11|18|4|2097|792|10325|Monday|2097Q4|N|N|N|2487279|2487582|2486931|2487204|N|N|N|N|N| +2487297|AAAAAAAABAEPFCAA|2097-11-19|2374|10326|792|2097|2|11|19|4|2097|792|10326|Tuesday|2097Q4|N|N|N|2487279|2487582|2486932|2487205|N|N|N|N|N| +2487298|AAAAAAAACAEPFCAA|2097-11-20|2374|10326|792|2097|3|11|20|4|2097|792|10326|Wednesday|2097Q4|N|N|N|2487279|2487582|2486933|2487206|N|N|N|N|N| +2487299|AAAAAAAADAEPFCAA|2097-11-21|2374|10326|792|2097|4|11|21|4|2097|792|10326|Thursday|2097Q4|N|N|N|2487279|2487582|2486934|2487207|N|N|N|N|N| +2487300|AAAAAAAAEAEPFCAA|2097-11-22|2374|10326|792|2097|5|11|22|4|2097|792|10326|Friday|2097Q4|N|Y|N|2487279|2487582|2486935|2487208|N|N|N|N|N| +2487301|AAAAAAAAFAEPFCAA|2097-11-23|2374|10326|792|2097|6|11|23|4|2097|792|10326|Saturday|2097Q4|N|Y|N|2487279|2487582|2486936|2487209|N|N|N|N|N| +2487302|AAAAAAAAGAEPFCAA|2097-11-24|2374|10326|792|2097|0|11|24|4|2097|792|10326|Sunday|2097Q4|N|N|N|2487279|2487582|2486937|2487210|N|N|N|N|N| +2487303|AAAAAAAAHAEPFCAA|2097-11-25|2374|10326|792|2097|1|11|25|4|2097|792|10326|Monday|2097Q4|N|N|N|2487279|2487582|2486938|2487211|N|N|N|N|N| +2487304|AAAAAAAAIAEPFCAA|2097-11-26|2374|10327|792|2097|2|11|26|4|2097|792|10327|Tuesday|2097Q4|N|N|N|2487279|2487582|2486939|2487212|N|N|N|N|N| +2487305|AAAAAAAAJAEPFCAA|2097-11-27|2374|10327|792|2097|3|11|27|4|2097|792|10327|Wednesday|2097Q4|N|N|N|2487279|2487582|2486940|2487213|N|N|N|N|N| +2487306|AAAAAAAAKAEPFCAA|2097-11-28|2374|10327|792|2097|4|11|28|4|2097|792|10327|Thursday|2097Q4|N|N|N|2487279|2487582|2486941|2487214|N|N|N|N|N| +2487307|AAAAAAAALAEPFCAA|2097-11-29|2374|10327|792|2097|5|11|29|4|2097|792|10327|Friday|2097Q4|N|Y|N|2487279|2487582|2486942|2487215|N|N|N|N|N| +2487308|AAAAAAAAMAEPFCAA|2097-11-30|2374|10327|792|2097|6|11|30|4|2097|792|10327|Saturday|2097Q4|N|Y|N|2487279|2487582|2486943|2487216|N|N|N|N|N| +2487309|AAAAAAAANAEPFCAA|2097-12-01|2375|10327|793|2097|0|12|1|4|2097|793|10327|Sunday|2097Q4|N|N|N|2487309|2487642|2486944|2487217|N|N|N|N|N| +2487310|AAAAAAAAOAEPFCAA|2097-12-02|2375|10327|793|2097|1|12|2|4|2097|793|10327|Monday|2097Q4|N|N|N|2487309|2487642|2486945|2487218|N|N|N|N|N| +2487311|AAAAAAAAPAEPFCAA|2097-12-03|2375|10328|793|2097|2|12|3|4|2097|793|10328|Tuesday|2097Q4|N|N|N|2487309|2487642|2486946|2487219|N|N|N|N|N| +2487312|AAAAAAAAABEPFCAA|2097-12-04|2375|10328|793|2097|3|12|4|4|2097|793|10328|Wednesday|2097Q4|N|N|N|2487309|2487642|2486947|2487220|N|N|N|N|N| +2487313|AAAAAAAABBEPFCAA|2097-12-05|2375|10328|793|2097|4|12|5|4|2097|793|10328|Thursday|2097Q4|N|N|N|2487309|2487642|2486948|2487221|N|N|N|N|N| +2487314|AAAAAAAACBEPFCAA|2097-12-06|2375|10328|793|2097|5|12|6|4|2097|793|10328|Friday|2097Q4|N|Y|N|2487309|2487642|2486949|2487222|N|N|N|N|N| +2487315|AAAAAAAADBEPFCAA|2097-12-07|2375|10328|793|2097|6|12|7|4|2097|793|10328|Saturday|2097Q4|N|Y|N|2487309|2487642|2486950|2487223|N|N|N|N|N| +2487316|AAAAAAAAEBEPFCAA|2097-12-08|2375|10328|793|2097|0|12|8|4|2097|793|10328|Sunday|2097Q4|N|N|N|2487309|2487642|2486951|2487224|N|N|N|N|N| +2487317|AAAAAAAAFBEPFCAA|2097-12-09|2375|10328|793|2097|1|12|9|4|2097|793|10328|Monday|2097Q4|N|N|N|2487309|2487642|2486952|2487225|N|N|N|N|N| +2487318|AAAAAAAAGBEPFCAA|2097-12-10|2375|10329|793|2097|2|12|10|4|2097|793|10329|Tuesday|2097Q4|N|N|N|2487309|2487642|2486953|2487226|N|N|N|N|N| +2487319|AAAAAAAAHBEPFCAA|2097-12-11|2375|10329|793|2097|3|12|11|4|2097|793|10329|Wednesday|2097Q4|N|N|N|2487309|2487642|2486954|2487227|N|N|N|N|N| +2487320|AAAAAAAAIBEPFCAA|2097-12-12|2375|10329|793|2097|4|12|12|4|2097|793|10329|Thursday|2097Q4|N|N|N|2487309|2487642|2486955|2487228|N|N|N|N|N| +2487321|AAAAAAAAJBEPFCAA|2097-12-13|2375|10329|793|2097|5|12|13|4|2097|793|10329|Friday|2097Q4|N|Y|N|2487309|2487642|2486956|2487229|N|N|N|N|N| +2487322|AAAAAAAAKBEPFCAA|2097-12-14|2375|10329|793|2097|6|12|14|4|2097|793|10329|Saturday|2097Q4|N|Y|N|2487309|2487642|2486957|2487230|N|N|N|N|N| +2487323|AAAAAAAALBEPFCAA|2097-12-15|2375|10329|793|2097|0|12|15|4|2097|793|10329|Sunday|2097Q4|N|N|N|2487309|2487642|2486958|2487231|N|N|N|N|N| +2487324|AAAAAAAAMBEPFCAA|2097-12-16|2375|10329|793|2097|1|12|16|4|2097|793|10329|Monday|2097Q4|N|N|N|2487309|2487642|2486959|2487232|N|N|N|N|N| +2487325|AAAAAAAANBEPFCAA|2097-12-17|2375|10330|793|2097|2|12|17|4|2097|793|10330|Tuesday|2097Q4|N|N|N|2487309|2487642|2486960|2487233|N|N|N|N|N| +2487326|AAAAAAAAOBEPFCAA|2097-12-18|2375|10330|793|2097|3|12|18|4|2097|793|10330|Wednesday|2097Q4|N|N|N|2487309|2487642|2486961|2487234|N|N|N|N|N| +2487327|AAAAAAAAPBEPFCAA|2097-12-19|2375|10330|793|2097|4|12|19|4|2097|793|10330|Thursday|2097Q4|N|N|N|2487309|2487642|2486962|2487235|N|N|N|N|N| +2487328|AAAAAAAAACEPFCAA|2097-12-20|2375|10330|793|2097|5|12|20|4|2097|793|10330|Friday|2097Q4|N|Y|N|2487309|2487642|2486963|2487236|N|N|N|N|N| +2487329|AAAAAAAABCEPFCAA|2097-12-21|2375|10330|793|2097|6|12|21|4|2097|793|10330|Saturday|2097Q4|N|Y|N|2487309|2487642|2486964|2487237|N|N|N|N|N| +2487330|AAAAAAAACCEPFCAA|2097-12-22|2375|10330|793|2097|0|12|22|4|2097|793|10330|Sunday|2097Q4|N|N|N|2487309|2487642|2486965|2487238|N|N|N|N|N| +2487331|AAAAAAAADCEPFCAA|2097-12-23|2375|10330|793|2097|1|12|23|4|2097|793|10330|Monday|2097Q4|N|N|N|2487309|2487642|2486966|2487239|N|N|N|N|N| +2487332|AAAAAAAAECEPFCAA|2097-12-24|2375|10331|793|2097|2|12|24|4|2097|793|10331|Tuesday|2097Q4|N|N|N|2487309|2487642|2486967|2487240|N|N|N|N|N| +2487333|AAAAAAAAFCEPFCAA|2097-12-25|2375|10331|793|2097|3|12|25|4|2097|793|10331|Wednesday|2097Q4|N|N|N|2487309|2487642|2486968|2487241|N|N|N|N|N| +2487334|AAAAAAAAGCEPFCAA|2097-12-26|2375|10331|793|2097|4|12|26|4|2097|793|10331|Thursday|2097Q4|Y|N|N|2487309|2487642|2486969|2487242|N|N|N|N|N| +2487335|AAAAAAAAHCEPFCAA|2097-12-27|2375|10331|793|2097|5|12|27|4|2097|793|10331|Friday|2097Q4|N|Y|Y|2487309|2487642|2486970|2487243|N|N|N|N|N| +2487336|AAAAAAAAICEPFCAA|2097-12-28|2375|10331|793|2097|6|12|28|4|2097|793|10331|Saturday|2097Q4|N|Y|N|2487309|2487642|2486971|2487244|N|N|N|N|N| +2487337|AAAAAAAAJCEPFCAA|2097-12-29|2375|10331|793|2097|0|12|29|4|2097|793|10331|Sunday|2097Q4|N|N|N|2487309|2487642|2486972|2487245|N|N|N|N|N| +2487338|AAAAAAAAKCEPFCAA|2097-12-30|2375|10331|793|2097|1|12|30|4|2097|793|10331|Monday|2097Q4|N|N|N|2487309|2487642|2486973|2487246|N|N|N|N|N| +2487339|AAAAAAAALCEPFCAA|2097-12-31|2375|10332|793|2097|2|12|31|4|2097|793|10332|Tuesday|2097Q4|N|N|N|2487309|2487642|2486974|2487247|N|N|N|N|N| +2487340|AAAAAAAAMCEPFCAA|2098-01-01|2376|10332|793|2098|3|1|1|1|2098|793|10332|Wednesday|2098Q1|Y|N|N|2487340|2487339|2486975|2487248|N|N|N|N|N| +2487341|AAAAAAAANCEPFCAA|2098-01-02|2376|10332|793|2098|4|1|2|1|2098|793|10332|Thursday|2098Q1|N|N|Y|2487340|2487339|2486976|2487249|N|N|N|N|N| +2487342|AAAAAAAAOCEPFCAA|2098-01-03|2376|10332|793|2098|5|1|3|1|2098|793|10332|Friday|2098Q1|N|Y|N|2487340|2487339|2486977|2487250|N|N|N|N|N| +2487343|AAAAAAAAPCEPFCAA|2098-01-04|2376|10332|793|2098|6|1|4|1|2098|793|10332|Saturday|2098Q1|N|Y|N|2487340|2487339|2486978|2487251|N|N|N|N|N| +2487344|AAAAAAAAADEPFCAA|2098-01-05|2376|10332|793|2098|0|1|5|1|2098|793|10332|Sunday|2098Q1|N|N|N|2487340|2487339|2486979|2487252|N|N|N|N|N| +2487345|AAAAAAAABDEPFCAA|2098-01-06|2376|10332|793|2098|1|1|6|1|2098|793|10332|Monday|2098Q1|N|N|N|2487340|2487339|2486980|2487253|N|N|N|N|N| +2487346|AAAAAAAACDEPFCAA|2098-01-07|2376|10333|793|2098|2|1|7|1|2098|793|10333|Tuesday|2098Q1|N|N|N|2487340|2487339|2486981|2487254|N|N|N|N|N| +2487347|AAAAAAAADDEPFCAA|2098-01-08|2376|10333|793|2098|3|1|8|1|2098|793|10333|Wednesday|2098Q1|N|N|N|2487340|2487339|2486982|2487255|N|N|N|N|N| +2487348|AAAAAAAAEDEPFCAA|2098-01-09|2376|10333|793|2098|4|1|9|1|2098|793|10333|Thursday|2098Q1|N|N|N|2487340|2487339|2486983|2487256|N|N|N|N|N| +2487349|AAAAAAAAFDEPFCAA|2098-01-10|2376|10333|793|2098|5|1|10|1|2098|793|10333|Friday|2098Q1|N|Y|N|2487340|2487339|2486984|2487257|N|N|N|N|N| +2487350|AAAAAAAAGDEPFCAA|2098-01-11|2376|10333|793|2098|6|1|11|1|2098|793|10333|Saturday|2098Q1|N|Y|N|2487340|2487339|2486985|2487258|N|N|N|N|N| +2487351|AAAAAAAAHDEPFCAA|2098-01-12|2376|10333|793|2098|0|1|12|1|2098|793|10333|Sunday|2098Q1|N|N|N|2487340|2487339|2486986|2487259|N|N|N|N|N| +2487352|AAAAAAAAIDEPFCAA|2098-01-13|2376|10333|793|2098|1|1|13|1|2098|793|10333|Monday|2098Q1|N|N|N|2487340|2487339|2486987|2487260|N|N|N|N|N| +2487353|AAAAAAAAJDEPFCAA|2098-01-14|2376|10334|793|2098|2|1|14|1|2098|793|10334|Tuesday|2098Q1|N|N|N|2487340|2487339|2486988|2487261|N|N|N|N|N| +2487354|AAAAAAAAKDEPFCAA|2098-01-15|2376|10334|793|2098|3|1|15|1|2098|793|10334|Wednesday|2098Q1|N|N|N|2487340|2487339|2486989|2487262|N|N|N|N|N| +2487355|AAAAAAAALDEPFCAA|2098-01-16|2376|10334|793|2098|4|1|16|1|2098|793|10334|Thursday|2098Q1|N|N|N|2487340|2487339|2486990|2487263|N|N|N|N|N| +2487356|AAAAAAAAMDEPFCAA|2098-01-17|2376|10334|793|2098|5|1|17|1|2098|793|10334|Friday|2098Q1|N|Y|N|2487340|2487339|2486991|2487264|N|N|N|N|N| +2487357|AAAAAAAANDEPFCAA|2098-01-18|2376|10334|793|2098|6|1|18|1|2098|793|10334|Saturday|2098Q1|N|Y|N|2487340|2487339|2486992|2487265|N|N|N|N|N| +2487358|AAAAAAAAODEPFCAA|2098-01-19|2376|10334|793|2098|0|1|19|1|2098|793|10334|Sunday|2098Q1|N|N|N|2487340|2487339|2486993|2487266|N|N|N|N|N| +2487359|AAAAAAAAPDEPFCAA|2098-01-20|2376|10334|793|2098|1|1|20|1|2098|793|10334|Monday|2098Q1|N|N|N|2487340|2487339|2486994|2487267|N|N|N|N|N| +2487360|AAAAAAAAAEEPFCAA|2098-01-21|2376|10335|793|2098|2|1|21|1|2098|793|10335|Tuesday|2098Q1|N|N|N|2487340|2487339|2486995|2487268|N|N|N|N|N| +2487361|AAAAAAAABEEPFCAA|2098-01-22|2376|10335|793|2098|3|1|22|1|2098|793|10335|Wednesday|2098Q1|N|N|N|2487340|2487339|2486996|2487269|N|N|N|N|N| +2487362|AAAAAAAACEEPFCAA|2098-01-23|2376|10335|793|2098|4|1|23|1|2098|793|10335|Thursday|2098Q1|N|N|N|2487340|2487339|2486997|2487270|N|N|N|N|N| +2487363|AAAAAAAADEEPFCAA|2098-01-24|2376|10335|793|2098|5|1|24|1|2098|793|10335|Friday|2098Q1|N|Y|N|2487340|2487339|2486998|2487271|N|N|N|N|N| +2487364|AAAAAAAAEEEPFCAA|2098-01-25|2376|10335|793|2098|6|1|25|1|2098|793|10335|Saturday|2098Q1|N|Y|N|2487340|2487339|2486999|2487272|N|N|N|N|N| +2487365|AAAAAAAAFEEPFCAA|2098-01-26|2376|10335|793|2098|0|1|26|1|2098|793|10335|Sunday|2098Q1|N|N|N|2487340|2487339|2487000|2487273|N|N|N|N|N| +2487366|AAAAAAAAGEEPFCAA|2098-01-27|2376|10335|793|2098|1|1|27|1|2098|793|10335|Monday|2098Q1|N|N|N|2487340|2487339|2487001|2487274|N|N|N|N|N| +2487367|AAAAAAAAHEEPFCAA|2098-01-28|2376|10336|793|2098|2|1|28|1|2098|793|10336|Tuesday|2098Q1|N|N|N|2487340|2487339|2487002|2487275|N|N|N|N|N| +2487368|AAAAAAAAIEEPFCAA|2098-01-29|2376|10336|793|2098|3|1|29|1|2098|793|10336|Wednesday|2098Q1|N|N|N|2487340|2487339|2487003|2487276|N|N|N|N|N| +2487369|AAAAAAAAJEEPFCAA|2098-01-30|2376|10336|793|2098|4|1|30|1|2098|793|10336|Thursday|2098Q1|N|N|N|2487340|2487339|2487004|2487277|N|N|N|N|N| +2487370|AAAAAAAAKEEPFCAA|2098-01-31|2376|10336|793|2098|5|1|31|1|2098|793|10336|Friday|2098Q1|N|Y|N|2487340|2487339|2487005|2487278|N|N|N|N|N| +2487371|AAAAAAAALEEPFCAA|2098-02-01|2377|10336|793|2098|6|2|1|1|2098|793|10336|Saturday|2098Q1|N|Y|N|2487371|2487401|2487006|2487279|N|N|N|N|N| +2487372|AAAAAAAAMEEPFCAA|2098-02-02|2377|10336|793|2098|0|2|2|1|2098|793|10336|Sunday|2098Q1|N|N|N|2487371|2487401|2487007|2487280|N|N|N|N|N| +2487373|AAAAAAAANEEPFCAA|2098-02-03|2377|10336|793|2098|1|2|3|1|2098|793|10336|Monday|2098Q1|N|N|N|2487371|2487401|2487008|2487281|N|N|N|N|N| +2487374|AAAAAAAAOEEPFCAA|2098-02-04|2377|10337|793|2098|2|2|4|1|2098|793|10337|Tuesday|2098Q1|N|N|N|2487371|2487401|2487009|2487282|N|N|N|N|N| +2487375|AAAAAAAAPEEPFCAA|2098-02-05|2377|10337|793|2098|3|2|5|1|2098|793|10337|Wednesday|2098Q1|N|N|N|2487371|2487401|2487010|2487283|N|N|N|N|N| +2487376|AAAAAAAAAFEPFCAA|2098-02-06|2377|10337|793|2098|4|2|6|1|2098|793|10337|Thursday|2098Q1|N|N|N|2487371|2487401|2487011|2487284|N|N|N|N|N| +2487377|AAAAAAAABFEPFCAA|2098-02-07|2377|10337|793|2098|5|2|7|1|2098|793|10337|Friday|2098Q1|N|Y|N|2487371|2487401|2487012|2487285|N|N|N|N|N| +2487378|AAAAAAAACFEPFCAA|2098-02-08|2377|10337|793|2098|6|2|8|1|2098|793|10337|Saturday|2098Q1|N|Y|N|2487371|2487401|2487013|2487286|N|N|N|N|N| +2487379|AAAAAAAADFEPFCAA|2098-02-09|2377|10337|793|2098|0|2|9|1|2098|793|10337|Sunday|2098Q1|N|N|N|2487371|2487401|2487014|2487287|N|N|N|N|N| +2487380|AAAAAAAAEFEPFCAA|2098-02-10|2377|10337|793|2098|1|2|10|1|2098|793|10337|Monday|2098Q1|N|N|N|2487371|2487401|2487015|2487288|N|N|N|N|N| +2487381|AAAAAAAAFFEPFCAA|2098-02-11|2377|10338|793|2098|2|2|11|1|2098|793|10338|Tuesday|2098Q1|N|N|N|2487371|2487401|2487016|2487289|N|N|N|N|N| +2487382|AAAAAAAAGFEPFCAA|2098-02-12|2377|10338|793|2098|3|2|12|1|2098|793|10338|Wednesday|2098Q1|N|N|N|2487371|2487401|2487017|2487290|N|N|N|N|N| +2487383|AAAAAAAAHFEPFCAA|2098-02-13|2377|10338|793|2098|4|2|13|1|2098|793|10338|Thursday|2098Q1|N|N|N|2487371|2487401|2487018|2487291|N|N|N|N|N| +2487384|AAAAAAAAIFEPFCAA|2098-02-14|2377|10338|793|2098|5|2|14|1|2098|793|10338|Friday|2098Q1|N|Y|N|2487371|2487401|2487019|2487292|N|N|N|N|N| +2487385|AAAAAAAAJFEPFCAA|2098-02-15|2377|10338|793|2098|6|2|15|1|2098|793|10338|Saturday|2098Q1|N|Y|N|2487371|2487401|2487020|2487293|N|N|N|N|N| +2487386|AAAAAAAAKFEPFCAA|2098-02-16|2377|10338|793|2098|0|2|16|1|2098|793|10338|Sunday|2098Q1|N|N|N|2487371|2487401|2487021|2487294|N|N|N|N|N| +2487387|AAAAAAAALFEPFCAA|2098-02-17|2377|10338|793|2098|1|2|17|1|2098|793|10338|Monday|2098Q1|N|N|N|2487371|2487401|2487022|2487295|N|N|N|N|N| +2487388|AAAAAAAAMFEPFCAA|2098-02-18|2377|10339|793|2098|2|2|18|1|2098|793|10339|Tuesday|2098Q1|N|N|N|2487371|2487401|2487023|2487296|N|N|N|N|N| +2487389|AAAAAAAANFEPFCAA|2098-02-19|2377|10339|793|2098|3|2|19|1|2098|793|10339|Wednesday|2098Q1|N|N|N|2487371|2487401|2487024|2487297|N|N|N|N|N| +2487390|AAAAAAAAOFEPFCAA|2098-02-20|2377|10339|793|2098|4|2|20|1|2098|793|10339|Thursday|2098Q1|N|N|N|2487371|2487401|2487025|2487298|N|N|N|N|N| +2487391|AAAAAAAAPFEPFCAA|2098-02-21|2377|10339|793|2098|5|2|21|1|2098|793|10339|Friday|2098Q1|N|Y|N|2487371|2487401|2487026|2487299|N|N|N|N|N| +2487392|AAAAAAAAAGEPFCAA|2098-02-22|2377|10339|793|2098|6|2|22|1|2098|793|10339|Saturday|2098Q1|N|Y|N|2487371|2487401|2487027|2487300|N|N|N|N|N| +2487393|AAAAAAAABGEPFCAA|2098-02-23|2377|10339|793|2098|0|2|23|1|2098|793|10339|Sunday|2098Q1|N|N|N|2487371|2487401|2487028|2487301|N|N|N|N|N| +2487394|AAAAAAAACGEPFCAA|2098-02-24|2377|10339|793|2098|1|2|24|1|2098|793|10339|Monday|2098Q1|N|N|N|2487371|2487401|2487029|2487302|N|N|N|N|N| +2487395|AAAAAAAADGEPFCAA|2098-02-25|2377|10340|793|2098|2|2|25|1|2098|793|10340|Tuesday|2098Q1|N|N|N|2487371|2487401|2487030|2487303|N|N|N|N|N| +2487396|AAAAAAAAEGEPFCAA|2098-02-26|2377|10340|793|2098|3|2|26|1|2098|793|10340|Wednesday|2098Q1|N|N|N|2487371|2487401|2487031|2487304|N|N|N|N|N| +2487397|AAAAAAAAFGEPFCAA|2098-02-27|2377|10340|793|2098|4|2|27|1|2098|793|10340|Thursday|2098Q1|N|N|N|2487371|2487401|2487032|2487305|N|N|N|N|N| +2487398|AAAAAAAAGGEPFCAA|2098-02-28|2377|10340|793|2098|5|2|28|1|2098|793|10340|Friday|2098Q1|N|Y|N|2487371|2487401|2487033|2487306|N|N|N|N|N| +2487399|AAAAAAAAHGEPFCAA|2098-03-01|2378|10340|794|2098|6|3|1|1|2098|794|10340|Saturday|2098Q1|N|Y|N|2487399|2487457|2487034|2487307|N|N|N|N|N| +2487400|AAAAAAAAIGEPFCAA|2098-03-02|2378|10340|794|2098|0|3|2|1|2098|794|10340|Sunday|2098Q1|N|N|N|2487399|2487457|2487035|2487308|N|N|N|N|N| +2487401|AAAAAAAAJGEPFCAA|2098-03-03|2378|10340|794|2098|1|3|3|1|2098|794|10340|Monday|2098Q1|N|N|N|2487399|2487457|2487036|2487309|N|N|N|N|N| +2487402|AAAAAAAAKGEPFCAA|2098-03-04|2378|10341|794|2098|2|3|4|1|2098|794|10341|Tuesday|2098Q1|N|N|N|2487399|2487457|2487037|2487310|N|N|N|N|N| +2487403|AAAAAAAALGEPFCAA|2098-03-05|2378|10341|794|2098|3|3|5|1|2098|794|10341|Wednesday|2098Q1|N|N|N|2487399|2487457|2487038|2487311|N|N|N|N|N| +2487404|AAAAAAAAMGEPFCAA|2098-03-06|2378|10341|794|2098|4|3|6|1|2098|794|10341|Thursday|2098Q1|N|N|N|2487399|2487457|2487039|2487312|N|N|N|N|N| +2487405|AAAAAAAANGEPFCAA|2098-03-07|2378|10341|794|2098|5|3|7|1|2098|794|10341|Friday|2098Q1|N|Y|N|2487399|2487457|2487040|2487313|N|N|N|N|N| +2487406|AAAAAAAAOGEPFCAA|2098-03-08|2378|10341|794|2098|6|3|8|1|2098|794|10341|Saturday|2098Q1|N|Y|N|2487399|2487457|2487041|2487314|N|N|N|N|N| +2487407|AAAAAAAAPGEPFCAA|2098-03-09|2378|10341|794|2098|0|3|9|1|2098|794|10341|Sunday|2098Q1|N|N|N|2487399|2487457|2487042|2487315|N|N|N|N|N| +2487408|AAAAAAAAAHEPFCAA|2098-03-10|2378|10341|794|2098|1|3|10|1|2098|794|10341|Monday|2098Q1|N|N|N|2487399|2487457|2487043|2487316|N|N|N|N|N| +2487409|AAAAAAAABHEPFCAA|2098-03-11|2378|10342|794|2098|2|3|11|1|2098|794|10342|Tuesday|2098Q1|N|N|N|2487399|2487457|2487044|2487317|N|N|N|N|N| +2487410|AAAAAAAACHEPFCAA|2098-03-12|2378|10342|794|2098|3|3|12|1|2098|794|10342|Wednesday|2098Q1|N|N|N|2487399|2487457|2487045|2487318|N|N|N|N|N| +2487411|AAAAAAAADHEPFCAA|2098-03-13|2378|10342|794|2098|4|3|13|1|2098|794|10342|Thursday|2098Q1|N|N|N|2487399|2487457|2487046|2487319|N|N|N|N|N| +2487412|AAAAAAAAEHEPFCAA|2098-03-14|2378|10342|794|2098|5|3|14|1|2098|794|10342|Friday|2098Q1|N|Y|N|2487399|2487457|2487047|2487320|N|N|N|N|N| +2487413|AAAAAAAAFHEPFCAA|2098-03-15|2378|10342|794|2098|6|3|15|1|2098|794|10342|Saturday|2098Q1|N|Y|N|2487399|2487457|2487048|2487321|N|N|N|N|N| +2487414|AAAAAAAAGHEPFCAA|2098-03-16|2378|10342|794|2098|0|3|16|1|2098|794|10342|Sunday|2098Q1|N|N|N|2487399|2487457|2487049|2487322|N|N|N|N|N| +2487415|AAAAAAAAHHEPFCAA|2098-03-17|2378|10342|794|2098|1|3|17|1|2098|794|10342|Monday|2098Q1|N|N|N|2487399|2487457|2487050|2487323|N|N|N|N|N| +2487416|AAAAAAAAIHEPFCAA|2098-03-18|2378|10343|794|2098|2|3|18|1|2098|794|10343|Tuesday|2098Q1|N|N|N|2487399|2487457|2487051|2487324|N|N|N|N|N| +2487417|AAAAAAAAJHEPFCAA|2098-03-19|2378|10343|794|2098|3|3|19|1|2098|794|10343|Wednesday|2098Q1|N|N|N|2487399|2487457|2487052|2487325|N|N|N|N|N| +2487418|AAAAAAAAKHEPFCAA|2098-03-20|2378|10343|794|2098|4|3|20|1|2098|794|10343|Thursday|2098Q1|N|N|N|2487399|2487457|2487053|2487326|N|N|N|N|N| +2487419|AAAAAAAALHEPFCAA|2098-03-21|2378|10343|794|2098|5|3|21|1|2098|794|10343|Friday|2098Q1|N|Y|N|2487399|2487457|2487054|2487327|N|N|N|N|N| +2487420|AAAAAAAAMHEPFCAA|2098-03-22|2378|10343|794|2098|6|3|22|1|2098|794|10343|Saturday|2098Q1|N|Y|N|2487399|2487457|2487055|2487328|N|N|N|N|N| +2487421|AAAAAAAANHEPFCAA|2098-03-23|2378|10343|794|2098|0|3|23|1|2098|794|10343|Sunday|2098Q1|N|N|N|2487399|2487457|2487056|2487329|N|N|N|N|N| +2487422|AAAAAAAAOHEPFCAA|2098-03-24|2378|10343|794|2098|1|3|24|1|2098|794|10343|Monday|2098Q1|N|N|N|2487399|2487457|2487057|2487330|N|N|N|N|N| +2487423|AAAAAAAAPHEPFCAA|2098-03-25|2378|10344|794|2098|2|3|25|1|2098|794|10344|Tuesday|2098Q1|N|N|N|2487399|2487457|2487058|2487331|N|N|N|N|N| +2487424|AAAAAAAAAIEPFCAA|2098-03-26|2378|10344|794|2098|3|3|26|1|2098|794|10344|Wednesday|2098Q1|N|N|N|2487399|2487457|2487059|2487332|N|N|N|N|N| +2487425|AAAAAAAABIEPFCAA|2098-03-27|2378|10344|794|2098|4|3|27|1|2098|794|10344|Thursday|2098Q1|N|N|N|2487399|2487457|2487060|2487333|N|N|N|N|N| +2487426|AAAAAAAACIEPFCAA|2098-03-28|2378|10344|794|2098|5|3|28|1|2098|794|10344|Friday|2098Q1|N|Y|N|2487399|2487457|2487061|2487334|N|N|N|N|N| +2487427|AAAAAAAADIEPFCAA|2098-03-29|2378|10344|794|2098|6|3|29|1|2098|794|10344|Saturday|2098Q1|N|Y|N|2487399|2487457|2487062|2487335|N|N|N|N|N| +2487428|AAAAAAAAEIEPFCAA|2098-03-30|2378|10344|794|2098|0|3|30|1|2098|794|10344|Sunday|2098Q1|N|N|N|2487399|2487457|2487063|2487336|N|N|N|N|N| +2487429|AAAAAAAAFIEPFCAA|2098-03-31|2378|10344|794|2098|1|3|31|1|2098|794|10344|Monday|2098Q1|N|N|N|2487399|2487457|2487064|2487337|N|N|N|N|N| +2487430|AAAAAAAAGIEPFCAA|2098-04-01|2379|10345|794|2098|2|4|1|1|2098|794|10345|Tuesday|2098Q1|N|N|N|2487430|2487519|2487065|2487340|N|N|N|N|N| +2487431|AAAAAAAAHIEPFCAA|2098-04-02|2379|10345|794|2098|3|4|2|2|2098|794|10345|Wednesday|2098Q2|N|N|N|2487430|2487519|2487066|2487341|N|N|N|N|N| +2487432|AAAAAAAAIIEPFCAA|2098-04-03|2379|10345|794|2098|4|4|3|2|2098|794|10345|Thursday|2098Q2|N|N|N|2487430|2487519|2487067|2487342|N|N|N|N|N| +2487433|AAAAAAAAJIEPFCAA|2098-04-04|2379|10345|794|2098|5|4|4|2|2098|794|10345|Friday|2098Q2|N|Y|N|2487430|2487519|2487068|2487343|N|N|N|N|N| +2487434|AAAAAAAAKIEPFCAA|2098-04-05|2379|10345|794|2098|6|4|5|2|2098|794|10345|Saturday|2098Q2|N|Y|N|2487430|2487519|2487069|2487344|N|N|N|N|N| +2487435|AAAAAAAALIEPFCAA|2098-04-06|2379|10345|794|2098|0|4|6|2|2098|794|10345|Sunday|2098Q2|N|N|N|2487430|2487519|2487070|2487345|N|N|N|N|N| +2487436|AAAAAAAAMIEPFCAA|2098-04-07|2379|10345|794|2098|1|4|7|2|2098|794|10345|Monday|2098Q2|N|N|N|2487430|2487519|2487071|2487346|N|N|N|N|N| +2487437|AAAAAAAANIEPFCAA|2098-04-08|2379|10346|794|2098|2|4|8|2|2098|794|10346|Tuesday|2098Q2|N|N|N|2487430|2487519|2487072|2487347|N|N|N|N|N| +2487438|AAAAAAAAOIEPFCAA|2098-04-09|2379|10346|794|2098|3|4|9|2|2098|794|10346|Wednesday|2098Q2|N|N|N|2487430|2487519|2487073|2487348|N|N|N|N|N| +2487439|AAAAAAAAPIEPFCAA|2098-04-10|2379|10346|794|2098|4|4|10|2|2098|794|10346|Thursday|2098Q2|N|N|N|2487430|2487519|2487074|2487349|N|N|N|N|N| +2487440|AAAAAAAAAJEPFCAA|2098-04-11|2379|10346|794|2098|5|4|11|2|2098|794|10346|Friday|2098Q2|N|Y|N|2487430|2487519|2487075|2487350|N|N|N|N|N| +2487441|AAAAAAAABJEPFCAA|2098-04-12|2379|10346|794|2098|6|4|12|2|2098|794|10346|Saturday|2098Q2|N|Y|N|2487430|2487519|2487076|2487351|N|N|N|N|N| +2487442|AAAAAAAACJEPFCAA|2098-04-13|2379|10346|794|2098|0|4|13|2|2098|794|10346|Sunday|2098Q2|N|N|N|2487430|2487519|2487077|2487352|N|N|N|N|N| +2487443|AAAAAAAADJEPFCAA|2098-04-14|2379|10346|794|2098|1|4|14|2|2098|794|10346|Monday|2098Q2|N|N|N|2487430|2487519|2487078|2487353|N|N|N|N|N| +2487444|AAAAAAAAEJEPFCAA|2098-04-15|2379|10347|794|2098|2|4|15|2|2098|794|10347|Tuesday|2098Q2|N|N|N|2487430|2487519|2487079|2487354|N|N|N|N|N| +2487445|AAAAAAAAFJEPFCAA|2098-04-16|2379|10347|794|2098|3|4|16|2|2098|794|10347|Wednesday|2098Q2|N|N|N|2487430|2487519|2487080|2487355|N|N|N|N|N| +2487446|AAAAAAAAGJEPFCAA|2098-04-17|2379|10347|794|2098|4|4|17|2|2098|794|10347|Thursday|2098Q2|N|N|N|2487430|2487519|2487081|2487356|N|N|N|N|N| +2487447|AAAAAAAAHJEPFCAA|2098-04-18|2379|10347|794|2098|5|4|18|2|2098|794|10347|Friday|2098Q2|N|Y|N|2487430|2487519|2487082|2487357|N|N|N|N|N| +2487448|AAAAAAAAIJEPFCAA|2098-04-19|2379|10347|794|2098|6|4|19|2|2098|794|10347|Saturday|2098Q2|N|Y|N|2487430|2487519|2487083|2487358|N|N|N|N|N| +2487449|AAAAAAAAJJEPFCAA|2098-04-20|2379|10347|794|2098|0|4|20|2|2098|794|10347|Sunday|2098Q2|N|N|N|2487430|2487519|2487084|2487359|N|N|N|N|N| +2487450|AAAAAAAAKJEPFCAA|2098-04-21|2379|10347|794|2098|1|4|21|2|2098|794|10347|Monday|2098Q2|N|N|N|2487430|2487519|2487085|2487360|N|N|N|N|N| +2487451|AAAAAAAALJEPFCAA|2098-04-22|2379|10348|794|2098|2|4|22|2|2098|794|10348|Tuesday|2098Q2|N|N|N|2487430|2487519|2487086|2487361|N|N|N|N|N| +2487452|AAAAAAAAMJEPFCAA|2098-04-23|2379|10348|794|2098|3|4|23|2|2098|794|10348|Wednesday|2098Q2|N|N|N|2487430|2487519|2487087|2487362|N|N|N|N|N| +2487453|AAAAAAAANJEPFCAA|2098-04-24|2379|10348|794|2098|4|4|24|2|2098|794|10348|Thursday|2098Q2|N|N|N|2487430|2487519|2487088|2487363|N|N|N|N|N| +2487454|AAAAAAAAOJEPFCAA|2098-04-25|2379|10348|794|2098|5|4|25|2|2098|794|10348|Friday|2098Q2|N|Y|N|2487430|2487519|2487089|2487364|N|N|N|N|N| +2487455|AAAAAAAAPJEPFCAA|2098-04-26|2379|10348|794|2098|6|4|26|2|2098|794|10348|Saturday|2098Q2|N|Y|N|2487430|2487519|2487090|2487365|N|N|N|N|N| +2487456|AAAAAAAAAKEPFCAA|2098-04-27|2379|10348|794|2098|0|4|27|2|2098|794|10348|Sunday|2098Q2|N|N|N|2487430|2487519|2487091|2487366|N|N|N|N|N| +2487457|AAAAAAAABKEPFCAA|2098-04-28|2379|10348|794|2098|1|4|28|2|2098|794|10348|Monday|2098Q2|N|N|N|2487430|2487519|2487092|2487367|N|N|N|N|N| +2487458|AAAAAAAACKEPFCAA|2098-04-29|2379|10349|794|2098|2|4|29|2|2098|794|10349|Tuesday|2098Q2|N|N|N|2487430|2487519|2487093|2487368|N|N|N|N|N| +2487459|AAAAAAAADKEPFCAA|2098-04-30|2379|10349|794|2098|3|4|30|2|2098|794|10349|Wednesday|2098Q2|N|N|N|2487430|2487519|2487094|2487369|N|N|N|N|N| +2487460|AAAAAAAAEKEPFCAA|2098-05-01|2380|10349|794|2098|4|5|1|2|2098|794|10349|Thursday|2098Q2|N|N|N|2487460|2487579|2487095|2487370|N|N|N|N|N| +2487461|AAAAAAAAFKEPFCAA|2098-05-02|2380|10349|794|2098|5|5|2|2|2098|794|10349|Friday|2098Q2|N|Y|N|2487460|2487579|2487096|2487371|N|N|N|N|N| +2487462|AAAAAAAAGKEPFCAA|2098-05-03|2380|10349|794|2098|6|5|3|2|2098|794|10349|Saturday|2098Q2|N|Y|N|2487460|2487579|2487097|2487372|N|N|N|N|N| +2487463|AAAAAAAAHKEPFCAA|2098-05-04|2380|10349|794|2098|0|5|4|2|2098|794|10349|Sunday|2098Q2|N|N|N|2487460|2487579|2487098|2487373|N|N|N|N|N| +2487464|AAAAAAAAIKEPFCAA|2098-05-05|2380|10349|794|2098|1|5|5|2|2098|794|10349|Monday|2098Q2|N|N|N|2487460|2487579|2487099|2487374|N|N|N|N|N| +2487465|AAAAAAAAJKEPFCAA|2098-05-06|2380|10350|794|2098|2|5|6|2|2098|794|10350|Tuesday|2098Q2|N|N|N|2487460|2487579|2487100|2487375|N|N|N|N|N| +2487466|AAAAAAAAKKEPFCAA|2098-05-07|2380|10350|794|2098|3|5|7|2|2098|794|10350|Wednesday|2098Q2|N|N|N|2487460|2487579|2487101|2487376|N|N|N|N|N| +2487467|AAAAAAAALKEPFCAA|2098-05-08|2380|10350|794|2098|4|5|8|2|2098|794|10350|Thursday|2098Q2|N|N|N|2487460|2487579|2487102|2487377|N|N|N|N|N| +2487468|AAAAAAAAMKEPFCAA|2098-05-09|2380|10350|794|2098|5|5|9|2|2098|794|10350|Friday|2098Q2|N|Y|N|2487460|2487579|2487103|2487378|N|N|N|N|N| +2487469|AAAAAAAANKEPFCAA|2098-05-10|2380|10350|794|2098|6|5|10|2|2098|794|10350|Saturday|2098Q2|N|Y|N|2487460|2487579|2487104|2487379|N|N|N|N|N| +2487470|AAAAAAAAOKEPFCAA|2098-05-11|2380|10350|794|2098|0|5|11|2|2098|794|10350|Sunday|2098Q2|N|N|N|2487460|2487579|2487105|2487380|N|N|N|N|N| +2487471|AAAAAAAAPKEPFCAA|2098-05-12|2380|10350|794|2098|1|5|12|2|2098|794|10350|Monday|2098Q2|N|N|N|2487460|2487579|2487106|2487381|N|N|N|N|N| +2487472|AAAAAAAAALEPFCAA|2098-05-13|2380|10351|794|2098|2|5|13|2|2098|794|10351|Tuesday|2098Q2|N|N|N|2487460|2487579|2487107|2487382|N|N|N|N|N| +2487473|AAAAAAAABLEPFCAA|2098-05-14|2380|10351|794|2098|3|5|14|2|2098|794|10351|Wednesday|2098Q2|N|N|N|2487460|2487579|2487108|2487383|N|N|N|N|N| +2487474|AAAAAAAACLEPFCAA|2098-05-15|2380|10351|794|2098|4|5|15|2|2098|794|10351|Thursday|2098Q2|N|N|N|2487460|2487579|2487109|2487384|N|N|N|N|N| +2487475|AAAAAAAADLEPFCAA|2098-05-16|2380|10351|794|2098|5|5|16|2|2098|794|10351|Friday|2098Q2|N|Y|N|2487460|2487579|2487110|2487385|N|N|N|N|N| +2487476|AAAAAAAAELEPFCAA|2098-05-17|2380|10351|794|2098|6|5|17|2|2098|794|10351|Saturday|2098Q2|N|Y|N|2487460|2487579|2487111|2487386|N|N|N|N|N| +2487477|AAAAAAAAFLEPFCAA|2098-05-18|2380|10351|794|2098|0|5|18|2|2098|794|10351|Sunday|2098Q2|N|N|N|2487460|2487579|2487112|2487387|N|N|N|N|N| +2487478|AAAAAAAAGLEPFCAA|2098-05-19|2380|10351|794|2098|1|5|19|2|2098|794|10351|Monday|2098Q2|N|N|N|2487460|2487579|2487113|2487388|N|N|N|N|N| +2487479|AAAAAAAAHLEPFCAA|2098-05-20|2380|10352|794|2098|2|5|20|2|2098|794|10352|Tuesday|2098Q2|N|N|N|2487460|2487579|2487114|2487389|N|N|N|N|N| +2487480|AAAAAAAAILEPFCAA|2098-05-21|2380|10352|794|2098|3|5|21|2|2098|794|10352|Wednesday|2098Q2|N|N|N|2487460|2487579|2487115|2487390|N|N|N|N|N| +2487481|AAAAAAAAJLEPFCAA|2098-05-22|2380|10352|794|2098|4|5|22|2|2098|794|10352|Thursday|2098Q2|N|N|N|2487460|2487579|2487116|2487391|N|N|N|N|N| +2487482|AAAAAAAAKLEPFCAA|2098-05-23|2380|10352|794|2098|5|5|23|2|2098|794|10352|Friday|2098Q2|N|Y|N|2487460|2487579|2487117|2487392|N|N|N|N|N| +2487483|AAAAAAAALLEPFCAA|2098-05-24|2380|10352|794|2098|6|5|24|2|2098|794|10352|Saturday|2098Q2|N|Y|N|2487460|2487579|2487118|2487393|N|N|N|N|N| +2487484|AAAAAAAAMLEPFCAA|2098-05-25|2380|10352|794|2098|0|5|25|2|2098|794|10352|Sunday|2098Q2|N|N|N|2487460|2487579|2487119|2487394|N|N|N|N|N| +2487485|AAAAAAAANLEPFCAA|2098-05-26|2380|10352|794|2098|1|5|26|2|2098|794|10352|Monday|2098Q2|N|N|N|2487460|2487579|2487120|2487395|N|N|N|N|N| +2487486|AAAAAAAAOLEPFCAA|2098-05-27|2380|10353|794|2098|2|5|27|2|2098|794|10353|Tuesday|2098Q2|N|N|N|2487460|2487579|2487121|2487396|N|N|N|N|N| +2487487|AAAAAAAAPLEPFCAA|2098-05-28|2380|10353|794|2098|3|5|28|2|2098|794|10353|Wednesday|2098Q2|N|N|N|2487460|2487579|2487122|2487397|N|N|N|N|N| +2487488|AAAAAAAAAMEPFCAA|2098-05-29|2380|10353|794|2098|4|5|29|2|2098|794|10353|Thursday|2098Q2|N|N|N|2487460|2487579|2487123|2487398|N|N|N|N|N| +2487489|AAAAAAAABMEPFCAA|2098-05-30|2380|10353|794|2098|5|5|30|2|2098|794|10353|Friday|2098Q2|N|Y|N|2487460|2487579|2487124|2487399|N|N|N|N|N| +2487490|AAAAAAAACMEPFCAA|2098-05-31|2380|10353|794|2098|6|5|31|2|2098|794|10353|Saturday|2098Q2|N|Y|N|2487460|2487579|2487125|2487400|N|N|N|N|N| +2487491|AAAAAAAADMEPFCAA|2098-06-01|2381|10353|795|2098|0|6|1|2|2098|795|10353|Sunday|2098Q2|N|N|N|2487491|2487641|2487126|2487401|N|N|N|N|N| +2487492|AAAAAAAAEMEPFCAA|2098-06-02|2381|10353|795|2098|1|6|2|2|2098|795|10353|Monday|2098Q2|N|N|N|2487491|2487641|2487127|2487402|N|N|N|N|N| +2487493|AAAAAAAAFMEPFCAA|2098-06-03|2381|10354|795|2098|2|6|3|2|2098|795|10354|Tuesday|2098Q2|N|N|N|2487491|2487641|2487128|2487403|N|N|N|N|N| +2487494|AAAAAAAAGMEPFCAA|2098-06-04|2381|10354|795|2098|3|6|4|2|2098|795|10354|Wednesday|2098Q2|N|N|N|2487491|2487641|2487129|2487404|N|N|N|N|N| +2487495|AAAAAAAAHMEPFCAA|2098-06-05|2381|10354|795|2098|4|6|5|2|2098|795|10354|Thursday|2098Q2|N|N|N|2487491|2487641|2487130|2487405|N|N|N|N|N| +2487496|AAAAAAAAIMEPFCAA|2098-06-06|2381|10354|795|2098|5|6|6|2|2098|795|10354|Friday|2098Q2|N|Y|N|2487491|2487641|2487131|2487406|N|N|N|N|N| +2487497|AAAAAAAAJMEPFCAA|2098-06-07|2381|10354|795|2098|6|6|7|2|2098|795|10354|Saturday|2098Q2|N|Y|N|2487491|2487641|2487132|2487407|N|N|N|N|N| +2487498|AAAAAAAAKMEPFCAA|2098-06-08|2381|10354|795|2098|0|6|8|2|2098|795|10354|Sunday|2098Q2|N|N|N|2487491|2487641|2487133|2487408|N|N|N|N|N| +2487499|AAAAAAAALMEPFCAA|2098-06-09|2381|10354|795|2098|1|6|9|2|2098|795|10354|Monday|2098Q2|N|N|N|2487491|2487641|2487134|2487409|N|N|N|N|N| +2487500|AAAAAAAAMMEPFCAA|2098-06-10|2381|10355|795|2098|2|6|10|2|2098|795|10355|Tuesday|2098Q2|N|N|N|2487491|2487641|2487135|2487410|N|N|N|N|N| +2487501|AAAAAAAANMEPFCAA|2098-06-11|2381|10355|795|2098|3|6|11|2|2098|795|10355|Wednesday|2098Q2|N|N|N|2487491|2487641|2487136|2487411|N|N|N|N|N| +2487502|AAAAAAAAOMEPFCAA|2098-06-12|2381|10355|795|2098|4|6|12|2|2098|795|10355|Thursday|2098Q2|N|N|N|2487491|2487641|2487137|2487412|N|N|N|N|N| +2487503|AAAAAAAAPMEPFCAA|2098-06-13|2381|10355|795|2098|5|6|13|2|2098|795|10355|Friday|2098Q2|N|Y|N|2487491|2487641|2487138|2487413|N|N|N|N|N| +2487504|AAAAAAAAANEPFCAA|2098-06-14|2381|10355|795|2098|6|6|14|2|2098|795|10355|Saturday|2098Q2|N|Y|N|2487491|2487641|2487139|2487414|N|N|N|N|N| +2487505|AAAAAAAABNEPFCAA|2098-06-15|2381|10355|795|2098|0|6|15|2|2098|795|10355|Sunday|2098Q2|N|N|N|2487491|2487641|2487140|2487415|N|N|N|N|N| +2487506|AAAAAAAACNEPFCAA|2098-06-16|2381|10355|795|2098|1|6|16|2|2098|795|10355|Monday|2098Q2|N|N|N|2487491|2487641|2487141|2487416|N|N|N|N|N| +2487507|AAAAAAAADNEPFCAA|2098-06-17|2381|10356|795|2098|2|6|17|2|2098|795|10356|Tuesday|2098Q2|N|N|N|2487491|2487641|2487142|2487417|N|N|N|N|N| +2487508|AAAAAAAAENEPFCAA|2098-06-18|2381|10356|795|2098|3|6|18|2|2098|795|10356|Wednesday|2098Q2|N|N|N|2487491|2487641|2487143|2487418|N|N|N|N|N| +2487509|AAAAAAAAFNEPFCAA|2098-06-19|2381|10356|795|2098|4|6|19|2|2098|795|10356|Thursday|2098Q2|N|N|N|2487491|2487641|2487144|2487419|N|N|N|N|N| +2487510|AAAAAAAAGNEPFCAA|2098-06-20|2381|10356|795|2098|5|6|20|2|2098|795|10356|Friday|2098Q2|N|Y|N|2487491|2487641|2487145|2487420|N|N|N|N|N| +2487511|AAAAAAAAHNEPFCAA|2098-06-21|2381|10356|795|2098|6|6|21|2|2098|795|10356|Saturday|2098Q2|N|Y|N|2487491|2487641|2487146|2487421|N|N|N|N|N| +2487512|AAAAAAAAINEPFCAA|2098-06-22|2381|10356|795|2098|0|6|22|2|2098|795|10356|Sunday|2098Q2|N|N|N|2487491|2487641|2487147|2487422|N|N|N|N|N| +2487513|AAAAAAAAJNEPFCAA|2098-06-23|2381|10356|795|2098|1|6|23|2|2098|795|10356|Monday|2098Q2|N|N|N|2487491|2487641|2487148|2487423|N|N|N|N|N| +2487514|AAAAAAAAKNEPFCAA|2098-06-24|2381|10357|795|2098|2|6|24|2|2098|795|10357|Tuesday|2098Q2|N|N|N|2487491|2487641|2487149|2487424|N|N|N|N|N| +2487515|AAAAAAAALNEPFCAA|2098-06-25|2381|10357|795|2098|3|6|25|2|2098|795|10357|Wednesday|2098Q2|N|N|N|2487491|2487641|2487150|2487425|N|N|N|N|N| +2487516|AAAAAAAAMNEPFCAA|2098-06-26|2381|10357|795|2098|4|6|26|2|2098|795|10357|Thursday|2098Q2|N|N|N|2487491|2487641|2487151|2487426|N|N|N|N|N| +2487517|AAAAAAAANNEPFCAA|2098-06-27|2381|10357|795|2098|5|6|27|2|2098|795|10357|Friday|2098Q2|N|Y|N|2487491|2487641|2487152|2487427|N|N|N|N|N| +2487518|AAAAAAAAONEPFCAA|2098-06-28|2381|10357|795|2098|6|6|28|2|2098|795|10357|Saturday|2098Q2|N|Y|N|2487491|2487641|2487153|2487428|N|N|N|N|N| +2487519|AAAAAAAAPNEPFCAA|2098-06-29|2381|10357|795|2098|0|6|29|2|2098|795|10357|Sunday|2098Q2|N|N|N|2487491|2487641|2487154|2487429|N|N|N|N|N| +2487520|AAAAAAAAAOEPFCAA|2098-06-30|2381|10357|795|2098|1|6|30|2|2098|795|10357|Monday|2098Q2|N|N|N|2487491|2487641|2487155|2487430|N|N|N|N|N| +2487521|AAAAAAAABOEPFCAA|2098-07-01|2382|10358|795|2098|2|7|1|2|2098|795|10358|Tuesday|2098Q2|N|N|N|2487521|2487701|2487156|2487430|N|N|N|N|N| +2487522|AAAAAAAACOEPFCAA|2098-07-02|2382|10358|795|2098|3|7|2|3|2098|795|10358|Wednesday|2098Q3|N|N|N|2487521|2487701|2487157|2487431|N|N|N|N|N| +2487523|AAAAAAAADOEPFCAA|2098-07-03|2382|10358|795|2098|4|7|3|3|2098|795|10358|Thursday|2098Q3|N|N|N|2487521|2487701|2487158|2487432|N|N|N|N|N| +2487524|AAAAAAAAEOEPFCAA|2098-07-04|2382|10358|795|2098|5|7|4|3|2098|795|10358|Friday|2098Q3|N|Y|N|2487521|2487701|2487159|2487433|N|N|N|N|N| +2487525|AAAAAAAAFOEPFCAA|2098-07-05|2382|10358|795|2098|6|7|5|3|2098|795|10358|Saturday|2098Q3|Y|Y|N|2487521|2487701|2487160|2487434|N|N|N|N|N| +2487526|AAAAAAAAGOEPFCAA|2098-07-06|2382|10358|795|2098|0|7|6|3|2098|795|10358|Sunday|2098Q3|N|N|Y|2487521|2487701|2487161|2487435|N|N|N|N|N| +2487527|AAAAAAAAHOEPFCAA|2098-07-07|2382|10358|795|2098|1|7|7|3|2098|795|10358|Monday|2098Q3|N|N|N|2487521|2487701|2487162|2487436|N|N|N|N|N| +2487528|AAAAAAAAIOEPFCAA|2098-07-08|2382|10359|795|2098|2|7|8|3|2098|795|10359|Tuesday|2098Q3|N|N|N|2487521|2487701|2487163|2487437|N|N|N|N|N| +2487529|AAAAAAAAJOEPFCAA|2098-07-09|2382|10359|795|2098|3|7|9|3|2098|795|10359|Wednesday|2098Q3|N|N|N|2487521|2487701|2487164|2487438|N|N|N|N|N| +2487530|AAAAAAAAKOEPFCAA|2098-07-10|2382|10359|795|2098|4|7|10|3|2098|795|10359|Thursday|2098Q3|N|N|N|2487521|2487701|2487165|2487439|N|N|N|N|N| +2487531|AAAAAAAALOEPFCAA|2098-07-11|2382|10359|795|2098|5|7|11|3|2098|795|10359|Friday|2098Q3|N|Y|N|2487521|2487701|2487166|2487440|N|N|N|N|N| +2487532|AAAAAAAAMOEPFCAA|2098-07-12|2382|10359|795|2098|6|7|12|3|2098|795|10359|Saturday|2098Q3|N|Y|N|2487521|2487701|2487167|2487441|N|N|N|N|N| +2487533|AAAAAAAANOEPFCAA|2098-07-13|2382|10359|795|2098|0|7|13|3|2098|795|10359|Sunday|2098Q3|N|N|N|2487521|2487701|2487168|2487442|N|N|N|N|N| +2487534|AAAAAAAAOOEPFCAA|2098-07-14|2382|10359|795|2098|1|7|14|3|2098|795|10359|Monday|2098Q3|N|N|N|2487521|2487701|2487169|2487443|N|N|N|N|N| +2487535|AAAAAAAAPOEPFCAA|2098-07-15|2382|10360|795|2098|2|7|15|3|2098|795|10360|Tuesday|2098Q3|N|N|N|2487521|2487701|2487170|2487444|N|N|N|N|N| +2487536|AAAAAAAAAPEPFCAA|2098-07-16|2382|10360|795|2098|3|7|16|3|2098|795|10360|Wednesday|2098Q3|N|N|N|2487521|2487701|2487171|2487445|N|N|N|N|N| +2487537|AAAAAAAABPEPFCAA|2098-07-17|2382|10360|795|2098|4|7|17|3|2098|795|10360|Thursday|2098Q3|N|N|N|2487521|2487701|2487172|2487446|N|N|N|N|N| +2487538|AAAAAAAACPEPFCAA|2098-07-18|2382|10360|795|2098|5|7|18|3|2098|795|10360|Friday|2098Q3|N|Y|N|2487521|2487701|2487173|2487447|N|N|N|N|N| +2487539|AAAAAAAADPEPFCAA|2098-07-19|2382|10360|795|2098|6|7|19|3|2098|795|10360|Saturday|2098Q3|N|Y|N|2487521|2487701|2487174|2487448|N|N|N|N|N| +2487540|AAAAAAAAEPEPFCAA|2098-07-20|2382|10360|795|2098|0|7|20|3|2098|795|10360|Sunday|2098Q3|N|N|N|2487521|2487701|2487175|2487449|N|N|N|N|N| +2487541|AAAAAAAAFPEPFCAA|2098-07-21|2382|10360|795|2098|1|7|21|3|2098|795|10360|Monday|2098Q3|N|N|N|2487521|2487701|2487176|2487450|N|N|N|N|N| +2487542|AAAAAAAAGPEPFCAA|2098-07-22|2382|10361|795|2098|2|7|22|3|2098|795|10361|Tuesday|2098Q3|N|N|N|2487521|2487701|2487177|2487451|N|N|N|N|N| +2487543|AAAAAAAAHPEPFCAA|2098-07-23|2382|10361|795|2098|3|7|23|3|2098|795|10361|Wednesday|2098Q3|N|N|N|2487521|2487701|2487178|2487452|N|N|N|N|N| +2487544|AAAAAAAAIPEPFCAA|2098-07-24|2382|10361|795|2098|4|7|24|3|2098|795|10361|Thursday|2098Q3|N|N|N|2487521|2487701|2487179|2487453|N|N|N|N|N| +2487545|AAAAAAAAJPEPFCAA|2098-07-25|2382|10361|795|2098|5|7|25|3|2098|795|10361|Friday|2098Q3|N|Y|N|2487521|2487701|2487180|2487454|N|N|N|N|N| +2487546|AAAAAAAAKPEPFCAA|2098-07-26|2382|10361|795|2098|6|7|26|3|2098|795|10361|Saturday|2098Q3|N|Y|N|2487521|2487701|2487181|2487455|N|N|N|N|N| +2487547|AAAAAAAALPEPFCAA|2098-07-27|2382|10361|795|2098|0|7|27|3|2098|795|10361|Sunday|2098Q3|N|N|N|2487521|2487701|2487182|2487456|N|N|N|N|N| +2487548|AAAAAAAAMPEPFCAA|2098-07-28|2382|10361|795|2098|1|7|28|3|2098|795|10361|Monday|2098Q3|N|N|N|2487521|2487701|2487183|2487457|N|N|N|N|N| +2487549|AAAAAAAANPEPFCAA|2098-07-29|2382|10362|795|2098|2|7|29|3|2098|795|10362|Tuesday|2098Q3|N|N|N|2487521|2487701|2487184|2487458|N|N|N|N|N| +2487550|AAAAAAAAOPEPFCAA|2098-07-30|2382|10362|795|2098|3|7|30|3|2098|795|10362|Wednesday|2098Q3|N|N|N|2487521|2487701|2487185|2487459|N|N|N|N|N| +2487551|AAAAAAAAPPEPFCAA|2098-07-31|2382|10362|795|2098|4|7|31|3|2098|795|10362|Thursday|2098Q3|N|N|N|2487521|2487701|2487186|2487460|N|N|N|N|N| +2487552|AAAAAAAAAAFPFCAA|2098-08-01|2383|10362|795|2098|5|8|1|3|2098|795|10362|Friday|2098Q3|N|Y|N|2487552|2487763|2487187|2487461|N|N|N|N|N| +2487553|AAAAAAAABAFPFCAA|2098-08-02|2383|10362|795|2098|6|8|2|3|2098|795|10362|Saturday|2098Q3|N|Y|N|2487552|2487763|2487188|2487462|N|N|N|N|N| +2487554|AAAAAAAACAFPFCAA|2098-08-03|2383|10362|795|2098|0|8|3|3|2098|795|10362|Sunday|2098Q3|N|N|N|2487552|2487763|2487189|2487463|N|N|N|N|N| +2487555|AAAAAAAADAFPFCAA|2098-08-04|2383|10362|795|2098|1|8|4|3|2098|795|10362|Monday|2098Q3|N|N|N|2487552|2487763|2487190|2487464|N|N|N|N|N| +2487556|AAAAAAAAEAFPFCAA|2098-08-05|2383|10363|795|2098|2|8|5|3|2098|795|10363|Tuesday|2098Q3|N|N|N|2487552|2487763|2487191|2487465|N|N|N|N|N| +2487557|AAAAAAAAFAFPFCAA|2098-08-06|2383|10363|795|2098|3|8|6|3|2098|795|10363|Wednesday|2098Q3|N|N|N|2487552|2487763|2487192|2487466|N|N|N|N|N| +2487558|AAAAAAAAGAFPFCAA|2098-08-07|2383|10363|795|2098|4|8|7|3|2098|795|10363|Thursday|2098Q3|N|N|N|2487552|2487763|2487193|2487467|N|N|N|N|N| +2487559|AAAAAAAAHAFPFCAA|2098-08-08|2383|10363|795|2098|5|8|8|3|2098|795|10363|Friday|2098Q3|N|Y|N|2487552|2487763|2487194|2487468|N|N|N|N|N| +2487560|AAAAAAAAIAFPFCAA|2098-08-09|2383|10363|795|2098|6|8|9|3|2098|795|10363|Saturday|2098Q3|N|Y|N|2487552|2487763|2487195|2487469|N|N|N|N|N| +2487561|AAAAAAAAJAFPFCAA|2098-08-10|2383|10363|795|2098|0|8|10|3|2098|795|10363|Sunday|2098Q3|N|N|N|2487552|2487763|2487196|2487470|N|N|N|N|N| +2487562|AAAAAAAAKAFPFCAA|2098-08-11|2383|10363|795|2098|1|8|11|3|2098|795|10363|Monday|2098Q3|N|N|N|2487552|2487763|2487197|2487471|N|N|N|N|N| +2487563|AAAAAAAALAFPFCAA|2098-08-12|2383|10364|795|2098|2|8|12|3|2098|795|10364|Tuesday|2098Q3|N|N|N|2487552|2487763|2487198|2487472|N|N|N|N|N| +2487564|AAAAAAAAMAFPFCAA|2098-08-13|2383|10364|795|2098|3|8|13|3|2098|795|10364|Wednesday|2098Q3|N|N|N|2487552|2487763|2487199|2487473|N|N|N|N|N| +2487565|AAAAAAAANAFPFCAA|2098-08-14|2383|10364|795|2098|4|8|14|3|2098|795|10364|Thursday|2098Q3|N|N|N|2487552|2487763|2487200|2487474|N|N|N|N|N| +2487566|AAAAAAAAOAFPFCAA|2098-08-15|2383|10364|795|2098|5|8|15|3|2098|795|10364|Friday|2098Q3|N|Y|N|2487552|2487763|2487201|2487475|N|N|N|N|N| +2487567|AAAAAAAAPAFPFCAA|2098-08-16|2383|10364|795|2098|6|8|16|3|2098|795|10364|Saturday|2098Q3|N|Y|N|2487552|2487763|2487202|2487476|N|N|N|N|N| +2487568|AAAAAAAAABFPFCAA|2098-08-17|2383|10364|795|2098|0|8|17|3|2098|795|10364|Sunday|2098Q3|N|N|N|2487552|2487763|2487203|2487477|N|N|N|N|N| +2487569|AAAAAAAABBFPFCAA|2098-08-18|2383|10364|795|2098|1|8|18|3|2098|795|10364|Monday|2098Q3|N|N|N|2487552|2487763|2487204|2487478|N|N|N|N|N| +2487570|AAAAAAAACBFPFCAA|2098-08-19|2383|10365|795|2098|2|8|19|3|2098|795|10365|Tuesday|2098Q3|N|N|N|2487552|2487763|2487205|2487479|N|N|N|N|N| +2487571|AAAAAAAADBFPFCAA|2098-08-20|2383|10365|795|2098|3|8|20|3|2098|795|10365|Wednesday|2098Q3|N|N|N|2487552|2487763|2487206|2487480|N|N|N|N|N| +2487572|AAAAAAAAEBFPFCAA|2098-08-21|2383|10365|795|2098|4|8|21|3|2098|795|10365|Thursday|2098Q3|N|N|N|2487552|2487763|2487207|2487481|N|N|N|N|N| +2487573|AAAAAAAAFBFPFCAA|2098-08-22|2383|10365|795|2098|5|8|22|3|2098|795|10365|Friday|2098Q3|N|Y|N|2487552|2487763|2487208|2487482|N|N|N|N|N| +2487574|AAAAAAAAGBFPFCAA|2098-08-23|2383|10365|795|2098|6|8|23|3|2098|795|10365|Saturday|2098Q3|N|Y|N|2487552|2487763|2487209|2487483|N|N|N|N|N| +2487575|AAAAAAAAHBFPFCAA|2098-08-24|2383|10365|795|2098|0|8|24|3|2098|795|10365|Sunday|2098Q3|N|N|N|2487552|2487763|2487210|2487484|N|N|N|N|N| +2487576|AAAAAAAAIBFPFCAA|2098-08-25|2383|10365|795|2098|1|8|25|3|2098|795|10365|Monday|2098Q3|N|N|N|2487552|2487763|2487211|2487485|N|N|N|N|N| +2487577|AAAAAAAAJBFPFCAA|2098-08-26|2383|10366|795|2098|2|8|26|3|2098|795|10366|Tuesday|2098Q3|N|N|N|2487552|2487763|2487212|2487486|N|N|N|N|N| +2487578|AAAAAAAAKBFPFCAA|2098-08-27|2383|10366|795|2098|3|8|27|3|2098|795|10366|Wednesday|2098Q3|N|N|N|2487552|2487763|2487213|2487487|N|N|N|N|N| +2487579|AAAAAAAALBFPFCAA|2098-08-28|2383|10366|795|2098|4|8|28|3|2098|795|10366|Thursday|2098Q3|N|N|N|2487552|2487763|2487214|2487488|N|N|N|N|N| +2487580|AAAAAAAAMBFPFCAA|2098-08-29|2383|10366|795|2098|5|8|29|3|2098|795|10366|Friday|2098Q3|N|Y|N|2487552|2487763|2487215|2487489|N|N|N|N|N| +2487581|AAAAAAAANBFPFCAA|2098-08-30|2383|10366|795|2098|6|8|30|3|2098|795|10366|Saturday|2098Q3|N|Y|N|2487552|2487763|2487216|2487490|N|N|N|N|N| +2487582|AAAAAAAAOBFPFCAA|2098-08-31|2383|10366|795|2098|0|8|31|3|2098|795|10366|Sunday|2098Q3|N|N|N|2487552|2487763|2487217|2487491|N|N|N|N|N| +2487583|AAAAAAAAPBFPFCAA|2098-09-01|2384|10366|796|2098|1|9|1|3|2098|796|10366|Monday|2098Q3|N|N|N|2487583|2487825|2487218|2487492|N|N|N|N|N| +2487584|AAAAAAAAACFPFCAA|2098-09-02|2384|10367|796|2098|2|9|2|3|2098|796|10367|Tuesday|2098Q3|N|N|N|2487583|2487825|2487219|2487493|N|N|N|N|N| +2487585|AAAAAAAABCFPFCAA|2098-09-03|2384|10367|796|2098|3|9|3|3|2098|796|10367|Wednesday|2098Q3|N|N|N|2487583|2487825|2487220|2487494|N|N|N|N|N| +2487586|AAAAAAAACCFPFCAA|2098-09-04|2384|10367|796|2098|4|9|4|3|2098|796|10367|Thursday|2098Q3|N|N|N|2487583|2487825|2487221|2487495|N|N|N|N|N| +2487587|AAAAAAAADCFPFCAA|2098-09-05|2384|10367|796|2098|5|9|5|3|2098|796|10367|Friday|2098Q3|N|Y|N|2487583|2487825|2487222|2487496|N|N|N|N|N| +2487588|AAAAAAAAECFPFCAA|2098-09-06|2384|10367|796|2098|6|9|6|3|2098|796|10367|Saturday|2098Q3|N|Y|N|2487583|2487825|2487223|2487497|N|N|N|N|N| +2487589|AAAAAAAAFCFPFCAA|2098-09-07|2384|10367|796|2098|0|9|7|3|2098|796|10367|Sunday|2098Q3|N|N|N|2487583|2487825|2487224|2487498|N|N|N|N|N| +2487590|AAAAAAAAGCFPFCAA|2098-09-08|2384|10367|796|2098|1|9|8|3|2098|796|10367|Monday|2098Q3|N|N|N|2487583|2487825|2487225|2487499|N|N|N|N|N| +2487591|AAAAAAAAHCFPFCAA|2098-09-09|2384|10368|796|2098|2|9|9|3|2098|796|10368|Tuesday|2098Q3|N|N|N|2487583|2487825|2487226|2487500|N|N|N|N|N| +2487592|AAAAAAAAICFPFCAA|2098-09-10|2384|10368|796|2098|3|9|10|3|2098|796|10368|Wednesday|2098Q3|N|N|N|2487583|2487825|2487227|2487501|N|N|N|N|N| +2487593|AAAAAAAAJCFPFCAA|2098-09-11|2384|10368|796|2098|4|9|11|3|2098|796|10368|Thursday|2098Q3|N|N|N|2487583|2487825|2487228|2487502|N|N|N|N|N| +2487594|AAAAAAAAKCFPFCAA|2098-09-12|2384|10368|796|2098|5|9|12|3|2098|796|10368|Friday|2098Q3|N|Y|N|2487583|2487825|2487229|2487503|N|N|N|N|N| +2487595|AAAAAAAALCFPFCAA|2098-09-13|2384|10368|796|2098|6|9|13|3|2098|796|10368|Saturday|2098Q3|N|Y|N|2487583|2487825|2487230|2487504|N|N|N|N|N| +2487596|AAAAAAAAMCFPFCAA|2098-09-14|2384|10368|796|2098|0|9|14|3|2098|796|10368|Sunday|2098Q3|N|N|N|2487583|2487825|2487231|2487505|N|N|N|N|N| +2487597|AAAAAAAANCFPFCAA|2098-09-15|2384|10368|796|2098|1|9|15|3|2098|796|10368|Monday|2098Q3|N|N|N|2487583|2487825|2487232|2487506|N|N|N|N|N| +2487598|AAAAAAAAOCFPFCAA|2098-09-16|2384|10369|796|2098|2|9|16|3|2098|796|10369|Tuesday|2098Q3|N|N|N|2487583|2487825|2487233|2487507|N|N|N|N|N| +2487599|AAAAAAAAPCFPFCAA|2098-09-17|2384|10369|796|2098|3|9|17|3|2098|796|10369|Wednesday|2098Q3|N|N|N|2487583|2487825|2487234|2487508|N|N|N|N|N| +2487600|AAAAAAAAADFPFCAA|2098-09-18|2384|10369|796|2098|4|9|18|3|2098|796|10369|Thursday|2098Q3|N|N|N|2487583|2487825|2487235|2487509|N|N|N|N|N| +2487601|AAAAAAAABDFPFCAA|2098-09-19|2384|10369|796|2098|5|9|19|3|2098|796|10369|Friday|2098Q3|N|Y|N|2487583|2487825|2487236|2487510|N|N|N|N|N| +2487602|AAAAAAAACDFPFCAA|2098-09-20|2384|10369|796|2098|6|9|20|3|2098|796|10369|Saturday|2098Q3|N|Y|N|2487583|2487825|2487237|2487511|N|N|N|N|N| +2487603|AAAAAAAADDFPFCAA|2098-09-21|2384|10369|796|2098|0|9|21|3|2098|796|10369|Sunday|2098Q3|N|N|N|2487583|2487825|2487238|2487512|N|N|N|N|N| +2487604|AAAAAAAAEDFPFCAA|2098-09-22|2384|10369|796|2098|1|9|22|3|2098|796|10369|Monday|2098Q3|N|N|N|2487583|2487825|2487239|2487513|N|N|N|N|N| +2487605|AAAAAAAAFDFPFCAA|2098-09-23|2384|10370|796|2098|2|9|23|3|2098|796|10370|Tuesday|2098Q3|N|N|N|2487583|2487825|2487240|2487514|N|N|N|N|N| +2487606|AAAAAAAAGDFPFCAA|2098-09-24|2384|10370|796|2098|3|9|24|3|2098|796|10370|Wednesday|2098Q3|N|N|N|2487583|2487825|2487241|2487515|N|N|N|N|N| +2487607|AAAAAAAAHDFPFCAA|2098-09-25|2384|10370|796|2098|4|9|25|3|2098|796|10370|Thursday|2098Q3|N|N|N|2487583|2487825|2487242|2487516|N|N|N|N|N| +2487608|AAAAAAAAIDFPFCAA|2098-09-26|2384|10370|796|2098|5|9|26|3|2098|796|10370|Friday|2098Q3|N|Y|N|2487583|2487825|2487243|2487517|N|N|N|N|N| +2487609|AAAAAAAAJDFPFCAA|2098-09-27|2384|10370|796|2098|6|9|27|3|2098|796|10370|Saturday|2098Q3|N|Y|N|2487583|2487825|2487244|2487518|N|N|N|N|N| +2487610|AAAAAAAAKDFPFCAA|2098-09-28|2384|10370|796|2098|0|9|28|3|2098|796|10370|Sunday|2098Q3|N|N|N|2487583|2487825|2487245|2487519|N|N|N|N|N| +2487611|AAAAAAAALDFPFCAA|2098-09-29|2384|10370|796|2098|1|9|29|3|2098|796|10370|Monday|2098Q3|N|N|N|2487583|2487825|2487246|2487520|N|N|N|N|N| +2487612|AAAAAAAAMDFPFCAA|2098-09-30|2384|10371|796|2098|2|9|30|3|2098|796|10371|Tuesday|2098Q3|N|N|N|2487583|2487825|2487247|2487521|N|N|N|N|N| +2487613|AAAAAAAANDFPFCAA|2098-10-01|2385|10371|796|2098|3|10|1|3|2098|796|10371|Wednesday|2098Q3|N|N|N|2487613|2487885|2487248|2487521|N|N|N|N|N| +2487614|AAAAAAAAODFPFCAA|2098-10-02|2385|10371|796|2098|4|10|2|4|2098|796|10371|Thursday|2098Q4|N|N|N|2487613|2487885|2487249|2487522|N|N|N|N|N| +2487615|AAAAAAAAPDFPFCAA|2098-10-03|2385|10371|796|2098|5|10|3|4|2098|796|10371|Friday|2098Q4|N|Y|N|2487613|2487885|2487250|2487523|N|N|N|N|N| +2487616|AAAAAAAAAEFPFCAA|2098-10-04|2385|10371|796|2098|6|10|4|4|2098|796|10371|Saturday|2098Q4|N|Y|N|2487613|2487885|2487251|2487524|N|N|N|N|N| +2487617|AAAAAAAABEFPFCAA|2098-10-05|2385|10371|796|2098|0|10|5|4|2098|796|10371|Sunday|2098Q4|N|N|N|2487613|2487885|2487252|2487525|N|N|N|N|N| +2487618|AAAAAAAACEFPFCAA|2098-10-06|2385|10371|796|2098|1|10|6|4|2098|796|10371|Monday|2098Q4|N|N|N|2487613|2487885|2487253|2487526|N|N|N|N|N| +2487619|AAAAAAAADEFPFCAA|2098-10-07|2385|10372|796|2098|2|10|7|4|2098|796|10372|Tuesday|2098Q4|N|N|N|2487613|2487885|2487254|2487527|N|N|N|N|N| +2487620|AAAAAAAAEEFPFCAA|2098-10-08|2385|10372|796|2098|3|10|8|4|2098|796|10372|Wednesday|2098Q4|N|N|N|2487613|2487885|2487255|2487528|N|N|N|N|N| +2487621|AAAAAAAAFEFPFCAA|2098-10-09|2385|10372|796|2098|4|10|9|4|2098|796|10372|Thursday|2098Q4|N|N|N|2487613|2487885|2487256|2487529|N|N|N|N|N| +2487622|AAAAAAAAGEFPFCAA|2098-10-10|2385|10372|796|2098|5|10|10|4|2098|796|10372|Friday|2098Q4|N|Y|N|2487613|2487885|2487257|2487530|N|N|N|N|N| +2487623|AAAAAAAAHEFPFCAA|2098-10-11|2385|10372|796|2098|6|10|11|4|2098|796|10372|Saturday|2098Q4|N|Y|N|2487613|2487885|2487258|2487531|N|N|N|N|N| +2487624|AAAAAAAAIEFPFCAA|2098-10-12|2385|10372|796|2098|0|10|12|4|2098|796|10372|Sunday|2098Q4|N|N|N|2487613|2487885|2487259|2487532|N|N|N|N|N| +2487625|AAAAAAAAJEFPFCAA|2098-10-13|2385|10372|796|2098|1|10|13|4|2098|796|10372|Monday|2098Q4|N|N|N|2487613|2487885|2487260|2487533|N|N|N|N|N| +2487626|AAAAAAAAKEFPFCAA|2098-10-14|2385|10373|796|2098|2|10|14|4|2098|796|10373|Tuesday|2098Q4|N|N|N|2487613|2487885|2487261|2487534|N|N|N|N|N| +2487627|AAAAAAAALEFPFCAA|2098-10-15|2385|10373|796|2098|3|10|15|4|2098|796|10373|Wednesday|2098Q4|N|N|N|2487613|2487885|2487262|2487535|N|N|N|N|N| +2487628|AAAAAAAAMEFPFCAA|2098-10-16|2385|10373|796|2098|4|10|16|4|2098|796|10373|Thursday|2098Q4|N|N|N|2487613|2487885|2487263|2487536|N|N|N|N|N| +2487629|AAAAAAAANEFPFCAA|2098-10-17|2385|10373|796|2098|5|10|17|4|2098|796|10373|Friday|2098Q4|N|Y|N|2487613|2487885|2487264|2487537|N|N|N|N|N| +2487630|AAAAAAAAOEFPFCAA|2098-10-18|2385|10373|796|2098|6|10|18|4|2098|796|10373|Saturday|2098Q4|N|Y|N|2487613|2487885|2487265|2487538|N|N|N|N|N| +2487631|AAAAAAAAPEFPFCAA|2098-10-19|2385|10373|796|2098|0|10|19|4|2098|796|10373|Sunday|2098Q4|N|N|N|2487613|2487885|2487266|2487539|N|N|N|N|N| +2487632|AAAAAAAAAFFPFCAA|2098-10-20|2385|10373|796|2098|1|10|20|4|2098|796|10373|Monday|2098Q4|N|N|N|2487613|2487885|2487267|2487540|N|N|N|N|N| +2487633|AAAAAAAABFFPFCAA|2098-10-21|2385|10374|796|2098|2|10|21|4|2098|796|10374|Tuesday|2098Q4|N|N|N|2487613|2487885|2487268|2487541|N|N|N|N|N| +2487634|AAAAAAAACFFPFCAA|2098-10-22|2385|10374|796|2098|3|10|22|4|2098|796|10374|Wednesday|2098Q4|N|N|N|2487613|2487885|2487269|2487542|N|N|N|N|N| +2487635|AAAAAAAADFFPFCAA|2098-10-23|2385|10374|796|2098|4|10|23|4|2098|796|10374|Thursday|2098Q4|N|N|N|2487613|2487885|2487270|2487543|N|N|N|N|N| +2487636|AAAAAAAAEFFPFCAA|2098-10-24|2385|10374|796|2098|5|10|24|4|2098|796|10374|Friday|2098Q4|N|Y|N|2487613|2487885|2487271|2487544|N|N|N|N|N| +2487637|AAAAAAAAFFFPFCAA|2098-10-25|2385|10374|796|2098|6|10|25|4|2098|796|10374|Saturday|2098Q4|N|Y|N|2487613|2487885|2487272|2487545|N|N|N|N|N| +2487638|AAAAAAAAGFFPFCAA|2098-10-26|2385|10374|796|2098|0|10|26|4|2098|796|10374|Sunday|2098Q4|N|N|N|2487613|2487885|2487273|2487546|N|N|N|N|N| +2487639|AAAAAAAAHFFPFCAA|2098-10-27|2385|10374|796|2098|1|10|27|4|2098|796|10374|Monday|2098Q4|N|N|N|2487613|2487885|2487274|2487547|N|N|N|N|N| +2487640|AAAAAAAAIFFPFCAA|2098-10-28|2385|10375|796|2098|2|10|28|4|2098|796|10375|Tuesday|2098Q4|N|N|N|2487613|2487885|2487275|2487548|N|N|N|N|N| +2487641|AAAAAAAAJFFPFCAA|2098-10-29|2385|10375|796|2098|3|10|29|4|2098|796|10375|Wednesday|2098Q4|N|N|N|2487613|2487885|2487276|2487549|N|N|N|N|N| +2487642|AAAAAAAAKFFPFCAA|2098-10-30|2385|10375|796|2098|4|10|30|4|2098|796|10375|Thursday|2098Q4|N|N|N|2487613|2487885|2487277|2487550|N|N|N|N|N| +2487643|AAAAAAAALFFPFCAA|2098-10-31|2385|10375|796|2098|5|10|31|4|2098|796|10375|Friday|2098Q4|N|Y|N|2487613|2487885|2487278|2487551|N|N|N|N|N| +2487644|AAAAAAAAMFFPFCAA|2098-11-01|2386|10375|796|2098|6|11|1|4|2098|796|10375|Saturday|2098Q4|N|Y|N|2487644|2487947|2487279|2487552|N|N|N|N|N| +2487645|AAAAAAAANFFPFCAA|2098-11-02|2386|10375|796|2098|0|11|2|4|2098|796|10375|Sunday|2098Q4|N|N|N|2487644|2487947|2487280|2487553|N|N|N|N|N| +2487646|AAAAAAAAOFFPFCAA|2098-11-03|2386|10375|796|2098|1|11|3|4|2098|796|10375|Monday|2098Q4|N|N|N|2487644|2487947|2487281|2487554|N|N|N|N|N| +2487647|AAAAAAAAPFFPFCAA|2098-11-04|2386|10376|796|2098|2|11|4|4|2098|796|10376|Tuesday|2098Q4|N|N|N|2487644|2487947|2487282|2487555|N|N|N|N|N| +2487648|AAAAAAAAAGFPFCAA|2098-11-05|2386|10376|796|2098|3|11|5|4|2098|796|10376|Wednesday|2098Q4|N|N|N|2487644|2487947|2487283|2487556|N|N|N|N|N| +2487649|AAAAAAAABGFPFCAA|2098-11-06|2386|10376|796|2098|4|11|6|4|2098|796|10376|Thursday|2098Q4|N|N|N|2487644|2487947|2487284|2487557|N|N|N|N|N| +2487650|AAAAAAAACGFPFCAA|2098-11-07|2386|10376|796|2098|5|11|7|4|2098|796|10376|Friday|2098Q4|N|Y|N|2487644|2487947|2487285|2487558|N|N|N|N|N| +2487651|AAAAAAAADGFPFCAA|2098-11-08|2386|10376|796|2098|6|11|8|4|2098|796|10376|Saturday|2098Q4|N|Y|N|2487644|2487947|2487286|2487559|N|N|N|N|N| +2487652|AAAAAAAAEGFPFCAA|2098-11-09|2386|10376|796|2098|0|11|9|4|2098|796|10376|Sunday|2098Q4|N|N|N|2487644|2487947|2487287|2487560|N|N|N|N|N| +2487653|AAAAAAAAFGFPFCAA|2098-11-10|2386|10376|796|2098|1|11|10|4|2098|796|10376|Monday|2098Q4|N|N|N|2487644|2487947|2487288|2487561|N|N|N|N|N| +2487654|AAAAAAAAGGFPFCAA|2098-11-11|2386|10377|796|2098|2|11|11|4|2098|796|10377|Tuesday|2098Q4|N|N|N|2487644|2487947|2487289|2487562|N|N|N|N|N| +2487655|AAAAAAAAHGFPFCAA|2098-11-12|2386|10377|796|2098|3|11|12|4|2098|796|10377|Wednesday|2098Q4|N|N|N|2487644|2487947|2487290|2487563|N|N|N|N|N| +2487656|AAAAAAAAIGFPFCAA|2098-11-13|2386|10377|796|2098|4|11|13|4|2098|796|10377|Thursday|2098Q4|N|N|N|2487644|2487947|2487291|2487564|N|N|N|N|N| +2487657|AAAAAAAAJGFPFCAA|2098-11-14|2386|10377|796|2098|5|11|14|4|2098|796|10377|Friday|2098Q4|N|Y|N|2487644|2487947|2487292|2487565|N|N|N|N|N| +2487658|AAAAAAAAKGFPFCAA|2098-11-15|2386|10377|796|2098|6|11|15|4|2098|796|10377|Saturday|2098Q4|N|Y|N|2487644|2487947|2487293|2487566|N|N|N|N|N| +2487659|AAAAAAAALGFPFCAA|2098-11-16|2386|10377|796|2098|0|11|16|4|2098|796|10377|Sunday|2098Q4|N|N|N|2487644|2487947|2487294|2487567|N|N|N|N|N| +2487660|AAAAAAAAMGFPFCAA|2098-11-17|2386|10377|796|2098|1|11|17|4|2098|796|10377|Monday|2098Q4|N|N|N|2487644|2487947|2487295|2487568|N|N|N|N|N| +2487661|AAAAAAAANGFPFCAA|2098-11-18|2386|10378|796|2098|2|11|18|4|2098|796|10378|Tuesday|2098Q4|N|N|N|2487644|2487947|2487296|2487569|N|N|N|N|N| +2487662|AAAAAAAAOGFPFCAA|2098-11-19|2386|10378|796|2098|3|11|19|4|2098|796|10378|Wednesday|2098Q4|N|N|N|2487644|2487947|2487297|2487570|N|N|N|N|N| +2487663|AAAAAAAAPGFPFCAA|2098-11-20|2386|10378|796|2098|4|11|20|4|2098|796|10378|Thursday|2098Q4|N|N|N|2487644|2487947|2487298|2487571|N|N|N|N|N| +2487664|AAAAAAAAAHFPFCAA|2098-11-21|2386|10378|796|2098|5|11|21|4|2098|796|10378|Friday|2098Q4|N|Y|N|2487644|2487947|2487299|2487572|N|N|N|N|N| +2487665|AAAAAAAABHFPFCAA|2098-11-22|2386|10378|796|2098|6|11|22|4|2098|796|10378|Saturday|2098Q4|N|Y|N|2487644|2487947|2487300|2487573|N|N|N|N|N| +2487666|AAAAAAAACHFPFCAA|2098-11-23|2386|10378|796|2098|0|11|23|4|2098|796|10378|Sunday|2098Q4|N|N|N|2487644|2487947|2487301|2487574|N|N|N|N|N| +2487667|AAAAAAAADHFPFCAA|2098-11-24|2386|10378|796|2098|1|11|24|4|2098|796|10378|Monday|2098Q4|N|N|N|2487644|2487947|2487302|2487575|N|N|N|N|N| +2487668|AAAAAAAAEHFPFCAA|2098-11-25|2386|10379|796|2098|2|11|25|4|2098|796|10379|Tuesday|2098Q4|N|N|N|2487644|2487947|2487303|2487576|N|N|N|N|N| +2487669|AAAAAAAAFHFPFCAA|2098-11-26|2386|10379|796|2098|3|11|26|4|2098|796|10379|Wednesday|2098Q4|N|N|N|2487644|2487947|2487304|2487577|N|N|N|N|N| +2487670|AAAAAAAAGHFPFCAA|2098-11-27|2386|10379|796|2098|4|11|27|4|2098|796|10379|Thursday|2098Q4|N|N|N|2487644|2487947|2487305|2487578|N|N|N|N|N| +2487671|AAAAAAAAHHFPFCAA|2098-11-28|2386|10379|796|2098|5|11|28|4|2098|796|10379|Friday|2098Q4|N|Y|N|2487644|2487947|2487306|2487579|N|N|N|N|N| +2487672|AAAAAAAAIHFPFCAA|2098-11-29|2386|10379|796|2098|6|11|29|4|2098|796|10379|Saturday|2098Q4|N|Y|N|2487644|2487947|2487307|2487580|N|N|N|N|N| +2487673|AAAAAAAAJHFPFCAA|2098-11-30|2386|10379|796|2098|0|11|30|4|2098|796|10379|Sunday|2098Q4|N|N|N|2487644|2487947|2487308|2487581|N|N|N|N|N| +2487674|AAAAAAAAKHFPFCAA|2098-12-01|2387|10379|797|2098|1|12|1|4|2098|797|10379|Monday|2098Q4|N|N|N|2487674|2488007|2487309|2487582|N|N|N|N|N| +2487675|AAAAAAAALHFPFCAA|2098-12-02|2387|10380|797|2098|2|12|2|4|2098|797|10380|Tuesday|2098Q4|N|N|N|2487674|2488007|2487310|2487583|N|N|N|N|N| +2487676|AAAAAAAAMHFPFCAA|2098-12-03|2387|10380|797|2098|3|12|3|4|2098|797|10380|Wednesday|2098Q4|N|N|N|2487674|2488007|2487311|2487584|N|N|N|N|N| +2487677|AAAAAAAANHFPFCAA|2098-12-04|2387|10380|797|2098|4|12|4|4|2098|797|10380|Thursday|2098Q4|N|N|N|2487674|2488007|2487312|2487585|N|N|N|N|N| +2487678|AAAAAAAAOHFPFCAA|2098-12-05|2387|10380|797|2098|5|12|5|4|2098|797|10380|Friday|2098Q4|N|Y|N|2487674|2488007|2487313|2487586|N|N|N|N|N| +2487679|AAAAAAAAPHFPFCAA|2098-12-06|2387|10380|797|2098|6|12|6|4|2098|797|10380|Saturday|2098Q4|N|Y|N|2487674|2488007|2487314|2487587|N|N|N|N|N| +2487680|AAAAAAAAAIFPFCAA|2098-12-07|2387|10380|797|2098|0|12|7|4|2098|797|10380|Sunday|2098Q4|N|N|N|2487674|2488007|2487315|2487588|N|N|N|N|N| +2487681|AAAAAAAABIFPFCAA|2098-12-08|2387|10380|797|2098|1|12|8|4|2098|797|10380|Monday|2098Q4|N|N|N|2487674|2488007|2487316|2487589|N|N|N|N|N| +2487682|AAAAAAAACIFPFCAA|2098-12-09|2387|10381|797|2098|2|12|9|4|2098|797|10381|Tuesday|2098Q4|N|N|N|2487674|2488007|2487317|2487590|N|N|N|N|N| +2487683|AAAAAAAADIFPFCAA|2098-12-10|2387|10381|797|2098|3|12|10|4|2098|797|10381|Wednesday|2098Q4|N|N|N|2487674|2488007|2487318|2487591|N|N|N|N|N| +2487684|AAAAAAAAEIFPFCAA|2098-12-11|2387|10381|797|2098|4|12|11|4|2098|797|10381|Thursday|2098Q4|N|N|N|2487674|2488007|2487319|2487592|N|N|N|N|N| +2487685|AAAAAAAAFIFPFCAA|2098-12-12|2387|10381|797|2098|5|12|12|4|2098|797|10381|Friday|2098Q4|N|Y|N|2487674|2488007|2487320|2487593|N|N|N|N|N| +2487686|AAAAAAAAGIFPFCAA|2098-12-13|2387|10381|797|2098|6|12|13|4|2098|797|10381|Saturday|2098Q4|N|Y|N|2487674|2488007|2487321|2487594|N|N|N|N|N| +2487687|AAAAAAAAHIFPFCAA|2098-12-14|2387|10381|797|2098|0|12|14|4|2098|797|10381|Sunday|2098Q4|N|N|N|2487674|2488007|2487322|2487595|N|N|N|N|N| +2487688|AAAAAAAAIIFPFCAA|2098-12-15|2387|10381|797|2098|1|12|15|4|2098|797|10381|Monday|2098Q4|N|N|N|2487674|2488007|2487323|2487596|N|N|N|N|N| +2487689|AAAAAAAAJIFPFCAA|2098-12-16|2387|10382|797|2098|2|12|16|4|2098|797|10382|Tuesday|2098Q4|N|N|N|2487674|2488007|2487324|2487597|N|N|N|N|N| +2487690|AAAAAAAAKIFPFCAA|2098-12-17|2387|10382|797|2098|3|12|17|4|2098|797|10382|Wednesday|2098Q4|N|N|N|2487674|2488007|2487325|2487598|N|N|N|N|N| +2487691|AAAAAAAALIFPFCAA|2098-12-18|2387|10382|797|2098|4|12|18|4|2098|797|10382|Thursday|2098Q4|N|N|N|2487674|2488007|2487326|2487599|N|N|N|N|N| +2487692|AAAAAAAAMIFPFCAA|2098-12-19|2387|10382|797|2098|5|12|19|4|2098|797|10382|Friday|2098Q4|N|Y|N|2487674|2488007|2487327|2487600|N|N|N|N|N| +2487693|AAAAAAAANIFPFCAA|2098-12-20|2387|10382|797|2098|6|12|20|4|2098|797|10382|Saturday|2098Q4|N|Y|N|2487674|2488007|2487328|2487601|N|N|N|N|N| +2487694|AAAAAAAAOIFPFCAA|2098-12-21|2387|10382|797|2098|0|12|21|4|2098|797|10382|Sunday|2098Q4|N|N|N|2487674|2488007|2487329|2487602|N|N|N|N|N| +2487695|AAAAAAAAPIFPFCAA|2098-12-22|2387|10382|797|2098|1|12|22|4|2098|797|10382|Monday|2098Q4|N|N|N|2487674|2488007|2487330|2487603|N|N|N|N|N| +2487696|AAAAAAAAAJFPFCAA|2098-12-23|2387|10383|797|2098|2|12|23|4|2098|797|10383|Tuesday|2098Q4|N|N|N|2487674|2488007|2487331|2487604|N|N|N|N|N| +2487697|AAAAAAAABJFPFCAA|2098-12-24|2387|10383|797|2098|3|12|24|4|2098|797|10383|Wednesday|2098Q4|N|N|N|2487674|2488007|2487332|2487605|N|N|N|N|N| +2487698|AAAAAAAACJFPFCAA|2098-12-25|2387|10383|797|2098|4|12|25|4|2098|797|10383|Thursday|2098Q4|N|N|N|2487674|2488007|2487333|2487606|N|N|N|N|N| +2487699|AAAAAAAADJFPFCAA|2098-12-26|2387|10383|797|2098|5|12|26|4|2098|797|10383|Friday|2098Q4|Y|Y|N|2487674|2488007|2487334|2487607|N|N|N|N|N| +2487700|AAAAAAAAEJFPFCAA|2098-12-27|2387|10383|797|2098|6|12|27|4|2098|797|10383|Saturday|2098Q4|N|Y|Y|2487674|2488007|2487335|2487608|N|N|N|N|N| +2487701|AAAAAAAAFJFPFCAA|2098-12-28|2387|10383|797|2098|0|12|28|4|2098|797|10383|Sunday|2098Q4|N|N|N|2487674|2488007|2487336|2487609|N|N|N|N|N| +2487702|AAAAAAAAGJFPFCAA|2098-12-29|2387|10383|797|2098|1|12|29|4|2098|797|10383|Monday|2098Q4|N|N|N|2487674|2488007|2487337|2487610|N|N|N|N|N| +2487703|AAAAAAAAHJFPFCAA|2098-12-30|2387|10384|797|2098|2|12|30|4|2098|797|10384|Tuesday|2098Q4|N|N|N|2487674|2488007|2487338|2487611|N|N|N|N|N| +2487704|AAAAAAAAIJFPFCAA|2098-12-31|2387|10384|797|2098|3|12|31|4|2098|797|10384|Wednesday|2098Q4|N|N|N|2487674|2488007|2487339|2487612|N|N|N|N|N| +2487705|AAAAAAAAJJFPFCAA|2099-01-01|2388|10384|797|2099|4|1|1|1|2099|797|10384|Thursday|2099Q1|Y|N|N|2487705|2487704|2487340|2487613|N|N|N|N|N| +2487706|AAAAAAAAKJFPFCAA|2099-01-02|2388|10384|797|2099|5|1|2|1|2099|797|10384|Friday|2099Q1|N|Y|Y|2487705|2487704|2487341|2487614|N|N|N|N|N| +2487707|AAAAAAAALJFPFCAA|2099-01-03|2388|10384|797|2099|6|1|3|1|2099|797|10384|Saturday|2099Q1|N|Y|N|2487705|2487704|2487342|2487615|N|N|N|N|N| +2487708|AAAAAAAAMJFPFCAA|2099-01-04|2388|10384|797|2099|0|1|4|1|2099|797|10384|Sunday|2099Q1|N|N|N|2487705|2487704|2487343|2487616|N|N|N|N|N| +2487709|AAAAAAAANJFPFCAA|2099-01-05|2388|10384|797|2099|1|1|5|1|2099|797|10384|Monday|2099Q1|N|N|N|2487705|2487704|2487344|2487617|N|N|N|N|N| +2487710|AAAAAAAAOJFPFCAA|2099-01-06|2388|10385|797|2099|2|1|6|1|2099|797|10385|Tuesday|2099Q1|N|N|N|2487705|2487704|2487345|2487618|N|N|N|N|N| +2487711|AAAAAAAAPJFPFCAA|2099-01-07|2388|10385|797|2099|3|1|7|1|2099|797|10385|Wednesday|2099Q1|N|N|N|2487705|2487704|2487346|2487619|N|N|N|N|N| +2487712|AAAAAAAAAKFPFCAA|2099-01-08|2388|10385|797|2099|4|1|8|1|2099|797|10385|Thursday|2099Q1|N|N|N|2487705|2487704|2487347|2487620|N|N|N|N|N| +2487713|AAAAAAAABKFPFCAA|2099-01-09|2388|10385|797|2099|5|1|9|1|2099|797|10385|Friday|2099Q1|N|Y|N|2487705|2487704|2487348|2487621|N|N|N|N|N| +2487714|AAAAAAAACKFPFCAA|2099-01-10|2388|10385|797|2099|6|1|10|1|2099|797|10385|Saturday|2099Q1|N|Y|N|2487705|2487704|2487349|2487622|N|N|N|N|N| +2487715|AAAAAAAADKFPFCAA|2099-01-11|2388|10385|797|2099|0|1|11|1|2099|797|10385|Sunday|2099Q1|N|N|N|2487705|2487704|2487350|2487623|N|N|N|N|N| +2487716|AAAAAAAAEKFPFCAA|2099-01-12|2388|10385|797|2099|1|1|12|1|2099|797|10385|Monday|2099Q1|N|N|N|2487705|2487704|2487351|2487624|N|N|N|N|N| +2487717|AAAAAAAAFKFPFCAA|2099-01-13|2388|10386|797|2099|2|1|13|1|2099|797|10386|Tuesday|2099Q1|N|N|N|2487705|2487704|2487352|2487625|N|N|N|N|N| +2487718|AAAAAAAAGKFPFCAA|2099-01-14|2388|10386|797|2099|3|1|14|1|2099|797|10386|Wednesday|2099Q1|N|N|N|2487705|2487704|2487353|2487626|N|N|N|N|N| +2487719|AAAAAAAAHKFPFCAA|2099-01-15|2388|10386|797|2099|4|1|15|1|2099|797|10386|Thursday|2099Q1|N|N|N|2487705|2487704|2487354|2487627|N|N|N|N|N| +2487720|AAAAAAAAIKFPFCAA|2099-01-16|2388|10386|797|2099|5|1|16|1|2099|797|10386|Friday|2099Q1|N|Y|N|2487705|2487704|2487355|2487628|N|N|N|N|N| +2487721|AAAAAAAAJKFPFCAA|2099-01-17|2388|10386|797|2099|6|1|17|1|2099|797|10386|Saturday|2099Q1|N|Y|N|2487705|2487704|2487356|2487629|N|N|N|N|N| +2487722|AAAAAAAAKKFPFCAA|2099-01-18|2388|10386|797|2099|0|1|18|1|2099|797|10386|Sunday|2099Q1|N|N|N|2487705|2487704|2487357|2487630|N|N|N|N|N| +2487723|AAAAAAAALKFPFCAA|2099-01-19|2388|10386|797|2099|1|1|19|1|2099|797|10386|Monday|2099Q1|N|N|N|2487705|2487704|2487358|2487631|N|N|N|N|N| +2487724|AAAAAAAAMKFPFCAA|2099-01-20|2388|10387|797|2099|2|1|20|1|2099|797|10387|Tuesday|2099Q1|N|N|N|2487705|2487704|2487359|2487632|N|N|N|N|N| +2487725|AAAAAAAANKFPFCAA|2099-01-21|2388|10387|797|2099|3|1|21|1|2099|797|10387|Wednesday|2099Q1|N|N|N|2487705|2487704|2487360|2487633|N|N|N|N|N| +2487726|AAAAAAAAOKFPFCAA|2099-01-22|2388|10387|797|2099|4|1|22|1|2099|797|10387|Thursday|2099Q1|N|N|N|2487705|2487704|2487361|2487634|N|N|N|N|N| +2487727|AAAAAAAAPKFPFCAA|2099-01-23|2388|10387|797|2099|5|1|23|1|2099|797|10387|Friday|2099Q1|N|Y|N|2487705|2487704|2487362|2487635|N|N|N|N|N| +2487728|AAAAAAAAALFPFCAA|2099-01-24|2388|10387|797|2099|6|1|24|1|2099|797|10387|Saturday|2099Q1|N|Y|N|2487705|2487704|2487363|2487636|N|N|N|N|N| +2487729|AAAAAAAABLFPFCAA|2099-01-25|2388|10387|797|2099|0|1|25|1|2099|797|10387|Sunday|2099Q1|N|N|N|2487705|2487704|2487364|2487637|N|N|N|N|N| +2487730|AAAAAAAACLFPFCAA|2099-01-26|2388|10387|797|2099|1|1|26|1|2099|797|10387|Monday|2099Q1|N|N|N|2487705|2487704|2487365|2487638|N|N|N|N|N| +2487731|AAAAAAAADLFPFCAA|2099-01-27|2388|10388|797|2099|2|1|27|1|2099|797|10388|Tuesday|2099Q1|N|N|N|2487705|2487704|2487366|2487639|N|N|N|N|N| +2487732|AAAAAAAAELFPFCAA|2099-01-28|2388|10388|797|2099|3|1|28|1|2099|797|10388|Wednesday|2099Q1|N|N|N|2487705|2487704|2487367|2487640|N|N|N|N|N| +2487733|AAAAAAAAFLFPFCAA|2099-01-29|2388|10388|797|2099|4|1|29|1|2099|797|10388|Thursday|2099Q1|N|N|N|2487705|2487704|2487368|2487641|N|N|N|N|N| +2487734|AAAAAAAAGLFPFCAA|2099-01-30|2388|10388|797|2099|5|1|30|1|2099|797|10388|Friday|2099Q1|N|Y|N|2487705|2487704|2487369|2487642|N|N|N|N|N| +2487735|AAAAAAAAHLFPFCAA|2099-01-31|2388|10388|797|2099|6|1|31|1|2099|797|10388|Saturday|2099Q1|N|Y|N|2487705|2487704|2487370|2487643|N|N|N|N|N| +2487736|AAAAAAAAILFPFCAA|2099-02-01|2389|10388|797|2099|0|2|1|1|2099|797|10388|Sunday|2099Q1|N|N|N|2487736|2487766|2487371|2487644|N|N|N|N|N| +2487737|AAAAAAAAJLFPFCAA|2099-02-02|2389|10388|797|2099|1|2|2|1|2099|797|10388|Monday|2099Q1|N|N|N|2487736|2487766|2487372|2487645|N|N|N|N|N| +2487738|AAAAAAAAKLFPFCAA|2099-02-03|2389|10389|797|2099|2|2|3|1|2099|797|10389|Tuesday|2099Q1|N|N|N|2487736|2487766|2487373|2487646|N|N|N|N|N| +2487739|AAAAAAAALLFPFCAA|2099-02-04|2389|10389|797|2099|3|2|4|1|2099|797|10389|Wednesday|2099Q1|N|N|N|2487736|2487766|2487374|2487647|N|N|N|N|N| +2487740|AAAAAAAAMLFPFCAA|2099-02-05|2389|10389|797|2099|4|2|5|1|2099|797|10389|Thursday|2099Q1|N|N|N|2487736|2487766|2487375|2487648|N|N|N|N|N| +2487741|AAAAAAAANLFPFCAA|2099-02-06|2389|10389|797|2099|5|2|6|1|2099|797|10389|Friday|2099Q1|N|Y|N|2487736|2487766|2487376|2487649|N|N|N|N|N| +2487742|AAAAAAAAOLFPFCAA|2099-02-07|2389|10389|797|2099|6|2|7|1|2099|797|10389|Saturday|2099Q1|N|Y|N|2487736|2487766|2487377|2487650|N|N|N|N|N| +2487743|AAAAAAAAPLFPFCAA|2099-02-08|2389|10389|797|2099|0|2|8|1|2099|797|10389|Sunday|2099Q1|N|N|N|2487736|2487766|2487378|2487651|N|N|N|N|N| +2487744|AAAAAAAAAMFPFCAA|2099-02-09|2389|10389|797|2099|1|2|9|1|2099|797|10389|Monday|2099Q1|N|N|N|2487736|2487766|2487379|2487652|N|N|N|N|N| +2487745|AAAAAAAABMFPFCAA|2099-02-10|2389|10390|797|2099|2|2|10|1|2099|797|10390|Tuesday|2099Q1|N|N|N|2487736|2487766|2487380|2487653|N|N|N|N|N| +2487746|AAAAAAAACMFPFCAA|2099-02-11|2389|10390|797|2099|3|2|11|1|2099|797|10390|Wednesday|2099Q1|N|N|N|2487736|2487766|2487381|2487654|N|N|N|N|N| +2487747|AAAAAAAADMFPFCAA|2099-02-12|2389|10390|797|2099|4|2|12|1|2099|797|10390|Thursday|2099Q1|N|N|N|2487736|2487766|2487382|2487655|N|N|N|N|N| +2487748|AAAAAAAAEMFPFCAA|2099-02-13|2389|10390|797|2099|5|2|13|1|2099|797|10390|Friday|2099Q1|N|Y|N|2487736|2487766|2487383|2487656|N|N|N|N|N| +2487749|AAAAAAAAFMFPFCAA|2099-02-14|2389|10390|797|2099|6|2|14|1|2099|797|10390|Saturday|2099Q1|N|Y|N|2487736|2487766|2487384|2487657|N|N|N|N|N| +2487750|AAAAAAAAGMFPFCAA|2099-02-15|2389|10390|797|2099|0|2|15|1|2099|797|10390|Sunday|2099Q1|N|N|N|2487736|2487766|2487385|2487658|N|N|N|N|N| +2487751|AAAAAAAAHMFPFCAA|2099-02-16|2389|10390|797|2099|1|2|16|1|2099|797|10390|Monday|2099Q1|N|N|N|2487736|2487766|2487386|2487659|N|N|N|N|N| +2487752|AAAAAAAAIMFPFCAA|2099-02-17|2389|10391|797|2099|2|2|17|1|2099|797|10391|Tuesday|2099Q1|N|N|N|2487736|2487766|2487387|2487660|N|N|N|N|N| +2487753|AAAAAAAAJMFPFCAA|2099-02-18|2389|10391|797|2099|3|2|18|1|2099|797|10391|Wednesday|2099Q1|N|N|N|2487736|2487766|2487388|2487661|N|N|N|N|N| +2487754|AAAAAAAAKMFPFCAA|2099-02-19|2389|10391|797|2099|4|2|19|1|2099|797|10391|Thursday|2099Q1|N|N|N|2487736|2487766|2487389|2487662|N|N|N|N|N| +2487755|AAAAAAAALMFPFCAA|2099-02-20|2389|10391|797|2099|5|2|20|1|2099|797|10391|Friday|2099Q1|N|Y|N|2487736|2487766|2487390|2487663|N|N|N|N|N| +2487756|AAAAAAAAMMFPFCAA|2099-02-21|2389|10391|797|2099|6|2|21|1|2099|797|10391|Saturday|2099Q1|N|Y|N|2487736|2487766|2487391|2487664|N|N|N|N|N| +2487757|AAAAAAAANMFPFCAA|2099-02-22|2389|10391|797|2099|0|2|22|1|2099|797|10391|Sunday|2099Q1|N|N|N|2487736|2487766|2487392|2487665|N|N|N|N|N| +2487758|AAAAAAAAOMFPFCAA|2099-02-23|2389|10391|797|2099|1|2|23|1|2099|797|10391|Monday|2099Q1|N|N|N|2487736|2487766|2487393|2487666|N|N|N|N|N| +2487759|AAAAAAAAPMFPFCAA|2099-02-24|2389|10392|797|2099|2|2|24|1|2099|797|10392|Tuesday|2099Q1|N|N|N|2487736|2487766|2487394|2487667|N|N|N|N|N| +2487760|AAAAAAAAANFPFCAA|2099-02-25|2389|10392|797|2099|3|2|25|1|2099|797|10392|Wednesday|2099Q1|N|N|N|2487736|2487766|2487395|2487668|N|N|N|N|N| +2487761|AAAAAAAABNFPFCAA|2099-02-26|2389|10392|797|2099|4|2|26|1|2099|797|10392|Thursday|2099Q1|N|N|N|2487736|2487766|2487396|2487669|N|N|N|N|N| +2487762|AAAAAAAACNFPFCAA|2099-02-27|2389|10392|797|2099|5|2|27|1|2099|797|10392|Friday|2099Q1|N|Y|N|2487736|2487766|2487397|2487670|N|N|N|N|N| +2487763|AAAAAAAADNFPFCAA|2099-02-28|2389|10392|797|2099|6|2|28|1|2099|797|10392|Saturday|2099Q1|N|Y|N|2487736|2487766|2487398|2487671|N|N|N|N|N| +2487764|AAAAAAAAENFPFCAA|2099-03-01|2390|10392|798|2099|0|3|1|1|2099|798|10392|Sunday|2099Q1|N|N|N|2487764|2487822|2487399|2487672|N|N|N|N|N| +2487765|AAAAAAAAFNFPFCAA|2099-03-02|2390|10392|798|2099|1|3|2|1|2099|798|10392|Monday|2099Q1|N|N|N|2487764|2487822|2487400|2487673|N|N|N|N|N| +2487766|AAAAAAAAGNFPFCAA|2099-03-03|2390|10393|798|2099|2|3|3|1|2099|798|10393|Tuesday|2099Q1|N|N|N|2487764|2487822|2487401|2487674|N|N|N|N|N| +2487767|AAAAAAAAHNFPFCAA|2099-03-04|2390|10393|798|2099|3|3|4|1|2099|798|10393|Wednesday|2099Q1|N|N|N|2487764|2487822|2487402|2487675|N|N|N|N|N| +2487768|AAAAAAAAINFPFCAA|2099-03-05|2390|10393|798|2099|4|3|5|1|2099|798|10393|Thursday|2099Q1|N|N|N|2487764|2487822|2487403|2487676|N|N|N|N|N| +2487769|AAAAAAAAJNFPFCAA|2099-03-06|2390|10393|798|2099|5|3|6|1|2099|798|10393|Friday|2099Q1|N|Y|N|2487764|2487822|2487404|2487677|N|N|N|N|N| +2487770|AAAAAAAAKNFPFCAA|2099-03-07|2390|10393|798|2099|6|3|7|1|2099|798|10393|Saturday|2099Q1|N|Y|N|2487764|2487822|2487405|2487678|N|N|N|N|N| +2487771|AAAAAAAALNFPFCAA|2099-03-08|2390|10393|798|2099|0|3|8|1|2099|798|10393|Sunday|2099Q1|N|N|N|2487764|2487822|2487406|2487679|N|N|N|N|N| +2487772|AAAAAAAAMNFPFCAA|2099-03-09|2390|10393|798|2099|1|3|9|1|2099|798|10393|Monday|2099Q1|N|N|N|2487764|2487822|2487407|2487680|N|N|N|N|N| +2487773|AAAAAAAANNFPFCAA|2099-03-10|2390|10394|798|2099|2|3|10|1|2099|798|10394|Tuesday|2099Q1|N|N|N|2487764|2487822|2487408|2487681|N|N|N|N|N| +2487774|AAAAAAAAONFPFCAA|2099-03-11|2390|10394|798|2099|3|3|11|1|2099|798|10394|Wednesday|2099Q1|N|N|N|2487764|2487822|2487409|2487682|N|N|N|N|N| +2487775|AAAAAAAAPNFPFCAA|2099-03-12|2390|10394|798|2099|4|3|12|1|2099|798|10394|Thursday|2099Q1|N|N|N|2487764|2487822|2487410|2487683|N|N|N|N|N| +2487776|AAAAAAAAAOFPFCAA|2099-03-13|2390|10394|798|2099|5|3|13|1|2099|798|10394|Friday|2099Q1|N|Y|N|2487764|2487822|2487411|2487684|N|N|N|N|N| +2487777|AAAAAAAABOFPFCAA|2099-03-14|2390|10394|798|2099|6|3|14|1|2099|798|10394|Saturday|2099Q1|N|Y|N|2487764|2487822|2487412|2487685|N|N|N|N|N| +2487778|AAAAAAAACOFPFCAA|2099-03-15|2390|10394|798|2099|0|3|15|1|2099|798|10394|Sunday|2099Q1|N|N|N|2487764|2487822|2487413|2487686|N|N|N|N|N| +2487779|AAAAAAAADOFPFCAA|2099-03-16|2390|10394|798|2099|1|3|16|1|2099|798|10394|Monday|2099Q1|N|N|N|2487764|2487822|2487414|2487687|N|N|N|N|N| +2487780|AAAAAAAAEOFPFCAA|2099-03-17|2390|10395|798|2099|2|3|17|1|2099|798|10395|Tuesday|2099Q1|N|N|N|2487764|2487822|2487415|2487688|N|N|N|N|N| +2487781|AAAAAAAAFOFPFCAA|2099-03-18|2390|10395|798|2099|3|3|18|1|2099|798|10395|Wednesday|2099Q1|N|N|N|2487764|2487822|2487416|2487689|N|N|N|N|N| +2487782|AAAAAAAAGOFPFCAA|2099-03-19|2390|10395|798|2099|4|3|19|1|2099|798|10395|Thursday|2099Q1|N|N|N|2487764|2487822|2487417|2487690|N|N|N|N|N| +2487783|AAAAAAAAHOFPFCAA|2099-03-20|2390|10395|798|2099|5|3|20|1|2099|798|10395|Friday|2099Q1|N|Y|N|2487764|2487822|2487418|2487691|N|N|N|N|N| +2487784|AAAAAAAAIOFPFCAA|2099-03-21|2390|10395|798|2099|6|3|21|1|2099|798|10395|Saturday|2099Q1|N|Y|N|2487764|2487822|2487419|2487692|N|N|N|N|N| +2487785|AAAAAAAAJOFPFCAA|2099-03-22|2390|10395|798|2099|0|3|22|1|2099|798|10395|Sunday|2099Q1|N|N|N|2487764|2487822|2487420|2487693|N|N|N|N|N| +2487786|AAAAAAAAKOFPFCAA|2099-03-23|2390|10395|798|2099|1|3|23|1|2099|798|10395|Monday|2099Q1|N|N|N|2487764|2487822|2487421|2487694|N|N|N|N|N| +2487787|AAAAAAAALOFPFCAA|2099-03-24|2390|10396|798|2099|2|3|24|1|2099|798|10396|Tuesday|2099Q1|N|N|N|2487764|2487822|2487422|2487695|N|N|N|N|N| +2487788|AAAAAAAAMOFPFCAA|2099-03-25|2390|10396|798|2099|3|3|25|1|2099|798|10396|Wednesday|2099Q1|N|N|N|2487764|2487822|2487423|2487696|N|N|N|N|N| +2487789|AAAAAAAANOFPFCAA|2099-03-26|2390|10396|798|2099|4|3|26|1|2099|798|10396|Thursday|2099Q1|N|N|N|2487764|2487822|2487424|2487697|N|N|N|N|N| +2487790|AAAAAAAAOOFPFCAA|2099-03-27|2390|10396|798|2099|5|3|27|1|2099|798|10396|Friday|2099Q1|N|Y|N|2487764|2487822|2487425|2487698|N|N|N|N|N| +2487791|AAAAAAAAPOFPFCAA|2099-03-28|2390|10396|798|2099|6|3|28|1|2099|798|10396|Saturday|2099Q1|N|Y|N|2487764|2487822|2487426|2487699|N|N|N|N|N| +2487792|AAAAAAAAAPFPFCAA|2099-03-29|2390|10396|798|2099|0|3|29|1|2099|798|10396|Sunday|2099Q1|N|N|N|2487764|2487822|2487427|2487700|N|N|N|N|N| +2487793|AAAAAAAABPFPFCAA|2099-03-30|2390|10396|798|2099|1|3|30|1|2099|798|10396|Monday|2099Q1|N|N|N|2487764|2487822|2487428|2487701|N|N|N|N|N| +2487794|AAAAAAAACPFPFCAA|2099-03-31|2390|10397|798|2099|2|3|31|1|2099|798|10397|Tuesday|2099Q1|N|N|N|2487764|2487822|2487429|2487702|N|N|N|N|N| +2487795|AAAAAAAADPFPFCAA|2099-04-01|2391|10397|798|2099|3|4|1|1|2099|798|10397|Wednesday|2099Q1|N|N|N|2487795|2487884|2487430|2487705|N|N|N|N|N| +2487796|AAAAAAAAEPFPFCAA|2099-04-02|2391|10397|798|2099|4|4|2|2|2099|798|10397|Thursday|2099Q2|N|N|N|2487795|2487884|2487431|2487706|N|N|N|N|N| +2487797|AAAAAAAAFPFPFCAA|2099-04-03|2391|10397|798|2099|5|4|3|2|2099|798|10397|Friday|2099Q2|N|Y|N|2487795|2487884|2487432|2487707|N|N|N|N|N| +2487798|AAAAAAAAGPFPFCAA|2099-04-04|2391|10397|798|2099|6|4|4|2|2099|798|10397|Saturday|2099Q2|N|Y|N|2487795|2487884|2487433|2487708|N|N|N|N|N| +2487799|AAAAAAAAHPFPFCAA|2099-04-05|2391|10397|798|2099|0|4|5|2|2099|798|10397|Sunday|2099Q2|N|N|N|2487795|2487884|2487434|2487709|N|N|N|N|N| +2487800|AAAAAAAAIPFPFCAA|2099-04-06|2391|10397|798|2099|1|4|6|2|2099|798|10397|Monday|2099Q2|N|N|N|2487795|2487884|2487435|2487710|N|N|N|N|N| +2487801|AAAAAAAAJPFPFCAA|2099-04-07|2391|10398|798|2099|2|4|7|2|2099|798|10398|Tuesday|2099Q2|N|N|N|2487795|2487884|2487436|2487711|N|N|N|N|N| +2487802|AAAAAAAAKPFPFCAA|2099-04-08|2391|10398|798|2099|3|4|8|2|2099|798|10398|Wednesday|2099Q2|N|N|N|2487795|2487884|2487437|2487712|N|N|N|N|N| +2487803|AAAAAAAALPFPFCAA|2099-04-09|2391|10398|798|2099|4|4|9|2|2099|798|10398|Thursday|2099Q2|N|N|N|2487795|2487884|2487438|2487713|N|N|N|N|N| +2487804|AAAAAAAAMPFPFCAA|2099-04-10|2391|10398|798|2099|5|4|10|2|2099|798|10398|Friday|2099Q2|N|Y|N|2487795|2487884|2487439|2487714|N|N|N|N|N| +2487805|AAAAAAAANPFPFCAA|2099-04-11|2391|10398|798|2099|6|4|11|2|2099|798|10398|Saturday|2099Q2|N|Y|N|2487795|2487884|2487440|2487715|N|N|N|N|N| +2487806|AAAAAAAAOPFPFCAA|2099-04-12|2391|10398|798|2099|0|4|12|2|2099|798|10398|Sunday|2099Q2|N|N|N|2487795|2487884|2487441|2487716|N|N|N|N|N| +2487807|AAAAAAAAPPFPFCAA|2099-04-13|2391|10398|798|2099|1|4|13|2|2099|798|10398|Monday|2099Q2|N|N|N|2487795|2487884|2487442|2487717|N|N|N|N|N| +2487808|AAAAAAAAAAGPFCAA|2099-04-14|2391|10399|798|2099|2|4|14|2|2099|798|10399|Tuesday|2099Q2|N|N|N|2487795|2487884|2487443|2487718|N|N|N|N|N| +2487809|AAAAAAAABAGPFCAA|2099-04-15|2391|10399|798|2099|3|4|15|2|2099|798|10399|Wednesday|2099Q2|N|N|N|2487795|2487884|2487444|2487719|N|N|N|N|N| +2487810|AAAAAAAACAGPFCAA|2099-04-16|2391|10399|798|2099|4|4|16|2|2099|798|10399|Thursday|2099Q2|N|N|N|2487795|2487884|2487445|2487720|N|N|N|N|N| +2487811|AAAAAAAADAGPFCAA|2099-04-17|2391|10399|798|2099|5|4|17|2|2099|798|10399|Friday|2099Q2|N|Y|N|2487795|2487884|2487446|2487721|N|N|N|N|N| +2487812|AAAAAAAAEAGPFCAA|2099-04-18|2391|10399|798|2099|6|4|18|2|2099|798|10399|Saturday|2099Q2|N|Y|N|2487795|2487884|2487447|2487722|N|N|N|N|N| +2487813|AAAAAAAAFAGPFCAA|2099-04-19|2391|10399|798|2099|0|4|19|2|2099|798|10399|Sunday|2099Q2|N|N|N|2487795|2487884|2487448|2487723|N|N|N|N|N| +2487814|AAAAAAAAGAGPFCAA|2099-04-20|2391|10399|798|2099|1|4|20|2|2099|798|10399|Monday|2099Q2|N|N|N|2487795|2487884|2487449|2487724|N|N|N|N|N| +2487815|AAAAAAAAHAGPFCAA|2099-04-21|2391|10400|798|2099|2|4|21|2|2099|798|10400|Tuesday|2099Q2|N|N|N|2487795|2487884|2487450|2487725|N|N|N|N|N| +2487816|AAAAAAAAIAGPFCAA|2099-04-22|2391|10400|798|2099|3|4|22|2|2099|798|10400|Wednesday|2099Q2|N|N|N|2487795|2487884|2487451|2487726|N|N|N|N|N| +2487817|AAAAAAAAJAGPFCAA|2099-04-23|2391|10400|798|2099|4|4|23|2|2099|798|10400|Thursday|2099Q2|N|N|N|2487795|2487884|2487452|2487727|N|N|N|N|N| +2487818|AAAAAAAAKAGPFCAA|2099-04-24|2391|10400|798|2099|5|4|24|2|2099|798|10400|Friday|2099Q2|N|Y|N|2487795|2487884|2487453|2487728|N|N|N|N|N| +2487819|AAAAAAAALAGPFCAA|2099-04-25|2391|10400|798|2099|6|4|25|2|2099|798|10400|Saturday|2099Q2|N|Y|N|2487795|2487884|2487454|2487729|N|N|N|N|N| +2487820|AAAAAAAAMAGPFCAA|2099-04-26|2391|10400|798|2099|0|4|26|2|2099|798|10400|Sunday|2099Q2|N|N|N|2487795|2487884|2487455|2487730|N|N|N|N|N| +2487821|AAAAAAAANAGPFCAA|2099-04-27|2391|10400|798|2099|1|4|27|2|2099|798|10400|Monday|2099Q2|N|N|N|2487795|2487884|2487456|2487731|N|N|N|N|N| +2487822|AAAAAAAAOAGPFCAA|2099-04-28|2391|10401|798|2099|2|4|28|2|2099|798|10401|Tuesday|2099Q2|N|N|N|2487795|2487884|2487457|2487732|N|N|N|N|N| +2487823|AAAAAAAAPAGPFCAA|2099-04-29|2391|10401|798|2099|3|4|29|2|2099|798|10401|Wednesday|2099Q2|N|N|N|2487795|2487884|2487458|2487733|N|N|N|N|N| +2487824|AAAAAAAAABGPFCAA|2099-04-30|2391|10401|798|2099|4|4|30|2|2099|798|10401|Thursday|2099Q2|N|N|N|2487795|2487884|2487459|2487734|N|N|N|N|N| +2487825|AAAAAAAABBGPFCAA|2099-05-01|2392|10401|798|2099|5|5|1|2|2099|798|10401|Friday|2099Q2|N|Y|N|2487825|2487944|2487460|2487735|N|N|N|N|N| +2487826|AAAAAAAACBGPFCAA|2099-05-02|2392|10401|798|2099|6|5|2|2|2099|798|10401|Saturday|2099Q2|N|Y|N|2487825|2487944|2487461|2487736|N|N|N|N|N| +2487827|AAAAAAAADBGPFCAA|2099-05-03|2392|10401|798|2099|0|5|3|2|2099|798|10401|Sunday|2099Q2|N|N|N|2487825|2487944|2487462|2487737|N|N|N|N|N| +2487828|AAAAAAAAEBGPFCAA|2099-05-04|2392|10401|798|2099|1|5|4|2|2099|798|10401|Monday|2099Q2|N|N|N|2487825|2487944|2487463|2487738|N|N|N|N|N| +2487829|AAAAAAAAFBGPFCAA|2099-05-05|2392|10402|798|2099|2|5|5|2|2099|798|10402|Tuesday|2099Q2|N|N|N|2487825|2487944|2487464|2487739|N|N|N|N|N| +2487830|AAAAAAAAGBGPFCAA|2099-05-06|2392|10402|798|2099|3|5|6|2|2099|798|10402|Wednesday|2099Q2|N|N|N|2487825|2487944|2487465|2487740|N|N|N|N|N| +2487831|AAAAAAAAHBGPFCAA|2099-05-07|2392|10402|798|2099|4|5|7|2|2099|798|10402|Thursday|2099Q2|N|N|N|2487825|2487944|2487466|2487741|N|N|N|N|N| +2487832|AAAAAAAAIBGPFCAA|2099-05-08|2392|10402|798|2099|5|5|8|2|2099|798|10402|Friday|2099Q2|N|Y|N|2487825|2487944|2487467|2487742|N|N|N|N|N| +2487833|AAAAAAAAJBGPFCAA|2099-05-09|2392|10402|798|2099|6|5|9|2|2099|798|10402|Saturday|2099Q2|N|Y|N|2487825|2487944|2487468|2487743|N|N|N|N|N| +2487834|AAAAAAAAKBGPFCAA|2099-05-10|2392|10402|798|2099|0|5|10|2|2099|798|10402|Sunday|2099Q2|N|N|N|2487825|2487944|2487469|2487744|N|N|N|N|N| +2487835|AAAAAAAALBGPFCAA|2099-05-11|2392|10402|798|2099|1|5|11|2|2099|798|10402|Monday|2099Q2|N|N|N|2487825|2487944|2487470|2487745|N|N|N|N|N| +2487836|AAAAAAAAMBGPFCAA|2099-05-12|2392|10403|798|2099|2|5|12|2|2099|798|10403|Tuesday|2099Q2|N|N|N|2487825|2487944|2487471|2487746|N|N|N|N|N| +2487837|AAAAAAAANBGPFCAA|2099-05-13|2392|10403|798|2099|3|5|13|2|2099|798|10403|Wednesday|2099Q2|N|N|N|2487825|2487944|2487472|2487747|N|N|N|N|N| +2487838|AAAAAAAAOBGPFCAA|2099-05-14|2392|10403|798|2099|4|5|14|2|2099|798|10403|Thursday|2099Q2|N|N|N|2487825|2487944|2487473|2487748|N|N|N|N|N| +2487839|AAAAAAAAPBGPFCAA|2099-05-15|2392|10403|798|2099|5|5|15|2|2099|798|10403|Friday|2099Q2|N|Y|N|2487825|2487944|2487474|2487749|N|N|N|N|N| +2487840|AAAAAAAAACGPFCAA|2099-05-16|2392|10403|798|2099|6|5|16|2|2099|798|10403|Saturday|2099Q2|N|Y|N|2487825|2487944|2487475|2487750|N|N|N|N|N| +2487841|AAAAAAAABCGPFCAA|2099-05-17|2392|10403|798|2099|0|5|17|2|2099|798|10403|Sunday|2099Q2|N|N|N|2487825|2487944|2487476|2487751|N|N|N|N|N| +2487842|AAAAAAAACCGPFCAA|2099-05-18|2392|10403|798|2099|1|5|18|2|2099|798|10403|Monday|2099Q2|N|N|N|2487825|2487944|2487477|2487752|N|N|N|N|N| +2487843|AAAAAAAADCGPFCAA|2099-05-19|2392|10404|798|2099|2|5|19|2|2099|798|10404|Tuesday|2099Q2|N|N|N|2487825|2487944|2487478|2487753|N|N|N|N|N| +2487844|AAAAAAAAECGPFCAA|2099-05-20|2392|10404|798|2099|3|5|20|2|2099|798|10404|Wednesday|2099Q2|N|N|N|2487825|2487944|2487479|2487754|N|N|N|N|N| +2487845|AAAAAAAAFCGPFCAA|2099-05-21|2392|10404|798|2099|4|5|21|2|2099|798|10404|Thursday|2099Q2|N|N|N|2487825|2487944|2487480|2487755|N|N|N|N|N| +2487846|AAAAAAAAGCGPFCAA|2099-05-22|2392|10404|798|2099|5|5|22|2|2099|798|10404|Friday|2099Q2|N|Y|N|2487825|2487944|2487481|2487756|N|N|N|N|N| +2487847|AAAAAAAAHCGPFCAA|2099-05-23|2392|10404|798|2099|6|5|23|2|2099|798|10404|Saturday|2099Q2|N|Y|N|2487825|2487944|2487482|2487757|N|N|N|N|N| +2487848|AAAAAAAAICGPFCAA|2099-05-24|2392|10404|798|2099|0|5|24|2|2099|798|10404|Sunday|2099Q2|N|N|N|2487825|2487944|2487483|2487758|N|N|N|N|N| +2487849|AAAAAAAAJCGPFCAA|2099-05-25|2392|10404|798|2099|1|5|25|2|2099|798|10404|Monday|2099Q2|N|N|N|2487825|2487944|2487484|2487759|N|N|N|N|N| +2487850|AAAAAAAAKCGPFCAA|2099-05-26|2392|10405|798|2099|2|5|26|2|2099|798|10405|Tuesday|2099Q2|N|N|N|2487825|2487944|2487485|2487760|N|N|N|N|N| +2487851|AAAAAAAALCGPFCAA|2099-05-27|2392|10405|798|2099|3|5|27|2|2099|798|10405|Wednesday|2099Q2|N|N|N|2487825|2487944|2487486|2487761|N|N|N|N|N| +2487852|AAAAAAAAMCGPFCAA|2099-05-28|2392|10405|798|2099|4|5|28|2|2099|798|10405|Thursday|2099Q2|N|N|N|2487825|2487944|2487487|2487762|N|N|N|N|N| +2487853|AAAAAAAANCGPFCAA|2099-05-29|2392|10405|798|2099|5|5|29|2|2099|798|10405|Friday|2099Q2|N|Y|N|2487825|2487944|2487488|2487763|N|N|N|N|N| +2487854|AAAAAAAAOCGPFCAA|2099-05-30|2392|10405|798|2099|6|5|30|2|2099|798|10405|Saturday|2099Q2|N|Y|N|2487825|2487944|2487489|2487764|N|N|N|N|N| +2487855|AAAAAAAAPCGPFCAA|2099-05-31|2392|10405|798|2099|0|5|31|2|2099|798|10405|Sunday|2099Q2|N|N|N|2487825|2487944|2487490|2487765|N|N|N|N|N| +2487856|AAAAAAAAADGPFCAA|2099-06-01|2393|10405|799|2099|1|6|1|2|2099|799|10405|Monday|2099Q2|N|N|N|2487856|2488006|2487491|2487766|N|N|N|N|N| +2487857|AAAAAAAABDGPFCAA|2099-06-02|2393|10406|799|2099|2|6|2|2|2099|799|10406|Tuesday|2099Q2|N|N|N|2487856|2488006|2487492|2487767|N|N|N|N|N| +2487858|AAAAAAAACDGPFCAA|2099-06-03|2393|10406|799|2099|3|6|3|2|2099|799|10406|Wednesday|2099Q2|N|N|N|2487856|2488006|2487493|2487768|N|N|N|N|N| +2487859|AAAAAAAADDGPFCAA|2099-06-04|2393|10406|799|2099|4|6|4|2|2099|799|10406|Thursday|2099Q2|N|N|N|2487856|2488006|2487494|2487769|N|N|N|N|N| +2487860|AAAAAAAAEDGPFCAA|2099-06-05|2393|10406|799|2099|5|6|5|2|2099|799|10406|Friday|2099Q2|N|Y|N|2487856|2488006|2487495|2487770|N|N|N|N|N| +2487861|AAAAAAAAFDGPFCAA|2099-06-06|2393|10406|799|2099|6|6|6|2|2099|799|10406|Saturday|2099Q2|N|Y|N|2487856|2488006|2487496|2487771|N|N|N|N|N| +2487862|AAAAAAAAGDGPFCAA|2099-06-07|2393|10406|799|2099|0|6|7|2|2099|799|10406|Sunday|2099Q2|N|N|N|2487856|2488006|2487497|2487772|N|N|N|N|N| +2487863|AAAAAAAAHDGPFCAA|2099-06-08|2393|10406|799|2099|1|6|8|2|2099|799|10406|Monday|2099Q2|N|N|N|2487856|2488006|2487498|2487773|N|N|N|N|N| +2487864|AAAAAAAAIDGPFCAA|2099-06-09|2393|10407|799|2099|2|6|9|2|2099|799|10407|Tuesday|2099Q2|N|N|N|2487856|2488006|2487499|2487774|N|N|N|N|N| +2487865|AAAAAAAAJDGPFCAA|2099-06-10|2393|10407|799|2099|3|6|10|2|2099|799|10407|Wednesday|2099Q2|N|N|N|2487856|2488006|2487500|2487775|N|N|N|N|N| +2487866|AAAAAAAAKDGPFCAA|2099-06-11|2393|10407|799|2099|4|6|11|2|2099|799|10407|Thursday|2099Q2|N|N|N|2487856|2488006|2487501|2487776|N|N|N|N|N| +2487867|AAAAAAAALDGPFCAA|2099-06-12|2393|10407|799|2099|5|6|12|2|2099|799|10407|Friday|2099Q2|N|Y|N|2487856|2488006|2487502|2487777|N|N|N|N|N| +2487868|AAAAAAAAMDGPFCAA|2099-06-13|2393|10407|799|2099|6|6|13|2|2099|799|10407|Saturday|2099Q2|N|Y|N|2487856|2488006|2487503|2487778|N|N|N|N|N| +2487869|AAAAAAAANDGPFCAA|2099-06-14|2393|10407|799|2099|0|6|14|2|2099|799|10407|Sunday|2099Q2|N|N|N|2487856|2488006|2487504|2487779|N|N|N|N|N| +2487870|AAAAAAAAODGPFCAA|2099-06-15|2393|10407|799|2099|1|6|15|2|2099|799|10407|Monday|2099Q2|N|N|N|2487856|2488006|2487505|2487780|N|N|N|N|N| +2487871|AAAAAAAAPDGPFCAA|2099-06-16|2393|10408|799|2099|2|6|16|2|2099|799|10408|Tuesday|2099Q2|N|N|N|2487856|2488006|2487506|2487781|N|N|N|N|N| +2487872|AAAAAAAAAEGPFCAA|2099-06-17|2393|10408|799|2099|3|6|17|2|2099|799|10408|Wednesday|2099Q2|N|N|N|2487856|2488006|2487507|2487782|N|N|N|N|N| +2487873|AAAAAAAABEGPFCAA|2099-06-18|2393|10408|799|2099|4|6|18|2|2099|799|10408|Thursday|2099Q2|N|N|N|2487856|2488006|2487508|2487783|N|N|N|N|N| +2487874|AAAAAAAACEGPFCAA|2099-06-19|2393|10408|799|2099|5|6|19|2|2099|799|10408|Friday|2099Q2|N|Y|N|2487856|2488006|2487509|2487784|N|N|N|N|N| +2487875|AAAAAAAADEGPFCAA|2099-06-20|2393|10408|799|2099|6|6|20|2|2099|799|10408|Saturday|2099Q2|N|Y|N|2487856|2488006|2487510|2487785|N|N|N|N|N| +2487876|AAAAAAAAEEGPFCAA|2099-06-21|2393|10408|799|2099|0|6|21|2|2099|799|10408|Sunday|2099Q2|N|N|N|2487856|2488006|2487511|2487786|N|N|N|N|N| +2487877|AAAAAAAAFEGPFCAA|2099-06-22|2393|10408|799|2099|1|6|22|2|2099|799|10408|Monday|2099Q2|N|N|N|2487856|2488006|2487512|2487787|N|N|N|N|N| +2487878|AAAAAAAAGEGPFCAA|2099-06-23|2393|10409|799|2099|2|6|23|2|2099|799|10409|Tuesday|2099Q2|N|N|N|2487856|2488006|2487513|2487788|N|N|N|N|N| +2487879|AAAAAAAAHEGPFCAA|2099-06-24|2393|10409|799|2099|3|6|24|2|2099|799|10409|Wednesday|2099Q2|N|N|N|2487856|2488006|2487514|2487789|N|N|N|N|N| +2487880|AAAAAAAAIEGPFCAA|2099-06-25|2393|10409|799|2099|4|6|25|2|2099|799|10409|Thursday|2099Q2|N|N|N|2487856|2488006|2487515|2487790|N|N|N|N|N| +2487881|AAAAAAAAJEGPFCAA|2099-06-26|2393|10409|799|2099|5|6|26|2|2099|799|10409|Friday|2099Q2|N|Y|N|2487856|2488006|2487516|2487791|N|N|N|N|N| +2487882|AAAAAAAAKEGPFCAA|2099-06-27|2393|10409|799|2099|6|6|27|2|2099|799|10409|Saturday|2099Q2|N|Y|N|2487856|2488006|2487517|2487792|N|N|N|N|N| +2487883|AAAAAAAALEGPFCAA|2099-06-28|2393|10409|799|2099|0|6|28|2|2099|799|10409|Sunday|2099Q2|N|N|N|2487856|2488006|2487518|2487793|N|N|N|N|N| +2487884|AAAAAAAAMEGPFCAA|2099-06-29|2393|10409|799|2099|1|6|29|2|2099|799|10409|Monday|2099Q2|N|N|N|2487856|2488006|2487519|2487794|N|N|N|N|N| +2487885|AAAAAAAANEGPFCAA|2099-06-30|2393|10410|799|2099|2|6|30|2|2099|799|10410|Tuesday|2099Q2|N|N|N|2487856|2488006|2487520|2487795|N|N|N|N|N| +2487886|AAAAAAAAOEGPFCAA|2099-07-01|2394|10410|799|2099|3|7|1|2|2099|799|10410|Wednesday|2099Q2|N|N|N|2487886|2488066|2487521|2487795|N|N|N|N|N| +2487887|AAAAAAAAPEGPFCAA|2099-07-02|2394|10410|799|2099|4|7|2|3|2099|799|10410|Thursday|2099Q3|N|N|N|2487886|2488066|2487522|2487796|N|N|N|N|N| +2487888|AAAAAAAAAFGPFCAA|2099-07-03|2394|10410|799|2099|5|7|3|3|2099|799|10410|Friday|2099Q3|N|Y|N|2487886|2488066|2487523|2487797|N|N|N|N|N| +2487889|AAAAAAAABFGPFCAA|2099-07-04|2394|10410|799|2099|6|7|4|3|2099|799|10410|Saturday|2099Q3|N|Y|N|2487886|2488066|2487524|2487798|N|N|N|N|N| +2487890|AAAAAAAACFGPFCAA|2099-07-05|2394|10410|799|2099|0|7|5|3|2099|799|10410|Sunday|2099Q3|Y|N|N|2487886|2488066|2487525|2487799|N|N|N|N|N| +2487891|AAAAAAAADFGPFCAA|2099-07-06|2394|10410|799|2099|1|7|6|3|2099|799|10410|Monday|2099Q3|N|N|Y|2487886|2488066|2487526|2487800|N|N|N|N|N| +2487892|AAAAAAAAEFGPFCAA|2099-07-07|2394|10411|799|2099|2|7|7|3|2099|799|10411|Tuesday|2099Q3|N|N|N|2487886|2488066|2487527|2487801|N|N|N|N|N| +2487893|AAAAAAAAFFGPFCAA|2099-07-08|2394|10411|799|2099|3|7|8|3|2099|799|10411|Wednesday|2099Q3|N|N|N|2487886|2488066|2487528|2487802|N|N|N|N|N| +2487894|AAAAAAAAGFGPFCAA|2099-07-09|2394|10411|799|2099|4|7|9|3|2099|799|10411|Thursday|2099Q3|N|N|N|2487886|2488066|2487529|2487803|N|N|N|N|N| +2487895|AAAAAAAAHFGPFCAA|2099-07-10|2394|10411|799|2099|5|7|10|3|2099|799|10411|Friday|2099Q3|N|Y|N|2487886|2488066|2487530|2487804|N|N|N|N|N| +2487896|AAAAAAAAIFGPFCAA|2099-07-11|2394|10411|799|2099|6|7|11|3|2099|799|10411|Saturday|2099Q3|N|Y|N|2487886|2488066|2487531|2487805|N|N|N|N|N| +2487897|AAAAAAAAJFGPFCAA|2099-07-12|2394|10411|799|2099|0|7|12|3|2099|799|10411|Sunday|2099Q3|N|N|N|2487886|2488066|2487532|2487806|N|N|N|N|N| +2487898|AAAAAAAAKFGPFCAA|2099-07-13|2394|10411|799|2099|1|7|13|3|2099|799|10411|Monday|2099Q3|N|N|N|2487886|2488066|2487533|2487807|N|N|N|N|N| +2487899|AAAAAAAALFGPFCAA|2099-07-14|2394|10412|799|2099|2|7|14|3|2099|799|10412|Tuesday|2099Q3|N|N|N|2487886|2488066|2487534|2487808|N|N|N|N|N| +2487900|AAAAAAAAMFGPFCAA|2099-07-15|2394|10412|799|2099|3|7|15|3|2099|799|10412|Wednesday|2099Q3|N|N|N|2487886|2488066|2487535|2487809|N|N|N|N|N| +2487901|AAAAAAAANFGPFCAA|2099-07-16|2394|10412|799|2099|4|7|16|3|2099|799|10412|Thursday|2099Q3|N|N|N|2487886|2488066|2487536|2487810|N|N|N|N|N| +2487902|AAAAAAAAOFGPFCAA|2099-07-17|2394|10412|799|2099|5|7|17|3|2099|799|10412|Friday|2099Q3|N|Y|N|2487886|2488066|2487537|2487811|N|N|N|N|N| +2487903|AAAAAAAAPFGPFCAA|2099-07-18|2394|10412|799|2099|6|7|18|3|2099|799|10412|Saturday|2099Q3|N|Y|N|2487886|2488066|2487538|2487812|N|N|N|N|N| +2487904|AAAAAAAAAGGPFCAA|2099-07-19|2394|10412|799|2099|0|7|19|3|2099|799|10412|Sunday|2099Q3|N|N|N|2487886|2488066|2487539|2487813|N|N|N|N|N| +2487905|AAAAAAAABGGPFCAA|2099-07-20|2394|10412|799|2099|1|7|20|3|2099|799|10412|Monday|2099Q3|N|N|N|2487886|2488066|2487540|2487814|N|N|N|N|N| +2487906|AAAAAAAACGGPFCAA|2099-07-21|2394|10413|799|2099|2|7|21|3|2099|799|10413|Tuesday|2099Q3|N|N|N|2487886|2488066|2487541|2487815|N|N|N|N|N| +2487907|AAAAAAAADGGPFCAA|2099-07-22|2394|10413|799|2099|3|7|22|3|2099|799|10413|Wednesday|2099Q3|N|N|N|2487886|2488066|2487542|2487816|N|N|N|N|N| +2487908|AAAAAAAAEGGPFCAA|2099-07-23|2394|10413|799|2099|4|7|23|3|2099|799|10413|Thursday|2099Q3|N|N|N|2487886|2488066|2487543|2487817|N|N|N|N|N| +2487909|AAAAAAAAFGGPFCAA|2099-07-24|2394|10413|799|2099|5|7|24|3|2099|799|10413|Friday|2099Q3|N|Y|N|2487886|2488066|2487544|2487818|N|N|N|N|N| +2487910|AAAAAAAAGGGPFCAA|2099-07-25|2394|10413|799|2099|6|7|25|3|2099|799|10413|Saturday|2099Q3|N|Y|N|2487886|2488066|2487545|2487819|N|N|N|N|N| +2487911|AAAAAAAAHGGPFCAA|2099-07-26|2394|10413|799|2099|0|7|26|3|2099|799|10413|Sunday|2099Q3|N|N|N|2487886|2488066|2487546|2487820|N|N|N|N|N| +2487912|AAAAAAAAIGGPFCAA|2099-07-27|2394|10413|799|2099|1|7|27|3|2099|799|10413|Monday|2099Q3|N|N|N|2487886|2488066|2487547|2487821|N|N|N|N|N| +2487913|AAAAAAAAJGGPFCAA|2099-07-28|2394|10414|799|2099|2|7|28|3|2099|799|10414|Tuesday|2099Q3|N|N|N|2487886|2488066|2487548|2487822|N|N|N|N|N| +2487914|AAAAAAAAKGGPFCAA|2099-07-29|2394|10414|799|2099|3|7|29|3|2099|799|10414|Wednesday|2099Q3|N|N|N|2487886|2488066|2487549|2487823|N|N|N|N|N| +2487915|AAAAAAAALGGPFCAA|2099-07-30|2394|10414|799|2099|4|7|30|3|2099|799|10414|Thursday|2099Q3|N|N|N|2487886|2488066|2487550|2487824|N|N|N|N|N| +2487916|AAAAAAAAMGGPFCAA|2099-07-31|2394|10414|799|2099|5|7|31|3|2099|799|10414|Friday|2099Q3|N|Y|N|2487886|2488066|2487551|2487825|N|N|N|N|N| +2487917|AAAAAAAANGGPFCAA|2099-08-01|2395|10414|799|2099|6|8|1|3|2099|799|10414|Saturday|2099Q3|N|Y|N|2487917|2488128|2487552|2487826|N|N|N|N|N| +2487918|AAAAAAAAOGGPFCAA|2099-08-02|2395|10414|799|2099|0|8|2|3|2099|799|10414|Sunday|2099Q3|N|N|N|2487917|2488128|2487553|2487827|N|N|N|N|N| +2487919|AAAAAAAAPGGPFCAA|2099-08-03|2395|10414|799|2099|1|8|3|3|2099|799|10414|Monday|2099Q3|N|N|N|2487917|2488128|2487554|2487828|N|N|N|N|N| +2487920|AAAAAAAAAHGPFCAA|2099-08-04|2395|10415|799|2099|2|8|4|3|2099|799|10415|Tuesday|2099Q3|N|N|N|2487917|2488128|2487555|2487829|N|N|N|N|N| +2487921|AAAAAAAABHGPFCAA|2099-08-05|2395|10415|799|2099|3|8|5|3|2099|799|10415|Wednesday|2099Q3|N|N|N|2487917|2488128|2487556|2487830|N|N|N|N|N| +2487922|AAAAAAAACHGPFCAA|2099-08-06|2395|10415|799|2099|4|8|6|3|2099|799|10415|Thursday|2099Q3|N|N|N|2487917|2488128|2487557|2487831|N|N|N|N|N| +2487923|AAAAAAAADHGPFCAA|2099-08-07|2395|10415|799|2099|5|8|7|3|2099|799|10415|Friday|2099Q3|N|Y|N|2487917|2488128|2487558|2487832|N|N|N|N|N| +2487924|AAAAAAAAEHGPFCAA|2099-08-08|2395|10415|799|2099|6|8|8|3|2099|799|10415|Saturday|2099Q3|N|Y|N|2487917|2488128|2487559|2487833|N|N|N|N|N| +2487925|AAAAAAAAFHGPFCAA|2099-08-09|2395|10415|799|2099|0|8|9|3|2099|799|10415|Sunday|2099Q3|N|N|N|2487917|2488128|2487560|2487834|N|N|N|N|N| +2487926|AAAAAAAAGHGPFCAA|2099-08-10|2395|10415|799|2099|1|8|10|3|2099|799|10415|Monday|2099Q3|N|N|N|2487917|2488128|2487561|2487835|N|N|N|N|N| +2487927|AAAAAAAAHHGPFCAA|2099-08-11|2395|10416|799|2099|2|8|11|3|2099|799|10416|Tuesday|2099Q3|N|N|N|2487917|2488128|2487562|2487836|N|N|N|N|N| +2487928|AAAAAAAAIHGPFCAA|2099-08-12|2395|10416|799|2099|3|8|12|3|2099|799|10416|Wednesday|2099Q3|N|N|N|2487917|2488128|2487563|2487837|N|N|N|N|N| +2487929|AAAAAAAAJHGPFCAA|2099-08-13|2395|10416|799|2099|4|8|13|3|2099|799|10416|Thursday|2099Q3|N|N|N|2487917|2488128|2487564|2487838|N|N|N|N|N| +2487930|AAAAAAAAKHGPFCAA|2099-08-14|2395|10416|799|2099|5|8|14|3|2099|799|10416|Friday|2099Q3|N|Y|N|2487917|2488128|2487565|2487839|N|N|N|N|N| +2487931|AAAAAAAALHGPFCAA|2099-08-15|2395|10416|799|2099|6|8|15|3|2099|799|10416|Saturday|2099Q3|N|Y|N|2487917|2488128|2487566|2487840|N|N|N|N|N| +2487932|AAAAAAAAMHGPFCAA|2099-08-16|2395|10416|799|2099|0|8|16|3|2099|799|10416|Sunday|2099Q3|N|N|N|2487917|2488128|2487567|2487841|N|N|N|N|N| +2487933|AAAAAAAANHGPFCAA|2099-08-17|2395|10416|799|2099|1|8|17|3|2099|799|10416|Monday|2099Q3|N|N|N|2487917|2488128|2487568|2487842|N|N|N|N|N| +2487934|AAAAAAAAOHGPFCAA|2099-08-18|2395|10417|799|2099|2|8|18|3|2099|799|10417|Tuesday|2099Q3|N|N|N|2487917|2488128|2487569|2487843|N|N|N|N|N| +2487935|AAAAAAAAPHGPFCAA|2099-08-19|2395|10417|799|2099|3|8|19|3|2099|799|10417|Wednesday|2099Q3|N|N|N|2487917|2488128|2487570|2487844|N|N|N|N|N| +2487936|AAAAAAAAAIGPFCAA|2099-08-20|2395|10417|799|2099|4|8|20|3|2099|799|10417|Thursday|2099Q3|N|N|N|2487917|2488128|2487571|2487845|N|N|N|N|N| +2487937|AAAAAAAABIGPFCAA|2099-08-21|2395|10417|799|2099|5|8|21|3|2099|799|10417|Friday|2099Q3|N|Y|N|2487917|2488128|2487572|2487846|N|N|N|N|N| +2487938|AAAAAAAACIGPFCAA|2099-08-22|2395|10417|799|2099|6|8|22|3|2099|799|10417|Saturday|2099Q3|N|Y|N|2487917|2488128|2487573|2487847|N|N|N|N|N| +2487939|AAAAAAAADIGPFCAA|2099-08-23|2395|10417|799|2099|0|8|23|3|2099|799|10417|Sunday|2099Q3|N|N|N|2487917|2488128|2487574|2487848|N|N|N|N|N| +2487940|AAAAAAAAEIGPFCAA|2099-08-24|2395|10417|799|2099|1|8|24|3|2099|799|10417|Monday|2099Q3|N|N|N|2487917|2488128|2487575|2487849|N|N|N|N|N| +2487941|AAAAAAAAFIGPFCAA|2099-08-25|2395|10418|799|2099|2|8|25|3|2099|799|10418|Tuesday|2099Q3|N|N|N|2487917|2488128|2487576|2487850|N|N|N|N|N| +2487942|AAAAAAAAGIGPFCAA|2099-08-26|2395|10418|799|2099|3|8|26|3|2099|799|10418|Wednesday|2099Q3|N|N|N|2487917|2488128|2487577|2487851|N|N|N|N|N| +2487943|AAAAAAAAHIGPFCAA|2099-08-27|2395|10418|799|2099|4|8|27|3|2099|799|10418|Thursday|2099Q3|N|N|N|2487917|2488128|2487578|2487852|N|N|N|N|N| +2487944|AAAAAAAAIIGPFCAA|2099-08-28|2395|10418|799|2099|5|8|28|3|2099|799|10418|Friday|2099Q3|N|Y|N|2487917|2488128|2487579|2487853|N|N|N|N|N| +2487945|AAAAAAAAJIGPFCAA|2099-08-29|2395|10418|799|2099|6|8|29|3|2099|799|10418|Saturday|2099Q3|N|Y|N|2487917|2488128|2487580|2487854|N|N|N|N|N| +2487946|AAAAAAAAKIGPFCAA|2099-08-30|2395|10418|799|2099|0|8|30|3|2099|799|10418|Sunday|2099Q3|N|N|N|2487917|2488128|2487581|2487855|N|N|N|N|N| +2487947|AAAAAAAALIGPFCAA|2099-08-31|2395|10418|799|2099|1|8|31|3|2099|799|10418|Monday|2099Q3|N|N|N|2487917|2488128|2487582|2487856|N|N|N|N|N| +2487948|AAAAAAAAMIGPFCAA|2099-09-01|2396|10419|800|2099|2|9|1|3|2099|800|10419|Tuesday|2099Q3|N|N|N|2487948|2488190|2487583|2487857|N|N|N|N|N| +2487949|AAAAAAAANIGPFCAA|2099-09-02|2396|10419|800|2099|3|9|2|3|2099|800|10419|Wednesday|2099Q3|N|N|N|2487948|2488190|2487584|2487858|N|N|N|N|N| +2487950|AAAAAAAAOIGPFCAA|2099-09-03|2396|10419|800|2099|4|9|3|3|2099|800|10419|Thursday|2099Q3|N|N|N|2487948|2488190|2487585|2487859|N|N|N|N|N| +2487951|AAAAAAAAPIGPFCAA|2099-09-04|2396|10419|800|2099|5|9|4|3|2099|800|10419|Friday|2099Q3|N|Y|N|2487948|2488190|2487586|2487860|N|N|N|N|N| +2487952|AAAAAAAAAJGPFCAA|2099-09-05|2396|10419|800|2099|6|9|5|3|2099|800|10419|Saturday|2099Q3|N|Y|N|2487948|2488190|2487587|2487861|N|N|N|N|N| +2487953|AAAAAAAABJGPFCAA|2099-09-06|2396|10419|800|2099|0|9|6|3|2099|800|10419|Sunday|2099Q3|N|N|N|2487948|2488190|2487588|2487862|N|N|N|N|N| +2487954|AAAAAAAACJGPFCAA|2099-09-07|2396|10419|800|2099|1|9|7|3|2099|800|10419|Monday|2099Q3|N|N|N|2487948|2488190|2487589|2487863|N|N|N|N|N| +2487955|AAAAAAAADJGPFCAA|2099-09-08|2396|10420|800|2099|2|9|8|3|2099|800|10420|Tuesday|2099Q3|N|N|N|2487948|2488190|2487590|2487864|N|N|N|N|N| +2487956|AAAAAAAAEJGPFCAA|2099-09-09|2396|10420|800|2099|3|9|9|3|2099|800|10420|Wednesday|2099Q3|N|N|N|2487948|2488190|2487591|2487865|N|N|N|N|N| +2487957|AAAAAAAAFJGPFCAA|2099-09-10|2396|10420|800|2099|4|9|10|3|2099|800|10420|Thursday|2099Q3|N|N|N|2487948|2488190|2487592|2487866|N|N|N|N|N| +2487958|AAAAAAAAGJGPFCAA|2099-09-11|2396|10420|800|2099|5|9|11|3|2099|800|10420|Friday|2099Q3|N|Y|N|2487948|2488190|2487593|2487867|N|N|N|N|N| +2487959|AAAAAAAAHJGPFCAA|2099-09-12|2396|10420|800|2099|6|9|12|3|2099|800|10420|Saturday|2099Q3|N|Y|N|2487948|2488190|2487594|2487868|N|N|N|N|N| +2487960|AAAAAAAAIJGPFCAA|2099-09-13|2396|10420|800|2099|0|9|13|3|2099|800|10420|Sunday|2099Q3|N|N|N|2487948|2488190|2487595|2487869|N|N|N|N|N| +2487961|AAAAAAAAJJGPFCAA|2099-09-14|2396|10420|800|2099|1|9|14|3|2099|800|10420|Monday|2099Q3|N|N|N|2487948|2488190|2487596|2487870|N|N|N|N|N| +2487962|AAAAAAAAKJGPFCAA|2099-09-15|2396|10421|800|2099|2|9|15|3|2099|800|10421|Tuesday|2099Q3|N|N|N|2487948|2488190|2487597|2487871|N|N|N|N|N| +2487963|AAAAAAAALJGPFCAA|2099-09-16|2396|10421|800|2099|3|9|16|3|2099|800|10421|Wednesday|2099Q3|N|N|N|2487948|2488190|2487598|2487872|N|N|N|N|N| +2487964|AAAAAAAAMJGPFCAA|2099-09-17|2396|10421|800|2099|4|9|17|3|2099|800|10421|Thursday|2099Q3|N|N|N|2487948|2488190|2487599|2487873|N|N|N|N|N| +2487965|AAAAAAAANJGPFCAA|2099-09-18|2396|10421|800|2099|5|9|18|3|2099|800|10421|Friday|2099Q3|N|Y|N|2487948|2488190|2487600|2487874|N|N|N|N|N| +2487966|AAAAAAAAOJGPFCAA|2099-09-19|2396|10421|800|2099|6|9|19|3|2099|800|10421|Saturday|2099Q3|N|Y|N|2487948|2488190|2487601|2487875|N|N|N|N|N| +2487967|AAAAAAAAPJGPFCAA|2099-09-20|2396|10421|800|2099|0|9|20|3|2099|800|10421|Sunday|2099Q3|N|N|N|2487948|2488190|2487602|2487876|N|N|N|N|N| +2487968|AAAAAAAAAKGPFCAA|2099-09-21|2396|10421|800|2099|1|9|21|3|2099|800|10421|Monday|2099Q3|N|N|N|2487948|2488190|2487603|2487877|N|N|N|N|N| +2487969|AAAAAAAABKGPFCAA|2099-09-22|2396|10422|800|2099|2|9|22|3|2099|800|10422|Tuesday|2099Q3|N|N|N|2487948|2488190|2487604|2487878|N|N|N|N|N| +2487970|AAAAAAAACKGPFCAA|2099-09-23|2396|10422|800|2099|3|9|23|3|2099|800|10422|Wednesday|2099Q3|N|N|N|2487948|2488190|2487605|2487879|N|N|N|N|N| +2487971|AAAAAAAADKGPFCAA|2099-09-24|2396|10422|800|2099|4|9|24|3|2099|800|10422|Thursday|2099Q3|N|N|N|2487948|2488190|2487606|2487880|N|N|N|N|N| +2487972|AAAAAAAAEKGPFCAA|2099-09-25|2396|10422|800|2099|5|9|25|3|2099|800|10422|Friday|2099Q3|N|Y|N|2487948|2488190|2487607|2487881|N|N|N|N|N| +2487973|AAAAAAAAFKGPFCAA|2099-09-26|2396|10422|800|2099|6|9|26|3|2099|800|10422|Saturday|2099Q3|N|Y|N|2487948|2488190|2487608|2487882|N|N|N|N|N| +2487974|AAAAAAAAGKGPFCAA|2099-09-27|2396|10422|800|2099|0|9|27|3|2099|800|10422|Sunday|2099Q3|N|N|N|2487948|2488190|2487609|2487883|N|N|N|N|N| +2487975|AAAAAAAAHKGPFCAA|2099-09-28|2396|10422|800|2099|1|9|28|3|2099|800|10422|Monday|2099Q3|N|N|N|2487948|2488190|2487610|2487884|N|N|N|N|N| +2487976|AAAAAAAAIKGPFCAA|2099-09-29|2396|10423|800|2099|2|9|29|3|2099|800|10423|Tuesday|2099Q3|N|N|N|2487948|2488190|2487611|2487885|N|N|N|N|N| +2487977|AAAAAAAAJKGPFCAA|2099-09-30|2396|10423|800|2099|3|9|30|3|2099|800|10423|Wednesday|2099Q3|N|N|N|2487948|2488190|2487612|2487886|N|N|N|N|N| +2487978|AAAAAAAAKKGPFCAA|2099-10-01|2397|10423|800|2099|4|10|1|3|2099|800|10423|Thursday|2099Q3|N|N|N|2487978|2488250|2487613|2487886|N|N|N|N|N| +2487979|AAAAAAAALKGPFCAA|2099-10-02|2397|10423|800|2099|5|10|2|4|2099|800|10423|Friday|2099Q4|N|Y|N|2487978|2488250|2487614|2487887|N|N|N|N|N| +2487980|AAAAAAAAMKGPFCAA|2099-10-03|2397|10423|800|2099|6|10|3|4|2099|800|10423|Saturday|2099Q4|N|Y|N|2487978|2488250|2487615|2487888|N|N|N|N|N| +2487981|AAAAAAAANKGPFCAA|2099-10-04|2397|10423|800|2099|0|10|4|4|2099|800|10423|Sunday|2099Q4|N|N|N|2487978|2488250|2487616|2487889|N|N|N|N|N| +2487982|AAAAAAAAOKGPFCAA|2099-10-05|2397|10423|800|2099|1|10|5|4|2099|800|10423|Monday|2099Q4|N|N|N|2487978|2488250|2487617|2487890|N|N|N|N|N| +2487983|AAAAAAAAPKGPFCAA|2099-10-06|2397|10424|800|2099|2|10|6|4|2099|800|10424|Tuesday|2099Q4|N|N|N|2487978|2488250|2487618|2487891|N|N|N|N|N| +2487984|AAAAAAAAALGPFCAA|2099-10-07|2397|10424|800|2099|3|10|7|4|2099|800|10424|Wednesday|2099Q4|N|N|N|2487978|2488250|2487619|2487892|N|N|N|N|N| +2487985|AAAAAAAABLGPFCAA|2099-10-08|2397|10424|800|2099|4|10|8|4|2099|800|10424|Thursday|2099Q4|N|N|N|2487978|2488250|2487620|2487893|N|N|N|N|N| +2487986|AAAAAAAACLGPFCAA|2099-10-09|2397|10424|800|2099|5|10|9|4|2099|800|10424|Friday|2099Q4|N|Y|N|2487978|2488250|2487621|2487894|N|N|N|N|N| +2487987|AAAAAAAADLGPFCAA|2099-10-10|2397|10424|800|2099|6|10|10|4|2099|800|10424|Saturday|2099Q4|N|Y|N|2487978|2488250|2487622|2487895|N|N|N|N|N| +2487988|AAAAAAAAELGPFCAA|2099-10-11|2397|10424|800|2099|0|10|11|4|2099|800|10424|Sunday|2099Q4|N|N|N|2487978|2488250|2487623|2487896|N|N|N|N|N| +2487989|AAAAAAAAFLGPFCAA|2099-10-12|2397|10424|800|2099|1|10|12|4|2099|800|10424|Monday|2099Q4|N|N|N|2487978|2488250|2487624|2487897|N|N|N|N|N| +2487990|AAAAAAAAGLGPFCAA|2099-10-13|2397|10425|800|2099|2|10|13|4|2099|800|10425|Tuesday|2099Q4|N|N|N|2487978|2488250|2487625|2487898|N|N|N|N|N| +2487991|AAAAAAAAHLGPFCAA|2099-10-14|2397|10425|800|2099|3|10|14|4|2099|800|10425|Wednesday|2099Q4|N|N|N|2487978|2488250|2487626|2487899|N|N|N|N|N| +2487992|AAAAAAAAILGPFCAA|2099-10-15|2397|10425|800|2099|4|10|15|4|2099|800|10425|Thursday|2099Q4|N|N|N|2487978|2488250|2487627|2487900|N|N|N|N|N| +2487993|AAAAAAAAJLGPFCAA|2099-10-16|2397|10425|800|2099|5|10|16|4|2099|800|10425|Friday|2099Q4|N|Y|N|2487978|2488250|2487628|2487901|N|N|N|N|N| +2487994|AAAAAAAAKLGPFCAA|2099-10-17|2397|10425|800|2099|6|10|17|4|2099|800|10425|Saturday|2099Q4|N|Y|N|2487978|2488250|2487629|2487902|N|N|N|N|N| +2487995|AAAAAAAALLGPFCAA|2099-10-18|2397|10425|800|2099|0|10|18|4|2099|800|10425|Sunday|2099Q4|N|N|N|2487978|2488250|2487630|2487903|N|N|N|N|N| +2487996|AAAAAAAAMLGPFCAA|2099-10-19|2397|10425|800|2099|1|10|19|4|2099|800|10425|Monday|2099Q4|N|N|N|2487978|2488250|2487631|2487904|N|N|N|N|N| +2487997|AAAAAAAANLGPFCAA|2099-10-20|2397|10426|800|2099|2|10|20|4|2099|800|10426|Tuesday|2099Q4|N|N|N|2487978|2488250|2487632|2487905|N|N|N|N|N| +2487998|AAAAAAAAOLGPFCAA|2099-10-21|2397|10426|800|2099|3|10|21|4|2099|800|10426|Wednesday|2099Q4|N|N|N|2487978|2488250|2487633|2487906|N|N|N|N|N| +2487999|AAAAAAAAPLGPFCAA|2099-10-22|2397|10426|800|2099|4|10|22|4|2099|800|10426|Thursday|2099Q4|N|N|N|2487978|2488250|2487634|2487907|N|N|N|N|N| +2488000|AAAAAAAAAMGPFCAA|2099-10-23|2397|10426|800|2099|5|10|23|4|2099|800|10426|Friday|2099Q4|N|Y|N|2487978|2488250|2487635|2487908|N|N|N|N|N| +2488001|AAAAAAAABMGPFCAA|2099-10-24|2397|10426|800|2099|6|10|24|4|2099|800|10426|Saturday|2099Q4|N|Y|N|2487978|2488250|2487636|2487909|N|N|N|N|N| +2488002|AAAAAAAACMGPFCAA|2099-10-25|2397|10426|800|2099|0|10|25|4|2099|800|10426|Sunday|2099Q4|N|N|N|2487978|2488250|2487637|2487910|N|N|N|N|N| +2488003|AAAAAAAADMGPFCAA|2099-10-26|2397|10426|800|2099|1|10|26|4|2099|800|10426|Monday|2099Q4|N|N|N|2487978|2488250|2487638|2487911|N|N|N|N|N| +2488004|AAAAAAAAEMGPFCAA|2099-10-27|2397|10427|800|2099|2|10|27|4|2099|800|10427|Tuesday|2099Q4|N|N|N|2487978|2488250|2487639|2487912|N|N|N|N|N| +2488005|AAAAAAAAFMGPFCAA|2099-10-28|2397|10427|800|2099|3|10|28|4|2099|800|10427|Wednesday|2099Q4|N|N|N|2487978|2488250|2487640|2487913|N|N|N|N|N| +2488006|AAAAAAAAGMGPFCAA|2099-10-29|2397|10427|800|2099|4|10|29|4|2099|800|10427|Thursday|2099Q4|N|N|N|2487978|2488250|2487641|2487914|N|N|N|N|N| +2488007|AAAAAAAAHMGPFCAA|2099-10-30|2397|10427|800|2099|5|10|30|4|2099|800|10427|Friday|2099Q4|N|Y|N|2487978|2488250|2487642|2487915|N|N|N|N|N| +2488008|AAAAAAAAIMGPFCAA|2099-10-31|2397|10427|800|2099|6|10|31|4|2099|800|10427|Saturday|2099Q4|N|Y|N|2487978|2488250|2487643|2487916|N|N|N|N|N| +2488009|AAAAAAAAJMGPFCAA|2099-11-01|2398|10427|800|2099|0|11|1|4|2099|800|10427|Sunday|2099Q4|N|N|N|2488009|2488312|2487644|2487917|N|N|N|N|N| +2488010|AAAAAAAAKMGPFCAA|2099-11-02|2398|10427|800|2099|1|11|2|4|2099|800|10427|Monday|2099Q4|N|N|N|2488009|2488312|2487645|2487918|N|N|N|N|N| +2488011|AAAAAAAALMGPFCAA|2099-11-03|2398|10428|800|2099|2|11|3|4|2099|800|10428|Tuesday|2099Q4|N|N|N|2488009|2488312|2487646|2487919|N|N|N|N|N| +2488012|AAAAAAAAMMGPFCAA|2099-11-04|2398|10428|800|2099|3|11|4|4|2099|800|10428|Wednesday|2099Q4|N|N|N|2488009|2488312|2487647|2487920|N|N|N|N|N| +2488013|AAAAAAAANMGPFCAA|2099-11-05|2398|10428|800|2099|4|11|5|4|2099|800|10428|Thursday|2099Q4|N|N|N|2488009|2488312|2487648|2487921|N|N|N|N|N| +2488014|AAAAAAAAOMGPFCAA|2099-11-06|2398|10428|800|2099|5|11|6|4|2099|800|10428|Friday|2099Q4|N|Y|N|2488009|2488312|2487649|2487922|N|N|N|N|N| +2488015|AAAAAAAAPMGPFCAA|2099-11-07|2398|10428|800|2099|6|11|7|4|2099|800|10428|Saturday|2099Q4|N|Y|N|2488009|2488312|2487650|2487923|N|N|N|N|N| +2488016|AAAAAAAAANGPFCAA|2099-11-08|2398|10428|800|2099|0|11|8|4|2099|800|10428|Sunday|2099Q4|N|N|N|2488009|2488312|2487651|2487924|N|N|N|N|N| +2488017|AAAAAAAABNGPFCAA|2099-11-09|2398|10428|800|2099|1|11|9|4|2099|800|10428|Monday|2099Q4|N|N|N|2488009|2488312|2487652|2487925|N|N|N|N|N| +2488018|AAAAAAAACNGPFCAA|2099-11-10|2398|10429|800|2099|2|11|10|4|2099|800|10429|Tuesday|2099Q4|N|N|N|2488009|2488312|2487653|2487926|N|N|N|N|N| +2488019|AAAAAAAADNGPFCAA|2099-11-11|2398|10429|800|2099|3|11|11|4|2099|800|10429|Wednesday|2099Q4|N|N|N|2488009|2488312|2487654|2487927|N|N|N|N|N| +2488020|AAAAAAAAENGPFCAA|2099-11-12|2398|10429|800|2099|4|11|12|4|2099|800|10429|Thursday|2099Q4|N|N|N|2488009|2488312|2487655|2487928|N|N|N|N|N| +2488021|AAAAAAAAFNGPFCAA|2099-11-13|2398|10429|800|2099|5|11|13|4|2099|800|10429|Friday|2099Q4|N|Y|N|2488009|2488312|2487656|2487929|N|N|N|N|N| +2488022|AAAAAAAAGNGPFCAA|2099-11-14|2398|10429|800|2099|6|11|14|4|2099|800|10429|Saturday|2099Q4|N|Y|N|2488009|2488312|2487657|2487930|N|N|N|N|N| +2488023|AAAAAAAAHNGPFCAA|2099-11-15|2398|10429|800|2099|0|11|15|4|2099|800|10429|Sunday|2099Q4|N|N|N|2488009|2488312|2487658|2487931|N|N|N|N|N| +2488024|AAAAAAAAINGPFCAA|2099-11-16|2398|10429|800|2099|1|11|16|4|2099|800|10429|Monday|2099Q4|N|N|N|2488009|2488312|2487659|2487932|N|N|N|N|N| +2488025|AAAAAAAAJNGPFCAA|2099-11-17|2398|10430|800|2099|2|11|17|4|2099|800|10430|Tuesday|2099Q4|N|N|N|2488009|2488312|2487660|2487933|N|N|N|N|N| +2488026|AAAAAAAAKNGPFCAA|2099-11-18|2398|10430|800|2099|3|11|18|4|2099|800|10430|Wednesday|2099Q4|N|N|N|2488009|2488312|2487661|2487934|N|N|N|N|N| +2488027|AAAAAAAALNGPFCAA|2099-11-19|2398|10430|800|2099|4|11|19|4|2099|800|10430|Thursday|2099Q4|N|N|N|2488009|2488312|2487662|2487935|N|N|N|N|N| +2488028|AAAAAAAAMNGPFCAA|2099-11-20|2398|10430|800|2099|5|11|20|4|2099|800|10430|Friday|2099Q4|N|Y|N|2488009|2488312|2487663|2487936|N|N|N|N|N| +2488029|AAAAAAAANNGPFCAA|2099-11-21|2398|10430|800|2099|6|11|21|4|2099|800|10430|Saturday|2099Q4|N|Y|N|2488009|2488312|2487664|2487937|N|N|N|N|N| +2488030|AAAAAAAAONGPFCAA|2099-11-22|2398|10430|800|2099|0|11|22|4|2099|800|10430|Sunday|2099Q4|N|N|N|2488009|2488312|2487665|2487938|N|N|N|N|N| +2488031|AAAAAAAAPNGPFCAA|2099-11-23|2398|10430|800|2099|1|11|23|4|2099|800|10430|Monday|2099Q4|N|N|N|2488009|2488312|2487666|2487939|N|N|N|N|N| +2488032|AAAAAAAAAOGPFCAA|2099-11-24|2398|10431|800|2099|2|11|24|4|2099|800|10431|Tuesday|2099Q4|N|N|N|2488009|2488312|2487667|2487940|N|N|N|N|N| +2488033|AAAAAAAABOGPFCAA|2099-11-25|2398|10431|800|2099|3|11|25|4|2099|800|10431|Wednesday|2099Q4|N|N|N|2488009|2488312|2487668|2487941|N|N|N|N|N| +2488034|AAAAAAAACOGPFCAA|2099-11-26|2398|10431|800|2099|4|11|26|4|2099|800|10431|Thursday|2099Q4|N|N|N|2488009|2488312|2487669|2487942|N|N|N|N|N| +2488035|AAAAAAAADOGPFCAA|2099-11-27|2398|10431|800|2099|5|11|27|4|2099|800|10431|Friday|2099Q4|N|Y|N|2488009|2488312|2487670|2487943|N|N|N|N|N| +2488036|AAAAAAAAEOGPFCAA|2099-11-28|2398|10431|800|2099|6|11|28|4|2099|800|10431|Saturday|2099Q4|N|Y|N|2488009|2488312|2487671|2487944|N|N|N|N|N| +2488037|AAAAAAAAFOGPFCAA|2099-11-29|2398|10431|800|2099|0|11|29|4|2099|800|10431|Sunday|2099Q4|N|N|N|2488009|2488312|2487672|2487945|N|N|N|N|N| +2488038|AAAAAAAAGOGPFCAA|2099-11-30|2398|10431|800|2099|1|11|30|4|2099|800|10431|Monday|2099Q4|N|N|N|2488009|2488312|2487673|2487946|N|N|N|N|N| +2488039|AAAAAAAAHOGPFCAA|2099-12-01|2399|10432|801|2099|2|12|1|4|2099|801|10432|Tuesday|2099Q4|N|N|N|2488039|2488372|2487674|2487947|N|N|N|N|N| +2488040|AAAAAAAAIOGPFCAA|2099-12-02|2399|10432|801|2099|3|12|2|4|2099|801|10432|Wednesday|2099Q4|N|N|N|2488039|2488372|2487675|2487948|N|N|N|N|N| +2488041|AAAAAAAAJOGPFCAA|2099-12-03|2399|10432|801|2099|4|12|3|4|2099|801|10432|Thursday|2099Q4|N|N|N|2488039|2488372|2487676|2487949|N|N|N|N|N| +2488042|AAAAAAAAKOGPFCAA|2099-12-04|2399|10432|801|2099|5|12|4|4|2099|801|10432|Friday|2099Q4|N|Y|N|2488039|2488372|2487677|2487950|N|N|N|N|N| +2488043|AAAAAAAALOGPFCAA|2099-12-05|2399|10432|801|2099|6|12|5|4|2099|801|10432|Saturday|2099Q4|N|Y|N|2488039|2488372|2487678|2487951|N|N|N|N|N| +2488044|AAAAAAAAMOGPFCAA|2099-12-06|2399|10432|801|2099|0|12|6|4|2099|801|10432|Sunday|2099Q4|N|N|N|2488039|2488372|2487679|2487952|N|N|N|N|N| +2488045|AAAAAAAANOGPFCAA|2099-12-07|2399|10432|801|2099|1|12|7|4|2099|801|10432|Monday|2099Q4|N|N|N|2488039|2488372|2487680|2487953|N|N|N|N|N| +2488046|AAAAAAAAOOGPFCAA|2099-12-08|2399|10433|801|2099|2|12|8|4|2099|801|10433|Tuesday|2099Q4|N|N|N|2488039|2488372|2487681|2487954|N|N|N|N|N| +2488047|AAAAAAAAPOGPFCAA|2099-12-09|2399|10433|801|2099|3|12|9|4|2099|801|10433|Wednesday|2099Q4|N|N|N|2488039|2488372|2487682|2487955|N|N|N|N|N| +2488048|AAAAAAAAAPGPFCAA|2099-12-10|2399|10433|801|2099|4|12|10|4|2099|801|10433|Thursday|2099Q4|N|N|N|2488039|2488372|2487683|2487956|N|N|N|N|N| +2488049|AAAAAAAABPGPFCAA|2099-12-11|2399|10433|801|2099|5|12|11|4|2099|801|10433|Friday|2099Q4|N|Y|N|2488039|2488372|2487684|2487957|N|N|N|N|N| +2488050|AAAAAAAACPGPFCAA|2099-12-12|2399|10433|801|2099|6|12|12|4|2099|801|10433|Saturday|2099Q4|N|Y|N|2488039|2488372|2487685|2487958|N|N|N|N|N| +2488051|AAAAAAAADPGPFCAA|2099-12-13|2399|10433|801|2099|0|12|13|4|2099|801|10433|Sunday|2099Q4|N|N|N|2488039|2488372|2487686|2487959|N|N|N|N|N| +2488052|AAAAAAAAEPGPFCAA|2099-12-14|2399|10433|801|2099|1|12|14|4|2099|801|10433|Monday|2099Q4|N|N|N|2488039|2488372|2487687|2487960|N|N|N|N|N| +2488053|AAAAAAAAFPGPFCAA|2099-12-15|2399|10434|801|2099|2|12|15|4|2099|801|10434|Tuesday|2099Q4|N|N|N|2488039|2488372|2487688|2487961|N|N|N|N|N| +2488054|AAAAAAAAGPGPFCAA|2099-12-16|2399|10434|801|2099|3|12|16|4|2099|801|10434|Wednesday|2099Q4|N|N|N|2488039|2488372|2487689|2487962|N|N|N|N|N| +2488055|AAAAAAAAHPGPFCAA|2099-12-17|2399|10434|801|2099|4|12|17|4|2099|801|10434|Thursday|2099Q4|N|N|N|2488039|2488372|2487690|2487963|N|N|N|N|N| +2488056|AAAAAAAAIPGPFCAA|2099-12-18|2399|10434|801|2099|5|12|18|4|2099|801|10434|Friday|2099Q4|N|Y|N|2488039|2488372|2487691|2487964|N|N|N|N|N| +2488057|AAAAAAAAJPGPFCAA|2099-12-19|2399|10434|801|2099|6|12|19|4|2099|801|10434|Saturday|2099Q4|N|Y|N|2488039|2488372|2487692|2487965|N|N|N|N|N| +2488058|AAAAAAAAKPGPFCAA|2099-12-20|2399|10434|801|2099|0|12|20|4|2099|801|10434|Sunday|2099Q4|N|N|N|2488039|2488372|2487693|2487966|N|N|N|N|N| +2488059|AAAAAAAALPGPFCAA|2099-12-21|2399|10434|801|2099|1|12|21|4|2099|801|10434|Monday|2099Q4|N|N|N|2488039|2488372|2487694|2487967|N|N|N|N|N| +2488060|AAAAAAAAMPGPFCAA|2099-12-22|2399|10435|801|2099|2|12|22|4|2099|801|10435|Tuesday|2099Q4|N|N|N|2488039|2488372|2487695|2487968|N|N|N|N|N| +2488061|AAAAAAAANPGPFCAA|2099-12-23|2399|10435|801|2099|3|12|23|4|2099|801|10435|Wednesday|2099Q4|N|N|N|2488039|2488372|2487696|2487969|N|N|N|N|N| +2488062|AAAAAAAAOPGPFCAA|2099-12-24|2399|10435|801|2099|4|12|24|4|2099|801|10435|Thursday|2099Q4|N|N|N|2488039|2488372|2487697|2487970|N|N|N|N|N| +2488063|AAAAAAAAPPGPFCAA|2099-12-25|2399|10435|801|2099|5|12|25|4|2099|801|10435|Friday|2099Q4|N|Y|N|2488039|2488372|2487698|2487971|N|N|N|N|N| +2488064|AAAAAAAAAAHPFCAA|2099-12-26|2399|10435|801|2099|6|12|26|4|2099|801|10435|Saturday|2099Q4|Y|Y|N|2488039|2488372|2487699|2487972|N|N|N|N|N| +2488065|AAAAAAAABAHPFCAA|2099-12-27|2399|10435|801|2099|0|12|27|4|2099|801|10435|Sunday|2099Q4|N|N|Y|2488039|2488372|2487700|2487973|N|N|N|N|N| +2488066|AAAAAAAACAHPFCAA|2099-12-28|2399|10435|801|2099|1|12|28|4|2099|801|10435|Monday|2099Q4|N|N|N|2488039|2488372|2487701|2487974|N|N|N|N|N| +2488067|AAAAAAAADAHPFCAA|2099-12-29|2399|10436|801|2099|2|12|29|4|2099|801|10436|Tuesday|2099Q4|N|N|N|2488039|2488372|2487702|2487975|N|N|N|N|N| +2488068|AAAAAAAAEAHPFCAA|2099-12-30|2399|10436|801|2099|3|12|30|4|2099|801|10436|Wednesday|2099Q4|N|N|N|2488039|2488372|2487703|2487976|N|N|N|N|N| +2488069|AAAAAAAAFAHPFCAA|2099-12-31|2399|10436|801|2099|4|12|31|4|2099|801|10436|Thursday|2099Q4|N|N|N|2488039|2488372|2487704|2487977|N|N|N|N|N| +2488070|AAAAAAAAGAHPFCAA|2100-01-01|2400|10436|801|2100|4|1|1|1|2100|801|10436|Thursday|2100Q1|Y|N|N|2488070|2488069|2487705|2487978|N|N|N|N|N| diff --git a/spark-connector/spark-common/src/test/resources/data/tpcds/dbgen_version/dbgen_version.dat b/spark-connector/spark-common/src/test/resources/data/tpcds/dbgen_version/dbgen_version.dat new file mode 100644 index 00000000000..3b421123d10 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/data/tpcds/dbgen_version/dbgen_version.dat @@ -0,0 +1 @@ +2.13.0|2024-06-25|17:25:35|-SCALE 1 -DIR /tmp/tpcds | diff --git a/spark-connector/spark-common/src/test/resources/data/tpcds/household_demographics/household_demographics.dat b/spark-connector/spark-common/src/test/resources/data/tpcds/household_demographics/household_demographics.dat new file mode 100644 index 00000000000..9f07aba0ddf --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/data/tpcds/household_demographics/household_demographics.dat @@ -0,0 +1,7200 @@ +1|2|0-500|0|0| +2|3|0-500|0|0| +3|4|0-500|0|0| +4|5|0-500|0|0| +5|6|0-500|0|0| +6|7|0-500|0|0| +7|8|0-500|0|0| +8|9|0-500|0|0| +9|10|0-500|0|0| +10|11|0-500|0|0| +11|12|0-500|0|0| +12|13|0-500|0|0| +13|14|0-500|0|0| +14|15|0-500|0|0| +15|16|0-500|0|0| +16|17|0-500|0|0| +17|18|0-500|0|0| +18|19|0-500|0|0| +19|20|0-500|0|0| +20|1|501-1000|0|0| +21|2|501-1000|0|0| +22|3|501-1000|0|0| +23|4|501-1000|0|0| +24|5|501-1000|0|0| +25|6|501-1000|0|0| +26|7|501-1000|0|0| +27|8|501-1000|0|0| +28|9|501-1000|0|0| +29|10|501-1000|0|0| +30|11|501-1000|0|0| +31|12|501-1000|0|0| +32|13|501-1000|0|0| +33|14|501-1000|0|0| +34|15|501-1000|0|0| +35|16|501-1000|0|0| +36|17|501-1000|0|0| +37|18|501-1000|0|0| +38|19|501-1000|0|0| +39|20|501-1000|0|0| +40|1|1001-5000|0|0| +41|2|1001-5000|0|0| +42|3|1001-5000|0|0| +43|4|1001-5000|0|0| +44|5|1001-5000|0|0| +45|6|1001-5000|0|0| +46|7|1001-5000|0|0| +47|8|1001-5000|0|0| +48|9|1001-5000|0|0| +49|10|1001-5000|0|0| +50|11|1001-5000|0|0| +51|12|1001-5000|0|0| +52|13|1001-5000|0|0| +53|14|1001-5000|0|0| +54|15|1001-5000|0|0| +55|16|1001-5000|0|0| +56|17|1001-5000|0|0| +57|18|1001-5000|0|0| +58|19|1001-5000|0|0| +59|20|1001-5000|0|0| +60|1|5001-10000|0|0| +61|2|5001-10000|0|0| +62|3|5001-10000|0|0| +63|4|5001-10000|0|0| +64|5|5001-10000|0|0| +65|6|5001-10000|0|0| +66|7|5001-10000|0|0| +67|8|5001-10000|0|0| +68|9|5001-10000|0|0| +69|10|5001-10000|0|0| +70|11|5001-10000|0|0| +71|12|5001-10000|0|0| +72|13|5001-10000|0|0| +73|14|5001-10000|0|0| +74|15|5001-10000|0|0| +75|16|5001-10000|0|0| +76|17|5001-10000|0|0| +77|18|5001-10000|0|0| +78|19|5001-10000|0|0| +79|20|5001-10000|0|0| +80|1|>10000|0|0| +81|2|>10000|0|0| +82|3|>10000|0|0| +83|4|>10000|0|0| +84|5|>10000|0|0| +85|6|>10000|0|0| +86|7|>10000|0|0| +87|8|>10000|0|0| +88|9|>10000|0|0| +89|10|>10000|0|0| +90|11|>10000|0|0| +91|12|>10000|0|0| +92|13|>10000|0|0| +93|14|>10000|0|0| +94|15|>10000|0|0| +95|16|>10000|0|0| +96|17|>10000|0|0| +97|18|>10000|0|0| +98|19|>10000|0|0| +99|20|>10000|0|0| +100|1|Unknown|0|0| +101|2|Unknown|0|0| +102|3|Unknown|0|0| +103|4|Unknown|0|0| +104|5|Unknown|0|0| +105|6|Unknown|0|0| +106|7|Unknown|0|0| +107|8|Unknown|0|0| +108|9|Unknown|0|0| +109|10|Unknown|0|0| +110|11|Unknown|0|0| +111|12|Unknown|0|0| +112|13|Unknown|0|0| +113|14|Unknown|0|0| +114|15|Unknown|0|0| +115|16|Unknown|0|0| +116|17|Unknown|0|0| +117|18|Unknown|0|0| +118|19|Unknown|0|0| +119|20|Unknown|0|0| +120|1|0-500|1|0| +121|2|0-500|1|0| +122|3|0-500|1|0| +123|4|0-500|1|0| +124|5|0-500|1|0| +125|6|0-500|1|0| +126|7|0-500|1|0| +127|8|0-500|1|0| +128|9|0-500|1|0| +129|10|0-500|1|0| +130|11|0-500|1|0| +131|12|0-500|1|0| +132|13|0-500|1|0| +133|14|0-500|1|0| +134|15|0-500|1|0| +135|16|0-500|1|0| +136|17|0-500|1|0| +137|18|0-500|1|0| +138|19|0-500|1|0| +139|20|0-500|1|0| +140|1|501-1000|1|0| +141|2|501-1000|1|0| +142|3|501-1000|1|0| +143|4|501-1000|1|0| +144|5|501-1000|1|0| +145|6|501-1000|1|0| +146|7|501-1000|1|0| +147|8|501-1000|1|0| +148|9|501-1000|1|0| +149|10|501-1000|1|0| +150|11|501-1000|1|0| +151|12|501-1000|1|0| +152|13|501-1000|1|0| +153|14|501-1000|1|0| +154|15|501-1000|1|0| +155|16|501-1000|1|0| +156|17|501-1000|1|0| +157|18|501-1000|1|0| +158|19|501-1000|1|0| +159|20|501-1000|1|0| +160|1|1001-5000|1|0| +161|2|1001-5000|1|0| +162|3|1001-5000|1|0| +163|4|1001-5000|1|0| +164|5|1001-5000|1|0| +165|6|1001-5000|1|0| +166|7|1001-5000|1|0| +167|8|1001-5000|1|0| +168|9|1001-5000|1|0| +169|10|1001-5000|1|0| +170|11|1001-5000|1|0| +171|12|1001-5000|1|0| +172|13|1001-5000|1|0| +173|14|1001-5000|1|0| +174|15|1001-5000|1|0| +175|16|1001-5000|1|0| +176|17|1001-5000|1|0| +177|18|1001-5000|1|0| +178|19|1001-5000|1|0| +179|20|1001-5000|1|0| +180|1|5001-10000|1|0| +181|2|5001-10000|1|0| +182|3|5001-10000|1|0| +183|4|5001-10000|1|0| +184|5|5001-10000|1|0| +185|6|5001-10000|1|0| +186|7|5001-10000|1|0| +187|8|5001-10000|1|0| +188|9|5001-10000|1|0| +189|10|5001-10000|1|0| +190|11|5001-10000|1|0| +191|12|5001-10000|1|0| +192|13|5001-10000|1|0| +193|14|5001-10000|1|0| +194|15|5001-10000|1|0| +195|16|5001-10000|1|0| +196|17|5001-10000|1|0| +197|18|5001-10000|1|0| +198|19|5001-10000|1|0| +199|20|5001-10000|1|0| +200|1|>10000|1|0| +201|2|>10000|1|0| +202|3|>10000|1|0| +203|4|>10000|1|0| +204|5|>10000|1|0| +205|6|>10000|1|0| +206|7|>10000|1|0| +207|8|>10000|1|0| +208|9|>10000|1|0| +209|10|>10000|1|0| +210|11|>10000|1|0| +211|12|>10000|1|0| +212|13|>10000|1|0| +213|14|>10000|1|0| +214|15|>10000|1|0| +215|16|>10000|1|0| +216|17|>10000|1|0| +217|18|>10000|1|0| +218|19|>10000|1|0| +219|20|>10000|1|0| +220|1|Unknown|1|0| +221|2|Unknown|1|0| +222|3|Unknown|1|0| +223|4|Unknown|1|0| +224|5|Unknown|1|0| +225|6|Unknown|1|0| +226|7|Unknown|1|0| +227|8|Unknown|1|0| +228|9|Unknown|1|0| +229|10|Unknown|1|0| +230|11|Unknown|1|0| +231|12|Unknown|1|0| +232|13|Unknown|1|0| +233|14|Unknown|1|0| +234|15|Unknown|1|0| +235|16|Unknown|1|0| +236|17|Unknown|1|0| +237|18|Unknown|1|0| +238|19|Unknown|1|0| +239|20|Unknown|1|0| +240|1|0-500|2|0| +241|2|0-500|2|0| +242|3|0-500|2|0| +243|4|0-500|2|0| +244|5|0-500|2|0| +245|6|0-500|2|0| +246|7|0-500|2|0| +247|8|0-500|2|0| +248|9|0-500|2|0| +249|10|0-500|2|0| +250|11|0-500|2|0| +251|12|0-500|2|0| +252|13|0-500|2|0| +253|14|0-500|2|0| +254|15|0-500|2|0| +255|16|0-500|2|0| +256|17|0-500|2|0| +257|18|0-500|2|0| +258|19|0-500|2|0| +259|20|0-500|2|0| +260|1|501-1000|2|0| +261|2|501-1000|2|0| +262|3|501-1000|2|0| +263|4|501-1000|2|0| +264|5|501-1000|2|0| +265|6|501-1000|2|0| +266|7|501-1000|2|0| +267|8|501-1000|2|0| +268|9|501-1000|2|0| +269|10|501-1000|2|0| +270|11|501-1000|2|0| +271|12|501-1000|2|0| +272|13|501-1000|2|0| +273|14|501-1000|2|0| +274|15|501-1000|2|0| +275|16|501-1000|2|0| +276|17|501-1000|2|0| +277|18|501-1000|2|0| +278|19|501-1000|2|0| +279|20|501-1000|2|0| +280|1|1001-5000|2|0| +281|2|1001-5000|2|0| +282|3|1001-5000|2|0| +283|4|1001-5000|2|0| +284|5|1001-5000|2|0| +285|6|1001-5000|2|0| +286|7|1001-5000|2|0| +287|8|1001-5000|2|0| +288|9|1001-5000|2|0| +289|10|1001-5000|2|0| +290|11|1001-5000|2|0| +291|12|1001-5000|2|0| +292|13|1001-5000|2|0| +293|14|1001-5000|2|0| +294|15|1001-5000|2|0| +295|16|1001-5000|2|0| +296|17|1001-5000|2|0| +297|18|1001-5000|2|0| +298|19|1001-5000|2|0| +299|20|1001-5000|2|0| +300|1|5001-10000|2|0| +301|2|5001-10000|2|0| +302|3|5001-10000|2|0| +303|4|5001-10000|2|0| +304|5|5001-10000|2|0| +305|6|5001-10000|2|0| +306|7|5001-10000|2|0| +307|8|5001-10000|2|0| +308|9|5001-10000|2|0| +309|10|5001-10000|2|0| +310|11|5001-10000|2|0| +311|12|5001-10000|2|0| +312|13|5001-10000|2|0| +313|14|5001-10000|2|0| +314|15|5001-10000|2|0| +315|16|5001-10000|2|0| +316|17|5001-10000|2|0| +317|18|5001-10000|2|0| +318|19|5001-10000|2|0| +319|20|5001-10000|2|0| +320|1|>10000|2|0| +321|2|>10000|2|0| +322|3|>10000|2|0| +323|4|>10000|2|0| +324|5|>10000|2|0| +325|6|>10000|2|0| +326|7|>10000|2|0| +327|8|>10000|2|0| +328|9|>10000|2|0| +329|10|>10000|2|0| +330|11|>10000|2|0| +331|12|>10000|2|0| +332|13|>10000|2|0| +333|14|>10000|2|0| +334|15|>10000|2|0| +335|16|>10000|2|0| +336|17|>10000|2|0| +337|18|>10000|2|0| +338|19|>10000|2|0| +339|20|>10000|2|0| +340|1|Unknown|2|0| +341|2|Unknown|2|0| +342|3|Unknown|2|0| +343|4|Unknown|2|0| +344|5|Unknown|2|0| +345|6|Unknown|2|0| +346|7|Unknown|2|0| +347|8|Unknown|2|0| +348|9|Unknown|2|0| +349|10|Unknown|2|0| +350|11|Unknown|2|0| +351|12|Unknown|2|0| +352|13|Unknown|2|0| +353|14|Unknown|2|0| +354|15|Unknown|2|0| +355|16|Unknown|2|0| +356|17|Unknown|2|0| +357|18|Unknown|2|0| +358|19|Unknown|2|0| +359|20|Unknown|2|0| +360|1|0-500|3|0| +361|2|0-500|3|0| +362|3|0-500|3|0| +363|4|0-500|3|0| +364|5|0-500|3|0| +365|6|0-500|3|0| +366|7|0-500|3|0| +367|8|0-500|3|0| +368|9|0-500|3|0| +369|10|0-500|3|0| +370|11|0-500|3|0| +371|12|0-500|3|0| +372|13|0-500|3|0| +373|14|0-500|3|0| +374|15|0-500|3|0| +375|16|0-500|3|0| +376|17|0-500|3|0| +377|18|0-500|3|0| +378|19|0-500|3|0| +379|20|0-500|3|0| +380|1|501-1000|3|0| +381|2|501-1000|3|0| +382|3|501-1000|3|0| +383|4|501-1000|3|0| +384|5|501-1000|3|0| +385|6|501-1000|3|0| +386|7|501-1000|3|0| +387|8|501-1000|3|0| +388|9|501-1000|3|0| +389|10|501-1000|3|0| +390|11|501-1000|3|0| +391|12|501-1000|3|0| +392|13|501-1000|3|0| +393|14|501-1000|3|0| +394|15|501-1000|3|0| +395|16|501-1000|3|0| +396|17|501-1000|3|0| +397|18|501-1000|3|0| +398|19|501-1000|3|0| +399|20|501-1000|3|0| +400|1|1001-5000|3|0| +401|2|1001-5000|3|0| +402|3|1001-5000|3|0| +403|4|1001-5000|3|0| +404|5|1001-5000|3|0| +405|6|1001-5000|3|0| +406|7|1001-5000|3|0| +407|8|1001-5000|3|0| +408|9|1001-5000|3|0| +409|10|1001-5000|3|0| +410|11|1001-5000|3|0| +411|12|1001-5000|3|0| +412|13|1001-5000|3|0| +413|14|1001-5000|3|0| +414|15|1001-5000|3|0| +415|16|1001-5000|3|0| +416|17|1001-5000|3|0| +417|18|1001-5000|3|0| +418|19|1001-5000|3|0| +419|20|1001-5000|3|0| +420|1|5001-10000|3|0| +421|2|5001-10000|3|0| +422|3|5001-10000|3|0| +423|4|5001-10000|3|0| +424|5|5001-10000|3|0| +425|6|5001-10000|3|0| +426|7|5001-10000|3|0| +427|8|5001-10000|3|0| +428|9|5001-10000|3|0| +429|10|5001-10000|3|0| +430|11|5001-10000|3|0| +431|12|5001-10000|3|0| +432|13|5001-10000|3|0| +433|14|5001-10000|3|0| +434|15|5001-10000|3|0| +435|16|5001-10000|3|0| +436|17|5001-10000|3|0| +437|18|5001-10000|3|0| +438|19|5001-10000|3|0| +439|20|5001-10000|3|0| +440|1|>10000|3|0| +441|2|>10000|3|0| +442|3|>10000|3|0| +443|4|>10000|3|0| +444|5|>10000|3|0| +445|6|>10000|3|0| +446|7|>10000|3|0| +447|8|>10000|3|0| +448|9|>10000|3|0| +449|10|>10000|3|0| +450|11|>10000|3|0| +451|12|>10000|3|0| +452|13|>10000|3|0| +453|14|>10000|3|0| +454|15|>10000|3|0| +455|16|>10000|3|0| +456|17|>10000|3|0| +457|18|>10000|3|0| +458|19|>10000|3|0| +459|20|>10000|3|0| +460|1|Unknown|3|0| +461|2|Unknown|3|0| +462|3|Unknown|3|0| +463|4|Unknown|3|0| +464|5|Unknown|3|0| +465|6|Unknown|3|0| +466|7|Unknown|3|0| +467|8|Unknown|3|0| +468|9|Unknown|3|0| +469|10|Unknown|3|0| +470|11|Unknown|3|0| +471|12|Unknown|3|0| +472|13|Unknown|3|0| +473|14|Unknown|3|0| +474|15|Unknown|3|0| +475|16|Unknown|3|0| +476|17|Unknown|3|0| +477|18|Unknown|3|0| +478|19|Unknown|3|0| +479|20|Unknown|3|0| +480|1|0-500|4|0| +481|2|0-500|4|0| +482|3|0-500|4|0| +483|4|0-500|4|0| +484|5|0-500|4|0| +485|6|0-500|4|0| +486|7|0-500|4|0| +487|8|0-500|4|0| +488|9|0-500|4|0| +489|10|0-500|4|0| +490|11|0-500|4|0| +491|12|0-500|4|0| +492|13|0-500|4|0| +493|14|0-500|4|0| +494|15|0-500|4|0| +495|16|0-500|4|0| +496|17|0-500|4|0| +497|18|0-500|4|0| +498|19|0-500|4|0| +499|20|0-500|4|0| +500|1|501-1000|4|0| +501|2|501-1000|4|0| +502|3|501-1000|4|0| +503|4|501-1000|4|0| +504|5|501-1000|4|0| +505|6|501-1000|4|0| +506|7|501-1000|4|0| +507|8|501-1000|4|0| +508|9|501-1000|4|0| +509|10|501-1000|4|0| +510|11|501-1000|4|0| +511|12|501-1000|4|0| +512|13|501-1000|4|0| +513|14|501-1000|4|0| +514|15|501-1000|4|0| +515|16|501-1000|4|0| +516|17|501-1000|4|0| +517|18|501-1000|4|0| +518|19|501-1000|4|0| +519|20|501-1000|4|0| +520|1|1001-5000|4|0| +521|2|1001-5000|4|0| +522|3|1001-5000|4|0| +523|4|1001-5000|4|0| +524|5|1001-5000|4|0| +525|6|1001-5000|4|0| +526|7|1001-5000|4|0| +527|8|1001-5000|4|0| +528|9|1001-5000|4|0| +529|10|1001-5000|4|0| +530|11|1001-5000|4|0| +531|12|1001-5000|4|0| +532|13|1001-5000|4|0| +533|14|1001-5000|4|0| +534|15|1001-5000|4|0| +535|16|1001-5000|4|0| +536|17|1001-5000|4|0| +537|18|1001-5000|4|0| +538|19|1001-5000|4|0| +539|20|1001-5000|4|0| +540|1|5001-10000|4|0| +541|2|5001-10000|4|0| +542|3|5001-10000|4|0| +543|4|5001-10000|4|0| +544|5|5001-10000|4|0| +545|6|5001-10000|4|0| +546|7|5001-10000|4|0| +547|8|5001-10000|4|0| +548|9|5001-10000|4|0| +549|10|5001-10000|4|0| +550|11|5001-10000|4|0| +551|12|5001-10000|4|0| +552|13|5001-10000|4|0| +553|14|5001-10000|4|0| +554|15|5001-10000|4|0| +555|16|5001-10000|4|0| +556|17|5001-10000|4|0| +557|18|5001-10000|4|0| +558|19|5001-10000|4|0| +559|20|5001-10000|4|0| +560|1|>10000|4|0| +561|2|>10000|4|0| +562|3|>10000|4|0| +563|4|>10000|4|0| +564|5|>10000|4|0| +565|6|>10000|4|0| +566|7|>10000|4|0| +567|8|>10000|4|0| +568|9|>10000|4|0| +569|10|>10000|4|0| +570|11|>10000|4|0| +571|12|>10000|4|0| +572|13|>10000|4|0| +573|14|>10000|4|0| +574|15|>10000|4|0| +575|16|>10000|4|0| +576|17|>10000|4|0| +577|18|>10000|4|0| +578|19|>10000|4|0| +579|20|>10000|4|0| +580|1|Unknown|4|0| +581|2|Unknown|4|0| +582|3|Unknown|4|0| +583|4|Unknown|4|0| +584|5|Unknown|4|0| +585|6|Unknown|4|0| +586|7|Unknown|4|0| +587|8|Unknown|4|0| +588|9|Unknown|4|0| +589|10|Unknown|4|0| +590|11|Unknown|4|0| +591|12|Unknown|4|0| +592|13|Unknown|4|0| +593|14|Unknown|4|0| +594|15|Unknown|4|0| +595|16|Unknown|4|0| +596|17|Unknown|4|0| +597|18|Unknown|4|0| +598|19|Unknown|4|0| +599|20|Unknown|4|0| +600|1|0-500|5|0| +601|2|0-500|5|0| +602|3|0-500|5|0| +603|4|0-500|5|0| +604|5|0-500|5|0| +605|6|0-500|5|0| +606|7|0-500|5|0| +607|8|0-500|5|0| +608|9|0-500|5|0| +609|10|0-500|5|0| +610|11|0-500|5|0| +611|12|0-500|5|0| +612|13|0-500|5|0| +613|14|0-500|5|0| +614|15|0-500|5|0| +615|16|0-500|5|0| +616|17|0-500|5|0| +617|18|0-500|5|0| +618|19|0-500|5|0| +619|20|0-500|5|0| +620|1|501-1000|5|0| +621|2|501-1000|5|0| +622|3|501-1000|5|0| +623|4|501-1000|5|0| +624|5|501-1000|5|0| +625|6|501-1000|5|0| +626|7|501-1000|5|0| +627|8|501-1000|5|0| +628|9|501-1000|5|0| +629|10|501-1000|5|0| +630|11|501-1000|5|0| +631|12|501-1000|5|0| +632|13|501-1000|5|0| +633|14|501-1000|5|0| +634|15|501-1000|5|0| +635|16|501-1000|5|0| +636|17|501-1000|5|0| +637|18|501-1000|5|0| +638|19|501-1000|5|0| +639|20|501-1000|5|0| +640|1|1001-5000|5|0| +641|2|1001-5000|5|0| +642|3|1001-5000|5|0| +643|4|1001-5000|5|0| +644|5|1001-5000|5|0| +645|6|1001-5000|5|0| +646|7|1001-5000|5|0| +647|8|1001-5000|5|0| +648|9|1001-5000|5|0| +649|10|1001-5000|5|0| +650|11|1001-5000|5|0| +651|12|1001-5000|5|0| +652|13|1001-5000|5|0| +653|14|1001-5000|5|0| +654|15|1001-5000|5|0| +655|16|1001-5000|5|0| +656|17|1001-5000|5|0| +657|18|1001-5000|5|0| +658|19|1001-5000|5|0| +659|20|1001-5000|5|0| +660|1|5001-10000|5|0| +661|2|5001-10000|5|0| +662|3|5001-10000|5|0| +663|4|5001-10000|5|0| +664|5|5001-10000|5|0| +665|6|5001-10000|5|0| +666|7|5001-10000|5|0| +667|8|5001-10000|5|0| +668|9|5001-10000|5|0| +669|10|5001-10000|5|0| +670|11|5001-10000|5|0| +671|12|5001-10000|5|0| +672|13|5001-10000|5|0| +673|14|5001-10000|5|0| +674|15|5001-10000|5|0| +675|16|5001-10000|5|0| +676|17|5001-10000|5|0| +677|18|5001-10000|5|0| +678|19|5001-10000|5|0| +679|20|5001-10000|5|0| +680|1|>10000|5|0| +681|2|>10000|5|0| +682|3|>10000|5|0| +683|4|>10000|5|0| +684|5|>10000|5|0| +685|6|>10000|5|0| +686|7|>10000|5|0| +687|8|>10000|5|0| +688|9|>10000|5|0| +689|10|>10000|5|0| +690|11|>10000|5|0| +691|12|>10000|5|0| +692|13|>10000|5|0| +693|14|>10000|5|0| +694|15|>10000|5|0| +695|16|>10000|5|0| +696|17|>10000|5|0| +697|18|>10000|5|0| +698|19|>10000|5|0| +699|20|>10000|5|0| +700|1|Unknown|5|0| +701|2|Unknown|5|0| +702|3|Unknown|5|0| +703|4|Unknown|5|0| +704|5|Unknown|5|0| +705|6|Unknown|5|0| +706|7|Unknown|5|0| +707|8|Unknown|5|0| +708|9|Unknown|5|0| +709|10|Unknown|5|0| +710|11|Unknown|5|0| +711|12|Unknown|5|0| +712|13|Unknown|5|0| +713|14|Unknown|5|0| +714|15|Unknown|5|0| +715|16|Unknown|5|0| +716|17|Unknown|5|0| +717|18|Unknown|5|0| +718|19|Unknown|5|0| +719|20|Unknown|5|0| +720|1|0-500|6|0| +721|2|0-500|6|0| +722|3|0-500|6|0| +723|4|0-500|6|0| +724|5|0-500|6|0| +725|6|0-500|6|0| +726|7|0-500|6|0| +727|8|0-500|6|0| +728|9|0-500|6|0| +729|10|0-500|6|0| +730|11|0-500|6|0| +731|12|0-500|6|0| +732|13|0-500|6|0| +733|14|0-500|6|0| +734|15|0-500|6|0| +735|16|0-500|6|0| +736|17|0-500|6|0| +737|18|0-500|6|0| +738|19|0-500|6|0| +739|20|0-500|6|0| +740|1|501-1000|6|0| +741|2|501-1000|6|0| +742|3|501-1000|6|0| +743|4|501-1000|6|0| +744|5|501-1000|6|0| +745|6|501-1000|6|0| +746|7|501-1000|6|0| +747|8|501-1000|6|0| +748|9|501-1000|6|0| +749|10|501-1000|6|0| +750|11|501-1000|6|0| +751|12|501-1000|6|0| +752|13|501-1000|6|0| +753|14|501-1000|6|0| +754|15|501-1000|6|0| +755|16|501-1000|6|0| +756|17|501-1000|6|0| +757|18|501-1000|6|0| +758|19|501-1000|6|0| +759|20|501-1000|6|0| +760|1|1001-5000|6|0| +761|2|1001-5000|6|0| +762|3|1001-5000|6|0| +763|4|1001-5000|6|0| +764|5|1001-5000|6|0| +765|6|1001-5000|6|0| +766|7|1001-5000|6|0| +767|8|1001-5000|6|0| +768|9|1001-5000|6|0| +769|10|1001-5000|6|0| +770|11|1001-5000|6|0| +771|12|1001-5000|6|0| +772|13|1001-5000|6|0| +773|14|1001-5000|6|0| +774|15|1001-5000|6|0| +775|16|1001-5000|6|0| +776|17|1001-5000|6|0| +777|18|1001-5000|6|0| +778|19|1001-5000|6|0| +779|20|1001-5000|6|0| +780|1|5001-10000|6|0| +781|2|5001-10000|6|0| +782|3|5001-10000|6|0| +783|4|5001-10000|6|0| +784|5|5001-10000|6|0| +785|6|5001-10000|6|0| +786|7|5001-10000|6|0| +787|8|5001-10000|6|0| +788|9|5001-10000|6|0| +789|10|5001-10000|6|0| +790|11|5001-10000|6|0| +791|12|5001-10000|6|0| +792|13|5001-10000|6|0| +793|14|5001-10000|6|0| +794|15|5001-10000|6|0| +795|16|5001-10000|6|0| +796|17|5001-10000|6|0| +797|18|5001-10000|6|0| +798|19|5001-10000|6|0| +799|20|5001-10000|6|0| +800|1|>10000|6|0| +801|2|>10000|6|0| +802|3|>10000|6|0| +803|4|>10000|6|0| +804|5|>10000|6|0| +805|6|>10000|6|0| +806|7|>10000|6|0| +807|8|>10000|6|0| +808|9|>10000|6|0| +809|10|>10000|6|0| +810|11|>10000|6|0| +811|12|>10000|6|0| +812|13|>10000|6|0| +813|14|>10000|6|0| +814|15|>10000|6|0| +815|16|>10000|6|0| +816|17|>10000|6|0| +817|18|>10000|6|0| +818|19|>10000|6|0| +819|20|>10000|6|0| +820|1|Unknown|6|0| +821|2|Unknown|6|0| +822|3|Unknown|6|0| +823|4|Unknown|6|0| +824|5|Unknown|6|0| +825|6|Unknown|6|0| +826|7|Unknown|6|0| +827|8|Unknown|6|0| +828|9|Unknown|6|0| +829|10|Unknown|6|0| +830|11|Unknown|6|0| +831|12|Unknown|6|0| +832|13|Unknown|6|0| +833|14|Unknown|6|0| +834|15|Unknown|6|0| +835|16|Unknown|6|0| +836|17|Unknown|6|0| +837|18|Unknown|6|0| +838|19|Unknown|6|0| +839|20|Unknown|6|0| +840|1|0-500|7|0| +841|2|0-500|7|0| +842|3|0-500|7|0| +843|4|0-500|7|0| +844|5|0-500|7|0| +845|6|0-500|7|0| +846|7|0-500|7|0| +847|8|0-500|7|0| +848|9|0-500|7|0| +849|10|0-500|7|0| +850|11|0-500|7|0| +851|12|0-500|7|0| +852|13|0-500|7|0| +853|14|0-500|7|0| +854|15|0-500|7|0| +855|16|0-500|7|0| +856|17|0-500|7|0| +857|18|0-500|7|0| +858|19|0-500|7|0| +859|20|0-500|7|0| +860|1|501-1000|7|0| +861|2|501-1000|7|0| +862|3|501-1000|7|0| +863|4|501-1000|7|0| +864|5|501-1000|7|0| +865|6|501-1000|7|0| +866|7|501-1000|7|0| +867|8|501-1000|7|0| +868|9|501-1000|7|0| +869|10|501-1000|7|0| +870|11|501-1000|7|0| +871|12|501-1000|7|0| +872|13|501-1000|7|0| +873|14|501-1000|7|0| +874|15|501-1000|7|0| +875|16|501-1000|7|0| +876|17|501-1000|7|0| +877|18|501-1000|7|0| +878|19|501-1000|7|0| +879|20|501-1000|7|0| +880|1|1001-5000|7|0| +881|2|1001-5000|7|0| +882|3|1001-5000|7|0| +883|4|1001-5000|7|0| +884|5|1001-5000|7|0| +885|6|1001-5000|7|0| +886|7|1001-5000|7|0| +887|8|1001-5000|7|0| +888|9|1001-5000|7|0| +889|10|1001-5000|7|0| +890|11|1001-5000|7|0| +891|12|1001-5000|7|0| +892|13|1001-5000|7|0| +893|14|1001-5000|7|0| +894|15|1001-5000|7|0| +895|16|1001-5000|7|0| +896|17|1001-5000|7|0| +897|18|1001-5000|7|0| +898|19|1001-5000|7|0| +899|20|1001-5000|7|0| +900|1|5001-10000|7|0| +901|2|5001-10000|7|0| +902|3|5001-10000|7|0| +903|4|5001-10000|7|0| +904|5|5001-10000|7|0| +905|6|5001-10000|7|0| +906|7|5001-10000|7|0| +907|8|5001-10000|7|0| +908|9|5001-10000|7|0| +909|10|5001-10000|7|0| +910|11|5001-10000|7|0| +911|12|5001-10000|7|0| +912|13|5001-10000|7|0| +913|14|5001-10000|7|0| +914|15|5001-10000|7|0| +915|16|5001-10000|7|0| +916|17|5001-10000|7|0| +917|18|5001-10000|7|0| +918|19|5001-10000|7|0| +919|20|5001-10000|7|0| +920|1|>10000|7|0| +921|2|>10000|7|0| +922|3|>10000|7|0| +923|4|>10000|7|0| +924|5|>10000|7|0| +925|6|>10000|7|0| +926|7|>10000|7|0| +927|8|>10000|7|0| +928|9|>10000|7|0| +929|10|>10000|7|0| +930|11|>10000|7|0| +931|12|>10000|7|0| +932|13|>10000|7|0| +933|14|>10000|7|0| +934|15|>10000|7|0| +935|16|>10000|7|0| +936|17|>10000|7|0| +937|18|>10000|7|0| +938|19|>10000|7|0| +939|20|>10000|7|0| +940|1|Unknown|7|0| +941|2|Unknown|7|0| +942|3|Unknown|7|0| +943|4|Unknown|7|0| +944|5|Unknown|7|0| +945|6|Unknown|7|0| +946|7|Unknown|7|0| +947|8|Unknown|7|0| +948|9|Unknown|7|0| +949|10|Unknown|7|0| +950|11|Unknown|7|0| +951|12|Unknown|7|0| +952|13|Unknown|7|0| +953|14|Unknown|7|0| +954|15|Unknown|7|0| +955|16|Unknown|7|0| +956|17|Unknown|7|0| +957|18|Unknown|7|0| +958|19|Unknown|7|0| +959|20|Unknown|7|0| +960|1|0-500|8|0| +961|2|0-500|8|0| +962|3|0-500|8|0| +963|4|0-500|8|0| +964|5|0-500|8|0| +965|6|0-500|8|0| +966|7|0-500|8|0| +967|8|0-500|8|0| +968|9|0-500|8|0| +969|10|0-500|8|0| +970|11|0-500|8|0| +971|12|0-500|8|0| +972|13|0-500|8|0| +973|14|0-500|8|0| +974|15|0-500|8|0| +975|16|0-500|8|0| +976|17|0-500|8|0| +977|18|0-500|8|0| +978|19|0-500|8|0| +979|20|0-500|8|0| +980|1|501-1000|8|0| +981|2|501-1000|8|0| +982|3|501-1000|8|0| +983|4|501-1000|8|0| +984|5|501-1000|8|0| +985|6|501-1000|8|0| +986|7|501-1000|8|0| +987|8|501-1000|8|0| +988|9|501-1000|8|0| +989|10|501-1000|8|0| +990|11|501-1000|8|0| +991|12|501-1000|8|0| +992|13|501-1000|8|0| +993|14|501-1000|8|0| +994|15|501-1000|8|0| +995|16|501-1000|8|0| +996|17|501-1000|8|0| +997|18|501-1000|8|0| +998|19|501-1000|8|0| +999|20|501-1000|8|0| +1000|1|1001-5000|8|0| +1001|2|1001-5000|8|0| +1002|3|1001-5000|8|0| +1003|4|1001-5000|8|0| +1004|5|1001-5000|8|0| +1005|6|1001-5000|8|0| +1006|7|1001-5000|8|0| +1007|8|1001-5000|8|0| +1008|9|1001-5000|8|0| +1009|10|1001-5000|8|0| +1010|11|1001-5000|8|0| +1011|12|1001-5000|8|0| +1012|13|1001-5000|8|0| +1013|14|1001-5000|8|0| +1014|15|1001-5000|8|0| +1015|16|1001-5000|8|0| +1016|17|1001-5000|8|0| +1017|18|1001-5000|8|0| +1018|19|1001-5000|8|0| +1019|20|1001-5000|8|0| +1020|1|5001-10000|8|0| +1021|2|5001-10000|8|0| +1022|3|5001-10000|8|0| +1023|4|5001-10000|8|0| +1024|5|5001-10000|8|0| +1025|6|5001-10000|8|0| +1026|7|5001-10000|8|0| +1027|8|5001-10000|8|0| +1028|9|5001-10000|8|0| +1029|10|5001-10000|8|0| +1030|11|5001-10000|8|0| +1031|12|5001-10000|8|0| +1032|13|5001-10000|8|0| +1033|14|5001-10000|8|0| +1034|15|5001-10000|8|0| +1035|16|5001-10000|8|0| +1036|17|5001-10000|8|0| +1037|18|5001-10000|8|0| +1038|19|5001-10000|8|0| +1039|20|5001-10000|8|0| +1040|1|>10000|8|0| +1041|2|>10000|8|0| +1042|3|>10000|8|0| +1043|4|>10000|8|0| +1044|5|>10000|8|0| +1045|6|>10000|8|0| +1046|7|>10000|8|0| +1047|8|>10000|8|0| +1048|9|>10000|8|0| +1049|10|>10000|8|0| +1050|11|>10000|8|0| +1051|12|>10000|8|0| +1052|13|>10000|8|0| +1053|14|>10000|8|0| +1054|15|>10000|8|0| +1055|16|>10000|8|0| +1056|17|>10000|8|0| +1057|18|>10000|8|0| +1058|19|>10000|8|0| +1059|20|>10000|8|0| +1060|1|Unknown|8|0| +1061|2|Unknown|8|0| +1062|3|Unknown|8|0| +1063|4|Unknown|8|0| +1064|5|Unknown|8|0| +1065|6|Unknown|8|0| +1066|7|Unknown|8|0| +1067|8|Unknown|8|0| +1068|9|Unknown|8|0| +1069|10|Unknown|8|0| +1070|11|Unknown|8|0| +1071|12|Unknown|8|0| +1072|13|Unknown|8|0| +1073|14|Unknown|8|0| +1074|15|Unknown|8|0| +1075|16|Unknown|8|0| +1076|17|Unknown|8|0| +1077|18|Unknown|8|0| +1078|19|Unknown|8|0| +1079|20|Unknown|8|0| +1080|1|0-500|9|0| +1081|2|0-500|9|0| +1082|3|0-500|9|0| +1083|4|0-500|9|0| +1084|5|0-500|9|0| +1085|6|0-500|9|0| +1086|7|0-500|9|0| +1087|8|0-500|9|0| +1088|9|0-500|9|0| +1089|10|0-500|9|0| +1090|11|0-500|9|0| +1091|12|0-500|9|0| +1092|13|0-500|9|0| +1093|14|0-500|9|0| +1094|15|0-500|9|0| +1095|16|0-500|9|0| +1096|17|0-500|9|0| +1097|18|0-500|9|0| +1098|19|0-500|9|0| +1099|20|0-500|9|0| +1100|1|501-1000|9|0| +1101|2|501-1000|9|0| +1102|3|501-1000|9|0| +1103|4|501-1000|9|0| +1104|5|501-1000|9|0| +1105|6|501-1000|9|0| +1106|7|501-1000|9|0| +1107|8|501-1000|9|0| +1108|9|501-1000|9|0| +1109|10|501-1000|9|0| +1110|11|501-1000|9|0| +1111|12|501-1000|9|0| +1112|13|501-1000|9|0| +1113|14|501-1000|9|0| +1114|15|501-1000|9|0| +1115|16|501-1000|9|0| +1116|17|501-1000|9|0| +1117|18|501-1000|9|0| +1118|19|501-1000|9|0| +1119|20|501-1000|9|0| +1120|1|1001-5000|9|0| +1121|2|1001-5000|9|0| +1122|3|1001-5000|9|0| +1123|4|1001-5000|9|0| +1124|5|1001-5000|9|0| +1125|6|1001-5000|9|0| +1126|7|1001-5000|9|0| +1127|8|1001-5000|9|0| +1128|9|1001-5000|9|0| +1129|10|1001-5000|9|0| +1130|11|1001-5000|9|0| +1131|12|1001-5000|9|0| +1132|13|1001-5000|9|0| +1133|14|1001-5000|9|0| +1134|15|1001-5000|9|0| +1135|16|1001-5000|9|0| +1136|17|1001-5000|9|0| +1137|18|1001-5000|9|0| +1138|19|1001-5000|9|0| +1139|20|1001-5000|9|0| +1140|1|5001-10000|9|0| +1141|2|5001-10000|9|0| +1142|3|5001-10000|9|0| +1143|4|5001-10000|9|0| +1144|5|5001-10000|9|0| +1145|6|5001-10000|9|0| +1146|7|5001-10000|9|0| +1147|8|5001-10000|9|0| +1148|9|5001-10000|9|0| +1149|10|5001-10000|9|0| +1150|11|5001-10000|9|0| +1151|12|5001-10000|9|0| +1152|13|5001-10000|9|0| +1153|14|5001-10000|9|0| +1154|15|5001-10000|9|0| +1155|16|5001-10000|9|0| +1156|17|5001-10000|9|0| +1157|18|5001-10000|9|0| +1158|19|5001-10000|9|0| +1159|20|5001-10000|9|0| +1160|1|>10000|9|0| +1161|2|>10000|9|0| +1162|3|>10000|9|0| +1163|4|>10000|9|0| +1164|5|>10000|9|0| +1165|6|>10000|9|0| +1166|7|>10000|9|0| +1167|8|>10000|9|0| +1168|9|>10000|9|0| +1169|10|>10000|9|0| +1170|11|>10000|9|0| +1171|12|>10000|9|0| +1172|13|>10000|9|0| +1173|14|>10000|9|0| +1174|15|>10000|9|0| +1175|16|>10000|9|0| +1176|17|>10000|9|0| +1177|18|>10000|9|0| +1178|19|>10000|9|0| +1179|20|>10000|9|0| +1180|1|Unknown|9|0| +1181|2|Unknown|9|0| +1182|3|Unknown|9|0| +1183|4|Unknown|9|0| +1184|5|Unknown|9|0| +1185|6|Unknown|9|0| +1186|7|Unknown|9|0| +1187|8|Unknown|9|0| +1188|9|Unknown|9|0| +1189|10|Unknown|9|0| +1190|11|Unknown|9|0| +1191|12|Unknown|9|0| +1192|13|Unknown|9|0| +1193|14|Unknown|9|0| +1194|15|Unknown|9|0| +1195|16|Unknown|9|0| +1196|17|Unknown|9|0| +1197|18|Unknown|9|0| +1198|19|Unknown|9|0| +1199|20|Unknown|9|0| +1200|1|0-500|0|1| +1201|2|0-500|0|1| +1202|3|0-500|0|1| +1203|4|0-500|0|1| +1204|5|0-500|0|1| +1205|6|0-500|0|1| +1206|7|0-500|0|1| +1207|8|0-500|0|1| +1208|9|0-500|0|1| +1209|10|0-500|0|1| +1210|11|0-500|0|1| +1211|12|0-500|0|1| +1212|13|0-500|0|1| +1213|14|0-500|0|1| +1214|15|0-500|0|1| +1215|16|0-500|0|1| +1216|17|0-500|0|1| +1217|18|0-500|0|1| +1218|19|0-500|0|1| +1219|20|0-500|0|1| +1220|1|501-1000|0|1| +1221|2|501-1000|0|1| +1222|3|501-1000|0|1| +1223|4|501-1000|0|1| +1224|5|501-1000|0|1| +1225|6|501-1000|0|1| +1226|7|501-1000|0|1| +1227|8|501-1000|0|1| +1228|9|501-1000|0|1| +1229|10|501-1000|0|1| +1230|11|501-1000|0|1| +1231|12|501-1000|0|1| +1232|13|501-1000|0|1| +1233|14|501-1000|0|1| +1234|15|501-1000|0|1| +1235|16|501-1000|0|1| +1236|17|501-1000|0|1| +1237|18|501-1000|0|1| +1238|19|501-1000|0|1| +1239|20|501-1000|0|1| +1240|1|1001-5000|0|1| +1241|2|1001-5000|0|1| +1242|3|1001-5000|0|1| +1243|4|1001-5000|0|1| +1244|5|1001-5000|0|1| +1245|6|1001-5000|0|1| +1246|7|1001-5000|0|1| +1247|8|1001-5000|0|1| +1248|9|1001-5000|0|1| +1249|10|1001-5000|0|1| +1250|11|1001-5000|0|1| +1251|12|1001-5000|0|1| +1252|13|1001-5000|0|1| +1253|14|1001-5000|0|1| +1254|15|1001-5000|0|1| +1255|16|1001-5000|0|1| +1256|17|1001-5000|0|1| +1257|18|1001-5000|0|1| +1258|19|1001-5000|0|1| +1259|20|1001-5000|0|1| +1260|1|5001-10000|0|1| +1261|2|5001-10000|0|1| +1262|3|5001-10000|0|1| +1263|4|5001-10000|0|1| +1264|5|5001-10000|0|1| +1265|6|5001-10000|0|1| +1266|7|5001-10000|0|1| +1267|8|5001-10000|0|1| +1268|9|5001-10000|0|1| +1269|10|5001-10000|0|1| +1270|11|5001-10000|0|1| +1271|12|5001-10000|0|1| +1272|13|5001-10000|0|1| +1273|14|5001-10000|0|1| +1274|15|5001-10000|0|1| +1275|16|5001-10000|0|1| +1276|17|5001-10000|0|1| +1277|18|5001-10000|0|1| +1278|19|5001-10000|0|1| +1279|20|5001-10000|0|1| +1280|1|>10000|0|1| +1281|2|>10000|0|1| +1282|3|>10000|0|1| +1283|4|>10000|0|1| +1284|5|>10000|0|1| +1285|6|>10000|0|1| +1286|7|>10000|0|1| +1287|8|>10000|0|1| +1288|9|>10000|0|1| +1289|10|>10000|0|1| +1290|11|>10000|0|1| +1291|12|>10000|0|1| +1292|13|>10000|0|1| +1293|14|>10000|0|1| +1294|15|>10000|0|1| +1295|16|>10000|0|1| +1296|17|>10000|0|1| +1297|18|>10000|0|1| +1298|19|>10000|0|1| +1299|20|>10000|0|1| +1300|1|Unknown|0|1| +1301|2|Unknown|0|1| +1302|3|Unknown|0|1| +1303|4|Unknown|0|1| +1304|5|Unknown|0|1| +1305|6|Unknown|0|1| +1306|7|Unknown|0|1| +1307|8|Unknown|0|1| +1308|9|Unknown|0|1| +1309|10|Unknown|0|1| +1310|11|Unknown|0|1| +1311|12|Unknown|0|1| +1312|13|Unknown|0|1| +1313|14|Unknown|0|1| +1314|15|Unknown|0|1| +1315|16|Unknown|0|1| +1316|17|Unknown|0|1| +1317|18|Unknown|0|1| +1318|19|Unknown|0|1| +1319|20|Unknown|0|1| +1320|1|0-500|1|1| +1321|2|0-500|1|1| +1322|3|0-500|1|1| +1323|4|0-500|1|1| +1324|5|0-500|1|1| +1325|6|0-500|1|1| +1326|7|0-500|1|1| +1327|8|0-500|1|1| +1328|9|0-500|1|1| +1329|10|0-500|1|1| +1330|11|0-500|1|1| +1331|12|0-500|1|1| +1332|13|0-500|1|1| +1333|14|0-500|1|1| +1334|15|0-500|1|1| +1335|16|0-500|1|1| +1336|17|0-500|1|1| +1337|18|0-500|1|1| +1338|19|0-500|1|1| +1339|20|0-500|1|1| +1340|1|501-1000|1|1| +1341|2|501-1000|1|1| +1342|3|501-1000|1|1| +1343|4|501-1000|1|1| +1344|5|501-1000|1|1| +1345|6|501-1000|1|1| +1346|7|501-1000|1|1| +1347|8|501-1000|1|1| +1348|9|501-1000|1|1| +1349|10|501-1000|1|1| +1350|11|501-1000|1|1| +1351|12|501-1000|1|1| +1352|13|501-1000|1|1| +1353|14|501-1000|1|1| +1354|15|501-1000|1|1| +1355|16|501-1000|1|1| +1356|17|501-1000|1|1| +1357|18|501-1000|1|1| +1358|19|501-1000|1|1| +1359|20|501-1000|1|1| +1360|1|1001-5000|1|1| +1361|2|1001-5000|1|1| +1362|3|1001-5000|1|1| +1363|4|1001-5000|1|1| +1364|5|1001-5000|1|1| +1365|6|1001-5000|1|1| +1366|7|1001-5000|1|1| +1367|8|1001-5000|1|1| +1368|9|1001-5000|1|1| +1369|10|1001-5000|1|1| +1370|11|1001-5000|1|1| +1371|12|1001-5000|1|1| +1372|13|1001-5000|1|1| +1373|14|1001-5000|1|1| +1374|15|1001-5000|1|1| +1375|16|1001-5000|1|1| +1376|17|1001-5000|1|1| +1377|18|1001-5000|1|1| +1378|19|1001-5000|1|1| +1379|20|1001-5000|1|1| +1380|1|5001-10000|1|1| +1381|2|5001-10000|1|1| +1382|3|5001-10000|1|1| +1383|4|5001-10000|1|1| +1384|5|5001-10000|1|1| +1385|6|5001-10000|1|1| +1386|7|5001-10000|1|1| +1387|8|5001-10000|1|1| +1388|9|5001-10000|1|1| +1389|10|5001-10000|1|1| +1390|11|5001-10000|1|1| +1391|12|5001-10000|1|1| +1392|13|5001-10000|1|1| +1393|14|5001-10000|1|1| +1394|15|5001-10000|1|1| +1395|16|5001-10000|1|1| +1396|17|5001-10000|1|1| +1397|18|5001-10000|1|1| +1398|19|5001-10000|1|1| +1399|20|5001-10000|1|1| +1400|1|>10000|1|1| +1401|2|>10000|1|1| +1402|3|>10000|1|1| +1403|4|>10000|1|1| +1404|5|>10000|1|1| +1405|6|>10000|1|1| +1406|7|>10000|1|1| +1407|8|>10000|1|1| +1408|9|>10000|1|1| +1409|10|>10000|1|1| +1410|11|>10000|1|1| +1411|12|>10000|1|1| +1412|13|>10000|1|1| +1413|14|>10000|1|1| +1414|15|>10000|1|1| +1415|16|>10000|1|1| +1416|17|>10000|1|1| +1417|18|>10000|1|1| +1418|19|>10000|1|1| +1419|20|>10000|1|1| +1420|1|Unknown|1|1| +1421|2|Unknown|1|1| +1422|3|Unknown|1|1| +1423|4|Unknown|1|1| +1424|5|Unknown|1|1| +1425|6|Unknown|1|1| +1426|7|Unknown|1|1| +1427|8|Unknown|1|1| +1428|9|Unknown|1|1| +1429|10|Unknown|1|1| +1430|11|Unknown|1|1| +1431|12|Unknown|1|1| +1432|13|Unknown|1|1| +1433|14|Unknown|1|1| +1434|15|Unknown|1|1| +1435|16|Unknown|1|1| +1436|17|Unknown|1|1| +1437|18|Unknown|1|1| +1438|19|Unknown|1|1| +1439|20|Unknown|1|1| +1440|1|0-500|2|1| +1441|2|0-500|2|1| +1442|3|0-500|2|1| +1443|4|0-500|2|1| +1444|5|0-500|2|1| +1445|6|0-500|2|1| +1446|7|0-500|2|1| +1447|8|0-500|2|1| +1448|9|0-500|2|1| +1449|10|0-500|2|1| +1450|11|0-500|2|1| +1451|12|0-500|2|1| +1452|13|0-500|2|1| +1453|14|0-500|2|1| +1454|15|0-500|2|1| +1455|16|0-500|2|1| +1456|17|0-500|2|1| +1457|18|0-500|2|1| +1458|19|0-500|2|1| +1459|20|0-500|2|1| +1460|1|501-1000|2|1| +1461|2|501-1000|2|1| +1462|3|501-1000|2|1| +1463|4|501-1000|2|1| +1464|5|501-1000|2|1| +1465|6|501-1000|2|1| +1466|7|501-1000|2|1| +1467|8|501-1000|2|1| +1468|9|501-1000|2|1| +1469|10|501-1000|2|1| +1470|11|501-1000|2|1| +1471|12|501-1000|2|1| +1472|13|501-1000|2|1| +1473|14|501-1000|2|1| +1474|15|501-1000|2|1| +1475|16|501-1000|2|1| +1476|17|501-1000|2|1| +1477|18|501-1000|2|1| +1478|19|501-1000|2|1| +1479|20|501-1000|2|1| +1480|1|1001-5000|2|1| +1481|2|1001-5000|2|1| +1482|3|1001-5000|2|1| +1483|4|1001-5000|2|1| +1484|5|1001-5000|2|1| +1485|6|1001-5000|2|1| +1486|7|1001-5000|2|1| +1487|8|1001-5000|2|1| +1488|9|1001-5000|2|1| +1489|10|1001-5000|2|1| +1490|11|1001-5000|2|1| +1491|12|1001-5000|2|1| +1492|13|1001-5000|2|1| +1493|14|1001-5000|2|1| +1494|15|1001-5000|2|1| +1495|16|1001-5000|2|1| +1496|17|1001-5000|2|1| +1497|18|1001-5000|2|1| +1498|19|1001-5000|2|1| +1499|20|1001-5000|2|1| +1500|1|5001-10000|2|1| +1501|2|5001-10000|2|1| +1502|3|5001-10000|2|1| +1503|4|5001-10000|2|1| +1504|5|5001-10000|2|1| +1505|6|5001-10000|2|1| +1506|7|5001-10000|2|1| +1507|8|5001-10000|2|1| +1508|9|5001-10000|2|1| +1509|10|5001-10000|2|1| +1510|11|5001-10000|2|1| +1511|12|5001-10000|2|1| +1512|13|5001-10000|2|1| +1513|14|5001-10000|2|1| +1514|15|5001-10000|2|1| +1515|16|5001-10000|2|1| +1516|17|5001-10000|2|1| +1517|18|5001-10000|2|1| +1518|19|5001-10000|2|1| +1519|20|5001-10000|2|1| +1520|1|>10000|2|1| +1521|2|>10000|2|1| +1522|3|>10000|2|1| +1523|4|>10000|2|1| +1524|5|>10000|2|1| +1525|6|>10000|2|1| +1526|7|>10000|2|1| +1527|8|>10000|2|1| +1528|9|>10000|2|1| +1529|10|>10000|2|1| +1530|11|>10000|2|1| +1531|12|>10000|2|1| +1532|13|>10000|2|1| +1533|14|>10000|2|1| +1534|15|>10000|2|1| +1535|16|>10000|2|1| +1536|17|>10000|2|1| +1537|18|>10000|2|1| +1538|19|>10000|2|1| +1539|20|>10000|2|1| +1540|1|Unknown|2|1| +1541|2|Unknown|2|1| +1542|3|Unknown|2|1| +1543|4|Unknown|2|1| +1544|5|Unknown|2|1| +1545|6|Unknown|2|1| +1546|7|Unknown|2|1| +1547|8|Unknown|2|1| +1548|9|Unknown|2|1| +1549|10|Unknown|2|1| +1550|11|Unknown|2|1| +1551|12|Unknown|2|1| +1552|13|Unknown|2|1| +1553|14|Unknown|2|1| +1554|15|Unknown|2|1| +1555|16|Unknown|2|1| +1556|17|Unknown|2|1| +1557|18|Unknown|2|1| +1558|19|Unknown|2|1| +1559|20|Unknown|2|1| +1560|1|0-500|3|1| +1561|2|0-500|3|1| +1562|3|0-500|3|1| +1563|4|0-500|3|1| +1564|5|0-500|3|1| +1565|6|0-500|3|1| +1566|7|0-500|3|1| +1567|8|0-500|3|1| +1568|9|0-500|3|1| +1569|10|0-500|3|1| +1570|11|0-500|3|1| +1571|12|0-500|3|1| +1572|13|0-500|3|1| +1573|14|0-500|3|1| +1574|15|0-500|3|1| +1575|16|0-500|3|1| +1576|17|0-500|3|1| +1577|18|0-500|3|1| +1578|19|0-500|3|1| +1579|20|0-500|3|1| +1580|1|501-1000|3|1| +1581|2|501-1000|3|1| +1582|3|501-1000|3|1| +1583|4|501-1000|3|1| +1584|5|501-1000|3|1| +1585|6|501-1000|3|1| +1586|7|501-1000|3|1| +1587|8|501-1000|3|1| +1588|9|501-1000|3|1| +1589|10|501-1000|3|1| +1590|11|501-1000|3|1| +1591|12|501-1000|3|1| +1592|13|501-1000|3|1| +1593|14|501-1000|3|1| +1594|15|501-1000|3|1| +1595|16|501-1000|3|1| +1596|17|501-1000|3|1| +1597|18|501-1000|3|1| +1598|19|501-1000|3|1| +1599|20|501-1000|3|1| +1600|1|1001-5000|3|1| +1601|2|1001-5000|3|1| +1602|3|1001-5000|3|1| +1603|4|1001-5000|3|1| +1604|5|1001-5000|3|1| +1605|6|1001-5000|3|1| +1606|7|1001-5000|3|1| +1607|8|1001-5000|3|1| +1608|9|1001-5000|3|1| +1609|10|1001-5000|3|1| +1610|11|1001-5000|3|1| +1611|12|1001-5000|3|1| +1612|13|1001-5000|3|1| +1613|14|1001-5000|3|1| +1614|15|1001-5000|3|1| +1615|16|1001-5000|3|1| +1616|17|1001-5000|3|1| +1617|18|1001-5000|3|1| +1618|19|1001-5000|3|1| +1619|20|1001-5000|3|1| +1620|1|5001-10000|3|1| +1621|2|5001-10000|3|1| +1622|3|5001-10000|3|1| +1623|4|5001-10000|3|1| +1624|5|5001-10000|3|1| +1625|6|5001-10000|3|1| +1626|7|5001-10000|3|1| +1627|8|5001-10000|3|1| +1628|9|5001-10000|3|1| +1629|10|5001-10000|3|1| +1630|11|5001-10000|3|1| +1631|12|5001-10000|3|1| +1632|13|5001-10000|3|1| +1633|14|5001-10000|3|1| +1634|15|5001-10000|3|1| +1635|16|5001-10000|3|1| +1636|17|5001-10000|3|1| +1637|18|5001-10000|3|1| +1638|19|5001-10000|3|1| +1639|20|5001-10000|3|1| +1640|1|>10000|3|1| +1641|2|>10000|3|1| +1642|3|>10000|3|1| +1643|4|>10000|3|1| +1644|5|>10000|3|1| +1645|6|>10000|3|1| +1646|7|>10000|3|1| +1647|8|>10000|3|1| +1648|9|>10000|3|1| +1649|10|>10000|3|1| +1650|11|>10000|3|1| +1651|12|>10000|3|1| +1652|13|>10000|3|1| +1653|14|>10000|3|1| +1654|15|>10000|3|1| +1655|16|>10000|3|1| +1656|17|>10000|3|1| +1657|18|>10000|3|1| +1658|19|>10000|3|1| +1659|20|>10000|3|1| +1660|1|Unknown|3|1| +1661|2|Unknown|3|1| +1662|3|Unknown|3|1| +1663|4|Unknown|3|1| +1664|5|Unknown|3|1| +1665|6|Unknown|3|1| +1666|7|Unknown|3|1| +1667|8|Unknown|3|1| +1668|9|Unknown|3|1| +1669|10|Unknown|3|1| +1670|11|Unknown|3|1| +1671|12|Unknown|3|1| +1672|13|Unknown|3|1| +1673|14|Unknown|3|1| +1674|15|Unknown|3|1| +1675|16|Unknown|3|1| +1676|17|Unknown|3|1| +1677|18|Unknown|3|1| +1678|19|Unknown|3|1| +1679|20|Unknown|3|1| +1680|1|0-500|4|1| +1681|2|0-500|4|1| +1682|3|0-500|4|1| +1683|4|0-500|4|1| +1684|5|0-500|4|1| +1685|6|0-500|4|1| +1686|7|0-500|4|1| +1687|8|0-500|4|1| +1688|9|0-500|4|1| +1689|10|0-500|4|1| +1690|11|0-500|4|1| +1691|12|0-500|4|1| +1692|13|0-500|4|1| +1693|14|0-500|4|1| +1694|15|0-500|4|1| +1695|16|0-500|4|1| +1696|17|0-500|4|1| +1697|18|0-500|4|1| +1698|19|0-500|4|1| +1699|20|0-500|4|1| +1700|1|501-1000|4|1| +1701|2|501-1000|4|1| +1702|3|501-1000|4|1| +1703|4|501-1000|4|1| +1704|5|501-1000|4|1| +1705|6|501-1000|4|1| +1706|7|501-1000|4|1| +1707|8|501-1000|4|1| +1708|9|501-1000|4|1| +1709|10|501-1000|4|1| +1710|11|501-1000|4|1| +1711|12|501-1000|4|1| +1712|13|501-1000|4|1| +1713|14|501-1000|4|1| +1714|15|501-1000|4|1| +1715|16|501-1000|4|1| +1716|17|501-1000|4|1| +1717|18|501-1000|4|1| +1718|19|501-1000|4|1| +1719|20|501-1000|4|1| +1720|1|1001-5000|4|1| +1721|2|1001-5000|4|1| +1722|3|1001-5000|4|1| +1723|4|1001-5000|4|1| +1724|5|1001-5000|4|1| +1725|6|1001-5000|4|1| +1726|7|1001-5000|4|1| +1727|8|1001-5000|4|1| +1728|9|1001-5000|4|1| +1729|10|1001-5000|4|1| +1730|11|1001-5000|4|1| +1731|12|1001-5000|4|1| +1732|13|1001-5000|4|1| +1733|14|1001-5000|4|1| +1734|15|1001-5000|4|1| +1735|16|1001-5000|4|1| +1736|17|1001-5000|4|1| +1737|18|1001-5000|4|1| +1738|19|1001-5000|4|1| +1739|20|1001-5000|4|1| +1740|1|5001-10000|4|1| +1741|2|5001-10000|4|1| +1742|3|5001-10000|4|1| +1743|4|5001-10000|4|1| +1744|5|5001-10000|4|1| +1745|6|5001-10000|4|1| +1746|7|5001-10000|4|1| +1747|8|5001-10000|4|1| +1748|9|5001-10000|4|1| +1749|10|5001-10000|4|1| +1750|11|5001-10000|4|1| +1751|12|5001-10000|4|1| +1752|13|5001-10000|4|1| +1753|14|5001-10000|4|1| +1754|15|5001-10000|4|1| +1755|16|5001-10000|4|1| +1756|17|5001-10000|4|1| +1757|18|5001-10000|4|1| +1758|19|5001-10000|4|1| +1759|20|5001-10000|4|1| +1760|1|>10000|4|1| +1761|2|>10000|4|1| +1762|3|>10000|4|1| +1763|4|>10000|4|1| +1764|5|>10000|4|1| +1765|6|>10000|4|1| +1766|7|>10000|4|1| +1767|8|>10000|4|1| +1768|9|>10000|4|1| +1769|10|>10000|4|1| +1770|11|>10000|4|1| +1771|12|>10000|4|1| +1772|13|>10000|4|1| +1773|14|>10000|4|1| +1774|15|>10000|4|1| +1775|16|>10000|4|1| +1776|17|>10000|4|1| +1777|18|>10000|4|1| +1778|19|>10000|4|1| +1779|20|>10000|4|1| +1780|1|Unknown|4|1| +1781|2|Unknown|4|1| +1782|3|Unknown|4|1| +1783|4|Unknown|4|1| +1784|5|Unknown|4|1| +1785|6|Unknown|4|1| +1786|7|Unknown|4|1| +1787|8|Unknown|4|1| +1788|9|Unknown|4|1| +1789|10|Unknown|4|1| +1790|11|Unknown|4|1| +1791|12|Unknown|4|1| +1792|13|Unknown|4|1| +1793|14|Unknown|4|1| +1794|15|Unknown|4|1| +1795|16|Unknown|4|1| +1796|17|Unknown|4|1| +1797|18|Unknown|4|1| +1798|19|Unknown|4|1| +1799|20|Unknown|4|1| +1800|1|0-500|5|1| +1801|2|0-500|5|1| +1802|3|0-500|5|1| +1803|4|0-500|5|1| +1804|5|0-500|5|1| +1805|6|0-500|5|1| +1806|7|0-500|5|1| +1807|8|0-500|5|1| +1808|9|0-500|5|1| +1809|10|0-500|5|1| +1810|11|0-500|5|1| +1811|12|0-500|5|1| +1812|13|0-500|5|1| +1813|14|0-500|5|1| +1814|15|0-500|5|1| +1815|16|0-500|5|1| +1816|17|0-500|5|1| +1817|18|0-500|5|1| +1818|19|0-500|5|1| +1819|20|0-500|5|1| +1820|1|501-1000|5|1| +1821|2|501-1000|5|1| +1822|3|501-1000|5|1| +1823|4|501-1000|5|1| +1824|5|501-1000|5|1| +1825|6|501-1000|5|1| +1826|7|501-1000|5|1| +1827|8|501-1000|5|1| +1828|9|501-1000|5|1| +1829|10|501-1000|5|1| +1830|11|501-1000|5|1| +1831|12|501-1000|5|1| +1832|13|501-1000|5|1| +1833|14|501-1000|5|1| +1834|15|501-1000|5|1| +1835|16|501-1000|5|1| +1836|17|501-1000|5|1| +1837|18|501-1000|5|1| +1838|19|501-1000|5|1| +1839|20|501-1000|5|1| +1840|1|1001-5000|5|1| +1841|2|1001-5000|5|1| +1842|3|1001-5000|5|1| +1843|4|1001-5000|5|1| +1844|5|1001-5000|5|1| +1845|6|1001-5000|5|1| +1846|7|1001-5000|5|1| +1847|8|1001-5000|5|1| +1848|9|1001-5000|5|1| +1849|10|1001-5000|5|1| +1850|11|1001-5000|5|1| +1851|12|1001-5000|5|1| +1852|13|1001-5000|5|1| +1853|14|1001-5000|5|1| +1854|15|1001-5000|5|1| +1855|16|1001-5000|5|1| +1856|17|1001-5000|5|1| +1857|18|1001-5000|5|1| +1858|19|1001-5000|5|1| +1859|20|1001-5000|5|1| +1860|1|5001-10000|5|1| +1861|2|5001-10000|5|1| +1862|3|5001-10000|5|1| +1863|4|5001-10000|5|1| +1864|5|5001-10000|5|1| +1865|6|5001-10000|5|1| +1866|7|5001-10000|5|1| +1867|8|5001-10000|5|1| +1868|9|5001-10000|5|1| +1869|10|5001-10000|5|1| +1870|11|5001-10000|5|1| +1871|12|5001-10000|5|1| +1872|13|5001-10000|5|1| +1873|14|5001-10000|5|1| +1874|15|5001-10000|5|1| +1875|16|5001-10000|5|1| +1876|17|5001-10000|5|1| +1877|18|5001-10000|5|1| +1878|19|5001-10000|5|1| +1879|20|5001-10000|5|1| +1880|1|>10000|5|1| +1881|2|>10000|5|1| +1882|3|>10000|5|1| +1883|4|>10000|5|1| +1884|5|>10000|5|1| +1885|6|>10000|5|1| +1886|7|>10000|5|1| +1887|8|>10000|5|1| +1888|9|>10000|5|1| +1889|10|>10000|5|1| +1890|11|>10000|5|1| +1891|12|>10000|5|1| +1892|13|>10000|5|1| +1893|14|>10000|5|1| +1894|15|>10000|5|1| +1895|16|>10000|5|1| +1896|17|>10000|5|1| +1897|18|>10000|5|1| +1898|19|>10000|5|1| +1899|20|>10000|5|1| +1900|1|Unknown|5|1| +1901|2|Unknown|5|1| +1902|3|Unknown|5|1| +1903|4|Unknown|5|1| +1904|5|Unknown|5|1| +1905|6|Unknown|5|1| +1906|7|Unknown|5|1| +1907|8|Unknown|5|1| +1908|9|Unknown|5|1| +1909|10|Unknown|5|1| +1910|11|Unknown|5|1| +1911|12|Unknown|5|1| +1912|13|Unknown|5|1| +1913|14|Unknown|5|1| +1914|15|Unknown|5|1| +1915|16|Unknown|5|1| +1916|17|Unknown|5|1| +1917|18|Unknown|5|1| +1918|19|Unknown|5|1| +1919|20|Unknown|5|1| +1920|1|0-500|6|1| +1921|2|0-500|6|1| +1922|3|0-500|6|1| +1923|4|0-500|6|1| +1924|5|0-500|6|1| +1925|6|0-500|6|1| +1926|7|0-500|6|1| +1927|8|0-500|6|1| +1928|9|0-500|6|1| +1929|10|0-500|6|1| +1930|11|0-500|6|1| +1931|12|0-500|6|1| +1932|13|0-500|6|1| +1933|14|0-500|6|1| +1934|15|0-500|6|1| +1935|16|0-500|6|1| +1936|17|0-500|6|1| +1937|18|0-500|6|1| +1938|19|0-500|6|1| +1939|20|0-500|6|1| +1940|1|501-1000|6|1| +1941|2|501-1000|6|1| +1942|3|501-1000|6|1| +1943|4|501-1000|6|1| +1944|5|501-1000|6|1| +1945|6|501-1000|6|1| +1946|7|501-1000|6|1| +1947|8|501-1000|6|1| +1948|9|501-1000|6|1| +1949|10|501-1000|6|1| +1950|11|501-1000|6|1| +1951|12|501-1000|6|1| +1952|13|501-1000|6|1| +1953|14|501-1000|6|1| +1954|15|501-1000|6|1| +1955|16|501-1000|6|1| +1956|17|501-1000|6|1| +1957|18|501-1000|6|1| +1958|19|501-1000|6|1| +1959|20|501-1000|6|1| +1960|1|1001-5000|6|1| +1961|2|1001-5000|6|1| +1962|3|1001-5000|6|1| +1963|4|1001-5000|6|1| +1964|5|1001-5000|6|1| +1965|6|1001-5000|6|1| +1966|7|1001-5000|6|1| +1967|8|1001-5000|6|1| +1968|9|1001-5000|6|1| +1969|10|1001-5000|6|1| +1970|11|1001-5000|6|1| +1971|12|1001-5000|6|1| +1972|13|1001-5000|6|1| +1973|14|1001-5000|6|1| +1974|15|1001-5000|6|1| +1975|16|1001-5000|6|1| +1976|17|1001-5000|6|1| +1977|18|1001-5000|6|1| +1978|19|1001-5000|6|1| +1979|20|1001-5000|6|1| +1980|1|5001-10000|6|1| +1981|2|5001-10000|6|1| +1982|3|5001-10000|6|1| +1983|4|5001-10000|6|1| +1984|5|5001-10000|6|1| +1985|6|5001-10000|6|1| +1986|7|5001-10000|6|1| +1987|8|5001-10000|6|1| +1988|9|5001-10000|6|1| +1989|10|5001-10000|6|1| +1990|11|5001-10000|6|1| +1991|12|5001-10000|6|1| +1992|13|5001-10000|6|1| +1993|14|5001-10000|6|1| +1994|15|5001-10000|6|1| +1995|16|5001-10000|6|1| +1996|17|5001-10000|6|1| +1997|18|5001-10000|6|1| +1998|19|5001-10000|6|1| +1999|20|5001-10000|6|1| +2000|1|>10000|6|1| +2001|2|>10000|6|1| +2002|3|>10000|6|1| +2003|4|>10000|6|1| +2004|5|>10000|6|1| +2005|6|>10000|6|1| +2006|7|>10000|6|1| +2007|8|>10000|6|1| +2008|9|>10000|6|1| +2009|10|>10000|6|1| +2010|11|>10000|6|1| +2011|12|>10000|6|1| +2012|13|>10000|6|1| +2013|14|>10000|6|1| +2014|15|>10000|6|1| +2015|16|>10000|6|1| +2016|17|>10000|6|1| +2017|18|>10000|6|1| +2018|19|>10000|6|1| +2019|20|>10000|6|1| +2020|1|Unknown|6|1| +2021|2|Unknown|6|1| +2022|3|Unknown|6|1| +2023|4|Unknown|6|1| +2024|5|Unknown|6|1| +2025|6|Unknown|6|1| +2026|7|Unknown|6|1| +2027|8|Unknown|6|1| +2028|9|Unknown|6|1| +2029|10|Unknown|6|1| +2030|11|Unknown|6|1| +2031|12|Unknown|6|1| +2032|13|Unknown|6|1| +2033|14|Unknown|6|1| +2034|15|Unknown|6|1| +2035|16|Unknown|6|1| +2036|17|Unknown|6|1| +2037|18|Unknown|6|1| +2038|19|Unknown|6|1| +2039|20|Unknown|6|1| +2040|1|0-500|7|1| +2041|2|0-500|7|1| +2042|3|0-500|7|1| +2043|4|0-500|7|1| +2044|5|0-500|7|1| +2045|6|0-500|7|1| +2046|7|0-500|7|1| +2047|8|0-500|7|1| +2048|9|0-500|7|1| +2049|10|0-500|7|1| +2050|11|0-500|7|1| +2051|12|0-500|7|1| +2052|13|0-500|7|1| +2053|14|0-500|7|1| +2054|15|0-500|7|1| +2055|16|0-500|7|1| +2056|17|0-500|7|1| +2057|18|0-500|7|1| +2058|19|0-500|7|1| +2059|20|0-500|7|1| +2060|1|501-1000|7|1| +2061|2|501-1000|7|1| +2062|3|501-1000|7|1| +2063|4|501-1000|7|1| +2064|5|501-1000|7|1| +2065|6|501-1000|7|1| +2066|7|501-1000|7|1| +2067|8|501-1000|7|1| +2068|9|501-1000|7|1| +2069|10|501-1000|7|1| +2070|11|501-1000|7|1| +2071|12|501-1000|7|1| +2072|13|501-1000|7|1| +2073|14|501-1000|7|1| +2074|15|501-1000|7|1| +2075|16|501-1000|7|1| +2076|17|501-1000|7|1| +2077|18|501-1000|7|1| +2078|19|501-1000|7|1| +2079|20|501-1000|7|1| +2080|1|1001-5000|7|1| +2081|2|1001-5000|7|1| +2082|3|1001-5000|7|1| +2083|4|1001-5000|7|1| +2084|5|1001-5000|7|1| +2085|6|1001-5000|7|1| +2086|7|1001-5000|7|1| +2087|8|1001-5000|7|1| +2088|9|1001-5000|7|1| +2089|10|1001-5000|7|1| +2090|11|1001-5000|7|1| +2091|12|1001-5000|7|1| +2092|13|1001-5000|7|1| +2093|14|1001-5000|7|1| +2094|15|1001-5000|7|1| +2095|16|1001-5000|7|1| +2096|17|1001-5000|7|1| +2097|18|1001-5000|7|1| +2098|19|1001-5000|7|1| +2099|20|1001-5000|7|1| +2100|1|5001-10000|7|1| +2101|2|5001-10000|7|1| +2102|3|5001-10000|7|1| +2103|4|5001-10000|7|1| +2104|5|5001-10000|7|1| +2105|6|5001-10000|7|1| +2106|7|5001-10000|7|1| +2107|8|5001-10000|7|1| +2108|9|5001-10000|7|1| +2109|10|5001-10000|7|1| +2110|11|5001-10000|7|1| +2111|12|5001-10000|7|1| +2112|13|5001-10000|7|1| +2113|14|5001-10000|7|1| +2114|15|5001-10000|7|1| +2115|16|5001-10000|7|1| +2116|17|5001-10000|7|1| +2117|18|5001-10000|7|1| +2118|19|5001-10000|7|1| +2119|20|5001-10000|7|1| +2120|1|>10000|7|1| +2121|2|>10000|7|1| +2122|3|>10000|7|1| +2123|4|>10000|7|1| +2124|5|>10000|7|1| +2125|6|>10000|7|1| +2126|7|>10000|7|1| +2127|8|>10000|7|1| +2128|9|>10000|7|1| +2129|10|>10000|7|1| +2130|11|>10000|7|1| +2131|12|>10000|7|1| +2132|13|>10000|7|1| +2133|14|>10000|7|1| +2134|15|>10000|7|1| +2135|16|>10000|7|1| +2136|17|>10000|7|1| +2137|18|>10000|7|1| +2138|19|>10000|7|1| +2139|20|>10000|7|1| +2140|1|Unknown|7|1| +2141|2|Unknown|7|1| +2142|3|Unknown|7|1| +2143|4|Unknown|7|1| +2144|5|Unknown|7|1| +2145|6|Unknown|7|1| +2146|7|Unknown|7|1| +2147|8|Unknown|7|1| +2148|9|Unknown|7|1| +2149|10|Unknown|7|1| +2150|11|Unknown|7|1| +2151|12|Unknown|7|1| +2152|13|Unknown|7|1| +2153|14|Unknown|7|1| +2154|15|Unknown|7|1| +2155|16|Unknown|7|1| +2156|17|Unknown|7|1| +2157|18|Unknown|7|1| +2158|19|Unknown|7|1| +2159|20|Unknown|7|1| +2160|1|0-500|8|1| +2161|2|0-500|8|1| +2162|3|0-500|8|1| +2163|4|0-500|8|1| +2164|5|0-500|8|1| +2165|6|0-500|8|1| +2166|7|0-500|8|1| +2167|8|0-500|8|1| +2168|9|0-500|8|1| +2169|10|0-500|8|1| +2170|11|0-500|8|1| +2171|12|0-500|8|1| +2172|13|0-500|8|1| +2173|14|0-500|8|1| +2174|15|0-500|8|1| +2175|16|0-500|8|1| +2176|17|0-500|8|1| +2177|18|0-500|8|1| +2178|19|0-500|8|1| +2179|20|0-500|8|1| +2180|1|501-1000|8|1| +2181|2|501-1000|8|1| +2182|3|501-1000|8|1| +2183|4|501-1000|8|1| +2184|5|501-1000|8|1| +2185|6|501-1000|8|1| +2186|7|501-1000|8|1| +2187|8|501-1000|8|1| +2188|9|501-1000|8|1| +2189|10|501-1000|8|1| +2190|11|501-1000|8|1| +2191|12|501-1000|8|1| +2192|13|501-1000|8|1| +2193|14|501-1000|8|1| +2194|15|501-1000|8|1| +2195|16|501-1000|8|1| +2196|17|501-1000|8|1| +2197|18|501-1000|8|1| +2198|19|501-1000|8|1| +2199|20|501-1000|8|1| +2200|1|1001-5000|8|1| +2201|2|1001-5000|8|1| +2202|3|1001-5000|8|1| +2203|4|1001-5000|8|1| +2204|5|1001-5000|8|1| +2205|6|1001-5000|8|1| +2206|7|1001-5000|8|1| +2207|8|1001-5000|8|1| +2208|9|1001-5000|8|1| +2209|10|1001-5000|8|1| +2210|11|1001-5000|8|1| +2211|12|1001-5000|8|1| +2212|13|1001-5000|8|1| +2213|14|1001-5000|8|1| +2214|15|1001-5000|8|1| +2215|16|1001-5000|8|1| +2216|17|1001-5000|8|1| +2217|18|1001-5000|8|1| +2218|19|1001-5000|8|1| +2219|20|1001-5000|8|1| +2220|1|5001-10000|8|1| +2221|2|5001-10000|8|1| +2222|3|5001-10000|8|1| +2223|4|5001-10000|8|1| +2224|5|5001-10000|8|1| +2225|6|5001-10000|8|1| +2226|7|5001-10000|8|1| +2227|8|5001-10000|8|1| +2228|9|5001-10000|8|1| +2229|10|5001-10000|8|1| +2230|11|5001-10000|8|1| +2231|12|5001-10000|8|1| +2232|13|5001-10000|8|1| +2233|14|5001-10000|8|1| +2234|15|5001-10000|8|1| +2235|16|5001-10000|8|1| +2236|17|5001-10000|8|1| +2237|18|5001-10000|8|1| +2238|19|5001-10000|8|1| +2239|20|5001-10000|8|1| +2240|1|>10000|8|1| +2241|2|>10000|8|1| +2242|3|>10000|8|1| +2243|4|>10000|8|1| +2244|5|>10000|8|1| +2245|6|>10000|8|1| +2246|7|>10000|8|1| +2247|8|>10000|8|1| +2248|9|>10000|8|1| +2249|10|>10000|8|1| +2250|11|>10000|8|1| +2251|12|>10000|8|1| +2252|13|>10000|8|1| +2253|14|>10000|8|1| +2254|15|>10000|8|1| +2255|16|>10000|8|1| +2256|17|>10000|8|1| +2257|18|>10000|8|1| +2258|19|>10000|8|1| +2259|20|>10000|8|1| +2260|1|Unknown|8|1| +2261|2|Unknown|8|1| +2262|3|Unknown|8|1| +2263|4|Unknown|8|1| +2264|5|Unknown|8|1| +2265|6|Unknown|8|1| +2266|7|Unknown|8|1| +2267|8|Unknown|8|1| +2268|9|Unknown|8|1| +2269|10|Unknown|8|1| +2270|11|Unknown|8|1| +2271|12|Unknown|8|1| +2272|13|Unknown|8|1| +2273|14|Unknown|8|1| +2274|15|Unknown|8|1| +2275|16|Unknown|8|1| +2276|17|Unknown|8|1| +2277|18|Unknown|8|1| +2278|19|Unknown|8|1| +2279|20|Unknown|8|1| +2280|1|0-500|9|1| +2281|2|0-500|9|1| +2282|3|0-500|9|1| +2283|4|0-500|9|1| +2284|5|0-500|9|1| +2285|6|0-500|9|1| +2286|7|0-500|9|1| +2287|8|0-500|9|1| +2288|9|0-500|9|1| +2289|10|0-500|9|1| +2290|11|0-500|9|1| +2291|12|0-500|9|1| +2292|13|0-500|9|1| +2293|14|0-500|9|1| +2294|15|0-500|9|1| +2295|16|0-500|9|1| +2296|17|0-500|9|1| +2297|18|0-500|9|1| +2298|19|0-500|9|1| +2299|20|0-500|9|1| +2300|1|501-1000|9|1| +2301|2|501-1000|9|1| +2302|3|501-1000|9|1| +2303|4|501-1000|9|1| +2304|5|501-1000|9|1| +2305|6|501-1000|9|1| +2306|7|501-1000|9|1| +2307|8|501-1000|9|1| +2308|9|501-1000|9|1| +2309|10|501-1000|9|1| +2310|11|501-1000|9|1| +2311|12|501-1000|9|1| +2312|13|501-1000|9|1| +2313|14|501-1000|9|1| +2314|15|501-1000|9|1| +2315|16|501-1000|9|1| +2316|17|501-1000|9|1| +2317|18|501-1000|9|1| +2318|19|501-1000|9|1| +2319|20|501-1000|9|1| +2320|1|1001-5000|9|1| +2321|2|1001-5000|9|1| +2322|3|1001-5000|9|1| +2323|4|1001-5000|9|1| +2324|5|1001-5000|9|1| +2325|6|1001-5000|9|1| +2326|7|1001-5000|9|1| +2327|8|1001-5000|9|1| +2328|9|1001-5000|9|1| +2329|10|1001-5000|9|1| +2330|11|1001-5000|9|1| +2331|12|1001-5000|9|1| +2332|13|1001-5000|9|1| +2333|14|1001-5000|9|1| +2334|15|1001-5000|9|1| +2335|16|1001-5000|9|1| +2336|17|1001-5000|9|1| +2337|18|1001-5000|9|1| +2338|19|1001-5000|9|1| +2339|20|1001-5000|9|1| +2340|1|5001-10000|9|1| +2341|2|5001-10000|9|1| +2342|3|5001-10000|9|1| +2343|4|5001-10000|9|1| +2344|5|5001-10000|9|1| +2345|6|5001-10000|9|1| +2346|7|5001-10000|9|1| +2347|8|5001-10000|9|1| +2348|9|5001-10000|9|1| +2349|10|5001-10000|9|1| +2350|11|5001-10000|9|1| +2351|12|5001-10000|9|1| +2352|13|5001-10000|9|1| +2353|14|5001-10000|9|1| +2354|15|5001-10000|9|1| +2355|16|5001-10000|9|1| +2356|17|5001-10000|9|1| +2357|18|5001-10000|9|1| +2358|19|5001-10000|9|1| +2359|20|5001-10000|9|1| +2360|1|>10000|9|1| +2361|2|>10000|9|1| +2362|3|>10000|9|1| +2363|4|>10000|9|1| +2364|5|>10000|9|1| +2365|6|>10000|9|1| +2366|7|>10000|9|1| +2367|8|>10000|9|1| +2368|9|>10000|9|1| +2369|10|>10000|9|1| +2370|11|>10000|9|1| +2371|12|>10000|9|1| +2372|13|>10000|9|1| +2373|14|>10000|9|1| +2374|15|>10000|9|1| +2375|16|>10000|9|1| +2376|17|>10000|9|1| +2377|18|>10000|9|1| +2378|19|>10000|9|1| +2379|20|>10000|9|1| +2380|1|Unknown|9|1| +2381|2|Unknown|9|1| +2382|3|Unknown|9|1| +2383|4|Unknown|9|1| +2384|5|Unknown|9|1| +2385|6|Unknown|9|1| +2386|7|Unknown|9|1| +2387|8|Unknown|9|1| +2388|9|Unknown|9|1| +2389|10|Unknown|9|1| +2390|11|Unknown|9|1| +2391|12|Unknown|9|1| +2392|13|Unknown|9|1| +2393|14|Unknown|9|1| +2394|15|Unknown|9|1| +2395|16|Unknown|9|1| +2396|17|Unknown|9|1| +2397|18|Unknown|9|1| +2398|19|Unknown|9|1| +2399|20|Unknown|9|1| +2400|1|0-500|0|2| +2401|2|0-500|0|2| +2402|3|0-500|0|2| +2403|4|0-500|0|2| +2404|5|0-500|0|2| +2405|6|0-500|0|2| +2406|7|0-500|0|2| +2407|8|0-500|0|2| +2408|9|0-500|0|2| +2409|10|0-500|0|2| +2410|11|0-500|0|2| +2411|12|0-500|0|2| +2412|13|0-500|0|2| +2413|14|0-500|0|2| +2414|15|0-500|0|2| +2415|16|0-500|0|2| +2416|17|0-500|0|2| +2417|18|0-500|0|2| +2418|19|0-500|0|2| +2419|20|0-500|0|2| +2420|1|501-1000|0|2| +2421|2|501-1000|0|2| +2422|3|501-1000|0|2| +2423|4|501-1000|0|2| +2424|5|501-1000|0|2| +2425|6|501-1000|0|2| +2426|7|501-1000|0|2| +2427|8|501-1000|0|2| +2428|9|501-1000|0|2| +2429|10|501-1000|0|2| +2430|11|501-1000|0|2| +2431|12|501-1000|0|2| +2432|13|501-1000|0|2| +2433|14|501-1000|0|2| +2434|15|501-1000|0|2| +2435|16|501-1000|0|2| +2436|17|501-1000|0|2| +2437|18|501-1000|0|2| +2438|19|501-1000|0|2| +2439|20|501-1000|0|2| +2440|1|1001-5000|0|2| +2441|2|1001-5000|0|2| +2442|3|1001-5000|0|2| +2443|4|1001-5000|0|2| +2444|5|1001-5000|0|2| +2445|6|1001-5000|0|2| +2446|7|1001-5000|0|2| +2447|8|1001-5000|0|2| +2448|9|1001-5000|0|2| +2449|10|1001-5000|0|2| +2450|11|1001-5000|0|2| +2451|12|1001-5000|0|2| +2452|13|1001-5000|0|2| +2453|14|1001-5000|0|2| +2454|15|1001-5000|0|2| +2455|16|1001-5000|0|2| +2456|17|1001-5000|0|2| +2457|18|1001-5000|0|2| +2458|19|1001-5000|0|2| +2459|20|1001-5000|0|2| +2460|1|5001-10000|0|2| +2461|2|5001-10000|0|2| +2462|3|5001-10000|0|2| +2463|4|5001-10000|0|2| +2464|5|5001-10000|0|2| +2465|6|5001-10000|0|2| +2466|7|5001-10000|0|2| +2467|8|5001-10000|0|2| +2468|9|5001-10000|0|2| +2469|10|5001-10000|0|2| +2470|11|5001-10000|0|2| +2471|12|5001-10000|0|2| +2472|13|5001-10000|0|2| +2473|14|5001-10000|0|2| +2474|15|5001-10000|0|2| +2475|16|5001-10000|0|2| +2476|17|5001-10000|0|2| +2477|18|5001-10000|0|2| +2478|19|5001-10000|0|2| +2479|20|5001-10000|0|2| +2480|1|>10000|0|2| +2481|2|>10000|0|2| +2482|3|>10000|0|2| +2483|4|>10000|0|2| +2484|5|>10000|0|2| +2485|6|>10000|0|2| +2486|7|>10000|0|2| +2487|8|>10000|0|2| +2488|9|>10000|0|2| +2489|10|>10000|0|2| +2490|11|>10000|0|2| +2491|12|>10000|0|2| +2492|13|>10000|0|2| +2493|14|>10000|0|2| +2494|15|>10000|0|2| +2495|16|>10000|0|2| +2496|17|>10000|0|2| +2497|18|>10000|0|2| +2498|19|>10000|0|2| +2499|20|>10000|0|2| +2500|1|Unknown|0|2| +2501|2|Unknown|0|2| +2502|3|Unknown|0|2| +2503|4|Unknown|0|2| +2504|5|Unknown|0|2| +2505|6|Unknown|0|2| +2506|7|Unknown|0|2| +2507|8|Unknown|0|2| +2508|9|Unknown|0|2| +2509|10|Unknown|0|2| +2510|11|Unknown|0|2| +2511|12|Unknown|0|2| +2512|13|Unknown|0|2| +2513|14|Unknown|0|2| +2514|15|Unknown|0|2| +2515|16|Unknown|0|2| +2516|17|Unknown|0|2| +2517|18|Unknown|0|2| +2518|19|Unknown|0|2| +2519|20|Unknown|0|2| +2520|1|0-500|1|2| +2521|2|0-500|1|2| +2522|3|0-500|1|2| +2523|4|0-500|1|2| +2524|5|0-500|1|2| +2525|6|0-500|1|2| +2526|7|0-500|1|2| +2527|8|0-500|1|2| +2528|9|0-500|1|2| +2529|10|0-500|1|2| +2530|11|0-500|1|2| +2531|12|0-500|1|2| +2532|13|0-500|1|2| +2533|14|0-500|1|2| +2534|15|0-500|1|2| +2535|16|0-500|1|2| +2536|17|0-500|1|2| +2537|18|0-500|1|2| +2538|19|0-500|1|2| +2539|20|0-500|1|2| +2540|1|501-1000|1|2| +2541|2|501-1000|1|2| +2542|3|501-1000|1|2| +2543|4|501-1000|1|2| +2544|5|501-1000|1|2| +2545|6|501-1000|1|2| +2546|7|501-1000|1|2| +2547|8|501-1000|1|2| +2548|9|501-1000|1|2| +2549|10|501-1000|1|2| +2550|11|501-1000|1|2| +2551|12|501-1000|1|2| +2552|13|501-1000|1|2| +2553|14|501-1000|1|2| +2554|15|501-1000|1|2| +2555|16|501-1000|1|2| +2556|17|501-1000|1|2| +2557|18|501-1000|1|2| +2558|19|501-1000|1|2| +2559|20|501-1000|1|2| +2560|1|1001-5000|1|2| +2561|2|1001-5000|1|2| +2562|3|1001-5000|1|2| +2563|4|1001-5000|1|2| +2564|5|1001-5000|1|2| +2565|6|1001-5000|1|2| +2566|7|1001-5000|1|2| +2567|8|1001-5000|1|2| +2568|9|1001-5000|1|2| +2569|10|1001-5000|1|2| +2570|11|1001-5000|1|2| +2571|12|1001-5000|1|2| +2572|13|1001-5000|1|2| +2573|14|1001-5000|1|2| +2574|15|1001-5000|1|2| +2575|16|1001-5000|1|2| +2576|17|1001-5000|1|2| +2577|18|1001-5000|1|2| +2578|19|1001-5000|1|2| +2579|20|1001-5000|1|2| +2580|1|5001-10000|1|2| +2581|2|5001-10000|1|2| +2582|3|5001-10000|1|2| +2583|4|5001-10000|1|2| +2584|5|5001-10000|1|2| +2585|6|5001-10000|1|2| +2586|7|5001-10000|1|2| +2587|8|5001-10000|1|2| +2588|9|5001-10000|1|2| +2589|10|5001-10000|1|2| +2590|11|5001-10000|1|2| +2591|12|5001-10000|1|2| +2592|13|5001-10000|1|2| +2593|14|5001-10000|1|2| +2594|15|5001-10000|1|2| +2595|16|5001-10000|1|2| +2596|17|5001-10000|1|2| +2597|18|5001-10000|1|2| +2598|19|5001-10000|1|2| +2599|20|5001-10000|1|2| +2600|1|>10000|1|2| +2601|2|>10000|1|2| +2602|3|>10000|1|2| +2603|4|>10000|1|2| +2604|5|>10000|1|2| +2605|6|>10000|1|2| +2606|7|>10000|1|2| +2607|8|>10000|1|2| +2608|9|>10000|1|2| +2609|10|>10000|1|2| +2610|11|>10000|1|2| +2611|12|>10000|1|2| +2612|13|>10000|1|2| +2613|14|>10000|1|2| +2614|15|>10000|1|2| +2615|16|>10000|1|2| +2616|17|>10000|1|2| +2617|18|>10000|1|2| +2618|19|>10000|1|2| +2619|20|>10000|1|2| +2620|1|Unknown|1|2| +2621|2|Unknown|1|2| +2622|3|Unknown|1|2| +2623|4|Unknown|1|2| +2624|5|Unknown|1|2| +2625|6|Unknown|1|2| +2626|7|Unknown|1|2| +2627|8|Unknown|1|2| +2628|9|Unknown|1|2| +2629|10|Unknown|1|2| +2630|11|Unknown|1|2| +2631|12|Unknown|1|2| +2632|13|Unknown|1|2| +2633|14|Unknown|1|2| +2634|15|Unknown|1|2| +2635|16|Unknown|1|2| +2636|17|Unknown|1|2| +2637|18|Unknown|1|2| +2638|19|Unknown|1|2| +2639|20|Unknown|1|2| +2640|1|0-500|2|2| +2641|2|0-500|2|2| +2642|3|0-500|2|2| +2643|4|0-500|2|2| +2644|5|0-500|2|2| +2645|6|0-500|2|2| +2646|7|0-500|2|2| +2647|8|0-500|2|2| +2648|9|0-500|2|2| +2649|10|0-500|2|2| +2650|11|0-500|2|2| +2651|12|0-500|2|2| +2652|13|0-500|2|2| +2653|14|0-500|2|2| +2654|15|0-500|2|2| +2655|16|0-500|2|2| +2656|17|0-500|2|2| +2657|18|0-500|2|2| +2658|19|0-500|2|2| +2659|20|0-500|2|2| +2660|1|501-1000|2|2| +2661|2|501-1000|2|2| +2662|3|501-1000|2|2| +2663|4|501-1000|2|2| +2664|5|501-1000|2|2| +2665|6|501-1000|2|2| +2666|7|501-1000|2|2| +2667|8|501-1000|2|2| +2668|9|501-1000|2|2| +2669|10|501-1000|2|2| +2670|11|501-1000|2|2| +2671|12|501-1000|2|2| +2672|13|501-1000|2|2| +2673|14|501-1000|2|2| +2674|15|501-1000|2|2| +2675|16|501-1000|2|2| +2676|17|501-1000|2|2| +2677|18|501-1000|2|2| +2678|19|501-1000|2|2| +2679|20|501-1000|2|2| +2680|1|1001-5000|2|2| +2681|2|1001-5000|2|2| +2682|3|1001-5000|2|2| +2683|4|1001-5000|2|2| +2684|5|1001-5000|2|2| +2685|6|1001-5000|2|2| +2686|7|1001-5000|2|2| +2687|8|1001-5000|2|2| +2688|9|1001-5000|2|2| +2689|10|1001-5000|2|2| +2690|11|1001-5000|2|2| +2691|12|1001-5000|2|2| +2692|13|1001-5000|2|2| +2693|14|1001-5000|2|2| +2694|15|1001-5000|2|2| +2695|16|1001-5000|2|2| +2696|17|1001-5000|2|2| +2697|18|1001-5000|2|2| +2698|19|1001-5000|2|2| +2699|20|1001-5000|2|2| +2700|1|5001-10000|2|2| +2701|2|5001-10000|2|2| +2702|3|5001-10000|2|2| +2703|4|5001-10000|2|2| +2704|5|5001-10000|2|2| +2705|6|5001-10000|2|2| +2706|7|5001-10000|2|2| +2707|8|5001-10000|2|2| +2708|9|5001-10000|2|2| +2709|10|5001-10000|2|2| +2710|11|5001-10000|2|2| +2711|12|5001-10000|2|2| +2712|13|5001-10000|2|2| +2713|14|5001-10000|2|2| +2714|15|5001-10000|2|2| +2715|16|5001-10000|2|2| +2716|17|5001-10000|2|2| +2717|18|5001-10000|2|2| +2718|19|5001-10000|2|2| +2719|20|5001-10000|2|2| +2720|1|>10000|2|2| +2721|2|>10000|2|2| +2722|3|>10000|2|2| +2723|4|>10000|2|2| +2724|5|>10000|2|2| +2725|6|>10000|2|2| +2726|7|>10000|2|2| +2727|8|>10000|2|2| +2728|9|>10000|2|2| +2729|10|>10000|2|2| +2730|11|>10000|2|2| +2731|12|>10000|2|2| +2732|13|>10000|2|2| +2733|14|>10000|2|2| +2734|15|>10000|2|2| +2735|16|>10000|2|2| +2736|17|>10000|2|2| +2737|18|>10000|2|2| +2738|19|>10000|2|2| +2739|20|>10000|2|2| +2740|1|Unknown|2|2| +2741|2|Unknown|2|2| +2742|3|Unknown|2|2| +2743|4|Unknown|2|2| +2744|5|Unknown|2|2| +2745|6|Unknown|2|2| +2746|7|Unknown|2|2| +2747|8|Unknown|2|2| +2748|9|Unknown|2|2| +2749|10|Unknown|2|2| +2750|11|Unknown|2|2| +2751|12|Unknown|2|2| +2752|13|Unknown|2|2| +2753|14|Unknown|2|2| +2754|15|Unknown|2|2| +2755|16|Unknown|2|2| +2756|17|Unknown|2|2| +2757|18|Unknown|2|2| +2758|19|Unknown|2|2| +2759|20|Unknown|2|2| +2760|1|0-500|3|2| +2761|2|0-500|3|2| +2762|3|0-500|3|2| +2763|4|0-500|3|2| +2764|5|0-500|3|2| +2765|6|0-500|3|2| +2766|7|0-500|3|2| +2767|8|0-500|3|2| +2768|9|0-500|3|2| +2769|10|0-500|3|2| +2770|11|0-500|3|2| +2771|12|0-500|3|2| +2772|13|0-500|3|2| +2773|14|0-500|3|2| +2774|15|0-500|3|2| +2775|16|0-500|3|2| +2776|17|0-500|3|2| +2777|18|0-500|3|2| +2778|19|0-500|3|2| +2779|20|0-500|3|2| +2780|1|501-1000|3|2| +2781|2|501-1000|3|2| +2782|3|501-1000|3|2| +2783|4|501-1000|3|2| +2784|5|501-1000|3|2| +2785|6|501-1000|3|2| +2786|7|501-1000|3|2| +2787|8|501-1000|3|2| +2788|9|501-1000|3|2| +2789|10|501-1000|3|2| +2790|11|501-1000|3|2| +2791|12|501-1000|3|2| +2792|13|501-1000|3|2| +2793|14|501-1000|3|2| +2794|15|501-1000|3|2| +2795|16|501-1000|3|2| +2796|17|501-1000|3|2| +2797|18|501-1000|3|2| +2798|19|501-1000|3|2| +2799|20|501-1000|3|2| +2800|1|1001-5000|3|2| +2801|2|1001-5000|3|2| +2802|3|1001-5000|3|2| +2803|4|1001-5000|3|2| +2804|5|1001-5000|3|2| +2805|6|1001-5000|3|2| +2806|7|1001-5000|3|2| +2807|8|1001-5000|3|2| +2808|9|1001-5000|3|2| +2809|10|1001-5000|3|2| +2810|11|1001-5000|3|2| +2811|12|1001-5000|3|2| +2812|13|1001-5000|3|2| +2813|14|1001-5000|3|2| +2814|15|1001-5000|3|2| +2815|16|1001-5000|3|2| +2816|17|1001-5000|3|2| +2817|18|1001-5000|3|2| +2818|19|1001-5000|3|2| +2819|20|1001-5000|3|2| +2820|1|5001-10000|3|2| +2821|2|5001-10000|3|2| +2822|3|5001-10000|3|2| +2823|4|5001-10000|3|2| +2824|5|5001-10000|3|2| +2825|6|5001-10000|3|2| +2826|7|5001-10000|3|2| +2827|8|5001-10000|3|2| +2828|9|5001-10000|3|2| +2829|10|5001-10000|3|2| +2830|11|5001-10000|3|2| +2831|12|5001-10000|3|2| +2832|13|5001-10000|3|2| +2833|14|5001-10000|3|2| +2834|15|5001-10000|3|2| +2835|16|5001-10000|3|2| +2836|17|5001-10000|3|2| +2837|18|5001-10000|3|2| +2838|19|5001-10000|3|2| +2839|20|5001-10000|3|2| +2840|1|>10000|3|2| +2841|2|>10000|3|2| +2842|3|>10000|3|2| +2843|4|>10000|3|2| +2844|5|>10000|3|2| +2845|6|>10000|3|2| +2846|7|>10000|3|2| +2847|8|>10000|3|2| +2848|9|>10000|3|2| +2849|10|>10000|3|2| +2850|11|>10000|3|2| +2851|12|>10000|3|2| +2852|13|>10000|3|2| +2853|14|>10000|3|2| +2854|15|>10000|3|2| +2855|16|>10000|3|2| +2856|17|>10000|3|2| +2857|18|>10000|3|2| +2858|19|>10000|3|2| +2859|20|>10000|3|2| +2860|1|Unknown|3|2| +2861|2|Unknown|3|2| +2862|3|Unknown|3|2| +2863|4|Unknown|3|2| +2864|5|Unknown|3|2| +2865|6|Unknown|3|2| +2866|7|Unknown|3|2| +2867|8|Unknown|3|2| +2868|9|Unknown|3|2| +2869|10|Unknown|3|2| +2870|11|Unknown|3|2| +2871|12|Unknown|3|2| +2872|13|Unknown|3|2| +2873|14|Unknown|3|2| +2874|15|Unknown|3|2| +2875|16|Unknown|3|2| +2876|17|Unknown|3|2| +2877|18|Unknown|3|2| +2878|19|Unknown|3|2| +2879|20|Unknown|3|2| +2880|1|0-500|4|2| +2881|2|0-500|4|2| +2882|3|0-500|4|2| +2883|4|0-500|4|2| +2884|5|0-500|4|2| +2885|6|0-500|4|2| +2886|7|0-500|4|2| +2887|8|0-500|4|2| +2888|9|0-500|4|2| +2889|10|0-500|4|2| +2890|11|0-500|4|2| +2891|12|0-500|4|2| +2892|13|0-500|4|2| +2893|14|0-500|4|2| +2894|15|0-500|4|2| +2895|16|0-500|4|2| +2896|17|0-500|4|2| +2897|18|0-500|4|2| +2898|19|0-500|4|2| +2899|20|0-500|4|2| +2900|1|501-1000|4|2| +2901|2|501-1000|4|2| +2902|3|501-1000|4|2| +2903|4|501-1000|4|2| +2904|5|501-1000|4|2| +2905|6|501-1000|4|2| +2906|7|501-1000|4|2| +2907|8|501-1000|4|2| +2908|9|501-1000|4|2| +2909|10|501-1000|4|2| +2910|11|501-1000|4|2| +2911|12|501-1000|4|2| +2912|13|501-1000|4|2| +2913|14|501-1000|4|2| +2914|15|501-1000|4|2| +2915|16|501-1000|4|2| +2916|17|501-1000|4|2| +2917|18|501-1000|4|2| +2918|19|501-1000|4|2| +2919|20|501-1000|4|2| +2920|1|1001-5000|4|2| +2921|2|1001-5000|4|2| +2922|3|1001-5000|4|2| +2923|4|1001-5000|4|2| +2924|5|1001-5000|4|2| +2925|6|1001-5000|4|2| +2926|7|1001-5000|4|2| +2927|8|1001-5000|4|2| +2928|9|1001-5000|4|2| +2929|10|1001-5000|4|2| +2930|11|1001-5000|4|2| +2931|12|1001-5000|4|2| +2932|13|1001-5000|4|2| +2933|14|1001-5000|4|2| +2934|15|1001-5000|4|2| +2935|16|1001-5000|4|2| +2936|17|1001-5000|4|2| +2937|18|1001-5000|4|2| +2938|19|1001-5000|4|2| +2939|20|1001-5000|4|2| +2940|1|5001-10000|4|2| +2941|2|5001-10000|4|2| +2942|3|5001-10000|4|2| +2943|4|5001-10000|4|2| +2944|5|5001-10000|4|2| +2945|6|5001-10000|4|2| +2946|7|5001-10000|4|2| +2947|8|5001-10000|4|2| +2948|9|5001-10000|4|2| +2949|10|5001-10000|4|2| +2950|11|5001-10000|4|2| +2951|12|5001-10000|4|2| +2952|13|5001-10000|4|2| +2953|14|5001-10000|4|2| +2954|15|5001-10000|4|2| +2955|16|5001-10000|4|2| +2956|17|5001-10000|4|2| +2957|18|5001-10000|4|2| +2958|19|5001-10000|4|2| +2959|20|5001-10000|4|2| +2960|1|>10000|4|2| +2961|2|>10000|4|2| +2962|3|>10000|4|2| +2963|4|>10000|4|2| +2964|5|>10000|4|2| +2965|6|>10000|4|2| +2966|7|>10000|4|2| +2967|8|>10000|4|2| +2968|9|>10000|4|2| +2969|10|>10000|4|2| +2970|11|>10000|4|2| +2971|12|>10000|4|2| +2972|13|>10000|4|2| +2973|14|>10000|4|2| +2974|15|>10000|4|2| +2975|16|>10000|4|2| +2976|17|>10000|4|2| +2977|18|>10000|4|2| +2978|19|>10000|4|2| +2979|20|>10000|4|2| +2980|1|Unknown|4|2| +2981|2|Unknown|4|2| +2982|3|Unknown|4|2| +2983|4|Unknown|4|2| +2984|5|Unknown|4|2| +2985|6|Unknown|4|2| +2986|7|Unknown|4|2| +2987|8|Unknown|4|2| +2988|9|Unknown|4|2| +2989|10|Unknown|4|2| +2990|11|Unknown|4|2| +2991|12|Unknown|4|2| +2992|13|Unknown|4|2| +2993|14|Unknown|4|2| +2994|15|Unknown|4|2| +2995|16|Unknown|4|2| +2996|17|Unknown|4|2| +2997|18|Unknown|4|2| +2998|19|Unknown|4|2| +2999|20|Unknown|4|2| +3000|1|0-500|5|2| +3001|2|0-500|5|2| +3002|3|0-500|5|2| +3003|4|0-500|5|2| +3004|5|0-500|5|2| +3005|6|0-500|5|2| +3006|7|0-500|5|2| +3007|8|0-500|5|2| +3008|9|0-500|5|2| +3009|10|0-500|5|2| +3010|11|0-500|5|2| +3011|12|0-500|5|2| +3012|13|0-500|5|2| +3013|14|0-500|5|2| +3014|15|0-500|5|2| +3015|16|0-500|5|2| +3016|17|0-500|5|2| +3017|18|0-500|5|2| +3018|19|0-500|5|2| +3019|20|0-500|5|2| +3020|1|501-1000|5|2| +3021|2|501-1000|5|2| +3022|3|501-1000|5|2| +3023|4|501-1000|5|2| +3024|5|501-1000|5|2| +3025|6|501-1000|5|2| +3026|7|501-1000|5|2| +3027|8|501-1000|5|2| +3028|9|501-1000|5|2| +3029|10|501-1000|5|2| +3030|11|501-1000|5|2| +3031|12|501-1000|5|2| +3032|13|501-1000|5|2| +3033|14|501-1000|5|2| +3034|15|501-1000|5|2| +3035|16|501-1000|5|2| +3036|17|501-1000|5|2| +3037|18|501-1000|5|2| +3038|19|501-1000|5|2| +3039|20|501-1000|5|2| +3040|1|1001-5000|5|2| +3041|2|1001-5000|5|2| +3042|3|1001-5000|5|2| +3043|4|1001-5000|5|2| +3044|5|1001-5000|5|2| +3045|6|1001-5000|5|2| +3046|7|1001-5000|5|2| +3047|8|1001-5000|5|2| +3048|9|1001-5000|5|2| +3049|10|1001-5000|5|2| +3050|11|1001-5000|5|2| +3051|12|1001-5000|5|2| +3052|13|1001-5000|5|2| +3053|14|1001-5000|5|2| +3054|15|1001-5000|5|2| +3055|16|1001-5000|5|2| +3056|17|1001-5000|5|2| +3057|18|1001-5000|5|2| +3058|19|1001-5000|5|2| +3059|20|1001-5000|5|2| +3060|1|5001-10000|5|2| +3061|2|5001-10000|5|2| +3062|3|5001-10000|5|2| +3063|4|5001-10000|5|2| +3064|5|5001-10000|5|2| +3065|6|5001-10000|5|2| +3066|7|5001-10000|5|2| +3067|8|5001-10000|5|2| +3068|9|5001-10000|5|2| +3069|10|5001-10000|5|2| +3070|11|5001-10000|5|2| +3071|12|5001-10000|5|2| +3072|13|5001-10000|5|2| +3073|14|5001-10000|5|2| +3074|15|5001-10000|5|2| +3075|16|5001-10000|5|2| +3076|17|5001-10000|5|2| +3077|18|5001-10000|5|2| +3078|19|5001-10000|5|2| +3079|20|5001-10000|5|2| +3080|1|>10000|5|2| +3081|2|>10000|5|2| +3082|3|>10000|5|2| +3083|4|>10000|5|2| +3084|5|>10000|5|2| +3085|6|>10000|5|2| +3086|7|>10000|5|2| +3087|8|>10000|5|2| +3088|9|>10000|5|2| +3089|10|>10000|5|2| +3090|11|>10000|5|2| +3091|12|>10000|5|2| +3092|13|>10000|5|2| +3093|14|>10000|5|2| +3094|15|>10000|5|2| +3095|16|>10000|5|2| +3096|17|>10000|5|2| +3097|18|>10000|5|2| +3098|19|>10000|5|2| +3099|20|>10000|5|2| +3100|1|Unknown|5|2| +3101|2|Unknown|5|2| +3102|3|Unknown|5|2| +3103|4|Unknown|5|2| +3104|5|Unknown|5|2| +3105|6|Unknown|5|2| +3106|7|Unknown|5|2| +3107|8|Unknown|5|2| +3108|9|Unknown|5|2| +3109|10|Unknown|5|2| +3110|11|Unknown|5|2| +3111|12|Unknown|5|2| +3112|13|Unknown|5|2| +3113|14|Unknown|5|2| +3114|15|Unknown|5|2| +3115|16|Unknown|5|2| +3116|17|Unknown|5|2| +3117|18|Unknown|5|2| +3118|19|Unknown|5|2| +3119|20|Unknown|5|2| +3120|1|0-500|6|2| +3121|2|0-500|6|2| +3122|3|0-500|6|2| +3123|4|0-500|6|2| +3124|5|0-500|6|2| +3125|6|0-500|6|2| +3126|7|0-500|6|2| +3127|8|0-500|6|2| +3128|9|0-500|6|2| +3129|10|0-500|6|2| +3130|11|0-500|6|2| +3131|12|0-500|6|2| +3132|13|0-500|6|2| +3133|14|0-500|6|2| +3134|15|0-500|6|2| +3135|16|0-500|6|2| +3136|17|0-500|6|2| +3137|18|0-500|6|2| +3138|19|0-500|6|2| +3139|20|0-500|6|2| +3140|1|501-1000|6|2| +3141|2|501-1000|6|2| +3142|3|501-1000|6|2| +3143|4|501-1000|6|2| +3144|5|501-1000|6|2| +3145|6|501-1000|6|2| +3146|7|501-1000|6|2| +3147|8|501-1000|6|2| +3148|9|501-1000|6|2| +3149|10|501-1000|6|2| +3150|11|501-1000|6|2| +3151|12|501-1000|6|2| +3152|13|501-1000|6|2| +3153|14|501-1000|6|2| +3154|15|501-1000|6|2| +3155|16|501-1000|6|2| +3156|17|501-1000|6|2| +3157|18|501-1000|6|2| +3158|19|501-1000|6|2| +3159|20|501-1000|6|2| +3160|1|1001-5000|6|2| +3161|2|1001-5000|6|2| +3162|3|1001-5000|6|2| +3163|4|1001-5000|6|2| +3164|5|1001-5000|6|2| +3165|6|1001-5000|6|2| +3166|7|1001-5000|6|2| +3167|8|1001-5000|6|2| +3168|9|1001-5000|6|2| +3169|10|1001-5000|6|2| +3170|11|1001-5000|6|2| +3171|12|1001-5000|6|2| +3172|13|1001-5000|6|2| +3173|14|1001-5000|6|2| +3174|15|1001-5000|6|2| +3175|16|1001-5000|6|2| +3176|17|1001-5000|6|2| +3177|18|1001-5000|6|2| +3178|19|1001-5000|6|2| +3179|20|1001-5000|6|2| +3180|1|5001-10000|6|2| +3181|2|5001-10000|6|2| +3182|3|5001-10000|6|2| +3183|4|5001-10000|6|2| +3184|5|5001-10000|6|2| +3185|6|5001-10000|6|2| +3186|7|5001-10000|6|2| +3187|8|5001-10000|6|2| +3188|9|5001-10000|6|2| +3189|10|5001-10000|6|2| +3190|11|5001-10000|6|2| +3191|12|5001-10000|6|2| +3192|13|5001-10000|6|2| +3193|14|5001-10000|6|2| +3194|15|5001-10000|6|2| +3195|16|5001-10000|6|2| +3196|17|5001-10000|6|2| +3197|18|5001-10000|6|2| +3198|19|5001-10000|6|2| +3199|20|5001-10000|6|2| +3200|1|>10000|6|2| +3201|2|>10000|6|2| +3202|3|>10000|6|2| +3203|4|>10000|6|2| +3204|5|>10000|6|2| +3205|6|>10000|6|2| +3206|7|>10000|6|2| +3207|8|>10000|6|2| +3208|9|>10000|6|2| +3209|10|>10000|6|2| +3210|11|>10000|6|2| +3211|12|>10000|6|2| +3212|13|>10000|6|2| +3213|14|>10000|6|2| +3214|15|>10000|6|2| +3215|16|>10000|6|2| +3216|17|>10000|6|2| +3217|18|>10000|6|2| +3218|19|>10000|6|2| +3219|20|>10000|6|2| +3220|1|Unknown|6|2| +3221|2|Unknown|6|2| +3222|3|Unknown|6|2| +3223|4|Unknown|6|2| +3224|5|Unknown|6|2| +3225|6|Unknown|6|2| +3226|7|Unknown|6|2| +3227|8|Unknown|6|2| +3228|9|Unknown|6|2| +3229|10|Unknown|6|2| +3230|11|Unknown|6|2| +3231|12|Unknown|6|2| +3232|13|Unknown|6|2| +3233|14|Unknown|6|2| +3234|15|Unknown|6|2| +3235|16|Unknown|6|2| +3236|17|Unknown|6|2| +3237|18|Unknown|6|2| +3238|19|Unknown|6|2| +3239|20|Unknown|6|2| +3240|1|0-500|7|2| +3241|2|0-500|7|2| +3242|3|0-500|7|2| +3243|4|0-500|7|2| +3244|5|0-500|7|2| +3245|6|0-500|7|2| +3246|7|0-500|7|2| +3247|8|0-500|7|2| +3248|9|0-500|7|2| +3249|10|0-500|7|2| +3250|11|0-500|7|2| +3251|12|0-500|7|2| +3252|13|0-500|7|2| +3253|14|0-500|7|2| +3254|15|0-500|7|2| +3255|16|0-500|7|2| +3256|17|0-500|7|2| +3257|18|0-500|7|2| +3258|19|0-500|7|2| +3259|20|0-500|7|2| +3260|1|501-1000|7|2| +3261|2|501-1000|7|2| +3262|3|501-1000|7|2| +3263|4|501-1000|7|2| +3264|5|501-1000|7|2| +3265|6|501-1000|7|2| +3266|7|501-1000|7|2| +3267|8|501-1000|7|2| +3268|9|501-1000|7|2| +3269|10|501-1000|7|2| +3270|11|501-1000|7|2| +3271|12|501-1000|7|2| +3272|13|501-1000|7|2| +3273|14|501-1000|7|2| +3274|15|501-1000|7|2| +3275|16|501-1000|7|2| +3276|17|501-1000|7|2| +3277|18|501-1000|7|2| +3278|19|501-1000|7|2| +3279|20|501-1000|7|2| +3280|1|1001-5000|7|2| +3281|2|1001-5000|7|2| +3282|3|1001-5000|7|2| +3283|4|1001-5000|7|2| +3284|5|1001-5000|7|2| +3285|6|1001-5000|7|2| +3286|7|1001-5000|7|2| +3287|8|1001-5000|7|2| +3288|9|1001-5000|7|2| +3289|10|1001-5000|7|2| +3290|11|1001-5000|7|2| +3291|12|1001-5000|7|2| +3292|13|1001-5000|7|2| +3293|14|1001-5000|7|2| +3294|15|1001-5000|7|2| +3295|16|1001-5000|7|2| +3296|17|1001-5000|7|2| +3297|18|1001-5000|7|2| +3298|19|1001-5000|7|2| +3299|20|1001-5000|7|2| +3300|1|5001-10000|7|2| +3301|2|5001-10000|7|2| +3302|3|5001-10000|7|2| +3303|4|5001-10000|7|2| +3304|5|5001-10000|7|2| +3305|6|5001-10000|7|2| +3306|7|5001-10000|7|2| +3307|8|5001-10000|7|2| +3308|9|5001-10000|7|2| +3309|10|5001-10000|7|2| +3310|11|5001-10000|7|2| +3311|12|5001-10000|7|2| +3312|13|5001-10000|7|2| +3313|14|5001-10000|7|2| +3314|15|5001-10000|7|2| +3315|16|5001-10000|7|2| +3316|17|5001-10000|7|2| +3317|18|5001-10000|7|2| +3318|19|5001-10000|7|2| +3319|20|5001-10000|7|2| +3320|1|>10000|7|2| +3321|2|>10000|7|2| +3322|3|>10000|7|2| +3323|4|>10000|7|2| +3324|5|>10000|7|2| +3325|6|>10000|7|2| +3326|7|>10000|7|2| +3327|8|>10000|7|2| +3328|9|>10000|7|2| +3329|10|>10000|7|2| +3330|11|>10000|7|2| +3331|12|>10000|7|2| +3332|13|>10000|7|2| +3333|14|>10000|7|2| +3334|15|>10000|7|2| +3335|16|>10000|7|2| +3336|17|>10000|7|2| +3337|18|>10000|7|2| +3338|19|>10000|7|2| +3339|20|>10000|7|2| +3340|1|Unknown|7|2| +3341|2|Unknown|7|2| +3342|3|Unknown|7|2| +3343|4|Unknown|7|2| +3344|5|Unknown|7|2| +3345|6|Unknown|7|2| +3346|7|Unknown|7|2| +3347|8|Unknown|7|2| +3348|9|Unknown|7|2| +3349|10|Unknown|7|2| +3350|11|Unknown|7|2| +3351|12|Unknown|7|2| +3352|13|Unknown|7|2| +3353|14|Unknown|7|2| +3354|15|Unknown|7|2| +3355|16|Unknown|7|2| +3356|17|Unknown|7|2| +3357|18|Unknown|7|2| +3358|19|Unknown|7|2| +3359|20|Unknown|7|2| +3360|1|0-500|8|2| +3361|2|0-500|8|2| +3362|3|0-500|8|2| +3363|4|0-500|8|2| +3364|5|0-500|8|2| +3365|6|0-500|8|2| +3366|7|0-500|8|2| +3367|8|0-500|8|2| +3368|9|0-500|8|2| +3369|10|0-500|8|2| +3370|11|0-500|8|2| +3371|12|0-500|8|2| +3372|13|0-500|8|2| +3373|14|0-500|8|2| +3374|15|0-500|8|2| +3375|16|0-500|8|2| +3376|17|0-500|8|2| +3377|18|0-500|8|2| +3378|19|0-500|8|2| +3379|20|0-500|8|2| +3380|1|501-1000|8|2| +3381|2|501-1000|8|2| +3382|3|501-1000|8|2| +3383|4|501-1000|8|2| +3384|5|501-1000|8|2| +3385|6|501-1000|8|2| +3386|7|501-1000|8|2| +3387|8|501-1000|8|2| +3388|9|501-1000|8|2| +3389|10|501-1000|8|2| +3390|11|501-1000|8|2| +3391|12|501-1000|8|2| +3392|13|501-1000|8|2| +3393|14|501-1000|8|2| +3394|15|501-1000|8|2| +3395|16|501-1000|8|2| +3396|17|501-1000|8|2| +3397|18|501-1000|8|2| +3398|19|501-1000|8|2| +3399|20|501-1000|8|2| +3400|1|1001-5000|8|2| +3401|2|1001-5000|8|2| +3402|3|1001-5000|8|2| +3403|4|1001-5000|8|2| +3404|5|1001-5000|8|2| +3405|6|1001-5000|8|2| +3406|7|1001-5000|8|2| +3407|8|1001-5000|8|2| +3408|9|1001-5000|8|2| +3409|10|1001-5000|8|2| +3410|11|1001-5000|8|2| +3411|12|1001-5000|8|2| +3412|13|1001-5000|8|2| +3413|14|1001-5000|8|2| +3414|15|1001-5000|8|2| +3415|16|1001-5000|8|2| +3416|17|1001-5000|8|2| +3417|18|1001-5000|8|2| +3418|19|1001-5000|8|2| +3419|20|1001-5000|8|2| +3420|1|5001-10000|8|2| +3421|2|5001-10000|8|2| +3422|3|5001-10000|8|2| +3423|4|5001-10000|8|2| +3424|5|5001-10000|8|2| +3425|6|5001-10000|8|2| +3426|7|5001-10000|8|2| +3427|8|5001-10000|8|2| +3428|9|5001-10000|8|2| +3429|10|5001-10000|8|2| +3430|11|5001-10000|8|2| +3431|12|5001-10000|8|2| +3432|13|5001-10000|8|2| +3433|14|5001-10000|8|2| +3434|15|5001-10000|8|2| +3435|16|5001-10000|8|2| +3436|17|5001-10000|8|2| +3437|18|5001-10000|8|2| +3438|19|5001-10000|8|2| +3439|20|5001-10000|8|2| +3440|1|>10000|8|2| +3441|2|>10000|8|2| +3442|3|>10000|8|2| +3443|4|>10000|8|2| +3444|5|>10000|8|2| +3445|6|>10000|8|2| +3446|7|>10000|8|2| +3447|8|>10000|8|2| +3448|9|>10000|8|2| +3449|10|>10000|8|2| +3450|11|>10000|8|2| +3451|12|>10000|8|2| +3452|13|>10000|8|2| +3453|14|>10000|8|2| +3454|15|>10000|8|2| +3455|16|>10000|8|2| +3456|17|>10000|8|2| +3457|18|>10000|8|2| +3458|19|>10000|8|2| +3459|20|>10000|8|2| +3460|1|Unknown|8|2| +3461|2|Unknown|8|2| +3462|3|Unknown|8|2| +3463|4|Unknown|8|2| +3464|5|Unknown|8|2| +3465|6|Unknown|8|2| +3466|7|Unknown|8|2| +3467|8|Unknown|8|2| +3468|9|Unknown|8|2| +3469|10|Unknown|8|2| +3470|11|Unknown|8|2| +3471|12|Unknown|8|2| +3472|13|Unknown|8|2| +3473|14|Unknown|8|2| +3474|15|Unknown|8|2| +3475|16|Unknown|8|2| +3476|17|Unknown|8|2| +3477|18|Unknown|8|2| +3478|19|Unknown|8|2| +3479|20|Unknown|8|2| +3480|1|0-500|9|2| +3481|2|0-500|9|2| +3482|3|0-500|9|2| +3483|4|0-500|9|2| +3484|5|0-500|9|2| +3485|6|0-500|9|2| +3486|7|0-500|9|2| +3487|8|0-500|9|2| +3488|9|0-500|9|2| +3489|10|0-500|9|2| +3490|11|0-500|9|2| +3491|12|0-500|9|2| +3492|13|0-500|9|2| +3493|14|0-500|9|2| +3494|15|0-500|9|2| +3495|16|0-500|9|2| +3496|17|0-500|9|2| +3497|18|0-500|9|2| +3498|19|0-500|9|2| +3499|20|0-500|9|2| +3500|1|501-1000|9|2| +3501|2|501-1000|9|2| +3502|3|501-1000|9|2| +3503|4|501-1000|9|2| +3504|5|501-1000|9|2| +3505|6|501-1000|9|2| +3506|7|501-1000|9|2| +3507|8|501-1000|9|2| +3508|9|501-1000|9|2| +3509|10|501-1000|9|2| +3510|11|501-1000|9|2| +3511|12|501-1000|9|2| +3512|13|501-1000|9|2| +3513|14|501-1000|9|2| +3514|15|501-1000|9|2| +3515|16|501-1000|9|2| +3516|17|501-1000|9|2| +3517|18|501-1000|9|2| +3518|19|501-1000|9|2| +3519|20|501-1000|9|2| +3520|1|1001-5000|9|2| +3521|2|1001-5000|9|2| +3522|3|1001-5000|9|2| +3523|4|1001-5000|9|2| +3524|5|1001-5000|9|2| +3525|6|1001-5000|9|2| +3526|7|1001-5000|9|2| +3527|8|1001-5000|9|2| +3528|9|1001-5000|9|2| +3529|10|1001-5000|9|2| +3530|11|1001-5000|9|2| +3531|12|1001-5000|9|2| +3532|13|1001-5000|9|2| +3533|14|1001-5000|9|2| +3534|15|1001-5000|9|2| +3535|16|1001-5000|9|2| +3536|17|1001-5000|9|2| +3537|18|1001-5000|9|2| +3538|19|1001-5000|9|2| +3539|20|1001-5000|9|2| +3540|1|5001-10000|9|2| +3541|2|5001-10000|9|2| +3542|3|5001-10000|9|2| +3543|4|5001-10000|9|2| +3544|5|5001-10000|9|2| +3545|6|5001-10000|9|2| +3546|7|5001-10000|9|2| +3547|8|5001-10000|9|2| +3548|9|5001-10000|9|2| +3549|10|5001-10000|9|2| +3550|11|5001-10000|9|2| +3551|12|5001-10000|9|2| +3552|13|5001-10000|9|2| +3553|14|5001-10000|9|2| +3554|15|5001-10000|9|2| +3555|16|5001-10000|9|2| +3556|17|5001-10000|9|2| +3557|18|5001-10000|9|2| +3558|19|5001-10000|9|2| +3559|20|5001-10000|9|2| +3560|1|>10000|9|2| +3561|2|>10000|9|2| +3562|3|>10000|9|2| +3563|4|>10000|9|2| +3564|5|>10000|9|2| +3565|6|>10000|9|2| +3566|7|>10000|9|2| +3567|8|>10000|9|2| +3568|9|>10000|9|2| +3569|10|>10000|9|2| +3570|11|>10000|9|2| +3571|12|>10000|9|2| +3572|13|>10000|9|2| +3573|14|>10000|9|2| +3574|15|>10000|9|2| +3575|16|>10000|9|2| +3576|17|>10000|9|2| +3577|18|>10000|9|2| +3578|19|>10000|9|2| +3579|20|>10000|9|2| +3580|1|Unknown|9|2| +3581|2|Unknown|9|2| +3582|3|Unknown|9|2| +3583|4|Unknown|9|2| +3584|5|Unknown|9|2| +3585|6|Unknown|9|2| +3586|7|Unknown|9|2| +3587|8|Unknown|9|2| +3588|9|Unknown|9|2| +3589|10|Unknown|9|2| +3590|11|Unknown|9|2| +3591|12|Unknown|9|2| +3592|13|Unknown|9|2| +3593|14|Unknown|9|2| +3594|15|Unknown|9|2| +3595|16|Unknown|9|2| +3596|17|Unknown|9|2| +3597|18|Unknown|9|2| +3598|19|Unknown|9|2| +3599|20|Unknown|9|2| +3600|1|0-500|0|3| +3601|2|0-500|0|3| +3602|3|0-500|0|3| +3603|4|0-500|0|3| +3604|5|0-500|0|3| +3605|6|0-500|0|3| +3606|7|0-500|0|3| +3607|8|0-500|0|3| +3608|9|0-500|0|3| +3609|10|0-500|0|3| +3610|11|0-500|0|3| +3611|12|0-500|0|3| +3612|13|0-500|0|3| +3613|14|0-500|0|3| +3614|15|0-500|0|3| +3615|16|0-500|0|3| +3616|17|0-500|0|3| +3617|18|0-500|0|3| +3618|19|0-500|0|3| +3619|20|0-500|0|3| +3620|1|501-1000|0|3| +3621|2|501-1000|0|3| +3622|3|501-1000|0|3| +3623|4|501-1000|0|3| +3624|5|501-1000|0|3| +3625|6|501-1000|0|3| +3626|7|501-1000|0|3| +3627|8|501-1000|0|3| +3628|9|501-1000|0|3| +3629|10|501-1000|0|3| +3630|11|501-1000|0|3| +3631|12|501-1000|0|3| +3632|13|501-1000|0|3| +3633|14|501-1000|0|3| +3634|15|501-1000|0|3| +3635|16|501-1000|0|3| +3636|17|501-1000|0|3| +3637|18|501-1000|0|3| +3638|19|501-1000|0|3| +3639|20|501-1000|0|3| +3640|1|1001-5000|0|3| +3641|2|1001-5000|0|3| +3642|3|1001-5000|0|3| +3643|4|1001-5000|0|3| +3644|5|1001-5000|0|3| +3645|6|1001-5000|0|3| +3646|7|1001-5000|0|3| +3647|8|1001-5000|0|3| +3648|9|1001-5000|0|3| +3649|10|1001-5000|0|3| +3650|11|1001-5000|0|3| +3651|12|1001-5000|0|3| +3652|13|1001-5000|0|3| +3653|14|1001-5000|0|3| +3654|15|1001-5000|0|3| +3655|16|1001-5000|0|3| +3656|17|1001-5000|0|3| +3657|18|1001-5000|0|3| +3658|19|1001-5000|0|3| +3659|20|1001-5000|0|3| +3660|1|5001-10000|0|3| +3661|2|5001-10000|0|3| +3662|3|5001-10000|0|3| +3663|4|5001-10000|0|3| +3664|5|5001-10000|0|3| +3665|6|5001-10000|0|3| +3666|7|5001-10000|0|3| +3667|8|5001-10000|0|3| +3668|9|5001-10000|0|3| +3669|10|5001-10000|0|3| +3670|11|5001-10000|0|3| +3671|12|5001-10000|0|3| +3672|13|5001-10000|0|3| +3673|14|5001-10000|0|3| +3674|15|5001-10000|0|3| +3675|16|5001-10000|0|3| +3676|17|5001-10000|0|3| +3677|18|5001-10000|0|3| +3678|19|5001-10000|0|3| +3679|20|5001-10000|0|3| +3680|1|>10000|0|3| +3681|2|>10000|0|3| +3682|3|>10000|0|3| +3683|4|>10000|0|3| +3684|5|>10000|0|3| +3685|6|>10000|0|3| +3686|7|>10000|0|3| +3687|8|>10000|0|3| +3688|9|>10000|0|3| +3689|10|>10000|0|3| +3690|11|>10000|0|3| +3691|12|>10000|0|3| +3692|13|>10000|0|3| +3693|14|>10000|0|3| +3694|15|>10000|0|3| +3695|16|>10000|0|3| +3696|17|>10000|0|3| +3697|18|>10000|0|3| +3698|19|>10000|0|3| +3699|20|>10000|0|3| +3700|1|Unknown|0|3| +3701|2|Unknown|0|3| +3702|3|Unknown|0|3| +3703|4|Unknown|0|3| +3704|5|Unknown|0|3| +3705|6|Unknown|0|3| +3706|7|Unknown|0|3| +3707|8|Unknown|0|3| +3708|9|Unknown|0|3| +3709|10|Unknown|0|3| +3710|11|Unknown|0|3| +3711|12|Unknown|0|3| +3712|13|Unknown|0|3| +3713|14|Unknown|0|3| +3714|15|Unknown|0|3| +3715|16|Unknown|0|3| +3716|17|Unknown|0|3| +3717|18|Unknown|0|3| +3718|19|Unknown|0|3| +3719|20|Unknown|0|3| +3720|1|0-500|1|3| +3721|2|0-500|1|3| +3722|3|0-500|1|3| +3723|4|0-500|1|3| +3724|5|0-500|1|3| +3725|6|0-500|1|3| +3726|7|0-500|1|3| +3727|8|0-500|1|3| +3728|9|0-500|1|3| +3729|10|0-500|1|3| +3730|11|0-500|1|3| +3731|12|0-500|1|3| +3732|13|0-500|1|3| +3733|14|0-500|1|3| +3734|15|0-500|1|3| +3735|16|0-500|1|3| +3736|17|0-500|1|3| +3737|18|0-500|1|3| +3738|19|0-500|1|3| +3739|20|0-500|1|3| +3740|1|501-1000|1|3| +3741|2|501-1000|1|3| +3742|3|501-1000|1|3| +3743|4|501-1000|1|3| +3744|5|501-1000|1|3| +3745|6|501-1000|1|3| +3746|7|501-1000|1|3| +3747|8|501-1000|1|3| +3748|9|501-1000|1|3| +3749|10|501-1000|1|3| +3750|11|501-1000|1|3| +3751|12|501-1000|1|3| +3752|13|501-1000|1|3| +3753|14|501-1000|1|3| +3754|15|501-1000|1|3| +3755|16|501-1000|1|3| +3756|17|501-1000|1|3| +3757|18|501-1000|1|3| +3758|19|501-1000|1|3| +3759|20|501-1000|1|3| +3760|1|1001-5000|1|3| +3761|2|1001-5000|1|3| +3762|3|1001-5000|1|3| +3763|4|1001-5000|1|3| +3764|5|1001-5000|1|3| +3765|6|1001-5000|1|3| +3766|7|1001-5000|1|3| +3767|8|1001-5000|1|3| +3768|9|1001-5000|1|3| +3769|10|1001-5000|1|3| +3770|11|1001-5000|1|3| +3771|12|1001-5000|1|3| +3772|13|1001-5000|1|3| +3773|14|1001-5000|1|3| +3774|15|1001-5000|1|3| +3775|16|1001-5000|1|3| +3776|17|1001-5000|1|3| +3777|18|1001-5000|1|3| +3778|19|1001-5000|1|3| +3779|20|1001-5000|1|3| +3780|1|5001-10000|1|3| +3781|2|5001-10000|1|3| +3782|3|5001-10000|1|3| +3783|4|5001-10000|1|3| +3784|5|5001-10000|1|3| +3785|6|5001-10000|1|3| +3786|7|5001-10000|1|3| +3787|8|5001-10000|1|3| +3788|9|5001-10000|1|3| +3789|10|5001-10000|1|3| +3790|11|5001-10000|1|3| +3791|12|5001-10000|1|3| +3792|13|5001-10000|1|3| +3793|14|5001-10000|1|3| +3794|15|5001-10000|1|3| +3795|16|5001-10000|1|3| +3796|17|5001-10000|1|3| +3797|18|5001-10000|1|3| +3798|19|5001-10000|1|3| +3799|20|5001-10000|1|3| +3800|1|>10000|1|3| +3801|2|>10000|1|3| +3802|3|>10000|1|3| +3803|4|>10000|1|3| +3804|5|>10000|1|3| +3805|6|>10000|1|3| +3806|7|>10000|1|3| +3807|8|>10000|1|3| +3808|9|>10000|1|3| +3809|10|>10000|1|3| +3810|11|>10000|1|3| +3811|12|>10000|1|3| +3812|13|>10000|1|3| +3813|14|>10000|1|3| +3814|15|>10000|1|3| +3815|16|>10000|1|3| +3816|17|>10000|1|3| +3817|18|>10000|1|3| +3818|19|>10000|1|3| +3819|20|>10000|1|3| +3820|1|Unknown|1|3| +3821|2|Unknown|1|3| +3822|3|Unknown|1|3| +3823|4|Unknown|1|3| +3824|5|Unknown|1|3| +3825|6|Unknown|1|3| +3826|7|Unknown|1|3| +3827|8|Unknown|1|3| +3828|9|Unknown|1|3| +3829|10|Unknown|1|3| +3830|11|Unknown|1|3| +3831|12|Unknown|1|3| +3832|13|Unknown|1|3| +3833|14|Unknown|1|3| +3834|15|Unknown|1|3| +3835|16|Unknown|1|3| +3836|17|Unknown|1|3| +3837|18|Unknown|1|3| +3838|19|Unknown|1|3| +3839|20|Unknown|1|3| +3840|1|0-500|2|3| +3841|2|0-500|2|3| +3842|3|0-500|2|3| +3843|4|0-500|2|3| +3844|5|0-500|2|3| +3845|6|0-500|2|3| +3846|7|0-500|2|3| +3847|8|0-500|2|3| +3848|9|0-500|2|3| +3849|10|0-500|2|3| +3850|11|0-500|2|3| +3851|12|0-500|2|3| +3852|13|0-500|2|3| +3853|14|0-500|2|3| +3854|15|0-500|2|3| +3855|16|0-500|2|3| +3856|17|0-500|2|3| +3857|18|0-500|2|3| +3858|19|0-500|2|3| +3859|20|0-500|2|3| +3860|1|501-1000|2|3| +3861|2|501-1000|2|3| +3862|3|501-1000|2|3| +3863|4|501-1000|2|3| +3864|5|501-1000|2|3| +3865|6|501-1000|2|3| +3866|7|501-1000|2|3| +3867|8|501-1000|2|3| +3868|9|501-1000|2|3| +3869|10|501-1000|2|3| +3870|11|501-1000|2|3| +3871|12|501-1000|2|3| +3872|13|501-1000|2|3| +3873|14|501-1000|2|3| +3874|15|501-1000|2|3| +3875|16|501-1000|2|3| +3876|17|501-1000|2|3| +3877|18|501-1000|2|3| +3878|19|501-1000|2|3| +3879|20|501-1000|2|3| +3880|1|1001-5000|2|3| +3881|2|1001-5000|2|3| +3882|3|1001-5000|2|3| +3883|4|1001-5000|2|3| +3884|5|1001-5000|2|3| +3885|6|1001-5000|2|3| +3886|7|1001-5000|2|3| +3887|8|1001-5000|2|3| +3888|9|1001-5000|2|3| +3889|10|1001-5000|2|3| +3890|11|1001-5000|2|3| +3891|12|1001-5000|2|3| +3892|13|1001-5000|2|3| +3893|14|1001-5000|2|3| +3894|15|1001-5000|2|3| +3895|16|1001-5000|2|3| +3896|17|1001-5000|2|3| +3897|18|1001-5000|2|3| +3898|19|1001-5000|2|3| +3899|20|1001-5000|2|3| +3900|1|5001-10000|2|3| +3901|2|5001-10000|2|3| +3902|3|5001-10000|2|3| +3903|4|5001-10000|2|3| +3904|5|5001-10000|2|3| +3905|6|5001-10000|2|3| +3906|7|5001-10000|2|3| +3907|8|5001-10000|2|3| +3908|9|5001-10000|2|3| +3909|10|5001-10000|2|3| +3910|11|5001-10000|2|3| +3911|12|5001-10000|2|3| +3912|13|5001-10000|2|3| +3913|14|5001-10000|2|3| +3914|15|5001-10000|2|3| +3915|16|5001-10000|2|3| +3916|17|5001-10000|2|3| +3917|18|5001-10000|2|3| +3918|19|5001-10000|2|3| +3919|20|5001-10000|2|3| +3920|1|>10000|2|3| +3921|2|>10000|2|3| +3922|3|>10000|2|3| +3923|4|>10000|2|3| +3924|5|>10000|2|3| +3925|6|>10000|2|3| +3926|7|>10000|2|3| +3927|8|>10000|2|3| +3928|9|>10000|2|3| +3929|10|>10000|2|3| +3930|11|>10000|2|3| +3931|12|>10000|2|3| +3932|13|>10000|2|3| +3933|14|>10000|2|3| +3934|15|>10000|2|3| +3935|16|>10000|2|3| +3936|17|>10000|2|3| +3937|18|>10000|2|3| +3938|19|>10000|2|3| +3939|20|>10000|2|3| +3940|1|Unknown|2|3| +3941|2|Unknown|2|3| +3942|3|Unknown|2|3| +3943|4|Unknown|2|3| +3944|5|Unknown|2|3| +3945|6|Unknown|2|3| +3946|7|Unknown|2|3| +3947|8|Unknown|2|3| +3948|9|Unknown|2|3| +3949|10|Unknown|2|3| +3950|11|Unknown|2|3| +3951|12|Unknown|2|3| +3952|13|Unknown|2|3| +3953|14|Unknown|2|3| +3954|15|Unknown|2|3| +3955|16|Unknown|2|3| +3956|17|Unknown|2|3| +3957|18|Unknown|2|3| +3958|19|Unknown|2|3| +3959|20|Unknown|2|3| +3960|1|0-500|3|3| +3961|2|0-500|3|3| +3962|3|0-500|3|3| +3963|4|0-500|3|3| +3964|5|0-500|3|3| +3965|6|0-500|3|3| +3966|7|0-500|3|3| +3967|8|0-500|3|3| +3968|9|0-500|3|3| +3969|10|0-500|3|3| +3970|11|0-500|3|3| +3971|12|0-500|3|3| +3972|13|0-500|3|3| +3973|14|0-500|3|3| +3974|15|0-500|3|3| +3975|16|0-500|3|3| +3976|17|0-500|3|3| +3977|18|0-500|3|3| +3978|19|0-500|3|3| +3979|20|0-500|3|3| +3980|1|501-1000|3|3| +3981|2|501-1000|3|3| +3982|3|501-1000|3|3| +3983|4|501-1000|3|3| +3984|5|501-1000|3|3| +3985|6|501-1000|3|3| +3986|7|501-1000|3|3| +3987|8|501-1000|3|3| +3988|9|501-1000|3|3| +3989|10|501-1000|3|3| +3990|11|501-1000|3|3| +3991|12|501-1000|3|3| +3992|13|501-1000|3|3| +3993|14|501-1000|3|3| +3994|15|501-1000|3|3| +3995|16|501-1000|3|3| +3996|17|501-1000|3|3| +3997|18|501-1000|3|3| +3998|19|501-1000|3|3| +3999|20|501-1000|3|3| +4000|1|1001-5000|3|3| +4001|2|1001-5000|3|3| +4002|3|1001-5000|3|3| +4003|4|1001-5000|3|3| +4004|5|1001-5000|3|3| +4005|6|1001-5000|3|3| +4006|7|1001-5000|3|3| +4007|8|1001-5000|3|3| +4008|9|1001-5000|3|3| +4009|10|1001-5000|3|3| +4010|11|1001-5000|3|3| +4011|12|1001-5000|3|3| +4012|13|1001-5000|3|3| +4013|14|1001-5000|3|3| +4014|15|1001-5000|3|3| +4015|16|1001-5000|3|3| +4016|17|1001-5000|3|3| +4017|18|1001-5000|3|3| +4018|19|1001-5000|3|3| +4019|20|1001-5000|3|3| +4020|1|5001-10000|3|3| +4021|2|5001-10000|3|3| +4022|3|5001-10000|3|3| +4023|4|5001-10000|3|3| +4024|5|5001-10000|3|3| +4025|6|5001-10000|3|3| +4026|7|5001-10000|3|3| +4027|8|5001-10000|3|3| +4028|9|5001-10000|3|3| +4029|10|5001-10000|3|3| +4030|11|5001-10000|3|3| +4031|12|5001-10000|3|3| +4032|13|5001-10000|3|3| +4033|14|5001-10000|3|3| +4034|15|5001-10000|3|3| +4035|16|5001-10000|3|3| +4036|17|5001-10000|3|3| +4037|18|5001-10000|3|3| +4038|19|5001-10000|3|3| +4039|20|5001-10000|3|3| +4040|1|>10000|3|3| +4041|2|>10000|3|3| +4042|3|>10000|3|3| +4043|4|>10000|3|3| +4044|5|>10000|3|3| +4045|6|>10000|3|3| +4046|7|>10000|3|3| +4047|8|>10000|3|3| +4048|9|>10000|3|3| +4049|10|>10000|3|3| +4050|11|>10000|3|3| +4051|12|>10000|3|3| +4052|13|>10000|3|3| +4053|14|>10000|3|3| +4054|15|>10000|3|3| +4055|16|>10000|3|3| +4056|17|>10000|3|3| +4057|18|>10000|3|3| +4058|19|>10000|3|3| +4059|20|>10000|3|3| +4060|1|Unknown|3|3| +4061|2|Unknown|3|3| +4062|3|Unknown|3|3| +4063|4|Unknown|3|3| +4064|5|Unknown|3|3| +4065|6|Unknown|3|3| +4066|7|Unknown|3|3| +4067|8|Unknown|3|3| +4068|9|Unknown|3|3| +4069|10|Unknown|3|3| +4070|11|Unknown|3|3| +4071|12|Unknown|3|3| +4072|13|Unknown|3|3| +4073|14|Unknown|3|3| +4074|15|Unknown|3|3| +4075|16|Unknown|3|3| +4076|17|Unknown|3|3| +4077|18|Unknown|3|3| +4078|19|Unknown|3|3| +4079|20|Unknown|3|3| +4080|1|0-500|4|3| +4081|2|0-500|4|3| +4082|3|0-500|4|3| +4083|4|0-500|4|3| +4084|5|0-500|4|3| +4085|6|0-500|4|3| +4086|7|0-500|4|3| +4087|8|0-500|4|3| +4088|9|0-500|4|3| +4089|10|0-500|4|3| +4090|11|0-500|4|3| +4091|12|0-500|4|3| +4092|13|0-500|4|3| +4093|14|0-500|4|3| +4094|15|0-500|4|3| +4095|16|0-500|4|3| +4096|17|0-500|4|3| +4097|18|0-500|4|3| +4098|19|0-500|4|3| +4099|20|0-500|4|3| +4100|1|501-1000|4|3| +4101|2|501-1000|4|3| +4102|3|501-1000|4|3| +4103|4|501-1000|4|3| +4104|5|501-1000|4|3| +4105|6|501-1000|4|3| +4106|7|501-1000|4|3| +4107|8|501-1000|4|3| +4108|9|501-1000|4|3| +4109|10|501-1000|4|3| +4110|11|501-1000|4|3| +4111|12|501-1000|4|3| +4112|13|501-1000|4|3| +4113|14|501-1000|4|3| +4114|15|501-1000|4|3| +4115|16|501-1000|4|3| +4116|17|501-1000|4|3| +4117|18|501-1000|4|3| +4118|19|501-1000|4|3| +4119|20|501-1000|4|3| +4120|1|1001-5000|4|3| +4121|2|1001-5000|4|3| +4122|3|1001-5000|4|3| +4123|4|1001-5000|4|3| +4124|5|1001-5000|4|3| +4125|6|1001-5000|4|3| +4126|7|1001-5000|4|3| +4127|8|1001-5000|4|3| +4128|9|1001-5000|4|3| +4129|10|1001-5000|4|3| +4130|11|1001-5000|4|3| +4131|12|1001-5000|4|3| +4132|13|1001-5000|4|3| +4133|14|1001-5000|4|3| +4134|15|1001-5000|4|3| +4135|16|1001-5000|4|3| +4136|17|1001-5000|4|3| +4137|18|1001-5000|4|3| +4138|19|1001-5000|4|3| +4139|20|1001-5000|4|3| +4140|1|5001-10000|4|3| +4141|2|5001-10000|4|3| +4142|3|5001-10000|4|3| +4143|4|5001-10000|4|3| +4144|5|5001-10000|4|3| +4145|6|5001-10000|4|3| +4146|7|5001-10000|4|3| +4147|8|5001-10000|4|3| +4148|9|5001-10000|4|3| +4149|10|5001-10000|4|3| +4150|11|5001-10000|4|3| +4151|12|5001-10000|4|3| +4152|13|5001-10000|4|3| +4153|14|5001-10000|4|3| +4154|15|5001-10000|4|3| +4155|16|5001-10000|4|3| +4156|17|5001-10000|4|3| +4157|18|5001-10000|4|3| +4158|19|5001-10000|4|3| +4159|20|5001-10000|4|3| +4160|1|>10000|4|3| +4161|2|>10000|4|3| +4162|3|>10000|4|3| +4163|4|>10000|4|3| +4164|5|>10000|4|3| +4165|6|>10000|4|3| +4166|7|>10000|4|3| +4167|8|>10000|4|3| +4168|9|>10000|4|3| +4169|10|>10000|4|3| +4170|11|>10000|4|3| +4171|12|>10000|4|3| +4172|13|>10000|4|3| +4173|14|>10000|4|3| +4174|15|>10000|4|3| +4175|16|>10000|4|3| +4176|17|>10000|4|3| +4177|18|>10000|4|3| +4178|19|>10000|4|3| +4179|20|>10000|4|3| +4180|1|Unknown|4|3| +4181|2|Unknown|4|3| +4182|3|Unknown|4|3| +4183|4|Unknown|4|3| +4184|5|Unknown|4|3| +4185|6|Unknown|4|3| +4186|7|Unknown|4|3| +4187|8|Unknown|4|3| +4188|9|Unknown|4|3| +4189|10|Unknown|4|3| +4190|11|Unknown|4|3| +4191|12|Unknown|4|3| +4192|13|Unknown|4|3| +4193|14|Unknown|4|3| +4194|15|Unknown|4|3| +4195|16|Unknown|4|3| +4196|17|Unknown|4|3| +4197|18|Unknown|4|3| +4198|19|Unknown|4|3| +4199|20|Unknown|4|3| +4200|1|0-500|5|3| +4201|2|0-500|5|3| +4202|3|0-500|5|3| +4203|4|0-500|5|3| +4204|5|0-500|5|3| +4205|6|0-500|5|3| +4206|7|0-500|5|3| +4207|8|0-500|5|3| +4208|9|0-500|5|3| +4209|10|0-500|5|3| +4210|11|0-500|5|3| +4211|12|0-500|5|3| +4212|13|0-500|5|3| +4213|14|0-500|5|3| +4214|15|0-500|5|3| +4215|16|0-500|5|3| +4216|17|0-500|5|3| +4217|18|0-500|5|3| +4218|19|0-500|5|3| +4219|20|0-500|5|3| +4220|1|501-1000|5|3| +4221|2|501-1000|5|3| +4222|3|501-1000|5|3| +4223|4|501-1000|5|3| +4224|5|501-1000|5|3| +4225|6|501-1000|5|3| +4226|7|501-1000|5|3| +4227|8|501-1000|5|3| +4228|9|501-1000|5|3| +4229|10|501-1000|5|3| +4230|11|501-1000|5|3| +4231|12|501-1000|5|3| +4232|13|501-1000|5|3| +4233|14|501-1000|5|3| +4234|15|501-1000|5|3| +4235|16|501-1000|5|3| +4236|17|501-1000|5|3| +4237|18|501-1000|5|3| +4238|19|501-1000|5|3| +4239|20|501-1000|5|3| +4240|1|1001-5000|5|3| +4241|2|1001-5000|5|3| +4242|3|1001-5000|5|3| +4243|4|1001-5000|5|3| +4244|5|1001-5000|5|3| +4245|6|1001-5000|5|3| +4246|7|1001-5000|5|3| +4247|8|1001-5000|5|3| +4248|9|1001-5000|5|3| +4249|10|1001-5000|5|3| +4250|11|1001-5000|5|3| +4251|12|1001-5000|5|3| +4252|13|1001-5000|5|3| +4253|14|1001-5000|5|3| +4254|15|1001-5000|5|3| +4255|16|1001-5000|5|3| +4256|17|1001-5000|5|3| +4257|18|1001-5000|5|3| +4258|19|1001-5000|5|3| +4259|20|1001-5000|5|3| +4260|1|5001-10000|5|3| +4261|2|5001-10000|5|3| +4262|3|5001-10000|5|3| +4263|4|5001-10000|5|3| +4264|5|5001-10000|5|3| +4265|6|5001-10000|5|3| +4266|7|5001-10000|5|3| +4267|8|5001-10000|5|3| +4268|9|5001-10000|5|3| +4269|10|5001-10000|5|3| +4270|11|5001-10000|5|3| +4271|12|5001-10000|5|3| +4272|13|5001-10000|5|3| +4273|14|5001-10000|5|3| +4274|15|5001-10000|5|3| +4275|16|5001-10000|5|3| +4276|17|5001-10000|5|3| +4277|18|5001-10000|5|3| +4278|19|5001-10000|5|3| +4279|20|5001-10000|5|3| +4280|1|>10000|5|3| +4281|2|>10000|5|3| +4282|3|>10000|5|3| +4283|4|>10000|5|3| +4284|5|>10000|5|3| +4285|6|>10000|5|3| +4286|7|>10000|5|3| +4287|8|>10000|5|3| +4288|9|>10000|5|3| +4289|10|>10000|5|3| +4290|11|>10000|5|3| +4291|12|>10000|5|3| +4292|13|>10000|5|3| +4293|14|>10000|5|3| +4294|15|>10000|5|3| +4295|16|>10000|5|3| +4296|17|>10000|5|3| +4297|18|>10000|5|3| +4298|19|>10000|5|3| +4299|20|>10000|5|3| +4300|1|Unknown|5|3| +4301|2|Unknown|5|3| +4302|3|Unknown|5|3| +4303|4|Unknown|5|3| +4304|5|Unknown|5|3| +4305|6|Unknown|5|3| +4306|7|Unknown|5|3| +4307|8|Unknown|5|3| +4308|9|Unknown|5|3| +4309|10|Unknown|5|3| +4310|11|Unknown|5|3| +4311|12|Unknown|5|3| +4312|13|Unknown|5|3| +4313|14|Unknown|5|3| +4314|15|Unknown|5|3| +4315|16|Unknown|5|3| +4316|17|Unknown|5|3| +4317|18|Unknown|5|3| +4318|19|Unknown|5|3| +4319|20|Unknown|5|3| +4320|1|0-500|6|3| +4321|2|0-500|6|3| +4322|3|0-500|6|3| +4323|4|0-500|6|3| +4324|5|0-500|6|3| +4325|6|0-500|6|3| +4326|7|0-500|6|3| +4327|8|0-500|6|3| +4328|9|0-500|6|3| +4329|10|0-500|6|3| +4330|11|0-500|6|3| +4331|12|0-500|6|3| +4332|13|0-500|6|3| +4333|14|0-500|6|3| +4334|15|0-500|6|3| +4335|16|0-500|6|3| +4336|17|0-500|6|3| +4337|18|0-500|6|3| +4338|19|0-500|6|3| +4339|20|0-500|6|3| +4340|1|501-1000|6|3| +4341|2|501-1000|6|3| +4342|3|501-1000|6|3| +4343|4|501-1000|6|3| +4344|5|501-1000|6|3| +4345|6|501-1000|6|3| +4346|7|501-1000|6|3| +4347|8|501-1000|6|3| +4348|9|501-1000|6|3| +4349|10|501-1000|6|3| +4350|11|501-1000|6|3| +4351|12|501-1000|6|3| +4352|13|501-1000|6|3| +4353|14|501-1000|6|3| +4354|15|501-1000|6|3| +4355|16|501-1000|6|3| +4356|17|501-1000|6|3| +4357|18|501-1000|6|3| +4358|19|501-1000|6|3| +4359|20|501-1000|6|3| +4360|1|1001-5000|6|3| +4361|2|1001-5000|6|3| +4362|3|1001-5000|6|3| +4363|4|1001-5000|6|3| +4364|5|1001-5000|6|3| +4365|6|1001-5000|6|3| +4366|7|1001-5000|6|3| +4367|8|1001-5000|6|3| +4368|9|1001-5000|6|3| +4369|10|1001-5000|6|3| +4370|11|1001-5000|6|3| +4371|12|1001-5000|6|3| +4372|13|1001-5000|6|3| +4373|14|1001-5000|6|3| +4374|15|1001-5000|6|3| +4375|16|1001-5000|6|3| +4376|17|1001-5000|6|3| +4377|18|1001-5000|6|3| +4378|19|1001-5000|6|3| +4379|20|1001-5000|6|3| +4380|1|5001-10000|6|3| +4381|2|5001-10000|6|3| +4382|3|5001-10000|6|3| +4383|4|5001-10000|6|3| +4384|5|5001-10000|6|3| +4385|6|5001-10000|6|3| +4386|7|5001-10000|6|3| +4387|8|5001-10000|6|3| +4388|9|5001-10000|6|3| +4389|10|5001-10000|6|3| +4390|11|5001-10000|6|3| +4391|12|5001-10000|6|3| +4392|13|5001-10000|6|3| +4393|14|5001-10000|6|3| +4394|15|5001-10000|6|3| +4395|16|5001-10000|6|3| +4396|17|5001-10000|6|3| +4397|18|5001-10000|6|3| +4398|19|5001-10000|6|3| +4399|20|5001-10000|6|3| +4400|1|>10000|6|3| +4401|2|>10000|6|3| +4402|3|>10000|6|3| +4403|4|>10000|6|3| +4404|5|>10000|6|3| +4405|6|>10000|6|3| +4406|7|>10000|6|3| +4407|8|>10000|6|3| +4408|9|>10000|6|3| +4409|10|>10000|6|3| +4410|11|>10000|6|3| +4411|12|>10000|6|3| +4412|13|>10000|6|3| +4413|14|>10000|6|3| +4414|15|>10000|6|3| +4415|16|>10000|6|3| +4416|17|>10000|6|3| +4417|18|>10000|6|3| +4418|19|>10000|6|3| +4419|20|>10000|6|3| +4420|1|Unknown|6|3| +4421|2|Unknown|6|3| +4422|3|Unknown|6|3| +4423|4|Unknown|6|3| +4424|5|Unknown|6|3| +4425|6|Unknown|6|3| +4426|7|Unknown|6|3| +4427|8|Unknown|6|3| +4428|9|Unknown|6|3| +4429|10|Unknown|6|3| +4430|11|Unknown|6|3| +4431|12|Unknown|6|3| +4432|13|Unknown|6|3| +4433|14|Unknown|6|3| +4434|15|Unknown|6|3| +4435|16|Unknown|6|3| +4436|17|Unknown|6|3| +4437|18|Unknown|6|3| +4438|19|Unknown|6|3| +4439|20|Unknown|6|3| +4440|1|0-500|7|3| +4441|2|0-500|7|3| +4442|3|0-500|7|3| +4443|4|0-500|7|3| +4444|5|0-500|7|3| +4445|6|0-500|7|3| +4446|7|0-500|7|3| +4447|8|0-500|7|3| +4448|9|0-500|7|3| +4449|10|0-500|7|3| +4450|11|0-500|7|3| +4451|12|0-500|7|3| +4452|13|0-500|7|3| +4453|14|0-500|7|3| +4454|15|0-500|7|3| +4455|16|0-500|7|3| +4456|17|0-500|7|3| +4457|18|0-500|7|3| +4458|19|0-500|7|3| +4459|20|0-500|7|3| +4460|1|501-1000|7|3| +4461|2|501-1000|7|3| +4462|3|501-1000|7|3| +4463|4|501-1000|7|3| +4464|5|501-1000|7|3| +4465|6|501-1000|7|3| +4466|7|501-1000|7|3| +4467|8|501-1000|7|3| +4468|9|501-1000|7|3| +4469|10|501-1000|7|3| +4470|11|501-1000|7|3| +4471|12|501-1000|7|3| +4472|13|501-1000|7|3| +4473|14|501-1000|7|3| +4474|15|501-1000|7|3| +4475|16|501-1000|7|3| +4476|17|501-1000|7|3| +4477|18|501-1000|7|3| +4478|19|501-1000|7|3| +4479|20|501-1000|7|3| +4480|1|1001-5000|7|3| +4481|2|1001-5000|7|3| +4482|3|1001-5000|7|3| +4483|4|1001-5000|7|3| +4484|5|1001-5000|7|3| +4485|6|1001-5000|7|3| +4486|7|1001-5000|7|3| +4487|8|1001-5000|7|3| +4488|9|1001-5000|7|3| +4489|10|1001-5000|7|3| +4490|11|1001-5000|7|3| +4491|12|1001-5000|7|3| +4492|13|1001-5000|7|3| +4493|14|1001-5000|7|3| +4494|15|1001-5000|7|3| +4495|16|1001-5000|7|3| +4496|17|1001-5000|7|3| +4497|18|1001-5000|7|3| +4498|19|1001-5000|7|3| +4499|20|1001-5000|7|3| +4500|1|5001-10000|7|3| +4501|2|5001-10000|7|3| +4502|3|5001-10000|7|3| +4503|4|5001-10000|7|3| +4504|5|5001-10000|7|3| +4505|6|5001-10000|7|3| +4506|7|5001-10000|7|3| +4507|8|5001-10000|7|3| +4508|9|5001-10000|7|3| +4509|10|5001-10000|7|3| +4510|11|5001-10000|7|3| +4511|12|5001-10000|7|3| +4512|13|5001-10000|7|3| +4513|14|5001-10000|7|3| +4514|15|5001-10000|7|3| +4515|16|5001-10000|7|3| +4516|17|5001-10000|7|3| +4517|18|5001-10000|7|3| +4518|19|5001-10000|7|3| +4519|20|5001-10000|7|3| +4520|1|>10000|7|3| +4521|2|>10000|7|3| +4522|3|>10000|7|3| +4523|4|>10000|7|3| +4524|5|>10000|7|3| +4525|6|>10000|7|3| +4526|7|>10000|7|3| +4527|8|>10000|7|3| +4528|9|>10000|7|3| +4529|10|>10000|7|3| +4530|11|>10000|7|3| +4531|12|>10000|7|3| +4532|13|>10000|7|3| +4533|14|>10000|7|3| +4534|15|>10000|7|3| +4535|16|>10000|7|3| +4536|17|>10000|7|3| +4537|18|>10000|7|3| +4538|19|>10000|7|3| +4539|20|>10000|7|3| +4540|1|Unknown|7|3| +4541|2|Unknown|7|3| +4542|3|Unknown|7|3| +4543|4|Unknown|7|3| +4544|5|Unknown|7|3| +4545|6|Unknown|7|3| +4546|7|Unknown|7|3| +4547|8|Unknown|7|3| +4548|9|Unknown|7|3| +4549|10|Unknown|7|3| +4550|11|Unknown|7|3| +4551|12|Unknown|7|3| +4552|13|Unknown|7|3| +4553|14|Unknown|7|3| +4554|15|Unknown|7|3| +4555|16|Unknown|7|3| +4556|17|Unknown|7|3| +4557|18|Unknown|7|3| +4558|19|Unknown|7|3| +4559|20|Unknown|7|3| +4560|1|0-500|8|3| +4561|2|0-500|8|3| +4562|3|0-500|8|3| +4563|4|0-500|8|3| +4564|5|0-500|8|3| +4565|6|0-500|8|3| +4566|7|0-500|8|3| +4567|8|0-500|8|3| +4568|9|0-500|8|3| +4569|10|0-500|8|3| +4570|11|0-500|8|3| +4571|12|0-500|8|3| +4572|13|0-500|8|3| +4573|14|0-500|8|3| +4574|15|0-500|8|3| +4575|16|0-500|8|3| +4576|17|0-500|8|3| +4577|18|0-500|8|3| +4578|19|0-500|8|3| +4579|20|0-500|8|3| +4580|1|501-1000|8|3| +4581|2|501-1000|8|3| +4582|3|501-1000|8|3| +4583|4|501-1000|8|3| +4584|5|501-1000|8|3| +4585|6|501-1000|8|3| +4586|7|501-1000|8|3| +4587|8|501-1000|8|3| +4588|9|501-1000|8|3| +4589|10|501-1000|8|3| +4590|11|501-1000|8|3| +4591|12|501-1000|8|3| +4592|13|501-1000|8|3| +4593|14|501-1000|8|3| +4594|15|501-1000|8|3| +4595|16|501-1000|8|3| +4596|17|501-1000|8|3| +4597|18|501-1000|8|3| +4598|19|501-1000|8|3| +4599|20|501-1000|8|3| +4600|1|1001-5000|8|3| +4601|2|1001-5000|8|3| +4602|3|1001-5000|8|3| +4603|4|1001-5000|8|3| +4604|5|1001-5000|8|3| +4605|6|1001-5000|8|3| +4606|7|1001-5000|8|3| +4607|8|1001-5000|8|3| +4608|9|1001-5000|8|3| +4609|10|1001-5000|8|3| +4610|11|1001-5000|8|3| +4611|12|1001-5000|8|3| +4612|13|1001-5000|8|3| +4613|14|1001-5000|8|3| +4614|15|1001-5000|8|3| +4615|16|1001-5000|8|3| +4616|17|1001-5000|8|3| +4617|18|1001-5000|8|3| +4618|19|1001-5000|8|3| +4619|20|1001-5000|8|3| +4620|1|5001-10000|8|3| +4621|2|5001-10000|8|3| +4622|3|5001-10000|8|3| +4623|4|5001-10000|8|3| +4624|5|5001-10000|8|3| +4625|6|5001-10000|8|3| +4626|7|5001-10000|8|3| +4627|8|5001-10000|8|3| +4628|9|5001-10000|8|3| +4629|10|5001-10000|8|3| +4630|11|5001-10000|8|3| +4631|12|5001-10000|8|3| +4632|13|5001-10000|8|3| +4633|14|5001-10000|8|3| +4634|15|5001-10000|8|3| +4635|16|5001-10000|8|3| +4636|17|5001-10000|8|3| +4637|18|5001-10000|8|3| +4638|19|5001-10000|8|3| +4639|20|5001-10000|8|3| +4640|1|>10000|8|3| +4641|2|>10000|8|3| +4642|3|>10000|8|3| +4643|4|>10000|8|3| +4644|5|>10000|8|3| +4645|6|>10000|8|3| +4646|7|>10000|8|3| +4647|8|>10000|8|3| +4648|9|>10000|8|3| +4649|10|>10000|8|3| +4650|11|>10000|8|3| +4651|12|>10000|8|3| +4652|13|>10000|8|3| +4653|14|>10000|8|3| +4654|15|>10000|8|3| +4655|16|>10000|8|3| +4656|17|>10000|8|3| +4657|18|>10000|8|3| +4658|19|>10000|8|3| +4659|20|>10000|8|3| +4660|1|Unknown|8|3| +4661|2|Unknown|8|3| +4662|3|Unknown|8|3| +4663|4|Unknown|8|3| +4664|5|Unknown|8|3| +4665|6|Unknown|8|3| +4666|7|Unknown|8|3| +4667|8|Unknown|8|3| +4668|9|Unknown|8|3| +4669|10|Unknown|8|3| +4670|11|Unknown|8|3| +4671|12|Unknown|8|3| +4672|13|Unknown|8|3| +4673|14|Unknown|8|3| +4674|15|Unknown|8|3| +4675|16|Unknown|8|3| +4676|17|Unknown|8|3| +4677|18|Unknown|8|3| +4678|19|Unknown|8|3| +4679|20|Unknown|8|3| +4680|1|0-500|9|3| +4681|2|0-500|9|3| +4682|3|0-500|9|3| +4683|4|0-500|9|3| +4684|5|0-500|9|3| +4685|6|0-500|9|3| +4686|7|0-500|9|3| +4687|8|0-500|9|3| +4688|9|0-500|9|3| +4689|10|0-500|9|3| +4690|11|0-500|9|3| +4691|12|0-500|9|3| +4692|13|0-500|9|3| +4693|14|0-500|9|3| +4694|15|0-500|9|3| +4695|16|0-500|9|3| +4696|17|0-500|9|3| +4697|18|0-500|9|3| +4698|19|0-500|9|3| +4699|20|0-500|9|3| +4700|1|501-1000|9|3| +4701|2|501-1000|9|3| +4702|3|501-1000|9|3| +4703|4|501-1000|9|3| +4704|5|501-1000|9|3| +4705|6|501-1000|9|3| +4706|7|501-1000|9|3| +4707|8|501-1000|9|3| +4708|9|501-1000|9|3| +4709|10|501-1000|9|3| +4710|11|501-1000|9|3| +4711|12|501-1000|9|3| +4712|13|501-1000|9|3| +4713|14|501-1000|9|3| +4714|15|501-1000|9|3| +4715|16|501-1000|9|3| +4716|17|501-1000|9|3| +4717|18|501-1000|9|3| +4718|19|501-1000|9|3| +4719|20|501-1000|9|3| +4720|1|1001-5000|9|3| +4721|2|1001-5000|9|3| +4722|3|1001-5000|9|3| +4723|4|1001-5000|9|3| +4724|5|1001-5000|9|3| +4725|6|1001-5000|9|3| +4726|7|1001-5000|9|3| +4727|8|1001-5000|9|3| +4728|9|1001-5000|9|3| +4729|10|1001-5000|9|3| +4730|11|1001-5000|9|3| +4731|12|1001-5000|9|3| +4732|13|1001-5000|9|3| +4733|14|1001-5000|9|3| +4734|15|1001-5000|9|3| +4735|16|1001-5000|9|3| +4736|17|1001-5000|9|3| +4737|18|1001-5000|9|3| +4738|19|1001-5000|9|3| +4739|20|1001-5000|9|3| +4740|1|5001-10000|9|3| +4741|2|5001-10000|9|3| +4742|3|5001-10000|9|3| +4743|4|5001-10000|9|3| +4744|5|5001-10000|9|3| +4745|6|5001-10000|9|3| +4746|7|5001-10000|9|3| +4747|8|5001-10000|9|3| +4748|9|5001-10000|9|3| +4749|10|5001-10000|9|3| +4750|11|5001-10000|9|3| +4751|12|5001-10000|9|3| +4752|13|5001-10000|9|3| +4753|14|5001-10000|9|3| +4754|15|5001-10000|9|3| +4755|16|5001-10000|9|3| +4756|17|5001-10000|9|3| +4757|18|5001-10000|9|3| +4758|19|5001-10000|9|3| +4759|20|5001-10000|9|3| +4760|1|>10000|9|3| +4761|2|>10000|9|3| +4762|3|>10000|9|3| +4763|4|>10000|9|3| +4764|5|>10000|9|3| +4765|6|>10000|9|3| +4766|7|>10000|9|3| +4767|8|>10000|9|3| +4768|9|>10000|9|3| +4769|10|>10000|9|3| +4770|11|>10000|9|3| +4771|12|>10000|9|3| +4772|13|>10000|9|3| +4773|14|>10000|9|3| +4774|15|>10000|9|3| +4775|16|>10000|9|3| +4776|17|>10000|9|3| +4777|18|>10000|9|3| +4778|19|>10000|9|3| +4779|20|>10000|9|3| +4780|1|Unknown|9|3| +4781|2|Unknown|9|3| +4782|3|Unknown|9|3| +4783|4|Unknown|9|3| +4784|5|Unknown|9|3| +4785|6|Unknown|9|3| +4786|7|Unknown|9|3| +4787|8|Unknown|9|3| +4788|9|Unknown|9|3| +4789|10|Unknown|9|3| +4790|11|Unknown|9|3| +4791|12|Unknown|9|3| +4792|13|Unknown|9|3| +4793|14|Unknown|9|3| +4794|15|Unknown|9|3| +4795|16|Unknown|9|3| +4796|17|Unknown|9|3| +4797|18|Unknown|9|3| +4798|19|Unknown|9|3| +4799|20|Unknown|9|3| +4800|1|0-500|0|4| +4801|2|0-500|0|4| +4802|3|0-500|0|4| +4803|4|0-500|0|4| +4804|5|0-500|0|4| +4805|6|0-500|0|4| +4806|7|0-500|0|4| +4807|8|0-500|0|4| +4808|9|0-500|0|4| +4809|10|0-500|0|4| +4810|11|0-500|0|4| +4811|12|0-500|0|4| +4812|13|0-500|0|4| +4813|14|0-500|0|4| +4814|15|0-500|0|4| +4815|16|0-500|0|4| +4816|17|0-500|0|4| +4817|18|0-500|0|4| +4818|19|0-500|0|4| +4819|20|0-500|0|4| +4820|1|501-1000|0|4| +4821|2|501-1000|0|4| +4822|3|501-1000|0|4| +4823|4|501-1000|0|4| +4824|5|501-1000|0|4| +4825|6|501-1000|0|4| +4826|7|501-1000|0|4| +4827|8|501-1000|0|4| +4828|9|501-1000|0|4| +4829|10|501-1000|0|4| +4830|11|501-1000|0|4| +4831|12|501-1000|0|4| +4832|13|501-1000|0|4| +4833|14|501-1000|0|4| +4834|15|501-1000|0|4| +4835|16|501-1000|0|4| +4836|17|501-1000|0|4| +4837|18|501-1000|0|4| +4838|19|501-1000|0|4| +4839|20|501-1000|0|4| +4840|1|1001-5000|0|4| +4841|2|1001-5000|0|4| +4842|3|1001-5000|0|4| +4843|4|1001-5000|0|4| +4844|5|1001-5000|0|4| +4845|6|1001-5000|0|4| +4846|7|1001-5000|0|4| +4847|8|1001-5000|0|4| +4848|9|1001-5000|0|4| +4849|10|1001-5000|0|4| +4850|11|1001-5000|0|4| +4851|12|1001-5000|0|4| +4852|13|1001-5000|0|4| +4853|14|1001-5000|0|4| +4854|15|1001-5000|0|4| +4855|16|1001-5000|0|4| +4856|17|1001-5000|0|4| +4857|18|1001-5000|0|4| +4858|19|1001-5000|0|4| +4859|20|1001-5000|0|4| +4860|1|5001-10000|0|4| +4861|2|5001-10000|0|4| +4862|3|5001-10000|0|4| +4863|4|5001-10000|0|4| +4864|5|5001-10000|0|4| +4865|6|5001-10000|0|4| +4866|7|5001-10000|0|4| +4867|8|5001-10000|0|4| +4868|9|5001-10000|0|4| +4869|10|5001-10000|0|4| +4870|11|5001-10000|0|4| +4871|12|5001-10000|0|4| +4872|13|5001-10000|0|4| +4873|14|5001-10000|0|4| +4874|15|5001-10000|0|4| +4875|16|5001-10000|0|4| +4876|17|5001-10000|0|4| +4877|18|5001-10000|0|4| +4878|19|5001-10000|0|4| +4879|20|5001-10000|0|4| +4880|1|>10000|0|4| +4881|2|>10000|0|4| +4882|3|>10000|0|4| +4883|4|>10000|0|4| +4884|5|>10000|0|4| +4885|6|>10000|0|4| +4886|7|>10000|0|4| +4887|8|>10000|0|4| +4888|9|>10000|0|4| +4889|10|>10000|0|4| +4890|11|>10000|0|4| +4891|12|>10000|0|4| +4892|13|>10000|0|4| +4893|14|>10000|0|4| +4894|15|>10000|0|4| +4895|16|>10000|0|4| +4896|17|>10000|0|4| +4897|18|>10000|0|4| +4898|19|>10000|0|4| +4899|20|>10000|0|4| +4900|1|Unknown|0|4| +4901|2|Unknown|0|4| +4902|3|Unknown|0|4| +4903|4|Unknown|0|4| +4904|5|Unknown|0|4| +4905|6|Unknown|0|4| +4906|7|Unknown|0|4| +4907|8|Unknown|0|4| +4908|9|Unknown|0|4| +4909|10|Unknown|0|4| +4910|11|Unknown|0|4| +4911|12|Unknown|0|4| +4912|13|Unknown|0|4| +4913|14|Unknown|0|4| +4914|15|Unknown|0|4| +4915|16|Unknown|0|4| +4916|17|Unknown|0|4| +4917|18|Unknown|0|4| +4918|19|Unknown|0|4| +4919|20|Unknown|0|4| +4920|1|0-500|1|4| +4921|2|0-500|1|4| +4922|3|0-500|1|4| +4923|4|0-500|1|4| +4924|5|0-500|1|4| +4925|6|0-500|1|4| +4926|7|0-500|1|4| +4927|8|0-500|1|4| +4928|9|0-500|1|4| +4929|10|0-500|1|4| +4930|11|0-500|1|4| +4931|12|0-500|1|4| +4932|13|0-500|1|4| +4933|14|0-500|1|4| +4934|15|0-500|1|4| +4935|16|0-500|1|4| +4936|17|0-500|1|4| +4937|18|0-500|1|4| +4938|19|0-500|1|4| +4939|20|0-500|1|4| +4940|1|501-1000|1|4| +4941|2|501-1000|1|4| +4942|3|501-1000|1|4| +4943|4|501-1000|1|4| +4944|5|501-1000|1|4| +4945|6|501-1000|1|4| +4946|7|501-1000|1|4| +4947|8|501-1000|1|4| +4948|9|501-1000|1|4| +4949|10|501-1000|1|4| +4950|11|501-1000|1|4| +4951|12|501-1000|1|4| +4952|13|501-1000|1|4| +4953|14|501-1000|1|4| +4954|15|501-1000|1|4| +4955|16|501-1000|1|4| +4956|17|501-1000|1|4| +4957|18|501-1000|1|4| +4958|19|501-1000|1|4| +4959|20|501-1000|1|4| +4960|1|1001-5000|1|4| +4961|2|1001-5000|1|4| +4962|3|1001-5000|1|4| +4963|4|1001-5000|1|4| +4964|5|1001-5000|1|4| +4965|6|1001-5000|1|4| +4966|7|1001-5000|1|4| +4967|8|1001-5000|1|4| +4968|9|1001-5000|1|4| +4969|10|1001-5000|1|4| +4970|11|1001-5000|1|4| +4971|12|1001-5000|1|4| +4972|13|1001-5000|1|4| +4973|14|1001-5000|1|4| +4974|15|1001-5000|1|4| +4975|16|1001-5000|1|4| +4976|17|1001-5000|1|4| +4977|18|1001-5000|1|4| +4978|19|1001-5000|1|4| +4979|20|1001-5000|1|4| +4980|1|5001-10000|1|4| +4981|2|5001-10000|1|4| +4982|3|5001-10000|1|4| +4983|4|5001-10000|1|4| +4984|5|5001-10000|1|4| +4985|6|5001-10000|1|4| +4986|7|5001-10000|1|4| +4987|8|5001-10000|1|4| +4988|9|5001-10000|1|4| +4989|10|5001-10000|1|4| +4990|11|5001-10000|1|4| +4991|12|5001-10000|1|4| +4992|13|5001-10000|1|4| +4993|14|5001-10000|1|4| +4994|15|5001-10000|1|4| +4995|16|5001-10000|1|4| +4996|17|5001-10000|1|4| +4997|18|5001-10000|1|4| +4998|19|5001-10000|1|4| +4999|20|5001-10000|1|4| +5000|1|>10000|1|4| +5001|2|>10000|1|4| +5002|3|>10000|1|4| +5003|4|>10000|1|4| +5004|5|>10000|1|4| +5005|6|>10000|1|4| +5006|7|>10000|1|4| +5007|8|>10000|1|4| +5008|9|>10000|1|4| +5009|10|>10000|1|4| +5010|11|>10000|1|4| +5011|12|>10000|1|4| +5012|13|>10000|1|4| +5013|14|>10000|1|4| +5014|15|>10000|1|4| +5015|16|>10000|1|4| +5016|17|>10000|1|4| +5017|18|>10000|1|4| +5018|19|>10000|1|4| +5019|20|>10000|1|4| +5020|1|Unknown|1|4| +5021|2|Unknown|1|4| +5022|3|Unknown|1|4| +5023|4|Unknown|1|4| +5024|5|Unknown|1|4| +5025|6|Unknown|1|4| +5026|7|Unknown|1|4| +5027|8|Unknown|1|4| +5028|9|Unknown|1|4| +5029|10|Unknown|1|4| +5030|11|Unknown|1|4| +5031|12|Unknown|1|4| +5032|13|Unknown|1|4| +5033|14|Unknown|1|4| +5034|15|Unknown|1|4| +5035|16|Unknown|1|4| +5036|17|Unknown|1|4| +5037|18|Unknown|1|4| +5038|19|Unknown|1|4| +5039|20|Unknown|1|4| +5040|1|0-500|2|4| +5041|2|0-500|2|4| +5042|3|0-500|2|4| +5043|4|0-500|2|4| +5044|5|0-500|2|4| +5045|6|0-500|2|4| +5046|7|0-500|2|4| +5047|8|0-500|2|4| +5048|9|0-500|2|4| +5049|10|0-500|2|4| +5050|11|0-500|2|4| +5051|12|0-500|2|4| +5052|13|0-500|2|4| +5053|14|0-500|2|4| +5054|15|0-500|2|4| +5055|16|0-500|2|4| +5056|17|0-500|2|4| +5057|18|0-500|2|4| +5058|19|0-500|2|4| +5059|20|0-500|2|4| +5060|1|501-1000|2|4| +5061|2|501-1000|2|4| +5062|3|501-1000|2|4| +5063|4|501-1000|2|4| +5064|5|501-1000|2|4| +5065|6|501-1000|2|4| +5066|7|501-1000|2|4| +5067|8|501-1000|2|4| +5068|9|501-1000|2|4| +5069|10|501-1000|2|4| +5070|11|501-1000|2|4| +5071|12|501-1000|2|4| +5072|13|501-1000|2|4| +5073|14|501-1000|2|4| +5074|15|501-1000|2|4| +5075|16|501-1000|2|4| +5076|17|501-1000|2|4| +5077|18|501-1000|2|4| +5078|19|501-1000|2|4| +5079|20|501-1000|2|4| +5080|1|1001-5000|2|4| +5081|2|1001-5000|2|4| +5082|3|1001-5000|2|4| +5083|4|1001-5000|2|4| +5084|5|1001-5000|2|4| +5085|6|1001-5000|2|4| +5086|7|1001-5000|2|4| +5087|8|1001-5000|2|4| +5088|9|1001-5000|2|4| +5089|10|1001-5000|2|4| +5090|11|1001-5000|2|4| +5091|12|1001-5000|2|4| +5092|13|1001-5000|2|4| +5093|14|1001-5000|2|4| +5094|15|1001-5000|2|4| +5095|16|1001-5000|2|4| +5096|17|1001-5000|2|4| +5097|18|1001-5000|2|4| +5098|19|1001-5000|2|4| +5099|20|1001-5000|2|4| +5100|1|5001-10000|2|4| +5101|2|5001-10000|2|4| +5102|3|5001-10000|2|4| +5103|4|5001-10000|2|4| +5104|5|5001-10000|2|4| +5105|6|5001-10000|2|4| +5106|7|5001-10000|2|4| +5107|8|5001-10000|2|4| +5108|9|5001-10000|2|4| +5109|10|5001-10000|2|4| +5110|11|5001-10000|2|4| +5111|12|5001-10000|2|4| +5112|13|5001-10000|2|4| +5113|14|5001-10000|2|4| +5114|15|5001-10000|2|4| +5115|16|5001-10000|2|4| +5116|17|5001-10000|2|4| +5117|18|5001-10000|2|4| +5118|19|5001-10000|2|4| +5119|20|5001-10000|2|4| +5120|1|>10000|2|4| +5121|2|>10000|2|4| +5122|3|>10000|2|4| +5123|4|>10000|2|4| +5124|5|>10000|2|4| +5125|6|>10000|2|4| +5126|7|>10000|2|4| +5127|8|>10000|2|4| +5128|9|>10000|2|4| +5129|10|>10000|2|4| +5130|11|>10000|2|4| +5131|12|>10000|2|4| +5132|13|>10000|2|4| +5133|14|>10000|2|4| +5134|15|>10000|2|4| +5135|16|>10000|2|4| +5136|17|>10000|2|4| +5137|18|>10000|2|4| +5138|19|>10000|2|4| +5139|20|>10000|2|4| +5140|1|Unknown|2|4| +5141|2|Unknown|2|4| +5142|3|Unknown|2|4| +5143|4|Unknown|2|4| +5144|5|Unknown|2|4| +5145|6|Unknown|2|4| +5146|7|Unknown|2|4| +5147|8|Unknown|2|4| +5148|9|Unknown|2|4| +5149|10|Unknown|2|4| +5150|11|Unknown|2|4| +5151|12|Unknown|2|4| +5152|13|Unknown|2|4| +5153|14|Unknown|2|4| +5154|15|Unknown|2|4| +5155|16|Unknown|2|4| +5156|17|Unknown|2|4| +5157|18|Unknown|2|4| +5158|19|Unknown|2|4| +5159|20|Unknown|2|4| +5160|1|0-500|3|4| +5161|2|0-500|3|4| +5162|3|0-500|3|4| +5163|4|0-500|3|4| +5164|5|0-500|3|4| +5165|6|0-500|3|4| +5166|7|0-500|3|4| +5167|8|0-500|3|4| +5168|9|0-500|3|4| +5169|10|0-500|3|4| +5170|11|0-500|3|4| +5171|12|0-500|3|4| +5172|13|0-500|3|4| +5173|14|0-500|3|4| +5174|15|0-500|3|4| +5175|16|0-500|3|4| +5176|17|0-500|3|4| +5177|18|0-500|3|4| +5178|19|0-500|3|4| +5179|20|0-500|3|4| +5180|1|501-1000|3|4| +5181|2|501-1000|3|4| +5182|3|501-1000|3|4| +5183|4|501-1000|3|4| +5184|5|501-1000|3|4| +5185|6|501-1000|3|4| +5186|7|501-1000|3|4| +5187|8|501-1000|3|4| +5188|9|501-1000|3|4| +5189|10|501-1000|3|4| +5190|11|501-1000|3|4| +5191|12|501-1000|3|4| +5192|13|501-1000|3|4| +5193|14|501-1000|3|4| +5194|15|501-1000|3|4| +5195|16|501-1000|3|4| +5196|17|501-1000|3|4| +5197|18|501-1000|3|4| +5198|19|501-1000|3|4| +5199|20|501-1000|3|4| +5200|1|1001-5000|3|4| +5201|2|1001-5000|3|4| +5202|3|1001-5000|3|4| +5203|4|1001-5000|3|4| +5204|5|1001-5000|3|4| +5205|6|1001-5000|3|4| +5206|7|1001-5000|3|4| +5207|8|1001-5000|3|4| +5208|9|1001-5000|3|4| +5209|10|1001-5000|3|4| +5210|11|1001-5000|3|4| +5211|12|1001-5000|3|4| +5212|13|1001-5000|3|4| +5213|14|1001-5000|3|4| +5214|15|1001-5000|3|4| +5215|16|1001-5000|3|4| +5216|17|1001-5000|3|4| +5217|18|1001-5000|3|4| +5218|19|1001-5000|3|4| +5219|20|1001-5000|3|4| +5220|1|5001-10000|3|4| +5221|2|5001-10000|3|4| +5222|3|5001-10000|3|4| +5223|4|5001-10000|3|4| +5224|5|5001-10000|3|4| +5225|6|5001-10000|3|4| +5226|7|5001-10000|3|4| +5227|8|5001-10000|3|4| +5228|9|5001-10000|3|4| +5229|10|5001-10000|3|4| +5230|11|5001-10000|3|4| +5231|12|5001-10000|3|4| +5232|13|5001-10000|3|4| +5233|14|5001-10000|3|4| +5234|15|5001-10000|3|4| +5235|16|5001-10000|3|4| +5236|17|5001-10000|3|4| +5237|18|5001-10000|3|4| +5238|19|5001-10000|3|4| +5239|20|5001-10000|3|4| +5240|1|>10000|3|4| +5241|2|>10000|3|4| +5242|3|>10000|3|4| +5243|4|>10000|3|4| +5244|5|>10000|3|4| +5245|6|>10000|3|4| +5246|7|>10000|3|4| +5247|8|>10000|3|4| +5248|9|>10000|3|4| +5249|10|>10000|3|4| +5250|11|>10000|3|4| +5251|12|>10000|3|4| +5252|13|>10000|3|4| +5253|14|>10000|3|4| +5254|15|>10000|3|4| +5255|16|>10000|3|4| +5256|17|>10000|3|4| +5257|18|>10000|3|4| +5258|19|>10000|3|4| +5259|20|>10000|3|4| +5260|1|Unknown|3|4| +5261|2|Unknown|3|4| +5262|3|Unknown|3|4| +5263|4|Unknown|3|4| +5264|5|Unknown|3|4| +5265|6|Unknown|3|4| +5266|7|Unknown|3|4| +5267|8|Unknown|3|4| +5268|9|Unknown|3|4| +5269|10|Unknown|3|4| +5270|11|Unknown|3|4| +5271|12|Unknown|3|4| +5272|13|Unknown|3|4| +5273|14|Unknown|3|4| +5274|15|Unknown|3|4| +5275|16|Unknown|3|4| +5276|17|Unknown|3|4| +5277|18|Unknown|3|4| +5278|19|Unknown|3|4| +5279|20|Unknown|3|4| +5280|1|0-500|4|4| +5281|2|0-500|4|4| +5282|3|0-500|4|4| +5283|4|0-500|4|4| +5284|5|0-500|4|4| +5285|6|0-500|4|4| +5286|7|0-500|4|4| +5287|8|0-500|4|4| +5288|9|0-500|4|4| +5289|10|0-500|4|4| +5290|11|0-500|4|4| +5291|12|0-500|4|4| +5292|13|0-500|4|4| +5293|14|0-500|4|4| +5294|15|0-500|4|4| +5295|16|0-500|4|4| +5296|17|0-500|4|4| +5297|18|0-500|4|4| +5298|19|0-500|4|4| +5299|20|0-500|4|4| +5300|1|501-1000|4|4| +5301|2|501-1000|4|4| +5302|3|501-1000|4|4| +5303|4|501-1000|4|4| +5304|5|501-1000|4|4| +5305|6|501-1000|4|4| +5306|7|501-1000|4|4| +5307|8|501-1000|4|4| +5308|9|501-1000|4|4| +5309|10|501-1000|4|4| +5310|11|501-1000|4|4| +5311|12|501-1000|4|4| +5312|13|501-1000|4|4| +5313|14|501-1000|4|4| +5314|15|501-1000|4|4| +5315|16|501-1000|4|4| +5316|17|501-1000|4|4| +5317|18|501-1000|4|4| +5318|19|501-1000|4|4| +5319|20|501-1000|4|4| +5320|1|1001-5000|4|4| +5321|2|1001-5000|4|4| +5322|3|1001-5000|4|4| +5323|4|1001-5000|4|4| +5324|5|1001-5000|4|4| +5325|6|1001-5000|4|4| +5326|7|1001-5000|4|4| +5327|8|1001-5000|4|4| +5328|9|1001-5000|4|4| +5329|10|1001-5000|4|4| +5330|11|1001-5000|4|4| +5331|12|1001-5000|4|4| +5332|13|1001-5000|4|4| +5333|14|1001-5000|4|4| +5334|15|1001-5000|4|4| +5335|16|1001-5000|4|4| +5336|17|1001-5000|4|4| +5337|18|1001-5000|4|4| +5338|19|1001-5000|4|4| +5339|20|1001-5000|4|4| +5340|1|5001-10000|4|4| +5341|2|5001-10000|4|4| +5342|3|5001-10000|4|4| +5343|4|5001-10000|4|4| +5344|5|5001-10000|4|4| +5345|6|5001-10000|4|4| +5346|7|5001-10000|4|4| +5347|8|5001-10000|4|4| +5348|9|5001-10000|4|4| +5349|10|5001-10000|4|4| +5350|11|5001-10000|4|4| +5351|12|5001-10000|4|4| +5352|13|5001-10000|4|4| +5353|14|5001-10000|4|4| +5354|15|5001-10000|4|4| +5355|16|5001-10000|4|4| +5356|17|5001-10000|4|4| +5357|18|5001-10000|4|4| +5358|19|5001-10000|4|4| +5359|20|5001-10000|4|4| +5360|1|>10000|4|4| +5361|2|>10000|4|4| +5362|3|>10000|4|4| +5363|4|>10000|4|4| +5364|5|>10000|4|4| +5365|6|>10000|4|4| +5366|7|>10000|4|4| +5367|8|>10000|4|4| +5368|9|>10000|4|4| +5369|10|>10000|4|4| +5370|11|>10000|4|4| +5371|12|>10000|4|4| +5372|13|>10000|4|4| +5373|14|>10000|4|4| +5374|15|>10000|4|4| +5375|16|>10000|4|4| +5376|17|>10000|4|4| +5377|18|>10000|4|4| +5378|19|>10000|4|4| +5379|20|>10000|4|4| +5380|1|Unknown|4|4| +5381|2|Unknown|4|4| +5382|3|Unknown|4|4| +5383|4|Unknown|4|4| +5384|5|Unknown|4|4| +5385|6|Unknown|4|4| +5386|7|Unknown|4|4| +5387|8|Unknown|4|4| +5388|9|Unknown|4|4| +5389|10|Unknown|4|4| +5390|11|Unknown|4|4| +5391|12|Unknown|4|4| +5392|13|Unknown|4|4| +5393|14|Unknown|4|4| +5394|15|Unknown|4|4| +5395|16|Unknown|4|4| +5396|17|Unknown|4|4| +5397|18|Unknown|4|4| +5398|19|Unknown|4|4| +5399|20|Unknown|4|4| +5400|1|0-500|5|4| +5401|2|0-500|5|4| +5402|3|0-500|5|4| +5403|4|0-500|5|4| +5404|5|0-500|5|4| +5405|6|0-500|5|4| +5406|7|0-500|5|4| +5407|8|0-500|5|4| +5408|9|0-500|5|4| +5409|10|0-500|5|4| +5410|11|0-500|5|4| +5411|12|0-500|5|4| +5412|13|0-500|5|4| +5413|14|0-500|5|4| +5414|15|0-500|5|4| +5415|16|0-500|5|4| +5416|17|0-500|5|4| +5417|18|0-500|5|4| +5418|19|0-500|5|4| +5419|20|0-500|5|4| +5420|1|501-1000|5|4| +5421|2|501-1000|5|4| +5422|3|501-1000|5|4| +5423|4|501-1000|5|4| +5424|5|501-1000|5|4| +5425|6|501-1000|5|4| +5426|7|501-1000|5|4| +5427|8|501-1000|5|4| +5428|9|501-1000|5|4| +5429|10|501-1000|5|4| +5430|11|501-1000|5|4| +5431|12|501-1000|5|4| +5432|13|501-1000|5|4| +5433|14|501-1000|5|4| +5434|15|501-1000|5|4| +5435|16|501-1000|5|4| +5436|17|501-1000|5|4| +5437|18|501-1000|5|4| +5438|19|501-1000|5|4| +5439|20|501-1000|5|4| +5440|1|1001-5000|5|4| +5441|2|1001-5000|5|4| +5442|3|1001-5000|5|4| +5443|4|1001-5000|5|4| +5444|5|1001-5000|5|4| +5445|6|1001-5000|5|4| +5446|7|1001-5000|5|4| +5447|8|1001-5000|5|4| +5448|9|1001-5000|5|4| +5449|10|1001-5000|5|4| +5450|11|1001-5000|5|4| +5451|12|1001-5000|5|4| +5452|13|1001-5000|5|4| +5453|14|1001-5000|5|4| +5454|15|1001-5000|5|4| +5455|16|1001-5000|5|4| +5456|17|1001-5000|5|4| +5457|18|1001-5000|5|4| +5458|19|1001-5000|5|4| +5459|20|1001-5000|5|4| +5460|1|5001-10000|5|4| +5461|2|5001-10000|5|4| +5462|3|5001-10000|5|4| +5463|4|5001-10000|5|4| +5464|5|5001-10000|5|4| +5465|6|5001-10000|5|4| +5466|7|5001-10000|5|4| +5467|8|5001-10000|5|4| +5468|9|5001-10000|5|4| +5469|10|5001-10000|5|4| +5470|11|5001-10000|5|4| +5471|12|5001-10000|5|4| +5472|13|5001-10000|5|4| +5473|14|5001-10000|5|4| +5474|15|5001-10000|5|4| +5475|16|5001-10000|5|4| +5476|17|5001-10000|5|4| +5477|18|5001-10000|5|4| +5478|19|5001-10000|5|4| +5479|20|5001-10000|5|4| +5480|1|>10000|5|4| +5481|2|>10000|5|4| +5482|3|>10000|5|4| +5483|4|>10000|5|4| +5484|5|>10000|5|4| +5485|6|>10000|5|4| +5486|7|>10000|5|4| +5487|8|>10000|5|4| +5488|9|>10000|5|4| +5489|10|>10000|5|4| +5490|11|>10000|5|4| +5491|12|>10000|5|4| +5492|13|>10000|5|4| +5493|14|>10000|5|4| +5494|15|>10000|5|4| +5495|16|>10000|5|4| +5496|17|>10000|5|4| +5497|18|>10000|5|4| +5498|19|>10000|5|4| +5499|20|>10000|5|4| +5500|1|Unknown|5|4| +5501|2|Unknown|5|4| +5502|3|Unknown|5|4| +5503|4|Unknown|5|4| +5504|5|Unknown|5|4| +5505|6|Unknown|5|4| +5506|7|Unknown|5|4| +5507|8|Unknown|5|4| +5508|9|Unknown|5|4| +5509|10|Unknown|5|4| +5510|11|Unknown|5|4| +5511|12|Unknown|5|4| +5512|13|Unknown|5|4| +5513|14|Unknown|5|4| +5514|15|Unknown|5|4| +5515|16|Unknown|5|4| +5516|17|Unknown|5|4| +5517|18|Unknown|5|4| +5518|19|Unknown|5|4| +5519|20|Unknown|5|4| +5520|1|0-500|6|4| +5521|2|0-500|6|4| +5522|3|0-500|6|4| +5523|4|0-500|6|4| +5524|5|0-500|6|4| +5525|6|0-500|6|4| +5526|7|0-500|6|4| +5527|8|0-500|6|4| +5528|9|0-500|6|4| +5529|10|0-500|6|4| +5530|11|0-500|6|4| +5531|12|0-500|6|4| +5532|13|0-500|6|4| +5533|14|0-500|6|4| +5534|15|0-500|6|4| +5535|16|0-500|6|4| +5536|17|0-500|6|4| +5537|18|0-500|6|4| +5538|19|0-500|6|4| +5539|20|0-500|6|4| +5540|1|501-1000|6|4| +5541|2|501-1000|6|4| +5542|3|501-1000|6|4| +5543|4|501-1000|6|4| +5544|5|501-1000|6|4| +5545|6|501-1000|6|4| +5546|7|501-1000|6|4| +5547|8|501-1000|6|4| +5548|9|501-1000|6|4| +5549|10|501-1000|6|4| +5550|11|501-1000|6|4| +5551|12|501-1000|6|4| +5552|13|501-1000|6|4| +5553|14|501-1000|6|4| +5554|15|501-1000|6|4| +5555|16|501-1000|6|4| +5556|17|501-1000|6|4| +5557|18|501-1000|6|4| +5558|19|501-1000|6|4| +5559|20|501-1000|6|4| +5560|1|1001-5000|6|4| +5561|2|1001-5000|6|4| +5562|3|1001-5000|6|4| +5563|4|1001-5000|6|4| +5564|5|1001-5000|6|4| +5565|6|1001-5000|6|4| +5566|7|1001-5000|6|4| +5567|8|1001-5000|6|4| +5568|9|1001-5000|6|4| +5569|10|1001-5000|6|4| +5570|11|1001-5000|6|4| +5571|12|1001-5000|6|4| +5572|13|1001-5000|6|4| +5573|14|1001-5000|6|4| +5574|15|1001-5000|6|4| +5575|16|1001-5000|6|4| +5576|17|1001-5000|6|4| +5577|18|1001-5000|6|4| +5578|19|1001-5000|6|4| +5579|20|1001-5000|6|4| +5580|1|5001-10000|6|4| +5581|2|5001-10000|6|4| +5582|3|5001-10000|6|4| +5583|4|5001-10000|6|4| +5584|5|5001-10000|6|4| +5585|6|5001-10000|6|4| +5586|7|5001-10000|6|4| +5587|8|5001-10000|6|4| +5588|9|5001-10000|6|4| +5589|10|5001-10000|6|4| +5590|11|5001-10000|6|4| +5591|12|5001-10000|6|4| +5592|13|5001-10000|6|4| +5593|14|5001-10000|6|4| +5594|15|5001-10000|6|4| +5595|16|5001-10000|6|4| +5596|17|5001-10000|6|4| +5597|18|5001-10000|6|4| +5598|19|5001-10000|6|4| +5599|20|5001-10000|6|4| +5600|1|>10000|6|4| +5601|2|>10000|6|4| +5602|3|>10000|6|4| +5603|4|>10000|6|4| +5604|5|>10000|6|4| +5605|6|>10000|6|4| +5606|7|>10000|6|4| +5607|8|>10000|6|4| +5608|9|>10000|6|4| +5609|10|>10000|6|4| +5610|11|>10000|6|4| +5611|12|>10000|6|4| +5612|13|>10000|6|4| +5613|14|>10000|6|4| +5614|15|>10000|6|4| +5615|16|>10000|6|4| +5616|17|>10000|6|4| +5617|18|>10000|6|4| +5618|19|>10000|6|4| +5619|20|>10000|6|4| +5620|1|Unknown|6|4| +5621|2|Unknown|6|4| +5622|3|Unknown|6|4| +5623|4|Unknown|6|4| +5624|5|Unknown|6|4| +5625|6|Unknown|6|4| +5626|7|Unknown|6|4| +5627|8|Unknown|6|4| +5628|9|Unknown|6|4| +5629|10|Unknown|6|4| +5630|11|Unknown|6|4| +5631|12|Unknown|6|4| +5632|13|Unknown|6|4| +5633|14|Unknown|6|4| +5634|15|Unknown|6|4| +5635|16|Unknown|6|4| +5636|17|Unknown|6|4| +5637|18|Unknown|6|4| +5638|19|Unknown|6|4| +5639|20|Unknown|6|4| +5640|1|0-500|7|4| +5641|2|0-500|7|4| +5642|3|0-500|7|4| +5643|4|0-500|7|4| +5644|5|0-500|7|4| +5645|6|0-500|7|4| +5646|7|0-500|7|4| +5647|8|0-500|7|4| +5648|9|0-500|7|4| +5649|10|0-500|7|4| +5650|11|0-500|7|4| +5651|12|0-500|7|4| +5652|13|0-500|7|4| +5653|14|0-500|7|4| +5654|15|0-500|7|4| +5655|16|0-500|7|4| +5656|17|0-500|7|4| +5657|18|0-500|7|4| +5658|19|0-500|7|4| +5659|20|0-500|7|4| +5660|1|501-1000|7|4| +5661|2|501-1000|7|4| +5662|3|501-1000|7|4| +5663|4|501-1000|7|4| +5664|5|501-1000|7|4| +5665|6|501-1000|7|4| +5666|7|501-1000|7|4| +5667|8|501-1000|7|4| +5668|9|501-1000|7|4| +5669|10|501-1000|7|4| +5670|11|501-1000|7|4| +5671|12|501-1000|7|4| +5672|13|501-1000|7|4| +5673|14|501-1000|7|4| +5674|15|501-1000|7|4| +5675|16|501-1000|7|4| +5676|17|501-1000|7|4| +5677|18|501-1000|7|4| +5678|19|501-1000|7|4| +5679|20|501-1000|7|4| +5680|1|1001-5000|7|4| +5681|2|1001-5000|7|4| +5682|3|1001-5000|7|4| +5683|4|1001-5000|7|4| +5684|5|1001-5000|7|4| +5685|6|1001-5000|7|4| +5686|7|1001-5000|7|4| +5687|8|1001-5000|7|4| +5688|9|1001-5000|7|4| +5689|10|1001-5000|7|4| +5690|11|1001-5000|7|4| +5691|12|1001-5000|7|4| +5692|13|1001-5000|7|4| +5693|14|1001-5000|7|4| +5694|15|1001-5000|7|4| +5695|16|1001-5000|7|4| +5696|17|1001-5000|7|4| +5697|18|1001-5000|7|4| +5698|19|1001-5000|7|4| +5699|20|1001-5000|7|4| +5700|1|5001-10000|7|4| +5701|2|5001-10000|7|4| +5702|3|5001-10000|7|4| +5703|4|5001-10000|7|4| +5704|5|5001-10000|7|4| +5705|6|5001-10000|7|4| +5706|7|5001-10000|7|4| +5707|8|5001-10000|7|4| +5708|9|5001-10000|7|4| +5709|10|5001-10000|7|4| +5710|11|5001-10000|7|4| +5711|12|5001-10000|7|4| +5712|13|5001-10000|7|4| +5713|14|5001-10000|7|4| +5714|15|5001-10000|7|4| +5715|16|5001-10000|7|4| +5716|17|5001-10000|7|4| +5717|18|5001-10000|7|4| +5718|19|5001-10000|7|4| +5719|20|5001-10000|7|4| +5720|1|>10000|7|4| +5721|2|>10000|7|4| +5722|3|>10000|7|4| +5723|4|>10000|7|4| +5724|5|>10000|7|4| +5725|6|>10000|7|4| +5726|7|>10000|7|4| +5727|8|>10000|7|4| +5728|9|>10000|7|4| +5729|10|>10000|7|4| +5730|11|>10000|7|4| +5731|12|>10000|7|4| +5732|13|>10000|7|4| +5733|14|>10000|7|4| +5734|15|>10000|7|4| +5735|16|>10000|7|4| +5736|17|>10000|7|4| +5737|18|>10000|7|4| +5738|19|>10000|7|4| +5739|20|>10000|7|4| +5740|1|Unknown|7|4| +5741|2|Unknown|7|4| +5742|3|Unknown|7|4| +5743|4|Unknown|7|4| +5744|5|Unknown|7|4| +5745|6|Unknown|7|4| +5746|7|Unknown|7|4| +5747|8|Unknown|7|4| +5748|9|Unknown|7|4| +5749|10|Unknown|7|4| +5750|11|Unknown|7|4| +5751|12|Unknown|7|4| +5752|13|Unknown|7|4| +5753|14|Unknown|7|4| +5754|15|Unknown|7|4| +5755|16|Unknown|7|4| +5756|17|Unknown|7|4| +5757|18|Unknown|7|4| +5758|19|Unknown|7|4| +5759|20|Unknown|7|4| +5760|1|0-500|8|4| +5761|2|0-500|8|4| +5762|3|0-500|8|4| +5763|4|0-500|8|4| +5764|5|0-500|8|4| +5765|6|0-500|8|4| +5766|7|0-500|8|4| +5767|8|0-500|8|4| +5768|9|0-500|8|4| +5769|10|0-500|8|4| +5770|11|0-500|8|4| +5771|12|0-500|8|4| +5772|13|0-500|8|4| +5773|14|0-500|8|4| +5774|15|0-500|8|4| +5775|16|0-500|8|4| +5776|17|0-500|8|4| +5777|18|0-500|8|4| +5778|19|0-500|8|4| +5779|20|0-500|8|4| +5780|1|501-1000|8|4| +5781|2|501-1000|8|4| +5782|3|501-1000|8|4| +5783|4|501-1000|8|4| +5784|5|501-1000|8|4| +5785|6|501-1000|8|4| +5786|7|501-1000|8|4| +5787|8|501-1000|8|4| +5788|9|501-1000|8|4| +5789|10|501-1000|8|4| +5790|11|501-1000|8|4| +5791|12|501-1000|8|4| +5792|13|501-1000|8|4| +5793|14|501-1000|8|4| +5794|15|501-1000|8|4| +5795|16|501-1000|8|4| +5796|17|501-1000|8|4| +5797|18|501-1000|8|4| +5798|19|501-1000|8|4| +5799|20|501-1000|8|4| +5800|1|1001-5000|8|4| +5801|2|1001-5000|8|4| +5802|3|1001-5000|8|4| +5803|4|1001-5000|8|4| +5804|5|1001-5000|8|4| +5805|6|1001-5000|8|4| +5806|7|1001-5000|8|4| +5807|8|1001-5000|8|4| +5808|9|1001-5000|8|4| +5809|10|1001-5000|8|4| +5810|11|1001-5000|8|4| +5811|12|1001-5000|8|4| +5812|13|1001-5000|8|4| +5813|14|1001-5000|8|4| +5814|15|1001-5000|8|4| +5815|16|1001-5000|8|4| +5816|17|1001-5000|8|4| +5817|18|1001-5000|8|4| +5818|19|1001-5000|8|4| +5819|20|1001-5000|8|4| +5820|1|5001-10000|8|4| +5821|2|5001-10000|8|4| +5822|3|5001-10000|8|4| +5823|4|5001-10000|8|4| +5824|5|5001-10000|8|4| +5825|6|5001-10000|8|4| +5826|7|5001-10000|8|4| +5827|8|5001-10000|8|4| +5828|9|5001-10000|8|4| +5829|10|5001-10000|8|4| +5830|11|5001-10000|8|4| +5831|12|5001-10000|8|4| +5832|13|5001-10000|8|4| +5833|14|5001-10000|8|4| +5834|15|5001-10000|8|4| +5835|16|5001-10000|8|4| +5836|17|5001-10000|8|4| +5837|18|5001-10000|8|4| +5838|19|5001-10000|8|4| +5839|20|5001-10000|8|4| +5840|1|>10000|8|4| +5841|2|>10000|8|4| +5842|3|>10000|8|4| +5843|4|>10000|8|4| +5844|5|>10000|8|4| +5845|6|>10000|8|4| +5846|7|>10000|8|4| +5847|8|>10000|8|4| +5848|9|>10000|8|4| +5849|10|>10000|8|4| +5850|11|>10000|8|4| +5851|12|>10000|8|4| +5852|13|>10000|8|4| +5853|14|>10000|8|4| +5854|15|>10000|8|4| +5855|16|>10000|8|4| +5856|17|>10000|8|4| +5857|18|>10000|8|4| +5858|19|>10000|8|4| +5859|20|>10000|8|4| +5860|1|Unknown|8|4| +5861|2|Unknown|8|4| +5862|3|Unknown|8|4| +5863|4|Unknown|8|4| +5864|5|Unknown|8|4| +5865|6|Unknown|8|4| +5866|7|Unknown|8|4| +5867|8|Unknown|8|4| +5868|9|Unknown|8|4| +5869|10|Unknown|8|4| +5870|11|Unknown|8|4| +5871|12|Unknown|8|4| +5872|13|Unknown|8|4| +5873|14|Unknown|8|4| +5874|15|Unknown|8|4| +5875|16|Unknown|8|4| +5876|17|Unknown|8|4| +5877|18|Unknown|8|4| +5878|19|Unknown|8|4| +5879|20|Unknown|8|4| +5880|1|0-500|9|4| +5881|2|0-500|9|4| +5882|3|0-500|9|4| +5883|4|0-500|9|4| +5884|5|0-500|9|4| +5885|6|0-500|9|4| +5886|7|0-500|9|4| +5887|8|0-500|9|4| +5888|9|0-500|9|4| +5889|10|0-500|9|4| +5890|11|0-500|9|4| +5891|12|0-500|9|4| +5892|13|0-500|9|4| +5893|14|0-500|9|4| +5894|15|0-500|9|4| +5895|16|0-500|9|4| +5896|17|0-500|9|4| +5897|18|0-500|9|4| +5898|19|0-500|9|4| +5899|20|0-500|9|4| +5900|1|501-1000|9|4| +5901|2|501-1000|9|4| +5902|3|501-1000|9|4| +5903|4|501-1000|9|4| +5904|5|501-1000|9|4| +5905|6|501-1000|9|4| +5906|7|501-1000|9|4| +5907|8|501-1000|9|4| +5908|9|501-1000|9|4| +5909|10|501-1000|9|4| +5910|11|501-1000|9|4| +5911|12|501-1000|9|4| +5912|13|501-1000|9|4| +5913|14|501-1000|9|4| +5914|15|501-1000|9|4| +5915|16|501-1000|9|4| +5916|17|501-1000|9|4| +5917|18|501-1000|9|4| +5918|19|501-1000|9|4| +5919|20|501-1000|9|4| +5920|1|1001-5000|9|4| +5921|2|1001-5000|9|4| +5922|3|1001-5000|9|4| +5923|4|1001-5000|9|4| +5924|5|1001-5000|9|4| +5925|6|1001-5000|9|4| +5926|7|1001-5000|9|4| +5927|8|1001-5000|9|4| +5928|9|1001-5000|9|4| +5929|10|1001-5000|9|4| +5930|11|1001-5000|9|4| +5931|12|1001-5000|9|4| +5932|13|1001-5000|9|4| +5933|14|1001-5000|9|4| +5934|15|1001-5000|9|4| +5935|16|1001-5000|9|4| +5936|17|1001-5000|9|4| +5937|18|1001-5000|9|4| +5938|19|1001-5000|9|4| +5939|20|1001-5000|9|4| +5940|1|5001-10000|9|4| +5941|2|5001-10000|9|4| +5942|3|5001-10000|9|4| +5943|4|5001-10000|9|4| +5944|5|5001-10000|9|4| +5945|6|5001-10000|9|4| +5946|7|5001-10000|9|4| +5947|8|5001-10000|9|4| +5948|9|5001-10000|9|4| +5949|10|5001-10000|9|4| +5950|11|5001-10000|9|4| +5951|12|5001-10000|9|4| +5952|13|5001-10000|9|4| +5953|14|5001-10000|9|4| +5954|15|5001-10000|9|4| +5955|16|5001-10000|9|4| +5956|17|5001-10000|9|4| +5957|18|5001-10000|9|4| +5958|19|5001-10000|9|4| +5959|20|5001-10000|9|4| +5960|1|>10000|9|4| +5961|2|>10000|9|4| +5962|3|>10000|9|4| +5963|4|>10000|9|4| +5964|5|>10000|9|4| +5965|6|>10000|9|4| +5966|7|>10000|9|4| +5967|8|>10000|9|4| +5968|9|>10000|9|4| +5969|10|>10000|9|4| +5970|11|>10000|9|4| +5971|12|>10000|9|4| +5972|13|>10000|9|4| +5973|14|>10000|9|4| +5974|15|>10000|9|4| +5975|16|>10000|9|4| +5976|17|>10000|9|4| +5977|18|>10000|9|4| +5978|19|>10000|9|4| +5979|20|>10000|9|4| +5980|1|Unknown|9|4| +5981|2|Unknown|9|4| +5982|3|Unknown|9|4| +5983|4|Unknown|9|4| +5984|5|Unknown|9|4| +5985|6|Unknown|9|4| +5986|7|Unknown|9|4| +5987|8|Unknown|9|4| +5988|9|Unknown|9|4| +5989|10|Unknown|9|4| +5990|11|Unknown|9|4| +5991|12|Unknown|9|4| +5992|13|Unknown|9|4| +5993|14|Unknown|9|4| +5994|15|Unknown|9|4| +5995|16|Unknown|9|4| +5996|17|Unknown|9|4| +5997|18|Unknown|9|4| +5998|19|Unknown|9|4| +5999|20|Unknown|9|4| +6000|1|0-500|0|-1| +6001|2|0-500|0|-1| +6002|3|0-500|0|-1| +6003|4|0-500|0|-1| +6004|5|0-500|0|-1| +6005|6|0-500|0|-1| +6006|7|0-500|0|-1| +6007|8|0-500|0|-1| +6008|9|0-500|0|-1| +6009|10|0-500|0|-1| +6010|11|0-500|0|-1| +6011|12|0-500|0|-1| +6012|13|0-500|0|-1| +6013|14|0-500|0|-1| +6014|15|0-500|0|-1| +6015|16|0-500|0|-1| +6016|17|0-500|0|-1| +6017|18|0-500|0|-1| +6018|19|0-500|0|-1| +6019|20|0-500|0|-1| +6020|1|501-1000|0|-1| +6021|2|501-1000|0|-1| +6022|3|501-1000|0|-1| +6023|4|501-1000|0|-1| +6024|5|501-1000|0|-1| +6025|6|501-1000|0|-1| +6026|7|501-1000|0|-1| +6027|8|501-1000|0|-1| +6028|9|501-1000|0|-1| +6029|10|501-1000|0|-1| +6030|11|501-1000|0|-1| +6031|12|501-1000|0|-1| +6032|13|501-1000|0|-1| +6033|14|501-1000|0|-1| +6034|15|501-1000|0|-1| +6035|16|501-1000|0|-1| +6036|17|501-1000|0|-1| +6037|18|501-1000|0|-1| +6038|19|501-1000|0|-1| +6039|20|501-1000|0|-1| +6040|1|1001-5000|0|-1| +6041|2|1001-5000|0|-1| +6042|3|1001-5000|0|-1| +6043|4|1001-5000|0|-1| +6044|5|1001-5000|0|-1| +6045|6|1001-5000|0|-1| +6046|7|1001-5000|0|-1| +6047|8|1001-5000|0|-1| +6048|9|1001-5000|0|-1| +6049|10|1001-5000|0|-1| +6050|11|1001-5000|0|-1| +6051|12|1001-5000|0|-1| +6052|13|1001-5000|0|-1| +6053|14|1001-5000|0|-1| +6054|15|1001-5000|0|-1| +6055|16|1001-5000|0|-1| +6056|17|1001-5000|0|-1| +6057|18|1001-5000|0|-1| +6058|19|1001-5000|0|-1| +6059|20|1001-5000|0|-1| +6060|1|5001-10000|0|-1| +6061|2|5001-10000|0|-1| +6062|3|5001-10000|0|-1| +6063|4|5001-10000|0|-1| +6064|5|5001-10000|0|-1| +6065|6|5001-10000|0|-1| +6066|7|5001-10000|0|-1| +6067|8|5001-10000|0|-1| +6068|9|5001-10000|0|-1| +6069|10|5001-10000|0|-1| +6070|11|5001-10000|0|-1| +6071|12|5001-10000|0|-1| +6072|13|5001-10000|0|-1| +6073|14|5001-10000|0|-1| +6074|15|5001-10000|0|-1| +6075|16|5001-10000|0|-1| +6076|17|5001-10000|0|-1| +6077|18|5001-10000|0|-1| +6078|19|5001-10000|0|-1| +6079|20|5001-10000|0|-1| +6080|1|>10000|0|-1| +6081|2|>10000|0|-1| +6082|3|>10000|0|-1| +6083|4|>10000|0|-1| +6084|5|>10000|0|-1| +6085|6|>10000|0|-1| +6086|7|>10000|0|-1| +6087|8|>10000|0|-1| +6088|9|>10000|0|-1| +6089|10|>10000|0|-1| +6090|11|>10000|0|-1| +6091|12|>10000|0|-1| +6092|13|>10000|0|-1| +6093|14|>10000|0|-1| +6094|15|>10000|0|-1| +6095|16|>10000|0|-1| +6096|17|>10000|0|-1| +6097|18|>10000|0|-1| +6098|19|>10000|0|-1| +6099|20|>10000|0|-1| +6100|1|Unknown|0|-1| +6101|2|Unknown|0|-1| +6102|3|Unknown|0|-1| +6103|4|Unknown|0|-1| +6104|5|Unknown|0|-1| +6105|6|Unknown|0|-1| +6106|7|Unknown|0|-1| +6107|8|Unknown|0|-1| +6108|9|Unknown|0|-1| +6109|10|Unknown|0|-1| +6110|11|Unknown|0|-1| +6111|12|Unknown|0|-1| +6112|13|Unknown|0|-1| +6113|14|Unknown|0|-1| +6114|15|Unknown|0|-1| +6115|16|Unknown|0|-1| +6116|17|Unknown|0|-1| +6117|18|Unknown|0|-1| +6118|19|Unknown|0|-1| +6119|20|Unknown|0|-1| +6120|1|0-500|1|-1| +6121|2|0-500|1|-1| +6122|3|0-500|1|-1| +6123|4|0-500|1|-1| +6124|5|0-500|1|-1| +6125|6|0-500|1|-1| +6126|7|0-500|1|-1| +6127|8|0-500|1|-1| +6128|9|0-500|1|-1| +6129|10|0-500|1|-1| +6130|11|0-500|1|-1| +6131|12|0-500|1|-1| +6132|13|0-500|1|-1| +6133|14|0-500|1|-1| +6134|15|0-500|1|-1| +6135|16|0-500|1|-1| +6136|17|0-500|1|-1| +6137|18|0-500|1|-1| +6138|19|0-500|1|-1| +6139|20|0-500|1|-1| +6140|1|501-1000|1|-1| +6141|2|501-1000|1|-1| +6142|3|501-1000|1|-1| +6143|4|501-1000|1|-1| +6144|5|501-1000|1|-1| +6145|6|501-1000|1|-1| +6146|7|501-1000|1|-1| +6147|8|501-1000|1|-1| +6148|9|501-1000|1|-1| +6149|10|501-1000|1|-1| +6150|11|501-1000|1|-1| +6151|12|501-1000|1|-1| +6152|13|501-1000|1|-1| +6153|14|501-1000|1|-1| +6154|15|501-1000|1|-1| +6155|16|501-1000|1|-1| +6156|17|501-1000|1|-1| +6157|18|501-1000|1|-1| +6158|19|501-1000|1|-1| +6159|20|501-1000|1|-1| +6160|1|1001-5000|1|-1| +6161|2|1001-5000|1|-1| +6162|3|1001-5000|1|-1| +6163|4|1001-5000|1|-1| +6164|5|1001-5000|1|-1| +6165|6|1001-5000|1|-1| +6166|7|1001-5000|1|-1| +6167|8|1001-5000|1|-1| +6168|9|1001-5000|1|-1| +6169|10|1001-5000|1|-1| +6170|11|1001-5000|1|-1| +6171|12|1001-5000|1|-1| +6172|13|1001-5000|1|-1| +6173|14|1001-5000|1|-1| +6174|15|1001-5000|1|-1| +6175|16|1001-5000|1|-1| +6176|17|1001-5000|1|-1| +6177|18|1001-5000|1|-1| +6178|19|1001-5000|1|-1| +6179|20|1001-5000|1|-1| +6180|1|5001-10000|1|-1| +6181|2|5001-10000|1|-1| +6182|3|5001-10000|1|-1| +6183|4|5001-10000|1|-1| +6184|5|5001-10000|1|-1| +6185|6|5001-10000|1|-1| +6186|7|5001-10000|1|-1| +6187|8|5001-10000|1|-1| +6188|9|5001-10000|1|-1| +6189|10|5001-10000|1|-1| +6190|11|5001-10000|1|-1| +6191|12|5001-10000|1|-1| +6192|13|5001-10000|1|-1| +6193|14|5001-10000|1|-1| +6194|15|5001-10000|1|-1| +6195|16|5001-10000|1|-1| +6196|17|5001-10000|1|-1| +6197|18|5001-10000|1|-1| +6198|19|5001-10000|1|-1| +6199|20|5001-10000|1|-1| +6200|1|>10000|1|-1| +6201|2|>10000|1|-1| +6202|3|>10000|1|-1| +6203|4|>10000|1|-1| +6204|5|>10000|1|-1| +6205|6|>10000|1|-1| +6206|7|>10000|1|-1| +6207|8|>10000|1|-1| +6208|9|>10000|1|-1| +6209|10|>10000|1|-1| +6210|11|>10000|1|-1| +6211|12|>10000|1|-1| +6212|13|>10000|1|-1| +6213|14|>10000|1|-1| +6214|15|>10000|1|-1| +6215|16|>10000|1|-1| +6216|17|>10000|1|-1| +6217|18|>10000|1|-1| +6218|19|>10000|1|-1| +6219|20|>10000|1|-1| +6220|1|Unknown|1|-1| +6221|2|Unknown|1|-1| +6222|3|Unknown|1|-1| +6223|4|Unknown|1|-1| +6224|5|Unknown|1|-1| +6225|6|Unknown|1|-1| +6226|7|Unknown|1|-1| +6227|8|Unknown|1|-1| +6228|9|Unknown|1|-1| +6229|10|Unknown|1|-1| +6230|11|Unknown|1|-1| +6231|12|Unknown|1|-1| +6232|13|Unknown|1|-1| +6233|14|Unknown|1|-1| +6234|15|Unknown|1|-1| +6235|16|Unknown|1|-1| +6236|17|Unknown|1|-1| +6237|18|Unknown|1|-1| +6238|19|Unknown|1|-1| +6239|20|Unknown|1|-1| +6240|1|0-500|2|-1| +6241|2|0-500|2|-1| +6242|3|0-500|2|-1| +6243|4|0-500|2|-1| +6244|5|0-500|2|-1| +6245|6|0-500|2|-1| +6246|7|0-500|2|-1| +6247|8|0-500|2|-1| +6248|9|0-500|2|-1| +6249|10|0-500|2|-1| +6250|11|0-500|2|-1| +6251|12|0-500|2|-1| +6252|13|0-500|2|-1| +6253|14|0-500|2|-1| +6254|15|0-500|2|-1| +6255|16|0-500|2|-1| +6256|17|0-500|2|-1| +6257|18|0-500|2|-1| +6258|19|0-500|2|-1| +6259|20|0-500|2|-1| +6260|1|501-1000|2|-1| +6261|2|501-1000|2|-1| +6262|3|501-1000|2|-1| +6263|4|501-1000|2|-1| +6264|5|501-1000|2|-1| +6265|6|501-1000|2|-1| +6266|7|501-1000|2|-1| +6267|8|501-1000|2|-1| +6268|9|501-1000|2|-1| +6269|10|501-1000|2|-1| +6270|11|501-1000|2|-1| +6271|12|501-1000|2|-1| +6272|13|501-1000|2|-1| +6273|14|501-1000|2|-1| +6274|15|501-1000|2|-1| +6275|16|501-1000|2|-1| +6276|17|501-1000|2|-1| +6277|18|501-1000|2|-1| +6278|19|501-1000|2|-1| +6279|20|501-1000|2|-1| +6280|1|1001-5000|2|-1| +6281|2|1001-5000|2|-1| +6282|3|1001-5000|2|-1| +6283|4|1001-5000|2|-1| +6284|5|1001-5000|2|-1| +6285|6|1001-5000|2|-1| +6286|7|1001-5000|2|-1| +6287|8|1001-5000|2|-1| +6288|9|1001-5000|2|-1| +6289|10|1001-5000|2|-1| +6290|11|1001-5000|2|-1| +6291|12|1001-5000|2|-1| +6292|13|1001-5000|2|-1| +6293|14|1001-5000|2|-1| +6294|15|1001-5000|2|-1| +6295|16|1001-5000|2|-1| +6296|17|1001-5000|2|-1| +6297|18|1001-5000|2|-1| +6298|19|1001-5000|2|-1| +6299|20|1001-5000|2|-1| +6300|1|5001-10000|2|-1| +6301|2|5001-10000|2|-1| +6302|3|5001-10000|2|-1| +6303|4|5001-10000|2|-1| +6304|5|5001-10000|2|-1| +6305|6|5001-10000|2|-1| +6306|7|5001-10000|2|-1| +6307|8|5001-10000|2|-1| +6308|9|5001-10000|2|-1| +6309|10|5001-10000|2|-1| +6310|11|5001-10000|2|-1| +6311|12|5001-10000|2|-1| +6312|13|5001-10000|2|-1| +6313|14|5001-10000|2|-1| +6314|15|5001-10000|2|-1| +6315|16|5001-10000|2|-1| +6316|17|5001-10000|2|-1| +6317|18|5001-10000|2|-1| +6318|19|5001-10000|2|-1| +6319|20|5001-10000|2|-1| +6320|1|>10000|2|-1| +6321|2|>10000|2|-1| +6322|3|>10000|2|-1| +6323|4|>10000|2|-1| +6324|5|>10000|2|-1| +6325|6|>10000|2|-1| +6326|7|>10000|2|-1| +6327|8|>10000|2|-1| +6328|9|>10000|2|-1| +6329|10|>10000|2|-1| +6330|11|>10000|2|-1| +6331|12|>10000|2|-1| +6332|13|>10000|2|-1| +6333|14|>10000|2|-1| +6334|15|>10000|2|-1| +6335|16|>10000|2|-1| +6336|17|>10000|2|-1| +6337|18|>10000|2|-1| +6338|19|>10000|2|-1| +6339|20|>10000|2|-1| +6340|1|Unknown|2|-1| +6341|2|Unknown|2|-1| +6342|3|Unknown|2|-1| +6343|4|Unknown|2|-1| +6344|5|Unknown|2|-1| +6345|6|Unknown|2|-1| +6346|7|Unknown|2|-1| +6347|8|Unknown|2|-1| +6348|9|Unknown|2|-1| +6349|10|Unknown|2|-1| +6350|11|Unknown|2|-1| +6351|12|Unknown|2|-1| +6352|13|Unknown|2|-1| +6353|14|Unknown|2|-1| +6354|15|Unknown|2|-1| +6355|16|Unknown|2|-1| +6356|17|Unknown|2|-1| +6357|18|Unknown|2|-1| +6358|19|Unknown|2|-1| +6359|20|Unknown|2|-1| +6360|1|0-500|3|-1| +6361|2|0-500|3|-1| +6362|3|0-500|3|-1| +6363|4|0-500|3|-1| +6364|5|0-500|3|-1| +6365|6|0-500|3|-1| +6366|7|0-500|3|-1| +6367|8|0-500|3|-1| +6368|9|0-500|3|-1| +6369|10|0-500|3|-1| +6370|11|0-500|3|-1| +6371|12|0-500|3|-1| +6372|13|0-500|3|-1| +6373|14|0-500|3|-1| +6374|15|0-500|3|-1| +6375|16|0-500|3|-1| +6376|17|0-500|3|-1| +6377|18|0-500|3|-1| +6378|19|0-500|3|-1| +6379|20|0-500|3|-1| +6380|1|501-1000|3|-1| +6381|2|501-1000|3|-1| +6382|3|501-1000|3|-1| +6383|4|501-1000|3|-1| +6384|5|501-1000|3|-1| +6385|6|501-1000|3|-1| +6386|7|501-1000|3|-1| +6387|8|501-1000|3|-1| +6388|9|501-1000|3|-1| +6389|10|501-1000|3|-1| +6390|11|501-1000|3|-1| +6391|12|501-1000|3|-1| +6392|13|501-1000|3|-1| +6393|14|501-1000|3|-1| +6394|15|501-1000|3|-1| +6395|16|501-1000|3|-1| +6396|17|501-1000|3|-1| +6397|18|501-1000|3|-1| +6398|19|501-1000|3|-1| +6399|20|501-1000|3|-1| +6400|1|1001-5000|3|-1| +6401|2|1001-5000|3|-1| +6402|3|1001-5000|3|-1| +6403|4|1001-5000|3|-1| +6404|5|1001-5000|3|-1| +6405|6|1001-5000|3|-1| +6406|7|1001-5000|3|-1| +6407|8|1001-5000|3|-1| +6408|9|1001-5000|3|-1| +6409|10|1001-5000|3|-1| +6410|11|1001-5000|3|-1| +6411|12|1001-5000|3|-1| +6412|13|1001-5000|3|-1| +6413|14|1001-5000|3|-1| +6414|15|1001-5000|3|-1| +6415|16|1001-5000|3|-1| +6416|17|1001-5000|3|-1| +6417|18|1001-5000|3|-1| +6418|19|1001-5000|3|-1| +6419|20|1001-5000|3|-1| +6420|1|5001-10000|3|-1| +6421|2|5001-10000|3|-1| +6422|3|5001-10000|3|-1| +6423|4|5001-10000|3|-1| +6424|5|5001-10000|3|-1| +6425|6|5001-10000|3|-1| +6426|7|5001-10000|3|-1| +6427|8|5001-10000|3|-1| +6428|9|5001-10000|3|-1| +6429|10|5001-10000|3|-1| +6430|11|5001-10000|3|-1| +6431|12|5001-10000|3|-1| +6432|13|5001-10000|3|-1| +6433|14|5001-10000|3|-1| +6434|15|5001-10000|3|-1| +6435|16|5001-10000|3|-1| +6436|17|5001-10000|3|-1| +6437|18|5001-10000|3|-1| +6438|19|5001-10000|3|-1| +6439|20|5001-10000|3|-1| +6440|1|>10000|3|-1| +6441|2|>10000|3|-1| +6442|3|>10000|3|-1| +6443|4|>10000|3|-1| +6444|5|>10000|3|-1| +6445|6|>10000|3|-1| +6446|7|>10000|3|-1| +6447|8|>10000|3|-1| +6448|9|>10000|3|-1| +6449|10|>10000|3|-1| +6450|11|>10000|3|-1| +6451|12|>10000|3|-1| +6452|13|>10000|3|-1| +6453|14|>10000|3|-1| +6454|15|>10000|3|-1| +6455|16|>10000|3|-1| +6456|17|>10000|3|-1| +6457|18|>10000|3|-1| +6458|19|>10000|3|-1| +6459|20|>10000|3|-1| +6460|1|Unknown|3|-1| +6461|2|Unknown|3|-1| +6462|3|Unknown|3|-1| +6463|4|Unknown|3|-1| +6464|5|Unknown|3|-1| +6465|6|Unknown|3|-1| +6466|7|Unknown|3|-1| +6467|8|Unknown|3|-1| +6468|9|Unknown|3|-1| +6469|10|Unknown|3|-1| +6470|11|Unknown|3|-1| +6471|12|Unknown|3|-1| +6472|13|Unknown|3|-1| +6473|14|Unknown|3|-1| +6474|15|Unknown|3|-1| +6475|16|Unknown|3|-1| +6476|17|Unknown|3|-1| +6477|18|Unknown|3|-1| +6478|19|Unknown|3|-1| +6479|20|Unknown|3|-1| +6480|1|0-500|4|-1| +6481|2|0-500|4|-1| +6482|3|0-500|4|-1| +6483|4|0-500|4|-1| +6484|5|0-500|4|-1| +6485|6|0-500|4|-1| +6486|7|0-500|4|-1| +6487|8|0-500|4|-1| +6488|9|0-500|4|-1| +6489|10|0-500|4|-1| +6490|11|0-500|4|-1| +6491|12|0-500|4|-1| +6492|13|0-500|4|-1| +6493|14|0-500|4|-1| +6494|15|0-500|4|-1| +6495|16|0-500|4|-1| +6496|17|0-500|4|-1| +6497|18|0-500|4|-1| +6498|19|0-500|4|-1| +6499|20|0-500|4|-1| +6500|1|501-1000|4|-1| +6501|2|501-1000|4|-1| +6502|3|501-1000|4|-1| +6503|4|501-1000|4|-1| +6504|5|501-1000|4|-1| +6505|6|501-1000|4|-1| +6506|7|501-1000|4|-1| +6507|8|501-1000|4|-1| +6508|9|501-1000|4|-1| +6509|10|501-1000|4|-1| +6510|11|501-1000|4|-1| +6511|12|501-1000|4|-1| +6512|13|501-1000|4|-1| +6513|14|501-1000|4|-1| +6514|15|501-1000|4|-1| +6515|16|501-1000|4|-1| +6516|17|501-1000|4|-1| +6517|18|501-1000|4|-1| +6518|19|501-1000|4|-1| +6519|20|501-1000|4|-1| +6520|1|1001-5000|4|-1| +6521|2|1001-5000|4|-1| +6522|3|1001-5000|4|-1| +6523|4|1001-5000|4|-1| +6524|5|1001-5000|4|-1| +6525|6|1001-5000|4|-1| +6526|7|1001-5000|4|-1| +6527|8|1001-5000|4|-1| +6528|9|1001-5000|4|-1| +6529|10|1001-5000|4|-1| +6530|11|1001-5000|4|-1| +6531|12|1001-5000|4|-1| +6532|13|1001-5000|4|-1| +6533|14|1001-5000|4|-1| +6534|15|1001-5000|4|-1| +6535|16|1001-5000|4|-1| +6536|17|1001-5000|4|-1| +6537|18|1001-5000|4|-1| +6538|19|1001-5000|4|-1| +6539|20|1001-5000|4|-1| +6540|1|5001-10000|4|-1| +6541|2|5001-10000|4|-1| +6542|3|5001-10000|4|-1| +6543|4|5001-10000|4|-1| +6544|5|5001-10000|4|-1| +6545|6|5001-10000|4|-1| +6546|7|5001-10000|4|-1| +6547|8|5001-10000|4|-1| +6548|9|5001-10000|4|-1| +6549|10|5001-10000|4|-1| +6550|11|5001-10000|4|-1| +6551|12|5001-10000|4|-1| +6552|13|5001-10000|4|-1| +6553|14|5001-10000|4|-1| +6554|15|5001-10000|4|-1| +6555|16|5001-10000|4|-1| +6556|17|5001-10000|4|-1| +6557|18|5001-10000|4|-1| +6558|19|5001-10000|4|-1| +6559|20|5001-10000|4|-1| +6560|1|>10000|4|-1| +6561|2|>10000|4|-1| +6562|3|>10000|4|-1| +6563|4|>10000|4|-1| +6564|5|>10000|4|-1| +6565|6|>10000|4|-1| +6566|7|>10000|4|-1| +6567|8|>10000|4|-1| +6568|9|>10000|4|-1| +6569|10|>10000|4|-1| +6570|11|>10000|4|-1| +6571|12|>10000|4|-1| +6572|13|>10000|4|-1| +6573|14|>10000|4|-1| +6574|15|>10000|4|-1| +6575|16|>10000|4|-1| +6576|17|>10000|4|-1| +6577|18|>10000|4|-1| +6578|19|>10000|4|-1| +6579|20|>10000|4|-1| +6580|1|Unknown|4|-1| +6581|2|Unknown|4|-1| +6582|3|Unknown|4|-1| +6583|4|Unknown|4|-1| +6584|5|Unknown|4|-1| +6585|6|Unknown|4|-1| +6586|7|Unknown|4|-1| +6587|8|Unknown|4|-1| +6588|9|Unknown|4|-1| +6589|10|Unknown|4|-1| +6590|11|Unknown|4|-1| +6591|12|Unknown|4|-1| +6592|13|Unknown|4|-1| +6593|14|Unknown|4|-1| +6594|15|Unknown|4|-1| +6595|16|Unknown|4|-1| +6596|17|Unknown|4|-1| +6597|18|Unknown|4|-1| +6598|19|Unknown|4|-1| +6599|20|Unknown|4|-1| +6600|1|0-500|5|-1| +6601|2|0-500|5|-1| +6602|3|0-500|5|-1| +6603|4|0-500|5|-1| +6604|5|0-500|5|-1| +6605|6|0-500|5|-1| +6606|7|0-500|5|-1| +6607|8|0-500|5|-1| +6608|9|0-500|5|-1| +6609|10|0-500|5|-1| +6610|11|0-500|5|-1| +6611|12|0-500|5|-1| +6612|13|0-500|5|-1| +6613|14|0-500|5|-1| +6614|15|0-500|5|-1| +6615|16|0-500|5|-1| +6616|17|0-500|5|-1| +6617|18|0-500|5|-1| +6618|19|0-500|5|-1| +6619|20|0-500|5|-1| +6620|1|501-1000|5|-1| +6621|2|501-1000|5|-1| +6622|3|501-1000|5|-1| +6623|4|501-1000|5|-1| +6624|5|501-1000|5|-1| +6625|6|501-1000|5|-1| +6626|7|501-1000|5|-1| +6627|8|501-1000|5|-1| +6628|9|501-1000|5|-1| +6629|10|501-1000|5|-1| +6630|11|501-1000|5|-1| +6631|12|501-1000|5|-1| +6632|13|501-1000|5|-1| +6633|14|501-1000|5|-1| +6634|15|501-1000|5|-1| +6635|16|501-1000|5|-1| +6636|17|501-1000|5|-1| +6637|18|501-1000|5|-1| +6638|19|501-1000|5|-1| +6639|20|501-1000|5|-1| +6640|1|1001-5000|5|-1| +6641|2|1001-5000|5|-1| +6642|3|1001-5000|5|-1| +6643|4|1001-5000|5|-1| +6644|5|1001-5000|5|-1| +6645|6|1001-5000|5|-1| +6646|7|1001-5000|5|-1| +6647|8|1001-5000|5|-1| +6648|9|1001-5000|5|-1| +6649|10|1001-5000|5|-1| +6650|11|1001-5000|5|-1| +6651|12|1001-5000|5|-1| +6652|13|1001-5000|5|-1| +6653|14|1001-5000|5|-1| +6654|15|1001-5000|5|-1| +6655|16|1001-5000|5|-1| +6656|17|1001-5000|5|-1| +6657|18|1001-5000|5|-1| +6658|19|1001-5000|5|-1| +6659|20|1001-5000|5|-1| +6660|1|5001-10000|5|-1| +6661|2|5001-10000|5|-1| +6662|3|5001-10000|5|-1| +6663|4|5001-10000|5|-1| +6664|5|5001-10000|5|-1| +6665|6|5001-10000|5|-1| +6666|7|5001-10000|5|-1| +6667|8|5001-10000|5|-1| +6668|9|5001-10000|5|-1| +6669|10|5001-10000|5|-1| +6670|11|5001-10000|5|-1| +6671|12|5001-10000|5|-1| +6672|13|5001-10000|5|-1| +6673|14|5001-10000|5|-1| +6674|15|5001-10000|5|-1| +6675|16|5001-10000|5|-1| +6676|17|5001-10000|5|-1| +6677|18|5001-10000|5|-1| +6678|19|5001-10000|5|-1| +6679|20|5001-10000|5|-1| +6680|1|>10000|5|-1| +6681|2|>10000|5|-1| +6682|3|>10000|5|-1| +6683|4|>10000|5|-1| +6684|5|>10000|5|-1| +6685|6|>10000|5|-1| +6686|7|>10000|5|-1| +6687|8|>10000|5|-1| +6688|9|>10000|5|-1| +6689|10|>10000|5|-1| +6690|11|>10000|5|-1| +6691|12|>10000|5|-1| +6692|13|>10000|5|-1| +6693|14|>10000|5|-1| +6694|15|>10000|5|-1| +6695|16|>10000|5|-1| +6696|17|>10000|5|-1| +6697|18|>10000|5|-1| +6698|19|>10000|5|-1| +6699|20|>10000|5|-1| +6700|1|Unknown|5|-1| +6701|2|Unknown|5|-1| +6702|3|Unknown|5|-1| +6703|4|Unknown|5|-1| +6704|5|Unknown|5|-1| +6705|6|Unknown|5|-1| +6706|7|Unknown|5|-1| +6707|8|Unknown|5|-1| +6708|9|Unknown|5|-1| +6709|10|Unknown|5|-1| +6710|11|Unknown|5|-1| +6711|12|Unknown|5|-1| +6712|13|Unknown|5|-1| +6713|14|Unknown|5|-1| +6714|15|Unknown|5|-1| +6715|16|Unknown|5|-1| +6716|17|Unknown|5|-1| +6717|18|Unknown|5|-1| +6718|19|Unknown|5|-1| +6719|20|Unknown|5|-1| +6720|1|0-500|6|-1| +6721|2|0-500|6|-1| +6722|3|0-500|6|-1| +6723|4|0-500|6|-1| +6724|5|0-500|6|-1| +6725|6|0-500|6|-1| +6726|7|0-500|6|-1| +6727|8|0-500|6|-1| +6728|9|0-500|6|-1| +6729|10|0-500|6|-1| +6730|11|0-500|6|-1| +6731|12|0-500|6|-1| +6732|13|0-500|6|-1| +6733|14|0-500|6|-1| +6734|15|0-500|6|-1| +6735|16|0-500|6|-1| +6736|17|0-500|6|-1| +6737|18|0-500|6|-1| +6738|19|0-500|6|-1| +6739|20|0-500|6|-1| +6740|1|501-1000|6|-1| +6741|2|501-1000|6|-1| +6742|3|501-1000|6|-1| +6743|4|501-1000|6|-1| +6744|5|501-1000|6|-1| +6745|6|501-1000|6|-1| +6746|7|501-1000|6|-1| +6747|8|501-1000|6|-1| +6748|9|501-1000|6|-1| +6749|10|501-1000|6|-1| +6750|11|501-1000|6|-1| +6751|12|501-1000|6|-1| +6752|13|501-1000|6|-1| +6753|14|501-1000|6|-1| +6754|15|501-1000|6|-1| +6755|16|501-1000|6|-1| +6756|17|501-1000|6|-1| +6757|18|501-1000|6|-1| +6758|19|501-1000|6|-1| +6759|20|501-1000|6|-1| +6760|1|1001-5000|6|-1| +6761|2|1001-5000|6|-1| +6762|3|1001-5000|6|-1| +6763|4|1001-5000|6|-1| +6764|5|1001-5000|6|-1| +6765|6|1001-5000|6|-1| +6766|7|1001-5000|6|-1| +6767|8|1001-5000|6|-1| +6768|9|1001-5000|6|-1| +6769|10|1001-5000|6|-1| +6770|11|1001-5000|6|-1| +6771|12|1001-5000|6|-1| +6772|13|1001-5000|6|-1| +6773|14|1001-5000|6|-1| +6774|15|1001-5000|6|-1| +6775|16|1001-5000|6|-1| +6776|17|1001-5000|6|-1| +6777|18|1001-5000|6|-1| +6778|19|1001-5000|6|-1| +6779|20|1001-5000|6|-1| +6780|1|5001-10000|6|-1| +6781|2|5001-10000|6|-1| +6782|3|5001-10000|6|-1| +6783|4|5001-10000|6|-1| +6784|5|5001-10000|6|-1| +6785|6|5001-10000|6|-1| +6786|7|5001-10000|6|-1| +6787|8|5001-10000|6|-1| +6788|9|5001-10000|6|-1| +6789|10|5001-10000|6|-1| +6790|11|5001-10000|6|-1| +6791|12|5001-10000|6|-1| +6792|13|5001-10000|6|-1| +6793|14|5001-10000|6|-1| +6794|15|5001-10000|6|-1| +6795|16|5001-10000|6|-1| +6796|17|5001-10000|6|-1| +6797|18|5001-10000|6|-1| +6798|19|5001-10000|6|-1| +6799|20|5001-10000|6|-1| +6800|1|>10000|6|-1| +6801|2|>10000|6|-1| +6802|3|>10000|6|-1| +6803|4|>10000|6|-1| +6804|5|>10000|6|-1| +6805|6|>10000|6|-1| +6806|7|>10000|6|-1| +6807|8|>10000|6|-1| +6808|9|>10000|6|-1| +6809|10|>10000|6|-1| +6810|11|>10000|6|-1| +6811|12|>10000|6|-1| +6812|13|>10000|6|-1| +6813|14|>10000|6|-1| +6814|15|>10000|6|-1| +6815|16|>10000|6|-1| +6816|17|>10000|6|-1| +6817|18|>10000|6|-1| +6818|19|>10000|6|-1| +6819|20|>10000|6|-1| +6820|1|Unknown|6|-1| +6821|2|Unknown|6|-1| +6822|3|Unknown|6|-1| +6823|4|Unknown|6|-1| +6824|5|Unknown|6|-1| +6825|6|Unknown|6|-1| +6826|7|Unknown|6|-1| +6827|8|Unknown|6|-1| +6828|9|Unknown|6|-1| +6829|10|Unknown|6|-1| +6830|11|Unknown|6|-1| +6831|12|Unknown|6|-1| +6832|13|Unknown|6|-1| +6833|14|Unknown|6|-1| +6834|15|Unknown|6|-1| +6835|16|Unknown|6|-1| +6836|17|Unknown|6|-1| +6837|18|Unknown|6|-1| +6838|19|Unknown|6|-1| +6839|20|Unknown|6|-1| +6840|1|0-500|7|-1| +6841|2|0-500|7|-1| +6842|3|0-500|7|-1| +6843|4|0-500|7|-1| +6844|5|0-500|7|-1| +6845|6|0-500|7|-1| +6846|7|0-500|7|-1| +6847|8|0-500|7|-1| +6848|9|0-500|7|-1| +6849|10|0-500|7|-1| +6850|11|0-500|7|-1| +6851|12|0-500|7|-1| +6852|13|0-500|7|-1| +6853|14|0-500|7|-1| +6854|15|0-500|7|-1| +6855|16|0-500|7|-1| +6856|17|0-500|7|-1| +6857|18|0-500|7|-1| +6858|19|0-500|7|-1| +6859|20|0-500|7|-1| +6860|1|501-1000|7|-1| +6861|2|501-1000|7|-1| +6862|3|501-1000|7|-1| +6863|4|501-1000|7|-1| +6864|5|501-1000|7|-1| +6865|6|501-1000|7|-1| +6866|7|501-1000|7|-1| +6867|8|501-1000|7|-1| +6868|9|501-1000|7|-1| +6869|10|501-1000|7|-1| +6870|11|501-1000|7|-1| +6871|12|501-1000|7|-1| +6872|13|501-1000|7|-1| +6873|14|501-1000|7|-1| +6874|15|501-1000|7|-1| +6875|16|501-1000|7|-1| +6876|17|501-1000|7|-1| +6877|18|501-1000|7|-1| +6878|19|501-1000|7|-1| +6879|20|501-1000|7|-1| +6880|1|1001-5000|7|-1| +6881|2|1001-5000|7|-1| +6882|3|1001-5000|7|-1| +6883|4|1001-5000|7|-1| +6884|5|1001-5000|7|-1| +6885|6|1001-5000|7|-1| +6886|7|1001-5000|7|-1| +6887|8|1001-5000|7|-1| +6888|9|1001-5000|7|-1| +6889|10|1001-5000|7|-1| +6890|11|1001-5000|7|-1| +6891|12|1001-5000|7|-1| +6892|13|1001-5000|7|-1| +6893|14|1001-5000|7|-1| +6894|15|1001-5000|7|-1| +6895|16|1001-5000|7|-1| +6896|17|1001-5000|7|-1| +6897|18|1001-5000|7|-1| +6898|19|1001-5000|7|-1| +6899|20|1001-5000|7|-1| +6900|1|5001-10000|7|-1| +6901|2|5001-10000|7|-1| +6902|3|5001-10000|7|-1| +6903|4|5001-10000|7|-1| +6904|5|5001-10000|7|-1| +6905|6|5001-10000|7|-1| +6906|7|5001-10000|7|-1| +6907|8|5001-10000|7|-1| +6908|9|5001-10000|7|-1| +6909|10|5001-10000|7|-1| +6910|11|5001-10000|7|-1| +6911|12|5001-10000|7|-1| +6912|13|5001-10000|7|-1| +6913|14|5001-10000|7|-1| +6914|15|5001-10000|7|-1| +6915|16|5001-10000|7|-1| +6916|17|5001-10000|7|-1| +6917|18|5001-10000|7|-1| +6918|19|5001-10000|7|-1| +6919|20|5001-10000|7|-1| +6920|1|>10000|7|-1| +6921|2|>10000|7|-1| +6922|3|>10000|7|-1| +6923|4|>10000|7|-1| +6924|5|>10000|7|-1| +6925|6|>10000|7|-1| +6926|7|>10000|7|-1| +6927|8|>10000|7|-1| +6928|9|>10000|7|-1| +6929|10|>10000|7|-1| +6930|11|>10000|7|-1| +6931|12|>10000|7|-1| +6932|13|>10000|7|-1| +6933|14|>10000|7|-1| +6934|15|>10000|7|-1| +6935|16|>10000|7|-1| +6936|17|>10000|7|-1| +6937|18|>10000|7|-1| +6938|19|>10000|7|-1| +6939|20|>10000|7|-1| +6940|1|Unknown|7|-1| +6941|2|Unknown|7|-1| +6942|3|Unknown|7|-1| +6943|4|Unknown|7|-1| +6944|5|Unknown|7|-1| +6945|6|Unknown|7|-1| +6946|7|Unknown|7|-1| +6947|8|Unknown|7|-1| +6948|9|Unknown|7|-1| +6949|10|Unknown|7|-1| +6950|11|Unknown|7|-1| +6951|12|Unknown|7|-1| +6952|13|Unknown|7|-1| +6953|14|Unknown|7|-1| +6954|15|Unknown|7|-1| +6955|16|Unknown|7|-1| +6956|17|Unknown|7|-1| +6957|18|Unknown|7|-1| +6958|19|Unknown|7|-1| +6959|20|Unknown|7|-1| +6960|1|0-500|8|-1| +6961|2|0-500|8|-1| +6962|3|0-500|8|-1| +6963|4|0-500|8|-1| +6964|5|0-500|8|-1| +6965|6|0-500|8|-1| +6966|7|0-500|8|-1| +6967|8|0-500|8|-1| +6968|9|0-500|8|-1| +6969|10|0-500|8|-1| +6970|11|0-500|8|-1| +6971|12|0-500|8|-1| +6972|13|0-500|8|-1| +6973|14|0-500|8|-1| +6974|15|0-500|8|-1| +6975|16|0-500|8|-1| +6976|17|0-500|8|-1| +6977|18|0-500|8|-1| +6978|19|0-500|8|-1| +6979|20|0-500|8|-1| +6980|1|501-1000|8|-1| +6981|2|501-1000|8|-1| +6982|3|501-1000|8|-1| +6983|4|501-1000|8|-1| +6984|5|501-1000|8|-1| +6985|6|501-1000|8|-1| +6986|7|501-1000|8|-1| +6987|8|501-1000|8|-1| +6988|9|501-1000|8|-1| +6989|10|501-1000|8|-1| +6990|11|501-1000|8|-1| +6991|12|501-1000|8|-1| +6992|13|501-1000|8|-1| +6993|14|501-1000|8|-1| +6994|15|501-1000|8|-1| +6995|16|501-1000|8|-1| +6996|17|501-1000|8|-1| +6997|18|501-1000|8|-1| +6998|19|501-1000|8|-1| +6999|20|501-1000|8|-1| +7000|1|1001-5000|8|-1| +7001|2|1001-5000|8|-1| +7002|3|1001-5000|8|-1| +7003|4|1001-5000|8|-1| +7004|5|1001-5000|8|-1| +7005|6|1001-5000|8|-1| +7006|7|1001-5000|8|-1| +7007|8|1001-5000|8|-1| +7008|9|1001-5000|8|-1| +7009|10|1001-5000|8|-1| +7010|11|1001-5000|8|-1| +7011|12|1001-5000|8|-1| +7012|13|1001-5000|8|-1| +7013|14|1001-5000|8|-1| +7014|15|1001-5000|8|-1| +7015|16|1001-5000|8|-1| +7016|17|1001-5000|8|-1| +7017|18|1001-5000|8|-1| +7018|19|1001-5000|8|-1| +7019|20|1001-5000|8|-1| +7020|1|5001-10000|8|-1| +7021|2|5001-10000|8|-1| +7022|3|5001-10000|8|-1| +7023|4|5001-10000|8|-1| +7024|5|5001-10000|8|-1| +7025|6|5001-10000|8|-1| +7026|7|5001-10000|8|-1| +7027|8|5001-10000|8|-1| +7028|9|5001-10000|8|-1| +7029|10|5001-10000|8|-1| +7030|11|5001-10000|8|-1| +7031|12|5001-10000|8|-1| +7032|13|5001-10000|8|-1| +7033|14|5001-10000|8|-1| +7034|15|5001-10000|8|-1| +7035|16|5001-10000|8|-1| +7036|17|5001-10000|8|-1| +7037|18|5001-10000|8|-1| +7038|19|5001-10000|8|-1| +7039|20|5001-10000|8|-1| +7040|1|>10000|8|-1| +7041|2|>10000|8|-1| +7042|3|>10000|8|-1| +7043|4|>10000|8|-1| +7044|5|>10000|8|-1| +7045|6|>10000|8|-1| +7046|7|>10000|8|-1| +7047|8|>10000|8|-1| +7048|9|>10000|8|-1| +7049|10|>10000|8|-1| +7050|11|>10000|8|-1| +7051|12|>10000|8|-1| +7052|13|>10000|8|-1| +7053|14|>10000|8|-1| +7054|15|>10000|8|-1| +7055|16|>10000|8|-1| +7056|17|>10000|8|-1| +7057|18|>10000|8|-1| +7058|19|>10000|8|-1| +7059|20|>10000|8|-1| +7060|1|Unknown|8|-1| +7061|2|Unknown|8|-1| +7062|3|Unknown|8|-1| +7063|4|Unknown|8|-1| +7064|5|Unknown|8|-1| +7065|6|Unknown|8|-1| +7066|7|Unknown|8|-1| +7067|8|Unknown|8|-1| +7068|9|Unknown|8|-1| +7069|10|Unknown|8|-1| +7070|11|Unknown|8|-1| +7071|12|Unknown|8|-1| +7072|13|Unknown|8|-1| +7073|14|Unknown|8|-1| +7074|15|Unknown|8|-1| +7075|16|Unknown|8|-1| +7076|17|Unknown|8|-1| +7077|18|Unknown|8|-1| +7078|19|Unknown|8|-1| +7079|20|Unknown|8|-1| +7080|1|0-500|9|-1| +7081|2|0-500|9|-1| +7082|3|0-500|9|-1| +7083|4|0-500|9|-1| +7084|5|0-500|9|-1| +7085|6|0-500|9|-1| +7086|7|0-500|9|-1| +7087|8|0-500|9|-1| +7088|9|0-500|9|-1| +7089|10|0-500|9|-1| +7090|11|0-500|9|-1| +7091|12|0-500|9|-1| +7092|13|0-500|9|-1| +7093|14|0-500|9|-1| +7094|15|0-500|9|-1| +7095|16|0-500|9|-1| +7096|17|0-500|9|-1| +7097|18|0-500|9|-1| +7098|19|0-500|9|-1| +7099|20|0-500|9|-1| +7100|1|501-1000|9|-1| +7101|2|501-1000|9|-1| +7102|3|501-1000|9|-1| +7103|4|501-1000|9|-1| +7104|5|501-1000|9|-1| +7105|6|501-1000|9|-1| +7106|7|501-1000|9|-1| +7107|8|501-1000|9|-1| +7108|9|501-1000|9|-1| +7109|10|501-1000|9|-1| +7110|11|501-1000|9|-1| +7111|12|501-1000|9|-1| +7112|13|501-1000|9|-1| +7113|14|501-1000|9|-1| +7114|15|501-1000|9|-1| +7115|16|501-1000|9|-1| +7116|17|501-1000|9|-1| +7117|18|501-1000|9|-1| +7118|19|501-1000|9|-1| +7119|20|501-1000|9|-1| +7120|1|1001-5000|9|-1| +7121|2|1001-5000|9|-1| +7122|3|1001-5000|9|-1| +7123|4|1001-5000|9|-1| +7124|5|1001-5000|9|-1| +7125|6|1001-5000|9|-1| +7126|7|1001-5000|9|-1| +7127|8|1001-5000|9|-1| +7128|9|1001-5000|9|-1| +7129|10|1001-5000|9|-1| +7130|11|1001-5000|9|-1| +7131|12|1001-5000|9|-1| +7132|13|1001-5000|9|-1| +7133|14|1001-5000|9|-1| +7134|15|1001-5000|9|-1| +7135|16|1001-5000|9|-1| +7136|17|1001-5000|9|-1| +7137|18|1001-5000|9|-1| +7138|19|1001-5000|9|-1| +7139|20|1001-5000|9|-1| +7140|1|5001-10000|9|-1| +7141|2|5001-10000|9|-1| +7142|3|5001-10000|9|-1| +7143|4|5001-10000|9|-1| +7144|5|5001-10000|9|-1| +7145|6|5001-10000|9|-1| +7146|7|5001-10000|9|-1| +7147|8|5001-10000|9|-1| +7148|9|5001-10000|9|-1| +7149|10|5001-10000|9|-1| +7150|11|5001-10000|9|-1| +7151|12|5001-10000|9|-1| +7152|13|5001-10000|9|-1| +7153|14|5001-10000|9|-1| +7154|15|5001-10000|9|-1| +7155|16|5001-10000|9|-1| +7156|17|5001-10000|9|-1| +7157|18|5001-10000|9|-1| +7158|19|5001-10000|9|-1| +7159|20|5001-10000|9|-1| +7160|1|>10000|9|-1| +7161|2|>10000|9|-1| +7162|3|>10000|9|-1| +7163|4|>10000|9|-1| +7164|5|>10000|9|-1| +7165|6|>10000|9|-1| +7166|7|>10000|9|-1| +7167|8|>10000|9|-1| +7168|9|>10000|9|-1| +7169|10|>10000|9|-1| +7170|11|>10000|9|-1| +7171|12|>10000|9|-1| +7172|13|>10000|9|-1| +7173|14|>10000|9|-1| +7174|15|>10000|9|-1| +7175|16|>10000|9|-1| +7176|17|>10000|9|-1| +7177|18|>10000|9|-1| +7178|19|>10000|9|-1| +7179|20|>10000|9|-1| +7180|1|Unknown|9|-1| +7181|2|Unknown|9|-1| +7182|3|Unknown|9|-1| +7183|4|Unknown|9|-1| +7184|5|Unknown|9|-1| +7185|6|Unknown|9|-1| +7186|7|Unknown|9|-1| +7187|8|Unknown|9|-1| +7188|9|Unknown|9|-1| +7189|10|Unknown|9|-1| +7190|11|Unknown|9|-1| +7191|12|Unknown|9|-1| +7192|13|Unknown|9|-1| +7193|14|Unknown|9|-1| +7194|15|Unknown|9|-1| +7195|16|Unknown|9|-1| +7196|17|Unknown|9|-1| +7197|18|Unknown|9|-1| +7198|19|Unknown|9|-1| +7199|20|Unknown|9|-1| +7200|1|0-500|0|0| diff --git a/spark-connector/spark-common/src/test/resources/data/tpcds/income_band/income_band.dat b/spark-connector/spark-common/src/test/resources/data/tpcds/income_band/income_band.dat new file mode 100644 index 00000000000..18c2f70d265 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/data/tpcds/income_band/income_band.dat @@ -0,0 +1,20 @@ +1|0|10000| +2|10001|20000| +3|20001|30000| +4|30001|40000| +5|40001|50000| +6|50001|60000| +7|60001|70000| +8|70001|80000| +9|80001|90000| +10|90001|100000| +11|100001|110000| +12|110001|120000| +13|120001|130000| +14|130001|140000| +15|140001|150000| +16|150001|160000| +17|160001|170000| +18|170001|180000| +19|180001|190000| +20|190001|200000| diff --git a/spark-connector/spark-common/src/test/resources/data/tpcds/inventory/inventory.dat b/spark-connector/spark-common/src/test/resources/data/tpcds/inventory/inventory.dat new file mode 100644 index 00000000000..b5381e8d072 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/data/tpcds/inventory/inventory.dat @@ -0,0 +1,117450 @@ +2450815|1|1|211| +2450815|2|1|235| +2450815|4|1|859| +2450815|7|1|704| +2450815|8|1|891| +2450815|10|1|834| +2450815|13|1|591| +2450815|14|1|579| +2450815|16|1|622| +2450815|19|1|745| +2450815|20|1|405| +2450815|22|1|| +2450815|25|1|676| +2450815|26|1|865| +2450815|28|1|24| +2450815|31|1|170| +2450815|32|1|797| +2450815|34|1|332| +2450815|37|1|618| +2450815|38|1|909| +2450815|40|1|738| +2450815|43|1|699| +2450815|44|1|975| +2450815|46|1|551| +2450815|49|1|857| +2450815|50|1|894| +2450815|52|1|279| +2450815|55|1|206| +2450815|56|1|746| +2450815|58|1|| +2450815|61|1|154| +2450815|62|1|888| +2450815|64|1|773| +2450815|67|1|107| +2450815|68|1|801| +2450815|70|1|749| +2450815|73|1|419| +2450815|74|1|541| +2450815|76|1|359| +2450815|79|1|578| +2450815|80|1|42| +2450815|82|1|83| +2450815|85|1|192| +2450815|86|1|567| +2450815|88|1|710| +2450815|91|1|282| +2450815|92|1|631| +2450815|94|1|587| +2450815|97|1|645| +2450815|98|1|667| +2450815|100|1|26| +2450815|103|1|224| +2450815|104|1|901| +2450815|106|1|887| +2450815|109|1|455| +2450815|110|1|784| +2450815|112|1|742| +2450815|115|1|978| +2450815|116|1|207| +2450815|118|1|462| +2450815|121|1|406| +2450815|122|1|559| +2450815|124|1|999| +2450815|127|1|870| +2450815|128|1|982| +2450815|130|1|700| +2450815|133|1|26| +2450815|134|1|356| +2450815|136|1|143| +2450815|139|1|838| +2450815|140|1|846| +2450815|142|1|657| +2450815|145|1|181| +2450815|146|1|730| +2450815|148|1|49| +2450815|151|1|652| +2450815|152|1|861| +2450815|154|1|329| +2450815|157|1|286| +2450815|158|1|| +2450815|160|1|62| +2450815|163|1|894| +2450815|164|1|463| +2450815|166|1|458| +2450815|169|1|696| +2450815|170|1|450| +2450815|172|1|842| +2450815|175|1|79| +2450815|176|1|260| +2450815|178|1|894| +2450815|181|1|4| +2450815|182|1|281| +2450815|184|1|797| +2450815|187|1|981| +2450815|188|1|58| +2450815|190|1|834| +2450815|193|1|54| +2450815|194|1|856| +2450815|196|1|803| +2450815|199|1|296| +2450815|200|1|163| +2450815|202|1|154| +2450815|205|1|392| +2450815|206|1|469| +2450815|208|1|472| +2450815|211|1|324| +2450815|212|1|307| +2450815|214|1|94| +2450815|217|1|954| +2450815|218|1|435| +2450815|220|1|903| +2450815|223|1|777| +2450815|224|1|300| +2450815|226|1|740| +2450815|229|1|593| +2450815|230|1|589| +2450815|232|1|424| +2450815|235|1|600| +2450815|236|1|847| +2450815|238|1|847| +2450815|241|1|696| +2450815|242|1|96| +2450815|244|1|895| +2450815|247|1|899| +2450815|248|1|902| +2450815|250|1|734| +2450815|253|1|164| +2450815|254|1|335| +2450815|256|1|993| +2450815|259|1|531| +2450815|260|1|456| +2450815|262|1|738| +2450815|265|1|529| +2450815|266|1|292| +2450815|268|1|495| +2450815|271|1|360| +2450815|272|1|677| +2450815|274|1|908| +2450815|277|1|294| +2450815|278|1|734| +2450815|280|1|953| +2450815|283|1|887| +2450815|284|1|966| +2450815|286|1|720| +2450815|289|1|212| +2450815|290|1|793| +2450815|292|1|491| +2450815|295|1|62| +2450815|296|1|266| +2450815|298|1|206| +2450815|301|1|679| +2450815|302|1|116| +2450815|304|1|702| +2450815|307|1|922| +2450815|308|1|982| +2450815|310|1|783| +2450815|313|1|614| +2450815|314|1|855| +2450815|316|1|487| +2450815|319|1|829| +2450815|320|1|| +2450815|322|1|926| +2450815|325|1|349| +2450815|326|1|663| +2450815|328|1|95| +2450815|331|1|476| +2450815|332|1|956| +2450815|334|1|55| +2450815|337|1|564| +2450815|338|1|650| +2450815|340|1|688| +2450815|343|1|827| +2450815|344|1|716| +2450815|346|1|85| +2450815|349|1|559| +2450815|350|1|437| +2450815|352|1|215| +2450815|355|1|429| +2450815|356|1|614| +2450815|358|1|238| +2450815|361|1|331| +2450815|362|1|201| +2450815|364|1|544| +2450815|367|1|624| +2450815|368|1|100| +2450815|370|1|666| +2450815|373|1|732| +2450815|374|1|143| +2450815|376|1|171| +2450815|379|1|122| +2450815|380|1|859| +2450815|382|1|438| +2450815|385|1|768| +2450815|386|1|696| +2450815|388|1|711| +2450815|391|1|920| +2450815|392|1|405| +2450815|394|1|462| +2450815|397|1|699| +2450815|398|1|342| +2450815|400|1|487| +2450815|403|1|| +2450815|404|1|447| +2450815|406|1|18| +2450815|409|1|672| +2450815|410|1|56| +2450815|412|1|642| +2450815|415|1|196| +2450815|416|1|375| +2450815|418|1|950| +2450815|421|1|837| +2450815|422|1|456| +2450815|424|1|256| +2450815|427|1|283| +2450815|428|1|813| +2450815|430|1|431| +2450815|433|1|286| +2450815|434|1|858| +2450815|436|1|740| +2450815|439|1|7| +2450815|440|1|952| +2450815|442|1|309| +2450815|445|1|209| +2450815|446|1|269| +2450815|448|1|638| +2450815|451|1|448| +2450815|452|1|890| +2450815|454|1|209| +2450815|457|1|950| +2450815|458|1|72| +2450815|460|1|235| +2450815|463|1|289| +2450815|464|1|492| +2450815|466|1|311| +2450815|469|1|943| +2450815|470|1|995| +2450815|472|1|314| +2450815|475|1|601| +2450815|476|1|228| +2450815|478|1|170| +2450815|481|1|988| +2450815|482|1|728| +2450815|484|1|852| +2450815|487|1|832| +2450815|488|1|131| +2450815|490|1|894| +2450815|493|1|682| +2450815|494|1|634| +2450815|496|1|538| +2450815|499|1|460| +2450815|500|1|| +2450815|502|1|94| +2450815|505|1|151| +2450815|506|1|360| +2450815|508|1|| +2450815|511|1|767| +2450815|512|1|236| +2450815|514|1|142| +2450815|517|1|273| +2450815|518|1|| +2450815|520|1|57| +2450815|523|1|743| +2450815|524|1|273| +2450815|526|1|73| +2450815|529|1|514| +2450815|530|1|149| +2450815|532|1|924| +2450815|535|1|282| +2450815|536|1|993| +2450815|538|1|734| +2450815|541|1|876| +2450815|542|1|237| +2450815|544|1|949| +2450815|547|1|23| +2450815|548|1|443| +2450815|550|1|749| +2450815|553|1|729| +2450815|554|1|865| +2450815|556|1|630| +2450815|559|1|832| +2450815|560|1|933| +2450815|562|1|65| +2450815|565|1|10| +2450815|566|1|37| +2450815|568|1|616| +2450815|571|1|716| +2450815|572|1|263| +2450815|574|1|334| +2450815|577|1|874| +2450815|578|1|5| +2450815|580|1|719| +2450815|583|1|303| +2450815|584|1|319| +2450815|586|1|953| +2450815|589|1|69| +2450815|590|1|870| +2450815|592|1|157| +2450815|595|1|274| +2450815|596|1|770| +2450815|598|1|406| +2450815|601|1|564| +2450815|602|1|695| +2450815|604|1|60| +2450815|607|1|933| +2450815|608|1|637| +2450815|610|1|627| +2450815|613|1|483| +2450815|614|1|262| +2450815|616|1|716| +2450815|619|1|723| +2450815|620|1|604| +2450815|622|1|951| +2450815|625|1|602| +2450815|626|1|551| +2450815|628|1|323| +2450815|631|1|736| +2450815|632|1|579| +2450815|634|1|429| +2450815|637|1|833| +2450815|638|1|679| +2450815|640|1|631| +2450815|643|1|700| +2450815|644|1|423| +2450815|646|1|897| +2450815|649|1|818| +2450815|650|1|540| +2450815|652|1|62| +2450815|655|1|72| +2450815|656|1|40| +2450815|658|1|110| +2450815|661|1|772| +2450815|662|1|686| +2450815|664|1|300| +2450815|667|1|959| +2450815|668|1|| +2450815|670|1|646| +2450815|673|1|94| +2450815|674|1|329| +2450815|676|1|601| +2450815|679|1|370| +2450815|680|1|150| +2450815|682|1|144| +2450815|685|1|350| +2450815|686|1|295| +2450815|688|1|628| +2450815|691|1|| +2450815|692|1|22| +2450815|694|1|16| +2450815|697|1|707| +2450815|698|1|453| +2450815|700|1|112| +2450815|703|1|393| +2450815|704|1|798| +2450815|706|1|504| +2450815|709|1|148| +2450815|710|1|885| +2450815|712|1|828| +2450815|715|1|118| +2450815|716|1|917| +2450815|718|1|849| +2450815|721|1|11| +2450815|722|1|51| +2450815|724|1|893| +2450815|727|1|254| +2450815|728|1|82| +2450815|730|1|782| +2450815|733|1|199| +2450815|734|1|588| +2450815|736|1|390| +2450815|739|1|576| +2450815|740|1|327| +2450815|742|1|107| +2450815|745|1|305| +2450815|746|1|979| +2450815|748|1|214| +2450815|751|1|706| +2450815|752|1|16| +2450815|754|1|760| +2450815|757|1|391| +2450815|758|1|186| +2450815|760|1|385| +2450815|763|1|491| +2450815|764|1|736| +2450815|766|1|| +2450815|769|1|747| +2450815|770|1|599| +2450815|772|1|604| +2450815|775|1|184| +2450815|776|1|847| +2450815|778|1|81| +2450815|781|1|251| +2450815|782|1|116| +2450815|784|1|609| +2450815|787|1|196| +2450815|788|1|909| +2450815|790|1|779| +2450815|793|1|664| +2450815|794|1|124| +2450815|796|1|788| +2450815|799|1|206| +2450815|800|1|529| +2450815|802|1|213| +2450815|805|1|233| +2450815|806|1|115| +2450815|808|1|664| +2450815|811|1|723| +2450815|812|1|384| +2450815|814|1|344| +2450815|817|1|617| +2450815|818|1|672| +2450815|820|1|377| +2450815|823|1|497| +2450815|824|1|321| +2450815|826|1|932| +2450815|829|1|13| +2450815|830|1|119| +2450815|832|1|675| +2450815|835|1|721| +2450815|836|1|303| +2450815|838|1|109| +2450815|841|1|627| +2450815|842|1|| +2450815|844|1|678| +2450815|847|1|907| +2450815|848|1|950| +2450815|850|1|594| +2450815|853|1|746| +2450815|854|1|979| +2450815|856|1|556| +2450815|859|1|939| +2450815|860|1|916| +2450815|862|1|| +2450815|865|1|171| +2450815|866|1|372| +2450815|868|1|120| +2450815|871|1|386| +2450815|872|1|646| +2450815|874|1|507| +2450815|877|1|688| +2450815|878|1|132| +2450815|880|1|466| +2450815|883|1|272| +2450815|884|1|464| +2450815|886|1|872| +2450815|889|1|928| +2450815|890|1|699| +2450815|892|1|994| +2450815|895|1|544| +2450815|896|1|385| +2450815|898|1|924| +2450815|901|1|26| +2450815|902|1|634| +2450815|904|1|25| +2450815|907|1|960| +2450815|908|1|124| +2450815|910|1|918| +2450815|913|1|589| +2450815|914|1|84| +2450815|916|1|16| +2450815|919|1|502| +2450815|920|1|934| +2450815|922|1|134| +2450815|925|1|125| +2450815|926|1|534| +2450815|928|1|515| +2450815|931|1|979| +2450815|932|1|107| +2450815|934|1|635| +2450815|937|1|282| +2450815|938|1|299| +2450815|940|1|| +2450815|943|1|974| +2450815|944|1|825| +2450815|946|1|879| +2450815|949|1|287| +2450815|950|1|179| +2450815|952|1|864| +2450815|955|1|590| +2450815|956|1|162| +2450815|958|1|| +2450815|961|1|837| +2450815|962|1|412| +2450815|964|1|585| +2450815|967|1|811| +2450815|968|1|23| +2450815|970|1|592| +2450815|973|1|7| +2450815|974|1|105| +2450815|976|1|222| +2450815|979|1|545| +2450815|980|1|533| +2450815|982|1|983| +2450815|985|1|913| +2450815|986|1|934| +2450815|988|1|| +2450815|991|1|423| +2450815|992|1|397| +2450815|994|1|77| +2450815|997|1|249| +2450815|998|1|373| +2450815|1000|1|916| +2450815|1003|1|887| +2450815|1004|1|598| +2450815|1006|1|270| +2450815|1009|1|199| +2450815|1010|1|665| +2450815|1012|1|239| +2450815|1015|1|782| +2450815|1016|1|46| +2450815|1018|1|584| +2450815|1021|1|631| +2450815|1022|1|516| +2450815|1024|1|235| +2450815|1027|1|53| +2450815|1028|1|487| +2450815|1030|1|589| +2450815|1033|1|510| +2450815|1034|1|566| +2450815|1036|1|561| +2450815|1039|1|329| +2450815|1040|1|| +2450815|1042|1|629| +2450815|1045|1|528| +2450815|1046|1|794| +2450815|1048|1|371| +2450815|1051|1|15| +2450815|1052|1|703| +2450815|1054|1|| +2450815|1057|1|580| +2450815|1058|1|347| +2450815|1060|1|292| +2450815|1063|1|247| +2450815|1064|1|843| +2450815|1066|1|39| +2450815|1069|1|401| +2450815|1070|1|641| +2450815|1072|1|208| +2450815|1075|1|194| +2450815|1076|1|14| +2450815|1078|1|43| +2450815|1081|1|557| +2450815|1082|1|346| +2450815|1084|1|994| +2450815|1087|1|532| +2450815|1088|1|729| +2450815|1090|1|321| +2450815|1093|1|934| +2450815|1094|1|608| +2450815|1096|1|646| +2450815|1099|1|298| +2450815|1100|1|910| +2450815|1102|1|935| +2450815|1105|1|525| +2450815|1106|1|36| +2450815|1108|1|541| +2450815|1111|1|407| +2450815|1112|1|| +2450815|1114|1|765| +2450815|1117|1|85| +2450815|1118|1|460| +2450815|1120|1|49| +2450815|1123|1|248| +2450815|1124|1|| +2450815|1126|1|369| +2450815|1129|1|338| +2450815|1130|1|258| +2450815|1132|1|321| +2450815|1135|1|955| +2450815|1136|1|905| +2450815|1138|1|968| +2450815|1141|1|332| +2450815|1142|1|| +2450815|1144|1|491| +2450815|1147|1|684| +2450815|1148|1|796| +2450815|1150|1|451| +2450815|1153|1|427| +2450815|1154|1|255| +2450815|1156|1|356| +2450815|1159|1|848| +2450815|1160|1|931| +2450815|1162|1|464| +2450815|1165|1|517| +2450815|1166|1|969| +2450815|1168|1|398| +2450815|1171|1|263| +2450815|1172|1|926| +2450815|1174|1|608| +2450815|1177|1|956| +2450815|1178|1|237| +2450815|1180|1|597| +2450815|1183|1|520| +2450815|1184|1|721| +2450815|1186|1|637| +2450815|1189|1|470| +2450815|1190|1|849| +2450815|1192|1|905| +2450815|1195|1|965| +2450815|1196|1|80| +2450815|1198|1|788| +2450815|1201|1|293| +2450815|1202|1|635| +2450815|1204|1|513| +2450815|1207|1|356| +2450815|1208|1|633| +2450815|1210|1|230| +2450815|1213|1|403| +2450815|1214|1|| +2450815|1216|1|894| +2450815|1219|1|174| +2450815|1220|1|38| +2450815|1222|1|292| +2450815|1225|1|418| +2450815|1226|1|716| +2450815|1228|1|698| +2450815|1231|1|| +2450815|1232|1|11| +2450815|1234|1|866| +2450815|1237|1|594| +2450815|1238|1|214| +2450815|1240|1|876| +2450815|1243|1|247| +2450815|1244|1|| +2450815|1246|1|287| +2450815|1249|1|615| +2450815|1250|1|185| +2450815|1252|1|39| +2450815|1255|1|| +2450815|1256|1|281| +2450815|1258|1|586| +2450815|1261|1|805| +2450815|1262|1|796| +2450815|1264|1|971| +2450815|1267|1|698| +2450815|1268|1|3| +2450815|1270|1|335| +2450815|1273|1|589| +2450815|1274|1|76| +2450815|1276|1|867| +2450815|1279|1|431| +2450815|1280|1|| +2450815|1282|1|366| +2450815|1285|1|620| +2450815|1286|1|858| +2450815|1288|1|28| +2450815|1291|1|860| +2450815|1292|1|43| +2450815|1294|1|570| +2450815|1297|1|586| +2450815|1298|1|552| +2450815|1300|1|32| +2450815|1303|1|677| +2450815|1304|1|757| +2450815|1306|1|328| +2450815|1309|1|777| +2450815|1310|1|458| +2450815|1312|1|912| +2450815|1315|1|17| +2450815|1316|1|520| +2450815|1318|1|367| +2450815|1321|1|619| +2450815|1322|1|167| +2450815|1324|1|553| +2450815|1327|1|329| +2450815|1328|1|384| +2450815|1330|1|774| +2450815|1333|1|743| +2450815|1334|1|824| +2450815|1336|1|485| +2450815|1339|1|310| +2450815|1340|1|188| +2450815|1342|1|234| +2450815|1345|1|773| +2450815|1346|1|433| +2450815|1348|1|276| +2450815|1351|1|75| +2450815|1352|1|521| +2450815|1354|1|218| +2450815|1357|1|771| +2450815|1358|1|786| +2450815|1360|1|570| +2450815|1363|1|985| +2450815|1364|1|230| +2450815|1366|1|501| +2450815|1369|1|| +2450815|1370|1|875| +2450815|1372|1|| +2450815|1375|1|596| +2450815|1376|1|249| +2450815|1378|1|266| +2450815|1381|1|15| +2450815|1382|1|232| +2450815|1384|1|674| +2450815|1387|1|647| +2450815|1388|1|934| +2450815|1390|1|935| +2450815|1393|1|130| +2450815|1394|1|6| +2450815|1396|1|544| +2450815|1399|1|222| +2450815|1400|1|22| +2450815|1402|1|589| +2450815|1405|1|563| +2450815|1406|1|559| +2450815|1408|1|93| +2450815|1411|1|590| +2450815|1412|1|972| +2450815|1414|1|94| +2450815|1417|1|151| +2450815|1418|1|791| +2450815|1420|1|190| +2450815|1423|1|928| +2450815|1424|1|| +2450815|1426|1|385| +2450815|1429|1|665| +2450815|1430|1|971| +2450815|1432|1|778| +2450815|1435|1|682| +2450815|1436|1|877| +2450815|1438|1|3| +2450815|1441|1|335| +2450815|1442|1|20| +2450815|1444|1|524| +2450815|1447|1|631| +2450815|1448|1|799| +2450815|1450|1|137| +2450815|1453|1|| +2450815|1454|1|437| +2450815|1456|1|488| +2450815|1459|1|45| +2450815|1460|1|716| +2450815|1462|1|67| +2450815|1465|1|| +2450815|1466|1|428| +2450815|1468|1|74| +2450815|1471|1|177| +2450815|1472|1|635| +2450815|1474|1|233| +2450815|1477|1|345| +2450815|1478|1|717| +2450815|1480|1|47| +2450815|1483|1|266| +2450815|1484|1|539| +2450815|1486|1|331| +2450815|1489|1|58| +2450815|1490|1|441| +2450815|1492|1|764| +2450815|1495|1|9| +2450815|1496|1|35| +2450815|1498|1|461| +2450815|1501|1|230| +2450815|1502|1|214| +2450815|1504|1|121| +2450815|1507|1|475| +2450815|1508|1|203| +2450815|1510|1|47| +2450815|1513|1|118| +2450815|1514|1|773| +2450815|1516|1|517| +2450815|1519|1|857| +2450815|1520|1|37| +2450815|1522|1|959| +2450815|1525|1|620| +2450815|1526|1|239| +2450815|1528|1|968| +2450815|1531|1|747| +2450815|1532|1|255| +2450815|1534|1|69| +2450815|1537|1|537| +2450815|1538|1|142| +2450815|1540|1|626| +2450815|1543|1|924| +2450815|1544|1|742| +2450815|1546|1|499| +2450815|1549|1|166| +2450815|1550|1|720| +2450815|1552|1|284| +2450815|1555|1|426| +2450815|1556|1|960| +2450815|1558|1|132| +2450815|1561|1|141| +2450815|1562|1|162| +2450815|1564|1|351| +2450815|1567|1|43| +2450815|1568|1|187| +2450815|1570|1|416| +2450815|1573|1|984| +2450815|1574|1|794| +2450815|1576|1|711| +2450815|1579|1|| +2450815|1580|1|612| +2450815|1582|1|393| +2450815|1585|1|867| +2450815|1586|1|335| +2450815|1588|1|701| +2450815|1591|1|117| +2450815|1592|1|| +2450815|1594|1|763| +2450815|1597|1|526| +2450815|1598|1|204| +2450815|1600|1|16| +2450815|1603|1|999| +2450815|1604|1|213| +2450815|1606|1|484| +2450815|1609|1|| +2450815|1610|1|923| +2450815|1612|1|518| +2450815|1615|1|557| +2450815|1616|1|596| +2450815|1618|1|413| +2450815|1621|1|795| +2450815|1622|1|573| +2450815|1624|1|443| +2450815|1627|1|375| +2450815|1628|1|15| +2450815|1630|1|37| +2450815|1633|1|233| +2450815|1634|1|733| +2450815|1636|1|267| +2450815|1639|1|981| +2450815|1640|1|268| +2450815|1642|1|598| +2450815|1645|1|831| +2450815|1646|1|436| +2450815|1648|1|804| +2450815|1651|1|228| +2450815|1652|1|756| +2450815|1654|1|807| +2450815|1657|1|952| +2450815|1658|1|793| +2450815|1660|1|539| +2450815|1663|1|474| +2450815|1664|1|367| +2450815|1666|1|481| +2450815|1669|1|338| +2450815|1670|1|320| +2450815|1672|1|599| +2450815|1675|1|450| +2450815|1676|1|574| +2450815|1678|1|800| +2450815|1681|1|104| +2450815|1682|1|85| +2450815|1684|1|230| +2450815|1687|1|971| +2450815|1688|1|914| +2450815|1690|1|748| +2450815|1693|1|803| +2450815|1694|1|214| +2450815|1696|1|283| +2450815|1699|1|584| +2450815|1700|1|639| +2450815|1702|1|472| +2450815|1705|1|651| +2450815|1706|1|777| +2450815|1708|1|434| +2450815|1711|1|614| +2450815|1712|1|97| +2450815|1714|1|927| +2450815|1717|1|237| +2450815|1718|1|57| +2450815|1720|1|76| +2450815|1723|1|839| +2450815|1724|1|123| +2450815|1726|1|327| +2450815|1729|1|666| +2450815|1730|1|950| +2450815|1732|1|23| +2450815|1735|1|212| +2450815|1736|1|455| +2450815|1738|1|83| +2450815|1741|1|221| +2450815|1742|1|479| +2450815|1744|1|271| +2450815|1747|1|52| +2450815|1748|1|901| +2450815|1750|1|991| +2450815|1753|1|510| +2450815|1754|1|| +2450815|1756|1|694| +2450815|1759|1|138| +2450815|1760|1|632| +2450815|1762|1|369| +2450815|1765|1|473| +2450815|1766|1|595| +2450815|1768|1|144| +2450815|1771|1|840| +2450815|1772|1|578| +2450815|1774|1|444| +2450815|1777|1|427| +2450815|1778|1|276| +2450815|1780|1|998| +2450815|1783|1|823| +2450815|1784|1|906| +2450815|1786|1|329| +2450815|1789|1|794| +2450815|1790|1|375| +2450815|1792|1|450| +2450815|1795|1|58| +2450815|1796|1|828| +2450815|1798|1|39| +2450815|1801|1|852| +2450815|1802|1|800| +2450815|1804|1|481| +2450815|1807|1|135| +2450815|1808|1|75| +2450815|1810|1|402| +2450815|1813|1|128| +2450815|1814|1|243| +2450815|1816|1|695| +2450815|1819|1|426| +2450815|1820|1|623| +2450815|1822|1|214| +2450815|1825|1|284| +2450815|1826|1|951| +2450815|1828|1|| +2450815|1831|1|561| +2450815|1832|1|575| +2450815|1834|1|| +2450815|1837|1|752| +2450815|1838|1|396| +2450815|1840|1|580| +2450815|1843|1|| +2450815|1844|1|376| +2450815|1846|1|515| +2450815|1849|1|632| +2450815|1850|1|768| +2450815|1852|1|89| +2450815|1855|1|335| +2450815|1856|1|980| +2450815|1858|1|158| +2450815|1861|1|996| +2450815|1862|1|| +2450815|1864|1|507| +2450815|1867|1|979| +2450815|1868|1|396| +2450815|1870|1|203| +2450815|1873|1|400| +2450815|1874|1|695| +2450815|1876|1|846| +2450815|1879|1|175| +2450815|1880|1|| +2450815|1882|1|149| +2450815|1885|1|710| +2450815|1886|1|690| +2450815|1888|1|334| +2450815|1891|1|477| +2450815|1892|1|| +2450815|1894|1|385| +2450815|1897|1|608| +2450815|1898|1|938| +2450815|1900|1|565| +2450815|1903|1|756| +2450815|1904|1|92| +2450815|1906|1|598| +2450815|1909|1|880| +2450815|1910|1|744| +2450815|1912|1|822| +2450815|1915|1|96| +2450815|1916|1|194| +2450815|1918|1|758| +2450815|1921|1|2| +2450815|1922|1|211| +2450815|1924|1|755| +2450815|1927|1|698| +2450815|1928|1|40| +2450815|1930|1|595| +2450815|1933|1|669| +2450815|1934|1|338| +2450815|1936|1|648| +2450815|1939|1|993| +2450815|1940|1|640| +2450815|1942|1|587| +2450815|1945|1|45| +2450815|1946|1|156| +2450815|1948|1|380| +2450815|1951|1|30| +2450815|1952|1|280| +2450815|1954|1|976| +2450815|1957|1|242| +2450815|1958|1|399| +2450815|1960|1|| +2450815|1963|1|888| +2450815|1964|1|991| +2450815|1966|1|68| +2450815|1969|1|986| +2450815|1970|1|822| +2450815|1972|1|581| +2450815|1975|1|671| +2450815|1976|1|428| +2450815|1978|1|825| +2450815|1981|1|364| +2450815|1982|1|528| +2450815|1984|1|989| +2450815|1987|1|3| +2450815|1988|1|513| +2450815|1990|1|84| +2450815|1993|1|234| +2450815|1994|1|450| +2450815|1996|1|76| +2450815|1999|1|906| +2450815|2000|1|431| +2450815|2002|1|692| +2450815|2005|1|821| +2450815|2006|1|829| +2450815|2008|1|283| +2450815|2011|1|410| +2450815|2012|1|183| +2450815|2014|1|| +2450815|2017|1|867| +2450815|2018|1|296| +2450815|2020|1|849| +2450815|2023|1|556| +2450815|2024|1|122| +2450815|2026|1|767| +2450815|2029|1|938| +2450815|2030|1|221| +2450815|2032|1|470| +2450815|2035|1|383| +2450815|2036|1|767| +2450815|2038|1|866| +2450815|2041|1|435| +2450815|2042|1|55| +2450815|2044|1|857| +2450815|2047|1|900| +2450815|2048|1|773| +2450815|2050|1|304| +2450815|2053|1|596| +2450815|2054|1|692| +2450815|2056|1|488| +2450815|2059|1|635| +2450815|2060|1|997| +2450815|2062|1|722| +2450815|2065|1|103| +2450815|2066|1|899| +2450815|2068|1|223| +2450815|2071|1|116| +2450815|2072|1|983| +2450815|2074|1|865| +2450815|2077|1|49| +2450815|2078|1|628| +2450815|2080|1|154| +2450815|2083|1|850| +2450815|2084|1|917| +2450815|2086|1|854| +2450815|2089|1|817| +2450815|2090|1|930| +2450815|2092|1|743| +2450815|2095|1|959| +2450815|2096|1|| +2450815|2098|1|665| +2450815|2101|1|944| +2450815|2102|1|638| +2450815|2104|1|837| +2450815|2107|1|99| +2450815|2108|1|| +2450815|2110|1|158| +2450815|2113|1|956| +2450815|2114|1|633| +2450815|2116|1|547| +2450815|2119|1|266| +2450815|2120|1|863| +2450815|2122|1|688| +2450815|2125|1|119| +2450815|2126|1|102| +2450815|2128|1|178| +2450815|2131|1|388| +2450815|2132|1|261| +2450815|2134|1|196| +2450815|2137|1|756| +2450815|2138|1|842| +2450815|2140|1|484| +2450815|2143|1|688| +2450815|2144|1|481| +2450815|2146|1|342| +2450815|2149|1|877| +2450815|2150|1|711| +2450815|2152|1|799| +2450815|2155|1|170| +2450815|2156|1|905| +2450815|2158|1|42| +2450815|2161|1|557| +2450815|2162|1|992| +2450815|2164|1|369| +2450815|2167|1|847| +2450815|2168|1|1000| +2450815|2170|1|377| +2450815|2173|1|665| +2450815|2174|1|615| +2450815|2176|1|891| +2450815|2179|1|761| +2450815|2180|1|45| +2450815|2182|1|712| +2450815|2185|1|510| +2450815|2186|1|683| +2450815|2188|1|912| +2450815|2191|1|14| +2450815|2192|1|422| +2450815|2194|1|218| +2450815|2197|1|338| +2450815|2198|1|227| +2450815|2200|1|266| +2450815|2203|1|284| +2450815|2204|1|| +2450815|2206|1|196| +2450815|2209|1|206| +2450815|2210|1|641| +2450815|2212|1|776| +2450815|2215|1|623| +2450815|2216|1|333| +2450815|2218|1|392| +2450815|2221|1|| +2450815|2222|1|201| +2450815|2224|1|35| +2450815|2227|1|686| +2450815|2228|1|98| +2450815|2230|1|864| +2450815|2233|1|662| +2450815|2234|1|823| +2450815|2236|1|422| +2450815|2239|1|466| +2450815|2240|1|334| +2450815|2242|1|395| +2450815|2245|1|536| +2450815|2246|1|107| +2450815|2248|1|998| +2450815|2251|1|754| +2450815|2252|1|361| +2450815|2254|1|138| +2450815|2257|1|| +2450815|2258|1|239| +2450815|2260|1|585| +2450815|2263|1|108| +2450815|2264|1|940| +2450815|2266|1|615| +2450815|2269|1|605| +2450815|2270|1|632| +2450815|2272|1|205| +2450815|2275|1|874| +2450815|2276|1|954| +2450815|2278|1|89| +2450815|2281|1|519| +2450815|2282|1|918| +2450815|2284|1|205| +2450815|2287|1|680| +2450815|2288|1|979| +2450815|2290|1|61| +2450815|2293|1|846| +2450815|2294|1|785| +2450815|2296|1|843| +2450815|2299|1|805| +2450815|2300|1|950| +2450815|2302|1|121| +2450815|2305|1|98| +2450815|2306|1|856| +2450815|2308|1|73| +2450815|2311|1|663| +2450815|2312|1|674| +2450815|2314|1|961| +2450815|2317|1|| +2450815|2318|1|299| +2450815|2320|1|615| +2450815|2323|1|236| +2450815|2324|1|875| +2450815|2326|1|691| +2450815|2329|1|189| +2450815|2330|1|925| +2450815|2332|1|69| +2450815|2335|1|607| +2450815|2336|1|444| +2450815|2338|1|251| +2450815|2341|1|753| +2450815|2342|1|612| +2450815|2344|1|646| +2450815|2347|1|92| +2450815|2348|1|151| +2450815|2350|1|212| +2450815|2353|1|181| +2450815|2354|1|846| +2450815|2356|1|| +2450815|2359|1|660| +2450815|2360|1|982| +2450815|2362|1|0| +2450815|2365|1|449| +2450815|2366|1|87| +2450815|2368|1|798| +2450815|2371|1|637| +2450815|2372|1|491| +2450815|2374|1|595| +2450815|2377|1|488| +2450815|2378|1|399| +2450815|2380|1|5| +2450815|2383|1|805| +2450815|2384|1|303| +2450815|2386|1|915| +2450815|2389|1|565| +2450815|2390|1|| +2450815|2392|1|612| +2450815|2395|1|956| +2450815|2396|1|796| +2450815|2398|1|952| +2450815|2401|1|| +2450815|2402|1|48| +2450815|2404|1|820| +2450815|2407|1|1| +2450815|2408|1|68| +2450815|2410|1|86| +2450815|2413|1|464| +2450815|2414|1|177| +2450815|2416|1|747| +2450815|2419|1|776| +2450815|2420|1|989| +2450815|2422|1|920| +2450815|2425|1|884| +2450815|2426|1|91| +2450815|2428|1|596| +2450815|2431|1|902| +2450815|2432|1|739| +2450815|2434|1|110| +2450815|2437|1|732| +2450815|2438|1|239| +2450815|2440|1|306| +2450815|2443|1|923| +2450815|2444|1|724| +2450815|2446|1|572| +2450815|2449|1|154| +2450815|2450|1|114| +2450815|2452|1|570| +2450815|2455|1|590| +2450815|2456|1|657| +2450815|2458|1|651| +2450815|2461|1|469| +2450815|2462|1|350| +2450815|2464|1|18| +2450815|2467|1|401| +2450815|2468|1|49| +2450815|2470|1|473| +2450815|2473|1|440| +2450815|2474|1|639| +2450815|2476|1|276| +2450815|2479|1|731| +2450815|2480|1|369| +2450815|2482|1|563| +2450815|2485|1|827| +2450815|2486|1|179| +2450815|2488|1|223| +2450815|2491|1|703| +2450815|2492|1|830| +2450815|2494|1|978| +2450815|2497|1|986| +2450815|2498|1|907| +2450815|2500|1|986| +2450815|2503|1|| +2450815|2504|1|11| +2450815|2506|1|| +2450815|2509|1|609| +2450815|2510|1|944| +2450815|2512|1|279| +2450815|2515|1|601| +2450815|2516|1|114| +2450815|2518|1|819| +2450815|2521|1|601| +2450815|2522|1|900| +2450815|2524|1|48| +2450815|2527|1|84| +2450815|2528|1|904| +2450815|2530|1|919| +2450815|2533|1|882| +2450815|2534|1|845| +2450815|2536|1|| +2450815|2539|1|346| +2450815|2540|1|921| +2450815|2542|1|| +2450815|2545|1|479| +2450815|2546|1|231| +2450815|2548|1|239| +2450815|2551|1|994| +2450815|2552|1|492| +2450815|2554|1|884| +2450815|2557|1|429| +2450815|2558|1|503| +2450815|2560|1|292| +2450815|2563|1|729| +2450815|2564|1|| +2450815|2566|1|964| +2450815|2569|1|918| +2450815|2570|1|418| +2450815|2572|1|799| +2450815|2575|1|294| +2450815|2576|1|141| +2450815|2578|1|705| +2450815|2581|1|382| +2450815|2582|1|501| +2450815|2584|1|458| +2450815|2587|1|964| +2450815|2588|1|303| +2450815|2590|1|588| +2450815|2593|1|294| +2450815|2594|1|153| +2450815|2596|1|767| +2450815|2599|1|806| +2450815|2600|1|914| +2450815|2602|1|277| +2450815|2605|1|176| +2450815|2606|1|160| +2450815|2608|1|| +2450815|2611|1|543| +2450815|2612|1|462| +2450815|2614|1|240| +2450815|2617|1|530| +2450815|2618|1|425| +2450815|2620|1|523| +2450815|2623|1|157| +2450815|2624|1|624| +2450815|2626|1|904| +2450815|2629|1|544| +2450815|2630|1|391| +2450815|2632|1|379| +2450815|2635|1|169| +2450815|2636|1|608| +2450815|2638|1|233| +2450815|2641|1|706| +2450815|2642|1|132| +2450815|2644|1|295| +2450815|2647|1|445| +2450815|2648|1|839| +2450815|2650|1|171| +2450815|2653|1|243| +2450815|2654|1|780| +2450815|2656|1|762| +2450815|2659|1|436| +2450815|2660|1|588| +2450815|2662|1|726| +2450815|2665|1|319| +2450815|2666|1|644| +2450815|2668|1|635| +2450815|2671|1|318| +2450815|2672|1|513| +2450815|2674|1|869| +2450815|2677|1|206| +2450815|2678|1|603| +2450815|2680|1|540| +2450815|2683|1|824| +2450815|2684|1|852| +2450815|2686|1|983| +2450815|2689|1|888| +2450815|2690|1|825| +2450815|2692|1|999| +2450815|2695|1|597| +2450815|2696|1|977| +2450815|2698|1|350| +2450815|2701|1|432| +2450815|2702|1|494| +2450815|2704|1|25| +2450815|2707|1|755| +2450815|2708|1|951| +2450815|2710|1|579| +2450815|2713|1|643| +2450815|2714|1|582| +2450815|2716|1|80| +2450815|2719|1|742| +2450815|2720|1|27| +2450815|2722|1|918| +2450815|2725|1|451| +2450815|2726|1|432| +2450815|2728|1|546| +2450815|2731|1|460| +2450815|2732|1|643| +2450815|2734|1|893| +2450815|2737|1|313| +2450815|2738|1|573| +2450815|2740|1|662| +2450815|2743|1|138| +2450815|2744|1|321| +2450815|2746|1|262| +2450815|2749|1|46| +2450815|2750|1|467| +2450815|2752|1|960| +2450815|2755|1|557| +2450815|2756|1|937| +2450815|2758|1|929| +2450815|2761|1|115| +2450815|2762|1|66| +2450815|2764|1|808| +2450815|2767|1|286| +2450815|2768|1|| +2450815|2770|1|940| +2450815|2773|1|248| +2450815|2774|1|968| +2450815|2776|1|374| +2450815|2779|1|35| +2450815|2780|1|773| +2450815|2782|1|942| +2450815|2785|1|591| +2450815|2786|1|359| +2450815|2788|1|682| +2450815|2791|1|545| +2450815|2792|1|866| +2450815|2794|1|700| +2450815|2797|1|389| +2450815|2798|1|666| +2450815|2800|1|848| +2450815|2803|1|36| +2450815|2804|1|576| +2450815|2806|1|110| +2450815|2809|1|383| +2450815|2810|1|714| +2450815|2812|1|740| +2450815|2815|1|521| +2450815|2816|1|447| +2450815|2818|1|23| +2450815|2821|1|503| +2450815|2822|1|498| +2450815|2824|1|489| +2450815|2827|1|809| +2450815|2828|1|507| +2450815|2830|1|368| +2450815|2833|1|84| +2450815|2834|1|58| +2450815|2836|1|120| +2450815|2839|1|| +2450815|2840|1|373| +2450815|2842|1|335| +2450815|2845|1|| +2450815|2846|1|931| +2450815|2848|1|285| +2450815|2851|1|964| +2450815|2852|1|414| +2450815|2854|1|905| +2450815|2857|1|474| +2450815|2858|1|803| +2450815|2860|1|799| +2450815|2863|1|123| +2450815|2864|1|510| +2450815|2866|1|780| +2450815|2869|1|999| +2450815|2870|1|303| +2450815|2872|1|315| +2450815|2875|1|752| +2450815|2876|1|714| +2450815|2878|1|53| +2450815|2881|1|322| +2450815|2882|1|348| +2450815|2884|1|865| +2450815|2887|1|119| +2450815|2888|1|262| +2450815|2890|1|46| +2450815|2893|1|360| +2450815|2894|1|477| +2450815|2896|1|547| +2450815|2899|1|439| +2450815|2900|1|254| +2450815|2902|1|911| +2450815|2905|1|968| +2450815|2906|1|543| +2450815|2908|1|864| +2450815|2911|1|668| +2450815|2912|1|339| +2450815|2914|1|575| +2450815|2917|1|163| +2450815|2918|1|597| +2450815|2920|1|624| +2450815|2923|1|872| +2450815|2924|1|92| +2450815|2926|1|885| +2450815|2929|1|20| +2450815|2930|1|| +2450815|2932|1|859| +2450815|2935|1|491| +2450815|2936|1|676| +2450815|2938|1|517| +2450815|2941|1|872| +2450815|2942|1|322| +2450815|2944|1|413| +2450815|2947|1|313| +2450815|2948|1|356| +2450815|2950|1|252| +2450815|2953|1|718| +2450815|2954|1|154| +2450815|2956|1|335| +2450815|2959|1|890| +2450815|2960|1|345| +2450815|2962|1|154| +2450815|2965|1|168| +2450815|2966|1|523| +2450815|2968|1|324| +2450815|2971|1|817| +2450815|2972|1|250| +2450815|2974|1|| +2450815|2977|1|399| +2450815|2978|1|430| +2450815|2980|1|685| +2450815|2983|1|878| +2450815|2984|1|207| +2450815|2986|1|346| +2450815|2989|1|941| +2450815|2990|1|812| +2450815|2992|1|674| +2450815|2995|1|125| +2450815|2996|1|690| +2450815|2998|1|493| +2450815|3001|1|793| +2450815|3002|1|791| +2450815|3004|1|95| +2450815|3007|1|941| +2450815|3008|1|706| +2450815|3010|1|994| +2450815|3013|1|252| +2450815|3014|1|859| +2450815|3016|1|83| +2450815|3019|1|715| +2450815|3020|1|529| +2450815|3022|1|800| +2450815|3025|1|842| +2450815|3026|1|226| +2450815|3028|1|236| +2450815|3031|1|735| +2450815|3032|1|676| +2450815|3034|1|803| +2450815|3037|1|822| +2450815|3038|1|87| +2450815|3040|1|802| +2450815|3043|1|| +2450815|3044|1|161| +2450815|3046|1|708| +2450815|3049|1|309| +2450815|3050|1|778| +2450815|3052|1|454| +2450815|3055|1|614| +2450815|3056|1|502| +2450815|3058|1|568| +2450815|3061|1|206| +2450815|3062|1|361| +2450815|3064|1|505| +2450815|3067|1|154| +2450815|3068|1|915| +2450815|3070|1|623| +2450815|3073|1|| +2450815|3074|1|816| +2450815|3076|1|737| +2450815|3079|1|719| +2450815|3080|1|587| +2450815|3082|1|| +2450815|3085|1|5| +2450815|3086|1|| +2450815|3088|1|421| +2450815|3091|1|191| +2450815|3092|1|735| +2450815|3094|1|543| +2450815|3097|1|1| +2450815|3098|1|988| +2450815|3100|1|117| +2450815|3103|1|772| +2450815|3104|1|946| +2450815|3106|1|593| +2450815|3109|1|924| +2450815|3110|1|850| +2450815|3112|1|195| +2450815|3115|1|354| +2450815|3116|1|469| +2450815|3118|1|572| +2450815|3121|1|141| +2450815|3122|1|| +2450815|3124|1|516| +2450815|3127|1|310| +2450815|3128|1|543| +2450815|3130|1|626| +2450815|3133|1|43| +2450815|3134|1|743| +2450815|3136|1|941| +2450815|3139|1|960| +2450815|3140|1|477| +2450815|3142|1|521| +2450815|3145|1|694| +2450815|3146|1|599| +2450815|3148|1|593| +2450815|3151|1|797| +2450815|3152|1|908| +2450815|3154|1|850| +2450815|3157|1|573| +2450815|3158|1|967| +2450815|3160|1|32| +2450815|3163|1|451| +2450815|3164|1|562| +2450815|3166|1|979| +2450815|3169|1|835| +2450815|3170|1|990| +2450815|3172|1|168| +2450815|3175|1|103| +2450815|3176|1|961| +2450815|3178|1|502| +2450815|3181|1|598| +2450815|3182|1|32| +2450815|3184|1|990| +2450815|3187|1|772| +2450815|3188|1|186| +2450815|3190|1|781| +2450815|3193|1|536| +2450815|3194|1|5| +2450815|3196|1|77| +2450815|3199|1|670| +2450815|3200|1|991| +2450815|3202|1|18| +2450815|3205|1|766| +2450815|3206|1|755| +2450815|3208|1|197| +2450815|3211|1|622| +2450815|3212|1|133| +2450815|3214|1|363| +2450815|3217|1|387| +2450815|3218|1|78| +2450815|3220|1|586| +2450815|3223|1|185| +2450815|3224|1|95| +2450815|3226|1|617| +2450815|3229|1|798| +2450815|3230|1|395| +2450815|3232|1|| +2450815|3235|1|180| +2450815|3236|1|874| +2450815|3238|1|92| +2450815|3241|1|11| +2450815|3242|1|563| +2450815|3244|1|947| +2450815|3247|1|512| +2450815|3248|1|287| +2450815|3250|1|957| +2450815|3253|1|131| +2450815|3254|1|151| +2450815|3256|1|373| +2450815|3259|1|138| +2450815|3260|1|293| +2450815|3262|1|64| +2450815|3265|1|751| +2450815|3266|1|481| +2450815|3268|1|43| +2450815|3271|1|977| +2450815|3272|1|997| +2450815|3274|1|228| +2450815|3277|1|583| +2450815|3278|1|269| +2450815|3280|1|| +2450815|3283|1|159| +2450815|3284|1|347| +2450815|3286|1|696| +2450815|3289|1|846| +2450815|3290|1|811| +2450815|3292|1|56| +2450815|3295|1|207| +2450815|3296|1|174| +2450815|3298|1|258| +2450815|3301|1|828| +2450815|3302|1|574| +2450815|3304|1|835| +2450815|3307|1|615| +2450815|3308|1|49| +2450815|3310|1|312| +2450815|3313|1|| +2450815|3314|1|725| +2450815|3316|1|542| +2450815|3319|1|679| +2450815|3320|1|313| +2450815|3322|1|900| +2450815|3325|1|684| +2450815|3326|1|229| +2450815|3328|1|251| +2450815|3331|1|867| +2450815|3332|1|86| +2450815|3334|1|444| +2450815|3337|1|195| +2450815|3338|1|552| +2450815|3340|1|358| +2450815|3343|1|743| +2450815|3344|1|55| +2450815|3346|1|755| +2450815|3349|1|235| +2450815|3350|1|45| +2450815|3352|1|257| +2450815|3355|1|200| +2450815|3356|1|865| +2450815|3358|1|166| +2450815|3361|1|560| +2450815|3362|1|763| +2450815|3364|1|727| +2450815|3367|1|843| +2450815|3368|1|715| +2450815|3370|1|775| +2450815|3373|1|112| +2450815|3374|1|| +2450815|3376|1|341| +2450815|3379|1|781| +2450815|3380|1|524| +2450815|3382|1|872| +2450815|3385|1|| +2450815|3386|1|936| +2450815|3388|1|100| +2450815|3391|1|139| +2450815|3392|1|464| +2450815|3394|1|393| +2450815|3397|1|| +2450815|3398|1|642| +2450815|3400|1|901| +2450815|3403|1|570| +2450815|3404|1|993| +2450815|3406|1|492| +2450815|3409|1|65| +2450815|3410|1|814| +2450815|3412|1|731| +2450815|3415|1|597| +2450815|3416|1|352| +2450815|3418|1|527| +2450815|3421|1|354| +2450815|3422|1|972| +2450815|3424|1|73| +2450815|3427|1|251| +2450815|3428|1|173| +2450815|3430|1|797| +2450815|3433|1|98| +2450815|3434|1|527| +2450815|3436|1|545| +2450815|3439|1|687| +2450815|3440|1|622| +2450815|3442|1|760| +2450815|3445|1|706| +2450815|3446|1|780| +2450815|3448|1|335| +2450815|3451|1|672| +2450815|3452|1|680| +2450815|3454|1|139| +2450815|3457|1|400| +2450815|3458|1|309| +2450815|3460|1|86| +2450815|3463|1|976| +2450815|3464|1|897| +2450815|3466|1|522| +2450815|3469|1|898| +2450815|3470|1|271| +2450815|3472|1|648| +2450815|3475|1|564| +2450815|3476|1|650| +2450815|3478|1|595| +2450815|3481|1|359| +2450815|3482|1|272| +2450815|3484|1|470| +2450815|3487|1|384| +2450815|3488|1|831| +2450815|3490|1|877| +2450815|3493|1|630| +2450815|3494|1|391| +2450815|3496|1|238| +2450815|3499|1|990| +2450815|3500|1|| +2450815|3502|1|172| +2450815|3505|1|53| +2450815|3506|1|826| +2450815|3508|1|297| +2450815|3511|1|860| +2450815|3512|1|245| +2450815|3514|1|394| +2450815|3517|1|829| +2450815|3518|1|981| +2450815|3520|1|101| +2450815|3523|1|905| +2450815|3524|1|412| +2450815|3526|1|403| +2450815|3529|1|669| +2450815|3530|1|387| +2450815|3532|1|349| +2450815|3535|1|749| +2450815|3536|1|257| +2450815|3538|1|50| +2450815|3541|1|| +2450815|3542|1|582| +2450815|3544|1|663| +2450815|3547|1|306| +2450815|3548|1|| +2450815|3550|1|337| +2450815|3553|1|480| +2450815|3554|1|579| +2450815|3556|1|601| +2450815|3559|1|991| +2450815|3560|1|908| +2450815|3562|1|399| +2450815|3565|1|303| +2450815|3566|1|883| +2450815|3568|1|560| +2450815|3571|1|968| +2450815|3572|1|814| +2450815|3574|1|450| +2450815|3577|1|977| +2450815|3578|1|44| +2450815|3580|1|547| +2450815|3583|1|414| +2450815|3584|1|830| +2450815|3586|1|373| +2450815|3589|1|727| +2450815|3590|1|187| +2450815|3592|1|240| +2450815|3595|1|985| +2450815|3596|1|402| +2450815|3598|1|744| +2450815|3601|1|191| +2450815|3602|1|865| +2450815|3604|1|865| +2450815|3607|1|389| +2450815|3608|1|412| +2450815|3610|1|370| +2450815|3613|1|168| +2450815|3614|1|159| +2450815|3616|1|81| +2450815|3619|1|693| +2450815|3620|1|967| +2450815|3622|1|720| +2450815|3625|1|537| +2450815|3626|1|419| +2450815|3628|1|738| +2450815|3631|1|400| +2450815|3632|1|324| +2450815|3634|1|617| +2450815|3637|1|109| +2450815|3638|1|17| +2450815|3640|1|428| +2450815|3643|1|103| +2450815|3644|1|182| +2450815|3646|1|637| +2450815|3649|1|798| +2450815|3650|1|25| +2450815|3652|1|118| +2450815|3655|1|608| +2450815|3656|1|636| +2450815|3658|1|163| +2450815|3661|1|35| +2450815|3662|1|329| +2450815|3664|1|618| +2450815|3667|1|544| +2450815|3668|1|562| +2450815|3670|1|| +2450815|3673|1|823| +2450815|3674|1|412| +2450815|3676|1|862| +2450815|3679|1|296| +2450815|3680|1|645| +2450815|3682|1|562| +2450815|3685|1|538| +2450815|3686|1|66| +2450815|3688|1|102| +2450815|3691|1|413| +2450815|3692|1|671| +2450815|3694|1|218| +2450815|3697|1|806| +2450815|3698|1|269| +2450815|3700|1|8| +2450815|3703|1|940| +2450815|3704|1|783| +2450815|3706|1|668| +2450815|3709|1|| +2450815|3710|1|460| +2450815|3712|1|510| +2450815|3715|1|778| +2450815|3716|1|734| +2450815|3718|1|609| +2450815|3721|1|915| +2450815|3722|1|650| +2450815|3724|1|753| +2450815|3727|1|696| +2450815|3728|1|755| +2450815|3730|1|298| +2450815|3733|1|627| +2450815|3734|1|213| +2450815|3736|1|462| +2450815|3739|1|563| +2450815|3740|1|106| +2450815|3742|1|30| +2450815|3745|1|674| +2450815|3746|1|| +2450815|3748|1|796| +2450815|3751|1|758| +2450815|3752|1|933| +2450815|3754|1|523| +2450815|3757|1|235| +2450815|3758|1|42| +2450815|3760|1|629| +2450815|3763|1|161| +2450815|3764|1|739| +2450815|3766|1|575| +2450815|3769|1|688| +2450815|3770|1|539| +2450815|3772|1|384| +2450815|3775|1|631| +2450815|3776|1|737| +2450815|3778|1|798| +2450815|3781|1|761| +2450815|3782|1|399| +2450815|3784|1|948| +2450815|3787|1|597| +2450815|3788|1|117| +2450815|3790|1|918| +2450815|3793|1|984| +2450815|3794|1|844| +2450815|3796|1|730| +2450815|3799|1|685| +2450815|3800|1|331| +2450815|3802|1|465| +2450815|3805|1|423| +2450815|3806|1|587| +2450815|3808|1|891| +2450815|3811|1|426| +2450815|3812|1|262| +2450815|3814|1|464| +2450815|3817|1|272| +2450815|3818|1|647| +2450815|3820|1|388| +2450815|3823|1|694| +2450815|3824|1|727| +2450815|3826|1|852| +2450815|3829|1|| +2450815|3830|1|585| +2450815|3832|1|64| +2450815|3835|1|586| +2450815|3836|1|146| +2450815|3838|1|169| +2450815|3841|1|393| +2450815|3842|1|118| +2450815|3844|1|845| +2450815|3847|1|| +2450815|3848|1|388| +2450815|3850|1|831| +2450815|3853|1|841| +2450815|3854|1|578| +2450815|3856|1|823| +2450815|3859|1|974| +2450815|3860|1|153| +2450815|3862|1|264| +2450815|3865|1|98| +2450815|3866|1|| +2450815|3868|1|942| +2450815|3871|1|819| +2450815|3872|1|936| +2450815|3874|1|124| +2450815|3877|1|54| +2450815|3878|1|846| +2450815|3880|1|949| +2450815|3883|1|52| +2450815|3884|1|809| +2450815|3886|1|651| +2450815|3889|1|339| +2450815|3890|1|679| +2450815|3892|1|205| +2450815|3895|1|392| +2450815|3896|1|97| +2450815|3898|1|790| +2450815|3901|1|376| +2450815|3902|1|242| +2450815|3904|1|948| +2450815|3907|1|713| +2450815|3908|1|946| +2450815|3910|1|921| +2450815|3913|1|588| +2450815|3914|1|30| +2450815|3916|1|416| +2450815|3919|1|823| +2450815|3920|1|580| +2450815|3922|1|997| +2450815|3925|1|738| +2450815|3926|1|437| +2450815|3928|1|782| +2450815|3931|1|271| +2450815|3932|1|238| +2450815|3934|1|31| +2450815|3937|1|700| +2450815|3938|1|841| +2450815|3940|1|175| +2450815|3943|1|| +2450815|3944|1|| +2450815|3946|1|90| +2450815|3949|1|840| +2450815|3950|1|701| +2450815|3952|1|667| +2450815|3955|1|803| +2450815|3956|1|7| +2450815|3958|1|884| +2450815|3961|1|337| +2450815|3962|1|984| +2450815|3964|1|222| +2450815|3967|1|835| +2450815|3968|1|209| +2450815|3970|1|503| +2450815|3973|1|670| +2450815|3974|1|24| +2450815|3976|1|842| +2450815|3979|1|| +2450815|3980|1|215| +2450815|3982|1|120| +2450815|3985|1|6| +2450815|3986|1|129| +2450815|3988|1|751| +2450815|3991|1|626| +2450815|3992|1|162| +2450815|3994|1|974| +2450815|3997|1|11| +2450815|3998|1|802| +2450815|4000|1|562| +2450815|4003|1|549| +2450815|4004|1|32| +2450815|4006|1|120| +2450815|4009|1|741| +2450815|4010|1|569| +2450815|4012|1|749| +2450815|4015|1|48| +2450815|4016|1|344| +2450815|4018|1|742| +2450815|4021|1|435| +2450815|4022|1|625| +2450815|4024|1|749| +2450815|4027|1|31| +2450815|4028|1|852| +2450815|4030|1|671| +2450815|4033|1|768| +2450815|4034|1|783| +2450815|4036|1|333| +2450815|4039|1|171| +2450815|4040|1|131| +2450815|4042|1|787| +2450815|4045|1|68| +2450815|4046|1|490| +2450815|4048|1|54| +2450815|4051|1|955| +2450815|4052|1|416| +2450815|4054|1|20| +2450815|4057|1|67| +2450815|4058|1|387| +2450815|4060|1|679| +2450815|4063|1|122| +2450815|4064|1|931| +2450815|4066|1|738| +2450815|4069|1|425| +2450815|4070|1|931| +2450815|4072|1|385| +2450815|4075|1|684| +2450815|4076|1|311| +2450815|4078|1|527| +2450815|4081|1|98| +2450815|4082|1|53| +2450815|4084|1|941| +2450815|4087|1|914| +2450815|4088|1|554| +2450815|4090|1|146| +2450815|4093|1|724| +2450815|4094|1|605| +2450815|4096|1|321| +2450815|4099|1|542| +2450815|4100|1|137| +2450815|4102|1|36| +2450815|4105|1|| +2450815|4106|1|530| +2450815|4108|1|969| +2450815|4111|1|601| +2450815|4112|1|291| +2450815|4114|1|173| +2450815|4117|1|518| +2450815|4118|1|109| +2450815|4120|1|534| +2450815|4123|1|634| +2450815|4124|1|710| +2450815|4126|1|683| +2450815|4129|1|| +2450815|4130|1|637| +2450815|4132|1|693| +2450815|4135|1|413| +2450815|4136|1|452| +2450815|4138|1|| +2450815|4141|1|87| +2450815|4142|1|| +2450815|4144|1|524| +2450815|4147|1|| +2450815|4148|1|504| +2450815|4150|1|694| +2450815|4153|1|783| +2450815|4154|1|576| +2450815|4156|1|652| +2450815|4159|1|923| +2450815|4160|1|584| +2450815|4162|1|792| +2450815|4165|1|876| +2450815|4166|1|549| +2450815|4168|1|4| +2450815|4171|1|346| +2450815|4172|1|799| +2450815|4174|1|119| +2450815|4177|1|268| +2450815|4178|1|717| +2450815|4180|1|247| +2450815|4183|1|665| +2450815|4184|1|279| +2450815|4186|1|908| +2450815|4189|1|352| +2450815|4190|1|34| +2450815|4192|1|588| +2450815|4195|1|209| +2450815|4196|1|| +2450815|4198|1|376| +2450815|4201|1|413| +2450815|4202|1|627| +2450815|4204|1|732| +2450815|4207|1|446| +2450815|4208|1|392| +2450815|4210|1|670| +2450815|4213|1|555| +2450815|4214|1|747| +2450815|4216|1|881| +2450815|4219|1|928| +2450815|4220|1|303| +2450815|4222|1|507| +2450815|4225|1|480| +2450815|4226|1|330| +2450815|4228|1|530| +2450815|4231|1|810| +2450815|4232|1|770| +2450815|4234|1|59| +2450815|4237|1|727| +2450815|4238|1|751| +2450815|4240|1|| +2450815|4243|1|24| +2450815|4244|1|207| +2450815|4246|1|990| +2450815|4249|1|396| +2450815|4250|1|340| +2450815|4252|1|186| +2450815|4255|1|447| +2450815|4256|1|518| +2450815|4258|1|259| +2450815|4261|1|714| +2450815|4262|1|404| +2450815|4264|1|937| +2450815|4267|1|788| +2450815|4268|1|666| +2450815|4270|1|303| +2450815|4273|1|831| +2450815|4274|1|579| +2450815|4276|1|138| +2450815|4279|1|979| +2450815|4280|1|156| +2450815|4282|1|704| +2450815|4285|1|880| +2450815|4286|1|932| +2450815|4288|1|734| +2450815|4291|1|170| +2450815|4292|1|870| +2450815|4294|1|825| +2450815|4297|1|966| +2450815|4298|1|731| +2450815|4300|1|358| +2450815|4303|1|435| +2450815|4304|1|733| +2450815|4306|1|669| +2450815|4309|1|873| +2450815|4310|1|726| +2450815|4312|1|171| +2450815|4315|1|850| +2450815|4316|1|340| +2450815|4318|1|96| +2450815|4321|1|637| +2450815|4322|1|659| +2450815|4324|1|172| +2450815|4327|1|268| +2450815|4328|1|667| +2450815|4330|1|589| +2450815|4333|1|700| +2450815|4334|1|686| +2450815|4336|1|889| +2450815|4339|1|22| +2450815|4340|1|730| +2450815|4342|1|332| +2450815|4345|1|504| +2450815|4346|1|355| +2450815|4348|1|654| +2450815|4351|1|961| +2450815|4352|1|802| +2450815|4354|1|387| +2450815|4357|1|572| +2450815|4358|1|438| +2450815|4360|1|730| +2450815|4363|1|59| +2450815|4364|1|| +2450815|4366|1|848| +2450815|4369|1|87| +2450815|4370|1|985| +2450815|4372|1|485| +2450815|4375|1|608| +2450815|4376|1|| +2450815|4378|1|629| +2450815|4381|1|952| +2450815|4382|1|257| +2450815|4384|1|117| +2450815|4387|1|| +2450815|4388|1|706| +2450815|4390|1|735| +2450815|4393|1|79| +2450815|4394|1|737| +2450815|4396|1|182| +2450815|4399|1|622| +2450815|4400|1|168| +2450815|4402|1|918| +2450815|4405|1|175| +2450815|4406|1|714| +2450815|4408|1|940| +2450815|4411|1|253| +2450815|4412|1|535| +2450815|4414|1|178| +2450815|4417|1|74| +2450815|4418|1|135| +2450815|4420|1|| +2450815|4423|1|989| +2450815|4424|1|937| +2450815|4426|1|695| +2450815|4429|1|| +2450815|4430|1|731| +2450815|4432|1|741| +2450815|4435|1|119| +2450815|4436|1|636| +2450815|4438|1|957| +2450815|4441|1|825| +2450815|4442|1|966| +2450815|4444|1|314| +2450815|4447|1|193| +2450815|4448|1|351| +2450815|4450|1|644| +2450815|4453|1|19| +2450815|4454|1|236| +2450815|4456|1|105| +2450815|4459|1|255| +2450815|4460|1|571| +2450815|4462|1|0| +2450815|4465|1|544| +2450815|4466|1|889| +2450815|4468|1|276| +2450815|4471|1|261| +2450815|4472|1|14| +2450815|4474|1|591| +2450815|4477|1|736| +2450815|4478|1|743| +2450815|4480|1|856| +2450815|4483|1|919| +2450815|4484|1|401| +2450815|4486|1|425| +2450815|4489|1|84| +2450815|4490|1|197| +2450815|4492|1|97| +2450815|4495|1|551| +2450815|4496|1|848| +2450815|4498|1|558| +2450815|4501|1|799| +2450815|4502|1|48| +2450815|4504|1|542| +2450815|4507|1|340| +2450815|4508|1|436| +2450815|4510|1|621| +2450815|4513|1|367| +2450815|4514|1|203| +2450815|4516|1|228| +2450815|4519|1|225| +2450815|4520|1|279| +2450815|4522|1|308| +2450815|4525|1|122| +2450815|4526|1|678| +2450815|4528|1|27| +2450815|4531|1|396| +2450815|4532|1|195| +2450815|4534|1|870| +2450815|4537|1|545| +2450815|4538|1|680| +2450815|4540|1|238| +2450815|4543|1|873| +2450815|4544|1|553| +2450815|4546|1|756| +2450815|4549|1|274| +2450815|4550|1|517| +2450815|4552|1|928| +2450815|4555|1|587| +2450815|4556|1|131| +2450815|4558|1|133| +2450815|4561|1|858| +2450815|4562|1|| +2450815|4564|1|228| +2450815|4567|1|703| +2450815|4568|1|| +2450815|4570|1|164| +2450815|4573|1|19| +2450815|4574|1|383| +2450815|4576|1|824| +2450815|4579|1|456| +2450815|4580|1|719| +2450815|4582|1|| +2450815|4585|1|173| +2450815|4586|1|629| +2450815|4588|1|929| +2450815|4591|1|369| +2450815|4592|1|| +2450815|4594|1|563| +2450815|4597|1|481| +2450815|4598|1|950| +2450815|4600|1|957| +2450815|4603|1|958| +2450815|4604|1|892| +2450815|4606|1|878| +2450815|4609|1|196| +2450815|4610|1|800| +2450815|4612|1|222| +2450815|4615|1|627| +2450815|4616|1|683| +2450815|4618|1|75| +2450815|4621|1|283| +2450815|4622|1|267| +2450815|4624|1|626| +2450815|4627|1|111| +2450815|4628|1|419| +2450815|4630|1|235| +2450815|4633|1|266| +2450815|4634|1|650| +2450815|4636|1|909| +2450815|4639|1|755| +2450815|4640|1|59| +2450815|4642|1|506| +2450815|4645|1|512| +2450815|4646|1|667| +2450815|4648|1|835| +2450815|4651|1|923| +2450815|4652|1|72| +2450815|4654|1|783| +2450815|4657|1|965| +2450815|4658|1|404| +2450815|4660|1|967| +2450815|4663|1|164| +2450815|4664|1|618| +2450815|4666|1|209| +2450815|4669|1|139| +2450815|4670|1|474| +2450815|4672|1|328| +2450815|4675|1|738| +2450815|4676|1|874| +2450815|4678|1|923| +2450815|4681|1|575| +2450815|4682|1|799| +2450815|4684|1|258| +2450815|4687|1|403| +2450815|4688|1|503| +2450815|4690|1|| +2450815|4693|1|649| +2450815|4694|1|704| +2450815|4696|1|243| +2450815|4699|1|866| +2450815|4700|1|795| +2450815|4702|1|624| +2450815|4705|1|42| +2450815|4706|1|987| +2450815|4708|1|222| +2450815|4711|1|571| +2450815|4712|1|23| +2450815|4714|1|964| +2450815|4717|1|289| +2450815|4718|1|972| +2450815|4720|1|520| +2450815|4723|1|706| +2450815|4724|1|9| +2450815|4726|1|540| +2450815|4729|1|145| +2450815|4730|1|767| +2450815|4732|1|560| +2450815|4735|1|811| +2450815|4736|1|289| +2450815|4738|1|590| +2450815|4741|1|114| +2450815|4742|1|862| +2450815|4744|1|128| +2450815|4747|1|458| +2450815|4748|1|58| +2450815|4750|1|534| +2450815|4753|1|| +2450815|4754|1|563| +2450815|4756|1|| +2450815|4759|1|| +2450815|4760|1|839| +2450815|4762|1|717| +2450815|4765|1|276| +2450815|4766|1|804| +2450815|4768|1|546| +2450815|4771|1|988| +2450815|4772|1|| +2450815|4774|1|326| +2450815|4777|1|970| +2450815|4778|1|821| +2450815|4780|1|472| +2450815|4783|1|405| +2450815|4784|1|900| +2450815|4786|1|117| +2450815|4789|1|133| +2450815|4790|1|342| +2450815|4792|1|338| +2450815|4795|1|481| +2450815|4796|1|610| +2450815|4798|1|861| +2450815|4801|1|756| +2450815|4802|1|947| +2450815|4804|1|521| +2450815|4807|1|965| +2450815|4808|1|29| +2450815|4810|1|131| +2450815|4813|1|913| +2450815|4814|1|1| +2450815|4816|1|294| +2450815|4819|1|427| +2450815|4820|1|934| +2450815|4822|1|771| +2450815|4825|1|672| +2450815|4826|1|756| +2450815|4828|1|970| +2450815|4831|1|563| +2450815|4832|1|423| +2450815|4834|1|631| +2450815|4837|1|374| +2450815|4838|1|389| +2450815|4840|1|774| +2450815|4843|1|| +2450815|4844|1|| +2450815|4846|1|653| +2450815|4849|1|225| +2450815|4850|1|422| +2450815|4852|1|376| +2450815|4855|1|115| +2450815|4856|1|258| +2450815|4858|1|23| +2450815|4861|1|124| +2450815|4862|1|90| +2450815|4864|1|| +2450815|4867|1|743| +2450815|4868|1|| +2450815|4870|1|526| +2450815|4873|1|293| +2450815|4874|1|366| +2450815|4876|1|25| +2450815|4879|1|301| +2450815|4880|1|826| +2450815|4882|1|138| +2450815|4885|1|737| +2450815|4886|1|| +2450815|4888|1|906| +2450815|4891|1|125| +2450815|4892|1|22| +2450815|4894|1|| +2450815|4897|1|| +2450815|4898|1|| +2450815|4900|1|530| +2450815|4903|1|874| +2450815|4904|1|| +2450815|4906|1|383| +2450815|4909|1|830| +2450815|4910|1|106| +2450815|4912|1|418| +2450815|4915|1|27| +2450815|4916|1|614| +2450815|4918|1|262| +2450815|4921|1|174| +2450815|4922|1|71| +2450815|4924|1|265| +2450815|4927|1|30| +2450815|4928|1|721| +2450815|4930|1|27| +2450815|4933|1|482| +2450815|4934|1|107| +2450815|4936|1|797| +2450815|4939|1|| +2450815|4940|1|895| +2450815|4942|1|858| +2450815|4945|1|887| +2450815|4946|1|567| +2450815|4948|1|655| +2450815|4951|1|546| +2450815|4952|1|155| +2450815|4954|1|822| +2450815|4957|1|| +2450815|4958|1|404| +2450815|4960|1|| +2450815|4963|1|| +2450815|4964|1|614| +2450815|4966|1|291| +2450815|4969|1|295| +2450815|4970|1|828| +2450815|4972|1|398| +2450815|4975|1|325| +2450815|4976|1|140| +2450815|4978|1|774| +2450815|4981|1|830| +2450815|4982|1|602| +2450815|4984|1|| +2450815|4987|1|233| +2450815|4988|1|801| +2450815|4990|1|764| +2450815|4993|1|121| +2450815|4994|1|523| +2450815|4996|1|693| +2450815|4999|1|166| +2450815|5000|1|606| +2450815|5002|1|499| +2450815|5005|1|274| +2450815|5006|1|226| +2450815|5008|1|758| +2450815|5011|1|700| +2450815|5012|1|361| +2450815|5014|1|| +2450815|5017|1|71| +2450815|5018|1|30| +2450815|5020|1|496| +2450815|5023|1|817| +2450815|5024|1|490| +2450815|5026|1|784| +2450815|5029|1|519| +2450815|5030|1|281| +2450815|5032|1|652| +2450815|5035|1|961| +2450815|5036|1|763| +2450815|5038|1|822| +2450815|5041|1|239| +2450815|5042|1|582| +2450815|5044|1|56| +2450815|5047|1|254| +2450815|5048|1|89| +2450815|5050|1|159| +2450815|5053|1|808| +2450815|5054|1|105| +2450815|5056|1|427| +2450815|5059|1|791| +2450815|5060|1|356| +2450815|5062|1|113| +2450815|5065|1|905| +2450815|5066|1|| +2450815|5068|1|123| +2450815|5071|1|441| +2450815|5072|1|494| +2450815|5074|1|716| +2450815|5077|1|24| +2450815|5078|1|325| +2450815|5080|1|747| +2450815|5083|1|381| +2450815|5084|1|1| +2450815|5086|1|941| +2450815|5089|1|792| +2450815|5090|1|426| +2450815|5092|1|126| +2450815|5095|1|50| +2450815|5096|1|833| +2450815|5098|1|564| +2450815|5101|1|624| +2450815|5102|1|143| +2450815|5104|1|910| +2450815|5107|1|886| +2450815|5108|1|196| +2450815|5110|1|184| +2450815|5113|1|| +2450815|5114|1|164| +2450815|5116|1|174| +2450815|5119|1|582| +2450815|5120|1|689| +2450815|5122|1|930| +2450815|5125|1|693| +2450815|5126|1|195| +2450815|5128|1|| +2450815|5131|1|309| +2450815|5132|1|92| +2450815|5134|1|930| +2450815|5137|1|567| +2450815|5138|1|446| +2450815|5140|1|491| +2450815|5143|1|767| +2450815|5144|1|422| +2450815|5146|1|490| +2450815|5149|1|601| +2450815|5150|1|267| +2450815|5152|1|973| +2450815|5155|1|730| +2450815|5156|1|748| +2450815|5158|1|532| +2450815|5161|1|448| +2450815|5162|1|889| +2450815|5164|1|2| +2450815|5167|1|679| +2450815|5168|1|540| +2450815|5170|1|482| +2450815|5173|1|555| +2450815|5174|1|790| +2450815|5176|1|28| +2450815|5179|1|808| +2450815|5180|1|898| +2450815|5182|1|967| +2450815|5185|1|808| +2450815|5186|1|88| +2450815|5188|1|453| +2450815|5191|1|410| +2450815|5192|1|217| +2450815|5194|1|78| +2450815|5197|1|822| +2450815|5198|1|852| +2450815|5200|1|58| +2450815|5203|1|798| +2450815|5204|1|812| +2450815|5206|1|77| +2450815|5209|1|514| +2450815|5210|1|792| +2450815|5212|1|| +2450815|5215|1|883| +2450815|5216|1|296| +2450815|5218|1|839| +2450815|5221|1|219| +2450815|5222|1|269| +2450815|5224|1|518| +2450815|5227|1|| +2450815|5228|1|489| +2450815|5230|1|345| +2450815|5233|1|53| +2450815|5234|1|653| +2450815|5236|1|588| +2450815|5239|1|351| +2450815|5240|1|1| +2450815|5242|1|| +2450815|5245|1|128| +2450815|5246|1|506| +2450815|5248|1|97| +2450815|5251|1|916| +2450815|5252|1|707| +2450815|5254|1|859| +2450815|5257|1|816| +2450815|5258|1|743| +2450815|5260|1|621| +2450815|5263|1|138| +2450815|5264|1|669| +2450815|5266|1|27| +2450815|5269|1|584| +2450815|5270|1|169| +2450815|5272|1|645| +2450815|5275|1|549| +2450815|5276|1|341| +2450815|5278|1|5| +2450815|5281|1|775| +2450815|5282|1|34| +2450815|5284|1|995| +2450815|5287|1|133| +2450815|5288|1|462| +2450815|5290|1|679| +2450815|5293|1|310| +2450815|5294|1|47| +2450815|5296|1|| +2450815|5299|1|376| +2450815|5300|1|100| +2450815|5302|1|501| +2450815|5305|1|290| +2450815|5306|1|654| +2450815|5308|1|586| +2450815|5311|1|347| +2450815|5312|1|496| +2450815|5314|1|754| +2450815|5317|1|487| +2450815|5318|1|812| +2450815|5320|1|528| +2450815|5323|1|199| +2450815|5324|1|545| +2450815|5326|1|290| +2450815|5329|1|364| +2450815|5330|1|686| +2450815|5332|1|795| +2450815|5335|1|526| +2450815|5336|1|717| +2450815|5338|1|251| +2450815|5341|1|118| +2450815|5342|1|519| +2450815|5344|1|427| +2450815|5347|1|245| +2450815|5348|1|695| +2450815|5350|1|162| +2450815|5353|1|228| +2450815|5354|1|613| +2450815|5356|1|842| +2450815|5359|1|964| +2450815|5360|1|342| +2450815|5362|1|28| +2450815|5365|1|186| +2450815|5366|1|928| +2450815|5368|1|173| +2450815|5371|1|130| +2450815|5372|1|352| +2450815|5374|1|595| +2450815|5377|1|953| +2450815|5378|1|665| +2450815|5380|1|731| +2450815|5383|1|| +2450815|5384|1|752| +2450815|5386|1|640| +2450815|5389|1|662| +2450815|5390|1|740| +2450815|5392|1|569| +2450815|5395|1|413| +2450815|5396|1|| +2450815|5398|1|899| +2450815|5401|1|31| +2450815|5402|1|709| +2450815|5404|1|816| +2450815|5407|1|95| +2450815|5408|1|6| +2450815|5410|1|463| +2450815|5413|1|957| +2450815|5414|1|440| +2450815|5416|1|886| +2450815|5419|1|382| +2450815|5420|1|700| +2450815|5422|1|378| +2450815|5425|1|41| +2450815|5426|1|490| +2450815|5428|1|418| +2450815|5431|1|634| +2450815|5432|1|111| +2450815|5434|1|533| +2450815|5437|1|799| +2450815|5438|1|857| +2450815|5440|1|204| +2450815|5443|1|480| +2450815|5444|1|995| +2450815|5446|1|620| +2450815|5449|1|663| +2450815|5450|1|843| +2450815|5452|1|620| +2450815|5455|1|318| +2450815|5456|1|57| +2450815|5458|1|416| +2450815|5461|1|875| +2450815|5462|1|110| +2450815|5464|1|183| +2450815|5467|1|184| +2450815|5468|1|286| +2450815|5470|1|51| +2450815|5473|1|118| +2450815|5474|1|115| +2450815|5476|1|312| +2450815|5479|1|889| +2450815|5480|1|776| +2450815|5482|1|139| +2450815|5485|1|538| +2450815|5486|1|582| +2450815|5488|1|106| +2450815|5491|1|345| +2450815|5492|1|| +2450815|5494|1|| +2450815|5497|1|782| +2450815|5498|1|237| +2450815|5500|1|245| +2450815|5503|1|951| +2450815|5504|1|703| +2450815|5506|1|530| +2450815|5509|1|127| +2450815|5510|1|12| +2450815|5512|1|552| +2450815|5515|1|521| +2450815|5516|1|876| +2450815|5518|1|847| +2450815|5521|1|83| +2450815|5522|1|966| +2450815|5524|1|99| +2450815|5527|1|59| +2450815|5528|1|958| +2450815|5530|1|198| +2450815|5533|1|899| +2450815|5534|1|585| +2450815|5536|1|467| +2450815|5539|1|546| +2450815|5540|1|326| +2450815|5542|1|419| +2450815|5545|1|654| +2450815|5546|1|391| +2450815|5548|1|369| +2450815|5551|1|609| +2450815|5552|1|750| +2450815|5554|1|421| +2450815|5557|1|584| +2450815|5558|1|606| +2450815|5560|1|105| +2450815|5563|1|449| +2450815|5564|1|146| +2450815|5566|1|475| +2450815|5569|1|89| +2450815|5570|1|624| +2450815|5572|1|689| +2450815|5575|1|369| +2450815|5576|1|6| +2450815|5578|1|353| +2450815|5581|1|424| +2450815|5582|1|98| +2450815|5584|1|184| +2450815|5587|1|928| +2450815|5588|1|967| +2450815|5590|1|36| +2450815|5593|1|414| +2450815|5594|1|416| +2450815|5596|1|908| +2450815|5599|1|484| +2450815|5600|1|335| +2450815|5602|1|698| +2450815|5605|1|615| +2450815|5606|1|51| +2450815|5608|1|452| +2450815|5611|1|971| +2450815|5612|1|302| +2450815|5614|1|982| +2450815|5617|1|641| +2450815|5618|1|5| +2450815|5620|1|613| +2450815|5623|1|8| +2450815|5624|1|658| +2450815|5626|1|565| +2450815|5629|1|227| +2450815|5630|1|795| +2450815|5632|1|85| +2450815|5635|1|789| +2450815|5636|1|360| +2450815|5638|1|460| +2450815|5641|1|504| +2450815|5642|1|19| +2450815|5644|1|593| +2450815|5647|1|922| +2450815|5648|1|580| +2450815|5650|1|| +2450815|5653|1|51| +2450815|5654|1|149| +2450815|5656|1|254| +2450815|5659|1|561| +2450815|5660|1|871| +2450815|5662|1|| +2450815|5665|1|488| +2450815|5666|1|582| +2450815|5668|1|711| +2450815|5671|1|427| +2450815|5672|1|416| +2450815|5674|1|776| +2450815|5677|1|791| +2450815|5678|1|622| +2450815|5680|1|739| +2450815|5683|1|441| +2450815|5684|1|810| +2450815|5686|1|| +2450815|5689|1|258| +2450815|5690|1|305| +2450815|5692|1|| +2450815|5695|1|596| +2450815|5696|1|73| +2450815|5698|1|459| +2450815|5701|1|980| +2450815|5702|1|237| +2450815|5704|1|473| +2450815|5707|1|747| +2450815|5708|1|768| +2450815|5710|1|548| +2450815|5713|1|131| +2450815|5714|1|678| +2450815|5716|1|397| +2450815|5719|1|802| +2450815|5720|1|211| +2450815|5722|1|398| +2450815|5725|1|207| +2450815|5726|1|922| +2450815|5728|1|124| +2450815|5731|1|62| +2450815|5732|1|651| +2450815|5734|1|594| +2450815|5737|1|956| +2450815|5738|1|185| +2450815|5740|1|906| +2450815|5743|1|34| +2450815|5744|1|| +2450815|5746|1|295| +2450815|5749|1|894| +2450815|5750|1|575| +2450815|5752|1|313| +2450815|5755|1|34| +2450815|5756|1|620| +2450815|5758|1|971| +2450815|5761|1|369| +2450815|5762|1|554| +2450815|5764|1|490| +2450815|5767|1|607| +2450815|5768|1|33| +2450815|5770|1|11| +2450815|5773|1|902| +2450815|5774|1|917| +2450815|5776|1|364| +2450815|5779|1|7| +2450815|5780|1|793| +2450815|5782|1|457| +2450815|5785|1|580| +2450815|5786|1|604| +2450815|5788|1|476| +2450815|5791|1|907| +2450815|5792|1|70| +2450815|5794|1|854| +2450815|5797|1|216| +2450815|5798|1|518| +2450815|5800|1|161| +2450815|5803|1|358| +2450815|5804|1|170| +2450815|5806|1|| +2450815|5809|1|111| +2450815|5810|1|546| +2450815|5812|1|303| +2450815|5815|1|138| +2450815|5816|1|721| +2450815|5818|1|947| +2450815|5821|1|606| +2450815|5822|1|24| +2450815|5824|1|149| +2450815|5827|1|816| +2450815|5828|1|251| +2450815|5830|1|349| +2450815|5833|1|551| +2450815|5834|1|209| +2450815|5836|1|495| +2450815|5839|1|57| +2450815|5840|1|740| +2450815|5842|1|862| +2450815|5845|1|292| +2450815|5846|1|800| +2450815|5848|1|43| +2450815|5851|1|551| +2450815|5852|1|391| +2450815|5854|1|923| +2450815|5857|1|576| +2450815|5858|1|957| +2450815|5860|1|365| +2450815|5863|1|817| +2450815|5864|1|18| +2450815|5866|1|510| +2450815|5869|1|646| +2450815|5870|1|855| +2450815|5872|1|569| +2450815|5875|1|846| +2450815|5876|1|989| +2450815|5878|1|815| +2450815|5881|1|778| +2450815|5882|1|616| +2450815|5884|1|| +2450815|5887|1|171| +2450815|5888|1|225| +2450815|5890|1|922| +2450815|5893|1|46| +2450815|5894|1|| +2450815|5896|1|0| +2450815|5899|1|457| +2450815|5900|1|484| +2450815|5902|1|496| +2450815|5905|1|762| +2450815|5906|1|854| +2450815|5908|1|523| +2450815|5911|1|349| +2450815|5912|1|144| +2450815|5914|1|859| +2450815|5917|1|299| +2450815|5918|1|123| +2450815|5920|1|859| +2450815|5923|1|| +2450815|5924|1|387| +2450815|5926|1|633| +2450815|5929|1|40| +2450815|5930|1|644| +2450815|5932|1|897| +2450815|5935|1|27| +2450815|5936|1|454| +2450815|5938|1|208| +2450815|5941|1|132| +2450815|5942|1|124| +2450815|5944|1|352| +2450815|5947|1|531| +2450815|5948|1|724| +2450815|5950|1|644| +2450815|5953|1|838| +2450815|5954|1|578| +2450815|5956|1|265| +2450815|5959|1|| +2450815|5960|1|972| +2450815|5962|1|148| +2450815|5965|1|224| +2450815|5966|1|209| +2450815|5968|1|| +2450815|5971|1|485| +2450815|5972|1|655| +2450815|5974|1|405| +2450815|5977|1|397| +2450815|5978|1|60| +2450815|5980|1|955| +2450815|5983|1|31| +2450815|5984|1|372| +2450815|5986|1|583| +2450815|5989|1|349| +2450815|5990|1|73| +2450815|5992|1|825| +2450815|5995|1|887| +2450815|5996|1|737| +2450815|5998|1|| +2450815|6001|1|804| +2450815|6002|1|895| +2450815|6004|1|815| +2450815|6007|1|732| +2450815|6008|1|376| +2450815|6010|1|825| +2450815|6013|1|932| +2450815|6014|1|504| +2450815|6016|1|462| +2450815|6019|1|678| +2450815|6020|1|964| +2450815|6022|1|84| +2450815|6025|1|790| +2450815|6026|1|858| +2450815|6028|1|494| +2450815|6031|1|855| +2450815|6032|1|845| +2450815|6034|1|402| +2450815|6037|1|413| +2450815|6038|1|879| +2450815|6040|1|238| +2450815|6043|1|373| +2450815|6044|1|217| +2450815|6046|1|471| +2450815|6049|1|535| +2450815|6050|1|439| +2450815|6052|1|719| +2450815|6055|1|117| +2450815|6056|1|586| +2450815|6058|1|774| +2450815|6061|1|884| +2450815|6062|1|481| +2450815|6064|1|324| +2450815|6067|1|969| +2450815|6068|1|537| +2450815|6070|1|192| +2450815|6073|1|52| +2450815|6074|1|129| +2450815|6076|1|31| +2450815|6079|1|316| +2450815|6080|1|910| +2450815|6082|1|817| +2450815|6085|1|393| +2450815|6086|1|400| +2450815|6088|1|500| +2450815|6091|1|484| +2450815|6092|1|387| +2450815|6094|1|59| +2450815|6097|1|363| +2450815|6098|1|384| +2450815|6100|1|| +2450815|6103|1|733| +2450815|6104|1|693| +2450815|6106|1|931| +2450815|6109|1|840| +2450815|6110|1|853| +2450815|6112|1|548| +2450815|6115|1|45| +2450815|6116|1|124| +2450815|6118|1|289| +2450815|6121|1|575| +2450815|6122|1|312| +2450815|6124|1|442| +2450815|6127|1|110| +2450815|6128|1|36| +2450815|6130|1|152| +2450815|6133|1|| +2450815|6134|1|687| +2450815|6136|1|990| +2450815|6139|1|367| +2450815|6140|1|257| +2450815|6142|1|705| +2450815|6145|1|16| +2450815|6146|1|182| +2450815|6148|1|390| +2450815|6151|1|| +2450815|6152|1|479| +2450815|6154|1|683| +2450815|6157|1|703| +2450815|6158|1|157| +2450815|6160|1|763| +2450815|6163|1|560| +2450815|6164|1|730| +2450815|6166|1|591| +2450815|6169|1|170| +2450815|6170|1|750| +2450815|6172|1|417| +2450815|6175|1|850| +2450815|6176|1|538| +2450815|6178|1|297| +2450815|6181|1|411| +2450815|6182|1|199| +2450815|6184|1|854| +2450815|6187|1|300| +2450815|6188|1|296| +2450815|6190|1|975| +2450815|6193|1|11| +2450815|6194|1|| +2450815|6196|1|500| +2450815|6199|1|734| +2450815|6200|1|410| +2450815|6202|1|751| +2450815|6205|1|485| +2450815|6206|1|338| +2450815|6208|1|719| +2450815|6211|1|| +2450815|6212|1|604| +2450815|6214|1|858| +2450815|6217|1|102| +2450815|6218|1|528| +2450815|6220|1|896| +2450815|6223|1|501| +2450815|6224|1|939| +2450815|6226|1|34| +2450815|6229|1|961| +2450815|6230|1|926| +2450815|6232|1|610| +2450815|6235|1|510| +2450815|6236|1|579| +2450815|6238|1|100| +2450815|6241|1|893| +2450815|6242|1|736| +2450815|6244|1|925| +2450815|6247|1|501| +2450815|6248|1|397| +2450815|6250|1|147| +2450815|6253|1|577| +2450815|6254|1|1| +2450815|6256|1|748| +2450815|6259|1|112| +2450815|6260|1|896| +2450815|6262|1|132| +2450815|6265|1|917| +2450815|6266|1|344| +2450815|6268|1|830| +2450815|6271|1|46| +2450815|6272|1|155| +2450815|6274|1|552| +2450815|6277|1|482| +2450815|6278|1|15| +2450815|6280|1|398| +2450815|6283|1|170| +2450815|6284|1|727| +2450815|6286|1|723| +2450815|6289|1|533| +2450815|6290|1|321| +2450815|6292|1|803| +2450815|6295|1|| +2450815|6296|1|765| +2450815|6298|1|591| +2450815|6301|1|207| +2450815|6302|1|755| +2450815|6304|1|147| +2450815|6307|1|994| +2450815|6308|1|800| +2450815|6310|1|380| +2450815|6313|1|392| +2450815|6314|1|886| +2450815|6316|1|| +2450815|6319|1|946| +2450815|6320|1|684| +2450815|6322|1|717| +2450815|6325|1|948| +2450815|6326|1|602| +2450815|6328|1|607| +2450815|6331|1|970| +2450815|6332|1|337| +2450815|6334|1|12| +2450815|6337|1|882| +2450815|6338|1|485| +2450815|6340|1|627| +2450815|6343|1|510| +2450815|6344|1|162| +2450815|6346|1|618| +2450815|6349|1|832| +2450815|6350|1|869| +2450815|6352|1|880| +2450815|6355|1|850| +2450815|6356|1|965| +2450815|6358|1|305| +2450815|6361|1|688| +2450815|6362|1|55| +2450815|6364|1|715| +2450815|6367|1|960| +2450815|6368|1|361| +2450815|6370|1|796| +2450815|6373|1|595| +2450815|6374|1|132| +2450815|6376|1|397| +2450815|6379|1|102| +2450815|6380|1|925| +2450815|6382|1|335| +2450815|6385|1|269| +2450815|6386|1|440| +2450815|6388|1|841| +2450815|6391|1|202| +2450815|6392|1|700| +2450815|6394|1|988| +2450815|6397|1|438| +2450815|6398|1|376| +2450815|6400|1|33| +2450815|6403|1|730| +2450815|6404|1|162| +2450815|6406|1|155| +2450815|6409|1|435| +2450815|6410|1|507| +2450815|6412|1|558| +2450815|6415|1|836| +2450815|6416|1|455| +2450815|6418|1|679| +2450815|6421|1|337| +2450815|6422|1|238| +2450815|6424|1|732| +2450815|6427|1|851| +2450815|6428|1|672| +2450815|6430|1|987| +2450815|6433|1|342| +2450815|6434|1|334| +2450815|6436|1|134| +2450815|6439|1|950| +2450815|6440|1|471| +2450815|6442|1|990| +2450815|6445|1|50| +2450815|6446|1|709| +2450815|6448|1|763| +2450815|6451|1|862| +2450815|6452|1|991| +2450815|6454|1|218| +2450815|6457|1|701| +2450815|6458|1|477| +2450815|6460|1|729| +2450815|6463|1|745| +2450815|6464|1|500| +2450815|6466|1|| +2450815|6469|1|446| +2450815|6470|1|916| +2450815|6472|1|570| +2450815|6475|1|916| +2450815|6476|1|| +2450815|6478|1|571| +2450815|6481|1|194| +2450815|6482|1|383| +2450815|6484|1|729| +2450815|6487|1|278| +2450815|6488|1|154| +2450815|6490|1|224| +2450815|6493|1|491| +2450815|6494|1|| +2450815|6496|1|992| +2450815|6499|1|7| +2450815|6500|1|571| +2450815|6502|1|339| +2450815|6505|1|422| +2450815|6506|1|350| +2450815|6508|1|229| +2450815|6511|1|803| +2450815|6512|1|959| +2450815|6514|1|87| +2450815|6517|1|217| +2450815|6518|1|725| +2450815|6520|1|874| +2450815|6523|1|989| +2450815|6524|1|626| +2450815|6526|1|355| +2450815|6529|1|688| +2450815|6530|1|8| +2450815|6532|1|571| +2450815|6535|1|1000| +2450815|6536|1|441| +2450815|6538|1|806| +2450815|6541|1|819| +2450815|6542|1|200| +2450815|6544|1|| +2450815|6547|1|199| +2450815|6548|1|189| +2450815|6550|1|135| +2450815|6553|1|78| +2450815|6554|1|50| +2450815|6556|1|734| +2450815|6559|1|773| +2450815|6560|1|2| +2450815|6562|1|738| +2450815|6565|1|36| +2450815|6566|1|790| +2450815|6568|1|253| +2450815|6571|1|214| +2450815|6572|1|574| +2450815|6574|1|92| +2450815|6577|1|959| +2450815|6578|1|685| +2450815|6580|1|502| +2450815|6583|1|569| +2450815|6584|1|318| +2450815|6586|1|748| +2450815|6589|1|476| +2450815|6590|1|559| +2450815|6592|1|750| +2450815|6595|1|114| +2450815|6596|1|271| +2450815|6598|1|56| +2450815|6601|1|137| +2450815|6602|1|894| +2450815|6604|1|616| +2450815|6607|1|257| +2450815|6608|1|521| +2450815|6610|1|110| +2450815|6613|1|435| +2450815|6614|1|771| +2450815|6616|1|995| +2450815|6619|1|| +2450815|6620|1|165| +2450815|6622|1|739| +2450815|6625|1|937| +2450815|6626|1|836| +2450815|6628|1|468| +2450815|6631|1|852| +2450815|6632|1|449| +2450815|6634|1|725| +2450815|6637|1|368| +2450815|6638|1|710| +2450815|6640|1|684| +2450815|6643|1|186| +2450815|6644|1|| +2450815|6646|1|886| +2450815|6649|1|447| +2450815|6650|1|| +2450815|6652|1|433| +2450815|6655|1|240| +2450815|6656|1|463| +2450815|6658|1|846| +2450815|6661|1|458| +2450815|6662|1|454| +2450815|6664|1|652| +2450815|6667|1|124| +2450815|6668|1|565| +2450815|6670|1|512| +2450815|6673|1|940| +2450815|6674|1|340| +2450815|6676|1|574| +2450815|6679|1|31| +2450815|6680|1|183| +2450815|6682|1|959| +2450815|6685|1|27| +2450815|6686|1|595| +2450815|6688|1|173| +2450815|6691|1|911| +2450815|6692|1|930| +2450815|6694|1|994| +2450815|6697|1|601| +2450815|6698|1|852| +2450815|6700|1|7| +2450815|6703|1|818| +2450815|6704|1|876| +2450815|6706|1|253| +2450815|6709|1|| +2450815|6710|1|731| +2450815|6712|1|194| +2450815|6715|1|500| +2450815|6716|1|821| +2450815|6718|1|188| +2450815|6721|1|491| +2450815|6722|1|759| +2450815|6724|1|746| +2450815|6727|1|71| +2450815|6728|1|351| +2450815|6730|1|478| +2450815|6733|1|432| +2450815|6734|1|761| +2450815|6736|1|152| +2450815|6739|1|220| +2450815|6740|1|678| +2450815|6742|1|749| +2450815|6745|1|556| +2450815|6746|1|209| +2450815|6748|1|190| +2450815|6751|1|960| +2450815|6752|1|347| +2450815|6754|1|101| +2450815|6757|1|936| +2450815|6758|1|81| +2450815|6760|1|355| +2450815|6763|1|279| +2450815|6764|1|467| +2450815|6766|1|625| +2450815|6769|1|749| +2450815|6770|1|| +2450815|6772|1|76| +2450815|6775|1|454| +2450815|6776|1|747| +2450815|6778|1|383| +2450815|6781|1|918| +2450815|6782|1|450| +2450815|6784|1|764| +2450815|6787|1|329| +2450815|6788|1|471| +2450815|6790|1|589| +2450815|6793|1|713| +2450815|6794|1|461| +2450815|6796|1|66| +2450815|6799|1|539| +2450815|6800|1|816| +2450815|6802|1|838| +2450815|6805|1|172| +2450815|6806|1|129| +2450815|6808|1|647| +2450815|6811|1|918| +2450815|6812|1|582| +2450815|6814|1|469| +2450815|6817|1|589| +2450815|6818|1|566| +2450815|6820|1|145| +2450815|6823|1|610| +2450815|6824|1|789| +2450815|6826|1|3| +2450815|6829|1|886| +2450815|6830|1|913| +2450815|6832|1|310| +2450815|6835|1|715| +2450815|6836|1|831| +2450815|6838|1|711| +2450815|6841|1|203| +2450815|6842|1|740| +2450815|6844|1|869| +2450815|6847|1|241| +2450815|6848|1|415| +2450815|6850|1|299| +2450815|6853|1|630| +2450815|6854|1|478| +2450815|6856|1|981| +2450815|6859|1|953| +2450815|6860|1|750| +2450815|6862|1|969| +2450815|6865|1|419| +2450815|6866|1|856| +2450815|6868|1|710| +2450815|6871|1|123| +2450815|6872|1|687| +2450815|6874|1|163| +2450815|6877|1|78| +2450815|6878|1|916| +2450815|6880|1|247| +2450815|6883|1|298| +2450815|6884|1|287| +2450815|6886|1|878| +2450815|6889|1|736| +2450815|6890|1|439| +2450815|6892|1|805| +2450815|6895|1|457| +2450815|6896|1|312| +2450815|6898|1|34| +2450815|6901|1|950| +2450815|6902|1|727| +2450815|6904|1|448| +2450815|6907|1|670| +2450815|6908|1|222| +2450815|6910|1|737| +2450815|6913|1|4| +2450815|6914|1|725| +2450815|6916|1|561| +2450815|6919|1|8| +2450815|6920|1|| +2450815|6922|1|605| +2450815|6925|1|247| +2450815|6926|1|618| +2450815|6928|1|796| +2450815|6931|1|172| +2450815|6932|1|477| +2450815|6934|1|767| +2450815|6937|1|94| +2450815|6938|1|121| +2450815|6940|1|249| +2450815|6943|1|356| +2450815|6944|1|824| +2450815|6946|1|46| +2450815|6949|1|130| +2450815|6950|1|427| +2450815|6952|1|375| +2450815|6955|1|734| +2450815|6956|1|826| +2450815|6958|1|475| +2450815|6961|1|68| +2450815|6962|1|174| +2450815|6964|1|| +2450815|6967|1|843| +2450815|6968|1|888| +2450815|6970|1|908| +2450815|6973|1|736| +2450815|6974|1|39| +2450815|6976|1|327| +2450815|6979|1|915| +2450815|6980|1|186| +2450815|6982|1|113| +2450815|6985|1|798| +2450815|6986|1|962| +2450815|6988|1|126| +2450815|6991|1|997| +2450815|6992|1|907| +2450815|6994|1|788| +2450815|6997|1|669| +2450815|6998|1|721| +2450815|7000|1|39| +2450815|7003|1|670| +2450815|7004|1|936| +2450815|7006|1|24| +2450815|7009|1|170| +2450815|7010|1|386| +2450815|7012|1|904| +2450815|7015|1|602| +2450815|7016|1|820| +2450815|7018|1|| +2450815|7021|1|494| +2450815|7022|1|967| +2450815|7024|1|501| +2450815|7027|1|329| +2450815|7028|1|768| +2450815|7030|1|878| +2450815|7033|1|386| +2450815|7034|1|881| +2450815|7036|1|329| +2450815|7039|1|625| +2450815|7040|1|234| +2450815|7042|1|269| +2450815|7045|1|| +2450815|7046|1|756| +2450815|7048|1|948| +2450815|7051|1|302| +2450815|7052|1|528| +2450815|7054|1|86| +2450815|7057|1|784| +2450815|7058|1|102| +2450815|7060|1|312| +2450815|7063|1|561| +2450815|7064|1|983| +2450815|7066|1|678| +2450815|7069|1|56| +2450815|7070|1|734| +2450815|7072|1|928| +2450815|7075|1|63| +2450815|7076|1|530| +2450815|7078|1|776| +2450815|7081|1|537| +2450815|7082|1|17| +2450815|7084|1|944| +2450815|7087|1|394| +2450815|7088|1|758| +2450815|7090|1|908| +2450815|7093|1|894| +2450815|7094|1|604| +2450815|7096|1|835| +2450815|7099|1|479| +2450815|7100|1|385| +2450815|7102|1|969| +2450815|7105|1|899| +2450815|7106|1|535| +2450815|7108|1|| +2450815|7111|1|| +2450815|7112|1|645| +2450815|7114|1|68| +2450815|7117|1|383| +2450815|7118|1|413| +2450815|7120|1|463| +2450815|7123|1|522| +2450815|7124|1|442| +2450815|7126|1|691| +2450815|7129|1|452| +2450815|7130|1|666| +2450815|7132|1|397| +2450815|7135|1|562| +2450815|7136|1|493| +2450815|7138|1|187| +2450815|7141|1|400| +2450815|7142|1|978| +2450815|7144|1|991| +2450815|7147|1|741| +2450815|7148|1|409| +2450815|7150|1|724| +2450815|7153|1|162| +2450815|7154|1|886| +2450815|7156|1|259| +2450815|7159|1|56| +2450815|7160|1|360| +2450815|7162|1|655| +2450815|7165|1|| +2450815|7166|1|20| +2450815|7168|1|170| +2450815|7171|1|561| +2450815|7172|1|993| +2450815|7174|1|638| +2450815|7177|1|772| +2450815|7178|1|54| +2450815|7180|1|74| +2450815|7183|1|206| +2450815|7184|1|526| +2450815|7186|1|557| +2450815|7189|1|155| +2450815|7190|1|579| +2450815|7192|1|1000| +2450815|7195|1|133| +2450815|7196|1|608| +2450815|7198|1|72| +2450815|7201|1|| +2450815|7202|1|384| +2450815|7204|1|108| +2450815|7207|1|905| +2450815|7208|1|766| +2450815|7210|1|| +2450815|7213|1|69| +2450815|7214|1|815| +2450815|7216|1|662| +2450815|7219|1|830| +2450815|7220|1|175| +2450815|7222|1|408| +2450815|7225|1|614| +2450815|7226|1|192| +2450815|7228|1|23| +2450815|7231|1|365| +2450815|7232|1|380| +2450815|7234|1|435| +2450815|7237|1|171| +2450815|7238|1|608| +2450815|7240|1|966| +2450815|7243|1|370| +2450815|7244|1|11| +2450815|7246|1|226| +2450815|7249|1|76| +2450815|7250|1|14| +2450815|7252|1|28| +2450815|7255|1|| +2450815|7256|1|| +2450815|7258|1|727| +2450815|7261|1|7| +2450815|7262|1|611| +2450815|7264|1|846| +2450815|7267|1|620| +2450815|7268|1|340| +2450815|7270|1|855| +2450815|7273|1|500| +2450815|7274|1|| +2450815|7276|1|219| +2450815|7279|1|820| +2450815|7280|1|518| +2450815|7282|1|375| +2450815|7285|1|739| +2450815|7286|1|66| +2450815|7288|1|280| +2450815|7291|1|549| +2450815|7292|1|902| +2450815|7294|1|403| +2450815|7297|1|343| +2450815|7298|1|313| +2450815|7300|1|217| +2450815|7303|1|271| +2450815|7304|1|657| +2450815|7306|1|337| +2450815|7309|1|564| +2450815|7310|1|667| +2450815|7312|1|531| +2450815|7315|1|722| +2450815|7316|1|705| +2450815|7318|1|471| +2450815|7321|1|| +2450815|7322|1|107| +2450815|7324|1|680| +2450815|7327|1|703| +2450815|7328|1|479| +2450815|7330|1|400| +2450815|7333|1|600| +2450815|7334|1|78| +2450815|7336|1|758| +2450815|7339|1|83| +2450815|7340|1|761| +2450815|7342|1|14| +2450815|7345|1|177| +2450815|7346|1|505| +2450815|7348|1|662| +2450815|7351|1|188| +2450815|7352|1|604| +2450815|7354|1|366| +2450815|7357|1|357| +2450815|7358|1|597| +2450815|7360|1|276| +2450815|7363|1|298| +2450815|7364|1|138| +2450815|7366|1|898| +2450815|7369|1|286| +2450815|7370|1|862| +2450815|7372|1|775| +2450815|7375|1|932| +2450815|7376|1|105| +2450815|7378|1|871| +2450815|7381|1|797| +2450815|7382|1|95| +2450815|7384|1|868| +2450815|7387|1|634| +2450815|7388|1|486| +2450815|7390|1|123| +2450815|7393|1|80| +2450815|7394|1|77| +2450815|7396|1|200| +2450815|7399|1|732| +2450815|7400|1|| +2450815|7402|1|442| +2450815|7405|1|254| +2450815|7406|1|918| +2450815|7408|1|385| +2450815|7411|1|518| +2450815|7412|1|716| +2450815|7414|1|31| +2450815|7417|1|878| +2450815|7418|1|376| +2450815|7420|1|955| +2450815|7423|1|335| +2450815|7424|1|867| +2450815|7426|1|442| +2450815|7429|1|855| +2450815|7430|1|509| +2450815|7432|1|467| +2450815|7435|1|646| +2450815|7436|1|554| +2450815|7438|1|365| +2450815|7441|1|827| +2450815|7442|1|25| +2450815|7444|1|371| +2450815|7447|1|104| +2450815|7448|1|470| +2450815|7450|1|149| +2450815|7453|1|218| +2450815|7454|1|33| +2450815|7456|1|190| +2450815|7459|1|999| +2450815|7460|1|430| +2450815|7462|1|654| +2450815|7465|1|509| +2450815|7466|1|650| +2450815|7468|1|119| +2450815|7471|1|104| +2450815|7472|1|262| +2450815|7474|1|219| +2450815|7477|1|269| +2450815|7478|1|41| +2450815|7480|1|896| +2450815|7483|1|966| +2450815|7484|1|932| +2450815|7486|1|686| +2450815|7489|1|185| +2450815|7490|1|999| +2450815|7492|1|532| +2450815|7495|1|703| +2450815|7496|1|565| +2450815|7498|1|973| +2450815|7501|1|619| +2450815|7502|1|284| +2450815|7504|1|295| +2450815|7507|1|372| +2450815|7508|1|527| +2450815|7510|1|827| +2450815|7513|1|715| +2450815|7514|1|790| +2450815|7516|1|636| +2450815|7519|1|285| +2450815|7520|1|398| +2450815|7522|1|164| +2450815|7525|1|657| +2450815|7526|1|697| +2450815|7528|1|869| +2450815|7531|1|77| +2450815|7532|1|203| +2450815|7534|1|809| +2450815|7537|1|689| +2450815|7538|1|195| +2450815|7540|1|800| +2450815|7543|1|271| +2450815|7544|1|120| +2450815|7546|1|469| +2450815|7549|1|280| +2450815|7550|1|992| +2450815|7552|1|911| +2450815|7555|1|605| +2450815|7556|1|566| +2450815|7558|1|606| +2450815|7561|1|762| +2450815|7562|1|475| +2450815|7564|1|67| +2450815|7567|1|228| +2450815|7568|1|394| +2450815|7570|1|805| +2450815|7573|1|36| +2450815|7574|1|16| +2450815|7576|1|983| +2450815|7579|1|595| +2450815|7580|1|992| +2450815|7582|1|222| +2450815|7585|1|325| +2450815|7586|1|900| +2450815|7588|1|887| +2450815|7591|1|867| +2450815|7592|1|531| +2450815|7594|1|513| +2450815|7597|1|982| +2450815|7598|1|385| +2450815|7600|1|345| +2450815|7603|1|882| +2450815|7604|1|934| +2450815|7606|1|119| +2450815|7609|1|186| +2450815|7610|1|255| +2450815|7612|1|578| +2450815|7615|1|| +2450815|7616|1|728| +2450815|7618|1|31| +2450815|7621|1|372| +2450815|7622|1|| +2450815|7624|1|535| +2450815|7627|1|905| +2450815|7628|1|902| +2450815|7630|1|865| +2450815|7633|1|995| +2450815|7634|1|627| +2450815|7636|1|503| +2450815|7639|1|947| +2450815|7640|1|58| +2450815|7642|1|539| +2450815|7645|1|388| +2450815|7646|1|935| +2450815|7648|1|634| +2450815|7651|1|796| +2450815|7652|1|776| +2450815|7654|1|927| +2450815|7657|1|575| +2450815|7658|1|63| +2450815|7660|1|883| +2450815|7663|1|260| +2450815|7664|1|586| +2450815|7666|1|45| +2450815|7669|1|435| +2450815|7670|1|491| +2450815|7672|1|61| +2450815|7675|1|83| +2450815|7676|1|193| +2450815|7678|1|296| +2450815|7681|1|797| +2450815|7682|1|785| +2450815|7684|1|292| +2450815|7687|1|151| +2450815|7688|1|760| +2450815|7690|1|973| +2450815|7693|1|395| +2450815|7694|1|732| +2450815|7696|1|18| +2450815|7699|1|223| +2450815|7700|1|586| +2450815|7702|1|613| +2450815|7705|1|318| +2450815|7706|1|258| +2450815|7708|1|414| +2450815|7711|1|942| +2450815|7712|1|586| +2450815|7714|1|973| +2450815|7717|1|171| +2450815|7718|1|373| +2450815|7720|1|568| +2450815|7723|1|149| +2450815|7724|1|2| +2450815|7726|1|550| +2450815|7729|1|958| +2450815|7730|1|992| +2450815|7732|1|| +2450815|7735|1|607| +2450815|7736|1|556| +2450815|7738|1|715| +2450815|7741|1|547| +2450815|7742|1|787| +2450815|7744|1|811| +2450815|7747|1|253| +2450815|7748|1|307| +2450815|7750|1|895| +2450815|7753|1|| +2450815|7754|1|191| +2450815|7756|1|315| +2450815|7759|1|716| +2450815|7760|1|590| +2450815|7762|1|597| +2450815|7765|1|47| +2450815|7766|1|110| +2450815|7768|1|408| +2450815|7771|1|702| +2450815|7772|1|233| +2450815|7774|1|53| +2450815|7777|1|731| +2450815|7778|1|854| +2450815|7780|1|6| +2450815|7783|1|393| +2450815|7784|1|79| +2450815|7786|1|303| +2450815|7789|1|97| +2450815|7790|1|| +2450815|7792|1|962| +2450815|7795|1|763| +2450815|7796|1|678| +2450815|7798|1|943| +2450815|7801|1|840| +2450815|7802|1|519| +2450815|7804|1|| +2450815|7807|1|888| +2450815|7808|1|224| +2450815|7810|1|65| +2450815|7813|1|440| +2450815|7814|1|871| +2450815|7816|1|979| +2450815|7819|1|748| +2450815|7820|1|280| +2450815|7822|1|440| +2450815|7825|1|394| +2450815|7826|1|994| +2450815|7828|1|969| +2450815|7831|1|| +2450815|7832|1|999| +2450815|7834|1|757| +2450815|7837|1|775| +2450815|7838|1|477| +2450815|7840|1|974| +2450815|7843|1|82| +2450815|7844|1|| +2450815|7846|1|681| +2450815|7849|1|631| +2450815|7850|1|524| +2450815|7852|1|111| +2450815|7855|1|406| +2450815|7856|1|58| +2450815|7858|1|186| +2450815|7861|1|686| +2450815|7862|1|441| +2450815|7864|1|337| +2450815|7867|1|345| +2450815|7868|1|171| +2450815|7870|1|| +2450815|7873|1|955| +2450815|7874|1|296| +2450815|7876|1|965| +2450815|7879|1|34| +2450815|7880|1|407| +2450815|7882|1|301| +2450815|7885|1|38| +2450815|7886|1|835| +2450815|7888|1|354| +2450815|7891|1|981| +2450815|7892|1|407| +2450815|7894|1|272| +2450815|7897|1|741| +2450815|7898|1|449| +2450815|7900|1|142| +2450815|7903|1|133| +2450815|7904|1|334| +2450815|7906|1|487| +2450815|7909|1|627| +2450815|7910|1|344| +2450815|7912|1|889| +2450815|7915|1|67| +2450815|7916|1|175| +2450815|7918|1|727| +2450815|7921|1|467| +2450815|7922|1|477| +2450815|7924|1|956| +2450815|7927|1|472| +2450815|7928|1|277| +2450815|7930|1|975| +2450815|7933|1|693| +2450815|7934|1|291| +2450815|7936|1|328| +2450815|7939|1|517| +2450815|7940|1|| +2450815|7942|1|335| +2450815|7945|1|4| +2450815|7946|1|983| +2450815|7948|1|938| +2450815|7951|1|597| +2450815|7952|1|247| +2450815|7954|1|33| +2450815|7957|1|158| +2450815|7958|1|311| +2450815|7960|1|496| +2450815|7963|1|291| +2450815|7964|1|204| +2450815|7966|1|26| +2450815|7969|1|| +2450815|7970|1|735| +2450815|7972|1|144| +2450815|7975|1|518| +2450815|7976|1|298| +2450815|7978|1|983| +2450815|7981|1|849| +2450815|7982|1|925| +2450815|7984|1|769| +2450815|7987|1|698| +2450815|7988|1|525| +2450815|7990|1|522| +2450815|7993|1|261| +2450815|7994|1|296| +2450815|7996|1|397| +2450815|7999|1|49| +2450815|8000|1|646| +2450815|8002|1|245| +2450815|8005|1|166| +2450815|8006|1|586| +2450815|8008|1|913| +2450815|8011|1|128| +2450815|8012|1|716| +2450815|8014|1|| +2450815|8017|1|| +2450815|8018|1|339| +2450815|8020|1|368| +2450815|8023|1|273| +2450815|8024|1|965| +2450815|8026|1|378| +2450815|8029|1|378| +2450815|8030|1|962| +2450815|8032|1|581| +2450815|8035|1|582| +2450815|8036|1|594| +2450815|8038|1|962| +2450815|8041|1|537| +2450815|8042|1|133| +2450815|8044|1|753| +2450815|8047|1|353| +2450815|8048|1|465| +2450815|8050|1|56| +2450815|8053|1|943| +2450815|8054|1|363| +2450815|8056|1|547| +2450815|8059|1|200| +2450815|8060|1|13| +2450815|8062|1|525| +2450815|8065|1|911| +2450815|8066|1|857| +2450815|8068|1|861| +2450815|8071|1|47| +2450815|8072|1|819| +2450815|8074|1|404| +2450815|8077|1|423| +2450815|8078|1|499| +2450815|8080|1|266| +2450815|8083|1|| +2450815|8084|1|279| +2450815|8086|1|180| +2450815|8089|1|321| +2450815|8090|1|884| +2450815|8092|1|| +2450815|8095|1|957| +2450815|8096|1|217| +2450815|8098|1|179| +2450815|8101|1|70| +2450815|8102|1|300| +2450815|8104|1|736| +2450815|8107|1|607| +2450815|8108|1|856| +2450815|8110|1|38| +2450815|8113|1|172| +2450815|8114|1|301| +2450815|8116|1|946| +2450815|8119|1|122| +2450815|8120|1|179| +2450815|8122|1|848| +2450815|8125|1|599| +2450815|8126|1|20| +2450815|8128|1|460| +2450815|8131|1|609| +2450815|8132|1|784| +2450815|8134|1|| +2450815|8137|1|838| +2450815|8138|1|403| +2450815|8140|1|773| +2450815|8143|1|878| +2450815|8144|1|21| +2450815|8146|1|419| +2450815|8149|1|106| +2450815|8150|1|| +2450815|8152|1|464| +2450815|8155|1|812| +2450815|8156|1|604| +2450815|8158|1|826| +2450815|8161|1|136| +2450815|8162|1|252| +2450815|8164|1|793| +2450815|8167|1|425| +2450815|8168|1|629| +2450815|8170|1|735| +2450815|8173|1|131| +2450815|8174|1|944| +2450815|8176|1|992| +2450815|8179|1|563| +2450815|8180|1|444| +2450815|8182|1|676| +2450815|8185|1|307| +2450815|8186|1|577| +2450815|8188|1|586| +2450815|8191|1|517| +2450815|8192|1|236| +2450815|8194|1|234| +2450815|8197|1|143| +2450815|8198|1|792| +2450815|8200|1|837| +2450815|8203|1|| +2450815|8204|1|609| +2450815|8206|1|695| +2450815|8209|1|255| +2450815|8210|1|945| +2450815|8212|1|585| +2450815|8215|1|681| +2450815|8216|1|375| +2450815|8218|1|| +2450815|8221|1|64| +2450815|8222|1|331| +2450815|8224|1|43| +2450815|8227|1|253| +2450815|8228|1|116| +2450815|8230|1|416| +2450815|8233|1|433| +2450815|8234|1|29| +2450815|8236|1|933| +2450815|8239|1|92| +2450815|8240|1|190| +2450815|8242|1|398| +2450815|8245|1|852| +2450815|8246|1|801| +2450815|8248|1|591| +2450815|8251|1|168| +2450815|8252|1|416| +2450815|8254|1|833| +2450815|8257|1|977| +2450815|8258|1|186| +2450815|8260|1|859| +2450815|8263|1|155| +2450815|8264|1|442| +2450815|8266|1|533| +2450815|8269|1|403| +2450815|8270|1|962| +2450815|8272|1|741| +2450815|8275|1|216| +2450815|8276|1|873| +2450815|8278|1|200| +2450815|8281|1|237| +2450815|8282|1|4| +2450815|8284|1|180| +2450815|8287|1|5| +2450815|8288|1|16| +2450815|8290|1|398| +2450815|8293|1|309| +2450815|8294|1|436| +2450815|8296|1|509| +2450815|8299|1|588| +2450815|8300|1|168| +2450815|8302|1|288| +2450815|8305|1|330| +2450815|8306|1|399| +2450815|8308|1|197| +2450815|8311|1|907| +2450815|8312|1|303| +2450815|8314|1|966| +2450815|8317|1|751| +2450815|8318|1|336| +2450815|8320|1|662| +2450815|8323|1|673| +2450815|8324|1|83| +2450815|8326|1|178| +2450815|8329|1|152| +2450815|8330|1|584| +2450815|8332|1|448| +2450815|8335|1|132| +2450815|8336|1|409| +2450815|8338|1|| +2450815|8341|1|148| +2450815|8342|1|727| +2450815|8344|1|870| +2450815|8347|1|634| +2450815|8348|1|33| +2450815|8350|1|648| +2450815|8353|1|89| +2450815|8354|1|303| +2450815|8356|1|761| +2450815|8359|1|972| +2450815|8360|1|231| +2450815|8362|1|631| +2450815|8365|1|978| +2450815|8366|1|98| +2450815|8368|1|1| +2450815|8371|1|974| +2450815|8372|1|532| +2450815|8374|1|54| +2450815|8377|1|186| +2450815|8378|1|909| +2450815|8380|1|198| +2450815|8383|1|17| +2450815|8384|1|753| +2450815|8386|1|12| +2450815|8389|1|850| +2450815|8390|1|154| +2450815|8392|1|334| +2450815|8395|1|142| +2450815|8396|1|108| +2450815|8398|1|57| +2450815|8401|1|833| +2450815|8402|1|626| +2450815|8404|1|624| +2450815|8407|1|283| +2450815|8408|1|219| +2450815|8410|1|239| +2450815|8413|1|924| +2450815|8414|1|27| +2450815|8416|1|396| +2450815|8419|1|678| +2450815|8420|1|37| +2450815|8422|1|138| +2450815|8425|1|466| +2450815|8426|1|922| +2450815|8428|1|424| +2450815|8431|1|6| +2450815|8432|1|693| +2450815|8434|1|209| +2450815|8437|1|395| +2450815|8438|1|400| +2450815|8440|1|| +2450815|8443|1|383| +2450815|8444|1|941| +2450815|8446|1|538| +2450815|8449|1|713| +2450815|8450|1|644| +2450815|8452|1|413| +2450815|8455|1|13| +2450815|8456|1|650| +2450815|8458|1|503| +2450815|8461|1|125| +2450815|8462|1|494| +2450815|8464|1|199| +2450815|8467|1|549| +2450815|8468|1|532| +2450815|8470|1|587| +2450815|8473|1|4| +2450815|8474|1|729| +2450815|8476|1|43| +2450815|8479|1|| +2450815|8480|1|55| +2450815|8482|1|224| +2450815|8485|1|819| +2450815|8486|1|619| +2450815|8488|1|856| +2450815|8491|1|391| +2450815|8492|1|206| +2450815|8494|1|401| +2450815|8497|1|880| +2450815|8498|1|813| +2450815|8500|1|198| +2450815|8503|1|908| +2450815|8504|1|555| +2450815|8506|1|275| +2450815|8509|1|14| +2450815|8510|1|783| +2450815|8512|1|838| +2450815|8515|1|461| +2450815|8516|1|247| +2450815|8518|1|309| +2450815|8521|1|141| +2450815|8522|1|386| +2450815|8524|1|326| +2450815|8527|1|327| +2450815|8528|1|448| +2450815|8530|1|406| +2450815|8533|1|339| +2450815|8534|1|513| +2450815|8536|1|560| +2450815|8539|1|747| +2450815|8540|1|995| +2450815|8542|1|891| +2450815|8545|1|884| +2450815|8546|1|488| +2450815|8548|1|167| +2450815|8551|1|491| +2450815|8552|1|989| +2450815|8554|1|361| +2450815|8557|1|963| +2450815|8558|1|319| +2450815|8560|1|705| +2450815|8563|1|856| +2450815|8564|1|940| +2450815|8566|1|87| +2450815|8569|1|235| +2450815|8570|1|39| +2450815|8572|1|375| +2450815|8575|1|524| +2450815|8576|1|911| +2450815|8578|1|552| +2450815|8581|1|109| +2450815|8582|1|| +2450815|8584|1|| +2450815|8587|1|94| +2450815|8588|1|202| +2450815|8590|1|993| +2450815|8593|1|814| +2450815|8594|1|557| +2450815|8596|1|319| +2450815|8599|1|159| +2450815|8600|1|421| +2450815|8602|1|209| +2450815|8605|1|290| +2450815|8606|1|91| +2450815|8608|1|222| +2450815|8611|1|204| +2450815|8612|1|841| +2450815|8614|1|287| +2450815|8617|1|733| +2450815|8618|1|298| +2450815|8620|1|350| +2450815|8623|1|934| +2450815|8624|1|680| +2450815|8626|1|459| +2450815|8629|1|| +2450815|8630|1|506| +2450815|8632|1|387| +2450815|8635|1|566| +2450815|8636|1|686| +2450815|8638|1|575| +2450815|8641|1|831| +2450815|8642|1|771| +2450815|8644|1|753| +2450815|8647|1|417| +2450815|8648|1|750| +2450815|8650|1|606| +2450815|8653|1|489| +2450815|8654|1|548| +2450815|8656|1|885| +2450815|8659|1|| +2450815|8660|1|921| +2450815|8662|1|759| +2450815|8665|1|423| +2450815|8666|1|488| +2450815|8668|1|275| +2450815|8671|1|524| +2450815|8672|1|118| +2450815|8674|1|475| +2450815|8677|1|694| +2450815|8678|1|662| +2450815|8680|1|928| +2450815|8683|1|524| +2450815|8684|1|277| +2450815|8686|1|876| +2450815|8689|1|613| +2450815|8690|1|157| +2450815|8692|1|633| +2450815|8695|1|679| +2450815|8696|1|769| +2450815|8698|1|621| +2450815|8701|1|707| +2450815|8702|1|215| +2450815|8704|1|266| +2450815|8707|1|932| +2450815|8708|1|364| +2450815|8710|1|828| +2450815|8713|1|386| +2450815|8714|1|440| +2450815|8716|1|885| +2450815|8719|1|478| +2450815|8720|1|283| +2450815|8722|1|128| +2450815|8725|1|83| +2450815|8726|1|597| +2450815|8728|1|682| +2450815|8731|1|214| +2450815|8732|1|960| +2450815|8734|1|561| +2450815|8737|1|777| +2450815|8738|1|957| +2450815|8740|1|810| +2450815|8743|1|615| +2450815|8744|1|| +2450815|8746|1|301| +2450815|8749|1|218| +2450815|8750|1|276| +2450815|8752|1|94| +2450815|8755|1|756| +2450815|8756|1|762| +2450815|8758|1|131| +2450815|8761|1|501| +2450815|8762|1|26| +2450815|8764|1|541| +2450815|8767|1|159| +2450815|8768|1|618| +2450815|8770|1|114| +2450815|8773|1|114| +2450815|8774|1|831| +2450815|8776|1|882| +2450815|8779|1|486| +2450815|8780|1|602| +2450815|8782|1|607| +2450815|8785|1|431| +2450815|8786|1|283| +2450815|8788|1|469| +2450815|8791|1|874| +2450815|8792|1|| +2450815|8794|1|777| +2450815|8797|1|42| +2450815|8798|1|842| +2450815|8800|1|844| +2450815|8803|1|968| +2450815|8804|1|165| +2450815|8806|1|657| +2450815|8809|1|953| +2450815|8810|1|738| +2450815|8812|1|461| +2450815|8815|1|382| +2450815|8816|1|385| +2450815|8818|1|293| +2450815|8821|1|45| +2450815|8822|1|675| +2450815|8824|1|471| +2450815|8827|1|466| +2450815|8828|1|932| +2450815|8830|1|449| +2450815|8833|1|256| +2450815|8834|1|624| +2450815|8836|1|785| +2450815|8839|1|123| +2450815|8840|1|322| +2450815|8842|1|33| +2450815|8845|1|760| +2450815|8846|1|424| +2450815|8848|1|125| +2450815|8851|1|984| +2450815|8852|1|757| +2450815|8854|1|324| +2450815|8857|1|930| +2450815|8858|1|895| +2450815|8860|1|603| +2450815|8863|1|677| +2450815|8864|1|44| +2450815|8866|1|290| +2450815|8869|1|| +2450815|8870|1|| +2450815|8872|1|983| +2450815|8875|1|531| +2450815|8876|1|976| +2450815|8878|1|928| +2450815|8881|1|974| +2450815|8882|1|964| +2450815|8884|1|3| +2450815|8887|1|369| +2450815|8888|1|719| +2450815|8890|1|16| +2450815|8893|1|783| +2450815|8894|1|673| +2450815|8896|1|737| +2450815|8899|1|779| +2450815|8900|1|845| +2450815|8902|1|979| +2450815|8905|1|313| +2450815|8906|1|866| +2450815|8908|1|107| +2450815|8911|1|338| +2450815|8912|1|384| +2450815|8914|1|515| +2450815|8917|1|248| +2450815|8918|1|597| +2450815|8920|1|761| +2450815|8923|1|695| +2450815|8924|1|464| +2450815|8926|1|835| +2450815|8929|1|603| +2450815|8930|1|963| +2450815|8932|1|956| +2450815|8935|1|835| +2450815|8936|1|205| +2450815|8938|1|796| +2450815|8941|1|278| +2450815|8942|1|379| +2450815|8944|1|494| +2450815|8947|1|590| +2450815|8948|1|721| +2450815|8950|1|6| +2450815|8953|1|954| +2450815|8954|1|308| +2450815|8956|1|118| +2450815|8959|1|821| +2450815|8960|1|575| +2450815|8962|1|271| +2450815|8965|1|765| +2450815|8966|1|701| +2450815|8968|1|192| +2450815|8971|1|180| +2450815|8972|1|401| +2450815|8974|1|496| +2450815|8977|1|203| +2450815|8978|1|481| +2450815|8980|1|151| +2450815|8983|1|250| +2450815|8984|1|901| +2450815|8986|1|826| +2450815|8989|1|90| +2450815|8990|1|662| +2450815|8992|1|365| +2450815|8995|1|1| +2450815|8996|1|734| +2450815|8998|1|738| +2450815|9001|1|258| +2450815|9002|1|799| +2450815|9004|1|936| +2450815|9007|1|140| +2450815|9008|1|115| +2450815|9010|1|655| +2450815|9013|1|757| +2450815|9014|1|799| +2450815|9016|1|54| +2450815|9019|1|891| +2450815|9020|1|149| +2450815|9022|1|141| +2450815|9025|1|727| +2450815|9026|1|958| +2450815|9028|1|3| +2450815|9031|1|373| +2450815|9032|1|421| +2450815|9034|1|354| +2450815|9037|1|743| +2450815|9038|1|612| +2450815|9040|1|206| +2450815|9043|1|965| +2450815|9044|1|| +2450815|9046|1|508| +2450815|9049|1|548| +2450815|9050|1|265| +2450815|9052|1|714| +2450815|9055|1|381| +2450815|9056|1|456| +2450815|9058|1|80| +2450815|9061|1|158| +2450815|9062|1|27| +2450815|9064|1|784| +2450815|9067|1|384| +2450815|9068|1|795| +2450815|9070|1|337| +2450815|9073|1|813| +2450815|9074|1|198| +2450815|9076|1|2| +2450815|9079|1|892| +2450815|9080|1|267| +2450815|9082|1|386| +2450815|9085|1|558| +2450815|9086|1|84| +2450815|9088|1|802| +2450815|9091|1|803| +2450815|9092|1|246| +2450815|9094|1|590| +2450815|9097|1|24| +2450815|9098|1|285| +2450815|9100|1|413| +2450815|9103|1|175| +2450815|9104|1|260| +2450815|9106|1|251| +2450815|9109|1|32| +2450815|9110|1|36| +2450815|9112|1|281| +2450815|9115|1|388| +2450815|9116|1|916| +2450815|9118|1|465| +2450815|9121|1|489| +2450815|9122|1|835| +2450815|9124|1|803| +2450815|9127|1|906| +2450815|9128|1|432| +2450815|9130|1|415| +2450815|9133|1|480| +2450815|9134|1|955| +2450815|9136|1|945| +2450815|9139|1|610| +2450815|9140|1|867| +2450815|9142|1|783| +2450815|9145|1|60| +2450815|9146|1|52| +2450815|9148|1|753| +2450815|9151|1|742| +2450815|9152|1|634| +2450815|9154|1|387| +2450815|9157|1|| +2450815|9158|1|778| +2450815|9160|1|872| +2450815|9163|1|204| +2450815|9164|1|136| +2450815|9166|1|185| +2450815|9169|1|| +2450815|9170|1|437| +2450815|9172|1|460| +2450815|9175|1|374| +2450815|9176|1|928| +2450815|9178|1|489| +2450815|9181|1|387| +2450815|9182|1|55| +2450815|9184|1|430| +2450815|9187|1|28| +2450815|9188|1|73| +2450815|9190|1|16| +2450815|9193|1|871| +2450815|9194|1|683| +2450815|9196|1|735| +2450815|9199|1|502| +2450815|9200|1|115| +2450815|9202|1|261| +2450815|9205|1|127| +2450815|9206|1|967| +2450815|9208|1|967| +2450815|9211|1|744| +2450815|9212|1|312| +2450815|9214|1|374| +2450815|9217|1|929| +2450815|9218|1|226| +2450815|9220|1|228| +2450815|9223|1|607| +2450815|9224|1|243| +2450815|9226|1|385| +2450815|9229|1|949| +2450815|9230|1|494| +2450815|9232|1|775| +2450815|9235|1|399| +2450815|9236|1|462| +2450815|9238|1|635| +2450815|9241|1|91| +2450815|9242|1|127| +2450815|9244|1|231| +2450815|9247|1|221| +2450815|9248|1|582| +2450815|9250|1|449| +2450815|9253|1|467| +2450815|9254|1|812| +2450815|9256|1|754| +2450815|9259|1|690| +2450815|9260|1|666| +2450815|9262|1|714| +2450815|9265|1|476| +2450815|9266|1|799| +2450815|9268|1|340| +2450815|9271|1|30| +2450815|9272|1|198| +2450815|9274|1|914| +2450815|9277|1|845| +2450815|9278|1|829| +2450815|9280|1|362| +2450815|9283|1|447| +2450815|9284|1|998| +2450815|9286|1|770| +2450815|9289|1|261| +2450815|9290|1|700| +2450815|9292|1|302| +2450815|9295|1|865| +2450815|9296|1|| +2450815|9298|1|165| +2450815|9301|1|42| +2450815|9302|1|524| +2450815|9304|1|550| +2450815|9307|1|728| +2450815|9308|1|584| +2450815|9310|1|909| +2450815|9313|1|360| +2450815|9314|1|| +2450815|9316|1|233| +2450815|9319|1|123| +2450815|9320|1|680| +2450815|9322|1|193| +2450815|9325|1|320| +2450815|9326|1|834| +2450815|9328|1|55| +2450815|9331|1|307| +2450815|9332|1|789| +2450815|9334|1|| +2450815|9337|1|937| +2450815|9338|1|320| +2450815|9340|1|131| +2450815|9343|1|227| +2450815|9344|1|| +2450815|9346|1|20| +2450815|9349|1|| +2450815|9350|1|921| +2450815|9352|1|773| +2450815|9355|1|116| +2450815|9356|1|530| +2450815|9358|1|377| +2450815|9361|1|284| +2450815|9362|1|283| +2450815|9364|1|947| +2450815|9367|1|891| +2450815|9368|1|698| +2450815|9370|1|391| +2450815|9373|1|643| +2450815|9374|1|288| +2450815|9376|1|313| +2450815|9379|1|289| +2450815|9380|1|74| +2450815|9382|1|235| +2450815|9385|1|386| +2450815|9386|1|121| +2450815|9388|1|142| +2450815|9391|1|85| +2450815|9392|1|448| +2450815|9394|1|441| +2450815|9397|1|32| +2450815|9398|1|365| +2450815|9400|1|3| +2450815|9403|1|993| +2450815|9404|1|| +2450815|9406|1|| +2450815|9409|1|518| +2450815|9410|1|915| +2450815|9412|1|375| +2450815|9415|1|762| +2450815|9416|1|839| +2450815|9418|1|243| +2450815|9421|1|491| +2450815|9422|1|704| +2450815|9424|1|446| +2450815|9427|1|886| +2450815|9428|1|464| +2450815|9430|1|876| +2450815|9433|1|954| +2450815|9434|1|495| +2450815|9436|1|716| +2450815|9439|1|671| +2450815|9440|1|205| +2450815|9442|1|558| +2450815|9445|1|946| +2450815|9446|1|633| +2450815|9448|1|| +2450815|9451|1|283| +2450815|9452|1|507| +2450815|9454|1|2| +2450815|9457|1|247| +2450815|9458|1|557| +2450815|9460|1|646| +2450815|9463|1|49| +2450815|9464|1|707| +2450815|9466|1|214| +2450815|9469|1|681| +2450815|9470|1|665| +2450815|9472|1|| +2450815|9475|1|367| +2450815|9476|1|| +2450815|9478|1|283| +2450815|9481|1|319| +2450815|9482|1|957| +2450815|9484|1|269| +2450815|9487|1|719| +2450815|9488|1|290| +2450815|9490|1|44| +2450815|9493|1|| +2450815|9494|1|96| +2450815|9496|1|411| +2450815|9499|1|409| +2450815|9500|1|720| +2450815|9502|1|59| +2450815|9505|1|754| +2450815|9506|1|707| +2450815|9508|1|528| +2450815|9511|1|962| +2450815|9512|1|362| +2450815|9514|1|222| +2450815|9517|1|907| +2450815|9518|1|771| +2450815|9520|1|199| +2450815|9523|1|689| +2450815|9524|1|396| +2450815|9526|1|593| +2450815|9529|1|201| +2450815|9530|1|156| +2450815|9532|1|646| +2450815|9535|1|415| +2450815|9536|1|175| +2450815|9538|1|598| +2450815|9541|1|495| +2450815|9542|1|471| +2450815|9544|1|660| +2450815|9547|1|| +2450815|9548|1|758| +2450815|9550|1|78| +2450815|9553|1|564| +2450815|9554|1|628| +2450815|9556|1|822| +2450815|9559|1|298| +2450815|9560|1|4| +2450815|9562|1|702| +2450815|9565|1|528| +2450815|9566|1|960| +2450815|9568|1|43| +2450815|9571|1|592| +2450815|9572|1|361| +2450815|9574|1|509| +2450815|9577|1|498| +2450815|9578|1|406| +2450815|9580|1|390| +2450815|9583|1|751| +2450815|9584|1|| +2450815|9586|1|94| +2450815|9589|1|88| +2450815|9590|1|347| +2450815|9592|1|264| +2450815|9595|1|270| +2450815|9596|1|746| +2450815|9598|1|| +2450815|9601|1|597| +2450815|9602|1|942| +2450815|9604|1|| +2450815|9607|1|| +2450815|9608|1|955| +2450815|9610|1|33| +2450815|9613|1|516| +2450815|9614|1|602| +2450815|9616|1|797| +2450815|9619|1|243| +2450815|9620|1|683| +2450815|9622|1|| +2450815|9625|1|845| +2450815|9626|1|485| +2450815|9628|1|528| +2450815|9631|1|513| +2450815|9632|1|765| +2450815|9634|1|863| +2450815|9637|1|882| +2450815|9638|1|27| +2450815|9640|1|418| +2450815|9643|1|| +2450815|9644|1|336| +2450815|9646|1|129| +2450815|9649|1|814| +2450815|9650|1|288| +2450815|9652|1|959| +2450815|9655|1|286| +2450815|9656|1|637| +2450815|9658|1|422| +2450815|9661|1|42| +2450815|9662|1|692| +2450815|9664|1|867| +2450815|9667|1|| +2450815|9668|1|666| +2450815|9670|1|| +2450815|9673|1|412| +2450815|9674|1|518| +2450815|9676|1|369| +2450815|9679|1|782| +2450815|9680|1|924| +2450815|9682|1|736| +2450815|9685|1|598| +2450815|9686|1|164| +2450815|9688|1|827| +2450815|9691|1|837| +2450815|9692|1|206| +2450815|9694|1|551| +2450815|9697|1|900| +2450815|9698|1|734| +2450815|9700|1|87| +2450815|9703|1|44| +2450815|9704|1|822| +2450815|9706|1|719| +2450815|9709|1|590| +2450815|9710|1|534| +2450815|9712|1|904| +2450815|9715|1|56| +2450815|9716|1|628| +2450815|9718|1|959| +2450815|9721|1|757| +2450815|9722|1|102| +2450815|9724|1|82| +2450815|9727|1|899| +2450815|9728|1|946| +2450815|9730|1|881| +2450815|9733|1|361| +2450815|9734|1|564| +2450815|9736|1|372| +2450815|9739|1|556| +2450815|9740|1|66| +2450815|9742|1|63| +2450815|9745|1|152| +2450815|9746|1|710| +2450815|9748|1|755| +2450815|9751|1|773| +2450815|9752|1|478| +2450815|9754|1|464| +2450815|9757|1|| +2450815|9758|1|| +2450815|9760|1|974| +2450815|9763|1|677| +2450815|9764|1|622| +2450815|9766|1|646| +2450815|9769|1|469| +2450815|9770|1|| +2450815|9772|1|| +2450815|9775|1|636| +2450815|9776|1|366| +2450815|9778|1|| +2450815|9781|1|663| +2450815|9782|1|856| +2450815|9784|1|309| +2450815|9787|1|846| +2450815|9788|1|95| +2450815|9790|1|165| +2450815|9793|1|| +2450815|9794|1|323| +2450815|9796|1|9| +2450815|9799|1|| +2450815|9800|1|621| +2450815|9802|1|| +2450815|9805|1|615| +2450815|9806|1|776| +2450815|9808|1|788| +2450815|9811|1|843| +2450815|9812|1|116| +2450815|9814|1|20| +2450815|9817|1|840| +2450815|9818|1|386| +2450815|9820|1|748| +2450815|9823|1|572| +2450815|9824|1|518| +2450815|9826|1|80| +2450815|9829|1|760| +2450815|9830|1|28| +2450815|9832|1|893| +2450815|9835|1|204| +2450815|9836|1|437| +2450815|9838|1|476| +2450815|9841|1|908| +2450815|9842|1|734| +2450815|9844|1|32| +2450815|9847|1|| +2450815|9848|1|777| +2450815|9850|1|857| +2450815|9853|1|424| +2450815|9854|1|843| +2450815|9856|1|99| +2450815|9859|1|532| +2450815|9860|1|151| +2450815|9862|1|156| +2450815|9865|1|932| +2450815|9866|1|534| +2450815|9868|1|541| +2450815|9871|1|580| +2450815|9872|1|539| +2450815|9874|1|190| +2450815|9877|1|958| +2450815|9878|1|475| +2450815|9880|1|596| +2450815|9883|1|820| +2450815|9884|1|813| +2450815|9886|1|49| +2450815|9889|1|265| +2450815|9890|1|852| +2450815|9892|1|228| +2450815|9895|1|210| +2450815|9896|1|893| +2450815|9898|1|143| +2450815|9901|1|371| +2450815|9902|1|368| +2450815|9904|1|944| +2450815|9907|1|440| +2450815|9908|1|813| +2450815|9910|1|396| +2450815|9913|1|789| +2450815|9914|1|120| +2450815|9916|1|882| +2450815|9919|1|134| +2450815|9920|1|537| +2450815|9922|1|810| +2450815|9925|1|870| +2450815|9926|1|199| +2450815|9928|1|356| +2450815|9931|1|903| +2450815|9932|1|867| +2450815|9934|1|22| +2450815|9937|1|345| +2450815|9938|1|489| +2450815|9940|1|123| +2450815|9943|1|858| +2450815|9944|1|35| +2450815|9946|1|203| +2450815|9949|1|638| +2450815|9950|1|348| +2450815|9952|1|882| +2450815|9955|1|252| +2450815|9956|1|113| +2450815|9958|1|138| +2450815|9961|1|16| +2450815|9962|1|769| +2450815|9964|1|456| +2450815|9967|1|367| +2450815|9968|1|718| +2450815|9970|1|781| +2450815|9973|1|144| +2450815|9974|1|624| +2450815|9976|1|217| +2450815|9979|1|756| +2450815|9980|1|304| +2450815|9982|1|982| +2450815|9985|1|654| +2450815|9986|1|295| +2450815|9988|1|432| +2450815|9991|1|662| +2450815|9992|1|395| +2450815|9994|1|969| +2450815|9997|1|631| +2450815|9998|1|10| +2450815|10000|1|452| +2450815|10003|1|693| +2450815|10004|1|735| +2450815|10006|1|30| +2450815|10009|1|434| +2450815|10010|1|202| +2450815|10012|1|616| +2450815|10015|1|525| +2450815|10016|1|142| +2450815|10018|1|219| +2450815|10021|1|129| +2450815|10022|1|270| +2450815|10024|1|623| +2450815|10027|1|740| +2450815|10028|1|301| +2450815|10030|1|| +2450815|10033|1|433| +2450815|10034|1|436| +2450815|10036|1|447| +2450815|10039|1|824| +2450815|10040|1|898| +2450815|10042|1|901| +2450815|10045|1|357| +2450815|10046|1|542| +2450815|10048|1|288| +2450815|10051|1|998| +2450815|10052|1|228| +2450815|10054|1|204| +2450815|10057|1|944| +2450815|10058|1|888| +2450815|10060|1|485| +2450815|10063|1|969| +2450815|10064|1|303| +2450815|10066|1|407| +2450815|10069|1|705| +2450815|10070|1|69| +2450815|10072|1|773| +2450815|10075|1|951| +2450815|10076|1|141| +2450815|10078|1|47| +2450815|10081|1|619| +2450815|10082|1|888| +2450815|10084|1|895| +2450815|10087|1|550| +2450815|10088|1|976| +2450815|10090|1|362| +2450815|10093|1|35| +2450815|10094|1|878| +2450815|10096|1|243| +2450815|10099|1|458| +2450815|10100|1|5| +2450815|10102|1|207| +2450815|10105|1|143| +2450815|10106|1|74| +2450815|10108|1|904| +2450815|10111|1|871| +2450815|10112|1|580| +2450815|10114|1|598| +2450815|10117|1|364| +2450815|10118|1|356| +2450815|10120|1|184| +2450815|10123|1|716| +2450815|10124|1|994| +2450815|10126|1|680| +2450815|10129|1|744| +2450815|10130|1|612| +2450815|10132|1|355| +2450815|10135|1|482| +2450815|10136|1|556| +2450815|10138|1|| +2450815|10141|1|212| +2450815|10142|1|374| +2450815|10144|1|| +2450815|10147|1|882| +2450815|10148|1|342| +2450815|10150|1|| +2450815|10153|1|721| +2450815|10154|1|119| +2450815|10156|1|733| +2450815|10159|1|60| +2450815|10160|1|725| +2450815|10162|1|815| +2450815|10165|1|230| +2450815|10166|1|577| +2450815|10168|1|738| +2450815|10171|1|283| +2450815|10172|1|513| +2450815|10174|1|817| +2450815|10177|1|795| +2450815|10178|1|114| +2450815|10180|1|| +2450815|10183|1|640| +2450815|10184|1|760| +2450815|10186|1|636| +2450815|10189|1|61| +2450815|10190|1|259| +2450815|10192|1|496| +2450815|10195|1|659| +2450815|10196|1|124| +2450815|10198|1|704| +2450815|10201|1|593| +2450815|10202|1|517| +2450815|10204|1|152| +2450815|10207|1|860| +2450815|10208|1|931| +2450815|10210|1|746| +2450815|10213|1|307| +2450815|10214|1|966| +2450815|10216|1|726| +2450815|10219|1|872| +2450815|10220|1|498| +2450815|10222|1|394| +2450815|10225|1|438| +2450815|10226|1|943| +2450815|10228|1|558| +2450815|10231|1|659| +2450815|10232|1|219| +2450815|10234|1|924| +2450815|10237|1|574| +2450815|10238|1|724| +2450815|10240|1|| +2450815|10243|1|896| +2450815|10244|1|149| +2450815|10246|1|846| +2450815|10249|1|917| +2450815|10250|1|| +2450815|10252|1|665| +2450815|10255|1|574| +2450815|10256|1|589| +2450815|10258|1|110| +2450815|10261|1|745| +2450815|10262|1|9| +2450815|10264|1|552| +2450815|10267|1|537| +2450815|10268|1|901| +2450815|10270|1|29| +2450815|10273|1|600| +2450815|10274|1|139| +2450815|10276|1|110| +2450815|10279|1|963| +2450815|10280|1|691| +2450815|10282|1|618| +2450815|10285|1|| +2450815|10286|1|405| +2450815|10288|1|24| +2450815|10291|1|656| +2450815|10292|1|588| +2450815|10294|1|305| +2450815|10297|1|138| +2450815|10298|1|813| +2450815|10300|1|899| +2450815|10303|1|928| +2450815|10304|1|540| +2450815|10306|1|275| +2450815|10309|1|206| +2450815|10310|1|885| +2450815|10312|1|732| +2450815|10315|1|552| +2450815|10316|1|547| +2450815|10318|1|745| +2450815|10321|1|10| +2450815|10322|1|18| +2450815|10324|1|211| +2450815|10327|1|236| +2450815|10328|1|711| +2450815|10330|1|511| +2450815|10333|1|668| +2450815|10334|1|401| +2450815|10336|1|863| +2450815|10339|1|336| +2450815|10340|1|162| +2450815|10342|1|774| +2450815|10345|1|699| +2450815|10346|1|403| +2450815|10348|1|464| +2450815|10351|1|592| +2450815|10352|1|752| +2450815|10354|1|443| +2450815|10357|1|180| +2450815|10358|1|312| +2450815|10360|1|147| +2450815|10363|1|7| +2450815|10364|1|208| +2450815|10366|1|797| +2450815|10369|1|116| +2450815|10370|1|206| +2450815|10372|1|27| +2450815|10375|1|306| +2450815|10376|1|423| +2450815|10378|1|993| +2450815|10381|1|984| +2450815|10382|1|115| +2450815|10384|1|265| +2450815|10387|1|484| +2450815|10388|1|144| +2450815|10390|1|685| +2450815|10393|1|532| +2450815|10394|1|473| +2450815|10396|1|406| +2450815|10399|1|528| +2450815|10400|1|283| +2450815|10402|1|975| +2450815|10405|1|| +2450815|10406|1|898| +2450815|10408|1|927| +2450815|10411|1|155| +2450815|10412|1|329| +2450815|10414|1|894| +2450815|10417|1|237| +2450815|10418|1|979| +2450815|10420|1|| +2450815|10423|1|305| +2450815|10424|1|330| +2450815|10426|1|305| +2450815|10429|1|60| +2450815|10430|1|| +2450815|10432|1|842| +2450815|10435|1|955| +2450815|10436|1|| +2450815|10438|1|| +2450815|10441|1|184| +2450815|10442|1|914| +2450815|10444|1|594| +2450815|10447|1|886| +2450815|10448|1|| +2450815|10450|1|191| +2450815|10453|1|304| +2450815|10454|1|101| +2450815|10456|1|| +2450815|10459|1|55| +2450815|10460|1|885| +2450815|10462|1|381| +2450815|10465|1|315| +2450815|10466|1|592| +2450815|10468|1|296| +2450815|10471|1|138| +2450815|10472|1|146| +2450815|10474|1|19| +2450815|10477|1|312| +2450815|10478|1|205| +2450815|10480|1|659| +2450815|10483|1|952| +2450815|10484|1|86| +2450815|10486|1|38| +2450815|10489|1|433| +2450815|10490|1|510| +2450815|10492|1|570| +2450815|10495|1|257| +2450815|10496|1|442| +2450815|10498|1|681| +2450815|10501|1|991| +2450815|10502|1|196| +2450815|10504|1|676| +2450815|10507|1|5| +2450815|10508|1|276| +2450815|10510|1|177| +2450815|10513|1|692| +2450815|10514|1|296| +2450815|10516|1|471| +2450815|10519|1|295| +2450815|10520|1|651| +2450815|10522|1|632| +2450815|10525|1|| +2450815|10526|1|99| +2450815|10528|1|212| +2450815|10531|1|502| +2450815|10532|1|471| +2450815|10534|1|92| +2450815|10537|1|545| +2450815|10538|1|190| +2450815|10540|1|635| +2450815|10543|1|661| +2450815|10544|1|49| +2450815|10546|1|467| +2450815|10549|1|699| +2450815|10550|1|| +2450815|10552|1|17| +2450815|10555|1|597| +2450815|10556|1|304| +2450815|10558|1|238| +2450815|10561|1|645| +2450815|10562|1|928| +2450815|10564|1|440| +2450815|10567|1|837| +2450815|10568|1|351| +2450815|10570|1|366| +2450815|10573|1|622| +2450815|10574|1|390| +2450815|10576|1|| +2450815|10579|1|112| +2450815|10580|1|843| +2450815|10582|1|191| +2450815|10585|1|530| +2450815|10586|1|608| +2450815|10588|1|383| +2450815|10591|1|363| +2450815|10592|1|778| +2450815|10594|1|552| +2450815|10597|1|951| +2450815|10598|1|101| +2450815|10600|1|425| +2450815|10603|1|358| +2450815|10604|1|| +2450815|10606|1|346| +2450815|10609|1|143| +2450815|10610|1|113| +2450815|10612|1|561| +2450815|10615|1|563| +2450815|10616|1|24| +2450815|10618|1|94| +2450815|10621|1|670| +2450815|10622|1|439| +2450815|10624|1|600| +2450815|10627|1|633| +2450815|10628|1|814| +2450815|10630|1|942| +2450815|10633|1|32| +2450815|10634|1|205| +2450815|10636|1|719| +2450815|10639|1|448| +2450815|10640|1|659| +2450815|10642|1|281| +2450815|10645|1|| +2450815|10646|1|| +2450815|10648|1|322| +2450815|10651|1|770| +2450815|10652|1|707| +2450815|10654|1|776| +2450815|10657|1|878| +2450815|10658|1|722| +2450815|10660|1|41| +2450815|10663|1|279| +2450815|10664|1|444| +2450815|10666|1|605| +2450815|10669|1|902| +2450815|10670|1|932| +2450815|10672|1|524| +2450815|10675|1|713| +2450815|10676|1|404| +2450815|10678|1|616| +2450815|10681|1|693| +2450815|10682|1|289| +2450815|10684|1|696| +2450815|10687|1|669| +2450815|10688|1|48| +2450815|10690|1|102| +2450815|10693|1|36| +2450815|10694|1|34| +2450815|10696|1|518| +2450815|10699|1|577| +2450815|10700|1|500| +2450815|10702|1|414| +2450815|10705|1|918| +2450815|10706|1|225| +2450815|10708|1|750| +2450815|10711|1|123| +2450815|10712|1|672| +2450815|10714|1|684| +2450815|10717|1|595| +2450815|10718|1|| +2450815|10720|1|335| +2450815|10723|1|249| +2450815|10724|1|722| +2450815|10726|1|22| +2450815|10729|1|376| +2450815|10730|1|203| +2450815|10732|1|884| +2450815|10735|1|799| +2450815|10736|1|532| +2450815|10738|1|452| +2450815|10741|1|680| +2450815|10742|1|718| +2450815|10744|1|822| +2450815|10747|1|455| +2450815|10748|1|535| +2450815|10750|1|559| +2450815|10753|1|675| +2450815|10754|1|| +2450815|10756|1|794| +2450815|10759|1|585| +2450815|10760|1|712| +2450815|10762|1|| +2450815|10765|1|232| +2450815|10766|1|| +2450815|10768|1|780| +2450815|10771|1|77| +2450815|10772|1|734| +2450815|10774|1|282| +2450815|10777|1|141| +2450815|10778|1|19| +2450815|10780|1|219| +2450815|10783|1|331| +2450815|10784|1|606| +2450815|10786|1|365| +2450815|10789|1|950| +2450815|10790|1|187| +2450815|10792|1|| +2450815|10795|1|280| +2450815|10796|1|645| +2450815|10798|1|456| +2450815|10801|1|945| +2450815|10802|1|961| +2450815|10804|1|828| +2450815|10807|1|80| +2450815|10808|1|618| +2450815|10810|1|975| +2450815|10813|1|819| +2450815|10814|1|765| +2450815|10816|1|231| +2450815|10819|1|372| +2450815|10820|1|28| +2450815|10822|1|849| +2450815|10825|1|746| +2450815|10826|1|531| +2450815|10828|1|661| +2450815|10831|1|837| +2450815|10832|1|577| +2450815|10834|1|38| +2450815|10837|1|261| +2450815|10838|1|| +2450815|10840|1|327| +2450815|10843|1|536| +2450815|10844|1|216| +2450815|10846|1|121| +2450815|10849|1|564| +2450815|10850|1|227| +2450815|10852|1|571| +2450815|10855|1|505| +2450815|10856|1|644| +2450815|10858|1|189| +2450815|10861|1|674| +2450815|10862|1|243| +2450815|10864|1|17| +2450815|10867|1|| +2450815|10868|1|68| +2450815|10870|1|600| +2450815|10873|1|510| +2450815|10874|1|66| +2450815|10876|1|247| +2450815|10879|1|268| +2450815|10880|1|41| +2450815|10882|1|251| +2450815|10885|1|721| +2450815|10886|1|793| +2450815|10888|1|582| +2450815|10891|1|673| +2450815|10892|1|711| +2450815|10894|1|909| +2450815|10897|1|291| +2450815|10898|1|146| +2450815|10900|1|480| +2450815|10903|1|540| +2450815|10904|1|119| +2450815|10906|1|698| +2450815|10909|1|904| +2450815|10910|1|163| +2450815|10912|1|635| +2450815|10915|1|814| +2450815|10916|1|340| +2450815|10918|1|737| +2450815|10921|1|199| +2450815|10922|1|745| +2450815|10924|1|369| +2450815|10927|1|71| +2450815|10928|1|410| +2450815|10930|1|436| +2450815|10933|1|537| +2450815|10934|1|483| +2450815|10936|1|865| +2450815|10939|1|693| +2450815|10940|1|733| +2450815|10942|1|167| +2450815|10945|1|| +2450815|10946|1|695| +2450815|10948|1|860| +2450815|10951|1|81| +2450815|10952|1|106| +2450815|10954|1|658| +2450815|10957|1|| +2450815|10958|1|495| +2450815|10960|1|252| +2450815|10963|1|593| +2450815|10964|1|409| +2450815|10966|1|864| +2450815|10969|1|804| +2450815|10970|1|| +2450815|10972|1|407| +2450815|10975|1|330| +2450815|10976|1|357| +2450815|10978|1|469| +2450815|10981|1|352| +2450815|10982|1|827| +2450815|10984|1|895| +2450815|10987|1|370| +2450815|10988|1|| +2450815|10990|1|654| +2450815|10993|1|538| +2450815|10994|1|93| +2450815|10996|1|625| +2450815|10999|1|276| +2450815|11000|1|566| +2450815|11002|1|529| +2450815|11005|1|430| +2450815|11006|1|622| +2450815|11008|1|675| +2450815|11011|1|46| +2450815|11012|1|332| +2450815|11014|1|460| +2450815|11017|1|423| +2450815|11018|1|228| +2450815|11020|1|904| +2450815|11023|1|843| +2450815|11024|1|261| +2450815|11026|1|963| +2450815|11029|1|383| +2450815|11030|1|713| +2450815|11032|1|678| +2450815|11035|1|902| +2450815|11036|1|667| +2450815|11038|1|| +2450815|11041|1|866| +2450815|11042|1|182| +2450815|11044|1|643| +2450815|11047|1|777| +2450815|11048|1|194| +2450815|11050|1|657| +2450815|11053|1|135| +2450815|11054|1|928| +2450815|11056|1|350| +2450815|11059|1|635| +2450815|11060|1|107| +2450815|11062|1|852| +2450815|11065|1|817| +2450815|11066|1|5| +2450815|11068|1|398| +2450815|11071|1|244| +2450815|11072|1|314| +2450815|11074|1|244| +2450815|11077|1|198| +2450815|11078|1|236| +2450815|11080|1|280| +2450815|11083|1|458| +2450815|11084|1|684| +2450815|11086|1|535| +2450815|11089|1|928| +2450815|11090|1|524| +2450815|11092|1|485| +2450815|11095|1|415| +2450815|11096|1|551| +2450815|11098|1|104| +2450815|11101|1|542| +2450815|11102|1|348| +2450815|11104|1|564| +2450815|11107|1|639| +2450815|11108|1|825| +2450815|11110|1|190| +2450815|11113|1|2| +2450815|11114|1|71| +2450815|11116|1|148| +2450815|11119|1|605| +2450815|11120|1|878| +2450815|11122|1|72| +2450815|11125|1|247| +2450815|11126|1|628| +2450815|11128|1|345| +2450815|11131|1|752| +2450815|11132|1|267| +2450815|11134|1|211| +2450815|11137|1|430| +2450815|11138|1|316| +2450815|11140|1|766| +2450815|11143|1|256| +2450815|11144|1|556| +2450815|11146|1|622| +2450815|11149|1|506| +2450815|11150|1|36| +2450815|11152|1|743| +2450815|11155|1|472| +2450815|11156|1|887| +2450815|11158|1|109| +2450815|11161|1|| +2450815|11162|1|256| +2450815|11164|1|639| +2450815|11167|1|746| +2450815|11168|1|710| +2450815|11170|1|604| +2450815|11173|1|| +2450815|11174|1|208| +2450815|11176|1|418| +2450815|11179|1|913| +2450815|11180|1|938| +2450815|11182|1|380| +2450815|11185|1|834| +2450815|11186|1|108| +2450815|11188|1|79| +2450815|11191|1|859| +2450815|11192|1|929| +2450815|11194|1|200| +2450815|11197|1|| +2450815|11198|1|830| +2450815|11200|1|641| +2450815|11203|1|595| +2450815|11204|1|904| +2450815|11206|1|606| +2450815|11209|1|198| +2450815|11210|1|535| +2450815|11212|1|732| +2450815|11215|1|832| +2450815|11216|1|492| +2450815|11218|1|886| +2450815|11221|1|| +2450815|11222|1|535| +2450815|11224|1|698| +2450815|11227|1|533| +2450815|11228|1|111| +2450815|11230|1|944| +2450815|11233|1|860| +2450815|11234|1|890| +2450815|11236|1|446| +2450815|11239|1|862| +2450815|11240|1|554| +2450815|11242|1|985| +2450815|11245|1|691| +2450815|11246|1|904| +2450815|11248|1|910| +2450815|11251|1|743| +2450815|11252|1|267| +2450815|11254|1|388| +2450815|11257|1|684| +2450815|11258|1|460| +2450815|11260|1|693| +2450815|11263|1|731| +2450815|11264|1|928| +2450815|11266|1|126| +2450815|11269|1|259| +2450815|11270|1|178| +2450815|11272|1|431| +2450815|11275|1|942| +2450815|11276|1|239| +2450815|11278|1|260| +2450815|11281|1|3| +2450815|11282|1|854| +2450815|11284|1|823| +2450815|11287|1|686| +2450815|11288|1|253| +2450815|11290|1|976| +2450815|11293|1|381| +2450815|11294|1|961| +2450815|11296|1|586| +2450815|11299|1|568| +2450815|11300|1|920| +2450815|11302|1|264| +2450815|11305|1|673| +2450815|11306|1|187| +2450815|11308|1|677| +2450815|11311|1|15| +2450815|11312|1|911| +2450815|11314|1|550| +2450815|11317|1|859| +2450815|11318|1|825| +2450815|11320|1|832| +2450815|11323|1|618| +2450815|11324|1|703| +2450815|11326|1|610| +2450815|11329|1|257| +2450815|11330|1|153| +2450815|11332|1|483| +2450815|11335|1|669| +2450815|11336|1|263| +2450815|11338|1|125| +2450815|11341|1|155| +2450815|11342|1|692| +2450815|11344|1|911| +2450815|11347|1|691| +2450815|11348|1|139| +2450815|11350|1|280| +2450815|11353|1|392| +2450815|11354|1|90| +2450815|11356|1|463| +2450815|11359|1|515| +2450815|11360|1|203| +2450815|11362|1|534| +2450815|11365|1|53| +2450815|11366|1|202| +2450815|11368|1|75| +2450815|11371|1|137| +2450815|11372|1|429| +2450815|11374|1|298| +2450815|11377|1|| +2450815|11378|1|801| +2450815|11380|1|772| +2450815|11383|1|55| +2450815|11384|1|300| +2450815|11386|1|| +2450815|11389|1|218| +2450815|11390|1|| +2450815|11392|1|746| +2450815|11395|1|794| +2450815|11396|1|| +2450815|11398|1|495| +2450815|11401|1|489| +2450815|11402|1|525| +2450815|11404|1|511| +2450815|11407|1|933| +2450815|11408|1|331| +2450815|11410|1|790| +2450815|11413|1|17| +2450815|11414|1|412| +2450815|11416|1|380| +2450815|11419|1|385| +2450815|11420|1|887| +2450815|11422|1|150| +2450815|11425|1|24| +2450815|11426|1|575| +2450815|11428|1|90| +2450815|11431|1|937| +2450815|11432|1|482| +2450815|11434|1|534| +2450815|11437|1|276| +2450815|11438|1|785| +2450815|11440|1|320| +2450815|11443|1|838| +2450815|11444|1|396| +2450815|11446|1|227| +2450815|11449|1|890| +2450815|11450|1|331| +2450815|11452|1|764| +2450815|11455|1|347| +2450815|11456|1|223| +2450815|11458|1|653| +2450815|11461|1|551| +2450815|11462|1|229| +2450815|11464|1|154| +2450815|11467|1|598| +2450815|11468|1|561| +2450815|11470|1|393| +2450815|11473|1|814| +2450815|11474|1|380| +2450815|11476|1|175| +2450815|11479|1|737| +2450815|11480|1|597| +2450815|11482|1|313| +2450815|11485|1|81| +2450815|11486|1|893| +2450815|11488|1|594| +2450815|11491|1|859| +2450815|11492|1|232| +2450815|11494|1|| +2450815|11497|1|87| +2450815|11498|1|554| +2450815|11500|1|686| +2450815|11503|1|616| +2450815|11504|1|375| +2450815|11506|1|853| +2450815|11509|1|548| +2450815|11510|1|188| +2450815|11512|1|411| +2450815|11515|1|276| +2450815|11516|1|197| +2450815|11518|1|150| +2450815|11521|1|966| +2450815|11522|1|887| +2450815|11524|1|454| +2450815|11527|1|224| +2450815|11528|1|5| +2450815|11530|1|318| +2450815|11533|1|540| +2450815|11534|1|466| +2450815|11536|1|520| +2450815|11539|1|265| +2450815|11540|1|574| +2450815|11542|1|212| +2450815|11545|1|317| +2450815|11546|1|| +2450815|11548|1|67| +2450815|11551|1|535| +2450815|11552|1|334| +2450815|11554|1|71| +2450815|11557|1|302| +2450815|11558|1|618| +2450815|11560|1|178| +2450815|11563|1|232| +2450815|11564|1|863| +2450815|11566|1|558| +2450815|11569|1|202| +2450815|11570|1|770| +2450815|11572|1|309| +2450815|11575|1|251| +2450815|11576|1|830| +2450815|11578|1|195| +2450815|11581|1|712| +2450815|11582|1|632| +2450815|11584|1|700| +2450815|11587|1|997| +2450815|11588|1|770| +2450815|11590|1|| +2450815|11593|1|10| +2450815|11594|1|831| +2450815|11596|1|400| +2450815|11599|1|341| +2450815|11600|1|55| +2450815|11602|1|393| +2450815|11605|1|973| +2450815|11606|1|432| +2450815|11608|1|606| +2450815|11611|1|423| +2450815|11612|1|785| +2450815|11614|1|494| +2450815|11617|1|23| +2450815|11618|1|511| +2450815|11620|1|787| +2450815|11623|1|315| +2450815|11624|1|26| +2450815|11626|1|167| +2450815|11629|1|124| +2450815|11630|1|404| +2450815|11632|1|235| +2450815|11635|1|| +2450815|11636|1|287| +2450815|11638|1|808| +2450815|11641|1|765| +2450815|11642|1|355| +2450815|11644|1|539| +2450815|11647|1|544| +2450815|11648|1|296| +2450815|11650|1|509| +2450815|11653|1|641| +2450815|11654|1|635| +2450815|11656|1|473| +2450815|11659|1|442| +2450815|11660|1|292| +2450815|11662|1|628| +2450815|11665|1|237| +2450815|11666|1|191| +2450815|11668|1|283| +2450815|11671|1|648| +2450815|11672|1|304| +2450815|11674|1|659| +2450815|11677|1|35| +2450815|11678|1|731| +2450815|11680|1|438| +2450815|11683|1|138| +2450815|11684|1|171| +2450815|11686|1|830| +2450815|11689|1|370| +2450815|11690|1|890| +2450815|11692|1|273| +2450815|11695|1|578| +2450815|11696|1|960| +2450815|11698|1|477| +2450815|11701|1|954| +2450815|11702|1|15| +2450815|11704|1|1000| +2450815|11707|1|167| +2450815|11708|1|| +2450815|11710|1|631| +2450815|11713|1|650| +2450815|11714|1|49| +2450815|11716|1|237| +2450815|11719|1|776| +2450815|11720|1|678| +2450815|11722|1|253| +2450815|11725|1|873| +2450815|11726|1|283| +2450815|11728|1|609| +2450815|11731|1|518| +2450815|11732|1|154| +2450815|11734|1|898| +2450815|11737|1|150| +2450815|11738|1|546| +2450815|11740|1|201| +2450815|11743|1|816| +2450815|11744|1|228| +2450815|11746|1|828| +2450815|11749|1|39| +2450815|11750|1|29| +2450815|11752|1|95| +2450815|11755|1|119| +2450815|11756|1|48| +2450815|11758|1|799| +2450815|11761|1|| +2450815|11762|1|| +2450815|11764|1|219| +2450815|11767|1|356| +2450815|11768|1|583| +2450815|11770|1|866| +2450815|11773|1|31| +2450815|11774|1|547| +2450815|11776|1|762| +2450815|11779|1|231| +2450815|11780|1|725| +2450815|11782|1|980| +2450815|11785|1|882| +2450815|11786|1|92| +2450815|11788|1|810| +2450815|11791|1|70| +2450815|11792|1|385| +2450815|11794|1|502| +2450815|11797|1|564| +2450815|11798|1|81| +2450815|11800|1|298| +2450815|11803|1|784| +2450815|11804|1|653| +2450815|11806|1|818| +2450815|11809|1|586| +2450815|11810|1|850| +2450815|11812|1|259| +2450815|11815|1|339| +2450815|11816|1|865| +2450815|11818|1|891| +2450815|11821|1|| +2450815|11822|1|778| +2450815|11824|1|389| +2450815|11827|1|361| +2450815|11828|1|806| +2450815|11830|1|327| +2450815|11833|1|726| +2450815|11834|1|862| +2450815|11836|1|664| +2450815|11839|1|196| +2450815|11840|1|876| +2450815|11842|1|74| +2450815|11845|1|306| +2450815|11846|1|5| +2450815|11848|1|707| +2450815|11851|1|252| +2450815|11852|1|323| +2450815|11854|1|686| +2450815|11857|1|941| +2450815|11858|1|| +2450815|11860|1|601| +2450815|11863|1|| +2450815|11864|1|535| +2450815|11866|1|496| +2450815|11869|1|616| +2450815|11870|1|99| +2450815|11872|1|677| +2450815|11875|1|191| +2450815|11876|1|746| +2450815|11878|1|528| +2450815|11881|1|527| +2450815|11882|1|469| +2450815|11884|1|50| +2450815|11887|1|87| +2450815|11888|1|992| +2450815|11890|1|778| +2450815|11893|1|994| +2450815|11894|1|712| +2450815|11896|1|21| +2450815|11899|1|800| +2450815|11900|1|393| +2450815|11902|1|| +2450815|11905|1|178| +2450815|11906|1|139| +2450815|11908|1|404| +2450815|11911|1|93| +2450815|11912|1|586| +2450815|11914|1|16| +2450815|11917|1|260| +2450815|11918|1|490| +2450815|11920|1|475| +2450815|11923|1|938| +2450815|11924|1|342| +2450815|11926|1|934| +2450815|11929|1|10| +2450815|11930|1|498| +2450815|11932|1|701| +2450815|11935|1|386| +2450815|11936|1|788| +2450815|11938|1|482| +2450815|11941|1|503| +2450815|11942|1|| +2450815|11944|1|104| +2450815|11947|1|535| +2450815|11948|1|531| +2450815|11950|1|35| +2450815|11953|1|659| +2450815|11954|1|63| +2450815|11956|1|393| +2450815|11959|1|125| +2450815|11960|1|243| +2450815|11962|1|232| +2450815|11965|1|612| +2450815|11966|1|42| +2450815|11968|1|561| +2450815|11971|1|490| +2450815|11972|1|162| +2450815|11974|1|304| +2450815|11977|1|719| +2450815|11978|1|673| +2450815|11980|1|103| +2450815|11983|1|428| +2450815|11984|1|196| +2450815|11986|1|854| +2450815|11989|1|774| +2450815|11990|1|348| +2450815|11992|1|124| +2450815|11995|1|417| +2450815|11996|1|1000| +2450815|11998|1|128| +2450815|12001|1|523| +2450815|12002|1|303| +2450815|12004|1|907| +2450815|12007|1|717| +2450815|12008|1|391| +2450815|12010|1|967| +2450815|12013|1|32| +2450815|12014|1|658| +2450815|12016|1|983| +2450815|12019|1|235| +2450815|12020|1|398| +2450815|12022|1|953| +2450815|12025|1|584| +2450815|12026|1|711| +2450815|12028|1|643| +2450815|12031|1|516| +2450815|12032|1|362| +2450815|12034|1|596| +2450815|12037|1|401| +2450815|12038|1|25| +2450815|12040|1|806| +2450815|12043|1|188| +2450815|12044|1|261| +2450815|12046|1|951| +2450815|12049|1|871| +2450815|12050|1|356| +2450815|12052|1|995| +2450815|12055|1|586| +2450815|12056|1|590| +2450815|12058|1|954| +2450815|12061|1|172| +2450815|12062|1|557| +2450815|12064|1|189| +2450815|12067|1|| +2450815|12068|1|872| +2450815|12070|1|196| +2450815|12073|1|357| +2450815|12074|1|534| +2450815|12076|1|7| +2450815|12079|1|468| +2450815|12080|1|318| +2450815|12082|1|300| +2450815|12085|1|152| +2450815|12086|1|22| +2450815|12088|1|231| +2450815|12091|1|322| +2450815|12092|1|546| +2450815|12094|1|944| +2450815|12097|1|723| +2450815|12098|1|993| +2450815|12100|1|705| +2450815|12103|1|973| +2450815|12104|1|804| +2450815|12106|1|610| +2450815|12109|1|880| +2450815|12110|1|| +2450815|12112|1|767| +2450815|12115|1|| +2450815|12116|1|75| +2450815|12118|1|781| +2450815|12121|1|691| +2450815|12122|1|198| +2450815|12124|1|276| +2450815|12127|1|506| +2450815|12128|1|296| +2450815|12130|1|953| +2450815|12133|1|929| +2450815|12134|1|820| +2450815|12136|1|156| +2450815|12139|1|289| +2450815|12140|1|484| +2450815|12142|1|970| +2450815|12145|1|852| +2450815|12146|1|571| +2450815|12148|1|494| +2450815|12151|1|116| +2450815|12152|1|409| +2450815|12154|1|426| +2450815|12157|1|95| +2450815|12158|1|94| +2450815|12160|1|445| +2450815|12163|1|| +2450815|12164|1|841| +2450815|12166|1|508| +2450815|12169|1|861| +2450815|12170|1|991| +2450815|12172|1|361| +2450815|12175|1|581| +2450815|12176|1|364| +2450815|12178|1|474| +2450815|12181|1|110| +2450815|12182|1|763| +2450815|12184|1|902| +2450815|12187|1|26| +2450815|12188|1|608| +2450815|12190|1|330| +2450815|12193|1|858| +2450815|12194|1|823| +2450815|12196|1|572| +2450815|12199|1|316| +2450815|12200|1|804| +2450815|12202|1|832| +2450815|12205|1|876| +2450815|12206|1|357| +2450815|12208|1|603| +2450815|12211|1|298| +2450815|12212|1|97| +2450815|12214|1|459| +2450815|12217|1|777| +2450815|12218|1|642| +2450815|12220|1|450| +2450815|12223|1|809| +2450815|12224|1|559| +2450815|12226|1|696| +2450815|12229|1|260| +2450815|12230|1|953| +2450815|12232|1|637| +2450815|12235|1|926| +2450815|12236|1|908| +2450815|12238|1|658| +2450815|12241|1|| +2450815|12242|1|399| +2450815|12244|1|274| +2450815|12247|1|870| +2450815|12248|1|527| +2450815|12250|1|309| +2450815|12253|1|208| +2450815|12254|1|684| +2450815|12256|1|775| +2450815|12259|1|708| +2450815|12260|1|724| +2450815|12262|1|796| +2450815|12265|1|659| +2450815|12266|1|324| +2450815|12268|1|943| +2450815|12271|1|462| +2450815|12272|1|800| +2450815|12274|1|337| +2450815|12277|1|784| +2450815|12278|1|891| +2450815|12280|1|419| +2450815|12283|1|172| +2450815|12284|1|21| +2450815|12286|1|929| +2450815|12289|1|497| +2450815|12290|1|578| +2450815|12292|1|224| +2450815|12295|1|743| +2450815|12296|1|246| +2450815|12298|1|256| +2450815|12301|1|296| +2450815|12302|1|628| +2450815|12304|1|379| +2450815|12307|1|| +2450815|12308|1|256| +2450815|12310|1|42| +2450815|12313|1|314| +2450815|12314|1|19| +2450815|12316|1|361| +2450815|12319|1|644| +2450815|12320|1|737| +2450815|12322|1|183| +2450815|12325|1|5| +2450815|12326|1|52| +2450815|12328|1|397| +2450815|12331|1|868| +2450815|12332|1|196| +2450815|12334|1|776| +2450815|12337|1|737| +2450815|12338|1|921| +2450815|12340|1|772| +2450815|12343|1|598| +2450815|12344|1|219| +2450815|12346|1|611| +2450815|12349|1|7| +2450815|12350|1|574| +2450815|12352|1|219| +2450815|12355|1|196| +2450815|12356|1|833| +2450815|12358|1|944| +2450815|12361|1|878| +2450815|12362|1|800| +2450815|12364|1|334| +2450815|12367|1|451| +2450815|12368|1|122| +2450815|12370|1|212| +2450815|12373|1|992| +2450815|12374|1|573| +2450815|12376|1|604| +2450815|12379|1|947| +2450815|12380|1|788| +2450815|12382|1|799| +2450815|12385|1|451| +2450815|12386|1|390| +2450815|12388|1|387| +2450815|12391|1|124| +2450815|12392|1|962| +2450815|12394|1|123| +2450815|12397|1|698| +2450815|12398|1|| +2450815|12400|1|410| +2450815|12403|1|47| +2450815|12404|1|177| +2450815|12406|1|788| +2450815|12409|1|136| +2450815|12410|1|304| +2450815|12412|1|536| +2450815|12415|1|924| +2450815|12416|1|657| +2450815|12418|1|453| +2450815|12421|1|185| +2450815|12422|1|371| +2450815|12424|1|971| +2450815|12427|1|845| +2450815|12428|1|461| +2450815|12430|1|971| +2450815|12433|1|471| +2450815|12434|1|410| +2450815|12436|1|291| +2450815|12439|1|275| +2450815|12440|1|199| +2450815|12442|1|171| +2450815|12445|1|497| +2450815|12446|1|607| +2450815|12448|1|364| +2450815|12451|1|437| +2450815|12452|1|316| +2450815|12454|1|703| +2450815|12457|1|534| +2450815|12458|1|263| +2450815|12460|1|870| +2450815|12463|1|556| +2450815|12464|1|698| +2450815|12466|1|431| +2450815|12469|1|669| +2450815|12470|1|458| +2450815|12472|1|479| +2450815|12475|1|56| +2450815|12476|1|632| +2450815|12478|1|145| +2450815|12481|1|228| +2450815|12482|1|976| +2450815|12484|1|| +2450815|12487|1|705| +2450815|12488|1|270| +2450815|12490|1|498| +2450815|12493|1|| +2450815|12494|1|121| +2450815|12496|1|143| +2450815|12499|1|70| +2450815|12500|1|168| +2450815|12502|1|307| +2450815|12505|1|915| +2450815|12506|1|86| +2450815|12508|1|100| +2450815|12511|1|526| +2450815|12512|1|223| +2450815|12514|1|283| +2450815|12517|1|112| +2450815|12518|1|163| +2450815|12520|1|311| +2450815|12523|1|272| +2450815|12524|1|912| +2450815|12526|1|79| +2450815|12529|1|694| +2450815|12530|1|134| +2450815|12532|1|558| +2450815|12535|1|346| +2450815|12536|1|407| +2450815|12538|1|103| +2450815|12541|1|80| +2450815|12542|1|492| +2450815|12544|1|308| +2450815|12547|1|368| +2450815|12548|1|78| +2450815|12550|1|859| +2450815|12553|1|52| +2450815|12554|1|755| +2450815|12556|1|786| +2450815|12559|1|742| +2450815|12560|1|237| +2450815|12562|1|699| +2450815|12565|1|215| +2450815|12566|1|836| +2450815|12568|1|19| +2450815|12571|1|462| +2450815|12572|1|751| +2450815|12574|1|112| +2450815|12577|1|689| +2450815|12578|1|179| +2450815|12580|1|60| +2450815|12583|1|955| +2450815|12584|1|755| +2450815|12586|1|927| +2450815|12589|1|40| +2450815|12590|1|230| +2450815|12592|1|164| +2450815|12595|1|352| +2450815|12596|1|954| +2450815|12598|1|780| +2450815|12601|1|505| +2450815|12602|1|45| +2450815|12604|1|380| +2450815|12607|1|943| +2450815|12608|1|532| +2450815|12610|1|895| +2450815|12613|1|211| +2450815|12614|1|17| +2450815|12616|1|365| +2450815|12619|1|451| +2450815|12620|1|733| +2450815|12622|1|264| +2450815|12625|1|275| +2450815|12626|1|308| +2450815|12628|1|602| +2450815|12631|1|32| +2450815|12632|1|496| +2450815|12634|1|102| +2450815|12637|1|992| +2450815|12638|1|832| +2450815|12640|1|970| +2450815|12643|1|68| +2450815|12644|1|642| +2450815|12646|1|944| +2450815|12649|1|494| +2450815|12650|1|841| +2450815|12652|1|182| +2450815|12655|1|579| +2450815|12656|1|170| +2450815|12658|1|513| +2450815|12661|1|20| +2450815|12662|1|223| +2450815|12664|1|741| +2450815|12667|1|843| +2450815|12668|1|391| +2450815|12670|1|456| +2450815|12673|1|191| +2450815|12674|1|657| +2450815|12676|1|460| +2450815|12679|1|414| +2450815|12680|1|878| +2450815|12682|1|122| +2450815|12685|1|887| +2450815|12686|1|522| +2450815|12688|1|83| +2450815|12691|1|528| +2450815|12692|1|165| +2450815|12694|1|535| +2450815|12697|1|625| +2450815|12698|1|60| +2450815|12700|1|506| +2450815|12703|1|224| +2450815|12704|1|173| +2450815|12706|1|288| +2450815|12709|1|94| +2450815|12710|1|58| +2450815|12712|1|668| +2450815|12715|1|305| +2450815|12716|1|428| +2450815|12718|1|53| +2450815|12721|1|334| +2450815|12722|1|399| +2450815|12724|1|956| +2450815|12727|1|211| +2450815|12728|1|995| +2450815|12730|1|406| +2450815|12733|1|783| +2450815|12734|1|679| +2450815|12736|1|319| +2450815|12739|1|227| +2450815|12740|1|228| +2450815|12742|1|634| +2450815|12745|1|53| +2450815|12746|1|829| +2450815|12748|1|669| +2450815|12751|1|887| +2450815|12752|1|623| +2450815|12754|1|492| +2450815|12757|1|235| +2450815|12758|1|830| +2450815|12760|1|134| +2450815|12763|1|981| +2450815|12764|1|155| +2450815|12766|1|251| +2450815|12769|1|991| +2450815|12770|1|272| +2450815|12772|1|179| +2450815|12775|1|622| +2450815|12776|1|954| +2450815|12778|1|801| +2450815|12781|1|695| +2450815|12782|1|619| +2450815|12784|1|877| +2450815|12787|1|42| +2450815|12788|1|113| +2450815|12790|1|974| +2450815|12793|1|653| +2450815|12794|1|173| +2450815|12796|1|388| +2450815|12799|1|133| +2450815|12800|1|642| +2450815|12802|1|409| +2450815|12805|1|504| +2450815|12806|1|463| +2450815|12808|1|733| +2450815|12811|1|405| +2450815|12812|1|238| +2450815|12814|1|600| +2450815|12817|1|859| +2450815|12818|1|918| +2450815|12820|1|274| +2450815|12823|1|590| +2450815|12824|1|662| +2450815|12826|1|364| +2450815|12829|1|553| +2450815|12830|1|608| +2450815|12832|1|133| +2450815|12835|1|524| +2450815|12836|1|826| +2450815|12838|1|960| +2450815|12841|1|214| +2450815|12842|1|612| +2450815|12844|1|838| +2450815|12847|1|347| +2450815|12848|1|221| +2450815|12850|1|798| +2450815|12853|1|193| +2450815|12854|1|813| +2450815|12856|1|963| +2450815|12859|1|461| +2450815|12860|1|65| +2450815|12862|1|479| +2450815|12865|1|320| +2450815|12866|1|690| +2450815|12868|1|645| +2450815|12871|1|430| +2450815|12872|1|532| +2450815|12874|1|716| +2450815|12877|1|622| +2450815|12878|1|623| +2450815|12880|1|714| +2450815|12883|1|693| +2450815|12884|1|231| +2450815|12886|1|602| +2450815|12889|1|492| +2450815|12890|1|767| +2450815|12892|1|213| +2450815|12895|1|792| +2450815|12896|1|475| +2450815|12898|1|392| +2450815|12901|1|343| +2450815|12902|1|385| +2450815|12904|1|928| +2450815|12907|1|276| +2450815|12908|1|302| +2450815|12910|1|393| +2450815|12913|1|737| +2450815|12914|1|434| +2450815|12916|1|834| +2450815|12919|1|565| +2450815|12920|1|726| +2450815|12922|1|216| +2450815|12925|1|303| +2450815|12926|1|| +2450815|12928|1|367| +2450815|12931|1|206| +2450815|12932|1|926| +2450815|12934|1|447| +2450815|12937|1|| +2450815|12938|1|568| +2450815|12940|1|484| +2450815|12943|1|524| +2450815|12944|1|40| +2450815|12946|1|453| +2450815|12949|1|| +2450815|12950|1|| +2450815|12952|1|553| +2450815|12955|1|851| +2450815|12956|1|113| +2450815|12958|1|723| +2450815|12961|1|376| +2450815|12962|1|92| +2450815|12964|1|863| +2450815|12967|1|854| +2450815|12968|1|930| +2450815|12970|1|369| +2450815|12973|1|571| +2450815|12974|1|486| +2450815|12976|1|838| +2450815|12979|1|84| +2450815|12980|1|405| +2450815|12982|1|844| +2450815|12985|1|389| +2450815|12986|1|131| +2450815|12988|1|567| +2450815|12991|1|| +2450815|12992|1|898| +2450815|12994|1|306| +2450815|12997|1|261| +2450815|12998|1|790| +2450815|13000|1|858| +2450815|13003|1|901| +2450815|13004|1|491| +2450815|13006|1|774| +2450815|13009|1|569| +2450815|13010|1|835| +2450815|13012|1|289| +2450815|13015|1|193| +2450815|13016|1|918| +2450815|13018|1|451| +2450815|13021|1|779| +2450815|13022|1|122| +2450815|13024|1|557| +2450815|13027|1|586| +2450815|13028|1|567| +2450815|13030|1|23| +2450815|13033|1|917| +2450815|13034|1|296| +2450815|13036|1|600| +2450815|13039|1|| +2450815|13040|1|776| +2450815|13042|1|679| +2450815|13045|1|856| +2450815|13046|1|32| +2450815|13048|1|155| +2450815|13051|1|556| +2450815|13052|1|| +2450815|13054|1|| +2450815|13057|1|213| +2450815|13058|1|259| +2450815|13060|1|656| +2450815|13063|1|225| +2450815|13064|1|765| +2450815|13066|1|761| +2450815|13069|1|827| +2450815|13070|1|544| +2450815|13072|1|27| +2450815|13075|1|119| +2450815|13076|1|792| +2450815|13078|1|665| +2450815|13081|1|740| +2450815|13082|1|164| +2450815|13084|1|731| +2450815|13087|1|562| +2450815|13088|1|662| +2450815|13090|1|98| +2450815|13093|1|234| +2450815|13094|1|355| +2450815|13096|1|668| +2450815|13099|1|| +2450815|13100|1|13| +2450815|13102|1|476| +2450815|13105|1|213| +2450815|13106|1|765| +2450815|13108|1|723| +2450815|13111|1|96| +2450815|13112|1|125| +2450815|13114|1|642| +2450815|13117|1|| +2450815|13118|1|427| +2450815|13120|1|357| +2450815|13123|1|533| +2450815|13124|1|635| +2450815|13126|1|| +2450815|13129|1|769| +2450815|13130|1|94| +2450815|13132|1|828| +2450815|13135|1|| +2450815|13136|1|886| +2450815|13138|1|967| +2450815|13141|1|989| +2450815|13142|1|512| +2450815|13144|1|335| +2450815|13147|1|471| +2450815|13148|1|210| +2450815|13150|1|598| +2450815|13153|1|4| +2450815|13154|1|606| +2450815|13156|1|426| +2450815|13159|1|822| +2450815|13160|1|491| +2450815|13162|1|440| +2450815|13165|1|48| +2450815|13166|1|866| +2450815|13168|1|317| +2450815|13171|1|941| +2450815|13172|1|341| +2450815|13174|1|600| +2450815|13177|1|413| +2450815|13178|1|654| +2450815|13180|1|189| +2450815|13183|1|455| +2450815|13184|1|164| +2450815|13186|1|620| +2450815|13189|1|714| +2450815|13190|1|538| +2450815|13192|1|356| +2450815|13195|1|860| +2450815|13196|1|131| +2450815|13198|1|84| +2450815|13201|1|656| +2450815|13202|1|79| +2450815|13204|1|166| +2450815|13207|1|856| +2450815|13208|1|956| +2450815|13210|1|251| +2450815|13213|1|865| +2450815|13214|1|673| +2450815|13216|1|750| +2450815|13219|1|477| +2450815|13220|1|766| +2450815|13222|1|667| +2450815|13225|1|679| +2450815|13226|1|435| +2450815|13228|1|387| +2450815|13231|1|694| +2450815|13232|1|131| +2450815|13234|1|489| +2450815|13237|1|353| +2450815|13238|1|924| +2450815|13240|1|285| +2450815|13243|1|915| +2450815|13244|1|871| +2450815|13246|1|26| +2450815|13249|1|215| +2450815|13250|1|240| +2450815|13252|1|15| +2450815|13255|1|70| +2450815|13256|1|200| +2450815|13258|1|945| +2450815|13261|1|162| +2450815|13262|1|824| +2450815|13264|1|180| +2450815|13267|1|482| +2450815|13268|1|983| +2450815|13270|1|133| +2450815|13273|1|548| +2450815|13274|1|148| +2450815|13276|1|| +2450815|13279|1|737| +2450815|13280|1|| +2450815|13282|1|| +2450815|13285|1|251| +2450815|13286|1|700| +2450815|13288|1|412| +2450815|13291|1|229| +2450815|13292|1|627| +2450815|13294|1|751| +2450815|13297|1|636| +2450815|13298|1|100| +2450815|13300|1|359| +2450815|13303|1|649| +2450815|13304|1|| +2450815|13306|1|484| +2450815|13309|1|204| +2450815|13310|1|973| +2450815|13312|1|319| +2450815|13315|1|2| +2450815|13316|1|778| +2450815|13318|1|133| +2450815|13321|1|65| +2450815|13322|1|219| +2450815|13324|1|816| +2450815|13327|1|575| +2450815|13328|1|87| +2450815|13330|1|213| +2450815|13333|1|133| +2450815|13334|1|374| +2450815|13336|1|518| +2450815|13339|1|314| +2450815|13340|1|69| +2450815|13342|1|679| +2450815|13345|1|162| +2450815|13346|1|127| +2450815|13348|1|233| +2450815|13351|1|989| +2450815|13352|1|393| +2450815|13354|1|189| +2450815|13357|1|979| +2450815|13358|1|859| +2450815|13360|1|551| +2450815|13363|1|438| +2450815|13364|1|639| +2450815|13366|1|635| +2450815|13369|1|| +2450815|13370|1|576| +2450815|13372|1|234| +2450815|13375|1|322| +2450815|13376|1|991| +2450815|13378|1|928| +2450815|13381|1|221| +2450815|13382|1|904| +2450815|13384|1|355| +2450815|13387|1|811| +2450815|13388|1|343| +2450815|13390|1|898| +2450815|13393|1|386| +2450815|13394|1|119| +2450815|13396|1|512| +2450815|13399|1|974| +2450815|13400|1|751| +2450815|13402|1|| +2450815|13405|1|360| +2450815|13406|1|| +2450815|13408|1|977| +2450815|13411|1|484| +2450815|13412|1|839| +2450815|13414|1|519| +2450815|13417|1|879| +2450815|13418|1|690| +2450815|13420|1|779| +2450815|13423|1|355| +2450815|13424|1|714| +2450815|13426|1|61| +2450815|13429|1|325| +2450815|13430|1|723| +2450815|13432|1|200| +2450815|13435|1|34| +2450815|13436|1|375| +2450815|13438|1|96| +2450815|13441|1|17| +2450815|13442|1|| +2450815|13444|1|245| +2450815|13447|1|703| +2450815|13448|1|505| +2450815|13450|1|405| +2450815|13453|1|244| +2450815|13454|1|239| +2450815|13456|1|971| +2450815|13459|1|| +2450815|13460|1|420| +2450815|13462|1|341| +2450815|13465|1|604| +2450815|13466|1|587| +2450815|13468|1|748| +2450815|13471|1|111| +2450815|13472|1|103| +2450815|13474|1|822| +2450815|13477|1|| +2450815|13478|1|| +2450815|13480|1|968| +2450815|13483|1|429| +2450815|13484|1|967| +2450815|13486|1|426| +2450815|13489|1|208| +2450815|13490|1|407| +2450815|13492|1|905| +2450815|13495|1|582| +2450815|13496|1|739| +2450815|13498|1|851| +2450815|13501|1|922| +2450815|13502|1|214| +2450815|13504|1|409| +2450815|13507|1|774| +2450815|13508|1|295| +2450815|13510|1|25| +2450815|13513|1|519| +2450815|13514|1|424| +2450815|13516|1|148| +2450815|13519|1|95| +2450815|13520|1|346| +2450815|13522|1|480| +2450815|13525|1|305| +2450815|13526|1|136| +2450815|13528|1|304| +2450815|13531|1|171| +2450815|13532|1|215| +2450815|13534|1|392| +2450815|13537|1|475| +2450815|13538|1|243| +2450815|13540|1|672| +2450815|13543|1|557| +2450815|13544|1|571| +2450815|13546|1|513| +2450815|13549|1|864| +2450815|13550|1|540| +2450815|13552|1|70| +2450815|13555|1|849| +2450815|13556|1|115| +2450815|13558|1|565| +2450815|13561|1|832| +2450815|13562|1|0| +2450815|13564|1|82| +2450815|13567|1|896| +2450815|13568|1|983| +2450815|13570|1|131| +2450815|13573|1|100| +2450815|13574|1|695| +2450815|13576|1|135| +2450815|13579|1|491| +2450815|13580|1|873| +2450815|13582|1|123| +2450815|13585|1|37| +2450815|13586|1|1000| +2450815|13588|1|494| +2450815|13591|1|164| +2450815|13592|1|120| +2450815|13594|1|579| +2450815|13597|1|| +2450815|13598|1|| +2450815|13600|1|973| +2450815|13603|1|570| +2450815|13604|1|638| +2450815|13606|1|545| +2450815|13609|1|593| +2450815|13610|1|570| +2450815|13612|1|290| +2450815|13615|1|990| +2450815|13616|1|27| +2450815|13618|1|747| +2450815|13621|1|516| +2450815|13622|1|536| +2450815|13624|1|802| +2450815|13627|1|801| +2450815|13628|1|947| +2450815|13630|1|152| +2450815|13633|1|809| +2450815|13634|1|| +2450815|13636|1|176| +2450815|13639|1|960| +2450815|13640|1|348| +2450815|13642|1|824| +2450815|13645|1|88| +2450815|13646|1|215| +2450815|13648|1|952| +2450815|13651|1|199| +2450815|13652|1|936| +2450815|13654|1|570| +2450815|13657|1|355| +2450815|13658|1|958| +2450815|13660|1|407| +2450815|13663|1|301| +2450815|13664|1|248| +2450815|13666|1|552| +2450815|13669|1|532| +2450815|13670|1|993| +2450815|13672|1|917| +2450815|13675|1|273| +2450815|13676|1|229| +2450815|13678|1|223| +2450815|13681|1|283| +2450815|13682|1|248| +2450815|13684|1|| +2450815|13687|1|| +2450815|13688|1|125| +2450815|13690|1|278| +2450815|13693|1|960| +2450815|13694|1|243| +2450815|13696|1|| +2450815|13699|1|146| +2450815|13700|1|978| +2450815|13702|1|200| +2450815|13705|1|978| +2450815|13706|1|331| +2450815|13708|1|163| +2450815|13711|1|769| +2450815|13712|1|503| +2450815|13714|1|653| +2450815|13717|1|160| +2450815|13718|1|831| +2450815|13720|1|975| +2450815|13723|1|922| +2450815|13724|1|773| +2450815|13726|1|115| +2450815|13729|1|520| +2450815|13730|1|622| +2450815|13732|1|369| +2450815|13735|1|81| +2450815|13736|1|268| +2450815|13738|1|732| +2450815|13741|1|682| +2450815|13742|1|798| +2450815|13744|1|452| +2450815|13747|1|107| +2450815|13748|1|255| +2450815|13750|1|864| +2450815|13753|1|181| +2450815|13754|1|568| +2450815|13756|1|505| +2450815|13759|1|988| +2450815|13760|1|720| +2450815|13762|1|98| +2450815|13765|1|769| +2450815|13766|1|743| +2450815|13768|1|394| +2450815|13771|1|563| +2450815|13772|1|624| +2450815|13774|1|272| +2450815|13777|1|342| +2450815|13778|1|437| +2450815|13780|1|917| +2450815|13783|1|475| +2450815|13784|1|734| +2450815|13786|1|868| +2450815|13789|1|172| +2450815|13790|1|740| +2450815|13792|1|441| +2450815|13795|1|843| +2450815|13796|1|144| +2450815|13798|1|204| +2450815|13801|1|102| +2450815|13802|1|997| +2450815|13804|1|525| +2450815|13807|1|| +2450815|13808|1|572| +2450815|13810|1|245| +2450815|13813|1|804| +2450815|13814|1|300| +2450815|13816|1|619| +2450815|13819|1|733| +2450815|13820|1|699| +2450815|13822|1|971| +2450815|13825|1|906| +2450815|13826|1|554| +2450815|13828|1|579| +2450815|13831|1|| +2450815|13832|1|176| +2450815|13834|1|904| +2450815|13837|1|441| +2450815|13838|1|855| +2450815|13840|1|118| +2450815|13843|1|65| +2450815|13844|1|633| +2450815|13846|1|| +2450815|13849|1|743| +2450815|13850|1|150| +2450815|13852|1|188| +2450815|13855|1|305| +2450815|13856|1|| +2450815|13858|1|59| +2450815|13861|1|852| +2450815|13862|1|653| +2450815|13864|1|707| +2450815|13867|1|433| +2450815|13868|1|924| +2450815|13870|1|341| +2450815|13873|1|825| +2450815|13874|1|285| +2450815|13876|1|350| +2450815|13879|1|| +2450815|13880|1|| +2450815|13882|1|584| +2450815|13885|1|10| +2450815|13886|1|316| +2450815|13888|1|982| +2450815|13891|1|421| +2450815|13892|1|| +2450815|13894|1|376| +2450815|13897|1|929| +2450815|13898|1|443| +2450815|13900|1|984| +2450815|13903|1|618| +2450815|13904|1|126| +2450815|13906|1|963| +2450815|13909|1|138| +2450815|13910|1|164| +2450815|13912|1|654| +2450815|13915|1|714| +2450815|13916|1|675| +2450815|13918|1|196| +2450815|13921|1|426| +2450815|13922|1|165| +2450815|13924|1|520| +2450815|13927|1|419| +2450815|13928|1|173| +2450815|13930|1|181| +2450815|13933|1|675| +2450815|13934|1|421| +2450815|13936|1|438| +2450815|13939|1|608| +2450815|13940|1|731| +2450815|13942|1|454| +2450815|13945|1|819| +2450815|13946|1|647| +2450815|13948|1|489| +2450815|13951|1|357| +2450815|13952|1|827| +2450815|13954|1|415| +2450815|13957|1|970| +2450815|13958|1|501| +2450815|13960|1|802| +2450815|13963|1|647| +2450815|13964|1|665| +2450815|13966|1|38| +2450815|13969|1|63| +2450815|13970|1|993| +2450815|13972|1|788| +2450815|13975|1|304| +2450815|13976|1|536| +2450815|13978|1|| +2450815|13981|1|559| +2450815|13982|1|489| +2450815|13984|1|962| +2450815|13987|1|506| +2450815|13988|1|222| +2450815|13990|1|892| +2450815|13993|1|755| +2450815|13994|1|303| +2450815|13996|1|880| +2450815|13999|1|847| +2450815|14000|1|120| +2450815|14002|1|159| +2450815|14005|1|100| +2450815|14006|1|557| +2450815|14008|1|800| +2450815|14011|1|529| +2450815|14012|1|710| +2450815|14014|1|| +2450815|14017|1|288| +2450815|14018|1|393| +2450815|14020|1|120| +2450815|14023|1|659| +2450815|14024|1|844| +2450815|14026|1|266| +2450815|14029|1|502| +2450815|14030|1|943| +2450815|14032|1|443| +2450815|14035|1|835| +2450815|14036|1|704| +2450815|14038|1|734| +2450815|14041|1|193| +2450815|14042|1|524| +2450815|14044|1|718| +2450815|14047|1|493| +2450815|14048|1|741| +2450815|14050|1|536| +2450815|14053|1|778| +2450815|14054|1|614| +2450815|14056|1|270| +2450815|14059|1|168| +2450815|14060|1|627| +2450815|14062|1|169| +2450815|14065|1|511| +2450815|14066|1|66| +2450815|14068|1|194| +2450815|14071|1|137| +2450815|14072|1|116| +2450815|14074|1|430| +2450815|14077|1|989| +2450815|14078|1|397| +2450815|14080|1|432| +2450815|14083|1|857| +2450815|14084|1|911| +2450815|14086|1|946| +2450815|14089|1|754| +2450815|14090|1|6| +2450815|14092|1|745| +2450815|14095|1|253| +2450815|14096|1|299| +2450815|14098|1|947| +2450815|14101|1|620| +2450815|14102|1|435| +2450815|14104|1|920| +2450815|14107|1|170| +2450815|14108|1|237| +2450815|14110|1|326| +2450815|14113|1|151| +2450815|14114|1|32| +2450815|14116|1|282| +2450815|14119|1|675| +2450815|14120|1|109| +2450815|14122|1|121| +2450815|14125|1|359| +2450815|14126|1|757| +2450815|14128|1|477| +2450815|14131|1|609| +2450815|14132|1|849| +2450815|14134|1|167| +2450815|14137|1|503| +2450815|14138|1|641| +2450815|14140|1|448| +2450815|14143|1|700| +2450815|14144|1|537| +2450815|14146|1|402| +2450815|14149|1|542| +2450815|14150|1|325| +2450815|14152|1|780| +2450815|14155|1|688| +2450815|14156|1|829| +2450815|14158|1|915| +2450815|14161|1|308| +2450815|14162|1|883| +2450815|14164|1|546| +2450815|14167|1|661| +2450815|14168|1|560| +2450815|14170|1|906| +2450815|14173|1|872| +2450815|14174|1|347| +2450815|14176|1|516| +2450815|14179|1|773| +2450815|14180|1|114| +2450815|14182|1|982| +2450815|14185|1|463| +2450815|14186|1|849| +2450815|14188|1|293| +2450815|14191|1|605| +2450815|14192|1|726| +2450815|14194|1|588| +2450815|14197|1|709| +2450815|14198|1|843| +2450815|14200|1|890| +2450815|14203|1|543| +2450815|14204|1|| +2450815|14206|1|| +2450815|14209|1|407| +2450815|14210|1|800| +2450815|14212|1|675| +2450815|14215|1|| +2450815|14216|1|296| +2450815|14218|1|899| +2450815|14221|1|662| +2450815|14222|1|879| +2450815|14224|1|349| +2450815|14227|1|1| +2450815|14228|1|399| +2450815|14230|1|261| +2450815|14233|1|294| +2450815|14234|1|283| +2450815|14236|1|114| +2450815|14239|1|931| +2450815|14240|1|814| +2450815|14242|1|12| +2450815|14245|1|114| +2450815|14246|1|126| +2450815|14248|1|182| +2450815|14251|1|883| +2450815|14252|1|531| +2450815|14254|1|267| +2450815|14257|1|578| +2450815|14258|1|881| +2450815|14260|1|283| +2450815|14263|1|784| +2450815|14264|1|100| +2450815|14266|1|382| +2450815|14269|1|764| +2450815|14270|1|934| +2450815|14272|1|45| +2450815|14275|1|118| +2450815|14276|1|131| +2450815|14278|1|870| +2450815|14281|1|362| +2450815|14282|1|118| +2450815|14284|1|425| +2450815|14287|1|253| +2450815|14288|1|301| +2450815|14290|1|479| +2450815|14293|1|68| +2450815|14294|1|554| +2450815|14296|1|115| +2450815|14299|1|936| +2450815|14300|1|315| +2450815|14302|1|565| +2450815|14305|1|273| +2450815|14306|1|532| +2450815|14308|1|277| +2450815|14311|1|771| +2450815|14312|1|198| +2450815|14314|1|220| +2450815|14317|1|650| +2450815|14318|1|978| +2450815|14320|1|341| +2450815|14323|1|773| +2450815|14324|1|658| +2450815|14326|1|207| +2450815|14329|1|922| +2450815|14330|1|247| +2450815|14332|1|634| +2450815|14335|1|493| +2450815|14336|1|796| +2450815|14338|1|836| +2450815|14341|1|831| +2450815|14342|1|583| +2450815|14344|1|| +2450815|14347|1|342| +2450815|14348|1|801| +2450815|14350|1|688| +2450815|14353|1|124| +2450815|14354|1|406| +2450815|14356|1|793| +2450815|14359|1|52| +2450815|14360|1|375| +2450815|14362|1|499| +2450815|14365|1|269| +2450815|14366|1|65| +2450815|14368|1|457| +2450815|14371|1|672| +2450815|14372|1|74| +2450815|14374|1|770| +2450815|14377|1|993| +2450815|14378|1|799| +2450815|14380|1|91| +2450815|14383|1|720| +2450815|14384|1|340| +2450815|14386|1|944| +2450815|14389|1|589| +2450815|14390|1|614| +2450815|14392|1|597| +2450815|14395|1|747| +2450815|14396|1|442| +2450815|14398|1|157| +2450815|14401|1|879| +2450815|14402|1|632| +2450815|14404|1|660| +2450815|14407|1|547| +2450815|14408|1|900| +2450815|14410|1|456| +2450815|14413|1|980| +2450815|14414|1|968| +2450815|14416|1|792| +2450815|14419|1|346| +2450815|14420|1|949| +2450815|14422|1|741| +2450815|14425|1|695| +2450815|14426|1|36| +2450815|14428|1|50| +2450815|14431|1|945| +2450815|14432|1|33| +2450815|14434|1|260| +2450815|14437|1|346| +2450815|14438|1|652| +2450815|14440|1|848| +2450815|14443|1|656| +2450815|14444|1|564| +2450815|14446|1|82| +2450815|14449|1|579| +2450815|14450|1|850| +2450815|14452|1|219| +2450815|14455|1|525| +2450815|14456|1|286| +2450815|14458|1|453| +2450815|14461|1|648| +2450815|14462|1|664| +2450815|14464|1|390| +2450815|14467|1|953| +2450815|14468|1|425| +2450815|14470|1|727| +2450815|14473|1|341| +2450815|14474|1|| +2450815|14476|1|460| +2450815|14479|1|275| +2450815|14480|1|965| +2450815|14482|1|206| +2450815|14485|1|289| +2450815|14486|1|650| +2450815|14488|1|552| +2450815|14491|1|433| +2450815|14492|1|97| +2450815|14494|1|107| +2450815|14497|1|101| +2450815|14498|1|664| +2450815|14500|1|457| +2450815|14503|1|| +2450815|14504|1|851| +2450815|14506|1|567| +2450815|14509|1|414| +2450815|14510|1|739| +2450815|14512|1|564| +2450815|14515|1|285| +2450815|14516|1|939| +2450815|14518|1|1| +2450815|14521|1|941| +2450815|14522|1|327| +2450815|14524|1|34| +2450815|14527|1|699| +2450815|14528|1|906| +2450815|14530|1|631| +2450815|14533|1|832| +2450815|14534|1|| +2450815|14536|1|459| +2450815|14539|1|122| +2450815|14540|1|452| +2450815|14542|1|390| +2450815|14545|1|453| +2450815|14546|1|694| +2450815|14548|1|138| +2450815|14551|1|302| +2450815|14552|1|377| +2450815|14554|1|| +2450815|14557|1|830| +2450815|14558|1|993| +2450815|14560|1|| +2450815|14563|1|293| +2450815|14564|1|706| +2450815|14566|1|61| +2450815|14569|1|6| +2450815|14570|1|331| +2450815|14572|1|862| +2450815|14575|1|559| +2450815|14576|1|952| +2450815|14578|1|645| +2450815|14581|1|937| +2450815|14582|1|375| +2450815|14584|1|241| +2450815|14587|1|566| +2450815|14588|1|185| +2450815|14590|1|646| +2450815|14593|1|221| +2450815|14594|1|414| +2450815|14596|1|84| +2450815|14599|1|706| +2450815|14600|1|| +2450815|14602|1|3| +2450815|14605|1|876| +2450815|14606|1|522| +2450815|14608|1|670| +2450815|14611|1|136| +2450815|14612|1|303| +2450815|14614|1|757| +2450815|14617|1|| +2450815|14618|1|529| +2450815|14620|1|876| +2450815|14623|1|954| +2450815|14624|1|431| +2450815|14626|1|145| +2450815|14629|1|794| +2450815|14630|1|855| +2450815|14632|1|771| +2450815|14635|1|231| +2450815|14636|1|444| +2450815|14638|1|| +2450815|14641|1|93| +2450815|14642|1|| +2450815|14644|1|850| +2450815|14647|1|733| +2450815|14648|1|759| +2450815|14650|1|574| +2450815|14653|1|652| +2450815|14654|1|498| +2450815|14656|1|886| +2450815|14659|1|17| +2450815|14660|1|442| +2450815|14662|1|206| +2450815|14665|1|279| +2450815|14666|1|372| +2450815|14668|1|673| +2450815|14671|1|191| +2450815|14672|1|363| +2450815|14674|1|291| +2450815|14677|1|| +2450815|14678|1|| +2450815|14680|1|991| +2450815|14683|1|790| +2450815|14684|1|801| +2450815|14686|1|816| +2450815|14689|1|132| +2450815|14690|1|632| +2450815|14692|1|317| +2450815|14695|1|647| +2450815|14696|1|588| +2450815|14698|1|611| +2450815|14701|1|928| +2450815|14702|1|610| +2450815|14704|1|104| +2450815|14707|1|808| +2450815|14708|1|329| +2450815|14710|1|805| +2450815|14713|1|738| +2450815|14714|1|758| +2450815|14716|1|758| +2450815|14719|1|790| +2450815|14720|1|88| +2450815|14722|1|232| +2450815|14725|1|571| +2450815|14726|1|64| +2450815|14728|1|624| +2450815|14731|1|344| +2450815|14732|1|547| +2450815|14734|1|343| +2450815|14737|1|199| +2450815|14738|1|892| +2450815|14740|1|904| +2450815|14743|1|655| +2450815|14744|1|| +2450815|14746|1|490| +2450815|14749|1|589| +2450815|14750|1|716| +2450815|14752|1|650| +2450815|14755|1|744| +2450815|14756|1|593| +2450815|14758|1|592| +2450815|14761|1|131| +2450815|14762|1|765| +2450815|14764|1|732| +2450815|14767|1|915| +2450815|14768|1|413| +2450815|14770|1|40| +2450815|14773|1|349| +2450815|14774|1|421| +2450815|14776|1|886| +2450815|14779|1|526| +2450815|14780|1|568| +2450815|14782|1|436| +2450815|14785|1|8| +2450815|14786|1|572| +2450815|14788|1|801| +2450815|14791|1|598| +2450815|14792|1|533| +2450815|14794|1|392| +2450815|14797|1|6| +2450815|14798|1|53| +2450815|14800|1|915| +2450815|14803|1|35| +2450815|14804|1|289| +2450815|14806|1|738| +2450815|14809|1|248| +2450815|14810|1|878| +2450815|14812|1|698| +2450815|14815|1|202| +2450815|14816|1|632| +2450815|14818|1|593| +2450815|14821|1|620| +2450815|14822|1|869| +2450815|14824|1|450| +2450815|14827|1|168| +2450815|14828|1|16| +2450815|14830|1|383| +2450815|14833|1|| +2450815|14834|1|555| +2450815|14836|1|601| +2450815|14839|1|750| +2450815|14840|1|407| +2450815|14842|1|91| +2450815|14845|1|180| +2450815|14846|1|708| +2450815|14848|1|471| +2450815|14851|1|901| +2450815|14852|1|341| +2450815|14854|1|595| +2450815|14857|1|326| +2450815|14858|1|| +2450815|14860|1|544| +2450815|14863|1|236| +2450815|14864|1|332| +2450815|14866|1|261| +2450815|14869|1|812| +2450815|14870|1|612| +2450815|14872|1|370| +2450815|14875|1|292| +2450815|14876|1|836| +2450815|14878|1|664| +2450815|14881|1|672| +2450815|14882|1|892| +2450815|14884|1|391| +2450815|14887|1|502| +2450815|14888|1|854| +2450815|14890|1|957| +2450815|14893|1|70| +2450815|14894|1|789| +2450815|14896|1|269| +2450815|14899|1|434| +2450815|14900|1|86| +2450815|14902|1|710| +2450815|14905|1|664| +2450815|14906|1|893| +2450815|14908|1|953| +2450815|14911|1|62| +2450815|14912|1|663| +2450815|14914|1|316| +2450815|14917|1|455| +2450815|14918|1|519| +2450815|14920|1|743| +2450815|14923|1|11| +2450815|14924|1|3| +2450815|14926|1|567| +2450815|14929|1|804| +2450815|14930|1|286| +2450815|14932|1|459| +2450815|14935|1|1000| +2450815|14936|1|146| +2450815|14938|1|501| +2450815|14941|1|906| +2450815|14942|1|933| +2450815|14944|1|67| +2450815|14947|1|660| +2450815|14948|1|901| +2450815|14950|1|725| +2450815|14953|1|935| +2450815|14954|1|787| +2450815|14956|1|400| +2450815|14959|1|385| +2450815|14960|1|993| +2450815|14962|1|564| +2450815|14965|1|715| +2450815|14966|1|800| +2450815|14968|1|| +2450815|14971|1|32| +2450815|14972|1|313| +2450815|14974|1|34| +2450815|14977|1|300| +2450815|14978|1|347| +2450815|14980|1|475| +2450815|14983|1|106| +2450815|14984|1|84| +2450815|14986|1|460| +2450815|14989|1|972| +2450815|14990|1|| +2450815|14992|1|15| +2450815|14995|1|935| +2450815|14996|1|703| +2450815|14998|1|32| +2450815|15001|1|149| +2450815|15002|1|788| +2450815|15004|1|918| +2450815|15007|1|| +2450815|15008|1|379| +2450815|15010|1|131| +2450815|15013|1|548| +2450815|15014|1|929| +2450815|15016|1|490| +2450815|15019|1|970| +2450815|15020|1|907| +2450815|15022|1|257| +2450815|15025|1|432| +2450815|15026|1|689| +2450815|15028|1|205| +2450815|15031|1|| +2450815|15032|1|959| +2450815|15034|1|504| +2450815|15037|1|963| +2450815|15038|1|917| +2450815|15040|1|866| +2450815|15043|1|218| +2450815|15044|1|725| +2450815|15046|1|977| +2450815|15049|1|761| +2450815|15050|1|208| +2450815|15052|1|503| +2450815|15055|1|375| +2450815|15056|1|775| +2450815|15058|1|813| +2450815|15061|1|236| +2450815|15062|1|648| +2450815|15064|1|906| +2450815|15067|1|950| +2450815|15068|1|515| +2450815|15070|1|923| +2450815|15073|1|237| +2450815|15074|1|150| +2450815|15076|1|377| +2450815|15079|1|914| +2450815|15080|1|993| +2450815|15082|1|355| +2450815|15085|1|| +2450815|15086|1|600| +2450815|15088|1|293| +2450815|15091|1|661| +2450815|15092|1|795| +2450815|15094|1|888| +2450815|15097|1|254| +2450815|15098|1|653| +2450815|15100|1|993| +2450815|15103|1|122| +2450815|15104|1|763| +2450815|15106|1|111| +2450815|15109|1|641| +2450815|15110|1|420| +2450815|15112|1|860| +2450815|15115|1|405| +2450815|15116|1|226| +2450815|15118|1|555| +2450815|15121|1|610| +2450815|15122|1|545| +2450815|15124|1|119| +2450815|15127|1|| +2450815|15128|1|234| +2450815|15130|1|860| +2450815|15133|1|49| +2450815|15134|1|271| +2450815|15136|1|505| +2450815|15139|1|382| +2450815|15140|1|968| +2450815|15142|1|434| +2450815|15145|1|262| +2450815|15146|1|823| +2450815|15148|1|876| +2450815|15151|1|755| +2450815|15152|1|576| +2450815|15154|1|602| +2450815|15157|1|909| +2450815|15158|1|144| +2450815|15160|1|325| +2450815|15163|1|508| +2450815|15164|1|864| +2450815|15166|1|756| +2450815|15169|1|72| +2450815|15170|1|409| +2450815|15172|1|695| +2450815|15175|1|853| +2450815|15176|1|512| +2450815|15178|1|302| +2450815|15181|1|736| +2450815|15182|1|636| +2450815|15184|1|223| +2450815|15187|1|864| +2450815|15188|1|939| +2450815|15190|1|281| +2450815|15193|1|475| +2450815|15194|1|145| +2450815|15196|1|395| +2450815|15199|1|12| +2450815|15200|1|235| +2450815|15202|1|904| +2450815|15205|1|309| +2450815|15206|1|438| +2450815|15208|1|132| +2450815|15211|1|308| +2450815|15212|1|64| +2450815|15214|1|86| +2450815|15217|1|222| +2450815|15218|1|155| +2450815|15220|1|998| +2450815|15223|1|48| +2450815|15224|1|986| +2450815|15226|1|686| +2450815|15229|1|243| +2450815|15230|1|193| +2450815|15232|1|793| +2450815|15235|1|871| +2450815|15236|1|108| +2450815|15238|1|938| +2450815|15241|1|863| +2450815|15242|1|631| +2450815|15244|1|433| +2450815|15247|1|615| +2450815|15248|1|782| +2450815|15250|1|560| +2450815|15253|1|740| +2450815|15254|1|448| +2450815|15256|1|380| +2450815|15259|1|378| +2450815|15260|1|846| +2450815|15262|1|606| +2450815|15265|1|964| +2450815|15266|1|481| +2450815|15268|1|536| +2450815|15271|1|297| +2450815|15272|1|720| +2450815|15274|1|866| +2450815|15277|1|734| +2450815|15278|1|465| +2450815|15280|1|404| +2450815|15283|1|783| +2450815|15284|1|| +2450815|15286|1|748| +2450815|15289|1|723| +2450815|15290|1|951| +2450815|15292|1|862| +2450815|15295|1|314| +2450815|15296|1|495| +2450815|15298|1|672| +2450815|15301|1|646| +2450815|15302|1|206| +2450815|15304|1|| +2450815|15307|1|794| +2450815|15308|1|204| +2450815|15310|1|160| +2450815|15313|1|| +2450815|15314|1|288| +2450815|15316|1|389| +2450815|15319|1|203| +2450815|15320|1|295| +2450815|15322|1|319| +2450815|15325|1|316| +2450815|15326|1|104| +2450815|15328|1|| +2450815|15331|1|| +2450815|15332|1|692| +2450815|15334|1|990| +2450815|15337|1|908| +2450815|15338|1|885| +2450815|15340|1|271| +2450815|15343|1|469| +2450815|15344|1|336| +2450815|15346|1|811| +2450815|15349|1|414| +2450815|15350|1|504| +2450815|15352|1|743| +2450815|15355|1|19| +2450815|15356|1|| +2450815|15358|1|890| +2450815|15361|1|573| +2450815|15362|1|152| +2450815|15364|1|521| +2450815|15367|1|493| +2450815|15368|1|802| +2450815|15370|1|413| +2450815|15373|1|241| +2450815|15374|1|104| +2450815|15376|1|642| +2450815|15379|1|907| +2450815|15380|1|669| +2450815|15382|1|724| +2450815|15385|1|818| +2450815|15386|1|743| +2450815|15388|1|892| +2450815|15391|1|803| +2450815|15392|1|484| +2450815|15394|1|557| +2450815|15397|1|416| +2450815|15398|1|64| +2450815|15400|1|921| +2450815|15403|1|378| +2450815|15404|1|977| +2450815|15406|1|605| +2450815|15409|1|829| +2450815|15410|1|627| +2450815|15412|1|368| +2450815|15415|1|637| +2450815|15416|1|761| +2450815|15418|1|781| +2450815|15421|1|749| +2450815|15422|1|102| +2450815|15424|1|595| +2450815|15427|1|499| +2450815|15428|1|968| +2450815|15430|1|914| +2450815|15433|1|772| +2450815|15434|1|| +2450815|15436|1|396| +2450815|15439|1|247| +2450815|15440|1|31| +2450815|15442|1|104| +2450815|15445|1|990| +2450815|15446|1|6| +2450815|15448|1|662| +2450815|15451|1|860| +2450815|15452|1|882| +2450815|15454|1|762| +2450815|15457|1|268| +2450815|15458|1|780| +2450815|15460|1|247| +2450815|15463|1|218| +2450815|15464|1|257| +2450815|15466|1|733| +2450815|15469|1|880| +2450815|15470|1|33| +2450815|15472|1|764| +2450815|15475|1|| +2450815|15476|1|604| +2450815|15478|1|595| +2450815|15481|1|199| +2450815|15482|1|2| +2450815|15484|1|283| +2450815|15487|1|191| +2450815|15488|1|523| +2450815|15490|1|194| +2450815|15493|1|509| +2450815|15494|1|598| +2450815|15496|1|645| +2450815|15499|1|568| +2450815|15500|1|944| +2450815|15502|1|363| +2450815|15505|1|192| +2450815|15506|1|409| +2450815|15508|1|577| +2450815|15511|1|398| +2450815|15512|1|221| +2450815|15514|1|356| +2450815|15517|1|98| +2450815|15518|1|960| +2450815|15520|1|970| +2450815|15523|1|726| +2450815|15524|1|498| +2450815|15526|1|103| +2450815|15529|1|812| +2450815|15530|1|551| +2450815|15532|1|| +2450815|15535|1|372| +2450815|15536|1|328| +2450815|15538|1|148| +2450815|15541|1|| +2450815|15542|1|934| +2450815|15544|1|774| +2450815|15547|1|975| +2450815|15548|1|717| +2450815|15550|1|175| +2450815|15553|1|811| +2450815|15554|1|348| +2450815|15556|1|674| +2450815|15559|1|518| +2450815|15560|1|439| +2450815|15562|1|991| +2450815|15565|1|390| +2450815|15566|1|121| +2450815|15568|1|853| +2450815|15571|1|136| +2450815|15572|1|838| +2450815|15574|1|930| +2450815|15577|1|644| +2450815|15578|1|83| +2450815|15580|1|786| +2450815|15583|1|937| +2450815|15584|1|208| +2450815|15586|1|183| +2450815|15589|1|901| +2450815|15590|1|850| +2450815|15592|1|813| +2450815|15595|1|43| +2450815|15596|1|659| +2450815|15598|1|668| +2450815|15601|1|527| +2450815|15602|1|228| +2450815|15604|1|207| +2450815|15607|1|852| +2450815|15608|1|737| +2450815|15610|1|559| +2450815|15613|1|909| +2450815|15614|1|1| +2450815|15616|1|644| +2450815|15619|1|154| +2450815|15620|1|717| +2450815|15622|1|19| +2450815|15625|1|293| +2450815|15626|1|958| +2450815|15628|1|323| +2450815|15631|1|371| +2450815|15632|1|115| +2450815|15634|1|923| +2450815|15637|1|330| +2450815|15638|1|| +2450815|15640|1|754| +2450815|15643|1|139| +2450815|15644|1|406| +2450815|15646|1|927| +2450815|15649|1|| +2450815|15650|1|224| +2450815|15652|1|674| +2450815|15655|1|748| +2450815|15656|1|407| +2450815|15658|1|43| +2450815|15661|1|487| +2450815|15662|1|944| +2450815|15664|1|180| +2450815|15667|1|460| +2450815|15668|1|653| +2450815|15670|1|869| +2450815|15673|1|142| +2450815|15674|1|389| +2450815|15676|1|74| +2450815|15679|1|430| +2450815|15680|1|286| +2450815|15682|1|937| +2450815|15685|1|635| +2450815|15686|1|57| +2450815|15688|1|151| +2450815|15691|1|760| +2450815|15692|1|577| +2450815|15694|1|628| +2450815|15697|1|175| +2450815|15698|1|633| +2450815|15700|1|507| +2450815|15703|1|568| +2450815|15704|1|| +2450815|15706|1|724| +2450815|15709|1|556| +2450815|15710|1|224| +2450815|15712|1|178| +2450815|15715|1|914| +2450815|15716|1|174| +2450815|15718|1|207| +2450815|15721|1|781| +2450815|15722|1|820| +2450815|15724|1|509| +2450815|15727|1|508| +2450815|15728|1|675| +2450815|15730|1|168| +2450815|15733|1|210| +2450815|15734|1|641| +2450815|15736|1|629| +2450815|15739|1|839| +2450815|15740|1|235| +2450815|15742|1|695| +2450815|15745|1|186| +2450815|15746|1|666| +2450815|15748|1|| +2450815|15751|1|285| +2450815|15752|1|63| +2450815|15754|1|953| +2450815|15757|1|| +2450815|15758|1|588| +2450815|15760|1|784| +2450815|15763|1|430| +2450815|15764|1|128| +2450815|15766|1|875| +2450815|15769|1|271| +2450815|15770|1|408| +2450815|15772|1|258| +2450815|15775|1|476| +2450815|15776|1|827| +2450815|15778|1|68| +2450815|15781|1|56| +2450815|15782|1|771| +2450815|15784|1|185| +2450815|15787|1|529| +2450815|15788|1|945| +2450815|15790|1|391| +2450815|15793|1|836| +2450815|15794|1|671| +2450815|15796|1|113| +2450815|15799|1|379| +2450815|15800|1|786| +2450815|15802|1|890| +2450815|15805|1|856| +2450815|15806|1|8| +2450815|15808|1|265| +2450815|15811|1|646| +2450815|15812|1|| +2450815|15814|1|663| +2450815|15817|1|357| +2450815|15818|1|702| +2450815|15820|1|727| +2450815|15823|1|907| +2450815|15824|1|489| +2450815|15826|1|| +2450815|15829|1|882| +2450815|15830|1|245| +2450815|15832|1|892| +2450815|15835|1|848| +2450815|15836|1|658| +2450815|15838|1|278| +2450815|15841|1|171| +2450815|15842|1|681| +2450815|15844|1|188| +2450815|15847|1|637| +2450815|15848|1|517| +2450815|15850|1|292| +2450815|15853|1|890| +2450815|15854|1|724| +2450815|15856|1|607| +2450815|15859|1|711| +2450815|15860|1|693| +2450815|15862|1|134| +2450815|15865|1|18| +2450815|15866|1|683| +2450815|15868|1|770| +2450815|15871|1|163| +2450815|15872|1|823| +2450815|15874|1|897| +2450815|15877|1|388| +2450815|15878|1|680| +2450815|15880|1|| +2450815|15883|1|836| +2450815|15884|1|847| +2450815|15886|1|93| +2450815|15889|1|418| +2450815|15890|1|856| +2450815|15892|1|618| +2450815|15895|1|657| +2450815|15896|1|467| +2450815|15898|1|206| +2450815|15901|1|67| +2450815|15902|1|| +2450815|15904|1|318| +2450815|15907|1|4| +2450815|15908|1|860| +2450815|15910|1|125| +2450815|15913|1|650| +2450815|15914|1|532| +2450815|15916|1|545| +2450815|15919|1|929| +2450815|15920|1|438| +2450815|15922|1|611| +2450815|15925|1|992| +2450815|15926|1|| +2450815|15928|1|99| +2450815|15931|1|16| +2450815|15932|1|667| +2450815|15934|1|| +2450815|15937|1|| +2450815|15938|1|412| +2450815|15940|1|997| +2450815|15943|1|172| +2450815|15944|1|| +2450815|15946|1|156| +2450815|15949|1|693| +2450815|15950|1|414| +2450815|15952|1|256| +2450815|15955|1|| +2450815|15956|1|912| +2450815|15958|1|990| +2450815|15961|1|| +2450815|15962|1|195| +2450815|15964|1|393| +2450815|15967|1|710| +2450815|15968|1|818| +2450815|15970|1|460| +2450815|15973|1|491| +2450815|15974|1|40| +2450815|15976|1|977| +2450815|15979|1|863| +2450815|15980|1|469| +2450815|15982|1|17| +2450815|15985|1|817| +2450815|15986|1|942| +2450815|15988|1|| +2450815|15991|1|780| +2450815|15992|1|242| +2450815|15994|1|701| +2450815|15997|1|| +2450815|15998|1|668| +2450815|16000|1|| +2450815|16003|1|477| +2450815|16004|1|319| +2450815|16006|1|348| +2450815|16009|1|179| +2450815|16010|1|796| +2450815|16012|1|284| +2450815|16015|1|125| +2450815|16016|1|88| +2450815|16018|1|833| +2450815|16021|1|982| +2450815|16022|1|432| +2450815|16024|1|409| +2450815|16027|1|321| +2450815|16028|1|578| +2450815|16030|1|55| +2450815|16033|1|595| +2450815|16034|1|169| +2450815|16036|1|883| +2450815|16039|1|956| +2450815|16040|1|31| +2450815|16042|1|409| +2450815|16045|1|431| +2450815|16046|1|733| +2450815|16048|1|897| +2450815|16051|1|120| +2450815|16052|1|699| +2450815|16054|1|397| +2450815|16057|1|51| +2450815|16058|1|239| +2450815|16060|1|127| +2450815|16063|1|191| +2450815|16064|1|980| +2450815|16066|1|709| +2450815|16069|1|338| +2450815|16070|1|396| +2450815|16072|1|376| +2450815|16075|1|812| +2450815|16076|1|47| +2450815|16078|1|799| +2450815|16081|1|614| +2450815|16082|1|120| +2450815|16084|1|488| +2450815|16087|1|663| +2450815|16088|1|141| +2450815|16090|1|471| +2450815|16093|1|842| +2450815|16094|1|860| +2450815|16096|1|870| +2450815|16099|1|468| +2450815|16100|1|331| +2450815|16102|1|925| +2450815|16105|1|560| +2450815|16106|1|27| +2450815|16108|1|438| +2450815|16111|1|57| +2450815|16112|1|705| +2450815|16114|1|360| +2450815|16117|1|438| +2450815|16118|1|| +2450815|16120|1|635| +2450815|16123|1|59| +2450815|16124|1|501| +2450815|16126|1|273| +2450815|16129|1|332| +2450815|16130|1|79| +2450815|16132|1|338| +2450815|16135|1|326| +2450815|16136|1|17| +2450815|16138|1|976| +2450815|16141|1|182| +2450815|16142|1|152| +2450815|16144|1|332| +2450815|16147|1|943| +2450815|16148|1|9| +2450815|16150|1|750| +2450815|16153|1|53| +2450815|16154|1|485| +2450815|16156|1|552| +2450815|16159|1|203| +2450815|16160|1|858| +2450815|16162|1|918| +2450815|16165|1|540| +2450815|16166|1|30| +2450815|16168|1|256| +2450815|16171|1|720| +2450815|16172|1|527| +2450815|16174|1|893| +2450815|16177|1|867| +2450815|16178|1|987| +2450815|16180|1|| +2450815|16183|1|119| +2450815|16184|1|957| +2450815|16186|1|733| +2450815|16189|1|542| +2450815|16190|1|103| +2450815|16192|1|798| +2450815|16195|1|425| +2450815|16196|1|761| +2450815|16198|1|830| +2450815|16201|1|880| +2450815|16202|1|960| +2450815|16204|1|638| +2450815|16207|1|21| +2450815|16208|1|355| +2450815|16210|1|558| +2450815|16213|1|295| +2450815|16214|1|677| +2450815|16216|1|673| +2450815|16219|1|20| +2450815|16220|1|37| +2450815|16222|1|384| +2450815|16225|1|899| +2450815|16226|1|562| +2450815|16228|1|239| +2450815|16231|1|37| +2450815|16232|1|37| +2450815|16234|1|111| +2450815|16237|1|971| +2450815|16238|1|602| +2450815|16240|1|774| +2450815|16243|1|295| +2450815|16244|1|160| +2450815|16246|1|200| +2450815|16249|1|795| +2450815|16250|1|323| +2450815|16252|1|660| +2450815|16255|1|415| +2450815|16256|1|333| +2450815|16258|1|100| +2450815|16261|1|434| +2450815|16262|1|316| +2450815|16264|1|355| +2450815|16267|1|659| +2450815|16268|1|259| +2450815|16270|1|766| +2450815|16273|1|191| +2450815|16274|1|572| +2450815|16276|1|899| +2450815|16279|1|418| +2450815|16280|1|9| +2450815|16282|1|875| +2450815|16285|1|432| +2450815|16286|1|165| +2450815|16288|1|| +2450815|16291|1|141| +2450815|16292|1|610| +2450815|16294|1|591| +2450815|16297|1|107| +2450815|16298|1|| +2450815|16300|1|937| +2450815|16303|1|305| +2450815|16304|1|91| +2450815|16306|1|488| +2450815|16309|1|384| +2450815|16310|1|924| +2450815|16312|1|985| +2450815|16315|1|987| +2450815|16316|1|349| +2450815|16318|1|318| +2450815|16321|1|777| +2450815|16322|1|283| +2450815|16324|1|789| +2450815|16327|1|409| +2450815|16328|1|438| +2450815|16330|1|424| +2450815|16333|1|137| +2450815|16334|1|71| +2450815|16336|1|902| +2450815|16339|1|698| +2450815|16340|1|618| +2450815|16342|1|435| +2450815|16345|1|307| +2450815|16346|1|526| +2450815|16348|1|887| +2450815|16351|1|558| +2450815|16352|1|825| +2450815|16354|1|739| +2450815|16357|1|579| +2450815|16358|1|130| +2450815|16360|1|| +2450815|16363|1|232| +2450815|16364|1|333| +2450815|16366|1|29| +2450815|16369|1|157| +2450815|16370|1|776| +2450815|16372|1|163| +2450815|16375|1|634| +2450815|16376|1|108| +2450815|16378|1|96| +2450815|16381|1|424| +2450815|16382|1|394| +2450815|16384|1|870| +2450815|16387|1|146| +2450815|16388|1|133| +2450815|16390|1|59| +2450815|16393|1|66| +2450815|16394|1|867| +2450815|16396|1|445| +2450815|16399|1|764| +2450815|16400|1|189| +2450815|16402|1|356| +2450815|16405|1|573| +2450815|16406|1|363| +2450815|16408|1|722| +2450815|16411|1|973| +2450815|16412|1|929| +2450815|16414|1|600| +2450815|16417|1|375| +2450815|16418|1|587| +2450815|16420|1|856| +2450815|16423|1|354| +2450815|16424|1|951| +2450815|16426|1|422| +2450815|16429|1|13| +2450815|16430|1|954| +2450815|16432|1|500| +2450815|16435|1|895| +2450815|16436|1|387| +2450815|16438|1|46| +2450815|16441|1|9| +2450815|16442|1|249| +2450815|16444|1|481| +2450815|16447|1|235| +2450815|16448|1|840| +2450815|16450|1|991| +2450815|16453|1|386| +2450815|16454|1|927| +2450815|16456|1|898| +2450815|16459|1|430| +2450815|16460|1|489| +2450815|16462|1|| +2450815|16465|1|642| +2450815|16466|1|885| +2450815|16468|1|199| +2450815|16471|1|916| +2450815|16472|1|261| +2450815|16474|1|923| +2450815|16477|1|412| +2450815|16478|1|549| +2450815|16480|1|41| +2450815|16483|1|641| +2450815|16484|1|467| +2450815|16486|1|248| +2450815|16489|1|544| +2450815|16490|1|406| +2450815|16492|1|871| +2450815|16495|1|348| +2450815|16496|1|247| +2450815|16498|1|232| +2450815|16501|1|929| +2450815|16502|1|249| +2450815|16504|1|550| +2450815|16507|1|450| +2450815|16508|1|320| +2450815|16510|1|467| +2450815|16513|1|552| +2450815|16514|1|463| +2450815|16516|1|876| +2450815|16519|1|509| +2450815|16520|1|24| +2450815|16522|1|640| +2450815|16525|1|79| +2450815|16526|1|847| +2450815|16528|1|173| +2450815|16531|1|60| +2450815|16532|1|530| +2450815|16534|1|917| +2450815|16537|1|433| +2450815|16538|1|65| +2450815|16540|1|226| +2450815|16543|1|390| +2450815|16544|1|523| +2450815|16546|1|140| +2450815|16549|1|201| +2450815|16550|1|709| +2450815|16552|1|730| +2450815|16555|1|783| +2450815|16556|1|700| +2450815|16558|1|369| +2450815|16561|1|347| +2450815|16562|1|711| +2450815|16564|1|946| +2450815|16567|1|887| +2450815|16568|1|619| +2450815|16570|1|642| +2450815|16573|1|264| +2450815|16574|1|20| +2450815|16576|1|778| +2450815|16579|1|497| +2450815|16580|1|351| +2450815|16582|1|618| +2450815|16585|1|188| +2450815|16586|1|372| +2450815|16588|1|227| +2450815|16591|1|147| +2450815|16592|1|236| +2450815|16594|1|623| +2450815|16597|1|419| +2450815|16598|1|147| +2450815|16600|1|244| +2450815|16603|1|691| +2450815|16604|1|89| +2450815|16606|1|215| +2450815|16609|1|611| +2450815|16610|1|574| +2450815|16612|1|101| +2450815|16615|1|673| +2450815|16616|1|918| +2450815|16618|1|435| +2450815|16621|1|481| +2450815|16622|1|947| +2450815|16624|1|57| +2450815|16627|1|342| +2450815|16628|1|901| +2450815|16630|1|258| +2450815|16633|1|84| +2450815|16634|1|| +2450815|16636|1|168| +2450815|16639|1|260| +2450815|16640|1|| +2450815|16642|1|455| +2450815|16645|1|646| +2450815|16646|1|211| +2450815|16648|1|114| +2450815|16651|1|274| +2450815|16652|1|295| +2450815|16654|1|555| +2450815|16657|1|180| +2450815|16658|1|718| +2450815|16660|1|572| +2450815|16663|1|761| +2450815|16664|1|899| +2450815|16666|1|761| +2450815|16669|1|613| +2450815|16670|1|456| +2450815|16672|1|373| +2450815|16675|1|486| +2450815|16676|1|173| +2450815|16678|1|947| +2450815|16681|1|458| +2450815|16682|1|345| +2450815|16684|1|555| +2450815|16687|1|601| +2450815|16688|1|948| +2450815|16690|1|445| +2450815|16693|1|215| +2450815|16694|1|719| +2450815|16696|1|566| +2450815|16699|1|826| +2450815|16700|1|555| +2450815|16702|1|447| +2450815|16705|1|100| +2450815|16706|1|| +2450815|16708|1|| +2450815|16711|1|380| +2450815|16712|1|135| +2450815|16714|1|314| +2450815|16717|1|712| +2450815|16718|1|686| +2450815|16720|1|432| +2450815|16723|1|117| +2450815|16724|1|730| +2450815|16726|1|230| +2450815|16729|1|400| +2450815|16730|1|575| +2450815|16732|1|535| +2450815|16735|1|750| +2450815|16736|1|87| +2450815|16738|1|434| +2450815|16741|1|65| +2450815|16742|1|952| +2450815|16744|1|944| +2450815|16747|1|280| +2450815|16748|1|850| +2450815|16750|1|493| +2450815|16753|1|318| +2450815|16754|1|27| +2450815|16756|1|513| +2450815|16759|1|32| +2450815|16760|1|105| +2450815|16762|1|249| +2450815|16765|1|81| +2450815|16766|1|109| +2450815|16768|1|741| +2450815|16771|1|121| +2450815|16772|1|625| +2450815|16774|1|180| +2450815|16777|1|620| +2450815|16778|1|576| +2450815|16780|1|736| +2450815|16783|1|234| +2450815|16784|1|988| +2450815|16786|1|654| +2450815|16789|1|710| +2450815|16790|1|897| +2450815|16792|1|257| +2450815|16795|1|371| +2450815|16796|1|893| +2450815|16798|1|221| +2450815|16801|1|56| +2450815|16802|1|37| +2450815|16804|1|907| +2450815|16807|1|189| +2450815|16808|1|867| +2450815|16810|1|381| +2450815|16813|1|821| +2450815|16814|1|7| +2450815|16816|1|965| +2450815|16819|1|617| +2450815|16820|1|854| +2450815|16822|1|694| +2450815|16825|1|745| +2450815|16826|1|662| +2450815|16828|1|974| +2450815|16831|1|396| +2450815|16832|1|310| +2450815|16834|1|487| +2450815|16837|1|613| +2450815|16838|1|272| +2450815|16840|1|851| +2450815|16843|1|624| +2450815|16844|1|708| +2450815|16846|1|26| +2450815|16849|1|541| +2450815|16850|1|689| +2450815|16852|1|533| +2450815|16855|1|400| +2450815|16856|1|96| +2450815|16858|1|655| +2450815|16861|1|843| +2450815|16862|1|375| +2450815|16864|1|871| +2450815|16867|1|494| +2450815|16868|1|599| +2450815|16870|1|936| +2450815|16873|1|557| +2450815|16874|1|74| +2450815|16876|1|990| +2450815|16879|1|837| +2450815|16880|1|576| +2450815|16882|1|431| +2450815|16885|1|212| +2450815|16886|1|712| +2450815|16888|1|598| +2450815|16891|1|86| +2450815|16892|1|621| +2450815|16894|1|838| +2450815|16897|1|822| +2450815|16898|1|959| +2450815|16900|1|616| +2450815|16903|1|668| +2450815|16904|1|34| +2450815|16906|1|| +2450815|16909|1|660| +2450815|16910|1|915| +2450815|16912|1|409| +2450815|16915|1|529| +2450815|16916|1|940| +2450815|16918|1|443| +2450815|16921|1|820| +2450815|16922|1|266| +2450815|16924|1|491| +2450815|16927|1|| +2450815|16928|1|67| +2450815|16930|1|235| +2450815|16933|1|993| +2450815|16934|1|925| +2450815|16936|1|14| +2450815|16939|1|741| +2450815|16940|1|165| +2450815|16942|1|719| +2450815|16945|1|378| +2450815|16946|1|170| +2450815|16948|1|978| +2450815|16951|1|951| +2450815|16952|1|838| +2450815|16954|1|| +2450815|16957|1|835| +2450815|16958|1|291| +2450815|16960|1|369| +2450815|16963|1|953| +2450815|16964|1|| +2450815|16966|1|109| +2450815|16969|1|677| +2450815|16970|1|7| +2450815|16972|1|816| +2450815|16975|1|683| +2450815|16976|1|325| +2450815|16978|1|304| +2450815|16981|1|914| +2450815|16982|1|809| +2450815|16984|1|709| +2450815|16987|1|532| +2450815|16988|1|176| +2450815|16990|1|93| +2450815|16993|1|65| +2450815|16994|1|281| +2450815|16996|1|574| +2450815|16999|1|806| +2450815|17000|1|718| +2450815|17002|1|485| +2450815|17005|1|651| +2450815|17006|1|813| +2450815|17008|1|66| +2450815|17011|1|775| +2450815|17012|1|656| +2450815|17014|1|635| +2450815|17017|1|935| +2450815|17018|1|349| +2450815|17020|1|341| +2450815|17023|1|16| +2450815|17024|1|753| +2450815|17026|1|173| +2450815|17029|1|986| +2450815|17030|1|287| +2450815|17032|1|640| +2450815|17035|1|42| +2450815|17036|1|829| +2450815|17038|1|2| +2450815|17041|1|478| +2450815|17042|1|139| +2450815|17044|1|565| +2450815|17047|1|722| +2450815|17048|1|846| +2450815|17050|1|854| +2450815|17053|1|59| +2450815|17054|1|463| +2450815|17056|1|975| +2450815|17059|1|43| +2450815|17060|1|927| +2450815|17062|1|165| +2450815|17065|1|846| +2450815|17066|1|955| +2450815|17068|1|145| +2450815|17071|1|155| +2450815|17072|1|| +2450815|17074|1|452| +2450815|17077|1|783| +2450815|17078|1|971| +2450815|17080|1|448| +2450815|17083|1|| +2450815|17084|1|44| +2450815|17086|1|379| +2450815|17089|1|| +2450815|17090|1|| +2450815|17092|1|483| +2450815|17095|1|153| +2450815|17096|1|| +2450815|17098|1|855| +2450815|17101|1|776| +2450815|17102|1|594| +2450815|17104|1|32| +2450815|17107|1|279| +2450815|17108|1|545| +2450815|17110|1|406| +2450815|17113|1|383| +2450815|17114|1|921| +2450815|17116|1|70| +2450815|17119|1|905| +2450815|17120|1|971| +2450815|17122|1|228| +2450815|17125|1|| +2450815|17126|1|78| +2450815|17128|1|755| +2450815|17131|1|633| +2450815|17132|1|44| +2450815|17134|1|176| +2450815|17137|1|12| +2450815|17138|1|224| +2450815|17140|1|915| +2450815|17143|1|401| +2450815|17144|1|519| +2450815|17146|1|173| +2450815|17149|1|306| +2450815|17150|1|135| +2450815|17152|1|682| +2450815|17155|1|798| +2450815|17156|1|123| +2450815|17158|1|814| +2450815|17161|1|362| +2450815|17162|1|686| +2450815|17164|1|527| +2450815|17167|1|844| +2450815|17168|1|307| +2450815|17170|1|260| +2450815|17173|1|203| +2450815|17174|1|164| +2450815|17176|1|16| +2450815|17179|1|586| +2450815|17180|1|249| +2450815|17182|1|479| +2450815|17185|1|507| +2450815|17186|1|995| +2450815|17188|1|244| +2450815|17191|1|22| +2450815|17192|1|51| +2450815|17194|1|839| +2450815|17197|1|112| +2450815|17198|1|853| +2450815|17200|1|581| +2450815|17203|1|382| +2450815|17204|1|535| +2450815|17206|1|742| +2450815|17209|1|214| +2450815|17210|1|527| +2450815|17212|1|848| +2450815|17215|1|198| +2450815|17216|1|524| +2450815|17218|1|296| +2450815|17221|1|129| +2450815|17222|1|482| +2450815|17224|1|457| +2450815|17227|1|181| +2450815|17228|1|| +2450815|17230|1|| +2450815|17233|1|253| +2450815|17234|1|483| +2450815|17236|1|402| +2450815|17239|1|339| +2450815|17240|1|267| +2450815|17242|1|155| +2450815|17245|1|560| +2450815|17246|1|550| +2450815|17248|1|819| +2450815|17251|1|| +2450815|17252|1|628| +2450815|17254|1|606| +2450815|17257|1|827| +2450815|17258|1|600| +2450815|17260|1|873| +2450815|17263|1|414| +2450815|17264|1|774| +2450815|17266|1|425| +2450815|17269|1|357| +2450815|17270|1|988| +2450815|17272|1|570| +2450815|17275|1|322| +2450815|17276|1|94| +2450815|17278|1|649| +2450815|17281|1|8| +2450815|17282|1|279| +2450815|17284|1|611| +2450815|17287|1|834| +2450815|17288|1|796| +2450815|17290|1|236| +2450815|17293|1|423| +2450815|17294|1|356| +2450815|17296|1|987| +2450815|17299|1|402| +2450815|17300|1|458| +2450815|17302|1|803| +2450815|17305|1|269| +2450815|17306|1|751| +2450815|17308|1|86| +2450815|17311|1|251| +2450815|17312|1|26| +2450815|17314|1|282| +2450815|17317|1|667| +2450815|17318|1|569| +2450815|17320|1|321| +2450815|17323|1|255| +2450815|17324|1|707| +2450815|17326|1|122| +2450815|17329|1|9| +2450815|17330|1|888| +2450815|17332|1|309| +2450815|17335|1|639| +2450815|17336|1|264| +2450815|17338|1|225| +2450815|17341|1|404| +2450815|17342|1|372| +2450815|17344|1|280| +2450815|17347|1|883| +2450815|17348|1|980| +2450815|17350|1|619| +2450815|17353|1|880| +2450815|17354|1|7| +2450815|17356|1|504| +2450815|17359|1|731| +2450815|17360|1|893| +2450815|17362|1|850| +2450815|17365|1|827| +2450815|17366|1|211| +2450815|17368|1|865| +2450815|17371|1|919| +2450815|17372|1|44| +2450815|17374|1|197| +2450815|17377|1|500| +2450815|17378|1|368| +2450815|17380|1|971| +2450815|17383|1|167| +2450815|17384|1|157| +2450815|17386|1|569| +2450815|17389|1|884| +2450815|17390|1|231| +2450815|17392|1|282| +2450815|17395|1|100| +2450815|17396|1|909| +2450815|17398|1|109| +2450815|17401|1|702| +2450815|17402|1|95| +2450815|17404|1|975| +2450815|17407|1|643| +2450815|17408|1|519| +2450815|17410|1|947| +2450815|17413|1|471| +2450815|17414|1|795| +2450815|17416|1|378| +2450815|17419|1|849| +2450815|17420|1|576| +2450815|17422|1|514| +2450815|17425|1|| +2450815|17426|1|70| +2450815|17428|1|491| +2450815|17431|1|339| +2450815|17432|1|502| +2450815|17434|1|883| +2450815|17437|1|879| +2450815|17438|1|| +2450815|17440|1|146| +2450815|17443|1|165| +2450815|17444|1|893| +2450815|17446|1|174| +2450815|17449|1|797| +2450815|17450|1|616| +2450815|17452|1|286| +2450815|17455|1|972| +2450815|17456|1|859| +2450815|17458|1|358| +2450815|17461|1|831| +2450815|17462|1|566| +2450815|17464|1|176| +2450815|17467|1|106| +2450815|17468|1|663| +2450815|17470|1|758| +2450815|17473|1|23| +2450815|17474|1|40| +2450815|17476|1|719| +2450815|17479|1|30| +2450815|17480|1|750| +2450815|17482|1|555| +2450815|17485|1|460| +2450815|17486|1|901| +2450815|17488|1|372| +2450815|17491|1|28| +2450815|17492|1|| +2450815|17494|1|384| +2450815|17497|1|523| +2450815|17498|1|247| +2450815|17500|1|749| +2450815|17503|1|918| +2450815|17504|1|880| +2450815|17506|1|681| +2450815|17509|1|4| +2450815|17510|1|174| +2450815|17512|1|582| +2450815|17515|1|49| +2450815|17516|1|| +2450815|17518|1|558| +2450815|17521|1|176| +2450815|17522|1|382| +2450815|17524|1|466| +2450815|17527|1|923| +2450815|17528|1|368| +2450815|17530|1|998| +2450815|17533|1|977| +2450815|17534|1|382| +2450815|17536|1|472| +2450815|17539|1|608| +2450815|17540|1|944| +2450815|17542|1|294| +2450815|17545|1|617| +2450815|17546|1|596| +2450815|17548|1|668| +2450815|17551|1|264| +2450815|17552|1|553| +2450815|17554|1|910| +2450815|17557|1|55| +2450815|17558|1|899| +2450815|17560|1|525| +2450815|17563|1|705| +2450815|17564|1|302| +2450815|17566|1|471| +2450815|17569|1|595| +2450815|17570|1|458| +2450815|17572|1|693| +2450815|17575|1|370| +2450815|17576|1|108| +2450815|17578|1|921| +2450815|17581|1|176| +2450815|17582|1|623| +2450815|17584|1|420| +2450815|17587|1|156| +2450815|17588|1|371| +2450815|17590|1|831| +2450815|17593|1|418| +2450815|17594|1|272| +2450815|17596|1|778| +2450815|17599|1|117| +2450815|17600|1|345| +2450815|17602|1|255| +2450815|17605|1|38| +2450815|17606|1|762| +2450815|17608|1|485| +2450815|17611|1|522| +2450815|17612|1|615| +2450815|17614|1|623| +2450815|17617|1|592| +2450815|17618|1|782| +2450815|17620|1|692| +2450815|17623|1|773| +2450815|17624|1|910| +2450815|17626|1|27| +2450815|17629|1|625| +2450815|17630|1|937| +2450815|17632|1|315| +2450815|17635|1|744| +2450815|17636|1|753| +2450815|17638|1|478| +2450815|17641|1|954| +2450815|17642|1|798| +2450815|17644|1|322| +2450815|17647|1|351| +2450815|17648|1|34| +2450815|17650|1|753| +2450815|17653|1|| +2450815|17654|1|785| +2450815|17656|1|| +2450815|17659|1|1000| +2450815|17660|1|230| +2450815|17662|1|897| +2450815|17665|1|755| +2450815|17666|1|767| +2450815|17668|1|277| +2450815|17671|1|543| +2450815|17672|1|57| +2450815|17674|1|863| +2450815|17677|1|735| +2450815|17678|1|673| +2450815|17680|1|730| +2450815|17683|1|38| +2450815|17684|1|812| +2450815|17686|1|396| +2450815|17689|1|249| +2450815|17690|1|128| +2450815|17692|1|892| +2450815|17695|1|5| +2450815|17696|1|670| +2450815|17698|1|956| +2450815|17701|1|504| +2450815|17702|1|967| +2450815|17704|1|113| +2450815|17707|1|253| +2450815|17708|1|557| +2450815|17710|1|729| +2450815|17713|1|653| +2450815|17714|1|272| +2450815|17716|1|51| +2450815|17719|1|853| +2450815|17720|1|863| +2450815|17722|1|249| +2450815|17725|1|149| +2450815|17726|1|942| +2450815|17728|1|19| +2450815|17731|1|192| +2450815|17732|1|| +2450815|17734|1|518| +2450815|17737|1|885| +2450815|17738|1|1| +2450815|17740|1|846| +2450815|17743|1|946| +2450815|17744|1|883| +2450815|17746|1|484| +2450815|17749|1|185| +2450815|17750|1|601| +2450815|17752|1|331| +2450815|17755|1|683| +2450815|17756|1|17| +2450815|17758|1|640| +2450815|17761|1|289| +2450815|17762|1|211| +2450815|17764|1|241| +2450815|17767|1|948| +2450815|17768|1|519| +2450815|17770|1|751| +2450815|17773|1|216| +2450815|17774|1|799| +2450815|17776|1|| +2450815|17779|1|283| +2450815|17780|1|635| +2450815|17782|1|55| +2450815|17785|1|840| +2450815|17786|1|494| +2450815|17788|1|566| +2450815|17791|1|931| +2450815|17792|1|769| +2450815|17794|1|| +2450815|17797|1|203| +2450815|17798|1|250| +2450815|17800|1|| +2450815|17803|1|| +2450815|17804|1|881| +2450815|17806|1|| +2450815|17809|1|794| +2450815|17810|1|128| +2450815|17812|1|| +2450815|17815|1|277| +2450815|17816|1|275| +2450815|17818|1|339| +2450815|17821|1|79| +2450815|17822|1|241| +2450815|17824|1|441| +2450815|17827|1|244| +2450815|17828|1|86| +2450815|17830|1|508| +2450815|17833|1|869| +2450815|17834|1|919| +2450815|17836|1|578| +2450815|17839|1|326| +2450815|17840|1|943| +2450815|17842|1|609| +2450815|17845|1|142| +2450815|17846|1|752| +2450815|17848|1|739| +2450815|17851|1|880| +2450815|17852|1|591| +2450815|17854|1|957| +2450815|17857|1|374| +2450815|17858|1|293| +2450815|17860|1|353| +2450815|17863|1|714| +2450815|17864|1|125| +2450815|17866|1|222| +2450815|17869|1|610| +2450815|17870|1|936| +2450815|17872|1|48| +2450815|17875|1|324| +2450815|17876|1|497| +2450815|17878|1|3| +2450815|17881|1|431| +2450815|17882|1|741| +2450815|17884|1|901| +2450815|17887|1|44| +2450815|17888|1|499| +2450815|17890|1|504| +2450815|17893|1|143| +2450815|17894|1|320| +2450815|17896|1|5| +2450815|17899|1|485| +2450815|17900|1|92| +2450815|17902|1|361| +2450815|17905|1|154| +2450815|17906|1|648| +2450815|17908|1|| +2450815|17911|1|555| +2450815|17912|1|825| +2450815|17914|1|382| +2450815|17917|1|443| +2450815|17918|1|85| +2450815|17920|1|844| +2450815|17923|1|104| +2450815|17924|1|541| +2450815|17926|1|577| +2450815|17929|1|475| +2450815|17930|1|129| +2450815|17932|1|376| +2450815|17935|1|566| +2450815|17936|1|787| +2450815|17938|1|421| +2450815|17941|1|972| +2450815|17942|1|349| +2450815|17944|1|11| +2450815|17947|1|755| +2450815|17948|1|52| +2450815|17950|1|789| +2450815|17953|1|816| +2450815|17954|1|974| +2450815|17956|1|253| +2450815|17959|1|595| +2450815|17960|1|534| +2450815|17962|1|378| +2450815|17965|1|372| +2450815|17966|1|685| +2450815|17968|1|405| +2450815|17971|1|494| +2450815|17972|1|299| +2450815|17974|1|951| +2450815|17977|1|211| +2450815|17978|1|349| +2450815|17980|1|350| +2450815|17983|1|677| +2450815|17984|1|651| +2450815|17986|1|516| +2450815|17989|1|774| +2450815|17990|1|570| +2450815|17992|1|366| +2450815|17995|1|951| +2450815|17996|1|953| +2450815|17998|1|50| +2450815|1|2|824| +2450815|2|2|269| +2450815|4|2|522| +2450815|7|2|568| +2450815|8|2|703| +2450815|10|2|919| +2450815|13|2|850| +2450815|14|2|547| +2450815|16|2|305| +2450815|19|2|823| +2450815|20|2|414| +2450815|22|2|458| +2450815|25|2|879| +2450815|26|2|725| +2450815|28|2|699| +2450815|31|2|119| +2450815|32|2|728| +2450815|34|2|481| +2450815|37|2|800| +2450815|38|2|421| +2450815|40|2|303| +2450815|43|2|525| +2450815|44|2|911| +2450815|46|2|262| +2450815|49|2|278| +2450815|50|2|851| +2450815|52|2|317| +2450815|55|2|690| +2450815|56|2|444| +2450815|58|2|197| +2450815|61|2|303| +2450815|62|2|55| +2450815|64|2|296| +2450815|67|2|993| +2450815|68|2|344| +2450815|70|2|63| +2450815|73|2|592| +2450815|74|2|260| +2450815|76|2|784| +2450815|79|2|591| +2450815|80|2|863| +2450815|82|2|281| +2450815|85|2|520| +2450815|86|2|527| +2450815|88|2|762| +2450815|91|2|975| +2450815|92|2|994| +2450815|94|2|900| +2450815|97|2|109| +2450815|98|2|859| +2450815|100|2|549| +2450815|103|2|496| +2450815|104|2|268| +2450815|106|2|801| +2450815|109|2|427| +2450815|110|2|114| +2450815|112|2|475| +2450815|115|2|515| +2450815|116|2|715| +2450815|118|2|| +2450815|121|2|674| +2450815|122|2|53| +2450815|124|2|147| +2450815|127|2|298| +2450815|128|2|103| +2450815|130|2|161| +2450815|133|2|500| +2450815|134|2|638| +2450815|136|2|859| +2450815|139|2|| +2450815|140|2|273| +2450815|142|2|362| +2450815|145|2|464| +2450815|146|2|416| +2450815|148|2|460| +2450815|151|2|546| +2450815|152|2|783| +2450815|154|2|880| +2450815|157|2|696| +2450815|158|2|168| +2450815|160|2|262| +2450815|163|2|631| +2450815|164|2|926| +2450815|166|2|284| +2450815|169|2|126| +2450815|170|2|| +2450815|172|2|808| +2450815|175|2|614| +2450815|176|2|529| +2450815|178|2|738| +2450815|181|2|890| +2450815|182|2|9| +2450815|184|2|272| +2450815|187|2|427| +2450815|188|2|835| +2450815|190|2|634| +2450815|193|2|875| +2450815|194|2|594| +2450815|196|2|641| +2450815|199|2|945| +2450815|200|2|112| +2450815|202|2|672| +2450815|205|2|961| +2450815|206|2|144| +2450815|208|2|424| +2450815|211|2|184| +2450815|212|2|812| +2450815|214|2|804| +2450815|217|2|578| +2450815|218|2|685| +2450815|220|2|851| +2450815|223|2|194| +2450815|224|2|683| +2450815|226|2|186| +2450815|229|2|62| +2450815|230|2|916| +2450815|232|2|238| +2450815|235|2|431| +2450815|236|2|985| +2450815|238|2|410| +2450815|241|2|689| +2450815|242|2|407| +2450815|244|2|568| +2450815|247|2|507| +2450815|248|2|174| +2450815|250|2|966| +2450815|253|2|503| +2450815|254|2|304| +2450815|256|2|117| +2450815|259|2|193| +2450815|260|2|597| +2450815|262|2|| +2450815|265|2|475| +2450815|266|2|51| +2450815|268|2|972| +2450815|271|2|139| +2450815|272|2|764| +2450815|274|2|394| +2450815|277|2|332| +2450815|278|2|901| +2450815|280|2|526| +2450815|283|2|582| +2450815|284|2|227| +2450815|286|2|245| +2450815|289|2|994| +2450815|290|2|637| +2450815|292|2|188| +2450815|295|2|14| +2450815|296|2|243| +2450815|298|2|960| +2450815|301|2|309| +2450815|302|2|440| +2450815|304|2|247| +2450815|307|2|218| +2450815|308|2|25| +2450815|310|2|528| +2450815|313|2|746| +2450815|314|2|474| +2450815|316|2|786| +2450815|319|2|912| +2450815|320|2|659| +2450815|322|2|360| +2450815|325|2|835| +2450815|326|2|150| +2450815|328|2|548| +2450815|331|2|116| +2450815|332|2|925| +2450815|334|2|168| +2450815|337|2|552| +2450815|338|2|| +2450815|340|2|534| +2450815|343|2|621| +2450815|344|2|523| +2450815|346|2|511| +2450815|349|2|753| +2450815|350|2|657| +2450815|352|2|710| +2450815|355|2|295| +2450815|356|2|501| +2450815|358|2|383| +2450815|361|2|195| +2450815|362|2|920| +2450815|364|2|130| +2450815|367|2|920| +2450815|368|2|182| +2450815|370|2|902| +2450815|373|2|534| +2450815|374|2|552| +2450815|376|2|506| +2450815|379|2|627| +2450815|380|2|99| +2450815|382|2|939| +2450815|385|2|35| +2450815|386|2|889| +2450815|388|2|158| +2450815|391|2|400| +2450815|392|2|502| +2450815|394|2|985| +2450815|397|2|818| +2450815|398|2|155| +2450815|400|2|63| +2450815|403|2|911| +2450815|404|2|819| +2450815|406|2|331| +2450815|409|2|245| +2450815|410|2|137| +2450815|412|2|903| +2450815|415|2|286| +2450815|416|2|220| +2450815|418|2|762| +2450815|421|2|206| +2450815|422|2|106| +2450815|424|2|372| +2450815|427|2|686| +2450815|428|2|29| +2450815|430|2|| +2450815|433|2|426| +2450815|434|2|546| +2450815|436|2|134| +2450815|439|2|247| +2450815|440|2|480| +2450815|442|2|622| +2450815|445|2|541| +2450815|446|2|890| +2450815|448|2|955| +2450815|451|2|512| +2450815|452|2|108| +2450815|454|2|880| +2450815|457|2|78| +2450815|458|2|305| +2450815|460|2|10| +2450815|463|2|902| +2450815|464|2|949| +2450815|466|2|379| +2450815|469|2|756| +2450815|470|2|706| +2450815|472|2|771| +2450815|475|2|| +2450815|476|2|609| +2450815|478|2|437| +2450815|481|2|6| +2450815|482|2|114| +2450815|484|2|| +2450815|487|2|645| +2450815|488|2|347| +2450815|490|2|835| +2450815|493|2|742| +2450815|494|2|728| +2450815|496|2|696| +2450815|499|2|15| +2450815|500|2|364| +2450815|502|2|832| +2450815|505|2|752| +2450815|506|2|11| +2450815|508|2|253| +2450815|511|2|883| +2450815|512|2|821| +2450815|514|2|919| +2450815|517|2|212| +2450815|518|2|21| +2450815|520|2|67| +2450815|523|2|| +2450815|524|2|444| +2450815|526|2|275| +2450815|529|2|36| +2450815|530|2|936| +2450815|532|2|471| +2450815|535|2|493| +2450815|536|2|428| +2450815|538|2|548| +2450815|541|2|718| +2450815|542|2|247| +2450815|544|2|382| +2450815|547|2|813| +2450815|548|2|88| +2450815|550|2|728| +2450815|553|2|925| +2450815|554|2|562| +2450815|556|2|808| +2450815|559|2|611| +2450815|560|2|926| +2450815|562|2|664| +2450815|565|2|| +2450815|566|2|4| +2450815|568|2|135| +2450815|571|2|236| +2450815|572|2|472| +2450815|574|2|673| +2450815|577|2|1000| +2450815|578|2|784| +2450815|580|2|240| +2450815|583|2|| +2450815|584|2|111| +2450815|586|2|871| +2450815|589|2|553| +2450815|590|2|407| +2450815|592|2|698| +2450815|595|2|758| +2450815|596|2|600| +2450815|598|2|20| +2450815|601|2|636| +2450815|602|2|| +2450815|604|2|974| +2450815|607|2|575| +2450815|608|2|433| +2450815|610|2|898| +2450815|613|2|254| +2450815|614|2|243| +2450815|616|2|731| +2450815|619|2|350| +2450815|620|2|742| +2450815|622|2|33| +2450815|625|2|110| +2450815|626|2|916| +2450815|628|2|20| +2450815|631|2|901| +2450815|632|2|944| +2450815|634|2|189| +2450815|637|2|345| +2450815|638|2|520| +2450815|640|2|891| +2450815|643|2|58| +2450815|644|2|264| +2450815|646|2|769| +2450815|649|2|1| +2450815|650|2|489| +2450815|652|2|827| +2450815|655|2|571| +2450815|656|2|285| +2450815|658|2|940| +2450815|661|2|850| +2450815|662|2|216| +2450815|664|2|926| +2450815|667|2|872| +2450815|668|2|383| +2450815|670|2|783| +2450815|673|2|85| +2450815|674|2|593| +2450815|676|2|342| +2450815|679|2|725| +2450815|680|2|12| +2450815|682|2|757| +2450815|685|2|945| +2450815|686|2|116| +2450815|688|2|90| +2450815|691|2|105| +2450815|692|2|719| +2450815|694|2|989| +2450815|697|2|141| +2450815|698|2|101| +2450815|700|2|809| +2450815|703|2|317| +2450815|704|2|481| +2450815|706|2|527| +2450815|709|2|95| +2450815|710|2|813| +2450815|712|2|223| +2450815|715|2|604| +2450815|716|2|888| +2450815|718|2|198| +2450815|721|2|66| +2450815|722|2|782| +2450815|724|2|168| +2450815|727|2|57| +2450815|728|2|741| +2450815|730|2|652| +2450815|733|2|116| +2450815|734|2|898| +2450815|736|2|| +2450815|739|2|813| +2450815|740|2|677| +2450815|742|2|801| +2450815|745|2|946| +2450815|746|2|899| +2450815|748|2|525| +2450815|751|2|13| +2450815|752|2|45| +2450815|754|2|475| +2450815|757|2|| +2450815|758|2|887| +2450815|760|2|132| +2450815|763|2|901| +2450815|764|2|747| +2450815|766|2|144| +2450815|769|2|935| +2450815|770|2|157| +2450815|772|2|352| +2450815|775|2|32| +2450815|776|2|740| +2450815|778|2|371| +2450815|781|2|| +2450815|782|2|54| +2450815|784|2|| +2450815|787|2|597| +2450815|788|2|93| +2450815|790|2|312| +2450815|793|2|433| +2450815|794|2|103| +2450815|796|2|999| +2450815|799|2|320| +2450815|800|2|701| +2450815|802|2|392| +2450815|805|2|684| +2450815|806|2|164| +2450815|808|2|183| +2450815|811|2|443| +2450815|812|2|325| +2450815|814|2|760| +2450815|817|2|763| +2450815|818|2|739| +2450815|820|2|573| +2450815|823|2|953| +2450815|824|2|334| +2450815|826|2|797| +2450815|829|2|384| +2450815|830|2|901| +2450815|832|2|276| +2450815|835|2|610| +2450815|836|2|799| +2450815|838|2|480| +2450815|841|2|43| +2450815|842|2|51| +2450815|844|2|83| +2450815|847|2|677| +2450815|848|2|939| +2450815|850|2|952| +2450815|853|2|565| +2450815|854|2|208| +2450815|856|2|402| +2450815|859|2|| +2450815|860|2|290| +2450815|862|2|10| +2450815|865|2|906| +2450815|866|2|237| +2450815|868|2|306| +2450815|871|2|| +2450815|872|2|709| +2450815|874|2|730| +2450815|877|2|195| +2450815|878|2|67| +2450815|880|2|747| +2450815|883|2|524| +2450815|884|2|472| +2450815|886|2|209| +2450815|889|2|202| +2450815|890|2|86| +2450815|892|2|742| +2450815|895|2|423| +2450815|896|2|593| +2450815|898|2|851| +2450815|901|2|961| +2450815|902|2|862| +2450815|904|2|158| +2450815|907|2|103| +2450815|908|2|675| +2450815|910|2|284| +2450815|913|2|504| +2450815|914|2|398| +2450815|916|2|600| +2450815|919|2|862| +2450815|920|2|135| +2450815|922|2|682| +2450815|925|2|660| +2450815|926|2|742| +2450815|928|2|79| +2450815|931|2|444| +2450815|932|2|27| +2450815|934|2|6| +2450815|937|2|984| +2450815|938|2|500| +2450815|940|2|649| +2450815|943|2|123| +2450815|944|2|383| +2450815|946|2|272| +2450815|949|2|621| +2450815|950|2|730| +2450815|952|2|748| +2450815|955|2|| +2450815|956|2|81| +2450815|958|2|364| +2450815|961|2|104| +2450815|962|2|423| +2450815|964|2|556| +2450815|967|2|837| +2450815|968|2|145| +2450815|970|2|67| +2450815|973|2|280| +2450815|974|2|953| +2450815|976|2|936| +2450815|979|2|68| +2450815|980|2|| +2450815|982|2|113| +2450815|985|2|437| +2450815|986|2|289| +2450815|988|2|26| +2450815|991|2|637| +2450815|992|2|409| +2450815|994|2|655| +2450815|997|2|421| +2450815|998|2|206| +2450815|1000|2|375| +2450815|1003|2|265| +2450815|1004|2|616| +2450815|1006|2|381| +2450815|1009|2|181| +2450815|1010|2|897| +2450815|1012|2|450| +2450815|1015|2|687| +2450815|1016|2|246| +2450815|1018|2|989| +2450815|1021|2|456| +2450815|1022|2|807| +2450815|1024|2|685| +2450815|1027|2|107| +2450815|1028|2|656| +2450815|1030|2|| +2450815|1033|2|297| +2450815|1034|2|174| +2450815|1036|2|409| +2450815|1039|2|622| +2450815|1040|2|672| +2450815|1042|2|| +2450815|1045|2|316| +2450815|1046|2|986| +2450815|1048|2|693| +2450815|1051|2|350| +2450815|1052|2|917| +2450815|1054|2|754| +2450815|1057|2|880| +2450815|1058|2|775| +2450815|1060|2|17| +2450815|1063|2|849| +2450815|1064|2|384| +2450815|1066|2|190| +2450815|1069|2|881| +2450815|1070|2|231| +2450815|1072|2|411| +2450815|1075|2|887| +2450815|1076|2|119| +2450815|1078|2|110| +2450815|1081|2|13| +2450815|1082|2|437| +2450815|1084|2|31| +2450815|1087|2|903| +2450815|1088|2|678| +2450815|1090|2|335| +2450815|1093|2|694| +2450815|1094|2|| +2450815|1096|2|4| +2450815|1099|2|73| +2450815|1100|2|545| +2450815|1102|2|954| +2450815|1105|2|584| +2450815|1106|2|221| +2450815|1108|2|452| +2450815|1111|2|| +2450815|1112|2|856| +2450815|1114|2|165| +2450815|1117|2|921| +2450815|1118|2|268| +2450815|1120|2|772| +2450815|1123|2|918| +2450815|1124|2|110| +2450815|1126|2|577| +2450815|1129|2|705| +2450815|1130|2|| +2450815|1132|2|591| +2450815|1135|2|602| +2450815|1136|2|324| +2450815|1138|2|721| +2450815|1141|2|537| +2450815|1142|2|493| +2450815|1144|2|913| +2450815|1147|2|| +2450815|1148|2|951| +2450815|1150|2|879| +2450815|1153|2|669| +2450815|1154|2|51| +2450815|1156|2|304| +2450815|1159|2|788| +2450815|1160|2|350| +2450815|1162|2|345| +2450815|1165|2|353| +2450815|1166|2|728| +2450815|1168|2|807| +2450815|1171|2|425| +2450815|1172|2|399| +2450815|1174|2|235| +2450815|1177|2|389| +2450815|1178|2|| +2450815|1180|2|726| +2450815|1183|2|976| +2450815|1184|2|694| +2450815|1186|2|478| +2450815|1189|2|572| +2450815|1190|2|355| +2450815|1192|2|381| +2450815|1195|2|716| +2450815|1196|2|178| +2450815|1198|2|94| +2450815|1201|2|397| +2450815|1202|2|| +2450815|1204|2|880| +2450815|1207|2|31| +2450815|1208|2|43| +2450815|1210|2|799| +2450815|1213|2|43| +2450815|1214|2|347| +2450815|1216|2|413| +2450815|1219|2|362| +2450815|1220|2|274| +2450815|1222|2|825| +2450815|1225|2|137| +2450815|1226|2|| +2450815|1228|2|424| +2450815|1231|2|865| +2450815|1232|2|291| +2450815|1234|2|548| +2450815|1237|2|741| +2450815|1238|2|88| +2450815|1240|2|395| +2450815|1243|2|103| +2450815|1244|2|436| +2450815|1246|2|38| +2450815|1249|2|677| +2450815|1250|2|36| +2450815|1252|2|824| +2450815|1255|2|827| +2450815|1256|2|554| +2450815|1258|2|819| +2450815|1261|2|714| +2450815|1262|2|| +2450815|1264|2|77| +2450815|1267|2|413| +2450815|1268|2|636| +2450815|1270|2|333| +2450815|1273|2|918| +2450815|1274|2|989| +2450815|1276|2|97| +2450815|1279|2|45| +2450815|1280|2|34| +2450815|1282|2|210| +2450815|1285|2|743| +2450815|1286|2|72| +2450815|1288|2|543| +2450815|1291|2|433| +2450815|1292|2|196| +2450815|1294|2|136| +2450815|1297|2|772| +2450815|1298|2|526| +2450815|1300|2|802| +2450815|1303|2|447| +2450815|1304|2|751| +2450815|1306|2|698| +2450815|1309|2|928| +2450815|1310|2|960| +2450815|1312|2|140| +2450815|1315|2|494| +2450815|1316|2|149| +2450815|1318|2|488| +2450815|1321|2|850| +2450815|1322|2|554| +2450815|1324|2|539| +2450815|1327|2|516| +2450815|1328|2|169| +2450815|1330|2|308| +2450815|1333|2|634| +2450815|1334|2|930| +2450815|1336|2|456| +2450815|1339|2|952| +2450815|1340|2|308| +2450815|1342|2|965| +2450815|1345|2|555| +2450815|1346|2|927| +2450815|1348|2|843| +2450815|1351|2|477| +2450815|1352|2|889| +2450815|1354|2|624| +2450815|1357|2|239| +2450815|1358|2|118| +2450815|1360|2|267| +2450815|1363|2|881| +2450815|1364|2|| +2450815|1366|2|710| +2450815|1369|2|48| +2450815|1370|2|760| +2450815|1372|2|282| +2450815|1375|2|710| +2450815|1376|2|715| +2450815|1378|2|891| +2450815|1381|2|692| +2450815|1382|2|319| +2450815|1384|2|545| +2450815|1387|2|344| +2450815|1388|2|199| +2450815|1390|2|217| +2450815|1393|2|| +2450815|1394|2|242| +2450815|1396|2|328| +2450815|1399|2|700| +2450815|1400|2|342| +2450815|1402|2|| +2450815|1405|2|44| +2450815|1406|2|450| +2450815|1408|2|525| +2450815|1411|2|701| +2450815|1412|2|387| +2450815|1414|2|540| +2450815|1417|2|11| +2450815|1418|2|855| +2450815|1420|2|925| +2450815|1423|2|515| +2450815|1424|2|199| +2450815|1426|2|541| +2450815|1429|2|929| +2450815|1430|2|324| +2450815|1432|2|187| +2450815|1435|2|| +2450815|1436|2|| +2450815|1438|2|498| +2450815|1441|2|517| +2450815|1442|2|270| +2450815|1444|2|738| +2450815|1447|2|27| +2450815|1448|2|659| +2450815|1450|2|403| +2450815|1453|2|767| +2450815|1454|2|859| +2450815|1456|2|939| +2450815|1459|2|977| +2450815|1460|2|584| +2450815|1462|2|583| +2450815|1465|2|541| +2450815|1466|2|733| +2450815|1468|2|966| +2450815|1471|2|542| +2450815|1472|2|478| +2450815|1474|2|347| +2450815|1477|2|281| +2450815|1478|2|397| +2450815|1480|2|435| +2450815|1483|2|536| +2450815|1484|2|744| +2450815|1486|2|861| +2450815|1489|2|362| +2450815|1490|2|220| +2450815|1492|2|733| +2450815|1495|2|| +2450815|1496|2|355| +2450815|1498|2|614| +2450815|1501|2|475| +2450815|1502|2|560| +2450815|1504|2|315| +2450815|1507|2|559| +2450815|1508|2|678| +2450815|1510|2|963| +2450815|1513|2|939| +2450815|1514|2|671| +2450815|1516|2|881| +2450815|1519|2|143| +2450815|1520|2|328| +2450815|1522|2|494| +2450815|1525|2|394| +2450815|1526|2|373| +2450815|1528|2|534| +2450815|1531|2|64| +2450815|1532|2|871| +2450815|1534|2|656| +2450815|1537|2|132| +2450815|1538|2|443| +2450815|1540|2|768| +2450815|1543|2|510| +2450815|1544|2|349| +2450815|1546|2|501| +2450815|1549|2|106| +2450815|1550|2|732| +2450815|1552|2|15| +2450815|1555|2|806| +2450815|1556|2|255| +2450815|1558|2|47| +2450815|1561|2|21| +2450815|1562|2|871| +2450815|1564|2|314| +2450815|1567|2|918| +2450815|1568|2|800| +2450815|1570|2|992| +2450815|1573|2|599| +2450815|1574|2|698| +2450815|1576|2|223| +2450815|1579|2|524| +2450815|1580|2|887| +2450815|1582|2|56| +2450815|1585|2|45| +2450815|1586|2|218| +2450815|1588|2|334| +2450815|1591|2|856| +2450815|1592|2|267| +2450815|1594|2|848| +2450815|1597|2|717| +2450815|1598|2|986| +2450815|1600|2|293| +2450815|1603|2|208| +2450815|1604|2|485| +2450815|1606|2|486| +2450815|1609|2|415| +2450815|1610|2|448| +2450815|1612|2|127| +2450815|1615|2|733| +2450815|1616|2|| +2450815|1618|2|992| +2450815|1621|2|415| +2450815|1622|2|713| +2450815|1624|2|| +2450815|1627|2|229| +2450815|1628|2|282| +2450815|1630|2|56| +2450815|1633|2|986| +2450815|1634|2|785| +2450815|1636|2|118| +2450815|1639|2|366| +2450815|1640|2|439| +2450815|1642|2|697| +2450815|1645|2|250| +2450815|1646|2|966| +2450815|1648|2|722| +2450815|1651|2|12| +2450815|1652|2|681| +2450815|1654|2|504| +2450815|1657|2|442| +2450815|1658|2|206| +2450815|1660|2|651| +2450815|1663|2|190| +2450815|1664|2|| +2450815|1666|2|835| +2450815|1669|2|262| +2450815|1670|2|236| +2450815|1672|2|235| +2450815|1675|2|781| +2450815|1676|2|546| +2450815|1678|2|400| +2450815|1681|2|835| +2450815|1682|2|158| +2450815|1684|2|362| +2450815|1687|2|49| +2450815|1688|2|518| +2450815|1690|2|967| +2450815|1693|2|55| +2450815|1694|2|196| +2450815|1696|2|290| +2450815|1699|2|985| +2450815|1700|2|346| +2450815|1702|2|632| +2450815|1705|2|676| +2450815|1706|2|497| +2450815|1708|2|589| +2450815|1711|2|733| +2450815|1712|2|612| +2450815|1714|2|515| +2450815|1717|2|| +2450815|1718|2|887| +2450815|1720|2|| +2450815|1723|2|226| +2450815|1724|2|| +2450815|1726|2|705| +2450815|1729|2|238| +2450815|1730|2|78| +2450815|1732|2|29| +2450815|1735|2|260| +2450815|1736|2|432| +2450815|1738|2|284| +2450815|1741|2|| +2450815|1742|2|| +2450815|1744|2|940| +2450815|1747|2|266| +2450815|1748|2|396| +2450815|1750|2|847| +2450815|1753|2|502| +2450815|1754|2|954| +2450815|1756|2|374| +2450815|1759|2|801| +2450815|1760|2|908| +2450815|1762|2|523| +2450815|1765|2|448| +2450815|1766|2|909| +2450815|1768|2|448| +2450815|1771|2|849| +2450815|1772|2|221| +2450815|1774|2|| +2450815|1777|2|638| +2450815|1778|2|712| +2450815|1780|2|244| +2450815|1783|2|881| +2450815|1784|2|356| +2450815|1786|2|451| +2450815|1789|2|298| +2450815|1790|2|872| +2450815|1792|2|144| +2450815|1795|2|328| +2450815|1796|2|478| +2450815|1798|2|829| +2450815|1801|2|118| +2450815|1802|2|493| +2450815|1804|2|422| +2450815|1807|2|186| +2450815|1808|2|574| +2450815|1810|2|755| +2450815|1813|2|33| +2450815|1814|2|773| +2450815|1816|2|739| +2450815|1819|2|| +2450815|1820|2|251| +2450815|1822|2|119| +2450815|1825|2|524| +2450815|1826|2|437| +2450815|1828|2|475| +2450815|1831|2|488| +2450815|1832|2|142| +2450815|1834|2|696| +2450815|1837|2|155| +2450815|1838|2|220| +2450815|1840|2|951| +2450815|1843|2|290| +2450815|1844|2|888| +2450815|1846|2|67| +2450815|1849|2|882| +2450815|1850|2|| +2450815|1852|2|821| +2450815|1855|2|547| +2450815|1856|2|672| +2450815|1858|2|689| +2450815|1861|2|742| +2450815|1862|2|394| +2450815|1864|2|721| +2450815|1867|2|192| +2450815|1868|2|268| +2450815|1870|2|304| +2450815|1873|2|548| +2450815|1874|2|969| +2450815|1876|2|331| +2450815|1879|2|| +2450815|1880|2|509| +2450815|1882|2|329| +2450815|1885|2|276| +2450815|1886|2|702| +2450815|1888|2|232| +2450815|1891|2|447| +2450815|1892|2|337| +2450815|1894|2|977| +2450815|1897|2|46| +2450815|1898|2|742| +2450815|1900|2|387| +2450815|1903|2|325| +2450815|1904|2|547| +2450815|1906|2|408| +2450815|1909|2|527| +2450815|1910|2|922| +2450815|1912|2|817| +2450815|1915|2|381| +2450815|1916|2|293| +2450815|1918|2|257| +2450815|1921|2|606| +2450815|1922|2|560| +2450815|1924|2|911| +2450815|1927|2|767| +2450815|1928|2|950| +2450815|1930|2|282| +2450815|1933|2|531| +2450815|1934|2|857| +2450815|1936|2|203| +2450815|1939|2|66| +2450815|1940|2|867| +2450815|1942|2|532| +2450815|1945|2|937| +2450815|1946|2|174| +2450815|1948|2|371| +2450815|1951|2|379| +2450815|1952|2|630| +2450815|1954|2|57| +2450815|1957|2|150| +2450815|1958|2|842| +2450815|1960|2|979| +2450815|1963|2|261| +2450815|1964|2|374| +2450815|1966|2|583| +2450815|1969|2|888| +2450815|1970|2|55| +2450815|1972|2|751| +2450815|1975|2|100| +2450815|1976|2|711| +2450815|1978|2|68| +2450815|1981|2|10| +2450815|1982|2|111| +2450815|1984|2|464| +2450815|1987|2|190| +2450815|1988|2|505| +2450815|1990|2|60| +2450815|1993|2|198| +2450815|1994|2|828| +2450815|1996|2|867| +2450815|1999|2|110| +2450815|2000|2|2| +2450815|2002|2|307| +2450815|2005|2|600| +2450815|2006|2|| +2450815|2008|2|967| +2450815|2011|2|585| +2450815|2012|2|392| +2450815|2014|2|643| +2450815|2017|2|516| +2450815|2018|2|984| +2450815|2020|2|695| +2450815|2023|2|885| +2450815|2024|2|613| +2450815|2026|2|126| +2450815|2029|2|600| +2450815|2030|2|796| +2450815|2032|2|101| +2450815|2035|2|140| +2450815|2036|2|320| +2450815|2038|2|814| +2450815|2041|2|567| +2450815|2042|2|203| +2450815|2044|2|636| +2450815|2047|2|| +2450815|2048|2|123| +2450815|2050|2|958| +2450815|2053|2|41| +2450815|2054|2|248| +2450815|2056|2|453| +2450815|2059|2|399| +2450815|2060|2|418| +2450815|2062|2|625| +2450815|2065|2|366| +2450815|2066|2|507| +2450815|2068|2|101| +2450815|2071|2|719| +2450815|2072|2|411| +2450815|2074|2|727| +2450815|2077|2|452| +2450815|2078|2|830| +2450815|2080|2|283| +2450815|2083|2|283| +2450815|2084|2|514| +2450815|2086|2|126| +2450815|2089|2|325| +2450815|2090|2|478| +2450815|2092|2|112| +2450815|2095|2|461| +2450815|2096|2|787| +2450815|2098|2|623| +2450815|2101|2|502| +2450815|2102|2|174| +2450815|2104|2|898| +2450815|2107|2|846| +2450815|2108|2|515| +2450815|2110|2|361| +2450815|2113|2|29| +2450815|2114|2|761| +2450815|2116|2|814| +2450815|2119|2|606| +2450815|2120|2|120| +2450815|2122|2|599| +2450815|2125|2|201| +2450815|2126|2|148| +2450815|2128|2|| +2450815|2131|2|859| +2450815|2132|2|371| +2450815|2134|2|891| +2450815|2137|2|635| +2450815|2138|2|976| +2450815|2140|2|154| +2450815|2143|2|178| +2450815|2144|2|350| +2450815|2146|2|903| +2450815|2149|2|143| +2450815|2150|2|655| +2450815|2152|2|| +2450815|2155|2|451| +2450815|2156|2|| +2450815|2158|2|516| +2450815|2161|2|| +2450815|2162|2|201| +2450815|2164|2|944| +2450815|2167|2|82| +2450815|2168|2|168| +2450815|2170|2|420| +2450815|2173|2|424| +2450815|2174|2|316| +2450815|2176|2|398| +2450815|2179|2|875| +2450815|2180|2|960| +2450815|2182|2|757| +2450815|2185|2|937| +2450815|2186|2|760| +2450815|2188|2|162| +2450815|2191|2|832| +2450815|2192|2|286| +2450815|2194|2|549| +2450815|2197|2|227| +2450815|2198|2|959| +2450815|2200|2|14| +2450815|2203|2|300| +2450815|2204|2|731| +2450815|2206|2|46| +2450815|2209|2|975| +2450815|2210|2|233| +2450815|2212|2|283| +2450815|2215|2|887| +2450815|2216|2|880| +2450815|2218|2|329| +2450815|2221|2|755| +2450815|2222|2|568| +2450815|2224|2|106| +2450815|2227|2|784| +2450815|2228|2|762| +2450815|2230|2|601| +2450815|2233|2|513| +2450815|2234|2|214| +2450815|2236|2|622| +2450815|2239|2|884| +2450815|2240|2|174| +2450815|2242|2|983| +2450815|2245|2|252| +2450815|2246|2|946| +2450815|2248|2|924| +2450815|2251|2|438| +2450815|2252|2|482| +2450815|2254|2|597| +2450815|2257|2|204| +2450815|2258|2|| +2450815|2260|2|464| +2450815|2263|2|660| +2450815|2264|2|686| +2450815|2266|2|838| +2450815|2269|2|738| +2450815|2270|2|582| +2450815|2272|2|374| +2450815|2275|2|123| +2450815|2276|2|453| +2450815|2278|2|80| +2450815|2281|2|133| +2450815|2282|2|956| +2450815|2284|2|117| +2450815|2287|2|390| +2450815|2288|2|| +2450815|2290|2|757| +2450815|2293|2|871| +2450815|2294|2|917| +2450815|2296|2|427| +2450815|2299|2|691| +2450815|2300|2|423| +2450815|2302|2|905| +2450815|2305|2|9| +2450815|2306|2|128| +2450815|2308|2|88| +2450815|2311|2|719| +2450815|2312|2|602| +2450815|2314|2|980| +2450815|2317|2|521| +2450815|2318|2|240| +2450815|2320|2|245| +2450815|2323|2|364| +2450815|2324|2|166| +2450815|2326|2|83| +2450815|2329|2|351| +2450815|2330|2|283| +2450815|2332|2|721| +2450815|2335|2|110| +2450815|2336|2|53| +2450815|2338|2|943| +2450815|2341|2|94| +2450815|2342|2|388| +2450815|2344|2|306| +2450815|2347|2|873| +2450815|2348|2|| +2450815|2350|2|594| +2450815|2353|2|912| +2450815|2354|2|770| +2450815|2356|2|481| +2450815|2359|2|569| +2450815|2360|2|567| +2450815|2362|2|| +2450815|2365|2|304| +2450815|2366|2|370| +2450815|2368|2|795| +2450815|2371|2|427| +2450815|2372|2|| +2450815|2374|2|399| +2450815|2377|2|830| +2450815|2378|2|417| +2450815|2380|2|691| +2450815|2383|2|449| +2450815|2384|2|105| +2450815|2386|2|| +2450815|2389|2|442| +2450815|2390|2|278| +2450815|2392|2|480| +2450815|2395|2|774| +2450815|2396|2|43| +2450815|2398|2|82| +2450815|2401|2|122| +2450815|2402|2|109| +2450815|2404|2|766| +2450815|2407|2|623| +2450815|2408|2|967| +2450815|2410|2|| +2450815|2413|2|364| +2450815|2414|2|259| +2450815|2416|2|80| +2450815|2419|2|976| +2450815|2420|2|712| +2450815|2422|2|59| +2450815|2425|2|842| +2450815|2426|2|537| +2450815|2428|2|158| +2450815|2431|2|767| +2450815|2432|2|541| +2450815|2434|2|0| +2450815|2437|2|4| +2450815|2438|2|924| +2450815|2440|2|111| +2450815|2443|2|728| +2450815|2444|2|233| +2450815|2446|2|343| +2450815|2449|2|803| +2450815|2450|2|743| +2450815|2452|2|492| +2450815|2455|2|597| +2450815|2456|2|455| +2450815|2458|2|71| +2450815|2461|2|87| +2450815|2462|2|465| +2450815|2464|2|405| +2450815|2467|2|240| +2450815|2468|2|808| +2450815|2470|2|784| +2450815|2473|2|892| +2450815|2474|2|1000| +2450815|2476|2|412| +2450815|2479|2|551| +2450815|2480|2|| +2450815|2482|2|554| +2450815|2485|2|955| +2450815|2486|2|532| +2450815|2488|2|583| +2450815|2491|2|798| +2450815|2492|2|894| +2450815|2494|2|155| +2450815|2497|2|963| +2450815|2498|2|719| +2450815|2500|2|448| +2450815|2503|2|389| +2450815|2504|2|813| +2450815|2506|2|232| +2450815|2509|2|410| +2450815|2510|2|| +2450815|2512|2|68| +2450815|2515|2|607| +2450815|2516|2|| +2450815|2518|2|965| +2450815|2521|2|875| +2450815|2522|2|935| +2450815|2524|2|784| +2450815|2527|2|158| +2450815|2528|2|294| +2450815|2530|2|300| +2450815|2533|2|924| +2450815|2534|2|792| +2450815|2536|2|711| +2450815|2539|2|468| +2450815|2540|2|537| +2450815|2542|2|| +2450815|2545|2|966| +2450815|2546|2|655| +2450815|2548|2|361| +2450815|2551|2|573| +2450815|2552|2|225| +2450815|2554|2|454| +2450815|2557|2|671| +2450815|2558|2|595| +2450815|2560|2|117| +2450815|2563|2|696| +2450815|2564|2|483| +2450815|2566|2|147| +2450815|2569|2|438| +2450815|2570|2|718| +2450815|2572|2|141| +2450815|2575|2|321| +2450815|2576|2|712| +2450815|2578|2|68| +2450815|2581|2|445| +2450815|2582|2|357| +2450815|2584|2|383| +2450815|2587|2|178| +2450815|2588|2|994| +2450815|2590|2|258| +2450815|2593|2|166| +2450815|2594|2|307| +2450815|2596|2|577| +2450815|2599|2|949| +2450815|2600|2|622| +2450815|2602|2|789| +2450815|2605|2|329| +2450815|2606|2|94| +2450815|2608|2|34| +2450815|2611|2|158| +2450815|2612|2|490| +2450815|2614|2|108| +2450815|2617|2|847| +2450815|2618|2|972| +2450815|2620|2|667| +2450815|2623|2|751| +2450815|2624|2|475| +2450815|2626|2|774| +2450815|2629|2|735| +2450815|2630|2|969| +2450815|2632|2|690| +2450815|2635|2|525| +2450815|2636|2|281| +2450815|2638|2|473| +2450815|2641|2|210| +2450815|2642|2|720| +2450815|2644|2|669| +2450815|2647|2|795| +2450815|2648|2|580| +2450815|2650|2|924| +2450815|2653|2|989| +2450815|2654|2|972| +2450815|2656|2|560| +2450815|2659|2|| +2450815|2660|2|217| +2450815|2662|2|| +2450815|2665|2|879| +2450815|2666|2|286| +2450815|2668|2|791| +2450815|2671|2|180| +2450815|2672|2|509| +2450815|2674|2|652| +2450815|2677|2|745| +2450815|2678|2|178| +2450815|2680|2|789| +2450815|2683|2|992| +2450815|2684|2|| +2450815|2686|2|346| +2450815|2689|2|397| +2450815|2690|2|| +2450815|2692|2|195| +2450815|2695|2|571| +2450815|2696|2|588| +2450815|2698|2|99| +2450815|2701|2|121| +2450815|2702|2|971| +2450815|2704|2|998| +2450815|2707|2|201| +2450815|2708|2|103| +2450815|2710|2|549| +2450815|2713|2|152| +2450815|2714|2|581| +2450815|2716|2|387| +2450815|2719|2|186| +2450815|2720|2|843| +2450815|2722|2|793| +2450815|2725|2|252| +2450815|2726|2|961| +2450815|2728|2|812| +2450815|2731|2|348| +2450815|2732|2|102| +2450815|2734|2|59| +2450815|2737|2|942| +2450815|2738|2|| +2450815|2740|2|24| +2450815|2743|2|68| +2450815|2744|2|293| +2450815|2746|2|535| +2450815|2749|2|114| +2450815|2750|2|686| +2450815|2752|2|234| +2450815|2755|2|628| +2450815|2756|2|| +2450815|2758|2|51| +2450815|2761|2|640| +2450815|2762|2|159| +2450815|2764|2|698| +2450815|2767|2|121| +2450815|2768|2|684| +2450815|2770|2|611| +2450815|2773|2|136| +2450815|2774|2|685| +2450815|2776|2|| +2450815|2779|2|1| +2450815|2780|2|410| +2450815|2782|2|743| +2450815|2785|2|904| +2450815|2786|2|261| +2450815|2788|2|463| +2450815|2791|2|177| +2450815|2792|2|481| +2450815|2794|2|96| +2450815|2797|2|99| +2450815|2798|2|23| +2450815|2800|2|256| +2450815|2803|2|172| +2450815|2804|2|605| +2450815|2806|2|497| +2450815|2809|2|927| +2450815|2810|2|193| +2450815|2812|2|878| +2450815|2815|2|75| +2450815|2816|2|156| +2450815|2818|2|414| +2450815|2821|2|194| +2450815|2822|2|565| +2450815|2824|2|587| +2450815|2827|2|440| +2450815|2828|2|661| +2450815|2830|2|313| +2450815|2833|2|621| +2450815|2834|2|659| +2450815|2836|2|103| +2450815|2839|2|319| +2450815|2840|2|885| +2450815|2842|2|374| +2450815|2845|2|| +2450815|2846|2|695| +2450815|2848|2|76| +2450815|2851|2|263| +2450815|2852|2|272| +2450815|2854|2|742| +2450815|2857|2|885| +2450815|2858|2|93| +2450815|2860|2|827| +2450815|2863|2|121| +2450815|2864|2|801| +2450815|2866|2|56| +2450815|2869|2|573| +2450815|2870|2|310| +2450815|2872|2|391| +2450815|2875|2|406| +2450815|2876|2|963| +2450815|2878|2|755| +2450815|2881|2|822| +2450815|2882|2|317| +2450815|2884|2|602| +2450815|2887|2|181| +2450815|2888|2|342| +2450815|2890|2|526| +2450815|2893|2|932| +2450815|2894|2|765| +2450815|2896|2|37| +2450815|2899|2|727| +2450815|2900|2|258| +2450815|2902|2|802| +2450815|2905|2|560| +2450815|2906|2|930| +2450815|2908|2|722| +2450815|2911|2|| +2450815|2912|2|865| +2450815|2914|2|231| +2450815|2917|2|787| +2450815|2918|2|993| +2450815|2920|2|399| +2450815|2923|2|353| +2450815|2924|2|224| +2450815|2926|2|638| +2450815|2929|2|903| +2450815|2930|2|281| +2450815|2932|2|683| +2450815|2935|2|738| +2450815|2936|2|210| +2450815|2938|2|163| +2450815|2941|2|623| +2450815|2942|2|659| +2450815|2944|2|334| +2450815|2947|2|| +2450815|2948|2|622| +2450815|2950|2|500| +2450815|2953|2|410| +2450815|2954|2|860| +2450815|2956|2|140| +2450815|2959|2|| +2450815|2960|2|989| +2450815|2962|2|370| +2450815|2965|2|88| +2450815|2966|2|519| +2450815|2968|2|| +2450815|2971|2|457| +2450815|2972|2|560| +2450815|2974|2|936| +2450815|2977|2|62| +2450815|2978|2|355| +2450815|2980|2|695| +2450815|2983|2|594| +2450815|2984|2|622| +2450815|2986|2|26| +2450815|2989|2|| +2450815|2990|2|| +2450815|2992|2|268| +2450815|2995|2|357| +2450815|2996|2|293| +2450815|2998|2|58| +2450815|3001|2|885| +2450815|3002|2|175| +2450815|3004|2|272| +2450815|3007|2|553| +2450815|3008|2|| +2450815|3010|2|199| +2450815|3013|2|416| +2450815|3014|2|853| +2450815|3016|2|498| +2450815|3019|2|63| +2450815|3020|2|595| +2450815|3022|2|974| +2450815|3025|2|482| +2450815|3026|2|813| +2450815|3028|2|508| +2450815|3031|2|182| +2450815|3032|2|768| +2450815|3034|2|988| +2450815|3037|2|553| +2450815|3038|2|719| +2450815|3040|2|131| +2450815|3043|2|885| +2450815|3044|2|129| +2450815|3046|2|635| +2450815|3049|2|752| +2450815|3050|2|270| +2450815|3052|2|450| +2450815|3055|2|821| +2450815|3056|2|645| +2450815|3058|2|663| +2450815|3061|2|394| +2450815|3062|2|885| +2450815|3064|2|577| +2450815|3067|2|990| +2450815|3068|2|755| +2450815|3070|2|| +2450815|3073|2|211| +2450815|3074|2|572| +2450815|3076|2|914| +2450815|3079|2|287| +2450815|3080|2|103| +2450815|3082|2|650| +2450815|3085|2|242| +2450815|3086|2|809| +2450815|3088|2|193| +2450815|3091|2|592| +2450815|3092|2|| +2450815|3094|2|832| +2450815|3097|2|| +2450815|3098|2|96| +2450815|3100|2|228| +2450815|3103|2|148| +2450815|3104|2|720| +2450815|3106|2|313| +2450815|3109|2|207| +2450815|3110|2|62| +2450815|3112|2|402| +2450815|3115|2|967| +2450815|3116|2|589| +2450815|3118|2|27| +2450815|3121|2|832| +2450815|3122|2|636| +2450815|3124|2|328| +2450815|3127|2|7| +2450815|3128|2|682| +2450815|3130|2|399| +2450815|3133|2|457| +2450815|3134|2|333| +2450815|3136|2|294| +2450815|3139|2|350| +2450815|3140|2|88| +2450815|3142|2|362| +2450815|3145|2|350| +2450815|3146|2|191| +2450815|3148|2|149| +2450815|3151|2|781| +2450815|3152|2|707| +2450815|3154|2|732| +2450815|3157|2|82| +2450815|3158|2|49| +2450815|3160|2|635| +2450815|3163|2|674| +2450815|3164|2|355| +2450815|3166|2|215| +2450815|3169|2|926| +2450815|3170|2|372| +2450815|3172|2|454| +2450815|3175|2|| +2450815|3176|2|965| +2450815|3178|2|0| +2450815|3181|2|913| +2450815|3182|2|396| +2450815|3184|2|862| +2450815|3187|2|690| +2450815|3188|2|319| +2450815|3190|2|617| +2450815|3193|2|608| +2450815|3194|2|| +2450815|3196|2|423| +2450815|3199|2|94| +2450815|3200|2|682| +2450815|3202|2|| +2450815|3205|2|970| +2450815|3206|2|424| +2450815|3208|2|935| +2450815|3211|2|564| +2450815|3212|2|170| +2450815|3214|2|| +2450815|3217|2|808| +2450815|3218|2|335| +2450815|3220|2|957| +2450815|3223|2|324| +2450815|3224|2|331| +2450815|3226|2|390| +2450815|3229|2|110| +2450815|3230|2|749| +2450815|3232|2|471| +2450815|3235|2|557| +2450815|3236|2|932| +2450815|3238|2|537| +2450815|3241|2|445| +2450815|3242|2|941| +2450815|3244|2|215| +2450815|3247|2|527| +2450815|3248|2|768| +2450815|3250|2|648| +2450815|3253|2|741| +2450815|3254|2|18| +2450815|3256|2|572| +2450815|3259|2|| +2450815|3260|2|101| +2450815|3262|2|| +2450815|3265|2|679| +2450815|3266|2|37| +2450815|3268|2|899| +2450815|3271|2|763| +2450815|3272|2|803| +2450815|3274|2|551| +2450815|3277|2|265| +2450815|3278|2|160| +2450815|3280|2|852| +2450815|3283|2|503| +2450815|3284|2|439| +2450815|3286|2|409| +2450815|3289|2|506| +2450815|3290|2|325| +2450815|3292|2|428| +2450815|3295|2|15| +2450815|3296|2|98| +2450815|3298|2|855| +2450815|3301|2|867| +2450815|3302|2|392| +2450815|3304|2|141| +2450815|3307|2|207| +2450815|3308|2|697| +2450815|3310|2|906| +2450815|3313|2|332| +2450815|3314|2|101| +2450815|3316|2|778| +2450815|3319|2|652| +2450815|3320|2|396| +2450815|3322|2|785| +2450815|3325|2|| +2450815|3326|2|338| +2450815|3328|2|625| +2450815|3331|2|478| +2450815|3332|2|743| +2450815|3334|2|641| +2450815|3337|2|119| +2450815|3338|2|900| +2450815|3340|2|20| +2450815|3343|2|752| +2450815|3344|2|235| +2450815|3346|2|562| +2450815|3349|2|99| +2450815|3350|2|700| +2450815|3352|2|918| +2450815|3355|2|793| +2450815|3356|2|593| +2450815|3358|2|120| +2450815|3361|2|865| +2450815|3362|2|921| +2450815|3364|2|499| +2450815|3367|2|883| +2450815|3368|2|799| +2450815|3370|2|779| +2450815|3373|2|211| +2450815|3374|2|642| +2450815|3376|2|155| +2450815|3379|2|549| +2450815|3380|2|411| +2450815|3382|2|286| +2450815|3385|2|284| +2450815|3386|2|687| +2450815|3388|2|265| +2450815|3391|2|862| +2450815|3392|2|859| +2450815|3394|2|348| +2450815|3397|2|383| +2450815|3398|2|102| +2450815|3400|2|873| +2450815|3403|2|793| +2450815|3404|2|216| +2450815|3406|2|414| +2450815|3409|2|326| +2450815|3410|2|314| +2450815|3412|2|292| +2450815|3415|2|309| +2450815|3416|2|634| +2450815|3418|2|554| +2450815|3421|2|256| +2450815|3422|2|161| +2450815|3424|2|887| +2450815|3427|2|70| +2450815|3428|2|739| +2450815|3430|2|968| +2450815|3433|2|259| +2450815|3434|2|229| +2450815|3436|2|| +2450815|3439|2|916| +2450815|3440|2|| +2450815|3442|2|999| +2450815|3445|2|750| +2450815|3446|2|373| +2450815|3448|2|458| +2450815|3451|2|344| +2450815|3452|2|| +2450815|3454|2|216| +2450815|3457|2|228| +2450815|3458|2|117| +2450815|3460|2|23| +2450815|3463|2|| +2450815|3464|2|390| +2450815|3466|2|412| +2450815|3469|2|120| +2450815|3470|2|655| +2450815|3472|2|535| +2450815|3475|2|491| +2450815|3476|2|209| +2450815|3478|2|119| +2450815|3481|2|221| +2450815|3482|2|115| +2450815|3484|2|846| +2450815|3487|2|509| +2450815|3488|2|823| +2450815|3490|2|96| +2450815|3493|2|931| +2450815|3494|2|617| +2450815|3496|2|122| +2450815|3499|2|420| +2450815|3500|2|414| +2450815|3502|2|831| +2450815|3505|2|| +2450815|3506|2|960| +2450815|3508|2|89| +2450815|3511|2|113| +2450815|3512|2|286| +2450815|3514|2|462| +2450815|3517|2|133| +2450815|3518|2|259| +2450815|3520|2|| +2450815|3523|2|958| +2450815|3524|2|173| +2450815|3526|2|743| +2450815|3529|2|351| +2450815|3530|2|718| +2450815|3532|2|476| +2450815|3535|2|177| +2450815|3536|2|487| +2450815|3538|2|918| +2450815|3541|2|38| +2450815|3542|2|382| +2450815|3544|2|354| +2450815|3547|2|543| +2450815|3548|2|993| +2450815|3550|2|145| +2450815|3553|2|674| +2450815|3554|2|271| +2450815|3556|2|309| +2450815|3559|2|514| +2450815|3560|2|841| +2450815|3562|2|992| +2450815|3565|2|368| +2450815|3566|2|917| +2450815|3568|2|893| +2450815|3571|2|| +2450815|3572|2|927| +2450815|3574|2|275| +2450815|3577|2|572| +2450815|3578|2|244| +2450815|3580|2|18| +2450815|3583|2|818| +2450815|3584|2|381| +2450815|3586|2|669| +2450815|3589|2|| +2450815|3590|2|819| +2450815|3592|2|993| +2450815|3595|2|788| +2450815|3596|2|345| +2450815|3598|2|254| +2450815|3601|2|50| +2450815|3602|2|645| +2450815|3604|2|562| +2450815|3607|2|| +2450815|3608|2|157| +2450815|3610|2|805| +2450815|3613|2|108| +2450815|3614|2|299| +2450815|3616|2|801| +2450815|3619|2|439| +2450815|3620|2|898| +2450815|3622|2|126| +2450815|3625|2|713| +2450815|3626|2|955| +2450815|3628|2|383| +2450815|3631|2|730| +2450815|3632|2|291| +2450815|3634|2|770| +2450815|3637|2|762| +2450815|3638|2|732| +2450815|3640|2|417| +2450815|3643|2|914| +2450815|3644|2|164| +2450815|3646|2|263| +2450815|3649|2|409| +2450815|3650|2|724| +2450815|3652|2|657| +2450815|3655|2|909| +2450815|3656|2|989| +2450815|3658|2|225| +2450815|3661|2|81| +2450815|3662|2|761| +2450815|3664|2|799| +2450815|3667|2|| +2450815|3668|2|608| +2450815|3670|2|867| +2450815|3673|2|75| +2450815|3674|2|665| +2450815|3676|2|750| +2450815|3679|2|727| +2450815|3680|2|512| +2450815|3682|2|913| +2450815|3685|2|258| +2450815|3686|2|888| +2450815|3688|2|887| +2450815|3691|2|402| +2450815|3692|2|109| +2450815|3694|2|150| +2450815|3697|2|537| +2450815|3698|2|149| +2450815|3700|2|| +2450815|3703|2|498| +2450815|3704|2|794| +2450815|3706|2|290| +2450815|3709|2|988| +2450815|3710|2|940| +2450815|3712|2|353| +2450815|3715|2|| +2450815|3716|2|798| +2450815|3718|2|885| +2450815|3721|2|223| +2450815|3722|2|391| +2450815|3724|2|477| +2450815|3727|2|154| +2450815|3728|2|764| +2450815|3730|2|525| +2450815|3733|2|698| +2450815|3734|2|695| +2450815|3736|2|958| +2450815|3739|2|967| +2450815|3740|2|117| +2450815|3742|2|442| +2450815|3745|2|488| +2450815|3746|2|192| +2450815|3748|2|| +2450815|3751|2|885| +2450815|3752|2|797| +2450815|3754|2|948| +2450815|3757|2|503| +2450815|3758|2|14| +2450815|3760|2|| +2450815|3763|2|179| +2450815|3764|2|665| +2450815|3766|2|137| +2450815|3769|2|325| +2450815|3770|2|699| +2450815|3772|2|908| +2450815|3775|2|887| +2450815|3776|2|116| +2450815|3778|2|| +2450815|3781|2|640| +2450815|3782|2|493| +2450815|3784|2|146| +2450815|3787|2|863| +2450815|3788|2|467| +2450815|3790|2|299| +2450815|3793|2|915| +2450815|3794|2|310| +2450815|3796|2|267| +2450815|3799|2|952| +2450815|3800|2|281| +2450815|3802|2|133| +2450815|3805|2|171| +2450815|3806|2|111| +2450815|3808|2|760| +2450815|3811|2|214| +2450815|3812|2|483| +2450815|3814|2|96| +2450815|3817|2|945| +2450815|3818|2|242| +2450815|3820|2|903| +2450815|3823|2|516| +2450815|3824|2|798| +2450815|3826|2|831| +2450815|3829|2|216| +2450815|3830|2|| +2450815|3832|2|513| +2450815|3835|2|217| +2450815|3836|2|728| +2450815|3838|2|771| +2450815|3841|2|681| +2450815|3842|2|770| +2450815|3844|2|809| +2450815|3847|2|846| +2450815|3848|2|719| +2450815|3850|2|883| +2450815|3853|2|926| +2450815|3854|2|34| +2450815|3856|2|56| +2450815|3859|2|939| +2450815|3860|2|424| +2450815|3862|2|178| +2450815|3865|2|856| +2450815|3866|2|492| +2450815|3868|2|849| +2450815|3871|2|412| +2450815|3872|2|916| +2450815|3874|2|46| +2450815|3877|2|181| +2450815|3878|2|644| +2450815|3880|2|687| +2450815|3883|2|472| +2450815|3884|2|616| +2450815|3886|2|303| +2450815|3889|2|231| +2450815|3890|2|637| +2450815|3892|2|80| +2450815|3895|2|201| +2450815|3896|2|630| +2450815|3898|2|394| +2450815|3901|2|| +2450815|3902|2|695| +2450815|3904|2|25| +2450815|3907|2|603| +2450815|3908|2|818| +2450815|3910|2|671| +2450815|3913|2|949| +2450815|3914|2|787| +2450815|3916|2|244| +2450815|3919|2|133| +2450815|3920|2|715| +2450815|3922|2|| +2450815|3925|2|291| +2450815|3926|2|| +2450815|3928|2|91| +2450815|3931|2|377| +2450815|3932|2|247| +2450815|3934|2|803| +2450815|3937|2|253| +2450815|3938|2|738| +2450815|3940|2|227| +2450815|3943|2|438| +2450815|3944|2|222| +2450815|3946|2|99| +2450815|3949|2|12| +2450815|3950|2|503| +2450815|3952|2|29| +2450815|3955|2|251| +2450815|3956|2|367| +2450815|3958|2|518| +2450815|3961|2|709| +2450815|3962|2|547| +2450815|3964|2|41| +2450815|3967|2|550| +2450815|3968|2|702| +2450815|3970|2|784| +2450815|3973|2|811| +2450815|3974|2|597| +2450815|3976|2|998| +2450815|3979|2|887| +2450815|3980|2|278| +2450815|3982|2|707| +2450815|3985|2|750| +2450815|3986|2|344| +2450815|3988|2|392| +2450815|3991|2|925| +2450815|3992|2|685| +2450815|3994|2|952| +2450815|3997|2|9| +2450815|3998|2|330| +2450815|4000|2|469| +2450815|4003|2|147| +2450815|4004|2|50| +2450815|4006|2|783| +2450815|4009|2|880| +2450815|4010|2|43| +2450815|4012|2|549| +2450815|4015|2|216| +2450815|4016|2|| +2450815|4018|2|917| +2450815|4021|2|895| +2450815|4022|2|642| +2450815|4024|2|664| +2450815|4027|2|436| +2450815|4028|2|348| +2450815|4030|2|78| +2450815|4033|2|474| +2450815|4034|2|31| +2450815|4036|2|797| +2450815|4039|2|608| +2450815|4040|2|415| +2450815|4042|2|242| +2450815|4045|2|189| +2450815|4046|2|670| +2450815|4048|2|11| +2450815|4051|2|605| +2450815|4052|2|149| +2450815|4054|2|852| +2450815|4057|2|133| +2450815|4058|2|360| +2450815|4060|2|403| +2450815|4063|2|362| +2450815|4064|2|| +2450815|4066|2|421| +2450815|4069|2|| +2450815|4070|2|328| +2450815|4072|2|325| +2450815|4075|2|513| +2450815|4076|2|64| +2450815|4078|2|807| +2450815|4081|2|216| +2450815|4082|2|128| +2450815|4084|2|553| +2450815|4087|2|311| +2450815|4088|2|592| +2450815|4090|2|221| +2450815|4093|2|95| +2450815|4094|2|538| +2450815|4096|2|732| +2450815|4099|2|702| +2450815|4100|2|316| +2450815|4102|2|778| +2450815|4105|2|221| +2450815|4106|2|34| +2450815|4108|2|328| +2450815|4111|2|698| +2450815|4112|2|| +2450815|4114|2|936| +2450815|4117|2|135| +2450815|4118|2|753| +2450815|4120|2|839| +2450815|4123|2|33| +2450815|4124|2|653| +2450815|4126|2|984| +2450815|4129|2|399| +2450815|4130|2|535| +2450815|4132|2|356| +2450815|4135|2|385| +2450815|4136|2|329| +2450815|4138|2|450| +2450815|4141|2|838| +2450815|4142|2|943| +2450815|4144|2|732| +2450815|4147|2|642| +2450815|4148|2|136| +2450815|4150|2|714| +2450815|4153|2|166| +2450815|4154|2|352| +2450815|4156|2|376| +2450815|4159|2|151| +2450815|4160|2|607| +2450815|4162|2|| +2450815|4165|2|295| +2450815|4166|2|339| +2450815|4168|2|638| +2450815|4171|2|351| +2450815|4172|2|682| +2450815|4174|2|527| +2450815|4177|2|454| +2450815|4178|2|93| +2450815|4180|2|580| +2450815|4183|2|458| +2450815|4184|2|452| +2450815|4186|2|256| +2450815|4189|2|562| +2450815|4190|2|14| +2450815|4192|2|452| +2450815|4195|2|825| +2450815|4196|2|574| +2450815|4198|2|| +2450815|4201|2|303| +2450815|4202|2|128| +2450815|4204|2|885| +2450815|4207|2|677| +2450815|4208|2|764| +2450815|4210|2|286| +2450815|4213|2|116| +2450815|4214|2|11| +2450815|4216|2|130| +2450815|4219|2|324| +2450815|4220|2|595| +2450815|4222|2|546| +2450815|4225|2|370| +2450815|4226|2|649| +2450815|4228|2|28| +2450815|4231|2|478| +2450815|4232|2|728| +2450815|4234|2|13| +2450815|4237|2|13| +2450815|4238|2|114| +2450815|4240|2|974| +2450815|4243|2|652| +2450815|4244|2|749| +2450815|4246|2|673| +2450815|4249|2|332| +2450815|4250|2|621| +2450815|4252|2|859| +2450815|4255|2|260| +2450815|4256|2|736| +2450815|4258|2|822| +2450815|4261|2|391| +2450815|4262|2|936| +2450815|4264|2|551| +2450815|4267|2|190| +2450815|4268|2|987| +2450815|4270|2|210| +2450815|4273|2|501| +2450815|4274|2|862| +2450815|4276|2|325| +2450815|4279|2|623| +2450815|4280|2|180| +2450815|4282|2|116| +2450815|4285|2|443| +2450815|4286|2|894| +2450815|4288|2|324| +2450815|4291|2|417| +2450815|4292|2|900| +2450815|4294|2|933| +2450815|4297|2|460| +2450815|4298|2|632| +2450815|4300|2|342| +2450815|4303|2|415| +2450815|4304|2|991| +2450815|4306|2|759| +2450815|4309|2|892| +2450815|4310|2|851| +2450815|4312|2|732| +2450815|4315|2|122| +2450815|4316|2|609| +2450815|4318|2|102| +2450815|4321|2|872| +2450815|4322|2|130| +2450815|4324|2|975| +2450815|4327|2|371| +2450815|4328|2|941| +2450815|4330|2|932| +2450815|4333|2|527| +2450815|4334|2|109| +2450815|4336|2|24| +2450815|4339|2|433| +2450815|4340|2|19| +2450815|4342|2|348| +2450815|4345|2|824| +2450815|4346|2|884| +2450815|4348|2|254| +2450815|4351|2|594| +2450815|4352|2|187| +2450815|4354|2|87| +2450815|4357|2|575| +2450815|4358|2|225| +2450815|4360|2|881| +2450815|4363|2|| +2450815|4364|2|525| +2450815|4366|2|| +2450815|4369|2|301| +2450815|4370|2|671| +2450815|4372|2|864| +2450815|4375|2|4| +2450815|4376|2|917| +2450815|4378|2|416| +2450815|4381|2|956| +2450815|4382|2|412| +2450815|4384|2|410| +2450815|4387|2|606| +2450815|4388|2|296| +2450815|4390|2|902| +2450815|4393|2|149| +2450815|4394|2|516| +2450815|4396|2|860| +2450815|4399|2|340| +2450815|4400|2|556| +2450815|4402|2|198| +2450815|4405|2|428| +2450815|4406|2|502| +2450815|4408|2|345| +2450815|4411|2|36| +2450815|4412|2|793| +2450815|4414|2|491| +2450815|4417|2|55| +2450815|4418|2|438| +2450815|4420|2|106| +2450815|4423|2|809| +2450815|4424|2|983| +2450815|4426|2|545| +2450815|4429|2|955| +2450815|4430|2|931| +2450815|4432|2|51| +2450815|4435|2|959| +2450815|4436|2|487| +2450815|4438|2|157| +2450815|4441|2|720| +2450815|4442|2|448| +2450815|4444|2|| +2450815|4447|2|503| +2450815|4448|2|794| +2450815|4450|2|982| +2450815|4453|2|640| +2450815|4454|2|410| +2450815|4456|2|658| +2450815|4459|2|721| +2450815|4460|2|288| +2450815|4462|2|586| +2450815|4465|2|842| +2450815|4466|2|739| +2450815|4468|2|414| +2450815|4471|2|473| +2450815|4472|2|557| +2450815|4474|2|50| +2450815|4477|2|59| +2450815|4478|2|661| +2450815|4480|2|260| +2450815|4483|2|919| +2450815|4484|2|481| +2450815|4486|2|992| +2450815|4489|2|211| +2450815|4490|2|8| +2450815|4492|2|641| +2450815|4495|2|674| +2450815|4496|2|513| +2450815|4498|2|125| +2450815|4501|2|205| +2450815|4502|2|868| +2450815|4504|2|814| +2450815|4507|2|651| +2450815|4508|2|258| +2450815|4510|2|746| +2450815|4513|2|326| +2450815|4514|2|669| +2450815|4516|2|16| +2450815|4519|2|139| +2450815|4520|2|| +2450815|4522|2|945| +2450815|4525|2|390| +2450815|4526|2|847| +2450815|4528|2|872| +2450815|4531|2|808| +2450815|4532|2|790| +2450815|4534|2|44| +2450815|4537|2|949| +2450815|4538|2|542| +2450815|4540|2|183| +2450815|4543|2|209| +2450815|4544|2|331| +2450815|4546|2|447| +2450815|4549|2|687| +2450815|4550|2|175| +2450815|4552|2|586| +2450815|4555|2|644| +2450815|4556|2|718| +2450815|4558|2|| +2450815|4561|2|365| +2450815|4562|2|581| +2450815|4564|2|428| +2450815|4567|2|51| +2450815|4568|2|480| +2450815|4570|2|564| +2450815|4573|2|354| +2450815|4574|2|698| +2450815|4576|2|| +2450815|4579|2|339| +2450815|4580|2|565| +2450815|4582|2|441| +2450815|4585|2|920| +2450815|4586|2|934| +2450815|4588|2|914| +2450815|4591|2|691| +2450815|4592|2|| +2450815|4594|2|441| +2450815|4597|2|912| +2450815|4598|2|933| +2450815|4600|2|690| +2450815|4603|2|946| +2450815|4604|2|476| +2450815|4606|2|443| +2450815|4609|2|282| +2450815|4610|2|135| +2450815|4612|2|| +2450815|4615|2|| +2450815|4616|2|892| +2450815|4618|2|224| +2450815|4621|2|331| +2450815|4622|2|560| +2450815|4624|2|894| +2450815|4627|2|| +2450815|4628|2|778| +2450815|4630|2|275| +2450815|4633|2|90| +2450815|4634|2|626| +2450815|4636|2|378| +2450815|4639|2|526| +2450815|4640|2|161| +2450815|4642|2|548| +2450815|4645|2|308| +2450815|4646|2|868| +2450815|4648|2|795| +2450815|4651|2|770| +2450815|4652|2|117| +2450815|4654|2|979| +2450815|4657|2|89| +2450815|4658|2|95| +2450815|4660|2|945| +2450815|4663|2|2| +2450815|4664|2|393| +2450815|4666|2|489| +2450815|4669|2|218| +2450815|4670|2|320| +2450815|4672|2|827| +2450815|4675|2|993| +2450815|4676|2|796| +2450815|4678|2|692| +2450815|4681|2|654| +2450815|4682|2|552| +2450815|4684|2|464| +2450815|4687|2|774| +2450815|4688|2|904| +2450815|4690|2|635| +2450815|4693|2|227| +2450815|4694|2|389| +2450815|4696|2|709| +2450815|4699|2|316| +2450815|4700|2|621| +2450815|4702|2|270| +2450815|4705|2|914| +2450815|4706|2|305| +2450815|4708|2|647| +2450815|4711|2|503| +2450815|4712|2|895| +2450815|4714|2|630| +2450815|4717|2|360| +2450815|4718|2|865| +2450815|4720|2|378| +2450815|4723|2|892| +2450815|4724|2|928| +2450815|4726|2|806| +2450815|4729|2|62| +2450815|4730|2|| +2450815|4732|2|688| +2450815|4735|2|753| +2450815|4736|2|334| +2450815|4738|2|462| +2450815|4741|2|26| +2450815|4742|2|407| +2450815|4744|2|207| +2450815|4747|2|68| +2450815|4748|2|915| +2450815|4750|2|721| +2450815|4753|2|334| +2450815|4754|2|960| +2450815|4756|2|469| +2450815|4759|2|183| +2450815|4760|2|528| +2450815|4762|2|180| +2450815|4765|2|94| +2450815|4766|2|480| +2450815|4768|2|975| +2450815|4771|2|768| +2450815|4772|2|431| +2450815|4774|2|68| +2450815|4777|2|36| +2450815|4778|2|114| +2450815|4780|2|600| +2450815|4783|2|464| +2450815|4784|2|747| +2450815|4786|2|301| +2450815|4789|2|576| +2450815|4790|2|188| +2450815|4792|2|947| +2450815|4795|2|931| +2450815|4796|2|542| +2450815|4798|2|761| +2450815|4801|2|258| +2450815|4802|2|617| +2450815|4804|2|| +2450815|4807|2|974| +2450815|4808|2|324| +2450815|4810|2|688| +2450815|4813|2|202| +2450815|4814|2|431| +2450815|4816|2|88| +2450815|4819|2|229| +2450815|4820|2|389| +2450815|4822|2|752| +2450815|4825|2|413| +2450815|4826|2|| +2450815|4828|2|548| +2450815|4831|2|174| +2450815|4832|2|682| +2450815|4834|2|457| +2450815|4837|2|768| +2450815|4838|2|779| +2450815|4840|2|814| +2450815|4843|2|7| +2450815|4844|2|104| +2450815|4846|2|569| +2450815|4849|2|16| +2450815|4850|2|962| +2450815|4852|2|145| +2450815|4855|2|478| +2450815|4856|2|| +2450815|4858|2|912| +2450815|4861|2|598| +2450815|4862|2|23| +2450815|4864|2|991| +2450815|4867|2|754| +2450815|4868|2|931| +2450815|4870|2|484| +2450815|4873|2|25| +2450815|4874|2|644| +2450815|4876|2|574| +2450815|4879|2|930| +2450815|4880|2|| +2450815|4882|2|121| +2450815|4885|2|530| +2450815|4886|2|225| +2450815|4888|2|389| +2450815|4891|2|| +2450815|4892|2|412| +2450815|4894|2|551| +2450815|4897|2|999| +2450815|4898|2|396| +2450815|4900|2|721| +2450815|4903|2|57| +2450815|4904|2|393| +2450815|4906|2|533| +2450815|4909|2|| +2450815|4910|2|173| +2450815|4912|2|| +2450815|4915|2|848| +2450815|4916|2|539| +2450815|4918|2|596| +2450815|4921|2|681| +2450815|4922|2|551| +2450815|4924|2|81| +2450815|4927|2|466| +2450815|4928|2|254| +2450815|4930|2|385| +2450815|4933|2|388| +2450815|4934|2|| +2450815|4936|2|| +2450815|4939|2|185| +2450815|4940|2|53| +2450815|4942|2|696| +2450815|4945|2|649| +2450815|4946|2|| +2450815|4948|2|260| +2450815|4951|2|516| +2450815|4952|2|827| +2450815|4954|2|600| +2450815|4957|2|209| +2450815|4958|2|180| +2450815|4960|2|365| +2450815|4963|2|960| +2450815|4964|2|981| +2450815|4966|2|383| +2450815|4969|2|773| +2450815|4970|2|427| +2450815|4972|2|635| +2450815|4975|2|653| +2450815|4976|2|513| +2450815|4978|2|917| +2450815|4981|2|114| +2450815|4982|2|111| +2450815|4984|2|44| +2450815|4987|2|789| +2450815|4988|2|954| +2450815|4990|2|490| +2450815|4993|2|509| +2450815|4994|2|625| +2450815|4996|2|380| +2450815|4999|2|290| +2450815|5000|2|990| +2450815|5002|2|520| +2450815|5005|2|725| +2450815|5006|2|615| +2450815|5008|2|731| +2450815|5011|2|536| +2450815|5012|2|667| +2450815|5014|2|541| +2450815|5017|2|619| +2450815|5018|2|665| +2450815|5020|2|524| +2450815|5023|2|349| +2450815|5024|2|743| +2450815|5026|2|176| +2450815|5029|2|735| +2450815|5030|2|427| +2450815|5032|2|69| +2450815|5035|2|910| +2450815|5036|2|| +2450815|5038|2|795| +2450815|5041|2|284| +2450815|5042|2|286| +2450815|5044|2|411| +2450815|5047|2|491| +2450815|5048|2|788| +2450815|5050|2|299| +2450815|5053|2|43| +2450815|5054|2|925| +2450815|5056|2|461| +2450815|5059|2|258| +2450815|5060|2|358| +2450815|5062|2|215| +2450815|5065|2|789| +2450815|5066|2|395| +2450815|5068|2|587| +2450815|5071|2|567| +2450815|5072|2|507| +2450815|5074|2|76| +2450815|5077|2|635| +2450815|5078|2|| +2450815|5080|2|246| +2450815|5083|2|771| +2450815|5084|2|| +2450815|5086|2|698| +2450815|5089|2|301| +2450815|5090|2|48| +2450815|5092|2|856| +2450815|5095|2|514| +2450815|5096|2|342| +2450815|5098|2|343| +2450815|5101|2|409| +2450815|5102|2|| +2450815|5104|2|940| +2450815|5107|2|496| +2450815|5108|2|145| +2450815|5110|2|209| +2450815|5113|2|246| +2450815|5114|2|944| +2450815|5116|2|| +2450815|5119|2|| +2450815|5120|2|497| +2450815|5122|2|336| +2450815|5125|2|730| +2450815|5126|2|891| +2450815|5128|2|441| +2450815|5131|2|83| +2450815|5132|2|76| +2450815|5134|2|620| +2450815|5137|2|811| +2450815|5138|2|759| +2450815|5140|2|655| +2450815|5143|2|876| +2450815|5144|2|148| +2450815|5146|2|12| +2450815|5149|2|434| +2450815|5150|2|735| +2450815|5152|2|548| +2450815|5155|2|480| +2450815|5156|2|707| +2450815|5158|2|842| +2450815|5161|2|545| +2450815|5162|2|244| +2450815|5164|2|267| +2450815|5167|2|382| +2450815|5168|2|109| +2450815|5170|2|723| +2450815|5173|2|577| +2450815|5174|2|409| +2450815|5176|2|280| +2450815|5179|2|107| +2450815|5180|2|397| +2450815|5182|2|346| +2450815|5185|2|284| +2450815|5186|2|759| +2450815|5188|2|130| +2450815|5191|2|125| +2450815|5192|2|413| +2450815|5194|2|578| +2450815|5197|2|832| +2450815|5198|2|826| +2450815|5200|2|529| +2450815|5203|2|362| +2450815|5204|2|643| +2450815|5206|2|499| +2450815|5209|2|808| +2450815|5210|2|851| +2450815|5212|2|| +2450815|5215|2|485| +2450815|5216|2|380| +2450815|5218|2|857| +2450815|5221|2|879| +2450815|5222|2|716| +2450815|5224|2|128| +2450815|5227|2|962| +2450815|5228|2|124| +2450815|5230|2|| +2450815|5233|2|421| +2450815|5234|2|630| +2450815|5236|2|820| +2450815|5239|2|346| +2450815|5240|2|339| +2450815|5242|2|| +2450815|5245|2|245| +2450815|5246|2|664| +2450815|5248|2|114| +2450815|5251|2|508| +2450815|5252|2|404| +2450815|5254|2|749| +2450815|5257|2|313| +2450815|5258|2|846| +2450815|5260|2|931| +2450815|5263|2|247| +2450815|5264|2|507| +2450815|5266|2|565| +2450815|5269|2|| +2450815|5270|2|348| +2450815|5272|2|558| +2450815|5275|2|295| +2450815|5276|2|69| +2450815|5278|2|862| +2450815|5281|2|305| +2450815|5282|2|871| +2450815|5284|2|39| +2450815|5287|2|242| +2450815|5288|2|205| +2450815|5290|2|893| +2450815|5293|2|45| +2450815|5294|2|337| +2450815|5296|2|670| +2450815|5299|2|62| +2450815|5300|2|786| +2450815|5302|2|89| +2450815|5305|2|691| +2450815|5306|2|942| +2450815|5308|2|291| +2450815|5311|2|| +2450815|5312|2|7| +2450815|5314|2|664| +2450815|5317|2|465| +2450815|5318|2|34| +2450815|5320|2|554| +2450815|5323|2|178| +2450815|5324|2|124| +2450815|5326|2|896| +2450815|5329|2|325| +2450815|5330|2|638| +2450815|5332|2|850| +2450815|5335|2|873| +2450815|5336|2|71| +2450815|5338|2|910| +2450815|5341|2|785| +2450815|5342|2|880| +2450815|5344|2|661| +2450815|5347|2|380| +2450815|5348|2|170| +2450815|5350|2|933| +2450815|5353|2|938| +2450815|5354|2|280| +2450815|5356|2|45| +2450815|5359|2|977| +2450815|5360|2|100| +2450815|5362|2|62| +2450815|5365|2|690| +2450815|5366|2|372| +2450815|5368|2|163| +2450815|5371|2|15| +2450815|5372|2|38| +2450815|5374|2|962| +2450815|5377|2|164| +2450815|5378|2|36| +2450815|5380|2|873| +2450815|5383|2|34| +2450815|5384|2|501| +2450815|5386|2|831| +2450815|5389|2|304| +2450815|5390|2|579| +2450815|5392|2|826| +2450815|5395|2|405| +2450815|5396|2|189| +2450815|5398|2|833| +2450815|5401|2|593| +2450815|5402|2|753| +2450815|5404|2|372| +2450815|5407|2|741| +2450815|5408|2|846| +2450815|5410|2|585| +2450815|5413|2|602| +2450815|5414|2|891| +2450815|5416|2|36| +2450815|5419|2|963| +2450815|5420|2|879| +2450815|5422|2|656| +2450815|5425|2|7| +2450815|5426|2|286| +2450815|5428|2|36| +2450815|5431|2|902| +2450815|5432|2|896| +2450815|5434|2|976| +2450815|5437|2|635| +2450815|5438|2|170| +2450815|5440|2|855| +2450815|5443|2|523| +2450815|5444|2|| +2450815|5446|2|990| +2450815|5449|2|359| +2450815|5450|2|109| +2450815|5452|2|424| +2450815|5455|2|845| +2450815|5456|2|489| +2450815|5458|2|843| +2450815|5461|2|716| +2450815|5462|2|434| +2450815|5464|2|| +2450815|5467|2|777| +2450815|5468|2|870| +2450815|5470|2|889| +2450815|5473|2|710| +2450815|5474|2|767| +2450815|5476|2|771| +2450815|5479|2|60| +2450815|5480|2|92| +2450815|5482|2|786| +2450815|5485|2|835| +2450815|5486|2|33| +2450815|5488|2|319| +2450815|5491|2|629| +2450815|5492|2|711| +2450815|5494|2|498| +2450815|5497|2|647| +2450815|5498|2|704| +2450815|5500|2|486| +2450815|5503|2|757| +2450815|5504|2|217| +2450815|5506|2|168| +2450815|5509|2|277| +2450815|5510|2|529| +2450815|5512|2|741| +2450815|5515|2|544| +2450815|5516|2|601| +2450815|5518|2|260| +2450815|5521|2|335| +2450815|5522|2|824| +2450815|5524|2|78| +2450815|5527|2|453| +2450815|5528|2|493| +2450815|5530|2|214| +2450815|5533|2|765| +2450815|5534|2|717| +2450815|5536|2|357| +2450815|5539|2|199| +2450815|5540|2|932| +2450815|5542|2|458| +2450815|5545|2|963| +2450815|5546|2|| +2450815|5548|2|353| +2450815|5551|2|139| +2450815|5552|2|981| +2450815|5554|2|351| +2450815|5557|2|604| +2450815|5558|2|| +2450815|5560|2|| +2450815|5563|2|526| +2450815|5564|2|441| +2450815|5566|2|865| +2450815|5569|2|682| +2450815|5570|2|651| +2450815|5572|2|149| +2450815|5575|2|283| +2450815|5576|2|849| +2450815|5578|2|11| +2450815|5581|2|407| +2450815|5582|2|852| +2450815|5584|2|867| +2450815|5587|2|50| +2450815|5588|2|870| +2450815|5590|2|662| +2450815|5593|2|193| +2450815|5594|2|910| +2450815|5596|2|855| +2450815|5599|2|572| +2450815|5600|2|148| +2450815|5602|2|513| +2450815|5605|2|413| +2450815|5606|2|616| +2450815|5608|2|879| +2450815|5611|2|487| +2450815|5612|2|266| +2450815|5614|2|282| +2450815|5617|2|| +2450815|5618|2|137| +2450815|5620|2|383| +2450815|5623|2|547| +2450815|5624|2|416| +2450815|5626|2|876| +2450815|5629|2|499| +2450815|5630|2|558| +2450815|5632|2|265| +2450815|5635|2|797| +2450815|5636|2|586| +2450815|5638|2|85| +2450815|5641|2|739| +2450815|5642|2|906| +2450815|5644|2|934| +2450815|5647|2|762| +2450815|5648|2|334| +2450815|5650|2|618| +2450815|5653|2|| +2450815|5654|2|66| +2450815|5656|2|302| +2450815|5659|2|325| +2450815|5660|2|275| +2450815|5662|2|724| +2450815|5665|2|| +2450815|5666|2|621| +2450815|5668|2|579| +2450815|5671|2|242| +2450815|5672|2|455| +2450815|5674|2|796| +2450815|5677|2|999| +2450815|5678|2|281| +2450815|5680|2|256| +2450815|5683|2|681| +2450815|5684|2|173| +2450815|5686|2|322| +2450815|5689|2|819| +2450815|5690|2|991| +2450815|5692|2|739| +2450815|5695|2|| +2450815|5696|2|841| +2450815|5698|2|47| +2450815|5701|2|385| +2450815|5702|2|325| +2450815|5704|2|615| +2450815|5707|2|970| +2450815|5708|2|894| +2450815|5710|2|478| +2450815|5713|2|723| +2450815|5714|2|302| +2450815|5716|2|698| +2450815|5719|2|447| +2450815|5720|2|410| +2450815|5722|2|644| +2450815|5725|2|393| +2450815|5726|2|355| +2450815|5728|2|472| +2450815|5731|2|238| +2450815|5732|2|977| +2450815|5734|2|985| +2450815|5737|2|736| +2450815|5738|2|980| +2450815|5740|2|515| +2450815|5743|2|106| +2450815|5744|2|489| +2450815|5746|2|564| +2450815|5749|2|585| +2450815|5750|2|215| +2450815|5752|2|332| +2450815|5755|2|576| +2450815|5756|2|981| +2450815|5758|2|639| +2450815|5761|2|506| +2450815|5762|2|| +2450815|5764|2|411| +2450815|5767|2|567| +2450815|5768|2|92| +2450815|5770|2|355| +2450815|5773|2|782| +2450815|5774|2|22| +2450815|5776|2|804| +2450815|5779|2|798| +2450815|5780|2|722| +2450815|5782|2|601| +2450815|5785|2|455| +2450815|5786|2|714| +2450815|5788|2|63| +2450815|5791|2|374| +2450815|5792|2|17| +2450815|5794|2|654| +2450815|5797|2|151| +2450815|5798|2|816| +2450815|5800|2|152| +2450815|5803|2|533| +2450815|5804|2|458| +2450815|5806|2|147| +2450815|5809|2|201| +2450815|5810|2|342| +2450815|5812|2|604| +2450815|5815|2|743| +2450815|5816|2|652| +2450815|5818|2|290| +2450815|5821|2|11| +2450815|5822|2|624| +2450815|5824|2|753| +2450815|5827|2|919| +2450815|5828|2|583| +2450815|5830|2|914| +2450815|5833|2|446| +2450815|5834|2|970| +2450815|5836|2|312| +2450815|5839|2|592| +2450815|5840|2|418| +2450815|5842|2|991| +2450815|5845|2|| +2450815|5846|2|264| +2450815|5848|2|53| +2450815|5851|2|209| +2450815|5852|2|355| +2450815|5854|2|758| +2450815|5857|2|196| +2450815|5858|2|640| +2450815|5860|2|858| +2450815|5863|2|993| +2450815|5864|2|536| +2450815|5866|2|520| +2450815|5869|2|364| +2450815|5870|2|758| +2450815|5872|2|155| +2450815|5875|2|617| +2450815|5876|2|858| +2450815|5878|2|432| +2450815|5881|2|914| +2450815|5882|2|557| +2450815|5884|2|934| +2450815|5887|2|206| +2450815|5888|2|726| +2450815|5890|2|20| +2450815|5893|2|137| +2450815|5894|2|972| +2450815|5896|2|26| +2450815|5899|2|526| +2450815|5900|2|263| +2450815|5902|2|715| +2450815|5905|2|567| +2450815|5906|2|523| +2450815|5908|2|486| +2450815|5911|2|811| +2450815|5912|2|751| +2450815|5914|2|319| +2450815|5917|2|649| +2450815|5918|2|400| +2450815|5920|2|293| +2450815|5923|2|594| +2450815|5924|2|861| +2450815|5926|2|465| +2450815|5929|2|756| +2450815|5930|2|72| +2450815|5932|2|879| +2450815|5935|2|379| +2450815|5936|2|535| +2450815|5938|2|351| +2450815|5941|2|| +2450815|5942|2|741| +2450815|5944|2|83| +2450815|5947|2|770| +2450815|5948|2|997| +2450815|5950|2|329| +2450815|5953|2|412| +2450815|5954|2|986| +2450815|5956|2|| +2450815|5959|2|845| +2450815|5960|2|335| +2450815|5962|2|661| +2450815|5965|2|795| +2450815|5966|2|328| +2450815|5968|2|952| +2450815|5971|2|14| +2450815|5972|2|294| +2450815|5974|2|845| +2450815|5977|2|600| +2450815|5978|2|| +2450815|5980|2|220| +2450815|5983|2|504| +2450815|5984|2|484| +2450815|5986|2|570| +2450815|5989|2|975| +2450815|5990|2|879| +2450815|5992|2|65| +2450815|5995|2|92| +2450815|5996|2|163| +2450815|5998|2|451| +2450815|6001|2|245| +2450815|6002|2|163| +2450815|6004|2|257| +2450815|6007|2|798| +2450815|6008|2|884| +2450815|6010|2|765| +2450815|6013|2|173| +2450815|6014|2|16| +2450815|6016|2|216| +2450815|6019|2|917| +2450815|6020|2|| +2450815|6022|2|155| +2450815|6025|2|504| +2450815|6026|2|343| +2450815|6028|2|137| +2450815|6031|2|104| +2450815|6032|2|208| +2450815|6034|2|536| +2450815|6037|2|727| +2450815|6038|2|852| +2450815|6040|2|727| +2450815|6043|2|569| +2450815|6044|2|765| +2450815|6046|2|459| +2450815|6049|2|297| +2450815|6050|2|92| +2450815|6052|2|85| +2450815|6055|2|643| +2450815|6056|2|750| +2450815|6058|2|966| +2450815|6061|2|716| +2450815|6062|2|526| +2450815|6064|2|384| +2450815|6067|2|104| +2450815|6068|2|532| +2450815|6070|2|830| +2450815|6073|2|458| +2450815|6074|2|100| +2450815|6076|2|983| +2450815|6079|2|139| +2450815|6080|2|307| +2450815|6082|2|535| +2450815|6085|2|663| +2450815|6086|2|| +2450815|6088|2|517| +2450815|6091|2|960| +2450815|6092|2|270| +2450815|6094|2|859| +2450815|6097|2|140| +2450815|6098|2|45| +2450815|6100|2|558| +2450815|6103|2|838| +2450815|6104|2|935| +2450815|6106|2|606| +2450815|6109|2|254| +2450815|6110|2|349| +2450815|6112|2|506| +2450815|6115|2|869| +2450815|6116|2|565| +2450815|6118|2|123| +2450815|6121|2|964| +2450815|6122|2|115| +2450815|6124|2|657| +2450815|6127|2|555| +2450815|6128|2|360| +2450815|6130|2|395| +2450815|6133|2|501| +2450815|6134|2|971| +2450815|6136|2|257| +2450815|6139|2|370| +2450815|6140|2|560| +2450815|6142|2|679| +2450815|6145|2|627| +2450815|6146|2|30| +2450815|6148|2|890| +2450815|6151|2|390| +2450815|6152|2|208| +2450815|6154|2|178| +2450815|6157|2|58| +2450815|6158|2|778| +2450815|6160|2|471| +2450815|6163|2|318| +2450815|6164|2|946| +2450815|6166|2|438| +2450815|6169|2|710| +2450815|6170|2|14| +2450815|6172|2|262| +2450815|6175|2|677| +2450815|6176|2|478| +2450815|6178|2|827| +2450815|6181|2|762| +2450815|6182|2|568| +2450815|6184|2|822| +2450815|6187|2|777| +2450815|6188|2|21| +2450815|6190|2|468| +2450815|6193|2|159| +2450815|6194|2|454| +2450815|6196|2|780| +2450815|6199|2|345| +2450815|6200|2|224| +2450815|6202|2|92| +2450815|6205|2|903| +2450815|6206|2|370| +2450815|6208|2|352| +2450815|6211|2|516| +2450815|6212|2|800| +2450815|6214|2|53| +2450815|6217|2|9| +2450815|6218|2|442| +2450815|6220|2|940| +2450815|6223|2|876| +2450815|6224|2|594| +2450815|6226|2|12| +2450815|6229|2|302| +2450815|6230|2|537| +2450815|6232|2|283| +2450815|6235|2|| +2450815|6236|2|59| +2450815|6238|2|33| +2450815|6241|2|440| +2450815|6242|2|880| +2450815|6244|2|921| +2450815|6247|2|997| +2450815|6248|2|418| +2450815|6250|2|51| +2450815|6253|2|766| +2450815|6254|2|195| +2450815|6256|2|34| +2450815|6259|2|144| +2450815|6260|2|586| +2450815|6262|2|| +2450815|6265|2|578| +2450815|6266|2|804| +2450815|6268|2|462| +2450815|6271|2|164| +2450815|6272|2|361| +2450815|6274|2|264| +2450815|6277|2|971| +2450815|6278|2|913| +2450815|6280|2|648| +2450815|6283|2|553| +2450815|6284|2|730| +2450815|6286|2|56| +2450815|6289|2|610| +2450815|6290|2|482| +2450815|6292|2|925| +2450815|6295|2|992| +2450815|6296|2|446| +2450815|6298|2|226| +2450815|6301|2|239| +2450815|6302|2|372| +2450815|6304|2|507| +2450815|6307|2|| +2450815|6308|2|643| +2450815|6310|2|859| +2450815|6313|2|998| +2450815|6314|2|865| +2450815|6316|2|409| +2450815|6319|2|318| +2450815|6320|2|416| +2450815|6322|2|922| +2450815|6325|2|786| +2450815|6326|2|684| +2450815|6328|2|681| +2450815|6331|2|15| +2450815|6332|2|878| +2450815|6334|2|480| +2450815|6337|2|387| +2450815|6338|2|407| +2450815|6340|2|913| +2450815|6343|2|619| +2450815|6344|2|494| +2450815|6346|2|754| +2450815|6349|2|254| +2450815|6350|2|591| +2450815|6352|2|200| +2450815|6355|2|522| +2450815|6356|2|391| +2450815|6358|2|13| +2450815|6361|2|823| +2450815|6362|2|246| +2450815|6364|2|488| +2450815|6367|2|910| +2450815|6368|2|172| +2450815|6370|2|974| +2450815|6373|2|260| +2450815|6374|2|693| +2450815|6376|2|864| +2450815|6379|2|689| +2450815|6380|2|765| +2450815|6382|2|283| +2450815|6385|2|109| +2450815|6386|2|259| +2450815|6388|2|376| +2450815|6391|2|68| +2450815|6392|2|702| +2450815|6394|2|969| +2450815|6397|2|337| +2450815|6398|2|771| +2450815|6400|2|178| +2450815|6403|2|704| +2450815|6404|2|678| +2450815|6406|2|| +2450815|6409|2|227| +2450815|6410|2|604| +2450815|6412|2|690| +2450815|6415|2|327| +2450815|6416|2|2| +2450815|6418|2|681| +2450815|6421|2|457| +2450815|6422|2|663| +2450815|6424|2|775| +2450815|6427|2|384| +2450815|6428|2|694| +2450815|6430|2|463| +2450815|6433|2|340| +2450815|6434|2|32| +2450815|6436|2|976| +2450815|6439|2|668| +2450815|6440|2|8| +2450815|6442|2|531| +2450815|6445|2|| +2450815|6446|2|342| +2450815|6448|2|496| +2450815|6451|2|208| +2450815|6452|2|236| +2450815|6454|2|34| +2450815|6457|2|475| +2450815|6458|2|689| +2450815|6460|2|210| +2450815|6463|2|148| +2450815|6464|2|578| +2450815|6466|2|41| +2450815|6469|2|40| +2450815|6470|2|| +2450815|6472|2|766| +2450815|6475|2|474| +2450815|6476|2|595| +2450815|6478|2|985| +2450815|6481|2|359| +2450815|6482|2|870| +2450815|6484|2|908| +2450815|6487|2|325| +2450815|6488|2|768| +2450815|6490|2|667| +2450815|6493|2|369| +2450815|6494|2|124| +2450815|6496|2|909| +2450815|6499|2|42| +2450815|6500|2|497| +2450815|6502|2|475| +2450815|6505|2|390| +2450815|6506|2|459| +2450815|6508|2|78| +2450815|6511|2|283| +2450815|6512|2|34| +2450815|6514|2|312| +2450815|6517|2|956| +2450815|6518|2|946| +2450815|6520|2|990| +2450815|6523|2|349| +2450815|6524|2|452| +2450815|6526|2|403| +2450815|6529|2|931| +2450815|6530|2|459| +2450815|6532|2|368| +2450815|6535|2|374| +2450815|6536|2|73| +2450815|6538|2|346| +2450815|6541|2|775| +2450815|6542|2|86| +2450815|6544|2|700| +2450815|6547|2|652| +2450815|6548|2|568| +2450815|6550|2|851| +2450815|6553|2|834| +2450815|6554|2|207| +2450815|6556|2|| +2450815|6559|2|103| +2450815|6560|2|778| +2450815|6562|2|503| +2450815|6565|2|504| +2450815|6566|2|461| +2450815|6568|2|49| +2450815|6571|2|810| +2450815|6572|2|432| +2450815|6574|2|744| +2450815|6577|2|953| +2450815|6578|2|684| +2450815|6580|2|694| +2450815|6583|2|112| +2450815|6584|2|564| +2450815|6586|2|954| +2450815|6589|2|290| +2450815|6590|2|824| +2450815|6592|2|656| +2450815|6595|2|405| +2450815|6596|2|983| +2450815|6598|2|613| +2450815|6601|2|128| +2450815|6602|2|976| +2450815|6604|2|792| +2450815|6607|2|940| +2450815|6608|2|484| +2450815|6610|2|996| +2450815|6613|2|390| +2450815|6614|2|193| +2450815|6616|2|866| +2450815|6619|2|405| +2450815|6620|2|415| +2450815|6622|2|986| +2450815|6625|2|143| +2450815|6626|2|419| +2450815|6628|2|167| +2450815|6631|2|914| +2450815|6632|2|478| +2450815|6634|2|715| +2450815|6637|2|565| +2450815|6638|2|846| +2450815|6640|2|291| +2450815|6643|2|114| +2450815|6644|2|381| +2450815|6646|2|601| +2450815|6649|2|627| +2450815|6650|2|908| +2450815|6652|2|840| +2450815|6655|2|| +2450815|6656|2|512| +2450815|6658|2|706| +2450815|6661|2|573| +2450815|6662|2|482| +2450815|6664|2|526| +2450815|6667|2|794| +2450815|6668|2|436| +2450815|6670|2|717| +2450815|6673|2|447| +2450815|6674|2|155| +2450815|6676|2|911| +2450815|6679|2|505| +2450815|6680|2|865| +2450815|6682|2|338| +2450815|6685|2|76| +2450815|6686|2|694| +2450815|6688|2|635| +2450815|6691|2|284| +2450815|6692|2|747| +2450815|6694|2|168| +2450815|6697|2|| +2450815|6698|2|762| +2450815|6700|2|438| +2450815|6703|2|121| +2450815|6704|2|393| +2450815|6706|2|564| +2450815|6709|2|508| +2450815|6710|2|227| +2450815|6712|2|786| +2450815|6715|2|| +2450815|6716|2|498| +2450815|6718|2|190| +2450815|6721|2|197| +2450815|6722|2|102| +2450815|6724|2|7| +2450815|6727|2|638| +2450815|6728|2|717| +2450815|6730|2|934| +2450815|6733|2|574| +2450815|6734|2|855| +2450815|6736|2|733| +2450815|6739|2|299| +2450815|6740|2|| +2450815|6742|2|569| +2450815|6745|2|746| +2450815|6746|2|863| +2450815|6748|2|218| +2450815|6751|2|310| +2450815|6752|2|698| +2450815|6754|2|215| +2450815|6757|2|920| +2450815|6758|2|953| +2450815|6760|2|992| +2450815|6763|2|54| +2450815|6764|2|739| +2450815|6766|2|843| +2450815|6769|2|63| +2450815|6770|2|144| +2450815|6772|2|41| +2450815|6775|2|250| +2450815|6776|2|831| +2450815|6778|2|136| +2450815|6781|2|197| +2450815|6782|2|624| +2450815|6784|2|980| +2450815|6787|2|| +2450815|6788|2|681| +2450815|6790|2|144| +2450815|6793|2|| +2450815|6794|2|798| +2450815|6796|2|390| +2450815|6799|2|232| +2450815|6800|2|732| +2450815|6802|2|26| +2450815|6805|2|3| +2450815|6806|2|325| +2450815|6808|2|856| +2450815|6811|2|199| +2450815|6812|2|465| +2450815|6814|2|168| +2450815|6817|2|574| +2450815|6818|2|979| +2450815|6820|2|504| +2450815|6823|2|74| +2450815|6824|2|23| +2450815|6826|2|151| +2450815|6829|2|7| +2450815|6830|2|730| +2450815|6832|2|379| +2450815|6835|2|423| +2450815|6836|2|373| +2450815|6838|2|853| +2450815|6841|2|336| +2450815|6842|2|471| +2450815|6844|2|262| +2450815|6847|2|188| +2450815|6848|2|104| +2450815|6850|2|55| +2450815|6853|2|129| +2450815|6854|2|554| +2450815|6856|2|733| +2450815|6859|2|890| +2450815|6860|2|681| +2450815|6862|2|416| +2450815|6865|2|200| +2450815|6866|2|316| +2450815|6868|2|14| +2450815|6871|2|627| +2450815|6872|2|523| +2450815|6874|2|721| +2450815|6877|2|405| +2450815|6878|2|327| +2450815|6880|2|378| +2450815|6883|2|581| +2450815|6884|2|| +2450815|6886|2|857| +2450815|6889|2|308| +2450815|6890|2|453| +2450815|6892|2|447| +2450815|6895|2|882| +2450815|6896|2|617| +2450815|6898|2|464| +2450815|6901|2|43| +2450815|6902|2|441| +2450815|6904|2|684| +2450815|6907|2|986| +2450815|6908|2|291| +2450815|6910|2|23| +2450815|6913|2|311| +2450815|6914|2|670| +2450815|6916|2|58| +2450815|6919|2|163| +2450815|6920|2|788| +2450815|6922|2|616| +2450815|6925|2|147| +2450815|6926|2|494| +2450815|6928|2|331| +2450815|6931|2|44| +2450815|6932|2|953| +2450815|6934|2|721| +2450815|6937|2|49| +2450815|6938|2|775| +2450815|6940|2|284| +2450815|6943|2|980| +2450815|6944|2|248| +2450815|6946|2|178| +2450815|6949|2|690| +2450815|6950|2|44| +2450815|6952|2|827| +2450815|6955|2|473| +2450815|6956|2|669| +2450815|6958|2|593| +2450815|6961|2|910| +2450815|6962|2|124| +2450815|6964|2|477| +2450815|6967|2|11| +2450815|6968|2|19| +2450815|6970|2|975| +2450815|6973|2|56| +2450815|6974|2|582| +2450815|6976|2|366| +2450815|6979|2|615| +2450815|6980|2|696| +2450815|6982|2|45| +2450815|6985|2|758| +2450815|6986|2|783| +2450815|6988|2|616| +2450815|6991|2|384| +2450815|6992|2|490| +2450815|6994|2|285| +2450815|6997|2|724| +2450815|6998|2|626| +2450815|7000|2|845| +2450815|7003|2|463| +2450815|7004|2|521| +2450815|7006|2|| +2450815|7009|2|653| +2450815|7010|2|333| +2450815|7012|2|699| +2450815|7015|2|699| +2450815|7016|2|417| +2450815|7018|2|22| +2450815|7021|2|687| +2450815|7022|2|264| +2450815|7024|2|| +2450815|7027|2|171| +2450815|7028|2|531| +2450815|7030|2|272| +2450815|7033|2|67| +2450815|7034|2|800| +2450815|7036|2|719| +2450815|7039|2|862| +2450815|7040|2|948| +2450815|7042|2|399| +2450815|7045|2|450| +2450815|7046|2|734| +2450815|7048|2|343| +2450815|7051|2|557| +2450815|7052|2|686| +2450815|7054|2|686| +2450815|7057|2|415| +2450815|7058|2|108| +2450815|7060|2|946| +2450815|7063|2|838| +2450815|7064|2|728| +2450815|7066|2|789| +2450815|7069|2|480| +2450815|7070|2|362| +2450815|7072|2|785| +2450815|7075|2|82| +2450815|7076|2|553| +2450815|7078|2|9| +2450815|7081|2|935| +2450815|7082|2|38| +2450815|7084|2|357| +2450815|7087|2|256| +2450815|7088|2|601| +2450815|7090|2|485| +2450815|7093|2|439| +2450815|7094|2|157| +2450815|7096|2|619| +2450815|7099|2|15| +2450815|7100|2|157| +2450815|7102|2|35| +2450815|7105|2|481| +2450815|7106|2|203| +2450815|7108|2|121| +2450815|7111|2|| +2450815|7112|2|299| +2450815|7114|2|819| +2450815|7117|2|290| +2450815|7118|2|530| +2450815|7120|2|358| +2450815|7123|2|531| +2450815|7124|2|752| +2450815|7126|2|884| +2450815|7129|2|651| +2450815|7130|2|489| +2450815|7132|2|59| +2450815|7135|2|17| +2450815|7136|2|| +2450815|7138|2|111| +2450815|7141|2|501| +2450815|7142|2|| +2450815|7144|2|864| +2450815|7147|2|980| +2450815|7148|2|24| +2450815|7150|2|223| +2450815|7153|2|148| +2450815|7154|2|578| +2450815|7156|2|153| +2450815|7159|2|561| +2450815|7160|2|47| +2450815|7162|2|367| +2450815|7165|2|324| +2450815|7166|2|586| +2450815|7168|2|609| +2450815|7171|2|| +2450815|7172|2|886| +2450815|7174|2|144| +2450815|7177|2|162| +2450815|7178|2|565| +2450815|7180|2|478| +2450815|7183|2|594| +2450815|7184|2|537| +2450815|7186|2|454| +2450815|7189|2|882| +2450815|7190|2|302| +2450815|7192|2|202| +2450815|7195|2|869| +2450815|7196|2|| +2450815|7198|2|564| +2450815|7201|2|807| +2450815|7202|2|726| +2450815|7204|2|269| +2450815|7207|2|| +2450815|7208|2|856| +2450815|7210|2|221| +2450815|7213|2|50| +2450815|7214|2|614| +2450815|7216|2|913| +2450815|7219|2|936| +2450815|7220|2|911| +2450815|7222|2|327| +2450815|7225|2|91| +2450815|7226|2|361| +2450815|7228|2|568| +2450815|7231|2|264| +2450815|7232|2|60| +2450815|7234|2|624| +2450815|7237|2|912| +2450815|7238|2|597| +2450815|7240|2|612| +2450815|7243|2|908| +2450815|7244|2|296| +2450815|7246|2|561| +2450815|7249|2|111| +2450815|7250|2|634| +2450815|7252|2|789| +2450815|7255|2|| +2450815|7256|2|152| +2450815|7258|2|644| +2450815|7261|2|| +2450815|7262|2|| +2450815|7264|2|143| +2450815|7267|2|930| +2450815|7268|2|974| +2450815|7270|2|342| +2450815|7273|2|957| +2450815|7274|2|655| +2450815|7276|2|171| +2450815|7279|2|765| +2450815|7280|2|783| +2450815|7282|2|738| +2450815|7285|2|599| +2450815|7286|2|719| +2450815|7288|2|433| +2450815|7291|2|664| +2450815|7292|2|95| +2450815|7294|2|| +2450815|7297|2|66| +2450815|7298|2|487| +2450815|7300|2|700| +2450815|7303|2|769| +2450815|7304|2|267| +2450815|7306|2|| +2450815|7309|2|806| +2450815|7310|2|776| +2450815|7312|2|723| +2450815|7315|2|930| +2450815|7316|2|73| +2450815|7318|2|952| +2450815|7321|2|137| +2450815|7322|2|262| +2450815|7324|2|977| +2450815|7327|2|482| +2450815|7328|2|525| +2450815|7330|2|75| +2450815|7333|2|628| +2450815|7334|2|| +2450815|7336|2|| +2450815|7339|2|473| +2450815|7340|2|99| +2450815|7342|2|988| +2450815|7345|2|980| +2450815|7346|2|795| +2450815|7348|2|228| +2450815|7351|2|446| +2450815|7352|2|276| +2450815|7354|2|154| +2450815|7357|2|322| +2450815|7358|2|891| +2450815|7360|2|401| +2450815|7363|2|953| +2450815|7364|2|318| +2450815|7366|2|38| +2450815|7369|2|| +2450815|7370|2|70| +2450815|7372|2|709| +2450815|7375|2|943| +2450815|7376|2|965| +2450815|7378|2|63| +2450815|7381|2|202| +2450815|7382|2|879| +2450815|7384|2|602| +2450815|7387|2|890| +2450815|7388|2|32| +2450815|7390|2|635| +2450815|7393|2|31| +2450815|7394|2|145| +2450815|7396|2|311| +2450815|7399|2|250| +2450815|7400|2|324| +2450815|7402|2|174| +2450815|7405|2|765| +2450815|7406|2|317| +2450815|7408|2|728| +2450815|7411|2|364| +2450815|7412|2|479| +2450815|7414|2|980| +2450815|7417|2|930| +2450815|7418|2|278| +2450815|7420|2|286| +2450815|7423|2|933| +2450815|7424|2|147| +2450815|7426|2|315| +2450815|7429|2|973| +2450815|7430|2|641| +2450815|7432|2|311| +2450815|7435|2|226| +2450815|7436|2|724| +2450815|7438|2|955| +2450815|7441|2|750| +2450815|7442|2|108| +2450815|7444|2|934| +2450815|7447|2|862| +2450815|7448|2|| +2450815|7450|2|979| +2450815|7453|2|393| +2450815|7454|2|672| +2450815|7456|2|380| +2450815|7459|2|549| +2450815|7460|2|139| +2450815|7462|2|66| +2450815|7465|2|550| +2450815|7466|2|248| +2450815|7468|2|558| +2450815|7471|2|777| +2450815|7472|2|711| +2450815|7474|2|284| +2450815|7477|2|625| +2450815|7478|2|| +2450815|7480|2|579| +2450815|7483|2|336| +2450815|7484|2|191| +2450815|7486|2|24| +2450815|7489|2|950| +2450815|7490|2|612| +2450815|7492|2|434| +2450815|7495|2|882| +2450815|7496|2|475| +2450815|7498|2|561| +2450815|7501|2|702| +2450815|7502|2|37| +2450815|7504|2|561| +2450815|7507|2|512| +2450815|7508|2|696| +2450815|7510|2|395| +2450815|7513|2|978| +2450815|7514|2|138| +2450815|7516|2|226| +2450815|7519|2|262| +2450815|7520|2|336| +2450815|7522|2|948| +2450815|7525|2|99| +2450815|7526|2|510| +2450815|7528|2|782| +2450815|7531|2|989| +2450815|7532|2|448| +2450815|7534|2|62| +2450815|7537|2|313| +2450815|7538|2|543| +2450815|7540|2|490| +2450815|7543|2|174| +2450815|7544|2|731| +2450815|7546|2|170| +2450815|7549|2|485| +2450815|7550|2|492| +2450815|7552|2|855| +2450815|7555|2|759| +2450815|7556|2|265| +2450815|7558|2|470| +2450815|7561|2|736| +2450815|7562|2|119| +2450815|7564|2|66| +2450815|7567|2|448| +2450815|7568|2|206| +2450815|7570|2|| +2450815|7573|2|987| +2450815|7574|2|94| +2450815|7576|2|757| +2450815|7579|2|746| +2450815|7580|2|920| +2450815|7582|2|594| +2450815|7585|2|25| +2450815|7586|2|283| +2450815|7588|2|225| +2450815|7591|2|931| +2450815|7592|2|927| +2450815|7594|2|392| +2450815|7597|2|728| +2450815|7598|2|565| +2450815|7600|2|665| +2450815|7603|2|407| +2450815|7604|2|942| +2450815|7606|2|288| +2450815|7609|2|385| +2450815|7610|2|460| +2450815|7612|2|478| +2450815|7615|2|| +2450815|7616|2|577| +2450815|7618|2|686| +2450815|7621|2|411| +2450815|7622|2|601| +2450815|7624|2|919| +2450815|7627|2|| +2450815|7628|2|688| +2450815|7630|2|907| +2450815|7633|2|286| +2450815|7634|2|462| +2450815|7636|2|198| +2450815|7639|2|607| +2450815|7640|2|411| +2450815|7642|2|886| +2450815|7645|2|100| +2450815|7646|2|755| +2450815|7648|2|879| +2450815|7651|2|238| +2450815|7652|2|400| +2450815|7654|2|195| +2450815|7657|2|720| +2450815|7658|2|901| +2450815|7660|2|960| +2450815|7663|2|811| +2450815|7664|2|405| +2450815|7666|2|254| +2450815|7669|2|449| +2450815|7670|2|775| +2450815|7672|2|603| +2450815|7675|2|587| +2450815|7676|2|960| +2450815|7678|2|643| +2450815|7681|2|202| +2450815|7682|2|531| +2450815|7684|2|298| +2450815|7687|2|298| +2450815|7688|2|787| +2450815|7690|2|942| +2450815|7693|2|176| +2450815|7694|2|199| +2450815|7696|2|458| +2450815|7699|2|270| +2450815|7700|2|| +2450815|7702|2|123| +2450815|7705|2|214| +2450815|7706|2|968| +2450815|7708|2|91| +2450815|7711|2|75| +2450815|7712|2|246| +2450815|7714|2|142| +2450815|7717|2|919| +2450815|7718|2|529| +2450815|7720|2|813| +2450815|7723|2|547| +2450815|7724|2|219| +2450815|7726|2|591| +2450815|7729|2|278| +2450815|7730|2|754| +2450815|7732|2|411| +2450815|7735|2|828| +2450815|7736|2|27| +2450815|7738|2|128| +2450815|7741|2|778| +2450815|7742|2|332| +2450815|7744|2|311| +2450815|7747|2|296| +2450815|7748|2|269| +2450815|7750|2|469| +2450815|7753|2|769| +2450815|7754|2|693| +2450815|7756|2|| +2450815|7759|2|357| +2450815|7760|2|566| +2450815|7762|2|370| +2450815|7765|2|342| +2450815|7766|2|793| +2450815|7768|2|440| +2450815|7771|2|981| +2450815|7772|2|769| +2450815|7774|2|757| +2450815|7777|2|154| +2450815|7778|2|466| +2450815|7780|2|700| +2450815|7783|2|922| +2450815|7784|2|520| +2450815|7786|2|163| +2450815|7789|2|167| +2450815|7790|2|247| +2450815|7792|2|577| +2450815|7795|2|914| +2450815|7796|2|669| +2450815|7798|2|666| +2450815|7801|2|692| +2450815|7802|2|| +2450815|7804|2|152| +2450815|7807|2|724| +2450815|7808|2|148| +2450815|7810|2|470| +2450815|7813|2|| +2450815|7814|2|| +2450815|7816|2|885| +2450815|7819|2|341| +2450815|7820|2|158| +2450815|7822|2|245| +2450815|7825|2|672| +2450815|7826|2|722| +2450815|7828|2|435| +2450815|7831|2|9| +2450815|7832|2|870| +2450815|7834|2|673| +2450815|7837|2|| +2450815|7838|2|164| +2450815|7840|2|714| +2450815|7843|2|634| +2450815|7844|2|53| +2450815|7846|2|| +2450815|7849|2|385| +2450815|7850|2|735| +2450815|7852|2|151| +2450815|7855|2|861| +2450815|7856|2|476| +2450815|7858|2|597| +2450815|7861|2|352| +2450815|7862|2|951| +2450815|7864|2|602| +2450815|7867|2|548| +2450815|7868|2|623| +2450815|7870|2|681| +2450815|7873|2|836| +2450815|7874|2|335| +2450815|7876|2|381| +2450815|7879|2|981| +2450815|7880|2|619| +2450815|7882|2|386| +2450815|7885|2|46| +2450815|7886|2|299| +2450815|7888|2|517| +2450815|7891|2|442| +2450815|7892|2|525| +2450815|7894|2|598| +2450815|7897|2|571| +2450815|7898|2|539| +2450815|7900|2|739| +2450815|7903|2|956| +2450815|7904|2|683| +2450815|7906|2|568| +2450815|7909|2|922| +2450815|7910|2|740| +2450815|7912|2|607| +2450815|7915|2|272| +2450815|7916|2|778| +2450815|7918|2|980| +2450815|7921|2|389| +2450815|7922|2|302| +2450815|7924|2|939| +2450815|7927|2|494| +2450815|7928|2|781| +2450815|7930|2|752| +2450815|7933|2|627| +2450815|7934|2|437| +2450815|7936|2|18| +2450815|7939|2|610| +2450815|7940|2|| +2450815|7942|2|370| +2450815|7945|2|| +2450815|7946|2|261| +2450815|7948|2|107| +2450815|7951|2|586| +2450815|7952|2|22| +2450815|7954|2|55| +2450815|7957|2|142| +2450815|7958|2|| +2450815|7960|2|241| +2450815|7963|2|505| +2450815|7964|2|914| +2450815|7966|2|70| +2450815|7969|2|290| +2450815|7970|2|735| +2450815|7972|2|428| +2450815|7975|2|858| +2450815|7976|2|705| +2450815|7978|2|603| +2450815|7981|2|921| +2450815|7982|2|464| +2450815|7984|2|929| +2450815|7987|2|153| +2450815|7988|2|326| +2450815|7990|2|141| +2450815|7993|2|958| +2450815|7994|2|601| +2450815|7996|2|820| +2450815|7999|2|517| +2450815|8000|2|372| +2450815|8002|2|172| +2450815|8005|2|| +2450815|8006|2|734| +2450815|8008|2|470| +2450815|8011|2|950| +2450815|8012|2|81| +2450815|8014|2|902| +2450815|8017|2|615| +2450815|8018|2|903| +2450815|8020|2|791| +2450815|8023|2|| +2450815|8024|2|844| +2450815|8026|2|835| +2450815|8029|2|591| +2450815|8030|2|27| +2450815|8032|2|730| +2450815|8035|2|960| +2450815|8036|2|304| +2450815|8038|2|215| +2450815|8041|2|813| +2450815|8042|2|921| +2450815|8044|2|489| +2450815|8047|2|424| +2450815|8048|2|775| +2450815|8050|2|411| +2450815|8053|2|116| +2450815|8054|2|181| +2450815|8056|2|419| +2450815|8059|2|897| +2450815|8060|2|43| +2450815|8062|2|328| +2450815|8065|2|445| +2450815|8066|2|747| +2450815|8068|2|819| +2450815|8071|2|589| +2450815|8072|2|325| +2450815|8074|2|326| +2450815|8077|2|30| +2450815|8078|2|| +2450815|8080|2|140| +2450815|8083|2|832| +2450815|8084|2|489| +2450815|8086|2|642| +2450815|8089|2|850| +2450815|8090|2|603| +2450815|8092|2|301| +2450815|8095|2|9| +2450815|8096|2|1000| +2450815|8098|2|288| +2450815|8101|2|608| +2450815|8102|2|526| +2450815|8104|2|620| +2450815|8107|2|228| +2450815|8108|2|698| +2450815|8110|2|368| +2450815|8113|2|653| +2450815|8114|2|694| +2450815|8116|2|577| +2450815|8119|2|27| +2450815|8120|2|557| +2450815|8122|2|620| +2450815|8125|2|824| +2450815|8126|2|449| +2450815|8128|2|| +2450815|8131|2|161| +2450815|8132|2|589| +2450815|8134|2|526| +2450815|8137|2|764| +2450815|8138|2|123| +2450815|8140|2|405| +2450815|8143|2|540| +2450815|8144|2|220| +2450815|8146|2|| +2450815|8149|2|65| +2450815|8150|2|291| +2450815|8152|2|849| +2450815|8155|2|597| +2450815|8156|2|285| +2450815|8158|2|795| +2450815|8161|2|39| +2450815|8162|2|814| +2450815|8164|2|482| +2450815|8167|2|978| +2450815|8168|2|| +2450815|8170|2|946| +2450815|8173|2|923| +2450815|8174|2|357| +2450815|8176|2|738| +2450815|8179|2|663| +2450815|8180|2|987| +2450815|8182|2|404| +2450815|8185|2|36| +2450815|8186|2|704| +2450815|8188|2|317| +2450815|8191|2|850| +2450815|8192|2|351| +2450815|8194|2|268| +2450815|8197|2|431| +2450815|8198|2|336| +2450815|8200|2|147| +2450815|8203|2|235| +2450815|8204|2|811| +2450815|8206|2|542| +2450815|8209|2|97| +2450815|8210|2|594| +2450815|8212|2|389| +2450815|8215|2|282| +2450815|8216|2|102| +2450815|8218|2|728| +2450815|8221|2|323| +2450815|8222|2|963| +2450815|8224|2|1000| +2450815|8227|2|375| +2450815|8228|2|246| +2450815|8230|2|734| +2450815|8233|2|569| +2450815|8234|2|166| +2450815|8236|2|969| +2450815|8239|2|556| +2450815|8240|2|498| +2450815|8242|2|547| +2450815|8245|2|33| +2450815|8246|2|275| +2450815|8248|2|760| +2450815|8251|2|509| +2450815|8252|2|585| +2450815|8254|2|269| +2450815|8257|2|258| +2450815|8258|2|379| +2450815|8260|2|585| +2450815|8263|2|958| +2450815|8264|2|553| +2450815|8266|2|164| +2450815|8269|2|196| +2450815|8270|2|569| +2450815|8272|2|584| +2450815|8275|2|594| +2450815|8276|2|317| +2450815|8278|2|329| +2450815|8281|2|303| +2450815|8282|2|632| +2450815|8284|2|387| +2450815|8287|2|696| +2450815|8288|2|584| +2450815|8290|2|129| +2450815|8293|2|330| +2450815|8294|2|360| +2450815|8296|2|539| +2450815|8299|2|841| +2450815|8300|2|476| +2450815|8302|2|495| +2450815|8305|2|161| +2450815|8306|2|130| +2450815|8308|2|107| +2450815|8311|2|413| +2450815|8312|2|98| +2450815|8314|2|| +2450815|8317|2|603| +2450815|8318|2|916| +2450815|8320|2|815| +2450815|8323|2|724| +2450815|8324|2|773| +2450815|8326|2|562| +2450815|8329|2|114| +2450815|8330|2|896| +2450815|8332|2|303| +2450815|8335|2|303| +2450815|8336|2|734| +2450815|8338|2|822| +2450815|8341|2|709| +2450815|8342|2|266| +2450815|8344|2|297| +2450815|8347|2|175| +2450815|8348|2|714| +2450815|8350|2|860| +2450815|8353|2|472| +2450815|8354|2|42| +2450815|8356|2|624| +2450815|8359|2|740| +2450815|8360|2|203| +2450815|8362|2|929| +2450815|8365|2|| +2450815|8366|2|198| +2450815|8368|2|50| +2450815|8371|2|695| +2450815|8372|2|311| +2450815|8374|2|170| +2450815|8377|2|177| +2450815|8378|2|631| +2450815|8380|2|742| +2450815|8383|2|502| +2450815|8384|2|142| +2450815|8386|2|698| +2450815|8389|2|755| +2450815|8390|2|652| +2450815|8392|2|827| +2450815|8395|2|183| +2450815|8396|2|390| +2450815|8398|2|177| +2450815|8401|2|516| +2450815|8402|2|471| +2450815|8404|2|10| +2450815|8407|2|956| +2450815|8408|2|337| +2450815|8410|2|369| +2450815|8413|2|590| +2450815|8414|2|208| +2450815|8416|2|934| +2450815|8419|2|680| +2450815|8420|2|147| +2450815|8422|2|216| +2450815|8425|2|447| +2450815|8426|2|878| +2450815|8428|2|43| +2450815|8431|2|486| +2450815|8432|2|346| +2450815|8434|2|180| +2450815|8437|2|233| +2450815|8438|2|618| +2450815|8440|2|594| +2450815|8443|2|517| +2450815|8444|2|941| +2450815|8446|2|892| +2450815|8449|2|224| +2450815|8450|2|936| +2450815|8452|2|926| +2450815|8455|2|947| +2450815|8456|2|549| +2450815|8458|2|366| +2450815|8461|2|349| +2450815|8462|2|700| +2450815|8464|2|968| +2450815|8467|2|117| +2450815|8468|2|929| +2450815|8470|2|785| +2450815|8473|2|193| +2450815|8474|2|74| +2450815|8476|2|299| +2450815|8479|2|374| +2450815|8480|2|95| +2450815|8482|2|350| +2450815|8485|2|845| +2450815|8486|2|99| +2450815|8488|2|297| +2450815|8491|2|131| +2450815|8492|2|569| +2450815|8494|2|708| +2450815|8497|2|874| +2450815|8498|2|328| +2450815|8500|2|586| +2450815|8503|2|369| +2450815|8504|2|270| +2450815|8506|2|817| +2450815|8509|2|650| +2450815|8510|2|795| +2450815|8512|2|239| +2450815|8515|2|137| +2450815|8516|2|905| +2450815|8518|2|396| +2450815|8521|2|358| +2450815|8522|2|79| +2450815|8524|2|726| +2450815|8527|2|755| +2450815|8528|2|780| +2450815|8530|2|167| +2450815|8533|2|527| +2450815|8534|2|541| +2450815|8536|2|407| +2450815|8539|2|674| +2450815|8540|2|912| +2450815|8542|2|124| +2450815|8545|2|69| +2450815|8546|2|781| +2450815|8548|2|967| +2450815|8551|2|26| +2450815|8552|2|944| +2450815|8554|2|686| +2450815|8557|2|145| +2450815|8558|2|711| +2450815|8560|2|46| +2450815|8563|2|404| +2450815|8564|2|| +2450815|8566|2|556| +2450815|8569|2|426| +2450815|8570|2|342| +2450815|8572|2|409| +2450815|8575|2|34| +2450815|8576|2|447| +2450815|8578|2|162| +2450815|8581|2|887| +2450815|8582|2|797| +2450815|8584|2|975| +2450815|8587|2|107| +2450815|8588|2|251| +2450815|8590|2|496| +2450815|8593|2|687| +2450815|8594|2|43| +2450815|8596|2|935| +2450815|8599|2|187| +2450815|8600|2|467| +2450815|8602|2|919| +2450815|8605|2|904| +2450815|8606|2|86| +2450815|8608|2|320| +2450815|8611|2|561| +2450815|8612|2|77| +2450815|8614|2|346| +2450815|8617|2|182| +2450815|8618|2|955| +2450815|8620|2|71| +2450815|8623|2|539| +2450815|8624|2|420| +2450815|8626|2|344| +2450815|8629|2|307| +2450815|8630|2|| +2450815|8632|2|897| +2450815|8635|2|52| +2450815|8636|2|472| +2450815|8638|2|60| +2450815|8641|2|147| +2450815|8642|2|585| +2450815|8644|2|534| +2450815|8647|2|943| +2450815|8648|2|909| +2450815|8650|2|189| +2450815|8653|2|213| +2450815|8654|2|753| +2450815|8656|2|691| +2450815|8659|2|65| +2450815|8660|2|445| +2450815|8662|2|561| +2450815|8665|2|327| +2450815|8666|2|980| +2450815|8668|2|640| +2450815|8671|2|239| +2450815|8672|2|| +2450815|8674|2|724| +2450815|8677|2|887| +2450815|8678|2|261| +2450815|8680|2|600| +2450815|8683|2|636| +2450815|8684|2|728| +2450815|8686|2|218| +2450815|8689|2|174| +2450815|8690|2|837| +2450815|8692|2|90| +2450815|8695|2|808| +2450815|8696|2|721| +2450815|8698|2|795| +2450815|8701|2|932| +2450815|8702|2|202| +2450815|8704|2|289| +2450815|8707|2|974| +2450815|8708|2|977| +2450815|8710|2|| +2450815|8713|2|939| +2450815|8714|2|616| +2450815|8716|2|682| +2450815|8719|2|172| +2450815|8720|2|642| +2450815|8722|2|302| +2450815|8725|2|398| +2450815|8726|2|192| +2450815|8728|2|718| +2450815|8731|2|0| +2450815|8732|2|326| +2450815|8734|2|972| +2450815|8737|2|314| +2450815|8738|2|565| +2450815|8740|2|616| +2450815|8743|2|240| +2450815|8744|2|835| +2450815|8746|2|129| +2450815|8749|2|| +2450815|8750|2|208| +2450815|8752|2|761| +2450815|8755|2|860| +2450815|8756|2|368| +2450815|8758|2|405| +2450815|8761|2|458| +2450815|8762|2|727| +2450815|8764|2|924| +2450815|8767|2|828| +2450815|8768|2|154| +2450815|8770|2|672| +2450815|8773|2|92| +2450815|8774|2|801| +2450815|8776|2|786| +2450815|8779|2|688| +2450815|8780|2|312| +2450815|8782|2|445| +2450815|8785|2|75| +2450815|8786|2|709| +2450815|8788|2|952| +2450815|8791|2|366| +2450815|8792|2|186| +2450815|8794|2|517| +2450815|8797|2|612| +2450815|8798|2|638| +2450815|8800|2|782| +2450815|8803|2|405| +2450815|8804|2|652| +2450815|8806|2|139| +2450815|8809|2|45| +2450815|8810|2|545| +2450815|8812|2|475| +2450815|8815|2|444| +2450815|8816|2|602| +2450815|8818|2|937| +2450815|8821|2|568| +2450815|8822|2|416| +2450815|8824|2|987| +2450815|8827|2|905| +2450815|8828|2|88| +2450815|8830|2|448| +2450815|8833|2|909| +2450815|8834|2|310| +2450815|8836|2|589| +2450815|8839|2|270| +2450815|8840|2|662| +2450815|8842|2|12| +2450815|8845|2|969| +2450815|8846|2|726| +2450815|8848|2|836| +2450815|8851|2|55| +2450815|8852|2|549| +2450815|8854|2|910| +2450815|8857|2|539| +2450815|8858|2|354| +2450815|8860|2|| +2450815|8863|2|333| +2450815|8864|2|653| +2450815|8866|2|536| +2450815|8869|2|72| +2450815|8870|2|43| +2450815|8872|2|390| +2450815|8875|2|259| +2450815|8876|2|864| +2450815|8878|2|708| +2450815|8881|2|774| +2450815|8882|2|22| +2450815|8884|2|63| +2450815|8887|2|902| +2450815|8888|2|351| +2450815|8890|2|566| +2450815|8893|2|774| +2450815|8894|2|183| +2450815|8896|2|607| +2450815|8899|2|912| +2450815|8900|2|50| +2450815|8902|2|825| +2450815|8905|2|895| +2450815|8906|2|570| +2450815|8908|2|428| +2450815|8911|2|376| +2450815|8912|2|640| +2450815|8914|2|960| +2450815|8917|2|983| +2450815|8918|2|751| +2450815|8920|2|445| +2450815|8923|2|308| +2450815|8924|2|326| +2450815|8926|2|569| +2450815|8929|2|415| +2450815|8930|2|404| +2450815|8932|2|312| +2450815|8935|2|291| +2450815|8936|2|774| +2450815|8938|2|247| +2450815|8941|2|| +2450815|8942|2|314| +2450815|8944|2|481| +2450815|8947|2|6| +2450815|8948|2|244| +2450815|8950|2|383| +2450815|8953|2|797| +2450815|8954|2|515| +2450815|8956|2|689| +2450815|8959|2|636| +2450815|8960|2|235| +2450815|8962|2|450| +2450815|8965|2|516| +2450815|8966|2|471| +2450815|8968|2|563| +2450815|8971|2|579| +2450815|8972|2|1000| +2450815|8974|2|210| +2450815|8977|2|252| +2450815|8978|2|903| +2450815|8980|2|363| +2450815|8983|2|| +2450815|8984|2|458| +2450815|8986|2|266| +2450815|8989|2|263| +2450815|8990|2|538| +2450815|8992|2|107| +2450815|8995|2|736| +2450815|8996|2|819| +2450815|8998|2|350| +2450815|9001|2|556| +2450815|9002|2|388| +2450815|9004|2|627| +2450815|9007|2|530| +2450815|9008|2|325| +2450815|9010|2|367| +2450815|9013|2|333| +2450815|9014|2|680| +2450815|9016|2|256| +2450815|9019|2|| +2450815|9020|2|941| +2450815|9022|2|216| +2450815|9025|2|866| +2450815|9026|2|844| +2450815|9028|2|414| +2450815|9031|2|16| +2450815|9032|2|528| +2450815|9034|2|675| +2450815|9037|2|258| +2450815|9038|2|573| +2450815|9040|2|975| +2450815|9043|2|946| +2450815|9044|2|| +2450815|9046|2|116| +2450815|9049|2|431| +2450815|9050|2|529| +2450815|9052|2|15| +2450815|9055|2|543| +2450815|9056|2|196| +2450815|9058|2|998| +2450815|9061|2|372| +2450815|9062|2|16| +2450815|9064|2|701| +2450815|9067|2|945| +2450815|9068|2|459| +2450815|9070|2|43| +2450815|9073|2|96| +2450815|9074|2|730| +2450815|9076|2|86| +2450815|9079|2|738| +2450815|9080|2|160| +2450815|9082|2|1000| +2450815|9085|2|| +2450815|9086|2|837| +2450815|9088|2|588| +2450815|9091|2|604| +2450815|9092|2|512| +2450815|9094|2|707| +2450815|9097|2|841| +2450815|9098|2|623| +2450815|9100|2|263| +2450815|9103|2|463| +2450815|9104|2|864| +2450815|9106|2|682| +2450815|9109|2|220| +2450815|9110|2|845| +2450815|9112|2|192| +2450815|9115|2|487| +2450815|9116|2|596| +2450815|9118|2|0| +2450815|9121|2|803| +2450815|9122|2|806| +2450815|9124|2|68| +2450815|9127|2|488| +2450815|9128|2|205| +2450815|9130|2|770| +2450815|9133|2|675| +2450815|9134|2|621| +2450815|9136|2|| +2450815|9139|2|| +2450815|9140|2|735| +2450815|9142|2|488| +2450815|9145|2|548| +2450815|9146|2|651| +2450815|9148|2|490| +2450815|9151|2|447| +2450815|9152|2|257| +2450815|9154|2|| +2450815|9157|2|316| +2450815|9158|2|876| +2450815|9160|2|696| +2450815|9163|2|42| +2450815|9164|2|569| +2450815|9166|2|973| +2450815|9169|2|644| +2450815|9170|2|517| +2450815|9172|2|200| +2450815|9175|2|910| +2450815|9176|2|203| +2450815|9178|2|422| +2450815|9181|2|908| +2450815|9182|2|412| +2450815|9184|2|637| +2450815|9187|2|264| +2450815|9188|2|460| +2450815|9190|2|557| +2450815|9193|2|569| +2450815|9194|2|286| +2450815|9196|2|848| +2450815|9199|2|976| +2450815|9200|2|| +2450815|9202|2|| +2450815|9205|2|369| +2450815|9206|2|237| +2450815|9208|2|943| +2450815|9211|2|320| +2450815|9212|2|303| +2450815|9214|2|898| +2450815|9217|2|219| +2450815|9218|2|808| +2450815|9220|2|| +2450815|9223|2|863| +2450815|9224|2|567| +2450815|9226|2|629| +2450815|9229|2|82| +2450815|9230|2|238| +2450815|9232|2|610| +2450815|9235|2|662| +2450815|9236|2|191| +2450815|9238|2|| +2450815|9241|2|167| +2450815|9242|2|678| +2450815|9244|2|| +2450815|9247|2|990| +2450815|9248|2|941| +2450815|9250|2|95| +2450815|9253|2|543| +2450815|9254|2|634| +2450815|9256|2|121| +2450815|9259|2|47| +2450815|9260|2|815| +2450815|9262|2|134| +2450815|9265|2|407| +2450815|9266|2|631| +2450815|9268|2|902| +2450815|9271|2|371| +2450815|9272|2|817| +2450815|9274|2|649| +2450815|9277|2|268| +2450815|9278|2|920| +2450815|9280|2|| +2450815|9283|2|886| +2450815|9284|2|866| +2450815|9286|2|1| +2450815|9289|2|954| +2450815|9290|2|9| +2450815|9292|2|346| +2450815|9295|2|635| +2450815|9296|2|103| +2450815|9298|2|356| +2450815|9301|2|618| +2450815|9302|2|952| +2450815|9304|2|227| +2450815|9307|2|532| +2450815|9308|2|455| +2450815|9310|2|44| +2450815|9313|2|| +2450815|9314|2|139| +2450815|9316|2|797| +2450815|9319|2|680| +2450815|9320|2|70| +2450815|9322|2|656| +2450815|9325|2|648| +2450815|9326|2|797| +2450815|9328|2|646| +2450815|9331|2|235| +2450815|9332|2|708| +2450815|9334|2|402| +2450815|9337|2|286| +2450815|9338|2|738| +2450815|9340|2|774| +2450815|9343|2|262| +2450815|9344|2|280| +2450815|9346|2|576| +2450815|9349|2|153| +2450815|9350|2|381| +2450815|9352|2|373| +2450815|9355|2|287| +2450815|9356|2|602| +2450815|9358|2|675| +2450815|9361|2|612| +2450815|9362|2|289| +2450815|9364|2|227| +2450815|9367|2|124| +2450815|9368|2|57| +2450815|9370|2|172| +2450815|9373|2|289| +2450815|9374|2|883| +2450815|9376|2|298| +2450815|9379|2|426| +2450815|9380|2|541| +2450815|9382|2|617| +2450815|9385|2|129| +2450815|9386|2|375| +2450815|9388|2|175| +2450815|9391|2|804| +2450815|9392|2|317| +2450815|9394|2|897| +2450815|9397|2|490| +2450815|9398|2|936| +2450815|9400|2|96| +2450815|9403|2|632| +2450815|9404|2|410| +2450815|9406|2|895| +2450815|9409|2|89| +2450815|9410|2|583| +2450815|9412|2|| +2450815|9415|2|145| +2450815|9416|2|564| +2450815|9418|2|201| +2450815|9421|2|154| +2450815|9422|2|734| +2450815|9424|2|570| +2450815|9427|2|683| +2450815|9428|2|516| +2450815|9430|2|42| +2450815|9433|2|905| +2450815|9434|2|475| +2450815|9436|2|610| +2450815|9439|2|52| +2450815|9440|2|271| +2450815|9442|2|251| +2450815|9445|2|725| +2450815|9446|2|892| +2450815|9448|2|667| +2450815|9451|2|| +2450815|9452|2|418| +2450815|9454|2|798| +2450815|9457|2|921| +2450815|9458|2|732| +2450815|9460|2|773| +2450815|9463|2|894| +2450815|9464|2|893| +2450815|9466|2|717| +2450815|9469|2|96| +2450815|9470|2|| +2450815|9472|2|483| +2450815|9475|2|882| +2450815|9476|2|122| +2450815|9478|2|862| +2450815|9481|2|41| +2450815|9482|2|405| +2450815|9484|2|636| +2450815|9487|2|306| +2450815|9488|2|394| +2450815|9490|2|774| +2450815|9493|2|467| +2450815|9494|2|976| +2450815|9496|2|| +2450815|9499|2|376| +2450815|9500|2|519| +2450815|9502|2|522| +2450815|9505|2|376| +2450815|9506|2|| +2450815|9508|2|222| +2450815|9511|2|647| +2450815|9512|2|818| +2450815|9514|2|313| +2450815|9517|2|586| +2450815|9518|2|| +2450815|9520|2|717| +2450815|9523|2|948| +2450815|9524|2|835| +2450815|9526|2|48| +2450815|9529|2|347| +2450815|9530|2|513| +2450815|9532|2|574| +2450815|9535|2|| +2450815|9536|2|462| +2450815|9538|2|83| +2450815|9541|2|685| +2450815|9542|2|823| +2450815|9544|2|468| +2450815|9547|2|143| +2450815|9548|2|162| +2450815|9550|2|856| +2450815|9553|2|832| +2450815|9554|2|763| +2450815|9556|2|95| +2450815|9559|2|352| +2450815|9560|2|760| +2450815|9562|2|794| +2450815|9565|2|277| +2450815|9566|2|970| +2450815|9568|2|943| +2450815|9571|2|31| +2450815|9572|2|314| +2450815|9574|2|638| +2450815|9577|2|39| +2450815|9578|2|| +2450815|9580|2|529| +2450815|9583|2|67| +2450815|9584|2|483| +2450815|9586|2|| +2450815|9589|2|643| +2450815|9590|2|951| +2450815|9592|2|435| +2450815|9595|2|363| +2450815|9596|2|882| +2450815|9598|2|282| +2450815|9601|2|584| +2450815|9602|2|867| +2450815|9604|2|320| +2450815|9607|2|325| +2450815|9608|2|267| +2450815|9610|2|| +2450815|9613|2|901| +2450815|9614|2|230| +2450815|9616|2|86| +2450815|9619|2|387| +2450815|9620|2|287| +2450815|9622|2|652| +2450815|9625|2|907| +2450815|9626|2|507| +2450815|9628|2|468| +2450815|9631|2|145| +2450815|9632|2|280| +2450815|9634|2|854| +2450815|9637|2|850| +2450815|9638|2|320| +2450815|9640|2|355| +2450815|9643|2|392| +2450815|9644|2|498| +2450815|9646|2|921| +2450815|9649|2|149| +2450815|9650|2|383| +2450815|9652|2|297| +2450815|9655|2|262| +2450815|9656|2|471| +2450815|9658|2|| +2450815|9661|2|260| +2450815|9662|2|851| +2450815|9664|2|558| +2450815|9667|2|957| +2450815|9668|2|624| +2450815|9670|2|339| +2450815|9673|2|| +2450815|9674|2|212| +2450815|9676|2|630| +2450815|9679|2|469| +2450815|9680|2|524| +2450815|9682|2|78| +2450815|9685|2|990| +2450815|9686|2|319| +2450815|9688|2|672| +2450815|9691|2|824| +2450815|9692|2|768| +2450815|9694|2|727| +2450815|9697|2|145| +2450815|9698|2|534| +2450815|9700|2|749| +2450815|9703|2|96| +2450815|9704|2|636| +2450815|9706|2|460| +2450815|9709|2|534| +2450815|9710|2|1| +2450815|9712|2|| +2450815|9715|2|305| +2450815|9716|2|| +2450815|9718|2|851| +2450815|9721|2|218| +2450815|9722|2|824| +2450815|9724|2|814| +2450815|9727|2|251| +2450815|9728|2|798| +2450815|9730|2|191| +2450815|9733|2|23| +2450815|9734|2|220| +2450815|9736|2|243| +2450815|9739|2|890| +2450815|9740|2|425| +2450815|9742|2|87| +2450815|9745|2|357| +2450815|9746|2|229| +2450815|9748|2|511| +2450815|9751|2|177| +2450815|9752|2|| +2450815|9754|2|315| +2450815|9757|2|282| +2450815|9758|2|779| +2450815|9760|2|228| +2450815|9763|2|721| +2450815|9764|2|922| +2450815|9766|2|63| +2450815|9769|2|512| +2450815|9770|2|319| +2450815|9772|2|271| +2450815|9775|2|619| +2450815|9776|2|23| +2450815|9778|2|161| +2450815|9781|2|933| +2450815|9782|2|632| +2450815|9784|2|105| +2450815|9787|2|728| +2450815|9788|2|327| +2450815|9790|2|653| +2450815|9793|2|270| +2450815|9794|2|279| +2450815|9796|2|30| +2450815|9799|2|910| +2450815|9800|2|456| +2450815|9802|2|103| +2450815|9805|2|269| +2450815|9806|2|270| +2450815|9808|2|515| +2450815|9811|2|682| +2450815|9812|2|238| +2450815|9814|2|827| +2450815|9817|2|1| +2450815|9818|2|869| +2450815|9820|2|372| +2450815|9823|2|643| +2450815|9824|2|712| +2450815|9826|2|327| +2450815|9829|2|945| +2450815|9830|2|904| +2450815|9832|2|| +2450815|9835|2|693| +2450815|9836|2|717| +2450815|9838|2|842| +2450815|9841|2|337| +2450815|9842|2|482| +2450815|9844|2|367| +2450815|9847|2|916| +2450815|9848|2|218| +2450815|9850|2|700| +2450815|9853|2|507| +2450815|9854|2|282| +2450815|9856|2|336| +2450815|9859|2|949| +2450815|9860|2|474| +2450815|9862|2|266| +2450815|9865|2|138| +2450815|9866|2|246| +2450815|9868|2|| +2450815|9871|2|15| +2450815|9872|2|486| +2450815|9874|2|742| +2450815|9877|2|327| +2450815|9878|2|107| +2450815|9880|2|66| +2450815|9883|2|515| +2450815|9884|2|889| +2450815|9886|2|777| +2450815|9889|2|872| +2450815|9890|2|938| +2450815|9892|2|829| +2450815|9895|2|830| +2450815|9896|2|670| +2450815|9898|2|497| +2450815|9901|2|240| +2450815|9902|2|911| +2450815|9904|2|865| +2450815|9907|2|68| +2450815|9908|2|410| +2450815|9910|2|161| +2450815|9913|2|536| +2450815|9914|2|870| +2450815|9916|2|455| +2450815|9919|2|887| +2450815|9920|2|112| +2450815|9922|2|525| +2450815|9925|2|990| +2450815|9926|2|368| +2450815|9928|2|573| +2450815|9931|2|354| +2450815|9932|2|448| +2450815|9934|2|703| +2450815|9937|2|885| +2450815|9938|2|171| +2450815|9940|2|770| +2450815|9943|2|990| +2450815|9944|2|369| +2450815|9946|2|773| +2450815|9949|2|409| +2450815|9950|2|570| +2450815|9952|2|905| +2450815|9955|2|151| +2450815|9956|2|447| +2450815|9958|2|525| +2450815|9961|2|| +2450815|9962|2|850| +2450815|9964|2|485| +2450815|9967|2|360| +2450815|9968|2|407| +2450815|9970|2|592| +2450815|9973|2|124| +2450815|9974|2|995| +2450815|9976|2|802| +2450815|9979|2|194| +2450815|9980|2|835| +2450815|9982|2|174| +2450815|9985|2|885| +2450815|9986|2|162| +2450815|9988|2|388| +2450815|9991|2|433| +2450815|9992|2|767| +2450815|9994|2|756| +2450815|9997|2|| +2450815|9998|2|559| +2450815|10000|2|765| +2450815|10003|2|583| +2450815|10004|2|316| +2450815|10006|2|847| +2450815|10009|2|76| +2450815|10010|2|270| +2450815|10012|2|875| +2450815|10015|2|499| +2450815|10016|2|1| +2450815|10018|2|125| +2450815|10021|2|416| +2450815|10022|2|689| +2450815|10024|2|358| +2450815|10027|2|57| +2450815|10028|2|839| +2450815|10030|2|798| +2450815|10033|2|362| +2450815|10034|2|300| +2450815|10036|2|682| +2450815|10039|2|872| +2450815|10040|2|593| +2450815|10042|2|105| +2450815|10045|2|45| +2450815|10046|2|513| +2450815|10048|2|349| +2450815|10051|2|680| +2450815|10052|2|514| +2450815|10054|2|60| +2450815|10057|2|589| +2450815|10058|2|944| +2450815|10060|2|322| +2450815|10063|2|609| +2450815|10064|2|994| +2450815|10066|2|521| +2450815|10069|2|295| +2450815|10070|2|664| +2450815|10072|2|863| +2450815|10075|2|845| +2450815|10076|2|4| +2450815|10078|2|599| +2450815|10081|2|471| +2450815|10082|2|| +2450815|10084|2|478| +2450815|10087|2|351| +2450815|10088|2|622| +2450815|10090|2|646| +2450815|10093|2|580| +2450815|10094|2|224| +2450815|10096|2|180| +2450815|10099|2|455| +2450815|10100|2|900| +2450815|10102|2|190| +2450815|10105|2|115| +2450815|10106|2|| +2450815|10108|2|467| +2450815|10111|2|924| +2450815|10112|2|588| +2450815|10114|2|167| +2450815|10117|2|743| +2450815|10118|2|525| +2450815|10120|2|685| +2450815|10123|2|49| +2450815|10124|2|434| +2450815|10126|2|785| +2450815|10129|2|2| +2450815|10130|2|288| +2450815|10132|2|800| +2450815|10135|2|322| +2450815|10136|2|34| +2450815|10138|2|763| +2450815|10141|2|256| +2450815|10142|2|115| +2450815|10144|2|| +2450815|10147|2|136| +2450815|10148|2|151| +2450815|10150|2|470| +2450815|10153|2|208| +2450815|10154|2|567| +2450815|10156|2|| +2450815|10159|2|607| +2450815|10160|2|301| +2450815|10162|2|443| +2450815|10165|2|632| +2450815|10166|2|402| +2450815|10168|2|182| +2450815|10171|2|714| +2450815|10172|2|85| +2450815|10174|2|602| +2450815|10177|2|527| +2450815|10178|2|617| +2450815|10180|2|565| +2450815|10183|2|209| +2450815|10184|2|967| +2450815|10186|2|587| +2450815|10189|2|715| +2450815|10190|2|911| +2450815|10192|2|200| +2450815|10195|2|553| +2450815|10196|2|111| +2450815|10198|2|920| +2450815|10201|2|790| +2450815|10202|2|639| +2450815|10204|2|587| +2450815|10207|2|986| +2450815|10208|2|988| +2450815|10210|2|622| +2450815|10213|2|338| +2450815|10214|2|789| +2450815|10216|2|700| +2450815|10219|2|911| +2450815|10220|2|169| +2450815|10222|2|720| +2450815|10225|2|135| +2450815|10226|2|925| +2450815|10228|2|913| +2450815|10231|2|711| +2450815|10232|2|180| +2450815|10234|2|634| +2450815|10237|2|813| +2450815|10238|2|113| +2450815|10240|2|617| +2450815|10243|2|550| +2450815|10244|2|924| +2450815|10246|2|681| +2450815|10249|2|78| +2450815|10250|2|611| +2450815|10252|2|824| +2450815|10255|2|109| +2450815|10256|2|958| +2450815|10258|2|281| +2450815|10261|2|276| +2450815|10262|2|659| +2450815|10264|2|810| +2450815|10267|2|220| +2450815|10268|2|438| +2450815|10270|2|782| +2450815|10273|2|76| +2450815|10274|2|| +2450815|10276|2|281| +2450815|10279|2|964| +2450815|10280|2|158| +2450815|10282|2|717| +2450815|10285|2|740| +2450815|10286|2|154| +2450815|10288|2|161| +2450815|10291|2|220| +2450815|10292|2|33| +2450815|10294|2|148| +2450815|10297|2|846| +2450815|10298|2|804| +2450815|10300|2|391| +2450815|10303|2|226| +2450815|10304|2|222| +2450815|10306|2|645| +2450815|10309|2|941| +2450815|10310|2|195| +2450815|10312|2|394| +2450815|10315|2|548| +2450815|10316|2|326| +2450815|10318|2|334| +2450815|10321|2|738| +2450815|10322|2|64| +2450815|10324|2|664| +2450815|10327|2|266| +2450815|10328|2|829| +2450815|10330|2|211| +2450815|10333|2|17| +2450815|10334|2|49| +2450815|10336|2|880| +2450815|10339|2|320| +2450815|10340|2|319| +2450815|10342|2|713| +2450815|10345|2|94| +2450815|10346|2|104| +2450815|10348|2|34| +2450815|10351|2|541| +2450815|10352|2|380| +2450815|10354|2|398| +2450815|10357|2|570| +2450815|10358|2|985| +2450815|10360|2|| +2450815|10363|2|760| +2450815|10364|2|618| +2450815|10366|2|843| +2450815|10369|2|786| +2450815|10370|2|558| +2450815|10372|2|285| +2450815|10375|2|| +2450815|10376|2|275| +2450815|10378|2|317| +2450815|10381|2|608| +2450815|10382|2|266| +2450815|10384|2|471| +2450815|10387|2|951| +2450815|10388|2|970| +2450815|10390|2|929| +2450815|10393|2|0| +2450815|10394|2|420| +2450815|10396|2|737| +2450815|10399|2|167| +2450815|10400|2|572| +2450815|10402|2|673| +2450815|10405|2|712| +2450815|10406|2|771| +2450815|10408|2|163| +2450815|10411|2|822| +2450815|10412|2|46| +2450815|10414|2|200| +2450815|10417|2|752| +2450815|10418|2|21| +2450815|10420|2|469| +2450815|10423|2|822| +2450815|10424|2|856| +2450815|10426|2|390| +2450815|10429|2|718| +2450815|10430|2|88| +2450815|10432|2|552| +2450815|10435|2|236| +2450815|10436|2|375| +2450815|10438|2|38| +2450815|10441|2|114| +2450815|10442|2|264| +2450815|10444|2|378| +2450815|10447|2|700| +2450815|10448|2|680| +2450815|10450|2|952| +2450815|10453|2|196| +2450815|10454|2|507| +2450815|10456|2|188| +2450815|10459|2|820| +2450815|10460|2|60| +2450815|10462|2|990| +2450815|10465|2|164| +2450815|10466|2|863| +2450815|10468|2|138| +2450815|10471|2|215| +2450815|10472|2|151| +2450815|10474|2|976| +2450815|10477|2|176| +2450815|10478|2|322| +2450815|10480|2|574| +2450815|10483|2|461| +2450815|10484|2|| +2450815|10486|2|612| +2450815|10489|2|| +2450815|10490|2|526| +2450815|10492|2|618| +2450815|10495|2|| +2450815|10496|2|836| +2450815|10498|2|731| +2450815|10501|2|936| +2450815|10502|2|146| +2450815|10504|2|421| +2450815|10507|2|552| +2450815|10508|2|638| +2450815|10510|2|758| +2450815|10513|2|758| +2450815|10514|2|200| +2450815|10516|2|| +2450815|10519|2|330| +2450815|10520|2|878| +2450815|10522|2|353| +2450815|10525|2|737| +2450815|10526|2|743| +2450815|10528|2|352| +2450815|10531|2|429| +2450815|10532|2|41| +2450815|10534|2|598| +2450815|10537|2|97| +2450815|10538|2|647| +2450815|10540|2|203| +2450815|10543|2|848| +2450815|10544|2|477| +2450815|10546|2|346| +2450815|10549|2|745| +2450815|10550|2|| +2450815|10552|2|403| +2450815|10555|2|| +2450815|10556|2|772| +2450815|10558|2|745| +2450815|10561|2|733| +2450815|10562|2|265| +2450815|10564|2|758| +2450815|10567|2|220| +2450815|10568|2|820| +2450815|10570|2|122| +2450815|10573|2|| +2450815|10574|2|68| +2450815|10576|2|344| +2450815|10579|2|127| +2450815|10580|2|287| +2450815|10582|2|398| +2450815|10585|2|914| +2450815|10586|2|160| +2450815|10588|2|967| +2450815|10591|2|568| +2450815|10592|2|569| +2450815|10594|2|898| +2450815|10597|2|676| +2450815|10598|2|785| +2450815|10600|2|152| +2450815|10603|2|841| +2450815|10604|2|469| +2450815|10606|2|470| +2450815|10609|2|679| +2450815|10610|2|252| +2450815|10612|2|204| +2450815|10615|2|888| +2450815|10616|2|680| +2450815|10618|2|536| +2450815|10621|2|575| +2450815|10622|2|487| +2450815|10624|2|181| +2450815|10627|2|354| +2450815|10628|2|576| +2450815|10630|2|378| +2450815|10633|2|258| +2450815|10634|2|975| +2450815|10636|2|560| +2450815|10639|2|475| +2450815|10640|2|966| +2450815|10642|2|5| +2450815|10645|2|| +2450815|10646|2|281| +2450815|10648|2|| +2450815|10651|2|800| +2450815|10652|2|267| +2450815|10654|2|631| +2450815|10657|2|112| +2450815|10658|2|254| +2450815|10660|2|707| +2450815|10663|2|691| +2450815|10664|2|49| +2450815|10666|2|372| +2450815|10669|2|178| +2450815|10670|2|425| +2450815|10672|2|646| +2450815|10675|2|871| +2450815|10676|2|| +2450815|10678|2|967| +2450815|10681|2|378| +2450815|10682|2|430| +2450815|10684|2|808| +2450815|10687|2|436| +2450815|10688|2|178| +2450815|10690|2|326| +2450815|10693|2|884| +2450815|10694|2|847| +2450815|10696|2|844| +2450815|10699|2|615| +2450815|10700|2|399| +2450815|10702|2|2| +2450815|10705|2|737| +2450815|10706|2|833| +2450815|10708|2|246| +2450815|10711|2|| +2450815|10712|2|409| +2450815|10714|2|419| +2450815|10717|2|| +2450815|10718|2|899| +2450815|10720|2|404| +2450815|10723|2|843| +2450815|10724|2|58| +2450815|10726|2|387| +2450815|10729|2|927| +2450815|10730|2|178| +2450815|10732|2|| +2450815|10735|2|180| +2450815|10736|2|677| +2450815|10738|2|348| +2450815|10741|2|863| +2450815|10742|2|975| +2450815|10744|2|561| +2450815|10747|2|216| +2450815|10748|2|908| +2450815|10750|2|335| +2450815|10753|2|160| +2450815|10754|2|501| +2450815|10756|2|10| +2450815|10759|2|21| +2450815|10760|2|677| +2450815|10762|2|332| +2450815|10765|2|658| +2450815|10766|2|893| +2450815|10768|2|694| +2450815|10771|2|| +2450815|10772|2|182| +2450815|10774|2|410| +2450815|10777|2|613| +2450815|10778|2|221| +2450815|10780|2|446| +2450815|10783|2|781| +2450815|10784|2|353| +2450815|10786|2|96| +2450815|10789|2|857| +2450815|10790|2|447| +2450815|10792|2|| +2450815|10795|2|265| +2450815|10796|2|38| +2450815|10798|2|752| +2450815|10801|2|495| +2450815|10802|2|527| +2450815|10804|2|505| +2450815|10807|2|981| +2450815|10808|2|48| +2450815|10810|2|839| +2450815|10813|2|181| +2450815|10814|2|142| +2450815|10816|2|661| +2450815|10819|2|58| +2450815|10820|2|128| +2450815|10822|2|252| +2450815|10825|2|302| +2450815|10826|2|884| +2450815|10828|2|678| +2450815|10831|2|242| +2450815|10832|2|143| +2450815|10834|2|269| +2450815|10837|2|| +2450815|10838|2|306| +2450815|10840|2|| +2450815|10843|2|748| +2450815|10844|2|160| +2450815|10846|2|255| +2450815|10849|2|522| +2450815|10850|2|671| +2450815|10852|2|994| +2450815|10855|2|989| +2450815|10856|2|| +2450815|10858|2|217| +2450815|10861|2|495| +2450815|10862|2|416| +2450815|10864|2|194| +2450815|10867|2|228| +2450815|10868|2|467| +2450815|10870|2|189| +2450815|10873|2|537| +2450815|10874|2|978| +2450815|10876|2|771| +2450815|10879|2|411| +2450815|10880|2|737| +2450815|10882|2|778| +2450815|10885|2|244| +2450815|10886|2|251| +2450815|10888|2|382| +2450815|10891|2|574| +2450815|10892|2|| +2450815|10894|2|41| +2450815|10897|2|713| +2450815|10898|2|478| +2450815|10900|2|605| +2450815|10903|2|| +2450815|10904|2|888| +2450815|10906|2|379| +2450815|10909|2|536| +2450815|10910|2|846| +2450815|10912|2|634| +2450815|10915|2|750| +2450815|10916|2|517| +2450815|10918|2|1000| +2450815|10921|2|829| +2450815|10922|2|522| +2450815|10924|2|481| +2450815|10927|2|730| +2450815|10928|2|0| +2450815|10930|2|74| +2450815|10933|2|331| +2450815|10934|2|670| +2450815|10936|2|| +2450815|10939|2|26| +2450815|10940|2|185| +2450815|10942|2|594| +2450815|10945|2|435| +2450815|10946|2|587| +2450815|10948|2|360| +2450815|10951|2|311| +2450815|10952|2|268| +2450815|10954|2|831| +2450815|10957|2|692| +2450815|10958|2|785| +2450815|10960|2|78| +2450815|10963|2|199| +2450815|10964|2|575| +2450815|10966|2|744| +2450815|10969|2|812| +2450815|10970|2|681| +2450815|10972|2|801| +2450815|10975|2|481| +2450815|10976|2|259| +2450815|10978|2|123| +2450815|10981|2|729| +2450815|10982|2|171| +2450815|10984|2|522| +2450815|10987|2|994| +2450815|10988|2|638| +2450815|10990|2|110| +2450815|10993|2|996| +2450815|10994|2|47| +2450815|10996|2|169| +2450815|10999|2|45| +2450815|11000|2|943| +2450815|11002|2|288| +2450815|11005|2|435| +2450815|11006|2|819| +2450815|11008|2|731| +2450815|11011|2|142| +2450815|11012|2|17| +2450815|11014|2|264| +2450815|11017|2|602| +2450815|11018|2|301| +2450815|11020|2|280| +2450815|11023|2|478| +2450815|11024|2|53| +2450815|11026|2|759| +2450815|11029|2|985| +2450815|11030|2|308| +2450815|11032|2|487| +2450815|11035|2|15| +2450815|11036|2|183| +2450815|11038|2|691| +2450815|11041|2|982| +2450815|11042|2|130| +2450815|11044|2|739| +2450815|11047|2|761| +2450815|11048|2|197| +2450815|11050|2|713| +2450815|11053|2|315| +2450815|11054|2|278| +2450815|11056|2|776| +2450815|11059|2|347| +2450815|11060|2|636| +2450815|11062|2|34| +2450815|11065|2|658| +2450815|11066|2|92| +2450815|11068|2|736| +2450815|11071|2|451| +2450815|11072|2|724| +2450815|11074|2|339| +2450815|11077|2|654| +2450815|11078|2|641| +2450815|11080|2|| +2450815|11083|2|883| +2450815|11084|2|| +2450815|11086|2|185| +2450815|11089|2|398| +2450815|11090|2|766| +2450815|11092|2|318| +2450815|11095|2|30| +2450815|11096|2|351| +2450815|11098|2|751| +2450815|11101|2|625| +2450815|11102|2|56| +2450815|11104|2|323| +2450815|11107|2|108| +2450815|11108|2|780| +2450815|11110|2|| +2450815|11113|2|489| +2450815|11114|2|706| +2450815|11116|2|746| +2450815|11119|2|783| +2450815|11120|2|608| +2450815|11122|2|179| +2450815|11125|2|722| +2450815|11126|2|37| +2450815|11128|2|449| +2450815|11131|2|763| +2450815|11132|2|159| +2450815|11134|2|147| +2450815|11137|2|15| +2450815|11138|2|782| +2450815|11140|2|88| +2450815|11143|2|| +2450815|11144|2|10| +2450815|11146|2|786| +2450815|11149|2|734| +2450815|11150|2|924| +2450815|11152|2|243| +2450815|11155|2|377| +2450815|11156|2|794| +2450815|11158|2|396| +2450815|11161|2|329| +2450815|11162|2|427| +2450815|11164|2|744| +2450815|11167|2|178| +2450815|11168|2|347| +2450815|11170|2|327| +2450815|11173|2|396| +2450815|11174|2|384| +2450815|11176|2|| +2450815|11179|2|| +2450815|11180|2|785| +2450815|11182|2|883| +2450815|11185|2|579| +2450815|11186|2|758| +2450815|11188|2|708| +2450815|11191|2|486| +2450815|11192|2|| +2450815|11194|2|434| +2450815|11197|2|160| +2450815|11198|2|687| +2450815|11200|2|83| +2450815|11203|2|746| +2450815|11204|2|162| +2450815|11206|2|278| +2450815|11209|2|122| +2450815|11210|2|705| +2450815|11212|2|191| +2450815|11215|2|366| +2450815|11216|2|204| +2450815|11218|2|575| +2450815|11221|2|234| +2450815|11222|2|34| +2450815|11224|2|973| +2450815|11227|2|824| +2450815|11228|2|791| +2450815|11230|2|326| +2450815|11233|2|472| +2450815|11234|2|538| +2450815|11236|2|8| +2450815|11239|2|988| +2450815|11240|2|679| +2450815|11242|2|603| +2450815|11245|2|| +2450815|11246|2|838| +2450815|11248|2|88| +2450815|11251|2|128| +2450815|11252|2|606| +2450815|11254|2|526| +2450815|11257|2|670| +2450815|11258|2|146| +2450815|11260|2|508| +2450815|11263|2|374| +2450815|11264|2|691| +2450815|11266|2|888| +2450815|11269|2|800| +2450815|11270|2|901| +2450815|11272|2|655| +2450815|11275|2|160| +2450815|11276|2|644| +2450815|11278|2|| +2450815|11281|2|150| +2450815|11282|2|732| +2450815|11284|2|398| +2450815|11287|2|837| +2450815|11288|2|68| +2450815|11290|2|478| +2450815|11293|2|787| +2450815|11294|2|36| +2450815|11296|2|| +2450815|11299|2|611| +2450815|11300|2|933| +2450815|11302|2|409| +2450815|11305|2|546| +2450815|11306|2|367| +2450815|11308|2|394| +2450815|11311|2|145| +2450815|11312|2|| +2450815|11314|2|832| +2450815|11317|2|814| +2450815|11318|2|687| +2450815|11320|2|455| +2450815|11323|2|87| +2450815|11324|2|509| +2450815|11326|2|884| +2450815|11329|2|995| +2450815|11330|2|377| +2450815|11332|2|675| +2450815|11335|2|2| +2450815|11336|2|964| +2450815|11338|2|585| +2450815|11341|2|921| +2450815|11342|2|568| +2450815|11344|2|895| +2450815|11347|2|805| +2450815|11348|2|137| +2450815|11350|2|430| +2450815|11353|2|622| +2450815|11354|2|470| +2450815|11356|2|500| +2450815|11359|2|626| +2450815|11360|2|334| +2450815|11362|2|394| +2450815|11365|2|414| +2450815|11366|2|550| +2450815|11368|2|860| +2450815|11371|2|155| +2450815|11372|2|318| +2450815|11374|2|| +2450815|11377|2|144| +2450815|11378|2|59| +2450815|11380|2|502| +2450815|11383|2|151| +2450815|11384|2|0| +2450815|11386|2|719| +2450815|11389|2|222| +2450815|11390|2|| +2450815|11392|2|| +2450815|11395|2|| +2450815|11396|2|899| +2450815|11398|2|41| +2450815|11401|2|54| +2450815|11402|2|675| +2450815|11404|2|993| +2450815|11407|2|322| +2450815|11408|2|238| +2450815|11410|2|187| +2450815|11413|2|47| +2450815|11414|2|807| +2450815|11416|2|723| +2450815|11419|2|888| +2450815|11420|2|788| +2450815|11422|2|540| +2450815|11425|2|702| +2450815|11426|2|74| +2450815|11428|2|| +2450815|11431|2|181| +2450815|11432|2|726| +2450815|11434|2|120| +2450815|11437|2|961| +2450815|11438|2|412| +2450815|11440|2|125| +2450815|11443|2|356| +2450815|11444|2|349| +2450815|11446|2|653| +2450815|11449|2|55| +2450815|11450|2|76| +2450815|11452|2|571| +2450815|11455|2|292| +2450815|11456|2|485| +2450815|11458|2|971| +2450815|11461|2|825| +2450815|11462|2|672| +2450815|11464|2|936| +2450815|11467|2|883| +2450815|11468|2|203| +2450815|11470|2|591| +2450815|11473|2|409| +2450815|11474|2|606| +2450815|11476|2|650| +2450815|11479|2|21| +2450815|11480|2|| +2450815|11482|2|| +2450815|11485|2|228| +2450815|11486|2|724| +2450815|11488|2|64| +2450815|11491|2|426| +2450815|11492|2|552| +2450815|11494|2|614| +2450815|11497|2|312| +2450815|11498|2|955| +2450815|11500|2|367| +2450815|11503|2|928| +2450815|11504|2|59| +2450815|11506|2|952| +2450815|11509|2|638| +2450815|11510|2|194| +2450815|11512|2|153| +2450815|11515|2|284| +2450815|11516|2|154| +2450815|11518|2|885| +2450815|11521|2|784| +2450815|11522|2|619| +2450815|11524|2|461| +2450815|11527|2|62| +2450815|11528|2|98| +2450815|11530|2|32| +2450815|11533|2|271| +2450815|11534|2|21| +2450815|11536|2|952| +2450815|11539|2|16| +2450815|11540|2|71| +2450815|11542|2|423| +2450815|11545|2|682| +2450815|11546|2|| +2450815|11548|2|619| +2450815|11551|2|491| +2450815|11552|2|592| +2450815|11554|2|253| +2450815|11557|2|391| +2450815|11558|2|806| +2450815|11560|2|625| +2450815|11563|2|543| +2450815|11564|2|469| +2450815|11566|2|830| +2450815|11569|2|644| +2450815|11570|2|755| +2450815|11572|2|312| +2450815|11575|2|606| +2450815|11576|2|| +2450815|11578|2|12| +2450815|11581|2|67| +2450815|11582|2|289| +2450815|11584|2|326| +2450815|11587|2|456| +2450815|11588|2|580| +2450815|11590|2|405| +2450815|11593|2|403| +2450815|11594|2|599| +2450815|11596|2|473| +2450815|11599|2|902| +2450815|11600|2|683| +2450815|11602|2|939| +2450815|11605|2|869| +2450815|11606|2|173| +2450815|11608|2|101| +2450815|11611|2|375| +2450815|11612|2|781| +2450815|11614|2|231| +2450815|11617|2|762| +2450815|11618|2|256| +2450815|11620|2|258| +2450815|11623|2|177| +2450815|11624|2|63| +2450815|11626|2|702| +2450815|11629|2|16| +2450815|11630|2|290| +2450815|11632|2|806| +2450815|11635|2|663| +2450815|11636|2|982| +2450815|11638|2|930| +2450815|11641|2|955| +2450815|11642|2|810| +2450815|11644|2|160| +2450815|11647|2|266| +2450815|11648|2|738| +2450815|11650|2|981| +2450815|11653|2|533| +2450815|11654|2|422| +2450815|11656|2|52| +2450815|11659|2|624| +2450815|11660|2|825| +2450815|11662|2|992| +2450815|11665|2|535| +2450815|11666|2|839| +2450815|11668|2|636| +2450815|11671|2|| +2450815|11672|2|702| +2450815|11674|2|601| +2450815|11677|2|227| +2450815|11678|2|970| +2450815|11680|2|996| +2450815|11683|2|139| +2450815|11684|2|777| +2450815|11686|2|944| +2450815|11689|2|555| +2450815|11690|2|357| +2450815|11692|2|191| +2450815|11695|2|854| +2450815|11696|2|766| +2450815|11698|2|469| +2450815|11701|2|817| +2450815|11702|2|89| +2450815|11704|2|760| +2450815|11707|2|212| +2450815|11708|2|18| +2450815|11710|2|495| +2450815|11713|2|872| +2450815|11714|2|537| +2450815|11716|2|792| +2450815|11719|2|855| +2450815|11720|2|883| +2450815|11722|2|227| +2450815|11725|2|759| +2450815|11726|2|298| +2450815|11728|2|689| +2450815|11731|2|27| +2450815|11732|2|552| +2450815|11734|2|185| +2450815|11737|2|86| +2450815|11738|2|242| +2450815|11740|2|500| +2450815|11743|2|367| +2450815|11744|2|798| +2450815|11746|2|59| +2450815|11749|2|414| +2450815|11750|2|261| +2450815|11752|2|874| +2450815|11755|2|783| +2450815|11756|2|410| +2450815|11758|2|218| +2450815|11761|2|66| +2450815|11762|2|939| +2450815|11764|2|83| +2450815|11767|2|674| +2450815|11768|2|255| +2450815|11770|2|630| +2450815|11773|2|561| +2450815|11774|2|997| +2450815|11776|2|992| +2450815|11779|2|233| +2450815|11780|2|895| +2450815|11782|2|197| +2450815|11785|2|59| +2450815|11786|2|893| +2450815|11788|2|510| +2450815|11791|2|532| +2450815|11792|2|18| +2450815|11794|2|197| +2450815|11797|2|537| +2450815|11798|2|428| +2450815|11800|2|558| +2450815|11803|2|717| +2450815|11804|2|530| +2450815|11806|2|491| +2450815|11809|2|877| +2450815|11810|2|189| +2450815|11812|2|557| +2450815|11815|2|188| +2450815|11816|2|157| +2450815|11818|2|378| +2450815|11821|2|223| +2450815|11822|2|822| +2450815|11824|2|221| +2450815|11827|2|165| +2450815|11828|2|279| +2450815|11830|2|665| +2450815|11833|2|996| +2450815|11834|2|592| +2450815|11836|2|841| +2450815|11839|2|957| +2450815|11840|2|849| +2450815|11842|2|825| +2450815|11845|2|688| +2450815|11846|2|15| +2450815|11848|2|348| +2450815|11851|2|598| +2450815|11852|2|385| +2450815|11854|2|959| +2450815|11857|2|646| +2450815|11858|2|127| +2450815|11860|2|734| +2450815|11863|2|661| +2450815|11864|2|139| +2450815|11866|2|539| +2450815|11869|2|166| +2450815|11870|2|623| +2450815|11872|2|661| +2450815|11875|2|980| +2450815|11876|2|721| +2450815|11878|2|861| +2450815|11881|2|989| +2450815|11882|2|| +2450815|11884|2|354| +2450815|11887|2|566| +2450815|11888|2|508| +2450815|11890|2|538| +2450815|11893|2|477| +2450815|11894|2|980| +2450815|11896|2|574| +2450815|11899|2|82| +2450815|11900|2|291| +2450815|11902|2|759| +2450815|11905|2|257| +2450815|11906|2|527| +2450815|11908|2|937| +2450815|11911|2|225| +2450815|11912|2|466| +2450815|11914|2|387| +2450815|11917|2|949| +2450815|11918|2|734| +2450815|11920|2|946| +2450815|11923|2|610| +2450815|11924|2|509| +2450815|11926|2|264| +2450815|11929|2|920| +2450815|11930|2|978| +2450815|11932|2|793| +2450815|11935|2|570| +2450815|11936|2|649| +2450815|11938|2|| +2450815|11941|2|103| +2450815|11942|2|954| +2450815|11944|2|84| +2450815|11947|2|490| +2450815|11948|2|571| +2450815|11950|2|844| +2450815|11953|2|620| +2450815|11954|2|| +2450815|11956|2|71| +2450815|11959|2|553| +2450815|11960|2|321| +2450815|11962|2|915| +2450815|11965|2|521| +2450815|11966|2|48| +2450815|11968|2|7| +2450815|11971|2|879| +2450815|11972|2|888| +2450815|11974|2|249| +2450815|11977|2|598| +2450815|11978|2|264| +2450815|11980|2|979| +2450815|11983|2|614| +2450815|11984|2|671| +2450815|11986|2|377| +2450815|11989|2|30| +2450815|11990|2|212| +2450815|11992|2|555| +2450815|11995|2|836| +2450815|11996|2|849| +2450815|11998|2|950| +2450815|12001|2|995| +2450815|12002|2|89| +2450815|12004|2|787| +2450815|12007|2|238| +2450815|12008|2|43| +2450815|12010|2|377| +2450815|12013|2|68| +2450815|12014|2|922| +2450815|12016|2|139| +2450815|12019|2|86| +2450815|12020|2|19| +2450815|12022|2|101| +2450815|12025|2|862| +2450815|12026|2|627| +2450815|12028|2|619| +2450815|12031|2|95| +2450815|12032|2|766| +2450815|12034|2|243| +2450815|12037|2|610| +2450815|12038|2|215| +2450815|12040|2|329| +2450815|12043|2|885| +2450815|12044|2|772| +2450815|12046|2|977| +2450815|12049|2|743| +2450815|12050|2|221| +2450815|12052|2|507| +2450815|12055|2|390| +2450815|12056|2|808| +2450815|12058|2|167| +2450815|12061|2|| +2450815|12062|2|735| +2450815|12064|2|261| +2450815|12067|2|197| +2450815|12068|2|266| +2450815|12070|2|414| +2450815|12073|2|24| +2450815|12074|2|117| +2450815|12076|2|| +2450815|12079|2|523| +2450815|12080|2|347| +2450815|12082|2|709| +2450815|12085|2|339| +2450815|12086|2|372| +2450815|12088|2|250| +2450815|12091|2|288| +2450815|12092|2|290| +2450815|12094|2|267| +2450815|12097|2|917| +2450815|12098|2|510| +2450815|12100|2|678| +2450815|12103|2|475| +2450815|12104|2|| +2450815|12106|2|707| +2450815|12109|2|222| +2450815|12110|2|433| +2450815|12112|2|924| +2450815|12115|2|713| +2450815|12116|2|776| +2450815|12118|2|610| +2450815|12121|2|84| +2450815|12122|2|| +2450815|12124|2|661| +2450815|12127|2|741| +2450815|12128|2|738| +2450815|12130|2|313| +2450815|12133|2|876| +2450815|12134|2|842| +2450815|12136|2|977| +2450815|12139|2|479| +2450815|12140|2|742| +2450815|12142|2|| +2450815|12145|2|| +2450815|12146|2|959| +2450815|12148|2|| +2450815|12151|2|210| +2450815|12152|2|195| +2450815|12154|2|757| +2450815|12157|2|851| +2450815|12158|2|353| +2450815|12160|2|160| +2450815|12163|2|637| +2450815|12164|2|| +2450815|12166|2|471| +2450815|12169|2|668| +2450815|12170|2|118| +2450815|12172|2|808| +2450815|12175|2|58| +2450815|12176|2|720| +2450815|12178|2|844| +2450815|12181|2|966| +2450815|12182|2|724| +2450815|12184|2|349| +2450815|12187|2|605| +2450815|12188|2|388| +2450815|12190|2|60| +2450815|12193|2|891| +2450815|12194|2|617| +2450815|12196|2|301| +2450815|12199|2|311| +2450815|12200|2|593| +2450815|12202|2|380| +2450815|12205|2|549| +2450815|12206|2|685| +2450815|12208|2|325| +2450815|12211|2|| +2450815|12212|2|525| +2450815|12214|2|234| +2450815|12217|2|739| +2450815|12218|2|747| +2450815|12220|2|734| +2450815|12223|2|885| +2450815|12224|2|324| +2450815|12226|2|513| +2450815|12229|2|956| +2450815|12230|2|729| +2450815|12232|2|| +2450815|12235|2|97| +2450815|12236|2|534| +2450815|12238|2|891| +2450815|12241|2|79| +2450815|12242|2|189| +2450815|12244|2|139| +2450815|12247|2|711| +2450815|12248|2|843| +2450815|12250|2|179| +2450815|12253|2|693| +2450815|12254|2|579| +2450815|12256|2|814| +2450815|12259|2|873| +2450815|12260|2|548| +2450815|12262|2|96| +2450815|12265|2|| +2450815|12266|2|751| +2450815|12268|2|384| +2450815|12271|2|290| +2450815|12272|2|621| +2450815|12274|2|722| +2450815|12277|2|395| +2450815|12278|2|432| +2450815|12280|2|295| +2450815|12283|2|353| +2450815|12284|2|252| +2450815|12286|2|80| +2450815|12289|2|499| +2450815|12290|2|310| +2450815|12292|2|424| +2450815|12295|2|622| +2450815|12296|2|6| +2450815|12298|2|563| +2450815|12301|2|220| +2450815|12302|2|429| +2450815|12304|2|914| +2450815|12307|2|810| +2450815|12308|2|681| +2450815|12310|2|465| +2450815|12313|2|133| +2450815|12314|2|710| +2450815|12316|2|461| +2450815|12319|2|793| +2450815|12320|2|353| +2450815|12322|2|886| +2450815|12325|2|802| +2450815|12326|2|| +2450815|12328|2|32| +2450815|12331|2|444| +2450815|12332|2|| +2450815|12334|2|553| +2450815|12337|2|756| +2450815|12338|2|509| +2450815|12340|2|609| +2450815|12343|2|333| +2450815|12344|2|81| +2450815|12346|2|| +2450815|12349|2|770| +2450815|12350|2|159| +2450815|12352|2|793| +2450815|12355|2|596| +2450815|12356|2|445| +2450815|12358|2|27| +2450815|12361|2|511| +2450815|12362|2|232| +2450815|12364|2|57| +2450815|12367|2|360| +2450815|12368|2|715| +2450815|12370|2|688| +2450815|12373|2|293| +2450815|12374|2|785| +2450815|12376|2|266| +2450815|12379|2|482| +2450815|12380|2|918| +2450815|12382|2|670| +2450815|12385|2|639| +2450815|12386|2|873| +2450815|12388|2|234| +2450815|12391|2|102| +2450815|12392|2|72| +2450815|12394|2|648| +2450815|12397|2|716| +2450815|12398|2|844| +2450815|12400|2|804| +2450815|12403|2|350| +2450815|12404|2|483| +2450815|12406|2|341| +2450815|12409|2|724| +2450815|12410|2|976| +2450815|12412|2|58| +2450815|12415|2|285| +2450815|12416|2|103| +2450815|12418|2|23| +2450815|12421|2|6| +2450815|12422|2|692| +2450815|12424|2|377| +2450815|12427|2|314| +2450815|12428|2|594| +2450815|12430|2|202| +2450815|12433|2|| +2450815|12434|2|685| +2450815|12436|2|55| +2450815|12439|2|528| +2450815|12440|2|893| +2450815|12442|2|117| +2450815|12445|2|56| +2450815|12446|2|521| +2450815|12448|2|480| +2450815|12451|2|172| +2450815|12452|2|515| +2450815|12454|2|| +2450815|12457|2|771| +2450815|12458|2|855| +2450815|12460|2|617| +2450815|12463|2|804| +2450815|12464|2|492| +2450815|12466|2|945| +2450815|12469|2|659| +2450815|12470|2|385| +2450815|12472|2|908| +2450815|12475|2|621| +2450815|12476|2|399| +2450815|12478|2|711| +2450815|12481|2|822| +2450815|12482|2|319| +2450815|12484|2|| +2450815|12487|2|913| +2450815|12488|2|902| +2450815|12490|2|| +2450815|12493|2|349| +2450815|12494|2|855| +2450815|12496|2|153| +2450815|12499|2|84| +2450815|12500|2|531| +2450815|12502|2|617| +2450815|12505|2|137| +2450815|12506|2|906| +2450815|12508|2|580| +2450815|12511|2|53| +2450815|12512|2|845| +2450815|12514|2|19| +2450815|12517|2|956| +2450815|12518|2|532| +2450815|12520|2|828| +2450815|12523|2|201| +2450815|12524|2|38| +2450815|12526|2|848| +2450815|12529|2|381| +2450815|12530|2|| +2450815|12532|2|852| +2450815|12535|2|543| +2450815|12536|2|359| +2450815|12538|2|177| +2450815|12541|2|176| +2450815|12542|2|874| +2450815|12544|2|809| +2450815|12547|2|749| +2450815|12548|2|237| +2450815|12550|2|455| +2450815|12553|2|943| +2450815|12554|2|989| +2450815|12556|2|277| +2450815|12559|2|395| +2450815|12560|2|149| +2450815|12562|2|| +2450815|12565|2|401| +2450815|12566|2|130| +2450815|12568|2|761| +2450815|12571|2|64| +2450815|12572|2|159| +2450815|12574|2|338| +2450815|12577|2|667| +2450815|12578|2|291| +2450815|12580|2|770| +2450815|12583|2|339| +2450815|12584|2|33| +2450815|12586|2|614| +2450815|12589|2|556| +2450815|12590|2|405| +2450815|12592|2|632| +2450815|12595|2|783| +2450815|12596|2|848| +2450815|12598|2|618| +2450815|12601|2|223| +2450815|12602|2|855| +2450815|12604|2|249| +2450815|12607|2|247| +2450815|12608|2|688| +2450815|12610|2|115| +2450815|12613|2|601| +2450815|12614|2|430| +2450815|12616|2|483| +2450815|12619|2|900| +2450815|12620|2|778| +2450815|12622|2|688| +2450815|12625|2|688| +2450815|12626|2|159| +2450815|12628|2|529| +2450815|12631|2|606| +2450815|12632|2|20| +2450815|12634|2|924| +2450815|12637|2|831| +2450815|12638|2|506| +2450815|12640|2|824| +2450815|12643|2|758| +2450815|12644|2|182| +2450815|12646|2|753| +2450815|12649|2|347| +2450815|12650|2|112| +2450815|12652|2|455| +2450815|12655|2|842| +2450815|12656|2|335| +2450815|12658|2|93| +2450815|12661|2|297| +2450815|12662|2|507| +2450815|12664|2|824| +2450815|12667|2|542| +2450815|12668|2|295| +2450815|12670|2|866| +2450815|12673|2|125| +2450815|12674|2|192| +2450815|12676|2|595| +2450815|12679|2|18| +2450815|12680|2|| +2450815|12682|2|146| +2450815|12685|2|936| +2450815|12686|2|973| +2450815|12688|2|603| +2450815|12691|2|127| +2450815|12692|2|749| +2450815|12694|2|540| +2450815|12697|2|922| +2450815|12698|2|36| +2450815|12700|2|194| +2450815|12703|2|284| +2450815|12704|2|124| +2450815|12706|2|229| +2450815|12709|2|943| +2450815|12710|2|403| +2450815|12712|2|95| +2450815|12715|2|357| +2450815|12716|2|229| +2450815|12718|2|119| +2450815|12721|2|660| +2450815|12722|2|126| +2450815|12724|2|865| +2450815|12727|2|948| +2450815|12728|2|243| +2450815|12730|2|839| +2450815|12733|2|928| +2450815|12734|2|948| +2450815|12736|2|639| +2450815|12739|2|603| +2450815|12740|2|396| +2450815|12742|2|968| +2450815|12745|2|931| +2450815|12746|2|193| +2450815|12748|2|984| +2450815|12751|2|612| +2450815|12752|2|| +2450815|12754|2|184| +2450815|12757|2|699| +2450815|12758|2|521| +2450815|12760|2|202| +2450815|12763|2|129| +2450815|12764|2|691| +2450815|12766|2|865| +2450815|12769|2|723| +2450815|12770|2|284| +2450815|12772|2|970| +2450815|12775|2|923| +2450815|12776|2|72| +2450815|12778|2|413| +2450815|12781|2|946| +2450815|12782|2|458| +2450815|12784|2|655| +2450815|12787|2|953| +2450815|12788|2|358| +2450815|12790|2|544| +2450815|12793|2|981| +2450815|12794|2|855| +2450815|12796|2|438| +2450815|12799|2|142| +2450815|12800|2|613| +2450815|12802|2|108| +2450815|12805|2|794| +2450815|12806|2|823| +2450815|12808|2|768| +2450815|12811|2|726| +2450815|12812|2|669| +2450815|12814|2|| +2450815|12817|2|908| +2450815|12818|2|6| +2450815|12820|2|983| +2450815|12823|2|617| +2450815|12824|2|102| +2450815|12826|2|881| +2450815|12829|2|1000| +2450815|12830|2|189| +2450815|12832|2|860| +2450815|12835|2|562| +2450815|12836|2|263| +2450815|12838|2|220| +2450815|12841|2|843| +2450815|12842|2|738| +2450815|12844|2|642| +2450815|12847|2|730| +2450815|12848|2|294| +2450815|12850|2|80| +2450815|12853|2|637| +2450815|12854|2|| +2450815|12856|2|421| +2450815|12859|2|112| +2450815|12860|2|662| +2450815|12862|2|330| +2450815|12865|2|773| +2450815|12866|2|| +2450815|12868|2|331| +2450815|12871|2|248| +2450815|12872|2|987| +2450815|12874|2|368| +2450815|12877|2|86| +2450815|12878|2|309| +2450815|12880|2|367| +2450815|12883|2|512| +2450815|12884|2|759| +2450815|12886|2|530| +2450815|12889|2|771| +2450815|12890|2|186| +2450815|12892|2|444| +2450815|12895|2|667| +2450815|12896|2|312| +2450815|12898|2|676| +2450815|12901|2|932| +2450815|12902|2|64| +2450815|12904|2|111| +2450815|12907|2|572| +2450815|12908|2|641| +2450815|12910|2|917| +2450815|12913|2|645| +2450815|12914|2|759| +2450815|12916|2|532| +2450815|12919|2|288| +2450815|12920|2|68| +2450815|12922|2|237| +2450815|12925|2|145| +2450815|12926|2|365| +2450815|12928|2|433| +2450815|12931|2|980| +2450815|12932|2|268| +2450815|12934|2|973| +2450815|12937|2|348| +2450815|12938|2|535| +2450815|12940|2|943| +2450815|12943|2|530| +2450815|12944|2|449| +2450815|12946|2|936| +2450815|12949|2|884| +2450815|12950|2|184| +2450815|12952|2|492| +2450815|12955|2|230| +2450815|12956|2|361| +2450815|12958|2|331| +2450815|12961|2|413| +2450815|12962|2|986| +2450815|12964|2|823| +2450815|12967|2|503| +2450815|12968|2|731| +2450815|12970|2|234| +2450815|12973|2|24| +2450815|12974|2|434| +2450815|12976|2|992| +2450815|12979|2|281| +2450815|12980|2|472| +2450815|12982|2|500| +2450815|12985|2|650| +2450815|12986|2|105| +2450815|12988|2|967| +2450815|12991|2|167| +2450815|12992|2|659| +2450815|12994|2|695| +2450815|12997|2|262| +2450815|12998|2|924| +2450815|13000|2|604| +2450815|13003|2|495| +2450815|13004|2|92| +2450815|13006|2|545| +2450815|13009|2|538| +2450815|13010|2|297| +2450815|13012|2|966| +2450815|13015|2|879| +2450815|13016|2|281| +2450815|13018|2|89| +2450815|13021|2|217| +2450815|13022|2|915| +2450815|13024|2|793| +2450815|13027|2|578| +2450815|13028|2|832| +2450815|13030|2|54| +2450815|13033|2|439| +2450815|13034|2|870| +2450815|13036|2|397| +2450815|13039|2|164| +2450815|13040|2|876| +2450815|13042|2|433| +2450815|13045|2|243| +2450815|13046|2|55| +2450815|13048|2|736| +2450815|13051|2|117| +2450815|13052|2|768| +2450815|13054|2|145| +2450815|13057|2|495| +2450815|13058|2|957| +2450815|13060|2|910| +2450815|13063|2|37| +2450815|13064|2|289| +2450815|13066|2|322| +2450815|13069|2|72| +2450815|13070|2|338| +2450815|13072|2|522| +2450815|13075|2|243| +2450815|13076|2|399| +2450815|13078|2|44| +2450815|13081|2|375| +2450815|13082|2|408| +2450815|13084|2|414| +2450815|13087|2|852| +2450815|13088|2|453| +2450815|13090|2|939| +2450815|13093|2|259| +2450815|13094|2|441| +2450815|13096|2|850| +2450815|13099|2|509| +2450815|13100|2|| +2450815|13102|2|17| +2450815|13105|2|778| +2450815|13106|2|772| +2450815|13108|2|379| +2450815|13111|2|263| +2450815|13112|2|| +2450815|13114|2|| +2450815|13117|2|722| +2450815|13118|2|12| +2450815|13120|2|241| +2450815|13123|2|611| +2450815|13124|2|938| +2450815|13126|2|687| +2450815|13129|2|554| +2450815|13130|2|562| +2450815|13132|2|782| +2450815|13135|2|669| +2450815|13136|2|345| +2450815|13138|2|771| +2450815|13141|2|415| +2450815|13142|2|| +2450815|13144|2|667| +2450815|13147|2|350| +2450815|13148|2|716| +2450815|13150|2|483| +2450815|13153|2|660| +2450815|13154|2|57| +2450815|13156|2|554| +2450815|13159|2|33| +2450815|13160|2|741| +2450815|13162|2|441| +2450815|13165|2|822| +2450815|13166|2|| +2450815|13168|2|304| +2450815|13171|2|767| +2450815|13172|2|837| +2450815|13174|2|969| +2450815|13177|2|948| +2450815|13178|2|577| +2450815|13180|2|932| +2450815|13183|2|444| +2450815|13184|2|591| +2450815|13186|2|869| +2450815|13189|2|301| +2450815|13190|2|993| +2450815|13192|2|175| +2450815|13195|2|180| +2450815|13196|2|211| +2450815|13198|2|945| +2450815|13201|2|65| +2450815|13202|2|67| +2450815|13204|2|992| +2450815|13207|2|543| +2450815|13208|2|315| +2450815|13210|2|584| +2450815|13213|2|30| +2450815|13214|2|297| +2450815|13216|2|345| +2450815|13219|2|| +2450815|13220|2|266| +2450815|13222|2|438| +2450815|13225|2|938| +2450815|13226|2|| +2450815|13228|2|278| +2450815|13231|2|972| +2450815|13232|2|737| +2450815|13234|2|908| +2450815|13237|2|891| +2450815|13238|2|769| +2450815|13240|2|817| +2450815|13243|2|14| +2450815|13244|2|321| +2450815|13246|2|821| +2450815|13249|2|351| +2450815|13250|2|886| +2450815|13252|2|744| +2450815|13255|2|424| +2450815|13256|2|805| +2450815|13258|2|540| +2450815|13261|2|168| +2450815|13262|2|639| +2450815|13264|2|364| +2450815|13267|2|571| +2450815|13268|2|528| +2450815|13270|2|832| +2450815|13273|2|619| +2450815|13274|2|494| +2450815|13276|2|238| +2450815|13279|2|521| +2450815|13280|2|113| +2450815|13282|2|61| +2450815|13285|2|163| +2450815|13286|2|313| +2450815|13288|2|909| +2450815|13291|2|| +2450815|13292|2|319| +2450815|13294|2|452| +2450815|13297|2|204| +2450815|13298|2|858| +2450815|13300|2|327| +2450815|13303|2|161| +2450815|13304|2|331| +2450815|13306|2|544| +2450815|13309|2|984| +2450815|13310|2|181| +2450815|13312|2|657| +2450815|13315|2|189| +2450815|13316|2|603| +2450815|13318|2|542| +2450815|13321|2|910| +2450815|13322|2|419| +2450815|13324|2|621| +2450815|13327|2|| +2450815|13328|2|569| +2450815|13330|2|671| +2450815|13333|2|128| +2450815|13334|2|486| +2450815|13336|2|636| +2450815|13339|2|253| +2450815|13340|2|| +2450815|13342|2|570| +2450815|13345|2|671| +2450815|13346|2|| +2450815|13348|2|701| +2450815|13351|2|354| +2450815|13352|2|94| +2450815|13354|2|679| +2450815|13357|2|154| +2450815|13358|2|116| +2450815|13360|2|812| +2450815|13363|2|736| +2450815|13364|2|940| +2450815|13366|2|807| +2450815|13369|2|73| +2450815|13370|2|978| +2450815|13372|2|488| +2450815|13375|2|524| +2450815|13376|2|812| +2450815|13378|2|550| +2450815|13381|2|419| +2450815|13382|2|194| +2450815|13384|2|862| +2450815|13387|2|740| +2450815|13388|2|| +2450815|13390|2|977| +2450815|13393|2|323| +2450815|13394|2|35| +2450815|13396|2|797| +2450815|13399|2|376| +2450815|13400|2|503| +2450815|13402|2|481| +2450815|13405|2|142| +2450815|13406|2|576| +2450815|13408|2|449| +2450815|13411|2|879| +2450815|13412|2|391| +2450815|13414|2|485| +2450815|13417|2|700| +2450815|13418|2|5| +2450815|13420|2|625| +2450815|13423|2|146| +2450815|13424|2|773| +2450815|13426|2|518| +2450815|13429|2|962| +2450815|13430|2|751| +2450815|13432|2|86| +2450815|13435|2|460| +2450815|13436|2|88| +2450815|13438|2|727| +2450815|13441|2|601| +2450815|13442|2|280| +2450815|13444|2|110| +2450815|13447|2|438| +2450815|13448|2|981| +2450815|13450|2|531| +2450815|13453|2|913| +2450815|13454|2|925| +2450815|13456|2|880| +2450815|13459|2|| +2450815|13460|2|422| +2450815|13462|2|229| +2450815|13465|2|271| +2450815|13466|2|276| +2450815|13468|2|885| +2450815|13471|2|226| +2450815|13472|2|503| +2450815|13474|2|46| +2450815|13477|2|121| +2450815|13478|2|833| +2450815|13480|2|| +2450815|13483|2|899| +2450815|13484|2|53| +2450815|13486|2|275| +2450815|13489|2|605| +2450815|13490|2|848| +2450815|13492|2|| +2450815|13495|2|566| +2450815|13496|2|409| +2450815|13498|2|532| +2450815|13501|2|846| +2450815|13502|2|833| +2450815|13504|2|303| +2450815|13507|2|655| +2450815|13508|2|74| +2450815|13510|2|824| +2450815|13513|2|348| +2450815|13514|2|277| +2450815|13516|2|| +2450815|13519|2|279| +2450815|13520|2|490| +2450815|13522|2|473| +2450815|13525|2|834| +2450815|13526|2|456| +2450815|13528|2|602| +2450815|13531|2|273| +2450815|13532|2|828| +2450815|13534|2|508| +2450815|13537|2|126| +2450815|13538|2|710| +2450815|13540|2|712| +2450815|13543|2|637| +2450815|13544|2|613| +2450815|13546|2|573| +2450815|13549|2|740| +2450815|13550|2|770| +2450815|13552|2|773| +2450815|13555|2|421| +2450815|13556|2|| +2450815|13558|2|266| +2450815|13561|2|239| +2450815|13562|2|366| +2450815|13564|2|182| +2450815|13567|2|1000| +2450815|13568|2|211| +2450815|13570|2|891| +2450815|13573|2|434| +2450815|13574|2|134| +2450815|13576|2|274| +2450815|13579|2|147| +2450815|13580|2|231| +2450815|13582|2|705| +2450815|13585|2|374| +2450815|13586|2|532| +2450815|13588|2|117| +2450815|13591|2|720| +2450815|13592|2|318| +2450815|13594|2|290| +2450815|13597|2|760| +2450815|13598|2|| +2450815|13600|2|764| +2450815|13603|2|834| +2450815|13604|2|506| +2450815|13606|2|339| +2450815|13609|2|| +2450815|13610|2|401| +2450815|13612|2|270| +2450815|13615|2|79| +2450815|13616|2|397| +2450815|13618|2|577| +2450815|13621|2|912| +2450815|13622|2|82| +2450815|13624|2|1000| +2450815|13627|2|77| +2450815|13628|2|861| +2450815|13630|2|132| +2450815|13633|2|249| +2450815|13634|2|| +2450815|13636|2|398| +2450815|13639|2|403| +2450815|13640|2|715| +2450815|13642|2|296| +2450815|13645|2|303| +2450815|13646|2|724| +2450815|13648|2|431| +2450815|13651|2|| +2450815|13652|2|38| +2450815|13654|2|874| +2450815|13657|2|736| +2450815|13658|2|750| +2450815|13660|2|124| +2450815|13663|2|328| +2450815|13664|2|631| +2450815|13666|2|179| +2450815|13669|2|630| +2450815|13670|2|40| +2450815|13672|2|675| +2450815|13675|2|610| +2450815|13676|2|10| +2450815|13678|2|187| +2450815|13681|2|127| +2450815|13682|2|241| +2450815|13684|2|869| +2450815|13687|2|383| +2450815|13688|2|900| +2450815|13690|2|192| +2450815|13693|2|58| +2450815|13694|2|178| +2450815|13696|2|| +2450815|13699|2|16| +2450815|13700|2|671| +2450815|13702|2|227| +2450815|13705|2|331| +2450815|13706|2|459| +2450815|13708|2|331| +2450815|13711|2|657| +2450815|13712|2|15| +2450815|13714|2|414| +2450815|13717|2|141| +2450815|13718|2|947| +2450815|13720|2|136| +2450815|13723|2|651| +2450815|13724|2|415| +2450815|13726|2|178| +2450815|13729|2|572| +2450815|13730|2|550| +2450815|13732|2|| +2450815|13735|2|720| +2450815|13736|2|599| +2450815|13738|2|37| +2450815|13741|2|276| +2450815|13742|2|138| +2450815|13744|2|522| +2450815|13747|2|571| +2450815|13748|2|604| +2450815|13750|2|712| +2450815|13753|2|80| +2450815|13754|2|231| +2450815|13756|2|17| +2450815|13759|2|636| +2450815|13760|2|470| +2450815|13762|2|| +2450815|13765|2|766| +2450815|13766|2|794| +2450815|13768|2|633| +2450815|13771|2|354| +2450815|13772|2|954| +2450815|13774|2|475| +2450815|13777|2|194| +2450815|13778|2|645| +2450815|13780|2|728| +2450815|13783|2|35| +2450815|13784|2|387| +2450815|13786|2|356| +2450815|13789|2|895| +2450815|13790|2|841| +2450815|13792|2|45| +2450815|13795|2|593| +2450815|13796|2|910| +2450815|13798|2|300| +2450815|13801|2|255| +2450815|13802|2|923| +2450815|13804|2|396| +2450815|13807|2|525| +2450815|13808|2|717| +2450815|13810|2|898| +2450815|13813|2|176| +2450815|13814|2|216| +2450815|13816|2|672| +2450815|13819|2|894| +2450815|13820|2|858| +2450815|13822|2|308| +2450815|13825|2|816| +2450815|13826|2|932| +2450815|13828|2|267| +2450815|13831|2|693| +2450815|13832|2|711| +2450815|13834|2|945| +2450815|13837|2|691| +2450815|13838|2|377| +2450815|13840|2|741| +2450815|13843|2|588| +2450815|13844|2|473| +2450815|13846|2|471| +2450815|13849|2|612| +2450815|13850|2|945| +2450815|13852|2|618| +2450815|13855|2|11| +2450815|13856|2|54| +2450815|13858|2|177| +2450815|13861|2|694| +2450815|13862|2|206| +2450815|13864|2|287| +2450815|13867|2|981| +2450815|13868|2|990| +2450815|13870|2|2| +2450815|13873|2|380| +2450815|13874|2|148| +2450815|13876|2|754| +2450815|13879|2|619| +2450815|13880|2|777| +2450815|13882|2|766| +2450815|13885|2|316| +2450815|13886|2|211| +2450815|13888|2|665| +2450815|13891|2|702| +2450815|13892|2|850| +2450815|13894|2|779| +2450815|13897|2|363| +2450815|13898|2|571| +2450815|13900|2|503| +2450815|13903|2|918| +2450815|13904|2|682| +2450815|13906|2|735| +2450815|13909|2|932| +2450815|13910|2|224| +2450815|13912|2|238| +2450815|13915|2|887| +2450815|13916|2|115| +2450815|13918|2|488| +2450815|13921|2|308| +2450815|13922|2|537| +2450815|13924|2|634| +2450815|13927|2|558| +2450815|13928|2|427| +2450815|13930|2|555| +2450815|13933|2|754| +2450815|13934|2|976| +2450815|13936|2|302| +2450815|13939|2|858| +2450815|13940|2|566| +2450815|13942|2|647| +2450815|13945|2|714| +2450815|13946|2|| +2450815|13948|2|176| +2450815|13951|2|130| +2450815|13952|2|94| +2450815|13954|2|970| +2450815|13957|2|90| +2450815|13958|2|758| +2450815|13960|2|347| +2450815|13963|2|162| +2450815|13964|2|539| +2450815|13966|2|545| +2450815|13969|2|966| +2450815|13970|2|685| +2450815|13972|2|744| +2450815|13975|2|777| +2450815|13976|2|809| +2450815|13978|2|545| +2450815|13981|2|544| +2450815|13982|2|1000| +2450815|13984|2|407| +2450815|13987|2|524| +2450815|13988|2|895| +2450815|13990|2|649| +2450815|13993|2|151| +2450815|13994|2|105| +2450815|13996|2|147| +2450815|13999|2|165| +2450815|14000|2|424| +2450815|14002|2|| +2450815|14005|2|950| +2450815|14006|2|722| +2450815|14008|2|484| +2450815|14011|2|709| +2450815|14012|2|421| +2450815|14014|2|| +2450815|14017|2|695| +2450815|14018|2|973| +2450815|14020|2|897| +2450815|14023|2|390| +2450815|14024|2|731| +2450815|14026|2|214| +2450815|14029|2|838| +2450815|14030|2|720| +2450815|14032|2|826| +2450815|14035|2|114| +2450815|14036|2|64| +2450815|14038|2|468| +2450815|14041|2|273| +2450815|14042|2|414| +2450815|14044|2|340| +2450815|14047|2|| +2450815|14048|2|577| +2450815|14050|2|| +2450815|14053|2|595| +2450815|14054|2|| +2450815|14056|2|359| +2450815|14059|2|640| +2450815|14060|2|987| +2450815|14062|2|258| +2450815|14065|2|241| +2450815|14066|2|17| +2450815|14068|2|615| +2450815|14071|2|174| +2450815|14072|2|915| +2450815|14074|2|465| +2450815|14077|2|784| +2450815|14078|2|875| +2450815|14080|2|98| +2450815|14083|2|809| +2450815|14084|2|65| +2450815|14086|2|907| +2450815|14089|2|940| +2450815|14090|2|132| +2450815|14092|2|21| +2450815|14095|2|865| +2450815|14096|2|718| +2450815|14098|2|693| +2450815|14101|2|728| +2450815|14102|2|431| +2450815|14104|2|483| +2450815|14107|2|409| +2450815|14108|2|905| +2450815|14110|2|849| +2450815|14113|2|207| +2450815|14114|2|136| +2450815|14116|2|926| +2450815|14119|2|327| +2450815|14120|2|644| +2450815|14122|2|797| +2450815|14125|2|699| +2450815|14126|2|180| +2450815|14128|2|583| +2450815|14131|2|845| +2450815|14132|2|645| +2450815|14134|2|618| +2450815|14137|2|907| +2450815|14138|2|567| +2450815|14140|2|260| +2450815|14143|2|669| +2450815|14144|2|616| +2450815|14146|2|482| +2450815|14149|2|446| +2450815|14150|2|| +2450815|14152|2|392| +2450815|14155|2|905| +2450815|14156|2|14| +2450815|14158|2|387| +2450815|14161|2|868| +2450815|14162|2|403| +2450815|14164|2|94| +2450815|14167|2|132| +2450815|14168|2|390| +2450815|14170|2|812| +2450815|14173|2|462| +2450815|14174|2|153| +2450815|14176|2|42| +2450815|14179|2|855| +2450815|14180|2|476| +2450815|14182|2|7| +2450815|14185|2|127| +2450815|14186|2|825| +2450815|14188|2|138| +2450815|14191|2|571| +2450815|14192|2|16| +2450815|14194|2|179| +2450815|14197|2|705| +2450815|14198|2|346| +2450815|14200|2|536| +2450815|14203|2|497| +2450815|14204|2|997| +2450815|14206|2|557| +2450815|14209|2|283| +2450815|14210|2|235| +2450815|14212|2|302| +2450815|14215|2|390| +2450815|14216|2|155| +2450815|14218|2|435| +2450815|14221|2|577| +2450815|14222|2|556| +2450815|14224|2|472| +2450815|14227|2|497| +2450815|14228|2|139| +2450815|14230|2|174| +2450815|14233|2|39| +2450815|14234|2|164| +2450815|14236|2|| +2450815|14239|2|897| +2450815|14240|2|982| +2450815|14242|2|785| +2450815|14245|2|959| +2450815|14246|2|330| +2450815|14248|2|380| +2450815|14251|2|794| +2450815|14252|2|668| +2450815|14254|2|215| +2450815|14257|2|25| +2450815|14258|2|| +2450815|14260|2|763| +2450815|14263|2|26| +2450815|14264|2|804| +2450815|14266|2|650| +2450815|14269|2|388| +2450815|14270|2|38| +2450815|14272|2|566| +2450815|14275|2|151| +2450815|14276|2|497| +2450815|14278|2|370| +2450815|14281|2|210| +2450815|14282|2|896| +2450815|14284|2|852| +2450815|14287|2|872| +2450815|14288|2|160| +2450815|14290|2|189| +2450815|14293|2|684| +2450815|14294|2|963| +2450815|14296|2|889| +2450815|14299|2|638| +2450815|14300|2|397| +2450815|14302|2|88| +2450815|14305|2|231| +2450815|14306|2|171| +2450815|14308|2|384| +2450815|14311|2|872| +2450815|14312|2|643| +2450815|14314|2|954| +2450815|14317|2|161| +2450815|14318|2|572| +2450815|14320|2|664| +2450815|14323|2|79| +2450815|14324|2|630| +2450815|14326|2|572| +2450815|14329|2|311| +2450815|14330|2|658| +2450815|14332|2|938| +2450815|14335|2|606| +2450815|14336|2|183| +2450815|14338|2|855| +2450815|14341|2|228| +2450815|14342|2|923| +2450815|14344|2|194| +2450815|14347|2|61| +2450815|14348|2|349| +2450815|14350|2|160| +2450815|14353|2|371| +2450815|14354|2|602| +2450815|14356|2|488| +2450815|14359|2|917| +2450815|14360|2|269| +2450815|14362|2|632| +2450815|14365|2|323| +2450815|14366|2|394| +2450815|14368|2|450| +2450815|14371|2|585| +2450815|14372|2|968| +2450815|14374|2|500| +2450815|14377|2|811| +2450815|14378|2|875| +2450815|14380|2|346| +2450815|14383|2|578| +2450815|14384|2|916| +2450815|14386|2|869| +2450815|14389|2|960| +2450815|14390|2|71| +2450815|14392|2|332| +2450815|14395|2|215| +2450815|14396|2|568| +2450815|14398|2|| +2450815|14401|2|260| +2450815|14402|2|| +2450815|14404|2|789| +2450815|14407|2|106| +2450815|14408|2|838| +2450815|14410|2|399| +2450815|14413|2|119| +2450815|14414|2|571| +2450815|14416|2|954| +2450815|14419|2|627| +2450815|14420|2|506| +2450815|14422|2|58| +2450815|14425|2|655| +2450815|14426|2|464| +2450815|14428|2|353| +2450815|14431|2|988| +2450815|14432|2|437| +2450815|14434|2|679| +2450815|14437|2|244| +2450815|14438|2|998| +2450815|14440|2|766| +2450815|14443|2|| +2450815|14444|2|287| +2450815|14446|2|107| +2450815|14449|2|562| +2450815|14450|2|473| +2450815|14452|2|337| +2450815|14455|2|53| +2450815|14456|2|958| +2450815|14458|2|85| +2450815|14461|2|42| +2450815|14462|2|497| +2450815|14464|2|30| +2450815|14467|2|459| +2450815|14468|2|92| +2450815|14470|2|455| +2450815|14473|2|10| +2450815|14474|2|2| +2450815|14476|2|180| +2450815|14479|2|935| +2450815|14480|2|126| +2450815|14482|2|646| +2450815|14485|2|984| +2450815|14486|2|66| +2450815|14488|2|| +2450815|14491|2|742| +2450815|14492|2|957| +2450815|14494|2|740| +2450815|14497|2|682| +2450815|14498|2|674| +2450815|14500|2|| +2450815|14503|2|582| +2450815|14504|2|234| +2450815|14506|2|216| +2450815|14509|2|| +2450815|14510|2|183| +2450815|14512|2|263| +2450815|14515|2|985| +2450815|14516|2|773| +2450815|14518|2|226| +2450815|14521|2|| +2450815|14522|2|282| +2450815|14524|2|618| +2450815|14527|2|13| +2450815|14528|2|712| +2450815|14530|2|939| +2450815|14533|2|596| +2450815|14534|2|259| +2450815|14536|2|871| +2450815|14539|2|905| +2450815|14540|2|859| +2450815|14542|2|635| +2450815|14545|2|6| +2450815|14546|2|868| +2450815|14548|2|720| +2450815|14551|2|474| +2450815|14552|2|39| +2450815|14554|2|948| +2450815|14557|2|307| +2450815|14558|2|291| +2450815|14560|2|575| +2450815|14563|2|316| +2450815|14564|2|338| +2450815|14566|2|971| +2450815|14569|2|73| +2450815|14570|2|513| +2450815|14572|2|977| +2450815|14575|2|561| +2450815|14576|2|838| +2450815|14578|2|94| +2450815|14581|2|77| +2450815|14582|2|249| +2450815|14584|2|414| +2450815|14587|2|59| +2450815|14588|2|120| +2450815|14590|2|502| +2450815|14593|2|42| +2450815|14594|2|269| +2450815|14596|2|97| +2450815|14599|2|641| +2450815|14600|2|115| +2450815|14602|2|406| +2450815|14605|2|1000| +2450815|14606|2|938| +2450815|14608|2|821| +2450815|14611|2|141| +2450815|14612|2|572| +2450815|14614|2|502| +2450815|14617|2|978| +2450815|14618|2|719| +2450815|14620|2|867| +2450815|14623|2|639| +2450815|14624|2|121| +2450815|14626|2|756| +2450815|14629|2|| +2450815|14630|2|426| +2450815|14632|2|| +2450815|14635|2|246| +2450815|14636|2|947| +2450815|14638|2|79| +2450815|14641|2|1000| +2450815|14642|2|85| +2450815|14644|2|174| +2450815|14647|2|739| +2450815|14648|2|432| +2450815|14650|2|76| +2450815|14653|2|774| +2450815|14654|2|916| +2450815|14656|2|423| +2450815|14659|2|933| +2450815|14660|2|272| +2450815|14662|2|666| +2450815|14665|2|| +2450815|14666|2|677| +2450815|14668|2|290| +2450815|14671|2|751| +2450815|14672|2|664| +2450815|14674|2|508| +2450815|14677|2|958| +2450815|14678|2|161| +2450815|14680|2|95| +2450815|14683|2|755| +2450815|14684|2|273| +2450815|14686|2|76| +2450815|14689|2|841| +2450815|14690|2|623| +2450815|14692|2|672| +2450815|14695|2|863| +2450815|14696|2|72| +2450815|14698|2|719| +2450815|14701|2|423| +2450815|14702|2|406| +2450815|14704|2|64| +2450815|14707|2|694| +2450815|14708|2|844| +2450815|14710|2|625| +2450815|14713|2|256| +2450815|14714|2|803| +2450815|14716|2|716| +2450815|14719|2|365| +2450815|14720|2|517| +2450815|14722|2|248| +2450815|14725|2|760| +2450815|14726|2|702| +2450815|14728|2|| +2450815|14731|2|304| +2450815|14732|2|705| +2450815|14734|2|954| +2450815|14737|2|227| +2450815|14738|2|113| +2450815|14740|2|324| +2450815|14743|2|| +2450815|14744|2|993| +2450815|14746|2|749| +2450815|14749|2|243| +2450815|14750|2|482| +2450815|14752|2|498| +2450815|14755|2|129| +2450815|14756|2|156| +2450815|14758|2|103| +2450815|14761|2|744| +2450815|14762|2|954| +2450815|14764|2|726| +2450815|14767|2|| +2450815|14768|2|489| +2450815|14770|2|722| +2450815|14773|2|151| +2450815|14774|2|33| +2450815|14776|2|746| +2450815|14779|2|931| +2450815|14780|2|432| +2450815|14782|2|163| +2450815|14785|2|| +2450815|14786|2|614| +2450815|14788|2|545| +2450815|14791|2|655| +2450815|14792|2|56| +2450815|14794|2|875| +2450815|14797|2|370| +2450815|14798|2|999| +2450815|14800|2|407| +2450815|14803|2|613| +2450815|14804|2|291| +2450815|14806|2|513| +2450815|14809|2|760| +2450815|14810|2|607| +2450815|14812|2|521| +2450815|14815|2|356| +2450815|14816|2|874| +2450815|14818|2|| +2450815|14821|2|138| +2450815|14822|2|781| +2450815|14824|2|760| +2450815|14827|2|937| +2450815|14828|2|228| +2450815|14830|2|377| +2450815|14833|2|831| +2450815|14834|2|426| +2450815|14836|2|531| +2450815|14839|2|227| +2450815|14840|2|426| +2450815|14842|2|572| +2450815|14845|2|521| +2450815|14846|2|162| +2450815|14848|2|565| +2450815|14851|2|467| +2450815|14852|2|| +2450815|14854|2|948| +2450815|14857|2|172| +2450815|14858|2|| +2450815|14860|2|996| +2450815|14863|2|589| +2450815|14864|2|298| +2450815|14866|2|462| +2450815|14869|2|933| +2450815|14870|2|375| +2450815|14872|2|| +2450815|14875|2|66| +2450815|14876|2|560| +2450815|14878|2|906| +2450815|14881|2|20| +2450815|14882|2|300| +2450815|14884|2|| +2450815|14887|2|| +2450815|14888|2|750| +2450815|14890|2|901| +2450815|14893|2|237| +2450815|14894|2|471| +2450815|14896|2|567| +2450815|14899|2|217| +2450815|14900|2|634| +2450815|14902|2|684| +2450815|14905|2|441| +2450815|14906|2|972| +2450815|14908|2|743| +2450815|14911|2|187| +2450815|14912|2|485| +2450815|14914|2|870| +2450815|14917|2|59| +2450815|14918|2|605| +2450815|14920|2|318| +2450815|14923|2|828| +2450815|14924|2|608| +2450815|14926|2|1000| +2450815|14929|2|704| +2450815|14930|2|827| +2450815|14932|2|601| +2450815|14935|2|| +2450815|14936|2|244| +2450815|14938|2|138| +2450815|14941|2|452| +2450815|14942|2|824| +2450815|14944|2|392| +2450815|14947|2|783| +2450815|14948|2|126| +2450815|14950|2|705| +2450815|14953|2|156| +2450815|14954|2|75| +2450815|14956|2|20| +2450815|14959|2|| +2450815|14960|2|| +2450815|14962|2|579| +2450815|14965|2|981| +2450815|14966|2|9| +2450815|14968|2|648| +2450815|14971|2|500| +2450815|14972|2|975| +2450815|14974|2|465| +2450815|14977|2|| +2450815|14978|2|510| +2450815|14980|2|110| +2450815|14983|2|762| +2450815|14984|2|6| +2450815|14986|2|508| +2450815|14989|2|443| +2450815|14990|2|395| +2450815|14992|2|646| +2450815|14995|2|320| +2450815|14996|2|518| +2450815|14998|2|523| +2450815|15001|2|244| +2450815|15002|2|928| +2450815|15004|2|214| +2450815|15007|2|141| +2450815|15008|2|551| +2450815|15010|2|627| +2450815|15013|2|602| +2450815|15014|2|944| +2450815|15016|2|429| +2450815|15019|2|645| +2450815|15020|2|133| +2450815|15022|2|982| +2450815|15025|2|567| +2450815|15026|2|811| +2450815|15028|2|562| +2450815|15031|2|979| +2450815|15032|2|989| +2450815|15034|2|566| +2450815|15037|2|3| +2450815|15038|2|303| +2450815|15040|2|953| +2450815|15043|2|644| +2450815|15044|2|121| +2450815|15046|2|725| +2450815|15049|2|848| +2450815|15050|2|130| +2450815|15052|2|209| +2450815|15055|2|701| +2450815|15056|2|32| +2450815|15058|2|544| +2450815|15061|2|226| +2450815|15062|2|59| +2450815|15064|2|542| +2450815|15067|2|524| +2450815|15068|2|481| +2450815|15070|2|763| +2450815|15073|2|751| +2450815|15074|2|337| +2450815|15076|2|815| +2450815|15079|2|27| +2450815|15080|2|206| +2450815|15082|2|269| +2450815|15085|2|64| +2450815|15086|2|929| +2450815|15088|2|738| +2450815|15091|2|19| +2450815|15092|2|585| +2450815|15094|2|199| +2450815|15097|2|980| +2450815|15098|2|915| +2450815|15100|2|563| +2450815|15103|2|661| +2450815|15104|2|429| +2450815|15106|2|7| +2450815|15109|2|611| +2450815|15110|2|7| +2450815|15112|2|110| +2450815|15115|2|483| +2450815|15116|2|811| +2450815|15118|2|766| +2450815|15121|2|540| +2450815|15122|2|357| +2450815|15124|2|134| +2450815|15127|2|150| +2450815|15128|2|396| +2450815|15130|2|901| +2450815|15133|2|93| +2450815|15134|2|8| +2450815|15136|2|368| +2450815|15139|2|354| +2450815|15140|2|| +2450815|15142|2|892| +2450815|15145|2|590| +2450815|15146|2|166| +2450815|15148|2|857| +2450815|15151|2|479| +2450815|15152|2|589| +2450815|15154|2|667| +2450815|15157|2|153| +2450815|15158|2|363| +2450815|15160|2|293| +2450815|15163|2|718| +2450815|15164|2|213| +2450815|15166|2|898| +2450815|15169|2|973| +2450815|15170|2|810| +2450815|15172|2|891| +2450815|15175|2|587| +2450815|15176|2|73| +2450815|15178|2|792| +2450815|15181|2|391| +2450815|15182|2|281| +2450815|15184|2|960| +2450815|15187|2|361| +2450815|15188|2|479| +2450815|15190|2|| +2450815|15193|2|945| +2450815|15194|2|579| +2450815|15196|2|919| +2450815|15199|2|719| +2450815|15200|2|56| +2450815|15202|2|922| +2450815|15205|2|877| +2450815|15206|2|367| +2450815|15208|2|569| +2450815|15211|2|16| +2450815|15212|2|685| +2450815|15214|2|727| +2450815|15217|2|585| +2450815|15218|2|265| +2450815|15220|2|179| +2450815|15223|2|36| +2450815|15224|2|615| +2450815|15226|2|876| +2450815|15229|2|799| +2450815|15230|2|633| +2450815|15232|2|926| +2450815|15235|2|789| +2450815|15236|2|766| +2450815|15238|2|759| +2450815|15241|2|91| +2450815|15242|2|464| +2450815|15244|2|498| +2450815|15247|2|263| +2450815|15248|2|914| +2450815|15250|2|624| +2450815|15253|2|566| +2450815|15254|2|552| +2450815|15256|2|458| +2450815|15259|2|74| +2450815|15260|2|550| +2450815|15262|2|355| +2450815|15265|2|347| +2450815|15266|2|566| +2450815|15268|2|805| +2450815|15271|2|75| +2450815|15272|2|530| +2450815|15274|2|882| +2450815|15277|2|369| +2450815|15278|2|183| +2450815|15280|2|410| +2450815|15283|2|361| +2450815|15284|2|408| +2450815|15286|2|| +2450815|15289|2|626| +2450815|15290|2|236| +2450815|15292|2|832| +2450815|15295|2|14| +2450815|15296|2|198| +2450815|15298|2|210| +2450815|15301|2|101| +2450815|15302|2|80| +2450815|15304|2|934| +2450815|15307|2|611| +2450815|15308|2|297| +2450815|15310|2|444| +2450815|15313|2|235| +2450815|15314|2|690| +2450815|15316|2|758| +2450815|15319|2|207| +2450815|15320|2|372| +2450815|15322|2|610| +2450815|15325|2|733| +2450815|15326|2|287| +2450815|15328|2|563| +2450815|15331|2|883| +2450815|15332|2|238| +2450815|15334|2|474| +2450815|15337|2|865| +2450815|15338|2|946| +2450815|15340|2|214| +2450815|15343|2|286| +2450815|15344|2|268| +2450815|15346|2|765| +2450815|15349|2|333| +2450815|15350|2|645| +2450815|15352|2|504| +2450815|15355|2|695| +2450815|15356|2|873| +2450815|15358|2|324| +2450815|15361|2|631| +2450815|15362|2|| +2450815|15364|2|507| +2450815|15367|2|943| +2450815|15368|2|282| +2450815|15370|2|679| +2450815|15373|2|414| +2450815|15374|2|674| +2450815|15376|2|782| +2450815|15379|2|541| +2450815|15380|2|803| +2450815|15382|2|448| +2450815|15385|2|312| +2450815|15386|2|433| +2450815|15388|2|578| +2450815|15391|2|455| +2450815|15392|2|337| +2450815|15394|2|153| +2450815|15397|2|983| +2450815|15398|2|600| +2450815|15400|2|227| +2450815|15403|2|404| +2450815|15404|2|595| +2450815|15406|2|409| +2450815|15409|2|354| +2450815|15410|2|144| +2450815|15412|2|207| +2450815|15415|2|370| +2450815|15416|2|241| +2450815|15418|2|565| +2450815|15421|2|549| +2450815|15422|2|398| +2450815|15424|2|650| +2450815|15427|2|436| +2450815|15428|2|815| +2450815|15430|2|253| +2450815|15433|2|599| +2450815|15434|2|17| +2450815|15436|2|483| +2450815|15439|2|721| +2450815|15440|2|| +2450815|15442|2|647| +2450815|15445|2|835| +2450815|15446|2|271| +2450815|15448|2|267| +2450815|15451|2|355| +2450815|15452|2|| +2450815|15454|2|701| +2450815|15457|2|| +2450815|15458|2|997| +2450815|15460|2|114| +2450815|15463|2|484| +2450815|15464|2|900| +2450815|15466|2|690| +2450815|15469|2|814| +2450815|15470|2|639| +2450815|15472|2|229| +2450815|15475|2|830| +2450815|15476|2|661| +2450815|15478|2|217| +2450815|15481|2|421| +2450815|15482|2|15| +2450815|15484|2|245| +2450815|15487|2|757| +2450815|15488|2|913| +2450815|15490|2|29| +2450815|15493|2|17| +2450815|15494|2|266| +2450815|15496|2|903| +2450815|15499|2|234| +2450815|15500|2|662| +2450815|15502|2|780| +2450815|15505|2|607| +2450815|15506|2|| +2450815|15508|2|760| +2450815|15511|2|149| +2450815|15512|2|598| +2450815|15514|2|996| +2450815|15517|2|| +2450815|15518|2|984| +2450815|15520|2|406| +2450815|15523|2|740| +2450815|15524|2|301| +2450815|15526|2|6| +2450815|15529|2|619| +2450815|15530|2|27| +2450815|15532|2|| +2450815|15535|2|107| +2450815|15536|2|627| +2450815|15538|2|624| +2450815|15541|2|578| +2450815|15542|2|320| +2450815|15544|2|836| +2450815|15547|2|123| +2450815|15548|2|768| +2450815|15550|2|614| +2450815|15553|2|177| +2450815|15554|2|47| +2450815|15556|2|223| +2450815|15559|2|908| +2450815|15560|2|585| +2450815|15562|2|974| +2450815|15565|2|299| +2450815|15566|2|255| +2450815|15568|2|363| +2450815|15571|2|572| +2450815|15572|2|604| +2450815|15574|2|690| +2450815|15577|2|982| +2450815|15578|2|949| +2450815|15580|2|399| +2450815|15583|2|192| +2450815|15584|2|482| +2450815|15586|2|802| +2450815|15589|2|428| +2450815|15590|2|624| +2450815|15592|2|292| +2450815|15595|2|693| +2450815|15596|2|| +2450815|15598|2|465| +2450815|15601|2|354| +2450815|15602|2|| +2450815|15604|2|136| +2450815|15607|2|689| +2450815|15608|2|193| +2450815|15610|2|414| +2450815|15613|2|343| +2450815|15614|2|994| +2450815|15616|2|217| +2450815|15619|2|| +2450815|15620|2|632| +2450815|15622|2|802| +2450815|15625|2|651| +2450815|15626|2|565| +2450815|15628|2|696| +2450815|15631|2|498| +2450815|15632|2|574| +2450815|15634|2|630| +2450815|15637|2|934| +2450815|15638|2|998| +2450815|15640|2|465| +2450815|15643|2|867| +2450815|15644|2|523| +2450815|15646|2|339| +2450815|15649|2|367| +2450815|15650|2|563| +2450815|15652|2|773| +2450815|15655|2|340| +2450815|15656|2|742| +2450815|15658|2|516| +2450815|15661|2|230| +2450815|15662|2|255| +2450815|15664|2|814| +2450815|15667|2|14| +2450815|15668|2|766| +2450815|15670|2|327| +2450815|15673|2|98| +2450815|15674|2|845| +2450815|15676|2|136| +2450815|15679|2|977| +2450815|15680|2|| +2450815|15682|2|711| +2450815|15685|2|717| +2450815|15686|2|697| +2450815|15688|2|529| +2450815|15691|2|992| +2450815|15692|2|636| +2450815|15694|2|287| +2450815|15697|2|140| +2450815|15698|2|319| +2450815|15700|2|819| +2450815|15703|2|622| +2450815|15704|2|872| +2450815|15706|2|150| +2450815|15709|2|663| +2450815|15710|2|840| +2450815|15712|2|507| +2450815|15715|2|199| +2450815|15716|2|317| +2450815|15718|2|| +2450815|15721|2|328| +2450815|15722|2|187| +2450815|15724|2|295| +2450815|15727|2|460| +2450815|15728|2|925| +2450815|15730|2|933| +2450815|15733|2|79| +2450815|15734|2|280| +2450815|15736|2|699| +2450815|15739|2|532| +2450815|15740|2|836| +2450815|15742|2|2| +2450815|15745|2|743| +2450815|15746|2|923| +2450815|15748|2|131| +2450815|15751|2|979| +2450815|15752|2|447| +2450815|15754|2|936| +2450815|15757|2|339| +2450815|15758|2|642| +2450815|15760|2|889| +2450815|15763|2|234| +2450815|15764|2|718| +2450815|15766|2|632| +2450815|15769|2|985| +2450815|15770|2|967| +2450815|15772|2|289| +2450815|15775|2|238| +2450815|15776|2|| +2450815|15778|2|444| +2450815|15781|2|676| +2450815|15782|2|591| +2450815|15784|2|691| +2450815|15787|2|661| +2450815|15788|2|210| +2450815|15790|2|106| +2450815|15793|2|169| +2450815|15794|2|910| +2450815|15796|2|21| +2450815|15799|2|81| +2450815|15800|2|844| +2450815|15802|2|840| +2450815|15805|2|944| +2450815|15806|2|369| +2450815|15808|2|597| +2450815|15811|2|902| +2450815|15812|2|501| +2450815|15814|2|612| +2450815|15817|2|811| +2450815|15818|2|327| +2450815|15820|2|342| +2450815|15823|2|942| +2450815|15824|2|893| +2450815|15826|2|278| +2450815|15829|2|649| +2450815|15830|2|675| +2450815|15832|2|946| +2450815|15835|2|| +2450815|15836|2|892| +2450815|15838|2|777| +2450815|15841|2|278| +2450815|15842|2|893| +2450815|15844|2|417| +2450815|15847|2|466| +2450815|15848|2|770| +2450815|15850|2|206| +2450815|15853|2|130| +2450815|15854|2|211| +2450815|15856|2|360| +2450815|15859|2|530| +2450815|15860|2|768| +2450815|15862|2|945| +2450815|15865|2|939| +2450815|15866|2|14| +2450815|15868|2|786| +2450815|15871|2|625| +2450815|15872|2|971| +2450815|15874|2|326| +2450815|15877|2|213| +2450815|15878|2|222| +2450815|15880|2|440| +2450815|15883|2|841| +2450815|15884|2|668| +2450815|15886|2|614| +2450815|15889|2|922| +2450815|15890|2|261| +2450815|15892|2|71| +2450815|15895|2|194| +2450815|15896|2|397| +2450815|15898|2|102| +2450815|15901|2|233| +2450815|15902|2|797| +2450815|15904|2|923| +2450815|15907|2|411| +2450815|15908|2|755| +2450815|15910|2|553| +2450815|15913|2|755| +2450815|15914|2|468| +2450815|15916|2|786| +2450815|15919|2|595| +2450815|15920|2|524| +2450815|15922|2|989| +2450815|15925|2|360| +2450815|15926|2|190| +2450815|15928|2|751| +2450815|15931|2|853| +2450815|15932|2|238| +2450815|15934|2|521| +2450815|15937|2|568| +2450815|15938|2|670| +2450815|15940|2|426| +2450815|15943|2|254| +2450815|15944|2|236| +2450815|15946|2|210| +2450815|15949|2|983| +2450815|15950|2|| +2450815|15952|2|503| +2450815|15955|2|| +2450815|15956|2|698| +2450815|15958|2|916| +2450815|15961|2|921| +2450815|15962|2|441| +2450815|15964|2|973| +2450815|15967|2|185| +2450815|15968|2|327| +2450815|15970|2|940| +2450815|15973|2|3| +2450815|15974|2|973| +2450815|15976|2|241| +2450815|15979|2|241| +2450815|15980|2|710| +2450815|15982|2|115| +2450815|15985|2|788| +2450815|15986|2|517| +2450815|15988|2|782| +2450815|15991|2|654| +2450815|15992|2|52| +2450815|15994|2|871| +2450815|15997|2|673| +2450815|15998|2|| +2450815|16000|2|590| +2450815|16003|2|236| +2450815|16004|2|336| +2450815|16006|2|373| +2450815|16009|2|98| +2450815|16010|2|245| +2450815|16012|2|538| +2450815|16015|2|635| +2450815|16016|2|923| +2450815|16018|2|88| +2450815|16021|2|506| +2450815|16022|2|789| +2450815|16024|2|804| +2450815|16027|2|819| +2450815|16028|2|651| +2450815|16030|2|72| +2450815|16033|2|251| +2450815|16034|2|990| +2450815|16036|2|279| +2450815|16039|2|290| +2450815|16040|2|| +2450815|16042|2|33| +2450815|16045|2|468| +2450815|16046|2|966| +2450815|16048|2|520| +2450815|16051|2|28| +2450815|16052|2|824| +2450815|16054|2|778| +2450815|16057|2|| +2450815|16058|2|881| +2450815|16060|2|383| +2450815|16063|2|58| +2450815|16064|2|544| +2450815|16066|2|514| +2450815|16069|2|86| +2450815|16070|2|801| +2450815|16072|2|35| +2450815|16075|2|389| +2450815|16076|2|899| +2450815|16078|2|126| +2450815|16081|2|816| +2450815|16082|2|754| +2450815|16084|2|924| +2450815|16087|2|82| +2450815|16088|2|155| +2450815|16090|2|756| +2450815|16093|2|219| +2450815|16094|2|811| +2450815|16096|2|| +2450815|16099|2|474| +2450815|16100|2|943| +2450815|16102|2|905| +2450815|16105|2|11| +2450815|16106|2|737| +2450815|16108|2|393| +2450815|16111|2|325| +2450815|16112|2|| +2450815|16114|2|| +2450815|16117|2|645| +2450815|16118|2|543| +2450815|16120|2|737| +2450815|16123|2|867| +2450815|16124|2|395| +2450815|16126|2|| +2450815|16129|2|914| +2450815|16130|2|390| +2450815|16132|2|220| +2450815|16135|2|72| +2450815|16136|2|857| +2450815|16138|2|465| +2450815|16141|2|647| +2450815|16142|2|687| +2450815|16144|2|| +2450815|16147|2|| +2450815|16148|2|109| +2450815|16150|2|638| +2450815|16153|2|146| +2450815|16154|2|126| +2450815|16156|2|305| +2450815|16159|2|895| +2450815|16160|2|949| +2450815|16162|2|356| +2450815|16165|2|260| +2450815|16166|2|907| +2450815|16168|2|| +2450815|16171|2|968| +2450815|16172|2|961| +2450815|16174|2|970| +2450815|16177|2|313| +2450815|16178|2|255| +2450815|16180|2|475| +2450815|16183|2|25| +2450815|16184|2|748| +2450815|16186|2|927| +2450815|16189|2|33| +2450815|16190|2|849| +2450815|16192|2|552| +2450815|16195|2|176| +2450815|16196|2|315| +2450815|16198|2|100| +2450815|16201|2|239| +2450815|16202|2|| +2450815|16204|2|| +2450815|16207|2|908| +2450815|16208|2|685| +2450815|16210|2|777| +2450815|16213|2|146| +2450815|16214|2|993| +2450815|16216|2|581| +2450815|16219|2|| +2450815|16220|2|441| +2450815|16222|2|730| +2450815|16225|2|627| +2450815|16226|2|624| +2450815|16228|2|198| +2450815|16231|2|482| +2450815|16232|2|853| +2450815|16234|2|146| +2450815|16237|2|776| +2450815|16238|2|33| +2450815|16240|2|931| +2450815|16243|2|849| +2450815|16244|2|885| +2450815|16246|2|812| +2450815|16249|2|673| +2450815|16250|2|533| +2450815|16252|2|595| +2450815|16255|2|519| +2450815|16256|2|153| +2450815|16258|2|387| +2450815|16261|2|| +2450815|16262|2|408| +2450815|16264|2|133| +2450815|16267|2|248| +2450815|16268|2|879| +2450815|16270|2|698| +2450815|16273|2|201| +2450815|16274|2|649| +2450815|16276|2|| +2450815|16279|2|910| +2450815|16280|2|152| +2450815|16282|2|992| +2450815|16285|2|225| +2450815|16286|2|843| +2450815|16288|2|914| +2450815|16291|2|306| +2450815|16292|2|639| +2450815|16294|2|870| +2450815|16297|2|646| +2450815|16298|2|872| +2450815|16300|2|891| +2450815|16303|2|165| +2450815|16304|2|781| +2450815|16306|2|484| +2450815|16309|2|856| +2450815|16310|2|213| +2450815|16312|2|237| +2450815|16315|2|618| +2450815|16316|2|94| +2450815|16318|2|10| +2450815|16321|2|720| +2450815|16322|2|355| +2450815|16324|2|808| +2450815|16327|2|656| +2450815|16328|2|353| +2450815|16330|2|203| +2450815|16333|2|177| +2450815|16334|2|205| +2450815|16336|2|350| +2450815|16339|2|269| +2450815|16340|2|101| +2450815|16342|2|218| +2450815|16345|2|316| +2450815|16346|2|974| +2450815|16348|2|991| +2450815|16351|2|284| +2450815|16352|2|68| +2450815|16354|2|542| +2450815|16357|2|346| +2450815|16358|2|168| +2450815|16360|2|60| +2450815|16363|2|451| +2450815|16364|2|517| +2450815|16366|2|674| +2450815|16369|2|709| +2450815|16370|2|32| +2450815|16372|2|517| +2450815|16375|2|830| +2450815|16376|2|644| +2450815|16378|2|243| +2450815|16381|2|67| +2450815|16382|2|526| +2450815|16384|2|415| +2450815|16387|2|851| +2450815|16388|2|53| +2450815|16390|2|489| +2450815|16393|2|525| +2450815|16394|2|708| +2450815|16396|2|33| +2450815|16399|2|989| +2450815|16400|2|455| +2450815|16402|2|563| +2450815|16405|2|775| +2450815|16406|2|253| +2450815|16408|2|524| +2450815|16411|2|784| +2450815|16412|2|494| +2450815|16414|2|329| +2450815|16417|2|990| +2450815|16418|2|672| +2450815|16420|2|809| +2450815|16423|2|332| +2450815|16424|2|966| +2450815|16426|2|197| +2450815|16429|2|461| +2450815|16430|2|224| +2450815|16432|2|275| +2450815|16435|2|570| +2450815|16436|2|443| +2450815|16438|2|326| +2450815|16441|2|| +2450815|16442|2|219| +2450815|16444|2|412| +2450815|16447|2|317| +2450815|16448|2|444| +2450815|16450|2|307| +2450815|16453|2|488| +2450815|16454|2|537| +2450815|16456|2|655| +2450815|16459|2|803| +2450815|16460|2|725| +2450815|16462|2|156| +2450815|16465|2|565| +2450815|16466|2|91| +2450815|16468|2|993| +2450815|16471|2|653| +2450815|16472|2|951| +2450815|16474|2|980| +2450815|16477|2|665| +2450815|16478|2|3| +2450815|16480|2|972| +2450815|16483|2|220| +2450815|16484|2|302| +2450815|16486|2|907| +2450815|16489|2|| +2450815|16490|2|83| +2450815|16492|2|921| +2450815|16495|2|486| +2450815|16496|2|188| +2450815|16498|2|393| +2450815|16501|2|137| +2450815|16502|2|963| +2450815|16504|2|446| +2450815|16507|2|| +2450815|16508|2|331| +2450815|16510|2|263| +2450815|16513|2|615| +2450815|16514|2|821| +2450815|16516|2|605| +2450815|16519|2|434| +2450815|16520|2|574| +2450815|16522|2|210| +2450815|16525|2|954| +2450815|16526|2|489| +2450815|16528|2|582| +2450815|16531|2|766| +2450815|16532|2|973| +2450815|16534|2|| +2450815|16537|2|559| +2450815|16538|2|926| +2450815|16540|2|292| +2450815|16543|2|630| +2450815|16544|2|910| +2450815|16546|2|508| +2450815|16549|2|376| +2450815|16550|2|827| +2450815|16552|2|328| +2450815|16555|2|589| +2450815|16556|2|828| +2450815|16558|2|416| +2450815|16561|2|881| +2450815|16562|2|662| +2450815|16564|2|345| +2450815|16567|2|533| +2450815|16568|2|148| +2450815|16570|2|857| +2450815|16573|2|496| +2450815|16574|2|664| +2450815|16576|2|66| +2450815|16579|2|745| +2450815|16580|2|360| +2450815|16582|2|471| +2450815|16585|2|727| +2450815|16586|2|182| +2450815|16588|2|278| +2450815|16591|2|95| +2450815|16592|2|189| +2450815|16594|2|278| +2450815|16597|2|| +2450815|16598|2|572| +2450815|16600|2|117| +2450815|16603|2|124| +2450815|16604|2|677| +2450815|16606|2|296| +2450815|16609|2|316| +2450815|16610|2|832| +2450815|16612|2|583| +2450815|16615|2|754| +2450815|16616|2|428| +2450815|16618|2|448| +2450815|16621|2|5| +2450815|16622|2|674| +2450815|16624|2|334| +2450815|16627|2|841| +2450815|16628|2|597| +2450815|16630|2|260| +2450815|16633|2|273| +2450815|16634|2|983| +2450815|16636|2|30| +2450815|16639|2|317| +2450815|16640|2|336| +2450815|16642|2|783| +2450815|16645|2|723| +2450815|16646|2|528| +2450815|16648|2|231| +2450815|16651|2|437| +2450815|16652|2|494| +2450815|16654|2|393| +2450815|16657|2|2| +2450815|16658|2|266| +2450815|16660|2|105| +2450815|16663|2|216| +2450815|16664|2|446| +2450815|16666|2|935| +2450815|16669|2|575| +2450815|16670|2|17| +2450815|16672|2|160| +2450815|16675|2|906| +2450815|16676|2|454| +2450815|16678|2|128| +2450815|16681|2|630| +2450815|16682|2|131| +2450815|16684|2|811| +2450815|16687|2|929| +2450815|16688|2|34| +2450815|16690|2|562| +2450815|16693|2|101| +2450815|16694|2|| +2450815|16696|2|388| +2450815|16699|2|199| +2450815|16700|2|593| +2450815|16702|2|879| +2450815|16705|2|254| +2450815|16706|2|298| +2450815|16708|2|994| +2450815|16711|2|173| +2450815|16712|2|656| +2450815|16714|2|371| +2450815|16717|2|408| +2450815|16718|2|900| +2450815|16720|2|501| +2450815|16723|2|380| +2450815|16724|2|245| +2450815|16726|2|424| +2450815|16729|2|743| +2450815|16730|2|586| +2450815|16732|2|519| +2450815|16735|2|42| +2450815|16736|2|| +2450815|16738|2|792| +2450815|16741|2|850| +2450815|16742|2|932| +2450815|16744|2|362| +2450815|16747|2|867| +2450815|16748|2|920| +2450815|16750|2|705| +2450815|16753|2|861| +2450815|16754|2|642| +2450815|16756|2|| +2450815|16759|2|254| +2450815|16760|2|668| +2450815|16762|2|649| +2450815|16765|2|774| +2450815|16766|2|951| +2450815|16768|2|296| +2450815|16771|2|738| +2450815|16772|2|674| +2450815|16774|2|117| +2450815|16777|2|895| +2450815|16778|2|913| +2450815|16780|2|788| +2450815|16783|2|722| +2450815|16784|2|229| +2450815|16786|2|429| +2450815|16789|2|177| +2450815|16790|2|440| +2450815|16792|2|424| +2450815|16795|2|31| +2450815|16796|2|348| +2450815|16798|2|398| +2450815|16801|2|84| +2450815|16802|2|548| +2450815|16804|2|293| +2450815|16807|2|644| +2450815|16808|2|104| +2450815|16810|2|630| +2450815|16813|2|722| +2450815|16814|2|259| +2450815|16816|2|509| +2450815|16819|2|602| +2450815|16820|2|522| +2450815|16822|2|517| +2450815|16825|2|1| +2450815|16826|2|85| +2450815|16828|2|196| +2450815|16831|2|305| +2450815|16832|2|807| +2450815|16834|2|869| +2450815|16837|2|| +2450815|16838|2|199| +2450815|16840|2|3| +2450815|16843|2|940| +2450815|16844|2|138| +2450815|16846|2|107| +2450815|16849|2|671| +2450815|16850|2|214| +2450815|16852|2|23| +2450815|16855|2|533| +2450815|16856|2|556| +2450815|16858|2|755| +2450815|16861|2|287| +2450815|16862|2|241| +2450815|16864|2|170| +2450815|16867|2|428| +2450815|16868|2|846| +2450815|16870|2|734| +2450815|16873|2|697| +2450815|16874|2|146| +2450815|16876|2|581| +2450815|16879|2|451| +2450815|16880|2|838| +2450815|16882|2|50| +2450815|16885|2|| +2450815|16886|2|477| +2450815|16888|2|989| +2450815|16891|2|964| +2450815|16892|2|936| +2450815|16894|2|510| +2450815|16897|2|223| +2450815|16898|2|819| +2450815|16900|2|993| +2450815|16903|2|691| +2450815|16904|2|416| +2450815|16906|2|62| +2450815|16909|2|728| +2450815|16910|2|450| +2450815|16912|2|111| +2450815|16915|2|788| +2450815|16916|2|752| +2450815|16918|2|527| +2450815|16921|2|79| +2450815|16922|2|625| +2450815|16924|2|363| +2450815|16927|2|779| +2450815|16928|2|284| +2450815|16930|2|605| +2450815|16933|2|49| +2450815|16934|2|| +2450815|16936|2|20| +2450815|16939|2|804| +2450815|16940|2|432| +2450815|16942|2|279| +2450815|16945|2|493| +2450815|16946|2|463| +2450815|16948|2|110| +2450815|16951|2|138| +2450815|16952|2|148| +2450815|16954|2|388| +2450815|16957|2|640| +2450815|16958|2|511| +2450815|16960|2|119| +2450815|16963|2|975| +2450815|16964|2|200| +2450815|16966|2|832| +2450815|16969|2|131| +2450815|16970|2|422| +2450815|16972|2|511| +2450815|16975|2|712| +2450815|16976|2|943| +2450815|16978|2|144| +2450815|16981|2|739| +2450815|16982|2|645| +2450815|16984|2|849| +2450815|16987|2|204| +2450815|16988|2|899| +2450815|16990|2|339| +2450815|16993|2|856| +2450815|16994|2|220| +2450815|16996|2|976| +2450815|16999|2|579| +2450815|17000|2|397| +2450815|17002|2|204| +2450815|17005|2|985| +2450815|17006|2|111| +2450815|17008|2|223| +2450815|17011|2|16| +2450815|17012|2|207| +2450815|17014|2|631| +2450815|17017|2|250| +2450815|17018|2|974| +2450815|17020|2|11| +2450815|17023|2|57| +2450815|17024|2|24| +2450815|17026|2|833| +2450815|17029|2|334| +2450815|17030|2|| +2450815|17032|2|776| +2450815|17035|2|967| +2450815|17036|2|95| +2450815|17038|2|869| +2450815|17041|2|776| +2450815|17042|2|515| +2450815|17044|2|531| +2450815|17047|2|792| +2450815|17048|2|12| +2450815|17050|2|71| +2450815|17053|2|263| +2450815|17054|2|646| +2450815|17056|2|624| +2450815|17059|2|716| +2450815|17060|2|909| +2450815|17062|2|842| +2450815|17065|2|557| +2450815|17066|2|594| +2450815|17068|2|888| +2450815|17071|2|147| +2450815|17072|2|905| +2450815|17074|2|913| +2450815|17077|2|876| +2450815|17078|2|492| +2450815|17080|2|241| +2450815|17083|2|320| +2450815|17084|2|532| +2450815|17086|2|467| +2450815|17089|2|296| +2450815|17090|2|182| +2450815|17092|2|286| +2450815|17095|2|106| +2450815|17096|2|782| +2450815|17098|2|611| +2450815|17101|2|35| +2450815|17102|2|316| +2450815|17104|2|970| +2450815|17107|2|144| +2450815|17108|2|243| +2450815|17110|2|575| +2450815|17113|2|880| +2450815|17114|2|364| +2450815|17116|2|211| +2450815|17119|2|650| +2450815|17120|2|342| +2450815|17122|2|237| +2450815|17125|2|495| +2450815|17126|2|601| +2450815|17128|2|308| +2450815|17131|2|514| +2450815|17132|2|350| +2450815|17134|2|57| +2450815|17137|2|537| +2450815|17138|2|139| +2450815|17140|2|71| +2450815|17143|2|15| +2450815|17144|2|608| +2450815|17146|2|66| +2450815|17149|2|128| +2450815|17150|2|617| +2450815|17152|2|572| +2450815|17155|2|629| +2450815|17156|2|534| +2450815|17158|2|948| +2450815|17161|2|721| +2450815|17162|2|869| +2450815|17164|2|514| +2450815|17167|2|431| +2450815|17168|2|391| +2450815|17170|2|194| +2450815|17173|2|632| +2450815|17174|2|128| +2450815|17176|2|381| +2450815|17179|2|698| +2450815|17180|2|365| +2450815|17182|2|293| +2450815|17185|2|655| +2450815|17186|2|945| +2450815|17188|2|94| +2450815|17191|2|802| +2450815|17192|2|| +2450815|17194|2|242| +2450815|17197|2|659| +2450815|17198|2|746| +2450815|17200|2|333| +2450815|17203|2|956| +2450815|17204|2|815| +2450815|17206|2|533| +2450815|17209|2|894| +2450815|17210|2|329| +2450815|17212|2|91| +2450815|17215|2|37| +2450815|17216|2|932| +2450815|17218|2|925| +2450815|17221|2|192| +2450815|17222|2|256| +2450815|17224|2|917| +2450815|17227|2|355| +2450815|17228|2|636| +2450815|17230|2|168| +2450815|17233|2|143| +2450815|17234|2|| +2450815|17236|2|400| +2450815|17239|2|557| +2450815|17240|2|941| +2450815|17242|2|102| +2450815|17245|2|955| +2450815|17246|2|596| +2450815|17248|2|557| +2450815|17251|2|606| +2450815|17252|2|808| +2450815|17254|2|685| +2450815|17257|2|671| +2450815|17258|2|597| +2450815|17260|2|711| +2450815|17263|2|860| +2450815|17264|2|859| +2450815|17266|2|336| +2450815|17269|2|749| +2450815|17270|2|16| +2450815|17272|2|297| +2450815|17275|2|854| +2450815|17276|2|24| +2450815|17278|2|| +2450815|17281|2|611| +2450815|17282|2|371| +2450815|17284|2|909| +2450815|17287|2|622| +2450815|17288|2|120| +2450815|17290|2|572| +2450815|17293|2|850| +2450815|17294|2|514| +2450815|17296|2|901| +2450815|17299|2|614| +2450815|17300|2|212| +2450815|17302|2|56| +2450815|17305|2|338| +2450815|17306|2|764| +2450815|17308|2|84| +2450815|17311|2|855| +2450815|17312|2|905| +2450815|17314|2|| +2450815|17317|2|788| +2450815|17318|2|43| +2450815|17320|2|597| +2450815|17323|2|133| +2450815|17324|2|434| +2450815|17326|2|462| +2450815|17329|2|76| +2450815|17330|2|124| +2450815|17332|2|146| +2450815|17335|2|978| +2450815|17336|2|271| +2450815|17338|2|658| +2450815|17341|2|350| +2450815|17342|2|179| +2450815|17344|2|513| +2450815|17347|2|772| +2450815|17348|2|284| +2450815|17350|2|627| +2450815|17353|2|628| +2450815|17354|2|489| +2450815|17356|2|869| +2450815|17359|2|290| +2450815|17360|2|701| +2450815|17362|2|158| +2450815|17365|2|895| +2450815|17366|2|558| +2450815|17368|2|72| +2450815|17371|2|301| +2450815|17372|2|157| +2450815|17374|2|577| +2450815|17377|2|634| +2450815|17378|2|727| +2450815|17380|2|225| +2450815|17383|2|639| +2450815|17384|2|930| +2450815|17386|2|760| +2450815|17389|2|187| +2450815|17390|2|137| +2450815|17392|2|741| +2450815|17395|2|345| +2450815|17396|2|985| +2450815|17398|2|899| +2450815|17401|2|116| +2450815|17402|2|556| +2450815|17404|2|263| +2450815|17407|2|154| +2450815|17408|2|1000| +2450815|17410|2|870| +2450815|17413|2|583| +2450815|17414|2|84| +2450815|17416|2|345| +2450815|17419|2|771| +2450815|17420|2|197| +2450815|17422|2|383| +2450815|17425|2|398| +2450815|17426|2|391| +2450815|17428|2|900| +2450815|17431|2|942| +2450815|17432|2|215| +2450815|17434|2|590| +2450815|17437|2|637| +2450815|17438|2|461| +2450815|17440|2|176| +2450815|17443|2|604| +2450815|17444|2|917| +2450815|17446|2|933| +2450815|17449|2|34| +2450815|17450|2|48| +2450815|17452|2|808| +2450815|17455|2|9| +2450815|17456|2|355| +2450815|17458|2|444| +2450815|17461|2|982| +2450815|17462|2|281| +2450815|17464|2|902| +2450815|17467|2|355| +2450815|17468|2|784| +2450815|17470|2|47| +2450815|17473|2|661| +2450815|17474|2|799| +2450815|17476|2|921| +2450815|17479|2|133| +2450815|17480|2|176| +2450815|17482|2|889| +2450815|17485|2|414| +2450815|17486|2|40| +2450815|17488|2|949| +2450815|17491|2|6| +2450815|17492|2|435| +2450815|17494|2|918| +2450815|17497|2|935| +2450815|17498|2|679| +2450815|17500|2|679| +2450815|17503|2|983| +2450815|17504|2|583| +2450815|17506|2|781| +2450815|17509|2|260| +2450815|17510|2|940| +2450815|17512|2|367| +2450815|17515|2|858| +2450815|17516|2|271| +2450815|17518|2|494| +2450815|17521|2|736| +2450815|17522|2|717| +2450815|17524|2|431| +2450815|17527|2|143| +2450815|17528|2|574| +2450815|17530|2|507| +2450815|17533|2|47| +2450815|17534|2|958| +2450815|17536|2|542| +2450815|17539|2|15| +2450815|17540|2|599| +2450815|17542|2|571| +2450815|17545|2|644| +2450815|17546|2|| +2450815|17548|2|168| +2450815|17551|2|496| +2450815|17552|2|242| +2450815|17554|2|236| +2450815|17557|2|65| +2450815|17558|2|355| +2450815|17560|2|369| +2450815|17563|2|734| +2450815|17564|2|403| +2450815|17566|2|256| +2450815|17569|2|733| +2450815|17570|2|46| +2450815|17572|2|863| +2450815|17575|2|801| +2450815|17576|2|361| +2450815|17578|2|787| +2450815|17581|2|622| +2450815|17582|2|976| +2450815|17584|2|695| +2450815|17587|2|122| +2450815|17588|2|531| +2450815|17590|2|194| +2450815|17593|2|1| +2450815|17594|2|78| +2450815|17596|2|958| +2450815|17599|2|| +2450815|17600|2|408| +2450815|17602|2|405| +2450815|17605|2|475| +2450815|17606|2|83| +2450815|17608|2|940| +2450815|17611|2|742| +2450815|17612|2|848| +2450815|17614|2|931| +2450815|17617|2|761| +2450815|17618|2|529| +2450815|17620|2|223| +2450815|17623|2|925| +2450815|17624|2|612| +2450815|17626|2|279| +2450815|17629|2|774| +2450815|17630|2|979| +2450815|17632|2|293| +2450815|17635|2|268| +2450815|17636|2|523| +2450815|17638|2|547| +2450815|17641|2|37| +2450815|17642|2|518| +2450815|17644|2|920| +2450815|17647|2|603| +2450815|17648|2|83| +2450815|17650|2|408| +2450815|17653|2|715| +2450815|17654|2|656| +2450815|17656|2|349| +2450815|17659|2|| +2450815|17660|2|805| +2450815|17662|2|197| +2450815|17665|2|314| +2450815|17666|2|622| +2450815|17668|2|175| +2450815|17671|2|152| +2450815|17672|2|422| +2450815|17674|2|463| +2450815|17677|2|940| +2450815|17678|2|383| +2450815|17680|2|433| +2450815|17683|2|| +2450815|17684|2|572| +2450815|17686|2|43| +2450815|17689|2|490| +2450815|17690|2|451| +2450815|17692|2|102| +2450815|17695|2|23| +2450815|17696|2|221| +2450815|17698|2|642| +2450815|17701|2|180| +2450815|17702|2|972| +2450815|17704|2|201| +2450815|17707|2|762| +2450815|17708|2|| +2450815|17710|2|51| +2450815|17713|2|| +2450815|17714|2|| +2450815|17716|2|4| +2450815|17719|2|917| +2450815|17720|2|153| +2450815|17722|2|958| +2450815|17725|2|205| +2450815|17726|2|| +2450815|17728|2|463| +2450815|17731|2|582| +2450815|17732|2|280| +2450815|17734|2|| +2450815|17737|2|972| +2450815|17738|2|418| +2450815|17740|2|855| +2450815|17743|2|710| +2450815|17744|2|991| +2450815|17746|2|389| +2450815|17749|2|931| +2450815|17750|2|101| +2450815|17752|2|528| +2450815|17755|2|980| +2450815|17756|2|434| +2450815|17758|2|159| +2450815|17761|2|29| +2450815|17762|2|896| +2450815|17764|2|| +2450815|17767|2|494| +2450815|17768|2|594| +2450815|17770|2|357| +2450815|17773|2|596| +2450815|17774|2|105| +2450815|17776|2|832| +2450815|17779|2|892| +2450815|17780|2|961| +2450815|17782|2|716| +2450815|17785|2|478| +2450815|17786|2|478| +2450815|17788|2|641| +2450815|17791|2|658| +2450815|17792|2|839| +2450815|17794|2|343| +2450815|17797|2|833| +2450815|17798|2|445| +2450815|17800|2|159| +2450815|17803|2|816| +2450815|17804|2|698| +2450815|17806|2|251| +2450815|17809|2|327| +2450815|17810|2|374| +2450815|17812|2|909| +2450815|17815|2|973| +2450815|17816|2|547| +2450815|17818|2|198| +2450815|17821|2|301| +2450815|17822|2|454| +2450815|17824|2|481| +2450815|17827|2|979| +2450815|17828|2|562| +2450815|17830|2|384| +2450815|17833|2|372| +2450815|17834|2|637| +2450815|17836|2|805| +2450815|17839|2|643| +2450815|17840|2|591| +2450815|17842|2|483| +2450815|17845|2|398| +2450815|17846|2|984| +2450815|17848|2|921| +2450815|17851|2|252| +2450815|17852|2|759| +2450815|17854|2|| +2450815|17857|2|215| +2450815|17858|2|172| +2450815|17860|2|827| +2450815|17863|2|150| +2450815|17864|2|826| +2450815|17866|2|246| +2450815|17869|2|762| +2450815|17870|2|974| +2450815|17872|2|319| +2450815|17875|2|510| +2450815|17876|2|| +2450815|17878|2|12| +2450815|17881|2|33| +2450815|17882|2|390| +2450815|17884|2|792| +2450815|17887|2|819| +2450815|17888|2|105| +2450815|17890|2|496| +2450815|17893|2|13| +2450815|17894|2|708| +2450815|17896|2|425| +2450815|17899|2|700| +2450815|17900|2|733| +2450815|17902|2|740| +2450815|17905|2|44| +2450815|17906|2|141| +2450815|17908|2|513| +2450815|17911|2|309| +2450815|17912|2|508| +2450815|17914|2|772| +2450815|17917|2|504| +2450815|17918|2|342| +2450815|17920|2|273| +2450815|17923|2|309| +2450815|17924|2|377| +2450815|17926|2|804| +2450815|17929|2|| +2450815|17930|2|757| +2450815|17932|2|735| +2450815|17935|2|254| +2450815|17936|2|836| +2450815|17938|2|| +2450815|17941|2|512| +2450815|17942|2|883| +2450815|17944|2|759| +2450815|17947|2|109| +2450815|17948|2|746| +2450815|17950|2|301| +2450815|17953|2|997| +2450815|17954|2|755| +2450815|17956|2|841| +2450815|17959|2|369| +2450815|17960|2|796| +2450815|17962|2|218| +2450815|17965|2|736| +2450815|17966|2|561| +2450815|17968|2|923| +2450815|17971|2|881| +2450815|17972|2|655| +2450815|17974|2|795| +2450815|17977|2|800| +2450815|17978|2|773| +2450815|17980|2|793| +2450815|17983|2|347| +2450815|17984|2|256| +2450815|17986|2|473| +2450815|17989|2|961| +2450815|17990|2|90| +2450815|17992|2|567| +2450815|17995|2|298| +2450815|17996|2|638| +2450815|17998|2|631| +2450815|1|3|494| +2450815|2|3|58| +2450815|4|3|780| +2450815|7|3|614| +2450815|8|3|975| +2450815|10|3|145| +2450815|13|3|917| +2450815|14|3|556| +2450815|16|3|161| +2450815|19|3|808| +2450815|20|3|409| +2450815|22|3|477| +2450815|25|3|570| +2450815|26|3|62| +2450815|28|3|530| +2450815|31|3|756| +2450815|32|3|970| +2450815|34|3|943| +2450815|37|3|980| +2450815|38|3|263| +2450815|40|3|556| +2450815|43|3|867| +2450815|44|3|574| +2450815|46|3|149| +2450815|49|3|579| +2450815|50|3|115| +2450815|52|3|782| +2450815|55|3|290| +2450815|56|3|452| +2450815|58|3|450| +2450815|61|3|661| +2450815|62|3|700| +2450815|64|3|175| +2450815|67|3|428| +2450815|68|3|957| +2450815|70|3|533| +2450815|73|3|657| +2450815|74|3|78| +2450815|76|3|26| +2450815|79|3|| +2450815|80|3|24| +2450815|82|3|446| +2450815|85|3|917| +2450815|86|3|979| +2450815|88|3|192| +2450815|91|3|479| +2450815|92|3|2| +2450815|94|3|654| +2450815|97|3|451| +2450815|98|3|801| +2450815|100|3|780| +2450815|103|3|300| +2450815|104|3|893| +2450815|106|3|130| +2450815|109|3|524| +2450815|110|3|733| +2450815|112|3|200| +2450815|115|3|761| +2450815|116|3|423| +2450815|118|3|470| +2450815|121|3|353| +2450815|122|3|781| +2450815|124|3|84| +2450815|127|3|833| +2450815|128|3|119| +2450815|130|3|807| +2450815|133|3|33| +2450815|134|3|789| +2450815|136|3|880| +2450815|139|3|928| +2450815|140|3|592| +2450815|142|3|534| +2450815|145|3|977| +2450815|146|3|737| +2450815|148|3|618| +2450815|151|3|359| +2450815|152|3|| +2450815|154|3|465| +2450815|157|3|370| +2450815|158|3|615| +2450815|160|3|632| +2450815|163|3|306| +2450815|164|3|600| +2450815|166|3|974| +2450815|169|3|799| +2450815|170|3|410| +2450815|172|3|465| +2450815|175|3|715| +2450815|176|3|184| +2450815|178|3|506| +2450815|181|3|582| +2450815|182|3|678| +2450815|184|3|290| +2450815|187|3|525| +2450815|188|3|994| +2450815|190|3|765| +2450815|193|3|223| +2450815|194|3|367| +2450815|196|3|449| +2450815|199|3|624| +2450815|200|3|870| +2450815|202|3|641| +2450815|205|3|59| +2450815|206|3|344| +2450815|208|3|783| +2450815|211|3|193| +2450815|212|3|88| +2450815|214|3|510| +2450815|217|3|327| +2450815|218|3|915| +2450815|220|3|5| +2450815|223|3|969| +2450815|224|3|672| +2450815|226|3|601| +2450815|229|3|| +2450815|230|3|983| +2450815|232|3|311| +2450815|235|3|578| +2450815|236|3|259| +2450815|238|3|240| +2450815|241|3|851| +2450815|242|3|505| +2450815|244|3|78| +2450815|247|3|46| +2450815|248|3|478| +2450815|250|3|384| +2450815|253|3|390| +2450815|254|3|883| +2450815|256|3|267| +2450815|259|3|656| +2450815|260|3|| +2450815|262|3|375| +2450815|265|3|616| +2450815|266|3|| +2450815|268|3|363| +2450815|271|3|945| +2450815|272|3|144| +2450815|274|3|287| +2450815|277|3|852| +2450815|278|3|240| +2450815|280|3|713| +2450815|283|3|743| +2450815|284|3|| +2450815|286|3|980| +2450815|289|3|752| +2450815|290|3|555| +2450815|292|3|741| +2450815|295|3|657| +2450815|296|3|43| +2450815|298|3|70| +2450815|301|3|268| +2450815|302|3|374| +2450815|304|3|415| +2450815|307|3|455| +2450815|308|3|873| +2450815|310|3|628| +2450815|313|3|678| +2450815|314|3|250| +2450815|316|3|633| +2450815|319|3|115| +2450815|320|3|| +2450815|322|3|43| +2450815|325|3|185| +2450815|326|3|271| +2450815|328|3|| +2450815|331|3|491| +2450815|332|3|| +2450815|334|3|116| +2450815|337|3|100| +2450815|338|3|674| +2450815|340|3|804| +2450815|343|3|329| +2450815|344|3|286| +2450815|346|3|297| +2450815|349|3|115| +2450815|350|3|| +2450815|352|3|363| +2450815|355|3|| +2450815|356|3|183| +2450815|358|3|558| +2450815|361|3|615| +2450815|362|3|604| +2450815|364|3|847| +2450815|367|3|829| +2450815|368|3|596| +2450815|370|3|84| +2450815|373|3|180| +2450815|374|3|239| +2450815|376|3|89| +2450815|379|3|533| +2450815|380|3|764| +2450815|382|3|157| +2450815|385|3|57| +2450815|386|3|578| +2450815|388|3|148| +2450815|391|3|933| +2450815|392|3|240| +2450815|394|3|205| +2450815|397|3|263| +2450815|398|3|801| +2450815|400|3|857| +2450815|403|3|110| +2450815|404|3|691| +2450815|406|3|938| +2450815|409|3|762| +2450815|410|3|| +2450815|412|3|216| +2450815|415|3|742| +2450815|416|3|189| +2450815|418|3|188| +2450815|421|3|679| +2450815|422|3|85| +2450815|424|3|401| +2450815|427|3|854| +2450815|428|3|813| +2450815|430|3|17| +2450815|433|3|390| +2450815|434|3|956| +2450815|436|3|466| +2450815|439|3|958| +2450815|440|3|763| +2450815|442|3|85| +2450815|445|3|707| +2450815|446|3|163| +2450815|448|3|232| +2450815|451|3|822| +2450815|452|3|500| +2450815|454|3|69| +2450815|457|3|171| +2450815|458|3|| +2450815|460|3|136| +2450815|463|3|197| +2450815|464|3|220| +2450815|466|3|607| +2450815|469|3|420| +2450815|470|3|848| +2450815|472|3|154| +2450815|475|3|174| +2450815|476|3|183| +2450815|478|3|173| +2450815|481|3|756| +2450815|482|3|571| +2450815|484|3|816| +2450815|487|3|224| +2450815|488|3|987| +2450815|490|3|611| +2450815|493|3|303| +2450815|494|3|59| +2450815|496|3|531| +2450815|499|3|1000| +2450815|500|3|561| +2450815|502|3|137| +2450815|505|3|642| +2450815|506|3|990| +2450815|508|3|60| +2450815|511|3|85| +2450815|512|3|237| +2450815|514|3|146| +2450815|517|3|288| +2450815|518|3|| +2450815|520|3|776| +2450815|523|3|852| +2450815|524|3|989| +2450815|526|3|374| +2450815|529|3|15| +2450815|530|3|438| +2450815|532|3|621| +2450815|535|3|225| +2450815|536|3|980| +2450815|538|3|437| +2450815|541|3|806| +2450815|542|3|712| +2450815|544|3|578| +2450815|547|3|221| +2450815|548|3|852| +2450815|550|3|236| +2450815|553|3|882| +2450815|554|3|124| +2450815|556|3|790| +2450815|559|3|661| +2450815|560|3|214| +2450815|562|3|512| +2450815|565|3|644| +2450815|566|3|356| +2450815|568|3|830| +2450815|571|3|412| +2450815|572|3|673| +2450815|574|3|872| +2450815|577|3|363| +2450815|578|3|674| +2450815|580|3|272| +2450815|583|3|545| +2450815|584|3|73| +2450815|586|3|780| +2450815|589|3|266| +2450815|590|3|528| +2450815|592|3|| +2450815|595|3|74| +2450815|596|3|149| +2450815|598|3|163| +2450815|601|3|246| +2450815|602|3|479| +2450815|604|3|835| +2450815|607|3|314| +2450815|608|3|745| +2450815|610|3|653| +2450815|613|3|229| +2450815|614|3|52| +2450815|616|3|687| +2450815|619|3|632| +2450815|620|3|103| +2450815|622|3|642| +2450815|625|3|830| +2450815|626|3|485| +2450815|628|3|184| +2450815|631|3|448| +2450815|632|3|387| +2450815|634|3|881| +2450815|637|3|379| +2450815|638|3|596| +2450815|640|3|176| +2450815|643|3|872| +2450815|644|3|759| +2450815|646|3|373| +2450815|649|3|578| +2450815|650|3|751| +2450815|652|3|313| +2450815|655|3|596| +2450815|656|3|984| +2450815|658|3|290| +2450815|661|3|398| +2450815|662|3|35| +2450815|664|3|| +2450815|667|3|436| +2450815|668|3|60| +2450815|670|3|| +2450815|673|3|499| +2450815|674|3|625| +2450815|676|3|103| +2450815|679|3|786| +2450815|680|3|58| +2450815|682|3|125| +2450815|685|3|283| +2450815|686|3|407| +2450815|688|3|214| +2450815|691|3|221| +2450815|692|3|847| +2450815|694|3|44| +2450815|697|3|939| +2450815|698|3|59| +2450815|700|3|316| +2450815|703|3|961| +2450815|704|3|380| +2450815|706|3|297| +2450815|709|3|58| +2450815|710|3|847| +2450815|712|3|892| +2450815|715|3|857| +2450815|716|3|122| +2450815|718|3|476| +2450815|721|3|197| +2450815|722|3|983| +2450815|724|3|695| +2450815|727|3|| +2450815|728|3|907| +2450815|730|3|462| +2450815|733|3|329| +2450815|734|3|476| +2450815|736|3|649| +2450815|739|3|346| +2450815|740|3|166| +2450815|742|3|920| +2450815|745|3|851| +2450815|746|3|774| +2450815|748|3|143| +2450815|751|3|430| +2450815|752|3|131| +2450815|754|3|323| +2450815|757|3|783| +2450815|758|3|| +2450815|760|3|462| +2450815|763|3|671| +2450815|764|3|22| +2450815|766|3|329| +2450815|769|3|543| +2450815|770|3|817| +2450815|772|3|18| +2450815|775|3|| +2450815|776|3|33| +2450815|778|3|547| +2450815|781|3|27| +2450815|782|3|17| +2450815|784|3|937| +2450815|787|3|9| +2450815|788|3|711| +2450815|790|3|896| +2450815|793|3|2| +2450815|794|3|131| +2450815|796|3|684| +2450815|799|3|276| +2450815|800|3|32| +2450815|802|3|503| +2450815|805|3|860| +2450815|806|3|902| +2450815|808|3|898| +2450815|811|3|907| +2450815|812|3|114| +2450815|814|3|602| +2450815|817|3|770| +2450815|818|3|436| +2450815|820|3|| +2450815|823|3|878| +2450815|824|3|94| +2450815|826|3|377| +2450815|829|3|64| +2450815|830|3|80| +2450815|832|3|589| +2450815|835|3|577| +2450815|836|3|484| +2450815|838|3|480| +2450815|841|3|35| +2450815|842|3|47| +2450815|844|3|718| +2450815|847|3|959| +2450815|848|3|49| +2450815|850|3|555| +2450815|853|3|392| +2450815|854|3|412| +2450815|856|3|172| +2450815|859|3|39| +2450815|860|3|737| +2450815|862|3|808| +2450815|865|3|27| +2450815|866|3|691| +2450815|868|3|167| +2450815|871|3|345| +2450815|872|3|51| +2450815|874|3|336| +2450815|877|3|870| +2450815|878|3|518| +2450815|880|3|623| +2450815|883|3|465| +2450815|884|3|302| +2450815|886|3|283| +2450815|889|3|831| +2450815|890|3|682| +2450815|892|3|909| +2450815|895|3|516| +2450815|896|3|| +2450815|898|3|108| +2450815|901|3|350| +2450815|902|3|625| +2450815|904|3|578| +2450815|907|3|174| +2450815|908|3|388| +2450815|910|3|791| +2450815|913|3|679| +2450815|914|3|459| +2450815|916|3|555| +2450815|919|3|944| +2450815|920|3|567| +2450815|922|3|343| +2450815|925|3|40| +2450815|926|3|129| +2450815|928|3|147| +2450815|931|3|338| +2450815|932|3|408| +2450815|934|3|540| +2450815|937|3|| +2450815|938|3|394| +2450815|940|3|106| +2450815|943|3|416| +2450815|944|3|966| +2450815|946|3|836| +2450815|949|3|898| +2450815|950|3|110| +2450815|952|3|663| +2450815|955|3|354| +2450815|956|3|509| +2450815|958|3|545| +2450815|961|3|632| +2450815|962|3|496| +2450815|964|3|929| +2450815|967|3|629| +2450815|968|3|464| +2450815|970|3|507| +2450815|973|3|180| +2450815|974|3|664| +2450815|976|3|| +2450815|979|3|552| +2450815|980|3|819| +2450815|982|3|556| +2450815|985|3|158| +2450815|986|3|31| +2450815|988|3|| +2450815|991|3|| +2450815|992|3|| +2450815|994|3|44| +2450815|997|3|599| +2450815|998|3|636| +2450815|1000|3|18| +2450815|1003|3|423| +2450815|1004|3|219| +2450815|1006|3|691| +2450815|1009|3|470| +2450815|1010|3|689| +2450815|1012|3|37| +2450815|1015|3|957| +2450815|1016|3|289| +2450815|1018|3|515| +2450815|1021|3|408| +2450815|1022|3|434| +2450815|1024|3|392| +2450815|1027|3|268| +2450815|1028|3|612| +2450815|1030|3|229| +2450815|1033|3|608| +2450815|1034|3|18| +2450815|1036|3|757| +2450815|1039|3|958| +2450815|1040|3|413| +2450815|1042|3|711| +2450815|1045|3|901| +2450815|1046|3|430| +2450815|1048|3|349| +2450815|1051|3|791| +2450815|1052|3|760| +2450815|1054|3|693| +2450815|1057|3|829| +2450815|1058|3|50| +2450815|1060|3|200| +2450815|1063|3|28| +2450815|1064|3|474| +2450815|1066|3|813| +2450815|1069|3|| +2450815|1070|3|430| +2450815|1072|3|511| +2450815|1075|3|205| +2450815|1076|3|797| +2450815|1078|3|148| +2450815|1081|3|293| +2450815|1082|3|393| +2450815|1084|3|13| +2450815|1087|3|317| +2450815|1088|3|224| +2450815|1090|3|249| +2450815|1093|3|| +2450815|1094|3|799| +2450815|1096|3|934| +2450815|1099|3|511| +2450815|1100|3|100| +2450815|1102|3|300| +2450815|1105|3|358| +2450815|1106|3|538| +2450815|1108|3|959| +2450815|1111|3|779| +2450815|1112|3|77| +2450815|1114|3|149| +2450815|1117|3|76| +2450815|1118|3|701| +2450815|1120|3|845| +2450815|1123|3|598| +2450815|1124|3|186| +2450815|1126|3|468| +2450815|1129|3|944| +2450815|1130|3|116| +2450815|1132|3|401| +2450815|1135|3|462| +2450815|1136|3|207| +2450815|1138|3|982| +2450815|1141|3|141| +2450815|1142|3|992| +2450815|1144|3|273| +2450815|1147|3|142| +2450815|1148|3|807| +2450815|1150|3|930| +2450815|1153|3|| +2450815|1154|3|619| +2450815|1156|3|597| +2450815|1159|3|273| +2450815|1160|3|931| +2450815|1162|3|79| +2450815|1165|3|| +2450815|1166|3|19| +2450815|1168|3|13| +2450815|1171|3|937| +2450815|1172|3|342| +2450815|1174|3|290| +2450815|1177|3|980| +2450815|1178|3|174| +2450815|1180|3|584| +2450815|1183|3|| +2450815|1184|3|929| +2450815|1186|3|246| +2450815|1189|3|347| +2450815|1190|3|767| +2450815|1192|3|231| +2450815|1195|3|757| +2450815|1196|3|797| +2450815|1198|3|204| +2450815|1201|3|571| +2450815|1202|3|595| +2450815|1204|3|745| +2450815|1207|3|766| +2450815|1208|3|557| +2450815|1210|3|| +2450815|1213|3|735| +2450815|1214|3|633| +2450815|1216|3|10| +2450815|1219|3|673| +2450815|1220|3|671| +2450815|1222|3|925| +2450815|1225|3|644| +2450815|1226|3|599| +2450815|1228|3|272| +2450815|1231|3|372| +2450815|1232|3|672| +2450815|1234|3|370| +2450815|1237|3|737| +2450815|1238|3|589| +2450815|1240|3|27| +2450815|1243|3|| +2450815|1244|3|947| +2450815|1246|3|177| +2450815|1249|3|163| +2450815|1250|3|587| +2450815|1252|3|823| +2450815|1255|3|760| +2450815|1256|3|334| +2450815|1258|3|450| +2450815|1261|3|391| +2450815|1262|3|80| +2450815|1264|3|859| +2450815|1267|3|938| +2450815|1268|3|435| +2450815|1270|3|592| +2450815|1273|3|398| +2450815|1274|3|919| +2450815|1276|3|671| +2450815|1279|3|107| +2450815|1280|3|449| +2450815|1282|3|441| +2450815|1285|3|983| +2450815|1286|3|| +2450815|1288|3|701| +2450815|1291|3|210| +2450815|1292|3|156| +2450815|1294|3|650| +2450815|1297|3|200| +2450815|1298|3|197| +2450815|1300|3|454| +2450815|1303|3|537| +2450815|1304|3|241| +2450815|1306|3|357| +2450815|1309|3|48| +2450815|1310|3|708| +2450815|1312|3|739| +2450815|1315|3|161| +2450815|1316|3|832| +2450815|1318|3|285| +2450815|1321|3|487| +2450815|1322|3|703| +2450815|1324|3|3| +2450815|1327|3|435| +2450815|1328|3|519| +2450815|1330|3|973| +2450815|1333|3|17| +2450815|1334|3|171| +2450815|1336|3|799| +2450815|1339|3|598| +2450815|1340|3|374| +2450815|1342|3|834| +2450815|1345|3|103| +2450815|1346|3|239| +2450815|1348|3|422| +2450815|1351|3|568| +2450815|1352|3|244| +2450815|1354|3|| +2450815|1357|3|| +2450815|1358|3|692| +2450815|1360|3|50| +2450815|1363|3|416| +2450815|1364|3|827| +2450815|1366|3|276| +2450815|1369|3|337| +2450815|1370|3|104| +2450815|1372|3|184| +2450815|1375|3|302| +2450815|1376|3|420| +2450815|1378|3|299| +2450815|1381|3|457| +2450815|1382|3|494| +2450815|1384|3|513| +2450815|1387|3|715| +2450815|1388|3|168| +2450815|1390|3|651| +2450815|1393|3|804| +2450815|1394|3|612| +2450815|1396|3|211| +2450815|1399|3|53| +2450815|1400|3|141| +2450815|1402|3|784| +2450815|1405|3|741| +2450815|1406|3|702| +2450815|1408|3|312| +2450815|1411|3|676| +2450815|1412|3|585| +2450815|1414|3|339| +2450815|1417|3|582| +2450815|1418|3|925| +2450815|1420|3|| +2450815|1423|3|518| +2450815|1424|3|435| +2450815|1426|3|211| +2450815|1429|3|636| +2450815|1430|3|766| +2450815|1432|3|206| +2450815|1435|3|249| +2450815|1436|3|260| +2450815|1438|3|45| +2450815|1441|3|473| +2450815|1442|3|| +2450815|1444|3|658| +2450815|1447|3|232| +2450815|1448|3|720| +2450815|1450|3|317| +2450815|1453|3|573| +2450815|1454|3|634| +2450815|1456|3|79| +2450815|1459|3|371| +2450815|1460|3|| +2450815|1462|3|126| +2450815|1465|3|652| +2450815|1466|3|752| +2450815|1468|3|714| +2450815|1471|3|712| +2450815|1472|3|475| +2450815|1474|3|113| +2450815|1477|3|717| +2450815|1478|3|| +2450815|1480|3|| +2450815|1483|3|228| +2450815|1484|3|581| +2450815|1486|3|609| +2450815|1489|3|158| +2450815|1490|3|162| +2450815|1492|3|746| +2450815|1495|3|838| +2450815|1496|3|342| +2450815|1498|3|328| +2450815|1501|3|506| +2450815|1502|3|585| +2450815|1504|3|902| +2450815|1507|3|125| +2450815|1508|3|972| +2450815|1510|3|765| +2450815|1513|3|462| +2450815|1514|3|546| +2450815|1516|3|399| +2450815|1519|3|880| +2450815|1520|3|835| +2450815|1522|3|758| +2450815|1525|3|481| +2450815|1526|3|897| +2450815|1528|3|633| +2450815|1531|3|341| +2450815|1532|3|809| +2450815|1534|3|402| +2450815|1537|3|118| +2450815|1538|3|81| +2450815|1540|3|586| +2450815|1543|3|576| +2450815|1544|3|425| +2450815|1546|3|321| +2450815|1549|3|653| +2450815|1550|3|128| +2450815|1552|3|447| +2450815|1555|3|| +2450815|1556|3|527| +2450815|1558|3|30| +2450815|1561|3|16| +2450815|1562|3|595| +2450815|1564|3|525| +2450815|1567|3|208| +2450815|1568|3|141| +2450815|1570|3|283| +2450815|1573|3|656| +2450815|1574|3|40| +2450815|1576|3|707| +2450815|1579|3|334| +2450815|1580|3|244| +2450815|1582|3|693| +2450815|1585|3|722| +2450815|1586|3|36| +2450815|1588|3|981| +2450815|1591|3|602| +2450815|1592|3|584| +2450815|1594|3|401| +2450815|1597|3|813| +2450815|1598|3|613| +2450815|1600|3|104| +2450815|1603|3|105| +2450815|1604|3|651| +2450815|1606|3|793| +2450815|1609|3|16| +2450815|1610|3|760| +2450815|1612|3|842| +2450815|1615|3|412| +2450815|1616|3|210| +2450815|1618|3|406| +2450815|1621|3|686| +2450815|1622|3|790| +2450815|1624|3|882| +2450815|1627|3|550| +2450815|1628|3|733| +2450815|1630|3|149| +2450815|1633|3|668| +2450815|1634|3|460| +2450815|1636|3|782| +2450815|1639|3|342| +2450815|1640|3|281| +2450815|1642|3|865| +2450815|1645|3|695| +2450815|1646|3|628| +2450815|1648|3|843| +2450815|1651|3|495| +2450815|1652|3|794| +2450815|1654|3|609| +2450815|1657|3|179| +2450815|1658|3|8| +2450815|1660|3|| +2450815|1663|3|561| +2450815|1664|3|128| +2450815|1666|3|148| +2450815|1669|3|816| +2450815|1670|3|157| +2450815|1672|3|838| +2450815|1675|3|9| +2450815|1676|3|587| +2450815|1678|3|188| +2450815|1681|3|958| +2450815|1682|3|727| +2450815|1684|3|748| +2450815|1687|3|283| +2450815|1688|3|883| +2450815|1690|3|356| +2450815|1693|3|91| +2450815|1694|3|290| +2450815|1696|3|633| +2450815|1699|3|986| +2450815|1700|3|63| +2450815|1702|3|633| +2450815|1705|3|770| +2450815|1706|3|106| +2450815|1708|3|439| +2450815|1711|3|603| +2450815|1712|3|144| +2450815|1714|3|427| +2450815|1717|3|943| +2450815|1718|3|318| +2450815|1720|3|47| +2450815|1723|3|331| +2450815|1724|3|250| +2450815|1726|3|832| +2450815|1729|3|783| +2450815|1730|3|79| +2450815|1732|3|310| +2450815|1735|3|457| +2450815|1736|3|910| +2450815|1738|3|923| +2450815|1741|3|558| +2450815|1742|3|407| +2450815|1744|3|244| +2450815|1747|3|664| +2450815|1748|3|802| +2450815|1750|3|151| +2450815|1753|3|253| +2450815|1754|3|680| +2450815|1756|3|421| +2450815|1759|3|862| +2450815|1760|3|118| +2450815|1762|3|481| +2450815|1765|3|292| +2450815|1766|3|943| +2450815|1768|3|456| +2450815|1771|3|650| +2450815|1772|3|652| +2450815|1774|3|410| +2450815|1777|3|901| +2450815|1778|3|518| +2450815|1780|3|24| +2450815|1783|3|561| +2450815|1784|3|793| +2450815|1786|3|971| +2450815|1789|3|937| +2450815|1790|3|851| +2450815|1792|3|828| +2450815|1795|3|| +2450815|1796|3|| +2450815|1798|3|87| +2450815|1801|3|635| +2450815|1802|3|298| +2450815|1804|3|951| +2450815|1807|3|566| +2450815|1808|3|886| +2450815|1810|3|85| +2450815|1813|3|213| +2450815|1814|3|694| +2450815|1816|3|839| +2450815|1819|3|894| +2450815|1820|3|651| +2450815|1822|3|463| +2450815|1825|3|859| +2450815|1826|3|98| +2450815|1828|3|791| +2450815|1831|3|922| +2450815|1832|3|740| +2450815|1834|3|24| +2450815|1837|3|108| +2450815|1838|3|109| +2450815|1840|3|451| +2450815|1843|3|668| +2450815|1844|3|747| +2450815|1846|3|121| +2450815|1849|3|579| +2450815|1850|3|785| +2450815|1852|3|967| +2450815|1855|3|829| +2450815|1856|3|354| +2450815|1858|3|973| +2450815|1861|3|148| +2450815|1862|3|823| +2450815|1864|3|432| +2450815|1867|3|914| +2450815|1868|3|877| +2450815|1870|3|9| +2450815|1873|3|531| +2450815|1874|3|196| +2450815|1876|3|302| +2450815|1879|3|11| +2450815|1880|3|339| +2450815|1882|3|450| +2450815|1885|3|962| +2450815|1886|3|830| +2450815|1888|3|250| +2450815|1891|3|925| +2450815|1892|3|50| +2450815|1894|3|30| +2450815|1897|3|213| +2450815|1898|3|223| +2450815|1900|3|185| +2450815|1903|3|447| +2450815|1904|3|786| +2450815|1906|3|170| +2450815|1909|3|33| +2450815|1910|3|594| +2450815|1912|3|297| +2450815|1915|3|808| +2450815|1916|3|144| +2450815|1918|3|545| +2450815|1921|3|465| +2450815|1922|3|833| +2450815|1924|3|524| +2450815|1927|3|858| +2450815|1928|3|952| +2450815|1930|3|766| +2450815|1933|3|537| +2450815|1934|3|116| +2450815|1936|3|284| +2450815|1939|3|923| +2450815|1940|3|639| +2450815|1942|3|957| +2450815|1945|3|344| +2450815|1946|3|576| +2450815|1948|3|971| +2450815|1951|3|737| +2450815|1952|3|461| +2450815|1954|3|812| +2450815|1957|3|361| +2450815|1958|3|970| +2450815|1960|3|714| +2450815|1963|3|519| +2450815|1964|3|901| +2450815|1966|3|730| +2450815|1969|3|220| +2450815|1970|3|267| +2450815|1972|3|293| +2450815|1975|3|438| +2450815|1976|3|485| +2450815|1978|3|706| +2450815|1981|3|| +2450815|1982|3|779| +2450815|1984|3|904| +2450815|1987|3|623| +2450815|1988|3|946| +2450815|1990|3|722| +2450815|1993|3|258| +2450815|1994|3|898| +2450815|1996|3|596| +2450815|1999|3|878| +2450815|2000|3|239| +2450815|2002|3|957| +2450815|2005|3|485| +2450815|2006|3|737| +2450815|2008|3|167| +2450815|2011|3|508| +2450815|2012|3|794| +2450815|2014|3|143| +2450815|2017|3|420| +2450815|2018|3|67| +2450815|2020|3|107| +2450815|2023|3|858| +2450815|2024|3|634| +2450815|2026|3|562| +2450815|2029|3|10| +2450815|2030|3|316| +2450815|2032|3|| +2450815|2035|3|| +2450815|2036|3|318| +2450815|2038|3|629| +2450815|2041|3|880| +2450815|2042|3|532| +2450815|2044|3|573| +2450815|2047|3|844| +2450815|2048|3|446| +2450815|2050|3|26| +2450815|2053|3|211| +2450815|2054|3|768| +2450815|2056|3|572| +2450815|2059|3|86| +2450815|2060|3|974| +2450815|2062|3|857| +2450815|2065|3|515| +2450815|2066|3|134| +2450815|2068|3|885| +2450815|2071|3|456| +2450815|2072|3|85| +2450815|2074|3|579| +2450815|2077|3|775| +2450815|2078|3|87| +2450815|2080|3|417| +2450815|2083|3|32| +2450815|2084|3|58| +2450815|2086|3|834| +2450815|2089|3|879| +2450815|2090|3|765| +2450815|2092|3|| +2450815|2095|3|344| +2450815|2096|3|217| +2450815|2098|3|140| +2450815|2101|3|888| +2450815|2102|3|783| +2450815|2104|3|958| +2450815|2107|3|76| +2450815|2108|3|317| +2450815|2110|3|91| +2450815|2113|3|35| +2450815|2114|3|138| +2450815|2116|3|401| +2450815|2119|3|714| +2450815|2120|3|441| +2450815|2122|3|271| +2450815|2125|3|334| +2450815|2126|3|| +2450815|2128|3|| +2450815|2131|3|61| +2450815|2132|3|713| +2450815|2134|3|27| +2450815|2137|3|454| +2450815|2138|3|300| +2450815|2140|3|645| +2450815|2143|3|675| +2450815|2144|3|146| +2450815|2146|3|637| +2450815|2149|3|50| +2450815|2150|3|426| +2450815|2152|3|956| +2450815|2155|3|175| +2450815|2156|3|688| +2450815|2158|3|652| +2450815|2161|3|188| +2450815|2162|3|194| +2450815|2164|3|178| +2450815|2167|3|993| +2450815|2168|3|608| +2450815|2170|3|471| +2450815|2173|3|91| +2450815|2174|3|574| +2450815|2176|3|221| +2450815|2179|3|411| +2450815|2180|3|814| +2450815|2182|3|761| +2450815|2185|3|167| +2450815|2186|3|917| +2450815|2188|3|| +2450815|2191|3|666| +2450815|2192|3|648| +2450815|2194|3|919| +2450815|2197|3|355| +2450815|2198|3|514| +2450815|2200|3|926| +2450815|2203|3|716| +2450815|2204|3|963| +2450815|2206|3|783| +2450815|2209|3|889| +2450815|2210|3|48| +2450815|2212|3|127| +2450815|2215|3|458| +2450815|2216|3|435| +2450815|2218|3|96| +2450815|2221|3|471| +2450815|2222|3|367| +2450815|2224|3|515| +2450815|2227|3|799| +2450815|2228|3|360| +2450815|2230|3|160| +2450815|2233|3|558| +2450815|2234|3|922| +2450815|2236|3|212| +2450815|2239|3|225| +2450815|2240|3|106| +2450815|2242|3|162| +2450815|2245|3|42| +2450815|2246|3|514| +2450815|2248|3|299| +2450815|2251|3|780| +2450815|2252|3|898| +2450815|2254|3|455| +2450815|2257|3|970| +2450815|2258|3|976| +2450815|2260|3|| +2450815|2263|3|939| +2450815|2264|3|468| +2450815|2266|3|639| +2450815|2269|3|940| +2450815|2270|3|547| +2450815|2272|3|985| +2450815|2275|3|169| +2450815|2276|3|241| +2450815|2278|3|513| +2450815|2281|3|392| +2450815|2282|3|196| +2450815|2284|3|23| +2450815|2287|3|395| +2450815|2288|3|773| +2450815|2290|3|986| +2450815|2293|3|3| +2450815|2294|3|| +2450815|2296|3|| +2450815|2299|3|430| +2450815|2300|3|560| +2450815|2302|3|440| +2450815|2305|3|| +2450815|2306|3|14| +2450815|2308|3|265| +2450815|2311|3|839| +2450815|2312|3|197| +2450815|2314|3|985| +2450815|2317|3|52| +2450815|2318|3|642| +2450815|2320|3|890| +2450815|2323|3|846| +2450815|2324|3|857| +2450815|2326|3|194| +2450815|2329|3|353| +2450815|2330|3|7| +2450815|2332|3|982| +2450815|2335|3|999| +2450815|2336|3|830| +2450815|2338|3|184| +2450815|2341|3|343| +2450815|2342|3|631| +2450815|2344|3|405| +2450815|2347|3|| +2450815|2348|3|438| +2450815|2350|3|250| +2450815|2353|3|408| +2450815|2354|3|892| +2450815|2356|3|277| +2450815|2359|3|674| +2450815|2360|3|369| +2450815|2362|3|306| +2450815|2365|3|753| +2450815|2366|3|11| +2450815|2368|3|60| +2450815|2371|3|129| +2450815|2372|3|783| +2450815|2374|3|65| +2450815|2377|3|| +2450815|2378|3|19| +2450815|2380|3|257| +2450815|2383|3|418| +2450815|2384|3|655| +2450815|2386|3|611| +2450815|2389|3|893| +2450815|2390|3|343| +2450815|2392|3|482| +2450815|2395|3|346| +2450815|2396|3|171| +2450815|2398|3|833| +2450815|2401|3|858| +2450815|2402|3|527| +2450815|2404|3|475| +2450815|2407|3|225| +2450815|2408|3|808| +2450815|2410|3|414| +2450815|2413|3|779| +2450815|2414|3|543| +2450815|2416|3|922| +2450815|2419|3|273| +2450815|2420|3|310| +2450815|2422|3|109| +2450815|2425|3|591| +2450815|2426|3|361| +2450815|2428|3|161| +2450815|2431|3|10| +2450815|2432|3|129| +2450815|2434|3|531| +2450815|2437|3|249| +2450815|2438|3|333| +2450815|2440|3|377| +2450815|2443|3|693| +2450815|2444|3|541| +2450815|2446|3|364| +2450815|2449|3|106| +2450815|2450|3|60| +2450815|2452|3|431| +2450815|2455|3|200| +2450815|2456|3|891| +2450815|2458|3|940| +2450815|2461|3|8| +2450815|2462|3|321| +2450815|2464|3|637| +2450815|2467|3|278| +2450815|2468|3|35| +2450815|2470|3|906| +2450815|2473|3|432| +2450815|2474|3|344| +2450815|2476|3|518| +2450815|2479|3|894| +2450815|2480|3|133| +2450815|2482|3|351| +2450815|2485|3|818| +2450815|2486|3|676| +2450815|2488|3|436| +2450815|2491|3|379| +2450815|2492|3|7| +2450815|2494|3|751| +2450815|2497|3|443| +2450815|2498|3|123| +2450815|2500|3|109| +2450815|2503|3|280| +2450815|2504|3|240| +2450815|2506|3|420| +2450815|2509|3|156| +2450815|2510|3|702| +2450815|2512|3|478| +2450815|2515|3|633| +2450815|2516|3|| +2450815|2518|3|728| +2450815|2521|3|81| +2450815|2522|3|936| +2450815|2524|3|494| +2450815|2527|3|136| +2450815|2528|3|552| +2450815|2530|3|147| +2450815|2533|3|231| +2450815|2534|3|910| +2450815|2536|3|| +2450815|2539|3|808| +2450815|2540|3|220| +2450815|2542|3|873| +2450815|2545|3|405| +2450815|2546|3|338| +2450815|2548|3|81| +2450815|2551|3|| +2450815|2552|3|132| +2450815|2554|3|752| +2450815|2557|3|692| +2450815|2558|3|73| +2450815|2560|3|510| +2450815|2563|3|176| +2450815|2564|3|701| +2450815|2566|3|457| +2450815|2569|3|| +2450815|2570|3|333| +2450815|2572|3|432| +2450815|2575|3|| +2450815|2576|3|573| +2450815|2578|3|594| +2450815|2581|3|91| +2450815|2582|3|474| +2450815|2584|3|59| +2450815|2587|3|984| +2450815|2588|3|411| +2450815|2590|3|| +2450815|2593|3|467| +2450815|2594|3|435| +2450815|2596|3|172| +2450815|2599|3|741| +2450815|2600|3|574| +2450815|2602|3|169| +2450815|2605|3|902| +2450815|2606|3|608| +2450815|2608|3|970| +2450815|2611|3|801| +2450815|2612|3|836| +2450815|2614|3|855| +2450815|2617|3|806| +2450815|2618|3|623| +2450815|2620|3|965| +2450815|2623|3|584| +2450815|2624|3|914| +2450815|2626|3|158| +2450815|2629|3|12| +2450815|2630|3|200| +2450815|2632|3|458| +2450815|2635|3|256| +2450815|2636|3|834| +2450815|2638|3|756| +2450815|2641|3|453| +2450815|2642|3|| +2450815|2644|3|689| +2450815|2647|3|960| +2450815|2648|3|524| +2450815|2650|3|| +2450815|2653|3|470| +2450815|2654|3|227| +2450815|2656|3|439| +2450815|2659|3|29| +2450815|2660|3|684| +2450815|2662|3|808| +2450815|2665|3|459| +2450815|2666|3|509| +2450815|2668|3|94| +2450815|2671|3|3| +2450815|2672|3|539| +2450815|2674|3|614| +2450815|2677|3|729| +2450815|2678|3|699| +2450815|2680|3|931| +2450815|2683|3|144| +2450815|2684|3|416| +2450815|2686|3|429| +2450815|2689|3|267| +2450815|2690|3|486| +2450815|2692|3|3| +2450815|2695|3|608| +2450815|2696|3|161| +2450815|2698|3|796| +2450815|2701|3|60| +2450815|2702|3|719| +2450815|2704|3|799| +2450815|2707|3|726| +2450815|2708|3|406| +2450815|2710|3|459| +2450815|2713|3|296| +2450815|2714|3|956| +2450815|2716|3|94| +2450815|2719|3|97| +2450815|2720|3|834| +2450815|2722|3|515| +2450815|2725|3|940| +2450815|2726|3|520| +2450815|2728|3|531| +2450815|2731|3|825| +2450815|2732|3|949| +2450815|2734|3|384| +2450815|2737|3|125| +2450815|2738|3|377| +2450815|2740|3|243| +2450815|2743|3|976| +2450815|2744|3|991| +2450815|2746|3|305| +2450815|2749|3|112| +2450815|2750|3|101| +2450815|2752|3|151| +2450815|2755|3|698| +2450815|2756|3|| +2450815|2758|3|335| +2450815|2761|3|268| +2450815|2762|3|249| +2450815|2764|3|222| +2450815|2767|3|203| +2450815|2768|3|933| +2450815|2770|3|| +2450815|2773|3|| +2450815|2774|3|932| +2450815|2776|3|79| +2450815|2779|3|314| +2450815|2780|3|40| +2450815|2782|3|953| +2450815|2785|3|518| +2450815|2786|3|126| +2450815|2788|3|938| +2450815|2791|3|796| +2450815|2792|3|| +2450815|2794|3|536| +2450815|2797|3|887| +2450815|2798|3|682| +2450815|2800|3|994| +2450815|2803|3|299| +2450815|2804|3|657| +2450815|2806|3|559| +2450815|2809|3|879| +2450815|2810|3|755| +2450815|2812|3|362| +2450815|2815|3|715| +2450815|2816|3|993| +2450815|2818|3|999| +2450815|2821|3|297| +2450815|2822|3|742| +2450815|2824|3|697| +2450815|2827|3|981| +2450815|2828|3|14| +2450815|2830|3|959| +2450815|2833|3|452| +2450815|2834|3|524| +2450815|2836|3|939| +2450815|2839|3|663| +2450815|2840|3|845| +2450815|2842|3|203| +2450815|2845|3|417| +2450815|2846|3|226| +2450815|2848|3|606| +2450815|2851|3|797| +2450815|2852|3|225| +2450815|2854|3|247| +2450815|2857|3|489| +2450815|2858|3|234| +2450815|2860|3|649| +2450815|2863|3|| +2450815|2864|3|788| +2450815|2866|3|522| +2450815|2869|3|561| +2450815|2870|3|486| +2450815|2872|3|118| +2450815|2875|3|92| +2450815|2876|3|875| +2450815|2878|3|381| +2450815|2881|3|267| +2450815|2882|3|716| +2450815|2884|3|522| +2450815|2887|3|703| +2450815|2888|3|856| +2450815|2890|3|791| +2450815|2893|3|594| +2450815|2894|3|230| +2450815|2896|3|224| +2450815|2899|3|271| +2450815|2900|3|932| +2450815|2902|3|649| +2450815|2905|3|| +2450815|2906|3|846| +2450815|2908|3|669| +2450815|2911|3|29| +2450815|2912|3|650| +2450815|2914|3|21| +2450815|2917|3|447| +2450815|2918|3|639| +2450815|2920|3|127| +2450815|2923|3|491| +2450815|2924|3|516| +2450815|2926|3|| +2450815|2929|3|869| +2450815|2930|3|786| +2450815|2932|3|882| +2450815|2935|3|222| +2450815|2936|3|220| +2450815|2938|3|798| +2450815|2941|3|208| +2450815|2942|3|243| +2450815|2944|3|404| +2450815|2947|3|196| +2450815|2948|3|548| +2450815|2950|3|4| +2450815|2953|3|266| +2450815|2954|3|63| +2450815|2956|3|958| +2450815|2959|3|860| +2450815|2960|3|786| +2450815|2962|3|473| +2450815|2965|3|409| +2450815|2966|3|601| +2450815|2968|3|339| +2450815|2971|3|| +2450815|2972|3|480| +2450815|2974|3|478| +2450815|2977|3|| +2450815|2978|3|698| +2450815|2980|3|972| +2450815|2983|3|771| +2450815|2984|3|332| +2450815|2986|3|516| +2450815|2989|3|959| +2450815|2990|3|816| +2450815|2992|3|766| +2450815|2995|3|759| +2450815|2996|3|343| +2450815|2998|3|| +2450815|3001|3|386| +2450815|3002|3|334| +2450815|3004|3|846| +2450815|3007|3|865| +2450815|3008|3|219| +2450815|3010|3|393| +2450815|3013|3|985| +2450815|3014|3|688| +2450815|3016|3|509| +2450815|3019|3|440| +2450815|3020|3|516| +2450815|3022|3|492| +2450815|3025|3|444| +2450815|3026|3|977| +2450815|3028|3|723| +2450815|3031|3|92| +2450815|3032|3|830| +2450815|3034|3|765| +2450815|3037|3|414| +2450815|3038|3|54| +2450815|3040|3|923| +2450815|3043|3|850| +2450815|3044|3|868| +2450815|3046|3|309| +2450815|3049|3|209| +2450815|3050|3|286| +2450815|3052|3|813| +2450815|3055|3|105| +2450815|3056|3|863| +2450815|3058|3|246| +2450815|3061|3|379| +2450815|3062|3|298| +2450815|3064|3|289| +2450815|3067|3|261| +2450815|3068|3|337| +2450815|3070|3|894| +2450815|3073|3|596| +2450815|3074|3|406| +2450815|3076|3|652| +2450815|3079|3|584| +2450815|3080|3|543| +2450815|3082|3|579| +2450815|3085|3|963| +2450815|3086|3|| +2450815|3088|3|206| +2450815|3091|3|346| +2450815|3092|3|204| +2450815|3094|3|180| +2450815|3097|3|| +2450815|3098|3|611| +2450815|3100|3|191| +2450815|3103|3|627| +2450815|3104|3|609| +2450815|3106|3|900| +2450815|3109|3|995| +2450815|3110|3|475| +2450815|3112|3|891| +2450815|3115|3|544| +2450815|3116|3|176| +2450815|3118|3|496| +2450815|3121|3|477| +2450815|3122|3|315| +2450815|3124|3|733| +2450815|3127|3|855| +2450815|3128|3|562| +2450815|3130|3|| +2450815|3133|3|289| +2450815|3134|3|475| +2450815|3136|3|703| +2450815|3139|3|234| +2450815|3140|3|545| +2450815|3142|3|26| +2450815|3145|3|258| +2450815|3146|3|369| +2450815|3148|3|825| +2450815|3151|3|810| +2450815|3152|3|726| +2450815|3154|3|841| +2450815|3157|3|940| +2450815|3158|3|686| +2450815|3160|3|437| +2450815|3163|3|811| +2450815|3164|3|939| +2450815|3166|3|84| +2450815|3169|3|799| +2450815|3170|3|258| +2450815|3172|3|| +2450815|3175|3|3| +2450815|3176|3|796| +2450815|3178|3|121| +2450815|3181|3|96| +2450815|3182|3|79| +2450815|3184|3|842| +2450815|3187|3|314| +2450815|3188|3|650| +2450815|3190|3|| +2450815|3193|3|965| +2450815|3194|3|806| +2450815|3196|3|432| +2450815|3199|3|552| +2450815|3200|3|468| +2450815|3202|3|127| +2450815|3205|3|487| +2450815|3206|3|258| +2450815|3208|3|467| +2450815|3211|3|275| +2450815|3212|3|161| +2450815|3214|3|302| +2450815|3217|3|760| +2450815|3218|3|20| +2450815|3220|3|289| +2450815|3223|3|101| +2450815|3224|3|187| +2450815|3226|3|| +2450815|3229|3|230| +2450815|3230|3|671| +2450815|3232|3|937| +2450815|3235|3|167| +2450815|3236|3|953| +2450815|3238|3|627| +2450815|3241|3|570| +2450815|3242|3|116| +2450815|3244|3|472| +2450815|3247|3|743| +2450815|3248|3|540| +2450815|3250|3|260| +2450815|3253|3|88| +2450815|3254|3|5| +2450815|3256|3|45| +2450815|3259|3|963| +2450815|3260|3|| +2450815|3262|3|252| +2450815|3265|3|53| +2450815|3266|3|0| +2450815|3268|3|423| +2450815|3271|3|328| +2450815|3272|3|258| +2450815|3274|3|376| +2450815|3277|3|886| +2450815|3278|3|840| +2450815|3280|3|585| +2450815|3283|3|635| +2450815|3284|3|57| +2450815|3286|3|171| +2450815|3289|3|353| +2450815|3290|3|630| +2450815|3292|3|549| +2450815|3295|3|792| +2450815|3296|3|734| +2450815|3298|3|264| +2450815|3301|3|476| +2450815|3302|3|22| +2450815|3304|3|406| +2450815|3307|3|692| +2450815|3308|3|196| +2450815|3310|3|9| +2450815|3313|3|146| +2450815|3314|3|716| +2450815|3316|3|858| +2450815|3319|3|968| +2450815|3320|3|572| +2450815|3322|3|124| +2450815|3325|3|414| +2450815|3326|3|921| +2450815|3328|3|352| +2450815|3331|3|875| +2450815|3332|3|281| +2450815|3334|3|867| +2450815|3337|3|0| +2450815|3338|3|493| +2450815|3340|3|126| +2450815|3343|3|706| +2450815|3344|3|704| +2450815|3346|3|256| +2450815|3349|3|| +2450815|3350|3|611| +2450815|3352|3|244| +2450815|3355|3|287| +2450815|3356|3|300| +2450815|3358|3|574| +2450815|3361|3|4| +2450815|3362|3|176| +2450815|3364|3|696| +2450815|3367|3|692| +2450815|3368|3|332| +2450815|3370|3|| +2450815|3373|3|235| +2450815|3374|3|| +2450815|3376|3|509| +2450815|3379|3|537| +2450815|3380|3|194| +2450815|3382|3|514| +2450815|3385|3|46| +2450815|3386|3|564| +2450815|3388|3|892| +2450815|3391|3|449| +2450815|3392|3|314| +2450815|3394|3|867| +2450815|3397|3|969| +2450815|3398|3|637| +2450815|3400|3|912| +2450815|3403|3|576| +2450815|3404|3|369| +2450815|3406|3|682| +2450815|3409|3|283| +2450815|3410|3|620| +2450815|3412|3|777| +2450815|3415|3|786| +2450815|3416|3|911| +2450815|3418|3|936| +2450815|3421|3|| +2450815|3422|3|318| +2450815|3424|3|271| +2450815|3427|3|653| +2450815|3428|3|525| +2450815|3430|3|149| +2450815|3433|3|911| +2450815|3434|3|237| +2450815|3436|3|612| +2450815|3439|3|735| +2450815|3440|3|736| +2450815|3442|3|776| +2450815|3445|3|454| +2450815|3446|3|368| +2450815|3448|3|638| +2450815|3451|3|762| +2450815|3452|3|286| +2450815|3454|3|758| +2450815|3457|3|370| +2450815|3458|3|164| +2450815|3460|3|998| +2450815|3463|3|949| +2450815|3464|3|867| +2450815|3466|3|587| +2450815|3469|3|537| +2450815|3470|3|622| +2450815|3472|3|341| +2450815|3475|3|443| +2450815|3476|3|839| +2450815|3478|3|468| +2450815|3481|3|54| +2450815|3482|3|833| +2450815|3484|3|544| +2450815|3487|3|889| +2450815|3488|3|150| +2450815|3490|3|735| +2450815|3493|3|416| +2450815|3494|3|55| +2450815|3496|3|115| +2450815|3499|3|468| +2450815|3500|3|529| +2450815|3502|3|152| +2450815|3505|3|984| +2450815|3506|3|773| +2450815|3508|3|605| +2450815|3511|3|0| +2450815|3512|3|167| +2450815|3514|3|| +2450815|3517|3|874| +2450815|3518|3|107| +2450815|3520|3|| +2450815|3523|3|789| +2450815|3524|3|503| +2450815|3526|3|358| +2450815|3529|3|780| +2450815|3530|3|929| +2450815|3532|3|610| +2450815|3535|3|23| +2450815|3536|3|762| +2450815|3538|3|689| +2450815|3541|3|745| +2450815|3542|3|172| +2450815|3544|3|970| +2450815|3547|3|230| +2450815|3548|3|956| +2450815|3550|3|313| +2450815|3553|3|627| +2450815|3554|3|872| +2450815|3556|3|549| +2450815|3559|3|580| +2450815|3560|3|842| +2450815|3562|3|116| +2450815|3565|3|980| +2450815|3566|3|156| +2450815|3568|3|134| +2450815|3571|3|184| +2450815|3572|3|22| +2450815|3574|3|787| +2450815|3577|3|586| +2450815|3578|3|585| +2450815|3580|3|874| +2450815|3583|3|335| +2450815|3584|3|404| +2450815|3586|3|431| +2450815|3589|3|892| +2450815|3590|3|156| +2450815|3592|3|212| +2450815|3595|3|798| +2450815|3596|3|31| +2450815|3598|3|483| +2450815|3601|3|628| +2450815|3602|3|574| +2450815|3604|3|| +2450815|3607|3|173| +2450815|3608|3|656| +2450815|3610|3|992| +2450815|3613|3|985| +2450815|3614|3|877| +2450815|3616|3|410| +2450815|3619|3|207| +2450815|3620|3|537| +2450815|3622|3|262| +2450815|3625|3|638| +2450815|3626|3|565| +2450815|3628|3|246| +2450815|3631|3|561| +2450815|3632|3|260| +2450815|3634|3|330| +2450815|3637|3|843| +2450815|3638|3|118| +2450815|3640|3|| +2450815|3643|3|93| +2450815|3644|3|254| +2450815|3646|3|491| +2450815|3649|3|217| +2450815|3650|3|749| +2450815|3652|3|345| +2450815|3655|3|103| +2450815|3656|3|771| +2450815|3658|3|785| +2450815|3661|3|| +2450815|3662|3|562| +2450815|3664|3|572| +2450815|3667|3|293| +2450815|3668|3|866| +2450815|3670|3|561| +2450815|3673|3|349| +2450815|3674|3|807| +2450815|3676|3|797| +2450815|3679|3|528| +2450815|3680|3|441| +2450815|3682|3|493| +2450815|3685|3|377| +2450815|3686|3|667| +2450815|3688|3|826| +2450815|3691|3|367| +2450815|3692|3|357| +2450815|3694|3|810| +2450815|3697|3|109| +2450815|3698|3|162| +2450815|3700|3|27| +2450815|3703|3|11| +2450815|3704|3|240| +2450815|3706|3|942| +2450815|3709|3|526| +2450815|3710|3|564| +2450815|3712|3|454| +2450815|3715|3|498| +2450815|3716|3|56| +2450815|3718|3|103| +2450815|3721|3|949| +2450815|3722|3|203| +2450815|3724|3|905| +2450815|3727|3|666| +2450815|3728|3|611| +2450815|3730|3|920| +2450815|3733|3|361| +2450815|3734|3|438| +2450815|3736|3|355| +2450815|3739|3|306| +2450815|3740|3|104| +2450815|3742|3|709| +2450815|3745|3|262| +2450815|3746|3|154| +2450815|3748|3|954| +2450815|3751|3|| +2450815|3752|3|427| +2450815|3754|3|622| +2450815|3757|3|797| +2450815|3758|3|164| +2450815|3760|3|702| +2450815|3763|3|195| +2450815|3764|3|774| +2450815|3766|3|107| +2450815|3769|3|819| +2450815|3770|3|138| +2450815|3772|3|254| +2450815|3775|3|749| +2450815|3776|3|890| +2450815|3778|3|139| +2450815|3781|3|390| +2450815|3782|3|647| +2450815|3784|3|624| +2450815|3787|3|65| +2450815|3788|3|468| +2450815|3790|3|15| +2450815|3793|3|286| +2450815|3794|3|877| +2450815|3796|3|150| +2450815|3799|3|782| +2450815|3800|3|963| +2450815|3802|3|733| +2450815|3805|3|784| +2450815|3806|3|213| +2450815|3808|3|497| +2450815|3811|3|345| +2450815|3812|3|356| +2450815|3814|3|701| +2450815|3817|3|84| +2450815|3818|3|841| +2450815|3820|3|882| +2450815|3823|3|700| +2450815|3824|3|371| +2450815|3826|3|5| +2450815|3829|3|729| +2450815|3830|3|827| +2450815|3832|3|436| +2450815|3835|3|395| +2450815|3836|3|811| +2450815|3838|3|920| +2450815|3841|3|728| +2450815|3842|3|812| +2450815|3844|3|960| +2450815|3847|3|| +2450815|3848|3|651| +2450815|3850|3|780| +2450815|3853|3|143| +2450815|3854|3|| +2450815|3856|3|314| +2450815|3859|3|250| +2450815|3860|3|279| +2450815|3862|3|419| +2450815|3865|3|875| +2450815|3866|3|188| +2450815|3868|3|132| +2450815|3871|3|54| +2450815|3872|3|450| +2450815|3874|3|641| +2450815|3877|3|237| +2450815|3878|3|991| +2450815|3880|3|966| +2450815|3883|3|681| +2450815|3884|3|586| +2450815|3886|3|308| +2450815|3889|3|771| +2450815|3890|3|987| +2450815|3892|3|253| +2450815|3895|3|748| +2450815|3896|3|716| +2450815|3898|3|742| +2450815|3901|3|70| +2450815|3902|3|999| +2450815|3904|3|874| +2450815|3907|3|878| +2450815|3908|3|720| +2450815|3910|3|631| +2450815|3913|3|12| +2450815|3914|3|962| +2450815|3916|3|715| +2450815|3919|3|828| +2450815|3920|3|471| +2450815|3922|3|876| +2450815|3925|3|327| +2450815|3926|3|98| +2450815|3928|3|791| +2450815|3931|3|786| +2450815|3932|3|563| +2450815|3934|3|488| +2450815|3937|3|151| +2450815|3938|3|228| +2450815|3940|3|173| +2450815|3943|3|334| +2450815|3944|3|538| +2450815|3946|3|657| +2450815|3949|3|99| +2450815|3950|3|256| +2450815|3952|3|419| +2450815|3955|3|208| +2450815|3956|3|| +2450815|3958|3|287| +2450815|3961|3|880| +2450815|3962|3|455| +2450815|3964|3|| +2450815|3967|3|77| +2450815|3968|3|900| +2450815|3970|3|906| +2450815|3973|3|430| +2450815|3974|3|481| +2450815|3976|3|327| +2450815|3979|3|426| +2450815|3980|3|800| +2450815|3982|3|780| +2450815|3985|3|601| +2450815|3986|3|29| +2450815|3988|3|155| +2450815|3991|3|595| +2450815|3992|3|196| +2450815|3994|3|865| +2450815|3997|3|560| +2450815|3998|3|| +2450815|4000|3|751| +2450815|4003|3|724| +2450815|4004|3|993| +2450815|4006|3|260| +2450815|4009|3|308| +2450815|4010|3|327| +2450815|4012|3|725| +2450815|4015|3|599| +2450815|4016|3|| +2450815|4018|3|971| +2450815|4021|3|| +2450815|4022|3|308| +2450815|4024|3|347| +2450815|4027|3|| +2450815|4028|3|23| +2450815|4030|3|305| +2450815|4033|3|680| +2450815|4034|3|839| +2450815|4036|3|346| +2450815|4039|3|969| +2450815|4040|3|603| +2450815|4042|3|615| +2450815|4045|3|842| +2450815|4046|3|234| +2450815|4048|3|166| +2450815|4051|3|411| +2450815|4052|3|11| +2450815|4054|3|241| +2450815|4057|3|71| +2450815|4058|3|540| +2450815|4060|3|268| +2450815|4063|3|796| +2450815|4064|3|605| +2450815|4066|3|370| +2450815|4069|3|968| +2450815|4070|3|838| +2450815|4072|3|317| +2450815|4075|3|577| +2450815|4076|3|704| +2450815|4078|3|258| +2450815|4081|3|37| +2450815|4082|3|621| +2450815|4084|3|899| +2450815|4087|3|766| +2450815|4088|3|747| +2450815|4090|3|646| +2450815|4093|3|543| +2450815|4094|3|316| +2450815|4096|3|18| +2450815|4099|3|150| +2450815|4100|3|484| +2450815|4102|3|145| +2450815|4105|3|535| +2450815|4106|3|267| +2450815|4108|3|| +2450815|4111|3|795| +2450815|4112|3|327| +2450815|4114|3|581| +2450815|4117|3|907| +2450815|4118|3|903| +2450815|4120|3|486| +2450815|4123|3|872| +2450815|4124|3|386| +2450815|4126|3|486| +2450815|4129|3|149| +2450815|4130|3|391| +2450815|4132|3|179| +2450815|4135|3|270| +2450815|4136|3|162| +2450815|4138|3|467| +2450815|4141|3|270| +2450815|4142|3|807| +2450815|4144|3|64| +2450815|4147|3|695| +2450815|4148|3|809| +2450815|4150|3|315| +2450815|4153|3|153| +2450815|4154|3|354| +2450815|4156|3|158| +2450815|4159|3|230| +2450815|4160|3|244| +2450815|4162|3|566| +2450815|4165|3|795| +2450815|4166|3|| +2450815|4168|3|619| +2450815|4171|3|998| +2450815|4172|3|432| +2450815|4174|3|770| +2450815|4177|3|280| +2450815|4178|3|625| +2450815|4180|3|463| +2450815|4183|3|620| +2450815|4184|3|262| +2450815|4186|3|300| +2450815|4189|3|544| +2450815|4190|3|874| +2450815|4192|3|505| +2450815|4195|3|389| +2450815|4196|3|900| +2450815|4198|3|61| +2450815|4201|3|386| +2450815|4202|3|429| +2450815|4204|3|275| +2450815|4207|3|297| +2450815|4208|3|385| +2450815|4210|3|579| +2450815|4213|3|229| +2450815|4214|3|415| +2450815|4216|3|879| +2450815|4219|3|643| +2450815|4220|3|63| +2450815|4222|3|670| +2450815|4225|3|97| +2450815|4226|3|133| +2450815|4228|3|191| +2450815|4231|3|851| +2450815|4232|3|241| +2450815|4234|3|787| +2450815|4237|3|556| +2450815|4238|3|743| +2450815|4240|3|769| +2450815|4243|3|926| +2450815|4244|3|987| +2450815|4246|3|717| +2450815|4249|3|886| +2450815|4250|3|727| +2450815|4252|3|216| +2450815|4255|3|50| +2450815|4256|3|250| +2450815|4258|3|74| +2450815|4261|3|15| +2450815|4262|3|615| +2450815|4264|3|617| +2450815|4267|3|21| +2450815|4268|3|560| +2450815|4270|3|932| +2450815|4273|3|| +2450815|4274|3|945| +2450815|4276|3|675| +2450815|4279|3|592| +2450815|4280|3|704| +2450815|4282|3|471| +2450815|4285|3|511| +2450815|4286|3|889| +2450815|4288|3|36| +2450815|4291|3|295| +2450815|4292|3|898| +2450815|4294|3|820| +2450815|4297|3|209| +2450815|4298|3|759| +2450815|4300|3|784| +2450815|4303|3|405| +2450815|4304|3|374| +2450815|4306|3|223| +2450815|4309|3|457| +2450815|4310|3|788| +2450815|4312|3|141| +2450815|4315|3|955| +2450815|4316|3|263| +2450815|4318|3|428| +2450815|4321|3|408| +2450815|4322|3|| +2450815|4324|3|367| +2450815|4327|3|365| +2450815|4328|3|434| +2450815|4330|3|919| +2450815|4333|3|940| +2450815|4334|3|43| +2450815|4336|3|931| +2450815|4339|3|613| +2450815|4340|3|967| +2450815|4342|3|715| +2450815|4345|3|656| +2450815|4346|3|142| +2450815|4348|3|833| +2450815|4351|3|167| +2450815|4352|3|315| +2450815|4354|3|685| +2450815|4357|3|476| +2450815|4358|3|820| +2450815|4360|3|722| +2450815|4363|3|602| +2450815|4364|3|| +2450815|4366|3|851| +2450815|4369|3|945| +2450815|4370|3|100| +2450815|4372|3|| +2450815|4375|3|786| +2450815|4376|3|660| +2450815|4378|3|4| +2450815|4381|3|221| +2450815|4382|3|726| +2450815|4384|3|834| +2450815|4387|3|941| +2450815|4388|3|639| +2450815|4390|3|949| +2450815|4393|3|406| +2450815|4394|3|831| +2450815|4396|3|70| +2450815|4399|3|545| +2450815|4400|3|790| +2450815|4402|3|863| +2450815|4405|3|398| +2450815|4406|3|452| +2450815|4408|3|124| +2450815|4411|3|504| +2450815|4412|3|618| +2450815|4414|3|583| +2450815|4417|3|543| +2450815|4418|3|333| +2450815|4420|3|771| +2450815|4423|3|801| +2450815|4424|3|345| +2450815|4426|3|825| +2450815|4429|3|823| +2450815|4430|3|170| +2450815|4432|3|398| +2450815|4435|3|891| +2450815|4436|3|133| +2450815|4438|3|478| +2450815|4441|3|426| +2450815|4442|3|338| +2450815|4444|3|990| +2450815|4447|3|90| +2450815|4448|3|17| +2450815|4450|3|497| +2450815|4453|3|947| +2450815|4454|3|259| +2450815|4456|3|358| +2450815|4459|3|206| +2450815|4460|3|482| +2450815|4462|3|23| +2450815|4465|3|224| +2450815|4466|3|91| +2450815|4468|3|374| +2450815|4471|3|995| +2450815|4472|3|688| +2450815|4474|3|240| +2450815|4477|3|702| +2450815|4478|3|142| +2450815|4480|3|223| +2450815|4483|3|40| +2450815|4484|3|819| +2450815|4486|3|769| +2450815|4489|3|626| +2450815|4490|3|383| +2450815|4492|3|840| +2450815|4495|3|814| +2450815|4496|3|152| +2450815|4498|3|525| +2450815|4501|3|73| +2450815|4502|3|581| +2450815|4504|3|| +2450815|4507|3|476| +2450815|4508|3|203| +2450815|4510|3|564| +2450815|4513|3|355| +2450815|4514|3|384| +2450815|4516|3|702| +2450815|4519|3|791| +2450815|4520|3|680| +2450815|4522|3|122| +2450815|4525|3|452| +2450815|4526|3|720| +2450815|4528|3|30| +2450815|4531|3|833| +2450815|4532|3|17| +2450815|4534|3|635| +2450815|4537|3|257| +2450815|4538|3|883| +2450815|4540|3|474| +2450815|4543|3|349| +2450815|4544|3|917| +2450815|4546|3|529| +2450815|4549|3|58| +2450815|4550|3|165| +2450815|4552|3|655| +2450815|4555|3|83| +2450815|4556|3|92| +2450815|4558|3|968| +2450815|4561|3|152| +2450815|4562|3|964| +2450815|4564|3|42| +2450815|4567|3|262| +2450815|4568|3|686| +2450815|4570|3|812| +2450815|4573|3|289| +2450815|4574|3|939| +2450815|4576|3|311| +2450815|4579|3|479| +2450815|4580|3|475| +2450815|4582|3|420| +2450815|4585|3|204| +2450815|4586|3|606| +2450815|4588|3|631| +2450815|4591|3|658| +2450815|4592|3|259| +2450815|4594|3|938| +2450815|4597|3|234| +2450815|4598|3|862| +2450815|4600|3|782| +2450815|4603|3|472| +2450815|4604|3|39| +2450815|4606|3|567| +2450815|4609|3|43| +2450815|4610|3|465| +2450815|4612|3|584| +2450815|4615|3|| +2450815|4616|3|223| +2450815|4618|3|603| +2450815|4621|3|519| +2450815|4622|3|108| +2450815|4624|3|347| +2450815|4627|3|884| +2450815|4628|3|| +2450815|4630|3|500| +2450815|4633|3|875| +2450815|4634|3|658| +2450815|4636|3|| +2450815|4639|3|903| +2450815|4640|3|471| +2450815|4642|3|13| +2450815|4645|3|598| +2450815|4646|3|382| +2450815|4648|3|221| +2450815|4651|3|115| +2450815|4652|3|851| +2450815|4654|3|132| +2450815|4657|3|725| +2450815|4658|3|5| +2450815|4660|3|287| +2450815|4663|3|531| +2450815|4664|3|872| +2450815|4666|3|873| +2450815|4669|3|999| +2450815|4670|3|577| +2450815|4672|3|457| +2450815|4675|3|92| +2450815|4676|3|854| +2450815|4678|3|523| +2450815|4681|3|582| +2450815|4682|3|899| +2450815|4684|3|457| +2450815|4687|3|50| +2450815|4688|3|80| +2450815|4690|3|160| +2450815|4693|3|83| +2450815|4694|3|720| +2450815|4696|3|721| +2450815|4699|3|281| +2450815|4700|3|945| +2450815|4702|3|420| +2450815|4705|3|22| +2450815|4706|3|218| +2450815|4708|3|836| +2450815|4711|3|908| +2450815|4712|3|984| +2450815|4714|3|807| +2450815|4717|3|867| +2450815|4718|3|27| +2450815|4720|3|359| +2450815|4723|3|147| +2450815|4724|3|685| +2450815|4726|3|9| +2450815|4729|3|976| +2450815|4730|3|95| +2450815|4732|3|398| +2450815|4735|3|823| +2450815|4736|3|169| +2450815|4738|3|139| +2450815|4741|3|729| +2450815|4742|3|778| +2450815|4744|3|539| +2450815|4747|3|247| +2450815|4748|3|568| +2450815|4750|3|661| +2450815|4753|3|366| +2450815|4754|3|| +2450815|4756|3|467| +2450815|4759|3|306| +2450815|4760|3|87| +2450815|4762|3|825| +2450815|4765|3|509| +2450815|4766|3|194| +2450815|4768|3|34| +2450815|4771|3|75| +2450815|4772|3|129| +2450815|4774|3|389| +2450815|4777|3|| +2450815|4778|3|755| +2450815|4780|3|992| +2450815|4783|3|697| +2450815|4784|3|| +2450815|4786|3|660| +2450815|4789|3|575| +2450815|4790|3|447| +2450815|4792|3|310| +2450815|4795|3|468| +2450815|4796|3|365| +2450815|4798|3|964| +2450815|4801|3|228| +2450815|4802|3|166| +2450815|4804|3|77| +2450815|4807|3|835| +2450815|4808|3|130| +2450815|4810|3|23| +2450815|4813|3|539| +2450815|4814|3|133| +2450815|4816|3|607| +2450815|4819|3|418| +2450815|4820|3|192| +2450815|4822|3|731| +2450815|4825|3|79| +2450815|4826|3|491| +2450815|4828|3|672| +2450815|4831|3|475| +2450815|4832|3|838| +2450815|4834|3|718| +2450815|4837|3|186| +2450815|4838|3|769| +2450815|4840|3|284| +2450815|4843|3|580| +2450815|4844|3|14| +2450815|4846|3|259| +2450815|4849|3|226| +2450815|4850|3|347| +2450815|4852|3|337| +2450815|4855|3|402| +2450815|4856|3|468| +2450815|4858|3|202| +2450815|4861|3|213| +2450815|4862|3|534| +2450815|4864|3|958| +2450815|4867|3|977| +2450815|4868|3|871| +2450815|4870|3|886| +2450815|4873|3|293| +2450815|4874|3|480| +2450815|4876|3|303| +2450815|4879|3|462| +2450815|4880|3|837| +2450815|4882|3|488| +2450815|4885|3|485| +2450815|4886|3|253| +2450815|4888|3|124| +2450815|4891|3|974| +2450815|4892|3|381| +2450815|4894|3|524| +2450815|4897|3|59| +2450815|4898|3|307| +2450815|4900|3|308| +2450815|4903|3|60| +2450815|4904|3|613| +2450815|4906|3|513| +2450815|4909|3|352| +2450815|4910|3|17| +2450815|4912|3|38| +2450815|4915|3|779| +2450815|4916|3|| +2450815|4918|3|53| +2450815|4921|3|653| +2450815|4922|3|518| +2450815|4924|3|790| +2450815|4927|3|973| +2450815|4928|3|926| +2450815|4930|3|21| +2450815|4933|3|491| +2450815|4934|3|654| +2450815|4936|3|476| +2450815|4939|3|13| +2450815|4940|3|555| +2450815|4942|3|733| +2450815|4945|3|262| +2450815|4946|3|949| +2450815|4948|3|658| +2450815|4951|3|789| +2450815|4952|3|| +2450815|4954|3|10| +2450815|4957|3|493| +2450815|4958|3|595| +2450815|4960|3|273| +2450815|4963|3|121| +2450815|4964|3|521| +2450815|4966|3|206| +2450815|4969|3|778| +2450815|4970|3|108| +2450815|4972|3|194| +2450815|4975|3|424| +2450815|4976|3|227| +2450815|4978|3|444| +2450815|4981|3|764| +2450815|4982|3|478| +2450815|4984|3|505| +2450815|4987|3|853| +2450815|4988|3|961| +2450815|4990|3|534| +2450815|4993|3|299| +2450815|4994|3|| +2450815|4996|3|76| +2450815|4999|3|20| +2450815|5000|3|938| +2450815|5002|3|70| +2450815|5005|3|47| +2450815|5006|3|518| +2450815|5008|3|678| +2450815|5011|3|2| +2450815|5012|3|526| +2450815|5014|3|801| +2450815|5017|3|313| +2450815|5018|3|| +2450815|5020|3|125| +2450815|5023|3|846| +2450815|5024|3|599| +2450815|5026|3|216| +2450815|5029|3|487| +2450815|5030|3|244| +2450815|5032|3|964| +2450815|5035|3|| +2450815|5036|3|92| +2450815|5038|3|133| +2450815|5041|3|146| +2450815|5042|3|363| +2450815|5044|3|791| +2450815|5047|3|861| +2450815|5048|3|144| +2450815|5050|3|299| +2450815|5053|3|415| +2450815|5054|3|242| +2450815|5056|3|444| +2450815|5059|3|62| +2450815|5060|3|602| +2450815|5062|3|943| +2450815|5065|3|327| +2450815|5066|3|803| +2450815|5068|3|140| +2450815|5071|3|112| +2450815|5072|3|482| +2450815|5074|3|450| +2450815|5077|3|236| +2450815|5078|3|469| +2450815|5080|3|626| +2450815|5083|3|86| +2450815|5084|3|593| +2450815|5086|3|99| +2450815|5089|3|267| +2450815|5090|3|231| +2450815|5092|3|39| +2450815|5095|3|447| +2450815|5096|3|1| +2450815|5098|3|| +2450815|5101|3|900| +2450815|5102|3|505| +2450815|5104|3|33| +2450815|5107|3|446| +2450815|5108|3|532| +2450815|5110|3|649| +2450815|5113|3|367| +2450815|5114|3|911| +2450815|5116|3|767| +2450815|5119|3|655| +2450815|5120|3|33| +2450815|5122|3|865| +2450815|5125|3|200| +2450815|5126|3|327| +2450815|5128|3|172| +2450815|5131|3|618| +2450815|5132|3|171| +2450815|5134|3|277| +2450815|5137|3|820| +2450815|5138|3|234| +2450815|5140|3|432| +2450815|5143|3|9| +2450815|5144|3|128| +2450815|5146|3|954| +2450815|5149|3|170| +2450815|5150|3|841| +2450815|5152|3|896| +2450815|5155|3|953| +2450815|5156|3|572| +2450815|5158|3|1| +2450815|5161|3|505| +2450815|5162|3|310| +2450815|5164|3|973| +2450815|5167|3|554| +2450815|5168|3|157| +2450815|5170|3|418| +2450815|5173|3|| +2450815|5174|3|| +2450815|5176|3|116| +2450815|5179|3|935| +2450815|5180|3|902| +2450815|5182|3|695| +2450815|5185|3|853| +2450815|5186|3|979| +2450815|5188|3|872| +2450815|5191|3|| +2450815|5192|3|846| +2450815|5194|3|25| +2450815|5197|3|236| +2450815|5198|3|912| +2450815|5200|3|413| +2450815|5203|3|285| +2450815|5204|3|27| +2450815|5206|3|807| +2450815|5209|3|923| +2450815|5210|3|794| +2450815|5212|3|86| +2450815|5215|3|761| +2450815|5216|3|878| +2450815|5218|3|786| +2450815|5221|3|678| +2450815|5222|3|905| +2450815|5224|3|912| +2450815|5227|3|808| +2450815|5228|3|3| +2450815|5230|3|475| +2450815|5233|3|664| +2450815|5234|3|225| +2450815|5236|3|628| +2450815|5239|3|| +2450815|5240|3|39| +2450815|5242|3|781| +2450815|5245|3|589| +2450815|5246|3|295| +2450815|5248|3|926| +2450815|5251|3|875| +2450815|5252|3|367| +2450815|5254|3|499| +2450815|5257|3|782| +2450815|5258|3|| +2450815|5260|3|479| +2450815|5263|3|13| +2450815|5264|3|485| +2450815|5266|3|639| +2450815|5269|3|325| +2450815|5270|3|490| +2450815|5272|3|748| +2450815|5275|3|69| +2450815|5276|3|177| +2450815|5278|3|240| +2450815|5281|3|496| +2450815|5282|3|392| +2450815|5284|3|| +2450815|5287|3|44| +2450815|5288|3|881| +2450815|5290|3|977| +2450815|5293|3|152| +2450815|5294|3|173| +2450815|5296|3|295| +2450815|5299|3|634| +2450815|5300|3|619| +2450815|5302|3|982| +2450815|5305|3|269| +2450815|5306|3|514| +2450815|5308|3|29| +2450815|5311|3|842| +2450815|5312|3|485| +2450815|5314|3|888| +2450815|5317|3|669| +2450815|5318|3|73| +2450815|5320|3|169| +2450815|5323|3|974| +2450815|5324|3|858| +2450815|5326|3|535| +2450815|5329|3|673| +2450815|5330|3|656| +2450815|5332|3|170| +2450815|5335|3|638| +2450815|5336|3|833| +2450815|5338|3|| +2450815|5341|3|368| +2450815|5342|3|96| +2450815|5344|3|324| +2450815|5347|3|861| +2450815|5348|3|236| +2450815|5350|3|26| +2450815|5353|3|870| +2450815|5354|3|818| +2450815|5356|3|370| +2450815|5359|3|284| +2450815|5360|3|786| +2450815|5362|3|249| +2450815|5365|3|467| +2450815|5366|3|394| +2450815|5368|3|351| +2450815|5371|3|948| +2450815|5372|3|709| +2450815|5374|3|406| +2450815|5377|3|30| +2450815|5378|3|590| +2450815|5380|3|801| +2450815|5383|3|752| +2450815|5384|3|793| +2450815|5386|3|613| +2450815|5389|3|367| +2450815|5390|3|650| +2450815|5392|3|745| +2450815|5395|3|481| +2450815|5396|3|309| +2450815|5398|3|154| +2450815|5401|3|34| +2450815|5402|3|295| +2450815|5404|3|863| +2450815|5407|3|716| +2450815|5408|3|316| +2450815|5410|3|610| +2450815|5413|3|130| +2450815|5414|3|444| +2450815|5416|3|739| +2450815|5419|3|833| +2450815|5420|3|541| +2450815|5422|3|629| +2450815|5425|3|107| +2450815|5426|3|177| +2450815|5428|3|225| +2450815|5431|3|647| +2450815|5432|3|608| +2450815|5434|3|942| +2450815|5437|3|717| +2450815|5438|3|105| +2450815|5440|3|62| +2450815|5443|3|465| +2450815|5444|3|497| +2450815|5446|3|564| +2450815|5449|3|592| +2450815|5450|3|12| +2450815|5452|3|267| +2450815|5455|3|194| +2450815|5456|3|821| +2450815|5458|3|511| +2450815|5461|3|974| +2450815|5462|3|108| +2450815|5464|3|419| +2450815|5467|3|459| +2450815|5468|3|245| +2450815|5470|3|422| +2450815|5473|3|567| +2450815|5474|3|657| +2450815|5476|3|612| +2450815|5479|3|289| +2450815|5480|3|| +2450815|5482|3|213| +2450815|5485|3|705| +2450815|5486|3|910| +2450815|5488|3|69| +2450815|5491|3|611| +2450815|5492|3|733| +2450815|5494|3|| +2450815|5497|3|600| +2450815|5498|3|615| +2450815|5500|3|914| +2450815|5503|3|224| +2450815|5504|3|301| +2450815|5506|3|831| +2450815|5509|3|699| +2450815|5510|3|203| +2450815|5512|3|858| +2450815|5515|3|217| +2450815|5516|3|412| +2450815|5518|3|908| +2450815|5521|3|| +2450815|5522|3|644| +2450815|5524|3|953| +2450815|5527|3|891| +2450815|5528|3|| +2450815|5530|3|922| +2450815|5533|3|840| +2450815|5534|3|371| +2450815|5536|3|| +2450815|5539|3|426| +2450815|5540|3|881| +2450815|5542|3|| +2450815|5545|3|214| +2450815|5546|3|195| +2450815|5548|3|226| +2450815|5551|3|5| +2450815|5552|3|93| +2450815|5554|3|756| +2450815|5557|3|| +2450815|5558|3|832| +2450815|5560|3|355| +2450815|5563|3|492| +2450815|5564|3|449| +2450815|5566|3|496| +2450815|5569|3|870| +2450815|5570|3|313| +2450815|5572|3|328| +2450815|5575|3|584| +2450815|5576|3|321| +2450815|5578|3|940| +2450815|5581|3|788| +2450815|5582|3|611| +2450815|5584|3|152| +2450815|5587|3|395| +2450815|5588|3|928| +2450815|5590|3|311| +2450815|5593|3|226| +2450815|5594|3|380| +2450815|5596|3|930| +2450815|5599|3|275| +2450815|5600|3|204| +2450815|5602|3|835| +2450815|5605|3|223| +2450815|5606|3|283| +2450815|5608|3|443| +2450815|5611|3|811| +2450815|5612|3|526| +2450815|5614|3|115| +2450815|5617|3|862| +2450815|5618|3|43| +2450815|5620|3|284| +2450815|5623|3|373| +2450815|5624|3|347| +2450815|5626|3|828| +2450815|5629|3|69| +2450815|5630|3|813| +2450815|5632|3|375| +2450815|5635|3|646| +2450815|5636|3|| +2450815|5638|3|115| +2450815|5641|3|259| +2450815|5642|3|182| +2450815|5644|3|481| +2450815|5647|3|809| +2450815|5648|3|895| +2450815|5650|3|| +2450815|5653|3|| +2450815|5654|3|647| +2450815|5656|3|536| +2450815|5659|3|868| +2450815|5660|3|551| +2450815|5662|3|365| +2450815|5665|3|886| +2450815|5666|3|| +2450815|5668|3|988| +2450815|5671|3|167| +2450815|5672|3|527| +2450815|5674|3|743| +2450815|5677|3|641| +2450815|5678|3|327| +2450815|5680|3|976| +2450815|5683|3|702| +2450815|5684|3|393| +2450815|5686|3|997| +2450815|5689|3|189| +2450815|5690|3|704| +2450815|5692|3|| +2450815|5695|3|355| +2450815|5696|3|77| +2450815|5698|3|152| +2450815|5701|3|280| +2450815|5702|3|66| +2450815|5704|3|7| +2450815|5707|3|282| +2450815|5708|3|225| +2450815|5710|3|158| +2450815|5713|3|387| +2450815|5714|3|64| +2450815|5716|3|537| +2450815|5719|3|317| +2450815|5720|3|682| +2450815|5722|3|980| +2450815|5725|3|850| +2450815|5726|3|862| +2450815|5728|3|944| +2450815|5731|3|561| +2450815|5732|3|882| +2450815|5734|3|630| +2450815|5737|3|346| +2450815|5738|3|544| +2450815|5740|3|91| +2450815|5743|3|280| +2450815|5744|3|684| +2450815|5746|3|161| +2450815|5749|3|145| +2450815|5750|3|791| +2450815|5752|3|175| +2450815|5755|3|453| +2450815|5756|3|886| +2450815|5758|3|525| +2450815|5761|3|353| +2450815|5762|3|278| +2450815|5764|3|818| +2450815|5767|3|267| +2450815|5768|3|970| +2450815|5770|3|352| +2450815|5773|3|160| +2450815|5774|3|477| +2450815|5776|3|568| +2450815|5779|3|813| +2450815|5780|3|939| +2450815|5782|3|424| +2450815|5785|3|662| +2450815|5786|3|855| +2450815|5788|3|592| +2450815|5791|3|622| +2450815|5792|3|589| +2450815|5794|3|896| +2450815|5797|3|196| +2450815|5798|3|374| +2450815|5800|3|178| +2450815|5803|3|802| +2450815|5804|3|554| +2450815|5806|3|217| +2450815|5809|3|321| +2450815|5810|3|419| +2450815|5812|3|830| +2450815|5815|3|102| +2450815|5816|3|850| +2450815|5818|3|795| +2450815|5821|3|911| +2450815|5822|3|410| +2450815|5824|3|422| +2450815|5827|3|| +2450815|5828|3|41| +2450815|5830|3|356| +2450815|5833|3|554| +2450815|5834|3|482| +2450815|5836|3|847| +2450815|5839|3|733| +2450815|5840|3|874| +2450815|5842|3|636| +2450815|5845|3|487| +2450815|5846|3|| +2450815|5848|3|250| +2450815|5851|3|473| +2450815|5852|3|964| +2450815|5854|3|978| +2450815|5857|3|913| +2450815|5858|3|885| +2450815|5860|3|297| +2450815|5863|3|577| +2450815|5864|3|| +2450815|5866|3|| +2450815|5869|3|11| +2450815|5870|3|967| +2450815|5872|3|35| +2450815|5875|3|742| +2450815|5876|3|| +2450815|5878|3|790| +2450815|5881|3|713| +2450815|5882|3|778| +2450815|5884|3|963| +2450815|5887|3|889| +2450815|5888|3|574| +2450815|5890|3|960| +2450815|5893|3|129| +2450815|5894|3|972| +2450815|5896|3|689| +2450815|5899|3|453| +2450815|5900|3|292| +2450815|5902|3|321| +2450815|5905|3|897| +2450815|5906|3|434| +2450815|5908|3|114| +2450815|5911|3|| +2450815|5912|3|846| +2450815|5914|3|954| +2450815|5917|3|346| +2450815|5918|3|176| +2450815|5920|3|202| +2450815|5923|3|635| +2450815|5924|3|268| +2450815|5926|3|891| +2450815|5929|3|426| +2450815|5930|3|857| +2450815|5932|3|914| +2450815|5935|3|860| +2450815|5936|3|57| +2450815|5938|3|798| +2450815|5941|3|581| +2450815|5942|3|110| +2450815|5944|3|879| +2450815|5947|3|842| +2450815|5948|3|284| +2450815|5950|3|557| +2450815|5953|3|937| +2450815|5954|3|402| +2450815|5956|3|| +2450815|5959|3|226| +2450815|5960|3|478| +2450815|5962|3|560| +2450815|5965|3|42| +2450815|5966|3|| +2450815|5968|3|55| +2450815|5971|3|261| +2450815|5972|3|991| +2450815|5974|3|594| +2450815|5977|3|359| +2450815|5978|3|966| +2450815|5980|3|674| +2450815|5983|3|911| +2450815|5984|3|544| +2450815|5986|3|| +2450815|5989|3|861| +2450815|5990|3|801| +2450815|5992|3|274| +2450815|5995|3|968| +2450815|5996|3|988| +2450815|5998|3|550| +2450815|6001|3|601| +2450815|6002|3|1| +2450815|6004|3|240| +2450815|6007|3|587| +2450815|6008|3|781| +2450815|6010|3|336| +2450815|6013|3|381| +2450815|6014|3|713| +2450815|6016|3|990| +2450815|6019|3|108| +2450815|6020|3|343| +2450815|6022|3|453| +2450815|6025|3|586| +2450815|6026|3|439| +2450815|6028|3|132| +2450815|6031|3|531| +2450815|6032|3|857| +2450815|6034|3|197| +2450815|6037|3|533| +2450815|6038|3|839| +2450815|6040|3|955| +2450815|6043|3|135| +2450815|6044|3|871| +2450815|6046|3|674| +2450815|6049|3|110| +2450815|6050|3|870| +2450815|6052|3|712| +2450815|6055|3|260| +2450815|6056|3|614| +2450815|6058|3|218| +2450815|6061|3|99| +2450815|6062|3|253| +2450815|6064|3|261| +2450815|6067|3|563| +2450815|6068|3|938| +2450815|6070|3|714| +2450815|6073|3|875| +2450815|6074|3|270| +2450815|6076|3|334| +2450815|6079|3|597| +2450815|6080|3|34| +2450815|6082|3|69| +2450815|6085|3|117| +2450815|6086|3|964| +2450815|6088|3|733| +2450815|6091|3|183| +2450815|6092|3|24| +2450815|6094|3|984| +2450815|6097|3|224| +2450815|6098|3|260| +2450815|6100|3|349| +2450815|6103|3|| +2450815|6104|3|634| +2450815|6106|3|577| +2450815|6109|3|240| +2450815|6110|3|461| +2450815|6112|3|164| +2450815|6115|3|40| +2450815|6116|3|888| +2450815|6118|3|243| +2450815|6121|3|510| +2450815|6122|3|972| +2450815|6124|3|934| +2450815|6127|3|218| +2450815|6128|3|265| +2450815|6130|3|612| +2450815|6133|3|65| +2450815|6134|3|578| +2450815|6136|3|195| +2450815|6139|3|757| +2450815|6140|3|22| +2450815|6142|3|61| +2450815|6145|3|163| +2450815|6146|3|872| +2450815|6148|3|396| +2450815|6151|3|602| +2450815|6152|3|953| +2450815|6154|3|450| +2450815|6157|3|152| +2450815|6158|3|949| +2450815|6160|3|160| +2450815|6163|3|780| +2450815|6164|3|825| +2450815|6166|3|40| +2450815|6169|3|225| +2450815|6170|3|715| +2450815|6172|3|363| +2450815|6175|3|937| +2450815|6176|3|310| +2450815|6178|3|937| +2450815|6181|3|979| +2450815|6182|3|430| +2450815|6184|3|288| +2450815|6187|3|969| +2450815|6188|3|888| +2450815|6190|3|80| +2450815|6193|3|386| +2450815|6194|3|159| +2450815|6196|3|987| +2450815|6199|3|910| +2450815|6200|3|460| +2450815|6202|3|4| +2450815|6205|3|753| +2450815|6206|3|209| +2450815|6208|3|785| +2450815|6211|3|684| +2450815|6212|3|26| +2450815|6214|3|424| +2450815|6217|3|490| +2450815|6218|3|582| +2450815|6220|3|663| +2450815|6223|3|847| +2450815|6224|3|722| +2450815|6226|3|822| +2450815|6229|3|| +2450815|6230|3|71| +2450815|6232|3|652| +2450815|6235|3|411| +2450815|6236|3|992| +2450815|6238|3|640| +2450815|6241|3|386| +2450815|6242|3|662| +2450815|6244|3|1000| +2450815|6247|3|936| +2450815|6248|3|173| +2450815|6250|3|632| +2450815|6253|3|93| +2450815|6254|3|949| +2450815|6256|3|750| +2450815|6259|3|| +2450815|6260|3|439| +2450815|6262|3|163| +2450815|6265|3|858| +2450815|6266|3|200| +2450815|6268|3|497| +2450815|6271|3|501| +2450815|6272|3|999| +2450815|6274|3|255| +2450815|6277|3|477| +2450815|6278|3|523| +2450815|6280|3|196| +2450815|6283|3|926| +2450815|6284|3|| +2450815|6286|3|458| +2450815|6289|3|520| +2450815|6290|3|601| +2450815|6292|3|698| +2450815|6295|3|117| +2450815|6296|3|386| +2450815|6298|3|423| +2450815|6301|3|557| +2450815|6302|3|156| +2450815|6304|3|896| +2450815|6307|3|586| +2450815|6308|3|31| +2450815|6310|3|1| +2450815|6313|3|| +2450815|6314|3|412| +2450815|6316|3|456| +2450815|6319|3|73| +2450815|6320|3|871| +2450815|6322|3|132| +2450815|6325|3|47| +2450815|6326|3|501| +2450815|6328|3|999| +2450815|6331|3|| +2450815|6332|3|707| +2450815|6334|3|393| +2450815|6337|3|712| +2450815|6338|3|685| +2450815|6340|3|364| +2450815|6343|3|43| +2450815|6344|3|924| +2450815|6346|3|460| +2450815|6349|3|913| +2450815|6350|3|917| +2450815|6352|3|243| +2450815|6355|3|701| +2450815|6356|3|900| +2450815|6358|3|48| +2450815|6361|3|701| +2450815|6362|3|257| +2450815|6364|3|25| +2450815|6367|3|| +2450815|6368|3|776| +2450815|6370|3|338| +2450815|6373|3|98| +2450815|6374|3|548| +2450815|6376|3|590| +2450815|6379|3|902| +2450815|6380|3|852| +2450815|6382|3|| +2450815|6385|3|| +2450815|6386|3|876| +2450815|6388|3|392| +2450815|6391|3|748| +2450815|6392|3|617| +2450815|6394|3|831| +2450815|6397|3|812| +2450815|6398|3|24| +2450815|6400|3|436| +2450815|6403|3|| +2450815|6404|3|284| +2450815|6406|3|619| +2450815|6409|3|629| +2450815|6410|3|188| +2450815|6412|3|279| +2450815|6415|3|92| +2450815|6416|3|| +2450815|6418|3|992| +2450815|6421|3|148| +2450815|6422|3|717| +2450815|6424|3|544| +2450815|6427|3|86| +2450815|6428|3|147| +2450815|6430|3|458| +2450815|6433|3|725| +2450815|6434|3|393| +2450815|6436|3|435| +2450815|6439|3|492| +2450815|6440|3|733| +2450815|6442|3|586| +2450815|6445|3|277| +2450815|6446|3|207| +2450815|6448|3|251| +2450815|6451|3|216| +2450815|6452|3|412| +2450815|6454|3|192| +2450815|6457|3|78| +2450815|6458|3|701| +2450815|6460|3|343| +2450815|6463|3|650| +2450815|6464|3|385| +2450815|6466|3|765| +2450815|6469|3|639| +2450815|6470|3|651| +2450815|6472|3|| +2450815|6475|3|533| +2450815|6476|3|814| +2450815|6478|3|8| +2450815|6481|3|155| +2450815|6482|3|42| +2450815|6484|3|1| +2450815|6487|3|632| +2450815|6488|3|897| +2450815|6490|3|891| +2450815|6493|3|847| +2450815|6494|3|87| +2450815|6496|3|697| +2450815|6499|3|440| +2450815|6500|3|855| +2450815|6502|3|739| +2450815|6505|3|| +2450815|6506|3|157| +2450815|6508|3|151| +2450815|6511|3|632| +2450815|6512|3|823| +2450815|6514|3|538| +2450815|6517|3|954| +2450815|6518|3|880| +2450815|6520|3|971| +2450815|6523|3|162| +2450815|6524|3|793| +2450815|6526|3|548| +2450815|6529|3|| +2450815|6530|3|124| +2450815|6532|3|743| +2450815|6535|3|57| +2450815|6536|3|76| +2450815|6538|3|726| +2450815|6541|3|213| +2450815|6542|3|558| +2450815|6544|3|168| +2450815|6547|3|602| +2450815|6548|3|696| +2450815|6550|3|| +2450815|6553|3|15| +2450815|6554|3|218| +2450815|6556|3|649| +2450815|6559|3|728| +2450815|6560|3|670| +2450815|6562|3|731| +2450815|6565|3|1000| +2450815|6566|3|546| +2450815|6568|3|495| +2450815|6571|3|715| +2450815|6572|3|486| +2450815|6574|3|706| +2450815|6577|3|936| +2450815|6578|3|870| +2450815|6580|3|115| +2450815|6583|3|596| +2450815|6584|3|451| +2450815|6586|3|728| +2450815|6589|3|29| +2450815|6590|3|30| +2450815|6592|3|505| +2450815|6595|3|966| +2450815|6596|3|878| +2450815|6598|3|828| +2450815|6601|3|899| +2450815|6602|3|| +2450815|6604|3|180| +2450815|6607|3|158| +2450815|6608|3|596| +2450815|6610|3|547| +2450815|6613|3|545| +2450815|6614|3|596| +2450815|6616|3|813| +2450815|6619|3|318| +2450815|6620|3|457| +2450815|6622|3|541| +2450815|6625|3|| +2450815|6626|3|| +2450815|6628|3|971| +2450815|6631|3|956| +2450815|6632|3|| +2450815|6634|3|75| +2450815|6637|3|529| +2450815|6638|3|216| +2450815|6640|3|986| +2450815|6643|3|167| +2450815|6644|3|190| +2450815|6646|3|580| +2450815|6649|3|99| +2450815|6650|3|291| +2450815|6652|3|760| +2450815|6655|3|360| +2450815|6656|3|317| +2450815|6658|3|217| +2450815|6661|3|| +2450815|6662|3|52| +2450815|6664|3|993| +2450815|6667|3|752| +2450815|6668|3|9| +2450815|6670|3|24| +2450815|6673|3|833| +2450815|6674|3|138| +2450815|6676|3|337| +2450815|6679|3|934| +2450815|6680|3|980| +2450815|6682|3|352| +2450815|6685|3|626| +2450815|6686|3|599| +2450815|6688|3|323| +2450815|6691|3|925| +2450815|6692|3|685| +2450815|6694|3|8| +2450815|6697|3|| +2450815|6698|3|757| +2450815|6700|3|154| +2450815|6703|3|978| +2450815|6704|3|| +2450815|6706|3|529| +2450815|6709|3|929| +2450815|6710|3|| +2450815|6712|3|365| +2450815|6715|3|958| +2450815|6716|3|868| +2450815|6718|3|236| +2450815|6721|3|400| +2450815|6722|3|164| +2450815|6724|3|| +2450815|6727|3|485| +2450815|6728|3|| +2450815|6730|3|407| +2450815|6733|3|455| +2450815|6734|3|62| +2450815|6736|3|319| +2450815|6739|3|711| +2450815|6740|3|979| +2450815|6742|3|672| +2450815|6745|3|567| +2450815|6746|3|421| +2450815|6748|3|363| +2450815|6751|3|960| +2450815|6752|3|85| +2450815|6754|3|866| +2450815|6757|3|812| +2450815|6758|3|157| +2450815|6760|3|726| +2450815|6763|3|764| +2450815|6764|3|956| +2450815|6766|3|240| +2450815|6769|3|957| +2450815|6770|3|559| +2450815|6772|3|353| +2450815|6775|3|570| +2450815|6776|3|225| +2450815|6778|3|51| +2450815|6781|3|486| +2450815|6782|3|755| +2450815|6784|3|880| +2450815|6787|3|616| +2450815|6788|3|618| +2450815|6790|3|37| +2450815|6793|3|744| +2450815|6794|3|586| +2450815|6796|3|709| +2450815|6799|3|988| +2450815|6800|3|277| +2450815|6802|3|955| +2450815|6805|3|34| +2450815|6806|3|273| +2450815|6808|3|631| +2450815|6811|3|973| +2450815|6812|3|| +2450815|6814|3|875| +2450815|6817|3|569| +2450815|6818|3|744| +2450815|6820|3|22| +2450815|6823|3|134| +2450815|6824|3|42| +2450815|6826|3|528| +2450815|6829|3|4| +2450815|6830|3|650| +2450815|6832|3|795| +2450815|6835|3|84| +2450815|6836|3|996| +2450815|6838|3|782| +2450815|6841|3|| +2450815|6842|3|173| +2450815|6844|3|1| +2450815|6847|3|725| +2450815|6848|3|296| +2450815|6850|3|205| +2450815|6853|3|65| +2450815|6854|3|285| +2450815|6856|3|134| +2450815|6859|3|273| +2450815|6860|3|546| +2450815|6862|3|594| +2450815|6865|3|648| +2450815|6866|3|723| +2450815|6868|3|542| +2450815|6871|3|740| +2450815|6872|3|100| +2450815|6874|3|465| +2450815|6877|3|471| +2450815|6878|3|973| +2450815|6880|3|838| +2450815|6883|3|353| +2450815|6884|3|910| +2450815|6886|3|73| +2450815|6889|3|545| +2450815|6890|3|196| +2450815|6892|3|270| +2450815|6895|3|108| +2450815|6896|3|111| +2450815|6898|3|100| +2450815|6901|3|249| +2450815|6902|3|814| +2450815|6904|3|545| +2450815|6907|3|279| +2450815|6908|3|70| +2450815|6910|3|2| +2450815|6913|3|658| +2450815|6914|3|632| +2450815|6916|3|845| +2450815|6919|3|585| +2450815|6920|3|318| +2450815|6922|3|717| +2450815|6925|3|23| +2450815|6926|3|529| +2450815|6928|3|593| +2450815|6931|3|315| +2450815|6932|3|318| +2450815|6934|3|478| +2450815|6937|3|644| +2450815|6938|3|79| +2450815|6940|3|419| +2450815|6943|3|737| +2450815|6944|3|64| +2450815|6946|3|214| +2450815|6949|3|742| +2450815|6950|3|759| +2450815|6952|3|211| +2450815|6955|3|269| +2450815|6956|3|665| +2450815|6958|3|212| +2450815|6961|3|2| +2450815|6962|3|| +2450815|6964|3|630| +2450815|6967|3|454| +2450815|6968|3|475| +2450815|6970|3|238| +2450815|6973|3|517| +2450815|6974|3|684| +2450815|6976|3|611| +2450815|6979|3|918| +2450815|6980|3|558| +2450815|6982|3|584| +2450815|6985|3|760| +2450815|6986|3|| +2450815|6988|3|770| +2450815|6991|3|629| +2450815|6992|3|142| +2450815|6994|3|690| +2450815|6997|3|933| +2450815|6998|3|965| +2450815|7000|3|493| +2450815|7003|3|| +2450815|7004|3|94| +2450815|7006|3|144| +2450815|7009|3|279| +2450815|7010|3|572| +2450815|7012|3|979| +2450815|7015|3|225| +2450815|7016|3|927| +2450815|7018|3|684| +2450815|7021|3|674| +2450815|7022|3|243| +2450815|7024|3|157| +2450815|7027|3|522| +2450815|7028|3|42| +2450815|7030|3|274| +2450815|7033|3|769| +2450815|7034|3|919| +2450815|7036|3|801| +2450815|7039|3|832| +2450815|7040|3|676| +2450815|7042|3|88| +2450815|7045|3|799| +2450815|7046|3|407| +2450815|7048|3|570| +2450815|7051|3|349| +2450815|7052|3|| +2450815|7054|3|822| +2450815|7057|3|543| +2450815|7058|3|988| +2450815|7060|3|847| +2450815|7063|3|577| +2450815|7064|3|549| +2450815|7066|3|218| +2450815|7069|3|749| +2450815|7070|3|60| +2450815|7072|3|432| +2450815|7075|3|| +2450815|7076|3|| +2450815|7078|3|222| +2450815|7081|3|293| +2450815|7082|3|644| +2450815|7084|3|91| +2450815|7087|3|268| +2450815|7088|3|594| +2450815|7090|3|812| +2450815|7093|3|574| +2450815|7094|3|141| +2450815|7096|3|858| +2450815|7099|3|197| +2450815|7100|3|510| +2450815|7102|3|84| +2450815|7105|3|247| +2450815|7106|3|748| +2450815|7108|3|938| +2450815|7111|3|474| +2450815|7112|3|25| +2450815|7114|3|31| +2450815|7117|3|810| +2450815|7118|3|708| +2450815|7120|3|265| +2450815|7123|3|336| +2450815|7124|3|450| +2450815|7126|3|328| +2450815|7129|3|557| +2450815|7130|3|663| +2450815|7132|3|176| +2450815|7135|3|705| +2450815|7136|3|766| +2450815|7138|3|828| +2450815|7141|3|584| +2450815|7142|3|460| +2450815|7144|3|902| +2450815|7147|3|203| +2450815|7148|3|977| +2450815|7150|3|467| +2450815|7153|3|969| +2450815|7154|3|948| +2450815|7156|3|35| +2450815|7159|3|614| +2450815|7160|3|453| +2450815|7162|3|249| +2450815|7165|3|574| +2450815|7166|3|822| +2450815|7168|3|876| +2450815|7171|3|836| +2450815|7172|3|649| +2450815|7174|3|200| +2450815|7177|3|865| +2450815|7178|3|195| +2450815|7180|3|24| +2450815|7183|3|560| +2450815|7184|3|157| +2450815|7186|3|964| +2450815|7189|3|138| +2450815|7190|3|64| +2450815|7192|3|537| +2450815|7195|3|527| +2450815|7196|3|681| +2450815|7198|3|508| +2450815|7201|3|657| +2450815|7202|3|886| +2450815|7204|3|33| +2450815|7207|3|525| +2450815|7208|3|814| +2450815|7210|3|647| +2450815|7213|3|0| +2450815|7214|3|312| +2450815|7216|3|83| +2450815|7219|3|148| +2450815|7220|3|952| +2450815|7222|3|766| +2450815|7225|3|455| +2450815|7226|3|594| +2450815|7228|3|159| +2450815|7231|3|21| +2450815|7232|3|675| +2450815|7234|3|515| +2450815|7237|3|282| +2450815|7238|3|1| +2450815|7240|3|74| +2450815|7243|3|874| +2450815|7244|3|646| +2450815|7246|3|484| +2450815|7249|3|720| +2450815|7250|3|203| +2450815|7252|3|680| +2450815|7255|3|| +2450815|7256|3|344| +2450815|7258|3|817| +2450815|7261|3|731| +2450815|7262|3|597| +2450815|7264|3|820| +2450815|7267|3|| +2450815|7268|3|837| +2450815|7270|3|| +2450815|7273|3|369| +2450815|7274|3|744| +2450815|7276|3|534| +2450815|7279|3|493| +2450815|7280|3|199| +2450815|7282|3|646| +2450815|7285|3|949| +2450815|7286|3|719| +2450815|7288|3|255| +2450815|7291|3|323| +2450815|7292|3|52| +2450815|7294|3|454| +2450815|7297|3|530| +2450815|7298|3|113| +2450815|7300|3|802| +2450815|7303|3|865| +2450815|7304|3|921| +2450815|7306|3|402| +2450815|7309|3|332| +2450815|7310|3|386| +2450815|7312|3|881| +2450815|7315|3|930| +2450815|7316|3|220| +2450815|7318|3|539| +2450815|7321|3|576| +2450815|7322|3|220| +2450815|7324|3|986| +2450815|7327|3|777| +2450815|7328|3|37| +2450815|7330|3|212| +2450815|7333|3|460| +2450815|7334|3|383| +2450815|7336|3|| +2450815|7339|3|98| +2450815|7340|3|950| +2450815|7342|3|315| +2450815|7345|3|17| +2450815|7346|3|637| +2450815|7348|3|135| +2450815|7351|3|97| +2450815|7352|3|700| +2450815|7354|3|104| +2450815|7357|3|410| +2450815|7358|3|446| +2450815|7360|3|70| +2450815|7363|3|586| +2450815|7364|3|512| +2450815|7366|3|752| +2450815|7369|3|220| +2450815|7370|3|652| +2450815|7372|3|418| +2450815|7375|3|169| +2450815|7376|3|778| +2450815|7378|3|839| +2450815|7381|3|505| +2450815|7382|3|109| +2450815|7384|3|100| +2450815|7387|3|881| +2450815|7388|3|362| +2450815|7390|3|73| +2450815|7393|3|544| +2450815|7394|3|64| +2450815|7396|3|317| +2450815|7399|3|734| +2450815|7400|3|991| +2450815|7402|3|826| +2450815|7405|3|568| +2450815|7406|3|333| +2450815|7408|3|| +2450815|7411|3|902| +2450815|7412|3|674| +2450815|7414|3|794| +2450815|7417|3|449| +2450815|7418|3|607| +2450815|7420|3|862| +2450815|7423|3|113| +2450815|7424|3|955| +2450815|7426|3|477| +2450815|7429|3|189| +2450815|7430|3|383| +2450815|7432|3|178| +2450815|7435|3|282| +2450815|7436|3|692| +2450815|7438|3|454| +2450815|7441|3|591| +2450815|7442|3|137| +2450815|7444|3|67| +2450815|7447|3|389| +2450815|7448|3|31| +2450815|7450|3|352| +2450815|7453|3|564| +2450815|7454|3|855| +2450815|7456|3|513| +2450815|7459|3|981| +2450815|7460|3|542| +2450815|7462|3|80| +2450815|7465|3|187| +2450815|7466|3|687| +2450815|7468|3|743| +2450815|7471|3|493| +2450815|7472|3|802| +2450815|7474|3|521| +2450815|7477|3|406| +2450815|7478|3|188| +2450815|7480|3|422| +2450815|7483|3|865| +2450815|7484|3|997| +2450815|7486|3|344| +2450815|7489|3|734| +2450815|7490|3|21| +2450815|7492|3|231| +2450815|7495|3|| +2450815|7496|3|232| +2450815|7498|3|940| +2450815|7501|3|327| +2450815|7502|3|318| +2450815|7504|3|688| +2450815|7507|3|231| +2450815|7508|3|934| +2450815|7510|3|435| +2450815|7513|3|799| +2450815|7514|3|248| +2450815|7516|3|673| +2450815|7519|3|193| +2450815|7520|3|376| +2450815|7522|3|730| +2450815|7525|3|852| +2450815|7526|3|524| +2450815|7528|3|839| +2450815|7531|3|87| +2450815|7532|3|800| +2450815|7534|3|118| +2450815|7537|3|853| +2450815|7538|3|56| +2450815|7540|3|589| +2450815|7543|3|36| +2450815|7544|3|916| +2450815|7546|3|861| +2450815|7549|3|249| +2450815|7550|3|| +2450815|7552|3|36| +2450815|7555|3|57| +2450815|7556|3|155| +2450815|7558|3|992| +2450815|7561|3|990| +2450815|7562|3|441| +2450815|7564|3|83| +2450815|7567|3|| +2450815|7568|3|657| +2450815|7570|3|231| +2450815|7573|3|498| +2450815|7574|3|102| +2450815|7576|3|176| +2450815|7579|3|826| +2450815|7580|3|262| +2450815|7582|3|381| +2450815|7585|3|552| +2450815|7586|3|816| +2450815|7588|3|994| +2450815|7591|3|512| +2450815|7592|3|672| +2450815|7594|3|101| +2450815|7597|3|844| +2450815|7598|3|808| +2450815|7600|3|108| +2450815|7603|3|243| +2450815|7604|3|995| +2450815|7606|3|270| +2450815|7609|3|168| +2450815|7610|3|501| +2450815|7612|3|749| +2450815|7615|3|181| +2450815|7616|3|417| +2450815|7618|3|702| +2450815|7621|3|43| +2450815|7622|3|183| +2450815|7624|3|758| +2450815|7627|3|889| +2450815|7628|3|568| +2450815|7630|3|383| +2450815|7633|3|960| +2450815|7634|3|915| +2450815|7636|3|20| +2450815|7639|3|176| +2450815|7640|3|764| +2450815|7642|3|815| +2450815|7645|3|301| +2450815|7646|3|512| +2450815|7648|3|713| +2450815|7651|3|591| +2450815|7652|3|185| +2450815|7654|3|292| +2450815|7657|3|246| +2450815|7658|3|848| +2450815|7660|3|531| +2450815|7663|3|| +2450815|7664|3|565| +2450815|7666|3|279| +2450815|7669|3|41| +2450815|7670|3|613| +2450815|7672|3|194| +2450815|7675|3|182| +2450815|7676|3|278| +2450815|7678|3|835| +2450815|7681|3|955| +2450815|7682|3|660| +2450815|7684|3|439| +2450815|7687|3|902| +2450815|7688|3|171| +2450815|7690|3|420| +2450815|7693|3|965| +2450815|7694|3|| +2450815|7696|3|191| +2450815|7699|3|904| +2450815|7700|3|307| +2450815|7702|3|326| +2450815|7705|3|817| +2450815|7706|3|751| +2450815|7708|3|789| +2450815|7711|3|838| +2450815|7712|3|483| +2450815|7714|3|981| +2450815|7717|3|52| +2450815|7718|3|680| +2450815|7720|3|865| +2450815|7723|3|570| +2450815|7724|3|| +2450815|7726|3|985| +2450815|7729|3|355| +2450815|7730|3|441| +2450815|7732|3|139| +2450815|7735|3|462| +2450815|7736|3|987| +2450815|7738|3|317| +2450815|7741|3|988| +2450815|7742|3|946| +2450815|7744|3|569| +2450815|7747|3|14| +2450815|7748|3|763| +2450815|7750|3|790| +2450815|7753|3|545| +2450815|7754|3|671| +2450815|7756|3|853| +2450815|7759|3|375| +2450815|7760|3|122| +2450815|7762|3|112| +2450815|7765|3|605| +2450815|7766|3|715| +2450815|7768|3|869| +2450815|7771|3|909| +2450815|7772|3|| +2450815|7774|3|381| +2450815|7777|3|| +2450815|7778|3|597| +2450815|7780|3|123| +2450815|7783|3|566| +2450815|7784|3|920| +2450815|7786|3|| +2450815|7789|3|300| +2450815|7790|3|846| +2450815|7792|3|| +2450815|7795|3|559| +2450815|7796|3|748| +2450815|7798|3|40| +2450815|7801|3|814| +2450815|7802|3|947| +2450815|7804|3|600| +2450815|7807|3|834| +2450815|7808|3|367| +2450815|7810|3|902| +2450815|7813|3|751| +2450815|7814|3|771| +2450815|7816|3|120| +2450815|7819|3|595| +2450815|7820|3|653| +2450815|7822|3|536| +2450815|7825|3|134| +2450815|7826|3|132| +2450815|7828|3|721| +2450815|7831|3|535| +2450815|7832|3|693| +2450815|7834|3|51| +2450815|7837|3|521| +2450815|7838|3|776| +2450815|7840|3|424| +2450815|7843|3|135| +2450815|7844|3|| +2450815|7846|3|875| +2450815|7849|3|779| +2450815|7850|3|470| +2450815|7852|3|46| +2450815|7855|3|| +2450815|7856|3|191| +2450815|7858|3|782| +2450815|7861|3|287| +2450815|7862|3|259| +2450815|7864|3|656| +2450815|7867|3|697| +2450815|7868|3|79| +2450815|7870|3|805| +2450815|7873|3|445| +2450815|7874|3|736| +2450815|7876|3|400| +2450815|7879|3|875| +2450815|7880|3|| +2450815|7882|3|370| +2450815|7885|3|163| +2450815|7886|3|775| +2450815|7888|3|185| +2450815|7891|3|368| +2450815|7892|3|835| +2450815|7894|3|432| +2450815|7897|3|613| +2450815|7898|3|354| +2450815|7900|3|597| +2450815|7903|3|626| +2450815|7904|3|256| +2450815|7906|3|631| +2450815|7909|3|114| +2450815|7910|3|| +2450815|7912|3|416| +2450815|7915|3|| +2450815|7916|3|822| +2450815|7918|3|619| +2450815|7921|3|669| +2450815|7922|3|137| +2450815|7924|3|672| +2450815|7927|3|819| +2450815|7928|3|240| +2450815|7930|3|975| +2450815|7933|3|617| +2450815|7934|3|588| +2450815|7936|3|| +2450815|7939|3|426| +2450815|7940|3|136| +2450815|7942|3|664| +2450815|7945|3|999| +2450815|7946|3|54| +2450815|7948|3|724| +2450815|7951|3|480| +2450815|7952|3|415| +2450815|7954|3|848| +2450815|7957|3|331| +2450815|7958|3|408| +2450815|7960|3|479| +2450815|7963|3|346| +2450815|7964|3|719| +2450815|7966|3|680| +2450815|7969|3|350| +2450815|7970|3|578| +2450815|7972|3|522| +2450815|7975|3|446| +2450815|7976|3|11| +2450815|7978|3|188| +2450815|7981|3|363| +2450815|7982|3|| +2450815|7984|3|950| +2450815|7987|3|330| +2450815|7988|3|355| +2450815|7990|3|760| +2450815|7993|3|860| +2450815|7994|3|669| +2450815|7996|3|260| +2450815|7999|3|132| +2450815|8000|3|311| +2450815|8002|3|873| +2450815|8005|3|712| +2450815|8006|3|478| +2450815|8008|3|968| +2450815|8011|3|497| +2450815|8012|3|632| +2450815|8014|3|954| +2450815|8017|3|190| +2450815|8018|3|109| +2450815|8020|3|10| +2450815|8023|3|769| +2450815|8024|3|832| +2450815|8026|3|340| +2450815|8029|3|201| +2450815|8030|3|407| +2450815|8032|3|859| +2450815|8035|3|766| +2450815|8036|3|565| +2450815|8038|3|231| +2450815|8041|3|630| +2450815|8042|3|794| +2450815|8044|3|270| +2450815|8047|3|318| +2450815|8048|3|589| +2450815|8050|3|26| +2450815|8053|3|902| +2450815|8054|3|917| +2450815|8056|3|298| +2450815|8059|3|951| +2450815|8060|3|676| +2450815|8062|3|350| +2450815|8065|3|192| +2450815|8066|3|249| +2450815|8068|3|285| +2450815|8071|3|893| +2450815|8072|3|858| +2450815|8074|3|394| +2450815|8077|3|75| +2450815|8078|3|133| +2450815|8080|3|31| +2450815|8083|3|438| +2450815|8084|3|502| +2450815|8086|3|388| +2450815|8089|3|444| +2450815|8090|3|613| +2450815|8092|3|632| +2450815|8095|3|137| +2450815|8096|3|795| +2450815|8098|3|649| +2450815|8101|3|858| +2450815|8102|3|204| +2450815|8104|3|861| +2450815|8107|3|480| +2450815|8108|3|167| +2450815|8110|3|17| +2450815|8113|3|595| +2450815|8114|3|109| +2450815|8116|3|238| +2450815|8119|3|500| +2450815|8120|3|549| +2450815|8122|3|582| +2450815|8125|3|| +2450815|8126|3|145| +2450815|8128|3|692| +2450815|8131|3|519| +2450815|8132|3|71| +2450815|8134|3|550| +2450815|8137|3|736| +2450815|8138|3|826| +2450815|8140|3|312| +2450815|8143|3|400| +2450815|8144|3|162| +2450815|8146|3|62| +2450815|8149|3|| +2450815|8150|3|369| +2450815|8152|3|107| +2450815|8155|3|201| +2450815|8156|3|333| +2450815|8158|3|267| +2450815|8161|3|303| +2450815|8162|3|71| +2450815|8164|3|690| +2450815|8167|3|388| +2450815|8168|3|346| +2450815|8170|3|| +2450815|8173|3|264| +2450815|8174|3|| +2450815|8176|3|492| +2450815|8179|3|710| +2450815|8180|3|| +2450815|8182|3|52| +2450815|8185|3|951| +2450815|8186|3|72| +2450815|8188|3|317| +2450815|8191|3|964| +2450815|8192|3|324| +2450815|8194|3|| +2450815|8197|3|| +2450815|8198|3|| +2450815|8200|3|878| +2450815|8203|3|697| +2450815|8204|3|| +2450815|8206|3|869| +2450815|8209|3|713| +2450815|8210|3|90| +2450815|8212|3|234| +2450815|8215|3|71| +2450815|8216|3|547| +2450815|8218|3|489| +2450815|8221|3|84| +2450815|8222|3|760| +2450815|8224|3|| +2450815|8227|3|397| +2450815|8228|3|148| +2450815|8230|3|320| +2450815|8233|3|962| +2450815|8234|3|58| +2450815|8236|3|617| +2450815|8239|3|926| +2450815|8240|3|130| +2450815|8242|3|6| +2450815|8245|3|423| +2450815|8246|3|989| +2450815|8248|3|988| +2450815|8251|3|439| +2450815|8252|3|765| +2450815|8254|3|557| +2450815|8257|3|243| +2450815|8258|3|686| +2450815|8260|3|208| +2450815|8263|3|90| +2450815|8264|3|997| +2450815|8266|3|64| +2450815|8269|3|315| +2450815|8270|3|79| +2450815|8272|3|531| +2450815|8275|3|155| +2450815|8276|3|954| +2450815|8278|3|| +2450815|8281|3|169| +2450815|8282|3|165| +2450815|8284|3|76| +2450815|8287|3|488| +2450815|8288|3|752| +2450815|8290|3|276| +2450815|8293|3|578| +2450815|8294|3|| +2450815|8296|3|768| +2450815|8299|3|771| +2450815|8300|3|284| +2450815|8302|3|380| +2450815|8305|3|754| +2450815|8306|3|928| +2450815|8308|3|33| +2450815|8311|3|485| +2450815|8312|3|888| +2450815|8314|3|9| +2450815|8317|3|653| +2450815|8318|3|982| +2450815|8320|3|725| +2450815|8323|3|200| +2450815|8324|3|591| +2450815|8326|3|746| +2450815|8329|3|186| +2450815|8330|3|191| +2450815|8332|3|177| +2450815|8335|3|588| +2450815|8336|3|140| +2450815|8338|3|414| +2450815|8341|3|317| +2450815|8342|3|785| +2450815|8344|3|560| +2450815|8347|3|290| +2450815|8348|3|212| +2450815|8350|3|462| +2450815|8353|3|624| +2450815|8354|3|586| +2450815|8356|3|267| +2450815|8359|3|17| +2450815|8360|3|| +2450815|8362|3|394| +2450815|8365|3|361| +2450815|8366|3|9| +2450815|8368|3|321| +2450815|8371|3|619| +2450815|8372|3|562| +2450815|8374|3|190| +2450815|8377|3|| +2450815|8378|3|| +2450815|8380|3|239| +2450815|8383|3|959| +2450815|8384|3|| +2450815|8386|3|825| +2450815|8389|3|368| +2450815|8390|3|706| +2450815|8392|3|918| +2450815|8395|3|794| +2450815|8396|3|902| +2450815|8398|3|687| +2450815|8401|3|495| +2450815|8402|3|981| +2450815|8404|3|522| +2450815|8407|3|698| +2450815|8408|3|480| +2450815|8410|3|719| +2450815|8413|3|703| +2450815|8414|3|214| +2450815|8416|3|346| +2450815|8419|3|191| +2450815|8420|3|429| +2450815|8422|3|947| +2450815|8425|3|461| +2450815|8426|3|348| +2450815|8428|3|906| +2450815|8431|3|928| +2450815|8432|3|545| +2450815|8434|3|42| +2450815|8437|3|663| +2450815|8438|3|203| +2450815|8440|3|107| +2450815|8443|3|769| +2450815|8444|3|762| +2450815|8446|3|719| +2450815|8449|3|261| +2450815|8450|3|684| +2450815|8452|3|| +2450815|8455|3|| +2450815|8456|3|109| +2450815|8458|3|46| +2450815|8461|3|940| +2450815|8462|3|152| +2450815|8464|3|402| +2450815|8467|3|112| +2450815|8468|3|787| +2450815|8470|3|873| +2450815|8473|3|583| +2450815|8474|3|157| +2450815|8476|3|953| +2450815|8479|3|472| +2450815|8480|3|247| +2450815|8482|3|130| +2450815|8485|3|383| +2450815|8486|3|993| +2450815|8488|3|950| +2450815|8491|3|25| +2450815|8492|3|510| +2450815|8494|3|688| +2450815|8497|3|410| +2450815|8498|3|780| +2450815|8500|3|939| +2450815|8503|3|936| +2450815|8504|3|438| +2450815|8506|3|730| +2450815|8509|3|788| +2450815|8510|3|113| +2450815|8512|3|299| +2450815|8515|3|686| +2450815|8516|3|465| +2450815|8518|3|12| +2450815|8521|3|| +2450815|8522|3|941| +2450815|8524|3|445| +2450815|8527|3|426| +2450815|8528|3|942| +2450815|8530|3|98| +2450815|8533|3|905| +2450815|8534|3|214| +2450815|8536|3|312| +2450815|8539|3|104| +2450815|8540|3|252| +2450815|8542|3|422| +2450815|8545|3|662| +2450815|8546|3|504| +2450815|8548|3|987| +2450815|8551|3|747| +2450815|8552|3|518| +2450815|8554|3|247| +2450815|8557|3|57| +2450815|8558|3|238| +2450815|8560|3|| +2450815|8563|3|171| +2450815|8564|3|257| +2450815|8566|3|773| +2450815|8569|3|665| +2450815|8570|3|280| +2450815|8572|3|631| +2450815|8575|3|160| +2450815|8576|3|902| +2450815|8578|3|262| +2450815|8581|3|619| +2450815|8582|3|895| +2450815|8584|3|627| +2450815|8587|3|829| +2450815|8588|3|818| +2450815|8590|3|583| +2450815|8593|3|981| +2450815|8594|3|642| +2450815|8596|3|95| +2450815|8599|3|946| +2450815|8600|3|665| +2450815|8602|3|342| +2450815|8605|3|978| +2450815|8606|3|877| +2450815|8608|3|308| +2450815|8611|3|559| +2450815|8612|3|20| +2450815|8614|3|| +2450815|8617|3|829| +2450815|8618|3|977| +2450815|8620|3|142| +2450815|8623|3|715| +2450815|8624|3|271| +2450815|8626|3|575| +2450815|8629|3|215| +2450815|8630|3|830| +2450815|8632|3|533| +2450815|8635|3|862| +2450815|8636|3|662| +2450815|8638|3|884| +2450815|8641|3|646| +2450815|8642|3|948| +2450815|8644|3|531| +2450815|8647|3|801| +2450815|8648|3|945| +2450815|8650|3|643| +2450815|8653|3|505| +2450815|8654|3|631| +2450815|8656|3|941| +2450815|8659|3|695| +2450815|8660|3|134| +2450815|8662|3|866| +2450815|8665|3|665| +2450815|8666|3|| +2450815|8668|3|377| +2450815|8671|3|797| +2450815|8672|3|381| +2450815|8674|3|595| +2450815|8677|3|588| +2450815|8678|3|391| +2450815|8680|3|603| +2450815|8683|3|976| +2450815|8684|3|329| +2450815|8686|3|685| +2450815|8689|3|| +2450815|8690|3|714| +2450815|8692|3|89| +2450815|8695|3|506| +2450815|8696|3|738| +2450815|8698|3|878| +2450815|8701|3|91| +2450815|8702|3|791| +2450815|8704|3|426| +2450815|8707|3|407| +2450815|8708|3|57| +2450815|8710|3|61| +2450815|8713|3|637| +2450815|8714|3|310| +2450815|8716|3|749| +2450815|8719|3|834| +2450815|8720|3|575| +2450815|8722|3|56| +2450815|8725|3|254| +2450815|8726|3|482| +2450815|8728|3|348| +2450815|8731|3|92| +2450815|8732|3|247| +2450815|8734|3|391| +2450815|8737|3|134| +2450815|8738|3|1000| +2450815|8740|3|767| +2450815|8743|3|| +2450815|8744|3|922| +2450815|8746|3|682| +2450815|8749|3|533| +2450815|8750|3|678| +2450815|8752|3|723| +2450815|8755|3|420| +2450815|8756|3|106| +2450815|8758|3|529| +2450815|8761|3|302| +2450815|8762|3|686| +2450815|8764|3|945| +2450815|8767|3|| +2450815|8768|3|127| +2450815|8770|3|580| +2450815|8773|3|795| +2450815|8774|3|950| +2450815|8776|3|413| +2450815|8779|3|290| +2450815|8780|3|213| +2450815|8782|3|994| +2450815|8785|3|876| +2450815|8786|3|| +2450815|8788|3|195| +2450815|8791|3|905| +2450815|8792|3|34| +2450815|8794|3|961| +2450815|8797|3|530| +2450815|8798|3|2| +2450815|8800|3|212| +2450815|8803|3|489| +2450815|8804|3|557| +2450815|8806|3|878| +2450815|8809|3|870| +2450815|8810|3|417| +2450815|8812|3|364| +2450815|8815|3|260| +2450815|8816|3|| +2450815|8818|3|858| +2450815|8821|3|398| +2450815|8822|3|846| +2450815|8824|3|467| +2450815|8827|3|779| +2450815|8828|3|263| +2450815|8830|3|527| +2450815|8833|3|988| +2450815|8834|3|274| +2450815|8836|3|574| +2450815|8839|3|343| +2450815|8840|3|29| +2450815|8842|3|404| +2450815|8845|3|638| +2450815|8846|3|658| +2450815|8848|3|413| +2450815|8851|3|450| +2450815|8852|3|616| +2450815|8854|3|966| +2450815|8857|3|241| +2450815|8858|3|217| +2450815|8860|3|665| +2450815|8863|3|852| +2450815|8864|3|154| +2450815|8866|3|751| +2450815|8869|3|102| +2450815|8870|3|966| +2450815|8872|3|953| +2450815|8875|3|597| +2450815|8876|3|805| +2450815|8878|3|374| +2450815|8881|3|542| +2450815|8882|3|932| +2450815|8884|3|890| +2450815|8887|3|515| +2450815|8888|3|162| +2450815|8890|3|967| +2450815|8893|3|290| +2450815|8894|3|680| +2450815|8896|3|959| +2450815|8899|3|| +2450815|8900|3|937| +2450815|8902|3|883| +2450815|8905|3|335| +2450815|8906|3|111| +2450815|8908|3|434| +2450815|8911|3|5| +2450815|8912|3|898| +2450815|8914|3|992| +2450815|8917|3|555| +2450815|8918|3|817| +2450815|8920|3|33| +2450815|8923|3|866| +2450815|8924|3|35| +2450815|8926|3|268| +2450815|8929|3|292| +2450815|8930|3|212| +2450815|8932|3|97| +2450815|8935|3|296| +2450815|8936|3|134| +2450815|8938|3|518| +2450815|8941|3|754| +2450815|8942|3|426| +2450815|8944|3|334| +2450815|8947|3|336| +2450815|8948|3|486| +2450815|8950|3|84| +2450815|8953|3|296| +2450815|8954|3|309| +2450815|8956|3|693| +2450815|8959|3|817| +2450815|8960|3|32| +2450815|8962|3|53| +2450815|8965|3|291| +2450815|8966|3|828| +2450815|8968|3|733| +2450815|8971|3|215| +2450815|8972|3|382| +2450815|8974|3|101| +2450815|8977|3|187| +2450815|8978|3|709| +2450815|8980|3|516| +2450815|8983|3|139| +2450815|8984|3|109| +2450815|8986|3|712| +2450815|8989|3|134| +2450815|8990|3|879| +2450815|8992|3|170| +2450815|8995|3|500| +2450815|8996|3|709| +2450815|8998|3|942| +2450815|9001|3|486| +2450815|9002|3|119| +2450815|9004|3|771| +2450815|9007|3|620| +2450815|9008|3|525| +2450815|9010|3|935| +2450815|9013|3|432| +2450815|9014|3|600| +2450815|9016|3|875| +2450815|9019|3|41| +2450815|9020|3|158| +2450815|9022|3|109| +2450815|9025|3|977| +2450815|9026|3|80| +2450815|9028|3|298| +2450815|9031|3|126| +2450815|9032|3|749| +2450815|9034|3|377| +2450815|9037|3|935| +2450815|9038|3|365| +2450815|9040|3|517| +2450815|9043|3|10| +2450815|9044|3|307| +2450815|9046|3|111| +2450815|9049|3|961| +2450815|9050|3|744| +2450815|9052|3|882| +2450815|9055|3|976| +2450815|9056|3|758| +2450815|9058|3|469| +2450815|9061|3|246| +2450815|9062|3|309| +2450815|9064|3|884| +2450815|9067|3|461| +2450815|9068|3|486| +2450815|9070|3|892| +2450815|9073|3|613| +2450815|9074|3|247| +2450815|9076|3|829| +2450815|9079|3|953| +2450815|9080|3|648| +2450815|9082|3|439| +2450815|9085|3|628| +2450815|9086|3|572| +2450815|9088|3|79| +2450815|9091|3|198| +2450815|9092|3|303| +2450815|9094|3|893| +2450815|9097|3|956| +2450815|9098|3|662| +2450815|9100|3|77| +2450815|9103|3|386| +2450815|9104|3|629| +2450815|9106|3|97| +2450815|9109|3|540| +2450815|9110|3|931| +2450815|9112|3|12| +2450815|9115|3|281| +2450815|9116|3|| +2450815|9118|3|| +2450815|9121|3|111| +2450815|9122|3|184| +2450815|9124|3|314| +2450815|9127|3|989| +2450815|9128|3|206| +2450815|9130|3|441| +2450815|9133|3|518| +2450815|9134|3|284| +2450815|9136|3|681| +2450815|9139|3|351| +2450815|9140|3|509| +2450815|9142|3|664| +2450815|9145|3|353| +2450815|9146|3|500| +2450815|9148|3|931| +2450815|9151|3|243| +2450815|9152|3|931| +2450815|9154|3|472| +2450815|9157|3|212| +2450815|9158|3|458| +2450815|9160|3|194| +2450815|9163|3|295| +2450815|9164|3|589| +2450815|9166|3|249| +2450815|9169|3|612| +2450815|9170|3|893| +2450815|9172|3|| +2450815|9175|3|723| +2450815|9176|3|2| +2450815|9178|3|957| +2450815|9181|3|576| +2450815|9182|3|223| +2450815|9184|3|141| +2450815|9187|3|377| +2450815|9188|3|127| +2450815|9190|3|| +2450815|9193|3|169| +2450815|9194|3|19| +2450815|9196|3|856| +2450815|9199|3|67| +2450815|9200|3|233| +2450815|9202|3|295| +2450815|9205|3|420| +2450815|9206|3|911| +2450815|9208|3|580| +2450815|9211|3|579| +2450815|9212|3|67| +2450815|9214|3|431| +2450815|9217|3|321| +2450815|9218|3|998| +2450815|9220|3|313| +2450815|9223|3|55| +2450815|9224|3|587| +2450815|9226|3|167| +2450815|9229|3|695| +2450815|9230|3|786| +2450815|9232|3|590| +2450815|9235|3|922| +2450815|9236|3|| +2450815|9238|3|939| +2450815|9241|3|114| +2450815|9242|3|589| +2450815|9244|3|594| +2450815|9247|3|491| +2450815|9248|3|43| +2450815|9250|3|407| +2450815|9253|3|837| +2450815|9254|3|513| +2450815|9256|3|557| +2450815|9259|3|654| +2450815|9260|3|| +2450815|9262|3|295| +2450815|9265|3|352| +2450815|9266|3|591| +2450815|9268|3|814| +2450815|9271|3|702| +2450815|9272|3|935| +2450815|9274|3|720| +2450815|9277|3|12| +2450815|9278|3|809| +2450815|9280|3|562| +2450815|9283|3|940| +2450815|9284|3|396| +2450815|9286|3|275| +2450815|9289|3|943| +2450815|9290|3|453| +2450815|9292|3|928| +2450815|9295|3|395| +2450815|9296|3|282| +2450815|9298|3|321| +2450815|9301|3|508| +2450815|9302|3|116| +2450815|9304|3|65| +2450815|9307|3|52| +2450815|9308|3|| +2450815|9310|3|722| +2450815|9313|3|725| +2450815|9314|3|859| +2450815|9316|3|382| +2450815|9319|3|370| +2450815|9320|3|219| +2450815|9322|3|367| +2450815|9325|3|718| +2450815|9326|3|430| +2450815|9328|3|36| +2450815|9331|3|348| +2450815|9332|3|853| +2450815|9334|3|185| +2450815|9337|3|22| +2450815|9338|3|243| +2450815|9340|3|752| +2450815|9343|3|538| +2450815|9344|3|552| +2450815|9346|3|559| +2450815|9349|3|340| +2450815|9350|3|716| +2450815|9352|3|640| +2450815|9355|3|425| +2450815|9356|3|437| +2450815|9358|3|198| +2450815|9361|3|711| +2450815|9362|3|563| +2450815|9364|3|156| +2450815|9367|3|507| +2450815|9368|3|300| +2450815|9370|3|717| +2450815|9373|3|347| +2450815|9374|3|559| +2450815|9376|3|362| +2450815|9379|3|241| +2450815|9380|3|711| +2450815|9382|3|416| +2450815|9385|3|569| +2450815|9386|3|228| +2450815|9388|3|910| +2450815|9391|3|3| +2450815|9392|3|534| +2450815|9394|3|49| +2450815|9397|3|359| +2450815|9398|3|496| +2450815|9400|3|129| +2450815|9403|3|| +2450815|9404|3|286| +2450815|9406|3|| +2450815|9409|3|85| +2450815|9410|3|336| +2450815|9412|3|330| +2450815|9415|3|82| +2450815|9416|3|479| +2450815|9418|3|| +2450815|9421|3|698| +2450815|9422|3|679| +2450815|9424|3|731| +2450815|9427|3|965| +2450815|9428|3|908| +2450815|9430|3|90| +2450815|9433|3|27| +2450815|9434|3|838| +2450815|9436|3|49| +2450815|9439|3|336| +2450815|9440|3|365| +2450815|9442|3|11| +2450815|9445|3|398| +2450815|9446|3|272| +2450815|9448|3|448| +2450815|9451|3|406| +2450815|9452|3|223| +2450815|9454|3|903| +2450815|9457|3|427| +2450815|9458|3|506| +2450815|9460|3|244| +2450815|9463|3|803| +2450815|9464|3|783| +2450815|9466|3|36| +2450815|9469|3|680| +2450815|9470|3|542| +2450815|9472|3|640| +2450815|9475|3|768| +2450815|9476|3|852| +2450815|9478|3|881| +2450815|9481|3|477| +2450815|9482|3|371| +2450815|9484|3|552| +2450815|9487|3|847| +2450815|9488|3|72| +2450815|9490|3|726| +2450815|9493|3|485| +2450815|9494|3|677| +2450815|9496|3|697| +2450815|9499|3|371| +2450815|9500|3|941| +2450815|9502|3|141| +2450815|9505|3|393| +2450815|9506|3|720| +2450815|9508|3|741| +2450815|9511|3|461| +2450815|9512|3|17| +2450815|9514|3|703| +2450815|9517|3|| +2450815|9518|3|459| +2450815|9520|3|316| +2450815|9523|3|976| +2450815|9524|3|244| +2450815|9526|3|391| +2450815|9529|3|756| +2450815|9530|3|658| +2450815|9532|3|116| +2450815|9535|3|222| +2450815|9536|3|840| +2450815|9538|3|709| +2450815|9541|3|791| +2450815|9542|3|1| +2450815|9544|3|420| +2450815|9547|3|744| +2450815|9548|3|71| +2450815|9550|3|124| +2450815|9553|3|205| +2450815|9554|3|784| +2450815|9556|3|547| +2450815|9559|3|310| +2450815|9560|3|119| +2450815|9562|3|919| +2450815|9565|3|672| +2450815|9566|3|327| +2450815|9568|3|70| +2450815|9571|3|967| +2450815|9572|3|901| +2450815|9574|3|492| +2450815|9577|3|166| +2450815|9578|3|890| +2450815|9580|3|746| +2450815|9583|3|416| +2450815|9584|3|443| +2450815|9586|3|637| +2450815|9589|3|937| +2450815|9590|3|734| +2450815|9592|3|89| +2450815|9595|3|671| +2450815|9596|3|418| +2450815|9598|3|60| +2450815|9601|3|40| +2450815|9602|3|524| +2450815|9604|3|328| +2450815|9607|3|946| +2450815|9608|3|537| +2450815|9610|3|513| +2450815|9613|3|570| +2450815|9614|3|806| +2450815|9616|3|614| +2450815|9619|3|563| +2450815|9620|3|| +2450815|9622|3|517| +2450815|9625|3|825| +2450815|9626|3|281| +2450815|9628|3|400| +2450815|9631|3|695| +2450815|9632|3|175| +2450815|9634|3|167| +2450815|9637|3|448| +2450815|9638|3|915| +2450815|9640|3|862| +2450815|9643|3|751| +2450815|9644|3|619| +2450815|9646|3|215| +2450815|9649|3|818| +2450815|9650|3|346| +2450815|9652|3|274| +2450815|9655|3|899| +2450815|9656|3|912| +2450815|9658|3|133| +2450815|9661|3|933| +2450815|9662|3|728| +2450815|9664|3|955| +2450815|9667|3|252| +2450815|9668|3|83| +2450815|9670|3|951| +2450815|9673|3|400| +2450815|9674|3|645| +2450815|9676|3|69| +2450815|9679|3|262| +2450815|9680|3|433| +2450815|9682|3|838| +2450815|9685|3|| +2450815|9686|3|963| +2450815|9688|3|842| +2450815|9691|3|786| +2450815|9692|3|134| +2450815|9694|3|904| +2450815|9697|3|859| +2450815|9698|3|387| +2450815|9700|3|892| +2450815|9703|3|533| +2450815|9704|3|1| +2450815|9706|3|690| +2450815|9709|3|887| +2450815|9710|3|578| +2450815|9712|3|968| +2450815|9715|3|300| +2450815|9716|3|147| +2450815|9718|3|650| +2450815|9721|3|323| +2450815|9722|3|743| +2450815|9724|3|888| +2450815|9727|3|307| +2450815|9728|3|136| +2450815|9730|3|70| +2450815|9733|3|541| +2450815|9734|3|834| +2450815|9736|3|64| +2450815|9739|3|312| +2450815|9740|3|367| +2450815|9742|3|73| +2450815|9745|3|707| +2450815|9746|3|| +2450815|9748|3|708| +2450815|9751|3|953| +2450815|9752|3|234| +2450815|9754|3|383| +2450815|9757|3|158| +2450815|9758|3|2| +2450815|9760|3|548| +2450815|9763|3|315| +2450815|9764|3|946| +2450815|9766|3|303| +2450815|9769|3|976| +2450815|9770|3|203| +2450815|9772|3|800| +2450815|9775|3|43| +2450815|9776|3|986| +2450815|9778|3|| +2450815|9781|3|391| +2450815|9782|3|583| +2450815|9784|3|841| +2450815|9787|3|213| +2450815|9788|3|165| +2450815|9790|3|812| +2450815|9793|3|965| +2450815|9794|3|| +2450815|9796|3|432| +2450815|9799|3|996| +2450815|9800|3|107| +2450815|9802|3|406| +2450815|9805|3|518| +2450815|9806|3|71| +2450815|9808|3|179| +2450815|9811|3|60| +2450815|9812|3|475| +2450815|9814|3|675| +2450815|9817|3|7| +2450815|9818|3|679| +2450815|9820|3|283| +2450815|9823|3|25| +2450815|9824|3|849| +2450815|9826|3|280| +2450815|9829|3|735| +2450815|9830|3|675| +2450815|9832|3|645| +2450815|9835|3|866| +2450815|9836|3|710| +2450815|9838|3|325| +2450815|9841|3|310| +2450815|9842|3|544| +2450815|9844|3|942| +2450815|9847|3|560| +2450815|9848|3|256| +2450815|9850|3|370| +2450815|9853|3|458| +2450815|9854|3|535| +2450815|9856|3|511| +2450815|9859|3|756| +2450815|9860|3|824| +2450815|9862|3|616| +2450815|9865|3|748| +2450815|9866|3|291| +2450815|9868|3|151| +2450815|9871|3|46| +2450815|9872|3|380| +2450815|9874|3|734| +2450815|9877|3|983| +2450815|9878|3|201| +2450815|9880|3|349| +2450815|9883|3|907| +2450815|9884|3|326| +2450815|9886|3|497| +2450815|9889|3|81| +2450815|9890|3|553| +2450815|9892|3|418| +2450815|9895|3|391| +2450815|9896|3|404| +2450815|9898|3|823| +2450815|9901|3|826| +2450815|9902|3|635| +2450815|9904|3|787| +2450815|9907|3|335| +2450815|9908|3|397| +2450815|9910|3|19| +2450815|9913|3|| +2450815|9914|3|61| +2450815|9916|3|859| +2450815|9919|3|290| +2450815|9920|3|392| +2450815|9922|3|744| +2450815|9925|3|504| +2450815|9926|3|889| +2450815|9928|3|445| +2450815|9931|3|87| +2450815|9932|3|987| +2450815|9934|3|141| +2450815|9937|3|554| +2450815|9938|3|284| +2450815|9940|3|308| +2450815|9943|3|947| +2450815|9944|3|967| +2450815|9946|3|547| +2450815|9949|3|210| +2450815|9950|3|343| +2450815|9952|3|881| +2450815|9955|3|| +2450815|9956|3|891| +2450815|9958|3|302| +2450815|9961|3|966| +2450815|9962|3|145| +2450815|9964|3|57| +2450815|9967|3|517| +2450815|9968|3|408| +2450815|9970|3|434| +2450815|9973|3|320| +2450815|9974|3|651| +2450815|9976|3|818| +2450815|9979|3|31| +2450815|9980|3|242| +2450815|9982|3|449| +2450815|9985|3|344| +2450815|9986|3|833| +2450815|9988|3|316| +2450815|9991|3|445| +2450815|9992|3|222| +2450815|9994|3|491| +2450815|9997|3|137| +2450815|9998|3|42| +2450815|10000|3|195| +2450815|10003|3|753| +2450815|10004|3|| +2450815|10006|3|255| +2450815|10009|3|550| +2450815|10010|3|868| +2450815|10012|3|767| +2450815|10015|3|159| +2450815|10016|3|66| +2450815|10018|3|996| +2450815|10021|3|315| +2450815|10022|3|420| +2450815|10024|3|471| +2450815|10027|3|735| +2450815|10028|3|450| +2450815|10030|3|15| +2450815|10033|3|94| +2450815|10034|3|| +2450815|10036|3|793| +2450815|10039|3|364| +2450815|10040|3|820| +2450815|10042|3|373| +2450815|10045|3|79| +2450815|10046|3|653| +2450815|10048|3|937| +2450815|10051|3|115| +2450815|10052|3|164| +2450815|10054|3|895| +2450815|10057|3|425| +2450815|10058|3|504| +2450815|10060|3|663| +2450815|10063|3|395| +2450815|10064|3|80| +2450815|10066|3|797| +2450815|10069|3|552| +2450815|10070|3|216| +2450815|10072|3|448| +2450815|10075|3|877| +2450815|10076|3|493| +2450815|10078|3|730| +2450815|10081|3|357| +2450815|10082|3|182| +2450815|10084|3|177| +2450815|10087|3|630| +2450815|10088|3|553| +2450815|10090|3|620| +2450815|10093|3|774| +2450815|10094|3|905| +2450815|10096|3|860| +2450815|10099|3|499| +2450815|10100|3|775| +2450815|10102|3|741| +2450815|10105|3|556| +2450815|10106|3|912| +2450815|10108|3|| +2450815|10111|3|106| +2450815|10112|3|471| +2450815|10114|3|96| +2450815|10117|3|705| +2450815|10118|3|145| +2450815|10120|3|| +2450815|10123|3|| +2450815|10124|3|| +2450815|10126|3|50| +2450815|10129|3|83| +2450815|10130|3|73| +2450815|10132|3|865| +2450815|10135|3|4| +2450815|10136|3|570| +2450815|10138|3|645| +2450815|10141|3|95| +2450815|10142|3|619| +2450815|10144|3|122| +2450815|10147|3|49| +2450815|10148|3|925| +2450815|10150|3|530| +2450815|10153|3|83| +2450815|10154|3|632| +2450815|10156|3|813| +2450815|10159|3|351| +2450815|10160|3|885| +2450815|10162|3|762| +2450815|10165|3|868| +2450815|10166|3|559| +2450815|10168|3|707| +2450815|10171|3|689| +2450815|10172|3|344| +2450815|10174|3|102| +2450815|10177|3|199| +2450815|10178|3|521| +2450815|10180|3|304| +2450815|10183|3|453| +2450815|10184|3|| +2450815|10186|3|408| +2450815|10189|3|367| +2450815|10190|3|77| +2450815|10192|3|235| +2450815|10195|3|925| +2450815|10196|3|776| +2450815|10198|3|696| +2450815|10201|3|49| +2450815|10202|3|195| +2450815|10204|3|679| +2450815|10207|3|602| +2450815|10208|3|| +2450815|10210|3|607| +2450815|10213|3|550| +2450815|10214|3|213| +2450815|10216|3|844| +2450815|10219|3|629| +2450815|10220|3|788| +2450815|10222|3|955| +2450815|10225|3|| +2450815|10226|3|56| +2450815|10228|3|912| +2450815|10231|3|423| +2450815|10232|3|503| +2450815|10234|3|155| +2450815|10237|3|216| +2450815|10238|3|742| +2450815|10240|3|171| +2450815|10243|3|491| +2450815|10244|3|515| +2450815|10246|3|434| +2450815|10249|3|562| +2450815|10250|3|304| +2450815|10252|3|582| +2450815|10255|3|708| +2450815|10256|3|| +2450815|10258|3|| +2450815|10261|3|| +2450815|10262|3|182| +2450815|10264|3|276| +2450815|10267|3|367| +2450815|10268|3|211| +2450815|10270|3|614| +2450815|10273|3|900| +2450815|10274|3|83| +2450815|10276|3|967| +2450815|10279|3|506| +2450815|10280|3|104| +2450815|10282|3|244| +2450815|10285|3|464| +2450815|10286|3|95| +2450815|10288|3|| +2450815|10291|3|488| +2450815|10292|3|298| +2450815|10294|3|201| +2450815|10297|3|718| +2450815|10298|3|84| +2450815|10300|3|920| +2450815|10303|3|444| +2450815|10304|3|621| +2450815|10306|3|118| +2450815|10309|3|538| +2450815|10310|3|880| +2450815|10312|3|542| +2450815|10315|3|633| +2450815|10316|3|145| +2450815|10318|3|501| +2450815|10321|3|711| +2450815|10322|3|| +2450815|10324|3|857| +2450815|10327|3|884| +2450815|10328|3|841| +2450815|10330|3|163| +2450815|10333|3|50| +2450815|10334|3|315| +2450815|10336|3|261| +2450815|10339|3|522| +2450815|10340|3|821| +2450815|10342|3|591| +2450815|10345|3|137| +2450815|10346|3|263| +2450815|10348|3|89| +2450815|10351|3|348| +2450815|10352|3|839| +2450815|10354|3|649| +2450815|10357|3|875| +2450815|10358|3|723| +2450815|10360|3|243| +2450815|10363|3|459| +2450815|10364|3|| +2450815|10366|3|275| +2450815|10369|3|391| +2450815|10370|3|424| +2450815|10372|3|565| +2450815|10375|3|34| +2450815|10376|3|220| +2450815|10378|3|808| +2450815|10381|3|| +2450815|10382|3|893| +2450815|10384|3|617| +2450815|10387|3|895| +2450815|10388|3|8| +2450815|10390|3|| +2450815|10393|3|48| +2450815|10394|3|328| +2450815|10396|3|466| +2450815|10399|3|136| +2450815|10400|3|996| +2450815|10402|3|68| +2450815|10405|3|710| +2450815|10406|3|700| +2450815|10408|3|65| +2450815|10411|3|857| +2450815|10412|3|101| +2450815|10414|3|960| +2450815|10417|3|386| +2450815|10418|3|49| +2450815|10420|3|674| +2450815|10423|3|366| +2450815|10424|3|658| +2450815|10426|3|174| +2450815|10429|3|131| +2450815|10430|3|958| +2450815|10432|3|995| +2450815|10435|3|589| +2450815|10436|3|14| +2450815|10438|3|394| +2450815|10441|3|188| +2450815|10442|3|828| +2450815|10444|3|534| +2450815|10447|3|598| +2450815|10448|3|647| +2450815|10450|3|500| +2450815|10453|3|946| +2450815|10454|3|610| +2450815|10456|3|786| +2450815|10459|3|816| +2450815|10460|3|422| +2450815|10462|3|674| +2450815|10465|3|| +2450815|10466|3|915| +2450815|10468|3|| +2450815|10471|3|496| +2450815|10472|3|191| +2450815|10474|3|535| +2450815|10477|3|338| +2450815|10478|3|662| +2450815|10480|3|13| +2450815|10483|3|54| +2450815|10484|3|630| +2450815|10486|3|724| +2450815|10489|3|568| +2450815|10490|3|180| +2450815|10492|3|209| +2450815|10495|3|362| +2450815|10496|3|649| +2450815|10498|3|973| +2450815|10501|3|84| +2450815|10502|3|596| +2450815|10504|3|323| +2450815|10507|3|41| +2450815|10508|3|68| +2450815|10510|3|514| +2450815|10513|3|729| +2450815|10514|3|289| +2450815|10516|3|978| +2450815|10519|3|807| +2450815|10520|3|399| +2450815|10522|3|426| +2450815|10525|3|| +2450815|10526|3|421| +2450815|10528|3|3| +2450815|10531|3|742| +2450815|10532|3|102| +2450815|10534|3|880| +2450815|10537|3|330| +2450815|10538|3|317| +2450815|10540|3|446| +2450815|10543|3|430| +2450815|10544|3|| +2450815|10546|3|156| +2450815|10549|3|339| +2450815|10550|3|423| +2450815|10552|3|896| +2450815|10555|3|34| +2450815|10556|3|208| +2450815|10558|3|106| +2450815|10561|3|869| +2450815|10562|3|736| +2450815|10564|3|379| +2450815|10567|3|27| +2450815|10568|3|| +2450815|10570|3|657| +2450815|10573|3|994| +2450815|10574|3|8| +2450815|10576|3|931| +2450815|10579|3|608| +2450815|10580|3|872| +2450815|10582|3|444| +2450815|10585|3|787| +2450815|10586|3|758| +2450815|10588|3|64| +2450815|10591|3|223| +2450815|10592|3|750| +2450815|10594|3|143| +2450815|10597|3|468| +2450815|10598|3|669| +2450815|10600|3|66| +2450815|10603|3|421| +2450815|10604|3|202| +2450815|10606|3|358| +2450815|10609|3|624| +2450815|10610|3|666| +2450815|10612|3|585| +2450815|10615|3|614| +2450815|10616|3|| +2450815|10618|3|366| +2450815|10621|3|408| +2450815|10622|3|595| +2450815|10624|3|24| +2450815|10627|3|778| +2450815|10628|3|277| +2450815|10630|3|964| +2450815|10633|3|654| +2450815|10634|3|309| +2450815|10636|3|782| +2450815|10639|3|961| +2450815|10640|3|682| +2450815|10642|3|620| +2450815|10645|3|520| +2450815|10646|3|624| +2450815|10648|3|613| +2450815|10651|3|660| +2450815|10652|3|492| +2450815|10654|3|234| +2450815|10657|3|916| +2450815|10658|3|626| +2450815|10660|3|875| +2450815|10663|3|302| +2450815|10664|3|793| +2450815|10666|3|159| +2450815|10669|3|236| +2450815|10670|3|649| +2450815|10672|3|161| +2450815|10675|3|821| +2450815|10676|3|| +2450815|10678|3|323| +2450815|10681|3|808| +2450815|10682|3|380| +2450815|10684|3|841| +2450815|10687|3|468| +2450815|10688|3|763| +2450815|10690|3|778| +2450815|10693|3|620| +2450815|10694|3|106| +2450815|10696|3|50| +2450815|10699|3|178| +2450815|10700|3|766| +2450815|10702|3|656| +2450815|10705|3|421| +2450815|10706|3|326| +2450815|10708|3|881| +2450815|10711|3|587| +2450815|10712|3|337| +2450815|10714|3|| +2450815|10717|3|| +2450815|10718|3|755| +2450815|10720|3|187| +2450815|10723|3|733| +2450815|10724|3|599| +2450815|10726|3|315| +2450815|10729|3|798| +2450815|10730|3|626| +2450815|10732|3|974| +2450815|10735|3|188| +2450815|10736|3|138| +2450815|10738|3|250| +2450815|10741|3|162| +2450815|10742|3|162| +2450815|10744|3|312| +2450815|10747|3|692| +2450815|10748|3|236| +2450815|10750|3|217| +2450815|10753|3|143| +2450815|10754|3|390| +2450815|10756|3|960| +2450815|10759|3|836| +2450815|10760|3|928| +2450815|10762|3|701| +2450815|10765|3|129| +2450815|10766|3|836| +2450815|10768|3|742| +2450815|10771|3|838| +2450815|10772|3|269| +2450815|10774|3|357| +2450815|10777|3|177| +2450815|10778|3|167| +2450815|10780|3|72| +2450815|10783|3|387| +2450815|10784|3|699| +2450815|10786|3|697| +2450815|10789|3|115| +2450815|10790|3|152| +2450815|10792|3|530| +2450815|10795|3|89| +2450815|10796|3|981| +2450815|10798|3|446| +2450815|10801|3|975| +2450815|10802|3|| +2450815|10804|3|657| +2450815|10807|3|760| +2450815|10808|3|622| +2450815|10810|3|147| +2450815|10813|3|897| +2450815|10814|3|371| +2450815|10816|3|979| +2450815|10819|3|45| +2450815|10820|3|22| +2450815|10822|3|171| +2450815|10825|3|629| +2450815|10826|3|900| +2450815|10828|3|1| +2450815|10831|3|453| +2450815|10832|3|105| +2450815|10834|3|501| +2450815|10837|3|111| +2450815|10838|3|242| +2450815|10840|3|742| +2450815|10843|3|418| +2450815|10844|3|241| +2450815|10846|3|| +2450815|10849|3|259| +2450815|10850|3|702| +2450815|10852|3|278| +2450815|10855|3|916| +2450815|10856|3|849| +2450815|10858|3|| +2450815|10861|3|818| +2450815|10862|3|348| +2450815|10864|3|135| +2450815|10867|3|790| +2450815|10868|3|356| +2450815|10870|3|211| +2450815|10873|3|871| +2450815|10874|3|527| +2450815|10876|3|257| +2450815|10879|3|14| +2450815|10880|3|843| +2450815|10882|3|827| +2450815|10885|3|597| +2450815|10886|3|629| +2450815|10888|3|118| +2450815|10891|3|| +2450815|10892|3|824| +2450815|10894|3|348| +2450815|10897|3|656| +2450815|10898|3|390| +2450815|10900|3|920| +2450815|10903|3|607| +2450815|10904|3|362| +2450815|10906|3|110| +2450815|10909|3|96| +2450815|10910|3|364| +2450815|10912|3|438| +2450815|10915|3|119| +2450815|10916|3|37| +2450815|10918|3|192| +2450815|10921|3|902| +2450815|10922|3|585| +2450815|10924|3|608| +2450815|10927|3|| +2450815|10928|3|33| +2450815|10930|3|846| +2450815|10933|3|566| +2450815|10934|3|974| +2450815|10936|3|379| +2450815|10939|3|271| +2450815|10940|3|193| +2450815|10942|3|393| +2450815|10945|3|75| +2450815|10946|3|269| +2450815|10948|3|590| +2450815|10951|3|955| +2450815|10952|3|501| +2450815|10954|3|850| +2450815|10957|3|99| +2450815|10958|3|324| +2450815|10960|3|816| +2450815|10963|3|204| +2450815|10964|3|96| +2450815|10966|3|627| +2450815|10969|3|41| +2450815|10970|3|933| +2450815|10972|3|649| +2450815|10975|3|| +2450815|10976|3|566| +2450815|10978|3|311| +2450815|10981|3|504| +2450815|10982|3|399| +2450815|10984|3|| +2450815|10987|3|| +2450815|10988|3|601| +2450815|10990|3|893| +2450815|10993|3|890| +2450815|10994|3|630| +2450815|10996|3|996| +2450815|10999|3|328| +2450815|11000|3|252| +2450815|11002|3|700| +2450815|11005|3|720| +2450815|11006|3|708| +2450815|11008|3|174| +2450815|11011|3|11| +2450815|11012|3|| +2450815|11014|3|993| +2450815|11017|3|780| +2450815|11018|3|956| +2450815|11020|3|932| +2450815|11023|3|281| +2450815|11024|3|779| +2450815|11026|3|500| +2450815|11029|3|722| +2450815|11030|3|502| +2450815|11032|3|893| +2450815|11035|3|464| +2450815|11036|3|127| +2450815|11038|3|226| +2450815|11041|3|192| +2450815|11042|3|129| +2450815|11044|3|127| +2450815|11047|3|521| +2450815|11048|3|307| +2450815|11050|3|892| +2450815|11053|3|312| +2450815|11054|3|800| +2450815|11056|3|777| +2450815|11059|3|369| +2450815|11060|3|158| +2450815|11062|3|927| +2450815|11065|3|568| +2450815|11066|3|501| +2450815|11068|3|1| +2450815|11071|3|572| +2450815|11072|3|753| +2450815|11074|3|764| +2450815|11077|3|378| +2450815|11078|3|312| +2450815|11080|3|905| +2450815|11083|3|304| +2450815|11084|3|908| +2450815|11086|3|219| +2450815|11089|3|125| +2450815|11090|3|343| +2450815|11092|3|986| +2450815|11095|3|453| +2450815|11096|3|19| +2450815|11098|3|271| +2450815|11101|3|20| +2450815|11102|3|| +2450815|11104|3|248| +2450815|11107|3|817| +2450815|11108|3|810| +2450815|11110|3|406| +2450815|11113|3|821| +2450815|11114|3|134| +2450815|11116|3|646| +2450815|11119|3|701| +2450815|11120|3|58| +2450815|11122|3|76| +2450815|11125|3|437| +2450815|11126|3|186| +2450815|11128|3|61| +2450815|11131|3|70| +2450815|11132|3|| +2450815|11134|3|146| +2450815|11137|3|878| +2450815|11138|3|610| +2450815|11140|3|575| +2450815|11143|3|785| +2450815|11144|3|906| +2450815|11146|3|388| +2450815|11149|3|338| +2450815|11150|3|| +2450815|11152|3|143| +2450815|11155|3|734| +2450815|11156|3|| +2450815|11158|3|218| +2450815|11161|3|39| +2450815|11162|3|689| +2450815|11164|3|1| +2450815|11167|3|842| +2450815|11168|3|242| +2450815|11170|3|714| +2450815|11173|3|400| +2450815|11174|3|568| +2450815|11176|3|638| +2450815|11179|3|25| +2450815|11180|3|655| +2450815|11182|3|232| +2450815|11185|3|114| +2450815|11186|3|962| +2450815|11188|3|613| +2450815|11191|3|768| +2450815|11192|3|109| +2450815|11194|3|882| +2450815|11197|3|637| +2450815|11198|3|551| +2450815|11200|3|234| +2450815|11203|3|320| +2450815|11204|3|671| +2450815|11206|3|| +2450815|11209|3|293| +2450815|11210|3|668| +2450815|11212|3|679| +2450815|11215|3|714| +2450815|11216|3|| +2450815|11218|3|713| +2450815|11221|3|156| +2450815|11222|3|385| +2450815|11224|3|2| +2450815|11227|3|831| +2450815|11228|3|570| +2450815|11230|3|412| +2450815|11233|3|769| +2450815|11234|3|304| +2450815|11236|3|910| +2450815|11239|3|872| +2450815|11240|3|982| +2450815|11242|3|993| +2450815|11245|3|25| +2450815|11246|3|165| +2450815|11248|3|941| +2450815|11251|3|224| +2450815|11252|3|42| +2450815|11254|3|334| +2450815|11257|3|682| +2450815|11258|3|933| +2450815|11260|3|859| +2450815|11263|3|535| +2450815|11264|3|951| +2450815|11266|3|296| +2450815|11269|3|847| +2450815|11270|3|494| +2450815|11272|3|626| +2450815|11275|3|| +2450815|11276|3|669| +2450815|11278|3|636| +2450815|11281|3|648| +2450815|11282|3|337| +2450815|11284|3|440| +2450815|11287|3|346| +2450815|11288|3|680| +2450815|11290|3|921| +2450815|11293|3|53| +2450815|11294|3|| +2450815|11296|3|27| +2450815|11299|3|160| +2450815|11300|3|| +2450815|11302|3|374| +2450815|11305|3|641| +2450815|11306|3|274| +2450815|11308|3|463| +2450815|11311|3|26| +2450815|11312|3|877| +2450815|11314|3|713| +2450815|11317|3|502| +2450815|11318|3|| +2450815|11320|3|36| +2450815|11323|3|781| +2450815|11324|3|945| +2450815|11326|3|947| +2450815|11329|3|13| +2450815|11330|3|965| +2450815|11332|3|740| +2450815|11335|3|360| +2450815|11336|3|28| +2450815|11338|3|603| +2450815|11341|3|318| +2450815|11342|3|210| +2450815|11344|3|367| +2450815|11347|3|396| +2450815|11348|3|231| +2450815|11350|3|320| +2450815|11353|3|996| +2450815|11354|3|825| +2450815|11356|3|89| +2450815|11359|3|350| +2450815|11360|3|277| +2450815|11362|3|256| +2450815|11365|3|167| +2450815|11366|3|329| +2450815|11368|3|241| +2450815|11371|3|499| +2450815|11372|3|744| +2450815|11374|3|255| +2450815|11377|3|120| +2450815|11378|3|438| +2450815|11380|3|58| +2450815|11383|3|651| +2450815|11384|3|678| +2450815|11386|3|| +2450815|11389|3|356| +2450815|11390|3|522| +2450815|11392|3|596| +2450815|11395|3|246| +2450815|11396|3|162| +2450815|11398|3|466| +2450815|11401|3|533| +2450815|11402|3|247| +2450815|11404|3|864| +2450815|11407|3|677| +2450815|11408|3|275| +2450815|11410|3|529| +2450815|11413|3|151| +2450815|11414|3|608| +2450815|11416|3|627| +2450815|11419|3|42| +2450815|11420|3|847| +2450815|11422|3|| +2450815|11425|3|607| +2450815|11426|3|614| +2450815|11428|3|898| +2450815|11431|3|516| +2450815|11432|3|520| +2450815|11434|3|562| +2450815|11437|3|209| +2450815|11438|3|866| +2450815|11440|3|968| +2450815|11443|3|| +2450815|11444|3|684| +2450815|11446|3|221| +2450815|11449|3|927| +2450815|11450|3|105| +2450815|11452|3|577| +2450815|11455|3|211| +2450815|11456|3|204| +2450815|11458|3|204| +2450815|11461|3|413| +2450815|11462|3|431| +2450815|11464|3|766| +2450815|11467|3|884| +2450815|11468|3|766| +2450815|11470|3|| +2450815|11473|3|866| +2450815|11474|3|98| +2450815|11476|3|913| +2450815|11479|3|285| +2450815|11480|3|401| +2450815|11482|3|877| +2450815|11485|3|567| +2450815|11486|3|576| +2450815|11488|3|345| +2450815|11491|3|184| +2450815|11492|3|147| +2450815|11494|3|354| +2450815|11497|3|735| +2450815|11498|3|74| +2450815|11500|3|825| +2450815|11503|3|| +2450815|11504|3|766| +2450815|11506|3|733| +2450815|11509|3|620| +2450815|11510|3|527| +2450815|11512|3|| +2450815|11515|3|955| +2450815|11516|3|917| +2450815|11518|3|411| +2450815|11521|3|490| +2450815|11522|3|591| +2450815|11524|3|557| +2450815|11527|3|975| +2450815|11528|3|785| +2450815|11530|3|955| +2450815|11533|3|782| +2450815|11534|3|741| +2450815|11536|3|143| +2450815|11539|3|737| +2450815|11540|3|79| +2450815|11542|3|759| +2450815|11545|3|57| +2450815|11546|3|997| +2450815|11548|3|| +2450815|11551|3|941| +2450815|11552|3|459| +2450815|11554|3|294| +2450815|11557|3|638| +2450815|11558|3|119| +2450815|11560|3|977| +2450815|11563|3|650| +2450815|11564|3|921| +2450815|11566|3|177| +2450815|11569|3|820| +2450815|11570|3|881| +2450815|11572|3|183| +2450815|11575|3|716| +2450815|11576|3|514| +2450815|11578|3|806| +2450815|11581|3|414| +2450815|11582|3|692| +2450815|11584|3|| +2450815|11587|3|813| +2450815|11588|3|348| +2450815|11590|3|875| +2450815|11593|3|903| +2450815|11594|3|959| +2450815|11596|3|683| +2450815|11599|3|630| +2450815|11600|3|90| +2450815|11602|3|160| +2450815|11605|3|375| +2450815|11606|3|656| +2450815|11608|3|456| +2450815|11611|3|| +2450815|11612|3|560| +2450815|11614|3|165| +2450815|11617|3|911| +2450815|11618|3|160| +2450815|11620|3|829| +2450815|11623|3|868| +2450815|11624|3|484| +2450815|11626|3|510| +2450815|11629|3|| +2450815|11630|3|966| +2450815|11632|3|975| +2450815|11635|3|277| +2450815|11636|3|905| +2450815|11638|3|928| +2450815|11641|3|127| +2450815|11642|3|843| +2450815|11644|3|12| +2450815|11647|3|364| +2450815|11648|3|55| +2450815|11650|3|571| +2450815|11653|3|216| +2450815|11654|3|616| +2450815|11656|3|303| +2450815|11659|3|370| +2450815|11660|3|568| +2450815|11662|3|279| +2450815|11665|3|311| +2450815|11666|3|576| +2450815|11668|3|946| +2450815|11671|3|797| +2450815|11672|3|909| +2450815|11674|3|51| +2450815|11677|3|71| +2450815|11678|3|164| +2450815|11680|3|172| +2450815|11683|3|310| +2450815|11684|3|353| +2450815|11686|3|955| +2450815|11689|3|| +2450815|11690|3|67| +2450815|11692|3|823| +2450815|11695|3|433| +2450815|11696|3|| +2450815|11698|3|959| +2450815|11701|3|519| +2450815|11702|3|194| +2450815|11704|3|958| +2450815|11707|3|921| +2450815|11708|3|471| +2450815|11710|3|71| +2450815|11713|3|451| +2450815|11714|3|918| +2450815|11716|3|246| +2450815|11719|3|10| +2450815|11720|3|350| +2450815|11722|3|847| +2450815|11725|3|595| +2450815|11726|3|435| +2450815|11728|3|973| +2450815|11731|3|884| +2450815|11732|3|29| +2450815|11734|3|109| +2450815|11737|3|759| +2450815|11738|3|276| +2450815|11740|3|230| +2450815|11743|3|376| +2450815|11744|3|234| +2450815|11746|3|852| +2450815|11749|3|456| +2450815|11750|3|206| +2450815|11752|3|504| +2450815|11755|3|481| +2450815|11756|3|| +2450815|11758|3|335| +2450815|11761|3|425| +2450815|11762|3|341| +2450815|11764|3|292| +2450815|11767|3|406| +2450815|11768|3|554| +2450815|11770|3|891| +2450815|11773|3|967| +2450815|11774|3|769| +2450815|11776|3|664| +2450815|11779|3|772| +2450815|11780|3|560| +2450815|11782|3|397| +2450815|11785|3|993| +2450815|11786|3|601| +2450815|11788|3|71| +2450815|11791|3|836| +2450815|11792|3|533| +2450815|11794|3|32| +2450815|11797|3|246| +2450815|11798|3|821| +2450815|11800|3|164| +2450815|11803|3|329| +2450815|11804|3|175| +2450815|11806|3|256| +2450815|11809|3|886| +2450815|11810|3|461| +2450815|11812|3|359| +2450815|11815|3|877| +2450815|11816|3|851| +2450815|11818|3|125| +2450815|11821|3|645| +2450815|11822|3|373| +2450815|11824|3|449| +2450815|11827|3|98| +2450815|11828|3|112| +2450815|11830|3|718| +2450815|11833|3|155| +2450815|11834|3|192| +2450815|11836|3|293| +2450815|11839|3|375| +2450815|11840|3|426| +2450815|11842|3|620| +2450815|11845|3|975| +2450815|11846|3|89| +2450815|11848|3|| +2450815|11851|3|696| +2450815|11852|3|106| +2450815|11854|3|37| +2450815|11857|3|964| +2450815|11858|3|855| +2450815|11860|3|202| +2450815|11863|3|420| +2450815|11864|3|897| +2450815|11866|3|896| +2450815|11869|3|958| +2450815|11870|3|265| +2450815|11872|3|183| +2450815|11875|3|401| +2450815|11876|3|504| +2450815|11878|3|170| +2450815|11881|3|921| +2450815|11882|3|| +2450815|11884|3|284| +2450815|11887|3|779| +2450815|11888|3|848| +2450815|11890|3|288| +2450815|11893|3|506| +2450815|11894|3|305| +2450815|11896|3|| +2450815|11899|3|945| +2450815|11900|3|759| +2450815|11902|3|443| +2450815|11905|3|269| +2450815|11906|3|732| +2450815|11908|3|454| +2450815|11911|3|774| +2450815|11912|3|387| +2450815|11914|3|708| +2450815|11917|3|165| +2450815|11918|3|669| +2450815|11920|3|| +2450815|11923|3|| +2450815|11924|3|488| +2450815|11926|3|350| +2450815|11929|3|840| +2450815|11930|3|657| +2450815|11932|3|258| +2450815|11935|3|306| +2450815|11936|3|778| +2450815|11938|3|306| +2450815|11941|3|358| +2450815|11942|3|996| +2450815|11944|3|53| +2450815|11947|3|156| +2450815|11948|3|405| +2450815|11950|3|137| +2450815|11953|3|377| +2450815|11954|3|873| +2450815|11956|3|588| +2450815|11959|3|860| +2450815|11960|3|140| +2450815|11962|3|16| +2450815|11965|3|376| +2450815|11966|3|62| +2450815|11968|3|856| +2450815|11971|3|691| +2450815|11972|3|932| +2450815|11974|3|| +2450815|11977|3|802| +2450815|11978|3|215| +2450815|11980|3|490| +2450815|11983|3|315| +2450815|11984|3|153| +2450815|11986|3|257| +2450815|11989|3|608| +2450815|11990|3|818| +2450815|11992|3|346| +2450815|11995|3|284| +2450815|11996|3|682| +2450815|11998|3|768| +2450815|12001|3|337| +2450815|12002|3|807| +2450815|12004|3|823| +2450815|12007|3|262| +2450815|12008|3|631| +2450815|12010|3|881| +2450815|12013|3|61| +2450815|12014|3|144| +2450815|12016|3|313| +2450815|12019|3|324| +2450815|12020|3|| +2450815|12022|3|316| +2450815|12025|3|730| +2450815|12026|3|810| +2450815|12028|3|621| +2450815|12031|3|833| +2450815|12032|3|772| +2450815|12034|3|241| +2450815|12037|3|685| +2450815|12038|3|109| +2450815|12040|3|593| +2450815|12043|3|600| +2450815|12044|3|271| +2450815|12046|3|280| +2450815|12049|3|28| +2450815|12050|3|556| +2450815|12052|3|205| +2450815|12055|3|580| +2450815|12056|3|646| +2450815|12058|3|17| +2450815|12061|3|472| +2450815|12062|3|790| +2450815|12064|3|343| +2450815|12067|3|226| +2450815|12068|3|854| +2450815|12070|3|49| +2450815|12073|3|660| +2450815|12074|3|658| +2450815|12076|3|87| +2450815|12079|3|531| +2450815|12080|3|759| +2450815|12082|3|529| +2450815|12085|3|737| +2450815|12086|3|994| +2450815|12088|3|768| +2450815|12091|3|547| +2450815|12092|3|| +2450815|12094|3|874| +2450815|12097|3|575| +2450815|12098|3|728| +2450815|12100|3|994| +2450815|12103|3|132| +2450815|12104|3|407| +2450815|12106|3|576| +2450815|12109|3|355| +2450815|12110|3|461| +2450815|12112|3|786| +2450815|12115|3|823| +2450815|12116|3|586| +2450815|12118|3|| +2450815|12121|3|207| +2450815|12122|3|18| +2450815|12124|3|134| +2450815|12127|3|867| +2450815|12128|3|281| +2450815|12130|3|21| +2450815|12133|3|543| +2450815|12134|3|665| +2450815|12136|3|425| +2450815|12139|3|379| +2450815|12140|3|48| +2450815|12142|3|233| +2450815|12145|3|371| +2450815|12146|3|468| +2450815|12148|3|263| +2450815|12151|3|570| +2450815|12152|3|492| +2450815|12154|3|636| +2450815|12157|3|596| +2450815|12158|3|66| +2450815|12160|3|939| +2450815|12163|3|745| +2450815|12164|3|554| +2450815|12166|3|33| +2450815|12169|3|121| +2450815|12170|3|642| +2450815|12172|3|52| +2450815|12175|3|396| +2450815|12176|3|617| +2450815|12178|3|773| +2450815|12181|3|646| +2450815|12182|3|645| +2450815|12184|3|11| +2450815|12187|3|548| +2450815|12188|3|493| +2450815|12190|3|362| +2450815|12193|3|175| +2450815|12194|3|924| +2450815|12196|3|768| +2450815|12199|3|163| +2450815|12200|3|76| +2450815|12202|3|120| +2450815|12205|3|131| +2450815|12206|3|| +2450815|12208|3|830| +2450815|12211|3|813| +2450815|12212|3|202| +2450815|12214|3|543| +2450815|12217|3|207| +2450815|12218|3|343| +2450815|12220|3|974| +2450815|12223|3|935| +2450815|12224|3|677| +2450815|12226|3|129| +2450815|12229|3|| +2450815|12230|3|723| +2450815|12232|3|21| +2450815|12235|3|812| +2450815|12236|3|563| +2450815|12238|3|952| +2450815|12241|3|675| +2450815|12242|3|47| +2450815|12244|3|575| +2450815|12247|3|278| +2450815|12248|3|245| +2450815|12250|3|873| +2450815|12253|3|539| +2450815|12254|3|612| +2450815|12256|3|752| +2450815|12259|3|889| +2450815|12260|3|353| +2450815|12262|3|826| +2450815|12265|3|36| +2450815|12266|3|851| +2450815|12268|3|555| +2450815|12271|3|581| +2450815|12272|3|374| +2450815|12274|3|783| +2450815|12277|3|616| +2450815|12278|3|910| +2450815|12280|3|901| +2450815|12283|3|350| +2450815|12284|3|180| +2450815|12286|3|39| +2450815|12289|3|837| +2450815|12290|3|791| +2450815|12292|3|55| +2450815|12295|3|129| +2450815|12296|3|977| +2450815|12298|3|43| +2450815|12301|3|691| +2450815|12302|3|357| +2450815|12304|3|224| +2450815|12307|3|871| +2450815|12308|3|382| +2450815|12310|3|948| +2450815|12313|3|781| +2450815|12314|3|846| +2450815|12316|3|380| +2450815|12319|3|408| +2450815|12320|3|240| +2450815|12322|3|79| +2450815|12325|3|643| +2450815|12326|3|841| +2450815|12328|3|575| +2450815|12331|3|820| +2450815|12332|3|319| +2450815|12334|3|829| +2450815|12337|3|657| +2450815|12338|3|835| +2450815|12340|3|850| +2450815|12343|3|17| +2450815|12344|3|| +2450815|12346|3|186| +2450815|12349|3|280| +2450815|12350|3|949| +2450815|12352|3|349| +2450815|12355|3|298| +2450815|12356|3|35| +2450815|12358|3|597| +2450815|12361|3|18| +2450815|12362|3|66| +2450815|12364|3|14| +2450815|12367|3|48| +2450815|12368|3|236| +2450815|12370|3|389| +2450815|12373|3|245| +2450815|12374|3|701| +2450815|12376|3|469| +2450815|12379|3|930| +2450815|12380|3|660| +2450815|12382|3|| +2450815|12385|3|984| +2450815|12386|3|169| +2450815|12388|3|744| +2450815|12391|3|480| +2450815|12392|3|937| +2450815|12394|3|36| +2450815|12397|3|208| +2450815|12398|3|517| +2450815|12400|3|279| +2450815|12403|3|742| +2450815|12404|3|83| +2450815|12406|3|85| +2450815|12409|3|495| +2450815|12410|3|453| +2450815|12412|3|315| +2450815|12415|3|968| +2450815|12416|3|881| +2450815|12418|3|656| +2450815|12421|3|667| +2450815|12422|3|322| +2450815|12424|3|386| +2450815|12427|3|781| +2450815|12428|3|577| +2450815|12430|3|892| +2450815|12433|3|804| +2450815|12434|3|660| +2450815|12436|3|796| +2450815|12439|3|998| +2450815|12440|3|615| +2450815|12442|3|366| +2450815|12445|3|673| +2450815|12446|3|580| +2450815|12448|3|261| +2450815|12451|3|295| +2450815|12452|3|352| +2450815|12454|3|285| +2450815|12457|3|79| +2450815|12458|3|753| +2450815|12460|3|| +2450815|12463|3|385| +2450815|12464|3|24| +2450815|12466|3|668| +2450815|12469|3|936| +2450815|12470|3|29| +2450815|12472|3|14| +2450815|12475|3|647| +2450815|12476|3|311| +2450815|12478|3|366| +2450815|12481|3|96| +2450815|12482|3|| +2450815|12484|3|581| +2450815|12487|3|110| +2450815|12488|3|600| +2450815|12490|3|58| +2450815|12493|3|626| +2450815|12494|3|386| +2450815|12496|3|946| +2450815|12499|3|286| +2450815|12500|3|| +2450815|12502|3|117| +2450815|12505|3|811| +2450815|12506|3|300| +2450815|12508|3|271| +2450815|12511|3|827| +2450815|12512|3|638| +2450815|12514|3|| +2450815|12517|3|468| +2450815|12518|3|196| +2450815|12520|3|110| +2450815|12523|3|580| +2450815|12524|3|762| +2450815|12526|3|| +2450815|12529|3|897| +2450815|12530|3|934| +2450815|12532|3|320| +2450815|12535|3|699| +2450815|12536|3|118| +2450815|12538|3|756| +2450815|12541|3|681| +2450815|12542|3|761| +2450815|12544|3|896| +2450815|12547|3|212| +2450815|12548|3|105| +2450815|12550|3|741| +2450815|12553|3|427| +2450815|12554|3|636| +2450815|12556|3|974| +2450815|12559|3|631| +2450815|12560|3|994| +2450815|12562|3|238| +2450815|12565|3|815| +2450815|12566|3|375| +2450815|12568|3|255| +2450815|12571|3|186| +2450815|12572|3|60| +2450815|12574|3|489| +2450815|12577|3|901| +2450815|12578|3|234| +2450815|12580|3|| +2450815|12583|3|269| +2450815|12584|3|75| +2450815|12586|3|647| +2450815|12589|3|31| +2450815|12590|3|289| +2450815|12592|3|720| +2450815|12595|3|513| +2450815|12596|3|734| +2450815|12598|3|816| +2450815|12601|3|817| +2450815|12602|3|243| +2450815|12604|3|94| +2450815|12607|3|458| +2450815|12608|3|51| +2450815|12610|3|337| +2450815|12613|3|375| +2450815|12614|3|600| +2450815|12616|3|871| +2450815|12619|3|76| +2450815|12620|3|381| +2450815|12622|3|333| +2450815|12625|3|161| +2450815|12626|3|472| +2450815|12628|3|306| +2450815|12631|3|564| +2450815|12632|3|600| +2450815|12634|3|156| +2450815|12637|3|192| +2450815|12638|3|887| +2450815|12640|3|693| +2450815|12643|3|166| +2450815|12644|3|307| +2450815|12646|3|149| +2450815|12649|3|212| +2450815|12650|3|339| +2450815|12652|3|327| +2450815|12655|3|703| +2450815|12656|3|368| +2450815|12658|3|883| +2450815|12661|3|427| +2450815|12662|3|549| +2450815|12664|3|545| +2450815|12667|3|167| +2450815|12668|3|872| +2450815|12670|3|517| +2450815|12673|3|539| +2450815|12674|3|99| +2450815|12676|3|423| +2450815|12679|3|258| +2450815|12680|3|869| +2450815|12682|3|817| +2450815|12685|3|378| +2450815|12686|3|991| +2450815|12688|3|14| +2450815|12691|3|49| +2450815|12692|3|517| +2450815|12694|3|471| +2450815|12697|3|635| +2450815|12698|3|21| +2450815|12700|3|74| +2450815|12703|3|438| +2450815|12704|3|305| +2450815|12706|3|576| +2450815|12709|3|502| +2450815|12710|3|586| +2450815|12712|3|474| +2450815|12715|3|158| +2450815|12716|3|821| +2450815|12718|3|753| +2450815|12721|3|361| +2450815|12722|3|760| +2450815|12724|3|874| +2450815|12727|3|621| +2450815|12728|3|305| +2450815|12730|3|314| +2450815|12733|3|512| +2450815|12734|3|445| +2450815|12736|3|667| +2450815|12739|3|637| +2450815|12740|3|275| +2450815|12742|3|529| +2450815|12745|3|197| +2450815|12746|3|684| +2450815|12748|3|71| +2450815|12751|3|547| +2450815|12752|3|976| +2450815|12754|3|| +2450815|12757|3|480| +2450815|12758|3|777| +2450815|12760|3|816| +2450815|12763|3|622| +2450815|12764|3|742| +2450815|12766|3|1| +2450815|12769|3|278| +2450815|12770|3|490| +2450815|12772|3|473| +2450815|12775|3|12| +2450815|12776|3|705| +2450815|12778|3|530| +2450815|12781|3|673| +2450815|12782|3|77| +2450815|12784|3|279| +2450815|12787|3|195| +2450815|12788|3|12| +2450815|12790|3|975| +2450815|12793|3|208| +2450815|12794|3|240| +2450815|12796|3|232| +2450815|12799|3|252| +2450815|12800|3|233| +2450815|12802|3|110| +2450815|12805|3|322| +2450815|12806|3|117| +2450815|12808|3|663| +2450815|12811|3|401| +2450815|12812|3|821| +2450815|12814|3|576| +2450815|12817|3|725| +2450815|12818|3|796| +2450815|12820|3|558| +2450815|12823|3|631| +2450815|12824|3|333| +2450815|12826|3|71| +2450815|12829|3|647| +2450815|12830|3|626| +2450815|12832|3|759| +2450815|12835|3|155| +2450815|12836|3|678| +2450815|12838|3|964| +2450815|12841|3|657| +2450815|12842|3|479| +2450815|12844|3|670| +2450815|12847|3|819| +2450815|12848|3|319| +2450815|12850|3|305| +2450815|12853|3|912| +2450815|12854|3|950| +2450815|12856|3|386| +2450815|12859|3|604| +2450815|12860|3|427| +2450815|12862|3|323| +2450815|12865|3|262| +2450815|12866|3|868| +2450815|12868|3|737| +2450815|12871|3|456| +2450815|12872|3|850| +2450815|12874|3|312| +2450815|12877|3|881| +2450815|12878|3|953| +2450815|12880|3|200| +2450815|12883|3|296| +2450815|12884|3|849| +2450815|12886|3|869| +2450815|12889|3|967| +2450815|12890|3|166| +2450815|12892|3|935| +2450815|12895|3|204| +2450815|12896|3|12| +2450815|12898|3|967| +2450815|12901|3|602| +2450815|12902|3|692| +2450815|12904|3|| +2450815|12907|3|95| +2450815|12908|3|491| +2450815|12910|3|465| +2450815|12913|3|170| +2450815|12914|3|245| +2450815|12916|3|586| +2450815|12919|3|| +2450815|12920|3|766| +2450815|12922|3|| +2450815|12925|3|739| +2450815|12926|3|445| +2450815|12928|3|| +2450815|12931|3|842| +2450815|12932|3|814| +2450815|12934|3|330| +2450815|12937|3|| +2450815|12938|3|523| +2450815|12940|3|638| +2450815|12943|3|176| +2450815|12944|3|233| +2450815|12946|3|0| +2450815|12949|3|711| +2450815|12950|3|794| +2450815|12952|3|324| +2450815|12955|3|493| +2450815|12956|3|736| +2450815|12958|3|| +2450815|12961|3|600| +2450815|12962|3|72| +2450815|12964|3|591| +2450815|12967|3|942| +2450815|12968|3|| +2450815|12970|3|256| +2450815|12973|3|725| +2450815|12974|3|839| +2450815|12976|3|500| +2450815|12979|3|149| +2450815|12980|3|229| +2450815|12982|3|340| +2450815|12985|3|357| +2450815|12986|3|355| +2450815|12988|3|580| +2450815|12991|3|185| +2450815|12992|3|722| +2450815|12994|3|456| +2450815|12997|3|| +2450815|12998|3|| +2450815|13000|3|466| +2450815|13003|3|663| +2450815|13004|3|394| +2450815|13006|3|304| +2450815|13009|3|697| +2450815|13010|3|830| +2450815|13012|3|177| +2450815|13015|3|133| +2450815|13016|3|477| +2450815|13018|3|533| +2450815|13021|3|849| +2450815|13022|3|323| +2450815|13024|3|899| +2450815|13027|3|579| +2450815|13028|3|| +2450815|13030|3|28| +2450815|13033|3|309| +2450815|13034|3|78| +2450815|13036|3|495| +2450815|13039|3|460| +2450815|13040|3|940| +2450815|13042|3|81| +2450815|13045|3|330| +2450815|13046|3|788| +2450815|13048|3|654| +2450815|13051|3|450| +2450815|13052|3|618| +2450815|13054|3|352| +2450815|13057|3|958| +2450815|13058|3|687| +2450815|13060|3|| +2450815|13063|3|714| +2450815|13064|3|818| +2450815|13066|3|775| +2450815|13069|3|802| +2450815|13070|3|832| +2450815|13072|3|976| +2450815|13075|3|935| +2450815|13076|3|389| +2450815|13078|3|324| +2450815|13081|3|8| +2450815|13082|3|| +2450815|13084|3|303| +2450815|13087|3|355| +2450815|13088|3|832| +2450815|13090|3|662| +2450815|13093|3|709| +2450815|13094|3|671| +2450815|13096|3|89| +2450815|13099|3|771| +2450815|13100|3|785| +2450815|13102|3|176| +2450815|13105|3|290| +2450815|13106|3|937| +2450815|13108|3|91| +2450815|13111|3|265| +2450815|13112|3|724| +2450815|13114|3|794| +2450815|13117|3|990| +2450815|13118|3|735| +2450815|13120|3|| +2450815|13123|3|540| +2450815|13124|3|355| +2450815|13126|3|78| +2450815|13129|3|214| +2450815|13130|3|59| +2450815|13132|3|414| +2450815|13135|3|| +2450815|13136|3|401| +2450815|13138|3|853| +2450815|13141|3|138| +2450815|13142|3|506| +2450815|13144|3|251| +2450815|13147|3|301| +2450815|13148|3|918| +2450815|13150|3|74| +2450815|13153|3|253| +2450815|13154|3|341| +2450815|13156|3|766| +2450815|13159|3|58| +2450815|13160|3|687| +2450815|13162|3|603| +2450815|13165|3|35| +2450815|13166|3|991| +2450815|13168|3|| +2450815|13171|3|54| +2450815|13172|3|11| +2450815|13174|3|198| +2450815|13177|3|90| +2450815|13178|3|923| +2450815|13180|3|460| +2450815|13183|3|597| +2450815|13184|3|418| +2450815|13186|3|119| +2450815|13189|3|798| +2450815|13190|3|918| +2450815|13192|3|521| +2450815|13195|3|581| +2450815|13196|3|234| +2450815|13198|3|364| +2450815|13201|3|469| +2450815|13202|3|323| +2450815|13204|3|713| +2450815|13207|3|221| +2450815|13208|3|38| +2450815|13210|3|466| +2450815|13213|3|14| +2450815|13214|3|| +2450815|13216|3|664| +2450815|13219|3|806| +2450815|13220|3|996| +2450815|13222|3|382| +2450815|13225|3|461| +2450815|13226|3|200| +2450815|13228|3|853| +2450815|13231|3|619| +2450815|13232|3|108| +2450815|13234|3|134| +2450815|13237|3|82| +2450815|13238|3|371| +2450815|13240|3|543| +2450815|13243|3|450| +2450815|13244|3|136| +2450815|13246|3|14| +2450815|13249|3|560| +2450815|13250|3|335| +2450815|13252|3|588| +2450815|13255|3|699| +2450815|13256|3|383| +2450815|13258|3|| +2450815|13261|3|318| +2450815|13262|3|380| +2450815|13264|3|687| +2450815|13267|3|462| +2450815|13268|3|667| +2450815|13270|3|585| +2450815|13273|3|273| +2450815|13274|3|990| +2450815|13276|3|614| +2450815|13279|3|420| +2450815|13280|3|161| +2450815|13282|3|410| +2450815|13285|3|242| +2450815|13286|3|678| +2450815|13288|3|331| +2450815|13291|3|852| +2450815|13292|3|983| +2450815|13294|3|421| +2450815|13297|3|| +2450815|13298|3|640| +2450815|13300|3|696| +2450815|13303|3|174| +2450815|13304|3|286| +2450815|13306|3|459| +2450815|13309|3|362| +2450815|13310|3|215| +2450815|13312|3|564| +2450815|13315|3|848| +2450815|13316|3|669| +2450815|13318|3|| +2450815|13321|3|708| +2450815|13322|3|629| +2450815|13324|3|994| +2450815|13327|3|| +2450815|13328|3|239| +2450815|13330|3|611| +2450815|13333|3|916| +2450815|13334|3|269| +2450815|13336|3|612| +2450815|13339|3|474| +2450815|13340|3|277| +2450815|13342|3|382| +2450815|13345|3|631| +2450815|13346|3|986| +2450815|13348|3|984| +2450815|13351|3|333| +2450815|13352|3|544| +2450815|13354|3|860| +2450815|13357|3|479| +2450815|13358|3|948| +2450815|13360|3|821| +2450815|13363|3|871| +2450815|13364|3|762| +2450815|13366|3|540| +2450815|13369|3|839| +2450815|13370|3|534| +2450815|13372|3|249| +2450815|13375|3|768| +2450815|13376|3|993| +2450815|13378|3|604| +2450815|13381|3|826| +2450815|13382|3|268| +2450815|13384|3|339| +2450815|13387|3|68| +2450815|13388|3|743| +2450815|13390|3|919| +2450815|13393|3|224| +2450815|13394|3|680| +2450815|13396|3|129| +2450815|13399|3|293| +2450815|13400|3|330| +2450815|13402|3|461| +2450815|13405|3|118| +2450815|13406|3|628| +2450815|13408|3|| +2450815|13411|3|147| +2450815|13412|3|155| +2450815|13414|3|| +2450815|13417|3|356| +2450815|13418|3|853| +2450815|13420|3|242| +2450815|13423|3|966| +2450815|13424|3|840| +2450815|13426|3|634| +2450815|13429|3|489| +2450815|13430|3|629| +2450815|13432|3|239| +2450815|13435|3|361| +2450815|13436|3|590| +2450815|13438|3|724| +2450815|13441|3|516| +2450815|13442|3|107| +2450815|13444|3|27| +2450815|13447|3|544| +2450815|13448|3|857| +2450815|13450|3|767| +2450815|13453|3|596| +2450815|13454|3|332| +2450815|13456|3|303| +2450815|13459|3|659| +2450815|13460|3|504| +2450815|13462|3|993| +2450815|13465|3|243| +2450815|13466|3|| +2450815|13468|3|150| +2450815|13471|3|3| +2450815|13472|3|332| +2450815|13474|3|998| +2450815|13477|3|533| +2450815|13478|3|632| +2450815|13480|3|195| +2450815|13483|3|829| +2450815|13484|3|516| +2450815|13486|3|534| +2450815|13489|3|961| +2450815|13490|3|354| +2450815|13492|3|431| +2450815|13495|3|291| +2450815|13496|3|277| +2450815|13498|3|| +2450815|13501|3|456| +2450815|13502|3|192| +2450815|13504|3|873| +2450815|13507|3|992| +2450815|13508|3|679| +2450815|13510|3|156| +2450815|13513|3|237| +2450815|13514|3|564| +2450815|13516|3|416| +2450815|13519|3|650| +2450815|13520|3|823| +2450815|13522|3|696| +2450815|13525|3|463| +2450815|13526|3|620| +2450815|13528|3|266| +2450815|13531|3|832| +2450815|13532|3|723| +2450815|13534|3|604| +2450815|13537|3|401| +2450815|13538|3|961| +2450815|13540|3|294| +2450815|13543|3|329| +2450815|13544|3|991| +2450815|13546|3|734| +2450815|13549|3|173| +2450815|13550|3|720| +2450815|13552|3|249| +2450815|13555|3|188| +2450815|13556|3|908| +2450815|13558|3|428| +2450815|13561|3|992| +2450815|13562|3|130| +2450815|13564|3|| +2450815|13567|3|496| +2450815|13568|3|732| +2450815|13570|3|250| +2450815|13573|3|866| +2450815|13574|3|133| +2450815|13576|3|351| +2450815|13579|3|75| +2450815|13580|3|902| +2450815|13582|3|1000| +2450815|13585|3|551| +2450815|13586|3|275| +2450815|13588|3|245| +2450815|13591|3|758| +2450815|13592|3|824| +2450815|13594|3|635| +2450815|13597|3|28| +2450815|13598|3|571| +2450815|13600|3|395| +2450815|13603|3|795| +2450815|13604|3|84| +2450815|13606|3|701| +2450815|13609|3|71| +2450815|13610|3|71| +2450815|13612|3|237| +2450815|13615|3|520| +2450815|13616|3|863| +2450815|13618|3|993| +2450815|13621|3|364| +2450815|13622|3|58| +2450815|13624|3|403| +2450815|13627|3|| +2450815|13628|3|376| +2450815|13630|3|419| +2450815|13633|3|939| +2450815|13634|3|444| +2450815|13636|3|706| +2450815|13639|3|415| +2450815|13640|3|689| +2450815|13642|3|949| +2450815|13645|3|463| +2450815|13646|3|187| +2450815|13648|3|406| +2450815|13651|3|584| +2450815|13652|3|866| +2450815|13654|3|442| +2450815|13657|3|391| +2450815|13658|3|100| +2450815|13660|3|454| +2450815|13663|3|211| +2450815|13664|3|377| +2450815|13666|3|91| +2450815|13669|3|490| +2450815|13670|3|824| +2450815|13672|3|53| +2450815|13675|3|92| +2450815|13676|3|271| +2450815|13678|3|149| +2450815|13681|3|478| +2450815|13682|3|543| +2450815|13684|3|210| +2450815|13687|3|809| +2450815|13688|3|351| +2450815|13690|3|232| +2450815|13693|3|162| +2450815|13694|3|670| +2450815|13696|3|38| +2450815|13699|3|209| +2450815|13700|3|562| +2450815|13702|3|830| +2450815|13705|3|550| +2450815|13706|3|434| +2450815|13708|3|| +2450815|13711|3|615| +2450815|13712|3|413| +2450815|13714|3|299| +2450815|13717|3|74| +2450815|13718|3|603| +2450815|13720|3|748| +2450815|13723|3|870| +2450815|13724|3|253| +2450815|13726|3|835| +2450815|13729|3|153| +2450815|13730|3|51| +2450815|13732|3|965| +2450815|13735|3|50| +2450815|13736|3|711| +2450815|13738|3|948| +2450815|13741|3|78| +2450815|13742|3|588| +2450815|13744|3|393| +2450815|13747|3|994| +2450815|13748|3|942| +2450815|13750|3|627| +2450815|13753|3|40| +2450815|13754|3|114| +2450815|13756|3|477| +2450815|13759|3|32| +2450815|13760|3|315| +2450815|13762|3|641| +2450815|13765|3|| +2450815|13766|3|592| +2450815|13768|3|124| +2450815|13771|3|871| +2450815|13772|3|841| +2450815|13774|3|455| +2450815|13777|3|45| +2450815|13778|3|488| +2450815|13780|3|211| +2450815|13783|3|89| +2450815|13784|3|321| +2450815|13786|3|232| +2450815|13789|3|926| +2450815|13790|3|739| +2450815|13792|3|459| +2450815|13795|3|264| +2450815|13796|3|21| +2450815|13798|3|810| +2450815|13801|3|59| +2450815|13802|3|| +2450815|13804|3|959| +2450815|13807|3|298| +2450815|13808|3|807| +2450815|13810|3|276| +2450815|13813|3|66| +2450815|13814|3|782| +2450815|13816|3|338| +2450815|13819|3|482| +2450815|13820|3|544| +2450815|13822|3|936| +2450815|13825|3|| +2450815|13826|3|825| +2450815|13828|3|728| +2450815|13831|3|986| +2450815|13832|3|990| +2450815|13834|3|419| +2450815|13837|3|659| +2450815|13838|3|356| +2450815|13840|3|118| +2450815|13843|3|753| +2450815|13844|3|254| +2450815|13846|3|142| +2450815|13849|3|640| +2450815|13850|3|185| +2450815|13852|3|469| +2450815|13855|3|755| +2450815|13856|3|650| +2450815|13858|3|576| +2450815|13861|3|362| +2450815|13862|3|106| +2450815|13864|3|919| +2450815|13867|3|20| +2450815|13868|3|688| +2450815|13870|3|536| +2450815|13873|3|307| +2450815|13874|3|59| +2450815|13876|3|286| +2450815|13879|3|737| +2450815|13880|3|498| +2450815|13882|3|234| +2450815|13885|3|576| +2450815|13886|3|664| +2450815|13888|3|629| +2450815|13891|3|| +2450815|13892|3|510| +2450815|13894|3|818| +2450815|13897|3|979| +2450815|13898|3|491| +2450815|13900|3|80| +2450815|13903|3|464| +2450815|13904|3|591| +2450815|13906|3|89| +2450815|13909|3|787| +2450815|13910|3|869| +2450815|13912|3|40| +2450815|13915|3|946| +2450815|13916|3|| +2450815|13918|3|232| +2450815|13921|3|93| +2450815|13922|3|987| +2450815|13924|3|529| +2450815|13927|3|784| +2450815|13928|3|635| +2450815|13930|3|685| +2450815|13933|3|779| +2450815|13934|3|213| +2450815|13936|3|527| +2450815|13939|3|457| +2450815|13940|3|797| +2450815|13942|3|202| +2450815|13945|3|647| +2450815|13946|3|507| +2450815|13948|3|806| +2450815|13951|3|620| +2450815|13952|3|115| +2450815|13954|3|533| +2450815|13957|3|73| +2450815|13958|3|956| +2450815|13960|3|918| +2450815|13963|3|75| +2450815|13964|3|582| +2450815|13966|3|501| +2450815|13969|3|758| +2450815|13970|3|39| +2450815|13972|3|326| +2450815|13975|3|519| +2450815|13976|3|165| +2450815|13978|3|839| +2450815|13981|3|928| +2450815|13982|3|| +2450815|13984|3|647| +2450815|13987|3|813| +2450815|13988|3|631| +2450815|13990|3|816| +2450815|13993|3|870| +2450815|13994|3|600| +2450815|13996|3|245| +2450815|13999|3|246| +2450815|14000|3|408| +2450815|14002|3|518| +2450815|14005|3|46| +2450815|14006|3|866| +2450815|14008|3|825| +2450815|14011|3|981| +2450815|14012|3|252| +2450815|14014|3|790| +2450815|14017|3|906| +2450815|14018|3|22| +2450815|14020|3|807| +2450815|14023|3|717| +2450815|14024|3|185| +2450815|14026|3|346| +2450815|14029|3|586| +2450815|14030|3|88| +2450815|14032|3|515| +2450815|14035|3|828| +2450815|14036|3|299| +2450815|14038|3|| +2450815|14041|3|533| +2450815|14042|3|368| +2450815|14044|3|527| +2450815|14047|3|615| +2450815|14048|3|504| +2450815|14050|3|262| +2450815|14053|3|108| +2450815|14054|3|510| +2450815|14056|3|108| +2450815|14059|3|119| +2450815|14060|3|375| +2450815|14062|3|324| +2450815|14065|3|698| +2450815|14066|3|862| +2450815|14068|3|912| +2450815|14071|3|593| +2450815|14072|3|119| +2450815|14074|3|287| +2450815|14077|3|514| +2450815|14078|3|943| +2450815|14080|3|460| +2450815|14083|3|449| +2450815|14084|3|946| +2450815|14086|3|211| +2450815|14089|3|58| +2450815|14090|3|359| +2450815|14092|3|881| +2450815|14095|3|| +2450815|14096|3|879| +2450815|14098|3|607| +2450815|14101|3|134| +2450815|14102|3|779| +2450815|14104|3|635| +2450815|14107|3|385| +2450815|14108|3|351| +2450815|14110|3|| +2450815|14113|3|286| +2450815|14114|3|52| +2450815|14116|3|438| +2450815|14119|3|456| +2450815|14120|3|181| +2450815|14122|3|435| +2450815|14125|3|648| +2450815|14126|3|172| +2450815|14128|3|666| +2450815|14131|3|552| +2450815|14132|3|943| +2450815|14134|3|318| +2450815|14137|3|87| +2450815|14138|3|800| +2450815|14140|3|324| +2450815|14143|3|469| +2450815|14144|3|147| +2450815|14146|3|370| +2450815|14149|3|992| +2450815|14150|3|122| +2450815|14152|3|| +2450815|14155|3|135| +2450815|14156|3|182| +2450815|14158|3|333| +2450815|14161|3|276| +2450815|14162|3|463| +2450815|14164|3|801| +2450815|14167|3|586| +2450815|14168|3|420| +2450815|14170|3|749| +2450815|14173|3|18| +2450815|14174|3|740| +2450815|14176|3|269| +2450815|14179|3|227| +2450815|14180|3|458| +2450815|14182|3|686| +2450815|14185|3|419| +2450815|14186|3|288| +2450815|14188|3|776| +2450815|14191|3|828| +2450815|14192|3|759| +2450815|14194|3|811| +2450815|14197|3|253| +2450815|14198|3|980| +2450815|14200|3|80| +2450815|14203|3|907| +2450815|14204|3|182| +2450815|14206|3|764| +2450815|14209|3|987| +2450815|14210|3|999| +2450815|14212|3|615| +2450815|14215|3|705| +2450815|14216|3|598| +2450815|14218|3|38| +2450815|14221|3|611| +2450815|14222|3|| +2450815|14224|3|833| +2450815|14227|3|853| +2450815|14228|3|959| +2450815|14230|3|579| +2450815|14233|3|154| +2450815|14234|3|266| +2450815|14236|3|858| +2450815|14239|3|38| +2450815|14240|3|611| +2450815|14242|3|724| +2450815|14245|3|404| +2450815|14246|3|3| +2450815|14248|3|160| +2450815|14251|3|459| +2450815|14252|3|491| +2450815|14254|3|944| +2450815|14257|3|605| +2450815|14258|3|390| +2450815|14260|3|116| +2450815|14263|3|827| +2450815|14264|3|954| +2450815|14266|3|96| +2450815|14269|3|919| +2450815|14270|3|892| +2450815|14272|3|896| +2450815|14275|3|73| +2450815|14276|3|320| +2450815|14278|3|274| +2450815|14281|3|967| +2450815|14282|3|465| +2450815|14284|3|16| +2450815|14287|3|965| +2450815|14288|3|995| +2450815|14290|3|635| +2450815|14293|3|980| +2450815|14294|3|0| +2450815|14296|3|203| +2450815|14299|3|329| +2450815|14300|3|952| +2450815|14302|3|603| +2450815|14305|3|707| +2450815|14306|3|332| +2450815|14308|3|892| +2450815|14311|3|576| +2450815|14312|3|340| +2450815|14314|3|471| +2450815|14317|3|505| +2450815|14318|3|706| +2450815|14320|3|711| +2450815|14323|3|256| +2450815|14324|3|140| +2450815|14326|3|997| +2450815|14329|3|545| +2450815|14330|3|201| +2450815|14332|3|651| +2450815|14335|3|606| +2450815|14336|3|484| +2450815|14338|3|593| +2450815|14341|3|946| +2450815|14342|3|950| +2450815|14344|3|825| +2450815|14347|3|397| +2450815|14348|3|682| +2450815|14350|3|358| +2450815|14353|3|132| +2450815|14354|3|522| +2450815|14356|3|186| +2450815|14359|3|383| +2450815|14360|3|725| +2450815|14362|3|447| +2450815|14365|3|| +2450815|14366|3|531| +2450815|14368|3|45| +2450815|14371|3|927| +2450815|14372|3|132| +2450815|14374|3|| +2450815|14377|3|451| +2450815|14378|3|317| +2450815|14380|3|640| +2450815|14383|3|837| +2450815|14384|3|288| +2450815|14386|3|497| +2450815|14389|3|969| +2450815|14390|3|213| +2450815|14392|3|282| +2450815|14395|3|249| +2450815|14396|3|124| +2450815|14398|3|392| +2450815|14401|3|20| +2450815|14402|3|44| +2450815|14404|3|464| +2450815|14407|3|558| +2450815|14408|3|358| +2450815|14410|3|877| +2450815|14413|3|282| +2450815|14414|3|412| +2450815|14416|3|170| +2450815|14419|3|889| +2450815|14420|3|| +2450815|14422|3|952| +2450815|14425|3|679| +2450815|14426|3|699| +2450815|14428|3|438| +2450815|14431|3|270| +2450815|14432|3|99| +2450815|14434|3|572| +2450815|14437|3|596| +2450815|14438|3|509| +2450815|14440|3|162| +2450815|14443|3|542| +2450815|14444|3|867| +2450815|14446|3|146| +2450815|14449|3|870| +2450815|14450|3|939| +2450815|14452|3|343| +2450815|14455|3|830| +2450815|14456|3|616| +2450815|14458|3|851| +2450815|14461|3|770| +2450815|14462|3|120| +2450815|14464|3|600| +2450815|14467|3|992| +2450815|14468|3|121| +2450815|14470|3|666| +2450815|14473|3|741| +2450815|14474|3|151| +2450815|14476|3|865| +2450815|14479|3|379| +2450815|14480|3|567| +2450815|14482|3|840| +2450815|14485|3|285| +2450815|14486|3|896| +2450815|14488|3|977| +2450815|14491|3|914| +2450815|14492|3|926| +2450815|14494|3|572| +2450815|14497|3|383| +2450815|14498|3|573| +2450815|14500|3|169| +2450815|14503|3|136| +2450815|14504|3|433| +2450815|14506|3|485| +2450815|14509|3|866| +2450815|14510|3|574| +2450815|14512|3|295| +2450815|14515|3|772| +2450815|14516|3|55| +2450815|14518|3|589| +2450815|14521|3|416| +2450815|14522|3|897| +2450815|14524|3|912| +2450815|14527|3|310| +2450815|14528|3|512| +2450815|14530|3|641| +2450815|14533|3|| +2450815|14534|3|762| +2450815|14536|3|711| +2450815|14539|3|636| +2450815|14540|3|| +2450815|14542|3|257| +2450815|14545|3|828| +2450815|14546|3|388| +2450815|14548|3|554| +2450815|14551|3|527| +2450815|14552|3|708| +2450815|14554|3|| +2450815|14557|3|496| +2450815|14558|3|460| +2450815|14560|3|640| +2450815|14563|3|296| +2450815|14564|3|40| +2450815|14566|3|476| +2450815|14569|3|631| +2450815|14570|3|959| +2450815|14572|3|376| +2450815|14575|3|651| +2450815|14576|3|294| +2450815|14578|3|199| +2450815|14581|3|344| +2450815|14582|3|259| +2450815|14584|3|| +2450815|14587|3|406| +2450815|14588|3|881| +2450815|14590|3|225| +2450815|14593|3|359| +2450815|14594|3|531| +2450815|14596|3|278| +2450815|14599|3|622| +2450815|14600|3|76| +2450815|14602|3|82| +2450815|14605|3|353| +2450815|14606|3|666| +2450815|14608|3|409| +2450815|14611|3|473| +2450815|14612|3|594| +2450815|14614|3|831| +2450815|14617|3|650| +2450815|14618|3|45| +2450815|14620|3|475| +2450815|14623|3|47| +2450815|14624|3|| +2450815|14626|3|876| +2450815|14629|3|121| +2450815|14630|3|138| +2450815|14632|3|702| +2450815|14635|3|179| +2450815|14636|3|| +2450815|14638|3|668| +2450815|14641|3|680| +2450815|14642|3|355| +2450815|14644|3|942| +2450815|14647|3|579| +2450815|14648|3|634| +2450815|14650|3|349| +2450815|14653|3|238| +2450815|14654|3|886| +2450815|14656|3|324| +2450815|14659|3|833| +2450815|14660|3|395| +2450815|14662|3|372| +2450815|14665|3|570| +2450815|14666|3|152| +2450815|14668|3|853| +2450815|14671|3|930| +2450815|14672|3|521| +2450815|14674|3|579| +2450815|14677|3|496| +2450815|14678|3|17| +2450815|14680|3|217| +2450815|14683|3|551| +2450815|14684|3|478| +2450815|14686|3|156| +2450815|14689|3|580| +2450815|14690|3|918| +2450815|14692|3|483| +2450815|14695|3|604| +2450815|14696|3|614| +2450815|14698|3|736| +2450815|14701|3|718| +2450815|14702|3|544| +2450815|14704|3|36| +2450815|14707|3|537| +2450815|14708|3|977| +2450815|14710|3|471| +2450815|14713|3|664| +2450815|14714|3|229| +2450815|14716|3|152| +2450815|14719|3|297| +2450815|14720|3|319| +2450815|14722|3|11| +2450815|14725|3|512| +2450815|14726|3|827| +2450815|14728|3|751| +2450815|14731|3|962| +2450815|14732|3|64| +2450815|14734|3|202| +2450815|14737|3|101| +2450815|14738|3|218| +2450815|14740|3|610| +2450815|14743|3|434| +2450815|14744|3|623| +2450815|14746|3|772| +2450815|14749|3|842| +2450815|14750|3|132| +2450815|14752|3|371| +2450815|14755|3|92| +2450815|14756|3|546| +2450815|14758|3|1| +2450815|14761|3|271| +2450815|14762|3|938| +2450815|14764|3|| +2450815|14767|3|790| +2450815|14768|3|166| +2450815|14770|3|118| +2450815|14773|3|803| +2450815|14774|3|433| +2450815|14776|3|746| +2450815|14779|3|655| +2450815|14780|3|739| +2450815|14782|3|28| +2450815|14785|3|140| +2450815|14786|3|927| +2450815|14788|3|450| +2450815|14791|3|300| +2450815|14792|3|459| +2450815|14794|3|927| +2450815|14797|3|967| +2450815|14798|3|121| +2450815|14800|3|870| +2450815|14803|3|452| +2450815|14804|3|| +2450815|14806|3|913| +2450815|14809|3|918| +2450815|14810|3|970| +2450815|14812|3|955| +2450815|14815|3|429| +2450815|14816|3|876| +2450815|14818|3|175| +2450815|14821|3|829| +2450815|14822|3|23| +2450815|14824|3|481| +2450815|14827|3|528| +2450815|14828|3|487| +2450815|14830|3|451| +2450815|14833|3|60| +2450815|14834|3|757| +2450815|14836|3|854| +2450815|14839|3|795| +2450815|14840|3|722| +2450815|14842|3|394| +2450815|14845|3|971| +2450815|14846|3|499| +2450815|14848|3|621| +2450815|14851|3|70| +2450815|14852|3|172| +2450815|14854|3|62| +2450815|14857|3|411| +2450815|14858|3|| +2450815|14860|3|313| +2450815|14863|3|287| +2450815|14864|3|36| +2450815|14866|3|406| +2450815|14869|3|863| +2450815|14870|3|491| +2450815|14872|3|860| +2450815|14875|3|939| +2450815|14876|3|379| +2450815|14878|3|377| +2450815|14881|3|475| +2450815|14882|3|959| +2450815|14884|3|554| +2450815|14887|3|846| +2450815|14888|3|30| +2450815|14890|3|441| +2450815|14893|3|232| +2450815|14894|3|429| +2450815|14896|3|128| +2450815|14899|3|568| +2450815|14900|3|315| +2450815|14902|3|367| +2450815|14905|3|676| +2450815|14906|3|80| +2450815|14908|3|54| +2450815|14911|3|35| +2450815|14912|3|5| +2450815|14914|3|183| +2450815|14917|3|316| +2450815|14918|3|255| +2450815|14920|3|539| +2450815|14923|3|372| +2450815|14924|3|717| +2450815|14926|3|760| +2450815|14929|3|709| +2450815|14930|3|859| +2450815|14932|3|233| +2450815|14935|3|103| +2450815|14936|3|201| +2450815|14938|3|212| +2450815|14941|3|894| +2450815|14942|3|549| +2450815|14944|3|689| +2450815|14947|3|922| +2450815|14948|3|| +2450815|14950|3|641| +2450815|14953|3|841| +2450815|14954|3|542| +2450815|14956|3|937| +2450815|14959|3|447| +2450815|14960|3|254| +2450815|14962|3|995| +2450815|14965|3|21| +2450815|14966|3|998| +2450815|14968|3|202| +2450815|14971|3|753| +2450815|14972|3|600| +2450815|14974|3|461| +2450815|14977|3|97| +2450815|14978|3|588| +2450815|14980|3|191| +2450815|14983|3|780| +2450815|14984|3|343| +2450815|14986|3|578| +2450815|14989|3|664| +2450815|14990|3|321| +2450815|14992|3|| +2450815|14995|3|622| +2450815|14996|3|26| +2450815|14998|3|459| +2450815|15001|3|898| +2450815|15002|3|286| +2450815|15004|3|914| +2450815|15007|3|666| +2450815|15008|3|| +2450815|15010|3|983| +2450815|15013|3|6| +2450815|15014|3|173| +2450815|15016|3|141| +2450815|15019|3|148| +2450815|15020|3|188| +2450815|15022|3|477| +2450815|15025|3|971| +2450815|15026|3|112| +2450815|15028|3|585| +2450815|15031|3|629| +2450815|15032|3|892| +2450815|15034|3|845| +2450815|15037|3|581| +2450815|15038|3|245| +2450815|15040|3|67| +2450815|15043|3|649| +2450815|15044|3|320| +2450815|15046|3|813| +2450815|15049|3|90| +2450815|15050|3|570| +2450815|15052|3|53| +2450815|15055|3|460| +2450815|15056|3|643| +2450815|15058|3|426| +2450815|15061|3|744| +2450815|15062|3|214| +2450815|15064|3|761| +2450815|15067|3|336| +2450815|15068|3|949| +2450815|15070|3|289| +2450815|15073|3|606| +2450815|15074|3|657| +2450815|15076|3|769| +2450815|15079|3|145| +2450815|15080|3|335| +2450815|15082|3|61| +2450815|15085|3|933| +2450815|15086|3|615| +2450815|15088|3|368| +2450815|15091|3|633| +2450815|15092|3|657| +2450815|15094|3|791| +2450815|15097|3|732| +2450815|15098|3|691| +2450815|15100|3|526| +2450815|15103|3|867| +2450815|15104|3|157| +2450815|15106|3|10| +2450815|15109|3|649| +2450815|15110|3|374| +2450815|15112|3|527| +2450815|15115|3|557| +2450815|15116|3|147| +2450815|15118|3|513| +2450815|15121|3|401| +2450815|15122|3|654| +2450815|15124|3|111| +2450815|15127|3|971| +2450815|15128|3|409| +2450815|15130|3|535| +2450815|15133|3|155| +2450815|15134|3|440| +2450815|15136|3|703| +2450815|15139|3|262| +2450815|15140|3|809| +2450815|15142|3|379| +2450815|15145|3|361| +2450815|15146|3|392| +2450815|15148|3|484| +2450815|15151|3|104| +2450815|15152|3|308| +2450815|15154|3|696| +2450815|15157|3|950| +2450815|15158|3|56| +2450815|15160|3|705| +2450815|15163|3|183| +2450815|15164|3|3| +2450815|15166|3|407| +2450815|15169|3|624| +2450815|15170|3|19| +2450815|15172|3|617| +2450815|15175|3|844| +2450815|15176|3|459| +2450815|15178|3|886| +2450815|15181|3|116| +2450815|15182|3|916| +2450815|15184|3|211| +2450815|15187|3|642| +2450815|15188|3|326| +2450815|15190|3|911| +2450815|15193|3|990| +2450815|15194|3|443| +2450815|15196|3|746| +2450815|15199|3|606| +2450815|15200|3|244| +2450815|15202|3|693| +2450815|15205|3|471| +2450815|15206|3|26| +2450815|15208|3|828| +2450815|15211|3|982| +2450815|15212|3|784| +2450815|15214|3|333| +2450815|15217|3|25| +2450815|15218|3|524| +2450815|15220|3|195| +2450815|15223|3|375| +2450815|15224|3|854| +2450815|15226|3|429| +2450815|15229|3|408| +2450815|15230|3|673| +2450815|15232|3|891| +2450815|15235|3|161| +2450815|15236|3|583| +2450815|15238|3|574| +2450815|15241|3|702| +2450815|15242|3|222| +2450815|15244|3|67| +2450815|15247|3|987| +2450815|15248|3|555| +2450815|15250|3|405| +2450815|15253|3|942| +2450815|15254|3|644| +2450815|15256|3|170| +2450815|15259|3|520| +2450815|15260|3|907| +2450815|15262|3|536| +2450815|15265|3|851| +2450815|15266|3|659| +2450815|15268|3|155| +2450815|15271|3|50| +2450815|15272|3|433| +2450815|15274|3|441| +2450815|15277|3|412| +2450815|15278|3|855| +2450815|15280|3|83| +2450815|15283|3|200| +2450815|15284|3|593| +2450815|15286|3|687| +2450815|15289|3|| +2450815|15290|3|210| +2450815|15292|3|410| +2450815|15295|3|247| +2450815|15296|3|780| +2450815|15298|3|| +2450815|15301|3|646| +2450815|15302|3|532| +2450815|15304|3|179| +2450815|15307|3|531| +2450815|15308|3|990| +2450815|15310|3|208| +2450815|15313|3|792| +2450815|15314|3|118| +2450815|15316|3|610| +2450815|15319|3|958| +2450815|15320|3|345| +2450815|15322|3|203| +2450815|15325|3|630| +2450815|15326|3|586| +2450815|15328|3|281| +2450815|15331|3|91| +2450815|15332|3|283| +2450815|15334|3|75| +2450815|15337|3|904| +2450815|15338|3|668| +2450815|15340|3|618| +2450815|15343|3|540| +2450815|15344|3|180| +2450815|15346|3|709| +2450815|15349|3|963| +2450815|15350|3|817| +2450815|15352|3|295| +2450815|15355|3|977| +2450815|15356|3|153| +2450815|15358|3|806| +2450815|15361|3|38| +2450815|15362|3|985| +2450815|15364|3|71| +2450815|15367|3|841| +2450815|15368|3|753| +2450815|15370|3|307| +2450815|15373|3|899| +2450815|15374|3|754| +2450815|15376|3|751| +2450815|15379|3|119| +2450815|15380|3|371| +2450815|15382|3|308| +2450815|15385|3|712| +2450815|15386|3|645| +2450815|15388|3|193| +2450815|15391|3|824| +2450815|15392|3|| +2450815|15394|3|200| +2450815|15397|3|736| +2450815|15398|3|23| +2450815|15400|3|708| +2450815|15403|3|767| +2450815|15404|3|473| +2450815|15406|3|438| +2450815|15409|3|| +2450815|15410|3|79| +2450815|15412|3|| +2450815|15415|3|614| +2450815|15416|3|893| +2450815|15418|3|676| +2450815|15421|3|924| +2450815|15422|3|| +2450815|15424|3|958| +2450815|15427|3|790| +2450815|15428|3|755| +2450815|15430|3|478| +2450815|15433|3|52| +2450815|15434|3|168| +2450815|15436|3|94| +2450815|15439|3|532| +2450815|15440|3|117| +2450815|15442|3|389| +2450815|15445|3|804| +2450815|15446|3|685| +2450815|15448|3|7| +2450815|15451|3|532| +2450815|15452|3|583| +2450815|15454|3|853| +2450815|15457|3|512| +2450815|15458|3|683| +2450815|15460|3|788| +2450815|15463|3|434| +2450815|15464|3|187| +2450815|15466|3|658| +2450815|15469|3|| +2450815|15470|3|224| +2450815|15472|3|238| +2450815|15475|3|775| +2450815|15476|3|728| +2450815|15478|3|534| +2450815|15481|3|798| +2450815|15482|3|683| +2450815|15484|3|426| +2450815|15487|3|852| +2450815|15488|3|552| +2450815|15490|3|981| +2450815|15493|3|109| +2450815|15494|3|| +2450815|15496|3|131| +2450815|15499|3|920| +2450815|15500|3|737| +2450815|15502|3|406| +2450815|15505|3|242| +2450815|15506|3|680| +2450815|15508|3|423| +2450815|15511|3|980| +2450815|15512|3|607| +2450815|15514|3|405| +2450815|15517|3|669| +2450815|15518|3|527| +2450815|15520|3|| +2450815|15523|3|795| +2450815|15524|3|456| +2450815|15526|3|86| +2450815|15529|3|156| +2450815|15530|3|314| +2450815|15532|3|871| +2450815|15535|3|931| +2450815|15536|3|996| +2450815|15538|3|675| +2450815|15541|3|443| +2450815|15542|3|398| +2450815|15544|3|694| +2450815|15547|3|623| +2450815|15548|3|606| +2450815|15550|3|251| +2450815|15553|3|691| +2450815|15554|3|144| +2450815|15556|3|132| +2450815|15559|3|258| +2450815|15560|3|114| +2450815|15562|3|675| +2450815|15565|3|| +2450815|15566|3|843| +2450815|15568|3|927| +2450815|15571|3|820| +2450815|15572|3|75| +2450815|15574|3|467| +2450815|15577|3|752| +2450815|15578|3|736| +2450815|15580|3|89| +2450815|15583|3|487| +2450815|15584|3|474| +2450815|15586|3|613| +2450815|15589|3|938| +2450815|15590|3|720| +2450815|15592|3|277| +2450815|15595|3|867| +2450815|15596|3|60| +2450815|15598|3|850| +2450815|15601|3|| +2450815|15602|3|798| +2450815|15604|3|143| +2450815|15607|3|38| +2450815|15608|3|509| +2450815|15610|3|827| +2450815|15613|3|614| +2450815|15614|3|49| +2450815|15616|3|13| +2450815|15619|3|675| +2450815|15620|3|868| +2450815|15622|3|703| +2450815|15625|3|38| +2450815|15626|3|307| +2450815|15628|3|615| +2450815|15631|3|342| +2450815|15632|3|982| +2450815|15634|3|97| +2450815|15637|3|0| +2450815|15638|3|741| +2450815|15640|3|| +2450815|15643|3|419| +2450815|15644|3|583| +2450815|15646|3|551| +2450815|15649|3|991| +2450815|15650|3|997| +2450815|15652|3|721| +2450815|15655|3|701| +2450815|15656|3|209| +2450815|15658|3|725| +2450815|15661|3|363| +2450815|15662|3|987| +2450815|15664|3|231| +2450815|15667|3|32| +2450815|15668|3|388| +2450815|15670|3|16| +2450815|15673|3|757| +2450815|15674|3|| +2450815|15676|3|374| +2450815|15679|3|788| +2450815|15680|3|923| +2450815|15682|3|224| +2450815|15685|3|929| +2450815|15686|3|516| +2450815|15688|3|577| +2450815|15691|3|313| +2450815|15692|3|450| +2450815|15694|3|776| +2450815|15697|3|| +2450815|15698|3|791| +2450815|15700|3|1| +2450815|15703|3|460| +2450815|15704|3|877| +2450815|15706|3|187| +2450815|15709|3|557| +2450815|15710|3|84| +2450815|15712|3|163| +2450815|15715|3|704| +2450815|15716|3|133| +2450815|15718|3|382| +2450815|15721|3|| +2450815|15722|3|340| +2450815|15724|3|927| +2450815|15727|3|781| +2450815|15728|3|408| +2450815|15730|3|124| +2450815|15733|3|60| +2450815|15734|3|126| +2450815|15736|3|214| +2450815|15739|3|710| +2450815|15740|3|51| +2450815|15742|3|253| +2450815|15745|3|762| +2450815|15746|3|| +2450815|15748|3|631| +2450815|15751|3|231| +2450815|15752|3|624| +2450815|15754|3|842| +2450815|15757|3|32| +2450815|15758|3|335| +2450815|15760|3|342| +2450815|15763|3|778| +2450815|15764|3|328| +2450815|15766|3|944| +2450815|15769|3|110| +2450815|15770|3|53| +2450815|15772|3|899| +2450815|15775|3|969| +2450815|15776|3|440| +2450815|15778|3|864| +2450815|15781|3|492| +2450815|15782|3|601| +2450815|15784|3|296| +2450815|15787|3|340| +2450815|15788|3|555| +2450815|15790|3|205| +2450815|15793|3|512| +2450815|15794|3|286| +2450815|15796|3|912| +2450815|15799|3|218| +2450815|15800|3|| +2450815|15802|3|176| +2450815|15805|3|365| +2450815|15806|3|404| +2450815|15808|3|310| +2450815|15811|3|377| +2450815|15812|3|575| +2450815|15814|3|433| +2450815|15817|3|959| +2450815|15818|3|257| +2450815|15820|3|366| +2450815|15823|3|429| +2450815|15824|3|39| +2450815|15826|3|96| +2450815|15829|3|133| +2450815|15830|3|601| +2450815|15832|3|233| +2450815|15835|3|556| +2450815|15836|3|843| +2450815|15838|3|784| +2450815|15841|3|71| +2450815|15842|3|275| +2450815|15844|3|306| +2450815|15847|3|| +2450815|15848|3|71| +2450815|15850|3|255| +2450815|15853|3|455| +2450815|15854|3|968| +2450815|15856|3|| +2450815|15859|3|454| +2450815|15860|3|259| +2450815|15862|3|27| +2450815|15865|3|65| +2450815|15866|3|431| +2450815|15868|3|617| +2450815|15871|3|888| +2450815|15872|3|414| +2450815|15874|3|887| +2450815|15877|3|941| +2450815|15878|3|| +2450815|15880|3|179| +2450815|15883|3|893| +2450815|15884|3|3| +2450815|15886|3|| +2450815|15889|3|662| +2450815|15890|3|954| +2450815|15892|3|12| +2450815|15895|3|799| +2450815|15896|3|662| +2450815|15898|3|99| +2450815|15901|3|677| +2450815|15902|3|495| +2450815|15904|3|122| +2450815|15907|3|| +2450815|15908|3|618| +2450815|15910|3|112| +2450815|15913|3|136| +2450815|15914|3|647| +2450815|15916|3|230| +2450815|15919|3|| +2450815|15920|3|939| +2450815|15922|3|906| +2450815|15925|3|889| +2450815|15926|3|| +2450815|15928|3|675| +2450815|15931|3|67| +2450815|15932|3|433| +2450815|15934|3|816| +2450815|15937|3|304| +2450815|15938|3|| +2450815|15940|3|383| +2450815|15943|3|106| +2450815|15944|3|975| +2450815|15946|3|429| +2450815|15949|3|912| +2450815|15950|3|160| +2450815|15952|3|99| +2450815|15955|3|398| +2450815|15956|3|710| +2450815|15958|3|701| +2450815|15961|3|175| +2450815|15962|3|757| +2450815|15964|3|524| +2450815|15967|3|608| +2450815|15968|3|621| +2450815|15970|3|209| +2450815|15973|3|228| +2450815|15974|3|159| +2450815|15976|3|35| +2450815|15979|3|901| +2450815|15980|3|864| +2450815|15982|3|33| +2450815|15985|3|796| +2450815|15986|3|157| +2450815|15988|3|449| +2450815|15991|3|430| +2450815|15992|3|| +2450815|15994|3|75| +2450815|15997|3|558| +2450815|15998|3|941| +2450815|16000|3|319| +2450815|16003|3|319| +2450815|16004|3|330| +2450815|16006|3|176| +2450815|16009|3|616| +2450815|16010|3|333| +2450815|16012|3|159| +2450815|16015|3|690| +2450815|16016|3|| +2450815|16018|3|364| +2450815|16021|3|385| +2450815|16022|3|541| +2450815|16024|3|208| +2450815|16027|3|130| +2450815|16028|3|573| +2450815|16030|3|729| +2450815|16033|3|509| +2450815|16034|3|42| +2450815|16036|3|| +2450815|16039|3|289| +2450815|16040|3|20| +2450815|16042|3|752| +2450815|16045|3|469| +2450815|16046|3|253| +2450815|16048|3|473| +2450815|16051|3|15| +2450815|16052|3|187| +2450815|16054|3|395| +2450815|16057|3|540| +2450815|16058|3|| +2450815|16060|3|252| +2450815|16063|3|221| +2450815|16064|3|531| +2450815|16066|3|828| +2450815|16069|3|420| +2450815|16070|3|653| +2450815|16072|3|748| +2450815|16075|3|860| +2450815|16076|3|605| +2450815|16078|3|74| +2450815|16081|3|280| +2450815|16082|3|238| +2450815|16084|3|| +2450815|16087|3|684| +2450815|16088|3|270| +2450815|16090|3|348| +2450815|16093|3|583| +2450815|16094|3|988| +2450815|16096|3|339| +2450815|16099|3|482| +2450815|16100|3|923| +2450815|16102|3|764| +2450815|16105|3|493| +2450815|16106|3|887| +2450815|16108|3|518| +2450815|16111|3|196| +2450815|16112|3|769| +2450815|16114|3|602| +2450815|16117|3|| +2450815|16118|3|879| +2450815|16120|3|158| +2450815|16123|3|135| +2450815|16124|3|533| +2450815|16126|3|677| +2450815|16129|3|874| +2450815|16130|3|489| +2450815|16132|3|775| +2450815|16135|3|614| +2450815|16136|3|820| +2450815|16138|3|906| +2450815|16141|3|184| +2450815|16142|3|| +2450815|16144|3|706| +2450815|16147|3|794| +2450815|16148|3|909| +2450815|16150|3|588| +2450815|16153|3|602| +2450815|16154|3|851| +2450815|16156|3|441| +2450815|16159|3|718| +2450815|16160|3|353| +2450815|16162|3|269| +2450815|16165|3|570| +2450815|16166|3|433| +2450815|16168|3|315| +2450815|16171|3|113| +2450815|16172|3|973| +2450815|16174|3|284| +2450815|16177|3|536| +2450815|16178|3|99| +2450815|16180|3|567| +2450815|16183|3|600| +2450815|16184|3|579| +2450815|16186|3|944| +2450815|16189|3|130| +2450815|16190|3|338| +2450815|16192|3|686| +2450815|16195|3|362| +2450815|16196|3|183| +2450815|16198|3|664| +2450815|16201|3|65| +2450815|16202|3|290| +2450815|16204|3|525| +2450815|16207|3|448| +2450815|16208|3|506| +2450815|16210|3|| +2450815|16213|3|90| +2450815|16214|3|81| +2450815|16216|3|353| +2450815|16219|3|834| +2450815|16220|3|883| +2450815|16222|3|800| +2450815|16225|3|568| +2450815|16226|3|540| +2450815|16228|3|574| +2450815|16231|3|551| +2450815|16232|3|494| +2450815|16234|3|488| +2450815|16237|3|301| +2450815|16238|3|240| +2450815|16240|3|771| +2450815|16243|3|958| +2450815|16244|3|714| +2450815|16246|3|782| +2450815|16249|3|177| +2450815|16250|3|693| +2450815|16252|3|706| +2450815|16255|3|376| +2450815|16256|3|513| +2450815|16258|3|956| +2450815|16261|3|404| +2450815|16262|3|208| +2450815|16264|3|953| +2450815|16267|3|63| +2450815|16268|3|256| +2450815|16270|3|199| +2450815|16273|3|772| +2450815|16274|3|327| +2450815|16276|3|249| +2450815|16279|3|938| +2450815|16280|3|142| +2450815|16282|3|576| +2450815|16285|3|414| +2450815|16286|3|464| +2450815|16288|3|311| +2450815|16291|3|565| +2450815|16292|3|33| +2450815|16294|3|715| +2450815|16297|3|726| +2450815|16298|3|4| +2450815|16300|3|281| +2450815|16303|3|538| +2450815|16304|3|536| +2450815|16306|3|| +2450815|16309|3|994| +2450815|16310|3|651| +2450815|16312|3|280| +2450815|16315|3|540| +2450815|16316|3|88| +2450815|16318|3|153| +2450815|16321|3|807| +2450815|16322|3|286| +2450815|16324|3|399| +2450815|16327|3|838| +2450815|16328|3|498| +2450815|16330|3|259| +2450815|16333|3|79| +2450815|16334|3|305| +2450815|16336|3|290| +2450815|16339|3|181| +2450815|16340|3|827| +2450815|16342|3|580| +2450815|16345|3|755| +2450815|16346|3|42| +2450815|16348|3|| +2450815|16351|3|531| +2450815|16352|3|612| +2450815|16354|3|581| +2450815|16357|3|689| +2450815|16358|3|563| +2450815|16360|3|986| +2450815|16363|3|631| +2450815|16364|3|926| +2450815|16366|3|166| +2450815|16369|3|620| +2450815|16370|3|594| +2450815|16372|3|606| +2450815|16375|3|86| +2450815|16376|3|890| +2450815|16378|3|419| +2450815|16381|3|840| +2450815|16382|3|375| +2450815|16384|3|105| +2450815|16387|3|826| +2450815|16388|3|736| +2450815|16390|3|178| +2450815|16393|3|922| +2450815|16394|3|320| +2450815|16396|3|55| +2450815|16399|3|320| +2450815|16400|3|858| +2450815|16402|3|427| +2450815|16405|3|304| +2450815|16406|3|75| +2450815|16408|3|711| +2450815|16411|3|670| +2450815|16412|3|401| +2450815|16414|3|427| +2450815|16417|3|472| +2450815|16418|3|436| +2450815|16420|3|691| +2450815|16423|3|959| +2450815|16424|3|512| +2450815|16426|3|650| +2450815|16429|3|519| +2450815|16430|3|| +2450815|16432|3|699| +2450815|16435|3|291| +2450815|16436|3|74| +2450815|16438|3|327| +2450815|16441|3|392| +2450815|16442|3|742| +2450815|16444|3|67| +2450815|16447|3|874| +2450815|16448|3|661| +2450815|16450|3|432| +2450815|16453|3|408| +2450815|16454|3|| +2450815|16456|3|262| +2450815|16459|3|463| +2450815|16460|3|| +2450815|16462|3|513| +2450815|16465|3|200| +2450815|16466|3|409| +2450815|16468|3|18| +2450815|16471|3|334| +2450815|16472|3|444| +2450815|16474|3|829| +2450815|16477|3|917| +2450815|16478|3|837| +2450815|16480|3|| +2450815|16483|3|929| +2450815|16484|3|730| +2450815|16486|3|285| +2450815|16489|3|278| +2450815|16490|3|842| +2450815|16492|3|121| +2450815|16495|3|598| +2450815|16496|3|180| +2450815|16498|3|440| +2450815|16501|3|733| +2450815|16502|3|242| +2450815|16504|3|444| +2450815|16507|3|647| +2450815|16508|3|401| +2450815|16510|3|592| +2450815|16513|3|316| +2450815|16514|3|643| +2450815|16516|3|24| +2450815|16519|3|489| +2450815|16520|3|| +2450815|16522|3|904| +2450815|16525|3|108| +2450815|16526|3|913| +2450815|16528|3|626| +2450815|16531|3|652| +2450815|16532|3|458| +2450815|16534|3|913| +2450815|16537|3|809| +2450815|16538|3|175| +2450815|16540|3|589| +2450815|16543|3|784| +2450815|16544|3|231| +2450815|16546|3|959| +2450815|16549|3|212| +2450815|16550|3|318| +2450815|16552|3|315| +2450815|16555|3|139| +2450815|16556|3|321| +2450815|16558|3|634| +2450815|16561|3|659| +2450815|16562|3|593| +2450815|16564|3|| +2450815|16567|3|571| +2450815|16568|3|721| +2450815|16570|3|180| +2450815|16573|3|687| +2450815|16574|3|887| +2450815|16576|3|695| +2450815|16579|3|484| +2450815|16580|3|415| +2450815|16582|3|880| +2450815|16585|3|487| +2450815|16586|3|238| +2450815|16588|3|181| +2450815|16591|3|12| +2450815|16592|3|646| +2450815|16594|3|966| +2450815|16597|3|252| +2450815|16598|3|806| +2450815|16600|3|182| +2450815|16603|3|3| +2450815|16604|3|127| +2450815|16606|3|371| +2450815|16609|3|791| +2450815|16610|3|621| +2450815|16612|3|21| +2450815|16615|3|342| +2450815|16616|3|52| +2450815|16618|3|182| +2450815|16621|3|36| +2450815|16622|3|528| +2450815|16624|3|| +2450815|16627|3|204| +2450815|16628|3|201| +2450815|16630|3|670| +2450815|16633|3|331| +2450815|16634|3|280| +2450815|16636|3|243| +2450815|16639|3|122| +2450815|16640|3|416| +2450815|16642|3|643| +2450815|16645|3|533| +2450815|16646|3|170| +2450815|16648|3|56| +2450815|16651|3|958| +2450815|16652|3|994| +2450815|16654|3|459| +2450815|16657|3|0| +2450815|16658|3|690| +2450815|16660|3|847| +2450815|16663|3|721| +2450815|16664|3|220| +2450815|16666|3|734| +2450815|16669|3|758| +2450815|16670|3|386| +2450815|16672|3|273| +2450815|16675|3|572| +2450815|16676|3|138| +2450815|16678|3|698| +2450815|16681|3|162| +2450815|16682|3|522| +2450815|16684|3|| +2450815|16687|3|| +2450815|16688|3|125| +2450815|16690|3|971| +2450815|16693|3|446| +2450815|16694|3|671| +2450815|16696|3|450| +2450815|16699|3|44| +2450815|16700|3|295| +2450815|16702|3|833| +2450815|16705|3|159| +2450815|16706|3|918| +2450815|16708|3|784| +2450815|16711|3|48| +2450815|16712|3|711| +2450815|16714|3|872| +2450815|16717|3|565| +2450815|16718|3|625| +2450815|16720|3|158| +2450815|16723|3|959| +2450815|16724|3|35| +2450815|16726|3|340| +2450815|16729|3|67| +2450815|16730|3|982| +2450815|16732|3|82| +2450815|16735|3|501| +2450815|16736|3|287| +2450815|16738|3|552| +2450815|16741|3|| +2450815|16742|3|238| +2450815|16744|3|| +2450815|16747|3|394| +2450815|16748|3|651| +2450815|16750|3|12| +2450815|16753|3|532| +2450815|16754|3|935| +2450815|16756|3|597| +2450815|16759|3|727| +2450815|16760|3|928| +2450815|16762|3|336| +2450815|16765|3|986| +2450815|16766|3|69| +2450815|16768|3|915| +2450815|16771|3|691| +2450815|16772|3|188| +2450815|16774|3|| +2450815|16777|3|312| +2450815|16778|3|779| +2450815|16780|3|994| +2450815|16783|3|869| +2450815|16784|3|890| +2450815|16786|3|403| +2450815|16789|3|379| +2450815|16790|3|964| +2450815|16792|3|| +2450815|16795|3|954| +2450815|16796|3|520| +2450815|16798|3|1| +2450815|16801|3|586| +2450815|16802|3|247| +2450815|16804|3|804| +2450815|16807|3|932| +2450815|16808|3|456| +2450815|16810|3|413| +2450815|16813|3|901| +2450815|16814|3|453| +2450815|16816|3|958| +2450815|16819|3|660| +2450815|16820|3|847| +2450815|16822|3|965| +2450815|16825|3|306| +2450815|16826|3|157| +2450815|16828|3|739| +2450815|16831|3|105| +2450815|16832|3|384| +2450815|16834|3|366| +2450815|16837|3|84| +2450815|16838|3|584| +2450815|16840|3|736| +2450815|16843|3|356| +2450815|16844|3|23| +2450815|16846|3|634| +2450815|16849|3|621| +2450815|16850|3|437| +2450815|16852|3|477| +2450815|16855|3|937| +2450815|16856|3|427| +2450815|16858|3|885| +2450815|16861|3|79| +2450815|16862|3|273| +2450815|16864|3|213| +2450815|16867|3|892| +2450815|16868|3|152| +2450815|16870|3|968| +2450815|16873|3|875| +2450815|16874|3|658| +2450815|16876|3|114| +2450815|16879|3|708| +2450815|16880|3|658| +2450815|16882|3|| +2450815|16885|3|55| +2450815|16886|3|42| +2450815|16888|3|423| +2450815|16891|3|85| +2450815|16892|3|880| +2450815|16894|3|577| +2450815|16897|3|497| +2450815|16898|3|796| +2450815|16900|3|822| +2450815|16903|3|523| +2450815|16904|3|| +2450815|16906|3|298| +2450815|16909|3|855| +2450815|16910|3|783| +2450815|16912|3|540| +2450815|16915|3|921| +2450815|16916|3|522| +2450815|16918|3|985| +2450815|16921|3|914| +2450815|16922|3|78| +2450815|16924|3|383| +2450815|16927|3|9| +2450815|16928|3|762| +2450815|16930|3|427| +2450815|16933|3|203| +2450815|16934|3|591| +2450815|16936|3|136| +2450815|16939|3|383| +2450815|16940|3|978| +2450815|16942|3|942| +2450815|16945|3|16| +2450815|16946|3|359| +2450815|16948|3|381| +2450815|16951|3|34| +2450815|16952|3|281| +2450815|16954|3|| +2450815|16957|3|627| +2450815|16958|3|164| +2450815|16960|3|285| +2450815|16963|3|82| +2450815|16964|3|741| +2450815|16966|3|718| +2450815|16969|3|342| +2450815|16970|3|428| +2450815|16972|3|705| +2450815|16975|3|321| +2450815|16976|3|979| +2450815|16978|3|920| +2450815|16981|3|882| +2450815|16982|3|706| +2450815|16984|3|535| +2450815|16987|3|165| +2450815|16988|3|396| +2450815|16990|3|46| +2450815|16993|3|280| +2450815|16994|3|521| +2450815|16996|3|799| +2450815|16999|3|480| +2450815|17000|3|221| +2450815|17002|3|922| +2450815|17005|3|627| +2450815|17006|3|112| +2450815|17008|3|880| +2450815|17011|3|330| +2450815|17012|3|243| +2450815|17014|3|917| +2450815|17017|3|924| +2450815|17018|3|513| +2450815|17020|3|800| +2450815|17023|3|| +2450815|17024|3|227| +2450815|17026|3|195| +2450815|17029|3|489| +2450815|17030|3|679| +2450815|17032|3|213| +2450815|17035|3|3| +2450815|17036|3|247| +2450815|17038|3|265| +2450815|17041|3|919| +2450815|17042|3|498| +2450815|17044|3|866| +2450815|17047|3|32| +2450815|17048|3|680| +2450815|17050|3|343| +2450815|17053|3|381| +2450815|17054|3|410| +2450815|17056|3|876| +2450815|17059|3|| +2450815|17060|3|45| +2450815|17062|3|775| +2450815|17065|3|56| +2450815|17066|3|842| +2450815|17068|3|442| +2450815|17071|3|194| +2450815|17072|3|424| +2450815|17074|3|700| +2450815|17077|3|547| +2450815|17078|3|459| +2450815|17080|3|957| +2450815|17083|3|800| +2450815|17084|3|220| +2450815|17086|3|631| +2450815|17089|3|243| +2450815|17090|3|870| +2450815|17092|3|121| +2450815|17095|3|69| +2450815|17096|3|850| +2450815|17098|3|256| +2450815|17101|3|106| +2450815|17102|3|620| +2450815|17104|3|279| +2450815|17107|3|656| +2450815|17108|3|799| +2450815|17110|3|403| +2450815|17113|3|543| +2450815|17114|3|30| +2450815|17116|3|484| +2450815|17119|3|| +2450815|17120|3|229| +2450815|17122|3|925| +2450815|17125|3|231| +2450815|17126|3|13| +2450815|17128|3|727| +2450815|17131|3|967| +2450815|17132|3|| +2450815|17134|3|162| +2450815|17137|3|699| +2450815|17138|3|901| +2450815|17140|3|57| +2450815|17143|3|749| +2450815|17144|3|979| +2450815|17146|3|109| +2450815|17149|3|563| +2450815|17150|3|620| +2450815|17152|3|66| +2450815|17155|3|487| +2450815|17156|3|797| +2450815|17158|3|698| +2450815|17161|3|459| +2450815|17162|3|74| +2450815|17164|3|0| +2450815|17167|3|226| +2450815|17168|3|842| +2450815|17170|3|335| +2450815|17173|3|351| +2450815|17174|3|672| +2450815|17176|3|812| +2450815|17179|3|700| +2450815|17180|3|328| +2450815|17182|3|787| +2450815|17185|3|46| +2450815|17186|3|739| +2450815|17188|3|152| +2450815|17191|3|| +2450815|17192|3|698| +2450815|17194|3|| +2450815|17197|3|590| +2450815|17198|3|256| +2450815|17200|3|248| +2450815|17203|3|410| +2450815|17204|3|312| +2450815|17206|3|153| +2450815|17209|3|151| +2450815|17210|3|924| +2450815|17212|3|535| +2450815|17215|3|428| +2450815|17216|3|644| +2450815|17218|3|| +2450815|17221|3|712| +2450815|17222|3|297| +2450815|17224|3|511| +2450815|17227|3|49| +2450815|17228|3|687| +2450815|17230|3|608| +2450815|17233|3|687| +2450815|17234|3|400| +2450815|17236|3|222| +2450815|17239|3|780| +2450815|17240|3|816| +2450815|17242|3|503| +2450815|17245|3|606| +2450815|17246|3|37| +2450815|17248|3|| +2450815|17251|3|56| +2450815|17252|3|134| +2450815|17254|3|963| +2450815|17257|3|579| +2450815|17258|3|727| +2450815|17260|3|803| +2450815|17263|3|212| +2450815|17264|3|518| +2450815|17266|3|951| +2450815|17269|3|138| +2450815|17270|3|25| +2450815|17272|3|896| +2450815|17275|3|| +2450815|17276|3|379| +2450815|17278|3|397| +2450815|17281|3|199| +2450815|17282|3|204| +2450815|17284|3|757| +2450815|17287|3|33| +2450815|17288|3|| +2450815|17290|3|203| +2450815|17293|3|380| +2450815|17294|3|185| +2450815|17296|3|930| +2450815|17299|3|535| +2450815|17300|3|800| +2450815|17302|3|275| +2450815|17305|3|165| +2450815|17306|3|203| +2450815|17308|3|940| +2450815|17311|3|270| +2450815|17312|3|549| +2450815|17314|3|437| +2450815|17317|3|937| +2450815|17318|3|12| +2450815|17320|3|971| +2450815|17323|3|| +2450815|17324|3|899| +2450815|17326|3|537| +2450815|17329|3|936| +2450815|17330|3|734| +2450815|17332|3|832| +2450815|17335|3|906| +2450815|17336|3|145| +2450815|17338|3|921| +2450815|17341|3|996| +2450815|17342|3|368| +2450815|17344|3|952| +2450815|17347|3|163| +2450815|17348|3|| +2450815|17350|3|258| +2450815|17353|3|855| +2450815|17354|3|440| +2450815|17356|3|4| +2450815|17359|3|335| +2450815|17360|3|368| +2450815|17362|3|550| +2450815|17365|3|| +2450815|17366|3|687| +2450815|17368|3|33| +2450815|17371|3|687| +2450815|17372|3|99| +2450815|17374|3|403| +2450815|17377|3|504| +2450815|17378|3|397| +2450815|17380|3|741| +2450815|17383|3|104| +2450815|17384|3|475| +2450815|17386|3|436| +2450815|17389|3|221| +2450815|17390|3|742| +2450815|17392|3|922| +2450815|17395|3|293| +2450815|17396|3|692| +2450815|17398|3|294| +2450815|17401|3|744| +2450815|17402|3|414| +2450815|17404|3|638| +2450815|17407|3|37| +2450815|17408|3|763| +2450815|17410|3|252| +2450815|17413|3|200| +2450815|17414|3|366| +2450815|17416|3|593| +2450815|17419|3|662| +2450815|17420|3|883| +2450815|17422|3|481| +2450815|17425|3|882| +2450815|17426|3|394| +2450815|17428|3|350| +2450815|17431|3|562| +2450815|17432|3|471| +2450815|17434|3|296| +2450815|17437|3|364| +2450815|17438|3|812| +2450815|17440|3|416| +2450815|17443|3|876| +2450815|17444|3|896| +2450815|17446|3|129| +2450815|17449|3|988| +2450815|17450|3|965| +2450815|17452|3|619| +2450815|17455|3|312| +2450815|17456|3|911| +2450815|17458|3|222| +2450815|17461|3|803| +2450815|17462|3|110| +2450815|17464|3|442| +2450815|17467|3|85| +2450815|17468|3|735| +2450815|17470|3|745| +2450815|17473|3|323| +2450815|17474|3|442| +2450815|17476|3|942| +2450815|17479|3|929| +2450815|17480|3|697| +2450815|17482|3|376| +2450815|17485|3|955| +2450815|17486|3|54| +2450815|17488|3|347| +2450815|17491|3|462| +2450815|17492|3|18| +2450815|17494|3|| +2450815|17497|3|420| +2450815|17498|3|184| +2450815|17500|3|242| +2450815|17503|3|116| +2450815|17504|3|19| +2450815|17506|3|653| +2450815|17509|3|123| +2450815|17510|3|537| +2450815|17512|3|317| +2450815|17515|3|219| +2450815|17516|3|660| +2450815|17518|3|319| +2450815|17521|3|587| +2450815|17522|3|523| +2450815|17524|3|980| +2450815|17527|3|519| +2450815|17528|3|608| +2450815|17530|3|264| +2450815|17533|3|427| +2450815|17534|3|843| +2450815|17536|3|119| +2450815|17539|3|311| +2450815|17540|3|254| +2450815|17542|3|456| +2450815|17545|3|922| +2450815|17546|3|842| +2450815|17548|3|6| +2450815|17551|3|691| +2450815|17552|3|769| +2450815|17554|3|743| +2450815|17557|3|477| +2450815|17558|3|795| +2450815|17560|3|781| +2450815|17563|3|661| +2450815|17564|3|75| +2450815|17566|3|221| +2450815|17569|3|333| +2450815|17570|3|543| +2450815|17572|3|919| +2450815|17575|3|481| +2450815|17576|3|204| +2450815|17578|3|778| +2450815|17581|3|983| +2450815|17582|3|100| +2450815|17584|3|796| +2450815|17587|3|271| +2450815|17588|3|870| +2450815|17590|3|213| +2450815|17593|3|448| +2450815|17594|3|252| +2450815|17596|3|321| +2450815|17599|3|616| +2450815|17600|3|839| +2450815|17602|3|427| +2450815|17605|3|885| +2450815|17606|3|722| +2450815|17608|3|950| +2450815|17611|3|836| +2450815|17612|3|| +2450815|17614|3|806| +2450815|17617|3|259| +2450815|17618|3|609| +2450815|17620|3|144| +2450815|17623|3|432| +2450815|17624|3|501| +2450815|17626|3|| +2450815|17629|3|613| +2450815|17630|3|773| +2450815|17632|3|64| +2450815|17635|3|566| +2450815|17636|3|574| +2450815|17638|3|56| +2450815|17641|3|258| +2450815|17642|3|192| +2450815|17644|3|158| +2450815|17647|3|751| +2450815|17648|3|543| +2450815|17650|3|521| +2450815|17653|3|992| +2450815|17654|3|336| +2450815|17656|3|674| +2450815|17659|3|171| +2450815|17660|3|700| +2450815|17662|3|40| +2450815|17665|3|| +2450815|17666|3|| +2450815|17668|3|173| +2450815|17671|3|771| +2450815|17672|3|812| +2450815|17674|3|985| +2450815|17677|3|711| +2450815|17678|3|674| +2450815|17680|3|33| +2450815|17683|3|352| +2450815|17684|3|527| +2450815|17686|3|447| +2450815|17689|3|263| +2450815|17690|3|126| +2450815|17692|3|| +2450815|17695|3|653| +2450815|17696|3|495| +2450815|17698|3|46| +2450815|17701|3|571| +2450815|17702|3|209| +2450815|17704|3|279| +2450815|17707|3|659| +2450815|17708|3|| +2450815|17710|3|642| +2450815|17713|3|746| +2450815|17714|3|| +2450815|17716|3|423| +2450815|17719|3|| +2450815|17720|3|398| +2450815|17722|3|371| +2450815|17725|3|594| +2450815|17726|3|624| +2450815|17728|3|685| +2450815|17731|3|52| +2450815|17732|3|599| +2450815|17734|3|795| +2450815|17737|3|257| +2450815|17738|3|426| +2450815|17740|3|791| +2450815|17743|3|931| +2450815|17744|3|969| +2450815|17746|3|768| +2450815|17749|3|| +2450815|17750|3|225| +2450815|17752|3|907| +2450815|17755|3|312| +2450815|17756|3|272| +2450815|17758|3|947| +2450815|17761|3|617| +2450815|17762|3|920| +2450815|17764|3|110| +2450815|17767|3|949| +2450815|17768|3|735| +2450815|17770|3|765| +2450815|17773|3|570| +2450815|17774|3|360| +2450815|17776|3|684| +2450815|17779|3|1| +2450815|17780|3|119| +2450815|17782|3|515| +2450815|17785|3|820| +2450815|17786|3|74| +2450815|17788|3|9| +2450815|17791|3|| +2450815|17792|3|172| +2450815|17794|3|739| +2450815|17797|3|137| +2450815|17798|3|740| +2450815|17800|3|| +2450815|17803|3|66| +2450815|17804|3|| +2450815|17806|3|92| +2450815|17809|3|459| +2450815|17810|3|631| +2450815|17812|3|99| +2450815|17815|3|57| +2450815|17816|3|768| +2450815|17818|3|838| +2450815|17821|3|479| +2450815|17822|3|729| +2450815|17824|3|179| +2450815|17827|3|114| +2450815|17828|3|818| +2450815|17830|3|174| +2450815|17833|3|128| +2450815|17834|3|482| +2450815|17836|3|807| +2450815|17839|3|786| +2450815|17840|3|429| +2450815|17842|3|803| +2450815|17845|3|248| +2450815|17846|3|892| +2450815|17848|3|773| +2450815|17851|3|968| +2450815|17852|3|88| +2450815|17854|3|8| +2450815|17857|3|900| +2450815|17858|3|466| +2450815|17860|3|152| +2450815|17863|3|136| +2450815|17864|3|835| +2450815|17866|3|401| +2450815|17869|3|910| +2450815|17870|3|32| +2450815|17872|3|937| +2450815|17875|3|411| +2450815|17876|3|989| +2450815|17878|3|204| +2450815|17881|3|667| +2450815|17882|3|624| +2450815|17884|3|743| +2450815|17887|3|709| +2450815|17888|3|833| +2450815|17890|3|377| +2450815|17893|3|998| +2450815|17894|3|705| +2450815|17896|3|| +2450815|17899|3|334| +2450815|17900|3|319| +2450815|17902|3|360| +2450815|17905|3|23| +2450815|17906|3|| +2450815|17908|3|35| +2450815|17911|3|395| +2450815|17912|3|456| +2450815|17914|3|971| +2450815|17917|3|625| +2450815|17918|3|898| +2450815|17920|3|740| +2450815|17923|3|452| +2450815|17924|3|959| +2450815|17926|3|383| +2450815|17929|3|579| +2450815|17930|3|379| +2450815|17932|3|318| +2450815|17935|3|938| +2450815|17936|3|| +2450815|17938|3|176| +2450815|17941|3|553| +2450815|17942|3|416| +2450815|17944|3|811| +2450815|17947|3|911| +2450815|17948|3|884| +2450815|17950|3|84| +2450815|17953|3|259| +2450815|17954|3|257| +2450815|17956|3|152| +2450815|17959|3|| +2450815|17960|3|507| +2450815|17962|3|710| +2450815|17965|3|845| +2450815|17966|3|402| +2450815|17968|3|325| +2450815|17971|3|34| +2450815|17972|3|697| +2450815|17974|3|667| +2450815|17977|3|90| +2450815|17978|3|779| +2450815|17980|3|637| +2450815|17983|3|92| +2450815|17984|3|80| +2450815|17986|3|864| +2450815|17989|3|50| +2450815|17990|3|509| +2450815|17992|3|276| +2450815|17995|3|320| +2450815|17996|3|654| +2450815|17998|3|36| +2450815|1|4|989| +2450815|2|4|85| +2450815|4|4|721| +2450815|7|4|38| +2450815|8|4|63| +2450815|10|4|957| +2450815|13|4|416| +2450815|14|4|96| +2450815|16|4|554| +2450815|19|4|392| +2450815|20|4|| +2450815|22|4|363| +2450815|25|4|926| +2450815|26|4|191| +2450815|28|4|556| +2450815|31|4|940| +2450815|32|4|319| +2450815|34|4|259| +2450815|37|4|321| +2450815|38|4|986| +2450815|40|4|219| +2450815|43|4|323| +2450815|44|4|578| +2450815|46|4|492| +2450815|49|4|1000| +2450815|50|4|647| +2450815|52|4|686| +2450815|55|4|818| +2450815|56|4|314| +2450815|58|4|50| +2450815|61|4|685| +2450815|62|4|420| +2450815|64|4|282| +2450815|67|4|499| +2450815|68|4|67| +2450815|70|4|| +2450815|73|4|650| +2450815|74|4|753| +2450815|76|4|538| +2450815|79|4|498| +2450815|80|4|160| +2450815|82|4|993| +2450815|85|4|807| +2450815|86|4|752| +2450815|88|4|779| +2450815|91|4|233| +2450815|92|4|133| +2450815|94|4|63| +2450815|97|4|59| +2450815|98|4|97| +2450815|100|4|592| +2450815|103|4|538| +2450815|104|4|25| +2450815|106|4|81| +2450815|109|4|149| +2450815|110|4|741| +2450815|112|4|261| +2450815|115|4|138| +2450815|116|4|78| +2450815|118|4|850| +2450815|121|4|330| +2450815|122|4|768| +2450815|124|4|804| +2450815|127|4|369| +2450815|128|4|125| +2450815|130|4|629| +2450815|133|4|869| +2450815|134|4|959| +2450815|136|4|611| +2450815|139|4|191| +2450815|140|4|908| +2450815|142|4|155| +2450815|145|4|529| +2450815|146|4|35| +2450815|148|4|247| +2450815|151|4|849| +2450815|152|4|530| +2450815|154|4|639| +2450815|157|4|869| +2450815|158|4|53| +2450815|160|4|644| +2450815|163|4|307| +2450815|164|4|744| +2450815|166|4|72| +2450815|169|4|78| +2450815|170|4|714| +2450815|172|4|396| +2450815|175|4|50| +2450815|176|4|652| +2450815|178|4|238| +2450815|181|4|193| +2450815|182|4|781| +2450815|184|4|918| +2450815|187|4|54| +2450815|188|4|584| +2450815|190|4|245| +2450815|193|4|140| +2450815|194|4|709| +2450815|196|4|452| +2450815|199|4|886| +2450815|200|4|22| +2450815|202|4|| +2450815|205|4|961| +2450815|206|4|863| +2450815|208|4|276| +2450815|211|4|997| +2450815|212|4|24| +2450815|214|4|61| +2450815|217|4|655| +2450815|218|4|974| +2450815|220|4|20| +2450815|223|4|529| +2450815|224|4|172| +2450815|226|4|742| +2450815|229|4|314| +2450815|230|4|589| +2450815|232|4|725| +2450815|235|4|665| +2450815|236|4|387| +2450815|238|4|41| +2450815|241|4|670| +2450815|242|4|700| +2450815|244|4|525| +2450815|247|4|| +2450815|248|4|302| +2450815|250|4|140| +2450815|253|4|394| +2450815|254|4|219| +2450815|256|4|847| +2450815|259|4|400| +2450815|260|4|713| +2450815|262|4|602| +2450815|265|4|153| +2450815|266|4|366| +2450815|268|4|339| +2450815|271|4|694| +2450815|272|4|629| +2450815|274|4|165| +2450815|277|4|300| +2450815|278|4|698| +2450815|280|4|249| +2450815|283|4|726| +2450815|284|4|179| +2450815|286|4|444| +2450815|289|4|209| +2450815|290|4|| +2450815|292|4|629| +2450815|295|4|441| +2450815|296|4|480| +2450815|298|4|277| +2450815|301|4|253| +2450815|302|4|30| +2450815|304|4|516| +2450815|307|4|52| +2450815|308|4|395| +2450815|310|4|409| +2450815|313|4|128| +2450815|314|4|731| +2450815|316|4|609| +2450815|319|4|994| +2450815|320|4|723| +2450815|322|4|689| +2450815|325|4|576| +2450815|326|4|| +2450815|328|4|595| +2450815|331|4|331| +2450815|332|4|| +2450815|334|4|781| +2450815|337|4|| +2450815|338|4|470| +2450815|340|4|| +2450815|343|4|507| +2450815|344|4|282| +2450815|346|4|567| +2450815|349|4|525| +2450815|350|4|833| +2450815|352|4|953| +2450815|355|4|495| +2450815|356|4|216| +2450815|358|4|327| +2450815|361|4|592| +2450815|362|4|551| +2450815|364|4|534| +2450815|367|4|331| +2450815|368|4|452| +2450815|370|4|38| +2450815|373|4|431| +2450815|374|4|245| +2450815|376|4|8| +2450815|379|4|775| +2450815|380|4|122| +2450815|382|4|24| +2450815|385|4|725| +2450815|386|4|547| +2450815|388|4|871| +2450815|391|4|209| +2450815|392|4|654| +2450815|394|4|764| +2450815|397|4|835| +2450815|398|4|916| +2450815|400|4|111| +2450815|403|4|285| +2450815|404|4|615| +2450815|406|4|738| +2450815|409|4|47| +2450815|410|4|641| +2450815|412|4|183| +2450815|415|4|533| +2450815|416|4|218| +2450815|418|4|984| +2450815|421|4|233| +2450815|422|4|355| +2450815|424|4|39| +2450815|427|4|868| +2450815|428|4|448| +2450815|430|4|781| +2450815|433|4|116| +2450815|434|4|606| +2450815|436|4|853| +2450815|439|4|80| +2450815|440|4|70| +2450815|442|4|935| +2450815|445|4|608| +2450815|446|4|494| +2450815|448|4|545| +2450815|451|4|984| +2450815|452|4|345| +2450815|454|4|730| +2450815|457|4|377| +2450815|458|4|858| +2450815|460|4|956| +2450815|463|4|24| +2450815|464|4|493| +2450815|466|4|174| +2450815|469|4|792| +2450815|470|4|790| +2450815|472|4|441| +2450815|475|4|377| +2450815|476|4|| +2450815|478|4|406| +2450815|481|4|724| +2450815|482|4|234| +2450815|484|4|184| +2450815|487|4|394| +2450815|488|4|476| +2450815|490|4|47| +2450815|493|4|617| +2450815|494|4|289| +2450815|496|4|389| +2450815|499|4|314| +2450815|500|4|673| +2450815|502|4|518| +2450815|505|4|912| +2450815|506|4|448| +2450815|508|4|107| +2450815|511|4|746| +2450815|512|4|551| +2450815|514|4|971| +2450815|517|4|670| +2450815|518|4|154| +2450815|520|4|384| +2450815|523|4|982| +2450815|524|4|964| +2450815|526|4|384| +2450815|529|4|622| +2450815|530|4|645| +2450815|532|4|73| +2450815|535|4|183| +2450815|536|4|510| +2450815|538|4|447| +2450815|541|4|356| +2450815|542|4|581| +2450815|544|4|39| +2450815|547|4|832| +2450815|548|4|222| +2450815|550|4|26| +2450815|553|4|140| +2450815|554|4|35| +2450815|556|4|182| +2450815|559|4|859| +2450815|560|4|124| +2450815|562|4|90| +2450815|565|4|869| +2450815|566|4|483| +2450815|568|4|303| +2450815|571|4|22| +2450815|572|4|| +2450815|574|4|75| +2450815|577|4|139| +2450815|578|4|736| +2450815|580|4|792| +2450815|583|4|168| +2450815|584|4|576| +2450815|586|4|995| +2450815|589|4|695| +2450815|590|4|606| +2450815|592|4|67| +2450815|595|4|522| +2450815|596|4|505| +2450815|598|4|958| +2450815|601|4|332| +2450815|602|4|990| +2450815|604|4|933| +2450815|607|4|948| +2450815|608|4|| +2450815|610|4|464| +2450815|613|4|109| +2450815|614|4|834| +2450815|616|4|779| +2450815|619|4|341| +2450815|620|4|353| +2450815|622|4|590| +2450815|625|4|285| +2450815|626|4|972| +2450815|628|4|940| +2450815|631|4|298| +2450815|632|4|820| +2450815|634|4|431| +2450815|637|4|923| +2450815|638|4|714| +2450815|640|4|105| +2450815|643|4|807| +2450815|644|4|790| +2450815|646|4|514| +2450815|649|4|988| +2450815|650|4|246| +2450815|652|4|551| +2450815|655|4|328| +2450815|656|4|494| +2450815|658|4|56| +2450815|661|4|57| +2450815|662|4|| +2450815|664|4|647| +2450815|667|4|268| +2450815|668|4|229| +2450815|670|4|400| +2450815|673|4|932| +2450815|674|4|504| +2450815|676|4|426| +2450815|679|4|279| +2450815|680|4|415| +2450815|682|4|898| +2450815|685|4|716| +2450815|686|4|788| +2450815|688|4|152| +2450815|691|4|977| +2450815|692|4|406| +2450815|694|4|275| +2450815|697|4|196| +2450815|698|4|116| +2450815|700|4|564| +2450815|703|4|148| +2450815|704|4|890| +2450815|706|4|395| +2450815|709|4|492| +2450815|710|4|172| +2450815|712|4|712| +2450815|715|4|74| +2450815|716|4|672| +2450815|718|4|128| +2450815|721|4|196| +2450815|722|4|661| +2450815|724|4|469| +2450815|727|4|848| +2450815|728|4|598| +2450815|730|4|185| +2450815|733|4|413| +2450815|734|4|529| +2450815|736|4|303| +2450815|739|4|487| +2450815|740|4|638| +2450815|742|4|3| +2450815|745|4|86| +2450815|746|4|547| +2450815|748|4|164| +2450815|751|4|783| +2450815|752|4|| +2450815|754|4|764| +2450815|757|4|491| +2450815|758|4|14| +2450815|760|4|39| +2450815|763|4|38| +2450815|764|4|705| +2450815|766|4|782| +2450815|769|4|301| +2450815|770|4|479| +2450815|772|4|604| +2450815|775|4|29| +2450815|776|4|78| +2450815|778|4|700| +2450815|781|4|809| +2450815|782|4|741| +2450815|784|4|693| +2450815|787|4|912| +2450815|788|4|| +2450815|790|4|436| +2450815|793|4|| +2450815|794|4|863| +2450815|796|4|933| +2450815|799|4|556| +2450815|800|4|606| +2450815|802|4|494| +2450815|805|4|779| +2450815|806|4|747| +2450815|808|4|645| +2450815|811|4|840| +2450815|812|4|536| +2450815|814|4|517| +2450815|817|4|163| +2450815|818|4|119| +2450815|820|4|5| +2450815|823|4|| +2450815|824|4|398| +2450815|826|4|413| +2450815|829|4|| +2450815|830|4|463| +2450815|832|4|235| +2450815|835|4|188| +2450815|836|4|| +2450815|838|4|650| +2450815|841|4|849| +2450815|842|4|305| +2450815|844|4|915| +2450815|847|4|752| +2450815|848|4|275| +2450815|850|4|792| +2450815|853|4|548| +2450815|854|4|149| +2450815|856|4|414| +2450815|859|4|223| +2450815|860|4|66| +2450815|862|4|571| +2450815|865|4|762| +2450815|866|4|829| +2450815|868|4|914| +2450815|871|4|284| +2450815|872|4|340| +2450815|874|4|116| +2450815|877|4|963| +2450815|878|4|543| +2450815|880|4|106| +2450815|883|4|522| +2450815|884|4|144| +2450815|886|4|754| +2450815|889|4|543| +2450815|890|4|930| +2450815|892|4|992| +2450815|895|4|555| +2450815|896|4|284| +2450815|898|4|258| +2450815|901|4|649| +2450815|902|4|14| +2450815|904|4|560| +2450815|907|4|877| +2450815|908|4|491| +2450815|910|4|730| +2450815|913|4|839| +2450815|914|4|537| +2450815|916|4|617| +2450815|919|4|412| +2450815|920|4|933| +2450815|922|4|985| +2450815|925|4|855| +2450815|926|4|234| +2450815|928|4|844| +2450815|931|4|| +2450815|932|4|316| +2450815|934|4|409| +2450815|937|4|997| +2450815|938|4|189| +2450815|940|4|592| +2450815|943|4|495| +2450815|944|4|401| +2450815|946|4|567| +2450815|949|4|298| +2450815|950|4|764| +2450815|952|4|51| +2450815|955|4|132| +2450815|956|4|| +2450815|958|4|272| +2450815|961|4|708| +2450815|962|4|216| +2450815|964|4|126| +2450815|967|4|16| +2450815|968|4|76| +2450815|970|4|210| +2450815|973|4|705| +2450815|974|4|96| +2450815|976|4|711| +2450815|979|4|147| +2450815|980|4|997| +2450815|982|4|460| +2450815|985|4|399| +2450815|986|4|448| +2450815|988|4|485| +2450815|991|4|8| +2450815|992|4|513| +2450815|994|4|875| +2450815|997|4|791| +2450815|998|4|272| +2450815|1000|4|636| +2450815|1003|4|362| +2450815|1004|4|583| +2450815|1006|4|93| +2450815|1009|4|232| +2450815|1010|4|949| +2450815|1012|4|| +2450815|1015|4|838| +2450815|1016|4|178| +2450815|1018|4|818| +2450815|1021|4|82| +2450815|1022|4|145| +2450815|1024|4|261| +2450815|1027|4|567| +2450815|1028|4|548| +2450815|1030|4|746| +2450815|1033|4|854| +2450815|1034|4|796| +2450815|1036|4|| +2450815|1039|4|246| +2450815|1040|4|688| +2450815|1042|4|646| +2450815|1045|4|659| +2450815|1046|4|389| +2450815|1048|4|471| +2450815|1051|4|922| +2450815|1052|4|200| +2450815|1054|4|654| +2450815|1057|4|217| +2450815|1058|4|773| +2450815|1060|4|171| +2450815|1063|4|826| +2450815|1064|4|793| +2450815|1066|4|423| +2450815|1069|4|178| +2450815|1070|4|405| +2450815|1072|4|312| +2450815|1075|4|354| +2450815|1076|4|643| +2450815|1078|4|| +2450815|1081|4|450| +2450815|1082|4|900| +2450815|1084|4|826| +2450815|1087|4|728| +2450815|1088|4|177| +2450815|1090|4|980| +2450815|1093|4|926| +2450815|1094|4|487| +2450815|1096|4|145| +2450815|1099|4|8| +2450815|1100|4|414| +2450815|1102|4|872| +2450815|1105|4|844| +2450815|1106|4|666| +2450815|1108|4|495| +2450815|1111|4|815| +2450815|1112|4|221| +2450815|1114|4|291| +2450815|1117|4|358| +2450815|1118|4|711| +2450815|1120|4|111| +2450815|1123|4|527| +2450815|1124|4|| +2450815|1126|4|761| +2450815|1129|4|345| +2450815|1130|4|152| +2450815|1132|4|401| +2450815|1135|4|446| +2450815|1136|4|56| +2450815|1138|4|432| +2450815|1141|4|594| +2450815|1142|4|578| +2450815|1144|4|432| +2450815|1147|4|968| +2450815|1148|4|881| +2450815|1150|4|522| +2450815|1153|4|348| +2450815|1154|4|199| +2450815|1156|4|903| +2450815|1159|4|228| +2450815|1160|4|625| +2450815|1162|4|292| +2450815|1165|4|177| +2450815|1166|4|87| +2450815|1168|4|659| +2450815|1171|4|463| +2450815|1172|4|340| +2450815|1174|4|379| +2450815|1177|4|701| +2450815|1178|4|197| +2450815|1180|4|656| +2450815|1183|4|724| +2450815|1184|4|163| +2450815|1186|4|552| +2450815|1189|4|43| +2450815|1190|4|104| +2450815|1192|4|759| +2450815|1195|4|897| +2450815|1196|4|154| +2450815|1198|4|574| +2450815|1201|4|530| +2450815|1202|4|417| +2450815|1204|4|257| +2450815|1207|4|343| +2450815|1208|4|472| +2450815|1210|4|112| +2450815|1213|4|984| +2450815|1214|4|535| +2450815|1216|4|182| +2450815|1219|4|663| +2450815|1220|4|924| +2450815|1222|4|627| +2450815|1225|4|274| +2450815|1226|4|165| +2450815|1228|4|938| +2450815|1231|4|314| +2450815|1232|4|989| +2450815|1234|4|62| +2450815|1237|4|472| +2450815|1238|4|457| +2450815|1240|4|379| +2450815|1243|4|559| +2450815|1244|4|| +2450815|1246|4|903| +2450815|1249|4|595| +2450815|1250|4|425| +2450815|1252|4|845| +2450815|1255|4|347| +2450815|1256|4|618| +2450815|1258|4|562| +2450815|1261|4|767| +2450815|1262|4|810| +2450815|1264|4|34| +2450815|1267|4|852| +2450815|1268|4|809| +2450815|1270|4|750| +2450815|1273|4|218| +2450815|1274|4|516| +2450815|1276|4|331| +2450815|1279|4|39| +2450815|1280|4|749| +2450815|1282|4|481| +2450815|1285|4|239| +2450815|1286|4|136| +2450815|1288|4|241| +2450815|1291|4|499| +2450815|1292|4|891| +2450815|1294|4|616| +2450815|1297|4|545| +2450815|1298|4|778| +2450815|1300|4|274| +2450815|1303|4|57| +2450815|1304|4|674| +2450815|1306|4|475| +2450815|1309|4|598| +2450815|1310|4|554| +2450815|1312|4|777| +2450815|1315|4|172| +2450815|1316|4|629| +2450815|1318|4|400| +2450815|1321|4|719| +2450815|1322|4|140| +2450815|1324|4|86| +2450815|1327|4|196| +2450815|1328|4|536| +2450815|1330|4|177| +2450815|1333|4|329| +2450815|1334|4|350| +2450815|1336|4|980| +2450815|1339|4|504| +2450815|1340|4|116| +2450815|1342|4|804| +2450815|1345|4|| +2450815|1346|4|48| +2450815|1348|4|382| +2450815|1351|4|414| +2450815|1352|4|833| +2450815|1354|4|985| +2450815|1357|4|873| +2450815|1358|4|315| +2450815|1360|4|654| +2450815|1363|4|643| +2450815|1364|4|977| +2450815|1366|4|336| +2450815|1369|4|816| +2450815|1370|4|747| +2450815|1372|4|227| +2450815|1375|4|202| +2450815|1376|4|170| +2450815|1378|4|722| +2450815|1381|4|348| +2450815|1382|4|116| +2450815|1384|4|855| +2450815|1387|4|865| +2450815|1388|4|722| +2450815|1390|4|897| +2450815|1393|4|| +2450815|1394|4|697| +2450815|1396|4|354| +2450815|1399|4|862| +2450815|1400|4|226| +2450815|1402|4|755| +2450815|1405|4|915| +2450815|1406|4|189| +2450815|1408|4|477| +2450815|1411|4|219| +2450815|1412|4|31| +2450815|1414|4|777| +2450815|1417|4|129| +2450815|1418|4|993| +2450815|1420|4|206| +2450815|1423|4|814| +2450815|1424|4|999| +2450815|1426|4|225| +2450815|1429|4|544| +2450815|1430|4|| +2450815|1432|4|623| +2450815|1435|4|71| +2450815|1436|4|779| +2450815|1438|4|997| +2450815|1441|4|669| +2450815|1442|4|468| +2450815|1444|4|433| +2450815|1447|4|653| +2450815|1448|4|193| +2450815|1450|4|601| +2450815|1453|4|698| +2450815|1454|4|849| +2450815|1456|4|961| +2450815|1459|4|148| +2450815|1460|4|80| +2450815|1462|4|340| +2450815|1465|4|367| +2450815|1466|4|352| +2450815|1468|4|791| +2450815|1471|4|706| +2450815|1472|4|224| +2450815|1474|4|467| +2450815|1477|4|682| +2450815|1478|4|657| +2450815|1480|4|708| +2450815|1483|4|528| +2450815|1484|4|690| +2450815|1486|4|334| +2450815|1489|4|684| +2450815|1490|4|873| +2450815|1492|4|333| +2450815|1495|4|171| +2450815|1496|4|901| +2450815|1498|4|981| +2450815|1501|4|363| +2450815|1502|4|775| +2450815|1504|4|303| +2450815|1507|4|644| +2450815|1508|4|307| +2450815|1510|4|925| +2450815|1513|4|350| +2450815|1514|4|814| +2450815|1516|4|165| +2450815|1519|4|728| +2450815|1520|4|156| +2450815|1522|4|| +2450815|1525|4|693| +2450815|1526|4|44| +2450815|1528|4|162| +2450815|1531|4|493| +2450815|1532|4|537| +2450815|1534|4|623| +2450815|1537|4|928| +2450815|1538|4|696| +2450815|1540|4|476| +2450815|1543|4|530| +2450815|1544|4|966| +2450815|1546|4|| +2450815|1549|4|379| +2450815|1550|4|501| +2450815|1552|4|827| +2450815|1555|4|454| +2450815|1556|4|996| +2450815|1558|4|18| +2450815|1561|4|| +2450815|1562|4|348| +2450815|1564|4|990| +2450815|1567|4|| +2450815|1568|4|894| +2450815|1570|4|357| +2450815|1573|4|151| +2450815|1574|4|537| +2450815|1576|4|532| +2450815|1579|4|36| +2450815|1580|4|987| +2450815|1582|4|495| +2450815|1585|4|981| +2450815|1586|4|987| +2450815|1588|4|504| +2450815|1591|4|559| +2450815|1592|4|750| +2450815|1594|4|724| +2450815|1597|4|436| +2450815|1598|4|402| +2450815|1600|4|970| +2450815|1603|4|534| +2450815|1604|4|512| +2450815|1606|4|6| +2450815|1609|4|379| +2450815|1610|4|993| +2450815|1612|4|992| +2450815|1615|4|814| +2450815|1616|4|455| +2450815|1618|4|644| +2450815|1621|4|920| +2450815|1622|4|442| +2450815|1624|4|934| +2450815|1627|4|519| +2450815|1628|4|73| +2450815|1630|4|731| +2450815|1633|4|863| +2450815|1634|4|854| +2450815|1636|4|848| +2450815|1639|4|101| +2450815|1640|4|839| +2450815|1642|4|175| +2450815|1645|4|211| +2450815|1646|4|821| +2450815|1648|4|294| +2450815|1651|4|585| +2450815|1652|4|937| +2450815|1654|4|38| +2450815|1657|4|531| +2450815|1658|4|503| +2450815|1660|4|304| +2450815|1663|4|516| +2450815|1664|4|677| +2450815|1666|4|31| +2450815|1669|4|196| +2450815|1670|4|808| +2450815|1672|4|426| +2450815|1675|4|618| +2450815|1676|4|723| +2450815|1678|4|773| +2450815|1681|4|315| +2450815|1682|4|655| +2450815|1684|4|294| +2450815|1687|4|45| +2450815|1688|4|626| +2450815|1690|4|793| +2450815|1693|4|758| +2450815|1694|4|870| +2450815|1696|4|699| +2450815|1699|4|390| +2450815|1700|4|| +2450815|1702|4|447| +2450815|1705|4|271| +2450815|1706|4|754| +2450815|1708|4|904| +2450815|1711|4|92| +2450815|1712|4|355| +2450815|1714|4|722| +2450815|1717|4|526| +2450815|1718|4|673| +2450815|1720|4|164| +2450815|1723|4|312| +2450815|1724|4|770| +2450815|1726|4|795| +2450815|1729|4|811| +2450815|1730|4|735| +2450815|1732|4|721| +2450815|1735|4|414| +2450815|1736|4|515| +2450815|1738|4|412| +2450815|1741|4|415| +2450815|1742|4|380| +2450815|1744|4|969| +2450815|1747|4|322| +2450815|1748|4|76| +2450815|1750|4|509| +2450815|1753|4|810| +2450815|1754|4|150| +2450815|1756|4|418| +2450815|1759|4|480| +2450815|1760|4|393| +2450815|1762|4|630| +2450815|1765|4|738| +2450815|1766|4|602| +2450815|1768|4|| +2450815|1771|4|514| +2450815|1772|4|91| +2450815|1774|4|852| +2450815|1777|4|| +2450815|1778|4|745| +2450815|1780|4|226| +2450815|1783|4|366| +2450815|1784|4|| +2450815|1786|4|139| +2450815|1789|4|58| +2450815|1790|4|887| +2450815|1792|4|340| +2450815|1795|4|177| +2450815|1796|4|139| +2450815|1798|4|624| +2450815|1801|4|81| +2450815|1802|4|560| +2450815|1804|4|469| +2450815|1807|4|735| +2450815|1808|4|144| +2450815|1810|4|42| +2450815|1813|4|| +2450815|1814|4|| +2450815|1816|4|422| +2450815|1819|4|50| +2450815|1820|4|421| +2450815|1822|4|805| +2450815|1825|4|270| +2450815|1826|4|514| +2450815|1828|4|888| +2450815|1831|4|750| +2450815|1832|4|941| +2450815|1834|4|311| +2450815|1837|4|406| +2450815|1838|4|423| +2450815|1840|4|761| +2450815|1843|4|521| +2450815|1844|4|339| +2450815|1846|4|579| +2450815|1849|4|422| +2450815|1850|4|553| +2450815|1852|4|682| +2450815|1855|4|688| +2450815|1856|4|651| +2450815|1858|4|687| +2450815|1861|4|750| +2450815|1862|4|761| +2450815|1864|4|689| +2450815|1867|4|244| +2450815|1868|4|173| +2450815|1870|4|932| +2450815|1873|4|145| +2450815|1874|4|437| +2450815|1876|4|800| +2450815|1879|4|216| +2450815|1880|4|126| +2450815|1882|4|741| +2450815|1885|4|302| +2450815|1886|4|916| +2450815|1888|4|586| +2450815|1891|4|199| +2450815|1892|4|355| +2450815|1894|4|| +2450815|1897|4|208| +2450815|1898|4|106| +2450815|1900|4|516| +2450815|1903|4|583| +2450815|1904|4|782| +2450815|1906|4|689| +2450815|1909|4|659| +2450815|1910|4|135| +2450815|1912|4|863| +2450815|1915|4|904| +2450815|1916|4|283| +2450815|1918|4|180| +2450815|1921|4|859| +2450815|1922|4|837| +2450815|1924|4|384| +2450815|1927|4|| +2450815|1928|4|763| +2450815|1930|4|739| +2450815|1933|4|481| +2450815|1934|4|902| +2450815|1936|4|746| +2450815|1939|4|282| +2450815|1940|4|884| +2450815|1942|4|233| +2450815|1945|4|549| +2450815|1946|4|664| +2450815|1948|4|549| +2450815|1951|4|276| +2450815|1952|4|263| +2450815|1954|4|785| +2450815|1957|4|235| +2450815|1958|4|449| +2450815|1960|4|780| +2450815|1963|4|301| +2450815|1964|4|259| +2450815|1966|4|148| +2450815|1969|4|655| +2450815|1970|4|452| +2450815|1972|4|28| +2450815|1975|4|167| +2450815|1976|4|681| +2450815|1978|4|802| +2450815|1981|4|975| +2450815|1982|4|459| +2450815|1984|4|277| +2450815|1987|4|61| +2450815|1988|4|727| +2450815|1990|4|415| +2450815|1993|4|575| +2450815|1994|4|545| +2450815|1996|4|882| +2450815|1999|4|847| +2450815|2000|4|716| +2450815|2002|4|| +2450815|2005|4|| +2450815|2006|4|982| +2450815|2008|4|871| +2450815|2011|4|642| +2450815|2012|4|214| +2450815|2014|4|725| +2450815|2017|4|855| +2450815|2018|4|317| +2450815|2020|4|69| +2450815|2023|4|156| +2450815|2024|4|681| +2450815|2026|4|| +2450815|2029|4|717| +2450815|2030|4|742| +2450815|2032|4|184| +2450815|2035|4|358| +2450815|2036|4|478| +2450815|2038|4|11| +2450815|2041|4|981| +2450815|2042|4|9| +2450815|2044|4|734| +2450815|2047|4|991| +2450815|2048|4|886| +2450815|2050|4|197| +2450815|2053|4|382| +2450815|2054|4|963| +2450815|2056|4|616| +2450815|2059|4|597| +2450815|2060|4|63| +2450815|2062|4|597| +2450815|2065|4|377| +2450815|2066|4|| +2450815|2068|4|103| +2450815|2071|4|720| +2450815|2072|4|89| +2450815|2074|4|986| +2450815|2077|4|18| +2450815|2078|4|817| +2450815|2080|4|983| +2450815|2083|4|815| +2450815|2084|4|31| +2450815|2086|4|114| +2450815|2089|4|250| +2450815|2090|4|765| +2450815|2092|4|854| +2450815|2095|4|30| +2450815|2096|4|323| +2450815|2098|4|877| +2450815|2101|4|148| +2450815|2102|4|| +2450815|2104|4|380| +2450815|2107|4|344| +2450815|2108|4|647| +2450815|2110|4|323| +2450815|2113|4|836| +2450815|2114|4|275| +2450815|2116|4|661| +2450815|2119|4|951| +2450815|2120|4|1000| +2450815|2122|4|813| +2450815|2125|4|240| +2450815|2126|4|755| +2450815|2128|4|32| +2450815|2131|4|629| +2450815|2132|4|79| +2450815|2134|4|56| +2450815|2137|4|452| +2450815|2138|4|708| +2450815|2140|4|285| +2450815|2143|4|598| +2450815|2144|4|349| +2450815|2146|4|734| +2450815|2149|4|265| +2450815|2150|4|534| +2450815|2152|4|| +2450815|2155|4|274| +2450815|2156|4|349| +2450815|2158|4|423| +2450815|2161|4|665| +2450815|2162|4|431| +2450815|2164|4|322| +2450815|2167|4|999| +2450815|2168|4|835| +2450815|2170|4|888| +2450815|2173|4|468| +2450815|2174|4|951| +2450815|2176|4|98| +2450815|2179|4|715| +2450815|2180|4|211| +2450815|2182|4|730| +2450815|2185|4|48| +2450815|2186|4|818| +2450815|2188|4|320| +2450815|2191|4|933| +2450815|2192|4|871| +2450815|2194|4|159| +2450815|2197|4|386| +2450815|2198|4|934| +2450815|2200|4|673| +2450815|2203|4|116| +2450815|2204|4|721| +2450815|2206|4|461| +2450815|2209|4|169| +2450815|2210|4|664| +2450815|2212|4|817| +2450815|2215|4|110| +2450815|2216|4|71| +2450815|2218|4|371| +2450815|2221|4|594| +2450815|2222|4|195| +2450815|2224|4|692| +2450815|2227|4|791| +2450815|2228|4|641| +2450815|2230|4|667| +2450815|2233|4|286| +2450815|2234|4|797| +2450815|2236|4|| +2450815|2239|4|582| +2450815|2240|4|353| +2450815|2242|4|870| +2450815|2245|4|895| +2450815|2246|4|747| +2450815|2248|4|797| +2450815|2251|4|768| +2450815|2252|4|143| +2450815|2254|4|675| +2450815|2257|4|475| +2450815|2258|4|226| +2450815|2260|4|737| +2450815|2263|4|733| +2450815|2264|4|777| +2450815|2266|4|112| +2450815|2269|4|| +2450815|2270|4|288| +2450815|2272|4|912| +2450815|2275|4|8| +2450815|2276|4|| +2450815|2278|4|377| +2450815|2281|4|484| +2450815|2282|4|268| +2450815|2284|4|504| +2450815|2287|4|789| +2450815|2288|4|750| +2450815|2290|4|944| +2450815|2293|4|680| +2450815|2294|4|733| +2450815|2296|4|437| +2450815|2299|4|424| +2450815|2300|4|834| +2450815|2302|4|737| +2450815|2305|4|914| +2450815|2306|4|177| +2450815|2308|4|983| +2450815|2311|4|153| +2450815|2312|4|412| +2450815|2314|4|544| +2450815|2317|4|419| +2450815|2318|4|394| +2450815|2320|4|983| +2450815|2323|4|661| +2450815|2324|4|428| +2450815|2326|4|869| +2450815|2329|4|724| +2450815|2330|4|264| +2450815|2332|4|233| +2450815|2335|4|947| +2450815|2336|4|792| +2450815|2338|4|233| +2450815|2341|4|780| +2450815|2342|4|931| +2450815|2344|4|| +2450815|2347|4|143| +2450815|2348|4|867| +2450815|2350|4|833| +2450815|2353|4|840| +2450815|2354|4|799| +2450815|2356|4|951| +2450815|2359|4|| +2450815|2360|4|577| +2450815|2362|4|967| +2450815|2365|4|882| +2450815|2366|4|417| +2450815|2368|4|636| +2450815|2371|4|655| +2450815|2372|4|178| +2450815|2374|4|663| +2450815|2377|4|775| +2450815|2378|4|593| +2450815|2380|4|197| +2450815|2383|4|826| +2450815|2384|4|365| +2450815|2386|4|740| +2450815|2389|4|857| +2450815|2390|4|460| +2450815|2392|4|527| +2450815|2395|4|381| +2450815|2396|4|952| +2450815|2398|4|766| +2450815|2401|4|384| +2450815|2402|4|632| +2450815|2404|4|790| +2450815|2407|4|210| +2450815|2408|4|107| +2450815|2410|4|157| +2450815|2413|4|175| +2450815|2414|4|377| +2450815|2416|4|401| +2450815|2419|4|820| +2450815|2420|4|655| +2450815|2422|4|293| +2450815|2425|4|456| +2450815|2426|4|231| +2450815|2428|4|182| +2450815|2431|4|| +2450815|2432|4|560| +2450815|2434|4|837| +2450815|2437|4|247| +2450815|2438|4|636| +2450815|2440|4|15| +2450815|2443|4|583| +2450815|2444|4|882| +2450815|2446|4|13| +2450815|2449|4|816| +2450815|2450|4|851| +2450815|2452|4|615| +2450815|2455|4|306| +2450815|2456|4|318| +2450815|2458|4|495| +2450815|2461|4|150| +2450815|2462|4|625| +2450815|2464|4|380| +2450815|2467|4|809| +2450815|2468|4|423| +2450815|2470|4|744| +2450815|2473|4|607| +2450815|2474|4|67| +2450815|2476|4|569| +2450815|2479|4|| +2450815|2480|4|237| +2450815|2482|4|782| +2450815|2485|4|31| +2450815|2486|4|659| +2450815|2488|4|139| +2450815|2491|4|93| +2450815|2492|4|379| +2450815|2494|4|321| +2450815|2497|4|524| +2450815|2498|4|704| +2450815|2500|4|686| +2450815|2503|4|266| +2450815|2504|4|541| +2450815|2506|4|842| +2450815|2509|4|299| +2450815|2510|4|351| +2450815|2512|4|873| +2450815|2515|4|587| +2450815|2516|4|241| +2450815|2518|4|892| +2450815|2521|4|3| +2450815|2522|4|342| +2450815|2524|4|64| +2450815|2527|4|918| +2450815|2528|4|179| +2450815|2530|4|340| +2450815|2533|4|304| +2450815|2534|4|940| +2450815|2536|4|764| +2450815|2539|4|81| +2450815|2540|4|27| +2450815|2542|4|359| +2450815|2545|4|930| +2450815|2546|4|80| +2450815|2548|4|| +2450815|2551|4|721| +2450815|2552|4|798| +2450815|2554|4|545| +2450815|2557|4|244| +2450815|2558|4|306| +2450815|2560|4|348| +2450815|2563|4|412| +2450815|2564|4|562| +2450815|2566|4|749| +2450815|2569|4|| +2450815|2570|4|711| +2450815|2572|4|753| +2450815|2575|4|708| +2450815|2576|4|257| +2450815|2578|4|403| +2450815|2581|4|195| +2450815|2582|4|721| +2450815|2584|4|183| +2450815|2587|4|352| +2450815|2588|4|49| +2450815|2590|4|882| +2450815|2593|4|765| +2450815|2594|4|381| +2450815|2596|4|723| +2450815|2599|4|597| +2450815|2600|4|261| +2450815|2602|4|417| +2450815|2605|4|56| +2450815|2606|4|377| +2450815|2608|4|2| +2450815|2611|4|567| +2450815|2612|4|894| +2450815|2614|4|338| +2450815|2617|4|56| +2450815|2618|4|995| +2450815|2620|4|755| +2450815|2623|4|86| +2450815|2624|4|| +2450815|2626|4|191| +2450815|2629|4|906| +2450815|2630|4|369| +2450815|2632|4|956| +2450815|2635|4|749| +2450815|2636|4|110| +2450815|2638|4|679| +2450815|2641|4|752| +2450815|2642|4|138| +2450815|2644|4|552| +2450815|2647|4|13| +2450815|2648|4|719| +2450815|2650|4|695| +2450815|2653|4|691| +2450815|2654|4|24| +2450815|2656|4|125| +2450815|2659|4|671| +2450815|2660|4|295| +2450815|2662|4|330| +2450815|2665|4|361| +2450815|2666|4|541| +2450815|2668|4|150| +2450815|2671|4|342| +2450815|2672|4|236| +2450815|2674|4|47| +2450815|2677|4|838| +2450815|2678|4|952| +2450815|2680|4|471| +2450815|2683|4|80| +2450815|2684|4|507| +2450815|2686|4|| +2450815|2689|4|496| +2450815|2690|4|980| +2450815|2692|4|172| +2450815|2695|4|680| +2450815|2696|4|979| +2450815|2698|4|311| +2450815|2701|4|599| +2450815|2702|4|650| +2450815|2704|4|658| +2450815|2707|4|103| +2450815|2708|4|81| +2450815|2710|4|431| +2450815|2713|4|421| +2450815|2714|4|| +2450815|2716|4|752| +2450815|2719|4|933| +2450815|2720|4|168| +2450815|2722|4|240| +2450815|2725|4|929| +2450815|2726|4|81| +2450815|2728|4|779| +2450815|2731|4|68| +2450815|2732|4|665| +2450815|2734|4|163| +2450815|2737|4|530| +2450815|2738|4|122| +2450815|2740|4|133| +2450815|2743|4|293| +2450815|2744|4|288| +2450815|2746|4|103| +2450815|2749|4|957| +2450815|2750|4|480| +2450815|2752|4|892| +2450815|2755|4|571| +2450815|2756|4|502| +2450815|2758|4|543| +2450815|2761|4|31| +2450815|2762|4|338| +2450815|2764|4|| +2450815|2767|4|| +2450815|2768|4|466| +2450815|2770|4|799| +2450815|2773|4|10| +2450815|2774|4|660| +2450815|2776|4|507| +2450815|2779|4|| +2450815|2780|4|280| +2450815|2782|4|397| +2450815|2785|4|180| +2450815|2786|4|935| +2450815|2788|4|| +2450815|2791|4|770| +2450815|2792|4|250| +2450815|2794|4|867| +2450815|2797|4|937| +2450815|2798|4|100| +2450815|2800|4|2| +2450815|2803|4|117| +2450815|2804|4|534| +2450815|2806|4|475| +2450815|2809|4|77| +2450815|2810|4|250| +2450815|2812|4|944| +2450815|2815|4|372| +2450815|2816|4|869| +2450815|2818|4|651| +2450815|2821|4|762| +2450815|2822|4|| +2450815|2824|4|356| +2450815|2827|4|392| +2450815|2828|4|536| +2450815|2830|4|573| +2450815|2833|4|699| +2450815|2834|4|989| +2450815|2836|4|910| +2450815|2839|4|283| +2450815|2840|4|303| +2450815|2842|4|365| +2450815|2845|4|863| +2450815|2846|4|242| +2450815|2848|4|279| +2450815|2851|4|1000| +2450815|2852|4|401| +2450815|2854|4|567| +2450815|2857|4|646| +2450815|2858|4|453| +2450815|2860|4|314| +2450815|2863|4|| +2450815|2864|4|699| +2450815|2866|4|883| +2450815|2869|4|223| +2450815|2870|4|956| +2450815|2872|4|446| +2450815|2875|4|59| +2450815|2876|4|935| +2450815|2878|4|638| +2450815|2881|4|847| +2450815|2882|4|192| +2450815|2884|4|46| +2450815|2887|4|135| +2450815|2888|4|713| +2450815|2890|4|707| +2450815|2893|4|194| +2450815|2894|4|866| +2450815|2896|4|449| +2450815|2899|4|213| +2450815|2900|4|252| +2450815|2902|4|336| +2450815|2905|4|190| +2450815|2906|4|531| +2450815|2908|4|883| +2450815|2911|4|67| +2450815|2912|4|192| +2450815|2914|4|631| +2450815|2917|4|983| +2450815|2918|4|589| +2450815|2920|4|39| +2450815|2923|4|399| +2450815|2924|4|994| +2450815|2926|4|539| +2450815|2929|4|148| +2450815|2930|4|474| +2450815|2932|4|334| +2450815|2935|4|728| +2450815|2936|4|93| +2450815|2938|4|431| +2450815|2941|4|394| +2450815|2942|4|120| +2450815|2944|4|791| +2450815|2947|4|55| +2450815|2948|4|291| +2450815|2950|4|902| +2450815|2953|4|190| +2450815|2954|4|413| +2450815|2956|4|51| +2450815|2959|4|569| +2450815|2960|4|848| +2450815|2962|4|26| +2450815|2965|4|| +2450815|2966|4|560| +2450815|2968|4|542| +2450815|2971|4|339| +2450815|2972|4|12| +2450815|2974|4|428| +2450815|2977|4|381| +2450815|2978|4|861| +2450815|2980|4|171| +2450815|2983|4|101| +2450815|2984|4|557| +2450815|2986|4|653| +2450815|2989|4|399| +2450815|2990|4|995| +2450815|2992|4|279| +2450815|2995|4|850| +2450815|2996|4|648| +2450815|2998|4|186| +2450815|3001|4|660| +2450815|3002|4|423| +2450815|3004|4|741| +2450815|3007|4|706| +2450815|3008|4|407| +2450815|3010|4|948| +2450815|3013|4|437| +2450815|3014|4|830| +2450815|3016|4|60| +2450815|3019|4|585| +2450815|3020|4|781| +2450815|3022|4|385| +2450815|3025|4|926| +2450815|3026|4|140| +2450815|3028|4|21| +2450815|3031|4|863| +2450815|3032|4|105| +2450815|3034|4|154| +2450815|3037|4|116| +2450815|3038|4|681| +2450815|3040|4|692| +2450815|3043|4|399| +2450815|3044|4|248| +2450815|3046|4|224| +2450815|3049|4|661| +2450815|3050|4|806| +2450815|3052|4|399| +2450815|3055|4|127| +2450815|3056|4|975| +2450815|3058|4|564| +2450815|3061|4|164| +2450815|3062|4|101| +2450815|3064|4|96| +2450815|3067|4|904| +2450815|3068|4|297| +2450815|3070|4|69| +2450815|3073|4|292| +2450815|3074|4|564| +2450815|3076|4|631| +2450815|3079|4|564| +2450815|3080|4|254| +2450815|3082|4|454| +2450815|3085|4|780| +2450815|3086|4|282| +2450815|3088|4|149| +2450815|3091|4|381| +2450815|3092|4|25| +2450815|3094|4|900| +2450815|3097|4|538| +2450815|3098|4|499| +2450815|3100|4|647| +2450815|3103|4|30| +2450815|3104|4|157| +2450815|3106|4|634| +2450815|3109|4|220| +2450815|3110|4|522| +2450815|3112|4|236| +2450815|3115|4|352| +2450815|3116|4|462| +2450815|3118|4|| +2450815|3121|4|639| +2450815|3122|4|62| +2450815|3124|4|433| +2450815|3127|4|849| +2450815|3128|4|10| +2450815|3130|4|183| +2450815|3133|4|742| +2450815|3134|4|944| +2450815|3136|4|529| +2450815|3139|4|967| +2450815|3140|4|441| +2450815|3142|4|901| +2450815|3145|4|872| +2450815|3146|4|910| +2450815|3148|4|840| +2450815|3151|4|729| +2450815|3152|4|| +2450815|3154|4|915| +2450815|3157|4|| +2450815|3158|4|852| +2450815|3160|4|265| +2450815|3163|4|54| +2450815|3164|4|772| +2450815|3166|4|746| +2450815|3169|4|| +2450815|3170|4|770| +2450815|3172|4|635| +2450815|3175|4|37| +2450815|3176|4|823| +2450815|3178|4|| +2450815|3181|4|321| +2450815|3182|4|426| +2450815|3184|4|118| +2450815|3187|4|105| +2450815|3188|4|750| +2450815|3190|4|470| +2450815|3193|4|72| +2450815|3194|4|663| +2450815|3196|4|139| +2450815|3199|4|362| +2450815|3200|4|614| +2450815|3202|4|668| +2450815|3205|4|18| +2450815|3206|4|956| +2450815|3208|4|313| +2450815|3211|4|700| +2450815|3212|4|879| +2450815|3214|4|549| +2450815|3217|4|56| +2450815|3218|4|396| +2450815|3220|4|714| +2450815|3223|4|479| +2450815|3224|4|| +2450815|3226|4|389| +2450815|3229|4|786| +2450815|3230|4|615| +2450815|3232|4|530| +2450815|3235|4|1000| +2450815|3236|4|318| +2450815|3238|4|779| +2450815|3241|4|127| +2450815|3242|4|963| +2450815|3244|4|938| +2450815|3247|4|247| +2450815|3248|4|226| +2450815|3250|4|504| +2450815|3253|4|14| +2450815|3254|4|832| +2450815|3256|4|611| +2450815|3259|4|287| +2450815|3260|4|398| +2450815|3262|4|685| +2450815|3265|4|969| +2450815|3266|4|417| +2450815|3268|4|327| +2450815|3271|4|389| +2450815|3272|4|49| +2450815|3274|4|618| +2450815|3277|4|438| +2450815|3278|4|14| +2450815|3280|4|946| +2450815|3283|4|473| +2450815|3284|4|216| +2450815|3286|4|880| +2450815|3289|4|552| +2450815|3290|4|985| +2450815|3292|4|660| +2450815|3295|4|209| +2450815|3296|4|103| +2450815|3298|4|| +2450815|3301|4|168| +2450815|3302|4|221| +2450815|3304|4|793| +2450815|3307|4|575| +2450815|3308|4|70| +2450815|3310|4|699| +2450815|3313|4|399| +2450815|3314|4|360| +2450815|3316|4|911| +2450815|3319|4|735| +2450815|3320|4|31| +2450815|3322|4|803| +2450815|3325|4|680| +2450815|3326|4|337| +2450815|3328|4|104| +2450815|3331|4|394| +2450815|3332|4|46| +2450815|3334|4|247| +2450815|3337|4|516| +2450815|3338|4|143| +2450815|3340|4|406| +2450815|3343|4|526| +2450815|3344|4|640| +2450815|3346|4|433| +2450815|3349|4|226| +2450815|3350|4|793| +2450815|3352|4|257| +2450815|3355|4|340| +2450815|3356|4|654| +2450815|3358|4|679| +2450815|3361|4|517| +2450815|3362|4|649| +2450815|3364|4|866| +2450815|3367|4|| +2450815|3368|4|642| +2450815|3370|4|948| +2450815|3373|4|820| +2450815|3374|4|631| +2450815|3376|4|256| +2450815|3379|4|79| +2450815|3380|4|948| +2450815|3382|4|659| +2450815|3385|4|279| +2450815|3386|4|415| +2450815|3388|4|827| +2450815|3391|4|754| +2450815|3392|4|418| +2450815|3394|4|32| +2450815|3397|4|| +2450815|3398|4|357| +2450815|3400|4|477| +2450815|3403|4|572| +2450815|3404|4|443| +2450815|3406|4|819| +2450815|3409|4|157| +2450815|3410|4|699| +2450815|3412|4|131| +2450815|3415|4|520| +2450815|3416|4|675| +2450815|3418|4|80| +2450815|3421|4|290| +2450815|3422|4|383| +2450815|3424|4|336| +2450815|3427|4|235| +2450815|3428|4|857| +2450815|3430|4|124| +2450815|3433|4|201| +2450815|3434|4|752| +2450815|3436|4|632| +2450815|3439|4|520| +2450815|3440|4|301| +2450815|3442|4|425| +2450815|3445|4|606| +2450815|3446|4|487| +2450815|3448|4|897| +2450815|3451|4|993| +2450815|3452|4|363| +2450815|3454|4|959| +2450815|3457|4|153| +2450815|3458|4|22| +2450815|3460|4|841| +2450815|3463|4|445| +2450815|3464|4|782| +2450815|3466|4|897| +2450815|3469|4|355| +2450815|3470|4|393| +2450815|3472|4|481| +2450815|3475|4|| +2450815|3476|4|738| +2450815|3478|4|76| +2450815|3481|4|880| +2450815|3482|4|169| +2450815|3484|4|| +2450815|3487|4|601| +2450815|3488|4|441| +2450815|3490|4|491| +2450815|3493|4|415| +2450815|3494|4|678| +2450815|3496|4|967| +2450815|3499|4|665| +2450815|3500|4|27| +2450815|3502|4|421| +2450815|3505|4|848| +2450815|3506|4|491| +2450815|3508|4|480| +2450815|3511|4|484| +2450815|3512|4|676| +2450815|3514|4|465| +2450815|3517|4|628| +2450815|3518|4|80| +2450815|3520|4|411| +2450815|3523|4|348| +2450815|3524|4|813| +2450815|3526|4|501| +2450815|3529|4|87| +2450815|3530|4|5| +2450815|3532|4|122| +2450815|3535|4|813| +2450815|3536|4|910| +2450815|3538|4|787| +2450815|3541|4|208| +2450815|3542|4|820| +2450815|3544|4|389| +2450815|3547|4|45| +2450815|3548|4|862| +2450815|3550|4|137| +2450815|3553|4|643| +2450815|3554|4|259| +2450815|3556|4|395| +2450815|3559|4|194| +2450815|3560|4|276| +2450815|3562|4|388| +2450815|3565|4|490| +2450815|3566|4|953| +2450815|3568|4|890| +2450815|3571|4|149| +2450815|3572|4|774| +2450815|3574|4|364| +2450815|3577|4|660| +2450815|3578|4|119| +2450815|3580|4|432| +2450815|3583|4|428| +2450815|3584|4|713| +2450815|3586|4|590| +2450815|3589|4|520| +2450815|3590|4|84| +2450815|3592|4|621| +2450815|3595|4|537| +2450815|3596|4|754| +2450815|3598|4|| +2450815|3601|4|| +2450815|3602|4|| +2450815|3604|4|724| +2450815|3607|4|815| +2450815|3608|4|702| +2450815|3610|4|257| +2450815|3613|4|247| +2450815|3614|4|330| +2450815|3616|4|71| +2450815|3619|4|| +2450815|3620|4|980| +2450815|3622|4|76| +2450815|3625|4|981| +2450815|3626|4|531| +2450815|3628|4|| +2450815|3631|4|306| +2450815|3632|4|43| +2450815|3634|4|364| +2450815|3637|4|707| +2450815|3638|4|181| +2450815|3640|4|2| +2450815|3643|4|141| +2450815|3644|4|893| +2450815|3646|4|398| +2450815|3649|4|| +2450815|3650|4|78| +2450815|3652|4|194| +2450815|3655|4|818| +2450815|3656|4|683| +2450815|3658|4|347| +2450815|3661|4|136| +2450815|3662|4|816| +2450815|3664|4|783| +2450815|3667|4|32| +2450815|3668|4|1000| +2450815|3670|4|667| +2450815|3673|4|281| +2450815|3674|4|506| +2450815|3676|4|908| +2450815|3679|4|436| +2450815|3680|4|596| +2450815|3682|4|197| +2450815|3685|4|750| +2450815|3686|4|578| +2450815|3688|4|207| +2450815|3691|4|834| +2450815|3692|4|267| +2450815|3694|4|501| +2450815|3697|4|290| +2450815|3698|4|134| +2450815|3700|4|443| +2450815|3703|4|| +2450815|3704|4|435| +2450815|3706|4|841| +2450815|3709|4|615| +2450815|3710|4|654| +2450815|3712|4|464| +2450815|3715|4|979| +2450815|3716|4|382| +2450815|3718|4|79| +2450815|3721|4|347| +2450815|3722|4|41| +2450815|3724|4|358| +2450815|3727|4|| +2450815|3728|4|887| +2450815|3730|4|455| +2450815|3733|4|198| +2450815|3734|4|729| +2450815|3736|4|513| +2450815|3739|4|231| +2450815|3740|4|820| +2450815|3742|4|| +2450815|3745|4|| +2450815|3746|4|850| +2450815|3748|4|125| +2450815|3751|4|| +2450815|3752|4|| +2450815|3754|4|905| +2450815|3757|4|407| +2450815|3758|4|898| +2450815|3760|4|821| +2450815|3763|4|69| +2450815|3764|4|| +2450815|3766|4|177| +2450815|3769|4|420| +2450815|3770|4|611| +2450815|3772|4|435| +2450815|3775|4|921| +2450815|3776|4|439| +2450815|3778|4|901| +2450815|3781|4|187| +2450815|3782|4|808| +2450815|3784|4|954| +2450815|3787|4|367| +2450815|3788|4|144| +2450815|3790|4|54| +2450815|3793|4|694| +2450815|3794|4|269| +2450815|3796|4|97| +2450815|3799|4|953| +2450815|3800|4|719| +2450815|3802|4|32| +2450815|3805|4|805| +2450815|3806|4|250| +2450815|3808|4|332| +2450815|3811|4|504| +2450815|3812|4|309| +2450815|3814|4|296| +2450815|3817|4|165| +2450815|3818|4|502| +2450815|3820|4|744| +2450815|3823|4|499| +2450815|3824|4|781| +2450815|3826|4|718| +2450815|3829|4|265| +2450815|3830|4|860| +2450815|3832|4|582| +2450815|3835|4|975| +2450815|3836|4|237| +2450815|3838|4|527| +2450815|3841|4|569| +2450815|3842|4|789| +2450815|3844|4|815| +2450815|3847|4|| +2450815|3848|4|144| +2450815|3850|4|79| +2450815|3853|4|495| +2450815|3854|4|652| +2450815|3856|4|781| +2450815|3859|4|155| +2450815|3860|4|454| +2450815|3862|4|58| +2450815|3865|4|| +2450815|3866|4|245| +2450815|3868|4|| +2450815|3871|4|619| +2450815|3872|4|345| +2450815|3874|4|648| +2450815|3877|4|693| +2450815|3878|4|375| +2450815|3880|4|521| +2450815|3883|4|195| +2450815|3884|4|782| +2450815|3886|4|608| +2450815|3889|4|998| +2450815|3890|4|515| +2450815|3892|4|387| +2450815|3895|4|825| +2450815|3896|4|848| +2450815|3898|4|364| +2450815|3901|4|102| +2450815|3902|4|777| +2450815|3904|4|569| +2450815|3907|4|173| +2450815|3908|4|579| +2450815|3910|4|333| +2450815|3913|4|883| +2450815|3914|4|221| +2450815|3916|4|716| +2450815|3919|4|787| +2450815|3920|4|16| +2450815|3922|4|535| +2450815|3925|4|634| +2450815|3926|4|551| +2450815|3928|4|416| +2450815|3931|4|680| +2450815|3932|4|87| +2450815|3934|4|393| +2450815|3937|4|24| +2450815|3938|4|509| +2450815|3940|4|303| +2450815|3943|4|597| +2450815|3944|4|385| +2450815|3946|4|382| +2450815|3949|4|178| +2450815|3950|4|604| +2450815|3952|4|406| +2450815|3955|4|618| +2450815|3956|4|541| +2450815|3958|4|934| +2450815|3961|4|612| +2450815|3962|4|697| +2450815|3964|4|821| +2450815|3967|4|990| +2450815|3968|4|72| +2450815|3970|4|994| +2450815|3973|4|350| +2450815|3974|4|560| +2450815|3976|4|89| +2450815|3979|4|751| +2450815|3980|4|675| +2450815|3982|4|277| +2450815|3985|4|34| +2450815|3986|4|399| +2450815|3988|4|315| +2450815|3991|4|530| +2450815|3992|4|156| +2450815|3994|4|412| +2450815|3997|4|213| +2450815|3998|4|787| +2450815|4000|4|841| +2450815|4003|4|290| +2450815|4004|4|723| +2450815|4006|4|724| +2450815|4009|4|465| +2450815|4010|4|184| +2450815|4012|4|622| +2450815|4015|4|268| +2450815|4016|4|220| +2450815|4018|4|412| +2450815|4021|4|86| +2450815|4022|4|1| +2450815|4024|4|| +2450815|4027|4|140| +2450815|4028|4|176| +2450815|4030|4|789| +2450815|4033|4|365| +2450815|4034|4|956| +2450815|4036|4|702| +2450815|4039|4|90| +2450815|4040|4|349| +2450815|4042|4|924| +2450815|4045|4|723| +2450815|4046|4|310| +2450815|4048|4|113| +2450815|4051|4|764| +2450815|4052|4|960| +2450815|4054|4|918| +2450815|4057|4|568| +2450815|4058|4|161| +2450815|4060|4|973| +2450815|4063|4|774| +2450815|4064|4|349| +2450815|4066|4|592| +2450815|4069|4|641| +2450815|4070|4|800| +2450815|4072|4|1| +2450815|4075|4|104| +2450815|4076|4|171| +2450815|4078|4|418| +2450815|4081|4|715| +2450815|4082|4|451| +2450815|4084|4|900| +2450815|4087|4|710| +2450815|4088|4|400| +2450815|4090|4|758| +2450815|4093|4|782| +2450815|4094|4|690| +2450815|4096|4|500| +2450815|4099|4|328| +2450815|4100|4|277| +2450815|4102|4|972| +2450815|4105|4|187| +2450815|4106|4|414| +2450815|4108|4|395| +2450815|4111|4|231| +2450815|4112|4|899| +2450815|4114|4|846| +2450815|4117|4|815| +2450815|4118|4|367| +2450815|4120|4|187| +2450815|4123|4|665| +2450815|4124|4|42| +2450815|4126|4|827| +2450815|4129|4|| +2450815|4130|4|219| +2450815|4132|4|747| +2450815|4135|4|876| +2450815|4136|4|649| +2450815|4138|4|214| +2450815|4141|4|106| +2450815|4142|4|196| +2450815|4144|4|309| +2450815|4147|4|31| +2450815|4148|4|784| +2450815|4150|4|157| +2450815|4153|4|761| +2450815|4154|4|179| +2450815|4156|4|574| +2450815|4159|4|799| +2450815|4160|4|223| +2450815|4162|4|557| +2450815|4165|4|969| +2450815|4166|4|356| +2450815|4168|4|950| +2450815|4171|4|315| +2450815|4172|4|194| +2450815|4174|4|399| +2450815|4177|4|824| +2450815|4178|4|741| +2450815|4180|4|658| +2450815|4183|4|235| +2450815|4184|4|786| +2450815|4186|4|301| +2450815|4189|4|1| +2450815|4190|4|709| +2450815|4192|4|151| +2450815|4195|4|232| +2450815|4196|4|18| +2450815|4198|4|267| +2450815|4201|4|485| +2450815|4202|4|218| +2450815|4204|4|407| +2450815|4207|4|966| +2450815|4208|4|527| +2450815|4210|4|| +2450815|4213|4|347| +2450815|4214|4|361| +2450815|4216|4|809| +2450815|4219|4|48| +2450815|4220|4|45| +2450815|4222|4|214| +2450815|4225|4|370| +2450815|4226|4|585| +2450815|4228|4|483| +2450815|4231|4|97| +2450815|4232|4|388| +2450815|4234|4|517| +2450815|4237|4|862| +2450815|4238|4|647| +2450815|4240|4|871| +2450815|4243|4|626| +2450815|4244|4|275| +2450815|4246|4|634| +2450815|4249|4|426| +2450815|4250|4|262| +2450815|4252|4|676| +2450815|4255|4|183| +2450815|4256|4|888| +2450815|4258|4|78| +2450815|4261|4|17| +2450815|4262|4|965| +2450815|4264|4|599| +2450815|4267|4|540| +2450815|4268|4|742| +2450815|4270|4|983| +2450815|4273|4|569| +2450815|4274|4|756| +2450815|4276|4|314| +2450815|4279|4|26| +2450815|4280|4|267| +2450815|4282|4|241| +2450815|4285|4|900| +2450815|4286|4|229| +2450815|4288|4|386| +2450815|4291|4|748| +2450815|4292|4|| +2450815|4294|4|251| +2450815|4297|4|249| +2450815|4298|4|899| +2450815|4300|4|| +2450815|4303|4|971| +2450815|4304|4|309| +2450815|4306|4|996| +2450815|4309|4|403| +2450815|4310|4|252| +2450815|4312|4|439| +2450815|4315|4|347| +2450815|4316|4|681| +2450815|4318|4|876| +2450815|4321|4|527| +2450815|4322|4|435| +2450815|4324|4|857| +2450815|4327|4|992| +2450815|4328|4|158| +2450815|4330|4|71| +2450815|4333|4|504| +2450815|4334|4|766| +2450815|4336|4|693| +2450815|4339|4|515| +2450815|4340|4|619| +2450815|4342|4|478| +2450815|4345|4|701| +2450815|4346|4|465| +2450815|4348|4|50| +2450815|4351|4|788| +2450815|4352|4|369| +2450815|4354|4|95| +2450815|4357|4|903| +2450815|4358|4|385| +2450815|4360|4|508| +2450815|4363|4|717| +2450815|4364|4|190| +2450815|4366|4|687| +2450815|4369|4|403| +2450815|4370|4|467| +2450815|4372|4|| +2450815|4375|4|521| +2450815|4376|4|91| +2450815|4378|4|417| +2450815|4381|4|489| +2450815|4382|4|266| +2450815|4384|4|406| +2450815|4387|4|913| +2450815|4388|4|177| +2450815|4390|4|469| +2450815|4393|4|340| +2450815|4394|4|124| +2450815|4396|4|989| +2450815|4399|4|398| +2450815|4400|4|384| +2450815|4402|4|293| +2450815|4405|4|935| +2450815|4406|4|579| +2450815|4408|4|314| +2450815|4411|4|666| +2450815|4412|4|102| +2450815|4414|4|700| +2450815|4417|4|184| +2450815|4418|4|923| +2450815|4420|4|680| +2450815|4423|4|901| +2450815|4424|4|96| +2450815|4426|4|594| +2450815|4429|4|617| +2450815|4430|4|130| +2450815|4432|4|85| +2450815|4435|4|218| +2450815|4436|4|409| +2450815|4438|4|249| +2450815|4441|4|132| +2450815|4442|4|535| +2450815|4444|4|396| +2450815|4447|4|964| +2450815|4448|4|144| +2450815|4450|4|234| +2450815|4453|4|352| +2450815|4454|4|697| +2450815|4456|4|978| +2450815|4459|4|878| +2450815|4460|4|864| +2450815|4462|4|913| +2450815|4465|4|934| +2450815|4466|4|781| +2450815|4468|4|756| +2450815|4471|4|980| +2450815|4472|4|216| +2450815|4474|4|476| +2450815|4477|4|67| +2450815|4478|4|| +2450815|4480|4|614| +2450815|4483|4|413| +2450815|4484|4|987| +2450815|4486|4|691| +2450815|4489|4|803| +2450815|4490|4|783| +2450815|4492|4|801| +2450815|4495|4|358| +2450815|4496|4|498| +2450815|4498|4|236| +2450815|4501|4|713| +2450815|4502|4|| +2450815|4504|4|935| +2450815|4507|4|377| +2450815|4508|4|411| +2450815|4510|4|836| +2450815|4513|4|644| +2450815|4514|4|183| +2450815|4516|4|506| +2450815|4519|4|888| +2450815|4520|4|438| +2450815|4522|4|217| +2450815|4525|4|814| +2450815|4526|4|814| +2450815|4528|4|| +2450815|4531|4|772| +2450815|4532|4|582| +2450815|4534|4|314| +2450815|4537|4|741| +2450815|4538|4|888| +2450815|4540|4|950| +2450815|4543|4|299| +2450815|4544|4|270| +2450815|4546|4|997| +2450815|4549|4|195| +2450815|4550|4|| +2450815|4552|4|| +2450815|4555|4|669| +2450815|4556|4|525| +2450815|4558|4|729| +2450815|4561|4|251| +2450815|4562|4|328| +2450815|4564|4|852| +2450815|4567|4|889| +2450815|4568|4|980| +2450815|4570|4|419| +2450815|4573|4|477| +2450815|4574|4|| +2450815|4576|4|749| +2450815|4579|4|15| +2450815|4580|4|712| +2450815|4582|4|145| +2450815|4585|4|541| +2450815|4586|4|765| +2450815|4588|4|36| +2450815|4591|4|122| +2450815|4592|4|42| +2450815|4594|4|164| +2450815|4597|4|665| +2450815|4598|4|570| +2450815|4600|4|361| +2450815|4603|4|| +2450815|4604|4|70| +2450815|4606|4|409| +2450815|4609|4|801| +2450815|4610|4|139| +2450815|4612|4|408| +2450815|4615|4|815| +2450815|4616|4|143| +2450815|4618|4|956| +2450815|4621|4|838| +2450815|4622|4|292| +2450815|4624|4|639| +2450815|4627|4|888| +2450815|4628|4|128| +2450815|4630|4|413| +2450815|4633|4|111| +2450815|4634|4|597| +2450815|4636|4|950| +2450815|4639|4|711| +2450815|4640|4|452| +2450815|4642|4|| +2450815|4645|4|409| +2450815|4646|4|100| +2450815|4648|4|295| +2450815|4651|4|372| +2450815|4652|4|806| +2450815|4654|4|556| +2450815|4657|4|457| +2450815|4658|4|242| +2450815|4660|4|965| +2450815|4663|4|962| +2450815|4664|4|418| +2450815|4666|4|972| +2450815|4669|4|768| +2450815|4670|4|727| +2450815|4672|4|419| +2450815|4675|4|428| +2450815|4676|4|669| +2450815|4678|4|176| +2450815|4681|4|970| +2450815|4682|4|169| +2450815|4684|4|635| +2450815|4687|4|168| +2450815|4688|4|766| +2450815|4690|4|634| +2450815|4693|4|555| +2450815|4694|4|469| +2450815|4696|4|125| +2450815|4699|4|14| +2450815|4700|4|697| +2450815|4702|4|756| +2450815|4705|4|977| +2450815|4706|4|373| +2450815|4708|4|720| +2450815|4711|4|377| +2450815|4712|4|662| +2450815|4714|4|196| +2450815|4717|4|319| +2450815|4718|4|422| +2450815|4720|4|866| +2450815|4723|4|36| +2450815|4724|4|| +2450815|4726|4|129| +2450815|4729|4|719| +2450815|4730|4|310| +2450815|4732|4|461| +2450815|4735|4|286| +2450815|4736|4|545| +2450815|4738|4|897| +2450815|4741|4|412| +2450815|4742|4|812| +2450815|4744|4|800| +2450815|4747|4|| +2450815|4748|4|895| +2450815|4750|4|975| +2450815|4753|4|344| +2450815|4754|4|480| +2450815|4756|4|637| +2450815|4759|4|351| +2450815|4760|4|302| +2450815|4762|4|591| +2450815|4765|4|371| +2450815|4766|4|931| +2450815|4768|4|807| +2450815|4771|4|1000| +2450815|4772|4|61| +2450815|4774|4|912| +2450815|4777|4|844| +2450815|4778|4|4| +2450815|4780|4|542| +2450815|4783|4|311| +2450815|4784|4|772| +2450815|4786|4|529| +2450815|4789|4|262| +2450815|4790|4|798| +2450815|4792|4|704| +2450815|4795|4|671| +2450815|4796|4|255| +2450815|4798|4|805| +2450815|4801|4|678| +2450815|4802|4|44| +2450815|4804|4|396| +2450815|4807|4|703| +2450815|4808|4|398| +2450815|4810|4|78| +2450815|4813|4|489| +2450815|4814|4|722| +2450815|4816|4|83| +2450815|4819|4|64| +2450815|4820|4|917| +2450815|4822|4|941| +2450815|4825|4|53| +2450815|4826|4|661| +2450815|4828|4|717| +2450815|4831|4|24| +2450815|4832|4|192| +2450815|4834|4|623| +2450815|4837|4|701| +2450815|4838|4|564| +2450815|4840|4|964| +2450815|4843|4|713| +2450815|4844|4|194| +2450815|4846|4|774| +2450815|4849|4|723| +2450815|4850|4|723| +2450815|4852|4|410| +2450815|4855|4|488| +2450815|4856|4|767| +2450815|4858|4|559| +2450815|4861|4|882| +2450815|4862|4|428| +2450815|4864|4|24| +2450815|4867|4|249| +2450815|4868|4|| +2450815|4870|4|110| +2450815|4873|4|723| +2450815|4874|4|| +2450815|4876|4|248| +2450815|4879|4|673| +2450815|4880|4|589| +2450815|4882|4|398| +2450815|4885|4|988| +2450815|4886|4|828| +2450815|4888|4|578| +2450815|4891|4|34| +2450815|4892|4|38| +2450815|4894|4|401| +2450815|4897|4|264| +2450815|4898|4|| +2450815|4900|4|908| +2450815|4903|4|299| +2450815|4904|4|| +2450815|4906|4|547| +2450815|4909|4|211| +2450815|4910|4|574| +2450815|4912|4|562| +2450815|4915|4|42| +2450815|4916|4|357| +2450815|4918|4|768| +2450815|4921|4|91| +2450815|4922|4|936| +2450815|4924|4|93| +2450815|4927|4|417| +2450815|4928|4|863| +2450815|4930|4|447| +2450815|4933|4|48| +2450815|4934|4|443| +2450815|4936|4|359| +2450815|4939|4|249| +2450815|4940|4|790| +2450815|4942|4|312| +2450815|4945|4|910| +2450815|4946|4|293| +2450815|4948|4|606| +2450815|4951|4|204| +2450815|4952|4|372| +2450815|4954|4|882| +2450815|4957|4|806| +2450815|4958|4|176| +2450815|4960|4|495| +2450815|4963|4|32| +2450815|4964|4|412| +2450815|4966|4|999| +2450815|4969|4|851| +2450815|4970|4|176| +2450815|4972|4|277| +2450815|4975|4|449| +2450815|4976|4|931| +2450815|4978|4|490| +2450815|4981|4|129| +2450815|4982|4|866| +2450815|4984|4|192| +2450815|4987|4|425| +2450815|4988|4|951| +2450815|4990|4|| +2450815|4993|4|248| +2450815|4994|4|947| +2450815|4996|4|112| +2450815|4999|4|899| +2450815|5000|4|781| +2450815|5002|4|362| +2450815|5005|4|615| +2450815|5006|4|842| +2450815|5008|4|185| +2450815|5011|4|257| +2450815|5012|4|994| +2450815|5014|4|364| +2450815|5017|4|714| +2450815|5018|4|544| +2450815|5020|4|359| +2450815|5023|4|157| +2450815|5024|4|122| +2450815|5026|4|221| +2450815|5029|4|856| +2450815|5030|4|910| +2450815|5032|4|798| +2450815|5035|4|68| +2450815|5036|4|116| +2450815|5038|4|400| +2450815|5041|4|793| +2450815|5042|4|335| +2450815|5044|4|259| +2450815|5047|4|318| +2450815|5048|4|13| +2450815|5050|4|209| +2450815|5053|4|330| +2450815|5054|4|492| +2450815|5056|4|875| +2450815|5059|4|960| +2450815|5060|4|533| +2450815|5062|4|932| +2450815|5065|4|814| +2450815|5066|4|421| +2450815|5068|4|346| +2450815|5071|4|221| +2450815|5072|4|883| +2450815|5074|4|649| +2450815|5077|4|849| +2450815|5078|4|945| +2450815|5080|4|890| +2450815|5083|4|648| +2450815|5084|4|59| +2450815|5086|4|660| +2450815|5089|4|67| +2450815|5090|4|68| +2450815|5092|4|635| +2450815|5095|4|58| +2450815|5096|4|758| +2450815|5098|4|153| +2450815|5101|4|145| +2450815|5102|4|279| +2450815|5104|4|827| +2450815|5107|4|325| +2450815|5108|4|485| +2450815|5110|4|118| +2450815|5113|4|458| +2450815|5114|4|158| +2450815|5116|4|397| +2450815|5119|4|85| +2450815|5120|4|| +2450815|5122|4|460| +2450815|5125|4|436| +2450815|5126|4|816| +2450815|5128|4|108| +2450815|5131|4|133| +2450815|5132|4|808| +2450815|5134|4|999| +2450815|5137|4|572| +2450815|5138|4|| +2450815|5140|4|410| +2450815|5143|4|718| +2450815|5144|4|2| +2450815|5146|4|675| +2450815|5149|4|697| +2450815|5150|4|35| +2450815|5152|4|848| +2450815|5155|4|985| +2450815|5156|4|630| +2450815|5158|4|430| +2450815|5161|4|792| +2450815|5162|4|776| +2450815|5164|4|524| +2450815|5167|4|463| +2450815|5168|4|512| +2450815|5170|4|687| +2450815|5173|4|761| +2450815|5174|4|98| +2450815|5176|4|622| +2450815|5179|4|716| +2450815|5180|4|344| +2450815|5182|4|451| +2450815|5185|4|891| +2450815|5186|4|395| +2450815|5188|4|290| +2450815|5191|4|169| +2450815|5192|4|768| +2450815|5194|4|979| +2450815|5197|4|180| +2450815|5198|4|157| +2450815|5200|4|42| +2450815|5203|4|211| +2450815|5204|4|304| +2450815|5206|4|415| +2450815|5209|4|4| +2450815|5210|4|644| +2450815|5212|4|943| +2450815|5215|4|| +2450815|5216|4|953| +2450815|5218|4|995| +2450815|5221|4|217| +2450815|5222|4|| +2450815|5224|4|816| +2450815|5227|4|341| +2450815|5228|4|20| +2450815|5230|4|921| +2450815|5233|4|707| +2450815|5234|4|230| +2450815|5236|4|153| +2450815|5239|4|319| +2450815|5240|4|478| +2450815|5242|4|303| +2450815|5245|4|321| +2450815|5246|4|667| +2450815|5248|4|749| +2450815|5251|4|631| +2450815|5252|4|600| +2450815|5254|4|| +2450815|5257|4|159| +2450815|5258|4|579| +2450815|5260|4|467| +2450815|5263|4|338| +2450815|5264|4|852| +2450815|5266|4|297| +2450815|5269|4|50| +2450815|5270|4|706| +2450815|5272|4|108| +2450815|5275|4|137| +2450815|5276|4|393| +2450815|5278|4|219| +2450815|5281|4|| +2450815|5282|4|941| +2450815|5284|4|541| +2450815|5287|4|601| +2450815|5288|4|794| +2450815|5290|4|165| +2450815|5293|4|455| +2450815|5294|4|913| +2450815|5296|4|458| +2450815|5299|4|733| +2450815|5300|4|139| +2450815|5302|4|719| +2450815|5305|4|769| +2450815|5306|4|473| +2450815|5308|4|591| +2450815|5311|4|992| +2450815|5312|4|995| +2450815|5314|4|442| +2450815|5317|4|795| +2450815|5318|4|| +2450815|5320|4|939| +2450815|5323|4|55| +2450815|5324|4|493| +2450815|5326|4|975| +2450815|5329|4|425| +2450815|5330|4|322| +2450815|5332|4|972| +2450815|5335|4|478| +2450815|5336|4|934| +2450815|5338|4|20| +2450815|5341|4|115| +2450815|5342|4|941| +2450815|5344|4|5| +2450815|5347|4|914| +2450815|5348|4|505| +2450815|5350|4|892| +2450815|5353|4|76| +2450815|5354|4|594| +2450815|5356|4|929| +2450815|5359|4|830| +2450815|5360|4|767| +2450815|5362|4|579| +2450815|5365|4|529| +2450815|5366|4|417| +2450815|5368|4|399| +2450815|5371|4|717| +2450815|5372|4|367| +2450815|5374|4|484| +2450815|5377|4|761| +2450815|5378|4|| +2450815|5380|4|66| +2450815|5383|4|965| +2450815|5384|4|318| +2450815|5386|4|814| +2450815|5389|4|878| +2450815|5390|4|942| +2450815|5392|4|104| +2450815|5395|4|598| +2450815|5396|4|704| +2450815|5398|4|225| +2450815|5401|4|192| +2450815|5402|4|740| +2450815|5404|4|381| +2450815|5407|4|933| +2450815|5408|4|427| +2450815|5410|4|67| +2450815|5413|4|848| +2450815|5414|4|960| +2450815|5416|4|211| +2450815|5419|4|730| +2450815|5420|4|577| +2450815|5422|4|316| +2450815|5425|4|59| +2450815|5426|4|814| +2450815|5428|4|20| +2450815|5431|4|873| +2450815|5432|4|| +2450815|5434|4|40| +2450815|5437|4|278| +2450815|5438|4|179| +2450815|5440|4|503| +2450815|5443|4|727| +2450815|5444|4|538| +2450815|5446|4|71| +2450815|5449|4|69| +2450815|5450|4|926| +2450815|5452|4|94| +2450815|5455|4|593| +2450815|5456|4|1000| +2450815|5458|4|476| +2450815|5461|4|505| +2450815|5462|4|966| +2450815|5464|4|102| +2450815|5467|4|488| +2450815|5468|4|887| +2450815|5470|4|612| +2450815|5473|4|773| +2450815|5474|4|126| +2450815|5476|4|568| +2450815|5479|4|626| +2450815|5480|4|78| +2450815|5482|4|186| +2450815|5485|4|911| +2450815|5486|4|458| +2450815|5488|4|392| +2450815|5491|4|632| +2450815|5492|4|833| +2450815|5494|4|812| +2450815|5497|4|328| +2450815|5498|4|592| +2450815|5500|4|488| +2450815|5503|4|877| +2450815|5504|4|546| +2450815|5506|4|352| +2450815|5509|4|312| +2450815|5510|4|192| +2450815|5512|4|949| +2450815|5515|4|175| +2450815|5516|4|262| +2450815|5518|4|290| +2450815|5521|4|517| +2450815|5522|4|985| +2450815|5524|4|681| +2450815|5527|4|778| +2450815|5528|4|966| +2450815|5530|4|840| +2450815|5533|4|362| +2450815|5534|4|816| +2450815|5536|4|421| +2450815|5539|4|42| +2450815|5540|4|327| +2450815|5542|4|337| +2450815|5545|4|97| +2450815|5546|4|925| +2450815|5548|4|47| +2450815|5551|4|963| +2450815|5552|4|184| +2450815|5554|4|164| +2450815|5557|4|136| +2450815|5558|4|771| +2450815|5560|4|685| +2450815|5563|4|194| +2450815|5564|4|72| +2450815|5566|4|488| +2450815|5569|4|975| +2450815|5570|4|513| +2450815|5572|4|338| +2450815|5575|4|652| +2450815|5576|4|971| +2450815|5578|4|486| +2450815|5581|4|314| +2450815|5582|4|922| +2450815|5584|4|605| +2450815|5587|4|738| +2450815|5588|4|823| +2450815|5590|4|417| +2450815|5593|4|990| +2450815|5594|4|340| +2450815|5596|4|789| +2450815|5599|4|496| +2450815|5600|4|533| +2450815|5602|4|| +2450815|5605|4|683| +2450815|5606|4|196| +2450815|5608|4|776| +2450815|5611|4|210| +2450815|5612|4|413| +2450815|5614|4|364| +2450815|5617|4|973| +2450815|5618|4|703| +2450815|5620|4|315| +2450815|5623|4|765| +2450815|5624|4|294| +2450815|5626|4|639| +2450815|5629|4|68| +2450815|5630|4|740| +2450815|5632|4|268| +2450815|5635|4|707| +2450815|5636|4|129| +2450815|5638|4|247| +2450815|5641|4|373| +2450815|5642|4|460| +2450815|5644|4|411| +2450815|5647|4|574| +2450815|5648|4|224| +2450815|5650|4|906| +2450815|5653|4|401| +2450815|5654|4|846| +2450815|5656|4|184| +2450815|5659|4|240| +2450815|5660|4|135| +2450815|5662|4|108| +2450815|5665|4|297| +2450815|5666|4|883| +2450815|5668|4|603| +2450815|5671|4|945| +2450815|5672|4|454| +2450815|5674|4|249| +2450815|5677|4|87| +2450815|5678|4|944| +2450815|5680|4|163| +2450815|5683|4|391| +2450815|5684|4|670| +2450815|5686|4|210| +2450815|5689|4|554| +2450815|5690|4|922| +2450815|5692|4|874| +2450815|5695|4|114| +2450815|5696|4|907| +2450815|5698|4|125| +2450815|5701|4|672| +2450815|5702|4|827| +2450815|5704|4|406| +2450815|5707|4|389| +2450815|5708|4|560| +2450815|5710|4|342| +2450815|5713|4|| +2450815|5714|4|412| +2450815|5716|4|461| +2450815|5719|4|964| +2450815|5720|4|435| +2450815|5722|4|36| +2450815|5725|4|10| +2450815|5726|4|67| +2450815|5728|4|125| +2450815|5731|4|410| +2450815|5732|4|870| +2450815|5734|4|38| +2450815|5737|4|857| +2450815|5738|4|588| +2450815|5740|4|332| +2450815|5743|4|| +2450815|5744|4|263| +2450815|5746|4|936| +2450815|5749|4|258| +2450815|5750|4|202| +2450815|5752|4|350| +2450815|5755|4|858| +2450815|5756|4|523| +2450815|5758|4|728| +2450815|5761|4|771| +2450815|5762|4|826| +2450815|5764|4|367| +2450815|5767|4|954| +2450815|5768|4|315| +2450815|5770|4|462| +2450815|5773|4|830| +2450815|5774|4|4| +2450815|5776|4|453| +2450815|5779|4|284| +2450815|5780|4|| +2450815|5782|4|47| +2450815|5785|4|225| +2450815|5786|4|851| +2450815|5788|4|307| +2450815|5791|4|852| +2450815|5792|4|278| +2450815|5794|4|157| +2450815|5797|4|604| +2450815|5798|4|245| +2450815|5800|4|632| +2450815|5803|4|345| +2450815|5804|4|721| +2450815|5806|4|961| +2450815|5809|4|19| +2450815|5810|4|883| +2450815|5812|4|870| +2450815|5815|4|226| +2450815|5816|4|965| +2450815|5818|4|| +2450815|5821|4|324| +2450815|5822|4|260| +2450815|5824|4|311| +2450815|5827|4|475| +2450815|5828|4|71| +2450815|5830|4|894| +2450815|5833|4|960| +2450815|5834|4|525| +2450815|5836|4|506| +2450815|5839|4|820| +2450815|5840|4|976| +2450815|5842|4|755| +2450815|5845|4|300| +2450815|5846|4|162| +2450815|5848|4|617| +2450815|5851|4|28| +2450815|5852|4|92| +2450815|5854|4|| +2450815|5857|4|429| +2450815|5858|4|253| +2450815|5860|4|470| +2450815|5863|4|361| +2450815|5864|4|208| +2450815|5866|4|96| +2450815|5869|4|951| +2450815|5870|4|480| +2450815|5872|4|847| +2450815|5875|4|82| +2450815|5876|4|380| +2450815|5878|4|369| +2450815|5881|4|929| +2450815|5882|4|883| +2450815|5884|4|89| +2450815|5887|4|700| +2450815|5888|4|446| +2450815|5890|4|110| +2450815|5893|4|160| +2450815|5894|4|226| +2450815|5896|4|140| +2450815|5899|4|813| +2450815|5900|4|| +2450815|5902|4|870| +2450815|5905|4|| +2450815|5906|4|308| +2450815|5908|4|52| +2450815|5911|4|454| +2450815|5912|4|341| +2450815|5914|4|469| +2450815|5917|4|19| +2450815|5918|4|943| +2450815|5920|4|439| +2450815|5923|4|497| +2450815|5924|4|207| +2450815|5926|4|957| +2450815|5929|4|| +2450815|5930|4|610| +2450815|5932|4|| +2450815|5935|4|876| +2450815|5936|4|421| +2450815|5938|4|691| +2450815|5941|4|77| +2450815|5942|4|634| +2450815|5944|4|| +2450815|5947|4|721| +2450815|5948|4|617| +2450815|5950|4|757| +2450815|5953|4|494| +2450815|5954|4|413| +2450815|5956|4|390| +2450815|5959|4|852| +2450815|5960|4|368| +2450815|5962|4|64| +2450815|5965|4|419| +2450815|5966|4|173| +2450815|5968|4|605| +2450815|5971|4|763| +2450815|5972|4|430| +2450815|5974|4|692| +2450815|5977|4|556| +2450815|5978|4|699| +2450815|5980|4|167| +2450815|5983|4|706| +2450815|5984|4|535| +2450815|5986|4|819| +2450815|5989|4|96| +2450815|5990|4|298| +2450815|5992|4|451| +2450815|5995|4|27| +2450815|5996|4|| +2450815|5998|4|51| +2450815|6001|4|493| +2450815|6002|4|451| +2450815|6004|4|920| +2450815|6007|4|168| +2450815|6008|4|639| +2450815|6010|4|699| +2450815|6013|4|441| +2450815|6014|4|922| +2450815|6016|4|96| +2450815|6019|4|465| +2450815|6020|4|647| +2450815|6022|4|| +2450815|6025|4|27| +2450815|6026|4|507| +2450815|6028|4|848| +2450815|6031|4|724| +2450815|6032|4|445| +2450815|6034|4|67| +2450815|6037|4|27| +2450815|6038|4|522| +2450815|6040|4|904| +2450815|6043|4|498| +2450815|6044|4|978| +2450815|6046|4|459| +2450815|6049|4|733| +2450815|6050|4|766| +2450815|6052|4|857| +2450815|6055|4|94| +2450815|6056|4|412| +2450815|6058|4|779| +2450815|6061|4|510| +2450815|6062|4|171| +2450815|6064|4|10| +2450815|6067|4|827| +2450815|6068|4|752| +2450815|6070|4|178| +2450815|6073|4|118| +2450815|6074|4|282| +2450815|6076|4|367| +2450815|6079|4|827| +2450815|6080|4|98| +2450815|6082|4|24| +2450815|6085|4|136| +2450815|6086|4|| +2450815|6088|4|300| +2450815|6091|4|670| +2450815|6092|4|488| +2450815|6094|4|64| +2450815|6097|4|| +2450815|6098|4|395| +2450815|6100|4|937| +2450815|6103|4|592| +2450815|6104|4|446| +2450815|6106|4|307| +2450815|6109|4|36| +2450815|6110|4|47| +2450815|6112|4|44| +2450815|6115|4|630| +2450815|6116|4|| +2450815|6118|4|558| +2450815|6121|4|437| +2450815|6122|4|371| +2450815|6124|4|687| +2450815|6127|4|452| +2450815|6128|4|44| +2450815|6130|4|846| +2450815|6133|4|830| +2450815|6134|4|789| +2450815|6136|4|501| +2450815|6139|4|277| +2450815|6140|4|90| +2450815|6142|4|| +2450815|6145|4|599| +2450815|6146|4|282| +2450815|6148|4|873| +2450815|6151|4|549| +2450815|6152|4|3| +2450815|6154|4|915| +2450815|6157|4|974| +2450815|6158|4|868| +2450815|6160|4|582| +2450815|6163|4|209| +2450815|6164|4|449| +2450815|6166|4|804| +2450815|6169|4|851| +2450815|6170|4|286| +2450815|6172|4|546| +2450815|6175|4|574| +2450815|6176|4|608| +2450815|6178|4|491| +2450815|6181|4|275| +2450815|6182|4|| +2450815|6184|4|942| +2450815|6187|4|926| +2450815|6188|4|432| +2450815|6190|4|124| +2450815|6193|4|764| +2450815|6194|4|841| +2450815|6196|4|401| +2450815|6199|4|913| +2450815|6200|4|828| +2450815|6202|4|51| +2450815|6205|4|798| +2450815|6206|4|| +2450815|6208|4|250| +2450815|6211|4|175| +2450815|6212|4|61| +2450815|6214|4|706| +2450815|6217|4|340| +2450815|6218|4|702| +2450815|6220|4|747| +2450815|6223|4|354| +2450815|6224|4|880| +2450815|6226|4|335| +2450815|6229|4|746| +2450815|6230|4|319| +2450815|6232|4|270| +2450815|6235|4|538| +2450815|6236|4|306| +2450815|6238|4|877| +2450815|6241|4|920| +2450815|6242|4|386| +2450815|6244|4|589| +2450815|6247|4|605| +2450815|6248|4|801| +2450815|6250|4|97| +2450815|6253|4|35| +2450815|6254|4|230| +2450815|6256|4|105| +2450815|6259|4|753| +2450815|6260|4|508| +2450815|6262|4|725| +2450815|6265|4|504| +2450815|6266|4|755| +2450815|6268|4|239| +2450815|6271|4|519| +2450815|6272|4|579| +2450815|6274|4|693| +2450815|6277|4|163| +2450815|6278|4|534| +2450815|6280|4|806| +2450815|6283|4|427| +2450815|6284|4|665| +2450815|6286|4|498| +2450815|6289|4|627| +2450815|6290|4|| +2450815|6292|4|275| +2450815|6295|4|920| +2450815|6296|4|811| +2450815|6298|4|608| +2450815|6301|4|962| +2450815|6302|4|860| +2450815|6304|4|976| +2450815|6307|4|631| +2450815|6308|4|985| +2450815|6310|4|38| +2450815|6313|4|74| +2450815|6314|4|146| +2450815|6316|4|290| +2450815|6319|4|246| +2450815|6320|4|489| +2450815|6322|4|198| +2450815|6325|4|711| +2450815|6326|4|238| +2450815|6328|4|633| +2450815|6331|4|362| +2450815|6332|4|830| +2450815|6334|4|82| +2450815|6337|4|467| +2450815|6338|4|833| +2450815|6340|4|550| +2450815|6343|4|780| +2450815|6344|4|806| +2450815|6346|4|379| +2450815|6349|4|761| +2450815|6350|4|63| +2450815|6352|4|560| +2450815|6355|4|442| +2450815|6356|4|551| +2450815|6358|4|987| +2450815|6361|4|409| +2450815|6362|4|471| +2450815|6364|4|750| +2450815|6367|4|524| +2450815|6368|4|| +2450815|6370|4|145| +2450815|6373|4|368| +2450815|6374|4|898| +2450815|6376|4|995| +2450815|6379|4|9| +2450815|6380|4|767| +2450815|6382|4|877| +2450815|6385|4|352| +2450815|6386|4|175| +2450815|6388|4|902| +2450815|6391|4|742| +2450815|6392|4|47| +2450815|6394|4|25| +2450815|6397|4|758| +2450815|6398|4|488| +2450815|6400|4|441| +2450815|6403|4|817| +2450815|6404|4|866| +2450815|6406|4|682| +2450815|6409|4|| +2450815|6410|4|317| +2450815|6412|4|29| +2450815|6415|4|433| +2450815|6416|4|92| +2450815|6418|4|629| +2450815|6421|4|515| +2450815|6422|4|549| +2450815|6424|4|775| +2450815|6427|4|344| +2450815|6428|4|268| +2450815|6430|4|718| +2450815|6433|4|90| +2450815|6434|4|344| +2450815|6436|4|625| +2450815|6439|4|40| +2450815|6440|4|548| +2450815|6442|4|909| +2450815|6445|4|126| +2450815|6446|4|51| +2450815|6448|4|443| +2450815|6451|4|356| +2450815|6452|4|762| +2450815|6454|4|434| +2450815|6457|4|152| +2450815|6458|4|366| +2450815|6460|4|960| +2450815|6463|4|758| +2450815|6464|4|546| +2450815|6466|4|2| +2450815|6469|4|527| +2450815|6470|4|239| +2450815|6472|4|550| +2450815|6475|4|690| +2450815|6476|4|461| +2450815|6478|4|434| +2450815|6481|4|299| +2450815|6482|4|750| +2450815|6484|4|518| +2450815|6487|4|255| +2450815|6488|4|130| +2450815|6490|4|947| +2450815|6493|4|381| +2450815|6494|4|333| +2450815|6496|4|316| +2450815|6499|4|571| +2450815|6500|4|| +2450815|6502|4|751| +2450815|6505|4|394| +2450815|6506|4|418| +2450815|6508|4|412| +2450815|6511|4|526| +2450815|6512|4|842| +2450815|6514|4|453| +2450815|6517|4|631| +2450815|6518|4|500| +2450815|6520|4|784| +2450815|6523|4|258| +2450815|6524|4|989| +2450815|6526|4|327| +2450815|6529|4|152| +2450815|6530|4|843| +2450815|6532|4|115| +2450815|6535|4|766| +2450815|6536|4|342| +2450815|6538|4|819| +2450815|6541|4|630| +2450815|6542|4|285| +2450815|6544|4|411| +2450815|6547|4|447| +2450815|6548|4|548| +2450815|6550|4|114| +2450815|6553|4|451| +2450815|6554|4|157| +2450815|6556|4|314| +2450815|6559|4|680| +2450815|6560|4|524| +2450815|6562|4|296| +2450815|6565|4|130| +2450815|6566|4|872| +2450815|6568|4|663| +2450815|6571|4|77| +2450815|6572|4|93| +2450815|6574|4|512| +2450815|6577|4|340| +2450815|6578|4|803| +2450815|6580|4|| +2450815|6583|4|675| +2450815|6584|4|181| +2450815|6586|4|651| +2450815|6589|4|70| +2450815|6590|4|535| +2450815|6592|4|159| +2450815|6595|4|| +2450815|6596|4|296| +2450815|6598|4|8| +2450815|6601|4|223| +2450815|6602|4|818| +2450815|6604|4|457| +2450815|6607|4|978| +2450815|6608|4|558| +2450815|6610|4|427| +2450815|6613|4|821| +2450815|6614|4|26| +2450815|6616|4|765| +2450815|6619|4|12| +2450815|6620|4|704| +2450815|6622|4|607| +2450815|6625|4|185| +2450815|6626|4|281| +2450815|6628|4|696| +2450815|6631|4|179| +2450815|6632|4|295| +2450815|6634|4|494| +2450815|6637|4|713| +2450815|6638|4|149| +2450815|6640|4|147| +2450815|6643|4|208| +2450815|6644|4|680| +2450815|6646|4|798| +2450815|6649|4|| +2450815|6650|4|770| +2450815|6652|4|803| +2450815|6655|4|881| +2450815|6656|4|998| +2450815|6658|4|680| +2450815|6661|4|423| +2450815|6662|4|509| +2450815|6664|4|937| +2450815|6667|4|960| +2450815|6668|4|846| +2450815|6670|4|170| +2450815|6673|4|282| +2450815|6674|4|77| +2450815|6676|4|819| +2450815|6679|4|983| +2450815|6680|4|456| +2450815|6682|4|397| +2450815|6685|4|950| +2450815|6686|4|374| +2450815|6688|4|784| +2450815|6691|4|131| +2450815|6692|4|493| +2450815|6694|4|963| +2450815|6697|4|679| +2450815|6698|4|619| +2450815|6700|4|95| +2450815|6703|4|308| +2450815|6704|4|802| +2450815|6706|4|| +2450815|6709|4|749| +2450815|6710|4|603| +2450815|6712|4|395| +2450815|6715|4|109| +2450815|6716|4|723| +2450815|6718|4|| +2450815|6721|4|930| +2450815|6722|4|152| +2450815|6724|4|709| +2450815|6727|4|247| +2450815|6728|4|646| +2450815|6730|4|600| +2450815|6733|4|879| +2450815|6734|4|146| +2450815|6736|4|362| +2450815|6739|4|250| +2450815|6740|4|375| +2450815|6742|4|952| +2450815|6745|4|742| +2450815|6746|4|633| +2450815|6748|4|908| +2450815|6751|4|791| +2450815|6752|4|613| +2450815|6754|4|844| +2450815|6757|4|329| +2450815|6758|4|970| +2450815|6760|4|279| +2450815|6763|4|904| +2450815|6764|4|548| +2450815|6766|4|836| +2450815|6769|4|225| +2450815|6770|4|38| +2450815|6772|4|193| +2450815|6775|4|517| +2450815|6776|4|522| +2450815|6778|4|931| +2450815|6781|4|635| +2450815|6782|4|| +2450815|6784|4|614| +2450815|6787|4|56| +2450815|6788|4|367| +2450815|6790|4|467| +2450815|6793|4|153| +2450815|6794|4|397| +2450815|6796|4|432| +2450815|6799|4|355| +2450815|6800|4|778| +2450815|6802|4|74| +2450815|6805|4|920| +2450815|6806|4|19| +2450815|6808|4|84| +2450815|6811|4|515| +2450815|6812|4|| +2450815|6814|4|210| +2450815|6817|4|811| +2450815|6818|4|249| +2450815|6820|4|581| +2450815|6823|4|714| +2450815|6824|4|307| +2450815|6826|4|705| +2450815|6829|4|156| +2450815|6830|4|329| +2450815|6832|4|891| +2450815|6835|4|49| +2450815|6836|4|675| +2450815|6838|4|955| +2450815|6841|4|812| +2450815|6842|4|320| +2450815|6844|4|836| +2450815|6847|4|980| +2450815|6848|4|826| +2450815|6850|4|202| +2450815|6853|4|477| +2450815|6854|4|411| +2450815|6856|4|488| +2450815|6859|4|134| +2450815|6860|4|667| +2450815|6862|4|478| +2450815|6865|4|413| +2450815|6866|4|628| +2450815|6868|4|64| +2450815|6871|4|525| +2450815|6872|4|256| +2450815|6874|4|62| +2450815|6877|4|424| +2450815|6878|4|813| +2450815|6880|4|657| +2450815|6883|4|295| +2450815|6884|4|694| +2450815|6886|4|46| +2450815|6889|4|| +2450815|6890|4|| +2450815|6892|4|164| +2450815|6895|4|684| +2450815|6896|4|788| +2450815|6898|4|417| +2450815|6901|4|126| +2450815|6902|4|237| +2450815|6904|4|| +2450815|6907|4|265| +2450815|6908|4|345| +2450815|6910|4|521| +2450815|6913|4|245| +2450815|6914|4|958| +2450815|6916|4|644| +2450815|6919|4|567| +2450815|6920|4|416| +2450815|6922|4|849| +2450815|6925|4|851| +2450815|6926|4|35| +2450815|6928|4|162| +2450815|6931|4|168| +2450815|6932|4|345| +2450815|6934|4|366| +2450815|6937|4|| +2450815|6938|4|788| +2450815|6940|4|| +2450815|6943|4|416| +2450815|6944|4|965| +2450815|6946|4|335| +2450815|6949|4|987| +2450815|6950|4|643| +2450815|6952|4|938| +2450815|6955|4|756| +2450815|6956|4|646| +2450815|6958|4|349| +2450815|6961|4|942| +2450815|6962|4|133| +2450815|6964|4|660| +2450815|6967|4|326| +2450815|6968|4|475| +2450815|6970|4|854| +2450815|6973|4|| +2450815|6974|4|104| +2450815|6976|4|182| +2450815|6979|4|187| +2450815|6980|4|228| +2450815|6982|4|296| +2450815|6985|4|529| +2450815|6986|4|147| +2450815|6988|4|105| +2450815|6991|4|451| +2450815|6992|4|147| +2450815|6994|4|101| +2450815|6997|4|590| +2450815|6998|4|285| +2450815|7000|4|581| +2450815|7003|4|929| +2450815|7004|4|253| +2450815|7006|4|434| +2450815|7009|4|376| +2450815|7010|4|911| +2450815|7012|4|764| +2450815|7015|4|208| +2450815|7016|4|822| +2450815|7018|4|939| +2450815|7021|4|252| +2450815|7022|4|766| +2450815|7024|4|85| +2450815|7027|4|701| +2450815|7028|4|203| +2450815|7030|4|565| +2450815|7033|4|613| +2450815|7034|4|715| +2450815|7036|4|934| +2450815|7039|4|743| +2450815|7040|4|365| +2450815|7042|4|81| +2450815|7045|4|84| +2450815|7046|4|880| +2450815|7048|4|391| +2450815|7051|4|935| +2450815|7052|4|357| +2450815|7054|4|685| +2450815|7057|4|716| +2450815|7058|4|199| +2450815|7060|4|249| +2450815|7063|4|795| +2450815|7064|4|524| +2450815|7066|4|728| +2450815|7069|4|762| +2450815|7070|4|796| +2450815|7072|4|35| +2450815|7075|4|396| +2450815|7076|4|20| +2450815|7078|4|148| +2450815|7081|4|972| +2450815|7082|4|655| +2450815|7084|4|768| +2450815|7087|4|608| +2450815|7088|4|595| +2450815|7090|4|583| +2450815|7093|4|16| +2450815|7094|4|639| +2450815|7096|4|940| +2450815|7099|4|414| +2450815|7100|4|481| +2450815|7102|4|38| +2450815|7105|4|| +2450815|7106|4|240| +2450815|7108|4|655| +2450815|7111|4|36| +2450815|7112|4|676| +2450815|7114|4|120| +2450815|7117|4|209| +2450815|7118|4|790| +2450815|7120|4|430| +2450815|7123|4|649| +2450815|7124|4|608| +2450815|7126|4|753| +2450815|7129|4|895| +2450815|7130|4|54| +2450815|7132|4|151| +2450815|7135|4|967| +2450815|7136|4|663| +2450815|7138|4|93| +2450815|7141|4|656| +2450815|7142|4|12| +2450815|7144|4|719| +2450815|7147|4|292| +2450815|7148|4|179| +2450815|7150|4|483| +2450815|7153|4|503| +2450815|7154|4|364| +2450815|7156|4|195| +2450815|7159|4|8| +2450815|7160|4|247| +2450815|7162|4|223| +2450815|7165|4|233| +2450815|7166|4|964| +2450815|7168|4|777| +2450815|7171|4|638| +2450815|7172|4|548| +2450815|7174|4|361| +2450815|7177|4|578| +2450815|7178|4|348| +2450815|7180|4|460| +2450815|7183|4|953| +2450815|7184|4|734| +2450815|7186|4|937| +2450815|7189|4|462| +2450815|7190|4|538| +2450815|7192|4|| +2450815|7195|4|641| +2450815|7196|4|140| +2450815|7198|4|512| +2450815|7201|4|292| +2450815|7202|4|669| +2450815|7204|4|693| +2450815|7207|4|594| +2450815|7208|4|810| +2450815|7210|4|459| +2450815|7213|4|950| +2450815|7214|4|855| +2450815|7216|4|| +2450815|7219|4|163| +2450815|7220|4|677| +2450815|7222|4|984| +2450815|7225|4|| +2450815|7226|4|388| +2450815|7228|4|569| +2450815|7231|4|699| +2450815|7232|4|401| +2450815|7234|4|685| +2450815|7237|4|| +2450815|7238|4|916| +2450815|7240|4|972| +2450815|7243|4|| +2450815|7244|4|88| +2450815|7246|4|89| +2450815|7249|4|900| +2450815|7250|4|776| +2450815|7252|4|375| +2450815|7255|4|607| +2450815|7256|4|156| +2450815|7258|4|767| +2450815|7261|4|920| +2450815|7262|4|947| +2450815|7264|4|872| +2450815|7267|4|926| +2450815|7268|4|194| +2450815|7270|4|462| +2450815|7273|4|| +2450815|7274|4|763| +2450815|7276|4|534| +2450815|7279|4|724| +2450815|7280|4|983| +2450815|7282|4|276| +2450815|7285|4|459| +2450815|7286|4|243| +2450815|7288|4|979| +2450815|7291|4|906| +2450815|7292|4|679| +2450815|7294|4|597| +2450815|7297|4|593| +2450815|7298|4|935| +2450815|7300|4|361| +2450815|7303|4|201| +2450815|7304|4|| +2450815|7306|4|823| +2450815|7309|4|919| +2450815|7310|4|572| +2450815|7312|4|782| +2450815|7315|4|715| +2450815|7316|4|320| +2450815|7318|4|345| +2450815|7321|4|900| +2450815|7322|4|7| +2450815|7324|4|| +2450815|7327|4|290| +2450815|7328|4|841| +2450815|7330|4|886| +2450815|7333|4|760| +2450815|7334|4|794| +2450815|7336|4|774| +2450815|7339|4|916| +2450815|7340|4|519| +2450815|7342|4|366| +2450815|7345|4|298| +2450815|7346|4|206| +2450815|7348|4|77| +2450815|7351|4|919| +2450815|7352|4|237| +2450815|7354|4|51| +2450815|7357|4|775| +2450815|7358|4|69| +2450815|7360|4|287| +2450815|7363|4|495| +2450815|7364|4|2| +2450815|7366|4|345| +2450815|7369|4|57| +2450815|7370|4|371| +2450815|7372|4|30| +2450815|7375|4|275| +2450815|7376|4|950| +2450815|7378|4|447| +2450815|7381|4|351| +2450815|7382|4|982| +2450815|7384|4|778| +2450815|7387|4|72| +2450815|7388|4|608| +2450815|7390|4|621| +2450815|7393|4|464| +2450815|7394|4|727| +2450815|7396|4|710| +2450815|7399|4|| +2450815|7400|4|898| +2450815|7402|4|687| +2450815|7405|4|125| +2450815|7406|4|333| +2450815|7408|4|| +2450815|7411|4|360| +2450815|7412|4|| +2450815|7414|4|148| +2450815|7417|4|294| +2450815|7418|4|45| +2450815|7420|4|605| +2450815|7423|4|422| +2450815|7424|4|188| +2450815|7426|4|371| +2450815|7429|4|943| +2450815|7430|4|992| +2450815|7432|4|409| +2450815|7435|4|276| +2450815|7436|4|127| +2450815|7438|4|64| +2450815|7441|4|379| +2450815|7442|4|343| +2450815|7444|4|401| +2450815|7447|4|132| +2450815|7448|4|331| +2450815|7450|4|520| +2450815|7453|4|850| +2450815|7454|4|285| +2450815|7456|4|448| +2450815|7459|4|857| +2450815|7460|4|| +2450815|7462|4|166| +2450815|7465|4|305| +2450815|7466|4|362| +2450815|7468|4|| +2450815|7471|4|847| +2450815|7472|4|294| +2450815|7474|4|35| +2450815|7477|4|562| +2450815|7478|4|295| +2450815|7480|4|791| +2450815|7483|4|660| +2450815|7484|4|284| +2450815|7486|4|641| +2450815|7489|4|323| +2450815|7490|4|| +2450815|7492|4|910| +2450815|7495|4|716| +2450815|7496|4|470| +2450815|7498|4|870| +2450815|7501|4|484| +2450815|7502|4|414| +2450815|7504|4|117| +2450815|7507|4|968| +2450815|7508|4|770| +2450815|7510|4|567| +2450815|7513|4|460| +2450815|7514|4|405| +2450815|7516|4|235| +2450815|7519|4|507| +2450815|7520|4|748| +2450815|7522|4|499| +2450815|7525|4|129| +2450815|7526|4|479| +2450815|7528|4|712| +2450815|7531|4|318| +2450815|7532|4|522| +2450815|7534|4|665| +2450815|7537|4|137| +2450815|7538|4|51| +2450815|7540|4|262| +2450815|7543|4|420| +2450815|7544|4|820| +2450815|7546|4|301| +2450815|7549|4|559| +2450815|7550|4|96| +2450815|7552|4|239| +2450815|7555|4|72| +2450815|7556|4|790| +2450815|7558|4|241| +2450815|7561|4|419| +2450815|7562|4|| +2450815|7564|4|366| +2450815|7567|4|437| +2450815|7568|4|816| +2450815|7570|4|22| +2450815|7573|4|155| +2450815|7574|4|957| +2450815|7576|4|29| +2450815|7579|4|341| +2450815|7580|4|914| +2450815|7582|4|442| +2450815|7585|4|122| +2450815|7586|4|334| +2450815|7588|4|323| +2450815|7591|4|191| +2450815|7592|4|387| +2450815|7594|4|39| +2450815|7597|4|72| +2450815|7598|4|595| +2450815|7600|4|321| +2450815|7603|4|425| +2450815|7604|4|919| +2450815|7606|4|359| +2450815|7609|4|628| +2450815|7610|4|268| +2450815|7612|4|114| +2450815|7615|4|387| +2450815|7616|4|693| +2450815|7618|4|530| +2450815|7621|4|133| +2450815|7622|4|647| +2450815|7624|4|| +2450815|7627|4|118| +2450815|7628|4|317| +2450815|7630|4|727| +2450815|7633|4|331| +2450815|7634|4|272| +2450815|7636|4|684| +2450815|7639|4|411| +2450815|7640|4|333| +2450815|7642|4|646| +2450815|7645|4|846| +2450815|7646|4|593| +2450815|7648|4|477| +2450815|7651|4|843| +2450815|7652|4|576| +2450815|7654|4|381| +2450815|7657|4|594| +2450815|7658|4|188| +2450815|7660|4|429| +2450815|7663|4|809| +2450815|7664|4|479| +2450815|7666|4|| +2450815|7669|4|414| +2450815|7670|4|76| +2450815|7672|4|328| +2450815|7675|4|389| +2450815|7676|4|942| +2450815|7678|4|454| +2450815|7681|4|25| +2450815|7682|4|482| +2450815|7684|4|679| +2450815|7687|4|157| +2450815|7688|4|404| +2450815|7690|4|526| +2450815|7693|4|442| +2450815|7694|4|61| +2450815|7696|4|983| +2450815|7699|4|110| +2450815|7700|4|198| +2450815|7702|4|485| +2450815|7705|4|| +2450815|7706|4|19| +2450815|7708|4|234| +2450815|7711|4|771| +2450815|7712|4|590| +2450815|7714|4|492| +2450815|7717|4|753| +2450815|7718|4|608| +2450815|7720|4|| +2450815|7723|4|741| +2450815|7724|4|930| +2450815|7726|4|960| +2450815|7729|4|921| +2450815|7730|4|897| +2450815|7732|4|143| +2450815|7735|4|608| +2450815|7736|4|634| +2450815|7738|4|564| +2450815|7741|4|859| +2450815|7742|4|154| +2450815|7744|4|315| +2450815|7747|4|379| +2450815|7748|4|321| +2450815|7750|4|350| +2450815|7753|4|518| +2450815|7754|4|666| +2450815|7756|4|532| +2450815|7759|4|410| +2450815|7760|4|694| +2450815|7762|4|762| +2450815|7765|4|| +2450815|7766|4|612| +2450815|7768|4|191| +2450815|7771|4|120| +2450815|7772|4|270| +2450815|7774|4|744| +2450815|7777|4|720| +2450815|7778|4|184| +2450815|7780|4|892| +2450815|7783|4|85| +2450815|7784|4|548| +2450815|7786|4|377| +2450815|7789|4|| +2450815|7790|4|315| +2450815|7792|4|240| +2450815|7795|4|121| +2450815|7796|4|487| +2450815|7798|4|873| +2450815|7801|4|266| +2450815|7802|4|732| +2450815|7804|4|837| +2450815|7807|4|861| +2450815|7808|4|156| +2450815|7810|4|868| +2450815|7813|4|577| +2450815|7814|4|619| +2450815|7816|4|367| +2450815|7819|4|171| +2450815|7820|4|746| +2450815|7822|4|385| +2450815|7825|4|192| +2450815|7826|4|106| +2450815|7828|4|961| +2450815|7831|4|180| +2450815|7832|4|706| +2450815|7834|4|600| +2450815|7837|4|50| +2450815|7838|4|| +2450815|7840|4|404| +2450815|7843|4|819| +2450815|7844|4|932| +2450815|7846|4|234| +2450815|7849|4|31| +2450815|7850|4|526| +2450815|7852|4|175| +2450815|7855|4|566| +2450815|7856|4|277| +2450815|7858|4|940| +2450815|7861|4|629| +2450815|7862|4|734| +2450815|7864|4|335| +2450815|7867|4|410| +2450815|7868|4|475| +2450815|7870|4|| +2450815|7873|4|686| +2450815|7874|4|178| +2450815|7876|4|32| +2450815|7879|4|183| +2450815|7880|4|| +2450815|7882|4|604| +2450815|7885|4|845| +2450815|7886|4|915| +2450815|7888|4|983| +2450815|7891|4|1| +2450815|7892|4|547| +2450815|7894|4|672| +2450815|7897|4|794| +2450815|7898|4|179| +2450815|7900|4|465| +2450815|7903|4|862| +2450815|7904|4|571| +2450815|7906|4|925| +2450815|7909|4|702| +2450815|7910|4|301| +2450815|7912|4|810| +2450815|7915|4|834| +2450815|7916|4|| +2450815|7918|4|630| +2450815|7921|4|292| +2450815|7922|4|292| +2450815|7924|4|| +2450815|7927|4|| +2450815|7928|4|522| +2450815|7930|4|517| +2450815|7933|4|752| +2450815|7934|4|465| +2450815|7936|4|941| +2450815|7939|4|581| +2450815|7940|4|504| +2450815|7942|4|829| +2450815|7945|4|96| +2450815|7946|4|121| +2450815|7948|4|944| +2450815|7951|4|854| +2450815|7952|4|255| +2450815|7954|4|239| +2450815|7957|4|573| +2450815|7958|4|112| +2450815|7960|4|413| +2450815|7963|4|752| +2450815|7964|4|597| +2450815|7966|4|325| +2450815|7969|4|391| +2450815|7970|4|892| +2450815|7972|4|941| +2450815|7975|4|737| +2450815|7976|4|452| +2450815|7978|4|683| +2450815|7981|4|434| +2450815|7982|4|369| +2450815|7984|4|164| +2450815|7987|4|| +2450815|7988|4|381| +2450815|7990|4|680| +2450815|7993|4|50| +2450815|7994|4|510| +2450815|7996|4|77| +2450815|7999|4|173| +2450815|8000|4|383| +2450815|8002|4|600| +2450815|8005|4|437| +2450815|8006|4|458| +2450815|8008|4|997| +2450815|8011|4|445| +2450815|8012|4|102| +2450815|8014|4|287| +2450815|8017|4|88| +2450815|8018|4|522| +2450815|8020|4|400| +2450815|8023|4|655| +2450815|8024|4|224| +2450815|8026|4|570| +2450815|8029|4|524| +2450815|8030|4|679| +2450815|8032|4|101| +2450815|8035|4|39| +2450815|8036|4|530| +2450815|8038|4|505| +2450815|8041|4|331| +2450815|8042|4|448| +2450815|8044|4|816| +2450815|8047|4|119| +2450815|8048|4|906| +2450815|8050|4|21| +2450815|8053|4|899| +2450815|8054|4|636| +2450815|8056|4|147| +2450815|8059|4|876| +2450815|8060|4|| +2450815|8062|4|380| +2450815|8065|4|400| +2450815|8066|4|163| +2450815|8068|4|281| +2450815|8071|4|761| +2450815|8072|4|39| +2450815|8074|4|925| +2450815|8077|4|| +2450815|8078|4|551| +2450815|8080|4|335| +2450815|8083|4|731| +2450815|8084|4|556| +2450815|8086|4|136| +2450815|8089|4|958| +2450815|8090|4|315| +2450815|8092|4|105| +2450815|8095|4|461| +2450815|8096|4|338| +2450815|8098|4|835| +2450815|8101|4|496| +2450815|8102|4|209| +2450815|8104|4|76| +2450815|8107|4|987| +2450815|8108|4|116| +2450815|8110|4|874| +2450815|8113|4|135| +2450815|8114|4|351| +2450815|8116|4|894| +2450815|8119|4|29| +2450815|8120|4|832| +2450815|8122|4|318| +2450815|8125|4|252| +2450815|8126|4|654| +2450815|8128|4|596| +2450815|8131|4|9| +2450815|8132|4|677| +2450815|8134|4|622| +2450815|8137|4|120| +2450815|8138|4|495| +2450815|8140|4|50| +2450815|8143|4|118| +2450815|8144|4|885| +2450815|8146|4|196| +2450815|8149|4|821| +2450815|8150|4|| +2450815|8152|4|5| +2450815|8155|4|215| +2450815|8156|4|356| +2450815|8158|4|757| +2450815|8161|4|| +2450815|8162|4|593| +2450815|8164|4|717| +2450815|8167|4|738| +2450815|8168|4|294| +2450815|8170|4|484| +2450815|8173|4|591| +2450815|8174|4|943| +2450815|8176|4|| +2450815|8179|4|166| +2450815|8180|4|889| +2450815|8182|4|885| +2450815|8185|4|66| +2450815|8186|4|274| +2450815|8188|4|749| +2450815|8191|4|78| +2450815|8192|4|221| +2450815|8194|4|153| +2450815|8197|4|361| +2450815|8198|4|50| +2450815|8200|4|76| +2450815|8203|4|73| +2450815|8204|4|268| +2450815|8206|4|51| +2450815|8209|4|974| +2450815|8210|4|249| +2450815|8212|4|50| +2450815|8215|4|| +2450815|8216|4|556| +2450815|8218|4|67| +2450815|8221|4|931| +2450815|8222|4|621| +2450815|8224|4|638| +2450815|8227|4|334| +2450815|8228|4|613| +2450815|8230|4|113| +2450815|8233|4|| +2450815|8234|4|5| +2450815|8236|4|79| +2450815|8239|4|480| +2450815|8240|4|103| +2450815|8242|4|806| +2450815|8245|4|440| +2450815|8246|4|565| +2450815|8248|4|829| +2450815|8251|4|50| +2450815|8252|4|720| +2450815|8254|4|426| +2450815|8257|4|431| +2450815|8258|4|| +2450815|8260|4|333| +2450815|8263|4|480| +2450815|8264|4|311| +2450815|8266|4|276| +2450815|8269|4|770| +2450815|8270|4|| +2450815|8272|4|283| +2450815|8275|4|757| +2450815|8276|4|706| +2450815|8278|4|730| +2450815|8281|4|436| +2450815|8282|4|347| +2450815|8284|4|872| +2450815|8287|4|230| +2450815|8288|4|148| +2450815|8290|4|940| +2450815|8293|4|50| +2450815|8294|4|624| +2450815|8296|4|793| +2450815|8299|4|588| +2450815|8300|4|697| +2450815|8302|4|124| +2450815|8305|4|76| +2450815|8306|4|336| +2450815|8308|4|456| +2450815|8311|4|222| +2450815|8312|4|316| +2450815|8314|4|496| +2450815|8317|4|559| +2450815|8318|4|663| +2450815|8320|4|528| +2450815|8323|4|700| +2450815|8324|4|538| +2450815|8326|4|795| +2450815|8329|4|421| +2450815|8330|4|627| +2450815|8332|4|816| +2450815|8335|4|633| +2450815|8336|4|456| +2450815|8338|4|99| +2450815|8341|4|327| +2450815|8342|4|99| +2450815|8344|4|376| +2450815|8347|4|798| +2450815|8348|4|736| +2450815|8350|4|358| +2450815|8353|4|83| +2450815|8354|4|207| +2450815|8356|4|| +2450815|8359|4|540| +2450815|8360|4|800| +2450815|8362|4|296| +2450815|8365|4|422| +2450815|8366|4|462| +2450815|8368|4|637| +2450815|8371|4|350| +2450815|8372|4|831| +2450815|8374|4|863| +2450815|8377|4|350| +2450815|8378|4|532| +2450815|8380|4|506| +2450815|8383|4|471| +2450815|8384|4|680| +2450815|8386|4|488| +2450815|8389|4|303| +2450815|8390|4|227| +2450815|8392|4|231| +2450815|8395|4|473| +2450815|8396|4|145| +2450815|8398|4|731| +2450815|8401|4|759| +2450815|8402|4|103| +2450815|8404|4|53| +2450815|8407|4|361| +2450815|8408|4|857| +2450815|8410|4|550| +2450815|8413|4|332| +2450815|8414|4|750| +2450815|8416|4|746| +2450815|8419|4|393| +2450815|8420|4|69| +2450815|8422|4|| +2450815|8425|4|738| +2450815|8426|4|674| +2450815|8428|4|379| +2450815|8431|4|503| +2450815|8432|4|| +2450815|8434|4|986| +2450815|8437|4|676| +2450815|8438|4|607| +2450815|8440|4|272| +2450815|8443|4|541| +2450815|8444|4|827| +2450815|8446|4|530| +2450815|8449|4|912| +2450815|8450|4|390| +2450815|8452|4|404| +2450815|8455|4|138| +2450815|8456|4|87| +2450815|8458|4|376| +2450815|8461|4|484| +2450815|8462|4|515| +2450815|8464|4|290| +2450815|8467|4|709| +2450815|8468|4|83| +2450815|8470|4|848| +2450815|8473|4|856| +2450815|8474|4|8| +2450815|8476|4|4| +2450815|8479|4|560| +2450815|8480|4|225| +2450815|8482|4|452| +2450815|8485|4|514| +2450815|8486|4|444| +2450815|8488|4|580| +2450815|8491|4|799| +2450815|8492|4|84| +2450815|8494|4|| +2450815|8497|4|302| +2450815|8498|4|16| +2450815|8500|4|269| +2450815|8503|4|174| +2450815|8504|4|322| +2450815|8506|4|430| +2450815|8509|4|873| +2450815|8510|4|33| +2450815|8512|4|501| +2450815|8515|4|859| +2450815|8516|4|937| +2450815|8518|4|581| +2450815|8521|4|21| +2450815|8522|4|32| +2450815|8524|4|481| +2450815|8527|4|618| +2450815|8528|4|199| +2450815|8530|4|683| +2450815|8533|4|750| +2450815|8534|4|984| +2450815|8536|4|| +2450815|8539|4|46| +2450815|8540|4|604| +2450815|8542|4|958| +2450815|8545|4|950| +2450815|8546|4|634| +2450815|8548|4|190| +2450815|8551|4|944| +2450815|8552|4|785| +2450815|8554|4|574| +2450815|8557|4|908| +2450815|8558|4|387| +2450815|8560|4|250| +2450815|8563|4|869| +2450815|8564|4|312| +2450815|8566|4|378| +2450815|8569|4|191| +2450815|8570|4|565| +2450815|8572|4|724| +2450815|8575|4|832| +2450815|8576|4|179| +2450815|8578|4|70| +2450815|8581|4|898| +2450815|8582|4|764| +2450815|8584|4|775| +2450815|8587|4|201| +2450815|8588|4|509| +2450815|8590|4|734| +2450815|8593|4|41| +2450815|8594|4|914| +2450815|8596|4|935| +2450815|8599|4|769| +2450815|8600|4|545| +2450815|8602|4|530| +2450815|8605|4|891| +2450815|8606|4|15| +2450815|8608|4|| +2450815|8611|4|419| +2450815|8612|4|114| +2450815|8614|4|567| +2450815|8617|4|419| +2450815|8618|4|783| +2450815|8620|4|721| +2450815|8623|4|539| +2450815|8624|4|715| +2450815|8626|4|465| +2450815|8629|4|806| +2450815|8630|4|185| +2450815|8632|4|558| +2450815|8635|4|195| +2450815|8636|4|72| +2450815|8638|4|519| +2450815|8641|4|140| +2450815|8642|4|436| +2450815|8644|4|188| +2450815|8647|4|937| +2450815|8648|4|482| +2450815|8650|4|583| +2450815|8653|4|600| +2450815|8654|4|| +2450815|8656|4|545| +2450815|8659|4|184| +2450815|8660|4|104| +2450815|8662|4|184| +2450815|8665|4|403| +2450815|8666|4|13| +2450815|8668|4|971| +2450815|8671|4|292| +2450815|8672|4|581| +2450815|8674|4|936| +2450815|8677|4|529| +2450815|8678|4|86| +2450815|8680|4|666| +2450815|8683|4|421| +2450815|8684|4|571| +2450815|8686|4|698| +2450815|8689|4|451| +2450815|8690|4|451| +2450815|8692|4|155| +2450815|8695|4|182| +2450815|8696|4|904| +2450815|8698|4|523| +2450815|8701|4|190| +2450815|8702|4|261| +2450815|8704|4|123| +2450815|8707|4|340| +2450815|8708|4|160| +2450815|8710|4|448| +2450815|8713|4|993| +2450815|8714|4|166| +2450815|8716|4|192| +2450815|8719|4|393| +2450815|8720|4|91| +2450815|8722|4|73| +2450815|8725|4|414| +2450815|8726|4|973| +2450815|8728|4|959| +2450815|8731|4|807| +2450815|8732|4|876| +2450815|8734|4|40| +2450815|8737|4|676| +2450815|8738|4|615| +2450815|8740|4|994| +2450815|8743|4|459| +2450815|8744|4|407| +2450815|8746|4|44| +2450815|8749|4|116| +2450815|8750|4|92| +2450815|8752|4|| +2450815|8755|4|55| +2450815|8756|4|237| +2450815|8758|4|773| +2450815|8761|4|631| +2450815|8762|4|340| +2450815|8764|4|991| +2450815|8767|4|142| +2450815|8768|4|660| +2450815|8770|4|329| +2450815|8773|4|323| +2450815|8774|4|100| +2450815|8776|4|436| +2450815|8779|4|834| +2450815|8780|4|652| +2450815|8782|4|412| +2450815|8785|4|933| +2450815|8786|4|638| +2450815|8788|4|129| +2450815|8791|4|705| +2450815|8792|4|888| +2450815|8794|4|597| +2450815|8797|4|918| +2450815|8798|4|199| +2450815|8800|4|758| +2450815|8803|4|293| +2450815|8804|4|69| +2450815|8806|4|942| +2450815|8809|4|722| +2450815|8810|4|682| +2450815|8812|4|979| +2450815|8815|4|804| +2450815|8816|4|681| +2450815|8818|4|522| +2450815|8821|4|968| +2450815|8822|4|644| +2450815|8824|4|490| +2450815|8827|4|847| +2450815|8828|4|230| +2450815|8830|4|647| +2450815|8833|4|512| +2450815|8834|4|453| +2450815|8836|4|647| +2450815|8839|4|256| +2450815|8840|4|232| +2450815|8842|4|500| +2450815|8845|4|686| +2450815|8846|4|522| +2450815|8848|4|527| +2450815|8851|4|342| +2450815|8852|4|603| +2450815|8854|4|233| +2450815|8857|4|507| +2450815|8858|4|740| +2450815|8860|4|966| +2450815|8863|4|130| +2450815|8864|4|826| +2450815|8866|4|327| +2450815|8869|4|921| +2450815|8870|4|477| +2450815|8872|4|455| +2450815|8875|4|488| +2450815|8876|4|795| +2450815|8878|4|80| +2450815|8881|4|300| +2450815|8882|4|527| +2450815|8884|4|232| +2450815|8887|4|3| +2450815|8888|4|312| +2450815|8890|4|874| +2450815|8893|4|883| +2450815|8894|4|351| +2450815|8896|4|314| +2450815|8899|4|620| +2450815|8900|4|984| +2450815|8902|4|907| +2450815|8905|4|| +2450815|8906|4|929| +2450815|8908|4|838| +2450815|8911|4|406| +2450815|8912|4|283| +2450815|8914|4|| +2450815|8917|4|30| +2450815|8918|4|331| +2450815|8920|4|978| +2450815|8923|4|28| +2450815|8924|4|827| +2450815|8926|4|| +2450815|8929|4|116| +2450815|8930|4|398| +2450815|8932|4|153| +2450815|8935|4|377| +2450815|8936|4|790| +2450815|8938|4|111| +2450815|8941|4|397| +2450815|8942|4|15| +2450815|8944|4|777| +2450815|8947|4|955| +2450815|8948|4|829| +2450815|8950|4|145| +2450815|8953|4|| +2450815|8954|4|151| +2450815|8956|4|280| +2450815|8959|4|687| +2450815|8960|4|939| +2450815|8962|4|590| +2450815|8965|4|972| +2450815|8966|4|80| +2450815|8968|4|145| +2450815|8971|4|64| +2450815|8972|4|256| +2450815|8974|4|466| +2450815|8977|4|79| +2450815|8978|4|683| +2450815|8980|4|18| +2450815|8983|4|| +2450815|8984|4|362| +2450815|8986|4|348| +2450815|8989|4|477| +2450815|8990|4|885| +2450815|8992|4|288| +2450815|8995|4|459| +2450815|8996|4|619| +2450815|8998|4|874| +2450815|9001|4|653| +2450815|9002|4|134| +2450815|9004|4|731| +2450815|9007|4|4| +2450815|9008|4|281| +2450815|9010|4|812| +2450815|9013|4|| +2450815|9014|4|249| +2450815|9016|4|960| +2450815|9019|4|804| +2450815|9020|4|862| +2450815|9022|4|726| +2450815|9025|4|349| +2450815|9026|4|| +2450815|9028|4|267| +2450815|9031|4|600| +2450815|9032|4|317| +2450815|9034|4|296| +2450815|9037|4|418| +2450815|9038|4|848| +2450815|9040|4|792| +2450815|9043|4|150| +2450815|9044|4|| +2450815|9046|4|821| +2450815|9049|4|732| +2450815|9050|4|317| +2450815|9052|4|504| +2450815|9055|4|758| +2450815|9056|4|410| +2450815|9058|4|808| +2450815|9061|4|469| +2450815|9062|4|509| +2450815|9064|4|618| +2450815|9067|4|268| +2450815|9068|4|795| +2450815|9070|4|385| +2450815|9073|4|589| +2450815|9074|4|875| +2450815|9076|4|624| +2450815|9079|4|129| +2450815|9080|4|417| +2450815|9082|4|891| +2450815|9085|4|582| +2450815|9086|4|706| +2450815|9088|4|325| +2450815|9091|4|808| +2450815|9092|4|21| +2450815|9094|4|46| +2450815|9097|4|44| +2450815|9098|4|591| +2450815|9100|4|6| +2450815|9103|4|895| +2450815|9104|4|744| +2450815|9106|4|294| +2450815|9109|4|415| +2450815|9110|4|183| +2450815|9112|4|223| +2450815|9115|4|396| +2450815|9116|4|280| +2450815|9118|4|855| +2450815|9121|4|374| +2450815|9122|4|473| +2450815|9124|4|149| +2450815|9127|4|938| +2450815|9128|4|700| +2450815|9130|4|260| +2450815|9133|4|541| +2450815|9134|4|517| +2450815|9136|4|814| +2450815|9139|4|64| +2450815|9140|4|541| +2450815|9142|4|691| +2450815|9145|4|859| +2450815|9146|4|965| +2450815|9148|4|702| +2450815|9151|4|673| +2450815|9152|4|497| +2450815|9154|4|996| +2450815|9157|4|620| +2450815|9158|4|790| +2450815|9160|4|713| +2450815|9163|4|785| +2450815|9164|4|913| +2450815|9166|4|| +2450815|9169|4|252| +2450815|9170|4|126| +2450815|9172|4|372| +2450815|9175|4|280| +2450815|9176|4|811| +2450815|9178|4|495| +2450815|9181|4|743| +2450815|9182|4|81| +2450815|9184|4|947| +2450815|9187|4|26| +2450815|9188|4|928| +2450815|9190|4|12| +2450815|9193|4|886| +2450815|9194|4|153| +2450815|9196|4|611| +2450815|9199|4|552| +2450815|9200|4|580| +2450815|9202|4|692| +2450815|9205|4|496| +2450815|9206|4|943| +2450815|9208|4|42| +2450815|9211|4|227| +2450815|9212|4|402| +2450815|9214|4|296| +2450815|9217|4|521| +2450815|9218|4|| +2450815|9220|4|817| +2450815|9223|4|950| +2450815|9224|4|185| +2450815|9226|4|550| +2450815|9229|4|235| +2450815|9230|4|19| +2450815|9232|4|274| +2450815|9235|4|886| +2450815|9236|4|696| +2450815|9238|4|| +2450815|9241|4|405| +2450815|9242|4|444| +2450815|9244|4|241| +2450815|9247|4|836| +2450815|9248|4|624| +2450815|9250|4|227| +2450815|9253|4|235| +2450815|9254|4|162| +2450815|9256|4|867| +2450815|9259|4|862| +2450815|9260|4|351| +2450815|9262|4|300| +2450815|9265|4|601| +2450815|9266|4|| +2450815|9268|4|751| +2450815|9271|4|541| +2450815|9272|4|652| +2450815|9274|4|628| +2450815|9277|4|193| +2450815|9278|4|239| +2450815|9280|4|264| +2450815|9283|4|229| +2450815|9284|4|424| +2450815|9286|4|608| +2450815|9289|4|16| +2450815|9290|4|227| +2450815|9292|4|822| +2450815|9295|4|347| +2450815|9296|4|577| +2450815|9298|4|896| +2450815|9301|4|843| +2450815|9302|4|940| +2450815|9304|4|429| +2450815|9307|4|380| +2450815|9308|4|403| +2450815|9310|4|164| +2450815|9313|4|598| +2450815|9314|4|278| +2450815|9316|4|595| +2450815|9319|4|106| +2450815|9320|4|76| +2450815|9322|4|339| +2450815|9325|4|782| +2450815|9326|4|200| +2450815|9328|4|899| +2450815|9331|4|294| +2450815|9332|4|322| +2450815|9334|4|439| +2450815|9337|4|904| +2450815|9338|4|764| +2450815|9340|4|487| +2450815|9343|4|998| +2450815|9344|4|786| +2450815|9346|4|399| +2450815|9349|4|899| +2450815|9350|4|857| +2450815|9352|4|| +2450815|9355|4|956| +2450815|9356|4|886| +2450815|9358|4|572| +2450815|9361|4|477| +2450815|9362|4|496| +2450815|9364|4|252| +2450815|9367|4|632| +2450815|9368|4|611| +2450815|9370|4|186| +2450815|9373|4|601| +2450815|9374|4|559| +2450815|9376|4|768| +2450815|9379|4|383| +2450815|9380|4|336| +2450815|9382|4|795| +2450815|9385|4|132| +2450815|9386|4|805| +2450815|9388|4|695| +2450815|9391|4|853| +2450815|9392|4|575| +2450815|9394|4|526| +2450815|9397|4|880| +2450815|9398|4|217| +2450815|9400|4|212| +2450815|9403|4|366| +2450815|9404|4|822| +2450815|9406|4|38| +2450815|9409|4|241| +2450815|9410|4|636| +2450815|9412|4|552| +2450815|9415|4|779| +2450815|9416|4|394| +2450815|9418|4|199| +2450815|9421|4|331| +2450815|9422|4|338| +2450815|9424|4|355| +2450815|9427|4|848| +2450815|9428|4|429| +2450815|9430|4|479| +2450815|9433|4|407| +2450815|9434|4|591| +2450815|9436|4|238| +2450815|9439|4|588| +2450815|9440|4|| +2450815|9442|4|772| +2450815|9445|4|485| +2450815|9446|4|341| +2450815|9448|4|885| +2450815|9451|4|579| +2450815|9452|4|592| +2450815|9454|4|765| +2450815|9457|4|892| +2450815|9458|4|348| +2450815|9460|4|135| +2450815|9463|4|6| +2450815|9464|4|278| +2450815|9466|4|665| +2450815|9469|4|151| +2450815|9470|4|850| +2450815|9472|4|988| +2450815|9475|4|763| +2450815|9476|4|| +2450815|9478|4|453| +2450815|9481|4|933| +2450815|9482|4|893| +2450815|9484|4|| +2450815|9487|4|22| +2450815|9488|4|238| +2450815|9490|4|414| +2450815|9493|4|448| +2450815|9494|4|| +2450815|9496|4|424| +2450815|9499|4|226| +2450815|9500|4|286| +2450815|9502|4|817| +2450815|9505|4|| +2450815|9506|4|977| +2450815|9508|4|156| +2450815|9511|4|275| +2450815|9512|4|826| +2450815|9514|4|369| +2450815|9517|4|479| +2450815|9518|4|244| +2450815|9520|4|481| +2450815|9523|4|155| +2450815|9524|4|327| +2450815|9526|4|65| +2450815|9529|4|556| +2450815|9530|4|416| +2450815|9532|4|816| +2450815|9535|4|565| +2450815|9536|4|579| +2450815|9538|4|758| +2450815|9541|4|842| +2450815|9542|4|645| +2450815|9544|4|36| +2450815|9547|4|622| +2450815|9548|4|231| +2450815|9550|4|636| +2450815|9553|4|549| +2450815|9554|4|285| +2450815|9556|4|| +2450815|9559|4|362| +2450815|9560|4|| +2450815|9562|4|335| +2450815|9565|4|984| +2450815|9566|4|276| +2450815|9568|4|949| +2450815|9571|4|240| +2450815|9572|4|34| +2450815|9574|4|118| +2450815|9577|4|879| +2450815|9578|4|935| +2450815|9580|4|439| +2450815|9583|4|485| +2450815|9584|4|196| +2450815|9586|4|769| +2450815|9589|4|0| +2450815|9590|4|590| +2450815|9592|4|448| +2450815|9595|4|183| +2450815|9596|4|431| +2450815|9598|4|222| +2450815|9601|4|900| +2450815|9602|4|631| +2450815|9604|4|629| +2450815|9607|4|802| +2450815|9608|4|239| +2450815|9610|4|119| +2450815|9613|4|| +2450815|9614|4|43| +2450815|9616|4|422| +2450815|9619|4|98| +2450815|9620|4|212| +2450815|9622|4|180| +2450815|9625|4|827| +2450815|9626|4|251| +2450815|9628|4|54| +2450815|9631|4|49| +2450815|9632|4|765| +2450815|9634|4|95| +2450815|9637|4|318| +2450815|9638|4|931| +2450815|9640|4|124| +2450815|9643|4|667| +2450815|9644|4|137| +2450815|9646|4|881| +2450815|9649|4|696| +2450815|9650|4|615| +2450815|9652|4|778| +2450815|9655|4|646| +2450815|9656|4|182| +2450815|9658|4|939| +2450815|9661|4|59| +2450815|9662|4|894| +2450815|9664|4|422| +2450815|9667|4|836| +2450815|9668|4|927| +2450815|9670|4|89| +2450815|9673|4|218| +2450815|9674|4|| +2450815|9676|4|759| +2450815|9679|4|649| +2450815|9680|4|844| +2450815|9682|4|715| +2450815|9685|4|268| +2450815|9686|4|55| +2450815|9688|4|23| +2450815|9691|4|862| +2450815|9692|4|4| +2450815|9694|4|614| +2450815|9697|4|889| +2450815|9698|4|992| +2450815|9700|4|693| +2450815|9703|4|796| +2450815|9704|4|183| +2450815|9706|4|645| +2450815|9709|4|338| +2450815|9710|4|902| +2450815|9712|4|675| +2450815|9715|4|320| +2450815|9716|4|929| +2450815|9718|4|382| +2450815|9721|4|176| +2450815|9722|4|434| +2450815|9724|4|623| +2450815|9727|4|428| +2450815|9728|4|684| +2450815|9730|4|48| +2450815|9733|4|376| +2450815|9734|4|689| +2450815|9736|4|816| +2450815|9739|4|572| +2450815|9740|4|800| +2450815|9742|4|914| +2450815|9745|4|78| +2450815|9746|4|830| +2450815|9748|4|| +2450815|9751|4|149| +2450815|9752|4|625| +2450815|9754|4|776| +2450815|9757|4|771| +2450815|9758|4|51| +2450815|9760|4|980| +2450815|9763|4|937| +2450815|9764|4|549| +2450815|9766|4|288| +2450815|9769|4|362| +2450815|9770|4|544| +2450815|9772|4|280| +2450815|9775|4|328| +2450815|9776|4|688| +2450815|9778|4|220| +2450815|9781|4|421| +2450815|9782|4|625| +2450815|9784|4|702| +2450815|9787|4|112| +2450815|9788|4|9| +2450815|9790|4|554| +2450815|9793|4|| +2450815|9794|4|117| +2450815|9796|4|675| +2450815|9799|4|254| +2450815|9800|4|220| +2450815|9802|4|328| +2450815|9805|4|406| +2450815|9806|4|54| +2450815|9808|4|543| +2450815|9811|4|279| +2450815|9812|4|324| +2450815|9814|4|226| +2450815|9817|4|92| +2450815|9818|4|572| +2450815|9820|4|933| +2450815|9823|4|562| +2450815|9824|4|674| +2450815|9826|4|475| +2450815|9829|4|813| +2450815|9830|4|325| +2450815|9832|4|949| +2450815|9835|4|593| +2450815|9836|4|528| +2450815|9838|4|401| +2450815|9841|4|990| +2450815|9842|4|747| +2450815|9844|4|122| +2450815|9847|4|544| +2450815|9848|4|768| +2450815|9850|4|40| +2450815|9853|4|767| +2450815|9854|4|274| +2450815|9856|4|| +2450815|9859|4|| +2450815|9860|4|472| +2450815|9862|4|| +2450815|9865|4|241| +2450815|9866|4|807| +2450815|9868|4|282| +2450815|9871|4|326| +2450815|9872|4|470| +2450815|9874|4|165| +2450815|9877|4|846| +2450815|9878|4|814| +2450815|9880|4|169| +2450815|9883|4|326| +2450815|9884|4|606| +2450815|9886|4|210| +2450815|9889|4|374| +2450815|9890|4|806| +2450815|9892|4|723| +2450815|9895|4|648| +2450815|9896|4|411| +2450815|9898|4|431| +2450815|9901|4|213| +2450815|9902|4|869| +2450815|9904|4|155| +2450815|9907|4|35| +2450815|9908|4|| +2450815|9910|4|36| +2450815|9913|4|479| +2450815|9914|4|889| +2450815|9916|4|535| +2450815|9919|4|233| +2450815|9920|4|531| +2450815|9922|4|614| +2450815|9925|4|638| +2450815|9926|4|147| +2450815|9928|4|715| +2450815|9931|4|459| +2450815|9932|4|40| +2450815|9934|4|653| +2450815|9937|4|378| +2450815|9938|4|91| +2450815|9940|4|829| +2450815|9943|4|483| +2450815|9944|4|985| +2450815|9946|4|569| +2450815|9949|4|276| +2450815|9950|4|446| +2450815|9952|4|135| +2450815|9955|4|241| +2450815|9956|4|153| +2450815|9958|4|867| +2450815|9961|4|150| +2450815|9962|4|504| +2450815|9964|4|787| +2450815|9967|4|554| +2450815|9968|4|6| +2450815|9970|4|451| +2450815|9973|4|172| +2450815|9974|4|266| +2450815|9976|4|873| +2450815|9979|4|155| +2450815|9980|4|404| +2450815|9982|4|202| +2450815|9985|4|107| +2450815|9986|4|921| +2450815|9988|4|664| +2450815|9991|4|77| +2450815|9992|4|246| +2450815|9994|4|998| +2450815|9997|4|506| +2450815|9998|4|71| +2450815|10000|4|474| +2450815|10003|4|| +2450815|10004|4|467| +2450815|10006|4|412| +2450815|10009|4|809| +2450815|10010|4|869| +2450815|10012|4|841| +2450815|10015|4|404| +2450815|10016|4|370| +2450815|10018|4|627| +2450815|10021|4|382| +2450815|10022|4|40| +2450815|10024|4|92| +2450815|10027|4|641| +2450815|10028|4|881| +2450815|10030|4|822| +2450815|10033|4|382| +2450815|10034|4|125| +2450815|10036|4|1| +2450815|10039|4|17| +2450815|10040|4|11| +2450815|10042|4|795| +2450815|10045|4|828| +2450815|10046|4|92| +2450815|10048|4|259| +2450815|10051|4|143| +2450815|10052|4|195| +2450815|10054|4|473| +2450815|10057|4|203| +2450815|10058|4|805| +2450815|10060|4|662| +2450815|10063|4|158| +2450815|10064|4|| +2450815|10066|4|953| +2450815|10069|4|321| +2450815|10070|4|| +2450815|10072|4|140| +2450815|10075|4|895| +2450815|10076|4|656| +2450815|10078|4|893| +2450815|10081|4|927| +2450815|10082|4|72| +2450815|10084|4|| +2450815|10087|4|956| +2450815|10088|4|459| +2450815|10090|4|889| +2450815|10093|4|181| +2450815|10094|4|217| +2450815|10096|4|566| +2450815|10099|4|434| +2450815|10100|4|423| +2450815|10102|4|649| +2450815|10105|4|743| +2450815|10106|4|241| +2450815|10108|4|723| +2450815|10111|4|91| +2450815|10112|4|986| +2450815|10114|4|872| +2450815|10117|4|945| +2450815|10118|4|935| +2450815|10120|4|616| +2450815|10123|4|856| +2450815|10124|4|633| +2450815|10126|4|903| +2450815|10129|4|690| +2450815|10130|4|378| +2450815|10132|4|163| +2450815|10135|4|868| +2450815|10136|4|367| +2450815|10138|4|254| +2450815|10141|4|299| +2450815|10142|4|284| +2450815|10144|4|757| +2450815|10147|4|551| +2450815|10148|4|| +2450815|10150|4|373| +2450815|10153|4|414| +2450815|10154|4|287| +2450815|10156|4|594| +2450815|10159|4|685| +2450815|10160|4|| +2450815|10162|4|173| +2450815|10165|4|259| +2450815|10166|4|980| +2450815|10168|4|554| +2450815|10171|4|212| +2450815|10172|4|484| +2450815|10174|4|295| +2450815|10177|4|522| +2450815|10178|4|208| +2450815|10180|4|15| +2450815|10183|4|384| +2450815|10184|4|99| +2450815|10186|4|626| +2450815|10189|4|54| +2450815|10190|4|630| +2450815|10192|4|| +2450815|10195|4|177| +2450815|10196|4|103| +2450815|10198|4|0| +2450815|10201|4|819| +2450815|10202|4|86| +2450815|10204|4|166| +2450815|10207|4|719| +2450815|10208|4|395| +2450815|10210|4|143| +2450815|10213|4|649| +2450815|10214|4|57| +2450815|10216|4|385| +2450815|10219|4|970| +2450815|10220|4|571| +2450815|10222|4|693| +2450815|10225|4|828| +2450815|10226|4|655| +2450815|10228|4|206| +2450815|10231|4|712| +2450815|10232|4|809| +2450815|10234|4|955| +2450815|10237|4|514| +2450815|10238|4|771| +2450815|10240|4|386| +2450815|10243|4|357| +2450815|10244|4|767| +2450815|10246|4|984| +2450815|10249|4|826| +2450815|10250|4|600| +2450815|10252|4|844| +2450815|10255|4|277| +2450815|10256|4|967| +2450815|10258|4|936| +2450815|10261|4|154| +2450815|10262|4|811| +2450815|10264|4|609| +2450815|10267|4|880| +2450815|10268|4|442| +2450815|10270|4|519| +2450815|10273|4|| +2450815|10274|4|129| +2450815|10276|4|61| +2450815|10279|4|330| +2450815|10280|4|31| +2450815|10282|4|921| +2450815|10285|4|737| +2450815|10286|4|683| +2450815|10288|4|946| +2450815|10291|4|890| +2450815|10292|4|406| +2450815|10294|4|151| +2450815|10297|4|652| +2450815|10298|4|233| +2450815|10300|4|859| +2450815|10303|4|551| +2450815|10304|4|174| +2450815|10306|4|959| +2450815|10309|4|354| +2450815|10310|4|579| +2450815|10312|4|149| +2450815|10315|4|397| +2450815|10316|4|871| +2450815|10318|4|273| +2450815|10321|4|527| +2450815|10322|4|799| +2450815|10324|4|| +2450815|10327|4|125| +2450815|10328|4|333| +2450815|10330|4|170| +2450815|10333|4|175| +2450815|10334|4|968| +2450815|10336|4|607| +2450815|10339|4|139| +2450815|10340|4|116| +2450815|10342|4|412| +2450815|10345|4|530| +2450815|10346|4|219| +2450815|10348|4|12| +2450815|10351|4|53| +2450815|10352|4|378| +2450815|10354|4|973| +2450815|10357|4|695| +2450815|10358|4|643| +2450815|10360|4|729| +2450815|10363|4|482| +2450815|10364|4|| +2450815|10366|4|906| +2450815|10369|4|654| +2450815|10370|4|201| +2450815|10372|4|750| +2450815|10375|4|220| +2450815|10376|4|156| +2450815|10378|4|785| +2450815|10381|4|675| +2450815|10382|4|429| +2450815|10384|4|723| +2450815|10387|4|642| +2450815|10388|4|862| +2450815|10390|4|192| +2450815|10393|4|712| +2450815|10394|4|172| +2450815|10396|4|613| +2450815|10399|4|120| +2450815|10400|4|756| +2450815|10402|4|723| +2450815|10405|4|894| +2450815|10406|4|674| +2450815|10408|4|721| +2450815|10411|4|877| +2450815|10412|4|906| +2450815|10414|4|623| +2450815|10417|4|709| +2450815|10418|4|125| +2450815|10420|4|294| +2450815|10423|4|645| +2450815|10424|4|730| +2450815|10426|4|366| +2450815|10429|4|724| +2450815|10430|4|914| +2450815|10432|4|526| +2450815|10435|4|370| +2450815|10436|4|886| +2450815|10438|4|958| +2450815|10441|4|880| +2450815|10442|4|433| +2450815|10444|4|489| +2450815|10447|4|433| +2450815|10448|4|273| +2450815|10450|4|357| +2450815|10453|4|354| +2450815|10454|4|351| +2450815|10456|4|831| +2450815|10459|4|| +2450815|10460|4|450| +2450815|10462|4|966| +2450815|10465|4|| +2450815|10466|4|544| +2450815|10468|4|988| +2450815|10471|4|952| +2450815|10472|4|245| +2450815|10474|4|329| +2450815|10477|4|327| +2450815|10478|4|905| +2450815|10480|4|687| +2450815|10483|4|892| +2450815|10484|4|589| +2450815|10486|4|553| +2450815|10489|4|231| +2450815|10490|4|368| +2450815|10492|4|464| +2450815|10495|4|25| +2450815|10496|4|129| +2450815|10498|4|601| +2450815|10501|4|463| +2450815|10502|4|263| +2450815|10504|4|406| +2450815|10507|4|561| +2450815|10508|4|388| +2450815|10510|4|595| +2450815|10513|4|506| +2450815|10514|4|864| +2450815|10516|4|291| +2450815|10519|4|471| +2450815|10520|4|557| +2450815|10522|4|549| +2450815|10525|4|25| +2450815|10526|4|56| +2450815|10528|4|| +2450815|10531|4|306| +2450815|10532|4|97| +2450815|10534|4|656| +2450815|10537|4|279| +2450815|10538|4|212| +2450815|10540|4|698| +2450815|10543|4|327| +2450815|10544|4|488| +2450815|10546|4|930| +2450815|10549|4|845| +2450815|10550|4|| +2450815|10552|4|249| +2450815|10555|4|94| +2450815|10556|4|192| +2450815|10558|4|858| +2450815|10561|4|785| +2450815|10562|4|3| +2450815|10564|4|106| +2450815|10567|4|519| +2450815|10568|4|709| +2450815|10570|4|262| +2450815|10573|4|557| +2450815|10574|4|58| +2450815|10576|4|617| +2450815|10579|4|864| +2450815|10580|4|147| +2450815|10582|4|327| +2450815|10585|4|| +2450815|10586|4|979| +2450815|10588|4|825| +2450815|10591|4|777| +2450815|10592|4|105| +2450815|10594|4|514| +2450815|10597|4|141| +2450815|10598|4|786| +2450815|10600|4|| +2450815|10603|4|| +2450815|10604|4|439| +2450815|10606|4|713| +2450815|10609|4|207| +2450815|10610|4|605| +2450815|10612|4|882| +2450815|10615|4|14| +2450815|10616|4|12| +2450815|10618|4|29| +2450815|10621|4|| +2450815|10622|4|856| +2450815|10624|4|178| +2450815|10627|4|102| +2450815|10628|4|817| +2450815|10630|4|381| +2450815|10633|4|989| +2450815|10634|4|45| +2450815|10636|4|472| +2450815|10639|4|215| +2450815|10640|4|69| +2450815|10642|4|207| +2450815|10645|4|| +2450815|10646|4|912| +2450815|10648|4|603| +2450815|10651|4|94| +2450815|10652|4|567| +2450815|10654|4|866| +2450815|10657|4|163| +2450815|10658|4|773| +2450815|10660|4|74| +2450815|10663|4|708| +2450815|10664|4|702| +2450815|10666|4|234| +2450815|10669|4|162| +2450815|10670|4|827| +2450815|10672|4|419| +2450815|10675|4|873| +2450815|10676|4|620| +2450815|10678|4|30| +2450815|10681|4|686| +2450815|10682|4|477| +2450815|10684|4|464| +2450815|10687|4|220| +2450815|10688|4|150| +2450815|10690|4|| +2450815|10693|4|| +2450815|10694|4|385| +2450815|10696|4|216| +2450815|10699|4|882| +2450815|10700|4|474| +2450815|10702|4|762| +2450815|10705|4|441| +2450815|10706|4|508| +2450815|10708|4|949| +2450815|10711|4|483| +2450815|10712|4|527| +2450815|10714|4|898| +2450815|10717|4|730| +2450815|10718|4|131| +2450815|10720|4|476| +2450815|10723|4|317| +2450815|10724|4|389| +2450815|10726|4|676| +2450815|10729|4|414| +2450815|10730|4|662| +2450815|10732|4|326| +2450815|10735|4|395| +2450815|10736|4|155| +2450815|10738|4|909| +2450815|10741|4|370| +2450815|10742|4|798| +2450815|10744|4|932| +2450815|10747|4|123| +2450815|10748|4|| +2450815|10750|4|346| +2450815|10753|4|788| +2450815|10754|4|969| +2450815|10756|4|333| +2450815|10759|4|912| +2450815|10760|4|947| +2450815|10762|4|854| +2450815|10765|4|796| +2450815|10766|4|939| +2450815|10768|4|448| +2450815|10771|4|50| +2450815|10772|4|765| +2450815|10774|4|229| +2450815|10777|4|868| +2450815|10778|4|763| +2450815|10780|4|885| +2450815|10783|4|305| +2450815|10784|4|751| +2450815|10786|4|542| +2450815|10789|4|404| +2450815|10790|4|8| +2450815|10792|4|112| +2450815|10795|4|565| +2450815|10796|4|156| +2450815|10798|4|205| +2450815|10801|4|568| +2450815|10802|4|897| +2450815|10804|4|247| +2450815|10807|4|713| +2450815|10808|4|200| +2450815|10810|4|59| +2450815|10813|4|335| +2450815|10814|4|242| +2450815|10816|4|139| +2450815|10819|4|575| +2450815|10820|4|976| +2450815|10822|4|654| +2450815|10825|4|162| +2450815|10826|4|861| +2450815|10828|4|738| +2450815|10831|4|538| +2450815|10832|4|924| +2450815|10834|4|| +2450815|10837|4|256| +2450815|10838|4|138| +2450815|10840|4|584| +2450815|10843|4|377| +2450815|10844|4|519| +2450815|10846|4|810| +2450815|10849|4|112| +2450815|10850|4|202| +2450815|10852|4|166| +2450815|10855|4|401| +2450815|10856|4|612| +2450815|10858|4|419| +2450815|10861|4|71| +2450815|10862|4|477| +2450815|10864|4|885| +2450815|10867|4|729| +2450815|10868|4|34| +2450815|10870|4|506| +2450815|10873|4|676| +2450815|10874|4|33| +2450815|10876|4|194| +2450815|10879|4|167| +2450815|10880|4|513| +2450815|10882|4|363| +2450815|10885|4|258| +2450815|10886|4|308| +2450815|10888|4|586| +2450815|10891|4|800| +2450815|10892|4|988| +2450815|10894|4|823| +2450815|10897|4|267| +2450815|10898|4|| +2450815|10900|4|531| +2450815|10903|4|31| +2450815|10904|4|976| +2450815|10906|4|876| +2450815|10909|4|290| +2450815|10910|4|110| +2450815|10912|4|539| +2450815|10915|4|989| +2450815|10916|4|593| +2450815|10918|4|417| +2450815|10921|4|639| +2450815|10922|4|207| +2450815|10924|4|174| +2450815|10927|4|776| +2450815|10928|4|| +2450815|10930|4|331| +2450815|10933|4|152| +2450815|10934|4|755| +2450815|10936|4|587| +2450815|10939|4|25| +2450815|10940|4|86| +2450815|10942|4|666| +2450815|10945|4|895| +2450815|10946|4|250| +2450815|10948|4|507| +2450815|10951|4|17| +2450815|10952|4|210| +2450815|10954|4|774| +2450815|10957|4|598| +2450815|10958|4|553| +2450815|10960|4|941| +2450815|10963|4|576| +2450815|10964|4|776| +2450815|10966|4|| +2450815|10969|4|686| +2450815|10970|4|910| +2450815|10972|4|775| +2450815|10975|4|412| +2450815|10976|4|791| +2450815|10978|4|932| +2450815|10981|4|436| +2450815|10982|4|219| +2450815|10984|4|716| +2450815|10987|4|484| +2450815|10988|4|649| +2450815|10990|4|138| +2450815|10993|4|210| +2450815|10994|4|418| +2450815|10996|4|997| +2450815|10999|4|471| +2450815|11000|4|302| +2450815|11002|4|402| +2450815|11005|4|236| +2450815|11006|4|621| +2450815|11008|4|845| +2450815|11011|4|618| +2450815|11012|4|116| +2450815|11014|4|26| +2450815|11017|4|239| +2450815|11018|4|767| +2450815|11020|4|158| +2450815|11023|4|392| +2450815|11024|4|660| +2450815|11026|4|258| +2450815|11029|4|675| +2450815|11030|4|618| +2450815|11032|4|451| +2450815|11035|4|22| +2450815|11036|4|26| +2450815|11038|4|20| +2450815|11041|4|449| +2450815|11042|4|552| +2450815|11044|4|519| +2450815|11047|4|482| +2450815|11048|4|207| +2450815|11050|4|689| +2450815|11053|4|71| +2450815|11054|4|247| +2450815|11056|4|756| +2450815|11059|4|149| +2450815|11060|4|219| +2450815|11062|4|114| +2450815|11065|4|189| +2450815|11066|4|325| +2450815|11068|4|547| +2450815|11071|4|414| +2450815|11072|4|114| +2450815|11074|4|557| +2450815|11077|4|495| +2450815|11078|4|845| +2450815|11080|4|728| +2450815|11083|4|312| +2450815|11084|4|559| +2450815|11086|4|877| +2450815|11089|4|177| +2450815|11090|4|569| +2450815|11092|4|522| +2450815|11095|4|686| +2450815|11096|4|717| +2450815|11098|4|219| +2450815|11101|4|949| +2450815|11102|4|805| +2450815|11104|4|782| +2450815|11107|4|144| +2450815|11108|4|339| +2450815|11110|4|988| +2450815|11113|4|657| +2450815|11114|4|810| +2450815|11116|4|90| +2450815|11119|4|168| +2450815|11120|4|285| +2450815|11122|4|| +2450815|11125|4|506| +2450815|11126|4|20| +2450815|11128|4|126| +2450815|11131|4|24| +2450815|11132|4|228| +2450815|11134|4|177| +2450815|11137|4|312| +2450815|11138|4|826| +2450815|11140|4|302| +2450815|11143|4|66| +2450815|11144|4|9| +2450815|11146|4|869| +2450815|11149|4|610| +2450815|11150|4|422| +2450815|11152|4|434| +2450815|11155|4|570| +2450815|11156|4|231| +2450815|11158|4|995| +2450815|11161|4|596| +2450815|11162|4|924| +2450815|11164|4|692| +2450815|11167|4|406| +2450815|11168|4|308| +2450815|11170|4|8| +2450815|11173|4|53| +2450815|11174|4|727| +2450815|11176|4|896| +2450815|11179|4|312| +2450815|11180|4|109| +2450815|11182|4|33| +2450815|11185|4|166| +2450815|11186|4|| +2450815|11188|4|300| +2450815|11191|4|488| +2450815|11192|4|25| +2450815|11194|4|670| +2450815|11197|4|302| +2450815|11198|4|| +2450815|11200|4|832| +2450815|11203|4|114| +2450815|11204|4|334| +2450815|11206|4|590| +2450815|11209|4|425| +2450815|11210|4|511| +2450815|11212|4|859| +2450815|11215|4|958| +2450815|11216|4|186| +2450815|11218|4|61| +2450815|11221|4|716| +2450815|11222|4|465| +2450815|11224|4|660| +2450815|11227|4|578| +2450815|11228|4|967| +2450815|11230|4|502| +2450815|11233|4|127| +2450815|11234|4|894| +2450815|11236|4|903| +2450815|11239|4|693| +2450815|11240|4|58| +2450815|11242|4|| +2450815|11245|4|987| +2450815|11246|4|478| +2450815|11248|4|871| +2450815|11251|4|145| +2450815|11252|4|875| +2450815|11254|4|437| +2450815|11257|4|729| +2450815|11258|4|204| +2450815|11260|4|352| +2450815|11263|4|346| +2450815|11264|4|614| +2450815|11266|4|714| +2450815|11269|4|743| +2450815|11270|4|531| +2450815|11272|4|466| +2450815|11275|4|191| +2450815|11276|4|675| +2450815|11278|4|56| +2450815|11281|4|399| +2450815|11282|4|| +2450815|11284|4|321| +2450815|11287|4|168| +2450815|11288|4|202| +2450815|11290|4|355| +2450815|11293|4|967| +2450815|11294|4|933| +2450815|11296|4|933| +2450815|11299|4|180| +2450815|11300|4|360| +2450815|11302|4|69| +2450815|11305|4|831| +2450815|11306|4|439| +2450815|11308|4|696| +2450815|11311|4|989| +2450815|11312|4|689| +2450815|11314|4|351| +2450815|11317|4|444| +2450815|11318|4|589| +2450815|11320|4|101| +2450815|11323|4|639| +2450815|11324|4|376| +2450815|11326|4|677| +2450815|11329|4|343| +2450815|11330|4|720| +2450815|11332|4|62| +2450815|11335|4|697| +2450815|11336|4|581| +2450815|11338|4|76| +2450815|11341|4|114| +2450815|11342|4|159| +2450815|11344|4|97| +2450815|11347|4|176| +2450815|11348|4|645| +2450815|11350|4|720| +2450815|11353|4|177| +2450815|11354|4|477| +2450815|11356|4|556| +2450815|11359|4|358| +2450815|11360|4|166| +2450815|11362|4|159| +2450815|11365|4|96| +2450815|11366|4|196| +2450815|11368|4|388| +2450815|11371|4|806| +2450815|11372|4|| +2450815|11374|4|322| +2450815|11377|4|370| +2450815|11378|4|440| +2450815|11380|4|622| +2450815|11383|4|177| +2450815|11384|4|504| +2450815|11386|4|411| +2450815|11389|4|632| +2450815|11390|4|865| +2450815|11392|4|183| +2450815|11395|4|452| +2450815|11396|4|281| +2450815|11398|4|562| +2450815|11401|4|505| +2450815|11402|4|261| +2450815|11404|4|530| +2450815|11407|4|93| +2450815|11408|4|488| +2450815|11410|4|782| +2450815|11413|4|515| +2450815|11414|4|121| +2450815|11416|4|539| +2450815|11419|4|954| +2450815|11420|4|522| +2450815|11422|4|734| +2450815|11425|4|372| +2450815|11426|4|822| +2450815|11428|4|102| +2450815|11431|4|188| +2450815|11432|4|615| +2450815|11434|4|650| +2450815|11437|4|647| +2450815|11438|4|792| +2450815|11440|4|983| +2450815|11443|4|172| +2450815|11444|4|362| +2450815|11446|4|160| +2450815|11449|4|316| +2450815|11450|4|686| +2450815|11452|4|523| +2450815|11455|4|450| +2450815|11456|4|670| +2450815|11458|4|832| +2450815|11461|4|521| +2450815|11462|4|41| +2450815|11464|4|454| +2450815|11467|4|3| +2450815|11468|4|848| +2450815|11470|4|351| +2450815|11473|4|50| +2450815|11474|4|4| +2450815|11476|4|77| +2450815|11479|4|573| +2450815|11480|4|646| +2450815|11482|4|504| +2450815|11485|4|309| +2450815|11486|4|49| +2450815|11488|4|245| +2450815|11491|4|301| +2450815|11492|4|763| +2450815|11494|4|599| +2450815|11497|4|501| +2450815|11498|4|407| +2450815|11500|4|199| +2450815|11503|4|113| +2450815|11504|4|873| +2450815|11506|4|689| +2450815|11509|4|361| +2450815|11510|4|910| +2450815|11512|4|696| +2450815|11515|4|653| +2450815|11516|4|503| +2450815|11518|4|308| +2450815|11521|4|748| +2450815|11522|4|222| +2450815|11524|4|454| +2450815|11527|4|| +2450815|11528|4|270| +2450815|11530|4|668| +2450815|11533|4|891| +2450815|11534|4|544| +2450815|11536|4|593| +2450815|11539|4|528| +2450815|11540|4|952| +2450815|11542|4|313| +2450815|11545|4|386| +2450815|11546|4|90| +2450815|11548|4|573| +2450815|11551|4|44| +2450815|11552|4|126| +2450815|11554|4|222| +2450815|11557|4|560| +2450815|11558|4|356| +2450815|11560|4|54| +2450815|11563|4|542| +2450815|11564|4|649| +2450815|11566|4|62| +2450815|11569|4|113| +2450815|11570|4|715| +2450815|11572|4|980| +2450815|11575|4|820| +2450815|11576|4|210| +2450815|11578|4|618| +2450815|11581|4|392| +2450815|11582|4|324| +2450815|11584|4|607| +2450815|11587|4|452| +2450815|11588|4|460| +2450815|11590|4|771| +2450815|11593|4|688| +2450815|11594|4|547| +2450815|11596|4|28| +2450815|11599|4|319| +2450815|11600|4|138| +2450815|11602|4|797| +2450815|11605|4|515| +2450815|11606|4|624| +2450815|11608|4|394| +2450815|11611|4|110| +2450815|11612|4|408| +2450815|11614|4|408| +2450815|11617|4|640| +2450815|11618|4|812| +2450815|11620|4|844| +2450815|11623|4|84| +2450815|11624|4|15| +2450815|11626|4|450| +2450815|11629|4|219| +2450815|11630|4|376| +2450815|11632|4|263| +2450815|11635|4|804| +2450815|11636|4|763| +2450815|11638|4|782| +2450815|11641|4|525| +2450815|11642|4|121| +2450815|11644|4|168| +2450815|11647|4|487| +2450815|11648|4|664| +2450815|11650|4|633| +2450815|11653|4|104| +2450815|11654|4|82| +2450815|11656|4|198| +2450815|11659|4|491| +2450815|11660|4|439| +2450815|11662|4|233| +2450815|11665|4|261| +2450815|11666|4|254| +2450815|11668|4|33| +2450815|11671|4|769| +2450815|11672|4|430| +2450815|11674|4|386| +2450815|11677|4|| +2450815|11678|4|501| +2450815|11680|4|479| +2450815|11683|4|938| +2450815|11684|4|450| +2450815|11686|4|797| +2450815|11689|4|896| +2450815|11690|4|37| +2450815|11692|4|324| +2450815|11695|4|671| +2450815|11696|4|963| +2450815|11698|4|841| +2450815|11701|4|991| +2450815|11702|4|| +2450815|11704|4|852| +2450815|11707|4|386| +2450815|11708|4|917| +2450815|11710|4|734| +2450815|11713|4|788| +2450815|11714|4|706| +2450815|11716|4|535| +2450815|11719|4|362| +2450815|11720|4|752| +2450815|11722|4|96| +2450815|11725|4|845| +2450815|11726|4|95| +2450815|11728|4|339| +2450815|11731|4|304| +2450815|11732|4|676| +2450815|11734|4|571| +2450815|11737|4|741| +2450815|11738|4|196| +2450815|11740|4|332| +2450815|11743|4|651| +2450815|11744|4|91| +2450815|11746|4|216| +2450815|11749|4|623| +2450815|11750|4|20| +2450815|11752|4|679| +2450815|11755|4|951| +2450815|11756|4|488| +2450815|11758|4|997| +2450815|11761|4|153| +2450815|11762|4|790| +2450815|11764|4|829| +2450815|11767|4|743| +2450815|11768|4|250| +2450815|11770|4|576| +2450815|11773|4|510| +2450815|11774|4|510| +2450815|11776|4|175| +2450815|11779|4|857| +2450815|11780|4|282| +2450815|11782|4|514| +2450815|11785|4|418| +2450815|11786|4|920| +2450815|11788|4|420| +2450815|11791|4|863| +2450815|11792|4|338| +2450815|11794|4|| +2450815|11797|4|26| +2450815|11798|4|675| +2450815|11800|4|381| +2450815|11803|4|546| +2450815|11804|4|879| +2450815|11806|4|162| +2450815|11809|4|| +2450815|11810|4|847| +2450815|11812|4|775| +2450815|11815|4|588| +2450815|11816|4|892| +2450815|11818|4|60| +2450815|11821|4|110| +2450815|11822|4|74| +2450815|11824|4|| +2450815|11827|4|| +2450815|11828|4|826| +2450815|11830|4|803| +2450815|11833|4|68| +2450815|11834|4|493| +2450815|11836|4|616| +2450815|11839|4|973| +2450815|11840|4|0| +2450815|11842|4|427| +2450815|11845|4|754| +2450815|11846|4|44| +2450815|11848|4|834| +2450815|11851|4|128| +2450815|11852|4|272| +2450815|11854|4|426| +2450815|11857|4|949| +2450815|11858|4|| +2450815|11860|4|9| +2450815|11863|4|| +2450815|11864|4|| +2450815|11866|4|420| +2450815|11869|4|112| +2450815|11870|4|88| +2450815|11872|4|164| +2450815|11875|4|476| +2450815|11876|4|549| +2450815|11878|4|777| +2450815|11881|4|134| +2450815|11882|4|251| +2450815|11884|4|400| +2450815|11887|4|458| +2450815|11888|4|377| +2450815|11890|4|197| +2450815|11893|4|594| +2450815|11894|4|679| +2450815|11896|4|934| +2450815|11899|4|870| +2450815|11900|4|478| +2450815|11902|4|198| +2450815|11905|4|93| +2450815|11906|4|435| +2450815|11908|4|569| +2450815|11911|4|764| +2450815|11912|4|177| +2450815|11914|4|587| +2450815|11917|4|928| +2450815|11918|4|892| +2450815|11920|4|107| +2450815|11923|4|| +2450815|11924|4|848| +2450815|11926|4|373| +2450815|11929|4|708| +2450815|11930|4|216| +2450815|11932|4|643| +2450815|11935|4|432| +2450815|11936|4|628| +2450815|11938|4|323| +2450815|11941|4|347| +2450815|11942|4|297| +2450815|11944|4|646| +2450815|11947|4|357| +2450815|11948|4|780| +2450815|11950|4|477| +2450815|11953|4|56| +2450815|11954|4|146| +2450815|11956|4|710| +2450815|11959|4|550| +2450815|11960|4|25| +2450815|11962|4|60| +2450815|11965|4|500| +2450815|11966|4|415| +2450815|11968|4|371| +2450815|11971|4|416| +2450815|11972|4|174| +2450815|11974|4|858| +2450815|11977|4|615| +2450815|11978|4|357| +2450815|11980|4|880| +2450815|11983|4|354| +2450815|11984|4|870| +2450815|11986|4|998| +2450815|11989|4|492| +2450815|11990|4|730| +2450815|11992|4|402| +2450815|11995|4|345| +2450815|11996|4|839| +2450815|11998|4|884| +2450815|12001|4|621| +2450815|12002|4|| +2450815|12004|4|63| +2450815|12007|4|643| +2450815|12008|4|100| +2450815|12010|4|426| +2450815|12013|4|969| +2450815|12014|4|406| +2450815|12016|4|384| +2450815|12019|4|248| +2450815|12020|4|575| +2450815|12022|4|163| +2450815|12025|4|874| +2450815|12026|4|69| +2450815|12028|4|414| +2450815|12031|4|903| +2450815|12032|4|102| +2450815|12034|4|493| +2450815|12037|4|87| +2450815|12038|4|124| +2450815|12040|4|570| +2450815|12043|4|125| +2450815|12044|4|340| +2450815|12046|4|744| +2450815|12049|4|522| +2450815|12050|4|895| +2450815|12052|4|927| +2450815|12055|4|579| +2450815|12056|4|82| +2450815|12058|4|729| +2450815|12061|4|408| +2450815|12062|4|34| +2450815|12064|4|390| +2450815|12067|4|827| +2450815|12068|4|101| +2450815|12070|4|939| +2450815|12073|4|573| +2450815|12074|4|66| +2450815|12076|4|227| +2450815|12079|4|832| +2450815|12080|4|159| +2450815|12082|4|260| +2450815|12085|4|853| +2450815|12086|4|959| +2450815|12088|4|| +2450815|12091|4|210| +2450815|12092|4|603| +2450815|12094|4|772| +2450815|12097|4|759| +2450815|12098|4|747| +2450815|12100|4|879| +2450815|12103|4|350| +2450815|12104|4|925| +2450815|12106|4|583| +2450815|12109|4|368| +2450815|12110|4|868| +2450815|12112|4|942| +2450815|12115|4|101| +2450815|12116|4|663| +2450815|12118|4|870| +2450815|12121|4|989| +2450815|12122|4|562| +2450815|12124|4|645| +2450815|12127|4|400| +2450815|12128|4|116| +2450815|12130|4|814| +2450815|12133|4|236| +2450815|12134|4|88| +2450815|12136|4|861| +2450815|12139|4|979| +2450815|12140|4|990| +2450815|12142|4|908| +2450815|12145|4|956| +2450815|12146|4|206| +2450815|12148|4|881| +2450815|12151|4|251| +2450815|12152|4|893| +2450815|12154|4|861| +2450815|12157|4|86| +2450815|12158|4|525| +2450815|12160|4|260| +2450815|12163|4|| +2450815|12164|4|616| +2450815|12166|4|128| +2450815|12169|4|786| +2450815|12170|4|135| +2450815|12172|4|215| +2450815|12175|4|| +2450815|12176|4|991| +2450815|12178|4|786| +2450815|12181|4|955| +2450815|12182|4|360| +2450815|12184|4|370| +2450815|12187|4|83| +2450815|12188|4|516| +2450815|12190|4|818| +2450815|12193|4|784| +2450815|12194|4|408| +2450815|12196|4|330| +2450815|12199|4|608| +2450815|12200|4|566| +2450815|12202|4|92| +2450815|12205|4|392| +2450815|12206|4|| +2450815|12208|4|63| +2450815|12211|4|535| +2450815|12212|4|163| +2450815|12214|4|| +2450815|12217|4|746| +2450815|12218|4|933| +2450815|12220|4|957| +2450815|12223|4|| +2450815|12224|4|586| +2450815|12226|4|953| +2450815|12229|4|| +2450815|12230|4|839| +2450815|12232|4|632| +2450815|12235|4|381| +2450815|12236|4|848| +2450815|12238|4|410| +2450815|12241|4|835| +2450815|12242|4|942| +2450815|12244|4|206| +2450815|12247|4|499| +2450815|12248|4|435| +2450815|12250|4|699| +2450815|12253|4|274| +2450815|12254|4|597| +2450815|12256|4|796| +2450815|12259|4|92| +2450815|12260|4|412| +2450815|12262|4|187| +2450815|12265|4|114| +2450815|12266|4|607| +2450815|12268|4|933| +2450815|12271|4|223| +2450815|12272|4|771| +2450815|12274|4|229| +2450815|12277|4|674| +2450815|12278|4|788| +2450815|12280|4|990| +2450815|12283|4|241| +2450815|12284|4|709| +2450815|12286|4|484| +2450815|12289|4|312| +2450815|12290|4|589| +2450815|12292|4|| +2450815|12295|4|259| +2450815|12296|4|820| +2450815|12298|4|823| +2450815|12301|4|52| +2450815|12302|4|185| +2450815|12304|4|482| +2450815|12307|4|217| +2450815|12308|4|718| +2450815|12310|4|474| +2450815|12313|4|269| +2450815|12314|4|328| +2450815|12316|4|861| +2450815|12319|4|993| +2450815|12320|4|665| +2450815|12322|4|401| +2450815|12325|4|326| +2450815|12326|4|711| +2450815|12328|4|725| +2450815|12331|4|862| +2450815|12332|4|725| +2450815|12334|4|809| +2450815|12337|4|722| +2450815|12338|4|509| +2450815|12340|4|737| +2450815|12343|4|696| +2450815|12344|4|920| +2450815|12346|4|883| +2450815|12349|4|563| +2450815|12350|4|353| +2450815|12352|4|155| +2450815|12355|4|554| +2450815|12356|4|317| +2450815|12358|4|| +2450815|12361|4|371| +2450815|12362|4|841| +2450815|12364|4|552| +2450815|12367|4|209| +2450815|12368|4|299| +2450815|12370|4|151| +2450815|12373|4|525| +2450815|12374|4|934| +2450815|12376|4|631| +2450815|12379|4|310| +2450815|12380|4|538| +2450815|12382|4|377| +2450815|12385|4|47| +2450815|12386|4|606| +2450815|12388|4|652| +2450815|12391|4|| +2450815|12392|4|703| +2450815|12394|4|566| +2450815|12397|4|856| +2450815|12398|4|298| +2450815|12400|4|136| +2450815|12403|4|770| +2450815|12404|4|760| +2450815|12406|4|380| +2450815|12409|4|604| +2450815|12410|4|135| +2450815|12412|4|840| +2450815|12415|4|467| +2450815|12416|4|298| +2450815|12418|4|178| +2450815|12421|4|422| +2450815|12422|4|314| +2450815|12424|4|625| +2450815|12427|4|451| +2450815|12428|4|773| +2450815|12430|4|732| +2450815|12433|4|534| +2450815|12434|4|645| +2450815|12436|4|902| +2450815|12439|4|735| +2450815|12440|4|963| +2450815|12442|4|328| +2450815|12445|4|99| +2450815|12446|4|747| +2450815|12448|4|20| +2450815|12451|4|656| +2450815|12452|4|973| +2450815|12454|4|600| +2450815|12457|4|964| +2450815|12458|4|261| +2450815|12460|4|100| +2450815|12463|4|771| +2450815|12464|4|848| +2450815|12466|4|706| +2450815|12469|4|46| +2450815|12470|4|375| +2450815|12472|4|913| +2450815|12475|4|761| +2450815|12476|4|818| +2450815|12478|4|350| +2450815|12481|4|311| +2450815|12482|4|133| +2450815|12484|4|915| +2450815|12487|4|542| +2450815|12488|4|425| +2450815|12490|4|978| +2450815|12493|4|487| +2450815|12494|4|130| +2450815|12496|4|144| +2450815|12499|4|906| +2450815|12500|4|523| +2450815|12502|4|146| +2450815|12505|4|622| +2450815|12506|4|949| +2450815|12508|4|321| +2450815|12511|4|586| +2450815|12512|4|689| +2450815|12514|4|516| +2450815|12517|4|246| +2450815|12518|4|102| +2450815|12520|4|428| +2450815|12523|4|382| +2450815|12524|4|488| +2450815|12526|4|830| +2450815|12529|4|588| +2450815|12530|4|821| +2450815|12532|4|241| +2450815|12535|4|309| +2450815|12536|4|292| +2450815|12538|4|598| +2450815|12541|4|521| +2450815|12542|4|640| +2450815|12544|4|121| +2450815|12547|4|971| +2450815|12548|4|387| +2450815|12550|4|839| +2450815|12553|4|457| +2450815|12554|4|961| +2450815|12556|4|924| +2450815|12559|4|161| +2450815|12560|4|507| +2450815|12562|4|741| +2450815|12565|4|972| +2450815|12566|4|663| +2450815|12568|4|871| +2450815|12571|4|799| +2450815|12572|4|703| +2450815|12574|4|64| +2450815|12577|4|385| +2450815|12578|4|122| +2450815|12580|4|381| +2450815|12583|4|957| +2450815|12584|4|460| +2450815|12586|4|530| +2450815|12589|4|953| +2450815|12590|4|61| +2450815|12592|4|206| +2450815|12595|4|995| +2450815|12596|4|192| +2450815|12598|4|542| +2450815|12601|4|673| +2450815|12602|4|492| +2450815|12604|4|679| +2450815|12607|4|283| +2450815|12608|4|645| +2450815|12610|4|366| +2450815|12613|4|699| +2450815|12614|4|271| +2450815|12616|4|848| +2450815|12619|4|550| +2450815|12620|4|581| +2450815|12622|4|| +2450815|12625|4|385| +2450815|12626|4|800| +2450815|12628|4|939| +2450815|12631|4|659| +2450815|12632|4|606| +2450815|12634|4|| +2450815|12637|4|370| +2450815|12638|4|| +2450815|12640|4|841| +2450815|12643|4|315| +2450815|12644|4|86| +2450815|12646|4|391| +2450815|12649|4|192| +2450815|12650|4|933| +2450815|12652|4|552| +2450815|12655|4|689| +2450815|12656|4|358| +2450815|12658|4|998| +2450815|12661|4|182| +2450815|12662|4|99| +2450815|12664|4|742| +2450815|12667|4|409| +2450815|12668|4|848| +2450815|12670|4|77| +2450815|12673|4|603| +2450815|12674|4|211| +2450815|12676|4|339| +2450815|12679|4|567| +2450815|12680|4|566| +2450815|12682|4|746| +2450815|12685|4|51| +2450815|12686|4|650| +2450815|12688|4|84| +2450815|12691|4|36| +2450815|12692|4|146| +2450815|12694|4|398| +2450815|12697|4|536| +2450815|12698|4|67| +2450815|12700|4|668| +2450815|12703|4|36| +2450815|12704|4|17| +2450815|12706|4|52| +2450815|12709|4|6| +2450815|12710|4|605| +2450815|12712|4|44| +2450815|12715|4|895| +2450815|12716|4|201| +2450815|12718|4|5| +2450815|12721|4|759| +2450815|12722|4|307| +2450815|12724|4|400| +2450815|12727|4|986| +2450815|12728|4|903| +2450815|12730|4|481| +2450815|12733|4|11| +2450815|12734|4|917| +2450815|12736|4|758| +2450815|12739|4|516| +2450815|12740|4|109| +2450815|12742|4|468| +2450815|12745|4|782| +2450815|12746|4|820| +2450815|12748|4|152| +2450815|12751|4|| +2450815|12752|4|740| +2450815|12754|4|681| +2450815|12757|4|748| +2450815|12758|4|497| +2450815|12760|4|731| +2450815|12763|4|908| +2450815|12764|4|798| +2450815|12766|4|515| +2450815|12769|4|313| +2450815|12770|4|447| +2450815|12772|4|580| +2450815|12775|4|| +2450815|12776|4|894| +2450815|12778|4|| +2450815|12781|4|872| +2450815|12782|4|169| +2450815|12784|4|883| +2450815|12787|4|| +2450815|12788|4|538| +2450815|12790|4|740| +2450815|12793|4|633| +2450815|12794|4|952| +2450815|12796|4|536| +2450815|12799|4|748| +2450815|12800|4|749| +2450815|12802|4|31| +2450815|12805|4|910| +2450815|12806|4|841| +2450815|12808|4|226| +2450815|12811|4|647| +2450815|12812|4|933| +2450815|12814|4|674| +2450815|12817|4|924| +2450815|12818|4|329| +2450815|12820|4|940| +2450815|12823|4|56| +2450815|12824|4|704| +2450815|12826|4|868| +2450815|12829|4|327| +2450815|12830|4|326| +2450815|12832|4|899| +2450815|12835|4|348| +2450815|12836|4|247| +2450815|12838|4|720| +2450815|12841|4|1| +2450815|12842|4|95| +2450815|12844|4|40| +2450815|12847|4|313| +2450815|12848|4|875| +2450815|12850|4|837| +2450815|12853|4|99| +2450815|12854|4|859| +2450815|12856|4|348| +2450815|12859|4|832| +2450815|12860|4|310| +2450815|12862|4|25| +2450815|12865|4|939| +2450815|12866|4|356| +2450815|12868|4|148| +2450815|12871|4|593| +2450815|12872|4|596| +2450815|12874|4|964| +2450815|12877|4|879| +2450815|12878|4|118| +2450815|12880|4|15| +2450815|12883|4|267| +2450815|12884|4|292| +2450815|12886|4|927| +2450815|12889|4|| +2450815|12890|4|958| +2450815|12892|4|652| +2450815|12895|4|508| +2450815|12896|4|425| +2450815|12898|4|505| +2450815|12901|4|376| +2450815|12902|4|138| +2450815|12904|4|49| +2450815|12907|4|163| +2450815|12908|4|573| +2450815|12910|4|21| +2450815|12913|4|607| +2450815|12914|4|447| +2450815|12916|4|| +2450815|12919|4|931| +2450815|12920|4|103| +2450815|12922|4|90| +2450815|12925|4|979| +2450815|12926|4|249| +2450815|12928|4|54| +2450815|12931|4|55| +2450815|12932|4|| +2450815|12934|4|564| +2450815|12937|4|303| +2450815|12938|4|526| +2450815|12940|4|699| +2450815|12943|4|190| +2450815|12944|4|487| +2450815|12946|4|67| +2450815|12949|4|802| +2450815|12950|4|733| +2450815|12952|4|104| +2450815|12955|4|277| +2450815|12956|4|98| +2450815|12958|4|306| +2450815|12961|4|240| +2450815|12962|4|514| +2450815|12964|4|273| +2450815|12967|4|573| +2450815|12968|4|578| +2450815|12970|4|723| +2450815|12973|4|293| +2450815|12974|4|26| +2450815|12976|4|110| +2450815|12979|4|787| +2450815|12980|4|574| +2450815|12982|4|438| +2450815|12985|4|631| +2450815|12986|4|951| +2450815|12988|4|235| +2450815|12991|4|708| +2450815|12992|4|7| +2450815|12994|4|174| +2450815|12997|4|606| +2450815|12998|4|466| +2450815|13000|4|35| +2450815|13003|4|474| +2450815|13004|4|882| +2450815|13006|4|751| +2450815|13009|4|411| +2450815|13010|4|704| +2450815|13012|4|211| +2450815|13015|4|678| +2450815|13016|4|825| +2450815|13018|4|424| +2450815|13021|4|72| +2450815|13022|4|479| +2450815|13024|4|590| +2450815|13027|4|378| +2450815|13028|4|464| +2450815|13030|4|391| +2450815|13033|4|313| +2450815|13034|4|109| +2450815|13036|4|818| +2450815|13039|4|| +2450815|13040|4|780| +2450815|13042|4|169| +2450815|13045|4|564| +2450815|13046|4|566| +2450815|13048|4|765| +2450815|13051|4|522| +2450815|13052|4|186| +2450815|13054|4|754| +2450815|13057|4|275| +2450815|13058|4|740| +2450815|13060|4|48| +2450815|13063|4|966| +2450815|13064|4|48| +2450815|13066|4|37| +2450815|13069|4|675| +2450815|13070|4|838| +2450815|13072|4|904| +2450815|13075|4|830| +2450815|13076|4|207| +2450815|13078|4|39| +2450815|13081|4|301| +2450815|13082|4|252| +2450815|13084|4|| +2450815|13087|4|343| +2450815|13088|4|925| +2450815|13090|4|63| +2450815|13093|4|805| +2450815|13094|4|709| +2450815|13096|4|357| +2450815|13099|4|989| +2450815|13100|4|669| +2450815|13102|4|331| +2450815|13105|4|806| +2450815|13106|4|993| +2450815|13108|4|695| +2450815|13111|4|96| +2450815|13112|4|938| +2450815|13114|4|86| +2450815|13117|4|403| +2450815|13118|4|747| +2450815|13120|4|541| +2450815|13123|4|633| +2450815|13124|4|701| +2450815|13126|4|882| +2450815|13129|4|894| +2450815|13130|4|228| +2450815|13132|4|980| +2450815|13135|4|502| +2450815|13136|4|781| +2450815|13138|4|456| +2450815|13141|4|976| +2450815|13142|4|518| +2450815|13144|4|162| +2450815|13147|4|699| +2450815|13148|4|158| +2450815|13150|4|310| +2450815|13153|4|962| +2450815|13154|4|355| +2450815|13156|4|863| +2450815|13159|4|969| +2450815|13160|4|459| +2450815|13162|4|609| +2450815|13165|4|136| +2450815|13166|4|5| +2450815|13168|4|293| +2450815|13171|4|761| +2450815|13172|4|34| +2450815|13174|4|49| +2450815|13177|4|916| +2450815|13178|4|146| +2450815|13180|4|498| +2450815|13183|4|914| +2450815|13184|4|744| +2450815|13186|4|982| +2450815|13189|4|539| +2450815|13190|4|486| +2450815|13192|4|| +2450815|13195|4|148| +2450815|13196|4|527| +2450815|13198|4|996| +2450815|13201|4|942| +2450815|13202|4|274| +2450815|13204|4|428| +2450815|13207|4|898| +2450815|13208|4|354| +2450815|13210|4|635| +2450815|13213|4|467| +2450815|13214|4|689| +2450815|13216|4|588| +2450815|13219|4|| +2450815|13220|4|620| +2450815|13222|4|809| +2450815|13225|4|582| +2450815|13226|4|192| +2450815|13228|4|327| +2450815|13231|4|125| +2450815|13232|4|512| +2450815|13234|4|730| +2450815|13237|4|317| +2450815|13238|4|96| +2450815|13240|4|74| +2450815|13243|4|417| +2450815|13244|4|51| +2450815|13246|4|991| +2450815|13249|4|210| +2450815|13250|4|| +2450815|13252|4|801| +2450815|13255|4|273| +2450815|13256|4|581| +2450815|13258|4|403| +2450815|13261|4|707| +2450815|13262|4|165| +2450815|13264|4|425| +2450815|13267|4|644| +2450815|13268|4|405| +2450815|13270|4|692| +2450815|13273|4|429| +2450815|13274|4|481| +2450815|13276|4|340| +2450815|13279|4|115| +2450815|13280|4|300| +2450815|13282|4|| +2450815|13285|4|454| +2450815|13286|4|45| +2450815|13288|4|| +2450815|13291|4|897| +2450815|13292|4|734| +2450815|13294|4|75| +2450815|13297|4|429| +2450815|13298|4|| +2450815|13300|4|776| +2450815|13303|4|569| +2450815|13304|4|589| +2450815|13306|4|134| +2450815|13309|4|323| +2450815|13310|4|731| +2450815|13312|4|| +2450815|13315|4|298| +2450815|13316|4|263| +2450815|13318|4|256| +2450815|13321|4|251| +2450815|13322|4|123| +2450815|13324|4|292| +2450815|13327|4|726| +2450815|13328|4|592| +2450815|13330|4|648| +2450815|13333|4|872| +2450815|13334|4|55| +2450815|13336|4|593| +2450815|13339|4|361| +2450815|13340|4|389| +2450815|13342|4|326| +2450815|13345|4|204| +2450815|13346|4|504| +2450815|13348|4|223| +2450815|13351|4|607| +2450815|13352|4|2| +2450815|13354|4|213| +2450815|13357|4|696| +2450815|13358|4|695| +2450815|13360|4|801| +2450815|13363|4|841| +2450815|13364|4|155| +2450815|13366|4|975| +2450815|13369|4|625| +2450815|13370|4|926| +2450815|13372|4|822| +2450815|13375|4|153| +2450815|13376|4|944| +2450815|13378|4|981| +2450815|13381|4|932| +2450815|13382|4|136| +2450815|13384|4|174| +2450815|13387|4|687| +2450815|13388|4|411| +2450815|13390|4|184| +2450815|13393|4|293| +2450815|13394|4|923| +2450815|13396|4|876| +2450815|13399|4|| +2450815|13400|4|311| +2450815|13402|4|904| +2450815|13405|4|364| +2450815|13406|4|149| +2450815|13408|4|269| +2450815|13411|4|718| +2450815|13412|4|743| +2450815|13414|4|343| +2450815|13417|4|813| +2450815|13418|4|988| +2450815|13420|4|853| +2450815|13423|4|648| +2450815|13424|4|230| +2450815|13426|4|999| +2450815|13429|4|70| +2450815|13430|4|767| +2450815|13432|4|637| +2450815|13435|4|444| +2450815|13436|4|95| +2450815|13438|4|362| +2450815|13441|4|989| +2450815|13442|4|358| +2450815|13444|4|673| +2450815|13447|4|427| +2450815|13448|4|450| +2450815|13450|4|669| +2450815|13453|4|163| +2450815|13454|4|719| +2450815|13456|4|12| +2450815|13459|4|286| +2450815|13460|4|952| +2450815|13462|4|868| +2450815|13465|4|930| +2450815|13466|4|330| +2450815|13468|4|1000| +2450815|13471|4|726| +2450815|13472|4|914| +2450815|13474|4|28| +2450815|13477|4|927| +2450815|13478|4|305| +2450815|13480|4|| +2450815|13483|4|986| +2450815|13484|4|937| +2450815|13486|4|830| +2450815|13489|4|930| +2450815|13490|4|51| +2450815|13492|4|228| +2450815|13495|4|310| +2450815|13496|4|498| +2450815|13498|4|793| +2450815|13501|4|108| +2450815|13502|4|752| +2450815|13504|4|456| +2450815|13507|4|611| +2450815|13508|4|64| +2450815|13510|4|157| +2450815|13513|4|702| +2450815|13514|4|463| +2450815|13516|4|484| +2450815|13519|4|716| +2450815|13520|4|479| +2450815|13522|4|422| +2450815|13525|4|499| +2450815|13526|4|23| +2450815|13528|4|921| +2450815|13531|4|115| +2450815|13532|4|| +2450815|13534|4|21| +2450815|13537|4|| +2450815|13538|4|50| +2450815|13540|4|472| +2450815|13543|4|820| +2450815|13544|4|321| +2450815|13546|4|950| +2450815|13549|4|409| +2450815|13550|4|515| +2450815|13552|4|560| +2450815|13555|4|33| +2450815|13556|4|151| +2450815|13558|4|672| +2450815|13561|4|412| +2450815|13562|4|382| +2450815|13564|4|369| +2450815|13567|4|471| +2450815|13568|4|| +2450815|13570|4|804| +2450815|13573|4|426| +2450815|13574|4|312| +2450815|13576|4|567| +2450815|13579|4|62| +2450815|13580|4|647| +2450815|13582|4|836| +2450815|13585|4|477| +2450815|13586|4|574| +2450815|13588|4|154| +2450815|13591|4|356| +2450815|13592|4|637| +2450815|13594|4|999| +2450815|13597|4|665| +2450815|13598|4|57| +2450815|13600|4|849| +2450815|13603|4|727| +2450815|13604|4|487| +2450815|13606|4|897| +2450815|13609|4|717| +2450815|13610|4|722| +2450815|13612|4|273| +2450815|13615|4|125| +2450815|13616|4|121| +2450815|13618|4|| +2450815|13621|4|684| +2450815|13622|4|930| +2450815|13624|4|985| +2450815|13627|4|64| +2450815|13628|4|915| +2450815|13630|4|404| +2450815|13633|4|819| +2450815|13634|4|971| +2450815|13636|4|200| +2450815|13639|4|334| +2450815|13640|4|365| +2450815|13642|4|347| +2450815|13645|4|996| +2450815|13646|4|708| +2450815|13648|4|894| +2450815|13651|4|940| +2450815|13652|4|645| +2450815|13654|4|454| +2450815|13657|4|847| +2450815|13658|4|757| +2450815|13660|4|957| +2450815|13663|4|625| +2450815|13664|4|650| +2450815|13666|4|82| +2450815|13669|4|122| +2450815|13670|4|560| +2450815|13672|4|| +2450815|13675|4|799| +2450815|13676|4|677| +2450815|13678|4|592| +2450815|13681|4|207| +2450815|13682|4|870| +2450815|13684|4|881| +2450815|13687|4|| +2450815|13688|4|216| +2450815|13690|4|925| +2450815|13693|4|894| +2450815|13694|4|437| +2450815|13696|4|718| +2450815|13699|4|973| +2450815|13700|4|343| +2450815|13702|4|580| +2450815|13705|4|109| +2450815|13706|4|160| +2450815|13708|4|| +2450815|13711|4|393| +2450815|13712|4|14| +2450815|13714|4|139| +2450815|13717|4|752| +2450815|13718|4|546| +2450815|13720|4|561| +2450815|13723|4|361| +2450815|13724|4|| +2450815|13726|4|322| +2450815|13729|4|426| +2450815|13730|4|907| +2450815|13732|4|15| +2450815|13735|4|260| +2450815|13736|4|852| +2450815|13738|4|349| +2450815|13741|4|867| +2450815|13742|4|930| +2450815|13744|4|94| +2450815|13747|4|846| +2450815|13748|4|487| +2450815|13750|4|155| +2450815|13753|4|410| +2450815|13754|4|509| +2450815|13756|4|245| +2450815|13759|4|266| +2450815|13760|4|| +2450815|13762|4|760| +2450815|13765|4|896| +2450815|13766|4|762| +2450815|13768|4|101| +2450815|13771|4|328| +2450815|13772|4|258| +2450815|13774|4|785| +2450815|13777|4|285| +2450815|13778|4|740| +2450815|13780|4|622| +2450815|13783|4|838| +2450815|13784|4|928| +2450815|13786|4|43| +2450815|13789|4|251| +2450815|13790|4|878| +2450815|13792|4|173| +2450815|13795|4|245| +2450815|13796|4|731| +2450815|13798|4|912| +2450815|13801|4|44| +2450815|13802|4|334| +2450815|13804|4|885| +2450815|13807|4|85| +2450815|13808|4|277| +2450815|13810|4|779| +2450815|13813|4|614| +2450815|13814|4|686| +2450815|13816|4|323| +2450815|13819|4|647| +2450815|13820|4|34| +2450815|13822|4|267| +2450815|13825|4|24| +2450815|13826|4|656| +2450815|13828|4|420| +2450815|13831|4|802| +2450815|13832|4|888| +2450815|13834|4|360| +2450815|13837|4|87| +2450815|13838|4|| +2450815|13840|4|566| +2450815|13843|4|| +2450815|13844|4|140| +2450815|13846|4|782| +2450815|13849|4|850| +2450815|13850|4|897| +2450815|13852|4|544| +2450815|13855|4|531| +2450815|13856|4|922| +2450815|13858|4|374| +2450815|13861|4|362| +2450815|13862|4|457| +2450815|13864|4|611| +2450815|13867|4|505| +2450815|13868|4|660| +2450815|13870|4|| +2450815|13873|4|915| +2450815|13874|4|284| +2450815|13876|4|398| +2450815|13879|4|229| +2450815|13880|4|956| +2450815|13882|4|979| +2450815|13885|4|339| +2450815|13886|4|552| +2450815|13888|4|816| +2450815|13891|4|990| +2450815|13892|4|113| +2450815|13894|4|720| +2450815|13897|4|462| +2450815|13898|4|252| +2450815|13900|4|19| +2450815|13903|4|632| +2450815|13904|4|912| +2450815|13906|4|832| +2450815|13909|4|| +2450815|13910|4|966| +2450815|13912|4|396| +2450815|13915|4|758| +2450815|13916|4|659| +2450815|13918|4|602| +2450815|13921|4|71| +2450815|13922|4|495| +2450815|13924|4|748| +2450815|13927|4|| +2450815|13928|4|362| +2450815|13930|4|593| +2450815|13933|4|330| +2450815|13934|4|269| +2450815|13936|4|| +2450815|13939|4|500| +2450815|13940|4|623| +2450815|13942|4|178| +2450815|13945|4|672| +2450815|13946|4|850| +2450815|13948|4|142| +2450815|13951|4|426| +2450815|13952|4|742| +2450815|13954|4|829| +2450815|13957|4|551| +2450815|13958|4|859| +2450815|13960|4|902| +2450815|13963|4|707| +2450815|13964|4|| +2450815|13966|4|72| +2450815|13969|4|657| +2450815|13970|4|268| +2450815|13972|4|886| +2450815|13975|4|62| +2450815|13976|4|358| +2450815|13978|4|808| +2450815|13981|4|431| +2450815|13982|4|11| +2450815|13984|4|183| +2450815|13987|4|136| +2450815|13988|4|983| +2450815|13990|4|372| +2450815|13993|4|232| +2450815|13994|4|779| +2450815|13996|4|638| +2450815|13999|4|880| +2450815|14000|4|337| +2450815|14002|4|825| +2450815|14005|4|576| +2450815|14006|4|982| +2450815|14008|4|894| +2450815|14011|4|418| +2450815|14012|4|256| +2450815|14014|4|415| +2450815|14017|4|364| +2450815|14018|4|71| +2450815|14020|4|73| +2450815|14023|4|87| +2450815|14024|4|946| +2450815|14026|4|377| +2450815|14029|4|584| +2450815|14030|4|| +2450815|14032|4|640| +2450815|14035|4|568| +2450815|14036|4|239| +2450815|14038|4|173| +2450815|14041|4|| +2450815|14042|4|439| +2450815|14044|4|685| +2450815|14047|4|230| +2450815|14048|4|774| +2450815|14050|4|354| +2450815|14053|4|701| +2450815|14054|4|305| +2450815|14056|4|397| +2450815|14059|4|1| +2450815|14060|4|977| +2450815|14062|4|51| +2450815|14065|4|246| +2450815|14066|4|846| +2450815|14068|4|437| +2450815|14071|4|137| +2450815|14072|4|482| +2450815|14074|4|645| +2450815|14077|4|274| +2450815|14078|4|693| +2450815|14080|4|502| +2450815|14083|4|181| +2450815|14084|4|749| +2450815|14086|4|850| +2450815|14089|4|196| +2450815|14090|4|143| +2450815|14092|4|638| +2450815|14095|4|245| +2450815|14096|4|241| +2450815|14098|4|342| +2450815|14101|4|281| +2450815|14102|4|191| +2450815|14104|4|412| +2450815|14107|4|836| +2450815|14108|4|683| +2450815|14110|4|944| +2450815|14113|4|911| +2450815|14114|4|285| +2450815|14116|4|725| +2450815|14119|4|486| +2450815|14120|4|949| +2450815|14122|4|548| +2450815|14125|4|| +2450815|14126|4|960| +2450815|14128|4|865| +2450815|14131|4|488| +2450815|14132|4|350| +2450815|14134|4|767| +2450815|14137|4|650| +2450815|14138|4|258| +2450815|14140|4|521| +2450815|14143|4|223| +2450815|14144|4|709| +2450815|14146|4|859| +2450815|14149|4|551| +2450815|14150|4|107| +2450815|14152|4|246| +2450815|14155|4|| +2450815|14156|4|394| +2450815|14158|4|964| +2450815|14161|4|139| +2450815|14162|4|683| +2450815|14164|4|67| +2450815|14167|4|582| +2450815|14168|4|819| +2450815|14170|4|329| +2450815|14173|4|993| +2450815|14174|4|564| +2450815|14176|4|474| +2450815|14179|4|440| +2450815|14180|4|215| +2450815|14182|4|834| +2450815|14185|4|391| +2450815|14186|4|278| +2450815|14188|4|211| +2450815|14191|4|392| +2450815|14192|4|922| +2450815|14194|4|203| +2450815|14197|4|645| +2450815|14198|4|138| +2450815|14200|4|745| +2450815|14203|4|204| +2450815|14204|4|84| +2450815|14206|4|502| +2450815|14209|4|155| +2450815|14210|4|498| +2450815|14212|4|271| +2450815|14215|4|907| +2450815|14216|4|242| +2450815|14218|4|3| +2450815|14221|4|159| +2450815|14222|4|182| +2450815|14224|4|941| +2450815|14227|4|424| +2450815|14228|4|14| +2450815|14230|4|372| +2450815|14233|4|488| +2450815|14234|4|576| +2450815|14236|4|228| +2450815|14239|4|182| +2450815|14240|4|807| +2450815|14242|4|368| +2450815|14245|4|949| +2450815|14246|4|53| +2450815|14248|4|213| +2450815|14251|4|800| +2450815|14252|4|737| +2450815|14254|4|984| +2450815|14257|4|316| +2450815|14258|4|755| +2450815|14260|4|592| +2450815|14263|4|944| +2450815|14264|4|104| +2450815|14266|4|632| +2450815|14269|4|817| +2450815|14270|4|736| +2450815|14272|4|367| +2450815|14275|4|418| +2450815|14276|4|911| +2450815|14278|4|738| +2450815|14281|4|475| +2450815|14282|4|436| +2450815|14284|4|25| +2450815|14287|4|41| +2450815|14288|4|605| +2450815|14290|4|498| +2450815|14293|4|| +2450815|14294|4|274| +2450815|14296|4|225| +2450815|14299|4|460| +2450815|14300|4|721| +2450815|14302|4|444| +2450815|14305|4|675| +2450815|14306|4|| +2450815|14308|4|519| +2450815|14311|4|294| +2450815|14312|4|360| +2450815|14314|4|963| +2450815|14317|4|481| +2450815|14318|4|108| +2450815|14320|4|411| +2450815|14323|4|642| +2450815|14324|4|452| +2450815|14326|4|206| +2450815|14329|4|730| +2450815|14330|4|269| +2450815|14332|4|576| +2450815|14335|4|701| +2450815|14336|4|512| +2450815|14338|4|820| +2450815|14341|4|154| +2450815|14342|4|46| +2450815|14344|4|| +2450815|14347|4|439| +2450815|14348|4|4| +2450815|14350|4|955| +2450815|14353|4|750| +2450815|14354|4|| +2450815|14356|4|117| +2450815|14359|4|630| +2450815|14360|4|452| +2450815|14362|4|157| +2450815|14365|4|412| +2450815|14366|4|829| +2450815|14368|4|204| +2450815|14371|4|535| +2450815|14372|4|152| +2450815|14374|4|724| +2450815|14377|4|155| +2450815|14378|4|553| +2450815|14380|4|716| +2450815|14383|4|557| +2450815|14384|4|356| +2450815|14386|4|985| +2450815|14389|4|76| +2450815|14390|4|| +2450815|14392|4|847| +2450815|14395|4|456| +2450815|14396|4|582| +2450815|14398|4|686| +2450815|14401|4|26| +2450815|14402|4|| +2450815|14404|4|984| +2450815|14407|4|587| +2450815|14408|4|478| +2450815|14410|4|259| +2450815|14413|4|314| +2450815|14414|4|775| +2450815|14416|4|101| +2450815|14419|4|| +2450815|14420|4|187| +2450815|14422|4|579| +2450815|14425|4|436| +2450815|14426|4|512| +2450815|14428|4|932| +2450815|14431|4|837| +2450815|14432|4|429| +2450815|14434|4|764| +2450815|14437|4|125| +2450815|14438|4|179| +2450815|14440|4|434| +2450815|14443|4|762| +2450815|14444|4|960| +2450815|14446|4|893| +2450815|14449|4|423| +2450815|14450|4|515| +2450815|14452|4|937| +2450815|14455|4|690| +2450815|14456|4|391| +2450815|14458|4|655| +2450815|14461|4|384| +2450815|14462|4|931| +2450815|14464|4|670| +2450815|14467|4|| +2450815|14468|4|745| +2450815|14470|4|762| +2450815|14473|4|976| +2450815|14474|4|671| +2450815|14476|4|377| +2450815|14479|4|139| +2450815|14480|4|129| +2450815|14482|4|492| +2450815|14485|4|323| +2450815|14486|4|386| +2450815|14488|4|738| +2450815|14491|4|948| +2450815|14492|4|781| +2450815|14494|4|700| +2450815|14497|4|287| +2450815|14498|4|406| +2450815|14500|4|989| +2450815|14503|4|19| +2450815|14504|4|487| +2450815|14506|4|590| +2450815|14509|4|743| +2450815|14510|4|425| +2450815|14512|4|136| +2450815|14515|4|861| +2450815|14516|4|815| +2450815|14518|4|395| +2450815|14521|4|320| +2450815|14522|4|586| +2450815|14524|4|294| +2450815|14527|4|610| +2450815|14528|4|12| +2450815|14530|4|236| +2450815|14533|4|86| +2450815|14534|4|737| +2450815|14536|4|781| +2450815|14539|4|338| +2450815|14540|4|44| +2450815|14542|4|875| +2450815|14545|4|1| +2450815|14546|4|648| +2450815|14548|4|| +2450815|14551|4|589| +2450815|14552|4|543| +2450815|14554|4|404| +2450815|14557|4|386| +2450815|14558|4|312| +2450815|14560|4|17| +2450815|14563|4|345| +2450815|14564|4|683| +2450815|14566|4|787| +2450815|14569|4|203| +2450815|14570|4|772| +2450815|14572|4|850| +2450815|14575|4|354| +2450815|14576|4|59| +2450815|14578|4|296| +2450815|14581|4|203| +2450815|14582|4|873| +2450815|14584|4|208| +2450815|14587|4|837| +2450815|14588|4|60| +2450815|14590|4|951| +2450815|14593|4|497| +2450815|14594|4|802| +2450815|14596|4|798| +2450815|14599|4|710| +2450815|14600|4|99| +2450815|14602|4|979| +2450815|14605|4|921| +2450815|14606|4|429| +2450815|14608|4|274| +2450815|14611|4|529| +2450815|14612|4|875| +2450815|14614|4|650| +2450815|14617|4|24| +2450815|14618|4|160| +2450815|14620|4|337| +2450815|14623|4|349| +2450815|14624|4|136| +2450815|14626|4|417| +2450815|14629|4|121| +2450815|14630|4|20| +2450815|14632|4|268| +2450815|14635|4|626| +2450815|14636|4|360| +2450815|14638|4|205| +2450815|14641|4|818| +2450815|14642|4|985| +2450815|14644|4|446| +2450815|14647|4|178| +2450815|14648|4|378| +2450815|14650|4|| +2450815|14653|4|531| +2450815|14654|4|123| +2450815|14656|4|983| +2450815|14659|4|327| +2450815|14660|4|30| +2450815|14662|4|886| +2450815|14665|4|554| +2450815|14666|4|99| +2450815|14668|4|665| +2450815|14671|4|922| +2450815|14672|4|188| +2450815|14674|4|606| +2450815|14677|4|81| +2450815|14678|4|128| +2450815|14680|4|81| +2450815|14683|4|898| +2450815|14684|4|691| +2450815|14686|4|562| +2450815|14689|4|734| +2450815|14690|4|374| +2450815|14692|4|794| +2450815|14695|4|7| +2450815|14696|4|| +2450815|14698|4|921| +2450815|14701|4|850| +2450815|14702|4|222| +2450815|14704|4|279| +2450815|14707|4|277| +2450815|14708|4|341| +2450815|14710|4|593| +2450815|14713|4|856| +2450815|14714|4|396| +2450815|14716|4|403| +2450815|14719|4|531| +2450815|14720|4|239| +2450815|14722|4|786| +2450815|14725|4|284| +2450815|14726|4|142| +2450815|14728|4|| +2450815|14731|4|629| +2450815|14732|4|552| +2450815|14734|4|546| +2450815|14737|4|287| +2450815|14738|4|182| +2450815|14740|4|764| +2450815|14743|4|861| +2450815|14744|4|578| +2450815|14746|4|977| +2450815|14749|4|55| +2450815|14750|4|797| +2450815|14752|4|945| +2450815|14755|4|| +2450815|14756|4|929| +2450815|14758|4|733| +2450815|14761|4|332| +2450815|14762|4|910| +2450815|14764|4|373| +2450815|14767|4|158| +2450815|14768|4|32| +2450815|14770|4|280| +2450815|14773|4|104| +2450815|14774|4|681| +2450815|14776|4|80| +2450815|14779|4|617| +2450815|14780|4|461| +2450815|14782|4|131| +2450815|14785|4|209| +2450815|14786|4|172| +2450815|14788|4|19| +2450815|14791|4|215| +2450815|14792|4|904| +2450815|14794|4|333| +2450815|14797|4|668| +2450815|14798|4|951| +2450815|14800|4|463| +2450815|14803|4|422| +2450815|14804|4|965| +2450815|14806|4|719| +2450815|14809|4|595| +2450815|14810|4|678| +2450815|14812|4|620| +2450815|14815|4|778| +2450815|14816|4|737| +2450815|14818|4|106| +2450815|14821|4|495| +2450815|14822|4|275| +2450815|14824|4|900| +2450815|14827|4|900| +2450815|14828|4|790| +2450815|14830|4|874| +2450815|14833|4|311| +2450815|14834|4|189| +2450815|14836|4|732| +2450815|14839|4|629| +2450815|14840|4|515| +2450815|14842|4|421| +2450815|14845|4|176| +2450815|14846|4|466| +2450815|14848|4|863| +2450815|14851|4|316| +2450815|14852|4|665| +2450815|14854|4|409| +2450815|14857|4|196| +2450815|14858|4|918| +2450815|14860|4|579| +2450815|14863|4|563| +2450815|14864|4|50| +2450815|14866|4|341| +2450815|14869|4|52| +2450815|14870|4|199| +2450815|14872|4|976| +2450815|14875|4|165| +2450815|14876|4|931| +2450815|14878|4|130| +2450815|14881|4|| +2450815|14882|4|582| +2450815|14884|4|51| +2450815|14887|4|546| +2450815|14888|4|219| +2450815|14890|4|| +2450815|14893|4|397| +2450815|14894|4|162| +2450815|14896|4|963| +2450815|14899|4|220| +2450815|14900|4|501| +2450815|14902|4|292| +2450815|14905|4|629| +2450815|14906|4|164| +2450815|14908|4|769| +2450815|14911|4|340| +2450815|14912|4|179| +2450815|14914|4|| +2450815|14917|4|164| +2450815|14918|4|993| +2450815|14920|4|558| +2450815|14923|4|692| +2450815|14924|4|627| +2450815|14926|4|418| +2450815|14929|4|871| +2450815|14930|4|983| +2450815|14932|4|323| +2450815|14935|4|436| +2450815|14936|4|377| +2450815|14938|4|720| +2450815|14941|4|| +2450815|14942|4|977| +2450815|14944|4|844| +2450815|14947|4|953| +2450815|14948|4|842| +2450815|14950|4|1| +2450815|14953|4|608| +2450815|14954|4|71| +2450815|14956|4|968| +2450815|14959|4|680| +2450815|14960|4|550| +2450815|14962|4|302| +2450815|14965|4|| +2450815|14966|4|726| +2450815|14968|4|840| +2450815|14971|4|70| +2450815|14972|4|145| +2450815|14974|4|921| +2450815|14977|4|361| +2450815|14978|4|626| +2450815|14980|4|990| +2450815|14983|4|333| +2450815|14984|4|420| +2450815|14986|4|297| +2450815|14989|4|489| +2450815|14990|4|812| +2450815|14992|4|367| +2450815|14995|4|831| +2450815|14996|4|556| +2450815|14998|4|647| +2450815|15001|4|402| +2450815|15002|4|585| +2450815|15004|4|338| +2450815|15007|4|127| +2450815|15008|4|344| +2450815|15010|4|126| +2450815|15013|4|888| +2450815|15014|4|303| +2450815|15016|4|201| +2450815|15019|4|264| +2450815|15020|4|245| +2450815|15022|4|514| +2450815|15025|4|165| +2450815|15026|4|576| +2450815|15028|4|| +2450815|15031|4|937| +2450815|15032|4|482| +2450815|15034|4|671| +2450815|15037|4|| +2450815|15038|4|963| +2450815|15040|4|547| +2450815|15043|4|484| +2450815|15044|4|800| +2450815|15046|4|216| +2450815|15049|4|528| +2450815|15050|4|217| +2450815|15052|4|277| +2450815|15055|4|479| +2450815|15056|4|799| +2450815|15058|4|738| +2450815|15061|4|752| +2450815|15062|4|395| +2450815|15064|4|538| +2450815|15067|4|49| +2450815|15068|4|783| +2450815|15070|4|857| +2450815|15073|4|467| +2450815|15074|4|419| +2450815|15076|4|509| +2450815|15079|4|131| +2450815|15080|4|263| +2450815|15082|4|155| +2450815|15085|4|619| +2450815|15086|4|896| +2450815|15088|4|112| +2450815|15091|4|696| +2450815|15092|4|483| +2450815|15094|4|6| +2450815|15097|4|47| +2450815|15098|4|237| +2450815|15100|4|480| +2450815|15103|4|260| +2450815|15104|4|740| +2450815|15106|4|127| +2450815|15109|4|626| +2450815|15110|4|741| +2450815|15112|4|618| +2450815|15115|4|800| +2450815|15116|4|949| +2450815|15118|4|504| +2450815|15121|4|109| +2450815|15122|4|793| +2450815|15124|4|429| +2450815|15127|4|464| +2450815|15128|4|709| +2450815|15130|4|832| +2450815|15133|4|83| +2450815|15134|4|421| +2450815|15136|4|578| +2450815|15139|4|| +2450815|15140|4|| +2450815|15142|4|258| +2450815|15145|4|132| +2450815|15146|4|| +2450815|15148|4|68| +2450815|15151|4|25| +2450815|15152|4|329| +2450815|15154|4|594| +2450815|15157|4|653| +2450815|15158|4|118| +2450815|15160|4|| +2450815|15163|4|564| +2450815|15164|4|791| +2450815|15166|4|178| +2450815|15169|4|403| +2450815|15170|4|588| +2450815|15172|4|173| +2450815|15175|4|376| +2450815|15176|4|518| +2450815|15178|4|467| +2450815|15181|4|813| +2450815|15182|4|203| +2450815|15184|4|913| +2450815|15187|4|304| +2450815|15188|4|617| +2450815|15190|4|861| +2450815|15193|4|78| +2450815|15194|4|181| +2450815|15196|4|279| +2450815|15199|4|189| +2450815|15200|4|745| +2450815|15202|4|805| +2450815|15205|4|460| +2450815|15206|4|572| +2450815|15208|4|647| +2450815|15211|4|724| +2450815|15212|4|418| +2450815|15214|4|824| +2450815|15217|4|400| +2450815|15218|4|407| +2450815|15220|4|478| +2450815|15223|4|741| +2450815|15224|4|159| +2450815|15226|4|476| +2450815|15229|4|396| +2450815|15230|4|124| +2450815|15232|4|600| +2450815|15235|4|236| +2450815|15236|4|722| +2450815|15238|4|687| +2450815|15241|4|403| +2450815|15242|4|907| +2450815|15244|4|773| +2450815|15247|4|376| +2450815|15248|4|406| +2450815|15250|4|337| +2450815|15253|4|206| +2450815|15254|4|39| +2450815|15256|4|| +2450815|15259|4|706| +2450815|15260|4|182| +2450815|15262|4|333| +2450815|15265|4|967| +2450815|15266|4|373| +2450815|15268|4|63| +2450815|15271|4|742| +2450815|15272|4|885| +2450815|15274|4|97| +2450815|15277|4|182| +2450815|15278|4|601| +2450815|15280|4|313| +2450815|15283|4|949| +2450815|15284|4|236| +2450815|15286|4|5| +2450815|15289|4|595| +2450815|15290|4|143| +2450815|15292|4|860| +2450815|15295|4|348| +2450815|15296|4|871| +2450815|15298|4|54| +2450815|15301|4|607| +2450815|15302|4|31| +2450815|15304|4|429| +2450815|15307|4|203| +2450815|15308|4|780| +2450815|15310|4|353| +2450815|15313|4|| +2450815|15314|4|20| +2450815|15316|4|748| +2450815|15319|4|769| +2450815|15320|4|945| +2450815|15322|4|320| +2450815|15325|4|128| +2450815|15326|4|108| +2450815|15328|4|544| +2450815|15331|4|344| +2450815|15332|4|116| +2450815|15334|4|606| +2450815|15337|4|14| +2450815|15338|4|181| +2450815|15340|4|608| +2450815|15343|4|566| +2450815|15344|4|998| +2450815|15346|4|747| +2450815|15349|4|604| +2450815|15350|4|692| +2450815|15352|4|760| +2450815|15355|4|917| +2450815|15356|4|378| +2450815|15358|4|190| +2450815|15361|4|687| +2450815|15362|4|508| +2450815|15364|4|623| +2450815|15367|4|318| +2450815|15368|4|558| +2450815|15370|4|956| +2450815|15373|4|293| +2450815|15374|4|720| +2450815|15376|4|170| +2450815|15379|4|371| +2450815|15380|4|876| +2450815|15382|4|132| +2450815|15385|4|917| +2450815|15386|4|625| +2450815|15388|4|498| +2450815|15391|4|768| +2450815|15392|4|696| +2450815|15394|4|357| +2450815|15397|4|526| +2450815|15398|4|343| +2450815|15400|4|932| +2450815|15403|4|854| +2450815|15404|4|790| +2450815|15406|4|444| +2450815|15409|4|828| +2450815|15410|4|385| +2450815|15412|4|894| +2450815|15415|4|93| +2450815|15416|4|692| +2450815|15418|4|843| +2450815|15421|4|706| +2450815|15422|4|712| +2450815|15424|4|364| +2450815|15427|4|128| +2450815|15428|4|707| +2450815|15430|4|310| +2450815|15433|4|445| +2450815|15434|4|354| +2450815|15436|4|377| +2450815|15439|4|702| +2450815|15440|4|831| +2450815|15442|4|408| +2450815|15445|4|467| +2450815|15446|4|283| +2450815|15448|4|613| +2450815|15451|4|885| +2450815|15452|4|248| +2450815|15454|4|81| +2450815|15457|4|542| +2450815|15458|4|577| +2450815|15460|4|420| +2450815|15463|4|332| +2450815|15464|4|600| +2450815|15466|4|813| +2450815|15469|4|363| +2450815|15470|4|123| +2450815|15472|4|240| +2450815|15475|4|454| +2450815|15476|4|717| +2450815|15478|4|664| +2450815|15481|4|772| +2450815|15482|4|885| +2450815|15484|4|229| +2450815|15487|4|137| +2450815|15488|4|71| +2450815|15490|4|706| +2450815|15493|4|716| +2450815|15494|4|951| +2450815|15496|4|153| +2450815|15499|4|708| +2450815|15500|4|| +2450815|15502|4|893| +2450815|15505|4|446| +2450815|15506|4|| +2450815|15508|4|606| +2450815|15511|4|447| +2450815|15512|4|444| +2450815|15514|4|422| +2450815|15517|4|827| +2450815|15518|4|876| +2450815|15520|4|196| +2450815|15523|4|654| +2450815|15524|4|456| +2450815|15526|4|635| +2450815|15529|4|816| +2450815|15530|4|551| +2450815|15532|4|53| +2450815|15535|4|537| +2450815|15536|4|213| +2450815|15538|4|594| +2450815|15541|4|721| +2450815|15542|4|930| +2450815|15544|4|449| +2450815|15547|4|| +2450815|15548|4|| +2450815|15550|4|521| +2450815|15553|4|966| +2450815|15554|4|456| +2450815|15556|4|646| +2450815|15559|4|483| +2450815|15560|4|354| +2450815|15562|4|841| +2450815|15565|4|974| +2450815|15566|4|852| +2450815|15568|4|| +2450815|15571|4|622| +2450815|15572|4|87| +2450815|15574|4|766| +2450815|15577|4|622| +2450815|15578|4|770| +2450815|15580|4|331| +2450815|15583|4|688| +2450815|15584|4|710| +2450815|15586|4|912| +2450815|15589|4|368| +2450815|15590|4|45| +2450815|15592|4|303| +2450815|15595|4|233| +2450815|15596|4|300| +2450815|15598|4|109| +2450815|15601|4|| +2450815|15602|4|78| +2450815|15604|4|861| +2450815|15607|4|68| +2450815|15608|4|42| +2450815|15610|4|746| +2450815|15613|4|106| +2450815|15614|4|698| +2450815|15616|4|445| +2450815|15619|4|779| +2450815|15620|4|| +2450815|15622|4|488| +2450815|15625|4|| +2450815|15626|4|969| +2450815|15628|4|850| +2450815|15631|4|611| +2450815|15632|4|905| +2450815|15634|4|527| +2450815|15637|4|209| +2450815|15638|4|487| +2450815|15640|4|104| +2450815|15643|4|770| +2450815|15644|4|109| +2450815|15646|4|498| +2450815|15649|4|531| +2450815|15650|4|38| +2450815|15652|4|938| +2450815|15655|4|28| +2450815|15656|4|756| +2450815|15658|4|546| +2450815|15661|4|256| +2450815|15662|4|724| +2450815|15664|4|333| +2450815|15667|4|285| +2450815|15668|4|292| +2450815|15670|4|527| +2450815|15673|4|265| +2450815|15674|4|215| +2450815|15676|4|106| +2450815|15679|4|854| +2450815|15680|4|346| +2450815|15682|4|81| +2450815|15685|4|835| +2450815|15686|4|638| +2450815|15688|4|904| +2450815|15691|4|| +2450815|15692|4|376| +2450815|15694|4|19| +2450815|15697|4|512| +2450815|15698|4|978| +2450815|15700|4|102| +2450815|15703|4|| +2450815|15704|4|393| +2450815|15706|4|345| +2450815|15709|4|748| +2450815|15710|4|516| +2450815|15712|4|656| +2450815|15715|4|52| +2450815|15716|4|38| +2450815|15718|4|77| +2450815|15721|4|234| +2450815|15722|4|43| +2450815|15724|4|982| +2450815|15727|4|907| +2450815|15728|4|253| +2450815|15730|4|931| +2450815|15733|4|62| +2450815|15734|4|83| +2450815|15736|4|295| +2450815|15739|4|573| +2450815|15740|4|522| +2450815|15742|4|150| +2450815|15745|4|354| +2450815|15746|4|17| +2450815|15748|4|521| +2450815|15751|4|796| +2450815|15752|4|435| +2450815|15754|4|364| +2450815|15757|4|805| +2450815|15758|4|743| +2450815|15760|4|422| +2450815|15763|4|245| +2450815|15764|4|851| +2450815|15766|4|131| +2450815|15769|4|915| +2450815|15770|4|173| +2450815|15772|4|868| +2450815|15775|4|546| +2450815|15776|4|866| +2450815|15778|4|614| +2450815|15781|4|278| +2450815|15782|4|205| +2450815|15784|4|788| +2450815|15787|4|810| +2450815|15788|4|782| +2450815|15790|4|224| +2450815|15793|4|543| +2450815|15794|4|913| +2450815|15796|4|796| +2450815|15799|4|320| +2450815|15800|4|481| +2450815|15802|4|33| +2450815|15805|4|887| +2450815|15806|4|600| +2450815|15808|4|615| +2450815|15811|4|569| +2450815|15812|4|289| +2450815|15814|4|9| +2450815|15817|4|258| +2450815|15818|4|464| +2450815|15820|4|175| +2450815|15823|4|89| +2450815|15824|4|174| +2450815|15826|4|654| +2450815|15829|4|162| +2450815|15830|4|782| +2450815|15832|4|170| +2450815|15835|4|| +2450815|15836|4|356| +2450815|15838|4|678| +2450815|15841|4|444| +2450815|15842|4|417| +2450815|15844|4|337| +2450815|15847|4|238| +2450815|15848|4|803| +2450815|15850|4|| +2450815|15853|4|123| +2450815|15854|4|| +2450815|15856|4|605| +2450815|15859|4|600| +2450815|15860|4|632| +2450815|15862|4|638| +2450815|15865|4|703| +2450815|15866|4|451| +2450815|15868|4|346| +2450815|15871|4|965| +2450815|15872|4|153| +2450815|15874|4|747| +2450815|15877|4|625| +2450815|15878|4|467| +2450815|15880|4|921| +2450815|15883|4|405| +2450815|15884|4|636| +2450815|15886|4|715| +2450815|15889|4|645| +2450815|15890|4|550| +2450815|15892|4|800| +2450815|15895|4|660| +2450815|15896|4|534| +2450815|15898|4|| +2450815|15901|4|813| +2450815|15902|4|| +2450815|15904|4|53| +2450815|15907|4|554| +2450815|15908|4|172| +2450815|15910|4|798| +2450815|15913|4|448| +2450815|15914|4|263| +2450815|15916|4|4| +2450815|15919|4|299| +2450815|15920|4|903| +2450815|15922|4|544| +2450815|15925|4|919| +2450815|15926|4|710| +2450815|15928|4|115| +2450815|15931|4|648| +2450815|15932|4|676| +2450815|15934|4|766| +2450815|15937|4|876| +2450815|15938|4|971| +2450815|15940|4|| +2450815|15943|4|644| +2450815|15944|4|838| +2450815|15946|4|| +2450815|15949|4|860| +2450815|15950|4|214| +2450815|15952|4|85| +2450815|15955|4|741| +2450815|15956|4|| +2450815|15958|4|2| +2450815|15961|4|588| +2450815|15962|4|878| +2450815|15964|4|683| +2450815|15967|4|773| +2450815|15968|4|686| +2450815|15970|4|23| +2450815|15973|4|920| +2450815|15974|4|660| +2450815|15976|4|713| +2450815|15979|4|967| +2450815|15980|4|426| +2450815|15982|4|812| +2450815|15985|4|250| +2450815|15986|4|142| +2450815|15988|4|300| +2450815|15991|4|525| +2450815|15992|4|496| +2450815|15994|4|835| +2450815|15997|4|300| +2450815|15998|4|858| +2450815|16000|4|248| +2450815|16003|4|618| +2450815|16004|4|858| +2450815|16006|4|519| +2450815|16009|4|475| +2450815|16010|4|524| +2450815|16012|4|878| +2450815|16015|4|37| +2450815|16016|4|583| +2450815|16018|4|33| +2450815|16021|4|129| +2450815|16022|4|247| +2450815|16024|4|802| +2450815|16027|4|453| +2450815|16028|4|982| +2450815|16030|4|532| +2450815|16033|4|352| +2450815|16034|4|877| +2450815|16036|4|722| +2450815|16039|4|652| +2450815|16040|4|533| +2450815|16042|4|856| +2450815|16045|4|33| +2450815|16046|4|77| +2450815|16048|4|776| +2450815|16051|4|684| +2450815|16052|4|147| +2450815|16054|4|73| +2450815|16057|4|726| +2450815|16058|4|82| +2450815|16060|4|781| +2450815|16063|4|8| +2450815|16064|4|688| +2450815|16066|4|349| +2450815|16069|4|13| +2450815|16070|4|622| +2450815|16072|4|935| +2450815|16075|4|694| +2450815|16076|4|867| +2450815|16078|4|923| +2450815|16081|4|44| +2450815|16082|4|770| +2450815|16084|4|169| +2450815|16087|4|332| +2450815|16088|4|232| +2450815|16090|4|812| +2450815|16093|4|664| +2450815|16094|4|800| +2450815|16096|4|20| +2450815|16099|4|824| +2450815|16100|4|103| +2450815|16102|4|393| +2450815|16105|4|| +2450815|16106|4|582| +2450815|16108|4|| +2450815|16111|4|184| +2450815|16112|4|29| +2450815|16114|4|457| +2450815|16117|4|472| +2450815|16118|4|790| +2450815|16120|4|500| +2450815|16123|4|785| +2450815|16124|4|786| +2450815|16126|4|96| +2450815|16129|4|569| +2450815|16130|4|340| +2450815|16132|4|785| +2450815|16135|4|84| +2450815|16136|4|498| +2450815|16138|4|495| +2450815|16141|4|92| +2450815|16142|4|888| +2450815|16144|4|732| +2450815|16147|4|957| +2450815|16148|4|883| +2450815|16150|4|175| +2450815|16153|4|800| +2450815|16154|4|213| +2450815|16156|4|497| +2450815|16159|4|206| +2450815|16160|4|10| +2450815|16162|4|542| +2450815|16165|4|88| +2450815|16166|4|75| +2450815|16168|4|841| +2450815|16171|4|843| +2450815|16172|4|63| +2450815|16174|4|| +2450815|16177|4|820| +2450815|16178|4|49| +2450815|16180|4|944| +2450815|16183|4|791| +2450815|16184|4|515| +2450815|16186|4|813| +2450815|16189|4|49| +2450815|16190|4|101| +2450815|16192|4|573| +2450815|16195|4|451| +2450815|16196|4|823| +2450815|16198|4|903| +2450815|16201|4|559| +2450815|16202|4|310| +2450815|16204|4|212| +2450815|16207|4|506| +2450815|16208|4|| +2450815|16210|4|515| +2450815|16213|4|193| +2450815|16214|4|343| +2450815|16216|4|171| +2450815|16219|4|172| +2450815|16220|4|447| +2450815|16222|4|123| +2450815|16225|4|374| +2450815|16226|4|569| +2450815|16228|4|| +2450815|16231|4|298| +2450815|16232|4|384| +2450815|16234|4|577| +2450815|16237|4|432| +2450815|16238|4|609| +2450815|16240|4|209| +2450815|16243|4|| +2450815|16244|4|798| +2450815|16246|4|363| +2450815|16249|4|971| +2450815|16250|4|540| +2450815|16252|4|357| +2450815|16255|4|709| +2450815|16256|4|725| +2450815|16258|4|271| +2450815|16261|4|595| +2450815|16262|4|63| +2450815|16264|4|706| +2450815|16267|4|828| +2450815|16268|4|315| +2450815|16270|4|665| +2450815|16273|4|369| +2450815|16274|4|903| +2450815|16276|4|872| +2450815|16279|4|603| +2450815|16280|4|511| +2450815|16282|4|656| +2450815|16285|4|242| +2450815|16286|4|889| +2450815|16288|4|202| +2450815|16291|4|754| +2450815|16292|4|971| +2450815|16294|4|729| +2450815|16297|4|56| +2450815|16298|4|242| +2450815|16300|4|401| +2450815|16303|4|978| +2450815|16304|4|211| +2450815|16306|4|19| +2450815|16309|4|322| +2450815|16310|4|73| +2450815|16312|4|68| +2450815|16315|4|291| +2450815|16316|4|820| +2450815|16318|4|343| +2450815|16321|4|893| +2450815|16322|4|613| +2450815|16324|4|235| +2450815|16327|4|125| +2450815|16328|4|215| +2450815|16330|4|861| +2450815|16333|4|536| +2450815|16334|4|774| +2450815|16336|4|849| +2450815|16339|4|869| +2450815|16340|4|632| +2450815|16342|4|402| +2450815|16345|4|501| +2450815|16346|4|826| +2450815|16348|4|986| +2450815|16351|4|995| +2450815|16352|4|702| +2450815|16354|4|279| +2450815|16357|4|627| +2450815|16358|4|| +2450815|16360|4|670| +2450815|16363|4|633| +2450815|16364|4|254| +2450815|16366|4|211| +2450815|16369|4|780| +2450815|16370|4|321| +2450815|16372|4|138| +2450815|16375|4|83| +2450815|16376|4|550| +2450815|16378|4|622| +2450815|16381|4|114| +2450815|16382|4|972| +2450815|16384|4|524| +2450815|16387|4|355| +2450815|16388|4|801| +2450815|16390|4|252| +2450815|16393|4|129| +2450815|16394|4|685| +2450815|16396|4|| +2450815|16399|4|482| +2450815|16400|4|346| +2450815|16402|4|229| +2450815|16405|4|2| +2450815|16406|4|316| +2450815|16408|4|391| +2450815|16411|4|74| +2450815|16412|4|728| +2450815|16414|4|909| +2450815|16417|4|648| +2450815|16418|4|650| +2450815|16420|4|801| +2450815|16423|4|880| +2450815|16424|4|186| +2450815|16426|4|530| +2450815|16429|4|415| +2450815|16430|4|731| +2450815|16432|4|322| +2450815|16435|4|421| +2450815|16436|4|608| +2450815|16438|4|892| +2450815|16441|4|889| +2450815|16442|4|556| +2450815|16444|4|407| +2450815|16447|4|867| +2450815|16448|4|846| +2450815|16450|4|638| +2450815|16453|4|4| +2450815|16454|4|356| +2450815|16456|4|147| +2450815|16459|4|801| +2450815|16460|4|3| +2450815|16462|4|| +2450815|16465|4|149| +2450815|16466|4|| +2450815|16468|4|618| +2450815|16471|4|912| +2450815|16472|4|8| +2450815|16474|4|85| +2450815|16477|4|679| +2450815|16478|4|420| +2450815|16480|4|81| +2450815|16483|4|842| +2450815|16484|4|| +2450815|16486|4|669| +2450815|16489|4|827| +2450815|16490|4|643| +2450815|16492|4|766| +2450815|16495|4|327| +2450815|16496|4|250| +2450815|16498|4|674| +2450815|16501|4|206| +2450815|16502|4|920| +2450815|16504|4|| +2450815|16507|4|990| +2450815|16508|4|889| +2450815|16510|4|395| +2450815|16513|4|675| +2450815|16514|4|79| +2450815|16516|4|755| +2450815|16519|4|57| +2450815|16520|4|789| +2450815|16522|4|281| +2450815|16525|4|283| +2450815|16526|4|662| +2450815|16528|4|106| +2450815|16531|4|228| +2450815|16532|4|469| +2450815|16534|4|963| +2450815|16537|4|882| +2450815|16538|4|54| +2450815|16540|4|551| +2450815|16543|4|951| +2450815|16544|4|529| +2450815|16546|4|223| +2450815|16549|4|486| +2450815|16550|4|241| +2450815|16552|4|590| +2450815|16555|4|| +2450815|16556|4|375| +2450815|16558|4|785| +2450815|16561|4|667| +2450815|16562|4|| +2450815|16564|4|612| +2450815|16567|4|500| +2450815|16568|4|999| +2450815|16570|4|88| +2450815|16573|4|799| +2450815|16574|4|414| +2450815|16576|4|493| +2450815|16579|4|274| +2450815|16580|4|495| +2450815|16582|4|812| +2450815|16585|4|875| +2450815|16586|4|772| +2450815|16588|4|126| +2450815|16591|4|186| +2450815|16592|4|230| +2450815|16594|4|714| +2450815|16597|4|472| +2450815|16598|4|22| +2450815|16600|4|579| +2450815|16603|4|524| +2450815|16604|4|682| +2450815|16606|4|215| +2450815|16609|4|814| +2450815|16610|4|386| +2450815|16612|4|535| +2450815|16615|4|985| +2450815|16616|4|892| +2450815|16618|4|701| +2450815|16621|4|344| +2450815|16622|4|1000| +2450815|16624|4|661| +2450815|16627|4|934| +2450815|16628|4|528| +2450815|16630|4|| +2450815|16633|4|| +2450815|16634|4|808| +2450815|16636|4|517| +2450815|16639|4|301| +2450815|16640|4|638| +2450815|16642|4|732| +2450815|16645|4|18| +2450815|16646|4|387| +2450815|16648|4|403| +2450815|16651|4|737| +2450815|16652|4|735| +2450815|16654|4|| +2450815|16657|4|339| +2450815|16658|4|948| +2450815|16660|4|570| +2450815|16663|4|62| +2450815|16664|4|588| +2450815|16666|4|180| +2450815|16669|4|821| +2450815|16670|4|706| +2450815|16672|4|917| +2450815|16675|4|944| +2450815|16676|4|299| +2450815|16678|4|107| +2450815|16681|4|197| +2450815|16682|4|| +2450815|16684|4|316| +2450815|16687|4|375| +2450815|16688|4|49| +2450815|16690|4|572| +2450815|16693|4|41| +2450815|16694|4|679| +2450815|16696|4|124| +2450815|16699|4|809| +2450815|16700|4|362| +2450815|16702|4|609| +2450815|16705|4|891| +2450815|16706|4|55| +2450815|16708|4|821| +2450815|16711|4|848| +2450815|16712|4|790| +2450815|16714|4|388| +2450815|16717|4|874| +2450815|16718|4|333| +2450815|16720|4|233| +2450815|16723|4|141| +2450815|16724|4|18| +2450815|16726|4|926| +2450815|16729|4|47| +2450815|16730|4|347| +2450815|16732|4|893| +2450815|16735|4|582| +2450815|16736|4|221| +2450815|16738|4|419| +2450815|16741|4|746| +2450815|16742|4|200| +2450815|16744|4|91| +2450815|16747|4|225| +2450815|16748|4|678| +2450815|16750|4|578| +2450815|16753|4|31| +2450815|16754|4|23| +2450815|16756|4|813| +2450815|16759|4|493| +2450815|16760|4|450| +2450815|16762|4|655| +2450815|16765|4|606| +2450815|16766|4|173| +2450815|16768|4|273| +2450815|16771|4|74| +2450815|16772|4|631| +2450815|16774|4|| +2450815|16777|4|25| +2450815|16778|4|368| +2450815|16780|4|131| +2450815|16783|4|192| +2450815|16784|4|734| +2450815|16786|4|736| +2450815|16789|4|260| +2450815|16790|4|900| +2450815|16792|4|139| +2450815|16795|4|427| +2450815|16796|4|803| +2450815|16798|4|458| +2450815|16801|4|424| +2450815|16802|4|572| +2450815|16804|4|849| +2450815|16807|4|916| +2450815|16808|4|815| +2450815|16810|4|| +2450815|16813|4|291| +2450815|16814|4|642| +2450815|16816|4|415| +2450815|16819|4|668| +2450815|16820|4|330| +2450815|16822|4|915| +2450815|16825|4|547| +2450815|16826|4|605| +2450815|16828|4|631| +2450815|16831|4|412| +2450815|16832|4|455| +2450815|16834|4|761| +2450815|16837|4|54| +2450815|16838|4|674| +2450815|16840|4|52| +2450815|16843|4|714| +2450815|16844|4|392| +2450815|16846|4|40| +2450815|16849|4|348| +2450815|16850|4|482| +2450815|16852|4|65| +2450815|16855|4|22| +2450815|16856|4|281| +2450815|16858|4|840| +2450815|16861|4|988| +2450815|16862|4|830| +2450815|16864|4|681| +2450815|16867|4|957| +2450815|16868|4|637| +2450815|16870|4|369| +2450815|16873|4|445| +2450815|16874|4|794| +2450815|16876|4|553| +2450815|16879|4|185| +2450815|16880|4|921| +2450815|16882|4|201| +2450815|16885|4|806| +2450815|16886|4|357| +2450815|16888|4|718| +2450815|16891|4|336| +2450815|16892|4|553| +2450815|16894|4|5| +2450815|16897|4|900| +2450815|16898|4|247| +2450815|16900|4|121| +2450815|16903|4|786| +2450815|16904|4|9| +2450815|16906|4|751| +2450815|16909|4|510| +2450815|16910|4|216| +2450815|16912|4|464| +2450815|16915|4|385| +2450815|16916|4|413| +2450815|16918|4|348| +2450815|16921|4|389| +2450815|16922|4|465| +2450815|16924|4|848| +2450815|16927|4|792| +2450815|16928|4|552| +2450815|16930|4|256| +2450815|16933|4|796| +2450815|16934|4|604| +2450815|16936|4|530| +2450815|16939|4|| +2450815|16940|4|846| +2450815|16942|4|337| +2450815|16945|4|137| +2450815|16946|4|412| +2450815|16948|4|467| +2450815|16951|4|583| +2450815|16952|4|954| +2450815|16954|4|216| +2450815|16957|4|235| +2450815|16958|4|894| +2450815|16960|4|967| +2450815|16963|4|876| +2450815|16964|4|465| +2450815|16966|4|132| +2450815|16969|4|836| +2450815|16970|4|750| +2450815|16972|4|805| +2450815|16975|4|17| +2450815|16976|4|92| +2450815|16978|4|520| +2450815|16981|4|45| +2450815|16982|4|94| +2450815|16984|4|585| +2450815|16987|4|852| +2450815|16988|4|328| +2450815|16990|4|943| +2450815|16993|4|363| +2450815|16994|4|551| +2450815|16996|4|274| +2450815|16999|4|451| +2450815|17000|4|242| +2450815|17002|4|288| +2450815|17005|4|740| +2450815|17006|4|389| +2450815|17008|4|402| +2450815|17011|4|680| +2450815|17012|4|762| +2450815|17014|4|973| +2450815|17017|4|875| +2450815|17018|4|325| +2450815|17020|4|669| +2450815|17023|4|905| +2450815|17024|4|825| +2450815|17026|4|514| +2450815|17029|4|476| +2450815|17030|4|425| +2450815|17032|4|16| +2450815|17035|4|543| +2450815|17036|4|290| +2450815|17038|4|509| +2450815|17041|4|222| +2450815|17042|4|639| +2450815|17044|4|7| +2450815|17047|4|44| +2450815|17048|4|| +2450815|17050|4|157| +2450815|17053|4|105| +2450815|17054|4|647| +2450815|17056|4|774| +2450815|17059|4|197| +2450815|17060|4|570| +2450815|17062|4|289| +2450815|17065|4|244| +2450815|17066|4|730| +2450815|17068|4|320| +2450815|17071|4|| +2450815|17072|4|424| +2450815|17074|4|469| +2450815|17077|4|575| +2450815|17078|4|333| +2450815|17080|4|| +2450815|17083|4|57| +2450815|17084|4|534| +2450815|17086|4|140| +2450815|17089|4|667| +2450815|17090|4|210| +2450815|17092|4|627| +2450815|17095|4|876| +2450815|17096|4|742| +2450815|17098|4|612| +2450815|17101|4|392| +2450815|17102|4|608| +2450815|17104|4|455| +2450815|17107|4|696| +2450815|17108|4|282| +2450815|17110|4|21| +2450815|17113|4|773| +2450815|17114|4|364| +2450815|17116|4|638| +2450815|17119|4|241| +2450815|17120|4|199| +2450815|17122|4|617| +2450815|17125|4|986| +2450815|17126|4|656| +2450815|17128|4|868| +2450815|17131|4|764| +2450815|17132|4|10| +2450815|17134|4|549| +2450815|17137|4|707| +2450815|17138|4|176| +2450815|17140|4|491| +2450815|17143|4|478| +2450815|17144|4|505| +2450815|17146|4|848| +2450815|17149|4|215| +2450815|17150|4|| +2450815|17152|4|891| +2450815|17155|4|195| +2450815|17156|4|76| +2450815|17158|4|238| +2450815|17161|4|105| +2450815|17162|4|175| +2450815|17164|4|815| +2450815|17167|4|319| +2450815|17168|4|133| +2450815|17170|4|755| +2450815|17173|4|536| +2450815|17174|4|610| +2450815|17176|4|| +2450815|17179|4|247| +2450815|17180|4|875| +2450815|17182|4|552| +2450815|17185|4|873| +2450815|17186|4|647| +2450815|17188|4|434| +2450815|17191|4|493| +2450815|17192|4|886| +2450815|17194|4|381| +2450815|17197|4|647| +2450815|17198|4|141| +2450815|17200|4|777| +2450815|17203|4|641| +2450815|17204|4|194| +2450815|17206|4|762| +2450815|17209|4|218| +2450815|17210|4|386| +2450815|17212|4|995| +2450815|17215|4|815| +2450815|17216|4|783| +2450815|17218|4|616| +2450815|17221|4|579| +2450815|17222|4|828| +2450815|17224|4|663| +2450815|17227|4|519| +2450815|17228|4|390| +2450815|17230|4|748| +2450815|17233|4|419| +2450815|17234|4|306| +2450815|17236|4|606| +2450815|17239|4|371| +2450815|17240|4|463| +2450815|17242|4|234| +2450815|17245|4|809| +2450815|17246|4|170| +2450815|17248|4|690| +2450815|17251|4|783| +2450815|17252|4|204| +2450815|17254|4|870| +2450815|17257|4|879| +2450815|17258|4|308| +2450815|17260|4|927| +2450815|17263|4|790| +2450815|17264|4|228| +2450815|17266|4|590| +2450815|17269|4|699| +2450815|17270|4|705| +2450815|17272|4|500| +2450815|17275|4|570| +2450815|17276|4|738| +2450815|17278|4|251| +2450815|17281|4|863| +2450815|17282|4|937| +2450815|17284|4|714| +2450815|17287|4|74| +2450815|17288|4|532| +2450815|17290|4|337| +2450815|17293|4|37| +2450815|17294|4|897| +2450815|17296|4|531| +2450815|17299|4|627| +2450815|17300|4|341| +2450815|17302|4|122| +2450815|17305|4|88| +2450815|17306|4|233| +2450815|17308|4|308| +2450815|17311|4|933| +2450815|17312|4|516| +2450815|17314|4|362| +2450815|17317|4|389| +2450815|17318|4|821| +2450815|17320|4|510| +2450815|17323|4|947| +2450815|17324|4|435| +2450815|17326|4|843| +2450815|17329|4|209| +2450815|17330|4|793| +2450815|17332|4|| +2450815|17335|4|177| +2450815|17336|4|371| +2450815|17338|4|534| +2450815|17341|4|505| +2450815|17342|4|| +2450815|17344|4|511| +2450815|17347|4|524| +2450815|17348|4|536| +2450815|17350|4|974| +2450815|17353|4|683| +2450815|17354|4|119| +2450815|17356|4|275| +2450815|17359|4|189| +2450815|17360|4|814| +2450815|17362|4|704| +2450815|17365|4|263| +2450815|17366|4|985| +2450815|17368|4|982| +2450815|17371|4|570| +2450815|17372|4|310| +2450815|17374|4|320| +2450815|17377|4|844| +2450815|17378|4|583| +2450815|17380|4|370| +2450815|17383|4|370| +2450815|17384|4|881| +2450815|17386|4|393| +2450815|17389|4|643| +2450815|17390|4|840| +2450815|17392|4|779| +2450815|17395|4|435| +2450815|17396|4|235| +2450815|17398|4|869| +2450815|17401|4|417| +2450815|17402|4|334| +2450815|17404|4|441| +2450815|17407|4|537| +2450815|17408|4|394| +2450815|17410|4|912| +2450815|17413|4|237| +2450815|17414|4|676| +2450815|17416|4|994| +2450815|17419|4|755| +2450815|17420|4|107| +2450815|17422|4|846| +2450815|17425|4|741| +2450815|17426|4|446| +2450815|17428|4|| +2450815|17431|4|732| +2450815|17432|4|222| +2450815|17434|4|948| +2450815|17437|4|116| +2450815|17438|4|572| +2450815|17440|4|804| +2450815|17443|4|629| +2450815|17444|4|| +2450815|17446|4|91| +2450815|17449|4|957| +2450815|17450|4|282| +2450815|17452|4|574| +2450815|17455|4|504| +2450815|17456|4|446| +2450815|17458|4|647| +2450815|17461|4|274| +2450815|17462|4|189| +2450815|17464|4|243| +2450815|17467|4|531| +2450815|17468|4|198| +2450815|17470|4|949| +2450815|17473|4|307| +2450815|17474|4|2| +2450815|17476|4|| +2450815|17479|4|387| +2450815|17480|4|536| +2450815|17482|4|673| +2450815|17485|4|836| +2450815|17486|4|23| +2450815|17488|4|658| +2450815|17491|4|792| +2450815|17492|4|26| +2450815|17494|4|924| +2450815|17497|4|749| +2450815|17498|4|51| +2450815|17500|4|492| +2450815|17503|4|604| +2450815|17504|4|991| +2450815|17506|4|995| +2450815|17509|4|880| +2450815|17510|4|121| +2450815|17512|4|776| +2450815|17515|4|773| +2450815|17516|4|836| +2450815|17518|4|| +2450815|17521|4|902| +2450815|17522|4|20| +2450815|17524|4|471| +2450815|17527|4|592| +2450815|17528|4|961| +2450815|17530|4|394| +2450815|17533|4|787| +2450815|17534|4|757| +2450815|17536|4|231| +2450815|17539|4|230| +2450815|17540|4|523| +2450815|17542|4|894| +2450815|17545|4|748| +2450815|17546|4|641| +2450815|17548|4|809| +2450815|17551|4|374| +2450815|17552|4|251| +2450815|17554|4|300| +2450815|17557|4|205| +2450815|17558|4|| +2450815|17560|4|490| +2450815|17563|4|562| +2450815|17564|4|242| +2450815|17566|4|281| +2450815|17569|4|615| +2450815|17570|4|43| +2450815|17572|4|367| +2450815|17575|4|434| +2450815|17576|4|194| +2450815|17578|4|642| +2450815|17581|4|721| +2450815|17582|4|564| +2450815|17584|4|621| +2450815|17587|4|957| +2450815|17588|4|516| +2450815|17590|4|31| +2450815|17593|4|371| +2450815|17594|4|310| +2450815|17596|4|767| +2450815|17599|4|31| +2450815|17600|4|65| +2450815|17602|4|427| +2450815|17605|4|119| +2450815|17606|4|777| +2450815|17608|4|339| +2450815|17611|4|141| +2450815|17612|4|929| +2450815|17614|4|18| +2450815|17617|4|505| +2450815|17618|4|930| +2450815|17620|4|767| +2450815|17623|4|258| +2450815|17624|4|669| +2450815|17626|4|414| +2450815|17629|4|548| +2450815|17630|4|153| +2450815|17632|4|390| +2450815|17635|4|173| +2450815|17636|4|905| +2450815|17638|4|486| +2450815|17641|4|433| +2450815|17642|4|944| +2450815|17644|4|179| +2450815|17647|4|514| +2450815|17648|4|899| +2450815|17650|4|305| +2450815|17653|4|99| +2450815|17654|4|840| +2450815|17656|4|407| +2450815|17659|4|636| +2450815|17660|4|713| +2450815|17662|4|555| +2450815|17665|4|260| +2450815|17666|4|| +2450815|17668|4|652| +2450815|17671|4|148| +2450815|17672|4|253| +2450815|17674|4|731| +2450815|17677|4|444| +2450815|17678|4|561| +2450815|17680|4|334| +2450815|17683|4|268| +2450815|17684|4|30| +2450815|17686|4|141| +2450815|17689|4|883| +2450815|17690|4|528| +2450815|17692|4|792| +2450815|17695|4|539| +2450815|17696|4|819| +2450815|17698|4|93| +2450815|17701|4|711| +2450815|17702|4|157| +2450815|17704|4|621| +2450815|17707|4|916| +2450815|17708|4|429| +2450815|17710|4|434| +2450815|17713|4|284| +2450815|17714|4|132| +2450815|17716|4|531| +2450815|17719|4|976| +2450815|17720|4|801| +2450815|17722|4|898| +2450815|17725|4|901| +2450815|17726|4|276| +2450815|17728|4|78| +2450815|17731|4|413| +2450815|17732|4|469| +2450815|17734|4|610| +2450815|17737|4|149| +2450815|17738|4|696| +2450815|17740|4|734| +2450815|17743|4|891| +2450815|17744|4|343| +2450815|17746|4|612| +2450815|17749|4|| +2450815|17750|4|865| +2450815|17752|4|302| +2450815|17755|4|926| +2450815|17756|4|870| +2450815|17758|4|506| +2450815|17761|4|665| +2450815|17762|4|202| +2450815|17764|4|6| +2450815|17767|4|643| +2450815|17768|4|803| +2450815|17770|4|916| +2450815|17773|4|81| +2450815|17774|4|261| +2450815|17776|4|561| +2450815|17779|4|345| +2450815|17780|4|487| +2450815|17782|4|274| +2450815|17785|4|955| +2450815|17786|4|645| +2450815|17788|4|599| +2450815|17791|4|660| +2450815|17792|4|195| +2450815|17794|4|413| +2450815|17797|4|776| +2450815|17798|4|298| +2450815|17800|4|591| +2450815|17803|4|508| +2450815|17804|4|| +2450815|17806|4|6| +2450815|17809|4|698| +2450815|17810|4|256| +2450815|17812|4|572| +2450815|17815|4|844| +2450815|17816|4|869| +2450815|17818|4|| +2450815|17821|4|572| +2450815|17822|4|362| +2450815|17824|4|422| +2450815|17827|4|276| +2450815|17828|4|62| +2450815|17830|4|802| +2450815|17833|4|165| +2450815|17834|4|82| +2450815|17836|4|899| +2450815|17839|4|368| +2450815|17840|4|579| +2450815|17842|4|164| +2450815|17845|4|922| +2450815|17846|4|823| +2450815|17848|4|496| +2450815|17851|4|384| +2450815|17852|4|940| +2450815|17854|4|191| +2450815|17857|4|395| +2450815|17858|4|197| +2450815|17860|4|749| +2450815|17863|4|312| +2450815|17864|4|537| +2450815|17866|4|204| +2450815|17869|4|49| +2450815|17870|4|151| +2450815|17872|4|60| +2450815|17875|4|106| +2450815|17876|4|842| +2450815|17878|4|531| +2450815|17881|4|547| +2450815|17882|4|135| +2450815|17884|4|19| +2450815|17887|4|18| +2450815|17888|4|967| +2450815|17890|4|626| +2450815|17893|4|427| +2450815|17894|4|913| +2450815|17896|4|481| +2450815|17899|4|566| +2450815|17900|4|132| +2450815|17902|4|742| +2450815|17905|4|783| +2450815|17906|4|691| +2450815|17908|4|708| +2450815|17911|4|| +2450815|17912|4|986| +2450815|17914|4|875| +2450815|17917|4|320| +2450815|17918|4|770| +2450815|17920|4|212| +2450815|17923|4|798| +2450815|17924|4|116| +2450815|17926|4|451| +2450815|17929|4|724| +2450815|17930|4|628| +2450815|17932|4|193| +2450815|17935|4|197| +2450815|17936|4|465| +2450815|17938|4|262| +2450815|17941|4|751| +2450815|17942|4|37| +2450815|17944|4|43| +2450815|17947|4|798| +2450815|17948|4|920| +2450815|17950|4|281| +2450815|17953|4|697| +2450815|17954|4|793| +2450815|17956|4|788| +2450815|17959|4|687| +2450815|17960|4|747| +2450815|17962|4|795| +2450815|17965|4|79| +2450815|17966|4|670| +2450815|17968|4|574| +2450815|17971|4|811| +2450815|17972|4|626| +2450815|17974|4|411| +2450815|17977|4|761| +2450815|17978|4|531| +2450815|17980|4|987| +2450815|17983|4|231| +2450815|17984|4|440| +2450815|17986|4|265| +2450815|17989|4|410| +2450815|17990|4|807| +2450815|17992|4|480| +2450815|17995|4|124| +2450815|17996|4|272| +2450815|17998|4|565| +2450815|1|5|873| +2450815|2|5|625| +2450815|4|5|360| +2450815|7|5|143| +2450815|8|5|886| +2450815|10|5|947| +2450815|13|5|884| +2450815|14|5|| +2450815|16|5|334| +2450815|19|5|628| +2450815|20|5|948| +2450815|22|5|| +2450815|25|5|| +2450815|26|5|395| +2450815|28|5|996| +2450815|31|5|831| +2450815|32|5|794| +2450815|34|5|970| +2450815|37|5|| +2450815|38|5|626| +2450815|40|5|18| +2450815|43|5|595| +2450815|44|5|125| +2450815|46|5|610| +2450815|49|5|354| +2450815|50|5|988| +2450815|52|5|302| +2450815|55|5|819| +2450815|56|5|| +2450815|58|5|586| +2450815|61|5|493| +2450815|62|5|483| +2450815|64|5|663| +2450815|67|5|243| +2450815|68|5|653| +2450815|70|5|| +2450815|73|5|273| +2450815|74|5|365| +2450815|76|5|| +2450815|79|5|974| +2450815|80|5|322| +2450815|82|5|259| +2450815|85|5|96| +2450815|86|5|| +2450815|88|5|199| +2450815|91|5|282| +2450815|92|5|| +2450815|94|5|514| +2450815|97|5|654| +2450815|98|5|324| +2450815|100|5|414| +2450815|103|5|381| +2450815|104|5|369| +2450815|106|5|576| +2450815|109|5|977| +2450815|110|5|24| +2450815|112|5|943| +2450815|115|5|880| +2450815|116|5|155| +2450815|118|5|348| +2450815|121|5|563| +2450815|122|5|307| +2450815|124|5|938| +2450815|127|5|185| +2450815|128|5|73| +2450815|130|5|853| +2450815|133|5|561| +2450815|134|5|561| +2450815|136|5|175| +2450815|139|5|734| +2450815|140|5|847| +2450815|142|5|130| +2450815|145|5|103| +2450815|146|5|483| +2450815|148|5|479| +2450815|151|5|243| +2450815|152|5|783| +2450815|154|5|711| +2450815|157|5|862| +2450815|158|5|591| +2450815|160|5|404| +2450815|163|5|365| +2450815|164|5|110| +2450815|166|5|68| +2450815|169|5|793| +2450815|170|5|428| +2450815|172|5|220| +2450815|175|5|517| +2450815|176|5|936| +2450815|178|5|465| +2450815|181|5|811| +2450815|182|5|686| +2450815|184|5|916| +2450815|187|5|912| +2450815|188|5|432| +2450815|190|5|38| +2450815|193|5|155| +2450815|194|5|736| +2450815|196|5|868| +2450815|199|5|455| +2450815|200|5|552| +2450815|202|5|569| +2450815|205|5|141| +2450815|206|5|109| +2450815|208|5|749| +2450815|211|5|655| +2450815|212|5|890| +2450815|214|5|361| +2450815|217|5|878| +2450815|218|5|274| +2450815|220|5|965| +2450815|223|5|972| +2450815|224|5|818| +2450815|226|5|794| +2450815|229|5|768| +2450815|230|5|98| +2450815|232|5|803| +2450815|235|5|186| +2450815|236|5|239| +2450815|238|5|40| +2450815|241|5|570| +2450815|242|5|665| +2450815|244|5|758| +2450815|247|5|| +2450815|248|5|648| +2450815|250|5|529| +2450815|253|5|25| +2450815|254|5|782| +2450815|256|5|438| +2450815|259|5|981| +2450815|260|5|456| +2450815|262|5|579| +2450815|265|5|944| +2450815|266|5|325| +2450815|268|5|516| +2450815|271|5|| +2450815|272|5|170| +2450815|274|5|157| +2450815|277|5|564| +2450815|278|5|579| +2450815|280|5|989| +2450815|283|5|843| +2450815|284|5|113| +2450815|286|5|293| +2450815|289|5|987| +2450815|290|5|5| +2450815|292|5|464| +2450815|295|5|382| +2450815|296|5|298| +2450815|298|5|360| +2450815|301|5|409| +2450815|302|5|448| +2450815|304|5|392| +2450815|307|5|228| +2450815|308|5|971| +2450815|310|5|776| +2450815|313|5|720| +2450815|314|5|| +2450815|316|5|886| +2450815|319|5|28| +2450815|320|5|209| +2450815|322|5|824| +2450815|325|5|927| +2450815|326|5|732| +2450815|328|5|24| +2450815|331|5|654| +2450815|332|5|530| +2450815|334|5|750| +2450815|337|5|953| +2450815|338|5|479| +2450815|340|5|695| +2450815|343|5|897| +2450815|344|5|413| +2450815|346|5|367| +2450815|349|5|106| +2450815|350|5|203| +2450815|352|5|451| +2450815|355|5|498| +2450815|356|5|869| +2450815|358|5|275| +2450815|361|5|11| +2450815|362|5|| +2450815|364|5|905| +2450815|367|5|621| +2450815|368|5|996| +2450815|370|5|429| +2450815|373|5|639| +2450815|374|5|221| +2450815|376|5|823| +2450815|379|5|744| +2450815|380|5|185| +2450815|382|5|493| +2450815|385|5|43| +2450815|386|5|309| +2450815|388|5|437| +2450815|391|5|275| +2450815|392|5|596| +2450815|394|5|34| +2450815|397|5|815| +2450815|398|5|79| +2450815|400|5|704| +2450815|403|5|232| +2450815|404|5|409| +2450815|406|5|540| +2450815|409|5|138| +2450815|410|5|605| +2450815|412|5|660| +2450815|415|5|932| +2450815|416|5|546| +2450815|418|5|896| +2450815|421|5|387| +2450815|422|5|908| +2450815|424|5|485| +2450815|427|5|842| +2450815|428|5|557| +2450815|430|5|70| +2450815|433|5|549| +2450815|434|5|362| +2450815|436|5|318| +2450815|439|5|280| +2450815|440|5|34| +2450815|442|5|773| +2450815|445|5|752| +2450815|446|5|928| +2450815|448|5|525| +2450815|451|5|623| +2450815|452|5|183| +2450815|454|5|304| +2450815|457|5|907| +2450815|458|5|132| +2450815|460|5|422| +2450815|463|5|909| +2450815|464|5|63| +2450815|466|5|| +2450815|469|5|472| +2450815|470|5|837| +2450815|472|5|873| +2450815|475|5|995| +2450815|476|5|392| +2450815|478|5|3| +2450815|481|5|892| +2450815|482|5|605| +2450815|484|5|836| +2450815|487|5|895| +2450815|488|5|841| +2450815|490|5|400| +2450815|493|5|642| +2450815|494|5|705| +2450815|496|5|600| +2450815|499|5|899| +2450815|500|5|996| +2450815|502|5|53| +2450815|505|5|537| +2450815|506|5|511| +2450815|508|5|543| +2450815|511|5|442| +2450815|512|5|329| +2450815|514|5|419| +2450815|517|5|852| +2450815|518|5|897| +2450815|520|5|894| +2450815|523|5|| +2450815|524|5|496| +2450815|526|5|380| +2450815|529|5|422| +2450815|530|5|72| +2450815|532|5|999| +2450815|535|5|645| +2450815|536|5|993| +2450815|538|5|23| +2450815|541|5|| +2450815|542|5|239| +2450815|544|5|394| +2450815|547|5|899| +2450815|548|5|501| +2450815|550|5|987| +2450815|553|5|303| +2450815|554|5|615| +2450815|556|5|646| +2450815|559|5|351| +2450815|560|5|| +2450815|562|5|641| +2450815|565|5|719| +2450815|566|5|981| +2450815|568|5|115| +2450815|571|5|688| +2450815|572|5|449| +2450815|574|5|637| +2450815|577|5|687| +2450815|578|5|794| +2450815|580|5|934| +2450815|583|5|12| +2450815|584|5|554| +2450815|586|5|51| +2450815|589|5|683| +2450815|590|5|469| +2450815|592|5|909| +2450815|595|5|291| +2450815|596|5|804| +2450815|598|5|723| +2450815|601|5|488| +2450815|602|5|612| +2450815|604|5|755| +2450815|607|5|207| +2450815|608|5|88| +2450815|610|5|348| +2450815|613|5|78| +2450815|614|5|352| +2450815|616|5|994| +2450815|619|5|596| +2450815|620|5|676| +2450815|622|5|992| +2450815|625|5|31| +2450815|626|5|476| +2450815|628|5|352| +2450815|631|5|81| +2450815|632|5|442| +2450815|634|5|650| +2450815|637|5|330| +2450815|638|5|505| +2450815|640|5|798| +2450815|643|5|451| +2450815|644|5|72| +2450815|646|5|844| +2450815|649|5|370| +2450815|650|5|393| +2450815|652|5|846| +2450815|655|5|622| +2450815|656|5|374| +2450815|658|5|567| +2450815|661|5|805| +2450815|662|5|730| +2450815|664|5|734| +2450815|667|5|959| +2450815|668|5|586| +2450815|670|5|574| +2450815|673|5|701| +2450815|674|5|300| +2450815|676|5|954| +2450815|679|5|573| +2450815|680|5|946| +2450815|682|5|364| +2450815|685|5|347| +2450815|686|5|302| +2450815|688|5|709| +2450815|691|5|594| +2450815|692|5|111| +2450815|694|5|444| +2450815|697|5|681| +2450815|698|5|616| +2450815|700|5|69| +2450815|703|5|545| +2450815|704|5|439| +2450815|706|5|233| +2450815|709|5|148| +2450815|710|5|759| +2450815|712|5|224| +2450815|715|5|157| +2450815|716|5|902| +2450815|718|5|931| +2450815|721|5|982| +2450815|722|5|464| +2450815|724|5|623| +2450815|727|5|188| +2450815|728|5|129| +2450815|730|5|804| +2450815|733|5|125| +2450815|734|5|866| +2450815|736|5|276| +2450815|739|5|172| +2450815|740|5|121| +2450815|742|5|941| +2450815|745|5|810| +2450815|746|5|894| +2450815|748|5|671| +2450815|751|5|471| +2450815|752|5|739| +2450815|754|5|296| +2450815|757|5|444| +2450815|758|5|899| +2450815|760|5|490| +2450815|763|5|417| +2450815|764|5|282| +2450815|766|5|791| +2450815|769|5|734| +2450815|770|5|553| +2450815|772|5|194| +2450815|775|5|519| +2450815|776|5|813| +2450815|778|5|780| +2450815|781|5|746| +2450815|782|5|633| +2450815|784|5|552| +2450815|787|5|50| +2450815|788|5|722| +2450815|790|5|| +2450815|793|5|854| +2450815|794|5|826| +2450815|796|5|| +2450815|799|5|323| +2450815|800|5|422| +2450815|802|5|153| +2450815|805|5|549| +2450815|806|5|533| +2450815|808|5|247| +2450815|811|5|361| +2450815|812|5|709| +2450815|814|5|59| +2450815|817|5|441| +2450815|818|5|275| +2450815|820|5|165| +2450815|823|5|3| +2450815|824|5|724| +2450815|826|5|598| +2450815|829|5|946| +2450815|830|5|979| +2450815|832|5|942| +2450815|835|5|665| +2450815|836|5|417| +2450815|838|5|| +2450815|841|5|355| +2450815|842|5|513| +2450815|844|5|229| +2450815|847|5|120| +2450815|848|5|1| +2450815|850|5|184| +2450815|853|5|728| +2450815|854|5|762| +2450815|856|5|670| +2450815|859|5|482| +2450815|860|5|148| +2450815|862|5|280| +2450815|865|5|351| +2450815|866|5|786| +2450815|868|5|112| +2450815|871|5|851| +2450815|872|5|66| +2450815|874|5|482| +2450815|877|5|147| +2450815|878|5|493| +2450815|880|5|| +2450815|883|5|310| +2450815|884|5|665| +2450815|886|5|473| +2450815|889|5|544| +2450815|890|5|127| +2450815|892|5|877| +2450815|895|5|430| +2450815|896|5|39| +2450815|898|5|362| +2450815|901|5|200| +2450815|902|5|492| +2450815|904|5|811| +2450815|907|5|517| +2450815|908|5|414| +2450815|910|5|26| +2450815|913|5|728| +2450815|914|5|281| +2450815|916|5|586| +2450815|919|5|341| +2450815|920|5|208| +2450815|922|5|17| +2450815|925|5|335| +2450815|926|5|118| +2450815|928|5|812| +2450815|931|5|481| +2450815|932|5|806| +2450815|934|5|246| +2450815|937|5|| +2450815|938|5|754| +2450815|940|5|430| +2450815|943|5|220| +2450815|944|5|546| +2450815|946|5|585| +2450815|949|5|332| +2450815|950|5|63| +2450815|952|5|180| +2450815|955|5|843| +2450815|956|5|410| +2450815|958|5|806| +2450815|961|5|79| +2450815|962|5|331| +2450815|964|5|663| +2450815|967|5|81| +2450815|968|5|654| +2450815|970|5|234| +2450815|973|5|870| +2450815|974|5|502| +2450815|976|5|895| +2450815|979|5|454| +2450815|980|5|299| +2450815|982|5|932| +2450815|985|5|650| +2450815|986|5|135| +2450815|988|5|475| +2450815|991|5|206| +2450815|992|5|841| +2450815|994|5|321| +2450815|997|5|447| +2450815|998|5|294| +2450815|1000|5|912| +2450815|1003|5|398| +2450815|1004|5|613| +2450815|1006|5|7| +2450815|1009|5|731| +2450815|1010|5|573| +2450815|1012|5|210| +2450815|1015|5|686| +2450815|1016|5|659| +2450815|1018|5|411| +2450815|1021|5|318| +2450815|1022|5|230| +2450815|1024|5|933| +2450815|1027|5|399| +2450815|1028|5|607| +2450815|1030|5|581| +2450815|1033|5|725| +2450815|1034|5|406| +2450815|1036|5|492| +2450815|1039|5|142| +2450815|1040|5|519| +2450815|1042|5|995| +2450815|1045|5|905| +2450815|1046|5|128| +2450815|1048|5|216| +2450815|1051|5|156| +2450815|1052|5|219| +2450815|1054|5|194| +2450815|1057|5|918| +2450815|1058|5|808| +2450815|1060|5|288| +2450815|1063|5|179| +2450815|1064|5|965| +2450815|1066|5|344| +2450815|1069|5|72| +2450815|1070|5|658| +2450815|1072|5|405| +2450815|1075|5|850| +2450815|1076|5|959| +2450815|1078|5|378| +2450815|1081|5|81| +2450815|1082|5|296| +2450815|1084|5|184| +2450815|1087|5|503| +2450815|1088|5|984| +2450815|1090|5|797| +2450815|1093|5|862| +2450815|1094|5|930| +2450815|1096|5|180| +2450815|1099|5|402| +2450815|1100|5|577| +2450815|1102|5|788| +2450815|1105|5|848| +2450815|1106|5|| +2450815|1108|5|883| +2450815|1111|5|325| +2450815|1112|5|648| +2450815|1114|5|| +2450815|1117|5|776| +2450815|1118|5|699| +2450815|1120|5|664| +2450815|1123|5|724| +2450815|1124|5|330| +2450815|1126|5|811| +2450815|1129|5|5| +2450815|1130|5|318| +2450815|1132|5|783| +2450815|1135|5|30| +2450815|1136|5|933| +2450815|1138|5|425| +2450815|1141|5|582| +2450815|1142|5|520| +2450815|1144|5|624| +2450815|1147|5|614| +2450815|1148|5|768| +2450815|1150|5|347| +2450815|1153|5|335| +2450815|1154|5|| +2450815|1156|5|874| +2450815|1159|5|486| +2450815|1160|5|155| +2450815|1162|5|766| +2450815|1165|5|956| +2450815|1166|5|106| +2450815|1168|5|471| +2450815|1171|5|| +2450815|1172|5|472| +2450815|1174|5|964| +2450815|1177|5|350| +2450815|1178|5|374| +2450815|1180|5|527| +2450815|1183|5|179| +2450815|1184|5|929| +2450815|1186|5|379| +2450815|1189|5|271| +2450815|1190|5|258| +2450815|1192|5|955| +2450815|1195|5|| +2450815|1196|5|903| +2450815|1198|5|661| +2450815|1201|5|792| +2450815|1202|5|211| +2450815|1204|5|759| +2450815|1207|5|145| +2450815|1208|5|316| +2450815|1210|5|612| +2450815|1213|5|794| +2450815|1214|5|473| +2450815|1216|5|| +2450815|1219|5|281| +2450815|1220|5|7| +2450815|1222|5|982| +2450815|1225|5|764| +2450815|1226|5|898| +2450815|1228|5|37| +2450815|1231|5|836| +2450815|1232|5|382| +2450815|1234|5|772| +2450815|1237|5|994| +2450815|1238|5|658| +2450815|1240|5|139| +2450815|1243|5|27| +2450815|1244|5|603| +2450815|1246|5|884| +2450815|1249|5|| +2450815|1250|5|448| +2450815|1252|5|99| +2450815|1255|5|434| +2450815|1256|5|| +2450815|1258|5|27| +2450815|1261|5|345| +2450815|1262|5|379| +2450815|1264|5|203| +2450815|1267|5|967| +2450815|1268|5|83| +2450815|1270|5|129| +2450815|1273|5|820| +2450815|1274|5|173| +2450815|1276|5|617| +2450815|1279|5|955| +2450815|1280|5|481| +2450815|1282|5|98| +2450815|1285|5|141| +2450815|1286|5|| +2450815|1288|5|785| +2450815|1291|5|816| +2450815|1292|5|55| +2450815|1294|5|221| +2450815|1297|5|349| +2450815|1298|5|7| +2450815|1300|5|862| +2450815|1303|5|893| +2450815|1304|5|807| +2450815|1306|5|890| +2450815|1309|5|731| +2450815|1310|5|831| +2450815|1312|5|445| +2450815|1315|5|510| +2450815|1316|5|563| +2450815|1318|5|204| +2450815|1321|5|885| +2450815|1322|5|520| +2450815|1324|5|553| +2450815|1327|5|47| +2450815|1328|5|862| +2450815|1330|5|94| +2450815|1333|5|835| +2450815|1334|5|900| +2450815|1336|5|54| +2450815|1339|5|113| +2450815|1340|5|637| +2450815|1342|5|608| +2450815|1345|5|| +2450815|1346|5|| +2450815|1348|5|853| +2450815|1351|5|432| +2450815|1352|5|310| +2450815|1354|5|855| +2450815|1357|5|656| +2450815|1358|5|494| +2450815|1360|5|101| +2450815|1363|5|403| +2450815|1364|5|728| +2450815|1366|5|794| +2450815|1369|5|635| +2450815|1370|5|666| +2450815|1372|5|| +2450815|1375|5|177| +2450815|1376|5|892| +2450815|1378|5|51| +2450815|1381|5|587| +2450815|1382|5|648| +2450815|1384|5|13| +2450815|1387|5|679| +2450815|1388|5|374| +2450815|1390|5|516| +2450815|1393|5|764| +2450815|1394|5|669| +2450815|1396|5|3| +2450815|1399|5|416| +2450815|1400|5|185| +2450815|1402|5|428| +2450815|1405|5|877| +2450815|1406|5|263| +2450815|1408|5|675| +2450815|1411|5|131| +2450815|1412|5|100| +2450815|1414|5|340| +2450815|1417|5|805| +2450815|1418|5|654| +2450815|1420|5|469| +2450815|1423|5|657| +2450815|1424|5|184| +2450815|1426|5|175| +2450815|1429|5|324| +2450815|1430|5|494| +2450815|1432|5|936| +2450815|1435|5|269| +2450815|1436|5|764| +2450815|1438|5|69| +2450815|1441|5|372| +2450815|1442|5|306| +2450815|1444|5|610| +2450815|1447|5|938| +2450815|1448|5|773| +2450815|1450|5|90| +2450815|1453|5|495| +2450815|1454|5|660| +2450815|1456|5|595| +2450815|1459|5|956| +2450815|1460|5|438| +2450815|1462|5|766| +2450815|1465|5|54| +2450815|1466|5|735| +2450815|1468|5|469| +2450815|1471|5|738| +2450815|1472|5|727| +2450815|1474|5|176| +2450815|1477|5|921| +2450815|1478|5|589| +2450815|1480|5|836| +2450815|1483|5|| +2450815|1484|5|29| +2450815|1486|5|666| +2450815|1489|5|| +2450815|1490|5|813| +2450815|1492|5|156| +2450815|1495|5|449| +2450815|1496|5|471| +2450815|1498|5|103| +2450815|1501|5|514| +2450815|1502|5|201| +2450815|1504|5|503| +2450815|1507|5|380| +2450815|1508|5|769| +2450815|1510|5|256| +2450815|1513|5|907| +2450815|1514|5|303| +2450815|1516|5|525| +2450815|1519|5|770| +2450815|1520|5|20| +2450815|1522|5|230| +2450815|1525|5|894| +2450815|1526|5|775| +2450815|1528|5|715| +2450815|1531|5|369| +2450815|1532|5|291| +2450815|1534|5|813| +2450815|1537|5|| +2450815|1538|5|958| +2450815|1540|5|995| +2450815|1543|5|395| +2450815|1544|5|253| +2450815|1546|5|439| +2450815|1549|5|481| +2450815|1550|5|528| +2450815|1552|5|882| +2450815|1555|5|696| +2450815|1556|5|802| +2450815|1558|5|155| +2450815|1561|5|76| +2450815|1562|5|693| +2450815|1564|5|334| +2450815|1567|5|41| +2450815|1568|5|382| +2450815|1570|5|761| +2450815|1573|5|83| +2450815|1574|5|621| +2450815|1576|5|896| +2450815|1579|5|646| +2450815|1580|5|800| +2450815|1582|5|806| +2450815|1585|5|412| +2450815|1586|5|18| +2450815|1588|5|630| +2450815|1591|5|295| +2450815|1592|5|400| +2450815|1594|5|52| +2450815|1597|5|761| +2450815|1598|5|861| +2450815|1600|5|| +2450815|1603|5|764| +2450815|1604|5|738| +2450815|1606|5|| +2450815|1609|5|| +2450815|1610|5|930| +2450815|1612|5|713| +2450815|1615|5|60| +2450815|1616|5|| +2450815|1618|5|953| +2450815|1621|5|828| +2450815|1622|5|587| +2450815|1624|5|482| +2450815|1627|5|929| +2450815|1628|5|160| +2450815|1630|5|502| +2450815|1633|5|978| +2450815|1634|5|784| +2450815|1636|5|854| +2450815|1639|5|62| +2450815|1640|5|573| +2450815|1642|5|| +2450815|1645|5|29| +2450815|1646|5|226| +2450815|1648|5|231| +2450815|1651|5|409| +2450815|1652|5|754| +2450815|1654|5|102| +2450815|1657|5|336| +2450815|1658|5|716| +2450815|1660|5|589| +2450815|1663|5|944| +2450815|1664|5|668| +2450815|1666|5|663| +2450815|1669|5|275| +2450815|1670|5|786| +2450815|1672|5|167| +2450815|1675|5|946| +2450815|1676|5|510| +2450815|1678|5|915| +2450815|1681|5|867| +2450815|1682|5|246| +2450815|1684|5|684| +2450815|1687|5|677| +2450815|1688|5|792| +2450815|1690|5|352| +2450815|1693|5|325| +2450815|1694|5|| +2450815|1696|5|729| +2450815|1699|5|306| +2450815|1700|5|45| +2450815|1702|5|698| +2450815|1705|5|25| +2450815|1706|5|836| +2450815|1708|5|172| +2450815|1711|5|279| +2450815|1712|5|965| +2450815|1714|5|937| +2450815|1717|5|934| +2450815|1718|5|401| +2450815|1720|5|653| +2450815|1723|5|661| +2450815|1724|5|225| +2450815|1726|5|647| +2450815|1729|5|954| +2450815|1730|5|277| +2450815|1732|5|349| +2450815|1735|5|165| +2450815|1736|5|758| +2450815|1738|5|527| +2450815|1741|5|620| +2450815|1742|5|792| +2450815|1744|5|639| +2450815|1747|5|199| +2450815|1748|5|524| +2450815|1750|5|378| +2450815|1753|5|504| +2450815|1754|5|846| +2450815|1756|5|902| +2450815|1759|5|318| +2450815|1760|5|361| +2450815|1762|5|797| +2450815|1765|5|477| +2450815|1766|5|422| +2450815|1768|5|352| +2450815|1771|5|681| +2450815|1772|5|506| +2450815|1774|5|160| +2450815|1777|5|960| +2450815|1778|5|850| +2450815|1780|5|350| +2450815|1783|5|698| +2450815|1784|5|107| +2450815|1786|5|964| +2450815|1789|5|494| +2450815|1790|5|48| +2450815|1792|5|930| +2450815|1795|5|460| +2450815|1796|5|189| +2450815|1798|5|281| +2450815|1801|5|455| +2450815|1802|5|554| +2450815|1804|5|371| +2450815|1807|5|207| +2450815|1808|5|362| +2450815|1810|5|885| +2450815|1813|5|195| +2450815|1814|5|666| +2450815|1816|5|566| +2450815|1819|5|906| +2450815|1820|5|430| +2450815|1822|5|754| +2450815|1825|5|523| +2450815|1826|5|140| +2450815|1828|5|675| +2450815|1831|5|616| +2450815|1832|5|282| +2450815|1834|5|43| +2450815|1837|5|2| +2450815|1838|5|946| +2450815|1840|5|189| +2450815|1843|5|525| +2450815|1844|5|683| +2450815|1846|5|582| +2450815|1849|5|957| +2450815|1850|5|575| +2450815|1852|5|336| +2450815|1855|5|79| +2450815|1856|5|779| +2450815|1858|5|627| +2450815|1861|5|355| +2450815|1862|5|433| +2450815|1864|5|| +2450815|1867|5|80| +2450815|1868|5|848| +2450815|1870|5|44| +2450815|1873|5|424| +2450815|1874|5|264| +2450815|1876|5|285| +2450815|1879|5|943| +2450815|1880|5|25| +2450815|1882|5|532| +2450815|1885|5|852| +2450815|1886|5|200| +2450815|1888|5|742| +2450815|1891|5|297| +2450815|1892|5|332| +2450815|1894|5|232| +2450815|1897|5|37| +2450815|1898|5|874| +2450815|1900|5|944| +2450815|1903|5|31| +2450815|1904|5|395| +2450815|1906|5|411| +2450815|1909|5|277| +2450815|1910|5|276| +2450815|1912|5|| +2450815|1915|5|443| +2450815|1916|5|848| +2450815|1918|5|618| +2450815|1921|5|97| +2450815|1922|5|301| +2450815|1924|5|1| +2450815|1927|5|507| +2450815|1928|5|891| +2450815|1930|5|930| +2450815|1933|5|750| +2450815|1934|5|513| +2450815|1936|5|250| +2450815|1939|5|478| +2450815|1940|5|33| +2450815|1942|5|164| +2450815|1945|5|127| +2450815|1946|5|| +2450815|1948|5|88| +2450815|1951|5|598| +2450815|1952|5|217| +2450815|1954|5|920| +2450815|1957|5|158| +2450815|1958|5|281| +2450815|1960|5|135| +2450815|1963|5|444| +2450815|1964|5|571| +2450815|1966|5|35| +2450815|1969|5|368| +2450815|1970|5|199| +2450815|1972|5|220| +2450815|1975|5|952| +2450815|1976|5|1| +2450815|1978|5|| +2450815|1981|5|137| +2450815|1982|5|697| +2450815|1984|5|65| +2450815|1987|5|897| +2450815|1988|5|537| +2450815|1990|5|637| +2450815|1993|5|616| +2450815|1994|5|955| +2450815|1996|5|108| +2450815|1999|5|85| +2450815|2000|5|292| +2450815|2002|5|397| +2450815|2005|5|1| +2450815|2006|5|998| +2450815|2008|5|140| +2450815|2011|5|465| +2450815|2012|5|549| +2450815|2014|5|679| +2450815|2017|5|447| +2450815|2018|5|102| +2450815|2020|5|658| +2450815|2023|5|542| +2450815|2024|5|843| +2450815|2026|5|983| +2450815|2029|5|257| +2450815|2030|5|333| +2450815|2032|5|105| +2450815|2035|5|858| +2450815|2036|5|162| +2450815|2038|5|159| +2450815|2041|5|708| +2450815|2042|5|505| +2450815|2044|5|523| +2450815|2047|5|739| +2450815|2048|5|690| +2450815|2050|5|991| +2450815|2053|5|734| +2450815|2054|5|461| +2450815|2056|5|886| +2450815|2059|5|| +2450815|2060|5|384| +2450815|2062|5|312| +2450815|2065|5|988| +2450815|2066|5|322| +2450815|2068|5|124| +2450815|2071|5|251| +2450815|2072|5|116| +2450815|2074|5|30| +2450815|2077|5|521| +2450815|2078|5|638| +2450815|2080|5|162| +2450815|2083|5|139| +2450815|2084|5|324| +2450815|2086|5|275| +2450815|2089|5|740| +2450815|2090|5|712| +2450815|2092|5|| +2450815|2095|5|415| +2450815|2096|5|708| +2450815|2098|5|228| +2450815|2101|5|353| +2450815|2102|5|225| +2450815|2104|5|567| +2450815|2107|5|791| +2450815|2108|5|496| +2450815|2110|5|753| +2450815|2113|5|250| +2450815|2114|5|139| +2450815|2116|5|732| +2450815|2119|5|142| +2450815|2120|5|68| +2450815|2122|5|172| +2450815|2125|5|| +2450815|2126|5|468| +2450815|2128|5|700| +2450815|2131|5|87| +2450815|2132|5|465| +2450815|2134|5|557| +2450815|2137|5|69| +2450815|2138|5|878| +2450815|2140|5|727| +2450815|2143|5|316| +2450815|2144|5|587| +2450815|2146|5|584| +2450815|2149|5|249| +2450815|2150|5|331| +2450815|2152|5|| +2450815|2155|5|| +2450815|2156|5|302| +2450815|2158|5|540| +2450815|2161|5|760| +2450815|2162|5|667| +2450815|2164|5|504| +2450815|2167|5|197| +2450815|2168|5|797| +2450815|2170|5|254| +2450815|2173|5|512| +2450815|2174|5|187| +2450815|2176|5|779| +2450815|2179|5|644| +2450815|2180|5|861| +2450815|2182|5|269| +2450815|2185|5|643| +2450815|2186|5|974| +2450815|2188|5|746| +2450815|2191|5|540| +2450815|2192|5|935| +2450815|2194|5|| +2450815|2197|5|663| +2450815|2198|5|| +2450815|2200|5|381| +2450815|2203|5|700| +2450815|2204|5|289| +2450815|2206|5|275| +2450815|2209|5|634| +2450815|2210|5|513| +2450815|2212|5|717| +2450815|2215|5|| +2450815|2216|5|994| +2450815|2218|5|517| +2450815|2221|5|434| +2450815|2222|5|520| +2450815|2224|5|992| +2450815|2227|5|388| +2450815|2228|5|458| +2450815|2230|5|363| +2450815|2233|5|443| +2450815|2234|5|666| +2450815|2236|5|932| +2450815|2239|5|953| +2450815|2240|5|798| +2450815|2242|5|147| +2450815|2245|5|946| +2450815|2246|5|835| +2450815|2248|5|794| +2450815|2251|5|| +2450815|2252|5|544| +2450815|2254|5|105| +2450815|2257|5|970| +2450815|2258|5|625| +2450815|2260|5|176| +2450815|2263|5|984| +2450815|2264|5|401| +2450815|2266|5|853| +2450815|2269|5|597| +2450815|2270|5|33| +2450815|2272|5|695| +2450815|2275|5|345| +2450815|2276|5|784| +2450815|2278|5|553| +2450815|2281|5|563| +2450815|2282|5|858| +2450815|2284|5|554| +2450815|2287|5|357| +2450815|2288|5|| +2450815|2290|5|61| +2450815|2293|5|719| +2450815|2294|5|869| +2450815|2296|5|933| +2450815|2299|5|413| +2450815|2300|5|254| +2450815|2302|5|1000| +2450815|2305|5|813| +2450815|2306|5|685| +2450815|2308|5|713| +2450815|2311|5|357| +2450815|2312|5|148| +2450815|2314|5|838| +2450815|2317|5|1000| +2450815|2318|5|108| +2450815|2320|5|571| +2450815|2323|5|537| +2450815|2324|5|402| +2450815|2326|5|671| +2450815|2329|5|446| +2450815|2330|5|473| +2450815|2332|5|80| +2450815|2335|5|724| +2450815|2336|5|460| +2450815|2338|5|248| +2450815|2341|5|567| +2450815|2342|5|960| +2450815|2344|5|867| +2450815|2347|5|206| +2450815|2348|5|103| +2450815|2350|5|657| +2450815|2353|5|810| +2450815|2354|5|568| +2450815|2356|5|485| +2450815|2359|5|925| +2450815|2360|5|658| +2450815|2362|5|981| +2450815|2365|5|700| +2450815|2366|5|713| +2450815|2368|5|35| +2450815|2371|5|259| +2450815|2372|5|691| +2450815|2374|5|155| +2450815|2377|5|461| +2450815|2378|5|63| +2450815|2380|5|154| +2450815|2383|5|411| +2450815|2384|5|509| +2450815|2386|5|705| +2450815|2389|5|574| +2450815|2390|5|432| +2450815|2392|5|272| +2450815|2395|5|518| +2450815|2396|5|| +2450815|2398|5|385| +2450815|2401|5|333| +2450815|2402|5|| +2450815|2404|5|979| +2450815|2407|5|670| +2450815|2408|5|37| +2450815|2410|5|673| +2450815|2413|5|| +2450815|2414|5|507| +2450815|2416|5|723| +2450815|2419|5|548| +2450815|2420|5|963| +2450815|2422|5|47| +2450815|2425|5|231| +2450815|2426|5|145| +2450815|2428|5|766| +2450815|2431|5|452| +2450815|2432|5|956| +2450815|2434|5|497| +2450815|2437|5|607| +2450815|2438|5|867| +2450815|2440|5|942| +2450815|2443|5|967| +2450815|2444|5|485| +2450815|2446|5|262| +2450815|2449|5|664| +2450815|2450|5|611| +2450815|2452|5|731| +2450815|2455|5|266| +2450815|2456|5|58| +2450815|2458|5|| +2450815|2461|5|842| +2450815|2462|5|438| +2450815|2464|5|465| +2450815|2467|5|394| +2450815|2468|5|62| +2450815|2470|5|613| +2450815|2473|5|688| +2450815|2474|5|430| +2450815|2476|5|| +2450815|2479|5|| +2450815|2480|5|819| +2450815|2482|5|764| +2450815|2485|5|327| +2450815|2486|5|701| +2450815|2488|5|77| +2450815|2491|5|609| +2450815|2492|5|515| +2450815|2494|5|617| +2450815|2497|5|290| +2450815|2498|5|369| +2450815|2500|5|853| +2450815|2503|5|938| +2450815|2504|5|582| +2450815|2506|5|581| +2450815|2509|5|723| +2450815|2510|5|501| +2450815|2512|5|448| +2450815|2515|5|100| +2450815|2516|5|102| +2450815|2518|5|190| +2450815|2521|5|584| +2450815|2522|5|695| +2450815|2524|5|40| +2450815|2527|5|58| +2450815|2528|5|607| +2450815|2530|5|985| +2450815|2533|5|17| +2450815|2534|5|356| +2450815|2536|5|915| +2450815|2539|5|487| +2450815|2540|5|970| +2450815|2542|5|585| +2450815|2545|5|403| +2450815|2546|5|549| +2450815|2548|5|197| +2450815|2551|5|439| +2450815|2552|5|255| +2450815|2554|5|487| +2450815|2557|5|719| +2450815|2558|5|675| +2450815|2560|5|517| +2450815|2563|5|528| +2450815|2564|5|652| +2450815|2566|5|269| +2450815|2569|5|246| +2450815|2570|5|661| +2450815|2572|5|772| +2450815|2575|5|751| +2450815|2576|5|752| +2450815|2578|5|923| +2450815|2581|5|105| +2450815|2582|5|265| +2450815|2584|5|| +2450815|2587|5|859| +2450815|2588|5|858| +2450815|2590|5|439| +2450815|2593|5|| +2450815|2594|5|548| +2450815|2596|5|459| +2450815|2599|5|12| +2450815|2600|5|0| +2450815|2602|5|501| +2450815|2605|5|492| +2450815|2606|5|176| +2450815|2608|5|752| +2450815|2611|5|9| +2450815|2612|5|730| +2450815|2614|5|280| +2450815|2617|5|783| +2450815|2618|5|362| +2450815|2620|5|700| +2450815|2623|5|738| +2450815|2624|5|664| +2450815|2626|5|517| +2450815|2629|5|825| +2450815|2630|5|331| +2450815|2632|5|954| +2450815|2635|5|711| +2450815|2636|5|0| +2450815|2638|5|221| +2450815|2641|5|181| +2450815|2642|5|825| +2450815|2644|5|299| +2450815|2647|5|717| +2450815|2648|5|108| +2450815|2650|5|423| +2450815|2653|5|26| +2450815|2654|5|636| +2450815|2656|5|484| +2450815|2659|5|| +2450815|2660|5|644| +2450815|2662|5|854| +2450815|2665|5|549| +2450815|2666|5|219| +2450815|2668|5|267| +2450815|2671|5|908| +2450815|2672|5|942| +2450815|2674|5|725| +2450815|2677|5|761| +2450815|2678|5|216| +2450815|2680|5|105| +2450815|2683|5|983| +2450815|2684|5|167| +2450815|2686|5|34| +2450815|2689|5|597| +2450815|2690|5|502| +2450815|2692|5|830| +2450815|2695|5|| +2450815|2696|5|864| +2450815|2698|5|994| +2450815|2701|5|| +2450815|2702|5|81| +2450815|2704|5|869| +2450815|2707|5|989| +2450815|2708|5|867| +2450815|2710|5|574| +2450815|2713|5|821| +2450815|2714|5|418| +2450815|2716|5|| +2450815|2719|5|3| +2450815|2720|5|941| +2450815|2722|5|820| +2450815|2725|5|553| +2450815|2726|5|178| +2450815|2728|5|625| +2450815|2731|5|834| +2450815|2732|5|371| +2450815|2734|5|426| +2450815|2737|5|953| +2450815|2738|5|660| +2450815|2740|5|133| +2450815|2743|5|823| +2450815|2744|5|276| +2450815|2746|5|662| +2450815|2749|5|| +2450815|2750|5|271| +2450815|2752|5|16| +2450815|2755|5|241| +2450815|2756|5|180| +2450815|2758|5|| +2450815|2761|5|28| +2450815|2762|5|777| +2450815|2764|5|322| +2450815|2767|5|| +2450815|2768|5|376| +2450815|2770|5|383| +2450815|2773|5|793| +2450815|2774|5|498| +2450815|2776|5|872| +2450815|2779|5|604| +2450815|2780|5|839| +2450815|2782|5|879| +2450815|2785|5|459| +2450815|2786|5|997| +2450815|2788|5|220| +2450815|2791|5|369| +2450815|2792|5|628| +2450815|2794|5|919| +2450815|2797|5|349| +2450815|2798|5|746| +2450815|2800|5|305| +2450815|2803|5|170| +2450815|2804|5|743| +2450815|2806|5|803| +2450815|2809|5|131| +2450815|2810|5|634| +2450815|2812|5|997| +2450815|2815|5|| +2450815|2816|5|897| +2450815|2818|5|678| +2450815|2821|5|164| +2450815|2822|5|519| +2450815|2824|5|928| +2450815|2827|5|1000| +2450815|2828|5|869| +2450815|2830|5|812| +2450815|2833|5|22| +2450815|2834|5|485| +2450815|2836|5|527| +2450815|2839|5|58| +2450815|2840|5|78| +2450815|2842|5|272| +2450815|2845|5|| +2450815|2846|5|76| +2450815|2848|5|536| +2450815|2851|5|854| +2450815|2852|5|722| +2450815|2854|5|119| +2450815|2857|5|| +2450815|2858|5|173| +2450815|2860|5|456| +2450815|2863|5|507| +2450815|2864|5|781| +2450815|2866|5|347| +2450815|2869|5|589| +2450815|2870|5|577| +2450815|2872|5|480| +2450815|2875|5|390| +2450815|2876|5|775| +2450815|2878|5|408| +2450815|2881|5|126| +2450815|2882|5|170| +2450815|2884|5|264| +2450815|2887|5|532| +2450815|2888|5|229| +2450815|2890|5|| +2450815|2893|5|661| +2450815|2894|5|935| +2450815|2896|5|437| +2450815|2899|5|285| +2450815|2900|5|212| +2450815|2902|5|60| +2450815|2905|5|928| +2450815|2906|5|218| +2450815|2908|5|925| +2450815|2911|5|| +2450815|2912|5|895| +2450815|2914|5|774| +2450815|2917|5|933| +2450815|2918|5|976| +2450815|2920|5|782| +2450815|2923|5|421| +2450815|2924|5|937| +2450815|2926|5|539| +2450815|2929|5|457| +2450815|2930|5|146| +2450815|2932|5|| +2450815|2935|5|259| +2450815|2936|5|994| +2450815|2938|5|884| +2450815|2941|5|882| +2450815|2942|5|920| +2450815|2944|5|564| +2450815|2947|5|271| +2450815|2948|5|186| +2450815|2950|5|145| +2450815|2953|5|782| +2450815|2954|5|216| +2450815|2956|5|| +2450815|2959|5|828| +2450815|2960|5|570| +2450815|2962|5|34| +2450815|2965|5|204| +2450815|2966|5|862| +2450815|2968|5|797| +2450815|2971|5|621| +2450815|2972|5|306| +2450815|2974|5|898| +2450815|2977|5|921| +2450815|2978|5|597| +2450815|2980|5|963| +2450815|2983|5|269| +2450815|2984|5|415| +2450815|2986|5|360| +2450815|2989|5|136| +2450815|2990|5|806| +2450815|2992|5|138| +2450815|2995|5|655| +2450815|2996|5|255| +2450815|2998|5|880| +2450815|3001|5|256| +2450815|3002|5|714| +2450815|3004|5|996| +2450815|3007|5|191| +2450815|3008|5|270| +2450815|3010|5|955| +2450815|3013|5|245| +2450815|3014|5|751| +2450815|3016|5|556| +2450815|3019|5|| +2450815|3020|5|654| +2450815|3022|5|918| +2450815|3025|5|524| +2450815|3026|5|891| +2450815|3028|5|427| +2450815|3031|5|495| +2450815|3032|5|190| +2450815|3034|5|238| +2450815|3037|5|87| +2450815|3038|5|149| +2450815|3040|5|59| +2450815|3043|5|601| +2450815|3044|5|708| +2450815|3046|5|| +2450815|3049|5|587| +2450815|3050|5|| +2450815|3052|5|| +2450815|3055|5|88| +2450815|3056|5|163| +2450815|3058|5|18| +2450815|3061|5|135| +2450815|3062|5|173| +2450815|3064|5|670| +2450815|3067|5|412| +2450815|3068|5|538| +2450815|3070|5|87| +2450815|3073|5|600| +2450815|3074|5|| +2450815|3076|5|232| +2450815|3079|5|| +2450815|3080|5|804| +2450815|3082|5|671| +2450815|3085|5|| +2450815|3086|5|565| +2450815|3088|5|996| +2450815|3091|5|233| +2450815|3092|5|405| +2450815|3094|5|59| +2450815|3097|5|735| +2450815|3098|5|983| +2450815|3100|5|840| +2450815|3103|5|475| +2450815|3104|5|978| +2450815|3106|5|430| +2450815|3109|5|645| +2450815|3110|5|246| +2450815|3112|5|564| +2450815|3115|5|964| +2450815|3116|5|367| +2450815|3118|5|992| +2450815|3121|5|59| +2450815|3122|5|62| +2450815|3124|5|23| +2450815|3127|5|131| +2450815|3128|5|975| +2450815|3130|5|475| +2450815|3133|5|954| +2450815|3134|5|287| +2450815|3136|5|867| +2450815|3139|5|467| +2450815|3140|5|130| +2450815|3142|5|896| +2450815|3145|5|354| +2450815|3146|5|278| +2450815|3148|5|3| +2450815|3151|5|271| +2450815|3152|5|17| +2450815|3154|5|580| +2450815|3157|5|272| +2450815|3158|5|27| +2450815|3160|5|226| +2450815|3163|5|809| +2450815|3164|5|330| +2450815|3166|5|829| +2450815|3169|5|516| +2450815|3170|5|899| +2450815|3172|5|414| +2450815|3175|5|226| +2450815|3176|5|510| +2450815|3178|5|204| +2450815|3181|5|107| +2450815|3182|5|533| +2450815|3184|5|414| +2450815|3187|5|388| +2450815|3188|5|209| +2450815|3190|5|143| +2450815|3193|5|981| +2450815|3194|5|860| +2450815|3196|5|421| +2450815|3199|5|440| +2450815|3200|5|179| +2450815|3202|5|560| +2450815|3205|5|836| +2450815|3206|5|740| +2450815|3208|5|403| +2450815|3211|5|177| +2450815|3212|5|111| +2450815|3214|5|688| +2450815|3217|5|543| +2450815|3218|5|7| +2450815|3220|5|146| +2450815|3223|5|318| +2450815|3224|5|579| +2450815|3226|5|51| +2450815|3229|5|| +2450815|3230|5|916| +2450815|3232|5|873| +2450815|3235|5|461| +2450815|3236|5|831| +2450815|3238|5|63| +2450815|3241|5|860| +2450815|3242|5|258| +2450815|3244|5|21| +2450815|3247|5|682| +2450815|3248|5|648| +2450815|3250|5|| +2450815|3253|5|396| +2450815|3254|5|462| +2450815|3256|5|567| +2450815|3259|5|389| +2450815|3260|5|370| +2450815|3262|5|| +2450815|3265|5|345| +2450815|3266|5|909| +2450815|3268|5|72| +2450815|3271|5|336| +2450815|3272|5|880| +2450815|3274|5|745| +2450815|3277|5|133| +2450815|3278|5|390| +2450815|3280|5|838| +2450815|3283|5|766| +2450815|3284|5|296| +2450815|3286|5|698| +2450815|3289|5|659| +2450815|3290|5|801| +2450815|3292|5|661| +2450815|3295|5|313| +2450815|3296|5|722| +2450815|3298|5|299| +2450815|3301|5|777| +2450815|3302|5|548| +2450815|3304|5|66| +2450815|3307|5|275| +2450815|3308|5|464| +2450815|3310|5|| +2450815|3313|5|430| +2450815|3314|5|276| +2450815|3316|5|291| +2450815|3319|5|187| +2450815|3320|5|237| +2450815|3322|5|709| +2450815|3325|5|| +2450815|3326|5|392| +2450815|3328|5|325| +2450815|3331|5|336| +2450815|3332|5|566| +2450815|3334|5|612| +2450815|3337|5|637| +2450815|3338|5|738| +2450815|3340|5|882| +2450815|3343|5|859| +2450815|3344|5|530| +2450815|3346|5|770| +2450815|3349|5|375| +2450815|3350|5|808| +2450815|3352|5|81| +2450815|3355|5|545| +2450815|3356|5|353| +2450815|3358|5|464| +2450815|3361|5|481| +2450815|3362|5|326| +2450815|3364|5|736| +2450815|3367|5|199| +2450815|3368|5|500| +2450815|3370|5|860| +2450815|3373|5|967| +2450815|3374|5|832| +2450815|3376|5|198| +2450815|3379|5|970| +2450815|3380|5|303| +2450815|3382|5|308| +2450815|3385|5|829| +2450815|3386|5|347| +2450815|3388|5|971| +2450815|3391|5|241| +2450815|3392|5|712| +2450815|3394|5|989| +2450815|3397|5|230| +2450815|3398|5|408| +2450815|3400|5|924| +2450815|3403|5|272| +2450815|3404|5|58| +2450815|3406|5|267| +2450815|3409|5|419| +2450815|3410|5|397| +2450815|3412|5|565| +2450815|3415|5|209| +2450815|3416|5|149| +2450815|3418|5|69| +2450815|3421|5|648| +2450815|3422|5|| +2450815|3424|5|618| +2450815|3427|5|902| +2450815|3428|5|534| +2450815|3430|5|973| +2450815|3433|5|479| +2450815|3434|5|| +2450815|3436|5|415| +2450815|3439|5|974| +2450815|3440|5|62| +2450815|3442|5|819| +2450815|3445|5|248| +2450815|3446|5|966| +2450815|3448|5|323| +2450815|3451|5|4| +2450815|3452|5|689| +2450815|3454|5|159| +2450815|3457|5|855| +2450815|3458|5|755| +2450815|3460|5|82| +2450815|3463|5|940| +2450815|3464|5|| +2450815|3466|5|345| +2450815|3469|5|642| +2450815|3470|5|223| +2450815|3472|5|199| +2450815|3475|5|863| +2450815|3476|5|849| +2450815|3478|5|88| +2450815|3481|5|714| +2450815|3482|5|393| +2450815|3484|5|862| +2450815|3487|5|107| +2450815|3488|5|| +2450815|3490|5|508| +2450815|3493|5|792| +2450815|3494|5|417| +2450815|3496|5|753| +2450815|3499|5|870| +2450815|3500|5|500| +2450815|3502|5|274| +2450815|3505|5|215| +2450815|3506|5|246| +2450815|3508|5|439| +2450815|3511|5|729| +2450815|3512|5|496| +2450815|3514|5|387| +2450815|3517|5|| +2450815|3518|5|642| +2450815|3520|5|302| +2450815|3523|5|36| +2450815|3524|5|903| +2450815|3526|5|157| +2450815|3529|5|971| +2450815|3530|5|964| +2450815|3532|5|944| +2450815|3535|5|432| +2450815|3536|5|145| +2450815|3538|5|105| +2450815|3541|5|971| +2450815|3542|5|457| +2450815|3544|5|108| +2450815|3547|5|48| +2450815|3548|5|944| +2450815|3550|5|685| +2450815|3553|5|164| +2450815|3554|5|650| +2450815|3556|5|967| +2450815|3559|5|662| +2450815|3560|5|824| +2450815|3562|5|161| +2450815|3565|5|645| +2450815|3566|5|384| +2450815|3568|5|| +2450815|3571|5|975| +2450815|3572|5|| +2450815|3574|5|| +2450815|3577|5|501| +2450815|3578|5|357| +2450815|3580|5|367| +2450815|3583|5|650| +2450815|3584|5|347| +2450815|3586|5|669| +2450815|3589|5|122| +2450815|3590|5|103| +2450815|3592|5|714| +2450815|3595|5|729| +2450815|3596|5|470| +2450815|3598|5|610| +2450815|3601|5|32| +2450815|3602|5|564| +2450815|3604|5|457| +2450815|3607|5|219| +2450815|3608|5|791| +2450815|3610|5|416| +2450815|3613|5|541| +2450815|3614|5|720| +2450815|3616|5|1| +2450815|3619|5|774| +2450815|3620|5|787| +2450815|3622|5|440| +2450815|3625|5|648| +2450815|3626|5|263| +2450815|3628|5|894| +2450815|3631|5|798| +2450815|3632|5|477| +2450815|3634|5|682| +2450815|3637|5|295| +2450815|3638|5|502| +2450815|3640|5|711| +2450815|3643|5|806| +2450815|3644|5|447| +2450815|3646|5|| +2450815|3649|5|580| +2450815|3650|5|759| +2450815|3652|5|811| +2450815|3655|5|976| +2450815|3656|5|730| +2450815|3658|5|681| +2450815|3661|5|5| +2450815|3662|5|957| +2450815|3664|5|82| +2450815|3667|5|882| +2450815|3668|5|319| +2450815|3670|5|369| +2450815|3673|5|479| +2450815|3674|5|430| +2450815|3676|5|371| +2450815|3679|5|967| +2450815|3680|5|405| +2450815|3682|5|791| +2450815|3685|5|570| +2450815|3686|5|794| +2450815|3688|5|| +2450815|3691|5|561| +2450815|3692|5|292| +2450815|3694|5|145| +2450815|3697|5|242| +2450815|3698|5|750| +2450815|3700|5|4| +2450815|3703|5|505| +2450815|3704|5|172| +2450815|3706|5|495| +2450815|3709|5|519| +2450815|3710|5|823| +2450815|3712|5|310| +2450815|3715|5|813| +2450815|3716|5|959| +2450815|3718|5|663| +2450815|3721|5|267| +2450815|3722|5|844| +2450815|3724|5|945| +2450815|3727|5|474| +2450815|3728|5|998| +2450815|3730|5|318| +2450815|3733|5|168| +2450815|3734|5|246| +2450815|3736|5|892| +2450815|3739|5|39| +2450815|3740|5|393| +2450815|3742|5|392| +2450815|3745|5|409| +2450815|3746|5|132| +2450815|3748|5|901| +2450815|3751|5|347| +2450815|3752|5|436| +2450815|3754|5|550| +2450815|3757|5|145| +2450815|3758|5|254| +2450815|3760|5|915| +2450815|3763|5|821| +2450815|3764|5|358| +2450815|3766|5|467| +2450815|3769|5|791| +2450815|3770|5|793| +2450815|3772|5|175| +2450815|3775|5|801| +2450815|3776|5|72| +2450815|3778|5|387| +2450815|3781|5|785| +2450815|3782|5|347| +2450815|3784|5|285| +2450815|3787|5|180| +2450815|3788|5|534| +2450815|3790|5|545| +2450815|3793|5|620| +2450815|3794|5|368| +2450815|3796|5|597| +2450815|3799|5|| +2450815|3800|5|242| +2450815|3802|5|425| +2450815|3805|5|274| +2450815|3806|5|609| +2450815|3808|5|106| +2450815|3811|5|| +2450815|3812|5|833| +2450815|3814|5|39| +2450815|3817|5|| +2450815|3818|5|78| +2450815|3820|5|109| +2450815|3823|5|72| +2450815|3824|5|105| +2450815|3826|5|914| +2450815|3829|5|632| +2450815|3830|5|999| +2450815|3832|5|453| +2450815|3835|5|5| +2450815|3836|5|217| +2450815|3838|5|247| +2450815|3841|5|502| +2450815|3842|5|598| +2450815|3844|5|689| +2450815|3847|5|273| +2450815|3848|5|297| +2450815|3850|5|282| +2450815|3853|5|682| +2450815|3854|5|589| +2450815|3856|5|185| +2450815|3859|5|542| +2450815|3860|5|87| +2450815|3862|5|153| +2450815|3865|5|754| +2450815|3866|5|| +2450815|3868|5|659| +2450815|3871|5|106| +2450815|3872|5|445| +2450815|3874|5|232| +2450815|3877|5|633| +2450815|3878|5|680| +2450815|3880|5|180| +2450815|3883|5|986| +2450815|3884|5|206| +2450815|3886|5|380| +2450815|3889|5|153| +2450815|3890|5|| +2450815|3892|5|386| +2450815|3895|5|593| +2450815|3896|5|39| +2450815|3898|5|848| +2450815|3901|5|| +2450815|3902|5|804| +2450815|3904|5|735| +2450815|3907|5|922| +2450815|3908|5|678| +2450815|3910|5|801| +2450815|3913|5|69| +2450815|3914|5|505| +2450815|3916|5|242| +2450815|3919|5|966| +2450815|3920|5|19| +2450815|3922|5|682| +2450815|3925|5|272| +2450815|3926|5|129| +2450815|3928|5|472| +2450815|3931|5|71| +2450815|3932|5|270| +2450815|3934|5|188| +2450815|3937|5|752| +2450815|3938|5|475| +2450815|3940|5|525| +2450815|3943|5|25| +2450815|3944|5|201| +2450815|3946|5|882| +2450815|3949|5|49| +2450815|3950|5|952| +2450815|3952|5|146| +2450815|3955|5|| +2450815|3956|5|4| +2450815|3958|5|599| +2450815|3961|5|562| +2450815|3962|5|147| +2450815|3964|5|682| +2450815|3967|5|210| +2450815|3968|5|505| +2450815|3970|5|921| +2450815|3973|5|225| +2450815|3974|5|267| +2450815|3976|5|380| +2450815|3979|5|6| +2450815|3980|5|253| +2450815|3982|5|100| +2450815|3985|5|464| +2450815|3986|5|284| +2450815|3988|5|162| +2450815|3991|5|749| +2450815|3992|5|121| +2450815|3994|5|796| +2450815|3997|5|689| +2450815|3998|5|410| +2450815|4000|5|416| +2450815|4003|5|141| +2450815|4004|5|159| +2450815|4006|5|376| +2450815|4009|5|615| +2450815|4010|5|490| +2450815|4012|5|73| +2450815|4015|5|95| +2450815|4016|5|82| +2450815|4018|5|374| +2450815|4021|5|732| +2450815|4022|5|862| +2450815|4024|5|663| +2450815|4027|5|609| +2450815|4028|5|244| +2450815|4030|5|744| +2450815|4033|5|280| +2450815|4034|5|477| +2450815|4036|5|996| +2450815|4039|5|335| +2450815|4040|5|425| +2450815|4042|5|960| +2450815|4045|5|569| +2450815|4046|5|51| +2450815|4048|5|899| +2450815|4051|5|84| +2450815|4052|5|841| +2450815|4054|5|263| +2450815|4057|5|732| +2450815|4058|5|417| +2450815|4060|5|661| +2450815|4063|5|535| +2450815|4064|5|953| +2450815|4066|5|| +2450815|4069|5|327| +2450815|4070|5|165| +2450815|4072|5|819| +2450815|4075|5|899| +2450815|4076|5|852| +2450815|4078|5|619| +2450815|4081|5|280| +2450815|4082|5|321| +2450815|4084|5|494| +2450815|4087|5|277| +2450815|4088|5|782| +2450815|4090|5|516| +2450815|4093|5|134| +2450815|4094|5|287| +2450815|4096|5|409| +2450815|4099|5|545| +2450815|4100|5|292| +2450815|4102|5|77| +2450815|4105|5|279| +2450815|4106|5|348| +2450815|4108|5|391| +2450815|4111|5|509| +2450815|4112|5|343| +2450815|4114|5|953| +2450815|4117|5|851| +2450815|4118|5|336| +2450815|4120|5|29| +2450815|4123|5|980| +2450815|4124|5|107| +2450815|4126|5|493| +2450815|4129|5|190| +2450815|4130|5|755| +2450815|4132|5|830| +2450815|4135|5|19| +2450815|4136|5|866| +2450815|4138|5|470| +2450815|4141|5|143| +2450815|4142|5|661| +2450815|4144|5|908| +2450815|4147|5|30| +2450815|4148|5|59| +2450815|4150|5|99| +2450815|4153|5|887| +2450815|4154|5|230| +2450815|4156|5|556| +2450815|4159|5|884| +2450815|4160|5|997| +2450815|4162|5|957| +2450815|4165|5|82| +2450815|4166|5|42| +2450815|4168|5|436| +2450815|4171|5|129| +2450815|4172|5|152| +2450815|4174|5|412| +2450815|4177|5|536| +2450815|4178|5|274| +2450815|4180|5|68| +2450815|4183|5|258| +2450815|4184|5|610| +2450815|4186|5|312| +2450815|4189|5|501| +2450815|4190|5|19| +2450815|4192|5|978| +2450815|4195|5|64| +2450815|4196|5|114| +2450815|4198|5|294| +2450815|4201|5|30| +2450815|4202|5|654| +2450815|4204|5|315| +2450815|4207|5|73| +2450815|4208|5|532| +2450815|4210|5|702| +2450815|4213|5|53| +2450815|4214|5|490| +2450815|4216|5|953| +2450815|4219|5|186| +2450815|4220|5|28| +2450815|4222|5|110| +2450815|4225|5|800| +2450815|4226|5|12| +2450815|4228|5|690| +2450815|4231|5|989| +2450815|4232|5|206| +2450815|4234|5|636| +2450815|4237|5|978| +2450815|4238|5|136| +2450815|4240|5|976| +2450815|4243|5|148| +2450815|4244|5|189| +2450815|4246|5|653| +2450815|4249|5|209| +2450815|4250|5|2| +2450815|4252|5|135| +2450815|4255|5|74| +2450815|4256|5|186| +2450815|4258|5|757| +2450815|4261|5|237| +2450815|4262|5|712| +2450815|4264|5|793| +2450815|4267|5|257| +2450815|4268|5|454| +2450815|4270|5|805| +2450815|4273|5|221| +2450815|4274|5|206| +2450815|4276|5|976| +2450815|4279|5|533| +2450815|4280|5|388| +2450815|4282|5|179| +2450815|4285|5|536| +2450815|4286|5|| +2450815|4288|5|639| +2450815|4291|5|306| +2450815|4292|5|798| +2450815|4294|5|896| +2450815|4297|5|82| +2450815|4298|5|778| +2450815|4300|5|362| +2450815|4303|5|542| +2450815|4304|5|101| +2450815|4306|5|566| +2450815|4309|5|116| +2450815|4310|5|942| +2450815|4312|5|596| +2450815|4315|5|503| +2450815|4316|5|249| +2450815|4318|5|834| +2450815|4321|5|270| +2450815|4322|5|329| +2450815|4324|5|130| +2450815|4327|5|973| +2450815|4328|5|443| +2450815|4330|5|294| +2450815|4333|5|473| +2450815|4334|5|755| +2450815|4336|5|595| +2450815|4339|5|| +2450815|4340|5|853| +2450815|4342|5|232| +2450815|4345|5|836| +2450815|4346|5|152| +2450815|4348|5|632| +2450815|4351|5|81| +2450815|4352|5|522| +2450815|4354|5|352| +2450815|4357|5|688| +2450815|4358|5|460| +2450815|4360|5|399| +2450815|4363|5|117| +2450815|4364|5|838| +2450815|4366|5|794| +2450815|4369|5|670| +2450815|4370|5|| +2450815|4372|5|518| +2450815|4375|5|373| +2450815|4376|5|397| +2450815|4378|5|264| +2450815|4381|5|793| +2450815|4382|5|431| +2450815|4384|5|523| +2450815|4387|5|| +2450815|4388|5|532| +2450815|4390|5|622| +2450815|4393|5|431| +2450815|4394|5|637| +2450815|4396|5|759| +2450815|4399|5|717| +2450815|4400|5|207| +2450815|4402|5|458| +2450815|4405|5|661| +2450815|4406|5|309| +2450815|4408|5|906| +2450815|4411|5|784| +2450815|4412|5|625| +2450815|4414|5|23| +2450815|4417|5|| +2450815|4418|5|511| +2450815|4420|5|676| +2450815|4423|5|90| +2450815|4424|5|792| +2450815|4426|5|146| +2450815|4429|5|412| +2450815|4430|5|768| +2450815|4432|5|125| +2450815|4435|5|838| +2450815|4436|5|8| +2450815|4438|5|597| +2450815|4441|5|| +2450815|4442|5|| +2450815|4444|5|946| +2450815|4447|5|27| +2450815|4448|5|2| +2450815|4450|5|| +2450815|4453|5|433| +2450815|4454|5|621| +2450815|4456|5|278| +2450815|4459|5|103| +2450815|4460|5|252| +2450815|4462|5|176| +2450815|4465|5|470| +2450815|4466|5|16| +2450815|4468|5|125| +2450815|4471|5|225| +2450815|4472|5|828| +2450815|4474|5|929| +2450815|4477|5|467| +2450815|4478|5|518| +2450815|4480|5|393| +2450815|4483|5|714| +2450815|4484|5|74| +2450815|4486|5|211| +2450815|4489|5|351| +2450815|4490|5|896| +2450815|4492|5|758| +2450815|4495|5|113| +2450815|4496|5|672| +2450815|4498|5|| +2450815|4501|5|191| +2450815|4502|5|477| +2450815|4504|5|376| +2450815|4507|5|998| +2450815|4508|5|480| +2450815|4510|5|301| +2450815|4513|5|374| +2450815|4514|5|161| +2450815|4516|5|772| +2450815|4519|5|643| +2450815|4520|5|822| +2450815|4522|5|458| +2450815|4525|5|943| +2450815|4526|5|239| +2450815|4528|5|302| +2450815|4531|5|518| +2450815|4532|5|921| +2450815|4534|5|682| +2450815|4537|5|729| +2450815|4538|5|104| +2450815|4540|5|858| +2450815|4543|5|120| +2450815|4544|5|406| +2450815|4546|5|71| +2450815|4549|5|978| +2450815|4550|5|356| +2450815|4552|5|924| +2450815|4555|5|113| +2450815|4556|5|913| +2450815|4558|5|838| +2450815|4561|5|251| +2450815|4562|5|659| +2450815|4564|5|369| +2450815|4567|5|918| +2450815|4568|5|747| +2450815|4570|5|2| +2450815|4573|5|571| +2450815|4574|5|658| +2450815|4576|5|526| +2450815|4579|5|778| +2450815|4580|5|924| +2450815|4582|5|621| +2450815|4585|5|652| +2450815|4586|5|787| +2450815|4588|5|135| +2450815|4591|5|742| +2450815|4592|5|587| +2450815|4594|5|458| +2450815|4597|5|174| +2450815|4598|5|763| +2450815|4600|5|97| +2450815|4603|5|150| +2450815|4604|5|294| +2450815|4606|5|485| +2450815|4609|5|992| +2450815|4610|5|305| +2450815|4612|5|839| +2450815|4615|5|150| +2450815|4616|5|766| +2450815|4618|5|189| +2450815|4621|5|524| +2450815|4622|5|153| +2450815|4624|5|429| +2450815|4627|5|| +2450815|4628|5|722| +2450815|4630|5|407| +2450815|4633|5|871| +2450815|4634|5|963| +2450815|4636|5|963| +2450815|4639|5|500| +2450815|4640|5|319| +2450815|4642|5|666| +2450815|4645|5|983| +2450815|4646|5|262| +2450815|4648|5|965| +2450815|4651|5|640| +2450815|4652|5|32| +2450815|4654|5|171| +2450815|4657|5|833| +2450815|4658|5|131| +2450815|4660|5|| +2450815|4663|5|572| +2450815|4664|5|383| +2450815|4666|5|928| +2450815|4669|5|218| +2450815|4670|5|932| +2450815|4672|5|733| +2450815|4675|5|128| +2450815|4676|5|939| +2450815|4678|5|923| +2450815|4681|5|307| +2450815|4682|5|566| +2450815|4684|5|701| +2450815|4687|5|855| +2450815|4688|5|220| +2450815|4690|5|953| +2450815|4693|5|870| +2450815|4694|5|349| +2450815|4696|5|406| +2450815|4699|5|455| +2450815|4700|5|57| +2450815|4702|5|875| +2450815|4705|5|440| +2450815|4706|5|306| +2450815|4708|5|715| +2450815|4711|5|903| +2450815|4712|5|443| +2450815|4714|5|671| +2450815|4717|5|987| +2450815|4718|5|293| +2450815|4720|5|631| +2450815|4723|5|615| +2450815|4724|5|744| +2450815|4726|5|829| +2450815|4729|5|623| +2450815|4730|5|857| +2450815|4732|5|509| +2450815|4735|5|825| +2450815|4736|5|510| +2450815|4738|5|312| +2450815|4741|5|114| +2450815|4742|5|642| +2450815|4744|5|407| +2450815|4747|5|507| +2450815|4748|5|954| +2450815|4750|5|142| +2450815|4753|5|588| +2450815|4754|5|423| +2450815|4756|5|740| +2450815|4759|5|613| +2450815|4760|5|702| +2450815|4762|5|918| +2450815|4765|5|951| +2450815|4766|5|241| +2450815|4768|5|206| +2450815|4771|5|10| +2450815|4772|5|277| +2450815|4774|5|194| +2450815|4777|5|671| +2450815|4778|5|94| +2450815|4780|5|923| +2450815|4783|5|106| +2450815|4784|5|514| +2450815|4786|5|847| +2450815|4789|5|358| +2450815|4790|5|284| +2450815|4792|5|744| +2450815|4795|5|163| +2450815|4796|5|260| +2450815|4798|5|496| +2450815|4801|5|175| +2450815|4802|5|920| +2450815|4804|5|| +2450815|4807|5|996| +2450815|4808|5|455| +2450815|4810|5|11| +2450815|4813|5|414| +2450815|4814|5|827| +2450815|4816|5|785| +2450815|4819|5|778| +2450815|4820|5|356| +2450815|4822|5|862| +2450815|4825|5|564| +2450815|4826|5|131| +2450815|4828|5|271| +2450815|4831|5|904| +2450815|4832|5|186| +2450815|4834|5|681| +2450815|4837|5|804| +2450815|4838|5|120| +2450815|4840|5|710| +2450815|4843|5|572| +2450815|4844|5|397| +2450815|4846|5|539| +2450815|4849|5|141| +2450815|4850|5|734| +2450815|4852|5|887| +2450815|4855|5|185| +2450815|4856|5|835| +2450815|4858|5|| +2450815|4861|5|| +2450815|4862|5|11| +2450815|4864|5|674| +2450815|4867|5|178| +2450815|4868|5|724| +2450815|4870|5|841| +2450815|4873|5|243| +2450815|4874|5|729| +2450815|4876|5|| +2450815|4879|5|66| +2450815|4880|5|381| +2450815|4882|5|670| +2450815|4885|5|283| +2450815|4886|5|635| +2450815|4888|5|451| +2450815|4891|5|474| +2450815|4892|5|441| +2450815|4894|5|861| +2450815|4897|5|342| +2450815|4898|5|695| +2450815|4900|5|473| +2450815|4903|5|553| +2450815|4904|5|511| +2450815|4906|5|196| +2450815|4909|5|312| +2450815|4910|5|235| +2450815|4912|5|431| +2450815|4915|5|| +2450815|4916|5|549| +2450815|4918|5|733| +2450815|4921|5|44| +2450815|4922|5|718| +2450815|4924|5|638| +2450815|4927|5|878| +2450815|4928|5|363| +2450815|4930|5|875| +2450815|4933|5|930| +2450815|4934|5|702| +2450815|4936|5|398| +2450815|4939|5|294| +2450815|4940|5|376| +2450815|4942|5|300| +2450815|4945|5|305| +2450815|4946|5|985| +2450815|4948|5|468| +2450815|4951|5|455| +2450815|4952|5|78| +2450815|4954|5|608| +2450815|4957|5|78| +2450815|4958|5|868| +2450815|4960|5|413| +2450815|4963|5|37| +2450815|4964|5|247| +2450815|4966|5|890| +2450815|4969|5|167| +2450815|4970|5|560| +2450815|4972|5|| +2450815|4975|5|508| +2450815|4976|5|909| +2450815|4978|5|640| +2450815|4981|5|94| +2450815|4982|5|57| +2450815|4984|5|274| +2450815|4987|5|329| +2450815|4988|5|92| +2450815|4990|5|849| +2450815|4993|5|286| +2450815|4994|5|159| +2450815|4996|5|856| +2450815|4999|5|756| +2450815|5000|5|295| +2450815|5002|5|772| +2450815|5005|5|263| +2450815|5006|5|128| +2450815|5008|5|45| +2450815|5011|5|152| +2450815|5012|5|821| +2450815|5014|5|671| +2450815|5017|5|307| +2450815|5018|5|968| +2450815|5020|5|920| +2450815|5023|5|737| +2450815|5024|5|138| +2450815|5026|5|776| +2450815|5029|5|136| +2450815|5030|5|34| +2450815|5032|5|319| +2450815|5035|5|20| +2450815|5036|5|917| +2450815|5038|5|677| +2450815|5041|5|78| +2450815|5042|5|| +2450815|5044|5|371| +2450815|5047|5|766| +2450815|5048|5|697| +2450815|5050|5|398| +2450815|5053|5|155| +2450815|5054|5|171| +2450815|5056|5|410| +2450815|5059|5|624| +2450815|5060|5|72| +2450815|5062|5|764| +2450815|5065|5|101| +2450815|5066|5|679| +2450815|5068|5|12| +2450815|5071|5|92| +2450815|5072|5|369| +2450815|5074|5|907| +2450815|5077|5|758| +2450815|5078|5|531| +2450815|5080|5|259| +2450815|5083|5|108| +2450815|5084|5|495| +2450815|5086|5|671| +2450815|5089|5|877| +2450815|5090|5|189| +2450815|5092|5|176| +2450815|5095|5|| +2450815|5096|5|361| +2450815|5098|5|95| +2450815|5101|5|71| +2450815|5102|5|82| +2450815|5104|5|839| +2450815|5107|5|654| +2450815|5108|5|844| +2450815|5110|5|3| +2450815|5113|5|949| +2450815|5114|5|694| +2450815|5116|5|983| +2450815|5119|5|903| +2450815|5120|5|56| +2450815|5122|5|55| +2450815|5125|5|475| +2450815|5126|5|843| +2450815|5128|5|219| +2450815|5131|5|812| +2450815|5132|5|908| +2450815|5134|5|732| +2450815|5137|5|255| +2450815|5138|5|589| +2450815|5140|5|304| +2450815|5143|5|154| +2450815|5144|5|713| +2450815|5146|5|256| +2450815|5149|5|910| +2450815|5150|5|93| +2450815|5152|5|868| +2450815|5155|5|822| +2450815|5156|5|759| +2450815|5158|5|710| +2450815|5161|5|800| +2450815|5162|5|597| +2450815|5164|5|3| +2450815|5167|5|982| +2450815|5168|5|776| +2450815|5170|5|27| +2450815|5173|5|723| +2450815|5174|5|563| +2450815|5176|5|578| +2450815|5179|5|149| +2450815|5180|5|448| +2450815|5182|5|105| +2450815|5185|5|0| +2450815|5186|5|387| +2450815|5188|5|445| +2450815|5191|5|310| +2450815|5192|5|749| +2450815|5194|5|860| +2450815|5197|5|824| +2450815|5198|5|921| +2450815|5200|5|252| +2450815|5203|5|933| +2450815|5204|5|530| +2450815|5206|5|580| +2450815|5209|5|186| +2450815|5210|5|872| +2450815|5212|5|649| +2450815|5215|5|418| +2450815|5216|5|687| +2450815|5218|5|370| +2450815|5221|5|897| +2450815|5222|5|931| +2450815|5224|5|699| +2450815|5227|5|211| +2450815|5228|5|52| +2450815|5230|5|921| +2450815|5233|5|762| +2450815|5234|5|875| +2450815|5236|5|605| +2450815|5239|5|145| +2450815|5240|5|176| +2450815|5242|5|551| +2450815|5245|5|565| +2450815|5246|5|836| +2450815|5248|5|240| +2450815|5251|5|409| +2450815|5252|5|16| +2450815|5254|5|647| +2450815|5257|5|872| +2450815|5258|5|13| +2450815|5260|5|973| +2450815|5263|5|334| +2450815|5264|5|644| +2450815|5266|5|155| +2450815|5269|5|172| +2450815|5270|5|| +2450815|5272|5|390| +2450815|5275|5|152| +2450815|5276|5|385| +2450815|5278|5|682| +2450815|5281|5|268| +2450815|5282|5|793| +2450815|5284|5|631| +2450815|5287|5|719| +2450815|5288|5|478| +2450815|5290|5|72| +2450815|5293|5|260| +2450815|5294|5|163| +2450815|5296|5|14| +2450815|5299|5|913| +2450815|5300|5|21| +2450815|5302|5|898| +2450815|5305|5|| +2450815|5306|5|143| +2450815|5308|5|913| +2450815|5311|5|857| +2450815|5312|5|296| +2450815|5314|5|51| +2450815|5317|5|264| +2450815|5318|5|337| +2450815|5320|5|825| +2450815|5323|5|439| +2450815|5324|5|244| +2450815|5326|5|66| +2450815|5329|5|987| +2450815|5330|5|124| +2450815|5332|5|100| +2450815|5335|5|851| +2450815|5336|5|591| +2450815|5338|5|755| +2450815|5341|5|113| +2450815|5342|5|858| +2450815|5344|5|166| +2450815|5347|5|869| +2450815|5348|5|586| +2450815|5350|5|699| +2450815|5353|5|627| +2450815|5354|5|890| +2450815|5356|5|516| +2450815|5359|5|798| +2450815|5360|5|543| +2450815|5362|5|305| +2450815|5365|5|967| +2450815|5366|5|216| +2450815|5368|5|150| +2450815|5371|5|37| +2450815|5372|5|391| +2450815|5374|5|694| +2450815|5377|5|37| +2450815|5378|5|267| +2450815|5380|5|326| +2450815|5383|5|795| +2450815|5384|5|158| +2450815|5386|5|873| +2450815|5389|5|696| +2450815|5390|5|75| +2450815|5392|5|460| +2450815|5395|5|195| +2450815|5396|5|442| +2450815|5398|5|36| +2450815|5401|5|641| +2450815|5402|5|347| +2450815|5404|5|991| +2450815|5407|5|814| +2450815|5408|5|825| +2450815|5410|5|123| +2450815|5413|5|156| +2450815|5414|5|786| +2450815|5416|5|59| +2450815|5419|5|117| +2450815|5420|5|368| +2450815|5422|5|808| +2450815|5425|5|352| +2450815|5426|5|237| +2450815|5428|5|367| +2450815|5431|5|663| +2450815|5432|5|373| +2450815|5434|5|111| +2450815|5437|5|184| +2450815|5438|5|91| +2450815|5440|5|357| +2450815|5443|5|334| +2450815|5444|5|| +2450815|5446|5|7| +2450815|5449|5|342| +2450815|5450|5|845| +2450815|5452|5|115| +2450815|5455|5|255| +2450815|5456|5|575| +2450815|5458|5|895| +2450815|5461|5|492| +2450815|5462|5|213| +2450815|5464|5|62| +2450815|5467|5|607| +2450815|5468|5|432| +2450815|5470|5|965| +2450815|5473|5|196| +2450815|5474|5|423| +2450815|5476|5|136| +2450815|5479|5|101| +2450815|5480|5|661| +2450815|5482|5|879| +2450815|5485|5|199| +2450815|5486|5|525| +2450815|5488|5|597| +2450815|5491|5|44| +2450815|5492|5|474| +2450815|5494|5|232| +2450815|5497|5|527| +2450815|5498|5|490| +2450815|5500|5|80| +2450815|5503|5|747| +2450815|5504|5|30| +2450815|5506|5|951| +2450815|5509|5|206| +2450815|5510|5|802| +2450815|5512|5|155| +2450815|5515|5|823| +2450815|5516|5|823| +2450815|5518|5|250| +2450815|5521|5|382| +2450815|5522|5|6| +2450815|5524|5|930| +2450815|5527|5|464| +2450815|5528|5|124| +2450815|5530|5|107| +2450815|5533|5|795| +2450815|5534|5|754| +2450815|5536|5|188| +2450815|5539|5|511| +2450815|5540|5|950| +2450815|5542|5|257| +2450815|5545|5|635| +2450815|5546|5|121| +2450815|5548|5|82| +2450815|5551|5|219| +2450815|5552|5|751| +2450815|5554|5|| +2450815|5557|5|25| +2450815|5558|5|198| +2450815|5560|5|119| +2450815|5563|5|| +2450815|5564|5|487| +2450815|5566|5|559| +2450815|5569|5|918| +2450815|5570|5|363| +2450815|5572|5|449| +2450815|5575|5|431| +2450815|5576|5|852| +2450815|5578|5|443| +2450815|5581|5|274| +2450815|5582|5|33| +2450815|5584|5|251| +2450815|5587|5|551| +2450815|5588|5|221| +2450815|5590|5|567| +2450815|5593|5|397| +2450815|5594|5|927| +2450815|5596|5|284| +2450815|5599|5|375| +2450815|5600|5|535| +2450815|5602|5|1000| +2450815|5605|5|114| +2450815|5606|5|321| +2450815|5608|5|99| +2450815|5611|5|431| +2450815|5612|5|763| +2450815|5614|5|430| +2450815|5617|5|631| +2450815|5618|5|537| +2450815|5620|5|753| +2450815|5623|5|547| +2450815|5624|5|803| +2450815|5626|5|584| +2450815|5629|5|462| +2450815|5630|5|938| +2450815|5632|5|679| +2450815|5635|5|870| +2450815|5636|5|123| +2450815|5638|5|264| +2450815|5641|5|237| +2450815|5642|5|712| +2450815|5644|5|475| +2450815|5647|5|120| +2450815|5648|5|800| +2450815|5650|5|113| +2450815|5653|5|416| +2450815|5654|5|700| +2450815|5656|5|916| +2450815|5659|5|836| +2450815|5660|5|525| +2450815|5662|5|646| +2450815|5665|5|480| +2450815|5666|5|676| +2450815|5668|5|18| +2450815|5671|5|485| +2450815|5672|5|406| +2450815|5674|5|685| +2450815|5677|5|324| +2450815|5678|5|890| +2450815|5680|5|| +2450815|5683|5|25| +2450815|5684|5|831| +2450815|5686|5|262| +2450815|5689|5|1000| +2450815|5690|5|224| +2450815|5692|5|553| +2450815|5695|5|718| +2450815|5696|5|396| +2450815|5698|5|785| +2450815|5701|5|519| +2450815|5702|5|168| +2450815|5704|5|24| +2450815|5707|5|905| +2450815|5708|5|617| +2450815|5710|5|680| +2450815|5713|5|704| +2450815|5714|5|463| +2450815|5716|5|28| +2450815|5719|5|125| +2450815|5720|5|782| +2450815|5722|5|18| +2450815|5725|5|838| +2450815|5726|5|631| +2450815|5728|5|| +2450815|5731|5|198| +2450815|5732|5|349| +2450815|5734|5|694| +2450815|5737|5|414| +2450815|5738|5|214| +2450815|5740|5|948| +2450815|5743|5|| +2450815|5744|5|450| +2450815|5746|5|496| +2450815|5749|5|106| +2450815|5750|5|857| +2450815|5752|5|| +2450815|5755|5|238| +2450815|5756|5|437| +2450815|5758|5|385| +2450815|5761|5|935| +2450815|5762|5|| +2450815|5764|5|790| +2450815|5767|5|| +2450815|5768|5|467| +2450815|5770|5|| +2450815|5773|5|687| +2450815|5774|5|265| +2450815|5776|5|322| +2450815|5779|5|149| +2450815|5780|5|380| +2450815|5782|5|606| +2450815|5785|5|441| +2450815|5786|5|119| +2450815|5788|5|711| +2450815|5791|5|395| +2450815|5792|5|908| +2450815|5794|5|942| +2450815|5797|5|924| +2450815|5798|5|321| +2450815|5800|5|464| +2450815|5803|5|| +2450815|5804|5|| +2450815|5806|5|951| +2450815|5809|5|385| +2450815|5810|5|650| +2450815|5812|5|298| +2450815|5815|5|161| +2450815|5816|5|740| +2450815|5818|5|| +2450815|5821|5|968| +2450815|5822|5|109| +2450815|5824|5|736| +2450815|5827|5|978| +2450815|5828|5|41| +2450815|5830|5|68| +2450815|5833|5|293| +2450815|5834|5|| +2450815|5836|5|398| +2450815|5839|5|980| +2450815|5840|5|415| +2450815|5842|5|94| +2450815|5845|5|458| +2450815|5846|5|599| +2450815|5848|5|625| +2450815|5851|5|310| +2450815|5852|5|929| +2450815|5854|5|320| +2450815|5857|5|682| +2450815|5858|5|244| +2450815|5860|5|935| +2450815|5863|5|9| +2450815|5864|5|750| +2450815|5866|5|188| +2450815|5869|5|6| +2450815|5870|5|355| +2450815|5872|5|82| +2450815|5875|5|249| +2450815|5876|5|984| +2450815|5878|5|568| +2450815|5881|5|739| +2450815|5882|5|989| +2450815|5884|5|370| +2450815|5887|5|368| +2450815|5888|5|692| +2450815|5890|5|539| +2450815|5893|5|215| +2450815|5894|5|48| +2450815|5896|5|79| +2450815|5899|5|218| +2450815|5900|5|475| +2450815|5902|5|211| +2450815|5905|5|189| +2450815|5906|5|999| +2450815|5908|5|526| +2450815|5911|5|415| +2450815|5912|5|216| +2450815|5914|5|823| +2450815|5917|5|916| +2450815|5918|5|902| +2450815|5920|5|847| +2450815|5923|5|193| +2450815|5924|5|426| +2450815|5926|5|504| +2450815|5929|5|145| +2450815|5930|5|591| +2450815|5932|5|995| +2450815|5935|5|35| +2450815|5936|5|103| +2450815|5938|5|1000| +2450815|5941|5|543| +2450815|5942|5|218| +2450815|5944|5|29| +2450815|5947|5|102| +2450815|5948|5|661| +2450815|5950|5|947| +2450815|5953|5|228| +2450815|5954|5|984| +2450815|5956|5|327| +2450815|5959|5|147| +2450815|5960|5|691| +2450815|5962|5|821| +2450815|5965|5|386| +2450815|5966|5|991| +2450815|5968|5|574| +2450815|5971|5|348| +2450815|5972|5|331| +2450815|5974|5|814| +2450815|5977|5|104| +2450815|5978|5|| +2450815|5980|5|939| +2450815|5983|5|721| +2450815|5984|5|622| +2450815|5986|5|31| +2450815|5989|5|387| +2450815|5990|5|593| +2450815|5992|5|148| +2450815|5995|5|462| +2450815|5996|5|206| +2450815|5998|5|220| +2450815|6001|5|606| +2450815|6002|5|992| +2450815|6004|5|617| +2450815|6007|5|306| +2450815|6008|5|345| +2450815|6010|5|951| +2450815|6013|5|955| +2450815|6014|5|508| +2450815|6016|5|369| +2450815|6019|5|190| +2450815|6020|5|663| +2450815|6022|5|693| +2450815|6025|5|632| +2450815|6026|5|440| +2450815|6028|5|237| +2450815|6031|5|954| +2450815|6032|5|790| +2450815|6034|5|765| +2450815|6037|5|254| +2450815|6038|5|799| +2450815|6040|5|922| +2450815|6043|5|995| +2450815|6044|5|670| +2450815|6046|5|179| +2450815|6049|5|325| +2450815|6050|5|83| +2450815|6052|5|188| +2450815|6055|5|557| +2450815|6056|5|406| +2450815|6058|5|526| +2450815|6061|5|503| +2450815|6062|5|829| +2450815|6064|5|8| +2450815|6067|5|452| +2450815|6068|5|121| +2450815|6070|5|809| +2450815|6073|5|441| +2450815|6074|5|443| +2450815|6076|5|949| +2450815|6079|5|267| +2450815|6080|5|164| +2450815|6082|5|180| +2450815|6085|5|800| +2450815|6086|5|785| +2450815|6088|5|50| +2450815|6091|5|455| +2450815|6092|5|471| +2450815|6094|5|165| +2450815|6097|5|736| +2450815|6098|5|754| +2450815|6100|5|559| +2450815|6103|5|| +2450815|6104|5|562| +2450815|6106|5|652| +2450815|6109|5|344| +2450815|6110|5|854| +2450815|6112|5|466| +2450815|6115|5|816| +2450815|6116|5|481| +2450815|6118|5|135| +2450815|6121|5|964| +2450815|6122|5|14| +2450815|6124|5|174| +2450815|6127|5|733| +2450815|6128|5|149| +2450815|6130|5|910| +2450815|6133|5|917| +2450815|6134|5|30| +2450815|6136|5|401| +2450815|6139|5|306| +2450815|6140|5|925| +2450815|6142|5|390| +2450815|6145|5|328| +2450815|6146|5|570| +2450815|6148|5|| +2450815|6151|5|616| +2450815|6152|5|305| +2450815|6154|5|227| +2450815|6157|5|| +2450815|6158|5|719| +2450815|6160|5|67| +2450815|6163|5|346| +2450815|6164|5|859| +2450815|6166|5|746| +2450815|6169|5|921| +2450815|6170|5|75| +2450815|6172|5|228| +2450815|6175|5|160| +2450815|6176|5|828| +2450815|6178|5|797| +2450815|6181|5|550| +2450815|6182|5|81| +2450815|6184|5|530| +2450815|6187|5|594| +2450815|6188|5|553| +2450815|6190|5|462| +2450815|6193|5|757| +2450815|6194|5|| +2450815|6196|5|312| +2450815|6199|5|538| +2450815|6200|5|64| +2450815|6202|5|462| +2450815|6205|5|545| +2450815|6206|5|75| +2450815|6208|5|343| +2450815|6211|5|53| +2450815|6212|5|686| +2450815|6214|5|198| +2450815|6217|5|895| +2450815|6218|5|252| +2450815|6220|5|254| +2450815|6223|5|492| +2450815|6224|5|585| +2450815|6226|5|289| +2450815|6229|5|600| +2450815|6230|5|501| +2450815|6232|5|401| +2450815|6235|5|311| +2450815|6236|5|755| +2450815|6238|5|458| +2450815|6241|5|793| +2450815|6242|5|900| +2450815|6244|5|359| +2450815|6247|5|603| +2450815|6248|5|723| +2450815|6250|5|487| +2450815|6253|5|542| +2450815|6254|5|| +2450815|6256|5|985| +2450815|6259|5|175| +2450815|6260|5|| +2450815|6262|5|685| +2450815|6265|5|315| +2450815|6266|5|857| +2450815|6268|5|687| +2450815|6271|5|196| +2450815|6272|5|292| +2450815|6274|5|888| +2450815|6277|5|467| +2450815|6278|5|827| +2450815|6280|5|927| +2450815|6283|5|38| +2450815|6284|5|447| +2450815|6286|5|424| +2450815|6289|5|38| +2450815|6290|5|342| +2450815|6292|5|700| +2450815|6295|5|579| +2450815|6296|5|544| +2450815|6298|5|11| +2450815|6301|5|658| +2450815|6302|5|576| +2450815|6304|5|266| +2450815|6307|5|85| +2450815|6308|5|632| +2450815|6310|5|| +2450815|6313|5|54| +2450815|6314|5|175| +2450815|6316|5|989| +2450815|6319|5|378| +2450815|6320|5|918| +2450815|6322|5|167| +2450815|6325|5|644| +2450815|6326|5|677| +2450815|6328|5|294| +2450815|6331|5|526| +2450815|6332|5|876| +2450815|6334|5|550| +2450815|6337|5|| +2450815|6338|5|881| +2450815|6340|5|787| +2450815|6343|5|977| +2450815|6344|5|694| +2450815|6346|5|124| +2450815|6349|5|558| +2450815|6350|5|435| +2450815|6352|5|832| +2450815|6355|5|166| +2450815|6356|5|667| +2450815|6358|5|671| +2450815|6361|5|572| +2450815|6362|5|423| +2450815|6364|5|900| +2450815|6367|5|186| +2450815|6368|5|425| +2450815|6370|5|467| +2450815|6373|5|645| +2450815|6374|5|48| +2450815|6376|5|416| +2450815|6379|5|497| +2450815|6380|5|505| +2450815|6382|5|288| +2450815|6385|5|838| +2450815|6386|5|440| +2450815|6388|5|604| +2450815|6391|5|262| +2450815|6392|5|578| +2450815|6394|5|651| +2450815|6397|5|577| +2450815|6398|5|461| +2450815|6400|5|788| +2450815|6403|5|257| +2450815|6404|5|423| +2450815|6406|5|765| +2450815|6409|5|106| +2450815|6410|5|305| +2450815|6412|5|330| +2450815|6415|5|156| +2450815|6416|5|346| +2450815|6418|5|696| +2450815|6421|5|528| +2450815|6422|5|134| +2450815|6424|5|693| +2450815|6427|5|548| +2450815|6428|5|592| +2450815|6430|5|90| +2450815|6433|5|706| +2450815|6434|5|795| +2450815|6436|5|459| +2450815|6439|5|884| +2450815|6440|5|609| +2450815|6442|5|817| +2450815|6445|5|274| +2450815|6446|5|367| +2450815|6448|5|147| +2450815|6451|5|248| +2450815|6452|5|249| +2450815|6454|5|618| +2450815|6457|5|373| +2450815|6458|5|941| +2450815|6460|5|94| +2450815|6463|5|335| +2450815|6464|5|841| +2450815|6466|5|774| +2450815|6469|5|307| +2450815|6470|5|278| +2450815|6472|5|3| +2450815|6475|5|737| +2450815|6476|5|240| +2450815|6478|5|514| +2450815|6481|5|404| +2450815|6482|5|723| +2450815|6484|5|78| +2450815|6487|5|476| +2450815|6488|5|270| +2450815|6490|5|215| +2450815|6493|5|878| +2450815|6494|5|314| +2450815|6496|5|267| +2450815|6499|5|145| +2450815|6500|5|917| +2450815|6502|5|280| +2450815|6505|5|238| +2450815|6506|5|80| +2450815|6508|5|277| +2450815|6511|5|61| +2450815|6512|5|526| +2450815|6514|5|517| +2450815|6517|5|520| +2450815|6518|5|968| +2450815|6520|5|139| +2450815|6523|5|729| +2450815|6524|5|229| +2450815|6526|5|447| +2450815|6529|5|455| +2450815|6530|5|457| +2450815|6532|5|895| +2450815|6535|5|552| +2450815|6536|5|331| +2450815|6538|5|452| +2450815|6541|5|291| +2450815|6542|5|33| +2450815|6544|5|907| +2450815|6547|5|271| +2450815|6548|5|995| +2450815|6550|5|394| +2450815|6553|5|590| +2450815|6554|5|664| +2450815|6556|5|488| +2450815|6559|5|564| +2450815|6560|5|734| +2450815|6562|5|805| +2450815|6565|5|638| +2450815|6566|5|808| +2450815|6568|5|534| +2450815|6571|5|113| +2450815|6572|5|918| +2450815|6574|5|333| +2450815|6577|5|422| +2450815|6578|5|80| +2450815|6580|5|229| +2450815|6583|5|434| +2450815|6584|5|670| +2450815|6586|5|911| +2450815|6589|5|| +2450815|6590|5|148| +2450815|6592|5|771| +2450815|6595|5|870| +2450815|6596|5|294| +2450815|6598|5|756| +2450815|6601|5|470| +2450815|6602|5|203| +2450815|6604|5|908| +2450815|6607|5|700| +2450815|6608|5|759| +2450815|6610|5|849| +2450815|6613|5|451| +2450815|6614|5|905| +2450815|6616|5|600| +2450815|6619|5|965| +2450815|6620|5|662| +2450815|6622|5|745| +2450815|6625|5|948| +2450815|6626|5|| +2450815|6628|5|715| +2450815|6631|5|440| +2450815|6632|5|| +2450815|6634|5|641| +2450815|6637|5|626| +2450815|6638|5|54| +2450815|6640|5|586| +2450815|6643|5|238| +2450815|6644|5|12| +2450815|6646|5|| +2450815|6649|5|653| +2450815|6650|5|294| +2450815|6652|5|822| +2450815|6655|5|666| +2450815|6656|5|506| +2450815|6658|5|606| +2450815|6661|5|788| +2450815|6662|5|88| +2450815|6664|5|460| +2450815|6667|5|188| +2450815|6668|5|338| +2450815|6670|5|146| +2450815|6673|5|653| +2450815|6674|5|172| +2450815|6676|5|414| +2450815|6679|5|586| +2450815|6680|5|529| +2450815|6682|5|627| +2450815|6685|5|846| +2450815|6686|5|489| +2450815|6688|5|903| +2450815|6691|5|885| +2450815|6692|5|884| +2450815|6694|5|761| +2450815|6697|5|268| +2450815|6698|5|454| +2450815|6700|5|220| +2450815|6703|5|631| +2450815|6704|5|49| +2450815|6706|5|523| +2450815|6709|5|126| +2450815|6710|5|248| +2450815|6712|5|369| +2450815|6715|5|476| +2450815|6716|5|981| +2450815|6718|5|177| +2450815|6721|5|944| +2450815|6722|5|919| +2450815|6724|5|312| +2450815|6727|5|943| +2450815|6728|5|703| +2450815|6730|5|720| +2450815|6733|5|659| +2450815|6734|5|478| +2450815|6736|5|293| +2450815|6739|5|78| +2450815|6740|5|324| +2450815|6742|5|370| +2450815|6745|5|316| +2450815|6746|5|483| +2450815|6748|5|861| +2450815|6751|5|726| +2450815|6752|5|16| +2450815|6754|5|721| +2450815|6757|5|| +2450815|6758|5|610| +2450815|6760|5|809| +2450815|6763|5|903| +2450815|6764|5|849| +2450815|6766|5|173| +2450815|6769|5|| +2450815|6770|5|846| +2450815|6772|5|175| +2450815|6775|5|842| +2450815|6776|5|995| +2450815|6778|5|199| +2450815|6781|5|102| +2450815|6782|5|764| +2450815|6784|5|251| +2450815|6787|5|437| +2450815|6788|5|607| +2450815|6790|5|22| +2450815|6793|5|103| +2450815|6794|5|42| +2450815|6796|5|913| +2450815|6799|5|47| +2450815|6800|5|805| +2450815|6802|5|46| +2450815|6805|5|215| +2450815|6806|5|215| +2450815|6808|5|849| +2450815|6811|5|477| +2450815|6812|5|172| +2450815|6814|5|966| +2450815|6817|5|984| +2450815|6818|5|546| +2450815|6820|5|391| +2450815|6823|5|502| +2450815|6824|5|660| +2450815|6826|5|610| +2450815|6829|5|384| +2450815|6830|5|968| +2450815|6832|5|789| +2450815|6835|5|963| +2450815|6836|5|961| +2450815|6838|5|267| +2450815|6841|5|328| +2450815|6842|5|464| +2450815|6844|5|466| +2450815|6847|5|911| +2450815|6848|5|131| +2450815|6850|5|5| +2450815|6853|5|939| +2450815|6854|5|289| +2450815|6856|5|115| +2450815|6859|5|857| +2450815|6860|5|96| +2450815|6862|5|924| +2450815|6865|5|393| +2450815|6866|5|803| +2450815|6868|5|254| +2450815|6871|5|872| +2450815|6872|5|464| +2450815|6874|5|321| +2450815|6877|5|294| +2450815|6878|5|794| +2450815|6880|5|206| +2450815|6883|5|453| +2450815|6884|5|186| +2450815|6886|5|970| +2450815|6889|5|344| +2450815|6890|5|145| +2450815|6892|5|87| +2450815|6895|5|529| +2450815|6896|5|219| +2450815|6898|5|689| +2450815|6901|5|| +2450815|6902|5|28| +2450815|6904|5|531| +2450815|6907|5|134| +2450815|6908|5|569| +2450815|6910|5|385| +2450815|6913|5|592| +2450815|6914|5|999| +2450815|6916|5|59| +2450815|6919|5|367| +2450815|6920|5|151| +2450815|6922|5|76| +2450815|6925|5|| +2450815|6926|5|866| +2450815|6928|5|166| +2450815|6931|5|79| +2450815|6932|5|285| +2450815|6934|5|433| +2450815|6937|5|885| +2450815|6938|5|232| +2450815|6940|5|997| +2450815|6943|5|890| +2450815|6944|5|579| +2450815|6946|5|818| +2450815|6949|5|397| +2450815|6950|5|777| +2450815|6952|5|635| +2450815|6955|5|883| +2450815|6956|5|464| +2450815|6958|5|227| +2450815|6961|5|249| +2450815|6962|5|795| +2450815|6964|5|293| +2450815|6967|5|151| +2450815|6968|5|403| +2450815|6970|5|109| +2450815|6973|5|679| +2450815|6974|5|735| +2450815|6976|5|857| +2450815|6979|5|572| +2450815|6980|5|907| +2450815|6982|5|171| +2450815|6985|5|| +2450815|6986|5|955| +2450815|6988|5|623| +2450815|6991|5|976| +2450815|6992|5|969| +2450815|6994|5|407| +2450815|6997|5|728| +2450815|6998|5|538| +2450815|7000|5|594| +2450815|7003|5|452| +2450815|7004|5|563| +2450815|7006|5|175| +2450815|7009|5|686| +2450815|7010|5|985| +2450815|7012|5|85| +2450815|7015|5|354| +2450815|7016|5|12| +2450815|7018|5|840| +2450815|7021|5|681| +2450815|7022|5|595| +2450815|7024|5|325| +2450815|7027|5|592| +2450815|7028|5|675| +2450815|7030|5|87| +2450815|7033|5|| +2450815|7034|5|| +2450815|7036|5|252| +2450815|7039|5|214| +2450815|7040|5|749| +2450815|7042|5|676| +2450815|7045|5|736| +2450815|7046|5|324| +2450815|7048|5|168| +2450815|7051|5|356| +2450815|7052|5|266| +2450815|7054|5|284| +2450815|7057|5|524| +2450815|7058|5|508| +2450815|7060|5|217| +2450815|7063|5|767| +2450815|7064|5|878| +2450815|7066|5|944| +2450815|7069|5|615| +2450815|7070|5|142| +2450815|7072|5|303| +2450815|7075|5|653| +2450815|7076|5|436| +2450815|7078|5|985| +2450815|7081|5|560| +2450815|7082|5|454| +2450815|7084|5|670| +2450815|7087|5|339| +2450815|7088|5|816| +2450815|7090|5|271| +2450815|7093|5|583| +2450815|7094|5|544| +2450815|7096|5|122| +2450815|7099|5|221| +2450815|7100|5|516| +2450815|7102|5|240| +2450815|7105|5|111| +2450815|7106|5|677| +2450815|7108|5|218| +2450815|7111|5|425| +2450815|7112|5|977| +2450815|7114|5|312| +2450815|7117|5|723| +2450815|7118|5|69| +2450815|7120|5|342| +2450815|7123|5|571| +2450815|7124|5|280| +2450815|7126|5|715| +2450815|7129|5|548| +2450815|7130|5|837| +2450815|7132|5|789| +2450815|7135|5|72| +2450815|7136|5|504| +2450815|7138|5|329| +2450815|7141|5|352| +2450815|7142|5|| +2450815|7144|5|386| +2450815|7147|5|648| +2450815|7148|5|539| +2450815|7150|5|808| +2450815|7153|5|237| +2450815|7154|5|610| +2450815|7156|5|367| +2450815|7159|5|90| +2450815|7160|5|853| +2450815|7162|5|968| +2450815|7165|5|629| +2450815|7166|5|647| +2450815|7168|5|490| +2450815|7171|5|931| +2450815|7172|5|10| +2450815|7174|5|101| +2450815|7177|5|662| +2450815|7178|5|263| +2450815|7180|5|755| +2450815|7183|5|252| +2450815|7184|5|981| +2450815|7186|5|230| +2450815|7189|5|427| +2450815|7190|5|970| +2450815|7192|5|773| +2450815|7195|5|809| +2450815|7196|5|788| +2450815|7198|5|480| +2450815|7201|5|81| +2450815|7202|5|850| +2450815|7204|5|471| +2450815|7207|5|583| +2450815|7208|5|212| +2450815|7210|5|386| +2450815|7213|5|684| +2450815|7214|5|80| +2450815|7216|5|830| +2450815|7219|5|628| +2450815|7220|5|807| +2450815|7222|5|934| +2450815|7225|5|178| +2450815|7226|5|165| +2450815|7228|5|61| +2450815|7231|5|517| +2450815|7232|5|51| +2450815|7234|5|800| +2450815|7237|5|| +2450815|7238|5|851| +2450815|7240|5|420| +2450815|7243|5|403| +2450815|7244|5|361| +2450815|7246|5|29| +2450815|7249|5|35| +2450815|7250|5|214| +2450815|7252|5|518| +2450815|7255|5|265| +2450815|7256|5|212| +2450815|7258|5|125| +2450815|7261|5|998| +2450815|7262|5|581| +2450815|7264|5|356| +2450815|7267|5|273| +2450815|7268|5|804| +2450815|7270|5|955| +2450815|7273|5|25| +2450815|7274|5|875| +2450815|7276|5|75| +2450815|7279|5|103| +2450815|7280|5|267| +2450815|7282|5|| +2450815|7285|5|36| +2450815|7286|5|460| +2450815|7288|5|464| +2450815|7291|5|207| +2450815|7292|5|927| +2450815|7294|5|196| +2450815|7297|5|454| +2450815|7298|5|755| +2450815|7300|5|200| +2450815|7303|5|833| +2450815|7304|5|449| +2450815|7306|5|409| +2450815|7309|5|750| +2450815|7310|5|593| +2450815|7312|5|519| +2450815|7315|5|938| +2450815|7316|5|474| +2450815|7318|5|103| +2450815|7321|5|795| +2450815|7322|5|321| +2450815|7324|5|493| +2450815|7327|5|481| +2450815|7328|5|248| +2450815|7330|5|1000| +2450815|7333|5|470| +2450815|7334|5|529| +2450815|7336|5|912| +2450815|7339|5|579| +2450815|7340|5|899| +2450815|7342|5|508| +2450815|7345|5|366| +2450815|7346|5|| +2450815|7348|5|71| +2450815|7351|5|657| +2450815|7352|5|553| +2450815|7354|5|663| +2450815|7357|5|731| +2450815|7358|5|830| +2450815|7360|5|514| +2450815|7363|5|189| +2450815|7364|5|716| +2450815|7366|5|479| +2450815|7369|5|743| +2450815|7370|5|81| +2450815|7372|5|340| +2450815|7375|5|456| +2450815|7376|5|677| +2450815|7378|5|37| +2450815|7381|5|829| +2450815|7382|5|837| +2450815|7384|5|784| +2450815|7387|5|303| +2450815|7388|5|694| +2450815|7390|5|380| +2450815|7393|5|620| +2450815|7394|5|483| +2450815|7396|5|623| +2450815|7399|5|281| +2450815|7400|5|777| +2450815|7402|5|870| +2450815|7405|5|51| +2450815|7406|5|723| +2450815|7408|5|725| +2450815|7411|5|405| +2450815|7412|5|820| +2450815|7414|5|| +2450815|7417|5|620| +2450815|7418|5|96| +2450815|7420|5|648| +2450815|7423|5|268| +2450815|7424|5|| +2450815|7426|5|168| +2450815|7429|5|301| +2450815|7430|5|847| +2450815|7432|5|360| +2450815|7435|5|768| +2450815|7436|5|847| +2450815|7438|5|| +2450815|7441|5|708| +2450815|7442|5|71| +2450815|7444|5|286| +2450815|7447|5|929| +2450815|7448|5|415| +2450815|7450|5|574| +2450815|7453|5|145| +2450815|7454|5|372| +2450815|7456|5|483| +2450815|7459|5|934| +2450815|7460|5|803| +2450815|7462|5|296| +2450815|7465|5|493| +2450815|7466|5|146| +2450815|7468|5|491| +2450815|7471|5|234| +2450815|7472|5|624| +2450815|7474|5|| +2450815|7477|5|592| +2450815|7478|5|141| +2450815|7480|5|319| +2450815|7483|5|396| +2450815|7484|5|569| +2450815|7486|5|738| +2450815|7489|5|647| +2450815|7490|5|117| +2450815|7492|5|742| +2450815|7495|5|| +2450815|7496|5|499| +2450815|7498|5|108| +2450815|7501|5|108| +2450815|7502|5|701| +2450815|7504|5|902| +2450815|7507|5|675| +2450815|7508|5|270| +2450815|7510|5|950| +2450815|7513|5|271| +2450815|7514|5|874| +2450815|7516|5|289| +2450815|7519|5|703| +2450815|7520|5|165| +2450815|7522|5|979| +2450815|7525|5|943| +2450815|7526|5|649| +2450815|7528|5|56| +2450815|7531|5|401| +2450815|7532|5|341| +2450815|7534|5|737| +2450815|7537|5|990| +2450815|7538|5|567| +2450815|7540|5|901| +2450815|7543|5|27| +2450815|7544|5|678| +2450815|7546|5|96| +2450815|7549|5|802| +2450815|7550|5|482| +2450815|7552|5|545| +2450815|7555|5|776| +2450815|7556|5|84| +2450815|7558|5|548| +2450815|7561|5|822| +2450815|7562|5|606| +2450815|7564|5|792| +2450815|7567|5|125| +2450815|7568|5|711| +2450815|7570|5|14| +2450815|7573|5|669| +2450815|7574|5|544| +2450815|7576|5|808| +2450815|7579|5|270| +2450815|7580|5|961| +2450815|7582|5|331| +2450815|7585|5|839| +2450815|7586|5|403| +2450815|7588|5|76| +2450815|7591|5|974| +2450815|7592|5|131| +2450815|7594|5|963| +2450815|7597|5|618| +2450815|7598|5|640| +2450815|7600|5|665| +2450815|7603|5|455| +2450815|7604|5|621| +2450815|7606|5|265| +2450815|7609|5|161| +2450815|7610|5|95| +2450815|7612|5|828| +2450815|7615|5|253| +2450815|7616|5|118| +2450815|7618|5|421| +2450815|7621|5|0| +2450815|7622|5|358| +2450815|7624|5|529| +2450815|7627|5|366| +2450815|7628|5|850| +2450815|7630|5|564| +2450815|7633|5|42| +2450815|7634|5|172| +2450815|7636|5|477| +2450815|7639|5|911| +2450815|7640|5|755| +2450815|7642|5|491| +2450815|7645|5|154| +2450815|7646|5|478| +2450815|7648|5|779| +2450815|7651|5|344| +2450815|7652|5|199| +2450815|7654|5|355| +2450815|7657|5|30| +2450815|7658|5|626| +2450815|7660|5|| +2450815|7663|5|152| +2450815|7664|5|82| +2450815|7666|5|577| +2450815|7669|5|694| +2450815|7670|5|185| +2450815|7672|5|374| +2450815|7675|5|438| +2450815|7676|5|767| +2450815|7678|5|| +2450815|7681|5|889| +2450815|7682|5|361| +2450815|7684|5|160| +2450815|7687|5|326| +2450815|7688|5|824| +2450815|7690|5|401| +2450815|7693|5|800| +2450815|7694|5|639| +2450815|7696|5|393| +2450815|7699|5|299| +2450815|7700|5|244| +2450815|7702|5|112| +2450815|7705|5|785| +2450815|7706|5|468| +2450815|7708|5|329| +2450815|7711|5|91| +2450815|7712|5|868| +2450815|7714|5|252| +2450815|7717|5|37| +2450815|7718|5|9| +2450815|7720|5|770| +2450815|7723|5|120| +2450815|7724|5|206| +2450815|7726|5|147| +2450815|7729|5|412| +2450815|7730|5|446| +2450815|7732|5|287| +2450815|7735|5|596| +2450815|7736|5|916| +2450815|7738|5|225| +2450815|7741|5|135| +2450815|7742|5|19| +2450815|7744|5|731| +2450815|7747|5|55| +2450815|7748|5|845| +2450815|7750|5|130| +2450815|7753|5|154| +2450815|7754|5|791| +2450815|7756|5|186| +2450815|7759|5|145| +2450815|7760|5|318| +2450815|7762|5|933| +2450815|7765|5|171| +2450815|7766|5|475| +2450815|7768|5|351| +2450815|7771|5|| +2450815|7772|5|536| +2450815|7774|5|339| +2450815|7777|5|291| +2450815|7778|5|984| +2450815|7780|5|61| +2450815|7783|5|989| +2450815|7784|5|600| +2450815|7786|5|222| +2450815|7789|5|898| +2450815|7790|5|91| +2450815|7792|5|976| +2450815|7795|5|407| +2450815|7796|5|775| +2450815|7798|5|850| +2450815|7801|5|625| +2450815|7802|5|545| +2450815|7804|5|863| +2450815|7807|5|805| +2450815|7808|5|997| +2450815|7810|5|854| +2450815|7813|5|650| +2450815|7814|5|821| +2450815|7816|5|| +2450815|7819|5|142| +2450815|7820|5|284| +2450815|7822|5|454| +2450815|7825|5|424| +2450815|7826|5|90| +2450815|7828|5|814| +2450815|7831|5|685| +2450815|7832|5|275| +2450815|7834|5|361| +2450815|7837|5|513| +2450815|7838|5|527| +2450815|7840|5|62| +2450815|7843|5|43| +2450815|7844|5|917| +2450815|7846|5|532| +2450815|7849|5|801| +2450815|7850|5|840| +2450815|7852|5|754| +2450815|7855|5|675| +2450815|7856|5|23| +2450815|7858|5|377| +2450815|7861|5|14| +2450815|7862|5|548| +2450815|7864|5|705| +2450815|7867|5|| +2450815|7868|5|643| +2450815|7870|5|379| +2450815|7873|5|251| +2450815|7874|5|11| +2450815|7876|5|584| +2450815|7879|5|162| +2450815|7880|5|807| +2450815|7882|5|53| +2450815|7885|5|926| +2450815|7886|5|40| +2450815|7888|5|77| +2450815|7891|5|934| +2450815|7892|5|888| +2450815|7894|5|275| +2450815|7897|5|895| +2450815|7898|5|| +2450815|7900|5|711| +2450815|7903|5|626| +2450815|7904|5|648| +2450815|7906|5|969| +2450815|7909|5|993| +2450815|7910|5|81| +2450815|7912|5|992| +2450815|7915|5|31| +2450815|7916|5|50| +2450815|7918|5|| +2450815|7921|5|| +2450815|7922|5|432| +2450815|7924|5|101| +2450815|7927|5|30| +2450815|7928|5|98| +2450815|7930|5|846| +2450815|7933|5|540| +2450815|7934|5|294| +2450815|7936|5|808| +2450815|7939|5|28| +2450815|7940|5|334| +2450815|7942|5|631| +2450815|7945|5|327| +2450815|7946|5|581| +2450815|7948|5|382| +2450815|7951|5|749| +2450815|7952|5|709| +2450815|7954|5|220| +2450815|7957|5|474| +2450815|7958|5|430| +2450815|7960|5|991| +2450815|7963|5|623| +2450815|7964|5|489| +2450815|7966|5|999| +2450815|7969|5|671| +2450815|7970|5|344| +2450815|7972|5|941| +2450815|7975|5|589| +2450815|7976|5|385| +2450815|7978|5|86| +2450815|7981|5|229| +2450815|7982|5|423| +2450815|7984|5|299| +2450815|7987|5|732| +2450815|7988|5|948| +2450815|7990|5|557| +2450815|7993|5|147| +2450815|7994|5|851| +2450815|7996|5|999| +2450815|7999|5|60| +2450815|8000|5|633| +2450815|8002|5|| +2450815|8005|5|925| +2450815|8006|5|434| +2450815|8008|5|516| +2450815|8011|5|580| +2450815|8012|5|326| +2450815|8014|5|376| +2450815|8017|5|568| +2450815|8018|5|958| +2450815|8020|5|711| +2450815|8023|5|153| +2450815|8024|5|580| +2450815|8026|5|| +2450815|8029|5|205| +2450815|8030|5|500| +2450815|8032|5|381| +2450815|8035|5|222| +2450815|8036|5|128| +2450815|8038|5|672| +2450815|8041|5|568| +2450815|8042|5|98| +2450815|8044|5|144| +2450815|8047|5|679| +2450815|8048|5|984| +2450815|8050|5|784| +2450815|8053|5|369| +2450815|8054|5|23| +2450815|8056|5|411| +2450815|8059|5|399| +2450815|8060|5|259| +2450815|8062|5|991| +2450815|8065|5|756| +2450815|8066|5|638| +2450815|8068|5|772| +2450815|8071|5|986| +2450815|8072|5|8| +2450815|8074|5|34| +2450815|8077|5|676| +2450815|8078|5|75| +2450815|8080|5|| +2450815|8083|5|999| +2450815|8084|5|974| +2450815|8086|5|607| +2450815|8089|5|772| +2450815|8090|5|334| +2450815|8092|5|276| +2450815|8095|5|465| +2450815|8096|5|651| +2450815|8098|5|453| +2450815|8101|5|235| +2450815|8102|5|431| +2450815|8104|5|103| +2450815|8107|5|104| +2450815|8108|5|43| +2450815|8110|5|204| +2450815|8113|5|5| +2450815|8114|5|754| +2450815|8116|5|306| +2450815|8119|5|847| +2450815|8120|5|865| +2450815|8122|5|582| +2450815|8125|5|596| +2450815|8126|5|328| +2450815|8128|5|660| +2450815|8131|5|241| +2450815|8132|5|561| +2450815|8134|5|290| +2450815|8137|5|27| +2450815|8138|5|632| +2450815|8140|5|734| +2450815|8143|5|128| +2450815|8144|5|579| +2450815|8146|5|93| +2450815|8149|5|317| +2450815|8150|5|644| +2450815|8152|5|144| +2450815|8155|5|259| +2450815|8156|5|933| +2450815|8158|5|701| +2450815|8161|5|| +2450815|8162|5|793| +2450815|8164|5|936| +2450815|8167|5|936| +2450815|8168|5|905| +2450815|8170|5|981| +2450815|8173|5|487| +2450815|8174|5|545| +2450815|8176|5|423| +2450815|8179|5|451| +2450815|8180|5|812| +2450815|8182|5|697| +2450815|8185|5|| +2450815|8186|5|669| +2450815|8188|5|86| +2450815|8191|5|671| +2450815|8192|5|273| +2450815|8194|5|277| +2450815|8197|5|154| +2450815|8198|5|367| +2450815|8200|5|520| +2450815|8203|5|483| +2450815|8204|5|441| +2450815|8206|5|583| +2450815|8209|5|522| +2450815|8210|5|529| +2450815|8212|5|916| +2450815|8215|5|| +2450815|8216|5|937| +2450815|8218|5|93| +2450815|8221|5|792| +2450815|8222|5|894| +2450815|8224|5|277| +2450815|8227|5|198| +2450815|8228|5|36| +2450815|8230|5|506| +2450815|8233|5|32| +2450815|8234|5|215| +2450815|8236|5|570| +2450815|8239|5|634| +2450815|8240|5|44| +2450815|8242|5|622| +2450815|8245|5|873| +2450815|8246|5|815| +2450815|8248|5|402| +2450815|8251|5|257| +2450815|8252|5|932| +2450815|8254|5|454| +2450815|8257|5|420| +2450815|8258|5|625| +2450815|8260|5|| +2450815|8263|5|153| +2450815|8264|5|395| +2450815|8266|5|873| +2450815|8269|5|381| +2450815|8270|5|841| +2450815|8272|5|536| +2450815|8275|5|510| +2450815|8276|5|886| +2450815|8278|5|555| +2450815|8281|5|695| +2450815|8282|5|178| +2450815|8284|5|600| +2450815|8287|5|467| +2450815|8288|5|589| +2450815|8290|5|375| +2450815|8293|5|884| +2450815|8294|5|508| +2450815|8296|5|80| +2450815|8299|5|902| +2450815|8300|5|585| +2450815|8302|5|243| +2450815|8305|5|860| +2450815|8306|5|731| +2450815|8308|5|147| +2450815|8311|5|538| +2450815|8312|5|934| +2450815|8314|5|329| +2450815|8317|5|787| +2450815|8318|5|902| +2450815|8320|5|505| +2450815|8323|5|635| +2450815|8324|5|188| +2450815|8326|5|345| +2450815|8329|5|830| +2450815|8330|5|182| +2450815|8332|5|164| +2450815|8335|5|283| +2450815|8336|5|794| +2450815|8338|5|769| +2450815|8341|5|432| +2450815|8342|5|0| +2450815|8344|5|| +2450815|8347|5|15| +2450815|8348|5|856| +2450815|8350|5|581| +2450815|8353|5|320| +2450815|8354|5|461| +2450815|8356|5|72| +2450815|8359|5|89| +2450815|8360|5|710| +2450815|8362|5|764| +2450815|8365|5|638| +2450815|8366|5|470| +2450815|8368|5|| +2450815|8371|5|721| +2450815|8372|5|396| +2450815|8374|5|787| +2450815|8377|5|324| +2450815|8378|5|376| +2450815|8380|5|559| +2450815|8383|5|875| +2450815|8384|5|247| +2450815|8386|5|308| +2450815|8389|5|216| +2450815|8390|5|439| +2450815|8392|5|940| +2450815|8395|5|884| +2450815|8396|5|317| +2450815|8398|5|268| +2450815|8401|5|690| +2450815|8402|5|103| +2450815|8404|5|570| +2450815|8407|5|983| +2450815|8408|5|919| +2450815|8410|5|794| +2450815|8413|5|178| +2450815|8414|5|86| +2450815|8416|5|707| +2450815|8419|5|954| +2450815|8420|5|336| +2450815|8422|5|704| +2450815|8425|5|560| +2450815|8426|5|852| +2450815|8428|5|612| +2450815|8431|5|302| +2450815|8432|5|985| +2450815|8434|5|754| +2450815|8437|5|843| +2450815|8438|5|2| +2450815|8440|5|165| +2450815|8443|5|690| +2450815|8444|5|708| +2450815|8446|5|331| +2450815|8449|5|670| +2450815|8450|5|571| +2450815|8452|5|368| +2450815|8455|5|664| +2450815|8456|5|622| +2450815|8458|5|748| +2450815|8461|5|42| +2450815|8462|5|1000| +2450815|8464|5|925| +2450815|8467|5|582| +2450815|8468|5|973| +2450815|8470|5|969| +2450815|8473|5|450| +2450815|8474|5|731| +2450815|8476|5|661| +2450815|8479|5|476| +2450815|8480|5|146| +2450815|8482|5|357| +2450815|8485|5|675| +2450815|8486|5|396| +2450815|8488|5|79| +2450815|8491|5|| +2450815|8492|5|296| +2450815|8494|5|833| +2450815|8497|5|941| +2450815|8498|5|48| +2450815|8500|5|491| +2450815|8503|5|980| +2450815|8504|5|874| +2450815|8506|5|471| +2450815|8509|5|877| +2450815|8510|5|739| +2450815|8512|5|| +2450815|8515|5|681| +2450815|8516|5|860| +2450815|8518|5|145| +2450815|8521|5|185| +2450815|8522|5|| +2450815|8524|5|749| +2450815|8527|5|694| +2450815|8528|5|458| +2450815|8530|5|554| +2450815|8533|5|356| +2450815|8534|5|987| +2450815|8536|5|654| +2450815|8539|5|150| +2450815|8540|5|338| +2450815|8542|5|864| +2450815|8545|5|526| +2450815|8546|5|277| +2450815|8548|5|869| +2450815|8551|5|458| +2450815|8552|5|739| +2450815|8554|5|69| +2450815|8557|5|318| +2450815|8558|5|68| +2450815|8560|5|133| +2450815|8563|5|928| +2450815|8564|5|916| +2450815|8566|5|126| +2450815|8569|5|| +2450815|8570|5|9| +2450815|8572|5|732| +2450815|8575|5|274| +2450815|8576|5|| +2450815|8578|5|898| +2450815|8581|5|774| +2450815|8582|5|768| +2450815|8584|5|764| +2450815|8587|5|64| +2450815|8588|5|28| +2450815|8590|5|374| +2450815|8593|5|216| +2450815|8594|5|225| +2450815|8596|5|166| +2450815|8599|5|666| +2450815|8600|5|283| +2450815|8602|5|598| +2450815|8605|5|374| +2450815|8606|5|| +2450815|8608|5|112| +2450815|8611|5|801| +2450815|8612|5|507| +2450815|8614|5|236| +2450815|8617|5|984| +2450815|8618|5|180| +2450815|8620|5|210| +2450815|8623|5|827| +2450815|8624|5|278| +2450815|8626|5|987| +2450815|8629|5|100| +2450815|8630|5|211| +2450815|8632|5|612| +2450815|8635|5|248| +2450815|8636|5|457| +2450815|8638|5|578| +2450815|8641|5|417| +2450815|8642|5|800| +2450815|8644|5|719| +2450815|8647|5|35| +2450815|8648|5|446| +2450815|8650|5|331| +2450815|8653|5|451| +2450815|8654|5|260| +2450815|8656|5|153| +2450815|8659|5|| +2450815|8660|5|213| +2450815|8662|5|860| +2450815|8665|5|391| +2450815|8666|5|371| +2450815|8668|5|87| +2450815|8671|5|799| +2450815|8672|5|| +2450815|8674|5|314| +2450815|8677|5|792| +2450815|8678|5|525| +2450815|8680|5|367| +2450815|8683|5|641| +2450815|8684|5|467| +2450815|8686|5|547| +2450815|8689|5|477| +2450815|8690|5|905| +2450815|8692|5|537| +2450815|8695|5|726| +2450815|8696|5|681| +2450815|8698|5|931| +2450815|8701|5|37| +2450815|8702|5|861| +2450815|8704|5|737| +2450815|8707|5|284| +2450815|8708|5|640| +2450815|8710|5|458| +2450815|8713|5|125| +2450815|8714|5|654| +2450815|8716|5|906| +2450815|8719|5|749| +2450815|8720|5|563| +2450815|8722|5|495| +2450815|8725|5|908| +2450815|8726|5|456| +2450815|8728|5|857| +2450815|8731|5|989| +2450815|8732|5|300| +2450815|8734|5|494| +2450815|8737|5|404| +2450815|8738|5|856| +2450815|8740|5|534| +2450815|8743|5|231| +2450815|8744|5|| +2450815|8746|5|419| +2450815|8749|5|853| +2450815|8750|5|72| +2450815|8752|5|398| +2450815|8755|5|47| +2450815|8756|5|228| +2450815|8758|5|435| +2450815|8761|5|901| +2450815|8762|5|203| +2450815|8764|5|144| +2450815|8767|5|686| +2450815|8768|5|903| +2450815|8770|5|22| +2450815|8773|5|407| +2450815|8774|5|978| +2450815|8776|5|7| +2450815|8779|5|261| +2450815|8780|5|571| +2450815|8782|5|806| +2450815|8785|5|461| +2450815|8786|5|138| +2450815|8788|5|420| +2450815|8791|5|545| +2450815|8792|5|709| +2450815|8794|5|196| +2450815|8797|5|| +2450815|8798|5|766| +2450815|8800|5|445| +2450815|8803|5|712| +2450815|8804|5|888| +2450815|8806|5|77| +2450815|8809|5|722| +2450815|8810|5|625| +2450815|8812|5|884| +2450815|8815|5|502| +2450815|8816|5|115| +2450815|8818|5|462| +2450815|8821|5|733| +2450815|8822|5|845| +2450815|8824|5|157| +2450815|8827|5|391| +2450815|8828|5|335| +2450815|8830|5|809| +2450815|8833|5|802| +2450815|8834|5|604| +2450815|8836|5|107| +2450815|8839|5|| +2450815|8840|5|754| +2450815|8842|5|509| +2450815|8845|5|764| +2450815|8846|5|556| +2450815|8848|5|| +2450815|8851|5|226| +2450815|8852|5|882| +2450815|8854|5|918| +2450815|8857|5|112| +2450815|8858|5|595| +2450815|8860|5|943| +2450815|8863|5|478| +2450815|8864|5|635| +2450815|8866|5|502| +2450815|8869|5|444| +2450815|8870|5|860| +2450815|8872|5|356| +2450815|8875|5|356| +2450815|8876|5|800| +2450815|8878|5|267| +2450815|8881|5|847| +2450815|8882|5|97| +2450815|8884|5|933| +2450815|8887|5|114| +2450815|8888|5|392| +2450815|8890|5|170| +2450815|8893|5|339| +2450815|8894|5|40| +2450815|8896|5|| +2450815|8899|5|| +2450815|8900|5|403| +2450815|8902|5|828| +2450815|8905|5|479| +2450815|8906|5|683| +2450815|8908|5|896| +2450815|8911|5|849| +2450815|8912|5|333| +2450815|8914|5|| +2450815|8917|5|412| +2450815|8918|5|851| +2450815|8920|5|991| +2450815|8923|5|692| +2450815|8924|5|20| +2450815|8926|5|| +2450815|8929|5|28| +2450815|8930|5|9| +2450815|8932|5|303| +2450815|8935|5|734| +2450815|8936|5|303| +2450815|8938|5|530| +2450815|8941|5|126| +2450815|8942|5|314| +2450815|8944|5|752| +2450815|8947|5|53| +2450815|8948|5|7| +2450815|8950|5|909| +2450815|8953|5|554| +2450815|8954|5|| +2450815|8956|5|525| +2450815|8959|5|935| +2450815|8960|5|232| +2450815|8962|5|517| +2450815|8965|5|812| +2450815|8966|5|838| +2450815|8968|5|4| +2450815|8971|5|206| +2450815|8972|5|238| +2450815|8974|5|463| +2450815|8977|5|211| +2450815|8978|5|175| +2450815|8980|5|111| +2450815|8983|5|989| +2450815|8984|5|886| +2450815|8986|5|| +2450815|8989|5|659| +2450815|8990|5|976| +2450815|8992|5|713| +2450815|8995|5|| +2450815|8996|5|960| +2450815|8998|5|932| +2450815|9001|5|658| +2450815|9002|5|| +2450815|9004|5|269| +2450815|9007|5|520| +2450815|9008|5|246| +2450815|9010|5|106| +2450815|9013|5|791| +2450815|9014|5|972| +2450815|9016|5|451| +2450815|9019|5|387| +2450815|9020|5|208| +2450815|9022|5|70| +2450815|9025|5|196| +2450815|9026|5|564| +2450815|9028|5|488| +2450815|9031|5|386| +2450815|9032|5|982| +2450815|9034|5|380| +2450815|9037|5|914| +2450815|9038|5|387| +2450815|9040|5|40| +2450815|9043|5|805| +2450815|9044|5|276| +2450815|9046|5|919| +2450815|9049|5|| +2450815|9050|5|66| +2450815|9052|5|745| +2450815|9055|5|238| +2450815|9056|5|555| +2450815|9058|5|618| +2450815|9061|5|750| +2450815|9062|5|196| +2450815|9064|5|971| +2450815|9067|5|509| +2450815|9068|5|723| +2450815|9070|5|426| +2450815|9073|5|587| +2450815|9074|5|10| +2450815|9076|5|323| +2450815|9079|5|| +2450815|9080|5|5| +2450815|9082|5|951| +2450815|9085|5|736| +2450815|9086|5|559| +2450815|9088|5|854| +2450815|9091|5|19| +2450815|9092|5|489| +2450815|9094|5|962| +2450815|9097|5|920| +2450815|9098|5|527| +2450815|9100|5|353| +2450815|9103|5|496| +2450815|9104|5|377| +2450815|9106|5|444| +2450815|9109|5|635| +2450815|9110|5|938| +2450815|9112|5|559| +2450815|9115|5|10| +2450815|9116|5|861| +2450815|9118|5|294| +2450815|9121|5|310| +2450815|9122|5|330| +2450815|9124|5|792| +2450815|9127|5|646| +2450815|9128|5|904| +2450815|9130|5|248| +2450815|9133|5|549| +2450815|9134|5|963| +2450815|9136|5|114| +2450815|9139|5|323| +2450815|9140|5|829| +2450815|9142|5|623| +2450815|9145|5|| +2450815|9146|5|872| +2450815|9148|5|613| +2450815|9151|5|808| +2450815|9152|5|67| +2450815|9154|5|375| +2450815|9157|5|837| +2450815|9158|5|55| +2450815|9160|5|808| +2450815|9163|5|452| +2450815|9164|5|19| +2450815|9166|5|| +2450815|9169|5|574| +2450815|9170|5|478| +2450815|9172|5|368| +2450815|9175|5|581| +2450815|9176|5|153| +2450815|9178|5|997| +2450815|9181|5|185| +2450815|9182|5|93| +2450815|9184|5|596| +2450815|9187|5|984| +2450815|9188|5|962| +2450815|9190|5|183| +2450815|9193|5|453| +2450815|9194|5|462| +2450815|9196|5|424| +2450815|9199|5|297| +2450815|9200|5|721| +2450815|9202|5|| +2450815|9205|5|302| +2450815|9206|5|841| +2450815|9208|5|199| +2450815|9211|5|566| +2450815|9212|5|902| +2450815|9214|5|262| +2450815|9217|5|220| +2450815|9218|5|278| +2450815|9220|5|274| +2450815|9223|5|996| +2450815|9224|5|489| +2450815|9226|5|808| +2450815|9229|5|725| +2450815|9230|5|122| +2450815|9232|5|645| +2450815|9235|5|432| +2450815|9236|5|956| +2450815|9238|5|322| +2450815|9241|5|276| +2450815|9242|5|| +2450815|9244|5|571| +2450815|9247|5|973| +2450815|9248|5|271| +2450815|9250|5|717| +2450815|9253|5|50| +2450815|9254|5|804| +2450815|9256|5|279| +2450815|9259|5|722| +2450815|9260|5|843| +2450815|9262|5|9| +2450815|9265|5|89| +2450815|9266|5|958| +2450815|9268|5|385| +2450815|9271|5|132| +2450815|9272|5|713| +2450815|9274|5|905| +2450815|9277|5|765| +2450815|9278|5|206| +2450815|9280|5|814| +2450815|9283|5|183| +2450815|9284|5|620| +2450815|9286|5|498| +2450815|9289|5|318| +2450815|9290|5|31| +2450815|9292|5|877| +2450815|9295|5|985| +2450815|9296|5|316| +2450815|9298|5|498| +2450815|9301|5|22| +2450815|9302|5|476| +2450815|9304|5|258| +2450815|9307|5|545| +2450815|9308|5|855| +2450815|9310|5|844| +2450815|9313|5|| +2450815|9314|5|485| +2450815|9316|5|408| +2450815|9319|5|278| +2450815|9320|5|206| +2450815|9322|5|915| +2450815|9325|5|991| +2450815|9326|5|822| +2450815|9328|5|732| +2450815|9331|5|943| +2450815|9332|5|683| +2450815|9334|5|661| +2450815|9337|5|261| +2450815|9338|5|840| +2450815|9340|5|45| +2450815|9343|5|941| +2450815|9344|5|629| +2450815|9346|5|762| +2450815|9349|5|204| +2450815|9350|5|103| +2450815|9352|5|568| +2450815|9355|5|946| +2450815|9356|5|612| +2450815|9358|5|394| +2450815|9361|5|535| +2450815|9362|5|297| +2450815|9364|5|297| +2450815|9367|5|581| +2450815|9368|5|219| +2450815|9370|5|746| +2450815|9373|5|81| +2450815|9374|5|11| +2450815|9376|5|896| +2450815|9379|5|403| +2450815|9380|5|524| +2450815|9382|5|363| +2450815|9385|5|283| +2450815|9386|5|989| +2450815|9388|5|735| +2450815|9391|5|757| +2450815|9392|5|40| +2450815|9394|5|760| +2450815|9397|5|983| +2450815|9398|5|256| +2450815|9400|5|313| +2450815|9403|5|258| +2450815|9404|5|86| +2450815|9406|5|757| +2450815|9409|5|916| +2450815|9410|5|454| +2450815|9412|5|564| +2450815|9415|5|| +2450815|9416|5|59| +2450815|9418|5|398| +2450815|9421|5|188| +2450815|9422|5|331| +2450815|9424|5|313| +2450815|9427|5|781| +2450815|9428|5|659| +2450815|9430|5|358| +2450815|9433|5|245| +2450815|9434|5|651| +2450815|9436|5|961| +2450815|9439|5|749| +2450815|9440|5|506| +2450815|9442|5|842| +2450815|9445|5|91| +2450815|9446|5|904| +2450815|9448|5|267| +2450815|9451|5|270| +2450815|9452|5|951| +2450815|9454|5|546| +2450815|9457|5|965| +2450815|9458|5|218| +2450815|9460|5|742| +2450815|9463|5|68| +2450815|9464|5|186| +2450815|9466|5|24| +2450815|9469|5|460| +2450815|9470|5|71| +2450815|9472|5|525| +2450815|9475|5|141| +2450815|9476|5|449| +2450815|9478|5|308| +2450815|9481|5|183| +2450815|9482|5|571| +2450815|9484|5|580| +2450815|9487|5|27| +2450815|9488|5|963| +2450815|9490|5|505| +2450815|9493|5|719| +2450815|9494|5|430| +2450815|9496|5|27| +2450815|9499|5|787| +2450815|9500|5|88| +2450815|9502|5|232| +2450815|9505|5|518| +2450815|9506|5|933| +2450815|9508|5|336| +2450815|9511|5|275| +2450815|9512|5|432| +2450815|9514|5|627| +2450815|9517|5|933| +2450815|9518|5|513| +2450815|9520|5|935| +2450815|9523|5|275| +2450815|9524|5|708| +2450815|9526|5|701| +2450815|9529|5|685| +2450815|9530|5|| +2450815|9532|5|511| +2450815|9535|5|736| +2450815|9536|5|323| +2450815|9538|5|402| +2450815|9541|5|758| +2450815|9542|5|358| +2450815|9544|5|654| +2450815|9547|5|260| +2450815|9548|5|114| +2450815|9550|5|707| +2450815|9553|5|415| +2450815|9554|5|73| +2450815|9556|5|231| +2450815|9559|5|717| +2450815|9560|5|145| +2450815|9562|5|355| +2450815|9565|5|137| +2450815|9566|5|90| +2450815|9568|5|642| +2450815|9571|5|11| +2450815|9572|5|297| +2450815|9574|5|601| +2450815|9577|5|157| +2450815|9578|5|495| +2450815|9580|5|725| +2450815|9583|5|532| +2450815|9584|5|107| +2450815|9586|5|916| +2450815|9589|5|506| +2450815|9590|5|477| +2450815|9592|5|582| +2450815|9595|5|145| +2450815|9596|5|930| +2450815|9598|5|444| +2450815|9601|5|| +2450815|9602|5|93| +2450815|9604|5|684| +2450815|9607|5|639| +2450815|9608|5|566| +2450815|9610|5|239| +2450815|9613|5|963| +2450815|9614|5|578| +2450815|9616|5|| +2450815|9619|5|776| +2450815|9620|5|598| +2450815|9622|5|713| +2450815|9625|5|873| +2450815|9626|5|862| +2450815|9628|5|564| +2450815|9631|5|813| +2450815|9632|5|894| +2450815|9634|5|476| +2450815|9637|5|612| +2450815|9638|5|897| +2450815|9640|5|958| +2450815|9643|5|840| +2450815|9644|5|606| +2450815|9646|5|| +2450815|9649|5|166| +2450815|9650|5|918| +2450815|9652|5|279| +2450815|9655|5|408| +2450815|9656|5|327| +2450815|9658|5|123| +2450815|9661|5|523| +2450815|9662|5|986| +2450815|9664|5|602| +2450815|9667|5|22| +2450815|9668|5|706| +2450815|9670|5|323| +2450815|9673|5|554| +2450815|9674|5|793| +2450815|9676|5|885| +2450815|9679|5|120| +2450815|9680|5|479| +2450815|9682|5|867| +2450815|9685|5|774| +2450815|9686|5|415| +2450815|9688|5|710| +2450815|9691|5|601| +2450815|9692|5|771| +2450815|9694|5|792| +2450815|9697|5|667| +2450815|9698|5|405| +2450815|9700|5|398| +2450815|9703|5|331| +2450815|9704|5|972| +2450815|9706|5|214| +2450815|9709|5|694| +2450815|9710|5|108| +2450815|9712|5|674| +2450815|9715|5|627| +2450815|9716|5|281| +2450815|9718|5|40| +2450815|9721|5|503| +2450815|9722|5|835| +2450815|9724|5|832| +2450815|9727|5|154| +2450815|9728|5|760| +2450815|9730|5|777| +2450815|9733|5|284| +2450815|9734|5|982| +2450815|9736|5|205| +2450815|9739|5|677| +2450815|9740|5|948| +2450815|9742|5|929| +2450815|9745|5|104| +2450815|9746|5|348| +2450815|9748|5|99| +2450815|9751|5|56| +2450815|9752|5|314| +2450815|9754|5|807| +2450815|9757|5|313| +2450815|9758|5|290| +2450815|9760|5|718| +2450815|9763|5|444| +2450815|9764|5|667| +2450815|9766|5|60| +2450815|9769|5|851| +2450815|9770|5|787| +2450815|9772|5|379| +2450815|9775|5|31| +2450815|9776|5|289| +2450815|9778|5|723| +2450815|9781|5|932| +2450815|9782|5|747| +2450815|9784|5|764| +2450815|9787|5|396| +2450815|9788|5|915| +2450815|9790|5|317| +2450815|9793|5|| +2450815|9794|5|489| +2450815|9796|5|174| +2450815|9799|5|237| +2450815|9800|5|227| +2450815|9802|5|290| +2450815|9805|5|972| +2450815|9806|5|631| +2450815|9808|5|669| +2450815|9811|5|248| +2450815|9812|5|249| +2450815|9814|5|979| +2450815|9817|5|115| +2450815|9818|5|331| +2450815|9820|5|267| +2450815|9823|5|261| +2450815|9824|5|220| +2450815|9826|5|954| +2450815|9829|5|675| +2450815|9830|5|| +2450815|9832|5|356| +2450815|9835|5|235| +2450815|9836|5|225| +2450815|9838|5|216| +2450815|9841|5|277| +2450815|9842|5|830| +2450815|9844|5|234| +2450815|9847|5|77| +2450815|9848|5|| +2450815|9850|5|884| +2450815|9853|5|66| +2450815|9854|5|116| +2450815|9856|5|358| +2450815|9859|5|660| +2450815|9860|5|248| +2450815|9862|5|751| +2450815|9865|5|547| +2450815|9866|5|540| +2450815|9868|5|705| +2450815|9871|5|566| +2450815|9872|5|340| +2450815|9874|5|211| +2450815|9877|5|1000| +2450815|9878|5|| +2450815|9880|5|689| +2450815|9883|5|156| +2450815|9884|5|763| +2450815|9886|5|315| +2450815|9889|5|220| +2450815|9890|5|486| +2450815|9892|5|| +2450815|9895|5|512| +2450815|9896|5|158| +2450815|9898|5|670| +2450815|9901|5|290| +2450815|9902|5|433| +2450815|9904|5|334| +2450815|9907|5|2| +2450815|9908|5|746| +2450815|9910|5|317| +2450815|9913|5|425| +2450815|9914|5|123| +2450815|9916|5|313| +2450815|9919|5|184| +2450815|9920|5|830| +2450815|9922|5|297| +2450815|9925|5|425| +2450815|9926|5|577| +2450815|9928|5|164| +2450815|9931|5|39| +2450815|9932|5|816| +2450815|9934|5|95| +2450815|9937|5|266| +2450815|9938|5|175| +2450815|9940|5|68| +2450815|9943|5|368| +2450815|9944|5|754| +2450815|9946|5|208| +2450815|9949|5|708| +2450815|9950|5|835| +2450815|9952|5|590| +2450815|9955|5|996| +2450815|9956|5|45| +2450815|9958|5|479| +2450815|9961|5|464| +2450815|9962|5|72| +2450815|9964|5|292| +2450815|9967|5|530| +2450815|9968|5|| +2450815|9970|5|568| +2450815|9973|5|882| +2450815|9974|5|956| +2450815|9976|5|308| +2450815|9979|5|315| +2450815|9980|5|264| +2450815|9982|5|266| +2450815|9985|5|| +2450815|9986|5|| +2450815|9988|5|879| +2450815|9991|5|581| +2450815|9992|5|309| +2450815|9994|5|491| +2450815|9997|5|566| +2450815|9998|5|364| +2450815|10000|5|208| +2450815|10003|5|36| +2450815|10004|5|640| +2450815|10006|5|90| +2450815|10009|5|901| +2450815|10010|5|46| +2450815|10012|5|756| +2450815|10015|5|417| +2450815|10016|5|| +2450815|10018|5|374| +2450815|10021|5|998| +2450815|10022|5|171| +2450815|10024|5|655| +2450815|10027|5|33| +2450815|10028|5|59| +2450815|10030|5|484| +2450815|10033|5|97| +2450815|10034|5|558| +2450815|10036|5|720| +2450815|10039|5|944| +2450815|10040|5|184| +2450815|10042|5|612| +2450815|10045|5|870| +2450815|10046|5|229| +2450815|10048|5|227| +2450815|10051|5|10| +2450815|10052|5|111| +2450815|10054|5|591| +2450815|10057|5|266| +2450815|10058|5|568| +2450815|10060|5|788| +2450815|10063|5|489| +2450815|10064|5|133| +2450815|10066|5|105| +2450815|10069|5|283| +2450815|10070|5|835| +2450815|10072|5|657| +2450815|10075|5|645| +2450815|10076|5|| +2450815|10078|5|602| +2450815|10081|5|286| +2450815|10082|5|36| +2450815|10084|5|194| +2450815|10087|5|552| +2450815|10088|5|442| +2450815|10090|5|649| +2450815|10093|5|627| +2450815|10094|5|545| +2450815|10096|5|685| +2450815|10099|5|288| +2450815|10100|5|| +2450815|10102|5|133| +2450815|10105|5|| +2450815|10106|5|893| +2450815|10108|5|508| +2450815|10111|5|532| +2450815|10112|5|| +2450815|10114|5|| +2450815|10117|5|567| +2450815|10118|5|548| +2450815|10120|5|807| +2450815|10123|5|724| +2450815|10124|5|618| +2450815|10126|5|264| +2450815|10129|5|940| +2450815|10130|5|951| +2450815|10132|5|445| +2450815|10135|5|905| +2450815|10136|5|265| +2450815|10138|5|414| +2450815|10141|5|168| +2450815|10142|5|414| +2450815|10144|5|663| +2450815|10147|5|60| +2450815|10148|5|696| +2450815|10150|5|911| +2450815|10153|5|274| +2450815|10154|5|423| +2450815|10156|5|916| +2450815|10159|5|153| +2450815|10160|5|102| +2450815|10162|5|876| +2450815|10165|5|750| +2450815|10166|5|283| +2450815|10168|5|906| +2450815|10171|5|605| +2450815|10172|5|540| +2450815|10174|5|56| +2450815|10177|5|940| +2450815|10178|5|| +2450815|10180|5|474| +2450815|10183|5|983| +2450815|10184|5|| +2450815|10186|5|529| +2450815|10189|5|171| +2450815|10190|5|524| +2450815|10192|5|456| +2450815|10195|5|343| +2450815|10196|5|677| +2450815|10198|5|740| +2450815|10201|5|381| +2450815|10202|5|| +2450815|10204|5|434| +2450815|10207|5|948| +2450815|10208|5|599| +2450815|10210|5|976| +2450815|10213|5|390| +2450815|10214|5|415| +2450815|10216|5|974| +2450815|10219|5|88| +2450815|10220|5|813| +2450815|10222|5|| +2450815|10225|5|373| +2450815|10226|5|436| +2450815|10228|5|317| +2450815|10231|5|534| +2450815|10232|5|783| +2450815|10234|5|892| +2450815|10237|5|499| +2450815|10238|5|413| +2450815|10240|5|513| +2450815|10243|5|100| +2450815|10244|5|35| +2450815|10246|5|441| +2450815|10249|5|121| +2450815|10250|5|838| +2450815|10252|5|681| +2450815|10255|5|925| +2450815|10256|5|229| +2450815|10258|5|| +2450815|10261|5|247| +2450815|10262|5|647| +2450815|10264|5|723| +2450815|10267|5|837| +2450815|10268|5|316| +2450815|10270|5|139| +2450815|10273|5|62| +2450815|10274|5|265| +2450815|10276|5|422| +2450815|10279|5|415| +2450815|10280|5|696| +2450815|10282|5|| +2450815|10285|5|638| +2450815|10286|5|219| +2450815|10288|5|126| +2450815|10291|5|355| +2450815|10292|5|797| +2450815|10294|5|133| +2450815|10297|5|478| +2450815|10298|5|208| +2450815|10300|5|734| +2450815|10303|5|153| +2450815|10304|5|527| +2450815|10306|5|411| +2450815|10309|5|953| +2450815|10310|5|368| +2450815|10312|5|705| +2450815|10315|5|567| +2450815|10316|5|581| +2450815|10318|5|958| +2450815|10321|5|233| +2450815|10322|5|816| +2450815|10324|5|543| +2450815|10327|5|466| +2450815|10328|5|336| +2450815|10330|5|579| +2450815|10333|5|485| +2450815|10334|5|282| +2450815|10336|5|776| +2450815|10339|5|582| +2450815|10340|5|575| +2450815|10342|5|983| +2450815|10345|5|616| +2450815|10346|5|315| +2450815|10348|5|635| +2450815|10351|5|173| +2450815|10352|5|255| +2450815|10354|5|659| +2450815|10357|5|951| +2450815|10358|5|433| +2450815|10360|5|792| +2450815|10363|5|904| +2450815|10364|5|| +2450815|10366|5|650| +2450815|10369|5|592| +2450815|10370|5|635| +2450815|10372|5|705| +2450815|10375|5|493| +2450815|10376|5|958| +2450815|10378|5|916| +2450815|10381|5|236| +2450815|10382|5|523| +2450815|10384|5|959| +2450815|10387|5|963| +2450815|10388|5|734| +2450815|10390|5|995| +2450815|10393|5|707| +2450815|10394|5|900| +2450815|10396|5|829| +2450815|10399|5|987| +2450815|10400|5|283| +2450815|10402|5|453| +2450815|10405|5|| +2450815|10406|5|274| +2450815|10408|5|352| +2450815|10411|5|| +2450815|10412|5|568| +2450815|10414|5|| +2450815|10417|5|13| +2450815|10418|5|242| +2450815|10420|5|181| +2450815|10423|5|934| +2450815|10424|5|378| +2450815|10426|5|575| +2450815|10429|5|522| +2450815|10430|5|725| +2450815|10432|5|| +2450815|10435|5|580| +2450815|10436|5|346| +2450815|10438|5|139| +2450815|10441|5|327| +2450815|10442|5|724| +2450815|10444|5|715| +2450815|10447|5|918| +2450815|10448|5|736| +2450815|10450|5|758| +2450815|10453|5|946| +2450815|10454|5|371| +2450815|10456|5|675| +2450815|10459|5|952| +2450815|10460|5|90| +2450815|10462|5|868| +2450815|10465|5|874| +2450815|10466|5|| +2450815|10468|5|132| +2450815|10471|5|503| +2450815|10472|5|845| +2450815|10474|5|84| +2450815|10477|5|535| +2450815|10478|5|387| +2450815|10480|5|886| +2450815|10483|5|613| +2450815|10484|5|600| +2450815|10486|5|206| +2450815|10489|5|625| +2450815|10490|5|626| +2450815|10492|5|10| +2450815|10495|5|561| +2450815|10496|5|852| +2450815|10498|5|241| +2450815|10501|5|327| +2450815|10502|5|891| +2450815|10504|5|628| +2450815|10507|5|91| +2450815|10508|5|195| +2450815|10510|5|699| +2450815|10513|5|199| +2450815|10514|5|743| +2450815|10516|5|789| +2450815|10519|5|806| +2450815|10520|5|906| +2450815|10522|5|80| +2450815|10525|5|13| +2450815|10526|5|220| +2450815|10528|5|800| +2450815|10531|5|645| +2450815|10532|5|540| +2450815|10534|5|539| +2450815|10537|5|647| +2450815|10538|5|950| +2450815|10540|5|621| +2450815|10543|5|33| +2450815|10544|5|42| +2450815|10546|5|222| +2450815|10549|5|833| +2450815|10550|5|596| +2450815|10552|5|593| +2450815|10555|5|279| +2450815|10556|5|635| +2450815|10558|5|0| +2450815|10561|5|390| +2450815|10562|5|247| +2450815|10564|5|199| +2450815|10567|5|937| +2450815|10568|5|981| +2450815|10570|5|650| +2450815|10573|5|72| +2450815|10574|5|608| +2450815|10576|5|571| +2450815|10579|5|645| +2450815|10580|5|492| +2450815|10582|5|703| +2450815|10585|5|905| +2450815|10586|5|981| +2450815|10588|5|558| +2450815|10591|5|507| +2450815|10592|5|793| +2450815|10594|5|571| +2450815|10597|5|70| +2450815|10598|5|54| +2450815|10600|5|112| +2450815|10603|5|602| +2450815|10604|5|760| +2450815|10606|5|725| +2450815|10609|5|264| +2450815|10610|5|468| +2450815|10612|5|418| +2450815|10615|5|339| +2450815|10616|5|17| +2450815|10618|5|114| +2450815|10621|5|225| +2450815|10622|5|872| +2450815|10624|5|5| +2450815|10627|5|490| +2450815|10628|5|720| +2450815|10630|5|822| +2450815|10633|5|892| +2450815|10634|5|631| +2450815|10636|5|345| +2450815|10639|5|787| +2450815|10640|5|505| +2450815|10642|5|76| +2450815|10645|5|226| +2450815|10646|5|657| +2450815|10648|5|831| +2450815|10651|5|618| +2450815|10652|5|445| +2450815|10654|5|148| +2450815|10657|5|674| +2450815|10658|5|332| +2450815|10660|5|796| +2450815|10663|5|552| +2450815|10664|5|635| +2450815|10666|5|91| +2450815|10669|5|924| +2450815|10670|5|766| +2450815|10672|5|721| +2450815|10675|5|352| +2450815|10676|5|918| +2450815|10678|5|990| +2450815|10681|5|545| +2450815|10682|5|782| +2450815|10684|5|433| +2450815|10687|5|48| +2450815|10688|5|527| +2450815|10690|5|20| +2450815|10693|5|691| +2450815|10694|5|926| +2450815|10696|5|856| +2450815|10699|5|210| +2450815|10700|5|435| +2450815|10702|5|287| +2450815|10705|5|297| +2450815|10706|5|799| +2450815|10708|5|898| +2450815|10711|5|111| +2450815|10712|5|151| +2450815|10714|5|81| +2450815|10717|5|797| +2450815|10718|5|7| +2450815|10720|5|274| +2450815|10723|5|546| +2450815|10724|5|145| +2450815|10726|5|42| +2450815|10729|5|418| +2450815|10730|5|722| +2450815|10732|5|77| +2450815|10735|5|| +2450815|10736|5|58| +2450815|10738|5|83| +2450815|10741|5|168| +2450815|10742|5|639| +2450815|10744|5|90| +2450815|10747|5|986| +2450815|10748|5|608| +2450815|10750|5|752| +2450815|10753|5|594| +2450815|10754|5|785| +2450815|10756|5|829| +2450815|10759|5|354| +2450815|10760|5|914| +2450815|10762|5|41| +2450815|10765|5|589| +2450815|10766|5|| +2450815|10768|5|1| +2450815|10771|5|73| +2450815|10772|5|127| +2450815|10774|5|948| +2450815|10777|5|25| +2450815|10778|5|985| +2450815|10780|5|322| +2450815|10783|5|4| +2450815|10784|5|913| +2450815|10786|5|237| +2450815|10789|5|89| +2450815|10790|5|847| +2450815|10792|5|457| +2450815|10795|5|382| +2450815|10796|5|735| +2450815|10798|5|590| +2450815|10801|5|45| +2450815|10802|5|424| +2450815|10804|5|751| +2450815|10807|5|582| +2450815|10808|5|252| +2450815|10810|5|186| +2450815|10813|5|947| +2450815|10814|5|429| +2450815|10816|5|657| +2450815|10819|5|589| +2450815|10820|5|127| +2450815|10822|5|523| +2450815|10825|5|350| +2450815|10826|5|| +2450815|10828|5|305| +2450815|10831|5|635| +2450815|10832|5|898| +2450815|10834|5|124| +2450815|10837|5|110| +2450815|10838|5|839| +2450815|10840|5|275| +2450815|10843|5|15| +2450815|10844|5|909| +2450815|10846|5|496| +2450815|10849|5|686| +2450815|10850|5|76| +2450815|10852|5|215| +2450815|10855|5|483| +2450815|10856|5|555| +2450815|10858|5|703| +2450815|10861|5|395| +2450815|10862|5|223| +2450815|10864|5|159| +2450815|10867|5|938| +2450815|10868|5|894| +2450815|10870|5|815| +2450815|10873|5|357| +2450815|10874|5|318| +2450815|10876|5|345| +2450815|10879|5|577| +2450815|10880|5|425| +2450815|10882|5|95| +2450815|10885|5|370| +2450815|10886|5|696| +2450815|10888|5|394| +2450815|10891|5|605| +2450815|10892|5|842| +2450815|10894|5|460| +2450815|10897|5|865| +2450815|10898|5|44| +2450815|10900|5|415| +2450815|10903|5|| +2450815|10904|5|917| +2450815|10906|5|535| +2450815|10909|5|43| +2450815|10910|5|453| +2450815|10912|5|261| +2450815|10915|5|302| +2450815|10916|5|749| +2450815|10918|5|277| +2450815|10921|5|364| +2450815|10922|5|299| +2450815|10924|5|336| +2450815|10927|5|| +2450815|10928|5|280| +2450815|10930|5|487| +2450815|10933|5|920| +2450815|10934|5|273| +2450815|10936|5|418| +2450815|10939|5|| +2450815|10940|5|725| +2450815|10942|5|949| +2450815|10945|5|218| +2450815|10946|5|392| +2450815|10948|5|877| +2450815|10951|5|464| +2450815|10952|5|87| +2450815|10954|5|717| +2450815|10957|5|853| +2450815|10958|5|166| +2450815|10960|5|359| +2450815|10963|5|425| +2450815|10964|5|779| +2450815|10966|5|| +2450815|10969|5|453| +2450815|10970|5|| +2450815|10972|5|306| +2450815|10975|5|635| +2450815|10976|5|99| +2450815|10978|5|339| +2450815|10981|5|82| +2450815|10982|5|962| +2450815|10984|5|626| +2450815|10987|5|784| +2450815|10988|5|487| +2450815|10990|5|45| +2450815|10993|5|626| +2450815|10994|5|806| +2450815|10996|5|656| +2450815|10999|5|741| +2450815|11000|5|841| +2450815|11002|5|869| +2450815|11005|5|232| +2450815|11006|5|538| +2450815|11008|5|64| +2450815|11011|5|565| +2450815|11012|5|523| +2450815|11014|5|556| +2450815|11017|5|624| +2450815|11018|5|317| +2450815|11020|5|230| +2450815|11023|5|427| +2450815|11024|5|42| +2450815|11026|5|194| +2450815|11029|5|769| +2450815|11030|5|| +2450815|11032|5|162| +2450815|11035|5|551| +2450815|11036|5|125| +2450815|11038|5|9| +2450815|11041|5|155| +2450815|11042|5|946| +2450815|11044|5|718| +2450815|11047|5|418| +2450815|11048|5|737| +2450815|11050|5|292| +2450815|11053|5|930| +2450815|11054|5|115| +2450815|11056|5|158| +2450815|11059|5|993| +2450815|11060|5|155| +2450815|11062|5|188| +2450815|11065|5|127| +2450815|11066|5|19| +2450815|11068|5|421| +2450815|11071|5|819| +2450815|11072|5|896| +2450815|11074|5|294| +2450815|11077|5|984| +2450815|11078|5|133| +2450815|11080|5|941| +2450815|11083|5|877| +2450815|11084|5|194| +2450815|11086|5|847| +2450815|11089|5|865| +2450815|11090|5|963| +2450815|11092|5|957| +2450815|11095|5|435| +2450815|11096|5|624| +2450815|11098|5|35| +2450815|11101|5|570| +2450815|11102|5|233| +2450815|11104|5|| +2450815|11107|5|261| +2450815|11108|5|66| +2450815|11110|5|341| +2450815|11113|5|450| +2450815|11114|5|181| +2450815|11116|5|125| +2450815|11119|5|57| +2450815|11120|5|255| +2450815|11122|5|932| +2450815|11125|5|| +2450815|11126|5|139| +2450815|11128|5|27| +2450815|11131|5|248| +2450815|11132|5|675| +2450815|11134|5|293| +2450815|11137|5|853| +2450815|11138|5|| +2450815|11140|5|358| +2450815|11143|5|640| +2450815|11144|5|929| +2450815|11146|5|986| +2450815|11149|5|681| +2450815|11150|5|286| +2450815|11152|5|901| +2450815|11155|5|833| +2450815|11156|5|109| +2450815|11158|5|142| +2450815|11161|5|648| +2450815|11162|5|31| +2450815|11164|5|285| +2450815|11167|5|850| +2450815|11168|5|827| +2450815|11170|5|780| +2450815|11173|5|38| +2450815|11174|5|| +2450815|11176|5|252| +2450815|11179|5|671| +2450815|11180|5|986| +2450815|11182|5|431| +2450815|11185|5|908| +2450815|11186|5|372| +2450815|11188|5|882| +2450815|11191|5|687| +2450815|11192|5|988| +2450815|11194|5|90| +2450815|11197|5|857| +2450815|11198|5|601| +2450815|11200|5|236| +2450815|11203|5|393| +2450815|11204|5|784| +2450815|11206|5|95| +2450815|11209|5|301| +2450815|11210|5|377| +2450815|11212|5|7| +2450815|11215|5|95| +2450815|11216|5|779| +2450815|11218|5|76| +2450815|11221|5|665| +2450815|11222|5|13| +2450815|11224|5|893| +2450815|11227|5|203| +2450815|11228|5|848| +2450815|11230|5|800| +2450815|11233|5|| +2450815|11234|5|42| +2450815|11236|5|208| +2450815|11239|5|| +2450815|11240|5|158| +2450815|11242|5|| +2450815|11245|5|116| +2450815|11246|5|615| +2450815|11248|5|492| +2450815|11251|5|| +2450815|11252|5|196| +2450815|11254|5|9| +2450815|11257|5|291| +2450815|11258|5|180| +2450815|11260|5|84| +2450815|11263|5|346| +2450815|11264|5|10| +2450815|11266|5|288| +2450815|11269|5|| +2450815|11270|5|403| +2450815|11272|5|447| +2450815|11275|5|998| +2450815|11276|5|209| +2450815|11278|5|287| +2450815|11281|5|122| +2450815|11282|5|913| +2450815|11284|5|| +2450815|11287|5|498| +2450815|11288|5|72| +2450815|11290|5|453| +2450815|11293|5|883| +2450815|11294|5|608| +2450815|11296|5|243| +2450815|11299|5|163| +2450815|11300|5|352| +2450815|11302|5|497| +2450815|11305|5|66| +2450815|11306|5|983| +2450815|11308|5|699| +2450815|11311|5|32| +2450815|11312|5|42| +2450815|11314|5|553| +2450815|11317|5|808| +2450815|11318|5|492| +2450815|11320|5|68| +2450815|11323|5|147| +2450815|11324|5|154| +2450815|11326|5|228| +2450815|11329|5|611| +2450815|11330|5|82| +2450815|11332|5|891| +2450815|11335|5|828| +2450815|11336|5|364| +2450815|11338|5|409| +2450815|11341|5|986| +2450815|11342|5|649| +2450815|11344|5|716| +2450815|11347|5|647| +2450815|11348|5|988| +2450815|11350|5|9| +2450815|11353|5|887| +2450815|11354|5|544| +2450815|11356|5|153| +2450815|11359|5|213| +2450815|11360|5|377| +2450815|11362|5|457| +2450815|11365|5|496| +2450815|11366|5|633| +2450815|11368|5|636| +2450815|11371|5|33| +2450815|11372|5|374| +2450815|11374|5|326| +2450815|11377|5|114| +2450815|11378|5|0| +2450815|11380|5|930| +2450815|11383|5|284| +2450815|11384|5|876| +2450815|11386|5|613| +2450815|11389|5|456| +2450815|11390|5|444| +2450815|11392|5|703| +2450815|11395|5|627| +2450815|11396|5|657| +2450815|11398|5|54| +2450815|11401|5|572| +2450815|11402|5|613| +2450815|11404|5|232| +2450815|11407|5|373| +2450815|11408|5|269| +2450815|11410|5|294| +2450815|11413|5|256| +2450815|11414|5|878| +2450815|11416|5|774| +2450815|11419|5|502| +2450815|11420|5|| +2450815|11422|5|471| +2450815|11425|5|913| +2450815|11426|5|678| +2450815|11428|5|856| +2450815|11431|5|258| +2450815|11432|5|107| +2450815|11434|5|341| +2450815|11437|5|840| +2450815|11438|5|269| +2450815|11440|5|949| +2450815|11443|5|866| +2450815|11444|5|909| +2450815|11446|5|537| +2450815|11449|5|777| +2450815|11450|5|775| +2450815|11452|5|389| +2450815|11455|5|583| +2450815|11456|5|654| +2450815|11458|5|553| +2450815|11461|5|713| +2450815|11462|5|258| +2450815|11464|5|363| +2450815|11467|5|256| +2450815|11468|5|660| +2450815|11470|5|245| +2450815|11473|5|| +2450815|11474|5|| +2450815|11476|5|793| +2450815|11479|5|321| +2450815|11480|5|240| +2450815|11482|5|800| +2450815|11485|5|168| +2450815|11486|5|492| +2450815|11488|5|696| +2450815|11491|5|935| +2450815|11492|5|222| +2450815|11494|5|476| +2450815|11497|5|315| +2450815|11498|5|802| +2450815|11500|5|72| +2450815|11503|5|24| +2450815|11504|5|255| +2450815|11506|5|994| +2450815|11509|5|620| +2450815|11510|5|221| +2450815|11512|5|487| +2450815|11515|5|250| +2450815|11516|5|665| +2450815|11518|5|612| +2450815|11521|5|664| +2450815|11522|5|464| +2450815|11524|5|422| +2450815|11527|5|866| +2450815|11528|5|843| +2450815|11530|5|873| +2450815|11533|5|876| +2450815|11534|5|602| +2450815|11536|5|348| +2450815|11539|5|678| +2450815|11540|5|| +2450815|11542|5|15| +2450815|11545|5|304| +2450815|11546|5|622| +2450815|11548|5|983| +2450815|11551|5|386| +2450815|11552|5|257| +2450815|11554|5|831| +2450815|11557|5|511| +2450815|11558|5|174| +2450815|11560|5|773| +2450815|11563|5|377| +2450815|11564|5|926| +2450815|11566|5|734| +2450815|11569|5|180| +2450815|11570|5|| +2450815|11572|5|579| +2450815|11575|5|731| +2450815|11576|5|215| +2450815|11578|5|770| +2450815|11581|5|457| +2450815|11582|5|528| +2450815|11584|5|941| +2450815|11587|5|189| +2450815|11588|5|846| +2450815|11590|5|406| +2450815|11593|5|167| +2450815|11594|5|807| +2450815|11596|5|990| +2450815|11599|5|453| +2450815|11600|5|81| +2450815|11602|5|698| +2450815|11605|5|108| +2450815|11606|5|109| +2450815|11608|5|586| +2450815|11611|5|67| +2450815|11612|5|75| +2450815|11614|5|501| +2450815|11617|5|629| +2450815|11618|5|| +2450815|11620|5|653| +2450815|11623|5|466| +2450815|11624|5|776| +2450815|11626|5|445| +2450815|11629|5|288| +2450815|11630|5|593| +2450815|11632|5|930| +2450815|11635|5|871| +2450815|11636|5|909| +2450815|11638|5|467| +2450815|11641|5|771| +2450815|11642|5|788| +2450815|11644|5|432| +2450815|11647|5|265| +2450815|11648|5|572| +2450815|11650|5|166| +2450815|11653|5|926| +2450815|11654|5|702| +2450815|11656|5|808| +2450815|11659|5|369| +2450815|11660|5|858| +2450815|11662|5|169| +2450815|11665|5|126| +2450815|11666|5|20| +2450815|11668|5|| +2450815|11671|5|562| +2450815|11672|5|665| +2450815|11674|5|606| +2450815|11677|5|289| +2450815|11678|5|837| +2450815|11680|5|327| +2450815|11683|5|261| +2450815|11684|5|143| +2450815|11686|5|668| +2450815|11689|5|276| +2450815|11690|5|538| +2450815|11692|5|103| +2450815|11695|5|907| +2450815|11696|5|511| +2450815|11698|5|| +2450815|11701|5|776| +2450815|11702|5|756| +2450815|11704|5|7| +2450815|11707|5|736| +2450815|11708|5|25| +2450815|11710|5|618| +2450815|11713|5|716| +2450815|11714|5|768| +2450815|11716|5|559| +2450815|11719|5|56| +2450815|11720|5|403| +2450815|11722|5|750| +2450815|11725|5|400| +2450815|11726|5|451| +2450815|11728|5|574| +2450815|11731|5|984| +2450815|11732|5|340| +2450815|11734|5|| +2450815|11737|5|907| +2450815|11738|5|968| +2450815|11740|5|352| +2450815|11743|5|929| +2450815|11744|5|885| +2450815|11746|5|15| +2450815|11749|5|8| +2450815|11750|5|97| +2450815|11752|5|139| +2450815|11755|5|593| +2450815|11756|5|765| +2450815|11758|5|894| +2450815|11761|5|65| +2450815|11762|5|84| +2450815|11764|5|39| +2450815|11767|5|656| +2450815|11768|5|658| +2450815|11770|5|907| +2450815|11773|5|379| +2450815|11774|5|523| +2450815|11776|5|409| +2450815|11779|5|427| +2450815|11780|5|22| +2450815|11782|5|927| +2450815|11785|5|895| +2450815|11786|5|107| +2450815|11788|5|786| +2450815|11791|5|191| +2450815|11792|5|769| +2450815|11794|5|825| +2450815|11797|5|58| +2450815|11798|5|148| +2450815|11800|5|699| +2450815|11803|5|862| +2450815|11804|5|771| +2450815|11806|5|864| +2450815|11809|5|823| +2450815|11810|5|402| +2450815|11812|5|263| +2450815|11815|5|100| +2450815|11816|5|614| +2450815|11818|5|165| +2450815|11821|5|152| +2450815|11822|5|439| +2450815|11824|5|148| +2450815|11827|5|425| +2450815|11828|5|272| +2450815|11830|5|213| +2450815|11833|5|262| +2450815|11834|5|873| +2450815|11836|5|978| +2450815|11839|5|410| +2450815|11840|5|763| +2450815|11842|5|744| +2450815|11845|5|532| +2450815|11846|5|141| +2450815|11848|5|492| +2450815|11851|5|519| +2450815|11852|5|819| +2450815|11854|5|742| +2450815|11857|5|325| +2450815|11858|5|863| +2450815|11860|5|220| +2450815|11863|5|177| +2450815|11864|5|| +2450815|11866|5|770| +2450815|11869|5|484| +2450815|11870|5|78| +2450815|11872|5|694| +2450815|11875|5|754| +2450815|11876|5|525| +2450815|11878|5|921| +2450815|11881|5|384| +2450815|11882|5|536| +2450815|11884|5|627| +2450815|11887|5|186| +2450815|11888|5|527| +2450815|11890|5|785| +2450815|11893|5|956| +2450815|11894|5|566| +2450815|11896|5|164| +2450815|11899|5|131| +2450815|11900|5|428| +2450815|11902|5|153| +2450815|11905|5|833| +2450815|11906|5|848| +2450815|11908|5|342| +2450815|11911|5|584| +2450815|11912|5|347| +2450815|11914|5|461| +2450815|11917|5|411| +2450815|11918|5|24| +2450815|11920|5|377| +2450815|11923|5|751| +2450815|11924|5|274| +2450815|11926|5|215| +2450815|11929|5|870| +2450815|11930|5|998| +2450815|11932|5|111| +2450815|11935|5|944| +2450815|11936|5|131| +2450815|11938|5|| +2450815|11941|5|345| +2450815|11942|5|58| +2450815|11944|5|897| +2450815|11947|5|546| +2450815|11948|5|317| +2450815|11950|5|495| +2450815|11953|5|970| +2450815|11954|5|808| +2450815|11956|5|627| +2450815|11959|5|80| +2450815|11960|5|314| +2450815|11962|5|285| +2450815|11965|5|340| +2450815|11966|5|840| +2450815|11968|5|205| +2450815|11971|5|142| +2450815|11972|5|277| +2450815|11974|5|909| +2450815|11977|5|21| +2450815|11978|5|440| +2450815|11980|5|272| +2450815|11983|5|106| +2450815|11984|5|687| +2450815|11986|5|465| +2450815|11989|5|850| +2450815|11990|5|143| +2450815|11992|5|259| +2450815|11995|5|74| +2450815|11996|5|926| +2450815|11998|5|581| +2450815|12001|5|151| +2450815|12002|5|503| +2450815|12004|5|| +2450815|12007|5|923| +2450815|12008|5|369| +2450815|12010|5|447| +2450815|12013|5|785| +2450815|12014|5|369| +2450815|12016|5|489| +2450815|12019|5|865| +2450815|12020|5|42| +2450815|12022|5|359| +2450815|12025|5|772| +2450815|12026|5|578| +2450815|12028|5|367| +2450815|12031|5|794| +2450815|12032|5|885| +2450815|12034|5|185| +2450815|12037|5|526| +2450815|12038|5|228| +2450815|12040|5|966| +2450815|12043|5|601| +2450815|12044|5|386| +2450815|12046|5|324| +2450815|12049|5|460| +2450815|12050|5|411| +2450815|12052|5|244| +2450815|12055|5|365| +2450815|12056|5|725| +2450815|12058|5|727| +2450815|12061|5|999| +2450815|12062|5|215| +2450815|12064|5|565| +2450815|12067|5|333| +2450815|12068|5|184| +2450815|12070|5|620| +2450815|12073|5|603| +2450815|12074|5|268| +2450815|12076|5|162| +2450815|12079|5|781| +2450815|12080|5|964| +2450815|12082|5|723| +2450815|12085|5|433| +2450815|12086|5|707| +2450815|12088|5|706| +2450815|12091|5|807| +2450815|12092|5|664| +2450815|12094|5|39| +2450815|12097|5|| +2450815|12098|5|925| +2450815|12100|5|25| +2450815|12103|5|193| +2450815|12104|5|761| +2450815|12106|5|727| +2450815|12109|5|985| +2450815|12110|5|939| +2450815|12112|5|566| +2450815|12115|5|532| +2450815|12116|5|705| +2450815|12118|5|71| +2450815|12121|5|682| +2450815|12122|5|659| +2450815|12124|5|663| +2450815|12127|5|285| +2450815|12128|5|6| +2450815|12130|5|872| +2450815|12133|5|23| +2450815|12134|5|930| +2450815|12136|5|119| +2450815|12139|5|872| +2450815|12140|5|839| +2450815|12142|5|827| +2450815|12145|5|432| +2450815|12146|5|837| +2450815|12148|5|239| +2450815|12151|5|981| +2450815|12152|5|580| +2450815|12154|5|592| +2450815|12157|5|601| +2450815|12158|5|896| +2450815|12160|5|284| +2450815|12163|5|791| +2450815|12164|5|222| +2450815|12166|5|103| +2450815|12169|5|196| +2450815|12170|5|407| +2450815|12172|5|457| +2450815|12175|5|308| +2450815|12176|5|75| +2450815|12178|5|174| +2450815|12181|5|546| +2450815|12182|5|506| +2450815|12184|5|35| +2450815|12187|5|537| +2450815|12188|5|5| +2450815|12190|5|520| +2450815|12193|5|830| +2450815|12194|5|217| +2450815|12196|5|763| +2450815|12199|5|| +2450815|12200|5|716| +2450815|12202|5|471| +2450815|12205|5|724| +2450815|12206|5|309| +2450815|12208|5|864| +2450815|12211|5|145| +2450815|12212|5|358| +2450815|12214|5|547| +2450815|12217|5|976| +2450815|12218|5|| +2450815|12220|5|156| +2450815|12223|5|| +2450815|12224|5|941| +2450815|12226|5|701| +2450815|12229|5|54| +2450815|12230|5|497| +2450815|12232|5|495| +2450815|12235|5|987| +2450815|12236|5|415| +2450815|12238|5|94| +2450815|12241|5|65| +2450815|12242|5|410| +2450815|12244|5|649| +2450815|12247|5|168| +2450815|12248|5|273| +2450815|12250|5|977| +2450815|12253|5|38| +2450815|12254|5|591| +2450815|12256|5|235| +2450815|12259|5|18| +2450815|12260|5|191| +2450815|12262|5|96| +2450815|12265|5|823| +2450815|12266|5|348| +2450815|12268|5|687| +2450815|12271|5|468| +2450815|12272|5|893| +2450815|12274|5|956| +2450815|12277|5|| +2450815|12278|5|985| +2450815|12280|5|320| +2450815|12283|5|667| +2450815|12284|5|238| +2450815|12286|5|| +2450815|12289|5|98| +2450815|12290|5|508| +2450815|12292|5|| +2450815|12295|5|148| +2450815|12296|5|391| +2450815|12298|5|530| +2450815|12301|5|999| +2450815|12302|5|988| +2450815|12304|5|159| +2450815|12307|5|545| +2450815|12308|5|430| +2450815|12310|5|875| +2450815|12313|5|| +2450815|12314|5|| +2450815|12316|5|742| +2450815|12319|5|832| +2450815|12320|5|| +2450815|12322|5|| +2450815|12325|5|129| +2450815|12326|5|156| +2450815|12328|5|610| +2450815|12331|5|755| +2450815|12332|5|737| +2450815|12334|5|368| +2450815|12337|5|428| +2450815|12338|5|241| +2450815|12340|5|852| +2450815|12343|5|106| +2450815|12344|5|106| +2450815|12346|5|380| +2450815|12349|5|702| +2450815|12350|5|942| +2450815|12352|5|432| +2450815|12355|5|698| +2450815|12356|5|289| +2450815|12358|5|862| +2450815|12361|5|268| +2450815|12362|5|751| +2450815|12364|5|679| +2450815|12367|5|522| +2450815|12368|5|703| +2450815|12370|5|380| +2450815|12373|5|| +2450815|12374|5|804| +2450815|12376|5|895| +2450815|12379|5|655| +2450815|12380|5|| +2450815|12382|5|648| +2450815|12385|5|639| +2450815|12386|5|410| +2450815|12388|5|373| +2450815|12391|5|509| +2450815|12392|5|199| +2450815|12394|5|476| +2450815|12397|5|486| +2450815|12398|5|488| +2450815|12400|5|307| +2450815|12403|5|936| +2450815|12404|5|278| +2450815|12406|5|852| +2450815|12409|5|571| +2450815|12410|5|79| +2450815|12412|5|156| +2450815|12415|5|523| +2450815|12416|5|607| +2450815|12418|5|915| +2450815|12421|5|9| +2450815|12422|5|42| +2450815|12424|5|977| +2450815|12427|5|709| +2450815|12428|5|77| +2450815|12430|5|561| +2450815|12433|5|748| +2450815|12434|5|632| +2450815|12436|5|184| +2450815|12439|5|156| +2450815|12440|5|808| +2450815|12442|5|494| +2450815|12445|5|511| +2450815|12446|5|845| +2450815|12448|5|708| +2450815|12451|5|| +2450815|12452|5|200| +2450815|12454|5|911| +2450815|12457|5|746| +2450815|12458|5|831| +2450815|12460|5|256| +2450815|12463|5|135| +2450815|12464|5|682| +2450815|12466|5|61| +2450815|12469|5|473| +2450815|12470|5|447| +2450815|12472|5|907| +2450815|12475|5|269| +2450815|12476|5|414| +2450815|12478|5|22| +2450815|12481|5|434| +2450815|12482|5|422| +2450815|12484|5|837| +2450815|12487|5|120| +2450815|12488|5|584| +2450815|12490|5|663| +2450815|12493|5|127| +2450815|12494|5|47| +2450815|12496|5|| +2450815|12499|5|68| +2450815|12500|5|436| +2450815|12502|5|556| +2450815|12505|5|962| +2450815|12506|5|437| +2450815|12508|5|453| +2450815|12511|5|919| +2450815|12512|5|67| +2450815|12514|5|540| +2450815|12517|5|389| +2450815|12518|5|843| +2450815|12520|5|919| +2450815|12523|5|585| +2450815|12524|5|9| +2450815|12526|5|312| +2450815|12529|5|725| +2450815|12530|5|716| +2450815|12532|5|478| +2450815|12535|5|667| +2450815|12536|5|242| +2450815|12538|5|659| +2450815|12541|5|943| +2450815|12542|5|| +2450815|12544|5|613| +2450815|12547|5|789| +2450815|12548|5|78| +2450815|12550|5|256| +2450815|12553|5|202| +2450815|12554|5|539| +2450815|12556|5|935| +2450815|12559|5|967| +2450815|12560|5|229| +2450815|12562|5|607| +2450815|12565|5|529| +2450815|12566|5|789| +2450815|12568|5|205| +2450815|12571|5|67| +2450815|12572|5|89| +2450815|12574|5|820| +2450815|12577|5|993| +2450815|12578|5|838| +2450815|12580|5|731| +2450815|12583|5|156| +2450815|12584|5|539| +2450815|12586|5|262| +2450815|12589|5|51| +2450815|12590|5|| +2450815|12592|5|700| +2450815|12595|5|599| +2450815|12596|5|750| +2450815|12598|5|930| +2450815|12601|5|980| +2450815|12602|5|768| +2450815|12604|5|101| +2450815|12607|5|190| +2450815|12608|5|679| +2450815|12610|5|934| +2450815|12613|5|933| +2450815|12614|5|553| +2450815|12616|5|910| +2450815|12619|5|662| +2450815|12620|5|906| +2450815|12622|5|383| +2450815|12625|5|781| +2450815|12626|5|330| +2450815|12628|5|919| +2450815|12631|5|780| +2450815|12632|5|391| +2450815|12634|5|827| +2450815|12637|5|603| +2450815|12638|5|205| +2450815|12640|5|33| +2450815|12643|5|548| +2450815|12644|5|902| +2450815|12646|5|287| +2450815|12649|5|315| +2450815|12650|5|254| +2450815|12652|5|508| +2450815|12655|5|85| +2450815|12656|5|994| +2450815|12658|5|516| +2450815|12661|5|410| +2450815|12662|5|978| +2450815|12664|5|562| +2450815|12667|5|381| +2450815|12668|5|132| +2450815|12670|5|564| +2450815|12673|5|785| +2450815|12674|5|463| +2450815|12676|5|169| +2450815|12679|5|31| +2450815|12680|5|304| +2450815|12682|5|849| +2450815|12685|5|37| +2450815|12686|5|270| +2450815|12688|5|67| +2450815|12691|5|814| +2450815|12692|5|833| +2450815|12694|5|386| +2450815|12697|5|137| +2450815|12698|5|965| +2450815|12700|5|16| +2450815|12703|5|279| +2450815|12704|5|372| +2450815|12706|5|5| +2450815|12709|5|911| +2450815|12710|5|166| +2450815|12712|5|4| +2450815|12715|5|110| +2450815|12716|5|722| +2450815|12718|5|498| +2450815|12721|5|691| +2450815|12722|5|295| +2450815|12724|5|614| +2450815|12727|5|352| +2450815|12728|5|912| +2450815|12730|5|83| +2450815|12733|5|920| +2450815|12734|5|987| +2450815|12736|5|385| +2450815|12739|5|781| +2450815|12740|5|204| +2450815|12742|5|38| +2450815|12745|5|522| +2450815|12746|5|266| +2450815|12748|5|494| +2450815|12751|5|182| +2450815|12752|5|142| +2450815|12754|5|697| +2450815|12757|5|129| +2450815|12758|5|| +2450815|12760|5|498| +2450815|12763|5|458| +2450815|12764|5|707| +2450815|12766|5|1000| +2450815|12769|5|830| +2450815|12770|5|845| +2450815|12772|5|495| +2450815|12775|5|4| +2450815|12776|5|85| +2450815|12778|5|864| +2450815|12781|5|213| +2450815|12782|5|663| +2450815|12784|5|625| +2450815|12787|5|744| +2450815|12788|5|74| +2450815|12790|5|122| +2450815|12793|5|795| +2450815|12794|5|597| +2450815|12796|5|463| +2450815|12799|5|221| +2450815|12800|5|291| +2450815|12802|5|484| +2450815|12805|5|264| +2450815|12806|5|359| +2450815|12808|5|267| +2450815|12811|5|880| +2450815|12812|5|947| +2450815|12814|5|196| +2450815|12817|5|69| +2450815|12818|5|943| +2450815|12820|5|559| +2450815|12823|5|492| +2450815|12824|5|647| +2450815|12826|5|154| +2450815|12829|5|207| +2450815|12830|5|873| +2450815|12832|5|284| +2450815|12835|5|684| +2450815|12836|5|526| +2450815|12838|5|20| +2450815|12841|5|| +2450815|12842|5|900| +2450815|12844|5|851| +2450815|12847|5|545| +2450815|12848|5|318| +2450815|12850|5|156| +2450815|12853|5|0| +2450815|12854|5|894| +2450815|12856|5|353| +2450815|12859|5|674| +2450815|12860|5|993| +2450815|12862|5|861| +2450815|12865|5|7| +2450815|12866|5|414| +2450815|12868|5|369| +2450815|12871|5|922| +2450815|12872|5|781| +2450815|12874|5|244| +2450815|12877|5|342| +2450815|12878|5|| +2450815|12880|5|830| +2450815|12883|5|172| +2450815|12884|5|632| +2450815|12886|5|464| +2450815|12889|5|392| +2450815|12890|5|720| +2450815|12892|5|| +2450815|12895|5|672| +2450815|12896|5|974| +2450815|12898|5|458| +2450815|12901|5|778| +2450815|12902|5|719| +2450815|12904|5|405| +2450815|12907|5|205| +2450815|12908|5|874| +2450815|12910|5|960| +2450815|12913|5|235| +2450815|12914|5|236| +2450815|12916|5|814| +2450815|12919|5|364| +2450815|12920|5|| +2450815|12922|5|424| +2450815|12925|5|770| +2450815|12926|5|721| +2450815|12928|5|459| +2450815|12931|5|258| +2450815|12932|5|59| +2450815|12934|5|789| +2450815|12937|5|912| +2450815|12938|5|598| +2450815|12940|5|527| +2450815|12943|5|630| +2450815|12944|5|217| +2450815|12946|5|710| +2450815|12949|5|372| +2450815|12950|5|585| +2450815|12952|5|521| +2450815|12955|5|| +2450815|12956|5|904| +2450815|12958|5|705| +2450815|12961|5|399| +2450815|12962|5|371| +2450815|12964|5|29| +2450815|12967|5|807| +2450815|12968|5|70| +2450815|12970|5|914| +2450815|12973|5|709| +2450815|12974|5|293| +2450815|12976|5|6| +2450815|12979|5|255| +2450815|12980|5|109| +2450815|12982|5|829| +2450815|12985|5|10| +2450815|12986|5|362| +2450815|12988|5|219| +2450815|12991|5|536| +2450815|12992|5|266| +2450815|12994|5|328| +2450815|12997|5|269| +2450815|12998|5|483| +2450815|13000|5|| +2450815|13003|5|173| +2450815|13004|5|336| +2450815|13006|5|193| +2450815|13009|5|495| +2450815|13010|5|154| +2450815|13012|5|835| +2450815|13015|5|740| +2450815|13016|5|670| +2450815|13018|5|167| +2450815|13021|5|479| +2450815|13022|5|| +2450815|13024|5|805| +2450815|13027|5|632| +2450815|13028|5|100| +2450815|13030|5|187| +2450815|13033|5|907| +2450815|13034|5|600| +2450815|13036|5|903| +2450815|13039|5|794| +2450815|13040|5|974| +2450815|13042|5|264| +2450815|13045|5|386| +2450815|13046|5|100| +2450815|13048|5|175| +2450815|13051|5|474| +2450815|13052|5|162| +2450815|13054|5|632| +2450815|13057|5|805| +2450815|13058|5|870| +2450815|13060|5|362| +2450815|13063|5|972| +2450815|13064|5|474| +2450815|13066|5|787| +2450815|13069|5|858| +2450815|13070|5|440| +2450815|13072|5|573| +2450815|13075|5|82| +2450815|13076|5|185| +2450815|13078|5|126| +2450815|13081|5|744| +2450815|13082|5|340| +2450815|13084|5|674| +2450815|13087|5|| +2450815|13088|5|701| +2450815|13090|5|360| +2450815|13093|5|863| +2450815|13094|5|204| +2450815|13096|5|875| +2450815|13099|5|41| +2450815|13100|5|936| +2450815|13102|5|627| +2450815|13105|5|537| +2450815|13106|5|17| +2450815|13108|5|998| +2450815|13111|5|490| +2450815|13112|5|779| +2450815|13114|5|54| +2450815|13117|5|| +2450815|13118|5|223| +2450815|13120|5|407| +2450815|13123|5|358| +2450815|13124|5|86| +2450815|13126|5|869| +2450815|13129|5|999| +2450815|13130|5|72| +2450815|13132|5|863| +2450815|13135|5|918| +2450815|13136|5|| +2450815|13138|5|385| +2450815|13141|5|67| +2450815|13142|5|418| +2450815|13144|5|3| +2450815|13147|5|128| +2450815|13148|5|423| +2450815|13150|5|654| +2450815|13153|5|503| +2450815|13154|5|897| +2450815|13156|5|282| +2450815|13159|5|802| +2450815|13160|5|| +2450815|13162|5|532| +2450815|13165|5|| +2450815|13166|5|977| +2450815|13168|5|674| +2450815|13171|5|991| +2450815|13172|5|816| +2450815|13174|5|990| +2450815|13177|5|438| +2450815|13178|5|392| +2450815|13180|5|39| +2450815|13183|5|427| +2450815|13184|5|172| +2450815|13186|5|509| +2450815|13189|5|195| +2450815|13190|5|866| +2450815|13192|5|454| +2450815|13195|5|509| +2450815|13196|5|782| +2450815|13198|5|613| +2450815|13201|5|629| +2450815|13202|5|890| +2450815|13204|5|| +2450815|13207|5|847| +2450815|13208|5|731| +2450815|13210|5|476| +2450815|13213|5|560| +2450815|13214|5|739| +2450815|13216|5|648| +2450815|13219|5|358| +2450815|13220|5|464| +2450815|13222|5|578| +2450815|13225|5|651| +2450815|13226|5|711| +2450815|13228|5|902| +2450815|13231|5|694| +2450815|13232|5|| +2450815|13234|5|317| +2450815|13237|5|494| +2450815|13238|5|| +2450815|13240|5|2| +2450815|13243|5|213| +2450815|13244|5|471| +2450815|13246|5|273| +2450815|13249|5|691| +2450815|13250|5|999| +2450815|13252|5|378| +2450815|13255|5|971| +2450815|13256|5|347| +2450815|13258|5|185| +2450815|13261|5|430| +2450815|13262|5|979| +2450815|13264|5|| +2450815|13267|5|767| +2450815|13268|5|112| +2450815|13270|5|479| +2450815|13273|5|201| +2450815|13274|5|62| +2450815|13276|5|284| +2450815|13279|5|216| +2450815|13280|5|383| +2450815|13282|5|478| +2450815|13285|5|528| +2450815|13286|5|405| +2450815|13288|5|716| +2450815|13291|5|928| +2450815|13292|5|272| +2450815|13294|5|339| +2450815|13297|5|927| +2450815|13298|5|118| +2450815|13300|5|466| +2450815|13303|5|822| +2450815|13304|5|305| +2450815|13306|5|466| +2450815|13309|5|377| +2450815|13310|5|547| +2450815|13312|5|639| +2450815|13315|5|572| +2450815|13316|5|187| +2450815|13318|5|669| +2450815|13321|5|1000| +2450815|13322|5|839| +2450815|13324|5|437| +2450815|13327|5|291| +2450815|13328|5|600| +2450815|13330|5|793| +2450815|13333|5|463| +2450815|13334|5|345| +2450815|13336|5|988| +2450815|13339|5|223| +2450815|13340|5|998| +2450815|13342|5|796| +2450815|13345|5|201| +2450815|13346|5|815| +2450815|13348|5|472| +2450815|13351|5|330| +2450815|13352|5|902| +2450815|13354|5|168| +2450815|13357|5|435| +2450815|13358|5|444| +2450815|13360|5|595| +2450815|13363|5|645| +2450815|13364|5|450| +2450815|13366|5|810| +2450815|13369|5|682| +2450815|13370|5|521| +2450815|13372|5|512| +2450815|13375|5|948| +2450815|13376|5|144| +2450815|13378|5|415| +2450815|13381|5|769| +2450815|13382|5|778| +2450815|13384|5|897| +2450815|13387|5|205| +2450815|13388|5|735| +2450815|13390|5|850| +2450815|13393|5|937| +2450815|13394|5|940| +2450815|13396|5|172| +2450815|13399|5|464| +2450815|13400|5|476| +2450815|13402|5|7| +2450815|13405|5|45| +2450815|13406|5|424| +2450815|13408|5|488| +2450815|13411|5|169| +2450815|13412|5|85| +2450815|13414|5|515| +2450815|13417|5|985| +2450815|13418|5|843| +2450815|13420|5|642| +2450815|13423|5|812| +2450815|13424|5|265| +2450815|13426|5|641| +2450815|13429|5|125| +2450815|13430|5|49| +2450815|13432|5|855| +2450815|13435|5|558| +2450815|13436|5|930| +2450815|13438|5|673| +2450815|13441|5|936| +2450815|13442|5|718| +2450815|13444|5|967| +2450815|13447|5|62| +2450815|13448|5|181| +2450815|13450|5|843| +2450815|13453|5|827| +2450815|13454|5|332| +2450815|13456|5|187| +2450815|13459|5|144| +2450815|13460|5|160| +2450815|13462|5|780| +2450815|13465|5|427| +2450815|13466|5|814| +2450815|13468|5|78| +2450815|13471|5|778| +2450815|13472|5|532| +2450815|13474|5|498| +2450815|13477|5|141| +2450815|13478|5|428| +2450815|13480|5|686| +2450815|13483|5|252| +2450815|13484|5|184| +2450815|13486|5|753| +2450815|13489|5|697| +2450815|13490|5|626| +2450815|13492|5|990| +2450815|13495|5|734| +2450815|13496|5|214| +2450815|13498|5|890| +2450815|13501|5|78| +2450815|13502|5|741| +2450815|13504|5|474| +2450815|13507|5|88| +2450815|13508|5|832| +2450815|13510|5|120| +2450815|13513|5|364| +2450815|13514|5|591| +2450815|13516|5|| +2450815|13519|5|234| +2450815|13520|5|632| +2450815|13522|5|134| +2450815|13525|5|| +2450815|13526|5|82| +2450815|13528|5|202| +2450815|13531|5|| +2450815|13532|5|827| +2450815|13534|5|524| +2450815|13537|5|94| +2450815|13538|5|142| +2450815|13540|5|792| +2450815|13543|5|66| +2450815|13544|5|| +2450815|13546|5|872| +2450815|13549|5|364| +2450815|13550|5|122| +2450815|13552|5|691| +2450815|13555|5|635| +2450815|13556|5|194| +2450815|13558|5|848| +2450815|13561|5|337| +2450815|13562|5|506| +2450815|13564|5|| +2450815|13567|5|899| +2450815|13568|5|513| +2450815|13570|5|779| +2450815|13573|5|82| +2450815|13574|5|244| +2450815|13576|5|629| +2450815|13579|5|612| +2450815|13580|5|24| +2450815|13582|5|881| +2450815|13585|5|911| +2450815|13586|5|509| +2450815|13588|5|334| +2450815|13591|5|692| +2450815|13592|5|91| +2450815|13594|5|161| +2450815|13597|5|753| +2450815|13598|5|519| +2450815|13600|5|968| +2450815|13603|5|846| +2450815|13604|5|842| +2450815|13606|5|448| +2450815|13609|5|926| +2450815|13610|5|208| +2450815|13612|5|359| +2450815|13615|5|91| +2450815|13616|5|798| +2450815|13618|5|228| +2450815|13621|5|939| +2450815|13622|5|929| +2450815|13624|5|606| +2450815|13627|5|380| +2450815|13628|5|89| +2450815|13630|5|44| +2450815|13633|5|452| +2450815|13634|5|619| +2450815|13636|5|495| +2450815|13639|5|171| +2450815|13640|5|73| +2450815|13642|5|116| +2450815|13645|5|953| +2450815|13646|5|568| +2450815|13648|5|214| +2450815|13651|5|172| +2450815|13652|5|605| +2450815|13654|5|775| +2450815|13657|5|879| +2450815|13658|5|728| +2450815|13660|5|| +2450815|13663|5|264| +2450815|13664|5|844| +2450815|13666|5|787| +2450815|13669|5|739| +2450815|13670|5|633| +2450815|13672|5|919| +2450815|13675|5|602| +2450815|13676|5|234| +2450815|13678|5|656| +2450815|13681|5|632| +2450815|13682|5|978| +2450815|13684|5|266| +2450815|13687|5|26| +2450815|13688|5|816| +2450815|13690|5|4| +2450815|13693|5|504| +2450815|13694|5|98| +2450815|13696|5|861| +2450815|13699|5|35| +2450815|13700|5|270| +2450815|13702|5|845| +2450815|13705|5|940| +2450815|13706|5|754| +2450815|13708|5|812| +2450815|13711|5|887| +2450815|13712|5|652| +2450815|13714|5|734| +2450815|13717|5|225| +2450815|13718|5|998| +2450815|13720|5|579| +2450815|13723|5|890| +2450815|13724|5|| +2450815|13726|5|449| +2450815|13729|5|389| +2450815|13730|5|491| +2450815|13732|5|101| +2450815|13735|5|992| +2450815|13736|5|586| +2450815|13738|5|911| +2450815|13741|5|499| +2450815|13742|5|872| +2450815|13744|5|418| +2450815|13747|5|204| +2450815|13748|5|131| +2450815|13750|5|219| +2450815|13753|5|300| +2450815|13754|5|396| +2450815|13756|5|59| +2450815|13759|5|464| +2450815|13760|5|844| +2450815|13762|5|131| +2450815|13765|5|| +2450815|13766|5|592| +2450815|13768|5|282| +2450815|13771|5|13| +2450815|13772|5|305| +2450815|13774|5|23| +2450815|13777|5|656| +2450815|13778|5|1000| +2450815|13780|5|584| +2450815|13783|5|651| +2450815|13784|5|830| +2450815|13786|5|| +2450815|13789|5|808| +2450815|13790|5|573| +2450815|13792|5|563| +2450815|13795|5|762| +2450815|13796|5|148| +2450815|13798|5|663| +2450815|13801|5|477| +2450815|13802|5|643| +2450815|13804|5|363| +2450815|13807|5|569| +2450815|13808|5|613| +2450815|13810|5|769| +2450815|13813|5|9| +2450815|13814|5|916| +2450815|13816|5|449| +2450815|13819|5|483| +2450815|13820|5|750| +2450815|13822|5|482| +2450815|13825|5|871| +2450815|13826|5|721| +2450815|13828|5|404| +2450815|13831|5|332| +2450815|13832|5|29| +2450815|13834|5|733| +2450815|13837|5|313| +2450815|13838|5|409| +2450815|13840|5|850| +2450815|13843|5|185| +2450815|13844|5|685| +2450815|13846|5|764| +2450815|13849|5|602| +2450815|13850|5|957| +2450815|13852|5|419| +2450815|13855|5|567| +2450815|13856|5|864| +2450815|13858|5|592| +2450815|13861|5|103| +2450815|13862|5|260| +2450815|13864|5|774| +2450815|13867|5|736| +2450815|13868|5|| +2450815|13870|5|542| +2450815|13873|5|52| +2450815|13874|5|857| +2450815|13876|5|888| +2450815|13879|5|317| +2450815|13880|5|467| +2450815|13882|5|949| +2450815|13885|5|152| +2450815|13886|5|546| +2450815|13888|5|308| +2450815|13891|5|350| +2450815|13892|5|390| +2450815|13894|5|674| +2450815|13897|5|91| +2450815|13898|5|337| +2450815|13900|5|| +2450815|13903|5|397| +2450815|13904|5|681| +2450815|13906|5|401| +2450815|13909|5|72| +2450815|13910|5|596| +2450815|13912|5|644| +2450815|13915|5|157| +2450815|13916|5|843| +2450815|13918|5|340| +2450815|13921|5|594| +2450815|13922|5|61| +2450815|13924|5|632| +2450815|13927|5|369| +2450815|13928|5|534| +2450815|13930|5|880| +2450815|13933|5|618| +2450815|13934|5|333| +2450815|13936|5|90| +2450815|13939|5|| +2450815|13940|5|2| +2450815|13942|5|| +2450815|13945|5|191| +2450815|13946|5|589| +2450815|13948|5|| +2450815|13951|5|843| +2450815|13952|5|45| +2450815|13954|5|948| +2450815|13957|5|461| +2450815|13958|5|697| +2450815|13960|5|943| +2450815|13963|5|286| +2450815|13964|5|287| +2450815|13966|5|523| +2450815|13969|5|806| +2450815|13970|5|141| +2450815|13972|5|910| +2450815|13975|5|990| +2450815|13976|5|50| +2450815|13978|5|139| +2450815|13981|5|395| +2450815|13982|5|634| +2450815|13984|5|985| +2450815|13987|5|243| +2450815|13988|5|663| +2450815|13990|5|163| +2450815|13993|5|772| +2450815|13994|5|464| +2450815|13996|5|783| +2450815|13999|5|955| +2450815|14000|5|544| +2450815|14002|5|615| +2450815|14005|5|563| +2450815|14006|5|355| +2450815|14008|5|54| +2450815|14011|5|| +2450815|14012|5|291| +2450815|14014|5|638| +2450815|14017|5|193| +2450815|14018|5|38| +2450815|14020|5|64| +2450815|14023|5|210| +2450815|14024|5|646| +2450815|14026|5|| +2450815|14029|5|584| +2450815|14030|5|| +2450815|14032|5|565| +2450815|14035|5|251| +2450815|14036|5|889| +2450815|14038|5|567| +2450815|14041|5|840| +2450815|14042|5|465| +2450815|14044|5|667| +2450815|14047|5|720| +2450815|14048|5|300| +2450815|14050|5|978| +2450815|14053|5|463| +2450815|14054|5|840| +2450815|14056|5|440| +2450815|14059|5|723| +2450815|14060|5|636| +2450815|14062|5|361| +2450815|14065|5|266| +2450815|14066|5|762| +2450815|14068|5|418| +2450815|14071|5|145| +2450815|14072|5|908| +2450815|14074|5|256| +2450815|14077|5|749| +2450815|14078|5|860| +2450815|14080|5|560| +2450815|14083|5|549| +2450815|14084|5|972| +2450815|14086|5|173| +2450815|14089|5|103| +2450815|14090|5|51| +2450815|14092|5|426| +2450815|14095|5|| +2450815|14096|5|248| +2450815|14098|5|996| +2450815|14101|5|941| +2450815|14102|5|449| +2450815|14104|5|419| +2450815|14107|5|758| +2450815|14108|5|977| +2450815|14110|5|793| +2450815|14113|5|540| +2450815|14114|5|857| +2450815|14116|5|596| +2450815|14119|5|536| +2450815|14120|5|| +2450815|14122|5|696| +2450815|14125|5|223| +2450815|14126|5|628| +2450815|14128|5|459| +2450815|14131|5|970| +2450815|14132|5|107| +2450815|14134|5|173| +2450815|14137|5|| +2450815|14138|5|41| +2450815|14140|5|83| +2450815|14143|5|205| +2450815|14144|5|565| +2450815|14146|5|519| +2450815|14149|5|671| +2450815|14150|5|484| +2450815|14152|5|603| +2450815|14155|5|638| +2450815|14156|5|277| +2450815|14158|5|564| +2450815|14161|5|435| +2450815|14162|5|770| +2450815|14164|5|427| +2450815|14167|5|480| +2450815|14168|5|629| +2450815|14170|5|338| +2450815|14173|5|852| +2450815|14174|5|222| +2450815|14176|5|866| +2450815|14179|5|466| +2450815|14180|5|567| +2450815|14182|5|983| +2450815|14185|5|155| +2450815|14186|5|26| +2450815|14188|5|515| +2450815|14191|5|972| +2450815|14192|5|491| +2450815|14194|5|712| +2450815|14197|5|| +2450815|14198|5|729| +2450815|14200|5|565| +2450815|14203|5|428| +2450815|14204|5|122| +2450815|14206|5|836| +2450815|14209|5|363| +2450815|14210|5|223| +2450815|14212|5|290| +2450815|14215|5|343| +2450815|14216|5|129| +2450815|14218|5|362| +2450815|14221|5|866| +2450815|14222|5|87| +2450815|14224|5|613| +2450815|14227|5|920| +2450815|14228|5|495| +2450815|14230|5|392| +2450815|14233|5|684| +2450815|14234|5|918| +2450815|14236|5|290| +2450815|14239|5|200| +2450815|14240|5|549| +2450815|14242|5|345| +2450815|14245|5|703| +2450815|14246|5|953| +2450815|14248|5|349| +2450815|14251|5|639| +2450815|14252|5|85| +2450815|14254|5|467| +2450815|14257|5|835| +2450815|14258|5|914| +2450815|14260|5|69| +2450815|14263|5|249| +2450815|14264|5|44| +2450815|14266|5|299| +2450815|14269|5|82| +2450815|14270|5|828| +2450815|14272|5|285| +2450815|14275|5|561| +2450815|14276|5|358| +2450815|14278|5|627| +2450815|14281|5|848| +2450815|14282|5|206| +2450815|14284|5|133| +2450815|14287|5|960| +2450815|14288|5|365| +2450815|14290|5|268| +2450815|14293|5|112| +2450815|14294|5|459| +2450815|14296|5|914| +2450815|14299|5|462| +2450815|14300|5|962| +2450815|14302|5|828| +2450815|14305|5|794| +2450815|14306|5|620| +2450815|14308|5|594| +2450815|14311|5|464| +2450815|14312|5|67| +2450815|14314|5|325| +2450815|14317|5|885| +2450815|14318|5|466| +2450815|14320|5|102| +2450815|14323|5|383| +2450815|14324|5|657| +2450815|14326|5|202| +2450815|14329|5|353| +2450815|14330|5|503| +2450815|14332|5|854| +2450815|14335|5|675| +2450815|14336|5|153| +2450815|14338|5|743| +2450815|14341|5|837| +2450815|14342|5|612| +2450815|14344|5|91| +2450815|14347|5|529| +2450815|14348|5|467| +2450815|14350|5|165| +2450815|14353|5|283| +2450815|14354|5|343| +2450815|14356|5|384| +2450815|14359|5|402| +2450815|14360|5|346| +2450815|14362|5|300| +2450815|14365|5|962| +2450815|14366|5|704| +2450815|14368|5|105| +2450815|14371|5|795| +2450815|14372|5|| +2450815|14374|5|475| +2450815|14377|5|407| +2450815|14378|5|153| +2450815|14380|5|665| +2450815|14383|5|| +2450815|14384|5|736| +2450815|14386|5|820| +2450815|14389|5|228| +2450815|14390|5|357| +2450815|14392|5|946| +2450815|14395|5|602| +2450815|14396|5|915| +2450815|14398|5|395| +2450815|14401|5|753| +2450815|14402|5|986| +2450815|14404|5|702| +2450815|14407|5|262| +2450815|14408|5|640| +2450815|14410|5|550| +2450815|14413|5|662| +2450815|14414|5|137| +2450815|14416|5|| +2450815|14419|5|984| +2450815|14420|5|184| +2450815|14422|5|445| +2450815|14425|5|221| +2450815|14426|5|497| +2450815|14428|5|742| +2450815|14431|5|484| +2450815|14432|5|902| +2450815|14434|5|140| +2450815|14437|5|995| +2450815|14438|5|25| +2450815|14440|5|114| +2450815|14443|5|660| +2450815|14444|5|166| +2450815|14446|5|345| +2450815|14449|5|284| +2450815|14450|5|| +2450815|14452|5|147| +2450815|14455|5|817| +2450815|14456|5|712| +2450815|14458|5|535| +2450815|14461|5|827| +2450815|14462|5|211| +2450815|14464|5|317| +2450815|14467|5|737| +2450815|14468|5|378| +2450815|14470|5|755| +2450815|14473|5|870| +2450815|14474|5|635| +2450815|14476|5|884| +2450815|14479|5|969| +2450815|14480|5|627| +2450815|14482|5|114| +2450815|14485|5|432| +2450815|14486|5|88| +2450815|14488|5|78| +2450815|14491|5|904| +2450815|14492|5|248| +2450815|14494|5|220| +2450815|14497|5|493| +2450815|14498|5|831| +2450815|14500|5|545| +2450815|14503|5|479| +2450815|14504|5|991| +2450815|14506|5|345| +2450815|14509|5|389| +2450815|14510|5|| +2450815|14512|5|775| +2450815|14515|5|702| +2450815|14516|5|845| +2450815|14518|5|565| +2450815|14521|5|| +2450815|14522|5|863| +2450815|14524|5|248| +2450815|14527|5|486| +2450815|14528|5|17| +2450815|14530|5|594| +2450815|14533|5|833| +2450815|14534|5|406| +2450815|14536|5|630| +2450815|14539|5|855| +2450815|14540|5|741| +2450815|14542|5|869| +2450815|14545|5|419| +2450815|14546|5|117| +2450815|14548|5|| +2450815|14551|5|729| +2450815|14552|5|674| +2450815|14554|5|115| +2450815|14557|5|436| +2450815|14558|5|563| +2450815|14560|5|701| +2450815|14563|5|799| +2450815|14564|5|788| +2450815|14566|5|291| +2450815|14569|5|793| +2450815|14570|5|639| +2450815|14572|5|851| +2450815|14575|5|388| +2450815|14576|5|61| +2450815|14578|5|| +2450815|14581|5|991| +2450815|14582|5|791| +2450815|14584|5|595| +2450815|14587|5|304| +2450815|14588|5|18| +2450815|14590|5|322| +2450815|14593|5|471| +2450815|14594|5|905| +2450815|14596|5|470| +2450815|14599|5|493| +2450815|14600|5|916| +2450815|14602|5|976| +2450815|14605|5|895| +2450815|14606|5|884| +2450815|14608|5|340| +2450815|14611|5|834| +2450815|14612|5|929| +2450815|14614|5|99| +2450815|14617|5|490| +2450815|14618|5|396| +2450815|14620|5|315| +2450815|14623|5|898| +2450815|14624|5|| +2450815|14626|5|640| +2450815|14629|5|452| +2450815|14630|5|243| +2450815|14632|5|696| +2450815|14635|5|601| +2450815|14636|5|339| +2450815|14638|5|289| +2450815|14641|5|848| +2450815|14642|5|903| +2450815|14644|5|550| +2450815|14647|5|468| +2450815|14648|5|976| +2450815|14650|5|266| +2450815|14653|5|549| +2450815|14654|5|704| +2450815|14656|5|592| +2450815|14659|5|898| +2450815|14660|5|126| +2450815|14662|5|639| +2450815|14665|5|319| +2450815|14666|5|608| +2450815|14668|5|| +2450815|14671|5|| +2450815|14672|5|872| +2450815|14674|5|790| +2450815|14677|5|938| +2450815|14678|5|863| +2450815|14680|5|252| +2450815|14683|5|532| +2450815|14684|5|407| +2450815|14686|5|678| +2450815|14689|5|881| +2450815|14690|5|151| +2450815|14692|5|674| +2450815|14695|5|531| +2450815|14696|5|648| +2450815|14698|5|810| +2450815|14701|5|| +2450815|14702|5|945| +2450815|14704|5|94| +2450815|14707|5|218| +2450815|14708|5|688| +2450815|14710|5|978| +2450815|14713|5|634| +2450815|14714|5|523| +2450815|14716|5|842| +2450815|14719|5|661| +2450815|14720|5|133| +2450815|14722|5|129| +2450815|14725|5|| +2450815|14726|5|312| +2450815|14728|5|139| +2450815|14731|5|663| +2450815|14732|5|881| +2450815|14734|5|575| +2450815|14737|5|167| +2450815|14738|5|776| +2450815|14740|5|490| +2450815|14743|5|437| +2450815|14744|5|354| +2450815|14746|5|417| +2450815|14749|5|582| +2450815|14750|5|466| +2450815|14752|5|963| +2450815|14755|5|892| +2450815|14756|5|785| +2450815|14758|5|275| +2450815|14761|5|175| +2450815|14762|5|334| +2450815|14764|5|895| +2450815|14767|5|386| +2450815|14768|5|44| +2450815|14770|5|700| +2450815|14773|5|74| +2450815|14774|5|431| +2450815|14776|5|349| +2450815|14779|5|132| +2450815|14780|5|146| +2450815|14782|5|328| +2450815|14785|5|253| +2450815|14786|5|635| +2450815|14788|5|168| +2450815|14791|5|600| +2450815|14792|5|26| +2450815|14794|5|193| +2450815|14797|5|978| +2450815|14798|5|783| +2450815|14800|5|| +2450815|14803|5|558| +2450815|14804|5|576| +2450815|14806|5|| +2450815|14809|5|179| +2450815|14810|5|90| +2450815|14812|5|919| +2450815|14815|5|312| +2450815|14816|5|497| +2450815|14818|5|968| +2450815|14821|5|187| +2450815|14822|5|562| +2450815|14824|5|580| +2450815|14827|5|361| +2450815|14828|5|775| +2450815|14830|5|890| +2450815|14833|5|384| +2450815|14834|5|505| +2450815|14836|5|773| +2450815|14839|5|680| +2450815|14840|5|395| +2450815|14842|5|113| +2450815|14845|5|481| +2450815|14846|5|758| +2450815|14848|5|583| +2450815|14851|5|699| +2450815|14852|5|902| +2450815|14854|5|690| +2450815|14857|5|418| +2450815|14858|5|678| +2450815|14860|5|272| +2450815|14863|5|| +2450815|14864|5|714| +2450815|14866|5|56| +2450815|14869|5|819| +2450815|14870|5|246| +2450815|14872|5|107| +2450815|14875|5|984| +2450815|14876|5|182| +2450815|14878|5|650| +2450815|14881|5|105| +2450815|14882|5|819| +2450815|14884|5|976| +2450815|14887|5|419| +2450815|14888|5|368| +2450815|14890|5|244| +2450815|14893|5|88| +2450815|14894|5|912| +2450815|14896|5|467| +2450815|14899|5|560| +2450815|14900|5|585| +2450815|14902|5|748| +2450815|14905|5|249| +2450815|14906|5|424| +2450815|14908|5|255| +2450815|14911|5|892| +2450815|14912|5|962| +2450815|14914|5|762| +2450815|14917|5|941| +2450815|14918|5|250| +2450815|14920|5|721| +2450815|14923|5|410| +2450815|14924|5|67| +2450815|14926|5|318| +2450815|14929|5|613| +2450815|14930|5|74| +2450815|14932|5|158| +2450815|14935|5|90| +2450815|14936|5|194| +2450815|14938|5|819| +2450815|14941|5|219| +2450815|14942|5|583| +2450815|14944|5|810| +2450815|14947|5|438| +2450815|14948|5|492| +2450815|14950|5|576| +2450815|14953|5|734| +2450815|14954|5|6| +2450815|14956|5|620| +2450815|14959|5|354| +2450815|14960|5|102| +2450815|14962|5|902| +2450815|14965|5|647| +2450815|14966|5|177| +2450815|14968|5|251| +2450815|14971|5|924| +2450815|14972|5|85| +2450815|14974|5|569| +2450815|14977|5|422| +2450815|14978|5|305| +2450815|14980|5|286| +2450815|14983|5|552| +2450815|14984|5|602| +2450815|14986|5|271| +2450815|14989|5|487| +2450815|14990|5|255| +2450815|14992|5|412| +2450815|14995|5|475| +2450815|14996|5|621| +2450815|14998|5|919| +2450815|15001|5|22| +2450815|15002|5|288| +2450815|15004|5|368| +2450815|15007|5|524| +2450815|15008|5|131| +2450815|15010|5|249| +2450815|15013|5|189| +2450815|15014|5|796| +2450815|15016|5|731| +2450815|15019|5|54| +2450815|15020|5|514| +2450815|15022|5|511| +2450815|15025|5|572| +2450815|15026|5|258| +2450815|15028|5|432| +2450815|15031|5|809| +2450815|15032|5|711| +2450815|15034|5|170| +2450815|15037|5|989| +2450815|15038|5|924| +2450815|15040|5|375| +2450815|15043|5|775| +2450815|15044|5|603| +2450815|15046|5|956| +2450815|15049|5|77| +2450815|15050|5|303| +2450815|15052|5|102| +2450815|15055|5|336| +2450815|15056|5|792| +2450815|15058|5|456| +2450815|15061|5|566| +2450815|15062|5|183| +2450815|15064|5|101| +2450815|15067|5|636| +2450815|15068|5|809| +2450815|15070|5|711| +2450815|15073|5|771| +2450815|15074|5|850| +2450815|15076|5|64| +2450815|15079|5|864| +2450815|15080|5|644| +2450815|15082|5|708| +2450815|15085|5|3| +2450815|15086|5|994| +2450815|15088|5|562| +2450815|15091|5|661| +2450815|15092|5|17| +2450815|15094|5|650| +2450815|15097|5|680| +2450815|15098|5|746| +2450815|15100|5|551| +2450815|15103|5|589| +2450815|15104|5|654| +2450815|15106|5|623| +2450815|15109|5|342| +2450815|15110|5|389| +2450815|15112|5|743| +2450815|15115|5|135| +2450815|15116|5|358| +2450815|15118|5|592| +2450815|15121|5|220| +2450815|15122|5|975| +2450815|15124|5|891| +2450815|15127|5|720| +2450815|15128|5|772| +2450815|15130|5|100| +2450815|15133|5|311| +2450815|15134|5|53| +2450815|15136|5|671| +2450815|15139|5|421| +2450815|15140|5|340| +2450815|15142|5|677| +2450815|15145|5|134| +2450815|15146|5|440| +2450815|15148|5|279| +2450815|15151|5|454| +2450815|15152|5|797| +2450815|15154|5|290| +2450815|15157|5|262| +2450815|15158|5|217| +2450815|15160|5|315| +2450815|15163|5|674| +2450815|15164|5|476| +2450815|15166|5|324| +2450815|15169|5|89| +2450815|15170|5|657| +2450815|15172|5|| +2450815|15175|5|618| +2450815|15176|5|752| +2450815|15178|5|884| +2450815|15181|5|410| +2450815|15182|5|715| +2450815|15184|5|885| +2450815|15187|5|933| +2450815|15188|5|54| +2450815|15190|5|866| +2450815|15193|5|945| +2450815|15194|5|839| +2450815|15196|5|993| +2450815|15199|5|945| +2450815|15200|5|154| +2450815|15202|5|911| +2450815|15205|5|679| +2450815|15206|5|495| +2450815|15208|5|826| +2450815|15211|5|921| +2450815|15212|5|834| +2450815|15214|5|| +2450815|15217|5|194| +2450815|15218|5|265| +2450815|15220|5|615| +2450815|15223|5|190| +2450815|15224|5|536| +2450815|15226|5|| +2450815|15229|5|478| +2450815|15230|5|10| +2450815|15232|5|943| +2450815|15235|5|19| +2450815|15236|5|660| +2450815|15238|5|267| +2450815|15241|5|102| +2450815|15242|5|640| +2450815|15244|5|412| +2450815|15247|5|869| +2450815|15248|5|319| +2450815|15250|5|359| +2450815|15253|5|850| +2450815|15254|5|581| +2450815|15256|5|477| +2450815|15259|5|100| +2450815|15260|5|644| +2450815|15262|5|93| +2450815|15265|5|56| +2450815|15266|5|760| +2450815|15268|5|607| +2450815|15271|5|662| +2450815|15272|5|731| +2450815|15274|5|840| +2450815|15277|5|785| +2450815|15278|5|986| +2450815|15280|5|156| +2450815|15283|5|753| +2450815|15284|5|87| +2450815|15286|5|160| +2450815|15289|5|788| +2450815|15290|5|556| +2450815|15292|5|50| +2450815|15295|5|353| +2450815|15296|5|433| +2450815|15298|5|442| +2450815|15301|5|734| +2450815|15302|5|650| +2450815|15304|5|| +2450815|15307|5|617| +2450815|15308|5|938| +2450815|15310|5|179| +2450815|15313|5|771| +2450815|15314|5|534| +2450815|15316|5|| +2450815|15319|5|846| +2450815|15320|5|533| +2450815|15322|5|98| +2450815|15325|5|660| +2450815|15326|5|883| +2450815|15328|5|264| +2450815|15331|5|286| +2450815|15332|5|951| +2450815|15334|5|763| +2450815|15337|5|775| +2450815|15338|5|284| +2450815|15340|5|760| +2450815|15343|5|572| +2450815|15344|5|62| +2450815|15346|5|256| +2450815|15349|5|966| +2450815|15350|5|823| +2450815|15352|5|286| +2450815|15355|5|452| +2450815|15356|5|| +2450815|15358|5|276| +2450815|15361|5|| +2450815|15362|5|808| +2450815|15364|5|131| +2450815|15367|5|43| +2450815|15368|5|192| +2450815|15370|5|371| +2450815|15373|5|| +2450815|15374|5|340| +2450815|15376|5|999| +2450815|15379|5|978| +2450815|15380|5|576| +2450815|15382|5|490| +2450815|15385|5|939| +2450815|15386|5|718| +2450815|15388|5|146| +2450815|15391|5|125| +2450815|15392|5|935| +2450815|15394|5|494| +2450815|15397|5|71| +2450815|15398|5|611| +2450815|15400|5|746| +2450815|15403|5|617| +2450815|15404|5|543| +2450815|15406|5|973| +2450815|15409|5|169| +2450815|15410|5|934| +2450815|15412|5|571| +2450815|15415|5|960| +2450815|15416|5|500| +2450815|15418|5|54| +2450815|15421|5|581| +2450815|15422|5|835| +2450815|15424|5|147| +2450815|15427|5|| +2450815|15428|5|206| +2450815|15430|5|307| +2450815|15433|5|435| +2450815|15434|5|127| +2450815|15436|5|239| +2450815|15439|5|809| +2450815|15440|5|517| +2450815|15442|5|| +2450815|15445|5|703| +2450815|15446|5|| +2450815|15448|5|415| +2450815|15451|5|338| +2450815|15452|5|799| +2450815|15454|5|415| +2450815|15457|5|505| +2450815|15458|5|367| +2450815|15460|5|217| +2450815|15463|5|| +2450815|15464|5|| +2450815|15466|5|574| +2450815|15469|5|587| +2450815|15470|5|932| +2450815|15472|5|701| +2450815|15475|5|87| +2450815|15476|5|478| +2450815|15478|5|580| +2450815|15481|5|690| +2450815|15482|5|101| +2450815|15484|5|472| +2450815|15487|5|524| +2450815|15488|5|346| +2450815|15490|5|748| +2450815|15493|5|873| +2450815|15494|5|| +2450815|15496|5|368| +2450815|15499|5|433| +2450815|15500|5|317| +2450815|15502|5|51| +2450815|15505|5|120| +2450815|15506|5|248| +2450815|15508|5|25| +2450815|15511|5|665| +2450815|15512|5|755| +2450815|15514|5|321| +2450815|15517|5|587| +2450815|15518|5|377| +2450815|15520|5|373| +2450815|15523|5|290| +2450815|15524|5|471| +2450815|15526|5|319| +2450815|15529|5|814| +2450815|15530|5|722| +2450815|15532|5|939| +2450815|15535|5|659| +2450815|15536|5|382| +2450815|15538|5|527| +2450815|15541|5|466| +2450815|15542|5|590| +2450815|15544|5|870| +2450815|15547|5|648| +2450815|15548|5|896| +2450815|15550|5|662| +2450815|15553|5|959| +2450815|15554|5|93| +2450815|15556|5|185| +2450815|15559|5|279| +2450815|15560|5|741| +2450815|15562|5|16| +2450815|15565|5|550| +2450815|15566|5|327| +2450815|15568|5|283| +2450815|15571|5|844| +2450815|15572|5|334| +2450815|15574|5|542| +2450815|15577|5|840| +2450815|15578|5|763| +2450815|15580|5|219| +2450815|15583|5|593| +2450815|15584|5|938| +2450815|15586|5|666| +2450815|15589|5|904| +2450815|15590|5|432| +2450815|15592|5|726| +2450815|15595|5|473| +2450815|15596|5|816| +2450815|15598|5|879| +2450815|15601|5|528| +2450815|15602|5|312| +2450815|15604|5|576| +2450815|15607|5|868| +2450815|15608|5|| +2450815|15610|5|| +2450815|15613|5|839| +2450815|15614|5|423| +2450815|15616|5|917| +2450815|15619|5|819| +2450815|15620|5|337| +2450815|15622|5|459| +2450815|15625|5|265| +2450815|15626|5|546| +2450815|15628|5|217| +2450815|15631|5|246| +2450815|15632|5|493| +2450815|15634|5|498| +2450815|15637|5|507| +2450815|15638|5|149| +2450815|15640|5|417| +2450815|15643|5|335| +2450815|15644|5|663| +2450815|15646|5|187| +2450815|15649|5|35| +2450815|15650|5|882| +2450815|15652|5|25| +2450815|15655|5|167| +2450815|15656|5|809| +2450815|15658|5|187| +2450815|15661|5|153| +2450815|15662|5|590| +2450815|15664|5|820| +2450815|15667|5|60| +2450815|15668|5|120| +2450815|15670|5|181| +2450815|15673|5|445| +2450815|15674|5|492| +2450815|15676|5|888| +2450815|15679|5|248| +2450815|15680|5|442| +2450815|15682|5|459| +2450815|15685|5|| +2450815|15686|5|689| +2450815|15688|5|360| +2450815|15691|5|180| +2450815|15692|5|742| +2450815|15694|5|68| +2450815|15697|5|933| +2450815|15698|5|740| +2450815|15700|5|725| +2450815|15703|5|835| +2450815|15704|5|330| +2450815|15706|5|438| +2450815|15709|5|745| +2450815|15710|5|718| +2450815|15712|5|386| +2450815|15715|5|588| +2450815|15716|5|412| +2450815|15718|5|110| +2450815|15721|5|| +2450815|15722|5|398| +2450815|15724|5|258| +2450815|15727|5|562| +2450815|15728|5|391| +2450815|15730|5|200| +2450815|15733|5|| +2450815|15734|5|65| +2450815|15736|5|686| +2450815|15739|5|752| +2450815|15740|5|358| +2450815|15742|5|840| +2450815|15745|5|470| +2450815|15746|5|134| +2450815|15748|5|719| +2450815|15751|5|702| +2450815|15752|5|365| +2450815|15754|5|| +2450815|15757|5|118| +2450815|15758|5|810| +2450815|15760|5|946| +2450815|15763|5|375| +2450815|15764|5|| +2450815|15766|5|731| +2450815|15769|5|| +2450815|15770|5|812| +2450815|15772|5|808| +2450815|15775|5|563| +2450815|15776|5|896| +2450815|15778|5|421| +2450815|15781|5|635| +2450815|15782|5|426| +2450815|15784|5|276| +2450815|15787|5|940| +2450815|15788|5|577| +2450815|15790|5|303| +2450815|15793|5|803| +2450815|15794|5|256| +2450815|15796|5|253| +2450815|15799|5|169| +2450815|15800|5|578| +2450815|15802|5|124| +2450815|15805|5|| +2450815|15806|5|686| +2450815|15808|5|476| +2450815|15811|5|873| +2450815|15812|5|366| +2450815|15814|5|| +2450815|15817|5|511| +2450815|15818|5|566| +2450815|15820|5|102| +2450815|15823|5|905| +2450815|15824|5|407| +2450815|15826|5|143| +2450815|15829|5|85| +2450815|15830|5|537| +2450815|15832|5|74| +2450815|15835|5|279| +2450815|15836|5|179| +2450815|15838|5|616| +2450815|15841|5|0| +2450815|15842|5|935| +2450815|15844|5|628| +2450815|15847|5|499| +2450815|15848|5|6| +2450815|15850|5|168| +2450815|15853|5|50| +2450815|15854|5|127| +2450815|15856|5|147| +2450815|15859|5|27| +2450815|15860|5|275| +2450815|15862|5|960| +2450815|15865|5|630| +2450815|15866|5|197| +2450815|15868|5|921| +2450815|15871|5|459| +2450815|15872|5|780| +2450815|15874|5|814| +2450815|15877|5|| +2450815|15878|5|123| +2450815|15880|5|453| +2450815|15883|5|949| +2450815|15884|5|206| +2450815|15886|5|939| +2450815|15889|5|95| +2450815|15890|5|582| +2450815|15892|5|805| +2450815|15895|5|342| +2450815|15896|5|772| +2450815|15898|5|180| +2450815|15901|5|713| +2450815|15902|5|| +2450815|15904|5|934| +2450815|15907|5|779| +2450815|15908|5|280| +2450815|15910|5|205| +2450815|15913|5|390| +2450815|15914|5|671| +2450815|15916|5|585| +2450815|15919|5|789| +2450815|15920|5|996| +2450815|15922|5|111| +2450815|15925|5|191| +2450815|15926|5|795| +2450815|15928|5|320| +2450815|15931|5|812| +2450815|15932|5|168| +2450815|15934|5|350| +2450815|15937|5|680| +2450815|15938|5|965| +2450815|15940|5|91| +2450815|15943|5|651| +2450815|15944|5|317| +2450815|15946|5|659| +2450815|15949|5|208| +2450815|15950|5|409| +2450815|15952|5|426| +2450815|15955|5|| +2450815|15956|5|320| +2450815|15958|5|150| +2450815|15961|5|867| +2450815|15962|5|944| +2450815|15964|5|152| +2450815|15967|5|181| +2450815|15968|5|894| +2450815|15970|5|7| +2450815|15973|5|6| +2450815|15974|5|499| +2450815|15976|5|771| +2450815|15979|5|791| +2450815|15980|5|600| +2450815|15982|5|854| +2450815|15985|5|407| +2450815|15986|5|239| +2450815|15988|5|978| +2450815|15991|5|676| +2450815|15992|5|98| +2450815|15994|5|315| +2450815|15997|5|468| +2450815|15998|5|478| +2450815|16000|5|526| +2450815|16003|5|468| +2450815|16004|5|969| +2450815|16006|5|996| +2450815|16009|5|962| +2450815|16010|5|234| +2450815|16012|5|81| +2450815|16015|5|752| +2450815|16016|5|643| +2450815|16018|5|368| +2450815|16021|5|504| +2450815|16022|5|731| +2450815|16024|5|302| +2450815|16027|5|799| +2450815|16028|5|500| +2450815|16030|5|131| +2450815|16033|5|46| +2450815|16034|5|820| +2450815|16036|5|446| +2450815|16039|5|502| +2450815|16040|5|162| +2450815|16042|5|381| +2450815|16045|5|990| +2450815|16046|5|613| +2450815|16048|5|510| +2450815|16051|5|184| +2450815|16052|5|174| +2450815|16054|5|811| +2450815|16057|5|115| +2450815|16058|5|380| +2450815|16060|5|902| +2450815|16063|5|411| +2450815|16064|5|256| +2450815|16066|5|427| +2450815|16069|5|894| +2450815|16070|5|747| +2450815|16072|5|227| +2450815|16075|5|399| +2450815|16076|5|244| +2450815|16078|5|421| +2450815|16081|5|117| +2450815|16082|5|176| +2450815|16084|5|306| +2450815|16087|5|213| +2450815|16088|5|327| +2450815|16090|5|670| +2450815|16093|5|277| +2450815|16094|5|737| +2450815|16096|5|442| +2450815|16099|5|813| +2450815|16100|5|432| +2450815|16102|5|521| +2450815|16105|5|789| +2450815|16106|5|904| +2450815|16108|5|409| +2450815|16111|5|490| +2450815|16112|5|226| +2450815|16114|5|610| +2450815|16117|5|724| +2450815|16118|5|266| +2450815|16120|5|668| +2450815|16123|5|757| +2450815|16124|5|378| +2450815|16126|5|928| +2450815|16129|5|890| +2450815|16130|5|215| +2450815|16132|5|407| +2450815|16135|5|366| +2450815|16136|5|108| +2450815|16138|5|357| +2450815|16141|5|353| +2450815|16142|5|740| +2450815|16144|5|798| +2450815|16147|5|765| +2450815|16148|5|968| +2450815|16150|5|435| +2450815|16153|5|1| +2450815|16154|5|869| +2450815|16156|5|316| +2450815|16159|5|529| +2450815|16160|5|710| +2450815|16162|5|720| +2450815|16165|5|913| +2450815|16166|5|956| +2450815|16168|5|188| +2450815|16171|5|144| +2450815|16172|5|558| +2450815|16174|5|237| +2450815|16177|5|283| +2450815|16178|5|27| +2450815|16180|5|687| +2450815|16183|5|878| +2450815|16184|5|763| +2450815|16186|5|780| +2450815|16189|5|| +2450815|16190|5|804| +2450815|16192|5|866| +2450815|16195|5|167| +2450815|16196|5|346| +2450815|16198|5|52| +2450815|16201|5|96| +2450815|16202|5|271| +2450815|16204|5|376| +2450815|16207|5|530| +2450815|16208|5|| +2450815|16210|5|926| +2450815|16213|5|243| +2450815|16214|5|398| +2450815|16216|5|865| +2450815|16219|5|914| +2450815|16220|5|| +2450815|16222|5|133| +2450815|16225|5|510| +2450815|16226|5|587| +2450815|16228|5|941| +2450815|16231|5|335| +2450815|16232|5|370| +2450815|16234|5|820| +2450815|16237|5|694| +2450815|16238|5|464| +2450815|16240|5|853| +2450815|16243|5|37| +2450815|16244|5|457| +2450815|16246|5|753| +2450815|16249|5|77| +2450815|16250|5|221| +2450815|16252|5|880| +2450815|16255|5|103| +2450815|16256|5|503| +2450815|16258|5|938| +2450815|16261|5|658| +2450815|16262|5|806| +2450815|16264|5|971| +2450815|16267|5|97| +2450815|16268|5|95| +2450815|16270|5|932| +2450815|16273|5|490| +2450815|16274|5|921| +2450815|16276|5|316| +2450815|16279|5|499| +2450815|16280|5|563| +2450815|16282|5|840| +2450815|16285|5|977| +2450815|16286|5|294| +2450815|16288|5|604| +2450815|16291|5|92| +2450815|16292|5|41| +2450815|16294|5|172| +2450815|16297|5|| +2450815|16298|5|405| +2450815|16300|5|171| +2450815|16303|5|718| +2450815|16304|5|52| +2450815|16306|5|497| +2450815|16309|5|792| +2450815|16310|5|665| +2450815|16312|5|550| +2450815|16315|5|786| +2450815|16316|5|522| +2450815|16318|5|881| +2450815|16321|5|740| +2450815|16322|5|20| +2450815|16324|5|243| +2450815|16327|5|865| +2450815|16328|5|493| +2450815|16330|5|981| +2450815|16333|5|646| +2450815|16334|5|400| +2450815|16336|5|498| +2450815|16339|5|87| +2450815|16340|5|126| +2450815|16342|5|303| +2450815|16345|5|716| +2450815|16346|5|772| +2450815|16348|5|516| +2450815|16351|5|787| +2450815|16352|5|954| +2450815|16354|5|| +2450815|16357|5|430| +2450815|16358|5|588| +2450815|16360|5|913| +2450815|16363|5|957| +2450815|16364|5|345| +2450815|16366|5|30| +2450815|16369|5|551| +2450815|16370|5|58| +2450815|16372|5|91| +2450815|16375|5|252| +2450815|16376|5|371| +2450815|16378|5|32| +2450815|16381|5|543| +2450815|16382|5|211| +2450815|16384|5|985| +2450815|16387|5|414| +2450815|16388|5|732| +2450815|16390|5|804| +2450815|16393|5|420| +2450815|16394|5|425| +2450815|16396|5|731| +2450815|16399|5|463| +2450815|16400|5|738| +2450815|16402|5|868| +2450815|16405|5|738| +2450815|16406|5|177| +2450815|16408|5|480| +2450815|16411|5|940| +2450815|16412|5|799| +2450815|16414|5|70| +2450815|16417|5|549| +2450815|16418|5|854| +2450815|16420|5|508| +2450815|16423|5|36| +2450815|16424|5|369| +2450815|16426|5|404| +2450815|16429|5|314| +2450815|16430|5|607| +2450815|16432|5|246| +2450815|16435|5|305| +2450815|16436|5|520| +2450815|16438|5|94| +2450815|16441|5|6| +2450815|16442|5|448| +2450815|16444|5|161| +2450815|16447|5|8| +2450815|16448|5|| +2450815|16450|5|126| +2450815|16453|5|409| +2450815|16454|5|644| +2450815|16456|5|87| +2450815|16459|5|363| +2450815|16460|5|164| +2450815|16462|5|734| +2450815|16465|5|688| +2450815|16466|5|424| +2450815|16468|5|8| +2450815|16471|5|429| +2450815|16472|5|660| +2450815|16474|5|758| +2450815|16477|5|211| +2450815|16478|5|215| +2450815|16480|5|205| +2450815|16483|5|873| +2450815|16484|5|797| +2450815|16486|5|789| +2450815|16489|5|257| +2450815|16490|5|210| +2450815|16492|5|578| +2450815|16495|5|583| +2450815|16496|5|518| +2450815|16498|5|504| +2450815|16501|5|56| +2450815|16502|5|221| +2450815|16504|5|683| +2450815|16507|5|895| +2450815|16508|5|149| +2450815|16510|5|447| +2450815|16513|5|628| +2450815|16514|5|275| +2450815|16516|5|773| +2450815|16519|5|163| +2450815|16520|5|369| +2450815|16522|5|183| +2450815|16525|5|207| +2450815|16526|5|605| +2450815|16528|5|248| +2450815|16531|5|748| +2450815|16532|5|| +2450815|16534|5|484| +2450815|16537|5|228| +2450815|16538|5|536| +2450815|16540|5|230| +2450815|16543|5|343| +2450815|16544|5|131| +2450815|16546|5|718| +2450815|16549|5|417| +2450815|16550|5|997| +2450815|16552|5|740| +2450815|16555|5|497| +2450815|16556|5|968| +2450815|16558|5|436| +2450815|16561|5|660| +2450815|16562|5|20| +2450815|16564|5|98| +2450815|16567|5|275| +2450815|16568|5|| +2450815|16570|5|388| +2450815|16573|5|327| +2450815|16574|5|542| +2450815|16576|5|| +2450815|16579|5|978| +2450815|16580|5|860| +2450815|16582|5|813| +2450815|16585|5|479| +2450815|16586|5|990| +2450815|16588|5|245| +2450815|16591|5|319| +2450815|16592|5|665| +2450815|16594|5|724| +2450815|16597|5|819| +2450815|16598|5|258| +2450815|16600|5|906| +2450815|16603|5|339| +2450815|16604|5|312| +2450815|16606|5|28| +2450815|16609|5|558| +2450815|16610|5|878| +2450815|16612|5|2| +2450815|16615|5|699| +2450815|16616|5|774| +2450815|16618|5|624| +2450815|16621|5|901| +2450815|16622|5|220| +2450815|16624|5|5| +2450815|16627|5|305| +2450815|16628|5|13| +2450815|16630|5|264| +2450815|16633|5|661| +2450815|16634|5|417| +2450815|16636|5|227| +2450815|16639|5|378| +2450815|16640|5|514| +2450815|16642|5|510| +2450815|16645|5|569| +2450815|16646|5|426| +2450815|16648|5|751| +2450815|16651|5|80| +2450815|16652|5|| +2450815|16654|5|343| +2450815|16657|5|448| +2450815|16658|5|610| +2450815|16660|5|102| +2450815|16663|5|| +2450815|16664|5|40| +2450815|16666|5|253| +2450815|16669|5|838| +2450815|16670|5|474| +2450815|16672|5|683| +2450815|16675|5|685| +2450815|16676|5|53| +2450815|16678|5|990| +2450815|16681|5|629| +2450815|16682|5|929| +2450815|16684|5|488| +2450815|16687|5|440| +2450815|16688|5|318| +2450815|16690|5|761| +2450815|16693|5|967| +2450815|16694|5|| +2450815|16696|5|891| +2450815|16699|5|442| +2450815|16700|5|936| +2450815|16702|5|575| +2450815|16705|5|397| +2450815|16706|5|69| +2450815|16708|5|30| +2450815|16711|5|412| +2450815|16712|5|416| +2450815|16714|5|177| +2450815|16717|5|306| +2450815|16718|5|4| +2450815|16720|5|523| +2450815|16723|5|584| +2450815|16724|5|516| +2450815|16726|5|288| +2450815|16729|5|225| +2450815|16730|5|216| +2450815|16732|5|774| +2450815|16735|5|57| +2450815|16736|5|853| +2450815|16738|5|207| +2450815|16741|5|865| +2450815|16742|5|541| +2450815|16744|5|580| +2450815|16747|5|693| +2450815|16748|5|943| +2450815|16750|5|183| +2450815|16753|5|584| +2450815|16754|5|283| +2450815|16756|5|870| +2450815|16759|5|727| +2450815|16760|5|497| +2450815|16762|5|991| +2450815|16765|5|618| +2450815|16766|5|865| +2450815|16768|5|813| +2450815|16771|5|516| +2450815|16772|5|978| +2450815|16774|5|348| +2450815|16777|5|684| +2450815|16778|5|622| +2450815|16780|5|585| +2450815|16783|5|643| +2450815|16784|5|413| +2450815|16786|5|992| +2450815|16789|5|463| +2450815|16790|5|270| +2450815|16792|5|5| +2450815|16795|5|458| +2450815|16796|5|574| +2450815|16798|5|897| +2450815|16801|5|520| +2450815|16802|5|977| +2450815|16804|5|16| +2450815|16807|5|682| +2450815|16808|5|944| +2450815|16810|5|607| +2450815|16813|5|56| +2450815|16814|5|697| +2450815|16816|5|| +2450815|16819|5|132| +2450815|16820|5|| +2450815|16822|5|726| +2450815|16825|5|71| +2450815|16826|5|715| +2450815|16828|5|456| +2450815|16831|5|939| +2450815|16832|5|989| +2450815|16834|5|50| +2450815|16837|5|34| +2450815|16838|5|860| +2450815|16840|5|| +2450815|16843|5|967| +2450815|16844|5|| +2450815|16846|5|128| +2450815|16849|5|313| +2450815|16850|5|181| +2450815|16852|5|552| +2450815|16855|5|366| +2450815|16856|5|266| +2450815|16858|5|659| +2450815|16861|5|554| +2450815|16862|5|97| +2450815|16864|5|211| +2450815|16867|5|477| +2450815|16868|5|354| +2450815|16870|5|78| +2450815|16873|5|820| +2450815|16874|5|994| +2450815|16876|5|613| +2450815|16879|5|15| +2450815|16880|5|887| +2450815|16882|5|204| +2450815|16885|5|297| +2450815|16886|5|236| +2450815|16888|5|142| +2450815|16891|5|994| +2450815|16892|5|26| +2450815|16894|5|460| +2450815|16897|5|881| +2450815|16898|5|270| +2450815|16900|5|149| +2450815|16903|5|883| +2450815|16904|5|180| +2450815|16906|5|840| +2450815|16909|5|147| +2450815|16910|5|572| +2450815|16912|5|561| +2450815|16915|5|887| +2450815|16916|5|527| +2450815|16918|5|861| +2450815|16921|5|331| +2450815|16922|5|733| +2450815|16924|5|705| +2450815|16927|5|53| +2450815|16928|5|548| +2450815|16930|5|739| +2450815|16933|5|169| +2450815|16934|5|771| +2450815|16936|5|116| +2450815|16939|5|904| +2450815|16940|5|472| +2450815|16942|5|506| +2450815|16945|5|682| +2450815|16946|5|821| +2450815|16948|5|753| +2450815|16951|5|341| +2450815|16952|5|885| +2450815|16954|5|161| +2450815|16957|5|905| +2450815|16958|5|| +2450815|16960|5|124| +2450815|16963|5|588| +2450815|16964|5|379| +2450815|16966|5|690| +2450815|16969|5|415| +2450815|16970|5|95| +2450815|16972|5|| +2450815|16975|5|997| +2450815|16976|5|809| +2450815|16978|5|305| +2450815|16981|5|763| +2450815|16982|5|996| +2450815|16984|5|931| +2450815|16987|5|| +2450815|16988|5|805| +2450815|16990|5|602| +2450815|16993|5|59| +2450815|16994|5|240| +2450815|16996|5|207| +2450815|16999|5|553| +2450815|17000|5|13| +2450815|17002|5|916| +2450815|17005|5|26| +2450815|17006|5|57| +2450815|17008|5|967| +2450815|17011|5|448| +2450815|17012|5|386| +2450815|17014|5|230| +2450815|17017|5|551| +2450815|17018|5|79| +2450815|17020|5|82| +2450815|17023|5|531| +2450815|17024|5|456| +2450815|17026|5|789| +2450815|17029|5|765| +2450815|17030|5|691| +2450815|17032|5|603| +2450815|17035|5|698| +2450815|17036|5|533| +2450815|17038|5|648| +2450815|17041|5|| +2450815|17042|5|556| +2450815|17044|5|423| +2450815|17047|5|46| +2450815|17048|5|249| +2450815|17050|5|279| +2450815|17053|5|865| +2450815|17054|5|194| +2450815|17056|5|89| +2450815|17059|5|95| +2450815|17060|5|442| +2450815|17062|5|498| +2450815|17065|5|743| +2450815|17066|5|15| +2450815|17068|5|490| +2450815|17071|5|920| +2450815|17072|5|85| +2450815|17074|5|818| +2450815|17077|5|227| +2450815|17078|5|64| +2450815|17080|5|511| +2450815|17083|5|168| +2450815|17084|5|814| +2450815|17086|5|56| +2450815|17089|5|141| +2450815|17090|5|718| +2450815|17092|5|151| +2450815|17095|5|986| +2450815|17096|5|379| +2450815|17098|5|176| +2450815|17101|5|835| +2450815|17102|5|950| +2450815|17104|5|929| +2450815|17107|5|99| +2450815|17108|5|89| +2450815|17110|5|187| +2450815|17113|5|763| +2450815|17114|5|422| +2450815|17116|5|349| +2450815|17119|5|661| +2450815|17120|5|109| +2450815|17122|5|| +2450815|17125|5|213| +2450815|17126|5|507| +2450815|17128|5|| +2450815|17131|5|99| +2450815|17132|5|189| +2450815|17134|5|577| +2450815|17137|5|145| +2450815|17138|5|439| +2450815|17140|5|221| +2450815|17143|5|958| +2450815|17144|5|70| +2450815|17146|5|374| +2450815|17149|5|263| +2450815|17150|5|217| +2450815|17152|5|| +2450815|17155|5|434| +2450815|17156|5|112| +2450815|17158|5|946| +2450815|17161|5|811| +2450815|17162|5|69| +2450815|17164|5|562| +2450815|17167|5|832| +2450815|17168|5|903| +2450815|17170|5|3| +2450815|17173|5|229| +2450815|17174|5|203| +2450815|17176|5|765| +2450815|17179|5|231| +2450815|17180|5|253| +2450815|17182|5|718| +2450815|17185|5|970| +2450815|17186|5|762| +2450815|17188|5|687| +2450815|17191|5|813| +2450815|17192|5|37| +2450815|17194|5|113| +2450815|17197|5|546| +2450815|17198|5|936| +2450815|17200|5|376| +2450815|17203|5|742| +2450815|17204|5|636| +2450815|17206|5|549| +2450815|17209|5|419| +2450815|17210|5|610| +2450815|17212|5|| +2450815|17215|5|297| +2450815|17216|5|921| +2450815|17218|5|713| +2450815|17221|5|909| +2450815|17222|5|646| +2450815|17224|5|123| +2450815|17227|5|| +2450815|17228|5|45| +2450815|17230|5|874| +2450815|17233|5|405| +2450815|17234|5|| +2450815|17236|5|763| +2450815|17239|5|304| +2450815|17240|5|362| +2450815|17242|5|| +2450815|17245|5|863| +2450815|17246|5|316| +2450815|17248|5|800| +2450815|17251|5|660| +2450815|17252|5|92| +2450815|17254|5|284| +2450815|17257|5|110| +2450815|17258|5|546| +2450815|17260|5|871| +2450815|17263|5|374| +2450815|17264|5|348| +2450815|17266|5|725| +2450815|17269|5|75| +2450815|17270|5|889| +2450815|17272|5|623| +2450815|17275|5|841| +2450815|17276|5|679| +2450815|17278|5|288| +2450815|17281|5|62| +2450815|17282|5|934| +2450815|17284|5|868| +2450815|17287|5|642| +2450815|17288|5|630| +2450815|17290|5|873| +2450815|17293|5|73| +2450815|17294|5|40| +2450815|17296|5|160| +2450815|17299|5|849| +2450815|17300|5|588| +2450815|17302|5|480| +2450815|17305|5|885| +2450815|17306|5|| +2450815|17308|5|361| +2450815|17311|5|637| +2450815|17312|5|123| +2450815|17314|5|242| +2450815|17317|5|309| +2450815|17318|5|329| +2450815|17320|5|473| +2450815|17323|5|863| +2450815|17324|5|668| +2450815|17326|5|594| +2450815|17329|5|2| +2450815|17330|5|572| +2450815|17332|5|| +2450815|17335|5|325| +2450815|17336|5|345| +2450815|17338|5|565| +2450815|17341|5|44| +2450815|17342|5|161| +2450815|17344|5|921| +2450815|17347|5|903| +2450815|17348|5|401| +2450815|17350|5|118| +2450815|17353|5|913| +2450815|17354|5|863| +2450815|17356|5|837| +2450815|17359|5|56| +2450815|17360|5|374| +2450815|17362|5|404| +2450815|17365|5|781| +2450815|17366|5|599| +2450815|17368|5|346| +2450815|17371|5|99| +2450815|17372|5|498| +2450815|17374|5|308| +2450815|17377|5|736| +2450815|17378|5|988| +2450815|17380|5|667| +2450815|17383|5|355| +2450815|17384|5|447| +2450815|17386|5|88| +2450815|17389|5|691| +2450815|17390|5|223| +2450815|17392|5|821| +2450815|17395|5|451| +2450815|17396|5|759| +2450815|17398|5|152| +2450815|17401|5|499| +2450815|17402|5|655| +2450815|17404|5|499| +2450815|17407|5|20| +2450815|17408|5|431| +2450815|17410|5|449| +2450815|17413|5|940| +2450815|17414|5|322| +2450815|17416|5|896| +2450815|17419|5|779| +2450815|17420|5|511| +2450815|17422|5|758| +2450815|17425|5|811| +2450815|17426|5|342| +2450815|17428|5|791| +2450815|17431|5|905| +2450815|17432|5|401| +2450815|17434|5|812| +2450815|17437|5|318| +2450815|17438|5|262| +2450815|17440|5|851| +2450815|17443|5|730| +2450815|17444|5|25| +2450815|17446|5|87| +2450815|17449|5|305| +2450815|17450|5|390| +2450815|17452|5|| +2450815|17455|5|205| +2450815|17456|5|505| +2450815|17458|5|78| +2450815|17461|5|521| +2450815|17462|5|935| +2450815|17464|5|710| +2450815|17467|5|843| +2450815|17468|5|473| +2450815|17470|5|183| +2450815|17473|5|719| +2450815|17474|5|834| +2450815|17476|5|527| +2450815|17479|5|8| +2450815|17480|5|26| +2450815|17482|5|745| +2450815|17485|5|413| +2450815|17486|5|942| +2450815|17488|5|576| +2450815|17491|5|82| +2450815|17492|5|993| +2450815|17494|5|727| +2450815|17497|5|183| +2450815|17498|5|862| +2450815|17500|5|184| +2450815|17503|5|515| +2450815|17504|5|983| +2450815|17506|5|23| +2450815|17509|5|88| +2450815|17510|5|681| +2450815|17512|5|890| +2450815|17515|5|668| +2450815|17516|5|257| +2450815|17518|5|121| +2450815|17521|5|514| +2450815|17522|5|517| +2450815|17524|5|| +2450815|17527|5|397| +2450815|17528|5|748| +2450815|17530|5|724| +2450815|17533|5|993| +2450815|17534|5|942| +2450815|17536|5|340| +2450815|17539|5|858| +2450815|17540|5|46| +2450815|17542|5|992| +2450815|17545|5|653| +2450815|17546|5|214| +2450815|17548|5|945| +2450815|17551|5|846| +2450815|17552|5|886| +2450815|17554|5|926| +2450815|17557|5|488| +2450815|17558|5|301| +2450815|17560|5|787| +2450815|17563|5|552| +2450815|17564|5|901| +2450815|17566|5|669| +2450815|17569|5|643| +2450815|17570|5|259| +2450815|17572|5|116| +2450815|17575|5|720| +2450815|17576|5|927| +2450815|17578|5|526| +2450815|17581|5|499| +2450815|17582|5|837| +2450815|17584|5|821| +2450815|17587|5|40| +2450815|17588|5|106| +2450815|17590|5|104| +2450815|17593|5|197| +2450815|17594|5|424| +2450815|17596|5|323| +2450815|17599|5|345| +2450815|17600|5|190| +2450815|17602|5|192| +2450815|17605|5|271| +2450815|17606|5|390| +2450815|17608|5|709| +2450815|17611|5|504| +2450815|17612|5|106| +2450815|17614|5|556| +2450815|17617|5|216| +2450815|17618|5|375| +2450815|17620|5|877| +2450815|17623|5|603| +2450815|17624|5|395| +2450815|17626|5|159| +2450815|17629|5|542| +2450815|17630|5|232| +2450815|17632|5|| +2450815|17635|5|816| +2450815|17636|5|626| +2450815|17638|5|376| +2450815|17641|5|573| +2450815|17642|5|449| +2450815|17644|5|553| +2450815|17647|5|436| +2450815|17648|5|824| +2450815|17650|5|| +2450815|17653|5|| +2450815|17654|5|237| +2450815|17656|5|931| +2450815|17659|5|791| +2450815|17660|5|937| +2450815|17662|5|435| +2450815|17665|5|216| +2450815|17666|5|818| +2450815|17668|5|624| +2450815|17671|5|20| +2450815|17672|5|535| +2450815|17674|5|654| +2450815|17677|5|746| +2450815|17678|5|459| +2450815|17680|5|113| +2450815|17683|5|588| +2450815|17684|5|894| +2450815|17686|5|| +2450815|17689|5|585| +2450815|17690|5|19| +2450815|17692|5|698| +2450815|17695|5|902| +2450815|17696|5|342| +2450815|17698|5|83| +2450815|17701|5|54| +2450815|17702|5|384| +2450815|17704|5|254| +2450815|17707|5|| +2450815|17708|5|| +2450815|17710|5|904| +2450815|17713|5|| +2450815|17714|5|735| +2450815|17716|5|310| +2450815|17719|5|756| +2450815|17720|5|603| +2450815|17722|5|435| +2450815|17725|5|83| +2450815|17726|5|869| +2450815|17728|5|731| +2450815|17731|5|| +2450815|17732|5|269| +2450815|17734|5|150| +2450815|17737|5|434| +2450815|17738|5|592| +2450815|17740|5|253| +2450815|17743|5|915| +2450815|17744|5|426| +2450815|17746|5|761| +2450815|17749|5|925| +2450815|17750|5|925| +2450815|17752|5|451| +2450815|17755|5|530| +2450815|17756|5|93| +2450815|17758|5|890| +2450815|17761|5|844| +2450815|17762|5|530| +2450815|17764|5|107| +2450815|17767|5|624| +2450815|17768|5|311| +2450815|17770|5|525| +2450815|17773|5|258| +2450815|17774|5|384| +2450815|17776|5|633| +2450815|17779|5|630| +2450815|17780|5|162| +2450815|17782|5|749| +2450815|17785|5|342| +2450815|17786|5|100| +2450815|17788|5|580| +2450815|17791|5|717| +2450815|17792|5|116| +2450815|17794|5|446| +2450815|17797|5|127| +2450815|17798|5|943| +2450815|17800|5|411| +2450815|17803|5|183| +2450815|17804|5|887| +2450815|17806|5|878| +2450815|17809|5|| +2450815|17810|5|460| +2450815|17812|5|114| +2450815|17815|5|298| +2450815|17816|5|343| +2450815|17818|5|397| +2450815|17821|5|394| +2450815|17822|5|728| +2450815|17824|5|621| +2450815|17827|5|123| +2450815|17828|5|46| +2450815|17830|5|976| +2450815|17833|5|| +2450815|17834|5|531| +2450815|17836|5|935| +2450815|17839|5|371| +2450815|17840|5|234| +2450815|17842|5|762| +2450815|17845|5|591| +2450815|17846|5|208| +2450815|17848|5|661| +2450815|17851|5|129| +2450815|17852|5|794| +2450815|17854|5|195| +2450815|17857|5|207| +2450815|17858|5|730| +2450815|17860|5|868| +2450815|17863|5|506| +2450815|17864|5|826| +2450815|17866|5|826| +2450815|17869|5|388| +2450815|17870|5|185| +2450815|17872|5|102| +2450815|17875|5|194| +2450815|17876|5|489| +2450815|17878|5|976| +2450815|17881|5|686| +2450815|17882|5|549| +2450815|17884|5|806| +2450815|17887|5|837| +2450815|17888|5|920| +2450815|17890|5|602| +2450815|17893|5|| +2450815|17894|5|576| +2450815|17896|5|153| +2450815|17899|5|270| +2450815|17900|5|508| +2450815|17902|5|| +2450815|17905|5|78| +2450815|17906|5|623| +2450815|17908|5|524| +2450815|17911|5|279| +2450815|17912|5|299| +2450815|17914|5|| +2450815|17917|5|341| +2450815|17918|5|22| +2450815|17920|5|240| +2450815|17923|5|711| +2450815|17924|5|158| +2450815|17926|5|389| +2450815|17929|5|996| +2450815|17930|5|884| +2450815|17932|5|139| +2450815|17935|5|506| +2450815|17936|5|897| +2450815|17938|5|483| +2450815|17941|5|832| +2450815|17942|5|743| +2450815|17944|5|387| +2450815|17947|5|621| +2450815|17948|5|479| +2450815|17950|5|820| +2450815|17953|5|| +2450815|17954|5|464| +2450815|17956|5|154| +2450815|17959|5|577| +2450815|17960|5|559| +2450815|17962|5|942| +2450815|17965|5|976| +2450815|17966|5|524| +2450815|17968|5|582| +2450815|17971|5|443| +2450815|17972|5|584| +2450815|17974|5|331| +2450815|17977|5|634| +2450815|17978|5|293| +2450815|17980|5|590| +2450815|17983|5|255| +2450815|17984|5|83| +2450815|17986|5|637| +2450815|17989|5|661| +2450815|17990|5|89| +2450815|17992|5|555| +2450815|17995|5|847| +2450815|17996|5|69| +2450815|17998|5|493| +2450822|1|1|522| +2450822|2|1|416| +2450822|4|1|491| +2450822|7|1|215| +2450822|8|1|735| +2450822|10|1|799| +2450822|13|1|776| +2450822|14|1|444| +2450822|16|1|262| +2450822|19|1|| +2450822|20|1|952| +2450822|22|1|107| +2450822|25|1|| +2450822|26|1|336| +2450822|28|1|419| +2450822|31|1|| +2450822|32|1|647| +2450822|34|1|896| +2450822|37|1|90| +2450822|38|1|998| +2450822|40|1|995| +2450822|43|1|981| +2450822|44|1|502| +2450822|46|1|178| +2450822|49|1|808| +2450822|50|1|180| +2450822|52|1|846| +2450822|55|1|260| +2450822|56|1|618| +2450822|58|1|613| +2450822|61|1|498| +2450822|62|1|66| +2450822|64|1|908| +2450822|67|1|612| +2450822|68|1|584| +2450822|70|1|378| +2450822|73|1|179| +2450822|74|1|448| +2450822|76|1|443| +2450822|79|1|13| +2450822|80|1|249| +2450822|82|1|710| +2450822|85|1|28| +2450822|86|1|762| +2450822|88|1|52| +2450822|91|1|759| +2450822|92|1|483| +2450822|94|1|174| +2450822|97|1|| +2450822|98|1|625| +2450822|100|1|505| +2450822|103|1|18| +2450822|104|1|435| +2450822|106|1|381| +2450822|109|1|525| +2450822|110|1|226| +2450822|112|1|136| +2450822|115|1|725| +2450822|116|1|690| +2450822|118|1|468| +2450822|121|1|836| +2450822|122|1|478| +2450822|124|1|638| +2450822|127|1|| +2450822|128|1|374| +2450822|130|1|895| +2450822|133|1|594| +2450822|134|1|660| +2450822|136|1|345| +2450822|139|1|385| +2450822|140|1|177| +2450822|142|1|976| +2450822|145|1|977| +2450822|146|1|53| +2450822|148|1|903| +2450822|151|1|488| +2450822|152|1|349| +2450822|154|1|498| +2450822|157|1|119| +2450822|158|1|315| +2450822|160|1|956| +2450822|163|1|298| +2450822|164|1|716| +2450822|166|1|999| +2450822|169|1|851| +2450822|170|1|977| +2450822|172|1|929| +2450822|175|1|859| +2450822|176|1|545| +2450822|178|1|727| +2450822|181|1|42| +2450822|182|1|909| +2450822|184|1|394| +2450822|187|1|764| +2450822|188|1|815| +2450822|190|1|605| +2450822|193|1|916| +2450822|194|1|| +2450822|196|1|887| +2450822|199|1|5| +2450822|200|1|289| +2450822|202|1|330| +2450822|205|1|237| +2450822|206|1|379| +2450822|208|1|| +2450822|211|1|715| +2450822|212|1|782| +2450822|214|1|599| +2450822|217|1|870| +2450822|218|1|365| +2450822|220|1|303| +2450822|223|1|853| +2450822|224|1|479| +2450822|226|1|142| +2450822|229|1|391| +2450822|230|1|870| +2450822|232|1|546| +2450822|235|1|965| +2450822|236|1|394| +2450822|238|1|700| +2450822|241|1|591| +2450822|242|1|47| +2450822|244|1|357| +2450822|247|1|| +2450822|248|1|275| +2450822|250|1|967| +2450822|253|1|214| +2450822|254|1|104| +2450822|256|1|603| +2450822|259|1|| +2450822|260|1|616| +2450822|262|1|481| +2450822|265|1|666| +2450822|266|1|643| +2450822|268|1|800| +2450822|271|1|135| +2450822|272|1|804| +2450822|274|1|107| +2450822|277|1|130| +2450822|278|1|254| +2450822|280|1|765| +2450822|283|1|544| +2450822|284|1|894| +2450822|286|1|397| +2450822|289|1|73| +2450822|290|1|925| +2450822|292|1|360| +2450822|295|1|741| +2450822|296|1|964| +2450822|298|1|652| +2450822|301|1|422| +2450822|302|1|588| +2450822|304|1|44| +2450822|307|1|196| +2450822|308|1|569| +2450822|310|1|408| +2450822|313|1|476| +2450822|314|1|227| +2450822|316|1|172| +2450822|319|1|180| +2450822|320|1|| +2450822|322|1|49| +2450822|325|1|167| +2450822|326|1|946| +2450822|328|1|112| +2450822|331|1|384| +2450822|332|1|30| +2450822|334|1|991| +2450822|337|1|186| +2450822|338|1|199| +2450822|340|1|875| +2450822|343|1|95| +2450822|344|1|609| +2450822|346|1|| +2450822|349|1|99| +2450822|350|1|755| +2450822|352|1|776| +2450822|355|1|615| +2450822|356|1|250| +2450822|358|1|60| +2450822|361|1|551| +2450822|362|1|823| +2450822|364|1|893| +2450822|367|1|487| +2450822|368|1|633| +2450822|370|1|101| +2450822|373|1|667| +2450822|374|1|507| +2450822|376|1|127| +2450822|379|1|178| +2450822|380|1|| +2450822|382|1|461| +2450822|385|1|287| +2450822|386|1|484| +2450822|388|1|158| +2450822|391|1|525| +2450822|392|1|189| +2450822|394|1|701| +2450822|397|1|96| +2450822|398|1|555| +2450822|400|1|30| +2450822|403|1|427| +2450822|404|1|660| +2450822|406|1|46| +2450822|409|1|848| +2450822|410|1|435| +2450822|412|1|39| +2450822|415|1|393| +2450822|416|1|373| +2450822|418|1|216| +2450822|421|1|803| +2450822|422|1|467| +2450822|424|1|883| +2450822|427|1|369| +2450822|428|1|456| +2450822|430|1|731| +2450822|433|1|133| +2450822|434|1|674| +2450822|436|1|593| +2450822|439|1|788| +2450822|440|1|227| +2450822|442|1|667| +2450822|445|1|869| +2450822|446|1|257| +2450822|448|1|825| +2450822|451|1|624| +2450822|452|1|205| +2450822|454|1|255| +2450822|457|1|377| +2450822|458|1|962| +2450822|460|1|903| +2450822|463|1|174| +2450822|464|1|975| +2450822|466|1|412| +2450822|469|1|300| +2450822|470|1|901| +2450822|472|1|27| +2450822|475|1|380| +2450822|476|1|792| +2450822|478|1|387| +2450822|481|1|998| +2450822|482|1|789| +2450822|484|1|643| +2450822|487|1|167| +2450822|488|1|789| +2450822|490|1|819| +2450822|493|1|479| +2450822|494|1|998| +2450822|496|1|919| +2450822|499|1|37| +2450822|500|1|551| +2450822|502|1|78| +2450822|505|1|265| +2450822|506|1|467| +2450822|508|1|803| +2450822|511|1|639| +2450822|512|1|769| +2450822|514|1|792| +2450822|517|1|98| +2450822|518|1|46| +2450822|520|1|254| +2450822|523|1|993| +2450822|524|1|507| +2450822|526|1|534| +2450822|529|1|422| +2450822|530|1|914| +2450822|532|1|126| +2450822|535|1|375| +2450822|536|1|964| +2450822|538|1|547| +2450822|541|1|459| +2450822|542|1|841| +2450822|544|1|311| +2450822|547|1|179| +2450822|548|1|198| +2450822|550|1|835| +2450822|553|1|868| +2450822|554|1|362| +2450822|556|1|689| +2450822|559|1|982| +2450822|560|1|| +2450822|562|1|150| +2450822|565|1|747| +2450822|566|1|343| +2450822|568|1|798| +2450822|571|1|157| +2450822|572|1|421| +2450822|574|1|177| +2450822|577|1|941| +2450822|578|1|861| +2450822|580|1|918| +2450822|583|1|| +2450822|584|1|462| +2450822|586|1|399| +2450822|589|1|347| +2450822|590|1|944| +2450822|592|1|106| +2450822|595|1|718| +2450822|596|1|165| +2450822|598|1|471| +2450822|601|1|440| +2450822|602|1|731| +2450822|604|1|248| +2450822|607|1|901| +2450822|608|1|639| +2450822|610|1|200| +2450822|613|1|171| +2450822|614|1|91| +2450822|616|1|314| +2450822|619|1|310| +2450822|620|1|20| +2450822|622|1|989| +2450822|625|1|162| +2450822|626|1|504| +2450822|628|1|214| +2450822|631|1|624| +2450822|632|1|222| +2450822|634|1|531| +2450822|637|1|808| +2450822|638|1|217| +2450822|640|1|568| +2450822|643|1|90| +2450822|644|1|502| +2450822|646|1|168| +2450822|649|1|513| +2450822|650|1|541| +2450822|652|1|215| +2450822|655|1|130| +2450822|656|1|275| +2450822|658|1|642| +2450822|661|1|992| +2450822|662|1|949| +2450822|664|1|396| +2450822|667|1|849| +2450822|668|1|681| +2450822|670|1|743| +2450822|673|1|904| +2450822|674|1|593| +2450822|676|1|218| +2450822|679|1|556| +2450822|680|1|425| +2450822|682|1|| +2450822|685|1|311| +2450822|686|1|534| +2450822|688|1|657| +2450822|691|1|243| +2450822|692|1|191| +2450822|694|1|| +2450822|697|1|458| +2450822|698|1|320| +2450822|700|1|374| +2450822|703|1|98| +2450822|704|1|931| +2450822|706|1|161| +2450822|709|1|665| +2450822|710|1|23| +2450822|712|1|101| +2450822|715|1|474| +2450822|716|1|41| +2450822|718|1|388| +2450822|721|1|108| +2450822|722|1|956| +2450822|724|1|860| +2450822|727|1|437| +2450822|728|1|924| +2450822|730|1|299| +2450822|733|1|478| +2450822|734|1|258| +2450822|736|1|794| +2450822|739|1|4| +2450822|740|1|| +2450822|742|1|198| +2450822|745|1|363| +2450822|746|1|122| +2450822|748|1|573| +2450822|751|1|733| +2450822|752|1|585| +2450822|754|1|392| +2450822|757|1|909| +2450822|758|1|241| +2450822|760|1|| +2450822|763|1|428| +2450822|764|1|607| +2450822|766|1|690| +2450822|769|1|| +2450822|770|1|829| +2450822|772|1|778| +2450822|775|1|689| +2450822|776|1|488| +2450822|778|1|794| +2450822|781|1|441| +2450822|782|1|501| +2450822|784|1|333| +2450822|787|1|215| +2450822|788|1|850| +2450822|790|1|405| +2450822|793|1|687| +2450822|794|1|798| +2450822|796|1|79| +2450822|799|1|775| +2450822|800|1|197| +2450822|802|1|932| +2450822|805|1|837| +2450822|806|1|163| +2450822|808|1|933| +2450822|811|1|254| +2450822|812|1|560| +2450822|814|1|893| +2450822|817|1|504| +2450822|818|1|30| +2450822|820|1|570| +2450822|823|1|123| +2450822|824|1|796| +2450822|826|1|158| +2450822|829|1|917| +2450822|830|1|521| +2450822|832|1|992| +2450822|835|1|166| +2450822|836|1|143| +2450822|838|1|| +2450822|841|1|530| +2450822|842|1|72| +2450822|844|1|949| +2450822|847|1|| +2450822|848|1|598| +2450822|850|1|449| +2450822|853|1|| +2450822|854|1|841| +2450822|856|1|954| +2450822|859|1|838| +2450822|860|1|565| +2450822|862|1|747| +2450822|865|1|| +2450822|866|1|811| +2450822|868|1|657| +2450822|871|1|155| +2450822|872|1|46| +2450822|874|1|122| +2450822|877|1|621| +2450822|878|1|180| +2450822|880|1|366| +2450822|883|1|566| +2450822|884|1|241| +2450822|886|1|342| +2450822|889|1|642| +2450822|890|1|82| +2450822|892|1|235| +2450822|895|1|21| +2450822|896|1|127| +2450822|898|1|320| +2450822|901|1|957| +2450822|902|1|566| +2450822|904|1|279| +2450822|907|1|477| +2450822|908|1|104| +2450822|910|1|580| +2450822|913|1|735| +2450822|914|1|388| +2450822|916|1|333| +2450822|919|1|704| +2450822|920|1|489| +2450822|922|1|449| +2450822|925|1|800| +2450822|926|1|213| +2450822|928|1|| +2450822|931|1|11| +2450822|932|1|768| +2450822|934|1|| +2450822|937|1|228| +2450822|938|1|304| +2450822|940|1|863| +2450822|943|1|63| +2450822|944|1|696| +2450822|946|1|323| +2450822|949|1|391| +2450822|950|1|243| +2450822|952|1|370| +2450822|955|1|| +2450822|956|1|59| +2450822|958|1|159| +2450822|961|1|736| +2450822|962|1|647| +2450822|964|1|396| +2450822|967|1|31| +2450822|968|1|999| +2450822|970|1|230| +2450822|973|1|726| +2450822|974|1|721| +2450822|976|1|870| +2450822|979|1|753| +2450822|980|1|691| +2450822|982|1|385| +2450822|985|1|| +2450822|986|1|886| +2450822|988|1|798| +2450822|991|1|953| +2450822|992|1|514| +2450822|994|1|607| +2450822|997|1|641| +2450822|998|1|159| +2450822|1000|1|144| +2450822|1003|1|488| +2450822|1004|1|958| +2450822|1006|1|163| +2450822|1009|1|272| +2450822|1010|1|621| +2450822|1012|1|647| +2450822|1015|1|394| +2450822|1016|1|| +2450822|1018|1|502| +2450822|1021|1|598| +2450822|1022|1|234| +2450822|1024|1|| +2450822|1027|1|726| +2450822|1028|1|341| +2450822|1030|1|| +2450822|1033|1|952| +2450822|1034|1|831| +2450822|1036|1|| +2450822|1039|1|789| +2450822|1040|1|522| +2450822|1042|1|383| +2450822|1045|1|324| +2450822|1046|1|69| +2450822|1048|1|518| +2450822|1051|1|147| +2450822|1052|1|| +2450822|1054|1|654| +2450822|1057|1|238| +2450822|1058|1|780| +2450822|1060|1|555| +2450822|1063|1|875| +2450822|1064|1|558| +2450822|1066|1|373| +2450822|1069|1|695| +2450822|1070|1|167| +2450822|1072|1|747| +2450822|1075|1|386| +2450822|1076|1|680| +2450822|1078|1|179| +2450822|1081|1|186| +2450822|1082|1|813| +2450822|1084|1|241| +2450822|1087|1|759| +2450822|1088|1|387| +2450822|1090|1|434| +2450822|1093|1|486| +2450822|1094|1|238| +2450822|1096|1|708| +2450822|1099|1|945| +2450822|1100|1|163| +2450822|1102|1|161| +2450822|1105|1|777| +2450822|1106|1|906| +2450822|1108|1|89| +2450822|1111|1|540| +2450822|1112|1|347| +2450822|1114|1|589| +2450822|1117|1|235| +2450822|1118|1|85| +2450822|1120|1|586| +2450822|1123|1|650| +2450822|1124|1|786| +2450822|1126|1|856| +2450822|1129|1|658| +2450822|1130|1|455| +2450822|1132|1|213| +2450822|1135|1|808| +2450822|1136|1|606| +2450822|1138|1|226| +2450822|1141|1|746| +2450822|1142|1|378| +2450822|1144|1|119| +2450822|1147|1|264| +2450822|1148|1|129| +2450822|1150|1|822| +2450822|1153|1|632| +2450822|1154|1|685| +2450822|1156|1|691| +2450822|1159|1|918| +2450822|1160|1|717| +2450822|1162|1|499| +2450822|1165|1|304| +2450822|1166|1|119| +2450822|1168|1|513| +2450822|1171|1|468| +2450822|1172|1|773| +2450822|1174|1|620| +2450822|1177|1|423| +2450822|1178|1|285| +2450822|1180|1|366| +2450822|1183|1|704| +2450822|1184|1|182| +2450822|1186|1|783| +2450822|1189|1|715| +2450822|1190|1|42| +2450822|1192|1|890| +2450822|1195|1|602| +2450822|1196|1|974| +2450822|1198|1|272| +2450822|1201|1|139| +2450822|1202|1|440| +2450822|1204|1|295| +2450822|1207|1|779| +2450822|1208|1|153| +2450822|1210|1|770| +2450822|1213|1|986| +2450822|1214|1|585| +2450822|1216|1|992| +2450822|1219|1|326| +2450822|1220|1|411| +2450822|1222|1|113| +2450822|1225|1|485| +2450822|1226|1|| +2450822|1228|1|673| +2450822|1231|1|576| +2450822|1232|1|950| +2450822|1234|1|538| +2450822|1237|1|654| +2450822|1238|1|| +2450822|1240|1|941| +2450822|1243|1|493| +2450822|1244|1|880| +2450822|1246|1|682| +2450822|1249|1|464| +2450822|1250|1|633| +2450822|1252|1|816| +2450822|1255|1|179| +2450822|1256|1|929| +2450822|1258|1|660| +2450822|1261|1|940| +2450822|1262|1|980| +2450822|1264|1|495| +2450822|1267|1|832| +2450822|1268|1|947| +2450822|1270|1|680| +2450822|1273|1|325| +2450822|1274|1|895| +2450822|1276|1|513| +2450822|1279|1|867| +2450822|1280|1|920| +2450822|1282|1|676| +2450822|1285|1|885| +2450822|1286|1|765| +2450822|1288|1|760| +2450822|1291|1|224| +2450822|1292|1|815| +2450822|1294|1|939| +2450822|1297|1|981| +2450822|1298|1|32| +2450822|1300|1|945| +2450822|1303|1|766| +2450822|1304|1|126| +2450822|1306|1|932| +2450822|1309|1|24| +2450822|1310|1|105| +2450822|1312|1|286| +2450822|1315|1|410| +2450822|1316|1|122| +2450822|1318|1|565| +2450822|1321|1|662| +2450822|1322|1|363| +2450822|1324|1|10| +2450822|1327|1|503| +2450822|1328|1|783| +2450822|1330|1|119| +2450822|1333|1|138| +2450822|1334|1|201| +2450822|1336|1|950| +2450822|1339|1|161| +2450822|1340|1|615| +2450822|1342|1|693| +2450822|1345|1|180| +2450822|1346|1|| +2450822|1348|1|60| +2450822|1351|1|4| +2450822|1352|1|565| +2450822|1354|1|928| +2450822|1357|1|965| +2450822|1358|1|573| +2450822|1360|1|| +2450822|1363|1|726| +2450822|1364|1|176| +2450822|1366|1|348| +2450822|1369|1|105| +2450822|1370|1|613| +2450822|1372|1|354| +2450822|1375|1|450| +2450822|1376|1|675| +2450822|1378|1|790| +2450822|1381|1|187| +2450822|1382|1|| +2450822|1384|1|189| +2450822|1387|1|253| +2450822|1388|1|260| +2450822|1390|1|704| +2450822|1393|1|47| +2450822|1394|1|444| +2450822|1396|1|847| +2450822|1399|1|527| +2450822|1400|1|242| +2450822|1402|1|138| +2450822|1405|1|639| +2450822|1406|1|601| +2450822|1408|1|813| +2450822|1411|1|28| +2450822|1412|1|20| +2450822|1414|1|299| +2450822|1417|1|543| +2450822|1418|1|809| +2450822|1420|1|439| +2450822|1423|1|967| +2450822|1424|1|625| +2450822|1426|1|| +2450822|1429|1|167| +2450822|1430|1|719| +2450822|1432|1|985| +2450822|1435|1|254| +2450822|1436|1|651| +2450822|1438|1|245| +2450822|1441|1|| +2450822|1442|1|426| +2450822|1444|1|316| +2450822|1447|1|725| +2450822|1448|1|| +2450822|1450|1|35| +2450822|1453|1|| +2450822|1454|1|282| +2450822|1456|1|915| +2450822|1459|1|12| +2450822|1460|1|| +2450822|1462|1|337| +2450822|1465|1|497| +2450822|1466|1|987| +2450822|1468|1|959| +2450822|1471|1|955| +2450822|1472|1|595| +2450822|1474|1|123| +2450822|1477|1|8| +2450822|1478|1|745| +2450822|1480|1|872| +2450822|1483|1|653| +2450822|1484|1|716| +2450822|1486|1|875| +2450822|1489|1|| +2450822|1490|1|425| +2450822|1492|1|270| +2450822|1495|1|806| +2450822|1496|1|281| +2450822|1498|1|794| +2450822|1501|1|945| +2450822|1502|1|611| +2450822|1504|1|882| +2450822|1507|1|709| +2450822|1508|1|756| +2450822|1510|1|| +2450822|1513|1|912| +2450822|1514|1|87| +2450822|1516|1|93| +2450822|1519|1|80| +2450822|1520|1|| +2450822|1522|1|120| +2450822|1525|1|862| +2450822|1526|1|689| +2450822|1528|1|218| +2450822|1531|1|696| +2450822|1532|1|876| +2450822|1534|1|598| +2450822|1537|1|437| +2450822|1538|1|821| +2450822|1540|1|| +2450822|1543|1|| +2450822|1544|1|914| +2450822|1546|1|650| +2450822|1549|1|602| +2450822|1550|1|670| +2450822|1552|1|179| +2450822|1555|1|581| +2450822|1556|1|828| +2450822|1558|1|514| +2450822|1561|1|345| +2450822|1562|1|716| +2450822|1564|1|| +2450822|1567|1|376| +2450822|1568|1|24| +2450822|1570|1|396| +2450822|1573|1|423| +2450822|1574|1|939| +2450822|1576|1|578| +2450822|1579|1|978| +2450822|1580|1|40| +2450822|1582|1|213| +2450822|1585|1|813| +2450822|1586|1|241| +2450822|1588|1|292| +2450822|1591|1|791| +2450822|1592|1|117| +2450822|1594|1|380| +2450822|1597|1|259| +2450822|1598|1|115| +2450822|1600|1|447| +2450822|1603|1|881| +2450822|1604|1|227| +2450822|1606|1|125| +2450822|1609|1|538| +2450822|1610|1|161| +2450822|1612|1|351| +2450822|1615|1|807| +2450822|1616|1|671| +2450822|1618|1|930| +2450822|1621|1|534| +2450822|1622|1|163| +2450822|1624|1|868| +2450822|1627|1|26| +2450822|1628|1|930| +2450822|1630|1|188| +2450822|1633|1|815| +2450822|1634|1|752| +2450822|1636|1|364| +2450822|1639|1|287| +2450822|1640|1|158| +2450822|1642|1|145| +2450822|1645|1|518| +2450822|1646|1|927| +2450822|1648|1|298| +2450822|1651|1|414| +2450822|1652|1|620| +2450822|1654|1|557| +2450822|1657|1|548| +2450822|1658|1|704| +2450822|1660|1|538| +2450822|1663|1|316| +2450822|1664|1|| +2450822|1666|1|934| +2450822|1669|1|983| +2450822|1670|1|458| +2450822|1672|1|795| +2450822|1675|1|147| +2450822|1676|1|597| +2450822|1678|1|304| +2450822|1681|1|365| +2450822|1682|1|588| +2450822|1684|1|381| +2450822|1687|1|76| +2450822|1688|1|131| +2450822|1690|1|565| +2450822|1693|1|304| +2450822|1694|1|859| +2450822|1696|1|781| +2450822|1699|1|| +2450822|1700|1|462| +2450822|1702|1|862| +2450822|1705|1|954| +2450822|1706|1|144| +2450822|1708|1|389| +2450822|1711|1|552| +2450822|1712|1|526| +2450822|1714|1|184| +2450822|1717|1|621| +2450822|1718|1|393| +2450822|1720|1|683| +2450822|1723|1|611| +2450822|1724|1|754| +2450822|1726|1|48| +2450822|1729|1|97| +2450822|1730|1|95| +2450822|1732|1|234| +2450822|1735|1|833| +2450822|1736|1|506| +2450822|1738|1|905| +2450822|1741|1|431| +2450822|1742|1|791| +2450822|1744|1|282| +2450822|1747|1|591| +2450822|1748|1|199| +2450822|1750|1|609| +2450822|1753|1|432| +2450822|1754|1|372| +2450822|1756|1|853| +2450822|1759|1|767| +2450822|1760|1|688| +2450822|1762|1|403| +2450822|1765|1|740| +2450822|1766|1|928| +2450822|1768|1|516| +2450822|1771|1|614| +2450822|1772|1|941| +2450822|1774|1|274| +2450822|1777|1|315| +2450822|1778|1|60| +2450822|1780|1|565| +2450822|1783|1|488| +2450822|1784|1|700| +2450822|1786|1|739| +2450822|1789|1|563| +2450822|1790|1|161| +2450822|1792|1|740| +2450822|1795|1|367| +2450822|1796|1|972| +2450822|1798|1|536| +2450822|1801|1|113| +2450822|1802|1|7| +2450822|1804|1|443| +2450822|1807|1|153| +2450822|1808|1|44| +2450822|1810|1|240| +2450822|1813|1|404| +2450822|1814|1|134| +2450822|1816|1|688| +2450822|1819|1|584| +2450822|1820|1|498| +2450822|1822|1|680| +2450822|1825|1|279| +2450822|1826|1|180| +2450822|1828|1|615| +2450822|1831|1|610| +2450822|1832|1|746| +2450822|1834|1|935| +2450822|1837|1|291| +2450822|1838|1|954| +2450822|1840|1|623| +2450822|1843|1|405| +2450822|1844|1|796| +2450822|1846|1|342| +2450822|1849|1|692| +2450822|1850|1|864| +2450822|1852|1|26| +2450822|1855|1|850| +2450822|1856|1|630| +2450822|1858|1|559| +2450822|1861|1|776| +2450822|1862|1|450| +2450822|1864|1|739| +2450822|1867|1|131| +2450822|1868|1|411| +2450822|1870|1|223| +2450822|1873|1|456| +2450822|1874|1|565| +2450822|1876|1|942| +2450822|1879|1|849| +2450822|1880|1|410| +2450822|1882|1|661| +2450822|1885|1|268| +2450822|1886|1|246| +2450822|1888|1|| +2450822|1891|1|905| +2450822|1892|1|301| +2450822|1894|1|274| +2450822|1897|1|568| +2450822|1898|1|924| +2450822|1900|1|434| +2450822|1903|1|411| +2450822|1904|1|321| +2450822|1906|1|504| +2450822|1909|1|| +2450822|1910|1|589| +2450822|1912|1|551| +2450822|1915|1|776| +2450822|1916|1|160| +2450822|1918|1|41| +2450822|1921|1|628| +2450822|1922|1|86| +2450822|1924|1|315| +2450822|1927|1|77| +2450822|1928|1|719| +2450822|1930|1|588| +2450822|1933|1|887| +2450822|1934|1|85| +2450822|1936|1|578| +2450822|1939|1|383| +2450822|1940|1|731| +2450822|1942|1|931| +2450822|1945|1|984| +2450822|1946|1|916| +2450822|1948|1|326| +2450822|1951|1|390| +2450822|1952|1|1| +2450822|1954|1|127| +2450822|1957|1|751| +2450822|1958|1|690| +2450822|1960|1|395| +2450822|1963|1|510| +2450822|1964|1|260| +2450822|1966|1|301| +2450822|1969|1|731| +2450822|1970|1|783| +2450822|1972|1|861| +2450822|1975|1|311| +2450822|1976|1|388| +2450822|1978|1|723| +2450822|1981|1|615| +2450822|1982|1|| +2450822|1984|1|653| +2450822|1987|1|112| +2450822|1988|1|651| +2450822|1990|1|378| +2450822|1993|1|558| +2450822|1994|1|725| +2450822|1996|1|852| +2450822|1999|1|270| +2450822|2000|1|289| +2450822|2002|1|884| +2450822|2005|1|631| +2450822|2006|1|687| +2450822|2008|1|620| +2450822|2011|1|520| +2450822|2012|1|632| +2450822|2014|1|85| +2450822|2017|1|| +2450822|2018|1|849| +2450822|2020|1|203| +2450822|2023|1|587| +2450822|2024|1|631| +2450822|2026|1|252| +2450822|2029|1|635| +2450822|2030|1|863| +2450822|2032|1|912| +2450822|2035|1|408| +2450822|2036|1|7| +2450822|2038|1|46| +2450822|2041|1|261| +2450822|2042|1|740| +2450822|2044|1|756| +2450822|2047|1|527| +2450822|2048|1|346| +2450822|2050|1|427| +2450822|2053|1|769| +2450822|2054|1|927| +2450822|2056|1|598| +2450822|2059|1|31| +2450822|2060|1|530| +2450822|2062|1|4| +2450822|2065|1|261| +2450822|2066|1|342| +2450822|2068|1|401| +2450822|2071|1|904| +2450822|2072|1|314| +2450822|2074|1|381| +2450822|2077|1|270| +2450822|2078|1|109| +2450822|2080|1|187| +2450822|2083|1|72| +2450822|2084|1|849| +2450822|2086|1|418| +2450822|2089|1|92| +2450822|2090|1|796| +2450822|2092|1|411| +2450822|2095|1|5| +2450822|2096|1|318| +2450822|2098|1|325| +2450822|2101|1|695| +2450822|2102|1|657| +2450822|2104|1|536| +2450822|2107|1|366| +2450822|2108|1|211| +2450822|2110|1|46| +2450822|2113|1|454| +2450822|2114|1|31| +2450822|2116|1|879| +2450822|2119|1|102| +2450822|2120|1|208| +2450822|2122|1|865| +2450822|2125|1|694| +2450822|2126|1|917| +2450822|2128|1|117| +2450822|2131|1|507| +2450822|2132|1|888| +2450822|2134|1|398| +2450822|2137|1|11| +2450822|2138|1|718| +2450822|2140|1|339| +2450822|2143|1|192| +2450822|2144|1|323| +2450822|2146|1|739| +2450822|2149|1|825| +2450822|2150|1|184| +2450822|2152|1|313| +2450822|2155|1|140| +2450822|2156|1|755| +2450822|2158|1|452| +2450822|2161|1|772| +2450822|2162|1|905| +2450822|2164|1|896| +2450822|2167|1|506| +2450822|2168|1|684| +2450822|2170|1|107| +2450822|2173|1|680| +2450822|2174|1|305| +2450822|2176|1|252| +2450822|2179|1|253| +2450822|2180|1|663| +2450822|2182|1|| +2450822|2185|1|319| +2450822|2186|1|606| +2450822|2188|1|252| +2450822|2191|1|413| +2450822|2192|1|1000| +2450822|2194|1|401| +2450822|2197|1|446| +2450822|2198|1|998| +2450822|2200|1|458| +2450822|2203|1|| +2450822|2204|1|932| +2450822|2206|1|516| +2450822|2209|1|256| +2450822|2210|1|917| +2450822|2212|1|183| +2450822|2215|1|619| +2450822|2216|1|165| +2450822|2218|1|| +2450822|2221|1|520| +2450822|2222|1|961| +2450822|2224|1|272| +2450822|2227|1|609| +2450822|2228|1|758| +2450822|2230|1|64| +2450822|2233|1|155| +2450822|2234|1|930| +2450822|2236|1|747| +2450822|2239|1|491| +2450822|2240|1|500| +2450822|2242|1|252| +2450822|2245|1|| +2450822|2246|1|526| +2450822|2248|1|131| +2450822|2251|1|539| +2450822|2252|1|659| +2450822|2254|1|321| +2450822|2257|1|321| +2450822|2258|1|400| +2450822|2260|1|515| +2450822|2263|1|629| +2450822|2264|1|255| +2450822|2266|1|129| +2450822|2269|1|| +2450822|2270|1|925| +2450822|2272|1|209| +2450822|2275|1|863| +2450822|2276|1|553| +2450822|2278|1|525| +2450822|2281|1|745| +2450822|2282|1|730| +2450822|2284|1|543| +2450822|2287|1|517| +2450822|2288|1|607| +2450822|2290|1|854| +2450822|2293|1|| +2450822|2294|1|128| +2450822|2296|1|521| +2450822|2299|1|272| +2450822|2300|1|309| +2450822|2302|1|427| +2450822|2305|1|234| +2450822|2306|1|365| +2450822|2308|1|835| +2450822|2311|1|373| +2450822|2312|1|890| +2450822|2314|1|596| +2450822|2317|1|982| +2450822|2318|1|999| +2450822|2320|1|469| +2450822|2323|1|699| +2450822|2324|1|949| +2450822|2326|1|383| +2450822|2329|1|50| +2450822|2330|1|342| +2450822|2332|1|728| +2450822|2335|1|244| +2450822|2336|1|64| +2450822|2338|1|89| +2450822|2341|1|553| +2450822|2342|1|757| +2450822|2344|1|791| +2450822|2347|1|64| +2450822|2348|1|105| +2450822|2350|1|685| +2450822|2353|1|368| +2450822|2354|1|401| +2450822|2356|1|635| +2450822|2359|1|128| +2450822|2360|1|971| +2450822|2362|1|843| +2450822|2365|1|401| +2450822|2366|1|264| +2450822|2368|1|386| +2450822|2371|1|216| +2450822|2372|1|664| +2450822|2374|1|824| +2450822|2377|1|595| +2450822|2378|1|40| +2450822|2380|1|807| +2450822|2383|1|938| +2450822|2384|1|743| +2450822|2386|1|| +2450822|2389|1|245| +2450822|2390|1|743| +2450822|2392|1|731| +2450822|2395|1|376| +2450822|2396|1|431| +2450822|2398|1|415| +2450822|2401|1|167| +2450822|2402|1|183| +2450822|2404|1|834| +2450822|2407|1|655| +2450822|2408|1|464| +2450822|2410|1|618| +2450822|2413|1|163| +2450822|2414|1|884| +2450822|2416|1|529| +2450822|2419|1|988| +2450822|2420|1|735| +2450822|2422|1|389| +2450822|2425|1|933| +2450822|2426|1|855| +2450822|2428|1|| +2450822|2431|1|344| +2450822|2432|1|498| +2450822|2434|1|568| +2450822|2437|1|941| +2450822|2438|1|108| +2450822|2440|1|564| +2450822|2443|1|198| +2450822|2444|1|580| +2450822|2446|1|536| +2450822|2449|1|764| +2450822|2450|1|648| +2450822|2452|1|219| +2450822|2455|1|569| +2450822|2456|1|183| +2450822|2458|1|135| +2450822|2461|1|581| +2450822|2462|1|205| +2450822|2464|1|240| +2450822|2467|1|12| +2450822|2468|1|135| +2450822|2470|1|366| +2450822|2473|1|708| +2450822|2474|1|532| +2450822|2476|1|729| +2450822|2479|1|508| +2450822|2480|1|666| +2450822|2482|1|369| +2450822|2485|1|302| +2450822|2486|1|789| +2450822|2488|1|49| +2450822|2491|1|276| +2450822|2492|1|688| +2450822|2494|1|763| +2450822|2497|1|597| +2450822|2498|1|398| +2450822|2500|1|458| +2450822|2503|1|643| +2450822|2504|1|249| +2450822|2506|1|515| +2450822|2509|1|797| +2450822|2510|1|191| +2450822|2512|1|181| +2450822|2515|1|534| +2450822|2516|1|847| +2450822|2518|1|953| +2450822|2521|1|198| +2450822|2522|1|270| +2450822|2524|1|262| +2450822|2527|1|792| +2450822|2528|1|400| +2450822|2530|1|564| +2450822|2533|1|612| +2450822|2534|1|| +2450822|2536|1|933| +2450822|2539|1|105| +2450822|2540|1|294| +2450822|2542|1|18| +2450822|2545|1|385| +2450822|2546|1|769| +2450822|2548|1|269| +2450822|2551|1|558| +2450822|2552|1|254| +2450822|2554|1|775| +2450822|2557|1|576| +2450822|2558|1|848| +2450822|2560|1|41| +2450822|2563|1|376| +2450822|2564|1|520| +2450822|2566|1|588| +2450822|2569|1|123| +2450822|2570|1|274| +2450822|2572|1|532| +2450822|2575|1|324| +2450822|2576|1|147| +2450822|2578|1|737| +2450822|2581|1|966| +2450822|2582|1|173| +2450822|2584|1|707| +2450822|2587|1|667| +2450822|2588|1|303| +2450822|2590|1|138| +2450822|2593|1|472| +2450822|2594|1|955| +2450822|2596|1|125| +2450822|2599|1|173| +2450822|2600|1|| +2450822|2602|1|385| +2450822|2605|1|332| +2450822|2606|1|621| +2450822|2608|1|99| +2450822|2611|1|875| +2450822|2612|1|343| +2450822|2614|1|611| +2450822|2617|1|| +2450822|2618|1|870| +2450822|2620|1|974| +2450822|2623|1|152| +2450822|2624|1|292| +2450822|2626|1|884| +2450822|2629|1|28| +2450822|2630|1|530| +2450822|2632|1|600| +2450822|2635|1|644| +2450822|2636|1|870| +2450822|2638|1|817| +2450822|2641|1|| +2450822|2642|1|940| +2450822|2644|1|936| +2450822|2647|1|813| +2450822|2648|1|202| +2450822|2650|1|227| +2450822|2653|1|663| +2450822|2654|1|197| +2450822|2656|1|925| +2450822|2659|1|858| +2450822|2660|1|683| +2450822|2662|1|156| +2450822|2665|1|257| +2450822|2666|1|892| +2450822|2668|1|68| +2450822|2671|1|174| +2450822|2672|1|628| +2450822|2674|1|288| +2450822|2677|1|740| +2450822|2678|1|632| +2450822|2680|1|115| +2450822|2683|1|525| +2450822|2684|1|935| +2450822|2686|1|499| +2450822|2689|1|179| +2450822|2690|1|498| +2450822|2692|1|135| +2450822|2695|1|39| +2450822|2696|1|584| +2450822|2698|1|555| +2450822|2701|1|350| +2450822|2702|1|802| +2450822|2704|1|514| +2450822|2707|1|190| +2450822|2708|1|192| +2450822|2710|1|831| +2450822|2713|1|975| +2450822|2714|1|389| +2450822|2716|1|998| +2450822|2719|1|207| +2450822|2720|1|499| +2450822|2722|1|222| +2450822|2725|1|645| +2450822|2726|1|360| +2450822|2728|1|209| +2450822|2731|1|274| +2450822|2732|1|84| +2450822|2734|1|897| +2450822|2737|1|| +2450822|2738|1|923| +2450822|2740|1|906| +2450822|2743|1|382| +2450822|2744|1|930| +2450822|2746|1|327| +2450822|2749|1|999| +2450822|2750|1|995| +2450822|2752|1|436| +2450822|2755|1|278| +2450822|2756|1|926| +2450822|2758|1|281| +2450822|2761|1|907| +2450822|2762|1|| +2450822|2764|1|| +2450822|2767|1|355| +2450822|2768|1|32| +2450822|2770|1|387| +2450822|2773|1|553| +2450822|2774|1|783| +2450822|2776|1|799| +2450822|2779|1|436| +2450822|2780|1|279| +2450822|2782|1|846| +2450822|2785|1|323| +2450822|2786|1|40| +2450822|2788|1|876| +2450822|2791|1|409| +2450822|2792|1|490| +2450822|2794|1|766| +2450822|2797|1|186| +2450822|2798|1|82| +2450822|2800|1|261| +2450822|2803|1|24| +2450822|2804|1|727| +2450822|2806|1|851| +2450822|2809|1|504| +2450822|2810|1|955| +2450822|2812|1|699| +2450822|2815|1|411| +2450822|2816|1|19| +2450822|2818|1|80| +2450822|2821|1|| +2450822|2822|1|710| +2450822|2824|1|972| +2450822|2827|1|9| +2450822|2828|1|641| +2450822|2830|1|855| +2450822|2833|1|330| +2450822|2834|1|570| +2450822|2836|1|934| +2450822|2839|1|977| +2450822|2840|1|581| +2450822|2842|1|609| +2450822|2845|1|486| +2450822|2846|1|109| +2450822|2848|1|906| +2450822|2851|1|938| +2450822|2852|1|198| +2450822|2854|1|841| +2450822|2857|1|707| +2450822|2858|1|812| +2450822|2860|1|| +2450822|2863|1|230| +2450822|2864|1|728| +2450822|2866|1|173| +2450822|2869|1|659| +2450822|2870|1|| +2450822|2872|1|140| +2450822|2875|1|849| +2450822|2876|1|135| +2450822|2878|1|357| +2450822|2881|1|179| +2450822|2882|1|333| +2450822|2884|1|351| +2450822|2887|1|47| +2450822|2888|1|30| +2450822|2890|1|863| +2450822|2893|1|382| +2450822|2894|1|290| +2450822|2896|1|418| +2450822|2899|1|26| +2450822|2900|1|718| +2450822|2902|1|893| +2450822|2905|1|150| +2450822|2906|1|961| +2450822|2908|1|642| +2450822|2911|1|841| +2450822|2912|1|525| +2450822|2914|1|743| +2450822|2917|1|| +2450822|2918|1|340| +2450822|2920|1|731| +2450822|2923|1|345| +2450822|2924|1|134| +2450822|2926|1|475| +2450822|2929|1|735| +2450822|2930|1|545| +2450822|2932|1|650| +2450822|2935|1|461| +2450822|2936|1|20| +2450822|2938|1|94| +2450822|2941|1|972| +2450822|2942|1|631| +2450822|2944|1|348| +2450822|2947|1|517| +2450822|2948|1|191| +2450822|2950|1|430| +2450822|2953|1|| +2450822|2954|1|245| +2450822|2956|1|594| +2450822|2959|1|646| +2450822|2960|1|52| +2450822|2962|1|175| +2450822|2965|1|143| +2450822|2966|1|292| +2450822|2968|1|881| +2450822|2971|1|915| +2450822|2972|1|756| +2450822|2974|1|435| +2450822|2977|1|614| +2450822|2978|1|216| +2450822|2980|1|373| +2450822|2983|1|0| +2450822|2984|1|715| +2450822|2986|1|842| +2450822|2989|1|173| +2450822|2990|1|167| +2450822|2992|1|644| +2450822|2995|1|793| +2450822|2996|1|64| +2450822|2998|1|918| +2450822|3001|1|412| +2450822|3002|1|782| +2450822|3004|1|682| +2450822|3007|1|115| +2450822|3008|1|325| +2450822|3010|1|721| +2450822|3013|1|4| +2450822|3014|1|124| +2450822|3016|1|104| +2450822|3019|1|741| +2450822|3020|1|23| +2450822|3022|1|32| +2450822|3025|1|91| +2450822|3026|1|521| +2450822|3028|1|918| +2450822|3031|1|104| +2450822|3032|1|447| +2450822|3034|1|76| +2450822|3037|1|958| +2450822|3038|1|136| +2450822|3040|1|465| +2450822|3043|1|319| +2450822|3044|1|591| +2450822|3046|1|144| +2450822|3049|1|748| +2450822|3050|1|644| +2450822|3052|1|132| +2450822|3055|1|296| +2450822|3056|1|377| +2450822|3058|1|150| +2450822|3061|1|401| +2450822|3062|1|943| +2450822|3064|1|205| +2450822|3067|1|632| +2450822|3068|1|479| +2450822|3070|1|522| +2450822|3073|1|767| +2450822|3074|1|357| +2450822|3076|1|602| +2450822|3079|1|773| +2450822|3080|1|167| +2450822|3082|1|457| +2450822|3085|1|654| +2450822|3086|1|609| +2450822|3088|1|| +2450822|3091|1|770| +2450822|3092|1|854| +2450822|3094|1|445| +2450822|3097|1|| +2450822|3098|1|803| +2450822|3100|1|962| +2450822|3103|1|204| +2450822|3104|1|437| +2450822|3106|1|243| +2450822|3109|1|56| +2450822|3110|1|398| +2450822|3112|1|652| +2450822|3115|1|988| +2450822|3116|1|918| +2450822|3118|1|355| +2450822|3121|1|817| +2450822|3122|1|853| +2450822|3124|1|| +2450822|3127|1|417| +2450822|3128|1|513| +2450822|3130|1|232| +2450822|3133|1|289| +2450822|3134|1|85| +2450822|3136|1|877| +2450822|3139|1|583| +2450822|3140|1|130| +2450822|3142|1|161| +2450822|3145|1|945| +2450822|3146|1|508| +2450822|3148|1|366| +2450822|3151|1|819| +2450822|3152|1|530| +2450822|3154|1|355| +2450822|3157|1|339| +2450822|3158|1|214| +2450822|3160|1|207| +2450822|3163|1|144| +2450822|3164|1|620| +2450822|3166|1|12| +2450822|3169|1|274| +2450822|3170|1|46| +2450822|3172|1|45| +2450822|3175|1|922| +2450822|3176|1|759| +2450822|3178|1|86| +2450822|3181|1|367| +2450822|3182|1|353| +2450822|3184|1|844| +2450822|3187|1|91| +2450822|3188|1|922| +2450822|3190|1|329| +2450822|3193|1|424| +2450822|3194|1|217| +2450822|3196|1|158| +2450822|3199|1|279| +2450822|3200|1|929| +2450822|3202|1|323| +2450822|3205|1|454| +2450822|3206|1|784| +2450822|3208|1|| +2450822|3211|1|210| +2450822|3212|1|571| +2450822|3214|1|364| +2450822|3217|1|822| +2450822|3218|1|549| +2450822|3220|1|423| +2450822|3223|1|520| +2450822|3224|1|576| +2450822|3226|1|670| +2450822|3229|1|615| +2450822|3230|1|124| +2450822|3232|1|850| +2450822|3235|1|806| +2450822|3236|1|742| +2450822|3238|1|12| +2450822|3241|1|598| +2450822|3242|1|300| +2450822|3244|1|308| +2450822|3247|1|405| +2450822|3248|1|686| +2450822|3250|1|601| +2450822|3253|1|619| +2450822|3254|1|56| +2450822|3256|1|853| +2450822|3259|1|248| +2450822|3260|1|449| +2450822|3262|1|21| +2450822|3265|1|18| +2450822|3266|1|| +2450822|3268|1|784| +2450822|3271|1|800| +2450822|3272|1|668| +2450822|3274|1|890| +2450822|3277|1|96| +2450822|3278|1|963| +2450822|3280|1|781| +2450822|3283|1|1000| +2450822|3284|1|348| +2450822|3286|1|619| +2450822|3289|1|| +2450822|3290|1|174| +2450822|3292|1|85| +2450822|3295|1|876| +2450822|3296|1|990| +2450822|3298|1|98| +2450822|3301|1|412| +2450822|3302|1|432| +2450822|3304|1|| +2450822|3307|1|443| +2450822|3308|1|| +2450822|3310|1|299| +2450822|3313|1|310| +2450822|3314|1|214| +2450822|3316|1|13| +2450822|3319|1|904| +2450822|3320|1|452| +2450822|3322|1|468| +2450822|3325|1|522| +2450822|3326|1|142| +2450822|3328|1|604| +2450822|3331|1|964| +2450822|3332|1|496| +2450822|3334|1|39| +2450822|3337|1|719| +2450822|3338|1|806| +2450822|3340|1|929| +2450822|3343|1|637| +2450822|3344|1|| +2450822|3346|1|444| +2450822|3349|1|80| +2450822|3350|1|806| +2450822|3352|1|142| +2450822|3355|1|808| +2450822|3356|1|154| +2450822|3358|1|623| +2450822|3361|1|315| +2450822|3362|1|310| +2450822|3364|1|155| +2450822|3367|1|182| +2450822|3368|1|593| +2450822|3370|1|442| +2450822|3373|1|379| +2450822|3374|1|134| +2450822|3376|1|613| +2450822|3379|1|257| +2450822|3380|1|494| +2450822|3382|1|967| +2450822|3385|1|180| +2450822|3386|1|907| +2450822|3388|1|710| +2450822|3391|1|638| +2450822|3392|1|306| +2450822|3394|1|795| +2450822|3397|1|656| +2450822|3398|1|9| +2450822|3400|1|66| +2450822|3403|1|491| +2450822|3404|1|429| +2450822|3406|1|650| +2450822|3409|1|472| +2450822|3410|1|290| +2450822|3412|1|58| +2450822|3415|1|614| +2450822|3416|1|812| +2450822|3418|1|489| +2450822|3421|1|760| +2450822|3422|1|| +2450822|3424|1|640| +2450822|3427|1|636| +2450822|3428|1|163| +2450822|3430|1|476| +2450822|3433|1|905| +2450822|3434|1|196| +2450822|3436|1|37| +2450822|3439|1|943| +2450822|3440|1|402| +2450822|3442|1|259| +2450822|3445|1|921| +2450822|3446|1|944| +2450822|3448|1|743| +2450822|3451|1|417| +2450822|3452|1|899| +2450822|3454|1|378| +2450822|3457|1|250| +2450822|3458|1|755| +2450822|3460|1|955| +2450822|3463|1|| +2450822|3464|1|770| +2450822|3466|1|570| +2450822|3469|1|645| +2450822|3470|1|567| +2450822|3472|1|934| +2450822|3475|1|475| +2450822|3476|1|359| +2450822|3478|1|116| +2450822|3481|1|903| +2450822|3482|1|147| +2450822|3484|1|287| +2450822|3487|1|518| +2450822|3488|1|775| +2450822|3490|1|186| +2450822|3493|1|216| +2450822|3494|1|674| +2450822|3496|1|732| +2450822|3499|1|40| +2450822|3500|1|67| +2450822|3502|1|343| +2450822|3505|1|723| +2450822|3506|1|333| +2450822|3508|1|72| +2450822|3511|1|459| +2450822|3512|1|663| +2450822|3514|1|67| +2450822|3517|1|101| +2450822|3518|1|461| +2450822|3520|1|774| +2450822|3523|1|904| +2450822|3524|1|449| +2450822|3526|1|486| +2450822|3529|1|10| +2450822|3530|1|644| +2450822|3532|1|109| +2450822|3535|1|604| +2450822|3536|1|450| +2450822|3538|1|93| +2450822|3541|1|435| +2450822|3542|1|572| +2450822|3544|1|464| +2450822|3547|1|541| +2450822|3548|1|330| +2450822|3550|1|21| +2450822|3553|1|343| +2450822|3554|1|806| +2450822|3556|1|261| +2450822|3559|1|161| +2450822|3560|1|750| +2450822|3562|1|305| +2450822|3565|1|208| +2450822|3566|1|432| +2450822|3568|1|661| +2450822|3571|1|411| +2450822|3572|1|793| +2450822|3574|1|227| +2450822|3577|1|330| +2450822|3578|1|67| +2450822|3580|1|666| +2450822|3583|1|170| +2450822|3584|1|658| +2450822|3586|1|963| +2450822|3589|1|634| +2450822|3590|1|458| +2450822|3592|1|373| +2450822|3595|1|416| +2450822|3596|1|822| +2450822|3598|1|651| +2450822|3601|1|827| +2450822|3602|1|172| +2450822|3604|1|572| +2450822|3607|1|755| +2450822|3608|1|253| +2450822|3610|1|503| +2450822|3613|1|45| +2450822|3614|1|414| +2450822|3616|1|772| +2450822|3619|1|201| +2450822|3620|1|505| +2450822|3622|1|43| +2450822|3625|1|| +2450822|3626|1|9| +2450822|3628|1|150| +2450822|3631|1|| +2450822|3632|1|658| +2450822|3634|1|593| +2450822|3637|1|590| +2450822|3638|1|610| +2450822|3640|1|622| +2450822|3643|1|284| +2450822|3644|1|910| +2450822|3646|1|15| +2450822|3649|1|177| +2450822|3650|1|423| +2450822|3652|1|934| +2450822|3655|1|| +2450822|3656|1|570| +2450822|3658|1|517| +2450822|3661|1|446| +2450822|3662|1|239| +2450822|3664|1|791| +2450822|3667|1|667| +2450822|3668|1|552| +2450822|3670|1|246| +2450822|3673|1|696| +2450822|3674|1|548| +2450822|3676|1|138| +2450822|3679|1|350| +2450822|3680|1|567| +2450822|3682|1|515| +2450822|3685|1|480| +2450822|3686|1|576| +2450822|3688|1|793| +2450822|3691|1|365| +2450822|3692|1|652| +2450822|3694|1|587| +2450822|3697|1|495| +2450822|3698|1|299| +2450822|3700|1|592| +2450822|3703|1|772| +2450822|3704|1|| +2450822|3706|1|324| +2450822|3709|1|309| +2450822|3710|1|83| +2450822|3712|1|6| +2450822|3715|1|150| +2450822|3716|1|977| +2450822|3718|1|145| +2450822|3721|1|394| +2450822|3722|1|237| +2450822|3724|1|791| +2450822|3727|1|915| +2450822|3728|1|348| +2450822|3730|1|922| +2450822|3733|1|945| +2450822|3734|1|654| +2450822|3736|1|684| +2450822|3739|1|93| +2450822|3740|1|459| +2450822|3742|1|553| +2450822|3745|1|754| +2450822|3746|1|472| +2450822|3748|1|451| +2450822|3751|1|366| +2450822|3752|1|451| +2450822|3754|1|| +2450822|3757|1|48| +2450822|3758|1|653| +2450822|3760|1|898| +2450822|3763|1|72| +2450822|3764|1|914| +2450822|3766|1|536| +2450822|3769|1|842| +2450822|3770|1|138| +2450822|3772|1|535| +2450822|3775|1|640| +2450822|3776|1|215| +2450822|3778|1|903| +2450822|3781|1|758| +2450822|3782|1|| +2450822|3784|1|365| +2450822|3787|1|911| +2450822|3788|1|545| +2450822|3790|1|900| +2450822|3793|1|636| +2450822|3794|1|674| +2450822|3796|1|896| +2450822|3799|1|268| +2450822|3800|1|769| +2450822|3802|1|208| +2450822|3805|1|209| +2450822|3806|1|61| +2450822|3808|1|972| +2450822|3811|1|8| +2450822|3812|1|98| +2450822|3814|1|755| +2450822|3817|1|982| +2450822|3818|1|155| +2450822|3820|1|889| +2450822|3823|1|179| +2450822|3824|1|568| +2450822|3826|1|719| +2450822|3829|1|148| +2450822|3830|1|818| +2450822|3832|1|339| +2450822|3835|1|582| +2450822|3836|1|987| +2450822|3838|1|404| +2450822|3841|1|239| +2450822|3842|1|620| +2450822|3844|1|44| +2450822|3847|1|196| +2450822|3848|1|904| +2450822|3850|1|23| +2450822|3853|1|477| +2450822|3854|1|902| +2450822|3856|1|681| +2450822|3859|1|155| +2450822|3860|1|741| +2450822|3862|1|549| +2450822|3865|1|809| +2450822|3866|1|962| +2450822|3868|1|894| +2450822|3871|1|169| +2450822|3872|1|141| +2450822|3874|1|778| +2450822|3877|1|631| +2450822|3878|1|383| +2450822|3880|1|273| +2450822|3883|1|704| +2450822|3884|1|625| +2450822|3886|1|272| +2450822|3889|1|215| +2450822|3890|1|619| +2450822|3892|1|212| +2450822|3895|1|229| +2450822|3896|1|397| +2450822|3898|1|134| +2450822|3901|1|211| +2450822|3902|1|491| +2450822|3904|1|863| +2450822|3907|1|460| +2450822|3908|1|| +2450822|3910|1|585| +2450822|3913|1|| +2450822|3914|1|498| +2450822|3916|1|659| +2450822|3919|1|740| +2450822|3920|1|48| +2450822|3922|1|307| +2450822|3925|1|28| +2450822|3926|1|| +2450822|3928|1|914| +2450822|3931|1|332| +2450822|3932|1|302| +2450822|3934|1|257| +2450822|3937|1|| +2450822|3938|1|40| +2450822|3940|1|591| +2450822|3943|1|916| +2450822|3944|1|456| +2450822|3946|1|289| +2450822|3949|1|946| +2450822|3950|1|833| +2450822|3952|1|236| +2450822|3955|1|665| +2450822|3956|1|847| +2450822|3958|1|323| +2450822|3961|1|160| +2450822|3962|1|823| +2450822|3964|1|649| +2450822|3967|1|611| +2450822|3968|1|490| +2450822|3970|1|691| +2450822|3973|1|550| +2450822|3974|1|935| +2450822|3976|1|203| +2450822|3979|1|672| +2450822|3980|1|564| +2450822|3982|1|472| +2450822|3985|1|647| +2450822|3986|1|383| +2450822|3988|1|664| +2450822|3991|1|576| +2450822|3992|1|324| +2450822|3994|1|190| +2450822|3997|1|583| +2450822|3998|1|174| +2450822|4000|1|430| +2450822|4003|1|269| +2450822|4004|1|74| +2450822|4006|1|| +2450822|4009|1|873| +2450822|4010|1|| +2450822|4012|1|222| +2450822|4015|1|706| +2450822|4016|1|578| +2450822|4018|1|407| +2450822|4021|1|834| +2450822|4022|1|822| +2450822|4024|1|508| +2450822|4027|1|| +2450822|4028|1|123| +2450822|4030|1|578| +2450822|4033|1|760| +2450822|4034|1|548| +2450822|4036|1|616| +2450822|4039|1|79| +2450822|4040|1|750| +2450822|4042|1|775| +2450822|4045|1|732| +2450822|4046|1|403| +2450822|4048|1|168| +2450822|4051|1|391| +2450822|4052|1|906| +2450822|4054|1|759| +2450822|4057|1|905| +2450822|4058|1|812| +2450822|4060|1|800| +2450822|4063|1|450| +2450822|4064|1|773| +2450822|4066|1|333| +2450822|4069|1|245| +2450822|4070|1|974| +2450822|4072|1|701| +2450822|4075|1|324| +2450822|4076|1|743| +2450822|4078|1|401| +2450822|4081|1|253| +2450822|4082|1|342| +2450822|4084|1|717| +2450822|4087|1|867| +2450822|4088|1|936| +2450822|4090|1|363| +2450822|4093|1|283| +2450822|4094|1|658| +2450822|4096|1|220| +2450822|4099|1|151| +2450822|4100|1|| +2450822|4102|1|806| +2450822|4105|1|110| +2450822|4106|1|967| +2450822|4108|1|319| +2450822|4111|1|602| +2450822|4112|1|197| +2450822|4114|1|790| +2450822|4117|1|833| +2450822|4118|1|32| +2450822|4120|1|114| +2450822|4123|1|890| +2450822|4124|1|386| +2450822|4126|1|90| +2450822|4129|1|917| +2450822|4130|1|788| +2450822|4132|1|| +2450822|4135|1|331| +2450822|4136|1|836| +2450822|4138|1|214| +2450822|4141|1|399| +2450822|4142|1|920| +2450822|4144|1|355| +2450822|4147|1|669| +2450822|4148|1|334| +2450822|4150|1|729| +2450822|4153|1|397| +2450822|4154|1|268| +2450822|4156|1|162| +2450822|4159|1|385| +2450822|4160|1|329| +2450822|4162|1|776| +2450822|4165|1|447| +2450822|4166|1|295| +2450822|4168|1|780| +2450822|4171|1|416| +2450822|4172|1|314| +2450822|4174|1|779| +2450822|4177|1|550| +2450822|4178|1|548| +2450822|4180|1|980| +2450822|4183|1|268| +2450822|4184|1|192| +2450822|4186|1|306| +2450822|4189|1|1| +2450822|4190|1|863| +2450822|4192|1|432| +2450822|4195|1|542| +2450822|4196|1|907| +2450822|4198|1|258| +2450822|4201|1|557| +2450822|4202|1|974| +2450822|4204|1|47| +2450822|4207|1|395| +2450822|4208|1|581| +2450822|4210|1|723| +2450822|4213|1|301| +2450822|4214|1|951| +2450822|4216|1|77| +2450822|4219|1|12| +2450822|4220|1|217| +2450822|4222|1|500| +2450822|4225|1|872| +2450822|4226|1|329| +2450822|4228|1|963| +2450822|4231|1|524| +2450822|4232|1|917| +2450822|4234|1|638| +2450822|4237|1|64| +2450822|4238|1|442| +2450822|4240|1|541| +2450822|4243|1|418| +2450822|4244|1|324| +2450822|4246|1|130| +2450822|4249|1|605| +2450822|4250|1|328| +2450822|4252|1|897| +2450822|4255|1|37| +2450822|4256|1|177| +2450822|4258|1|823| +2450822|4261|1|598| +2450822|4262|1|264| +2450822|4264|1|375| +2450822|4267|1|185| +2450822|4268|1|393| +2450822|4270|1|545| +2450822|4273|1|372| +2450822|4274|1|439| +2450822|4276|1|727| +2450822|4279|1|844| +2450822|4280|1|718| +2450822|4282|1|298| +2450822|4285|1|102| +2450822|4286|1|557| +2450822|4288|1|29| +2450822|4291|1|917| +2450822|4292|1|64| +2450822|4294|1|494| +2450822|4297|1|43| +2450822|4298|1|126| +2450822|4300|1|663| +2450822|4303|1|150| +2450822|4304|1|704| +2450822|4306|1|| +2450822|4309|1|754| +2450822|4310|1|293| +2450822|4312|1|639| +2450822|4315|1|728| +2450822|4316|1|621| +2450822|4318|1|55| +2450822|4321|1|679| +2450822|4322|1|9| +2450822|4324|1|221| +2450822|4327|1|547| +2450822|4328|1|156| +2450822|4330|1|569| +2450822|4333|1|528| +2450822|4334|1|| +2450822|4336|1|782| +2450822|4339|1|75| +2450822|4340|1|471| +2450822|4342|1|721| +2450822|4345|1|321| +2450822|4346|1|776| +2450822|4348|1|146| +2450822|4351|1|159| +2450822|4352|1|996| +2450822|4354|1|810| +2450822|4357|1|734| +2450822|4358|1|785| +2450822|4360|1|782| +2450822|4363|1|315| +2450822|4364|1|797| +2450822|4366|1|312| +2450822|4369|1|369| +2450822|4370|1|387| +2450822|4372|1|283| +2450822|4375|1|360| +2450822|4376|1|252| +2450822|4378|1|| +2450822|4381|1|517| +2450822|4382|1|607| +2450822|4384|1|477| +2450822|4387|1|481| +2450822|4388|1|377| +2450822|4390|1|365| +2450822|4393|1|160| +2450822|4394|1|967| +2450822|4396|1|313| +2450822|4399|1|70| +2450822|4400|1|347| +2450822|4402|1|452| +2450822|4405|1|96| +2450822|4406|1|329| +2450822|4408|1|749| +2450822|4411|1|623| +2450822|4412|1|382| +2450822|4414|1|768| +2450822|4417|1|784| +2450822|4418|1|857| +2450822|4420|1|669| +2450822|4423|1|192| +2450822|4424|1|563| +2450822|4426|1|472| +2450822|4429|1|309| +2450822|4430|1|140| +2450822|4432|1|588| +2450822|4435|1|| +2450822|4436|1|302| +2450822|4438|1|141| +2450822|4441|1|537| +2450822|4442|1|329| +2450822|4444|1|460| +2450822|4447|1|7| +2450822|4448|1|145| +2450822|4450|1|192| +2450822|4453|1|772| +2450822|4454|1|394| +2450822|4456|1|370| +2450822|4459|1|491| +2450822|4460|1|534| +2450822|4462|1|426| +2450822|4465|1|541| +2450822|4466|1|561| +2450822|4468|1|137| +2450822|4471|1|156| +2450822|4472|1|67| +2450822|4474|1|805| +2450822|4477|1|146| +2450822|4478|1|798| +2450822|4480|1|704| +2450822|4483|1|87| +2450822|4484|1|979| +2450822|4486|1|279| +2450822|4489|1|367| +2450822|4490|1|698| +2450822|4492|1|941| +2450822|4495|1|22| +2450822|4496|1|69| +2450822|4498|1|916| +2450822|4501|1|236| +2450822|4502|1|579| +2450822|4504|1|205| +2450822|4507|1|265| +2450822|4508|1|432| +2450822|4510|1|331| +2450822|4513|1|747| +2450822|4514|1|950| +2450822|4516|1|622| +2450822|4519|1|278| +2450822|4520|1|171| +2450822|4522|1|894| +2450822|4525|1|633| +2450822|4526|1|519| +2450822|4528|1|477| +2450822|4531|1|381| +2450822|4532|1|562| +2450822|4534|1|435| +2450822|4537|1|544| +2450822|4538|1|289| +2450822|4540|1|954| +2450822|4543|1|721| +2450822|4544|1|965| +2450822|4546|1|601| +2450822|4549|1|599| +2450822|4550|1|793| +2450822|4552|1|334| +2450822|4555|1|685| +2450822|4556|1|585| +2450822|4558|1|554| +2450822|4561|1|325| +2450822|4562|1|589| +2450822|4564|1|508| +2450822|4567|1|48| +2450822|4568|1|546| +2450822|4570|1|927| +2450822|4573|1|179| +2450822|4574|1|271| +2450822|4576|1|679| +2450822|4579|1|136| +2450822|4580|1|231| +2450822|4582|1|713| +2450822|4585|1|334| +2450822|4586|1|48| +2450822|4588|1|928| +2450822|4591|1|589| +2450822|4592|1|| +2450822|4594|1|981| +2450822|4597|1|173| +2450822|4598|1|50| +2450822|4600|1|477| +2450822|4603|1|980| +2450822|4604|1|530| +2450822|4606|1|637| +2450822|4609|1|517| +2450822|4610|1|33| +2450822|4612|1|605| +2450822|4615|1|602| +2450822|4616|1|62| +2450822|4618|1|480| +2450822|4621|1|120| +2450822|4622|1|5| +2450822|4624|1|898| +2450822|4627|1|547| +2450822|4628|1|475| +2450822|4630|1|709| +2450822|4633|1|14| +2450822|4634|1|766| +2450822|4636|1|315| +2450822|4639|1|270| +2450822|4640|1|358| +2450822|4642|1|959| +2450822|4645|1|807| +2450822|4646|1|891| +2450822|4648|1|702| +2450822|4651|1|827| +2450822|4652|1|627| +2450822|4654|1|562| +2450822|4657|1|646| +2450822|4658|1|415| +2450822|4660|1|191| +2450822|4663|1|942| +2450822|4664|1|28| +2450822|4666|1|766| +2450822|4669|1|203| +2450822|4670|1|304| +2450822|4672|1|596| +2450822|4675|1|568| +2450822|4676|1|245| +2450822|4678|1|545| +2450822|4681|1|412| +2450822|4682|1|830| +2450822|4684|1|341| +2450822|4687|1|659| +2450822|4688|1|307| +2450822|4690|1|39| +2450822|4693|1|911| +2450822|4694|1|112| +2450822|4696|1|| +2450822|4699|1|655| +2450822|4700|1|77| +2450822|4702|1|852| +2450822|4705|1|217| +2450822|4706|1|571| +2450822|4708|1|752| +2450822|4711|1|591| +2450822|4712|1|609| +2450822|4714|1|76| +2450822|4717|1|129| +2450822|4718|1|175| +2450822|4720|1|712| +2450822|4723|1|944| +2450822|4724|1|399| +2450822|4726|1|369| +2450822|4729|1|130| +2450822|4730|1|57| +2450822|4732|1|487| +2450822|4735|1|97| +2450822|4736|1|582| +2450822|4738|1|3| +2450822|4741|1|576| +2450822|4742|1|9| +2450822|4744|1|321| +2450822|4747|1|146| +2450822|4748|1|565| +2450822|4750|1|764| +2450822|4753|1|412| +2450822|4754|1|199| +2450822|4756|1|| +2450822|4759|1|373| +2450822|4760|1|604| +2450822|4762|1|572| +2450822|4765|1|272| +2450822|4766|1|143| +2450822|4768|1|79| +2450822|4771|1|718| +2450822|4772|1|595| +2450822|4774|1|511| +2450822|4777|1|942| +2450822|4778|1|669| +2450822|4780|1|33| +2450822|4783|1|764| +2450822|4784|1|795| +2450822|4786|1|| +2450822|4789|1|563| +2450822|4790|1|70| +2450822|4792|1|| +2450822|4795|1|943| +2450822|4796|1|464| +2450822|4798|1|480| +2450822|4801|1|647| +2450822|4802|1|495| +2450822|4804|1|890| +2450822|4807|1|50| +2450822|4808|1|827| +2450822|4810|1|680| +2450822|4813|1|488| +2450822|4814|1|290| +2450822|4816|1|744| +2450822|4819|1|320| +2450822|4820|1|526| +2450822|4822|1|177| +2450822|4825|1|876| +2450822|4826|1|753| +2450822|4828|1|713| +2450822|4831|1|251| +2450822|4832|1|874| +2450822|4834|1|515| +2450822|4837|1|581| +2450822|4838|1|288| +2450822|4840|1|83| +2450822|4843|1|388| +2450822|4844|1|723| +2450822|4846|1|545| +2450822|4849|1|157| +2450822|4850|1|16| +2450822|4852|1|70| +2450822|4855|1|882| +2450822|4856|1|541| +2450822|4858|1|448| +2450822|4861|1|292| +2450822|4862|1|911| +2450822|4864|1|45| +2450822|4867|1|294| +2450822|4868|1|441| +2450822|4870|1|315| +2450822|4873|1|491| +2450822|4874|1|498| +2450822|4876|1|197| +2450822|4879|1|954| +2450822|4880|1|612| +2450822|4882|1|733| +2450822|4885|1|87| +2450822|4886|1|302| +2450822|4888|1|551| +2450822|4891|1|45| +2450822|4892|1|303| +2450822|4894|1|724| +2450822|4897|1|553| +2450822|4898|1|90| +2450822|4900|1|65| +2450822|4903|1|88| +2450822|4904|1|122| +2450822|4906|1|967| +2450822|4909|1|414| +2450822|4910|1|128| +2450822|4912|1|692| +2450822|4915|1|757| +2450822|4916|1|432| +2450822|4918|1|468| +2450822|4921|1|693| +2450822|4922|1|988| +2450822|4924|1|239| +2450822|4927|1|357| +2450822|4928|1|716| +2450822|4930|1|193| +2450822|4933|1|162| +2450822|4934|1|24| +2450822|4936|1|603| +2450822|4939|1|10| +2450822|4940|1|850| +2450822|4942|1|459| +2450822|4945|1|767| +2450822|4946|1|326| +2450822|4948|1|260| +2450822|4951|1|998| +2450822|4952|1|755| +2450822|4954|1|| +2450822|4957|1|320| +2450822|4958|1|462| +2450822|4960|1|717| +2450822|4963|1|490| +2450822|4964|1|158| +2450822|4966|1|475| +2450822|4969|1|305| +2450822|4970|1|650| +2450822|4972|1|| +2450822|4975|1|668| +2450822|4976|1|705| +2450822|4978|1|975| +2450822|4981|1|71| +2450822|4982|1|590| +2450822|4984|1|375| +2450822|4987|1|74| +2450822|4988|1|2| +2450822|4990|1|421| +2450822|4993|1|| +2450822|4994|1|749| +2450822|4996|1|| +2450822|4999|1|349| +2450822|5000|1|243| +2450822|5002|1|991| +2450822|5005|1|96| +2450822|5006|1|207| +2450822|5008|1|188| +2450822|5011|1|500| +2450822|5012|1|520| +2450822|5014|1|| +2450822|5017|1|615| +2450822|5018|1|550| +2450822|5020|1|243| +2450822|5023|1|935| +2450822|5024|1|981| +2450822|5026|1|762| +2450822|5029|1|814| +2450822|5030|1|221| +2450822|5032|1|671| +2450822|5035|1|903| +2450822|5036|1|846| +2450822|5038|1|454| +2450822|5041|1|350| +2450822|5042|1|556| +2450822|5044|1|147| +2450822|5047|1|267| +2450822|5048|1|315| +2450822|5050|1|126| +2450822|5053|1|445| +2450822|5054|1|268| +2450822|5056|1|900| +2450822|5059|1|301| +2450822|5060|1|659| +2450822|5062|1|37| +2450822|5065|1|468| +2450822|5066|1|544| +2450822|5068|1|617| +2450822|5071|1|60| +2450822|5072|1|777| +2450822|5074|1|95| +2450822|5077|1|127| +2450822|5078|1|| +2450822|5080|1|386| +2450822|5083|1|| +2450822|5084|1|270| +2450822|5086|1|266| +2450822|5089|1|897| +2450822|5090|1|482| +2450822|5092|1|922| +2450822|5095|1|511| +2450822|5096|1|759| +2450822|5098|1|599| +2450822|5101|1|270| +2450822|5102|1|691| +2450822|5104|1|121| +2450822|5107|1|886| +2450822|5108|1|546| +2450822|5110|1|670| +2450822|5113|1|33| +2450822|5114|1|646| +2450822|5116|1|981| +2450822|5119|1|970| +2450822|5120|1|609| +2450822|5122|1|234| +2450822|5125|1|900| +2450822|5126|1|432| +2450822|5128|1|668| +2450822|5131|1|14| +2450822|5132|1|514| +2450822|5134|1|769| +2450822|5137|1|749| +2450822|5138|1|764| +2450822|5140|1|754| +2450822|5143|1|33| +2450822|5144|1|468| +2450822|5146|1|961| +2450822|5149|1|642| +2450822|5150|1|425| +2450822|5152|1|324| +2450822|5155|1|83| +2450822|5156|1|406| +2450822|5158|1|489| +2450822|5161|1|615| +2450822|5162|1|722| +2450822|5164|1|719| +2450822|5167|1|929| +2450822|5168|1|916| +2450822|5170|1|989| +2450822|5173|1|650| +2450822|5174|1|335| +2450822|5176|1|886| +2450822|5179|1|908| +2450822|5180|1|513| +2450822|5182|1|575| +2450822|5185|1|898| +2450822|5186|1|545| +2450822|5188|1|421| +2450822|5191|1|403| +2450822|5192|1|348| +2450822|5194|1|58| +2450822|5197|1|876| +2450822|5198|1|30| +2450822|5200|1|150| +2450822|5203|1|160| +2450822|5204|1|435| +2450822|5206|1|666| +2450822|5209|1|584| +2450822|5210|1|831| +2450822|5212|1|443| +2450822|5215|1|59| +2450822|5216|1|100| +2450822|5218|1|776| +2450822|5221|1|204| +2450822|5222|1|867| +2450822|5224|1|| +2450822|5227|1|444| +2450822|5228|1|149| +2450822|5230|1|171| +2450822|5233|1|393| +2450822|5234|1|104| +2450822|5236|1|582| +2450822|5239|1|930| +2450822|5240|1|494| +2450822|5242|1|438| +2450822|5245|1|969| +2450822|5246|1|| +2450822|5248|1|881| +2450822|5251|1|90| +2450822|5252|1|982| +2450822|5254|1|435| +2450822|5257|1|153| +2450822|5258|1|529| +2450822|5260|1|227| +2450822|5263|1|797| +2450822|5264|1|913| +2450822|5266|1|80| +2450822|5269|1|717| +2450822|5270|1|374| +2450822|5272|1|747| +2450822|5275|1|743| +2450822|5276|1|745| +2450822|5278|1|129| +2450822|5281|1|837| +2450822|5282|1|347| +2450822|5284|1|470| +2450822|5287|1|762| +2450822|5288|1|44| +2450822|5290|1|211| +2450822|5293|1|467| +2450822|5294|1|| +2450822|5296|1|343| +2450822|5299|1|803| +2450822|5300|1|479| +2450822|5302|1|530| +2450822|5305|1|966| +2450822|5306|1|901| +2450822|5308|1|822| +2450822|5311|1|470| +2450822|5312|1|538| +2450822|5314|1|839| +2450822|5317|1|964| +2450822|5318|1|846| +2450822|5320|1|994| +2450822|5323|1|989| +2450822|5324|1|844| +2450822|5326|1|906| +2450822|5329|1|604| +2450822|5330|1|633| +2450822|5332|1|940| +2450822|5335|1|326| +2450822|5336|1|1000| +2450822|5338|1|178| +2450822|5341|1|486| +2450822|5342|1|374| +2450822|5344|1|365| +2450822|5347|1|886| +2450822|5348|1|125| +2450822|5350|1|959| +2450822|5353|1|648| +2450822|5354|1|22| +2450822|5356|1|143| +2450822|5359|1|588| +2450822|5360|1|56| +2450822|5362|1|988| +2450822|5365|1|| +2450822|5366|1|306| +2450822|5368|1|62| +2450822|5371|1|23| +2450822|5372|1|861| +2450822|5374|1|675| +2450822|5377|1|959| +2450822|5378|1|39| +2450822|5380|1|228| +2450822|5383|1|737| +2450822|5384|1|671| +2450822|5386|1|882| +2450822|5389|1|692| +2450822|5390|1|| +2450822|5392|1|871| +2450822|5395|1|386| +2450822|5396|1|911| +2450822|5398|1|635| +2450822|5401|1|775| +2450822|5402|1|29| +2450822|5404|1|62| +2450822|5407|1|| +2450822|5408|1|| +2450822|5410|1|42| +2450822|5413|1|482| +2450822|5414|1|948| +2450822|5416|1|89| +2450822|5419|1|311| +2450822|5420|1|| +2450822|5422|1|476| +2450822|5425|1|667| +2450822|5426|1|631| +2450822|5428|1|670| +2450822|5431|1|16| +2450822|5432|1|435| +2450822|5434|1|84| +2450822|5437|1|707| +2450822|5438|1|372| +2450822|5440|1|904| +2450822|5443|1|298| +2450822|5444|1|170| +2450822|5446|1|213| +2450822|5449|1|296| +2450822|5450|1|549| +2450822|5452|1|550| +2450822|5455|1|| +2450822|5456|1|255| +2450822|5458|1|608| +2450822|5461|1|29| +2450822|5462|1|232| +2450822|5464|1|714| +2450822|5467|1|228| +2450822|5468|1|891| +2450822|5470|1|692| +2450822|5473|1|788| +2450822|5474|1|468| +2450822|5476|1|333| +2450822|5479|1|766| +2450822|5480|1|6| +2450822|5482|1|543| +2450822|5485|1|299| +2450822|5486|1|280| +2450822|5488|1|91| +2450822|5491|1|| +2450822|5492|1|205| +2450822|5494|1|198| +2450822|5497|1|| +2450822|5498|1|160| +2450822|5500|1|251| +2450822|5503|1|296| +2450822|5504|1|968| +2450822|5506|1|599| +2450822|5509|1|258| +2450822|5510|1|859| +2450822|5512|1|279| +2450822|5515|1|228| +2450822|5516|1|864| +2450822|5518|1|134| +2450822|5521|1|361| +2450822|5522|1|60| +2450822|5524|1|398| +2450822|5527|1|957| +2450822|5528|1|366| +2450822|5530|1|420| +2450822|5533|1|446| +2450822|5534|1|496| +2450822|5536|1|372| +2450822|5539|1|111| +2450822|5540|1|317| +2450822|5542|1|215| +2450822|5545|1|346| +2450822|5546|1|772| +2450822|5548|1|651| +2450822|5551|1|86| +2450822|5552|1|883| +2450822|5554|1|197| +2450822|5557|1|443| +2450822|5558|1|365| +2450822|5560|1|729| +2450822|5563|1|30| +2450822|5564|1|869| +2450822|5566|1|564| +2450822|5569|1|375| +2450822|5570|1|684| +2450822|5572|1|592| +2450822|5575|1|253| +2450822|5576|1|526| +2450822|5578|1|487| +2450822|5581|1|996| +2450822|5582|1|718| +2450822|5584|1|342| +2450822|5587|1|899| +2450822|5588|1|806| +2450822|5590|1|550| +2450822|5593|1|| +2450822|5594|1|309| +2450822|5596|1|134| +2450822|5599|1|765| +2450822|5600|1|243| +2450822|5602|1|271| +2450822|5605|1|583| +2450822|5606|1|356| +2450822|5608|1|613| +2450822|5611|1|252| +2450822|5612|1|591| +2450822|5614|1|919| +2450822|5617|1|674| +2450822|5618|1|688| +2450822|5620|1|793| +2450822|5623|1|965| +2450822|5624|1|222| +2450822|5626|1|241| +2450822|5629|1|517| +2450822|5630|1|68| +2450822|5632|1|822| +2450822|5635|1|554| +2450822|5636|1|722| +2450822|5638|1|644| +2450822|5641|1|948| +2450822|5642|1|966| +2450822|5644|1|557| +2450822|5647|1|816| +2450822|5648|1|934| +2450822|5650|1|262| +2450822|5653|1|583| +2450822|5654|1|143| +2450822|5656|1|915| +2450822|5659|1|910| +2450822|5660|1|814| +2450822|5662|1|684| +2450822|5665|1|803| +2450822|5666|1|674| +2450822|5668|1|69| +2450822|5671|1|40| +2450822|5672|1|606| +2450822|5674|1|264| +2450822|5677|1|790| +2450822|5678|1|860| +2450822|5680|1|413| +2450822|5683|1|213| +2450822|5684|1|613| +2450822|5686|1|61| +2450822|5689|1|679| +2450822|5690|1|866| +2450822|5692|1|374| +2450822|5695|1|887| +2450822|5696|1|408| +2450822|5698|1|190| +2450822|5701|1|387| +2450822|5702|1|739| +2450822|5704|1|289| +2450822|5707|1|229| +2450822|5708|1|840| +2450822|5710|1|365| +2450822|5713|1|400| +2450822|5714|1|296| +2450822|5716|1|436| +2450822|5719|1|| +2450822|5720|1|380| +2450822|5722|1|485| +2450822|5725|1|869| +2450822|5726|1|672| +2450822|5728|1|471| +2450822|5731|1|10| +2450822|5732|1|200| +2450822|5734|1|361| +2450822|5737|1|372| +2450822|5738|1|301| +2450822|5740|1|331| +2450822|5743|1|971| +2450822|5744|1|155| +2450822|5746|1|615| +2450822|5749|1|816| +2450822|5750|1|247| +2450822|5752|1|911| +2450822|5755|1|414| +2450822|5756|1|450| +2450822|5758|1|396| +2450822|5761|1|962| +2450822|5762|1|444| +2450822|5764|1|151| +2450822|5767|1|977| +2450822|5768|1|292| +2450822|5770|1|330| +2450822|5773|1|552| +2450822|5774|1|869| +2450822|5776|1|122| +2450822|5779|1|951| +2450822|5780|1|253| +2450822|5782|1|769| +2450822|5785|1|639| +2450822|5786|1|626| +2450822|5788|1|231| +2450822|5791|1|287| +2450822|5792|1|626| +2450822|5794|1|359| +2450822|5797|1|11| +2450822|5798|1|57| +2450822|5800|1|321| +2450822|5803|1|845| +2450822|5804|1|589| +2450822|5806|1|442| +2450822|5809|1|124| +2450822|5810|1|320| +2450822|5812|1|953| +2450822|5815|1|583| +2450822|5816|1|832| +2450822|5818|1|186| +2450822|5821|1|534| +2450822|5822|1|618| +2450822|5824|1|413| +2450822|5827|1|119| +2450822|5828|1|334| +2450822|5830|1|344| +2450822|5833|1|434| +2450822|5834|1|735| +2450822|5836|1|836| +2450822|5839|1|513| +2450822|5840|1|271| +2450822|5842|1|947| +2450822|5845|1|645| +2450822|5846|1|712| +2450822|5848|1|680| +2450822|5851|1|440| +2450822|5852|1|844| +2450822|5854|1|751| +2450822|5857|1|489| +2450822|5858|1|53| +2450822|5860|1|179| +2450822|5863|1|| +2450822|5864|1|13| +2450822|5866|1|830| +2450822|5869|1|785| +2450822|5870|1|138| +2450822|5872|1|715| +2450822|5875|1|777| +2450822|5876|1|924| +2450822|5878|1|464| +2450822|5881|1|944| +2450822|5882|1|442| +2450822|5884|1|136| +2450822|5887|1|623| +2450822|5888|1|896| +2450822|5890|1|86| +2450822|5893|1|454| +2450822|5894|1|111| +2450822|5896|1|459| +2450822|5899|1|571| +2450822|5900|1|704| +2450822|5902|1|547| +2450822|5905|1|764| +2450822|5906|1|468| +2450822|5908|1|912| +2450822|5911|1|402| +2450822|5912|1|924| +2450822|5914|1|855| +2450822|5917|1|612| +2450822|5918|1|266| +2450822|5920|1|905| +2450822|5923|1|556| +2450822|5924|1|125| +2450822|5926|1|298| +2450822|5929|1|703| +2450822|5930|1|975| +2450822|5932|1|586| +2450822|5935|1|154| +2450822|5936|1|697| +2450822|5938|1|816| +2450822|5941|1|271| +2450822|5942|1|525| +2450822|5944|1|401| +2450822|5947|1|313| +2450822|5948|1|562| +2450822|5950|1|402| +2450822|5953|1|649| +2450822|5954|1|535| +2450822|5956|1|285| +2450822|5959|1|780| +2450822|5960|1|628| +2450822|5962|1|283| +2450822|5965|1|726| +2450822|5966|1|133| +2450822|5968|1|| +2450822|5971|1|978| +2450822|5972|1|476| +2450822|5974|1|610| +2450822|5977|1|862| +2450822|5978|1|| +2450822|5980|1|722| +2450822|5983|1|514| +2450822|5984|1|| +2450822|5986|1|177| +2450822|5989|1|22| +2450822|5990|1|76| +2450822|5992|1|349| +2450822|5995|1|837| +2450822|5996|1|315| +2450822|5998|1|350| +2450822|6001|1|479| +2450822|6002|1|632| +2450822|6004|1|436| +2450822|6007|1|132| +2450822|6008|1|750| +2450822|6010|1|649| +2450822|6013|1|366| +2450822|6014|1|383| +2450822|6016|1|46| +2450822|6019|1|415| +2450822|6020|1|527| +2450822|6022|1|565| +2450822|6025|1|501| +2450822|6026|1|155| +2450822|6028|1|963| +2450822|6031|1|538| +2450822|6032|1|276| +2450822|6034|1|305| +2450822|6037|1|939| +2450822|6038|1|411| +2450822|6040|1|485| +2450822|6043|1|946| +2450822|6044|1|708| +2450822|6046|1|73| +2450822|6049|1|533| +2450822|6050|1|820| +2450822|6052|1|692| +2450822|6055|1|138| +2450822|6056|1|139| +2450822|6058|1|1000| +2450822|6061|1|735| +2450822|6062|1|580| +2450822|6064|1|| +2450822|6067|1|606| +2450822|6068|1|671| +2450822|6070|1|555| +2450822|6073|1|15| +2450822|6074|1|233| +2450822|6076|1|914| +2450822|6079|1|738| +2450822|6080|1|743| +2450822|6082|1|940| +2450822|6085|1|| +2450822|6086|1|104| +2450822|6088|1|812| +2450822|6091|1|603| +2450822|6092|1|864| +2450822|6094|1|865| +2450822|6097|1|368| +2450822|6098|1|881| +2450822|6100|1|562| +2450822|6103|1|126| +2450822|6104|1|230| +2450822|6106|1|359| +2450822|6109|1|346| +2450822|6110|1|146| +2450822|6112|1|| +2450822|6115|1|69| +2450822|6116|1|368| +2450822|6118|1|386| +2450822|6121|1|774| +2450822|6122|1|101| +2450822|6124|1|465| +2450822|6127|1|451| +2450822|6128|1|637| +2450822|6130|1|835| +2450822|6133|1|605| +2450822|6134|1|556| +2450822|6136|1|545| +2450822|6139|1|683| +2450822|6140|1|521| +2450822|6142|1|806| +2450822|6145|1|393| +2450822|6146|1|316| +2450822|6148|1|846| +2450822|6151|1|19| +2450822|6152|1|784| +2450822|6154|1|90| +2450822|6157|1|715| +2450822|6158|1|396| +2450822|6160|1|771| +2450822|6163|1|913| +2450822|6164|1|432| +2450822|6166|1|826| +2450822|6169|1|250| +2450822|6170|1|254| +2450822|6172|1|876| +2450822|6175|1|| +2450822|6176|1|232| +2450822|6178|1|920| +2450822|6181|1|978| +2450822|6182|1|43| +2450822|6184|1|995| +2450822|6187|1|762| +2450822|6188|1|643| +2450822|6190|1|414| +2450822|6193|1|894| +2450822|6194|1|957| +2450822|6196|1|982| +2450822|6199|1|489| +2450822|6200|1|397| +2450822|6202|1|848| +2450822|6205|1|| +2450822|6206|1|231| +2450822|6208|1|678| +2450822|6211|1|669| +2450822|6212|1|373| +2450822|6214|1|506| +2450822|6217|1|497| +2450822|6218|1|243| +2450822|6220|1|503| +2450822|6223|1|329| +2450822|6224|1|255| +2450822|6226|1|906| +2450822|6229|1|264| +2450822|6230|1|782| +2450822|6232|1|171| +2450822|6235|1|33| +2450822|6236|1|868| +2450822|6238|1|555| +2450822|6241|1|355| +2450822|6242|1|283| +2450822|6244|1|961| +2450822|6247|1|928| +2450822|6248|1|605| +2450822|6250|1|400| +2450822|6253|1|11| +2450822|6254|1|| +2450822|6256|1|420| +2450822|6259|1|172| +2450822|6260|1|868| +2450822|6262|1|716| +2450822|6265|1|289| +2450822|6266|1|746| +2450822|6268|1|314| +2450822|6271|1|663| +2450822|6272|1|394| +2450822|6274|1|891| +2450822|6277|1|392| +2450822|6278|1|368| +2450822|6280|1|127| +2450822|6283|1|751| +2450822|6284|1|604| +2450822|6286|1|745| +2450822|6289|1|31| +2450822|6290|1|924| +2450822|6292|1|989| +2450822|6295|1|275| +2450822|6296|1|143| +2450822|6298|1|247| +2450822|6301|1|689| +2450822|6302|1|246| +2450822|6304|1|137| +2450822|6307|1|950| +2450822|6308|1|812| +2450822|6310|1|500| +2450822|6313|1|| +2450822|6314|1|248| +2450822|6316|1|833| +2450822|6319|1|564| +2450822|6320|1|129| +2450822|6322|1|945| +2450822|6325|1|790| +2450822|6326|1|116| +2450822|6328|1|471| +2450822|6331|1|| +2450822|6332|1|160| +2450822|6334|1|431| +2450822|6337|1|178| +2450822|6338|1|560| +2450822|6340|1|301| +2450822|6343|1|529| +2450822|6344|1|588| +2450822|6346|1|227| +2450822|6349|1|849| +2450822|6350|1|368| +2450822|6352|1|233| +2450822|6355|1|268| +2450822|6356|1|29| +2450822|6358|1|614| +2450822|6361|1|40| +2450822|6362|1|814| +2450822|6364|1|786| +2450822|6367|1|206| +2450822|6368|1|33| +2450822|6370|1|580| +2450822|6373|1|739| +2450822|6374|1|458| +2450822|6376|1|548| +2450822|6379|1|551| +2450822|6380|1|136| +2450822|6382|1|436| +2450822|6385|1|448| +2450822|6386|1|482| +2450822|6388|1|917| +2450822|6391|1|73| +2450822|6392|1|12| +2450822|6394|1|28| +2450822|6397|1|173| +2450822|6398|1|803| +2450822|6400|1|570| +2450822|6403|1|29| +2450822|6404|1|876| +2450822|6406|1|803| +2450822|6409|1|125| +2450822|6410|1|291| +2450822|6412|1|870| +2450822|6415|1|102| +2450822|6416|1|72| +2450822|6418|1|98| +2450822|6421|1|356| +2450822|6422|1|938| +2450822|6424|1|605| +2450822|6427|1|119| +2450822|6428|1|418| +2450822|6430|1|561| +2450822|6433|1|769| +2450822|6434|1|113| +2450822|6436|1|155| +2450822|6439|1|537| +2450822|6440|1|782| +2450822|6442|1|797| +2450822|6445|1|653| +2450822|6446|1|452| +2450822|6448|1|945| +2450822|6451|1|767| +2450822|6452|1|59| +2450822|6454|1|324| +2450822|6457|1|460| +2450822|6458|1|9| +2450822|6460|1|209| +2450822|6463|1|308| +2450822|6464|1|252| +2450822|6466|1|510| +2450822|6469|1|552| +2450822|6470|1|737| +2450822|6472|1|883| +2450822|6475|1|445| +2450822|6476|1|834| +2450822|6478|1|48| +2450822|6481|1|264| +2450822|6482|1|456| +2450822|6484|1|828| +2450822|6487|1|794| +2450822|6488|1|41| +2450822|6490|1|195| +2450822|6493|1|233| +2450822|6494|1|565| +2450822|6496|1|982| +2450822|6499|1|212| +2450822|6500|1|869| +2450822|6502|1|542| +2450822|6505|1|557| +2450822|6506|1|229| +2450822|6508|1|954| +2450822|6511|1|955| +2450822|6512|1|825| +2450822|6514|1|755| +2450822|6517|1|119| +2450822|6518|1|860| +2450822|6520|1|| +2450822|6523|1|353| +2450822|6524|1|44| +2450822|6526|1|96| +2450822|6529|1|28| +2450822|6530|1|825| +2450822|6532|1|164| +2450822|6535|1|977| +2450822|6536|1|764| +2450822|6538|1|933| +2450822|6541|1|507| +2450822|6542|1|960| +2450822|6544|1|611| +2450822|6547|1|81| +2450822|6548|1|699| +2450822|6550|1|112| +2450822|6553|1|331| +2450822|6554|1|473| +2450822|6556|1|692| +2450822|6559|1|261| +2450822|6560|1|665| +2450822|6562|1|256| +2450822|6565|1|398| +2450822|6566|1|775| +2450822|6568|1|301| +2450822|6571|1|252| +2450822|6572|1|952| +2450822|6574|1|730| +2450822|6577|1|407| +2450822|6578|1|381| +2450822|6580|1|| +2450822|6583|1|6| +2450822|6584|1|27| +2450822|6586|1|104| +2450822|6589|1|260| +2450822|6590|1|630| +2450822|6592|1|333| +2450822|6595|1|847| +2450822|6596|1|118| +2450822|6598|1|193| +2450822|6601|1|474| +2450822|6602|1|| +2450822|6604|1|481| +2450822|6607|1|484| +2450822|6608|1|260| +2450822|6610|1|910| +2450822|6613|1|947| +2450822|6614|1|232| +2450822|6616|1|156| +2450822|6619|1|766| +2450822|6620|1|| +2450822|6622|1|261| +2450822|6625|1|| +2450822|6626|1|516| +2450822|6628|1|| +2450822|6631|1|812| +2450822|6632|1|| +2450822|6634|1|226| +2450822|6637|1|421| +2450822|6638|1|789| +2450822|6640|1|305| +2450822|6643|1|988| +2450822|6644|1|69| +2450822|6646|1|196| +2450822|6649|1|129| +2450822|6650|1|934| +2450822|6652|1|776| +2450822|6655|1|| +2450822|6656|1|892| +2450822|6658|1|716| +2450822|6661|1|728| +2450822|6662|1|870| +2450822|6664|1|94| +2450822|6667|1|| +2450822|6668|1|608| +2450822|6670|1|462| +2450822|6673|1|28| +2450822|6674|1|544| +2450822|6676|1|170| +2450822|6679|1|673| +2450822|6680|1|520| +2450822|6682|1|284| +2450822|6685|1|174| +2450822|6686|1|674| +2450822|6688|1|846| +2450822|6691|1|165| +2450822|6692|1|873| +2450822|6694|1|248| +2450822|6697|1|876| +2450822|6698|1|459| +2450822|6700|1|944| +2450822|6703|1|621| +2450822|6704|1|681| +2450822|6706|1|76| +2450822|6709|1|916| +2450822|6710|1|337| +2450822|6712|1|987| +2450822|6715|1|257| +2450822|6716|1|505| +2450822|6718|1|| +2450822|6721|1|580| +2450822|6722|1|630| +2450822|6724|1|698| +2450822|6727|1|888| +2450822|6728|1|954| +2450822|6730|1|233| +2450822|6733|1|734| +2450822|6734|1|780| +2450822|6736|1|12| +2450822|6739|1|399| +2450822|6740|1|888| +2450822|6742|1|385| +2450822|6745|1|729| +2450822|6746|1|443| +2450822|6748|1|140| +2450822|6751|1|195| +2450822|6752|1|997| +2450822|6754|1|638| +2450822|6757|1|891| +2450822|6758|1|665| +2450822|6760|1|240| +2450822|6763|1|719| +2450822|6764|1|822| +2450822|6766|1|148| +2450822|6769|1|252| +2450822|6770|1|817| +2450822|6772|1|708| +2450822|6775|1|712| +2450822|6776|1|787| +2450822|6778|1|649| +2450822|6781|1|241| +2450822|6782|1|988| +2450822|6784|1|879| +2450822|6787|1|748| +2450822|6788|1|349| +2450822|6790|1|71| +2450822|6793|1|427| +2450822|6794|1|865| +2450822|6796|1|905| +2450822|6799|1|973| +2450822|6800|1|265| +2450822|6802|1|16| +2450822|6805|1|787| +2450822|6806|1|398| +2450822|6808|1|422| +2450822|6811|1|763| +2450822|6812|1|549| +2450822|6814|1|| +2450822|6817|1|491| +2450822|6818|1|393| +2450822|6820|1|241| +2450822|6823|1|235| +2450822|6824|1|72| +2450822|6826|1|75| +2450822|6829|1|526| +2450822|6830|1|241| +2450822|6832|1|812| +2450822|6835|1|863| +2450822|6836|1|747| +2450822|6838|1|29| +2450822|6841|1|566| +2450822|6842|1|906| +2450822|6844|1|158| +2450822|6847|1|362| +2450822|6848|1|666| +2450822|6850|1|739| +2450822|6853|1|155| +2450822|6854|1|143| +2450822|6856|1|131| +2450822|6859|1|610| +2450822|6860|1|819| +2450822|6862|1|729| +2450822|6865|1|565| +2450822|6866|1|506| +2450822|6868|1|422| +2450822|6871|1|399| +2450822|6872|1|534| +2450822|6874|1|600| +2450822|6877|1|160| +2450822|6878|1|10| +2450822|6880|1|127| +2450822|6883|1|963| +2450822|6884|1|733| +2450822|6886|1|4| +2450822|6889|1|518| +2450822|6890|1|697| +2450822|6892|1|652| +2450822|6895|1|748| +2450822|6896|1|5| +2450822|6898|1|52| +2450822|6901|1|407| +2450822|6902|1|678| +2450822|6904|1|702| +2450822|6907|1|184| +2450822|6908|1|788| +2450822|6910|1|927| +2450822|6913|1|913| +2450822|6914|1|127| +2450822|6916|1|832| +2450822|6919|1|491| +2450822|6920|1|708| +2450822|6922|1|878| +2450822|6925|1|796| +2450822|6926|1|300| +2450822|6928|1|491| +2450822|6931|1|393| +2450822|6932|1|729| +2450822|6934|1|202| +2450822|6937|1|959| +2450822|6938|1|| +2450822|6940|1|| +2450822|6943|1|924| +2450822|6944|1|405| +2450822|6946|1|316| +2450822|6949|1|133| +2450822|6950|1|370| +2450822|6952|1|458| +2450822|6955|1|892| +2450822|6956|1|301| +2450822|6958|1|492| +2450822|6961|1|779| +2450822|6962|1|197| +2450822|6964|1|330| +2450822|6967|1|593| +2450822|6968|1|235| +2450822|6970|1|307| +2450822|6973|1|553| +2450822|6974|1|75| +2450822|6976|1|312| +2450822|6979|1|19| +2450822|6980|1|131| +2450822|6982|1|975| +2450822|6985|1|31| +2450822|6986|1|545| +2450822|6988|1|427| +2450822|6991|1|230| +2450822|6992|1|688| +2450822|6994|1|527| +2450822|6997|1|| +2450822|6998|1|589| +2450822|7000|1|750| +2450822|7003|1|588| +2450822|7004|1|206| +2450822|7006|1|62| +2450822|7009|1|769| +2450822|7010|1|464| +2450822|7012|1|729| +2450822|7015|1|833| +2450822|7016|1|349| +2450822|7018|1|385| +2450822|7021|1|| +2450822|7022|1|223| +2450822|7024|1|599| +2450822|7027|1|867| +2450822|7028|1|335| +2450822|7030|1|229| +2450822|7033|1|| +2450822|7034|1|415| +2450822|7036|1|53| +2450822|7039|1|29| +2450822|7040|1|740| +2450822|7042|1|| +2450822|7045|1|98| +2450822|7046|1|375| +2450822|7048|1|777| +2450822|7051|1|514| +2450822|7052|1|842| +2450822|7054|1|147| +2450822|7057|1|297| +2450822|7058|1|90| +2450822|7060|1|961| +2450822|7063|1|923| +2450822|7064|1|592| +2450822|7066|1|424| +2450822|7069|1|141| +2450822|7070|1|201| +2450822|7072|1|455| +2450822|7075|1|712| +2450822|7076|1|938| +2450822|7078|1|584| +2450822|7081|1|166| +2450822|7082|1|840| +2450822|7084|1|651| +2450822|7087|1|98| +2450822|7088|1|| +2450822|7090|1|546| +2450822|7093|1|| +2450822|7094|1|249| +2450822|7096|1|792| +2450822|7099|1|33| +2450822|7100|1|390| +2450822|7102|1|196| +2450822|7105|1|714| +2450822|7106|1|785| +2450822|7108|1|7| +2450822|7111|1|806| +2450822|7112|1|918| +2450822|7114|1|| +2450822|7117|1|481| +2450822|7118|1|500| +2450822|7120|1|829| +2450822|7123|1|865| +2450822|7124|1|42| +2450822|7126|1|618| +2450822|7129|1|935| +2450822|7130|1|| +2450822|7132|1|103| +2450822|7135|1|683| +2450822|7136|1|680| +2450822|7138|1|973| +2450822|7141|1|885| +2450822|7142|1|53| +2450822|7144|1|774| +2450822|7147|1|972| +2450822|7148|1|474| +2450822|7150|1|589| +2450822|7153|1|953| +2450822|7154|1|926| +2450822|7156|1|125| +2450822|7159|1|529| +2450822|7160|1|614| +2450822|7162|1|81| +2450822|7165|1|191| +2450822|7166|1|276| +2450822|7168|1|171| +2450822|7171|1|355| +2450822|7172|1|491| +2450822|7174|1|25| +2450822|7177|1|276| +2450822|7178|1|926| +2450822|7180|1|74| +2450822|7183|1|94| +2450822|7184|1|549| +2450822|7186|1|74| +2450822|7189|1|133| +2450822|7190|1|941| +2450822|7192|1|467| +2450822|7195|1|139| +2450822|7196|1|213| +2450822|7198|1|750| +2450822|7201|1|962| +2450822|7202|1|875| +2450822|7204|1|257| +2450822|7207|1|225| +2450822|7208|1|364| +2450822|7210|1|425| +2450822|7213|1|784| +2450822|7214|1|45| +2450822|7216|1|666| +2450822|7219|1|722| +2450822|7220|1|963| +2450822|7222|1|663| +2450822|7225|1|991| +2450822|7226|1|| +2450822|7228|1|813| +2450822|7231|1|897| +2450822|7232|1|58| +2450822|7234|1|895| +2450822|7237|1|90| +2450822|7238|1|977| +2450822|7240|1|463| +2450822|7243|1|481| +2450822|7244|1|460| +2450822|7246|1|249| +2450822|7249|1|85| +2450822|7250|1|834| +2450822|7252|1|25| +2450822|7255|1|73| +2450822|7256|1|150| +2450822|7258|1|562| +2450822|7261|1|146| +2450822|7262|1|749| +2450822|7264|1|| +2450822|7267|1|| +2450822|7268|1|926| +2450822|7270|1|998| +2450822|7273|1|586| +2450822|7274|1|641| +2450822|7276|1|817| +2450822|7279|1|922| +2450822|7280|1|134| +2450822|7282|1|205| +2450822|7285|1|716| +2450822|7286|1|547| +2450822|7288|1|424| +2450822|7291|1|441| +2450822|7292|1|467| +2450822|7294|1|629| +2450822|7297|1|417| +2450822|7298|1|104| +2450822|7300|1|339| +2450822|7303|1|447| +2450822|7304|1|216| +2450822|7306|1|401| +2450822|7309|1|180| +2450822|7310|1|180| +2450822|7312|1|54| +2450822|7315|1|38| +2450822|7316|1|604| +2450822|7318|1|634| +2450822|7321|1|653| +2450822|7322|1|351| +2450822|7324|1|10| +2450822|7327|1|649| +2450822|7328|1|330| +2450822|7330|1|268| +2450822|7333|1|1000| +2450822|7334|1|923| +2450822|7336|1|828| +2450822|7339|1|699| +2450822|7340|1|747| +2450822|7342|1|604| +2450822|7345|1|| +2450822|7346|1|541| +2450822|7348|1|91| +2450822|7351|1|248| +2450822|7352|1|795| +2450822|7354|1|221| +2450822|7357|1|873| +2450822|7358|1|116| +2450822|7360|1|444| +2450822|7363|1|503| +2450822|7364|1|727| +2450822|7366|1|349| +2450822|7369|1|603| +2450822|7370|1|678| +2450822|7372|1|913| +2450822|7375|1|308| +2450822|7376|1|905| +2450822|7378|1|265| +2450822|7381|1|802| +2450822|7382|1|197| +2450822|7384|1|53| +2450822|7387|1|201| +2450822|7388|1|507| +2450822|7390|1|859| +2450822|7393|1|687| +2450822|7394|1|310| +2450822|7396|1|910| +2450822|7399|1|851| +2450822|7400|1|82| +2450822|7402|1|350| +2450822|7405|1|569| +2450822|7406|1|37| +2450822|7408|1|400| +2450822|7411|1|231| +2450822|7412|1|375| +2450822|7414|1|594| +2450822|7417|1|| +2450822|7418|1|160| +2450822|7420|1|554| +2450822|7423|1|62| +2450822|7424|1|246| +2450822|7426|1|903| +2450822|7429|1|406| +2450822|7430|1|216| +2450822|7432|1|447| +2450822|7435|1|179| +2450822|7436|1|555| +2450822|7438|1|256| +2450822|7441|1|948| +2450822|7442|1|619| +2450822|7444|1|341| +2450822|7447|1|364| +2450822|7448|1|644| +2450822|7450|1|131| +2450822|7453|1|17| +2450822|7454|1|622| +2450822|7456|1|433| +2450822|7459|1|541| +2450822|7460|1|147| +2450822|7462|1|25| +2450822|7465|1|546| +2450822|7466|1|| +2450822|7468|1|461| +2450822|7471|1|| +2450822|7472|1|581| +2450822|7474|1|149| +2450822|7477|1|717| +2450822|7478|1|237| +2450822|7480|1|245| +2450822|7483|1|| +2450822|7484|1|273| +2450822|7486|1|407| +2450822|7489|1|478| +2450822|7490|1|108| +2450822|7492|1|266| +2450822|7495|1|710| +2450822|7496|1|133| +2450822|7498|1|345| +2450822|7501|1|403| +2450822|7502|1|810| +2450822|7504|1|614| +2450822|7507|1|858| +2450822|7508|1|29| +2450822|7510|1|596| +2450822|7513|1|107| +2450822|7514|1|914| +2450822|7516|1|822| +2450822|7519|1|77| +2450822|7520|1|610| +2450822|7522|1|| +2450822|7525|1|398| +2450822|7526|1|1| +2450822|7528|1|497| +2450822|7531|1|511| +2450822|7532|1|728| +2450822|7534|1|761| +2450822|7537|1|| +2450822|7538|1|660| +2450822|7540|1|732| +2450822|7543|1|745| +2450822|7544|1|50| +2450822|7546|1|60| +2450822|7549|1|177| +2450822|7550|1|366| +2450822|7552|1|487| +2450822|7555|1|488| +2450822|7556|1|924| +2450822|7558|1|| +2450822|7561|1|176| +2450822|7562|1|908| +2450822|7564|1|| +2450822|7567|1|60| +2450822|7568|1|52| +2450822|7570|1|679| +2450822|7573|1|52| +2450822|7574|1|251| +2450822|7576|1|319| +2450822|7579|1|243| +2450822|7580|1|749| +2450822|7582|1|542| +2450822|7585|1|726| +2450822|7586|1|334| +2450822|7588|1|603| +2450822|7591|1|586| +2450822|7592|1|928| +2450822|7594|1|777| +2450822|7597|1|507| +2450822|7598|1|180| +2450822|7600|1|522| +2450822|7603|1|825| +2450822|7604|1|298| +2450822|7606|1|274| +2450822|7609|1|516| +2450822|7610|1|847| +2450822|7612|1|870| +2450822|7615|1|155| +2450822|7616|1|812| +2450822|7618|1|| +2450822|7621|1|| +2450822|7622|1|797| +2450822|7624|1|176| +2450822|7627|1|223| +2450822|7628|1|238| +2450822|7630|1|971| +2450822|7633|1|767| +2450822|7634|1|| +2450822|7636|1|817| +2450822|7639|1|649| +2450822|7640|1|7| +2450822|7642|1|967| +2450822|7645|1|662| +2450822|7646|1|766| +2450822|7648|1|710| +2450822|7651|1|836| +2450822|7652|1|725| +2450822|7654|1|455| +2450822|7657|1|383| +2450822|7658|1|741| +2450822|7660|1|219| +2450822|7663|1|754| +2450822|7664|1|560| +2450822|7666|1|674| +2450822|7669|1|63| +2450822|7670|1|758| +2450822|7672|1|| +2450822|7675|1|758| +2450822|7676|1|700| +2450822|7678|1|352| +2450822|7681|1|876| +2450822|7682|1|341| +2450822|7684|1|186| +2450822|7687|1|758| +2450822|7688|1|111| +2450822|7690|1|| +2450822|7693|1|913| +2450822|7694|1|206| +2450822|7696|1|461| +2450822|7699|1|992| +2450822|7700|1|78| +2450822|7702|1|504| +2450822|7705|1|136| +2450822|7706|1|| +2450822|7708|1|266| +2450822|7711|1|413| +2450822|7712|1|923| +2450822|7714|1|690| +2450822|7717|1|888| +2450822|7718|1|231| +2450822|7720|1|81| +2450822|7723|1|220| +2450822|7724|1|98| +2450822|7726|1|265| +2450822|7729|1|205| +2450822|7730|1|941| +2450822|7732|1|619| +2450822|7735|1|| +2450822|7736|1|783| +2450822|7738|1|135| +2450822|7741|1|684| +2450822|7742|1|714| +2450822|7744|1|888| +2450822|7747|1|178| +2450822|7748|1|856| +2450822|7750|1|677| +2450822|7753|1|243| +2450822|7754|1|366| +2450822|7756|1|211| +2450822|7759|1|455| +2450822|7760|1|987| +2450822|7762|1|850| +2450822|7765|1|315| +2450822|7766|1|610| +2450822|7768|1|170| +2450822|7771|1|346| +2450822|7772|1|837| +2450822|7774|1|297| +2450822|7777|1|334| +2450822|7778|1|347| +2450822|7780|1|202| +2450822|7783|1|919| +2450822|7784|1|976| +2450822|7786|1|893| +2450822|7789|1|892| +2450822|7790|1|150| +2450822|7792|1|456| +2450822|7795|1|| +2450822|7796|1|283| +2450822|7798|1|814| +2450822|7801|1|151| +2450822|7802|1|112| +2450822|7804|1|319| +2450822|7807|1|876| +2450822|7808|1|809| +2450822|7810|1|91| +2450822|7813|1|386| +2450822|7814|1|455| +2450822|7816|1|579| +2450822|7819|1|573| +2450822|7820|1|650| +2450822|7822|1|474| +2450822|7825|1|859| +2450822|7826|1|469| +2450822|7828|1|674| +2450822|7831|1|785| +2450822|7832|1|666| +2450822|7834|1|777| +2450822|7837|1|15| +2450822|7838|1|647| +2450822|7840|1|635| +2450822|7843|1|55| +2450822|7844|1|568| +2450822|7846|1|102| +2450822|7849|1|709| +2450822|7850|1|529| +2450822|7852|1|249| +2450822|7855|1|610| +2450822|7856|1|744| +2450822|7858|1|| +2450822|7861|1|| +2450822|7862|1|719| +2450822|7864|1|565| +2450822|7867|1|178| +2450822|7868|1|842| +2450822|7870|1|637| +2450822|7873|1|636| +2450822|7874|1|994| +2450822|7876|1|99| +2450822|7879|1|579| +2450822|7880|1|119| +2450822|7882|1|577| +2450822|7885|1|510| +2450822|7886|1|816| +2450822|7888|1|891| +2450822|7891|1|124| +2450822|7892|1|673| +2450822|7894|1|622| +2450822|7897|1|159| +2450822|7898|1|423| +2450822|7900|1|64| +2450822|7903|1|35| +2450822|7904|1|819| +2450822|7906|1|404| +2450822|7909|1|675| +2450822|7910|1|297| +2450822|7912|1|762| +2450822|7915|1|478| +2450822|7916|1|182| +2450822|7918|1|552| +2450822|7921|1|23| +2450822|7922|1|| +2450822|7924|1|669| +2450822|7927|1|491| +2450822|7928|1|397| +2450822|7930|1|226| +2450822|7933|1|500| +2450822|7934|1|250| +2450822|7936|1|926| +2450822|7939|1|240| +2450822|7940|1|153| +2450822|7942|1|46| +2450822|7945|1|208| +2450822|7946|1|59| +2450822|7948|1|322| +2450822|7951|1|58| +2450822|7952|1|711| +2450822|7954|1|591| +2450822|7957|1|557| +2450822|7958|1|464| +2450822|7960|1|930| +2450822|7963|1|80| +2450822|7964|1|| +2450822|7966|1|76| +2450822|7969|1|157| +2450822|7970|1|332| +2450822|7972|1|289| +2450822|7975|1|860| +2450822|7976|1|538| +2450822|7978|1|877| +2450822|7981|1|817| +2450822|7982|1|106| +2450822|7984|1|437| +2450822|7987|1|678| +2450822|7988|1|| +2450822|7990|1|633| +2450822|7993|1|696| +2450822|7994|1|202| +2450822|7996|1|718| +2450822|7999|1|58| +2450822|8000|1|720| +2450822|8002|1|766| +2450822|8005|1|80| +2450822|8006|1|120| +2450822|8008|1|756| +2450822|8011|1|47| +2450822|8012|1|699| +2450822|8014|1|169| +2450822|8017|1|987| +2450822|8018|1|25| +2450822|8020|1|875| +2450822|8023|1|825| +2450822|8024|1|| +2450822|8026|1|154| +2450822|8029|1|430| +2450822|8030|1|726| +2450822|8032|1|520| +2450822|8035|1|901| +2450822|8036|1|716| +2450822|8038|1|700| +2450822|8041|1|643| +2450822|8042|1|226| +2450822|8044|1|91| +2450822|8047|1|40| +2450822|8048|1|708| +2450822|8050|1|101| +2450822|8053|1|951| +2450822|8054|1|765| +2450822|8056|1|144| +2450822|8059|1|283| +2450822|8060|1|622| +2450822|8062|1|32| +2450822|8065|1|97| +2450822|8066|1|487| +2450822|8068|1|321| +2450822|8071|1|381| +2450822|8072|1|924| +2450822|8074|1|577| +2450822|8077|1|509| +2450822|8078|1|946| +2450822|8080|1|695| +2450822|8083|1|183| +2450822|8084|1|145| +2450822|8086|1|511| +2450822|8089|1|853| +2450822|8090|1|652| +2450822|8092|1|726| +2450822|8095|1|788| +2450822|8096|1|475| +2450822|8098|1|424| +2450822|8101|1|172| +2450822|8102|1|83| +2450822|8104|1|462| +2450822|8107|1|216| +2450822|8108|1|962| +2450822|8110|1|381| +2450822|8113|1|186| +2450822|8114|1|714| +2450822|8116|1|7| +2450822|8119|1|591| +2450822|8120|1|424| +2450822|8122|1|692| +2450822|8125|1|677| +2450822|8126|1|385| +2450822|8128|1|712| +2450822|8131|1|189| +2450822|8132|1|| +2450822|8134|1|190| +2450822|8137|1|522| +2450822|8138|1|115| +2450822|8140|1|15| +2450822|8143|1|192| +2450822|8144|1|225| +2450822|8146|1|326| +2450822|8149|1|225| +2450822|8150|1|787| +2450822|8152|1|246| +2450822|8155|1|738| +2450822|8156|1|770| +2450822|8158|1|522| +2450822|8161|1|93| +2450822|8162|1|700| +2450822|8164|1|424| +2450822|8167|1|761| +2450822|8168|1|320| +2450822|8170|1|886| +2450822|8173|1|408| +2450822|8174|1|209| +2450822|8176|1|40| +2450822|8179|1|50| +2450822|8180|1|755| +2450822|8182|1|608| +2450822|8185|1|245| +2450822|8186|1|706| +2450822|8188|1|733| +2450822|8191|1|905| +2450822|8192|1|170| +2450822|8194|1|832| +2450822|8197|1|560| +2450822|8198|1|754| +2450822|8200|1|456| +2450822|8203|1|284| +2450822|8204|1|861| +2450822|8206|1|372| +2450822|8209|1|873| +2450822|8210|1|998| +2450822|8212|1|129| +2450822|8215|1|385| +2450822|8216|1|736| +2450822|8218|1|811| +2450822|8221|1|562| +2450822|8222|1|31| +2450822|8224|1|535| +2450822|8227|1|122| +2450822|8228|1|959| +2450822|8230|1|546| +2450822|8233|1|232| +2450822|8234|1|996| +2450822|8236|1|903| +2450822|8239|1|703| +2450822|8240|1|359| +2450822|8242|1|974| +2450822|8245|1|990| +2450822|8246|1|156| +2450822|8248|1|929| +2450822|8251|1|79| +2450822|8252|1|442| +2450822|8254|1|304| +2450822|8257|1|348| +2450822|8258|1|| +2450822|8260|1|636| +2450822|8263|1|468| +2450822|8264|1|848| +2450822|8266|1|| +2450822|8269|1|404| +2450822|8270|1|347| +2450822|8272|1|295| +2450822|8275|1|984| +2450822|8276|1|851| +2450822|8278|1|501| +2450822|8281|1|15| +2450822|8282|1|83| +2450822|8284|1|222| +2450822|8287|1|750| +2450822|8288|1|394| +2450822|8290|1|504| +2450822|8293|1|108| +2450822|8294|1|335| +2450822|8296|1|707| +2450822|8299|1|169| +2450822|8300|1|553| +2450822|8302|1|301| +2450822|8305|1|530| +2450822|8306|1|50| +2450822|8308|1|801| +2450822|8311|1|97| +2450822|8312|1|127| +2450822|8314|1|281| +2450822|8317|1|2| +2450822|8318|1|267| +2450822|8320|1|8| +2450822|8323|1|392| +2450822|8324|1|231| +2450822|8326|1|| +2450822|8329|1|489| +2450822|8330|1|| +2450822|8332|1|359| +2450822|8335|1|651| +2450822|8336|1|703| +2450822|8338|1|| +2450822|8341|1|273| +2450822|8342|1|77| +2450822|8344|1|971| +2450822|8347|1|249| +2450822|8348|1|374| +2450822|8350|1|214| +2450822|8353|1|537| +2450822|8354|1|763| +2450822|8356|1|35| +2450822|8359|1|| +2450822|8360|1|293| +2450822|8362|1|695| +2450822|8365|1|| +2450822|8366|1|299| +2450822|8368|1|453| +2450822|8371|1|691| +2450822|8372|1|555| +2450822|8374|1|125| +2450822|8377|1|189| +2450822|8378|1|918| +2450822|8380|1|976| +2450822|8383|1|633| +2450822|8384|1|922| +2450822|8386|1|680| +2450822|8389|1|46| +2450822|8390|1|571| +2450822|8392|1|576| +2450822|8395|1|219| +2450822|8396|1|771| +2450822|8398|1|619| +2450822|8401|1|155| +2450822|8402|1|908| +2450822|8404|1|938| +2450822|8407|1|137| +2450822|8408|1|| +2450822|8410|1|| +2450822|8413|1|511| +2450822|8414|1|766| +2450822|8416|1|| +2450822|8419|1|25| +2450822|8420|1|508| +2450822|8422|1|| +2450822|8425|1|43| +2450822|8426|1|540| +2450822|8428|1|610| +2450822|8431|1|139| +2450822|8432|1|635| +2450822|8434|1|58| +2450822|8437|1|762| +2450822|8438|1|685| +2450822|8440|1|324| +2450822|8443|1|9| +2450822|8444|1|414| +2450822|8446|1|205| +2450822|8449|1|880| +2450822|8450|1|386| +2450822|8452|1|62| +2450822|8455|1|463| +2450822|8456|1|295| +2450822|8458|1|508| +2450822|8461|1|604| +2450822|8462|1|69| +2450822|8464|1|423| +2450822|8467|1|520| +2450822|8468|1|221| +2450822|8470|1|621| +2450822|8473|1|820| +2450822|8474|1|825| +2450822|8476|1|898| +2450822|8479|1|733| +2450822|8480|1|393| +2450822|8482|1|0| +2450822|8485|1|804| +2450822|8486|1|655| +2450822|8488|1|42| +2450822|8491|1|292| +2450822|8492|1|939| +2450822|8494|1|643| +2450822|8497|1|580| +2450822|8498|1|162| +2450822|8500|1|38| +2450822|8503|1|| +2450822|8504|1|336| +2450822|8506|1|292| +2450822|8509|1|| +2450822|8510|1|716| +2450822|8512|1|138| +2450822|8515|1|184| +2450822|8516|1|385| +2450822|8518|1|457| +2450822|8521|1|330| +2450822|8522|1|39| +2450822|8524|1|494| +2450822|8527|1|443| +2450822|8528|1|671| +2450822|8530|1|636| +2450822|8533|1|467| +2450822|8534|1|14| +2450822|8536|1|737| +2450822|8539|1|997| +2450822|8540|1|636| +2450822|8542|1|669| +2450822|8545|1|112| +2450822|8546|1|319| +2450822|8548|1|484| +2450822|8551|1|142| +2450822|8552|1|622| +2450822|8554|1|231| +2450822|8557|1|293| +2450822|8558|1|355| +2450822|8560|1|626| +2450822|8563|1|| +2450822|8564|1|65| +2450822|8566|1|487| +2450822|8569|1|211| +2450822|8570|1|12| +2450822|8572|1|676| +2450822|8575|1|731| +2450822|8576|1|276| +2450822|8578|1|778| +2450822|8581|1|97| +2450822|8582|1|477| +2450822|8584|1|653| +2450822|8587|1|773| +2450822|8588|1|988| +2450822|8590|1|| +2450822|8593|1|844| +2450822|8594|1|300| +2450822|8596|1|855| +2450822|8599|1|669| +2450822|8600|1|| +2450822|8602|1|149| +2450822|8605|1|575| +2450822|8606|1|822| +2450822|8608|1|| +2450822|8611|1|716| +2450822|8612|1|937| +2450822|8614|1|970| +2450822|8617|1|131| +2450822|8618|1|984| +2450822|8620|1|876| +2450822|8623|1|448| +2450822|8624|1|827| +2450822|8626|1|727| +2450822|8629|1|882| +2450822|8630|1|30| +2450822|8632|1|487| +2450822|8635|1|21| +2450822|8636|1|778| +2450822|8638|1|152| +2450822|8641|1|465| +2450822|8642|1|821| +2450822|8644|1|73| +2450822|8647|1|590| +2450822|8648|1|599| +2450822|8650|1|34| +2450822|8653|1|289| +2450822|8654|1|573| +2450822|8656|1|29| +2450822|8659|1|359| +2450822|8660|1|855| +2450822|8662|1|666| +2450822|8665|1|352| +2450822|8666|1|115| +2450822|8668|1|197| +2450822|8671|1|939| +2450822|8672|1|867| +2450822|8674|1|269| +2450822|8677|1|713| +2450822|8678|1|66| +2450822|8680|1|442| +2450822|8683|1|225| +2450822|8684|1|653| +2450822|8686|1|820| +2450822|8689|1|583| +2450822|8690|1|| +2450822|8692|1|280| +2450822|8695|1|488| +2450822|8696|1|865| +2450822|8698|1|787| +2450822|8701|1|517| +2450822|8702|1|346| +2450822|8704|1|295| +2450822|8707|1|276| +2450822|8708|1|100| +2450822|8710|1|| +2450822|8713|1|844| +2450822|8714|1|| +2450822|8716|1|343| +2450822|8719|1|768| +2450822|8720|1|67| +2450822|8722|1|443| +2450822|8725|1|277| +2450822|8726|1|55| +2450822|8728|1|411| +2450822|8731|1|426| +2450822|8732|1|838| +2450822|8734|1|382| +2450822|8737|1|304| +2450822|8738|1|442| +2450822|8740|1|556| +2450822|8743|1|776| +2450822|8744|1|200| +2450822|8746|1|784| +2450822|8749|1|393| +2450822|8750|1|604| +2450822|8752|1|337| +2450822|8755|1|521| +2450822|8756|1|982| +2450822|8758|1|439| +2450822|8761|1|534| +2450822|8762|1|816| +2450822|8764|1|354| +2450822|8767|1|314| +2450822|8768|1|681| +2450822|8770|1|188| +2450822|8773|1|903| +2450822|8774|1|939| +2450822|8776|1|514| +2450822|8779|1|304| +2450822|8780|1|88| +2450822|8782|1|954| +2450822|8785|1|807| +2450822|8786|1|173| +2450822|8788|1|310| +2450822|8791|1|969| +2450822|8792|1|505| +2450822|8794|1|216| +2450822|8797|1|712| +2450822|8798|1|128| +2450822|8800|1|648| +2450822|8803|1|618| +2450822|8804|1|426| +2450822|8806|1|896| +2450822|8809|1|875| +2450822|8810|1|108| +2450822|8812|1|211| +2450822|8815|1|561| +2450822|8816|1|621| +2450822|8818|1|541| +2450822|8821|1|433| +2450822|8822|1|629| +2450822|8824|1|300| +2450822|8827|1|443| +2450822|8828|1|174| +2450822|8830|1|| +2450822|8833|1|887| +2450822|8834|1|867| +2450822|8836|1|759| +2450822|8839|1|671| +2450822|8840|1|| +2450822|8842|1|432| +2450822|8845|1|122| +2450822|8846|1|180| +2450822|8848|1|875| +2450822|8851|1|842| +2450822|8852|1|814| +2450822|8854|1|210| +2450822|8857|1|28| +2450822|8858|1|343| +2450822|8860|1|589| +2450822|8863|1|52| +2450822|8864|1|170| +2450822|8866|1|70| +2450822|8869|1|660| +2450822|8870|1|312| +2450822|8872|1|| +2450822|8875|1|900| +2450822|8876|1|140| +2450822|8878|1|| +2450822|8881|1|94| +2450822|8882|1|356| +2450822|8884|1|388| +2450822|8887|1|105| +2450822|8888|1|494| +2450822|8890|1|678| +2450822|8893|1|360| +2450822|8894|1|964| +2450822|8896|1|478| +2450822|8899|1|513| +2450822|8900|1|333| +2450822|8902|1|679| +2450822|8905|1|467| +2450822|8906|1|602| +2450822|8908|1|349| +2450822|8911|1|717| +2450822|8912|1|97| +2450822|8914|1|987| +2450822|8917|1|924| +2450822|8918|1|903| +2450822|8920|1|568| +2450822|8923|1|443| +2450822|8924|1|362| +2450822|8926|1|977| +2450822|8929|1|| +2450822|8930|1|302| +2450822|8932|1|284| +2450822|8935|1|479| +2450822|8936|1|918| +2450822|8938|1|534| +2450822|8941|1|968| +2450822|8942|1|595| +2450822|8944|1|899| +2450822|8947|1|848| +2450822|8948|1|588| +2450822|8950|1|606| +2450822|8953|1|996| +2450822|8954|1|559| +2450822|8956|1|763| +2450822|8959|1|747| +2450822|8960|1|897| +2450822|8962|1|367| +2450822|8965|1|419| +2450822|8966|1|260| +2450822|8968|1|577| +2450822|8971|1|36| +2450822|8972|1|607| +2450822|8974|1|421| +2450822|8977|1|356| +2450822|8978|1|831| +2450822|8980|1|517| +2450822|8983|1|652| +2450822|8984|1|527| +2450822|8986|1|694| +2450822|8989|1|824| +2450822|8990|1|215| +2450822|8992|1|618| +2450822|8995|1|588| +2450822|8996|1|704| +2450822|8998|1|352| +2450822|9001|1|448| +2450822|9002|1|292| +2450822|9004|1|501| +2450822|9007|1|123| +2450822|9008|1|270| +2450822|9010|1|417| +2450822|9013|1|310| +2450822|9014|1|501| +2450822|9016|1|930| +2450822|9019|1|| +2450822|9020|1|415| +2450822|9022|1|59| +2450822|9025|1|979| +2450822|9026|1|969| +2450822|9028|1|580| +2450822|9031|1|602| +2450822|9032|1|530| +2450822|9034|1|590| +2450822|9037|1|97| +2450822|9038|1|17| +2450822|9040|1|39| +2450822|9043|1|698| +2450822|9044|1|875| +2450822|9046|1|412| +2450822|9049|1|519| +2450822|9050|1|156| +2450822|9052|1|933| +2450822|9055|1|925| +2450822|9056|1|375| +2450822|9058|1|251| +2450822|9061|1|910| +2450822|9062|1|840| +2450822|9064|1|| +2450822|9067|1|267| +2450822|9068|1|103| +2450822|9070|1|764| +2450822|9073|1|478| +2450822|9074|1|585| +2450822|9076|1|197| +2450822|9079|1|154| +2450822|9080|1|679| +2450822|9082|1|972| +2450822|9085|1|3| +2450822|9086|1|136| +2450822|9088|1|839| +2450822|9091|1|119| +2450822|9092|1|485| +2450822|9094|1|875| +2450822|9097|1|976| +2450822|9098|1|296| +2450822|9100|1|303| +2450822|9103|1|289| +2450822|9104|1|283| +2450822|9106|1|222| +2450822|9109|1|928| +2450822|9110|1|| +2450822|9112|1|107| +2450822|9115|1|234| +2450822|9116|1|482| +2450822|9118|1|306| +2450822|9121|1|398| +2450822|9122|1|558| +2450822|9124|1|| +2450822|9127|1|409| +2450822|9128|1|615| +2450822|9130|1|991| +2450822|9133|1|274| +2450822|9134|1|| +2450822|9136|1|77| +2450822|9139|1|571| +2450822|9140|1|662| +2450822|9142|1|638| +2450822|9145|1|197| +2450822|9146|1|475| +2450822|9148|1|880| +2450822|9151|1|299| +2450822|9152|1|966| +2450822|9154|1|169| +2450822|9157|1|| +2450822|9158|1|617| +2450822|9160|1|772| +2450822|9163|1|186| +2450822|9164|1|| +2450822|9166|1|86| +2450822|9169|1|418| +2450822|9170|1|667| +2450822|9172|1|260| +2450822|9175|1|412| +2450822|9176|1|635| +2450822|9178|1|47| +2450822|9181|1|254| +2450822|9182|1|498| +2450822|9184|1|129| +2450822|9187|1|783| +2450822|9188|1|107| +2450822|9190|1|789| +2450822|9193|1|554| +2450822|9194|1|226| +2450822|9196|1|470| +2450822|9199|1|940| +2450822|9200|1|| +2450822|9202|1|573| +2450822|9205|1|975| +2450822|9206|1|415| +2450822|9208|1|409| +2450822|9211|1|488| +2450822|9212|1|589| +2450822|9214|1|602| +2450822|9217|1|751| +2450822|9218|1|430| +2450822|9220|1|491| +2450822|9223|1|245| +2450822|9224|1|400| +2450822|9226|1|982| +2450822|9229|1|930| +2450822|9230|1|231| +2450822|9232|1|228| +2450822|9235|1|745| +2450822|9236|1|872| +2450822|9238|1|161| +2450822|9241|1|542| +2450822|9242|1|942| +2450822|9244|1|537| +2450822|9247|1|7| +2450822|9248|1|613| +2450822|9250|1|383| +2450822|9253|1|228| +2450822|9254|1|| +2450822|9256|1|654| +2450822|9259|1|474| +2450822|9260|1|894| +2450822|9262|1|325| +2450822|9265|1|868| +2450822|9266|1|230| +2450822|9268|1|735| +2450822|9271|1|464| +2450822|9272|1|1000| +2450822|9274|1|561| +2450822|9277|1|868| +2450822|9278|1|401| +2450822|9280|1|723| +2450822|9283|1|210| +2450822|9284|1|681| +2450822|9286|1|494| +2450822|9289|1|164| +2450822|9290|1|460| +2450822|9292|1|327| +2450822|9295|1|| +2450822|9296|1|568| +2450822|9298|1|513| +2450822|9301|1|| +2450822|9302|1|14| +2450822|9304|1|820| +2450822|9307|1|| +2450822|9308|1|181| +2450822|9310|1|745| +2450822|9313|1|292| +2450822|9314|1|307| +2450822|9316|1|587| +2450822|9319|1|773| +2450822|9320|1|0| +2450822|9322|1|314| +2450822|9325|1|532| +2450822|9326|1|566| +2450822|9328|1|750| +2450822|9331|1|781| +2450822|9332|1|813| +2450822|9334|1|640| +2450822|9337|1|370| +2450822|9338|1|222| +2450822|9340|1|16| +2450822|9343|1|100| +2450822|9344|1|476| +2450822|9346|1|922| +2450822|9349|1|108| +2450822|9350|1|255| +2450822|9352|1|| +2450822|9355|1|563| +2450822|9356|1|| +2450822|9358|1|958| +2450822|9361|1|458| +2450822|9362|1|404| +2450822|9364|1|276| +2450822|9367|1|447| +2450822|9368|1|851| +2450822|9370|1|881| +2450822|9373|1|320| +2450822|9374|1|503| +2450822|9376|1|949| +2450822|9379|1|861| +2450822|9380|1|472| +2450822|9382|1|31| +2450822|9385|1|980| +2450822|9386|1|561| +2450822|9388|1|480| +2450822|9391|1|371| +2450822|9392|1|710| +2450822|9394|1|289| +2450822|9397|1|260| +2450822|9398|1|739| +2450822|9400|1|265| +2450822|9403|1|903| +2450822|9404|1|189| +2450822|9406|1|756| +2450822|9409|1|658| +2450822|9410|1|313| +2450822|9412|1|738| +2450822|9415|1|457| +2450822|9416|1|569| +2450822|9418|1|155| +2450822|9421|1|236| +2450822|9422|1|| +2450822|9424|1|285| +2450822|9427|1|978| +2450822|9428|1|291| +2450822|9430|1|632| +2450822|9433|1|498| +2450822|9434|1|680| +2450822|9436|1|912| +2450822|9439|1|57| +2450822|9440|1|558| +2450822|9442|1|259| +2450822|9445|1|311| +2450822|9446|1|226| +2450822|9448|1|| +2450822|9451|1|702| +2450822|9452|1|150| +2450822|9454|1|733| +2450822|9457|1|453| +2450822|9458|1|587| +2450822|9460|1|222| +2450822|9463|1|597| +2450822|9464|1|96| +2450822|9466|1|151| +2450822|9469|1|807| +2450822|9470|1|356| +2450822|9472|1|884| +2450822|9475|1|51| +2450822|9476|1|| +2450822|9478|1|737| +2450822|9481|1|463| +2450822|9482|1|761| +2450822|9484|1|229| +2450822|9487|1|565| +2450822|9488|1|| +2450822|9490|1|580| +2450822|9493|1|175| +2450822|9494|1|540| +2450822|9496|1|265| +2450822|9499|1|798| +2450822|9500|1|707| +2450822|9502|1|866| +2450822|9505|1|360| +2450822|9506|1|927| +2450822|9508|1|145| +2450822|9511|1|11| +2450822|9512|1|111| +2450822|9514|1|226| +2450822|9517|1|532| +2450822|9518|1|16| +2450822|9520|1|853| +2450822|9523|1|753| +2450822|9524|1|535| +2450822|9526|1|593| +2450822|9529|1|421| +2450822|9530|1|932| +2450822|9532|1|506| +2450822|9535|1|| +2450822|9536|1|| +2450822|9538|1|6| +2450822|9541|1|340| +2450822|9542|1|254| +2450822|9544|1|554| +2450822|9547|1|143| +2450822|9548|1|944| +2450822|9550|1|| +2450822|9553|1|693| +2450822|9554|1|303| +2450822|9556|1|705| +2450822|9559|1|228| +2450822|9560|1|956| +2450822|9562|1|398| +2450822|9565|1|410| +2450822|9566|1|589| +2450822|9568|1|472| +2450822|9571|1|927| +2450822|9572|1|671| +2450822|9574|1|25| +2450822|9577|1|301| +2450822|9578|1|271| +2450822|9580|1|459| +2450822|9583|1|618| +2450822|9584|1|223| +2450822|9586|1|985| +2450822|9589|1|185| +2450822|9590|1|933| +2450822|9592|1|454| +2450822|9595|1|290| +2450822|9596|1|670| +2450822|9598|1|501| +2450822|9601|1|755| +2450822|9602|1|302| +2450822|9604|1|248| +2450822|9607|1|673| +2450822|9608|1|587| +2450822|9610|1|87| +2450822|9613|1|830| +2450822|9614|1|672| +2450822|9616|1|673| +2450822|9619|1|611| +2450822|9620|1|| +2450822|9622|1|709| +2450822|9625|1|108| +2450822|9626|1|80| +2450822|9628|1|916| +2450822|9631|1|803| +2450822|9632|1|295| +2450822|9634|1|303| +2450822|9637|1|769| +2450822|9638|1|265| +2450822|9640|1|659| +2450822|9643|1|412| +2450822|9644|1|332| +2450822|9646|1|368| +2450822|9649|1|285| +2450822|9650|1|318| +2450822|9652|1|663| +2450822|9655|1|623| +2450822|9656|1|269| +2450822|9658|1|663| +2450822|9661|1|544| +2450822|9662|1|681| +2450822|9664|1|126| +2450822|9667|1|796| +2450822|9668|1|165| +2450822|9670|1|467| +2450822|9673|1|414| +2450822|9674|1|935| +2450822|9676|1|526| +2450822|9679|1|774| +2450822|9680|1|127| +2450822|9682|1|257| +2450822|9685|1|944| +2450822|9686|1|92| +2450822|9688|1|783| +2450822|9691|1|987| +2450822|9692|1|638| +2450822|9694|1|904| +2450822|9697|1|504| +2450822|9698|1|723| +2450822|9700|1|276| +2450822|9703|1|76| +2450822|9704|1|449| +2450822|9706|1|451| +2450822|9709|1|252| +2450822|9710|1|86| +2450822|9712|1|284| +2450822|9715|1|702| +2450822|9716|1|493| +2450822|9718|1|520| +2450822|9721|1|476| +2450822|9722|1|884| +2450822|9724|1|352| +2450822|9727|1|694| +2450822|9728|1|810| +2450822|9730|1|167| +2450822|9733|1|554| +2450822|9734|1|340| +2450822|9736|1|540| +2450822|9739|1|721| +2450822|9740|1|123| +2450822|9742|1|963| +2450822|9745|1|987| +2450822|9746|1|990| +2450822|9748|1|275| +2450822|9751|1|96| +2450822|9752|1|395| +2450822|9754|1|399| +2450822|9757|1|412| +2450822|9758|1|869| +2450822|9760|1|989| +2450822|9763|1|408| +2450822|9764|1|637| +2450822|9766|1|419| +2450822|9769|1|861| +2450822|9770|1|97| +2450822|9772|1|305| +2450822|9775|1|106| +2450822|9776|1|844| +2450822|9778|1|641| +2450822|9781|1|419| +2450822|9782|1|926| +2450822|9784|1|357| +2450822|9787|1|57| +2450822|9788|1|190| +2450822|9790|1|889| +2450822|9793|1|435| +2450822|9794|1|684| +2450822|9796|1|969| +2450822|9799|1|612| +2450822|9800|1|966| +2450822|9802|1|| +2450822|9805|1|581| +2450822|9806|1|815| +2450822|9808|1|986| +2450822|9811|1|775| +2450822|9812|1|871| +2450822|9814|1|278| +2450822|9817|1|375| +2450822|9818|1|282| +2450822|9820|1|744| +2450822|9823|1|620| +2450822|9824|1|506| +2450822|9826|1|927| +2450822|9829|1|348| +2450822|9830|1|379| +2450822|9832|1|1| +2450822|9835|1|173| +2450822|9836|1|873| +2450822|9838|1|343| +2450822|9841|1|262| +2450822|9842|1|234| +2450822|9844|1|369| +2450822|9847|1|250| +2450822|9848|1|376| +2450822|9850|1|800| +2450822|9853|1|326| +2450822|9854|1|540| +2450822|9856|1|745| +2450822|9859|1|166| +2450822|9860|1|490| +2450822|9862|1|298| +2450822|9865|1|90| +2450822|9866|1|659| +2450822|9868|1|545| +2450822|9871|1|| +2450822|9872|1|161| +2450822|9874|1|390| +2450822|9877|1|197| +2450822|9878|1|435| +2450822|9880|1|160| +2450822|9883|1|934| +2450822|9884|1|825| +2450822|9886|1|389| +2450822|9889|1|215| +2450822|9890|1|779| +2450822|9892|1|844| +2450822|9895|1|244| +2450822|9896|1|| +2450822|9898|1|501| +2450822|9901|1|651| +2450822|9902|1|289| +2450822|9904|1|174| +2450822|9907|1|993| +2450822|9908|1|24| +2450822|9910|1|496| +2450822|9913|1|456| +2450822|9914|1|813| +2450822|9916|1|901| +2450822|9919|1|690| +2450822|9920|1|579| +2450822|9922|1|681| +2450822|9925|1|796| +2450822|9926|1|365| +2450822|9928|1|618| +2450822|9931|1|685| +2450822|9932|1|846| +2450822|9934|1|647| +2450822|9937|1|179| +2450822|9938|1|332| +2450822|9940|1|234| +2450822|9943|1|959| +2450822|9944|1|847| +2450822|9946|1|| +2450822|9949|1|548| +2450822|9950|1|924| +2450822|9952|1|861| +2450822|9955|1|773| +2450822|9956|1|539| +2450822|9958|1|300| +2450822|9961|1|841| +2450822|9962|1|909| +2450822|9964|1|441| +2450822|9967|1|643| +2450822|9968|1|594| +2450822|9970|1|76| +2450822|9973|1|374| +2450822|9974|1|851| +2450822|9976|1|812| +2450822|9979|1|886| +2450822|9980|1|118| +2450822|9982|1|613| +2450822|9985|1|876| +2450822|9986|1|313| +2450822|9988|1|783| +2450822|9991|1|979| +2450822|9992|1|644| +2450822|9994|1|524| +2450822|9997|1|682| +2450822|9998|1|151| +2450822|10000|1|935| +2450822|10003|1|818| +2450822|10004|1|184| +2450822|10006|1|197| +2450822|10009|1|177| +2450822|10010|1|512| +2450822|10012|1|86| +2450822|10015|1|501| +2450822|10016|1|650| +2450822|10018|1|89| +2450822|10021|1|13| +2450822|10022|1|911| +2450822|10024|1|937| +2450822|10027|1|410| +2450822|10028|1|308| +2450822|10030|1|281| +2450822|10033|1|975| +2450822|10034|1|200| +2450822|10036|1|735| +2450822|10039|1|299| +2450822|10040|1|659| +2450822|10042|1|126| +2450822|10045|1|| +2450822|10046|1|741| +2450822|10048|1|361| +2450822|10051|1|608| +2450822|10052|1|| +2450822|10054|1|8| +2450822|10057|1|653| +2450822|10058|1|885| +2450822|10060|1|917| +2450822|10063|1|120| +2450822|10064|1|520| +2450822|10066|1|201| +2450822|10069|1|501| +2450822|10070|1|989| +2450822|10072|1|280| +2450822|10075|1|143| +2450822|10076|1|873| +2450822|10078|1|566| +2450822|10081|1|951| +2450822|10082|1|439| +2450822|10084|1|956| +2450822|10087|1|36| +2450822|10088|1|177| +2450822|10090|1|974| +2450822|10093|1|951| +2450822|10094|1|850| +2450822|10096|1|477| +2450822|10099|1|211| +2450822|10100|1|18| +2450822|10102|1|565| +2450822|10105|1|639| +2450822|10106|1|464| +2450822|10108|1|557| +2450822|10111|1|538| +2450822|10112|1|367| +2450822|10114|1|967| +2450822|10117|1|109| +2450822|10118|1|418| +2450822|10120|1|716| +2450822|10123|1|| +2450822|10124|1|295| +2450822|10126|1|839| +2450822|10129|1|437| +2450822|10130|1|361| +2450822|10132|1|294| +2450822|10135|1|740| +2450822|10136|1|186| +2450822|10138|1|448| +2450822|10141|1|94| +2450822|10142|1|276| +2450822|10144|1|373| +2450822|10147|1|931| +2450822|10148|1|97| +2450822|10150|1|284| +2450822|10153|1|888| +2450822|10154|1|567| +2450822|10156|1|469| +2450822|10159|1|912| +2450822|10160|1|99| +2450822|10162|1|582| +2450822|10165|1|632| +2450822|10166|1|271| +2450822|10168|1|717| +2450822|10171|1|689| +2450822|10172|1|230| +2450822|10174|1|| +2450822|10177|1|562| +2450822|10178|1|423| +2450822|10180|1|510| +2450822|10183|1|905| +2450822|10184|1|643| +2450822|10186|1|975| +2450822|10189|1|852| +2450822|10190|1|605| +2450822|10192|1|489| +2450822|10195|1|988| +2450822|10196|1|644| +2450822|10198|1|503| +2450822|10201|1|| +2450822|10202|1|477| +2450822|10204|1|948| +2450822|10207|1|368| +2450822|10208|1|943| +2450822|10210|1|831| +2450822|10213|1|266| +2450822|10214|1|33| +2450822|10216|1|371| +2450822|10219|1|190| +2450822|10220|1|702| +2450822|10222|1|152| +2450822|10225|1|313| +2450822|10226|1|410| +2450822|10228|1|783| +2450822|10231|1|578| +2450822|10232|1|120| +2450822|10234|1|668| +2450822|10237|1|216| +2450822|10238|1|546| +2450822|10240|1|549| +2450822|10243|1|512| +2450822|10244|1|754| +2450822|10246|1|966| +2450822|10249|1|976| +2450822|10250|1|202| +2450822|10252|1|825| +2450822|10255|1|665| +2450822|10256|1|366| +2450822|10258|1|797| +2450822|10261|1|239| +2450822|10262|1|489| +2450822|10264|1|314| +2450822|10267|1|| +2450822|10268|1|520| +2450822|10270|1|| +2450822|10273|1|765| +2450822|10274|1|754| +2450822|10276|1|| +2450822|10279|1|965| +2450822|10280|1|983| +2450822|10282|1|330| +2450822|10285|1|859| +2450822|10286|1|257| +2450822|10288|1|825| +2450822|10291|1|804| +2450822|10292|1|695| +2450822|10294|1|957| +2450822|10297|1|969| +2450822|10298|1|190| +2450822|10300|1|705| +2450822|10303|1|464| +2450822|10304|1|684| +2450822|10306|1|89| +2450822|10309|1|569| +2450822|10310|1|93| +2450822|10312|1|827| +2450822|10315|1|598| +2450822|10316|1|302| +2450822|10318|1|704| +2450822|10321|1|691| +2450822|10322|1|196| +2450822|10324|1|| +2450822|10327|1|553| +2450822|10328|1|695| +2450822|10330|1|| +2450822|10333|1|70| +2450822|10334|1|822| +2450822|10336|1|227| +2450822|10339|1|800| +2450822|10340|1|901| +2450822|10342|1|914| +2450822|10345|1|323| +2450822|10346|1|983| +2450822|10348|1|269| +2450822|10351|1|39| +2450822|10352|1|527| +2450822|10354|1|185| +2450822|10357|1|195| +2450822|10358|1|894| +2450822|10360|1|286| +2450822|10363|1|157| +2450822|10364|1|433| +2450822|10366|1|343| +2450822|10369|1|103| +2450822|10370|1|447| +2450822|10372|1|290| +2450822|10375|1|401| +2450822|10376|1|948| +2450822|10378|1|57| +2450822|10381|1|338| +2450822|10382|1|936| +2450822|10384|1|747| +2450822|10387|1|527| +2450822|10388|1|994| +2450822|10390|1|861| +2450822|10393|1|| +2450822|10394|1|166| +2450822|10396|1|796| +2450822|10399|1|308| +2450822|10400|1|963| +2450822|10402|1|718| +2450822|10405|1|703| +2450822|10406|1|197| +2450822|10408|1|| +2450822|10411|1|562| +2450822|10412|1|117| +2450822|10414|1|605| +2450822|10417|1|619| +2450822|10418|1|87| +2450822|10420|1|353| +2450822|10423|1|654| +2450822|10424|1|73| +2450822|10426|1|890| +2450822|10429|1|734| +2450822|10430|1|750| +2450822|10432|1|494| +2450822|10435|1|| +2450822|10436|1|116| +2450822|10438|1|451| +2450822|10441|1|155| +2450822|10442|1|197| +2450822|10444|1|741| +2450822|10447|1|64| +2450822|10448|1|111| +2450822|10450|1|| +2450822|10453|1|218| +2450822|10454|1|739| +2450822|10456|1|371| +2450822|10459|1|832| +2450822|10460|1|336| +2450822|10462|1|109| +2450822|10465|1|714| +2450822|10466|1|587| +2450822|10468|1|519| +2450822|10471|1|147| +2450822|10472|1|245| +2450822|10474|1|963| +2450822|10477|1|| +2450822|10478|1|91| +2450822|10480|1|588| +2450822|10483|1|717| +2450822|10484|1|510| +2450822|10486|1|617| +2450822|10489|1|277| +2450822|10490|1|330| +2450822|10492|1|| +2450822|10495|1|315| +2450822|10496|1|133| +2450822|10498|1|793| +2450822|10501|1|351| +2450822|10502|1|767| +2450822|10504|1|509| +2450822|10507|1|556| +2450822|10508|1|874| +2450822|10510|1|543| +2450822|10513|1|944| +2450822|10514|1|236| +2450822|10516|1|229| +2450822|10519|1|975| +2450822|10520|1|577| +2450822|10522|1|| +2450822|10525|1|749| +2450822|10526|1|356| +2450822|10528|1|853| +2450822|10531|1|718| +2450822|10532|1|918| +2450822|10534|1|22| +2450822|10537|1|560| +2450822|10538|1|396| +2450822|10540|1|754| +2450822|10543|1|165| +2450822|10544|1|687| +2450822|10546|1|909| +2450822|10549|1|275| +2450822|10550|1|455| +2450822|10552|1|284| +2450822|10555|1|941| +2450822|10556|1|55| +2450822|10558|1|574| +2450822|10561|1|366| +2450822|10562|1|683| +2450822|10564|1|56| +2450822|10567|1|392| +2450822|10568|1|476| +2450822|10570|1|612| +2450822|10573|1|762| +2450822|10574|1|276| +2450822|10576|1|160| +2450822|10579|1|431| +2450822|10580|1|421| +2450822|10582|1|406| +2450822|10585|1|64| +2450822|10586|1|484| +2450822|10588|1|110| +2450822|10591|1|624| +2450822|10592|1|631| +2450822|10594|1|79| +2450822|10597|1|810| +2450822|10598|1|620| +2450822|10600|1|766| +2450822|10603|1|806| +2450822|10604|1|290| +2450822|10606|1|| +2450822|10609|1|296| +2450822|10610|1|874| +2450822|10612|1|228| +2450822|10615|1|384| +2450822|10616|1|65| +2450822|10618|1|| +2450822|10621|1|370| +2450822|10622|1|642| +2450822|10624|1|494| +2450822|10627|1|77| +2450822|10628|1|751| +2450822|10630|1|344| +2450822|10633|1|445| +2450822|10634|1|10| +2450822|10636|1|642| +2450822|10639|1|183| +2450822|10640|1|29| +2450822|10642|1|580| +2450822|10645|1|883| +2450822|10646|1|702| +2450822|10648|1|850| +2450822|10651|1|801| +2450822|10652|1|107| +2450822|10654|1|523| +2450822|10657|1|976| +2450822|10658|1|958| +2450822|10660|1|203| +2450822|10663|1|661| +2450822|10664|1|445| +2450822|10666|1|224| +2450822|10669|1|217| +2450822|10670|1|363| +2450822|10672|1|940| +2450822|10675|1|712| +2450822|10676|1|31| +2450822|10678|1|574| +2450822|10681|1|962| +2450822|10682|1|185| +2450822|10684|1|322| +2450822|10687|1|986| +2450822|10688|1|134| +2450822|10690|1|27| +2450822|10693|1|109| +2450822|10694|1|497| +2450822|10696|1|52| +2450822|10699|1|67| +2450822|10700|1|548| +2450822|10702|1|565| +2450822|10705|1|502| +2450822|10706|1|680| +2450822|10708|1|823| +2450822|10711|1|847| +2450822|10712|1|744| +2450822|10714|1|721| +2450822|10717|1|183| +2450822|10718|1|59| +2450822|10720|1|287| +2450822|10723|1|455| +2450822|10724|1|173| +2450822|10726|1|105| +2450822|10729|1|160| +2450822|10730|1|| +2450822|10732|1|657| +2450822|10735|1|781| +2450822|10736|1|842| +2450822|10738|1|228| +2450822|10741|1|234| +2450822|10742|1|352| +2450822|10744|1|121| +2450822|10747|1|29| +2450822|10748|1|620| +2450822|10750|1|195| +2450822|10753|1|659| +2450822|10754|1|113| +2450822|10756|1|491| +2450822|10759|1|143| +2450822|10760|1|600| +2450822|10762|1|758| +2450822|10765|1|520| +2450822|10766|1|287| +2450822|10768|1|557| +2450822|10771|1|139| +2450822|10772|1|504| +2450822|10774|1|545| +2450822|10777|1|643| +2450822|10778|1|404| +2450822|10780|1|522| +2450822|10783|1|105| +2450822|10784|1|908| +2450822|10786|1|409| +2450822|10789|1|743| +2450822|10790|1|687| +2450822|10792|1|| +2450822|10795|1|424| +2450822|10796|1|633| +2450822|10798|1|900| +2450822|10801|1|10| +2450822|10802|1|155| +2450822|10804|1|952| +2450822|10807|1|552| +2450822|10808|1|| +2450822|10810|1|614| +2450822|10813|1|747| +2450822|10814|1|622| +2450822|10816|1|180| +2450822|10819|1|| +2450822|10820|1|123| +2450822|10822|1|283| +2450822|10825|1|337| +2450822|10826|1|820| +2450822|10828|1|635| +2450822|10831|1|303| +2450822|10832|1|683| +2450822|10834|1|155| +2450822|10837|1|694| +2450822|10838|1|209| +2450822|10840|1|579| +2450822|10843|1|891| +2450822|10844|1|917| +2450822|10846|1|650| +2450822|10849|1|275| +2450822|10850|1|846| +2450822|10852|1|536| +2450822|10855|1|733| +2450822|10856|1|658| +2450822|10858|1|818| +2450822|10861|1|943| +2450822|10862|1|350| +2450822|10864|1|294| +2450822|10867|1|584| +2450822|10868|1|74| +2450822|10870|1|448| +2450822|10873|1|898| +2450822|10874|1|732| +2450822|10876|1|467| +2450822|10879|1|973| +2450822|10880|1|520| +2450822|10882|1|624| +2450822|10885|1|414| +2450822|10886|1|126| +2450822|10888|1|202| +2450822|10891|1|914| +2450822|10892|1|741| +2450822|10894|1|62| +2450822|10897|1|48| +2450822|10898|1|91| +2450822|10900|1|120| +2450822|10903|1|597| +2450822|10904|1|164| +2450822|10906|1|709| +2450822|10909|1|| +2450822|10910|1|| +2450822|10912|1|277| +2450822|10915|1|80| +2450822|10916|1|653| +2450822|10918|1|923| +2450822|10921|1|755| +2450822|10922|1|107| +2450822|10924|1|815| +2450822|10927|1|560| +2450822|10928|1|371| +2450822|10930|1|799| +2450822|10933|1|925| +2450822|10934|1|818| +2450822|10936|1|711| +2450822|10939|1|725| +2450822|10940|1|507| +2450822|10942|1|543| +2450822|10945|1|404| +2450822|10946|1|331| +2450822|10948|1|561| +2450822|10951|1|572| +2450822|10952|1|429| +2450822|10954|1|663| +2450822|10957|1|647| +2450822|10958|1|325| +2450822|10960|1|392| +2450822|10963|1|416| +2450822|10964|1|934| +2450822|10966|1|964| +2450822|10969|1|10| +2450822|10970|1|403| +2450822|10972|1|462| +2450822|10975|1|627| +2450822|10976|1|| +2450822|10978|1|270| +2450822|10981|1|130| +2450822|10982|1|98| +2450822|10984|1|996| +2450822|10987|1|914| +2450822|10988|1|438| +2450822|10990|1|679| +2450822|10993|1|159| +2450822|10994|1|393| +2450822|10996|1|297| +2450822|10999|1|115| +2450822|11000|1|649| +2450822|11002|1|528| +2450822|11005|1|424| +2450822|11006|1|220| +2450822|11008|1|834| +2450822|11011|1|683| +2450822|11012|1|404| +2450822|11014|1|512| +2450822|11017|1|504| +2450822|11018|1|719| +2450822|11020|1|838| +2450822|11023|1|525| +2450822|11024|1|730| +2450822|11026|1|466| +2450822|11029|1|337| +2450822|11030|1|610| +2450822|11032|1|489| +2450822|11035|1|984| +2450822|11036|1|| +2450822|11038|1|675| +2450822|11041|1|46| +2450822|11042|1|61| +2450822|11044|1|309| +2450822|11047|1|339| +2450822|11048|1|710| +2450822|11050|1|363| +2450822|11053|1|710| +2450822|11054|1|331| +2450822|11056|1|627| +2450822|11059|1|| +2450822|11060|1|576| +2450822|11062|1|619| +2450822|11065|1|22| +2450822|11066|1|111| +2450822|11068|1|396| +2450822|11071|1|422| +2450822|11072|1|1000| +2450822|11074|1|598| +2450822|11077|1|215| +2450822|11078|1|458| +2450822|11080|1|135| +2450822|11083|1|789| +2450822|11084|1|365| +2450822|11086|1|85| +2450822|11089|1|584| +2450822|11090|1|477| +2450822|11092|1|706| +2450822|11095|1|436| +2450822|11096|1|661| +2450822|11098|1|332| +2450822|11101|1|392| +2450822|11102|1|919| +2450822|11104|1|509| +2450822|11107|1|991| +2450822|11108|1|814| +2450822|11110|1|137| +2450822|11113|1|941| +2450822|11114|1|| +2450822|11116|1|578| +2450822|11119|1|866| +2450822|11120|1|440| +2450822|11122|1|427| +2450822|11125|1|801| +2450822|11126|1|746| +2450822|11128|1|353| +2450822|11131|1|142| +2450822|11132|1|971| +2450822|11134|1|617| +2450822|11137|1|151| +2450822|11138|1|41| +2450822|11140|1|215| +2450822|11143|1|23| +2450822|11144|1|868| +2450822|11146|1|684| +2450822|11149|1|507| +2450822|11150|1|33| +2450822|11152|1|350| +2450822|11155|1|896| +2450822|11156|1|691| +2450822|11158|1|276| +2450822|11161|1|72| +2450822|11162|1|450| +2450822|11164|1|47| +2450822|11167|1|739| +2450822|11168|1|211| +2450822|11170|1|804| +2450822|11173|1|31| +2450822|11174|1|947| +2450822|11176|1|283| +2450822|11179|1|629| +2450822|11180|1|583| +2450822|11182|1|831| +2450822|11185|1|| +2450822|11186|1|385| +2450822|11188|1|839| +2450822|11191|1|22| +2450822|11192|1|479| +2450822|11194|1|293| +2450822|11197|1|524| +2450822|11198|1|599| +2450822|11200|1|921| +2450822|11203|1|| +2450822|11204|1|158| +2450822|11206|1|454| +2450822|11209|1|783| +2450822|11210|1|533| +2450822|11212|1|528| +2450822|11215|1|764| +2450822|11216|1|396| +2450822|11218|1|380| +2450822|11221|1|513| +2450822|11222|1|597| +2450822|11224|1|416| +2450822|11227|1|693| +2450822|11228|1|178| +2450822|11230|1|370| +2450822|11233|1|388| +2450822|11234|1|926| +2450822|11236|1|532| +2450822|11239|1|| +2450822|11240|1|775| +2450822|11242|1|538| +2450822|11245|1|719| +2450822|11246|1|725| +2450822|11248|1|596| +2450822|11251|1|608| +2450822|11252|1|380| +2450822|11254|1|952| +2450822|11257|1|336| +2450822|11258|1|16| +2450822|11260|1|316| +2450822|11263|1|301| +2450822|11264|1|107| +2450822|11266|1|1| +2450822|11269|1|581| +2450822|11270|1|464| +2450822|11272|1|3| +2450822|11275|1|79| +2450822|11276|1|| +2450822|11278|1|653| +2450822|11281|1|376| +2450822|11282|1|641| +2450822|11284|1|217| +2450822|11287|1|313| +2450822|11288|1|291| +2450822|11290|1|397| +2450822|11293|1|685| +2450822|11294|1|977| +2450822|11296|1|201| +2450822|11299|1|884| +2450822|11300|1|765| +2450822|11302|1|723| +2450822|11305|1|631| +2450822|11306|1|824| +2450822|11308|1|706| +2450822|11311|1|22| +2450822|11312|1|89| +2450822|11314|1|235| +2450822|11317|1|614| +2450822|11318|1|443| +2450822|11320|1|778| +2450822|11323|1|| +2450822|11324|1|204| +2450822|11326|1|1| +2450822|11329|1|873| +2450822|11330|1|416| +2450822|11332|1|961| +2450822|11335|1|| +2450822|11336|1|505| +2450822|11338|1|909| +2450822|11341|1|488| +2450822|11342|1|79| +2450822|11344|1|938| +2450822|11347|1|644| +2450822|11348|1|531| +2450822|11350|1|298| +2450822|11353|1|969| +2450822|11354|1|| +2450822|11356|1|502| +2450822|11359|1|898| +2450822|11360|1|562| +2450822|11362|1|486| +2450822|11365|1|461| +2450822|11366|1|1| +2450822|11368|1|634| +2450822|11371|1|484| +2450822|11372|1|345| +2450822|11374|1|804| +2450822|11377|1|98| +2450822|11378|1|| +2450822|11380|1|527| +2450822|11383|1|923| +2450822|11384|1|247| +2450822|11386|1|560| +2450822|11389|1|652| +2450822|11390|1|122| +2450822|11392|1|44| +2450822|11395|1|770| +2450822|11396|1|524| +2450822|11398|1|514| +2450822|11401|1|946| +2450822|11402|1|511| +2450822|11404|1|303| +2450822|11407|1|965| +2450822|11408|1|287| +2450822|11410|1|80| +2450822|11413|1|30| +2450822|11414|1|631| +2450822|11416|1|752| +2450822|11419|1|12| +2450822|11420|1|776| +2450822|11422|1|538| +2450822|11425|1|240| +2450822|11426|1|425| +2450822|11428|1|851| +2450822|11431|1|715| +2450822|11432|1|189| +2450822|11434|1|814| +2450822|11437|1|807| +2450822|11438|1|523| +2450822|11440|1|816| +2450822|11443|1|57| +2450822|11444|1|348| +2450822|11446|1|635| +2450822|11449|1|716| +2450822|11450|1|149| +2450822|11452|1|401| +2450822|11455|1|424| +2450822|11456|1|72| +2450822|11458|1|823| +2450822|11461|1|333| +2450822|11462|1|502| +2450822|11464|1|573| +2450822|11467|1|976| +2450822|11468|1|171| +2450822|11470|1|454| +2450822|11473|1|195| +2450822|11474|1|427| +2450822|11476|1|| +2450822|11479|1|390| +2450822|11480|1|578| +2450822|11482|1|| +2450822|11485|1|905| +2450822|11486|1|474| +2450822|11488|1|954| +2450822|11491|1|| +2450822|11492|1|418| +2450822|11494|1|580| +2450822|11497|1|892| +2450822|11498|1|285| +2450822|11500|1|385| +2450822|11503|1|664| +2450822|11504|1|598| +2450822|11506|1|50| +2450822|11509|1|132| +2450822|11510|1|671| +2450822|11512|1|679| +2450822|11515|1|717| +2450822|11516|1|984| +2450822|11518|1|168| +2450822|11521|1|73| +2450822|11522|1|367| +2450822|11524|1|58| +2450822|11527|1|21| +2450822|11528|1|251| +2450822|11530|1|422| +2450822|11533|1|246| +2450822|11534|1|20| +2450822|11536|1|614| +2450822|11539|1|| +2450822|11540|1|152| +2450822|11542|1|30| +2450822|11545|1|150| +2450822|11546|1|411| +2450822|11548|1|860| +2450822|11551|1|87| +2450822|11552|1|782| +2450822|11554|1|| +2450822|11557|1|692| +2450822|11558|1|932| +2450822|11560|1|592| +2450822|11563|1|694| +2450822|11564|1|758| +2450822|11566|1|94| +2450822|11569|1|931| +2450822|11570|1|604| +2450822|11572|1|966| +2450822|11575|1|519| +2450822|11576|1|852| +2450822|11578|1|89| +2450822|11581|1|30| +2450822|11582|1|165| +2450822|11584|1|118| +2450822|11587|1|880| +2450822|11588|1|975| +2450822|11590|1|474| +2450822|11593|1|189| +2450822|11594|1|894| +2450822|11596|1|848| +2450822|11599|1|106| +2450822|11600|1|644| +2450822|11602|1|53| +2450822|11605|1|61| +2450822|11606|1|525| +2450822|11608|1|346| +2450822|11611|1|468| +2450822|11612|1|681| +2450822|11614|1|88| +2450822|11617|1|669| +2450822|11618|1|26| +2450822|11620|1|927| +2450822|11623|1|258| +2450822|11624|1|327| +2450822|11626|1|995| +2450822|11629|1|529| +2450822|11630|1|59| +2450822|11632|1|272| +2450822|11635|1|546| +2450822|11636|1|760| +2450822|11638|1|998| +2450822|11641|1|417| +2450822|11642|1|552| +2450822|11644|1|820| +2450822|11647|1|824| +2450822|11648|1|326| +2450822|11650|1|| +2450822|11653|1|947| +2450822|11654|1|64| +2450822|11656|1|642| +2450822|11659|1|342| +2450822|11660|1|130| +2450822|11662|1|41| +2450822|11665|1|461| +2450822|11666|1|| +2450822|11668|1|900| +2450822|11671|1|356| +2450822|11672|1|248| +2450822|11674|1|589| +2450822|11677|1|416| +2450822|11678|1|313| +2450822|11680|1|635| +2450822|11683|1|476| +2450822|11684|1|510| +2450822|11686|1|294| +2450822|11689|1|37| +2450822|11690|1|81| +2450822|11692|1|382| +2450822|11695|1|782| +2450822|11696|1|408| +2450822|11698|1|114| +2450822|11701|1|950| +2450822|11702|1|898| +2450822|11704|1|69| +2450822|11707|1|918| +2450822|11708|1|710| +2450822|11710|1|308| +2450822|11713|1|401| +2450822|11714|1|714| +2450822|11716|1|619| +2450822|11719|1|567| +2450822|11720|1|973| +2450822|11722|1|431| +2450822|11725|1|144| +2450822|11726|1|627| +2450822|11728|1|744| +2450822|11731|1|312| +2450822|11732|1|115| +2450822|11734|1|766| +2450822|11737|1|888| +2450822|11738|1|492| +2450822|11740|1|718| +2450822|11743|1|448| +2450822|11744|1|| +2450822|11746|1|590| +2450822|11749|1|669| +2450822|11750|1|967| +2450822|11752|1|28| +2450822|11755|1|65| +2450822|11756|1|679| +2450822|11758|1|599| +2450822|11761|1|304| +2450822|11762|1|41| +2450822|11764|1|366| +2450822|11767|1|1| +2450822|11768|1|786| +2450822|11770|1|| +2450822|11773|1|751| +2450822|11774|1|839| +2450822|11776|1|977| +2450822|11779|1|50| +2450822|11780|1|308| +2450822|11782|1|111| +2450822|11785|1|141| +2450822|11786|1|428| +2450822|11788|1|| +2450822|11791|1|613| +2450822|11792|1|871| +2450822|11794|1|144| +2450822|11797|1|595| +2450822|11798|1|601| +2450822|11800|1|114| +2450822|11803|1|757| +2450822|11804|1|59| +2450822|11806|1|138| +2450822|11809|1|750| +2450822|11810|1|379| +2450822|11812|1|833| +2450822|11815|1|866| +2450822|11816|1|467| +2450822|11818|1|935| +2450822|11821|1|108| +2450822|11822|1|| +2450822|11824|1|538| +2450822|11827|1|326| +2450822|11828|1|317| +2450822|11830|1|51| +2450822|11833|1|205| +2450822|11834|1|83| +2450822|11836|1|512| +2450822|11839|1|628| +2450822|11840|1|479| +2450822|11842|1|| +2450822|11845|1|787| +2450822|11846|1|712| +2450822|11848|1|698| +2450822|11851|1|167| +2450822|11852|1|409| +2450822|11854|1|975| +2450822|11857|1|323| +2450822|11858|1|345| +2450822|11860|1|134| +2450822|11863|1|813| +2450822|11864|1|50| +2450822|11866|1|601| +2450822|11869|1|67| +2450822|11870|1|63| +2450822|11872|1|447| +2450822|11875|1|| +2450822|11876|1|483| +2450822|11878|1|895| +2450822|11881|1|741| +2450822|11882|1|587| +2450822|11884|1|277| +2450822|11887|1|639| +2450822|11888|1|360| +2450822|11890|1|473| +2450822|11893|1|247| +2450822|11894|1|925| +2450822|11896|1|694| +2450822|11899|1|613| +2450822|11900|1|818| +2450822|11902|1|799| +2450822|11905|1|926| +2450822|11906|1|566| +2450822|11908|1|858| +2450822|11911|1|639| +2450822|11912|1|511| +2450822|11914|1|267| +2450822|11917|1|494| +2450822|11918|1|813| +2450822|11920|1|99| +2450822|11923|1|67| +2450822|11924|1|195| +2450822|11926|1|841| +2450822|11929|1|30| +2450822|11930|1|212| +2450822|11932|1|314| +2450822|11935|1|449| +2450822|11936|1|676| +2450822|11938|1|546| +2450822|11941|1|587| +2450822|11942|1|113| +2450822|11944|1|173| +2450822|11947|1|132| +2450822|11948|1|366| +2450822|11950|1|705| +2450822|11953|1|364| +2450822|11954|1|273| +2450822|11956|1|591| +2450822|11959|1|375| +2450822|11960|1|313| +2450822|11962|1|111| +2450822|11965|1|107| +2450822|11966|1|348| +2450822|11968|1|799| +2450822|11971|1|280| +2450822|11972|1|750| +2450822|11974|1|467| +2450822|11977|1|227| +2450822|11978|1|729| +2450822|11980|1|209| +2450822|11983|1|| +2450822|11984|1|| +2450822|11986|1|934| +2450822|11989|1|452| +2450822|11990|1|65| +2450822|11992|1|833| +2450822|11995|1|848| +2450822|11996|1|219| +2450822|11998|1|39| +2450822|12001|1|463| +2450822|12002|1|464| +2450822|12004|1|708| +2450822|12007|1|599| +2450822|12008|1|224| +2450822|12010|1|846| +2450822|12013|1|46| +2450822|12014|1|479| +2450822|12016|1|723| +2450822|12019|1|775| +2450822|12020|1|340| +2450822|12022|1|852| +2450822|12025|1|165| +2450822|12026|1|716| +2450822|12028|1|731| +2450822|12031|1|87| +2450822|12032|1|955| +2450822|12034|1|630| +2450822|12037|1|711| +2450822|12038|1|362| +2450822|12040|1|956| +2450822|12043|1|269| +2450822|12044|1|396| +2450822|12046|1|991| +2450822|12049|1|779| +2450822|12050|1|848| +2450822|12052|1|157| +2450822|12055|1|329| +2450822|12056|1|847| +2450822|12058|1|179| +2450822|12061|1|791| +2450822|12062|1|659| +2450822|12064|1|151| +2450822|12067|1|900| +2450822|12068|1|606| +2450822|12070|1|286| +2450822|12073|1|223| +2450822|12074|1|731| +2450822|12076|1|383| +2450822|12079|1|126| +2450822|12080|1|| +2450822|12082|1|905| +2450822|12085|1|526| +2450822|12086|1|693| +2450822|12088|1|232| +2450822|12091|1|103| +2450822|12092|1|58| +2450822|12094|1|659| +2450822|12097|1|16| +2450822|12098|1|497| +2450822|12100|1|859| +2450822|12103|1|894| +2450822|12104|1|534| +2450822|12106|1|480| +2450822|12109|1|25| +2450822|12110|1|266| +2450822|12112|1|348| +2450822|12115|1|995| +2450822|12116|1|240| +2450822|12118|1|60| +2450822|12121|1|107| +2450822|12122|1|| +2450822|12124|1|131| +2450822|12127|1|643| +2450822|12128|1|441| +2450822|12130|1|127| +2450822|12133|1|643| +2450822|12134|1|276| +2450822|12136|1|441| +2450822|12139|1|32| +2450822|12140|1|951| +2450822|12142|1|325| +2450822|12145|1|457| +2450822|12146|1|981| +2450822|12148|1|| +2450822|12151|1|522| +2450822|12152|1|296| +2450822|12154|1|215| +2450822|12157|1|506| +2450822|12158|1|639| +2450822|12160|1|| +2450822|12163|1|| +2450822|12164|1|344| +2450822|12166|1|383| +2450822|12169|1|211| +2450822|12170|1|240| +2450822|12172|1|361| +2450822|12175|1|266| +2450822|12176|1|805| +2450822|12178|1|463| +2450822|12181|1|888| +2450822|12182|1|145| +2450822|12184|1|448| +2450822|12187|1|22| +2450822|12188|1|70| +2450822|12190|1|120| +2450822|12193|1|915| +2450822|12194|1|603| +2450822|12196|1|316| +2450822|12199|1|889| +2450822|12200|1|622| +2450822|12202|1|527| +2450822|12205|1|340| +2450822|12206|1|14| +2450822|12208|1|389| +2450822|12211|1|344| +2450822|12212|1|141| +2450822|12214|1|654| +2450822|12217|1|15| +2450822|12218|1|671| +2450822|12220|1|499| +2450822|12223|1|77| +2450822|12224|1|647| +2450822|12226|1|570| +2450822|12229|1|271| +2450822|12230|1|646| +2450822|12232|1|418| +2450822|12235|1|399| +2450822|12236|1|300| +2450822|12238|1|| +2450822|12241|1|460| +2450822|12242|1|744| +2450822|12244|1|225| +2450822|12247|1|180| +2450822|12248|1|573| +2450822|12250|1|642| +2450822|12253|1|924| +2450822|12254|1|275| +2450822|12256|1|509| +2450822|12259|1|420| +2450822|12260|1|865| +2450822|12262|1|972| +2450822|12265|1|| +2450822|12266|1|779| +2450822|12268|1|986| +2450822|12271|1|147| +2450822|12272|1|397| +2450822|12274|1|295| +2450822|12277|1|| +2450822|12278|1|458| +2450822|12280|1|| +2450822|12283|1|11| +2450822|12284|1|113| +2450822|12286|1|277| +2450822|12289|1|696| +2450822|12290|1|18| +2450822|12292|1|726| +2450822|12295|1|794| +2450822|12296|1|231| +2450822|12298|1|802| +2450822|12301|1|| +2450822|12302|1|548| +2450822|12304|1|762| +2450822|12307|1|458| +2450822|12308|1|591| +2450822|12310|1|| +2450822|12313|1|910| +2450822|12314|1|153| +2450822|12316|1|650| +2450822|12319|1|377| +2450822|12320|1|625| +2450822|12322|1|660| +2450822|12325|1|97| +2450822|12326|1|616| +2450822|12328|1|284| +2450822|12331|1|232| +2450822|12332|1|830| +2450822|12334|1|724| +2450822|12337|1|776| +2450822|12338|1|47| +2450822|12340|1|680| +2450822|12343|1|818| +2450822|12344|1|772| +2450822|12346|1|679| +2450822|12349|1|211| +2450822|12350|1|38| +2450822|12352|1|574| +2450822|12355|1|483| +2450822|12356|1|86| +2450822|12358|1|288| +2450822|12361|1|622| +2450822|12362|1|| +2450822|12364|1|665| +2450822|12367|1|784| +2450822|12368|1|523| +2450822|12370|1|210| +2450822|12373|1|519| +2450822|12374|1|800| +2450822|12376|1|876| +2450822|12379|1|447| +2450822|12380|1|423| +2450822|12382|1|382| +2450822|12385|1|650| +2450822|12386|1|508| +2450822|12388|1|384| +2450822|12391|1|868| +2450822|12392|1|455| +2450822|12394|1|412| +2450822|12397|1|595| +2450822|12398|1|207| +2450822|12400|1|240| +2450822|12403|1|362| +2450822|12404|1|622| +2450822|12406|1|251| +2450822|12409|1|704| +2450822|12410|1|234| +2450822|12412|1|614| +2450822|12415|1|456| +2450822|12416|1|749| +2450822|12418|1|587| +2450822|12421|1|388| +2450822|12422|1|| +2450822|12424|1|978| +2450822|12427|1|885| +2450822|12428|1|420| +2450822|12430|1|621| +2450822|12433|1|947| +2450822|12434|1|822| +2450822|12436|1|702| +2450822|12439|1|783| +2450822|12440|1|425| +2450822|12442|1|774| +2450822|12445|1|539| +2450822|12446|1|415| +2450822|12448|1|319| +2450822|12451|1|503| +2450822|12452|1|267| +2450822|12454|1|850| +2450822|12457|1|226| +2450822|12458|1|677| +2450822|12460|1|430| +2450822|12463|1|125| +2450822|12464|1|392| +2450822|12466|1|661| +2450822|12469|1|934| +2450822|12470|1|123| +2450822|12472|1|841| +2450822|12475|1|979| +2450822|12476|1|| +2450822|12478|1|210| +2450822|12481|1|311| +2450822|12482|1|852| +2450822|12484|1|793| +2450822|12487|1|684| +2450822|12488|1|67| +2450822|12490|1|| +2450822|12493|1|162| +2450822|12494|1|260| +2450822|12496|1|375| +2450822|12499|1|706| +2450822|12500|1|173| +2450822|12502|1|598| +2450822|12505|1|469| +2450822|12506|1|522| +2450822|12508|1|848| +2450822|12511|1|176| +2450822|12512|1|357| +2450822|12514|1|912| +2450822|12517|1|764| +2450822|12518|1|266| +2450822|12520|1|786| +2450822|12523|1|499| +2450822|12524|1|675| +2450822|12526|1|480| +2450822|12529|1|427| +2450822|12530|1|| +2450822|12532|1|526| +2450822|12535|1|701| +2450822|12536|1|456| +2450822|12538|1|837| +2450822|12541|1|893| +2450822|12542|1|982| +2450822|12544|1|492| +2450822|12547|1|674| +2450822|12548|1|842| +2450822|12550|1|649| +2450822|12553|1|276| +2450822|12554|1|667| +2450822|12556|1|310| +2450822|12559|1|505| +2450822|12560|1|775| +2450822|12562|1|801| +2450822|12565|1|972| +2450822|12566|1|712| +2450822|12568|1|282| +2450822|12571|1|78| +2450822|12572|1|| +2450822|12574|1|794| +2450822|12577|1|970| +2450822|12578|1|565| +2450822|12580|1|722| +2450822|12583|1|546| +2450822|12584|1|775| +2450822|12586|1|230| +2450822|12589|1|517| +2450822|12590|1|| +2450822|12592|1|445| +2450822|12595|1|825| +2450822|12596|1|879| +2450822|12598|1|6| +2450822|12601|1|405| +2450822|12602|1|182| +2450822|12604|1|163| +2450822|12607|1|538| +2450822|12608|1|54| +2450822|12610|1|733| +2450822|12613|1|476| +2450822|12614|1|584| +2450822|12616|1|616| +2450822|12619|1|872| +2450822|12620|1|998| +2450822|12622|1|392| +2450822|12625|1|285| +2450822|12626|1|534| +2450822|12628|1|410| +2450822|12631|1|924| +2450822|12632|1|868| +2450822|12634|1|704| +2450822|12637|1|660| +2450822|12638|1|123| +2450822|12640|1|73| +2450822|12643|1|| +2450822|12644|1|77| +2450822|12646|1|704| +2450822|12649|1|292| +2450822|12650|1|99| +2450822|12652|1|71| +2450822|12655|1|65| +2450822|12656|1|245| +2450822|12658|1|634| +2450822|12661|1|| +2450822|12662|1|933| +2450822|12664|1|351| +2450822|12667|1|624| +2450822|12668|1|861| +2450822|12670|1|22| +2450822|12673|1|3| +2450822|12674|1|441| +2450822|12676|1|486| +2450822|12679|1|180| +2450822|12680|1|460| +2450822|12682|1|121| +2450822|12685|1|925| +2450822|12686|1|49| +2450822|12688|1|355| +2450822|12691|1|168| +2450822|12692|1|134| +2450822|12694|1|261| +2450822|12697|1|825| +2450822|12698|1|468| +2450822|12700|1|697| +2450822|12703|1|860| +2450822|12704|1|540| +2450822|12706|1|158| +2450822|12709|1|659| +2450822|12710|1|898| +2450822|12712|1|616| +2450822|12715|1|499| +2450822|12716|1|516| +2450822|12718|1|902| +2450822|12721|1|649| +2450822|12722|1|980| +2450822|12724|1|317| +2450822|12727|1|485| +2450822|12728|1|221| +2450822|12730|1|109| +2450822|12733|1|507| +2450822|12734|1|510| +2450822|12736|1|| +2450822|12739|1|399| +2450822|12740|1|391| +2450822|12742|1|823| +2450822|12745|1|118| +2450822|12746|1|445| +2450822|12748|1|260| +2450822|12751|1|659| +2450822|12752|1|553| +2450822|12754|1|735| +2450822|12757|1|729| +2450822|12758|1|437| +2450822|12760|1|632| +2450822|12763|1|233| +2450822|12764|1|956| +2450822|12766|1|628| +2450822|12769|1|116| +2450822|12770|1|603| +2450822|12772|1|785| +2450822|12775|1|51| +2450822|12776|1|11| +2450822|12778|1|194| +2450822|12781|1|240| +2450822|12782|1|393| +2450822|12784|1|260| +2450822|12787|1|331| +2450822|12788|1|239| +2450822|12790|1|407| +2450822|12793|1|82| +2450822|12794|1|51| +2450822|12796|1|754| +2450822|12799|1|221| +2450822|12800|1|146| +2450822|12802|1|965| +2450822|12805|1|164| +2450822|12806|1|443| +2450822|12808|1|413| +2450822|12811|1|663| +2450822|12812|1|251| +2450822|12814|1|803| +2450822|12817|1|833| +2450822|12818|1|75| +2450822|12820|1|724| +2450822|12823|1|583| +2450822|12824|1|519| +2450822|12826|1|760| +2450822|12829|1|76| +2450822|12830|1|604| +2450822|12832|1|788| +2450822|12835|1|675| +2450822|12836|1|376| +2450822|12838|1|777| +2450822|12841|1|95| +2450822|12842|1|171| +2450822|12844|1|142| +2450822|12847|1|511| +2450822|12848|1|923| +2450822|12850|1|746| +2450822|12853|1|972| +2450822|12854|1|938| +2450822|12856|1|385| +2450822|12859|1|143| +2450822|12860|1|432| +2450822|12862|1|854| +2450822|12865|1|459| +2450822|12866|1|419| +2450822|12868|1|926| +2450822|12871|1|149| +2450822|12872|1|518| +2450822|12874|1|252| +2450822|12877|1|633| +2450822|12878|1|526| +2450822|12880|1|323| +2450822|12883|1|482| +2450822|12884|1|594| +2450822|12886|1|848| +2450822|12889|1|515| +2450822|12890|1|845| +2450822|12892|1|675| +2450822|12895|1|201| +2450822|12896|1|998| +2450822|12898|1|70| +2450822|12901|1|646| +2450822|12902|1|12| +2450822|12904|1|132| +2450822|12907|1|682| +2450822|12908|1|997| +2450822|12910|1|322| +2450822|12913|1|| +2450822|12914|1|600| +2450822|12916|1|277| +2450822|12919|1|887| +2450822|12920|1|| +2450822|12922|1|93| +2450822|12925|1|678| +2450822|12926|1|257| +2450822|12928|1|121| +2450822|12931|1|562| +2450822|12932|1|771| +2450822|12934|1|708| +2450822|12937|1|| +2450822|12938|1|477| +2450822|12940|1|39| +2450822|12943|1|868| +2450822|12944|1|552| +2450822|12946|1|281| +2450822|12949|1|412| +2450822|12950|1|874| +2450822|12952|1|610| +2450822|12955|1|745| +2450822|12956|1|996| +2450822|12958|1|| +2450822|12961|1|841| +2450822|12962|1|245| +2450822|12964|1|504| +2450822|12967|1|683| +2450822|12968|1|991| +2450822|12970|1|| +2450822|12973|1|781| +2450822|12974|1|847| +2450822|12976|1|131| +2450822|12979|1|442| +2450822|12980|1|735| +2450822|12982|1|327| +2450822|12985|1|737| +2450822|12986|1|296| +2450822|12988|1|722| +2450822|12991|1|608| +2450822|12992|1|818| +2450822|12994|1|485| +2450822|12997|1|39| +2450822|12998|1|490| +2450822|13000|1|660| +2450822|13003|1|588| +2450822|13004|1|450| +2450822|13006|1|204| +2450822|13009|1|721| +2450822|13010|1|98| +2450822|13012|1|942| +2450822|13015|1|103| +2450822|13016|1|272| +2450822|13018|1|548| +2450822|13021|1|978| +2450822|13022|1|273| +2450822|13024|1|861| +2450822|13027|1|956| +2450822|13028|1|941| +2450822|13030|1|339| +2450822|13033|1|904| +2450822|13034|1|298| +2450822|13036|1|447| +2450822|13039|1|925| +2450822|13040|1|366| +2450822|13042|1|850| +2450822|13045|1|760| +2450822|13046|1|876| +2450822|13048|1|407| +2450822|13051|1|54| +2450822|13052|1|992| +2450822|13054|1|| +2450822|13057|1|310| +2450822|13058|1|353| +2450822|13060|1|528| +2450822|13063|1|656| +2450822|13064|1|127| +2450822|13066|1|530| +2450822|13069|1|910| +2450822|13070|1|498| +2450822|13072|1|722| +2450822|13075|1|224| +2450822|13076|1|427| +2450822|13078|1|507| +2450822|13081|1|933| +2450822|13082|1|746| +2450822|13084|1|743| +2450822|13087|1|102| +2450822|13088|1|781| +2450822|13090|1|587| +2450822|13093|1|204| +2450822|13094|1|735| +2450822|13096|1|25| +2450822|13099|1|948| +2450822|13100|1|398| +2450822|13102|1|195| +2450822|13105|1|| +2450822|13106|1|| +2450822|13108|1|143| +2450822|13111|1|918| +2450822|13112|1|576| +2450822|13114|1|648| +2450822|13117|1|810| +2450822|13118|1|775| +2450822|13120|1|343| +2450822|13123|1|29| +2450822|13124|1|404| +2450822|13126|1|798| +2450822|13129|1|225| +2450822|13130|1|709| +2450822|13132|1|| +2450822|13135|1|619| +2450822|13136|1|99| +2450822|13138|1|328| +2450822|13141|1|783| +2450822|13142|1|26| +2450822|13144|1|503| +2450822|13147|1|107| +2450822|13148|1|946| +2450822|13150|1|856| +2450822|13153|1|604| +2450822|13154|1|893| +2450822|13156|1|368| +2450822|13159|1|910| +2450822|13160|1|893| +2450822|13162|1|776| +2450822|13165|1|932| +2450822|13166|1|884| +2450822|13168|1|971| +2450822|13171|1|802| +2450822|13172|1|68| +2450822|13174|1|93| +2450822|13177|1|832| +2450822|13178|1|135| +2450822|13180|1|326| +2450822|13183|1|700| +2450822|13184|1|325| +2450822|13186|1|946| +2450822|13189|1|883| +2450822|13190|1|248| +2450822|13192|1|192| +2450822|13195|1|292| +2450822|13196|1|116| +2450822|13198|1|546| +2450822|13201|1|615| +2450822|13202|1|831| +2450822|13204|1|734| +2450822|13207|1|466| +2450822|13208|1|843| +2450822|13210|1|540| +2450822|13213|1|262| +2450822|13214|1|164| +2450822|13216|1|950| +2450822|13219|1|704| +2450822|13220|1|565| +2450822|13222|1|842| +2450822|13225|1|833| +2450822|13226|1|102| +2450822|13228|1|218| +2450822|13231|1|786| +2450822|13232|1|91| +2450822|13234|1|422| +2450822|13237|1|205| +2450822|13238|1|50| +2450822|13240|1|27| +2450822|13243|1|771| +2450822|13244|1|394| +2450822|13246|1|5| +2450822|13249|1|216| +2450822|13250|1|602| +2450822|13252|1|275| +2450822|13255|1|336| +2450822|13256|1|289| +2450822|13258|1|446| +2450822|13261|1|959| +2450822|13262|1|483| +2450822|13264|1|731| +2450822|13267|1|875| +2450822|13268|1|741| +2450822|13270|1|969| +2450822|13273|1|416| +2450822|13274|1|875| +2450822|13276|1|854| +2450822|13279|1|49| +2450822|13280|1|962| +2450822|13282|1|602| +2450822|13285|1|244| +2450822|13286|1|967| +2450822|13288|1|464| +2450822|13291|1|797| +2450822|13292|1|343| +2450822|13294|1|469| +2450822|13297|1|717| +2450822|13298|1|273| +2450822|13300|1|844| +2450822|13303|1|524| +2450822|13304|1|740| +2450822|13306|1|185| +2450822|13309|1|669| +2450822|13310|1|576| +2450822|13312|1|850| +2450822|13315|1|695| +2450822|13316|1|36| +2450822|13318|1|26| +2450822|13321|1|410| +2450822|13322|1|844| +2450822|13324|1|404| +2450822|13327|1|348| +2450822|13328|1|760| +2450822|13330|1|480| +2450822|13333|1|638| +2450822|13334|1|426| +2450822|13336|1|326| +2450822|13339|1|280| +2450822|13340|1|820| +2450822|13342|1|894| +2450822|13345|1|16| +2450822|13346|1|600| +2450822|13348|1|232| +2450822|13351|1|450| +2450822|13352|1|371| +2450822|13354|1|821| +2450822|13357|1|286| +2450822|13358|1|513| +2450822|13360|1|496| +2450822|13363|1|840| +2450822|13364|1|| +2450822|13366|1|909| +2450822|13369|1|448| +2450822|13370|1|| +2450822|13372|1|455| +2450822|13375|1|807| +2450822|13376|1|788| +2450822|13378|1|510| +2450822|13381|1|| +2450822|13382|1|| +2450822|13384|1|260| +2450822|13387|1|214| +2450822|13388|1|302| +2450822|13390|1|| +2450822|13393|1|562| +2450822|13394|1|203| +2450822|13396|1|326| +2450822|13399|1|352| +2450822|13400|1|348| +2450822|13402|1|| +2450822|13405|1|763| +2450822|13406|1|795| +2450822|13408|1|90| +2450822|13411|1|555| +2450822|13412|1|549| +2450822|13414|1|356| +2450822|13417|1|483| +2450822|13418|1|80| +2450822|13420|1|866| +2450822|13423|1|150| +2450822|13424|1|891| +2450822|13426|1|696| +2450822|13429|1|532| +2450822|13430|1|914| +2450822|13432|1|569| +2450822|13435|1|378| +2450822|13436|1|738| +2450822|13438|1|10| +2450822|13441|1|369| +2450822|13442|1|712| +2450822|13444|1|392| +2450822|13447|1|669| +2450822|13448|1|223| +2450822|13450|1|605| +2450822|13453|1|483| +2450822|13454|1|408| +2450822|13456|1|| +2450822|13459|1|742| +2450822|13460|1|485| +2450822|13462|1|621| +2450822|13465|1|56| +2450822|13466|1|330| +2450822|13468|1|493| +2450822|13471|1|471| +2450822|13472|1|719| +2450822|13474|1|295| +2450822|13477|1|83| +2450822|13478|1|109| +2450822|13480|1|707| +2450822|13483|1|574| +2450822|13484|1|306| +2450822|13486|1|365| +2450822|13489|1|| +2450822|13490|1|603| +2450822|13492|1|876| +2450822|13495|1|693| +2450822|13496|1|695| +2450822|13498|1|800| +2450822|13501|1|436| +2450822|13502|1|980| +2450822|13504|1|| +2450822|13507|1|| +2450822|13508|1|329| +2450822|13510|1|548| +2450822|13513|1|826| +2450822|13514|1|639| +2450822|13516|1|| +2450822|13519|1|81| +2450822|13520|1|311| +2450822|13522|1|825| +2450822|13525|1|944| +2450822|13526|1|556| +2450822|13528|1|86| +2450822|13531|1|720| +2450822|13532|1|966| +2450822|13534|1|209| +2450822|13537|1|647| +2450822|13538|1|23| +2450822|13540|1|696| +2450822|13543|1|373| +2450822|13544|1|161| +2450822|13546|1|729| +2450822|13549|1|527| +2450822|13550|1|629| +2450822|13552|1|239| +2450822|13555|1|764| +2450822|13556|1|307| +2450822|13558|1|163| +2450822|13561|1|271| +2450822|13562|1|299| +2450822|13564|1|288| +2450822|13567|1|853| +2450822|13568|1|882| +2450822|13570|1|511| +2450822|13573|1|236| +2450822|13574|1|547| +2450822|13576|1|25| +2450822|13579|1|741| +2450822|13580|1|973| +2450822|13582|1|513| +2450822|13585|1|340| +2450822|13586|1|769| +2450822|13588|1|910| +2450822|13591|1|254| +2450822|13592|1|499| +2450822|13594|1|527| +2450822|13597|1|567| +2450822|13598|1|662| +2450822|13600|1|529| +2450822|13603|1|679| +2450822|13604|1|857| +2450822|13606|1|571| +2450822|13609|1|569| +2450822|13610|1|469| +2450822|13612|1|689| +2450822|13615|1|962| +2450822|13616|1|397| +2450822|13618|1|996| +2450822|13621|1|443| +2450822|13622|1|278| +2450822|13624|1|61| +2450822|13627|1|252| +2450822|13628|1|471| +2450822|13630|1|309| +2450822|13633|1|164| +2450822|13634|1|892| +2450822|13636|1|372| +2450822|13639|1|| +2450822|13640|1|362| +2450822|13642|1|768| +2450822|13645|1|944| +2450822|13646|1|705| +2450822|13648|1|324| +2450822|13651|1|395| +2450822|13652|1|415| +2450822|13654|1|411| +2450822|13657|1|749| +2450822|13658|1|33| +2450822|13660|1|174| +2450822|13663|1|226| +2450822|13664|1|318| +2450822|13666|1|57| +2450822|13669|1|894| +2450822|13670|1|340| +2450822|13672|1|491| +2450822|13675|1|379| +2450822|13676|1|869| +2450822|13678|1|| +2450822|13681|1|504| +2450822|13682|1|351| +2450822|13684|1|890| +2450822|13687|1|81| +2450822|13688|1|80| +2450822|13690|1|458| +2450822|13693|1|706| +2450822|13694|1|873| +2450822|13696|1|345| +2450822|13699|1|115| +2450822|13700|1|669| +2450822|13702|1|939| +2450822|13705|1|| +2450822|13706|1|250| +2450822|13708|1|833| +2450822|13711|1|189| +2450822|13712|1|284| +2450822|13714|1|164| +2450822|13717|1|18| +2450822|13718|1|588| +2450822|13720|1|531| +2450822|13723|1|713| +2450822|13724|1|246| +2450822|13726|1|994| +2450822|13729|1|15| +2450822|13730|1|884| +2450822|13732|1|633| +2450822|13735|1|214| +2450822|13736|1|445| +2450822|13738|1|737| +2450822|13741|1|919| +2450822|13742|1|185| +2450822|13744|1|213| +2450822|13747|1|| +2450822|13748|1|873| +2450822|13750|1|811| +2450822|13753|1|880| +2450822|13754|1|233| +2450822|13756|1|282| +2450822|13759|1|| +2450822|13760|1|793| +2450822|13762|1|333| +2450822|13765|1|435| +2450822|13766|1|| +2450822|13768|1|279| +2450822|13771|1|25| +2450822|13772|1|663| +2450822|13774|1|223| +2450822|13777|1|555| +2450822|13778|1|851| +2450822|13780|1|464| +2450822|13783|1|892| +2450822|13784|1|644| +2450822|13786|1|754| +2450822|13789|1|540| +2450822|13790|1|862| +2450822|13792|1|708| +2450822|13795|1|795| +2450822|13796|1|| +2450822|13798|1|553| +2450822|13801|1|| +2450822|13802|1|87| +2450822|13804|1|592| +2450822|13807|1|18| +2450822|13808|1|89| +2450822|13810|1|656| +2450822|13813|1|| +2450822|13814|1|596| +2450822|13816|1|838| +2450822|13819|1|616| +2450822|13820|1|455| +2450822|13822|1|462| +2450822|13825|1|305| +2450822|13826|1|782| +2450822|13828|1|306| +2450822|13831|1|463| +2450822|13832|1|573| +2450822|13834|1|| +2450822|13837|1|| +2450822|13838|1|754| +2450822|13840|1|843| +2450822|13843|1|939| +2450822|13844|1|313| +2450822|13846|1|771| +2450822|13849|1|562| +2450822|13850|1|| +2450822|13852|1|867| +2450822|13855|1|907| +2450822|13856|1|261| +2450822|13858|1|598| +2450822|13861|1|71| +2450822|13862|1|860| +2450822|13864|1|855| +2450822|13867|1|523| +2450822|13868|1|519| +2450822|13870|1|960| +2450822|13873|1|630| +2450822|13874|1|155| +2450822|13876|1|531| +2450822|13879|1|141| +2450822|13880|1|311| +2450822|13882|1|729| +2450822|13885|1|23| +2450822|13886|1|218| +2450822|13888|1|393| +2450822|13891|1|6| +2450822|13892|1|| +2450822|13894|1|792| +2450822|13897|1|58| +2450822|13898|1|651| +2450822|13900|1|480| +2450822|13903|1|532| +2450822|13904|1|314| +2450822|13906|1|893| +2450822|13909|1|433| +2450822|13910|1|491| +2450822|13912|1|899| +2450822|13915|1|424| +2450822|13916|1|895| +2450822|13918|1|251| +2450822|13921|1|441| +2450822|13922|1|214| +2450822|13924|1|503| +2450822|13927|1|151| +2450822|13928|1|838| +2450822|13930|1|660| +2450822|13933|1|934| +2450822|13934|1|935| +2450822|13936|1|524| +2450822|13939|1|878| +2450822|13940|1|903| +2450822|13942|1|647| +2450822|13945|1|962| +2450822|13946|1|364| +2450822|13948|1|36| +2450822|13951|1|864| +2450822|13952|1|649| +2450822|13954|1|871| +2450822|13957|1|482| +2450822|13958|1|261| +2450822|13960|1|605| +2450822|13963|1|932| +2450822|13964|1|| +2450822|13966|1|575| +2450822|13969|1|841| +2450822|13970|1|557| +2450822|13972|1|149| +2450822|13975|1|417| +2450822|13976|1|763| +2450822|13978|1|313| +2450822|13981|1|554| +2450822|13982|1|491| +2450822|13984|1|298| +2450822|13987|1|138| +2450822|13988|1|989| +2450822|13990|1|682| +2450822|13993|1|| +2450822|13994|1|520| +2450822|13996|1|634| +2450822|13999|1|252| +2450822|14000|1|292| +2450822|14002|1|242| +2450822|14005|1|613| +2450822|14006|1|551| +2450822|14008|1|418| +2450822|14011|1|696| +2450822|14012|1|715| +2450822|14014|1|223| +2450822|14017|1|169| +2450822|14018|1|542| +2450822|14020|1|623| +2450822|14023|1|874| +2450822|14024|1|22| +2450822|14026|1|196| +2450822|14029|1|968| +2450822|14030|1|228| +2450822|14032|1|373| +2450822|14035|1|563| +2450822|14036|1|721| +2450822|14038|1|821| +2450822|14041|1|710| +2450822|14042|1|396| +2450822|14044|1|533| +2450822|14047|1|601| +2450822|14048|1|110| +2450822|14050|1|757| +2450822|14053|1|852| +2450822|14054|1|459| +2450822|14056|1|| +2450822|14059|1|891| +2450822|14060|1|645| +2450822|14062|1|456| +2450822|14065|1|817| +2450822|14066|1|229| +2450822|14068|1|340| +2450822|14071|1|584| +2450822|14072|1|220| +2450822|14074|1|457| +2450822|14077|1|966| +2450822|14078|1|91| +2450822|14080|1|976| +2450822|14083|1|811| +2450822|14084|1|| +2450822|14086|1|455| +2450822|14089|1|162| +2450822|14090|1|53| +2450822|14092|1|641| +2450822|14095|1|977| +2450822|14096|1|435| +2450822|14098|1|765| +2450822|14101|1|449| +2450822|14102|1|380| +2450822|14104|1|492| +2450822|14107|1|978| +2450822|14108|1|467| +2450822|14110|1|712| +2450822|14113|1|890| +2450822|14114|1|957| +2450822|14116|1|592| +2450822|14119|1|275| +2450822|14120|1|408| +2450822|14122|1|102| +2450822|14125|1|553| +2450822|14126|1|| +2450822|14128|1|345| +2450822|14131|1|434| +2450822|14132|1|346| +2450822|14134|1|824| +2450822|14137|1|76| +2450822|14138|1|714| +2450822|14140|1|784| +2450822|14143|1|406| +2450822|14144|1|301| +2450822|14146|1|470| +2450822|14149|1|867| +2450822|14150|1|487| +2450822|14152|1|906| +2450822|14155|1|510| +2450822|14156|1|256| +2450822|14158|1|665| +2450822|14161|1|423| +2450822|14162|1|982| +2450822|14164|1|132| +2450822|14167|1|52| +2450822|14168|1|337| +2450822|14170|1|57| +2450822|14173|1|257| +2450822|14174|1|318| +2450822|14176|1|252| +2450822|14179|1|1000| +2450822|14180|1|357| +2450822|14182|1|114| +2450822|14185|1|555| +2450822|14186|1|708| +2450822|14188|1|| +2450822|14191|1|708| +2450822|14192|1|789| +2450822|14194|1|591| +2450822|14197|1|887| +2450822|14198|1|371| +2450822|14200|1|752| +2450822|14203|1|58| +2450822|14204|1|240| +2450822|14206|1|108| +2450822|14209|1|418| +2450822|14210|1|261| +2450822|14212|1|384| +2450822|14215|1|363| +2450822|14216|1|43| +2450822|14218|1|204| +2450822|14221|1|882| +2450822|14222|1|793| +2450822|14224|1|567| +2450822|14227|1|57| +2450822|14228|1|487| +2450822|14230|1|909| +2450822|14233|1|832| +2450822|14234|1|570| +2450822|14236|1|459| +2450822|14239|1|400| +2450822|14240|1|544| +2450822|14242|1|937| +2450822|14245|1|733| +2450822|14246|1|889| +2450822|14248|1|961| +2450822|14251|1|563| +2450822|14252|1|89| +2450822|14254|1|284| +2450822|14257|1|36| +2450822|14258|1|576| +2450822|14260|1|613| +2450822|14263|1|633| +2450822|14264|1|280| +2450822|14266|1|623| +2450822|14269|1|220| +2450822|14270|1|670| +2450822|14272|1|814| +2450822|14275|1|671| +2450822|14276|1|623| +2450822|14278|1|37| +2450822|14281|1|172| +2450822|14282|1|| +2450822|14284|1|971| +2450822|14287|1|826| +2450822|14288|1|62| +2450822|14290|1|770| +2450822|14293|1|245| +2450822|14294|1|574| +2450822|14296|1|240| +2450822|14299|1|| +2450822|14300|1|398| +2450822|14302|1|849| +2450822|14305|1|966| +2450822|14306|1|3| +2450822|14308|1|487| +2450822|14311|1|852| +2450822|14312|1|26| +2450822|14314|1|206| +2450822|14317|1|879| +2450822|14318|1|663| +2450822|14320|1|916| +2450822|14323|1|88| +2450822|14324|1|397| +2450822|14326|1|909| +2450822|14329|1|362| +2450822|14330|1|222| +2450822|14332|1|852| +2450822|14335|1|152| +2450822|14336|1|363| +2450822|14338|1|864| +2450822|14341|1|681| +2450822|14342|1|545| +2450822|14344|1|933| +2450822|14347|1|21| +2450822|14348|1|654| +2450822|14350|1|587| +2450822|14353|1|466| +2450822|14354|1|939| +2450822|14356|1|568| +2450822|14359|1|456| +2450822|14360|1|942| +2450822|14362|1|443| +2450822|14365|1|841| +2450822|14366|1|310| +2450822|14368|1|50| +2450822|14371|1|813| +2450822|14372|1|597| +2450822|14374|1|399| +2450822|14377|1|877| +2450822|14378|1|676| +2450822|14380|1|590| +2450822|14383|1|254| +2450822|14384|1|336| +2450822|14386|1|134| +2450822|14389|1|36| +2450822|14390|1|572| +2450822|14392|1|666| +2450822|14395|1|963| +2450822|14396|1|317| +2450822|14398|1|| +2450822|14401|1|937| +2450822|14402|1|756| +2450822|14404|1|6| +2450822|14407|1|674| +2450822|14408|1|97| +2450822|14410|1|838| +2450822|14413|1|533| +2450822|14414|1|130| +2450822|14416|1|935| +2450822|14419|1|210| +2450822|14420|1|371| +2450822|14422|1|437| +2450822|14425|1|689| +2450822|14426|1|502| +2450822|14428|1|134| +2450822|14431|1|905| +2450822|14432|1|911| +2450822|14434|1|277| +2450822|14437|1|732| +2450822|14438|1|613| +2450822|14440|1|427| +2450822|14443|1|41| +2450822|14444|1|474| +2450822|14446|1|224| +2450822|14449|1|39| +2450822|14450|1|822| +2450822|14452|1|573| +2450822|14455|1|766| +2450822|14456|1|99| +2450822|14458|1|67| +2450822|14461|1|821| +2450822|14462|1|322| +2450822|14464|1|260| +2450822|14467|1|826| +2450822|14468|1|12| +2450822|14470|1|744| +2450822|14473|1|430| +2450822|14474|1|| +2450822|14476|1|718| +2450822|14479|1|126| +2450822|14480|1|542| +2450822|14482|1|489| +2450822|14485|1|38| +2450822|14486|1|750| +2450822|14488|1|380| +2450822|14491|1|484| +2450822|14492|1|935| +2450822|14494|1|757| +2450822|14497|1|| +2450822|14498|1|428| +2450822|14500|1|784| +2450822|14503|1|343| +2450822|14504|1|546| +2450822|14506|1|100| +2450822|14509|1|| +2450822|14510|1|669| +2450822|14512|1|983| +2450822|14515|1|242| +2450822|14516|1|690| +2450822|14518|1|938| +2450822|14521|1|581| +2450822|14522|1|925| +2450822|14524|1|30| +2450822|14527|1|309| +2450822|14528|1|253| +2450822|14530|1|424| +2450822|14533|1|855| +2450822|14534|1|945| +2450822|14536|1|912| +2450822|14539|1|327| +2450822|14540|1|195| +2450822|14542|1|| +2450822|14545|1|884| +2450822|14546|1|490| +2450822|14548|1|546| +2450822|14551|1|602| +2450822|14552|1|150| +2450822|14554|1|334| +2450822|14557|1|551| +2450822|14558|1|528| +2450822|14560|1|114| +2450822|14563|1|624| +2450822|14564|1|340| +2450822|14566|1|448| +2450822|14569|1|125| +2450822|14570|1|517| +2450822|14572|1|93| +2450822|14575|1|401| +2450822|14576|1|242| +2450822|14578|1|617| +2450822|14581|1|644| +2450822|14582|1|794| +2450822|14584|1|350| +2450822|14587|1|936| +2450822|14588|1|233| +2450822|14590|1|587| +2450822|14593|1|348| +2450822|14594|1|200| +2450822|14596|1|909| +2450822|14599|1|42| +2450822|14600|1|20| +2450822|14602|1|868| +2450822|14605|1|783| +2450822|14606|1|455| +2450822|14608|1|373| +2450822|14611|1|947| +2450822|14612|1|494| +2450822|14614|1|486| +2450822|14617|1|905| +2450822|14618|1|874| +2450822|14620|1|54| +2450822|14623|1|424| +2450822|14624|1|282| +2450822|14626|1|115| +2450822|14629|1|146| +2450822|14630|1|551| +2450822|14632|1|484| +2450822|14635|1|835| +2450822|14636|1|818| +2450822|14638|1|329| +2450822|14641|1|72| +2450822|14642|1|8| +2450822|14644|1|297| +2450822|14647|1|832| +2450822|14648|1|645| +2450822|14650|1|232| +2450822|14653|1|662| +2450822|14654|1|605| +2450822|14656|1|119| +2450822|14659|1|398| +2450822|14660|1|846| +2450822|14662|1|933| +2450822|14665|1|693| +2450822|14666|1|197| +2450822|14668|1|350| +2450822|14671|1|992| +2450822|14672|1|126| +2450822|14674|1|241| +2450822|14677|1|283| +2450822|14678|1|416| +2450822|14680|1|74| +2450822|14683|1|53| +2450822|14684|1|511| +2450822|14686|1|332| +2450822|14689|1|857| +2450822|14690|1|| +2450822|14692|1|96| +2450822|14695|1|789| +2450822|14696|1|772| +2450822|14698|1|647| +2450822|14701|1|168| +2450822|14702|1|837| +2450822|14704|1|290| +2450822|14707|1|210| +2450822|14708|1|684| +2450822|14710|1|241| +2450822|14713|1|647| +2450822|14714|1|383| +2450822|14716|1|994| +2450822|14719|1|568| +2450822|14720|1|423| +2450822|14722|1|264| +2450822|14725|1|426| +2450822|14726|1|927| +2450822|14728|1|715| +2450822|14731|1|316| +2450822|14732|1|440| +2450822|14734|1|912| +2450822|14737|1|539| +2450822|14738|1|588| +2450822|14740|1|878| +2450822|14743|1|83| +2450822|14744|1|727| +2450822|14746|1|846| +2450822|14749|1|600| +2450822|14750|1|| +2450822|14752|1|770| +2450822|14755|1|413| +2450822|14756|1|329| +2450822|14758|1|465| +2450822|14761|1|135| +2450822|14762|1|908| +2450822|14764|1|294| +2450822|14767|1|911| +2450822|14768|1|875| +2450822|14770|1|142| +2450822|14773|1|596| +2450822|14774|1|159| +2450822|14776|1|655| +2450822|14779|1|302| +2450822|14780|1|124| +2450822|14782|1|868| +2450822|14785|1|751| +2450822|14786|1|497| +2450822|14788|1|112| +2450822|14791|1|109| +2450822|14792|1|379| +2450822|14794|1|936| +2450822|14797|1|279| +2450822|14798|1|183| +2450822|14800|1|846| +2450822|14803|1|395| +2450822|14804|1|767| +2450822|14806|1|75| +2450822|14809|1|| +2450822|14810|1|828| +2450822|14812|1|964| +2450822|14815|1|266| +2450822|14816|1|48| +2450822|14818|1|372| +2450822|14821|1|984| +2450822|14822|1|442| +2450822|14824|1|599| +2450822|14827|1|913| +2450822|14828|1|173| +2450822|14830|1|221| +2450822|14833|1|584| +2450822|14834|1|| +2450822|14836|1|786| +2450822|14839|1|388| +2450822|14840|1|925| +2450822|14842|1|100| +2450822|14845|1|869| +2450822|14846|1|519| +2450822|14848|1|777| +2450822|14851|1|783| +2450822|14852|1|487| +2450822|14854|1|204| +2450822|14857|1|| +2450822|14858|1|226| +2450822|14860|1|387| +2450822|14863|1|341| +2450822|14864|1|104| +2450822|14866|1|| +2450822|14869|1|302| +2450822|14870|1|| +2450822|14872|1|742| +2450822|14875|1|97| +2450822|14876|1|| +2450822|14878|1|202| +2450822|14881|1|703| +2450822|14882|1|208| +2450822|14884|1|413| +2450822|14887|1|892| +2450822|14888|1|74| +2450822|14890|1|543| +2450822|14893|1|722| +2450822|14894|1|664| +2450822|14896|1|75| +2450822|14899|1|147| +2450822|14900|1|16| +2450822|14902|1|427| +2450822|14905|1|38| +2450822|14906|1|675| +2450822|14908|1|| +2450822|14911|1|432| +2450822|14912|1|945| +2450822|14914|1|808| +2450822|14917|1|288| +2450822|14918|1|100| +2450822|14920|1|679| +2450822|14923|1|709| +2450822|14924|1|300| +2450822|14926|1|949| +2450822|14929|1|| +2450822|14930|1|| +2450822|14932|1|561| +2450822|14935|1|807| +2450822|14936|1|905| +2450822|14938|1|593| +2450822|14941|1|869| +2450822|14942|1|377| +2450822|14944|1|64| +2450822|14947|1|220| +2450822|14948|1|324| +2450822|14950|1|942| +2450822|14953|1|67| +2450822|14954|1|743| +2450822|14956|1|796| +2450822|14959|1|992| +2450822|14960|1|345| +2450822|14962|1|151| +2450822|14965|1|294| +2450822|14966|1|246| +2450822|14968|1|| +2450822|14971|1|527| +2450822|14972|1|453| +2450822|14974|1|178| +2450822|14977|1|| +2450822|14978|1|186| +2450822|14980|1|817| +2450822|14983|1|172| +2450822|14984|1|213| +2450822|14986|1|| +2450822|14989|1|508| +2450822|14990|1|889| +2450822|14992|1|974| +2450822|14995|1|726| +2450822|14996|1|136| +2450822|14998|1|564| +2450822|15001|1|365| +2450822|15002|1|738| +2450822|15004|1|750| +2450822|15007|1|741| +2450822|15008|1|222| +2450822|15010|1|810| +2450822|15013|1|883| +2450822|15014|1|29| +2450822|15016|1|125| +2450822|15019|1|297| +2450822|15020|1|573| +2450822|15022|1|66| +2450822|15025|1|842| +2450822|15026|1|699| +2450822|15028|1|718| +2450822|15031|1|683| +2450822|15032|1|492| +2450822|15034|1|730| +2450822|15037|1|296| +2450822|15038|1|773| +2450822|15040|1|955| +2450822|15043|1|54| +2450822|15044|1|82| +2450822|15046|1|| +2450822|15049|1|221| +2450822|15050|1|577| +2450822|15052|1|854| +2450822|15055|1|358| +2450822|15056|1|143| +2450822|15058|1|136| +2450822|15061|1|576| +2450822|15062|1|223| +2450822|15064|1|297| +2450822|15067|1|147| +2450822|15068|1|717| +2450822|15070|1|690| +2450822|15073|1|976| +2450822|15074|1|398| +2450822|15076|1|638| +2450822|15079|1|184| +2450822|15080|1|107| +2450822|15082|1|457| +2450822|15085|1|393| +2450822|15086|1|430| +2450822|15088|1|723| +2450822|15091|1|817| +2450822|15092|1|986| +2450822|15094|1|350| +2450822|15097|1|567| +2450822|15098|1|307| +2450822|15100|1|765| +2450822|15103|1|247| +2450822|15104|1|937| +2450822|15106|1|761| +2450822|15109|1|74| +2450822|15110|1|401| +2450822|15112|1|722| +2450822|15115|1|623| +2450822|15116|1|209| +2450822|15118|1|422| +2450822|15121|1|743| +2450822|15122|1|413| +2450822|15124|1|928| +2450822|15127|1|455| +2450822|15128|1|448| +2450822|15130|1|352| +2450822|15133|1|| +2450822|15134|1|660| +2450822|15136|1|70| +2450822|15139|1|713| +2450822|15140|1|858| +2450822|15142|1|150| +2450822|15145|1|39| +2450822|15146|1|396| +2450822|15148|1|| +2450822|15151|1|326| +2450822|15152|1|441| +2450822|15154|1|419| +2450822|15157|1|326| +2450822|15158|1|409| +2450822|15160|1|| +2450822|15163|1|385| +2450822|15164|1|785| +2450822|15166|1|633| +2450822|15169|1|525| +2450822|15170|1|34| +2450822|15172|1|944| +2450822|15175|1|329| +2450822|15176|1|784| +2450822|15178|1|846| +2450822|15181|1|879| +2450822|15182|1|281| +2450822|15184|1|499| +2450822|15187|1|139| +2450822|15188|1|939| +2450822|15190|1|817| +2450822|15193|1|583| +2450822|15194|1|822| +2450822|15196|1|623| +2450822|15199|1|902| +2450822|15200|1|99| +2450822|15202|1|950| +2450822|15205|1|227| +2450822|15206|1|995| +2450822|15208|1|155| +2450822|15211|1|501| +2450822|15212|1|571| +2450822|15214|1|449| +2450822|15217|1|744| +2450822|15218|1|| +2450822|15220|1|505| +2450822|15223|1|863| +2450822|15224|1|170| +2450822|15226|1|311| +2450822|15229|1|| +2450822|15230|1|547| +2450822|15232|1|518| +2450822|15235|1|148| +2450822|15236|1|720| +2450822|15238|1|703| +2450822|15241|1|802| +2450822|15242|1|252| +2450822|15244|1|687| +2450822|15247|1|347| +2450822|15248|1|662| +2450822|15250|1|366| +2450822|15253|1|876| +2450822|15254|1|30| +2450822|15256|1|74| +2450822|15259|1|708| +2450822|15260|1|773| +2450822|15262|1|36| +2450822|15265|1|965| +2450822|15266|1|651| +2450822|15268|1|505| +2450822|15271|1|919| +2450822|15272|1|284| +2450822|15274|1|568| +2450822|15277|1|794| +2450822|15278|1|478| +2450822|15280|1|804| +2450822|15283|1|458| +2450822|15284|1|219| +2450822|15286|1|177| +2450822|15289|1|850| +2450822|15290|1|76| +2450822|15292|1|229| +2450822|15295|1|65| +2450822|15296|1|439| +2450822|15298|1|489| +2450822|15301|1|245| +2450822|15302|1|859| +2450822|15304|1|139| +2450822|15307|1|| +2450822|15308|1|979| +2450822|15310|1|988| +2450822|15313|1|806| +2450822|15314|1|141| +2450822|15316|1|154| +2450822|15319|1|13| +2450822|15320|1|53| +2450822|15322|1|| +2450822|15325|1|518| +2450822|15326|1|362| +2450822|15328|1|769| +2450822|15331|1|711| +2450822|15332|1|262| +2450822|15334|1|964| +2450822|15337|1|113| +2450822|15338|1|835| +2450822|15340|1|747| +2450822|15343|1|313| +2450822|15344|1|634| +2450822|15346|1|826| +2450822|15349|1|96| +2450822|15350|1|738| +2450822|15352|1|830| +2450822|15355|1|560| +2450822|15356|1|260| +2450822|15358|1|282| +2450822|15361|1|380| +2450822|15362|1|63| +2450822|15364|1|371| +2450822|15367|1|430| +2450822|15368|1|227| +2450822|15370|1|827| +2450822|15373|1|115| +2450822|15374|1|225| +2450822|15376|1|382| +2450822|15379|1|682| +2450822|15380|1|115| +2450822|15382|1|122| +2450822|15385|1|86| +2450822|15386|1|319| +2450822|15388|1|573| +2450822|15391|1|66| +2450822|15392|1|74| +2450822|15394|1|453| +2450822|15397|1|316| +2450822|15398|1|434| +2450822|15400|1|140| +2450822|15403|1|340| +2450822|15404|1|462| +2450822|15406|1|105| +2450822|15409|1|42| +2450822|15410|1|| +2450822|15412|1|383| +2450822|15415|1|528| +2450822|15416|1|216| +2450822|15418|1|341| +2450822|15421|1|790| +2450822|15422|1|133| +2450822|15424|1|694| +2450822|15427|1|282| +2450822|15428|1|912| +2450822|15430|1|527| +2450822|15433|1|12| +2450822|15434|1|457| +2450822|15436|1|| +2450822|15439|1|434| +2450822|15440|1|570| +2450822|15442|1|522| +2450822|15445|1|888| +2450822|15446|1|474| +2450822|15448|1|759| +2450822|15451|1|676| +2450822|15452|1|637| +2450822|15454|1|553| +2450822|15457|1|165| +2450822|15458|1|667| +2450822|15460|1|493| +2450822|15463|1|748| +2450822|15464|1|210| +2450822|15466|1|301| +2450822|15469|1|649| +2450822|15470|1|513| +2450822|15472|1|642| +2450822|15475|1|761| +2450822|15476|1|5| +2450822|15478|1|859| +2450822|15481|1|172| +2450822|15482|1|203| +2450822|15484|1|444| +2450822|15487|1|305| +2450822|15488|1|166| +2450822|15490|1|100| +2450822|15493|1|94| +2450822|15494|1|912| +2450822|15496|1|419| +2450822|15499|1|241| +2450822|15500|1|84| +2450822|15502|1|835| +2450822|15505|1|296| +2450822|15506|1|391| +2450822|15508|1|365| +2450822|15511|1|311| +2450822|15512|1|413| +2450822|15514|1|409| +2450822|15517|1|351| +2450822|15518|1|56| +2450822|15520|1|693| +2450822|15523|1|| +2450822|15524|1|492| +2450822|15526|1|| +2450822|15529|1|602| +2450822|15530|1|268| +2450822|15532|1|538| +2450822|15535|1|954| +2450822|15536|1|796| +2450822|15538|1|940| +2450822|15541|1|842| +2450822|15542|1|612| +2450822|15544|1|140| +2450822|15547|1|239| +2450822|15548|1|96| +2450822|15550|1|822| +2450822|15553|1|900| +2450822|15554|1|667| +2450822|15556|1|824| +2450822|15559|1|44| +2450822|15560|1|493| +2450822|15562|1|259| +2450822|15565|1|| +2450822|15566|1|911| +2450822|15568|1|| +2450822|15571|1|652| +2450822|15572|1|957| +2450822|15574|1|141| +2450822|15577|1|53| +2450822|15578|1|806| +2450822|15580|1|57| +2450822|15583|1|24| +2450822|15584|1|304| +2450822|15586|1|90| +2450822|15589|1|878| +2450822|15590|1|485| +2450822|15592|1|154| +2450822|15595|1|251| +2450822|15596|1|233| +2450822|15598|1|676| +2450822|15601|1|353| +2450822|15602|1|238| +2450822|15604|1|789| +2450822|15607|1|93| +2450822|15608|1|751| +2450822|15610|1|523| +2450822|15613|1|| +2450822|15614|1|76| +2450822|15616|1|588| +2450822|15619|1|890| +2450822|15620|1|861| +2450822|15622|1|99| +2450822|15625|1|408| +2450822|15626|1|470| +2450822|15628|1|669| +2450822|15631|1|391| +2450822|15632|1|596| +2450822|15634|1|690| +2450822|15637|1|612| +2450822|15638|1|188| +2450822|15640|1|361| +2450822|15643|1|827| +2450822|15644|1|725| +2450822|15646|1|93| +2450822|15649|1|| +2450822|15650|1|301| +2450822|15652|1|24| +2450822|15655|1|493| +2450822|15656|1|954| +2450822|15658|1|487| +2450822|15661|1|590| +2450822|15662|1|134| +2450822|15664|1|677| +2450822|15667|1|646| +2450822|15668|1|129| +2450822|15670|1|849| +2450822|15673|1|438| +2450822|15674|1|182| +2450822|15676|1|581| +2450822|15679|1|694| +2450822|15680|1|850| +2450822|15682|1|840| +2450822|15685|1|907| +2450822|15686|1|953| +2450822|15688|1|634| +2450822|15691|1|885| +2450822|15692|1|368| +2450822|15694|1|928| +2450822|15697|1|416| +2450822|15698|1|461| +2450822|15700|1|695| +2450822|15703|1|602| +2450822|15704|1|603| +2450822|15706|1|790| +2450822|15709|1|789| +2450822|15710|1|373| +2450822|15712|1|337| +2450822|15715|1|978| +2450822|15716|1|634| +2450822|15718|1|677| +2450822|15721|1|716| +2450822|15722|1|361| +2450822|15724|1|742| +2450822|15727|1|850| +2450822|15728|1|192| +2450822|15730|1|568| +2450822|15733|1|349| +2450822|15734|1|| +2450822|15736|1|| +2450822|15739|1|295| +2450822|15740|1|490| +2450822|15742|1|574| +2450822|15745|1|213| +2450822|15746|1|309| +2450822|15748|1|864| +2450822|15751|1|426| +2450822|15752|1|878| +2450822|15754|1|836| +2450822|15757|1|812| +2450822|15758|1|754| +2450822|15760|1|203| +2450822|15763|1|354| +2450822|15764|1|561| +2450822|15766|1|950| +2450822|15769|1|862| +2450822|15770|1|870| +2450822|15772|1|743| +2450822|15775|1|245| +2450822|15776|1|259| +2450822|15778|1|24| +2450822|15781|1|624| +2450822|15782|1|124| +2450822|15784|1|252| +2450822|15787|1|644| +2450822|15788|1|724| +2450822|15790|1|803| +2450822|15793|1|940| +2450822|15794|1|977| +2450822|15796|1|619| +2450822|15799|1|280| +2450822|15800|1|334| +2450822|15802|1|4| +2450822|15805|1|838| +2450822|15806|1|934| +2450822|15808|1|661| +2450822|15811|1|901| +2450822|15812|1|| +2450822|15814|1|642| +2450822|15817|1|780| +2450822|15818|1|240| +2450822|15820|1|375| +2450822|15823|1|884| +2450822|15824|1|714| +2450822|15826|1|415| +2450822|15829|1|642| +2450822|15830|1|571| +2450822|15832|1|375| +2450822|15835|1|286| +2450822|15836|1|691| +2450822|15838|1|567| +2450822|15841|1|509| +2450822|15842|1|691| +2450822|15844|1|813| +2450822|15847|1|421| +2450822|15848|1|32| +2450822|15850|1|768| +2450822|15853|1|389| +2450822|15854|1|411| +2450822|15856|1|162| +2450822|15859|1|657| +2450822|15860|1|162| +2450822|15862|1|136| +2450822|15865|1|980| +2450822|15866|1|925| +2450822|15868|1|216| +2450822|15871|1|996| +2450822|15872|1|908| +2450822|15874|1|1| +2450822|15877|1|350| +2450822|15878|1|| +2450822|15880|1|194| +2450822|15883|1|794| +2450822|15884|1|528| +2450822|15886|1|863| +2450822|15889|1|963| +2450822|15890|1|704| +2450822|15892|1|887| +2450822|15895|1|462| +2450822|15896|1|619| +2450822|15898|1|792| +2450822|15901|1|731| +2450822|15902|1|19| +2450822|15904|1|403| +2450822|15907|1|925| +2450822|15908|1|227| +2450822|15910|1|852| +2450822|15913|1|876| +2450822|15914|1|182| +2450822|15916|1|481| +2450822|15919|1|494| +2450822|15920|1|564| +2450822|15922|1|990| +2450822|15925|1|921| +2450822|15926|1|755| +2450822|15928|1|662| +2450822|15931|1|282| +2450822|15932|1|651| +2450822|15934|1|825| +2450822|15937|1|861| +2450822|15938|1|759| +2450822|15940|1|174| +2450822|15943|1|579| +2450822|15944|1|281| +2450822|15946|1|531| +2450822|15949|1|107| +2450822|15950|1|807| +2450822|15952|1|| +2450822|15955|1|301| +2450822|15956|1|551| +2450822|15958|1|| +2450822|15961|1|509| +2450822|15962|1|932| +2450822|15964|1|909| +2450822|15967|1|446| +2450822|15968|1|857| +2450822|15970|1|750| +2450822|15973|1|352| +2450822|15974|1|616| +2450822|15976|1|72| +2450822|15979|1|479| +2450822|15980|1|160| +2450822|15982|1|996| +2450822|15985|1|688| +2450822|15986|1|537| +2450822|15988|1|997| +2450822|15991|1|110| +2450822|15992|1|101| +2450822|15994|1|| +2450822|15997|1|582| +2450822|15998|1|218| +2450822|16000|1|60| +2450822|16003|1|| +2450822|16004|1|84| +2450822|16006|1|736| +2450822|16009|1|301| +2450822|16010|1|| +2450822|16012|1|335| +2450822|16015|1|504| +2450822|16016|1|959| +2450822|16018|1|683| +2450822|16021|1|248| +2450822|16022|1|207| +2450822|16024|1|547| +2450822|16027|1|733| +2450822|16028|1|368| +2450822|16030|1|327| +2450822|16033|1|19| +2450822|16034|1|541| +2450822|16036|1|245| +2450822|16039|1|650| +2450822|16040|1|338| +2450822|16042|1|836| +2450822|16045|1|13| +2450822|16046|1|987| +2450822|16048|1|201| +2450822|16051|1|994| +2450822|16052|1|60| +2450822|16054|1|475| +2450822|16057|1|866| +2450822|16058|1|787| +2450822|16060|1|228| +2450822|16063|1|452| +2450822|16064|1|717| +2450822|16066|1|961| +2450822|16069|1|317| +2450822|16070|1|| +2450822|16072|1|| +2450822|16075|1|289| +2450822|16076|1|141| +2450822|16078|1|496| +2450822|16081|1|632| +2450822|16082|1|788| +2450822|16084|1|758| +2450822|16087|1|776| +2450822|16088|1|180| +2450822|16090|1|| +2450822|16093|1|432| +2450822|16094|1|475| +2450822|16096|1|523| +2450822|16099|1|186| +2450822|16100|1|347| +2450822|16102|1|566| +2450822|16105|1|139| +2450822|16106|1|| +2450822|16108|1|964| +2450822|16111|1|260| +2450822|16112|1|396| +2450822|16114|1|155| +2450822|16117|1|310| +2450822|16118|1|774| +2450822|16120|1|793| +2450822|16123|1|453| +2450822|16124|1|756| +2450822|16126|1|68| +2450822|16129|1|748| +2450822|16130|1|740| +2450822|16132|1|4| +2450822|16135|1|454| +2450822|16136|1|729| +2450822|16138|1|| +2450822|16141|1|431| +2450822|16142|1|434| +2450822|16144|1|413| +2450822|16147|1|314| +2450822|16148|1|216| +2450822|16150|1|665| +2450822|16153|1|803| +2450822|16154|1|478| +2450822|16156|1|881| +2450822|16159|1|101| +2450822|16160|1|| +2450822|16162|1|806| +2450822|16165|1|236| +2450822|16166|1|447| +2450822|16168|1|560| +2450822|16171|1|342| +2450822|16172|1|209| +2450822|16174|1|112| +2450822|16177|1|86| +2450822|16178|1|450| +2450822|16180|1|211| +2450822|16183|1|56| +2450822|16184|1|220| +2450822|16186|1|21| +2450822|16189|1|| +2450822|16190|1|3| +2450822|16192|1|958| +2450822|16195|1|743| +2450822|16196|1|847| +2450822|16198|1|660| +2450822|16201|1|565| +2450822|16202|1|97| +2450822|16204|1|974| +2450822|16207|1|1000| +2450822|16208|1|729| +2450822|16210|1|955| +2450822|16213|1|248| +2450822|16214|1|522| +2450822|16216|1|573| +2450822|16219|1|446| +2450822|16220|1|559| +2450822|16222|1|937| +2450822|16225|1|271| +2450822|16226|1|9| +2450822|16228|1|365| +2450822|16231|1|545| +2450822|16232|1|842| +2450822|16234|1|530| +2450822|16237|1|67| +2450822|16238|1|951| +2450822|16240|1|815| +2450822|16243|1|877| +2450822|16244|1|911| +2450822|16246|1|331| +2450822|16249|1|606| +2450822|16250|1|780| +2450822|16252|1|601| +2450822|16255|1|73| +2450822|16256|1|723| +2450822|16258|1|152| +2450822|16261|1|730| +2450822|16262|1|550| +2450822|16264|1|262| +2450822|16267|1|| +2450822|16268|1|567| +2450822|16270|1|380| +2450822|16273|1|861| +2450822|16274|1|12| +2450822|16276|1|215| +2450822|16279|1|973| +2450822|16280|1|898| +2450822|16282|1|989| +2450822|16285|1|972| +2450822|16286|1|906| +2450822|16288|1|64| +2450822|16291|1|974| +2450822|16292|1|791| +2450822|16294|1|784| +2450822|16297|1|654| +2450822|16298|1|780| +2450822|16300|1|321| +2450822|16303|1|991| +2450822|16304|1|786| +2450822|16306|1|350| +2450822|16309|1|409| +2450822|16310|1|277| +2450822|16312|1|264| +2450822|16315|1|| +2450822|16316|1|925| +2450822|16318|1|259| +2450822|16321|1|160| +2450822|16322|1|909| +2450822|16324|1|337| +2450822|16327|1|373| +2450822|16328|1|438| +2450822|16330|1|560| +2450822|16333|1|894| +2450822|16334|1|881| +2450822|16336|1|726| +2450822|16339|1|597| +2450822|16340|1|| +2450822|16342|1|679| +2450822|16345|1|479| +2450822|16346|1|397| +2450822|16348|1|137| +2450822|16351|1|401| +2450822|16352|1|824| +2450822|16354|1|341| +2450822|16357|1|314| +2450822|16358|1|841| +2450822|16360|1|80| +2450822|16363|1|977| +2450822|16364|1|958| +2450822|16366|1|578| +2450822|16369|1|719| +2450822|16370|1|925| +2450822|16372|1|888| +2450822|16375|1|988| +2450822|16376|1|730| +2450822|16378|1|939| +2450822|16381|1|288| +2450822|16382|1|722| +2450822|16384|1|759| +2450822|16387|1|978| +2450822|16388|1|219| +2450822|16390|1|312| +2450822|16393|1|786| +2450822|16394|1|121| +2450822|16396|1|345| +2450822|16399|1|763| +2450822|16400|1|200| +2450822|16402|1|382| +2450822|16405|1|832| +2450822|16406|1|758| +2450822|16408|1|798| +2450822|16411|1|192| +2450822|16412|1|813| +2450822|16414|1|576| +2450822|16417|1|133| +2450822|16418|1|490| +2450822|16420|1|856| +2450822|16423|1|75| +2450822|16424|1|897| +2450822|16426|1|| +2450822|16429|1|719| +2450822|16430|1|639| +2450822|16432|1|972| +2450822|16435|1|565| +2450822|16436|1|433| +2450822|16438|1|446| +2450822|16441|1|298| +2450822|16442|1|782| +2450822|16444|1|849| +2450822|16447|1|547| +2450822|16448|1|135| +2450822|16450|1|| +2450822|16453|1|942| +2450822|16454|1|11| +2450822|16456|1|133| +2450822|16459|1|264| +2450822|16460|1|663| +2450822|16462|1|| +2450822|16465|1|585| +2450822|16466|1|533| +2450822|16468|1|271| +2450822|16471|1|965| +2450822|16472|1|496| +2450822|16474|1|211| +2450822|16477|1|994| +2450822|16478|1|167| +2450822|16480|1|336| +2450822|16483|1|468| +2450822|16484|1|801| +2450822|16486|1|6| +2450822|16489|1|336| +2450822|16490|1|95| +2450822|16492|1|1000| +2450822|16495|1|589| +2450822|16496|1|808| +2450822|16498|1|775| +2450822|16501|1|143| +2450822|16502|1|273| +2450822|16504|1|351| +2450822|16507|1|670| +2450822|16508|1|990| +2450822|16510|1|40| +2450822|16513|1|714| +2450822|16514|1|730| +2450822|16516|1|890| +2450822|16519|1|553| +2450822|16520|1|499| +2450822|16522|1|663| +2450822|16525|1|441| +2450822|16526|1|552| +2450822|16528|1|903| +2450822|16531|1|653| +2450822|16532|1|536| +2450822|16534|1|329| +2450822|16537|1|896| +2450822|16538|1|| +2450822|16540|1|148| +2450822|16543|1|848| +2450822|16544|1|771| +2450822|16546|1|258| +2450822|16549|1|380| +2450822|16550|1|733| +2450822|16552|1|669| +2450822|16555|1|730| +2450822|16556|1|157| +2450822|16558|1|142| +2450822|16561|1|432| +2450822|16562|1|679| +2450822|16564|1|484| +2450822|16567|1|694| +2450822|16568|1|775| +2450822|16570|1|224| +2450822|16573|1|363| +2450822|16574|1|992| +2450822|16576|1|674| +2450822|16579|1|231| +2450822|16580|1|781| +2450822|16582|1|103| +2450822|16585|1|640| +2450822|16586|1|598| +2450822|16588|1|149| +2450822|16591|1|407| +2450822|16592|1|701| +2450822|16594|1|834| +2450822|16597|1|259| +2450822|16598|1|742| +2450822|16600|1|2| +2450822|16603|1|26| +2450822|16604|1|675| +2450822|16606|1|171| +2450822|16609|1|813| +2450822|16610|1|878| +2450822|16612|1|537| +2450822|16615|1|255| +2450822|16616|1|941| +2450822|16618|1|29| +2450822|16621|1|837| +2450822|16622|1|758| +2450822|16624|1|591| +2450822|16627|1|424| +2450822|16628|1|335| +2450822|16630|1|317| +2450822|16633|1|741| +2450822|16634|1|565| +2450822|16636|1|641| +2450822|16639|1|703| +2450822|16640|1|230| +2450822|16642|1|509| +2450822|16645|1|165| +2450822|16646|1|329| +2450822|16648|1|935| +2450822|16651|1|616| +2450822|16652|1|992| +2450822|16654|1|715| +2450822|16657|1|305| +2450822|16658|1|43| +2450822|16660|1|712| +2450822|16663|1|575| +2450822|16664|1|384| +2450822|16666|1|656| +2450822|16669|1|532| +2450822|16670|1|753| +2450822|16672|1|685| +2450822|16675|1|911| +2450822|16676|1|318| +2450822|16678|1|500| +2450822|16681|1|247| +2450822|16682|1|46| +2450822|16684|1|531| +2450822|16687|1|745| +2450822|16688|1|615| +2450822|16690|1|108| +2450822|16693|1|341| +2450822|16694|1|214| +2450822|16696|1|| +2450822|16699|1|169| +2450822|16700|1|233| +2450822|16702|1|506| +2450822|16705|1|| +2450822|16706|1|951| +2450822|16708|1|264| +2450822|16711|1|514| +2450822|16712|1|23| +2450822|16714|1|18| +2450822|16717|1|943| +2450822|16718|1|817| +2450822|16720|1|| +2450822|16723|1|712| +2450822|16724|1|634| +2450822|16726|1|542| +2450822|16729|1|| +2450822|16730|1|317| +2450822|16732|1|978| +2450822|16735|1|2| +2450822|16736|1|560| +2450822|16738|1|475| +2450822|16741|1|293| +2450822|16742|1|511| +2450822|16744|1|667| +2450822|16747|1|290| +2450822|16748|1|893| +2450822|16750|1|220| +2450822|16753|1|859| +2450822|16754|1|613| +2450822|16756|1|455| +2450822|16759|1|767| +2450822|16760|1|76| +2450822|16762|1|473| +2450822|16765|1|682| +2450822|16766|1|593| +2450822|16768|1|748| +2450822|16771|1|320| +2450822|16772|1|73| +2450822|16774|1|109| +2450822|16777|1|| +2450822|16778|1|412| +2450822|16780|1|437| +2450822|16783|1|72| +2450822|16784|1|737| +2450822|16786|1|103| +2450822|16789|1|853| +2450822|16790|1|108| +2450822|16792|1|264| +2450822|16795|1|566| +2450822|16796|1|114| +2450822|16798|1|| +2450822|16801|1|925| +2450822|16802|1|| +2450822|16804|1|350| +2450822|16807|1|750| +2450822|16808|1|128| +2450822|16810|1|563| +2450822|16813|1|489| +2450822|16814|1|655| +2450822|16816|1|691| +2450822|16819|1|572| +2450822|16820|1|407| +2450822|16822|1|20| +2450822|16825|1|592| +2450822|16826|1|127| +2450822|16828|1|45| +2450822|16831|1|661| +2450822|16832|1|215| +2450822|16834|1|592| +2450822|16837|1|839| +2450822|16838|1|826| +2450822|16840|1|164| +2450822|16843|1|547| +2450822|16844|1|761| +2450822|16846|1|607| +2450822|16849|1|634| +2450822|16850|1|| +2450822|16852|1|| +2450822|16855|1|293| +2450822|16856|1|568| +2450822|16858|1|862| +2450822|16861|1|834| +2450822|16862|1|703| +2450822|16864|1|356| +2450822|16867|1|48| +2450822|16868|1|662| +2450822|16870|1|736| +2450822|16873|1|697| +2450822|16874|1|369| +2450822|16876|1|737| +2450822|16879|1|881| +2450822|16880|1|207| +2450822|16882|1|816| +2450822|16885|1|| +2450822|16886|1|235| +2450822|16888|1|892| +2450822|16891|1|214| +2450822|16892|1|902| +2450822|16894|1|958| +2450822|16897|1|851| +2450822|16898|1|52| +2450822|16900|1|47| +2450822|16903|1|455| +2450822|16904|1|947| +2450822|16906|1|555| +2450822|16909|1|693| +2450822|16910|1|211| +2450822|16912|1|722| +2450822|16915|1|487| +2450822|16916|1|487| +2450822|16918|1|388| +2450822|16921|1|88| +2450822|16922|1|| +2450822|16924|1|158| +2450822|16927|1|195| +2450822|16928|1|818| +2450822|16930|1|| +2450822|16933|1|998| +2450822|16934|1|322| +2450822|16936|1|463| +2450822|16939|1|164| +2450822|16940|1|| +2450822|16942|1|324| +2450822|16945|1|672| +2450822|16946|1|825| +2450822|16948|1|431| +2450822|16951|1|624| +2450822|16952|1|922| +2450822|16954|1|998| +2450822|16957|1|930| +2450822|16958|1|73| +2450822|16960|1|141| +2450822|16963|1|570| +2450822|16964|1|78| +2450822|16966|1|67| +2450822|16969|1|66| +2450822|16970|1|425| +2450822|16972|1|115| +2450822|16975|1|111| +2450822|16976|1|968| +2450822|16978|1|513| +2450822|16981|1|753| +2450822|16982|1|137| +2450822|16984|1|655| +2450822|16987|1|| +2450822|16988|1|633| +2450822|16990|1|754| +2450822|16993|1|961| +2450822|16994|1|944| +2450822|16996|1|741| +2450822|16999|1|139| +2450822|17000|1|976| +2450822|17002|1|768| +2450822|17005|1|483| +2450822|17006|1|| +2450822|17008|1|920| +2450822|17011|1|63| +2450822|17012|1|645| +2450822|17014|1|282| +2450822|17017|1|21| +2450822|17018|1|513| +2450822|17020|1|828| +2450822|17023|1|917| +2450822|17024|1|222| +2450822|17026|1|335| +2450822|17029|1|699| +2450822|17030|1|59| +2450822|17032|1|263| +2450822|17035|1|878| +2450822|17036|1|975| +2450822|17038|1|688| +2450822|17041|1|77| +2450822|17042|1|481| +2450822|17044|1|241| +2450822|17047|1|826| +2450822|17048|1|422| +2450822|17050|1|337| +2450822|17053|1|| +2450822|17054|1|752| +2450822|17056|1|522| +2450822|17059|1|586| +2450822|17060|1|350| +2450822|17062|1|| +2450822|17065|1|188| +2450822|17066|1|545| +2450822|17068|1|925| +2450822|17071|1|668| +2450822|17072|1|426| +2450822|17074|1|677| +2450822|17077|1|777| +2450822|17078|1|593| +2450822|17080|1|414| +2450822|17083|1|703| +2450822|17084|1|159| +2450822|17086|1|431| +2450822|17089|1|467| +2450822|17090|1|736| +2450822|17092|1|230| +2450822|17095|1|331| +2450822|17096|1|886| +2450822|17098|1|465| +2450822|17101|1|338| +2450822|17102|1|687| +2450822|17104|1|311| +2450822|17107|1|771| +2450822|17108|1|200| +2450822|17110|1|943| +2450822|17113|1|238| +2450822|17114|1|736| +2450822|17116|1|550| +2450822|17119|1|336| +2450822|17120|1|| +2450822|17122|1|743| +2450822|17125|1|887| +2450822|17126|1|780| +2450822|17128|1|416| +2450822|17131|1|831| +2450822|17132|1|34| +2450822|17134|1|534| +2450822|17137|1|| +2450822|17138|1|140| +2450822|17140|1|466| +2450822|17143|1|685| +2450822|17144|1|120| +2450822|17146|1|219| +2450822|17149|1|135| +2450822|17150|1|129| +2450822|17152|1|979| +2450822|17155|1|628| +2450822|17156|1|738| +2450822|17158|1|122| +2450822|17161|1|219| +2450822|17162|1|294| +2450822|17164|1|433| +2450822|17167|1|489| +2450822|17168|1|852| +2450822|17170|1|201| +2450822|17173|1|721| +2450822|17174|1|313| +2450822|17176|1|| +2450822|17179|1|618| +2450822|17180|1|905| +2450822|17182|1|720| +2450822|17185|1|950| +2450822|17186|1|| +2450822|17188|1|| +2450822|17191|1|615| +2450822|17192|1|806| +2450822|17194|1|584| +2450822|17197|1|766| +2450822|17198|1|121| +2450822|17200|1|739| +2450822|17203|1|606| +2450822|17204|1|536| +2450822|17206|1|219| +2450822|17209|1|311| +2450822|17210|1|338| +2450822|17212|1|624| +2450822|17215|1|207| +2450822|17216|1|902| +2450822|17218|1|540| +2450822|17221|1|| +2450822|17222|1|378| +2450822|17224|1|604| +2450822|17227|1|589| +2450822|17228|1|534| +2450822|17230|1|967| +2450822|17233|1|741| +2450822|17234|1|62| +2450822|17236|1|263| +2450822|17239|1|969| +2450822|17240|1|252| +2450822|17242|1|212| +2450822|17245|1|883| +2450822|17246|1|891| +2450822|17248|1|865| +2450822|17251|1|825| +2450822|17252|1|631| +2450822|17254|1|560| +2450822|17257|1|315| +2450822|17258|1|| +2450822|17260|1|744| +2450822|17263|1|761| +2450822|17264|1|473| +2450822|17266|1|883| +2450822|17269|1|393| +2450822|17270|1|777| +2450822|17272|1|672| +2450822|17275|1|633| +2450822|17276|1|238| +2450822|17278|1|79| +2450822|17281|1|274| +2450822|17282|1|671| +2450822|17284|1|454| +2450822|17287|1|880| +2450822|17288|1|136| +2450822|17290|1|232| +2450822|17293|1|208| +2450822|17294|1|351| +2450822|17296|1|398| +2450822|17299|1|540| +2450822|17300|1|279| +2450822|17302|1|381| +2450822|17305|1|528| +2450822|17306|1|427| +2450822|17308|1|83| +2450822|17311|1|875| +2450822|17312|1|426| +2450822|17314|1|928| +2450822|17317|1|855| +2450822|17318|1|699| +2450822|17320|1|40| +2450822|17323|1|571| +2450822|17324|1|392| +2450822|17326|1|816| +2450822|17329|1|273| +2450822|17330|1|257| +2450822|17332|1|320| +2450822|17335|1|806| +2450822|17336|1|952| +2450822|17338|1|964| +2450822|17341|1|591| +2450822|17342|1|72| +2450822|17344|1|276| +2450822|17347|1|| +2450822|17348|1|937| +2450822|17350|1|182| +2450822|17353|1|| +2450822|17354|1|894| +2450822|17356|1|527| +2450822|17359|1|369| +2450822|17360|1|334| +2450822|17362|1|828| +2450822|17365|1|199| +2450822|17366|1|461| +2450822|17368|1|458| +2450822|17371|1|631| +2450822|17372|1|402| +2450822|17374|1|117| +2450822|17377|1|640| +2450822|17378|1|383| +2450822|17380|1|205| +2450822|17383|1|856| +2450822|17384|1|29| +2450822|17386|1|317| +2450822|17389|1|678| +2450822|17390|1|223| +2450822|17392|1|549| +2450822|17395|1|490| +2450822|17396|1|949| +2450822|17398|1|174| +2450822|17401|1|860| +2450822|17402|1|335| +2450822|17404|1|| +2450822|17407|1|49| +2450822|17408|1|384| +2450822|17410|1|961| +2450822|17413|1|| +2450822|17414|1|110| +2450822|17416|1|341| +2450822|17419|1|229| +2450822|17420|1|502| +2450822|17422|1|310| +2450822|17425|1|552| +2450822|17426|1|539| +2450822|17428|1|544| +2450822|17431|1|438| +2450822|17432|1|992| +2450822|17434|1|462| +2450822|17437|1|784| +2450822|17438|1|995| +2450822|17440|1|757| +2450822|17443|1|588| +2450822|17444|1|941| +2450822|17446|1|142| +2450822|17449|1|398| +2450822|17450|1|879| +2450822|17452|1|403| +2450822|17455|1|959| +2450822|17456|1|516| +2450822|17458|1|859| +2450822|17461|1|| +2450822|17462|1|442| +2450822|17464|1|46| +2450822|17467|1|562| +2450822|17468|1|452| +2450822|17470|1|10| +2450822|17473|1|206| +2450822|17474|1|582| +2450822|17476|1|168| +2450822|17479|1|477| +2450822|17480|1|145| +2450822|17482|1|649| +2450822|17485|1|349| +2450822|17486|1|740| +2450822|17488|1|489| +2450822|17491|1|882| +2450822|17492|1|244| +2450822|17494|1|127| +2450822|17497|1|603| +2450822|17498|1|380| +2450822|17500|1|622| +2450822|17503|1|203| +2450822|17504|1|| +2450822|17506|1|| +2450822|17509|1|488| +2450822|17510|1|294| +2450822|17512|1|436| +2450822|17515|1|593| +2450822|17516|1|279| +2450822|17518|1|117| +2450822|17521|1|648| +2450822|17522|1|63| +2450822|17524|1|953| +2450822|17527|1|244| +2450822|17528|1|744| +2450822|17530|1|573| +2450822|17533|1|399| +2450822|17534|1|990| +2450822|17536|1|222| +2450822|17539|1|| +2450822|17540|1|525| +2450822|17542|1|950| +2450822|17545|1|51| +2450822|17546|1|792| +2450822|17548|1|352| +2450822|17551|1|641| +2450822|17552|1|985| +2450822|17554|1|833| +2450822|17557|1|251| +2450822|17558|1|917| +2450822|17560|1|603| +2450822|17563|1|523| +2450822|17564|1|834| +2450822|17566|1|127| +2450822|17569|1|1| +2450822|17570|1|558| +2450822|17572|1|198| +2450822|17575|1|919| +2450822|17576|1|2| +2450822|17578|1|844| +2450822|17581|1|987| +2450822|17582|1|877| +2450822|17584|1|702| +2450822|17587|1|269| +2450822|17588|1|571| +2450822|17590|1|495| +2450822|17593|1|403| +2450822|17594|1|770| +2450822|17596|1|451| +2450822|17599|1|403| +2450822|17600|1|163| +2450822|17602|1|79| +2450822|17605|1|686| +2450822|17606|1|787| +2450822|17608|1|343| +2450822|17611|1|544| +2450822|17612|1|523| +2450822|17614|1|612| +2450822|17617|1|177| +2450822|17618|1|575| +2450822|17620|1|298| +2450822|17623|1|98| +2450822|17624|1|787| +2450822|17626|1|696| +2450822|17629|1|169| +2450822|17630|1|| +2450822|17632|1|469| +2450822|17635|1|135| +2450822|17636|1|605| +2450822|17638|1|702| +2450822|17641|1|130| +2450822|17642|1|812| +2450822|17644|1|304| +2450822|17647|1|765| +2450822|17648|1|802| +2450822|17650|1|121| +2450822|17653|1|138| +2450822|17654|1|485| +2450822|17656|1|257| +2450822|17659|1|455| +2450822|17660|1|387| +2450822|17662|1|666| +2450822|17665|1|934| +2450822|17666|1|193| +2450822|17668|1|367| +2450822|17671|1|7| +2450822|17672|1|705| +2450822|17674|1|975| +2450822|17677|1|479| +2450822|17678|1|185| +2450822|17680|1|418| +2450822|17683|1|329| +2450822|17684|1|889| +2450822|17686|1|82| +2450822|17689|1|957| +2450822|17690|1|542| +2450822|17692|1|5| +2450822|17695|1|815| +2450822|17696|1|352| +2450822|17698|1|884| +2450822|17701|1|299| +2450822|17702|1|| +2450822|17704|1|71| +2450822|17707|1|610| +2450822|17708|1|579| +2450822|17710|1|229| +2450822|17713|1|44| +2450822|17714|1|743| +2450822|17716|1|310| +2450822|17719|1|394| +2450822|17720|1|54| +2450822|17722|1|275| +2450822|17725|1|107| +2450822|17726|1|743| +2450822|17728|1|151| +2450822|17731|1|612| +2450822|17732|1|705| +2450822|17734|1|288| +2450822|17737|1|783| +2450822|17738|1|176| +2450822|17740|1|258| +2450822|17743|1|842| +2450822|17744|1|253| +2450822|17746|1|551| +2450822|17749|1|164| +2450822|17750|1|593| +2450822|17752|1|52| +2450822|17755|1|356| +2450822|17756|1|260| +2450822|17758|1|758| +2450822|17761|1|919| +2450822|17762|1|602| +2450822|17764|1|966| +2450822|17767|1|816| +2450822|17768|1|397| +2450822|17770|1|615| +2450822|17773|1|631| +2450822|17774|1|317| +2450822|17776|1|497| +2450822|17779|1|670| +2450822|17780|1|891| +2450822|17782|1|311| +2450822|17785|1|485| +2450822|17786|1|962| +2450822|17788|1|611| +2450822|17791|1|644| +2450822|17792|1|630| +2450822|17794|1|188| +2450822|17797|1|979| +2450822|17798|1|956| +2450822|17800|1|941| +2450822|17803|1|612| +2450822|17804|1|685| +2450822|17806|1|52| +2450822|17809|1|745| +2450822|17810|1|633| +2450822|17812|1|867| +2450822|17815|1|87| +2450822|17816|1|513| +2450822|17818|1|217| +2450822|17821|1|268| +2450822|17822|1|| +2450822|17824|1|815| +2450822|17827|1|329| +2450822|17828|1|787| +2450822|17830|1|808| +2450822|17833|1|664| +2450822|17834|1|809| +2450822|17836|1|290| +2450822|17839|1|887| +2450822|17840|1|792| +2450822|17842|1|13| +2450822|17845|1|714| +2450822|17846|1|204| +2450822|17848|1|813| +2450822|17851|1|166| +2450822|17852|1|960| +2450822|17854|1|359| +2450822|17857|1|| +2450822|17858|1|899| +2450822|17860|1|312| +2450822|17863|1|396| +2450822|17864|1|863| +2450822|17866|1|575| +2450822|17869|1|| +2450822|17870|1|276| +2450822|17872|1|134| +2450822|17875|1|677| +2450822|17876|1|920| +2450822|17878|1|7| +2450822|17881|1|254| +2450822|17882|1|596| +2450822|17884|1|302| +2450822|17887|1|385| +2450822|17888|1|538| +2450822|17890|1|39| +2450822|17893|1|46| +2450822|17894|1|| +2450822|17896|1|565| +2450822|17899|1|140| +2450822|17900|1|| +2450822|17902|1|16| +2450822|17905|1|930| +2450822|17906|1|935| +2450822|17908|1|284| +2450822|17911|1|33| +2450822|17912|1|702| +2450822|17914|1|985| +2450822|17917|1|868| +2450822|17918|1|462| +2450822|17920|1|398| +2450822|17923|1|| +2450822|17924|1|869| +2450822|17926|1|288| +2450822|17929|1|148| +2450822|17930|1|267| +2450822|17932|1|450| +2450822|17935|1|480| +2450822|17936|1|134| +2450822|17938|1|502| +2450822|17941|1|546| +2450822|17942|1|815| +2450822|17944|1|183| +2450822|17947|1|851| +2450822|17948|1|28| +2450822|17950|1|346| +2450822|17953|1|881| +2450822|17954|1|830| +2450822|17956|1|488| +2450822|17959|1|627| +2450822|17960|1|250| +2450822|17962|1|| +2450822|17965|1|598| +2450822|17966|1|77| +2450822|17968|1|889| +2450822|17971|1|349| +2450822|17972|1|334| +2450822|17974|1|177| +2450822|17977|1|| +2450822|17978|1|592| +2450822|17980|1|| +2450822|17983|1|273| +2450822|17984|1|330| +2450822|17986|1|764| +2450822|17989|1|557| +2450822|17990|1|268| +2450822|17992|1|203| +2450822|17995|1|195| +2450822|17996|1|657| +2450822|17998|1|711| +2450822|1|2|415| +2450822|2|2|317| +2450822|4|2|580| +2450822|7|2|138| +2450822|8|2|333| +2450822|10|2|168| +2450822|13|2|391| +2450822|14|2|290| +2450822|16|2|209| +2450822|19|2|964| +2450822|20|2|415| +2450822|22|2|185| +2450822|25|2|247| +2450822|26|2|192| +2450822|28|2|119| +2450822|31|2|37| +2450822|32|2|18| +2450822|34|2|425| +2450822|37|2|992| +2450822|38|2|822| +2450822|40|2|732| +2450822|43|2|| +2450822|44|2|392| +2450822|46|2|629| +2450822|49|2|38| +2450822|50|2|501| +2450822|52|2|799| +2450822|55|2|714| +2450822|56|2|450| +2450822|58|2|268| +2450822|61|2|589| +2450822|62|2|494| +2450822|64|2|539| +2450822|67|2|791| +2450822|68|2|924| +2450822|70|2|614| +2450822|73|2|841| +2450822|74|2|73| +2450822|76|2|225| +2450822|79|2|353| +2450822|80|2|105| +2450822|82|2|974| +2450822|85|2|519| +2450822|86|2|273| +2450822|88|2|741| +2450822|91|2|892| +2450822|92|2|717| +2450822|94|2|320| +2450822|97|2|965| +2450822|98|2|760| +2450822|100|2|769| +2450822|103|2|11| +2450822|104|2|84| +2450822|106|2|561| +2450822|109|2|116| +2450822|110|2|916| +2450822|112|2|46| +2450822|115|2|990| +2450822|116|2|475| +2450822|118|2|240| +2450822|121|2|716| +2450822|122|2|427| +2450822|124|2|855| +2450822|127|2|107| +2450822|128|2|951| +2450822|130|2|768| +2450822|133|2|913| +2450822|134|2|440| +2450822|136|2|804| +2450822|139|2|207| +2450822|140|2|232| +2450822|142|2|780| +2450822|145|2|608| +2450822|146|2|220| +2450822|148|2|94| +2450822|151|2|490| +2450822|152|2|552| +2450822|154|2|981| +2450822|157|2|895| +2450822|158|2|648| +2450822|160|2|1000| +2450822|163|2|304| +2450822|164|2|18| +2450822|166|2|533| +2450822|169|2|| +2450822|170|2|980| +2450822|172|2|69| +2450822|175|2|500| +2450822|176|2|945| +2450822|178|2|551| +2450822|181|2|989| +2450822|182|2|874| +2450822|184|2|791| +2450822|187|2|472| +2450822|188|2|434| +2450822|190|2|800| +2450822|193|2|409| +2450822|194|2|808| +2450822|196|2|| +2450822|199|2|576| +2450822|200|2|920| +2450822|202|2|856| +2450822|205|2|| +2450822|206|2|850| +2450822|208|2|118| +2450822|211|2|271| +2450822|212|2|827| +2450822|214|2|563| +2450822|217|2|940| +2450822|218|2|621| +2450822|220|2|245| +2450822|223|2|293| +2450822|224|2|520| +2450822|226|2|211| +2450822|229|2|684| +2450822|230|2|371| +2450822|232|2|963| +2450822|235|2|519| +2450822|236|2|432| +2450822|238|2|38| +2450822|241|2|842| +2450822|242|2|263| +2450822|244|2|693| +2450822|247|2|585| +2450822|248|2|639| +2450822|250|2|34| +2450822|253|2|777| +2450822|254|2|616| +2450822|256|2|623| +2450822|259|2|18| +2450822|260|2|118| +2450822|262|2|305| +2450822|265|2|995| +2450822|266|2|767| +2450822|268|2|732| +2450822|271|2|539| +2450822|272|2|45| +2450822|274|2|445| +2450822|277|2|591| +2450822|278|2|399| +2450822|280|2|517| +2450822|283|2|372| +2450822|284|2|89| +2450822|286|2|399| +2450822|289|2|918| +2450822|290|2|286| +2450822|292|2|702| +2450822|295|2|517| +2450822|296|2|9| +2450822|298|2|698| +2450822|301|2|842| +2450822|302|2|947| +2450822|304|2|131| +2450822|307|2|441| +2450822|308|2|924| +2450822|310|2|691| +2450822|313|2|151| +2450822|314|2|529| +2450822|316|2|399| +2450822|319|2|140| +2450822|320|2|926| +2450822|322|2|700| +2450822|325|2|512| +2450822|326|2|48| +2450822|328|2|243| +2450822|331|2|494| +2450822|332|2|658| +2450822|334|2|504| +2450822|337|2|322| +2450822|338|2|308| +2450822|340|2|7| +2450822|343|2|266| +2450822|344|2|736| +2450822|346|2|462| +2450822|349|2|969| +2450822|350|2|246| +2450822|352|2|291| +2450822|355|2|287| +2450822|356|2|636| +2450822|358|2|113| +2450822|361|2|989| +2450822|362|2|813| +2450822|364|2|570| +2450822|367|2|1000| +2450822|368|2|292| +2450822|370|2|446| +2450822|373|2|94| +2450822|374|2|724| +2450822|376|2|225| +2450822|379|2|711| +2450822|380|2|654| +2450822|382|2|367| +2450822|385|2|425| +2450822|386|2|524| +2450822|388|2|828| +2450822|391|2|604| +2450822|392|2|172| +2450822|394|2|| +2450822|397|2|43| +2450822|398|2|691| +2450822|400|2|593| +2450822|403|2|84| +2450822|404|2|261| +2450822|406|2|506| +2450822|409|2|88| +2450822|410|2|91| +2450822|412|2|737| +2450822|415|2|1000| +2450822|416|2|20| +2450822|418|2|46| +2450822|421|2|792| +2450822|422|2|582| +2450822|424|2|554| +2450822|427|2|446| +2450822|428|2|51| +2450822|430|2|453| +2450822|433|2|397| +2450822|434|2|98| +2450822|436|2|903| +2450822|439|2|789| +2450822|440|2|66| +2450822|442|2|940| +2450822|445|2|402| +2450822|446|2|50| +2450822|448|2|391| +2450822|451|2|98| +2450822|452|2|326| +2450822|454|2|818| +2450822|457|2|505| +2450822|458|2|703| +2450822|460|2|999| +2450822|463|2|959| +2450822|464|2|951| +2450822|466|2|839| +2450822|469|2|796| +2450822|470|2|988| +2450822|472|2|854| +2450822|475|2|308| +2450822|476|2|720| +2450822|478|2|897| +2450822|481|2|869| +2450822|482|2|331| +2450822|484|2|949| +2450822|487|2|995| +2450822|488|2|525| +2450822|490|2|909| +2450822|493|2|323| +2450822|494|2|608| +2450822|496|2|71| +2450822|499|2|841| +2450822|500|2|631| +2450822|502|2|136| +2450822|505|2|88| +2450822|506|2|731| +2450822|508|2|331| +2450822|511|2|817| +2450822|512|2|48| +2450822|514|2|746| +2450822|517|2|340| +2450822|518|2|| +2450822|520|2|307| +2450822|523|2|676| +2450822|524|2|990| +2450822|526|2|242| +2450822|529|2|612| +2450822|530|2|351| +2450822|532|2|97| +2450822|535|2|355| +2450822|536|2|331| +2450822|538|2|965| +2450822|541|2|18| +2450822|542|2|877| +2450822|544|2|610| +2450822|547|2|819| +2450822|548|2|125| +2450822|550|2|855| +2450822|553|2|878| +2450822|554|2|795| +2450822|556|2|609| +2450822|559|2|981| +2450822|560|2|296| +2450822|562|2|10| +2450822|565|2|3| +2450822|566|2|| +2450822|568|2|26| +2450822|571|2|527| +2450822|572|2|878| +2450822|574|2|863| +2450822|577|2|184| +2450822|578|2|635| +2450822|580|2|621| +2450822|583|2|955| +2450822|584|2|209| +2450822|586|2|405| +2450822|589|2|394| +2450822|590|2|135| +2450822|592|2|931| +2450822|595|2|300| +2450822|596|2|459| +2450822|598|2|230| +2450822|601|2|| +2450822|602|2|89| +2450822|604|2|432| +2450822|607|2|668| +2450822|608|2|549| +2450822|610|2|185| +2450822|613|2|875| +2450822|614|2|731| +2450822|616|2|249| +2450822|619|2|721| +2450822|620|2|281| +2450822|622|2|| +2450822|625|2|565| +2450822|626|2|| +2450822|628|2|579| +2450822|631|2|171| +2450822|632|2|922| +2450822|634|2|270| +2450822|637|2|16| +2450822|638|2|276| +2450822|640|2|508| +2450822|643|2|548| +2450822|644|2|3| +2450822|646|2|542| +2450822|649|2|988| +2450822|650|2|668| +2450822|652|2|789| +2450822|655|2|912| +2450822|656|2|| +2450822|658|2|413| +2450822|661|2|937| +2450822|662|2|610| +2450822|664|2|| +2450822|667|2|424| +2450822|668|2|144| +2450822|670|2|973| +2450822|673|2|594| +2450822|674|2|608| +2450822|676|2|651| +2450822|679|2|945| +2450822|680|2|1000| +2450822|682|2|300| +2450822|685|2|660| +2450822|686|2|459| +2450822|688|2|971| +2450822|691|2|230| +2450822|692|2|131| +2450822|694|2|225| +2450822|697|2|127| +2450822|698|2|896| +2450822|700|2|95| +2450822|703|2|35| +2450822|704|2|884| +2450822|706|2|734| +2450822|709|2|883| +2450822|710|2|450| +2450822|712|2|244| +2450822|715|2|560| +2450822|716|2|590| +2450822|718|2|262| +2450822|721|2|624| +2450822|722|2|1| +2450822|724|2|140| +2450822|727|2|836| +2450822|728|2|958| +2450822|730|2|340| +2450822|733|2|440| +2450822|734|2|993| +2450822|736|2|621| +2450822|739|2|32| +2450822|740|2|915| +2450822|742|2|774| +2450822|745|2|370| +2450822|746|2|692| +2450822|748|2|420| +2450822|751|2|567| +2450822|752|2|24| +2450822|754|2|119| +2450822|757|2|184| +2450822|758|2|386| +2450822|760|2|933| +2450822|763|2|| +2450822|764|2|498| +2450822|766|2|610| +2450822|769|2|121| +2450822|770|2|82| +2450822|772|2|237| +2450822|775|2|245| +2450822|776|2|416| +2450822|778|2|872| +2450822|781|2|857| +2450822|782|2|850| +2450822|784|2|354| +2450822|787|2|582| +2450822|788|2|139| +2450822|790|2|823| +2450822|793|2|677| +2450822|794|2|| +2450822|796|2|160| +2450822|799|2|299| +2450822|800|2|26| +2450822|802|2|31| +2450822|805|2|32| +2450822|806|2|606| +2450822|808|2|27| +2450822|811|2|533| +2450822|812|2|953| +2450822|814|2|157| +2450822|817|2|420| +2450822|818|2|642| +2450822|820|2|482| +2450822|823|2|704| +2450822|824|2|181| +2450822|826|2|429| +2450822|829|2|| +2450822|830|2|3| +2450822|832|2|579| +2450822|835|2|648| +2450822|836|2|957| +2450822|838|2|912| +2450822|841|2|860| +2450822|842|2|543| +2450822|844|2|508| +2450822|847|2|500| +2450822|848|2|714| +2450822|850|2|150| +2450822|853|2|870| +2450822|854|2|60| +2450822|856|2|399| +2450822|859|2|967| +2450822|860|2|281| +2450822|862|2|903| +2450822|865|2|944| +2450822|866|2|656| +2450822|868|2|843| +2450822|871|2|570| +2450822|872|2|306| +2450822|874|2|663| +2450822|877|2|580| +2450822|878|2|481| +2450822|880|2|29| +2450822|883|2|483| +2450822|884|2|612| +2450822|886|2|408| +2450822|889|2|208| +2450822|890|2|559| +2450822|892|2|821| +2450822|895|2|920| +2450822|896|2|801| +2450822|898|2|894| +2450822|901|2|863| +2450822|902|2|519| +2450822|904|2|952| +2450822|907|2|17| +2450822|908|2|483| +2450822|910|2|240| +2450822|913|2|612| +2450822|914|2|253| +2450822|916|2|691| +2450822|919|2|7| +2450822|920|2|920| +2450822|922|2|| +2450822|925|2|285| +2450822|926|2|798| +2450822|928|2|210| +2450822|931|2|943| +2450822|932|2|805| +2450822|934|2|498| +2450822|937|2|| +2450822|938|2|677| +2450822|940|2|469| +2450822|943|2|228| +2450822|944|2|405| +2450822|946|2|987| +2450822|949|2|332| +2450822|950|2|508| +2450822|952|2|675| +2450822|955|2|613| +2450822|956|2|450| +2450822|958|2|7| +2450822|961|2|944| +2450822|962|2|420| +2450822|964|2|959| +2450822|967|2|827| +2450822|968|2|207| +2450822|970|2|485| +2450822|973|2|596| +2450822|974|2|342| +2450822|976|2|449| +2450822|979|2|839| +2450822|980|2|472| +2450822|982|2|90| +2450822|985|2|412| +2450822|986|2|431| +2450822|988|2|528| +2450822|991|2|536| +2450822|992|2|464| +2450822|994|2|952| +2450822|997|2|742| +2450822|998|2|444| +2450822|1000|2|903| +2450822|1003|2|656| +2450822|1004|2|906| +2450822|1006|2|897| +2450822|1009|2|| +2450822|1010|2|117| +2450822|1012|2|440| +2450822|1015|2|585| +2450822|1016|2|| +2450822|1018|2|111| +2450822|1021|2|224| +2450822|1022|2|593| +2450822|1024|2|959| +2450822|1027|2|71| +2450822|1028|2|595| +2450822|1030|2|834| +2450822|1033|2|324| +2450822|1034|2|752| +2450822|1036|2|107| +2450822|1039|2|897| +2450822|1040|2|482| +2450822|1042|2|795| +2450822|1045|2|435| +2450822|1046|2|696| +2450822|1048|2|180| +2450822|1051|2|671| +2450822|1052|2|385| +2450822|1054|2|574| +2450822|1057|2|713| +2450822|1058|2|714| +2450822|1060|2|629| +2450822|1063|2|297| +2450822|1064|2|0| +2450822|1066|2|345| +2450822|1069|2|960| +2450822|1070|2|965| +2450822|1072|2|683| +2450822|1075|2|864| +2450822|1076|2|974| +2450822|1078|2|22| +2450822|1081|2|581| +2450822|1082|2|582| +2450822|1084|2|210| +2450822|1087|2|60| +2450822|1088|2|943| +2450822|1090|2|722| +2450822|1093|2|655| +2450822|1094|2|587| +2450822|1096|2|266| +2450822|1099|2|677| +2450822|1100|2|565| +2450822|1102|2|15| +2450822|1105|2|79| +2450822|1106|2|982| +2450822|1108|2|331| +2450822|1111|2|933| +2450822|1112|2|661| +2450822|1114|2|908| +2450822|1117|2|388| +2450822|1118|2|383| +2450822|1120|2|477| +2450822|1123|2|420| +2450822|1124|2|268| +2450822|1126|2|264| +2450822|1129|2|149| +2450822|1130|2|201| +2450822|1132|2|639| +2450822|1135|2|| +2450822|1136|2|988| +2450822|1138|2|794| +2450822|1141|2|412| +2450822|1142|2|439| +2450822|1144|2|269| +2450822|1147|2|47| +2450822|1148|2|406| +2450822|1150|2|387| +2450822|1153|2|563| +2450822|1154|2|247| +2450822|1156|2|383| +2450822|1159|2|481| +2450822|1160|2|465| +2450822|1162|2|284| +2450822|1165|2|274| +2450822|1166|2|223| +2450822|1168|2|663| +2450822|1171|2|251| +2450822|1172|2|135| +2450822|1174|2|552| +2450822|1177|2|879| +2450822|1178|2|437| +2450822|1180|2|314| +2450822|1183|2|251| +2450822|1184|2|267| +2450822|1186|2|| +2450822|1189|2|407| +2450822|1190|2|586| +2450822|1192|2|895| +2450822|1195|2|656| +2450822|1196|2|543| +2450822|1198|2|368| +2450822|1201|2|570| +2450822|1202|2|917| +2450822|1204|2|364| +2450822|1207|2|208| +2450822|1208|2|371| +2450822|1210|2|377| +2450822|1213|2|875| +2450822|1214|2|| +2450822|1216|2|588| +2450822|1219|2|817| +2450822|1220|2|793| +2450822|1222|2|799| +2450822|1225|2|635| +2450822|1226|2|500| +2450822|1228|2|36| +2450822|1231|2|160| +2450822|1232|2|628| +2450822|1234|2|118| +2450822|1237|2|824| +2450822|1238|2|475| +2450822|1240|2|979| +2450822|1243|2|479| +2450822|1244|2|499| +2450822|1246|2|73| +2450822|1249|2|696| +2450822|1250|2|118| +2450822|1252|2|356| +2450822|1255|2|356| +2450822|1256|2|543| +2450822|1258|2|697| +2450822|1261|2|708| +2450822|1262|2|3| +2450822|1264|2|463| +2450822|1267|2|832| +2450822|1268|2|383| +2450822|1270|2|971| +2450822|1273|2|237| +2450822|1274|2|813| +2450822|1276|2|417| +2450822|1279|2|3| +2450822|1280|2|404| +2450822|1282|2|221| +2450822|1285|2|632| +2450822|1286|2|553| +2450822|1288|2|45| +2450822|1291|2|154| +2450822|1292|2|61| +2450822|1294|2|267| +2450822|1297|2|817| +2450822|1298|2|932| +2450822|1300|2|545| +2450822|1303|2|24| +2450822|1304|2|387| +2450822|1306|2|801| +2450822|1309|2|484| +2450822|1310|2|414| +2450822|1312|2|41| +2450822|1315|2|127| +2450822|1316|2|797| +2450822|1318|2|948| +2450822|1321|2|507| +2450822|1322|2|956| +2450822|1324|2|737| +2450822|1327|2|114| +2450822|1328|2|46| +2450822|1330|2|929| +2450822|1333|2|244| +2450822|1334|2|117| +2450822|1336|2|636| +2450822|1339|2|652| +2450822|1340|2|971| +2450822|1342|2|| +2450822|1345|2|134| +2450822|1346|2|460| +2450822|1348|2|600| +2450822|1351|2|742| +2450822|1352|2|478| +2450822|1354|2|531| +2450822|1357|2|560| +2450822|1358|2|379| +2450822|1360|2|934| +2450822|1363|2|999| +2450822|1364|2|118| +2450822|1366|2|262| +2450822|1369|2|794| +2450822|1370|2|85| +2450822|1372|2|497| +2450822|1375|2|225| +2450822|1376|2|137| +2450822|1378|2|229| +2450822|1381|2|674| +2450822|1382|2|10| +2450822|1384|2|339| +2450822|1387|2|71| +2450822|1388|2|372| +2450822|1390|2|171| +2450822|1393|2|165| +2450822|1394|2|950| +2450822|1396|2|804| +2450822|1399|2|637| +2450822|1400|2|369| +2450822|1402|2|237| +2450822|1405|2|387| +2450822|1406|2|533| +2450822|1408|2|329| +2450822|1411|2|7| +2450822|1412|2|624| +2450822|1414|2|620| +2450822|1417|2|275| +2450822|1418|2|733| +2450822|1420|2|262| +2450822|1423|2|512| +2450822|1424|2|492| +2450822|1426|2|989| +2450822|1429|2|866| +2450822|1430|2|184| +2450822|1432|2|227| +2450822|1435|2|211| +2450822|1436|2|962| +2450822|1438|2|333| +2450822|1441|2|893| +2450822|1442|2|248| +2450822|1444|2|452| +2450822|1447|2|958| +2450822|1448|2|348| +2450822|1450|2|864| +2450822|1453|2|196| +2450822|1454|2|520| +2450822|1456|2|549| +2450822|1459|2|878| +2450822|1460|2|123| +2450822|1462|2|961| +2450822|1465|2|7| +2450822|1466|2|443| +2450822|1468|2|376| +2450822|1471|2|| +2450822|1472|2|720| +2450822|1474|2|272| +2450822|1477|2|307| +2450822|1478|2|143| +2450822|1480|2|469| +2450822|1483|2|727| +2450822|1484|2|490| +2450822|1486|2|289| +2450822|1489|2|551| +2450822|1490|2|276| +2450822|1492|2|279| +2450822|1495|2|| +2450822|1496|2|500| +2450822|1498|2|755| +2450822|1501|2|615| +2450822|1502|2|373| +2450822|1504|2|752| +2450822|1507|2|131| +2450822|1508|2|202| +2450822|1510|2|140| +2450822|1513|2|614| +2450822|1514|2|412| +2450822|1516|2|527| +2450822|1519|2|818| +2450822|1520|2|609| +2450822|1522|2|458| +2450822|1525|2|847| +2450822|1526|2|75| +2450822|1528|2|650| +2450822|1531|2|854| +2450822|1532|2|675| +2450822|1534|2|841| +2450822|1537|2|617| +2450822|1538|2|395| +2450822|1540|2|820| +2450822|1543|2|285| +2450822|1544|2|206| +2450822|1546|2|408| +2450822|1549|2|172| +2450822|1550|2|890| +2450822|1552|2|227| +2450822|1555|2|609| +2450822|1556|2|94| +2450822|1558|2|| +2450822|1561|2|260| +2450822|1562|2|796| +2450822|1564|2|580| +2450822|1567|2|439| +2450822|1568|2|261| +2450822|1570|2|850| +2450822|1573|2|667| +2450822|1574|2|24| +2450822|1576|2|648| +2450822|1579|2|885| +2450822|1580|2|210| +2450822|1582|2|34| +2450822|1585|2|333| +2450822|1586|2|270| +2450822|1588|2|523| +2450822|1591|2|269| +2450822|1592|2|998| +2450822|1594|2|543| +2450822|1597|2|328| +2450822|1598|2|2| +2450822|1600|2|125| +2450822|1603|2|209| +2450822|1604|2|89| +2450822|1606|2|161| +2450822|1609|2|257| +2450822|1610|2|167| +2450822|1612|2|824| +2450822|1615|2|292| +2450822|1616|2|549| +2450822|1618|2|365| +2450822|1621|2|153| +2450822|1622|2|149| +2450822|1624|2|66| +2450822|1627|2|132| +2450822|1628|2|381| +2450822|1630|2|478| +2450822|1633|2|308| +2450822|1634|2|668| +2450822|1636|2|| +2450822|1639|2|197| +2450822|1640|2|778| +2450822|1642|2|300| +2450822|1645|2|391| +2450822|1646|2|69| +2450822|1648|2|| +2450822|1651|2|693| +2450822|1652|2|102| +2450822|1654|2|466| +2450822|1657|2|563| +2450822|1658|2|| +2450822|1660|2|433| +2450822|1663|2|377| +2450822|1664|2|473| +2450822|1666|2|| +2450822|1669|2|602| +2450822|1670|2|780| +2450822|1672|2|160| +2450822|1675|2|358| +2450822|1676|2|116| +2450822|1678|2|| +2450822|1681|2|70| +2450822|1682|2|750| +2450822|1684|2|971| +2450822|1687|2|897| +2450822|1688|2|835| +2450822|1690|2|734| +2450822|1693|2|198| +2450822|1694|2|947| +2450822|1696|2|190| +2450822|1699|2|915| +2450822|1700|2|507| +2450822|1702|2|498| +2450822|1705|2|412| +2450822|1706|2|22| +2450822|1708|2|45| +2450822|1711|2|404| +2450822|1712|2|962| +2450822|1714|2|669| +2450822|1717|2|138| +2450822|1718|2|788| +2450822|1720|2|279| +2450822|1723|2|649| +2450822|1724|2|96| +2450822|1726|2|830| +2450822|1729|2|| +2450822|1730|2|432| +2450822|1732|2|330| +2450822|1735|2|707| +2450822|1736|2|196| +2450822|1738|2|268| +2450822|1741|2|685| +2450822|1742|2|607| +2450822|1744|2|877| +2450822|1747|2|746| +2450822|1748|2|71| +2450822|1750|2|475| +2450822|1753|2|827| +2450822|1754|2|482| +2450822|1756|2|540| +2450822|1759|2|172| +2450822|1760|2|163| +2450822|1762|2|2| +2450822|1765|2|67| +2450822|1766|2|951| +2450822|1768|2|413| +2450822|1771|2|682| +2450822|1772|2|978| +2450822|1774|2|563| +2450822|1777|2|908| +2450822|1778|2|330| +2450822|1780|2|989| +2450822|1783|2|304| +2450822|1784|2|269| +2450822|1786|2|749| +2450822|1789|2|507| +2450822|1790|2|950| +2450822|1792|2|947| +2450822|1795|2|696| +2450822|1796|2|942| +2450822|1798|2|518| +2450822|1801|2|562| +2450822|1802|2|708| +2450822|1804|2|938| +2450822|1807|2|14| +2450822|1808|2|612| +2450822|1810|2|891| +2450822|1813|2|| +2450822|1814|2|367| +2450822|1816|2|267| +2450822|1819|2|98| +2450822|1820|2|400| +2450822|1822|2|980| +2450822|1825|2|479| +2450822|1826|2|769| +2450822|1828|2|800| +2450822|1831|2|64| +2450822|1832|2|527| +2450822|1834|2|876| +2450822|1837|2|939| +2450822|1838|2|229| +2450822|1840|2|614| +2450822|1843|2|566| +2450822|1844|2|818| +2450822|1846|2|911| +2450822|1849|2|44| +2450822|1850|2|962| +2450822|1852|2|484| +2450822|1855|2|926| +2450822|1856|2|907| +2450822|1858|2|662| +2450822|1861|2|526| +2450822|1862|2|210| +2450822|1864|2|315| +2450822|1867|2|962| +2450822|1868|2|34| +2450822|1870|2|687| +2450822|1873|2|261| +2450822|1874|2|895| +2450822|1876|2|9| +2450822|1879|2|292| +2450822|1880|2|698| +2450822|1882|2|713| +2450822|1885|2|730| +2450822|1886|2|313| +2450822|1888|2|| +2450822|1891|2|488| +2450822|1892|2|267| +2450822|1894|2|970| +2450822|1897|2|898| +2450822|1898|2|994| +2450822|1900|2|824| +2450822|1903|2|718| +2450822|1904|2|481| +2450822|1906|2|84| +2450822|1909|2|370| +2450822|1910|2|601| +2450822|1912|2|689| +2450822|1915|2|862| +2450822|1916|2|638| +2450822|1918|2|461| +2450822|1921|2|871| +2450822|1922|2|3| +2450822|1924|2|248| +2450822|1927|2|940| +2450822|1928|2|696| +2450822|1930|2|164| +2450822|1933|2|393| +2450822|1934|2|190| +2450822|1936|2|747| +2450822|1939|2|71| +2450822|1940|2|615| +2450822|1942|2|926| +2450822|1945|2|165| +2450822|1946|2|491| +2450822|1948|2|517| +2450822|1951|2|677| +2450822|1952|2|618| +2450822|1954|2|404| +2450822|1957|2|406| +2450822|1958|2|757| +2450822|1960|2|144| +2450822|1963|2|600| +2450822|1964|2|213| +2450822|1966|2|429| +2450822|1969|2|644| +2450822|1970|2|121| +2450822|1972|2|170| +2450822|1975|2|195| +2450822|1976|2|352| +2450822|1978|2|625| +2450822|1981|2|265| +2450822|1982|2|3| +2450822|1984|2|287| +2450822|1987|2|409| +2450822|1988|2|160| +2450822|1990|2|| +2450822|1993|2|760| +2450822|1994|2|689| +2450822|1996|2|481| +2450822|1999|2|591| +2450822|2000|2|956| +2450822|2002|2|470| +2450822|2005|2|595| +2450822|2006|2|753| +2450822|2008|2|269| +2450822|2011|2|409| +2450822|2012|2|397| +2450822|2014|2|715| +2450822|2017|2|626| +2450822|2018|2|200| +2450822|2020|2|763| +2450822|2023|2|553| +2450822|2024|2|936| +2450822|2026|2|196| +2450822|2029|2|39| +2450822|2030|2|357| +2450822|2032|2|157| +2450822|2035|2|831| +2450822|2036|2|723| +2450822|2038|2|214| +2450822|2041|2|267| +2450822|2042|2|370| +2450822|2044|2|733| +2450822|2047|2|863| +2450822|2048|2|| +2450822|2050|2|642| +2450822|2053|2|356| +2450822|2054|2|504| +2450822|2056|2|997| +2450822|2059|2|412| +2450822|2060|2|| +2450822|2062|2|820| +2450822|2065|2|639| +2450822|2066|2|810| +2450822|2068|2|645| +2450822|2071|2|313| +2450822|2072|2|760| +2450822|2074|2|116| +2450822|2077|2|650| +2450822|2078|2|269| +2450822|2080|2|160| +2450822|2083|2|961| +2450822|2084|2|436| +2450822|2086|2|17| +2450822|2089|2|295| +2450822|2090|2|747| +2450822|2092|2|694| +2450822|2095|2|685| +2450822|2096|2|969| +2450822|2098|2|327| +2450822|2101|2|454| +2450822|2102|2|40| +2450822|2104|2|291| +2450822|2107|2|718| +2450822|2108|2|411| +2450822|2110|2|981| +2450822|2113|2|64| +2450822|2114|2|246| +2450822|2116|2|461| +2450822|2119|2|| +2450822|2120|2|464| +2450822|2122|2|159| +2450822|2125|2|| +2450822|2126|2|321| +2450822|2128|2|997| +2450822|2131|2|858| +2450822|2132|2|687| +2450822|2134|2|923| +2450822|2137|2|368| +2450822|2138|2|460| +2450822|2140|2|472| +2450822|2143|2|612| +2450822|2144|2|656| +2450822|2146|2|724| +2450822|2149|2|433| +2450822|2150|2|47| +2450822|2152|2|703| +2450822|2155|2|545| +2450822|2156|2|192| +2450822|2158|2|818| +2450822|2161|2|779| +2450822|2162|2|409| +2450822|2164|2|567| +2450822|2167|2|725| +2450822|2168|2|320| +2450822|2170|2|165| +2450822|2173|2|388| +2450822|2174|2|781| +2450822|2176|2|360| +2450822|2179|2|409| +2450822|2180|2|121| +2450822|2182|2|| +2450822|2185|2|493| +2450822|2186|2|949| +2450822|2188|2|718| +2450822|2191|2|419| +2450822|2192|2|141| +2450822|2194|2|159| +2450822|2197|2|| +2450822|2198|2|| +2450822|2200|2|755| +2450822|2203|2|266| +2450822|2204|2|483| +2450822|2206|2|184| +2450822|2209|2|288| +2450822|2210|2|561| +2450822|2212|2|555| +2450822|2215|2|168| +2450822|2216|2|639| +2450822|2218|2|149| +2450822|2221|2|81| +2450822|2222|2|707| +2450822|2224|2|898| +2450822|2227|2|543| +2450822|2228|2|582| +2450822|2230|2|134| +2450822|2233|2|72| +2450822|2234|2|954| +2450822|2236|2|945| +2450822|2239|2|976| +2450822|2240|2|123| +2450822|2242|2|841| +2450822|2245|2|983| +2450822|2246|2|591| +2450822|2248|2|471| +2450822|2251|2|951| +2450822|2252|2|443| +2450822|2254|2|581| +2450822|2257|2|160| +2450822|2258|2|90| +2450822|2260|2|10| +2450822|2263|2|875| +2450822|2264|2|463| +2450822|2266|2|171| +2450822|2269|2|| +2450822|2270|2|387| +2450822|2272|2|413| +2450822|2275|2|408| +2450822|2276|2|70| +2450822|2278|2|975| +2450822|2281|2|303| +2450822|2282|2|884| +2450822|2284|2|887| +2450822|2287|2|817| +2450822|2288|2|957| +2450822|2290|2|275| +2450822|2293|2|| +2450822|2294|2|| +2450822|2296|2|65| +2450822|2299|2|261| +2450822|2300|2|| +2450822|2302|2|896| +2450822|2305|2|988| +2450822|2306|2|305| +2450822|2308|2|193| +2450822|2311|2|634| +2450822|2312|2|369| +2450822|2314|2|438| +2450822|2317|2|| +2450822|2318|2|361| +2450822|2320|2|141| +2450822|2323|2|590| +2450822|2324|2|440| +2450822|2326|2|37| +2450822|2329|2|328| +2450822|2330|2|290| +2450822|2332|2|271| +2450822|2335|2|375| +2450822|2336|2|964| +2450822|2338|2|| +2450822|2341|2|381| +2450822|2342|2|216| +2450822|2344|2|284| +2450822|2347|2|466| +2450822|2348|2|751| +2450822|2350|2|541| +2450822|2353|2|549| +2450822|2354|2|650| +2450822|2356|2|713| +2450822|2359|2|959| +2450822|2360|2|0| +2450822|2362|2|994| +2450822|2365|2|195| +2450822|2366|2|460| +2450822|2368|2|74| +2450822|2371|2|713| +2450822|2372|2|163| +2450822|2374|2|770| +2450822|2377|2|315| +2450822|2378|2|370| +2450822|2380|2|| +2450822|2383|2|185| +2450822|2384|2|795| +2450822|2386|2|| +2450822|2389|2|443| +2450822|2390|2|912| +2450822|2392|2|| +2450822|2395|2|708| +2450822|2396|2|391| +2450822|2398|2|671| +2450822|2401|2|571| +2450822|2402|2|417| +2450822|2404|2|| +2450822|2407|2|| +2450822|2408|2|363| +2450822|2410|2|347| +2450822|2413|2|188| +2450822|2414|2|167| +2450822|2416|2|775| +2450822|2419|2|158| +2450822|2420|2|550| +2450822|2422|2|349| +2450822|2425|2|488| +2450822|2426|2|672| +2450822|2428|2|879| +2450822|2431|2|610| +2450822|2432|2|881| +2450822|2434|2|446| +2450822|2437|2|566| +2450822|2438|2|678| +2450822|2440|2|| +2450822|2443|2|935| +2450822|2444|2|598| +2450822|2446|2|385| +2450822|2449|2|361| +2450822|2450|2|| +2450822|2452|2|527| +2450822|2455|2|684| +2450822|2456|2|672| +2450822|2458|2|890| +2450822|2461|2|933| +2450822|2462|2|285| +2450822|2464|2|376| +2450822|2467|2|| +2450822|2468|2|934| +2450822|2470|2|743| +2450822|2473|2|494| +2450822|2474|2|186| +2450822|2476|2|955| +2450822|2479|2|659| +2450822|2480|2|764| +2450822|2482|2|884| +2450822|2485|2|732| +2450822|2486|2|701| +2450822|2488|2|828| +2450822|2491|2|50| +2450822|2492|2|934| +2450822|2494|2|572| +2450822|2497|2|377| +2450822|2498|2|648| +2450822|2500|2|624| +2450822|2503|2|522| +2450822|2504|2|181| +2450822|2506|2|538| +2450822|2509|2|942| +2450822|2510|2|| +2450822|2512|2|50| +2450822|2515|2|452| +2450822|2516|2|49| +2450822|2518|2|452| +2450822|2521|2|733| +2450822|2522|2|740| +2450822|2524|2|794| +2450822|2527|2|850| +2450822|2528|2|9| +2450822|2530|2|202| +2450822|2533|2|502| +2450822|2534|2|615| +2450822|2536|2|241| +2450822|2539|2|95| +2450822|2540|2|488| +2450822|2542|2|348| +2450822|2545|2|845| +2450822|2546|2|458| +2450822|2548|2|517| +2450822|2551|2|27| +2450822|2552|2|559| +2450822|2554|2|870| +2450822|2557|2|| +2450822|2558|2|781| +2450822|2560|2|161| +2450822|2563|2|558| +2450822|2564|2|977| +2450822|2566|2|476| +2450822|2569|2|731| +2450822|2570|2|| +2450822|2572|2|783| +2450822|2575|2|624| +2450822|2576|2|930| +2450822|2578|2|610| +2450822|2581|2|602| +2450822|2582|2|364| +2450822|2584|2|671| +2450822|2587|2|| +2450822|2588|2|425| +2450822|2590|2|83| +2450822|2593|2|56| +2450822|2594|2|447| +2450822|2596|2|837| +2450822|2599|2|792| +2450822|2600|2|549| +2450822|2602|2|320| +2450822|2605|2|599| +2450822|2606|2|119| +2450822|2608|2|208| +2450822|2611|2|627| +2450822|2612|2|56| +2450822|2614|2|959| +2450822|2617|2|| +2450822|2618|2|856| +2450822|2620|2|833| +2450822|2623|2|835| +2450822|2624|2|552| +2450822|2626|2|596| +2450822|2629|2|247| +2450822|2630|2|659| +2450822|2632|2|127| +2450822|2635|2|481| +2450822|2636|2|37| +2450822|2638|2|389| +2450822|2641|2|57| +2450822|2642|2|552| +2450822|2644|2|109| +2450822|2647|2|477| +2450822|2648|2|420| +2450822|2650|2|458| +2450822|2653|2|282| +2450822|2654|2|314| +2450822|2656|2|406| +2450822|2659|2|404| +2450822|2660|2|317| +2450822|2662|2|943| +2450822|2665|2|| +2450822|2666|2|134| +2450822|2668|2|740| +2450822|2671|2|118| +2450822|2672|2|94| +2450822|2674|2|522| +2450822|2677|2|50| +2450822|2678|2|313| +2450822|2680|2|| +2450822|2683|2|540| +2450822|2684|2|532| +2450822|2686|2|108| +2450822|2689|2|44| +2450822|2690|2|732| +2450822|2692|2|839| +2450822|2695|2|755| +2450822|2696|2|149| +2450822|2698|2|940| +2450822|2701|2|702| +2450822|2702|2|393| +2450822|2704|2|342| +2450822|2707|2|243| +2450822|2708|2|986| +2450822|2710|2|147| +2450822|2713|2|| +2450822|2714|2|359| +2450822|2716|2|800| +2450822|2719|2|350| +2450822|2720|2|237| +2450822|2722|2|655| +2450822|2725|2|495| +2450822|2726|2|736| +2450822|2728|2|163| +2450822|2731|2|79| +2450822|2732|2|768| +2450822|2734|2|922| +2450822|2737|2|466| +2450822|2738|2|| +2450822|2740|2|153| +2450822|2743|2|92| +2450822|2744|2|679| +2450822|2746|2|898| +2450822|2749|2|309| +2450822|2750|2|290| +2450822|2752|2|436| +2450822|2755|2|359| +2450822|2756|2|890| +2450822|2758|2|634| +2450822|2761|2|| +2450822|2762|2|820| +2450822|2764|2|702| +2450822|2767|2|743| +2450822|2768|2|310| +2450822|2770|2|668| +2450822|2773|2|491| +2450822|2774|2|763| +2450822|2776|2|483| +2450822|2779|2|558| +2450822|2780|2|864| +2450822|2782|2|155| +2450822|2785|2|959| +2450822|2786|2|| +2450822|2788|2|856| +2450822|2791|2|402| +2450822|2792|2|830| +2450822|2794|2|944| +2450822|2797|2|506| +2450822|2798|2|381| +2450822|2800|2|609| +2450822|2803|2|543| +2450822|2804|2|30| +2450822|2806|2|880| +2450822|2809|2|342| +2450822|2810|2|176| +2450822|2812|2|219| +2450822|2815|2|643| +2450822|2816|2|248| +2450822|2818|2|117| +2450822|2821|2|316| +2450822|2822|2|326| +2450822|2824|2|760| +2450822|2827|2|75| +2450822|2828|2|947| +2450822|2830|2|291| +2450822|2833|2|317| +2450822|2834|2|399| +2450822|2836|2|784| +2450822|2839|2|330| +2450822|2840|2|62| +2450822|2842|2|56| +2450822|2845|2|285| +2450822|2846|2|610| +2450822|2848|2|131| +2450822|2851|2|144| +2450822|2852|2|690| +2450822|2854|2|891| +2450822|2857|2|961| +2450822|2858|2|991| +2450822|2860|2|366| +2450822|2863|2|639| +2450822|2864|2|714| +2450822|2866|2|671| +2450822|2869|2|321| +2450822|2870|2|| +2450822|2872|2|347| +2450822|2875|2|554| +2450822|2876|2|15| +2450822|2878|2|609| +2450822|2881|2|262| +2450822|2882|2|961| +2450822|2884|2|| +2450822|2887|2|43| +2450822|2888|2|1000| +2450822|2890|2|930| +2450822|2893|2|308| +2450822|2894|2|285| +2450822|2896|2|745| +2450822|2899|2|922| +2450822|2900|2|484| +2450822|2902|2|114| +2450822|2905|2|326| +2450822|2906|2|996| +2450822|2908|2|689| +2450822|2911|2|954| +2450822|2912|2|958| +2450822|2914|2|863| +2450822|2917|2|971| +2450822|2918|2|662| +2450822|2920|2|65| +2450822|2923|2|465| +2450822|2924|2|| +2450822|2926|2|907| +2450822|2929|2|778| +2450822|2930|2|989| +2450822|2932|2|124| +2450822|2935|2|358| +2450822|2936|2|872| +2450822|2938|2|928| +2450822|2941|2|41| +2450822|2942|2|496| +2450822|2944|2|895| +2450822|2947|2|611| +2450822|2948|2|220| +2450822|2950|2|545| +2450822|2953|2|| +2450822|2954|2|997| +2450822|2956|2|| +2450822|2959|2|342| +2450822|2960|2|882| +2450822|2962|2|846| +2450822|2965|2|105| +2450822|2966|2|418| +2450822|2968|2|784| +2450822|2971|2|600| +2450822|2972|2|323| +2450822|2974|2|378| +2450822|2977|2|150| +2450822|2978|2|907| +2450822|2980|2|714| +2450822|2983|2|887| +2450822|2984|2|| +2450822|2986|2|564| +2450822|2989|2|255| +2450822|2990|2|484| +2450822|2992|2|777| +2450822|2995|2|429| +2450822|2996|2|744| +2450822|2998|2|248| +2450822|3001|2|| +2450822|3002|2|695| +2450822|3004|2|758| +2450822|3007|2|136| +2450822|3008|2|551| +2450822|3010|2|631| +2450822|3013|2|402| +2450822|3014|2|381| +2450822|3016|2|575| +2450822|3019|2|56| +2450822|3020|2|| +2450822|3022|2|620| +2450822|3025|2|736| +2450822|3026|2|720| +2450822|3028|2|402| +2450822|3031|2|67| +2450822|3032|2|50| +2450822|3034|2|15| +2450822|3037|2|525| +2450822|3038|2|700| +2450822|3040|2|8| +2450822|3043|2|816| +2450822|3044|2|869| +2450822|3046|2|766| +2450822|3049|2|960| +2450822|3050|2|551| +2450822|3052|2|583| +2450822|3055|2|390| +2450822|3056|2|239| +2450822|3058|2|857| +2450822|3061|2|41| +2450822|3062|2|391| +2450822|3064|2|356| +2450822|3067|2|580| +2450822|3068|2|580| +2450822|3070|2|655| +2450822|3073|2|339| +2450822|3074|2|692| +2450822|3076|2|829| +2450822|3079|2|22| +2450822|3080|2|586| +2450822|3082|2|728| +2450822|3085|2|972| +2450822|3086|2|412| +2450822|3088|2|25| +2450822|3091|2|165| +2450822|3092|2|633| +2450822|3094|2|844| +2450822|3097|2|595| +2450822|3098|2|951| +2450822|3100|2|265| +2450822|3103|2|610| +2450822|3104|2|891| +2450822|3106|2|65| +2450822|3109|2|452| +2450822|3110|2|104| +2450822|3112|2|270| +2450822|3115|2|39| +2450822|3116|2|630| +2450822|3118|2|703| +2450822|3121|2|385| +2450822|3122|2|451| +2450822|3124|2|| +2450822|3127|2|54| +2450822|3128|2|415| +2450822|3130|2|573| +2450822|3133|2|867| +2450822|3134|2|777| +2450822|3136|2|333| +2450822|3139|2|| +2450822|3140|2|489| +2450822|3142|2|428| +2450822|3145|2|850| +2450822|3146|2|910| +2450822|3148|2|996| +2450822|3151|2|266| +2450822|3152|2|190| +2450822|3154|2|886| +2450822|3157|2|731| +2450822|3158|2|273| +2450822|3160|2|746| +2450822|3163|2|5| +2450822|3164|2|313| +2450822|3166|2|820| +2450822|3169|2|874| +2450822|3170|2|297| +2450822|3172|2|563| +2450822|3175|2|280| +2450822|3176|2|285| +2450822|3178|2|131| +2450822|3181|2|100| +2450822|3182|2|494| +2450822|3184|2|115| +2450822|3187|2|962| +2450822|3188|2|796| +2450822|3190|2|348| +2450822|3193|2|142| +2450822|3194|2|480| +2450822|3196|2|527| +2450822|3199|2|778| +2450822|3200|2|109| +2450822|3202|2|210| +2450822|3205|2|740| +2450822|3206|2|128| +2450822|3208|2|| +2450822|3211|2|420| +2450822|3212|2|852| +2450822|3214|2|51| +2450822|3217|2|557| +2450822|3218|2|35| +2450822|3220|2|521| +2450822|3223|2|203| +2450822|3224|2|874| +2450822|3226|2|| +2450822|3229|2|516| +2450822|3230|2|406| +2450822|3232|2|83| +2450822|3235|2|1| +2450822|3236|2|367| +2450822|3238|2|27| +2450822|3241|2|861| +2450822|3242|2|328| +2450822|3244|2|959| +2450822|3247|2|269| +2450822|3248|2|236| +2450822|3250|2|695| +2450822|3253|2|140| +2450822|3254|2|751| +2450822|3256|2|738| +2450822|3259|2|15| +2450822|3260|2|416| +2450822|3262|2|984| +2450822|3265|2|| +2450822|3266|2|74| +2450822|3268|2|600| +2450822|3271|2|102| +2450822|3272|2|124| +2450822|3274|2|346| +2450822|3277|2|566| +2450822|3278|2|417| +2450822|3280|2|401| +2450822|3283|2|221| +2450822|3284|2|180| +2450822|3286|2|326| +2450822|3289|2|178| +2450822|3290|2|487| +2450822|3292|2|458| +2450822|3295|2|783| +2450822|3296|2|225| +2450822|3298|2|882| +2450822|3301|2|298| +2450822|3302|2|925| +2450822|3304|2|152| +2450822|3307|2|682| +2450822|3308|2|457| +2450822|3310|2|359| +2450822|3313|2|634| +2450822|3314|2|306| +2450822|3316|2|561| +2450822|3319|2|239| +2450822|3320|2|877| +2450822|3322|2|| +2450822|3325|2|762| +2450822|3326|2|959| +2450822|3328|2|624| +2450822|3331|2|505| +2450822|3332|2|261| +2450822|3334|2|999| +2450822|3337|2|122| +2450822|3338|2|437| +2450822|3340|2|543| +2450822|3343|2|432| +2450822|3344|2|915| +2450822|3346|2|681| +2450822|3349|2|1000| +2450822|3350|2|923| +2450822|3352|2|604| +2450822|3355|2|| +2450822|3356|2|275| +2450822|3358|2|362| +2450822|3361|2|553| +2450822|3362|2|562| +2450822|3364|2|767| +2450822|3367|2|741| +2450822|3368|2|996| +2450822|3370|2|638| +2450822|3373|2|855| +2450822|3374|2|445| +2450822|3376|2|632| +2450822|3379|2|460| +2450822|3380|2|147| +2450822|3382|2|637| +2450822|3385|2|64| +2450822|3386|2|7| +2450822|3388|2|881| +2450822|3391|2|539| +2450822|3392|2|450| +2450822|3394|2|514| +2450822|3397|2|226| +2450822|3398|2|584| +2450822|3400|2|323| +2450822|3403|2|318| +2450822|3404|2|207| +2450822|3406|2|773| +2450822|3409|2|125| +2450822|3410|2|984| +2450822|3412|2|136| +2450822|3415|2|721| +2450822|3416|2|303| +2450822|3418|2|781| +2450822|3421|2|290| +2450822|3422|2|474| +2450822|3424|2|682| +2450822|3427|2|553| +2450822|3428|2|409| +2450822|3430|2|615| +2450822|3433|2|508| +2450822|3434|2|345| +2450822|3436|2|115| +2450822|3439|2|268| +2450822|3440|2|11| +2450822|3442|2|74| +2450822|3445|2|570| +2450822|3446|2|599| +2450822|3448|2|991| +2450822|3451|2|417| +2450822|3452|2|644| +2450822|3454|2|103| +2450822|3457|2|33| +2450822|3458|2|830| +2450822|3460|2|888| +2450822|3463|2|958| +2450822|3464|2|1| +2450822|3466|2|144| +2450822|3469|2|680| +2450822|3470|2|| +2450822|3472|2|161| +2450822|3475|2|333| +2450822|3476|2|856| +2450822|3478|2|249| +2450822|3481|2|884| +2450822|3482|2|354| +2450822|3484|2|596| +2450822|3487|2|153| +2450822|3488|2|154| +2450822|3490|2|227| +2450822|3493|2|| +2450822|3494|2|| +2450822|3496|2|765| +2450822|3499|2|287| +2450822|3500|2|| +2450822|3502|2|222| +2450822|3505|2|911| +2450822|3506|2|875| +2450822|3508|2|759| +2450822|3511|2|231| +2450822|3512|2|308| +2450822|3514|2|| +2450822|3517|2|748| +2450822|3518|2|625| +2450822|3520|2|787| +2450822|3523|2|227| +2450822|3524|2|748| +2450822|3526|2|74| +2450822|3529|2|774| +2450822|3530|2|180| +2450822|3532|2|671| +2450822|3535|2|997| +2450822|3536|2|917| +2450822|3538|2|607| +2450822|3541|2|302| +2450822|3542|2|561| +2450822|3544|2|525| +2450822|3547|2|| +2450822|3548|2|960| +2450822|3550|2|865| +2450822|3553|2|970| +2450822|3554|2|22| +2450822|3556|2|194| +2450822|3559|2|654| +2450822|3560|2|625| +2450822|3562|2|864| +2450822|3565|2|814| +2450822|3566|2|541| +2450822|3568|2|936| +2450822|3571|2|270| +2450822|3572|2|235| +2450822|3574|2|931| +2450822|3577|2|311| +2450822|3578|2|487| +2450822|3580|2|215| +2450822|3583|2|558| +2450822|3584|2|455| +2450822|3586|2|759| +2450822|3589|2|| +2450822|3590|2|29| +2450822|3592|2|601| +2450822|3595|2|420| +2450822|3596|2|629| +2450822|3598|2|960| +2450822|3601|2|421| +2450822|3602|2|194| +2450822|3604|2|993| +2450822|3607|2|408| +2450822|3608|2|| +2450822|3610|2|422| +2450822|3613|2|926| +2450822|3614|2|948| +2450822|3616|2|470| +2450822|3619|2|649| +2450822|3620|2|948| +2450822|3622|2|322| +2450822|3625|2|206| +2450822|3626|2|283| +2450822|3628|2|883| +2450822|3631|2|682| +2450822|3632|2|137| +2450822|3634|2|250| +2450822|3637|2|782| +2450822|3638|2|422| +2450822|3640|2|63| +2450822|3643|2|747| +2450822|3644|2|146| +2450822|3646|2|859| +2450822|3649|2|272| +2450822|3650|2|941| +2450822|3652|2|383| +2450822|3655|2|358| +2450822|3656|2|673| +2450822|3658|2|392| +2450822|3661|2|574| +2450822|3662|2|627| +2450822|3664|2|964| +2450822|3667|2|138| +2450822|3668|2|62| +2450822|3670|2|332| +2450822|3673|2|329| +2450822|3674|2|720| +2450822|3676|2|731| +2450822|3679|2|414| +2450822|3680|2|359| +2450822|3682|2|126| +2450822|3685|2|378| +2450822|3686|2|905| +2450822|3688|2|394| +2450822|3691|2|804| +2450822|3692|2|968| +2450822|3694|2|260| +2450822|3697|2|102| +2450822|3698|2|851| +2450822|3700|2|220| +2450822|3703|2|132| +2450822|3704|2|203| +2450822|3706|2|508| +2450822|3709|2|| +2450822|3710|2|500| +2450822|3712|2|214| +2450822|3715|2|744| +2450822|3716|2|794| +2450822|3718|2|31| +2450822|3721|2|429| +2450822|3722|2|552| +2450822|3724|2|401| +2450822|3727|2|80| +2450822|3728|2|463| +2450822|3730|2|991| +2450822|3733|2|389| +2450822|3734|2|505| +2450822|3736|2|90| +2450822|3739|2|363| +2450822|3740|2|560| +2450822|3742|2|223| +2450822|3745|2|729| +2450822|3746|2|574| +2450822|3748|2|951| +2450822|3751|2|| +2450822|3752|2|795| +2450822|3754|2|28| +2450822|3757|2|362| +2450822|3758|2|585| +2450822|3760|2|867| +2450822|3763|2|295| +2450822|3764|2|675| +2450822|3766|2|362| +2450822|3769|2|863| +2450822|3770|2|882| +2450822|3772|2|788| +2450822|3775|2|475| +2450822|3776|2|951| +2450822|3778|2|238| +2450822|3781|2|882| +2450822|3782|2|997| +2450822|3784|2|119| +2450822|3787|2|165| +2450822|3788|2|14| +2450822|3790|2|530| +2450822|3793|2|853| +2450822|3794|2|852| +2450822|3796|2|297| +2450822|3799|2|164| +2450822|3800|2|990| +2450822|3802|2|260| +2450822|3805|2|609| +2450822|3806|2|716| +2450822|3808|2|979| +2450822|3811|2|949| +2450822|3812|2|| +2450822|3814|2|571| +2450822|3817|2|463| +2450822|3818|2|497| +2450822|3820|2|533| +2450822|3823|2|46| +2450822|3824|2|948| +2450822|3826|2|745| +2450822|3829|2|614| +2450822|3830|2|651| +2450822|3832|2|795| +2450822|3835|2|1| +2450822|3836|2|612| +2450822|3838|2|334| +2450822|3841|2|279| +2450822|3842|2|526| +2450822|3844|2|879| +2450822|3847|2|61| +2450822|3848|2|310| +2450822|3850|2|842| +2450822|3853|2|959| +2450822|3854|2|947| +2450822|3856|2|796| +2450822|3859|2|11| +2450822|3860|2|133| +2450822|3862|2|| +2450822|3865|2|109| +2450822|3866|2|261| +2450822|3868|2|101| +2450822|3871|2|53| +2450822|3872|2|826| +2450822|3874|2|293| +2450822|3877|2|637| +2450822|3878|2|375| +2450822|3880|2|664| +2450822|3883|2|933| +2450822|3884|2|409| +2450822|3886|2|15| +2450822|3889|2|637| +2450822|3890|2|116| +2450822|3892|2|98| +2450822|3895|2|771| +2450822|3896|2|965| +2450822|3898|2|485| +2450822|3901|2|831| +2450822|3902|2|579| +2450822|3904|2|346| +2450822|3907|2|518| +2450822|3908|2|94| +2450822|3910|2|651| +2450822|3913|2|789| +2450822|3914|2|601| +2450822|3916|2|220| +2450822|3919|2|| +2450822|3920|2|296| +2450822|3922|2|491| +2450822|3925|2|702| +2450822|3926|2|| +2450822|3928|2|834| +2450822|3931|2|212| +2450822|3932|2|224| +2450822|3934|2|301| +2450822|3937|2|235| +2450822|3938|2|876| +2450822|3940|2|233| +2450822|3943|2|473| +2450822|3944|2|524| +2450822|3946|2|402| +2450822|3949|2|694| +2450822|3950|2|761| +2450822|3952|2|287| +2450822|3955|2|272| +2450822|3956|2|446| +2450822|3958|2|563| +2450822|3961|2|705| +2450822|3962|2|107| +2450822|3964|2|696| +2450822|3967|2|502| +2450822|3968|2|398| +2450822|3970|2|165| +2450822|3973|2|353| +2450822|3974|2|856| +2450822|3976|2|531| +2450822|3979|2|575| +2450822|3980|2|10| +2450822|3982|2|273| +2450822|3985|2|97| +2450822|3986|2|487| +2450822|3988|2|530| +2450822|3991|2|858| +2450822|3992|2|328| +2450822|3994|2|397| +2450822|3997|2|39| +2450822|3998|2|916| +2450822|4000|2|963| +2450822|4003|2|860| +2450822|4004|2|872| +2450822|4006|2|237| +2450822|4009|2|320| +2450822|4010|2|213| +2450822|4012|2|828| +2450822|4015|2|675| +2450822|4016|2|567| +2450822|4018|2|499| +2450822|4021|2|434| +2450822|4022|2|304| +2450822|4024|2|57| +2450822|4027|2|488| +2450822|4028|2|708| +2450822|4030|2|65| +2450822|4033|2|141| +2450822|4034|2|867| +2450822|4036|2|252| +2450822|4039|2|493| +2450822|4040|2|910| +2450822|4042|2|138| +2450822|4045|2|772| +2450822|4046|2|559| +2450822|4048|2|552| +2450822|4051|2|710| +2450822|4052|2|234| +2450822|4054|2|55| +2450822|4057|2|340| +2450822|4058|2|202| +2450822|4060|2|19| +2450822|4063|2|829| +2450822|4064|2|143| +2450822|4066|2|921| +2450822|4069|2|74| +2450822|4070|2|442| +2450822|4072|2|250| +2450822|4075|2|600| +2450822|4076|2|349| +2450822|4078|2|184| +2450822|4081|2|676| +2450822|4082|2|738| +2450822|4084|2|492| +2450822|4087|2|907| +2450822|4088|2|607| +2450822|4090|2|458| +2450822|4093|2|565| +2450822|4094|2|211| +2450822|4096|2|123| +2450822|4099|2|19| +2450822|4100|2|603| +2450822|4102|2|603| +2450822|4105|2|113| +2450822|4106|2|25| +2450822|4108|2|414| +2450822|4111|2|779| +2450822|4112|2|538| +2450822|4114|2|16| +2450822|4117|2|20| +2450822|4118|2|606| +2450822|4120|2|238| +2450822|4123|2|588| +2450822|4124|2|| +2450822|4126|2|242| +2450822|4129|2|550| +2450822|4130|2|535| +2450822|4132|2|681| +2450822|4135|2|443| +2450822|4136|2|891| +2450822|4138|2|634| +2450822|4141|2|864| +2450822|4142|2|662| +2450822|4144|2|247| +2450822|4147|2|| +2450822|4148|2|248| +2450822|4150|2|471| +2450822|4153|2|322| +2450822|4154|2|535| +2450822|4156|2|733| +2450822|4159|2|291| +2450822|4160|2|365| +2450822|4162|2|79| +2450822|4165|2|443| +2450822|4166|2|489| +2450822|4168|2|981| +2450822|4171|2|381| +2450822|4172|2|803| +2450822|4174|2|982| +2450822|4177|2|209| +2450822|4178|2|899| +2450822|4180|2|428| +2450822|4183|2|453| +2450822|4184|2|104| +2450822|4186|2|422| +2450822|4189|2|466| +2450822|4190|2|237| +2450822|4192|2|599| +2450822|4195|2|954| +2450822|4196|2|876| +2450822|4198|2|799| +2450822|4201|2|86| +2450822|4202|2|852| +2450822|4204|2|669| +2450822|4207|2|424| +2450822|4208|2|803| +2450822|4210|2|606| +2450822|4213|2|297| +2450822|4214|2|152| +2450822|4216|2|706| +2450822|4219|2|255| +2450822|4220|2|351| +2450822|4222|2|245| +2450822|4225|2|872| +2450822|4226|2|913| +2450822|4228|2|221| +2450822|4231|2|688| +2450822|4232|2|713| +2450822|4234|2|229| +2450822|4237|2|166| +2450822|4238|2|793| +2450822|4240|2|472| +2450822|4243|2|422| +2450822|4244|2|438| +2450822|4246|2|652| +2450822|4249|2|| +2450822|4250|2|679| +2450822|4252|2|216| +2450822|4255|2|595| +2450822|4256|2|764| +2450822|4258|2|851| +2450822|4261|2|258| +2450822|4262|2|16| +2450822|4264|2|805| +2450822|4267|2|594| +2450822|4268|2|735| +2450822|4270|2|894| +2450822|4273|2|349| +2450822|4274|2|206| +2450822|4276|2|842| +2450822|4279|2|440| +2450822|4280|2|475| +2450822|4282|2|| +2450822|4285|2|995| +2450822|4286|2|709| +2450822|4288|2|347| +2450822|4291|2|873| +2450822|4292|2|402| +2450822|4294|2|15| +2450822|4297|2|527| +2450822|4298|2|166| +2450822|4300|2|655| +2450822|4303|2|210| +2450822|4304|2|548| +2450822|4306|2|| +2450822|4309|2|861| +2450822|4310|2|550| +2450822|4312|2|6| +2450822|4315|2|429| +2450822|4316|2|545| +2450822|4318|2|591| +2450822|4321|2|784| +2450822|4322|2|717| +2450822|4324|2|394| +2450822|4327|2|891| +2450822|4328|2|268| +2450822|4330|2|738| +2450822|4333|2|35| +2450822|4334|2|176| +2450822|4336|2|69| +2450822|4339|2|21| +2450822|4340|2|540| +2450822|4342|2|756| +2450822|4345|2|| +2450822|4346|2|532| +2450822|4348|2|917| +2450822|4351|2|247| +2450822|4352|2|795| +2450822|4354|2|798| +2450822|4357|2|518| +2450822|4358|2|| +2450822|4360|2|761| +2450822|4363|2|563| +2450822|4364|2|563| +2450822|4366|2|609| +2450822|4369|2|664| +2450822|4370|2|738| +2450822|4372|2|673| +2450822|4375|2|303| +2450822|4376|2|531| +2450822|4378|2|314| +2450822|4381|2|381| +2450822|4382|2|128| +2450822|4384|2|90| +2450822|4387|2|623| +2450822|4388|2|538| +2450822|4390|2|166| +2450822|4393|2|837| +2450822|4394|2|61| +2450822|4396|2|627| +2450822|4399|2|186| +2450822|4400|2|180| +2450822|4402|2|208| +2450822|4405|2|859| +2450822|4406|2|623| +2450822|4408|2|490| +2450822|4411|2|730| +2450822|4412|2|84| +2450822|4414|2|519| +2450822|4417|2|371| +2450822|4418|2|272| +2450822|4420|2|829| +2450822|4423|2|952| +2450822|4424|2|267| +2450822|4426|2|388| +2450822|4429|2|813| +2450822|4430|2|714| +2450822|4432|2|996| +2450822|4435|2|214| +2450822|4436|2|819| +2450822|4438|2|764| +2450822|4441|2|500| +2450822|4442|2|107| +2450822|4444|2|286| +2450822|4447|2|431| +2450822|4448|2|272| +2450822|4450|2|84| +2450822|4453|2|662| +2450822|4454|2|712| +2450822|4456|2|46| +2450822|4459|2|472| +2450822|4460|2|628| +2450822|4462|2|12| +2450822|4465|2|784| +2450822|4466|2|111| +2450822|4468|2|665| +2450822|4471|2|371| +2450822|4472|2|746| +2450822|4474|2|778| +2450822|4477|2|709| +2450822|4478|2|924| +2450822|4480|2|302| +2450822|4483|2|49| +2450822|4484|2|86| +2450822|4486|2|884| +2450822|4489|2|| +2450822|4490|2|791| +2450822|4492|2|170| +2450822|4495|2|659| +2450822|4496|2|349| +2450822|4498|2|527| +2450822|4501|2|301| +2450822|4502|2|809| +2450822|4504|2|993| +2450822|4507|2|109| +2450822|4508|2|524| +2450822|4510|2|916| +2450822|4513|2|388| +2450822|4514|2|472| +2450822|4516|2|127| +2450822|4519|2|127| +2450822|4520|2|311| +2450822|4522|2|220| +2450822|4525|2|413| +2450822|4526|2|312| +2450822|4528|2|183| +2450822|4531|2|741| +2450822|4532|2|532| +2450822|4534|2|552| +2450822|4537|2|469| +2450822|4538|2|266| +2450822|4540|2|656| +2450822|4543|2|352| +2450822|4544|2|139| +2450822|4546|2|635| +2450822|4549|2|269| +2450822|4550|2|1| +2450822|4552|2|678| +2450822|4555|2|14| +2450822|4556|2|12| +2450822|4558|2|844| +2450822|4561|2|390| +2450822|4562|2|910| +2450822|4564|2|818| +2450822|4567|2|105| +2450822|4568|2|488| +2450822|4570|2|736| +2450822|4573|2|81| +2450822|4574|2|610| +2450822|4576|2|344| +2450822|4579|2|525| +2450822|4580|2|763| +2450822|4582|2|404| +2450822|4585|2|413| +2450822|4586|2|752| +2450822|4588|2|90| +2450822|4591|2|693| +2450822|4592|2|696| +2450822|4594|2|983| +2450822|4597|2|864| +2450822|4598|2|631| +2450822|4600|2|133| +2450822|4603|2|160| +2450822|4604|2|139| +2450822|4606|2|463| +2450822|4609|2|251| +2450822|4610|2|907| +2450822|4612|2|411| +2450822|4615|2|576| +2450822|4616|2|421| +2450822|4618|2|216| +2450822|4621|2|704| +2450822|4622|2|| +2450822|4624|2|639| +2450822|4627|2|| +2450822|4628|2|644| +2450822|4630|2|475| +2450822|4633|2|556| +2450822|4634|2|934| +2450822|4636|2|617| +2450822|4639|2|| +2450822|4640|2|367| +2450822|4642|2|372| +2450822|4645|2|660| +2450822|4646|2|698| +2450822|4648|2|387| +2450822|4651|2|137| +2450822|4652|2|412| +2450822|4654|2|363| +2450822|4657|2|801| +2450822|4658|2|616| +2450822|4660|2|462| +2450822|4663|2|655| +2450822|4664|2|140| +2450822|4666|2|| +2450822|4669|2|623| +2450822|4670|2|| +2450822|4672|2|150| +2450822|4675|2|| +2450822|4676|2|377| +2450822|4678|2|790| +2450822|4681|2|778| +2450822|4682|2|155| +2450822|4684|2|397| +2450822|4687|2|76| +2450822|4688|2|730| +2450822|4690|2|977| +2450822|4693|2|942| +2450822|4694|2|32| +2450822|4696|2|57| +2450822|4699|2|245| +2450822|4700|2|858| +2450822|4702|2|228| +2450822|4705|2|749| +2450822|4706|2|318| +2450822|4708|2|478| +2450822|4711|2|136| +2450822|4712|2|670| +2450822|4714|2|207| +2450822|4717|2|250| +2450822|4718|2|156| +2450822|4720|2|811| +2450822|4723|2|84| +2450822|4724|2|665| +2450822|4726|2|874| +2450822|4729|2|356| +2450822|4730|2|672| +2450822|4732|2|| +2450822|4735|2|910| +2450822|4736|2|632| +2450822|4738|2|347| +2450822|4741|2|658| +2450822|4742|2|974| +2450822|4744|2|479| +2450822|4747|2|375| +2450822|4748|2|584| +2450822|4750|2|255| +2450822|4753|2|550| +2450822|4754|2|675| +2450822|4756|2|661| +2450822|4759|2|938| +2450822|4760|2|437| +2450822|4762|2|494| +2450822|4765|2|875| +2450822|4766|2|515| +2450822|4768|2|126| +2450822|4771|2|814| +2450822|4772|2|101| +2450822|4774|2|94| +2450822|4777|2|391| +2450822|4778|2|11| +2450822|4780|2|529| +2450822|4783|2|518| +2450822|4784|2|382| +2450822|4786|2|877| +2450822|4789|2|432| +2450822|4790|2|970| +2450822|4792|2|745| +2450822|4795|2|| +2450822|4796|2|238| +2450822|4798|2|621| +2450822|4801|2|939| +2450822|4802|2|387| +2450822|4804|2|325| +2450822|4807|2|822| +2450822|4808|2|809| +2450822|4810|2|| +2450822|4813|2|66| +2450822|4814|2|430| +2450822|4816|2|614| +2450822|4819|2|142| +2450822|4820|2|11| +2450822|4822|2|103| +2450822|4825|2|153| +2450822|4826|2|824| +2450822|4828|2|395| +2450822|4831|2|691| +2450822|4832|2|2| +2450822|4834|2|| +2450822|4837|2|301| +2450822|4838|2|259| +2450822|4840|2|168| +2450822|4843|2|248| +2450822|4844|2|814| +2450822|4846|2|950| +2450822|4849|2|784| +2450822|4850|2|782| +2450822|4852|2|250| +2450822|4855|2|883| +2450822|4856|2|869| +2450822|4858|2|268| +2450822|4861|2|197| +2450822|4862|2|7| +2450822|4864|2|938| +2450822|4867|2|808| +2450822|4868|2|778| +2450822|4870|2|532| +2450822|4873|2|33| +2450822|4874|2|468| +2450822|4876|2|959| +2450822|4879|2|437| +2450822|4880|2|536| +2450822|4882|2|363| +2450822|4885|2|306| +2450822|4886|2|163| +2450822|4888|2|963| +2450822|4891|2|16| +2450822|4892|2|575| +2450822|4894|2|810| +2450822|4897|2|676| +2450822|4898|2|410| +2450822|4900|2|644| +2450822|4903|2|135| +2450822|4904|2|981| +2450822|4906|2|476| +2450822|4909|2|362| +2450822|4910|2|190| +2450822|4912|2|493| +2450822|4915|2|555| +2450822|4916|2|366| +2450822|4918|2|555| +2450822|4921|2|411| +2450822|4922|2|810| +2450822|4924|2|851| +2450822|4927|2|141| +2450822|4928|2|586| +2450822|4930|2|56| +2450822|4933|2|77| +2450822|4934|2|796| +2450822|4936|2|926| +2450822|4939|2|895| +2450822|4940|2|727| +2450822|4942|2|913| +2450822|4945|2|333| +2450822|4946|2|733| +2450822|4948|2|739| +2450822|4951|2|608| +2450822|4952|2|977| +2450822|4954|2|4| +2450822|4957|2|| +2450822|4958|2|701| +2450822|4960|2|531| +2450822|4963|2|888| +2450822|4964|2|974| +2450822|4966|2|968| +2450822|4969|2|459| +2450822|4970|2|841| +2450822|4972|2|394| +2450822|4975|2|75| +2450822|4976|2|706| +2450822|4978|2|710| +2450822|4981|2|115| +2450822|4982|2|927| +2450822|4984|2|581| +2450822|4987|2|899| +2450822|4988|2|546| +2450822|4990|2|995| +2450822|4993|2|514| +2450822|4994|2|54| +2450822|4996|2|709| +2450822|4999|2|57| +2450822|5000|2|297| +2450822|5002|2|378| +2450822|5005|2|501| +2450822|5006|2|333| +2450822|5008|2|| +2450822|5011|2|111| +2450822|5012|2|179| +2450822|5014|2|148| +2450822|5017|2|528| +2450822|5018|2|533| +2450822|5020|2|551| +2450822|5023|2|330| +2450822|5024|2|24| +2450822|5026|2|582| +2450822|5029|2|329| +2450822|5030|2|508| +2450822|5032|2|149| +2450822|5035|2|884| +2450822|5036|2|417| +2450822|5038|2|81| +2450822|5041|2|| +2450822|5042|2|129| +2450822|5044|2|4| +2450822|5047|2|227| +2450822|5048|2|539| +2450822|5050|2|7| +2450822|5053|2|115| +2450822|5054|2|685| +2450822|5056|2|564| +2450822|5059|2|27| +2450822|5060|2|422| +2450822|5062|2|800| +2450822|5065|2|283| +2450822|5066|2|506| +2450822|5068|2|202| +2450822|5071|2|331| +2450822|5072|2|603| +2450822|5074|2|| +2450822|5077|2|914| +2450822|5078|2|635| +2450822|5080|2|411| +2450822|5083|2|624| +2450822|5084|2|926| +2450822|5086|2|219| +2450822|5089|2|426| +2450822|5090|2|421| +2450822|5092|2|755| +2450822|5095|2|166| +2450822|5096|2|350| +2450822|5098|2|505| +2450822|5101|2|328| +2450822|5102|2|874| +2450822|5104|2|| +2450822|5107|2|582| +2450822|5108|2|757| +2450822|5110|2|853| +2450822|5113|2|606| +2450822|5114|2|1000| +2450822|5116|2|593| +2450822|5119|2|164| +2450822|5120|2|85| +2450822|5122|2|780| +2450822|5125|2|152| +2450822|5126|2|817| +2450822|5128|2|794| +2450822|5131|2|198| +2450822|5132|2|342| +2450822|5134|2|902| +2450822|5137|2|251| +2450822|5138|2|239| +2450822|5140|2|721| +2450822|5143|2|579| +2450822|5144|2|894| +2450822|5146|2|423| +2450822|5149|2|464| +2450822|5150|2|984| +2450822|5152|2|313| +2450822|5155|2|635| +2450822|5156|2|| +2450822|5158|2|629| +2450822|5161|2|783| +2450822|5162|2|301| +2450822|5164|2|342| +2450822|5167|2|539| +2450822|5168|2|212| +2450822|5170|2|14| +2450822|5173|2|650| +2450822|5174|2|247| +2450822|5176|2|590| +2450822|5179|2|230| +2450822|5180|2|856| +2450822|5182|2|173| +2450822|5185|2|| +2450822|5186|2|503| +2450822|5188|2|992| +2450822|5191|2|615| +2450822|5192|2|790| +2450822|5194|2|959| +2450822|5197|2|323| +2450822|5198|2|950| +2450822|5200|2|772| +2450822|5203|2|512| +2450822|5204|2|314| +2450822|5206|2|181| +2450822|5209|2|999| +2450822|5210|2|14| +2450822|5212|2|891| +2450822|5215|2|6| +2450822|5216|2|513| +2450822|5218|2|814| +2450822|5221|2|98| +2450822|5222|2|27| +2450822|5224|2|17| +2450822|5227|2|377| +2450822|5228|2|| +2450822|5230|2|949| +2450822|5233|2|493| +2450822|5234|2|206| +2450822|5236|2|972| +2450822|5239|2|287| +2450822|5240|2|943| +2450822|5242|2|991| +2450822|5245|2|489| +2450822|5246|2|676| +2450822|5248|2|603| +2450822|5251|2|416| +2450822|5252|2|424| +2450822|5254|2|| +2450822|5257|2|277| +2450822|5258|2|578| +2450822|5260|2|552| +2450822|5263|2|634| +2450822|5264|2|319| +2450822|5266|2|621| +2450822|5269|2|| +2450822|5270|2|594| +2450822|5272|2|234| +2450822|5275|2|273| +2450822|5276|2|334| +2450822|5278|2|421| +2450822|5281|2|544| +2450822|5282|2|639| +2450822|5284|2|430| +2450822|5287|2|210| +2450822|5288|2|106| +2450822|5290|2|472| +2450822|5293|2|252| +2450822|5294|2|607| +2450822|5296|2|670| +2450822|5299|2|84| +2450822|5300|2|67| +2450822|5302|2|386| +2450822|5305|2|675| +2450822|5306|2|502| +2450822|5308|2|907| +2450822|5311|2|841| +2450822|5312|2|943| +2450822|5314|2|800| +2450822|5317|2|930| +2450822|5318|2|625| +2450822|5320|2|173| +2450822|5323|2|915| +2450822|5324|2|153| +2450822|5326|2|668| +2450822|5329|2|188| +2450822|5330|2|221| +2450822|5332|2|204| +2450822|5335|2|421| +2450822|5336|2|315| +2450822|5338|2|271| +2450822|5341|2|641| +2450822|5342|2|788| +2450822|5344|2|962| +2450822|5347|2|468| +2450822|5348|2|183| +2450822|5350|2|890| +2450822|5353|2|933| +2450822|5354|2|240| +2450822|5356|2|568| +2450822|5359|2|794| +2450822|5360|2|517| +2450822|5362|2|78| +2450822|5365|2|452| +2450822|5366|2|58| +2450822|5368|2|349| +2450822|5371|2|279| +2450822|5372|2|507| +2450822|5374|2|875| +2450822|5377|2|| +2450822|5378|2|350| +2450822|5380|2|608| +2450822|5383|2|682| +2450822|5384|2|140| +2450822|5386|2|197| +2450822|5389|2|320| +2450822|5390|2|450| +2450822|5392|2|37| +2450822|5395|2|745| +2450822|5396|2|994| +2450822|5398|2|824| +2450822|5401|2|659| +2450822|5402|2|507| +2450822|5404|2|769| +2450822|5407|2|| +2450822|5408|2|685| +2450822|5410|2|978| +2450822|5413|2|898| +2450822|5414|2|991| +2450822|5416|2|423| +2450822|5419|2|145| +2450822|5420|2|701| +2450822|5422|2|540| +2450822|5425|2|275| +2450822|5426|2|676| +2450822|5428|2|596| +2450822|5431|2|63| +2450822|5432|2|287| +2450822|5434|2|439| +2450822|5437|2|661| +2450822|5438|2|983| +2450822|5440|2|407| +2450822|5443|2|777| +2450822|5444|2|87| +2450822|5446|2|106| +2450822|5449|2|39| +2450822|5450|2|197| +2450822|5452|2|208| +2450822|5455|2|961| +2450822|5456|2|319| +2450822|5458|2|260| +2450822|5461|2|846| +2450822|5462|2|846| +2450822|5464|2|528| +2450822|5467|2|791| +2450822|5468|2|994| +2450822|5470|2|| +2450822|5473|2|648| +2450822|5474|2|449| +2450822|5476|2|195| +2450822|5479|2|283| +2450822|5480|2|| +2450822|5482|2|369| +2450822|5485|2|215| +2450822|5486|2|76| +2450822|5488|2|925| +2450822|5491|2|201| +2450822|5492|2|296| +2450822|5494|2|493| +2450822|5497|2|422| +2450822|5498|2|105| +2450822|5500|2|962| +2450822|5503|2|447| +2450822|5504|2|| +2450822|5506|2|945| +2450822|5509|2|378| +2450822|5510|2|329| +2450822|5512|2|184| +2450822|5515|2|90| +2450822|5516|2|952| +2450822|5518|2|397| +2450822|5521|2|242| +2450822|5522|2|843| +2450822|5524|2|128| +2450822|5527|2|178| +2450822|5528|2|706| +2450822|5530|2|356| +2450822|5533|2|673| +2450822|5534|2|392| +2450822|5536|2|240| +2450822|5539|2|816| +2450822|5540|2|128| +2450822|5542|2|246| +2450822|5545|2|871| +2450822|5546|2|566| +2450822|5548|2|467| +2450822|5551|2|634| +2450822|5552|2|948| +2450822|5554|2|866| +2450822|5557|2|231| +2450822|5558|2|136| +2450822|5560|2|712| +2450822|5563|2|832| +2450822|5564|2|378| +2450822|5566|2|989| +2450822|5569|2|904| +2450822|5570|2|671| +2450822|5572|2|735| +2450822|5575|2|760| +2450822|5576|2|7| +2450822|5578|2|583| +2450822|5581|2|681| +2450822|5582|2|843| +2450822|5584|2|236| +2450822|5587|2|206| +2450822|5588|2|809| +2450822|5590|2|743| +2450822|5593|2|| +2450822|5594|2|522| +2450822|5596|2|683| +2450822|5599|2|535| +2450822|5600|2|785| +2450822|5602|2|955| +2450822|5605|2|785| +2450822|5606|2|441| +2450822|5608|2|198| +2450822|5611|2|480| +2450822|5612|2|482| +2450822|5614|2|210| +2450822|5617|2|447| +2450822|5618|2|8| +2450822|5620|2|274| +2450822|5623|2|317| +2450822|5624|2|56| +2450822|5626|2|286| +2450822|5629|2|736| +2450822|5630|2|657| +2450822|5632|2|760| +2450822|5635|2|428| +2450822|5636|2|492| +2450822|5638|2|210| +2450822|5641|2|550| +2450822|5642|2|498| +2450822|5644|2|265| +2450822|5647|2|897| +2450822|5648|2|729| +2450822|5650|2|293| +2450822|5653|2|162| +2450822|5654|2|515| +2450822|5656|2|993| +2450822|5659|2|| +2450822|5660|2|190| +2450822|5662|2|286| +2450822|5665|2|12| +2450822|5666|2|806| +2450822|5668|2|263| +2450822|5671|2|334| +2450822|5672|2|670| +2450822|5674|2|849| +2450822|5677|2|| +2450822|5678|2|73| +2450822|5680|2|479| +2450822|5683|2|166| +2450822|5684|2|111| +2450822|5686|2|206| +2450822|5689|2|209| +2450822|5690|2|966| +2450822|5692|2|892| +2450822|5695|2|642| +2450822|5696|2|456| +2450822|5698|2|280| +2450822|5701|2|886| +2450822|5702|2|945| +2450822|5704|2|158| +2450822|5707|2|69| +2450822|5708|2|952| +2450822|5710|2|699| +2450822|5713|2|166| +2450822|5714|2|528| +2450822|5716|2|294| +2450822|5719|2|771| +2450822|5720|2|699| +2450822|5722|2|948| +2450822|5725|2|733| +2450822|5726|2|504| +2450822|5728|2|868| +2450822|5731|2|| +2450822|5732|2|727| +2450822|5734|2|851| +2450822|5737|2|981| +2450822|5738|2|589| +2450822|5740|2|264| +2450822|5743|2|620| +2450822|5744|2|902| +2450822|5746|2|637| +2450822|5749|2|721| +2450822|5750|2|378| +2450822|5752|2|455| +2450822|5755|2|542| +2450822|5756|2|532| +2450822|5758|2|415| +2450822|5761|2|822| +2450822|5762|2|524| +2450822|5764|2|493| +2450822|5767|2|751| +2450822|5768|2|363| +2450822|5770|2|893| +2450822|5773|2|375| +2450822|5774|2|369| +2450822|5776|2|949| +2450822|5779|2|637| +2450822|5780|2|455| +2450822|5782|2|816| +2450822|5785|2|834| +2450822|5786|2|807| +2450822|5788|2|558| +2450822|5791|2|278| +2450822|5792|2|513| +2450822|5794|2|| +2450822|5797|2|679| +2450822|5798|2|769| +2450822|5800|2|141| +2450822|5803|2|760| +2450822|5804|2|906| +2450822|5806|2|996| +2450822|5809|2|289| +2450822|5810|2|626| +2450822|5812|2|| +2450822|5815|2|649| +2450822|5816|2|837| +2450822|5818|2|338| +2450822|5821|2|464| +2450822|5822|2|395| +2450822|5824|2|460| +2450822|5827|2|| +2450822|5828|2|184| +2450822|5830|2|780| +2450822|5833|2|703| +2450822|5834|2|206| +2450822|5836|2|943| +2450822|5839|2|714| +2450822|5840|2|501| +2450822|5842|2|442| +2450822|5845|2|419| +2450822|5846|2|191| +2450822|5848|2|682| +2450822|5851|2|765| +2450822|5852|2|373| +2450822|5854|2|817| +2450822|5857|2|406| +2450822|5858|2|510| +2450822|5860|2|415| +2450822|5863|2|476| +2450822|5864|2|647| +2450822|5866|2|106| +2450822|5869|2|92| +2450822|5870|2|225| +2450822|5872|2|874| +2450822|5875|2|951| +2450822|5876|2|976| +2450822|5878|2|878| +2450822|5881|2|671| +2450822|5882|2|266| +2450822|5884|2|331| +2450822|5887|2|627| +2450822|5888|2|477| +2450822|5890|2|866| +2450822|5893|2|132| +2450822|5894|2|796| +2450822|5896|2|156| +2450822|5899|2|993| +2450822|5900|2|124| +2450822|5902|2|361| +2450822|5905|2|712| +2450822|5906|2|702| +2450822|5908|2|| +2450822|5911|2|560| +2450822|5912|2|704| +2450822|5914|2|164| +2450822|5917|2|920| +2450822|5918|2|509| +2450822|5920|2|25| +2450822|5923|2|484| +2450822|5924|2|281| +2450822|5926|2|7| +2450822|5929|2|293| +2450822|5930|2|58| +2450822|5932|2|524| +2450822|5935|2|151| +2450822|5936|2|840| +2450822|5938|2|585| +2450822|5941|2|826| +2450822|5942|2|731| +2450822|5944|2|761| +2450822|5947|2|636| +2450822|5948|2|502| +2450822|5950|2|480| +2450822|5953|2|52| +2450822|5954|2|315| +2450822|5956|2|| +2450822|5959|2|297| +2450822|5960|2|868| +2450822|5962|2|201| +2450822|5965|2|64| +2450822|5966|2|945| +2450822|5968|2|518| +2450822|5971|2|773| +2450822|5972|2|670| +2450822|5974|2|724| +2450822|5977|2|778| +2450822|5978|2|841| +2450822|5980|2|485| +2450822|5983|2|338| +2450822|5984|2|| +2450822|5986|2|147| +2450822|5989|2|321| +2450822|5990|2|| +2450822|5992|2|479| +2450822|5995|2|132| +2450822|5996|2|693| +2450822|5998|2|396| +2450822|6001|2|| +2450822|6002|2|30| +2450822|6004|2|147| +2450822|6007|2|288| +2450822|6008|2|106| +2450822|6010|2|232| +2450822|6013|2|330| +2450822|6014|2|239| +2450822|6016|2|362| +2450822|6019|2|602| +2450822|6020|2|402| +2450822|6022|2|278| +2450822|6025|2|475| +2450822|6026|2|660| +2450822|6028|2|165| +2450822|6031|2|232| +2450822|6032|2|109| +2450822|6034|2|57| +2450822|6037|2|883| +2450822|6038|2|879| +2450822|6040|2|917| +2450822|6043|2|372| +2450822|6044|2|841| +2450822|6046|2|637| +2450822|6049|2|29| +2450822|6050|2|325| +2450822|6052|2|749| +2450822|6055|2|727| +2450822|6056|2|281| +2450822|6058|2|691| +2450822|6061|2|230| +2450822|6062|2|38| +2450822|6064|2|625| +2450822|6067|2|899| +2450822|6068|2|799| +2450822|6070|2|398| +2450822|6073|2|642| +2450822|6074|2|27| +2450822|6076|2|299| +2450822|6079|2|463| +2450822|6080|2|299| +2450822|6082|2|800| +2450822|6085|2|231| +2450822|6086|2|| +2450822|6088|2|169| +2450822|6091|2|882| +2450822|6092|2|| +2450822|6094|2|469| +2450822|6097|2|450| +2450822|6098|2|347| +2450822|6100|2|| +2450822|6103|2|894| +2450822|6104|2|943| +2450822|6106|2|529| +2450822|6109|2|248| +2450822|6110|2|811| +2450822|6112|2|| +2450822|6115|2|327| +2450822|6116|2|| +2450822|6118|2|562| +2450822|6121|2|524| +2450822|6122|2|905| +2450822|6124|2|191| +2450822|6127|2|621| +2450822|6128|2|445| +2450822|6130|2|618| +2450822|6133|2|853| +2450822|6134|2|756| +2450822|6136|2|705| +2450822|6139|2|641| +2450822|6140|2|847| +2450822|6142|2|865| +2450822|6145|2|982| +2450822|6146|2|719| +2450822|6148|2|982| +2450822|6151|2|867| +2450822|6152|2|158| +2450822|6154|2|551| +2450822|6157|2|294| +2450822|6158|2|443| +2450822|6160|2|380| +2450822|6163|2|821| +2450822|6164|2|485| +2450822|6166|2|300| +2450822|6169|2|938| +2450822|6170|2|286| +2450822|6172|2|86| +2450822|6175|2|671| +2450822|6176|2|886| +2450822|6178|2|967| +2450822|6181|2|473| +2450822|6182|2|33| +2450822|6184|2|250| +2450822|6187|2|231| +2450822|6188|2|189| +2450822|6190|2|698| +2450822|6193|2|851| +2450822|6194|2|54| +2450822|6196|2|251| +2450822|6199|2|556| +2450822|6200|2|874| +2450822|6202|2|472| +2450822|6205|2|141| +2450822|6206|2|434| +2450822|6208|2|714| +2450822|6211|2|| +2450822|6212|2|531| +2450822|6214|2|528| +2450822|6217|2|211| +2450822|6218|2|248| +2450822|6220|2|160| +2450822|6223|2|69| +2450822|6224|2|680| +2450822|6226|2|954| +2450822|6229|2|341| +2450822|6230|2|537| +2450822|6232|2|62| +2450822|6235|2|152| +2450822|6236|2|| +2450822|6238|2|944| +2450822|6241|2|548| +2450822|6242|2|342| +2450822|6244|2|934| +2450822|6247|2|750| +2450822|6248|2|157| +2450822|6250|2|122| +2450822|6253|2|585| +2450822|6254|2|607| +2450822|6256|2|355| +2450822|6259|2|714| +2450822|6260|2|88| +2450822|6262|2|256| +2450822|6265|2|278| +2450822|6266|2|172| +2450822|6268|2|769| +2450822|6271|2|989| +2450822|6272|2|414| +2450822|6274|2|299| +2450822|6277|2|| +2450822|6278|2|741| +2450822|6280|2|73| +2450822|6283|2|500| +2450822|6284|2|750| +2450822|6286|2|353| +2450822|6289|2|330| +2450822|6290|2|311| +2450822|6292|2|| +2450822|6295|2|618| +2450822|6296|2|642| +2450822|6298|2|| +2450822|6301|2|176| +2450822|6302|2|981| +2450822|6304|2|795| +2450822|6307|2|770| +2450822|6308|2|995| +2450822|6310|2|285| +2450822|6313|2|464| +2450822|6314|2|394| +2450822|6316|2|31| +2450822|6319|2|580| +2450822|6320|2|374| +2450822|6322|2|671| +2450822|6325|2|865| +2450822|6326|2|328| +2450822|6328|2|| +2450822|6331|2|723| +2450822|6332|2|528| +2450822|6334|2|353| +2450822|6337|2|635| +2450822|6338|2|954| +2450822|6340|2|312| +2450822|6343|2|599| +2450822|6344|2|480| +2450822|6346|2|953| +2450822|6349|2|930| +2450822|6350|2|250| +2450822|6352|2|459| +2450822|6355|2|726| +2450822|6356|2|492| +2450822|6358|2|984| +2450822|6361|2|221| +2450822|6362|2|44| +2450822|6364|2|358| +2450822|6367|2|| +2450822|6368|2|287| +2450822|6370|2|277| +2450822|6373|2|645| +2450822|6374|2|429| +2450822|6376|2|695| +2450822|6379|2|746| +2450822|6380|2|| +2450822|6382|2|977| +2450822|6385|2|397| +2450822|6386|2|30| +2450822|6388|2|781| +2450822|6391|2|475| +2450822|6392|2|582| +2450822|6394|2|540| +2450822|6397|2|368| +2450822|6398|2|586| +2450822|6400|2|942| +2450822|6403|2|547| +2450822|6404|2|725| +2450822|6406|2|| +2450822|6409|2|388| +2450822|6410|2|778| +2450822|6412|2|808| +2450822|6415|2|7| +2450822|6416|2|580| +2450822|6418|2|613| +2450822|6421|2|| +2450822|6422|2|436| +2450822|6424|2|762| +2450822|6427|2|923| +2450822|6428|2|645| +2450822|6430|2|49| +2450822|6433|2|957| +2450822|6434|2|834| +2450822|6436|2|265| +2450822|6439|2|833| +2450822|6440|2|410| +2450822|6442|2|86| +2450822|6445|2|700| +2450822|6446|2|799| +2450822|6448|2|210| +2450822|6451|2|317| +2450822|6452|2|277| +2450822|6454|2|938| +2450822|6457|2|219| +2450822|6458|2|575| +2450822|6460|2|708| +2450822|6463|2|621| +2450822|6464|2|75| +2450822|6466|2|377| +2450822|6469|2|649| +2450822|6470|2|955| +2450822|6472|2|8| +2450822|6475|2|154| +2450822|6476|2|980| +2450822|6478|2|700| +2450822|6481|2|469| +2450822|6482|2|740| +2450822|6484|2|970| +2450822|6487|2|99| +2450822|6488|2|411| +2450822|6490|2|13| +2450822|6493|2|271| +2450822|6494|2|162| +2450822|6496|2|838| +2450822|6499|2|360| +2450822|6500|2|809| +2450822|6502|2|588| +2450822|6505|2|215| +2450822|6506|2|473| +2450822|6508|2|179| +2450822|6511|2|469| +2450822|6512|2|884| +2450822|6514|2|117| +2450822|6517|2|127| +2450822|6518|2|202| +2450822|6520|2|740| +2450822|6523|2|962| +2450822|6524|2|142| +2450822|6526|2|500| +2450822|6529|2|742| +2450822|6530|2|| +2450822|6532|2|512| +2450822|6535|2|733| +2450822|6536|2|828| +2450822|6538|2|27| +2450822|6541|2|1| +2450822|6542|2|420| +2450822|6544|2|919| +2450822|6547|2|789| +2450822|6548|2|182| +2450822|6550|2|875| +2450822|6553|2|446| +2450822|6554|2|438| +2450822|6556|2|293| +2450822|6559|2|463| +2450822|6560|2|577| +2450822|6562|2|247| +2450822|6565|2|365| +2450822|6566|2|999| +2450822|6568|2|924| +2450822|6571|2|755| +2450822|6572|2|151| +2450822|6574|2|54| +2450822|6577|2|745| +2450822|6578|2|964| +2450822|6580|2|566| +2450822|6583|2|71| +2450822|6584|2|396| +2450822|6586|2|545| +2450822|6589|2|644| +2450822|6590|2|474| +2450822|6592|2|66| +2450822|6595|2|328| +2450822|6596|2|821| +2450822|6598|2|975| +2450822|6601|2|136| +2450822|6602|2|724| +2450822|6604|2|468| +2450822|6607|2|642| +2450822|6608|2|112| +2450822|6610|2|599| +2450822|6613|2|154| +2450822|6614|2|164| +2450822|6616|2|| +2450822|6619|2|707| +2450822|6620|2|990| +2450822|6622|2|508| +2450822|6625|2|653| +2450822|6626|2|397| +2450822|6628|2|636| +2450822|6631|2|| +2450822|6632|2|| +2450822|6634|2|583| +2450822|6637|2|696| +2450822|6638|2|| +2450822|6640|2|596| +2450822|6643|2|659| +2450822|6644|2|972| +2450822|6646|2|90| +2450822|6649|2|845| +2450822|6650|2|193| +2450822|6652|2|137| +2450822|6655|2|169| +2450822|6656|2|124| +2450822|6658|2|306| +2450822|6661|2|809| +2450822|6662|2|502| +2450822|6664|2|121| +2450822|6667|2|| +2450822|6668|2|77| +2450822|6670|2|306| +2450822|6673|2|740| +2450822|6674|2|61| +2450822|6676|2|431| +2450822|6679|2|82| +2450822|6680|2|209| +2450822|6682|2|198| +2450822|6685|2|454| +2450822|6686|2|790| +2450822|6688|2|410| +2450822|6691|2|377| +2450822|6692|2|484| +2450822|6694|2|882| +2450822|6697|2|892| +2450822|6698|2|313| +2450822|6700|2|3| +2450822|6703|2|171| +2450822|6704|2|456| +2450822|6706|2|568| +2450822|6709|2|563| +2450822|6710|2|911| +2450822|6712|2|695| +2450822|6715|2|678| +2450822|6716|2|797| +2450822|6718|2|7| +2450822|6721|2|| +2450822|6722|2|152| +2450822|6724|2|113| +2450822|6727|2|340| +2450822|6728|2|512| +2450822|6730|2|285| +2450822|6733|2|914| +2450822|6734|2|424| +2450822|6736|2|24| +2450822|6739|2|243| +2450822|6740|2|853| +2450822|6742|2|| +2450822|6745|2|604| +2450822|6746|2|919| +2450822|6748|2|896| +2450822|6751|2|812| +2450822|6752|2|633| +2450822|6754|2|679| +2450822|6757|2|442| +2450822|6758|2|47| +2450822|6760|2|621| +2450822|6763|2|398| +2450822|6764|2|694| +2450822|6766|2|851| +2450822|6769|2|302| +2450822|6770|2|944| +2450822|6772|2|458| +2450822|6775|2|7| +2450822|6776|2|772| +2450822|6778|2|144| +2450822|6781|2|47| +2450822|6782|2|955| +2450822|6784|2|493| +2450822|6787|2|803| +2450822|6788|2|385| +2450822|6790|2|174| +2450822|6793|2|61| +2450822|6794|2|258| +2450822|6796|2|302| +2450822|6799|2|180| +2450822|6800|2|503| +2450822|6802|2|| +2450822|6805|2|827| +2450822|6806|2|361| +2450822|6808|2|184| +2450822|6811|2|807| +2450822|6812|2|636| +2450822|6814|2|194| +2450822|6817|2|856| +2450822|6818|2|795| +2450822|6820|2|744| +2450822|6823|2|961| +2450822|6824|2|345| +2450822|6826|2|291| +2450822|6829|2|644| +2450822|6830|2|354| +2450822|6832|2|444| +2450822|6835|2|69| +2450822|6836|2|277| +2450822|6838|2|734| +2450822|6841|2|785| +2450822|6842|2|133| +2450822|6844|2|78| +2450822|6847|2|497| +2450822|6848|2|63| +2450822|6850|2|738| +2450822|6853|2|881| +2450822|6854|2|327| +2450822|6856|2|454| +2450822|6859|2|50| +2450822|6860|2|| +2450822|6862|2|616| +2450822|6865|2|719| +2450822|6866|2|369| +2450822|6868|2|940| +2450822|6871|2|29| +2450822|6872|2|998| +2450822|6874|2|761| +2450822|6877|2|114| +2450822|6878|2|639| +2450822|6880|2|239| +2450822|6883|2|946| +2450822|6884|2|| +2450822|6886|2|966| +2450822|6889|2|713| +2450822|6890|2|360| +2450822|6892|2|282| +2450822|6895|2|509| +2450822|6896|2|215| +2450822|6898|2|378| +2450822|6901|2|507| +2450822|6902|2|812| +2450822|6904|2|484| +2450822|6907|2|243| +2450822|6908|2|756| +2450822|6910|2|432| +2450822|6913|2|85| +2450822|6914|2|615| +2450822|6916|2|240| +2450822|6919|2|242| +2450822|6920|2|185| +2450822|6922|2|714| +2450822|6925|2|| +2450822|6926|2|870| +2450822|6928|2|860| +2450822|6931|2|315| +2450822|6932|2|116| +2450822|6934|2|718| +2450822|6937|2|686| +2450822|6938|2|648| +2450822|6940|2|330| +2450822|6943|2|77| +2450822|6944|2|681| +2450822|6946|2|664| +2450822|6949|2|382| +2450822|6950|2|230| +2450822|6952|2|974| +2450822|6955|2|| +2450822|6956|2|557| +2450822|6958|2|565| +2450822|6961|2|632| +2450822|6962|2|588| +2450822|6964|2|618| +2450822|6967|2|944| +2450822|6968|2|550| +2450822|6970|2|900| +2450822|6973|2|390| +2450822|6974|2|268| +2450822|6976|2|468| +2450822|6979|2|50| +2450822|6980|2|505| +2450822|6982|2|112| +2450822|6985|2|9| +2450822|6986|2|279| +2450822|6988|2|209| +2450822|6991|2|205| +2450822|6992|2|530| +2450822|6994|2|641| +2450822|6997|2|988| +2450822|6998|2|479| +2450822|7000|2|652| +2450822|7003|2|557| +2450822|7004|2|950| +2450822|7006|2|366| +2450822|7009|2|696| +2450822|7010|2|176| +2450822|7012|2|372| +2450822|7015|2|675| +2450822|7016|2|179| +2450822|7018|2|197| +2450822|7021|2|881| +2450822|7022|2|93| +2450822|7024|2|222| +2450822|7027|2|3| +2450822|7028|2|| +2450822|7030|2|| +2450822|7033|2|657| +2450822|7034|2|457| +2450822|7036|2|178| +2450822|7039|2|629| +2450822|7040|2|458| +2450822|7042|2|824| +2450822|7045|2|| +2450822|7046|2|607| +2450822|7048|2|572| +2450822|7051|2|939| +2450822|7052|2|263| +2450822|7054|2|902| +2450822|7057|2|614| +2450822|7058|2|439| +2450822|7060|2|| +2450822|7063|2|454| +2450822|7064|2|87| +2450822|7066|2|627| +2450822|7069|2|698| +2450822|7070|2|654| +2450822|7072|2|22| +2450822|7075|2|202| +2450822|7076|2|321| +2450822|7078|2|474| +2450822|7081|2|72| +2450822|7082|2|77| +2450822|7084|2|701| +2450822|7087|2|621| +2450822|7088|2|946| +2450822|7090|2|424| +2450822|7093|2|503| +2450822|7094|2|268| +2450822|7096|2|490| +2450822|7099|2|830| +2450822|7100|2|728| +2450822|7102|2|229| +2450822|7105|2|596| +2450822|7106|2|508| +2450822|7108|2|19| +2450822|7111|2|432| +2450822|7112|2|253| +2450822|7114|2|539| +2450822|7117|2|746| +2450822|7118|2|39| +2450822|7120|2|725| +2450822|7123|2|802| +2450822|7124|2|585| +2450822|7126|2|53| +2450822|7129|2|865| +2450822|7130|2|168| +2450822|7132|2|359| +2450822|7135|2|128| +2450822|7136|2|925| +2450822|7138|2|235| +2450822|7141|2|853| +2450822|7142|2|99| +2450822|7144|2|928| +2450822|7147|2|845| +2450822|7148|2|183| +2450822|7150|2|502| +2450822|7153|2|478| +2450822|7154|2|202| +2450822|7156|2|863| +2450822|7159|2|246| +2450822|7160|2|501| +2450822|7162|2|| +2450822|7165|2|709| +2450822|7166|2|375| +2450822|7168|2|61| +2450822|7171|2|288| +2450822|7172|2|528| +2450822|7174|2|213| +2450822|7177|2|156| +2450822|7178|2|932| +2450822|7180|2|124| +2450822|7183|2|44| +2450822|7184|2|337| +2450822|7186|2|333| +2450822|7189|2|193| +2450822|7190|2|92| +2450822|7192|2|688| +2450822|7195|2|953| +2450822|7196|2|15| +2450822|7198|2|75| +2450822|7201|2|213| +2450822|7202|2|202| +2450822|7204|2|| +2450822|7207|2|488| +2450822|7208|2|660| +2450822|7210|2|31| +2450822|7213|2|79| +2450822|7214|2|114| +2450822|7216|2|547| +2450822|7219|2|594| +2450822|7220|2|908| +2450822|7222|2|666| +2450822|7225|2|54| +2450822|7226|2|636| +2450822|7228|2|746| +2450822|7231|2|953| +2450822|7232|2|541| +2450822|7234|2|202| +2450822|7237|2|899| +2450822|7238|2|56| +2450822|7240|2|| +2450822|7243|2|618| +2450822|7244|2|177| +2450822|7246|2|581| +2450822|7249|2|767| +2450822|7250|2|310| +2450822|7252|2|20| +2450822|7255|2|965| +2450822|7256|2|64| +2450822|7258|2|922| +2450822|7261|2|768| +2450822|7262|2|| +2450822|7264|2|152| +2450822|7267|2|| +2450822|7268|2|474| +2450822|7270|2|636| +2450822|7273|2|41| +2450822|7274|2|12| +2450822|7276|2|125| +2450822|7279|2|795| +2450822|7280|2|337| +2450822|7282|2|96| +2450822|7285|2|727| +2450822|7286|2|37| +2450822|7288|2|471| +2450822|7291|2|648| +2450822|7292|2|193| +2450822|7294|2|409| +2450822|7297|2|717| +2450822|7298|2|364| +2450822|7300|2|295| +2450822|7303|2|216| +2450822|7304|2|138| +2450822|7306|2|381| +2450822|7309|2|667| +2450822|7310|2|126| +2450822|7312|2|512| +2450822|7315|2|855| +2450822|7316|2|888| +2450822|7318|2|238| +2450822|7321|2|104| +2450822|7322|2|498| +2450822|7324|2|470| +2450822|7327|2|207| +2450822|7328|2|645| +2450822|7330|2|808| +2450822|7333|2|492| +2450822|7334|2|793| +2450822|7336|2|373| +2450822|7339|2|860| +2450822|7340|2|801| +2450822|7342|2|714| +2450822|7345|2|727| +2450822|7346|2|520| +2450822|7348|2|731| +2450822|7351|2|| +2450822|7352|2|526| +2450822|7354|2|812| +2450822|7357|2|946| +2450822|7358|2|396| +2450822|7360|2|484| +2450822|7363|2|940| +2450822|7364|2|390| +2450822|7366|2|518| +2450822|7369|2|883| +2450822|7370|2|663| +2450822|7372|2|489| +2450822|7375|2|120| +2450822|7376|2|609| +2450822|7378|2|862| +2450822|7381|2|18| +2450822|7382|2|537| +2450822|7384|2|645| +2450822|7387|2|767| +2450822|7388|2|681| +2450822|7390|2|251| +2450822|7393|2|623| +2450822|7394|2|905| +2450822|7396|2|285| +2450822|7399|2|794| +2450822|7400|2|308| +2450822|7402|2|937| +2450822|7405|2|50| +2450822|7406|2|241| +2450822|7408|2|| +2450822|7411|2|460| +2450822|7412|2|| +2450822|7414|2|793| +2450822|7417|2|401| +2450822|7418|2|459| +2450822|7420|2|828| +2450822|7423|2|67| +2450822|7424|2|105| +2450822|7426|2|| +2450822|7429|2|78| +2450822|7430|2|750| +2450822|7432|2|993| +2450822|7435|2|382| +2450822|7436|2|879| +2450822|7438|2|569| +2450822|7441|2|784| +2450822|7442|2|917| +2450822|7444|2|| +2450822|7447|2|283| +2450822|7448|2|773| +2450822|7450|2|787| +2450822|7453|2|851| +2450822|7454|2|564| +2450822|7456|2|219| +2450822|7459|2|219| +2450822|7460|2|482| +2450822|7462|2|932| +2450822|7465|2|152| +2450822|7466|2|105| +2450822|7468|2|332| +2450822|7471|2|791| +2450822|7472|2|779| +2450822|7474|2|203| +2450822|7477|2|135| +2450822|7478|2|880| +2450822|7480|2|752| +2450822|7483|2|110| +2450822|7484|2|71| +2450822|7486|2|153| +2450822|7489|2|122| +2450822|7490|2|277| +2450822|7492|2|528| +2450822|7495|2|489| +2450822|7496|2|880| +2450822|7498|2|| +2450822|7501|2|977| +2450822|7502|2|756| +2450822|7504|2|260| +2450822|7507|2|753| +2450822|7508|2|498| +2450822|7510|2|| +2450822|7513|2|742| +2450822|7514|2|702| +2450822|7516|2|212| +2450822|7519|2|532| +2450822|7520|2|440| +2450822|7522|2|449| +2450822|7525|2|408| +2450822|7526|2|244| +2450822|7528|2|644| +2450822|7531|2|881| +2450822|7532|2|417| +2450822|7534|2|391| +2450822|7537|2|837| +2450822|7538|2|73| +2450822|7540|2|776| +2450822|7543|2|228| +2450822|7544|2|25| +2450822|7546|2|709| +2450822|7549|2|578| +2450822|7550|2|501| +2450822|7552|2|663| +2450822|7555|2|660| +2450822|7556|2|352| +2450822|7558|2|544| +2450822|7561|2|510| +2450822|7562|2|409| +2450822|7564|2|67| +2450822|7567|2|650| +2450822|7568|2|962| +2450822|7570|2|353| +2450822|7573|2|917| +2450822|7574|2|242| +2450822|7576|2|837| +2450822|7579|2|568| +2450822|7580|2|783| +2450822|7582|2|274| +2450822|7585|2|482| +2450822|7586|2|973| +2450822|7588|2|| +2450822|7591|2|605| +2450822|7592|2|781| +2450822|7594|2|890| +2450822|7597|2|940| +2450822|7598|2|360| +2450822|7600|2|157| +2450822|7603|2|317| +2450822|7604|2|71| +2450822|7606|2|985| +2450822|7609|2|468| +2450822|7610|2|854| +2450822|7612|2|413| +2450822|7615|2|610| +2450822|7616|2|116| +2450822|7618|2|682| +2450822|7621|2|791| +2450822|7622|2|126| +2450822|7624|2|918| +2450822|7627|2|464| +2450822|7628|2|141| +2450822|7630|2|534| +2450822|7633|2|250| +2450822|7634|2|222| +2450822|7636|2|| +2450822|7639|2|844| +2450822|7640|2|547| +2450822|7642|2|705| +2450822|7645|2|441| +2450822|7646|2|49| +2450822|7648|2|508| +2450822|7651|2|52| +2450822|7652|2|901| +2450822|7654|2|59| +2450822|7657|2|267| +2450822|7658|2|394| +2450822|7660|2|544| +2450822|7663|2|620| +2450822|7664|2|438| +2450822|7666|2|950| +2450822|7669|2|499| +2450822|7670|2|121| +2450822|7672|2|289| +2450822|7675|2|578| +2450822|7676|2|263| +2450822|7678|2|54| +2450822|7681|2|885| +2450822|7682|2|728| +2450822|7684|2|219| +2450822|7687|2|61| +2450822|7688|2|872| +2450822|7690|2|731| +2450822|7693|2|103| +2450822|7694|2|692| +2450822|7696|2|620| +2450822|7699|2|404| +2450822|7700|2|135| +2450822|7702|2|385| +2450822|7705|2|702| +2450822|7706|2|583| +2450822|7708|2|265| +2450822|7711|2|934| +2450822|7712|2|587| +2450822|7714|2|769| +2450822|7717|2|842| +2450822|7718|2|14| +2450822|7720|2|370| +2450822|7723|2|20| +2450822|7724|2|966| +2450822|7726|2|100| +2450822|7729|2|425| +2450822|7730|2|293| +2450822|7732|2|255| +2450822|7735|2|644| +2450822|7736|2|438| +2450822|7738|2|442| +2450822|7741|2|974| +2450822|7742|2|315| +2450822|7744|2|493| +2450822|7747|2|351| +2450822|7748|2|632| +2450822|7750|2|| +2450822|7753|2|947| +2450822|7754|2|916| +2450822|7756|2|120| +2450822|7759|2|165| +2450822|7760|2|295| +2450822|7762|2|465| +2450822|7765|2|155| +2450822|7766|2|822| +2450822|7768|2|94| +2450822|7771|2|713| +2450822|7772|2|653| +2450822|7774|2|537| +2450822|7777|2|344| +2450822|7778|2|689| +2450822|7780|2|348| +2450822|7783|2|145| +2450822|7784|2|702| +2450822|7786|2|333| +2450822|7789|2|108| +2450822|7790|2|829| +2450822|7792|2|517| +2450822|7795|2|728| +2450822|7796|2|24| +2450822|7798|2|15| +2450822|7801|2|351| +2450822|7802|2|547| +2450822|7804|2|252| +2450822|7807|2|571| +2450822|7808|2|106| +2450822|7810|2|169| +2450822|7813|2|601| +2450822|7814|2|12| +2450822|7816|2|550| +2450822|7819|2|587| +2450822|7820|2|110| +2450822|7822|2|320| +2450822|7825|2|311| +2450822|7826|2|778| +2450822|7828|2|795| +2450822|7831|2|558| +2450822|7832|2|996| +2450822|7834|2|935| +2450822|7837|2|13| +2450822|7838|2|491| +2450822|7840|2|366| +2450822|7843|2|265| +2450822|7844|2|58| +2450822|7846|2|987| +2450822|7849|2|425| +2450822|7850|2|215| +2450822|7852|2|104| +2450822|7855|2|60| +2450822|7856|2|541| +2450822|7858|2|561| +2450822|7861|2|224| +2450822|7862|2|169| +2450822|7864|2|794| +2450822|7867|2|524| +2450822|7868|2|471| +2450822|7870|2|370| +2450822|7873|2|906| +2450822|7874|2|870| +2450822|7876|2|326| +2450822|7879|2|296| +2450822|7880|2|| +2450822|7882|2|| +2450822|7885|2|473| +2450822|7886|2|494| +2450822|7888|2|845| +2450822|7891|2|613| +2450822|7892|2|552| +2450822|7894|2|923| +2450822|7897|2|255| +2450822|7898|2|147| +2450822|7900|2|156| +2450822|7903|2|755| +2450822|7904|2|472| +2450822|7906|2|566| +2450822|7909|2|99| +2450822|7910|2|129| +2450822|7912|2|7| +2450822|7915|2|902| +2450822|7916|2|846| +2450822|7918|2|140| +2450822|7921|2|444| +2450822|7922|2|632| +2450822|7924|2|887| +2450822|7927|2|396| +2450822|7928|2|300| +2450822|7930|2|93| +2450822|7933|2|272| +2450822|7934|2|639| +2450822|7936|2|580| +2450822|7939|2|576| +2450822|7940|2|203| +2450822|7942|2|426| +2450822|7945|2|845| +2450822|7946|2|396| +2450822|7948|2|172| +2450822|7951|2|649| +2450822|7952|2|268| +2450822|7954|2|| +2450822|7957|2|404| +2450822|7958|2|179| +2450822|7960|2|95| +2450822|7963|2|253| +2450822|7964|2|932| +2450822|7966|2|846| +2450822|7969|2|79| +2450822|7970|2|293| +2450822|7972|2|572| +2450822|7975|2|650| +2450822|7976|2|971| +2450822|7978|2|265| +2450822|7981|2|336| +2450822|7982|2|| +2450822|7984|2|300| +2450822|7987|2|573| +2450822|7988|2|753| +2450822|7990|2|661| +2450822|7993|2|534| +2450822|7994|2|112| +2450822|7996|2|271| +2450822|7999|2|36| +2450822|8000|2|776| +2450822|8002|2|717| +2450822|8005|2|785| +2450822|8006|2|11| +2450822|8008|2|480| +2450822|8011|2|56| +2450822|8012|2|26| +2450822|8014|2|51| +2450822|8017|2|771| +2450822|8018|2|150| +2450822|8020|2|825| +2450822|8023|2|41| +2450822|8024|2|669| +2450822|8026|2|990| +2450822|8029|2|322| +2450822|8030|2|376| +2450822|8032|2|383| +2450822|8035|2|732| +2450822|8036|2|346| +2450822|8038|2|316| +2450822|8041|2|893| +2450822|8042|2|322| +2450822|8044|2|836| +2450822|8047|2|493| +2450822|8048|2|593| +2450822|8050|2|468| +2450822|8053|2|195| +2450822|8054|2|337| +2450822|8056|2|340| +2450822|8059|2|662| +2450822|8060|2|569| +2450822|8062|2|157| +2450822|8065|2|297| +2450822|8066|2|501| +2450822|8068|2|917| +2450822|8071|2|272| +2450822|8072|2|902| +2450822|8074|2|331| +2450822|8077|2|714| +2450822|8078|2|664| +2450822|8080|2|| +2450822|8083|2|832| +2450822|8084|2|741| +2450822|8086|2|679| +2450822|8089|2|| +2450822|8090|2|529| +2450822|8092|2|763| +2450822|8095|2|307| +2450822|8096|2|464| +2450822|8098|2|712| +2450822|8101|2|13| +2450822|8102|2|| +2450822|8104|2|983| +2450822|8107|2|585| +2450822|8108|2|58| +2450822|8110|2|438| +2450822|8113|2|251| +2450822|8114|2|33| +2450822|8116|2|478| +2450822|8119|2|484| +2450822|8120|2|67| +2450822|8122|2|| +2450822|8125|2|757| +2450822|8126|2|133| +2450822|8128|2|977| +2450822|8131|2|26| +2450822|8132|2|805| +2450822|8134|2|160| +2450822|8137|2|497| +2450822|8138|2|13| +2450822|8140|2|690| +2450822|8143|2|443| +2450822|8144|2|| +2450822|8146|2|959| +2450822|8149|2|682| +2450822|8150|2|64| +2450822|8152|2|363| +2450822|8155|2|349| +2450822|8156|2|426| +2450822|8158|2|649| +2450822|8161|2|578| +2450822|8162|2|493| +2450822|8164|2|262| +2450822|8167|2|313| +2450822|8168|2|262| +2450822|8170|2|282| +2450822|8173|2|904| +2450822|8174|2|790| +2450822|8176|2|575| +2450822|8179|2|261| +2450822|8180|2|414| +2450822|8182|2|682| +2450822|8185|2|650| +2450822|8186|2|261| +2450822|8188|2|417| +2450822|8191|2|393| +2450822|8192|2|37| +2450822|8194|2|634| +2450822|8197|2|907| +2450822|8198|2|698| +2450822|8200|2|862| +2450822|8203|2|174| +2450822|8204|2|5| +2450822|8206|2|978| +2450822|8209|2|959| +2450822|8210|2|476| +2450822|8212|2|220| +2450822|8215|2|166| +2450822|8216|2|918| +2450822|8218|2|808| +2450822|8221|2|93| +2450822|8222|2|283| +2450822|8224|2|505| +2450822|8227|2|331| +2450822|8228|2|613| +2450822|8230|2|847| +2450822|8233|2|58| +2450822|8234|2|309| +2450822|8236|2|900| +2450822|8239|2|944| +2450822|8240|2|338| +2450822|8242|2|78| +2450822|8245|2|948| +2450822|8246|2|145| +2450822|8248|2|488| +2450822|8251|2|466| +2450822|8252|2|795| +2450822|8254|2|576| +2450822|8257|2|358| +2450822|8258|2|616| +2450822|8260|2|582| +2450822|8263|2|438| +2450822|8264|2|712| +2450822|8266|2|304| +2450822|8269|2|635| +2450822|8270|2|995| +2450822|8272|2|690| +2450822|8275|2|675| +2450822|8276|2|794| +2450822|8278|2|388| +2450822|8281|2|21| +2450822|8282|2|356| +2450822|8284|2|| +2450822|8287|2|9| +2450822|8288|2|6| +2450822|8290|2|501| +2450822|8293|2|| +2450822|8294|2|371| +2450822|8296|2|767| +2450822|8299|2|742| +2450822|8300|2|593| +2450822|8302|2|196| +2450822|8305|2|163| +2450822|8306|2|698| +2450822|8308|2|911| +2450822|8311|2|302| +2450822|8312|2|987| +2450822|8314|2|884| +2450822|8317|2|829| +2450822|8318|2|74| +2450822|8320|2|505| +2450822|8323|2|601| +2450822|8324|2|85| +2450822|8326|2|212| +2450822|8329|2|774| +2450822|8330|2|78| +2450822|8332|2|110| +2450822|8335|2|15| +2450822|8336|2|26| +2450822|8338|2|| +2450822|8341|2|841| +2450822|8342|2|149| +2450822|8344|2|268| +2450822|8347|2|801| +2450822|8348|2|843| +2450822|8350|2|182| +2450822|8353|2|216| +2450822|8354|2|435| +2450822|8356|2|570| +2450822|8359|2|| +2450822|8360|2|688| +2450822|8362|2|224| +2450822|8365|2|561| +2450822|8366|2|525| +2450822|8368|2|273| +2450822|8371|2|535| +2450822|8372|2|61| +2450822|8374|2|686| +2450822|8377|2|874| +2450822|8378|2|576| +2450822|8380|2|456| +2450822|8383|2|452| +2450822|8384|2|551| +2450822|8386|2|195| +2450822|8389|2|828| +2450822|8390|2|169| +2450822|8392|2|677| +2450822|8395|2|496| +2450822|8396|2|722| +2450822|8398|2|644| +2450822|8401|2|826| +2450822|8402|2|917| +2450822|8404|2|530| +2450822|8407|2|601| +2450822|8408|2|547| +2450822|8410|2|45| +2450822|8413|2|602| +2450822|8414|2|652| +2450822|8416|2|| +2450822|8419|2|641| +2450822|8420|2|| +2450822|8422|2|256| +2450822|8425|2|785| +2450822|8426|2|988| +2450822|8428|2|532| +2450822|8431|2|871| +2450822|8432|2|| +2450822|8434|2|3| +2450822|8437|2|496| +2450822|8438|2|910| +2450822|8440|2|18| +2450822|8443|2|| +2450822|8444|2|822| +2450822|8446|2|472| +2450822|8449|2|| +2450822|8450|2|556| +2450822|8452|2|785| +2450822|8455|2|116| +2450822|8456|2|985| +2450822|8458|2|452| +2450822|8461|2|398| +2450822|8462|2|625| +2450822|8464|2|279| +2450822|8467|2|153| +2450822|8468|2|823| +2450822|8470|2|837| +2450822|8473|2|231| +2450822|8474|2|297| +2450822|8476|2|105| +2450822|8479|2|499| +2450822|8480|2|247| +2450822|8482|2|60| +2450822|8485|2|| +2450822|8486|2|802| +2450822|8488|2|695| +2450822|8491|2|974| +2450822|8492|2|783| +2450822|8494|2|78| +2450822|8497|2|638| +2450822|8498|2|432| +2450822|8500|2|10| +2450822|8503|2|333| +2450822|8504|2|301| +2450822|8506|2|547| +2450822|8509|2|428| +2450822|8510|2|676| +2450822|8512|2|192| +2450822|8515|2|495| +2450822|8516|2|696| +2450822|8518|2|331| +2450822|8521|2|980| +2450822|8522|2|| +2450822|8524|2|862| +2450822|8527|2|208| +2450822|8528|2|815| +2450822|8530|2|213| +2450822|8533|2|775| +2450822|8534|2|374| +2450822|8536|2|839| +2450822|8539|2|404| +2450822|8540|2|994| +2450822|8542|2|155| +2450822|8545|2|643| +2450822|8546|2|663| +2450822|8548|2|302| +2450822|8551|2|320| +2450822|8552|2|566| +2450822|8554|2|155| +2450822|8557|2|271| +2450822|8558|2|314| +2450822|8560|2|138| +2450822|8563|2|441| +2450822|8564|2|106| +2450822|8566|2|621| +2450822|8569|2|328| +2450822|8570|2|670| +2450822|8572|2|885| +2450822|8575|2|831| +2450822|8576|2|871| +2450822|8578|2|22| +2450822|8581|2|983| +2450822|8582|2|278| +2450822|8584|2|429| +2450822|8587|2|662| +2450822|8588|2|755| +2450822|8590|2|455| +2450822|8593|2|278| +2450822|8594|2|841| +2450822|8596|2|387| +2450822|8599|2|473| +2450822|8600|2|816| +2450822|8602|2|984| +2450822|8605|2|684| +2450822|8606|2|32| +2450822|8608|2|613| +2450822|8611|2|732| +2450822|8612|2|300| +2450822|8614|2|596| +2450822|8617|2|434| +2450822|8618|2|388| +2450822|8620|2|880| +2450822|8623|2|465| +2450822|8624|2|785| +2450822|8626|2|744| +2450822|8629|2|639| +2450822|8630|2|| +2450822|8632|2|515| +2450822|8635|2|219| +2450822|8636|2|| +2450822|8638|2|907| +2450822|8641|2|| +2450822|8642|2|689| +2450822|8644|2|541| +2450822|8647|2|677| +2450822|8648|2|482| +2450822|8650|2|697| +2450822|8653|2|6| +2450822|8654|2|604| +2450822|8656|2|817| +2450822|8659|2|| +2450822|8660|2|921| +2450822|8662|2|282| +2450822|8665|2|387| +2450822|8666|2|633| +2450822|8668|2|308| +2450822|8671|2|| +2450822|8672|2|316| +2450822|8674|2|403| +2450822|8677|2|596| +2450822|8678|2|489| +2450822|8680|2|755| +2450822|8683|2|512| +2450822|8684|2|675| +2450822|8686|2|390| +2450822|8689|2|799| +2450822|8690|2|287| +2450822|8692|2|422| +2450822|8695|2|398| +2450822|8696|2|| +2450822|8698|2|712| +2450822|8701|2|263| +2450822|8702|2|625| +2450822|8704|2|345| +2450822|8707|2|122| +2450822|8708|2|217| +2450822|8710|2|494| +2450822|8713|2|412| +2450822|8714|2|558| +2450822|8716|2|381| +2450822|8719|2|957| +2450822|8720|2|184| +2450822|8722|2|501| +2450822|8725|2|265| +2450822|8726|2|43| +2450822|8728|2|972| +2450822|8731|2|770| +2450822|8732|2|871| +2450822|8734|2|438| +2450822|8737|2|876| +2450822|8738|2|839| +2450822|8740|2|34| +2450822|8743|2|233| +2450822|8744|2|277| +2450822|8746|2|536| +2450822|8749|2|893| +2450822|8750|2|| +2450822|8752|2|584| +2450822|8755|2|686| +2450822|8756|2|975| +2450822|8758|2|637| +2450822|8761|2|501| +2450822|8762|2|812| +2450822|8764|2|630| +2450822|8767|2|737| +2450822|8768|2|223| +2450822|8770|2|232| +2450822|8773|2|546| +2450822|8774|2|399| +2450822|8776|2|30| +2450822|8779|2|887| +2450822|8780|2|620| +2450822|8782|2|847| +2450822|8785|2|490| +2450822|8786|2|923| +2450822|8788|2|866| +2450822|8791|2|936| +2450822|8792|2|505| +2450822|8794|2|508| +2450822|8797|2|120| +2450822|8798|2|888| +2450822|8800|2|410| +2450822|8803|2|30| +2450822|8804|2|824| +2450822|8806|2|499| +2450822|8809|2|235| +2450822|8810|2|559| +2450822|8812|2|857| +2450822|8815|2|927| +2450822|8816|2|843| +2450822|8818|2|178| +2450822|8821|2|77| +2450822|8822|2|909| +2450822|8824|2|381| +2450822|8827|2|82| +2450822|8828|2|611| +2450822|8830|2|524| +2450822|8833|2|510| +2450822|8834|2|409| +2450822|8836|2|39| +2450822|8839|2|318| +2450822|8840|2|374| +2450822|8842|2|894| +2450822|8845|2|722| +2450822|8846|2|286| +2450822|8848|2|164| +2450822|8851|2|101| +2450822|8852|2|466| +2450822|8854|2|28| +2450822|8857|2|203| +2450822|8858|2|267| +2450822|8860|2|323| +2450822|8863|2|| +2450822|8864|2|72| +2450822|8866|2|817| +2450822|8869|2|883| +2450822|8870|2|167| +2450822|8872|2|114| +2450822|8875|2|534| +2450822|8876|2|206| +2450822|8878|2|892| +2450822|8881|2|398| +2450822|8882|2|907| +2450822|8884|2|882| +2450822|8887|2|641| +2450822|8888|2|873| +2450822|8890|2|520| +2450822|8893|2|482| +2450822|8894|2|646| +2450822|8896|2|599| +2450822|8899|2|470| +2450822|8900|2|56| +2450822|8902|2|445| +2450822|8905|2|| +2450822|8906|2|191| +2450822|8908|2|798| +2450822|8911|2|637| +2450822|8912|2|369| +2450822|8914|2|451| +2450822|8917|2|636| +2450822|8918|2|581| +2450822|8920|2|763| +2450822|8923|2|413| +2450822|8924|2|505| +2450822|8926|2|503| +2450822|8929|2|626| +2450822|8930|2|478| +2450822|8932|2|682| +2450822|8935|2|134| +2450822|8936|2|925| +2450822|8938|2|660| +2450822|8941|2|409| +2450822|8942|2|242| +2450822|8944|2|107| +2450822|8947|2|| +2450822|8948|2|| +2450822|8950|2|856| +2450822|8953|2|442| +2450822|8954|2|905| +2450822|8956|2|428| +2450822|8959|2|827| +2450822|8960|2|425| +2450822|8962|2|784| +2450822|8965|2|538| +2450822|8966|2|389| +2450822|8968|2|119| +2450822|8971|2|567| +2450822|8972|2|59| +2450822|8974|2|679| +2450822|8977|2|181| +2450822|8978|2|964| +2450822|8980|2|309| +2450822|8983|2|738| +2450822|8984|2|| +2450822|8986|2|527| +2450822|8989|2|757| +2450822|8990|2|398| +2450822|8992|2|61| +2450822|8995|2|738| +2450822|8996|2|312| +2450822|8998|2|687| +2450822|9001|2|268| +2450822|9002|2|917| +2450822|9004|2|377| +2450822|9007|2|847| +2450822|9008|2|972| +2450822|9010|2|325| +2450822|9013|2|516| +2450822|9014|2|238| +2450822|9016|2|636| +2450822|9019|2|594| +2450822|9020|2|392| +2450822|9022|2|311| +2450822|9025|2|494| +2450822|9026|2|837| +2450822|9028|2|413| +2450822|9031|2|768| +2450822|9032|2|238| +2450822|9034|2|675| +2450822|9037|2|798| +2450822|9038|2|744| +2450822|9040|2|916| +2450822|9043|2|137| +2450822|9044|2|971| +2450822|9046|2|801| +2450822|9049|2|574| +2450822|9050|2|98| +2450822|9052|2|| +2450822|9055|2|277| +2450822|9056|2|492| +2450822|9058|2|221| +2450822|9061|2|630| +2450822|9062|2|753| +2450822|9064|2|51| +2450822|9067|2|873| +2450822|9068|2|796| +2450822|9070|2|508| +2450822|9073|2|500| +2450822|9074|2|644| +2450822|9076|2|871| +2450822|9079|2|| +2450822|9080|2|915| +2450822|9082|2|115| +2450822|9085|2|732| +2450822|9086|2|4| +2450822|9088|2|621| +2450822|9091|2|126| +2450822|9092|2|892| +2450822|9094|2|804| +2450822|9097|2|454| +2450822|9098|2|312| +2450822|9100|2|825| +2450822|9103|2|925| +2450822|9104|2|24| +2450822|9106|2|558| +2450822|9109|2|754| +2450822|9110|2|620| +2450822|9112|2|890| +2450822|9115|2|648| +2450822|9116|2|453| +2450822|9118|2|398| +2450822|9121|2|817| +2450822|9122|2|402| +2450822|9124|2|269| +2450822|9127|2|742| +2450822|9128|2|622| +2450822|9130|2|213| +2450822|9133|2|568| +2450822|9134|2|968| +2450822|9136|2|642| +2450822|9139|2|174| +2450822|9140|2|374| +2450822|9142|2|172| +2450822|9145|2|833| +2450822|9146|2|732| +2450822|9148|2|928| +2450822|9151|2|897| +2450822|9152|2|339| +2450822|9154|2|916| +2450822|9157|2|70| +2450822|9158|2|74| +2450822|9160|2|| +2450822|9163|2|519| +2450822|9164|2|490| +2450822|9166|2|690| +2450822|9169|2|| +2450822|9170|2|244| +2450822|9172|2|615| +2450822|9175|2|812| +2450822|9176|2|121| +2450822|9178|2|298| +2450822|9181|2|920| +2450822|9182|2|757| +2450822|9184|2|629| +2450822|9187|2|894| +2450822|9188|2|49| +2450822|9190|2|497| +2450822|9193|2|856| +2450822|9194|2|49| +2450822|9196|2|377| +2450822|9199|2|| +2450822|9200|2|936| +2450822|9202|2|79| +2450822|9205|2|912| +2450822|9206|2|767| +2450822|9208|2|922| +2450822|9211|2|314| +2450822|9212|2|262| +2450822|9214|2|456| +2450822|9217|2|90| +2450822|9218|2|864| +2450822|9220|2|557| +2450822|9223|2|83| +2450822|9224|2|574| +2450822|9226|2|27| +2450822|9229|2|172| +2450822|9230|2|1000| +2450822|9232|2|538| +2450822|9235|2|389| +2450822|9236|2|617| +2450822|9238|2|62| +2450822|9241|2|100| +2450822|9242|2|897| +2450822|9244|2|453| +2450822|9247|2|872| +2450822|9248|2|568| +2450822|9250|2|563| +2450822|9253|2|411| +2450822|9254|2|900| +2450822|9256|2|305| +2450822|9259|2|14| +2450822|9260|2|411| +2450822|9262|2|329| +2450822|9265|2|550| +2450822|9266|2|770| +2450822|9268|2|884| +2450822|9271|2|204| +2450822|9272|2|47| +2450822|9274|2|707| +2450822|9277|2|185| +2450822|9278|2|4| +2450822|9280|2|840| +2450822|9283|2|32| +2450822|9284|2|78| +2450822|9286|2|587| +2450822|9289|2|260| +2450822|9290|2|| +2450822|9292|2|792| +2450822|9295|2|633| +2450822|9296|2|836| +2450822|9298|2|441| +2450822|9301|2|524| +2450822|9302|2|376| +2450822|9304|2|329| +2450822|9307|2|184| +2450822|9308|2|726| +2450822|9310|2|| +2450822|9313|2|782| +2450822|9314|2|469| +2450822|9316|2|77| +2450822|9319|2|649| +2450822|9320|2|907| +2450822|9322|2|583| +2450822|9325|2|841| +2450822|9326|2|718| +2450822|9328|2|726| +2450822|9331|2|227| +2450822|9332|2|695| +2450822|9334|2|635| +2450822|9337|2|387| +2450822|9338|2|397| +2450822|9340|2|968| +2450822|9343|2|996| +2450822|9344|2|| +2450822|9346|2|427| +2450822|9349|2|21| +2450822|9350|2|559| +2450822|9352|2|460| +2450822|9355|2|121| +2450822|9356|2|966| +2450822|9358|2|380| +2450822|9361|2|114| +2450822|9362|2|264| +2450822|9364|2|565| +2450822|9367|2|245| +2450822|9368|2|224| +2450822|9370|2|799| +2450822|9373|2|720| +2450822|9374|2|| +2450822|9376|2|| +2450822|9379|2|745| +2450822|9380|2|789| +2450822|9382|2|532| +2450822|9385|2|239| +2450822|9386|2|320| +2450822|9388|2|883| +2450822|9391|2|268| +2450822|9392|2|101| +2450822|9394|2|225| +2450822|9397|2|141| +2450822|9398|2|107| +2450822|9400|2|| +2450822|9403|2|493| +2450822|9404|2|139| +2450822|9406|2|874| +2450822|9409|2|789| +2450822|9410|2|707| +2450822|9412|2|828| +2450822|9415|2|257| +2450822|9416|2|167| +2450822|9418|2|229| +2450822|9421|2|97| +2450822|9422|2|354| +2450822|9424|2|| +2450822|9427|2|81| +2450822|9428|2|440| +2450822|9430|2|325| +2450822|9433|2|421| +2450822|9434|2|324| +2450822|9436|2|858| +2450822|9439|2|595| +2450822|9440|2|664| +2450822|9442|2|91| +2450822|9445|2|575| +2450822|9446|2|11| +2450822|9448|2|517| +2450822|9451|2|262| +2450822|9452|2|734| +2450822|9454|2|252| +2450822|9457|2|291| +2450822|9458|2|309| +2450822|9460|2|193| +2450822|9463|2|466| +2450822|9464|2|618| +2450822|9466|2|596| +2450822|9469|2|906| +2450822|9470|2|569| +2450822|9472|2|73| +2450822|9475|2|528| +2450822|9476|2|789| +2450822|9478|2|737| +2450822|9481|2|754| +2450822|9482|2|824| +2450822|9484|2|681| +2450822|9487|2|636| +2450822|9488|2|252| +2450822|9490|2|169| +2450822|9493|2|344| +2450822|9494|2|761| +2450822|9496|2|965| +2450822|9499|2|2| +2450822|9500|2|124| +2450822|9502|2|517| +2450822|9505|2|56| +2450822|9506|2|941| +2450822|9508|2|861| +2450822|9511|2|475| +2450822|9512|2|991| +2450822|9514|2|943| +2450822|9517|2|250| +2450822|9518|2|28| +2450822|9520|2|683| +2450822|9523|2|25| +2450822|9524|2|446| +2450822|9526|2|617| +2450822|9529|2|497| +2450822|9530|2|592| +2450822|9532|2|| +2450822|9535|2|337| +2450822|9536|2|893| +2450822|9538|2|958| +2450822|9541|2|848| +2450822|9542|2|923| +2450822|9544|2|450| +2450822|9547|2|677| +2450822|9548|2|| +2450822|9550|2|997| +2450822|9553|2|301| +2450822|9554|2|349| +2450822|9556|2|504| +2450822|9559|2|347| +2450822|9560|2|886| +2450822|9562|2|933| +2450822|9565|2|573| +2450822|9566|2|866| +2450822|9568|2|836| +2450822|9571|2|202| +2450822|9572|2|16| +2450822|9574|2|46| +2450822|9577|2|927| +2450822|9578|2|96| +2450822|9580|2|| +2450822|9583|2|875| +2450822|9584|2|469| +2450822|9586|2|562| +2450822|9589|2|871| +2450822|9590|2|364| +2450822|9592|2|117| +2450822|9595|2|832| +2450822|9596|2|713| +2450822|9598|2|327| +2450822|9601|2|71| +2450822|9602|2|492| +2450822|9604|2|443| +2450822|9607|2|183| +2450822|9608|2|608| +2450822|9610|2|589| +2450822|9613|2|113| +2450822|9614|2|586| +2450822|9616|2|588| +2450822|9619|2|2| +2450822|9620|2|383| +2450822|9622|2|718| +2450822|9625|2|792| +2450822|9626|2|471| +2450822|9628|2|517| +2450822|9631|2|545| +2450822|9632|2|918| +2450822|9634|2|757| +2450822|9637|2|387| +2450822|9638|2|518| +2450822|9640|2|641| +2450822|9643|2|180| +2450822|9644|2|758| +2450822|9646|2|670| +2450822|9649|2|0| +2450822|9650|2|289| +2450822|9652|2|440| +2450822|9655|2|412| +2450822|9656|2|663| +2450822|9658|2|242| +2450822|9661|2|126| +2450822|9662|2|469| +2450822|9664|2|416| +2450822|9667|2|141| +2450822|9668|2|415| +2450822|9670|2|706| +2450822|9673|2|981| +2450822|9674|2|392| +2450822|9676|2|564| +2450822|9679|2|| +2450822|9680|2|871| +2450822|9682|2|476| +2450822|9685|2|110| +2450822|9686|2|701| +2450822|9688|2|741| +2450822|9691|2|78| +2450822|9692|2|538| +2450822|9694|2|612| +2450822|9697|2|| +2450822|9698|2|506| +2450822|9700|2|406| +2450822|9703|2|831| +2450822|9704|2|473| +2450822|9706|2|544| +2450822|9709|2|580| +2450822|9710|2|413| +2450822|9712|2|413| +2450822|9715|2|110| +2450822|9716|2|909| +2450822|9718|2|687| +2450822|9721|2|238| +2450822|9722|2|454| +2450822|9724|2|802| +2450822|9727|2|729| +2450822|9728|2|531| +2450822|9730|2|603| +2450822|9733|2|399| +2450822|9734|2|55| +2450822|9736|2|147| +2450822|9739|2|185| +2450822|9740|2|| +2450822|9742|2|815| +2450822|9745|2|322| +2450822|9746|2|582| +2450822|9748|2|293| +2450822|9751|2|826| +2450822|9752|2|327| +2450822|9754|2|746| +2450822|9757|2|354| +2450822|9758|2|660| +2450822|9760|2|881| +2450822|9763|2|42| +2450822|9764|2|941| +2450822|9766|2|290| +2450822|9769|2|909| +2450822|9770|2|363| +2450822|9772|2|548| +2450822|9775|2|460| +2450822|9776|2|522| +2450822|9778|2|305| +2450822|9781|2|619| +2450822|9782|2|834| +2450822|9784|2|151| +2450822|9787|2|682| +2450822|9788|2|735| +2450822|9790|2|788| +2450822|9793|2|282| +2450822|9794|2|827| +2450822|9796|2|186| +2450822|9799|2|170| +2450822|9800|2|753| +2450822|9802|2|535| +2450822|9805|2|619| +2450822|9806|2|830| +2450822|9808|2|363| +2450822|9811|2|800| +2450822|9812|2|256| +2450822|9814|2|707| +2450822|9817|2|628| +2450822|9818|2|366| +2450822|9820|2|300| +2450822|9823|2|121| +2450822|9824|2|905| +2450822|9826|2|817| +2450822|9829|2|166| +2450822|9830|2|421| +2450822|9832|2|190| +2450822|9835|2|119| +2450822|9836|2|842| +2450822|9838|2|390| +2450822|9841|2|417| +2450822|9842|2|299| +2450822|9844|2|935| +2450822|9847|2|769| +2450822|9848|2|533| +2450822|9850|2|542| +2450822|9853|2|837| +2450822|9854|2|851| +2450822|9856|2|679| +2450822|9859|2|773| +2450822|9860|2|972| +2450822|9862|2|590| +2450822|9865|2|784| +2450822|9866|2|454| +2450822|9868|2|724| +2450822|9871|2|365| +2450822|9872|2|115| +2450822|9874|2|513| +2450822|9877|2|877| +2450822|9878|2|884| +2450822|9880|2|677| +2450822|9883|2|214| +2450822|9884|2|1000| +2450822|9886|2|190| +2450822|9889|2|855| +2450822|9890|2|480| +2450822|9892|2|| +2450822|9895|2|501| +2450822|9896|2|257| +2450822|9898|2|110| +2450822|9901|2|568| +2450822|9902|2|730| +2450822|9904|2|793| +2450822|9907|2|609| +2450822|9908|2|752| +2450822|9910|2|149| +2450822|9913|2|120| +2450822|9914|2|352| +2450822|9916|2|469| +2450822|9919|2|324| +2450822|9920|2|205| +2450822|9922|2|793| +2450822|9925|2|417| +2450822|9926|2|234| +2450822|9928|2|640| +2450822|9931|2|719| +2450822|9932|2|464| +2450822|9934|2|54| +2450822|9937|2|158| +2450822|9938|2|173| +2450822|9940|2|| +2450822|9943|2|953| +2450822|9944|2|20| +2450822|9946|2|857| +2450822|9949|2|399| +2450822|9950|2|899| +2450822|9952|2|579| +2450822|9955|2|294| +2450822|9956|2|30| +2450822|9958|2|793| +2450822|9961|2|379| +2450822|9962|2|882| +2450822|9964|2|733| +2450822|9967|2|644| +2450822|9968|2|697| +2450822|9970|2|872| +2450822|9973|2|411| +2450822|9974|2|654| +2450822|9976|2|398| +2450822|9979|2|844| +2450822|9980|2|928| +2450822|9982|2|| +2450822|9985|2|652| +2450822|9986|2|562| +2450822|9988|2|751| +2450822|9991|2|553| +2450822|9992|2|108| +2450822|9994|2|855| +2450822|9997|2|46| +2450822|9998|2|675| +2450822|10000|2|704| +2450822|10003|2|893| +2450822|10004|2|157| +2450822|10006|2|259| +2450822|10009|2|295| +2450822|10010|2|679| +2450822|10012|2|622| +2450822|10015|2|474| +2450822|10016|2|673| +2450822|10018|2|373| +2450822|10021|2|832| +2450822|10022|2|965| +2450822|10024|2|965| +2450822|10027|2|989| +2450822|10028|2|559| +2450822|10030|2|| +2450822|10033|2|659| +2450822|10034|2|177| +2450822|10036|2|752| +2450822|10039|2|744| +2450822|10040|2|| +2450822|10042|2|| +2450822|10045|2|40| +2450822|10046|2|602| +2450822|10048|2|11| +2450822|10051|2|| +2450822|10052|2|870| +2450822|10054|2|597| +2450822|10057|2|328| +2450822|10058|2|242| +2450822|10060|2|253| +2450822|10063|2|939| +2450822|10064|2|787| +2450822|10066|2|835| +2450822|10069|2|540| +2450822|10070|2|613| +2450822|10072|2|125| +2450822|10075|2|396| +2450822|10076|2|385| +2450822|10078|2|191| +2450822|10081|2|117| +2450822|10082|2|161| +2450822|10084|2|875| +2450822|10087|2|116| +2450822|10088|2|859| +2450822|10090|2|109| +2450822|10093|2|645| +2450822|10094|2|579| +2450822|10096|2|71| +2450822|10099|2|| +2450822|10100|2|270| +2450822|10102|2|55| +2450822|10105|2|35| +2450822|10106|2|59| +2450822|10108|2|792| +2450822|10111|2|88| +2450822|10112|2|165| +2450822|10114|2|52| +2450822|10117|2|433| +2450822|10118|2|518| +2450822|10120|2|570| +2450822|10123|2|107| +2450822|10124|2|805| +2450822|10126|2|762| +2450822|10129|2|705| +2450822|10130|2|703| +2450822|10132|2|334| +2450822|10135|2|945| +2450822|10136|2|219| +2450822|10138|2|472| +2450822|10141|2|959| +2450822|10142|2|172| +2450822|10144|2|777| +2450822|10147|2|676| +2450822|10148|2|111| +2450822|10150|2|571| +2450822|10153|2|785| +2450822|10154|2|472| +2450822|10156|2|37| +2450822|10159|2|884| +2450822|10160|2|545| +2450822|10162|2|87| +2450822|10165|2|48| +2450822|10166|2|538| +2450822|10168|2|238| +2450822|10171|2|358| +2450822|10172|2|316| +2450822|10174|2|530| +2450822|10177|2|125| +2450822|10178|2|52| +2450822|10180|2|890| +2450822|10183|2|| +2450822|10184|2|634| +2450822|10186|2|636| +2450822|10189|2|23| +2450822|10190|2|263| +2450822|10192|2|789| +2450822|10195|2|970| +2450822|10196|2|517| +2450822|10198|2|279| +2450822|10201|2|339| +2450822|10202|2|515| +2450822|10204|2|78| +2450822|10207|2|444| +2450822|10208|2|780| +2450822|10210|2|743| +2450822|10213|2|525| +2450822|10214|2|769| +2450822|10216|2|655| +2450822|10219|2|974| +2450822|10220|2|147| +2450822|10222|2|711| +2450822|10225|2|195| +2450822|10226|2|46| +2450822|10228|2|733| +2450822|10231|2|75| +2450822|10232|2|357| +2450822|10234|2|932| +2450822|10237|2|228| +2450822|10238|2|214| +2450822|10240|2|837| +2450822|10243|2|645| +2450822|10244|2|403| +2450822|10246|2|288| +2450822|10249|2|343| +2450822|10250|2|900| +2450822|10252|2|96| +2450822|10255|2|294| +2450822|10256|2|744| +2450822|10258|2|891| +2450822|10261|2|272| +2450822|10262|2|294| +2450822|10264|2|972| +2450822|10267|2|134| +2450822|10268|2|448| +2450822|10270|2|122| +2450822|10273|2|570| +2450822|10274|2|837| +2450822|10276|2|724| +2450822|10279|2|372| +2450822|10280|2|133| +2450822|10282|2|886| +2450822|10285|2|304| +2450822|10286|2|121| +2450822|10288|2|245| +2450822|10291|2|387| +2450822|10292|2|957| +2450822|10294|2|9| +2450822|10297|2|985| +2450822|10298|2|227| +2450822|10300|2|904| +2450822|10303|2|695| +2450822|10304|2|654| +2450822|10306|2|91| +2450822|10309|2|| +2450822|10310|2|| +2450822|10312|2|959| +2450822|10315|2|7| +2450822|10316|2|674| +2450822|10318|2|130| +2450822|10321|2|48| +2450822|10322|2|86| +2450822|10324|2|646| +2450822|10327|2|515| +2450822|10328|2|962| +2450822|10330|2|552| +2450822|10333|2|926| +2450822|10334|2|511| +2450822|10336|2|298| +2450822|10339|2|182| +2450822|10340|2|739| +2450822|10342|2|136| +2450822|10345|2|143| +2450822|10346|2|813| +2450822|10348|2|| +2450822|10351|2|998| +2450822|10352|2|616| +2450822|10354|2|45| +2450822|10357|2|179| +2450822|10358|2|417| +2450822|10360|2|893| +2450822|10363|2|629| +2450822|10364|2|514| +2450822|10366|2|539| +2450822|10369|2|168| +2450822|10370|2|862| +2450822|10372|2|771| +2450822|10375|2|558| +2450822|10376|2|929| +2450822|10378|2|886| +2450822|10381|2|945| +2450822|10382|2|284| +2450822|10384|2|623| +2450822|10387|2|199| +2450822|10388|2|439| +2450822|10390|2|964| +2450822|10393|2|714| +2450822|10394|2|| +2450822|10396|2|688| +2450822|10399|2|240| +2450822|10400|2|369| +2450822|10402|2|477| +2450822|10405|2|27| +2450822|10406|2|428| +2450822|10408|2|58| +2450822|10411|2|167| +2450822|10412|2|167| +2450822|10414|2|475| +2450822|10417|2|593| +2450822|10418|2|| +2450822|10420|2|153| +2450822|10423|2|590| +2450822|10424|2|951| +2450822|10426|2|641| +2450822|10429|2|404| +2450822|10430|2|434| +2450822|10432|2|455| +2450822|10435|2|767| +2450822|10436|2|| +2450822|10438|2|847| +2450822|10441|2|446| +2450822|10442|2|653| +2450822|10444|2|561| +2450822|10447|2|430| +2450822|10448|2|538| +2450822|10450|2|442| +2450822|10453|2|598| +2450822|10454|2|458| +2450822|10456|2|875| +2450822|10459|2|942| +2450822|10460|2|749| +2450822|10462|2|852| +2450822|10465|2|556| +2450822|10466|2|183| +2450822|10468|2|253| +2450822|10471|2|976| +2450822|10472|2|| +2450822|10474|2|570| +2450822|10477|2|939| +2450822|10478|2|616| +2450822|10480|2|694| +2450822|10483|2|442| +2450822|10484|2|738| +2450822|10486|2|571| +2450822|10489|2|873| +2450822|10490|2|619| +2450822|10492|2|27| +2450822|10495|2|135| +2450822|10496|2|838| +2450822|10498|2|531| +2450822|10501|2|355| +2450822|10502|2|412| +2450822|10504|2|894| +2450822|10507|2|526| +2450822|10508|2|989| +2450822|10510|2|256| +2450822|10513|2|54| +2450822|10514|2|136| +2450822|10516|2|200| +2450822|10519|2|254| +2450822|10520|2|622| +2450822|10522|2|431| +2450822|10525|2|847| +2450822|10526|2|316| +2450822|10528|2|327| +2450822|10531|2|924| +2450822|10532|2|177| +2450822|10534|2|659| +2450822|10537|2|664| +2450822|10538|2|593| +2450822|10540|2|359| +2450822|10543|2|996| +2450822|10544|2|405| +2450822|10546|2|150| +2450822|10549|2|815| +2450822|10550|2|857| +2450822|10552|2|523| +2450822|10555|2|719| +2450822|10556|2|385| +2450822|10558|2|35| +2450822|10561|2|451| +2450822|10562|2|924| +2450822|10564|2|860| +2450822|10567|2|| +2450822|10568|2|27| +2450822|10570|2|995| +2450822|10573|2|896| +2450822|10574|2|195| +2450822|10576|2|| +2450822|10579|2|139| +2450822|10580|2|52| +2450822|10582|2|256| +2450822|10585|2|638| +2450822|10586|2|801| +2450822|10588|2|915| +2450822|10591|2|463| +2450822|10592|2|803| +2450822|10594|2|123| +2450822|10597|2|| +2450822|10598|2|864| +2450822|10600|2|613| +2450822|10603|2|537| +2450822|10604|2|308| +2450822|10606|2|809| +2450822|10609|2|664| +2450822|10610|2|274| +2450822|10612|2|9| +2450822|10615|2|749| +2450822|10616|2|73| +2450822|10618|2|787| +2450822|10621|2|411| +2450822|10622|2|844| +2450822|10624|2|707| +2450822|10627|2|949| +2450822|10628|2|12| +2450822|10630|2|241| +2450822|10633|2|141| +2450822|10634|2|226| +2450822|10636|2|355| +2450822|10639|2|510| +2450822|10640|2|361| +2450822|10642|2|866| +2450822|10645|2|697| +2450822|10646|2|648| +2450822|10648|2|514| +2450822|10651|2|255| +2450822|10652|2|432| +2450822|10654|2|138| +2450822|10657|2|129| +2450822|10658|2|909| +2450822|10660|2|694| +2450822|10663|2|368| +2450822|10664|2|960| +2450822|10666|2|791| +2450822|10669|2|578| +2450822|10670|2|616| +2450822|10672|2|806| +2450822|10675|2|420| +2450822|10676|2|51| +2450822|10678|2|408| +2450822|10681|2|498| +2450822|10682|2|645| +2450822|10684|2|141| +2450822|10687|2|| +2450822|10688|2|416| +2450822|10690|2|822| +2450822|10693|2|793| +2450822|10694|2|564| +2450822|10696|2|695| +2450822|10699|2|2| +2450822|10700|2|614| +2450822|10702|2|746| +2450822|10705|2|750| +2450822|10706|2|365| +2450822|10708|2|763| +2450822|10711|2|749| +2450822|10712|2|576| +2450822|10714|2|607| +2450822|10717|2|746| +2450822|10718|2|295| +2450822|10720|2|591| +2450822|10723|2|298| +2450822|10724|2|492| +2450822|10726|2|386| +2450822|10729|2|408| +2450822|10730|2|35| +2450822|10732|2|| +2450822|10735|2|387| +2450822|10736|2|267| +2450822|10738|2|808| +2450822|10741|2|508| +2450822|10742|2|784| +2450822|10744|2|26| +2450822|10747|2|67| +2450822|10748|2|853| +2450822|10750|2|943| +2450822|10753|2|200| +2450822|10754|2|605| +2450822|10756|2|28| +2450822|10759|2|358| +2450822|10760|2|668| +2450822|10762|2|758| +2450822|10765|2|472| +2450822|10766|2|315| +2450822|10768|2|302| +2450822|10771|2|676| +2450822|10772|2|279| +2450822|10774|2|652| +2450822|10777|2|204| +2450822|10778|2|203| +2450822|10780|2|709| +2450822|10783|2|976| +2450822|10784|2|796| +2450822|10786|2|939| +2450822|10789|2|994| +2450822|10790|2|469| +2450822|10792|2|151| +2450822|10795|2|493| +2450822|10796|2|717| +2450822|10798|2|36| +2450822|10801|2|746| +2450822|10802|2|| +2450822|10804|2|576| +2450822|10807|2|680| +2450822|10808|2|990| +2450822|10810|2|581| +2450822|10813|2|153| +2450822|10814|2|334| +2450822|10816|2|634| +2450822|10819|2|759| +2450822|10820|2|104| +2450822|10822|2|891| +2450822|10825|2|480| +2450822|10826|2|525| +2450822|10828|2|901| +2450822|10831|2|859| +2450822|10832|2|409| +2450822|10834|2|62| +2450822|10837|2|433| +2450822|10838|2|105| +2450822|10840|2|714| +2450822|10843|2|932| +2450822|10844|2|927| +2450822|10846|2|718| +2450822|10849|2|595| +2450822|10850|2|585| +2450822|10852|2|886| +2450822|10855|2|444| +2450822|10856|2|692| +2450822|10858|2|940| +2450822|10861|2|693| +2450822|10862|2|601| +2450822|10864|2|357| +2450822|10867|2|625| +2450822|10868|2|880| +2450822|10870|2|266| +2450822|10873|2|| +2450822|10874|2|211| +2450822|10876|2|490| +2450822|10879|2|0| +2450822|10880|2|433| +2450822|10882|2|| +2450822|10885|2|95| +2450822|10886|2|260| +2450822|10888|2|824| +2450822|10891|2|144| +2450822|10892|2|339| +2450822|10894|2|| +2450822|10897|2|705| +2450822|10898|2|275| +2450822|10900|2|215| +2450822|10903|2|980| +2450822|10904|2|598| +2450822|10906|2|102| +2450822|10909|2|962| +2450822|10910|2|644| +2450822|10912|2|795| +2450822|10915|2|554| +2450822|10916|2|655| +2450822|10918|2|835| +2450822|10921|2|301| +2450822|10922|2|507| +2450822|10924|2|139| +2450822|10927|2|359| +2450822|10928|2|78| +2450822|10930|2|238| +2450822|10933|2|477| +2450822|10934|2|327| +2450822|10936|2|341| +2450822|10939|2|| +2450822|10940|2|233| +2450822|10942|2|200| +2450822|10945|2|850| +2450822|10946|2|| +2450822|10948|2|707| +2450822|10951|2|641| +2450822|10952|2|457| +2450822|10954|2|22| +2450822|10957|2|92| +2450822|10958|2|731| +2450822|10960|2|708| +2450822|10963|2|520| +2450822|10964|2|860| +2450822|10966|2|339| +2450822|10969|2|599| +2450822|10970|2|575| +2450822|10972|2|801| +2450822|10975|2|678| +2450822|10976|2|782| +2450822|10978|2|285| +2450822|10981|2|735| +2450822|10982|2|276| +2450822|10984|2|142| +2450822|10987|2|946| +2450822|10988|2|461| +2450822|10990|2|172| +2450822|10993|2|994| +2450822|10994|2|736| +2450822|10996|2|521| +2450822|10999|2|805| +2450822|11000|2|207| +2450822|11002|2|161| +2450822|11005|2|490| +2450822|11006|2|866| +2450822|11008|2|468| +2450822|11011|2|609| +2450822|11012|2|886| +2450822|11014|2|116| +2450822|11017|2|374| +2450822|11018|2|484| +2450822|11020|2|693| +2450822|11023|2|780| +2450822|11024|2|439| +2450822|11026|2|599| +2450822|11029|2|802| +2450822|11030|2|255| +2450822|11032|2|951| +2450822|11035|2|464| +2450822|11036|2|348| +2450822|11038|2|274| +2450822|11041|2|| +2450822|11042|2|730| +2450822|11044|2|129| +2450822|11047|2|875| +2450822|11048|2|990| +2450822|11050|2|315| +2450822|11053|2|822| +2450822|11054|2|134| +2450822|11056|2|0| +2450822|11059|2|165| +2450822|11060|2|485| +2450822|11062|2|819| +2450822|11065|2|591| +2450822|11066|2|452| +2450822|11068|2|| +2450822|11071|2|739| +2450822|11072|2|795| +2450822|11074|2|| +2450822|11077|2|758| +2450822|11078|2|672| +2450822|11080|2|42| +2450822|11083|2|352| +2450822|11084|2|969| +2450822|11086|2|55| +2450822|11089|2|113| +2450822|11090|2|967| +2450822|11092|2|858| +2450822|11095|2|283| +2450822|11096|2|400| +2450822|11098|2|959| +2450822|11101|2|65| +2450822|11102|2|181| +2450822|11104|2|517| +2450822|11107|2|967| +2450822|11108|2|422| +2450822|11110|2|108| +2450822|11113|2|81| +2450822|11114|2|367| +2450822|11116|2|683| +2450822|11119|2|562| +2450822|11120|2|190| +2450822|11122|2|807| +2450822|11125|2|515| +2450822|11126|2|332| +2450822|11128|2|921| +2450822|11131|2|711| +2450822|11132|2|384| +2450822|11134|2|681| +2450822|11137|2|838| +2450822|11138|2|410| +2450822|11140|2|521| +2450822|11143|2|782| +2450822|11144|2|439| +2450822|11146|2|421| +2450822|11149|2|418| +2450822|11150|2|485| +2450822|11152|2|321| +2450822|11155|2|638| +2450822|11156|2|821| +2450822|11158|2|417| +2450822|11161|2|296| +2450822|11162|2|951| +2450822|11164|2|212| +2450822|11167|2|404| +2450822|11168|2|73| +2450822|11170|2|| +2450822|11173|2|256| +2450822|11174|2|340| +2450822|11176|2|978| +2450822|11179|2|465| +2450822|11180|2|961| +2450822|11182|2|778| +2450822|11185|2|443| +2450822|11186|2|460| +2450822|11188|2|867| +2450822|11191|2|708| +2450822|11192|2|162| +2450822|11194|2|| +2450822|11197|2|570| +2450822|11198|2|487| +2450822|11200|2|999| +2450822|11203|2|634| +2450822|11204|2|824| +2450822|11206|2|238| +2450822|11209|2|398| +2450822|11210|2|7| +2450822|11212|2|521| +2450822|11215|2|206| +2450822|11216|2|246| +2450822|11218|2|601| +2450822|11221|2|709| +2450822|11222|2|491| +2450822|11224|2|292| +2450822|11227|2|971| +2450822|11228|2|788| +2450822|11230|2|359| +2450822|11233|2|895| +2450822|11234|2|104| +2450822|11236|2|581| +2450822|11239|2|276| +2450822|11240|2|833| +2450822|11242|2|949| +2450822|11245|2|40| +2450822|11246|2|427| +2450822|11248|2|34| +2450822|11251|2|507| +2450822|11252|2|912| +2450822|11254|2|620| +2450822|11257|2|783| +2450822|11258|2|647| +2450822|11260|2|158| +2450822|11263|2|492| +2450822|11264|2|965| +2450822|11266|2|| +2450822|11269|2|947| +2450822|11270|2|540| +2450822|11272|2|542| +2450822|11275|2|48| +2450822|11276|2|| +2450822|11278|2|649| +2450822|11281|2|269| +2450822|11282|2|458| +2450822|11284|2|264| +2450822|11287|2|488| +2450822|11288|2|254| +2450822|11290|2|669| +2450822|11293|2|868| +2450822|11294|2|| +2450822|11296|2|190| +2450822|11299|2|803| +2450822|11300|2|729| +2450822|11302|2|373| +2450822|11305|2|289| +2450822|11306|2|796| +2450822|11308|2|993| +2450822|11311|2|683| +2450822|11312|2|2| +2450822|11314|2|802| +2450822|11317|2|252| +2450822|11318|2|433| +2450822|11320|2|195| +2450822|11323|2|536| +2450822|11324|2|912| +2450822|11326|2|677| +2450822|11329|2|| +2450822|11330|2|288| +2450822|11332|2|362| +2450822|11335|2|417| +2450822|11336|2|| +2450822|11338|2|183| +2450822|11341|2|| +2450822|11342|2|926| +2450822|11344|2|207| +2450822|11347|2|996| +2450822|11348|2|774| +2450822|11350|2|722| +2450822|11353|2|252| +2450822|11354|2|204| +2450822|11356|2|282| +2450822|11359|2|46| +2450822|11360|2|60| +2450822|11362|2|473| +2450822|11365|2|743| +2450822|11366|2|857| +2450822|11368|2|1000| +2450822|11371|2|986| +2450822|11372|2|100| +2450822|11374|2|6| +2450822|11377|2|830| +2450822|11378|2|831| +2450822|11380|2|373| +2450822|11383|2|129| +2450822|11384|2|981| +2450822|11386|2|132| +2450822|11389|2|402| +2450822|11390|2|532| +2450822|11392|2|933| +2450822|11395|2|680| +2450822|11396|2|828| +2450822|11398|2|750| +2450822|11401|2|846| +2450822|11402|2|143| +2450822|11404|2|503| +2450822|11407|2|948| +2450822|11408|2|783| +2450822|11410|2|897| +2450822|11413|2|720| +2450822|11414|2|492| +2450822|11416|2|22| +2450822|11419|2|816| +2450822|11420|2|| +2450822|11422|2|440| +2450822|11425|2|| +2450822|11426|2|303| +2450822|11428|2|674| +2450822|11431|2|267| +2450822|11432|2|408| +2450822|11434|2|174| +2450822|11437|2|| +2450822|11438|2|745| +2450822|11440|2|551| +2450822|11443|2|856| +2450822|11444|2|910| +2450822|11446|2|518| +2450822|11449|2|763| +2450822|11450|2|952| +2450822|11452|2|551| +2450822|11455|2|308| +2450822|11456|2|408| +2450822|11458|2|803| +2450822|11461|2|954| +2450822|11462|2|55| +2450822|11464|2|21| +2450822|11467|2|567| +2450822|11468|2|366| +2450822|11470|2|691| +2450822|11473|2|187| +2450822|11474|2|807| +2450822|11476|2|299| +2450822|11479|2|332| +2450822|11480|2|272| +2450822|11482|2|152| +2450822|11485|2|980| +2450822|11486|2|992| +2450822|11488|2|684| +2450822|11491|2|540| +2450822|11492|2|47| +2450822|11494|2|681| +2450822|11497|2|971| +2450822|11498|2|798| +2450822|11500|2|758| +2450822|11503|2|422| +2450822|11504|2|341| +2450822|11506|2|351| +2450822|11509|2|18| +2450822|11510|2|738| +2450822|11512|2|249| +2450822|11515|2|814| +2450822|11516|2|56| +2450822|11518|2|220| +2450822|11521|2|152| +2450822|11522|2|219| +2450822|11524|2|153| +2450822|11527|2|607| +2450822|11528|2|897| +2450822|11530|2|947| +2450822|11533|2|260| +2450822|11534|2|| +2450822|11536|2|223| +2450822|11539|2|| +2450822|11540|2|550| +2450822|11542|2|385| +2450822|11545|2|588| +2450822|11546|2|287| +2450822|11548|2|567| +2450822|11551|2|326| +2450822|11552|2|573| +2450822|11554|2|663| +2450822|11557|2|54| +2450822|11558|2|7| +2450822|11560|2|608| +2450822|11563|2|965| +2450822|11564|2|835| +2450822|11566|2|122| +2450822|11569|2|286| +2450822|11570|2|532| +2450822|11572|2|412| +2450822|11575|2|697| +2450822|11576|2|699| +2450822|11578|2|49| +2450822|11581|2|120| +2450822|11582|2|| +2450822|11584|2|412| +2450822|11587|2|128| +2450822|11588|2|826| +2450822|11590|2|828| +2450822|11593|2|37| +2450822|11594|2|463| +2450822|11596|2|171| +2450822|11599|2|991| +2450822|11600|2|832| +2450822|11602|2|222| +2450822|11605|2|| +2450822|11606|2|630| +2450822|11608|2|969| +2450822|11611|2|380| +2450822|11612|2|139| +2450822|11614|2|70| +2450822|11617|2|12| +2450822|11618|2|57| +2450822|11620|2|427| +2450822|11623|2|195| +2450822|11624|2|752| +2450822|11626|2|525| +2450822|11629|2|564| +2450822|11630|2|69| +2450822|11632|2|242| +2450822|11635|2|761| +2450822|11636|2|350| +2450822|11638|2|176| +2450822|11641|2|870| +2450822|11642|2|204| +2450822|11644|2|617| +2450822|11647|2|423| +2450822|11648|2|183| +2450822|11650|2|170| +2450822|11653|2|127| +2450822|11654|2|868| +2450822|11656|2|339| +2450822|11659|2|838| +2450822|11660|2|291| +2450822|11662|2|298| +2450822|11665|2|485| +2450822|11666|2|690| +2450822|11668|2|677| +2450822|11671|2|792| +2450822|11672|2|165| +2450822|11674|2|34| +2450822|11677|2|570| +2450822|11678|2|472| +2450822|11680|2|120| +2450822|11683|2|998| +2450822|11684|2|966| +2450822|11686|2|| +2450822|11689|2|860| +2450822|11690|2|709| +2450822|11692|2|984| +2450822|11695|2|854| +2450822|11696|2|932| +2450822|11698|2|857| +2450822|11701|2|209| +2450822|11702|2|980| +2450822|11704|2|591| +2450822|11707|2|952| +2450822|11708|2|367| +2450822|11710|2|436| +2450822|11713|2|242| +2450822|11714|2|536| +2450822|11716|2|834| +2450822|11719|2|193| +2450822|11720|2|292| +2450822|11722|2|979| +2450822|11725|2|677| +2450822|11726|2|941| +2450822|11728|2|985| +2450822|11731|2|367| +2450822|11732|2|696| +2450822|11734|2|829| +2450822|11737|2|664| +2450822|11738|2|39| +2450822|11740|2|822| +2450822|11743|2|288| +2450822|11744|2|23| +2450822|11746|2|941| +2450822|11749|2|501| +2450822|11750|2|638| +2450822|11752|2|86| +2450822|11755|2|404| +2450822|11756|2|938| +2450822|11758|2|159| +2450822|11761|2|323| +2450822|11762|2|468| +2450822|11764|2|72| +2450822|11767|2|286| +2450822|11768|2|673| +2450822|11770|2|625| +2450822|11773|2|918| +2450822|11774|2|211| +2450822|11776|2|659| +2450822|11779|2|| +2450822|11780|2|207| +2450822|11782|2|474| +2450822|11785|2|739| +2450822|11786|2|160| +2450822|11788|2|453| +2450822|11791|2|412| +2450822|11792|2|912| +2450822|11794|2|601| +2450822|11797|2|520| +2450822|11798|2|783| +2450822|11800|2|36| +2450822|11803|2|909| +2450822|11804|2|585| +2450822|11806|2|757| +2450822|11809|2|304| +2450822|11810|2|822| +2450822|11812|2|40| +2450822|11815|2|978| +2450822|11816|2|805| +2450822|11818|2|564| +2450822|11821|2|795| +2450822|11822|2|28| +2450822|11824|2|933| +2450822|11827|2|788| +2450822|11828|2|439| +2450822|11830|2|555| +2450822|11833|2|230| +2450822|11834|2|491| +2450822|11836|2|370| +2450822|11839|2|971| +2450822|11840|2|272| +2450822|11842|2|392| +2450822|11845|2|| +2450822|11846|2|474| +2450822|11848|2|959| +2450822|11851|2|481| +2450822|11852|2|146| +2450822|11854|2|| +2450822|11857|2|408| +2450822|11858|2|678| +2450822|11860|2|820| +2450822|11863|2|| +2450822|11864|2|870| +2450822|11866|2|892| +2450822|11869|2|40| +2450822|11870|2|208| +2450822|11872|2|997| +2450822|11875|2|472| +2450822|11876|2|731| +2450822|11878|2|782| +2450822|11881|2|517| +2450822|11882|2|463| +2450822|11884|2|596| +2450822|11887|2|834| +2450822|11888|2|230| +2450822|11890|2|450| +2450822|11893|2|991| +2450822|11894|2|275| +2450822|11896|2|801| +2450822|11899|2|418| +2450822|11900|2|19| +2450822|11902|2|636| +2450822|11905|2|614| +2450822|11906|2|196| +2450822|11908|2|476| +2450822|11911|2|801| +2450822|11912|2|42| +2450822|11914|2|855| +2450822|11917|2|515| +2450822|11918|2|974| +2450822|11920|2|121| +2450822|11923|2|664| +2450822|11924|2|310| +2450822|11926|2|309| +2450822|11929|2|447| +2450822|11930|2|825| +2450822|11932|2|15| +2450822|11935|2|257| +2450822|11936|2|964| +2450822|11938|2|934| +2450822|11941|2|587| +2450822|11942|2|615| +2450822|11944|2|222| +2450822|11947|2|707| +2450822|11948|2|352| +2450822|11950|2|329| +2450822|11953|2|7| +2450822|11954|2|796| +2450822|11956|2|222| +2450822|11959|2|318| +2450822|11960|2|81| +2450822|11962|2|703| +2450822|11965|2|462| +2450822|11966|2|692| +2450822|11968|2|226| +2450822|11971|2|893| +2450822|11972|2|251| +2450822|11974|2|237| +2450822|11977|2|848| +2450822|11978|2|269| +2450822|11980|2|921| +2450822|11983|2|372| +2450822|11984|2|150| +2450822|11986|2|761| +2450822|11989|2|526| +2450822|11990|2|825| +2450822|11992|2|126| +2450822|11995|2|246| +2450822|11996|2|196| +2450822|11998|2|114| +2450822|12001|2|519| +2450822|12002|2|385| +2450822|12004|2|306| +2450822|12007|2|8| +2450822|12008|2|948| +2450822|12010|2|688| +2450822|12013|2|591| +2450822|12014|2|248| +2450822|12016|2|| +2450822|12019|2|935| +2450822|12020|2|257| +2450822|12022|2|885| +2450822|12025|2|835| +2450822|12026|2|44| +2450822|12028|2|265| +2450822|12031|2|337| +2450822|12032|2|631| +2450822|12034|2|605| +2450822|12037|2|309| +2450822|12038|2|8| +2450822|12040|2|106| +2450822|12043|2|786| +2450822|12044|2|203| +2450822|12046|2|339| +2450822|12049|2|953| +2450822|12050|2|63| +2450822|12052|2|55| +2450822|12055|2|461| +2450822|12056|2|284| +2450822|12058|2|737| +2450822|12061|2|297| +2450822|12062|2|823| +2450822|12064|2|98| +2450822|12067|2|767| +2450822|12068|2|904| +2450822|12070|2|993| +2450822|12073|2|678| +2450822|12074|2|428| +2450822|12076|2|345| +2450822|12079|2|195| +2450822|12080|2|109| +2450822|12082|2|661| +2450822|12085|2|581| +2450822|12086|2|758| +2450822|12088|2|368| +2450822|12091|2|666| +2450822|12092|2|867| +2450822|12094|2|| +2450822|12097|2|731| +2450822|12098|2|694| +2450822|12100|2|446| +2450822|12103|2|58| +2450822|12104|2|616| +2450822|12106|2|261| +2450822|12109|2|402| +2450822|12110|2|415| +2450822|12112|2|973| +2450822|12115|2|5| +2450822|12116|2|199| +2450822|12118|2|237| +2450822|12121|2|90| +2450822|12122|2|413| +2450822|12124|2|558| +2450822|12127|2|576| +2450822|12128|2|658| +2450822|12130|2|881| +2450822|12133|2|355| +2450822|12134|2|111| +2450822|12136|2|700| +2450822|12139|2|599| +2450822|12140|2|882| +2450822|12142|2|16| +2450822|12145|2|255| +2450822|12146|2|85| +2450822|12148|2|700| +2450822|12151|2|680| +2450822|12152|2|227| +2450822|12154|2|811| +2450822|12157|2|| +2450822|12158|2|320| +2450822|12160|2|927| +2450822|12163|2|413| +2450822|12164|2|226| +2450822|12166|2|866| +2450822|12169|2|651| +2450822|12170|2|660| +2450822|12172|2|738| +2450822|12175|2|902| +2450822|12176|2|100| +2450822|12178|2|277| +2450822|12181|2|732| +2450822|12182|2|253| +2450822|12184|2|58| +2450822|12187|2|33| +2450822|12188|2|275| +2450822|12190|2|582| +2450822|12193|2|91| +2450822|12194|2|163| +2450822|12196|2|548| +2450822|12199|2|502| +2450822|12200|2|36| +2450822|12202|2|212| +2450822|12205|2|911| +2450822|12206|2|890| +2450822|12208|2|212| +2450822|12211|2|561| +2450822|12212|2|| +2450822|12214|2|146| +2450822|12217|2|202| +2450822|12218|2|875| +2450822|12220|2|972| +2450822|12223|2|877| +2450822|12224|2|690| +2450822|12226|2|507| +2450822|12229|2|141| +2450822|12230|2|434| +2450822|12232|2|340| +2450822|12235|2|514| +2450822|12236|2|797| +2450822|12238|2|395| +2450822|12241|2|107| +2450822|12242|2|325| +2450822|12244|2|108| +2450822|12247|2|398| +2450822|12248|2|323| +2450822|12250|2|475| +2450822|12253|2|152| +2450822|12254|2|227| +2450822|12256|2|511| +2450822|12259|2|366| +2450822|12260|2|677| +2450822|12262|2|366| +2450822|12265|2|193| +2450822|12266|2|969| +2450822|12268|2|62| +2450822|12271|2|191| +2450822|12272|2|| +2450822|12274|2|63| +2450822|12277|2|60| +2450822|12278|2|221| +2450822|12280|2|913| +2450822|12283|2|956| +2450822|12284|2|494| +2450822|12286|2|| +2450822|12289|2|156| +2450822|12290|2|774| +2450822|12292|2|103| +2450822|12295|2|927| +2450822|12296|2|352| +2450822|12298|2|346| +2450822|12301|2|105| +2450822|12302|2|931| +2450822|12304|2|148| +2450822|12307|2|762| +2450822|12308|2|274| +2450822|12310|2|665| +2450822|12313|2|879| +2450822|12314|2|620| +2450822|12316|2|205| +2450822|12319|2|924| +2450822|12320|2|67| +2450822|12322|2|904| +2450822|12325|2|840| +2450822|12326|2|761| +2450822|12328|2|| +2450822|12331|2|270| +2450822|12332|2|118| +2450822|12334|2|862| +2450822|12337|2|109| +2450822|12338|2|527| +2450822|12340|2|231| +2450822|12343|2|141| +2450822|12344|2|589| +2450822|12346|2|631| +2450822|12349|2|| +2450822|12350|2|901| +2450822|12352|2|679| +2450822|12355|2|854| +2450822|12356|2|331| +2450822|12358|2|683| +2450822|12361|2|666| +2450822|12362|2|9| +2450822|12364|2|507| +2450822|12367|2|860| +2450822|12368|2|869| +2450822|12370|2|546| +2450822|12373|2|393| +2450822|12374|2|593| +2450822|12376|2|860| +2450822|12379|2|991| +2450822|12380|2|60| +2450822|12382|2|798| +2450822|12385|2|307| +2450822|12386|2|993| +2450822|12388|2|50| +2450822|12391|2|244| +2450822|12392|2|839| +2450822|12394|2|143| +2450822|12397|2|19| +2450822|12398|2|905| +2450822|12400|2|264| +2450822|12403|2|999| +2450822|12404|2|714| +2450822|12406|2|654| +2450822|12409|2|167| +2450822|12410|2|273| +2450822|12412|2|118| +2450822|12415|2|103| +2450822|12416|2|486| +2450822|12418|2|607| +2450822|12421|2|796| +2450822|12422|2|931| +2450822|12424|2|981| +2450822|12427|2|| +2450822|12428|2|743| +2450822|12430|2|874| +2450822|12433|2|71| +2450822|12434|2|669| +2450822|12436|2|| +2450822|12439|2|338| +2450822|12440|2|| +2450822|12442|2|483| +2450822|12445|2|387| +2450822|12446|2|521| +2450822|12448|2|797| +2450822|12451|2|781| +2450822|12452|2|539| +2450822|12454|2|578| +2450822|12457|2|401| +2450822|12458|2|926| +2450822|12460|2|588| +2450822|12463|2|381| +2450822|12464|2|947| +2450822|12466|2|222| +2450822|12469|2|838| +2450822|12470|2|743| +2450822|12472|2|861| +2450822|12475|2|623| +2450822|12476|2|783| +2450822|12478|2|32| +2450822|12481|2|583| +2450822|12482|2|944| +2450822|12484|2|115| +2450822|12487|2|817| +2450822|12488|2|254| +2450822|12490|2|892| +2450822|12493|2|585| +2450822|12494|2|526| +2450822|12496|2|913| +2450822|12499|2|831| +2450822|12500|2|350| +2450822|12502|2|712| +2450822|12505|2|66| +2450822|12506|2|986| +2450822|12508|2|208| +2450822|12511|2|132| +2450822|12512|2|154| +2450822|12514|2|878| +2450822|12517|2|181| +2450822|12518|2|862| +2450822|12520|2|653| +2450822|12523|2|30| +2450822|12524|2|509| +2450822|12526|2|465| +2450822|12529|2|714| +2450822|12530|2|712| +2450822|12532|2|686| +2450822|12535|2|220| +2450822|12536|2|100| +2450822|12538|2|529| +2450822|12541|2|963| +2450822|12542|2|257| +2450822|12544|2|156| +2450822|12547|2|703| +2450822|12548|2|148| +2450822|12550|2|922| +2450822|12553|2|384| +2450822|12554|2|652| +2450822|12556|2|42| +2450822|12559|2|247| +2450822|12560|2|918| +2450822|12562|2|613| +2450822|12565|2|580| +2450822|12566|2|948| +2450822|12568|2|879| +2450822|12571|2|| +2450822|12572|2|68| +2450822|12574|2|969| +2450822|12577|2|| +2450822|12578|2|676| +2450822|12580|2|108| +2450822|12583|2|850| +2450822|12584|2|199| +2450822|12586|2|592| +2450822|12589|2|149| +2450822|12590|2|886| +2450822|12592|2|754| +2450822|12595|2|846| +2450822|12596|2|786| +2450822|12598|2|452| +2450822|12601|2|899| +2450822|12602|2|190| +2450822|12604|2|825| +2450822|12607|2|844| +2450822|12608|2|1000| +2450822|12610|2|441| +2450822|12613|2|225| +2450822|12614|2|524| +2450822|12616|2|772| +2450822|12619|2|178| +2450822|12620|2|461| +2450822|12622|2|966| +2450822|12625|2|| +2450822|12626|2|611| +2450822|12628|2|769| +2450822|12631|2|397| +2450822|12632|2|271| +2450822|12634|2|98| +2450822|12637|2|715| +2450822|12638|2|423| +2450822|12640|2|188| +2450822|12643|2|859| +2450822|12644|2|176| +2450822|12646|2|768| +2450822|12649|2|443| +2450822|12650|2|312| +2450822|12652|2|5| +2450822|12655|2|994| +2450822|12656|2|576| +2450822|12658|2|473| +2450822|12661|2|652| +2450822|12662|2|982| +2450822|12664|2|219| +2450822|12667|2|269| +2450822|12668|2|651| +2450822|12670|2|752| +2450822|12673|2|620| +2450822|12674|2|455| +2450822|12676|2|326| +2450822|12679|2|390| +2450822|12680|2|748| +2450822|12682|2|390| +2450822|12685|2|585| +2450822|12686|2|121| +2450822|12688|2|15| +2450822|12691|2|79| +2450822|12692|2|250| +2450822|12694|2|49| +2450822|12697|2|384| +2450822|12698|2|215| +2450822|12700|2|510| +2450822|12703|2|11| +2450822|12704|2|360| +2450822|12706|2|741| +2450822|12709|2|414| +2450822|12710|2|878| +2450822|12712|2|556| +2450822|12715|2|503| +2450822|12716|2|995| +2450822|12718|2|259| +2450822|12721|2|704| +2450822|12722|2|50| +2450822|12724|2|| +2450822|12727|2|192| +2450822|12728|2|302| +2450822|12730|2|| +2450822|12733|2|929| +2450822|12734|2|275| +2450822|12736|2|531| +2450822|12739|2|305| +2450822|12740|2|899| +2450822|12742|2|233| +2450822|12745|2|901| +2450822|12746|2|361| +2450822|12748|2|192| +2450822|12751|2|911| +2450822|12752|2|525| +2450822|12754|2|891| +2450822|12757|2|925| +2450822|12758|2|661| +2450822|12760|2|255| +2450822|12763|2|489| +2450822|12764|2|207| +2450822|12766|2|119| +2450822|12769|2|471| +2450822|12770|2|89| +2450822|12772|2|761| +2450822|12775|2|823| +2450822|12776|2|529| +2450822|12778|2|282| +2450822|12781|2|367| +2450822|12782|2|957| +2450822|12784|2|183| +2450822|12787|2|142| +2450822|12788|2|607| +2450822|12790|2|359| +2450822|12793|2|366| +2450822|12794|2|| +2450822|12796|2|673| +2450822|12799|2|902| +2450822|12800|2|979| +2450822|12802|2|37| +2450822|12805|2|883| +2450822|12806|2|16| +2450822|12808|2|685| +2450822|12811|2|735| +2450822|12812|2|57| +2450822|12814|2|964| +2450822|12817|2|536| +2450822|12818|2|5| +2450822|12820|2|978| +2450822|12823|2|881| +2450822|12824|2|176| +2450822|12826|2|966| +2450822|12829|2|996| +2450822|12830|2|541| +2450822|12832|2|246| +2450822|12835|2|117| +2450822|12836|2|977| +2450822|12838|2|905| +2450822|12841|2|773| +2450822|12842|2|696| +2450822|12844|2|942| +2450822|12847|2|590| +2450822|12848|2|152| +2450822|12850|2|853| +2450822|12853|2|931| +2450822|12854|2|895| +2450822|12856|2|962| +2450822|12859|2|779| +2450822|12860|2|301| +2450822|12862|2|649| +2450822|12865|2|381| +2450822|12866|2|821| +2450822|12868|2|12| +2450822|12871|2|561| +2450822|12872|2|870| +2450822|12874|2|867| +2450822|12877|2|| +2450822|12878|2|| +2450822|12880|2|578| +2450822|12883|2|584| +2450822|12884|2|857| +2450822|12886|2|596| +2450822|12889|2|337| +2450822|12890|2|419| +2450822|12892|2|406| +2450822|12895|2|903| +2450822|12896|2|453| +2450822|12898|2|587| +2450822|12901|2|452| +2450822|12902|2|897| +2450822|12904|2|810| +2450822|12907|2|493| +2450822|12908|2|970| +2450822|12910|2|452| +2450822|12913|2|| +2450822|12914|2|560| +2450822|12916|2|640| +2450822|12919|2|428| +2450822|12920|2|329| +2450822|12922|2|145| +2450822|12925|2|861| +2450822|12926|2|608| +2450822|12928|2|616| +2450822|12931|2|526| +2450822|12932|2|521| +2450822|12934|2|811| +2450822|12937|2|354| +2450822|12938|2|| +2450822|12940|2|283| +2450822|12943|2|612| +2450822|12944|2|308| +2450822|12946|2|82| +2450822|12949|2|990| +2450822|12950|2|| +2450822|12952|2|898| +2450822|12955|2|927| +2450822|12956|2|553| +2450822|12958|2|423| +2450822|12961|2|431| +2450822|12962|2|83| +2450822|12964|2|814| +2450822|12967|2|281| +2450822|12968|2|173| +2450822|12970|2|594| +2450822|12973|2|156| +2450822|12974|2|646| +2450822|12976|2|159| +2450822|12979|2|431| +2450822|12980|2|272| +2450822|12982|2|398| +2450822|12985|2|161| +2450822|12986|2|536| +2450822|12988|2|121| +2450822|12991|2|562| +2450822|12992|2|382| +2450822|12994|2|593| +2450822|12997|2|119| +2450822|12998|2|52| +2450822|13000|2|762| +2450822|13003|2|407| +2450822|13004|2|128| +2450822|13006|2|983| +2450822|13009|2|883| +2450822|13010|2|932| +2450822|13012|2|601| +2450822|13015|2|28| +2450822|13016|2|| +2450822|13018|2|694| +2450822|13021|2|762| +2450822|13022|2|417| +2450822|13024|2|606| +2450822|13027|2|663| +2450822|13028|2|528| +2450822|13030|2|383| +2450822|13033|2|56| +2450822|13034|2|565| +2450822|13036|2|808| +2450822|13039|2|620| +2450822|13040|2|521| +2450822|13042|2|879| +2450822|13045|2|259| +2450822|13046|2|660| +2450822|13048|2|314| +2450822|13051|2|731| +2450822|13052|2|19| +2450822|13054|2|125| +2450822|13057|2|691| +2450822|13058|2|342| +2450822|13060|2|873| +2450822|13063|2|864| +2450822|13064|2|715| +2450822|13066|2|633| +2450822|13069|2|483| +2450822|13070|2|620| +2450822|13072|2|427| +2450822|13075|2|715| +2450822|13076|2|508| +2450822|13078|2|950| +2450822|13081|2|938| +2450822|13082|2|107| +2450822|13084|2|528| +2450822|13087|2|51| +2450822|13088|2|572| +2450822|13090|2|486| +2450822|13093|2|689| +2450822|13094|2|910| +2450822|13096|2|368| +2450822|13099|2|807| +2450822|13100|2|655| +2450822|13102|2|838| +2450822|13105|2|932| +2450822|13106|2|282| +2450822|13108|2|896| +2450822|13111|2|616| +2450822|13112|2|555| +2450822|13114|2|702| +2450822|13117|2|212| +2450822|13118|2|50| +2450822|13120|2|209| +2450822|13123|2|631| +2450822|13124|2|186| +2450822|13126|2|182| +2450822|13129|2|393| +2450822|13130|2|900| +2450822|13132|2|830| +2450822|13135|2|756| +2450822|13136|2|18| +2450822|13138|2|332| +2450822|13141|2|26| +2450822|13142|2|603| +2450822|13144|2|328| +2450822|13147|2|320| +2450822|13148|2|925| +2450822|13150|2|985| +2450822|13153|2|23| +2450822|13154|2|650| +2450822|13156|2|824| +2450822|13159|2|623| +2450822|13160|2|455| +2450822|13162|2|637| +2450822|13165|2|375| +2450822|13166|2|258| +2450822|13168|2|100| +2450822|13171|2|646| +2450822|13172|2|631| +2450822|13174|2|532| +2450822|13177|2|915| +2450822|13178|2|6| +2450822|13180|2|640| +2450822|13183|2|589| +2450822|13184|2|645| +2450822|13186|2|588| +2450822|13189|2|256| +2450822|13190|2|924| +2450822|13192|2|783| +2450822|13195|2|394| +2450822|13196|2|877| +2450822|13198|2|711| +2450822|13201|2|192| +2450822|13202|2|296| +2450822|13204|2|965| +2450822|13207|2|749| +2450822|13208|2|834| +2450822|13210|2|636| +2450822|13213|2|794| +2450822|13214|2|158| +2450822|13216|2|307| +2450822|13219|2|737| +2450822|13220|2|263| +2450822|13222|2|128| +2450822|13225|2|392| +2450822|13226|2|878| +2450822|13228|2|350| +2450822|13231|2|627| +2450822|13232|2|500| +2450822|13234|2|| +2450822|13237|2|129| +2450822|13238|2|976| +2450822|13240|2|34| +2450822|13243|2|298| +2450822|13244|2|999| +2450822|13246|2|714| +2450822|13249|2|513| +2450822|13250|2|340| +2450822|13252|2|185| +2450822|13255|2|47| +2450822|13256|2|380| +2450822|13258|2|383| +2450822|13261|2|308| +2450822|13262|2|984| +2450822|13264|2|758| +2450822|13267|2|294| +2450822|13268|2|525| +2450822|13270|2|44| +2450822|13273|2|859| +2450822|13274|2|991| +2450822|13276|2|913| +2450822|13279|2|934| +2450822|13280|2|440| +2450822|13282|2|298| +2450822|13285|2|951| +2450822|13286|2|965| +2450822|13288|2|475| +2450822|13291|2|789| +2450822|13292|2|533| +2450822|13294|2|846| +2450822|13297|2|703| +2450822|13298|2|934| +2450822|13300|2|637| +2450822|13303|2|790| +2450822|13304|2|603| +2450822|13306|2|852| +2450822|13309|2|484| +2450822|13310|2|993| +2450822|13312|2|822| +2450822|13315|2|987| +2450822|13316|2|829| +2450822|13318|2|652| +2450822|13321|2|129| +2450822|13322|2|813| +2450822|13324|2|986| +2450822|13327|2|953| +2450822|13328|2|642| +2450822|13330|2|359| +2450822|13333|2|686| +2450822|13334|2|442| +2450822|13336|2|541| +2450822|13339|2|729| +2450822|13340|2|123| +2450822|13342|2|591| +2450822|13345|2|937| +2450822|13346|2|324| +2450822|13348|2|891| +2450822|13351|2|228| +2450822|13352|2|501| +2450822|13354|2|759| +2450822|13357|2|627| +2450822|13358|2|394| +2450822|13360|2|457| +2450822|13363|2|601| +2450822|13364|2|29| +2450822|13366|2|770| +2450822|13369|2|893| +2450822|13370|2|538| +2450822|13372|2|787| +2450822|13375|2|232| +2450822|13376|2|42| +2450822|13378|2|843| +2450822|13381|2|357| +2450822|13382|2|655| +2450822|13384|2|721| +2450822|13387|2|24| +2450822|13388|2|305| +2450822|13390|2|551| +2450822|13393|2|406| +2450822|13394|2|795| +2450822|13396|2|248| +2450822|13399|2|3| +2450822|13400|2|419| +2450822|13402|2|830| +2450822|13405|2|236| +2450822|13406|2|8| +2450822|13408|2|659| +2450822|13411|2|98| +2450822|13412|2|| +2450822|13414|2|62| +2450822|13417|2|169| +2450822|13418|2|581| +2450822|13420|2|363| +2450822|13423|2|329| +2450822|13424|2|674| +2450822|13426|2|| +2450822|13429|2|525| +2450822|13430|2|784| +2450822|13432|2|380| +2450822|13435|2|152| +2450822|13436|2|469| +2450822|13438|2|334| +2450822|13441|2|688| +2450822|13442|2|766| +2450822|13444|2|626| +2450822|13447|2|71| +2450822|13448|2|918| +2450822|13450|2|528| +2450822|13453|2|878| +2450822|13454|2|843| +2450822|13456|2|263| +2450822|13459|2|592| +2450822|13460|2|67| +2450822|13462|2|420| +2450822|13465|2|862| +2450822|13466|2|319| +2450822|13468|2|| +2450822|13471|2|989| +2450822|13472|2|347| +2450822|13474|2|936| +2450822|13477|2|504| +2450822|13478|2|243| +2450822|13480|2|691| +2450822|13483|2|31| +2450822|13484|2|75| +2450822|13486|2|660| +2450822|13489|2|934| +2450822|13490|2|383| +2450822|13492|2|732| +2450822|13495|2|722| +2450822|13496|2|357| +2450822|13498|2|787| +2450822|13501|2|696| +2450822|13502|2|631| +2450822|13504|2|318| +2450822|13507|2|535| +2450822|13508|2|218| +2450822|13510|2|46| +2450822|13513|2|944| +2450822|13514|2|272| +2450822|13516|2|427| +2450822|13519|2|663| +2450822|13520|2|493| +2450822|13522|2|485| +2450822|13525|2|525| +2450822|13526|2|806| +2450822|13528|2|345| +2450822|13531|2|276| +2450822|13532|2|223| +2450822|13534|2|281| +2450822|13537|2|605| +2450822|13538|2|| +2450822|13540|2|807| +2450822|13543|2|702| +2450822|13544|2|| +2450822|13546|2|105| +2450822|13549|2|820| +2450822|13550|2|222| +2450822|13552|2|620| +2450822|13555|2|319| +2450822|13556|2|915| +2450822|13558|2|101| +2450822|13561|2|255| +2450822|13562|2|177| +2450822|13564|2|149| +2450822|13567|2|725| +2450822|13568|2|527| +2450822|13570|2|849| +2450822|13573|2|749| +2450822|13574|2|511| +2450822|13576|2|995| +2450822|13579|2|95| +2450822|13580|2|315| +2450822|13582|2|106| +2450822|13585|2|| +2450822|13586|2|266| +2450822|13588|2|163| +2450822|13591|2|747| +2450822|13592|2|541| +2450822|13594|2|229| +2450822|13597|2|461| +2450822|13598|2|627| +2450822|13600|2|460| +2450822|13603|2|234| +2450822|13604|2|290| +2450822|13606|2|497| +2450822|13609|2|500| +2450822|13610|2|421| +2450822|13612|2|716| +2450822|13615|2|694| +2450822|13616|2|318| +2450822|13618|2|233| +2450822|13621|2|309| +2450822|13622|2|586| +2450822|13624|2|939| +2450822|13627|2|282| +2450822|13628|2|340| +2450822|13630|2|436| +2450822|13633|2|83| +2450822|13634|2|836| +2450822|13636|2|475| +2450822|13639|2|207| +2450822|13640|2|164| +2450822|13642|2|| +2450822|13645|2|228| +2450822|13646|2|57| +2450822|13648|2|194| +2450822|13651|2|400| +2450822|13652|2|994| +2450822|13654|2|403| +2450822|13657|2|739| +2450822|13658|2|390| +2450822|13660|2|39| +2450822|13663|2|476| +2450822|13664|2|435| +2450822|13666|2|586| +2450822|13669|2|432| +2450822|13670|2|797| +2450822|13672|2|436| +2450822|13675|2|735| +2450822|13676|2|463| +2450822|13678|2|979| +2450822|13681|2|340| +2450822|13682|2|22| +2450822|13684|2|713| +2450822|13687|2|195| +2450822|13688|2|834| +2450822|13690|2|218| +2450822|13693|2|871| +2450822|13694|2|629| +2450822|13696|2|187| +2450822|13699|2|882| +2450822|13700|2|761| +2450822|13702|2|842| +2450822|13705|2|274| +2450822|13706|2|208| +2450822|13708|2|122| +2450822|13711|2|749| +2450822|13712|2|118| +2450822|13714|2|19| +2450822|13717|2|618| +2450822|13718|2|612| +2450822|13720|2|794| +2450822|13723|2|733| +2450822|13724|2|412| +2450822|13726|2|475| +2450822|13729|2|153| +2450822|13730|2|545| +2450822|13732|2|498| +2450822|13735|2|605| +2450822|13736|2|558| +2450822|13738|2|943| +2450822|13741|2|987| +2450822|13742|2|704| +2450822|13744|2|212| +2450822|13747|2|137| +2450822|13748|2|283| +2450822|13750|2|349| +2450822|13753|2|202| +2450822|13754|2|614| +2450822|13756|2|692| +2450822|13759|2|141| +2450822|13760|2|753| +2450822|13762|2|420| +2450822|13765|2|531| +2450822|13766|2|688| +2450822|13768|2|500| +2450822|13771|2|69| +2450822|13772|2|292| +2450822|13774|2|210| +2450822|13777|2|823| +2450822|13778|2|233| +2450822|13780|2|838| +2450822|13783|2|790| +2450822|13784|2|46| +2450822|13786|2|154| +2450822|13789|2|235| +2450822|13790|2|1| +2450822|13792|2|77| +2450822|13795|2|891| +2450822|13796|2|182| +2450822|13798|2|457| +2450822|13801|2|736| +2450822|13802|2|662| +2450822|13804|2|364| +2450822|13807|2|259| +2450822|13808|2|163| +2450822|13810|2|633| +2450822|13813|2|650| +2450822|13814|2|266| +2450822|13816|2|966| +2450822|13819|2|137| +2450822|13820|2|356| +2450822|13822|2|345| +2450822|13825|2|151| +2450822|13826|2|120| +2450822|13828|2|646| +2450822|13831|2|709| +2450822|13832|2|598| +2450822|13834|2|503| +2450822|13837|2|158| +2450822|13838|2|262| +2450822|13840|2|132| +2450822|13843|2|238| +2450822|13844|2|645| +2450822|13846|2|662| +2450822|13849|2|194| +2450822|13850|2|| +2450822|13852|2|846| +2450822|13855|2|5| +2450822|13856|2|492| +2450822|13858|2|376| +2450822|13861|2|155| +2450822|13862|2|495| +2450822|13864|2|300| +2450822|13867|2|983| +2450822|13868|2|457| +2450822|13870|2|186| +2450822|13873|2|682| +2450822|13874|2|440| +2450822|13876|2|474| +2450822|13879|2|933| +2450822|13880|2|| +2450822|13882|2|486| +2450822|13885|2|864| +2450822|13886|2|581| +2450822|13888|2|814| +2450822|13891|2|288| +2450822|13892|2|118| +2450822|13894|2|748| +2450822|13897|2|973| +2450822|13898|2|17| +2450822|13900|2|| +2450822|13903|2|954| +2450822|13904|2|869| +2450822|13906|2|585| +2450822|13909|2|710| +2450822|13910|2|109| +2450822|13912|2|935| +2450822|13915|2|722| +2450822|13916|2|880| +2450822|13918|2|691| +2450822|13921|2|467| +2450822|13922|2|299| +2450822|13924|2|36| +2450822|13927|2|375| +2450822|13928|2|117| +2450822|13930|2|88| +2450822|13933|2|882| +2450822|13934|2|564| +2450822|13936|2|718| +2450822|13939|2|580| +2450822|13940|2|814| +2450822|13942|2|146| +2450822|13945|2|939| +2450822|13946|2|945| +2450822|13948|2|895| +2450822|13951|2|| +2450822|13952|2|630| +2450822|13954|2|672| +2450822|13957|2|50| +2450822|13958|2|626| +2450822|13960|2|106| +2450822|13963|2|473| +2450822|13964|2|557| +2450822|13966|2|978| +2450822|13969|2|79| +2450822|13970|2|619| +2450822|13972|2|680| +2450822|13975|2|541| +2450822|13976|2|795| +2450822|13978|2|| +2450822|13981|2|918| +2450822|13982|2|878| +2450822|13984|2|500| +2450822|13987|2|37| +2450822|13988|2|405| +2450822|13990|2|342| +2450822|13993|2|138| +2450822|13994|2|| +2450822|13996|2|579| +2450822|13999|2|991| +2450822|14000|2|496| +2450822|14002|2|332| +2450822|14005|2|190| +2450822|14006|2|718| +2450822|14008|2|421| +2450822|14011|2|502| +2450822|14012|2|898| +2450822|14014|2|489| +2450822|14017|2|251| +2450822|14018|2|718| +2450822|14020|2|522| +2450822|14023|2|919| +2450822|14024|2|929| +2450822|14026|2|419| +2450822|14029|2|503| +2450822|14030|2|524| +2450822|14032|2|291| +2450822|14035|2|111| +2450822|14036|2|468| +2450822|14038|2|416| +2450822|14041|2|517| +2450822|14042|2|455| +2450822|14044|2|620| +2450822|14047|2|106| +2450822|14048|2|| +2450822|14050|2|377| +2450822|14053|2|864| +2450822|14054|2|367| +2450822|14056|2|467| +2450822|14059|2|300| +2450822|14060|2|758| +2450822|14062|2|857| +2450822|14065|2|3| +2450822|14066|2|994| +2450822|14068|2|481| +2450822|14071|2|916| +2450822|14072|2|100| +2450822|14074|2|326| +2450822|14077|2|547| +2450822|14078|2|602| +2450822|14080|2|| +2450822|14083|2|277| +2450822|14084|2|492| +2450822|14086|2|767| +2450822|14089|2|607| +2450822|14090|2|530| +2450822|14092|2|224| +2450822|14095|2|726| +2450822|14096|2|978| +2450822|14098|2|68| +2450822|14101|2|100| +2450822|14102|2|27| +2450822|14104|2|416| +2450822|14107|2|81| +2450822|14108|2|861| +2450822|14110|2|975| +2450822|14113|2|414| +2450822|14114|2|619| +2450822|14116|2|100| +2450822|14119|2|790| +2450822|14120|2|834| +2450822|14122|2|151| +2450822|14125|2|275| +2450822|14126|2|| +2450822|14128|2|560| +2450822|14131|2|706| +2450822|14132|2|743| +2450822|14134|2|954| +2450822|14137|2|166| +2450822|14138|2|215| +2450822|14140|2|53| +2450822|14143|2|390| +2450822|14144|2|| +2450822|14146|2|592| +2450822|14149|2|| +2450822|14150|2|783| +2450822|14152|2|336| +2450822|14155|2|753| +2450822|14156|2|849| +2450822|14158|2|693| +2450822|14161|2|282| +2450822|14162|2|921| +2450822|14164|2|| +2450822|14167|2|643| +2450822|14168|2|115| +2450822|14170|2|781| +2450822|14173|2|570| +2450822|14174|2|477| +2450822|14176|2|| +2450822|14179|2|821| +2450822|14180|2|380| +2450822|14182|2|38| +2450822|14185|2|485| +2450822|14186|2|798| +2450822|14188|2|789| +2450822|14191|2|775| +2450822|14192|2|528| +2450822|14194|2|395| +2450822|14197|2|| +2450822|14198|2|258| +2450822|14200|2|483| +2450822|14203|2|| +2450822|14204|2|373| +2450822|14206|2|12| +2450822|14209|2|293| +2450822|14210|2|841| +2450822|14212|2|13| +2450822|14215|2|704| +2450822|14216|2|981| +2450822|14218|2|854| +2450822|14221|2|864| +2450822|14222|2|154| +2450822|14224|2|761| +2450822|14227|2|634| +2450822|14228|2|715| +2450822|14230|2|970| +2450822|14233|2|682| +2450822|14234|2|305| +2450822|14236|2|| +2450822|14239|2|218| +2450822|14240|2|586| +2450822|14242|2|918| +2450822|14245|2|416| +2450822|14246|2|960| +2450822|14248|2|174| +2450822|14251|2|298| +2450822|14252|2|304| +2450822|14254|2|970| +2450822|14257|2|134| +2450822|14258|2|297| +2450822|14260|2|866| +2450822|14263|2|| +2450822|14264|2|151| +2450822|14266|2|| +2450822|14269|2|320| +2450822|14270|2|263| +2450822|14272|2|265| +2450822|14275|2|335| +2450822|14276|2|961| +2450822|14278|2|211| +2450822|14281|2|63| +2450822|14282|2|304| +2450822|14284|2|36| +2450822|14287|2|318| +2450822|14288|2|462| +2450822|14290|2|94| +2450822|14293|2|781| +2450822|14294|2|910| +2450822|14296|2|399| +2450822|14299|2|590| +2450822|14300|2|172| +2450822|14302|2|951| +2450822|14305|2|587| +2450822|14306|2|704| +2450822|14308|2|435| +2450822|14311|2|566| +2450822|14312|2|521| +2450822|14314|2|431| +2450822|14317|2|470| +2450822|14318|2|518| +2450822|14320|2|862| +2450822|14323|2|115| +2450822|14324|2|290| +2450822|14326|2|852| +2450822|14329|2|300| +2450822|14330|2|| +2450822|14332|2|884| +2450822|14335|2|909| +2450822|14336|2|582| +2450822|14338|2|279| +2450822|14341|2|317| +2450822|14342|2|558| +2450822|14344|2|815| +2450822|14347|2|390| +2450822|14348|2|871| +2450822|14350|2|437| +2450822|14353|2|18| +2450822|14354|2|153| +2450822|14356|2|654| +2450822|14359|2|166| +2450822|14360|2|311| +2450822|14362|2|996| +2450822|14365|2|961| +2450822|14366|2|278| +2450822|14368|2|262| +2450822|14371|2|290| +2450822|14372|2|292| +2450822|14374|2|430| +2450822|14377|2|962| +2450822|14378|2|845| +2450822|14380|2|| +2450822|14383|2|744| +2450822|14384|2|475| +2450822|14386|2|831| +2450822|14389|2|897| +2450822|14390|2|275| +2450822|14392|2|351| +2450822|14395|2|497| +2450822|14396|2|405| +2450822|14398|2|563| +2450822|14401|2|903| +2450822|14402|2|956| +2450822|14404|2|184| +2450822|14407|2|836| +2450822|14408|2|650| +2450822|14410|2|335| +2450822|14413|2|| +2450822|14414|2|501| +2450822|14416|2|184| +2450822|14419|2|719| +2450822|14420|2|293| +2450822|14422|2|968| +2450822|14425|2|544| +2450822|14426|2|321| +2450822|14428|2|363| +2450822|14431|2|985| +2450822|14432|2|835| +2450822|14434|2|517| +2450822|14437|2|239| +2450822|14438|2|470| +2450822|14440|2|807| +2450822|14443|2|107| +2450822|14444|2|786| +2450822|14446|2|41| +2450822|14449|2|266| +2450822|14450|2|736| +2450822|14452|2|885| +2450822|14455|2|534| +2450822|14456|2|440| +2450822|14458|2|867| +2450822|14461|2|742| +2450822|14462|2|175| +2450822|14464|2|822| +2450822|14467|2|929| +2450822|14468|2|719| +2450822|14470|2|972| +2450822|14473|2|747| +2450822|14474|2|642| +2450822|14476|2|407| +2450822|14479|2|772| +2450822|14480|2|544| +2450822|14482|2|549| +2450822|14485|2|324| +2450822|14486|2|986| +2450822|14488|2|658| +2450822|14491|2|321| +2450822|14492|2|927| +2450822|14494|2|494| +2450822|14497|2|973| +2450822|14498|2|454| +2450822|14500|2|222| +2450822|14503|2|484| +2450822|14504|2|202| +2450822|14506|2|742| +2450822|14509|2|666| +2450822|14510|2|608| +2450822|14512|2|| +2450822|14515|2|864| +2450822|14516|2|876| +2450822|14518|2|721| +2450822|14521|2|565| +2450822|14522|2|281| +2450822|14524|2|872| +2450822|14527|2|887| +2450822|14528|2|957| +2450822|14530|2|635| +2450822|14533|2|438| +2450822|14534|2|| +2450822|14536|2|280| +2450822|14539|2|729| +2450822|14540|2|577| +2450822|14542|2|287| +2450822|14545|2|923| +2450822|14546|2|6| +2450822|14548|2|531| +2450822|14551|2|670| +2450822|14552|2|487| +2450822|14554|2|79| +2450822|14557|2|836| +2450822|14558|2|527| +2450822|14560|2|373| +2450822|14563|2|228| +2450822|14564|2|388| +2450822|14566|2|522| +2450822|14569|2|566| +2450822|14570|2|| +2450822|14572|2|87| +2450822|14575|2|| +2450822|14576|2|326| +2450822|14578|2|435| +2450822|14581|2|993| +2450822|14582|2|914| +2450822|14584|2|451| +2450822|14587|2|562| +2450822|14588|2|60| +2450822|14590|2|384| +2450822|14593|2|| +2450822|14594|2|274| +2450822|14596|2|553| +2450822|14599|2|313| +2450822|14600|2|88| +2450822|14602|2|| +2450822|14605|2|361| +2450822|14606|2|622| +2450822|14608|2|543| +2450822|14611|2|149| +2450822|14612|2|435| +2450822|14614|2|660| +2450822|14617|2|815| +2450822|14618|2|661| +2450822|14620|2|433| +2450822|14623|2|648| +2450822|14624|2|141| +2450822|14626|2|679| +2450822|14629|2|569| +2450822|14630|2|329| +2450822|14632|2|433| +2450822|14635|2|826| +2450822|14636|2|165| +2450822|14638|2|107| +2450822|14641|2|840| +2450822|14642|2|470| +2450822|14644|2|804| +2450822|14647|2|337| +2450822|14648|2|706| +2450822|14650|2|704| +2450822|14653|2|926| +2450822|14654|2|338| +2450822|14656|2|438| +2450822|14659|2|196| +2450822|14660|2|160| +2450822|14662|2|116| +2450822|14665|2|150| +2450822|14666|2|924| +2450822|14668|2|263| +2450822|14671|2|754| +2450822|14672|2|73| +2450822|14674|2|296| +2450822|14677|2|678| +2450822|14678|2|724| +2450822|14680|2|| +2450822|14683|2|204| +2450822|14684|2|458| +2450822|14686|2|126| +2450822|14689|2|452| +2450822|14690|2|750| +2450822|14692|2|114| +2450822|14695|2|74| +2450822|14696|2|850| +2450822|14698|2|829| +2450822|14701|2|33| +2450822|14702|2|251| +2450822|14704|2|22| +2450822|14707|2|563| +2450822|14708|2|569| +2450822|14710|2|220| +2450822|14713|2|295| +2450822|14714|2|568| +2450822|14716|2|964| +2450822|14719|2|341| +2450822|14720|2|137| +2450822|14722|2|999| +2450822|14725|2|196| +2450822|14726|2|881| +2450822|14728|2|868| +2450822|14731|2|293| +2450822|14732|2|539| +2450822|14734|2|876| +2450822|14737|2|743| +2450822|14738|2|| +2450822|14740|2|623| +2450822|14743|2|288| +2450822|14744|2|79| +2450822|14746|2|293| +2450822|14749|2|329| +2450822|14750|2|218| +2450822|14752|2|476| +2450822|14755|2|196| +2450822|14756|2|633| +2450822|14758|2|516| +2450822|14761|2|294| +2450822|14762|2|956| +2450822|14764|2|185| +2450822|14767|2|441| +2450822|14768|2|228| +2450822|14770|2|912| +2450822|14773|2|| +2450822|14774|2|873| +2450822|14776|2|662| +2450822|14779|2|257| +2450822|14780|2|256| +2450822|14782|2|515| +2450822|14785|2|96| +2450822|14786|2|824| +2450822|14788|2|523| +2450822|14791|2|94| +2450822|14792|2|543| +2450822|14794|2|765| +2450822|14797|2|599| +2450822|14798|2|229| +2450822|14800|2|200| +2450822|14803|2|721| +2450822|14804|2|151| +2450822|14806|2|35| +2450822|14809|2|411| +2450822|14810|2|110| +2450822|14812|2|425| +2450822|14815|2|163| +2450822|14816|2|272| +2450822|14818|2|78| +2450822|14821|2|370| +2450822|14822|2|981| +2450822|14824|2|459| +2450822|14827|2|743| +2450822|14828|2|176| +2450822|14830|2|372| +2450822|14833|2|76| +2450822|14834|2|656| +2450822|14836|2|172| +2450822|14839|2|753| +2450822|14840|2|306| +2450822|14842|2|761| +2450822|14845|2|91| +2450822|14846|2|958| +2450822|14848|2|512| +2450822|14851|2|600| +2450822|14852|2|415| +2450822|14854|2|| +2450822|14857|2|314| +2450822|14858|2|188| +2450822|14860|2|674| +2450822|14863|2|779| +2450822|14864|2|699| +2450822|14866|2|132| +2450822|14869|2|966| +2450822|14870|2|979| +2450822|14872|2|847| +2450822|14875|2|473| +2450822|14876|2|229| +2450822|14878|2|775| +2450822|14881|2|430| +2450822|14882|2|586| +2450822|14884|2|317| +2450822|14887|2|56| +2450822|14888|2|648| +2450822|14890|2|868| +2450822|14893|2|945| +2450822|14894|2|829| +2450822|14896|2|| +2450822|14899|2|208| +2450822|14900|2|477| +2450822|14902|2|237| +2450822|14905|2|467| +2450822|14906|2|954| +2450822|14908|2|309| +2450822|14911|2|518| +2450822|14912|2|506| +2450822|14914|2|329| +2450822|14917|2|654| +2450822|14918|2|0| +2450822|14920|2|844| +2450822|14923|2|380| +2450822|14924|2|660| +2450822|14926|2|419| +2450822|14929|2|| +2450822|14930|2|918| +2450822|14932|2|324| +2450822|14935|2|3| +2450822|14936|2|91| +2450822|14938|2|566| +2450822|14941|2|728| +2450822|14942|2|597| +2450822|14944|2|280| +2450822|14947|2|| +2450822|14948|2|77| +2450822|14950|2|| +2450822|14953|2|124| +2450822|14954|2|478| +2450822|14956|2|878| +2450822|14959|2|247| +2450822|14960|2|438| +2450822|14962|2|753| +2450822|14965|2|134| +2450822|14966|2|662| +2450822|14968|2|251| +2450822|14971|2|774| +2450822|14972|2|441| +2450822|14974|2|567| +2450822|14977|2|644| +2450822|14978|2|42| +2450822|14980|2|473| +2450822|14983|2|848| +2450822|14984|2|627| +2450822|14986|2|508| +2450822|14989|2|579| +2450822|14990|2|397| +2450822|14992|2|| +2450822|14995|2|871| +2450822|14996|2|616| +2450822|14998|2|958| +2450822|15001|2|559| +2450822|15002|2|238| +2450822|15004|2|738| +2450822|15007|2|280| +2450822|15008|2|945| +2450822|15010|2|567| +2450822|15013|2|285| +2450822|15014|2|543| +2450822|15016|2|298| +2450822|15019|2|651| +2450822|15020|2|821| +2450822|15022|2|350| +2450822|15025|2|110| +2450822|15026|2|| +2450822|15028|2|477| +2450822|15031|2|639| +2450822|15032|2|538| +2450822|15034|2|69| +2450822|15037|2|769| +2450822|15038|2|833| +2450822|15040|2|145| +2450822|15043|2|304| +2450822|15044|2|965| +2450822|15046|2|48| +2450822|15049|2|596| +2450822|15050|2|37| +2450822|15052|2|426| +2450822|15055|2|455| +2450822|15056|2|870| +2450822|15058|2|675| +2450822|15061|2|153| +2450822|15062|2|455| +2450822|15064|2|367| +2450822|15067|2|749| +2450822|15068|2|886| +2450822|15070|2|75| +2450822|15073|2|869| +2450822|15074|2|395| +2450822|15076|2|541| +2450822|15079|2|361| +2450822|15080|2|126| +2450822|15082|2|578| +2450822|15085|2|740| +2450822|15086|2|449| +2450822|15088|2|304| +2450822|15091|2|537| +2450822|15092|2|198| +2450822|15094|2|881| +2450822|15097|2|992| +2450822|15098|2|315| +2450822|15100|2|579| +2450822|15103|2|962| +2450822|15104|2|536| +2450822|15106|2|517| +2450822|15109|2|427| +2450822|15110|2|187| +2450822|15112|2|597| +2450822|15115|2|321| +2450822|15116|2|158| +2450822|15118|2|211| +2450822|15121|2|705| +2450822|15122|2|432| +2450822|15124|2|521| +2450822|15127|2|591| +2450822|15128|2|25| +2450822|15130|2|544| +2450822|15133|2|247| +2450822|15134|2|850| +2450822|15136|2|258| +2450822|15139|2|512| +2450822|15140|2|556| +2450822|15142|2|728| +2450822|15145|2|23| +2450822|15146|2|302| +2450822|15148|2|44| +2450822|15151|2|185| +2450822|15152|2|987| +2450822|15154|2|904| +2450822|15157|2|| +2450822|15158|2|59| +2450822|15160|2|406| +2450822|15163|2|434| +2450822|15164|2|579| +2450822|15166|2|753| +2450822|15169|2|856| +2450822|15170|2|780| +2450822|15172|2|24| +2450822|15175|2|518| +2450822|15176|2|400| +2450822|15178|2|359| +2450822|15181|2|825| +2450822|15182|2|758| +2450822|15184|2|163| +2450822|15187|2|940| +2450822|15188|2|468| +2450822|15190|2|351| +2450822|15193|2|| +2450822|15194|2|433| +2450822|15196|2|861| +2450822|15199|2|674| +2450822|15200|2|123| +2450822|15202|2|| +2450822|15205|2|329| +2450822|15206|2|409| +2450822|15208|2|140| +2450822|15211|2|525| +2450822|15212|2|566| +2450822|15214|2|863| +2450822|15217|2|462| +2450822|15218|2|241| +2450822|15220|2|847| +2450822|15223|2|| +2450822|15224|2|852| +2450822|15226|2|574| +2450822|15229|2|616| +2450822|15230|2|81| +2450822|15232|2|922| +2450822|15235|2|17| +2450822|15236|2|203| +2450822|15238|2|120| +2450822|15241|2|860| +2450822|15242|2|| +2450822|15244|2|82| +2450822|15247|2|| +2450822|15248|2|913| +2450822|15250|2|381| +2450822|15253|2|912| +2450822|15254|2|265| +2450822|15256|2|587| +2450822|15259|2|60| +2450822|15260|2|651| +2450822|15262|2|701| +2450822|15265|2|643| +2450822|15266|2|113| +2450822|15268|2|597| +2450822|15271|2|903| +2450822|15272|2|653| +2450822|15274|2|264| +2450822|15277|2|816| +2450822|15278|2|255| +2450822|15280|2|361| +2450822|15283|2|596| +2450822|15284|2|598| +2450822|15286|2|587| +2450822|15289|2|754| +2450822|15290|2|886| +2450822|15292|2|29| +2450822|15295|2|652| +2450822|15296|2|682| +2450822|15298|2|30| +2450822|15301|2|70| +2450822|15302|2|423| +2450822|15304|2|510| +2450822|15307|2|978| +2450822|15308|2|474| +2450822|15310|2|584| +2450822|15313|2|728| +2450822|15314|2|464| +2450822|15316|2|178| +2450822|15319|2|788| +2450822|15320|2|| +2450822|15322|2|199| +2450822|15325|2|295| +2450822|15326|2|| +2450822|15328|2|317| +2450822|15331|2|| +2450822|15332|2|763| +2450822|15334|2|| +2450822|15337|2|132| +2450822|15338|2|465| +2450822|15340|2|930| +2450822|15343|2|122| +2450822|15344|2|763| +2450822|15346|2|103| +2450822|15349|2|903| +2450822|15350|2|194| +2450822|15352|2|12| +2450822|15355|2|135| +2450822|15356|2|371| +2450822|15358|2|640| +2450822|15361|2|743| +2450822|15362|2|638| +2450822|15364|2|865| +2450822|15367|2|580| +2450822|15368|2|810| +2450822|15370|2|755| +2450822|15373|2|373| +2450822|15374|2|796| +2450822|15376|2|82| +2450822|15379|2|821| +2450822|15380|2|527| +2450822|15382|2|6| +2450822|15385|2|694| +2450822|15386|2|203| +2450822|15388|2|967| +2450822|15391|2|349| +2450822|15392|2|269| +2450822|15394|2|1000| +2450822|15397|2|101| +2450822|15398|2|158| +2450822|15400|2|904| +2450822|15403|2|410| +2450822|15404|2|421| +2450822|15406|2|441| +2450822|15409|2|806| +2450822|15410|2|563| +2450822|15412|2|589| +2450822|15415|2|926| +2450822|15416|2|193| +2450822|15418|2|876| +2450822|15421|2|795| +2450822|15422|2|| +2450822|15424|2|791| +2450822|15427|2|479| +2450822|15428|2|394| +2450822|15430|2|881| +2450822|15433|2|595| +2450822|15434|2|373| +2450822|15436|2|279| +2450822|15439|2|160| +2450822|15440|2|743| +2450822|15442|2|370| +2450822|15445|2|341| +2450822|15446|2|544| +2450822|15448|2|343| +2450822|15451|2|832| +2450822|15452|2|137| +2450822|15454|2|967| +2450822|15457|2|259| +2450822|15458|2|921| +2450822|15460|2|821| +2450822|15463|2|635| +2450822|15464|2|832| +2450822|15466|2|2| +2450822|15469|2|337| +2450822|15470|2|647| +2450822|15472|2|960| +2450822|15475|2|42| +2450822|15476|2|706| +2450822|15478|2|955| +2450822|15481|2|644| +2450822|15482|2|75| +2450822|15484|2|56| +2450822|15487|2|866| +2450822|15488|2|751| +2450822|15490|2|553| +2450822|15493|2|298| +2450822|15494|2|420| +2450822|15496|2|781| +2450822|15499|2|857| +2450822|15500|2|| +2450822|15502|2|885| +2450822|15505|2|807| +2450822|15506|2|370| +2450822|15508|2|468| +2450822|15511|2|| +2450822|15512|2|597| +2450822|15514|2|138| +2450822|15517|2|905| +2450822|15518|2|161| +2450822|15520|2|842| +2450822|15523|2|255| +2450822|15524|2|352| +2450822|15526|2|708| +2450822|15529|2|30| +2450822|15530|2|201| +2450822|15532|2|94| +2450822|15535|2|917| +2450822|15536|2|| +2450822|15538|2|691| +2450822|15541|2|467| +2450822|15542|2|827| +2450822|15544|2|66| +2450822|15547|2|598| +2450822|15548|2|421| +2450822|15550|2|98| +2450822|15553|2|297| +2450822|15554|2|503| +2450822|15556|2|253| +2450822|15559|2|142| +2450822|15560|2|178| +2450822|15562|2|540| +2450822|15565|2|736| +2450822|15566|2|499| +2450822|15568|2|| +2450822|15571|2|762| +2450822|15572|2|| +2450822|15574|2|567| +2450822|15577|2|306| +2450822|15578|2|325| +2450822|15580|2|355| +2450822|15583|2|415| +2450822|15584|2|38| +2450822|15586|2|688| +2450822|15589|2|489| +2450822|15590|2|883| +2450822|15592|2|91| +2450822|15595|2|466| +2450822|15596|2|735| +2450822|15598|2|344| +2450822|15601|2|| +2450822|15602|2|780| +2450822|15604|2|818| +2450822|15607|2|173| +2450822|15608|2|878| +2450822|15610|2|500| +2450822|15613|2|595| +2450822|15614|2|785| +2450822|15616|2|825| +2450822|15619|2|671| +2450822|15620|2|753| +2450822|15622|2|279| +2450822|15625|2|470| +2450822|15626|2|476| +2450822|15628|2|669| +2450822|15631|2|888| +2450822|15632|2|344| +2450822|15634|2|841| +2450822|15637|2|512| +2450822|15638|2|953| +2450822|15640|2|429| +2450822|15643|2|240| +2450822|15644|2|237| +2450822|15646|2|753| +2450822|15649|2|591| +2450822|15650|2|135| +2450822|15652|2|682| +2450822|15655|2|328| +2450822|15656|2|987| +2450822|15658|2|701| +2450822|15661|2|809| +2450822|15662|2|94| +2450822|15664|2|795| +2450822|15667|2|782| +2450822|15668|2|99| +2450822|15670|2|578| +2450822|15673|2|537| +2450822|15674|2|851| +2450822|15676|2|432| +2450822|15679|2|803| +2450822|15680|2|233| +2450822|15682|2|416| +2450822|15685|2|672| +2450822|15686|2|334| +2450822|15688|2|894| +2450822|15691|2|337| +2450822|15692|2|909| +2450822|15694|2|831| +2450822|15697|2|147| +2450822|15698|2|762| +2450822|15700|2|580| +2450822|15703|2|292| +2450822|15704|2|505| +2450822|15706|2|818| +2450822|15709|2|29| +2450822|15710|2|719| +2450822|15712|2|6| +2450822|15715|2|| +2450822|15716|2|572| +2450822|15718|2|285| +2450822|15721|2|164| +2450822|15722|2|644| +2450822|15724|2|485| +2450822|15727|2|83| +2450822|15728|2|| +2450822|15730|2|455| +2450822|15733|2|| +2450822|15734|2|972| +2450822|15736|2|737| +2450822|15739|2|250| +2450822|15740|2|84| +2450822|15742|2|119| +2450822|15745|2|734| +2450822|15746|2|311| +2450822|15748|2|840| +2450822|15751|2|945| +2450822|15752|2|322| +2450822|15754|2|177| +2450822|15757|2|962| +2450822|15758|2|388| +2450822|15760|2|61| +2450822|15763|2|561| +2450822|15764|2|879| +2450822|15766|2|921| +2450822|15769|2|30| +2450822|15770|2|760| +2450822|15772|2|472| +2450822|15775|2|448| +2450822|15776|2|319| +2450822|15778|2|229| +2450822|15781|2|709| +2450822|15782|2|216| +2450822|15784|2|395| +2450822|15787|2|709| +2450822|15788|2|241| +2450822|15790|2|116| +2450822|15793|2|822| +2450822|15794|2|129| +2450822|15796|2|589| +2450822|15799|2|131| +2450822|15800|2|344| +2450822|15802|2|25| +2450822|15805|2|301| +2450822|15806|2|433| +2450822|15808|2|817| +2450822|15811|2|843| +2450822|15812|2|780| +2450822|15814|2|946| +2450822|15817|2|690| +2450822|15818|2|461| +2450822|15820|2|215| +2450822|15823|2|172| +2450822|15824|2|98| +2450822|15826|2|| +2450822|15829|2|155| +2450822|15830|2|466| +2450822|15832|2|672| +2450822|15835|2|262| +2450822|15836|2|251| +2450822|15838|2|763| +2450822|15841|2|54| +2450822|15842|2|44| +2450822|15844|2|267| +2450822|15847|2|465| +2450822|15848|2|306| +2450822|15850|2|519| +2450822|15853|2|493| +2450822|15854|2|323| +2450822|15856|2|60| +2450822|15859|2|364| +2450822|15860|2|615| +2450822|15862|2|691| +2450822|15865|2|345| +2450822|15866|2|| +2450822|15868|2|964| +2450822|15871|2|630| +2450822|15872|2|530| +2450822|15874|2|753| +2450822|15877|2|23| +2450822|15878|2|830| +2450822|15880|2|29| +2450822|15883|2|956| +2450822|15884|2|270| +2450822|15886|2|891| +2450822|15889|2|970| +2450822|15890|2|744| +2450822|15892|2|92| +2450822|15895|2|778| +2450822|15896|2|903| +2450822|15898|2|594| +2450822|15901|2|704| +2450822|15902|2|629| +2450822|15904|2|680| +2450822|15907|2|586| +2450822|15908|2|263| +2450822|15910|2|290| +2450822|15913|2|337| +2450822|15914|2|469| +2450822|15916|2|1| +2450822|15919|2|538| +2450822|15920|2|943| +2450822|15922|2|| +2450822|15925|2|665| +2450822|15926|2|518| +2450822|15928|2|97| +2450822|15931|2|38| +2450822|15932|2|745| +2450822|15934|2|552| +2450822|15937|2|727| +2450822|15938|2|628| +2450822|15940|2|| +2450822|15943|2|277| +2450822|15944|2|865| +2450822|15946|2|| +2450822|15949|2|776| +2450822|15950|2|694| +2450822|15952|2|341| +2450822|15955|2|225| +2450822|15956|2|336| +2450822|15958|2|955| +2450822|15961|2|856| +2450822|15962|2|738| +2450822|15964|2|465| +2450822|15967|2|643| +2450822|15968|2|519| +2450822|15970|2|591| +2450822|15973|2|| +2450822|15974|2|142| +2450822|15976|2|296| +2450822|15979|2|889| +2450822|15980|2|294| +2450822|15982|2|994| +2450822|15985|2|458| +2450822|15986|2|418| +2450822|15988|2|378| +2450822|15991|2|729| +2450822|15992|2|594| +2450822|15994|2|691| +2450822|15997|2|206| +2450822|15998|2|533| +2450822|16000|2|192| +2450822|16003|2|16| +2450822|16004|2|618| +2450822|16006|2|413| +2450822|16009|2|793| +2450822|16010|2|42| +2450822|16012|2|918| +2450822|16015|2|412| +2450822|16016|2|986| +2450822|16018|2|535| +2450822|16021|2|618| +2450822|16022|2|545| +2450822|16024|2|895| +2450822|16027|2|752| +2450822|16028|2|643| +2450822|16030|2|| +2450822|16033|2|883| +2450822|16034|2|3| +2450822|16036|2|385| +2450822|16039|2|520| +2450822|16040|2|859| +2450822|16042|2|395| +2450822|16045|2|901| +2450822|16046|2|777| +2450822|16048|2|14| +2450822|16051|2|| +2450822|16052|2|669| +2450822|16054|2|843| +2450822|16057|2|155| +2450822|16058|2|607| +2450822|16060|2|730| +2450822|16063|2|| +2450822|16064|2|| +2450822|16066|2|100| +2450822|16069|2|246| +2450822|16070|2|832| +2450822|16072|2|139| +2450822|16075|2|741| +2450822|16076|2|153| +2450822|16078|2|462| +2450822|16081|2|484| +2450822|16082|2|462| +2450822|16084|2|964| +2450822|16087|2|| +2450822|16088|2|986| +2450822|16090|2|743| +2450822|16093|2|556| +2450822|16094|2|658| +2450822|16096|2|585| +2450822|16099|2|403| +2450822|16100|2|645| +2450822|16102|2|498| +2450822|16105|2|856| +2450822|16106|2|705| +2450822|16108|2|369| +2450822|16111|2|177| +2450822|16112|2|321| +2450822|16114|2|573| +2450822|16117|2|506| +2450822|16118|2|621| +2450822|16120|2|904| +2450822|16123|2|262| +2450822|16124|2|695| +2450822|16126|2|275| +2450822|16129|2|672| +2450822|16130|2|973| +2450822|16132|2|239| +2450822|16135|2|397| +2450822|16136|2|658| +2450822|16138|2|370| +2450822|16141|2|787| +2450822|16142|2|748| +2450822|16144|2|501| +2450822|16147|2|744| +2450822|16148|2|193| +2450822|16150|2|960| +2450822|16153|2|181| +2450822|16154|2|| +2450822|16156|2|624| +2450822|16159|2|969| +2450822|16160|2|386| +2450822|16162|2|190| +2450822|16165|2|36| +2450822|16166|2|957| +2450822|16168|2|650| +2450822|16171|2|867| +2450822|16172|2|522| +2450822|16174|2|343| +2450822|16177|2|751| +2450822|16178|2|| +2450822|16180|2|905| +2450822|16183|2|428| +2450822|16184|2|341| +2450822|16186|2|503| +2450822|16189|2|528| +2450822|16190|2|599| +2450822|16192|2|950| +2450822|16195|2|986| +2450822|16196|2|366| +2450822|16198|2|117| +2450822|16201|2|874| +2450822|16202|2|697| +2450822|16204|2|982| +2450822|16207|2|876| +2450822|16208|2|745| +2450822|16210|2|920| +2450822|16213|2|822| +2450822|16214|2|459| +2450822|16216|2|306| +2450822|16219|2|830| +2450822|16220|2|197| +2450822|16222|2|32| +2450822|16225|2|440| +2450822|16226|2|794| +2450822|16228|2|14| +2450822|16231|2|763| +2450822|16232|2|552| +2450822|16234|2|18| +2450822|16237|2|429| +2450822|16238|2|488| +2450822|16240|2|543| +2450822|16243|2|601| +2450822|16244|2|331| +2450822|16246|2|598| +2450822|16249|2|917| +2450822|16250|2|935| +2450822|16252|2|453| +2450822|16255|2|186| +2450822|16256|2|125| +2450822|16258|2|529| +2450822|16261|2|3| +2450822|16262|2|543| +2450822|16264|2|463| +2450822|16267|2|470| +2450822|16268|2|556| +2450822|16270|2|27| +2450822|16273|2|220| +2450822|16274|2|825| +2450822|16276|2|538| +2450822|16279|2|388| +2450822|16280|2|79| +2450822|16282|2|559| +2450822|16285|2|313| +2450822|16286|2|186| +2450822|16288|2|588| +2450822|16291|2|77| +2450822|16292|2|471| +2450822|16294|2|975| +2450822|16297|2|534| +2450822|16298|2|202| +2450822|16300|2|589| +2450822|16303|2|465| +2450822|16304|2|642| +2450822|16306|2|636| +2450822|16309|2|912| +2450822|16310|2|457| +2450822|16312|2|359| +2450822|16315|2|217| +2450822|16316|2|35| +2450822|16318|2|517| +2450822|16321|2|573| +2450822|16322|2|99| +2450822|16324|2|587| +2450822|16327|2|56| +2450822|16328|2|949| +2450822|16330|2|336| +2450822|16333|2|725| +2450822|16334|2|278| +2450822|16336|2|951| +2450822|16339|2|497| +2450822|16340|2|797| +2450822|16342|2|893| +2450822|16345|2|457| +2450822|16346|2|395| +2450822|16348|2|107| +2450822|16351|2|| +2450822|16352|2|204| +2450822|16354|2|355| +2450822|16357|2|68| +2450822|16358|2|589| +2450822|16360|2|800| +2450822|16363|2|218| +2450822|16364|2|718| +2450822|16366|2|285| +2450822|16369|2|520| +2450822|16370|2|257| +2450822|16372|2|300| +2450822|16375|2|| +2450822|16376|2|208| +2450822|16378|2|308| +2450822|16381|2|56| +2450822|16382|2|313| +2450822|16384|2|999| +2450822|16387|2|100| +2450822|16388|2|720| +2450822|16390|2|889| +2450822|16393|2|34| +2450822|16394|2|832| +2450822|16396|2|952| +2450822|16399|2|480| +2450822|16400|2|273| +2450822|16402|2|446| +2450822|16405|2|643| +2450822|16406|2|796| +2450822|16408|2|963| +2450822|16411|2|498| +2450822|16412|2|528| +2450822|16414|2|651| +2450822|16417|2|700| +2450822|16418|2|727| +2450822|16420|2|454| +2450822|16423|2|901| +2450822|16424|2|189| +2450822|16426|2|771| +2450822|16429|2|12| +2450822|16430|2|302| +2450822|16432|2|811| +2450822|16435|2|310| +2450822|16436|2|475| +2450822|16438|2|922| +2450822|16441|2|363| +2450822|16442|2|241| +2450822|16444|2|899| +2450822|16447|2|23| +2450822|16448|2|638| +2450822|16450|2|339| +2450822|16453|2|12| +2450822|16454|2|10| +2450822|16456|2|976| +2450822|16459|2|312| +2450822|16460|2|199| +2450822|16462|2|164| +2450822|16465|2|890| +2450822|16466|2|812| +2450822|16468|2|385| +2450822|16471|2|301| +2450822|16472|2|242| +2450822|16474|2|732| +2450822|16477|2|142| +2450822|16478|2|407| +2450822|16480|2|10| +2450822|16483|2|190| +2450822|16484|2|873| +2450822|16486|2|104| +2450822|16489|2|943| +2450822|16490|2|894| +2450822|16492|2|324| +2450822|16495|2|624| +2450822|16496|2|198| +2450822|16498|2|97| +2450822|16501|2|| +2450822|16502|2|562| +2450822|16504|2|981| +2450822|16507|2|681| +2450822|16508|2|911| +2450822|16510|2|462| +2450822|16513|2|169| +2450822|16514|2|| +2450822|16516|2|114| +2450822|16519|2|| +2450822|16520|2|648| +2450822|16522|2|924| +2450822|16525|2|403| +2450822|16526|2|53| +2450822|16528|2|617| +2450822|16531|2|518| +2450822|16532|2|112| +2450822|16534|2|531| +2450822|16537|2|275| +2450822|16538|2|927| +2450822|16540|2|485| +2450822|16543|2|983| +2450822|16544|2|873| +2450822|16546|2|89| +2450822|16549|2|622| +2450822|16550|2|768| +2450822|16552|2|955| +2450822|16555|2|473| +2450822|16556|2|746| +2450822|16558|2|900| +2450822|16561|2|557| +2450822|16562|2|79| +2450822|16564|2|| +2450822|16567|2|63| +2450822|16568|2|46| +2450822|16570|2|658| +2450822|16573|2|967| +2450822|16574|2|798| +2450822|16576|2|487| +2450822|16579|2|397| +2450822|16580|2|757| +2450822|16582|2|782| +2450822|16585|2|117| +2450822|16586|2|431| +2450822|16588|2|374| +2450822|16591|2|280| +2450822|16592|2|520| +2450822|16594|2|377| +2450822|16597|2|285| +2450822|16598|2|864| +2450822|16600|2|468| +2450822|16603|2|771| +2450822|16604|2|750| +2450822|16606|2|963| +2450822|16609|2|800| +2450822|16610|2|763| +2450822|16612|2|2| +2450822|16615|2|408| +2450822|16616|2|| +2450822|16618|2|138| +2450822|16621|2|394| +2450822|16622|2|139| +2450822|16624|2|270| +2450822|16627|2|134| +2450822|16628|2|352| +2450822|16630|2|461| +2450822|16633|2|370| +2450822|16634|2|127| +2450822|16636|2|185| +2450822|16639|2|831| +2450822|16640|2|983| +2450822|16642|2|387| +2450822|16645|2|494| +2450822|16646|2|| +2450822|16648|2|915| +2450822|16651|2|| +2450822|16652|2|912| +2450822|16654|2|626| +2450822|16657|2|328| +2450822|16658|2|636| +2450822|16660|2|832| +2450822|16663|2|253| +2450822|16664|2|434| +2450822|16666|2|387| +2450822|16669|2|| +2450822|16670|2|435| +2450822|16672|2|399| +2450822|16675|2|189| +2450822|16676|2|153| +2450822|16678|2|256| +2450822|16681|2|400| +2450822|16682|2|173| +2450822|16684|2|740| +2450822|16687|2|248| +2450822|16688|2|664| +2450822|16690|2|943| +2450822|16693|2|533| +2450822|16694|2|20| +2450822|16696|2|870| +2450822|16699|2|242| +2450822|16700|2|194| +2450822|16702|2|357| +2450822|16705|2|74| +2450822|16706|2|433| +2450822|16708|2|652| +2450822|16711|2|376| +2450822|16712|2|377| +2450822|16714|2|477| +2450822|16717|2|403| +2450822|16718|2|483| +2450822|16720|2|555| +2450822|16723|2|442| +2450822|16724|2|602| +2450822|16726|2|214| +2450822|16729|2|335| +2450822|16730|2|572| +2450822|16732|2|731| +2450822|16735|2|720| +2450822|16736|2|581| +2450822|16738|2|921| +2450822|16741|2|504| +2450822|16742|2|35| +2450822|16744|2|776| +2450822|16747|2|541| +2450822|16748|2|846| +2450822|16750|2|862| +2450822|16753|2|560| +2450822|16754|2|| +2450822|16756|2|351| +2450822|16759|2|176| +2450822|16760|2|| +2450822|16762|2|123| +2450822|16765|2|70| +2450822|16766|2|667| +2450822|16768|2|967| +2450822|16771|2|792| +2450822|16772|2|| +2450822|16774|2|563| +2450822|16777|2|100| +2450822|16778|2|258| +2450822|16780|2|793| +2450822|16783|2|821| +2450822|16784|2|49| +2450822|16786|2|433| +2450822|16789|2|69| +2450822|16790|2|922| +2450822|16792|2|512| +2450822|16795|2|381| +2450822|16796|2|219| +2450822|16798|2|895| +2450822|16801|2|468| +2450822|16802|2|534| +2450822|16804|2|701| +2450822|16807|2|9| +2450822|16808|2|442| +2450822|16810|2|111| +2450822|16813|2|760| +2450822|16814|2|679| +2450822|16816|2|848| +2450822|16819|2|566| +2450822|16820|2|885| +2450822|16822|2|959| +2450822|16825|2|| +2450822|16826|2|963| +2450822|16828|2|640| +2450822|16831|2|494| +2450822|16832|2|189| +2450822|16834|2|| +2450822|16837|2|918| +2450822|16838|2|431| +2450822|16840|2|87| +2450822|16843|2|176| +2450822|16844|2|19| +2450822|16846|2|| +2450822|16849|2|164| +2450822|16850|2|604| +2450822|16852|2|893| +2450822|16855|2|981| +2450822|16856|2|599| +2450822|16858|2|211| +2450822|16861|2|133| +2450822|16862|2|662| +2450822|16864|2|323| +2450822|16867|2|719| +2450822|16868|2|917| +2450822|16870|2|| +2450822|16873|2|| +2450822|16874|2|859| +2450822|16876|2|214| +2450822|16879|2|955| +2450822|16880|2|770| +2450822|16882|2|984| +2450822|16885|2|486| +2450822|16886|2|10| +2450822|16888|2|247| +2450822|16891|2|378| +2450822|16892|2|314| +2450822|16894|2|295| +2450822|16897|2|270| +2450822|16898|2|205| +2450822|16900|2|266| +2450822|16903|2|369| +2450822|16904|2|106| +2450822|16906|2|620| +2450822|16909|2|262| +2450822|16910|2|923| +2450822|16912|2|59| +2450822|16915|2|521| +2450822|16916|2|193| +2450822|16918|2|384| +2450822|16921|2|559| +2450822|16922|2|899| +2450822|16924|2|699| +2450822|16927|2|672| +2450822|16928|2|35| +2450822|16930|2|987| +2450822|16933|2|| +2450822|16934|2|24| +2450822|16936|2|946| +2450822|16939|2|922| +2450822|16940|2|163| +2450822|16942|2|978| +2450822|16945|2|691| +2450822|16946|2|385| +2450822|16948|2|815| +2450822|16951|2|100| +2450822|16952|2|260| +2450822|16954|2|106| +2450822|16957|2|679| +2450822|16958|2|260| +2450822|16960|2|81| +2450822|16963|2|587| +2450822|16964|2|985| +2450822|16966|2|| +2450822|16969|2|549| +2450822|16970|2|574| +2450822|16972|2|955| +2450822|16975|2|562| +2450822|16976|2|422| +2450822|16978|2|42| +2450822|16981|2|681| +2450822|16982|2|534| +2450822|16984|2|101| +2450822|16987|2|270| +2450822|16988|2|737| +2450822|16990|2|| +2450822|16993|2|681| +2450822|16994|2|584| +2450822|16996|2|570| +2450822|16999|2|424| +2450822|17000|2|735| +2450822|17002|2|172| +2450822|17005|2|513| +2450822|17006|2|651| +2450822|17008|2|283| +2450822|17011|2|213| +2450822|17012|2|467| +2450822|17014|2|42| +2450822|17017|2|519| +2450822|17018|2|650| +2450822|17020|2|353| +2450822|17023|2|483| +2450822|17024|2|537| +2450822|17026|2|51| +2450822|17029|2|952| +2450822|17030|2|14| +2450822|17032|2|431| +2450822|17035|2|422| +2450822|17036|2|168| +2450822|17038|2|717| +2450822|17041|2|562| +2450822|17042|2|124| +2450822|17044|2|172| +2450822|17047|2|964| +2450822|17048|2|419| +2450822|17050|2|800| +2450822|17053|2|811| +2450822|17054|2|41| +2450822|17056|2|375| +2450822|17059|2|256| +2450822|17060|2|548| +2450822|17062|2|473| +2450822|17065|2|836| +2450822|17066|2|659| +2450822|17068|2|462| +2450822|17071|2|427| +2450822|17072|2|553| +2450822|17074|2|978| +2450822|17077|2|318| +2450822|17078|2|172| +2450822|17080|2|15| +2450822|17083|2|958| +2450822|17084|2|| +2450822|17086|2|130| +2450822|17089|2|953| +2450822|17090|2|66| +2450822|17092|2|909| +2450822|17095|2|613| +2450822|17096|2|520| +2450822|17098|2|170| +2450822|17101|2|| +2450822|17102|2|50| +2450822|17104|2|27| +2450822|17107|2|27| +2450822|17108|2|965| +2450822|17110|2|836| +2450822|17113|2|619| +2450822|17114|2|71| +2450822|17116|2|558| +2450822|17119|2|345| +2450822|17120|2|489| +2450822|17122|2|548| +2450822|17125|2|495| +2450822|17126|2|818| +2450822|17128|2|649| +2450822|17131|2|565| +2450822|17132|2|347| +2450822|17134|2|816| +2450822|17137|2|546| +2450822|17138|2|| +2450822|17140|2|857| +2450822|17143|2|321| +2450822|17144|2|689| +2450822|17146|2|197| +2450822|17149|2|163| +2450822|17150|2|412| +2450822|17152|2|131| +2450822|17155|2|739| +2450822|17156|2|609| +2450822|17158|2|13| +2450822|17161|2|312| +2450822|17162|2|535| +2450822|17164|2|450| +2450822|17167|2|30| +2450822|17168|2|| +2450822|17170|2|805| +2450822|17173|2|719| +2450822|17174|2|591| +2450822|17176|2|789| +2450822|17179|2|656| +2450822|17180|2|16| +2450822|17182|2|| +2450822|17185|2|274| +2450822|17186|2|41| +2450822|17188|2|| +2450822|17191|2|301| +2450822|17192|2|256| +2450822|17194|2|466| +2450822|17197|2|15| +2450822|17198|2|831| +2450822|17200|2|99| +2450822|17203|2|732| +2450822|17204|2|752| +2450822|17206|2|545| +2450822|17209|2|813| +2450822|17210|2|469| +2450822|17212|2|222| +2450822|17215|2|909| +2450822|17216|2|39| +2450822|17218|2|686| +2450822|17221|2|176| +2450822|17222|2|| +2450822|17224|2|108| +2450822|17227|2|662| +2450822|17228|2|468| +2450822|17230|2|893| +2450822|17233|2|947| +2450822|17234|2|208| +2450822|17236|2|904| +2450822|17239|2|681| +2450822|17240|2|723| +2450822|17242|2|| +2450822|17245|2|268| +2450822|17246|2|378| +2450822|17248|2|319| +2450822|17251|2|| +2450822|17252|2|135| +2450822|17254|2|836| +2450822|17257|2|593| +2450822|17258|2|671| +2450822|17260|2|705| +2450822|17263|2|710| +2450822|17264|2|800| +2450822|17266|2|843| +2450822|17269|2|417| +2450822|17270|2|882| +2450822|17272|2|383| +2450822|17275|2|648| +2450822|17276|2|479| +2450822|17278|2|125| +2450822|17281|2|56| +2450822|17282|2|598| +2450822|17284|2|584| +2450822|17287|2|744| +2450822|17288|2|175| +2450822|17290|2|553| +2450822|17293|2|480| +2450822|17294|2|432| +2450822|17296|2|120| +2450822|17299|2|61| +2450822|17300|2|609| +2450822|17302|2|401| +2450822|17305|2|860| +2450822|17306|2|486| +2450822|17308|2|738| +2450822|17311|2|551| +2450822|17312|2|213| +2450822|17314|2|333| +2450822|17317|2|908| +2450822|17318|2|194| +2450822|17320|2|654| +2450822|17323|2|820| +2450822|17324|2|174| +2450822|17326|2|233| +2450822|17329|2|131| +2450822|17330|2|214| +2450822|17332|2|262| +2450822|17335|2|667| +2450822|17336|2|| +2450822|17338|2|379| +2450822|17341|2|745| +2450822|17342|2|389| +2450822|17344|2|767| +2450822|17347|2|935| +2450822|17348|2|846| +2450822|17350|2|369| +2450822|17353|2|568| +2450822|17354|2|643| +2450822|17356|2|390| +2450822|17359|2|703| +2450822|17360|2|803| +2450822|17362|2|426| +2450822|17365|2|987| +2450822|17366|2|567| +2450822|17368|2|433| +2450822|17371|2|335| +2450822|17372|2|494| +2450822|17374|2|635| +2450822|17377|2|622| +2450822|17378|2|602| +2450822|17380|2|774| +2450822|17383|2|474| +2450822|17384|2|154| +2450822|17386|2|44| +2450822|17389|2|604| +2450822|17390|2|| +2450822|17392|2|253| +2450822|17395|2|533| +2450822|17396|2|629| +2450822|17398|2|611| +2450822|17401|2|235| +2450822|17402|2|164| +2450822|17404|2|668| +2450822|17407|2|432| +2450822|17408|2|| +2450822|17410|2|654| +2450822|17413|2|516| +2450822|17414|2|645| +2450822|17416|2|784| +2450822|17419|2|155| +2450822|17420|2|641| +2450822|17422|2|950| +2450822|17425|2|709| +2450822|17426|2|767| +2450822|17428|2|701| +2450822|17431|2|363| +2450822|17432|2|179| +2450822|17434|2|480| +2450822|17437|2|807| +2450822|17438|2|532| +2450822|17440|2|322| +2450822|17443|2|645| +2450822|17444|2|692| +2450822|17446|2|714| +2450822|17449|2|616| +2450822|17450|2|918| +2450822|17452|2|| +2450822|17455|2|654| +2450822|17456|2|145| +2450822|17458|2|995| +2450822|17461|2|213| +2450822|17462|2|174| +2450822|17464|2|669| +2450822|17467|2|85| +2450822|17468|2|814| +2450822|17470|2|694| +2450822|17473|2|857| +2450822|17474|2|974| +2450822|17476|2|220| +2450822|17479|2|852| +2450822|17480|2|327| +2450822|17482|2|738| +2450822|17485|2|270| +2450822|17486|2|258| +2450822|17488|2|277| +2450822|17491|2|8| +2450822|17492|2|180| +2450822|17494|2|7| +2450822|17497|2|52| +2450822|17498|2|6| +2450822|17500|2|812| +2450822|17503|2|492| +2450822|17504|2|801| +2450822|17506|2|970| +2450822|17509|2|454| +2450822|17510|2|85| +2450822|17512|2|439| +2450822|17515|2|542| +2450822|17516|2|123| +2450822|17518|2|127| +2450822|17521|2|| +2450822|17522|2|441| +2450822|17524|2|801| +2450822|17527|2|735| +2450822|17528|2|699| +2450822|17530|2|9| +2450822|17533|2|620| +2450822|17534|2|628| +2450822|17536|2|448| +2450822|17539|2|399| +2450822|17540|2|979| +2450822|17542|2|185| +2450822|17545|2|351| +2450822|17546|2|569| +2450822|17548|2|945| +2450822|17551|2|472| +2450822|17552|2|497| +2450822|17554|2|36| +2450822|17557|2|196| +2450822|17558|2|559| +2450822|17560|2|884| +2450822|17563|2|528| +2450822|17564|2|661| +2450822|17566|2|753| +2450822|17569|2|| +2450822|17570|2|708| +2450822|17572|2|816| +2450822|17575|2|722| +2450822|17576|2|216| +2450822|17578|2|406| +2450822|17581|2|690| +2450822|17582|2|826| +2450822|17584|2|770| +2450822|17587|2|| +2450822|17588|2|372| +2450822|17590|2|844| +2450822|17593|2|| +2450822|17594|2|996| +2450822|17596|2|159| +2450822|17599|2|362| +2450822|17600|2|716| +2450822|17602|2|992| +2450822|17605|2|448| +2450822|17606|2|541| +2450822|17608|2|753| +2450822|17611|2|7| +2450822|17612|2|925| +2450822|17614|2|26| +2450822|17617|2|918| +2450822|17618|2|555| +2450822|17620|2|666| +2450822|17623|2|952| +2450822|17624|2|298| +2450822|17626|2|408| +2450822|17629|2|52| +2450822|17630|2|903| +2450822|17632|2|233| +2450822|17635|2|151| +2450822|17636|2|671| +2450822|17638|2|632| +2450822|17641|2|59| +2450822|17642|2|468| +2450822|17644|2|10| +2450822|17647|2|9| +2450822|17648|2|524| +2450822|17650|2|380| +2450822|17653|2|163| +2450822|17654|2|237| +2450822|17656|2|474| +2450822|17659|2|150| +2450822|17660|2|112| +2450822|17662|2|780| +2450822|17665|2|654| +2450822|17666|2|472| +2450822|17668|2|242| +2450822|17671|2|433| +2450822|17672|2|952| +2450822|17674|2|268| +2450822|17677|2|821| +2450822|17678|2|755| +2450822|17680|2|352| +2450822|17683|2|39| +2450822|17684|2|434| +2450822|17686|2|488| +2450822|17689|2|294| +2450822|17690|2|910| +2450822|17692|2|809| +2450822|17695|2|832| +2450822|17696|2|378| +2450822|17698|2|259| +2450822|17701|2|640| +2450822|17702|2|795| +2450822|17704|2|58| +2450822|17707|2|481| +2450822|17708|2|268| +2450822|17710|2|428| +2450822|17713|2|| +2450822|17714|2|294| +2450822|17716|2|933| +2450822|17719|2|835| +2450822|17720|2|79| +2450822|17722|2|55| +2450822|17725|2|607| +2450822|17726|2|878| +2450822|17728|2|349| +2450822|17731|2|914| +2450822|17732|2|730| +2450822|17734|2|748| +2450822|17737|2|249| +2450822|17738|2|469| +2450822|17740|2|318| +2450822|17743|2|615| +2450822|17744|2|969| +2450822|17746|2|764| +2450822|17749|2|852| +2450822|17750|2|329| +2450822|17752|2|| +2450822|17755|2|233| +2450822|17756|2|403| +2450822|17758|2|106| +2450822|17761|2|315| +2450822|17762|2|164| +2450822|17764|2|354| +2450822|17767|2|909| +2450822|17768|2|255| +2450822|17770|2|780| +2450822|17773|2|| +2450822|17774|2|| +2450822|17776|2|219| +2450822|17779|2|843| +2450822|17780|2|437| +2450822|17782|2|273| +2450822|17785|2|633| +2450822|17786|2|699| +2450822|17788|2|840| +2450822|17791|2|991| +2450822|17792|2|267| +2450822|17794|2|632| +2450822|17797|2|533| +2450822|17798|2|59| +2450822|17800|2|859| +2450822|17803|2|248| +2450822|17804|2|99| +2450822|17806|2|398| +2450822|17809|2|8| +2450822|17810|2|636| +2450822|17812|2|| +2450822|17815|2|449| +2450822|17816|2|135| +2450822|17818|2|424| +2450822|17821|2|838| +2450822|17822|2|673| +2450822|17824|2|864| +2450822|17827|2|509| +2450822|17828|2|556| +2450822|17830|2|250| +2450822|17833|2|239| +2450822|17834|2|262| +2450822|17836|2|379| +2450822|17839|2|131| +2450822|17840|2|11| +2450822|17842|2|825| +2450822|17845|2|752| +2450822|17846|2|332| +2450822|17848|2|792| +2450822|17851|2|781| +2450822|17852|2|231| +2450822|17854|2|960| +2450822|17857|2|134| +2450822|17858|2|123| +2450822|17860|2|729| +2450822|17863|2|440| +2450822|17864|2|362| +2450822|17866|2|228| +2450822|17869|2|0| +2450822|17870|2|746| +2450822|17872|2|705| +2450822|17875|2|366| +2450822|17876|2|813| +2450822|17878|2|659| +2450822|17881|2|730| +2450822|17882|2|664| +2450822|17884|2|200| +2450822|17887|2|478| +2450822|17888|2|932| +2450822|17890|2|452| +2450822|17893|2|583| +2450822|17894|2|97| +2450822|17896|2|533| +2450822|17899|2|704| +2450822|17900|2|402| +2450822|17902|2|18| +2450822|17905|2|307| +2450822|17906|2|593| +2450822|17908|2|241| +2450822|17911|2|654| +2450822|17912|2|157| +2450822|17914|2|892| +2450822|17917|2|457| +2450822|17918|2|524| +2450822|17920|2|689| +2450822|17923|2|902| +2450822|17924|2|629| +2450822|17926|2|473| +2450822|17929|2|261| +2450822|17930|2|467| +2450822|17932|2|711| +2450822|17935|2|811| +2450822|17936|2|417| +2450822|17938|2|547| +2450822|17941|2|18| +2450822|17942|2|71| +2450822|17944|2|918| +2450822|17947|2|123| +2450822|17948|2|701| +2450822|17950|2|813| +2450822|17953|2|957| +2450822|17954|2|285| +2450822|17956|2|77| +2450822|17959|2|634| +2450822|17960|2|923| +2450822|17962|2|985| +2450822|17965|2|493| +2450822|17966|2|433| +2450822|17968|2|773| +2450822|17971|2|152| +2450822|17972|2|236| +2450822|17974|2|919| +2450822|17977|2|917| +2450822|17978|2|59| +2450822|17980|2|791| +2450822|17983|2|903| +2450822|17984|2|292| +2450822|17986|2|| +2450822|17989|2|383| +2450822|17990|2|25| +2450822|17992|2|177| +2450822|17995|2|| +2450822|17996|2|374| +2450822|17998|2|241| +2450822|1|3|507| +2450822|2|3|699| +2450822|4|3|270| +2450822|7|3|503| +2450822|8|3|898| +2450822|10|3|623| +2450822|13|3|204| +2450822|14|3|684| +2450822|16|3|577| +2450822|19|3|104| +2450822|20|3|408| +2450822|22|3|589| +2450822|25|3|131| +2450822|26|3|| +2450822|28|3|652| +2450822|31|3|158| +2450822|32|3|527| +2450822|34|3|536| +2450822|37|3|248| +2450822|38|3|269| +2450822|40|3|81| +2450822|43|3|799| +2450822|44|3|176| +2450822|46|3|253| +2450822|49|3|261| +2450822|50|3|220| +2450822|52|3|302| +2450822|55|3|445| +2450822|56|3|558| +2450822|58|3|978| +2450822|61|3|724| +2450822|62|3|864| +2450822|64|3|819| +2450822|67|3|336| +2450822|68|3|307| +2450822|70|3|304| +2450822|73|3|850| +2450822|74|3|289| +2450822|76|3|919| +2450822|79|3|839| +2450822|80|3|484| +2450822|82|3|198| +2450822|85|3|44| +2450822|86|3|206| +2450822|88|3|251| +2450822|91|3|182| +2450822|92|3|397| +2450822|94|3|| +2450822|97|3|89| +2450822|98|3|804| +2450822|100|3|383| +2450822|103|3|297| +2450822|104|3|174| +2450822|106|3|84| +2450822|109|3|684| +2450822|110|3|567| +2450822|112|3|242| +2450822|115|3|863| +2450822|116|3|925| +2450822|118|3|190| +2450822|121|3|135| +2450822|122|3|446| +2450822|124|3|717| +2450822|127|3|721| +2450822|128|3|667| +2450822|130|3|| +2450822|133|3|770| +2450822|134|3|426| +2450822|136|3|691| +2450822|139|3|460| +2450822|140|3|665| +2450822|142|3|443| +2450822|145|3|759| +2450822|146|3|792| +2450822|148|3|585| +2450822|151|3|389| +2450822|152|3|464| +2450822|154|3|516| +2450822|157|3|517| +2450822|158|3|545| +2450822|160|3|646| +2450822|163|3|243| +2450822|164|3|747| +2450822|166|3|617| +2450822|169|3|780| +2450822|170|3|931| +2450822|172|3|265| +2450822|175|3|165| +2450822|176|3|197| +2450822|178|3|586| +2450822|181|3|732| +2450822|182|3|673| +2450822|184|3|227| +2450822|187|3|170| +2450822|188|3|117| +2450822|190|3|992| +2450822|193|3|910| +2450822|194|3|230| +2450822|196|3|224| +2450822|199|3|209| +2450822|200|3|390| +2450822|202|3|888| +2450822|205|3|987| +2450822|206|3|140| +2450822|208|3|289| +2450822|211|3|977| +2450822|212|3|235| +2450822|214|3|420| +2450822|217|3|938| +2450822|218|3|434| +2450822|220|3|745| +2450822|223|3|602| +2450822|224|3|354| +2450822|226|3|694| +2450822|229|3|835| +2450822|230|3|876| +2450822|232|3|902| +2450822|235|3|967| +2450822|236|3|270| +2450822|238|3|857| +2450822|241|3|703| +2450822|242|3|226| +2450822|244|3|608| +2450822|247|3|400| +2450822|248|3|39| +2450822|250|3|870| +2450822|253|3|99| +2450822|254|3|114| +2450822|256|3|951| +2450822|259|3|968| +2450822|260|3|454| +2450822|262|3|882| +2450822|265|3|327| +2450822|266|3|701| +2450822|268|3|483| +2450822|271|3|308| +2450822|272|3|360| +2450822|274|3|219| +2450822|277|3|399| +2450822|278|3|479| +2450822|280|3|335| +2450822|283|3|132| +2450822|284|3|516| +2450822|286|3|829| +2450822|289|3|579| +2450822|290|3|197| +2450822|292|3|998| +2450822|295|3|440| +2450822|296|3|940| +2450822|298|3|199| +2450822|301|3|789| +2450822|302|3|743| +2450822|304|3|388| +2450822|307|3|909| +2450822|308|3|186| +2450822|310|3|578| +2450822|313|3|791| +2450822|314|3|331| +2450822|316|3|0| +2450822|319|3|274| +2450822|320|3|5| +2450822|322|3|200| +2450822|325|3|179| +2450822|326|3|52| +2450822|328|3|890| +2450822|331|3|679| +2450822|332|3|571| +2450822|334|3|531| +2450822|337|3|931| +2450822|338|3|732| +2450822|340|3|508| +2450822|343|3|256| +2450822|344|3|606| +2450822|346|3|| +2450822|349|3|939| +2450822|350|3|241| +2450822|352|3|582| +2450822|355|3|612| +2450822|356|3|970| +2450822|358|3|755| +2450822|361|3|947| +2450822|362|3|823| +2450822|364|3|928| +2450822|367|3|559| +2450822|368|3|199| +2450822|370|3|536| +2450822|373|3|825| +2450822|374|3|538| +2450822|376|3|| +2450822|379|3|929| +2450822|380|3|707| +2450822|382|3|710| +2450822|385|3|389| +2450822|386|3|599| +2450822|388|3|780| +2450822|391|3|924| +2450822|392|3|781| +2450822|394|3|329| +2450822|397|3|371| +2450822|398|3|333| +2450822|400|3|180| +2450822|403|3|308| +2450822|404|3|382| +2450822|406|3|313| +2450822|409|3|495| +2450822|410|3|494| +2450822|412|3|971| +2450822|415|3|426| +2450822|416|3|203| +2450822|418|3|775| +2450822|421|3|289| +2450822|422|3|201| +2450822|424|3|834| +2450822|427|3|680| +2450822|428|3|470| +2450822|430|3|227| +2450822|433|3|40| +2450822|434|3|280| +2450822|436|3|722| +2450822|439|3|884| +2450822|440|3|826| +2450822|442|3|951| +2450822|445|3|921| +2450822|446|3|931| +2450822|448|3|647| +2450822|451|3|508| +2450822|452|3|324| +2450822|454|3|52| +2450822|457|3|665| +2450822|458|3|309| +2450822|460|3|816| +2450822|463|3|769| +2450822|464|3|954| +2450822|466|3|11| +2450822|469|3|340| +2450822|470|3|444| +2450822|472|3|| +2450822|475|3|430| +2450822|476|3|339| +2450822|478|3|360| +2450822|481|3|83| +2450822|482|3|736| +2450822|484|3|807| +2450822|487|3|307| +2450822|488|3|325| +2450822|490|3|997| +2450822|493|3|241| +2450822|494|3|473| +2450822|496|3|535| +2450822|499|3|731| +2450822|500|3|499| +2450822|502|3|801| +2450822|505|3|23| +2450822|506|3|716| +2450822|508|3|1000| +2450822|511|3|210| +2450822|512|3|317| +2450822|514|3|996| +2450822|517|3|747| +2450822|518|3|992| +2450822|520|3|47| +2450822|523|3|409| +2450822|524|3|148| +2450822|526|3|51| +2450822|529|3|897| +2450822|530|3|675| +2450822|532|3|964| +2450822|535|3|987| +2450822|536|3|429| +2450822|538|3|841| +2450822|541|3|228| +2450822|542|3|974| +2450822|544|3|887| +2450822|547|3|485| +2450822|548|3|838| +2450822|550|3|172| +2450822|553|3|588| +2450822|554|3|893| +2450822|556|3|319| +2450822|559|3|139| +2450822|560|3|512| +2450822|562|3|287| +2450822|565|3|91| +2450822|566|3|813| +2450822|568|3|522| +2450822|571|3|548| +2450822|572|3|703| +2450822|574|3|362| +2450822|577|3|226| +2450822|578|3|546| +2450822|580|3|779| +2450822|583|3|55| +2450822|584|3|160| +2450822|586|3|749| +2450822|589|3|95| +2450822|590|3|565| +2450822|592|3|526| +2450822|595|3|105| +2450822|596|3|192| +2450822|598|3|194| +2450822|601|3|884| +2450822|602|3|| +2450822|604|3|703| +2450822|607|3|151| +2450822|608|3|| +2450822|610|3|200| +2450822|613|3|303| +2450822|614|3|587| +2450822|616|3|990| +2450822|619|3|837| +2450822|620|3|656| +2450822|622|3|372| +2450822|625|3|272| +2450822|626|3|562| +2450822|628|3|414| +2450822|631|3|330| +2450822|632|3|172| +2450822|634|3|427| +2450822|637|3|| +2450822|638|3|656| +2450822|640|3|6| +2450822|643|3|40| +2450822|644|3|800| +2450822|646|3|375| +2450822|649|3|303| +2450822|650|3|69| +2450822|652|3|503| +2450822|655|3|457| +2450822|656|3|74| +2450822|658|3|115| +2450822|661|3|519| +2450822|662|3|77| +2450822|664|3|783| +2450822|667|3|493| +2450822|668|3|789| +2450822|670|3|86| +2450822|673|3|244| +2450822|674|3|783| +2450822|676|3|806| +2450822|679|3|55| +2450822|680|3|515| +2450822|682|3|0| +2450822|685|3|956| +2450822|686|3|333| +2450822|688|3|220| +2450822|691|3|941| +2450822|692|3|293| +2450822|694|3|810| +2450822|697|3|156| +2450822|698|3|| +2450822|700|3|836| +2450822|703|3|234| +2450822|704|3|118| +2450822|706|3|| +2450822|709|3|957| +2450822|710|3|38| +2450822|712|3|751| +2450822|715|3|360| +2450822|716|3|714| +2450822|718|3|508| +2450822|721|3|821| +2450822|722|3|890| +2450822|724|3|703| +2450822|727|3|177| +2450822|728|3|45| +2450822|730|3|95| +2450822|733|3|232| +2450822|734|3|671| +2450822|736|3|98| +2450822|739|3|293| +2450822|740|3|301| +2450822|742|3|93| +2450822|745|3|160| +2450822|746|3|252| +2450822|748|3|297| +2450822|751|3|313| +2450822|752|3|40| +2450822|754|3|230| +2450822|757|3|947| +2450822|758|3|775| +2450822|760|3|767| +2450822|763|3|421| +2450822|764|3|693| +2450822|766|3|412| +2450822|769|3|| +2450822|770|3|230| +2450822|772|3|173| +2450822|775|3|38| +2450822|776|3|260| +2450822|778|3|554| +2450822|781|3|896| +2450822|782|3|260| +2450822|784|3|291| +2450822|787|3|472| +2450822|788|3|| +2450822|790|3|989| +2450822|793|3|124| +2450822|794|3|909| +2450822|796|3|195| +2450822|799|3|567| +2450822|800|3|318| +2450822|802|3|525| +2450822|805|3|171| +2450822|806|3|697| +2450822|808|3|676| +2450822|811|3|831| +2450822|812|3|873| +2450822|814|3|741| +2450822|817|3|321| +2450822|818|3|517| +2450822|820|3|703| +2450822|823|3|| +2450822|824|3|538| +2450822|826|3|380| +2450822|829|3|133| +2450822|830|3|645| +2450822|832|3|404| +2450822|835|3|957| +2450822|836|3|305| +2450822|838|3|187| +2450822|841|3|251| +2450822|842|3|136| +2450822|844|3|727| +2450822|847|3|446| +2450822|848|3|79| +2450822|850|3|173| +2450822|853|3|685| +2450822|854|3|445| +2450822|856|3|327| +2450822|859|3|895| +2450822|860|3|327| +2450822|862|3|745| +2450822|865|3|924| +2450822|866|3|508| +2450822|868|3|480| +2450822|871|3|382| +2450822|872|3|679| +2450822|874|3|300| +2450822|877|3|| +2450822|878|3|458| +2450822|880|3|676| +2450822|883|3|692| +2450822|884|3|706| +2450822|886|3|524| +2450822|889|3|165| +2450822|890|3|486| +2450822|892|3|454| +2450822|895|3|449| +2450822|896|3|818| +2450822|898|3|| +2450822|901|3|511| +2450822|902|3|911| +2450822|904|3|225| +2450822|907|3|424| +2450822|908|3|907| +2450822|910|3|508| +2450822|913|3|921| +2450822|914|3|963| +2450822|916|3|631| +2450822|919|3|614| +2450822|920|3|509| +2450822|922|3|179| +2450822|925|3|810| +2450822|926|3|250| +2450822|928|3|485| +2450822|931|3|997| +2450822|932|3|276| +2450822|934|3|15| +2450822|937|3|362| +2450822|938|3|962| +2450822|940|3|584| +2450822|943|3|703| +2450822|944|3|36| +2450822|946|3|659| +2450822|949|3|141| +2450822|950|3|439| +2450822|952|3|659| +2450822|955|3|71| +2450822|956|3|963| +2450822|958|3|316| +2450822|961|3|184| +2450822|962|3|114| +2450822|964|3|507| +2450822|967|3|676| +2450822|968|3|8| +2450822|970|3|235| +2450822|973|3|262| +2450822|974|3|164| +2450822|976|3|344| +2450822|979|3|816| +2450822|980|3|697| +2450822|982|3|149| +2450822|985|3|751| +2450822|986|3|217| +2450822|988|3|913| +2450822|991|3|880| +2450822|992|3|61| +2450822|994|3|91| +2450822|997|3|973| +2450822|998|3|666| +2450822|1000|3|878| +2450822|1003|3|959| +2450822|1004|3|611| +2450822|1006|3|| +2450822|1009|3|751| +2450822|1010|3|281| +2450822|1012|3|984| +2450822|1015|3|926| +2450822|1016|3|859| +2450822|1018|3|944| +2450822|1021|3|876| +2450822|1022|3|303| +2450822|1024|3|259| +2450822|1027|3|835| +2450822|1028|3|243| +2450822|1030|3|| +2450822|1033|3|455| +2450822|1034|3|670| +2450822|1036|3|873| +2450822|1039|3|786| +2450822|1040|3|622| +2450822|1042|3|650| +2450822|1045|3|828| +2450822|1046|3|411| +2450822|1048|3|310| +2450822|1051|3|157| +2450822|1052|3|251| +2450822|1054|3|427| +2450822|1057|3|650| +2450822|1058|3|7| +2450822|1060|3|529| +2450822|1063|3|948| +2450822|1064|3|172| +2450822|1066|3|503| +2450822|1069|3|722| +2450822|1070|3|440| +2450822|1072|3|968| +2450822|1075|3|536| +2450822|1076|3|15| +2450822|1078|3|928| +2450822|1081|3|105| +2450822|1082|3|435| +2450822|1084|3|987| +2450822|1087|3|742| +2450822|1088|3|537| +2450822|1090|3|| +2450822|1093|3|689| +2450822|1094|3|748| +2450822|1096|3|495| +2450822|1099|3|| +2450822|1100|3|148| +2450822|1102|3|920| +2450822|1105|3|369| +2450822|1106|3|860| +2450822|1108|3|248| +2450822|1111|3|| +2450822|1112|3|774| +2450822|1114|3|75| +2450822|1117|3|188| +2450822|1118|3|825| +2450822|1120|3|577| +2450822|1123|3|425| +2450822|1124|3|195| +2450822|1126|3|26| +2450822|1129|3|| +2450822|1130|3|665| +2450822|1132|3|379| +2450822|1135|3|964| +2450822|1136|3|802| +2450822|1138|3|124| +2450822|1141|3|760| +2450822|1142|3|281| +2450822|1144|3|323| +2450822|1147|3|982| +2450822|1148|3|306| +2450822|1150|3|243| +2450822|1153|3|224| +2450822|1154|3|260| +2450822|1156|3|605| +2450822|1159|3|889| +2450822|1160|3|567| +2450822|1162|3|83| +2450822|1165|3|212| +2450822|1166|3|716| +2450822|1168|3|395| +2450822|1171|3|229| +2450822|1172|3|832| +2450822|1174|3|444| +2450822|1177|3|239| +2450822|1178|3|394| +2450822|1180|3|422| +2450822|1183|3|840| +2450822|1184|3|772| +2450822|1186|3|890| +2450822|1189|3|247| +2450822|1190|3|178| +2450822|1192|3|762| +2450822|1195|3|575| +2450822|1196|3|33| +2450822|1198|3|417| +2450822|1201|3|133| +2450822|1202|3|537| +2450822|1204|3|566| +2450822|1207|3|137| +2450822|1208|3|302| +2450822|1210|3|355| +2450822|1213|3|496| +2450822|1214|3|597| +2450822|1216|3|181| +2450822|1219|3|51| +2450822|1220|3|668| +2450822|1222|3|53| +2450822|1225|3|741| +2450822|1226|3|396| +2450822|1228|3|343| +2450822|1231|3|123| +2450822|1232|3|459| +2450822|1234|3|686| +2450822|1237|3|646| +2450822|1238|3|| +2450822|1240|3|408| +2450822|1243|3|755| +2450822|1244|3|157| +2450822|1246|3|520| +2450822|1249|3|511| +2450822|1250|3|226| +2450822|1252|3|991| +2450822|1255|3|210| +2450822|1256|3|496| +2450822|1258|3|96| +2450822|1261|3|792| +2450822|1262|3|151| +2450822|1264|3|971| +2450822|1267|3|162| +2450822|1268|3|691| +2450822|1270|3|971| +2450822|1273|3|| +2450822|1274|3|22| +2450822|1276|3|940| +2450822|1279|3|856| +2450822|1280|3|202| +2450822|1282|3|763| +2450822|1285|3|251| +2450822|1286|3|485| +2450822|1288|3|418| +2450822|1291|3|735| +2450822|1292|3|25| +2450822|1294|3|776| +2450822|1297|3|124| +2450822|1298|3|196| +2450822|1300|3|917| +2450822|1303|3|377| +2450822|1304|3|452| +2450822|1306|3|472| +2450822|1309|3|883| +2450822|1310|3|527| +2450822|1312|3|678| +2450822|1315|3|869| +2450822|1316|3|648| +2450822|1318|3|46| +2450822|1321|3|| +2450822|1322|3|153| +2450822|1324|3|286| +2450822|1327|3|516| +2450822|1328|3|997| +2450822|1330|3|679| +2450822|1333|3|528| +2450822|1334|3|708| +2450822|1336|3|| +2450822|1339|3|673| +2450822|1340|3|119| +2450822|1342|3|567| +2450822|1345|3|945| +2450822|1346|3|564| +2450822|1348|3|214| +2450822|1351|3|814| +2450822|1352|3|876| +2450822|1354|3|859| +2450822|1357|3|50| +2450822|1358|3|489| +2450822|1360|3|436| +2450822|1363|3|917| +2450822|1364|3|57| +2450822|1366|3|940| +2450822|1369|3|646| +2450822|1370|3|271| +2450822|1372|3|778| +2450822|1375|3|801| +2450822|1376|3|324| +2450822|1378|3|316| +2450822|1381|3|457| +2450822|1382|3|978| +2450822|1384|3|| +2450822|1387|3|706| +2450822|1388|3|336| +2450822|1390|3|671| +2450822|1393|3|72| +2450822|1394|3|28| +2450822|1396|3|945| +2450822|1399|3|821| +2450822|1400|3|19| +2450822|1402|3|8| +2450822|1405|3|503| +2450822|1406|3|979| +2450822|1408|3|| +2450822|1411|3|600| +2450822|1412|3|664| +2450822|1414|3|916| +2450822|1417|3|731| +2450822|1418|3|613| +2450822|1420|3|240| +2450822|1423|3|105| +2450822|1424|3|653| +2450822|1426|3|805| +2450822|1429|3|949| +2450822|1430|3|932| +2450822|1432|3|694| +2450822|1435|3|672| +2450822|1436|3|985| +2450822|1438|3|966| +2450822|1441|3|401| +2450822|1442|3|521| +2450822|1444|3|41| +2450822|1447|3|174| +2450822|1448|3|421| +2450822|1450|3|334| +2450822|1453|3|388| +2450822|1454|3|385| +2450822|1456|3|258| +2450822|1459|3|88| +2450822|1460|3|611| +2450822|1462|3|5| +2450822|1465|3|448| +2450822|1466|3|977| +2450822|1468|3|223| +2450822|1471|3|304| +2450822|1472|3|567| +2450822|1474|3|433| +2450822|1477|3|| +2450822|1478|3|686| +2450822|1480|3|875| +2450822|1483|3|949| +2450822|1484|3|620| +2450822|1486|3|| +2450822|1489|3|166| +2450822|1490|3|182| +2450822|1492|3|268| +2450822|1495|3|564| +2450822|1496|3|25| +2450822|1498|3|301| +2450822|1501|3|74| +2450822|1502|3|252| +2450822|1504|3|0| +2450822|1507|3|363| +2450822|1508|3|620| +2450822|1510|3|642| +2450822|1513|3|283| +2450822|1514|3|604| +2450822|1516|3|| +2450822|1519|3|503| +2450822|1520|3|74| +2450822|1522|3|349| +2450822|1525|3|478| +2450822|1526|3|647| +2450822|1528|3|735| +2450822|1531|3|| +2450822|1532|3|500| +2450822|1534|3|846| +2450822|1537|3|516| +2450822|1538|3|487| +2450822|1540|3|810| +2450822|1543|3|460| +2450822|1544|3|| +2450822|1546|3|545| +2450822|1549|3|548| +2450822|1550|3|109| +2450822|1552|3|649| +2450822|1555|3|527| +2450822|1556|3|88| +2450822|1558|3|696| +2450822|1561|3|| +2450822|1562|3|137| +2450822|1564|3|470| +2450822|1567|3|172| +2450822|1568|3|981| +2450822|1570|3|346| +2450822|1573|3|194| +2450822|1574|3|688| +2450822|1576|3|904| +2450822|1579|3|502| +2450822|1580|3|962| +2450822|1582|3|174| +2450822|1585|3|446| +2450822|1586|3|424| +2450822|1588|3|592| +2450822|1591|3|667| +2450822|1592|3|426| +2450822|1594|3|26| +2450822|1597|3|408| +2450822|1598|3|75| +2450822|1600|3|938| +2450822|1603|3|875| +2450822|1604|3|479| +2450822|1606|3|326| +2450822|1609|3|938| +2450822|1610|3|372| +2450822|1612|3|853| +2450822|1615|3|627| +2450822|1616|3|128| +2450822|1618|3|429| +2450822|1621|3|65| +2450822|1622|3|360| +2450822|1624|3|664| +2450822|1627|3|60| +2450822|1628|3|377| +2450822|1630|3|640| +2450822|1633|3|478| +2450822|1634|3|983| +2450822|1636|3|951| +2450822|1639|3|500| +2450822|1640|3|23| +2450822|1642|3|| +2450822|1645|3|650| +2450822|1646|3|910| +2450822|1648|3|590| +2450822|1651|3|692| +2450822|1652|3|935| +2450822|1654|3|106| +2450822|1657|3|593| +2450822|1658|3|162| +2450822|1660|3|237| +2450822|1663|3|356| +2450822|1664|3|104| +2450822|1666|3|381| +2450822|1669|3|| +2450822|1670|3|349| +2450822|1672|3|480| +2450822|1675|3|862| +2450822|1676|3|301| +2450822|1678|3|805| +2450822|1681|3|904| +2450822|1682|3|143| +2450822|1684|3|265| +2450822|1687|3|617| +2450822|1688|3|241| +2450822|1690|3|849| +2450822|1693|3|374| +2450822|1694|3|| +2450822|1696|3|570| +2450822|1699|3|327| +2450822|1700|3|94| +2450822|1702|3|423| +2450822|1705|3|318| +2450822|1706|3|997| +2450822|1708|3|122| +2450822|1711|3|837| +2450822|1712|3|887| +2450822|1714|3|827| +2450822|1717|3|620| +2450822|1718|3|5| +2450822|1720|3|352| +2450822|1723|3|53| +2450822|1724|3|36| +2450822|1726|3|815| +2450822|1729|3|406| +2450822|1730|3|642| +2450822|1732|3|308| +2450822|1735|3|| +2450822|1736|3|886| +2450822|1738|3|985| +2450822|1741|3|455| +2450822|1742|3|816| +2450822|1744|3|983| +2450822|1747|3|494| +2450822|1748|3|655| +2450822|1750|3|387| +2450822|1753|3|848| +2450822|1754|3|| +2450822|1756|3|| +2450822|1759|3|210| +2450822|1760|3|532| +2450822|1762|3|187| +2450822|1765|3|732| +2450822|1766|3|147| +2450822|1768|3|275| +2450822|1771|3|655| +2450822|1772|3|752| +2450822|1774|3|454| +2450822|1777|3|627| +2450822|1778|3|331| +2450822|1780|3|655| +2450822|1783|3|218| +2450822|1784|3|| +2450822|1786|3|265| +2450822|1789|3|219| +2450822|1790|3|294| +2450822|1792|3|| +2450822|1795|3|993| +2450822|1796|3|983| +2450822|1798|3|311| +2450822|1801|3|994| +2450822|1802|3|742| +2450822|1804|3|942| +2450822|1807|3|657| +2450822|1808|3|488| +2450822|1810|3|| +2450822|1813|3|837| +2450822|1814|3|51| +2450822|1816|3|807| +2450822|1819|3|749| +2450822|1820|3|483| +2450822|1822|3|941| +2450822|1825|3|| +2450822|1826|3|390| +2450822|1828|3|| +2450822|1831|3|517| +2450822|1832|3|281| +2450822|1834|3|761| +2450822|1837|3|975| +2450822|1838|3|672| +2450822|1840|3|974| +2450822|1843|3|918| +2450822|1844|3|264| +2450822|1846|3|585| +2450822|1849|3|846| +2450822|1850|3|536| +2450822|1852|3|603| +2450822|1855|3|14| +2450822|1856|3|309| +2450822|1858|3|872| +2450822|1861|3|962| +2450822|1862|3|320| +2450822|1864|3|256| +2450822|1867|3|826| +2450822|1868|3|52| +2450822|1870|3|764| +2450822|1873|3|785| +2450822|1874|3|147| +2450822|1876|3|112| +2450822|1879|3|677| +2450822|1880|3|59| +2450822|1882|3|467| +2450822|1885|3|83| +2450822|1886|3|763| +2450822|1888|3|114| +2450822|1891|3|681| +2450822|1892|3|399| +2450822|1894|3|682| +2450822|1897|3|481| +2450822|1898|3|914| +2450822|1900|3|97| +2450822|1903|3|389| +2450822|1904|3|889| +2450822|1906|3|785| +2450822|1909|3|339| +2450822|1910|3|294| +2450822|1912|3|634| +2450822|1915|3|685| +2450822|1916|3|499| +2450822|1918|3|677| +2450822|1921|3|653| +2450822|1922|3|640| +2450822|1924|3|226| +2450822|1927|3|614| +2450822|1928|3|820| +2450822|1930|3|369| +2450822|1933|3|335| +2450822|1934|3|234| +2450822|1936|3|968| +2450822|1939|3|16| +2450822|1940|3|| +2450822|1942|3|338| +2450822|1945|3|188| +2450822|1946|3|223| +2450822|1948|3|486| +2450822|1951|3|576| +2450822|1952|3|327| +2450822|1954|3|717| +2450822|1957|3|858| +2450822|1958|3|326| +2450822|1960|3|419| +2450822|1963|3|| +2450822|1964|3|874| +2450822|1966|3|761| +2450822|1969|3|501| +2450822|1970|3|954| +2450822|1972|3|304| +2450822|1975|3|8| +2450822|1976|3|861| +2450822|1978|3|614| +2450822|1981|3|564| +2450822|1982|3|232| +2450822|1984|3|943| +2450822|1987|3|636| +2450822|1988|3|15| +2450822|1990|3|270| +2450822|1993|3|170| +2450822|1994|3|532| +2450822|1996|3|910| +2450822|1999|3|864| +2450822|2000|3|| +2450822|2002|3|641| +2450822|2005|3|419| +2450822|2006|3|110| +2450822|2008|3|858| +2450822|2011|3|| +2450822|2012|3|372| +2450822|2014|3|116| +2450822|2017|3|421| +2450822|2018|3|975| +2450822|2020|3|691| +2450822|2023|3|67| +2450822|2024|3|343| +2450822|2026|3|964| +2450822|2029|3|486| +2450822|2030|3|718| +2450822|2032|3|969| +2450822|2035|3|317| +2450822|2036|3|651| +2450822|2038|3|469| +2450822|2041|3|943| +2450822|2042|3|| +2450822|2044|3|358| +2450822|2047|3|486| +2450822|2048|3|896| +2450822|2050|3|87| +2450822|2053|3|630| +2450822|2054|3|483| +2450822|2056|3|| +2450822|2059|3|212| +2450822|2060|3|454| +2450822|2062|3|625| +2450822|2065|3|285| +2450822|2066|3|369| +2450822|2068|3|820| +2450822|2071|3|720| +2450822|2072|3|227| +2450822|2074|3|615| +2450822|2077|3|477| +2450822|2078|3|793| +2450822|2080|3|404| +2450822|2083|3|861| +2450822|2084|3|| +2450822|2086|3|95| +2450822|2089|3|341| +2450822|2090|3|758| +2450822|2092|3|120| +2450822|2095|3|405| +2450822|2096|3|960| +2450822|2098|3|458| +2450822|2101|3|467| +2450822|2102|3|428| +2450822|2104|3|974| +2450822|2107|3|329| +2450822|2108|3|981| +2450822|2110|3|457| +2450822|2113|3|817| +2450822|2114|3|602| +2450822|2116|3|193| +2450822|2119|3|| +2450822|2120|3|614| +2450822|2122|3|759| +2450822|2125|3|781| +2450822|2126|3|873| +2450822|2128|3|| +2450822|2131|3|45| +2450822|2132|3|572| +2450822|2134|3|218| +2450822|2137|3|1000| +2450822|2138|3|699| +2450822|2140|3|927| +2450822|2143|3|226| +2450822|2144|3|343| +2450822|2146|3|873| +2450822|2149|3|830| +2450822|2150|3|628| +2450822|2152|3|605| +2450822|2155|3|700| +2450822|2156|3|834| +2450822|2158|3|101| +2450822|2161|3|793| +2450822|2162|3|193| +2450822|2164|3|836| +2450822|2167|3|577| +2450822|2168|3|449| +2450822|2170|3|912| +2450822|2173|3|751| +2450822|2174|3|471| +2450822|2176|3|920| +2450822|2179|3|796| +2450822|2180|3|373| +2450822|2182|3|193| +2450822|2185|3|| +2450822|2186|3|775| +2450822|2188|3|396| +2450822|2191|3|147| +2450822|2192|3|659| +2450822|2194|3|182| +2450822|2197|3|201| +2450822|2198|3|478| +2450822|2200|3|621| +2450822|2203|3|771| +2450822|2204|3|580| +2450822|2206|3|491| +2450822|2209|3|10| +2450822|2210|3|909| +2450822|2212|3|681| +2450822|2215|3|| +2450822|2216|3|680| +2450822|2218|3|836| +2450822|2221|3|411| +2450822|2222|3|427| +2450822|2224|3|852| +2450822|2227|3|371| +2450822|2228|3|| +2450822|2230|3|657| +2450822|2233|3|93| +2450822|2234|3|821| +2450822|2236|3|879| +2450822|2239|3|819| +2450822|2240|3|10| +2450822|2242|3|887| +2450822|2245|3|596| +2450822|2246|3|275| +2450822|2248|3|746| +2450822|2251|3|531| +2450822|2252|3|370| +2450822|2254|3|225| +2450822|2257|3|741| +2450822|2258|3|999| +2450822|2260|3|409| +2450822|2263|3|872| +2450822|2264|3|558| +2450822|2266|3|846| +2450822|2269|3|| +2450822|2270|3|895| +2450822|2272|3|756| +2450822|2275|3|2| +2450822|2276|3|57| +2450822|2278|3|223| +2450822|2281|3|831| +2450822|2282|3|430| +2450822|2284|3|315| +2450822|2287|3|982| +2450822|2288|3|551| +2450822|2290|3|526| +2450822|2293|3|135| +2450822|2294|3|| +2450822|2296|3|874| +2450822|2299|3|477| +2450822|2300|3|969| +2450822|2302|3|387| +2450822|2305|3|855| +2450822|2306|3|898| +2450822|2308|3|157| +2450822|2311|3|| +2450822|2312|3|846| +2450822|2314|3|402| +2450822|2317|3|846| +2450822|2318|3|361| +2450822|2320|3|569| +2450822|2323|3|182| +2450822|2324|3|925| +2450822|2326|3|209| +2450822|2329|3|586| +2450822|2330|3|256| +2450822|2332|3|415| +2450822|2335|3|639| +2450822|2336|3|339| +2450822|2338|3|67| +2450822|2341|3|141| +2450822|2342|3|563| +2450822|2344|3|209| +2450822|2347|3|515| +2450822|2348|3|654| +2450822|2350|3|696| +2450822|2353|3|| +2450822|2354|3|766| +2450822|2356|3|175| +2450822|2359|3|380| +2450822|2360|3|764| +2450822|2362|3|719| +2450822|2365|3|601| +2450822|2366|3|265| +2450822|2368|3|95| +2450822|2371|3|197| +2450822|2372|3|596| +2450822|2374|3|59| +2450822|2377|3|238| +2450822|2378|3|94| +2450822|2380|3|129| +2450822|2383|3|601| +2450822|2384|3|539| +2450822|2386|3|103| +2450822|2389|3|70| +2450822|2390|3|180| +2450822|2392|3|341| +2450822|2395|3|736| +2450822|2396|3|4| +2450822|2398|3|256| +2450822|2401|3|993| +2450822|2402|3|350| +2450822|2404|3|442| +2450822|2407|3|911| +2450822|2408|3|101| +2450822|2410|3|414| +2450822|2413|3|977| +2450822|2414|3|951| +2450822|2416|3|100| +2450822|2419|3|247| +2450822|2420|3|942| +2450822|2422|3|| +2450822|2425|3|39| +2450822|2426|3|97| +2450822|2428|3|487| +2450822|2431|3|| +2450822|2432|3|150| +2450822|2434|3|844| +2450822|2437|3|921| +2450822|2438|3|765| +2450822|2440|3|126| +2450822|2443|3|894| +2450822|2444|3|654| +2450822|2446|3|907| +2450822|2449|3|476| +2450822|2450|3|621| +2450822|2452|3|882| +2450822|2455|3|208| +2450822|2456|3|247| +2450822|2458|3|637| +2450822|2461|3|158| +2450822|2462|3|134| +2450822|2464|3|413| +2450822|2467|3|20| +2450822|2468|3|244| +2450822|2470|3|533| +2450822|2473|3|843| +2450822|2474|3|579| +2450822|2476|3|482| +2450822|2479|3|576| +2450822|2480|3|442| +2450822|2482|3|203| +2450822|2485|3|494| +2450822|2486|3|67| +2450822|2488|3|792| +2450822|2491|3|248| +2450822|2492|3|59| +2450822|2494|3|480| +2450822|2497|3|391| +2450822|2498|3|556| +2450822|2500|3|280| +2450822|2503|3|492| +2450822|2504|3|823| +2450822|2506|3|763| +2450822|2509|3|843| +2450822|2510|3|187| +2450822|2512|3|947| +2450822|2515|3|405| +2450822|2516|3|| +2450822|2518|3|499| +2450822|2521|3|457| +2450822|2522|3|998| +2450822|2524|3|27| +2450822|2527|3|280| +2450822|2528|3|124| +2450822|2530|3|877| +2450822|2533|3|364| +2450822|2534|3|824| +2450822|2536|3|153| +2450822|2539|3|611| +2450822|2540|3|216| +2450822|2542|3|160| +2450822|2545|3|834| +2450822|2546|3|39| +2450822|2548|3|| +2450822|2551|3|859| +2450822|2552|3|21| +2450822|2554|3|| +2450822|2557|3|179| +2450822|2558|3|37| +2450822|2560|3|786| +2450822|2563|3|928| +2450822|2564|3|85| +2450822|2566|3|684| +2450822|2569|3|677| +2450822|2570|3|992| +2450822|2572|3|286| +2450822|2575|3|647| +2450822|2576|3|221| +2450822|2578|3|364| +2450822|2581|3|212| +2450822|2582|3|226| +2450822|2584|3|93| +2450822|2587|3|369| +2450822|2588|3|148| +2450822|2590|3|461| +2450822|2593|3|| +2450822|2594|3|208| +2450822|2596|3|433| +2450822|2599|3|121| +2450822|2600|3|| +2450822|2602|3|18| +2450822|2605|3|731| +2450822|2606|3|745| +2450822|2608|3|693| +2450822|2611|3|869| +2450822|2612|3|46| +2450822|2614|3|985| +2450822|2617|3|754| +2450822|2618|3|860| +2450822|2620|3|150| +2450822|2623|3|865| +2450822|2624|3|767| +2450822|2626|3|568| +2450822|2629|3|850| +2450822|2630|3|371| +2450822|2632|3|474| +2450822|2635|3|216| +2450822|2636|3|48| +2450822|2638|3|248| +2450822|2641|3|766| +2450822|2642|3|824| +2450822|2644|3|671| +2450822|2647|3|441| +2450822|2648|3|30| +2450822|2650|3|| +2450822|2653|3|508| +2450822|2654|3|760| +2450822|2656|3|843| +2450822|2659|3|143| +2450822|2660|3|236| +2450822|2662|3|579| +2450822|2665|3|126| +2450822|2666|3|139| +2450822|2668|3|364| +2450822|2671|3|855| +2450822|2672|3|519| +2450822|2674|3|921| +2450822|2677|3|510| +2450822|2678|3|276| +2450822|2680|3|165| +2450822|2683|3|770| +2450822|2684|3|581| +2450822|2686|3|156| +2450822|2689|3|| +2450822|2690|3|215| +2450822|2692|3|624| +2450822|2695|3|529| +2450822|2696|3|327| +2450822|2698|3|973| +2450822|2701|3|504| +2450822|2702|3|173| +2450822|2704|3|860| +2450822|2707|3|92| +2450822|2708|3|824| +2450822|2710|3|535| +2450822|2713|3|357| +2450822|2714|3|838| +2450822|2716|3|583| +2450822|2719|3|99| +2450822|2720|3|699| +2450822|2722|3|264| +2450822|2725|3|413| +2450822|2726|3|145| +2450822|2728|3|988| +2450822|2731|3|949| +2450822|2732|3|163| +2450822|2734|3|391| +2450822|2737|3|662| +2450822|2738|3|860| +2450822|2740|3|639| +2450822|2743|3|373| +2450822|2744|3|246| +2450822|2746|3|529| +2450822|2749|3|243| +2450822|2750|3|819| +2450822|2752|3|650| +2450822|2755|3|935| +2450822|2756|3|738| +2450822|2758|3|629| +2450822|2761|3|376| +2450822|2762|3|37| +2450822|2764|3|159| +2450822|2767|3|331| +2450822|2768|3|50| +2450822|2770|3|395| +2450822|2773|3|317| +2450822|2774|3|974| +2450822|2776|3|369| +2450822|2779|3|980| +2450822|2780|3|687| +2450822|2782|3|513| +2450822|2785|3|832| +2450822|2786|3|287| +2450822|2788|3|539| +2450822|2791|3|389| +2450822|2792|3|512| +2450822|2794|3|295| +2450822|2797|3|53| +2450822|2798|3|789| +2450822|2800|3|923| +2450822|2803|3|277| +2450822|2804|3|14| +2450822|2806|3|473| +2450822|2809|3|809| +2450822|2810|3|333| +2450822|2812|3|700| +2450822|2815|3|189| +2450822|2816|3|216| +2450822|2818|3|590| +2450822|2821|3|19| +2450822|2822|3|688| +2450822|2824|3|356| +2450822|2827|3|305| +2450822|2828|3|298| +2450822|2830|3|932| +2450822|2833|3|908| +2450822|2834|3|254| +2450822|2836|3|407| +2450822|2839|3|936| +2450822|2840|3|598| +2450822|2842|3|623| +2450822|2845|3|560| +2450822|2846|3|338| +2450822|2848|3|259| +2450822|2851|3|289| +2450822|2852|3|638| +2450822|2854|3|33| +2450822|2857|3|958| +2450822|2858|3|346| +2450822|2860|3|409| +2450822|2863|3|351| +2450822|2864|3|654| +2450822|2866|3|843| +2450822|2869|3|155| +2450822|2870|3|646| +2450822|2872|3|829| +2450822|2875|3|941| +2450822|2876|3|350| +2450822|2878|3|268| +2450822|2881|3|768| +2450822|2882|3|872| +2450822|2884|3|29| +2450822|2887|3|493| +2450822|2888|3|657| +2450822|2890|3|276| +2450822|2893|3|547| +2450822|2894|3|926| +2450822|2896|3|417| +2450822|2899|3|780| +2450822|2900|3|531| +2450822|2902|3|666| +2450822|2905|3|856| +2450822|2906|3|395| +2450822|2908|3|245| +2450822|2911|3|708| +2450822|2912|3|| +2450822|2914|3|91| +2450822|2917|3|163| +2450822|2918|3|617| +2450822|2920|3|4| +2450822|2923|3|816| +2450822|2924|3|472| +2450822|2926|3|54| +2450822|2929|3|518| +2450822|2930|3|150| +2450822|2932|3|580| +2450822|2935|3|114| +2450822|2936|3|| +2450822|2938|3|161| +2450822|2941|3|522| +2450822|2942|3|843| +2450822|2944|3|358| +2450822|2947|3|383| +2450822|2948|3|152| +2450822|2950|3|794| +2450822|2953|3|889| +2450822|2954|3|317| +2450822|2956|3|309| +2450822|2959|3|795| +2450822|2960|3|116| +2450822|2962|3|779| +2450822|2965|3|16| +2450822|2966|3|170| +2450822|2968|3|619| +2450822|2971|3|198| +2450822|2972|3|| +2450822|2974|3|312| +2450822|2977|3|379| +2450822|2978|3|436| +2450822|2980|3|534| +2450822|2983|3|136| +2450822|2984|3|336| +2450822|2986|3|930| +2450822|2989|3|952| +2450822|2990|3|557| +2450822|2992|3|604| +2450822|2995|3|335| +2450822|2996|3|611| +2450822|2998|3|900| +2450822|3001|3|226| +2450822|3002|3|322| +2450822|3004|3|836| +2450822|3007|3|939| +2450822|3008|3|355| +2450822|3010|3|| +2450822|3013|3|497| +2450822|3014|3|412| +2450822|3016|3|183| +2450822|3019|3|490| +2450822|3020|3|846| +2450822|3022|3|267| +2450822|3025|3|569| +2450822|3026|3|859| +2450822|3028|3|424| +2450822|3031|3|233| +2450822|3032|3|162| +2450822|3034|3|346| +2450822|3037|3|268| +2450822|3038|3|589| +2450822|3040|3|46| +2450822|3043|3|775| +2450822|3044|3|142| +2450822|3046|3|466| +2450822|3049|3|634| +2450822|3050|3|265| +2450822|3052|3|943| +2450822|3055|3|256| +2450822|3056|3|779| +2450822|3058|3|793| +2450822|3061|3|624| +2450822|3062|3|515| +2450822|3064|3|725| +2450822|3067|3|| +2450822|3068|3|743| +2450822|3070|3|545| +2450822|3073|3|875| +2450822|3074|3|171| +2450822|3076|3|792| +2450822|3079|3|968| +2450822|3080|3|418| +2450822|3082|3|47| +2450822|3085|3|394| +2450822|3086|3|451| +2450822|3088|3|682| +2450822|3091|3|639| +2450822|3092|3|713| +2450822|3094|3|794| +2450822|3097|3|479| +2450822|3098|3|| +2450822|3100|3|769| +2450822|3103|3|325| +2450822|3104|3|196| +2450822|3106|3|969| +2450822|3109|3|550| +2450822|3110|3|477| +2450822|3112|3|712| +2450822|3115|3|137| +2450822|3116|3|474| +2450822|3118|3|202| +2450822|3121|3|132| +2450822|3122|3|857| +2450822|3124|3|142| +2450822|3127|3|400| +2450822|3128|3|510| +2450822|3130|3|425| +2450822|3133|3|473| +2450822|3134|3|752| +2450822|3136|3|667| +2450822|3139|3|400| +2450822|3140|3|417| +2450822|3142|3|371| +2450822|3145|3|904| +2450822|3146|3|815| +2450822|3148|3|734| +2450822|3151|3|835| +2450822|3152|3|866| +2450822|3154|3|18| +2450822|3157|3|427| +2450822|3158|3|904| +2450822|3160|3|816| +2450822|3163|3|| +2450822|3164|3|577| +2450822|3166|3|363| +2450822|3169|3|501| +2450822|3170|3|781| +2450822|3172|3|799| +2450822|3175|3|939| +2450822|3176|3|468| +2450822|3178|3|111| +2450822|3181|3|218| +2450822|3182|3|324| +2450822|3184|3|372| +2450822|3187|3|159| +2450822|3188|3|826| +2450822|3190|3|574| +2450822|3193|3|548| +2450822|3194|3|325| +2450822|3196|3|659| +2450822|3199|3|693| +2450822|3200|3|976| +2450822|3202|3|740| +2450822|3205|3|953| +2450822|3206|3|202| +2450822|3208|3|429| +2450822|3211|3|844| +2450822|3212|3|996| +2450822|3214|3|741| +2450822|3217|3|171| +2450822|3218|3|| +2450822|3220|3|784| +2450822|3223|3|128| +2450822|3224|3|788| +2450822|3226|3|721| +2450822|3229|3|641| +2450822|3230|3|717| +2450822|3232|3|950| +2450822|3235|3|518| +2450822|3236|3|398| +2450822|3238|3|292| +2450822|3241|3|35| +2450822|3242|3|| +2450822|3244|3|509| +2450822|3247|3|841| +2450822|3248|3|373| +2450822|3250|3|829| +2450822|3253|3|97| +2450822|3254|3|| +2450822|3256|3|457| +2450822|3259|3|214| +2450822|3260|3|158| +2450822|3262|3|492| +2450822|3265|3|160| +2450822|3266|3|815| +2450822|3268|3|831| +2450822|3271|3|322| +2450822|3272|3|108| +2450822|3274|3|71| +2450822|3277|3|846| +2450822|3278|3|690| +2450822|3280|3|752| +2450822|3283|3|104| +2450822|3284|3|3| +2450822|3286|3|454| +2450822|3289|3|439| +2450822|3290|3|494| +2450822|3292|3|739| +2450822|3295|3|133| +2450822|3296|3|363| +2450822|3298|3|972| +2450822|3301|3|26| +2450822|3302|3|668| +2450822|3304|3|187| +2450822|3307|3|454| +2450822|3308|3|498| +2450822|3310|3|349| +2450822|3313|3|893| +2450822|3314|3|24| +2450822|3316|3|308| +2450822|3319|3|666| +2450822|3320|3|448| +2450822|3322|3|382| +2450822|3325|3|366| +2450822|3326|3|121| +2450822|3328|3|113| +2450822|3331|3|460| +2450822|3332|3|710| +2450822|3334|3|641| +2450822|3337|3|147| +2450822|3338|3|655| +2450822|3340|3|| +2450822|3343|3|651| +2450822|3344|3|707| +2450822|3346|3|342| +2450822|3349|3|467| +2450822|3350|3|877| +2450822|3352|3|91| +2450822|3355|3|476| +2450822|3356|3|477| +2450822|3358|3|748| +2450822|3361|3|346| +2450822|3362|3|750| +2450822|3364|3|140| +2450822|3367|3|411| +2450822|3368|3|824| +2450822|3370|3|587| +2450822|3373|3|372| +2450822|3374|3|861| +2450822|3376|3|77| +2450822|3379|3|310| +2450822|3380|3|577| +2450822|3382|3|948| +2450822|3385|3|152| +2450822|3386|3|105| +2450822|3388|3|939| +2450822|3391|3|361| +2450822|3392|3|29| +2450822|3394|3|178| +2450822|3397|3|501| +2450822|3398|3|196| +2450822|3400|3|888| +2450822|3403|3|50| +2450822|3404|3|| +2450822|3406|3|465| +2450822|3409|3|999| +2450822|3410|3|428| +2450822|3412|3|948| +2450822|3415|3|744| +2450822|3416|3|505| +2450822|3418|3|433| +2450822|3421|3|446| +2450822|3422|3|636| +2450822|3424|3|352| +2450822|3427|3|297| +2450822|3428|3|864| +2450822|3430|3|75| +2450822|3433|3|357| +2450822|3434|3|316| +2450822|3436|3|530| +2450822|3439|3|841| +2450822|3440|3|| +2450822|3442|3|512| +2450822|3445|3|203| +2450822|3446|3|842| +2450822|3448|3|525| +2450822|3451|3|257| +2450822|3452|3|194| +2450822|3454|3|361| +2450822|3457|3|364| +2450822|3458|3|164| +2450822|3460|3|356| +2450822|3463|3|| +2450822|3464|3|962| +2450822|3466|3|126| +2450822|3469|3|725| +2450822|3470|3|206| +2450822|3472|3|931| +2450822|3475|3|267| +2450822|3476|3|510| +2450822|3478|3|567| +2450822|3481|3|519| +2450822|3482|3|| +2450822|3484|3|841| +2450822|3487|3|645| +2450822|3488|3|70| +2450822|3490|3|576| +2450822|3493|3|650| +2450822|3494|3|328| +2450822|3496|3|339| +2450822|3499|3|519| +2450822|3500|3|791| +2450822|3502|3|741| +2450822|3505|3|678| +2450822|3506|3|103| +2450822|3508|3|255| +2450822|3511|3|958| +2450822|3512|3|685| +2450822|3514|3|946| +2450822|3517|3|429| +2450822|3518|3|938| +2450822|3520|3|851| +2450822|3523|3|749| +2450822|3524|3|878| +2450822|3526|3|374| +2450822|3529|3|979| +2450822|3530|3|180| +2450822|3532|3|22| +2450822|3535|3|201| +2450822|3536|3|273| +2450822|3538|3|142| +2450822|3541|3|60| +2450822|3542|3|933| +2450822|3544|3|657| +2450822|3547|3|| +2450822|3548|3|277| +2450822|3550|3|196| +2450822|3553|3|851| +2450822|3554|3|821| +2450822|3556|3|227| +2450822|3559|3|463| +2450822|3560|3|446| +2450822|3562|3|180| +2450822|3565|3|116| +2450822|3566|3|63| +2450822|3568|3|215| +2450822|3571|3|| +2450822|3572|3|427| +2450822|3574|3|389| +2450822|3577|3|46| +2450822|3578|3|656| +2450822|3580|3|520| +2450822|3583|3|878| +2450822|3584|3|63| +2450822|3586|3|730| +2450822|3589|3|271| +2450822|3590|3|382| +2450822|3592|3|176| +2450822|3595|3|| +2450822|3596|3|878| +2450822|3598|3|249| +2450822|3601|3|617| +2450822|3602|3|233| +2450822|3604|3|| +2450822|3607|3|454| +2450822|3608|3|562| +2450822|3610|3|807| +2450822|3613|3|418| +2450822|3614|3|997| +2450822|3616|3|507| +2450822|3619|3|728| +2450822|3620|3|909| +2450822|3622|3|319| +2450822|3625|3|180| +2450822|3626|3|826| +2450822|3628|3|793| +2450822|3631|3|837| +2450822|3632|3|589| +2450822|3634|3|827| +2450822|3637|3|| +2450822|3638|3|247| +2450822|3640|3|388| +2450822|3643|3|242| +2450822|3644|3|361| +2450822|3646|3|617| +2450822|3649|3|868| +2450822|3650|3|462| +2450822|3652|3|972| +2450822|3655|3|3| +2450822|3656|3|275| +2450822|3658|3|24| +2450822|3661|3|711| +2450822|3662|3|185| +2450822|3664|3|850| +2450822|3667|3|705| +2450822|3668|3|19| +2450822|3670|3|524| +2450822|3673|3|759| +2450822|3674|3|139| +2450822|3676|3|657| +2450822|3679|3|776| +2450822|3680|3|576| +2450822|3682|3|341| +2450822|3685|3|394| +2450822|3686|3|529| +2450822|3688|3|245| +2450822|3691|3|989| +2450822|3692|3|124| +2450822|3694|3|124| +2450822|3697|3|595| +2450822|3698|3|205| +2450822|3700|3|612| +2450822|3703|3|737| +2450822|3704|3|473| +2450822|3706|3|749| +2450822|3709|3|632| +2450822|3710|3|166| +2450822|3712|3|280| +2450822|3715|3|704| +2450822|3716|3|606| +2450822|3718|3|272| +2450822|3721|3|737| +2450822|3722|3|14| +2450822|3724|3|95| +2450822|3727|3|635| +2450822|3728|3|875| +2450822|3730|3|983| +2450822|3733|3|119| +2450822|3734|3|419| +2450822|3736|3|596| +2450822|3739|3|31| +2450822|3740|3|174| +2450822|3742|3|239| +2450822|3745|3|741| +2450822|3746|3|668| +2450822|3748|3|544| +2450822|3751|3|893| +2450822|3752|3|311| +2450822|3754|3|| +2450822|3757|3|427| +2450822|3758|3|448| +2450822|3760|3|805| +2450822|3763|3|263| +2450822|3764|3|| +2450822|3766|3|| +2450822|3769|3|254| +2450822|3770|3|701| +2450822|3772|3|867| +2450822|3775|3|155| +2450822|3776|3|566| +2450822|3778|3|747| +2450822|3781|3|932| +2450822|3782|3|629| +2450822|3784|3|882| +2450822|3787|3|768| +2450822|3788|3|131| +2450822|3790|3|491| +2450822|3793|3|615| +2450822|3794|3|476| +2450822|3796|3|713| +2450822|3799|3|491| +2450822|3800|3|433| +2450822|3802|3|841| +2450822|3805|3|475| +2450822|3806|3|33| +2450822|3808|3|498| +2450822|3811|3|7| +2450822|3812|3|708| +2450822|3814|3|501| +2450822|3817|3|754| +2450822|3818|3|646| +2450822|3820|3|| +2450822|3823|3|48| +2450822|3824|3|688| +2450822|3826|3|893| +2450822|3829|3|585| +2450822|3830|3|560| +2450822|3832|3|87| +2450822|3835|3|809| +2450822|3836|3|635| +2450822|3838|3|90| +2450822|3841|3|338| +2450822|3842|3|472| +2450822|3844|3|197| +2450822|3847|3|687| +2450822|3848|3|162| +2450822|3850|3|613| +2450822|3853|3|455| +2450822|3854|3|| +2450822|3856|3|295| +2450822|3859|3|178| +2450822|3860|3|469| +2450822|3862|3|833| +2450822|3865|3|103| +2450822|3866|3|450| +2450822|3868|3|841| +2450822|3871|3|501| +2450822|3872|3|| +2450822|3874|3|646| +2450822|3877|3|299| +2450822|3878|3|339| +2450822|3880|3|149| +2450822|3883|3|150| +2450822|3884|3|132| +2450822|3886|3|181| +2450822|3889|3|405| +2450822|3890|3|337| +2450822|3892|3|11| +2450822|3895|3|349| +2450822|3896|3|168| +2450822|3898|3|134| +2450822|3901|3|299| +2450822|3902|3|683| +2450822|3904|3|607| +2450822|3907|3|435| +2450822|3908|3|48| +2450822|3910|3|30| +2450822|3913|3|978| +2450822|3914|3|142| +2450822|3916|3|927| +2450822|3919|3|731| +2450822|3920|3|56| +2450822|3922|3|| +2450822|3925|3|660| +2450822|3926|3|364| +2450822|3928|3|111| +2450822|3931|3|762| +2450822|3932|3|660| +2450822|3934|3|13| +2450822|3937|3|623| +2450822|3938|3|716| +2450822|3940|3|539| +2450822|3943|3|583| +2450822|3944|3|370| +2450822|3946|3|57| +2450822|3949|3|169| +2450822|3950|3|912| +2450822|3952|3|737| +2450822|3955|3|853| +2450822|3956|3|637| +2450822|3958|3|461| +2450822|3961|3|645| +2450822|3962|3|793| +2450822|3964|3|433| +2450822|3967|3|681| +2450822|3968|3|433| +2450822|3970|3|536| +2450822|3973|3|392| +2450822|3974|3|142| +2450822|3976|3|321| +2450822|3979|3|367| +2450822|3980|3|76| +2450822|3982|3|688| +2450822|3985|3|356| +2450822|3986|3|782| +2450822|3988|3|678| +2450822|3991|3|796| +2450822|3992|3|217| +2450822|3994|3|799| +2450822|3997|3|786| +2450822|3998|3|49| +2450822|4000|3|259| +2450822|4003|3|354| +2450822|4004|3|13| +2450822|4006|3|141| +2450822|4009|3|164| +2450822|4010|3|733| +2450822|4012|3|616| +2450822|4015|3|835| +2450822|4016|3|155| +2450822|4018|3|851| +2450822|4021|3|490| +2450822|4022|3|124| +2450822|4024|3|90| +2450822|4027|3|567| +2450822|4028|3|406| +2450822|4030|3|537| +2450822|4033|3|187| +2450822|4034|3|92| +2450822|4036|3|407| +2450822|4039|3|924| +2450822|4040|3|15| +2450822|4042|3|710| +2450822|4045|3|850| +2450822|4046|3|71| +2450822|4048|3|718| +2450822|4051|3|657| +2450822|4052|3|444| +2450822|4054|3|529| +2450822|4057|3|4| +2450822|4058|3|498| +2450822|4060|3|478| +2450822|4063|3|74| +2450822|4064|3|124| +2450822|4066|3|649| +2450822|4069|3|554| +2450822|4070|3|539| +2450822|4072|3|234| +2450822|4075|3|968| +2450822|4076|3|946| +2450822|4078|3|515| +2450822|4081|3|359| +2450822|4082|3|978| +2450822|4084|3|660| +2450822|4087|3|345| +2450822|4088|3|860| +2450822|4090|3|767| +2450822|4093|3|602| +2450822|4094|3|211| +2450822|4096|3|949| +2450822|4099|3|911| +2450822|4100|3|852| +2450822|4102|3|506| +2450822|4105|3|91| +2450822|4106|3|796| +2450822|4108|3|296| +2450822|4111|3|| +2450822|4112|3|640| +2450822|4114|3|909| +2450822|4117|3|150| +2450822|4118|3|155| +2450822|4120|3|293| +2450822|4123|3|303| +2450822|4124|3|493| +2450822|4126|3|| +2450822|4129|3|591| +2450822|4130|3|| +2450822|4132|3|629| +2450822|4135|3|470| +2450822|4136|3|785| +2450822|4138|3|879| +2450822|4141|3|876| +2450822|4142|3|738| +2450822|4144|3|365| +2450822|4147|3|573| +2450822|4148|3|36| +2450822|4150|3|125| +2450822|4153|3|999| +2450822|4154|3|815| +2450822|4156|3|301| +2450822|4159|3|56| +2450822|4160|3|| +2450822|4162|3|852| +2450822|4165|3|| +2450822|4166|3|5| +2450822|4168|3|439| +2450822|4171|3|761| +2450822|4172|3|869| +2450822|4174|3|| +2450822|4177|3|103| +2450822|4178|3|89| +2450822|4180|3|519| +2450822|4183|3|658| +2450822|4184|3|520| +2450822|4186|3|526| +2450822|4189|3|255| +2450822|4190|3|370| +2450822|4192|3|931| +2450822|4195|3|301| +2450822|4196|3|228| +2450822|4198|3|380| +2450822|4201|3|213| +2450822|4202|3|759| +2450822|4204|3|580| +2450822|4207|3|825| +2450822|4208|3|200| +2450822|4210|3|636| +2450822|4213|3|| +2450822|4214|3|873| +2450822|4216|3|87| +2450822|4219|3|| +2450822|4220|3|522| +2450822|4222|3|240| +2450822|4225|3|23| +2450822|4226|3|209| +2450822|4228|3|779| +2450822|4231|3|466| +2450822|4232|3|239| +2450822|4234|3|303| +2450822|4237|3|419| +2450822|4238|3|998| +2450822|4240|3|229| +2450822|4243|3|420| +2450822|4244|3|60| +2450822|4246|3|192| +2450822|4249|3|878| +2450822|4250|3|956| +2450822|4252|3|800| +2450822|4255|3|525| +2450822|4256|3|869| +2450822|4258|3|355| +2450822|4261|3|171| +2450822|4262|3|448| +2450822|4264|3|837| +2450822|4267|3|702| +2450822|4268|3|907| +2450822|4270|3|990| +2450822|4273|3|117| +2450822|4274|3|997| +2450822|4276|3|534| +2450822|4279|3|655| +2450822|4280|3|949| +2450822|4282|3|553| +2450822|4285|3|798| +2450822|4286|3|757| +2450822|4288|3|135| +2450822|4291|3|981| +2450822|4292|3|635| +2450822|4294|3|711| +2450822|4297|3|360| +2450822|4298|3|306| +2450822|4300|3|422| +2450822|4303|3|110| +2450822|4304|3|385| +2450822|4306|3|469| +2450822|4309|3|687| +2450822|4310|3|903| +2450822|4312|3|319| +2450822|4315|3|995| +2450822|4316|3|122| +2450822|4318|3|272| +2450822|4321|3|602| +2450822|4322|3|| +2450822|4324|3|999| +2450822|4327|3|919| +2450822|4328|3|246| +2450822|4330|3|310| +2450822|4333|3|505| +2450822|4334|3|| +2450822|4336|3|512| +2450822|4339|3|874| +2450822|4340|3|190| +2450822|4342|3|165| +2450822|4345|3|563| +2450822|4346|3|89| +2450822|4348|3|124| +2450822|4351|3|138| +2450822|4352|3|633| +2450822|4354|3|917| +2450822|4357|3|787| +2450822|4358|3|196| +2450822|4360|3|572| +2450822|4363|3|756| +2450822|4364|3|378| +2450822|4366|3|373| +2450822|4369|3|928| +2450822|4370|3|74| +2450822|4372|3|575| +2450822|4375|3|239| +2450822|4376|3|759| +2450822|4378|3|675| +2450822|4381|3|974| +2450822|4382|3|672| +2450822|4384|3|212| +2450822|4387|3|125| +2450822|4388|3|526| +2450822|4390|3|985| +2450822|4393|3|962| +2450822|4394|3|860| +2450822|4396|3|97| +2450822|4399|3|802| +2450822|4400|3|795| +2450822|4402|3|61| +2450822|4405|3|494| +2450822|4406|3|449| +2450822|4408|3|922| +2450822|4411|3|809| +2450822|4412|3|861| +2450822|4414|3|954| +2450822|4417|3|906| +2450822|4418|3|386| +2450822|4420|3|826| +2450822|4423|3|320| +2450822|4424|3|954| +2450822|4426|3|446| +2450822|4429|3|77| +2450822|4430|3|244| +2450822|4432|3|383| +2450822|4435|3|253| +2450822|4436|3|168| +2450822|4438|3|582| +2450822|4441|3|242| +2450822|4442|3|859| +2450822|4444|3|636| +2450822|4447|3|263| +2450822|4448|3|288| +2450822|4450|3|443| +2450822|4453|3|758| +2450822|4454|3|280| +2450822|4456|3|517| +2450822|4459|3|992| +2450822|4460|3|252| +2450822|4462|3|760| +2450822|4465|3|127| +2450822|4466|3|903| +2450822|4468|3|234| +2450822|4471|3|536| +2450822|4472|3|77| +2450822|4474|3|| +2450822|4477|3|| +2450822|4478|3|| +2450822|4480|3|| +2450822|4483|3|189| +2450822|4484|3|165| +2450822|4486|3|| +2450822|4489|3|615| +2450822|4490|3|622| +2450822|4492|3|169| +2450822|4495|3|244| +2450822|4496|3|439| +2450822|4498|3|508| +2450822|4501|3|651| +2450822|4502|3|115| +2450822|4504|3|208| +2450822|4507|3|782| +2450822|4508|3|324| +2450822|4510|3|581| +2450822|4513|3|692| +2450822|4514|3|680| +2450822|4516|3|45| +2450822|4519|3|891| +2450822|4520|3|813| +2450822|4522|3|94| +2450822|4525|3|334| +2450822|4526|3|92| +2450822|4528|3|749| +2450822|4531|3|176| +2450822|4532|3|774| +2450822|4534|3|312| +2450822|4537|3|733| +2450822|4538|3|792| +2450822|4540|3|738| +2450822|4543|3|219| +2450822|4544|3|443| +2450822|4546|3|293| +2450822|4549|3|813| +2450822|4550|3|| +2450822|4552|3|874| +2450822|4555|3|431| +2450822|4556|3|766| +2450822|4558|3|15| +2450822|4561|3|720| +2450822|4562|3|746| +2450822|4564|3|92| +2450822|4567|3|392| +2450822|4568|3|310| +2450822|4570|3|224| +2450822|4573|3|342| +2450822|4574|3|737| +2450822|4576|3|433| +2450822|4579|3|515| +2450822|4580|3|470| +2450822|4582|3|632| +2450822|4585|3|543| +2450822|4586|3|52| +2450822|4588|3|299| +2450822|4591|3|329| +2450822|4592|3|895| +2450822|4594|3|698| +2450822|4597|3|225| +2450822|4598|3|202| +2450822|4600|3|604| +2450822|4603|3|568| +2450822|4604|3|402| +2450822|4606|3|739| +2450822|4609|3|476| +2450822|4610|3|736| +2450822|4612|3|977| +2450822|4615|3|609| +2450822|4616|3|994| +2450822|4618|3|45| +2450822|4621|3|289| +2450822|4622|3|874| +2450822|4624|3|218| +2450822|4627|3|275| +2450822|4628|3|749| +2450822|4630|3|| +2450822|4633|3|152| +2450822|4634|3|291| +2450822|4636|3|697| +2450822|4639|3|500| +2450822|4640|3|792| +2450822|4642|3|472| +2450822|4645|3|44| +2450822|4646|3|265| +2450822|4648|3|157| +2450822|4651|3|263| +2450822|4652|3|687| +2450822|4654|3|217| +2450822|4657|3|813| +2450822|4658|3|81| +2450822|4660|3|838| +2450822|4663|3|565| +2450822|4664|3|973| +2450822|4666|3|170| +2450822|4669|3|263| +2450822|4670|3|668| +2450822|4672|3|727| +2450822|4675|3|354| +2450822|4676|3|6| +2450822|4678|3|798| +2450822|4681|3|437| +2450822|4682|3|345| +2450822|4684|3|229| +2450822|4687|3|830| +2450822|4688|3|983| +2450822|4690|3|471| +2450822|4693|3|294| +2450822|4694|3|551| +2450822|4696|3|| +2450822|4699|3|963| +2450822|4700|3|974| +2450822|4702|3|901| +2450822|4705|3|476| +2450822|4706|3|281| +2450822|4708|3|735| +2450822|4711|3|771| +2450822|4712|3|212| +2450822|4714|3|172| +2450822|4717|3|280| +2450822|4718|3|891| +2450822|4720|3|965| +2450822|4723|3|526| +2450822|4724|3|741| +2450822|4726|3|281| +2450822|4729|3|135| +2450822|4730|3|962| +2450822|4732|3|| +2450822|4735|3|484| +2450822|4736|3|377| +2450822|4738|3|30| +2450822|4741|3|758| +2450822|4742|3|974| +2450822|4744|3|222| +2450822|4747|3|112| +2450822|4748|3|799| +2450822|4750|3|960| +2450822|4753|3|259| +2450822|4754|3|323| +2450822|4756|3|63| +2450822|4759|3|181| +2450822|4760|3|| +2450822|4762|3|641| +2450822|4765|3|654| +2450822|4766|3|217| +2450822|4768|3|564| +2450822|4771|3|573| +2450822|4772|3|638| +2450822|4774|3|164| +2450822|4777|3|860| +2450822|4778|3|| +2450822|4780|3|| +2450822|4783|3|565| +2450822|4784|3|317| +2450822|4786|3|430| +2450822|4789|3|509| +2450822|4790|3|| +2450822|4792|3|932| +2450822|4795|3|49| +2450822|4796|3|741| +2450822|4798|3|600| +2450822|4801|3|387| +2450822|4802|3|554| +2450822|4804|3|352| +2450822|4807|3|733| +2450822|4808|3|603| +2450822|4810|3|345| +2450822|4813|3|535| +2450822|4814|3|176| +2450822|4816|3|35| +2450822|4819|3|81| +2450822|4820|3|873| +2450822|4822|3|88| +2450822|4825|3|580| +2450822|4826|3|| +2450822|4828|3|401| +2450822|4831|3|929| +2450822|4832|3|762| +2450822|4834|3|370| +2450822|4837|3|892| +2450822|4838|3|782| +2450822|4840|3|77| +2450822|4843|3|929| +2450822|4844|3|239| +2450822|4846|3|593| +2450822|4849|3|20| +2450822|4850|3|786| +2450822|4852|3|177| +2450822|4855|3|993| +2450822|4856|3|46| +2450822|4858|3|708| +2450822|4861|3|| +2450822|4862|3|768| +2450822|4864|3|829| +2450822|4867|3|816| +2450822|4868|3|998| +2450822|4870|3|864| +2450822|4873|3|492| +2450822|4874|3|136| +2450822|4876|3|905| +2450822|4879|3|221| +2450822|4880|3|101| +2450822|4882|3|312| +2450822|4885|3|780| +2450822|4886|3|315| +2450822|4888|3|227| +2450822|4891|3|431| +2450822|4892|3|410| +2450822|4894|3|61| +2450822|4897|3|325| +2450822|4898|3|251| +2450822|4900|3|284| +2450822|4903|3|505| +2450822|4904|3|370| +2450822|4906|3|118| +2450822|4909|3|859| +2450822|4910|3|866| +2450822|4912|3|258| +2450822|4915|3|299| +2450822|4916|3|436| +2450822|4918|3|757| +2450822|4921|3|210| +2450822|4922|3|365| +2450822|4924|3|| +2450822|4927|3|315| +2450822|4928|3|27| +2450822|4930|3|698| +2450822|4933|3|518| +2450822|4934|3|49| +2450822|4936|3|972| +2450822|4939|3|606| +2450822|4940|3|527| +2450822|4942|3|153| +2450822|4945|3|836| +2450822|4946|3|625| +2450822|4948|3|| +2450822|4951|3|| +2450822|4952|3|843| +2450822|4954|3|491| +2450822|4957|3|| +2450822|4958|3|707| +2450822|4960|3|497| +2450822|4963|3|996| +2450822|4964|3|910| +2450822|4966|3|746| +2450822|4969|3|816| +2450822|4970|3|747| +2450822|4972|3|967| +2450822|4975|3|702| +2450822|4976|3|24| +2450822|4978|3|625| +2450822|4981|3|686| +2450822|4982|3|987| +2450822|4984|3|932| +2450822|4987|3|972| +2450822|4988|3|308| +2450822|4990|3|587| +2450822|4993|3|384| +2450822|4994|3|477| +2450822|4996|3|180| +2450822|4999|3|459| +2450822|5000|3|494| +2450822|5002|3|390| +2450822|5005|3|950| +2450822|5006|3|926| +2450822|5008|3|847| +2450822|5011|3|693| +2450822|5012|3|824| +2450822|5014|3|623| +2450822|5017|3|823| +2450822|5018|3|354| +2450822|5020|3|695| +2450822|5023|3|980| +2450822|5024|3|445| +2450822|5026|3|797| +2450822|5029|3|125| +2450822|5030|3|818| +2450822|5032|3|868| +2450822|5035|3|799| +2450822|5036|3|860| +2450822|5038|3|229| +2450822|5041|3|877| +2450822|5042|3|977| +2450822|5044|3|365| +2450822|5047|3|733| +2450822|5048|3|468| +2450822|5050|3|643| +2450822|5053|3|767| +2450822|5054|3|783| +2450822|5056|3|768| +2450822|5059|3|771| +2450822|5060|3|710| +2450822|5062|3|| +2450822|5065|3|269| +2450822|5066|3|142| +2450822|5068|3|759| +2450822|5071|3|696| +2450822|5072|3|819| +2450822|5074|3|804| +2450822|5077|3|688| +2450822|5078|3|912| +2450822|5080|3|25| +2450822|5083|3|869| +2450822|5084|3|972| +2450822|5086|3|| +2450822|5089|3|256| +2450822|5090|3|542| +2450822|5092|3|569| +2450822|5095|3|388| +2450822|5096|3|795| +2450822|5098|3|170| +2450822|5101|3|451| +2450822|5102|3|562| +2450822|5104|3|998| +2450822|5107|3|906| +2450822|5108|3|536| +2450822|5110|3|| +2450822|5113|3|226| +2450822|5114|3|633| +2450822|5116|3|563| +2450822|5119|3|250| +2450822|5120|3|151| +2450822|5122|3|52| +2450822|5125|3|607| +2450822|5126|3|345| +2450822|5128|3|537| +2450822|5131|3|| +2450822|5132|3|34| +2450822|5134|3|493| +2450822|5137|3|624| +2450822|5138|3|597| +2450822|5140|3|661| +2450822|5143|3|772| +2450822|5144|3|757| +2450822|5146|3|445| +2450822|5149|3|518| +2450822|5150|3|979| +2450822|5152|3|239| +2450822|5155|3|83| +2450822|5156|3|690| +2450822|5158|3|213| +2450822|5161|3|765| +2450822|5162|3|887| +2450822|5164|3|781| +2450822|5167|3|| +2450822|5168|3|968| +2450822|5170|3|916| +2450822|5173|3|| +2450822|5174|3|754| +2450822|5176|3|272| +2450822|5179|3|975| +2450822|5180|3|874| +2450822|5182|3|189| +2450822|5185|3|412| +2450822|5186|3|| +2450822|5188|3|923| +2450822|5191|3|416| +2450822|5192|3|234| +2450822|5194|3|540| +2450822|5197|3|210| +2450822|5198|3|329| +2450822|5200|3|240| +2450822|5203|3|211| +2450822|5204|3|728| +2450822|5206|3|497| +2450822|5209|3|735| +2450822|5210|3|262| +2450822|5212|3|904| +2450822|5215|3|| +2450822|5216|3|231| +2450822|5218|3|135| +2450822|5221|3|152| +2450822|5222|3|302| +2450822|5224|3|288| +2450822|5227|3|10| +2450822|5228|3|175| +2450822|5230|3|266| +2450822|5233|3|545| +2450822|5234|3|809| +2450822|5236|3|254| +2450822|5239|3|676| +2450822|5240|3|444| +2450822|5242|3|796| +2450822|5245|3|| +2450822|5246|3|456| +2450822|5248|3|600| +2450822|5251|3|277| +2450822|5252|3|947| +2450822|5254|3|693| +2450822|5257|3|182| +2450822|5258|3|207| +2450822|5260|3|152| +2450822|5263|3|864| +2450822|5264|3|55| +2450822|5266|3|825| +2450822|5269|3|852| +2450822|5270|3|| +2450822|5272|3|36| +2450822|5275|3|505| +2450822|5276|3|237| +2450822|5278|3|| +2450822|5281|3|829| +2450822|5282|3|843| +2450822|5284|3|908| +2450822|5287|3|425| +2450822|5288|3|379| +2450822|5290|3|771| +2450822|5293|3|132| +2450822|5294|3|95| +2450822|5296|3|580| +2450822|5299|3|556| +2450822|5300|3|886| +2450822|5302|3|253| +2450822|5305|3|178| +2450822|5306|3|785| +2450822|5308|3|788| +2450822|5311|3|119| +2450822|5312|3|124| +2450822|5314|3|715| +2450822|5317|3|84| +2450822|5318|3|441| +2450822|5320|3|419| +2450822|5323|3|116| +2450822|5324|3|570| +2450822|5326|3|505| +2450822|5329|3|874| +2450822|5330|3|242| +2450822|5332|3|546| +2450822|5335|3|3| +2450822|5336|3|189| +2450822|5338|3|492| +2450822|5341|3|728| +2450822|5342|3|358| +2450822|5344|3|115| +2450822|5347|3|205| +2450822|5348|3|75| +2450822|5350|3|842| +2450822|5353|3|770| +2450822|5354|3|907| +2450822|5356|3|246| +2450822|5359|3|224| +2450822|5360|3|266| +2450822|5362|3|633| +2450822|5365|3|967| +2450822|5366|3|523| +2450822|5368|3|641| +2450822|5371|3|31| +2450822|5372|3|430| +2450822|5374|3|867| +2450822|5377|3|810| +2450822|5378|3|739| +2450822|5380|3|100| +2450822|5383|3|506| +2450822|5384|3|508| +2450822|5386|3|884| +2450822|5389|3|299| +2450822|5390|3|615| +2450822|5392|3|787| +2450822|5395|3|304| +2450822|5396|3|290| +2450822|5398|3|352| +2450822|5401|3|773| +2450822|5402|3|218| +2450822|5404|3|778| +2450822|5407|3|994| +2450822|5408|3|855| +2450822|5410|3|308| +2450822|5413|3|478| +2450822|5414|3|15| +2450822|5416|3|510| +2450822|5419|3|702| +2450822|5420|3|916| +2450822|5422|3|822| +2450822|5425|3|165| +2450822|5426|3|952| +2450822|5428|3|648| +2450822|5431|3|502| +2450822|5432|3|898| +2450822|5434|3|46| +2450822|5437|3|498| +2450822|5438|3|591| +2450822|5440|3|964| +2450822|5443|3|938| +2450822|5444|3|537| +2450822|5446|3|289| +2450822|5449|3|238| +2450822|5450|3|419| +2450822|5452|3|396| +2450822|5455|3|42| +2450822|5456|3|261| +2450822|5458|3|553| +2450822|5461|3|176| +2450822|5462|3|532| +2450822|5464|3|844| +2450822|5467|3|753| +2450822|5468|3|675| +2450822|5470|3|164| +2450822|5473|3|79| +2450822|5474|3|662| +2450822|5476|3|228| +2450822|5479|3|470| +2450822|5480|3|374| +2450822|5482|3|965| +2450822|5485|3|512| +2450822|5486|3|921| +2450822|5488|3|323| +2450822|5491|3|229| +2450822|5492|3|770| +2450822|5494|3|93| +2450822|5497|3|18| +2450822|5498|3|767| +2450822|5500|3|| +2450822|5503|3|27| +2450822|5504|3|867| +2450822|5506|3|818| +2450822|5509|3|983| +2450822|5510|3|682| +2450822|5512|3|94| +2450822|5515|3|342| +2450822|5516|3|313| +2450822|5518|3|755| +2450822|5521|3|998| +2450822|5522|3|| +2450822|5524|3|208| +2450822|5527|3|547| +2450822|5528|3|552| +2450822|5530|3|639| +2450822|5533|3|524| +2450822|5534|3|793| +2450822|5536|3|24| +2450822|5539|3|424| +2450822|5540|3|256| +2450822|5542|3|93| +2450822|5545|3|913| +2450822|5546|3|81| +2450822|5548|3|103| +2450822|5551|3|986| +2450822|5552|3|548| +2450822|5554|3|383| +2450822|5557|3|498| +2450822|5558|3|637| +2450822|5560|3|234| +2450822|5563|3|278| +2450822|5564|3|501| +2450822|5566|3|122| +2450822|5569|3|682| +2450822|5570|3|412| +2450822|5572|3|906| +2450822|5575|3|501| +2450822|5576|3|117| +2450822|5578|3|901| +2450822|5581|3|668| +2450822|5582|3|| +2450822|5584|3|396| +2450822|5587|3|616| +2450822|5588|3|| +2450822|5590|3|516| +2450822|5593|3|455| +2450822|5594|3|70| +2450822|5596|3|331| +2450822|5599|3|959| +2450822|5600|3|44| +2450822|5602|3|| +2450822|5605|3|109| +2450822|5606|3|485| +2450822|5608|3|689| +2450822|5611|3|26| +2450822|5612|3|96| +2450822|5614|3|950| +2450822|5617|3|31| +2450822|5618|3|534| +2450822|5620|3|261| +2450822|5623|3|| +2450822|5624|3|120| +2450822|5626|3|38| +2450822|5629|3|990| +2450822|5630|3|348| +2450822|5632|3|193| +2450822|5635|3|214| +2450822|5636|3|926| +2450822|5638|3|315| +2450822|5641|3|704| +2450822|5642|3|384| +2450822|5644|3|220| +2450822|5647|3|780| +2450822|5648|3|1000| +2450822|5650|3|287| +2450822|5653|3|| +2450822|5654|3|845| +2450822|5656|3|713| +2450822|5659|3|796| +2450822|5660|3|79| +2450822|5662|3|352| +2450822|5665|3|677| +2450822|5666|3|397| +2450822|5668|3|159| +2450822|5671|3|421| +2450822|5672|3|539| +2450822|5674|3|282| +2450822|5677|3|882| +2450822|5678|3|988| +2450822|5680|3|47| +2450822|5683|3|902| +2450822|5684|3|881| +2450822|5686|3|471| +2450822|5689|3|664| +2450822|5690|3|620| +2450822|5692|3|600| +2450822|5695|3|641| +2450822|5696|3|114| +2450822|5698|3|729| +2450822|5701|3|655| +2450822|5702|3|826| +2450822|5704|3|641| +2450822|5707|3|194| +2450822|5708|3|837| +2450822|5710|3|685| +2450822|5713|3|977| +2450822|5714|3|640| +2450822|5716|3|793| +2450822|5719|3|801| +2450822|5720|3|284| +2450822|5722|3|629| +2450822|5725|3|673| +2450822|5726|3|252| +2450822|5728|3|452| +2450822|5731|3|677| +2450822|5732|3|245| +2450822|5734|3|643| +2450822|5737|3|15| +2450822|5738|3|| +2450822|5740|3|370| +2450822|5743|3|495| +2450822|5744|3|96| +2450822|5746|3|353| +2450822|5749|3|268| +2450822|5750|3|530| +2450822|5752|3|558| +2450822|5755|3|487| +2450822|5756|3|712| +2450822|5758|3|684| +2450822|5761|3|181| +2450822|5762|3|616| +2450822|5764|3|581| +2450822|5767|3|254| +2450822|5768|3|868| +2450822|5770|3|106| +2450822|5773|3|637| +2450822|5774|3|275| +2450822|5776|3|941| +2450822|5779|3|457| +2450822|5780|3|206| +2450822|5782|3|147| +2450822|5785|3|325| +2450822|5786|3|300| +2450822|5788|3|952| +2450822|5791|3|786| +2450822|5792|3|595| +2450822|5794|3|870| +2450822|5797|3|324| +2450822|5798|3|| +2450822|5800|3|942| +2450822|5803|3|| +2450822|5804|3|223| +2450822|5806|3|670| +2450822|5809|3|749| +2450822|5810|3|615| +2450822|5812|3|857| +2450822|5815|3|683| +2450822|5816|3|438| +2450822|5818|3|228| +2450822|5821|3|| +2450822|5822|3|911| +2450822|5824|3|| +2450822|5827|3|445| +2450822|5828|3|518| +2450822|5830|3|930| +2450822|5833|3|916| +2450822|5834|3|706| +2450822|5836|3|465| +2450822|5839|3|867| +2450822|5840|3|56| +2450822|5842|3|621| +2450822|5845|3|267| +2450822|5846|3|649| +2450822|5848|3|761| +2450822|5851|3|971| +2450822|5852|3|903| +2450822|5854|3|162| +2450822|5857|3|986| +2450822|5858|3|117| +2450822|5860|3|362| +2450822|5863|3|57| +2450822|5864|3|402| +2450822|5866|3|206| +2450822|5869|3|594| +2450822|5870|3|345| +2450822|5872|3|254| +2450822|5875|3|403| +2450822|5876|3|904| +2450822|5878|3|119| +2450822|5881|3|440| +2450822|5882|3|843| +2450822|5884|3|507| +2450822|5887|3|15| +2450822|5888|3|807| +2450822|5890|3|457| +2450822|5893|3|307| +2450822|5894|3|341| +2450822|5896|3|| +2450822|5899|3|358| +2450822|5900|3|436| +2450822|5902|3|773| +2450822|5905|3|422| +2450822|5906|3|226| +2450822|5908|3|628| +2450822|5911|3|736| +2450822|5912|3|867| +2450822|5914|3|919| +2450822|5917|3|544| +2450822|5918|3|| +2450822|5920|3|905| +2450822|5923|3|252| +2450822|5924|3|163| +2450822|5926|3|38| +2450822|5929|3|715| +2450822|5930|3|651| +2450822|5932|3|648| +2450822|5935|3|903| +2450822|5936|3|210| +2450822|5938|3|647| +2450822|5941|3|839| +2450822|5942|3|584| +2450822|5944|3|132| +2450822|5947|3|909| +2450822|5948|3|526| +2450822|5950|3|823| +2450822|5953|3|15| +2450822|5954|3|331| +2450822|5956|3|671| +2450822|5959|3|17| +2450822|5960|3|39| +2450822|5962|3|989| +2450822|5965|3|486| +2450822|5966|3|248| +2450822|5968|3|| +2450822|5971|3|8| +2450822|5972|3|74| +2450822|5974|3|694| +2450822|5977|3|374| +2450822|5978|3|989| +2450822|5980|3|136| +2450822|5983|3|957| +2450822|5984|3|570| +2450822|5986|3|781| +2450822|5989|3|385| +2450822|5990|3|577| +2450822|5992|3|| +2450822|5995|3|944| +2450822|5996|3|459| +2450822|5998|3|621| +2450822|6001|3|115| +2450822|6002|3|905| +2450822|6004|3|686| +2450822|6007|3|773| +2450822|6008|3|428| +2450822|6010|3|89| +2450822|6013|3|| +2450822|6014|3|753| +2450822|6016|3|638| +2450822|6019|3|860| +2450822|6020|3|213| +2450822|6022|3|25| +2450822|6025|3|554| +2450822|6026|3|87| +2450822|6028|3|475| +2450822|6031|3|889| +2450822|6032|3|763| +2450822|6034|3|481| +2450822|6037|3|504| +2450822|6038|3|840| +2450822|6040|3|840| +2450822|6043|3|46| +2450822|6044|3|746| +2450822|6046|3|705| +2450822|6049|3|361| +2450822|6050|3|189| +2450822|6052|3|448| +2450822|6055|3|821| +2450822|6056|3|307| +2450822|6058|3|286| +2450822|6061|3|37| +2450822|6062|3|170| +2450822|6064|3|868| +2450822|6067|3|837| +2450822|6068|3|983| +2450822|6070|3|400| +2450822|6073|3|940| +2450822|6074|3|993| +2450822|6076|3|723| +2450822|6079|3|562| +2450822|6080|3|269| +2450822|6082|3|450| +2450822|6085|3|900| +2450822|6086|3|955| +2450822|6088|3|982| +2450822|6091|3|480| +2450822|6092|3|463| +2450822|6094|3|817| +2450822|6097|3|124| +2450822|6098|3|386| +2450822|6100|3|938| +2450822|6103|3|961| +2450822|6104|3|670| +2450822|6106|3|185| +2450822|6109|3|490| +2450822|6110|3|153| +2450822|6112|3|982| +2450822|6115|3|445| +2450822|6116|3|386| +2450822|6118|3|733| +2450822|6121|3|748| +2450822|6122|3|205| +2450822|6124|3|678| +2450822|6127|3|853| +2450822|6128|3|564| +2450822|6130|3|171| +2450822|6133|3|666| +2450822|6134|3|437| +2450822|6136|3|182| +2450822|6139|3|682| +2450822|6140|3|466| +2450822|6142|3|770| +2450822|6145|3|603| +2450822|6146|3|393| +2450822|6148|3|934| +2450822|6151|3|528| +2450822|6152|3|872| +2450822|6154|3|995| +2450822|6157|3|808| +2450822|6158|3|947| +2450822|6160|3|884| +2450822|6163|3|443| +2450822|6164|3|733| +2450822|6166|3|438| +2450822|6169|3|878| +2450822|6170|3|892| +2450822|6172|3|243| +2450822|6175|3|557| +2450822|6176|3|816| +2450822|6178|3|873| +2450822|6181|3|829| +2450822|6182|3|677| +2450822|6184|3|946| +2450822|6187|3|884| +2450822|6188|3|778| +2450822|6190|3|372| +2450822|6193|3|122| +2450822|6194|3|174| +2450822|6196|3|652| +2450822|6199|3|735| +2450822|6200|3|480| +2450822|6202|3|549| +2450822|6205|3|754| +2450822|6206|3|| +2450822|6208|3|| +2450822|6211|3|| +2450822|6212|3|71| +2450822|6214|3|696| +2450822|6217|3|263| +2450822|6218|3|354| +2450822|6220|3|854| +2450822|6223|3|996| +2450822|6224|3|939| +2450822|6226|3|588| +2450822|6229|3|97| +2450822|6230|3|530| +2450822|6232|3|399| +2450822|6235|3|822| +2450822|6236|3|655| +2450822|6238|3|904| +2450822|6241|3|71| +2450822|6242|3|775| +2450822|6244|3|899| +2450822|6247|3|415| +2450822|6248|3|210| +2450822|6250|3|404| +2450822|6253|3|257| +2450822|6254|3|684| +2450822|6256|3|403| +2450822|6259|3|661| +2450822|6260|3|786| +2450822|6262|3|115| +2450822|6265|3|430| +2450822|6266|3|9| +2450822|6268|3|974| +2450822|6271|3|900| +2450822|6272|3|488| +2450822|6274|3|552| +2450822|6277|3|814| +2450822|6278|3|145| +2450822|6280|3|88| +2450822|6283|3|624| +2450822|6284|3|357| +2450822|6286|3|660| +2450822|6289|3|959| +2450822|6290|3|555| +2450822|6292|3|203| +2450822|6295|3|353| +2450822|6296|3|836| +2450822|6298|3|351| +2450822|6301|3|65| +2450822|6302|3|120| +2450822|6304|3|375| +2450822|6307|3|194| +2450822|6308|3|567| +2450822|6310|3|413| +2450822|6313|3|604| +2450822|6314|3|496| +2450822|6316|3|896| +2450822|6319|3|188| +2450822|6320|3|505| +2450822|6322|3|627| +2450822|6325|3|| +2450822|6326|3|622| +2450822|6328|3|999| +2450822|6331|3|0| +2450822|6332|3|394| +2450822|6334|3|331| +2450822|6337|3|287| +2450822|6338|3|901| +2450822|6340|3|471| +2450822|6343|3|102| +2450822|6344|3|604| +2450822|6346|3|| +2450822|6349|3|875| +2450822|6350|3|358| +2450822|6352|3|| +2450822|6355|3|127| +2450822|6356|3|314| +2450822|6358|3|725| +2450822|6361|3|917| +2450822|6362|3|863| +2450822|6364|3|495| +2450822|6367|3|935| +2450822|6368|3|207| +2450822|6370|3|912| +2450822|6373|3|433| +2450822|6374|3|160| +2450822|6376|3|607| +2450822|6379|3|605| +2450822|6380|3|271| +2450822|6382|3|988| +2450822|6385|3|919| +2450822|6386|3|909| +2450822|6388|3|919| +2450822|6391|3|875| +2450822|6392|3|574| +2450822|6394|3|163| +2450822|6397|3|486| +2450822|6398|3|| +2450822|6400|3|374| +2450822|6403|3|87| +2450822|6404|3|879| +2450822|6406|3|669| +2450822|6409|3|249| +2450822|6410|3|| +2450822|6412|3|262| +2450822|6415|3|684| +2450822|6416|3|26| +2450822|6418|3|797| +2450822|6421|3|549| +2450822|6422|3|762| +2450822|6424|3|498| +2450822|6427|3|33| +2450822|6428|3|6| +2450822|6430|3|590| +2450822|6433|3|724| +2450822|6434|3|41| +2450822|6436|3|651| +2450822|6439|3|702| +2450822|6440|3|91| +2450822|6442|3|266| +2450822|6445|3|851| +2450822|6446|3|842| +2450822|6448|3|111| +2450822|6451|3|151| +2450822|6452|3|266| +2450822|6454|3|910| +2450822|6457|3|| +2450822|6458|3|775| +2450822|6460|3|131| +2450822|6463|3|170| +2450822|6464|3|773| +2450822|6466|3|580| +2450822|6469|3|984| +2450822|6470|3|92| +2450822|6472|3|781| +2450822|6475|3|991| +2450822|6476|3|65| +2450822|6478|3|88| +2450822|6481|3|251| +2450822|6482|3|220| +2450822|6484|3|717| +2450822|6487|3|612| +2450822|6488|3|892| +2450822|6490|3|394| +2450822|6493|3|673| +2450822|6494|3|397| +2450822|6496|3|548| +2450822|6499|3|786| +2450822|6500|3|272| +2450822|6502|3|181| +2450822|6505|3|65| +2450822|6506|3|360| +2450822|6508|3|386| +2450822|6511|3|119| +2450822|6512|3|978| +2450822|6514|3|47| +2450822|6517|3|252| +2450822|6518|3|301| +2450822|6520|3|846| +2450822|6523|3|129| +2450822|6524|3|778| +2450822|6526|3|| +2450822|6529|3|| +2450822|6530|3|242| +2450822|6532|3|946| +2450822|6535|3|805| +2450822|6536|3|893| +2450822|6538|3|824| +2450822|6541|3|| +2450822|6542|3|132| +2450822|6544|3|644| +2450822|6547|3|691| +2450822|6548|3|392| +2450822|6550|3|585| +2450822|6553|3|749| +2450822|6554|3|630| +2450822|6556|3|581| +2450822|6559|3|30| +2450822|6560|3|636| +2450822|6562|3|865| +2450822|6565|3|235| +2450822|6566|3|| +2450822|6568|3|| +2450822|6571|3|185| +2450822|6572|3|96| +2450822|6574|3|289| +2450822|6577|3|264| +2450822|6578|3|904| +2450822|6580|3|817| +2450822|6583|3|433| +2450822|6584|3|517| +2450822|6586|3|518| +2450822|6589|3|244| +2450822|6590|3|614| +2450822|6592|3|195| +2450822|6595|3|456| +2450822|6596|3|262| +2450822|6598|3|271| +2450822|6601|3|449| +2450822|6602|3|271| +2450822|6604|3|720| +2450822|6607|3|489| +2450822|6608|3|383| +2450822|6610|3|631| +2450822|6613|3|17| +2450822|6614|3|954| +2450822|6616|3|422| +2450822|6619|3|868| +2450822|6620|3|461| +2450822|6622|3|508| +2450822|6625|3|908| +2450822|6626|3|520| +2450822|6628|3|278| +2450822|6631|3|861| +2450822|6632|3|22| +2450822|6634|3|916| +2450822|6637|3|54| +2450822|6638|3|377| +2450822|6640|3|178| +2450822|6643|3|852| +2450822|6644|3|426| +2450822|6646|3|893| +2450822|6649|3|925| +2450822|6650|3|151| +2450822|6652|3|862| +2450822|6655|3|948| +2450822|6656|3|43| +2450822|6658|3|22| +2450822|6661|3|554| +2450822|6662|3|79| +2450822|6664|3|988| +2450822|6667|3|220| +2450822|6668|3|947| +2450822|6670|3|500| +2450822|6673|3|573| +2450822|6674|3|648| +2450822|6676|3|145| +2450822|6679|3|567| +2450822|6680|3|851| +2450822|6682|3|16| +2450822|6685|3|860| +2450822|6686|3|374| +2450822|6688|3|496| +2450822|6691|3|866| +2450822|6692|3|787| +2450822|6694|3|949| +2450822|6697|3|455| +2450822|6698|3|552| +2450822|6700|3|111| +2450822|6703|3|833| +2450822|6704|3|483| +2450822|6706|3|64| +2450822|6709|3|233| +2450822|6710|3|492| +2450822|6712|3|308| +2450822|6715|3|869| +2450822|6716|3|662| +2450822|6718|3|265| +2450822|6721|3|3| +2450822|6722|3|454| +2450822|6724|3|106| +2450822|6727|3|951| +2450822|6728|3|| +2450822|6730|3|20| +2450822|6733|3|636| +2450822|6734|3|456| +2450822|6736|3|601| +2450822|6739|3|522| +2450822|6740|3|| +2450822|6742|3|755| +2450822|6745|3|583| +2450822|6746|3|134| +2450822|6748|3|253| +2450822|6751|3|356| +2450822|6752|3|738| +2450822|6754|3|818| +2450822|6757|3|722| +2450822|6758|3|427| +2450822|6760|3|856| +2450822|6763|3|416| +2450822|6764|3|826| +2450822|6766|3|398| +2450822|6769|3|973| +2450822|6770|3|463| +2450822|6772|3|| +2450822|6775|3|580| +2450822|6776|3|481| +2450822|6778|3|963| +2450822|6781|3|489| +2450822|6782|3|720| +2450822|6784|3|428| +2450822|6787|3|508| +2450822|6788|3|212| +2450822|6790|3|157| +2450822|6793|3|556| +2450822|6794|3|980| +2450822|6796|3|374| +2450822|6799|3|991| +2450822|6800|3|713| +2450822|6802|3|813| +2450822|6805|3|679| +2450822|6806|3|214| +2450822|6808|3|354| +2450822|6811|3|45| +2450822|6812|3|249| +2450822|6814|3|486| +2450822|6817|3|358| +2450822|6818|3|122| +2450822|6820|3|3| +2450822|6823|3|639| +2450822|6824|3|927| +2450822|6826|3|483| +2450822|6829|3|400| +2450822|6830|3|14| +2450822|6832|3|385| +2450822|6835|3|624| +2450822|6836|3|825| +2450822|6838|3|179| +2450822|6841|3|321| +2450822|6842|3|848| +2450822|6844|3|675| +2450822|6847|3|348| +2450822|6848|3|| +2450822|6850|3|505| +2450822|6853|3|559| +2450822|6854|3|112| +2450822|6856|3|927| +2450822|6859|3|780| +2450822|6860|3|683| +2450822|6862|3|393| +2450822|6865|3|346| +2450822|6866|3|286| +2450822|6868|3|542| +2450822|6871|3|258| +2450822|6872|3|750| +2450822|6874|3|397| +2450822|6877|3|300| +2450822|6878|3|719| +2450822|6880|3|888| +2450822|6883|3|149| +2450822|6884|3|532| +2450822|6886|3|270| +2450822|6889|3|602| +2450822|6890|3|907| +2450822|6892|3|| +2450822|6895|3|137| +2450822|6896|3|812| +2450822|6898|3|279| +2450822|6901|3|901| +2450822|6902|3|648| +2450822|6904|3|416| +2450822|6907|3|377| +2450822|6908|3|637| +2450822|6910|3|| +2450822|6913|3|135| +2450822|6914|3|296| +2450822|6916|3|150| +2450822|6919|3|604| +2450822|6920|3|183| +2450822|6922|3|632| +2450822|6925|3|205| +2450822|6926|3|411| +2450822|6928|3|916| +2450822|6931|3|601| +2450822|6932|3|39| +2450822|6934|3|11| +2450822|6937|3|713| +2450822|6938|3|101| +2450822|6940|3|| +2450822|6943|3|547| +2450822|6944|3|268| +2450822|6946|3|580| +2450822|6949|3|664| +2450822|6950|3|110| +2450822|6952|3|685| +2450822|6955|3|657| +2450822|6956|3|363| +2450822|6958|3|315| +2450822|6961|3|883| +2450822|6962|3|835| +2450822|6964|3|675| +2450822|6967|3|9| +2450822|6968|3|8| +2450822|6970|3|113| +2450822|6973|3|282| +2450822|6974|3|415| +2450822|6976|3|| +2450822|6979|3|51| +2450822|6980|3|600| +2450822|6982|3|719| +2450822|6985|3|9| +2450822|6986|3|370| +2450822|6988|3|188| +2450822|6991|3|486| +2450822|6992|3|233| +2450822|6994|3|49| +2450822|6997|3|1000| +2450822|6998|3|662| +2450822|7000|3|644| +2450822|7003|3|7| +2450822|7004|3|922| +2450822|7006|3|| +2450822|7009|3|397| +2450822|7010|3|693| +2450822|7012|3|119| +2450822|7015|3|189| +2450822|7016|3|492| +2450822|7018|3|764| +2450822|7021|3|757| +2450822|7022|3|664| +2450822|7024|3|392| +2450822|7027|3|149| +2450822|7028|3|603| +2450822|7030|3|420| +2450822|7033|3|785| +2450822|7034|3|685| +2450822|7036|3|351| +2450822|7039|3|483| +2450822|7040|3|395| +2450822|7042|3|332| +2450822|7045|3|748| +2450822|7046|3|969| +2450822|7048|3|799| +2450822|7051|3|473| +2450822|7052|3|133| +2450822|7054|3|817| +2450822|7057|3|782| +2450822|7058|3|14| +2450822|7060|3|151| +2450822|7063|3|195| +2450822|7064|3|| +2450822|7066|3|154| +2450822|7069|3|427| +2450822|7070|3|726| +2450822|7072|3|311| +2450822|7075|3|238| +2450822|7076|3|953| +2450822|7078|3|501| +2450822|7081|3|47| +2450822|7082|3|49| +2450822|7084|3|807| +2450822|7087|3|853| +2450822|7088|3|470| +2450822|7090|3|814| +2450822|7093|3|408| +2450822|7094|3|| +2450822|7096|3|689| +2450822|7099|3|788| +2450822|7100|3|191| +2450822|7102|3|436| +2450822|7105|3|| +2450822|7106|3|636| +2450822|7108|3|960| +2450822|7111|3|745| +2450822|7112|3|111| +2450822|7114|3|133| +2450822|7117|3|607| +2450822|7118|3|9| +2450822|7120|3|527| +2450822|7123|3|303| +2450822|7124|3|861| +2450822|7126|3|988| +2450822|7129|3|927| +2450822|7130|3|384| +2450822|7132|3|450| +2450822|7135|3|204| +2450822|7136|3|377| +2450822|7138|3|233| +2450822|7141|3|343| +2450822|7142|3|186| +2450822|7144|3|655| +2450822|7147|3|913| +2450822|7148|3|600| +2450822|7150|3|825| +2450822|7153|3|535| +2450822|7154|3|863| +2450822|7156|3|327| +2450822|7159|3|| +2450822|7160|3|2| +2450822|7162|3|269| +2450822|7165|3|600| +2450822|7166|3|964| +2450822|7168|3|407| +2450822|7171|3|247| +2450822|7172|3|380| +2450822|7174|3|| +2450822|7177|3|428| +2450822|7178|3|333| +2450822|7180|3|984| +2450822|7183|3|429| +2450822|7184|3|712| +2450822|7186|3|922| +2450822|7189|3|230| +2450822|7190|3|115| +2450822|7192|3|525| +2450822|7195|3|777| +2450822|7196|3|524| +2450822|7198|3|83| +2450822|7201|3|968| +2450822|7202|3|| +2450822|7204|3|418| +2450822|7207|3|275| +2450822|7208|3|148| +2450822|7210|3|329| +2450822|7213|3|731| +2450822|7214|3|288| +2450822|7216|3|165| +2450822|7219|3|396| +2450822|7220|3|78| +2450822|7222|3|702| +2450822|7225|3|369| +2450822|7226|3|898| +2450822|7228|3|575| +2450822|7231|3|657| +2450822|7232|3|379| +2450822|7234|3|116| +2450822|7237|3|592| +2450822|7238|3|529| +2450822|7240|3|812| +2450822|7243|3|745| +2450822|7244|3|152| +2450822|7246|3|46| +2450822|7249|3|36| +2450822|7250|3|503| +2450822|7252|3|919| +2450822|7255|3|856| +2450822|7256|3|365| +2450822|7258|3|799| +2450822|7261|3|| +2450822|7262|3|359| +2450822|7264|3|764| +2450822|7267|3|684| +2450822|7268|3|626| +2450822|7270|3|750| +2450822|7273|3|327| +2450822|7274|3|862| +2450822|7276|3|101| +2450822|7279|3|530| +2450822|7280|3|257| +2450822|7282|3|224| +2450822|7285|3|160| +2450822|7286|3|14| +2450822|7288|3|182| +2450822|7291|3|331| +2450822|7292|3|507| +2450822|7294|3|| +2450822|7297|3|512| +2450822|7298|3|318| +2450822|7300|3|464| +2450822|7303|3|620| +2450822|7304|3|577| +2450822|7306|3|10| +2450822|7309|3|634| +2450822|7310|3|917| +2450822|7312|3|693| +2450822|7315|3|613| +2450822|7316|3|104| +2450822|7318|3|339| +2450822|7321|3|182| +2450822|7322|3|908| +2450822|7324|3|345| +2450822|7327|3|548| +2450822|7328|3|441| +2450822|7330|3|225| +2450822|7333|3|806| +2450822|7334|3|228| +2450822|7336|3|934| +2450822|7339|3|170| +2450822|7340|3|874| +2450822|7342|3|| +2450822|7345|3|854| +2450822|7346|3|739| +2450822|7348|3|946| +2450822|7351|3|138| +2450822|7352|3|202| +2450822|7354|3|384| +2450822|7357|3|308| +2450822|7358|3|142| +2450822|7360|3|389| +2450822|7363|3|486| +2450822|7364|3|690| +2450822|7366|3|117| +2450822|7369|3|12| +2450822|7370|3|722| +2450822|7372|3|691| +2450822|7375|3|856| +2450822|7376|3|65| +2450822|7378|3|395| +2450822|7381|3|682| +2450822|7382|3|569| +2450822|7384|3|884| +2450822|7387|3|447| +2450822|7388|3|736| +2450822|7390|3|346| +2450822|7393|3|612| +2450822|7394|3|292| +2450822|7396|3|134| +2450822|7399|3|196| +2450822|7400|3|924| +2450822|7402|3|| +2450822|7405|3|156| +2450822|7406|3|130| +2450822|7408|3|409| +2450822|7411|3|502| +2450822|7412|3|| +2450822|7414|3|953| +2450822|7417|3|866| +2450822|7418|3|182| +2450822|7420|3|113| +2450822|7423|3|984| +2450822|7424|3|| +2450822|7426|3|174| +2450822|7429|3|344| +2450822|7430|3|52| +2450822|7432|3|494| +2450822|7435|3|437| +2450822|7436|3|531| +2450822|7438|3|479| +2450822|7441|3|557| +2450822|7442|3|638| +2450822|7444|3|372| +2450822|7447|3|74| +2450822|7448|3|841| +2450822|7450|3|596| +2450822|7453|3|322| +2450822|7454|3|710| +2450822|7456|3|75| +2450822|7459|3|549| +2450822|7460|3|914| +2450822|7462|3|251| +2450822|7465|3|518| +2450822|7466|3|678| +2450822|7468|3|603| +2450822|7471|3|387| +2450822|7472|3|633| +2450822|7474|3|493| +2450822|7477|3|462| +2450822|7478|3|230| +2450822|7480|3|585| +2450822|7483|3|824| +2450822|7484|3|475| +2450822|7486|3|45| +2450822|7489|3|926| +2450822|7490|3|822| +2450822|7492|3|76| +2450822|7495|3|581| +2450822|7496|3|192| +2450822|7498|3|325| +2450822|7501|3|690| +2450822|7502|3|605| +2450822|7504|3|492| +2450822|7507|3|763| +2450822|7508|3|192| +2450822|7510|3|486| +2450822|7513|3|45| +2450822|7514|3|83| +2450822|7516|3|575| +2450822|7519|3|| +2450822|7520|3|148| +2450822|7522|3|| +2450822|7525|3|914| +2450822|7526|3|223| +2450822|7528|3|774| +2450822|7531|3|856| +2450822|7532|3|66| +2450822|7534|3|106| +2450822|7537|3|220| +2450822|7538|3|340| +2450822|7540|3|289| +2450822|7543|3|691| +2450822|7544|3|990| +2450822|7546|3|979| +2450822|7549|3|893| +2450822|7550|3|663| +2450822|7552|3|426| +2450822|7555|3|999| +2450822|7556|3|326| +2450822|7558|3|683| +2450822|7561|3|| +2450822|7562|3|586| +2450822|7564|3|435| +2450822|7567|3|569| +2450822|7568|3|81| +2450822|7570|3|314| +2450822|7573|3|241| +2450822|7574|3|| +2450822|7576|3|388| +2450822|7579|3|614| +2450822|7580|3|309| +2450822|7582|3|545| +2450822|7585|3|236| +2450822|7586|3|241| +2450822|7588|3|272| +2450822|7591|3|748| +2450822|7592|3|52| +2450822|7594|3|449| +2450822|7597|3|| +2450822|7598|3|867| +2450822|7600|3|55| +2450822|7603|3|534| +2450822|7604|3|122| +2450822|7606|3|296| +2450822|7609|3|56| +2450822|7610|3|360| +2450822|7612|3|| +2450822|7615|3|118| +2450822|7616|3|593| +2450822|7618|3|811| +2450822|7621|3|222| +2450822|7622|3|858| +2450822|7624|3|858| +2450822|7627|3|532| +2450822|7628|3|522| +2450822|7630|3|242| +2450822|7633|3|684| +2450822|7634|3|229| +2450822|7636|3|126| +2450822|7639|3|863| +2450822|7640|3|612| +2450822|7642|3|896| +2450822|7645|3|377| +2450822|7646|3|706| +2450822|7648|3|| +2450822|7651|3|924| +2450822|7652|3|237| +2450822|7654|3|722| +2450822|7657|3|56| +2450822|7658|3|971| +2450822|7660|3|| +2450822|7663|3|309| +2450822|7664|3|94| +2450822|7666|3|642| +2450822|7669|3|505| +2450822|7670|3|266| +2450822|7672|3|278| +2450822|7675|3|761| +2450822|7676|3|302| +2450822|7678|3|18| +2450822|7681|3|432| +2450822|7682|3|753| +2450822|7684|3|928| +2450822|7687|3|710| +2450822|7688|3|636| +2450822|7690|3|849| +2450822|7693|3|558| +2450822|7694|3|735| +2450822|7696|3|560| +2450822|7699|3|942| +2450822|7700|3|| +2450822|7702|3|908| +2450822|7705|3|571| +2450822|7706|3|374| +2450822|7708|3|806| +2450822|7711|3|116| +2450822|7712|3|465| +2450822|7714|3|471| +2450822|7717|3|446| +2450822|7718|3|490| +2450822|7720|3|513| +2450822|7723|3|580| +2450822|7724|3|347| +2450822|7726|3|561| +2450822|7729|3|87| +2450822|7730|3|77| +2450822|7732|3|402| +2450822|7735|3|36| +2450822|7736|3|350| +2450822|7738|3|| +2450822|7741|3|529| +2450822|7742|3|628| +2450822|7744|3|831| +2450822|7747|3|933| +2450822|7748|3|420| +2450822|7750|3|653| +2450822|7753|3|733| +2450822|7754|3|38| +2450822|7756|3|887| +2450822|7759|3|767| +2450822|7760|3|230| +2450822|7762|3|357| +2450822|7765|3|420| +2450822|7766|3|450| +2450822|7768|3|880| +2450822|7771|3|359| +2450822|7772|3|293| +2450822|7774|3|887| +2450822|7777|3|77| +2450822|7778|3|906| +2450822|7780|3|347| +2450822|7783|3|919| +2450822|7784|3|859| +2450822|7786|3|821| +2450822|7789|3|830| +2450822|7790|3|259| +2450822|7792|3|433| +2450822|7795|3|401| +2450822|7796|3|| +2450822|7798|3|854| +2450822|7801|3|596| +2450822|7802|3|413| +2450822|7804|3|659| +2450822|7807|3|369| +2450822|7808|3|949| +2450822|7810|3|522| +2450822|7813|3|649| +2450822|7814|3|849| +2450822|7816|3|375| +2450822|7819|3|682| +2450822|7820|3|679| +2450822|7822|3|349| +2450822|7825|3|475| +2450822|7826|3|32| +2450822|7828|3|889| +2450822|7831|3|765| +2450822|7832|3|343| +2450822|7834|3|93| +2450822|7837|3|490| +2450822|7838|3|505| +2450822|7840|3|173| +2450822|7843|3|573| +2450822|7844|3|403| +2450822|7846|3|134| +2450822|7849|3|489| +2450822|7850|3|660| +2450822|7852|3|932| +2450822|7855|3|95| +2450822|7856|3|537| +2450822|7858|3|644| +2450822|7861|3|902| +2450822|7862|3|230| +2450822|7864|3|996| +2450822|7867|3|813| +2450822|7868|3|183| +2450822|7870|3|979| +2450822|7873|3|805| +2450822|7874|3|802| +2450822|7876|3|275| +2450822|7879|3|940| +2450822|7880|3|460| +2450822|7882|3|216| +2450822|7885|3|494| +2450822|7886|3|174| +2450822|7888|3|104| +2450822|7891|3|428| +2450822|7892|3|943| +2450822|7894|3|| +2450822|7897|3|357| +2450822|7898|3|170| +2450822|7900|3|11| +2450822|7903|3|158| +2450822|7904|3|967| +2450822|7906|3|556| +2450822|7909|3|680| +2450822|7910|3|| +2450822|7912|3|201| +2450822|7915|3|508| +2450822|7916|3|783| +2450822|7918|3|100| +2450822|7921|3|747| +2450822|7922|3|83| +2450822|7924|3|525| +2450822|7927|3|786| +2450822|7928|3|35| +2450822|7930|3|549| +2450822|7933|3|606| +2450822|7934|3|130| +2450822|7936|3|704| +2450822|7939|3|444| +2450822|7940|3|835| +2450822|7942|3|18| +2450822|7945|3|832| +2450822|7946|3|285| +2450822|7948|3|| +2450822|7951|3|239| +2450822|7952|3|987| +2450822|7954|3|197| +2450822|7957|3|65| +2450822|7958|3|459| +2450822|7960|3|455| +2450822|7963|3|815| +2450822|7964|3|392| +2450822|7966|3|604| +2450822|7969|3|879| +2450822|7970|3|17| +2450822|7972|3|221| +2450822|7975|3|709| +2450822|7976|3|228| +2450822|7978|3|545| +2450822|7981|3|934| +2450822|7982|3|723| +2450822|7984|3|824| +2450822|7987|3|462| +2450822|7988|3|554| +2450822|7990|3|436| +2450822|7993|3|908| +2450822|7994|3|866| +2450822|7996|3|938| +2450822|7999|3|643| +2450822|8000|3|845| +2450822|8002|3|107| +2450822|8005|3|| +2450822|8006|3|257| +2450822|8008|3|302| +2450822|8011|3|368| +2450822|8012|3|218| +2450822|8014|3|765| +2450822|8017|3|885| +2450822|8018|3|568| +2450822|8020|3|91| +2450822|8023|3|91| +2450822|8024|3|281| +2450822|8026|3|81| +2450822|8029|3|642| +2450822|8030|3|702| +2450822|8032|3|212| +2450822|8035|3|943| +2450822|8036|3|44| +2450822|8038|3|547| +2450822|8041|3|406| +2450822|8042|3|224| +2450822|8044|3|757| +2450822|8047|3|279| +2450822|8048|3|| +2450822|8050|3|477| +2450822|8053|3|370| +2450822|8054|3|780| +2450822|8056|3|278| +2450822|8059|3|529| +2450822|8060|3|417| +2450822|8062|3|66| +2450822|8065|3|157| +2450822|8066|3|131| +2450822|8068|3|758| +2450822|8071|3|820| +2450822|8072|3|854| +2450822|8074|3|693| +2450822|8077|3|150| +2450822|8078|3|389| +2450822|8080|3|760| +2450822|8083|3|0| +2450822|8084|3|306| +2450822|8086|3|533| +2450822|8089|3|387| +2450822|8090|3|944| +2450822|8092|3|82| +2450822|8095|3|110| +2450822|8096|3|250| +2450822|8098|3|966| +2450822|8101|3|452| +2450822|8102|3|799| +2450822|8104|3|372| +2450822|8107|3|167| +2450822|8108|3|523| +2450822|8110|3|374| +2450822|8113|3|502| +2450822|8114|3|204| +2450822|8116|3|919| +2450822|8119|3|66| +2450822|8120|3|243| +2450822|8122|3|87| +2450822|8125|3|998| +2450822|8126|3|446| +2450822|8128|3|703| +2450822|8131|3|660| +2450822|8132|3|689| +2450822|8134|3|668| +2450822|8137|3|851| +2450822|8138|3|794| +2450822|8140|3|734| +2450822|8143|3|| +2450822|8144|3|998| +2450822|8146|3|505| +2450822|8149|3|790| +2450822|8150|3|45| +2450822|8152|3|672| +2450822|8155|3|68| +2450822|8156|3|700| +2450822|8158|3|88| +2450822|8161|3|226| +2450822|8162|3|347| +2450822|8164|3|462| +2450822|8167|3|433| +2450822|8168|3|901| +2450822|8170|3|956| +2450822|8173|3|208| +2450822|8174|3|600| +2450822|8176|3|| +2450822|8179|3|638| +2450822|8180|3|813| +2450822|8182|3|379| +2450822|8185|3|737| +2450822|8186|3|573| +2450822|8188|3|789| +2450822|8191|3|711| +2450822|8192|3|73| +2450822|8194|3|570| +2450822|8197|3|747| +2450822|8198|3|494| +2450822|8200|3|515| +2450822|8203|3|298| +2450822|8204|3|826| +2450822|8206|3|210| +2450822|8209|3|903| +2450822|8210|3|538| +2450822|8212|3|| +2450822|8215|3|817| +2450822|8216|3|893| +2450822|8218|3|290| +2450822|8221|3|746| +2450822|8222|3|693| +2450822|8224|3|49| +2450822|8227|3|704| +2450822|8228|3|170| +2450822|8230|3|186| +2450822|8233|3|487| +2450822|8234|3|606| +2450822|8236|3|689| +2450822|8239|3|908| +2450822|8240|3|366| +2450822|8242|3|685| +2450822|8245|3|989| +2450822|8246|3|394| +2450822|8248|3|| +2450822|8251|3|514| +2450822|8252|3|477| +2450822|8254|3|121| +2450822|8257|3|745| +2450822|8258|3|| +2450822|8260|3|288| +2450822|8263|3|975| +2450822|8264|3|601| +2450822|8266|3|706| +2450822|8269|3|920| +2450822|8270|3|464| +2450822|8272|3|881| +2450822|8275|3|973| +2450822|8276|3|| +2450822|8278|3|923| +2450822|8281|3|117| +2450822|8282|3|354| +2450822|8284|3|322| +2450822|8287|3|573| +2450822|8288|3|| +2450822|8290|3|749| +2450822|8293|3|226| +2450822|8294|3|752| +2450822|8296|3|486| +2450822|8299|3|260| +2450822|8300|3|63| +2450822|8302|3|345| +2450822|8305|3|490| +2450822|8306|3|676| +2450822|8308|3|49| +2450822|8311|3|932| +2450822|8312|3|130| +2450822|8314|3|557| +2450822|8317|3|240| +2450822|8318|3|511| +2450822|8320|3|830| +2450822|8323|3|940| +2450822|8324|3|541| +2450822|8326|3|18| +2450822|8329|3|793| +2450822|8330|3|614| +2450822|8332|3|241| +2450822|8335|3|68| +2450822|8336|3|556| +2450822|8338|3|430| +2450822|8341|3|316| +2450822|8342|3|32| +2450822|8344|3|892| +2450822|8347|3|593| +2450822|8348|3|23| +2450822|8350|3|364| +2450822|8353|3|194| +2450822|8354|3|535| +2450822|8356|3|208| +2450822|8359|3|44| +2450822|8360|3|988| +2450822|8362|3|758| +2450822|8365|3|825| +2450822|8366|3|954| +2450822|8368|3|123| +2450822|8371|3|665| +2450822|8372|3|68| +2450822|8374|3|45| +2450822|8377|3|125| +2450822|8378|3|| +2450822|8380|3|692| +2450822|8383|3|299| +2450822|8384|3|459| +2450822|8386|3|676| +2450822|8389|3|135| +2450822|8390|3|909| +2450822|8392|3|486| +2450822|8395|3|360| +2450822|8396|3|380| +2450822|8398|3|897| +2450822|8401|3|589| +2450822|8402|3|672| +2450822|8404|3|487| +2450822|8407|3|203| +2450822|8408|3|139| +2450822|8410|3|918| +2450822|8413|3|254| +2450822|8414|3|536| +2450822|8416|3|877| +2450822|8419|3|802| +2450822|8420|3|314| +2450822|8422|3|864| +2450822|8425|3|655| +2450822|8426|3|88| +2450822|8428|3|993| +2450822|8431|3|342| +2450822|8432|3|562| +2450822|8434|3|335| +2450822|8437|3|| +2450822|8438|3|86| +2450822|8440|3|905| +2450822|8443|3|504| +2450822|8444|3|238| +2450822|8446|3|| +2450822|8449|3|223| +2450822|8450|3|442| +2450822|8452|3|406| +2450822|8455|3|864| +2450822|8456|3|29| +2450822|8458|3|18| +2450822|8461|3|459| +2450822|8462|3|859| +2450822|8464|3|40| +2450822|8467|3|481| +2450822|8468|3|52| +2450822|8470|3|643| +2450822|8473|3|455| +2450822|8474|3|557| +2450822|8476|3|153| +2450822|8479|3|530| +2450822|8480|3|146| +2450822|8482|3|289| +2450822|8485|3|582| +2450822|8486|3|628| +2450822|8488|3|998| +2450822|8491|3|478| +2450822|8492|3|259| +2450822|8494|3|444| +2450822|8497|3|551| +2450822|8498|3|219| +2450822|8500|3|497| +2450822|8503|3|881| +2450822|8504|3|336| +2450822|8506|3|373| +2450822|8509|3|697| +2450822|8510|3|77| +2450822|8512|3|563| +2450822|8515|3|71| +2450822|8516|3|18| +2450822|8518|3|859| +2450822|8521|3|418| +2450822|8522|3|862| +2450822|8524|3|783| +2450822|8527|3|876| +2450822|8528|3|384| +2450822|8530|3|799| +2450822|8533|3|566| +2450822|8534|3|259| +2450822|8536|3|429| +2450822|8539|3|294| +2450822|8540|3|964| +2450822|8542|3|473| +2450822|8545|3|| +2450822|8546|3|942| +2450822|8548|3|575| +2450822|8551|3|861| +2450822|8552|3|33| +2450822|8554|3|85| +2450822|8557|3|213| +2450822|8558|3|382| +2450822|8560|3|384| +2450822|8563|3|700| +2450822|8564|3|550| +2450822|8566|3|978| +2450822|8569|3|| +2450822|8570|3|602| +2450822|8572|3|275| +2450822|8575|3|253| +2450822|8576|3|499| +2450822|8578|3|899| +2450822|8581|3|970| +2450822|8582|3|746| +2450822|8584|3|38| +2450822|8587|3|67| +2450822|8588|3|918| +2450822|8590|3|917| +2450822|8593|3|395| +2450822|8594|3|538| +2450822|8596|3|442| +2450822|8599|3|690| +2450822|8600|3|791| +2450822|8602|3|814| +2450822|8605|3|560| +2450822|8606|3|764| +2450822|8608|3|608| +2450822|8611|3|759| +2450822|8612|3|287| +2450822|8614|3|379| +2450822|8617|3|719| +2450822|8618|3|531| +2450822|8620|3|510| +2450822|8623|3|| +2450822|8624|3|177| +2450822|8626|3|386| +2450822|8629|3|226| +2450822|8630|3|837| +2450822|8632|3|578| +2450822|8635|3|678| +2450822|8636|3|518| +2450822|8638|3|949| +2450822|8641|3|48| +2450822|8642|3|900| +2450822|8644|3|120| +2450822|8647|3|174| +2450822|8648|3|562| +2450822|8650|3|934| +2450822|8653|3|749| +2450822|8654|3|585| +2450822|8656|3|743| +2450822|8659|3|756| +2450822|8660|3|388| +2450822|8662|3|601| +2450822|8665|3|278| +2450822|8666|3|346| +2450822|8668|3|354| +2450822|8671|3|253| +2450822|8672|3|276| +2450822|8674|3|136| +2450822|8677|3|799| +2450822|8678|3|45| +2450822|8680|3|70| +2450822|8683|3|939| +2450822|8684|3|896| +2450822|8686|3|47| +2450822|8689|3|701| +2450822|8690|3|12| +2450822|8692|3|329| +2450822|8695|3|739| +2450822|8696|3|697| +2450822|8698|3|133| +2450822|8701|3|617| +2450822|8702|3|401| +2450822|8704|3|104| +2450822|8707|3|348| +2450822|8708|3|261| +2450822|8710|3|254| +2450822|8713|3|972| +2450822|8714|3|67| +2450822|8716|3|101| +2450822|8719|3|792| +2450822|8720|3|112| +2450822|8722|3|427| +2450822|8725|3|292| +2450822|8726|3|436| +2450822|8728|3|975| +2450822|8731|3|234| +2450822|8732|3|348| +2450822|8734|3|393| +2450822|8737|3|240| +2450822|8738|3|447| +2450822|8740|3|144| +2450822|8743|3|867| +2450822|8744|3|309| +2450822|8746|3|757| +2450822|8749|3|77| +2450822|8750|3|202| +2450822|8752|3|976| +2450822|8755|3|234| +2450822|8756|3|315| +2450822|8758|3|962| +2450822|8761|3|| +2450822|8762|3|742| +2450822|8764|3|156| +2450822|8767|3|753| +2450822|8768|3|693| +2450822|8770|3|634| +2450822|8773|3|696| +2450822|8774|3|362| +2450822|8776|3|4| +2450822|8779|3|675| +2450822|8780|3|654| +2450822|8782|3|946| +2450822|8785|3|446| +2450822|8786|3|849| +2450822|8788|3|726| +2450822|8791|3|702| +2450822|8792|3|| +2450822|8794|3|954| +2450822|8797|3|157| +2450822|8798|3|887| +2450822|8800|3|751| +2450822|8803|3|550| +2450822|8804|3|882| +2450822|8806|3|| +2450822|8809|3|463| +2450822|8810|3|251| +2450822|8812|3|217| +2450822|8815|3|210| +2450822|8816|3|36| +2450822|8818|3|753| +2450822|8821|3|212| +2450822|8822|3|698| +2450822|8824|3|571| +2450822|8827|3|271| +2450822|8828|3|| +2450822|8830|3|673| +2450822|8833|3|914| +2450822|8834|3|947| +2450822|8836|3|349| +2450822|8839|3|562| +2450822|8840|3|42| +2450822|8842|3|888| +2450822|8845|3|471| +2450822|8846|3|720| +2450822|8848|3|86| +2450822|8851|3|20| +2450822|8852|3|373| +2450822|8854|3|667| +2450822|8857|3|152| +2450822|8858|3|801| +2450822|8860|3|415| +2450822|8863|3|394| +2450822|8864|3|712| +2450822|8866|3|503| +2450822|8869|3|234| +2450822|8870|3|208| +2450822|8872|3|209| +2450822|8875|3|174| +2450822|8876|3|749| +2450822|8878|3|168| +2450822|8881|3|505| +2450822|8882|3|798| +2450822|8884|3|526| +2450822|8887|3|257| +2450822|8888|3|83| +2450822|8890|3|912| +2450822|8893|3|548| +2450822|8894|3|775| +2450822|8896|3|815| +2450822|8899|3|266| +2450822|8900|3|845| +2450822|8902|3|163| +2450822|8905|3|619| +2450822|8906|3|347| +2450822|8908|3|395| +2450822|8911|3|167| +2450822|8912|3|750| +2450822|8914|3|36| +2450822|8917|3|197| +2450822|8918|3|730| +2450822|8920|3|68| +2450822|8923|3|733| +2450822|8924|3|803| +2450822|8926|3|809| +2450822|8929|3|932| +2450822|8930|3|838| +2450822|8932|3|949| +2450822|8935|3|293| +2450822|8936|3|709| +2450822|8938|3|816| +2450822|8941|3|279| +2450822|8942|3|879| +2450822|8944|3|602| +2450822|8947|3|574| +2450822|8948|3|355| +2450822|8950|3|571| +2450822|8953|3|200| +2450822|8954|3|764| +2450822|8956|3|225| +2450822|8959|3|359| +2450822|8960|3|232| +2450822|8962|3|902| +2450822|8965|3|911| +2450822|8966|3|985| +2450822|8968|3|475| +2450822|8971|3|136| +2450822|8972|3|148| +2450822|8974|3|701| +2450822|8977|3|396| +2450822|8978|3|652| +2450822|8980|3|924| +2450822|8983|3|365| +2450822|8984|3|797| +2450822|8986|3|851| +2450822|8989|3|292| +2450822|8990|3|499| +2450822|8992|3|440| +2450822|8995|3|675| +2450822|8996|3|611| +2450822|8998|3|454| +2450822|9001|3|286| +2450822|9002|3|696| +2450822|9004|3|463| +2450822|9007|3|938| +2450822|9008|3|270| +2450822|9010|3|568| +2450822|9013|3|14| +2450822|9014|3|104| +2450822|9016|3|816| +2450822|9019|3|411| +2450822|9020|3|747| +2450822|9022|3|751| +2450822|9025|3|932| +2450822|9026|3|403| +2450822|9028|3|961| +2450822|9031|3|776| +2450822|9032|3|| +2450822|9034|3|408| +2450822|9037|3|353| +2450822|9038|3|313| +2450822|9040|3|136| +2450822|9043|3|319| +2450822|9044|3|601| +2450822|9046|3|416| +2450822|9049|3|663| +2450822|9050|3|74| +2450822|9052|3|790| +2450822|9055|3|587| +2450822|9056|3|710| +2450822|9058|3|477| +2450822|9061|3|837| +2450822|9062|3|420| +2450822|9064|3|732| +2450822|9067|3|409| +2450822|9068|3|| +2450822|9070|3|2| +2450822|9073|3|271| +2450822|9074|3|980| +2450822|9076|3|437| +2450822|9079|3|817| +2450822|9080|3|912| +2450822|9082|3|289| +2450822|9085|3|526| +2450822|9086|3|400| +2450822|9088|3|579| +2450822|9091|3|109| +2450822|9092|3|665| +2450822|9094|3|560| +2450822|9097|3|568| +2450822|9098|3|829| +2450822|9100|3|620| +2450822|9103|3|422| +2450822|9104|3|136| +2450822|9106|3|520| +2450822|9109|3|22| +2450822|9110|3|884| +2450822|9112|3|677| +2450822|9115|3|693| +2450822|9116|3|812| +2450822|9118|3|996| +2450822|9121|3|820| +2450822|9122|3|271| +2450822|9124|3|525| +2450822|9127|3|259| +2450822|9128|3|| +2450822|9130|3|308| +2450822|9133|3|858| +2450822|9134|3|609| +2450822|9136|3|223| +2450822|9139|3|815| +2450822|9140|3|554| +2450822|9142|3|288| +2450822|9145|3|129| +2450822|9146|3|| +2450822|9148|3|331| +2450822|9151|3|662| +2450822|9152|3|182| +2450822|9154|3|983| +2450822|9157|3|670| +2450822|9158|3|629| +2450822|9160|3|307| +2450822|9163|3|850| +2450822|9164|3|817| +2450822|9166|3|671| +2450822|9169|3|855| +2450822|9170|3|| +2450822|9172|3|434| +2450822|9175|3|625| +2450822|9176|3|280| +2450822|9178|3|| +2450822|9181|3|193| +2450822|9182|3|633| +2450822|9184|3|73| +2450822|9187|3|637| +2450822|9188|3|129| +2450822|9190|3|43| +2450822|9193|3|601| +2450822|9194|3|719| +2450822|9196|3|240| +2450822|9199|3|646| +2450822|9200|3|824| +2450822|9202|3|436| +2450822|9205|3|157| +2450822|9206|3|684| +2450822|9208|3|668| +2450822|9211|3|98| +2450822|9212|3|95| +2450822|9214|3|| +2450822|9217|3|962| +2450822|9218|3|38| +2450822|9220|3|844| +2450822|9223|3|516| +2450822|9224|3|117| +2450822|9226|3|890| +2450822|9229|3|781| +2450822|9230|3|830| +2450822|9232|3|708| +2450822|9235|3|565| +2450822|9236|3|538| +2450822|9238|3|745| +2450822|9241|3|465| +2450822|9242|3|373| +2450822|9244|3|631| +2450822|9247|3|743| +2450822|9248|3|893| +2450822|9250|3|258| +2450822|9253|3|720| +2450822|9254|3|612| +2450822|9256|3|816| +2450822|9259|3|303| +2450822|9260|3|142| +2450822|9262|3|55| +2450822|9265|3|998| +2450822|9266|3|721| +2450822|9268|3|742| +2450822|9271|3|307| +2450822|9272|3|628| +2450822|9274|3|121| +2450822|9277|3|355| +2450822|9278|3|835| +2450822|9280|3|| +2450822|9283|3|966| +2450822|9284|3|180| +2450822|9286|3|446| +2450822|9289|3|310| +2450822|9290|3|680| +2450822|9292|3|615| +2450822|9295|3|712| +2450822|9296|3|142| +2450822|9298|3|371| +2450822|9301|3|877| +2450822|9302|3|| +2450822|9304|3|485| +2450822|9307|3|48| +2450822|9308|3|773| +2450822|9310|3|339| +2450822|9313|3|797| +2450822|9314|3|417| +2450822|9316|3|400| +2450822|9319|3|982| +2450822|9320|3|62| +2450822|9322|3|559| +2450822|9325|3|277| +2450822|9326|3|254| +2450822|9328|3|449| +2450822|9331|3|991| +2450822|9332|3|311| +2450822|9334|3|225| +2450822|9337|3|253| +2450822|9338|3|163| +2450822|9340|3|504| +2450822|9343|3|583| +2450822|9344|3|254| +2450822|9346|3|634| +2450822|9349|3|503| +2450822|9350|3|905| +2450822|9352|3|461| +2450822|9355|3|955| +2450822|9356|3|492| +2450822|9358|3|132| +2450822|9361|3|139| +2450822|9362|3|378| +2450822|9364|3|956| +2450822|9367|3|997| +2450822|9368|3|807| +2450822|9370|3|538| +2450822|9373|3|| +2450822|9374|3|987| +2450822|9376|3|842| +2450822|9379|3|632| +2450822|9380|3|135| +2450822|9382|3|77| +2450822|9385|3|993| +2450822|9386|3|310| +2450822|9388|3|905| +2450822|9391|3|654| +2450822|9392|3|820| +2450822|9394|3|381| +2450822|9397|3|505| +2450822|9398|3|647| +2450822|9400|3|73| +2450822|9403|3|707| +2450822|9404|3|85| +2450822|9406|3|615| +2450822|9409|3|148| +2450822|9410|3|324| +2450822|9412|3|206| +2450822|9415|3|982| +2450822|9416|3|299| +2450822|9418|3|294| +2450822|9421|3|742| +2450822|9422|3|614| +2450822|9424|3|617| +2450822|9427|3|106| +2450822|9428|3|601| +2450822|9430|3|7| +2450822|9433|3|664| +2450822|9434|3|554| +2450822|9436|3|385| +2450822|9439|3|87| +2450822|9440|3|950| +2450822|9442|3|531| +2450822|9445|3|803| +2450822|9446|3|663| +2450822|9448|3|263| +2450822|9451|3|786| +2450822|9452|3|700| +2450822|9454|3|946| +2450822|9457|3|876| +2450822|9458|3|226| +2450822|9460|3|570| +2450822|9463|3|345| +2450822|9464|3|252| +2450822|9466|3|596| +2450822|9469|3|651| +2450822|9470|3|263| +2450822|9472|3|496| +2450822|9475|3|917| +2450822|9476|3|639| +2450822|9478|3|419| +2450822|9481|3|10| +2450822|9482|3|273| +2450822|9484|3|693| +2450822|9487|3|896| +2450822|9488|3|231| +2450822|9490|3|356| +2450822|9493|3|202| +2450822|9494|3|535| +2450822|9496|3|168| +2450822|9499|3|883| +2450822|9500|3|310| +2450822|9502|3|408| +2450822|9505|3|886| +2450822|9506|3|672| +2450822|9508|3|| +2450822|9511|3|216| +2450822|9512|3|106| +2450822|9514|3|693| +2450822|9517|3|929| +2450822|9518|3|598| +2450822|9520|3|425| +2450822|9523|3|915| +2450822|9524|3|94| +2450822|9526|3|697| +2450822|9529|3|573| +2450822|9530|3|939| +2450822|9532|3|31| +2450822|9535|3|804| +2450822|9536|3|803| +2450822|9538|3|13| +2450822|9541|3|615| +2450822|9542|3|376| +2450822|9544|3|35| +2450822|9547|3|48| +2450822|9548|3|511| +2450822|9550|3|659| +2450822|9553|3|593| +2450822|9554|3|385| +2450822|9556|3|443| +2450822|9559|3|366| +2450822|9560|3|139| +2450822|9562|3|176| +2450822|9565|3|551| +2450822|9566|3|752| +2450822|9568|3|126| +2450822|9571|3|83| +2450822|9572|3|| +2450822|9574|3|511| +2450822|9577|3|695| +2450822|9578|3|604| +2450822|9580|3|932| +2450822|9583|3|949| +2450822|9584|3|798| +2450822|9586|3|940| +2450822|9589|3|266| +2450822|9590|3|177| +2450822|9592|3|305| +2450822|9595|3|891| +2450822|9596|3|| +2450822|9598|3|455| +2450822|9601|3|956| +2450822|9602|3|548| +2450822|9604|3|310| +2450822|9607|3|357| +2450822|9608|3|532| +2450822|9610|3|501| +2450822|9613|3|637| +2450822|9614|3|495| +2450822|9616|3|113| +2450822|9619|3|| +2450822|9620|3|| +2450822|9622|3|236| +2450822|9625|3|54| +2450822|9626|3|712| +2450822|9628|3|127| +2450822|9631|3|786| +2450822|9632|3|27| +2450822|9634|3|101| +2450822|9637|3|124| +2450822|9638|3|37| +2450822|9640|3|565| +2450822|9643|3|746| +2450822|9644|3|153| +2450822|9646|3|530| +2450822|9649|3|860| +2450822|9650|3|209| +2450822|9652|3|616| +2450822|9655|3|318| +2450822|9656|3|733| +2450822|9658|3|908| +2450822|9661|3|576| +2450822|9662|3|713| +2450822|9664|3|418| +2450822|9667|3|297| +2450822|9668|3|587| +2450822|9670|3|387| +2450822|9673|3|687| +2450822|9674|3|701| +2450822|9676|3|871| +2450822|9679|3|331| +2450822|9680|3|317| +2450822|9682|3|297| +2450822|9685|3|255| +2450822|9686|3|940| +2450822|9688|3|| +2450822|9691|3|605| +2450822|9692|3|129| +2450822|9694|3|260| +2450822|9697|3|215| +2450822|9698|3|206| +2450822|9700|3|725| +2450822|9703|3|162| +2450822|9704|3|957| +2450822|9706|3|734| +2450822|9709|3|177| +2450822|9710|3|243| +2450822|9712|3|514| +2450822|9715|3|611| +2450822|9716|3|356| +2450822|9718|3|34| +2450822|9721|3|623| +2450822|9722|3|| +2450822|9724|3|749| +2450822|9727|3|26| +2450822|9728|3|968| +2450822|9730|3|225| +2450822|9733|3|932| +2450822|9734|3|511| +2450822|9736|3|749| +2450822|9739|3|476| +2450822|9740|3|666| +2450822|9742|3|490| +2450822|9745|3|483| +2450822|9746|3|559| +2450822|9748|3|994| +2450822|9751|3|915| +2450822|9752|3|270| +2450822|9754|3|805| +2450822|9757|3|625| +2450822|9758|3|847| +2450822|9760|3|51| +2450822|9763|3|826| +2450822|9764|3|868| +2450822|9766|3|283| +2450822|9769|3|| +2450822|9770|3|152| +2450822|9772|3|100| +2450822|9775|3|| +2450822|9776|3|44| +2450822|9778|3|10| +2450822|9781|3|765| +2450822|9782|3|673| +2450822|9784|3|349| +2450822|9787|3|| +2450822|9788|3|862| +2450822|9790|3|74| +2450822|9793|3|394| +2450822|9794|3|486| +2450822|9796|3|193| +2450822|9799|3|824| +2450822|9800|3|215| +2450822|9802|3|692| +2450822|9805|3|969| +2450822|9806|3|778| +2450822|9808|3|420| +2450822|9811|3|595| +2450822|9812|3|12| +2450822|9814|3|831| +2450822|9817|3|602| +2450822|9818|3|381| +2450822|9820|3|308| +2450822|9823|3|163| +2450822|9824|3|319| +2450822|9826|3|613| +2450822|9829|3|911| +2450822|9830|3|162| +2450822|9832|3|26| +2450822|9835|3|850| +2450822|9836|3|34| +2450822|9838|3|736| +2450822|9841|3|114| +2450822|9842|3|65| +2450822|9844|3|45| +2450822|9847|3|121| +2450822|9848|3|889| +2450822|9850|3|854| +2450822|9853|3|99| +2450822|9854|3|193| +2450822|9856|3|497| +2450822|9859|3|114| +2450822|9860|3|885| +2450822|9862|3|620| +2450822|9865|3|902| +2450822|9866|3|20| +2450822|9868|3|809| +2450822|9871|3|189| +2450822|9872|3|58| +2450822|9874|3|607| +2450822|9877|3|212| +2450822|9878|3|| +2450822|9880|3|439| +2450822|9883|3|591| +2450822|9884|3|663| +2450822|9886|3|517| +2450822|9889|3|862| +2450822|9890|3|400| +2450822|9892|3|387| +2450822|9895|3|32| +2450822|9896|3|704| +2450822|9898|3|396| +2450822|9901|3|| +2450822|9902|3|309| +2450822|9904|3|294| +2450822|9907|3|994| +2450822|9908|3|408| +2450822|9910|3|705| +2450822|9913|3|795| +2450822|9914|3|608| +2450822|9916|3|750| +2450822|9919|3|955| +2450822|9920|3|854| +2450822|9922|3|8| +2450822|9925|3|661| +2450822|9926|3|474| +2450822|9928|3|| +2450822|9931|3|| +2450822|9932|3|308| +2450822|9934|3|79| +2450822|9937|3|720| +2450822|9938|3|740| +2450822|9940|3|| +2450822|9943|3|| +2450822|9944|3|917| +2450822|9946|3|| +2450822|9949|3|239| +2450822|9950|3|900| +2450822|9952|3|722| +2450822|9955|3|19| +2450822|9956|3|901| +2450822|9958|3|164| +2450822|9961|3|| +2450822|9962|3|918| +2450822|9964|3|459| +2450822|9967|3|509| +2450822|9968|3|610| +2450822|9970|3|| +2450822|9973|3|| +2450822|9974|3|844| +2450822|9976|3|392| +2450822|9979|3|560| +2450822|9980|3|336| +2450822|9982|3|639| +2450822|9985|3|476| +2450822|9986|3|57| +2450822|9988|3|359| +2450822|9991|3|427| +2450822|9992|3|832| +2450822|9994|3|411| +2450822|9997|3|695| +2450822|9998|3|881| +2450822|10000|3|639| +2450822|10003|3|354| +2450822|10004|3|310| +2450822|10006|3|637| +2450822|10009|3|722| +2450822|10010|3|666| +2450822|10012|3|56| +2450822|10015|3|31| +2450822|10016|3|| +2450822|10018|3|463| +2450822|10021|3|487| +2450822|10022|3|445| +2450822|10024|3|537| +2450822|10027|3|692| +2450822|10028|3|| +2450822|10030|3|991| +2450822|10033|3|383| +2450822|10034|3|362| +2450822|10036|3|898| +2450822|10039|3|440| +2450822|10040|3|544| +2450822|10042|3|279| +2450822|10045|3|135| +2450822|10046|3|| +2450822|10048|3|332| +2450822|10051|3|693| +2450822|10052|3|657| +2450822|10054|3|| +2450822|10057|3|141| +2450822|10058|3|457| +2450822|10060|3|992| +2450822|10063|3|270| +2450822|10064|3|353| +2450822|10066|3|252| +2450822|10069|3|220| +2450822|10070|3|85| +2450822|10072|3|242| +2450822|10075|3|303| +2450822|10076|3|366| +2450822|10078|3|78| +2450822|10081|3|641| +2450822|10082|3|935| +2450822|10084|3|187| +2450822|10087|3|| +2450822|10088|3|152| +2450822|10090|3|931| +2450822|10093|3|775| +2450822|10094|3|| +2450822|10096|3|487| +2450822|10099|3|932| +2450822|10100|3|251| +2450822|10102|3|247| +2450822|10105|3|194| +2450822|10106|3|773| +2450822|10108|3|416| +2450822|10111|3|174| +2450822|10112|3|452| +2450822|10114|3|197| +2450822|10117|3|863| +2450822|10118|3|72| +2450822|10120|3|68| +2450822|10123|3|849| +2450822|10124|3|299| +2450822|10126|3|764| +2450822|10129|3|981| +2450822|10130|3|694| +2450822|10132|3|224| +2450822|10135|3|| +2450822|10136|3|704| +2450822|10138|3|615| +2450822|10141|3|516| +2450822|10142|3|497| +2450822|10144|3|| +2450822|10147|3|308| +2450822|10148|3|445| +2450822|10150|3|| +2450822|10153|3|486| +2450822|10154|3|478| +2450822|10156|3|312| +2450822|10159|3|62| +2450822|10160|3|565| +2450822|10162|3|710| +2450822|10165|3|333| +2450822|10166|3|353| +2450822|10168|3|| +2450822|10171|3|433| +2450822|10172|3|21| +2450822|10174|3|429| +2450822|10177|3|619| +2450822|10178|3|187| +2450822|10180|3|151| +2450822|10183|3|463| +2450822|10184|3|949| +2450822|10186|3|312| +2450822|10189|3|719| +2450822|10190|3|492| +2450822|10192|3|700| +2450822|10195|3|394| +2450822|10196|3|890| +2450822|10198|3|63| +2450822|10201|3|208| +2450822|10202|3|135| +2450822|10204|3|770| +2450822|10207|3|965| +2450822|10208|3|829| +2450822|10210|3|902| +2450822|10213|3|75| +2450822|10214|3|396| +2450822|10216|3|712| +2450822|10219|3|199| +2450822|10220|3|324| +2450822|10222|3|148| +2450822|10225|3|353| +2450822|10226|3|728| +2450822|10228|3|| +2450822|10231|3|985| +2450822|10232|3|373| +2450822|10234|3|910| +2450822|10237|3|648| +2450822|10238|3|307| +2450822|10240|3|971| +2450822|10243|3|539| +2450822|10244|3|328| +2450822|10246|3|315| +2450822|10249|3|650| +2450822|10250|3|278| +2450822|10252|3|369| +2450822|10255|3|392| +2450822|10256|3|180| +2450822|10258|3|7| +2450822|10261|3|649| +2450822|10262|3|379| +2450822|10264|3|246| +2450822|10267|3|68| +2450822|10268|3|473| +2450822|10270|3|319| +2450822|10273|3|889| +2450822|10274|3|553| +2450822|10276|3|210| +2450822|10279|3|611| +2450822|10280|3|133| +2450822|10282|3|310| +2450822|10285|3|923| +2450822|10286|3|220| +2450822|10288|3|146| +2450822|10291|3|61| +2450822|10292|3|166| +2450822|10294|3|924| +2450822|10297|3|778| +2450822|10298|3|438| +2450822|10300|3|388| +2450822|10303|3|608| +2450822|10304|3|213| +2450822|10306|3|647| +2450822|10309|3|884| +2450822|10310|3|| +2450822|10312|3|593| +2450822|10315|3|528| +2450822|10316|3|473| +2450822|10318|3|192| +2450822|10321|3|581| +2450822|10322|3|125| +2450822|10324|3|90| +2450822|10327|3|156| +2450822|10328|3|368| +2450822|10330|3|798| +2450822|10333|3|362| +2450822|10334|3|375| +2450822|10336|3|867| +2450822|10339|3|111| +2450822|10340|3|134| +2450822|10342|3|260| +2450822|10345|3|975| +2450822|10346|3|706| +2450822|10348|3|550| +2450822|10351|3|958| +2450822|10352|3|564| +2450822|10354|3|950| +2450822|10357|3|202| +2450822|10358|3|380| +2450822|10360|3|904| +2450822|10363|3|93| +2450822|10364|3|329| +2450822|10366|3|418| +2450822|10369|3|71| +2450822|10370|3|159| +2450822|10372|3|341| +2450822|10375|3|975| +2450822|10376|3|466| +2450822|10378|3|228| +2450822|10381|3|600| +2450822|10382|3|830| +2450822|10384|3|660| +2450822|10387|3|925| +2450822|10388|3|65| +2450822|10390|3|309| +2450822|10393|3|389| +2450822|10394|3|281| +2450822|10396|3|315| +2450822|10399|3|577| +2450822|10400|3|322| +2450822|10402|3|323| +2450822|10405|3|655| +2450822|10406|3|630| +2450822|10408|3|925| +2450822|10411|3|225| +2450822|10412|3|| +2450822|10414|3|614| +2450822|10417|3|413| +2450822|10418|3|812| +2450822|10420|3|| +2450822|10423|3|394| +2450822|10424|3|9| +2450822|10426|3|315| +2450822|10429|3|378| +2450822|10430|3|39| +2450822|10432|3|43| +2450822|10435|3|487| +2450822|10436|3|75| +2450822|10438|3|853| +2450822|10441|3|776| +2450822|10442|3|84| +2450822|10444|3|339| +2450822|10447|3|760| +2450822|10448|3|| +2450822|10450|3|858| +2450822|10453|3|414| +2450822|10454|3|922| +2450822|10456|3|832| +2450822|10459|3|86| +2450822|10460|3|135| +2450822|10462|3|213| +2450822|10465|3|233| +2450822|10466|3|699| +2450822|10468|3|781| +2450822|10471|3|681| +2450822|10472|3|399| +2450822|10474|3|209| +2450822|10477|3|378| +2450822|10478|3|527| +2450822|10480|3|619| +2450822|10483|3|94| +2450822|10484|3|521| +2450822|10486|3|114| +2450822|10489|3|939| +2450822|10490|3|724| +2450822|10492|3|97| +2450822|10495|3|692| +2450822|10496|3|670| +2450822|10498|3|| +2450822|10501|3|617| +2450822|10502|3|67| +2450822|10504|3|465| +2450822|10507|3|636| +2450822|10508|3|23| +2450822|10510|3|183| +2450822|10513|3|593| +2450822|10514|3|517| +2450822|10516|3|76| +2450822|10519|3|434| +2450822|10520|3|226| +2450822|10522|3|12| +2450822|10525|3|173| +2450822|10526|3|697| +2450822|10528|3|3| +2450822|10531|3|573| +2450822|10532|3|806| +2450822|10534|3|956| +2450822|10537|3|984| +2450822|10538|3|| +2450822|10540|3|725| +2450822|10543|3|30| +2450822|10544|3|756| +2450822|10546|3|796| +2450822|10549|3|237| +2450822|10550|3|749| +2450822|10552|3|789| +2450822|10555|3|966| +2450822|10556|3|126| +2450822|10558|3|518| +2450822|10561|3|47| +2450822|10562|3|485| +2450822|10564|3|745| +2450822|10567|3|550| +2450822|10568|3|286| +2450822|10570|3|108| +2450822|10573|3|269| +2450822|10574|3|96| +2450822|10576|3|567| +2450822|10579|3|| +2450822|10580|3|923| +2450822|10582|3|447| +2450822|10585|3|472| +2450822|10586|3|948| +2450822|10588|3|937| +2450822|10591|3|412| +2450822|10592|3|121| +2450822|10594|3|786| +2450822|10597|3|817| +2450822|10598|3|618| +2450822|10600|3|| +2450822|10603|3|382| +2450822|10604|3|786| +2450822|10606|3|33| +2450822|10609|3|984| +2450822|10610|3|161| +2450822|10612|3|344| +2450822|10615|3|656| +2450822|10616|3|662| +2450822|10618|3|762| +2450822|10621|3|95| +2450822|10622|3|17| +2450822|10624|3|483| +2450822|10627|3|300| +2450822|10628|3|827| +2450822|10630|3|342| +2450822|10633|3|220| +2450822|10634|3|903| +2450822|10636|3|971| +2450822|10639|3|865| +2450822|10640|3|339| +2450822|10642|3|250| +2450822|10645|3|633| +2450822|10646|3|502| +2450822|10648|3|270| +2450822|10651|3|| +2450822|10652|3|737| +2450822|10654|3|224| +2450822|10657|3|900| +2450822|10658|3|571| +2450822|10660|3|220| +2450822|10663|3|621| +2450822|10664|3|107| +2450822|10666|3|302| +2450822|10669|3|930| +2450822|10670|3|948| +2450822|10672|3|193| +2450822|10675|3|210| +2450822|10676|3|206| +2450822|10678|3|963| +2450822|10681|3|963| +2450822|10682|3|768| +2450822|10684|3|195| +2450822|10687|3|167| +2450822|10688|3|576| +2450822|10690|3|99| +2450822|10693|3|462| +2450822|10694|3|653| +2450822|10696|3|175| +2450822|10699|3|52| +2450822|10700|3|99| +2450822|10702|3|511| +2450822|10705|3|422| +2450822|10706|3|592| +2450822|10708|3|808| +2450822|10711|3|862| +2450822|10712|3|784| +2450822|10714|3|659| +2450822|10717|3|849| +2450822|10718|3|| +2450822|10720|3|861| +2450822|10723|3|869| +2450822|10724|3|252| +2450822|10726|3|339| +2450822|10729|3|369| +2450822|10730|3|558| +2450822|10732|3|999| +2450822|10735|3|731| +2450822|10736|3|917| +2450822|10738|3|220| +2450822|10741|3|887| +2450822|10742|3|414| +2450822|10744|3|179| +2450822|10747|3|339| +2450822|10748|3|133| +2450822|10750|3|20| +2450822|10753|3|86| +2450822|10754|3|121| +2450822|10756|3|| +2450822|10759|3|248| +2450822|10760|3|327| +2450822|10762|3|128| +2450822|10765|3|176| +2450822|10766|3|413| +2450822|10768|3|| +2450822|10771|3|208| +2450822|10772|3|938| +2450822|10774|3|957| +2450822|10777|3|246| +2450822|10778|3|577| +2450822|10780|3|272| +2450822|10783|3|976| +2450822|10784|3|841| +2450822|10786|3|936| +2450822|10789|3|766| +2450822|10790|3|319| +2450822|10792|3|624| +2450822|10795|3|836| +2450822|10796|3|622| +2450822|10798|3|852| +2450822|10801|3|958| +2450822|10802|3|974| +2450822|10804|3|728| +2450822|10807|3|672| +2450822|10808|3|995| +2450822|10810|3|890| +2450822|10813|3|781| +2450822|10814|3|123| +2450822|10816|3|359| +2450822|10819|3|635| +2450822|10820|3|101| +2450822|10822|3|572| +2450822|10825|3|941| +2450822|10826|3|554| +2450822|10828|3|| +2450822|10831|3|139| +2450822|10832|3|228| +2450822|10834|3|70| +2450822|10837|3|784| +2450822|10838|3|246| +2450822|10840|3|781| +2450822|10843|3|320| +2450822|10844|3|523| +2450822|10846|3|953| +2450822|10849|3|518| +2450822|10850|3|800| +2450822|10852|3|955| +2450822|10855|3|60| +2450822|10856|3|431| +2450822|10858|3|| +2450822|10861|3|586| +2450822|10862|3|787| +2450822|10864|3|267| +2450822|10867|3|247| +2450822|10868|3|764| +2450822|10870|3|490| +2450822|10873|3|218| +2450822|10874|3|506| +2450822|10876|3|522| +2450822|10879|3|40| +2450822|10880|3|23| +2450822|10882|3|804| +2450822|10885|3|513| +2450822|10886|3|210| +2450822|10888|3|460| +2450822|10891|3|165| +2450822|10892|3|308| +2450822|10894|3|489| +2450822|10897|3|580| +2450822|10898|3|903| +2450822|10900|3|622| +2450822|10903|3|237| +2450822|10904|3|930| +2450822|10906|3|873| +2450822|10909|3|72| +2450822|10910|3|850| +2450822|10912|3|542| +2450822|10915|3|| +2450822|10916|3|821| +2450822|10918|3|289| +2450822|10921|3|492| +2450822|10922|3|456| +2450822|10924|3|716| +2450822|10927|3|221| +2450822|10928|3|693| +2450822|10930|3|39| +2450822|10933|3|392| +2450822|10934|3|640| +2450822|10936|3|161| +2450822|10939|3|823| +2450822|10940|3|| +2450822|10942|3|760| +2450822|10945|3|552| +2450822|10946|3|15| +2450822|10948|3|101| +2450822|10951|3|774| +2450822|10952|3|330| +2450822|10954|3|137| +2450822|10957|3|581| +2450822|10958|3|582| +2450822|10960|3|148| +2450822|10963|3|359| +2450822|10964|3|614| +2450822|10966|3|127| +2450822|10969|3|159| +2450822|10970|3|204| +2450822|10972|3|79| +2450822|10975|3|867| +2450822|10976|3|398| +2450822|10978|3|453| +2450822|10981|3|532| +2450822|10982|3|646| +2450822|10984|3|247| +2450822|10987|3|17| +2450822|10988|3|534| +2450822|10990|3|934| +2450822|10993|3|707| +2450822|10994|3|990| +2450822|10996|3|394| +2450822|10999|3|236| +2450822|11000|3|327| +2450822|11002|3|765| +2450822|11005|3|677| +2450822|11006|3|766| +2450822|11008|3|741| +2450822|11011|3|669| +2450822|11012|3|701| +2450822|11014|3|450| +2450822|11017|3|260| +2450822|11018|3|978| +2450822|11020|3|626| +2450822|11023|3|414| +2450822|11024|3|735| +2450822|11026|3|182| +2450822|11029|3|785| +2450822|11030|3|997| +2450822|11032|3|188| +2450822|11035|3|679| +2450822|11036|3|99| +2450822|11038|3|461| +2450822|11041|3|187| +2450822|11042|3|848| +2450822|11044|3|141| +2450822|11047|3|187| +2450822|11048|3|707| +2450822|11050|3|105| +2450822|11053|3|607| +2450822|11054|3|408| +2450822|11056|3|640| +2450822|11059|3|| +2450822|11060|3|234| +2450822|11062|3|612| +2450822|11065|3|934| +2450822|11066|3|| +2450822|11068|3|932| +2450822|11071|3|806| +2450822|11072|3|140| +2450822|11074|3|817| +2450822|11077|3|311| +2450822|11078|3|670| +2450822|11080|3|858| +2450822|11083|3|292| +2450822|11084|3|691| +2450822|11086|3|243| +2450822|11089|3|891| +2450822|11090|3|287| +2450822|11092|3|678| +2450822|11095|3|991| +2450822|11096|3|0| +2450822|11098|3|657| +2450822|11101|3|721| +2450822|11102|3|320| +2450822|11104|3|723| +2450822|11107|3|373| +2450822|11108|3|| +2450822|11110|3|761| +2450822|11113|3|425| +2450822|11114|3|904| +2450822|11116|3|| +2450822|11119|3|779| +2450822|11120|3|883| +2450822|11122|3|445| +2450822|11125|3|67| +2450822|11126|3|485| +2450822|11128|3|245| +2450822|11131|3|181| +2450822|11132|3|| +2450822|11134|3|527| +2450822|11137|3|37| +2450822|11138|3|871| +2450822|11140|3|177| +2450822|11143|3|787| +2450822|11144|3|926| +2450822|11146|3|811| +2450822|11149|3|133| +2450822|11150|3|440| +2450822|11152|3|26| +2450822|11155|3|179| +2450822|11156|3|817| +2450822|11158|3|969| +2450822|11161|3|269| +2450822|11162|3|192| +2450822|11164|3|78| +2450822|11167|3|408| +2450822|11168|3|581| +2450822|11170|3|223| +2450822|11173|3|485| +2450822|11174|3|489| +2450822|11176|3|76| +2450822|11179|3|404| +2450822|11180|3|959| +2450822|11182|3|897| +2450822|11185|3|599| +2450822|11186|3|860| +2450822|11188|3|387| +2450822|11191|3|136| +2450822|11192|3|229| +2450822|11194|3|323| +2450822|11197|3|769| +2450822|11198|3|91| +2450822|11200|3|59| +2450822|11203|3|114| +2450822|11204|3|786| +2450822|11206|3|701| +2450822|11209|3|549| +2450822|11210|3|436| +2450822|11212|3|439| +2450822|11215|3|696| +2450822|11216|3|598| +2450822|11218|3|92| +2450822|11221|3|575| +2450822|11222|3|253| +2450822|11224|3|353| +2450822|11227|3|962| +2450822|11228|3|774| +2450822|11230|3|| +2450822|11233|3|9| +2450822|11234|3|| +2450822|11236|3|580| +2450822|11239|3|449| +2450822|11240|3|779| +2450822|11242|3|460| +2450822|11245|3|348| +2450822|11246|3|393| +2450822|11248|3|| +2450822|11251|3|371| +2450822|11252|3|884| +2450822|11254|3|798| +2450822|11257|3|135| +2450822|11258|3|163| +2450822|11260|3|291| +2450822|11263|3|451| +2450822|11264|3|322| +2450822|11266|3|33| +2450822|11269|3|297| +2450822|11270|3|343| +2450822|11272|3|| +2450822|11275|3|686| +2450822|11276|3|151| +2450822|11278|3|707| +2450822|11281|3|| +2450822|11282|3|211| +2450822|11284|3|972| +2450822|11287|3|976| +2450822|11288|3|613| +2450822|11290|3|55| +2450822|11293|3|255| +2450822|11294|3|329| +2450822|11296|3|553| +2450822|11299|3|944| +2450822|11300|3|873| +2450822|11302|3|458| +2450822|11305|3|607| +2450822|11306|3|| +2450822|11308|3|371| +2450822|11311|3|383| +2450822|11312|3|810| +2450822|11314|3|579| +2450822|11317|3|140| +2450822|11318|3|873| +2450822|11320|3|309| +2450822|11323|3|34| +2450822|11324|3|683| +2450822|11326|3|879| +2450822|11329|3|712| +2450822|11330|3|685| +2450822|11332|3|505| +2450822|11335|3|152| +2450822|11336|3|362| +2450822|11338|3|326| +2450822|11341|3|150| +2450822|11342|3|108| +2450822|11344|3|17| +2450822|11347|3|671| +2450822|11348|3|949| +2450822|11350|3|274| +2450822|11353|3|344| +2450822|11354|3|629| +2450822|11356|3|410| +2450822|11359|3|784| +2450822|11360|3|65| +2450822|11362|3|43| +2450822|11365|3|790| +2450822|11366|3|897| +2450822|11368|3|| +2450822|11371|3|390| +2450822|11372|3|866| +2450822|11374|3|754| +2450822|11377|3|444| +2450822|11378|3|723| +2450822|11380|3|755| +2450822|11383|3|839| +2450822|11384|3|514| +2450822|11386|3|183| +2450822|11389|3|409| +2450822|11390|3|645| +2450822|11392|3|86| +2450822|11395|3|249| +2450822|11396|3|15| +2450822|11398|3|846| +2450822|11401|3|352| +2450822|11402|3|499| +2450822|11404|3|378| +2450822|11407|3|381| +2450822|11408|3|10| +2450822|11410|3|| +2450822|11413|3|445| +2450822|11414|3|195| +2450822|11416|3|29| +2450822|11419|3|54| +2450822|11420|3|524| +2450822|11422|3|219| +2450822|11425|3|216| +2450822|11426|3|573| +2450822|11428|3|560| +2450822|11431|3|263| +2450822|11432|3|178| +2450822|11434|3|832| +2450822|11437|3|434| +2450822|11438|3|942| +2450822|11440|3|300| +2450822|11443|3|907| +2450822|11444|3|609| +2450822|11446|3|635| +2450822|11449|3|404| +2450822|11450|3|936| +2450822|11452|3|545| +2450822|11455|3|891| +2450822|11456|3|262| +2450822|11458|3|4| +2450822|11461|3|643| +2450822|11462|3|494| +2450822|11464|3|269| +2450822|11467|3|741| +2450822|11468|3|851| +2450822|11470|3|273| +2450822|11473|3|491| +2450822|11474|3|532| +2450822|11476|3|845| +2450822|11479|3|647| +2450822|11480|3|281| +2450822|11482|3|217| +2450822|11485|3|11| +2450822|11486|3|| +2450822|11488|3|959| +2450822|11491|3|749| +2450822|11492|3|305| +2450822|11494|3|324| +2450822|11497|3|654| +2450822|11498|3|120| +2450822|11500|3|414| +2450822|11503|3|209| +2450822|11504|3|33| +2450822|11506|3|62| +2450822|11509|3|13| +2450822|11510|3|791| +2450822|11512|3|31| +2450822|11515|3|966| +2450822|11516|3|934| +2450822|11518|3|655| +2450822|11521|3|645| +2450822|11522|3|352| +2450822|11524|3|994| +2450822|11527|3|146| +2450822|11528|3|558| +2450822|11530|3|21| +2450822|11533|3|217| +2450822|11534|3|200| +2450822|11536|3|537| +2450822|11539|3|558| +2450822|11540|3|653| +2450822|11542|3|494| +2450822|11545|3|161| +2450822|11546|3|554| +2450822|11548|3|241| +2450822|11551|3|753| +2450822|11552|3|498| +2450822|11554|3|| +2450822|11557|3|729| +2450822|11558|3|482| +2450822|11560|3|835| +2450822|11563|3|782| +2450822|11564|3|464| +2450822|11566|3|486| +2450822|11569|3|896| +2450822|11570|3|718| +2450822|11572|3|478| +2450822|11575|3|619| +2450822|11576|3|556| +2450822|11578|3|| +2450822|11581|3|102| +2450822|11582|3|659| +2450822|11584|3|882| +2450822|11587|3|617| +2450822|11588|3|646| +2450822|11590|3|429| +2450822|11593|3|340| +2450822|11594|3|236| +2450822|11596|3|300| +2450822|11599|3|645| +2450822|11600|3|840| +2450822|11602|3|619| +2450822|11605|3|185| +2450822|11606|3|236| +2450822|11608|3|609| +2450822|11611|3|401| +2450822|11612|3|178| +2450822|11614|3|864| +2450822|11617|3|165| +2450822|11618|3|125| +2450822|11620|3|998| +2450822|11623|3|179| +2450822|11624|3|188| +2450822|11626|3|298| +2450822|11629|3|| +2450822|11630|3|742| +2450822|11632|3|796| +2450822|11635|3|787| +2450822|11636|3|457| +2450822|11638|3|774| +2450822|11641|3|324| +2450822|11642|3|979| +2450822|11644|3|71| +2450822|11647|3|775| +2450822|11648|3|42| +2450822|11650|3|357| +2450822|11653|3|931| +2450822|11654|3|104| +2450822|11656|3|| +2450822|11659|3|728| +2450822|11660|3|176| +2450822|11662|3|579| +2450822|11665|3|281| +2450822|11666|3|310| +2450822|11668|3|57| +2450822|11671|3|308| +2450822|11672|3|20| +2450822|11674|3|939| +2450822|11677|3|115| +2450822|11678|3|109| +2450822|11680|3|950| +2450822|11683|3|888| +2450822|11684|3|986| +2450822|11686|3|412| +2450822|11689|3|112| +2450822|11690|3|600| +2450822|11692|3|242| +2450822|11695|3|683| +2450822|11696|3|904| +2450822|11698|3|385| +2450822|11701|3|884| +2450822|11702|3|579| +2450822|11704|3|836| +2450822|11707|3|132| +2450822|11708|3|799| +2450822|11710|3|734| +2450822|11713|3|| +2450822|11714|3|511| +2450822|11716|3|640| +2450822|11719|3|69| +2450822|11720|3|850| +2450822|11722|3|404| +2450822|11725|3|149| +2450822|11726|3|427| +2450822|11728|3|902| +2450822|11731|3|155| +2450822|11732|3|328| +2450822|11734|3|229| +2450822|11737|3|658| +2450822|11738|3|820| +2450822|11740|3|497| +2450822|11743|3|841| +2450822|11744|3|230| +2450822|11746|3|| +2450822|11749|3|608| +2450822|11750|3|41| +2450822|11752|3|795| +2450822|11755|3|193| +2450822|11756|3|144| +2450822|11758|3|223| +2450822|11761|3|790| +2450822|11762|3|322| +2450822|11764|3|38| +2450822|11767|3|325| +2450822|11768|3|183| +2450822|11770|3|534| +2450822|11773|3|681| +2450822|11774|3|557| +2450822|11776|3|173| +2450822|11779|3|114| +2450822|11780|3|87| +2450822|11782|3|602| +2450822|11785|3|686| +2450822|11786|3|175| +2450822|11788|3|468| +2450822|11791|3|332| +2450822|11792|3|459| +2450822|11794|3|363| +2450822|11797|3|760| +2450822|11798|3|714| +2450822|11800|3|829| +2450822|11803|3|739| +2450822|11804|3|759| +2450822|11806|3|14| +2450822|11809|3|286| +2450822|11810|3|956| +2450822|11812|3|178| +2450822|11815|3|750| +2450822|11816|3|253| +2450822|11818|3|488| +2450822|11821|3|243| +2450822|11822|3|873| +2450822|11824|3|971| +2450822|11827|3|972| +2450822|11828|3|| +2450822|11830|3|755| +2450822|11833|3|374| +2450822|11834|3|56| +2450822|11836|3|289| +2450822|11839|3|498| +2450822|11840|3|| +2450822|11842|3|481| +2450822|11845|3|12| +2450822|11846|3|925| +2450822|11848|3|202| +2450822|11851|3|582| +2450822|11852|3|155| +2450822|11854|3|241| +2450822|11857|3|826| +2450822|11858|3|533| +2450822|11860|3|53| +2450822|11863|3|105| +2450822|11864|3|571| +2450822|11866|3|55| +2450822|11869|3|677| +2450822|11870|3|| +2450822|11872|3|154| +2450822|11875|3|37| +2450822|11876|3|314| +2450822|11878|3|726| +2450822|11881|3|670| +2450822|11882|3|793| +2450822|11884|3|879| +2450822|11887|3|452| +2450822|11888|3|770| +2450822|11890|3|919| +2450822|11893|3|106| +2450822|11894|3|547| +2450822|11896|3|944| +2450822|11899|3|479| +2450822|11900|3|215| +2450822|11902|3|148| +2450822|11905|3|73| +2450822|11906|3|994| +2450822|11908|3|194| +2450822|11911|3|302| +2450822|11912|3|251| +2450822|11914|3|945| +2450822|11917|3|469| +2450822|11918|3|649| +2450822|11920|3|790| +2450822|11923|3|213| +2450822|11924|3|830| +2450822|11926|3|105| +2450822|11929|3|303| +2450822|11930|3|957| +2450822|11932|3|411| +2450822|11935|3|| +2450822|11936|3|459| +2450822|11938|3|33| +2450822|11941|3|576| +2450822|11942|3|446| +2450822|11944|3|308| +2450822|11947|3|831| +2450822|11948|3|818| +2450822|11950|3|281| +2450822|11953|3|424| +2450822|11954|3|904| +2450822|11956|3|| +2450822|11959|3|795| +2450822|11960|3|102| +2450822|11962|3|551| +2450822|11965|3|48| +2450822|11966|3|570| +2450822|11968|3|286| +2450822|11971|3|210| +2450822|11972|3|151| +2450822|11974|3|608| +2450822|11977|3|61| +2450822|11978|3|925| +2450822|11980|3|138| +2450822|11983|3|192| +2450822|11984|3|13| +2450822|11986|3|367| +2450822|11989|3|194| +2450822|11990|3|978| +2450822|11992|3|| +2450822|11995|3|610| +2450822|11996|3|440| +2450822|11998|3|21| +2450822|12001|3|853| +2450822|12002|3|521| +2450822|12004|3|577| +2450822|12007|3|810| +2450822|12008|3|537| +2450822|12010|3|434| +2450822|12013|3|477| +2450822|12014|3|436| +2450822|12016|3|631| +2450822|12019|3|470| +2450822|12020|3|545| +2450822|12022|3|715| +2450822|12025|3|466| +2450822|12026|3|141| +2450822|12028|3|710| +2450822|12031|3|393| +2450822|12032|3|432| +2450822|12034|3|6| +2450822|12037|3|859| +2450822|12038|3|664| +2450822|12040|3|809| +2450822|12043|3|832| +2450822|12044|3|774| +2450822|12046|3|308| +2450822|12049|3|547| +2450822|12050|3|976| +2450822|12052|3|598| +2450822|12055|3|588| +2450822|12056|3|744| +2450822|12058|3|489| +2450822|12061|3|192| +2450822|12062|3|23| +2450822|12064|3|740| +2450822|12067|3|180| +2450822|12068|3|556| +2450822|12070|3|353| +2450822|12073|3|424| +2450822|12074|3|760| +2450822|12076|3|783| +2450822|12079|3|355| +2450822|12080|3|663| +2450822|12082|3|294| +2450822|12085|3|690| +2450822|12086|3|345| +2450822|12088|3|133| +2450822|12091|3|486| +2450822|12092|3|936| +2450822|12094|3|239| +2450822|12097|3|60| +2450822|12098|3|720| +2450822|12100|3|79| +2450822|12103|3|781| +2450822|12104|3|146| +2450822|12106|3|575| +2450822|12109|3|33| +2450822|12110|3|553| +2450822|12112|3|215| +2450822|12115|3|940| +2450822|12116|3|228| +2450822|12118|3|471| +2450822|12121|3|714| +2450822|12122|3|872| +2450822|12124|3|785| +2450822|12127|3|183| +2450822|12128|3|635| +2450822|12130|3|452| +2450822|12133|3|822| +2450822|12134|3|821| +2450822|12136|3|832| +2450822|12139|3|962| +2450822|12140|3|976| +2450822|12142|3|367| +2450822|12145|3|610| +2450822|12146|3|925| +2450822|12148|3|342| +2450822|12151|3|886| +2450822|12152|3|363| +2450822|12154|3|| +2450822|12157|3|773| +2450822|12158|3|541| +2450822|12160|3|533| +2450822|12163|3|832| +2450822|12164|3|419| +2450822|12166|3|958| +2450822|12169|3|995| +2450822|12170|3|736| +2450822|12172|3|711| +2450822|12175|3|468| +2450822|12176|3|733| +2450822|12178|3|| +2450822|12181|3|160| +2450822|12182|3|427| +2450822|12184|3|261| +2450822|12187|3|| +2450822|12188|3|| +2450822|12190|3|55| +2450822|12193|3|38| +2450822|12194|3|913| +2450822|12196|3|331| +2450822|12199|3|326| +2450822|12200|3|280| +2450822|12202|3|344| +2450822|12205|3|931| +2450822|12206|3|699| +2450822|12208|3|82| +2450822|12211|3|362| +2450822|12212|3|10| +2450822|12214|3|869| +2450822|12217|3|321| +2450822|12218|3|| +2450822|12220|3|547| +2450822|12223|3|254| +2450822|12224|3|448| +2450822|12226|3|846| +2450822|12229|3|582| +2450822|12230|3|454| +2450822|12232|3|799| +2450822|12235|3|134| +2450822|12236|3|181| +2450822|12238|3|353| +2450822|12241|3|225| +2450822|12242|3|447| +2450822|12244|3|665| +2450822|12247|3|316| +2450822|12248|3|704| +2450822|12250|3|126| +2450822|12253|3|385| +2450822|12254|3|493| +2450822|12256|3|400| +2450822|12259|3|278| +2450822|12260|3|965| +2450822|12262|3|726| +2450822|12265|3|873| +2450822|12266|3|657| +2450822|12268|3|891| +2450822|12271|3|57| +2450822|12272|3|354| +2450822|12274|3|925| +2450822|12277|3|684| +2450822|12278|3|148| +2450822|12280|3|260| +2450822|12283|3|403| +2450822|12284|3|253| +2450822|12286|3|613| +2450822|12289|3|126| +2450822|12290|3|981| +2450822|12292|3|875| +2450822|12295|3|858| +2450822|12296|3|309| +2450822|12298|3|291| +2450822|12301|3|734| +2450822|12302|3|34| +2450822|12304|3|809| +2450822|12307|3|389| +2450822|12308|3|356| +2450822|12310|3|| +2450822|12313|3|155| +2450822|12314|3|451| +2450822|12316|3|723| +2450822|12319|3|333| +2450822|12320|3|376| +2450822|12322|3|297| +2450822|12325|3|414| +2450822|12326|3|438| +2450822|12328|3|350| +2450822|12331|3|898| +2450822|12332|3|79| +2450822|12334|3|932| +2450822|12337|3|521| +2450822|12338|3|959| +2450822|12340|3|386| +2450822|12343|3|770| +2450822|12344|3|543| +2450822|12346|3|921| +2450822|12349|3|470| +2450822|12350|3|694| +2450822|12352|3|100| +2450822|12355|3|466| +2450822|12356|3|131| +2450822|12358|3|20| +2450822|12361|3|818| +2450822|12362|3|60| +2450822|12364|3|478| +2450822|12367|3|293| +2450822|12368|3|795| +2450822|12370|3|381| +2450822|12373|3|375| +2450822|12374|3|851| +2450822|12376|3|975| +2450822|12379|3|873| +2450822|12380|3|790| +2450822|12382|3|290| +2450822|12385|3|149| +2450822|12386|3|130| +2450822|12388|3|209| +2450822|12391|3|961| +2450822|12392|3|303| +2450822|12394|3|| +2450822|12397|3|63| +2450822|12398|3|932| +2450822|12400|3|| +2450822|12403|3|231| +2450822|12404|3|693| +2450822|12406|3|870| +2450822|12409|3|355| +2450822|12410|3|269| +2450822|12412|3|342| +2450822|12415|3|816| +2450822|12416|3|350| +2450822|12418|3|591| +2450822|12421|3|664| +2450822|12422|3|659| +2450822|12424|3|465| +2450822|12427|3|713| +2450822|12428|3|930| +2450822|12430|3|185| +2450822|12433|3|388| +2450822|12434|3|575| +2450822|12436|3|718| +2450822|12439|3|351| +2450822|12440|3|518| +2450822|12442|3|251| +2450822|12445|3|659| +2450822|12446|3|111| +2450822|12448|3|128| +2450822|12451|3|791| +2450822|12452|3|240| +2450822|12454|3|966| +2450822|12457|3|| +2450822|12458|3|569| +2450822|12460|3|762| +2450822|12463|3|955| +2450822|12464|3|794| +2450822|12466|3|542| +2450822|12469|3|465| +2450822|12470|3|449| +2450822|12472|3|383| +2450822|12475|3|606| +2450822|12476|3|| +2450822|12478|3|632| +2450822|12481|3|433| +2450822|12482|3|543| +2450822|12484|3|| +2450822|12487|3|477| +2450822|12488|3|500| +2450822|12490|3|502| +2450822|12493|3|433| +2450822|12494|3|588| +2450822|12496|3|54| +2450822|12499|3|907| +2450822|12500|3|567| +2450822|12502|3|353| +2450822|12505|3|269| +2450822|12506|3|978| +2450822|12508|3|188| +2450822|12511|3|926| +2450822|12512|3|973| +2450822|12514|3|649| +2450822|12517|3|949| +2450822|12518|3|679| +2450822|12520|3|980| +2450822|12523|3|213| +2450822|12524|3|118| +2450822|12526|3|339| +2450822|12529|3|98| +2450822|12530|3|528| +2450822|12532|3|254| +2450822|12535|3|207| +2450822|12536|3|412| +2450822|12538|3|705| +2450822|12541|3|831| +2450822|12542|3|13| +2450822|12544|3|408| +2450822|12547|3|283| +2450822|12548|3|902| +2450822|12550|3|35| +2450822|12553|3|557| +2450822|12554|3|795| +2450822|12556|3|406| +2450822|12559|3|986| +2450822|12560|3|720| +2450822|12562|3|945| +2450822|12565|3|88| +2450822|12566|3|443| +2450822|12568|3|79| +2450822|12571|3|765| +2450822|12572|3|940| +2450822|12574|3|928| +2450822|12577|3|616| +2450822|12578|3|145| +2450822|12580|3|247| +2450822|12583|3|722| +2450822|12584|3|118| +2450822|12586|3|183| +2450822|12589|3|241| +2450822|12590|3|202| +2450822|12592|3|999| +2450822|12595|3|796| +2450822|12596|3|890| +2450822|12598|3|478| +2450822|12601|3|44| +2450822|12602|3|955| +2450822|12604|3|890| +2450822|12607|3|161| +2450822|12608|3|408| +2450822|12610|3|450| +2450822|12613|3|544| +2450822|12614|3|367| +2450822|12616|3|624| +2450822|12619|3|526| +2450822|12620|3|428| +2450822|12622|3|224| +2450822|12625|3|904| +2450822|12626|3|463| +2450822|12628|3|443| +2450822|12631|3|828| +2450822|12632|3|160| +2450822|12634|3|232| +2450822|12637|3|51| +2450822|12638|3|| +2450822|12640|3|559| +2450822|12643|3|417| +2450822|12644|3|623| +2450822|12646|3|723| +2450822|12649|3|426| +2450822|12650|3|391| +2450822|12652|3|337| +2450822|12655|3|656| +2450822|12656|3|942| +2450822|12658|3|715| +2450822|12661|3|842| +2450822|12662|3|50| +2450822|12664|3|447| +2450822|12667|3|369| +2450822|12668|3|887| +2450822|12670|3|357| +2450822|12673|3|584| +2450822|12674|3|560| +2450822|12676|3|121| +2450822|12679|3|896| +2450822|12680|3|663| +2450822|12682|3|874| +2450822|12685|3|124| +2450822|12686|3|106| +2450822|12688|3|119| +2450822|12691|3|1| +2450822|12692|3|276| +2450822|12694|3|| +2450822|12697|3|113| +2450822|12698|3|34| +2450822|12700|3|905| +2450822|12703|3|279| +2450822|12704|3|35| +2450822|12706|3|726| +2450822|12709|3|640| +2450822|12710|3|625| +2450822|12712|3|164| +2450822|12715|3|381| +2450822|12716|3|613| +2450822|12718|3|430| +2450822|12721|3|923| +2450822|12722|3|52| +2450822|12724|3|564| +2450822|12727|3|| +2450822|12728|3|522| +2450822|12730|3|623| +2450822|12733|3|184| +2450822|12734|3|747| +2450822|12736|3|| +2450822|12739|3|627| +2450822|12740|3|670| +2450822|12742|3|694| +2450822|12745|3|828| +2450822|12746|3|986| +2450822|12748|3|952| +2450822|12751|3|26| +2450822|12752|3|529| +2450822|12754|3|377| +2450822|12757|3|356| +2450822|12758|3|814| +2450822|12760|3|462| +2450822|12763|3|835| +2450822|12764|3|162| +2450822|12766|3|335| +2450822|12769|3|299| +2450822|12770|3|993| +2450822|12772|3|138| +2450822|12775|3|633| +2450822|12776|3|551| +2450822|12778|3|858| +2450822|12781|3|202| +2450822|12782|3|591| +2450822|12784|3|444| +2450822|12787|3|215| +2450822|12788|3|493| +2450822|12790|3|100| +2450822|12793|3|618| +2450822|12794|3|706| +2450822|12796|3|847| +2450822|12799|3|603| +2450822|12800|3|160| +2450822|12802|3|715| +2450822|12805|3|511| +2450822|12806|3|43| +2450822|12808|3|997| +2450822|12811|3|127| +2450822|12812|3|829| +2450822|12814|3|55| +2450822|12817|3|829| +2450822|12818|3|581| +2450822|12820|3|243| +2450822|12823|3|283| +2450822|12824|3|839| +2450822|12826|3|4| +2450822|12829|3|546| +2450822|12830|3|393| +2450822|12832|3|560| +2450822|12835|3|470| +2450822|12836|3|188| +2450822|12838|3|617| +2450822|12841|3|818| +2450822|12842|3|689| +2450822|12844|3|40| +2450822|12847|3|184| +2450822|12848|3|811| +2450822|12850|3|88| +2450822|12853|3|524| +2450822|12854|3|558| +2450822|12856|3|328| +2450822|12859|3|577| +2450822|12860|3|454| +2450822|12862|3|416| +2450822|12865|3|491| +2450822|12866|3|| +2450822|12868|3|305| +2450822|12871|3|693| +2450822|12872|3|580| +2450822|12874|3|21| +2450822|12877|3|826| +2450822|12878|3|463| +2450822|12880|3|824| +2450822|12883|3|683| +2450822|12884|3|885| +2450822|12886|3|188| +2450822|12889|3|76| +2450822|12890|3|574| +2450822|12892|3|50| +2450822|12895|3|832| +2450822|12896|3|647| +2450822|12898|3|282| +2450822|12901|3|64| +2450822|12902|3|914| +2450822|12904|3|28| +2450822|12907|3|822| +2450822|12908|3|149| +2450822|12910|3|751| +2450822|12913|3|| +2450822|12914|3|72| +2450822|12916|3|954| +2450822|12919|3|699| +2450822|12920|3|| +2450822|12922|3|263| +2450822|12925|3|347| +2450822|12926|3|543| +2450822|12928|3|345| +2450822|12931|3|824| +2450822|12932|3|69| +2450822|12934|3|986| +2450822|12937|3|313| +2450822|12938|3|783| +2450822|12940|3|925| +2450822|12943|3|293| +2450822|12944|3|110| +2450822|12946|3|945| +2450822|12949|3|901| +2450822|12950|3|137| +2450822|12952|3|294| +2450822|12955|3|990| +2450822|12956|3|744| +2450822|12958|3|198| +2450822|12961|3|324| +2450822|12962|3|202| +2450822|12964|3|601| +2450822|12967|3|39| +2450822|12968|3|817| +2450822|12970|3|120| +2450822|12973|3|331| +2450822|12974|3|420| +2450822|12976|3|754| +2450822|12979|3|187| +2450822|12980|3|673| +2450822|12982|3|563| +2450822|12985|3|166| +2450822|12986|3|888| +2450822|12988|3|439| +2450822|12991|3|481| +2450822|12992|3|466| +2450822|12994|3|462| +2450822|12997|3|| +2450822|12998|3|2| +2450822|13000|3|483| +2450822|13003|3|783| +2450822|13004|3|982| +2450822|13006|3|682| +2450822|13009|3|442| +2450822|13010|3|50| +2450822|13012|3|949| +2450822|13015|3|112| +2450822|13016|3|907| +2450822|13018|3|314| +2450822|13021|3|52| +2450822|13022|3|268| +2450822|13024|3|226| +2450822|13027|3|| +2450822|13028|3|302| +2450822|13030|3|203| +2450822|13033|3|13| +2450822|13034|3|971| +2450822|13036|3|417| +2450822|13039|3|479| +2450822|13040|3|450| +2450822|13042|3|500| +2450822|13045|3|85| +2450822|13046|3|640| +2450822|13048|3|967| +2450822|13051|3|988| +2450822|13052|3|507| +2450822|13054|3|43| +2450822|13057|3|199| +2450822|13058|3|931| +2450822|13060|3|795| +2450822|13063|3|757| +2450822|13064|3|897| +2450822|13066|3|198| +2450822|13069|3|650| +2450822|13070|3|312| +2450822|13072|3|846| +2450822|13075|3|494| +2450822|13076|3|233| +2450822|13078|3|306| +2450822|13081|3|| +2450822|13082|3|12| +2450822|13084|3|730| +2450822|13087|3|421| +2450822|13088|3|210| +2450822|13090|3|769| +2450822|13093|3|35| +2450822|13094|3|654| +2450822|13096|3|154| +2450822|13099|3|918| +2450822|13100|3|551| +2450822|13102|3|642| +2450822|13105|3|333| +2450822|13106|3|507| +2450822|13108|3|37| +2450822|13111|3|833| +2450822|13112|3|600| +2450822|13114|3|7| +2450822|13117|3|600| +2450822|13118|3|162| +2450822|13120|3|757| +2450822|13123|3|665| +2450822|13124|3|562| +2450822|13126|3|66| +2450822|13129|3|385| +2450822|13130|3|474| +2450822|13132|3|376| +2450822|13135|3|| +2450822|13136|3|427| +2450822|13138|3|367| +2450822|13141|3|119| +2450822|13142|3|| +2450822|13144|3|945| +2450822|13147|3|783| +2450822|13148|3|825| +2450822|13150|3|901| +2450822|13153|3|133| +2450822|13154|3|833| +2450822|13156|3|863| +2450822|13159|3|566| +2450822|13160|3|754| +2450822|13162|3|876| +2450822|13165|3|84| +2450822|13166|3|492| +2450822|13168|3|552| +2450822|13171|3|9| +2450822|13172|3|677| +2450822|13174|3|124| +2450822|13177|3|652| +2450822|13178|3|412| +2450822|13180|3|729| +2450822|13183|3|332| +2450822|13184|3|196| +2450822|13186|3|771| +2450822|13189|3|838| +2450822|13190|3|814| +2450822|13192|3|93| +2450822|13195|3|623| +2450822|13196|3|66| +2450822|13198|3|37| +2450822|13201|3|534| +2450822|13202|3|130| +2450822|13204|3|500| +2450822|13207|3|329| +2450822|13208|3|442| +2450822|13210|3|756| +2450822|13213|3|974| +2450822|13214|3|69| +2450822|13216|3|955| +2450822|13219|3|| +2450822|13220|3|632| +2450822|13222|3|955| +2450822|13225|3|67| +2450822|13226|3|106| +2450822|13228|3|402| +2450822|13231|3|962| +2450822|13232|3|900| +2450822|13234|3|201| +2450822|13237|3|724| +2450822|13238|3|955| +2450822|13240|3|22| +2450822|13243|3|| +2450822|13244|3|648| +2450822|13246|3|45| +2450822|13249|3|631| +2450822|13250|3|660| +2450822|13252|3|716| +2450822|13255|3|519| +2450822|13256|3|731| +2450822|13258|3|233| +2450822|13261|3|108| +2450822|13262|3|189| +2450822|13264|3|255| +2450822|13267|3|454| +2450822|13268|3|| +2450822|13270|3|120| +2450822|13273|3|530| +2450822|13274|3|755| +2450822|13276|3|616| +2450822|13279|3|178| +2450822|13280|3|413| +2450822|13282|3|79| +2450822|13285|3|993| +2450822|13286|3|666| +2450822|13288|3|995| +2450822|13291|3|689| +2450822|13292|3|107| +2450822|13294|3|452| +2450822|13297|3|421| +2450822|13298|3|598| +2450822|13300|3|395| +2450822|13303|3|543| +2450822|13304|3|793| +2450822|13306|3|960| +2450822|13309|3|378| +2450822|13310|3|886| +2450822|13312|3|976| +2450822|13315|3|9| +2450822|13316|3|689| +2450822|13318|3|503| +2450822|13321|3|126| +2450822|13322|3|271| +2450822|13324|3|662| +2450822|13327|3|| +2450822|13328|3|262| +2450822|13330|3|222| +2450822|13333|3|521| +2450822|13334|3|943| +2450822|13336|3|93| +2450822|13339|3|323| +2450822|13340|3|214| +2450822|13342|3|472| +2450822|13345|3|980| +2450822|13346|3|23| +2450822|13348|3|485| +2450822|13351|3|652| +2450822|13352|3|602| +2450822|13354|3|262| +2450822|13357|3|97| +2450822|13358|3|| +2450822|13360|3|584| +2450822|13363|3|920| +2450822|13364|3|| +2450822|13366|3|361| +2450822|13369|3|659| +2450822|13370|3|161| +2450822|13372|3|| +2450822|13375|3|282| +2450822|13376|3|243| +2450822|13378|3|554| +2450822|13381|3|783| +2450822|13382|3|258| +2450822|13384|3|850| +2450822|13387|3|165| +2450822|13388|3|244| +2450822|13390|3|765| +2450822|13393|3|392| +2450822|13394|3|244| +2450822|13396|3|508| +2450822|13399|3|| +2450822|13400|3|| +2450822|13402|3|310| +2450822|13405|3|680| +2450822|13406|3|555| +2450822|13408|3|215| +2450822|13411|3|703| +2450822|13412|3|| +2450822|13414|3|467| +2450822|13417|3|347| +2450822|13418|3|825| +2450822|13420|3|145| +2450822|13423|3|444| +2450822|13424|3|686| +2450822|13426|3|502| +2450822|13429|3|278| +2450822|13430|3|661| +2450822|13432|3|581| +2450822|13435|3|290| +2450822|13436|3|995| +2450822|13438|3|141| +2450822|13441|3|| +2450822|13442|3|448| +2450822|13444|3|149| +2450822|13447|3|549| +2450822|13448|3|901| +2450822|13450|3|81| +2450822|13453|3|958| +2450822|13454|3|814| +2450822|13456|3|697| +2450822|13459|3|103| +2450822|13460|3|384| +2450822|13462|3|125| +2450822|13465|3|82| +2450822|13466|3|| +2450822|13468|3|702| +2450822|13471|3|233| +2450822|13472|3|767| +2450822|13474|3|87| +2450822|13477|3|804| +2450822|13478|3|798| +2450822|13480|3|791| +2450822|13483|3|640| +2450822|13484|3|978| +2450822|13486|3|359| +2450822|13489|3|368| +2450822|13490|3|269| +2450822|13492|3|765| +2450822|13495|3|529| +2450822|13496|3|245| +2450822|13498|3|191| +2450822|13501|3|935| +2450822|13502|3|892| +2450822|13504|3|55| +2450822|13507|3|939| +2450822|13508|3|186| +2450822|13510|3|| +2450822|13513|3|945| +2450822|13514|3|965| +2450822|13516|3|420| +2450822|13519|3|666| +2450822|13520|3|296| +2450822|13522|3|172| +2450822|13525|3|244| +2450822|13526|3|380| +2450822|13528|3|416| +2450822|13531|3|387| +2450822|13532|3|838| +2450822|13534|3|1000| +2450822|13537|3|238| +2450822|13538|3|629| +2450822|13540|3|5| +2450822|13543|3|668| +2450822|13544|3|879| +2450822|13546|3|292| +2450822|13549|3|886| +2450822|13550|3|423| +2450822|13552|3|437| +2450822|13555|3|726| +2450822|13556|3|626| +2450822|13558|3|186| +2450822|13561|3|62| +2450822|13562|3|748| +2450822|13564|3|167| +2450822|13567|3|258| +2450822|13568|3|366| +2450822|13570|3|557| +2450822|13573|3|310| +2450822|13574|3|293| +2450822|13576|3|907| +2450822|13579|3|776| +2450822|13580|3|412| +2450822|13582|3|162| +2450822|13585|3|| +2450822|13586|3|861| +2450822|13588|3|139| +2450822|13591|3|392| +2450822|13592|3|399| +2450822|13594|3|258| +2450822|13597|3|394| +2450822|13598|3|278| +2450822|13600|3|226| +2450822|13603|3|415| +2450822|13604|3|204| +2450822|13606|3|503| +2450822|13609|3|974| +2450822|13610|3|749| +2450822|13612|3|160| +2450822|13615|3|927| +2450822|13616|3|288| +2450822|13618|3|716| +2450822|13621|3|173| +2450822|13622|3|| +2450822|13624|3|819| +2450822|13627|3|634| +2450822|13628|3|346| +2450822|13630|3|47| +2450822|13633|3|30| +2450822|13634|3|722| +2450822|13636|3|300| +2450822|13639|3|501| +2450822|13640|3|757| +2450822|13642|3|966| +2450822|13645|3|437| +2450822|13646|3|643| +2450822|13648|3|22| +2450822|13651|3|35| +2450822|13652|3|877| +2450822|13654|3|462| +2450822|13657|3|480| +2450822|13658|3|680| +2450822|13660|3|| +2450822|13663|3|134| +2450822|13664|3|892| +2450822|13666|3|552| +2450822|13669|3|530| +2450822|13670|3|518| +2450822|13672|3|489| +2450822|13675|3|180| +2450822|13676|3|193| +2450822|13678|3|848| +2450822|13681|3|19| +2450822|13682|3|244| +2450822|13684|3|566| +2450822|13687|3|125| +2450822|13688|3|750| +2450822|13690|3|792| +2450822|13693|3|45| +2450822|13694|3|559| +2450822|13696|3|954| +2450822|13699|3|704| +2450822|13700|3|14| +2450822|13702|3|685| +2450822|13705|3|562| +2450822|13706|3|881| +2450822|13708|3|336| +2450822|13711|3|469| +2450822|13712|3|385| +2450822|13714|3|| +2450822|13717|3|| +2450822|13718|3|979| +2450822|13720|3|129| +2450822|13723|3|757| +2450822|13724|3|726| +2450822|13726|3|725| +2450822|13729|3|114| +2450822|13730|3|676| +2450822|13732|3|373| +2450822|13735|3|754| +2450822|13736|3|132| +2450822|13738|3|802| +2450822|13741|3|992| +2450822|13742|3|70| +2450822|13744|3|819| +2450822|13747|3|175| +2450822|13748|3|849| +2450822|13750|3|848| +2450822|13753|3|978| +2450822|13754|3|198| +2450822|13756|3|7| +2450822|13759|3|604| +2450822|13760|3|387| +2450822|13762|3|219| +2450822|13765|3|595| +2450822|13766|3|100| +2450822|13768|3|366| +2450822|13771|3|984| +2450822|13772|3|520| +2450822|13774|3|831| +2450822|13777|3|65| +2450822|13778|3|906| +2450822|13780|3|517| +2450822|13783|3|462| +2450822|13784|3|851| +2450822|13786|3|509| +2450822|13789|3|154| +2450822|13790|3|689| +2450822|13792|3|393| +2450822|13795|3|31| +2450822|13796|3|50| +2450822|13798|3|654| +2450822|13801|3|148| +2450822|13802|3|424| +2450822|13804|3|426| +2450822|13807|3|834| +2450822|13808|3|553| +2450822|13810|3|997| +2450822|13813|3|379| +2450822|13814|3|776| +2450822|13816|3|835| +2450822|13819|3|| +2450822|13820|3|990| +2450822|13822|3|93| +2450822|13825|3|855| +2450822|13826|3|720| +2450822|13828|3|609| +2450822|13831|3|284| +2450822|13832|3|975| +2450822|13834|3|997| +2450822|13837|3|349| +2450822|13838|3|889| +2450822|13840|3|922| +2450822|13843|3|149| +2450822|13844|3|155| +2450822|13846|3|715| +2450822|13849|3|807| +2450822|13850|3|162| +2450822|13852|3|699| +2450822|13855|3|157| +2450822|13856|3|937| +2450822|13858|3|579| +2450822|13861|3|72| +2450822|13862|3|579| +2450822|13864|3|846| +2450822|13867|3|629| +2450822|13868|3|266| +2450822|13870|3|280| +2450822|13873|3|24| +2450822|13874|3|341| +2450822|13876|3|686| +2450822|13879|3|765| +2450822|13880|3|607| +2450822|13882|3|73| +2450822|13885|3|972| +2450822|13886|3|494| +2450822|13888|3|354| +2450822|13891|3|944| +2450822|13892|3|539| +2450822|13894|3|295| +2450822|13897|3|556| +2450822|13898|3|232| +2450822|13900|3|714| +2450822|13903|3|372| +2450822|13904|3|418| +2450822|13906|3|143| +2450822|13909|3|293| +2450822|13910|3|285| +2450822|13912|3|196| +2450822|13915|3|763| +2450822|13916|3|831| +2450822|13918|3|29| +2450822|13921|3|362| +2450822|13922|3|477| +2450822|13924|3|399| +2450822|13927|3|885| +2450822|13928|3|824| +2450822|13930|3|738| +2450822|13933|3|817| +2450822|13934|3|939| +2450822|13936|3|686| +2450822|13939|3|250| +2450822|13940|3|7| +2450822|13942|3|35| +2450822|13945|3|756| +2450822|13946|3|12| +2450822|13948|3|460| +2450822|13951|3|852| +2450822|13952|3|154| +2450822|13954|3|431| +2450822|13957|3|921| +2450822|13958|3|681| +2450822|13960|3|596| +2450822|13963|3|496| +2450822|13964|3|58| +2450822|13966|3|922| +2450822|13969|3|24| +2450822|13970|3|487| +2450822|13972|3|191| +2450822|13975|3|549| +2450822|13976|3|505| +2450822|13978|3|196| +2450822|13981|3|| +2450822|13982|3|189| +2450822|13984|3|587| +2450822|13987|3|900| +2450822|13988|3|831| +2450822|13990|3|245| +2450822|13993|3|401| +2450822|13994|3|44| +2450822|13996|3|258| +2450822|13999|3|43| +2450822|14000|3|590| +2450822|14002|3|843| +2450822|14005|3|839| +2450822|14006|3|671| +2450822|14008|3|759| +2450822|14011|3|234| +2450822|14012|3|690| +2450822|14014|3|173| +2450822|14017|3|454| +2450822|14018|3|53| +2450822|14020|3|458| +2450822|14023|3|988| +2450822|14024|3|870| +2450822|14026|3|39| +2450822|14029|3|122| +2450822|14030|3|955| +2450822|14032|3|| +2450822|14035|3|606| +2450822|14036|3|255| +2450822|14038|3|126| +2450822|14041|3|847| +2450822|14042|3|55| +2450822|14044|3|796| +2450822|14047|3|417| +2450822|14048|3|703| +2450822|14050|3|| +2450822|14053|3|636| +2450822|14054|3|695| +2450822|14056|3|341| +2450822|14059|3|250| +2450822|14060|3|250| +2450822|14062|3|892| +2450822|14065|3|615| +2450822|14066|3|44| +2450822|14068|3|762| +2450822|14071|3|518| +2450822|14072|3|385| +2450822|14074|3|948| +2450822|14077|3|443| +2450822|14078|3|434| +2450822|14080|3|23| +2450822|14083|3|574| +2450822|14084|3|188| +2450822|14086|3|771| +2450822|14089|3|761| +2450822|14090|3|683| +2450822|14092|3|564| +2450822|14095|3|758| +2450822|14096|3|280| +2450822|14098|3|108| +2450822|14101|3|432| +2450822|14102|3|476| +2450822|14104|3|427| +2450822|14107|3|443| +2450822|14108|3|29| +2450822|14110|3|920| +2450822|14113|3|242| +2450822|14114|3|573| +2450822|14116|3|249| +2450822|14119|3|634| +2450822|14120|3|803| +2450822|14122|3|124| +2450822|14125|3|559| +2450822|14126|3|215| +2450822|14128|3|154| +2450822|14131|3|759| +2450822|14132|3|436| +2450822|14134|3|882| +2450822|14137|3|852| +2450822|14138|3|469| +2450822|14140|3|48| +2450822|14143|3|516| +2450822|14144|3|13| +2450822|14146|3|964| +2450822|14149|3|368| +2450822|14150|3|437| +2450822|14152|3|645| +2450822|14155|3|871| +2450822|14156|3|310| +2450822|14158|3|801| +2450822|14161|3|193| +2450822|14162|3|145| +2450822|14164|3|841| +2450822|14167|3|356| +2450822|14168|3|953| +2450822|14170|3|98| +2450822|14173|3|348| +2450822|14174|3|690| +2450822|14176|3|215| +2450822|14179|3|621| +2450822|14180|3|740| +2450822|14182|3|54| +2450822|14185|3|614| +2450822|14186|3|421| +2450822|14188|3|620| +2450822|14191|3|346| +2450822|14192|3|352| +2450822|14194|3|129| +2450822|14197|3|| +2450822|14198|3|525| +2450822|14200|3|49| +2450822|14203|3|454| +2450822|14204|3|994| +2450822|14206|3|621| +2450822|14209|3|974| +2450822|14210|3|735| +2450822|14212|3|560| +2450822|14215|3|| +2450822|14216|3|199| +2450822|14218|3|548| +2450822|14221|3|307| +2450822|14222|3|466| +2450822|14224|3|655| +2450822|14227|3|736| +2450822|14228|3|556| +2450822|14230|3|962| +2450822|14233|3|487| +2450822|14234|3|| +2450822|14236|3|803| +2450822|14239|3|279| +2450822|14240|3|689| +2450822|14242|3|217| +2450822|14245|3|496| +2450822|14246|3|688| +2450822|14248|3|558| +2450822|14251|3|| +2450822|14252|3|558| +2450822|14254|3|513| +2450822|14257|3|| +2450822|14258|3|837| +2450822|14260|3|728| +2450822|14263|3|989| +2450822|14264|3|| +2450822|14266|3|716| +2450822|14269|3|20| +2450822|14270|3|150| +2450822|14272|3|103| +2450822|14275|3|995| +2450822|14276|3|470| +2450822|14278|3|| +2450822|14281|3|802| +2450822|14282|3|509| +2450822|14284|3|290| +2450822|14287|3|563| +2450822|14288|3|28| +2450822|14290|3|704| +2450822|14293|3|743| +2450822|14294|3|996| +2450822|14296|3|37| +2450822|14299|3|800| +2450822|14300|3|521| +2450822|14302|3|850| +2450822|14305|3|669| +2450822|14306|3|900| +2450822|14308|3|987| +2450822|14311|3|974| +2450822|14312|3|313| +2450822|14314|3|430| +2450822|14317|3|199| +2450822|14318|3|885| +2450822|14320|3|176| +2450822|14323|3|563| +2450822|14324|3|882| +2450822|14326|3|77| +2450822|14329|3|338| +2450822|14330|3|43| +2450822|14332|3|498| +2450822|14335|3|333| +2450822|14336|3|270| +2450822|14338|3|974| +2450822|14341|3|191| +2450822|14342|3|133| +2450822|14344|3|448| +2450822|14347|3|612| +2450822|14348|3|26| +2450822|14350|3|821| +2450822|14353|3|| +2450822|14354|3|103| +2450822|14356|3|798| +2450822|14359|3|553| +2450822|14360|3|519| +2450822|14362|3|275| +2450822|14365|3|| +2450822|14366|3|843| +2450822|14368|3|375| +2450822|14371|3|399| +2450822|14372|3|871| +2450822|14374|3|246| +2450822|14377|3|317| +2450822|14378|3|365| +2450822|14380|3|702| +2450822|14383|3|926| +2450822|14384|3|181| +2450822|14386|3|179| +2450822|14389|3|316| +2450822|14390|3|602| +2450822|14392|3|257| +2450822|14395|3|792| +2450822|14396|3|914| +2450822|14398|3|979| +2450822|14401|3|459| +2450822|14402|3|514| +2450822|14404|3|29| +2450822|14407|3|482| +2450822|14408|3|116| +2450822|14410|3|880| +2450822|14413|3|779| +2450822|14414|3|431| +2450822|14416|3|221| +2450822|14419|3|315| +2450822|14420|3|141| +2450822|14422|3|271| +2450822|14425|3|610| +2450822|14426|3|109| +2450822|14428|3|260| +2450822|14431|3|205| +2450822|14432|3|93| +2450822|14434|3|142| +2450822|14437|3|387| +2450822|14438|3|499| +2450822|14440|3|96| +2450822|14443|3|993| +2450822|14444|3|490| +2450822|14446|3|468| +2450822|14449|3|709| +2450822|14450|3|836| +2450822|14452|3|473| +2450822|14455|3|969| +2450822|14456|3|746| +2450822|14458|3|588| +2450822|14461|3|20| +2450822|14462|3|32| +2450822|14464|3|238| +2450822|14467|3|845| +2450822|14468|3|33| +2450822|14470|3|770| +2450822|14473|3|808| +2450822|14474|3|965| +2450822|14476|3|803| +2450822|14479|3|310| +2450822|14480|3|321| +2450822|14482|3|428| +2450822|14485|3|354| +2450822|14486|3|78| +2450822|14488|3|| +2450822|14491|3|525| +2450822|14492|3|341| +2450822|14494|3|36| +2450822|14497|3|948| +2450822|14498|3|626| +2450822|14500|3|377| +2450822|14503|3|993| +2450822|14504|3|626| +2450822|14506|3|930| +2450822|14509|3|429| +2450822|14510|3|252| +2450822|14512|3|11| +2450822|14515|3|425| +2450822|14516|3|790| +2450822|14518|3|954| +2450822|14521|3|618| +2450822|14522|3|498| +2450822|14524|3|780| +2450822|14527|3|49| +2450822|14528|3|642| +2450822|14530|3|79| +2450822|14533|3|651| +2450822|14534|3|13| +2450822|14536|3|816| +2450822|14539|3|568| +2450822|14540|3|609| +2450822|14542|3|86| +2450822|14545|3|726| +2450822|14546|3|608| +2450822|14548|3|463| +2450822|14551|3|161| +2450822|14552|3|| +2450822|14554|3|846| +2450822|14557|3|167| +2450822|14558|3|383| +2450822|14560|3|180| +2450822|14563|3|786| +2450822|14564|3|568| +2450822|14566|3|| +2450822|14569|3|135| +2450822|14570|3|420| +2450822|14572|3|850| +2450822|14575|3|735| +2450822|14576|3|280| +2450822|14578|3|120| +2450822|14581|3|233| +2450822|14582|3|798| +2450822|14584|3|139| +2450822|14587|3|747| +2450822|14588|3|185| +2450822|14590|3|614| +2450822|14593|3|186| +2450822|14594|3|45| +2450822|14596|3|852| +2450822|14599|3|266| +2450822|14600|3|| +2450822|14602|3|987| +2450822|14605|3|633| +2450822|14606|3|672| +2450822|14608|3|92| +2450822|14611|3|201| +2450822|14612|3|165| +2450822|14614|3|320| +2450822|14617|3|15| +2450822|14618|3|555| +2450822|14620|3|310| +2450822|14623|3|882| +2450822|14624|3|133| +2450822|14626|3|522| +2450822|14629|3|16| +2450822|14630|3|92| +2450822|14632|3|| +2450822|14635|3|502| +2450822|14636|3|340| +2450822|14638|3|16| +2450822|14641|3|574| +2450822|14642|3|561| +2450822|14644|3|430| +2450822|14647|3|829| +2450822|14648|3|308| +2450822|14650|3|827| +2450822|14653|3|397| +2450822|14654|3|373| +2450822|14656|3|261| +2450822|14659|3|469| +2450822|14660|3|858| +2450822|14662|3|45| +2450822|14665|3|489| +2450822|14666|3|149| +2450822|14668|3|547| +2450822|14671|3|577| +2450822|14672|3|| +2450822|14674|3|| +2450822|14677|3|643| +2450822|14678|3|948| +2450822|14680|3|215| +2450822|14683|3|629| +2450822|14684|3|831| +2450822|14686|3|433| +2450822|14689|3|750| +2450822|14690|3|338| +2450822|14692|3|32| +2450822|14695|3|91| +2450822|14696|3|350| +2450822|14698|3|450| +2450822|14701|3|566| +2450822|14702|3|404| +2450822|14704|3|487| +2450822|14707|3|166| +2450822|14708|3|72| +2450822|14710|3|540| +2450822|14713|3|389| +2450822|14714|3|89| +2450822|14716|3|269| +2450822|14719|3|260| +2450822|14720|3|490| +2450822|14722|3|554| +2450822|14725|3|90| +2450822|14726|3|59| +2450822|14728|3|615| +2450822|14731|3|724| +2450822|14732|3|438| +2450822|14734|3|337| +2450822|14737|3|539| +2450822|14738|3|918| +2450822|14740|3|758| +2450822|14743|3|511| +2450822|14744|3|584| +2450822|14746|3|357| +2450822|14749|3|58| +2450822|14750|3|244| +2450822|14752|3|808| +2450822|14755|3|82| +2450822|14756|3|816| +2450822|14758|3|476| +2450822|14761|3|607| +2450822|14762|3|380| +2450822|14764|3|671| +2450822|14767|3|241| +2450822|14768|3|838| +2450822|14770|3|50| +2450822|14773|3|22| +2450822|14774|3|865| +2450822|14776|3|204| +2450822|14779|3|217| +2450822|14780|3|472| +2450822|14782|3|978| +2450822|14785|3|| +2450822|14786|3|945| +2450822|14788|3|479| +2450822|14791|3|861| +2450822|14792|3|357| +2450822|14794|3|946| +2450822|14797|3|| +2450822|14798|3|505| +2450822|14800|3|802| +2450822|14803|3|489| +2450822|14804|3|548| +2450822|14806|3|620| +2450822|14809|3|971| +2450822|14810|3|865| +2450822|14812|3|| +2450822|14815|3|684| +2450822|14816|3|659| +2450822|14818|3|42| +2450822|14821|3|148| +2450822|14822|3|109| +2450822|14824|3|993| +2450822|14827|3|216| +2450822|14828|3|10| +2450822|14830|3|132| +2450822|14833|3|241| +2450822|14834|3|955| +2450822|14836|3|14| +2450822|14839|3|893| +2450822|14840|3|714| +2450822|14842|3|242| +2450822|14845|3|| +2450822|14846|3|863| +2450822|14848|3|793| +2450822|14851|3|547| +2450822|14852|3|550| +2450822|14854|3|| +2450822|14857|3|859| +2450822|14858|3|834| +2450822|14860|3|811| +2450822|14863|3|335| +2450822|14864|3|82| +2450822|14866|3|63| +2450822|14869|3|320| +2450822|14870|3|818| +2450822|14872|3|973| +2450822|14875|3|141| +2450822|14876|3|339| +2450822|14878|3|986| +2450822|14881|3|293| +2450822|14882|3|520| +2450822|14884|3|956| +2450822|14887|3|503| +2450822|14888|3|438| +2450822|14890|3|456| +2450822|14893|3|238| +2450822|14894|3|177| +2450822|14896|3|116| +2450822|14899|3|653| +2450822|14900|3|10| +2450822|14902|3|801| +2450822|14905|3|856| +2450822|14906|3|216| +2450822|14908|3|| +2450822|14911|3|155| +2450822|14912|3|801| +2450822|14914|3|654| +2450822|14917|3|837| +2450822|14918|3|518| +2450822|14920|3|739| +2450822|14923|3|485| +2450822|14924|3|756| +2450822|14926|3|376| +2450822|14929|3|569| +2450822|14930|3|71| +2450822|14932|3|275| +2450822|14935|3|364| +2450822|14936|3|741| +2450822|14938|3|612| +2450822|14941|3|28| +2450822|14942|3|750| +2450822|14944|3|378| +2450822|14947|3|337| +2450822|14948|3|752| +2450822|14950|3|689| +2450822|14953|3|693| +2450822|14954|3|328| +2450822|14956|3|718| +2450822|14959|3|964| +2450822|14960|3|798| +2450822|14962|3|382| +2450822|14965|3|774| +2450822|14966|3|636| +2450822|14968|3|571| +2450822|14971|3|365| +2450822|14972|3|259| +2450822|14974|3|651| +2450822|14977|3|707| +2450822|14978|3|859| +2450822|14980|3|4| +2450822|14983|3|211| +2450822|14984|3|| +2450822|14986|3|151| +2450822|14989|3|455| +2450822|14990|3|716| +2450822|14992|3|| +2450822|14995|3|844| +2450822|14996|3|644| +2450822|14998|3|461| +2450822|15001|3|934| +2450822|15002|3|987| +2450822|15004|3|494| +2450822|15007|3|960| +2450822|15008|3|766| +2450822|15010|3|350| +2450822|15013|3|871| +2450822|15014|3|222| +2450822|15016|3|38| +2450822|15019|3|665| +2450822|15020|3|87| +2450822|15022|3|434| +2450822|15025|3|261| +2450822|15026|3|758| +2450822|15028|3|220| +2450822|15031|3|653| +2450822|15032|3|754| +2450822|15034|3|385| +2450822|15037|3|613| +2450822|15038|3|384| +2450822|15040|3|439| +2450822|15043|3|449| +2450822|15044|3|624| +2450822|15046|3|| +2450822|15049|3|314| +2450822|15050|3|911| +2450822|15052|3|370| +2450822|15055|3|28| +2450822|15056|3|693| +2450822|15058|3|861| +2450822|15061|3|33| +2450822|15062|3|385| +2450822|15064|3|953| +2450822|15067|3|341| +2450822|15068|3|167| +2450822|15070|3|628| +2450822|15073|3|575| +2450822|15074|3|57| +2450822|15076|3|530| +2450822|15079|3|404| +2450822|15080|3|784| +2450822|15082|3|118| +2450822|15085|3|644| +2450822|15086|3|719| +2450822|15088|3|913| +2450822|15091|3|375| +2450822|15092|3|72| +2450822|15094|3|965| +2450822|15097|3|22| +2450822|15098|3|516| +2450822|15100|3|927| +2450822|15103|3|675| +2450822|15104|3|597| +2450822|15106|3|207| +2450822|15109|3|689| +2450822|15110|3|158| +2450822|15112|3|287| +2450822|15115|3|131| +2450822|15116|3|73| +2450822|15118|3|29| +2450822|15121|3|598| +2450822|15122|3|607| +2450822|15124|3|258| +2450822|15127|3|18| +2450822|15128|3|892| +2450822|15130|3|305| +2450822|15133|3|718| +2450822|15134|3|| +2450822|15136|3|983| +2450822|15139|3|970| +2450822|15140|3|567| +2450822|15142|3|81| +2450822|15145|3|965| +2450822|15146|3|169| +2450822|15148|3|694| +2450822|15151|3|587| +2450822|15152|3|39| +2450822|15154|3|605| +2450822|15157|3|682| +2450822|15158|3|247| +2450822|15160|3|32| +2450822|15163|3|| +2450822|15164|3|598| +2450822|15166|3|733| +2450822|15169|3|| +2450822|15170|3|403| +2450822|15172|3|96| +2450822|15175|3|914| +2450822|15176|3|85| +2450822|15178|3|804| +2450822|15181|3|99| +2450822|15182|3|629| +2450822|15184|3|420| +2450822|15187|3|964| +2450822|15188|3|333| +2450822|15190|3|711| +2450822|15193|3|227| +2450822|15194|3|543| +2450822|15196|3|712| +2450822|15199|3|| +2450822|15200|3|275| +2450822|15202|3|168| +2450822|15205|3|793| +2450822|15206|3|156| +2450822|15208|3|524| +2450822|15211|3|324| +2450822|15212|3|322| +2450822|15214|3|458| +2450822|15217|3|539| +2450822|15218|3|198| +2450822|15220|3|177| +2450822|15223|3|240| +2450822|15224|3|953| +2450822|15226|3|311| +2450822|15229|3|421| +2450822|15230|3|833| +2450822|15232|3|113| +2450822|15235|3|| +2450822|15236|3|118| +2450822|15238|3|171| +2450822|15241|3|58| +2450822|15242|3|998| +2450822|15244|3|673| +2450822|15247|3|541| +2450822|15248|3|755| +2450822|15250|3|655| +2450822|15253|3|770| +2450822|15254|3|463| +2450822|15256|3|367| +2450822|15259|3|960| +2450822|15260|3|403| +2450822|15262|3|595| +2450822|15265|3|307| +2450822|15266|3|104| +2450822|15268|3|377| +2450822|15271|3|46| +2450822|15272|3|485| +2450822|15274|3|286| +2450822|15277|3|19| +2450822|15278|3|568| +2450822|15280|3|173| +2450822|15283|3|| +2450822|15284|3|606| +2450822|15286|3|622| +2450822|15289|3|47| +2450822|15290|3|633| +2450822|15292|3|260| +2450822|15295|3|167| +2450822|15296|3|118| +2450822|15298|3|700| +2450822|15301|3|307| +2450822|15302|3|195| +2450822|15304|3|918| +2450822|15307|3|648| +2450822|15308|3|808| +2450822|15310|3|162| +2450822|15313|3|428| +2450822|15314|3|242| +2450822|15316|3|564| +2450822|15319|3|327| +2450822|15320|3|952| +2450822|15322|3|68| +2450822|15325|3|229| +2450822|15326|3|| +2450822|15328|3|203| +2450822|15331|3|321| +2450822|15332|3|516| +2450822|15334|3|134| +2450822|15337|3|22| +2450822|15338|3|537| +2450822|15340|3|96| +2450822|15343|3|761| +2450822|15344|3|0| +2450822|15346|3|272| +2450822|15349|3|164| +2450822|15350|3|8| +2450822|15352|3|56| +2450822|15355|3|782| +2450822|15356|3|696| +2450822|15358|3|853| +2450822|15361|3|493| +2450822|15362|3|771| +2450822|15364|3|899| +2450822|15367|3|670| +2450822|15368|3|878| +2450822|15370|3|544| +2450822|15373|3|940| +2450822|15374|3|711| +2450822|15376|3|177| +2450822|15379|3|589| +2450822|15380|3|636| +2450822|15382|3|553| +2450822|15385|3|450| +2450822|15386|3|897| +2450822|15388|3|283| +2450822|15391|3|242| +2450822|15392|3|793| +2450822|15394|3|787| +2450822|15397|3|968| +2450822|15398|3|161| +2450822|15400|3|798| +2450822|15403|3|135| +2450822|15404|3|848| +2450822|15406|3|465| +2450822|15409|3|955| +2450822|15410|3|478| +2450822|15412|3|825| +2450822|15415|3|214| +2450822|15416|3|895| +2450822|15418|3|159| +2450822|15421|3|979| +2450822|15422|3|659| +2450822|15424|3|334| +2450822|15427|3|962| +2450822|15428|3|178| +2450822|15430|3|| +2450822|15433|3|335| +2450822|15434|3|19| +2450822|15436|3|509| +2450822|15439|3|689| +2450822|15440|3|280| +2450822|15442|3|| +2450822|15445|3|| +2450822|15446|3|186| +2450822|15448|3|| +2450822|15451|3|375| +2450822|15452|3|497| +2450822|15454|3|370| +2450822|15457|3|34| +2450822|15458|3|523| +2450822|15460|3|60| +2450822|15463|3|124| +2450822|15464|3|700| +2450822|15466|3|395| +2450822|15469|3|704| +2450822|15470|3|427| +2450822|15472|3|498| +2450822|15475|3|851| +2450822|15476|3|672| +2450822|15478|3|103| +2450822|15481|3|635| +2450822|15482|3|850| +2450822|15484|3|349| +2450822|15487|3|984| +2450822|15488|3|682| +2450822|15490|3|921| +2450822|15493|3|845| +2450822|15494|3|744| +2450822|15496|3|693| +2450822|15499|3|162| +2450822|15500|3|968| +2450822|15502|3|599| +2450822|15505|3|273| +2450822|15506|3|547| +2450822|15508|3|388| +2450822|15511|3|136| +2450822|15512|3|609| +2450822|15514|3|232| +2450822|15517|3|894| +2450822|15518|3|49| +2450822|15520|3|347| +2450822|15523|3|353| +2450822|15524|3|961| +2450822|15526|3|895| +2450822|15529|3|143| +2450822|15530|3|986| +2450822|15532|3|84| +2450822|15535|3|883| +2450822|15536|3|78| +2450822|15538|3|921| +2450822|15541|3|149| +2450822|15542|3|718| +2450822|15544|3|327| +2450822|15547|3|223| +2450822|15548|3|| +2450822|15550|3|812| +2450822|15553|3|416| +2450822|15554|3|| +2450822|15556|3|151| +2450822|15559|3|916| +2450822|15560|3|463| +2450822|15562|3|242| +2450822|15565|3|529| +2450822|15566|3|566| +2450822|15568|3|146| +2450822|15571|3|310| +2450822|15572|3|916| +2450822|15574|3|143| +2450822|15577|3|214| +2450822|15578|3|278| +2450822|15580|3|477| +2450822|15583|3|258| +2450822|15584|3|74| +2450822|15586|3|293| +2450822|15589|3|15| +2450822|15590|3|66| +2450822|15592|3|873| +2450822|15595|3|817| +2450822|15596|3|973| +2450822|15598|3|580| +2450822|15601|3|321| +2450822|15602|3|1| +2450822|15604|3|897| +2450822|15607|3|98| +2450822|15608|3|182| +2450822|15610|3|736| +2450822|15613|3|534| +2450822|15614|3|28| +2450822|15616|3|688| +2450822|15619|3|656| +2450822|15620|3|325| +2450822|15622|3|367| +2450822|15625|3|737| +2450822|15626|3|158| +2450822|15628|3|389| +2450822|15631|3|932| +2450822|15632|3|750| +2450822|15634|3|870| +2450822|15637|3|565| +2450822|15638|3|463| +2450822|15640|3|900| +2450822|15643|3|0| +2450822|15644|3|169| +2450822|15646|3|445| +2450822|15649|3|531| +2450822|15650|3|547| +2450822|15652|3|267| +2450822|15655|3|559| +2450822|15656|3|416| +2450822|15658|3|307| +2450822|15661|3|792| +2450822|15662|3|644| +2450822|15664|3|767| +2450822|15667|3|19| +2450822|15668|3|417| +2450822|15670|3|175| +2450822|15673|3|942| +2450822|15674|3|299| +2450822|15676|3|896| +2450822|15679|3|65| +2450822|15680|3|377| +2450822|15682|3|342| +2450822|15685|3|463| +2450822|15686|3|774| +2450822|15688|3|| +2450822|15691|3|924| +2450822|15692|3|949| +2450822|15694|3|658| +2450822|15697|3|53| +2450822|15698|3|| +2450822|15700|3|335| +2450822|15703|3|11| +2450822|15704|3|462| +2450822|15706|3|| +2450822|15709|3|130| +2450822|15710|3|682| +2450822|15712|3|533| +2450822|15715|3|392| +2450822|15716|3|16| +2450822|15718|3|145| +2450822|15721|3|462| +2450822|15722|3|548| +2450822|15724|3|855| +2450822|15727|3|833| +2450822|15728|3|556| +2450822|15730|3|| +2450822|15733|3|186| +2450822|15734|3|313| +2450822|15736|3|67| +2450822|15739|3|570| +2450822|15740|3|875| +2450822|15742|3|306| +2450822|15745|3|333| +2450822|15746|3|60| +2450822|15748|3|661| +2450822|15751|3|41| +2450822|15752|3|719| +2450822|15754|3|541| +2450822|15757|3|708| +2450822|15758|3|531| +2450822|15760|3|658| +2450822|15763|3|989| +2450822|15764|3|889| +2450822|15766|3|446| +2450822|15769|3|| +2450822|15770|3|548| +2450822|15772|3|319| +2450822|15775|3|222| +2450822|15776|3|630| +2450822|15778|3|79| +2450822|15781|3|723| +2450822|15782|3|58| +2450822|15784|3|799| +2450822|15787|3|| +2450822|15788|3|150| +2450822|15790|3|158| +2450822|15793|3|592| +2450822|15794|3|190| +2450822|15796|3|261| +2450822|15799|3|918| +2450822|15800|3|138| +2450822|15802|3|338| +2450822|15805|3|244| +2450822|15806|3|165| +2450822|15808|3|140| +2450822|15811|3|| +2450822|15812|3|910| +2450822|15814|3|562| +2450822|15817|3|909| +2450822|15818|3|626| +2450822|15820|3|656| +2450822|15823|3|701| +2450822|15824|3|940| +2450822|15826|3|293| +2450822|15829|3|583| +2450822|15830|3|423| +2450822|15832|3|840| +2450822|15835|3|899| +2450822|15836|3|478| +2450822|15838|3|370| +2450822|15841|3|978| +2450822|15842|3|962| +2450822|15844|3|322| +2450822|15847|3|118| +2450822|15848|3|521| +2450822|15850|3|354| +2450822|15853|3|624| +2450822|15854|3|731| +2450822|15856|3|847| +2450822|15859|3|587| +2450822|15860|3|| +2450822|15862|3|677| +2450822|15865|3|502| +2450822|15866|3|316| +2450822|15868|3|851| +2450822|15871|3|917| +2450822|15872|3|325| +2450822|15874|3|527| +2450822|15877|3|| +2450822|15878|3|306| +2450822|15880|3|51| +2450822|15883|3|824| +2450822|15884|3|176| +2450822|15886|3|328| +2450822|15889|3|306| +2450822|15890|3|182| +2450822|15892|3|| +2450822|15895|3|721| +2450822|15896|3|51| +2450822|15898|3|98| +2450822|15901|3|538| +2450822|15902|3|736| +2450822|15904|3|110| +2450822|15907|3|236| +2450822|15908|3|887| +2450822|15910|3|491| +2450822|15913|3|121| +2450822|15914|3|431| +2450822|15916|3|477| +2450822|15919|3|265| +2450822|15920|3|802| +2450822|15922|3|800| +2450822|15925|3|214| +2450822|15926|3|365| +2450822|15928|3|251| +2450822|15931|3|670| +2450822|15932|3|209| +2450822|15934|3|84| +2450822|15937|3|653| +2450822|15938|3|443| +2450822|15940|3|267| +2450822|15943|3|721| +2450822|15944|3|17| +2450822|15946|3|768| +2450822|15949|3|317| +2450822|15950|3|765| +2450822|15952|3|291| +2450822|15955|3|281| +2450822|15956|3|356| +2450822|15958|3|88| +2450822|15961|3|104| +2450822|15962|3|762| +2450822|15964|3|947| +2450822|15967|3|917| +2450822|15968|3|252| +2450822|15970|3|118| +2450822|15973|3|979| +2450822|15974|3|| +2450822|15976|3|739| +2450822|15979|3|194| +2450822|15980|3|266| +2450822|15982|3|711| +2450822|15985|3|339| +2450822|15986|3|934| +2450822|15988|3|553| +2450822|15991|3|589| +2450822|15992|3|373| +2450822|15994|3|437| +2450822|15997|3|173| +2450822|15998|3|697| +2450822|16000|3|348| +2450822|16003|3|972| +2450822|16004|3|663| +2450822|16006|3|834| +2450822|16009|3|57| +2450822|16010|3|642| +2450822|16012|3|375| +2450822|16015|3|600| +2450822|16016|3|194| +2450822|16018|3|923| +2450822|16021|3|454| +2450822|16022|3|511| +2450822|16024|3|154| +2450822|16027|3|190| +2450822|16028|3|454| +2450822|16030|3|285| +2450822|16033|3|90| +2450822|16034|3|752| +2450822|16036|3|443| +2450822|16039|3|284| +2450822|16040|3|200| +2450822|16042|3|754| +2450822|16045|3|957| +2450822|16046|3|| +2450822|16048|3|731| +2450822|16051|3|602| +2450822|16052|3|862| +2450822|16054|3|454| +2450822|16057|3|420| +2450822|16058|3|857| +2450822|16060|3|356| +2450822|16063|3|241| +2450822|16064|3|393| +2450822|16066|3|599| +2450822|16069|3|815| +2450822|16070|3|780| +2450822|16072|3|666| +2450822|16075|3|83| +2450822|16076|3|440| +2450822|16078|3|20| +2450822|16081|3|856| +2450822|16082|3|531| +2450822|16084|3|615| +2450822|16087|3|350| +2450822|16088|3|650| +2450822|16090|3|911| +2450822|16093|3|634| +2450822|16094|3|391| +2450822|16096|3|118| +2450822|16099|3|316| +2450822|16100|3|564| +2450822|16102|3|932| +2450822|16105|3|987| +2450822|16106|3|474| +2450822|16108|3|720| +2450822|16111|3|63| +2450822|16112|3|| +2450822|16114|3|658| +2450822|16117|3|267| +2450822|16118|3|326| +2450822|16120|3|327| +2450822|16123|3|370| +2450822|16124|3|388| +2450822|16126|3|126| +2450822|16129|3|137| +2450822|16130|3|6| +2450822|16132|3|966| +2450822|16135|3|494| +2450822|16136|3|742| +2450822|16138|3|159| +2450822|16141|3|469| +2450822|16142|3|986| +2450822|16144|3|259| +2450822|16147|3|198| +2450822|16148|3|| +2450822|16150|3|140| +2450822|16153|3|440| +2450822|16154|3|54| +2450822|16156|3|686| +2450822|16159|3|829| +2450822|16160|3|0| +2450822|16162|3|747| +2450822|16165|3|24| +2450822|16166|3|547| +2450822|16168|3|114| +2450822|16171|3|910| +2450822|16172|3|120| +2450822|16174|3|694| +2450822|16177|3|194| +2450822|16178|3|361| +2450822|16180|3|894| +2450822|16183|3|372| +2450822|16184|3|127| +2450822|16186|3|530| +2450822|16189|3|572| +2450822|16190|3|182| +2450822|16192|3|666| +2450822|16195|3|807| +2450822|16196|3|825| +2450822|16198|3|80| +2450822|16201|3|200| +2450822|16202|3|234| +2450822|16204|3|788| +2450822|16207|3|710| +2450822|16208|3|711| +2450822|16210|3|501| +2450822|16213|3|304| +2450822|16214|3|410| +2450822|16216|3|798| +2450822|16219|3|294| +2450822|16220|3|428| +2450822|16222|3|554| +2450822|16225|3|330| +2450822|16226|3|843| +2450822|16228|3|637| +2450822|16231|3|873| +2450822|16232|3|925| +2450822|16234|3|237| +2450822|16237|3|915| +2450822|16238|3|569| +2450822|16240|3|292| +2450822|16243|3|909| +2450822|16244|3|592| +2450822|16246|3|104| +2450822|16249|3|1000| +2450822|16250|3|687| +2450822|16252|3|258| +2450822|16255|3|232| +2450822|16256|3|680| +2450822|16258|3|78| +2450822|16261|3|251| +2450822|16262|3|454| +2450822|16264|3|739| +2450822|16267|3|511| +2450822|16268|3|948| +2450822|16270|3|7| +2450822|16273|3|| +2450822|16274|3|| +2450822|16276|3|479| +2450822|16279|3|650| +2450822|16280|3|| +2450822|16282|3|208| +2450822|16285|3|280| +2450822|16286|3|662| +2450822|16288|3|| +2450822|16291|3|612| +2450822|16292|3|825| +2450822|16294|3|742| +2450822|16297|3|628| +2450822|16298|3|343| +2450822|16300|3|687| +2450822|16303|3|938| +2450822|16304|3|916| +2450822|16306|3|327| +2450822|16309|3|23| +2450822|16310|3|400| +2450822|16312|3|211| +2450822|16315|3|| +2450822|16316|3|38| +2450822|16318|3|90| +2450822|16321|3|609| +2450822|16322|3|321| +2450822|16324|3|637| +2450822|16327|3|517| +2450822|16328|3|10| +2450822|16330|3|311| +2450822|16333|3|185| +2450822|16334|3|707| +2450822|16336|3|770| +2450822|16339|3|259| +2450822|16340|3|859| +2450822|16342|3|| +2450822|16345|3|824| +2450822|16346|3|416| +2450822|16348|3|976| +2450822|16351|3|433| +2450822|16352|3|837| +2450822|16354|3|854| +2450822|16357|3|| +2450822|16358|3|238| +2450822|16360|3|200| +2450822|16363|3|21| +2450822|16364|3|545| +2450822|16366|3|178| +2450822|16369|3|190| +2450822|16370|3|| +2450822|16372|3|992| +2450822|16375|3|| +2450822|16376|3|429| +2450822|16378|3|206| +2450822|16381|3|293| +2450822|16382|3|144| +2450822|16384|3|95| +2450822|16387|3|62| +2450822|16388|3|190| +2450822|16390|3|21| +2450822|16393|3|749| +2450822|16394|3|558| +2450822|16396|3|653| +2450822|16399|3|703| +2450822|16400|3|505| +2450822|16402|3|349| +2450822|16405|3|279| +2450822|16406|3|979| +2450822|16408|3|67| +2450822|16411|3|946| +2450822|16412|3|469| +2450822|16414|3|767| +2450822|16417|3|969| +2450822|16418|3|863| +2450822|16420|3|399| +2450822|16423|3|| +2450822|16424|3|387| +2450822|16426|3|255| +2450822|16429|3|479| +2450822|16430|3|316| +2450822|16432|3|416| +2450822|16435|3|406| +2450822|16436|3|692| +2450822|16438|3|746| +2450822|16441|3|502| +2450822|16442|3|279| +2450822|16444|3|| +2450822|16447|3|84| +2450822|16448|3|124| +2450822|16450|3|270| +2450822|16453|3|324| +2450822|16454|3|409| +2450822|16456|3|416| +2450822|16459|3|487| +2450822|16460|3|823| +2450822|16462|3|381| +2450822|16465|3|307| +2450822|16466|3|103| +2450822|16468|3|691| +2450822|16471|3|547| +2450822|16472|3|292| +2450822|16474|3|467| +2450822|16477|3|| +2450822|16478|3|38| +2450822|16480|3|830| +2450822|16483|3|448| +2450822|16484|3|338| +2450822|16486|3|497| +2450822|16489|3|505| +2450822|16490|3|240| +2450822|16492|3|196| +2450822|16495|3|445| +2450822|16496|3|353| +2450822|16498|3|199| +2450822|16501|3|301| +2450822|16502|3|215| +2450822|16504|3|406| +2450822|16507|3|635| +2450822|16508|3|562| +2450822|16510|3|264| +2450822|16513|3|140| +2450822|16514|3|752| +2450822|16516|3|262| +2450822|16519|3|290| +2450822|16520|3|| +2450822|16522|3|337| +2450822|16525|3|896| +2450822|16526|3|| +2450822|16528|3|333| +2450822|16531|3|307| +2450822|16532|3|643| +2450822|16534|3|635| +2450822|16537|3|28| +2450822|16538|3|35| +2450822|16540|3|174| +2450822|16543|3|916| +2450822|16544|3|911| +2450822|16546|3|236| +2450822|16549|3|73| +2450822|16550|3|294| +2450822|16552|3|173| +2450822|16555|3|192| +2450822|16556|3|968| +2450822|16558|3|273| +2450822|16561|3|68| +2450822|16562|3|824| +2450822|16564|3|801| +2450822|16567|3|0| +2450822|16568|3|614| +2450822|16570|3|193| +2450822|16573|3|602| +2450822|16574|3|145| +2450822|16576|3|628| +2450822|16579|3|140| +2450822|16580|3|113| +2450822|16582|3|255| +2450822|16585|3|796| +2450822|16586|3|988| +2450822|16588|3|355| +2450822|16591|3|184| +2450822|16592|3|904| +2450822|16594|3|806| +2450822|16597|3|976| +2450822|16598|3|431| +2450822|16600|3|217| +2450822|16603|3|820| +2450822|16604|3|794| +2450822|16606|3|476| +2450822|16609|3|43| +2450822|16610|3|879| +2450822|16612|3|404| +2450822|16615|3|139| +2450822|16616|3|484| +2450822|16618|3|920| +2450822|16621|3|265| +2450822|16622|3|63| +2450822|16624|3|436| +2450822|16627|3|1000| +2450822|16628|3|964| +2450822|16630|3|379| +2450822|16633|3|118| +2450822|16634|3|736| +2450822|16636|3|426| +2450822|16639|3|785| +2450822|16640|3|415| +2450822|16642|3|669| +2450822|16645|3|533| +2450822|16646|3|803| +2450822|16648|3|333| +2450822|16651|3|| +2450822|16652|3|| +2450822|16654|3|231| +2450822|16657|3|| +2450822|16658|3|541| +2450822|16660|3|982| +2450822|16663|3|664| +2450822|16664|3|296| +2450822|16666|3|478| +2450822|16669|3|50| +2450822|16670|3|638| +2450822|16672|3|542| +2450822|16675|3|611| +2450822|16676|3|883| +2450822|16678|3|710| +2450822|16681|3|277| +2450822|16682|3|664| +2450822|16684|3|793| +2450822|16687|3|871| +2450822|16688|3|497| +2450822|16690|3|9| +2450822|16693|3|929| +2450822|16694|3|237| +2450822|16696|3|11| +2450822|16699|3|259| +2450822|16700|3|424| +2450822|16702|3|39| +2450822|16705|3|152| +2450822|16706|3|266| +2450822|16708|3|780| +2450822|16711|3|347| +2450822|16712|3|408| +2450822|16714|3|733| +2450822|16717|3|77| +2450822|16718|3|457| +2450822|16720|3|180| +2450822|16723|3|730| +2450822|16724|3|644| +2450822|16726|3|100| +2450822|16729|3|777| +2450822|16730|3|183| +2450822|16732|3|4| +2450822|16735|3|500| +2450822|16736|3|329| +2450822|16738|3|49| +2450822|16741|3|103| +2450822|16742|3|| +2450822|16744|3|313| +2450822|16747|3|151| +2450822|16748|3|620| +2450822|16750|3|106| +2450822|16753|3|466| +2450822|16754|3|688| +2450822|16756|3|425| +2450822|16759|3|667| +2450822|16760|3|581| +2450822|16762|3|442| +2450822|16765|3|504| +2450822|16766|3|| +2450822|16768|3|884| +2450822|16771|3|| +2450822|16772|3|459| +2450822|16774|3|755| +2450822|16777|3|622| +2450822|16778|3|868| +2450822|16780|3|| +2450822|16783|3|774| +2450822|16784|3|545| +2450822|16786|3|76| +2450822|16789|3|| +2450822|16790|3|650| +2450822|16792|3|165| +2450822|16795|3|246| +2450822|16796|3|349| +2450822|16798|3|881| +2450822|16801|3|619| +2450822|16802|3|594| +2450822|16804|3|251| +2450822|16807|3|234| +2450822|16808|3|461| +2450822|16810|3|996| +2450822|16813|3|352| +2450822|16814|3|148| +2450822|16816|3|303| +2450822|16819|3|906| +2450822|16820|3|570| +2450822|16822|3|4| +2450822|16825|3|876| +2450822|16826|3|594| +2450822|16828|3|667| +2450822|16831|3|381| +2450822|16832|3|173| +2450822|16834|3|418| +2450822|16837|3|69| +2450822|16838|3|561| +2450822|16840|3|687| +2450822|16843|3|615| +2450822|16844|3|150| +2450822|16846|3|866| +2450822|16849|3|479| +2450822|16850|3|534| +2450822|16852|3|611| +2450822|16855|3|| +2450822|16856|3|664| +2450822|16858|3|724| +2450822|16861|3|| +2450822|16862|3|556| +2450822|16864|3|858| +2450822|16867|3|326| +2450822|16868|3|646| +2450822|16870|3|572| +2450822|16873|3|709| +2450822|16874|3|746| +2450822|16876|3|47| +2450822|16879|3|384| +2450822|16880|3|327| +2450822|16882|3|837| +2450822|16885|3|974| +2450822|16886|3|883| +2450822|16888|3|179| +2450822|16891|3|896| +2450822|16892|3|544| +2450822|16894|3|599| +2450822|16897|3|69| +2450822|16898|3|452| +2450822|16900|3|589| +2450822|16903|3|235| +2450822|16904|3|363| +2450822|16906|3|942| +2450822|16909|3|9| +2450822|16910|3|573| +2450822|16912|3|901| +2450822|16915|3|520| +2450822|16916|3|| +2450822|16918|3|987| +2450822|16921|3|62| +2450822|16922|3|655| +2450822|16924|3|825| +2450822|16927|3|713| +2450822|16928|3|574| +2450822|16930|3|44| +2450822|16933|3|703| +2450822|16934|3|929| +2450822|16936|3|| +2450822|16939|3|970| +2450822|16940|3|221| +2450822|16942|3|413| +2450822|16945|3|949| +2450822|16946|3|956| +2450822|16948|3|181| +2450822|16951|3|971| +2450822|16952|3|95| +2450822|16954|3|85| +2450822|16957|3|885| +2450822|16958|3|381| +2450822|16960|3|438| +2450822|16963|3|871| +2450822|16964|3|962| +2450822|16966|3|954| +2450822|16969|3|549| +2450822|16970|3|261| +2450822|16972|3|246| +2450822|16975|3|621| +2450822|16976|3|3| +2450822|16978|3|503| +2450822|16981|3|151| +2450822|16982|3|8| +2450822|16984|3|458| +2450822|16987|3|811| +2450822|16988|3|501| +2450822|16990|3|957| +2450822|16993|3|619| +2450822|16994|3|239| +2450822|16996|3|162| +2450822|16999|3|130| +2450822|17000|3|655| +2450822|17002|3|702| +2450822|17005|3|| +2450822|17006|3|512| +2450822|17008|3|744| +2450822|17011|3|842| +2450822|17012|3|795| +2450822|17014|3|888| +2450822|17017|3|790| +2450822|17018|3|874| +2450822|17020|3|872| +2450822|17023|3|296| +2450822|17024|3|227| +2450822|17026|3|109| +2450822|17029|3|109| +2450822|17030|3|| +2450822|17032|3|879| +2450822|17035|3|300| +2450822|17036|3|637| +2450822|17038|3|159| +2450822|17041|3|845| +2450822|17042|3|456| +2450822|17044|3|626| +2450822|17047|3|176| +2450822|17048|3|769| +2450822|17050|3|888| +2450822|17053|3|348| +2450822|17054|3|349| +2450822|17056|3|88| +2450822|17059|3|266| +2450822|17060|3|661| +2450822|17062|3|101| +2450822|17065|3|771| +2450822|17066|3|874| +2450822|17068|3|275| +2450822|17071|3|717| +2450822|17072|3|212| +2450822|17074|3|| +2450822|17077|3|578| +2450822|17078|3|690| +2450822|17080|3|25| +2450822|17083|3|91| +2450822|17084|3|479| +2450822|17086|3|511| +2450822|17089|3|17| +2450822|17090|3|58| +2450822|17092|3|648| +2450822|17095|3|640| +2450822|17096|3|| +2450822|17098|3|515| +2450822|17101|3|907| +2450822|17102|3|719| +2450822|17104|3|32| +2450822|17107|3|943| +2450822|17108|3|483| +2450822|17110|3|325| +2450822|17113|3|718| +2450822|17114|3|| +2450822|17116|3|128| +2450822|17119|3|159| +2450822|17120|3|396| +2450822|17122|3|921| +2450822|17125|3|941| +2450822|17126|3|522| +2450822|17128|3|415| +2450822|17131|3|425| +2450822|17132|3|441| +2450822|17134|3|260| +2450822|17137|3|284| +2450822|17138|3|922| +2450822|17140|3|203| +2450822|17143|3|5| +2450822|17144|3|651| +2450822|17146|3|97| +2450822|17149|3|657| +2450822|17150|3|616| +2450822|17152|3|541| +2450822|17155|3|699| +2450822|17156|3|346| +2450822|17158|3|739| +2450822|17161|3|478| +2450822|17162|3|540| +2450822|17164|3|247| +2450822|17167|3|59| +2450822|17168|3|90| +2450822|17170|3|363| +2450822|17173|3|617| +2450822|17174|3|605| +2450822|17176|3|845| +2450822|17179|3|578| +2450822|17180|3|447| +2450822|17182|3|119| +2450822|17185|3|981| +2450822|17186|3|826| +2450822|17188|3|59| +2450822|17191|3|730| +2450822|17192|3|133| +2450822|17194|3|834| +2450822|17197|3|73| +2450822|17198|3|805| +2450822|17200|3|764| +2450822|17203|3|769| +2450822|17204|3|377| +2450822|17206|3|897| +2450822|17209|3|970| +2450822|17210|3|405| +2450822|17212|3|935| +2450822|17215|3|698| +2450822|17216|3|255| +2450822|17218|3|628| +2450822|17221|3|571| +2450822|17222|3|543| +2450822|17224|3|244| +2450822|17227|3|672| +2450822|17228|3|65| +2450822|17230|3|| +2450822|17233|3|207| +2450822|17234|3|510| +2450822|17236|3|204| +2450822|17239|3|166| +2450822|17240|3|789| +2450822|17242|3|80| +2450822|17245|3|830| +2450822|17246|3|640| +2450822|17248|3|635| +2450822|17251|3|420| +2450822|17252|3|356| +2450822|17254|3|26| +2450822|17257|3|705| +2450822|17258|3|217| +2450822|17260|3|250| +2450822|17263|3|870| +2450822|17264|3|93| +2450822|17266|3|196| +2450822|17269|3|950| +2450822|17270|3|808| +2450822|17272|3|251| +2450822|17275|3|320| +2450822|17276|3|397| +2450822|17278|3|324| +2450822|17281|3|85| +2450822|17282|3|820| +2450822|17284|3|616| +2450822|17287|3|720| +2450822|17288|3|651| +2450822|17290|3|677| +2450822|17293|3|152| +2450822|17294|3|87| +2450822|17296|3|473| +2450822|17299|3|835| +2450822|17300|3|590| +2450822|17302|3|733| +2450822|17305|3|888| +2450822|17306|3|705| +2450822|17308|3|692| +2450822|17311|3|139| +2450822|17312|3|884| +2450822|17314|3|775| +2450822|17317|3|37| +2450822|17318|3|458| +2450822|17320|3|78| +2450822|17323|3|510| +2450822|17324|3|200| +2450822|17326|3|913| +2450822|17329|3|472| +2450822|17330|3|56| +2450822|17332|3|845| +2450822|17335|3|652| +2450822|17336|3|961| +2450822|17338|3|697| +2450822|17341|3|179| +2450822|17342|3|82| +2450822|17344|3|52| +2450822|17347|3|488| +2450822|17348|3|80| +2450822|17350|3|984| +2450822|17353|3|239| +2450822|17354|3|913| +2450822|17356|3|778| +2450822|17359|3|25| +2450822|17360|3|397| +2450822|17362|3|778| +2450822|17365|3|614| +2450822|17366|3|633| +2450822|17368|3|386| +2450822|17371|3|122| +2450822|17372|3|816| +2450822|17374|3|766| +2450822|17377|3|772| +2450822|17378|3|108| +2450822|17380|3|779| +2450822|17383|3|128| +2450822|17384|3|817| +2450822|17386|3|481| +2450822|17389|3|346| +2450822|17390|3|459| +2450822|17392|3|71| +2450822|17395|3|11| +2450822|17396|3|765| +2450822|17398|3|460| +2450822|17401|3|162| +2450822|17402|3|825| +2450822|17404|3|383| +2450822|17407|3|248| +2450822|17408|3|294| +2450822|17410|3|361| +2450822|17413|3|180| +2450822|17414|3|190| +2450822|17416|3|922| +2450822|17419|3|755| +2450822|17420|3|48| +2450822|17422|3|461| +2450822|17425|3|102| +2450822|17426|3|744| +2450822|17428|3|587| +2450822|17431|3|555| +2450822|17432|3|926| +2450822|17434|3|377| +2450822|17437|3|740| +2450822|17438|3|328| +2450822|17440|3|816| +2450822|17443|3|950| +2450822|17444|3|891| +2450822|17446|3|449| +2450822|17449|3|639| +2450822|17450|3|820| +2450822|17452|3|225| +2450822|17455|3|359| +2450822|17456|3|196| +2450822|17458|3|221| +2450822|17461|3|| +2450822|17462|3|2| +2450822|17464|3|578| +2450822|17467|3|286| +2450822|17468|3|22| +2450822|17470|3|665| +2450822|17473|3|| +2450822|17474|3|294| +2450822|17476|3|663| +2450822|17479|3|930| +2450822|17480|3|753| +2450822|17482|3|292| +2450822|17485|3|431| +2450822|17486|3|289| +2450822|17488|3|604| +2450822|17491|3|152| +2450822|17492|3|546| +2450822|17494|3|654| +2450822|17497|3|713| +2450822|17498|3|672| +2450822|17500|3|939| +2450822|17503|3|| +2450822|17504|3|619| +2450822|17506|3|132| +2450822|17509|3|776| +2450822|17510|3|852| +2450822|17512|3|48| +2450822|17515|3|977| +2450822|17516|3|106| +2450822|17518|3|| +2450822|17521|3|158| +2450822|17522|3|237| +2450822|17524|3|725| +2450822|17527|3|| +2450822|17528|3|143| +2450822|17530|3|497| +2450822|17533|3|612| +2450822|17534|3|660| +2450822|17536|3|834| +2450822|17539|3|675| +2450822|17540|3|50| +2450822|17542|3|897| +2450822|17545|3|592| +2450822|17546|3|39| +2450822|17548|3|158| +2450822|17551|3|101| +2450822|17552|3|725| +2450822|17554|3|432| +2450822|17557|3|440| +2450822|17558|3|565| +2450822|17560|3|131| +2450822|17563|3|499| +2450822|17564|3|574| +2450822|17566|3|901| +2450822|17569|3|491| +2450822|17570|3|181| +2450822|17572|3|861| +2450822|17575|3|188| +2450822|17576|3|225| +2450822|17578|3|413| +2450822|17581|3|540| +2450822|17582|3|712| +2450822|17584|3|515| +2450822|17587|3|88| +2450822|17588|3|385| +2450822|17590|3|169| +2450822|17593|3|| +2450822|17594|3|839| +2450822|17596|3|142| +2450822|17599|3|| +2450822|17600|3|51| +2450822|17602|3|415| +2450822|17605|3|710| +2450822|17606|3|418| +2450822|17608|3|863| +2450822|17611|3|| +2450822|17612|3|127| +2450822|17614|3|597| +2450822|17617|3|925| +2450822|17618|3|819| +2450822|17620|3|99| +2450822|17623|3|408| +2450822|17624|3|588| +2450822|17626|3|53| +2450822|17629|3|939| +2450822|17630|3|990| +2450822|17632|3|184| +2450822|17635|3|165| +2450822|17636|3|474| +2450822|17638|3|340| +2450822|17641|3|150| +2450822|17642|3|405| +2450822|17644|3|302| +2450822|17647|3|376| +2450822|17648|3|891| +2450822|17650|3|397| +2450822|17653|3|872| +2450822|17654|3|| +2450822|17656|3|398| +2450822|17659|3|2| +2450822|17660|3|840| +2450822|17662|3|810| +2450822|17665|3|55| +2450822|17666|3|767| +2450822|17668|3|673| +2450822|17671|3|643| +2450822|17672|3|173| +2450822|17674|3|484| +2450822|17677|3|177| +2450822|17678|3|632| +2450822|17680|3|150| +2450822|17683|3|6| +2450822|17684|3|19| +2450822|17686|3|824| +2450822|17689|3|758| +2450822|17690|3|369| +2450822|17692|3|325| +2450822|17695|3|541| +2450822|17696|3|164| +2450822|17698|3|464| +2450822|17701|3|979| +2450822|17702|3|543| +2450822|17704|3|169| +2450822|17707|3|458| +2450822|17708|3|510| +2450822|17710|3|163| +2450822|17713|3|326| +2450822|17714|3|670| +2450822|17716|3|276| +2450822|17719|3|542| +2450822|17720|3|1000| +2450822|17722|3|767| +2450822|17725|3|548| +2450822|17726|3|957| +2450822|17728|3|421| +2450822|17731|3|630| +2450822|17732|3|856| +2450822|17734|3|772| +2450822|17737|3|846| +2450822|17738|3|471| +2450822|17740|3|805| +2450822|17743|3|645| +2450822|17744|3|289| +2450822|17746|3|228| +2450822|17749|3|967| +2450822|17750|3|336| +2450822|17752|3|486| +2450822|17755|3|935| +2450822|17756|3|927| +2450822|17758|3|830| +2450822|17761|3|610| +2450822|17762|3|198| +2450822|17764|3|866| +2450822|17767|3|359| +2450822|17768|3|198| +2450822|17770|3|581| +2450822|17773|3|324| +2450822|17774|3|437| +2450822|17776|3|155| +2450822|17779|3|537| +2450822|17780|3|716| +2450822|17782|3|48| +2450822|17785|3|845| +2450822|17786|3|345| +2450822|17788|3|557| +2450822|17791|3|721| +2450822|17792|3|177| +2450822|17794|3|310| +2450822|17797|3|247| +2450822|17798|3|241| +2450822|17800|3|| +2450822|17803|3|924| +2450822|17804|3|122| +2450822|17806|3|590| +2450822|17809|3|884| +2450822|17810|3|985| +2450822|17812|3|524| +2450822|17815|3|512| +2450822|17816|3|854| +2450822|17818|3|356| +2450822|17821|3|29| +2450822|17822|3|467| +2450822|17824|3|388| +2450822|17827|3|282| +2450822|17828|3|444| +2450822|17830|3|860| +2450822|17833|3|149| +2450822|17834|3|925| +2450822|17836|3|717| +2450822|17839|3|112| +2450822|17840|3|612| +2450822|17842|3|| +2450822|17845|3|57| +2450822|17846|3|950| +2450822|17848|3|| +2450822|17851|3|300| +2450822|17852|3|473| +2450822|17854|3|31| +2450822|17857|3|800| +2450822|17858|3|487| +2450822|17860|3|410| +2450822|17863|3|222| +2450822|17864|3|683| +2450822|17866|3|680| +2450822|17869|3|| +2450822|17870|3|686| +2450822|17872|3|74| +2450822|17875|3|491| +2450822|17876|3|302| +2450822|17878|3|597| +2450822|17881|3|852| +2450822|17882|3|412| +2450822|17884|3|136| +2450822|17887|3|537| +2450822|17888|3|122| +2450822|17890|3|| +2450822|17893|3|452| +2450822|17894|3|292| +2450822|17896|3|601| +2450822|17899|3|381| +2450822|17900|3|848| +2450822|17902|3|200| +2450822|17905|3|969| +2450822|17906|3|331| +2450822|17908|3|490| +2450822|17911|3|780| +2450822|17912|3|437| +2450822|17914|3|344| +2450822|17917|3|278| +2450822|17918|3|21| +2450822|17920|3|734| +2450822|17923|3|711| +2450822|17924|3|46| +2450822|17926|3|884| +2450822|17929|3|242| +2450822|17930|3|866| +2450822|17932|3|697| +2450822|17935|3|985| +2450822|17936|3|456| +2450822|17938|3|237| +2450822|17941|3|876| +2450822|17942|3|457| +2450822|17944|3|433| +2450822|17947|3|811| +2450822|17948|3|554| +2450822|17950|3|776| +2450822|17953|3|371| +2450822|17954|3|909| +2450822|17956|3|280| +2450822|17959|3|555| +2450822|17960|3|688| +2450822|17962|3|806| +2450822|17965|3|530| +2450822|17966|3|374| +2450822|17968|3|271| +2450822|17971|3|591| +2450822|17972|3|373| +2450822|17974|3|180| +2450822|17977|3|531| +2450822|17978|3|941| +2450822|17980|3|460| +2450822|17983|3|613| +2450822|17984|3|14| +2450822|17986|3|564| +2450822|17989|3|445| +2450822|17990|3|724| +2450822|17992|3|742| +2450822|17995|3|691| +2450822|17996|3|289| +2450822|17998|3|985| +2450822|1|4|636| +2450822|2|4|916| +2450822|4|4|460| +2450822|7|4|400| +2450822|8|4|870| +2450822|10|4|218| +2450822|13|4|713| +2450822|14|4|996| +2450822|16|4|817| +2450822|19|4|494| +2450822|20|4|517| +2450822|22|4|686| +2450822|25|4|| +2450822|26|4|278| +2450822|28|4|810| +2450822|31|4|927| +2450822|32|4|103| +2450822|34|4|570| +2450822|37|4|919| +2450822|38|4|758| +2450822|40|4|875| +2450822|43|4|724| +2450822|44|4|476| +2450822|46|4|| +2450822|49|4|289| +2450822|50|4|921| +2450822|52|4|638| +2450822|55|4|9| +2450822|56|4|964| +2450822|58|4|424| +2450822|61|4|826| +2450822|62|4|579| +2450822|64|4|131| +2450822|67|4|500| +2450822|68|4|881| +2450822|70|4|555| +2450822|73|4|100| +2450822|74|4|358| +2450822|76|4|584| +2450822|79|4|547| +2450822|80|4|123| +2450822|82|4|86| +2450822|85|4|583| +2450822|86|4|329| +2450822|88|4|800| +2450822|91|4|805| +2450822|92|4|818| +2450822|94|4|773| +2450822|97|4|65| +2450822|98|4|104| +2450822|100|4|882| +2450822|103|4|9| +2450822|104|4|| +2450822|106|4|683| +2450822|109|4|161| +2450822|110|4|711| +2450822|112|4|236| +2450822|115|4|565| +2450822|116|4|156| +2450822|118|4|397| +2450822|121|4|327| +2450822|122|4|385| +2450822|124|4|799| +2450822|127|4|416| +2450822|128|4|429| +2450822|130|4|659| +2450822|133|4|871| +2450822|134|4|431| +2450822|136|4|968| +2450822|139|4|500| +2450822|140|4|931| +2450822|142|4|841| +2450822|145|4|959| +2450822|146|4|331| +2450822|148|4|558| +2450822|151|4|929| +2450822|152|4|756| +2450822|154|4|35| +2450822|157|4|557| +2450822|158|4|61| +2450822|160|4|201| +2450822|163|4|590| +2450822|164|4|417| +2450822|166|4|284| +2450822|169|4|558| +2450822|170|4|678| +2450822|172|4|211| +2450822|175|4|797| +2450822|176|4|871| +2450822|178|4|393| +2450822|181|4|137| +2450822|182|4|964| +2450822|184|4|687| +2450822|187|4|315| +2450822|188|4|214| +2450822|190|4|| +2450822|193|4|955| +2450822|194|4|41| +2450822|196|4|703| +2450822|199|4|849| +2450822|200|4|963| +2450822|202|4|467| +2450822|205|4|402| +2450822|206|4|886| +2450822|208|4|320| +2450822|211|4|745| +2450822|212|4|73| +2450822|214|4|712| +2450822|217|4|66| +2450822|218|4|841| +2450822|220|4|435| +2450822|223|4|906| +2450822|224|4|467| +2450822|226|4|| +2450822|229|4|17| +2450822|230|4|392| +2450822|232|4|170| +2450822|235|4|| +2450822|236|4|386| +2450822|238|4|537| +2450822|241|4|293| +2450822|242|4|590| +2450822|244|4|958| +2450822|247|4|389| +2450822|248|4|308| +2450822|250|4|852| +2450822|253|4|457| +2450822|254|4|78| +2450822|256|4|159| +2450822|259|4|12| +2450822|260|4|272| +2450822|262|4|561| +2450822|265|4|152| +2450822|266|4|590| +2450822|268|4|236| +2450822|271|4|553| +2450822|272|4|577| +2450822|274|4|903| +2450822|277|4|321| +2450822|278|4|375| +2450822|280|4|145| +2450822|283|4|146| +2450822|284|4|458| +2450822|286|4|8| +2450822|289|4|355| +2450822|290|4|111| +2450822|292|4|899| +2450822|295|4|851| +2450822|296|4|| +2450822|298|4|542| +2450822|301|4|916| +2450822|302|4|869| +2450822|304|4|742| +2450822|307|4|733| +2450822|308|4|715| +2450822|310|4|834| +2450822|313|4|571| +2450822|314|4|278| +2450822|316|4|| +2450822|319|4|467| +2450822|320|4|881| +2450822|322|4|975| +2450822|325|4|704| +2450822|326|4|170| +2450822|328|4|457| +2450822|331|4|505| +2450822|332|4|80| +2450822|334|4|116| +2450822|337|4|143| +2450822|338|4|| +2450822|340|4|5| +2450822|343|4|79| +2450822|344|4|722| +2450822|346|4|959| +2450822|349|4|878| +2450822|350|4|421| +2450822|352|4|112| +2450822|355|4|794| +2450822|356|4|2| +2450822|358|4|88| +2450822|361|4|146| +2450822|362|4|802| +2450822|364|4|557| +2450822|367|4|450| +2450822|368|4|151| +2450822|370|4|379| +2450822|373|4|945| +2450822|374|4|262| +2450822|376|4|964| +2450822|379|4|837| +2450822|380|4|283| +2450822|382|4|244| +2450822|385|4|308| +2450822|386|4|597| +2450822|388|4|481| +2450822|391|4|78| +2450822|392|4|378| +2450822|394|4|742| +2450822|397|4|979| +2450822|398|4|6| +2450822|400|4|110| +2450822|403|4|| +2450822|404|4|853| +2450822|406|4|670| +2450822|409|4|1000| +2450822|410|4|731| +2450822|412|4|199| +2450822|415|4|227| +2450822|416|4|778| +2450822|418|4|254| +2450822|421|4|449| +2450822|422|4|38| +2450822|424|4|405| +2450822|427|4|651| +2450822|428|4|527| +2450822|430|4|102| +2450822|433|4|817| +2450822|434|4|324| +2450822|436|4|778| +2450822|439|4|9| +2450822|440|4|4| +2450822|442|4|80| +2450822|445|4|818| +2450822|446|4|852| +2450822|448|4|211| +2450822|451|4|579| +2450822|452|4|194| +2450822|454|4|967| +2450822|457|4|738| +2450822|458|4|673| +2450822|460|4|262| +2450822|463|4|387| +2450822|464|4|772| +2450822|466|4|477| +2450822|469|4|356| +2450822|470|4|20| +2450822|472|4|190| +2450822|475|4|136| +2450822|476|4|833| +2450822|478|4|477| +2450822|481|4|646| +2450822|482|4|11| +2450822|484|4|882| +2450822|487|4|274| +2450822|488|4|690| +2450822|490|4|844| +2450822|493|4|234| +2450822|494|4|281| +2450822|496|4|429| +2450822|499|4|6| +2450822|500|4|727| +2450822|502|4|364| +2450822|505|4|525| +2450822|506|4|931| +2450822|508|4|625| +2450822|511|4|392| +2450822|512|4|457| +2450822|514|4|436| +2450822|517|4|43| +2450822|518|4|799| +2450822|520|4|387| +2450822|523|4|372| +2450822|524|4|888| +2450822|526|4|575| +2450822|529|4|373| +2450822|530|4|463| +2450822|532|4|649| +2450822|535|4|857| +2450822|536|4|6| +2450822|538|4|10| +2450822|541|4|| +2450822|542|4|927| +2450822|544|4|528| +2450822|547|4|871| +2450822|548|4|881| +2450822|550|4|4| +2450822|553|4|166| +2450822|554|4|| +2450822|556|4|884| +2450822|559|4|946| +2450822|560|4|803| +2450822|562|4|233| +2450822|565|4|17| +2450822|566|4|896| +2450822|568|4|414| +2450822|571|4|765| +2450822|572|4|218| +2450822|574|4|130| +2450822|577|4|132| +2450822|578|4|541| +2450822|580|4|54| +2450822|583|4|13| +2450822|584|4|623| +2450822|586|4|532| +2450822|589|4|922| +2450822|590|4|837| +2450822|592|4|652| +2450822|595|4|480| +2450822|596|4|353| +2450822|598|4|44| +2450822|601|4|680| +2450822|602|4|758| +2450822|604|4|781| +2450822|607|4|498| +2450822|608|4|199| +2450822|610|4|| +2450822|613|4|227| +2450822|614|4|497| +2450822|616|4|672| +2450822|619|4|139| +2450822|620|4|98| +2450822|622|4|651| +2450822|625|4|487| +2450822|626|4|382| +2450822|628|4|823| +2450822|631|4|408| +2450822|632|4|433| +2450822|634|4|783| +2450822|637|4|174| +2450822|638|4|680| +2450822|640|4|526| +2450822|643|4|554| +2450822|644|4|875| +2450822|646|4|80| +2450822|649|4|395| +2450822|650|4|563| +2450822|652|4|900| +2450822|655|4|462| +2450822|656|4|268| +2450822|658|4|999| +2450822|661|4|324| +2450822|662|4|23| +2450822|664|4|615| +2450822|667|4|621| +2450822|668|4|12| +2450822|670|4|171| +2450822|673|4|921| +2450822|674|4|495| +2450822|676|4|184| +2450822|679|4|977| +2450822|680|4|866| +2450822|682|4|389| +2450822|685|4|855| +2450822|686|4|636| +2450822|688|4|355| +2450822|691|4|481| +2450822|692|4|663| +2450822|694|4|526| +2450822|697|4|325| +2450822|698|4|916| +2450822|700|4|30| +2450822|703|4|360| +2450822|704|4|927| +2450822|706|4|671| +2450822|709|4|837| +2450822|710|4|136| +2450822|712|4|878| +2450822|715|4|765| +2450822|716|4|189| +2450822|718|4|265| +2450822|721|4|690| +2450822|722|4|951| +2450822|724|4|| +2450822|727|4|871| +2450822|728|4|223| +2450822|730|4|716| +2450822|733|4|861| +2450822|734|4|458| +2450822|736|4|910| +2450822|739|4|690| +2450822|740|4|| +2450822|742|4|171| +2450822|745|4|49| +2450822|746|4|398| +2450822|748|4|512| +2450822|751|4|102| +2450822|752|4|| +2450822|754|4|761| +2450822|757|4|906| +2450822|758|4|436| +2450822|760|4|244| +2450822|763|4|655| +2450822|764|4|408| +2450822|766|4|634| +2450822|769|4|246| +2450822|770|4|394| +2450822|772|4|659| +2450822|775|4|| +2450822|776|4|360| +2450822|778|4|778| +2450822|781|4|959| +2450822|782|4|705| +2450822|784|4|670| +2450822|787|4|532| +2450822|788|4|72| +2450822|790|4|263| +2450822|793|4|32| +2450822|794|4|616| +2450822|796|4|642| +2450822|799|4|327| +2450822|800|4|910| +2450822|802|4|439| +2450822|805|4|687| +2450822|806|4|757| +2450822|808|4|917| +2450822|811|4|188| +2450822|812|4|705| +2450822|814|4|430| +2450822|817|4|129| +2450822|818|4|970| +2450822|820|4|303| +2450822|823|4|419| +2450822|824|4|205| +2450822|826|4|183| +2450822|829|4|318| +2450822|830|4|202| +2450822|832|4|229| +2450822|835|4|419| +2450822|836|4|527| +2450822|838|4|220| +2450822|841|4|961| +2450822|842|4|667| +2450822|844|4|522| +2450822|847|4|611| +2450822|848|4|166| +2450822|850|4|343| +2450822|853|4|899| +2450822|854|4|55| +2450822|856|4|18| +2450822|859|4|575| +2450822|860|4|310| +2450822|862|4|906| +2450822|865|4|157| +2450822|866|4|697| +2450822|868|4|431| +2450822|871|4|155| +2450822|872|4|949| +2450822|874|4|881| +2450822|877|4|820| +2450822|878|4|969| +2450822|880|4|866| +2450822|883|4|287| +2450822|884|4|998| +2450822|886|4|57| +2450822|889|4|479| +2450822|890|4|46| +2450822|892|4|363| +2450822|895|4|633| +2450822|896|4|296| +2450822|898|4|653| +2450822|901|4|558| +2450822|902|4|830| +2450822|904|4|601| +2450822|907|4|557| +2450822|908|4|746| +2450822|910|4|386| +2450822|913|4|958| +2450822|914|4|635| +2450822|916|4|484| +2450822|919|4|847| +2450822|920|4|352| +2450822|922|4|601| +2450822|925|4|479| +2450822|926|4|716| +2450822|928|4|467| +2450822|931|4|879| +2450822|932|4|403| +2450822|934|4|875| +2450822|937|4|789| +2450822|938|4|536| +2450822|940|4|749| +2450822|943|4|392| +2450822|944|4|53| +2450822|946|4|974| +2450822|949|4|279| +2450822|950|4|968| +2450822|952|4|434| +2450822|955|4|205| +2450822|956|4|26| +2450822|958|4|800| +2450822|961|4|865| +2450822|962|4|260| +2450822|964|4|222| +2450822|967|4|638| +2450822|968|4|253| +2450822|970|4|310| +2450822|973|4|923| +2450822|974|4|166| +2450822|976|4|918| +2450822|979|4|431| +2450822|980|4|237| +2450822|982|4|397| +2450822|985|4|66| +2450822|986|4|| +2450822|988|4|264| +2450822|991|4|465| +2450822|992|4|721| +2450822|994|4|852| +2450822|997|4|720| +2450822|998|4|673| +2450822|1000|4|563| +2450822|1003|4|842| +2450822|1004|4|82| +2450822|1006|4|944| +2450822|1009|4|215| +2450822|1010|4|8| +2450822|1012|4|493| +2450822|1015|4|516| +2450822|1016|4|32| +2450822|1018|4|145| +2450822|1021|4|338| +2450822|1022|4|901| +2450822|1024|4|200| +2450822|1027|4|908| +2450822|1028|4|367| +2450822|1030|4|854| +2450822|1033|4|996| +2450822|1034|4|50| +2450822|1036|4|108| +2450822|1039|4|970| +2450822|1040|4|131| +2450822|1042|4|249| +2450822|1045|4|121| +2450822|1046|4|279| +2450822|1048|4|332| +2450822|1051|4|87| +2450822|1052|4|11| +2450822|1054|4|736| +2450822|1057|4|212| +2450822|1058|4|330| +2450822|1060|4|629| +2450822|1063|4|361| +2450822|1064|4|538| +2450822|1066|4|242| +2450822|1069|4|466| +2450822|1070|4|513| +2450822|1072|4|903| +2450822|1075|4|654| +2450822|1076|4|804| +2450822|1078|4|497| +2450822|1081|4|591| +2450822|1082|4|136| +2450822|1084|4|61| +2450822|1087|4|724| +2450822|1088|4|68| +2450822|1090|4|790| +2450822|1093|4|786| +2450822|1094|4|175| +2450822|1096|4|665| +2450822|1099|4|410| +2450822|1100|4|| +2450822|1102|4|612| +2450822|1105|4|332| +2450822|1106|4|57| +2450822|1108|4|841| +2450822|1111|4|39| +2450822|1112|4|44| +2450822|1114|4|326| +2450822|1117|4|223| +2450822|1118|4|637| +2450822|1120|4|178| +2450822|1123|4|693| +2450822|1124|4|333| +2450822|1126|4|904| +2450822|1129|4|736| +2450822|1130|4|791| +2450822|1132|4|89| +2450822|1135|4|63| +2450822|1136|4|908| +2450822|1138|4|643| +2450822|1141|4|446| +2450822|1142|4|794| +2450822|1144|4|617| +2450822|1147|4|577| +2450822|1148|4|549| +2450822|1150|4|315| +2450822|1153|4|193| +2450822|1154|4|822| +2450822|1156|4|135| +2450822|1159|4|525| +2450822|1160|4|440| +2450822|1162|4|754| +2450822|1165|4|172| +2450822|1166|4|826| +2450822|1168|4|569| +2450822|1171|4|345| +2450822|1172|4|| +2450822|1174|4|313| +2450822|1177|4|938| +2450822|1178|4|912| +2450822|1180|4|290| +2450822|1183|4|816| +2450822|1184|4|46| +2450822|1186|4|699| +2450822|1189|4|966| +2450822|1190|4|8| +2450822|1192|4|774| +2450822|1195|4|| +2450822|1196|4|693| +2450822|1198|4|142| +2450822|1201|4|350| +2450822|1202|4|836| +2450822|1204|4|80| +2450822|1207|4|165| +2450822|1208|4|| +2450822|1210|4|888| +2450822|1213|4|943| +2450822|1214|4|49| +2450822|1216|4|515| +2450822|1219|4|750| +2450822|1220|4|463| +2450822|1222|4|118| +2450822|1225|4|275| +2450822|1226|4|549| +2450822|1228|4|670| +2450822|1231|4|652| +2450822|1232|4|390| +2450822|1234|4|26| +2450822|1237|4|| +2450822|1238|4|691| +2450822|1240|4|404| +2450822|1243|4|887| +2450822|1244|4|169| +2450822|1246|4|9| +2450822|1249|4|216| +2450822|1250|4|462| +2450822|1252|4|132| +2450822|1255|4|121| +2450822|1256|4|962| +2450822|1258|4|338| +2450822|1261|4|975| +2450822|1262|4|8| +2450822|1264|4|452| +2450822|1267|4|882| +2450822|1268|4|972| +2450822|1270|4|| +2450822|1273|4|231| +2450822|1274|4|520| +2450822|1276|4|631| +2450822|1279|4|80| +2450822|1280|4|481| +2450822|1282|4|537| +2450822|1285|4|810| +2450822|1286|4|603| +2450822|1288|4|564| +2450822|1291|4|319| +2450822|1292|4|501| +2450822|1294|4|8| +2450822|1297|4|865| +2450822|1298|4|244| +2450822|1300|4|753| +2450822|1303|4|966| +2450822|1304|4|980| +2450822|1306|4|913| +2450822|1309|4|635| +2450822|1310|4|323| +2450822|1312|4|662| +2450822|1315|4|927| +2450822|1316|4|96| +2450822|1318|4|826| +2450822|1321|4|530| +2450822|1322|4|696| +2450822|1324|4|990| +2450822|1327|4|94| +2450822|1328|4|288| +2450822|1330|4|184| +2450822|1333|4|62| +2450822|1334|4|315| +2450822|1336|4|997| +2450822|1339|4|786| +2450822|1340|4|267| +2450822|1342|4|245| +2450822|1345|4|914| +2450822|1346|4|623| +2450822|1348|4|403| +2450822|1351|4|824| +2450822|1352|4|437| +2450822|1354|4|676| +2450822|1357|4|249| +2450822|1358|4|| +2450822|1360|4|966| +2450822|1363|4|142| +2450822|1364|4|264| +2450822|1366|4|607| +2450822|1369|4|911| +2450822|1370|4|43| +2450822|1372|4|743| +2450822|1375|4|| +2450822|1376|4|907| +2450822|1378|4|667| +2450822|1381|4|758| +2450822|1382|4|489| +2450822|1384|4|227| +2450822|1387|4|982| +2450822|1388|4|795| +2450822|1390|4|720| +2450822|1393|4|548| +2450822|1394|4|901| +2450822|1396|4|773| +2450822|1399|4|851| +2450822|1400|4|182| +2450822|1402|4|563| +2450822|1405|4|593| +2450822|1406|4|191| +2450822|1408|4|924| +2450822|1411|4|426| +2450822|1412|4|71| +2450822|1414|4|845| +2450822|1417|4|928| +2450822|1418|4|818| +2450822|1420|4|676| +2450822|1423|4|194| +2450822|1424|4|82| +2450822|1426|4|17| +2450822|1429|4|679| +2450822|1430|4|900| +2450822|1432|4|66| +2450822|1435|4|548| +2450822|1436|4|667| +2450822|1438|4|78| +2450822|1441|4|457| +2450822|1442|4|677| +2450822|1444|4|959| +2450822|1447|4|934| +2450822|1448|4|478| +2450822|1450|4|275| +2450822|1453|4|853| +2450822|1454|4|635| +2450822|1456|4|330| +2450822|1459|4|82| +2450822|1460|4|298| +2450822|1462|4|5| +2450822|1465|4|614| +2450822|1466|4|805| +2450822|1468|4|218| +2450822|1471|4|| +2450822|1472|4|| +2450822|1474|4|38| +2450822|1477|4|494| +2450822|1478|4|588| +2450822|1480|4|905| +2450822|1483|4|153| +2450822|1484|4|878| +2450822|1486|4|367| +2450822|1489|4|239| +2450822|1490|4|404| +2450822|1492|4|36| +2450822|1495|4|634| +2450822|1496|4|| +2450822|1498|4|| +2450822|1501|4|514| +2450822|1502|4|293| +2450822|1504|4|505| +2450822|1507|4|| +2450822|1508|4|866| +2450822|1510|4|418| +2450822|1513|4|46| +2450822|1514|4|503| +2450822|1516|4|290| +2450822|1519|4|600| +2450822|1520|4|551| +2450822|1522|4|627| +2450822|1525|4|646| +2450822|1526|4|948| +2450822|1528|4|594| +2450822|1531|4|279| +2450822|1532|4|481| +2450822|1534|4|| +2450822|1537|4|832| +2450822|1538|4|311| +2450822|1540|4|775| +2450822|1543|4|768| +2450822|1544|4|344| +2450822|1546|4|435| +2450822|1549|4|601| +2450822|1550|4|809| +2450822|1552|4|| +2450822|1555|4|68| +2450822|1556|4|161| +2450822|1558|4|71| +2450822|1561|4|397| +2450822|1562|4|751| +2450822|1564|4|83| +2450822|1567|4|883| +2450822|1568|4|833| +2450822|1570|4|569| +2450822|1573|4|94| +2450822|1574|4|561| +2450822|1576|4|15| +2450822|1579|4|530| +2450822|1580|4|888| +2450822|1582|4|| +2450822|1585|4|130| +2450822|1586|4|550| +2450822|1588|4|351| +2450822|1591|4|780| +2450822|1592|4|15| +2450822|1594|4|175| +2450822|1597|4|506| +2450822|1598|4|61| +2450822|1600|4|637| +2450822|1603|4|976| +2450822|1604|4|852| +2450822|1606|4|| +2450822|1609|4|797| +2450822|1610|4|725| +2450822|1612|4|190| +2450822|1615|4|281| +2450822|1616|4|| +2450822|1618|4|575| +2450822|1621|4|692| +2450822|1622|4|875| +2450822|1624|4|| +2450822|1627|4|519| +2450822|1628|4|646| +2450822|1630|4|935| +2450822|1633|4|913| +2450822|1634|4|490| +2450822|1636|4|| +2450822|1639|4|295| +2450822|1640|4|909| +2450822|1642|4|645| +2450822|1645|4|864| +2450822|1646|4|255| +2450822|1648|4|225| +2450822|1651|4|| +2450822|1652|4|247| +2450822|1654|4|573| +2450822|1657|4|213| +2450822|1658|4|482| +2450822|1660|4|988| +2450822|1663|4|613| +2450822|1664|4|657| +2450822|1666|4|387| +2450822|1669|4|870| +2450822|1670|4|224| +2450822|1672|4|134| +2450822|1675|4|503| +2450822|1676|4|831| +2450822|1678|4|385| +2450822|1681|4|971| +2450822|1682|4|83| +2450822|1684|4|139| +2450822|1687|4|610| +2450822|1688|4|139| +2450822|1690|4|580| +2450822|1693|4|185| +2450822|1694|4|533| +2450822|1696|4|961| +2450822|1699|4|271| +2450822|1700|4|92| +2450822|1702|4|503| +2450822|1705|4|755| +2450822|1706|4|135| +2450822|1708|4|479| +2450822|1711|4|574| +2450822|1712|4|457| +2450822|1714|4|320| +2450822|1717|4|517| +2450822|1718|4|127| +2450822|1720|4|250| +2450822|1723|4|611| +2450822|1724|4|549| +2450822|1726|4|283| +2450822|1729|4|291| +2450822|1730|4|968| +2450822|1732|4|168| +2450822|1735|4|95| +2450822|1736|4|399| +2450822|1738|4|480| +2450822|1741|4|357| +2450822|1742|4|497| +2450822|1744|4|397| +2450822|1747|4|| +2450822|1748|4|304| +2450822|1750|4|665| +2450822|1753|4|654| +2450822|1754|4|887| +2450822|1756|4|| +2450822|1759|4|672| +2450822|1760|4|986| +2450822|1762|4|554| +2450822|1765|4|273| +2450822|1766|4|316| +2450822|1768|4|951| +2450822|1771|4|682| +2450822|1772|4|163| +2450822|1774|4|781| +2450822|1777|4|| +2450822|1778|4|526| +2450822|1780|4|18| +2450822|1783|4|660| +2450822|1784|4|70| +2450822|1786|4|207| +2450822|1789|4|881| +2450822|1790|4|880| +2450822|1792|4|208| +2450822|1795|4|426| +2450822|1796|4|782| +2450822|1798|4|313| +2450822|1801|4|419| +2450822|1802|4|857| +2450822|1804|4|941| +2450822|1807|4|492| +2450822|1808|4|493| +2450822|1810|4|363| +2450822|1813|4|148| +2450822|1814|4|379| +2450822|1816|4|540| +2450822|1819|4|767| +2450822|1820|4|274| +2450822|1822|4|752| +2450822|1825|4|607| +2450822|1826|4|| +2450822|1828|4|896| +2450822|1831|4|210| +2450822|1832|4|576| +2450822|1834|4|296| +2450822|1837|4|52| +2450822|1838|4|502| +2450822|1840|4|969| +2450822|1843|4|43| +2450822|1844|4|584| +2450822|1846|4|877| +2450822|1849|4|435| +2450822|1850|4|85| +2450822|1852|4|282| +2450822|1855|4|| +2450822|1856|4|138| +2450822|1858|4|55| +2450822|1861|4|533| +2450822|1862|4|189| +2450822|1864|4|904| +2450822|1867|4|621| +2450822|1868|4|765| +2450822|1870|4|340| +2450822|1873|4|529| +2450822|1874|4|463| +2450822|1876|4|866| +2450822|1879|4|332| +2450822|1880|4|484| +2450822|1882|4|986| +2450822|1885|4|590| +2450822|1886|4|471| +2450822|1888|4|313| +2450822|1891|4|867| +2450822|1892|4|975| +2450822|1894|4|940| +2450822|1897|4|| +2450822|1898|4|185| +2450822|1900|4|84| +2450822|1903|4|324| +2450822|1904|4|919| +2450822|1906|4|115| +2450822|1909|4|716| +2450822|1910|4|934| +2450822|1912|4|416| +2450822|1915|4|464| +2450822|1916|4|314| +2450822|1918|4|326| +2450822|1921|4|389| +2450822|1922|4|755| +2450822|1924|4|235| +2450822|1927|4|491| +2450822|1928|4|19| +2450822|1930|4|382| +2450822|1933|4|166| +2450822|1934|4|241| +2450822|1936|4|712| +2450822|1939|4|591| +2450822|1940|4|| +2450822|1942|4|953| +2450822|1945|4|987| +2450822|1946|4|132| +2450822|1948|4|620| +2450822|1951|4|790| +2450822|1952|4|911| +2450822|1954|4|796| +2450822|1957|4|387| +2450822|1958|4|105| +2450822|1960|4|934| +2450822|1963|4|879| +2450822|1964|4|502| +2450822|1966|4|235| +2450822|1969|4|884| +2450822|1970|4|129| +2450822|1972|4|514| +2450822|1975|4|282| +2450822|1976|4|| +2450822|1978|4|342| +2450822|1981|4|603| +2450822|1982|4|20| +2450822|1984|4|| +2450822|1987|4|156| +2450822|1988|4|415| +2450822|1990|4|704| +2450822|1993|4|976| +2450822|1994|4|974| +2450822|1996|4|668| +2450822|1999|4|455| +2450822|2000|4|424| +2450822|2002|4|122| +2450822|2005|4|811| +2450822|2006|4|423| +2450822|2008|4|804| +2450822|2011|4|139| +2450822|2012|4|144| +2450822|2014|4|320| +2450822|2017|4|| +2450822|2018|4|575| +2450822|2020|4|785| +2450822|2023|4|971| +2450822|2024|4|| +2450822|2026|4|682| +2450822|2029|4|548| +2450822|2030|4|552| +2450822|2032|4|749| +2450822|2035|4|406| +2450822|2036|4|826| +2450822|2038|4|838| +2450822|2041|4|760| +2450822|2042|4|886| +2450822|2044|4|469| +2450822|2047|4|667| +2450822|2048|4|896| +2450822|2050|4|398| +2450822|2053|4|219| +2450822|2054|4|557| +2450822|2056|4|451| +2450822|2059|4|411| +2450822|2060|4|| +2450822|2062|4|765| +2450822|2065|4|84| +2450822|2066|4|121| +2450822|2068|4|947| +2450822|2071|4|833| +2450822|2072|4|66| +2450822|2074|4|683| +2450822|2077|4|324| +2450822|2078|4|378| +2450822|2080|4|7| +2450822|2083|4|308| +2450822|2084|4|869| +2450822|2086|4|937| +2450822|2089|4|788| +2450822|2090|4|699| +2450822|2092|4|407| +2450822|2095|4|753| +2450822|2096|4|961| +2450822|2098|4|208| +2450822|2101|4|537| +2450822|2102|4|444| +2450822|2104|4|536| +2450822|2107|4|215| +2450822|2108|4|871| +2450822|2110|4|167| +2450822|2113|4|55| +2450822|2114|4|375| +2450822|2116|4|81| +2450822|2119|4|331| +2450822|2120|4|609| +2450822|2122|4|968| +2450822|2125|4|932| +2450822|2126|4|703| +2450822|2128|4|421| +2450822|2131|4|648| +2450822|2132|4|802| +2450822|2134|4|155| +2450822|2137|4|721| +2450822|2138|4|547| +2450822|2140|4|196| +2450822|2143|4|573| +2450822|2144|4|636| +2450822|2146|4|384| +2450822|2149|4|820| +2450822|2150|4|864| +2450822|2152|4|236| +2450822|2155|4|888| +2450822|2156|4|151| +2450822|2158|4|318| +2450822|2161|4|152| +2450822|2162|4|889| +2450822|2164|4|352| +2450822|2167|4|18| +2450822|2168|4|916| +2450822|2170|4|312| +2450822|2173|4|905| +2450822|2174|4|941| +2450822|2176|4|573| +2450822|2179|4|28| +2450822|2180|4|430| +2450822|2182|4|685| +2450822|2185|4|409| +2450822|2186|4|178| +2450822|2188|4|| +2450822|2191|4|179| +2450822|2192|4|895| +2450822|2194|4|189| +2450822|2197|4|984| +2450822|2198|4|644| +2450822|2200|4|784| +2450822|2203|4|923| +2450822|2204|4|347| +2450822|2206|4|333| +2450822|2209|4|681| +2450822|2210|4|32| +2450822|2212|4|46| +2450822|2215|4|135| +2450822|2216|4|73| +2450822|2218|4|880| +2450822|2221|4|368| +2450822|2222|4|847| +2450822|2224|4|408| +2450822|2227|4|679| +2450822|2228|4|774| +2450822|2230|4|775| +2450822|2233|4|312| +2450822|2234|4|753| +2450822|2236|4|| +2450822|2239|4|773| +2450822|2240|4|69| +2450822|2242|4|393| +2450822|2245|4|676| +2450822|2246|4|939| +2450822|2248|4|972| +2450822|2251|4|646| +2450822|2252|4|510| +2450822|2254|4|560| +2450822|2257|4|610| +2450822|2258|4|641| +2450822|2260|4|230| +2450822|2263|4|574| +2450822|2264|4|950| +2450822|2266|4|251| +2450822|2269|4|619| +2450822|2270|4|926| +2450822|2272|4|220| +2450822|2275|4|941| +2450822|2276|4|764| +2450822|2278|4|43| +2450822|2281|4|303| +2450822|2282|4|501| +2450822|2284|4|| +2450822|2287|4|579| +2450822|2288|4|787| +2450822|2290|4|628| +2450822|2293|4|813| +2450822|2294|4|126| +2450822|2296|4|447| +2450822|2299|4|319| +2450822|2300|4|318| +2450822|2302|4|387| +2450822|2305|4|122| +2450822|2306|4|107| +2450822|2308|4|| +2450822|2311|4|704| +2450822|2312|4|474| +2450822|2314|4|429| +2450822|2317|4|803| +2450822|2318|4|317| +2450822|2320|4|543| +2450822|2323|4|985| +2450822|2324|4|932| +2450822|2326|4|751| +2450822|2329|4|943| +2450822|2330|4|395| +2450822|2332|4|89| +2450822|2335|4|670| +2450822|2336|4|149| +2450822|2338|4|142| +2450822|2341|4|787| +2450822|2342|4|277| +2450822|2344|4|761| +2450822|2347|4|992| +2450822|2348|4|425| +2450822|2350|4|561| +2450822|2353|4|554| +2450822|2354|4|| +2450822|2356|4|185| +2450822|2359|4|741| +2450822|2360|4|436| +2450822|2362|4|48| +2450822|2365|4|429| +2450822|2366|4|266| +2450822|2368|4|122| +2450822|2371|4|127| +2450822|2372|4|141| +2450822|2374|4|276| +2450822|2377|4|57| +2450822|2378|4|506| +2450822|2380|4|769| +2450822|2383|4|724| +2450822|2384|4|551| +2450822|2386|4|697| +2450822|2389|4|559| +2450822|2390|4|303| +2450822|2392|4|319| +2450822|2395|4|| +2450822|2396|4|202| +2450822|2398|4|930| +2450822|2401|4|958| +2450822|2402|4|105| +2450822|2404|4|320| +2450822|2407|4|21| +2450822|2408|4|11| +2450822|2410|4|949| +2450822|2413|4|21| +2450822|2414|4|123| +2450822|2416|4|585| +2450822|2419|4|233| +2450822|2420|4|333| +2450822|2422|4|472| +2450822|2425|4|732| +2450822|2426|4|963| +2450822|2428|4|| +2450822|2431|4|215| +2450822|2432|4|942| +2450822|2434|4|455| +2450822|2437|4|620| +2450822|2438|4|888| +2450822|2440|4|811| +2450822|2443|4|644| +2450822|2444|4|713| +2450822|2446|4|572| +2450822|2449|4|586| +2450822|2450|4|635| +2450822|2452|4|192| +2450822|2455|4|525| +2450822|2456|4|351| +2450822|2458|4|172| +2450822|2461|4|195| +2450822|2462|4|305| +2450822|2464|4|567| +2450822|2467|4|77| +2450822|2468|4|728| +2450822|2470|4|825| +2450822|2473|4|670| +2450822|2474|4|461| +2450822|2476|4|908| +2450822|2479|4|232| +2450822|2480|4|578| +2450822|2482|4|910| +2450822|2485|4|69| +2450822|2486|4|952| +2450822|2488|4|190| +2450822|2491|4|| +2450822|2492|4|852| +2450822|2494|4|616| +2450822|2497|4|884| +2450822|2498|4|264| +2450822|2500|4|69| +2450822|2503|4|541| +2450822|2504|4|60| +2450822|2506|4|566| +2450822|2509|4|871| +2450822|2510|4|88| +2450822|2512|4|361| +2450822|2515|4|782| +2450822|2516|4|327| +2450822|2518|4|367| +2450822|2521|4|679| +2450822|2522|4|944| +2450822|2524|4|271| +2450822|2527|4|169| +2450822|2528|4|747| +2450822|2530|4|913| +2450822|2533|4|419| +2450822|2534|4|968| +2450822|2536|4|27| +2450822|2539|4|798| +2450822|2540|4|| +2450822|2542|4|872| +2450822|2545|4|2| +2450822|2546|4|696| +2450822|2548|4|154| +2450822|2551|4|900| +2450822|2552|4|| +2450822|2554|4|564| +2450822|2557|4|868| +2450822|2558|4|879| +2450822|2560|4|640| +2450822|2563|4|| +2450822|2564|4|437| +2450822|2566|4|853| +2450822|2569|4|231| +2450822|2570|4|148| +2450822|2572|4|252| +2450822|2575|4|| +2450822|2576|4|775| +2450822|2578|4|862| +2450822|2581|4|480| +2450822|2582|4|681| +2450822|2584|4|245| +2450822|2587|4|151| +2450822|2588|4|551| +2450822|2590|4|78| +2450822|2593|4|691| +2450822|2594|4|14| +2450822|2596|4|402| +2450822|2599|4|444| +2450822|2600|4|832| +2450822|2602|4|180| +2450822|2605|4|981| +2450822|2606|4|848| +2450822|2608|4|| +2450822|2611|4|| +2450822|2612|4|| +2450822|2614|4|858| +2450822|2617|4|851| +2450822|2618|4|462| +2450822|2620|4|| +2450822|2623|4|302| +2450822|2624|4|250| +2450822|2626|4|859| +2450822|2629|4|| +2450822|2630|4|111| +2450822|2632|4|764| +2450822|2635|4|405| +2450822|2636|4|310| +2450822|2638|4|758| +2450822|2641|4|166| +2450822|2642|4|941| +2450822|2644|4|199| +2450822|2647|4|829| +2450822|2648|4|36| +2450822|2650|4|434| +2450822|2653|4|885| +2450822|2654|4|547| +2450822|2656|4|973| +2450822|2659|4|198| +2450822|2660|4|495| +2450822|2662|4|306| +2450822|2665|4|805| +2450822|2666|4|200| +2450822|2668|4|402| +2450822|2671|4|253| +2450822|2672|4|127| +2450822|2674|4|70| +2450822|2677|4|753| +2450822|2678|4|597| +2450822|2680|4|| +2450822|2683|4|763| +2450822|2684|4|364| +2450822|2686|4|293| +2450822|2689|4|619| +2450822|2690|4|6| +2450822|2692|4|528| +2450822|2695|4|338| +2450822|2696|4|257| +2450822|2698|4|511| +2450822|2701|4|255| +2450822|2702|4|94| +2450822|2704|4|73| +2450822|2707|4|631| +2450822|2708|4|522| +2450822|2710|4|380| +2450822|2713|4|557| +2450822|2714|4|568| +2450822|2716|4|761| +2450822|2719|4|50| +2450822|2720|4|583| +2450822|2722|4|882| +2450822|2725|4|949| +2450822|2726|4|132| +2450822|2728|4|116| +2450822|2731|4|482| +2450822|2732|4|202| +2450822|2734|4|86| +2450822|2737|4|134| +2450822|2738|4|479| +2450822|2740|4|268| +2450822|2743|4|902| +2450822|2744|4|620| +2450822|2746|4|113| +2450822|2749|4|| +2450822|2750|4|512| +2450822|2752|4|241| +2450822|2755|4|992| +2450822|2756|4|676| +2450822|2758|4|864| +2450822|2761|4|911| +2450822|2762|4|473| +2450822|2764|4|668| +2450822|2767|4|80| +2450822|2768|4|878| +2450822|2770|4|994| +2450822|2773|4|187| +2450822|2774|4|457| +2450822|2776|4|418| +2450822|2779|4|335| +2450822|2780|4|453| +2450822|2782|4|142| +2450822|2785|4|176| +2450822|2786|4|237| +2450822|2788|4|598| +2450822|2791|4|920| +2450822|2792|4|683| +2450822|2794|4|431| +2450822|2797|4|275| +2450822|2798|4|265| +2450822|2800|4|867| +2450822|2803|4|67| +2450822|2804|4|466| +2450822|2806|4|797| +2450822|2809|4|549| +2450822|2810|4|919| +2450822|2812|4|365| +2450822|2815|4|966| +2450822|2816|4|745| +2450822|2818|4|499| +2450822|2821|4|97| +2450822|2822|4|430| +2450822|2824|4|204| +2450822|2827|4|750| +2450822|2828|4|366| +2450822|2830|4|931| +2450822|2833|4|53| +2450822|2834|4|806| +2450822|2836|4|521| +2450822|2839|4|308| +2450822|2840|4|568| +2450822|2842|4|134| +2450822|2845|4|96| +2450822|2846|4|970| +2450822|2848|4|514| +2450822|2851|4|774| +2450822|2852|4|47| +2450822|2854|4|313| +2450822|2857|4|598| +2450822|2858|4|147| +2450822|2860|4|109| +2450822|2863|4|468| +2450822|2864|4|816| +2450822|2866|4|| +2450822|2869|4|591| +2450822|2870|4|29| +2450822|2872|4|641| +2450822|2875|4|846| +2450822|2876|4|374| +2450822|2878|4|209| +2450822|2881|4|109| +2450822|2882|4|545| +2450822|2884|4|620| +2450822|2887|4|560| +2450822|2888|4|942| +2450822|2890|4|369| +2450822|2893|4|908| +2450822|2894|4|0| +2450822|2896|4|631| +2450822|2899|4|123| +2450822|2900|4|548| +2450822|2902|4|645| +2450822|2905|4|529| +2450822|2906|4|843| +2450822|2908|4|273| +2450822|2911|4|22| +2450822|2912|4|176| +2450822|2914|4|455| +2450822|2917|4|718| +2450822|2918|4|638| +2450822|2920|4|848| +2450822|2923|4|572| +2450822|2924|4|697| +2450822|2926|4|610| +2450822|2929|4|392| +2450822|2930|4|21| +2450822|2932|4|| +2450822|2935|4|996| +2450822|2936|4|393| +2450822|2938|4|380| +2450822|2941|4|365| +2450822|2942|4|219| +2450822|2944|4|435| +2450822|2947|4|679| +2450822|2948|4|95| +2450822|2950|4|| +2450822|2953|4|679| +2450822|2954|4|| +2450822|2956|4|488| +2450822|2959|4|607| +2450822|2960|4|218| +2450822|2962|4|451| +2450822|2965|4|515| +2450822|2966|4|802| +2450822|2968|4|485| +2450822|2971|4|267| +2450822|2972|4|43| +2450822|2974|4|347| +2450822|2977|4|410| +2450822|2978|4|795| +2450822|2980|4|829| +2450822|2983|4|172| +2450822|2984|4|39| +2450822|2986|4|653| +2450822|2989|4|335| +2450822|2990|4|789| +2450822|2992|4|419| +2450822|2995|4|764| +2450822|2996|4|530| +2450822|2998|4|414| +2450822|3001|4|479| +2450822|3002|4|270| +2450822|3004|4|825| +2450822|3007|4|334| +2450822|3008|4|670| +2450822|3010|4|364| +2450822|3013|4|979| +2450822|3014|4|200| +2450822|3016|4|823| +2450822|3019|4|702| +2450822|3020|4|100| +2450822|3022|4|527| +2450822|3025|4|610| +2450822|3026|4|16| +2450822|3028|4|388| +2450822|3031|4|303| +2450822|3032|4|97| +2450822|3034|4|557| +2450822|3037|4|651| +2450822|3038|4|81| +2450822|3040|4|35| +2450822|3043|4|982| +2450822|3044|4|279| +2450822|3046|4|267| +2450822|3049|4|20| +2450822|3050|4|525| +2450822|3052|4|750| +2450822|3055|4|92| +2450822|3056|4|519| +2450822|3058|4|499| +2450822|3061|4|357| +2450822|3062|4|406| +2450822|3064|4|821| +2450822|3067|4|| +2450822|3068|4|393| +2450822|3070|4|179| +2450822|3073|4|991| +2450822|3074|4|323| +2450822|3076|4|723| +2450822|3079|4|347| +2450822|3080|4|210| +2450822|3082|4|112| +2450822|3085|4|305| +2450822|3086|4|860| +2450822|3088|4|| +2450822|3091|4|341| +2450822|3092|4|526| +2450822|3094|4|95| +2450822|3097|4|601| +2450822|3098|4|| +2450822|3100|4|776| +2450822|3103|4|252| +2450822|3104|4|974| +2450822|3106|4|| +2450822|3109|4|587| +2450822|3110|4|240| +2450822|3112|4|938| +2450822|3115|4|901| +2450822|3116|4|158| +2450822|3118|4|190| +2450822|3121|4|666| +2450822|3122|4|618| +2450822|3124|4|401| +2450822|3127|4|470| +2450822|3128|4|560| +2450822|3130|4|660| +2450822|3133|4|278| +2450822|3134|4|798| +2450822|3136|4|640| +2450822|3139|4|265| +2450822|3140|4|| +2450822|3142|4|614| +2450822|3145|4|561| +2450822|3146|4|604| +2450822|3148|4|223| +2450822|3151|4|336| +2450822|3152|4|15| +2450822|3154|4|52| +2450822|3157|4|261| +2450822|3158|4|457| +2450822|3160|4|563| +2450822|3163|4|708| +2450822|3164|4|602| +2450822|3166|4|283| +2450822|3169|4|896| +2450822|3170|4|| +2450822|3172|4|489| +2450822|3175|4|842| +2450822|3176|4|990| +2450822|3178|4|180| +2450822|3181|4|539| +2450822|3182|4|| +2450822|3184|4|395| +2450822|3187|4|626| +2450822|3188|4|712| +2450822|3190|4|346| +2450822|3193|4|18| +2450822|3194|4|912| +2450822|3196|4|740| +2450822|3199|4|| +2450822|3200|4|668| +2450822|3202|4|81| +2450822|3205|4|685| +2450822|3206|4|274| +2450822|3208|4|985| +2450822|3211|4|403| +2450822|3212|4|540| +2450822|3214|4|831| +2450822|3217|4|341| +2450822|3218|4|415| +2450822|3220|4|967| +2450822|3223|4|782| +2450822|3224|4|421| +2450822|3226|4|965| +2450822|3229|4|738| +2450822|3230|4|126| +2450822|3232|4|515| +2450822|3235|4|277| +2450822|3236|4|123| +2450822|3238|4|56| +2450822|3241|4|105| +2450822|3242|4|65| +2450822|3244|4|573| +2450822|3247|4|456| +2450822|3248|4|350| +2450822|3250|4|| +2450822|3253|4|659| +2450822|3254|4|318| +2450822|3256|4|215| +2450822|3259|4|928| +2450822|3260|4|969| +2450822|3262|4|36| +2450822|3265|4|518| +2450822|3266|4|410| +2450822|3268|4|891| +2450822|3271|4|883| +2450822|3272|4|775| +2450822|3274|4|557| +2450822|3277|4|440| +2450822|3278|4|713| +2450822|3280|4|336| +2450822|3283|4|249| +2450822|3284|4|770| +2450822|3286|4|70| +2450822|3289|4|471| +2450822|3290|4|331| +2450822|3292|4|728| +2450822|3295|4|954| +2450822|3296|4|420| +2450822|3298|4|151| +2450822|3301|4|886| +2450822|3302|4|549| +2450822|3304|4|813| +2450822|3307|4|811| +2450822|3308|4|646| +2450822|3310|4|| +2450822|3313|4|830| +2450822|3314|4|817| +2450822|3316|4|354| +2450822|3319|4|760| +2450822|3320|4|568| +2450822|3322|4|428| +2450822|3325|4|286| +2450822|3326|4|972| +2450822|3328|4|838| +2450822|3331|4|109| +2450822|3332|4|44| +2450822|3334|4|567| +2450822|3337|4|453| +2450822|3338|4|794| +2450822|3340|4|909| +2450822|3343|4|658| +2450822|3344|4|882| +2450822|3346|4|838| +2450822|3349|4|278| +2450822|3350|4|1| +2450822|3352|4|279| +2450822|3355|4|546| +2450822|3356|4|852| +2450822|3358|4|250| +2450822|3361|4|| +2450822|3362|4|152| +2450822|3364|4|88| +2450822|3367|4|278| +2450822|3368|4|374| +2450822|3370|4|755| +2450822|3373|4|23| +2450822|3374|4|380| +2450822|3376|4|1000| +2450822|3379|4|627| +2450822|3380|4|649| +2450822|3382|4|207| +2450822|3385|4|100| +2450822|3386|4|785| +2450822|3388|4|286| +2450822|3391|4|62| +2450822|3392|4|381| +2450822|3394|4|720| +2450822|3397|4|113| +2450822|3398|4|186| +2450822|3400|4|956| +2450822|3403|4|920| +2450822|3404|4|| +2450822|3406|4|519| +2450822|3409|4|541| +2450822|3410|4|352| +2450822|3412|4|57| +2450822|3415|4|296| +2450822|3416|4|11| +2450822|3418|4|237| +2450822|3421|4|32| +2450822|3422|4|846| +2450822|3424|4|675| +2450822|3427|4|527| +2450822|3428|4|998| +2450822|3430|4|700| +2450822|3433|4|958| +2450822|3434|4|45| +2450822|3436|4|666| +2450822|3439|4|| +2450822|3440|4|549| +2450822|3442|4|905| +2450822|3445|4|784| +2450822|3446|4|444| +2450822|3448|4|206| +2450822|3451|4|340| +2450822|3452|4|783| +2450822|3454|4|567| +2450822|3457|4|323| +2450822|3458|4|525| +2450822|3460|4|919| +2450822|3463|4|48| +2450822|3464|4|880| +2450822|3466|4|165| +2450822|3469|4|656| +2450822|3470|4|307| +2450822|3472|4|69| +2450822|3475|4|281| +2450822|3476|4|892| +2450822|3478|4|562| +2450822|3481|4|171| +2450822|3482|4|637| +2450822|3484|4|788| +2450822|3487|4|315| +2450822|3488|4|| +2450822|3490|4|663| +2450822|3493|4|151| +2450822|3494|4|413| +2450822|3496|4|19| +2450822|3499|4|584| +2450822|3500|4|352| +2450822|3502|4|945| +2450822|3505|4|781| +2450822|3506|4|918| +2450822|3508|4|264| +2450822|3511|4|583| +2450822|3512|4|774| +2450822|3514|4|948| +2450822|3517|4|733| +2450822|3518|4|233| +2450822|3520|4|850| +2450822|3523|4|955| +2450822|3524|4|| +2450822|3526|4|615| +2450822|3529|4|940| +2450822|3530|4|268| +2450822|3532|4|173| +2450822|3535|4|292| +2450822|3536|4|666| +2450822|3538|4|965| +2450822|3541|4|777| +2450822|3542|4|696| +2450822|3544|4|97| +2450822|3547|4|962| +2450822|3548|4|423| +2450822|3550|4|243| +2450822|3553|4|312| +2450822|3554|4|673| +2450822|3556|4|963| +2450822|3559|4|63| +2450822|3560|4|173| +2450822|3562|4|229| +2450822|3565|4|426| +2450822|3566|4|328| +2450822|3568|4|229| +2450822|3571|4|16| +2450822|3572|4|533| +2450822|3574|4|772| +2450822|3577|4|962| +2450822|3578|4|74| +2450822|3580|4|706| +2450822|3583|4|652| +2450822|3584|4|566| +2450822|3586|4|326| +2450822|3589|4|736| +2450822|3590|4|3| +2450822|3592|4|385| +2450822|3595|4|274| +2450822|3596|4|594| +2450822|3598|4|928| +2450822|3601|4|998| +2450822|3602|4|675| +2450822|3604|4|947| +2450822|3607|4|382| +2450822|3608|4|103| +2450822|3610|4|829| +2450822|3613|4|967| +2450822|3614|4|8| +2450822|3616|4|911| +2450822|3619|4|359| +2450822|3620|4|323| +2450822|3622|4|31| +2450822|3625|4|243| +2450822|3626|4|859| +2450822|3628|4|249| +2450822|3631|4|50| +2450822|3632|4|312| +2450822|3634|4|217| +2450822|3637|4|| +2450822|3638|4|35| +2450822|3640|4|989| +2450822|3643|4|105| +2450822|3644|4|924| +2450822|3646|4|123| +2450822|3649|4|721| +2450822|3650|4|| +2450822|3652|4|853| +2450822|3655|4|658| +2450822|3656|4|293| +2450822|3658|4|415| +2450822|3661|4|832| +2450822|3662|4|772| +2450822|3664|4|863| +2450822|3667|4|857| +2450822|3668|4|882| +2450822|3670|4|222| +2450822|3673|4|810| +2450822|3674|4|60| +2450822|3676|4|192| +2450822|3679|4|846| +2450822|3680|4|119| +2450822|3682|4|992| +2450822|3685|4|540| +2450822|3686|4|263| +2450822|3688|4|739| +2450822|3691|4|938| +2450822|3692|4|556| +2450822|3694|4|39| +2450822|3697|4|921| +2450822|3698|4|995| +2450822|3700|4|| +2450822|3703|4|639| +2450822|3704|4|| +2450822|3706|4|923| +2450822|3709|4|435| +2450822|3710|4|986| +2450822|3712|4|308| +2450822|3715|4|685| +2450822|3716|4|699| +2450822|3718|4|977| +2450822|3721|4|392| +2450822|3722|4|745| +2450822|3724|4|327| +2450822|3727|4|231| +2450822|3728|4|734| +2450822|3730|4|224| +2450822|3733|4|137| +2450822|3734|4|166| +2450822|3736|4|805| +2450822|3739|4|197| +2450822|3740|4|904| +2450822|3742|4|177| +2450822|3745|4|336| +2450822|3746|4|450| +2450822|3748|4|924| +2450822|3751|4|353| +2450822|3752|4|675| +2450822|3754|4|665| +2450822|3757|4|913| +2450822|3758|4|865| +2450822|3760|4|517| +2450822|3763|4|803| +2450822|3764|4|693| +2450822|3766|4|121| +2450822|3769|4|383| +2450822|3770|4|384| +2450822|3772|4|362| +2450822|3775|4|343| +2450822|3776|4|848| +2450822|3778|4|395| +2450822|3781|4|744| +2450822|3782|4|115| +2450822|3784|4|550| +2450822|3787|4|999| +2450822|3788|4|145| +2450822|3790|4|597| +2450822|3793|4|668| +2450822|3794|4|623| +2450822|3796|4|655| +2450822|3799|4|166| +2450822|3800|4|414| +2450822|3802|4|768| +2450822|3805|4|543| +2450822|3806|4|426| +2450822|3808|4|641| +2450822|3811|4|365| +2450822|3812|4|534| +2450822|3814|4|358| +2450822|3817|4|168| +2450822|3818|4|945| +2450822|3820|4|323| +2450822|3823|4|58| +2450822|3824|4|756| +2450822|3826|4|177| +2450822|3829|4|166| +2450822|3830|4|826| +2450822|3832|4|495| +2450822|3835|4|748| +2450822|3836|4|26| +2450822|3838|4|102| +2450822|3841|4|500| +2450822|3842|4|511| +2450822|3844|4|500| +2450822|3847|4|841| +2450822|3848|4|976| +2450822|3850|4|375| +2450822|3853|4|757| +2450822|3854|4|401| +2450822|3856|4|| +2450822|3859|4|488| +2450822|3860|4|594| +2450822|3862|4|| +2450822|3865|4|| +2450822|3866|4|742| +2450822|3868|4|| +2450822|3871|4|343| +2450822|3872|4|949| +2450822|3874|4|12| +2450822|3877|4|513| +2450822|3878|4|533| +2450822|3880|4|83| +2450822|3883|4|757| +2450822|3884|4|29| +2450822|3886|4|834| +2450822|3889|4|16| +2450822|3890|4|130| +2450822|3892|4|140| +2450822|3895|4|731| +2450822|3896|4|880| +2450822|3898|4|760| +2450822|3901|4|90| +2450822|3902|4|312| +2450822|3904|4|331| +2450822|3907|4|669| +2450822|3908|4|853| +2450822|3910|4|| +2450822|3913|4|442| +2450822|3914|4|242| +2450822|3916|4|547| +2450822|3919|4|234| +2450822|3920|4|616| +2450822|3922|4|435| +2450822|3925|4|| +2450822|3926|4|486| +2450822|3928|4|372| +2450822|3931|4|44| +2450822|3932|4|104| +2450822|3934|4|990| +2450822|3937|4|145| +2450822|3938|4|140| +2450822|3940|4|628| +2450822|3943|4|969| +2450822|3944|4|832| +2450822|3946|4|92| +2450822|3949|4|246| +2450822|3950|4|800| +2450822|3952|4|695| +2450822|3955|4|759| +2450822|3956|4|205| +2450822|3958|4|307| +2450822|3961|4|511| +2450822|3962|4|455| +2450822|3964|4|152| +2450822|3967|4|665| +2450822|3968|4|178| +2450822|3970|4|255| +2450822|3973|4|664| +2450822|3974|4|102| +2450822|3976|4|728| +2450822|3979|4|171| +2450822|3980|4|255| +2450822|3982|4|713| +2450822|3985|4|932| +2450822|3986|4|546| +2450822|3988|4|232| +2450822|3991|4|47| +2450822|3992|4|539| +2450822|3994|4|133| +2450822|3997|4|646| +2450822|3998|4|772| +2450822|4000|4|497| +2450822|4003|4|560| +2450822|4004|4|894| +2450822|4006|4|190| +2450822|4009|4|663| +2450822|4010|4|372| +2450822|4012|4|342| +2450822|4015|4|300| +2450822|4016|4|780| +2450822|4018|4|178| +2450822|4021|4|796| +2450822|4022|4|701| +2450822|4024|4|884| +2450822|4027|4|72| +2450822|4028|4|736| +2450822|4030|4|158| +2450822|4033|4|190| +2450822|4034|4|577| +2450822|4036|4|400| +2450822|4039|4|477| +2450822|4040|4|790| +2450822|4042|4|149| +2450822|4045|4|| +2450822|4046|4|859| +2450822|4048|4|| +2450822|4051|4|145| +2450822|4052|4|30| +2450822|4054|4|852| +2450822|4057|4|274| +2450822|4058|4|684| +2450822|4060|4|0| +2450822|4063|4|402| +2450822|4064|4|538| +2450822|4066|4|154| +2450822|4069|4|386| +2450822|4070|4|| +2450822|4072|4|232| +2450822|4075|4|351| +2450822|4076|4|418| +2450822|4078|4|576| +2450822|4081|4|128| +2450822|4082|4|902| +2450822|4084|4|830| +2450822|4087|4|208| +2450822|4088|4|| +2450822|4090|4|694| +2450822|4093|4|920| +2450822|4094|4|832| +2450822|4096|4|9| +2450822|4099|4|967| +2450822|4100|4|| +2450822|4102|4|229| +2450822|4105|4|174| +2450822|4106|4|663| +2450822|4108|4|397| +2450822|4111|4|926| +2450822|4112|4|| +2450822|4114|4|154| +2450822|4117|4|1000| +2450822|4118|4|| +2450822|4120|4|57| +2450822|4123|4|266| +2450822|4124|4|2| +2450822|4126|4|126| +2450822|4129|4|141| +2450822|4130|4|92| +2450822|4132|4|391| +2450822|4135|4|265| +2450822|4136|4|593| +2450822|4138|4|694| +2450822|4141|4|335| +2450822|4142|4|483| +2450822|4144|4|406| +2450822|4147|4|| +2450822|4148|4|416| +2450822|4150|4|513| +2450822|4153|4|982| +2450822|4154|4|325| +2450822|4156|4|263| +2450822|4159|4|402| +2450822|4160|4|186| +2450822|4162|4|529| +2450822|4165|4|227| +2450822|4166|4|371| +2450822|4168|4|769| +2450822|4171|4|306| +2450822|4172|4|206| +2450822|4174|4|167| +2450822|4177|4|533| +2450822|4178|4|678| +2450822|4180|4|864| +2450822|4183|4|529| +2450822|4184|4|545| +2450822|4186|4|395| +2450822|4189|4|| +2450822|4190|4|878| +2450822|4192|4|719| +2450822|4195|4|503| +2450822|4196|4|176| +2450822|4198|4|497| +2450822|4201|4|438| +2450822|4202|4|321| +2450822|4204|4|489| +2450822|4207|4|705| +2450822|4208|4|194| +2450822|4210|4|7| +2450822|4213|4|| +2450822|4214|4|44| +2450822|4216|4|786| +2450822|4219|4|393| +2450822|4220|4|740| +2450822|4222|4|347| +2450822|4225|4|306| +2450822|4226|4|850| +2450822|4228|4|575| +2450822|4231|4|| +2450822|4232|4|763| +2450822|4234|4|230| +2450822|4237|4|| +2450822|4238|4|37| +2450822|4240|4|14| +2450822|4243|4|| +2450822|4244|4|767| +2450822|4246|4|393| +2450822|4249|4|430| +2450822|4250|4|156| +2450822|4252|4|566| +2450822|4255|4|67| +2450822|4256|4|308| +2450822|4258|4|47| +2450822|4261|4|284| +2450822|4262|4|753| +2450822|4264|4|751| +2450822|4267|4|311| +2450822|4268|4|| +2450822|4270|4|474| +2450822|4273|4|129| +2450822|4274|4|529| +2450822|4276|4|538| +2450822|4279|4|221| +2450822|4280|4|370| +2450822|4282|4|903| +2450822|4285|4|707| +2450822|4286|4|952| +2450822|4288|4|714| +2450822|4291|4|233| +2450822|4292|4|660| +2450822|4294|4|| +2450822|4297|4|428| +2450822|4298|4|509| +2450822|4300|4|154| +2450822|4303|4|529| +2450822|4304|4|373| +2450822|4306|4|305| +2450822|4309|4|546| +2450822|4310|4|| +2450822|4312|4|287| +2450822|4315|4|31| +2450822|4316|4|199| +2450822|4318|4|231| +2450822|4321|4|626| +2450822|4322|4|169| +2450822|4324|4|802| +2450822|4327|4|138| +2450822|4328|4|321| +2450822|4330|4|209| +2450822|4333|4|500| +2450822|4334|4|219| +2450822|4336|4|4| +2450822|4339|4|314| +2450822|4340|4|657| +2450822|4342|4|577| +2450822|4345|4|334| +2450822|4346|4|780| +2450822|4348|4|296| +2450822|4351|4|| +2450822|4352|4|246| +2450822|4354|4|484| +2450822|4357|4|352| +2450822|4358|4|| +2450822|4360|4|850| +2450822|4363|4|| +2450822|4364|4|275| +2450822|4366|4|625| +2450822|4369|4|436| +2450822|4370|4|654| +2450822|4372|4|585| +2450822|4375|4|425| +2450822|4376|4|276| +2450822|4378|4|380| +2450822|4381|4|992| +2450822|4382|4|952| +2450822|4384|4|753| +2450822|4387|4|773| +2450822|4388|4|990| +2450822|4390|4|46| +2450822|4393|4|437| +2450822|4394|4|163| +2450822|4396|4|634| +2450822|4399|4|240| +2450822|4400|4|902| +2450822|4402|4|349| +2450822|4405|4|917| +2450822|4406|4|645| +2450822|4408|4|547| +2450822|4411|4|515| +2450822|4412|4|87| +2450822|4414|4|492| +2450822|4417|4|94| +2450822|4418|4|727| +2450822|4420|4|842| +2450822|4423|4|832| +2450822|4424|4|814| +2450822|4426|4|218| +2450822|4429|4|559| +2450822|4430|4|748| +2450822|4432|4|839| +2450822|4435|4|271| +2450822|4436|4|293| +2450822|4438|4|536| +2450822|4441|4|565| +2450822|4442|4|43| +2450822|4444|4|686| +2450822|4447|4|528| +2450822|4448|4|93| +2450822|4450|4|181| +2450822|4453|4|926| +2450822|4454|4|721| +2450822|4456|4|94| +2450822|4459|4|209| +2450822|4460|4|470| +2450822|4462|4|700| +2450822|4465|4|500| +2450822|4466|4|202| +2450822|4468|4|235| +2450822|4471|4|234| +2450822|4472|4|746| +2450822|4474|4|25| +2450822|4477|4|672| +2450822|4478|4|578| +2450822|4480|4|692| +2450822|4483|4|595| +2450822|4484|4|24| +2450822|4486|4|663| +2450822|4489|4|471| +2450822|4490|4|997| +2450822|4492|4|195| +2450822|4495|4|876| +2450822|4496|4|903| +2450822|4498|4|31| +2450822|4501|4|335| +2450822|4502|4|299| +2450822|4504|4|730| +2450822|4507|4|620| +2450822|4508|4|815| +2450822|4510|4|994| +2450822|4513|4|768| +2450822|4514|4|744| +2450822|4516|4|800| +2450822|4519|4|573| +2450822|4520|4|35| +2450822|4522|4|788| +2450822|4525|4|445| +2450822|4526|4|| +2450822|4528|4|704| +2450822|4531|4|40| +2450822|4532|4|930| +2450822|4534|4|501| +2450822|4537|4|874| +2450822|4538|4|774| +2450822|4540|4|792| +2450822|4543|4|987| +2450822|4544|4|773| +2450822|4546|4|733| +2450822|4549|4|66| +2450822|4550|4|137| +2450822|4552|4|476| +2450822|4555|4|855| +2450822|4556|4|137| +2450822|4558|4|939| +2450822|4561|4|887| +2450822|4562|4|322| +2450822|4564|4|428| +2450822|4567|4|936| +2450822|4568|4|| +2450822|4570|4|| +2450822|4573|4|47| +2450822|4574|4|173| +2450822|4576|4|822| +2450822|4579|4|688| +2450822|4580|4|280| +2450822|4582|4|338| +2450822|4585|4|303| +2450822|4586|4|857| +2450822|4588|4|328| +2450822|4591|4|176| +2450822|4592|4|734| +2450822|4594|4|353| +2450822|4597|4|408| +2450822|4598|4|468| +2450822|4600|4|555| +2450822|4603|4|152| +2450822|4604|4|191| +2450822|4606|4|248| +2450822|4609|4|409| +2450822|4610|4|987| +2450822|4612|4|135| +2450822|4615|4|132| +2450822|4616|4|777| +2450822|4618|4|294| +2450822|4621|4|55| +2450822|4622|4|513| +2450822|4624|4|149| +2450822|4627|4|566| +2450822|4628|4|703| +2450822|4630|4|860| +2450822|4633|4|362| +2450822|4634|4|809| +2450822|4636|4|339| +2450822|4639|4|924| +2450822|4640|4|941| +2450822|4642|4|840| +2450822|4645|4|738| +2450822|4646|4|129| +2450822|4648|4|511| +2450822|4651|4|130| +2450822|4652|4|998| +2450822|4654|4|999| +2450822|4657|4|412| +2450822|4658|4|998| +2450822|4660|4|746| +2450822|4663|4|228| +2450822|4664|4|801| +2450822|4666|4|899| +2450822|4669|4|662| +2450822|4670|4|812| +2450822|4672|4|78| +2450822|4675|4|69| +2450822|4676|4|305| +2450822|4678|4|380| +2450822|4681|4|128| +2450822|4682|4|705| +2450822|4684|4|845| +2450822|4687|4|427| +2450822|4688|4|426| +2450822|4690|4|571| +2450822|4693|4|608| +2450822|4694|4|230| +2450822|4696|4|882| +2450822|4699|4|702| +2450822|4700|4|544| +2450822|4702|4|367| +2450822|4705|4|882| +2450822|4706|4|928| +2450822|4708|4|382| +2450822|4711|4|305| +2450822|4712|4|175| +2450822|4714|4|866| +2450822|4717|4|148| +2450822|4718|4|283| +2450822|4720|4|244| +2450822|4723|4|388| +2450822|4724|4|687| +2450822|4726|4|974| +2450822|4729|4|324| +2450822|4730|4|713| +2450822|4732|4|674| +2450822|4735|4|396| +2450822|4736|4|509| +2450822|4738|4|849| +2450822|4741|4|582| +2450822|4742|4|592| +2450822|4744|4|781| +2450822|4747|4|570| +2450822|4748|4|364| +2450822|4750|4|513| +2450822|4753|4|133| +2450822|4754|4|100| +2450822|4756|4|386| +2450822|4759|4|65| +2450822|4760|4|477| +2450822|4762|4|413| +2450822|4765|4|468| +2450822|4766|4|363| +2450822|4768|4|949| +2450822|4771|4|436| +2450822|4772|4|63| +2450822|4774|4|186| +2450822|4777|4|691| +2450822|4778|4|593| +2450822|4780|4|632| +2450822|4783|4|725| +2450822|4784|4|523| +2450822|4786|4|17| +2450822|4789|4|236| +2450822|4790|4|250| +2450822|4792|4|562| +2450822|4795|4|377| +2450822|4796|4|453| +2450822|4798|4|786| +2450822|4801|4|769| +2450822|4802|4|370| +2450822|4804|4|328| +2450822|4807|4|582| +2450822|4808|4|338| +2450822|4810|4|420| +2450822|4813|4|100| +2450822|4814|4|114| +2450822|4816|4|585| +2450822|4819|4|296| +2450822|4820|4|436| +2450822|4822|4|382| +2450822|4825|4|970| +2450822|4826|4|802| +2450822|4828|4|498| +2450822|4831|4|96| +2450822|4832|4|157| +2450822|4834|4|8| +2450822|4837|4|| +2450822|4838|4|40| +2450822|4840|4|113| +2450822|4843|4|76| +2450822|4844|4|133| +2450822|4846|4|746| +2450822|4849|4|271| +2450822|4850|4|628| +2450822|4852|4|20| +2450822|4855|4|240| +2450822|4856|4|827| +2450822|4858|4|841| +2450822|4861|4|831| +2450822|4862|4|30| +2450822|4864|4|214| +2450822|4867|4|284| +2450822|4868|4|176| +2450822|4870|4|452| +2450822|4873|4|95| +2450822|4874|4|186| +2450822|4876|4|357| +2450822|4879|4|968| +2450822|4880|4|980| +2450822|4882|4|821| +2450822|4885|4|796| +2450822|4886|4|763| +2450822|4888|4|483| +2450822|4891|4|285| +2450822|4892|4|535| +2450822|4894|4|523| +2450822|4897|4|684| +2450822|4898|4|221| +2450822|4900|4|391| +2450822|4903|4|779| +2450822|4904|4|738| +2450822|4906|4|151| +2450822|4909|4|921| +2450822|4910|4|334| +2450822|4912|4|168| +2450822|4915|4|273| +2450822|4916|4|429| +2450822|4918|4|857| +2450822|4921|4|925| +2450822|4922|4|793| +2450822|4924|4|388| +2450822|4927|4|767| +2450822|4928|4|318| +2450822|4930|4|624| +2450822|4933|4|509| +2450822|4934|4|| +2450822|4936|4|325| +2450822|4939|4|750| +2450822|4940|4|443| +2450822|4942|4|858| +2450822|4945|4|324| +2450822|4946|4|431| +2450822|4948|4|458| +2450822|4951|4|974| +2450822|4952|4|365| +2450822|4954|4|935| +2450822|4957|4|980| +2450822|4958|4|503| +2450822|4960|4|66| +2450822|4963|4|134| +2450822|4964|4|618| +2450822|4966|4|852| +2450822|4969|4|948| +2450822|4970|4|210| +2450822|4972|4|28| +2450822|4975|4|914| +2450822|4976|4|941| +2450822|4978|4|57| +2450822|4981|4|443| +2450822|4982|4|144| +2450822|4984|4|| +2450822|4987|4|377| +2450822|4988|4|118| +2450822|4990|4|819| +2450822|4993|4|313| +2450822|4994|4|| +2450822|4996|4|377| +2450822|4999|4|238| +2450822|5000|4|811| +2450822|5002|4|185| +2450822|5005|4|56| +2450822|5006|4|609| +2450822|5008|4|712| +2450822|5011|4|103| +2450822|5012|4|905| +2450822|5014|4|394| +2450822|5017|4|66| +2450822|5018|4|966| +2450822|5020|4|467| +2450822|5023|4|888| +2450822|5024|4|994| +2450822|5026|4|626| +2450822|5029|4|62| +2450822|5030|4|578| +2450822|5032|4|52| +2450822|5035|4|284| +2450822|5036|4|294| +2450822|5038|4|60| +2450822|5041|4|952| +2450822|5042|4|833| +2450822|5044|4|912| +2450822|5047|4|673| +2450822|5048|4|60| +2450822|5050|4|214| +2450822|5053|4|446| +2450822|5054|4|668| +2450822|5056|4|399| +2450822|5059|4|909| +2450822|5060|4|234| +2450822|5062|4|270| +2450822|5065|4|314| +2450822|5066|4|157| +2450822|5068|4|558| +2450822|5071|4|984| +2450822|5072|4|| +2450822|5074|4|855| +2450822|5077|4|117| +2450822|5078|4|107| +2450822|5080|4|999| +2450822|5083|4|| +2450822|5084|4|447| +2450822|5086|4|2| +2450822|5089|4|438| +2450822|5090|4|972| +2450822|5092|4|940| +2450822|5095|4|75| +2450822|5096|4|443| +2450822|5098|4|952| +2450822|5101|4|54| +2450822|5102|4|| +2450822|5104|4|650| +2450822|5107|4|613| +2450822|5108|4|561| +2450822|5110|4|12| +2450822|5113|4|669| +2450822|5114|4|876| +2450822|5116|4|890| +2450822|5119|4|659| +2450822|5120|4|289| +2450822|5122|4|491| +2450822|5125|4|| +2450822|5126|4|125| +2450822|5128|4|897| +2450822|5131|4|922| +2450822|5132|4|894| +2450822|5134|4|849| +2450822|5137|4|483| +2450822|5138|4|754| +2450822|5140|4|40| +2450822|5143|4|293| +2450822|5144|4|227| +2450822|5146|4|848| +2450822|5149|4|33| +2450822|5150|4|| +2450822|5152|4|507| +2450822|5155|4|767| +2450822|5156|4|964| +2450822|5158|4|824| +2450822|5161|4|120| +2450822|5162|4|64| +2450822|5164|4|110| +2450822|5167|4|177| +2450822|5168|4|| +2450822|5170|4|850| +2450822|5173|4|352| +2450822|5174|4|555| +2450822|5176|4|350| +2450822|5179|4|443| +2450822|5180|4|364| +2450822|5182|4|1000| +2450822|5185|4|662| +2450822|5186|4|324| +2450822|5188|4|107| +2450822|5191|4|988| +2450822|5192|4|480| +2450822|5194|4|119| +2450822|5197|4|35| +2450822|5198|4|142| +2450822|5200|4|830| +2450822|5203|4|924| +2450822|5204|4|914| +2450822|5206|4|835| +2450822|5209|4|680| +2450822|5210|4|992| +2450822|5212|4|| +2450822|5215|4|337| +2450822|5216|4|874| +2450822|5218|4|354| +2450822|5221|4|526| +2450822|5222|4|80| +2450822|5224|4|306| +2450822|5227|4|988| +2450822|5228|4|226| +2450822|5230|4|623| +2450822|5233|4|815| +2450822|5234|4|137| +2450822|5236|4|71| +2450822|5239|4|287| +2450822|5240|4|788| +2450822|5242|4|587| +2450822|5245|4|| +2450822|5246|4|969| +2450822|5248|4|630| +2450822|5251|4|355| +2450822|5252|4|868| +2450822|5254|4|356| +2450822|5257|4|641| +2450822|5258|4|337| +2450822|5260|4|125| +2450822|5263|4|568| +2450822|5264|4|797| +2450822|5266|4|439| +2450822|5269|4|93| +2450822|5270|4|820| +2450822|5272|4|228| +2450822|5275|4|299| +2450822|5276|4|494| +2450822|5278|4|793| +2450822|5281|4|255| +2450822|5282|4|281| +2450822|5284|4|161| +2450822|5287|4|718| +2450822|5288|4|310| +2450822|5290|4|824| +2450822|5293|4|843| +2450822|5294|4|548| +2450822|5296|4|| +2450822|5299|4|548| +2450822|5300|4|78| +2450822|5302|4|619| +2450822|5305|4|89| +2450822|5306|4|83| +2450822|5308|4|368| +2450822|5311|4|368| +2450822|5312|4|318| +2450822|5314|4|209| +2450822|5317|4|140| +2450822|5318|4|814| +2450822|5320|4|379| +2450822|5323|4|441| +2450822|5324|4|159| +2450822|5326|4|110| +2450822|5329|4|| +2450822|5330|4|156| +2450822|5332|4|647| +2450822|5335|4|766| +2450822|5336|4|974| +2450822|5338|4|582| +2450822|5341|4|62| +2450822|5342|4|331| +2450822|5344|4|225| +2450822|5347|4|966| +2450822|5348|4|25| +2450822|5350|4|662| +2450822|5353|4|975| +2450822|5354|4|309| +2450822|5356|4|269| +2450822|5359|4|128| +2450822|5360|4|583| +2450822|5362|4|752| +2450822|5365|4|651| +2450822|5366|4|439| +2450822|5368|4|140| +2450822|5371|4|72| +2450822|5372|4|285| +2450822|5374|4|76| +2450822|5377|4|| +2450822|5378|4|506| +2450822|5380|4|855| +2450822|5383|4|878| +2450822|5384|4|387| +2450822|5386|4|146| +2450822|5389|4|547| +2450822|5390|4|824| +2450822|5392|4|429| +2450822|5395|4|743| +2450822|5396|4|154| +2450822|5398|4|478| +2450822|5401|4|888| +2450822|5402|4|313| +2450822|5404|4|140| +2450822|5407|4|918| +2450822|5408|4|305| +2450822|5410|4|162| +2450822|5413|4|243| +2450822|5414|4|173| +2450822|5416|4|219| +2450822|5419|4|384| +2450822|5420|4|591| +2450822|5422|4|37| +2450822|5425|4|201| +2450822|5426|4|372| +2450822|5428|4|| +2450822|5431|4|271| +2450822|5432|4|385| +2450822|5434|4|337| +2450822|5437|4|639| +2450822|5438|4|262| +2450822|5440|4|657| +2450822|5443|4|20| +2450822|5444|4|962| +2450822|5446|4|203| +2450822|5449|4|883| +2450822|5450|4|874| +2450822|5452|4|612| +2450822|5455|4|715| +2450822|5456|4|441| +2450822|5458|4|764| +2450822|5461|4|595| +2450822|5462|4|330| +2450822|5464|4|968| +2450822|5467|4|1| +2450822|5468|4|371| +2450822|5470|4|214| +2450822|5473|4|| +2450822|5474|4|201| +2450822|5476|4|668| +2450822|5479|4|767| +2450822|5480|4|252| +2450822|5482|4|312| +2450822|5485|4|867| +2450822|5486|4|617| +2450822|5488|4|509| +2450822|5491|4|221| +2450822|5492|4|518| +2450822|5494|4|227| +2450822|5497|4|423| +2450822|5498|4|451| +2450822|5500|4|946| +2450822|5503|4|856| +2450822|5504|4|870| +2450822|5506|4|165| +2450822|5509|4|157| +2450822|5510|4|620| +2450822|5512|4|739| +2450822|5515|4|817| +2450822|5516|4|161| +2450822|5518|4|451| +2450822|5521|4|403| +2450822|5522|4|113| +2450822|5524|4|233| +2450822|5527|4|493| +2450822|5528|4|575| +2450822|5530|4|194| +2450822|5533|4|716| +2450822|5534|4|561| +2450822|5536|4|151| +2450822|5539|4|204| +2450822|5540|4|440| +2450822|5542|4|52| +2450822|5545|4|310| +2450822|5546|4|429| +2450822|5548|4|337| +2450822|5551|4|108| +2450822|5552|4|| +2450822|5554|4|104| +2450822|5557|4|671| +2450822|5558|4|| +2450822|5560|4|164| +2450822|5563|4|816| +2450822|5564|4|896| +2450822|5566|4|90| +2450822|5569|4|663| +2450822|5570|4|230| +2450822|5572|4|547| +2450822|5575|4|815| +2450822|5576|4|151| +2450822|5578|4|836| +2450822|5581|4|280| +2450822|5582|4|433| +2450822|5584|4|876| +2450822|5587|4|964| +2450822|5588|4|483| +2450822|5590|4|666| +2450822|5593|4|150| +2450822|5594|4|874| +2450822|5596|4|806| +2450822|5599|4|649| +2450822|5600|4|751| +2450822|5602|4|367| +2450822|5605|4|951| +2450822|5606|4|620| +2450822|5608|4|898| +2450822|5611|4|379| +2450822|5612|4|729| +2450822|5614|4|53| +2450822|5617|4|717| +2450822|5618|4|782| +2450822|5620|4|250| +2450822|5623|4|992| +2450822|5624|4|407| +2450822|5626|4|426| +2450822|5629|4|260| +2450822|5630|4|941| +2450822|5632|4|697| +2450822|5635|4|258| +2450822|5636|4|36| +2450822|5638|4|358| +2450822|5641|4|354| +2450822|5642|4|973| +2450822|5644|4|782| +2450822|5647|4|89| +2450822|5648|4|581| +2450822|5650|4|459| +2450822|5653|4|248| +2450822|5654|4|329| +2450822|5656|4|788| +2450822|5659|4|956| +2450822|5660|4|988| +2450822|5662|4|675| +2450822|5665|4|911| +2450822|5666|4|190| +2450822|5668|4|694| +2450822|5671|4|548| +2450822|5672|4|352| +2450822|5674|4|387| +2450822|5677|4|| +2450822|5678|4|922| +2450822|5680|4|718| +2450822|5683|4|42| +2450822|5684|4|251| +2450822|5686|4|616| +2450822|5689|4|622| +2450822|5690|4|710| +2450822|5692|4|270| +2450822|5695|4|324| +2450822|5696|4|139| +2450822|5698|4|229| +2450822|5701|4|828| +2450822|5702|4|382| +2450822|5704|4|639| +2450822|5707|4|386| +2450822|5708|4|479| +2450822|5710|4|856| +2450822|5713|4|510| +2450822|5714|4|985| +2450822|5716|4|95| +2450822|5719|4|998| +2450822|5720|4|431| +2450822|5722|4|853| +2450822|5725|4|380| +2450822|5726|4|464| +2450822|5728|4|558| +2450822|5731|4|| +2450822|5732|4|671| +2450822|5734|4|875| +2450822|5737|4|1| +2450822|5738|4|99| +2450822|5740|4|996| +2450822|5743|4|147| +2450822|5744|4|536| +2450822|5746|4|152| +2450822|5749|4|661| +2450822|5750|4|444| +2450822|5752|4|160| +2450822|5755|4|206| +2450822|5756|4|168| +2450822|5758|4|604| +2450822|5761|4|330| +2450822|5762|4|250| +2450822|5764|4|543| +2450822|5767|4|93| +2450822|5768|4|728| +2450822|5770|4|188| +2450822|5773|4|868| +2450822|5774|4|| +2450822|5776|4|779| +2450822|5779|4|746| +2450822|5780|4|772| +2450822|5782|4|971| +2450822|5785|4|189| +2450822|5786|4|| +2450822|5788|4|135| +2450822|5791|4|272| +2450822|5792|4|60| +2450822|5794|4|372| +2450822|5797|4|961| +2450822|5798|4|153| +2450822|5800|4|380| +2450822|5803|4|987| +2450822|5804|4|554| +2450822|5806|4|326| +2450822|5809|4|535| +2450822|5810|4|517| +2450822|5812|4|4| +2450822|5815|4|488| +2450822|5816|4|814| +2450822|5818|4|716| +2450822|5821|4|| +2450822|5822|4|259| +2450822|5824|4|962| +2450822|5827|4|685| +2450822|5828|4|313| +2450822|5830|4|872| +2450822|5833|4|16| +2450822|5834|4|919| +2450822|5836|4|936| +2450822|5839|4|205| +2450822|5840|4|256| +2450822|5842|4|549| +2450822|5845|4|83| +2450822|5846|4|883| +2450822|5848|4|189| +2450822|5851|4|343| +2450822|5852|4|7| +2450822|5854|4|903| +2450822|5857|4|383| +2450822|5858|4|899| +2450822|5860|4|| +2450822|5863|4|241| +2450822|5864|4|| +2450822|5866|4|368| +2450822|5869|4|39| +2450822|5870|4|| +2450822|5872|4|593| +2450822|5875|4|415| +2450822|5876|4|384| +2450822|5878|4|472| +2450822|5881|4|20| +2450822|5882|4|787| +2450822|5884|4|241| +2450822|5887|4|977| +2450822|5888|4|592| +2450822|5890|4|147| +2450822|5893|4|705| +2450822|5894|4|638| +2450822|5896|4|335| +2450822|5899|4|533| +2450822|5900|4|22| +2450822|5902|4|537| +2450822|5905|4|345| +2450822|5906|4|| +2450822|5908|4|523| +2450822|5911|4|919| +2450822|5912|4|360| +2450822|5914|4|528| +2450822|5917|4|451| +2450822|5918|4|359| +2450822|5920|4|23| +2450822|5923|4|727| +2450822|5924|4|689| +2450822|5926|4|891| +2450822|5929|4|357| +2450822|5930|4|304| +2450822|5932|4|421| +2450822|5935|4|| +2450822|5936|4|793| +2450822|5938|4|325| +2450822|5941|4|907| +2450822|5942|4|47| +2450822|5944|4|271| +2450822|5947|4|244| +2450822|5948|4|783| +2450822|5950|4|50| +2450822|5953|4|904| +2450822|5954|4|127| +2450822|5956|4|111| +2450822|5959|4|74| +2450822|5960|4|462| +2450822|5962|4|305| +2450822|5965|4|653| +2450822|5966|4|334| +2450822|5968|4|202| +2450822|5971|4|885| +2450822|5972|4|467| +2450822|5974|4|673| +2450822|5977|4|989| +2450822|5978|4|71| +2450822|5980|4|960| +2450822|5983|4|54| +2450822|5984|4|782| +2450822|5986|4|411| +2450822|5989|4|741| +2450822|5990|4|342| +2450822|5992|4|216| +2450822|5995|4|| +2450822|5996|4|16| +2450822|5998|4|290| +2450822|6001|4|824| +2450822|6002|4|770| +2450822|6004|4|103| +2450822|6007|4|786| +2450822|6008|4|279| +2450822|6010|4|579| +2450822|6013|4|607| +2450822|6014|4|418| +2450822|6016|4|124| +2450822|6019|4|523| +2450822|6020|4|681| +2450822|6022|4|524| +2450822|6025|4|169| +2450822|6026|4|700| +2450822|6028|4|478| +2450822|6031|4|98| +2450822|6032|4|74| +2450822|6034|4|321| +2450822|6037|4|803| +2450822|6038|4|343| +2450822|6040|4|801| +2450822|6043|4|540| +2450822|6044|4|754| +2450822|6046|4|| +2450822|6049|4|229| +2450822|6050|4|635| +2450822|6052|4|290| +2450822|6055|4|35| +2450822|6056|4|| +2450822|6058|4|230| +2450822|6061|4|543| +2450822|6062|4|310| +2450822|6064|4|90| +2450822|6067|4|878| +2450822|6068|4|857| +2450822|6070|4|254| +2450822|6073|4|317| +2450822|6074|4|672| +2450822|6076|4|398| +2450822|6079|4|387| +2450822|6080|4|261| +2450822|6082|4|429| +2450822|6085|4|216| +2450822|6086|4|592| +2450822|6088|4|601| +2450822|6091|4|167| +2450822|6092|4|676| +2450822|6094|4|648| +2450822|6097|4|624| +2450822|6098|4|314| +2450822|6100|4|669| +2450822|6103|4|558| +2450822|6104|4|891| +2450822|6106|4|492| +2450822|6109|4|292| +2450822|6110|4|490| +2450822|6112|4|874| +2450822|6115|4|458| +2450822|6116|4|527| +2450822|6118|4|960| +2450822|6121|4|981| +2450822|6122|4|696| +2450822|6124|4|899| +2450822|6127|4|869| +2450822|6128|4|427| +2450822|6130|4|459| +2450822|6133|4|144| +2450822|6134|4|976| +2450822|6136|4|416| +2450822|6139|4|505| +2450822|6140|4|160| +2450822|6142|4|873| +2450822|6145|4|261| +2450822|6146|4|734| +2450822|6148|4|549| +2450822|6151|4|791| +2450822|6152|4|504| +2450822|6154|4|957| +2450822|6157|4|523| +2450822|6158|4|530| +2450822|6160|4|224| +2450822|6163|4|998| +2450822|6164|4|236| +2450822|6166|4|7| +2450822|6169|4|411| +2450822|6170|4|242| +2450822|6172|4|797| +2450822|6175|4|623| +2450822|6176|4|358| +2450822|6178|4|25| +2450822|6181|4|38| +2450822|6182|4|| +2450822|6184|4|802| +2450822|6187|4|177| +2450822|6188|4|930| +2450822|6190|4|711| +2450822|6193|4|485| +2450822|6194|4|273| +2450822|6196|4|268| +2450822|6199|4|472| +2450822|6200|4|725| +2450822|6202|4|225| +2450822|6205|4|222| +2450822|6206|4|118| +2450822|6208|4|335| +2450822|6211|4|997| +2450822|6212|4|892| +2450822|6214|4|559| +2450822|6217|4|461| +2450822|6218|4|639| +2450822|6220|4|621| +2450822|6223|4|413| +2450822|6224|4|313| +2450822|6226|4|175| +2450822|6229|4|389| +2450822|6230|4|634| +2450822|6232|4|589| +2450822|6235|4|823| +2450822|6236|4|656| +2450822|6238|4|225| +2450822|6241|4|102| +2450822|6242|4|944| +2450822|6244|4|273| +2450822|6247|4|179| +2450822|6248|4|325| +2450822|6250|4|763| +2450822|6253|4|606| +2450822|6254|4|932| +2450822|6256|4|562| +2450822|6259|4|965| +2450822|6260|4|414| +2450822|6262|4|21| +2450822|6265|4|306| +2450822|6266|4|719| +2450822|6268|4|290| +2450822|6271|4|935| +2450822|6272|4|382| +2450822|6274|4|773| +2450822|6277|4|875| +2450822|6278|4|159| +2450822|6280|4|752| +2450822|6283|4|437| +2450822|6284|4|81| +2450822|6286|4|405| +2450822|6289|4|740| +2450822|6290|4|421| +2450822|6292|4|435| +2450822|6295|4|125| +2450822|6296|4|147| +2450822|6298|4|938| +2450822|6301|4|889| +2450822|6302|4|188| +2450822|6304|4|861| +2450822|6307|4|314| +2450822|6308|4|131| +2450822|6310|4|79| +2450822|6313|4|96| +2450822|6314|4|29| +2450822|6316|4|176| +2450822|6319|4|645| +2450822|6320|4|343| +2450822|6322|4|305| +2450822|6325|4|76| +2450822|6326|4|195| +2450822|6328|4|761| +2450822|6331|4|221| +2450822|6332|4|521| +2450822|6334|4|90| +2450822|6337|4|746| +2450822|6338|4|854| +2450822|6340|4|264| +2450822|6343|4|713| +2450822|6344|4|153| +2450822|6346|4|654| +2450822|6349|4|48| +2450822|6350|4|| +2450822|6352|4|55| +2450822|6355|4|668| +2450822|6356|4|755| +2450822|6358|4|531| +2450822|6361|4|53| +2450822|6362|4|998| +2450822|6364|4|663| +2450822|6367|4|511| +2450822|6368|4|857| +2450822|6370|4|776| +2450822|6373|4|| +2450822|6374|4|126| +2450822|6376|4|694| +2450822|6379|4|| +2450822|6380|4|778| +2450822|6382|4|736| +2450822|6385|4|676| +2450822|6386|4|758| +2450822|6388|4|835| +2450822|6391|4|817| +2450822|6392|4|457| +2450822|6394|4|672| +2450822|6397|4|870| +2450822|6398|4|809| +2450822|6400|4|635| +2450822|6403|4|95| +2450822|6404|4|935| +2450822|6406|4|33| +2450822|6409|4|902| +2450822|6410|4|867| +2450822|6412|4|165| +2450822|6415|4|529| +2450822|6416|4|| +2450822|6418|4|187| +2450822|6421|4|581| +2450822|6422|4|28| +2450822|6424|4|275| +2450822|6427|4|| +2450822|6428|4|847| +2450822|6430|4|| +2450822|6433|4|720| +2450822|6434|4|927| +2450822|6436|4|218| +2450822|6439|4|458| +2450822|6440|4|259| +2450822|6442|4|217| +2450822|6445|4|505| +2450822|6446|4|357| +2450822|6448|4|158| +2450822|6451|4|446| +2450822|6452|4|963| +2450822|6454|4|172| +2450822|6457|4|558| +2450822|6458|4|274| +2450822|6460|4|219| +2450822|6463|4|676| +2450822|6464|4|144| +2450822|6466|4|367| +2450822|6469|4|842| +2450822|6470|4|358| +2450822|6472|4|413| +2450822|6475|4|602| +2450822|6476|4|992| +2450822|6478|4|364| +2450822|6481|4|697| +2450822|6482|4|883| +2450822|6484|4|237| +2450822|6487|4|69| +2450822|6488|4|| +2450822|6490|4|249| +2450822|6493|4|500| +2450822|6494|4|513| +2450822|6496|4|392| +2450822|6499|4|913| +2450822|6500|4|495| +2450822|6502|4|360| +2450822|6505|4|193| +2450822|6506|4|296| +2450822|6508|4|196| +2450822|6511|4|642| +2450822|6512|4|418| +2450822|6514|4|176| +2450822|6517|4|67| +2450822|6518|4|779| +2450822|6520|4|872| +2450822|6523|4|658| +2450822|6524|4|884| +2450822|6526|4|434| +2450822|6529|4|861| +2450822|6530|4|390| +2450822|6532|4|416| +2450822|6535|4|436| +2450822|6536|4|822| +2450822|6538|4|629| +2450822|6541|4|775| +2450822|6542|4|903| +2450822|6544|4|173| +2450822|6547|4|998| +2450822|6548|4|15| +2450822|6550|4|700| +2450822|6553|4|362| +2450822|6554|4|757| +2450822|6556|4|649| +2450822|6559|4|706| +2450822|6560|4|633| +2450822|6562|4|895| +2450822|6565|4|317| +2450822|6566|4|106| +2450822|6568|4|665| +2450822|6571|4|140| +2450822|6572|4|384| +2450822|6574|4|30| +2450822|6577|4|318| +2450822|6578|4|0| +2450822|6580|4|750| +2450822|6583|4|884| +2450822|6584|4|647| +2450822|6586|4|338| +2450822|6589|4|498| +2450822|6590|4|81| +2450822|6592|4|97| +2450822|6595|4|597| +2450822|6596|4|985| +2450822|6598|4|32| +2450822|6601|4|384| +2450822|6602|4|772| +2450822|6604|4|701| +2450822|6607|4|788| +2450822|6608|4|915| +2450822|6610|4|| +2450822|6613|4|223| +2450822|6614|4|421| +2450822|6616|4|566| +2450822|6619|4|461| +2450822|6620|4|416| +2450822|6622|4|719| +2450822|6625|4|135| +2450822|6626|4|| +2450822|6628|4|556| +2450822|6631|4|878| +2450822|6632|4|487| +2450822|6634|4|472| +2450822|6637|4|226| +2450822|6638|4|354| +2450822|6640|4|860| +2450822|6643|4|948| +2450822|6644|4|309| +2450822|6646|4|489| +2450822|6649|4|435| +2450822|6650|4|392| +2450822|6652|4|64| +2450822|6655|4|713| +2450822|6656|4|87| +2450822|6658|4|551| +2450822|6661|4|441| +2450822|6662|4|543| +2450822|6664|4|277| +2450822|6667|4|365| +2450822|6668|4|708| +2450822|6670|4|559| +2450822|6673|4|584| +2450822|6674|4|611| +2450822|6676|4|976| +2450822|6679|4|165| +2450822|6680|4|296| +2450822|6682|4|927| +2450822|6685|4|393| +2450822|6686|4|472| +2450822|6688|4|367| +2450822|6691|4|926| +2450822|6692|4|181| +2450822|6694|4|45| +2450822|6697|4|412| +2450822|6698|4|439| +2450822|6700|4|660| +2450822|6703|4|58| +2450822|6704|4|142| +2450822|6706|4|464| +2450822|6709|4|927| +2450822|6710|4|259| +2450822|6712|4|24| +2450822|6715|4|| +2450822|6716|4|| +2450822|6718|4|827| +2450822|6721|4|536| +2450822|6722|4|136| +2450822|6724|4|824| +2450822|6727|4|410| +2450822|6728|4|260| +2450822|6730|4|481| +2450822|6733|4|322| +2450822|6734|4|780| +2450822|6736|4|550| +2450822|6739|4|847| +2450822|6740|4|479| +2450822|6742|4|125| +2450822|6745|4|| +2450822|6746|4|638| +2450822|6748|4|7| +2450822|6751|4|147| +2450822|6752|4|465| +2450822|6754|4|790| +2450822|6757|4|752| +2450822|6758|4|433| +2450822|6760|4|903| +2450822|6763|4|566| +2450822|6764|4|824| +2450822|6766|4|364| +2450822|6769|4|388| +2450822|6770|4|475| +2450822|6772|4|360| +2450822|6775|4|| +2450822|6776|4|921| +2450822|6778|4|540| +2450822|6781|4|58| +2450822|6782|4|337| +2450822|6784|4|441| +2450822|6787|4|264| +2450822|6788|4|799| +2450822|6790|4|207| +2450822|6793|4|627| +2450822|6794|4|724| +2450822|6796|4|700| +2450822|6799|4|534| +2450822|6800|4|866| +2450822|6802|4|324| +2450822|6805|4|750| +2450822|6806|4|983| +2450822|6808|4|922| +2450822|6811|4|932| +2450822|6812|4|354| +2450822|6814|4|523| +2450822|6817|4|502| +2450822|6818|4|300| +2450822|6820|4|580| +2450822|6823|4|347| +2450822|6824|4|558| +2450822|6826|4|| +2450822|6829|4|106| +2450822|6830|4|249| +2450822|6832|4|| +2450822|6835|4|| +2450822|6836|4|| +2450822|6838|4|833| +2450822|6841|4|758| +2450822|6842|4|533| +2450822|6844|4|114| +2450822|6847|4|466| +2450822|6848|4|912| +2450822|6850|4|171| +2450822|6853|4|13| +2450822|6854|4|685| +2450822|6856|4|782| +2450822|6859|4|709| +2450822|6860|4|51| +2450822|6862|4|626| +2450822|6865|4|687| +2450822|6866|4|147| +2450822|6868|4|127| +2450822|6871|4|422| +2450822|6872|4|805| +2450822|6874|4|| +2450822|6877|4|423| +2450822|6878|4|695| +2450822|6880|4|23| +2450822|6883|4|807| +2450822|6884|4|152| +2450822|6886|4|528| +2450822|6889|4|299| +2450822|6890|4|743| +2450822|6892|4|371| +2450822|6895|4|16| +2450822|6896|4|711| +2450822|6898|4|215| +2450822|6901|4|255| +2450822|6902|4|484| +2450822|6904|4|370| +2450822|6907|4|383| +2450822|6908|4|165| +2450822|6910|4|877| +2450822|6913|4|260| +2450822|6914|4|373| +2450822|6916|4|278| +2450822|6919|4|750| +2450822|6920|4|193| +2450822|6922|4|863| +2450822|6925|4|859| +2450822|6926|4|937| +2450822|6928|4|77| +2450822|6931|4|941| +2450822|6932|4|960| +2450822|6934|4|611| +2450822|6937|4|425| +2450822|6938|4|857| +2450822|6940|4|196| +2450822|6943|4|326| +2450822|6944|4|568| +2450822|6946|4|422| +2450822|6949|4|382| +2450822|6950|4|136| +2450822|6952|4|280| +2450822|6955|4|136| +2450822|6956|4|426| +2450822|6958|4|615| +2450822|6961|4|221| +2450822|6962|4|129| +2450822|6964|4|78| +2450822|6967|4|382| +2450822|6968|4|676| +2450822|6970|4|442| +2450822|6973|4|790| +2450822|6974|4|928| +2450822|6976|4|743| +2450822|6979|4|348| +2450822|6980|4|386| +2450822|6982|4|203| +2450822|6985|4|136| +2450822|6986|4|88| +2450822|6988|4|786| +2450822|6991|4|923| +2450822|6992|4|3| +2450822|6994|4|669| +2450822|6997|4|976| +2450822|6998|4|| +2450822|7000|4|325| +2450822|7003|4|600| +2450822|7004|4|544| +2450822|7006|4|357| +2450822|7009|4|996| +2450822|7010|4|945| +2450822|7012|4|| +2450822|7015|4|755| +2450822|7016|4|298| +2450822|7018|4|16| +2450822|7021|4|675| +2450822|7022|4|734| +2450822|7024|4|757| +2450822|7027|4|182| +2450822|7028|4|703| +2450822|7030|4|94| +2450822|7033|4|820| +2450822|7034|4|57| +2450822|7036|4|| +2450822|7039|4|938| +2450822|7040|4|988| +2450822|7042|4|1000| +2450822|7045|4|463| +2450822|7046|4|775| +2450822|7048|4|298| +2450822|7051|4|743| +2450822|7052|4|| +2450822|7054|4|8| +2450822|7057|4|992| +2450822|7058|4|703| +2450822|7060|4|913| +2450822|7063|4|121| +2450822|7064|4|208| +2450822|7066|4|308| +2450822|7069|4|740| +2450822|7070|4|832| +2450822|7072|4|769| +2450822|7075|4|487| +2450822|7076|4|21| +2450822|7078|4|594| +2450822|7081|4|280| +2450822|7082|4|326| +2450822|7084|4|512| +2450822|7087|4|158| +2450822|7088|4|905| +2450822|7090|4|89| +2450822|7093|4|971| +2450822|7094|4|751| +2450822|7096|4|| +2450822|7099|4|287| +2450822|7100|4|876| +2450822|7102|4|269| +2450822|7105|4|793| +2450822|7106|4|410| +2450822|7108|4|778| +2450822|7111|4|906| +2450822|7112|4|133| +2450822|7114|4|951| +2450822|7117|4|612| +2450822|7118|4|| +2450822|7120|4|364| +2450822|7123|4|488| +2450822|7124|4|995| +2450822|7126|4|988| +2450822|7129|4|378| +2450822|7130|4|250| +2450822|7132|4|320| +2450822|7135|4|780| +2450822|7136|4|| +2450822|7138|4|| +2450822|7141|4|377| +2450822|7142|4|105| +2450822|7144|4|527| +2450822|7147|4|| +2450822|7148|4|20| +2450822|7150|4|666| +2450822|7153|4|539| +2450822|7154|4|167| +2450822|7156|4|886| +2450822|7159|4|178| +2450822|7160|4|123| +2450822|7162|4|172| +2450822|7165|4|932| +2450822|7166|4|731| +2450822|7168|4|61| +2450822|7171|4|282| +2450822|7172|4|263| +2450822|7174|4|744| +2450822|7177|4|772| +2450822|7178|4|6| +2450822|7180|4|942| +2450822|7183|4|276| +2450822|7184|4|912| +2450822|7186|4|| +2450822|7189|4|585| +2450822|7190|4|759| +2450822|7192|4|100| +2450822|7195|4|593| +2450822|7196|4|143| +2450822|7198|4|| +2450822|7201|4|668| +2450822|7202|4|309| +2450822|7204|4|997| +2450822|7207|4|126| +2450822|7208|4|779| +2450822|7210|4|956| +2450822|7213|4|226| +2450822|7214|4|576| +2450822|7216|4|776| +2450822|7219|4|302| +2450822|7220|4|693| +2450822|7222|4|171| +2450822|7225|4|303| +2450822|7226|4|958| +2450822|7228|4|648| +2450822|7231|4|251| +2450822|7232|4|514| +2450822|7234|4|77| +2450822|7237|4|534| +2450822|7238|4|876| +2450822|7240|4|796| +2450822|7243|4|135| +2450822|7244|4|478| +2450822|7246|4|773| +2450822|7249|4|18| +2450822|7250|4|| +2450822|7252|4|148| +2450822|7255|4|498| +2450822|7256|4|750| +2450822|7258|4|420| +2450822|7261|4|236| +2450822|7262|4|253| +2450822|7264|4|784| +2450822|7267|4|726| +2450822|7268|4|351| +2450822|7270|4|801| +2450822|7273|4|866| +2450822|7274|4|361| +2450822|7276|4|667| +2450822|7279|4|565| +2450822|7280|4|835| +2450822|7282|4|168| +2450822|7285|4|953| +2450822|7286|4|773| +2450822|7288|4|851| +2450822|7291|4|6| +2450822|7292|4|472| +2450822|7294|4|152| +2450822|7297|4|239| +2450822|7298|4|286| +2450822|7300|4|412| +2450822|7303|4|567| +2450822|7304|4|467| +2450822|7306|4|413| +2450822|7309|4|261| +2450822|7310|4|686| +2450822|7312|4|883| +2450822|7315|4|164| +2450822|7316|4|29| +2450822|7318|4|177| +2450822|7321|4|837| +2450822|7322|4|899| +2450822|7324|4|79| +2450822|7327|4|408| +2450822|7328|4|7| +2450822|7330|4|454| +2450822|7333|4|| +2450822|7334|4|825| +2450822|7336|4|114| +2450822|7339|4|928| +2450822|7340|4|788| +2450822|7342|4|603| +2450822|7345|4|411| +2450822|7346|4|575| +2450822|7348|4|294| +2450822|7351|4|184| +2450822|7352|4|405| +2450822|7354|4|| +2450822|7357|4|758| +2450822|7358|4|450| +2450822|7360|4|645| +2450822|7363|4|756| +2450822|7364|4|632| +2450822|7366|4|177| +2450822|7369|4|220| +2450822|7370|4|836| +2450822|7372|4|717| +2450822|7375|4|984| +2450822|7376|4|207| +2450822|7378|4|199| +2450822|7381|4|929| +2450822|7382|4|885| +2450822|7384|4|675| +2450822|7387|4|684| +2450822|7388|4|987| +2450822|7390|4|979| +2450822|7393|4|897| +2450822|7394|4|578| +2450822|7396|4|463| +2450822|7399|4|542| +2450822|7400|4|816| +2450822|7402|4|270| +2450822|7405|4|989| +2450822|7406|4|271| +2450822|7408|4|909| +2450822|7411|4|826| +2450822|7412|4|821| +2450822|7414|4|286| +2450822|7417|4|| +2450822|7418|4|348| +2450822|7420|4|759| +2450822|7423|4|603| +2450822|7424|4|519| +2450822|7426|4|251| +2450822|7429|4|634| +2450822|7430|4|871| +2450822|7432|4|873| +2450822|7435|4|338| +2450822|7436|4|398| +2450822|7438|4|434| +2450822|7441|4|207| +2450822|7442|4|975| +2450822|7444|4|837| +2450822|7447|4|645| +2450822|7448|4|148| +2450822|7450|4|481| +2450822|7453|4|793| +2450822|7454|4|676| +2450822|7456|4|99| +2450822|7459|4|162| +2450822|7460|4|652| +2450822|7462|4|44| +2450822|7465|4|876| +2450822|7466|4|383| +2450822|7468|4|503| +2450822|7471|4|95| +2450822|7472|4|229| +2450822|7474|4|635| +2450822|7477|4|597| +2450822|7478|4|734| +2450822|7480|4|546| +2450822|7483|4|331| +2450822|7484|4|676| +2450822|7486|4|510| +2450822|7489|4|228| +2450822|7490|4|696| +2450822|7492|4|499| +2450822|7495|4|156| +2450822|7496|4|942| +2450822|7498|4|51| +2450822|7501|4|124| +2450822|7502|4|303| +2450822|7504|4|703| +2450822|7507|4|748| +2450822|7508|4|533| +2450822|7510|4|315| +2450822|7513|4|517| +2450822|7514|4|498| +2450822|7516|4|| +2450822|7519|4|862| +2450822|7520|4|120| +2450822|7522|4|237| +2450822|7525|4|52| +2450822|7526|4|| +2450822|7528|4|367| +2450822|7531|4|976| +2450822|7532|4|103| +2450822|7534|4|797| +2450822|7537|4|| +2450822|7538|4|103| +2450822|7540|4|927| +2450822|7543|4|381| +2450822|7544|4|| +2450822|7546|4|981| +2450822|7549|4|195| +2450822|7550|4|536| +2450822|7552|4|654| +2450822|7555|4|424| +2450822|7556|4|206| +2450822|7558|4|912| +2450822|7561|4|609| +2450822|7562|4|11| +2450822|7564|4|807| +2450822|7567|4|332| +2450822|7568|4|402| +2450822|7570|4|717| +2450822|7573|4|921| +2450822|7574|4|694| +2450822|7576|4|505| +2450822|7579|4|673| +2450822|7580|4|563| +2450822|7582|4|900| +2450822|7585|4|119| +2450822|7586|4|793| +2450822|7588|4|989| +2450822|7591|4|| +2450822|7592|4|904| +2450822|7594|4|7| +2450822|7597|4|21| +2450822|7598|4|| +2450822|7600|4|312| +2450822|7603|4|| +2450822|7604|4|487| +2450822|7606|4|254| +2450822|7609|4|277| +2450822|7610|4|822| +2450822|7612|4|883| +2450822|7615|4|54| +2450822|7616|4|| +2450822|7618|4|155| +2450822|7621|4|498| +2450822|7622|4|205| +2450822|7624|4|131| +2450822|7627|4|476| +2450822|7628|4|383| +2450822|7630|4|595| +2450822|7633|4|225| +2450822|7634|4|534| +2450822|7636|4|568| +2450822|7639|4|369| +2450822|7640|4|452| +2450822|7642|4|82| +2450822|7645|4|556| +2450822|7646|4|777| +2450822|7648|4|596| +2450822|7651|4|| +2450822|7652|4|750| +2450822|7654|4|383| +2450822|7657|4|376| +2450822|7658|4|600| +2450822|7660|4|| +2450822|7663|4|470| +2450822|7664|4|373| +2450822|7666|4|714| +2450822|7669|4|88| +2450822|7670|4|791| +2450822|7672|4|| +2450822|7675|4|629| +2450822|7676|4|546| +2450822|7678|4|989| +2450822|7681|4|161| +2450822|7682|4|22| +2450822|7684|4|| +2450822|7687|4|221| +2450822|7688|4|| +2450822|7690|4|698| +2450822|7693|4|962| +2450822|7694|4|870| +2450822|7696|4|528| +2450822|7699|4|372| +2450822|7700|4|792| +2450822|7702|4|197| +2450822|7705|4|727| +2450822|7706|4|143| +2450822|7708|4|560| +2450822|7711|4|948| +2450822|7712|4|883| +2450822|7714|4|369| +2450822|7717|4|174| +2450822|7718|4|407| +2450822|7720|4|988| +2450822|7723|4|364| +2450822|7724|4|681| +2450822|7726|4|532| +2450822|7729|4|903| +2450822|7730|4|| +2450822|7732|4|762| +2450822|7735|4|983| +2450822|7736|4|416| +2450822|7738|4|581| +2450822|7741|4|193| +2450822|7742|4|388| +2450822|7744|4|710| +2450822|7747|4|726| +2450822|7748|4|854| +2450822|7750|4|861| +2450822|7753|4|851| +2450822|7754|4|996| +2450822|7756|4|262| +2450822|7759|4|| +2450822|7760|4|728| +2450822|7762|4|430| +2450822|7765|4|990| +2450822|7766|4|155| +2450822|7768|4|143| +2450822|7771|4|106| +2450822|7772|4|990| +2450822|7774|4|421| +2450822|7777|4|| +2450822|7778|4|736| +2450822|7780|4|316| +2450822|7783|4|716| +2450822|7784|4|810| +2450822|7786|4|662| +2450822|7789|4|837| +2450822|7790|4|473| +2450822|7792|4|512| +2450822|7795|4|679| +2450822|7796|4|97| +2450822|7798|4|776| +2450822|7801|4|281| +2450822|7802|4|884| +2450822|7804|4|129| +2450822|7807|4|376| +2450822|7808|4|49| +2450822|7810|4|222| +2450822|7813|4|437| +2450822|7814|4|154| +2450822|7816|4|497| +2450822|7819|4|814| +2450822|7820|4|954| +2450822|7822|4|238| +2450822|7825|4|262| +2450822|7826|4|543| +2450822|7828|4|968| +2450822|7831|4|508| +2450822|7832|4|874| +2450822|7834|4|746| +2450822|7837|4|813| +2450822|7838|4|126| +2450822|7840|4|438| +2450822|7843|4|58| +2450822|7844|4|| +2450822|7846|4|170| +2450822|7849|4|50| +2450822|7850|4|441| +2450822|7852|4|113| +2450822|7855|4|581| +2450822|7856|4|276| +2450822|7858|4|246| +2450822|7861|4|668| +2450822|7862|4|462| +2450822|7864|4|363| +2450822|7867|4|236| +2450822|7868|4|252| +2450822|7870|4|785| +2450822|7873|4|531| +2450822|7874|4|500| +2450822|7876|4|689| +2450822|7879|4|893| +2450822|7880|4|286| +2450822|7882|4|297| +2450822|7885|4|613| +2450822|7886|4|175| +2450822|7888|4|105| +2450822|7891|4|719| +2450822|7892|4|793| +2450822|7894|4|26| +2450822|7897|4|825| +2450822|7898|4|436| +2450822|7900|4|722| +2450822|7903|4|280| +2450822|7904|4|924| +2450822|7906|4|739| +2450822|7909|4|578| +2450822|7910|4|807| +2450822|7912|4|85| +2450822|7915|4|884| +2450822|7916|4|274| +2450822|7918|4|265| +2450822|7921|4|152| +2450822|7922|4|261| +2450822|7924|4|990| +2450822|7927|4|60| +2450822|7928|4|238| +2450822|7930|4|616| +2450822|7933|4|681| +2450822|7934|4|114| +2450822|7936|4|633| +2450822|7939|4|394| +2450822|7940|4|936| +2450822|7942|4|| +2450822|7945|4|| +2450822|7946|4|801| +2450822|7948|4|814| +2450822|7951|4|148| +2450822|7952|4|579| +2450822|7954|4|680| +2450822|7957|4|881| +2450822|7958|4|947| +2450822|7960|4|386| +2450822|7963|4|163| +2450822|7964|4|251| +2450822|7966|4|426| +2450822|7969|4|195| +2450822|7970|4|294| +2450822|7972|4|289| +2450822|7975|4|226| +2450822|7976|4|736| +2450822|7978|4|401| +2450822|7981|4|741| +2450822|7982|4|675| +2450822|7984|4|834| +2450822|7987|4|551| +2450822|7988|4|636| +2450822|7990|4|790| +2450822|7993|4|696| +2450822|7994|4|913| +2450822|7996|4|136| +2450822|7999|4|422| +2450822|8000|4|215| +2450822|8002|4|23| +2450822|8005|4|715| +2450822|8006|4|614| +2450822|8008|4|973| +2450822|8011|4|899| +2450822|8012|4|638| +2450822|8014|4|277| +2450822|8017|4|607| +2450822|8018|4|171| +2450822|8020|4|512| +2450822|8023|4|665| +2450822|8024|4|672| +2450822|8026|4|211| +2450822|8029|4|858| +2450822|8030|4|103| +2450822|8032|4|588| +2450822|8035|4|497| +2450822|8036|4|609| +2450822|8038|4|352| +2450822|8041|4|| +2450822|8042|4|256| +2450822|8044|4|904| +2450822|8047|4|281| +2450822|8048|4|365| +2450822|8050|4|895| +2450822|8053|4|404| +2450822|8054|4|415| +2450822|8056|4|295| +2450822|8059|4|259| +2450822|8060|4|134| +2450822|8062|4|969| +2450822|8065|4|935| +2450822|8066|4|337| +2450822|8068|4|959| +2450822|8071|4|984| +2450822|8072|4|65| +2450822|8074|4|305| +2450822|8077|4|726| +2450822|8078|4|430| +2450822|8080|4|269| +2450822|8083|4|249| +2450822|8084|4|972| +2450822|8086|4|929| +2450822|8089|4|615| +2450822|8090|4|242| +2450822|8092|4|636| +2450822|8095|4|0| +2450822|8096|4|499| +2450822|8098|4|298| +2450822|8101|4|726| +2450822|8102|4|71| +2450822|8104|4|514| +2450822|8107|4|159| +2450822|8108|4|60| +2450822|8110|4|798| +2450822|8113|4|94| +2450822|8114|4|11| +2450822|8116|4|116| +2450822|8119|4|590| +2450822|8120|4|898| +2450822|8122|4|513| +2450822|8125|4|190| +2450822|8126|4|803| +2450822|8128|4|57| +2450822|8131|4|428| +2450822|8132|4|613| +2450822|8134|4|997| +2450822|8137|4|832| +2450822|8138|4|134| +2450822|8140|4|506| +2450822|8143|4|190| +2450822|8144|4|9| +2450822|8146|4|643| +2450822|8149|4|| +2450822|8150|4|75| +2450822|8152|4|941| +2450822|8155|4|945| +2450822|8156|4|968| +2450822|8158|4|284| +2450822|8161|4|229| +2450822|8162|4|163| +2450822|8164|4|683| +2450822|8167|4|80| +2450822|8168|4|695| +2450822|8170|4|434| +2450822|8173|4|376| +2450822|8174|4|996| +2450822|8176|4|13| +2450822|8179|4|655| +2450822|8180|4|671| +2450822|8182|4|791| +2450822|8185|4|365| +2450822|8186|4|50| +2450822|8188|4|514| +2450822|8191|4|343| +2450822|8192|4|904| +2450822|8194|4|934| +2450822|8197|4|568| +2450822|8198|4|776| +2450822|8200|4|595| +2450822|8203|4|335| +2450822|8204|4|727| +2450822|8206|4|416| +2450822|8209|4|775| +2450822|8210|4|733| +2450822|8212|4|613| +2450822|8215|4|139| +2450822|8216|4|979| +2450822|8218|4|402| +2450822|8221|4|896| +2450822|8222|4|58| +2450822|8224|4|793| +2450822|8227|4|255| +2450822|8228|4|861| +2450822|8230|4|188| +2450822|8233|4|724| +2450822|8234|4|| +2450822|8236|4|789| +2450822|8239|4|507| +2450822|8240|4|927| +2450822|8242|4|986| +2450822|8245|4|983| +2450822|8246|4|257| +2450822|8248|4|599| +2450822|8251|4|218| +2450822|8252|4|79| +2450822|8254|4|577| +2450822|8257|4|524| +2450822|8258|4|54| +2450822|8260|4|910| +2450822|8263|4|3| +2450822|8264|4|606| +2450822|8266|4|640| +2450822|8269|4|504| +2450822|8270|4|180| +2450822|8272|4|9| +2450822|8275|4|191| +2450822|8276|4|174| +2450822|8278|4|614| +2450822|8281|4|58| +2450822|8282|4|141| +2450822|8284|4|361| +2450822|8287|4|695| +2450822|8288|4|369| +2450822|8290|4|259| +2450822|8293|4|| +2450822|8294|4|444| +2450822|8296|4|414| +2450822|8299|4|15| +2450822|8300|4|925| +2450822|8302|4|8| +2450822|8305|4|394| +2450822|8306|4|561| +2450822|8308|4|903| +2450822|8311|4|233| +2450822|8312|4|357| +2450822|8314|4|420| +2450822|8317|4|| +2450822|8318|4|932| +2450822|8320|4|419| +2450822|8323|4|309| +2450822|8324|4|54| +2450822|8326|4|346| +2450822|8329|4|669| +2450822|8330|4|52| +2450822|8332|4|151| +2450822|8335|4|408| +2450822|8336|4|524| +2450822|8338|4|413| +2450822|8341|4|| +2450822|8342|4|393| +2450822|8344|4|956| +2450822|8347|4|686| +2450822|8348|4|356| +2450822|8350|4|672| +2450822|8353|4|283| +2450822|8354|4|703| +2450822|8356|4|641| +2450822|8359|4|171| +2450822|8360|4|715| +2450822|8362|4|136| +2450822|8365|4|883| +2450822|8366|4|843| +2450822|8368|4|982| +2450822|8371|4|788| +2450822|8372|4|783| +2450822|8374|4|324| +2450822|8377|4|524| +2450822|8378|4|462| +2450822|8380|4|963| +2450822|8383|4|637| +2450822|8384|4|838| +2450822|8386|4|5| +2450822|8389|4|410| +2450822|8390|4|| +2450822|8392|4|22| +2450822|8395|4|130| +2450822|8396|4|820| +2450822|8398|4|962| +2450822|8401|4|964| +2450822|8402|4|72| +2450822|8404|4|272| +2450822|8407|4|37| +2450822|8408|4|476| +2450822|8410|4|926| +2450822|8413|4|201| +2450822|8414|4|604| +2450822|8416|4|104| +2450822|8419|4|| +2450822|8420|4|| +2450822|8422|4|155| +2450822|8425|4|431| +2450822|8426|4|831| +2450822|8428|4|187| +2450822|8431|4|698| +2450822|8432|4|719| +2450822|8434|4|454| +2450822|8437|4|244| +2450822|8438|4|919| +2450822|8440|4|292| +2450822|8443|4|220| +2450822|8444|4|| +2450822|8446|4|755| +2450822|8449|4|546| +2450822|8450|4|659| +2450822|8452|4|210| +2450822|8455|4|63| +2450822|8456|4|72| +2450822|8458|4|398| +2450822|8461|4|435| +2450822|8462|4|512| +2450822|8464|4|158| +2450822|8467|4|338| +2450822|8468|4|679| +2450822|8470|4|485| +2450822|8473|4|413| +2450822|8474|4|946| +2450822|8476|4|261| +2450822|8479|4|421| +2450822|8480|4|686| +2450822|8482|4|602| +2450822|8485|4|2| +2450822|8486|4|526| +2450822|8488|4|372| +2450822|8491|4|54| +2450822|8492|4|804| +2450822|8494|4|986| +2450822|8497|4|417| +2450822|8498|4|753| +2450822|8500|4|677| +2450822|8503|4|500| +2450822|8504|4|12| +2450822|8506|4|163| +2450822|8509|4|564| +2450822|8510|4|871| +2450822|8512|4|909| +2450822|8515|4|554| +2450822|8516|4|784| +2450822|8518|4|514| +2450822|8521|4|151| +2450822|8522|4|590| +2450822|8524|4|144| +2450822|8527|4|368| +2450822|8528|4|708| +2450822|8530|4|49| +2450822|8533|4|133| +2450822|8534|4|| +2450822|8536|4|692| +2450822|8539|4|702| +2450822|8540|4|907| +2450822|8542|4|487| +2450822|8545|4|622| +2450822|8546|4|569| +2450822|8548|4|8| +2450822|8551|4|656| +2450822|8552|4|171| +2450822|8554|4|561| +2450822|8557|4|442| +2450822|8558|4|987| +2450822|8560|4|760| +2450822|8563|4|145| +2450822|8564|4|127| +2450822|8566|4|374| +2450822|8569|4|| +2450822|8570|4|428| +2450822|8572|4|132| +2450822|8575|4|254| +2450822|8576|4|335| +2450822|8578|4|80| +2450822|8581|4|277| +2450822|8582|4|492| +2450822|8584|4|830| +2450822|8587|4|245| +2450822|8588|4|634| +2450822|8590|4|692| +2450822|8593|4|534| +2450822|8594|4|661| +2450822|8596|4|36| +2450822|8599|4|952| +2450822|8600|4|803| +2450822|8602|4|38| +2450822|8605|4|360| +2450822|8606|4|927| +2450822|8608|4|685| +2450822|8611|4|439| +2450822|8612|4|37| +2450822|8614|4|517| +2450822|8617|4|370| +2450822|8618|4|990| +2450822|8620|4|459| +2450822|8623|4|416| +2450822|8624|4|| +2450822|8626|4|438| +2450822|8629|4|76| +2450822|8630|4|229| +2450822|8632|4|982| +2450822|8635|4|994| +2450822|8636|4|208| +2450822|8638|4|178| +2450822|8641|4|323| +2450822|8642|4|235| +2450822|8644|4|268| +2450822|8647|4|964| +2450822|8648|4|512| +2450822|8650|4|264| +2450822|8653|4|809| +2450822|8654|4|995| +2450822|8656|4|706| +2450822|8659|4|106| +2450822|8660|4|304| +2450822|8662|4|525| +2450822|8665|4|362| +2450822|8666|4|97| +2450822|8668|4|506| +2450822|8671|4|698| +2450822|8672|4|971| +2450822|8674|4|498| +2450822|8677|4|31| +2450822|8678|4|825| +2450822|8680|4|995| +2450822|8683|4|179| +2450822|8684|4|620| +2450822|8686|4|118| +2450822|8689|4|36| +2450822|8690|4|640| +2450822|8692|4|39| +2450822|8695|4|665| +2450822|8696|4|648| +2450822|8698|4|648| +2450822|8701|4|79| +2450822|8702|4|372| +2450822|8704|4|325| +2450822|8707|4|217| +2450822|8708|4|349| +2450822|8710|4|876| +2450822|8713|4|874| +2450822|8714|4|776| +2450822|8716|4|925| +2450822|8719|4|445| +2450822|8720|4|465| +2450822|8722|4|121| +2450822|8725|4|911| +2450822|8726|4|317| +2450822|8728|4|629| +2450822|8731|4|114| +2450822|8732|4|330| +2450822|8734|4|439| +2450822|8737|4|12| +2450822|8738|4|7| +2450822|8740|4|| +2450822|8743|4|| +2450822|8744|4|358| +2450822|8746|4|764| +2450822|8749|4|| +2450822|8750|4|0| +2450822|8752|4|975| +2450822|8755|4|491| +2450822|8756|4|780| +2450822|8758|4|601| +2450822|8761|4|135| +2450822|8762|4|115| +2450822|8764|4|481| +2450822|8767|4|324| +2450822|8768|4|45| +2450822|8770|4|645| +2450822|8773|4|279| +2450822|8774|4|68| +2450822|8776|4|| +2450822|8779|4|423| +2450822|8780|4|560| +2450822|8782|4|551| +2450822|8785|4|365| +2450822|8786|4|121| +2450822|8788|4|807| +2450822|8791|4|946| +2450822|8792|4|587| +2450822|8794|4|149| +2450822|8797|4|56| +2450822|8798|4|975| +2450822|8800|4|33| +2450822|8803|4|153| +2450822|8804|4|252| +2450822|8806|4|757| +2450822|8809|4|889| +2450822|8810|4|406| +2450822|8812|4|637| +2450822|8815|4|753| +2450822|8816|4|429| +2450822|8818|4|726| +2450822|8821|4|390| +2450822|8822|4|867| +2450822|8824|4|931| +2450822|8827|4|56| +2450822|8828|4|| +2450822|8830|4|588| +2450822|8833|4|568| +2450822|8834|4|240| +2450822|8836|4|926| +2450822|8839|4|717| +2450822|8840|4|968| +2450822|8842|4|793| +2450822|8845|4|118| +2450822|8846|4|533| +2450822|8848|4|416| +2450822|8851|4|926| +2450822|8852|4|669| +2450822|8854|4|475| +2450822|8857|4|767| +2450822|8858|4|315| +2450822|8860|4|374| +2450822|8863|4|680| +2450822|8864|4|314| +2450822|8866|4|838| +2450822|8869|4|248| +2450822|8870|4|808| +2450822|8872|4|289| +2450822|8875|4|962| +2450822|8876|4|183| +2450822|8878|4|134| +2450822|8881|4|803| +2450822|8882|4|| +2450822|8884|4|928| +2450822|8887|4|422| +2450822|8888|4|706| +2450822|8890|4|546| +2450822|8893|4|579| +2450822|8894|4|45| +2450822|8896|4|104| +2450822|8899|4|236| +2450822|8900|4|727| +2450822|8902|4|525| +2450822|8905|4|385| +2450822|8906|4|988| +2450822|8908|4|550| +2450822|8911|4|189| +2450822|8912|4|156| +2450822|8914|4|369| +2450822|8917|4|397| +2450822|8918|4|822| +2450822|8920|4|192| +2450822|8923|4|762| +2450822|8924|4|810| +2450822|8926|4|| +2450822|8929|4|186| +2450822|8930|4|71| +2450822|8932|4|811| +2450822|8935|4|889| +2450822|8936|4|309| +2450822|8938|4|104| +2450822|8941|4|851| +2450822|8942|4|787| +2450822|8944|4|595| +2450822|8947|4|| +2450822|8948|4|886| +2450822|8950|4|221| +2450822|8953|4|938| +2450822|8954|4|960| +2450822|8956|4|340| +2450822|8959|4|194| +2450822|8960|4|683| +2450822|8962|4|185| +2450822|8965|4|540| +2450822|8966|4|100| +2450822|8968|4|560| +2450822|8971|4|629| +2450822|8972|4|865| +2450822|8974|4|714| +2450822|8977|4|433| +2450822|8978|4|116| +2450822|8980|4|640| +2450822|8983|4|206| +2450822|8984|4|744| +2450822|8986|4|891| +2450822|8989|4|122| +2450822|8990|4|| +2450822|8992|4|883| +2450822|8995|4|961| +2450822|8996|4|834| +2450822|8998|4|172| +2450822|9001|4|266| +2450822|9002|4|808| +2450822|9004|4|35| +2450822|9007|4|371| +2450822|9008|4|969| +2450822|9010|4|341| +2450822|9013|4|867| +2450822|9014|4|254| +2450822|9016|4|413| +2450822|9019|4|| +2450822|9020|4|| +2450822|9022|4|96| +2450822|9025|4|800| +2450822|9026|4|971| +2450822|9028|4|676| +2450822|9031|4|90| +2450822|9032|4|102| +2450822|9034|4|349| +2450822|9037|4|6| +2450822|9038|4|394| +2450822|9040|4|669| +2450822|9043|4|698| +2450822|9044|4|16| +2450822|9046|4|841| +2450822|9049|4|271| +2450822|9050|4|934| +2450822|9052|4|512| +2450822|9055|4|503| +2450822|9056|4|271| +2450822|9058|4|207| +2450822|9061|4|| +2450822|9062|4|778| +2450822|9064|4|501| +2450822|9067|4|| +2450822|9068|4|547| +2450822|9070|4|90| +2450822|9073|4|333| +2450822|9074|4|726| +2450822|9076|4|442| +2450822|9079|4|585| +2450822|9080|4|809| +2450822|9082|4|996| +2450822|9085|4|280| +2450822|9086|4|813| +2450822|9088|4|806| +2450822|9091|4|204| +2450822|9092|4|618| +2450822|9094|4|862| +2450822|9097|4|808| +2450822|9098|4|269| +2450822|9100|4|453| +2450822|9103|4|98| +2450822|9104|4|0| +2450822|9106|4|932| +2450822|9109|4|574| +2450822|9110|4|503| +2450822|9112|4|323| +2450822|9115|4|567| +2450822|9116|4|630| +2450822|9118|4|604| +2450822|9121|4|398| +2450822|9122|4|864| +2450822|9124|4|386| +2450822|9127|4|513| +2450822|9128|4|934| +2450822|9130|4|426| +2450822|9133|4|468| +2450822|9134|4|369| +2450822|9136|4|770| +2450822|9139|4|375| +2450822|9140|4|| +2450822|9142|4|976| +2450822|9145|4|61| +2450822|9146|4|981| +2450822|9148|4|280| +2450822|9151|4|472| +2450822|9152|4|601| +2450822|9154|4|449| +2450822|9157|4|675| +2450822|9158|4|935| +2450822|9160|4|139| +2450822|9163|4|551| +2450822|9164|4|805| +2450822|9166|4|91| +2450822|9169|4|674| +2450822|9170|4|945| +2450822|9172|4|290| +2450822|9175|4|842| +2450822|9176|4|924| +2450822|9178|4|335| +2450822|9181|4|599| +2450822|9182|4|611| +2450822|9184|4|880| +2450822|9187|4|896| +2450822|9188|4|251| +2450822|9190|4|298| +2450822|9193|4|494| +2450822|9194|4|183| +2450822|9196|4|10| +2450822|9199|4|954| +2450822|9200|4|709| +2450822|9202|4|744| +2450822|9205|4|132| +2450822|9206|4|397| +2450822|9208|4|170| +2450822|9211|4|612| +2450822|9212|4|122| +2450822|9214|4|464| +2450822|9217|4|996| +2450822|9218|4|563| +2450822|9220|4|392| +2450822|9223|4|131| +2450822|9224|4|464| +2450822|9226|4|622| +2450822|9229|4|688| +2450822|9230|4|671| +2450822|9232|4|458| +2450822|9235|4|114| +2450822|9236|4|143| +2450822|9238|4|140| +2450822|9241|4|282| +2450822|9242|4|92| +2450822|9244|4|127| +2450822|9247|4|508| +2450822|9248|4|774| +2450822|9250|4|756| +2450822|9253|4|393| +2450822|9254|4|195| +2450822|9256|4|477| +2450822|9259|4|889| +2450822|9260|4|303| +2450822|9262|4|923| +2450822|9265|4|329| +2450822|9266|4|605| +2450822|9268|4|620| +2450822|9271|4|952| +2450822|9272|4|121| +2450822|9274|4|527| +2450822|9277|4|910| +2450822|9278|4|636| +2450822|9280|4|918| +2450822|9283|4|909| +2450822|9284|4|658| +2450822|9286|4|100| +2450822|9289|4|938| +2450822|9290|4|280| +2450822|9292|4|38| +2450822|9295|4|894| +2450822|9296|4|608| +2450822|9298|4|602| +2450822|9301|4|218| +2450822|9302|4|37| +2450822|9304|4|937| +2450822|9307|4|649| +2450822|9308|4|| +2450822|9310|4|227| +2450822|9313|4|17| +2450822|9314|4|360| +2450822|9316|4|407| +2450822|9319|4|134| +2450822|9320|4|401| +2450822|9322|4|408| +2450822|9325|4|540| +2450822|9326|4|954| +2450822|9328|4|565| +2450822|9331|4|36| +2450822|9332|4|473| +2450822|9334|4|541| +2450822|9337|4|557| +2450822|9338|4|34| +2450822|9340|4|412| +2450822|9343|4|826| +2450822|9344|4|958| +2450822|9346|4|699| +2450822|9349|4|466| +2450822|9350|4|675| +2450822|9352|4|225| +2450822|9355|4|464| +2450822|9356|4|280| +2450822|9358|4|229| +2450822|9361|4|429| +2450822|9362|4|374| +2450822|9364|4|232| +2450822|9367|4|81| +2450822|9368|4|5| +2450822|9370|4|585| +2450822|9373|4|761| +2450822|9374|4|958| +2450822|9376|4|996| +2450822|9379|4|190| +2450822|9380|4|679| +2450822|9382|4|100| +2450822|9385|4|44| +2450822|9386|4|303| +2450822|9388|4|| +2450822|9391|4|50| +2450822|9392|4|| +2450822|9394|4|443| +2450822|9397|4|193| +2450822|9398|4|214| +2450822|9400|4|523| +2450822|9403|4|370| +2450822|9404|4|521| +2450822|9406|4|243| +2450822|9409|4|| +2450822|9410|4|252| +2450822|9412|4|264| +2450822|9415|4|97| +2450822|9416|4|731| +2450822|9418|4|203| +2450822|9421|4|366| +2450822|9422|4|53| +2450822|9424|4|| +2450822|9427|4|574| +2450822|9428|4|| +2450822|9430|4|899| +2450822|9433|4|940| +2450822|9434|4|203| +2450822|9436|4|497| +2450822|9439|4|226| +2450822|9440|4|65| +2450822|9442|4|435| +2450822|9445|4|506| +2450822|9446|4|798| +2450822|9448|4|568| +2450822|9451|4|878| +2450822|9452|4|110| +2450822|9454|4|| +2450822|9457|4|719| +2450822|9458|4|498| +2450822|9460|4|90| +2450822|9463|4|99| +2450822|9464|4|299| +2450822|9466|4|781| +2450822|9469|4|459| +2450822|9470|4|775| +2450822|9472|4|989| +2450822|9475|4|337| +2450822|9476|4|872| +2450822|9478|4|714| +2450822|9481|4|99| +2450822|9482|4|| +2450822|9484|4|831| +2450822|9487|4|540| +2450822|9488|4|43| +2450822|9490|4|422| +2450822|9493|4|14| +2450822|9494|4|48| +2450822|9496|4|250| +2450822|9499|4|555| +2450822|9500|4|539| +2450822|9502|4|931| +2450822|9505|4|412| +2450822|9506|4|908| +2450822|9508|4|496| +2450822|9511|4|316| +2450822|9512|4|642| +2450822|9514|4|638| +2450822|9517|4|37| +2450822|9518|4|833| +2450822|9520|4|250| +2450822|9523|4|837| +2450822|9524|4|408| +2450822|9526|4|53| +2450822|9529|4|499| +2450822|9530|4|684| +2450822|9532|4|605| +2450822|9535|4|823| +2450822|9536|4|928| +2450822|9538|4|345| +2450822|9541|4|4| +2450822|9542|4|95| +2450822|9544|4|492| +2450822|9547|4|795| +2450822|9548|4|134| +2450822|9550|4|94| +2450822|9553|4|349| +2450822|9554|4|521| +2450822|9556|4|| +2450822|9559|4|889| +2450822|9560|4|0| +2450822|9562|4|204| +2450822|9565|4|261| +2450822|9566|4|822| +2450822|9568|4|223| +2450822|9571|4|624| +2450822|9572|4|| +2450822|9574|4|361| +2450822|9577|4|9| +2450822|9578|4|| +2450822|9580|4|70| +2450822|9583|4|951| +2450822|9584|4|277| +2450822|9586|4|531| +2450822|9589|4|924| +2450822|9590|4|689| +2450822|9592|4|610| +2450822|9595|4|565| +2450822|9596|4|847| +2450822|9598|4|520| +2450822|9601|4|59| +2450822|9602|4|86| +2450822|9604|4|402| +2450822|9607|4|488| +2450822|9608|4|794| +2450822|9610|4|237| +2450822|9613|4|648| +2450822|9614|4|522| +2450822|9616|4|996| +2450822|9619|4|985| +2450822|9620|4|658| +2450822|9622|4|860| +2450822|9625|4|574| +2450822|9626|4|246| +2450822|9628|4|795| +2450822|9631|4|536| +2450822|9632|4|503| +2450822|9634|4|808| +2450822|9637|4|686| +2450822|9638|4|| +2450822|9640|4|616| +2450822|9643|4|622| +2450822|9644|4|772| +2450822|9646|4|663| +2450822|9649|4|653| +2450822|9650|4|734| +2450822|9652|4|416| +2450822|9655|4|210| +2450822|9656|4|878| +2450822|9658|4|| +2450822|9661|4|553| +2450822|9662|4|41| +2450822|9664|4|712| +2450822|9667|4|846| +2450822|9668|4|68| +2450822|9670|4|319| +2450822|9673|4|816| +2450822|9674|4|96| +2450822|9676|4|534| +2450822|9679|4|465| +2450822|9680|4|547| +2450822|9682|4|742| +2450822|9685|4|700| +2450822|9686|4|170| +2450822|9688|4|347| +2450822|9691|4|437| +2450822|9692|4|998| +2450822|9694|4|616| +2450822|9697|4|434| +2450822|9698|4|880| +2450822|9700|4|304| +2450822|9703|4|122| +2450822|9704|4|87| +2450822|9706|4|102| +2450822|9709|4|878| +2450822|9710|4|64| +2450822|9712|4|382| +2450822|9715|4|581| +2450822|9716|4|136| +2450822|9718|4|345| +2450822|9721|4|459| +2450822|9722|4|551| +2450822|9724|4|967| +2450822|9727|4|243| +2450822|9728|4|431| +2450822|9730|4|151| +2450822|9733|4|174| +2450822|9734|4|910| +2450822|9736|4|242| +2450822|9739|4|527| +2450822|9740|4|125| +2450822|9742|4|627| +2450822|9745|4|305| +2450822|9746|4|735| +2450822|9748|4|846| +2450822|9751|4|718| +2450822|9752|4|425| +2450822|9754|4|230| +2450822|9757|4|389| +2450822|9758|4|797| +2450822|9760|4|585| +2450822|9763|4|782| +2450822|9764|4|332| +2450822|9766|4|921| +2450822|9769|4|168| +2450822|9770|4|948| +2450822|9772|4|700| +2450822|9775|4|804| +2450822|9776|4|169| +2450822|9778|4|788| +2450822|9781|4|786| +2450822|9782|4|764| +2450822|9784|4|477| +2450822|9787|4|196| +2450822|9788|4|541| +2450822|9790|4|723| +2450822|9793|4|373| +2450822|9794|4|614| +2450822|9796|4|948| +2450822|9799|4|346| +2450822|9800|4|87| +2450822|9802|4|198| +2450822|9805|4|960| +2450822|9806|4|636| +2450822|9808|4|222| +2450822|9811|4|16| +2450822|9812|4|9| +2450822|9814|4|799| +2450822|9817|4|716| +2450822|9818|4|| +2450822|9820|4|38| +2450822|9823|4|864| +2450822|9824|4|666| +2450822|9826|4|947| +2450822|9829|4|564| +2450822|9830|4|235| +2450822|9832|4|576| +2450822|9835|4|393| +2450822|9836|4|177| +2450822|9838|4|65| +2450822|9841|4|564| +2450822|9842|4|845| +2450822|9844|4|510| +2450822|9847|4|757| +2450822|9848|4|914| +2450822|9850|4|81| +2450822|9853|4|13| +2450822|9854|4|66| +2450822|9856|4|869| +2450822|9859|4|788| +2450822|9860|4|194| +2450822|9862|4|| +2450822|9865|4|347| +2450822|9866|4|573| +2450822|9868|4|413| +2450822|9871|4|5| +2450822|9872|4|13| +2450822|9874|4|14| +2450822|9877|4|294| +2450822|9878|4|313| +2450822|9880|4|629| +2450822|9883|4|530| +2450822|9884|4|673| +2450822|9886|4|421| +2450822|9889|4|277| +2450822|9890|4|712| +2450822|9892|4|20| +2450822|9895|4|588| +2450822|9896|4|936| +2450822|9898|4|536| +2450822|9901|4|617| +2450822|9902|4|711| +2450822|9904|4|610| +2450822|9907|4|419| +2450822|9908|4|374| +2450822|9910|4|583| +2450822|9913|4|87| +2450822|9914|4|111| +2450822|9916|4|164| +2450822|9919|4|99| +2450822|9920|4|379| +2450822|9922|4|60| +2450822|9925|4|224| +2450822|9926|4|578| +2450822|9928|4|118| +2450822|9931|4|681| +2450822|9932|4|186| +2450822|9934|4|122| +2450822|9937|4|617| +2450822|9938|4|905| +2450822|9940|4|950| +2450822|9943|4|588| +2450822|9944|4|390| +2450822|9946|4|528| +2450822|9949|4|968| +2450822|9950|4|154| +2450822|9952|4|158| +2450822|9955|4|430| +2450822|9956|4|| +2450822|9958|4|54| +2450822|9961|4|3| +2450822|9962|4|501| +2450822|9964|4|705| +2450822|9967|4|929| +2450822|9968|4|467| +2450822|9970|4|156| +2450822|9973|4|225| +2450822|9974|4|245| +2450822|9976|4|544| +2450822|9979|4|882| +2450822|9980|4|409| +2450822|9982|4|46| +2450822|9985|4|409| +2450822|9986|4|914| +2450822|9988|4|580| +2450822|9991|4|40| +2450822|9992|4|199| +2450822|9994|4|629| +2450822|9997|4|692| +2450822|9998|4|508| +2450822|10000|4|979| +2450822|10003|4|605| +2450822|10004|4|778| +2450822|10006|4|213| +2450822|10009|4|701| +2450822|10010|4|849| +2450822|10012|4|406| +2450822|10015|4|378| +2450822|10016|4|354| +2450822|10018|4|816| +2450822|10021|4|330| +2450822|10022|4|491| +2450822|10024|4|111| +2450822|10027|4|652| +2450822|10028|4|722| +2450822|10030|4|148| +2450822|10033|4|158| +2450822|10034|4|| +2450822|10036|4|701| +2450822|10039|4|| +2450822|10040|4|960| +2450822|10042|4|544| +2450822|10045|4|592| +2450822|10046|4|922| +2450822|10048|4|409| +2450822|10051|4|3| +2450822|10052|4|722| +2450822|10054|4|921| +2450822|10057|4|192| +2450822|10058|4|200| +2450822|10060|4|663| +2450822|10063|4|943| +2450822|10064|4|487| +2450822|10066|4|166| +2450822|10069|4|146| +2450822|10070|4|484| +2450822|10072|4|164| +2450822|10075|4|748| +2450822|10076|4|445| +2450822|10078|4|158| +2450822|10081|4|771| +2450822|10082|4|39| +2450822|10084|4|645| +2450822|10087|4|721| +2450822|10088|4|939| +2450822|10090|4|580| +2450822|10093|4|169| +2450822|10094|4|180| +2450822|10096|4|44| +2450822|10099|4|461| +2450822|10100|4|524| +2450822|10102|4|780| +2450822|10105|4|164| +2450822|10106|4|693| +2450822|10108|4|913| +2450822|10111|4|| +2450822|10112|4|506| +2450822|10114|4|570| +2450822|10117|4|914| +2450822|10118|4|816| +2450822|10120|4|834| +2450822|10123|4|265| +2450822|10124|4|522| +2450822|10126|4|329| +2450822|10129|4|967| +2450822|10130|4|573| +2450822|10132|4|288| +2450822|10135|4|127| +2450822|10136|4|651| +2450822|10138|4|796| +2450822|10141|4|| +2450822|10142|4|960| +2450822|10144|4|401| +2450822|10147|4|953| +2450822|10148|4|527| +2450822|10150|4|797| +2450822|10153|4|928| +2450822|10154|4|995| +2450822|10156|4|| +2450822|10159|4|693| +2450822|10160|4|708| +2450822|10162|4|898| +2450822|10165|4|496| +2450822|10166|4|144| +2450822|10168|4|113| +2450822|10171|4|| +2450822|10172|4|504| +2450822|10174|4|164| +2450822|10177|4|657| +2450822|10178|4|354| +2450822|10180|4|366| +2450822|10183|4|727| +2450822|10184|4|286| +2450822|10186|4|512| +2450822|10189|4|170| +2450822|10190|4|380| +2450822|10192|4|520| +2450822|10195|4|980| +2450822|10196|4|903| +2450822|10198|4|814| +2450822|10201|4|23| +2450822|10202|4|971| +2450822|10204|4|200| +2450822|10207|4|323| +2450822|10208|4|| +2450822|10210|4|108| +2450822|10213|4|509| +2450822|10214|4|859| +2450822|10216|4|367| +2450822|10219|4|493| +2450822|10220|4|615| +2450822|10222|4|109| +2450822|10225|4|755| +2450822|10226|4|369| +2450822|10228|4|905| +2450822|10231|4|100| +2450822|10232|4|78| +2450822|10234|4|666| +2450822|10237|4|55| +2450822|10238|4|912| +2450822|10240|4|747| +2450822|10243|4|7| +2450822|10244|4|248| +2450822|10246|4|241| +2450822|10249|4|496| +2450822|10250|4|86| +2450822|10252|4|240| +2450822|10255|4|809| +2450822|10256|4|901| +2450822|10258|4|848| +2450822|10261|4|265| +2450822|10262|4|625| +2450822|10264|4|389| +2450822|10267|4|460| +2450822|10268|4|193| +2450822|10270|4|964| +2450822|10273|4|961| +2450822|10274|4|| +2450822|10276|4|| +2450822|10279|4|628| +2450822|10280|4|483| +2450822|10282|4|73| +2450822|10285|4|478| +2450822|10286|4|40| +2450822|10288|4|559| +2450822|10291|4|942| +2450822|10292|4|575| +2450822|10294|4|879| +2450822|10297|4|422| +2450822|10298|4|558| +2450822|10300|4|173| +2450822|10303|4|221| +2450822|10304|4|242| +2450822|10306|4|931| +2450822|10309|4|551| +2450822|10310|4|932| +2450822|10312|4|426| +2450822|10315|4|794| +2450822|10316|4|50| +2450822|10318|4|740| +2450822|10321|4|| +2450822|10322|4|85| +2450822|10324|4|210| +2450822|10327|4|387| +2450822|10328|4|777| +2450822|10330|4|601| +2450822|10333|4|481| +2450822|10334|4|984| +2450822|10336|4|713| +2450822|10339|4|902| +2450822|10340|4|853| +2450822|10342|4|411| +2450822|10345|4|910| +2450822|10346|4|627| +2450822|10348|4|898| +2450822|10351|4|509| +2450822|10352|4|698| +2450822|10354|4|22| +2450822|10357|4|16| +2450822|10358|4|514| +2450822|10360|4|171| +2450822|10363|4|500| +2450822|10364|4|855| +2450822|10366|4|601| +2450822|10369|4|155| +2450822|10370|4|853| +2450822|10372|4|785| +2450822|10375|4|657| +2450822|10376|4|362| +2450822|10378|4|382| +2450822|10381|4|772| +2450822|10382|4|750| +2450822|10384|4|875| +2450822|10387|4|910| +2450822|10388|4|199| +2450822|10390|4|459| +2450822|10393|4|495| +2450822|10394|4|401| +2450822|10396|4|387| +2450822|10399|4|223| +2450822|10400|4|331| +2450822|10402|4|346| +2450822|10405|4|414| +2450822|10406|4|778| +2450822|10408|4|67| +2450822|10411|4|92| +2450822|10412|4|857| +2450822|10414|4|990| +2450822|10417|4|254| +2450822|10418|4|268| +2450822|10420|4|235| +2450822|10423|4|232| +2450822|10424|4|176| +2450822|10426|4|225| +2450822|10429|4|406| +2450822|10430|4|709| +2450822|10432|4|17| +2450822|10435|4|550| +2450822|10436|4|97| +2450822|10438|4|336| +2450822|10441|4|623| +2450822|10442|4|460| +2450822|10444|4|189| +2450822|10447|4|182| +2450822|10448|4|188| +2450822|10450|4|555| +2450822|10453|4|| +2450822|10454|4|931| +2450822|10456|4|268| +2450822|10459|4|630| +2450822|10460|4|947| +2450822|10462|4|800| +2450822|10465|4|35| +2450822|10466|4|592| +2450822|10468|4|| +2450822|10471|4|184| +2450822|10472|4|1000| +2450822|10474|4|710| +2450822|10477|4|837| +2450822|10478|4|982| +2450822|10480|4|545| +2450822|10483|4|150| +2450822|10484|4|810| +2450822|10486|4|617| +2450822|10489|4|373| +2450822|10490|4|972| +2450822|10492|4|736| +2450822|10495|4|701| +2450822|10496|4|471| +2450822|10498|4|804| +2450822|10501|4|50| +2450822|10502|4|453| +2450822|10504|4|622| +2450822|10507|4|981| +2450822|10508|4|995| +2450822|10510|4|354| +2450822|10513|4|758| +2450822|10514|4|282| +2450822|10516|4|618| +2450822|10519|4|17| +2450822|10520|4|962| +2450822|10522|4|767| +2450822|10525|4|| +2450822|10526|4|332| +2450822|10528|4|662| +2450822|10531|4|824| +2450822|10532|4|969| +2450822|10534|4|363| +2450822|10537|4|772| +2450822|10538|4|722| +2450822|10540|4|778| +2450822|10543|4|969| +2450822|10544|4|6| +2450822|10546|4|401| +2450822|10549|4|878| +2450822|10550|4|611| +2450822|10552|4|115| +2450822|10555|4|270| +2450822|10556|4|98| +2450822|10558|4|946| +2450822|10561|4|120| +2450822|10562|4|659| +2450822|10564|4|105| +2450822|10567|4|183| +2450822|10568|4|503| +2450822|10570|4|832| +2450822|10573|4|88| +2450822|10574|4|212| +2450822|10576|4|238| +2450822|10579|4|97| +2450822|10580|4|889| +2450822|10582|4|630| +2450822|10585|4|598| +2450822|10586|4|844| +2450822|10588|4|438| +2450822|10591|4|590| +2450822|10592|4|317| +2450822|10594|4|697| +2450822|10597|4|419| +2450822|10598|4|579| +2450822|10600|4|287| +2450822|10603|4|156| +2450822|10604|4|335| +2450822|10606|4|714| +2450822|10609|4|448| +2450822|10610|4|624| +2450822|10612|4|604| +2450822|10615|4|989| +2450822|10616|4|128| +2450822|10618|4|962| +2450822|10621|4|641| +2450822|10622|4|494| +2450822|10624|4|622| +2450822|10627|4|256| +2450822|10628|4|639| +2450822|10630|4|458| +2450822|10633|4|799| +2450822|10634|4|823| +2450822|10636|4|461| +2450822|10639|4|450| +2450822|10640|4|728| +2450822|10642|4|533| +2450822|10645|4|951| +2450822|10646|4|648| +2450822|10648|4|292| +2450822|10651|4|898| +2450822|10652|4|155| +2450822|10654|4|134| +2450822|10657|4|876| +2450822|10658|4|862| +2450822|10660|4|857| +2450822|10663|4|42| +2450822|10664|4|412| +2450822|10666|4|859| +2450822|10669|4|776| +2450822|10670|4|63| +2450822|10672|4|697| +2450822|10675|4|113| +2450822|10676|4|630| +2450822|10678|4|610| +2450822|10681|4|434| +2450822|10682|4|398| +2450822|10684|4|217| +2450822|10687|4|122| +2450822|10688|4|508| +2450822|10690|4|907| +2450822|10693|4|997| +2450822|10694|4|659| +2450822|10696|4|16| +2450822|10699|4|747| +2450822|10700|4|479| +2450822|10702|4|149| +2450822|10705|4|950| +2450822|10706|4|474| +2450822|10708|4|757| +2450822|10711|4|658| +2450822|10712|4|426| +2450822|10714|4|746| +2450822|10717|4|326| +2450822|10718|4|944| +2450822|10720|4|284| +2450822|10723|4|442| +2450822|10724|4|233| +2450822|10726|4|253| +2450822|10729|4|531| +2450822|10730|4|778| +2450822|10732|4|381| +2450822|10735|4|864| +2450822|10736|4|99| +2450822|10738|4|571| +2450822|10741|4|642| +2450822|10742|4|719| +2450822|10744|4|508| +2450822|10747|4|243| +2450822|10748|4|702| +2450822|10750|4|939| +2450822|10753|4|119| +2450822|10754|4|672| +2450822|10756|4|877| +2450822|10759|4|306| +2450822|10760|4|955| +2450822|10762|4|833| +2450822|10765|4|443| +2450822|10766|4|| +2450822|10768|4|813| +2450822|10771|4|718| +2450822|10772|4|722| +2450822|10774|4|262| +2450822|10777|4|| +2450822|10778|4|468| +2450822|10780|4|663| +2450822|10783|4|93| +2450822|10784|4|376| +2450822|10786|4|734| +2450822|10789|4|203| +2450822|10790|4|648| +2450822|10792|4|872| +2450822|10795|4|467| +2450822|10796|4|571| +2450822|10798|4|842| +2450822|10801|4|670| +2450822|10802|4|590| +2450822|10804|4|440| +2450822|10807|4|733| +2450822|10808|4|371| +2450822|10810|4|549| +2450822|10813|4|517| +2450822|10814|4|180| +2450822|10816|4|965| +2450822|10819|4|957| +2450822|10820|4|326| +2450822|10822|4|40| +2450822|10825|4|910| +2450822|10826|4|| +2450822|10828|4|588| +2450822|10831|4|886| +2450822|10832|4|766| +2450822|10834|4|546| +2450822|10837|4|514| +2450822|10838|4|184| +2450822|10840|4|367| +2450822|10843|4|| +2450822|10844|4|96| +2450822|10846|4|665| +2450822|10849|4|554| +2450822|10850|4|28| +2450822|10852|4|240| +2450822|10855|4|56| +2450822|10856|4|147| +2450822|10858|4|163| +2450822|10861|4|608| +2450822|10862|4|841| +2450822|10864|4|12| +2450822|10867|4|138| +2450822|10868|4|305| +2450822|10870|4|46| +2450822|10873|4|681| +2450822|10874|4|576| +2450822|10876|4|| +2450822|10879|4|999| +2450822|10880|4|| +2450822|10882|4|341| +2450822|10885|4|100| +2450822|10886|4|| +2450822|10888|4|317| +2450822|10891|4|16| +2450822|10892|4|45| +2450822|10894|4|209| +2450822|10897|4|279| +2450822|10898|4|458| +2450822|10900|4|791| +2450822|10903|4|| +2450822|10904|4|874| +2450822|10906|4|183| +2450822|10909|4|130| +2450822|10910|4|430| +2450822|10912|4|273| +2450822|10915|4|108| +2450822|10916|4|43| +2450822|10918|4|211| +2450822|10921|4|201| +2450822|10922|4|316| +2450822|10924|4|412| +2450822|10927|4|111| +2450822|10928|4|337| +2450822|10930|4|831| +2450822|10933|4|808| +2450822|10934|4|338| +2450822|10936|4|133| +2450822|10939|4|13| +2450822|10940|4|331| +2450822|10942|4|1| +2450822|10945|4|753| +2450822|10946|4|695| +2450822|10948|4|239| +2450822|10951|4|539| +2450822|10952|4|483| +2450822|10954|4|976| +2450822|10957|4|328| +2450822|10958|4|290| +2450822|10960|4|936| +2450822|10963|4|956| +2450822|10964|4|| +2450822|10966|4|994| +2450822|10969|4|556| +2450822|10970|4|32| +2450822|10972|4|387| +2450822|10975|4|553| +2450822|10976|4|634| +2450822|10978|4|366| +2450822|10981|4|732| +2450822|10982|4|761| +2450822|10984|4|849| +2450822|10987|4|457| +2450822|10988|4|127| +2450822|10990|4|441| +2450822|10993|4|462| +2450822|10994|4|141| +2450822|10996|4|719| +2450822|10999|4|690| +2450822|11000|4|144| +2450822|11002|4|900| +2450822|11005|4|965| +2450822|11006|4|498| +2450822|11008|4|956| +2450822|11011|4|22| +2450822|11012|4|880| +2450822|11014|4|699| +2450822|11017|4|405| +2450822|11018|4|166| +2450822|11020|4|28| +2450822|11023|4|66| +2450822|11024|4|822| +2450822|11026|4|123| +2450822|11029|4|878| +2450822|11030|4|844| +2450822|11032|4|618| +2450822|11035|4|773| +2450822|11036|4|489| +2450822|11038|4|705| +2450822|11041|4|791| +2450822|11042|4|768| +2450822|11044|4|3| +2450822|11047|4|302| +2450822|11048|4|792| +2450822|11050|4|704| +2450822|11053|4|735| +2450822|11054|4|66| +2450822|11056|4|580| +2450822|11059|4|281| +2450822|11060|4|751| +2450822|11062|4|427| +2450822|11065|4|297| +2450822|11066|4|381| +2450822|11068|4|743| +2450822|11071|4|706| +2450822|11072|4|250| +2450822|11074|4|547| +2450822|11077|4|926| +2450822|11078|4|17| +2450822|11080|4|348| +2450822|11083|4|186| +2450822|11084|4|768| +2450822|11086|4|939| +2450822|11089|4|405| +2450822|11090|4|241| +2450822|11092|4|601| +2450822|11095|4|723| +2450822|11096|4|| +2450822|11098|4|807| +2450822|11101|4|896| +2450822|11102|4|469| +2450822|11104|4|847| +2450822|11107|4|338| +2450822|11108|4|219| +2450822|11110|4|200| +2450822|11113|4|808| +2450822|11114|4|808| +2450822|11116|4|508| +2450822|11119|4|137| +2450822|11120|4|174| +2450822|11122|4|268| +2450822|11125|4|741| +2450822|11126|4|179| +2450822|11128|4|816| +2450822|11131|4|755| +2450822|11132|4|661| +2450822|11134|4|775| +2450822|11137|4|966| +2450822|11138|4|556| +2450822|11140|4|366| +2450822|11143|4|380| +2450822|11144|4|668| +2450822|11146|4|465| +2450822|11149|4|916| +2450822|11150|4|685| +2450822|11152|4|104| +2450822|11155|4|666| +2450822|11156|4|454| +2450822|11158|4|182| +2450822|11161|4|924| +2450822|11162|4|776| +2450822|11164|4|121| +2450822|11167|4|803| +2450822|11168|4|693| +2450822|11170|4|926| +2450822|11173|4|550| +2450822|11174|4|149| +2450822|11176|4|785| +2450822|11179|4|571| +2450822|11180|4|736| +2450822|11182|4|576| +2450822|11185|4|545| +2450822|11186|4|65| +2450822|11188|4|364| +2450822|11191|4|215| +2450822|11192|4|815| +2450822|11194|4|42| +2450822|11197|4|509| +2450822|11198|4|527| +2450822|11200|4|882| +2450822|11203|4|892| +2450822|11204|4|681| +2450822|11206|4|412| +2450822|11209|4|491| +2450822|11210|4|869| +2450822|11212|4|604| +2450822|11215|4|22| +2450822|11216|4|405| +2450822|11218|4|981| +2450822|11221|4|631| +2450822|11222|4|280| +2450822|11224|4|536| +2450822|11227|4|987| +2450822|11228|4|384| +2450822|11230|4|502| +2450822|11233|4|449| +2450822|11234|4|321| +2450822|11236|4|662| +2450822|11239|4|162| +2450822|11240|4|623| +2450822|11242|4|405| +2450822|11245|4|| +2450822|11246|4|636| +2450822|11248|4|108| +2450822|11251|4|219| +2450822|11252|4|45| +2450822|11254|4|402| +2450822|11257|4|777| +2450822|11258|4|479| +2450822|11260|4|499| +2450822|11263|4|509| +2450822|11264|4|781| +2450822|11266|4|404| +2450822|11269|4|31| +2450822|11270|4|685| +2450822|11272|4|472| +2450822|11275|4|994| +2450822|11276|4|700| +2450822|11278|4|634| +2450822|11281|4|756| +2450822|11282|4|184| +2450822|11284|4|905| +2450822|11287|4|920| +2450822|11288|4|407| +2450822|11290|4|32| +2450822|11293|4|287| +2450822|11294|4|951| +2450822|11296|4|525| +2450822|11299|4|116| +2450822|11300|4|796| +2450822|11302|4|861| +2450822|11305|4|435| +2450822|11306|4|770| +2450822|11308|4|291| +2450822|11311|4|276| +2450822|11312|4|443| +2450822|11314|4|485| +2450822|11317|4|485| +2450822|11318|4|33| +2450822|11320|4|7| +2450822|11323|4|415| +2450822|11324|4|979| +2450822|11326|4|42| +2450822|11329|4|671| +2450822|11330|4|152| +2450822|11332|4|573| +2450822|11335|4|300| +2450822|11336|4|30| +2450822|11338|4|577| +2450822|11341|4|606| +2450822|11342|4|| +2450822|11344|4|787| +2450822|11347|4|208| +2450822|11348|4|987| +2450822|11350|4|805| +2450822|11353|4|485| +2450822|11354|4|402| +2450822|11356|4|192| +2450822|11359|4|810| +2450822|11360|4|67| +2450822|11362|4|556| +2450822|11365|4|193| +2450822|11366|4|518| +2450822|11368|4|504| +2450822|11371|4|647| +2450822|11372|4|448| +2450822|11374|4|| +2450822|11377|4|575| +2450822|11378|4|663| +2450822|11380|4|454| +2450822|11383|4|926| +2450822|11384|4|726| +2450822|11386|4|978| +2450822|11389|4|249| +2450822|11390|4|964| +2450822|11392|4|111| +2450822|11395|4|982| +2450822|11396|4|| +2450822|11398|4|94| +2450822|11401|4|94| +2450822|11402|4|466| +2450822|11404|4|152| +2450822|11407|4|350| +2450822|11408|4|935| +2450822|11410|4|16| +2450822|11413|4|639| +2450822|11414|4|| +2450822|11416|4|821| +2450822|11419|4|119| +2450822|11420|4|117| +2450822|11422|4|206| +2450822|11425|4|139| +2450822|11426|4|206| +2450822|11428|4|128| +2450822|11431|4|266| +2450822|11432|4|129| +2450822|11434|4|73| +2450822|11437|4|509| +2450822|11438|4|599| +2450822|11440|4|416| +2450822|11443|4|375| +2450822|11444|4|102| +2450822|11446|4|708| +2450822|11449|4|994| +2450822|11450|4|536| +2450822|11452|4|60| +2450822|11455|4|108| +2450822|11456|4|601| +2450822|11458|4|992| +2450822|11461|4|372| +2450822|11462|4|481| +2450822|11464|4|130| +2450822|11467|4|922| +2450822|11468|4|743| +2450822|11470|4|984| +2450822|11473|4|326| +2450822|11474|4|191| +2450822|11476|4|590| +2450822|11479|4|8| +2450822|11480|4|932| +2450822|11482|4|229| +2450822|11485|4|943| +2450822|11486|4|295| +2450822|11488|4|372| +2450822|11491|4|191| +2450822|11492|4|| +2450822|11494|4|466| +2450822|11497|4|115| +2450822|11498|4|807| +2450822|11500|4|193| +2450822|11503|4|802| +2450822|11504|4|496| +2450822|11506|4|982| +2450822|11509|4|806| +2450822|11510|4|747| +2450822|11512|4|123| +2450822|11515|4|212| +2450822|11516|4|703| +2450822|11518|4|679| +2450822|11521|4|25| +2450822|11522|4|940| +2450822|11524|4|4| +2450822|11527|4|650| +2450822|11528|4|437| +2450822|11530|4|921| +2450822|11533|4|185| +2450822|11534|4|853| +2450822|11536|4|835| +2450822|11539|4|468| +2450822|11540|4|870| +2450822|11542|4|298| +2450822|11545|4|179| +2450822|11546|4|51| +2450822|11548|4|772| +2450822|11551|4|881| +2450822|11552|4|46| +2450822|11554|4|48| +2450822|11557|4|| +2450822|11558|4|| +2450822|11560|4|596| +2450822|11563|4|| +2450822|11564|4|756| +2450822|11566|4|336| +2450822|11569|4|721| +2450822|11570|4|479| +2450822|11572|4|481| +2450822|11575|4|305| +2450822|11576|4|421| +2450822|11578|4|356| +2450822|11581|4|693| +2450822|11582|4|721| +2450822|11584|4|266| +2450822|11587|4|723| +2450822|11588|4|908| +2450822|11590|4|122| +2450822|11593|4|579| +2450822|11594|4|717| +2450822|11596|4|659| +2450822|11599|4|183| +2450822|11600|4|140| +2450822|11602|4|404| +2450822|11605|4|84| +2450822|11606|4|252| +2450822|11608|4|996| +2450822|11611|4|693| +2450822|11612|4|838| +2450822|11614|4|122| +2450822|11617|4|331| +2450822|11618|4|244| +2450822|11620|4|442| +2450822|11623|4|697| +2450822|11624|4|222| +2450822|11626|4|209| +2450822|11629|4|98| +2450822|11630|4|948| +2450822|11632|4|736| +2450822|11635|4|407| +2450822|11636|4|557| +2450822|11638|4|858| +2450822|11641|4|372| +2450822|11642|4|340| +2450822|11644|4|36| +2450822|11647|4|171| +2450822|11648|4|986| +2450822|11650|4|963| +2450822|11653|4|880| +2450822|11654|4|515| +2450822|11656|4|152| +2450822|11659|4|949| +2450822|11660|4|964| +2450822|11662|4|983| +2450822|11665|4|165| +2450822|11666|4|385| +2450822|11668|4|346| +2450822|11671|4|391| +2450822|11672|4|238| +2450822|11674|4|237| +2450822|11677|4|528| +2450822|11678|4|230| +2450822|11680|4|508| +2450822|11683|4|42| +2450822|11684|4|786| +2450822|11686|4|33| +2450822|11689|4|725| +2450822|11690|4|999| +2450822|11692|4|5| +2450822|11695|4|611| +2450822|11696|4|773| +2450822|11698|4|967| +2450822|11701|4|1000| +2450822|11702|4|968| +2450822|11704|4|349| +2450822|11707|4|492| +2450822|11708|4|765| +2450822|11710|4|| +2450822|11713|4|264| +2450822|11714|4|410| +2450822|11716|4|456| +2450822|11719|4|984| +2450822|11720|4|484| +2450822|11722|4|361| +2450822|11725|4|970| +2450822|11726|4|618| +2450822|11728|4|209| +2450822|11731|4|953| +2450822|11732|4|23| +2450822|11734|4|609| +2450822|11737|4|185| +2450822|11738|4|| +2450822|11740|4|563| +2450822|11743|4|709| +2450822|11744|4|| +2450822|11746|4|717| +2450822|11749|4|578| +2450822|11750|4|247| +2450822|11752|4|268| +2450822|11755|4|449| +2450822|11756|4|874| +2450822|11758|4|413| +2450822|11761|4|252| +2450822|11762|4|954| +2450822|11764|4|| +2450822|11767|4|244| +2450822|11768|4|544| +2450822|11770|4|496| +2450822|11773|4|764| +2450822|11774|4|545| +2450822|11776|4|760| +2450822|11779|4|953| +2450822|11780|4|910| +2450822|11782|4|253| +2450822|11785|4|| +2450822|11786|4|592| +2450822|11788|4|929| +2450822|11791|4|712| +2450822|11792|4|864| +2450822|11794|4|607| +2450822|11797|4|826| +2450822|11798|4|606| +2450822|11800|4|122| +2450822|11803|4|525| +2450822|11804|4|643| +2450822|11806|4|409| +2450822|11809|4|947| +2450822|11810|4|596| +2450822|11812|4|| +2450822|11815|4|945| +2450822|11816|4|824| +2450822|11818|4|650| +2450822|11821|4|917| +2450822|11822|4|869| +2450822|11824|4|326| +2450822|11827|4|939| +2450822|11828|4|155| +2450822|11830|4|8| +2450822|11833|4|192| +2450822|11834|4|423| +2450822|11836|4|38| +2450822|11839|4|889| +2450822|11840|4|| +2450822|11842|4|| +2450822|11845|4|837| +2450822|11846|4|935| +2450822|11848|4|429| +2450822|11851|4|820| +2450822|11852|4|33| +2450822|11854|4|804| +2450822|11857|4|829| +2450822|11858|4|681| +2450822|11860|4|385| +2450822|11863|4|90| +2450822|11864|4|922| +2450822|11866|4|126| +2450822|11869|4|836| +2450822|11870|4|863| +2450822|11872|4|753| +2450822|11875|4|645| +2450822|11876|4|729| +2450822|11878|4|606| +2450822|11881|4|871| +2450822|11882|4|296| +2450822|11884|4|844| +2450822|11887|4|| +2450822|11888|4|238| +2450822|11890|4|809| +2450822|11893|4|213| +2450822|11894|4|178| +2450822|11896|4|242| +2450822|11899|4|302| +2450822|11900|4|389| +2450822|11902|4|240| +2450822|11905|4|431| +2450822|11906|4|614| +2450822|11908|4|921| +2450822|11911|4|697| +2450822|11912|4|148| +2450822|11914|4|| +2450822|11917|4|230| +2450822|11918|4|412| +2450822|11920|4|564| +2450822|11923|4|803| +2450822|11924|4|38| +2450822|11926|4|726| +2450822|11929|4|574| +2450822|11930|4|19| +2450822|11932|4|| +2450822|11935|4|835| +2450822|11936|4|900| +2450822|11938|4|775| +2450822|11941|4|968| +2450822|11942|4|39| +2450822|11944|4|54| +2450822|11947|4|514| +2450822|11948|4|229| +2450822|11950|4|221| +2450822|11953|4|154| +2450822|11954|4|918| +2450822|11956|4|720| +2450822|11959|4|544| +2450822|11960|4|321| +2450822|11962|4|| +2450822|11965|4|693| +2450822|11966|4|331| +2450822|11968|4|101| +2450822|11971|4|925| +2450822|11972|4|622| +2450822|11974|4|544| +2450822|11977|4|861| +2450822|11978|4|0| +2450822|11980|4|80| +2450822|11983|4|425| +2450822|11984|4|390| +2450822|11986|4|598| +2450822|11989|4|113| +2450822|11990|4|| +2450822|11992|4|482| +2450822|11995|4|321| +2450822|11996|4|87| +2450822|11998|4|471| +2450822|12001|4|295| +2450822|12002|4|937| +2450822|12004|4|29| +2450822|12007|4|732| +2450822|12008|4|435| +2450822|12010|4|193| +2450822|12013|4|641| +2450822|12014|4|324| +2450822|12016|4|481| +2450822|12019|4|822| +2450822|12020|4|131| +2450822|12022|4|902| +2450822|12025|4|79| +2450822|12026|4|429| +2450822|12028|4|5| +2450822|12031|4|683| +2450822|12032|4|106| +2450822|12034|4|259| +2450822|12037|4|812| +2450822|12038|4|959| +2450822|12040|4|715| +2450822|12043|4|8| +2450822|12044|4|563| +2450822|12046|4|870| +2450822|12049|4|400| +2450822|12050|4|365| +2450822|12052|4|381| +2450822|12055|4|974| +2450822|12056|4|197| +2450822|12058|4|210| +2450822|12061|4|663| +2450822|12062|4|729| +2450822|12064|4|926| +2450822|12067|4|711| +2450822|12068|4|612| +2450822|12070|4|491| +2450822|12073|4|654| +2450822|12074|4|453| +2450822|12076|4|869| +2450822|12079|4|571| +2450822|12080|4|81| +2450822|12082|4|762| +2450822|12085|4|393| +2450822|12086|4|377| +2450822|12088|4|782| +2450822|12091|4|148| +2450822|12092|4|899| +2450822|12094|4|800| +2450822|12097|4|60| +2450822|12098|4|149| +2450822|12100|4|598| +2450822|12103|4|867| +2450822|12104|4|146| +2450822|12106|4|940| +2450822|12109|4|218| +2450822|12110|4|330| +2450822|12112|4|667| +2450822|12115|4|900| +2450822|12116|4|777| +2450822|12118|4|738| +2450822|12121|4|402| +2450822|12122|4|417| +2450822|12124|4|756| +2450822|12127|4|160| +2450822|12128|4|473| +2450822|12130|4|735| +2450822|12133|4|990| +2450822|12134|4|520| +2450822|12136|4|836| +2450822|12139|4|38| +2450822|12140|4|260| +2450822|12142|4|969| +2450822|12145|4|414| +2450822|12146|4|537| +2450822|12148|4|19| +2450822|12151|4|348| +2450822|12152|4|675| +2450822|12154|4|782| +2450822|12157|4|197| +2450822|12158|4|599| +2450822|12160|4|33| +2450822|12163|4|171| +2450822|12164|4|935| +2450822|12166|4|381| +2450822|12169|4|172| +2450822|12170|4|733| +2450822|12172|4|960| +2450822|12175|4|574| +2450822|12176|4|| +2450822|12178|4|604| +2450822|12181|4|649| +2450822|12182|4|300| +2450822|12184|4|460| +2450822|12187|4|115| +2450822|12188|4|| +2450822|12190|4|532| +2450822|12193|4|767| +2450822|12194|4|170| +2450822|12196|4|131| +2450822|12199|4|440| +2450822|12200|4|209| +2450822|12202|4|680| +2450822|12205|4|760| +2450822|12206|4|| +2450822|12208|4|914| +2450822|12211|4|84| +2450822|12212|4|21| +2450822|12214|4|881| +2450822|12217|4|839| +2450822|12218|4|999| +2450822|12220|4|976| +2450822|12223|4|502| +2450822|12224|4|138| +2450822|12226|4|480| +2450822|12229|4|375| +2450822|12230|4|214| +2450822|12232|4|57| +2450822|12235|4|355| +2450822|12236|4|126| +2450822|12238|4|49| +2450822|12241|4|428| +2450822|12242|4|309| +2450822|12244|4|266| +2450822|12247|4|953| +2450822|12248|4|671| +2450822|12250|4|651| +2450822|12253|4|926| +2450822|12254|4|937| +2450822|12256|4|492| +2450822|12259|4|643| +2450822|12260|4|875| +2450822|12262|4|259| +2450822|12265|4|24| +2450822|12266|4|| +2450822|12268|4|974| +2450822|12271|4|972| +2450822|12272|4|6| +2450822|12274|4|79| +2450822|12277|4|704| +2450822|12278|4|828| +2450822|12280|4|381| +2450822|12283|4|943| +2450822|12284|4|227| +2450822|12286|4|817| +2450822|12289|4|667| +2450822|12290|4|11| +2450822|12292|4|829| +2450822|12295|4|996| +2450822|12296|4|14| +2450822|12298|4|701| +2450822|12301|4|266| +2450822|12302|4|172| +2450822|12304|4|536| +2450822|12307|4|107| +2450822|12308|4|835| +2450822|12310|4|619| +2450822|12313|4|620| +2450822|12314|4|696| +2450822|12316|4|468| +2450822|12319|4|437| +2450822|12320|4|409| +2450822|12322|4|467| +2450822|12325|4|706| +2450822|12326|4|179| +2450822|12328|4|750| +2450822|12331|4|763| +2450822|12332|4|919| +2450822|12334|4|93| +2450822|12337|4|711| +2450822|12338|4|614| +2450822|12340|4|407| +2450822|12343|4|727| +2450822|12344|4|126| +2450822|12346|4|362| +2450822|12349|4|562| +2450822|12350|4|489| +2450822|12352|4|284| +2450822|12355|4|297| +2450822|12356|4|572| +2450822|12358|4|565| +2450822|12361|4|834| +2450822|12362|4|889| +2450822|12364|4|824| +2450822|12367|4|982| +2450822|12368|4|16| +2450822|12370|4|928| +2450822|12373|4|169| +2450822|12374|4|545| +2450822|12376|4|573| +2450822|12379|4|369| +2450822|12380|4|410| +2450822|12382|4|530| +2450822|12385|4|631| +2450822|12386|4|87| +2450822|12388|4|506| +2450822|12391|4|323| +2450822|12392|4|377| +2450822|12394|4|825| +2450822|12397|4|447| +2450822|12398|4|777| +2450822|12400|4|649| +2450822|12403|4|785| +2450822|12404|4|408| +2450822|12406|4|515| +2450822|12409|4|648| +2450822|12410|4|189| +2450822|12412|4|242| +2450822|12415|4|397| +2450822|12416|4|935| +2450822|12418|4|| +2450822|12421|4|339| +2450822|12422|4|131| +2450822|12424|4|603| +2450822|12427|4|604| +2450822|12428|4|228| +2450822|12430|4|831| +2450822|12433|4|398| +2450822|12434|4|131| +2450822|12436|4|953| +2450822|12439|4|179| +2450822|12440|4|491| +2450822|12442|4|9| +2450822|12445|4|76| +2450822|12446|4|600| +2450822|12448|4|719| +2450822|12451|4|124| +2450822|12452|4|485| +2450822|12454|4|155| +2450822|12457|4|636| +2450822|12458|4|667| +2450822|12460|4|929| +2450822|12463|4|237| +2450822|12464|4|700| +2450822|12466|4|965| +2450822|12469|4|798| +2450822|12470|4|926| +2450822|12472|4|840| +2450822|12475|4|63| +2450822|12476|4|417| +2450822|12478|4|610| +2450822|12481|4|305| +2450822|12482|4|545| +2450822|12484|4|294| +2450822|12487|4|271| +2450822|12488|4|| +2450822|12490|4|70| +2450822|12493|4|682| +2450822|12494|4|211| +2450822|12496|4|343| +2450822|12499|4|413| +2450822|12500|4|276| +2450822|12502|4|901| +2450822|12505|4|190| +2450822|12506|4|789| +2450822|12508|4|400| +2450822|12511|4|483| +2450822|12512|4|665| +2450822|12514|4|439| +2450822|12517|4|308| +2450822|12518|4|385| +2450822|12520|4|730| +2450822|12523|4|330| +2450822|12524|4|202| +2450822|12526|4|| +2450822|12529|4|497| +2450822|12530|4|94| +2450822|12532|4|627| +2450822|12535|4|971| +2450822|12536|4|15| +2450822|12538|4|939| +2450822|12541|4|10| +2450822|12542|4|19| +2450822|12544|4|408| +2450822|12547|4|442| +2450822|12548|4|345| +2450822|12550|4|265| +2450822|12553|4|322| +2450822|12554|4|920| +2450822|12556|4|53| +2450822|12559|4|243| +2450822|12560|4|895| +2450822|12562|4|471| +2450822|12565|4|704| +2450822|12566|4|201| +2450822|12568|4|557| +2450822|12571|4|398| +2450822|12572|4|463| +2450822|12574|4|190| +2450822|12577|4|753| +2450822|12578|4|369| +2450822|12580|4|625| +2450822|12583|4|148| +2450822|12584|4|573| +2450822|12586|4|406| +2450822|12589|4|283| +2450822|12590|4|130| +2450822|12592|4|26| +2450822|12595|4|905| +2450822|12596|4|291| +2450822|12598|4|999| +2450822|12601|4|734| +2450822|12602|4|621| +2450822|12604|4|365| +2450822|12607|4|891| +2450822|12608|4|147| +2450822|12610|4|222| +2450822|12613|4|517| +2450822|12614|4|622| +2450822|12616|4|357| +2450822|12619|4|626| +2450822|12620|4|122| +2450822|12622|4|84| +2450822|12625|4|142| +2450822|12626|4|856| +2450822|12628|4|125| +2450822|12631|4|461| +2450822|12632|4|924| +2450822|12634|4|743| +2450822|12637|4|374| +2450822|12638|4|682| +2450822|12640|4|182| +2450822|12643|4|93| +2450822|12644|4|536| +2450822|12646|4|450| +2450822|12649|4|213| +2450822|12650|4|184| +2450822|12652|4|188| +2450822|12655|4|112| +2450822|12656|4|304| +2450822|12658|4|796| +2450822|12661|4|6| +2450822|12662|4|756| +2450822|12664|4|281| +2450822|12667|4|18| +2450822|12668|4|603| +2450822|12670|4|804| +2450822|12673|4|461| +2450822|12674|4|236| +2450822|12676|4|432| +2450822|12679|4|472| +2450822|12680|4|989| +2450822|12682|4|173| +2450822|12685|4|661| +2450822|12686|4|38| +2450822|12688|4|150| +2450822|12691|4|185| +2450822|12692|4|930| +2450822|12694|4|73| +2450822|12697|4|854| +2450822|12698|4|819| +2450822|12700|4|173| +2450822|12703|4|66| +2450822|12704|4|790| +2450822|12706|4|46| +2450822|12709|4|870| +2450822|12710|4|211| +2450822|12712|4|801| +2450822|12715|4|716| +2450822|12716|4|193| +2450822|12718|4|166| +2450822|12721|4|434| +2450822|12722|4|690| +2450822|12724|4|241| +2450822|12727|4|713| +2450822|12728|4|253| +2450822|12730|4|511| +2450822|12733|4|523| +2450822|12734|4|245| +2450822|12736|4|124| +2450822|12739|4|260| +2450822|12740|4|208| +2450822|12742|4|9| +2450822|12745|4|491| +2450822|12746|4|117| +2450822|12748|4|0| +2450822|12751|4|249| +2450822|12752|4|812| +2450822|12754|4|469| +2450822|12757|4|644| +2450822|12758|4|412| +2450822|12760|4|709| +2450822|12763|4|306| +2450822|12764|4|743| +2450822|12766|4|796| +2450822|12769|4|367| +2450822|12770|4|264| +2450822|12772|4|262| +2450822|12775|4|860| +2450822|12776|4|8| +2450822|12778|4|428| +2450822|12781|4|127| +2450822|12782|4|132| +2450822|12784|4|27| +2450822|12787|4|| +2450822|12788|4|34| +2450822|12790|4|170| +2450822|12793|4|400| +2450822|12794|4|277| +2450822|12796|4|717| +2450822|12799|4|29| +2450822|12800|4|756| +2450822|12802|4|204| +2450822|12805|4|561| +2450822|12806|4|505| +2450822|12808|4|| +2450822|12811|4|230| +2450822|12812|4|377| +2450822|12814|4|210| +2450822|12817|4|7| +2450822|12818|4|283| +2450822|12820|4|244| +2450822|12823|4|| +2450822|12824|4|564| +2450822|12826|4|58| +2450822|12829|4|| +2450822|12830|4|113| +2450822|12832|4|115| +2450822|12835|4|572| +2450822|12836|4|151| +2450822|12838|4|497| +2450822|12841|4|278| +2450822|12842|4|745| +2450822|12844|4|347| +2450822|12847|4|142| +2450822|12848|4|552| +2450822|12850|4|621| +2450822|12853|4|686| +2450822|12854|4|644| +2450822|12856|4|704| +2450822|12859|4|553| +2450822|12860|4|736| +2450822|12862|4|139| +2450822|12865|4|998| +2450822|12866|4|274| +2450822|12868|4|616| +2450822|12871|4|526| +2450822|12872|4|208| +2450822|12874|4|275| +2450822|12877|4|955| +2450822|12878|4|| +2450822|12880|4|300| +2450822|12883|4|378| +2450822|12884|4|| +2450822|12886|4|247| +2450822|12889|4|163| +2450822|12890|4|426| +2450822|12892|4|166| +2450822|12895|4|292| +2450822|12896|4|988| +2450822|12898|4|25| +2450822|12901|4|799| +2450822|12902|4|93| +2450822|12904|4|941| +2450822|12907|4|3| +2450822|12908|4|221| +2450822|12910|4|160| +2450822|12913|4|742| +2450822|12914|4|519| +2450822|12916|4|216| +2450822|12919|4|149| +2450822|12920|4|58| +2450822|12922|4|535| +2450822|12925|4|812| +2450822|12926|4|948| +2450822|12928|4|573| +2450822|12931|4|483| +2450822|12932|4|231| +2450822|12934|4|799| +2450822|12937|4|179| +2450822|12938|4|545| +2450822|12940|4|198| +2450822|12943|4|105| +2450822|12944|4|154| +2450822|12946|4|517| +2450822|12949|4|475| +2450822|12950|4|284| +2450822|12952|4|989| +2450822|12955|4|177| +2450822|12956|4|331| +2450822|12958|4|227| +2450822|12961|4|81| +2450822|12962|4|564| +2450822|12964|4|89| +2450822|12967|4|981| +2450822|12968|4|454| +2450822|12970|4|580| +2450822|12973|4|389| +2450822|12974|4|104| +2450822|12976|4|725| +2450822|12979|4|76| +2450822|12980|4|762| +2450822|12982|4|828| +2450822|12985|4|145| +2450822|12986|4|641| +2450822|12988|4|493| +2450822|12991|4|821| +2450822|12992|4|403| +2450822|12994|4|910| +2450822|12997|4|203| +2450822|12998|4|350| +2450822|13000|4|450| +2450822|13003|4|185| +2450822|13004|4|43| +2450822|13006|4|689| +2450822|13009|4|202| +2450822|13010|4|278| +2450822|13012|4|531| +2450822|13015|4|446| +2450822|13016|4|326| +2450822|13018|4|640| +2450822|13021|4|236| +2450822|13022|4|277| +2450822|13024|4|683| +2450822|13027|4|311| +2450822|13028|4|943| +2450822|13030|4|201| +2450822|13033|4|304| +2450822|13034|4|766| +2450822|13036|4|292| +2450822|13039|4|100| +2450822|13040|4|317| +2450822|13042|4|302| +2450822|13045|4|772| +2450822|13046|4|226| +2450822|13048|4|409| +2450822|13051|4|64| +2450822|13052|4|64| +2450822|13054|4|713| +2450822|13057|4|577| +2450822|13058|4|281| +2450822|13060|4|916| +2450822|13063|4|17| +2450822|13064|4|794| +2450822|13066|4|507| +2450822|13069|4|| +2450822|13070|4|366| +2450822|13072|4|676| +2450822|13075|4|960| +2450822|13076|4|734| +2450822|13078|4|722| +2450822|13081|4|911| +2450822|13082|4|175| +2450822|13084|4|380| +2450822|13087|4|958| +2450822|13088|4|214| +2450822|13090|4|28| +2450822|13093|4|| +2450822|13094|4|94| +2450822|13096|4|975| +2450822|13099|4|588| +2450822|13100|4|415| +2450822|13102|4|| +2450822|13105|4|779| +2450822|13106|4|965| +2450822|13108|4|108| +2450822|13111|4|876| +2450822|13112|4|352| +2450822|13114|4|257| +2450822|13117|4|470| +2450822|13118|4|506| +2450822|13120|4|786| +2450822|13123|4|393| +2450822|13124|4|666| +2450822|13126|4|403| +2450822|13129|4|840| +2450822|13130|4|191| +2450822|13132|4|| +2450822|13135|4|682| +2450822|13136|4|482| +2450822|13138|4|| +2450822|13141|4|913| +2450822|13142|4|67| +2450822|13144|4|691| +2450822|13147|4|213| +2450822|13148|4|35| +2450822|13150|4|550| +2450822|13153|4|925| +2450822|13154|4|613| +2450822|13156|4|108| +2450822|13159|4|799| +2450822|13160|4|728| +2450822|13162|4|20| +2450822|13165|4|430| +2450822|13166|4|849| +2450822|13168|4|361| +2450822|13171|4|824| +2450822|13172|4|617| +2450822|13174|4|134| +2450822|13177|4|903| +2450822|13178|4|953| +2450822|13180|4|687| +2450822|13183|4|133| +2450822|13184|4|241| +2450822|13186|4|689| +2450822|13189|4|441| +2450822|13190|4|380| +2450822|13192|4|267| +2450822|13195|4|655| +2450822|13196|4|698| +2450822|13198|4|520| +2450822|13201|4|85| +2450822|13202|4|501| +2450822|13204|4|798| +2450822|13207|4|593| +2450822|13208|4|803| +2450822|13210|4|256| +2450822|13213|4|659| +2450822|13214|4|| +2450822|13216|4|381| +2450822|13219|4|623| +2450822|13220|4|251| +2450822|13222|4|15| +2450822|13225|4|483| +2450822|13226|4|910| +2450822|13228|4|424| +2450822|13231|4|658| +2450822|13232|4|323| +2450822|13234|4|589| +2450822|13237|4|282| +2450822|13238|4|911| +2450822|13240|4|711| +2450822|13243|4|18| +2450822|13244|4|497| +2450822|13246|4|329| +2450822|13249|4|| +2450822|13250|4|63| +2450822|13252|4|99| +2450822|13255|4|831| +2450822|13256|4|| +2450822|13258|4|447| +2450822|13261|4|535| +2450822|13262|4|663| +2450822|13264|4|743| +2450822|13267|4|538| +2450822|13268|4|7| +2450822|13270|4|797| +2450822|13273|4|619| +2450822|13274|4|995| +2450822|13276|4|781| +2450822|13279|4|651| +2450822|13280|4|604| +2450822|13282|4|74| +2450822|13285|4|| +2450822|13286|4|669| +2450822|13288|4|956| +2450822|13291|4|248| +2450822|13292|4|965| +2450822|13294|4|120| +2450822|13297|4|984| +2450822|13298|4|985| +2450822|13300|4|32| +2450822|13303|4|193| +2450822|13304|4|844| +2450822|13306|4|719| +2450822|13309|4|915| +2450822|13310|4|13| +2450822|13312|4|763| +2450822|13315|4|11| +2450822|13316|4|833| +2450822|13318|4|668| +2450822|13321|4|870| +2450822|13322|4|483| +2450822|13324|4|886| +2450822|13327|4|| +2450822|13328|4|654| +2450822|13330|4|289| +2450822|13333|4|984| +2450822|13334|4|638| +2450822|13336|4|94| +2450822|13339|4|683| +2450822|13340|4|262| +2450822|13342|4|180| +2450822|13345|4|575| +2450822|13346|4|173| +2450822|13348|4|386| +2450822|13351|4|| +2450822|13352|4|412| +2450822|13354|4|544| +2450822|13357|4|781| +2450822|13358|4|941| +2450822|13360|4|230| +2450822|13363|4|616| +2450822|13364|4|671| +2450822|13366|4|457| +2450822|13369|4|75| +2450822|13370|4|45| +2450822|13372|4|211| +2450822|13375|4|458| +2450822|13376|4|| +2450822|13378|4|322| +2450822|13381|4|235| +2450822|13382|4|827| +2450822|13384|4|486| +2450822|13387|4|668| +2450822|13388|4|803| +2450822|13390|4|330| +2450822|13393|4|156| +2450822|13394|4|954| +2450822|13396|4|347| +2450822|13399|4|676| +2450822|13400|4|697| +2450822|13402|4|247| +2450822|13405|4|893| +2450822|13406|4|815| +2450822|13408|4|350| +2450822|13411|4|441| +2450822|13412|4|442| +2450822|13414|4|941| +2450822|13417|4|530| +2450822|13418|4|230| +2450822|13420|4|748| +2450822|13423|4|240| +2450822|13424|4|955| +2450822|13426|4|285| +2450822|13429|4|531| +2450822|13430|4|46| +2450822|13432|4|689| +2450822|13435|4|323| +2450822|13436|4|| +2450822|13438|4|185| +2450822|13441|4|952| +2450822|13442|4|622| +2450822|13444|4|121| +2450822|13447|4|489| +2450822|13448|4|340| +2450822|13450|4|25| +2450822|13453|4|292| +2450822|13454|4|298| +2450822|13456|4|584| +2450822|13459|4|958| +2450822|13460|4|38| +2450822|13462|4|414| +2450822|13465|4|825| +2450822|13466|4|140| +2450822|13468|4|942| +2450822|13471|4|283| +2450822|13472|4|155| +2450822|13474|4|253| +2450822|13477|4|846| +2450822|13478|4|17| +2450822|13480|4|598| +2450822|13483|4|804| +2450822|13484|4|733| +2450822|13486|4|257| +2450822|13489|4|| +2450822|13490|4|364| +2450822|13492|4|| +2450822|13495|4|936| +2450822|13496|4|895| +2450822|13498|4|703| +2450822|13501|4|863| +2450822|13502|4|4| +2450822|13504|4|279| +2450822|13507|4|973| +2450822|13508|4|267| +2450822|13510|4|282| +2450822|13513|4|855| +2450822|13514|4|873| +2450822|13516|4|807| +2450822|13519|4|699| +2450822|13520|4|29| +2450822|13522|4|218| +2450822|13525|4|884| +2450822|13526|4|487| +2450822|13528|4|437| +2450822|13531|4|296| +2450822|13532|4|631| +2450822|13534|4|599| +2450822|13537|4|415| +2450822|13538|4|957| +2450822|13540|4|788| +2450822|13543|4|325| +2450822|13544|4|459| +2450822|13546|4|| +2450822|13549|4|28| +2450822|13550|4|778| +2450822|13552|4|714| +2450822|13555|4|296| +2450822|13556|4|514| +2450822|13558|4|318| +2450822|13561|4|755| +2450822|13562|4|878| +2450822|13564|4|445| +2450822|13567|4|461| +2450822|13568|4|82| +2450822|13570|4|913| +2450822|13573|4|726| +2450822|13574|4|41| +2450822|13576|4|149| +2450822|13579|4|930| +2450822|13580|4|681| +2450822|13582|4|472| +2450822|13585|4|550| +2450822|13586|4|284| +2450822|13588|4|973| +2450822|13591|4|669| +2450822|13592|4|740| +2450822|13594|4|344| +2450822|13597|4|238| +2450822|13598|4|199| +2450822|13600|4|295| +2450822|13603|4|264| +2450822|13604|4|1000| +2450822|13606|4|467| +2450822|13609|4|| +2450822|13610|4|509| +2450822|13612|4|687| +2450822|13615|4|166| +2450822|13616|4|330| +2450822|13618|4|375| +2450822|13621|4|| +2450822|13622|4|565| +2450822|13624|4|265| +2450822|13627|4|881| +2450822|13628|4|921| +2450822|13630|4|528| +2450822|13633|4|316| +2450822|13634|4|814| +2450822|13636|4|963| +2450822|13639|4|698| +2450822|13640|4|| +2450822|13642|4|316| +2450822|13645|4|| +2450822|13646|4|143| +2450822|13648|4|577| +2450822|13651|4|219| +2450822|13652|4|372| +2450822|13654|4|797| +2450822|13657|4|905| +2450822|13658|4|942| +2450822|13660|4|724| +2450822|13663|4|893| +2450822|13664|4|92| +2450822|13666|4|444| +2450822|13669|4|325| +2450822|13670|4|529| +2450822|13672|4|187| +2450822|13675|4|743| +2450822|13676|4|134| +2450822|13678|4|| +2450822|13681|4|| +2450822|13682|4|750| +2450822|13684|4|370| +2450822|13687|4|4| +2450822|13688|4|209| +2450822|13690|4|481| +2450822|13693|4|103| +2450822|13694|4|368| +2450822|13696|4|7| +2450822|13699|4|13| +2450822|13700|4|108| +2450822|13702|4|760| +2450822|13705|4|524| +2450822|13706|4|414| +2450822|13708|4|433| +2450822|13711|4|615| +2450822|13712|4|581| +2450822|13714|4|567| +2450822|13717|4|860| +2450822|13718|4|267| +2450822|13720|4|90| +2450822|13723|4|394| +2450822|13724|4|159| +2450822|13726|4|481| +2450822|13729|4|701| +2450822|13730|4|931| +2450822|13732|4|432| +2450822|13735|4|814| +2450822|13736|4|675| +2450822|13738|4|679| +2450822|13741|4|359| +2450822|13742|4|545| +2450822|13744|4|809| +2450822|13747|4|800| +2450822|13748|4|653| +2450822|13750|4|221| +2450822|13753|4|969| +2450822|13754|4|418| +2450822|13756|4|638| +2450822|13759|4|160| +2450822|13760|4|| +2450822|13762|4|639| +2450822|13765|4|238| +2450822|13766|4|72| +2450822|13768|4|153| +2450822|13771|4|11| +2450822|13772|4|| +2450822|13774|4|670| +2450822|13777|4|304| +2450822|13778|4|824| +2450822|13780|4|22| +2450822|13783|4|384| +2450822|13784|4|326| +2450822|13786|4|278| +2450822|13789|4|354| +2450822|13790|4|610| +2450822|13792|4|870| +2450822|13795|4|612| +2450822|13796|4|529| +2450822|13798|4|666| +2450822|13801|4|626| +2450822|13802|4|| +2450822|13804|4|973| +2450822|13807|4|774| +2450822|13808|4|411| +2450822|13810|4|266| +2450822|13813|4|11| +2450822|13814|4|189| +2450822|13816|4|11| +2450822|13819|4|576| +2450822|13820|4|834| +2450822|13822|4|77| +2450822|13825|4|758| +2450822|13826|4|553| +2450822|13828|4|491| +2450822|13831|4|719| +2450822|13832|4|854| +2450822|13834|4|315| +2450822|13837|4|133| +2450822|13838|4|798| +2450822|13840|4|283| +2450822|13843|4|978| +2450822|13844|4|418| +2450822|13846|4|913| +2450822|13849|4|788| +2450822|13850|4|984| +2450822|13852|4|705| +2450822|13855|4|821| +2450822|13856|4|284| +2450822|13858|4|781| +2450822|13861|4|| +2450822|13862|4|265| +2450822|13864|4|158| +2450822|13867|4|652| +2450822|13868|4|598| +2450822|13870|4|| +2450822|13873|4|17| +2450822|13874|4|468| +2450822|13876|4|763| +2450822|13879|4|306| +2450822|13880|4|731| +2450822|13882|4|535| +2450822|13885|4|559| +2450822|13886|4|| +2450822|13888|4|518| +2450822|13891|4|682| +2450822|13892|4|| +2450822|13894|4|738| +2450822|13897|4|581| +2450822|13898|4|203| +2450822|13900|4|630| +2450822|13903|4|557| +2450822|13904|4|939| +2450822|13906|4|386| +2450822|13909|4|661| +2450822|13910|4|988| +2450822|13912|4|| +2450822|13915|4|411| +2450822|13916|4|264| +2450822|13918|4|152| +2450822|13921|4|327| +2450822|13922|4|374| +2450822|13924|4|550| +2450822|13927|4|978| +2450822|13928|4|613| +2450822|13930|4|282| +2450822|13933|4|634| +2450822|13934|4|629| +2450822|13936|4|653| +2450822|13939|4|605| +2450822|13940|4|154| +2450822|13942|4|302| +2450822|13945|4|778| +2450822|13946|4|802| +2450822|13948|4|642| +2450822|13951|4|37| +2450822|13952|4|168| +2450822|13954|4|677| +2450822|13957|4|805| +2450822|13958|4|315| +2450822|13960|4|853| +2450822|13963|4|| +2450822|13964|4|444| +2450822|13966|4|961| +2450822|13969|4|148| +2450822|13970|4|710| +2450822|13972|4|666| +2450822|13975|4|688| +2450822|13976|4|963| +2450822|13978|4|781| +2450822|13981|4|234| +2450822|13982|4|562| +2450822|13984|4|116| +2450822|13987|4|639| +2450822|13988|4|246| +2450822|13990|4|116| +2450822|13993|4|519| +2450822|13994|4|928| +2450822|13996|4|85| +2450822|13999|4|641| +2450822|14000|4|765| +2450822|14002|4|859| +2450822|14005|4|831| +2450822|14006|4|760| +2450822|14008|4|39| +2450822|14011|4|763| +2450822|14012|4|205| +2450822|14014|4|994| +2450822|14017|4|746| +2450822|14018|4|922| +2450822|14020|4|990| +2450822|14023|4|455| +2450822|14024|4|916| +2450822|14026|4|194| +2450822|14029|4|741| +2450822|14030|4|601| +2450822|14032|4|642| +2450822|14035|4|| +2450822|14036|4|487| +2450822|14038|4|146| +2450822|14041|4|62| +2450822|14042|4|455| +2450822|14044|4|302| +2450822|14047|4|475| +2450822|14048|4|206| +2450822|14050|4|576| +2450822|14053|4|880| +2450822|14054|4|473| +2450822|14056|4|262| +2450822|14059|4|868| +2450822|14060|4|477| +2450822|14062|4|771| +2450822|14065|4|696| +2450822|14066|4|589| +2450822|14068|4|498| +2450822|14071|4|37| +2450822|14072|4|717| +2450822|14074|4|991| +2450822|14077|4|424| +2450822|14078|4|132| +2450822|14080|4|604| +2450822|14083|4|648| +2450822|14084|4|904| +2450822|14086|4|331| +2450822|14089|4|294| +2450822|14090|4|851| +2450822|14092|4|514| +2450822|14095|4|348| +2450822|14096|4|66| +2450822|14098|4|694| +2450822|14101|4|186| +2450822|14102|4|614| +2450822|14104|4|689| +2450822|14107|4|835| +2450822|14108|4|127| +2450822|14110|4|426| +2450822|14113|4|462| +2450822|14114|4|871| +2450822|14116|4|485| +2450822|14119|4|922| +2450822|14120|4|38| +2450822|14122|4|948| +2450822|14125|4|222| +2450822|14126|4|56| +2450822|14128|4|589| +2450822|14131|4|833| +2450822|14132|4|317| +2450822|14134|4|751| +2450822|14137|4|929| +2450822|14138|4|499| +2450822|14140|4|158| +2450822|14143|4|| +2450822|14144|4|247| +2450822|14146|4|602| +2450822|14149|4|780| +2450822|14150|4|559| +2450822|14152|4|1| +2450822|14155|4|118| +2450822|14156|4|155| +2450822|14158|4|885| +2450822|14161|4|436| +2450822|14162|4|59| +2450822|14164|4|258| +2450822|14167|4|409| +2450822|14168|4|708| +2450822|14170|4|| +2450822|14173|4|293| +2450822|14174|4|213| +2450822|14176|4|712| +2450822|14179|4|| +2450822|14180|4|642| +2450822|14182|4|59| +2450822|14185|4|189| +2450822|14186|4|766| +2450822|14188|4|820| +2450822|14191|4|165| +2450822|14192|4|283| +2450822|14194|4|38| +2450822|14197|4|891| +2450822|14198|4|226| +2450822|14200|4|898| +2450822|14203|4|945| +2450822|14204|4|313| +2450822|14206|4|784| +2450822|14209|4|577| +2450822|14210|4|299| +2450822|14212|4|175| +2450822|14215|4|795| +2450822|14216|4|533| +2450822|14218|4|772| +2450822|14221|4|572| +2450822|14222|4|868| +2450822|14224|4|827| +2450822|14227|4|357| +2450822|14228|4|519| +2450822|14230|4|| +2450822|14233|4|958| +2450822|14234|4|211| +2450822|14236|4|876| +2450822|14239|4|891| +2450822|14240|4|368| +2450822|14242|4|810| +2450822|14245|4|281| +2450822|14246|4|230| +2450822|14248|4|250| +2450822|14251|4|822| +2450822|14252|4|995| +2450822|14254|4|505| +2450822|14257|4|70| +2450822|14258|4|379| +2450822|14260|4|602| +2450822|14263|4|526| +2450822|14264|4|613| +2450822|14266|4|237| +2450822|14269|4|362| +2450822|14270|4|456| +2450822|14272|4|891| +2450822|14275|4|679| +2450822|14276|4|198| +2450822|14278|4|473| +2450822|14281|4|113| +2450822|14282|4|144| +2450822|14284|4|12| +2450822|14287|4|984| +2450822|14288|4|955| +2450822|14290|4|43| +2450822|14293|4|446| +2450822|14294|4|968| +2450822|14296|4|858| +2450822|14299|4|161| +2450822|14300|4|512| +2450822|14302|4|170| +2450822|14305|4|85| +2450822|14306|4|968| +2450822|14308|4|| +2450822|14311|4|236| +2450822|14312|4|580| +2450822|14314|4|782| +2450822|14317|4|282| +2450822|14318|4|91| +2450822|14320|4|858| +2450822|14323|4|908| +2450822|14324|4|446| +2450822|14326|4|535| +2450822|14329|4|851| +2450822|14330|4|810| +2450822|14332|4|226| +2450822|14335|4|| +2450822|14336|4|431| +2450822|14338|4|953| +2450822|14341|4|949| +2450822|14342|4|708| +2450822|14344|4|192| +2450822|14347|4|349| +2450822|14348|4|884| +2450822|14350|4|337| +2450822|14353|4|319| +2450822|14354|4|200| +2450822|14356|4|83| +2450822|14359|4|884| +2450822|14360|4|23| +2450822|14362|4|762| +2450822|14365|4|621| +2450822|14366|4|510| +2450822|14368|4|816| +2450822|14371|4|798| +2450822|14372|4|551| +2450822|14374|4|269| +2450822|14377|4|120| +2450822|14378|4|857| +2450822|14380|4|636| +2450822|14383|4|641| +2450822|14384|4|142| +2450822|14386|4|278| +2450822|14389|4|272| +2450822|14390|4|903| +2450822|14392|4|909| +2450822|14395|4|147| +2450822|14396|4|178| +2450822|14398|4|511| +2450822|14401|4|667| +2450822|14402|4|485| +2450822|14404|4|203| +2450822|14407|4|640| +2450822|14408|4|449| +2450822|14410|4|200| +2450822|14413|4|138| +2450822|14414|4|198| +2450822|14416|4|19| +2450822|14419|4|232| +2450822|14420|4|631| +2450822|14422|4|767| +2450822|14425|4|265| +2450822|14426|4|800| +2450822|14428|4|160| +2450822|14431|4|1| +2450822|14432|4|194| +2450822|14434|4|454| +2450822|14437|4|407| +2450822|14438|4|146| +2450822|14440|4|580| +2450822|14443|4|108| +2450822|14444|4|645| +2450822|14446|4|374| +2450822|14449|4|289| +2450822|14450|4|891| +2450822|14452|4|186| +2450822|14455|4|607| +2450822|14456|4|599| +2450822|14458|4|392| +2450822|14461|4|475| +2450822|14462|4|276| +2450822|14464|4|652| +2450822|14467|4|| +2450822|14468|4|| +2450822|14470|4|316| +2450822|14473|4|262| +2450822|14474|4|968| +2450822|14476|4|786| +2450822|14479|4|850| +2450822|14480|4|405| +2450822|14482|4|228| +2450822|14485|4|967| +2450822|14486|4|947| +2450822|14488|4|| +2450822|14491|4|974| +2450822|14492|4|414| +2450822|14494|4|87| +2450822|14497|4|590| +2450822|14498|4|| +2450822|14500|4|318| +2450822|14503|4|291| +2450822|14504|4|407| +2450822|14506|4|742| +2450822|14509|4|833| +2450822|14510|4|594| +2450822|14512|4|415| +2450822|14515|4|12| +2450822|14516|4|921| +2450822|14518|4|504| +2450822|14521|4|573| +2450822|14522|4|334| +2450822|14524|4|919| +2450822|14527|4|71| +2450822|14528|4|408| +2450822|14530|4|893| +2450822|14533|4|246| +2450822|14534|4|258| +2450822|14536|4|| +2450822|14539|4|708| +2450822|14540|4|482| +2450822|14542|4|498| +2450822|14545|4|119| +2450822|14546|4|788| +2450822|14548|4|516| +2450822|14551|4|117| +2450822|14552|4|207| +2450822|14554|4|812| +2450822|14557|4|524| +2450822|14558|4|513| +2450822|14560|4|242| +2450822|14563|4|172| +2450822|14564|4|720| +2450822|14566|4|68| +2450822|14569|4|| +2450822|14570|4|32| +2450822|14572|4|677| +2450822|14575|4|698| +2450822|14576|4|636| +2450822|14578|4|503| +2450822|14581|4|607| +2450822|14582|4|819| +2450822|14584|4|395| +2450822|14587|4|31| +2450822|14588|4|878| +2450822|14590|4|457| +2450822|14593|4|982| +2450822|14594|4|603| +2450822|14596|4|| +2450822|14599|4|398| +2450822|14600|4|| +2450822|14602|4|399| +2450822|14605|4|628| +2450822|14606|4|684| +2450822|14608|4|258| +2450822|14611|4|836| +2450822|14612|4|668| +2450822|14614|4|914| +2450822|14617|4|114| +2450822|14618|4|710| +2450822|14620|4|127| +2450822|14623|4|371| +2450822|14624|4|112| +2450822|14626|4|875| +2450822|14629|4|895| +2450822|14630|4|569| +2450822|14632|4|935| +2450822|14635|4|697| +2450822|14636|4|| +2450822|14638|4|399| +2450822|14641|4|374| +2450822|14642|4|219| +2450822|14644|4|33| +2450822|14647|4|898| +2450822|14648|4|882| +2450822|14650|4|| +2450822|14653|4|917| +2450822|14654|4|830| +2450822|14656|4|621| +2450822|14659|4|711| +2450822|14660|4|676| +2450822|14662|4|456| +2450822|14665|4|| +2450822|14666|4|244| +2450822|14668|4|964| +2450822|14671|4|203| +2450822|14672|4|76| +2450822|14674|4|714| +2450822|14677|4|986| +2450822|14678|4|667| +2450822|14680|4|604| +2450822|14683|4|739| +2450822|14684|4|621| +2450822|14686|4|722| +2450822|14689|4|646| +2450822|14690|4|575| +2450822|14692|4|57| +2450822|14695|4|64| +2450822|14696|4|48| +2450822|14698|4|240| +2450822|14701|4|896| +2450822|14702|4|644| +2450822|14704|4|466| +2450822|14707|4|636| +2450822|14708|4|190| +2450822|14710|4|553| +2450822|14713|4|| +2450822|14714|4|| +2450822|14716|4|728| +2450822|14719|4|134| +2450822|14720|4|873| +2450822|14722|4|349| +2450822|14725|4|2| +2450822|14726|4|268| +2450822|14728|4|935| +2450822|14731|4|729| +2450822|14732|4|824| +2450822|14734|4|732| +2450822|14737|4|117| +2450822|14738|4|197| +2450822|14740|4|95| +2450822|14743|4|380| +2450822|14744|4|542| +2450822|14746|4|862| +2450822|14749|4|263| +2450822|14750|4|670| +2450822|14752|4|878| +2450822|14755|4|945| +2450822|14756|4|867| +2450822|14758|4|221| +2450822|14761|4|858| +2450822|14762|4|823| +2450822|14764|4|835| +2450822|14767|4|729| +2450822|14768|4|971| +2450822|14770|4|812| +2450822|14773|4|838| +2450822|14774|4|944| +2450822|14776|4|309| +2450822|14779|4|767| +2450822|14780|4|651| +2450822|14782|4|341| +2450822|14785|4|1| +2450822|14786|4|788| +2450822|14788|4|901| +2450822|14791|4|| +2450822|14792|4|527| +2450822|14794|4|147| +2450822|14797|4|126| +2450822|14798|4|73| +2450822|14800|4|946| +2450822|14803|4|879| +2450822|14804|4|635| +2450822|14806|4|311| +2450822|14809|4|826| +2450822|14810|4|143| +2450822|14812|4|302| +2450822|14815|4|208| +2450822|14816|4|922| +2450822|14818|4|232| +2450822|14821|4|485| +2450822|14822|4|93| +2450822|14824|4|856| +2450822|14827|4|176| +2450822|14828|4|85| +2450822|14830|4|112| +2450822|14833|4|566| +2450822|14834|4|214| +2450822|14836|4|602| +2450822|14839|4|787| +2450822|14840|4|40| +2450822|14842|4|989| +2450822|14845|4|457| +2450822|14846|4|652| +2450822|14848|4|824| +2450822|14851|4|43| +2450822|14852|4|| +2450822|14854|4|499| +2450822|14857|4|283| +2450822|14858|4|882| +2450822|14860|4|165| +2450822|14863|4|750| +2450822|14864|4|331| +2450822|14866|4|980| +2450822|14869|4|976| +2450822|14870|4|799| +2450822|14872|4|103| +2450822|14875|4|722| +2450822|14876|4|682| +2450822|14878|4|185| +2450822|14881|4|970| +2450822|14882|4|236| +2450822|14884|4|495| +2450822|14887|4|902| +2450822|14888|4|882| +2450822|14890|4|146| +2450822|14893|4|831| +2450822|14894|4|| +2450822|14896|4|332| +2450822|14899|4|988| +2450822|14900|4|209| +2450822|14902|4|853| +2450822|14905|4|900| +2450822|14906|4|190| +2450822|14908|4|266| +2450822|14911|4|736| +2450822|14912|4|804| +2450822|14914|4|789| +2450822|14917|4|523| +2450822|14918|4|543| +2450822|14920|4|609| +2450822|14923|4|31| +2450822|14924|4|98| +2450822|14926|4|926| +2450822|14929|4|557| +2450822|14930|4|807| +2450822|14932|4|11| +2450822|14935|4|474| +2450822|14936|4|| +2450822|14938|4|362| +2450822|14941|4|163| +2450822|14942|4|25| +2450822|14944|4|896| +2450822|14947|4|38| +2450822|14948|4|183| +2450822|14950|4|683| +2450822|14953|4|324| +2450822|14954|4|368| +2450822|14956|4|75| +2450822|14959|4|19| +2450822|14960|4|961| +2450822|14962|4|853| +2450822|14965|4|696| +2450822|14966|4|216| +2450822|14968|4|115| +2450822|14971|4|830| +2450822|14972|4|903| +2450822|14974|4|115| +2450822|14977|4|822| +2450822|14978|4|693| +2450822|14980|4|160| +2450822|14983|4|971| +2450822|14984|4|610| +2450822|14986|4|603| +2450822|14989|4|502| +2450822|14990|4|903| +2450822|14992|4|| +2450822|14995|4|224| +2450822|14996|4|822| +2450822|14998|4|864| +2450822|15001|4|674| +2450822|15002|4|96| +2450822|15004|4|795| +2450822|15007|4|| +2450822|15008|4|331| +2450822|15010|4|380| +2450822|15013|4|548| +2450822|15014|4|686| +2450822|15016|4|461| +2450822|15019|4|165| +2450822|15020|4|628| +2450822|15022|4|104| +2450822|15025|4|923| +2450822|15026|4|650| +2450822|15028|4|681| +2450822|15031|4|464| +2450822|15032|4|50| +2450822|15034|4|69| +2450822|15037|4|329| +2450822|15038|4|614| +2450822|15040|4|153| +2450822|15043|4|907| +2450822|15044|4|319| +2450822|15046|4|679| +2450822|15049|4|898| +2450822|15050|4|840| +2450822|15052|4|888| +2450822|15055|4|104| +2450822|15056|4|624| +2450822|15058|4|689| +2450822|15061|4|718| +2450822|15062|4|910| +2450822|15064|4|122| +2450822|15067|4|136| +2450822|15068|4|892| +2450822|15070|4|915| +2450822|15073|4|831| +2450822|15074|4|962| +2450822|15076|4|956| +2450822|15079|4|385| +2450822|15080|4|575| +2450822|15082|4|766| +2450822|15085|4|392| +2450822|15086|4|503| +2450822|15088|4|237| +2450822|15091|4|529| +2450822|15092|4|0| +2450822|15094|4|462| +2450822|15097|4|| +2450822|15098|4|929| +2450822|15100|4|498| +2450822|15103|4|782| +2450822|15104|4|513| +2450822|15106|4|870| +2450822|15109|4|770| +2450822|15110|4|| +2450822|15112|4|149| +2450822|15115|4|45| +2450822|15116|4|653| +2450822|15118|4|870| +2450822|15121|4|688| +2450822|15122|4|194| +2450822|15124|4|480| +2450822|15127|4|578| +2450822|15128|4|67| +2450822|15130|4|455| +2450822|15133|4|598| +2450822|15134|4|520| +2450822|15136|4|688| +2450822|15139|4|791| +2450822|15140|4|907| +2450822|15142|4|949| +2450822|15145|4|| +2450822|15146|4|979| +2450822|15148|4|379| +2450822|15151|4|571| +2450822|15152|4|594| +2450822|15154|4|663| +2450822|15157|4|111| +2450822|15158|4|497| +2450822|15160|4|729| +2450822|15163|4|176| +2450822|15164|4|816| +2450822|15166|4|798| +2450822|15169|4|797| +2450822|15170|4|507| +2450822|15172|4|939| +2450822|15175|4|555| +2450822|15176|4|561| +2450822|15178|4|798| +2450822|15181|4|314| +2450822|15182|4|1| +2450822|15184|4|162| +2450822|15187|4|| +2450822|15188|4|551| +2450822|15190|4|640| +2450822|15193|4|217| +2450822|15194|4|942| +2450822|15196|4|961| +2450822|15199|4|622| +2450822|15200|4|278| +2450822|15202|4|477| +2450822|15205|4|| +2450822|15206|4|| +2450822|15208|4|| +2450822|15211|4|779| +2450822|15212|4|283| +2450822|15214|4|361| +2450822|15217|4|991| +2450822|15218|4|452| +2450822|15220|4|184| +2450822|15223|4|498| +2450822|15224|4|632| +2450822|15226|4|568| +2450822|15229|4|160| +2450822|15230|4|770| +2450822|15232|4|300| +2450822|15235|4|214| +2450822|15236|4|442| +2450822|15238|4|795| +2450822|15241|4|161| +2450822|15242|4|332| +2450822|15244|4|418| +2450822|15247|4|253| +2450822|15248|4|560| +2450822|15250|4|902| +2450822|15253|4|987| +2450822|15254|4|793| +2450822|15256|4|842| +2450822|15259|4|294| +2450822|15260|4|145| +2450822|15262|4|322| +2450822|15265|4|619| +2450822|15266|4|756| +2450822|15268|4|252| +2450822|15271|4|99| +2450822|15272|4|373| +2450822|15274|4|| +2450822|15277|4|308| +2450822|15278|4|180| +2450822|15280|4|104| +2450822|15283|4|83| +2450822|15284|4|48| +2450822|15286|4|93| +2450822|15289|4|| +2450822|15290|4|840| +2450822|15292|4|756| +2450822|15295|4|460| +2450822|15296|4|| +2450822|15298|4|199| +2450822|15301|4|268| +2450822|15302|4|739| +2450822|15304|4|143| +2450822|15307|4|31| +2450822|15308|4|862| +2450822|15310|4|696| +2450822|15313|4|479| +2450822|15314|4|| +2450822|15316|4|880| +2450822|15319|4|814| +2450822|15320|4|713| +2450822|15322|4|117| +2450822|15325|4|926| +2450822|15326|4|321| +2450822|15328|4|861| +2450822|15331|4|716| +2450822|15332|4|788| +2450822|15334|4|129| +2450822|15337|4|710| +2450822|15338|4|948| +2450822|15340|4|816| +2450822|15343|4|663| +2450822|15344|4|166| +2450822|15346|4|250| +2450822|15349|4|379| +2450822|15350|4|290| +2450822|15352|4|797| +2450822|15355|4|715| +2450822|15356|4|226| +2450822|15358|4|413| +2450822|15361|4|971| +2450822|15362|4|431| +2450822|15364|4|566| +2450822|15367|4|336| +2450822|15368|4|311| +2450822|15370|4|757| +2450822|15373|4|| +2450822|15374|4|302| +2450822|15376|4|442| +2450822|15379|4|5| +2450822|15380|4|| +2450822|15382|4|830| +2450822|15385|4|187| +2450822|15386|4|381| +2450822|15388|4|565| +2450822|15391|4|597| +2450822|15392|4|572| +2450822|15394|4|138| +2450822|15397|4|244| +2450822|15398|4|103| +2450822|15400|4|724| +2450822|15403|4|670| +2450822|15404|4|434| +2450822|15406|4|519| +2450822|15409|4|528| +2450822|15410|4|547| +2450822|15412|4|787| +2450822|15415|4|516| +2450822|15416|4|| +2450822|15418|4|990| +2450822|15421|4|949| +2450822|15422|4|309| +2450822|15424|4|599| +2450822|15427|4|572| +2450822|15428|4|818| +2450822|15430|4|468| +2450822|15433|4|| +2450822|15434|4|586| +2450822|15436|4|878| +2450822|15439|4|202| +2450822|15440|4|109| +2450822|15442|4|651| +2450822|15445|4|| +2450822|15446|4|63| +2450822|15448|4|675| +2450822|15451|4|473| +2450822|15452|4|205| +2450822|15454|4|82| +2450822|15457|4|558| +2450822|15458|4|119| +2450822|15460|4|867| +2450822|15463|4|58| +2450822|15464|4|445| +2450822|15466|4|593| +2450822|15469|4|423| +2450822|15470|4|58| +2450822|15472|4|819| +2450822|15475|4|991| +2450822|15476|4|706| +2450822|15478|4|0| +2450822|15481|4|177| +2450822|15482|4|270| +2450822|15484|4|797| +2450822|15487|4|461| +2450822|15488|4|461| +2450822|15490|4|916| +2450822|15493|4|123| +2450822|15494|4|592| +2450822|15496|4|941| +2450822|15499|4|679| +2450822|15500|4|387| +2450822|15502|4|944| +2450822|15505|4|683| +2450822|15506|4|706| +2450822|15508|4|864| +2450822|15511|4|272| +2450822|15512|4|20| +2450822|15514|4|720| +2450822|15517|4|201| +2450822|15518|4|903| +2450822|15520|4|899| +2450822|15523|4|748| +2450822|15524|4|101| +2450822|15526|4|810| +2450822|15529|4|13| +2450822|15530|4|455| +2450822|15532|4|267| +2450822|15535|4|118| +2450822|15536|4|578| +2450822|15538|4|848| +2450822|15541|4|| +2450822|15542|4|296| +2450822|15544|4|263| +2450822|15547|4|302| +2450822|15548|4|68| +2450822|15550|4|538| +2450822|15553|4|376| +2450822|15554|4|| +2450822|15556|4|992| +2450822|15559|4|818| +2450822|15560|4|548| +2450822|15562|4|860| +2450822|15565|4|6| +2450822|15566|4|676| +2450822|15568|4|| +2450822|15571|4|934| +2450822|15572|4|437| +2450822|15574|4|913| +2450822|15577|4|332| +2450822|15578|4|821| +2450822|15580|4|329| +2450822|15583|4|740| +2450822|15584|4|890| +2450822|15586|4|504| +2450822|15589|4|383| +2450822|15590|4|106| +2450822|15592|4|196| +2450822|15595|4|256| +2450822|15596|4|618| +2450822|15598|4|145| +2450822|15601|4|293| +2450822|15602|4|| +2450822|15604|4|683| +2450822|15607|4|832| +2450822|15608|4|262| +2450822|15610|4|378| +2450822|15613|4|638| +2450822|15614|4|821| +2450822|15616|4|894| +2450822|15619|4|676| +2450822|15620|4|815| +2450822|15622|4|522| +2450822|15625|4|257| +2450822|15626|4|452| +2450822|15628|4|295| +2450822|15631|4|918| +2450822|15632|4|71| +2450822|15634|4|740| +2450822|15637|4|268| +2450822|15638|4|174| +2450822|15640|4|221| +2450822|15643|4|421| +2450822|15644|4|261| +2450822|15646|4|64| +2450822|15649|4|904| +2450822|15650|4|243| +2450822|15652|4|921| +2450822|15655|4|6| +2450822|15656|4|211| +2450822|15658|4|558| +2450822|15661|4|216| +2450822|15662|4|182| +2450822|15664|4|390| +2450822|15667|4|764| +2450822|15668|4|972| +2450822|15670|4|769| +2450822|15673|4|736| +2450822|15674|4|66| +2450822|15676|4|488| +2450822|15679|4|770| +2450822|15680|4|872| +2450822|15682|4|189| +2450822|15685|4|574| +2450822|15686|4|375| +2450822|15688|4|77| +2450822|15691|4|842| +2450822|15692|4|822| +2450822|15694|4|850| +2450822|15697|4|839| +2450822|15698|4|450| +2450822|15700|4|780| +2450822|15703|4|804| +2450822|15704|4|303| +2450822|15706|4|817| +2450822|15709|4|405| +2450822|15710|4|266| +2450822|15712|4|90| +2450822|15715|4|662| +2450822|15716|4|938| +2450822|15718|4|191| +2450822|15721|4|108| +2450822|15722|4|295| +2450822|15724|4|360| +2450822|15727|4|683| +2450822|15728|4|981| +2450822|15730|4|180| +2450822|15733|4|712| +2450822|15734|4|235| +2450822|15736|4|61| +2450822|15739|4|413| +2450822|15740|4|628| +2450822|15742|4|| +2450822|15745|4|79| +2450822|15746|4|404| +2450822|15748|4|| +2450822|15751|4|44| +2450822|15752|4|18| +2450822|15754|4|358| +2450822|15757|4|19| +2450822|15758|4|701| +2450822|15760|4|353| +2450822|15763|4|459| +2450822|15764|4|262| +2450822|15766|4|707| +2450822|15769|4|125| +2450822|15770|4|344| +2450822|15772|4|712| +2450822|15775|4|973| +2450822|15776|4|76| +2450822|15778|4|280| +2450822|15781|4|796| +2450822|15782|4|615| +2450822|15784|4|156| +2450822|15787|4|418| +2450822|15788|4|136| +2450822|15790|4|48| +2450822|15793|4|947| +2450822|15794|4|591| +2450822|15796|4|250| +2450822|15799|4|135| +2450822|15800|4|36| +2450822|15802|4|888| +2450822|15805|4|601| +2450822|15806|4|694| +2450822|15808|4|630| +2450822|15811|4|617| +2450822|15812|4|286| +2450822|15814|4|925| +2450822|15817|4|714| +2450822|15818|4|892| +2450822|15820|4|610| +2450822|15823|4|202| +2450822|15824|4|120| +2450822|15826|4|960| +2450822|15829|4|777| +2450822|15830|4|267| +2450822|15832|4|991| +2450822|15835|4|267| +2450822|15836|4|881| +2450822|15838|4|940| +2450822|15841|4|500| +2450822|15842|4|596| +2450822|15844|4|749| +2450822|15847|4|711| +2450822|15848|4|402| +2450822|15850|4|522| +2450822|15853|4|| +2450822|15854|4|435| +2450822|15856|4|31| +2450822|15859|4|882| +2450822|15860|4|756| +2450822|15862|4|559| +2450822|15865|4|79| +2450822|15866|4|| +2450822|15868|4|927| +2450822|15871|4|531| +2450822|15872|4|865| +2450822|15874|4|380| +2450822|15877|4|557| +2450822|15878|4|134| +2450822|15880|4|448| +2450822|15883|4|267| +2450822|15884|4|34| +2450822|15886|4|63| +2450822|15889|4|102| +2450822|15890|4|811| +2450822|15892|4|297| +2450822|15895|4|664| +2450822|15896|4|478| +2450822|15898|4|908| +2450822|15901|4|368| +2450822|15902|4|315| +2450822|15904|4|377| +2450822|15907|4|131| +2450822|15908|4|404| +2450822|15910|4|58| +2450822|15913|4|696| +2450822|15914|4|63| +2450822|15916|4|159| +2450822|15919|4|579| +2450822|15920|4|920| +2450822|15922|4|183| +2450822|15925|4|835| +2450822|15926|4|59| +2450822|15928|4|597| +2450822|15931|4|192| +2450822|15932|4|855| +2450822|15934|4|781| +2450822|15937|4|678| +2450822|15938|4|225| +2450822|15940|4|220| +2450822|15943|4|124| +2450822|15944|4|968| +2450822|15946|4|70| +2450822|15949|4|600| +2450822|15950|4|142| +2450822|15952|4|144| +2450822|15955|4|392| +2450822|15956|4|662| +2450822|15958|4|812| +2450822|15961|4|643| +2450822|15962|4|| +2450822|15964|4|850| +2450822|15967|4|62| +2450822|15968|4|191| +2450822|15970|4|221| +2450822|15973|4|| +2450822|15974|4|247| +2450822|15976|4|150| +2450822|15979|4|451| +2450822|15980|4|784| +2450822|15982|4|238| +2450822|15985|4|198| +2450822|15986|4|24| +2450822|15988|4|141| +2450822|15991|4|548| +2450822|15992|4|259| +2450822|15994|4|958| +2450822|15997|4|3| +2450822|15998|4|48| +2450822|16000|4|616| +2450822|16003|4|167| +2450822|16004|4|477| +2450822|16006|4|154| +2450822|16009|4|237| +2450822|16010|4|71| +2450822|16012|4|898| +2450822|16015|4|479| +2450822|16016|4|103| +2450822|16018|4|493| +2450822|16021|4|924| +2450822|16022|4|847| +2450822|16024|4|464| +2450822|16027|4|| +2450822|16028|4|733| +2450822|16030|4|171| +2450822|16033|4|578| +2450822|16034|4|993| +2450822|16036|4|658| +2450822|16039|4|| +2450822|16040|4|660| +2450822|16042|4|175| +2450822|16045|4|823| +2450822|16046|4|274| +2450822|16048|4|| +2450822|16051|4|705| +2450822|16052|4|291| +2450822|16054|4|965| +2450822|16057|4|931| +2450822|16058|4|387| +2450822|16060|4|144| +2450822|16063|4|557| +2450822|16064|4|812| +2450822|16066|4|291| +2450822|16069|4|272| +2450822|16070|4|321| +2450822|16072|4|133| +2450822|16075|4|645| +2450822|16076|4|867| +2450822|16078|4|276| +2450822|16081|4|144| +2450822|16082|4|462| +2450822|16084|4|981| +2450822|16087|4|409| +2450822|16088|4|781| +2450822|16090|4|777| +2450822|16093|4|161| +2450822|16094|4|290| +2450822|16096|4|934| +2450822|16099|4|297| +2450822|16100|4|700| +2450822|16102|4|802| +2450822|16105|4|325| +2450822|16106|4|614| +2450822|16108|4|257| +2450822|16111|4|955| +2450822|16112|4|675| +2450822|16114|4|138| +2450822|16117|4|598| +2450822|16118|4|101| +2450822|16120|4|151| +2450822|16123|4|945| +2450822|16124|4|985| +2450822|16126|4|660| +2450822|16129|4|645| +2450822|16130|4|668| +2450822|16132|4|190| +2450822|16135|4|71| +2450822|16136|4|912| +2450822|16138|4|669| +2450822|16141|4|702| +2450822|16142|4|566| +2450822|16144|4|2| +2450822|16147|4|428| +2450822|16148|4|852| +2450822|16150|4|450| +2450822|16153|4|198| +2450822|16154|4|434| +2450822|16156|4|538| +2450822|16159|4|134| +2450822|16160|4|260| +2450822|16162|4|993| +2450822|16165|4|403| +2450822|16166|4|714| +2450822|16168|4|918| +2450822|16171|4|712| +2450822|16172|4|499| +2450822|16174|4|433| +2450822|16177|4|381| +2450822|16178|4|166| +2450822|16180|4|867| +2450822|16183|4|521| +2450822|16184|4|| +2450822|16186|4|558| +2450822|16189|4|878| +2450822|16190|4|528| +2450822|16192|4|524| +2450822|16195|4|107| +2450822|16196|4|967| +2450822|16198|4|| +2450822|16201|4|393| +2450822|16202|4|418| +2450822|16204|4|242| +2450822|16207|4|429| +2450822|16208|4|720| +2450822|16210|4|110| +2450822|16213|4|709| +2450822|16214|4|128| +2450822|16216|4|915| +2450822|16219|4|67| +2450822|16220|4|413| +2450822|16222|4|919| +2450822|16225|4|668| +2450822|16226|4|368| +2450822|16228|4|987| +2450822|16231|4|786| +2450822|16232|4|959| +2450822|16234|4|858| +2450822|16237|4|481| +2450822|16238|4|158| +2450822|16240|4|261| +2450822|16243|4|521| +2450822|16244|4|99| +2450822|16246|4|82| +2450822|16249|4|293| +2450822|16250|4|845| +2450822|16252|4|313| +2450822|16255|4|777| +2450822|16256|4|623| +2450822|16258|4|60| +2450822|16261|4|709| +2450822|16262|4|| +2450822|16264|4|89| +2450822|16267|4|800| +2450822|16268|4|446| +2450822|16270|4|313| +2450822|16273|4|948| +2450822|16274|4|700| +2450822|16276|4|184| +2450822|16279|4|190| +2450822|16280|4|716| +2450822|16282|4|970| +2450822|16285|4|677| +2450822|16286|4|913| +2450822|16288|4|117| +2450822|16291|4|586| +2450822|16292|4|6| +2450822|16294|4|665| +2450822|16297|4|37| +2450822|16298|4|280| +2450822|16300|4|102| +2450822|16303|4|750| +2450822|16304|4|883| +2450822|16306|4|536| +2450822|16309|4|341| +2450822|16310|4|733| +2450822|16312|4|573| +2450822|16315|4|| +2450822|16316|4|133| +2450822|16318|4|515| +2450822|16321|4|803| +2450822|16322|4|141| +2450822|16324|4|241| +2450822|16327|4|586| +2450822|16328|4|28| +2450822|16330|4|910| +2450822|16333|4|440| +2450822|16334|4|550| +2450822|16336|4|2| +2450822|16339|4|975| +2450822|16340|4|37| +2450822|16342|4|649| +2450822|16345|4|494| +2450822|16346|4|592| +2450822|16348|4|878| +2450822|16351|4|818| +2450822|16352|4|312| +2450822|16354|4|734| +2450822|16357|4|636| +2450822|16358|4|| +2450822|16360|4|689| +2450822|16363|4|563| +2450822|16364|4|291| +2450822|16366|4|807| +2450822|16369|4|484| +2450822|16370|4|280| +2450822|16372|4|951| +2450822|16375|4|674| +2450822|16376|4|356| +2450822|16378|4|958| +2450822|16381|4|241| +2450822|16382|4|393| +2450822|16384|4|472| +2450822|16387|4|| +2450822|16388|4|757| +2450822|16390|4|361| +2450822|16393|4|665| +2450822|16394|4|954| +2450822|16396|4|697| +2450822|16399|4|589| +2450822|16400|4|690| +2450822|16402|4|442| +2450822|16405|4|183| +2450822|16406|4|| +2450822|16408|4|32| +2450822|16411|4|203| +2450822|16412|4|41| +2450822|16414|4|668| +2450822|16417|4|923| +2450822|16418|4|798| +2450822|16420|4|34| +2450822|16423|4|496| +2450822|16424|4|| +2450822|16426|4|719| +2450822|16429|4|245| +2450822|16430|4|250| +2450822|16432|4|549| +2450822|16435|4|839| +2450822|16436|4|694| +2450822|16438|4|656| +2450822|16441|4|255| +2450822|16442|4|714| +2450822|16444|4|934| +2450822|16447|4|978| +2450822|16448|4|882| +2450822|16450|4|597| +2450822|16453|4|458| +2450822|16454|4|927| +2450822|16456|4|832| +2450822|16459|4|901| +2450822|16460|4|481| +2450822|16462|4|309| +2450822|16465|4|548| +2450822|16466|4|182| +2450822|16468|4|694| +2450822|16471|4|92| +2450822|16472|4|836| +2450822|16474|4|911| +2450822|16477|4|999| +2450822|16478|4|32| +2450822|16480|4|682| +2450822|16483|4|888| +2450822|16484|4|221| +2450822|16486|4|408| +2450822|16489|4|840| +2450822|16490|4|828| +2450822|16492|4|327| +2450822|16495|4|665| +2450822|16496|4|75| +2450822|16498|4|472| +2450822|16501|4|483| +2450822|16502|4|415| +2450822|16504|4|772| +2450822|16507|4|251| +2450822|16508|4|12| +2450822|16510|4|420| +2450822|16513|4|759| +2450822|16514|4|969| +2450822|16516|4|417| +2450822|16519|4|650| +2450822|16520|4|271| +2450822|16522|4|389| +2450822|16525|4|| +2450822|16526|4|633| +2450822|16528|4|521| +2450822|16531|4|788| +2450822|16532|4|2| +2450822|16534|4|975| +2450822|16537|4|646| +2450822|16538|4|115| +2450822|16540|4|634| +2450822|16543|4|| +2450822|16544|4|20| +2450822|16546|4|377| +2450822|16549|4|975| +2450822|16550|4|| +2450822|16552|4|669| +2450822|16555|4|160| +2450822|16556|4|818| +2450822|16558|4|| +2450822|16561|4|55| +2450822|16562|4|267| +2450822|16564|4|155| +2450822|16567|4|116| +2450822|16568|4|274| +2450822|16570|4|877| +2450822|16573|4|530| +2450822|16574|4|845| +2450822|16576|4|94| +2450822|16579|4|0| +2450822|16580|4|79| +2450822|16582|4|786| +2450822|16585|4|591| +2450822|16586|4|404| +2450822|16588|4|96| +2450822|16591|4|419| +2450822|16592|4|575| +2450822|16594|4|134| +2450822|16597|4|2| +2450822|16598|4|199| +2450822|16600|4|64| +2450822|16603|4|514| +2450822|16604|4|286| +2450822|16606|4|662| +2450822|16609|4|659| +2450822|16610|4|889| +2450822|16612|4|749| +2450822|16615|4|478| +2450822|16616|4|90| +2450822|16618|4|959| +2450822|16621|4|131| +2450822|16622|4|832| +2450822|16624|4|534| +2450822|16627|4|196| +2450822|16628|4|28| +2450822|16630|4|894| +2450822|16633|4|820| +2450822|16634|4|853| +2450822|16636|4|516| +2450822|16639|4|416| +2450822|16640|4|333| +2450822|16642|4|427| +2450822|16645|4|238| +2450822|16646|4|803| +2450822|16648|4|| +2450822|16651|4|317| +2450822|16652|4|310| +2450822|16654|4|890| +2450822|16657|4|157| +2450822|16658|4|321| +2450822|16660|4|529| +2450822|16663|4|607| +2450822|16664|4|734| +2450822|16666|4|| +2450822|16669|4|451| +2450822|16670|4|| +2450822|16672|4|973| +2450822|16675|4|744| +2450822|16676|4|| +2450822|16678|4|726| +2450822|16681|4|182| +2450822|16682|4|445| +2450822|16684|4|941| +2450822|16687|4|741| +2450822|16688|4|70| +2450822|16690|4|751| +2450822|16693|4|504| +2450822|16694|4|150| +2450822|16696|4|102| +2450822|16699|4|818| +2450822|16700|4|991| +2450822|16702|4|950| +2450822|16705|4|59| +2450822|16706|4|84| +2450822|16708|4|965| +2450822|16711|4|430| +2450822|16712|4|966| +2450822|16714|4|823| +2450822|16717|4|320| +2450822|16718|4|822| +2450822|16720|4|146| +2450822|16723|4|102| +2450822|16724|4|343| +2450822|16726|4|543| +2450822|16729|4|646| +2450822|16730|4|78| +2450822|16732|4|42| +2450822|16735|4|997| +2450822|16736|4|69| +2450822|16738|4|279| +2450822|16741|4|455| +2450822|16742|4|984| +2450822|16744|4|686| +2450822|16747|4|82| +2450822|16748|4|563| +2450822|16750|4|610| +2450822|16753|4|186| +2450822|16754|4|835| +2450822|16756|4|10| +2450822|16759|4|901| +2450822|16760|4|474| +2450822|16762|4|99| +2450822|16765|4|| +2450822|16766|4|268| +2450822|16768|4|586| +2450822|16771|4|463| +2450822|16772|4|704| +2450822|16774|4|915| +2450822|16777|4|587| +2450822|16778|4|908| +2450822|16780|4|86| +2450822|16783|4|| +2450822|16784|4|737| +2450822|16786|4|746| +2450822|16789|4|125| +2450822|16790|4|624| +2450822|16792|4|394| +2450822|16795|4|877| +2450822|16796|4|717| +2450822|16798|4|34| +2450822|16801|4|324| +2450822|16802|4|542| +2450822|16804|4|241| +2450822|16807|4|137| +2450822|16808|4|333| +2450822|16810|4|62| +2450822|16813|4|919| +2450822|16814|4|653| +2450822|16816|4|318| +2450822|16819|4|182| +2450822|16820|4|889| +2450822|16822|4|554| +2450822|16825|4|307| +2450822|16826|4|672| +2450822|16828|4|952| +2450822|16831|4|618| +2450822|16832|4|403| +2450822|16834|4|46| +2450822|16837|4|416| +2450822|16838|4|769| +2450822|16840|4|712| +2450822|16843|4|767| +2450822|16844|4|108| +2450822|16846|4|586| +2450822|16849|4|882| +2450822|16850|4|513| +2450822|16852|4|62| +2450822|16855|4|183| +2450822|16856|4|958| +2450822|16858|4|886| +2450822|16861|4|726| +2450822|16862|4|24| +2450822|16864|4|186| +2450822|16867|4|476| +2450822|16868|4|434| +2450822|16870|4|203| +2450822|16873|4|922| +2450822|16874|4|391| +2450822|16876|4|718| +2450822|16879|4|| +2450822|16880|4|750| +2450822|16882|4|325| +2450822|16885|4|606| +2450822|16886|4|977| +2450822|16888|4|170| +2450822|16891|4|| +2450822|16892|4|335| +2450822|16894|4|469| +2450822|16897|4|367| +2450822|16898|4|255| +2450822|16900|4|291| +2450822|16903|4|784| +2450822|16904|4|117| +2450822|16906|4|884| +2450822|16909|4|244| +2450822|16910|4|74| +2450822|16912|4|117| +2450822|16915|4|124| +2450822|16916|4|964| +2450822|16918|4|376| +2450822|16921|4|282| +2450822|16922|4|450| +2450822|16924|4|848| +2450822|16927|4|324| +2450822|16928|4|111| +2450822|16930|4|871| +2450822|16933|4|138| +2450822|16934|4|596| +2450822|16936|4|244| +2450822|16939|4|643| +2450822|16940|4|246| +2450822|16942|4|933| +2450822|16945|4|408| +2450822|16946|4|37| +2450822|16948|4|276| +2450822|16951|4|91| +2450822|16952|4|791| +2450822|16954|4|702| +2450822|16957|4|21| +2450822|16958|4|785| +2450822|16960|4|885| +2450822|16963|4|744| +2450822|16964|4|663| +2450822|16966|4|| +2450822|16969|4|844| +2450822|16970|4|| +2450822|16972|4|578| +2450822|16975|4|777| +2450822|16976|4|232| +2450822|16978|4|164| +2450822|16981|4|193| +2450822|16982|4|348| +2450822|16984|4|70| +2450822|16987|4|732| +2450822|16988|4|556| +2450822|16990|4|430| +2450822|16993|4|403| +2450822|16994|4|487| +2450822|16996|4|227| +2450822|16999|4|403| +2450822|17000|4|344| +2450822|17002|4|| +2450822|17005|4|5| +2450822|17006|4|871| +2450822|17008|4|998| +2450822|17011|4|963| +2450822|17012|4|918| +2450822|17014|4|527| +2450822|17017|4|922| +2450822|17018|4|129| +2450822|17020|4|762| +2450822|17023|4|127| +2450822|17024|4|949| +2450822|17026|4|750| +2450822|17029|4|60| +2450822|17030|4|612| +2450822|17032|4|75| +2450822|17035|4|787| +2450822|17036|4|477| +2450822|17038|4|85| +2450822|17041|4|202| +2450822|17042|4|186| +2450822|17044|4|82| +2450822|17047|4|791| +2450822|17048|4|924| +2450822|17050|4|498| +2450822|17053|4|219| +2450822|17054|4|114| +2450822|17056|4|702| +2450822|17059|4|640| +2450822|17060|4|11| +2450822|17062|4|605| +2450822|17065|4|894| +2450822|17066|4|655| +2450822|17068|4|377| +2450822|17071|4|240| +2450822|17072|4|217| +2450822|17074|4|511| +2450822|17077|4|711| +2450822|17078|4|564| +2450822|17080|4|404| +2450822|17083|4|81| +2450822|17084|4|| +2450822|17086|4|670| +2450822|17089|4|335| +2450822|17090|4|334| +2450822|17092|4|211| +2450822|17095|4|865| +2450822|17096|4|707| +2450822|17098|4|| +2450822|17101|4|429| +2450822|17102|4|937| +2450822|17104|4|435| +2450822|17107|4|549| +2450822|17108|4|510| +2450822|17110|4|469| +2450822|17113|4|838| +2450822|17114|4|781| +2450822|17116|4|475| +2450822|17119|4|729| +2450822|17120|4|209| +2450822|17122|4|793| +2450822|17125|4|926| +2450822|17126|4|157| +2450822|17128|4|954| +2450822|17131|4|29| +2450822|17132|4|626| +2450822|17134|4|630| +2450822|17137|4|810| +2450822|17138|4|| +2450822|17140|4|934| +2450822|17143|4|| +2450822|17144|4|856| +2450822|17146|4|851| +2450822|17149|4|638| +2450822|17150|4|676| +2450822|17152|4|442| +2450822|17155|4|214| +2450822|17156|4|389| +2450822|17158|4|134| +2450822|17161|4|576| +2450822|17162|4|618| +2450822|17164|4|294| +2450822|17167|4|680| +2450822|17168|4|627| +2450822|17170|4|839| +2450822|17173|4|925| +2450822|17174|4|695| +2450822|17176|4|215| +2450822|17179|4|590| +2450822|17180|4|360| +2450822|17182|4|191| +2450822|17185|4|716| +2450822|17186|4|703| +2450822|17188|4|703| +2450822|17191|4|368| +2450822|17192|4|492| +2450822|17194|4|538| +2450822|17197|4|| +2450822|17198|4|968| +2450822|17200|4|145| +2450822|17203|4|946| +2450822|17204|4|909| +2450822|17206|4|404| +2450822|17209|4|425| +2450822|17210|4|58| +2450822|17212|4|389| +2450822|17215|4|676| +2450822|17216|4|831| +2450822|17218|4|800| +2450822|17221|4|452| +2450822|17222|4|279| +2450822|17224|4|628| +2450822|17227|4|325| +2450822|17228|4|466| +2450822|17230|4|872| +2450822|17233|4|86| +2450822|17234|4|2| +2450822|17236|4|504| +2450822|17239|4|768| +2450822|17240|4|540| +2450822|17242|4|522| +2450822|17245|4|120| +2450822|17246|4|121| +2450822|17248|4|902| +2450822|17251|4|375| +2450822|17252|4|993| +2450822|17254|4|666| +2450822|17257|4|37| +2450822|17258|4|734| +2450822|17260|4|850| +2450822|17263|4|| +2450822|17264|4|613| +2450822|17266|4|180| +2450822|17269|4|606| +2450822|17270|4|795| +2450822|17272|4|158| +2450822|17275|4|591| +2450822|17276|4|521| +2450822|17278|4|362| +2450822|17281|4|855| +2450822|17282|4|| +2450822|17284|4|| +2450822|17287|4|269| +2450822|17288|4|829| +2450822|17290|4|675| +2450822|17293|4|354| +2450822|17294|4|754| +2450822|17296|4|520| +2450822|17299|4|917| +2450822|17300|4|536| +2450822|17302|4|480| +2450822|17305|4|192| +2450822|17306|4|297| +2450822|17308|4|950| +2450822|17311|4|165| +2450822|17312|4|702| +2450822|17314|4|199| +2450822|17317|4|157| +2450822|17318|4|216| +2450822|17320|4|92| +2450822|17323|4|207| +2450822|17324|4|149| +2450822|17326|4|| +2450822|17329|4|807| +2450822|17330|4|478| +2450822|17332|4|473| +2450822|17335|4|71| +2450822|17336|4|649| +2450822|17338|4|602| +2450822|17341|4|19| +2450822|17342|4|| +2450822|17344|4|805| +2450822|17347|4|55| +2450822|17348|4|204| +2450822|17350|4|971| +2450822|17353|4|477| +2450822|17354|4|33| +2450822|17356|4|447| +2450822|17359|4|594| +2450822|17360|4|40| +2450822|17362|4|713| +2450822|17365|4|399| +2450822|17366|4|243| +2450822|17368|4|321| +2450822|17371|4|453| +2450822|17372|4|732| +2450822|17374|4|885| +2450822|17377|4|420| +2450822|17378|4|298| +2450822|17380|4|346| +2450822|17383|4|524| +2450822|17384|4|| +2450822|17386|4|102| +2450822|17389|4|7| +2450822|17390|4|407| +2450822|17392|4|167| +2450822|17395|4|224| +2450822|17396|4|92| +2450822|17398|4|675| +2450822|17401|4|983| +2450822|17402|4|660| +2450822|17404|4|57| +2450822|17407|4|628| +2450822|17408|4|414| +2450822|17410|4|815| +2450822|17413|4|200| +2450822|17414|4|660| +2450822|17416|4|448| +2450822|17419|4|634| +2450822|17420|4|554| +2450822|17422|4|948| +2450822|17425|4|551| +2450822|17426|4|801| +2450822|17428|4|86| +2450822|17431|4|615| +2450822|17432|4|620| +2450822|17434|4|338| +2450822|17437|4|836| +2450822|17438|4|864| +2450822|17440|4|750| +2450822|17443|4|331| +2450822|17444|4|693| +2450822|17446|4|747| +2450822|17449|4|400| +2450822|17450|4|781| +2450822|17452|4|939| +2450822|17455|4|114| +2450822|17456|4|636| +2450822|17458|4|325| +2450822|17461|4|360| +2450822|17462|4|| +2450822|17464|4|334| +2450822|17467|4|341| +2450822|17468|4|0| +2450822|17470|4|220| +2450822|17473|4|996| +2450822|17474|4|217| +2450822|17476|4|822| +2450822|17479|4|540| +2450822|17480|4|917| +2450822|17482|4|214| +2450822|17485|4|| +2450822|17486|4|695| +2450822|17488|4|538| +2450822|17491|4|136| +2450822|17492|4|722| +2450822|17494|4|42| +2450822|17497|4|545| +2450822|17498|4|689| +2450822|17500|4|453| +2450822|17503|4|301| +2450822|17504|4|582| +2450822|17506|4|938| +2450822|17509|4|960| +2450822|17510|4|186| +2450822|17512|4|751| +2450822|17515|4|457| +2450822|17516|4|76| +2450822|17518|4|544| +2450822|17521|4|359| +2450822|17522|4|646| +2450822|17524|4|490| +2450822|17527|4|139| +2450822|17528|4|948| +2450822|17530|4|105| +2450822|17533|4|514| +2450822|17534|4|16| +2450822|17536|4|63| +2450822|17539|4|767| +2450822|17540|4|575| +2450822|17542|4|135| +2450822|17545|4|956| +2450822|17546|4|52| +2450822|17548|4|944| +2450822|17551|4|940| +2450822|17552|4|830| +2450822|17554|4|569| +2450822|17557|4|628| +2450822|17558|4|806| +2450822|17560|4|607| +2450822|17563|4|942| +2450822|17564|4|20| +2450822|17566|4|555| +2450822|17569|4|140| +2450822|17570|4|598| +2450822|17572|4|721| +2450822|17575|4|27| +2450822|17576|4|529| +2450822|17578|4|226| +2450822|17581|4|| +2450822|17582|4|579| +2450822|17584|4|155| +2450822|17587|4|323| +2450822|17588|4|626| +2450822|17590|4|482| +2450822|17593|4|190| +2450822|17594|4|844| +2450822|17596|4|848| +2450822|17599|4|983| +2450822|17600|4|| +2450822|17602|4|868| +2450822|17605|4|838| +2450822|17606|4|263| +2450822|17608|4|200| +2450822|17611|4|287| +2450822|17612|4|518| +2450822|17614|4|814| +2450822|17617|4|982| +2450822|17618|4|439| +2450822|17620|4|285| +2450822|17623|4|542| +2450822|17624|4|832| +2450822|17626|4|668| +2450822|17629|4|485| +2450822|17630|4|669| +2450822|17632|4|199| +2450822|17635|4|52| +2450822|17636|4|186| +2450822|17638|4|739| +2450822|17641|4|893| +2450822|17642|4|834| +2450822|17644|4|348| +2450822|17647|4|848| +2450822|17648|4|767| +2450822|17650|4|496| +2450822|17653|4|652| +2450822|17654|4|293| +2450822|17656|4|291| +2450822|17659|4|670| +2450822|17660|4|12| +2450822|17662|4|927| +2450822|17665|4|93| +2450822|17666|4|742| +2450822|17668|4|254| +2450822|17671|4|223| +2450822|17672|4|893| +2450822|17674|4|965| +2450822|17677|4|126| +2450822|17678|4|397| +2450822|17680|4|887| +2450822|17683|4|430| +2450822|17684|4|| +2450822|17686|4|23| +2450822|17689|4|353| +2450822|17690|4|945| +2450822|17692|4|31| +2450822|17695|4|318| +2450822|17696|4|694| +2450822|17698|4|467| +2450822|17701|4|833| +2450822|17702|4|680| +2450822|17704|4|716| +2450822|17707|4|491| +2450822|17708|4|837| +2450822|17710|4|936| +2450822|17713|4|132| +2450822|17714|4|631| +2450822|17716|4|714| +2450822|17719|4|954| +2450822|17720|4|766| +2450822|17722|4|394| +2450822|17725|4|370| +2450822|17726|4|678| +2450822|17728|4|825| +2450822|17731|4|269| +2450822|17732|4|798| +2450822|17734|4|747| +2450822|17737|4|494| +2450822|17738|4|889| +2450822|17740|4|294| +2450822|17743|4|781| +2450822|17744|4|610| +2450822|17746|4|402| +2450822|17749|4|815| +2450822|17750|4|784| +2450822|17752|4|55| +2450822|17755|4|471| +2450822|17756|4|349| +2450822|17758|4|76| +2450822|17761|4|214| +2450822|17762|4|302| +2450822|17764|4|587| +2450822|17767|4|951| +2450822|17768|4|49| +2450822|17770|4|602| +2450822|17773|4|770| +2450822|17774|4|500| +2450822|17776|4|382| +2450822|17779|4|| +2450822|17780|4|445| +2450822|17782|4|164| +2450822|17785|4|779| +2450822|17786|4|269| +2450822|17788|4|847| +2450822|17791|4|75| +2450822|17792|4|495| +2450822|17794|4|324| +2450822|17797|4|550| +2450822|17798|4|846| +2450822|17800|4|790| +2450822|17803|4|356| +2450822|17804|4|258| +2450822|17806|4|575| +2450822|17809|4|747| +2450822|17810|4|944| +2450822|17812|4|456| +2450822|17815|4|819| +2450822|17816|4|481| +2450822|17818|4|109| +2450822|17821|4|155| +2450822|17822|4|284| +2450822|17824|4|145| +2450822|17827|4|685| +2450822|17828|4|43| +2450822|17830|4|390| +2450822|17833|4|496| +2450822|17834|4|122| +2450822|17836|4|527| +2450822|17839|4|742| +2450822|17840|4|390| +2450822|17842|4|25| +2450822|17845|4|629| +2450822|17846|4|801| +2450822|17848|4|826| +2450822|17851|4|859| +2450822|17852|4|117| +2450822|17854|4|918| +2450822|17857|4|449| +2450822|17858|4|758| +2450822|17860|4|664| +2450822|17863|4|| +2450822|17864|4|26| +2450822|17866|4|626| +2450822|17869|4|90| +2450822|17870|4|527| +2450822|17872|4|969| +2450822|17875|4|788| +2450822|17876|4|678| +2450822|17878|4|626| +2450822|17881|4|191| +2450822|17882|4|53| +2450822|17884|4|464| +2450822|17887|4|| +2450822|17888|4|274| +2450822|17890|4|715| +2450822|17893|4|14| +2450822|17894|4|902| +2450822|17896|4|438| +2450822|17899|4|125| +2450822|17900|4|563| +2450822|17902|4|888| +2450822|17905|4|682| +2450822|17906|4|168| +2450822|17908|4|423| +2450822|17911|4|815| +2450822|17912|4|414| +2450822|17914|4|322| +2450822|17917|4|215| +2450822|17918|4|489| +2450822|17920|4|587| +2450822|17923|4|722| +2450822|17924|4|179| +2450822|17926|4|24| +2450822|17929|4|743| +2450822|17930|4|752| +2450822|17932|4|284| +2450822|17935|4|814| +2450822|17936|4|993| +2450822|17938|4|740| +2450822|17941|4|157| +2450822|17942|4|440| +2450822|17944|4|381| +2450822|17947|4|831| +2450822|17948|4|667| +2450822|17950|4|163| +2450822|17953|4|771| +2450822|17954|4|949| +2450822|17956|4|769| +2450822|17959|4|418| +2450822|17960|4|716| +2450822|17962|4|452| +2450822|17965|4|421| +2450822|17966|4|542| +2450822|17968|4|973| +2450822|17971|4|850| +2450822|17972|4|185| +2450822|17974|4|619| +2450822|17977|4|121| +2450822|17978|4|445| +2450822|17980|4|427| +2450822|17983|4|851| +2450822|17984|4|393| +2450822|17986|4|758| +2450822|17989|4|672| +2450822|17990|4|900| +2450822|17992|4|933| +2450822|17995|4|63| +2450822|17996|4|| +2450822|17998|4|971| +2450822|1|5|42| +2450822|2|5|472| +2450822|4|5|312| +2450822|7|5|195| +2450822|8|5|0| +2450822|10|5|739| +2450822|13|5|4| +2450822|14|5|545| +2450822|16|5|1000| +2450822|19|5|999| +2450822|20|5|786| +2450822|22|5|178| +2450822|25|5|388| +2450822|26|5|759| +2450822|28|5|983| +2450822|31|5|679| +2450822|32|5|111| +2450822|34|5|308| +2450822|37|5|1000| +2450822|38|5|965| +2450822|40|5|836| +2450822|43|5|| +2450822|44|5|566| +2450822|46|5|7| +2450822|49|5|336| +2450822|50|5|938| +2450822|52|5|819| +2450822|55|5|| +2450822|56|5|836| +2450822|58|5|306| +2450822|61|5|52| +2450822|62|5|| +2450822|64|5|| +2450822|67|5|222| +2450822|68|5|504| +2450822|70|5|979| +2450822|73|5|426| +2450822|74|5|953| +2450822|76|5|650| +2450822|79|5|350| +2450822|80|5|967| +2450822|82|5|72| +2450822|85|5|96| +2450822|86|5|| +2450822|88|5|655| +2450822|91|5|638| +2450822|92|5|753| +2450822|94|5|69| +2450822|97|5|2| +2450822|98|5|51| +2450822|100|5|522| +2450822|103|5|193| +2450822|104|5|642| +2450822|106|5|484| +2450822|109|5|739| +2450822|110|5|134| +2450822|112|5|507| +2450822|115|5|194| +2450822|116|5|497| +2450822|118|5|57| +2450822|121|5|| +2450822|122|5|562| +2450822|124|5|288| +2450822|127|5|482| +2450822|128|5|90| +2450822|130|5|981| +2450822|133|5|249| +2450822|134|5|413| +2450822|136|5|133| +2450822|139|5|960| +2450822|140|5|992| +2450822|142|5|921| +2450822|145|5|336| +2450822|146|5|| +2450822|148|5|295| +2450822|151|5|450| +2450822|152|5|434| +2450822|154|5|307| +2450822|157|5|561| +2450822|158|5|602| +2450822|160|5|361| +2450822|163|5|790| +2450822|164|5|838| +2450822|166|5|205| +2450822|169|5|525| +2450822|170|5|446| +2450822|172|5|646| +2450822|175|5|| +2450822|176|5|419| +2450822|178|5|887| +2450822|181|5|703| +2450822|182|5|374| +2450822|184|5|801| +2450822|187|5|444| +2450822|188|5|532| +2450822|190|5|361| +2450822|193|5|943| +2450822|194|5|15| +2450822|196|5|150| +2450822|199|5|605| +2450822|200|5|581| +2450822|202|5|831| +2450822|205|5|720| +2450822|206|5|37| +2450822|208|5|551| +2450822|211|5|412| +2450822|212|5|225| +2450822|214|5|43| +2450822|217|5|708| +2450822|218|5|40| +2450822|220|5|532| +2450822|223|5|520| +2450822|224|5|97| +2450822|226|5|264| +2450822|229|5|750| +2450822|230|5|469| +2450822|232|5|544| +2450822|235|5|5| +2450822|236|5|433| +2450822|238|5|291| +2450822|241|5|693| +2450822|242|5|| +2450822|244|5|65| +2450822|247|5|396| +2450822|248|5|| +2450822|250|5|53| +2450822|253|5|867| +2450822|254|5|555| +2450822|256|5|224| +2450822|259|5|339| +2450822|260|5|358| +2450822|262|5|616| +2450822|265|5|513| +2450822|266|5|681| +2450822|268|5|540| +2450822|271|5|454| +2450822|272|5|153| +2450822|274|5|188| +2450822|277|5|109| +2450822|278|5|958| +2450822|280|5|267| +2450822|283|5|243| +2450822|284|5|283| +2450822|286|5|671| +2450822|289|5|273| +2450822|290|5|552| +2450822|292|5|555| +2450822|295|5|279| +2450822|296|5|665| +2450822|298|5|96| +2450822|301|5|365| +2450822|302|5|432| +2450822|304|5|968| +2450822|307|5|26| +2450822|308|5|818| +2450822|310|5|390| +2450822|313|5|565| +2450822|314|5|143| +2450822|316|5|512| +2450822|319|5|| +2450822|320|5|30| +2450822|322|5|563| +2450822|325|5|138| +2450822|326|5|196| +2450822|328|5|314| +2450822|331|5|537| +2450822|332|5|179| +2450822|334|5|44| +2450822|337|5|224| +2450822|338|5|566| +2450822|340|5|684| +2450822|343|5|624| +2450822|344|5|758| +2450822|346|5|978| +2450822|349|5|917| +2450822|350|5|579| +2450822|352|5|924| +2450822|355|5|269| +2450822|356|5|846| +2450822|358|5|871| +2450822|361|5|562| +2450822|362|5|599| +2450822|364|5|540| +2450822|367|5|193| +2450822|368|5|731| +2450822|370|5|209| +2450822|373|5|658| +2450822|374|5|66| +2450822|376|5|209| +2450822|379|5|98| +2450822|380|5|448| +2450822|382|5|319| +2450822|385|5|122| +2450822|386|5|508| +2450822|388|5|557| +2450822|391|5|693| +2450822|392|5|858| +2450822|394|5|125| +2450822|397|5|493| +2450822|398|5|406| +2450822|400|5|583| +2450822|403|5|461| +2450822|404|5|575| +2450822|406|5|344| +2450822|409|5|591| +2450822|410|5|858| +2450822|412|5|246| +2450822|415|5|358| +2450822|416|5|| +2450822|418|5|| +2450822|421|5|974| +2450822|422|5|920| +2450822|424|5|743| +2450822|427|5|528| +2450822|428|5|986| +2450822|430|5|380| +2450822|433|5|970| +2450822|434|5|| +2450822|436|5|451| +2450822|439|5|964| +2450822|440|5|636| +2450822|442|5|955| +2450822|445|5|| +2450822|446|5|50| +2450822|448|5|669| +2450822|451|5|62| +2450822|452|5|716| +2450822|454|5|137| +2450822|457|5|385| +2450822|458|5|737| +2450822|460|5|926| +2450822|463|5|36| +2450822|464|5|368| +2450822|466|5|364| +2450822|469|5|694| +2450822|470|5|| +2450822|472|5|744| +2450822|475|5|45| +2450822|476|5|595| +2450822|478|5|| +2450822|481|5|| +2450822|482|5|579| +2450822|484|5|199| +2450822|487|5|315| +2450822|488|5|266| +2450822|490|5|319| +2450822|493|5|679| +2450822|494|5|173| +2450822|496|5|508| +2450822|499|5|222| +2450822|500|5|337| +2450822|502|5|393| +2450822|505|5|864| +2450822|506|5|3| +2450822|508|5|604| +2450822|511|5|467| +2450822|512|5|888| +2450822|514|5|326| +2450822|517|5|318| +2450822|518|5|908| +2450822|520|5|86| +2450822|523|5|298| +2450822|524|5|797| +2450822|526|5|277| +2450822|529|5|811| +2450822|530|5|631| +2450822|532|5|479| +2450822|535|5|175| +2450822|536|5|626| +2450822|538|5|505| +2450822|541|5|821| +2450822|542|5|69| +2450822|544|5|576| +2450822|547|5|45| +2450822|548|5|| +2450822|550|5|226| +2450822|553|5|742| +2450822|554|5|602| +2450822|556|5|225| +2450822|559|5|840| +2450822|560|5|522| +2450822|562|5|127| +2450822|565|5|818| +2450822|566|5|633| +2450822|568|5|619| +2450822|571|5|13| +2450822|572|5|18| +2450822|574|5|735| +2450822|577|5|69| +2450822|578|5|546| +2450822|580|5|476| +2450822|583|5|674| +2450822|584|5|449| +2450822|586|5|599| +2450822|589|5|28| +2450822|590|5|684| +2450822|592|5|342| +2450822|595|5|618| +2450822|596|5|864| +2450822|598|5|4| +2450822|601|5|580| +2450822|602|5|756| +2450822|604|5|633| +2450822|607|5|764| +2450822|608|5|915| +2450822|610|5|391| +2450822|613|5|870| +2450822|614|5|891| +2450822|616|5|618| +2450822|619|5|644| +2450822|620|5|477| +2450822|622|5|311| +2450822|625|5|295| +2450822|626|5|959| +2450822|628|5|576| +2450822|631|5|157| +2450822|632|5|172| +2450822|634|5|210| +2450822|637|5|213| +2450822|638|5|325| +2450822|640|5|761| +2450822|643|5|702| +2450822|644|5|825| +2450822|646|5|295| +2450822|649|5|421| +2450822|650|5|231| +2450822|652|5|1| +2450822|655|5|345| +2450822|656|5|201| +2450822|658|5|492| +2450822|661|5|382| +2450822|662|5|904| +2450822|664|5|294| +2450822|667|5|741| +2450822|668|5|896| +2450822|670|5|969| +2450822|673|5|885| +2450822|674|5|221| +2450822|676|5|807| +2450822|679|5|856| +2450822|680|5|170| +2450822|682|5|896| +2450822|685|5|109| +2450822|686|5|463| +2450822|688|5|626| +2450822|691|5|397| +2450822|692|5|728| +2450822|694|5|592| +2450822|697|5|455| +2450822|698|5|694| +2450822|700|5|216| +2450822|703|5|| +2450822|704|5|971| +2450822|706|5|583| +2450822|709|5|852| +2450822|710|5|447| +2450822|712|5|458| +2450822|715|5|778| +2450822|716|5|111| +2450822|718|5|312| +2450822|721|5|| +2450822|722|5|220| +2450822|724|5|100| +2450822|727|5|799| +2450822|728|5|713| +2450822|730|5|309| +2450822|733|5|187| +2450822|734|5|783| +2450822|736|5|939| +2450822|739|5|188| +2450822|740|5|226| +2450822|742|5|132| +2450822|745|5|429| +2450822|746|5|| +2450822|748|5|86| +2450822|751|5|268| +2450822|752|5|224| +2450822|754|5|774| +2450822|757|5|20| +2450822|758|5|540| +2450822|760|5|17| +2450822|763|5|963| +2450822|764|5|214| +2450822|766|5|747| +2450822|769|5|119| +2450822|770|5|728| +2450822|772|5|661| +2450822|775|5|555| +2450822|776|5|341| +2450822|778|5|76| +2450822|781|5|363| +2450822|782|5|588| +2450822|784|5|771| +2450822|787|5|477| +2450822|788|5|102| +2450822|790|5|462| +2450822|793|5|273| +2450822|794|5|805| +2450822|796|5|161| +2450822|799|5|296| +2450822|800|5|407| +2450822|802|5|829| +2450822|805|5|838| +2450822|806|5|970| +2450822|808|5|219| +2450822|811|5|480| +2450822|812|5|557| +2450822|814|5|812| +2450822|817|5|692| +2450822|818|5|66| +2450822|820|5|222| +2450822|823|5|723| +2450822|824|5|574| +2450822|826|5|974| +2450822|829|5|534| +2450822|830|5|420| +2450822|832|5|936| +2450822|835|5|634| +2450822|836|5|724| +2450822|838|5|980| +2450822|841|5|884| +2450822|842|5|449| +2450822|844|5|| +2450822|847|5|755| +2450822|848|5|385| +2450822|850|5|392| +2450822|853|5|337| +2450822|854|5|907| +2450822|856|5|| +2450822|859|5|186| +2450822|860|5|521| +2450822|862|5|624| +2450822|865|5|268| +2450822|866|5|519| +2450822|868|5|302| +2450822|871|5|| +2450822|872|5|104| +2450822|874|5|437| +2450822|877|5|811| +2450822|878|5|348| +2450822|880|5|991| +2450822|883|5|1000| +2450822|884|5|360| +2450822|886|5|604| +2450822|889|5|88| +2450822|890|5|416| +2450822|892|5|607| +2450822|895|5|333| +2450822|896|5|277| +2450822|898|5|943| +2450822|901|5|12| +2450822|902|5|440| +2450822|904|5|767| +2450822|907|5|982| +2450822|908|5|224| +2450822|910|5|394| +2450822|913|5|553| +2450822|914|5|231| +2450822|916|5|720| +2450822|919|5|360| +2450822|920|5|335| +2450822|922|5|663| +2450822|925|5|274| +2450822|926|5|48| +2450822|928|5|608| +2450822|931|5|745| +2450822|932|5|492| +2450822|934|5|72| +2450822|937|5|389| +2450822|938|5|244| +2450822|940|5|938| +2450822|943|5|145| +2450822|944|5|295| +2450822|946|5|955| +2450822|949|5|714| +2450822|950|5|225| +2450822|952|5|959| +2450822|955|5|585| +2450822|956|5|564| +2450822|958|5|786| +2450822|961|5|818| +2450822|962|5|765| +2450822|964|5|413| +2450822|967|5|784| +2450822|968|5|906| +2450822|970|5|358| +2450822|973|5|546| +2450822|974|5|| +2450822|976|5|885| +2450822|979|5|733| +2450822|980|5|47| +2450822|982|5|952| +2450822|985|5|96| +2450822|986|5|| +2450822|988|5|351| +2450822|991|5|347| +2450822|992|5|35| +2450822|994|5|216| +2450822|997|5|429| +2450822|998|5|737| +2450822|1000|5|322| +2450822|1003|5|233| +2450822|1004|5|166| +2450822|1006|5|24| +2450822|1009|5|711| +2450822|1010|5|315| +2450822|1012|5|653| +2450822|1015|5|781| +2450822|1016|5|695| +2450822|1018|5|703| +2450822|1021|5|736| +2450822|1022|5|601| +2450822|1024|5|684| +2450822|1027|5|45| +2450822|1028|5|454| +2450822|1030|5|940| +2450822|1033|5|976| +2450822|1034|5|584| +2450822|1036|5|150| +2450822|1039|5|191| +2450822|1040|5|11| +2450822|1042|5|939| +2450822|1045|5|831| +2450822|1046|5|523| +2450822|1048|5|906| +2450822|1051|5|400| +2450822|1052|5|| +2450822|1054|5|506| +2450822|1057|5|478| +2450822|1058|5|768| +2450822|1060|5|167| +2450822|1063|5|367| +2450822|1064|5|507| +2450822|1066|5|260| +2450822|1069|5|483| +2450822|1070|5|| +2450822|1072|5|674| +2450822|1075|5|| +2450822|1076|5|75| +2450822|1078|5|315| +2450822|1081|5|998| +2450822|1082|5|569| +2450822|1084|5|35| +2450822|1087|5|809| +2450822|1088|5|527| +2450822|1090|5|590| +2450822|1093|5|957| +2450822|1094|5|617| +2450822|1096|5|591| +2450822|1099|5|567| +2450822|1100|5|444| +2450822|1102|5|| +2450822|1105|5|643| +2450822|1106|5|581| +2450822|1108|5|746| +2450822|1111|5|5| +2450822|1112|5|226| +2450822|1114|5|541| +2450822|1117|5|620| +2450822|1118|5|396| +2450822|1120|5|938| +2450822|1123|5|833| +2450822|1124|5|865| +2450822|1126|5|860| +2450822|1129|5|178| +2450822|1130|5|984| +2450822|1132|5|| +2450822|1135|5|875| +2450822|1136|5|517| +2450822|1138|5|85| +2450822|1141|5|332| +2450822|1142|5|635| +2450822|1144|5|82| +2450822|1147|5|713| +2450822|1148|5|598| +2450822|1150|5|346| +2450822|1153|5|697| +2450822|1154|5|268| +2450822|1156|5|809| +2450822|1159|5|909| +2450822|1160|5|195| +2450822|1162|5|| +2450822|1165|5|509| +2450822|1166|5|491| +2450822|1168|5|956| +2450822|1171|5|313| +2450822|1172|5|580| +2450822|1174|5|726| +2450822|1177|5|720| +2450822|1178|5|111| +2450822|1180|5|95| +2450822|1183|5|740| +2450822|1184|5|716| +2450822|1186|5|708| +2450822|1189|5|475| +2450822|1190|5|426| +2450822|1192|5|65| +2450822|1195|5|299| +2450822|1196|5|326| +2450822|1198|5|455| +2450822|1201|5|596| +2450822|1202|5|58| +2450822|1204|5|920| +2450822|1207|5|691| +2450822|1208|5|| +2450822|1210|5|928| +2450822|1213|5|288| +2450822|1214|5|62| +2450822|1216|5|32| +2450822|1219|5|869| +2450822|1220|5|113| +2450822|1222|5|823| +2450822|1225|5|849| +2450822|1226|5|292| +2450822|1228|5|76| +2450822|1231|5|228| +2450822|1232|5|140| +2450822|1234|5|934| +2450822|1237|5|712| +2450822|1238|5|893| +2450822|1240|5|724| +2450822|1243|5|93| +2450822|1244|5|341| +2450822|1246|5|814| +2450822|1249|5|439| +2450822|1250|5|325| +2450822|1252|5|452| +2450822|1255|5|684| +2450822|1256|5|873| +2450822|1258|5|739| +2450822|1261|5|871| +2450822|1262|5|54| +2450822|1264|5|274| +2450822|1267|5|472| +2450822|1268|5|193| +2450822|1270|5|786| +2450822|1273|5|530| +2450822|1274|5|933| +2450822|1276|5|172| +2450822|1279|5|206| +2450822|1280|5|510| +2450822|1282|5|23| +2450822|1285|5|67| +2450822|1286|5|| +2450822|1288|5|557| +2450822|1291|5|898| +2450822|1292|5|784| +2450822|1294|5|201| +2450822|1297|5|34| +2450822|1298|5|425| +2450822|1300|5|908| +2450822|1303|5|701| +2450822|1304|5|671| +2450822|1306|5|871| +2450822|1309|5|8| +2450822|1310|5|1| +2450822|1312|5|767| +2450822|1315|5|464| +2450822|1316|5|696| +2450822|1318|5|519| +2450822|1321|5|694| +2450822|1322|5|311| +2450822|1324|5|62| +2450822|1327|5|633| +2450822|1328|5|405| +2450822|1330|5|981| +2450822|1333|5|248| +2450822|1334|5|156| +2450822|1336|5|| +2450822|1339|5|844| +2450822|1340|5|117| +2450822|1342|5|33| +2450822|1345|5|898| +2450822|1346|5|204| +2450822|1348|5|327| +2450822|1351|5|513| +2450822|1352|5|544| +2450822|1354|5|655| +2450822|1357|5|259| +2450822|1358|5|400| +2450822|1360|5|821| +2450822|1363|5|773| +2450822|1364|5|392| +2450822|1366|5|704| +2450822|1369|5|141| +2450822|1370|5|| +2450822|1372|5|215| +2450822|1375|5|968| +2450822|1376|5|415| +2450822|1378|5|19| +2450822|1381|5|419| +2450822|1382|5|197| +2450822|1384|5|693| +2450822|1387|5|817| +2450822|1388|5|413| +2450822|1390|5|414| +2450822|1393|5|89| +2450822|1394|5|792| +2450822|1396|5|527| +2450822|1399|5|224| +2450822|1400|5|325| +2450822|1402|5|865| +2450822|1405|5|817| +2450822|1406|5|733| +2450822|1408|5|729| +2450822|1411|5|10| +2450822|1412|5|889| +2450822|1414|5|829| +2450822|1417|5|| +2450822|1418|5|298| +2450822|1420|5|843| +2450822|1423|5|545| +2450822|1424|5|147| +2450822|1426|5|542| +2450822|1429|5|250| +2450822|1430|5|859| +2450822|1432|5|128| +2450822|1435|5|568| +2450822|1436|5|519| +2450822|1438|5|806| +2450822|1441|5|218| +2450822|1442|5|414| +2450822|1444|5|409| +2450822|1447|5|911| +2450822|1448|5|748| +2450822|1450|5|4| +2450822|1453|5|140| +2450822|1454|5|103| +2450822|1456|5|461| +2450822|1459|5|501| +2450822|1460|5|503| +2450822|1462|5|508| +2450822|1465|5|417| +2450822|1466|5|981| +2450822|1468|5|505| +2450822|1471|5|608| +2450822|1472|5|99| +2450822|1474|5|842| +2450822|1477|5|411| +2450822|1478|5|814| +2450822|1480|5|144| +2450822|1483|5|947| +2450822|1484|5|728| +2450822|1486|5|559| +2450822|1489|5|39| +2450822|1490|5|| +2450822|1492|5|776| +2450822|1495|5|281| +2450822|1496|5|581| +2450822|1498|5|95| +2450822|1501|5|791| +2450822|1502|5|461| +2450822|1504|5|290| +2450822|1507|5|901| +2450822|1508|5|712| +2450822|1510|5|883| +2450822|1513|5|217| +2450822|1514|5|949| +2450822|1516|5|899| +2450822|1519|5|932| +2450822|1520|5|863| +2450822|1522|5|365| +2450822|1525|5|225| +2450822|1526|5|642| +2450822|1528|5|724| +2450822|1531|5|938| +2450822|1532|5|799| +2450822|1534|5|925| +2450822|1537|5|783| +2450822|1538|5|862| +2450822|1540|5|641| +2450822|1543|5|512| +2450822|1544|5|448| +2450822|1546|5|674| +2450822|1549|5|902| +2450822|1550|5|89| +2450822|1552|5|548| +2450822|1555|5|691| +2450822|1556|5|703| +2450822|1558|5|642| +2450822|1561|5|356| +2450822|1562|5|214| +2450822|1564|5|759| +2450822|1567|5|959| +2450822|1568|5|360| +2450822|1570|5|943| +2450822|1573|5|620| +2450822|1574|5|871| +2450822|1576|5|989| +2450822|1579|5|432| +2450822|1580|5|514| +2450822|1582|5|952| +2450822|1585|5|809| +2450822|1586|5|25| +2450822|1588|5|12| +2450822|1591|5|990| +2450822|1592|5|852| +2450822|1594|5|663| +2450822|1597|5|902| +2450822|1598|5|389| +2450822|1600|5|700| +2450822|1603|5|996| +2450822|1604|5|216| +2450822|1606|5|223| +2450822|1609|5|299| +2450822|1610|5|917| +2450822|1612|5|988| +2450822|1615|5|329| +2450822|1616|5|38| +2450822|1618|5|560| +2450822|1621|5|856| +2450822|1622|5|101| +2450822|1624|5|995| +2450822|1627|5|969| +2450822|1628|5|243| +2450822|1630|5|548| +2450822|1633|5|469| +2450822|1634|5|192| +2450822|1636|5|268| +2450822|1639|5|168| +2450822|1640|5|930| +2450822|1642|5|| +2450822|1645|5|592| +2450822|1646|5|426| +2450822|1648|5|448| +2450822|1651|5|695| +2450822|1652|5|823| +2450822|1654|5|796| +2450822|1657|5|469| +2450822|1658|5|223| +2450822|1660|5|163| +2450822|1663|5|792| +2450822|1664|5|955| +2450822|1666|5|885| +2450822|1669|5|44| +2450822|1670|5|393| +2450822|1672|5|| +2450822|1675|5|687| +2450822|1676|5|250| +2450822|1678|5|158| +2450822|1681|5|34| +2450822|1682|5|682| +2450822|1684|5|216| +2450822|1687|5|82| +2450822|1688|5|331| +2450822|1690|5|438| +2450822|1693|5|127| +2450822|1694|5|945| +2450822|1696|5|566| +2450822|1699|5|289| +2450822|1700|5|794| +2450822|1702|5|109| +2450822|1705|5|| +2450822|1706|5|604| +2450822|1708|5|858| +2450822|1711|5|488| +2450822|1712|5|634| +2450822|1714|5|688| +2450822|1717|5|874| +2450822|1718|5|122| +2450822|1720|5|43| +2450822|1723|5|686| +2450822|1724|5|212| +2450822|1726|5|134| +2450822|1729|5|767| +2450822|1730|5|991| +2450822|1732|5|516| +2450822|1735|5|920| +2450822|1736|5|843| +2450822|1738|5|529| +2450822|1741|5|860| +2450822|1742|5|415| +2450822|1744|5|134| +2450822|1747|5|309| +2450822|1748|5|637| +2450822|1750|5|81| +2450822|1753|5|213| +2450822|1754|5|4| +2450822|1756|5|158| +2450822|1759|5|765| +2450822|1760|5|970| +2450822|1762|5|608| +2450822|1765|5|230| +2450822|1766|5|412| +2450822|1768|5|17| +2450822|1771|5|170| +2450822|1772|5|528| +2450822|1774|5|475| +2450822|1777|5|923| +2450822|1778|5|626| +2450822|1780|5|759| +2450822|1783|5|734| +2450822|1784|5|475| +2450822|1786|5|709| +2450822|1789|5|81| +2450822|1790|5|345| +2450822|1792|5|| +2450822|1795|5|364| +2450822|1796|5|548| +2450822|1798|5|433| +2450822|1801|5|385| +2450822|1802|5|160| +2450822|1804|5|205| +2450822|1807|5|523| +2450822|1808|5|68| +2450822|1810|5|68| +2450822|1813|5|542| +2450822|1814|5|| +2450822|1816|5|| +2450822|1819|5|929| +2450822|1820|5|929| +2450822|1822|5|153| +2450822|1825|5|156| +2450822|1826|5|| +2450822|1828|5|766| +2450822|1831|5|493| +2450822|1832|5|762| +2450822|1834|5|160| +2450822|1837|5|754| +2450822|1838|5|631| +2450822|1840|5|71| +2450822|1843|5|103| +2450822|1844|5|203| +2450822|1846|5|77| +2450822|1849|5|316| +2450822|1850|5|864| +2450822|1852|5|188| +2450822|1855|5|447| +2450822|1856|5|708| +2450822|1858|5|338| +2450822|1861|5|871| +2450822|1862|5|597| +2450822|1864|5|809| +2450822|1867|5|542| +2450822|1868|5|970| +2450822|1870|5|216| +2450822|1873|5|561| +2450822|1874|5|375| +2450822|1876|5|417| +2450822|1879|5|356| +2450822|1880|5|515| +2450822|1882|5|860| +2450822|1885|5|436| +2450822|1886|5|450| +2450822|1888|5|824| +2450822|1891|5|| +2450822|1892|5|603| +2450822|1894|5|926| +2450822|1897|5|230| +2450822|1898|5|| +2450822|1900|5|89| +2450822|1903|5|419| +2450822|1904|5|383| +2450822|1906|5|49| +2450822|1909|5|745| +2450822|1910|5|728| +2450822|1912|5|17| +2450822|1915|5|721| +2450822|1916|5|506| +2450822|1918|5|619| +2450822|1921|5|1000| +2450822|1922|5|608| +2450822|1924|5|859| +2450822|1927|5|446| +2450822|1928|5|674| +2450822|1930|5|680| +2450822|1933|5|893| +2450822|1934|5|541| +2450822|1936|5|718| +2450822|1939|5|| +2450822|1940|5|182| +2450822|1942|5|343| +2450822|1945|5|560| +2450822|1946|5|664| +2450822|1948|5|842| +2450822|1951|5|125| +2450822|1952|5|90| +2450822|1954|5|413| +2450822|1957|5|220| +2450822|1958|5|979| +2450822|1960|5|560| +2450822|1963|5|668| +2450822|1964|5|238| +2450822|1966|5|596| +2450822|1969|5|492| +2450822|1970|5|930| +2450822|1972|5|742| +2450822|1975|5|84| +2450822|1976|5|141| +2450822|1978|5|326| +2450822|1981|5|813| +2450822|1982|5|21| +2450822|1984|5|775| +2450822|1987|5|23| +2450822|1988|5|162| +2450822|1990|5|423| +2450822|1993|5|122| +2450822|1994|5|673| +2450822|1996|5|369| +2450822|1999|5|442| +2450822|2000|5|| +2450822|2002|5|404| +2450822|2005|5|46| +2450822|2006|5|381| +2450822|2008|5|441| +2450822|2011|5|911| +2450822|2012|5|792| +2450822|2014|5|343| +2450822|2017|5|921| +2450822|2018|5|739| +2450822|2020|5|792| +2450822|2023|5|534| +2450822|2024|5|434| +2450822|2026|5|42| +2450822|2029|5|821| +2450822|2030|5|720| +2450822|2032|5|185| +2450822|2035|5|950| +2450822|2036|5|791| +2450822|2038|5|458| +2450822|2041|5|830| +2450822|2042|5|683| +2450822|2044|5|59| +2450822|2047|5|0| +2450822|2048|5|886| +2450822|2050|5|195| +2450822|2053|5|971| +2450822|2054|5|626| +2450822|2056|5|749| +2450822|2059|5|743| +2450822|2060|5|372| +2450822|2062|5|744| +2450822|2065|5|338| +2450822|2066|5|127| +2450822|2068|5|56| +2450822|2071|5|242| +2450822|2072|5|64| +2450822|2074|5|903| +2450822|2077|5|929| +2450822|2078|5|794| +2450822|2080|5|| +2450822|2083|5|678| +2450822|2084|5|866| +2450822|2086|5|186| +2450822|2089|5|279| +2450822|2090|5|339| +2450822|2092|5|388| +2450822|2095|5|284| +2450822|2096|5|732| +2450822|2098|5|848| +2450822|2101|5|967| +2450822|2102|5|1000| +2450822|2104|5|697| +2450822|2107|5|331| +2450822|2108|5|17| +2450822|2110|5|337| +2450822|2113|5|710| +2450822|2114|5|223| +2450822|2116|5|| +2450822|2119|5|| +2450822|2120|5|656| +2450822|2122|5|| +2450822|2125|5|998| +2450822|2126|5|25| +2450822|2128|5|627| +2450822|2131|5|160| +2450822|2132|5|372| +2450822|2134|5|223| +2450822|2137|5|| +2450822|2138|5|985| +2450822|2140|5|| +2450822|2143|5|32| +2450822|2144|5|21| +2450822|2146|5|604| +2450822|2149|5|745| +2450822|2150|5|938| +2450822|2152|5|206| +2450822|2155|5|549| +2450822|2156|5|520| +2450822|2158|5|952| +2450822|2161|5|678| +2450822|2162|5|380| +2450822|2164|5|221| +2450822|2167|5|702| +2450822|2168|5|618| +2450822|2170|5|288| +2450822|2173|5|778| +2450822|2174|5|821| +2450822|2176|5|780| +2450822|2179|5|| +2450822|2180|5|467| +2450822|2182|5|598| +2450822|2185|5|185| +2450822|2186|5|539| +2450822|2188|5|| +2450822|2191|5|942| +2450822|2192|5|136| +2450822|2194|5|439| +2450822|2197|5|97| +2450822|2198|5|328| +2450822|2200|5|219| +2450822|2203|5|264| +2450822|2204|5|612| +2450822|2206|5|782| +2450822|2209|5|661| +2450822|2210|5|487| +2450822|2212|5|248| +2450822|2215|5|617| +2450822|2216|5|281| +2450822|2218|5|635| +2450822|2221|5|425| +2450822|2222|5|780| +2450822|2224|5|21| +2450822|2227|5|942| +2450822|2228|5|279| +2450822|2230|5|706| +2450822|2233|5|624| +2450822|2234|5|507| +2450822|2236|5|227| +2450822|2239|5|| +2450822|2240|5|70| +2450822|2242|5|51| +2450822|2245|5|69| +2450822|2246|5|| +2450822|2248|5|790| +2450822|2251|5|109| +2450822|2252|5|| +2450822|2254|5|248| +2450822|2257|5|148| +2450822|2258|5|968| +2450822|2260|5|784| +2450822|2263|5|246| +2450822|2264|5|309| +2450822|2266|5|564| +2450822|2269|5|561| +2450822|2270|5|243| +2450822|2272|5|330| +2450822|2275|5|335| +2450822|2276|5|590| +2450822|2278|5|852| +2450822|2281|5|284| +2450822|2282|5|79| +2450822|2284|5|589| +2450822|2287|5|706| +2450822|2288|5|588| +2450822|2290|5|679| +2450822|2293|5|258| +2450822|2294|5|729| +2450822|2296|5|346| +2450822|2299|5|216| +2450822|2300|5|503| +2450822|2302|5|498| +2450822|2305|5|209| +2450822|2306|5|530| +2450822|2308|5|992| +2450822|2311|5|415| +2450822|2312|5|| +2450822|2314|5|42| +2450822|2317|5|188| +2450822|2318|5|542| +2450822|2320|5|848| +2450822|2323|5|608| +2450822|2324|5|364| +2450822|2326|5|440| +2450822|2329|5|423| +2450822|2330|5|858| +2450822|2332|5|154| +2450822|2335|5|138| +2450822|2336|5|199| +2450822|2338|5|398| +2450822|2341|5|238| +2450822|2342|5|816| +2450822|2344|5|960| +2450822|2347|5|745| +2450822|2348|5|853| +2450822|2350|5|252| +2450822|2353|5|619| +2450822|2354|5|760| +2450822|2356|5|152| +2450822|2359|5|| +2450822|2360|5|549| +2450822|2362|5|787| +2450822|2365|5|669| +2450822|2366|5|115| +2450822|2368|5|432| +2450822|2371|5|481| +2450822|2372|5|791| +2450822|2374|5|74| +2450822|2377|5|53| +2450822|2378|5|439| +2450822|2380|5|593| +2450822|2383|5|279| +2450822|2384|5|303| +2450822|2386|5|698| +2450822|2389|5|442| +2450822|2390|5|326| +2450822|2392|5|451| +2450822|2395|5|761| +2450822|2396|5|84| +2450822|2398|5|219| +2450822|2401|5|551| +2450822|2402|5|297| +2450822|2404|5|221| +2450822|2407|5|187| +2450822|2408|5|249| +2450822|2410|5|81| +2450822|2413|5|313| +2450822|2414|5|925| +2450822|2416|5|271| +2450822|2419|5|59| +2450822|2420|5|197| +2450822|2422|5|317| +2450822|2425|5|367| +2450822|2426|5|550| +2450822|2428|5|615| +2450822|2431|5|757| +2450822|2432|5|797| +2450822|2434|5|| +2450822|2437|5|635| +2450822|2438|5|29| +2450822|2440|5|321| +2450822|2443|5|438| +2450822|2444|5|907| +2450822|2446|5|358| +2450822|2449|5|202| +2450822|2450|5|713| +2450822|2452|5|79| +2450822|2455|5|683| +2450822|2456|5|854| +2450822|2458|5|847| +2450822|2461|5|423| +2450822|2462|5|945| +2450822|2464|5|599| +2450822|2467|5|840| +2450822|2468|5|586| +2450822|2470|5|| +2450822|2473|5|547| +2450822|2474|5|529| +2450822|2476|5|236| +2450822|2479|5|68| +2450822|2480|5|408| +2450822|2482|5|710| +2450822|2485|5|684| +2450822|2486|5|235| +2450822|2488|5|879| +2450822|2491|5|44| +2450822|2492|5|490| +2450822|2494|5|835| +2450822|2497|5|958| +2450822|2498|5|287| +2450822|2500|5|487| +2450822|2503|5|645| +2450822|2504|5|480| +2450822|2506|5|317| +2450822|2509|5|757| +2450822|2510|5|970| +2450822|2512|5|60| +2450822|2515|5|334| +2450822|2516|5|303| +2450822|2518|5|726| +2450822|2521|5|171| +2450822|2522|5|566| +2450822|2524|5|839| +2450822|2527|5|711| +2450822|2528|5|262| +2450822|2530|5|455| +2450822|2533|5|276| +2450822|2534|5|942| +2450822|2536|5|877| +2450822|2539|5|459| +2450822|2540|5|977| +2450822|2542|5|981| +2450822|2545|5|667| +2450822|2546|5|296| +2450822|2548|5|706| +2450822|2551|5|394| +2450822|2552|5|33| +2450822|2554|5|968| +2450822|2557|5|762| +2450822|2558|5|697| +2450822|2560|5|977| +2450822|2563|5|511| +2450822|2564|5|111| +2450822|2566|5|309| +2450822|2569|5|396| +2450822|2570|5|253| +2450822|2572|5|224| +2450822|2575|5|249| +2450822|2576|5|65| +2450822|2578|5|429| +2450822|2581|5|574| +2450822|2582|5|462| +2450822|2584|5|638| +2450822|2587|5|552| +2450822|2588|5|478| +2450822|2590|5|184| +2450822|2593|5|194| +2450822|2594|5|218| +2450822|2596|5|294| +2450822|2599|5|4| +2450822|2600|5|362| +2450822|2602|5|0| +2450822|2605|5|596| +2450822|2606|5|906| +2450822|2608|5|238| +2450822|2611|5|426| +2450822|2612|5|4| +2450822|2614|5|946| +2450822|2617|5|787| +2450822|2618|5|481| +2450822|2620|5|925| +2450822|2623|5|442| +2450822|2624|5|538| +2450822|2626|5|771| +2450822|2629|5|816| +2450822|2630|5|635| +2450822|2632|5|466| +2450822|2635|5|119| +2450822|2636|5|787| +2450822|2638|5|377| +2450822|2641|5|645| +2450822|2642|5|424| +2450822|2644|5|512| +2450822|2647|5|807| +2450822|2648|5|366| +2450822|2650|5|103| +2450822|2653|5|| +2450822|2654|5|| +2450822|2656|5|| +2450822|2659|5|651| +2450822|2660|5|494| +2450822|2662|5|543| +2450822|2665|5|482| +2450822|2666|5|823| +2450822|2668|5|293| +2450822|2671|5|498| +2450822|2672|5|562| +2450822|2674|5|413| +2450822|2677|5|763| +2450822|2678|5|756| +2450822|2680|5|988| +2450822|2683|5|287| +2450822|2684|5|| +2450822|2686|5|58| +2450822|2689|5|761| +2450822|2690|5|68| +2450822|2692|5|972| +2450822|2695|5|194| +2450822|2696|5|341| +2450822|2698|5|985| +2450822|2701|5|184| +2450822|2702|5|401| +2450822|2704|5|69| +2450822|2707|5|262| +2450822|2708|5|342| +2450822|2710|5|309| +2450822|2713|5|392| +2450822|2714|5|126| +2450822|2716|5|143| +2450822|2719|5|547| +2450822|2720|5|103| +2450822|2722|5|522| +2450822|2725|5|208| +2450822|2726|5|420| +2450822|2728|5|721| +2450822|2731|5|697| +2450822|2732|5|228| +2450822|2734|5|843| +2450822|2737|5|623| +2450822|2738|5|162| +2450822|2740|5|895| +2450822|2743|5|3| +2450822|2744|5|| +2450822|2746|5|25| +2450822|2749|5|614| +2450822|2750|5|836| +2450822|2752|5|431| +2450822|2755|5|667| +2450822|2756|5|858| +2450822|2758|5|392| +2450822|2761|5|756| +2450822|2762|5|970| +2450822|2764|5|536| +2450822|2767|5|907| +2450822|2768|5|35| +2450822|2770|5|980| +2450822|2773|5|| +2450822|2774|5|341| +2450822|2776|5|837| +2450822|2779|5|| +2450822|2780|5|634| +2450822|2782|5|553| +2450822|2785|5|482| +2450822|2786|5|681| +2450822|2788|5|322| +2450822|2791|5|187| +2450822|2792|5|20| +2450822|2794|5|300| +2450822|2797|5|911| +2450822|2798|5|349| +2450822|2800|5|385| +2450822|2803|5|673| +2450822|2804|5|255| +2450822|2806|5|503| +2450822|2809|5|813| +2450822|2810|5|306| +2450822|2812|5|205| +2450822|2815|5|861| +2450822|2816|5|144| +2450822|2818|5|287| +2450822|2821|5|560| +2450822|2822|5|106| +2450822|2824|5|61| +2450822|2827|5|716| +2450822|2828|5|356| +2450822|2830|5|718| +2450822|2833|5|286| +2450822|2834|5|300| +2450822|2836|5|713| +2450822|2839|5|502| +2450822|2840|5|626| +2450822|2842|5|724| +2450822|2845|5|| +2450822|2846|5|712| +2450822|2848|5|398| +2450822|2851|5|770| +2450822|2852|5|281| +2450822|2854|5|912| +2450822|2857|5|930| +2450822|2858|5|296| +2450822|2860|5|456| +2450822|2863|5|497| +2450822|2864|5|359| +2450822|2866|5|350| +2450822|2869|5|854| +2450822|2870|5|233| +2450822|2872|5|715| +2450822|2875|5|24| +2450822|2876|5|245| +2450822|2878|5|268| +2450822|2881|5|215| +2450822|2882|5|518| +2450822|2884|5|581| +2450822|2887|5|578| +2450822|2888|5|640| +2450822|2890|5|501| +2450822|2893|5|258| +2450822|2894|5|100| +2450822|2896|5|784| +2450822|2899|5|891| +2450822|2900|5|483| +2450822|2902|5|24| +2450822|2905|5|941| +2450822|2906|5|74| +2450822|2908|5|107| +2450822|2911|5|| +2450822|2912|5|889| +2450822|2914|5|158| +2450822|2917|5|312| +2450822|2918|5|807| +2450822|2920|5|184| +2450822|2923|5|816| +2450822|2924|5|856| +2450822|2926|5|368| +2450822|2929|5|879| +2450822|2930|5|385| +2450822|2932|5|127| +2450822|2935|5|939| +2450822|2936|5|951| +2450822|2938|5|994| +2450822|2941|5|484| +2450822|2942|5|306| +2450822|2944|5|| +2450822|2947|5|643| +2450822|2948|5|| +2450822|2950|5|160| +2450822|2953|5|836| +2450822|2954|5|13| +2450822|2956|5|919| +2450822|2959|5|72| +2450822|2960|5|597| +2450822|2962|5|775| +2450822|2965|5|306| +2450822|2966|5|360| +2450822|2968|5|988| +2450822|2971|5|552| +2450822|2972|5|534| +2450822|2974|5|632| +2450822|2977|5|262| +2450822|2978|5|53| +2450822|2980|5|117| +2450822|2983|5|482| +2450822|2984|5|747| +2450822|2986|5|362| +2450822|2989|5|156| +2450822|2990|5|737| +2450822|2992|5|611| +2450822|2995|5|405| +2450822|2996|5|713| +2450822|2998|5|698| +2450822|3001|5|983| +2450822|3002|5|175| +2450822|3004|5|259| +2450822|3007|5|571| +2450822|3008|5|| +2450822|3010|5|| +2450822|3013|5|858| +2450822|3014|5|380| +2450822|3016|5|506| +2450822|3019|5|600| +2450822|3020|5|547| +2450822|3022|5|444| +2450822|3025|5|530| +2450822|3026|5|863| +2450822|3028|5|706| +2450822|3031|5|674| +2450822|3032|5|128| +2450822|3034|5|893| +2450822|3037|5|618| +2450822|3038|5|491| +2450822|3040|5|901| +2450822|3043|5|847| +2450822|3044|5|291| +2450822|3046|5|929| +2450822|3049|5|844| +2450822|3050|5|974| +2450822|3052|5|325| +2450822|3055|5|301| +2450822|3056|5|264| +2450822|3058|5|841| +2450822|3061|5|623| +2450822|3062|5|533| +2450822|3064|5|544| +2450822|3067|5|563| +2450822|3068|5|928| +2450822|3070|5|835| +2450822|3073|5|585| +2450822|3074|5|732| +2450822|3076|5|228| +2450822|3079|5|624| +2450822|3080|5|176| +2450822|3082|5|744| +2450822|3085|5|328| +2450822|3086|5|433| +2450822|3088|5|427| +2450822|3091|5|185| +2450822|3092|5|127| +2450822|3094|5|948| +2450822|3097|5|751| +2450822|3098|5|584| +2450822|3100|5|410| +2450822|3103|5|860| +2450822|3104|5|299| +2450822|3106|5|313| +2450822|3109|5|| +2450822|3110|5|92| +2450822|3112|5|960| +2450822|3115|5|480| +2450822|3116|5|513| +2450822|3118|5|270| +2450822|3121|5|996| +2450822|3122|5|704| +2450822|3124|5|854| +2450822|3127|5|137| +2450822|3128|5|888| +2450822|3130|5|393| +2450822|3133|5|565| +2450822|3134|5|147| +2450822|3136|5|335| +2450822|3139|5|980| +2450822|3140|5|25| +2450822|3142|5|343| +2450822|3145|5|| +2450822|3146|5|384| +2450822|3148|5|187| +2450822|3151|5|971| +2450822|3152|5|177| +2450822|3154|5|448| +2450822|3157|5|677| +2450822|3158|5|335| +2450822|3160|5|631| +2450822|3163|5|37| +2450822|3164|5|191| +2450822|3166|5|638| +2450822|3169|5|651| +2450822|3170|5|609| +2450822|3172|5|863| +2450822|3175|5|288| +2450822|3176|5|856| +2450822|3178|5|931| +2450822|3181|5|884| +2450822|3182|5|709| +2450822|3184|5|548| +2450822|3187|5|398| +2450822|3188|5|41| +2450822|3190|5|281| +2450822|3193|5|224| +2450822|3194|5|645| +2450822|3196|5|| +2450822|3199|5|800| +2450822|3200|5|201| +2450822|3202|5|187| +2450822|3205|5|207| +2450822|3206|5|973| +2450822|3208|5|664| +2450822|3211|5|177| +2450822|3212|5|727| +2450822|3214|5|803| +2450822|3217|5|336| +2450822|3218|5|| +2450822|3220|5|959| +2450822|3223|5|814| +2450822|3224|5|222| +2450822|3226|5|441| +2450822|3229|5|160| +2450822|3230|5|540| +2450822|3232|5|916| +2450822|3235|5|| +2450822|3236|5|652| +2450822|3238|5|101| +2450822|3241|5|880| +2450822|3242|5|280| +2450822|3244|5|373| +2450822|3247|5|153| +2450822|3248|5|341| +2450822|3250|5|664| +2450822|3253|5|450| +2450822|3254|5|925| +2450822|3256|5|817| +2450822|3259|5|543| +2450822|3260|5|986| +2450822|3262|5|273| +2450822|3265|5|554| +2450822|3266|5|413| +2450822|3268|5|619| +2450822|3271|5|178| +2450822|3272|5|135| +2450822|3274|5|419| +2450822|3277|5|468| +2450822|3278|5|55| +2450822|3280|5|690| +2450822|3283|5|389| +2450822|3284|5|837| +2450822|3286|5|513| +2450822|3289|5|| +2450822|3290|5|268| +2450822|3292|5|430| +2450822|3295|5|1000| +2450822|3296|5|355| +2450822|3298|5|978| +2450822|3301|5|486| +2450822|3302|5|143| +2450822|3304|5|507| +2450822|3307|5|325| +2450822|3308|5|76| +2450822|3310|5|792| +2450822|3313|5|416| +2450822|3314|5|380| +2450822|3316|5|275| +2450822|3319|5|560| +2450822|3320|5|159| +2450822|3322|5|887| +2450822|3325|5|720| +2450822|3326|5|703| +2450822|3328|5|| +2450822|3331|5|380| +2450822|3332|5|| +2450822|3334|5|730| +2450822|3337|5|747| +2450822|3338|5|681| +2450822|3340|5|57| +2450822|3343|5|967| +2450822|3344|5|536| +2450822|3346|5|81| +2450822|3349|5|30| +2450822|3350|5|553| +2450822|3352|5|379| +2450822|3355|5|778| +2450822|3356|5|563| +2450822|3358|5|581| +2450822|3361|5|| +2450822|3362|5|71| +2450822|3364|5|878| +2450822|3367|5|821| +2450822|3368|5|500| +2450822|3370|5|756| +2450822|3373|5|265| +2450822|3374|5|638| +2450822|3376|5|876| +2450822|3379|5|635| +2450822|3380|5|897| +2450822|3382|5|265| +2450822|3385|5|247| +2450822|3386|5|218| +2450822|3388|5|143| +2450822|3391|5|449| +2450822|3392|5|728| +2450822|3394|5|562| +2450822|3397|5|132| +2450822|3398|5|423| +2450822|3400|5|371| +2450822|3403|5|813| +2450822|3404|5|478| +2450822|3406|5|140| +2450822|3409|5|580| +2450822|3410|5|515| +2450822|3412|5|932| +2450822|3415|5|851| +2450822|3416|5|914| +2450822|3418|5|717| +2450822|3421|5|825| +2450822|3422|5|801| +2450822|3424|5|711| +2450822|3427|5|224| +2450822|3428|5|449| +2450822|3430|5|813| +2450822|3433|5|354| +2450822|3434|5|962| +2450822|3436|5|955| +2450822|3439|5|539| +2450822|3440|5|195| +2450822|3442|5|56| +2450822|3445|5|516| +2450822|3446|5|905| +2450822|3448|5|912| +2450822|3451|5|601| +2450822|3452|5|838| +2450822|3454|5|920| +2450822|3457|5|691| +2450822|3458|5|291| +2450822|3460|5|390| +2450822|3463|5|263| +2450822|3464|5|364| +2450822|3466|5|| +2450822|3469|5|904| +2450822|3470|5|511| +2450822|3472|5|169| +2450822|3475|5|669| +2450822|3476|5|724| +2450822|3478|5|541| +2450822|3481|5|324| +2450822|3482|5|999| +2450822|3484|5|197| +2450822|3487|5|752| +2450822|3488|5|304| +2450822|3490|5|699| +2450822|3493|5|950| +2450822|3494|5|966| +2450822|3496|5|389| +2450822|3499|5|| +2450822|3500|5|59| +2450822|3502|5|| +2450822|3505|5|812| +2450822|3506|5|530| +2450822|3508|5|589| +2450822|3511|5|829| +2450822|3512|5|205| +2450822|3514|5|746| +2450822|3517|5|646| +2450822|3518|5|126| +2450822|3520|5|639| +2450822|3523|5|183| +2450822|3524|5|298| +2450822|3526|5|926| +2450822|3529|5|67| +2450822|3530|5|686| +2450822|3532|5|420| +2450822|3535|5|303| +2450822|3536|5|| +2450822|3538|5|346| +2450822|3541|5|460| +2450822|3542|5|183| +2450822|3544|5|504| +2450822|3547|5|696| +2450822|3548|5|751| +2450822|3550|5|125| +2450822|3553|5|188| +2450822|3554|5|716| +2450822|3556|5|892| +2450822|3559|5|704| +2450822|3560|5|668| +2450822|3562|5|331| +2450822|3565|5|436| +2450822|3566|5|680| +2450822|3568|5|901| +2450822|3571|5|150| +2450822|3572|5|248| +2450822|3574|5|720| +2450822|3577|5|186| +2450822|3578|5|2| +2450822|3580|5|746| +2450822|3583|5|372| +2450822|3584|5|884| +2450822|3586|5|392| +2450822|3589|5|843| +2450822|3590|5|418| +2450822|3592|5|875| +2450822|3595|5|432| +2450822|3596|5|473| +2450822|3598|5|3| +2450822|3601|5|940| +2450822|3602|5|789| +2450822|3604|5|831| +2450822|3607|5|519| +2450822|3608|5|528| +2450822|3610|5|255| +2450822|3613|5|680| +2450822|3614|5|514| +2450822|3616|5|108| +2450822|3619|5|512| +2450822|3620|5|289| +2450822|3622|5|368| +2450822|3625|5|495| +2450822|3626|5|668| +2450822|3628|5|432| +2450822|3631|5|315| +2450822|3632|5|366| +2450822|3634|5|628| +2450822|3637|5|882| +2450822|3638|5|652| +2450822|3640|5|982| +2450822|3643|5|958| +2450822|3644|5|406| +2450822|3646|5|949| +2450822|3649|5|272| +2450822|3650|5|328| +2450822|3652|5|208| +2450822|3655|5|532| +2450822|3656|5|19| +2450822|3658|5|300| +2450822|3661|5|205| +2450822|3662|5|926| +2450822|3664|5|733| +2450822|3667|5|270| +2450822|3668|5|49| +2450822|3670|5|953| +2450822|3673|5|390| +2450822|3674|5|649| +2450822|3676|5|| +2450822|3679|5|116| +2450822|3680|5|510| +2450822|3682|5|217| +2450822|3685|5|645| +2450822|3686|5|795| +2450822|3688|5|83| +2450822|3691|5|93| +2450822|3692|5|315| +2450822|3694|5|947| +2450822|3697|5|895| +2450822|3698|5|548| +2450822|3700|5|362| +2450822|3703|5|452| +2450822|3704|5|645| +2450822|3706|5|488| +2450822|3709|5|402| +2450822|3710|5|708| +2450822|3712|5|37| +2450822|3715|5|286| +2450822|3716|5|73| +2450822|3718|5|926| +2450822|3721|5|803| +2450822|3722|5|626| +2450822|3724|5|717| +2450822|3727|5|195| +2450822|3728|5|998| +2450822|3730|5|355| +2450822|3733|5|16| +2450822|3734|5|| +2450822|3736|5|633| +2450822|3739|5|117| +2450822|3740|5|510| +2450822|3742|5|836| +2450822|3745|5|| +2450822|3746|5|504| +2450822|3748|5|266| +2450822|3751|5|316| +2450822|3752|5|70| +2450822|3754|5|509| +2450822|3757|5|532| +2450822|3758|5|114| +2450822|3760|5|959| +2450822|3763|5|244| +2450822|3764|5|601| +2450822|3766|5|354| +2450822|3769|5|514| +2450822|3770|5|806| +2450822|3772|5|465| +2450822|3775|5|33| +2450822|3776|5|652| +2450822|3778|5|807| +2450822|3781|5|505| +2450822|3782|5|220| +2450822|3784|5|| +2450822|3787|5|141| +2450822|3788|5|823| +2450822|3790|5|232| +2450822|3793|5|| +2450822|3794|5|744| +2450822|3796|5|486| +2450822|3799|5|37| +2450822|3800|5|946| +2450822|3802|5|774| +2450822|3805|5|259| +2450822|3806|5|649| +2450822|3808|5|145| +2450822|3811|5|973| +2450822|3812|5|46| +2450822|3814|5|98| +2450822|3817|5|303| +2450822|3818|5|561| +2450822|3820|5|451| +2450822|3823|5|320| +2450822|3824|5|244| +2450822|3826|5|278| +2450822|3829|5|85| +2450822|3830|5|875| +2450822|3832|5|451| +2450822|3835|5|| +2450822|3836|5|459| +2450822|3838|5|940| +2450822|3841|5|610| +2450822|3842|5|440| +2450822|3844|5|545| +2450822|3847|5|606| +2450822|3848|5|266| +2450822|3850|5|49| +2450822|3853|5|261| +2450822|3854|5|250| +2450822|3856|5|930| +2450822|3859|5|60| +2450822|3860|5|146| +2450822|3862|5|461| +2450822|3865|5|53| +2450822|3866|5|958| +2450822|3868|5|144| +2450822|3871|5|884| +2450822|3872|5|203| +2450822|3874|5|586| +2450822|3877|5|70| +2450822|3878|5|712| +2450822|3880|5|118| +2450822|3883|5|413| +2450822|3884|5|599| +2450822|3886|5|622| +2450822|3889|5|542| +2450822|3890|5|208| +2450822|3892|5|437| +2450822|3895|5|917| +2450822|3896|5|772| +2450822|3898|5|773| +2450822|3901|5|| +2450822|3902|5|904| +2450822|3904|5|574| +2450822|3907|5|14| +2450822|3908|5|533| +2450822|3910|5|9| +2450822|3913|5|693| +2450822|3914|5|543| +2450822|3916|5|956| +2450822|3919|5|678| +2450822|3920|5|236| +2450822|3922|5|| +2450822|3925|5|91| +2450822|3926|5|368| +2450822|3928|5|317| +2450822|3931|5|691| +2450822|3932|5|602| +2450822|3934|5|304| +2450822|3937|5|717| +2450822|3938|5|880| +2450822|3940|5|165| +2450822|3943|5|990| +2450822|3944|5|495| +2450822|3946|5|80| +2450822|3949|5|572| +2450822|3950|5|454| +2450822|3952|5|283| +2450822|3955|5|326| +2450822|3956|5|801| +2450822|3958|5|325| +2450822|3961|5|120| +2450822|3962|5|729| +2450822|3964|5|720| +2450822|3967|5|40| +2450822|3968|5|734| +2450822|3970|5|380| +2450822|3973|5|861| +2450822|3974|5|333| +2450822|3976|5|505| +2450822|3979|5|421| +2450822|3980|5|355| +2450822|3982|5|764| +2450822|3985|5|966| +2450822|3986|5|661| +2450822|3988|5|744| +2450822|3991|5|| +2450822|3992|5|977| +2450822|3994|5|620| +2450822|3997|5|133| +2450822|3998|5|382| +2450822|4000|5|246| +2450822|4003|5|597| +2450822|4004|5|184| +2450822|4006|5|449| +2450822|4009|5|662| +2450822|4010|5|253| +2450822|4012|5|350| +2450822|4015|5|739| +2450822|4016|5|594| +2450822|4018|5|860| +2450822|4021|5|7| +2450822|4022|5|33| +2450822|4024|5|778| +2450822|4027|5|414| +2450822|4028|5|540| +2450822|4030|5|| +2450822|4033|5|672| +2450822|4034|5|240| +2450822|4036|5|293| +2450822|4039|5|30| +2450822|4040|5|793| +2450822|4042|5|558| +2450822|4045|5|130| +2450822|4046|5|661| +2450822|4048|5|352| +2450822|4051|5|255| +2450822|4052|5|643| +2450822|4054|5|| +2450822|4057|5|957| +2450822|4058|5|352| +2450822|4060|5|724| +2450822|4063|5|864| +2450822|4064|5|| +2450822|4066|5|164| +2450822|4069|5|25| +2450822|4070|5|397| +2450822|4072|5|454| +2450822|4075|5|76| +2450822|4076|5|560| +2450822|4078|5|208| +2450822|4081|5|129| +2450822|4082|5|97| +2450822|4084|5|618| +2450822|4087|5|742| +2450822|4088|5|927| +2450822|4090|5|492| +2450822|4093|5|878| +2450822|4094|5|419| +2450822|4096|5|974| +2450822|4099|5|67| +2450822|4100|5|118| +2450822|4102|5|613| +2450822|4105|5|802| +2450822|4106|5|502| +2450822|4108|5|926| +2450822|4111|5|720| +2450822|4112|5|464| +2450822|4114|5|| +2450822|4117|5|319| +2450822|4118|5|602| +2450822|4120|5|513| +2450822|4123|5|749| +2450822|4124|5|| +2450822|4126|5|659| +2450822|4129|5|| +2450822|4130|5|64| +2450822|4132|5|753| +2450822|4135|5|340| +2450822|4136|5|757| +2450822|4138|5|873| +2450822|4141|5|318| +2450822|4142|5|566| +2450822|4144|5|742| +2450822|4147|5|879| +2450822|4148|5|903| +2450822|4150|5|896| +2450822|4153|5|466| +2450822|4154|5|596| +2450822|4156|5|960| +2450822|4159|5|837| +2450822|4160|5|830| +2450822|4162|5|327| +2450822|4165|5|141| +2450822|4166|5|410| +2450822|4168|5|677| +2450822|4171|5|512| +2450822|4172|5|247| +2450822|4174|5|4| +2450822|4177|5|320| +2450822|4178|5|412| +2450822|4180|5|150| +2450822|4183|5|576| +2450822|4184|5|| +2450822|4186|5|971| +2450822|4189|5|785| +2450822|4190|5|704| +2450822|4192|5|217| +2450822|4195|5|| +2450822|4196|5|404| +2450822|4198|5|147| +2450822|4201|5|399| +2450822|4202|5|31| +2450822|4204|5|879| +2450822|4207|5|119| +2450822|4208|5|207| +2450822|4210|5|823| +2450822|4213|5|758| +2450822|4214|5|258| +2450822|4216|5|948| +2450822|4219|5|342| +2450822|4220|5|569| +2450822|4222|5|812| +2450822|4225|5|449| +2450822|4226|5|687| +2450822|4228|5|310| +2450822|4231|5|477| +2450822|4232|5|748| +2450822|4234|5|894| +2450822|4237|5|213| +2450822|4238|5|157| +2450822|4240|5|649| +2450822|4243|5|969| +2450822|4244|5|100| +2450822|4246|5|626| +2450822|4249|5|217| +2450822|4250|5|793| +2450822|4252|5|304| +2450822|4255|5|764| +2450822|4256|5|72| +2450822|4258|5|34| +2450822|4261|5|636| +2450822|4262|5|824| +2450822|4264|5|384| +2450822|4267|5|340| +2450822|4268|5|390| +2450822|4270|5|297| +2450822|4273|5|426| +2450822|4274|5|75| +2450822|4276|5|30| +2450822|4279|5|| +2450822|4280|5|876| +2450822|4282|5|| +2450822|4285|5|890| +2450822|4286|5|| +2450822|4288|5|939| +2450822|4291|5|149| +2450822|4292|5|475| +2450822|4294|5|910| +2450822|4297|5|453| +2450822|4298|5|| +2450822|4300|5|247| +2450822|4303|5|839| +2450822|4304|5|850| +2450822|4306|5|| +2450822|4309|5|145| +2450822|4310|5|769| +2450822|4312|5|813| +2450822|4315|5|235| +2450822|4316|5|594| +2450822|4318|5|732| +2450822|4321|5|910| +2450822|4322|5|793| +2450822|4324|5|77| +2450822|4327|5|373| +2450822|4328|5|785| +2450822|4330|5|| +2450822|4333|5|810| +2450822|4334|5|199| +2450822|4336|5|202| +2450822|4339|5|26| +2450822|4340|5|867| +2450822|4342|5|391| +2450822|4345|5|752| +2450822|4346|5|28| +2450822|4348|5|| +2450822|4351|5|613| +2450822|4352|5|790| +2450822|4354|5|926| +2450822|4357|5|291| +2450822|4358|5|645| +2450822|4360|5|493| +2450822|4363|5|640| +2450822|4364|5|813| +2450822|4366|5|463| +2450822|4369|5|270| +2450822|4370|5|804| +2450822|4372|5|357| +2450822|4375|5|839| +2450822|4376|5|924| +2450822|4378|5|247| +2450822|4381|5|745| +2450822|4382|5|410| +2450822|4384|5|704| +2450822|4387|5|708| +2450822|4388|5|260| +2450822|4390|5|774| +2450822|4393|5|391| +2450822|4394|5|283| +2450822|4396|5|603| +2450822|4399|5|957| +2450822|4400|5|779| +2450822|4402|5|910| +2450822|4405|5|585| +2450822|4406|5|| +2450822|4408|5|952| +2450822|4411|5|618| +2450822|4412|5|24| +2450822|4414|5|172| +2450822|4417|5|858| +2450822|4418|5|870| +2450822|4420|5|518| +2450822|4423|5|89| +2450822|4424|5|167| +2450822|4426|5|579| +2450822|4429|5|275| +2450822|4430|5|629| +2450822|4432|5|146| +2450822|4435|5|491| +2450822|4436|5|893| +2450822|4438|5|| +2450822|4441|5|892| +2450822|4442|5|| +2450822|4444|5|| +2450822|4447|5|9| +2450822|4448|5|444| +2450822|4450|5|| +2450822|4453|5|304| +2450822|4454|5|897| +2450822|4456|5|498| +2450822|4459|5|280| +2450822|4460|5|451| +2450822|4462|5|812| +2450822|4465|5|564| +2450822|4466|5|811| +2450822|4468|5|538| +2450822|4471|5|668| +2450822|4472|5|707| +2450822|4474|5|845| +2450822|4477|5|117| +2450822|4478|5|115| +2450822|4480|5|92| +2450822|4483|5|957| +2450822|4484|5|596| +2450822|4486|5|868| +2450822|4489|5|739| +2450822|4490|5|388| +2450822|4492|5|852| +2450822|4495|5|559| +2450822|4496|5|783| +2450822|4498|5|668| +2450822|4501|5|45| +2450822|4502|5|881| +2450822|4504|5|148| +2450822|4507|5|427| +2450822|4508|5|535| +2450822|4510|5|703| +2450822|4513|5|156| +2450822|4514|5|322| +2450822|4516|5|853| +2450822|4519|5|216| +2450822|4520|5|804| +2450822|4522|5|195| +2450822|4525|5|623| +2450822|4526|5|138| +2450822|4528|5|39| +2450822|4531|5|497| +2450822|4532|5|36| +2450822|4534|5|674| +2450822|4537|5|461| +2450822|4538|5|563| +2450822|4540|5|825| +2450822|4543|5|7| +2450822|4544|5|251| +2450822|4546|5|447| +2450822|4549|5|934| +2450822|4550|5|838| +2450822|4552|5|103| +2450822|4555|5|629| +2450822|4556|5|742| +2450822|4558|5|208| +2450822|4561|5|184| +2450822|4562|5|775| +2450822|4564|5|561| +2450822|4567|5|| +2450822|4568|5|379| +2450822|4570|5|452| +2450822|4573|5|950| +2450822|4574|5|127| +2450822|4576|5|858| +2450822|4579|5|375| +2450822|4580|5|145| +2450822|4582|5|962| +2450822|4585|5|| +2450822|4586|5|54| +2450822|4588|5|59| +2450822|4591|5|703| +2450822|4592|5|93| +2450822|4594|5|645| +2450822|4597|5|501| +2450822|4598|5|713| +2450822|4600|5|361| +2450822|4603|5|517| +2450822|4604|5|| +2450822|4606|5|381| +2450822|4609|5|461| +2450822|4610|5|688| +2450822|4612|5|999| +2450822|4615|5|445| +2450822|4616|5|950| +2450822|4618|5|269| +2450822|4621|5|14| +2450822|4622|5|427| +2450822|4624|5|65| +2450822|4627|5|139| +2450822|4628|5|431| +2450822|4630|5|478| +2450822|4633|5|173| +2450822|4634|5|137| +2450822|4636|5|539| +2450822|4639|5|275| +2450822|4640|5|488| +2450822|4642|5|829| +2450822|4645|5|736| +2450822|4646|5|189| +2450822|4648|5|855| +2450822|4651|5|651| +2450822|4652|5|| +2450822|4654|5|57| +2450822|4657|5|371| +2450822|4658|5|366| +2450822|4660|5|| +2450822|4663|5|116| +2450822|4664|5|904| +2450822|4666|5|176| +2450822|4669|5|857| +2450822|4670|5|984| +2450822|4672|5|677| +2450822|4675|5|416| +2450822|4676|5|104| +2450822|4678|5|281| +2450822|4681|5|616| +2450822|4682|5|574| +2450822|4684|5|69| +2450822|4687|5|771| +2450822|4688|5|239| +2450822|4690|5|859| +2450822|4693|5|961| +2450822|4694|5|601| +2450822|4696|5|692| +2450822|4699|5|226| +2450822|4700|5|327| +2450822|4702|5|645| +2450822|4705|5|918| +2450822|4706|5|292| +2450822|4708|5|585| +2450822|4711|5|103| +2450822|4712|5|89| +2450822|4714|5|981| +2450822|4717|5|737| +2450822|4718|5|586| +2450822|4720|5|918| +2450822|4723|5|697| +2450822|4724|5|612| +2450822|4726|5|764| +2450822|4729|5|225| +2450822|4730|5|896| +2450822|4732|5|108| +2450822|4735|5|205| +2450822|4736|5|162| +2450822|4738|5|247| +2450822|4741|5|685| +2450822|4742|5|911| +2450822|4744|5|713| +2450822|4747|5|452| +2450822|4748|5|812| +2450822|4750|5|58| +2450822|4753|5|968| +2450822|4754|5|548| +2450822|4756|5|175| +2450822|4759|5|424| +2450822|4760|5|54| +2450822|4762|5|831| +2450822|4765|5|592| +2450822|4766|5|140| +2450822|4768|5|889| +2450822|4771|5|632| +2450822|4772|5|170| +2450822|4774|5|450| +2450822|4777|5|27| +2450822|4778|5|88| +2450822|4780|5|213| +2450822|4783|5|374| +2450822|4784|5|512| +2450822|4786|5|11| +2450822|4789|5|458| +2450822|4790|5|893| +2450822|4792|5|369| +2450822|4795|5|699| +2450822|4796|5|194| +2450822|4798|5|58| +2450822|4801|5|122| +2450822|4802|5|343| +2450822|4804|5|828| +2450822|4807|5|292| +2450822|4808|5|182| +2450822|4810|5|53| +2450822|4813|5|63| +2450822|4814|5|31| +2450822|4816|5|923| +2450822|4819|5|590| +2450822|4820|5|969| +2450822|4822|5|774| +2450822|4825|5|520| +2450822|4826|5|| +2450822|4828|5|187| +2450822|4831|5|340| +2450822|4832|5|64| +2450822|4834|5|535| +2450822|4837|5|456| +2450822|4838|5|924| +2450822|4840|5|453| +2450822|4843|5|171| +2450822|4844|5|228| +2450822|4846|5|236| +2450822|4849|5|559| +2450822|4850|5|364| +2450822|4852|5|621| +2450822|4855|5|371| +2450822|4856|5|42| +2450822|4858|5|734| +2450822|4861|5|796| +2450822|4862|5|149| +2450822|4864|5|634| +2450822|4867|5|709| +2450822|4868|5|865| +2450822|4870|5|108| +2450822|4873|5|952| +2450822|4874|5|55| +2450822|4876|5|957| +2450822|4879|5|519| +2450822|4880|5|642| +2450822|4882|5|943| +2450822|4885|5|310| +2450822|4886|5|118| +2450822|4888|5|459| +2450822|4891|5|80| +2450822|4892|5|484| +2450822|4894|5|850| +2450822|4897|5|836| +2450822|4898|5|15| +2450822|4900|5|119| +2450822|4903|5|633| +2450822|4904|5|728| +2450822|4906|5|| +2450822|4909|5|142| +2450822|4910|5|826| +2450822|4912|5|415| +2450822|4915|5|337| +2450822|4916|5|830| +2450822|4918|5|757| +2450822|4921|5|130| +2450822|4922|5|676| +2450822|4924|5|216| +2450822|4927|5|189| +2450822|4928|5|773| +2450822|4930|5|996| +2450822|4933|5|391| +2450822|4934|5|715| +2450822|4936|5|307| +2450822|4939|5|9| +2450822|4940|5|935| +2450822|4942|5|130| +2450822|4945|5|408| +2450822|4946|5|845| +2450822|4948|5|115| +2450822|4951|5|986| +2450822|4952|5|| +2450822|4954|5|| +2450822|4957|5|565| +2450822|4958|5|| +2450822|4960|5|52| +2450822|4963|5|507| +2450822|4964|5|748| +2450822|4966|5|51| +2450822|4969|5|40| +2450822|4970|5|316| +2450822|4972|5|641| +2450822|4975|5|124| +2450822|4976|5|350| +2450822|4978|5|277| +2450822|4981|5|191| +2450822|4982|5|484| +2450822|4984|5|595| +2450822|4987|5|108| +2450822|4988|5|395| +2450822|4990|5|821| +2450822|4993|5|895| +2450822|4994|5|449| +2450822|4996|5|| +2450822|4999|5|580| +2450822|5000|5|462| +2450822|5002|5|368| +2450822|5005|5|843| +2450822|5006|5|| +2450822|5008|5|696| +2450822|5011|5|18| +2450822|5012|5|892| +2450822|5014|5|789| +2450822|5017|5|286| +2450822|5018|5|283| +2450822|5020|5|948| +2450822|5023|5|113| +2450822|5024|5|461| +2450822|5026|5|890| +2450822|5029|5|395| +2450822|5030|5|84| +2450822|5032|5|13| +2450822|5035|5|276| +2450822|5036|5|921| +2450822|5038|5|448| +2450822|5041|5|598| +2450822|5042|5|545| +2450822|5044|5|861| +2450822|5047|5|945| +2450822|5048|5|612| +2450822|5050|5|401| +2450822|5053|5|946| +2450822|5054|5|471| +2450822|5056|5|699| +2450822|5059|5|971| +2450822|5060|5|109| +2450822|5062|5|993| +2450822|5065|5|430| +2450822|5066|5|232| +2450822|5068|5|531| +2450822|5071|5|779| +2450822|5072|5|183| +2450822|5074|5|676| +2450822|5077|5|943| +2450822|5078|5|914| +2450822|5080|5|| +2450822|5083|5|342| +2450822|5084|5|529| +2450822|5086|5|899| +2450822|5089|5|588| +2450822|5090|5|427| +2450822|5092|5|313| +2450822|5095|5|873| +2450822|5096|5|750| +2450822|5098|5|282| +2450822|5101|5|916| +2450822|5102|5|698| +2450822|5104|5|379| +2450822|5107|5|675| +2450822|5108|5|524| +2450822|5110|5|203| +2450822|5113|5|| +2450822|5114|5|996| +2450822|5116|5|463| +2450822|5119|5|418| +2450822|5120|5|337| +2450822|5122|5|417| +2450822|5125|5|442| +2450822|5126|5|885| +2450822|5128|5|95| +2450822|5131|5|| +2450822|5132|5|40| +2450822|5134|5|865| +2450822|5137|5|103| +2450822|5138|5|979| +2450822|5140|5|834| +2450822|5143|5|114| +2450822|5144|5|349| +2450822|5146|5|952| +2450822|5149|5|524| +2450822|5150|5|34| +2450822|5152|5|377| +2450822|5155|5|821| +2450822|5156|5|198| +2450822|5158|5|631| +2450822|5161|5|96| +2450822|5162|5|686| +2450822|5164|5|| +2450822|5167|5|27| +2450822|5168|5|368| +2450822|5170|5|804| +2450822|5173|5|333| +2450822|5174|5|524| +2450822|5176|5|137| +2450822|5179|5|997| +2450822|5180|5|333| +2450822|5182|5|812| +2450822|5185|5|875| +2450822|5186|5|459| +2450822|5188|5|0| +2450822|5191|5|745| +2450822|5192|5|438| +2450822|5194|5|599| +2450822|5197|5|149| +2450822|5198|5|272| +2450822|5200|5|586| +2450822|5203|5|130| +2450822|5204|5|257| +2450822|5206|5|440| +2450822|5209|5|506| +2450822|5210|5|451| +2450822|5212|5|495| +2450822|5215|5|819| +2450822|5216|5|768| +2450822|5218|5|34| +2450822|5221|5|811| +2450822|5222|5|681| +2450822|5224|5|161| +2450822|5227|5|551| +2450822|5228|5|22| +2450822|5230|5|929| +2450822|5233|5|264| +2450822|5234|5|106| +2450822|5236|5|339| +2450822|5239|5|343| +2450822|5240|5|804| +2450822|5242|5|55| +2450822|5245|5|414| +2450822|5246|5|15| +2450822|5248|5|336| +2450822|5251|5|987| +2450822|5252|5|875| +2450822|5254|5|842| +2450822|5257|5|166| +2450822|5258|5|469| +2450822|5260|5|586| +2450822|5263|5|568| +2450822|5264|5|274| +2450822|5266|5|755| +2450822|5269|5|386| +2450822|5270|5|578| +2450822|5272|5|625| +2450822|5275|5|872| +2450822|5276|5|869| +2450822|5278|5|| +2450822|5281|5|786| +2450822|5282|5|62| +2450822|5284|5|484| +2450822|5287|5|357| +2450822|5288|5|323| +2450822|5290|5|964| +2450822|5293|5|468| +2450822|5294|5|121| +2450822|5296|5|862| +2450822|5299|5|594| +2450822|5300|5|261| +2450822|5302|5|154| +2450822|5305|5|176| +2450822|5306|5|118| +2450822|5308|5|849| +2450822|5311|5|408| +2450822|5312|5|| +2450822|5314|5|139| +2450822|5317|5|919| +2450822|5318|5|436| +2450822|5320|5|772| +2450822|5323|5|51| +2450822|5324|5|544| +2450822|5326|5|400| +2450822|5329|5|| +2450822|5330|5|838| +2450822|5332|5|849| +2450822|5335|5|900| +2450822|5336|5|795| +2450822|5338|5|237| +2450822|5341|5|677| +2450822|5342|5|686| +2450822|5344|5|69| +2450822|5347|5|370| +2450822|5348|5|37| +2450822|5350|5|256| +2450822|5353|5|368| +2450822|5354|5|531| +2450822|5356|5|880| +2450822|5359|5|435| +2450822|5360|5|958| +2450822|5362|5|593| +2450822|5365|5|866| +2450822|5366|5|998| +2450822|5368|5|847| +2450822|5371|5|495| +2450822|5372|5|356| +2450822|5374|5|80| +2450822|5377|5|636| +2450822|5378|5|180| +2450822|5380|5|500| +2450822|5383|5|792| +2450822|5384|5|658| +2450822|5386|5|813| +2450822|5389|5|602| +2450822|5390|5|427| +2450822|5392|5|956| +2450822|5395|5|615| +2450822|5396|5|562| +2450822|5398|5|475| +2450822|5401|5|75| +2450822|5402|5|373| +2450822|5404|5|560| +2450822|5407|5|962| +2450822|5408|5|792| +2450822|5410|5|651| +2450822|5413|5|656| +2450822|5414|5|165| +2450822|5416|5|191| +2450822|5419|5|826| +2450822|5420|5|104| +2450822|5422|5|746| +2450822|5425|5|136| +2450822|5426|5|798| +2450822|5428|5|84| +2450822|5431|5|752| +2450822|5432|5|653| +2450822|5434|5|| +2450822|5437|5|19| +2450822|5438|5|713| +2450822|5440|5|546| +2450822|5443|5|192| +2450822|5444|5|743| +2450822|5446|5|266| +2450822|5449|5|797| +2450822|5450|5|84| +2450822|5452|5|652| +2450822|5455|5|836| +2450822|5456|5|644| +2450822|5458|5|514| +2450822|5461|5|766| +2450822|5462|5|100| +2450822|5464|5|600| +2450822|5467|5|834| +2450822|5468|5|817| +2450822|5470|5|| +2450822|5473|5|639| +2450822|5474|5|676| +2450822|5476|5|392| +2450822|5479|5|502| +2450822|5480|5|477| +2450822|5482|5|261| +2450822|5485|5|732| +2450822|5486|5|894| +2450822|5488|5|367| +2450822|5491|5|409| +2450822|5492|5|264| +2450822|5494|5|21| +2450822|5497|5|941| +2450822|5498|5|491| +2450822|5500|5|900| +2450822|5503|5|198| +2450822|5504|5|790| +2450822|5506|5|84| +2450822|5509|5|519| +2450822|5510|5|752| +2450822|5512|5|351| +2450822|5515|5|585| +2450822|5516|5|812| +2450822|5518|5|238| +2450822|5521|5|308| +2450822|5522|5|280| +2450822|5524|5|319| +2450822|5527|5|| +2450822|5528|5|699| +2450822|5530|5|296| +2450822|5533|5|| +2450822|5534|5|910| +2450822|5536|5|582| +2450822|5539|5|166| +2450822|5540|5|717| +2450822|5542|5|151| +2450822|5545|5|874| +2450822|5546|5|734| +2450822|5548|5|555| +2450822|5551|5|| +2450822|5552|5|833| +2450822|5554|5|148| +2450822|5557|5|807| +2450822|5558|5|290| +2450822|5560|5|359| +2450822|5563|5|843| +2450822|5564|5|631| +2450822|5566|5|461| +2450822|5569|5|813| +2450822|5570|5|77| +2450822|5572|5|546| +2450822|5575|5|656| +2450822|5576|5|28| +2450822|5578|5|320| +2450822|5581|5|78| +2450822|5582|5|687| +2450822|5584|5|403| +2450822|5587|5|566| +2450822|5588|5|608| +2450822|5590|5|524| +2450822|5593|5|541| +2450822|5594|5|122| +2450822|5596|5|22| +2450822|5599|5|51| +2450822|5600|5|293| +2450822|5602|5|2| +2450822|5605|5|466| +2450822|5606|5|317| +2450822|5608|5|533| +2450822|5611|5|261| +2450822|5612|5|572| +2450822|5614|5|| +2450822|5617|5|702| +2450822|5618|5|710| +2450822|5620|5|613| +2450822|5623|5|952| +2450822|5624|5|564| +2450822|5626|5|51| +2450822|5629|5|4| +2450822|5630|5|904| +2450822|5632|5|645| +2450822|5635|5|293| +2450822|5636|5|943| +2450822|5638|5|441| +2450822|5641|5|783| +2450822|5642|5|603| +2450822|5644|5|265| +2450822|5647|5|447| +2450822|5648|5|170| +2450822|5650|5|29| +2450822|5653|5|503| +2450822|5654|5|541| +2450822|5656|5|311| +2450822|5659|5|771| +2450822|5660|5|749| +2450822|5662|5|780| +2450822|5665|5|638| +2450822|5666|5|948| +2450822|5668|5|979| +2450822|5671|5|973| +2450822|5672|5|625| +2450822|5674|5|442| +2450822|5677|5|149| +2450822|5678|5|755| +2450822|5680|5|555| +2450822|5683|5|30| +2450822|5684|5|186| +2450822|5686|5|| +2450822|5689|5|401| +2450822|5690|5|29| +2450822|5692|5|781| +2450822|5695|5|| +2450822|5696|5|717| +2450822|5698|5|892| +2450822|5701|5|567| +2450822|5702|5|348| +2450822|5704|5|430| +2450822|5707|5|782| +2450822|5708|5|292| +2450822|5710|5|161| +2450822|5713|5|48| +2450822|5714|5|779| +2450822|5716|5|220| +2450822|5719|5|232| +2450822|5720|5|907| +2450822|5722|5|823| +2450822|5725|5|453| +2450822|5726|5|590| +2450822|5728|5|868| +2450822|5731|5|| +2450822|5732|5|352| +2450822|5734|5|534| +2450822|5737|5|724| +2450822|5738|5|826| +2450822|5740|5|542| +2450822|5743|5|136| +2450822|5744|5|287| +2450822|5746|5|65| +2450822|5749|5|624| +2450822|5750|5|591| +2450822|5752|5|288| +2450822|5755|5|295| +2450822|5756|5|252| +2450822|5758|5|392| +2450822|5761|5|625| +2450822|5762|5|194| +2450822|5764|5|117| +2450822|5767|5|625| +2450822|5768|5|344| +2450822|5770|5|121| +2450822|5773|5|590| +2450822|5774|5|702| +2450822|5776|5|954| +2450822|5779|5|956| +2450822|5780|5|105| +2450822|5782|5|943| +2450822|5785|5|469| +2450822|5786|5|375| +2450822|5788|5|984| +2450822|5791|5|734| +2450822|5792|5|466| +2450822|5794|5|183| +2450822|5797|5|| +2450822|5798|5|373| +2450822|5800|5|325| +2450822|5803|5|955| +2450822|5804|5|891| +2450822|5806|5|687| +2450822|5809|5|56| +2450822|5810|5|410| +2450822|5812|5|872| +2450822|5815|5|886| +2450822|5816|5|172| +2450822|5818|5|89| +2450822|5821|5|976| +2450822|5822|5|908| +2450822|5824|5|173| +2450822|5827|5|878| +2450822|5828|5|445| +2450822|5830|5|871| +2450822|5833|5|989| +2450822|5834|5|113| +2450822|5836|5|136| +2450822|5839|5|833| +2450822|5840|5|234| +2450822|5842|5|241| +2450822|5845|5|475| +2450822|5846|5|709| +2450822|5848|5|748| +2450822|5851|5|385| +2450822|5852|5|671| +2450822|5854|5|459| +2450822|5857|5|838| +2450822|5858|5|670| +2450822|5860|5|657| +2450822|5863|5|161| +2450822|5864|5|988| +2450822|5866|5|27| +2450822|5869|5|283| +2450822|5870|5|31| +2450822|5872|5|70| +2450822|5875|5|705| +2450822|5876|5|686| +2450822|5878|5|53| +2450822|5881|5|636| +2450822|5882|5|360| +2450822|5884|5|307| +2450822|5887|5|348| +2450822|5888|5|862| +2450822|5890|5|938| +2450822|5893|5|386| +2450822|5894|5|134| +2450822|5896|5|249| +2450822|5899|5|| +2450822|5900|5|818| +2450822|5902|5|971| +2450822|5905|5|504| +2450822|5906|5|22| +2450822|5908|5|12| +2450822|5911|5|457| +2450822|5912|5|343| +2450822|5914|5|135| +2450822|5917|5|216| +2450822|5918|5|492| +2450822|5920|5|805| +2450822|5923|5|254| +2450822|5924|5|307| +2450822|5926|5|186| +2450822|5929|5|416| +2450822|5930|5|123| +2450822|5932|5|681| +2450822|5935|5|57| +2450822|5936|5|264| +2450822|5938|5|789| +2450822|5941|5|801| +2450822|5942|5|749| +2450822|5944|5|881| +2450822|5947|5|72| +2450822|5948|5|856| +2450822|5950|5|| +2450822|5953|5|360| +2450822|5954|5|361| +2450822|5956|5|331| +2450822|5959|5|152| +2450822|5960|5|618| +2450822|5962|5|595| +2450822|5965|5|497| +2450822|5966|5|64| +2450822|5968|5|196| +2450822|5971|5|402| +2450822|5972|5|477| +2450822|5974|5|| +2450822|5977|5|564| +2450822|5978|5|157| +2450822|5980|5|598| +2450822|5983|5|349| +2450822|5984|5|98| +2450822|5986|5|97| +2450822|5989|5|877| +2450822|5990|5|258| +2450822|5992|5|792| +2450822|5995|5|517| +2450822|5996|5|388| +2450822|5998|5|55| +2450822|6001|5|714| +2450822|6002|5|78| +2450822|6004|5|700| +2450822|6007|5|616| +2450822|6008|5|890| +2450822|6010|5|864| +2450822|6013|5|873| +2450822|6014|5|885| +2450822|6016|5|865| +2450822|6019|5|434| +2450822|6020|5|834| +2450822|6022|5|764| +2450822|6025|5|286| +2450822|6026|5|464| +2450822|6028|5|829| +2450822|6031|5|692| +2450822|6032|5|150| +2450822|6034|5|| +2450822|6037|5|979| +2450822|6038|5|935| +2450822|6040|5|381| +2450822|6043|5|905| +2450822|6044|5|442| +2450822|6046|5|498| +2450822|6049|5|| +2450822|6050|5|107| +2450822|6052|5|234| +2450822|6055|5|551| +2450822|6056|5|63| +2450822|6058|5|570| +2450822|6061|5|717| +2450822|6062|5|703| +2450822|6064|5|215| +2450822|6067|5|504| +2450822|6068|5|16| +2450822|6070|5|801| +2450822|6073|5|667| +2450822|6074|5|816| +2450822|6076|5|256| +2450822|6079|5|581| +2450822|6080|5|95| +2450822|6082|5|649| +2450822|6085|5|697| +2450822|6086|5|118| +2450822|6088|5|265| +2450822|6091|5|367| +2450822|6092|5|479| +2450822|6094|5|137| +2450822|6097|5|787| +2450822|6098|5|117| +2450822|6100|5|494| +2450822|6103|5|223| +2450822|6104|5|| +2450822|6106|5|201| +2450822|6109|5|451| +2450822|6110|5|367| +2450822|6112|5|615| +2450822|6115|5|1| +2450822|6116|5|452| +2450822|6118|5|25| +2450822|6121|5|173| +2450822|6122|5|| +2450822|6124|5|817| +2450822|6127|5|302| +2450822|6128|5|930| +2450822|6130|5|897| +2450822|6133|5|106| +2450822|6134|5|553| +2450822|6136|5|111| +2450822|6139|5|143| +2450822|6140|5|795| +2450822|6142|5|894| +2450822|6145|5|328| +2450822|6146|5|780| +2450822|6148|5|744| +2450822|6151|5|288| +2450822|6152|5|225| +2450822|6154|5|899| +2450822|6157|5|770| +2450822|6158|5|12| +2450822|6160|5|561| +2450822|6163|5|707| +2450822|6164|5|450| +2450822|6166|5|90| +2450822|6169|5|819| +2450822|6170|5|963| +2450822|6172|5|1| +2450822|6175|5|514| +2450822|6176|5|105| +2450822|6178|5|536| +2450822|6181|5|444| +2450822|6182|5|507| +2450822|6184|5|360| +2450822|6187|5|797| +2450822|6188|5|473| +2450822|6190|5|367| +2450822|6193|5|287| +2450822|6194|5|596| +2450822|6196|5|46| +2450822|6199|5|524| +2450822|6200|5|68| +2450822|6202|5|168| +2450822|6205|5|226| +2450822|6206|5|241| +2450822|6208|5|478| +2450822|6211|5|571| +2450822|6212|5|282| +2450822|6214|5|136| +2450822|6217|5|129| +2450822|6218|5|522| +2450822|6220|5|126| +2450822|6223|5|552| +2450822|6224|5|803| +2450822|6226|5|731| +2450822|6229|5|500| +2450822|6230|5|187| +2450822|6232|5|502| +2450822|6235|5|599| +2450822|6236|5|858| +2450822|6238|5|659| +2450822|6241|5|733| +2450822|6242|5|92| +2450822|6244|5|742| +2450822|6247|5|890| +2450822|6248|5|124| +2450822|6250|5|| +2450822|6253|5|723| +2450822|6254|5|617| +2450822|6256|5|33| +2450822|6259|5|586| +2450822|6260|5|296| +2450822|6262|5|611| +2450822|6265|5|357| +2450822|6266|5|695| +2450822|6268|5|953| +2450822|6271|5|114| +2450822|6272|5|560| +2450822|6274|5|418| +2450822|6277|5|276| +2450822|6278|5|951| +2450822|6280|5|274| +2450822|6283|5|492| +2450822|6284|5|30| +2450822|6286|5|74| +2450822|6289|5|111| +2450822|6290|5|709| +2450822|6292|5|602| +2450822|6295|5|388| +2450822|6296|5|979| +2450822|6298|5|155| +2450822|6301|5|484| +2450822|6302|5|817| +2450822|6304|5|488| +2450822|6307|5|725| +2450822|6308|5|468| +2450822|6310|5|621| +2450822|6313|5|536| +2450822|6314|5|131| +2450822|6316|5|523| +2450822|6319|5|523| +2450822|6320|5|609| +2450822|6322|5|315| +2450822|6325|5|908| +2450822|6326|5|629| +2450822|6328|5|549| +2450822|6331|5|36| +2450822|6332|5|624| +2450822|6334|5|678| +2450822|6337|5|| +2450822|6338|5|458| +2450822|6340|5|30| +2450822|6343|5|| +2450822|6344|5|165| +2450822|6346|5|999| +2450822|6349|5|629| +2450822|6350|5|178| +2450822|6352|5|953| +2450822|6355|5|890| +2450822|6356|5|655| +2450822|6358|5|724| +2450822|6361|5|986| +2450822|6362|5|595| +2450822|6364|5|913| +2450822|6367|5|415| +2450822|6368|5|917| +2450822|6370|5|893| +2450822|6373|5|| +2450822|6374|5|445| +2450822|6376|5|45| +2450822|6379|5|959| +2450822|6380|5|237| +2450822|6382|5|724| +2450822|6385|5|63| +2450822|6386|5|876| +2450822|6388|5|538| +2450822|6391|5|958| +2450822|6392|5|450| +2450822|6394|5|636| +2450822|6397|5|807| +2450822|6398|5|300| +2450822|6400|5|238| +2450822|6403|5|148| +2450822|6404|5|993| +2450822|6406|5|180| +2450822|6409|5|1000| +2450822|6410|5|841| +2450822|6412|5|128| +2450822|6415|5|113| +2450822|6416|5|243| +2450822|6418|5|981| +2450822|6421|5|38| +2450822|6422|5|696| +2450822|6424|5|371| +2450822|6427|5|891| +2450822|6428|5|783| +2450822|6430|5|449| +2450822|6433|5|745| +2450822|6434|5|300| +2450822|6436|5|883| +2450822|6439|5|827| +2450822|6440|5|2| +2450822|6442|5|149| +2450822|6445|5|278| +2450822|6446|5|951| +2450822|6448|5|123| +2450822|6451|5|546| +2450822|6452|5|545| +2450822|6454|5|158| +2450822|6457|5|558| +2450822|6458|5|| +2450822|6460|5|726| +2450822|6463|5|959| +2450822|6464|5|333| +2450822|6466|5|931| +2450822|6469|5|479| +2450822|6470|5|932| +2450822|6472|5|226| +2450822|6475|5|167| +2450822|6476|5|55| +2450822|6478|5|389| +2450822|6481|5|247| +2450822|6482|5|625| +2450822|6484|5|839| +2450822|6487|5|911| +2450822|6488|5|601| +2450822|6490|5|539| +2450822|6493|5|154| +2450822|6494|5|306| +2450822|6496|5|| +2450822|6499|5|449| +2450822|6500|5|73| +2450822|6502|5|169| +2450822|6505|5|305| +2450822|6506|5|773| +2450822|6508|5|172| +2450822|6511|5|913| +2450822|6512|5|70| +2450822|6514|5|385| +2450822|6517|5|974| +2450822|6518|5|320| +2450822|6520|5|43| +2450822|6523|5|7| +2450822|6524|5|972| +2450822|6526|5|809| +2450822|6529|5|182| +2450822|6530|5|795| +2450822|6532|5|688| +2450822|6535|5|275| +2450822|6536|5|645| +2450822|6538|5|946| +2450822|6541|5|570| +2450822|6542|5|225| +2450822|6544|5|975| +2450822|6547|5|150| +2450822|6548|5|925| +2450822|6550|5|787| +2450822|6553|5|346| +2450822|6554|5|268| +2450822|6556|5|676| +2450822|6559|5|938| +2450822|6560|5|718| +2450822|6562|5|362| +2450822|6565|5|758| +2450822|6566|5|120| +2450822|6568|5|125| +2450822|6571|5|963| +2450822|6572|5|293| +2450822|6574|5|618| +2450822|6577|5|635| +2450822|6578|5|738| +2450822|6580|5|564| +2450822|6583|5|925| +2450822|6584|5|499| +2450822|6586|5|938| +2450822|6589|5|530| +2450822|6590|5|376| +2450822|6592|5|682| +2450822|6595|5|842| +2450822|6596|5|724| +2450822|6598|5|457| +2450822|6601|5|| +2450822|6602|5|805| +2450822|6604|5|852| +2450822|6607|5|21| +2450822|6608|5|327| +2450822|6610|5|34| +2450822|6613|5|390| +2450822|6614|5|29| +2450822|6616|5|17| +2450822|6619|5|666| +2450822|6620|5|920| +2450822|6622|5|928| +2450822|6625|5|674| +2450822|6626|5|569| +2450822|6628|5|725| +2450822|6631|5|794| +2450822|6632|5|773| +2450822|6634|5|580| +2450822|6637|5|862| +2450822|6638|5|922| +2450822|6640|5|| +2450822|6643|5|276| +2450822|6644|5|646| +2450822|6646|5|| +2450822|6649|5|929| +2450822|6650|5|967| +2450822|6652|5|345| +2450822|6655|5|548| +2450822|6656|5|697| +2450822|6658|5|246| +2450822|6661|5|171| +2450822|6662|5|439| +2450822|6664|5|424| +2450822|6667|5|113| +2450822|6668|5|662| +2450822|6670|5|770| +2450822|6673|5|232| +2450822|6674|5|578| +2450822|6676|5|23| +2450822|6679|5|608| +2450822|6680|5|496| +2450822|6682|5|252| +2450822|6685|5|176| +2450822|6686|5|480| +2450822|6688|5|800| +2450822|6691|5|982| +2450822|6692|5|738| +2450822|6694|5|702| +2450822|6697|5|23| +2450822|6698|5|775| +2450822|6700|5|205| +2450822|6703|5|882| +2450822|6704|5|133| +2450822|6706|5|510| +2450822|6709|5|188| +2450822|6710|5|606| +2450822|6712|5|24| +2450822|6715|5|641| +2450822|6716|5|5| +2450822|6718|5|7| +2450822|6721|5|684| +2450822|6722|5|182| +2450822|6724|5|463| +2450822|6727|5|483| +2450822|6728|5|779| +2450822|6730|5|731| +2450822|6733|5|198| +2450822|6734|5|82| +2450822|6736|5|900| +2450822|6739|5|359| +2450822|6740|5|290| +2450822|6742|5|491| +2450822|6745|5|| +2450822|6746|5|555| +2450822|6748|5|| +2450822|6751|5|481| +2450822|6752|5|340| +2450822|6754|5|387| +2450822|6757|5|974| +2450822|6758|5|67| +2450822|6760|5|651| +2450822|6763|5|79| +2450822|6764|5|710| +2450822|6766|5|523| +2450822|6769|5|325| +2450822|6770|5|135| +2450822|6772|5|494| +2450822|6775|5|777| +2450822|6776|5|777| +2450822|6778|5|199| +2450822|6781|5|414| +2450822|6782|5|90| +2450822|6784|5|432| +2450822|6787|5|8| +2450822|6788|5|801| +2450822|6790|5|316| +2450822|6793|5|| +2450822|6794|5|847| +2450822|6796|5|750| +2450822|6799|5|156| +2450822|6800|5|| +2450822|6802|5|409| +2450822|6805|5|203| +2450822|6806|5|675| +2450822|6808|5|329| +2450822|6811|5|61| +2450822|6812|5|| +2450822|6814|5|906| +2450822|6817|5|9| +2450822|6818|5|200| +2450822|6820|5|831| +2450822|6823|5|262| +2450822|6824|5|882| +2450822|6826|5|303| +2450822|6829|5|262| +2450822|6830|5|630| +2450822|6832|5|49| +2450822|6835|5|273| +2450822|6836|5|115| +2450822|6838|5|24| +2450822|6841|5|381| +2450822|6842|5|915| +2450822|6844|5|585| +2450822|6847|5|590| +2450822|6848|5|5| +2450822|6850|5|781| +2450822|6853|5|667| +2450822|6854|5|23| +2450822|6856|5|182| +2450822|6859|5|147| +2450822|6860|5|650| +2450822|6862|5|| +2450822|6865|5|352| +2450822|6866|5|734| +2450822|6868|5|705| +2450822|6871|5|27| +2450822|6872|5|961| +2450822|6874|5|58| +2450822|6877|5|639| +2450822|6878|5|439| +2450822|6880|5|64| +2450822|6883|5|511| +2450822|6884|5|377| +2450822|6886|5|157| +2450822|6889|5|160| +2450822|6890|5|| +2450822|6892|5|944| +2450822|6895|5|29| +2450822|6896|5|644| +2450822|6898|5|811| +2450822|6901|5|461| +2450822|6902|5|| +2450822|6904|5|722| +2450822|6907|5|172| +2450822|6908|5|328| +2450822|6910|5|890| +2450822|6913|5|416| +2450822|6914|5|764| +2450822|6916|5|119| +2450822|6919|5|209| +2450822|6920|5|62| +2450822|6922|5|699| +2450822|6925|5|652| +2450822|6926|5|834| +2450822|6928|5|169| +2450822|6931|5|954| +2450822|6932|5|755| +2450822|6934|5|974| +2450822|6937|5|| +2450822|6938|5|478| +2450822|6940|5|203| +2450822|6943|5|106| +2450822|6944|5|826| +2450822|6946|5|708| +2450822|6949|5|519| +2450822|6950|5|962| +2450822|6952|5|228| +2450822|6955|5|634| +2450822|6956|5|117| +2450822|6958|5|548| +2450822|6961|5|979| +2450822|6962|5|942| +2450822|6964|5|929| +2450822|6967|5|203| +2450822|6968|5|290| +2450822|6970|5|| +2450822|6973|5|759| +2450822|6974|5|939| +2450822|6976|5|657| +2450822|6979|5|813| +2450822|6980|5|903| +2450822|6982|5|282| +2450822|6985|5|939| +2450822|6986|5|199| +2450822|6988|5|511| +2450822|6991|5|359| +2450822|6992|5|399| +2450822|6994|5|978| +2450822|6997|5|273| +2450822|6998|5|793| +2450822|7000|5|960| +2450822|7003|5|153| +2450822|7004|5|453| +2450822|7006|5|39| +2450822|7009|5|576| +2450822|7010|5|257| +2450822|7012|5|493| +2450822|7015|5|581| +2450822|7016|5|357| +2450822|7018|5|448| +2450822|7021|5|238| +2450822|7022|5|988| +2450822|7024|5|102| +2450822|7027|5|3| +2450822|7028|5|161| +2450822|7030|5|| +2450822|7033|5|819| +2450822|7034|5|236| +2450822|7036|5|603| +2450822|7039|5|32| +2450822|7040|5|111| +2450822|7042|5|446| +2450822|7045|5|34| +2450822|7046|5|95| +2450822|7048|5|7| +2450822|7051|5|802| +2450822|7052|5|927| +2450822|7054|5|938| +2450822|7057|5|275| +2450822|7058|5|72| +2450822|7060|5|622| +2450822|7063|5|649| +2450822|7064|5|110| +2450822|7066|5|3| +2450822|7069|5|862| +2450822|7070|5|970| +2450822|7072|5|29| +2450822|7075|5|959| +2450822|7076|5|796| +2450822|7078|5|616| +2450822|7081|5|966| +2450822|7082|5|850| +2450822|7084|5|524| +2450822|7087|5|546| +2450822|7088|5|492| +2450822|7090|5|816| +2450822|7093|5|307| +2450822|7094|5|322| +2450822|7096|5|417| +2450822|7099|5|420| +2450822|7100|5|765| +2450822|7102|5|591| +2450822|7105|5|77| +2450822|7106|5|111| +2450822|7108|5|639| +2450822|7111|5|923| +2450822|7112|5|411| +2450822|7114|5|406| +2450822|7117|5|64| +2450822|7118|5|946| +2450822|7120|5|642| +2450822|7123|5|812| +2450822|7124|5|737| +2450822|7126|5|397| +2450822|7129|5|87| +2450822|7130|5|759| +2450822|7132|5|351| +2450822|7135|5|22| +2450822|7136|5|| +2450822|7138|5|882| +2450822|7141|5|82| +2450822|7142|5|425| +2450822|7144|5|964| +2450822|7147|5|143| +2450822|7148|5|552| +2450822|7150|5|411| +2450822|7153|5|332| +2450822|7154|5|93| +2450822|7156|5|59| +2450822|7159|5|125| +2450822|7160|5|179| +2450822|7162|5|322| +2450822|7165|5|241| +2450822|7166|5|396| +2450822|7168|5|| +2450822|7171|5|573| +2450822|7172|5|215| +2450822|7174|5|312| +2450822|7177|5|261| +2450822|7178|5|375| +2450822|7180|5|418| +2450822|7183|5|547| +2450822|7184|5|133| +2450822|7186|5|550| +2450822|7189|5|203| +2450822|7190|5|641| +2450822|7192|5|908| +2450822|7195|5|136| +2450822|7196|5|557| +2450822|7198|5|650| +2450822|7201|5|352| +2450822|7202|5|214| +2450822|7204|5|6| +2450822|7207|5|800| +2450822|7208|5|695| +2450822|7210|5|89| +2450822|7213|5|438| +2450822|7214|5|562| +2450822|7216|5|468| +2450822|7219|5|586| +2450822|7220|5|244| +2450822|7222|5|170| +2450822|7225|5|14| +2450822|7226|5|35| +2450822|7228|5|322| +2450822|7231|5|334| +2450822|7232|5|137| +2450822|7234|5|223| +2450822|7237|5|371| +2450822|7238|5|532| +2450822|7240|5|471| +2450822|7243|5|60| +2450822|7244|5|| +2450822|7246|5|533| +2450822|7249|5|536| +2450822|7250|5|924| +2450822|7252|5|399| +2450822|7255|5|78| +2450822|7256|5|189| +2450822|7258|5|495| +2450822|7261|5|876| +2450822|7262|5|353| +2450822|7264|5|589| +2450822|7267|5|357| +2450822|7268|5|202| +2450822|7270|5|715| +2450822|7273|5|612| +2450822|7274|5|253| +2450822|7276|5|756| +2450822|7279|5|542| +2450822|7280|5|221| +2450822|7282|5|800| +2450822|7285|5|179| +2450822|7286|5|724| +2450822|7288|5|705| +2450822|7291|5|701| +2450822|7292|5|35| +2450822|7294|5|29| +2450822|7297|5|819| +2450822|7298|5|940| +2450822|7300|5|| +2450822|7303|5|| +2450822|7304|5|870| +2450822|7306|5|504| +2450822|7309|5|458| +2450822|7310|5|678| +2450822|7312|5|64| +2450822|7315|5|405| +2450822|7316|5|132| +2450822|7318|5|60| +2450822|7321|5|615| +2450822|7322|5|445| +2450822|7324|5|194| +2450822|7327|5|697| +2450822|7328|5|645| +2450822|7330|5|12| +2450822|7333|5|612| +2450822|7334|5|504| +2450822|7336|5|866| +2450822|7339|5|511| +2450822|7340|5|305| +2450822|7342|5|858| +2450822|7345|5|503| +2450822|7346|5|945| +2450822|7348|5|269| +2450822|7351|5|265| +2450822|7352|5|738| +2450822|7354|5|746| +2450822|7357|5|61| +2450822|7358|5|566| +2450822|7360|5|577| +2450822|7363|5|289| +2450822|7364|5|221| +2450822|7366|5|762| +2450822|7369|5|364| +2450822|7370|5|215| +2450822|7372|5|749| +2450822|7375|5|| +2450822|7376|5|609| +2450822|7378|5|612| +2450822|7381|5|461| +2450822|7382|5|760| +2450822|7384|5|457| +2450822|7387|5|636| +2450822|7388|5|| +2450822|7390|5|923| +2450822|7393|5|379| +2450822|7394|5|918| +2450822|7396|5|409| +2450822|7399|5|480| +2450822|7400|5|376| +2450822|7402|5|334| +2450822|7405|5|155| +2450822|7406|5|594| +2450822|7408|5|173| +2450822|7411|5|978| +2450822|7412|5|555| +2450822|7414|5|256| +2450822|7417|5|532| +2450822|7418|5|136| +2450822|7420|5|640| +2450822|7423|5|558| +2450822|7424|5|970| +2450822|7426|5|927| +2450822|7429|5|227| +2450822|7430|5|357| +2450822|7432|5|10| +2450822|7435|5|182| +2450822|7436|5|547| +2450822|7438|5|661| +2450822|7441|5|| +2450822|7442|5|378| +2450822|7444|5|49| +2450822|7447|5|845| +2450822|7448|5|732| +2450822|7450|5|443| +2450822|7453|5|378| +2450822|7454|5|710| +2450822|7456|5|559| +2450822|7459|5|949| +2450822|7460|5|249| +2450822|7462|5|226| +2450822|7465|5|375| +2450822|7466|5|583| +2450822|7468|5|952| +2450822|7471|5|447| +2450822|7472|5|379| +2450822|7474|5|214| +2450822|7477|5|98| +2450822|7478|5|| +2450822|7480|5|428| +2450822|7483|5|300| +2450822|7484|5|351| +2450822|7486|5|247| +2450822|7489|5|778| +2450822|7490|5|639| +2450822|7492|5|646| +2450822|7495|5|78| +2450822|7496|5|492| +2450822|7498|5|673| +2450822|7501|5|430| +2450822|7502|5|2| +2450822|7504|5|426| +2450822|7507|5|| +2450822|7508|5|600| +2450822|7510|5|627| +2450822|7513|5|277| +2450822|7514|5|186| +2450822|7516|5|751| +2450822|7519|5|765| +2450822|7520|5|836| +2450822|7522|5|79| +2450822|7525|5|167| +2450822|7526|5|358| +2450822|7528|5|150| +2450822|7531|5|919| +2450822|7532|5|541| +2450822|7534|5|691| +2450822|7537|5|125| +2450822|7538|5|451| +2450822|7540|5|306| +2450822|7543|5|61| +2450822|7544|5|837| +2450822|7546|5|465| +2450822|7549|5|855| +2450822|7550|5|613| +2450822|7552|5|436| +2450822|7555|5|716| +2450822|7556|5|378| +2450822|7558|5|293| +2450822|7561|5|70| +2450822|7562|5|566| +2450822|7564|5|966| +2450822|7567|5|376| +2450822|7568|5|870| +2450822|7570|5|208| +2450822|7573|5|582| +2450822|7574|5|786| +2450822|7576|5|308| +2450822|7579|5|99| +2450822|7580|5|917| +2450822|7582|5|370| +2450822|7585|5|125| +2450822|7586|5|584| +2450822|7588|5|194| +2450822|7591|5|909| +2450822|7592|5|| +2450822|7594|5|214| +2450822|7597|5|278| +2450822|7598|5|754| +2450822|7600|5|422| +2450822|7603|5|266| +2450822|7604|5|308| +2450822|7606|5|410| +2450822|7609|5|450| +2450822|7610|5|639| +2450822|7612|5|925| +2450822|7615|5|175| +2450822|7616|5|| +2450822|7618|5|| +2450822|7621|5|200| +2450822|7622|5|418| +2450822|7624|5|938| +2450822|7627|5|262| +2450822|7628|5|516| +2450822|7630|5|571| +2450822|7633|5|92| +2450822|7634|5|266| +2450822|7636|5|598| +2450822|7639|5|166| +2450822|7640|5|779| +2450822|7642|5|847| +2450822|7645|5|775| +2450822|7646|5|952| +2450822|7648|5|359| +2450822|7651|5|552| +2450822|7652|5|897| +2450822|7654|5|37| +2450822|7657|5|107| +2450822|7658|5|101| +2450822|7660|5|1| +2450822|7663|5|336| +2450822|7664|5|787| +2450822|7666|5|859| +2450822|7669|5|507| +2450822|7670|5|18| +2450822|7672|5|| +2450822|7675|5|82| +2450822|7676|5|993| +2450822|7678|5|410| +2450822|7681|5|851| +2450822|7682|5|373| +2450822|7684|5|| +2450822|7687|5|26| +2450822|7688|5|685| +2450822|7690|5|827| +2450822|7693|5|805| +2450822|7694|5|985| +2450822|7696|5|515| +2450822|7699|5|648| +2450822|7700|5|948| +2450822|7702|5|946| +2450822|7705|5|716| +2450822|7706|5|171| +2450822|7708|5|873| +2450822|7711|5|974| +2450822|7712|5|570| +2450822|7714|5|298| +2450822|7717|5|442| +2450822|7718|5|265| +2450822|7720|5|351| +2450822|7723|5|655| +2450822|7724|5|587| +2450822|7726|5|798| +2450822|7729|5|648| +2450822|7730|5|616| +2450822|7732|5|254| +2450822|7735|5|471| +2450822|7736|5|188| +2450822|7738|5|189| +2450822|7741|5|887| +2450822|7742|5|952| +2450822|7744|5|37| +2450822|7747|5|945| +2450822|7748|5|930| +2450822|7750|5|217| +2450822|7753|5|747| +2450822|7754|5|321| +2450822|7756|5|147| +2450822|7759|5|364| +2450822|7760|5|998| +2450822|7762|5|984| +2450822|7765|5|607| +2450822|7766|5|596| +2450822|7768|5|631| +2450822|7771|5|268| +2450822|7772|5|978| +2450822|7774|5|847| +2450822|7777|5|433| +2450822|7778|5|68| +2450822|7780|5|338| +2450822|7783|5|545| +2450822|7784|5|530| +2450822|7786|5|14| +2450822|7789|5|258| +2450822|7790|5|420| +2450822|7792|5|312| +2450822|7795|5|6| +2450822|7796|5|329| +2450822|7798|5|968| +2450822|7801|5|699| +2450822|7802|5|798| +2450822|7804|5|920| +2450822|7807|5|467| +2450822|7808|5|183| +2450822|7810|5|779| +2450822|7813|5|262| +2450822|7814|5|865| +2450822|7816|5|895| +2450822|7819|5|355| +2450822|7820|5|893| +2450822|7822|5|479| +2450822|7825|5|570| +2450822|7826|5|70| +2450822|7828|5|942| +2450822|7831|5|972| +2450822|7832|5|815| +2450822|7834|5|294| +2450822|7837|5|291| +2450822|7838|5|275| +2450822|7840|5|3| +2450822|7843|5|205| +2450822|7844|5|653| +2450822|7846|5|939| +2450822|7849|5|979| +2450822|7850|5|233| +2450822|7852|5|579| +2450822|7855|5|259| +2450822|7856|5|734| +2450822|7858|5|794| +2450822|7861|5|700| +2450822|7862|5|| +2450822|7864|5|270| +2450822|7867|5|568| +2450822|7868|5|710| +2450822|7870|5|611| +2450822|7873|5|640| +2450822|7874|5|271| +2450822|7876|5|| +2450822|7879|5|102| +2450822|7880|5|320| +2450822|7882|5|183| +2450822|7885|5|156| +2450822|7886|5|783| +2450822|7888|5|995| +2450822|7891|5|| +2450822|7892|5|972| +2450822|7894|5|220| +2450822|7897|5|49| +2450822|7898|5|410| +2450822|7900|5|830| +2450822|7903|5|707| +2450822|7904|5|829| +2450822|7906|5|86| +2450822|7909|5|| +2450822|7910|5|| +2450822|7912|5|205| +2450822|7915|5|92| +2450822|7916|5|352| +2450822|7918|5|1000| +2450822|7921|5|700| +2450822|7922|5|328| +2450822|7924|5|393| +2450822|7927|5|335| +2450822|7928|5|639| +2450822|7930|5|332| +2450822|7933|5|141| +2450822|7934|5|234| +2450822|7936|5|684| +2450822|7939|5|681| +2450822|7940|5|266| +2450822|7942|5|329| +2450822|7945|5|242| +2450822|7946|5|98| +2450822|7948|5|203| +2450822|7951|5|764| +2450822|7952|5|15| +2450822|7954|5|828| +2450822|7957|5|104| +2450822|7958|5|577| +2450822|7960|5|775| +2450822|7963|5|499| +2450822|7964|5|3| +2450822|7966|5|874| +2450822|7969|5|828| +2450822|7970|5|147| +2450822|7972|5|438| +2450822|7975|5|681| +2450822|7976|5|87| +2450822|7978|5|| +2450822|7981|5|756| +2450822|7982|5|196| +2450822|7984|5|163| +2450822|7987|5|710| +2450822|7988|5|4| +2450822|7990|5|7| +2450822|7993|5|346| +2450822|7994|5|354| +2450822|7996|5|347| +2450822|7999|5|| +2450822|8000|5|576| +2450822|8002|5|| +2450822|8005|5|| +2450822|8006|5|640| +2450822|8008|5|247| +2450822|8011|5|921| +2450822|8012|5|792| +2450822|8014|5|341| +2450822|8017|5|536| +2450822|8018|5|360| +2450822|8020|5|129| +2450822|8023|5|165| +2450822|8024|5|654| +2450822|8026|5|714| +2450822|8029|5|933| +2450822|8030|5|864| +2450822|8032|5|599| +2450822|8035|5|428| +2450822|8036|5|92| +2450822|8038|5|5| +2450822|8041|5|97| +2450822|8042|5|126| +2450822|8044|5|642| +2450822|8047|5|762| +2450822|8048|5|464| +2450822|8050|5|209| +2450822|8053|5|242| +2450822|8054|5|232| +2450822|8056|5|905| +2450822|8059|5|159| +2450822|8060|5|318| +2450822|8062|5|227| +2450822|8065|5|842| +2450822|8066|5|333| +2450822|8068|5|127| +2450822|8071|5|101| +2450822|8072|5|910| +2450822|8074|5|4| +2450822|8077|5|530| +2450822|8078|5|932| +2450822|8080|5|| +2450822|8083|5|546| +2450822|8084|5|342| +2450822|8086|5|117| +2450822|8089|5|518| +2450822|8090|5|523| +2450822|8092|5|316| +2450822|8095|5|385| +2450822|8096|5|441| +2450822|8098|5|43| +2450822|8101|5|126| +2450822|8102|5|193| +2450822|8104|5|572| +2450822|8107|5|982| +2450822|8108|5|269| +2450822|8110|5|858| +2450822|8113|5|469| +2450822|8114|5|968| +2450822|8116|5|584| +2450822|8119|5|497| +2450822|8120|5|488| +2450822|8122|5|670| +2450822|8125|5|799| +2450822|8126|5|565| +2450822|8128|5|135| +2450822|8131|5|230| +2450822|8132|5|660| +2450822|8134|5|995| +2450822|8137|5|682| +2450822|8138|5|211| +2450822|8140|5|250| +2450822|8143|5|804| +2450822|8144|5|383| +2450822|8146|5|246| +2450822|8149|5|982| +2450822|8150|5|158| +2450822|8152|5|394| +2450822|8155|5|620| +2450822|8156|5|202| +2450822|8158|5|904| +2450822|8161|5|964| +2450822|8162|5|485| +2450822|8164|5|480| +2450822|8167|5|204| +2450822|8168|5|489| +2450822|8170|5|| +2450822|8173|5|532| +2450822|8174|5|173| +2450822|8176|5|715| +2450822|8179|5|956| +2450822|8180|5|411| +2450822|8182|5|707| +2450822|8185|5|928| +2450822|8186|5|225| +2450822|8188|5|534| +2450822|8191|5|517| +2450822|8192|5|895| +2450822|8194|5|484| +2450822|8197|5|998| +2450822|8198|5|94| +2450822|8200|5|648| +2450822|8203|5|853| +2450822|8204|5|244| +2450822|8206|5|477| +2450822|8209|5|188| +2450822|8210|5|399| +2450822|8212|5|699| +2450822|8215|5|189| +2450822|8216|5|693| +2450822|8218|5|| +2450822|8221|5|525| +2450822|8222|5|825| +2450822|8224|5|947| +2450822|8227|5|111| +2450822|8228|5|826| +2450822|8230|5|669| +2450822|8233|5|385| +2450822|8234|5|49| +2450822|8236|5|178| +2450822|8239|5|123| +2450822|8240|5|440| +2450822|8242|5|89| +2450822|8245|5|583| +2450822|8246|5|450| +2450822|8248|5|943| +2450822|8251|5|| +2450822|8252|5|232| +2450822|8254|5|764| +2450822|8257|5|455| +2450822|8258|5|748| +2450822|8260|5|700| +2450822|8263|5|911| +2450822|8264|5|448| +2450822|8266|5|343| +2450822|8269|5|918| +2450822|8270|5|| +2450822|8272|5|113| +2450822|8275|5|515| +2450822|8276|5|115| +2450822|8278|5|656| +2450822|8281|5|606| +2450822|8282|5|11| +2450822|8284|5|737| +2450822|8287|5|601| +2450822|8288|5|146| +2450822|8290|5|319| +2450822|8293|5|963| +2450822|8294|5|68| +2450822|8296|5|993| +2450822|8299|5|699| +2450822|8300|5|351| +2450822|8302|5|| +2450822|8305|5|924| +2450822|8306|5|694| +2450822|8308|5|895| +2450822|8311|5|374| +2450822|8312|5|492| +2450822|8314|5|882| +2450822|8317|5|231| +2450822|8318|5|803| +2450822|8320|5|784| +2450822|8323|5|298| +2450822|8324|5|542| +2450822|8326|5|953| +2450822|8329|5|322| +2450822|8330|5|40| +2450822|8332|5|382| +2450822|8335|5|908| +2450822|8336|5|213| +2450822|8338|5|336| +2450822|8341|5|267| +2450822|8342|5|84| +2450822|8344|5|| +2450822|8347|5|48| +2450822|8348|5|411| +2450822|8350|5|439| +2450822|8353|5|155| +2450822|8354|5|91| +2450822|8356|5|627| +2450822|8359|5|548| +2450822|8360|5|850| +2450822|8362|5|735| +2450822|8365|5|479| +2450822|8366|5|681| +2450822|8368|5|| +2450822|8371|5|323| +2450822|8372|5|491| +2450822|8374|5|530| +2450822|8377|5|| +2450822|8378|5|976| +2450822|8380|5|674| +2450822|8383|5|68| +2450822|8384|5|188| +2450822|8386|5|533| +2450822|8389|5|960| +2450822|8390|5|349| +2450822|8392|5|287| +2450822|8395|5|781| +2450822|8396|5|454| +2450822|8398|5|403| +2450822|8401|5|885| +2450822|8402|5|924| +2450822|8404|5|996| +2450822|8407|5|85| +2450822|8408|5|722| +2450822|8410|5|35| +2450822|8413|5|80| +2450822|8414|5|568| +2450822|8416|5|772| +2450822|8419|5|778| +2450822|8420|5|297| +2450822|8422|5|| +2450822|8425|5|| +2450822|8426|5|1000| +2450822|8428|5|540| +2450822|8431|5|437| +2450822|8432|5|666| +2450822|8434|5|741| +2450822|8437|5|344| +2450822|8438|5|716| +2450822|8440|5|208| +2450822|8443|5|606| +2450822|8444|5|3| +2450822|8446|5|857| +2450822|8449|5|158| +2450822|8450|5|5| +2450822|8452|5|82| +2450822|8455|5|195| +2450822|8456|5|714| +2450822|8458|5|740| +2450822|8461|5|660| +2450822|8462|5|766| +2450822|8464|5|224| +2450822|8467|5|| +2450822|8468|5|86| +2450822|8470|5|310| +2450822|8473|5|833| +2450822|8474|5|973| +2450822|8476|5|279| +2450822|8479|5|533| +2450822|8480|5|880| +2450822|8482|5|595| +2450822|8485|5|38| +2450822|8486|5|311| +2450822|8488|5|| +2450822|8491|5|292| +2450822|8492|5|| +2450822|8494|5|| +2450822|8497|5|415| +2450822|8498|5|1000| +2450822|8500|5|750| +2450822|8503|5|437| +2450822|8504|5|717| +2450822|8506|5|363| +2450822|8509|5|355| +2450822|8510|5|608| +2450822|8512|5|292| +2450822|8515|5|234| +2450822|8516|5|276| +2450822|8518|5|802| +2450822|8521|5|855| +2450822|8522|5|298| +2450822|8524|5|381| +2450822|8527|5|538| +2450822|8528|5|390| +2450822|8530|5|253| +2450822|8533|5|698| +2450822|8534|5|983| +2450822|8536|5|556| +2450822|8539|5|56| +2450822|8540|5|977| +2450822|8542|5|483| +2450822|8545|5|801| +2450822|8546|5|689| +2450822|8548|5|372| +2450822|8551|5|954| +2450822|8552|5|| +2450822|8554|5|801| +2450822|8557|5|878| +2450822|8558|5|54| +2450822|8560|5|594| +2450822|8563|5|847| +2450822|8564|5|329| +2450822|8566|5|972| +2450822|8569|5|670| +2450822|8570|5|151| +2450822|8572|5|196| +2450822|8575|5|678| +2450822|8576|5|984| +2450822|8578|5|582| +2450822|8581|5|| +2450822|8582|5|484| +2450822|8584|5|993| +2450822|8587|5|258| +2450822|8588|5|179| +2450822|8590|5|278| +2450822|8593|5|428| +2450822|8594|5|132| +2450822|8596|5|990| +2450822|8599|5|894| +2450822|8600|5|106| +2450822|8602|5|554| +2450822|8605|5|448| +2450822|8606|5|909| +2450822|8608|5|403| +2450822|8611|5|322| +2450822|8612|5|241| +2450822|8614|5|459| +2450822|8617|5|488| +2450822|8618|5|759| +2450822|8620|5|283| +2450822|8623|5|124| +2450822|8624|5|853| +2450822|8626|5|734| +2450822|8629|5|333| +2450822|8630|5|881| +2450822|8632|5|163| +2450822|8635|5|442| +2450822|8636|5|643| +2450822|8638|5|478| +2450822|8641|5|846| +2450822|8642|5|692| +2450822|8644|5|591| +2450822|8647|5|357| +2450822|8648|5|456| +2450822|8650|5|| +2450822|8653|5|156| +2450822|8654|5|644| +2450822|8656|5|509| +2450822|8659|5|476| +2450822|8660|5|| +2450822|8662|5|309| +2450822|8665|5|177| +2450822|8666|5|127| +2450822|8668|5|255| +2450822|8671|5|390| +2450822|8672|5|248| +2450822|8674|5|740| +2450822|8677|5|422| +2450822|8678|5|173| +2450822|8680|5|207| +2450822|8683|5|855| +2450822|8684|5|943| +2450822|8686|5|| +2450822|8689|5|94| +2450822|8690|5|559| +2450822|8692|5|316| +2450822|8695|5|954| +2450822|8696|5|195| +2450822|8698|5|795| +2450822|8701|5|763| +2450822|8702|5|439| +2450822|8704|5|487| +2450822|8707|5|268| +2450822|8708|5|330| +2450822|8710|5|221| +2450822|8713|5|166| +2450822|8714|5|422| +2450822|8716|5|189| +2450822|8719|5|569| +2450822|8720|5|85| +2450822|8722|5|92| +2450822|8725|5|728| +2450822|8726|5|243| +2450822|8728|5|947| +2450822|8731|5|36| +2450822|8732|5|636| +2450822|8734|5|684| +2450822|8737|5|158| +2450822|8738|5|177| +2450822|8740|5|551| +2450822|8743|5|931| +2450822|8744|5|766| +2450822|8746|5|288| +2450822|8749|5|329| +2450822|8750|5|328| +2450822|8752|5|130| +2450822|8755|5|869| +2450822|8756|5|403| +2450822|8758|5|289| +2450822|8761|5|237| +2450822|8762|5|912| +2450822|8764|5|269| +2450822|8767|5|| +2450822|8768|5|676| +2450822|8770|5|178| +2450822|8773|5|887| +2450822|8774|5|500| +2450822|8776|5|542| +2450822|8779|5|774| +2450822|8780|5|771| +2450822|8782|5|637| +2450822|8785|5|479| +2450822|8786|5|979| +2450822|8788|5|205| +2450822|8791|5|948| +2450822|8792|5|913| +2450822|8794|5|271| +2450822|8797|5|936| +2450822|8798|5|515| +2450822|8800|5|25| +2450822|8803|5|970| +2450822|8804|5|830| +2450822|8806|5|756| +2450822|8809|5|889| +2450822|8810|5|274| +2450822|8812|5|558| +2450822|8815|5|337| +2450822|8816|5|812| +2450822|8818|5|474| +2450822|8821|5|512| +2450822|8822|5|951| +2450822|8824|5|310| +2450822|8827|5|693| +2450822|8828|5|446| +2450822|8830|5|3| +2450822|8833|5|352| +2450822|8834|5|64| +2450822|8836|5|414| +2450822|8839|5|103| +2450822|8840|5|700| +2450822|8842|5|899| +2450822|8845|5|889| +2450822|8846|5|415| +2450822|8848|5|933| +2450822|8851|5|197| +2450822|8852|5|900| +2450822|8854|5|435| +2450822|8857|5|68| +2450822|8858|5|763| +2450822|8860|5|376| +2450822|8863|5|222| +2450822|8864|5|539| +2450822|8866|5|597| +2450822|8869|5|237| +2450822|8870|5|587| +2450822|8872|5|377| +2450822|8875|5|390| +2450822|8876|5|156| +2450822|8878|5|820| +2450822|8881|5|292| +2450822|8882|5|159| +2450822|8884|5|151| +2450822|8887|5|164| +2450822|8888|5|404| +2450822|8890|5|354| +2450822|8893|5|681| +2450822|8894|5|48| +2450822|8896|5|243| +2450822|8899|5|| +2450822|8900|5|986| +2450822|8902|5|879| +2450822|8905|5|977| +2450822|8906|5|58| +2450822|8908|5|377| +2450822|8911|5|437| +2450822|8912|5|634| +2450822|8914|5|564| +2450822|8917|5|398| +2450822|8918|5|837| +2450822|8920|5|634| +2450822|8923|5|952| +2450822|8924|5|497| +2450822|8926|5|501| +2450822|8929|5|75| +2450822|8930|5|645| +2450822|8932|5|915| +2450822|8935|5|334| +2450822|8936|5|437| +2450822|8938|5|406| +2450822|8941|5|915| +2450822|8942|5|257| +2450822|8944|5|636| +2450822|8947|5|869| +2450822|8948|5|840| +2450822|8950|5|236| +2450822|8953|5|859| +2450822|8954|5|828| +2450822|8956|5|779| +2450822|8959|5|902| +2450822|8960|5|338| +2450822|8962|5|685| +2450822|8965|5|918| +2450822|8966|5|131| +2450822|8968|5|497| +2450822|8971|5|48| +2450822|8972|5|386| +2450822|8974|5|432| +2450822|8977|5|742| +2450822|8978|5|357| +2450822|8980|5|757| +2450822|8983|5|38| +2450822|8984|5|745| +2450822|8986|5|472| +2450822|8989|5|504| +2450822|8990|5|217| +2450822|8992|5|938| +2450822|8995|5|638| +2450822|8996|5|841| +2450822|8998|5|951| +2450822|9001|5|28| +2450822|9002|5|823| +2450822|9004|5|413| +2450822|9007|5|791| +2450822|9008|5|599| +2450822|9010|5|511| +2450822|9013|5|827| +2450822|9014|5|| +2450822|9016|5|176| +2450822|9019|5|756| +2450822|9020|5|335| +2450822|9022|5|398| +2450822|9025|5|880| +2450822|9026|5|475| +2450822|9028|5|415| +2450822|9031|5|760| +2450822|9032|5|655| +2450822|9034|5|785| +2450822|9037|5|| +2450822|9038|5|558| +2450822|9040|5|966| +2450822|9043|5|810| +2450822|9044|5|754| +2450822|9046|5|120| +2450822|9049|5|343| +2450822|9050|5|| +2450822|9052|5|368| +2450822|9055|5|845| +2450822|9056|5|448| +2450822|9058|5|762| +2450822|9061|5|812| +2450822|9062|5|634| +2450822|9064|5|764| +2450822|9067|5|515| +2450822|9068|5|120| +2450822|9070|5|455| +2450822|9073|5|536| +2450822|9074|5|182| +2450822|9076|5|978| +2450822|9079|5|32| +2450822|9080|5|545| +2450822|9082|5|281| +2450822|9085|5|650| +2450822|9086|5|437| +2450822|9088|5|347| +2450822|9091|5|0| +2450822|9092|5|875| +2450822|9094|5|728| +2450822|9097|5|| +2450822|9098|5|509| +2450822|9100|5|926| +2450822|9103|5|229| +2450822|9104|5|686| +2450822|9106|5|630| +2450822|9109|5|482| +2450822|9110|5|145| +2450822|9112|5|504| +2450822|9115|5|280| +2450822|9116|5|988| +2450822|9118|5|181| +2450822|9121|5|442| +2450822|9122|5|319| +2450822|9124|5|581| +2450822|9127|5|686| +2450822|9128|5|857| +2450822|9130|5|194| +2450822|9133|5|820| +2450822|9134|5|140| +2450822|9136|5|733| +2450822|9139|5|| +2450822|9140|5|970| +2450822|9142|5|426| +2450822|9145|5|959| +2450822|9146|5|78| +2450822|9148|5|968| +2450822|9151|5|122| +2450822|9152|5|629| +2450822|9154|5|198| +2450822|9157|5|323| +2450822|9158|5|742| +2450822|9160|5|487| +2450822|9163|5|381| +2450822|9164|5|609| +2450822|9166|5|496| +2450822|9169|5|17| +2450822|9170|5|351| +2450822|9172|5|965| +2450822|9175|5|426| +2450822|9176|5|935| +2450822|9178|5|989| +2450822|9181|5|736| +2450822|9182|5|900| +2450822|9184|5|150| +2450822|9187|5|373| +2450822|9188|5|899| +2450822|9190|5|380| +2450822|9193|5|212| +2450822|9194|5|877| +2450822|9196|5|114| +2450822|9199|5|788| +2450822|9200|5|404| +2450822|9202|5|359| +2450822|9205|5|825| +2450822|9206|5|| +2450822|9208|5|| +2450822|9211|5|595| +2450822|9212|5|619| +2450822|9214|5|219| +2450822|9217|5|674| +2450822|9218|5|718| +2450822|9220|5|502| +2450822|9223|5|742| +2450822|9224|5|583| +2450822|9226|5|149| +2450822|9229|5|654| +2450822|9230|5|863| +2450822|9232|5|173| +2450822|9235|5|958| +2450822|9236|5|753| +2450822|9238|5|578| +2450822|9241|5|774| +2450822|9242|5|| +2450822|9244|5|799| +2450822|9247|5|142| +2450822|9248|5|236| +2450822|9250|5|928| +2450822|9253|5|384| +2450822|9254|5|239| +2450822|9256|5|914| +2450822|9259|5|743| +2450822|9260|5|716| +2450822|9262|5|294| +2450822|9265|5|638| +2450822|9266|5|56| +2450822|9268|5|498| +2450822|9271|5|113| +2450822|9272|5|749| +2450822|9274|5|851| +2450822|9277|5|| +2450822|9278|5|| +2450822|9280|5|110| +2450822|9283|5|506| +2450822|9284|5|945| +2450822|9286|5|870| +2450822|9289|5|301| +2450822|9290|5|485| +2450822|9292|5|21| +2450822|9295|5|32| +2450822|9296|5|872| +2450822|9298|5|65| +2450822|9301|5|852| +2450822|9302|5|169| +2450822|9304|5|776| +2450822|9307|5|126| +2450822|9308|5|898| +2450822|9310|5|937| +2450822|9313|5|837| +2450822|9314|5|730| +2450822|9316|5|432| +2450822|9319|5|509| +2450822|9320|5|303| +2450822|9322|5|126| +2450822|9325|5|151| +2450822|9326|5|235| +2450822|9328|5|456| +2450822|9331|5|910| +2450822|9332|5|61| +2450822|9334|5|641| +2450822|9337|5|897| +2450822|9338|5|77| +2450822|9340|5|833| +2450822|9343|5|0| +2450822|9344|5|| +2450822|9346|5|385| +2450822|9349|5|885| +2450822|9350|5|692| +2450822|9352|5|824| +2450822|9355|5|921| +2450822|9356|5|872| +2450822|9358|5|13| +2450822|9361|5|341| +2450822|9362|5|707| +2450822|9364|5|697| +2450822|9367|5|646| +2450822|9368|5|315| +2450822|9370|5|900| +2450822|9373|5|499| +2450822|9374|5|939| +2450822|9376|5|70| +2450822|9379|5|686| +2450822|9380|5|484| +2450822|9382|5|490| +2450822|9385|5|811| +2450822|9386|5|884| +2450822|9388|5|| +2450822|9391|5|190| +2450822|9392|5|51| +2450822|9394|5|536| +2450822|9397|5|71| +2450822|9398|5|184| +2450822|9400|5|500| +2450822|9403|5|681| +2450822|9404|5|527| +2450822|9406|5|656| +2450822|9409|5|682| +2450822|9410|5|255| +2450822|9412|5|| +2450822|9415|5|124| +2450822|9416|5|46| +2450822|9418|5|631| +2450822|9421|5|| +2450822|9422|5|299| +2450822|9424|5|125| +2450822|9427|5|737| +2450822|9428|5|957| +2450822|9430|5|| +2450822|9433|5|479| +2450822|9434|5|839| +2450822|9436|5|962| +2450822|9439|5|518| +2450822|9440|5|384| +2450822|9442|5|753| +2450822|9445|5|160| +2450822|9446|5|956| +2450822|9448|5|744| +2450822|9451|5|182| +2450822|9452|5|132| +2450822|9454|5|| +2450822|9457|5|294| +2450822|9458|5|530| +2450822|9460|5|236| +2450822|9463|5|962| +2450822|9464|5|314| +2450822|9466|5|682| +2450822|9469|5|115| +2450822|9470|5|586| +2450822|9472|5|956| +2450822|9475|5|623| +2450822|9476|5|39| +2450822|9478|5|466| +2450822|9481|5|996| +2450822|9482|5|36| +2450822|9484|5|745| +2450822|9487|5|931| +2450822|9488|5|955| +2450822|9490|5|846| +2450822|9493|5|71| +2450822|9494|5|875| +2450822|9496|5|688| +2450822|9499|5|655| +2450822|9500|5|44| +2450822|9502|5|301| +2450822|9505|5|862| +2450822|9506|5|760| +2450822|9508|5|689| +2450822|9511|5|328| +2450822|9512|5|361| +2450822|9514|5|545| +2450822|9517|5|39| +2450822|9518|5|401| +2450822|9520|5|131| +2450822|9523|5|743| +2450822|9524|5|526| +2450822|9526|5|271| +2450822|9529|5|877| +2450822|9530|5|599| +2450822|9532|5|28| +2450822|9535|5|214| +2450822|9536|5|791| +2450822|9538|5|854| +2450822|9541|5|115| +2450822|9542|5|594| +2450822|9544|5|667| +2450822|9547|5|918| +2450822|9548|5|22| +2450822|9550|5|764| +2450822|9553|5|808| +2450822|9554|5|647| +2450822|9556|5|294| +2450822|9559|5|556| +2450822|9560|5|873| +2450822|9562|5|| +2450822|9565|5|| +2450822|9566|5|| +2450822|9568|5|328| +2450822|9571|5|21| +2450822|9572|5|376| +2450822|9574|5|831| +2450822|9577|5|765| +2450822|9578|5|990| +2450822|9580|5|957| +2450822|9583|5|850| +2450822|9584|5|681| +2450822|9586|5|718| +2450822|9589|5|663| +2450822|9590|5|337| +2450822|9592|5|766| +2450822|9595|5|22| +2450822|9596|5|894| +2450822|9598|5|943| +2450822|9601|5|717| +2450822|9602|5|609| +2450822|9604|5|347| +2450822|9607|5|598| +2450822|9608|5|439| +2450822|9610|5|357| +2450822|9613|5|685| +2450822|9614|5|36| +2450822|9616|5|931| +2450822|9619|5|| +2450822|9620|5|343| +2450822|9622|5|604| +2450822|9625|5|814| +2450822|9626|5|126| +2450822|9628|5|720| +2450822|9631|5|203| +2450822|9632|5|483| +2450822|9634|5|552| +2450822|9637|5|254| +2450822|9638|5|447| +2450822|9640|5|976| +2450822|9643|5|237| +2450822|9644|5|80| +2450822|9646|5|567| +2450822|9649|5|505| +2450822|9650|5|878| +2450822|9652|5|716| +2450822|9655|5|282| +2450822|9656|5|456| +2450822|9658|5|374| +2450822|9661|5|988| +2450822|9662|5|846| +2450822|9664|5|107| +2450822|9667|5|147| +2450822|9668|5|870| +2450822|9670|5|119| +2450822|9673|5|879| +2450822|9674|5|232| +2450822|9676|5|549| +2450822|9679|5|445| +2450822|9680|5|448| +2450822|9682|5|912| +2450822|9685|5|979| +2450822|9686|5|479| +2450822|9688|5|470| +2450822|9691|5|891| +2450822|9692|5|249| +2450822|9694|5|483| +2450822|9697|5|750| +2450822|9698|5|21| +2450822|9700|5|420| +2450822|9703|5|430| +2450822|9704|5|886| +2450822|9706|5|| +2450822|9709|5|780| +2450822|9710|5|372| +2450822|9712|5|739| +2450822|9715|5|| +2450822|9716|5|181| +2450822|9718|5|233| +2450822|9721|5|703| +2450822|9722|5|414| +2450822|9724|5|| +2450822|9727|5|538| +2450822|9728|5|785| +2450822|9730|5|572| +2450822|9733|5|996| +2450822|9734|5|846| +2450822|9736|5|447| +2450822|9739|5|797| +2450822|9740|5|217| +2450822|9742|5|542| +2450822|9745|5|755| +2450822|9746|5|620| +2450822|9748|5|133| +2450822|9751|5|662| +2450822|9752|5|392| +2450822|9754|5|194| +2450822|9757|5|738| +2450822|9758|5|452| +2450822|9760|5|586| +2450822|9763|5|885| +2450822|9764|5|628| +2450822|9766|5|823| +2450822|9769|5|420| +2450822|9770|5|280| +2450822|9772|5|180| +2450822|9775|5|331| +2450822|9776|5|488| +2450822|9778|5|502| +2450822|9781|5|232| +2450822|9782|5|383| +2450822|9784|5|716| +2450822|9787|5|916| +2450822|9788|5|610| +2450822|9790|5|922| +2450822|9793|5|764| +2450822|9794|5|720| +2450822|9796|5|| +2450822|9799|5|768| +2450822|9800|5|918| +2450822|9802|5|56| +2450822|9805|5|677| +2450822|9806|5|923| +2450822|9808|5|538| +2450822|9811|5|885| +2450822|9812|5|845| +2450822|9814|5|112| +2450822|9817|5|997| +2450822|9818|5|309| +2450822|9820|5|854| +2450822|9823|5|879| +2450822|9824|5|945| +2450822|9826|5|853| +2450822|9829|5|419| +2450822|9830|5|191| +2450822|9832|5|652| +2450822|9835|5|974| +2450822|9836|5|195| +2450822|9838|5|497| +2450822|9841|5|96| +2450822|9842|5|442| +2450822|9844|5|265| +2450822|9847|5|95| +2450822|9848|5|740| +2450822|9850|5|181| +2450822|9853|5|473| +2450822|9854|5|977| +2450822|9856|5|545| +2450822|9859|5|551| +2450822|9860|5|463| +2450822|9862|5|127| +2450822|9865|5|173| +2450822|9866|5|239| +2450822|9868|5|782| +2450822|9871|5|987| +2450822|9872|5|806| +2450822|9874|5|149| +2450822|9877|5|87| +2450822|9878|5|2| +2450822|9880|5|133| +2450822|9883|5|351| +2450822|9884|5|528| +2450822|9886|5|305| +2450822|9889|5|373| +2450822|9890|5|222| +2450822|9892|5|826| +2450822|9895|5|640| +2450822|9896|5|895| +2450822|9898|5|531| +2450822|9901|5|787| +2450822|9902|5|442| +2450822|9904|5|689| +2450822|9907|5|644| +2450822|9908|5|| +2450822|9910|5|158| +2450822|9913|5|644| +2450822|9914|5|975| +2450822|9916|5|196| +2450822|9919|5|665| +2450822|9920|5|839| +2450822|9922|5|642| +2450822|9925|5|837| +2450822|9926|5|11| +2450822|9928|5|| +2450822|9931|5|199| +2450822|9932|5|518| +2450822|9934|5|179| +2450822|9937|5|628| +2450822|9938|5|12| +2450822|9940|5|778| +2450822|9943|5|684| +2450822|9944|5|597| +2450822|9946|5|680| +2450822|9949|5|72| +2450822|9950|5|346| +2450822|9952|5|96| +2450822|9955|5|| +2450822|9956|5|551| +2450822|9958|5|836| +2450822|9961|5|936| +2450822|9962|5|956| +2450822|9964|5|611| +2450822|9967|5|358| +2450822|9968|5|137| +2450822|9970|5|545| +2450822|9973|5|691| +2450822|9974|5|717| +2450822|9976|5|207| +2450822|9979|5|640| +2450822|9980|5|179| +2450822|9982|5|818| +2450822|9985|5|549| +2450822|9986|5|94| +2450822|9988|5|70| +2450822|9991|5|449| +2450822|9992|5|646| +2450822|9994|5|777| +2450822|9997|5|446| +2450822|9998|5|231| +2450822|10000|5|723| +2450822|10003|5|582| +2450822|10004|5|833| +2450822|10006|5|994| +2450822|10009|5|994| +2450822|10010|5|355| +2450822|10012|5|287| +2450822|10015|5|950| +2450822|10016|5|388| +2450822|10018|5|3| +2450822|10021|5|776| +2450822|10022|5|136| +2450822|10024|5|643| +2450822|10027|5|216| +2450822|10028|5|944| +2450822|10030|5|352| +2450822|10033|5|853| +2450822|10034|5|724| +2450822|10036|5|338| +2450822|10039|5|289| +2450822|10040|5|801| +2450822|10042|5|95| +2450822|10045|5|| +2450822|10046|5|532| +2450822|10048|5|110| +2450822|10051|5|451| +2450822|10052|5|882| +2450822|10054|5|102| +2450822|10057|5|801| +2450822|10058|5|455| +2450822|10060|5|701| +2450822|10063|5|40| +2450822|10064|5|605| +2450822|10066|5|52| +2450822|10069|5|896| +2450822|10070|5|425| +2450822|10072|5|751| +2450822|10075|5|122| +2450822|10076|5|997| +2450822|10078|5|29| +2450822|10081|5|783| +2450822|10082|5|184| +2450822|10084|5|280| +2450822|10087|5|287| +2450822|10088|5|135| +2450822|10090|5|709| +2450822|10093|5|400| +2450822|10094|5|624| +2450822|10096|5|771| +2450822|10099|5|| +2450822|10100|5|769| +2450822|10102|5|84| +2450822|10105|5|313| +2450822|10106|5|255| +2450822|10108|5|999| +2450822|10111|5|364| +2450822|10112|5|533| +2450822|10114|5|| +2450822|10117|5|666| +2450822|10118|5|381| +2450822|10120|5|271| +2450822|10123|5|357| +2450822|10124|5|16| +2450822|10126|5|799| +2450822|10129|5|83| +2450822|10130|5|744| +2450822|10132|5|737| +2450822|10135|5|596| +2450822|10136|5|350| +2450822|10138|5|445| +2450822|10141|5|157| +2450822|10142|5|965| +2450822|10144|5|750| +2450822|10147|5|230| +2450822|10148|5|57| +2450822|10150|5|100| +2450822|10153|5|207| +2450822|10154|5|111| +2450822|10156|5|250| +2450822|10159|5|802| +2450822|10160|5|589| +2450822|10162|5|344| +2450822|10165|5|118| +2450822|10166|5|688| +2450822|10168|5|131| +2450822|10171|5|206| +2450822|10172|5|2| +2450822|10174|5|253| +2450822|10177|5|140| +2450822|10178|5|| +2450822|10180|5|681| +2450822|10183|5|53| +2450822|10184|5|658| +2450822|10186|5|452| +2450822|10189|5|789| +2450822|10190|5|516| +2450822|10192|5|22| +2450822|10195|5|969| +2450822|10196|5|920| +2450822|10198|5|| +2450822|10201|5|| +2450822|10202|5|874| +2450822|10204|5|284| +2450822|10207|5|1| +2450822|10208|5|202| +2450822|10210|5|384| +2450822|10213|5|83| +2450822|10214|5|482| +2450822|10216|5|710| +2450822|10219|5|667| +2450822|10220|5|97| +2450822|10222|5|841| +2450822|10225|5|40| +2450822|10226|5|334| +2450822|10228|5|849| +2450822|10231|5|265| +2450822|10232|5|659| +2450822|10234|5|255| +2450822|10237|5|747| +2450822|10238|5|478| +2450822|10240|5|342| +2450822|10243|5|232| +2450822|10244|5|69| +2450822|10246|5|710| +2450822|10249|5|874| +2450822|10250|5|491| +2450822|10252|5|809| +2450822|10255|5|60| +2450822|10256|5|906| +2450822|10258|5|510| +2450822|10261|5|398| +2450822|10262|5|535| +2450822|10264|5|246| +2450822|10267|5|503| +2450822|10268|5|296| +2450822|10270|5|637| +2450822|10273|5|502| +2450822|10274|5|313| +2450822|10276|5|47| +2450822|10279|5|196| +2450822|10280|5|322| +2450822|10282|5|419| +2450822|10285|5|933| +2450822|10286|5|34| +2450822|10288|5|308| +2450822|10291|5|238| +2450822|10292|5|771| +2450822|10294|5|488| +2450822|10297|5|489| +2450822|10298|5|27| +2450822|10300|5|122| +2450822|10303|5|| +2450822|10304|5|27| +2450822|10306|5|750| +2450822|10309|5|631| +2450822|10310|5|833| +2450822|10312|5|782| +2450822|10315|5|818| +2450822|10316|5|402| +2450822|10318|5|716| +2450822|10321|5|390| +2450822|10322|5|246| +2450822|10324|5|561| +2450822|10327|5|527| +2450822|10328|5|684| +2450822|10330|5|529| +2450822|10333|5|725| +2450822|10334|5|729| +2450822|10336|5|471| +2450822|10339|5|571| +2450822|10340|5|378| +2450822|10342|5|499| +2450822|10345|5|87| +2450822|10346|5|733| +2450822|10348|5|999| +2450822|10351|5|716| +2450822|10352|5|829| +2450822|10354|5|840| +2450822|10357|5|| +2450822|10358|5|66| +2450822|10360|5|646| +2450822|10363|5|820| +2450822|10364|5|274| +2450822|10366|5|743| +2450822|10369|5|507| +2450822|10370|5|493| +2450822|10372|5|92| +2450822|10375|5|91| +2450822|10376|5|806| +2450822|10378|5|509| +2450822|10381|5|601| +2450822|10382|5|757| +2450822|10384|5|910| +2450822|10387|5|167| +2450822|10388|5|738| +2450822|10390|5|908| +2450822|10393|5|343| +2450822|10394|5|818| +2450822|10396|5|551| +2450822|10399|5|849| +2450822|10400|5|468| +2450822|10402|5|57| +2450822|10405|5|326| +2450822|10406|5|769| +2450822|10408|5|34| +2450822|10411|5|476| +2450822|10412|5|738| +2450822|10414|5|788| +2450822|10417|5|936| +2450822|10418|5|| +2450822|10420|5|707| +2450822|10423|5|902| +2450822|10424|5|625| +2450822|10426|5|343| +2450822|10429|5|426| +2450822|10430|5|| +2450822|10432|5|| +2450822|10435|5|147| +2450822|10436|5|469| +2450822|10438|5|66| +2450822|10441|5|96| +2450822|10442|5|655| +2450822|10444|5|370| +2450822|10447|5|964| +2450822|10448|5|175| +2450822|10450|5|15| +2450822|10453|5|304| +2450822|10454|5|259| +2450822|10456|5|| +2450822|10459|5|365| +2450822|10460|5|110| +2450822|10462|5|609| +2450822|10465|5|892| +2450822|10466|5|602| +2450822|10468|5|894| +2450822|10471|5|676| +2450822|10472|5|25| +2450822|10474|5|| +2450822|10477|5|643| +2450822|10478|5|| +2450822|10480|5|636| +2450822|10483|5|216| +2450822|10484|5|899| +2450822|10486|5|85| +2450822|10489|5|452| +2450822|10490|5|296| +2450822|10492|5|334| +2450822|10495|5|| +2450822|10496|5|883| +2450822|10498|5|46| +2450822|10501|5|937| +2450822|10502|5|922| +2450822|10504|5|750| +2450822|10507|5|701| +2450822|10508|5|737| +2450822|10510|5|313| +2450822|10513|5|109| +2450822|10514|5|117| +2450822|10516|5|27| +2450822|10519|5|661| +2450822|10520|5|810| +2450822|10522|5|225| +2450822|10525|5|122| +2450822|10526|5|985| +2450822|10528|5|282| +2450822|10531|5|345| +2450822|10532|5|922| +2450822|10534|5|753| +2450822|10537|5|443| +2450822|10538|5|618| +2450822|10540|5|673| +2450822|10543|5|81| +2450822|10544|5|625| +2450822|10546|5|919| +2450822|10549|5|936| +2450822|10550|5|868| +2450822|10552|5|792| +2450822|10555|5|60| +2450822|10556|5|653| +2450822|10558|5|762| +2450822|10561|5|766| +2450822|10562|5|53| +2450822|10564|5|583| +2450822|10567|5|204| +2450822|10568|5|43| +2450822|10570|5|480| +2450822|10573|5|800| +2450822|10574|5|227| +2450822|10576|5|135| +2450822|10579|5|779| +2450822|10580|5|17| +2450822|10582|5|181| +2450822|10585|5|56| +2450822|10586|5|496| +2450822|10588|5|944| +2450822|10591|5|966| +2450822|10592|5|970| +2450822|10594|5|591| +2450822|10597|5|470| +2450822|10598|5|875| +2450822|10600|5|498| +2450822|10603|5|940| +2450822|10604|5|761| +2450822|10606|5|28| +2450822|10609|5|| +2450822|10610|5|58| +2450822|10612|5|611| +2450822|10615|5|635| +2450822|10616|5|773| +2450822|10618|5|802| +2450822|10621|5|668| +2450822|10622|5|512| +2450822|10624|5|779| +2450822|10627|5|562| +2450822|10628|5|652| +2450822|10630|5|229| +2450822|10633|5|606| +2450822|10634|5|890| +2450822|10636|5|266| +2450822|10639|5|51| +2450822|10640|5|164| +2450822|10642|5|| +2450822|10645|5|732| +2450822|10646|5|721| +2450822|10648|5|350| +2450822|10651|5|450| +2450822|10652|5|794| +2450822|10654|5|377| +2450822|10657|5|219| +2450822|10658|5|739| +2450822|10660|5|116| +2450822|10663|5|307| +2450822|10664|5|499| +2450822|10666|5|| +2450822|10669|5|231| +2450822|10670|5|865| +2450822|10672|5|19| +2450822|10675|5|| +2450822|10676|5|202| +2450822|10678|5|316| +2450822|10681|5|| +2450822|10682|5|66| +2450822|10684|5|216| +2450822|10687|5|443| +2450822|10688|5|698| +2450822|10690|5|522| +2450822|10693|5|496| +2450822|10694|5|315| +2450822|10696|5|| +2450822|10699|5|460| +2450822|10700|5|0| +2450822|10702|5|990| +2450822|10705|5|693| +2450822|10706|5|311| +2450822|10708|5|137| +2450822|10711|5|172| +2450822|10712|5|976| +2450822|10714|5|668| +2450822|10717|5|807| +2450822|10718|5|| +2450822|10720|5|813| +2450822|10723|5|869| +2450822|10724|5|| +2450822|10726|5|754| +2450822|10729|5|| +2450822|10730|5|| +2450822|10732|5|241| +2450822|10735|5|829| +2450822|10736|5|320| +2450822|10738|5|563| +2450822|10741|5|184| +2450822|10742|5|784| +2450822|10744|5|136| +2450822|10747|5|731| +2450822|10748|5|626| +2450822|10750|5|495| +2450822|10753|5|203| +2450822|10754|5|830| +2450822|10756|5|648| +2450822|10759|5|920| +2450822|10760|5|427| +2450822|10762|5|838| +2450822|10765|5|560| +2450822|10766|5|421| +2450822|10768|5|832| +2450822|10771|5|| +2450822|10772|5|| +2450822|10774|5|207| +2450822|10777|5|| +2450822|10778|5|| +2450822|10780|5|654| +2450822|10783|5|540| +2450822|10784|5|884| +2450822|10786|5|863| +2450822|10789|5|351| +2450822|10790|5|524| +2450822|10792|5|| +2450822|10795|5|872| +2450822|10796|5|126| +2450822|10798|5|504| +2450822|10801|5|824| +2450822|10802|5|154| +2450822|10804|5|666| +2450822|10807|5|525| +2450822|10808|5|13| +2450822|10810|5|553| +2450822|10813|5|44| +2450822|10814|5|140| +2450822|10816|5|669| +2450822|10819|5|522| +2450822|10820|5|307| +2450822|10822|5|538| +2450822|10825|5|325| +2450822|10826|5|| +2450822|10828|5|123| +2450822|10831|5|997| +2450822|10832|5|175| +2450822|10834|5|27| +2450822|10837|5|894| +2450822|10838|5|819| +2450822|10840|5|791| +2450822|10843|5|992| +2450822|10844|5|| +2450822|10846|5|754| +2450822|10849|5|646| +2450822|10850|5|789| +2450822|10852|5|774| +2450822|10855|5|337| +2450822|10856|5|883| +2450822|10858|5|625| +2450822|10861|5|377| +2450822|10862|5|111| +2450822|10864|5|95| +2450822|10867|5|13| +2450822|10868|5|152| +2450822|10870|5|887| +2450822|10873|5|| +2450822|10874|5|241| +2450822|10876|5|86| +2450822|10879|5|843| +2450822|10880|5|271| +2450822|10882|5|220| +2450822|10885|5|669| +2450822|10886|5|284| +2450822|10888|5|491| +2450822|10891|5|101| +2450822|10892|5|980| +2450822|10894|5|758| +2450822|10897|5|733| +2450822|10898|5|321| +2450822|10900|5|10| +2450822|10903|5|83| +2450822|10904|5|195| +2450822|10906|5|933| +2450822|10909|5|149| +2450822|10910|5|333| +2450822|10912|5|50| +2450822|10915|5|530| +2450822|10916|5|59| +2450822|10918|5|645| +2450822|10921|5|841| +2450822|10922|5|| +2450822|10924|5|482| +2450822|10927|5|715| +2450822|10928|5|768| +2450822|10930|5|902| +2450822|10933|5|| +2450822|10934|5|2| +2450822|10936|5|904| +2450822|10939|5|96| +2450822|10940|5|528| +2450822|10942|5|160| +2450822|10945|5|23| +2450822|10946|5|243| +2450822|10948|5|23| +2450822|10951|5|442| +2450822|10952|5|160| +2450822|10954|5|724| +2450822|10957|5|151| +2450822|10958|5|630| +2450822|10960|5|328| +2450822|10963|5|98| +2450822|10964|5|9| +2450822|10966|5|461| +2450822|10969|5|455| +2450822|10970|5|955| +2450822|10972|5|758| +2450822|10975|5|177| +2450822|10976|5|259| +2450822|10978|5|354| +2450822|10981|5|239| +2450822|10982|5|351| +2450822|10984|5|617| +2450822|10987|5|744| +2450822|10988|5|187| +2450822|10990|5|142| +2450822|10993|5|39| +2450822|10994|5|27| +2450822|10996|5|438| +2450822|10999|5|103| +2450822|11000|5|754| +2450822|11002|5|155| +2450822|11005|5|46| +2450822|11006|5|366| +2450822|11008|5|708| +2450822|11011|5|408| +2450822|11012|5|513| +2450822|11014|5|67| +2450822|11017|5|978| +2450822|11018|5|345| +2450822|11020|5|451| +2450822|11023|5|73| +2450822|11024|5|976| +2450822|11026|5|323| +2450822|11029|5|76| +2450822|11030|5|967| +2450822|11032|5|897| +2450822|11035|5|55| +2450822|11036|5|47| +2450822|11038|5|111| +2450822|11041|5|416| +2450822|11042|5|130| +2450822|11044|5|167| +2450822|11047|5|486| +2450822|11048|5|290| +2450822|11050|5|993| +2450822|11053|5|463| +2450822|11054|5|626| +2450822|11056|5|813| +2450822|11059|5|88| +2450822|11060|5|819| +2450822|11062|5|453| +2450822|11065|5|335| +2450822|11066|5|335| +2450822|11068|5|618| +2450822|11071|5|473| +2450822|11072|5|695| +2450822|11074|5|32| +2450822|11077|5|655| +2450822|11078|5|| +2450822|11080|5|407| +2450822|11083|5|308| +2450822|11084|5|137| +2450822|11086|5|847| +2450822|11089|5|739| +2450822|11090|5|656| +2450822|11092|5|289| +2450822|11095|5|408| +2450822|11096|5|336| +2450822|11098|5|177| +2450822|11101|5|425| +2450822|11102|5|605| +2450822|11104|5|906| +2450822|11107|5|240| +2450822|11108|5|299| +2450822|11110|5|| +2450822|11113|5|919| +2450822|11114|5|838| +2450822|11116|5|103| +2450822|11119|5|147| +2450822|11120|5|229| +2450822|11122|5|348| +2450822|11125|5|167| +2450822|11126|5|890| +2450822|11128|5|| +2450822|11131|5|653| +2450822|11132|5|634| +2450822|11134|5|941| +2450822|11137|5|592| +2450822|11138|5|651| +2450822|11140|5|126| +2450822|11143|5|71| +2450822|11144|5|806| +2450822|11146|5|348| +2450822|11149|5|585| +2450822|11150|5|745| +2450822|11152|5|473| +2450822|11155|5|635| +2450822|11156|5|808| +2450822|11158|5|64| +2450822|11161|5|67| +2450822|11162|5|| +2450822|11164|5|515| +2450822|11167|5|638| +2450822|11168|5|961| +2450822|11170|5|| +2450822|11173|5|621| +2450822|11174|5|158| +2450822|11176|5|242| +2450822|11179|5|663| +2450822|11180|5|534| +2450822|11182|5|82| +2450822|11185|5|825| +2450822|11186|5|| +2450822|11188|5|883| +2450822|11191|5|262| +2450822|11192|5|606| +2450822|11194|5|896| +2450822|11197|5|444| +2450822|11198|5|57| +2450822|11200|5|991| +2450822|11203|5|283| +2450822|11204|5|556| +2450822|11206|5|510| +2450822|11209|5|| +2450822|11210|5|879| +2450822|11212|5|506| +2450822|11215|5|850| +2450822|11216|5|569| +2450822|11218|5|762| +2450822|11221|5|269| +2450822|11222|5|843| +2450822|11224|5|831| +2450822|11227|5|732| +2450822|11228|5|| +2450822|11230|5|168| +2450822|11233|5|716| +2450822|11234|5|185| +2450822|11236|5|296| +2450822|11239|5|227| +2450822|11240|5|836| +2450822|11242|5|231| +2450822|11245|5|387| +2450822|11246|5|721| +2450822|11248|5|180| +2450822|11251|5|833| +2450822|11252|5|6| +2450822|11254|5|655| +2450822|11257|5|249| +2450822|11258|5|652| +2450822|11260|5|255| +2450822|11263|5|262| +2450822|11264|5|913| +2450822|11266|5|8| +2450822|11269|5|656| +2450822|11270|5|570| +2450822|11272|5|443| +2450822|11275|5|322| +2450822|11276|5|1| +2450822|11278|5|389| +2450822|11281|5|104| +2450822|11282|5|602| +2450822|11284|5|383| +2450822|11287|5|136| +2450822|11288|5|818| +2450822|11290|5|924| +2450822|11293|5|218| +2450822|11294|5|474| +2450822|11296|5|2| +2450822|11299|5|741| +2450822|11300|5|226| +2450822|11302|5|708| +2450822|11305|5|80| +2450822|11306|5|182| +2450822|11308|5|847| +2450822|11311|5|716| +2450822|11312|5|90| +2450822|11314|5|50| +2450822|11317|5|412| +2450822|11318|5|416| +2450822|11320|5|764| +2450822|11323|5|720| +2450822|11324|5|501| +2450822|11326|5|678| +2450822|11329|5|687| +2450822|11330|5|554| +2450822|11332|5|24| +2450822|11335|5|250| +2450822|11336|5|| +2450822|11338|5|324| +2450822|11341|5|710| +2450822|11342|5|685| +2450822|11344|5|748| +2450822|11347|5|911| +2450822|11348|5|175| +2450822|11350|5|780| +2450822|11353|5|389| +2450822|11354|5|559| +2450822|11356|5|927| +2450822|11359|5|399| +2450822|11360|5|84| +2450822|11362|5|670| +2450822|11365|5|3| +2450822|11366|5|| +2450822|11368|5|95| +2450822|11371|5|| +2450822|11372|5|686| +2450822|11374|5|644| +2450822|11377|5|845| +2450822|11378|5|795| +2450822|11380|5|784| +2450822|11383|5|96| +2450822|11384|5|540| +2450822|11386|5|558| +2450822|11389|5|| +2450822|11390|5|432| +2450822|11392|5|699| +2450822|11395|5|785| +2450822|11396|5|985| +2450822|11398|5|578| +2450822|11401|5|249| +2450822|11402|5|188| +2450822|11404|5|569| +2450822|11407|5|26| +2450822|11408|5|728| +2450822|11410|5|135| +2450822|11413|5|333| +2450822|11414|5|772| +2450822|11416|5|661| +2450822|11419|5|476| +2450822|11420|5|868| +2450822|11422|5|42| +2450822|11425|5|893| +2450822|11426|5|521| +2450822|11428|5|570| +2450822|11431|5|627| +2450822|11432|5|361| +2450822|11434|5|119| +2450822|11437|5|839| +2450822|11438|5|150| +2450822|11440|5|371| +2450822|11443|5|237| +2450822|11444|5|960| +2450822|11446|5|141| +2450822|11449|5|993| +2450822|11450|5|281| +2450822|11452|5|853| +2450822|11455|5|91| +2450822|11456|5|466| +2450822|11458|5|346| +2450822|11461|5|538| +2450822|11462|5|838| +2450822|11464|5|664| +2450822|11467|5|231| +2450822|11468|5|404| +2450822|11470|5|454| +2450822|11473|5|201| +2450822|11474|5|757| +2450822|11476|5|843| +2450822|11479|5|507| +2450822|11480|5|955| +2450822|11482|5|937| +2450822|11485|5|944| +2450822|11486|5|391| +2450822|11488|5|648| +2450822|11491|5|771| +2450822|11492|5|868| +2450822|11494|5|190| +2450822|11497|5|588| +2450822|11498|5|788| +2450822|11500|5|671| +2450822|11503|5|414| +2450822|11504|5|727| +2450822|11506|5|332| +2450822|11509|5|304| +2450822|11510|5|113| +2450822|11512|5|499| +2450822|11515|5|274| +2450822|11516|5|760| +2450822|11518|5|188| +2450822|11521|5|341| +2450822|11522|5|151| +2450822|11524|5|947| +2450822|11527|5|7| +2450822|11528|5|57| +2450822|11530|5|55| +2450822|11533|5|6| +2450822|11534|5|608| +2450822|11536|5|372| +2450822|11539|5|918| +2450822|11540|5|414| +2450822|11542|5|778| +2450822|11545|5|818| +2450822|11546|5|323| +2450822|11548|5|168| +2450822|11551|5|561| +2450822|11552|5|433| +2450822|11554|5|540| +2450822|11557|5|669| +2450822|11558|5|241| +2450822|11560|5|| +2450822|11563|5|| +2450822|11564|5|174| +2450822|11566|5|794| +2450822|11569|5|806| +2450822|11570|5|201| +2450822|11572|5|| +2450822|11575|5|495| +2450822|11576|5|744| +2450822|11578|5|135| +2450822|11581|5|120| +2450822|11582|5|66| +2450822|11584|5|650| +2450822|11587|5|858| +2450822|11588|5|58| +2450822|11590|5|519| +2450822|11593|5|107| +2450822|11594|5|965| +2450822|11596|5|682| +2450822|11599|5|845| +2450822|11600|5|782| +2450822|11602|5|676| +2450822|11605|5|945| +2450822|11606|5|828| +2450822|11608|5|168| +2450822|11611|5|425| +2450822|11612|5|471| +2450822|11614|5|| +2450822|11617|5|884| +2450822|11618|5|781| +2450822|11620|5|612| +2450822|11623|5|168| +2450822|11624|5|| +2450822|11626|5|627| +2450822|11629|5|671| +2450822|11630|5|20| +2450822|11632|5|810| +2450822|11635|5|738| +2450822|11636|5|176| +2450822|11638|5|374| +2450822|11641|5|113| +2450822|11642|5|814| +2450822|11644|5|448| +2450822|11647|5|692| +2450822|11648|5|577| +2450822|11650|5|482| +2450822|11653|5|907| +2450822|11654|5|161| +2450822|11656|5|| +2450822|11659|5|131| +2450822|11660|5|907| +2450822|11662|5|121| +2450822|11665|5|| +2450822|11666|5|711| +2450822|11668|5|703| +2450822|11671|5|278| +2450822|11672|5|484| +2450822|11674|5|668| +2450822|11677|5|726| +2450822|11678|5|880| +2450822|11680|5|727| +2450822|11683|5|154| +2450822|11684|5|| +2450822|11686|5|469| +2450822|11689|5|938| +2450822|11690|5|| +2450822|11692|5|460| +2450822|11695|5|360| +2450822|11696|5|87| +2450822|11698|5|683| +2450822|11701|5|| +2450822|11702|5|| +2450822|11704|5|885| +2450822|11707|5|284| +2450822|11708|5|372| +2450822|11710|5|980| +2450822|11713|5|287| +2450822|11714|5|123| +2450822|11716|5|106| +2450822|11719|5|745| +2450822|11720|5|811| +2450822|11722|5|879| +2450822|11725|5|528| +2450822|11726|5|38| +2450822|11728|5|576| +2450822|11731|5|988| +2450822|11732|5|458| +2450822|11734|5|212| +2450822|11737|5|949| +2450822|11738|5|929| +2450822|11740|5|656| +2450822|11743|5|301| +2450822|11744|5|185| +2450822|11746|5|454| +2450822|11749|5|347| +2450822|11750|5|396| +2450822|11752|5|297| +2450822|11755|5|658| +2450822|11756|5|781| +2450822|11758|5|653| +2450822|11761|5|17| +2450822|11762|5|201| +2450822|11764|5|512| +2450822|11767|5|289| +2450822|11768|5|914| +2450822|11770|5|472| +2450822|11773|5|445| +2450822|11774|5|605| +2450822|11776|5|860| +2450822|11779|5|239| +2450822|11780|5|880| +2450822|11782|5|482| +2450822|11785|5|344| +2450822|11786|5|527| +2450822|11788|5|156| +2450822|11791|5|352| +2450822|11792|5|116| +2450822|11794|5|646| +2450822|11797|5|943| +2450822|11798|5|688| +2450822|11800|5|504| +2450822|11803|5|962| +2450822|11804|5|648| +2450822|11806|5|26| +2450822|11809|5|967| +2450822|11810|5|468| +2450822|11812|5|51| +2450822|11815|5|768| +2450822|11816|5|657| +2450822|11818|5|803| +2450822|11821|5|244| +2450822|11822|5|457| +2450822|11824|5|859| +2450822|11827|5|| +2450822|11828|5|411| +2450822|11830|5|809| +2450822|11833|5|175| +2450822|11834|5|842| +2450822|11836|5|104| +2450822|11839|5|542| +2450822|11840|5|426| +2450822|11842|5|300| +2450822|11845|5|501| +2450822|11846|5|| +2450822|11848|5|229| +2450822|11851|5|695| +2450822|11852|5|96| +2450822|11854|5|139| +2450822|11857|5|762| +2450822|11858|5|666| +2450822|11860|5|902| +2450822|11863|5|114| +2450822|11864|5|48| +2450822|11866|5|217| +2450822|11869|5|349| +2450822|11870|5|12| +2450822|11872|5|760| +2450822|11875|5|344| +2450822|11876|5|60| +2450822|11878|5|95| +2450822|11881|5|777| +2450822|11882|5|501| +2450822|11884|5|613| +2450822|11887|5|762| +2450822|11888|5|550| +2450822|11890|5|551| +2450822|11893|5|279| +2450822|11894|5|651| +2450822|11896|5|818| +2450822|11899|5|| +2450822|11900|5|866| +2450822|11902|5|231| +2450822|11905|5|219| +2450822|11906|5|342| +2450822|11908|5|120| +2450822|11911|5|380| +2450822|11912|5|936| +2450822|11914|5|593| +2450822|11917|5|344| +2450822|11918|5|162| +2450822|11920|5|938| +2450822|11923|5|976| +2450822|11924|5|268| +2450822|11926|5|139| +2450822|11929|5|7| +2450822|11930|5|345| +2450822|11932|5|356| +2450822|11935|5|553| +2450822|11936|5|174| +2450822|11938|5|643| +2450822|11941|5|41| +2450822|11942|5|505| +2450822|11944|5|625| +2450822|11947|5|789| +2450822|11948|5|| +2450822|11950|5|479| +2450822|11953|5|15| +2450822|11954|5|958| +2450822|11956|5|436| +2450822|11959|5|748| +2450822|11960|5|747| +2450822|11962|5|956| +2450822|11965|5|966| +2450822|11966|5|182| +2450822|11968|5|624| +2450822|11971|5|295| +2450822|11972|5|43| +2450822|11974|5|316| +2450822|11977|5|706| +2450822|11978|5|967| +2450822|11980|5|176| +2450822|11983|5|584| +2450822|11984|5|183| +2450822|11986|5|11| +2450822|11989|5|448| +2450822|11990|5|449| +2450822|11992|5|139| +2450822|11995|5|400| +2450822|11996|5|202| +2450822|11998|5|471| +2450822|12001|5|| +2450822|12002|5|1000| +2450822|12004|5|543| +2450822|12007|5|| +2450822|12008|5|872| +2450822|12010|5|138| +2450822|12013|5|700| +2450822|12014|5|699| +2450822|12016|5|608| +2450822|12019|5|368| +2450822|12020|5|595| +2450822|12022|5|606| +2450822|12025|5|801| +2450822|12026|5|856| +2450822|12028|5|154| +2450822|12031|5|872| +2450822|12032|5|476| +2450822|12034|5|411| +2450822|12037|5|644| +2450822|12038|5|365| +2450822|12040|5|856| +2450822|12043|5|79| +2450822|12044|5|133| +2450822|12046|5|345| +2450822|12049|5|785| +2450822|12050|5|560| +2450822|12052|5|618| +2450822|12055|5|718| +2450822|12056|5|| +2450822|12058|5|633| +2450822|12061|5|137| +2450822|12062|5|688| +2450822|12064|5|609| +2450822|12067|5|437| +2450822|12068|5|737| +2450822|12070|5|874| +2450822|12073|5|379| +2450822|12074|5|103| +2450822|12076|5|943| +2450822|12079|5|783| +2450822|12080|5|68| +2450822|12082|5|| +2450822|12085|5|102| +2450822|12086|5|787| +2450822|12088|5|80| +2450822|12091|5|356| +2450822|12092|5|192| +2450822|12094|5|10| +2450822|12097|5|635| +2450822|12098|5|276| +2450822|12100|5|777| +2450822|12103|5|684| +2450822|12104|5|67| +2450822|12106|5|523| +2450822|12109|5|943| +2450822|12110|5|805| +2450822|12112|5|223| +2450822|12115|5|429| +2450822|12116|5|332| +2450822|12118|5|620| +2450822|12121|5|405| +2450822|12122|5|484| +2450822|12124|5|271| +2450822|12127|5|974| +2450822|12128|5|576| +2450822|12130|5|134| +2450822|12133|5|42| +2450822|12134|5|775| +2450822|12136|5|461| +2450822|12139|5|285| +2450822|12140|5|161| +2450822|12142|5|145| +2450822|12145|5|542| +2450822|12146|5|210| +2450822|12148|5|622| +2450822|12151|5|239| +2450822|12152|5|20| +2450822|12154|5|834| +2450822|12157|5|884| +2450822|12158|5|240| +2450822|12160|5|248| +2450822|12163|5|404| +2450822|12164|5|156| +2450822|12166|5|40| +2450822|12169|5|710| +2450822|12170|5|242| +2450822|12172|5|269| +2450822|12175|5|157| +2450822|12176|5|571| +2450822|12178|5|739| +2450822|12181|5|125| +2450822|12182|5|442| +2450822|12184|5|987| +2450822|12187|5|206| +2450822|12188|5|563| +2450822|12190|5|42| +2450822|12193|5|754| +2450822|12194|5|985| +2450822|12196|5|483| +2450822|12199|5|958| +2450822|12200|5|31| +2450822|12202|5|177| +2450822|12205|5|873| +2450822|12206|5|631| +2450822|12208|5|748| +2450822|12211|5|430| +2450822|12212|5|282| +2450822|12214|5|716| +2450822|12217|5|555| +2450822|12218|5|565| +2450822|12220|5|639| +2450822|12223|5|838| +2450822|12224|5|553| +2450822|12226|5|802| +2450822|12229|5|75| +2450822|12230|5|466| +2450822|12232|5|695| +2450822|12235|5|968| +2450822|12236|5|955| +2450822|12238|5|216| +2450822|12241|5|354| +2450822|12242|5|861| +2450822|12244|5|959| +2450822|12247|5|| +2450822|12248|5|869| +2450822|12250|5|520| +2450822|12253|5|49| +2450822|12254|5|563| +2450822|12256|5|258| +2450822|12259|5|87| +2450822|12260|5|156| +2450822|12262|5|10| +2450822|12265|5|166| +2450822|12266|5|460| +2450822|12268|5|964| +2450822|12271|5|113| +2450822|12272|5|110| +2450822|12274|5|819| +2450822|12277|5|473| +2450822|12278|5|470| +2450822|12280|5|451| +2450822|12283|5|137| +2450822|12284|5|841| +2450822|12286|5|753| +2450822|12289|5|| +2450822|12290|5|729| +2450822|12292|5|153| +2450822|12295|5|177| +2450822|12296|5|994| +2450822|12298|5|410| +2450822|12301|5|893| +2450822|12302|5|967| +2450822|12304|5|653| +2450822|12307|5|567| +2450822|12308|5|370| +2450822|12310|5|639| +2450822|12313|5|247| +2450822|12314|5|| +2450822|12316|5|846| +2450822|12319|5|113| +2450822|12320|5|523| +2450822|12322|5|46| +2450822|12325|5|988| +2450822|12326|5|675| +2450822|12328|5|219| +2450822|12331|5|785| +2450822|12332|5|265| +2450822|12334|5|919| +2450822|12337|5|571| +2450822|12338|5|138| +2450822|12340|5|560| +2450822|12343|5|647| +2450822|12344|5|507| +2450822|12346|5|162| +2450822|12349|5|128| +2450822|12350|5|501| +2450822|12352|5|| +2450822|12355|5|832| +2450822|12356|5|449| +2450822|12358|5|744| +2450822|12361|5|693| +2450822|12362|5|967| +2450822|12364|5|134| +2450822|12367|5|364| +2450822|12368|5|| +2450822|12370|5|466| +2450822|12373|5|| +2450822|12374|5|317| +2450822|12376|5|410| +2450822|12379|5|182| +2450822|12380|5|708| +2450822|12382|5|266| +2450822|12385|5|16| +2450822|12386|5|799| +2450822|12388|5|493| +2450822|12391|5|799| +2450822|12392|5|370| +2450822|12394|5|711| +2450822|12397|5|351| +2450822|12398|5|527| +2450822|12400|5|335| +2450822|12403|5|363| +2450822|12404|5|421| +2450822|12406|5|375| +2450822|12409|5|198| +2450822|12410|5|| +2450822|12412|5|803| +2450822|12415|5|456| +2450822|12416|5|938| +2450822|12418|5|589| +2450822|12421|5|780| +2450822|12422|5|240| +2450822|12424|5|| +2450822|12427|5|46| +2450822|12428|5|4| +2450822|12430|5|289| +2450822|12433|5|743| +2450822|12434|5|613| +2450822|12436|5|926| +2450822|12439|5|268| +2450822|12440|5|514| +2450822|12442|5|| +2450822|12445|5|547| +2450822|12446|5|285| +2450822|12448|5|94| +2450822|12451|5|371| +2450822|12452|5|166| +2450822|12454|5|87| +2450822|12457|5|| +2450822|12458|5|181| +2450822|12460|5|358| +2450822|12463|5|523| +2450822|12464|5|192| +2450822|12466|5|652| +2450822|12469|5|957| +2450822|12470|5|28| +2450822|12472|5|425| +2450822|12475|5|387| +2450822|12476|5|355| +2450822|12478|5|500| +2450822|12481|5|757| +2450822|12482|5|855| +2450822|12484|5|287| +2450822|12487|5|187| +2450822|12488|5|121| +2450822|12490|5|544| +2450822|12493|5|305| +2450822|12494|5|32| +2450822|12496|5|786| +2450822|12499|5|327| +2450822|12500|5|216| +2450822|12502|5|379| +2450822|12505|5|490| +2450822|12506|5|457| +2450822|12508|5|212| +2450822|12511|5|236| +2450822|12512|5|300| +2450822|12514|5|728| +2450822|12517|5|630| +2450822|12518|5|166| +2450822|12520|5|483| +2450822|12523|5|665| +2450822|12524|5|301| +2450822|12526|5|208| +2450822|12529|5|1| +2450822|12530|5|916| +2450822|12532|5|878| +2450822|12535|5|462| +2450822|12536|5|| +2450822|12538|5|539| +2450822|12541|5|50| +2450822|12542|5|86| +2450822|12544|5|925| +2450822|12547|5|945| +2450822|12548|5|652| +2450822|12550|5|825| +2450822|12553|5|513| +2450822|12554|5|172| +2450822|12556|5|322| +2450822|12559|5|721| +2450822|12560|5|405| +2450822|12562|5|451| +2450822|12565|5|605| +2450822|12566|5|156| +2450822|12568|5|64| +2450822|12571|5|535| +2450822|12572|5|857| +2450822|12574|5|436| +2450822|12577|5|695| +2450822|12578|5|657| +2450822|12580|5|334| +2450822|12583|5|254| +2450822|12584|5|341| +2450822|12586|5|482| +2450822|12589|5|218| +2450822|12590|5|| +2450822|12592|5|4| +2450822|12595|5|| +2450822|12596|5|842| +2450822|12598|5|103| +2450822|12601|5|417| +2450822|12602|5|705| +2450822|12604|5|159| +2450822|12607|5|854| +2450822|12608|5|120| +2450822|12610|5|880| +2450822|12613|5|999| +2450822|12614|5|378| +2450822|12616|5|371| +2450822|12619|5|46| +2450822|12620|5|953| +2450822|12622|5|725| +2450822|12625|5|51| +2450822|12626|5|904| +2450822|12628|5|516| +2450822|12631|5|468| +2450822|12632|5|766| +2450822|12634|5|137| +2450822|12637|5|632| +2450822|12638|5|781| +2450822|12640|5|779| +2450822|12643|5|757| +2450822|12644|5|366| +2450822|12646|5|21| +2450822|12649|5|369| +2450822|12650|5|1| +2450822|12652|5|| +2450822|12655|5|| +2450822|12656|5|722| +2450822|12658|5|710| +2450822|12661|5|466| +2450822|12662|5|747| +2450822|12664|5|78| +2450822|12667|5|400| +2450822|12668|5|911| +2450822|12670|5|514| +2450822|12673|5|654| +2450822|12674|5|186| +2450822|12676|5|32| +2450822|12679|5|956| +2450822|12680|5|| +2450822|12682|5|609| +2450822|12685|5|456| +2450822|12686|5|473| +2450822|12688|5|135| +2450822|12691|5|268| +2450822|12692|5|767| +2450822|12694|5|258| +2450822|12697|5|819| +2450822|12698|5|651| +2450822|12700|5|418| +2450822|12703|5|454| +2450822|12704|5|704| +2450822|12706|5|940| +2450822|12709|5|681| +2450822|12710|5|334| +2450822|12712|5|547| +2450822|12715|5|760| +2450822|12716|5|571| +2450822|12718|5|114| +2450822|12721|5|590| +2450822|12722|5|381| +2450822|12724|5|897| +2450822|12727|5|437| +2450822|12728|5|637| +2450822|12730|5|828| +2450822|12733|5|127| +2450822|12734|5|521| +2450822|12736|5|169| +2450822|12739|5|95| +2450822|12740|5|927| +2450822|12742|5|996| +2450822|12745|5|991| +2450822|12746|5|668| +2450822|12748|5|831| +2450822|12751|5|717| +2450822|12752|5|948| +2450822|12754|5|289| +2450822|12757|5|790| +2450822|12758|5|528| +2450822|12760|5|545| +2450822|12763|5|206| +2450822|12764|5|250| +2450822|12766|5|735| +2450822|12769|5|634| +2450822|12770|5|116| +2450822|12772|5|738| +2450822|12775|5|534| +2450822|12776|5|524| +2450822|12778|5|625| +2450822|12781|5|840| +2450822|12782|5|993| +2450822|12784|5|841| +2450822|12787|5|639| +2450822|12788|5|154| +2450822|12790|5|45| +2450822|12793|5|215| +2450822|12794|5|268| +2450822|12796|5|856| +2450822|12799|5|15| +2450822|12800|5|159| +2450822|12802|5|122| +2450822|12805|5|219| +2450822|12806|5|144| +2450822|12808|5|90| +2450822|12811|5|968| +2450822|12812|5|665| +2450822|12814|5|68| +2450822|12817|5|774| +2450822|12818|5|452| +2450822|12820|5|840| +2450822|12823|5|495| +2450822|12824|5|| +2450822|12826|5|898| +2450822|12829|5|49| +2450822|12830|5|544| +2450822|12832|5|849| +2450822|12835|5|730| +2450822|12836|5|0| +2450822|12838|5|688| +2450822|12841|5|266| +2450822|12842|5|764| +2450822|12844|5|24| +2450822|12847|5|378| +2450822|12848|5|926| +2450822|12850|5|815| +2450822|12853|5|208| +2450822|12854|5|524| +2450822|12856|5|946| +2450822|12859|5|69| +2450822|12860|5|769| +2450822|12862|5|777| +2450822|12865|5|872| +2450822|12866|5|43| +2450822|12868|5|61| +2450822|12871|5|535| +2450822|12872|5|336| +2450822|12874|5|463| +2450822|12877|5|791| +2450822|12878|5|731| +2450822|12880|5|212| +2450822|12883|5|970| +2450822|12884|5|437| +2450822|12886|5|889| +2450822|12889|5|576| +2450822|12890|5|655| +2450822|12892|5|25| +2450822|12895|5|52| +2450822|12896|5|285| +2450822|12898|5|406| +2450822|12901|5|835| +2450822|12902|5|233| +2450822|12904|5|919| +2450822|12907|5|389| +2450822|12908|5|125| +2450822|12910|5|869| +2450822|12913|5|| +2450822|12914|5|732| +2450822|12916|5|205| +2450822|12919|5|594| +2450822|12920|5|828| +2450822|12922|5|671| +2450822|12925|5|58| +2450822|12926|5|641| +2450822|12928|5|78| +2450822|12931|5|277| +2450822|12932|5|373| +2450822|12934|5|174| +2450822|12937|5|111| +2450822|12938|5|223| +2450822|12940|5|684| +2450822|12943|5|797| +2450822|12944|5|828| +2450822|12946|5|537| +2450822|12949|5|242| +2450822|12950|5|588| +2450822|12952|5|550| +2450822|12955|5|432| +2450822|12956|5|29| +2450822|12958|5|227| +2450822|12961|5|387| +2450822|12962|5|187| +2450822|12964|5|| +2450822|12967|5|823| +2450822|12968|5|470| +2450822|12970|5|1| +2450822|12973|5|353| +2450822|12974|5|937| +2450822|12976|5|858| +2450822|12979|5|298| +2450822|12980|5|443| +2450822|12982|5|265| +2450822|12985|5|201| +2450822|12986|5|239| +2450822|12988|5|777| +2450822|12991|5|569| +2450822|12992|5|626| +2450822|12994|5|291| +2450822|12997|5|707| +2450822|12998|5|554| +2450822|13000|5|772| +2450822|13003|5|110| +2450822|13004|5|924| +2450822|13006|5|415| +2450822|13009|5|687| +2450822|13010|5|47| +2450822|13012|5|669| +2450822|13015|5|974| +2450822|13016|5|925| +2450822|13018|5|161| +2450822|13021|5|336| +2450822|13022|5|928| +2450822|13024|5|634| +2450822|13027|5|731| +2450822|13028|5|916| +2450822|13030|5|164| +2450822|13033|5|226| +2450822|13034|5|134| +2450822|13036|5|392| +2450822|13039|5|894| +2450822|13040|5|629| +2450822|13042|5|653| +2450822|13045|5|142| +2450822|13046|5|686| +2450822|13048|5|30| +2450822|13051|5|799| +2450822|13052|5|863| +2450822|13054|5|372| +2450822|13057|5|117| +2450822|13058|5|233| +2450822|13060|5|160| +2450822|13063|5|741| +2450822|13064|5|718| +2450822|13066|5|645| +2450822|13069|5|771| +2450822|13070|5|599| +2450822|13072|5|601| +2450822|13075|5|219| +2450822|13076|5|| +2450822|13078|5|324| +2450822|13081|5|320| +2450822|13082|5|131| +2450822|13084|5|170| +2450822|13087|5|427| +2450822|13088|5|377| +2450822|13090|5|269| +2450822|13093|5|123| +2450822|13094|5|390| +2450822|13096|5|20| +2450822|13099|5|552| +2450822|13100|5|105| +2450822|13102|5|| +2450822|13105|5|935| +2450822|13106|5|155| +2450822|13108|5|54| +2450822|13111|5|773| +2450822|13112|5|455| +2450822|13114|5|917| +2450822|13117|5|261| +2450822|13118|5|170| +2450822|13120|5|809| +2450822|13123|5|926| +2450822|13124|5|864| +2450822|13126|5|248| +2450822|13129|5|400| +2450822|13130|5|785| +2450822|13132|5|41| +2450822|13135|5|273| +2450822|13136|5|| +2450822|13138|5|58| +2450822|13141|5|936| +2450822|13142|5|138| +2450822|13144|5|448| +2450822|13147|5|989| +2450822|13148|5|524| +2450822|13150|5|42| +2450822|13153|5|564| +2450822|13154|5|786| +2450822|13156|5|322| +2450822|13159|5|| +2450822|13160|5|301| +2450822|13162|5|641| +2450822|13165|5|901| +2450822|13166|5|129| +2450822|13168|5|37| +2450822|13171|5|463| +2450822|13172|5|591| +2450822|13174|5|137| +2450822|13177|5|321| +2450822|13178|5|541| +2450822|13180|5|849| +2450822|13183|5|498| +2450822|13184|5|763| +2450822|13186|5|466| +2450822|13189|5|299| +2450822|13190|5|326| +2450822|13192|5|898| +2450822|13195|5|471| +2450822|13196|5|969| +2450822|13198|5|978| +2450822|13201|5|13| +2450822|13202|5|204| +2450822|13204|5|399| +2450822|13207|5|315| +2450822|13208|5|169| +2450822|13210|5|105| +2450822|13213|5|21| +2450822|13214|5|40| +2450822|13216|5|861| +2450822|13219|5|461| +2450822|13220|5|110| +2450822|13222|5|355| +2450822|13225|5|604| +2450822|13226|5|421| +2450822|13228|5|686| +2450822|13231|5|328| +2450822|13232|5|566| +2450822|13234|5|347| +2450822|13237|5|305| +2450822|13238|5|239| +2450822|13240|5|91| +2450822|13243|5|442| +2450822|13244|5|163| +2450822|13246|5|988| +2450822|13249|5|295| +2450822|13250|5|274| +2450822|13252|5|441| +2450822|13255|5|61| +2450822|13256|5|620| +2450822|13258|5|40| +2450822|13261|5|464| +2450822|13262|5|295| +2450822|13264|5|| +2450822|13267|5|| +2450822|13268|5|69| +2450822|13270|5|480| +2450822|13273|5|844| +2450822|13274|5|327| +2450822|13276|5|734| +2450822|13279|5|439| +2450822|13280|5|906| +2450822|13282|5|584| +2450822|13285|5|563| +2450822|13286|5|944| +2450822|13288|5|148| +2450822|13291|5|174| +2450822|13292|5|195| +2450822|13294|5|481| +2450822|13297|5|18| +2450822|13298|5|287| +2450822|13300|5|720| +2450822|13303|5|148| +2450822|13304|5|95| +2450822|13306|5|565| +2450822|13309|5|607| +2450822|13310|5|| +2450822|13312|5|378| +2450822|13315|5|404| +2450822|13316|5|836| +2450822|13318|5|| +2450822|13321|5|511| +2450822|13322|5|557| +2450822|13324|5|488| +2450822|13327|5|188| +2450822|13328|5|469| +2450822|13330|5|746| +2450822|13333|5|751| +2450822|13334|5|976| +2450822|13336|5|433| +2450822|13339|5|103| +2450822|13340|5|367| +2450822|13342|5|508| +2450822|13345|5|140| +2450822|13346|5|| +2450822|13348|5|93| +2450822|13351|5|996| +2450822|13352|5|152| +2450822|13354|5|346| +2450822|13357|5|654| +2450822|13358|5|| +2450822|13360|5|366| +2450822|13363|5|569| +2450822|13364|5|312| +2450822|13366|5|353| +2450822|13369|5|106| +2450822|13370|5|361| +2450822|13372|5|520| +2450822|13375|5|972| +2450822|13376|5|969| +2450822|13378|5|142| +2450822|13381|5|252| +2450822|13382|5|805| +2450822|13384|5|215| +2450822|13387|5|| +2450822|13388|5|465| +2450822|13390|5|254| +2450822|13393|5|89| +2450822|13394|5|201| +2450822|13396|5|| +2450822|13399|5|30| +2450822|13400|5|305| +2450822|13402|5|538| +2450822|13405|5|549| +2450822|13406|5|376| +2450822|13408|5|649| +2450822|13411|5|812| +2450822|13412|5|464| +2450822|13414|5|809| +2450822|13417|5|184| +2450822|13418|5|599| +2450822|13420|5|373| +2450822|13423|5|686| +2450822|13424|5|628| +2450822|13426|5|| +2450822|13429|5|78| +2450822|13430|5|41| +2450822|13432|5|3| +2450822|13435|5|462| +2450822|13436|5|63| +2450822|13438|5|21| +2450822|13441|5|474| +2450822|13442|5|45| +2450822|13444|5|768| +2450822|13447|5|564| +2450822|13448|5|267| +2450822|13450|5|390| +2450822|13453|5|403| +2450822|13454|5|603| +2450822|13456|5|789| +2450822|13459|5|478| +2450822|13460|5|735| +2450822|13462|5|427| +2450822|13465|5|47| +2450822|13466|5|138| +2450822|13468|5|402| +2450822|13471|5|| +2450822|13472|5|759| +2450822|13474|5|| +2450822|13477|5|652| +2450822|13478|5|135| +2450822|13480|5|355| +2450822|13483|5|| +2450822|13484|5|611| +2450822|13486|5|774| +2450822|13489|5|607| +2450822|13490|5|177| +2450822|13492|5|546| +2450822|13495|5|| +2450822|13496|5|| +2450822|13498|5|64| +2450822|13501|5|523| +2450822|13502|5|944| +2450822|13504|5|114| +2450822|13507|5|339| +2450822|13508|5|833| +2450822|13510|5|275| +2450822|13513|5|313| +2450822|13514|5|251| +2450822|13516|5|77| +2450822|13519|5|618| +2450822|13520|5|571| +2450822|13522|5|846| +2450822|13525|5|891| +2450822|13526|5|133| +2450822|13528|5|189| +2450822|13531|5|| +2450822|13532|5|132| +2450822|13534|5|615| +2450822|13537|5|225| +2450822|13538|5|115| +2450822|13540|5|120| +2450822|13543|5|768| +2450822|13544|5|103| +2450822|13546|5|988| +2450822|13549|5|| +2450822|13550|5|501| +2450822|13552|5|660| +2450822|13555|5|395| +2450822|13556|5|686| +2450822|13558|5|84| +2450822|13561|5|| +2450822|13562|5|644| +2450822|13564|5|337| +2450822|13567|5|374| +2450822|13568|5|697| +2450822|13570|5|722| +2450822|13573|5|249| +2450822|13574|5|820| +2450822|13576|5|905| +2450822|13579|5|676| +2450822|13580|5|798| +2450822|13582|5|403| +2450822|13585|5|130| +2450822|13586|5|736| +2450822|13588|5|778| +2450822|13591|5|573| +2450822|13592|5|| +2450822|13594|5|38| +2450822|13597|5|602| +2450822|13598|5|521| +2450822|13600|5|921| +2450822|13603|5|939| +2450822|13604|5|634| +2450822|13606|5|726| +2450822|13609|5|343| +2450822|13610|5|173| +2450822|13612|5|893| +2450822|13615|5|964| +2450822|13616|5|602| +2450822|13618|5|404| +2450822|13621|5|660| +2450822|13622|5|872| +2450822|13624|5|917| +2450822|13627|5|996| +2450822|13628|5|404| +2450822|13630|5|898| +2450822|13633|5|80| +2450822|13634|5|569| +2450822|13636|5|| +2450822|13639|5|522| +2450822|13640|5|738| +2450822|13642|5|319| +2450822|13645|5|902| +2450822|13646|5|715| +2450822|13648|5|43| +2450822|13651|5|75| +2450822|13652|5|475| +2450822|13654|5|287| +2450822|13657|5|631| +2450822|13658|5|941| +2450822|13660|5|833| +2450822|13663|5|958| +2450822|13664|5|209| +2450822|13666|5|674| +2450822|13669|5|968| +2450822|13670|5|990| +2450822|13672|5|958| +2450822|13675|5|| +2450822|13676|5|648| +2450822|13678|5|569| +2450822|13681|5|868| +2450822|13682|5|974| +2450822|13684|5|814| +2450822|13687|5|244| +2450822|13688|5|588| +2450822|13690|5|623| +2450822|13693|5|500| +2450822|13694|5|21| +2450822|13696|5|309| +2450822|13699|5|357| +2450822|13700|5|820| +2450822|13702|5|431| +2450822|13705|5|791| +2450822|13706|5|733| +2450822|13708|5|790| +2450822|13711|5|375| +2450822|13712|5|319| +2450822|13714|5|205| +2450822|13717|5|215| +2450822|13718|5|463| +2450822|13720|5|567| +2450822|13723|5|693| +2450822|13724|5|908| +2450822|13726|5|605| +2450822|13729|5|980| +2450822|13730|5|725| +2450822|13732|5|173| +2450822|13735|5|838| +2450822|13736|5|76| +2450822|13738|5|895| +2450822|13741|5|165| +2450822|13742|5|802| +2450822|13744|5|616| +2450822|13747|5|365| +2450822|13748|5|| +2450822|13750|5|359| +2450822|13753|5|| +2450822|13754|5|543| +2450822|13756|5|436| +2450822|13759|5|736| +2450822|13760|5|516| +2450822|13762|5|364| +2450822|13765|5|346| +2450822|13766|5|576| +2450822|13768|5|455| +2450822|13771|5|64| +2450822|13772|5|284| +2450822|13774|5|| +2450822|13777|5|109| +2450822|13778|5|618| +2450822|13780|5|958| +2450822|13783|5|793| +2450822|13784|5|982| +2450822|13786|5|881| +2450822|13789|5|636| +2450822|13790|5|592| +2450822|13792|5|272| +2450822|13795|5|523| +2450822|13796|5|238| +2450822|13798|5|896| +2450822|13801|5|887| +2450822|13802|5|357| +2450822|13804|5|58| +2450822|13807|5|43| +2450822|13808|5|788| +2450822|13810|5|553| +2450822|13813|5|739| +2450822|13814|5|701| +2450822|13816|5|712| +2450822|13819|5|169| +2450822|13820|5|704| +2450822|13822|5|855| +2450822|13825|5|827| +2450822|13826|5|682| +2450822|13828|5|212| +2450822|13831|5|498| +2450822|13832|5|660| +2450822|13834|5|67| +2450822|13837|5|686| +2450822|13838|5|484| +2450822|13840|5|22| +2450822|13843|5|719| +2450822|13844|5|401| +2450822|13846|5|766| +2450822|13849|5|895| +2450822|13850|5|559| +2450822|13852|5|867| +2450822|13855|5|375| +2450822|13856|5|83| +2450822|13858|5|381| +2450822|13861|5|277| +2450822|13862|5|495| +2450822|13864|5|22| +2450822|13867|5|427| +2450822|13868|5|909| +2450822|13870|5|685| +2450822|13873|5|238| +2450822|13874|5|914| +2450822|13876|5|509| +2450822|13879|5|336| +2450822|13880|5|470| +2450822|13882|5|341| +2450822|13885|5|491| +2450822|13886|5|546| +2450822|13888|5|547| +2450822|13891|5|71| +2450822|13892|5|| +2450822|13894|5|2| +2450822|13897|5|590| +2450822|13898|5|728| +2450822|13900|5|417| +2450822|13903|5|287| +2450822|13904|5|277| +2450822|13906|5|804| +2450822|13909|5|192| +2450822|13910|5|825| +2450822|13912|5|754| +2450822|13915|5|72| +2450822|13916|5|902| +2450822|13918|5|621| +2450822|13921|5|182| +2450822|13922|5|667| +2450822|13924|5|659| +2450822|13927|5|190| +2450822|13928|5|940| +2450822|13930|5|677| +2450822|13933|5|510| +2450822|13934|5|144| +2450822|13936|5|950| +2450822|13939|5|804| +2450822|13940|5|353| +2450822|13942|5|974| +2450822|13945|5|812| +2450822|13946|5|899| +2450822|13948|5|148| +2450822|13951|5|305| +2450822|13952|5|766| +2450822|13954|5|| +2450822|13957|5|499| +2450822|13958|5|375| +2450822|13960|5|458| +2450822|13963|5|31| +2450822|13964|5|720| +2450822|13966|5|197| +2450822|13969|5|447| +2450822|13970|5|603| +2450822|13972|5|463| +2450822|13975|5|952| +2450822|13976|5|891| +2450822|13978|5|885| +2450822|13981|5|776| +2450822|13982|5|875| +2450822|13984|5|941| +2450822|13987|5|374| +2450822|13988|5|228| +2450822|13990|5|809| +2450822|13993|5|16| +2450822|13994|5|151| +2450822|13996|5|114| +2450822|13999|5|143| +2450822|14000|5|616| +2450822|14002|5|234| +2450822|14005|5|| +2450822|14006|5|53| +2450822|14008|5|415| +2450822|14011|5|899| +2450822|14012|5|682| +2450822|14014|5|719| +2450822|14017|5|514| +2450822|14018|5|28| +2450822|14020|5|828| +2450822|14023|5|480| +2450822|14024|5|144| +2450822|14026|5|841| +2450822|14029|5|322| +2450822|14030|5|| +2450822|14032|5|730| +2450822|14035|5|846| +2450822|14036|5|616| +2450822|14038|5|35| +2450822|14041|5|744| +2450822|14042|5|687| +2450822|14044|5|942| +2450822|14047|5|890| +2450822|14048|5|381| +2450822|14050|5|664| +2450822|14053|5|354| +2450822|14054|5|105| +2450822|14056|5|536| +2450822|14059|5|778| +2450822|14060|5|722| +2450822|14062|5|20| +2450822|14065|5|292| +2450822|14066|5|521| +2450822|14068|5|308| +2450822|14071|5|967| +2450822|14072|5|602| +2450822|14074|5|16| +2450822|14077|5|98| +2450822|14078|5|633| +2450822|14080|5|552| +2450822|14083|5|437| +2450822|14084|5|228| +2450822|14086|5|614| +2450822|14089|5|714| +2450822|14090|5|470| +2450822|14092|5|| +2450822|14095|5|403| +2450822|14096|5|894| +2450822|14098|5|658| +2450822|14101|5|556| +2450822|14102|5|161| +2450822|14104|5|373| +2450822|14107|5|980| +2450822|14108|5|616| +2450822|14110|5|973| +2450822|14113|5|532| +2450822|14114|5|813| +2450822|14116|5|765| +2450822|14119|5|169| +2450822|14120|5|44| +2450822|14122|5|207| +2450822|14125|5|907| +2450822|14126|5|474| +2450822|14128|5|557| +2450822|14131|5|489| +2450822|14132|5|997| +2450822|14134|5|17| +2450822|14137|5|966| +2450822|14138|5|785| +2450822|14140|5|562| +2450822|14143|5|179| +2450822|14144|5|45| +2450822|14146|5|| +2450822|14149|5|145| +2450822|14150|5|843| +2450822|14152|5|61| +2450822|14155|5|222| +2450822|14156|5|77| +2450822|14158|5|252| +2450822|14161|5|340| +2450822|14162|5|324| +2450822|14164|5|| +2450822|14167|5|298| +2450822|14168|5|65| +2450822|14170|5|967| +2450822|14173|5|994| +2450822|14174|5|131| +2450822|14176|5|749| +2450822|14179|5|983| +2450822|14180|5|964| +2450822|14182|5|624| +2450822|14185|5|999| +2450822|14186|5|754| +2450822|14188|5|562| +2450822|14191|5|389| +2450822|14192|5|808| +2450822|14194|5|593| +2450822|14197|5|137| +2450822|14198|5|665| +2450822|14200|5|212| +2450822|14203|5|664| +2450822|14204|5|813| +2450822|14206|5|149| +2450822|14209|5|959| +2450822|14210|5|74| +2450822|14212|5|600| +2450822|14215|5|788| +2450822|14216|5|| +2450822|14218|5|| +2450822|14221|5|| +2450822|14222|5|492| +2450822|14224|5|750| +2450822|14227|5|603| +2450822|14228|5|518| +2450822|14230|5|475| +2450822|14233|5|667| +2450822|14234|5|66| +2450822|14236|5|925| +2450822|14239|5|19| +2450822|14240|5|90| +2450822|14242|5|| +2450822|14245|5|571| +2450822|14246|5|382| +2450822|14248|5|145| +2450822|14251|5|742| +2450822|14252|5|474| +2450822|14254|5|830| +2450822|14257|5|878| +2450822|14258|5|844| +2450822|14260|5|72| +2450822|14263|5|636| +2450822|14264|5|144| +2450822|14266|5|236| +2450822|14269|5|765| +2450822|14270|5|583| +2450822|14272|5|618| +2450822|14275|5|615| +2450822|14276|5|843| +2450822|14278|5|105| +2450822|14281|5|885| +2450822|14282|5|167| +2450822|14284|5|217| +2450822|14287|5|846| +2450822|14288|5|807| +2450822|14290|5|7| +2450822|14293|5|863| +2450822|14294|5|334| +2450822|14296|5|877| +2450822|14299|5|651| +2450822|14300|5|901| +2450822|14302|5|818| +2450822|14305|5|689| +2450822|14306|5|351| +2450822|14308|5|851| +2450822|14311|5|971| +2450822|14312|5|329| +2450822|14314|5|907| +2450822|14317|5|769| +2450822|14318|5|910| +2450822|14320|5|656| +2450822|14323|5|949| +2450822|14324|5|853| +2450822|14326|5|| +2450822|14329|5|389| +2450822|14330|5|848| +2450822|14332|5|242| +2450822|14335|5|942| +2450822|14336|5|468| +2450822|14338|5|51| +2450822|14341|5|162| +2450822|14342|5|327| +2450822|14344|5|16| +2450822|14347|5|461| +2450822|14348|5|379| +2450822|14350|5|573| +2450822|14353|5|472| +2450822|14354|5|569| +2450822|14356|5|646| +2450822|14359|5|252| +2450822|14360|5|222| +2450822|14362|5|402| +2450822|14365|5|519| +2450822|14366|5|218| +2450822|14368|5|654| +2450822|14371|5|175| +2450822|14372|5|375| +2450822|14374|5|548| +2450822|14377|5|518| +2450822|14378|5|695| +2450822|14380|5|49| +2450822|14383|5|887| +2450822|14384|5|| +2450822|14386|5|929| +2450822|14389|5|362| +2450822|14390|5|582| +2450822|14392|5|268| +2450822|14395|5|536| +2450822|14396|5|29| +2450822|14398|5|266| +2450822|14401|5|391| +2450822|14402|5|605| +2450822|14404|5|174| +2450822|14407|5|967| +2450822|14408|5|969| +2450822|14410|5|174| +2450822|14413|5|287| +2450822|14414|5|401| +2450822|14416|5|883| +2450822|14419|5|774| +2450822|14420|5|594| +2450822|14422|5|538| +2450822|14425|5|450| +2450822|14426|5|216| +2450822|14428|5|840| +2450822|14431|5|584| +2450822|14432|5|266| +2450822|14434|5|995| +2450822|14437|5|101| +2450822|14438|5|660| +2450822|14440|5|506| +2450822|14443|5|594| +2450822|14444|5|450| +2450822|14446|5|| +2450822|14449|5|753| +2450822|14450|5|942| +2450822|14452|5|228| +2450822|14455|5|934| +2450822|14456|5|243| +2450822|14458|5|| +2450822|14461|5|60| +2450822|14462|5|| +2450822|14464|5|940| +2450822|14467|5|721| +2450822|14468|5|589| +2450822|14470|5|345| +2450822|14473|5|391| +2450822|14474|5|989| +2450822|14476|5|823| +2450822|14479|5|170| +2450822|14480|5|639| +2450822|14482|5|90| +2450822|14485|5|781| +2450822|14486|5|381| +2450822|14488|5|867| +2450822|14491|5|400| +2450822|14492|5|636| +2450822|14494|5|959| +2450822|14497|5|5| +2450822|14498|5|347| +2450822|14500|5|43| +2450822|14503|5|628| +2450822|14504|5|390| +2450822|14506|5|189| +2450822|14509|5|775| +2450822|14510|5|516| +2450822|14512|5|0| +2450822|14515|5|521| +2450822|14516|5|734| +2450822|14518|5|642| +2450822|14521|5|363| +2450822|14522|5|92| +2450822|14524|5|466| +2450822|14527|5|| +2450822|14528|5|729| +2450822|14530|5|433| +2450822|14533|5|451| +2450822|14534|5|189| +2450822|14536|5|842| +2450822|14539|5|610| +2450822|14540|5|292| +2450822|14542|5|710| +2450822|14545|5|780| +2450822|14546|5|191| +2450822|14548|5|897| +2450822|14551|5|637| +2450822|14552|5|185| +2450822|14554|5|911| +2450822|14557|5|534| +2450822|14558|5|693| +2450822|14560|5|587| +2450822|14563|5|175| +2450822|14564|5|988| +2450822|14566|5|116| +2450822|14569|5|337| +2450822|14570|5|799| +2450822|14572|5|757| +2450822|14575|5|919| +2450822|14576|5|| +2450822|14578|5|975| +2450822|14581|5|297| +2450822|14582|5|461| +2450822|14584|5|415| +2450822|14587|5|378| +2450822|14588|5|988| +2450822|14590|5|23| +2450822|14593|5|410| +2450822|14594|5|587| +2450822|14596|5|| +2450822|14599|5|508| +2450822|14600|5|401| +2450822|14602|5|818| +2450822|14605|5|276| +2450822|14606|5|315| +2450822|14608|5|212| +2450822|14611|5|349| +2450822|14612|5|590| +2450822|14614|5|53| +2450822|14617|5|760| +2450822|14618|5|4| +2450822|14620|5|288| +2450822|14623|5|176| +2450822|14624|5|512| +2450822|14626|5|732| +2450822|14629|5|36| +2450822|14630|5|532| +2450822|14632|5|347| +2450822|14635|5|860| +2450822|14636|5|645| +2450822|14638|5|386| +2450822|14641|5|742| +2450822|14642|5|336| +2450822|14644|5|779| +2450822|14647|5|403| +2450822|14648|5|138| +2450822|14650|5|267| +2450822|14653|5|732| +2450822|14654|5|905| +2450822|14656|5|459| +2450822|14659|5|832| +2450822|14660|5|98| +2450822|14662|5|647| +2450822|14665|5|58| +2450822|14666|5|614| +2450822|14668|5|960| +2450822|14671|5|936| +2450822|14672|5|392| +2450822|14674|5|401| +2450822|14677|5|687| +2450822|14678|5|340| +2450822|14680|5|498| +2450822|14683|5|820| +2450822|14684|5|189| +2450822|14686|5|235| +2450822|14689|5|607| +2450822|14690|5|| +2450822|14692|5|| +2450822|14695|5|550| +2450822|14696|5|68| +2450822|14698|5|648| +2450822|14701|5|| +2450822|14702|5|903| +2450822|14704|5|866| +2450822|14707|5|58| +2450822|14708|5|299| +2450822|14710|5|190| +2450822|14713|5|58| +2450822|14714|5|250| +2450822|14716|5|376| +2450822|14719|5|514| +2450822|14720|5|338| +2450822|14722|5|606| +2450822|14725|5|614| +2450822|14726|5|362| +2450822|14728|5|262| +2450822|14731|5|970| +2450822|14732|5|853| +2450822|14734|5|506| +2450822|14737|5|64| +2450822|14738|5|792| +2450822|14740|5|69| +2450822|14743|5|641| +2450822|14744|5|325| +2450822|14746|5|918| +2450822|14749|5|146| +2450822|14750|5|207| +2450822|14752|5|663| +2450822|14755|5|677| +2450822|14756|5|673| +2450822|14758|5|756| +2450822|14761|5|447| +2450822|14762|5|978| +2450822|14764|5|162| +2450822|14767|5|1000| +2450822|14768|5|12| +2450822|14770|5|| +2450822|14773|5|813| +2450822|14774|5|226| +2450822|14776|5|640| +2450822|14779|5|389| +2450822|14780|5|758| +2450822|14782|5|822| +2450822|14785|5|544| +2450822|14786|5|653| +2450822|14788|5|800| +2450822|14791|5|810| +2450822|14792|5|269| +2450822|14794|5|658| +2450822|14797|5|972| +2450822|14798|5|226| +2450822|14800|5|536| +2450822|14803|5|319| +2450822|14804|5|227| +2450822|14806|5|936| +2450822|14809|5|563| +2450822|14810|5|| +2450822|14812|5|929| +2450822|14815|5|393| +2450822|14816|5|15| +2450822|14818|5|376| +2450822|14821|5|888| +2450822|14822|5|599| +2450822|14824|5|924| +2450822|14827|5|986| +2450822|14828|5|904| +2450822|14830|5|249| +2450822|14833|5|479| +2450822|14834|5|373| +2450822|14836|5|836| +2450822|14839|5|380| +2450822|14840|5|835| +2450822|14842|5|981| +2450822|14845|5|198| +2450822|14846|5|647| +2450822|14848|5|| +2450822|14851|5|574| +2450822|14852|5|607| +2450822|14854|5|652| +2450822|14857|5|| +2450822|14858|5|422| +2450822|14860|5|617| +2450822|14863|5|103| +2450822|14864|5|533| +2450822|14866|5|724| +2450822|14869|5|904| +2450822|14870|5|| +2450822|14872|5|| +2450822|14875|5|809| +2450822|14876|5|22| +2450822|14878|5|563| +2450822|14881|5|945| +2450822|14882|5|261| +2450822|14884|5|189| +2450822|14887|5|942| +2450822|14888|5|759| +2450822|14890|5|475| +2450822|14893|5|771| +2450822|14894|5|749| +2450822|14896|5|47| +2450822|14899|5|796| +2450822|14900|5|800| +2450822|14902|5|58| +2450822|14905|5|633| +2450822|14906|5|211| +2450822|14908|5|931| +2450822|14911|5|141| +2450822|14912|5|203| +2450822|14914|5|670| +2450822|14917|5|454| +2450822|14918|5|185| +2450822|14920|5|897| +2450822|14923|5|159| +2450822|14924|5|497| +2450822|14926|5|171| +2450822|14929|5|285| +2450822|14930|5|88| +2450822|14932|5|494| +2450822|14935|5|246| +2450822|14936|5|645| +2450822|14938|5|445| +2450822|14941|5|83| +2450822|14942|5|114| +2450822|14944|5|352| +2450822|14947|5|141| +2450822|14948|5|857| +2450822|14950|5|513| +2450822|14953|5|923| +2450822|14954|5|| +2450822|14956|5|2| +2450822|14959|5|539| +2450822|14960|5|556| +2450822|14962|5|605| +2450822|14965|5|571| +2450822|14966|5|| +2450822|14968|5|640| +2450822|14971|5|71| +2450822|14972|5|490| +2450822|14974|5|858| +2450822|14977|5|785| +2450822|14978|5|416| +2450822|14980|5|143| +2450822|14983|5|558| +2450822|14984|5|362| +2450822|14986|5|180| +2450822|14989|5|179| +2450822|14990|5|410| +2450822|14992|5|185| +2450822|14995|5|725| +2450822|14996|5|208| +2450822|14998|5|418| +2450822|15001|5|444| +2450822|15002|5|41| +2450822|15004|5|40| +2450822|15007|5|472| +2450822|15008|5|496| +2450822|15010|5|378| +2450822|15013|5|326| +2450822|15014|5|819| +2450822|15016|5|933| +2450822|15019|5|560| +2450822|15020|5|773| +2450822|15022|5|505| +2450822|15025|5|208| +2450822|15026|5|626| +2450822|15028|5|| +2450822|15031|5|155| +2450822|15032|5|314| +2450822|15034|5|889| +2450822|15037|5|61| +2450822|15038|5|492| +2450822|15040|5|462| +2450822|15043|5|| +2450822|15044|5|684| +2450822|15046|5|338| +2450822|15049|5|8| +2450822|15050|5|924| +2450822|15052|5|366| +2450822|15055|5|233| +2450822|15056|5|894| +2450822|15058|5|17| +2450822|15061|5|465| +2450822|15062|5|523| +2450822|15064|5|138| +2450822|15067|5|886| +2450822|15068|5|60| +2450822|15070|5|590| +2450822|15073|5|368| +2450822|15074|5|916| +2450822|15076|5|627| +2450822|15079|5|515| +2450822|15080|5|38| +2450822|15082|5|453| +2450822|15085|5|741| +2450822|15086|5|703| +2450822|15088|5|548| +2450822|15091|5|129| +2450822|15092|5|564| +2450822|15094|5|728| +2450822|15097|5|614| +2450822|15098|5|962| +2450822|15100|5|854| +2450822|15103|5|920| +2450822|15104|5|354| +2450822|15106|5|359| +2450822|15109|5|804| +2450822|15110|5|674| +2450822|15112|5|568| +2450822|15115|5|447| +2450822|15116|5|135| +2450822|15118|5|398| +2450822|15121|5|889| +2450822|15122|5|861| +2450822|15124|5|45| +2450822|15127|5|800| +2450822|15128|5|389| +2450822|15130|5|450| +2450822|15133|5|878| +2450822|15134|5|226| +2450822|15136|5|580| +2450822|15139|5|802| +2450822|15140|5|98| +2450822|15142|5|415| +2450822|15145|5|298| +2450822|15146|5|815| +2450822|15148|5|| +2450822|15151|5|876| +2450822|15152|5|559| +2450822|15154|5|354| +2450822|15157|5|401| +2450822|15158|5|917| +2450822|15160|5|652| +2450822|15163|5|454| +2450822|15164|5|608| +2450822|15166|5|416| +2450822|15169|5|320| +2450822|15170|5|792| +2450822|15172|5|631| +2450822|15175|5|395| +2450822|15176|5|629| +2450822|15178|5|33| +2450822|15181|5|809| +2450822|15182|5|245| +2450822|15184|5|593| +2450822|15187|5|821| +2450822|15188|5|383| +2450822|15190|5|214| +2450822|15193|5|546| +2450822|15194|5|331| +2450822|15196|5|527| +2450822|15199|5|104| +2450822|15200|5|173| +2450822|15202|5|382| +2450822|15205|5|988| +2450822|15206|5|201| +2450822|15208|5|553| +2450822|15211|5|459| +2450822|15212|5|301| +2450822|15214|5|415| +2450822|15217|5|612| +2450822|15218|5|269| +2450822|15220|5|917| +2450822|15223|5|620| +2450822|15224|5|232| +2450822|15226|5|135| +2450822|15229|5|557| +2450822|15230|5|124| +2450822|15232|5|988| +2450822|15235|5|480| +2450822|15236|5|387| +2450822|15238|5|790| +2450822|15241|5|169| +2450822|15242|5|615| +2450822|15244|5|419| +2450822|15247|5|325| +2450822|15248|5|815| +2450822|15250|5|752| +2450822|15253|5|562| +2450822|15254|5|298| +2450822|15256|5|633| +2450822|15259|5|479| +2450822|15260|5|| +2450822|15262|5|370| +2450822|15265|5|119| +2450822|15266|5|369| +2450822|15268|5|693| +2450822|15271|5|191| +2450822|15272|5|319| +2450822|15274|5|213| +2450822|15277|5|115| +2450822|15278|5|632| +2450822|15280|5|| +2450822|15283|5|942| +2450822|15284|5|596| +2450822|15286|5|461| +2450822|15289|5|290| +2450822|15290|5|299| +2450822|15292|5|301| +2450822|15295|5|277| +2450822|15296|5|846| +2450822|15298|5|| +2450822|15301|5|454| +2450822|15302|5|972| +2450822|15304|5|540| +2450822|15307|5|692| +2450822|15308|5|278| +2450822|15310|5|395| +2450822|15313|5|108| +2450822|15314|5|8| +2450822|15316|5|572| +2450822|15319|5|578| +2450822|15320|5|503| +2450822|15322|5|95| +2450822|15325|5|319| +2450822|15326|5|657| +2450822|15328|5|378| +2450822|15331|5|196| +2450822|15332|5|965| +2450822|15334|5|355| +2450822|15337|5|267| +2450822|15338|5|613| +2450822|15340|5|890| +2450822|15343|5|398| +2450822|15344|5|331| +2450822|15346|5|291| +2450822|15349|5|263| +2450822|15350|5|60| +2450822|15352|5|374| +2450822|15355|5|| +2450822|15356|5|590| +2450822|15358|5|124| +2450822|15361|5|313| +2450822|15362|5|815| +2450822|15364|5|632| +2450822|15367|5|102| +2450822|15368|5|185| +2450822|15370|5|966| +2450822|15373|5|655| +2450822|15374|5|912| +2450822|15376|5|588| +2450822|15379|5|173| +2450822|15380|5|731| +2450822|15382|5|700| +2450822|15385|5|872| +2450822|15386|5|203| +2450822|15388|5|151| +2450822|15391|5|769| +2450822|15392|5|571| +2450822|15394|5|874| +2450822|15397|5|96| +2450822|15398|5|316| +2450822|15400|5|142| +2450822|15403|5|18| +2450822|15404|5|810| +2450822|15406|5|98| +2450822|15409|5|707| +2450822|15410|5|428| +2450822|15412|5|951| +2450822|15415|5|572| +2450822|15416|5|413| +2450822|15418|5|447| +2450822|15421|5|845| +2450822|15422|5|827| +2450822|15424|5|345| +2450822|15427|5|504| +2450822|15428|5|613| +2450822|15430|5|18| +2450822|15433|5|2| +2450822|15434|5|742| +2450822|15436|5|742| +2450822|15439|5|136| +2450822|15440|5|577| +2450822|15442|5|897| +2450822|15445|5|137| +2450822|15446|5|38| +2450822|15448|5|680| +2450822|15451|5|226| +2450822|15452|5|742| +2450822|15454|5|| +2450822|15457|5|213| +2450822|15458|5|91| +2450822|15460|5|337| +2450822|15463|5|353| +2450822|15464|5|443| +2450822|15466|5|14| +2450822|15469|5|174| +2450822|15470|5|779| +2450822|15472|5|198| +2450822|15475|5|705| +2450822|15476|5|216| +2450822|15478|5|983| +2450822|15481|5|474| +2450822|15482|5|655| +2450822|15484|5|245| +2450822|15487|5|876| +2450822|15488|5|974| +2450822|15490|5|836| +2450822|15493|5|624| +2450822|15494|5|363| +2450822|15496|5|572| +2450822|15499|5|266| +2450822|15500|5|119| +2450822|15502|5|533| +2450822|15505|5|632| +2450822|15506|5|459| +2450822|15508|5|897| +2450822|15511|5|107| +2450822|15512|5|283| +2450822|15514|5|721| +2450822|15517|5|152| +2450822|15518|5|443| +2450822|15520|5|260| +2450822|15523|5|| +2450822|15524|5|473| +2450822|15526|5|368| +2450822|15529|5|86| +2450822|15530|5|271| +2450822|15532|5|536| +2450822|15535|5|22| +2450822|15536|5|225| +2450822|15538|5|518| +2450822|15541|5|500| +2450822|15542|5|892| +2450822|15544|5|803| +2450822|15547|5|453| +2450822|15548|5|658| +2450822|15550|5|383| +2450822|15553|5|953| +2450822|15554|5|937| +2450822|15556|5|| +2450822|15559|5|215| +2450822|15560|5|62| +2450822|15562|5|640| +2450822|15565|5|597| +2450822|15566|5|187| +2450822|15568|5|566| +2450822|15571|5|600| +2450822|15572|5|730| +2450822|15574|5|429| +2450822|15577|5|325| +2450822|15578|5|531| +2450822|15580|5|325| +2450822|15583|5|79| +2450822|15584|5|813| +2450822|15586|5|175| +2450822|15589|5|| +2450822|15590|5|713| +2450822|15592|5|915| +2450822|15595|5|724| +2450822|15596|5|47| +2450822|15598|5|| +2450822|15601|5|123| +2450822|15602|5|533| +2450822|15604|5|948| +2450822|15607|5|425| +2450822|15608|5|112| +2450822|15610|5|184| +2450822|15613|5|153| +2450822|15614|5|853| +2450822|15616|5|765| +2450822|15619|5|135| +2450822|15620|5|618| +2450822|15622|5|564| +2450822|15625|5|736| +2450822|15626|5|706| +2450822|15628|5|996| +2450822|15631|5|414| +2450822|15632|5|915| +2450822|15634|5|874| +2450822|15637|5|316| +2450822|15638|5|107| +2450822|15640|5|495| +2450822|15643|5|894| +2450822|15644|5|847| +2450822|15646|5|569| +2450822|15649|5|167| +2450822|15650|5|916| +2450822|15652|5|556| +2450822|15655|5|837| +2450822|15656|5|463| +2450822|15658|5|204| +2450822|15661|5|243| +2450822|15662|5|335| +2450822|15664|5|432| +2450822|15667|5|575| +2450822|15668|5|704| +2450822|15670|5|119| +2450822|15673|5|300| +2450822|15674|5|823| +2450822|15676|5|391| +2450822|15679|5|50| +2450822|15680|5|29| +2450822|15682|5|476| +2450822|15685|5|504| +2450822|15686|5|417| +2450822|15688|5|133| +2450822|15691|5|462| +2450822|15692|5|381| +2450822|15694|5|898| +2450822|15697|5|761| +2450822|15698|5|200| +2450822|15700|5|567| +2450822|15703|5|404| +2450822|15704|5|| +2450822|15706|5|82| +2450822|15709|5|309| +2450822|15710|5|856| +2450822|15712|5|127| +2450822|15715|5|268| +2450822|15716|5|762| +2450822|15718|5|639| +2450822|15721|5|123| +2450822|15722|5|789| +2450822|15724|5|870| +2450822|15727|5|| +2450822|15728|5|930| +2450822|15730|5|316| +2450822|15733|5|696| +2450822|15734|5|891| +2450822|15736|5|691| +2450822|15739|5|563| +2450822|15740|5|809| +2450822|15742|5|637| +2450822|15745|5|52| +2450822|15746|5|433| +2450822|15748|5|983| +2450822|15751|5|94| +2450822|15752|5|836| +2450822|15754|5|40| +2450822|15757|5|268| +2450822|15758|5|529| +2450822|15760|5|283| +2450822|15763|5|955| +2450822|15764|5|173| +2450822|15766|5|554| +2450822|15769|5|282| +2450822|15770|5|316| +2450822|15772|5|890| +2450822|15775|5|198| +2450822|15776|5|411| +2450822|15778|5|390| +2450822|15781|5|450| +2450822|15782|5|245| +2450822|15784|5|749| +2450822|15787|5|445| +2450822|15788|5|673| +2450822|15790|5|521| +2450822|15793|5|411| +2450822|15794|5|73| +2450822|15796|5|363| +2450822|15799|5|249| +2450822|15800|5|50| +2450822|15802|5|245| +2450822|15805|5|638| +2450822|15806|5|659| +2450822|15808|5|942| +2450822|15811|5|107| +2450822|15812|5|631| +2450822|15814|5|36| +2450822|15817|5|854| +2450822|15818|5|| +2450822|15820|5|| +2450822|15823|5|411| +2450822|15824|5|843| +2450822|15826|5|397| +2450822|15829|5|806| +2450822|15830|5|960| +2450822|15832|5|93| +2450822|15835|5|963| +2450822|15836|5|160| +2450822|15838|5|929| +2450822|15841|5|321| +2450822|15842|5|331| +2450822|15844|5|135| +2450822|15847|5|676| +2450822|15848|5|539| +2450822|15850|5|469| +2450822|15853|5|190| +2450822|15854|5|810| +2450822|15856|5|819| +2450822|15859|5|843| +2450822|15860|5|591| +2450822|15862|5|150| +2450822|15865|5|288| +2450822|15866|5|755| +2450822|15868|5|202| +2450822|15871|5|249| +2450822|15872|5|152| +2450822|15874|5|| +2450822|15877|5|588| +2450822|15878|5|69| +2450822|15880|5|34| +2450822|15883|5|948| +2450822|15884|5|545| +2450822|15886|5|959| +2450822|15889|5|85| +2450822|15890|5|687| +2450822|15892|5|23| +2450822|15895|5|| +2450822|15896|5|49| +2450822|15898|5|428| +2450822|15901|5|832| +2450822|15902|5|109| +2450822|15904|5|572| +2450822|15907|5|784| +2450822|15908|5|967| +2450822|15910|5|249| +2450822|15913|5|175| +2450822|15914|5|568| +2450822|15916|5|| +2450822|15919|5|105| +2450822|15920|5|67| +2450822|15922|5|342| +2450822|15925|5|136| +2450822|15926|5|639| +2450822|15928|5|517| +2450822|15931|5|306| +2450822|15932|5|447| +2450822|15934|5|657| +2450822|15937|5|657| +2450822|15938|5|913| +2450822|15940|5|| +2450822|15943|5|283| +2450822|15944|5|483| +2450822|15946|5|897| +2450822|15949|5|523| +2450822|15950|5|368| +2450822|15952|5|477| +2450822|15955|5|854| +2450822|15956|5|976| +2450822|15958|5|148| +2450822|15961|5|3| +2450822|15962|5|| +2450822|15964|5|440| +2450822|15967|5|720| +2450822|15968|5|925| +2450822|15970|5|841| +2450822|15973|5|284| +2450822|15974|5|181| +2450822|15976|5|809| +2450822|15979|5|| +2450822|15980|5|449| +2450822|15982|5|221| +2450822|15985|5|719| +2450822|15986|5|170| +2450822|15988|5|872| +2450822|15991|5|399| +2450822|15992|5|910| +2450822|15994|5|819| +2450822|15997|5|909| +2450822|15998|5|104| +2450822|16000|5|745| +2450822|16003|5|206| +2450822|16004|5|840| +2450822|16006|5|485| +2450822|16009|5|718| +2450822|16010|5|523| +2450822|16012|5|457| +2450822|16015|5|369| +2450822|16016|5|290| +2450822|16018|5|86| +2450822|16021|5|10| +2450822|16022|5|386| +2450822|16024|5|372| +2450822|16027|5|97| +2450822|16028|5|902| +2450822|16030|5|274| +2450822|16033|5|211| +2450822|16034|5|900| +2450822|16036|5|193| +2450822|16039|5|299| +2450822|16040|5|925| +2450822|16042|5|310| +2450822|16045|5|791| +2450822|16046|5|525| +2450822|16048|5|537| +2450822|16051|5|213| +2450822|16052|5|31| +2450822|16054|5|596| +2450822|16057|5|453| +2450822|16058|5|508| +2450822|16060|5|484| +2450822|16063|5|| +2450822|16064|5|259| +2450822|16066|5|462| +2450822|16069|5|224| +2450822|16070|5|716| +2450822|16072|5|658| +2450822|16075|5|81| +2450822|16076|5|892| +2450822|16078|5|| +2450822|16081|5|29| +2450822|16082|5|77| +2450822|16084|5|757| +2450822|16087|5|915| +2450822|16088|5|800| +2450822|16090|5|689| +2450822|16093|5|938| +2450822|16094|5|144| +2450822|16096|5|165| +2450822|16099|5|222| +2450822|16100|5|51| +2450822|16102|5|988| +2450822|16105|5|675| +2450822|16106|5|360| +2450822|16108|5|867| +2450822|16111|5|54| +2450822|16112|5|271| +2450822|16114|5|632| +2450822|16117|5|110| +2450822|16118|5|| +2450822|16120|5|466| +2450822|16123|5|432| +2450822|16124|5|619| +2450822|16126|5|| +2450822|16129|5|546| +2450822|16130|5|447| +2450822|16132|5|889| +2450822|16135|5|964| +2450822|16136|5|892| +2450822|16138|5|486| +2450822|16141|5|880| +2450822|16142|5|783| +2450822|16144|5|610| +2450822|16147|5|60| +2450822|16148|5|231| +2450822|16150|5|890| +2450822|16153|5|318| +2450822|16154|5|260| +2450822|16156|5|588| +2450822|16159|5|213| +2450822|16160|5|173| +2450822|16162|5|742| +2450822|16165|5|945| +2450822|16166|5|364| +2450822|16168|5|881| +2450822|16171|5|912| +2450822|16172|5|290| +2450822|16174|5|621| +2450822|16177|5|17| +2450822|16178|5|630| +2450822|16180|5|170| +2450822|16183|5|244| +2450822|16184|5|871| +2450822|16186|5|617| +2450822|16189|5|599| +2450822|16190|5|259| +2450822|16192|5|632| +2450822|16195|5|397| +2450822|16196|5|545| +2450822|16198|5|55| +2450822|16201|5|841| +2450822|16202|5|220| +2450822|16204|5|847| +2450822|16207|5|109| +2450822|16208|5|801| +2450822|16210|5|42| +2450822|16213|5|377| +2450822|16214|5|773| +2450822|16216|5|553| +2450822|16219|5|120| +2450822|16220|5|799| +2450822|16222|5|815| +2450822|16225|5|633| +2450822|16226|5|617| +2450822|16228|5|368| +2450822|16231|5|687| +2450822|16232|5|805| +2450822|16234|5|581| +2450822|16237|5|610| +2450822|16238|5|982| +2450822|16240|5|248| +2450822|16243|5|730| +2450822|16244|5|49| +2450822|16246|5|922| +2450822|16249|5|101| +2450822|16250|5|287| +2450822|16252|5|828| +2450822|16255|5|29| +2450822|16256|5|114| +2450822|16258|5|567| +2450822|16261|5|| +2450822|16262|5|51| +2450822|16264|5|233| +2450822|16267|5|42| +2450822|16268|5|317| +2450822|16270|5|948| +2450822|16273|5|953| +2450822|16274|5|512| +2450822|16276|5|302| +2450822|16279|5|325| +2450822|16280|5|340| +2450822|16282|5|125| +2450822|16285|5|| +2450822|16286|5|959| +2450822|16288|5|168| +2450822|16291|5|76| +2450822|16292|5|| +2450822|16294|5|| +2450822|16297|5|479| +2450822|16298|5|529| +2450822|16300|5|710| +2450822|16303|5|317| +2450822|16304|5|46| +2450822|16306|5|775| +2450822|16309|5|420| +2450822|16310|5|434| +2450822|16312|5|992| +2450822|16315|5|62| +2450822|16316|5|345| +2450822|16318|5|849| +2450822|16321|5|408| +2450822|16322|5|5| +2450822|16324|5|994| +2450822|16327|5|893| +2450822|16328|5|849| +2450822|16330|5|922| +2450822|16333|5|709| +2450822|16334|5|407| +2450822|16336|5|692| +2450822|16339|5|320| +2450822|16340|5|163| +2450822|16342|5|433| +2450822|16345|5|645| +2450822|16346|5|956| +2450822|16348|5|447| +2450822|16351|5|177| +2450822|16352|5|706| +2450822|16354|5|204| +2450822|16357|5|816| +2450822|16358|5|1| +2450822|16360|5|845| +2450822|16363|5|214| +2450822|16364|5|309| +2450822|16366|5|433| +2450822|16369|5|768| +2450822|16370|5|127| +2450822|16372|5|885| +2450822|16375|5|684| +2450822|16376|5|893| +2450822|16378|5|109| +2450822|16381|5|723| +2450822|16382|5|578| +2450822|16384|5|932| +2450822|16387|5|95| +2450822|16388|5|786| +2450822|16390|5|891| +2450822|16393|5|775| +2450822|16394|5|981| +2450822|16396|5|14| +2450822|16399|5|421| +2450822|16400|5|53| +2450822|16402|5|714| +2450822|16405|5|8| +2450822|16406|5|907| +2450822|16408|5|206| +2450822|16411|5|733| +2450822|16412|5|597| +2450822|16414|5|184| +2450822|16417|5|262| +2450822|16418|5|89| +2450822|16420|5|59| +2450822|16423|5|298| +2450822|16424|5|730| +2450822|16426|5|457| +2450822|16429|5|652| +2450822|16430|5|932| +2450822|16432|5|181| +2450822|16435|5|775| +2450822|16436|5|74| +2450822|16438|5|64| +2450822|16441|5|298| +2450822|16442|5|261| +2450822|16444|5|824| +2450822|16447|5|486| +2450822|16448|5|859| +2450822|16450|5|263| +2450822|16453|5|220| +2450822|16454|5|856| +2450822|16456|5|967| +2450822|16459|5|160| +2450822|16460|5|395| +2450822|16462|5|925| +2450822|16465|5|449| +2450822|16466|5|692| +2450822|16468|5|| +2450822|16471|5|471| +2450822|16472|5|521| +2450822|16474|5|916| +2450822|16477|5|261| +2450822|16478|5|77| +2450822|16480|5|405| +2450822|16483|5|931| +2450822|16484|5|342| +2450822|16486|5|616| +2450822|16489|5|548| +2450822|16490|5|383| +2450822|16492|5|89| +2450822|16495|5|237| +2450822|16496|5|334| +2450822|16498|5|241| +2450822|16501|5|607| +2450822|16502|5|561| +2450822|16504|5|810| +2450822|16507|5|514| +2450822|16508|5|658| +2450822|16510|5|877| +2450822|16513|5|664| +2450822|16514|5|212| +2450822|16516|5|20| +2450822|16519|5|942| +2450822|16520|5|84| +2450822|16522|5|137| +2450822|16525|5|537| +2450822|16526|5|435| +2450822|16528|5|686| +2450822|16531|5|912| +2450822|16532|5|361| +2450822|16534|5|493| +2450822|16537|5|365| +2450822|16538|5|660| +2450822|16540|5|| +2450822|16543|5|875| +2450822|16544|5|954| +2450822|16546|5|650| +2450822|16549|5|478| +2450822|16550|5|343| +2450822|16552|5|229| +2450822|16555|5|756| +2450822|16556|5|674| +2450822|16558|5|706| +2450822|16561|5|329| +2450822|16562|5|602| +2450822|16564|5|464| +2450822|16567|5|202| +2450822|16568|5|908| +2450822|16570|5|915| +2450822|16573|5|991| +2450822|16574|5|2| +2450822|16576|5|904| +2450822|16579|5|26| +2450822|16580|5|248| +2450822|16582|5|173| +2450822|16585|5|186| +2450822|16586|5|960| +2450822|16588|5|97| +2450822|16591|5|474| +2450822|16592|5|879| +2450822|16594|5|907| +2450822|16597|5|387| +2450822|16598|5|591| +2450822|16600|5|212| +2450822|16603|5|104| +2450822|16604|5|| +2450822|16606|5|806| +2450822|16609|5|999| +2450822|16610|5|483| +2450822|16612|5|337| +2450822|16615|5|494| +2450822|16616|5|| +2450822|16618|5|485| +2450822|16621|5|825| +2450822|16622|5|| +2450822|16624|5|| +2450822|16627|5|| +2450822|16628|5|563| +2450822|16630|5|713| +2450822|16633|5|142| +2450822|16634|5|289| +2450822|16636|5|989| +2450822|16639|5|588| +2450822|16640|5|155| +2450822|16642|5|718| +2450822|16645|5|859| +2450822|16646|5|848| +2450822|16648|5|533| +2450822|16651|5|912| +2450822|16652|5|761| +2450822|16654|5|679| +2450822|16657|5|519| +2450822|16658|5|50| +2450822|16660|5|579| +2450822|16663|5|34| +2450822|16664|5|279| +2450822|16666|5|864| +2450822|16669|5|581| +2450822|16670|5|879| +2450822|16672|5|945| +2450822|16675|5|895| +2450822|16676|5|676| +2450822|16678|5|591| +2450822|16681|5|29| +2450822|16682|5|62| +2450822|16684|5|474| +2450822|16687|5|101| +2450822|16688|5|603| +2450822|16690|5|634| +2450822|16693|5|92| +2450822|16694|5|856| +2450822|16696|5|842| +2450822|16699|5|239| +2450822|16700|5|402| +2450822|16702|5|924| +2450822|16705|5|929| +2450822|16706|5|| +2450822|16708|5|691| +2450822|16711|5|768| +2450822|16712|5|217| +2450822|16714|5|935| +2450822|16717|5|795| +2450822|16718|5|339| +2450822|16720|5|962| +2450822|16723|5|708| +2450822|16724|5|743| +2450822|16726|5|188| +2450822|16729|5|177| +2450822|16730|5|781| +2450822|16732|5|806| +2450822|16735|5|502| +2450822|16736|5|978| +2450822|16738|5|575| +2450822|16741|5|616| +2450822|16742|5|227| +2450822|16744|5|730| +2450822|16747|5|811| +2450822|16748|5|332| +2450822|16750|5|653| +2450822|16753|5|894| +2450822|16754|5|442| +2450822|16756|5|752| +2450822|16759|5|452| +2450822|16760|5|701| +2450822|16762|5|38| +2450822|16765|5|827| +2450822|16766|5|803| +2450822|16768|5|875| +2450822|16771|5|632| +2450822|16772|5|31| +2450822|16774|5|18| +2450822|16777|5|635| +2450822|16778|5|177| +2450822|16780|5|332| +2450822|16783|5|691| +2450822|16784|5|71| +2450822|16786|5|668| +2450822|16789|5|950| +2450822|16790|5|703| +2450822|16792|5|436| +2450822|16795|5|738| +2450822|16796|5|| +2450822|16798|5|738| +2450822|16801|5|247| +2450822|16802|5|553| +2450822|16804|5|175| +2450822|16807|5|40| +2450822|16808|5|267| +2450822|16810|5|669| +2450822|16813|5|765| +2450822|16814|5|910| +2450822|16816|5|272| +2450822|16819|5|189| +2450822|16820|5|890| +2450822|16822|5|76| +2450822|16825|5|836| +2450822|16826|5|121| +2450822|16828|5|46| +2450822|16831|5|415| +2450822|16832|5|316| +2450822|16834|5|43| +2450822|16837|5|157| +2450822|16838|5|361| +2450822|16840|5|136| +2450822|16843|5|328| +2450822|16844|5|925| +2450822|16846|5|117| +2450822|16849|5|940| +2450822|16850|5|829| +2450822|16852|5|107| +2450822|16855|5|832| +2450822|16856|5|765| +2450822|16858|5|425| +2450822|16861|5|33| +2450822|16862|5|275| +2450822|16864|5|656| +2450822|16867|5|8| +2450822|16868|5|| +2450822|16870|5|1| +2450822|16873|5|501| +2450822|16874|5|715| +2450822|16876|5|660| +2450822|16879|5|554| +2450822|16880|5|320| +2450822|16882|5|413| +2450822|16885|5|237| +2450822|16886|5|870| +2450822|16888|5|247| +2450822|16891|5|688| +2450822|16892|5|914| +2450822|16894|5|340| +2450822|16897|5|| +2450822|16898|5|214| +2450822|16900|5|145| +2450822|16903|5|234| +2450822|16904|5|948| +2450822|16906|5|623| +2450822|16909|5|49| +2450822|16910|5|129| +2450822|16912|5|981| +2450822|16915|5|762| +2450822|16916|5|852| +2450822|16918|5|239| +2450822|16921|5|377| +2450822|16922|5|937| +2450822|16924|5|825| +2450822|16927|5|948| +2450822|16928|5|303| +2450822|16930|5|126| +2450822|16933|5|742| +2450822|16934|5|435| +2450822|16936|5|524| +2450822|16939|5|694| +2450822|16940|5|| +2450822|16942|5|737| +2450822|16945|5|361| +2450822|16946|5|686| +2450822|16948|5|605| +2450822|16951|5|935| +2450822|16952|5|500| +2450822|16954|5|987| +2450822|16957|5|646| +2450822|16958|5|323| +2450822|16960|5|895| +2450822|16963|5|710| +2450822|16964|5|267| +2450822|16966|5|629| +2450822|16969|5|830| +2450822|16970|5|171| +2450822|16972|5|142| +2450822|16975|5|320| +2450822|16976|5|249| +2450822|16978|5|156| +2450822|16981|5|689| +2450822|16982|5|374| +2450822|16984|5|145| +2450822|16987|5|883| +2450822|16988|5|851| +2450822|16990|5|708| +2450822|16993|5|638| +2450822|16994|5|265| +2450822|16996|5|439| +2450822|16999|5|85| +2450822|17000|5|78| +2450822|17002|5|435| +2450822|17005|5|315| +2450822|17006|5|263| +2450822|17008|5|| +2450822|17011|5|363| +2450822|17012|5|| +2450822|17014|5|487| +2450822|17017|5|591| +2450822|17018|5|692| +2450822|17020|5|188| +2450822|17023|5|63| +2450822|17024|5|| +2450822|17026|5|458| +2450822|17029|5|514| +2450822|17030|5|813| +2450822|17032|5|863| +2450822|17035|5|990| +2450822|17036|5|670| +2450822|17038|5|431| +2450822|17041|5|728| +2450822|17042|5|796| +2450822|17044|5|| +2450822|17047|5|152| +2450822|17048|5|| +2450822|17050|5|305| +2450822|17053|5|| +2450822|17054|5|31| +2450822|17056|5|539| +2450822|17059|5|87| +2450822|17060|5|443| +2450822|17062|5|837| +2450822|17065|5|970| +2450822|17066|5|818| +2450822|17068|5|491| +2450822|17071|5|522| +2450822|17072|5|| +2450822|17074|5|187| +2450822|17077|5|86| +2450822|17078|5|597| +2450822|17080|5|865| +2450822|17083|5|856| +2450822|17084|5|985| +2450822|17086|5|456| +2450822|17089|5|82| +2450822|17090|5|601| +2450822|17092|5|725| +2450822|17095|5|337| +2450822|17096|5|321| +2450822|17098|5|384| +2450822|17101|5|165| +2450822|17102|5|668| +2450822|17104|5|201| +2450822|17107|5|586| +2450822|17108|5|| +2450822|17110|5|964| +2450822|17113|5|799| +2450822|17114|5|315| +2450822|17116|5|885| +2450822|17119|5|29| +2450822|17120|5|371| +2450822|17122|5|774| +2450822|17125|5|68| +2450822|17126|5|305| +2450822|17128|5|803| +2450822|17131|5|| +2450822|17132|5|147| +2450822|17134|5|348| +2450822|17137|5|281| +2450822|17138|5|524| +2450822|17140|5|50| +2450822|17143|5|308| +2450822|17144|5|439| +2450822|17146|5|237| +2450822|17149|5|20| +2450822|17150|5|830| +2450822|17152|5|988| +2450822|17155|5|380| +2450822|17156|5|903| +2450822|17158|5|735| +2450822|17161|5|820| +2450822|17162|5|166| +2450822|17164|5|862| +2450822|17167|5|206| +2450822|17168|5|140| +2450822|17170|5|304| +2450822|17173|5|482| +2450822|17174|5|862| +2450822|17176|5|395| +2450822|17179|5|962| +2450822|17180|5|114| +2450822|17182|5|620| +2450822|17185|5|168| +2450822|17186|5|250| +2450822|17188|5|401| +2450822|17191|5|138| +2450822|17192|5|522| +2450822|17194|5|302| +2450822|17197|5|| +2450822|17198|5|786| +2450822|17200|5|718| +2450822|17203|5|287| +2450822|17204|5|308| +2450822|17206|5|8| +2450822|17209|5|135| +2450822|17210|5|276| +2450822|17212|5|979| +2450822|17215|5|418| +2450822|17216|5|559| +2450822|17218|5|386| +2450822|17221|5|61| +2450822|17222|5|474| +2450822|17224|5|629| +2450822|17227|5|812| +2450822|17228|5|105| +2450822|17230|5|633| +2450822|17233|5|265| +2450822|17234|5|816| +2450822|17236|5|641| +2450822|17239|5|6| +2450822|17240|5|789| +2450822|17242|5|247| +2450822|17245|5|548| +2450822|17246|5|115| +2450822|17248|5|673| +2450822|17251|5|89| +2450822|17252|5|998| +2450822|17254|5|754| +2450822|17257|5|827| +2450822|17258|5|185| +2450822|17260|5|112| +2450822|17263|5|462| +2450822|17264|5|585| +2450822|17266|5|52| +2450822|17269|5|216| +2450822|17270|5|800| +2450822|17272|5|433| +2450822|17275|5|895| +2450822|17276|5|7| +2450822|17278|5|353| +2450822|17281|5|746| +2450822|17282|5|892| +2450822|17284|5|458| +2450822|17287|5|122| +2450822|17288|5|317| +2450822|17290|5|904| +2450822|17293|5|537| +2450822|17294|5|369| +2450822|17296|5|187| +2450822|17299|5|697| +2450822|17300|5|243| +2450822|17302|5|625| +2450822|17305|5|828| +2450822|17306|5|598| +2450822|17308|5|128| +2450822|17311|5|898| +2450822|17312|5|121| +2450822|17314|5|134| +2450822|17317|5|712| +2450822|17318|5|63| +2450822|17320|5|85| +2450822|17323|5|523| +2450822|17324|5|757| +2450822|17326|5|571| +2450822|17329|5|840| +2450822|17330|5|616| +2450822|17332|5|417| +2450822|17335|5|383| +2450822|17336|5|116| +2450822|17338|5|236| +2450822|17341|5|| +2450822|17342|5|919| +2450822|17344|5|| +2450822|17347|5|34| +2450822|17348|5|633| +2450822|17350|5|595| +2450822|17353|5|671| +2450822|17354|5|599| +2450822|17356|5|811| +2450822|17359|5|214| +2450822|17360|5|320| +2450822|17362|5|652| +2450822|17365|5|523| +2450822|17366|5|329| +2450822|17368|5|162| +2450822|17371|5|664| +2450822|17372|5|882| +2450822|17374|5|186| +2450822|17377|5|995| +2450822|17378|5|480| +2450822|17380|5|840| +2450822|17383|5|82| +2450822|17384|5|396| +2450822|17386|5|105| +2450822|17389|5|651| +2450822|17390|5|153| +2450822|17392|5|775| +2450822|17395|5|665| +2450822|17396|5|937| +2450822|17398|5|552| +2450822|17401|5|495| +2450822|17402|5|843| +2450822|17404|5|722| +2450822|17407|5|903| +2450822|17408|5|589| +2450822|17410|5|10| +2450822|17413|5|| +2450822|17414|5|784| +2450822|17416|5|392| +2450822|17419|5|65| +2450822|17420|5|615| +2450822|17422|5|608| +2450822|17425|5|603| +2450822|17426|5|129| +2450822|17428|5|659| +2450822|17431|5|| +2450822|17432|5|567| +2450822|17434|5|261| +2450822|17437|5|384| +2450822|17438|5|697| +2450822|17440|5|258| +2450822|17443|5|173| +2450822|17444|5|377| +2450822|17446|5|841| +2450822|17449|5|199| +2450822|17450|5|737| +2450822|17452|5|493| +2450822|17455|5|879| +2450822|17456|5|97| +2450822|17458|5|134| +2450822|17461|5|843| +2450822|17462|5|920| +2450822|17464|5|531| +2450822|17467|5|72| +2450822|17468|5|876| +2450822|17470|5|76| +2450822|17473|5|265| +2450822|17474|5|145| +2450822|17476|5|770| +2450822|17479|5|816| +2450822|17480|5|40| +2450822|17482|5|626| +2450822|17485|5|255| +2450822|17486|5|112| +2450822|17488|5|400| +2450822|17491|5|93| +2450822|17492|5|845| +2450822|17494|5|625| +2450822|17497|5|920| +2450822|17498|5|107| +2450822|17500|5|575| +2450822|17503|5|503| +2450822|17504|5|189| +2450822|17506|5|202| +2450822|17509|5|176| +2450822|17510|5|283| +2450822|17512|5|442| +2450822|17515|5|754| +2450822|17516|5|934| +2450822|17518|5|62| +2450822|17521|5|191| +2450822|17522|5|619| +2450822|17524|5|749| +2450822|17527|5|785| +2450822|17528|5|997| +2450822|17530|5|286| +2450822|17533|5|130| +2450822|17534|5|106| +2450822|17536|5|| +2450822|17539|5|184| +2450822|17540|5|727| +2450822|17542|5|957| +2450822|17545|5|157| +2450822|17546|5|275| +2450822|17548|5|62| +2450822|17551|5|273| +2450822|17552|5|968| +2450822|17554|5|375| +2450822|17557|5|445| +2450822|17558|5|960| +2450822|17560|5|201| +2450822|17563|5|691| +2450822|17564|5|| +2450822|17566|5|831| +2450822|17569|5|874| +2450822|17570|5|566| +2450822|17572|5|330| +2450822|17575|5|657| +2450822|17576|5|80| +2450822|17578|5|995| +2450822|17581|5|774| +2450822|17582|5|212| +2450822|17584|5|125| +2450822|17587|5|43| +2450822|17588|5|763| +2450822|17590|5|761| +2450822|17593|5|| +2450822|17594|5|752| +2450822|17596|5|119| +2450822|17599|5|| +2450822|17600|5|131| +2450822|17602|5|311| +2450822|17605|5|341| +2450822|17606|5|228| +2450822|17608|5|676| +2450822|17611|5|657| +2450822|17612|5|814| +2450822|17614|5|675| +2450822|17617|5|867| +2450822|17618|5|305| +2450822|17620|5|121| +2450822|17623|5|143| +2450822|17624|5|784| +2450822|17626|5|221| +2450822|17629|5|| +2450822|17630|5|387| +2450822|17632|5|126| +2450822|17635|5|622| +2450822|17636|5|452| +2450822|17638|5|737| +2450822|17641|5|283| +2450822|17642|5|783| +2450822|17644|5|640| +2450822|17647|5|| +2450822|17648|5|496| +2450822|17650|5|635| +2450822|17653|5|116| +2450822|17654|5|713| +2450822|17656|5|453| +2450822|17659|5|144| +2450822|17660|5|468| +2450822|17662|5|788| +2450822|17665|5|155| +2450822|17666|5|196| +2450822|17668|5|131| +2450822|17671|5|795| +2450822|17672|5|789| +2450822|17674|5|| +2450822|17677|5|469| +2450822|17678|5|687| +2450822|17680|5|304| +2450822|17683|5|514| +2450822|17684|5|619| +2450822|17686|5|891| +2450822|17689|5|662| +2450822|17690|5|51| +2450822|17692|5|41| +2450822|17695|5|768| +2450822|17696|5|| +2450822|17698|5|| +2450822|17701|5|224| +2450822|17702|5|797| +2450822|17704|5|444| +2450822|17707|5|600| +2450822|17708|5|190| +2450822|17710|5|52| +2450822|17713|5|| +2450822|17714|5|559| +2450822|17716|5|58| +2450822|17719|5|391| +2450822|17720|5|733| +2450822|17722|5|422| +2450822|17725|5|870| +2450822|17726|5|16| +2450822|17728|5|994| +2450822|17731|5|447| +2450822|17732|5|455| +2450822|17734|5|| +2450822|17737|5|239| +2450822|17738|5|247| +2450822|17740|5|287| +2450822|17743|5|501| +2450822|17744|5|498| +2450822|17746|5|542| +2450822|17749|5|145| +2450822|17750|5|21| +2450822|17752|5|322| +2450822|17755|5|393| +2450822|17756|5|924| +2450822|17758|5|566| +2450822|17761|5|| +2450822|17762|5|909| +2450822|17764|5|713| +2450822|17767|5|545| +2450822|17768|5|| +2450822|17770|5|1| +2450822|17773|5|429| +2450822|17774|5|473| +2450822|17776|5|| +2450822|17779|5|557| +2450822|17780|5|243| +2450822|17782|5|853| +2450822|17785|5|171| +2450822|17786|5|408| +2450822|17788|5|514| +2450822|17791|5|935| +2450822|17792|5|| +2450822|17794|5|582| +2450822|17797|5|30| +2450822|17798|5|914| +2450822|17800|5|611| +2450822|17803|5|872| +2450822|17804|5|252| +2450822|17806|5|85| +2450822|17809|5|403| +2450822|17810|5|173| +2450822|17812|5|923| +2450822|17815|5|906| +2450822|17816|5|512| +2450822|17818|5|451| +2450822|17821|5|52| +2450822|17822|5|879| +2450822|17824|5|350| +2450822|17827|5|324| +2450822|17828|5|| +2450822|17830|5|103| +2450822|17833|5|675| +2450822|17834|5|554| +2450822|17836|5|1000| +2450822|17839|5|57| +2450822|17840|5|| +2450822|17842|5|295| +2450822|17845|5|56| +2450822|17846|5|306| +2450822|17848|5|495| +2450822|17851|5|779| +2450822|17852|5|387| +2450822|17854|5|816| +2450822|17857|5|850| +2450822|17858|5|614| +2450822|17860|5|961| +2450822|17863|5|568| +2450822|17864|5|523| +2450822|17866|5|62| +2450822|17869|5|129| +2450822|17870|5|842| +2450822|17872|5|943| +2450822|17875|5|173| +2450822|17876|5|1| +2450822|17878|5|32| +2450822|17881|5|235| +2450822|17882|5|352| +2450822|17884|5|358| +2450822|17887|5|397| +2450822|17888|5|674| +2450822|17890|5|941| +2450822|17893|5|771| +2450822|17894|5|651| +2450822|17896|5|247| +2450822|17899|5|913| +2450822|17900|5|370| +2450822|17902|5|863| +2450822|17905|5|| +2450822|17906|5|577| +2450822|17908|5|582| +2450822|17911|5|420| +2450822|17912|5|760| +2450822|17914|5|493| +2450822|17917|5|| +2450822|17918|5|| +2450822|17920|5|625| +2450822|17923|5|933| +2450822|17924|5|835| +2450822|17926|5|366| +2450822|17929|5|417| +2450822|17930|5|615| +2450822|17932|5|983| +2450822|17935|5|128| +2450822|17936|5|264| +2450822|17938|5|889| +2450822|17941|5|328| +2450822|17942|5|567| +2450822|17944|5|931| +2450822|17947|5|272| +2450822|17948|5|782| +2450822|17950|5|296| +2450822|17953|5|78| +2450822|17954|5|211| +2450822|17956|5|209| +2450822|17959|5|990| +2450822|17960|5|955| +2450822|17962|5|| +2450822|17965|5|474| +2450822|17966|5|582| +2450822|17968|5|108| +2450822|17971|5|845| +2450822|17972|5|678| +2450822|17974|5|912| +2450822|17977|5|996| +2450822|17978|5|237| +2450822|17980|5|548| +2450822|17983|5|820| +2450822|17984|5|620| +2450822|17986|5|135| +2450822|17989|5|776| +2450822|17990|5|61| +2450822|17992|5|382| +2450822|17995|5|988| +2450822|17996|5|305| +2450822|17998|5|874| +2450829|1|1|452| +2450829|2|1|84| +2450829|4|1|81| +2450829|7|1|| +2450829|8|1|306| +2450829|10|1|| +2450829|13|1|381| +2450829|14|1|687| +2450829|16|1|648| +2450829|19|1|671| +2450829|20|1|633| +2450829|22|1|501| +2450829|25|1|101| +2450829|26|1|164| +2450829|28|1|78| +2450829|31|1|157| +2450829|32|1|802| +2450829|34|1|647| +2450829|37|1|532| +2450829|38|1|684| +2450829|40|1|365| +2450829|43|1|587| +2450829|44|1|33| +2450829|46|1|564| +2450829|49|1|242| +2450829|50|1|855| +2450829|52|1|| +2450829|55|1|873| +2450829|56|1|| +2450829|58|1|980| +2450829|61|1|875| +2450829|62|1|4| +2450829|64|1|439| +2450829|67|1|6| +2450829|68|1|414| +2450829|70|1|473| +2450829|73|1|863| +2450829|74|1|323| +2450829|76|1|355| +2450829|79|1|713| +2450829|80|1|681| +2450829|82|1|247| +2450829|85|1|258| +2450829|86|1|137| +2450829|88|1|774| +2450829|91|1|171| +2450829|92|1|87| +2450829|94|1|| +2450829|97|1|93| +2450829|98|1|230| +2450829|100|1|956| +2450829|103|1|223| +2450829|104|1|35| +2450829|106|1|874| +2450829|109|1|586| +2450829|110|1|194| +2450829|112|1|595| +2450829|115|1|867| +2450829|116|1|948| +2450829|118|1|806| +2450829|121|1|919| +2450829|122|1|745| +2450829|124|1|299| +2450829|127|1|640| +2450829|128|1|157| +2450829|130|1|| +2450829|133|1|901| +2450829|134|1|161| +2450829|136|1|178| +2450829|139|1|190| +2450829|140|1|224| +2450829|142|1|701| +2450829|145|1|684| +2450829|146|1|548| +2450829|148|1|442| +2450829|151|1|334| +2450829|152|1|465| +2450829|154|1|495| +2450829|157|1|390| +2450829|158|1|735| +2450829|160|1|694| +2450829|163|1|693| +2450829|164|1|240| +2450829|166|1|702| +2450829|169|1|283| +2450829|170|1|614| +2450829|172|1|521| +2450829|175|1|480| +2450829|176|1|830| +2450829|178|1|598| +2450829|181|1|633| +2450829|182|1|659| +2450829|184|1|739| +2450829|187|1|918| +2450829|188|1|904| +2450829|190|1|118| +2450829|193|1|644| +2450829|194|1|309| +2450829|196|1|991| +2450829|199|1|351| +2450829|200|1|48| +2450829|202|1|960| +2450829|205|1|| +2450829|206|1|948| +2450829|208|1|464| +2450829|211|1|445| +2450829|212|1|44| +2450829|214|1|453| +2450829|217|1|570| +2450829|218|1|436| +2450829|220|1|29| +2450829|223|1|616| +2450829|224|1|360| +2450829|226|1|403| +2450829|229|1|62| +2450829|230|1|556| +2450829|232|1|891| +2450829|235|1|805| +2450829|236|1|490| +2450829|238|1|549| +2450829|241|1|460| +2450829|242|1|462| +2450829|244|1|215| +2450829|247|1|588| +2450829|248|1|386| +2450829|250|1|214| +2450829|253|1|420| +2450829|254|1|218| +2450829|256|1|722| +2450829|259|1|333| +2450829|260|1|87| +2450829|262|1|581| +2450829|265|1|104| +2450829|266|1|111| +2450829|268|1|584| +2450829|271|1|987| +2450829|272|1|720| +2450829|274|1|969| +2450829|277|1|453| +2450829|278|1|916| +2450829|280|1|956| +2450829|283|1|92| +2450829|284|1|897| +2450829|286|1|273| +2450829|289|1|384| +2450829|290|1|779| +2450829|292|1|993| +2450829|295|1|757| +2450829|296|1|397| +2450829|298|1|472| +2450829|301|1|88| +2450829|302|1|| +2450829|304|1|200| +2450829|307|1|621| +2450829|308|1|| +2450829|310|1|592| +2450829|313|1|279| +2450829|314|1|788| +2450829|316|1|| +2450829|319|1|173| +2450829|320|1|| +2450829|322|1|311| +2450829|325|1|113| +2450829|326|1|527| +2450829|328|1|824| +2450829|331|1|244| +2450829|332|1|347| +2450829|334|1|| +2450829|337|1|800| +2450829|338|1|505| +2450829|340|1|224| +2450829|343|1|503| +2450829|344|1|768| +2450829|346|1|825| +2450829|349|1|752| +2450829|350|1|219| +2450829|352|1|616| +2450829|355|1|484| +2450829|356|1|861| +2450829|358|1|| +2450829|361|1|623| +2450829|362|1|193| +2450829|364|1|108| +2450829|367|1|692| +2450829|368|1|296| +2450829|370|1|395| +2450829|373|1|455| +2450829|374|1|| +2450829|376|1|12| +2450829|379|1|887| +2450829|380|1|959| +2450829|382|1|714| +2450829|385|1|109| +2450829|386|1|770| +2450829|388|1|522| +2450829|391|1|660| +2450829|392|1|829| +2450829|394|1|559| +2450829|397|1|995| +2450829|398|1|498| +2450829|400|1|823| +2450829|403|1|696| +2450829|404|1|944| +2450829|406|1|355| +2450829|409|1|919| +2450829|410|1|963| +2450829|412|1|468| +2450829|415|1|955| +2450829|416|1|210| +2450829|418|1|134| +2450829|421|1|763| +2450829|422|1|680| +2450829|424|1|387| +2450829|427|1|997| +2450829|428|1|663| +2450829|430|1|703| +2450829|433|1|901| +2450829|434|1|| +2450829|436|1|89| +2450829|439|1|48| +2450829|440|1|353| +2450829|442|1|693| +2450829|445|1|785| +2450829|446|1|545| +2450829|448|1|590| +2450829|451|1|994| +2450829|452|1|717| +2450829|454|1|906| +2450829|457|1|804| +2450829|458|1|311| +2450829|460|1|544| +2450829|463|1|458| +2450829|464|1|51| +2450829|466|1|260| +2450829|469|1|145| +2450829|470|1|730| +2450829|472|1|355| +2450829|475|1|626| +2450829|476|1|904| +2450829|478|1|748| +2450829|481|1|758| +2450829|482|1|613| +2450829|484|1|400| +2450829|487|1|73| +2450829|488|1|743| +2450829|490|1|879| +2450829|493|1|850| +2450829|494|1|995| +2450829|496|1|888| +2450829|499|1|495| +2450829|500|1|622| +2450829|502|1|90| +2450829|505|1|316| +2450829|506|1|722| +2450829|508|1|994| +2450829|511|1|228| +2450829|512|1|649| +2450829|514|1|113| +2450829|517|1|314| +2450829|518|1|992| +2450829|520|1|184| +2450829|523|1|315| +2450829|524|1|186| +2450829|526|1|471| +2450829|529|1|325| +2450829|530|1|645| +2450829|532|1|284| +2450829|535|1|277| +2450829|536|1|665| +2450829|538|1|837| +2450829|541|1|6| +2450829|542|1|288| +2450829|544|1|938| +2450829|547|1|871| +2450829|548|1|137| +2450829|550|1|| +2450829|553|1|370| +2450829|554|1|596| +2450829|556|1|952| +2450829|559|1|782| +2450829|560|1|537| +2450829|562|1|519| +2450829|565|1|144| +2450829|566|1|676| +2450829|568|1|339| +2450829|571|1|667| +2450829|572|1|237| +2450829|574|1|| +2450829|577|1|| +2450829|578|1|49| +2450829|580|1|989| +2450829|583|1|470| +2450829|584|1|980| +2450829|586|1|929| +2450829|589|1|220| +2450829|590|1|277| +2450829|592|1|24| +2450829|595|1|376| +2450829|596|1|216| +2450829|598|1|189| +2450829|601|1|441| +2450829|602|1|| +2450829|604|1|403| +2450829|607|1|989| +2450829|608|1|378| +2450829|610|1|| +2450829|613|1|123| +2450829|614|1|| +2450829|616|1|896| +2450829|619|1|975| +2450829|620|1|196| +2450829|622|1|| +2450829|625|1|106| +2450829|626|1|993| +2450829|628|1|370| +2450829|631|1|417| +2450829|632|1|741| +2450829|634|1|676| +2450829|637|1|95| +2450829|638|1|546| +2450829|640|1|656| +2450829|643|1|563| +2450829|644|1|273| +2450829|646|1|535| +2450829|649|1|464| +2450829|650|1|403| +2450829|652|1|622| +2450829|655|1|331| +2450829|656|1|648| +2450829|658|1|713| +2450829|661|1|| +2450829|662|1|220| +2450829|664|1|| +2450829|667|1|810| +2450829|668|1|200| +2450829|670|1|527| +2450829|673|1|804| +2450829|674|1|421| +2450829|676|1|346| +2450829|679|1|13| +2450829|680|1|61| +2450829|682|1|360| +2450829|685|1|798| +2450829|686|1|212| +2450829|688|1|131| +2450829|691|1|377| +2450829|692|1|575| +2450829|694|1|666| +2450829|697|1|862| +2450829|698|1|597| +2450829|700|1|822| +2450829|703|1|905| +2450829|704|1|338| +2450829|706|1|128| +2450829|709|1|379| +2450829|710|1|123| +2450829|712|1|421| +2450829|715|1|897| +2450829|716|1|104| +2450829|718|1|275| +2450829|721|1|774| +2450829|722|1|264| +2450829|724|1|141| +2450829|727|1|483| +2450829|728|1|421| +2450829|730|1|72| +2450829|733|1|216| +2450829|734|1|683| +2450829|736|1|470| +2450829|739|1|105| +2450829|740|1|55| +2450829|742|1|520| +2450829|745|1|386| +2450829|746|1|529| +2450829|748|1|38| +2450829|751|1|645| +2450829|752|1|870| +2450829|754|1|379| +2450829|757|1|453| +2450829|758|1|388| +2450829|760|1|215| +2450829|763|1|| +2450829|764|1|6| +2450829|766|1|215| +2450829|769|1|665| +2450829|770|1|39| +2450829|772|1|172| +2450829|775|1|945| +2450829|776|1|231| +2450829|778|1|147| +2450829|781|1|887| +2450829|782|1|204| +2450829|784|1|636| +2450829|787|1|207| +2450829|788|1|623| +2450829|790|1|489| +2450829|793|1|736| +2450829|794|1|98| +2450829|796|1|919| +2450829|799|1|974| +2450829|800|1|| +2450829|802|1|796| +2450829|805|1|| +2450829|806|1|674| +2450829|808|1|304| +2450829|811|1|673| +2450829|812|1|856| +2450829|814|1|849| +2450829|817|1|912| +2450829|818|1|317| +2450829|820|1|936| +2450829|823|1|607| +2450829|824|1|114| +2450829|826|1|407| +2450829|829|1|260| +2450829|830|1|738| +2450829|832|1|707| +2450829|835|1|869| +2450829|836|1|265| +2450829|838|1|251| +2450829|841|1|246| +2450829|842|1|117| +2450829|844|1|720| +2450829|847|1|920| +2450829|848|1|674| +2450829|850|1|407| +2450829|853|1|344| +2450829|854|1|126| +2450829|856|1|58| +2450829|859|1|250| +2450829|860|1|820| +2450829|862|1|535| +2450829|865|1|888| +2450829|866|1|561| +2450829|868|1|59| +2450829|871|1|4| +2450829|872|1|137| +2450829|874|1|483| +2450829|877|1|496| +2450829|878|1|250| +2450829|880|1|655| +2450829|883|1|82| +2450829|884|1|960| +2450829|886|1|386| +2450829|889|1|2| +2450829|890|1|970| +2450829|892|1|146| +2450829|895|1|280| +2450829|896|1|733| +2450829|898|1|847| +2450829|901|1|303| +2450829|902|1|482| +2450829|904|1|371| +2450829|907|1|849| +2450829|908|1|132| +2450829|910|1|18| +2450829|913|1|132| +2450829|914|1|800| +2450829|916|1|492| +2450829|919|1|| +2450829|920|1|580| +2450829|922|1|566| +2450829|925|1|330| +2450829|926|1|857| +2450829|928|1|551| +2450829|931|1|59| +2450829|932|1|659| +2450829|934|1|8| +2450829|937|1|990| +2450829|938|1|| +2450829|940|1|632| +2450829|943|1|733| +2450829|944|1|286| +2450829|946|1|864| +2450829|949|1|172| +2450829|950|1|954| +2450829|952|1|938| +2450829|955|1|482| +2450829|956|1|26| +2450829|958|1|974| +2450829|961|1|204| +2450829|962|1|968| +2450829|964|1|440| +2450829|967|1|185| +2450829|968|1|463| +2450829|970|1|761| +2450829|973|1|480| +2450829|974|1|708| +2450829|976|1|660| +2450829|979|1|983| +2450829|980|1|972| +2450829|982|1|700| +2450829|985|1|662| +2450829|986|1|131| +2450829|988|1|57| +2450829|991|1|880| +2450829|992|1|544| +2450829|994|1|225| +2450829|997|1|627| +2450829|998|1|494| +2450829|1000|1|926| +2450829|1003|1|551| +2450829|1004|1|724| +2450829|1006|1|209| +2450829|1009|1|30| +2450829|1010|1|620| +2450829|1012|1|317| +2450829|1015|1|529| +2450829|1016|1|892| +2450829|1018|1|778| +2450829|1021|1|130| +2450829|1022|1|567| +2450829|1024|1|276| +2450829|1027|1|943| +2450829|1028|1|737| +2450829|1030|1|872| +2450829|1033|1|61| +2450829|1034|1|996| +2450829|1036|1|212| +2450829|1039|1|508| +2450829|1040|1|169| +2450829|1042|1|45| +2450829|1045|1|101| +2450829|1046|1|200| +2450829|1048|1|998| +2450829|1051|1|222| +2450829|1052|1|936| +2450829|1054|1|64| +2450829|1057|1|799| +2450829|1058|1|329| +2450829|1060|1|923| +2450829|1063|1|432| +2450829|1064|1|237| +2450829|1066|1|925| +2450829|1069|1|167| +2450829|1070|1|621| +2450829|1072|1|549| +2450829|1075|1|79| +2450829|1076|1|486| +2450829|1078|1|719| +2450829|1081|1|250| +2450829|1082|1|491| +2450829|1084|1|72| +2450829|1087|1|277| +2450829|1088|1|317| +2450829|1090|1|966| +2450829|1093|1|358| +2450829|1094|1|664| +2450829|1096|1|85| +2450829|1099|1|946| +2450829|1100|1|268| +2450829|1102|1|563| +2450829|1105|1|434| +2450829|1106|1|729| +2450829|1108|1|224| +2450829|1111|1|600| +2450829|1112|1|90| +2450829|1114|1|861| +2450829|1117|1|784| +2450829|1118|1|325| +2450829|1120|1|829| +2450829|1123|1|467| +2450829|1124|1|215| +2450829|1126|1|743| +2450829|1129|1|775| +2450829|1130|1|379| +2450829|1132|1|792| +2450829|1135|1|192| +2450829|1136|1|733| +2450829|1138|1|700| +2450829|1141|1|| +2450829|1142|1|292| +2450829|1144|1|263| +2450829|1147|1|240| +2450829|1148|1|360| +2450829|1150|1|408| +2450829|1153|1|331| +2450829|1154|1|740| +2450829|1156|1|652| +2450829|1159|1|76| +2450829|1160|1|604| +2450829|1162|1|175| +2450829|1165|1|837| +2450829|1166|1|625| +2450829|1168|1|406| +2450829|1171|1|538| +2450829|1172|1|735| +2450829|1174|1|528| +2450829|1177|1|792| +2450829|1178|1|851| +2450829|1180|1|273| +2450829|1183|1|287| +2450829|1184|1|812| +2450829|1186|1|952| +2450829|1189|1|827| +2450829|1190|1|412| +2450829|1192|1|253| +2450829|1195|1|102| +2450829|1196|1|525| +2450829|1198|1|258| +2450829|1201|1|361| +2450829|1202|1|| +2450829|1204|1|213| +2450829|1207|1|498| +2450829|1208|1|38| +2450829|1210|1|481| +2450829|1213|1|954| +2450829|1214|1|969| +2450829|1216|1|994| +2450829|1219|1|749| +2450829|1220|1|404| +2450829|1222|1|855| +2450829|1225|1|322| +2450829|1226|1|641| +2450829|1228|1|167| +2450829|1231|1|767| +2450829|1232|1|119| +2450829|1234|1|495| +2450829|1237|1|114| +2450829|1238|1|591| +2450829|1240|1|662| +2450829|1243|1|670| +2450829|1244|1|827| +2450829|1246|1|958| +2450829|1249|1|742| +2450829|1250|1|636| +2450829|1252|1|166| +2450829|1255|1|162| +2450829|1256|1|614| +2450829|1258|1|343| +2450829|1261|1|726| +2450829|1262|1|360| +2450829|1264|1|552| +2450829|1267|1|547| +2450829|1268|1|870| +2450829|1270|1|882| +2450829|1273|1|884| +2450829|1274|1|978| +2450829|1276|1|750| +2450829|1279|1|774| +2450829|1280|1|532| +2450829|1282|1|242| +2450829|1285|1|86| +2450829|1286|1|| +2450829|1288|1|820| +2450829|1291|1|611| +2450829|1292|1|64| +2450829|1294|1|886| +2450829|1297|1|310| +2450829|1298|1|606| +2450829|1300|1|82| +2450829|1303|1|196| +2450829|1304|1|315| +2450829|1306|1|271| +2450829|1309|1|757| +2450829|1310|1|840| +2450829|1312|1|395| +2450829|1315|1|491| +2450829|1316|1|930| +2450829|1318|1|761| +2450829|1321|1|643| +2450829|1322|1|628| +2450829|1324|1|497| +2450829|1327|1|335| +2450829|1328|1|561| +2450829|1330|1|298| +2450829|1333|1|436| +2450829|1334|1|584| +2450829|1336|1|440| +2450829|1339|1|838| +2450829|1340|1|768| +2450829|1342|1|476| +2450829|1345|1|115| +2450829|1346|1|655| +2450829|1348|1|157| +2450829|1351|1|593| +2450829|1352|1|647| +2450829|1354|1|480| +2450829|1357|1|78| +2450829|1358|1|248| +2450829|1360|1|293| +2450829|1363|1|233| +2450829|1364|1|| +2450829|1366|1|335| +2450829|1369|1|427| +2450829|1370|1|553| +2450829|1372|1|158| +2450829|1375|1|900| +2450829|1376|1|906| +2450829|1378|1|94| +2450829|1381|1|956| +2450829|1382|1|200| +2450829|1384|1|775| +2450829|1387|1|155| +2450829|1388|1|| +2450829|1390|1|619| +2450829|1393|1|22| +2450829|1394|1|29| +2450829|1396|1|288| +2450829|1399|1|949| +2450829|1400|1|985| +2450829|1402|1|453| +2450829|1405|1|646| +2450829|1406|1|481| +2450829|1408|1|964| +2450829|1411|1|593| +2450829|1412|1|325| +2450829|1414|1|701| +2450829|1417|1|556| +2450829|1418|1|78| +2450829|1420|1|978| +2450829|1423|1|799| +2450829|1424|1|843| +2450829|1426|1|442| +2450829|1429|1|95| +2450829|1430|1|681| +2450829|1432|1|587| +2450829|1435|1|836| +2450829|1436|1|746| +2450829|1438|1|619| +2450829|1441|1|| +2450829|1442|1|76| +2450829|1444|1|593| +2450829|1447|1|865| +2450829|1448|1|287| +2450829|1450|1|831| +2450829|1453|1|539| +2450829|1454|1|629| +2450829|1456|1|639| +2450829|1459|1|502| +2450829|1460|1|936| +2450829|1462|1|784| +2450829|1465|1|403| +2450829|1466|1|294| +2450829|1468|1|311| +2450829|1471|1|321| +2450829|1472|1|354| +2450829|1474|1|231| +2450829|1477|1|363| +2450829|1478|1|771| +2450829|1480|1|950| +2450829|1483|1|193| +2450829|1484|1|293| +2450829|1486|1|84| +2450829|1489|1|460| +2450829|1490|1|453| +2450829|1492|1|| +2450829|1495|1|834| +2450829|1496|1|848| +2450829|1498|1|765| +2450829|1501|1|283| +2450829|1502|1|| +2450829|1504|1|66| +2450829|1507|1|553| +2450829|1508|1|977| +2450829|1510|1|120| +2450829|1513|1|189| +2450829|1514|1|482| +2450829|1516|1|935| +2450829|1519|1|879| +2450829|1520|1|314| +2450829|1522|1|| +2450829|1525|1|588| +2450829|1526|1|| +2450829|1528|1|217| +2450829|1531|1|126| +2450829|1532|1|212| +2450829|1534|1|422| +2450829|1537|1|196| +2450829|1538|1|354| +2450829|1540|1|778| +2450829|1543|1|573| +2450829|1544|1|813| +2450829|1546|1|435| +2450829|1549|1|207| +2450829|1550|1|322| +2450829|1552|1|660| +2450829|1555|1|880| +2450829|1556|1|212| +2450829|1558|1|| +2450829|1561|1|816| +2450829|1562|1|918| +2450829|1564|1|646| +2450829|1567|1|401| +2450829|1568|1|| +2450829|1570|1|164| +2450829|1573|1|888| +2450829|1574|1|258| +2450829|1576|1|566| +2450829|1579|1|613| +2450829|1580|1|548| +2450829|1582|1|519| +2450829|1585|1|975| +2450829|1586|1|72| +2450829|1588|1|718| +2450829|1591|1|238| +2450829|1592|1|| +2450829|1594|1|484| +2450829|1597|1|104| +2450829|1598|1|841| +2450829|1600|1|| +2450829|1603|1|208| +2450829|1604|1|873| +2450829|1606|1|928| +2450829|1609|1|688| +2450829|1610|1|| +2450829|1612|1|933| +2450829|1615|1|958| +2450829|1616|1|| +2450829|1618|1|253| +2450829|1621|1|859| +2450829|1622|1|322| +2450829|1624|1|569| +2450829|1627|1|| +2450829|1628|1|301| +2450829|1630|1|554| +2450829|1633|1|404| +2450829|1634|1|686| +2450829|1636|1|952| +2450829|1639|1|469| +2450829|1640|1|356| +2450829|1642|1|44| +2450829|1645|1|556| +2450829|1646|1|158| +2450829|1648|1|173| +2450829|1651|1|997| +2450829|1652|1|891| +2450829|1654|1|697| +2450829|1657|1|871| +2450829|1658|1|| +2450829|1660|1|850| +2450829|1663|1|267| +2450829|1664|1|770| +2450829|1666|1|18| +2450829|1669|1|982| +2450829|1670|1|852| +2450829|1672|1|908| +2450829|1675|1|290| +2450829|1676|1|952| +2450829|1678|1|243| +2450829|1681|1|83| +2450829|1682|1|677| +2450829|1684|1|401| +2450829|1687|1|5| +2450829|1688|1|877| +2450829|1690|1|301| +2450829|1693|1|| +2450829|1694|1|683| +2450829|1696|1|125| +2450829|1699|1|852| +2450829|1700|1|104| +2450829|1702|1|169| +2450829|1705|1|782| +2450829|1706|1|202| +2450829|1708|1|644| +2450829|1711|1|846| +2450829|1712|1|850| +2450829|1714|1|789| +2450829|1717|1|952| +2450829|1718|1|249| +2450829|1720|1|897| +2450829|1723|1|984| +2450829|1724|1|494| +2450829|1726|1|| +2450829|1729|1|488| +2450829|1730|1|14| +2450829|1732|1|31| +2450829|1735|1|966| +2450829|1736|1|305| +2450829|1738|1|852| +2450829|1741|1|139| +2450829|1742|1|729| +2450829|1744|1|497| +2450829|1747|1|738| +2450829|1748|1|917| +2450829|1750|1|937| +2450829|1753|1|551| +2450829|1754|1|230| +2450829|1756|1|429| +2450829|1759|1|689| +2450829|1760|1|| +2450829|1762|1|432| +2450829|1765|1|717| +2450829|1766|1|588| +2450829|1768|1|| +2450829|1771|1|548| +2450829|1772|1|921| +2450829|1774|1|437| +2450829|1777|1|882| +2450829|1778|1|296| +2450829|1780|1|221| +2450829|1783|1|475| +2450829|1784|1|903| +2450829|1786|1|195| +2450829|1789|1|437| +2450829|1790|1|729| +2450829|1792|1|| +2450829|1795|1|853| +2450829|1796|1|260| +2450829|1798|1|796| +2450829|1801|1|250| +2450829|1802|1|420| +2450829|1804|1|822| +2450829|1807|1|880| +2450829|1808|1|338| +2450829|1810|1|52| +2450829|1813|1|530| +2450829|1814|1|304| +2450829|1816|1|721| +2450829|1819|1|304| +2450829|1820|1|711| +2450829|1822|1|983| +2450829|1825|1|546| +2450829|1826|1|534| +2450829|1828|1|382| +2450829|1831|1|547| +2450829|1832|1|46| +2450829|1834|1|441| +2450829|1837|1|230| +2450829|1838|1|112| +2450829|1840|1|543| +2450829|1843|1|653| +2450829|1844|1|27| +2450829|1846|1|270| +2450829|1849|1|942| +2450829|1850|1|467| +2450829|1852|1|724| +2450829|1855|1|552| +2450829|1856|1|108| +2450829|1858|1|227| +2450829|1861|1|801| +2450829|1862|1|494| +2450829|1864|1|664| +2450829|1867|1|722| +2450829|1868|1|391| +2450829|1870|1|471| +2450829|1873|1|874| +2450829|1874|1|365| +2450829|1876|1|258| +2450829|1879|1|673| +2450829|1880|1|788| +2450829|1882|1|443| +2450829|1885|1|788| +2450829|1886|1|447| +2450829|1888|1|161| +2450829|1891|1|369| +2450829|1892|1|754| +2450829|1894|1|631| +2450829|1897|1|227| +2450829|1898|1|6| +2450829|1900|1|711| +2450829|1903|1|738| +2450829|1904|1|427| +2450829|1906|1|334| +2450829|1909|1|580| +2450829|1910|1|500| +2450829|1912|1|23| +2450829|1915|1|92| +2450829|1916|1|539| +2450829|1918|1|51| +2450829|1921|1|133| +2450829|1922|1|175| +2450829|1924|1|751| +2450829|1927|1|901| +2450829|1928|1|680| +2450829|1930|1|| +2450829|1933|1|754| +2450829|1934|1|539| +2450829|1936|1|215| +2450829|1939|1|517| +2450829|1940|1|108| +2450829|1942|1|167| +2450829|1945|1|236| +2450829|1946|1|128| +2450829|1948|1|983| +2450829|1951|1|140| +2450829|1952|1|572| +2450829|1954|1|124| +2450829|1957|1|86| +2450829|1958|1|266| +2450829|1960|1|369| +2450829|1963|1|161| +2450829|1964|1|211| +2450829|1966|1|449| +2450829|1969|1|543| +2450829|1970|1|598| +2450829|1972|1|349| +2450829|1975|1|| +2450829|1976|1|677| +2450829|1978|1|427| +2450829|1981|1|266| +2450829|1982|1|| +2450829|1984|1|4| +2450829|1987|1|332| +2450829|1988|1|644| +2450829|1990|1|131| +2450829|1993|1|710| +2450829|1994|1|965| +2450829|1996|1|637| +2450829|1999|1|43| +2450829|2000|1|550| +2450829|2002|1|884| +2450829|2005|1|663| +2450829|2006|1|88| +2450829|2008|1|280| +2450829|2011|1|686| +2450829|2012|1|126| +2450829|2014|1|686| +2450829|2017|1|964| +2450829|2018|1|225| +2450829|2020|1|358| +2450829|2023|1|37| +2450829|2024|1|619| +2450829|2026|1|432| +2450829|2029|1|705| +2450829|2030|1|83| +2450829|2032|1|80| +2450829|2035|1|8| +2450829|2036|1|234| +2450829|2038|1|935| +2450829|2041|1|633| +2450829|2042|1|| +2450829|2044|1|708| +2450829|2047|1|370| +2450829|2048|1|225| +2450829|2050|1|| +2450829|2053|1|719| +2450829|2054|1|| +2450829|2056|1|376| +2450829|2059|1|157| +2450829|2060|1|899| +2450829|2062|1|750| +2450829|2065|1|191| +2450829|2066|1|68| +2450829|2068|1|89| +2450829|2071|1|992| +2450829|2072|1|389| +2450829|2074|1|| +2450829|2077|1|638| +2450829|2078|1|20| +2450829|2080|1|19| +2450829|2083|1|475| +2450829|2084|1|69| +2450829|2086|1|222| +2450829|2089|1|43| +2450829|2090|1|773| +2450829|2092|1|123| +2450829|2095|1|256| +2450829|2096|1|388| +2450829|2098|1|| +2450829|2101|1|| +2450829|2102|1|700| +2450829|2104|1|535| +2450829|2107|1|| +2450829|2108|1|791| +2450829|2110|1|116| +2450829|2113|1|262| +2450829|2114|1|339| +2450829|2116|1|425| +2450829|2119|1|465| +2450829|2120|1|553| +2450829|2122|1|432| +2450829|2125|1|180| +2450829|2126|1|720| +2450829|2128|1|692| +2450829|2131|1|831| +2450829|2132|1|196| +2450829|2134|1|497| +2450829|2137|1|654| +2450829|2138|1|956| +2450829|2140|1|28| +2450829|2143|1|967| +2450829|2144|1|751| +2450829|2146|1|986| +2450829|2149|1|679| +2450829|2150|1|455| +2450829|2152|1|492| +2450829|2155|1|566| +2450829|2156|1|351| +2450829|2158|1|823| +2450829|2161|1|225| +2450829|2162|1|570| +2450829|2164|1|904| +2450829|2167|1|615| +2450829|2168|1|184| +2450829|2170|1|202| +2450829|2173|1|995| +2450829|2174|1|964| +2450829|2176|1|30| +2450829|2179|1|793| +2450829|2180|1|| +2450829|2182|1|438| +2450829|2185|1|816| +2450829|2186|1|535| +2450829|2188|1|9| +2450829|2191|1|130| +2450829|2192|1|497| +2450829|2194|1|206| +2450829|2197|1|673| +2450829|2198|1|77| +2450829|2200|1|600| +2450829|2203|1|645| +2450829|2204|1|| +2450829|2206|1|419| +2450829|2209|1|774| +2450829|2210|1|546| +2450829|2212|1|430| +2450829|2215|1|18| +2450829|2216|1|216| +2450829|2218|1|658| +2450829|2221|1|501| +2450829|2222|1|| +2450829|2224|1|769| +2450829|2227|1|79| +2450829|2228|1|65| +2450829|2230|1|915| +2450829|2233|1|595| +2450829|2234|1|153| +2450829|2236|1|52| +2450829|2239|1|951| +2450829|2240|1|16| +2450829|2242|1|| +2450829|2245|1|703| +2450829|2246|1|421| +2450829|2248|1|986| +2450829|2251|1|399| +2450829|2252|1|562| +2450829|2254|1|242| +2450829|2257|1|973| +2450829|2258|1|712| +2450829|2260|1|472| +2450829|2263|1|997| +2450829|2264|1|215| +2450829|2266|1|752| +2450829|2269|1|978| +2450829|2270|1|996| +2450829|2272|1|466| +2450829|2275|1|501| +2450829|2276|1|771| +2450829|2278|1|940| +2450829|2281|1|269| +2450829|2282|1|941| +2450829|2284|1|| +2450829|2287|1|347| +2450829|2288|1|478| +2450829|2290|1|19| +2450829|2293|1|120| +2450829|2294|1|137| +2450829|2296|1|| +2450829|2299|1|951| +2450829|2300|1|344| +2450829|2302|1|776| +2450829|2305|1|118| +2450829|2306|1|307| +2450829|2308|1|317| +2450829|2311|1|189| +2450829|2312|1|673| +2450829|2314|1|723| +2450829|2317|1|993| +2450829|2318|1|213| +2450829|2320|1|650| +2450829|2323|1|438| +2450829|2324|1|184| +2450829|2326|1|252| +2450829|2329|1|461| +2450829|2330|1|478| +2450829|2332|1|518| +2450829|2335|1|952| +2450829|2336|1|852| +2450829|2338|1|514| +2450829|2341|1|163| +2450829|2342|1|396| +2450829|2344|1|371| +2450829|2347|1|24| +2450829|2348|1|716| +2450829|2350|1|76| +2450829|2353|1|831| +2450829|2354|1|464| +2450829|2356|1|775| +2450829|2359|1|765| +2450829|2360|1|133| +2450829|2362|1|829| +2450829|2365|1|982| +2450829|2366|1|85| +2450829|2368|1|826| +2450829|2371|1|423| +2450829|2372|1|184| +2450829|2374|1|716| +2450829|2377|1|236| +2450829|2378|1|| +2450829|2380|1|806| +2450829|2383|1|977| +2450829|2384|1|849| +2450829|2386|1|701| +2450829|2389|1|| +2450829|2390|1|341| +2450829|2392|1|36| +2450829|2395|1|956| +2450829|2396|1|513| +2450829|2398|1|35| +2450829|2401|1|544| +2450829|2402|1|414| +2450829|2404|1|782| +2450829|2407|1|519| +2450829|2408|1|32| +2450829|2410|1|353| +2450829|2413|1|627| +2450829|2414|1|281| +2450829|2416|1|425| +2450829|2419|1|219| +2450829|2420|1|421| +2450829|2422|1|308| +2450829|2425|1|| +2450829|2426|1|741| +2450829|2428|1|415| +2450829|2431|1|560| +2450829|2432|1|657| +2450829|2434|1|435| +2450829|2437|1|174| +2450829|2438|1|868| +2450829|2440|1|754| +2450829|2443|1|226| +2450829|2444|1|| +2450829|2446|1|297| +2450829|2449|1|715| +2450829|2450|1|899| +2450829|2452|1|834| +2450829|2455|1|39| +2450829|2456|1|39| +2450829|2458|1|255| +2450829|2461|1|769| +2450829|2462|1|1000| +2450829|2464|1|445| +2450829|2467|1|437| +2450829|2468|1|370| +2450829|2470|1|181| +2450829|2473|1|370| +2450829|2474|1|797| +2450829|2476|1|926| +2450829|2479|1|751| +2450829|2480|1|739| +2450829|2482|1|170| +2450829|2485|1|796| +2450829|2486|1|693| +2450829|2488|1|537| +2450829|2491|1|673| +2450829|2492|1|926| +2450829|2494|1|339| +2450829|2497|1|287| +2450829|2498|1|889| +2450829|2500|1|923| +2450829|2503|1|54| +2450829|2504|1|509| +2450829|2506|1|740| +2450829|2509|1|329| +2450829|2510|1|| +2450829|2512|1|44| +2450829|2515|1|358| +2450829|2516|1|747| +2450829|2518|1|159| +2450829|2521|1|154| +2450829|2522|1|210| +2450829|2524|1|662| +2450829|2527|1|812| +2450829|2528|1|435| +2450829|2530|1|199| +2450829|2533|1|912| +2450829|2534|1|807| +2450829|2536|1|282| +2450829|2539|1|552| +2450829|2540|1|168| +2450829|2542|1|| +2450829|2545|1|32| +2450829|2546|1|| +2450829|2548|1|47| +2450829|2551|1|98| +2450829|2552|1|414| +2450829|2554|1|247| +2450829|2557|1|341| +2450829|2558|1|916| +2450829|2560|1|897| +2450829|2563|1|808| +2450829|2564|1|942| +2450829|2566|1|734| +2450829|2569|1|432| +2450829|2570|1|553| +2450829|2572|1|729| +2450829|2575|1|52| +2450829|2576|1|122| +2450829|2578|1|67| +2450829|2581|1|118| +2450829|2582|1|505| +2450829|2584|1|| +2450829|2587|1|380| +2450829|2588|1|425| +2450829|2590|1|367| +2450829|2593|1|796| +2450829|2594|1|152| +2450829|2596|1|884| +2450829|2599|1|980| +2450829|2600|1|392| +2450829|2602|1|687| +2450829|2605|1|758| +2450829|2606|1|783| +2450829|2608|1|134| +2450829|2611|1|818| +2450829|2612|1|22| +2450829|2614|1|167| +2450829|2617|1|176| +2450829|2618|1|155| +2450829|2620|1|339| +2450829|2623|1|33| +2450829|2624|1|453| +2450829|2626|1|135| +2450829|2629|1|956| +2450829|2630|1|949| +2450829|2632|1|443| +2450829|2635|1|644| +2450829|2636|1|562| +2450829|2638|1|812| +2450829|2641|1|272| +2450829|2642|1|881| +2450829|2644|1|203| +2450829|2647|1|709| +2450829|2648|1|269| +2450829|2650|1|| +2450829|2653|1|441| +2450829|2654|1|261| +2450829|2656|1|815| +2450829|2659|1|624| +2450829|2660|1|540| +2450829|2662|1|50| +2450829|2665|1|559| +2450829|2666|1|| +2450829|2668|1|2| +2450829|2671|1|687| +2450829|2672|1|811| +2450829|2674|1|| +2450829|2677|1|960| +2450829|2678|1|285| +2450829|2680|1|460| +2450829|2683|1|358| +2450829|2684|1|10| +2450829|2686|1|83| +2450829|2689|1|953| +2450829|2690|1|981| +2450829|2692|1|735| +2450829|2695|1|183| +2450829|2696|1|709| +2450829|2698|1|261| +2450829|2701|1|501| +2450829|2702|1|840| +2450829|2704|1|947| +2450829|2707|1|476| +2450829|2708|1|6| +2450829|2710|1|223| +2450829|2713|1|275| +2450829|2714|1|971| +2450829|2716|1|831| +2450829|2719|1|127| +2450829|2720|1|42| +2450829|2722|1|676| +2450829|2725|1|950| +2450829|2726|1|431| +2450829|2728|1|883| +2450829|2731|1|401| +2450829|2732|1|536| +2450829|2734|1|854| +2450829|2737|1|781| +2450829|2738|1|120| +2450829|2740|1|563| +2450829|2743|1|716| +2450829|2744|1|549| +2450829|2746|1|438| +2450829|2749|1|269| +2450829|2750|1|837| +2450829|2752|1|751| +2450829|2755|1|763| +2450829|2756|1|430| +2450829|2758|1|425| +2450829|2761|1|761| +2450829|2762|1|94| +2450829|2764|1|470| +2450829|2767|1|366| +2450829|2768|1|374| +2450829|2770|1|584| +2450829|2773|1|273| +2450829|2774|1|266| +2450829|2776|1|520| +2450829|2779|1|253| +2450829|2780|1|| +2450829|2782|1|542| +2450829|2785|1|53| +2450829|2786|1|858| +2450829|2788|1|824| +2450829|2791|1|109| +2450829|2792|1|926| +2450829|2794|1|| +2450829|2797|1|737| +2450829|2798|1|560| +2450829|2800|1|23| +2450829|2803|1|476| +2450829|2804|1|981| +2450829|2806|1|984| +2450829|2809|1|597| +2450829|2810|1|| +2450829|2812|1|491| +2450829|2815|1|462| +2450829|2816|1|923| +2450829|2818|1|| +2450829|2821|1|216| +2450829|2822|1|142| +2450829|2824|1|556| +2450829|2827|1|592| +2450829|2828|1|| +2450829|2830|1|839| +2450829|2833|1|5| +2450829|2834|1|658| +2450829|2836|1|631| +2450829|2839|1|790| +2450829|2840|1|862| +2450829|2842|1|814| +2450829|2845|1|111| +2450829|2846|1|199| +2450829|2848|1|325| +2450829|2851|1|6| +2450829|2852|1|412| +2450829|2854|1|491| +2450829|2857|1|296| +2450829|2858|1|234| +2450829|2860|1|36| +2450829|2863|1|672| +2450829|2864|1|788| +2450829|2866|1|485| +2450829|2869|1|347| +2450829|2870|1|173| +2450829|2872|1|610| +2450829|2875|1|124| +2450829|2876|1|751| +2450829|2878|1|249| +2450829|2881|1|228| +2450829|2882|1|144| +2450829|2884|1|450| +2450829|2887|1|457| +2450829|2888|1|907| +2450829|2890|1|994| +2450829|2893|1|566| +2450829|2894|1|| +2450829|2896|1|191| +2450829|2899|1|61| +2450829|2900|1|433| +2450829|2902|1|32| +2450829|2905|1|600| +2450829|2906|1|609| +2450829|2908|1|7| +2450829|2911|1|344| +2450829|2912|1|805| +2450829|2914|1|192| +2450829|2917|1|785| +2450829|2918|1|703| +2450829|2920|1|233| +2450829|2923|1|733| +2450829|2924|1|0| +2450829|2926|1|467| +2450829|2929|1|14| +2450829|2930|1|4| +2450829|2932|1|314| +2450829|2935|1|55| +2450829|2936|1|44| +2450829|2938|1|254| +2450829|2941|1|909| +2450829|2942|1|415| +2450829|2944|1|221| +2450829|2947|1|945| +2450829|2948|1|90| +2450829|2950|1|368| +2450829|2953|1|999| +2450829|2954|1|519| +2450829|2956|1|72| +2450829|2959|1|863| +2450829|2960|1|818| +2450829|2962|1|503| +2450829|2965|1|180| +2450829|2966|1|797| +2450829|2968|1|953| +2450829|2971|1|882| +2450829|2972|1|787| +2450829|2974|1|855| +2450829|2977|1|939| +2450829|2978|1|946| +2450829|2980|1|669| +2450829|2983|1|217| +2450829|2984|1|722| +2450829|2986|1|| +2450829|2989|1|69| +2450829|2990|1|477| +2450829|2992|1|674| +2450829|2995|1|534| +2450829|2996|1|701| +2450829|2998|1|405| +2450829|3001|1|76| +2450829|3002|1|58| +2450829|3004|1|54| +2450829|3007|1|399| +2450829|3008|1|82| +2450829|3010|1|957| +2450829|3013|1|543| +2450829|3014|1|528| +2450829|3016|1|630| +2450829|3019|1|339| +2450829|3020|1|686| +2450829|3022|1|419| +2450829|3025|1|758| +2450829|3026|1|327| +2450829|3028|1|901| +2450829|3031|1|507| +2450829|3032|1|530| +2450829|3034|1|166| +2450829|3037|1|818| +2450829|3038|1|691| +2450829|3040|1|598| +2450829|3043|1|289| +2450829|3044|1|508| +2450829|3046|1|871| +2450829|3049|1|333| +2450829|3050|1|360| +2450829|3052|1|| +2450829|3055|1|641| +2450829|3056|1|365| +2450829|3058|1|578| +2450829|3061|1|| +2450829|3062|1|724| +2450829|3064|1|488| +2450829|3067|1|366| +2450829|3068|1|491| +2450829|3070|1|411| +2450829|3073|1|433| +2450829|3074|1|938| +2450829|3076|1|22| +2450829|3079|1|464| +2450829|3080|1|682| +2450829|3082|1|59| +2450829|3085|1|212| +2450829|3086|1|| +2450829|3088|1|862| +2450829|3091|1|976| +2450829|3092|1|827| +2450829|3094|1|640| +2450829|3097|1|4| +2450829|3098|1|972| +2450829|3100|1|210| +2450829|3103|1|164| +2450829|3104|1|720| +2450829|3106|1|181| +2450829|3109|1|114| +2450829|3110|1|668| +2450829|3112|1|16| +2450829|3115|1|677| +2450829|3116|1|735| +2450829|3118|1|356| +2450829|3121|1|33| +2450829|3122|1|159| +2450829|3124|1|810| +2450829|3127|1|468| +2450829|3128|1|88| +2450829|3130|1|150| +2450829|3133|1|457| +2450829|3134|1|339| +2450829|3136|1|623| +2450829|3139|1|417| +2450829|3140|1|308| +2450829|3142|1|843| +2450829|3145|1|806| +2450829|3146|1|894| +2450829|3148|1|980| +2450829|3151|1|464| +2450829|3152|1|938| +2450829|3154|1|846| +2450829|3157|1|779| +2450829|3158|1|250| +2450829|3160|1|| +2450829|3163|1|975| +2450829|3164|1|73| +2450829|3166|1|183| +2450829|3169|1|539| +2450829|3170|1|141| +2450829|3172|1|191| +2450829|3175|1|732| +2450829|3176|1|246| +2450829|3178|1|905| +2450829|3181|1|457| +2450829|3182|1|662| +2450829|3184|1|182| +2450829|3187|1|156| +2450829|3188|1|110| +2450829|3190|1|| +2450829|3193|1|632| +2450829|3194|1|318| +2450829|3196|1|834| +2450829|3199|1|851| +2450829|3200|1|822| +2450829|3202|1|874| +2450829|3205|1|909| +2450829|3206|1|525| +2450829|3208|1|557| +2450829|3211|1|771| +2450829|3212|1|483| +2450829|3214|1|271| +2450829|3217|1|367| +2450829|3218|1|626| +2450829|3220|1|742| +2450829|3223|1|395| +2450829|3224|1|422| +2450829|3226|1|486| +2450829|3229|1|530| +2450829|3230|1|226| +2450829|3232|1|312| +2450829|3235|1|777| +2450829|3236|1|601| +2450829|3238|1|725| +2450829|3241|1|352| +2450829|3242|1|291| +2450829|3244|1|844| +2450829|3247|1|869| +2450829|3248|1|236| +2450829|3250|1|388| +2450829|3253|1|594| +2450829|3254|1|765| +2450829|3256|1|789| +2450829|3259|1|735| +2450829|3260|1|645| +2450829|3262|1|32| +2450829|3265|1|788| +2450829|3266|1|886| +2450829|3268|1|402| +2450829|3271|1|282| +2450829|3272|1|168| +2450829|3274|1|764| +2450829|3277|1|42| +2450829|3278|1|851| +2450829|3280|1|684| +2450829|3283|1|488| +2450829|3284|1|819| +2450829|3286|1|127| +2450829|3289|1|769| +2450829|3290|1|358| +2450829|3292|1|960| +2450829|3295|1|| +2450829|3296|1|243| +2450829|3298|1|191| +2450829|3301|1|755| +2450829|3302|1|57| +2450829|3304|1|408| +2450829|3307|1|229| +2450829|3308|1|485| +2450829|3310|1|698| +2450829|3313|1|621| +2450829|3314|1|219| +2450829|3316|1|526| +2450829|3319|1|508| +2450829|3320|1|252| +2450829|3322|1|722| +2450829|3325|1|338| +2450829|3326|1|388| +2450829|3328|1|750| +2450829|3331|1|106| +2450829|3332|1|| +2450829|3334|1|822| +2450829|3337|1|918| +2450829|3338|1|196| +2450829|3340|1|245| +2450829|3343|1|184| +2450829|3344|1|903| +2450829|3346|1|874| +2450829|3349|1|254| +2450829|3350|1|552| +2450829|3352|1|4| +2450829|3355|1|909| +2450829|3356|1|12| +2450829|3358|1|| +2450829|3361|1|959| +2450829|3362|1|| +2450829|3364|1|906| +2450829|3367|1|917| +2450829|3368|1|19| +2450829|3370|1|749| +2450829|3373|1|655| +2450829|3374|1|812| +2450829|3376|1|476| +2450829|3379|1|583| +2450829|3380|1|990| +2450829|3382|1|716| +2450829|3385|1|282| +2450829|3386|1|927| +2450829|3388|1|890| +2450829|3391|1|485| +2450829|3392|1|363| +2450829|3394|1|| +2450829|3397|1|272| +2450829|3398|1|13| +2450829|3400|1|922| +2450829|3403|1|611| +2450829|3404|1|551| +2450829|3406|1|981| +2450829|3409|1|769| +2450829|3410|1|422| +2450829|3412|1|433| +2450829|3415|1|505| +2450829|3416|1|139| +2450829|3418|1|594| +2450829|3421|1|364| +2450829|3422|1|44| +2450829|3424|1|574| +2450829|3427|1|288| +2450829|3428|1|429| +2450829|3430|1|10| +2450829|3433|1|584| +2450829|3434|1|3| +2450829|3436|1|175| +2450829|3439|1|862| +2450829|3440|1|761| +2450829|3442|1|968| +2450829|3445|1|924| +2450829|3446|1|511| +2450829|3448|1|109| +2450829|3451|1|353| +2450829|3452|1|578| +2450829|3454|1|775| +2450829|3457|1|980| +2450829|3458|1|21| +2450829|3460|1|239| +2450829|3463|1|310| +2450829|3464|1|98| +2450829|3466|1|890| +2450829|3469|1|82| +2450829|3470|1|673| +2450829|3472|1|553| +2450829|3475|1|371| +2450829|3476|1|203| +2450829|3478|1|749| +2450829|3481|1|738| +2450829|3482|1|201| +2450829|3484|1|511| +2450829|3487|1|330| +2450829|3488|1|624| +2450829|3490|1|994| +2450829|3493|1|71| +2450829|3494|1|659| +2450829|3496|1|206| +2450829|3499|1|909| +2450829|3500|1|277| +2450829|3502|1|766| +2450829|3505|1|415| +2450829|3506|1|138| +2450829|3508|1|756| +2450829|3511|1|512| +2450829|3512|1|66| +2450829|3514|1|851| +2450829|3517|1|370| +2450829|3518|1|309| +2450829|3520|1|699| +2450829|3523|1|152| +2450829|3524|1|22| +2450829|3526|1|198| +2450829|3529|1|245| +2450829|3530|1|273| +2450829|3532|1|767| +2450829|3535|1|830| +2450829|3536|1|345| +2450829|3538|1|942| +2450829|3541|1|674| +2450829|3542|1|| +2450829|3544|1|131| +2450829|3547|1|402| +2450829|3548|1|557| +2450829|3550|1|892| +2450829|3553|1|585| +2450829|3554|1|| +2450829|3556|1|769| +2450829|3559|1|288| +2450829|3560|1|755| +2450829|3562|1|940| +2450829|3565|1|997| +2450829|3566|1|553| +2450829|3568|1|807| +2450829|3571|1|107| +2450829|3572|1|795| +2450829|3574|1|462| +2450829|3577|1|566| +2450829|3578|1|248| +2450829|3580|1|438| +2450829|3583|1|512| +2450829|3584|1|494| +2450829|3586|1|76| +2450829|3589|1|505| +2450829|3590|1|720| +2450829|3592|1|608| +2450829|3595|1|428| +2450829|3596|1|93| +2450829|3598|1|782| +2450829|3601|1|756| +2450829|3602|1|239| +2450829|3604|1|590| +2450829|3607|1|772| +2450829|3608|1|772| +2450829|3610|1|251| +2450829|3613|1|879| +2450829|3614|1|825| +2450829|3616|1|596| +2450829|3619|1|371| +2450829|3620|1|684| +2450829|3622|1|879| +2450829|3625|1|757| +2450829|3626|1|165| +2450829|3628|1|154| +2450829|3631|1|628| +2450829|3632|1|417| +2450829|3634|1|813| +2450829|3637|1|950| +2450829|3638|1|732| +2450829|3640|1|| +2450829|3643|1|718| +2450829|3644|1|729| +2450829|3646|1|647| +2450829|3649|1|230| +2450829|3650|1|89| +2450829|3652|1|645| +2450829|3655|1|871| +2450829|3656|1|310| +2450829|3658|1|546| +2450829|3661|1|801| +2450829|3662|1|555| +2450829|3664|1|503| +2450829|3667|1|487| +2450829|3668|1|310| +2450829|3670|1|91| +2450829|3673|1|596| +2450829|3674|1|920| +2450829|3676|1|127| +2450829|3679|1|274| +2450829|3680|1|692| +2450829|3682|1|78| +2450829|3685|1|856| +2450829|3686|1|153| +2450829|3688|1|725| +2450829|3691|1|835| +2450829|3692|1|928| +2450829|3694|1|306| +2450829|3697|1|345| +2450829|3698|1|76| +2450829|3700|1|375| +2450829|3703|1|838| +2450829|3704|1|18| +2450829|3706|1|992| +2450829|3709|1|757| +2450829|3710|1|935| +2450829|3712|1|304| +2450829|3715|1|540| +2450829|3716|1|758| +2450829|3718|1|2| +2450829|3721|1|687| +2450829|3722|1|| +2450829|3724|1|488| +2450829|3727|1|376| +2450829|3728|1|| +2450829|3730|1|925| +2450829|3733|1|622| +2450829|3734|1|967| +2450829|3736|1|559| +2450829|3739|1|112| +2450829|3740|1|438| +2450829|3742|1|876| +2450829|3745|1|21| +2450829|3746|1|224| +2450829|3748|1|990| +2450829|3751|1|303| +2450829|3752|1|167| +2450829|3754|1|455| +2450829|3757|1|63| +2450829|3758|1|305| +2450829|3760|1|368| +2450829|3763|1|620| +2450829|3764|1|427| +2450829|3766|1|910| +2450829|3769|1|| +2450829|3770|1|369| +2450829|3772|1|528| +2450829|3775|1|386| +2450829|3776|1|520| +2450829|3778|1|483| +2450829|3781|1|515| +2450829|3782|1|245| +2450829|3784|1|947| +2450829|3787|1|392| +2450829|3788|1|| +2450829|3790|1|286| +2450829|3793|1|742| +2450829|3794|1|730| +2450829|3796|1|470| +2450829|3799|1|324| +2450829|3800|1|| +2450829|3802|1|690| +2450829|3805|1|398| +2450829|3806|1|15| +2450829|3808|1|| +2450829|3811|1|625| +2450829|3812|1|234| +2450829|3814|1|| +2450829|3817|1|567| +2450829|3818|1|892| +2450829|3820|1|759| +2450829|3823|1|340| +2450829|3824|1|| +2450829|3826|1|282| +2450829|3829|1|542| +2450829|3830|1|170| +2450829|3832|1|38| +2450829|3835|1|264| +2450829|3836|1|591| +2450829|3838|1|| +2450829|3841|1|| +2450829|3842|1|814| +2450829|3844|1|837| +2450829|3847|1|751| +2450829|3848|1|745| +2450829|3850|1|909| +2450829|3853|1|652| +2450829|3854|1|18| +2450829|3856|1|951| +2450829|3859|1|520| +2450829|3860|1|835| +2450829|3862|1|128| +2450829|3865|1|188| +2450829|3866|1|908| +2450829|3868|1|688| +2450829|3871|1|118| +2450829|3872|1|202| +2450829|3874|1|890| +2450829|3877|1|183| +2450829|3878|1|739| +2450829|3880|1|343| +2450829|3883|1|725| +2450829|3884|1|691| +2450829|3886|1|680| +2450829|3889|1|133| +2450829|3890|1|46| +2450829|3892|1|904| +2450829|3895|1|820| +2450829|3896|1|165| +2450829|3898|1|435| +2450829|3901|1|719| +2450829|3902|1|528| +2450829|3904|1|465| +2450829|3907|1|112| +2450829|3908|1|535| +2450829|3910|1|310| +2450829|3913|1|569| +2450829|3914|1|288| +2450829|3916|1|749| +2450829|3919|1|637| +2450829|3920|1|827| +2450829|3922|1|685| +2450829|3925|1|796| +2450829|3926|1|675| +2450829|3928|1|848| +2450829|3931|1|833| +2450829|3932|1|540| +2450829|3934|1|844| +2450829|3937|1|880| +2450829|3938|1|332| +2450829|3940|1|60| +2450829|3943|1|565| +2450829|3944|1|325| +2450829|3946|1|980| +2450829|3949|1|958| +2450829|3950|1|315| +2450829|3952|1|188| +2450829|3955|1|118| +2450829|3956|1|276| +2450829|3958|1|733| +2450829|3961|1|620| +2450829|3962|1|| +2450829|3964|1|583| +2450829|3967|1|565| +2450829|3968|1|418| +2450829|3970|1|302| +2450829|3973|1|85| +2450829|3974|1|136| +2450829|3976|1|689| +2450829|3979|1|805| +2450829|3980|1|448| +2450829|3982|1|929| +2450829|3985|1|582| +2450829|3986|1|149| +2450829|3988|1|749| +2450829|3991|1|74| +2450829|3992|1|416| +2450829|3994|1|247| +2450829|3997|1|570| +2450829|3998|1|35| +2450829|4000|1|829| +2450829|4003|1|254| +2450829|4004|1|232| +2450829|4006|1|216| +2450829|4009|1|| +2450829|4010|1|796| +2450829|4012|1|50| +2450829|4015|1|851| +2450829|4016|1|853| +2450829|4018|1|643| +2450829|4021|1|287| +2450829|4022|1|302| +2450829|4024|1|879| +2450829|4027|1|735| +2450829|4028|1|264| +2450829|4030|1|910| +2450829|4033|1|442| +2450829|4034|1|512| +2450829|4036|1|683| +2450829|4039|1|525| +2450829|4040|1|48| +2450829|4042|1|979| +2450829|4045|1|883| +2450829|4046|1|| +2450829|4048|1|549| +2450829|4051|1|56| +2450829|4052|1|148| +2450829|4054|1|990| +2450829|4057|1|421| +2450829|4058|1|838| +2450829|4060|1|172| +2450829|4063|1|297| +2450829|4064|1|441| +2450829|4066|1|50| +2450829|4069|1|86| +2450829|4070|1|| +2450829|4072|1|148| +2450829|4075|1|456| +2450829|4076|1|901| +2450829|4078|1|350| +2450829|4081|1|374| +2450829|4082|1|546| +2450829|4084|1|90| +2450829|4087|1|908| +2450829|4088|1|326| +2450829|4090|1|604| +2450829|4093|1|236| +2450829|4094|1|573| +2450829|4096|1|592| +2450829|4099|1|147| +2450829|4100|1|| +2450829|4102|1|284| +2450829|4105|1|417| +2450829|4106|1|11| +2450829|4108|1|244| +2450829|4111|1|15| +2450829|4112|1|87| +2450829|4114|1|668| +2450829|4117|1|666| +2450829|4118|1|648| +2450829|4120|1|53| +2450829|4123|1|318| +2450829|4124|1|235| +2450829|4126|1|896| +2450829|4129|1|103| +2450829|4130|1|120| +2450829|4132|1|420| +2450829|4135|1|842| +2450829|4136|1|222| +2450829|4138|1|679| +2450829|4141|1|686| +2450829|4142|1|552| +2450829|4144|1|| +2450829|4147|1|404| +2450829|4148|1|157| +2450829|4150|1|858| +2450829|4153|1|934| +2450829|4154|1|154| +2450829|4156|1|715| +2450829|4159|1|635| +2450829|4160|1|473| +2450829|4162|1|| +2450829|4165|1|864| +2450829|4166|1|286| +2450829|4168|1|3| +2450829|4171|1|425| +2450829|4172|1|229| +2450829|4174|1|547| +2450829|4177|1|640| +2450829|4178|1|54| +2450829|4180|1|327| +2450829|4183|1|630| +2450829|4184|1|475| +2450829|4186|1|326| +2450829|4189|1|361| +2450829|4190|1|453| +2450829|4192|1|60| +2450829|4195|1|430| +2450829|4196|1|411| +2450829|4198|1|131| +2450829|4201|1|224| +2450829|4202|1|113| +2450829|4204|1|343| +2450829|4207|1|474| +2450829|4208|1|230| +2450829|4210|1|8| +2450829|4213|1|511| +2450829|4214|1|253| +2450829|4216|1|99| +2450829|4219|1|875| +2450829|4220|1|890| +2450829|4222|1|243| +2450829|4225|1|161| +2450829|4226|1|120| +2450829|4228|1|625| +2450829|4231|1|546| +2450829|4232|1|850| +2450829|4234|1|985| +2450829|4237|1|453| +2450829|4238|1|596| +2450829|4240|1|486| +2450829|4243|1|338| +2450829|4244|1|677| +2450829|4246|1|967| +2450829|4249|1|876| +2450829|4250|1|728| +2450829|4252|1|705| +2450829|4255|1|570| +2450829|4256|1|717| +2450829|4258|1|248| +2450829|4261|1|607| +2450829|4262|1|933| +2450829|4264|1|75| +2450829|4267|1|473| +2450829|4268|1|29| +2450829|4270|1|152| +2450829|4273|1|951| +2450829|4274|1|282| +2450829|4276|1|773| +2450829|4279|1|278| +2450829|4280|1|943| +2450829|4282|1|220| +2450829|4285|1|368| +2450829|4286|1|69| +2450829|4288|1|166| +2450829|4291|1|467| +2450829|4292|1|36| +2450829|4294|1|717| +2450829|4297|1|63| +2450829|4298|1|892| +2450829|4300|1|423| +2450829|4303|1|112| +2450829|4304|1|425| +2450829|4306|1|840| +2450829|4309|1|769| +2450829|4310|1|348| +2450829|4312|1|284| +2450829|4315|1|918| +2450829|4316|1|556| +2450829|4318|1|110| +2450829|4321|1|136| +2450829|4322|1|74| +2450829|4324|1|516| +2450829|4327|1|261| +2450829|4328|1|87| +2450829|4330|1|549| +2450829|4333|1|545| +2450829|4334|1|760| +2450829|4336|1|276| +2450829|4339|1|474| +2450829|4340|1|976| +2450829|4342|1|213| +2450829|4345|1|70| +2450829|4346|1|558| +2450829|4348|1|384| +2450829|4351|1|736| +2450829|4352|1|221| +2450829|4354|1|882| +2450829|4357|1|121| +2450829|4358|1|155| +2450829|4360|1|354| +2450829|4363|1|588| +2450829|4364|1|525| +2450829|4366|1|271| +2450829|4369|1|12| +2450829|4370|1|8| +2450829|4372|1|112| +2450829|4375|1|509| +2450829|4376|1|42| +2450829|4378|1|| +2450829|4381|1|961| +2450829|4382|1|556| +2450829|4384|1|991| +2450829|4387|1|| +2450829|4388|1|652| +2450829|4390|1|901| +2450829|4393|1|731| +2450829|4394|1|188| +2450829|4396|1|685| +2450829|4399|1|203| +2450829|4400|1|| +2450829|4402|1|198| +2450829|4405|1|242| +2450829|4406|1|| +2450829|4408|1|259| +2450829|4411|1|376| +2450829|4412|1|142| +2450829|4414|1|82| +2450829|4417|1|42| +2450829|4418|1|96| +2450829|4420|1|609| +2450829|4423|1|899| +2450829|4424|1|33| +2450829|4426|1|156| +2450829|4429|1|296| +2450829|4430|1|118| +2450829|4432|1|148| +2450829|4435|1|199| +2450829|4436|1|10| +2450829|4438|1|178| +2450829|4441|1|428| +2450829|4442|1|424| +2450829|4444|1|225| +2450829|4447|1|841| +2450829|4448|1|231| +2450829|4450|1|206| +2450829|4453|1|690| +2450829|4454|1|487| +2450829|4456|1|56| +2450829|4459|1|| +2450829|4460|1|482| +2450829|4462|1|101| +2450829|4465|1|850| +2450829|4466|1|850| +2450829|4468|1|938| +2450829|4471|1|533| +2450829|4472|1|204| +2450829|4474|1|213| +2450829|4477|1|989| +2450829|4478|1|32| +2450829|4480|1|969| +2450829|4483|1|429| +2450829|4484|1|410| +2450829|4486|1|523| +2450829|4489|1|210| +2450829|4490|1|716| +2450829|4492|1|288| +2450829|4495|1|535| +2450829|4496|1|794| +2450829|4498|1|302| +2450829|4501|1|706| +2450829|4502|1|775| +2450829|4504|1|207| +2450829|4507|1|475| +2450829|4508|1|574| +2450829|4510|1|926| +2450829|4513|1|491| +2450829|4514|1|97| +2450829|4516|1|142| +2450829|4519|1|712| +2450829|4520|1|444| +2450829|4522|1|2| +2450829|4525|1|800| +2450829|4526|1|918| +2450829|4528|1|177| +2450829|4531|1|766| +2450829|4532|1|493| +2450829|4534|1|152| +2450829|4537|1|304| +2450829|4538|1|408| +2450829|4540|1|799| +2450829|4543|1|431| +2450829|4544|1|899| +2450829|4546|1|763| +2450829|4549|1|219| +2450829|4550|1|434| +2450829|4552|1|307| +2450829|4555|1|145| +2450829|4556|1|987| +2450829|4558|1|742| +2450829|4561|1|893| +2450829|4562|1|167| +2450829|4564|1|909| +2450829|4567|1|125| +2450829|4568|1|937| +2450829|4570|1|90| +2450829|4573|1|803| +2450829|4574|1|432| +2450829|4576|1|896| +2450829|4579|1|25| +2450829|4580|1|201| +2450829|4582|1|217| +2450829|4585|1|469| +2450829|4586|1|810| +2450829|4588|1|603| +2450829|4591|1|27| +2450829|4592|1|188| +2450829|4594|1|805| +2450829|4597|1|166| +2450829|4598|1|264| +2450829|4600|1|857| +2450829|4603|1|| +2450829|4604|1|85| +2450829|4606|1|476| +2450829|4609|1|727| +2450829|4610|1|449| +2450829|4612|1|933| +2450829|4615|1|740| +2450829|4616|1|572| +2450829|4618|1|765| +2450829|4621|1|796| +2450829|4622|1|303| +2450829|4624|1|341| +2450829|4627|1|99| +2450829|4628|1|994| +2450829|4630|1|483| +2450829|4633|1|852| +2450829|4634|1|325| +2450829|4636|1|443| +2450829|4639|1|179| +2450829|4640|1|| +2450829|4642|1|553| +2450829|4645|1|948| +2450829|4646|1|59| +2450829|4648|1|930| +2450829|4651|1|480| +2450829|4652|1|965| +2450829|4654|1|177| +2450829|4657|1|708| +2450829|4658|1|736| +2450829|4660|1|305| +2450829|4663|1|| +2450829|4664|1|644| +2450829|4666|1|| +2450829|4669|1|136| +2450829|4670|1|235| +2450829|4672|1|641| +2450829|4675|1|88| +2450829|4676|1|722| +2450829|4678|1|258| +2450829|4681|1|898| +2450829|4682|1|181| +2450829|4684|1|918| +2450829|4687|1|353| +2450829|4688|1|137| +2450829|4690|1|613| +2450829|4693|1|652| +2450829|4694|1|73| +2450829|4696|1|499| +2450829|4699|1|662| +2450829|4700|1|713| +2450829|4702|1|120| +2450829|4705|1|825| +2450829|4706|1|69| +2450829|4708|1|461| +2450829|4711|1|426| +2450829|4712|1|896| +2450829|4714|1|703| +2450829|4717|1|373| +2450829|4718|1|250| +2450829|4720|1|430| +2450829|4723|1|| +2450829|4724|1|499| +2450829|4726|1|| +2450829|4729|1|455| +2450829|4730|1|832| +2450829|4732|1|784| +2450829|4735|1|778| +2450829|4736|1|877| +2450829|4738|1|706| +2450829|4741|1|884| +2450829|4742|1|107| +2450829|4744|1|386| +2450829|4747|1|454| +2450829|4748|1|| +2450829|4750|1|761| +2450829|4753|1|618| +2450829|4754|1|165| +2450829|4756|1|793| +2450829|4759|1|956| +2450829|4760|1|467| +2450829|4762|1|177| +2450829|4765|1|262| +2450829|4766|1|136| +2450829|4768|1|817| +2450829|4771|1|528| +2450829|4772|1|612| +2450829|4774|1|| +2450829|4777|1|327| +2450829|4778|1|216| +2450829|4780|1|928| +2450829|4783|1|609| +2450829|4784|1|207| +2450829|4786|1|219| +2450829|4789|1|179| +2450829|4790|1|25| +2450829|4792|1|287| +2450829|4795|1|178| +2450829|4796|1|305| +2450829|4798|1|998| +2450829|4801|1|848| +2450829|4802|1|866| +2450829|4804|1|730| +2450829|4807|1|308| +2450829|4808|1|176| +2450829|4810|1|209| +2450829|4813|1|879| +2450829|4814|1|436| +2450829|4816|1|502| +2450829|4819|1|480| +2450829|4820|1|177| +2450829|4822|1|213| +2450829|4825|1|925| +2450829|4826|1|960| +2450829|4828|1|| +2450829|4831|1|867| +2450829|4832|1|610| +2450829|4834|1|361| +2450829|4837|1|327| +2450829|4838|1|365| +2450829|4840|1|418| +2450829|4843|1|499| +2450829|4844|1|699| +2450829|4846|1|694| +2450829|4849|1|859| +2450829|4850|1|346| +2450829|4852|1|771| +2450829|4855|1|27| +2450829|4856|1|827| +2450829|4858|1|657| +2450829|4861|1|264| +2450829|4862|1|| +2450829|4864|1|| +2450829|4867|1|| +2450829|4868|1|735| +2450829|4870|1|360| +2450829|4873|1|284| +2450829|4874|1|643| +2450829|4876|1|169| +2450829|4879|1|619| +2450829|4880|1|678| +2450829|4882|1|281| +2450829|4885|1|873| +2450829|4886|1|568| +2450829|4888|1|120| +2450829|4891|1|792| +2450829|4892|1|53| +2450829|4894|1|85| +2450829|4897|1|721| +2450829|4898|1|123| +2450829|4900|1|926| +2450829|4903|1|92| +2450829|4904|1|275| +2450829|4906|1|306| +2450829|4909|1|706| +2450829|4910|1|116| +2450829|4912|1|886| +2450829|4915|1|120| +2450829|4916|1|263| +2450829|4918|1|394| +2450829|4921|1|847| +2450829|4922|1|79| +2450829|4924|1|609| +2450829|4927|1|922| +2450829|4928|1|32| +2450829|4930|1|861| +2450829|4933|1|37| +2450829|4934|1|867| +2450829|4936|1|294| +2450829|4939|1|410| +2450829|4940|1|800| +2450829|4942|1|| +2450829|4945|1|| +2450829|4946|1|112| +2450829|4948|1|184| +2450829|4951|1|534| +2450829|4952|1|799| +2450829|4954|1|603| +2450829|4957|1|213| +2450829|4958|1|284| +2450829|4960|1|392| +2450829|4963|1|594| +2450829|4964|1|375| +2450829|4966|1|307| +2450829|4969|1|615| +2450829|4970|1|697| +2450829|4972|1|445| +2450829|4975|1|| +2450829|4976|1|186| +2450829|4978|1|203| +2450829|4981|1|105| +2450829|4982|1|853| +2450829|4984|1|998| +2450829|4987|1|354| +2450829|4988|1|768| +2450829|4990|1|693| +2450829|4993|1|133| +2450829|4994|1|691| +2450829|4996|1|205| +2450829|4999|1|160| +2450829|5000|1|465| +2450829|5002|1|467| +2450829|5005|1|412| +2450829|5006|1|88| +2450829|5008|1|174| +2450829|5011|1|5| +2450829|5012|1|253| +2450829|5014|1|855| +2450829|5017|1|554| +2450829|5018|1|147| +2450829|5020|1|96| +2450829|5023|1|50| +2450829|5024|1|| +2450829|5026|1|| +2450829|5029|1|639| +2450829|5030|1|566| +2450829|5032|1|898| +2450829|5035|1|728| +2450829|5036|1|341| +2450829|5038|1|493| +2450829|5041|1|635| +2450829|5042|1|21| +2450829|5044|1|643| +2450829|5047|1|842| +2450829|5048|1|108| +2450829|5050|1|504| +2450829|5053|1|370| +2450829|5054|1|412| +2450829|5056|1|482| +2450829|5059|1|615| +2450829|5060|1|60| +2450829|5062|1|950| +2450829|5065|1|989| +2450829|5066|1|605| +2450829|5068|1|535| +2450829|5071|1|| +2450829|5072|1|799| +2450829|5074|1|296| +2450829|5077|1|115| +2450829|5078|1|533| +2450829|5080|1|245| +2450829|5083|1|973| +2450829|5084|1|921| +2450829|5086|1|1| +2450829|5089|1|842| +2450829|5090|1|632| +2450829|5092|1|42| +2450829|5095|1|62| +2450829|5096|1|671| +2450829|5098|1|718| +2450829|5101|1|606| +2450829|5102|1|775| +2450829|5104|1|787| +2450829|5107|1|243| +2450829|5108|1|832| +2450829|5110|1|923| +2450829|5113|1|606| +2450829|5114|1|0| +2450829|5116|1|598| +2450829|5119|1|920| +2450829|5120|1|614| +2450829|5122|1|674| +2450829|5125|1|103| +2450829|5126|1|106| +2450829|5128|1|385| +2450829|5131|1|261| +2450829|5132|1|886| +2450829|5134|1|771| +2450829|5137|1|22| +2450829|5138|1|| +2450829|5140|1|627| +2450829|5143|1|280| +2450829|5144|1|826| +2450829|5146|1|41| +2450829|5149|1|646| +2450829|5150|1|208| +2450829|5152|1|966| +2450829|5155|1|784| +2450829|5156|1|965| +2450829|5158|1|312| +2450829|5161|1|428| +2450829|5162|1|636| +2450829|5164|1|867| +2450829|5167|1|0| +2450829|5168|1|| +2450829|5170|1|976| +2450829|5173|1|313| +2450829|5174|1|185| +2450829|5176|1|716| +2450829|5179|1|571| +2450829|5180|1|681| +2450829|5182|1|909| +2450829|5185|1|827| +2450829|5186|1|472| +2450829|5188|1|206| +2450829|5191|1|434| +2450829|5192|1|621| +2450829|5194|1|251| +2450829|5197|1|248| +2450829|5198|1|458| +2450829|5200|1|558| +2450829|5203|1|674| +2450829|5204|1|22| +2450829|5206|1|988| +2450829|5209|1|190| +2450829|5210|1|632| +2450829|5212|1|249| +2450829|5215|1|304| +2450829|5216|1|934| +2450829|5218|1|746| +2450829|5221|1|68| +2450829|5222|1|970| +2450829|5224|1|801| +2450829|5227|1|808| +2450829|5228|1|241| +2450829|5230|1|176| +2450829|5233|1|368| +2450829|5234|1|947| +2450829|5236|1|539| +2450829|5239|1|766| +2450829|5240|1|534| +2450829|5242|1|820| +2450829|5245|1|724| +2450829|5246|1|135| +2450829|5248|1|878| +2450829|5251|1|219| +2450829|5252|1|513| +2450829|5254|1|520| +2450829|5257|1|253| +2450829|5258|1|839| +2450829|5260|1|202| +2450829|5263|1|423| +2450829|5264|1|454| +2450829|5266|1|637| +2450829|5269|1|64| +2450829|5270|1|532| +2450829|5272|1|620| +2450829|5275|1|356| +2450829|5276|1|215| +2450829|5278|1|916| +2450829|5281|1|101| +2450829|5282|1|349| +2450829|5284|1|689| +2450829|5287|1|1000| +2450829|5288|1|347| +2450829|5290|1|834| +2450829|5293|1|949| +2450829|5294|1|762| +2450829|5296|1|625| +2450829|5299|1|3| +2450829|5300|1|704| +2450829|5302|1|2| +2450829|5305|1|566| +2450829|5306|1|385| +2450829|5308|1|556| +2450829|5311|1|445| +2450829|5312|1|331| +2450829|5314|1|273| +2450829|5317|1|276| +2450829|5318|1|373| +2450829|5320|1|389| +2450829|5323|1|194| +2450829|5324|1|945| +2450829|5326|1|432| +2450829|5329|1|| +2450829|5330|1|479| +2450829|5332|1|880| +2450829|5335|1|| +2450829|5336|1|368| +2450829|5338|1|753| +2450829|5341|1|622| +2450829|5342|1|455| +2450829|5344|1|197| +2450829|5347|1|528| +2450829|5348|1|| +2450829|5350|1|727| +2450829|5353|1|644| +2450829|5354|1|160| +2450829|5356|1|87| +2450829|5359|1|573| +2450829|5360|1|951| +2450829|5362|1|536| +2450829|5365|1|954| +2450829|5366|1|693| +2450829|5368|1|495| +2450829|5371|1|632| +2450829|5372|1|405| +2450829|5374|1|680| +2450829|5377|1|720| +2450829|5378|1|492| +2450829|5380|1|567| +2450829|5383|1|430| +2450829|5384|1|468| +2450829|5386|1|922| +2450829|5389|1|713| +2450829|5390|1|368| +2450829|5392|1|672| +2450829|5395|1|699| +2450829|5396|1|66| +2450829|5398|1|845| +2450829|5401|1|573| +2450829|5402|1|213| +2450829|5404|1|530| +2450829|5407|1|911| +2450829|5408|1|959| +2450829|5410|1|703| +2450829|5413|1|965| +2450829|5414|1|235| +2450829|5416|1|135| +2450829|5419|1|154| +2450829|5420|1|205| +2450829|5422|1|287| +2450829|5425|1|422| +2450829|5426|1|927| +2450829|5428|1|466| +2450829|5431|1|317| +2450829|5432|1|850| +2450829|5434|1|610| +2450829|5437|1|192| +2450829|5438|1|892| +2450829|5440|1|404| +2450829|5443|1|913| +2450829|5444|1|758| +2450829|5446|1|212| +2450829|5449|1|253| +2450829|5450|1|781| +2450829|5452|1|678| +2450829|5455|1|634| +2450829|5456|1|753| +2450829|5458|1|943| +2450829|5461|1|95| +2450829|5462|1|578| +2450829|5464|1|562| +2450829|5467|1|624| +2450829|5468|1|287| +2450829|5470|1|773| +2450829|5473|1|879| +2450829|5474|1|652| +2450829|5476|1|266| +2450829|5479|1|259| +2450829|5480|1|122| +2450829|5482|1|237| +2450829|5485|1|| +2450829|5486|1|| +2450829|5488|1|788| +2450829|5491|1|902| +2450829|5492|1|300| +2450829|5494|1|80| +2450829|5497|1|622| +2450829|5498|1|210| +2450829|5500|1|| +2450829|5503|1|757| +2450829|5504|1|| +2450829|5506|1|735| +2450829|5509|1|750| +2450829|5510|1|598| +2450829|5512|1|590| +2450829|5515|1|514| +2450829|5516|1|586| +2450829|5518|1|336| +2450829|5521|1|83| +2450829|5522|1|370| +2450829|5524|1|248| +2450829|5527|1|918| +2450829|5528|1|244| +2450829|5530|1|341| +2450829|5533|1|175| +2450829|5534|1|727| +2450829|5536|1|559| +2450829|5539|1|501| +2450829|5540|1|586| +2450829|5542|1|90| +2450829|5545|1|292| +2450829|5546|1|422| +2450829|5548|1|896| +2450829|5551|1|86| +2450829|5552|1|454| +2450829|5554|1|260| +2450829|5557|1|892| +2450829|5558|1|275| +2450829|5560|1|| +2450829|5563|1|235| +2450829|5564|1|630| +2450829|5566|1|381| +2450829|5569|1|346| +2450829|5570|1|349| +2450829|5572|1|272| +2450829|5575|1|902| +2450829|5576|1|417| +2450829|5578|1|204| +2450829|5581|1|266| +2450829|5582|1|| +2450829|5584|1|354| +2450829|5587|1|193| +2450829|5588|1|184| +2450829|5590|1|| +2450829|5593|1|586| +2450829|5594|1|160| +2450829|5596|1|241| +2450829|5599|1|176| +2450829|5600|1|169| +2450829|5602|1|413| +2450829|5605|1|| +2450829|5606|1|445| +2450829|5608|1|359| +2450829|5611|1|948| +2450829|5612|1|| +2450829|5614|1|925| +2450829|5617|1|514| +2450829|5618|1|966| +2450829|5620|1|441| +2450829|5623|1|889| +2450829|5624|1|601| +2450829|5626|1|142| +2450829|5629|1|| +2450829|5630|1|872| +2450829|5632|1|608| +2450829|5635|1|179| +2450829|5636|1|979| +2450829|5638|1|171| +2450829|5641|1|981| +2450829|5642|1|447| +2450829|5644|1|967| +2450829|5647|1|299| +2450829|5648|1|976| +2450829|5650|1|297| +2450829|5653|1|634| +2450829|5654|1|760| +2450829|5656|1|433| +2450829|5659|1|50| +2450829|5660|1|472| +2450829|5662|1|506| +2450829|5665|1|540| +2450829|5666|1|775| +2450829|5668|1|539| +2450829|5671|1|911| +2450829|5672|1|25| +2450829|5674|1|| +2450829|5677|1|658| +2450829|5678|1|546| +2450829|5680|1|776| +2450829|5683|1|611| +2450829|5684|1|4| +2450829|5686|1|588| +2450829|5689|1|713| +2450829|5690|1|571| +2450829|5692|1|| +2450829|5695|1|540| +2450829|5696|1|218| +2450829|5698|1|886| +2450829|5701|1|948| +2450829|5702|1|550| +2450829|5704|1|663| +2450829|5707|1|| +2450829|5708|1|| +2450829|5710|1|12| +2450829|5713|1|567| +2450829|5714|1|943| +2450829|5716|1|604| +2450829|5719|1|419| +2450829|5720|1|326| +2450829|5722|1|543| +2450829|5725|1|343| +2450829|5726|1|899| +2450829|5728|1|177| +2450829|5731|1|189| +2450829|5732|1|385| +2450829|5734|1|24| +2450829|5737|1|634| +2450829|5738|1|460| +2450829|5740|1|19| +2450829|5743|1|994| +2450829|5744|1|527| +2450829|5746|1|704| +2450829|5749|1|75| +2450829|5750|1|968| +2450829|5752|1|470| +2450829|5755|1|824| +2450829|5756|1|411| +2450829|5758|1|670| +2450829|5761|1|668| +2450829|5762|1|891| +2450829|5764|1|880| +2450829|5767|1|145| +2450829|5768|1|996| +2450829|5770|1|499| +2450829|5773|1|134| +2450829|5774|1|353| +2450829|5776|1|906| +2450829|5779|1|422| +2450829|5780|1|294| +2450829|5782|1|894| +2450829|5785|1|735| +2450829|5786|1|75| +2450829|5788|1|93| +2450829|5791|1|952| +2450829|5792|1|587| +2450829|5794|1|967| +2450829|5797|1|669| +2450829|5798|1|793| +2450829|5800|1|264| +2450829|5803|1|994| +2450829|5804|1|498| +2450829|5806|1|453| +2450829|5809|1|261| +2450829|5810|1|366| +2450829|5812|1|910| +2450829|5815|1|196| +2450829|5816|1|819| +2450829|5818|1|910| +2450829|5821|1|831| +2450829|5822|1|413| +2450829|5824|1|696| +2450829|5827|1|553| +2450829|5828|1|779| +2450829|5830|1|161| +2450829|5833|1|606| +2450829|5834|1|663| +2450829|5836|1|130| +2450829|5839|1|571| +2450829|5840|1|870| +2450829|5842|1|514| +2450829|5845|1|796| +2450829|5846|1|53| +2450829|5848|1|437| +2450829|5851|1|464| +2450829|5852|1|667| +2450829|5854|1|614| +2450829|5857|1|950| +2450829|5858|1|| +2450829|5860|1|71| +2450829|5863|1|400| +2450829|5864|1|691| +2450829|5866|1|124| +2450829|5869|1|458| +2450829|5870|1|151| +2450829|5872|1|276| +2450829|5875|1|426| +2450829|5876|1|523| +2450829|5878|1|942| +2450829|5881|1|604| +2450829|5882|1|407| +2450829|5884|1|923| +2450829|5887|1|166| +2450829|5888|1|736| +2450829|5890|1|737| +2450829|5893|1|338| +2450829|5894|1|845| +2450829|5896|1|752| +2450829|5899|1|903| +2450829|5900|1|448| +2450829|5902|1|157| +2450829|5905|1|897| +2450829|5906|1|674| +2450829|5908|1|966| +2450829|5911|1|59| +2450829|5912|1|0| +2450829|5914|1|817| +2450829|5917|1|720| +2450829|5918|1|149| +2450829|5920|1|262| +2450829|5923|1|782| +2450829|5924|1|25| +2450829|5926|1|| +2450829|5929|1|876| +2450829|5930|1|688| +2450829|5932|1|522| +2450829|5935|1|362| +2450829|5936|1|230| +2450829|5938|1|515| +2450829|5941|1|87| +2450829|5942|1|702| +2450829|5944|1|706| +2450829|5947|1|253| +2450829|5948|1|659| +2450829|5950|1|522| +2450829|5953|1|634| +2450829|5954|1|622| +2450829|5956|1|239| +2450829|5959|1|571| +2450829|5960|1|130| +2450829|5962|1|874| +2450829|5965|1|312| +2450829|5966|1|792| +2450829|5968|1|770| +2450829|5971|1|718| +2450829|5972|1|654| +2450829|5974|1|872| +2450829|5977|1|205| +2450829|5978|1|749| +2450829|5980|1|901| +2450829|5983|1|424| +2450829|5984|1|202| +2450829|5986|1|966| +2450829|5989|1|8| +2450829|5990|1|433| +2450829|5992|1|812| +2450829|5995|1|666| +2450829|5996|1|572| +2450829|5998|1|992| +2450829|6001|1|518| +2450829|6002|1|601| +2450829|6004|1|958| +2450829|6007|1|769| +2450829|6008|1|889| +2450829|6010|1|54| +2450829|6013|1|| +2450829|6014|1|256| +2450829|6016|1|| +2450829|6019|1|210| +2450829|6020|1|263| +2450829|6022|1|11| +2450829|6025|1|386| +2450829|6026|1|618| +2450829|6028|1|183| +2450829|6031|1|120| +2450829|6032|1|249| +2450829|6034|1|914| +2450829|6037|1|409| +2450829|6038|1|427| +2450829|6040|1|104| +2450829|6043|1|327| +2450829|6044|1|545| +2450829|6046|1|| +2450829|6049|1|717| +2450829|6050|1|547| +2450829|6052|1|489| +2450829|6055|1|328| +2450829|6056|1|953| +2450829|6058|1|834| +2450829|6061|1|121| +2450829|6062|1|778| +2450829|6064|1|929| +2450829|6067|1|49| +2450829|6068|1|270| +2450829|6070|1|926| +2450829|6073|1|798| +2450829|6074|1|863| +2450829|6076|1|255| +2450829|6079|1|312| +2450829|6080|1|967| +2450829|6082|1|874| +2450829|6085|1|415| +2450829|6086|1|757| +2450829|6088|1|485| +2450829|6091|1|737| +2450829|6092|1|867| +2450829|6094|1|934| +2450829|6097|1|799| +2450829|6098|1|415| +2450829|6100|1|294| +2450829|6103|1|260| +2450829|6104|1|834| +2450829|6106|1|157| +2450829|6109|1|628| +2450829|6110|1|545| +2450829|6112|1|11| +2450829|6115|1|838| +2450829|6116|1|| +2450829|6118|1|637| +2450829|6121|1|840| +2450829|6122|1|766| +2450829|6124|1|990| +2450829|6127|1|182| +2450829|6128|1|60| +2450829|6130|1|496| +2450829|6133|1|977| +2450829|6134|1|| +2450829|6136|1|966| +2450829|6139|1|| +2450829|6140|1|762| +2450829|6142|1|319| +2450829|6145|1|647| +2450829|6146|1|255| +2450829|6148|1|847| +2450829|6151|1|156| +2450829|6152|1|307| +2450829|6154|1|970| +2450829|6157|1|238| +2450829|6158|1|85| +2450829|6160|1|642| +2450829|6163|1|| +2450829|6164|1|706| +2450829|6166|1|228| +2450829|6169|1|850| +2450829|6170|1|705| +2450829|6172|1|414| +2450829|6175|1|199| +2450829|6176|1|592| +2450829|6178|1|37| +2450829|6181|1|913| +2450829|6182|1|760| +2450829|6184|1|221| +2450829|6187|1|751| +2450829|6188|1|116| +2450829|6190|1|969| +2450829|6193|1|100| +2450829|6194|1|259| +2450829|6196|1|331| +2450829|6199|1|980| +2450829|6200|1|570| +2450829|6202|1|221| +2450829|6205|1|733| +2450829|6206|1|928| +2450829|6208|1|606| +2450829|6211|1|161| +2450829|6212|1|829| +2450829|6214|1|112| +2450829|6217|1|872| +2450829|6218|1|832| +2450829|6220|1|| +2450829|6223|1|170| +2450829|6224|1|456| +2450829|6226|1|733| +2450829|6229|1|390| +2450829|6230|1|481| +2450829|6232|1|26| +2450829|6235|1|956| +2450829|6236|1|400| +2450829|6238|1|172| +2450829|6241|1|346| +2450829|6242|1|527| +2450829|6244|1|412| +2450829|6247|1|328| +2450829|6248|1|823| +2450829|6250|1|959| +2450829|6253|1|721| +2450829|6254|1|485| +2450829|6256|1|247| +2450829|6259|1|493| +2450829|6260|1|334| +2450829|6262|1|229| +2450829|6265|1|735| +2450829|6266|1|277| +2450829|6268|1|728| +2450829|6271|1|90| +2450829|6272|1|552| +2450829|6274|1|133| +2450829|6277|1|922| +2450829|6278|1|807| +2450829|6280|1|745| +2450829|6283|1|255| +2450829|6284|1|911| +2450829|6286|1|84| +2450829|6289|1|199| +2450829|6290|1|698| +2450829|6292|1|250| +2450829|6295|1|146| +2450829|6296|1|539| +2450829|6298|1|766| +2450829|6301|1|409| +2450829|6302|1|922| +2450829|6304|1|209| +2450829|6307|1|164| +2450829|6308|1|879| +2450829|6310|1|554| +2450829|6313|1|968| +2450829|6314|1|103| +2450829|6316|1|444| +2450829|6319|1|246| +2450829|6320|1|454| +2450829|6322|1|648| +2450829|6325|1|842| +2450829|6326|1|784| +2450829|6328|1|691| +2450829|6331|1|524| +2450829|6332|1|231| +2450829|6334|1|476| +2450829|6337|1|18| +2450829|6338|1|582| +2450829|6340|1|316| +2450829|6343|1|344| +2450829|6344|1|138| +2450829|6346|1|871| +2450829|6349|1|10| +2450829|6350|1|813| +2450829|6352|1|120| +2450829|6355|1|| +2450829|6356|1|562| +2450829|6358|1|617| +2450829|6361|1|356| +2450829|6362|1|961| +2450829|6364|1|| +2450829|6367|1|419| +2450829|6368|1|838| +2450829|6370|1|932| +2450829|6373|1|753| +2450829|6374|1|848| +2450829|6376|1|721| +2450829|6379|1|482| +2450829|6380|1|882| +2450829|6382|1|468| +2450829|6385|1|552| +2450829|6386|1|696| +2450829|6388|1|611| +2450829|6391|1|735| +2450829|6392|1|835| +2450829|6394|1|679| +2450829|6397|1|134| +2450829|6398|1|375| +2450829|6400|1|833| +2450829|6403|1|102| +2450829|6404|1|870| +2450829|6406|1|989| +2450829|6409|1|974| +2450829|6410|1|596| +2450829|6412|1|231| +2450829|6415|1|192| +2450829|6416|1|592| +2450829|6418|1|870| +2450829|6421|1|793| +2450829|6422|1|166| +2450829|6424|1|797| +2450829|6427|1|43| +2450829|6428|1|610| +2450829|6430|1|898| +2450829|6433|1|146| +2450829|6434|1|740| +2450829|6436|1|676| +2450829|6439|1|781| +2450829|6440|1|597| +2450829|6442|1|305| +2450829|6445|1|543| +2450829|6446|1|270| +2450829|6448|1|38| +2450829|6451|1|565| +2450829|6452|1|478| +2450829|6454|1|375| +2450829|6457|1|128| +2450829|6458|1|452| +2450829|6460|1|845| +2450829|6463|1|688| +2450829|6464|1|449| +2450829|6466|1|427| +2450829|6469|1|715| +2450829|6470|1|915| +2450829|6472|1|22| +2450829|6475|1|165| +2450829|6476|1|684| +2450829|6478|1|612| +2450829|6481|1|593| +2450829|6482|1|576| +2450829|6484|1|715| +2450829|6487|1|479| +2450829|6488|1|138| +2450829|6490|1|176| +2450829|6493|1|578| +2450829|6494|1|474| +2450829|6496|1|311| +2450829|6499|1|611| +2450829|6500|1|157| +2450829|6502|1|413| +2450829|6505|1|223| +2450829|6506|1|703| +2450829|6508|1|686| +2450829|6511|1|35| +2450829|6512|1|574| +2450829|6514|1|206| +2450829|6517|1|40| +2450829|6518|1|736| +2450829|6520|1|238| +2450829|6523|1|227| +2450829|6524|1|805| +2450829|6526|1|| +2450829|6529|1|652| +2450829|6530|1|302| +2450829|6532|1|67| +2450829|6535|1|471| +2450829|6536|1|665| +2450829|6538|1|342| +2450829|6541|1|913| +2450829|6542|1|342| +2450829|6544|1|6| +2450829|6547|1|130| +2450829|6548|1|126| +2450829|6550|1|905| +2450829|6553|1|643| +2450829|6554|1|218| +2450829|6556|1|220| +2450829|6559|1|875| +2450829|6560|1|431| +2450829|6562|1|312| +2450829|6565|1|469| +2450829|6566|1|982| +2450829|6568|1|68| +2450829|6571|1|344| +2450829|6572|1|227| +2450829|6574|1|405| +2450829|6577|1|256| +2450829|6578|1|155| +2450829|6580|1|540| +2450829|6583|1|748| +2450829|6584|1|273| +2450829|6586|1|618| +2450829|6589|1|269| +2450829|6590|1|761| +2450829|6592|1|503| +2450829|6595|1|476| +2450829|6596|1|118| +2450829|6598|1|704| +2450829|6601|1|557| +2450829|6602|1|23| +2450829|6604|1|544| +2450829|6607|1|29| +2450829|6608|1|599| +2450829|6610|1|81| +2450829|6613|1|847| +2450829|6614|1|971| +2450829|6616|1|329| +2450829|6619|1|181| +2450829|6620|1|9| +2450829|6622|1|563| +2450829|6625|1|505| +2450829|6626|1|80| +2450829|6628|1|| +2450829|6631|1|641| +2450829|6632|1|56| +2450829|6634|1|780| +2450829|6637|1|783| +2450829|6638|1|203| +2450829|6640|1|143| +2450829|6643|1|907| +2450829|6644|1|129| +2450829|6646|1|895| +2450829|6649|1|830| +2450829|6650|1|746| +2450829|6652|1|67| +2450829|6655|1|436| +2450829|6656|1|520| +2450829|6658|1|891| +2450829|6661|1|261| +2450829|6662|1|207| +2450829|6664|1|454| +2450829|6667|1|553| +2450829|6668|1|527| +2450829|6670|1|493| +2450829|6673|1|141| +2450829|6674|1|1| +2450829|6676|1|597| +2450829|6679|1|230| +2450829|6680|1|102| +2450829|6682|1|568| +2450829|6685|1|213| +2450829|6686|1|108| +2450829|6688|1|754| +2450829|6691|1|839| +2450829|6692|1|839| +2450829|6694|1|136| +2450829|6697|1|975| +2450829|6698|1|367| +2450829|6700|1|371| +2450829|6703|1|203| +2450829|6704|1|658| +2450829|6706|1|475| +2450829|6709|1|956| +2450829|6710|1|857| +2450829|6712|1|624| +2450829|6715|1|260| +2450829|6716|1|181| +2450829|6718|1|871| +2450829|6721|1|583| +2450829|6722|1|115| +2450829|6724|1|511| +2450829|6727|1|694| +2450829|6728|1|739| +2450829|6730|1|359| +2450829|6733|1|910| +2450829|6734|1|488| +2450829|6736|1|185| +2450829|6739|1|182| +2450829|6740|1|847| +2450829|6742|1|728| +2450829|6745|1|919| +2450829|6746|1|537| +2450829|6748|1|590| +2450829|6751|1|277| +2450829|6752|1|98| +2450829|6754|1|729| +2450829|6757|1|391| +2450829|6758|1|56| +2450829|6760|1|654| +2450829|6763|1|900| +2450829|6764|1|245| +2450829|6766|1|655| +2450829|6769|1|139| +2450829|6770|1|615| +2450829|6772|1|| +2450829|6775|1|103| +2450829|6776|1|624| +2450829|6778|1|465| +2450829|6781|1|270| +2450829|6782|1|374| +2450829|6784|1|42| +2450829|6787|1|655| +2450829|6788|1|723| +2450829|6790|1|522| +2450829|6793|1|754| +2450829|6794|1|550| +2450829|6796|1|259| +2450829|6799|1|713| +2450829|6800|1|908| +2450829|6802|1|737| +2450829|6805|1|802| +2450829|6806|1|847| +2450829|6808|1|692| +2450829|6811|1|109| +2450829|6812|1|357| +2450829|6814|1|347| +2450829|6817|1|0| +2450829|6818|1|872| +2450829|6820|1|54| +2450829|6823|1|590| +2450829|6824|1|662| +2450829|6826|1|8| +2450829|6829|1|689| +2450829|6830|1|821| +2450829|6832|1|460| +2450829|6835|1|292| +2450829|6836|1|229| +2450829|6838|1|171| +2450829|6841|1|615| +2450829|6842|1|520| +2450829|6844|1|619| +2450829|6847|1|434| +2450829|6848|1|407| +2450829|6850|1|908| +2450829|6853|1|397| +2450829|6854|1|760| +2450829|6856|1|820| +2450829|6859|1|770| +2450829|6860|1|581| +2450829|6862|1|990| +2450829|6865|1|157| +2450829|6866|1|425| +2450829|6868|1|288| +2450829|6871|1|960| +2450829|6872|1|380| +2450829|6874|1|998| +2450829|6877|1|185| +2450829|6878|1|607| +2450829|6880|1|78| +2450829|6883|1|933| +2450829|6884|1|994| +2450829|6886|1|670| +2450829|6889|1|513| +2450829|6890|1|879| +2450829|6892|1|560| +2450829|6895|1|908| +2450829|6896|1|486| +2450829|6898|1|600| +2450829|6901|1|110| +2450829|6902|1|665| +2450829|6904|1|667| +2450829|6907|1|590| +2450829|6908|1|239| +2450829|6910|1|394| +2450829|6913|1|874| +2450829|6914|1|2| +2450829|6916|1|372| +2450829|6919|1|369| +2450829|6920|1|819| +2450829|6922|1|542| +2450829|6925|1|678| +2450829|6926|1|876| +2450829|6928|1|771| +2450829|6931|1|442| +2450829|6932|1|686| +2450829|6934|1|337| +2450829|6937|1|414| +2450829|6938|1|164| +2450829|6940|1|575| +2450829|6943|1|157| +2450829|6944|1|181| +2450829|6946|1|826| +2450829|6949|1|585| +2450829|6950|1|688| +2450829|6952|1|928| +2450829|6955|1|835| +2450829|6956|1|598| +2450829|6958|1|242| +2450829|6961|1|789| +2450829|6962|1|84| +2450829|6964|1|321| +2450829|6967|1|201| +2450829|6968|1|167| +2450829|6970|1|975| +2450829|6973|1|662| +2450829|6974|1|133| +2450829|6976|1|425| +2450829|6979|1|552| +2450829|6980|1|425| +2450829|6982|1|917| +2450829|6985|1|543| +2450829|6986|1|396| +2450829|6988|1|722| +2450829|6991|1|749| +2450829|6992|1|904| +2450829|6994|1|| +2450829|6997|1|489| +2450829|6998|1|372| +2450829|7000|1|833| +2450829|7003|1|501| +2450829|7004|1|273| +2450829|7006|1|642| +2450829|7009|1|223| +2450829|7010|1|683| +2450829|7012|1|580| +2450829|7015|1|907| +2450829|7016|1|758| +2450829|7018|1|67| +2450829|7021|1|976| +2450829|7022|1|845| +2450829|7024|1|451| +2450829|7027|1|452| +2450829|7028|1|83| +2450829|7030|1|830| +2450829|7033|1|654| +2450829|7034|1|401| +2450829|7036|1|724| +2450829|7039|1|159| +2450829|7040|1|162| +2450829|7042|1|762| +2450829|7045|1|155| +2450829|7046|1|589| +2450829|7048|1|666| +2450829|7051|1|564| +2450829|7052|1|996| +2450829|7054|1|332| +2450829|7057|1|26| +2450829|7058|1|389| +2450829|7060|1|947| +2450829|7063|1|354| +2450829|7064|1|400| +2450829|7066|1|| +2450829|7069|1|50| +2450829|7070|1|466| +2450829|7072|1|537| +2450829|7075|1|745| +2450829|7076|1|134| +2450829|7078|1|319| +2450829|7081|1|7| +2450829|7082|1|| +2450829|7084|1|161| +2450829|7087|1|725| +2450829|7088|1|677| +2450829|7090|1|845| +2450829|7093|1|647| +2450829|7094|1|599| +2450829|7096|1|791| +2450829|7099|1|273| +2450829|7100|1|296| +2450829|7102|1|426| +2450829|7105|1|386| +2450829|7106|1|| +2450829|7108|1|14| +2450829|7111|1|73| +2450829|7112|1|575| +2450829|7114|1|1000| +2450829|7117|1|276| +2450829|7118|1|606| +2450829|7120|1|599| +2450829|7123|1|338| +2450829|7124|1|734| +2450829|7126|1|895| +2450829|7129|1|651| +2450829|7130|1|583| +2450829|7132|1|657| +2450829|7135|1|597| +2450829|7136|1|207| +2450829|7138|1|127| +2450829|7141|1|247| +2450829|7142|1|626| +2450829|7144|1|869| +2450829|7147|1|854| +2450829|7148|1|449| +2450829|7150|1|724| +2450829|7153|1|668| +2450829|7154|1|317| +2450829|7156|1|16| +2450829|7159|1|676| +2450829|7160|1|463| +2450829|7162|1|844| +2450829|7165|1|685| +2450829|7166|1|364| +2450829|7168|1|968| +2450829|7171|1|935| +2450829|7172|1|363| +2450829|7174|1|589| +2450829|7177|1|136| +2450829|7178|1|105| +2450829|7180|1|336| +2450829|7183|1|594| +2450829|7184|1|604| +2450829|7186|1|183| +2450829|7189|1|23| +2450829|7190|1|617| +2450829|7192|1|488| +2450829|7195|1|| +2450829|7196|1|425| +2450829|7198|1|394| +2450829|7201|1|316| +2450829|7202|1|103| +2450829|7204|1|758| +2450829|7207|1|407| +2450829|7208|1|25| +2450829|7210|1|892| +2450829|7213|1|332| +2450829|7214|1|314| +2450829|7216|1|544| +2450829|7219|1|655| +2450829|7220|1|311| +2450829|7222|1|9| +2450829|7225|1|293| +2450829|7226|1|163| +2450829|7228|1|13| +2450829|7231|1|992| +2450829|7232|1|20| +2450829|7234|1|341| +2450829|7237|1|894| +2450829|7238|1|728| +2450829|7240|1|527| +2450829|7243|1|378| +2450829|7244|1|248| +2450829|7246|1|984| +2450829|7249|1|628| +2450829|7250|1|590| +2450829|7252|1|| +2450829|7255|1|841| +2450829|7256|1|181| +2450829|7258|1|990| +2450829|7261|1|361| +2450829|7262|1|943| +2450829|7264|1|547| +2450829|7267|1|403| +2450829|7268|1|698| +2450829|7270|1|615| +2450829|7273|1|737| +2450829|7274|1|15| +2450829|7276|1|371| +2450829|7279|1|663| +2450829|7280|1|579| +2450829|7282|1|895| +2450829|7285|1|335| +2450829|7286|1|619| +2450829|7288|1|459| +2450829|7291|1|501| +2450829|7292|1|798| +2450829|7294|1|114| +2450829|7297|1|272| +2450829|7298|1|802| +2450829|7300|1|180| +2450829|7303|1|619| +2450829|7304|1|582| +2450829|7306|1|340| +2450829|7309|1|| +2450829|7310|1|385| +2450829|7312|1|733| +2450829|7315|1|766| +2450829|7316|1|667| +2450829|7318|1|822| +2450829|7321|1|855| +2450829|7322|1|984| +2450829|7324|1|407| +2450829|7327|1|274| +2450829|7328|1|104| +2450829|7330|1|739| +2450829|7333|1|586| +2450829|7334|1|799| +2450829|7336|1|181| +2450829|7339|1|197| +2450829|7340|1|895| +2450829|7342|1|187| +2450829|7345|1|899| +2450829|7346|1|439| +2450829|7348|1|393| +2450829|7351|1|103| +2450829|7352|1|924| +2450829|7354|1|778| +2450829|7357|1|712| +2450829|7358|1|692| +2450829|7360|1|486| +2450829|7363|1|189| +2450829|7364|1|466| +2450829|7366|1|504| +2450829|7369|1|23| +2450829|7370|1|602| +2450829|7372|1|852| +2450829|7375|1|262| +2450829|7376|1|31| +2450829|7378|1|526| +2450829|7381|1|663| +2450829|7382|1|639| +2450829|7384|1|123| +2450829|7387|1|739| +2450829|7388|1|183| +2450829|7390|1|288| +2450829|7393|1|13| +2450829|7394|1|958| +2450829|7396|1|871| +2450829|7399|1|371| +2450829|7400|1|8| +2450829|7402|1|155| +2450829|7405|1|727| +2450829|7406|1|42| +2450829|7408|1|526| +2450829|7411|1|270| +2450829|7412|1|686| +2450829|7414|1|672| +2450829|7417|1|29| +2450829|7418|1|447| +2450829|7420|1|600| +2450829|7423|1|461| +2450829|7424|1|671| +2450829|7426|1|640| +2450829|7429|1|251| +2450829|7430|1|487| +2450829|7432|1|820| +2450829|7435|1|155| +2450829|7436|1|427| +2450829|7438|1|335| +2450829|7441|1|254| +2450829|7442|1|863| +2450829|7444|1|730| +2450829|7447|1|42| +2450829|7448|1|569| +2450829|7450|1|246| +2450829|7453|1|777| +2450829|7454|1|6| +2450829|7456|1|949| +2450829|7459|1|58| +2450829|7460|1|185| +2450829|7462|1|217| +2450829|7465|1|413| +2450829|7466|1|371| +2450829|7468|1|690| +2450829|7471|1|965| +2450829|7472|1|4| +2450829|7474|1|580| +2450829|7477|1|885| +2450829|7478|1|738| +2450829|7480|1|634| +2450829|7483|1|385| +2450829|7484|1|764| +2450829|7486|1|510| +2450829|7489|1|265| +2450829|7490|1|634| +2450829|7492|1|64| +2450829|7495|1|328| +2450829|7496|1|996| +2450829|7498|1|764| +2450829|7501|1|812| +2450829|7502|1|| +2450829|7504|1|| +2450829|7507|1|899| +2450829|7508|1|356| +2450829|7510|1|46| +2450829|7513|1|938| +2450829|7514|1|837| +2450829|7516|1|781| +2450829|7519|1|622| +2450829|7520|1|577| +2450829|7522|1|574| +2450829|7525|1|369| +2450829|7526|1|850| +2450829|7528|1|122| +2450829|7531|1|412| +2450829|7532|1|871| +2450829|7534|1|170| +2450829|7537|1|616| +2450829|7538|1|871| +2450829|7540|1|159| +2450829|7543|1|751| +2450829|7544|1|802| +2450829|7546|1|505| +2450829|7549|1|390| +2450829|7550|1|453| +2450829|7552|1|163| +2450829|7555|1|408| +2450829|7556|1|24| +2450829|7558|1|330| +2450829|7561|1|579| +2450829|7562|1|654| +2450829|7564|1|664| +2450829|7567|1|231| +2450829|7568|1|417| +2450829|7570|1|965| +2450829|7573|1|762| +2450829|7574|1|217| +2450829|7576|1|| +2450829|7579|1|273| +2450829|7580|1|485| +2450829|7582|1|169| +2450829|7585|1|870| +2450829|7586|1|648| +2450829|7588|1|946| +2450829|7591|1|375| +2450829|7592|1|233| +2450829|7594|1|214| +2450829|7597|1|721| +2450829|7598|1|15| +2450829|7600|1|93| +2450829|7603|1|541| +2450829|7604|1|81| +2450829|7606|1|870| +2450829|7609|1|834| +2450829|7610|1|697| +2450829|7612|1|819| +2450829|7615|1|505| +2450829|7616|1|729| +2450829|7618|1|435| +2450829|7621|1|711| +2450829|7622|1|697| +2450829|7624|1|639| +2450829|7627|1|521| +2450829|7628|1|559| +2450829|7630|1|976| +2450829|7633|1|647| +2450829|7634|1|473| +2450829|7636|1|798| +2450829|7639|1|718| +2450829|7640|1|91| +2450829|7642|1|223| +2450829|7645|1|976| +2450829|7646|1|754| +2450829|7648|1|715| +2450829|7651|1|272| +2450829|7652|1|227| +2450829|7654|1|967| +2450829|7657|1|502| +2450829|7658|1|793| +2450829|7660|1|156| +2450829|7663|1|118| +2450829|7664|1|210| +2450829|7666|1|208| +2450829|7669|1|278| +2450829|7670|1|611| +2450829|7672|1|216| +2450829|7675|1|919| +2450829|7676|1|952| +2450829|7678|1|9| +2450829|7681|1|932| +2450829|7682|1|134| +2450829|7684|1|837| +2450829|7687|1|| +2450829|7688|1|969| +2450829|7690|1|193| +2450829|7693|1|955| +2450829|7694|1|| +2450829|7696|1|642| +2450829|7699|1|382| +2450829|7700|1|| +2450829|7702|1|548| +2450829|7705|1|928| +2450829|7706|1|558| +2450829|7708|1|232| +2450829|7711|1|204| +2450829|7712|1|431| +2450829|7714|1|307| +2450829|7717|1|311| +2450829|7718|1|381| +2450829|7720|1|404| +2450829|7723|1|801| +2450829|7724|1|| +2450829|7726|1|86| +2450829|7729|1|723| +2450829|7730|1|763| +2450829|7732|1|757| +2450829|7735|1|975| +2450829|7736|1|30| +2450829|7738|1|680| +2450829|7741|1|391| +2450829|7742|1|935| +2450829|7744|1|764| +2450829|7747|1|625| +2450829|7748|1|334| +2450829|7750|1|685| +2450829|7753|1|337| +2450829|7754|1|244| +2450829|7756|1|227| +2450829|7759|1|20| +2450829|7760|1|802| +2450829|7762|1|741| +2450829|7765|1|457| +2450829|7766|1|850| +2450829|7768|1|275| +2450829|7771|1|226| +2450829|7772|1|479| +2450829|7774|1|765| +2450829|7777|1|721| +2450829|7778|1|855| +2450829|7780|1|166| +2450829|7783|1|576| +2450829|7784|1|180| +2450829|7786|1|790| +2450829|7789|1|644| +2450829|7790|1|30| +2450829|7792|1|441| +2450829|7795|1|668| +2450829|7796|1|514| +2450829|7798|1|723| +2450829|7801|1|552| +2450829|7802|1|644| +2450829|7804|1|296| +2450829|7807|1|973| +2450829|7808|1|| +2450829|7810|1|685| +2450829|7813|1|328| +2450829|7814|1|358| +2450829|7816|1|705| +2450829|7819|1|265| +2450829|7820|1|425| +2450829|7822|1|759| +2450829|7825|1|242| +2450829|7826|1|973| +2450829|7828|1|572| +2450829|7831|1|913| +2450829|7832|1|22| +2450829|7834|1|541| +2450829|7837|1|184| +2450829|7838|1|125| +2450829|7840|1|442| +2450829|7843|1|903| +2450829|7844|1|175| +2450829|7846|1|242| +2450829|7849|1|862| +2450829|7850|1|346| +2450829|7852|1|261| +2450829|7855|1|455| +2450829|7856|1|36| +2450829|7858|1|565| +2450829|7861|1|768| +2450829|7862|1|971| +2450829|7864|1|771| +2450829|7867|1|683| +2450829|7868|1|521| +2450829|7870|1|824| +2450829|7873|1|| +2450829|7874|1|913| +2450829|7876|1|510| +2450829|7879|1|| +2450829|7880|1|639| +2450829|7882|1|879| +2450829|7885|1|938| +2450829|7886|1|93| +2450829|7888|1|491| +2450829|7891|1|453| +2450829|7892|1|254| +2450829|7894|1|809| +2450829|7897|1|215| +2450829|7898|1|803| +2450829|7900|1|485| +2450829|7903|1|13| +2450829|7904|1|174| +2450829|7906|1|935| +2450829|7909|1|307| +2450829|7910|1|110| +2450829|7912|1|318| +2450829|7915|1|670| +2450829|7916|1|105| +2450829|7918|1|342| +2450829|7921|1|| +2450829|7922|1|927| +2450829|7924|1|| +2450829|7927|1|31| +2450829|7928|1|32| +2450829|7930|1|669| +2450829|7933|1|384| +2450829|7934|1|595| +2450829|7936|1|316| +2450829|7939|1|451| +2450829|7940|1|14| +2450829|7942|1|134| +2450829|7945|1|484| +2450829|7946|1|603| +2450829|7948|1|| +2450829|7951|1|281| +2450829|7952|1|300| +2450829|7954|1|35| +2450829|7957|1|270| +2450829|7958|1|842| +2450829|7960|1|931| +2450829|7963|1|390| +2450829|7964|1|68| +2450829|7966|1|793| +2450829|7969|1|892| +2450829|7970|1|344| +2450829|7972|1|213| +2450829|7975|1|993| +2450829|7976|1|202| +2450829|7978|1|601| +2450829|7981|1|912| +2450829|7982|1|351| +2450829|7984|1|160| +2450829|7987|1|16| +2450829|7988|1|| +2450829|7990|1|111| +2450829|7993|1|553| +2450829|7994|1|180| +2450829|7996|1|882| +2450829|7999|1|790| +2450829|8000|1|237| +2450829|8002|1|990| +2450829|8005|1|644| +2450829|8006|1|471| +2450829|8008|1|477| +2450829|8011|1|5| +2450829|8012|1|607| +2450829|8014|1|622| +2450829|8017|1|432| +2450829|8018|1|21| +2450829|8020|1|165| +2450829|8023|1|11| +2450829|8024|1|887| +2450829|8026|1|434| +2450829|8029|1|617| +2450829|8030|1|489| +2450829|8032|1|238| +2450829|8035|1|977| +2450829|8036|1|385| +2450829|8038|1|99| +2450829|8041|1|789| +2450829|8042|1|605| +2450829|8044|1|2| +2450829|8047|1|819| +2450829|8048|1|789| +2450829|8050|1|790| +2450829|8053|1|378| +2450829|8054|1|| +2450829|8056|1|498| +2450829|8059|1|150| +2450829|8060|1|880| +2450829|8062|1|213| +2450829|8065|1|842| +2450829|8066|1|38| +2450829|8068|1|| +2450829|8071|1|634| +2450829|8072|1|42| +2450829|8074|1|840| +2450829|8077|1|373| +2450829|8078|1|657| +2450829|8080|1|983| +2450829|8083|1|156| +2450829|8084|1|345| +2450829|8086|1|173| +2450829|8089|1|| +2450829|8090|1|755| +2450829|8092|1|300| +2450829|8095|1|90| +2450829|8096|1|| +2450829|8098|1|192| +2450829|8101|1|| +2450829|8102|1|114| +2450829|8104|1|79| +2450829|8107|1|674| +2450829|8108|1|187| +2450829|8110|1|388| +2450829|8113|1|94| +2450829|8114|1|374| +2450829|8116|1|338| +2450829|8119|1|899| +2450829|8120|1|456| +2450829|8122|1|985| +2450829|8125|1|306| +2450829|8126|1|377| +2450829|8128|1|569| +2450829|8131|1|866| +2450829|8132|1|171| +2450829|8134|1|333| +2450829|8137|1|| +2450829|8138|1|257| +2450829|8140|1|572| +2450829|8143|1|543| +2450829|8144|1|732| +2450829|8146|1|187| +2450829|8149|1|192| +2450829|8150|1|899| +2450829|8152|1|449| +2450829|8155|1|168| +2450829|8156|1|609| +2450829|8158|1|212| +2450829|8161|1|795| +2450829|8162|1|871| +2450829|8164|1|723| +2450829|8167|1|893| +2450829|8168|1|371| +2450829|8170|1|555| +2450829|8173|1|356| +2450829|8174|1|332| +2450829|8176|1|96| +2450829|8179|1|105| +2450829|8180|1|532| +2450829|8182|1|69| +2450829|8185|1|| +2450829|8186|1|| +2450829|8188|1|61| +2450829|8191|1|683| +2450829|8192|1|783| +2450829|8194|1|632| +2450829|8197|1|451| +2450829|8198|1|472| +2450829|8200|1|77| +2450829|8203|1|571| +2450829|8204|1|207| +2450829|8206|1|29| +2450829|8209|1|935| +2450829|8210|1|534| +2450829|8212|1|243| +2450829|8215|1|353| +2450829|8216|1|302| +2450829|8218|1|864| +2450829|8221|1|522| +2450829|8222|1|| +2450829|8224|1|92| +2450829|8227|1|123| +2450829|8228|1|288| +2450829|8230|1|498| +2450829|8233|1|429| +2450829|8234|1|354| +2450829|8236|1|590| +2450829|8239|1|662| +2450829|8240|1|234| +2450829|8242|1|617| +2450829|8245|1|923| +2450829|8246|1|676| +2450829|8248|1|349| +2450829|8251|1|757| +2450829|8252|1|889| +2450829|8254|1|943| +2450829|8257|1|901| +2450829|8258|1|58| +2450829|8260|1|53| +2450829|8263|1|322| +2450829|8264|1|870| +2450829|8266|1|893| +2450829|8269|1|747| +2450829|8270|1|| +2450829|8272|1|774| +2450829|8275|1|469| +2450829|8276|1|585| +2450829|8278|1|35| +2450829|8281|1|| +2450829|8282|1|912| +2450829|8284|1|71| +2450829|8287|1|893| +2450829|8288|1|834| +2450829|8290|1|308| +2450829|8293|1|199| +2450829|8294|1|390| +2450829|8296|1|374| +2450829|8299|1|196| +2450829|8300|1|741| +2450829|8302|1|722| +2450829|8305|1|682| +2450829|8306|1|846| +2450829|8308|1|34| +2450829|8311|1|22| +2450829|8312|1|922| +2450829|8314|1|394| +2450829|8317|1|31| +2450829|8318|1|111| +2450829|8320|1|627| +2450829|8323|1|72| +2450829|8324|1|| +2450829|8326|1|234| +2450829|8329|1|627| +2450829|8330|1|748| +2450829|8332|1|331| +2450829|8335|1|520| +2450829|8336|1|996| +2450829|8338|1|554| +2450829|8341|1|532| +2450829|8342|1|17| +2450829|8344|1|| +2450829|8347|1|230| +2450829|8348|1|947| +2450829|8350|1|950| +2450829|8353|1|| +2450829|8354|1|437| +2450829|8356|1|567| +2450829|8359|1|251| +2450829|8360|1|128| +2450829|8362|1|738| +2450829|8365|1|359| +2450829|8366|1|821| +2450829|8368|1|988| +2450829|8371|1|548| +2450829|8372|1|150| +2450829|8374|1|539| +2450829|8377|1|812| +2450829|8378|1|598| +2450829|8380|1|436| +2450829|8383|1|50| +2450829|8384|1|86| +2450829|8386|1|0| +2450829|8389|1|66| +2450829|8390|1|667| +2450829|8392|1|548| +2450829|8395|1|289| +2450829|8396|1|402| +2450829|8398|1|296| +2450829|8401|1|271| +2450829|8402|1|983| +2450829|8404|1|561| +2450829|8407|1|225| +2450829|8408|1|191| +2450829|8410|1|819| +2450829|8413|1|535| +2450829|8414|1|942| +2450829|8416|1|214| +2450829|8419|1|800| +2450829|8420|1|504| +2450829|8422|1|363| +2450829|8425|1|999| +2450829|8426|1|342| +2450829|8428|1|631| +2450829|8431|1|746| +2450829|8432|1|665| +2450829|8434|1|798| +2450829|8437|1|339| +2450829|8438|1|813| +2450829|8440|1|637| +2450829|8443|1|170| +2450829|8444|1|958| +2450829|8446|1|249| +2450829|8449|1|29| +2450829|8450|1|300| +2450829|8452|1|334| +2450829|8455|1|718| +2450829|8456|1|504| +2450829|8458|1|257| +2450829|8461|1|| +2450829|8462|1|272| +2450829|8464|1|662| +2450829|8467|1|439| +2450829|8468|1|872| +2450829|8470|1|698| +2450829|8473|1|539| +2450829|8474|1|867| +2450829|8476|1|826| +2450829|8479|1|101| +2450829|8480|1|572| +2450829|8482|1|120| +2450829|8485|1|203| +2450829|8486|1|320| +2450829|8488|1|26| +2450829|8491|1|247| +2450829|8492|1|876| +2450829|8494|1|207| +2450829|8497|1|42| +2450829|8498|1|23| +2450829|8500|1|642| +2450829|8503|1|367| +2450829|8504|1|13| +2450829|8506|1|931| +2450829|8509|1|63| +2450829|8510|1|910| +2450829|8512|1|694| +2450829|8515|1|610| +2450829|8516|1|796| +2450829|8518|1|490| +2450829|8521|1|618| +2450829|8522|1|937| +2450829|8524|1|833| +2450829|8527|1|735| +2450829|8528|1|438| +2450829|8530|1|82| +2450829|8533|1|659| +2450829|8534|1|509| +2450829|8536|1|275| +2450829|8539|1|742| +2450829|8540|1|725| +2450829|8542|1|665| +2450829|8545|1|579| +2450829|8546|1|625| +2450829|8548|1|76| +2450829|8551|1|651| +2450829|8552|1|224| +2450829|8554|1|870| +2450829|8557|1|511| +2450829|8558|1|229| +2450829|8560|1|24| +2450829|8563|1|640| +2450829|8564|1|467| +2450829|8566|1|982| +2450829|8569|1|773| +2450829|8570|1|485| +2450829|8572|1|796| +2450829|8575|1|569| +2450829|8576|1|294| +2450829|8578|1|887| +2450829|8581|1|| +2450829|8582|1|457| +2450829|8584|1|883| +2450829|8587|1|379| +2450829|8588|1|| +2450829|8590|1|726| +2450829|8593|1|243| +2450829|8594|1|316| +2450829|8596|1|828| +2450829|8599|1|731| +2450829|8600|1|46| +2450829|8602|1|989| +2450829|8605|1|| +2450829|8606|1|226| +2450829|8608|1|321| +2450829|8611|1|308| +2450829|8612|1|218| +2450829|8614|1|115| +2450829|8617|1|452| +2450829|8618|1|197| +2450829|8620|1|713| +2450829|8623|1|703| +2450829|8624|1|280| +2450829|8626|1|457| +2450829|8629|1|681| +2450829|8630|1|597| +2450829|8632|1|286| +2450829|8635|1|910| +2450829|8636|1|978| +2450829|8638|1|407| +2450829|8641|1|926| +2450829|8642|1|388| +2450829|8644|1|260| +2450829|8647|1|25| +2450829|8648|1|262| +2450829|8650|1|637| +2450829|8653|1|342| +2450829|8654|1|632| +2450829|8656|1|495| +2450829|8659|1|212| +2450829|8660|1|744| +2450829|8662|1|831| +2450829|8665|1|306| +2450829|8666|1|761| +2450829|8668|1|315| +2450829|8671|1|990| +2450829|8672|1|416| +2450829|8674|1|546| +2450829|8677|1|386| +2450829|8678|1|119| +2450829|8680|1|505| +2450829|8683|1|918| +2450829|8684|1|908| +2450829|8686|1|618| +2450829|8689|1|65| +2450829|8690|1|542| +2450829|8692|1|583| +2450829|8695|1|960| +2450829|8696|1|788| +2450829|8698|1|245| +2450829|8701|1|997| +2450829|8702|1|340| +2450829|8704|1|944| +2450829|8707|1|121| +2450829|8708|1|80| +2450829|8710|1|505| +2450829|8713|1|459| +2450829|8714|1|150| +2450829|8716|1|573| +2450829|8719|1|909| +2450829|8720|1|1000| +2450829|8722|1|117| +2450829|8725|1|144| +2450829|8726|1|801| +2450829|8728|1|798| +2450829|8731|1|473| +2450829|8732|1|8| +2450829|8734|1|273| +2450829|8737|1|598| +2450829|8738|1|189| +2450829|8740|1|247| +2450829|8743|1|688| +2450829|8744|1|320| +2450829|8746|1|562| +2450829|8749|1|788| +2450829|8750|1|495| +2450829|8752|1|748| +2450829|8755|1|773| +2450829|8756|1|818| +2450829|8758|1|| +2450829|8761|1|274| +2450829|8762|1|305| +2450829|8764|1|137| +2450829|8767|1|580| +2450829|8768|1|785| +2450829|8770|1|960| +2450829|8773|1|418| +2450829|8774|1|180| +2450829|8776|1|571| +2450829|8779|1|| +2450829|8780|1|933| +2450829|8782|1|185| +2450829|8785|1|874| +2450829|8786|1|656| +2450829|8788|1|65| +2450829|8791|1|464| +2450829|8792|1|846| +2450829|8794|1|990| +2450829|8797|1|| +2450829|8798|1|414| +2450829|8800|1|321| +2450829|8803|1|67| +2450829|8804|1|341| +2450829|8806|1|803| +2450829|8809|1|319| +2450829|8810|1|571| +2450829|8812|1|30| +2450829|8815|1|599| +2450829|8816|1|869| +2450829|8818|1|660| +2450829|8821|1|138| +2450829|8822|1|851| +2450829|8824|1|355| +2450829|8827|1|910| +2450829|8828|1|438| +2450829|8830|1|492| +2450829|8833|1|447| +2450829|8834|1|480| +2450829|8836|1|| +2450829|8839|1|136| +2450829|8840|1|463| +2450829|8842|1|465| +2450829|8845|1|661| +2450829|8846|1|718| +2450829|8848|1|631| +2450829|8851|1|453| +2450829|8852|1|891| +2450829|8854|1|261| +2450829|8857|1|219| +2450829|8858|1|564| +2450829|8860|1|81| +2450829|8863|1|500| +2450829|8864|1|177| +2450829|8866|1|565| +2450829|8869|1|177| +2450829|8870|1|645| +2450829|8872|1|176| +2450829|8875|1|609| +2450829|8876|1|649| +2450829|8878|1|107| +2450829|8881|1|883| +2450829|8882|1|286| +2450829|8884|1|663| +2450829|8887|1|334| +2450829|8888|1|348| +2450829|8890|1|798| +2450829|8893|1|288| +2450829|8894|1|532| +2450829|8896|1|457| +2450829|8899|1|690| +2450829|8900|1|408| +2450829|8902|1|597| +2450829|8905|1|467| +2450829|8906|1|917| +2450829|8908|1|440| +2450829|8911|1|992| +2450829|8912|1|548| +2450829|8914|1|921| +2450829|8917|1|246| +2450829|8918|1|208| +2450829|8920|1|707| +2450829|8923|1|939| +2450829|8924|1|912| +2450829|8926|1|663| +2450829|8929|1|935| +2450829|8930|1|856| +2450829|8932|1|583| +2450829|8935|1|568| +2450829|8936|1|| +2450829|8938|1|774| +2450829|8941|1|732| +2450829|8942|1|502| +2450829|8944|1|912| +2450829|8947|1|195| +2450829|8948|1|617| +2450829|8950|1|472| +2450829|8953|1|242| +2450829|8954|1|570| +2450829|8956|1|944| +2450829|8959|1|797| +2450829|8960|1|151| +2450829|8962|1|224| +2450829|8965|1|536| +2450829|8966|1|504| +2450829|8968|1|922| +2450829|8971|1|942| +2450829|8972|1|347| +2450829|8974|1|85| +2450829|8977|1|57| +2450829|8978|1|595| +2450829|8980|1|913| +2450829|8983|1|959| +2450829|8984|1|157| +2450829|8986|1|39| +2450829|8989|1|536| +2450829|8990|1|2| +2450829|8992|1|679| +2450829|8995|1|126| +2450829|8996|1|326| +2450829|8998|1|526| +2450829|9001|1|753| +2450829|9002|1|473| +2450829|9004|1|932| +2450829|9007|1|667| +2450829|9008|1|62| +2450829|9010|1|843| +2450829|9013|1|125| +2450829|9014|1|498| +2450829|9016|1|963| +2450829|9019|1|933| +2450829|9020|1|396| +2450829|9022|1|42| +2450829|9025|1|863| +2450829|9026|1|85| +2450829|9028|1|565| +2450829|9031|1|403| +2450829|9032|1|725| +2450829|9034|1|975| +2450829|9037|1|977| +2450829|9038|1|| +2450829|9040|1|63| +2450829|9043|1|95| +2450829|9044|1|868| +2450829|9046|1|497| +2450829|9049|1|478| +2450829|9050|1|772| +2450829|9052|1|136| +2450829|9055|1|394| +2450829|9056|1|111| +2450829|9058|1|702| +2450829|9061|1|164| +2450829|9062|1|687| +2450829|9064|1|155| +2450829|9067|1|88| +2450829|9068|1|716| +2450829|9070|1|823| +2450829|9073|1|639| +2450829|9074|1|142| +2450829|9076|1|374| +2450829|9079|1|776| +2450829|9080|1|943| +2450829|9082|1|909| +2450829|9085|1|| +2450829|9086|1|403| +2450829|9088|1|509| +2450829|9091|1|25| +2450829|9092|1|697| +2450829|9094|1|699| +2450829|9097|1|549| +2450829|9098|1|48| +2450829|9100|1|662| +2450829|9103|1|285| +2450829|9104|1|29| +2450829|9106|1|935| +2450829|9109|1|731| +2450829|9110|1|626| +2450829|9112|1|| +2450829|9115|1|71| +2450829|9116|1|924| +2450829|9118|1|665| +2450829|9121|1|334| +2450829|9122|1|698| +2450829|9124|1|897| +2450829|9127|1|863| +2450829|9128|1|538| +2450829|9130|1|| +2450829|9133|1|614| +2450829|9134|1|796| +2450829|9136|1|737| +2450829|9139|1|800| +2450829|9140|1|331| +2450829|9142|1|546| +2450829|9145|1|947| +2450829|9146|1|544| +2450829|9148|1|590| +2450829|9151|1|483| +2450829|9152|1|216| +2450829|9154|1|362| +2450829|9157|1|| +2450829|9158|1|742| +2450829|9160|1|24| +2450829|9163|1|907| +2450829|9164|1|85| +2450829|9166|1|531| +2450829|9169|1|605| +2450829|9170|1|103| +2450829|9172|1|521| +2450829|9175|1|269| +2450829|9176|1|307| +2450829|9178|1|40| +2450829|9181|1|| +2450829|9182|1|154| +2450829|9184|1|727| +2450829|9187|1|137| +2450829|9188|1|19| +2450829|9190|1|555| +2450829|9193|1|771| +2450829|9194|1|312| +2450829|9196|1|| +2450829|9199|1|286| +2450829|9200|1|325| +2450829|9202|1|79| +2450829|9205|1|284| +2450829|9206|1|677| +2450829|9208|1|793| +2450829|9211|1|523| +2450829|9212|1|765| +2450829|9214|1|319| +2450829|9217|1|461| +2450829|9218|1|700| +2450829|9220|1|| +2450829|9223|1|470| +2450829|9224|1|165| +2450829|9226|1|159| +2450829|9229|1|744| +2450829|9230|1|777| +2450829|9232|1|901| +2450829|9235|1|729| +2450829|9236|1|742| +2450829|9238|1|605| +2450829|9241|1|212| +2450829|9242|1|283| +2450829|9244|1|231| +2450829|9247|1|531| +2450829|9248|1|534| +2450829|9250|1|902| +2450829|9253|1|314| +2450829|9254|1|58| +2450829|9256|1|| +2450829|9259|1|254| +2450829|9260|1|280| +2450829|9262|1|137| +2450829|9265|1|354| +2450829|9266|1|821| +2450829|9268|1|53| +2450829|9271|1|225| +2450829|9272|1|307| +2450829|9274|1|880| +2450829|9277|1|| +2450829|9278|1|928| +2450829|9280|1|652| +2450829|9283|1|476| +2450829|9284|1|721| +2450829|9286|1|119| +2450829|9289|1|316| +2450829|9290|1|55| +2450829|9292|1|208| +2450829|9295|1|330| +2450829|9296|1|754| +2450829|9298|1|| +2450829|9301|1|251| +2450829|9302|1|454| +2450829|9304|1|800| +2450829|9307|1|530| +2450829|9308|1|894| +2450829|9310|1|628| +2450829|9313|1|95| +2450829|9314|1|791| +2450829|9316|1|144| +2450829|9319|1|809| +2450829|9320|1|671| +2450829|9322|1|98| +2450829|9325|1|876| +2450829|9326|1|923| +2450829|9328|1|194| +2450829|9331|1|393| +2450829|9332|1|801| +2450829|9334|1|541| +2450829|9337|1|997| +2450829|9338|1|869| +2450829|9340|1|661| +2450829|9343|1|679| +2450829|9344|1|922| +2450829|9346|1|897| +2450829|9349|1|272| +2450829|9350|1|| +2450829|9352|1|955| +2450829|9355|1|324| +2450829|9356|1|836| +2450829|9358|1|453| +2450829|9361|1|618| +2450829|9362|1|81| +2450829|9364|1|689| +2450829|9367|1|655| +2450829|9368|1|244| +2450829|9370|1|881| +2450829|9373|1|| +2450829|9374|1|422| +2450829|9376|1|229| +2450829|9379|1|76| +2450829|9380|1|380| +2450829|9382|1|425| +2450829|9385|1|522| +2450829|9386|1|43| +2450829|9388|1|235| +2450829|9391|1|775| +2450829|9392|1|984| +2450829|9394|1|651| +2450829|9397|1|464| +2450829|9398|1|310| +2450829|9400|1|366| +2450829|9403|1|368| +2450829|9404|1|490| +2450829|9406|1|420| +2450829|9409|1|544| +2450829|9410|1|342| +2450829|9412|1|870| +2450829|9415|1|23| +2450829|9416|1|| +2450829|9418|1|617| +2450829|9421|1|305| +2450829|9422|1|939| +2450829|9424|1|746| +2450829|9427|1|100| +2450829|9428|1|947| +2450829|9430|1|691| +2450829|9433|1|582| +2450829|9434|1|508| +2450829|9436|1|854| +2450829|9439|1|191| +2450829|9440|1|88| +2450829|9442|1|155| +2450829|9445|1|472| +2450829|9446|1|| +2450829|9448|1|1000| +2450829|9451|1|247| +2450829|9452|1|50| +2450829|9454|1|855| +2450829|9457|1|954| +2450829|9458|1|624| +2450829|9460|1|249| +2450829|9463|1|279| +2450829|9464|1|854| +2450829|9466|1|607| +2450829|9469|1|376| +2450829|9470|1|109| +2450829|9472|1|251| +2450829|9475|1|194| +2450829|9476|1|955| +2450829|9478|1|223| +2450829|9481|1|835| +2450829|9482|1|341| +2450829|9484|1|997| +2450829|9487|1|45| +2450829|9488|1|528| +2450829|9490|1|913| +2450829|9493|1|199| +2450829|9494|1|908| +2450829|9496|1|835| +2450829|9499|1|688| +2450829|9500|1|811| +2450829|9502|1|| +2450829|9505|1|| +2450829|9506|1|890| +2450829|9508|1|214| +2450829|9511|1|21| +2450829|9512|1|277| +2450829|9514|1|431| +2450829|9517|1|395| +2450829|9518|1|402| +2450829|9520|1|352| +2450829|9523|1|788| +2450829|9524|1|781| +2450829|9526|1|919| +2450829|9529|1|2| +2450829|9530|1|974| +2450829|9532|1|790| +2450829|9535|1|953| +2450829|9536|1|68| +2450829|9538|1|139| +2450829|9541|1|548| +2450829|9542|1|957| +2450829|9544|1|| +2450829|9547|1|731| +2450829|9548|1|329| +2450829|9550|1|1| +2450829|9553|1|436| +2450829|9554|1|134| +2450829|9556|1|591| +2450829|9559|1|681| +2450829|9560|1|922| +2450829|9562|1|358| +2450829|9565|1|759| +2450829|9566|1|930| +2450829|9568|1|471| +2450829|9571|1|238| +2450829|9572|1|659| +2450829|9574|1|856| +2450829|9577|1|942| +2450829|9578|1|297| +2450829|9580|1|56| +2450829|9583|1|211| +2450829|9584|1|61| +2450829|9586|1|123| +2450829|9589|1|351| +2450829|9590|1|900| +2450829|9592|1|709| +2450829|9595|1|6| +2450829|9596|1|505| +2450829|9598|1|406| +2450829|9601|1|371| +2450829|9602|1|580| +2450829|9604|1|578| +2450829|9607|1|262| +2450829|9608|1|610| +2450829|9610|1|806| +2450829|9613|1|603| +2450829|9614|1|306| +2450829|9616|1|672| +2450829|9619|1|176| +2450829|9620|1|500| +2450829|9622|1|804| +2450829|9625|1|921| +2450829|9626|1|107| +2450829|9628|1|420| +2450829|9631|1|928| +2450829|9632|1|425| +2450829|9634|1|679| +2450829|9637|1|946| +2450829|9638|1|778| +2450829|9640|1|27| +2450829|9643|1|911| +2450829|9644|1|304| +2450829|9646|1|| +2450829|9649|1|599| +2450829|9650|1|547| +2450829|9652|1|216| +2450829|9655|1|168| +2450829|9656|1|457| +2450829|9658|1|896| +2450829|9661|1|| +2450829|9662|1|79| +2450829|9664|1|105| +2450829|9667|1|965| +2450829|9668|1|166| +2450829|9670|1|900| +2450829|9673|1|276| +2450829|9674|1|| +2450829|9676|1|727| +2450829|9679|1|644| +2450829|9680|1|639| +2450829|9682|1|| +2450829|9685|1|460| +2450829|9686|1|133| +2450829|9688|1|| +2450829|9691|1|750| +2450829|9692|1|842| +2450829|9694|1|675| +2450829|9697|1|388| +2450829|9698|1|102| +2450829|9700|1|172| +2450829|9703|1|524| +2450829|9704|1|825| +2450829|9706|1|974| +2450829|9709|1|| +2450829|9710|1|| +2450829|9712|1|787| +2450829|9715|1|523| +2450829|9716|1|821| +2450829|9718|1|349| +2450829|9721|1|229| +2450829|9722|1|947| +2450829|9724|1|255| +2450829|9727|1|830| +2450829|9728|1|755| +2450829|9730|1|765| +2450829|9733|1|75| +2450829|9734|1|924| +2450829|9736|1|368| +2450829|9739|1|20| +2450829|9740|1|553| +2450829|9742|1|821| +2450829|9745|1|179| +2450829|9746|1|833| +2450829|9748|1|906| +2450829|9751|1|33| +2450829|9752|1|949| +2450829|9754|1|364| +2450829|9757|1|992| +2450829|9758|1|917| +2450829|9760|1|148| +2450829|9763|1|| +2450829|9764|1|39| +2450829|9766|1|458| +2450829|9769|1|653| +2450829|9770|1|860| +2450829|9772|1|| +2450829|9775|1|533| +2450829|9776|1|639| +2450829|9778|1|729| +2450829|9781|1|680| +2450829|9782|1|720| +2450829|9784|1|11| +2450829|9787|1|550| +2450829|9788|1|427| +2450829|9790|1|542| +2450829|9793|1|662| +2450829|9794|1|| +2450829|9796|1|83| +2450829|9799|1|649| +2450829|9800|1|354| +2450829|9802|1|176| +2450829|9805|1|713| +2450829|9806|1|637| +2450829|9808|1|663| +2450829|9811|1|213| +2450829|9812|1|34| +2450829|9814|1|540| +2450829|9817|1|728| +2450829|9818|1|88| +2450829|9820|1|816| +2450829|9823|1|816| +2450829|9824|1|477| +2450829|9826|1|86| +2450829|9829|1|592| +2450829|9830|1|525| +2450829|9832|1|588| +2450829|9835|1|379| +2450829|9836|1|256| +2450829|9838|1|368| +2450829|9841|1|840| +2450829|9842|1|208| +2450829|9844|1|479| +2450829|9847|1|146| +2450829|9848|1|894| +2450829|9850|1|433| +2450829|9853|1|752| +2450829|9854|1|391| +2450829|9856|1|401| +2450829|9859|1|983| +2450829|9860|1|184| +2450829|9862|1|268| +2450829|9865|1|505| +2450829|9866|1|139| +2450829|9868|1|936| +2450829|9871|1|930| +2450829|9872|1|929| +2450829|9874|1|927| +2450829|9877|1|964| +2450829|9878|1|956| +2450829|9880|1|353| +2450829|9883|1|205| +2450829|9884|1|996| +2450829|9886|1|333| +2450829|9889|1|94| +2450829|9890|1|596| +2450829|9892|1|362| +2450829|9895|1|215| +2450829|9896|1|192| +2450829|9898|1|333| +2450829|9901|1|952| +2450829|9902|1|451| +2450829|9904|1|166| +2450829|9907|1|414| +2450829|9908|1|194| +2450829|9910|1|331| +2450829|9913|1|799| +2450829|9914|1|| +2450829|9916|1|407| +2450829|9919|1|954| +2450829|9920|1|461| +2450829|9922|1|376| +2450829|9925|1|146| +2450829|9926|1|721| +2450829|9928|1|704| +2450829|9931|1|528| +2450829|9932|1|311| +2450829|9934|1|453| +2450829|9937|1|324| +2450829|9938|1|214| +2450829|9940|1|474| +2450829|9943|1|846| +2450829|9944|1|714| +2450829|9946|1|222| +2450829|9949|1|548| +2450829|9950|1|592| +2450829|9952|1|192| +2450829|9955|1|540| +2450829|9956|1|| +2450829|9958|1|177| +2450829|9961|1|950| +2450829|9962|1|62| +2450829|9964|1|852| +2450829|9967|1|599| +2450829|9968|1|105| +2450829|9970|1|| +2450829|9973|1|464| +2450829|9974|1|493| +2450829|9976|1|412| +2450829|9979|1|730| +2450829|9980|1|983| +2450829|9982|1|328| +2450829|9985|1|285| +2450829|9986|1|250| +2450829|9988|1|512| +2450829|9991|1|126| +2450829|9992|1|338| +2450829|9994|1|937| +2450829|9997|1|212| +2450829|9998|1|573| +2450829|10000|1|554| +2450829|10003|1|978| +2450829|10004|1|742| +2450829|10006|1|394| +2450829|10009|1|614| +2450829|10010|1|550| +2450829|10012|1|1000| +2450829|10015|1|202| +2450829|10016|1|| +2450829|10018|1|761| +2450829|10021|1|407| +2450829|10022|1|952| +2450829|10024|1|830| +2450829|10027|1|918| +2450829|10028|1|24| +2450829|10030|1|75| +2450829|10033|1|| +2450829|10034|1|917| +2450829|10036|1|979| +2450829|10039|1|444| +2450829|10040|1|426| +2450829|10042|1|931| +2450829|10045|1|839| +2450829|10046|1|564| +2450829|10048|1|169| +2450829|10051|1|39| +2450829|10052|1|47| +2450829|10054|1|101| +2450829|10057|1|612| +2450829|10058|1|52| +2450829|10060|1|| +2450829|10063|1|126| +2450829|10064|1|958| +2450829|10066|1|975| +2450829|10069|1|855| +2450829|10070|1|| +2450829|10072|1|684| +2450829|10075|1|602| +2450829|10076|1|666| +2450829|10078|1|805| +2450829|10081|1|749| +2450829|10082|1|| +2450829|10084|1|893| +2450829|10087|1|736| +2450829|10088|1|951| +2450829|10090|1|910| +2450829|10093|1|829| +2450829|10094|1|441| +2450829|10096|1|818| +2450829|10099|1|186| +2450829|10100|1|369| +2450829|10102|1|138| +2450829|10105|1|942| +2450829|10106|1|700| +2450829|10108|1|678| +2450829|10111|1|217| +2450829|10112|1|791| +2450829|10114|1|775| +2450829|10117|1|42| +2450829|10118|1|658| +2450829|10120|1|787| +2450829|10123|1|450| +2450829|10124|1|417| +2450829|10126|1|368| +2450829|10129|1|| +2450829|10130|1|19| +2450829|10132|1|139| +2450829|10135|1|192| +2450829|10136|1|923| +2450829|10138|1|351| +2450829|10141|1|172| +2450829|10142|1|213| +2450829|10144|1|561| +2450829|10147|1|813| +2450829|10148|1|947| +2450829|10150|1|571| +2450829|10153|1|591| +2450829|10154|1|291| +2450829|10156|1|526| +2450829|10159|1|729| +2450829|10160|1|722| +2450829|10162|1|74| +2450829|10165|1|58| +2450829|10166|1|683| +2450829|10168|1|966| +2450829|10171|1|205| +2450829|10172|1|| +2450829|10174|1|620| +2450829|10177|1|302| +2450829|10178|1|532| +2450829|10180|1|| +2450829|10183|1|168| +2450829|10184|1|486| +2450829|10186|1|131| +2450829|10189|1|875| +2450829|10190|1|2| +2450829|10192|1|781| +2450829|10195|1|723| +2450829|10196|1|| +2450829|10198|1|742| +2450829|10201|1|477| +2450829|10202|1|279| +2450829|10204|1|790| +2450829|10207|1|825| +2450829|10208|1|764| +2450829|10210|1|| +2450829|10213|1|683| +2450829|10214|1|852| +2450829|10216|1|436| +2450829|10219|1|609| +2450829|10220|1|893| +2450829|10222|1|732| +2450829|10225|1|642| +2450829|10226|1|853| +2450829|10228|1|185| +2450829|10231|1|823| +2450829|10232|1|773| +2450829|10234|1|687| +2450829|10237|1|344| +2450829|10238|1|756| +2450829|10240|1|386| +2450829|10243|1|41| +2450829|10244|1|155| +2450829|10246|1|668| +2450829|10249|1|803| +2450829|10250|1|325| +2450829|10252|1|145| +2450829|10255|1|741| +2450829|10256|1|757| +2450829|10258|1|986| +2450829|10261|1|822| +2450829|10262|1|246| +2450829|10264|1|188| +2450829|10267|1|36| +2450829|10268|1|933| +2450829|10270|1|160| +2450829|10273|1|645| +2450829|10274|1|564| +2450829|10276|1|749| +2450829|10279|1|4| +2450829|10280|1|661| +2450829|10282|1|148| +2450829|10285|1|291| +2450829|10286|1|530| +2450829|10288|1|514| +2450829|10291|1|850| +2450829|10292|1|891| +2450829|10294|1|987| +2450829|10297|1|272| +2450829|10298|1|844| +2450829|10300|1|| +2450829|10303|1|727| +2450829|10304|1|598| +2450829|10306|1|414| +2450829|10309|1|475| +2450829|10310|1|291| +2450829|10312|1|600| +2450829|10315|1|| +2450829|10316|1|830| +2450829|10318|1|990| +2450829|10321|1|228| +2450829|10322|1|326| +2450829|10324|1|91| +2450829|10327|1|255| +2450829|10328|1|350| +2450829|10330|1|19| +2450829|10333|1|68| +2450829|10334|1|180| +2450829|10336|1|254| +2450829|10339|1|| +2450829|10340|1|679| +2450829|10342|1|831| +2450829|10345|1|899| +2450829|10346|1|921| +2450829|10348|1|843| +2450829|10351|1|303| +2450829|10352|1|189| +2450829|10354|1|981| +2450829|10357|1|400| +2450829|10358|1|515| +2450829|10360|1|682| +2450829|10363|1|951| +2450829|10364|1|810| +2450829|10366|1|569| +2450829|10369|1|686| +2450829|10370|1|486| +2450829|10372|1|80| +2450829|10375|1|247| +2450829|10376|1|246| +2450829|10378|1|203| +2450829|10381|1|230| +2450829|10382|1|679| +2450829|10384|1|469| +2450829|10387|1|750| +2450829|10388|1|751| +2450829|10390|1|911| +2450829|10393|1|293| +2450829|10394|1|| +2450829|10396|1|70| +2450829|10399|1|230| +2450829|10400|1|824| +2450829|10402|1|337| +2450829|10405|1|338| +2450829|10406|1|476| +2450829|10408|1|591| +2450829|10411|1|175| +2450829|10412|1|158| +2450829|10414|1|591| +2450829|10417|1|368| +2450829|10418|1|292| +2450829|10420|1|11| +2450829|10423|1|976| +2450829|10424|1|162| +2450829|10426|1|651| +2450829|10429|1|436| +2450829|10430|1|570| +2450829|10432|1|794| +2450829|10435|1|969| +2450829|10436|1|144| +2450829|10438|1|389| +2450829|10441|1|874| +2450829|10442|1|780| +2450829|10444|1|749| +2450829|10447|1|848| +2450829|10448|1|275| +2450829|10450|1|889| +2450829|10453|1|981| +2450829|10454|1|625| +2450829|10456|1|198| +2450829|10459|1|804| +2450829|10460|1|619| +2450829|10462|1|771| +2450829|10465|1|74| +2450829|10466|1|207| +2450829|10468|1|578| +2450829|10471|1|447| +2450829|10472|1|996| +2450829|10474|1|659| +2450829|10477|1|| +2450829|10478|1|485| +2450829|10480|1|195| +2450829|10483|1|410| +2450829|10484|1|33| +2450829|10486|1|597| +2450829|10489|1|534| +2450829|10490|1|20| +2450829|10492|1|542| +2450829|10495|1|686| +2450829|10496|1|2| +2450829|10498|1|830| +2450829|10501|1|477| +2450829|10502|1|| +2450829|10504|1|956| +2450829|10507|1|318| +2450829|10508|1|| +2450829|10510|1|932| +2450829|10513|1|374| +2450829|10514|1|418| +2450829|10516|1|215| +2450829|10519|1|424| +2450829|10520|1|| +2450829|10522|1|405| +2450829|10525|1|111| +2450829|10526|1|162| +2450829|10528|1|788| +2450829|10531|1|981| +2450829|10532|1|440| +2450829|10534|1|436| +2450829|10537|1|867| +2450829|10538|1|476| +2450829|10540|1|459| +2450829|10543|1|624| +2450829|10544|1|358| +2450829|10546|1|545| +2450829|10549|1|843| +2450829|10550|1|605| +2450829|10552|1|639| +2450829|10555|1|| +2450829|10556|1|240| +2450829|10558|1|655| +2450829|10561|1|| +2450829|10562|1|317| +2450829|10564|1|115| +2450829|10567|1|460| +2450829|10568|1|853| +2450829|10570|1|15| +2450829|10573|1|84| +2450829|10574|1|470| +2450829|10576|1|579| +2450829|10579|1|120| +2450829|10580|1|47| +2450829|10582|1|434| +2450829|10585|1|980| +2450829|10586|1|507| +2450829|10588|1|108| +2450829|10591|1|993| +2450829|10592|1|832| +2450829|10594|1|169| +2450829|10597|1|797| +2450829|10598|1|374| +2450829|10600|1|310| +2450829|10603|1|553| +2450829|10604|1|277| +2450829|10606|1|759| +2450829|10609|1|63| +2450829|10610|1|87| +2450829|10612|1|102| +2450829|10615|1|473| +2450829|10616|1|302| +2450829|10618|1|312| +2450829|10621|1|160| +2450829|10622|1|| +2450829|10624|1|536| +2450829|10627|1|758| +2450829|10628|1|14| +2450829|10630|1|641| +2450829|10633|1|747| +2450829|10634|1|306| +2450829|10636|1|877| +2450829|10639|1|943| +2450829|10640|1|680| +2450829|10642|1|5| +2450829|10645|1|961| +2450829|10646|1|820| +2450829|10648|1|650| +2450829|10651|1|182| +2450829|10652|1|842| +2450829|10654|1|829| +2450829|10657|1|812| +2450829|10658|1|550| +2450829|10660|1|306| +2450829|10663|1|433| +2450829|10664|1|525| +2450829|10666|1|255| +2450829|10669|1|948| +2450829|10670|1|646| +2450829|10672|1|478| +2450829|10675|1|509| +2450829|10676|1|57| +2450829|10678|1|442| +2450829|10681|1|100| +2450829|10682|1|235| +2450829|10684|1|626| +2450829|10687|1|613| +2450829|10688|1|366| +2450829|10690|1|232| +2450829|10693|1|112| +2450829|10694|1|172| +2450829|10696|1|57| +2450829|10699|1|75| +2450829|10700|1|125| +2450829|10702|1|486| +2450829|10705|1|318| +2450829|10706|1|185| +2450829|10708|1|748| +2450829|10711|1|399| +2450829|10712|1|830| +2450829|10714|1|670| +2450829|10717|1|575| +2450829|10718|1|152| +2450829|10720|1|163| +2450829|10723|1|663| +2450829|10724|1|803| +2450829|10726|1|319| +2450829|10729|1|577| +2450829|10730|1|591| +2450829|10732|1|276| +2450829|10735|1|651| +2450829|10736|1|638| +2450829|10738|1|353| +2450829|10741|1|72| +2450829|10742|1|466| +2450829|10744|1|794| +2450829|10747|1|244| +2450829|10748|1|338| +2450829|10750|1|1| +2450829|10753|1|871| +2450829|10754|1|223| +2450829|10756|1|32| +2450829|10759|1|342| +2450829|10760|1|688| +2450829|10762|1|523| +2450829|10765|1|| +2450829|10766|1|791| +2450829|10768|1|926| +2450829|10771|1|246| +2450829|10772|1|838| +2450829|10774|1|834| +2450829|10777|1|471| +2450829|10778|1|531| +2450829|10780|1|742| +2450829|10783|1|951| +2450829|10784|1|452| +2450829|10786|1|455| +2450829|10789|1|307| +2450829|10790|1|790| +2450829|10792|1|541| +2450829|10795|1|563| +2450829|10796|1|70| +2450829|10798|1|654| +2450829|10801|1|480| +2450829|10802|1|180| +2450829|10804|1|178| +2450829|10807|1|852| +2450829|10808|1|460| +2450829|10810|1|492| +2450829|10813|1|704| +2450829|10814|1|6| +2450829|10816|1|248| +2450829|10819|1|23| +2450829|10820|1|503| +2450829|10822|1|577| +2450829|10825|1|56| +2450829|10826|1|486| +2450829|10828|1|74| +2450829|10831|1|901| +2450829|10832|1|897| +2450829|10834|1|780| +2450829|10837|1|676| +2450829|10838|1|203| +2450829|10840|1|89| +2450829|10843|1|774| +2450829|10844|1|469| +2450829|10846|1|426| +2450829|10849|1|538| +2450829|10850|1|765| +2450829|10852|1|| +2450829|10855|1|995| +2450829|10856|1|843| +2450829|10858|1|829| +2450829|10861|1|337| +2450829|10862|1|888| +2450829|10864|1|391| +2450829|10867|1|848| +2450829|10868|1|49| +2450829|10870|1|870| +2450829|10873|1|| +2450829|10874|1|514| +2450829|10876|1|591| +2450829|10879|1|702| +2450829|10880|1|627| +2450829|10882|1|106| +2450829|10885|1|651| +2450829|10886|1|259| +2450829|10888|1|| +2450829|10891|1|178| +2450829|10892|1|815| +2450829|10894|1|506| +2450829|10897|1|24| +2450829|10898|1|608| +2450829|10900|1|57| +2450829|10903|1|989| +2450829|10904|1|810| +2450829|10906|1|81| +2450829|10909|1|707| +2450829|10910|1|666| +2450829|10912|1|29| +2450829|10915|1|448| +2450829|10916|1|| +2450829|10918|1|94| +2450829|10921|1|992| +2450829|10922|1|1000| +2450829|10924|1|741| +2450829|10927|1|759| +2450829|10928|1|297| +2450829|10930|1|939| +2450829|10933|1|516| +2450829|10934|1|181| +2450829|10936|1|925| +2450829|10939|1|795| +2450829|10940|1|907| +2450829|10942|1|49| +2450829|10945|1|143| +2450829|10946|1|590| +2450829|10948|1|331| +2450829|10951|1|332| +2450829|10952|1|728| +2450829|10954|1|138| +2450829|10957|1|593| +2450829|10958|1|384| +2450829|10960|1|126| +2450829|10963|1|167| +2450829|10964|1|243| +2450829|10966|1|464| +2450829|10969|1|77| +2450829|10970|1|262| +2450829|10972|1|306| +2450829|10975|1|370| +2450829|10976|1|585| +2450829|10978|1|507| +2450829|10981|1|792| +2450829|10982|1|837| +2450829|10984|1|360| +2450829|10987|1|341| +2450829|10988|1|851| +2450829|10990|1|372| +2450829|10993|1|422| +2450829|10994|1|979| +2450829|10996|1|443| +2450829|10999|1|376| +2450829|11000|1|554| +2450829|11002|1|196| +2450829|11005|1|754| +2450829|11006|1|521| +2450829|11008|1|239| +2450829|11011|1|| +2450829|11012|1|362| +2450829|11014|1|687| +2450829|11017|1|440| +2450829|11018|1|654| +2450829|11020|1|207| +2450829|11023|1|| +2450829|11024|1|| +2450829|11026|1|968| +2450829|11029|1|| +2450829|11030|1|105| +2450829|11032|1|861| +2450829|11035|1|276| +2450829|11036|1|546| +2450829|11038|1|871| +2450829|11041|1|854| +2450829|11042|1|534| +2450829|11044|1|913| +2450829|11047|1|323| +2450829|11048|1|655| +2450829|11050|1|634| +2450829|11053|1|429| +2450829|11054|1|446| +2450829|11056|1|33| +2450829|11059|1|333| +2450829|11060|1|| +2450829|11062|1|373| +2450829|11065|1|117| +2450829|11066|1|875| +2450829|11068|1|854| +2450829|11071|1|257| +2450829|11072|1|474| +2450829|11074|1|582| +2450829|11077|1|295| +2450829|11078|1|699| +2450829|11080|1|914| +2450829|11083|1|67| +2450829|11084|1|519| +2450829|11086|1|463| +2450829|11089|1|923| +2450829|11090|1|745| +2450829|11092|1|348| +2450829|11095|1|678| +2450829|11096|1|159| +2450829|11098|1|210| +2450829|11101|1|435| +2450829|11102|1|732| +2450829|11104|1|947| +2450829|11107|1|123| +2450829|11108|1|417| +2450829|11110|1|870| +2450829|11113|1|| +2450829|11114|1|738| +2450829|11116|1|660| +2450829|11119|1|| +2450829|11120|1|78| +2450829|11122|1|868| +2450829|11125|1|858| +2450829|11126|1|250| +2450829|11128|1|965| +2450829|11131|1|1000| +2450829|11132|1|263| +2450829|11134|1|| +2450829|11137|1|202| +2450829|11138|1|378| +2450829|11140|1|604| +2450829|11143|1|523| +2450829|11144|1|195| +2450829|11146|1|808| +2450829|11149|1|530| +2450829|11150|1|852| +2450829|11152|1|128| +2450829|11155|1|611| +2450829|11156|1|80| +2450829|11158|1|123| +2450829|11161|1|884| +2450829|11162|1|706| +2450829|11164|1|807| +2450829|11167|1|634| +2450829|11168|1|68| +2450829|11170|1|358| +2450829|11173|1|429| +2450829|11174|1|625| +2450829|11176|1|555| +2450829|11179|1|723| +2450829|11180|1|715| +2450829|11182|1|989| +2450829|11185|1|25| +2450829|11186|1|500| +2450829|11188|1|698| +2450829|11191|1|548| +2450829|11192|1|279| +2450829|11194|1|387| +2450829|11197|1|697| +2450829|11198|1|587| +2450829|11200|1|829| +2450829|11203|1|402| +2450829|11204|1|306| +2450829|11206|1|923| +2450829|11209|1|576| +2450829|11210|1|407| +2450829|11212|1|873| +2450829|11215|1|993| +2450829|11216|1|| +2450829|11218|1|975| +2450829|11221|1|489| +2450829|11222|1|465| +2450829|11224|1|369| +2450829|11227|1|851| +2450829|11228|1|493| +2450829|11230|1|680| +2450829|11233|1|830| +2450829|11234|1|825| +2450829|11236|1|826| +2450829|11239|1|68| +2450829|11240|1|719| +2450829|11242|1|501| +2450829|11245|1|460| +2450829|11246|1|949| +2450829|11248|1|437| +2450829|11251|1|122| +2450829|11252|1|265| +2450829|11254|1|984| +2450829|11257|1|19| +2450829|11258|1|| +2450829|11260|1|592| +2450829|11263|1|| +2450829|11264|1|124| +2450829|11266|1|109| +2450829|11269|1|638| +2450829|11270|1|565| +2450829|11272|1|160| +2450829|11275|1|629| +2450829|11276|1|740| +2450829|11278|1|| +2450829|11281|1|15| +2450829|11282|1|883| +2450829|11284|1|579| +2450829|11287|1|322| +2450829|11288|1|756| +2450829|11290|1|810| +2450829|11293|1|588| +2450829|11294|1|| +2450829|11296|1|323| +2450829|11299|1|308| +2450829|11300|1|336| +2450829|11302|1|248| +2450829|11305|1|771| +2450829|11306|1|621| +2450829|11308|1|919| +2450829|11311|1|629| +2450829|11312|1|417| +2450829|11314|1|9| +2450829|11317|1|99| +2450829|11318|1|192| +2450829|11320|1|783| +2450829|11323|1|716| +2450829|11324|1|491| +2450829|11326|1|175| +2450829|11329|1|774| +2450829|11330|1|3| +2450829|11332|1|35| +2450829|11335|1|118| +2450829|11336|1|809| +2450829|11338|1|194| +2450829|11341|1|161| +2450829|11342|1|616| +2450829|11344|1|| +2450829|11347|1|195| +2450829|11348|1|786| +2450829|11350|1|81| +2450829|11353|1|143| +2450829|11354|1|926| +2450829|11356|1|777| +2450829|11359|1|806| +2450829|11360|1|185| +2450829|11362|1|638| +2450829|11365|1|977| +2450829|11366|1|565| +2450829|11368|1|73| +2450829|11371|1|475| +2450829|11372|1|368| +2450829|11374|1|297| +2450829|11377|1|439| +2450829|11378|1|912| +2450829|11380|1|227| +2450829|11383|1|339| +2450829|11384|1|273| +2450829|11386|1|115| +2450829|11389|1|| +2450829|11390|1|109| +2450829|11392|1|399| +2450829|11395|1|417| +2450829|11396|1|837| +2450829|11398|1|587| +2450829|11401|1|39| +2450829|11402|1|589| +2450829|11404|1|753| +2450829|11407|1|365| +2450829|11408|1|933| +2450829|11410|1|594| +2450829|11413|1|469| +2450829|11414|1|| +2450829|11416|1|688| +2450829|11419|1|39| +2450829|11420|1|311| +2450829|11422|1|62| +2450829|11425|1|854| +2450829|11426|1|710| +2450829|11428|1|| +2450829|11431|1|246| +2450829|11432|1|857| +2450829|11434|1|763| +2450829|11437|1|120| +2450829|11438|1|684| +2450829|11440|1|| +2450829|11443|1|591| +2450829|11444|1|17| +2450829|11446|1|418| +2450829|11449|1|711| +2450829|11450|1|630| +2450829|11452|1|346| +2450829|11455|1|696| +2450829|11456|1|80| +2450829|11458|1|609| +2450829|11461|1|923| +2450829|11462|1|376| +2450829|11464|1|736| +2450829|11467|1|788| +2450829|11468|1|269| +2450829|11470|1|856| +2450829|11473|1|335| +2450829|11474|1|987| +2450829|11476|1|385| +2450829|11479|1|430| +2450829|11480|1|| +2450829|11482|1|679| +2450829|11485|1|258| +2450829|11486|1|358| +2450829|11488|1|| +2450829|11491|1|802| +2450829|11492|1|233| +2450829|11494|1|187| +2450829|11497|1|479| +2450829|11498|1|113| +2450829|11500|1|96| +2450829|11503|1|157| +2450829|11504|1|740| +2450829|11506|1|656| +2450829|11509|1|547| +2450829|11510|1|455| +2450829|11512|1|651| +2450829|11515|1|112| +2450829|11516|1|293| +2450829|11518|1|541| +2450829|11521|1|449| +2450829|11522|1|180| +2450829|11524|1|522| +2450829|11527|1|800| +2450829|11528|1|678| +2450829|11530|1|877| +2450829|11533|1|391| +2450829|11534|1|17| +2450829|11536|1|680| +2450829|11539|1|140| +2450829|11540|1|531| +2450829|11542|1|740| +2450829|11545|1|646| +2450829|11546|1|418| +2450829|11548|1|678| +2450829|11551|1|775| +2450829|11552|1|373| +2450829|11554|1|322| +2450829|11557|1|735| +2450829|11558|1|32| +2450829|11560|1|558| +2450829|11563|1|373| +2450829|11564|1|309| +2450829|11566|1|183| +2450829|11569|1|738| +2450829|11570|1|513| +2450829|11572|1|838| +2450829|11575|1|256| +2450829|11576|1|494| +2450829|11578|1|597| +2450829|11581|1|27| +2450829|11582|1|680| +2450829|11584|1|59| +2450829|11587|1|739| +2450829|11588|1|943| +2450829|11590|1|712| +2450829|11593|1|999| +2450829|11594|1|88| +2450829|11596|1|167| +2450829|11599|1|325| +2450829|11600|1|893| +2450829|11602|1|794| +2450829|11605|1|724| +2450829|11606|1|824| +2450829|11608|1|929| +2450829|11611|1|77| +2450829|11612|1|464| +2450829|11614|1|972| +2450829|11617|1|264| +2450829|11618|1|844| +2450829|11620|1|454| +2450829|11623|1|461| +2450829|11624|1|748| +2450829|11626|1|296| +2450829|11629|1|484| +2450829|11630|1|903| +2450829|11632|1|449| +2450829|11635|1|976| +2450829|11636|1|616| +2450829|11638|1|60| +2450829|11641|1|502| +2450829|11642|1|854| +2450829|11644|1|111| +2450829|11647|1|627| +2450829|11648|1|872| +2450829|11650|1|160| +2450829|11653|1|747| +2450829|11654|1|345| +2450829|11656|1|796| +2450829|11659|1|216| +2450829|11660|1|476| +2450829|11662|1|186| +2450829|11665|1|735| +2450829|11666|1|60| +2450829|11668|1|384| +2450829|11671|1|312| +2450829|11672|1|542| +2450829|11674|1|427| +2450829|11677|1|24| +2450829|11678|1|859| +2450829|11680|1|737| +2450829|11683|1|787| +2450829|11684|1|| +2450829|11686|1|116| +2450829|11689|1|611| +2450829|11690|1|275| +2450829|11692|1|271| +2450829|11695|1|141| +2450829|11696|1|214| +2450829|11698|1|55| +2450829|11701|1|39| +2450829|11702|1|478| +2450829|11704|1|601| +2450829|11707|1|13| +2450829|11708|1|525| +2450829|11710|1|833| +2450829|11713|1|812| +2450829|11714|1|999| +2450829|11716|1|467| +2450829|11719|1|780| +2450829|11720|1|712| +2450829|11722|1|| +2450829|11725|1|199| +2450829|11726|1|109| +2450829|11728|1|518| +2450829|11731|1|141| +2450829|11732|1|313| +2450829|11734|1|901| +2450829|11737|1|11| +2450829|11738|1|297| +2450829|11740|1|211| +2450829|11743|1|669| +2450829|11744|1|265| +2450829|11746|1|326| +2450829|11749|1|971| +2450829|11750|1|215| +2450829|11752|1|971| +2450829|11755|1|865| +2450829|11756|1|783| +2450829|11758|1|659| +2450829|11761|1|184| +2450829|11762|1|138| +2450829|11764|1|358| +2450829|11767|1|471| +2450829|11768|1|| +2450829|11770|1|681| +2450829|11773|1|428| +2450829|11774|1|987| +2450829|11776|1|977| +2450829|11779|1|576| +2450829|11780|1|919| +2450829|11782|1|| +2450829|11785|1|228| +2450829|11786|1|341| +2450829|11788|1|653| +2450829|11791|1|155| +2450829|11792|1|586| +2450829|11794|1|947| +2450829|11797|1|571| +2450829|11798|1|315| +2450829|11800|1|395| +2450829|11803|1|448| +2450829|11804|1|110| +2450829|11806|1|483| +2450829|11809|1|| +2450829|11810|1|346| +2450829|11812|1|824| +2450829|11815|1|610| +2450829|11816|1|144| +2450829|11818|1|116| +2450829|11821|1|505| +2450829|11822|1|499| +2450829|11824|1|97| +2450829|11827|1|894| +2450829|11828|1|46| +2450829|11830|1|95| +2450829|11833|1|628| +2450829|11834|1|845| +2450829|11836|1|755| +2450829|11839|1|556| +2450829|11840|1|799| +2450829|11842|1|681| +2450829|11845|1|613| +2450829|11846|1|993| +2450829|11848|1|379| +2450829|11851|1|417| +2450829|11852|1|802| +2450829|11854|1|976| +2450829|11857|1|843| +2450829|11858|1|188| +2450829|11860|1|499| +2450829|11863|1|624| +2450829|11864|1|766| +2450829|11866|1|322| +2450829|11869|1|553| +2450829|11870|1|801| +2450829|11872|1|| +2450829|11875|1|700| +2450829|11876|1|383| +2450829|11878|1|431| +2450829|11881|1|631| +2450829|11882|1|174| +2450829|11884|1|942| +2450829|11887|1|151| +2450829|11888|1|33| +2450829|11890|1|110| +2450829|11893|1|326| +2450829|11894|1|476| +2450829|11896|1|566| +2450829|11899|1|237| +2450829|11900|1|651| +2450829|11902|1|461| +2450829|11905|1|52| +2450829|11906|1|783| +2450829|11908|1|710| +2450829|11911|1|159| +2450829|11912|1|126| +2450829|11914|1|457| +2450829|11917|1|985| +2450829|11918|1|972| +2450829|11920|1|567| +2450829|11923|1|805| +2450829|11924|1|341| +2450829|11926|1|49| +2450829|11929|1|895| +2450829|11930|1|970| +2450829|11932|1|329| +2450829|11935|1|17| +2450829|11936|1|218| +2450829|11938|1|726| +2450829|11941|1|45| +2450829|11942|1|490| +2450829|11944|1|748| +2450829|11947|1|351| +2450829|11948|1|661| +2450829|11950|1|954| +2450829|11953|1|101| +2450829|11954|1|477| +2450829|11956|1|568| +2450829|11959|1|812| +2450829|11960|1|1| +2450829|11962|1|649| +2450829|11965|1|789| +2450829|11966|1|233| +2450829|11968|1|600| +2450829|11971|1|841| +2450829|11972|1|239| +2450829|11974|1|287| +2450829|11977|1|136| +2450829|11978|1|643| +2450829|11980|1|757| +2450829|11983|1|484| +2450829|11984|1|| +2450829|11986|1|| +2450829|11989|1|879| +2450829|11990|1|520| +2450829|11992|1|759| +2450829|11995|1|878| +2450829|11996|1|893| +2450829|11998|1|232| +2450829|12001|1|223| +2450829|12002|1|512| +2450829|12004|1|371| +2450829|12007|1|574| +2450829|12008|1|835| +2450829|12010|1|650| +2450829|12013|1|464| +2450829|12014|1|392| +2450829|12016|1|94| +2450829|12019|1|700| +2450829|12020|1|833| +2450829|12022|1|269| +2450829|12025|1|455| +2450829|12026|1|391| +2450829|12028|1|355| +2450829|12031|1|655| +2450829|12032|1|633| +2450829|12034|1|268| +2450829|12037|1|204| +2450829|12038|1|252| +2450829|12040|1|691| +2450829|12043|1|373| +2450829|12044|1|176| +2450829|12046|1|46| +2450829|12049|1|| +2450829|12050|1|336| +2450829|12052|1|929| +2450829|12055|1|151| +2450829|12056|1|923| +2450829|12058|1|692| +2450829|12061|1|749| +2450829|12062|1|990| +2450829|12064|1|608| +2450829|12067|1|207| +2450829|12068|1|423| +2450829|12070|1|973| +2450829|12073|1|899| +2450829|12074|1|458| +2450829|12076|1|628| +2450829|12079|1|647| +2450829|12080|1|593| +2450829|12082|1|802| +2450829|12085|1|860| +2450829|12086|1|248| +2450829|12088|1|415| +2450829|12091|1|615| +2450829|12092|1|782| +2450829|12094|1|744| +2450829|12097|1|436| +2450829|12098|1|| +2450829|12100|1|780| +2450829|12103|1|598| +2450829|12104|1|623| +2450829|12106|1|725| +2450829|12109|1|272| +2450829|12110|1|461| +2450829|12112|1|755| +2450829|12115|1|426| +2450829|12116|1|830| +2450829|12118|1|15| +2450829|12121|1|311| +2450829|12122|1|270| +2450829|12124|1|816| +2450829|12127|1|948| +2450829|12128|1|102| +2450829|12130|1|549| +2450829|12133|1|371| +2450829|12134|1|914| +2450829|12136|1|937| +2450829|12139|1|561| +2450829|12140|1|828| +2450829|12142|1|235| +2450829|12145|1|514| +2450829|12146|1|682| +2450829|12148|1|320| +2450829|12151|1|7| +2450829|12152|1|455| +2450829|12154|1|926| +2450829|12157|1|97| +2450829|12158|1|590| +2450829|12160|1|375| +2450829|12163|1|474| +2450829|12164|1|125| +2450829|12166|1|766| +2450829|12169|1|| +2450829|12170|1|666| +2450829|12172|1|915| +2450829|12175|1|353| +2450829|12176|1|350| +2450829|12178|1|430| +2450829|12181|1|183| +2450829|12182|1|564| +2450829|12184|1|910| +2450829|12187|1|35| +2450829|12188|1|798| +2450829|12190|1|301| +2450829|12193|1|141| +2450829|12194|1|176| +2450829|12196|1|586| +2450829|12199|1|707| +2450829|12200|1|587| +2450829|12202|1|528| +2450829|12205|1|801| +2450829|12206|1|475| +2450829|12208|1|464| +2450829|12211|1|81| +2450829|12212|1|722| +2450829|12214|1|852| +2450829|12217|1|483| +2450829|12218|1|670| +2450829|12220|1|808| +2450829|12223|1|712| +2450829|12224|1|744| +2450829|12226|1|863| +2450829|12229|1|247| +2450829|12230|1|820| +2450829|12232|1|154| +2450829|12235|1|235| +2450829|12236|1|753| +2450829|12238|1|831| +2450829|12241|1|17| +2450829|12242|1|741| +2450829|12244|1|438| +2450829|12247|1|819| +2450829|12248|1|792| +2450829|12250|1|629| +2450829|12253|1|364| +2450829|12254|1|58| +2450829|12256|1|702| +2450829|12259|1|857| +2450829|12260|1|129| +2450829|12262|1|306| +2450829|12265|1|369| +2450829|12266|1|441| +2450829|12268|1|699| +2450829|12271|1|281| +2450829|12272|1|333| +2450829|12274|1|430| +2450829|12277|1|414| +2450829|12278|1|145| +2450829|12280|1|24| +2450829|12283|1|542| +2450829|12284|1|| +2450829|12286|1|624| +2450829|12289|1|620| +2450829|12290|1|803| +2450829|12292|1|348| +2450829|12295|1|525| +2450829|12296|1|930| +2450829|12298|1|374| +2450829|12301|1|532| +2450829|12302|1|720| +2450829|12304|1|626| +2450829|12307|1|781| +2450829|12308|1|444| +2450829|12310|1|| +2450829|12313|1|831| +2450829|12314|1|| +2450829|12316|1|570| +2450829|12319|1|346| +2450829|12320|1|754| +2450829|12322|1|577| +2450829|12325|1|812| +2450829|12326|1|953| +2450829|12328|1|20| +2450829|12331|1|172| +2450829|12332|1|917| +2450829|12334|1|704| +2450829|12337|1|114| +2450829|12338|1|194| +2450829|12340|1|113| +2450829|12343|1|241| +2450829|12344|1|384| +2450829|12346|1|589| +2450829|12349|1|225| +2450829|12350|1|325| +2450829|12352|1|305| +2450829|12355|1|905| +2450829|12356|1|680| +2450829|12358|1|294| +2450829|12361|1|| +2450829|12362|1|390| +2450829|12364|1|494| +2450829|12367|1|988| +2450829|12368|1|769| +2450829|12370|1|335| +2450829|12373|1|746| +2450829|12374|1|548| +2450829|12376|1|340| +2450829|12379|1|613| +2450829|12380|1|47| +2450829|12382|1|23| +2450829|12385|1|991| +2450829|12386|1|411| +2450829|12388|1|173| +2450829|12391|1|432| +2450829|12392|1|209| +2450829|12394|1|9| +2450829|12397|1|878| +2450829|12398|1|855| +2450829|12400|1|534| +2450829|12403|1|77| +2450829|12404|1|233| +2450829|12406|1|482| +2450829|12409|1|786| +2450829|12410|1|966| +2450829|12412|1|726| +2450829|12415|1|511| +2450829|12416|1|756| +2450829|12418|1|312| +2450829|12421|1|957| +2450829|12422|1|16| +2450829|12424|1|| +2450829|12427|1|170| +2450829|12428|1|852| +2450829|12430|1|804| +2450829|12433|1|530| +2450829|12434|1|225| +2450829|12436|1|523| +2450829|12439|1|193| +2450829|12440|1|968| +2450829|12442|1|| +2450829|12445|1|69| +2450829|12446|1|36| +2450829|12448|1|878| +2450829|12451|1|906| +2450829|12452|1|400| +2450829|12454|1|308| +2450829|12457|1|963| +2450829|12458|1|495| +2450829|12460|1|220| +2450829|12463|1|758| +2450829|12464|1|283| +2450829|12466|1|261| +2450829|12469|1|172| +2450829|12470|1|635| +2450829|12472|1|| +2450829|12475|1|268| +2450829|12476|1|211| +2450829|12478|1|544| +2450829|12481|1|112| +2450829|12482|1|978| +2450829|12484|1|800| +2450829|12487|1|788| +2450829|12488|1|823| +2450829|12490|1|276| +2450829|12493|1|750| +2450829|12494|1|161| +2450829|12496|1|71| +2450829|12499|1|710| +2450829|12500|1|850| +2450829|12502|1|876| +2450829|12505|1|779| +2450829|12506|1|462| +2450829|12508|1|981| +2450829|12511|1|629| +2450829|12512|1|712| +2450829|12514|1|| +2450829|12517|1|185| +2450829|12518|1|707| +2450829|12520|1|807| +2450829|12523|1|744| +2450829|12524|1|924| +2450829|12526|1|742| +2450829|12529|1|37| +2450829|12530|1|475| +2450829|12532|1|965| +2450829|12535|1|883| +2450829|12536|1|318| +2450829|12538|1|679| +2450829|12541|1|860| +2450829|12542|1|469| +2450829|12544|1|689| +2450829|12547|1|294| +2450829|12548|1|526| +2450829|12550|1|757| +2450829|12553|1|361| +2450829|12554|1|186| +2450829|12556|1|396| +2450829|12559|1|| +2450829|12560|1|214| +2450829|12562|1|236| +2450829|12565|1|732| +2450829|12566|1|67| +2450829|12568|1|287| +2450829|12571|1|| +2450829|12572|1|993| +2450829|12574|1|942| +2450829|12577|1|275| +2450829|12578|1|| +2450829|12580|1|663| +2450829|12583|1|963| +2450829|12584|1|63| +2450829|12586|1|380| +2450829|12589|1|368| +2450829|12590|1|332| +2450829|12592|1|405| +2450829|12595|1|678| +2450829|12596|1|343| +2450829|12598|1|665| +2450829|12601|1|605| +2450829|12602|1|99| +2450829|12604|1|299| +2450829|12607|1|534| +2450829|12608|1|390| +2450829|12610|1|| +2450829|12613|1|303| +2450829|12614|1|919| +2450829|12616|1|595| +2450829|12619|1|| +2450829|12620|1|120| +2450829|12622|1|914| +2450829|12625|1|174| +2450829|12626|1|597| +2450829|12628|1|120| +2450829|12631|1|308| +2450829|12632|1|| +2450829|12634|1|551| +2450829|12637|1|570| +2450829|12638|1|556| +2450829|12640|1|247| +2450829|12643|1|592| +2450829|12644|1|605| +2450829|12646|1|451| +2450829|12649|1|980| +2450829|12650|1|284| +2450829|12652|1|718| +2450829|12655|1|225| +2450829|12656|1|887| +2450829|12658|1|267| +2450829|12661|1|56| +2450829|12662|1|425| +2450829|12664|1|904| +2450829|12667|1|247| +2450829|12668|1|298| +2450829|12670|1|433| +2450829|12673|1|24| +2450829|12674|1|36| +2450829|12676|1|100| +2450829|12679|1|527| +2450829|12680|1|744| +2450829|12682|1|921| +2450829|12685|1|551| +2450829|12686|1|| +2450829|12688|1|831| +2450829|12691|1|328| +2450829|12692|1|474| +2450829|12694|1|614| +2450829|12697|1|599| +2450829|12698|1|987| +2450829|12700|1|383| +2450829|12703|1|242| +2450829|12704|1|73| +2450829|12706|1|459| +2450829|12709|1|44| +2450829|12710|1|490| +2450829|12712|1|424| +2450829|12715|1|380| +2450829|12716|1|839| +2450829|12718|1|694| +2450829|12721|1|150| +2450829|12722|1|278| +2450829|12724|1|443| +2450829|12727|1|102| +2450829|12728|1|7| +2450829|12730|1|988| +2450829|12733|1|584| +2450829|12734|1|436| +2450829|12736|1|364| +2450829|12739|1|446| +2450829|12740|1|984| +2450829|12742|1|817| +2450829|12745|1|287| +2450829|12746|1|912| +2450829|12748|1|929| +2450829|12751|1|| +2450829|12752|1|825| +2450829|12754|1|166| +2450829|12757|1|642| +2450829|12758|1|822| +2450829|12760|1|142| +2450829|12763|1|7| +2450829|12764|1|817| +2450829|12766|1|298| +2450829|12769|1|565| +2450829|12770|1|262| +2450829|12772|1|204| +2450829|12775|1|713| +2450829|12776|1|887| +2450829|12778|1|345| +2450829|12781|1|447| +2450829|12782|1|127| +2450829|12784|1|551| +2450829|12787|1|| +2450829|12788|1|991| +2450829|12790|1|996| +2450829|12793|1|283| +2450829|12794|1|721| +2450829|12796|1|709| +2450829|12799|1|886| +2450829|12800|1|968| +2450829|12802|1|531| +2450829|12805|1|175| +2450829|12806|1|76| +2450829|12808|1|785| +2450829|12811|1|593| +2450829|12812|1|573| +2450829|12814|1|969| +2450829|12817|1|812| +2450829|12818|1|207| +2450829|12820|1|884| +2450829|12823|1|837| +2450829|12824|1|142| +2450829|12826|1|1000| +2450829|12829|1|278| +2450829|12830|1|| +2450829|12832|1|91| +2450829|12835|1|| +2450829|12836|1|454| +2450829|12838|1|662| +2450829|12841|1|318| +2450829|12842|1|999| +2450829|12844|1|164| +2450829|12847|1|607| +2450829|12848|1|930| +2450829|12850|1|83| +2450829|12853|1|284| +2450829|12854|1|635| +2450829|12856|1|31| +2450829|12859|1|536| +2450829|12860|1|13| +2450829|12862|1|987| +2450829|12865|1|710| +2450829|12866|1|538| +2450829|12868|1|777| +2450829|12871|1|959| +2450829|12872|1|934| +2450829|12874|1|349| +2450829|12877|1|21| +2450829|12878|1|550| +2450829|12880|1|151| +2450829|12883|1|| +2450829|12884|1|445| +2450829|12886|1|57| +2450829|12889|1|76| +2450829|12890|1|| +2450829|12892|1|315| +2450829|12895|1|877| +2450829|12896|1|215| +2450829|12898|1|438| +2450829|12901|1|872| +2450829|12902|1|270| +2450829|12904|1|802| +2450829|12907|1|| +2450829|12908|1|116| +2450829|12910|1|784| +2450829|12913|1|525| +2450829|12914|1|160| +2450829|12916|1|910| +2450829|12919|1|210| +2450829|12920|1|429| +2450829|12922|1|1| +2450829|12925|1|126| +2450829|12926|1|889| +2450829|12928|1|951| +2450829|12931|1|329| +2450829|12932|1|34| +2450829|12934|1|596| +2450829|12937|1|452| +2450829|12938|1|708| +2450829|12940|1|153| +2450829|12943|1|644| +2450829|12944|1|683| +2450829|12946|1|507| +2450829|12949|1|972| +2450829|12950|1|136| +2450829|12952|1|507| +2450829|12955|1|333| +2450829|12956|1|489| +2450829|12958|1|248| +2450829|12961|1|34| +2450829|12962|1|615| +2450829|12964|1|875| +2450829|12967|1|603| +2450829|12968|1|828| +2450829|12970|1|323| +2450829|12973|1|213| +2450829|12974|1|881| +2450829|12976|1|274| +2450829|12979|1|255| +2450829|12980|1|742| +2450829|12982|1|746| +2450829|12985|1|7| +2450829|12986|1|817| +2450829|12988|1|536| +2450829|12991|1|373| +2450829|12992|1|546| +2450829|12994|1|364| +2450829|12997|1|368| +2450829|12998|1|| +2450829|13000|1|602| +2450829|13003|1|303| +2450829|13004|1|802| +2450829|13006|1|716| +2450829|13009|1|635| +2450829|13010|1|58| +2450829|13012|1|238| +2450829|13015|1|37| +2450829|13016|1|539| +2450829|13018|1|423| +2450829|13021|1|782| +2450829|13022|1|673| +2450829|13024|1|160| +2450829|13027|1|205| +2450829|13028|1|361| +2450829|13030|1|456| +2450829|13033|1|451| +2450829|13034|1|86| +2450829|13036|1|417| +2450829|13039|1|381| +2450829|13040|1|449| +2450829|13042|1|| +2450829|13045|1|519| +2450829|13046|1|958| +2450829|13048|1|651| +2450829|13051|1|327| +2450829|13052|1|590| +2450829|13054|1|362| +2450829|13057|1|813| +2450829|13058|1|513| +2450829|13060|1|752| +2450829|13063|1|335| +2450829|13064|1|306| +2450829|13066|1|94| +2450829|13069|1|596| +2450829|13070|1|989| +2450829|13072|1|379| +2450829|13075|1|496| +2450829|13076|1|229| +2450829|13078|1|934| +2450829|13081|1|869| +2450829|13082|1|751| +2450829|13084|1|262| +2450829|13087|1|181| +2450829|13088|1|634| +2450829|13090|1|119| +2450829|13093|1|69| +2450829|13094|1|658| +2450829|13096|1|| +2450829|13099|1|319| +2450829|13100|1|| +2450829|13102|1|803| +2450829|13105|1|| +2450829|13106|1|411| +2450829|13108|1|979| +2450829|13111|1|923| +2450829|13112|1|378| +2450829|13114|1|604| +2450829|13117|1|778| +2450829|13118|1|29| +2450829|13120|1|316| +2450829|13123|1|959| +2450829|13124|1|25| +2450829|13126|1|863| +2450829|13129|1|227| +2450829|13130|1|769| +2450829|13132|1|335| +2450829|13135|1|255| +2450829|13136|1|439| +2450829|13138|1|481| +2450829|13141|1|174| +2450829|13142|1|914| +2450829|13144|1|798| +2450829|13147|1|| +2450829|13148|1|701| +2450829|13150|1|31| +2450829|13153|1|122| +2450829|13154|1|333| +2450829|13156|1|96| +2450829|13159|1|72| +2450829|13160|1|100| +2450829|13162|1|466| +2450829|13165|1|75| +2450829|13166|1|923| +2450829|13168|1|639| +2450829|13171|1|66| +2450829|13172|1|660| +2450829|13174|1|250| +2450829|13177|1|611| +2450829|13178|1|810| +2450829|13180|1|209| +2450829|13183|1|910| +2450829|13184|1|883| +2450829|13186|1|839| +2450829|13189|1|350| +2450829|13190|1|103| +2450829|13192|1|| +2450829|13195|1|196| +2450829|13196|1|| +2450829|13198|1|325| +2450829|13201|1|738| +2450829|13202|1|732| +2450829|13204|1|| +2450829|13207|1|351| +2450829|13208|1|103| +2450829|13210|1|515| +2450829|13213|1|523| +2450829|13214|1|846| +2450829|13216|1|98| +2450829|13219|1|162| +2450829|13220|1|| +2450829|13222|1|363| +2450829|13225|1|734| +2450829|13226|1|496| +2450829|13228|1|762| +2450829|13231|1|34| +2450829|13232|1|801| +2450829|13234|1|84| +2450829|13237|1|596| +2450829|13238|1|190| +2450829|13240|1|405| +2450829|13243|1|328| +2450829|13244|1|313| +2450829|13246|1|460| +2450829|13249|1|769| +2450829|13250|1|208| +2450829|13252|1|523| +2450829|13255|1|683| +2450829|13256|1|579| +2450829|13258|1|583| +2450829|13261|1|| +2450829|13262|1|286| +2450829|13264|1|212| +2450829|13267|1|562| +2450829|13268|1|449| +2450829|13270|1|71| +2450829|13273|1|688| +2450829|13274|1|822| +2450829|13276|1|837| +2450829|13279|1|572| +2450829|13280|1|934| +2450829|13282|1|74| +2450829|13285|1|988| +2450829|13286|1|648| +2450829|13288|1|898| +2450829|13291|1|394| +2450829|13292|1|330| +2450829|13294|1|708| +2450829|13297|1|598| +2450829|13298|1|717| +2450829|13300|1|719| +2450829|13303|1|168| +2450829|13304|1|894| +2450829|13306|1|348| +2450829|13309|1|176| +2450829|13310|1|617| +2450829|13312|1|550| +2450829|13315|1|186| +2450829|13316|1|120| +2450829|13318|1|895| +2450829|13321|1|487| +2450829|13322|1|271| +2450829|13324|1|607| +2450829|13327|1|188| +2450829|13328|1|592| +2450829|13330|1|262| +2450829|13333|1|693| +2450829|13334|1|299| +2450829|13336|1|693| +2450829|13339|1|210| +2450829|13340|1|216| +2450829|13342|1|820| +2450829|13345|1|199| +2450829|13346|1|354| +2450829|13348|1|829| +2450829|13351|1|100| +2450829|13352|1|913| +2450829|13354|1|808| +2450829|13357|1|738| +2450829|13358|1|875| +2450829|13360|1|502| +2450829|13363|1|574| +2450829|13364|1|172| +2450829|13366|1|632| +2450829|13369|1|172| +2450829|13370|1|| +2450829|13372|1|110| +2450829|13375|1|82| +2450829|13376|1|532| +2450829|13378|1|492| +2450829|13381|1|159| +2450829|13382|1|681| +2450829|13384|1|567| +2450829|13387|1|790| +2450829|13388|1|832| +2450829|13390|1|892| +2450829|13393|1|49| +2450829|13394|1|907| +2450829|13396|1|| +2450829|13399|1|262| +2450829|13400|1|257| +2450829|13402|1|562| +2450829|13405|1|734| +2450829|13406|1|882| +2450829|13408|1|203| +2450829|13411|1|219| +2450829|13412|1|691| +2450829|13414|1|829| +2450829|13417|1|253| +2450829|13418|1|188| +2450829|13420|1|373| +2450829|13423|1|615| +2450829|13424|1|563| +2450829|13426|1|895| +2450829|13429|1|45| +2450829|13430|1|489| +2450829|13432|1|864| +2450829|13435|1|122| +2450829|13436|1|35| +2450829|13438|1|287| +2450829|13441|1|| +2450829|13442|1|284| +2450829|13444|1|570| +2450829|13447|1|985| +2450829|13448|1|228| +2450829|13450|1|429| +2450829|13453|1|210| +2450829|13454|1|696| +2450829|13456|1|637| +2450829|13459|1|136| +2450829|13460|1|332| +2450829|13462|1|604| +2450829|13465|1|310| +2450829|13466|1|916| +2450829|13468|1|808| +2450829|13471|1|76| +2450829|13472|1|462| +2450829|13474|1|| +2450829|13477|1|256| +2450829|13478|1|980| +2450829|13480|1|563| +2450829|13483|1|569| +2450829|13484|1|826| +2450829|13486|1|929| +2450829|13489|1|43| +2450829|13490|1|757| +2450829|13492|1|85| +2450829|13495|1|153| +2450829|13496|1|353| +2450829|13498|1|787| +2450829|13501|1|158| +2450829|13502|1|119| +2450829|13504|1|438| +2450829|13507|1|268| +2450829|13508|1|978| +2450829|13510|1|579| +2450829|13513|1|142| +2450829|13514|1|607| +2450829|13516|1|859| +2450829|13519|1|889| +2450829|13520|1|160| +2450829|13522|1|917| +2450829|13525|1|21| +2450829|13526|1|530| +2450829|13528|1|605| +2450829|13531|1|196| +2450829|13532|1|704| +2450829|13534|1|| +2450829|13537|1|545| +2450829|13538|1|776| +2450829|13540|1|146| +2450829|13543|1|382| +2450829|13544|1|760| +2450829|13546|1|291| +2450829|13549|1|592| +2450829|13550|1|600| +2450829|13552|1|507| +2450829|13555|1|761| +2450829|13556|1|| +2450829|13558|1|7| +2450829|13561|1|50| +2450829|13562|1|| +2450829|13564|1|918| +2450829|13567|1|951| +2450829|13568|1|829| +2450829|13570|1|142| +2450829|13573|1|322| +2450829|13574|1|419| +2450829|13576|1|791| +2450829|13579|1|137| +2450829|13580|1|527| +2450829|13582|1|512| +2450829|13585|1|121| +2450829|13586|1|993| +2450829|13588|1|108| +2450829|13591|1|621| +2450829|13592|1|900| +2450829|13594|1|170| +2450829|13597|1|1| +2450829|13598|1|3| +2450829|13600|1|425| +2450829|13603|1|246| +2450829|13604|1|253| +2450829|13606|1|730| +2450829|13609|1|732| +2450829|13610|1|191| +2450829|13612|1|56| +2450829|13615|1|876| +2450829|13616|1|293| +2450829|13618|1|768| +2450829|13621|1|554| +2450829|13622|1|300| +2450829|13624|1|130| +2450829|13627|1|319| +2450829|13628|1|716| +2450829|13630|1|569| +2450829|13633|1|488| +2450829|13634|1|672| +2450829|13636|1|653| +2450829|13639|1|655| +2450829|13640|1|129| +2450829|13642|1|334| +2450829|13645|1|332| +2450829|13646|1|239| +2450829|13648|1|851| +2450829|13651|1|456| +2450829|13652|1|93| +2450829|13654|1|176| +2450829|13657|1|477| +2450829|13658|1|227| +2450829|13660|1|243| +2450829|13663|1|522| +2450829|13664|1|218| +2450829|13666|1|150| +2450829|13669|1|722| +2450829|13670|1|653| +2450829|13672|1|147| +2450829|13675|1|3| +2450829|13676|1|| +2450829|13678|1|189| +2450829|13681|1|911| +2450829|13682|1|828| +2450829|13684|1|606| +2450829|13687|1|159| +2450829|13688|1|780| +2450829|13690|1|126| +2450829|13693|1|418| +2450829|13694|1|23| +2450829|13696|1|978| +2450829|13699|1|114| +2450829|13700|1|941| +2450829|13702|1|639| +2450829|13705|1|488| +2450829|13706|1|983| +2450829|13708|1|189| +2450829|13711|1|757| +2450829|13712|1|902| +2450829|13714|1|760| +2450829|13717|1|26| +2450829|13718|1|| +2450829|13720|1|517| +2450829|13723|1|917| +2450829|13724|1|508| +2450829|13726|1|963| +2450829|13729|1|555| +2450829|13730|1|150| +2450829|13732|1|381| +2450829|13735|1|2| +2450829|13736|1|12| +2450829|13738|1|221| +2450829|13741|1|20| +2450829|13742|1|409| +2450829|13744|1|739| +2450829|13747|1|463| +2450829|13748|1|551| +2450829|13750|1|877| +2450829|13753|1|68| +2450829|13754|1|192| +2450829|13756|1|| +2450829|13759|1|940| +2450829|13760|1|840| +2450829|13762|1|763| +2450829|13765|1|88| +2450829|13766|1|416| +2450829|13768|1|479| +2450829|13771|1|16| +2450829|13772|1|750| +2450829|13774|1|332| +2450829|13777|1|434| +2450829|13778|1|795| +2450829|13780|1|999| +2450829|13783|1|711| +2450829|13784|1|978| +2450829|13786|1|656| +2450829|13789|1|251| +2450829|13790|1|829| +2450829|13792|1|379| +2450829|13795|1|612| +2450829|13796|1|712| +2450829|13798|1|319| +2450829|13801|1|799| +2450829|13802|1|82| +2450829|13804|1|739| +2450829|13807|1|860| +2450829|13808|1|976| +2450829|13810|1|957| +2450829|13813|1|461| +2450829|13814|1|782| +2450829|13816|1|31| +2450829|13819|1|| +2450829|13820|1|376| +2450829|13822|1|702| +2450829|13825|1|628| +2450829|13826|1|953| +2450829|13828|1|| +2450829|13831|1|969| +2450829|13832|1|283| +2450829|13834|1|777| +2450829|13837|1|140| +2450829|13838|1|434| +2450829|13840|1|187| +2450829|13843|1|459| +2450829|13844|1|414| +2450829|13846|1|626| +2450829|13849|1|918| +2450829|13850|1|600| +2450829|13852|1|856| +2450829|13855|1|708| +2450829|13856|1|587| +2450829|13858|1|319| +2450829|13861|1|864| +2450829|13862|1|471| +2450829|13864|1|921| +2450829|13867|1|328| +2450829|13868|1|413| +2450829|13870|1|646| +2450829|13873|1|580| +2450829|13874|1|765| +2450829|13876|1|279| +2450829|13879|1|629| +2450829|13880|1|573| +2450829|13882|1|40| +2450829|13885|1|205| +2450829|13886|1|731| +2450829|13888|1|299| +2450829|13891|1|279| +2450829|13892|1|308| +2450829|13894|1|472| +2450829|13897|1|518| +2450829|13898|1|881| +2450829|13900|1|982| +2450829|13903|1|610| +2450829|13904|1|901| +2450829|13906|1|229| +2450829|13909|1|333| +2450829|13910|1|766| +2450829|13912|1|904| +2450829|13915|1|443| +2450829|13916|1|35| +2450829|13918|1|500| +2450829|13921|1|795| +2450829|13922|1|759| +2450829|13924|1|944| +2450829|13927|1|288| +2450829|13928|1|214| +2450829|13930|1|159| +2450829|13933|1|200| +2450829|13934|1|307| +2450829|13936|1|222| +2450829|13939|1|493| +2450829|13940|1|472| +2450829|13942|1|911| +2450829|13945|1|314| +2450829|13946|1|114| +2450829|13948|1|55| +2450829|13951|1|667| +2450829|13952|1|309| +2450829|13954|1|105| +2450829|13957|1|504| +2450829|13958|1|6| +2450829|13960|1|475| +2450829|13963|1|295| +2450829|13964|1|320| +2450829|13966|1|976| +2450829|13969|1|912| +2450829|13970|1|93| +2450829|13972|1|714| +2450829|13975|1|281| +2450829|13976|1|178| +2450829|13978|1|628| +2450829|13981|1|| +2450829|13982|1|653| +2450829|13984|1|514| +2450829|13987|1|634| +2450829|13988|1|134| +2450829|13990|1|558| +2450829|13993|1|551| +2450829|13994|1|677| +2450829|13996|1|770| +2450829|13999|1|92| +2450829|14000|1|340| +2450829|14002|1|303| +2450829|14005|1|807| +2450829|14006|1|155| +2450829|14008|1|974| +2450829|14011|1|829| +2450829|14012|1|978| +2450829|14014|1|345| +2450829|14017|1|| +2450829|14018|1|366| +2450829|14020|1|302| +2450829|14023|1|| +2450829|14024|1|649| +2450829|14026|1|287| +2450829|14029|1|642| +2450829|14030|1|589| +2450829|14032|1|565| +2450829|14035|1|890| +2450829|14036|1|403| +2450829|14038|1|189| +2450829|14041|1|707| +2450829|14042|1|138| +2450829|14044|1|772| +2450829|14047|1|631| +2450829|14048|1|9| +2450829|14050|1|937| +2450829|14053|1|828| +2450829|14054|1|940| +2450829|14056|1|907| +2450829|14059|1|481| +2450829|14060|1|57| +2450829|14062|1|669| +2450829|14065|1|356| +2450829|14066|1|256| +2450829|14068|1|60| +2450829|14071|1|459| +2450829|14072|1|245| +2450829|14074|1|835| +2450829|14077|1|349| +2450829|14078|1|303| +2450829|14080|1|675| +2450829|14083|1|870| +2450829|14084|1|419| +2450829|14086|1|614| +2450829|14089|1|958| +2450829|14090|1|18| +2450829|14092|1|360| +2450829|14095|1|614| +2450829|14096|1|105| +2450829|14098|1|193| +2450829|14101|1|840| +2450829|14102|1|| +2450829|14104|1|36| +2450829|14107|1|502| +2450829|14108|1|975| +2450829|14110|1|130| +2450829|14113|1|| +2450829|14114|1|499| +2450829|14116|1|743| +2450829|14119|1|26| +2450829|14120|1|797| +2450829|14122|1|954| +2450829|14125|1|524| +2450829|14126|1|568| +2450829|14128|1|454| +2450829|14131|1|494| +2450829|14132|1|708| +2450829|14134|1|425| +2450829|14137|1|578| +2450829|14138|1|977| +2450829|14140|1|890| +2450829|14143|1|216| +2450829|14144|1|541| +2450829|14146|1|391| +2450829|14149|1|71| +2450829|14150|1|348| +2450829|14152|1|409| +2450829|14155|1|572| +2450829|14156|1|887| +2450829|14158|1|425| +2450829|14161|1|359| +2450829|14162|1|457| +2450829|14164|1|76| +2450829|14167|1|648| +2450829|14168|1|504| +2450829|14170|1|940| +2450829|14173|1|364| +2450829|14174|1|110| +2450829|14176|1|825| +2450829|14179|1|752| +2450829|14180|1|531| +2450829|14182|1|911| +2450829|14185|1|110| +2450829|14186|1|666| +2450829|14188|1|315| +2450829|14191|1|826| +2450829|14192|1|556| +2450829|14194|1|846| +2450829|14197|1|795| +2450829|14198|1|793| +2450829|14200|1|870| +2450829|14203|1|641| +2450829|14204|1|847| +2450829|14206|1|363| +2450829|14209|1|849| +2450829|14210|1|848| +2450829|14212|1|839| +2450829|14215|1|880| +2450829|14216|1|498| +2450829|14218|1|813| +2450829|14221|1|107| +2450829|14222|1|398| +2450829|14224|1|571| +2450829|14227|1|944| +2450829|14228|1|455| +2450829|14230|1|844| +2450829|14233|1|699| +2450829|14234|1|| +2450829|14236|1|91| +2450829|14239|1|276| +2450829|14240|1|185| +2450829|14242|1|650| +2450829|14245|1|38| +2450829|14246|1|| +2450829|14248|1|11| +2450829|14251|1|795| +2450829|14252|1|548| +2450829|14254|1|| +2450829|14257|1|136| +2450829|14258|1|89| +2450829|14260|1|917| +2450829|14263|1|326| +2450829|14264|1|914| +2450829|14266|1|907| +2450829|14269|1|580| +2450829|14270|1|510| +2450829|14272|1|276| +2450829|14275|1|500| +2450829|14276|1|507| +2450829|14278|1|365| +2450829|14281|1|639| +2450829|14282|1|396| +2450829|14284|1|276| +2450829|14287|1|1000| +2450829|14288|1|791| +2450829|14290|1|588| +2450829|14293|1|475| +2450829|14294|1|369| +2450829|14296|1|679| +2450829|14299|1|145| +2450829|14300|1|460| +2450829|14302|1|163| +2450829|14305|1|913| +2450829|14306|1|778| +2450829|14308|1|597| +2450829|14311|1|138| +2450829|14312|1|989| +2450829|14314|1|474| +2450829|14317|1|204| +2450829|14318|1|540| +2450829|14320|1|583| +2450829|14323|1|829| +2450829|14324|1|534| +2450829|14326|1|972| +2450829|14329|1|437| +2450829|14330|1|557| +2450829|14332|1|212| +2450829|14335|1|224| +2450829|14336|1|902| +2450829|14338|1|378| +2450829|14341|1|750| +2450829|14342|1|517| +2450829|14344|1|857| +2450829|14347|1|409| +2450829|14348|1|926| +2450829|14350|1|768| +2450829|14353|1|587| +2450829|14354|1|385| +2450829|14356|1|998| +2450829|14359|1|669| +2450829|14360|1|465| +2450829|14362|1|527| +2450829|14365|1|700| +2450829|14366|1|317| +2450829|14368|1|622| +2450829|14371|1|777| +2450829|14372|1|163| +2450829|14374|1|37| +2450829|14377|1|408| +2450829|14378|1|981| +2450829|14380|1|907| +2450829|14383|1|804| +2450829|14384|1|273| +2450829|14386|1|383| +2450829|14389|1|993| +2450829|14390|1|242| +2450829|14392|1|96| +2450829|14395|1|937| +2450829|14396|1|| +2450829|14398|1|686| +2450829|14401|1|342| +2450829|14402|1|185| +2450829|14404|1|540| +2450829|14407|1|766| +2450829|14408|1|945| +2450829|14410|1|937| +2450829|14413|1|882| +2450829|14414|1|205| +2450829|14416|1|458| +2450829|14419|1|| +2450829|14420|1|449| +2450829|14422|1|256| +2450829|14425|1|36| +2450829|14426|1|418| +2450829|14428|1|337| +2450829|14431|1|681| +2450829|14432|1|95| +2450829|14434|1|516| +2450829|14437|1|959| +2450829|14438|1|774| +2450829|14440|1|658| +2450829|14443|1|792| +2450829|14444|1|670| +2450829|14446|1|255| +2450829|14449|1|104| +2450829|14450|1|596| +2450829|14452|1|252| +2450829|14455|1|107| +2450829|14456|1|339| +2450829|14458|1|982| +2450829|14461|1|930| +2450829|14462|1|435| +2450829|14464|1|827| +2450829|14467|1|855| +2450829|14468|1|501| +2450829|14470|1|998| +2450829|14473|1|420| +2450829|14474|1|818| +2450829|14476|1|445| +2450829|14479|1|397| +2450829|14480|1|514| +2450829|14482|1|957| +2450829|14485|1|619| +2450829|14486|1|85| +2450829|14488|1|155| +2450829|14491|1|821| +2450829|14492|1|956| +2450829|14494|1|677| +2450829|14497|1|477| +2450829|14498|1|647| +2450829|14500|1|289| +2450829|14503|1|673| +2450829|14504|1|| +2450829|14506|1|472| +2450829|14509|1|841| +2450829|14510|1|894| +2450829|14512|1|753| +2450829|14515|1|252| +2450829|14516|1|335| +2450829|14518|1|227| +2450829|14521|1|453| +2450829|14522|1|229| +2450829|14524|1|347| +2450829|14527|1|779| +2450829|14528|1|846| +2450829|14530|1|788| +2450829|14533|1|357| +2450829|14534|1|251| +2450829|14536|1|232| +2450829|14539|1|241| +2450829|14540|1|648| +2450829|14542|1|320| +2450829|14545|1|875| +2450829|14546|1|92| +2450829|14548|1|622| +2450829|14551|1|384| +2450829|14552|1|388| +2450829|14554|1|361| +2450829|14557|1|417| +2450829|14558|1|385| +2450829|14560|1|303| +2450829|14563|1|764| +2450829|14564|1|156| +2450829|14566|1|941| +2450829|14569|1|207| +2450829|14570|1|923| +2450829|14572|1|834| +2450829|14575|1|583| +2450829|14576|1|441| +2450829|14578|1|539| +2450829|14581|1|389| +2450829|14582|1|110| +2450829|14584|1|979| +2450829|14587|1|646| +2450829|14588|1|212| +2450829|14590|1|191| +2450829|14593|1|742| +2450829|14594|1|752| +2450829|14596|1|633| +2450829|14599|1|44| +2450829|14600|1|443| +2450829|14602|1|84| +2450829|14605|1|| +2450829|14606|1|726| +2450829|14608|1|922| +2450829|14611|1|808| +2450829|14612|1|323| +2450829|14614|1|35| +2450829|14617|1|850| +2450829|14618|1|790| +2450829|14620|1|| +2450829|14623|1|654| +2450829|14624|1|80| +2450829|14626|1|72| +2450829|14629|1|191| +2450829|14630|1|415| +2450829|14632|1|236| +2450829|14635|1|404| +2450829|14636|1|156| +2450829|14638|1|159| +2450829|14641|1|207| +2450829|14642|1|647| +2450829|14644|1|445| +2450829|14647|1|660| +2450829|14648|1|333| +2450829|14650|1|196| +2450829|14653|1|518| +2450829|14654|1|624| +2450829|14656|1|826| +2450829|14659|1|760| +2450829|14660|1|737| +2450829|14662|1|768| +2450829|14665|1|841| +2450829|14666|1|100| +2450829|14668|1|243| +2450829|14671|1|895| +2450829|14672|1|154| +2450829|14674|1|222| +2450829|14677|1|972| +2450829|14678|1|944| +2450829|14680|1|391| +2450829|14683|1|168| +2450829|14684|1|436| +2450829|14686|1|247| +2450829|14689|1|238| +2450829|14690|1|612| +2450829|14692|1|| +2450829|14695|1|216| +2450829|14696|1|425| +2450829|14698|1|238| +2450829|14701|1|114| +2450829|14702|1|816| +2450829|14704|1|561| +2450829|14707|1|910| +2450829|14708|1|91| +2450829|14710|1|417| +2450829|14713|1|945| +2450829|14714|1|370| +2450829|14716|1|835| +2450829|14719|1|327| +2450829|14720|1|822| +2450829|14722|1|624| +2450829|14725|1|867| +2450829|14726|1|535| +2450829|14728|1|454| +2450829|14731|1|| +2450829|14732|1|791| +2450829|14734|1|151| +2450829|14737|1|277| +2450829|14738|1|| +2450829|14740|1|40| +2450829|14743|1|347| +2450829|14744|1|107| +2450829|14746|1|160| +2450829|14749|1|233| +2450829|14750|1|10| +2450829|14752|1|915| +2450829|14755|1|| +2450829|14756|1|| +2450829|14758|1|569| +2450829|14761|1|685| +2450829|14762|1|757| +2450829|14764|1|| +2450829|14767|1|745| +2450829|14768|1|585| +2450829|14770|1|330| +2450829|14773|1|115| +2450829|14774|1|373| +2450829|14776|1|100| +2450829|14779|1|242| +2450829|14780|1|633| +2450829|14782|1|265| +2450829|14785|1|162| +2450829|14786|1|84| +2450829|14788|1|494| +2450829|14791|1|950| +2450829|14792|1|617| +2450829|14794|1|858| +2450829|14797|1|158| +2450829|14798|1|62| +2450829|14800|1|109| +2450829|14803|1|981| +2450829|14804|1|572| +2450829|14806|1|141| +2450829|14809|1|701| +2450829|14810|1|790| +2450829|14812|1|197| +2450829|14815|1|822| +2450829|14816|1|870| +2450829|14818|1|392| +2450829|14821|1|370| +2450829|14822|1|585| +2450829|14824|1|| +2450829|14827|1|834| +2450829|14828|1|| +2450829|14830|1|156| +2450829|14833|1|853| +2450829|14834|1|510| +2450829|14836|1|663| +2450829|14839|1|550| +2450829|14840|1|423| +2450829|14842|1|631| +2450829|14845|1|86| +2450829|14846|1|787| +2450829|14848|1|301| +2450829|14851|1|81| +2450829|14852|1|471| +2450829|14854|1|963| +2450829|14857|1|846| +2450829|14858|1|435| +2450829|14860|1|953| +2450829|14863|1|265| +2450829|14864|1|47| +2450829|14866|1|791| +2450829|14869|1|709| +2450829|14870|1|37| +2450829|14872|1|592| +2450829|14875|1|762| +2450829|14876|1|235| +2450829|14878|1|297| +2450829|14881|1|557| +2450829|14882|1|547| +2450829|14884|1|174| +2450829|14887|1|800| +2450829|14888|1|95| +2450829|14890|1|36| +2450829|14893|1|515| +2450829|14894|1|955| +2450829|14896|1|418| +2450829|14899|1|224| +2450829|14900|1|301| +2450829|14902|1|615| +2450829|14905|1|687| +2450829|14906|1|567| +2450829|14908|1|245| +2450829|14911|1|30| +2450829|14912|1|380| +2450829|14914|1|416| +2450829|14917|1|117| +2450829|14918|1|193| +2450829|14920|1|908| +2450829|14923|1|93| +2450829|14924|1|677| +2450829|14926|1|87| +2450829|14929|1|177| +2450829|14930|1|130| +2450829|14932|1|821| +2450829|14935|1|631| +2450829|14936|1|703| +2450829|14938|1|568| +2450829|14941|1|313| +2450829|14942|1|711| +2450829|14944|1|448| +2450829|14947|1|128| +2450829|14948|1|580| +2450829|14950|1|217| +2450829|14953|1|516| +2450829|14954|1|278| +2450829|14956|1|805| +2450829|14959|1|50| +2450829|14960|1|854| +2450829|14962|1|701| +2450829|14965|1|890| +2450829|14966|1|270| +2450829|14968|1|31| +2450829|14971|1|348| +2450829|14972|1|960| +2450829|14974|1|969| +2450829|14977|1|251| +2450829|14978|1|351| +2450829|14980|1|69| +2450829|14983|1|465| +2450829|14984|1|671| +2450829|14986|1|819| +2450829|14989|1|76| +2450829|14990|1|| +2450829|14992|1|170| +2450829|14995|1|37| +2450829|14996|1|69| +2450829|14998|1|308| +2450829|15001|1|| +2450829|15002|1|23| +2450829|15004|1|731| +2450829|15007|1|781| +2450829|15008|1|611| +2450829|15010|1|587| +2450829|15013|1|447| +2450829|15014|1|324| +2450829|15016|1|222| +2450829|15019|1|985| +2450829|15020|1|402| +2450829|15022|1|209| +2450829|15025|1|238| +2450829|15026|1|408| +2450829|15028|1|87| +2450829|15031|1|389| +2450829|15032|1|665| +2450829|15034|1|| +2450829|15037|1|353| +2450829|15038|1|679| +2450829|15040|1|35| +2450829|15043|1|804| +2450829|15044|1|446| +2450829|15046|1|512| +2450829|15049|1|| +2450829|15050|1|667| +2450829|15052|1|943| +2450829|15055|1|512| +2450829|15056|1|239| +2450829|15058|1|175| +2450829|15061|1|417| +2450829|15062|1|359| +2450829|15064|1|340| +2450829|15067|1|504| +2450829|15068|1|2| +2450829|15070|1|515| +2450829|15073|1|274| +2450829|15074|1|55| +2450829|15076|1|160| +2450829|15079|1|| +2450829|15080|1|416| +2450829|15082|1|201| +2450829|15085|1|138| +2450829|15086|1|| +2450829|15088|1|533| +2450829|15091|1|686| +2450829|15092|1|588| +2450829|15094|1|307| +2450829|15097|1|748| +2450829|15098|1|640| +2450829|15100|1|629| +2450829|15103|1|343| +2450829|15104|1|274| +2450829|15106|1|828| +2450829|15109|1|897| +2450829|15110|1|632| +2450829|15112|1|834| +2450829|15115|1|400| +2450829|15116|1|482| +2450829|15118|1|738| +2450829|15121|1|192| +2450829|15122|1|899| +2450829|15124|1|686| +2450829|15127|1|966| +2450829|15128|1|466| +2450829|15130|1|209| +2450829|15133|1|248| +2450829|15134|1|482| +2450829|15136|1|395| +2450829|15139|1|170| +2450829|15140|1|283| +2450829|15142|1|112| +2450829|15145|1|108| +2450829|15146|1|693| +2450829|15148|1|436| +2450829|15151|1|86| +2450829|15152|1|55| +2450829|15154|1|646| +2450829|15157|1|866| +2450829|15158|1|114| +2450829|15160|1|354| +2450829|15163|1|579| +2450829|15164|1|703| +2450829|15166|1|421| +2450829|15169|1|565| +2450829|15170|1|293| +2450829|15172|1|650| +2450829|15175|1|527| +2450829|15176|1|942| +2450829|15178|1|593| +2450829|15181|1|933| +2450829|15182|1|971| +2450829|15184|1|208| +2450829|15187|1|338| +2450829|15188|1|571| +2450829|15190|1|498| +2450829|15193|1|898| +2450829|15194|1|314| +2450829|15196|1|63| +2450829|15199|1|| +2450829|15200|1|788| +2450829|15202|1|443| +2450829|15205|1|347| +2450829|15206|1|617| +2450829|15208|1|882| +2450829|15211|1|977| +2450829|15212|1|727| +2450829|15214|1|393| +2450829|15217|1|| +2450829|15218|1|182| +2450829|15220|1|185| +2450829|15223|1|521| +2450829|15224|1|567| +2450829|15226|1|651| +2450829|15229|1|383| +2450829|15230|1|649| +2450829|15232|1|613| +2450829|15235|1|975| +2450829|15236|1|| +2450829|15238|1|989| +2450829|15241|1|945| +2450829|15242|1|977| +2450829|15244|1|874| +2450829|15247|1|458| +2450829|15248|1|313| +2450829|15250|1|738| +2450829|15253|1|582| +2450829|15254|1|467| +2450829|15256|1|670| +2450829|15259|1|207| +2450829|15260|1|377| +2450829|15262|1|951| +2450829|15265|1|523| +2450829|15266|1|708| +2450829|15268|1|441| +2450829|15271|1|399| +2450829|15272|1|119| +2450829|15274|1|687| +2450829|15277|1|979| +2450829|15278|1|411| +2450829|15280|1|633| +2450829|15283|1|319| +2450829|15284|1|615| +2450829|15286|1|650| +2450829|15289|1|899| +2450829|15290|1|484| +2450829|15292|1|518| +2450829|15295|1|767| +2450829|15296|1|137| +2450829|15298|1|351| +2450829|15301|1|934| +2450829|15302|1|426| +2450829|15304|1|288| +2450829|15307|1|853| +2450829|15308|1|419| +2450829|15310|1|579| +2450829|15313|1|708| +2450829|15314|1|262| +2450829|15316|1|247| +2450829|15319|1|429| +2450829|15320|1|479| +2450829|15322|1|743| +2450829|15325|1|143| +2450829|15326|1|14| +2450829|15328|1|135| +2450829|15331|1|66| +2450829|15332|1|47| +2450829|15334|1|992| +2450829|15337|1|499| +2450829|15338|1|443| +2450829|15340|1|655| +2450829|15343|1|104| +2450829|15344|1|972| +2450829|15346|1|629| +2450829|15349|1|245| +2450829|15350|1|180| +2450829|15352|1|324| +2450829|15355|1|885| +2450829|15356|1|| +2450829|15358|1|392| +2450829|15361|1|864| +2450829|15362|1|666| +2450829|15364|1|5| +2450829|15367|1|698| +2450829|15368|1|118| +2450829|15370|1|518| +2450829|15373|1|482| +2450829|15374|1|892| +2450829|15376|1|163| +2450829|15379|1|298| +2450829|15380|1|139| +2450829|15382|1|288| +2450829|15385|1|402| +2450829|15386|1|575| +2450829|15388|1|941| +2450829|15391|1|95| +2450829|15392|1|273| +2450829|15394|1|138| +2450829|15397|1|462| +2450829|15398|1|607| +2450829|15400|1|381| +2450829|15403|1|905| +2450829|15404|1|783| +2450829|15406|1|796| +2450829|15409|1|865| +2450829|15410|1|63| +2450829|15412|1|754| +2450829|15415|1|827| +2450829|15416|1|661| +2450829|15418|1|251| +2450829|15421|1|862| +2450829|15422|1|11| +2450829|15424|1|| +2450829|15427|1|255| +2450829|15428|1|693| +2450829|15430|1|688| +2450829|15433|1|903| +2450829|15434|1|381| +2450829|15436|1|208| +2450829|15439|1|202| +2450829|15440|1|630| +2450829|15442|1|698| +2450829|15445|1|253| +2450829|15446|1|723| +2450829|15448|1|660| +2450829|15451|1|419| +2450829|15452|1|919| +2450829|15454|1|481| +2450829|15457|1|837| +2450829|15458|1|937| +2450829|15460|1|276| +2450829|15463|1|854| +2450829|15464|1|184| +2450829|15466|1|343| +2450829|15469|1|515| +2450829|15470|1|502| +2450829|15472|1|852| +2450829|15475|1|119| +2450829|15476|1|714| +2450829|15478|1|534| +2450829|15481|1|47| +2450829|15482|1|803| +2450829|15484|1|488| +2450829|15487|1|466| +2450829|15488|1|920| +2450829|15490|1|580| +2450829|15493|1|833| +2450829|15494|1|473| +2450829|15496|1|400| +2450829|15499|1|593| +2450829|15500|1|| +2450829|15502|1|123| +2450829|15505|1|762| +2450829|15506|1|27| +2450829|15508|1|869| +2450829|15511|1|750| +2450829|15512|1|958| +2450829|15514|1|971| +2450829|15517|1|483| +2450829|15518|1|881| +2450829|15520|1|247| +2450829|15523|1|506| +2450829|15524|1|455| +2450829|15526|1|583| +2450829|15529|1|316| +2450829|15530|1|127| +2450829|15532|1|756| +2450829|15535|1|818| +2450829|15536|1|| +2450829|15538|1|576| +2450829|15541|1|360| +2450829|15542|1|721| +2450829|15544|1|123| +2450829|15547|1|503| +2450829|15548|1|527| +2450829|15550|1|81| +2450829|15553|1|39| +2450829|15554|1|560| +2450829|15556|1|268| +2450829|15559|1|177| +2450829|15560|1|180| +2450829|15562|1|666| +2450829|15565|1|301| +2450829|15566|1|99| +2450829|15568|1|730| +2450829|15571|1|339| +2450829|15572|1|553| +2450829|15574|1|521| +2450829|15577|1|846| +2450829|15578|1|424| +2450829|15580|1|844| +2450829|15583|1|| +2450829|15584|1|205| +2450829|15586|1|397| +2450829|15589|1|731| +2450829|15590|1|396| +2450829|15592|1|831| +2450829|15595|1|121| +2450829|15596|1|93| +2450829|15598|1|287| +2450829|15601|1|454| +2450829|15602|1|632| +2450829|15604|1|938| +2450829|15607|1|585| +2450829|15608|1|819| +2450829|15610|1|591| +2450829|15613|1|| +2450829|15614|1|358| +2450829|15616|1|436| +2450829|15619|1|311| +2450829|15620|1|758| +2450829|15622|1|| +2450829|15625|1|122| +2450829|15626|1|113| +2450829|15628|1|497| +2450829|15631|1|170| +2450829|15632|1|22| +2450829|15634|1|289| +2450829|15637|1|308| +2450829|15638|1|1000| +2450829|15640|1|59| +2450829|15643|1|614| +2450829|15644|1|584| +2450829|15646|1|28| +2450829|15649|1|779| +2450829|15650|1|366| +2450829|15652|1|888| +2450829|15655|1|| +2450829|15656|1|591| +2450829|15658|1|877| +2450829|15661|1|640| +2450829|15662|1|638| +2450829|15664|1|937| +2450829|15667|1|848| +2450829|15668|1|631| +2450829|15670|1|490| +2450829|15673|1|62| +2450829|15674|1|955| +2450829|15676|1|372| +2450829|15679|1|343| +2450829|15680|1|| +2450829|15682|1|921| +2450829|15685|1|366| +2450829|15686|1|866| +2450829|15688|1|272| +2450829|15691|1|999| +2450829|15692|1|160| +2450829|15694|1|628| +2450829|15697|1|| +2450829|15698|1|541| +2450829|15700|1|421| +2450829|15703|1|384| +2450829|15704|1|554| +2450829|15706|1|356| +2450829|15709|1|621| +2450829|15710|1|991| +2450829|15712|1|684| +2450829|15715|1|539| +2450829|15716|1|853| +2450829|15718|1|238| +2450829|15721|1|925| +2450829|15722|1|222| +2450829|15724|1|885| +2450829|15727|1|54| +2450829|15728|1|759| +2450829|15730|1|106| +2450829|15733|1|300| +2450829|15734|1|357| +2450829|15736|1|802| +2450829|15739|1|| +2450829|15740|1|744| +2450829|15742|1|842| +2450829|15745|1|| +2450829|15746|1|499| +2450829|15748|1|562| +2450829|15751|1|512| +2450829|15752|1|925| +2450829|15754|1|157| +2450829|15757|1|415| +2450829|15758|1|436| +2450829|15760|1|373| +2450829|15763|1|654| +2450829|15764|1|478| +2450829|15766|1|94| +2450829|15769|1|404| +2450829|15770|1|644| +2450829|15772|1|260| +2450829|15775|1|480| +2450829|15776|1|550| +2450829|15778|1|62| +2450829|15781|1|265| +2450829|15782|1|834| +2450829|15784|1|233| +2450829|15787|1|609| +2450829|15788|1|600| +2450829|15790|1|420| +2450829|15793|1|452| +2450829|15794|1|| +2450829|15796|1|416| +2450829|15799|1|173| +2450829|15800|1|595| +2450829|15802|1|485| +2450829|15805|1|833| +2450829|15806|1|801| +2450829|15808|1|617| +2450829|15811|1|397| +2450829|15812|1|534| +2450829|15814|1|816| +2450829|15817|1|226| +2450829|15818|1|838| +2450829|15820|1|586| +2450829|15823|1|282| +2450829|15824|1|| +2450829|15826|1|| +2450829|15829|1|612| +2450829|15830|1|947| +2450829|15832|1|276| +2450829|15835|1|162| +2450829|15836|1|854| +2450829|15838|1|818| +2450829|15841|1|775| +2450829|15842|1|467| +2450829|15844|1|243| +2450829|15847|1|933| +2450829|15848|1|465| +2450829|15850|1|| +2450829|15853|1|260| +2450829|15854|1|499| +2450829|15856|1|218| +2450829|15859|1|477| +2450829|15860|1|800| +2450829|15862|1|315| +2450829|15865|1|203| +2450829|15866|1|920| +2450829|15868|1|| +2450829|15871|1|111| +2450829|15872|1|453| +2450829|15874|1|593| +2450829|15877|1|312| +2450829|15878|1|908| +2450829|15880|1|814| +2450829|15883|1|316| +2450829|15884|1|22| +2450829|15886|1|450| +2450829|15889|1|717| +2450829|15890|1|355| +2450829|15892|1|37| +2450829|15895|1|903| +2450829|15896|1|626| +2450829|15898|1|257| +2450829|15901|1|580| +2450829|15902|1|878| +2450829|15904|1|| +2450829|15907|1|511| +2450829|15908|1|142| +2450829|15910|1|941| +2450829|15913|1|111| +2450829|15914|1|946| +2450829|15916|1|141| +2450829|15919|1|423| +2450829|15920|1|237| +2450829|15922|1|40| +2450829|15925|1|375| +2450829|15926|1|169| +2450829|15928|1|960| +2450829|15931|1|857| +2450829|15932|1|687| +2450829|15934|1|342| +2450829|15937|1|694| +2450829|15938|1|890| +2450829|15940|1|719| +2450829|15943|1|578| +2450829|15944|1|| +2450829|15946|1|981| +2450829|15949|1|279| +2450829|15950|1|819| +2450829|15952|1|913| +2450829|15955|1|310| +2450829|15956|1|851| +2450829|15958|1|18| +2450829|15961|1|382| +2450829|15962|1|443| +2450829|15964|1|893| +2450829|15967|1|152| +2450829|15968|1|534| +2450829|15970|1|798| +2450829|15973|1|895| +2450829|15974|1|215| +2450829|15976|1|309| +2450829|15979|1|434| +2450829|15980|1|963| +2450829|15982|1|570| +2450829|15985|1|931| +2450829|15986|1|568| +2450829|15988|1|766| +2450829|15991|1|477| +2450829|15992|1|305| +2450829|15994|1|131| +2450829|15997|1|959| +2450829|15998|1|476| +2450829|16000|1|233| +2450829|16003|1|810| +2450829|16004|1|| +2450829|16006|1|50| +2450829|16009|1|543| +2450829|16010|1|833| +2450829|16012|1|| +2450829|16015|1|809| +2450829|16016|1|981| +2450829|16018|1|387| +2450829|16021|1|743| +2450829|16022|1|231| +2450829|16024|1|447| +2450829|16027|1|672| +2450829|16028|1|334| +2450829|16030|1|689| +2450829|16033|1|243| +2450829|16034|1|633| +2450829|16036|1|728| +2450829|16039|1|303| +2450829|16040|1|984| +2450829|16042|1|532| +2450829|16045|1|971| +2450829|16046|1|949| +2450829|16048|1|449| +2450829|16051|1|111| +2450829|16052|1|533| +2450829|16054|1|272| +2450829|16057|1|349| +2450829|16058|1|797| +2450829|16060|1|855| +2450829|16063|1|787| +2450829|16064|1|324| +2450829|16066|1|420| +2450829|16069|1|563| +2450829|16070|1|39| +2450829|16072|1|190| +2450829|16075|1|656| +2450829|16076|1|741| +2450829|16078|1|704| +2450829|16081|1|440| +2450829|16082|1|96| +2450829|16084|1|106| +2450829|16087|1|933| +2450829|16088|1|250| +2450829|16090|1|701| +2450829|16093|1|75| +2450829|16094|1|614| +2450829|16096|1|34| +2450829|16099|1|834| +2450829|16100|1|525| +2450829|16102|1|327| +2450829|16105|1|327| +2450829|16106|1|828| +2450829|16108|1|518| +2450829|16111|1|504| +2450829|16112|1|585| +2450829|16114|1|28| +2450829|16117|1|113| +2450829|16118|1|776| +2450829|16120|1|991| +2450829|16123|1|160| +2450829|16124|1|214| +2450829|16126|1|382| +2450829|16129|1|378| +2450829|16130|1|226| +2450829|16132|1|151| +2450829|16135|1|140| +2450829|16136|1|576| +2450829|16138|1|12| +2450829|16141|1|501| +2450829|16142|1|563| +2450829|16144|1|578| +2450829|16147|1|14| +2450829|16148|1|634| +2450829|16150|1|817| +2450829|16153|1|735| +2450829|16154|1|518| +2450829|16156|1|593| +2450829|16159|1|539| +2450829|16160|1|17| +2450829|16162|1|| +2450829|16165|1|528| +2450829|16166|1|41| +2450829|16168|1|182| +2450829|16171|1|192| +2450829|16172|1|844| +2450829|16174|1|460| +2450829|16177|1|| +2450829|16178|1|966| +2450829|16180|1|750| +2450829|16183|1|345| +2450829|16184|1|376| +2450829|16186|1|819| +2450829|16189|1|429| +2450829|16190|1|837| +2450829|16192|1|571| +2450829|16195|1|268| +2450829|16196|1|867| +2450829|16198|1|53| +2450829|16201|1|646| +2450829|16202|1|812| +2450829|16204|1|994| +2450829|16207|1|522| +2450829|16208|1|597| +2450829|16210|1|280| +2450829|16213|1|610| +2450829|16214|1|226| +2450829|16216|1|243| +2450829|16219|1|863| +2450829|16220|1|844| +2450829|16222|1|985| +2450829|16225|1|619| +2450829|16226|1|90| +2450829|16228|1|670| +2450829|16231|1|930| +2450829|16232|1|5| +2450829|16234|1|380| +2450829|16237|1|120| +2450829|16238|1|768| +2450829|16240|1|649| +2450829|16243|1|992| +2450829|16244|1|426| +2450829|16246|1|879| +2450829|16249|1|690| +2450829|16250|1|488| +2450829|16252|1|771| +2450829|16255|1|527| +2450829|16256|1|226| +2450829|16258|1|650| +2450829|16261|1|492| +2450829|16262|1|647| +2450829|16264|1|878| +2450829|16267|1|762| +2450829|16268|1|330| +2450829|16270|1|165| +2450829|16273|1|775| +2450829|16274|1|362| +2450829|16276|1|87| +2450829|16279|1|467| +2450829|16280|1|821| +2450829|16282|1|92| +2450829|16285|1|722| +2450829|16286|1|446| +2450829|16288|1|265| +2450829|16291|1|828| +2450829|16292|1|280| +2450829|16294|1|130| +2450829|16297|1|535| +2450829|16298|1|982| +2450829|16300|1|916| +2450829|16303|1|308| +2450829|16304|1|914| +2450829|16306|1|888| +2450829|16309|1|975| +2450829|16310|1|10| +2450829|16312|1|194| +2450829|16315|1|600| +2450829|16316|1|835| +2450829|16318|1|557| +2450829|16321|1|572| +2450829|16322|1|462| +2450829|16324|1|733| +2450829|16327|1|118| +2450829|16328|1|743| +2450829|16330|1|311| +2450829|16333|1|106| +2450829|16334|1|142| +2450829|16336|1|511| +2450829|16339|1|705| +2450829|16340|1|274| +2450829|16342|1|745| +2450829|16345|1|774| +2450829|16346|1|83| +2450829|16348|1|| +2450829|16351|1|463| +2450829|16352|1|544| +2450829|16354|1|418| +2450829|16357|1|386| +2450829|16358|1|405| +2450829|16360|1|986| +2450829|16363|1|291| +2450829|16364|1|791| +2450829|16366|1|897| +2450829|16369|1|942| +2450829|16370|1|87| +2450829|16372|1|779| +2450829|16375|1|59| +2450829|16376|1|472| +2450829|16378|1|224| +2450829|16381|1|752| +2450829|16382|1|131| +2450829|16384|1|842| +2450829|16387|1|276| +2450829|16388|1|749| +2450829|16390|1|431| +2450829|16393|1|877| +2450829|16394|1|661| +2450829|16396|1|87| +2450829|16399|1|328| +2450829|16400|1|416| +2450829|16402|1|545| +2450829|16405|1|450| +2450829|16406|1|148| +2450829|16408|1|798| +2450829|16411|1|78| +2450829|16412|1|924| +2450829|16414|1|45| +2450829|16417|1|888| +2450829|16418|1|347| +2450829|16420|1|690| +2450829|16423|1|696| +2450829|16424|1|711| +2450829|16426|1|723| +2450829|16429|1|331| +2450829|16430|1|454| +2450829|16432|1|238| +2450829|16435|1|282| +2450829|16436|1|344| +2450829|16438|1|991| +2450829|16441|1|| +2450829|16442|1|554| +2450829|16444|1|97| +2450829|16447|1|16| +2450829|16448|1|554| +2450829|16450|1|245| +2450829|16453|1|169| +2450829|16454|1|875| +2450829|16456|1|271| +2450829|16459|1|742| +2450829|16460|1|324| +2450829|16462|1|704| +2450829|16465|1|| +2450829|16466|1|289| +2450829|16468|1|692| +2450829|16471|1|| +2450829|16472|1|278| +2450829|16474|1|756| +2450829|16477|1|263| +2450829|16478|1|607| +2450829|16480|1|172| +2450829|16483|1|385| +2450829|16484|1|854| +2450829|16486|1|153| +2450829|16489|1|560| +2450829|16490|1|849| +2450829|16492|1|957| +2450829|16495|1|801| +2450829|16496|1|179| +2450829|16498|1|177| +2450829|16501|1|718| +2450829|16502|1|168| +2450829|16504|1|639| +2450829|16507|1|283| +2450829|16508|1|959| +2450829|16510|1|220| +2450829|16513|1|290| +2450829|16514|1|791| +2450829|16516|1|442| +2450829|16519|1|124| +2450829|16520|1|454| +2450829|16522|1|393| +2450829|16525|1|489| +2450829|16526|1|123| +2450829|16528|1|80| +2450829|16531|1|263| +2450829|16532|1|250| +2450829|16534|1|110| +2450829|16537|1|2| +2450829|16538|1|649| +2450829|16540|1|580| +2450829|16543|1|| +2450829|16544|1|205| +2450829|16546|1|79| +2450829|16549|1|791| +2450829|16550|1|70| +2450829|16552|1|540| +2450829|16555|1|985| +2450829|16556|1|985| +2450829|16558|1|548| +2450829|16561|1|748| +2450829|16562|1|381| +2450829|16564|1|621| +2450829|16567|1|874| +2450829|16568|1|| +2450829|16570|1|96| +2450829|16573|1|810| +2450829|16574|1|148| +2450829|16576|1|750| +2450829|16579|1|695| +2450829|16580|1|170| +2450829|16582|1|181| +2450829|16585|1|610| +2450829|16586|1|552| +2450829|16588|1|| +2450829|16591|1|96| +2450829|16592|1|253| +2450829|16594|1|324| +2450829|16597|1|441| +2450829|16598|1|464| +2450829|16600|1|312| +2450829|16603|1|8| +2450829|16604|1|563| +2450829|16606|1|66| +2450829|16609|1|756| +2450829|16610|1|415| +2450829|16612|1|946| +2450829|16615|1|548| +2450829|16616|1|738| +2450829|16618|1|34| +2450829|16621|1|117| +2450829|16622|1|| +2450829|16624|1|731| +2450829|16627|1|795| +2450829|16628|1|373| +2450829|16630|1|812| +2450829|16633|1|502| +2450829|16634|1|61| +2450829|16636|1|568| +2450829|16639|1|78| +2450829|16640|1|961| +2450829|16642|1|399| +2450829|16645|1|345| +2450829|16646|1|178| +2450829|16648|1|| +2450829|16651|1|568| +2450829|16652|1|514| +2450829|16654|1|230| +2450829|16657|1|63| +2450829|16658|1|460| +2450829|16660|1|189| +2450829|16663|1|879| +2450829|16664|1|765| +2450829|16666|1|764| +2450829|16669|1|821| +2450829|16670|1|192| +2450829|16672|1|12| +2450829|16675|1|100| +2450829|16676|1|321| +2450829|16678|1|45| +2450829|16681|1|695| +2450829|16682|1|647| +2450829|16684|1|34| +2450829|16687|1|994| +2450829|16688|1|412| +2450829|16690|1|86| +2450829|16693|1|444| +2450829|16694|1|983| +2450829|16696|1|267| +2450829|16699|1|735| +2450829|16700|1|78| +2450829|16702|1|757| +2450829|16705|1|85| +2450829|16706|1|967| +2450829|16708|1|195| +2450829|16711|1|244| +2450829|16712|1|993| +2450829|16714|1|| +2450829|16717|1|173| +2450829|16718|1|145| +2450829|16720|1|996| +2450829|16723|1|44| +2450829|16724|1|271| +2450829|16726|1|698| +2450829|16729|1|374| +2450829|16730|1|757| +2450829|16732|1|479| +2450829|16735|1|588| +2450829|16736|1|422| +2450829|16738|1|24| +2450829|16741|1|692| +2450829|16742|1|493| +2450829|16744|1|486| +2450829|16747|1|329| +2450829|16748|1|104| +2450829|16750|1|45| +2450829|16753|1|684| +2450829|16754|1|737| +2450829|16756|1|545| +2450829|16759|1|628| +2450829|16760|1|316| +2450829|16762|1|407| +2450829|16765|1|639| +2450829|16766|1|433| +2450829|16768|1|176| +2450829|16771|1|407| +2450829|16772|1|355| +2450829|16774|1|| +2450829|16777|1|728| +2450829|16778|1|189| +2450829|16780|1|802| +2450829|16783|1|106| +2450829|16784|1|380| +2450829|16786|1|123| +2450829|16789|1|11| +2450829|16790|1|769| +2450829|16792|1|491| +2450829|16795|1|118| +2450829|16796|1|192| +2450829|16798|1|113| +2450829|16801|1|277| +2450829|16802|1|109| +2450829|16804|1|147| +2450829|16807|1|797| +2450829|16808|1|626| +2450829|16810|1|610| +2450829|16813|1|767| +2450829|16814|1|978| +2450829|16816|1|610| +2450829|16819|1|294| +2450829|16820|1|961| +2450829|16822|1|281| +2450829|16825|1|340| +2450829|16826|1|873| +2450829|16828|1|| +2450829|16831|1|654| +2450829|16832|1|809| +2450829|16834|1|700| +2450829|16837|1|148| +2450829|16838|1|564| +2450829|16840|1|726| +2450829|16843|1|792| +2450829|16844|1|426| +2450829|16846|1|317| +2450829|16849|1|5| +2450829|16850|1|464| +2450829|16852|1|642| +2450829|16855|1|10| +2450829|16856|1|163| +2450829|16858|1|337| +2450829|16861|1|625| +2450829|16862|1|218| +2450829|16864|1|545| +2450829|16867|1|427| +2450829|16868|1|324| +2450829|16870|1|917| +2450829|16873|1|664| +2450829|16874|1|39| +2450829|16876|1|690| +2450829|16879|1|749| +2450829|16880|1|317| +2450829|16882|1|909| +2450829|16885|1|335| +2450829|16886|1|421| +2450829|16888|1|232| +2450829|16891|1|866| +2450829|16892|1|395| +2450829|16894|1|425| +2450829|16897|1|123| +2450829|16898|1|643| +2450829|16900|1|868| +2450829|16903|1|929| +2450829|16904|1|62| +2450829|16906|1|577| +2450829|16909|1|329| +2450829|16910|1|758| +2450829|16912|1|| +2450829|16915|1|181| +2450829|16916|1|56| +2450829|16918|1|933| +2450829|16921|1|136| +2450829|16922|1|415| +2450829|16924|1|746| +2450829|16927|1|903| +2450829|16928|1|647| +2450829|16930|1|783| +2450829|16933|1|652| +2450829|16934|1|852| +2450829|16936|1|850| +2450829|16939|1|251| +2450829|16940|1|366| +2450829|16942|1|840| +2450829|16945|1|123| +2450829|16946|1|971| +2450829|16948|1|670| +2450829|16951|1|624| +2450829|16952|1|936| +2450829|16954|1|746| +2450829|16957|1|403| +2450829|16958|1|795| +2450829|16960|1|535| +2450829|16963|1|792| +2450829|16964|1|375| +2450829|16966|1|875| +2450829|16969|1|308| +2450829|16970|1|| +2450829|16972|1|240| +2450829|16975|1|916| +2450829|16976|1|785| +2450829|16978|1|291| +2450829|16981|1|574| +2450829|16982|1|44| +2450829|16984|1|107| +2450829|16987|1|973| +2450829|16988|1|73| +2450829|16990|1|494| +2450829|16993|1|223| +2450829|16994|1|311| +2450829|16996|1|125| +2450829|16999|1|650| +2450829|17000|1|383| +2450829|17002|1|6| +2450829|17005|1|264| +2450829|17006|1|| +2450829|17008|1|795| +2450829|17011|1|945| +2450829|17012|1|900| +2450829|17014|1|571| +2450829|17017|1|60| +2450829|17018|1|| +2450829|17020|1|730| +2450829|17023|1|912| +2450829|17024|1|923| +2450829|17026|1|906| +2450829|17029|1|716| +2450829|17030|1|890| +2450829|17032|1|595| +2450829|17035|1|904| +2450829|17036|1|396| +2450829|17038|1|199| +2450829|17041|1|315| +2450829|17042|1|925| +2450829|17044|1|499| +2450829|17047|1|402| +2450829|17048|1|922| +2450829|17050|1|495| +2450829|17053|1|827| +2450829|17054|1|43| +2450829|17056|1|765| +2450829|17059|1|428| +2450829|17060|1|75| +2450829|17062|1|592| +2450829|17065|1|285| +2450829|17066|1|136| +2450829|17068|1|58| +2450829|17071|1|879| +2450829|17072|1|416| +2450829|17074|1|902| +2450829|17077|1|914| +2450829|17078|1|981| +2450829|17080|1|19| +2450829|17083|1|319| +2450829|17084|1|90| +2450829|17086|1|7| +2450829|17089|1|328| +2450829|17090|1|133| +2450829|17092|1|696| +2450829|17095|1|816| +2450829|17096|1|446| +2450829|17098|1|854| +2450829|17101|1|786| +2450829|17102|1|53| +2450829|17104|1|891| +2450829|17107|1|11| +2450829|17108|1|955| +2450829|17110|1|64| +2450829|17113|1|890| +2450829|17114|1|650| +2450829|17116|1|316| +2450829|17119|1|589| +2450829|17120|1|284| +2450829|17122|1|| +2450829|17125|1|841| +2450829|17126|1|650| +2450829|17128|1|268| +2450829|17131|1|494| +2450829|17132|1|352| +2450829|17134|1|470| +2450829|17137|1|528| +2450829|17138|1|57| +2450829|17140|1|312| +2450829|17143|1|531| +2450829|17144|1|389| +2450829|17146|1|671| +2450829|17149|1|779| +2450829|17150|1|53| +2450829|17152|1|714| +2450829|17155|1|669| +2450829|17156|1|104| +2450829|17158|1|911| +2450829|17161|1|250| +2450829|17162|1|394| +2450829|17164|1|117| +2450829|17167|1|528| +2450829|17168|1|130| +2450829|17170|1|773| +2450829|17173|1|457| +2450829|17174|1|363| +2450829|17176|1|192| +2450829|17179|1|729| +2450829|17180|1|759| +2450829|17182|1|683| +2450829|17185|1|578| +2450829|17186|1|457| +2450829|17188|1|845| +2450829|17191|1|412| +2450829|17192|1|487| +2450829|17194|1|170| +2450829|17197|1|888| +2450829|17198|1|69| +2450829|17200|1|469| +2450829|17203|1|321| +2450829|17204|1|292| +2450829|17206|1|171| +2450829|17209|1|8| +2450829|17210|1|372| +2450829|17212|1|66| +2450829|17215|1|571| +2450829|17216|1|708| +2450829|17218|1|41| +2450829|17221|1|201| +2450829|17222|1|137| +2450829|17224|1|118| +2450829|17227|1|497| +2450829|17228|1|390| +2450829|17230|1|664| +2450829|17233|1|405| +2450829|17234|1|313| +2450829|17236|1|736| +2450829|17239|1|| +2450829|17240|1|691| +2450829|17242|1|295| +2450829|17245|1|477| +2450829|17246|1|238| +2450829|17248|1|600| +2450829|17251|1|562| +2450829|17252|1|417| +2450829|17254|1|| +2450829|17257|1|579| +2450829|17258|1|914| +2450829|17260|1|131| +2450829|17263|1|808| +2450829|17264|1|954| +2450829|17266|1|297| +2450829|17269|1|865| +2450829|17270|1|67| +2450829|17272|1|857| +2450829|17275|1|986| +2450829|17276|1|863| +2450829|17278|1|676| +2450829|17281|1|437| +2450829|17282|1|215| +2450829|17284|1|40| +2450829|17287|1|211| +2450829|17288|1|216| +2450829|17290|1|| +2450829|17293|1|471| +2450829|17294|1|950| +2450829|17296|1|709| +2450829|17299|1|| +2450829|17300|1|323| +2450829|17302|1|816| +2450829|17305|1|262| +2450829|17306|1|849| +2450829|17308|1|851| +2450829|17311|1|437| +2450829|17312|1|479| +2450829|17314|1|576| +2450829|17317|1|613| +2450829|17318|1|97| +2450829|17320|1|672| +2450829|17323|1|742| +2450829|17324|1|365| +2450829|17326|1|872| +2450829|17329|1|855| +2450829|17330|1|642| +2450829|17332|1|307| +2450829|17335|1|133| +2450829|17336|1|606| +2450829|17338|1|659| +2450829|17341|1|702| +2450829|17342|1|670| +2450829|17344|1|88| +2450829|17347|1|642| +2450829|17348|1|450| +2450829|17350|1|580| +2450829|17353|1|340| +2450829|17354|1|188| +2450829|17356|1|506| +2450829|17359|1|287| +2450829|17360|1|569| +2450829|17362|1|34| +2450829|17365|1|410| +2450829|17366|1|87| +2450829|17368|1|611| +2450829|17371|1|138| +2450829|17372|1|694| +2450829|17374|1|440| +2450829|17377|1|160| +2450829|17378|1|314| +2450829|17380|1|423| +2450829|17383|1|6| +2450829|17384|1|598| +2450829|17386|1|324| +2450829|17389|1|471| +2450829|17390|1|495| +2450829|17392|1|476| +2450829|17395|1|160| +2450829|17396|1|789| +2450829|17398|1|81| +2450829|17401|1|539| +2450829|17402|1|746| +2450829|17404|1|774| +2450829|17407|1|479| +2450829|17408|1|971| +2450829|17410|1|79| +2450829|17413|1|693| +2450829|17414|1|735| +2450829|17416|1|154| +2450829|17419|1|634| +2450829|17420|1|820| +2450829|17422|1|382| +2450829|17425|1|26| +2450829|17426|1|780| +2450829|17428|1|678| +2450829|17431|1|226| +2450829|17432|1|922| +2450829|17434|1|188| +2450829|17437|1|883| +2450829|17438|1|272| +2450829|17440|1|278| +2450829|17443|1|301| +2450829|17444|1|469| +2450829|17446|1|294| +2450829|17449|1|853| +2450829|17450|1|684| +2450829|17452|1|701| +2450829|17455|1|473| +2450829|17456|1|51| +2450829|17458|1|730| +2450829|17461|1|72| +2450829|17462|1|355| +2450829|17464|1|77| +2450829|17467|1|926| +2450829|17468|1|521| +2450829|17470|1|149| +2450829|17473|1|| +2450829|17474|1|732| +2450829|17476|1|578| +2450829|17479|1|745| +2450829|17480|1|804| +2450829|17482|1|393| +2450829|17485|1|169| +2450829|17486|1|397| +2450829|17488|1|645| +2450829|17491|1|943| +2450829|17492|1|974| +2450829|17494|1|671| +2450829|17497|1|527| +2450829|17498|1|15| +2450829|17500|1|230| +2450829|17503|1|753| +2450829|17504|1|259| +2450829|17506|1|358| +2450829|17509|1|532| +2450829|17510|1|91| +2450829|17512|1|376| +2450829|17515|1|272| +2450829|17516|1|866| +2450829|17518|1|552| +2450829|17521|1|59| +2450829|17522|1|71| +2450829|17524|1|436| +2450829|17527|1|305| +2450829|17528|1|259| +2450829|17530|1|809| +2450829|17533|1|19| +2450829|17534|1|525| +2450829|17536|1|292| +2450829|17539|1|111| +2450829|17540|1|574| +2450829|17542|1|756| +2450829|17545|1|545| +2450829|17546|1|255| +2450829|17548|1|96| +2450829|17551|1|541| +2450829|17552|1|827| +2450829|17554|1|| +2450829|17557|1|943| +2450829|17558|1|251| +2450829|17560|1|500| +2450829|17563|1|981| +2450829|17564|1|263| +2450829|17566|1|36| +2450829|17569|1|885| +2450829|17570|1|248| +2450829|17572|1|632| +2450829|17575|1|839| +2450829|17576|1|567| +2450829|17578|1|183| +2450829|17581|1|876| +2450829|17582|1|966| +2450829|17584|1|65| +2450829|17587|1|977| +2450829|17588|1|285| +2450829|17590|1|203| +2450829|17593|1|625| +2450829|17594|1|460| +2450829|17596|1|208| +2450829|17599|1|927| +2450829|17600|1|725| +2450829|17602|1|104| +2450829|17605|1|893| +2450829|17606|1|677| +2450829|17608|1|117| +2450829|17611|1|| +2450829|17612|1|188| +2450829|17614|1|687| +2450829|17617|1|826| +2450829|17618|1|417| +2450829|17620|1|630| +2450829|17623|1|157| +2450829|17624|1|522| +2450829|17626|1|985| +2450829|17629|1|264| +2450829|17630|1|878| +2450829|17632|1|350| +2450829|17635|1|683| +2450829|17636|1|332| +2450829|17638|1|709| +2450829|17641|1|942| +2450829|17642|1|493| +2450829|17644|1|202| +2450829|17647|1|263| +2450829|17648|1|389| +2450829|17650|1|997| +2450829|17653|1|269| +2450829|17654|1|350| +2450829|17656|1|967| +2450829|17659|1|49| +2450829|17660|1|776| +2450829|17662|1|305| +2450829|17665|1|786| +2450829|17666|1|133| +2450829|17668|1|981| +2450829|17671|1|398| +2450829|17672|1|549| +2450829|17674|1|513| +2450829|17677|1|2| +2450829|17678|1|195| +2450829|17680|1|734| +2450829|17683|1|233| +2450829|17684|1|611| +2450829|17686|1|652| +2450829|17689|1|521| +2450829|17690|1|921| +2450829|17692|1|659| +2450829|17695|1|770| +2450829|17696|1|170| +2450829|17698|1|585| +2450829|17701|1|685| +2450829|17702|1|749| +2450829|17704|1|808| +2450829|17707|1|482| +2450829|17708|1|888| +2450829|17710|1|64| +2450829|17713|1|495| +2450829|17714|1|417| +2450829|17716|1|376| +2450829|17719|1|27| +2450829|17720|1|34| +2450829|17722|1|522| +2450829|17725|1|652| +2450829|17726|1|487| +2450829|17728|1|905| +2450829|17731|1|178| +2450829|17732|1|87| +2450829|17734|1|309| +2450829|17737|1|699| +2450829|17738|1|248| +2450829|17740|1|456| +2450829|17743|1|533| +2450829|17744|1|786| +2450829|17746|1|151| +2450829|17749|1|| +2450829|17750|1|85| +2450829|17752|1|337| +2450829|17755|1|| +2450829|17756|1|235| +2450829|17758|1|922| +2450829|17761|1|774| +2450829|17762|1|309| +2450829|17764|1|147| +2450829|17767|1|676| +2450829|17768|1|880| +2450829|17770|1|675| +2450829|17773|1|112| +2450829|17774|1|566| +2450829|17776|1|339| +2450829|17779|1|218| +2450829|17780|1|259| +2450829|17782|1|908| +2450829|17785|1|164| +2450829|17786|1|342| +2450829|17788|1|30| +2450829|17791|1|82| +2450829|17792|1|977| +2450829|17794|1|702| +2450829|17797|1|203| +2450829|17798|1|385| +2450829|17800|1|124| +2450829|17803|1|685| +2450829|17804|1|899| +2450829|17806|1|205| +2450829|17809|1|575| +2450829|17810|1|909| +2450829|17812|1|938| +2450829|17815|1|371| +2450829|17816|1|848| +2450829|17818|1|617| +2450829|17821|1|899| +2450829|17822|1|636| +2450829|17824|1|85| +2450829|17827|1|201| +2450829|17828|1|121| +2450829|17830|1|991| +2450829|17833|1|390| +2450829|17834|1|0| +2450829|17836|1|619| +2450829|17839|1|101| +2450829|17840|1|997| +2450829|17842|1|823| +2450829|17845|1|333| +2450829|17846|1|542| +2450829|17848|1|882| +2450829|17851|1|173| +2450829|17852|1|513| +2450829|17854|1|28| +2450829|17857|1|174| +2450829|17858|1|666| +2450829|17860|1|965| +2450829|17863|1|520| +2450829|17864|1|738| +2450829|17866|1|957| +2450829|17869|1|186| +2450829|17870|1|311| +2450829|17872|1|691| +2450829|17875|1|51| +2450829|17876|1|221| +2450829|17878|1|976| +2450829|17881|1|164| +2450829|17882|1|364| +2450829|17884|1|338| +2450829|17887|1|815| +2450829|17888|1|350| +2450829|17890|1|761| +2450829|17893|1|236| +2450829|17894|1|223| +2450829|17896|1|205| +2450829|17899|1|689| +2450829|17900|1|533| +2450829|17902|1|132| +2450829|17905|1|912| +2450829|17906|1|845| +2450829|17908|1|249| +2450829|17911|1|538| +2450829|17912|1|373| +2450829|17914|1|172| +2450829|17917|1|456| +2450829|17918|1|54| +2450829|17920|1|290| +2450829|17923|1|695| +2450829|17924|1|664| +2450829|17926|1|817| +2450829|17929|1|460| +2450829|17930|1|497| +2450829|17932|1|956| +2450829|17935|1|985| +2450829|17936|1|791| +2450829|17938|1|851| +2450829|17941|1|699| +2450829|17942|1|| +2450829|17944|1|877| +2450829|17947|1|185| +2450829|17948|1|256| +2450829|17950|1|273| +2450829|17953|1|565| +2450829|17954|1|902| +2450829|17956|1|613| +2450829|17959|1|43| +2450829|17960|1|929| +2450829|17962|1|570| +2450829|17965|1|126| +2450829|17966|1|979| +2450829|17968|1|435| +2450829|17971|1|| +2450829|17972|1|811| +2450829|17974|1|97| +2450829|17977|1|238| +2450829|17978|1|658| +2450829|17980|1|652| +2450829|17983|1|89| +2450829|17984|1|778| +2450829|17986|1|943| +2450829|17989|1|748| +2450829|17990|1|30| +2450829|17992|1|63| +2450829|17995|1|70| +2450829|17996|1|240| +2450829|17998|1|507| +2450829|1|2|767| +2450829|2|2|684| +2450829|4|2|800| +2450829|7|2|998| +2450829|8|2|10| +2450829|10|2|140| +2450829|13|2|30| +2450829|14|2|416| +2450829|16|2|952| +2450829|19|2|270| +2450829|20|2|907| +2450829|22|2|322| +2450829|25|2|991| +2450829|26|2|550| +2450829|28|2|870| +2450829|31|2|160| +2450829|32|2|134| +2450829|34|2|419| +2450829|37|2|94| +2450829|38|2|653| +2450829|40|2|635| +2450829|43|2|804| +2450829|44|2|556| +2450829|46|2|358| +2450829|49|2|180| +2450829|50|2|192| +2450829|52|2|781| +2450829|55|2|335| +2450829|56|2|335| +2450829|58|2|692| +2450829|61|2|135| +2450829|62|2|119| +2450829|64|2|587| +2450829|67|2|316| +2450829|68|2|220| +2450829|70|2|904| +2450829|73|2|945| +2450829|74|2|124| +2450829|76|2|969| +2450829|79|2|327| +2450829|80|2|839| +2450829|82|2|342| +2450829|85|2|918| +2450829|86|2|308| +2450829|88|2|278| +2450829|91|2|781| +2450829|92|2|618| +2450829|94|2|119| +2450829|97|2|819| +2450829|98|2|494| +2450829|100|2|840| +2450829|103|2|780| +2450829|104|2|171| +2450829|106|2|714| +2450829|109|2|864| +2450829|110|2|848| +2450829|112|2|679| +2450829|115|2|199| +2450829|116|2|452| +2450829|118|2|129| +2450829|121|2|903| +2450829|122|2|398| +2450829|124|2|851| +2450829|127|2|801| +2450829|128|2|741| +2450829|130|2|584| +2450829|133|2|226| +2450829|134|2|61| +2450829|136|2|| +2450829|139|2|511| +2450829|140|2|406| +2450829|142|2|72| +2450829|145|2|167| +2450829|146|2|612| +2450829|148|2|366| +2450829|151|2|189| +2450829|152|2|91| +2450829|154|2|113| +2450829|157|2|133| +2450829|158|2|| +2450829|160|2|59| +2450829|163|2|164| +2450829|164|2|300| +2450829|166|2|442| +2450829|169|2|380| +2450829|170|2|651| +2450829|172|2|0| +2450829|175|2|176| +2450829|176|2|605| +2450829|178|2|413| +2450829|181|2|445| +2450829|182|2|| +2450829|184|2|134| +2450829|187|2|815| +2450829|188|2|393| +2450829|190|2|766| +2450829|193|2|803| +2450829|194|2|| +2450829|196|2|146| +2450829|199|2|693| +2450829|200|2|548| +2450829|202|2|446| +2450829|205|2|252| +2450829|206|2|96| +2450829|208|2|133| +2450829|211|2|737| +2450829|212|2|597| +2450829|214|2|309| +2450829|217|2|992| +2450829|218|2|632| +2450829|220|2|263| +2450829|223|2|267| +2450829|224|2|444| +2450829|226|2|858| +2450829|229|2|64| +2450829|230|2|391| +2450829|232|2|974| +2450829|235|2|413| +2450829|236|2|579| +2450829|238|2|553| +2450829|241|2|334| +2450829|242|2|373| +2450829|244|2|472| +2450829|247|2|728| +2450829|248|2|| +2450829|250|2|23| +2450829|253|2|439| +2450829|254|2|912| +2450829|256|2|710| +2450829|259|2|534| +2450829|260|2|516| +2450829|262|2|106| +2450829|265|2|678| +2450829|266|2|572| +2450829|268|2|686| +2450829|271|2|505| +2450829|272|2|204| +2450829|274|2|| +2450829|277|2|468| +2450829|278|2|395| +2450829|280|2|326| +2450829|283|2|839| +2450829|284|2|388| +2450829|286|2|247| +2450829|289|2|783| +2450829|290|2|125| +2450829|292|2|556| +2450829|295|2|459| +2450829|296|2|827| +2450829|298|2|426| +2450829|301|2|687| +2450829|302|2|299| +2450829|304|2|214| +2450829|307|2|1000| +2450829|308|2|662| +2450829|310|2|584| +2450829|313|2|793| +2450829|314|2|141| +2450829|316|2|512| +2450829|319|2|811| +2450829|320|2|951| +2450829|322|2|935| +2450829|325|2|444| +2450829|326|2|39| +2450829|328|2|| +2450829|331|2|506| +2450829|332|2|| +2450829|334|2|22| +2450829|337|2|82| +2450829|338|2|364| +2450829|340|2|990| +2450829|343|2|442| +2450829|344|2|870| +2450829|346|2|717| +2450829|349|2|708| +2450829|350|2|292| +2450829|352|2|77| +2450829|355|2|413| +2450829|356|2|225| +2450829|358|2|596| +2450829|361|2|902| +2450829|362|2|591| +2450829|364|2|813| +2450829|367|2|841| +2450829|368|2|26| +2450829|370|2|| +2450829|373|2|215| +2450829|374|2|947| +2450829|376|2|141| +2450829|379|2|613| +2450829|380|2|331| +2450829|382|2|349| +2450829|385|2|242| +2450829|386|2|873| +2450829|388|2|298| +2450829|391|2|992| +2450829|392|2|706| +2450829|394|2|897| +2450829|397|2|811| +2450829|398|2|255| +2450829|400|2|868| +2450829|403|2|700| +2450829|404|2|209| +2450829|406|2|717| +2450829|409|2|218| +2450829|410|2|970| +2450829|412|2|507| +2450829|415|2|982| +2450829|416|2|272| +2450829|418|2|649| +2450829|421|2|52| +2450829|422|2|484| +2450829|424|2|514| +2450829|427|2|956| +2450829|428|2|425| +2450829|430|2|| +2450829|433|2|494| +2450829|434|2|901| +2450829|436|2|8| +2450829|439|2|702| +2450829|440|2|594| +2450829|442|2|997| +2450829|445|2|3| +2450829|446|2|945| +2450829|448|2|237| +2450829|451|2|601| +2450829|452|2|410| +2450829|454|2|20| +2450829|457|2|997| +2450829|458|2|357| +2450829|460|2|516| +2450829|463|2|564| +2450829|464|2|591| +2450829|466|2|697| +2450829|469|2|32| +2450829|470|2|31| +2450829|472|2|186| +2450829|475|2|144| +2450829|476|2|4| +2450829|478|2|594| +2450829|481|2|206| +2450829|482|2|802| +2450829|484|2|762| +2450829|487|2|206| +2450829|488|2|548| +2450829|490|2|725| +2450829|493|2|781| +2450829|494|2|409| +2450829|496|2|407| +2450829|499|2|784| +2450829|500|2|321| +2450829|502|2|906| +2450829|505|2|63| +2450829|506|2|934| +2450829|508|2|783| +2450829|511|2|37| +2450829|512|2|161| +2450829|514|2|186| +2450829|517|2|435| +2450829|518|2|461| +2450829|520|2|22| +2450829|523|2|596| +2450829|524|2|776| +2450829|526|2|109| +2450829|529|2|22| +2450829|530|2|327| +2450829|532|2|823| +2450829|535|2|| +2450829|536|2|931| +2450829|538|2|308| +2450829|541|2|947| +2450829|542|2|31| +2450829|544|2|800| +2450829|547|2|161| +2450829|548|2|668| +2450829|550|2|952| +2450829|553|2|853| +2450829|554|2|601| +2450829|556|2|903| +2450829|559|2|11| +2450829|560|2|446| +2450829|562|2|| +2450829|565|2|106| +2450829|566|2|785| +2450829|568|2|949| +2450829|571|2|560| +2450829|572|2|34| +2450829|574|2|150| +2450829|577|2|254| +2450829|578|2|574| +2450829|580|2|175| +2450829|583|2|676| +2450829|584|2|507| +2450829|586|2|842| +2450829|589|2|7| +2450829|590|2|971| +2450829|592|2|46| +2450829|595|2|275| +2450829|596|2|15| +2450829|598|2|| +2450829|601|2|80| +2450829|602|2|134| +2450829|604|2|655| +2450829|607|2|573| +2450829|608|2|282| +2450829|610|2|929| +2450829|613|2|802| +2450829|614|2|612| +2450829|616|2|206| +2450829|619|2|502| +2450829|620|2|548| +2450829|622|2|669| +2450829|625|2|273| +2450829|626|2|845| +2450829|628|2|132| +2450829|631|2|327| +2450829|632|2|125| +2450829|634|2|999| +2450829|637|2|41| +2450829|638|2|779| +2450829|640|2|623| +2450829|643|2|11| +2450829|644|2|888| +2450829|646|2|339| +2450829|649|2|944| +2450829|650|2|865| +2450829|652|2|556| +2450829|655|2|5| +2450829|656|2|511| +2450829|658|2|776| +2450829|661|2|757| +2450829|662|2|753| +2450829|664|2|315| +2450829|667|2|494| +2450829|668|2|278| +2450829|670|2|342| +2450829|673|2|189| +2450829|674|2|260| +2450829|676|2|828| +2450829|679|2|718| +2450829|680|2|732| +2450829|682|2|367| +2450829|685|2|160| +2450829|686|2|545| +2450829|688|2|764| +2450829|691|2|| +2450829|692|2|198| +2450829|694|2|482| +2450829|697|2|872| +2450829|698|2|217| +2450829|700|2|322| +2450829|703|2|968| +2450829|704|2|53| +2450829|706|2|521| +2450829|709|2|946| +2450829|710|2|155| +2450829|712|2|792| +2450829|715|2|287| +2450829|716|2|739| +2450829|718|2|678| +2450829|721|2|910| +2450829|722|2|548| +2450829|724|2|502| +2450829|727|2|824| +2450829|728|2|779| +2450829|730|2|546| +2450829|733|2|521| +2450829|734|2|395| +2450829|736|2|203| +2450829|739|2|897| +2450829|740|2|383| +2450829|742|2|755| +2450829|745|2|| +2450829|746|2|688| +2450829|748|2|837| +2450829|751|2|470| +2450829|752|2|493| +2450829|754|2|97| +2450829|757|2|494| +2450829|758|2|403| +2450829|760|2|527| +2450829|763|2|960| +2450829|764|2|279| +2450829|766|2|45| +2450829|769|2|652| +2450829|770|2|756| +2450829|772|2|858| +2450829|775|2|456| +2450829|776|2|137| +2450829|778|2|533| +2450829|781|2|60| +2450829|782|2|987| +2450829|784|2|111| +2450829|787|2|522| +2450829|788|2|880| +2450829|790|2|692| +2450829|793|2|529| +2450829|794|2|973| +2450829|796|2|733| +2450829|799|2|659| +2450829|800|2|108| +2450829|802|2|535| +2450829|805|2|777| +2450829|806|2|280| +2450829|808|2|278| +2450829|811|2|212| +2450829|812|2|984| +2450829|814|2|931| +2450829|817|2|970| +2450829|818|2|398| +2450829|820|2|711| +2450829|823|2|378| +2450829|824|2|398| +2450829|826|2|85| +2450829|829|2|513| +2450829|830|2|522| +2450829|832|2|579| +2450829|835|2|685| +2450829|836|2|| +2450829|838|2|236| +2450829|841|2|932| +2450829|842|2|346| +2450829|844|2|289| +2450829|847|2|986| +2450829|848|2|610| +2450829|850|2|745| +2450829|853|2|306| +2450829|854|2|978| +2450829|856|2|42| +2450829|859|2|| +2450829|860|2|18| +2450829|862|2|674| +2450829|865|2|47| +2450829|866|2|873| +2450829|868|2|151| +2450829|871|2|514| +2450829|872|2|610| +2450829|874|2|571| +2450829|877|2|647| +2450829|878|2|210| +2450829|880|2|323| +2450829|883|2|185| +2450829|884|2|49| +2450829|886|2|563| +2450829|889|2|379| +2450829|890|2|885| +2450829|892|2|584| +2450829|895|2|751| +2450829|896|2|302| +2450829|898|2|640| +2450829|901|2|456| +2450829|902|2|333| +2450829|904|2|626| +2450829|907|2|593| +2450829|908|2|407| +2450829|910|2|234| +2450829|913|2|683| +2450829|914|2|962| +2450829|916|2|122| +2450829|919|2|250| +2450829|920|2|389| +2450829|922|2|581| +2450829|925|2|186| +2450829|926|2|517| +2450829|928|2|112| +2450829|931|2|| +2450829|932|2|| +2450829|934|2|327| +2450829|937|2|265| +2450829|938|2|| +2450829|940|2|618| +2450829|943|2|806| +2450829|944|2|453| +2450829|946|2|83| +2450829|949|2|713| +2450829|950|2|777| +2450829|952|2|| +2450829|955|2|643| +2450829|956|2|636| +2450829|958|2|144| +2450829|961|2|32| +2450829|962|2|326| +2450829|964|2|732| +2450829|967|2|284| +2450829|968|2|569| +2450829|970|2|968| +2450829|973|2|363| +2450829|974|2|63| +2450829|976|2|343| +2450829|979|2|369| +2450829|980|2|924| +2450829|982|2|369| +2450829|985|2|221| +2450829|986|2|| +2450829|988|2|935| +2450829|991|2|| +2450829|992|2|379| +2450829|994|2|87| +2450829|997|2|228| +2450829|998|2|368| +2450829|1000|2|834| +2450829|1003|2|586| +2450829|1004|2|515| +2450829|1006|2|996| +2450829|1009|2|516| +2450829|1010|2|542| +2450829|1012|2|19| +2450829|1015|2|178| +2450829|1016|2|109| +2450829|1018|2|| +2450829|1021|2|908| +2450829|1022|2|| +2450829|1024|2|387| +2450829|1027|2|779| +2450829|1028|2|854| +2450829|1030|2|320| +2450829|1033|2|940| +2450829|1034|2|49| +2450829|1036|2|516| +2450829|1039|2|656| +2450829|1040|2|445| +2450829|1042|2|919| +2450829|1045|2|362| +2450829|1046|2|376| +2450829|1048|2|7| +2450829|1051|2|325| +2450829|1052|2|550| +2450829|1054|2|907| +2450829|1057|2|630| +2450829|1058|2|| +2450829|1060|2|642| +2450829|1063|2|387| +2450829|1064|2|448| +2450829|1066|2|529| +2450829|1069|2|855| +2450829|1070|2|827| +2450829|1072|2|108| +2450829|1075|2|133| +2450829|1076|2|480| +2450829|1078|2|748| +2450829|1081|2|414| +2450829|1082|2|197| +2450829|1084|2|196| +2450829|1087|2|166| +2450829|1088|2|362| +2450829|1090|2|700| +2450829|1093|2|106| +2450829|1094|2|293| +2450829|1096|2|566| +2450829|1099|2|549| +2450829|1100|2|45| +2450829|1102|2|877| +2450829|1105|2|981| +2450829|1106|2|148| +2450829|1108|2|683| +2450829|1111|2|736| +2450829|1112|2|815| +2450829|1114|2|64| +2450829|1117|2|| +2450829|1118|2|905| +2450829|1120|2|367| +2450829|1123|2|665| +2450829|1124|2|645| +2450829|1126|2|607| +2450829|1129|2|727| +2450829|1130|2|167| +2450829|1132|2|275| +2450829|1135|2|246| +2450829|1136|2|684| +2450829|1138|2|575| +2450829|1141|2|187| +2450829|1142|2|946| +2450829|1144|2|18| +2450829|1147|2|742| +2450829|1148|2|616| +2450829|1150|2|908| +2450829|1153|2|549| +2450829|1154|2|303| +2450829|1156|2|183| +2450829|1159|2|129| +2450829|1160|2|979| +2450829|1162|2|287| +2450829|1165|2|573| +2450829|1166|2|270| +2450829|1168|2|45| +2450829|1171|2|771| +2450829|1172|2|558| +2450829|1174|2|501| +2450829|1177|2|415| +2450829|1178|2|189| +2450829|1180|2|313| +2450829|1183|2|297| +2450829|1184|2|348| +2450829|1186|2|506| +2450829|1189|2|413| +2450829|1190|2|656| +2450829|1192|2|458| +2450829|1195|2|399| +2450829|1196|2|393| +2450829|1198|2|966| +2450829|1201|2|748| +2450829|1202|2|687| +2450829|1204|2|534| +2450829|1207|2|19| +2450829|1208|2|789| +2450829|1210|2|756| +2450829|1213|2|619| +2450829|1214|2|312| +2450829|1216|2|915| +2450829|1219|2|578| +2450829|1220|2|282| +2450829|1222|2|932| +2450829|1225|2|643| +2450829|1226|2|57| +2450829|1228|2|739| +2450829|1231|2|257| +2450829|1232|2|60| +2450829|1234|2|339| +2450829|1237|2|535| +2450829|1238|2|978| +2450829|1240|2|434| +2450829|1243|2|381| +2450829|1244|2|805| +2450829|1246|2|833| +2450829|1249|2|24| +2450829|1250|2|310| +2450829|1252|2|916| +2450829|1255|2|731| +2450829|1256|2|405| +2450829|1258|2|674| +2450829|1261|2|209| +2450829|1262|2|501| +2450829|1264|2|410| +2450829|1267|2|368| +2450829|1268|2|612| +2450829|1270|2|525| +2450829|1273|2|430| +2450829|1274|2|858| +2450829|1276|2|273| +2450829|1279|2|935| +2450829|1280|2|415| +2450829|1282|2|598| +2450829|1285|2|| +2450829|1286|2|923| +2450829|1288|2|765| +2450829|1291|2|494| +2450829|1292|2|381| +2450829|1294|2|392| +2450829|1297|2|| +2450829|1298|2|375| +2450829|1300|2|94| +2450829|1303|2|500| +2450829|1304|2|349| +2450829|1306|2|843| +2450829|1309|2|133| +2450829|1310|2|| +2450829|1312|2|556| +2450829|1315|2|587| +2450829|1316|2|99| +2450829|1318|2|668| +2450829|1321|2|90| +2450829|1322|2|139| +2450829|1324|2|689| +2450829|1327|2|154| +2450829|1328|2|501| +2450829|1330|2|481| +2450829|1333|2|442| +2450829|1334|2|114| +2450829|1336|2|206| +2450829|1339|2|609| +2450829|1340|2|423| +2450829|1342|2|719| +2450829|1345|2|932| +2450829|1346|2|957| +2450829|1348|2|| +2450829|1351|2|773| +2450829|1352|2|476| +2450829|1354|2|161| +2450829|1357|2|665| +2450829|1358|2|732| +2450829|1360|2|506| +2450829|1363|2|861| +2450829|1364|2|43| +2450829|1366|2|820| +2450829|1369|2|879| +2450829|1370|2|126| +2450829|1372|2|124| +2450829|1375|2|753| +2450829|1376|2|241| +2450829|1378|2|700| +2450829|1381|2|873| +2450829|1382|2|472| +2450829|1384|2|96| +2450829|1387|2|981| +2450829|1388|2|714| +2450829|1390|2|618| +2450829|1393|2|| +2450829|1394|2|953| +2450829|1396|2|937| +2450829|1399|2|467| +2450829|1400|2|951| +2450829|1402|2|483| +2450829|1405|2|164| +2450829|1406|2|908| +2450829|1408|2|85| +2450829|1411|2|743| +2450829|1412|2|294| +2450829|1414|2|932| +2450829|1417|2|156| +2450829|1418|2|494| +2450829|1420|2|479| +2450829|1423|2|235| +2450829|1424|2|643| +2450829|1426|2|274| +2450829|1429|2|876| +2450829|1430|2|631| +2450829|1432|2|| +2450829|1435|2|| +2450829|1436|2|374| +2450829|1438|2|724| +2450829|1441|2|783| +2450829|1442|2|527| +2450829|1444|2|611| +2450829|1447|2|700| +2450829|1448|2|11| +2450829|1450|2|50| +2450829|1453|2|573| +2450829|1454|2|968| +2450829|1456|2|821| +2450829|1459|2|645| +2450829|1460|2|298| +2450829|1462|2|377| +2450829|1465|2|620| +2450829|1466|2|908| +2450829|1468|2|47| +2450829|1471|2|55| +2450829|1472|2|330| +2450829|1474|2|246| +2450829|1477|2|624| +2450829|1478|2|807| +2450829|1480|2|356| +2450829|1483|2|764| +2450829|1484|2|207| +2450829|1486|2|391| +2450829|1489|2|483| +2450829|1490|2|854| +2450829|1492|2|787| +2450829|1495|2|466| +2450829|1496|2|672| +2450829|1498|2|675| +2450829|1501|2|912| +2450829|1502|2|626| +2450829|1504|2|618| +2450829|1507|2|592| +2450829|1508|2|935| +2450829|1510|2|524| +2450829|1513|2|797| +2450829|1514|2|317| +2450829|1516|2|989| +2450829|1519|2|934| +2450829|1520|2|713| +2450829|1522|2|896| +2450829|1525|2|| +2450829|1526|2|733| +2450829|1528|2|| +2450829|1531|2|679| +2450829|1532|2|177| +2450829|1534|2|| +2450829|1537|2|917| +2450829|1538|2|80| +2450829|1540|2|833| +2450829|1543|2|257| +2450829|1544|2|208| +2450829|1546|2|| +2450829|1549|2|461| +2450829|1550|2|358| +2450829|1552|2|402| +2450829|1555|2|422| +2450829|1556|2|741| +2450829|1558|2|503| +2450829|1561|2|979| +2450829|1562|2|160| +2450829|1564|2|327| +2450829|1567|2|677| +2450829|1568|2|864| +2450829|1570|2|| +2450829|1573|2|284| +2450829|1574|2|463| +2450829|1576|2|960| +2450829|1579|2|90| +2450829|1580|2|642| +2450829|1582|2|811| +2450829|1585|2|559| +2450829|1586|2|113| +2450829|1588|2|567| +2450829|1591|2|234| +2450829|1592|2|301| +2450829|1594|2|488| +2450829|1597|2|990| +2450829|1598|2|405| +2450829|1600|2|34| +2450829|1603|2|844| +2450829|1604|2|445| +2450829|1606|2|457| +2450829|1609|2|410| +2450829|1610|2|519| +2450829|1612|2|597| +2450829|1615|2|702| +2450829|1616|2|904| +2450829|1618|2|471| +2450829|1621|2|| +2450829|1622|2|112| +2450829|1624|2|879| +2450829|1627|2|337| +2450829|1628|2|476| +2450829|1630|2|133| +2450829|1633|2|194| +2450829|1634|2|213| +2450829|1636|2|848| +2450829|1639|2|865| +2450829|1640|2|141| +2450829|1642|2|287| +2450829|1645|2|| +2450829|1646|2|596| +2450829|1648|2|853| +2450829|1651|2|697| +2450829|1652|2|498| +2450829|1654|2|262| +2450829|1657|2|602| +2450829|1658|2|197| +2450829|1660|2|313| +2450829|1663|2|453| +2450829|1664|2|860| +2450829|1666|2|496| +2450829|1669|2|345| +2450829|1670|2|7| +2450829|1672|2|285| +2450829|1675|2|255| +2450829|1676|2|56| +2450829|1678|2|488| +2450829|1681|2|528| +2450829|1682|2|931| +2450829|1684|2|771| +2450829|1687|2|421| +2450829|1688|2|118| +2450829|1690|2|157| +2450829|1693|2|473| +2450829|1694|2|253| +2450829|1696|2|264| +2450829|1699|2|961| +2450829|1700|2|544| +2450829|1702|2|912| +2450829|1705|2|494| +2450829|1706|2|971| +2450829|1708|2|| +2450829|1711|2|81| +2450829|1712|2|640| +2450829|1714|2|532| +2450829|1717|2|| +2450829|1718|2|494| +2450829|1720|2|200| +2450829|1723|2|180| +2450829|1724|2|167| +2450829|1726|2|82| +2450829|1729|2|665| +2450829|1730|2|605| +2450829|1732|2|| +2450829|1735|2|54| +2450829|1736|2|454| +2450829|1738|2|163| +2450829|1741|2|980| +2450829|1742|2|289| +2450829|1744|2|427| +2450829|1747|2|794| +2450829|1748|2|97| +2450829|1750|2|265| +2450829|1753|2|497| +2450829|1754|2|535| +2450829|1756|2|561| +2450829|1759|2|| +2450829|1760|2|660| +2450829|1762|2|44| +2450829|1765|2|396| +2450829|1766|2|195| +2450829|1768|2|403| +2450829|1771|2|827| +2450829|1772|2|889| +2450829|1774|2|633| +2450829|1777|2|861| +2450829|1778|2|183| +2450829|1780|2|225| +2450829|1783|2|921| +2450829|1784|2|272| +2450829|1786|2|790| +2450829|1789|2|827| +2450829|1790|2|483| +2450829|1792|2|461| +2450829|1795|2|737| +2450829|1796|2|112| +2450829|1798|2|729| +2450829|1801|2|821| +2450829|1802|2|205| +2450829|1804|2|| +2450829|1807|2|737| +2450829|1808|2|798| +2450829|1810|2|970| +2450829|1813|2|743| +2450829|1814|2|51| +2450829|1816|2|635| +2450829|1819|2|| +2450829|1820|2|796| +2450829|1822|2|642| +2450829|1825|2|584| +2450829|1826|2|14| +2450829|1828|2|819| +2450829|1831|2|412| +2450829|1832|2|709| +2450829|1834|2|343| +2450829|1837|2|78| +2450829|1838|2|119| +2450829|1840|2|427| +2450829|1843|2|960| +2450829|1844|2|324| +2450829|1846|2|983| +2450829|1849|2|| +2450829|1850|2|475| +2450829|1852|2|314| +2450829|1855|2|80| +2450829|1856|2|211| +2450829|1858|2|| +2450829|1861|2|60| +2450829|1862|2|790| +2450829|1864|2|610| +2450829|1867|2|911| +2450829|1868|2|54| +2450829|1870|2|261| +2450829|1873|2|756| +2450829|1874|2|990| +2450829|1876|2|894| +2450829|1879|2|90| +2450829|1880|2|343| +2450829|1882|2|524| +2450829|1885|2|705| +2450829|1886|2|856| +2450829|1888|2|796| +2450829|1891|2|12| +2450829|1892|2|906| +2450829|1894|2|248| +2450829|1897|2|434| +2450829|1898|2|388| +2450829|1900|2|219| +2450829|1903|2|389| +2450829|1904|2|172| +2450829|1906|2|726| +2450829|1909|2|875| +2450829|1910|2|| +2450829|1912|2|282| +2450829|1915|2|558| +2450829|1916|2|323| +2450829|1918|2|926| +2450829|1921|2|619| +2450829|1922|2|365| +2450829|1924|2|639| +2450829|1927|2|121| +2450829|1928|2|876| +2450829|1930|2|87| +2450829|1933|2|646| +2450829|1934|2|975| +2450829|1936|2|551| +2450829|1939|2|829| +2450829|1940|2|233| +2450829|1942|2|648| +2450829|1945|2|558| +2450829|1946|2|942| +2450829|1948|2|593| +2450829|1951|2|728| +2450829|1952|2|877| +2450829|1954|2|241| +2450829|1957|2|235| +2450829|1958|2|418| +2450829|1960|2|411| +2450829|1963|2|| +2450829|1964|2|| +2450829|1966|2|| +2450829|1969|2|| +2450829|1970|2|870| +2450829|1972|2|257| +2450829|1975|2|613| +2450829|1976|2|92| +2450829|1978|2|106| +2450829|1981|2|546| +2450829|1982|2|349| +2450829|1984|2|73| +2450829|1987|2|83| +2450829|1988|2|662| +2450829|1990|2|73| +2450829|1993|2|948| +2450829|1994|2|50| +2450829|1996|2|729| +2450829|1999|2|350| +2450829|2000|2|530| +2450829|2002|2|316| +2450829|2005|2|962| +2450829|2006|2|828| +2450829|2008|2|928| +2450829|2011|2|806| +2450829|2012|2|799| +2450829|2014|2|857| +2450829|2017|2|315| +2450829|2018|2|185| +2450829|2020|2|875| +2450829|2023|2|848| +2450829|2024|2|233| +2450829|2026|2|928| +2450829|2029|2|496| +2450829|2030|2|159| +2450829|2032|2|776| +2450829|2035|2|79| +2450829|2036|2|373| +2450829|2038|2|206| +2450829|2041|2|852| +2450829|2042|2|571| +2450829|2044|2|685| +2450829|2047|2|657| +2450829|2048|2|80| +2450829|2050|2|| +2450829|2053|2|663| +2450829|2054|2|242| +2450829|2056|2|96| +2450829|2059|2|631| +2450829|2060|2|545| +2450829|2062|2|737| +2450829|2065|2|319| +2450829|2066|2|851| +2450829|2068|2|552| +2450829|2071|2|| +2450829|2072|2|318| +2450829|2074|2|718| +2450829|2077|2|500| +2450829|2078|2|268| +2450829|2080|2|827| +2450829|2083|2|304| +2450829|2084|2|309| +2450829|2086|2|190| +2450829|2089|2|26| +2450829|2090|2|516| +2450829|2092|2|429| +2450829|2095|2|932| +2450829|2096|2|965| +2450829|2098|2|850| +2450829|2101|2|745| +2450829|2102|2|338| +2450829|2104|2|632| +2450829|2107|2|50| +2450829|2108|2|407| +2450829|2110|2|185| +2450829|2113|2|156| +2450829|2114|2|55| +2450829|2116|2|864| +2450829|2119|2|238| +2450829|2120|2|462| +2450829|2122|2|830| +2450829|2125|2|187| +2450829|2126|2|521| +2450829|2128|2|257| +2450829|2131|2|489| +2450829|2132|2|337| +2450829|2134|2|702| +2450829|2137|2|| +2450829|2138|2|958| +2450829|2140|2|532| +2450829|2143|2|437| +2450829|2144|2|864| +2450829|2146|2|292| +2450829|2149|2|77| +2450829|2150|2|99| +2450829|2152|2|859| +2450829|2155|2|690| +2450829|2156|2|918| +2450829|2158|2|806| +2450829|2161|2|150| +2450829|2162|2|561| +2450829|2164|2|627| +2450829|2167|2|352| +2450829|2168|2|711| +2450829|2170|2|606| +2450829|2173|2|385| +2450829|2174|2|| +2450829|2176|2|244| +2450829|2179|2|513| +2450829|2180|2|326| +2450829|2182|2|458| +2450829|2185|2|566| +2450829|2186|2|758| +2450829|2188|2|755| +2450829|2191|2|238| +2450829|2192|2|748| +2450829|2194|2|544| +2450829|2197|2|791| +2450829|2198|2|124| +2450829|2200|2|881| +2450829|2203|2|721| +2450829|2204|2|826| +2450829|2206|2|| +2450829|2209|2|490| +2450829|2210|2|682| +2450829|2212|2|284| +2450829|2215|2|42| +2450829|2216|2|32| +2450829|2218|2|305| +2450829|2221|2|380| +2450829|2222|2|371| +2450829|2224|2|657| +2450829|2227|2|| +2450829|2228|2|268| +2450829|2230|2|768| +2450829|2233|2|148| +2450829|2234|2|166| +2450829|2236|2|35| +2450829|2239|2|467| +2450829|2240|2|185| +2450829|2242|2|19| +2450829|2245|2|| +2450829|2246|2|374| +2450829|2248|2|859| +2450829|2251|2|981| +2450829|2252|2|489| +2450829|2254|2|728| +2450829|2257|2|286| +2450829|2258|2|418| +2450829|2260|2|321| +2450829|2263|2|385| +2450829|2264|2|922| +2450829|2266|2|174| +2450829|2269|2|562| +2450829|2270|2|704| +2450829|2272|2|316| +2450829|2275|2|| +2450829|2276|2|1| +2450829|2278|2|288| +2450829|2281|2|509| +2450829|2282|2|322| +2450829|2284|2|525| +2450829|2287|2|485| +2450829|2288|2|762| +2450829|2290|2|692| +2450829|2293|2|265| +2450829|2294|2|632| +2450829|2296|2|51| +2450829|2299|2|155| +2450829|2300|2|| +2450829|2302|2|28| +2450829|2305|2|264| +2450829|2306|2|73| +2450829|2308|2|239| +2450829|2311|2|777| +2450829|2312|2|| +2450829|2314|2|807| +2450829|2317|2|748| +2450829|2318|2|| +2450829|2320|2|891| +2450829|2323|2|855| +2450829|2324|2|982| +2450829|2326|2|997| +2450829|2329|2|875| +2450829|2330|2|12| +2450829|2332|2|758| +2450829|2335|2|207| +2450829|2336|2|| +2450829|2338|2|924| +2450829|2341|2|876| +2450829|2342|2|456| +2450829|2344|2|770| +2450829|2347|2|914| +2450829|2348|2|86| +2450829|2350|2|312| +2450829|2353|2|882| +2450829|2354|2|240| +2450829|2356|2|83| +2450829|2359|2|136| +2450829|2360|2|835| +2450829|2362|2|740| +2450829|2365|2|29| +2450829|2366|2|254| +2450829|2368|2|136| +2450829|2371|2|330| +2450829|2372|2|624| +2450829|2374|2|800| +2450829|2377|2|283| +2450829|2378|2|54| +2450829|2380|2|294| +2450829|2383|2|123| +2450829|2384|2|714| +2450829|2386|2|30| +2450829|2389|2|728| +2450829|2390|2|670| +2450829|2392|2|426| +2450829|2395|2|639| +2450829|2396|2|980| +2450829|2398|2|446| +2450829|2401|2|| +2450829|2402|2|621| +2450829|2404|2|395| +2450829|2407|2|652| +2450829|2408|2|816| +2450829|2410|2|715| +2450829|2413|2|611| +2450829|2414|2|203| +2450829|2416|2|219| +2450829|2419|2|332| +2450829|2420|2|115| +2450829|2422|2|670| +2450829|2425|2|| +2450829|2426|2|529| +2450829|2428|2|570| +2450829|2431|2|210| +2450829|2432|2|370| +2450829|2434|2|205| +2450829|2437|2|947| +2450829|2438|2|313| +2450829|2440|2|961| +2450829|2443|2|551| +2450829|2444|2|324| +2450829|2446|2|154| +2450829|2449|2|921| +2450829|2450|2|770| +2450829|2452|2|572| +2450829|2455|2|218| +2450829|2456|2|883| +2450829|2458|2|555| +2450829|2461|2|152| +2450829|2462|2|| +2450829|2464|2|136| +2450829|2467|2|659| +2450829|2468|2|765| +2450829|2470|2|747| +2450829|2473|2|4| +2450829|2474|2|408| +2450829|2476|2|882| +2450829|2479|2|| +2450829|2480|2|909| +2450829|2482|2|559| +2450829|2485|2|436| +2450829|2486|2|286| +2450829|2488|2|286| +2450829|2491|2|6| +2450829|2492|2|269| +2450829|2494|2|790| +2450829|2497|2|94| +2450829|2498|2|24| +2450829|2500|2|789| +2450829|2503|2|939| +2450829|2504|2|57| +2450829|2506|2|714| +2450829|2509|2|259| +2450829|2510|2|924| +2450829|2512|2|469| +2450829|2515|2|495| +2450829|2516|2|754| +2450829|2518|2|901| +2450829|2521|2|295| +2450829|2522|2|930| +2450829|2524|2|177| +2450829|2527|2|327| +2450829|2528|2|722| +2450829|2530|2|924| +2450829|2533|2|572| +2450829|2534|2|806| +2450829|2536|2|699| +2450829|2539|2|| +2450829|2540|2|682| +2450829|2542|2|559| +2450829|2545|2|961| +2450829|2546|2|31| +2450829|2548|2|249| +2450829|2551|2|798| +2450829|2552|2|258| +2450829|2554|2|824| +2450829|2557|2|202| +2450829|2558|2|144| +2450829|2560|2|178| +2450829|2563|2|262| +2450829|2564|2|101| +2450829|2566|2|953| +2450829|2569|2|292| +2450829|2570|2|251| +2450829|2572|2|542| +2450829|2575|2|704| +2450829|2576|2|956| +2450829|2578|2|41| +2450829|2581|2|761| +2450829|2582|2|840| +2450829|2584|2|481| +2450829|2587|2|403| +2450829|2588|2|602| +2450829|2590|2|138| +2450829|2593|2|959| +2450829|2594|2|910| +2450829|2596|2|333| +2450829|2599|2|707| +2450829|2600|2|144| +2450829|2602|2|126| +2450829|2605|2|304| +2450829|2606|2|| +2450829|2608|2|143| +2450829|2611|2|754| +2450829|2612|2|994| +2450829|2614|2|21| +2450829|2617|2|536| +2450829|2618|2|131| +2450829|2620|2|547| +2450829|2623|2|217| +2450829|2624|2|57| +2450829|2626|2|158| +2450829|2629|2|387| +2450829|2630|2|524| +2450829|2632|2|| +2450829|2635|2|833| +2450829|2636|2|327| +2450829|2638|2|487| +2450829|2641|2|799| +2450829|2642|2|441| +2450829|2644|2|860| +2450829|2647|2|11| +2450829|2648|2|490| +2450829|2650|2|851| +2450829|2653|2|907| +2450829|2654|2|576| +2450829|2656|2|732| +2450829|2659|2|98| +2450829|2660|2|567| +2450829|2662|2|257| +2450829|2665|2|943| +2450829|2666|2|3| +2450829|2668|2|566| +2450829|2671|2|354| +2450829|2672|2|63| +2450829|2674|2|512| +2450829|2677|2|| +2450829|2678|2|762| +2450829|2680|2|530| +2450829|2683|2|| +2450829|2684|2|684| +2450829|2686|2|385| +2450829|2689|2|829| +2450829|2690|2|941| +2450829|2692|2|785| +2450829|2695|2|416| +2450829|2696|2|587| +2450829|2698|2|674| +2450829|2701|2|737| +2450829|2702|2|430| +2450829|2704|2|172| +2450829|2707|2|| +2450829|2708|2|878| +2450829|2710|2|158| +2450829|2713|2|38| +2450829|2714|2|188| +2450829|2716|2|162| +2450829|2719|2|417| +2450829|2720|2|784| +2450829|2722|2|823| +2450829|2725|2|293| +2450829|2726|2|758| +2450829|2728|2|141| +2450829|2731|2|607| +2450829|2732|2|909| +2450829|2734|2|319| +2450829|2737|2|722| +2450829|2738|2|995| +2450829|2740|2|351| +2450829|2743|2|| +2450829|2744|2|365| +2450829|2746|2|0| +2450829|2749|2|499| +2450829|2750|2|983| +2450829|2752|2|485| +2450829|2755|2|306| +2450829|2756|2|92| +2450829|2758|2|131| +2450829|2761|2|105| +2450829|2762|2|264| +2450829|2764|2|996| +2450829|2767|2|295| +2450829|2768|2|492| +2450829|2770|2|787| +2450829|2773|2|990| +2450829|2774|2|929| +2450829|2776|2|| +2450829|2779|2|766| +2450829|2780|2|256| +2450829|2782|2|524| +2450829|2785|2|430| +2450829|2786|2|4| +2450829|2788|2|658| +2450829|2791|2|42| +2450829|2792|2|377| +2450829|2794|2|374| +2450829|2797|2|831| +2450829|2798|2|| +2450829|2800|2|79| +2450829|2803|2|358| +2450829|2804|2|986| +2450829|2806|2|276| +2450829|2809|2|446| +2450829|2810|2|919| +2450829|2812|2|159| +2450829|2815|2|132| +2450829|2816|2|599| +2450829|2818|2|346| +2450829|2821|2|79| +2450829|2822|2|905| +2450829|2824|2|255| +2450829|2827|2|815| +2450829|2828|2|62| +2450829|2830|2|566| +2450829|2833|2|598| +2450829|2834|2|539| +2450829|2836|2|679| +2450829|2839|2|8| +2450829|2840|2|| +2450829|2842|2|540| +2450829|2845|2|620| +2450829|2846|2|526| +2450829|2848|2|920| +2450829|2851|2|849| +2450829|2852|2|468| +2450829|2854|2|438| +2450829|2857|2|405| +2450829|2858|2|216| +2450829|2860|2|895| +2450829|2863|2|447| +2450829|2864|2|658| +2450829|2866|2|255| +2450829|2869|2|541| +2450829|2870|2|349| +2450829|2872|2|500| +2450829|2875|2|379| +2450829|2876|2|823| +2450829|2878|2|753| +2450829|2881|2|901| +2450829|2882|2|983| +2450829|2884|2|573| +2450829|2887|2|26| +2450829|2888|2|| +2450829|2890|2|830| +2450829|2893|2|248| +2450829|2894|2|451| +2450829|2896|2|39| +2450829|2899|2|484| +2450829|2900|2|970| +2450829|2902|2|522| +2450829|2905|2|861| +2450829|2906|2|746| +2450829|2908|2|146| +2450829|2911|2|80| +2450829|2912|2|454| +2450829|2914|2|978| +2450829|2917|2|205| +2450829|2918|2|956| +2450829|2920|2|564| +2450829|2923|2|| +2450829|2924|2|220| +2450829|2926|2|884| +2450829|2929|2|255| +2450829|2930|2|437| +2450829|2932|2|585| +2450829|2935|2|391| +2450829|2936|2|913| +2450829|2938|2|907| +2450829|2941|2|666| +2450829|2942|2|355| +2450829|2944|2|518| +2450829|2947|2|378| +2450829|2948|2|433| +2450829|2950|2|976| +2450829|2953|2|998| +2450829|2954|2|55| +2450829|2956|2|194| +2450829|2959|2|759| +2450829|2960|2|910| +2450829|2962|2|610| +2450829|2965|2|| +2450829|2966|2|194| +2450829|2968|2|851| +2450829|2971|2|180| +2450829|2972|2|262| +2450829|2974|2|660| +2450829|2977|2|501| +2450829|2978|2|931| +2450829|2980|2|873| +2450829|2983|2|647| +2450829|2984|2|69| +2450829|2986|2|86| +2450829|2989|2|707| +2450829|2990|2|387| +2450829|2992|2|520| +2450829|2995|2|790| +2450829|2996|2|593| +2450829|2998|2|69| +2450829|3001|2|28| +2450829|3002|2|821| +2450829|3004|2|126| +2450829|3007|2|352| +2450829|3008|2|555| +2450829|3010|2|628| +2450829|3013|2|826| +2450829|3014|2|682| +2450829|3016|2|683| +2450829|3019|2|192| +2450829|3020|2|97| +2450829|3022|2|822| +2450829|3025|2|603| +2450829|3026|2|167| +2450829|3028|2|629| +2450829|3031|2|| +2450829|3032|2|436| +2450829|3034|2|347| +2450829|3037|2|206| +2450829|3038|2|819| +2450829|3040|2|500| +2450829|3043|2|718| +2450829|3044|2|73| +2450829|3046|2|688| +2450829|3049|2|302| +2450829|3050|2|245| +2450829|3052|2|523| +2450829|3055|2|680| +2450829|3056|2|593| +2450829|3058|2|795| +2450829|3061|2|245| +2450829|3062|2|652| +2450829|3064|2|986| +2450829|3067|2|814| +2450829|3068|2|59| +2450829|3070|2|527| +2450829|3073|2|911| +2450829|3074|2|823| +2450829|3076|2|313| +2450829|3079|2|729| +2450829|3080|2|3| +2450829|3082|2|395| +2450829|3085|2|| +2450829|3086|2|330| +2450829|3088|2|725| +2450829|3091|2|705| +2450829|3092|2|425| +2450829|3094|2|907| +2450829|3097|2|713| +2450829|3098|2|554| +2450829|3100|2|639| +2450829|3103|2|702| +2450829|3104|2|544| +2450829|3106|2|| +2450829|3109|2|981| +2450829|3110|2|213| +2450829|3112|2|| +2450829|3115|2|51| +2450829|3116|2|299| +2450829|3118|2|853| +2450829|3121|2|721| +2450829|3122|2|423| +2450829|3124|2|48| +2450829|3127|2|234| +2450829|3128|2|695| +2450829|3130|2|750| +2450829|3133|2|652| +2450829|3134|2|743| +2450829|3136|2|437| +2450829|3139|2|460| +2450829|3140|2|25| +2450829|3142|2|146| +2450829|3145|2|503| +2450829|3146|2|483| +2450829|3148|2|604| +2450829|3151|2|477| +2450829|3152|2|158| +2450829|3154|2|149| +2450829|3157|2|408| +2450829|3158|2|386| +2450829|3160|2|917| +2450829|3163|2|597| +2450829|3164|2|769| +2450829|3166|2|219| +2450829|3169|2|308| +2450829|3170|2|407| +2450829|3172|2|39| +2450829|3175|2|107| +2450829|3176|2|206| +2450829|3178|2|209| +2450829|3181|2|826| +2450829|3182|2|172| +2450829|3184|2|100| +2450829|3187|2|486| +2450829|3188|2|540| +2450829|3190|2|30| +2450829|3193|2|257| +2450829|3194|2|508| +2450829|3196|2|870| +2450829|3199|2|61| +2450829|3200|2|970| +2450829|3202|2|290| +2450829|3205|2|345| +2450829|3206|2|556| +2450829|3208|2|822| +2450829|3211|2|846| +2450829|3212|2|925| +2450829|3214|2|478| +2450829|3217|2|619| +2450829|3218|2|597| +2450829|3220|2|1000| +2450829|3223|2|212| +2450829|3224|2|550| +2450829|3226|2|135| +2450829|3229|2|690| +2450829|3230|2|212| +2450829|3232|2|918| +2450829|3235|2|239| +2450829|3236|2|649| +2450829|3238|2|367| +2450829|3241|2|213| +2450829|3242|2|339| +2450829|3244|2|587| +2450829|3247|2|342| +2450829|3248|2|883| +2450829|3250|2|769| +2450829|3253|2|715| +2450829|3254|2|998| +2450829|3256|2|175| +2450829|3259|2|649| +2450829|3260|2|126| +2450829|3262|2|892| +2450829|3265|2|717| +2450829|3266|2|308| +2450829|3268|2|977| +2450829|3271|2|14| +2450829|3272|2|| +2450829|3274|2|360| +2450829|3277|2|747| +2450829|3278|2|| +2450829|3280|2|139| +2450829|3283|2|280| +2450829|3284|2|718| +2450829|3286|2|993| +2450829|3289|2|993| +2450829|3290|2|641| +2450829|3292|2|764| +2450829|3295|2|807| +2450829|3296|2|862| +2450829|3298|2|918| +2450829|3301|2|278| +2450829|3302|2|| +2450829|3304|2|584| +2450829|3307|2|154| +2450829|3308|2|156| +2450829|3310|2|656| +2450829|3313|2|797| +2450829|3314|2|| +2450829|3316|2|135| +2450829|3319|2|497| +2450829|3320|2|353| +2450829|3322|2|738| +2450829|3325|2|285| +2450829|3326|2|628| +2450829|3328|2|860| +2450829|3331|2|365| +2450829|3332|2|647| +2450829|3334|2|803| +2450829|3337|2|50| +2450829|3338|2|157| +2450829|3340|2|744| +2450829|3343|2|868| +2450829|3344|2|217| +2450829|3346|2|474| +2450829|3349|2|242| +2450829|3350|2|444| +2450829|3352|2|345| +2450829|3355|2|581| +2450829|3356|2|513| +2450829|3358|2|715| +2450829|3361|2|204| +2450829|3362|2|917| +2450829|3364|2|| +2450829|3367|2|708| +2450829|3368|2|289| +2450829|3370|2|31| +2450829|3373|2|874| +2450829|3374|2|417| +2450829|3376|2|738| +2450829|3379|2|236| +2450829|3380|2|834| +2450829|3382|2|812| +2450829|3385|2|386| +2450829|3386|2|168| +2450829|3388|2|| +2450829|3391|2|618| +2450829|3392|2|| +2450829|3394|2|424| +2450829|3397|2|557| +2450829|3398|2|263| +2450829|3400|2|199| +2450829|3403|2|347| +2450829|3404|2|910| +2450829|3406|2|276| +2450829|3409|2|835| +2450829|3410|2|740| +2450829|3412|2|961| +2450829|3415|2|885| +2450829|3416|2|275| +2450829|3418|2|161| +2450829|3421|2|785| +2450829|3422|2|955| +2450829|3424|2|263| +2450829|3427|2|280| +2450829|3428|2|962| +2450829|3430|2|650| +2450829|3433|2|450| +2450829|3434|2|9| +2450829|3436|2|730| +2450829|3439|2|275| +2450829|3440|2|596| +2450829|3442|2|759| +2450829|3445|2|200| +2450829|3446|2|43| +2450829|3448|2|572| +2450829|3451|2|125| +2450829|3452|2|365| +2450829|3454|2|995| +2450829|3457|2|128| +2450829|3458|2|817| +2450829|3460|2|830| +2450829|3463|2|409| +2450829|3464|2|796| +2450829|3466|2|771| +2450829|3469|2|286| +2450829|3470|2|498| +2450829|3472|2|806| +2450829|3475|2|553| +2450829|3476|2|0| +2450829|3478|2|407| +2450829|3481|2|58| +2450829|3482|2|105| +2450829|3484|2|464| +2450829|3487|2|| +2450829|3488|2|368| +2450829|3490|2|| +2450829|3493|2|475| +2450829|3494|2|163| +2450829|3496|2|56| +2450829|3499|2|654| +2450829|3500|2|952| +2450829|3502|2|722| +2450829|3505|2|518| +2450829|3506|2|130| +2450829|3508|2|932| +2450829|3511|2|267| +2450829|3512|2|64| +2450829|3514|2|502| +2450829|3517|2|702| +2450829|3518|2|| +2450829|3520|2|791| +2450829|3523|2|83| +2450829|3524|2|920| +2450829|3526|2|113| +2450829|3529|2|594| +2450829|3530|2|453| +2450829|3532|2|22| +2450829|3535|2|634| +2450829|3536|2|920| +2450829|3538|2|114| +2450829|3541|2|912| +2450829|3542|2|456| +2450829|3544|2|776| +2450829|3547|2|128| +2450829|3548|2|710| +2450829|3550|2|813| +2450829|3553|2|904| +2450829|3554|2|274| +2450829|3556|2|23| +2450829|3559|2|600| +2450829|3560|2|649| +2450829|3562|2|375| +2450829|3565|2|| +2450829|3566|2|596| +2450829|3568|2|708| +2450829|3571|2|845| +2450829|3572|2|828| +2450829|3574|2|284| +2450829|3577|2|430| +2450829|3578|2|76| +2450829|3580|2|191| +2450829|3583|2|418| +2450829|3584|2|944| +2450829|3586|2|569| +2450829|3589|2|350| +2450829|3590|2|227| +2450829|3592|2|616| +2450829|3595|2|591| +2450829|3596|2|208| +2450829|3598|2|182| +2450829|3601|2|317| +2450829|3602|2|989| +2450829|3604|2|922| +2450829|3607|2|725| +2450829|3608|2|150| +2450829|3610|2|109| +2450829|3613|2|81| +2450829|3614|2|628| +2450829|3616|2|845| +2450829|3619|2|200| +2450829|3620|2|755| +2450829|3622|2|897| +2450829|3625|2|392| +2450829|3626|2|| +2450829|3628|2|967| +2450829|3631|2|256| +2450829|3632|2|687| +2450829|3634|2|468| +2450829|3637|2|168| +2450829|3638|2|730| +2450829|3640|2|349| +2450829|3643|2|412| +2450829|3644|2|217| +2450829|3646|2|22| +2450829|3649|2|609| +2450829|3650|2|892| +2450829|3652|2|452| +2450829|3655|2|391| +2450829|3656|2|278| +2450829|3658|2|543| +2450829|3661|2|711| +2450829|3662|2|384| +2450829|3664|2|738| +2450829|3667|2|607| +2450829|3668|2|528| +2450829|3670|2|898| +2450829|3673|2|612| +2450829|3674|2|599| +2450829|3676|2|167| +2450829|3679|2|467| +2450829|3680|2|376| +2450829|3682|2|137| +2450829|3685|2|379| +2450829|3686|2|332| +2450829|3688|2|737| +2450829|3691|2|802| +2450829|3692|2|473| +2450829|3694|2|132| +2450829|3697|2|| +2450829|3698|2|367| +2450829|3700|2|510| +2450829|3703|2|237| +2450829|3704|2|371| +2450829|3706|2|500| +2450829|3709|2|568| +2450829|3710|2|833| +2450829|3712|2|899| +2450829|3715|2|821| +2450829|3716|2|520| +2450829|3718|2|910| +2450829|3721|2|754| +2450829|3722|2|98| +2450829|3724|2|470| +2450829|3727|2|940| +2450829|3728|2|284| +2450829|3730|2|321| +2450829|3733|2|727| +2450829|3734|2|890| +2450829|3736|2|749| +2450829|3739|2|705| +2450829|3740|2|238| +2450829|3742|2|549| +2450829|3745|2|263| +2450829|3746|2|407| +2450829|3748|2|154| +2450829|3751|2|979| +2450829|3752|2|426| +2450829|3754|2|619| +2450829|3757|2|584| +2450829|3758|2|122| +2450829|3760|2|175| +2450829|3763|2|543| +2450829|3764|2|152| +2450829|3766|2|558| +2450829|3769|2|562| +2450829|3770|2|102| +2450829|3772|2|125| +2450829|3775|2|854| +2450829|3776|2|975| +2450829|3778|2|337| +2450829|3781|2|782| +2450829|3782|2|419| +2450829|3784|2|559| +2450829|3787|2|525| +2450829|3788|2|638| +2450829|3790|2|342| +2450829|3793|2|593| +2450829|3794|2|354| +2450829|3796|2|499| +2450829|3799|2|243| +2450829|3800|2|655| +2450829|3802|2|672| +2450829|3805|2|748| +2450829|3806|2|168| +2450829|3808|2|981| +2450829|3811|2|131| +2450829|3812|2|689| +2450829|3814|2|| +2450829|3817|2|571| +2450829|3818|2|414| +2450829|3820|2|469| +2450829|3823|2|75| +2450829|3824|2|736| +2450829|3826|2|949| +2450829|3829|2|925| +2450829|3830|2|977| +2450829|3832|2|949| +2450829|3835|2|642| +2450829|3836|2|179| +2450829|3838|2|897| +2450829|3841|2|763| +2450829|3842|2|112| +2450829|3844|2|740| +2450829|3847|2|269| +2450829|3848|2|| +2450829|3850|2|453| +2450829|3853|2|431| +2450829|3854|2|369| +2450829|3856|2|494| +2450829|3859|2|711| +2450829|3860|2|470| +2450829|3862|2|180| +2450829|3865|2|9| +2450829|3866|2|183| +2450829|3868|2|569| +2450829|3871|2|998| +2450829|3872|2|611| +2450829|3874|2|868| +2450829|3877|2|598| +2450829|3878|2|80| +2450829|3880|2|438| +2450829|3883|2|| +2450829|3884|2|825| +2450829|3886|2|997| +2450829|3889|2|616| +2450829|3890|2|784| +2450829|3892|2|644| +2450829|3895|2|396| +2450829|3896|2|737| +2450829|3898|2|564| +2450829|3901|2|514| +2450829|3902|2|382| +2450829|3904|2|780| +2450829|3907|2|460| +2450829|3908|2|| +2450829|3910|2|819| +2450829|3913|2|214| +2450829|3914|2|296| +2450829|3916|2|674| +2450829|3919|2|812| +2450829|3920|2|670| +2450829|3922|2|| +2450829|3925|2|| +2450829|3926|2|757| +2450829|3928|2|140| +2450829|3931|2|148| +2450829|3932|2|992| +2450829|3934|2|582| +2450829|3937|2|211| +2450829|3938|2|580| +2450829|3940|2|925| +2450829|3943|2|736| +2450829|3944|2|| +2450829|3946|2|308| +2450829|3949|2|110| +2450829|3950|2|625| +2450829|3952|2|908| +2450829|3955|2|188| +2450829|3956|2|720| +2450829|3958|2|119| +2450829|3961|2|479| +2450829|3962|2|651| +2450829|3964|2|932| +2450829|3967|2|496| +2450829|3968|2|77| +2450829|3970|2|393| +2450829|3973|2|365| +2450829|3974|2|13| +2450829|3976|2|904| +2450829|3979|2|459| +2450829|3980|2|673| +2450829|3982|2|798| +2450829|3985|2|411| +2450829|3986|2|417| +2450829|3988|2|384| +2450829|3991|2|756| +2450829|3992|2|477| +2450829|3994|2|513| +2450829|3997|2|718| +2450829|3998|2|468| +2450829|4000|2|33| +2450829|4003|2|429| +2450829|4004|2|925| +2450829|4006|2|75| +2450829|4009|2|422| +2450829|4010|2|929| +2450829|4012|2|584| +2450829|4015|2|955| +2450829|4016|2|824| +2450829|4018|2|691| +2450829|4021|2|295| +2450829|4022|2|882| +2450829|4024|2|106| +2450829|4027|2|939| +2450829|4028|2|869| +2450829|4030|2|724| +2450829|4033|2|398| +2450829|4034|2|365| +2450829|4036|2|853| +2450829|4039|2|775| +2450829|4040|2|127| +2450829|4042|2|509| +2450829|4045|2|563| +2450829|4046|2|339| +2450829|4048|2|| +2450829|4051|2|181| +2450829|4052|2|760| +2450829|4054|2|835| +2450829|4057|2|269| +2450829|4058|2|194| +2450829|4060|2|857| +2450829|4063|2|| +2450829|4064|2|560| +2450829|4066|2|221| +2450829|4069|2|690| +2450829|4070|2|504| +2450829|4072|2|963| +2450829|4075|2|407| +2450829|4076|2|| +2450829|4078|2|159| +2450829|4081|2|72| +2450829|4082|2|114| +2450829|4084|2|895| +2450829|4087|2|500| +2450829|4088|2|969| +2450829|4090|2|767| +2450829|4093|2|492| +2450829|4094|2|302| +2450829|4096|2|| +2450829|4099|2|889| +2450829|4100|2|264| +2450829|4102|2|537| +2450829|4105|2|443| +2450829|4106|2|978| +2450829|4108|2|720| +2450829|4111|2|985| +2450829|4112|2|692| +2450829|4114|2|38| +2450829|4117|2|829| +2450829|4118|2|8| +2450829|4120|2|313| +2450829|4123|2|366| +2450829|4124|2|0| +2450829|4126|2|| +2450829|4129|2|192| +2450829|4130|2|997| +2450829|4132|2|886| +2450829|4135|2|14| +2450829|4136|2|713| +2450829|4138|2|430| +2450829|4141|2|743| +2450829|4142|2|923| +2450829|4144|2|220| +2450829|4147|2|818| +2450829|4148|2|592| +2450829|4150|2|191| +2450829|4153|2|837| +2450829|4154|2|119| +2450829|4156|2|806| +2450829|4159|2|491| +2450829|4160|2|125| +2450829|4162|2|138| +2450829|4165|2|614| +2450829|4166|2|518| +2450829|4168|2|559| +2450829|4171|2|321| +2450829|4172|2|531| +2450829|4174|2|528| +2450829|4177|2|989| +2450829|4178|2|739| +2450829|4180|2|71| +2450829|4183|2|110| +2450829|4184|2|511| +2450829|4186|2|406| +2450829|4189|2|847| +2450829|4190|2|63| +2450829|4192|2|119| +2450829|4195|2|550| +2450829|4196|2|437| +2450829|4198|2|656| +2450829|4201|2|5| +2450829|4202|2|742| +2450829|4204|2|196| +2450829|4207|2|904| +2450829|4208|2|932| +2450829|4210|2|11| +2450829|4213|2|353| +2450829|4214|2|280| +2450829|4216|2|836| +2450829|4219|2|893| +2450829|4220|2|737| +2450829|4222|2|77| +2450829|4225|2|703| +2450829|4226|2|558| +2450829|4228|2|959| +2450829|4231|2|884| +2450829|4232|2|996| +2450829|4234|2|983| +2450829|4237|2|519| +2450829|4238|2|415| +2450829|4240|2|732| +2450829|4243|2|257| +2450829|4244|2|| +2450829|4246|2|325| +2450829|4249|2|347| +2450829|4250|2|927| +2450829|4252|2|875| +2450829|4255|2|14| +2450829|4256|2|726| +2450829|4258|2|326| +2450829|4261|2|1000| +2450829|4262|2|| +2450829|4264|2|| +2450829|4267|2|282| +2450829|4268|2|220| +2450829|4270|2|934| +2450829|4273|2|207| +2450829|4274|2|468| +2450829|4276|2|900| +2450829|4279|2|551| +2450829|4280|2|889| +2450829|4282|2|868| +2450829|4285|2|187| +2450829|4286|2|575| +2450829|4288|2|474| +2450829|4291|2|877| +2450829|4292|2|128| +2450829|4294|2|856| +2450829|4297|2|219| +2450829|4298|2|933| +2450829|4300|2|562| +2450829|4303|2|287| +2450829|4304|2|969| +2450829|4306|2|989| +2450829|4309|2|| +2450829|4310|2|996| +2450829|4312|2|241| +2450829|4315|2|448| +2450829|4316|2|930| +2450829|4318|2|485| +2450829|4321|2|740| +2450829|4322|2|494| +2450829|4324|2|727| +2450829|4327|2|149| +2450829|4328|2|133| +2450829|4330|2|854| +2450829|4333|2|183| +2450829|4334|2|704| +2450829|4336|2|19| +2450829|4339|2|306| +2450829|4340|2|462| +2450829|4342|2|632| +2450829|4345|2|846| +2450829|4346|2|787| +2450829|4348|2|711| +2450829|4351|2|646| +2450829|4352|2|277| +2450829|4354|2|569| +2450829|4357|2|203| +2450829|4358|2|| +2450829|4360|2|685| +2450829|4363|2|547| +2450829|4364|2|166| +2450829|4366|2|979| +2450829|4369|2|388| +2450829|4370|2|642| +2450829|4372|2|948| +2450829|4375|2|671| +2450829|4376|2|502| +2450829|4378|2|550| +2450829|4381|2|880| +2450829|4382|2|617| +2450829|4384|2|832| +2450829|4387|2|806| +2450829|4388|2|729| +2450829|4390|2|995| +2450829|4393|2|734| +2450829|4394|2|897| +2450829|4396|2|563| +2450829|4399|2|859| +2450829|4400|2|624| +2450829|4402|2|487| +2450829|4405|2|1| +2450829|4406|2|88| +2450829|4408|2|774| +2450829|4411|2|81| +2450829|4412|2|| +2450829|4414|2|147| +2450829|4417|2|154| +2450829|4418|2|347| +2450829|4420|2|773| +2450829|4423|2|529| +2450829|4424|2|170| +2450829|4426|2|845| +2450829|4429|2|215| +2450829|4430|2|33| +2450829|4432|2|536| +2450829|4435|2|| +2450829|4436|2|403| +2450829|4438|2|936| +2450829|4441|2|| +2450829|4442|2|413| +2450829|4444|2|616| +2450829|4447|2|885| +2450829|4448|2|47| +2450829|4450|2|944| +2450829|4453|2|756| +2450829|4454|2|560| +2450829|4456|2|352| +2450829|4459|2|605| +2450829|4460|2|658| +2450829|4462|2|609| +2450829|4465|2|682| +2450829|4466|2|345| +2450829|4468|2|112| +2450829|4471|2|347| +2450829|4472|2|625| +2450829|4474|2|482| +2450829|4477|2|273| +2450829|4478|2|270| +2450829|4480|2|76| +2450829|4483|2|913| +2450829|4484|2|922| +2450829|4486|2|807| +2450829|4489|2|282| +2450829|4490|2|760| +2450829|4492|2|681| +2450829|4495|2|901| +2450829|4496|2|83| +2450829|4498|2|208| +2450829|4501|2|806| +2450829|4502|2|60| +2450829|4504|2|899| +2450829|4507|2|933| +2450829|4508|2|856| +2450829|4510|2|387| +2450829|4513|2|710| +2450829|4514|2|962| +2450829|4516|2|52| +2450829|4519|2|759| +2450829|4520|2|198| +2450829|4522|2|17| +2450829|4525|2|134| +2450829|4526|2|481| +2450829|4528|2|503| +2450829|4531|2|911| +2450829|4532|2|803| +2450829|4534|2|399| +2450829|4537|2|663| +2450829|4538|2|983| +2450829|4540|2|435| +2450829|4543|2|755| +2450829|4544|2|707| +2450829|4546|2|977| +2450829|4549|2|831| +2450829|4550|2|699| +2450829|4552|2|144| +2450829|4555|2|165| +2450829|4556|2|944| +2450829|4558|2|561| +2450829|4561|2|815| +2450829|4562|2|199| +2450829|4564|2|155| +2450829|4567|2|154| +2450829|4568|2|729| +2450829|4570|2|803| +2450829|4573|2|972| +2450829|4574|2|805| +2450829|4576|2|76| +2450829|4579|2|41| +2450829|4580|2|82| +2450829|4582|2|685| +2450829|4585|2|795| +2450829|4586|2|162| +2450829|4588|2|843| +2450829|4591|2|660| +2450829|4592|2|146| +2450829|4594|2|775| +2450829|4597|2|498| +2450829|4598|2|217| +2450829|4600|2|991| +2450829|4603|2|249| +2450829|4604|2|596| +2450829|4606|2|249| +2450829|4609|2|186| +2450829|4610|2|820| +2450829|4612|2|780| +2450829|4615|2|233| +2450829|4616|2|493| +2450829|4618|2|821| +2450829|4621|2|741| +2450829|4622|2|225| +2450829|4624|2|259| +2450829|4627|2|768| +2450829|4628|2|52| +2450829|4630|2|612| +2450829|4633|2|776| +2450829|4634|2|200| +2450829|4636|2|157| +2450829|4639|2|671| +2450829|4640|2|601| +2450829|4642|2|532| +2450829|4645|2|535| +2450829|4646|2|656| +2450829|4648|2|464| +2450829|4651|2|931| +2450829|4652|2|962| +2450829|4654|2|953| +2450829|4657|2|689| +2450829|4658|2|876| +2450829|4660|2|811| +2450829|4663|2|811| +2450829|4664|2|944| +2450829|4666|2|511| +2450829|4669|2|212| +2450829|4670|2|199| +2450829|4672|2|823| +2450829|4675|2|347| +2450829|4676|2|| +2450829|4678|2|449| +2450829|4681|2|290| +2450829|4682|2|43| +2450829|4684|2|250| +2450829|4687|2|668| +2450829|4688|2|335| +2450829|4690|2|267| +2450829|4693|2|325| +2450829|4694|2|79| +2450829|4696|2|983| +2450829|4699|2|271| +2450829|4700|2|674| +2450829|4702|2|808| +2450829|4705|2|145| +2450829|4706|2|83| +2450829|4708|2|800| +2450829|4711|2|435| +2450829|4712|2|| +2450829|4714|2|216| +2450829|4717|2|866| +2450829|4718|2|250| +2450829|4720|2|72| +2450829|4723|2|284| +2450829|4724|2|218| +2450829|4726|2|940| +2450829|4729|2|619| +2450829|4730|2|801| +2450829|4732|2|109| +2450829|4735|2|418| +2450829|4736|2|209| +2450829|4738|2|494| +2450829|4741|2|211| +2450829|4742|2|105| +2450829|4744|2|760| +2450829|4747|2|| +2450829|4748|2|| +2450829|4750|2|849| +2450829|4753|2|672| +2450829|4754|2|221| +2450829|4756|2|628| +2450829|4759|2|449| +2450829|4760|2|231| +2450829|4762|2|219| +2450829|4765|2|| +2450829|4766|2|894| +2450829|4768|2|401| +2450829|4771|2|512| +2450829|4772|2|88| +2450829|4774|2|769| +2450829|4777|2|44| +2450829|4778|2|545| +2450829|4780|2|167| +2450829|4783|2|966| +2450829|4784|2|54| +2450829|4786|2|367| +2450829|4789|2|370| +2450829|4790|2|524| +2450829|4792|2|257| +2450829|4795|2|598| +2450829|4796|2|210| +2450829|4798|2|651| +2450829|4801|2|643| +2450829|4802|2|783| +2450829|4804|2|| +2450829|4807|2|| +2450829|4808|2|547| +2450829|4810|2|682| +2450829|4813|2|215| +2450829|4814|2|719| +2450829|4816|2|768| +2450829|4819|2|501| +2450829|4820|2|818| +2450829|4822|2|602| +2450829|4825|2|674| +2450829|4826|2|837| +2450829|4828|2|286| +2450829|4831|2|848| +2450829|4832|2|946| +2450829|4834|2|846| +2450829|4837|2|76| +2450829|4838|2|244| +2450829|4840|2|747| +2450829|4843|2|747| +2450829|4844|2|255| +2450829|4846|2|325| +2450829|4849|2|1000| +2450829|4850|2|| +2450829|4852|2|| +2450829|4855|2|719| +2450829|4856|2|777| +2450829|4858|2|897| +2450829|4861|2|513| +2450829|4862|2|540| +2450829|4864|2|716| +2450829|4867|2|891| +2450829|4868|2|55| +2450829|4870|2|299| +2450829|4873|2|405| +2450829|4874|2|202| +2450829|4876|2|763| +2450829|4879|2|327| +2450829|4880|2|43| +2450829|4882|2|40| +2450829|4885|2|559| +2450829|4886|2|464| +2450829|4888|2|136| +2450829|4891|2|456| +2450829|4892|2|431| +2450829|4894|2|50| +2450829|4897|2|717| +2450829|4898|2|293| +2450829|4900|2|160| +2450829|4903|2|589| +2450829|4904|2|551| +2450829|4906|2|4| +2450829|4909|2|70| +2450829|4910|2|549| +2450829|4912|2|382| +2450829|4915|2|752| +2450829|4916|2|74| +2450829|4918|2|660| +2450829|4921|2|216| +2450829|4922|2|850| +2450829|4924|2|242| +2450829|4927|2|| +2450829|4928|2|467| +2450829|4930|2|813| +2450829|4933|2|97| +2450829|4934|2|926| +2450829|4936|2|769| +2450829|4939|2|593| +2450829|4940|2|503| +2450829|4942|2|774| +2450829|4945|2|254| +2450829|4946|2|428| +2450829|4948|2|934| +2450829|4951|2|545| +2450829|4952|2|| +2450829|4954|2|891| +2450829|4957|2|212| +2450829|4958|2|893| +2450829|4960|2|741| +2450829|4963|2|886| +2450829|4964|2|309| +2450829|4966|2|404| +2450829|4969|2|717| +2450829|4970|2|917| +2450829|4972|2|958| +2450829|4975|2|408| +2450829|4976|2|366| +2450829|4978|2|473| +2450829|4981|2|871| +2450829|4982|2|855| +2450829|4984|2|868| +2450829|4987|2|126| +2450829|4988|2|0| +2450829|4990|2|44| +2450829|4993|2|972| +2450829|4994|2|345| +2450829|4996|2|667| +2450829|4999|2|405| +2450829|5000|2|821| +2450829|5002|2|870| +2450829|5005|2|516| +2450829|5006|2|585| +2450829|5008|2|354| +2450829|5011|2|737| +2450829|5012|2|858| +2450829|5014|2|725| +2450829|5017|2|755| +2450829|5018|2|673| +2450829|5020|2|294| +2450829|5023|2|549| +2450829|5024|2|| +2450829|5026|2|165| +2450829|5029|2|805| +2450829|5030|2|926| +2450829|5032|2|691| +2450829|5035|2|569| +2450829|5036|2|863| +2450829|5038|2|330| +2450829|5041|2|| +2450829|5042|2|974| +2450829|5044|2|310| +2450829|5047|2|784| +2450829|5048|2|650| +2450829|5050|2|812| +2450829|5053|2|125| +2450829|5054|2|201| +2450829|5056|2|22| +2450829|5059|2|162| +2450829|5060|2|54| +2450829|5062|2|492| +2450829|5065|2|261| +2450829|5066|2|397| +2450829|5068|2|345| +2450829|5071|2|227| +2450829|5072|2|709| +2450829|5074|2|645| +2450829|5077|2|751| +2450829|5078|2|470| +2450829|5080|2|45| +2450829|5083|2|377| +2450829|5084|2|562| +2450829|5086|2|244| +2450829|5089|2|476| +2450829|5090|2|588| +2450829|5092|2|539| +2450829|5095|2|850| +2450829|5096|2|807| +2450829|5098|2|534| +2450829|5101|2|323| +2450829|5102|2|244| +2450829|5104|2|543| +2450829|5107|2|170| +2450829|5108|2|22| +2450829|5110|2|566| +2450829|5113|2|727| +2450829|5114|2|584| +2450829|5116|2|665| +2450829|5119|2|346| +2450829|5120|2|145| +2450829|5122|2|437| +2450829|5125|2|202| +2450829|5126|2|321| +2450829|5128|2|877| +2450829|5131|2|486| +2450829|5132|2|828| +2450829|5134|2|446| +2450829|5137|2|455| +2450829|5138|2|476| +2450829|5140|2|293| +2450829|5143|2|780| +2450829|5144|2|898| +2450829|5146|2|493| +2450829|5149|2|828| +2450829|5150|2|423| +2450829|5152|2|232| +2450829|5155|2|127| +2450829|5156|2|650| +2450829|5158|2|458| +2450829|5161|2|| +2450829|5162|2|885| +2450829|5164|2|99| +2450829|5167|2|727| +2450829|5168|2|856| +2450829|5170|2|| +2450829|5173|2|799| +2450829|5174|2|328| +2450829|5176|2|468| +2450829|5179|2|| +2450829|5180|2|565| +2450829|5182|2|499| +2450829|5185|2|212| +2450829|5186|2|522| +2450829|5188|2|| +2450829|5191|2|401| +2450829|5192|2|226| +2450829|5194|2|787| +2450829|5197|2|213| +2450829|5198|2|| +2450829|5200|2|166| +2450829|5203|2|| +2450829|5204|2|105| +2450829|5206|2|586| +2450829|5209|2|808| +2450829|5210|2|638| +2450829|5212|2|939| +2450829|5215|2|45| +2450829|5216|2|746| +2450829|5218|2|650| +2450829|5221|2|379| +2450829|5222|2|34| +2450829|5224|2|255| +2450829|5227|2|23| +2450829|5228|2|815| +2450829|5230|2|346| +2450829|5233|2|885| +2450829|5234|2|224| +2450829|5236|2|941| +2450829|5239|2|356| +2450829|5240|2|306| +2450829|5242|2|629| +2450829|5245|2|684| +2450829|5246|2|99| +2450829|5248|2|878| +2450829|5251|2|466| +2450829|5252|2|364| +2450829|5254|2|506| +2450829|5257|2|321| +2450829|5258|2|184| +2450829|5260|2|482| +2450829|5263|2|622| +2450829|5264|2|382| +2450829|5266|2|444| +2450829|5269|2|813| +2450829|5270|2|34| +2450829|5272|2|409| +2450829|5275|2|241| +2450829|5276|2|364| +2450829|5278|2|347| +2450829|5281|2|909| +2450829|5282|2|146| +2450829|5284|2|927| +2450829|5287|2|996| +2450829|5288|2|505| +2450829|5290|2|906| +2450829|5293|2|142| +2450829|5294|2|73| +2450829|5296|2|990| +2450829|5299|2|158| +2450829|5300|2|414| +2450829|5302|2|673| +2450829|5305|2|305| +2450829|5306|2|83| +2450829|5308|2|611| +2450829|5311|2|254| +2450829|5312|2|836| +2450829|5314|2|810| +2450829|5317|2|671| +2450829|5318|2|176| +2450829|5320|2|203| +2450829|5323|2|498| +2450829|5324|2|818| +2450829|5326|2|638| +2450829|5329|2|594| +2450829|5330|2|158| +2450829|5332|2|708| +2450829|5335|2|70| +2450829|5336|2|714| +2450829|5338|2|427| +2450829|5341|2|111| +2450829|5342|2|164| +2450829|5344|2|667| +2450829|5347|2|851| +2450829|5348|2|399| +2450829|5350|2|961| +2450829|5353|2|467| +2450829|5354|2|914| +2450829|5356|2|468| +2450829|5359|2|236| +2450829|5360|2|| +2450829|5362|2|425| +2450829|5365|2|967| +2450829|5366|2|37| +2450829|5368|2|544| +2450829|5371|2|191| +2450829|5372|2|42| +2450829|5374|2|| +2450829|5377|2|318| +2450829|5378|2|739| +2450829|5380|2|93| +2450829|5383|2|602| +2450829|5384|2|807| +2450829|5386|2|919| +2450829|5389|2|214| +2450829|5390|2|11| +2450829|5392|2|643| +2450829|5395|2|140| +2450829|5396|2|505| +2450829|5398|2|137| +2450829|5401|2|559| +2450829|5402|2|129| +2450829|5404|2|925| +2450829|5407|2|109| +2450829|5408|2|| +2450829|5410|2|170| +2450829|5413|2|246| +2450829|5414|2|129| +2450829|5416|2|751| +2450829|5419|2|13| +2450829|5420|2|888| +2450829|5422|2|924| +2450829|5425|2|540| +2450829|5426|2|582| +2450829|5428|2|995| +2450829|5431|2|886| +2450829|5432|2|320| +2450829|5434|2|801| +2450829|5437|2|38| +2450829|5438|2|973| +2450829|5440|2|661| +2450829|5443|2|663| +2450829|5444|2|695| +2450829|5446|2|480| +2450829|5449|2|102| +2450829|5450|2|163| +2450829|5452|2|366| +2450829|5455|2|| +2450829|5456|2|| +2450829|5458|2|502| +2450829|5461|2|177| +2450829|5462|2|322| +2450829|5464|2|153| +2450829|5467|2|613| +2450829|5468|2|541| +2450829|5470|2|69| +2450829|5473|2|14| +2450829|5474|2|980| +2450829|5476|2|| +2450829|5479|2|| +2450829|5480|2|774| +2450829|5482|2|593| +2450829|5485|2|870| +2450829|5486|2|366| +2450829|5488|2|145| +2450829|5491|2|877| +2450829|5492|2|709| +2450829|5494|2|197| +2450829|5497|2|97| +2450829|5498|2|223| +2450829|5500|2|779| +2450829|5503|2|615| +2450829|5504|2|199| +2450829|5506|2|| +2450829|5509|2|944| +2450829|5510|2|106| +2450829|5512|2|138| +2450829|5515|2|526| +2450829|5516|2|776| +2450829|5518|2|746| +2450829|5521|2|99| +2450829|5522|2|545| +2450829|5524|2|391| +2450829|5527|2|981| +2450829|5528|2|960| +2450829|5530|2|355| +2450829|5533|2|586| +2450829|5534|2|158| +2450829|5536|2|927| +2450829|5539|2|183| +2450829|5540|2|454| +2450829|5542|2|768| +2450829|5545|2|572| +2450829|5546|2|223| +2450829|5548|2|833| +2450829|5551|2|467| +2450829|5552|2|446| +2450829|5554|2|299| +2450829|5557|2|285| +2450829|5558|2|620| +2450829|5560|2|637| +2450829|5563|2|900| +2450829|5564|2|927| +2450829|5566|2|877| +2450829|5569|2|39| +2450829|5570|2|923| +2450829|5572|2|506| +2450829|5575|2|670| +2450829|5576|2|13| +2450829|5578|2|136| +2450829|5581|2|| +2450829|5582|2|289| +2450829|5584|2|946| +2450829|5587|2|179| +2450829|5588|2|227| +2450829|5590|2|722| +2450829|5593|2|676| +2450829|5594|2|817| +2450829|5596|2|434| +2450829|5599|2|251| +2450829|5600|2|377| +2450829|5602|2|81| +2450829|5605|2|889| +2450829|5606|2|635| +2450829|5608|2|482| +2450829|5611|2|566| +2450829|5612|2|314| +2450829|5614|2|667| +2450829|5617|2|82| +2450829|5618|2|211| +2450829|5620|2|800| +2450829|5623|2|782| +2450829|5624|2|864| +2450829|5626|2|956| +2450829|5629|2|966| +2450829|5630|2|454| +2450829|5632|2|42| +2450829|5635|2|802| +2450829|5636|2|650| +2450829|5638|2|| +2450829|5641|2|89| +2450829|5642|2|642| +2450829|5644|2|761| +2450829|5647|2|200| +2450829|5648|2|169| +2450829|5650|2|592| +2450829|5653|2|794| +2450829|5654|2|31| +2450829|5656|2|297| +2450829|5659|2|598| +2450829|5660|2|43| +2450829|5662|2|854| +2450829|5665|2|169| +2450829|5666|2|593| +2450829|5668|2|450| +2450829|5671|2|564| +2450829|5672|2|922| +2450829|5674|2|188| +2450829|5677|2|930| +2450829|5678|2|407| +2450829|5680|2|281| +2450829|5683|2|487| +2450829|5684|2|252| +2450829|5686|2|356| +2450829|5689|2|849| +2450829|5690|2|267| +2450829|5692|2|| +2450829|5695|2|125| +2450829|5696|2|397| +2450829|5698|2|311| +2450829|5701|2|943| +2450829|5702|2|670| +2450829|5704|2|78| +2450829|5707|2|322| +2450829|5708|2|463| +2450829|5710|2|680| +2450829|5713|2|933| +2450829|5714|2|802| +2450829|5716|2|811| +2450829|5719|2|183| +2450829|5720|2|200| +2450829|5722|2|729| +2450829|5725|2|480| +2450829|5726|2|925| +2450829|5728|2|| +2450829|5731|2|779| +2450829|5732|2|456| +2450829|5734|2|216| +2450829|5737|2|486| +2450829|5738|2|429| +2450829|5740|2|77| +2450829|5743|2|439| +2450829|5744|2|| +2450829|5746|2|738| +2450829|5749|2|888| +2450829|5750|2|861| +2450829|5752|2|587| +2450829|5755|2|296| +2450829|5756|2|629| +2450829|5758|2|28| +2450829|5761|2|572| +2450829|5762|2|956| +2450829|5764|2|468| +2450829|5767|2|210| +2450829|5768|2|933| +2450829|5770|2|689| +2450829|5773|2|950| +2450829|5774|2|312| +2450829|5776|2|294| +2450829|5779|2|128| +2450829|5780|2|408| +2450829|5782|2|43| +2450829|5785|2|439| +2450829|5786|2|542| +2450829|5788|2|117| +2450829|5791|2|434| +2450829|5792|2|| +2450829|5794|2|568| +2450829|5797|2|518| +2450829|5798|2|428| +2450829|5800|2|868| +2450829|5803|2|654| +2450829|5804|2|294| +2450829|5806|2|229| +2450829|5809|2|215| +2450829|5810|2|218| +2450829|5812|2|| +2450829|5815|2|270| +2450829|5816|2|283| +2450829|5818|2|307| +2450829|5821|2|849| +2450829|5822|2|249| +2450829|5824|2|633| +2450829|5827|2|886| +2450829|5828|2|64| +2450829|5830|2|831| +2450829|5833|2|14| +2450829|5834|2|812| +2450829|5836|2|164| +2450829|5839|2|877| +2450829|5840|2|944| +2450829|5842|2|512| +2450829|5845|2|907| +2450829|5846|2|155| +2450829|5848|2|891| +2450829|5851|2|910| +2450829|5852|2|176| +2450829|5854|2|745| +2450829|5857|2|382| +2450829|5858|2|783| +2450829|5860|2|858| +2450829|5863|2|817| +2450829|5864|2|977| +2450829|5866|2|720| +2450829|5869|2|816| +2450829|5870|2|836| +2450829|5872|2|381| +2450829|5875|2|369| +2450829|5876|2|757| +2450829|5878|2|973| +2450829|5881|2|718| +2450829|5882|2|908| +2450829|5884|2|697| +2450829|5887|2|172| +2450829|5888|2|77| +2450829|5890|2|860| +2450829|5893|2|282| +2450829|5894|2|780| +2450829|5896|2|25| +2450829|5899|2|883| +2450829|5900|2|| +2450829|5902|2|601| +2450829|5905|2|583| +2450829|5906|2|117| +2450829|5908|2|18| +2450829|5911|2|671| +2450829|5912|2|201| +2450829|5914|2|134| +2450829|5917|2|364| +2450829|5918|2|373| +2450829|5920|2|584| +2450829|5923|2|628| +2450829|5924|2|217| +2450829|5926|2|215| +2450829|5929|2|676| +2450829|5930|2|866| +2450829|5932|2|45| +2450829|5935|2|258| +2450829|5936|2|32| +2450829|5938|2|930| +2450829|5941|2|841| +2450829|5942|2|| +2450829|5944|2|900| +2450829|5947|2|| +2450829|5948|2|68| +2450829|5950|2|231| +2450829|5953|2|| +2450829|5954|2|53| +2450829|5956|2|132| +2450829|5959|2|49| +2450829|5960|2|180| +2450829|5962|2|935| +2450829|5965|2|661| +2450829|5966|2|732| +2450829|5968|2|49| +2450829|5971|2|935| +2450829|5972|2|513| +2450829|5974|2|890| +2450829|5977|2|981| +2450829|5978|2|800| +2450829|5980|2|604| +2450829|5983|2|312| +2450829|5984|2|578| +2450829|5986|2|729| +2450829|5989|2|315| +2450829|5990|2|810| +2450829|5992|2|608| +2450829|5995|2|52| +2450829|5996|2|716| +2450829|5998|2|121| +2450829|6001|2|| +2450829|6002|2|910| +2450829|6004|2|| +2450829|6007|2|743| +2450829|6008|2|510| +2450829|6010|2|987| +2450829|6013|2|423| +2450829|6014|2|889| +2450829|6016|2|17| +2450829|6019|2|640| +2450829|6020|2|| +2450829|6022|2|413| +2450829|6025|2|484| +2450829|6026|2|724| +2450829|6028|2|242| +2450829|6031|2|119| +2450829|6032|2|694| +2450829|6034|2|714| +2450829|6037|2|| +2450829|6038|2|532| +2450829|6040|2|965| +2450829|6043|2|296| +2450829|6044|2|827| +2450829|6046|2|701| +2450829|6049|2|927| +2450829|6050|2|376| +2450829|6052|2|641| +2450829|6055|2|735| +2450829|6056|2|118| +2450829|6058|2|48| +2450829|6061|2|406| +2450829|6062|2|825| +2450829|6064|2|514| +2450829|6067|2|223| +2450829|6068|2|796| +2450829|6070|2|706| +2450829|6073|2|384| +2450829|6074|2|202| +2450829|6076|2|17| +2450829|6079|2|132| +2450829|6080|2|549| +2450829|6082|2|634| +2450829|6085|2|| +2450829|6086|2|992| +2450829|6088|2|989| +2450829|6091|2|910| +2450829|6092|2|7| +2450829|6094|2|380| +2450829|6097|2|638| +2450829|6098|2|39| +2450829|6100|2|827| +2450829|6103|2|927| +2450829|6104|2|763| +2450829|6106|2|700| +2450829|6109|2|779| +2450829|6110|2|816| +2450829|6112|2|931| +2450829|6115|2|564| +2450829|6116|2|733| +2450829|6118|2|598| +2450829|6121|2|782| +2450829|6122|2|235| +2450829|6124|2|667| +2450829|6127|2|890| +2450829|6128|2|396| +2450829|6130|2|444| +2450829|6133|2|381| +2450829|6134|2|993| +2450829|6136|2|13| +2450829|6139|2|380| +2450829|6140|2|830| +2450829|6142|2|436| +2450829|6145|2|943| +2450829|6146|2|726| +2450829|6148|2|205| +2450829|6151|2|526| +2450829|6152|2|940| +2450829|6154|2|| +2450829|6157|2|334| +2450829|6158|2|700| +2450829|6160|2|890| +2450829|6163|2|884| +2450829|6164|2|731| +2450829|6166|2|422| +2450829|6169|2|232| +2450829|6170|2|901| +2450829|6172|2|678| +2450829|6175|2|258| +2450829|6176|2|509| +2450829|6178|2|696| +2450829|6181|2|387| +2450829|6182|2|26| +2450829|6184|2|213| +2450829|6187|2|104| +2450829|6188|2|442| +2450829|6190|2|363| +2450829|6193|2|182| +2450829|6194|2|879| +2450829|6196|2|350| +2450829|6199|2|196| +2450829|6200|2|123| +2450829|6202|2|658| +2450829|6205|2|974| +2450829|6206|2|671| +2450829|6208|2|571| +2450829|6211|2|| +2450829|6212|2|525| +2450829|6214|2|916| +2450829|6217|2|34| +2450829|6218|2|309| +2450829|6220|2|775| +2450829|6223|2|874| +2450829|6224|2|243| +2450829|6226|2|20| +2450829|6229|2|432| +2450829|6230|2|574| +2450829|6232|2|634| +2450829|6235|2|122| +2450829|6236|2|647| +2450829|6238|2|887| +2450829|6241|2|641| +2450829|6242|2|621| +2450829|6244|2|687| +2450829|6247|2|987| +2450829|6248|2|45| +2450829|6250|2|662| +2450829|6253|2|560| +2450829|6254|2|244| +2450829|6256|2|797| +2450829|6259|2|497| +2450829|6260|2|866| +2450829|6262|2|770| +2450829|6265|2|595| +2450829|6266|2|858| +2450829|6268|2|342| +2450829|6271|2|| +2450829|6272|2|845| +2450829|6274|2|486| +2450829|6277|2|250| +2450829|6278|2|959| +2450829|6280|2|384| +2450829|6283|2|846| +2450829|6284|2|279| +2450829|6286|2|| +2450829|6289|2|415| +2450829|6290|2|557| +2450829|6292|2|802| +2450829|6295|2|105| +2450829|6296|2|699| +2450829|6298|2|| +2450829|6301|2|504| +2450829|6302|2|358| +2450829|6304|2|681| +2450829|6307|2|589| +2450829|6308|2|631| +2450829|6310|2|607| +2450829|6313|2|905| +2450829|6314|2|217| +2450829|6316|2|804| +2450829|6319|2|949| +2450829|6320|2|810| +2450829|6322|2|878| +2450829|6325|2|192| +2450829|6326|2|107| +2450829|6328|2|950| +2450829|6331|2|565| +2450829|6332|2|51| +2450829|6334|2|303| +2450829|6337|2|973| +2450829|6338|2|27| +2450829|6340|2|633| +2450829|6343|2|550| +2450829|6344|2|670| +2450829|6346|2|417| +2450829|6349|2|565| +2450829|6350|2|511| +2450829|6352|2|114| +2450829|6355|2|953| +2450829|6356|2|655| +2450829|6358|2|921| +2450829|6361|2|737| +2450829|6362|2|385| +2450829|6364|2|384| +2450829|6367|2|170| +2450829|6368|2|688| +2450829|6370|2|| +2450829|6373|2|959| +2450829|6374|2|847| +2450829|6376|2|738| +2450829|6379|2|772| +2450829|6380|2|806| +2450829|6382|2|699| +2450829|6385|2|| +2450829|6386|2|782| +2450829|6388|2|54| +2450829|6391|2|656| +2450829|6392|2|71| +2450829|6394|2|23| +2450829|6397|2|0| +2450829|6398|2|648| +2450829|6400|2|343| +2450829|6403|2|135| +2450829|6404|2|423| +2450829|6406|2|791| +2450829|6409|2|970| +2450829|6410|2|| +2450829|6412|2|237| +2450829|6415|2|825| +2450829|6416|2|991| +2450829|6418|2|43| +2450829|6421|2|384| +2450829|6422|2|| +2450829|6424|2|435| +2450829|6427|2|480| +2450829|6428|2|880| +2450829|6430|2|788| +2450829|6433|2|702| +2450829|6434|2|913| +2450829|6436|2|779| +2450829|6439|2|326| +2450829|6440|2|507| +2450829|6442|2|533| +2450829|6445|2|931| +2450829|6446|2|592| +2450829|6448|2|435| +2450829|6451|2|55| +2450829|6452|2|46| +2450829|6454|2|918| +2450829|6457|2|629| +2450829|6458|2|509| +2450829|6460|2|361| +2450829|6463|2|411| +2450829|6464|2|443| +2450829|6466|2|716| +2450829|6469|2|719| +2450829|6470|2|331| +2450829|6472|2|| +2450829|6475|2|637| +2450829|6476|2|648| +2450829|6478|2|780| +2450829|6481|2|733| +2450829|6482|2|182| +2450829|6484|2|206| +2450829|6487|2|679| +2450829|6488|2|106| +2450829|6490|2|919| +2450829|6493|2|374| +2450829|6494|2|778| +2450829|6496|2|909| +2450829|6499|2|519| +2450829|6500|2|238| +2450829|6502|2|108| +2450829|6505|2|340| +2450829|6506|2|570| +2450829|6508|2|181| +2450829|6511|2|372| +2450829|6512|2|717| +2450829|6514|2|967| +2450829|6517|2|593| +2450829|6518|2|63| +2450829|6520|2|411| +2450829|6523|2|214| +2450829|6524|2|659| +2450829|6526|2|702| +2450829|6529|2|720| +2450829|6530|2|935| +2450829|6532|2|835| +2450829|6535|2|662| +2450829|6536|2|729| +2450829|6538|2|477| +2450829|6541|2|150| +2450829|6542|2|275| +2450829|6544|2|260| +2450829|6547|2|704| +2450829|6548|2|589| +2450829|6550|2|198| +2450829|6553|2|445| +2450829|6554|2|278| +2450829|6556|2|199| +2450829|6559|2|440| +2450829|6560|2|805| +2450829|6562|2|934| +2450829|6565|2|| +2450829|6566|2|269| +2450829|6568|2|321| +2450829|6571|2|514| +2450829|6572|2|538| +2450829|6574|2|573| +2450829|6577|2|66| +2450829|6578|2|742| +2450829|6580|2|175| +2450829|6583|2|122| +2450829|6584|2|503| +2450829|6586|2|714| +2450829|6589|2|700| +2450829|6590|2|564| +2450829|6592|2|449| +2450829|6595|2|681| +2450829|6596|2|781| +2450829|6598|2|550| +2450829|6601|2|566| +2450829|6602|2|323| +2450829|6604|2|773| +2450829|6607|2|254| +2450829|6608|2|433| +2450829|6610|2|405| +2450829|6613|2|48| +2450829|6614|2|863| +2450829|6616|2|804| +2450829|6619|2|735| +2450829|6620|2|832| +2450829|6622|2|672| +2450829|6625|2|843| +2450829|6626|2|220| +2450829|6628|2|609| +2450829|6631|2|123| +2450829|6632|2|631| +2450829|6634|2|894| +2450829|6637|2|322| +2450829|6638|2|233| +2450829|6640|2|951| +2450829|6643|2|758| +2450829|6644|2|| +2450829|6646|2|804| +2450829|6649|2|| +2450829|6650|2|571| +2450829|6652|2|| +2450829|6655|2|683| +2450829|6656|2|842| +2450829|6658|2|786| +2450829|6661|2|416| +2450829|6662|2|387| +2450829|6664|2|572| +2450829|6667|2|301| +2450829|6668|2|977| +2450829|6670|2|100| +2450829|6673|2|460| +2450829|6674|2|502| +2450829|6676|2|559| +2450829|6679|2|| +2450829|6680|2|27| +2450829|6682|2|605| +2450829|6685|2|354| +2450829|6686|2|579| +2450829|6688|2|629| +2450829|6691|2|919| +2450829|6692|2|708| +2450829|6694|2|726| +2450829|6697|2|779| +2450829|6698|2|721| +2450829|6700|2|270| +2450829|6703|2|43| +2450829|6704|2|820| +2450829|6706|2|902| +2450829|6709|2|271| +2450829|6710|2|108| +2450829|6712|2|713| +2450829|6715|2|742| +2450829|6716|2|564| +2450829|6718|2|153| +2450829|6721|2|170| +2450829|6722|2|385| +2450829|6724|2|92| +2450829|6727|2|| +2450829|6728|2|152| +2450829|6730|2|85| +2450829|6733|2|476| +2450829|6734|2|591| +2450829|6736|2|527| +2450829|6739|2|789| +2450829|6740|2|373| +2450829|6742|2|28| +2450829|6745|2|356| +2450829|6746|2|413| +2450829|6748|2|101| +2450829|6751|2|320| +2450829|6752|2|699| +2450829|6754|2|244| +2450829|6757|2|282| +2450829|6758|2|279| +2450829|6760|2|296| +2450829|6763|2|207| +2450829|6764|2|178| +2450829|6766|2|432| +2450829|6769|2|507| +2450829|6770|2|536| +2450829|6772|2|597| +2450829|6775|2|453| +2450829|6776|2|828| +2450829|6778|2|924| +2450829|6781|2|510| +2450829|6782|2|789| +2450829|6784|2|826| +2450829|6787|2|852| +2450829|6788|2|446| +2450829|6790|2|791| +2450829|6793|2|780| +2450829|6794|2|248| +2450829|6796|2|927| +2450829|6799|2|404| +2450829|6800|2|449| +2450829|6802|2|| +2450829|6805|2|| +2450829|6806|2|566| +2450829|6808|2|425| +2450829|6811|2|873| +2450829|6812|2|961| +2450829|6814|2|114| +2450829|6817|2|899| +2450829|6818|2|385| +2450829|6820|2|241| +2450829|6823|2|90| +2450829|6824|2|867| +2450829|6826|2|757| +2450829|6829|2|93| +2450829|6830|2|949| +2450829|6832|2|199| +2450829|6835|2|10| +2450829|6836|2|409| +2450829|6838|2|286| +2450829|6841|2|133| +2450829|6842|2|15| +2450829|6844|2|20| +2450829|6847|2|290| +2450829|6848|2|154| +2450829|6850|2|424| +2450829|6853|2|428| +2450829|6854|2|| +2450829|6856|2|895| +2450829|6859|2|500| +2450829|6860|2|752| +2450829|6862|2|99| +2450829|6865|2|| +2450829|6866|2|612| +2450829|6868|2|538| +2450829|6871|2|336| +2450829|6872|2|467| +2450829|6874|2|332| +2450829|6877|2|| +2450829|6878|2|412| +2450829|6880|2|235| +2450829|6883|2|616| +2450829|6884|2|601| +2450829|6886|2|600| +2450829|6889|2|| +2450829|6890|2|793| +2450829|6892|2|412| +2450829|6895|2|373| +2450829|6896|2|806| +2450829|6898|2|139| +2450829|6901|2|113| +2450829|6902|2|218| +2450829|6904|2|948| +2450829|6907|2|604| +2450829|6908|2|532| +2450829|6910|2|889| +2450829|6913|2|783| +2450829|6914|2|145| +2450829|6916|2|817| +2450829|6919|2|| +2450829|6920|2|985| +2450829|6922|2|443| +2450829|6925|2|237| +2450829|6926|2|143| +2450829|6928|2|221| +2450829|6931|2|414| +2450829|6932|2|532| +2450829|6934|2|790| +2450829|6937|2|114| +2450829|6938|2|84| +2450829|6940|2|427| +2450829|6943|2|41| +2450829|6944|2|806| +2450829|6946|2|831| +2450829|6949|2|986| +2450829|6950|2|753| +2450829|6952|2|679| +2450829|6955|2|289| +2450829|6956|2|435| +2450829|6958|2|246| +2450829|6961|2|457| +2450829|6962|2|962| +2450829|6964|2|135| +2450829|6967|2|619| +2450829|6968|2|334| +2450829|6970|2|994| +2450829|6973|2|400| +2450829|6974|2|457| +2450829|6976|2|| +2450829|6979|2|10| +2450829|6980|2|828| +2450829|6982|2|929| +2450829|6985|2|764| +2450829|6986|2|433| +2450829|6988|2|911| +2450829|6991|2|611| +2450829|6992|2|333| +2450829|6994|2|316| +2450829|6997|2|181| +2450829|6998|2|335| +2450829|7000|2|912| +2450829|7003|2|905| +2450829|7004|2|552| +2450829|7006|2|428| +2450829|7009|2|972| +2450829|7010|2|956| +2450829|7012|2|626| +2450829|7015|2|115| +2450829|7016|2|844| +2450829|7018|2|366| +2450829|7021|2|299| +2450829|7022|2|815| +2450829|7024|2|543| +2450829|7027|2|75| +2450829|7028|2|67| +2450829|7030|2|175| +2450829|7033|2|927| +2450829|7034|2|965| +2450829|7036|2|191| +2450829|7039|2|45| +2450829|7040|2|754| +2450829|7042|2|52| +2450829|7045|2|798| +2450829|7046|2|404| +2450829|7048|2|387| +2450829|7051|2|411| +2450829|7052|2|585| +2450829|7054|2|756| +2450829|7057|2|627| +2450829|7058|2|477| +2450829|7060|2|929| +2450829|7063|2|489| +2450829|7064|2|858| +2450829|7066|2|829| +2450829|7069|2|286| +2450829|7070|2|719| +2450829|7072|2|854| +2450829|7075|2|714| +2450829|7076|2|601| +2450829|7078|2|409| +2450829|7081|2|103| +2450829|7082|2|788| +2450829|7084|2|196| +2450829|7087|2|611| +2450829|7088|2|703| +2450829|7090|2|1000| +2450829|7093|2|271| +2450829|7094|2|181| +2450829|7096|2|257| +2450829|7099|2|| +2450829|7100|2|903| +2450829|7102|2|332| +2450829|7105|2|685| +2450829|7106|2|172| +2450829|7108|2|125| +2450829|7111|2|647| +2450829|7112|2|424| +2450829|7114|2|640| +2450829|7117|2|750| +2450829|7118|2|190| +2450829|7120|2|936| +2450829|7123|2|763| +2450829|7124|2|234| +2450829|7126|2|986| +2450829|7129|2|115| +2450829|7130|2|834| +2450829|7132|2|710| +2450829|7135|2|679| +2450829|7136|2|933| +2450829|7138|2|692| +2450829|7141|2|61| +2450829|7142|2|68| +2450829|7144|2|478| +2450829|7147|2|389| +2450829|7148|2|943| +2450829|7150|2|850| +2450829|7153|2|765| +2450829|7154|2|97| +2450829|7156|2|754| +2450829|7159|2|678| +2450829|7160|2|| +2450829|7162|2|234| +2450829|7165|2|| +2450829|7166|2|466| +2450829|7168|2|242| +2450829|7171|2|496| +2450829|7172|2|628| +2450829|7174|2|| +2450829|7177|2|427| +2450829|7178|2|301| +2450829|7180|2|224| +2450829|7183|2|498| +2450829|7184|2|104| +2450829|7186|2|747| +2450829|7189|2|254| +2450829|7190|2|612| +2450829|7192|2|| +2450829|7195|2|848| +2450829|7196|2|470| +2450829|7198|2|| +2450829|7201|2|880| +2450829|7202|2|128| +2450829|7204|2|389| +2450829|7207|2|581| +2450829|7208|2|923| +2450829|7210|2|449| +2450829|7213|2|| +2450829|7214|2|934| +2450829|7216|2|| +2450829|7219|2|529| +2450829|7220|2|562| +2450829|7222|2|842| +2450829|7225|2|23| +2450829|7226|2|712| +2450829|7228|2|232| +2450829|7231|2|189| +2450829|7232|2|623| +2450829|7234|2|232| +2450829|7237|2|482| +2450829|7238|2|883| +2450829|7240|2|664| +2450829|7243|2|208| +2450829|7244|2|557| +2450829|7246|2|173| +2450829|7249|2|101| +2450829|7250|2|322| +2450829|7252|2|649| +2450829|7255|2|635| +2450829|7256|2|693| +2450829|7258|2|833| +2450829|7261|2|851| +2450829|7262|2|911| +2450829|7264|2|685| +2450829|7267|2|733| +2450829|7268|2|328| +2450829|7270|2|878| +2450829|7273|2|325| +2450829|7274|2|450| +2450829|7276|2|123| +2450829|7279|2|482| +2450829|7280|2|947| +2450829|7282|2|231| +2450829|7285|2|| +2450829|7286|2|893| +2450829|7288|2|331| +2450829|7291|2|903| +2450829|7292|2|879| +2450829|7294|2|| +2450829|7297|2|719| +2450829|7298|2|381| +2450829|7300|2|477| +2450829|7303|2|28| +2450829|7304|2|417| +2450829|7306|2|416| +2450829|7309|2|864| +2450829|7310|2|666| +2450829|7312|2|220| +2450829|7315|2|442| +2450829|7316|2|332| +2450829|7318|2|734| +2450829|7321|2|204| +2450829|7322|2|982| +2450829|7324|2|218| +2450829|7327|2|754| +2450829|7328|2|51| +2450829|7330|2|| +2450829|7333|2|369| +2450829|7334|2|111| +2450829|7336|2|56| +2450829|7339|2|353| +2450829|7340|2|760| +2450829|7342|2|387| +2450829|7345|2|611| +2450829|7346|2|533| +2450829|7348|2|243| +2450829|7351|2|729| +2450829|7352|2|342| +2450829|7354|2|86| +2450829|7357|2|339| +2450829|7358|2|89| +2450829|7360|2|| +2450829|7363|2|949| +2450829|7364|2|785| +2450829|7366|2|180| +2450829|7369|2|310| +2450829|7370|2|720| +2450829|7372|2|706| +2450829|7375|2|587| +2450829|7376|2|690| +2450829|7378|2|486| +2450829|7381|2|381| +2450829|7382|2|962| +2450829|7384|2|961| +2450829|7387|2|473| +2450829|7388|2|17| +2450829|7390|2|535| +2450829|7393|2|598| +2450829|7394|2|137| +2450829|7396|2|716| +2450829|7399|2|127| +2450829|7400|2|182| +2450829|7402|2|430| +2450829|7405|2|234| +2450829|7406|2|525| +2450829|7408|2|40| +2450829|7411|2|841| +2450829|7412|2|81| +2450829|7414|2|849| +2450829|7417|2|358| +2450829|7418|2|717| +2450829|7420|2|899| +2450829|7423|2|| +2450829|7424|2|437| +2450829|7426|2|285| +2450829|7429|2|667| +2450829|7430|2|642| +2450829|7432|2|541| +2450829|7435|2|955| +2450829|7436|2|75| +2450829|7438|2|536| +2450829|7441|2|33| +2450829|7442|2|929| +2450829|7444|2|| +2450829|7447|2|838| +2450829|7448|2|196| +2450829|7450|2|157| +2450829|7453|2|204| +2450829|7454|2|770| +2450829|7456|2|671| +2450829|7459|2|593| +2450829|7460|2|851| +2450829|7462|2|265| +2450829|7465|2|761| +2450829|7466|2|71| +2450829|7468|2|181| +2450829|7471|2|120| +2450829|7472|2|490| +2450829|7474|2|451| +2450829|7477|2|893| +2450829|7478|2|306| +2450829|7480|2|735| +2450829|7483|2|630| +2450829|7484|2|671| +2450829|7486|2|971| +2450829|7489|2|713| +2450829|7490|2|988| +2450829|7492|2|934| +2450829|7495|2|937| +2450829|7496|2|670| +2450829|7498|2|820| +2450829|7501|2|506| +2450829|7502|2|983| +2450829|7504|2|82| +2450829|7507|2|90| +2450829|7508|2|195| +2450829|7510|2|| +2450829|7513|2|953| +2450829|7514|2|859| +2450829|7516|2|950| +2450829|7519|2|224| +2450829|7520|2|85| +2450829|7522|2|624| +2450829|7525|2|44| +2450829|7526|2|348| +2450829|7528|2|467| +2450829|7531|2|75| +2450829|7532|2|23| +2450829|7534|2|515| +2450829|7537|2|760| +2450829|7538|2|| +2450829|7540|2|852| +2450829|7543|2|134| +2450829|7544|2|216| +2450829|7546|2|848| +2450829|7549|2|278| +2450829|7550|2|73| +2450829|7552|2|408| +2450829|7555|2|159| +2450829|7556|2|348| +2450829|7558|2|| +2450829|7561|2|420| +2450829|7562|2|763| +2450829|7564|2|924| +2450829|7567|2|666| +2450829|7568|2|271| +2450829|7570|2|473| +2450829|7573|2|770| +2450829|7574|2|484| +2450829|7576|2|530| +2450829|7579|2|990| +2450829|7580|2|964| +2450829|7582|2|404| +2450829|7585|2|797| +2450829|7586|2|531| +2450829|7588|2|753| +2450829|7591|2|172| +2450829|7592|2|56| +2450829|7594|2|144| +2450829|7597|2|32| +2450829|7598|2|382| +2450829|7600|2|287| +2450829|7603|2|896| +2450829|7604|2|267| +2450829|7606|2|304| +2450829|7609|2|403| +2450829|7610|2|113| +2450829|7612|2|828| +2450829|7615|2|661| +2450829|7616|2|812| +2450829|7618|2|810| +2450829|7621|2|611| +2450829|7622|2|359| +2450829|7624|2|59| +2450829|7627|2|| +2450829|7628|2|373| +2450829|7630|2|212| +2450829|7633|2|759| +2450829|7634|2|| +2450829|7636|2|606| +2450829|7639|2|373| +2450829|7640|2|793| +2450829|7642|2|21| +2450829|7645|2|593| +2450829|7646|2|610| +2450829|7648|2|932| +2450829|7651|2|283| +2450829|7652|2|348| +2450829|7654|2|433| +2450829|7657|2|597| +2450829|7658|2|462| +2450829|7660|2|872| +2450829|7663|2|910| +2450829|7664|2|664| +2450829|7666|2|489| +2450829|7669|2|| +2450829|7670|2|950| +2450829|7672|2|211| +2450829|7675|2|841| +2450829|7676|2|| +2450829|7678|2|764| +2450829|7681|2|823| +2450829|7682|2|242| +2450829|7684|2|803| +2450829|7687|2|411| +2450829|7688|2|912| +2450829|7690|2|803| +2450829|7693|2|960| +2450829|7694|2|25| +2450829|7696|2|233| +2450829|7699|2|62| +2450829|7700|2|378| +2450829|7702|2|268| +2450829|7705|2|438| +2450829|7706|2|234| +2450829|7708|2|| +2450829|7711|2|483| +2450829|7712|2|1000| +2450829|7714|2|987| +2450829|7717|2|207| +2450829|7718|2|621| +2450829|7720|2|466| +2450829|7723|2|313| +2450829|7724|2|642| +2450829|7726|2|710| +2450829|7729|2|613| +2450829|7730|2|7| +2450829|7732|2|784| +2450829|7735|2|13| +2450829|7736|2|498| +2450829|7738|2|755| +2450829|7741|2|774| +2450829|7742|2|819| +2450829|7744|2|894| +2450829|7747|2|918| +2450829|7748|2|902| +2450829|7750|2|779| +2450829|7753|2|19| +2450829|7754|2|370| +2450829|7756|2|629| +2450829|7759|2|573| +2450829|7760|2|631| +2450829|7762|2|150| +2450829|7765|2|97| +2450829|7766|2|| +2450829|7768|2|278| +2450829|7771|2|535| +2450829|7772|2|294| +2450829|7774|2|119| +2450829|7777|2|209| +2450829|7778|2|837| +2450829|7780|2|160| +2450829|7783|2|155| +2450829|7784|2|943| +2450829|7786|2|989| +2450829|7789|2|78| +2450829|7790|2|116| +2450829|7792|2|252| +2450829|7795|2|448| +2450829|7796|2|564| +2450829|7798|2|436| +2450829|7801|2|194| +2450829|7802|2|861| +2450829|7804|2|239| +2450829|7807|2|855| +2450829|7808|2|374| +2450829|7810|2|6| +2450829|7813|2|| +2450829|7814|2|619| +2450829|7816|2|913| +2450829|7819|2|639| +2450829|7820|2|296| +2450829|7822|2|459| +2450829|7825|2|| +2450829|7826|2|245| +2450829|7828|2|527| +2450829|7831|2|849| +2450829|7832|2|958| +2450829|7834|2|855| +2450829|7837|2|98| +2450829|7838|2|825| +2450829|7840|2|541| +2450829|7843|2|676| +2450829|7844|2|634| +2450829|7846|2|| +2450829|7849|2|139| +2450829|7850|2|881| +2450829|7852|2|313| +2450829|7855|2|164| +2450829|7856|2|381| +2450829|7858|2|118| +2450829|7861|2|689| +2450829|7862|2|235| +2450829|7864|2|988| +2450829|7867|2|357| +2450829|7868|2|210| +2450829|7870|2|702| +2450829|7873|2|803| +2450829|7874|2|314| +2450829|7876|2|672| +2450829|7879|2|593| +2450829|7880|2|965| +2450829|7882|2|61| +2450829|7885|2|664| +2450829|7886|2|306| +2450829|7888|2|104| +2450829|7891|2|321| +2450829|7892|2|260| +2450829|7894|2|473| +2450829|7897|2|620| +2450829|7898|2|906| +2450829|7900|2|867| +2450829|7903|2|370| +2450829|7904|2|912| +2450829|7906|2|493| +2450829|7909|2|227| +2450829|7910|2|222| +2450829|7912|2|161| +2450829|7915|2|44| +2450829|7916|2|197| +2450829|7918|2|| +2450829|7921|2|486| +2450829|7922|2|877| +2450829|7924|2|889| +2450829|7927|2|634| +2450829|7928|2|103| +2450829|7930|2|134| +2450829|7933|2|657| +2450829|7934|2|965| +2450829|7936|2|904| +2450829|7939|2|651| +2450829|7940|2|927| +2450829|7942|2|654| +2450829|7945|2|873| +2450829|7946|2|442| +2450829|7948|2|858| +2450829|7951|2|304| +2450829|7952|2|582| +2450829|7954|2|341| +2450829|7957|2|214| +2450829|7958|2|720| +2450829|7960|2|644| +2450829|7963|2|463| +2450829|7964|2|334| +2450829|7966|2|736| +2450829|7969|2|339| +2450829|7970|2|519| +2450829|7972|2|575| +2450829|7975|2|900| +2450829|7976|2|121| +2450829|7978|2|410| +2450829|7981|2|430| +2450829|7982|2|564| +2450829|7984|2|23| +2450829|7987|2|745| +2450829|7988|2|652| +2450829|7990|2|| +2450829|7993|2|| +2450829|7994|2|838| +2450829|7996|2|249| +2450829|7999|2|945| +2450829|8000|2|| +2450829|8002|2|549| +2450829|8005|2|203| +2450829|8006|2|408| +2450829|8008|2|359| +2450829|8011|2|127| +2450829|8012|2|939| +2450829|8014|2|578| +2450829|8017|2|520| +2450829|8018|2|344| +2450829|8020|2|585| +2450829|8023|2|447| +2450829|8024|2|177| +2450829|8026|2|625| +2450829|8029|2|177| +2450829|8030|2|548| +2450829|8032|2|883| +2450829|8035|2|889| +2450829|8036|2|982| +2450829|8038|2|419| +2450829|8041|2|581| +2450829|8042|2|730| +2450829|8044|2|732| +2450829|8047|2|105| +2450829|8048|2|151| +2450829|8050|2|4| +2450829|8053|2|366| +2450829|8054|2|| +2450829|8056|2|229| +2450829|8059|2|710| +2450829|8060|2|201| +2450829|8062|2|253| +2450829|8065|2|186| +2450829|8066|2|204| +2450829|8068|2|476| +2450829|8071|2|972| +2450829|8072|2|524| +2450829|8074|2|996| +2450829|8077|2|459| +2450829|8078|2|894| +2450829|8080|2|135| +2450829|8083|2|528| +2450829|8084|2|120| +2450829|8086|2|657| +2450829|8089|2|163| +2450829|8090|2|398| +2450829|8092|2|| +2450829|8095|2|726| +2450829|8096|2|808| +2450829|8098|2|964| +2450829|8101|2|346| +2450829|8102|2|419| +2450829|8104|2|842| +2450829|8107|2|951| +2450829|8108|2|317| +2450829|8110|2|645| +2450829|8113|2|374| +2450829|8114|2|| +2450829|8116|2|493| +2450829|8119|2|603| +2450829|8120|2|271| +2450829|8122|2|618| +2450829|8125|2|253| +2450829|8126|2|622| +2450829|8128|2|196| +2450829|8131|2|375| +2450829|8132|2|769| +2450829|8134|2|852| +2450829|8137|2|589| +2450829|8138|2|883| +2450829|8140|2|429| +2450829|8143|2|802| +2450829|8144|2|970| +2450829|8146|2|| +2450829|8149|2|13| +2450829|8150|2|312| +2450829|8152|2|470| +2450829|8155|2|48| +2450829|8156|2|338| +2450829|8158|2|985| +2450829|8161|2|524| +2450829|8162|2|576| +2450829|8164|2|14| +2450829|8167|2|275| +2450829|8168|2|175| +2450829|8170|2|77| +2450829|8173|2|191| +2450829|8174|2|638| +2450829|8176|2|860| +2450829|8179|2|234| +2450829|8180|2|| +2450829|8182|2|813| +2450829|8185|2|905| +2450829|8186|2|589| +2450829|8188|2|17| +2450829|8191|2|298| +2450829|8192|2|400| +2450829|8194|2|71| +2450829|8197|2|166| +2450829|8198|2|890| +2450829|8200|2|823| +2450829|8203|2|779| +2450829|8204|2|923| +2450829|8206|2|935| +2450829|8209|2|692| +2450829|8210|2|303| +2450829|8212|2|284| +2450829|8215|2|319| +2450829|8216|2|410| +2450829|8218|2|868| +2450829|8221|2|226| +2450829|8222|2|7| +2450829|8224|2|479| +2450829|8227|2|684| +2450829|8228|2|292| +2450829|8230|2|392| +2450829|8233|2|114| +2450829|8234|2|160| +2450829|8236|2|558| +2450829|8239|2|450| +2450829|8240|2|871| +2450829|8242|2|223| +2450829|8245|2|283| +2450829|8246|2|794| +2450829|8248|2|| +2450829|8251|2|234| +2450829|8252|2|493| +2450829|8254|2|951| +2450829|8257|2|322| +2450829|8258|2|580| +2450829|8260|2|880| +2450829|8263|2|848| +2450829|8264|2|655| +2450829|8266|2|830| +2450829|8269|2|223| +2450829|8270|2|433| +2450829|8272|2|266| +2450829|8275|2|471| +2450829|8276|2|499| +2450829|8278|2|983| +2450829|8281|2|410| +2450829|8282|2|121| +2450829|8284|2|853| +2450829|8287|2|590| +2450829|8288|2|951| +2450829|8290|2|26| +2450829|8293|2|494| +2450829|8294|2|94| +2450829|8296|2|523| +2450829|8299|2|428| +2450829|8300|2|100| +2450829|8302|2|624| +2450829|8305|2|64| +2450829|8306|2|876| +2450829|8308|2|772| +2450829|8311|2|| +2450829|8312|2|533| +2450829|8314|2|228| +2450829|8317|2|162| +2450829|8318|2|| +2450829|8320|2|247| +2450829|8323|2|604| +2450829|8324|2|201| +2450829|8326|2|988| +2450829|8329|2|416| +2450829|8330|2|176| +2450829|8332|2|721| +2450829|8335|2|833| +2450829|8336|2|783| +2450829|8338|2|| +2450829|8341|2|102| +2450829|8342|2|998| +2450829|8344|2|477| +2450829|8347|2|924| +2450829|8348|2|638| +2450829|8350|2|674| +2450829|8353|2|187| +2450829|8354|2|48| +2450829|8356|2|761| +2450829|8359|2|337| +2450829|8360|2|151| +2450829|8362|2|177| +2450829|8365|2|701| +2450829|8366|2|693| +2450829|8368|2|434| +2450829|8371|2|965| +2450829|8372|2|89| +2450829|8374|2|150| +2450829|8377|2|168| +2450829|8378|2|605| +2450829|8380|2|612| +2450829|8383|2|397| +2450829|8384|2|315| +2450829|8386|2|| +2450829|8389|2|926| +2450829|8390|2|820| +2450829|8392|2|343| +2450829|8395|2|245| +2450829|8396|2|| +2450829|8398|2|143| +2450829|8401|2|454| +2450829|8402|2|724| +2450829|8404|2|343| +2450829|8407|2|919| +2450829|8408|2|439| +2450829|8410|2|381| +2450829|8413|2|968| +2450829|8414|2|119| +2450829|8416|2|983| +2450829|8419|2|825| +2450829|8420|2|641| +2450829|8422|2|909| +2450829|8425|2|129| +2450829|8426|2|430| +2450829|8428|2|393| +2450829|8431|2|968| +2450829|8432|2|333| +2450829|8434|2|831| +2450829|8437|2|333| +2450829|8438|2|875| +2450829|8440|2|6| +2450829|8443|2|419| +2450829|8444|2|75| +2450829|8446|2|128| +2450829|8449|2|135| +2450829|8450|2|31| +2450829|8452|2|570| +2450829|8455|2|848| +2450829|8456|2|694| +2450829|8458|2|491| +2450829|8461|2|857| +2450829|8462|2|706| +2450829|8464|2|389| +2450829|8467|2|158| +2450829|8468|2|64| +2450829|8470|2|317| +2450829|8473|2|746| +2450829|8474|2|979| +2450829|8476|2|571| +2450829|8479|2|41| +2450829|8480|2|985| +2450829|8482|2|576| +2450829|8485|2|736| +2450829|8486|2|651| +2450829|8488|2|738| +2450829|8491|2|| +2450829|8492|2|| +2450829|8494|2|35| +2450829|8497|2|713| +2450829|8498|2|907| +2450829|8500|2|406| +2450829|8503|2|182| +2450829|8504|2|264| +2450829|8506|2|851| +2450829|8509|2|355| +2450829|8510|2|20| +2450829|8512|2|740| +2450829|8515|2|238| +2450829|8516|2|380| +2450829|8518|2|503| +2450829|8521|2|614| +2450829|8522|2|504| +2450829|8524|2|75| +2450829|8527|2|| +2450829|8528|2|298| +2450829|8530|2|780| +2450829|8533|2|333| +2450829|8534|2|577| +2450829|8536|2|363| +2450829|8539|2|540| +2450829|8540|2|188| +2450829|8542|2|711| +2450829|8545|2|869| +2450829|8546|2|945| +2450829|8548|2|942| +2450829|8551|2|258| +2450829|8552|2|214| +2450829|8554|2|124| +2450829|8557|2|663| +2450829|8558|2|640| +2450829|8560|2|637| +2450829|8563|2|11| +2450829|8564|2|779| +2450829|8566|2|770| +2450829|8569|2|238| +2450829|8570|2|899| +2450829|8572|2|731| +2450829|8575|2|381| +2450829|8576|2|863| +2450829|8578|2|42| +2450829|8581|2|670| +2450829|8582|2|898| +2450829|8584|2|48| +2450829|8587|2|83| +2450829|8588|2|708| +2450829|8590|2|113| +2450829|8593|2|588| +2450829|8594|2|578| +2450829|8596|2|733| +2450829|8599|2|170| +2450829|8600|2|84| +2450829|8602|2|493| +2450829|8605|2|727| +2450829|8606|2|| +2450829|8608|2|142| +2450829|8611|2|906| +2450829|8612|2|299| +2450829|8614|2|186| +2450829|8617|2|335| +2450829|8618|2|797| +2450829|8620|2|239| +2450829|8623|2|976| +2450829|8624|2|759| +2450829|8626|2|| +2450829|8629|2|137| +2450829|8630|2|978| +2450829|8632|2|327| +2450829|8635|2|651| +2450829|8636|2|810| +2450829|8638|2|752| +2450829|8641|2|395| +2450829|8642|2|922| +2450829|8644|2|414| +2450829|8647|2|634| +2450829|8648|2|574| +2450829|8650|2|21| +2450829|8653|2|919| +2450829|8654|2|590| +2450829|8656|2|655| +2450829|8659|2|12| +2450829|8660|2|193| +2450829|8662|2|955| +2450829|8665|2|94| +2450829|8666|2|68| +2450829|8668|2|979| +2450829|8671|2|750| +2450829|8672|2|245| +2450829|8674|2|140| +2450829|8677|2|443| +2450829|8678|2|30| +2450829|8680|2|664| +2450829|8683|2|299| +2450829|8684|2|226| +2450829|8686|2|923| +2450829|8689|2|767| +2450829|8690|2|796| +2450829|8692|2|129| +2450829|8695|2|74| +2450829|8696|2|239| +2450829|8698|2|| +2450829|8701|2|734| +2450829|8702|2|715| +2450829|8704|2|778| +2450829|8707|2|251| +2450829|8708|2|687| +2450829|8710|2|160| +2450829|8713|2|550| +2450829|8714|2|154| +2450829|8716|2|547| +2450829|8719|2|244| +2450829|8720|2|276| +2450829|8722|2|438| +2450829|8725|2|924| +2450829|8726|2|363| +2450829|8728|2|764| +2450829|8731|2|927| +2450829|8732|2|21| +2450829|8734|2|573| +2450829|8737|2|28| +2450829|8738|2|| +2450829|8740|2|458| +2450829|8743|2|692| +2450829|8744|2|498| +2450829|8746|2|575| +2450829|8749|2|172| +2450829|8750|2|597| +2450829|8752|2|430| +2450829|8755|2|779| +2450829|8756|2|249| +2450829|8758|2|498| +2450829|8761|2|995| +2450829|8762|2|573| +2450829|8764|2|344| +2450829|8767|2|143| +2450829|8768|2|686| +2450829|8770|2|135| +2450829|8773|2|776| +2450829|8774|2|643| +2450829|8776|2|172| +2450829|8779|2|| +2450829|8780|2|551| +2450829|8782|2|620| +2450829|8785|2|576| +2450829|8786|2|71| +2450829|8788|2|654| +2450829|8791|2|846| +2450829|8792|2|110| +2450829|8794|2|535| +2450829|8797|2|80| +2450829|8798|2|463| +2450829|8800|2|976| +2450829|8803|2|701| +2450829|8804|2|705| +2450829|8806|2|874| +2450829|8809|2|618| +2450829|8810|2|462| +2450829|8812|2|691| +2450829|8815|2|740| +2450829|8816|2|625| +2450829|8818|2|554| +2450829|8821|2|326| +2450829|8822|2|320| +2450829|8824|2|640| +2450829|8827|2|873| +2450829|8828|2|873| +2450829|8830|2|95| +2450829|8833|2|534| +2450829|8834|2|661| +2450829|8836|2|559| +2450829|8839|2|917| +2450829|8840|2|205| +2450829|8842|2|651| +2450829|8845|2|17| +2450829|8846|2|27| +2450829|8848|2|265| +2450829|8851|2|586| +2450829|8852|2|883| +2450829|8854|2|234| +2450829|8857|2|571| +2450829|8858|2|27| +2450829|8860|2|67| +2450829|8863|2|971| +2450829|8864|2|989| +2450829|8866|2|240| +2450829|8869|2|775| +2450829|8870|2|| +2450829|8872|2|350| +2450829|8875|2|357| +2450829|8876|2|581| +2450829|8878|2|619| +2450829|8881|2|680| +2450829|8882|2|288| +2450829|8884|2|165| +2450829|8887|2|706| +2450829|8888|2|450| +2450829|8890|2|481| +2450829|8893|2|8| +2450829|8894|2|948| +2450829|8896|2|315| +2450829|8899|2|969| +2450829|8900|2|676| +2450829|8902|2|949| +2450829|8905|2|410| +2450829|8906|2|795| +2450829|8908|2|| +2450829|8911|2|939| +2450829|8912|2|631| +2450829|8914|2|187| +2450829|8917|2|959| +2450829|8918|2|184| +2450829|8920|2|622| +2450829|8923|2|869| +2450829|8924|2|70| +2450829|8926|2|418| +2450829|8929|2|984| +2450829|8930|2|599| +2450829|8932|2|| +2450829|8935|2|612| +2450829|8936|2|919| +2450829|8938|2|225| +2450829|8941|2|509| +2450829|8942|2|334| +2450829|8944|2|345| +2450829|8947|2|233| +2450829|8948|2|22| +2450829|8950|2|14| +2450829|8953|2|330| +2450829|8954|2|786| +2450829|8956|2|26| +2450829|8959|2|358| +2450829|8960|2|110| +2450829|8962|2|811| +2450829|8965|2|651| +2450829|8966|2|53| +2450829|8968|2|333| +2450829|8971|2|391| +2450829|8972|2|96| +2450829|8974|2|494| +2450829|8977|2|754| +2450829|8978|2|814| +2450829|8980|2|867| +2450829|8983|2|| +2450829|8984|2|15| +2450829|8986|2|492| +2450829|8989|2|957| +2450829|8990|2|829| +2450829|8992|2|825| +2450829|8995|2|972| +2450829|8996|2|516| +2450829|8998|2|51| +2450829|9001|2|899| +2450829|9002|2|615| +2450829|9004|2|571| +2450829|9007|2|541| +2450829|9008|2|779| +2450829|9010|2|830| +2450829|9013|2|837| +2450829|9014|2|864| +2450829|9016|2|809| +2450829|9019|2|818| +2450829|9020|2|68| +2450829|9022|2|156| +2450829|9025|2|608| +2450829|9026|2|242| +2450829|9028|2|440| +2450829|9031|2|693| +2450829|9032|2|62| +2450829|9034|2|96| +2450829|9037|2|783| +2450829|9038|2|707| +2450829|9040|2|| +2450829|9043|2|540| +2450829|9044|2|797| +2450829|9046|2|904| +2450829|9049|2|306| +2450829|9050|2|866| +2450829|9052|2|527| +2450829|9055|2|702| +2450829|9056|2|672| +2450829|9058|2|209| +2450829|9061|2|261| +2450829|9062|2|521| +2450829|9064|2|508| +2450829|9067|2|360| +2450829|9068|2|719| +2450829|9070|2|382| +2450829|9073|2|174| +2450829|9074|2|561| +2450829|9076|2|462| +2450829|9079|2|196| +2450829|9080|2|646| +2450829|9082|2|253| +2450829|9085|2|150| +2450829|9086|2|768| +2450829|9088|2|919| +2450829|9091|2|610| +2450829|9092|2|364| +2450829|9094|2|985| +2450829|9097|2|939| +2450829|9098|2|| +2450829|9100|2|946| +2450829|9103|2|605| +2450829|9104|2|988| +2450829|9106|2|302| +2450829|9109|2|415| +2450829|9110|2|534| +2450829|9112|2|| +2450829|9115|2|764| +2450829|9116|2|7| +2450829|9118|2|971| +2450829|9121|2|104| +2450829|9122|2|285| +2450829|9124|2|907| +2450829|9127|2|961| +2450829|9128|2|198| +2450829|9130|2|634| +2450829|9133|2|622| +2450829|9134|2|720| +2450829|9136|2|588| +2450829|9139|2|133| +2450829|9140|2|800| +2450829|9142|2|572| +2450829|9145|2|923| +2450829|9146|2|746| +2450829|9148|2|741| +2450829|9151|2|999| +2450829|9152|2|70| +2450829|9154|2|337| +2450829|9157|2|376| +2450829|9158|2|451| +2450829|9160|2|741| +2450829|9163|2|78| +2450829|9164|2|75| +2450829|9166|2|292| +2450829|9169|2|618| +2450829|9170|2|540| +2450829|9172|2|986| +2450829|9175|2|249| +2450829|9176|2|110| +2450829|9178|2|213| +2450829|9181|2|498| +2450829|9182|2|296| +2450829|9184|2|856| +2450829|9187|2|| +2450829|9188|2|| +2450829|9190|2|780| +2450829|9193|2|240| +2450829|9194|2|585| +2450829|9196|2|468| +2450829|9199|2|282| +2450829|9200|2|336| +2450829|9202|2|847| +2450829|9205|2|480| +2450829|9206|2|254| +2450829|9208|2|816| +2450829|9211|2|699| +2450829|9212|2|398| +2450829|9214|2|847| +2450829|9217|2|383| +2450829|9218|2|438| +2450829|9220|2|| +2450829|9223|2|240| +2450829|9224|2|870| +2450829|9226|2|526| +2450829|9229|2|779| +2450829|9230|2|788| +2450829|9232|2|389| +2450829|9235|2|525| +2450829|9236|2|314| +2450829|9238|2|573| +2450829|9241|2|926| +2450829|9242|2|437| +2450829|9244|2|622| +2450829|9247|2|980| +2450829|9248|2|893| +2450829|9250|2|927| +2450829|9253|2|725| +2450829|9254|2|299| +2450829|9256|2|43| +2450829|9259|2|529| +2450829|9260|2|412| +2450829|9262|2|351| +2450829|9265|2|177| +2450829|9266|2|676| +2450829|9268|2|682| +2450829|9271|2|920| +2450829|9272|2|835| +2450829|9274|2|989| +2450829|9277|2|778| +2450829|9278|2|269| +2450829|9280|2|289| +2450829|9283|2|440| +2450829|9284|2|949| +2450829|9286|2|954| +2450829|9289|2|197| +2450829|9290|2|264| +2450829|9292|2|551| +2450829|9295|2|807| +2450829|9296|2|0| +2450829|9298|2|382| +2450829|9301|2|| +2450829|9302|2|528| +2450829|9304|2|538| +2450829|9307|2|908| +2450829|9308|2|464| +2450829|9310|2|287| +2450829|9313|2|446| +2450829|9314|2|80| +2450829|9316|2|699| +2450829|9319|2|200| +2450829|9320|2|487| +2450829|9322|2|564| +2450829|9325|2|417| +2450829|9326|2|784| +2450829|9328|2|699| +2450829|9331|2|226| +2450829|9332|2|750| +2450829|9334|2|157| +2450829|9337|2|101| +2450829|9338|2|54| +2450829|9340|2|470| +2450829|9343|2|561| +2450829|9344|2|476| +2450829|9346|2|687| +2450829|9349|2|26| +2450829|9350|2|20| +2450829|9352|2|| +2450829|9355|2|718| +2450829|9356|2|813| +2450829|9358|2|712| +2450829|9361|2|150| +2450829|9362|2|358| +2450829|9364|2|908| +2450829|9367|2|981| +2450829|9368|2|414| +2450829|9370|2|618| +2450829|9373|2|513| +2450829|9374|2|467| +2450829|9376|2|503| +2450829|9379|2|434| +2450829|9380|2|747| +2450829|9382|2|951| +2450829|9385|2|334| +2450829|9386|2|499| +2450829|9388|2|16| +2450829|9391|2|582| +2450829|9392|2|160| +2450829|9394|2|204| +2450829|9397|2|248| +2450829|9398|2|696| +2450829|9400|2|97| +2450829|9403|2|708| +2450829|9404|2|298| +2450829|9406|2|767| +2450829|9409|2|371| +2450829|9410|2|960| +2450829|9412|2|262| +2450829|9415|2|493| +2450829|9416|2|670| +2450829|9418|2|689| +2450829|9421|2|805| +2450829|9422|2|812| +2450829|9424|2|265| +2450829|9427|2|67| +2450829|9428|2|179| +2450829|9430|2|950| +2450829|9433|2|33| +2450829|9434|2|209| +2450829|9436|2|533| +2450829|9439|2|995| +2450829|9440|2|324| +2450829|9442|2|494| +2450829|9445|2|549| +2450829|9446|2|790| +2450829|9448|2|898| +2450829|9451|2|414| +2450829|9452|2|213| +2450829|9454|2|866| +2450829|9457|2|54| +2450829|9458|2|308| +2450829|9460|2|392| +2450829|9463|2|853| +2450829|9464|2|131| +2450829|9466|2|169| +2450829|9469|2|| +2450829|9470|2|715| +2450829|9472|2|992| +2450829|9475|2|541| +2450829|9476|2|724| +2450829|9478|2|880| +2450829|9481|2|245| +2450829|9482|2|660| +2450829|9484|2|149| +2450829|9487|2|145| +2450829|9488|2|588| +2450829|9490|2|820| +2450829|9493|2|524| +2450829|9494|2|34| +2450829|9496|2|453| +2450829|9499|2|501| +2450829|9500|2|604| +2450829|9502|2|955| +2450829|9505|2|568| +2450829|9506|2|132| +2450829|9508|2|957| +2450829|9511|2|333| +2450829|9512|2|504| +2450829|9514|2|630| +2450829|9517|2|567| +2450829|9518|2|933| +2450829|9520|2|18| +2450829|9523|2|601| +2450829|9524|2|246| +2450829|9526|2|516| +2450829|9529|2|106| +2450829|9530|2|946| +2450829|9532|2|144| +2450829|9535|2|498| +2450829|9536|2|780| +2450829|9538|2|684| +2450829|9541|2|384| +2450829|9542|2|41| +2450829|9544|2|790| +2450829|9547|2|| +2450829|9548|2|777| +2450829|9550|2|242| +2450829|9553|2|237| +2450829|9554|2|724| +2450829|9556|2|922| +2450829|9559|2|111| +2450829|9560|2|366| +2450829|9562|2|| +2450829|9565|2|105| +2450829|9566|2|232| +2450829|9568|2|727| +2450829|9571|2|138| +2450829|9572|2|457| +2450829|9574|2|402| +2450829|9577|2|353| +2450829|9578|2|60| +2450829|9580|2|976| +2450829|9583|2|650| +2450829|9584|2|80| +2450829|9586|2|791| +2450829|9589|2|446| +2450829|9590|2|508| +2450829|9592|2|281| +2450829|9595|2|316| +2450829|9596|2|763| +2450829|9598|2|195| +2450829|9601|2|| +2450829|9602|2|705| +2450829|9604|2|431| +2450829|9607|2|317| +2450829|9608|2|888| +2450829|9610|2|939| +2450829|9613|2|443| +2450829|9614|2|| +2450829|9616|2|816| +2450829|9619|2|64| +2450829|9620|2|836| +2450829|9622|2|331| +2450829|9625|2|473| +2450829|9626|2|132| +2450829|9628|2|528| +2450829|9631|2|552| +2450829|9632|2|750| +2450829|9634|2|779| +2450829|9637|2|835| +2450829|9638|2|429| +2450829|9640|2|761| +2450829|9643|2|| +2450829|9644|2|176| +2450829|9646|2|179| +2450829|9649|2|409| +2450829|9650|2|523| +2450829|9652|2|190| +2450829|9655|2|989| +2450829|9656|2|646| +2450829|9658|2|275| +2450829|9661|2|181| +2450829|9662|2|97| +2450829|9664|2|222| +2450829|9667|2|677| +2450829|9668|2|| +2450829|9670|2|166| +2450829|9673|2|868| +2450829|9674|2|369| +2450829|9676|2|618| +2450829|9679|2|589| +2450829|9680|2|419| +2450829|9682|2|| +2450829|9685|2|89| +2450829|9686|2|428| +2450829|9688|2|641| +2450829|9691|2|257| +2450829|9692|2|297| +2450829|9694|2|966| +2450829|9697|2|594| +2450829|9698|2|97| +2450829|9700|2|630| +2450829|9703|2|869| +2450829|9704|2|736| +2450829|9706|2|774| +2450829|9709|2|421| +2450829|9710|2|7| +2450829|9712|2|760| +2450829|9715|2|426| +2450829|9716|2|904| +2450829|9718|2|100| +2450829|9721|2|168| +2450829|9722|2|718| +2450829|9724|2|653| +2450829|9727|2|907| +2450829|9728|2|355| +2450829|9730|2|284| +2450829|9733|2|745| +2450829|9734|2|500| +2450829|9736|2|25| +2450829|9739|2|104| +2450829|9740|2|604| +2450829|9742|2|833| +2450829|9745|2|76| +2450829|9746|2|740| +2450829|9748|2|926| +2450829|9751|2|69| +2450829|9752|2|640| +2450829|9754|2|910| +2450829|9757|2|| +2450829|9758|2|859| +2450829|9760|2|530| +2450829|9763|2|| +2450829|9764|2|224| +2450829|9766|2|956| +2450829|9769|2|478| +2450829|9770|2|748| +2450829|9772|2|673| +2450829|9775|2|166| +2450829|9776|2|625| +2450829|9778|2|824| +2450829|9781|2|840| +2450829|9782|2|368| +2450829|9784|2|244| +2450829|9787|2|575| +2450829|9788|2|942| +2450829|9790|2|578| +2450829|9793|2|600| +2450829|9794|2|7| +2450829|9796|2|129| +2450829|9799|2|61| +2450829|9800|2|219| +2450829|9802|2|| +2450829|9805|2|172| +2450829|9806|2|122| +2450829|9808|2|326| +2450829|9811|2|766| +2450829|9812|2|962| +2450829|9814|2|542| +2450829|9817|2|426| +2450829|9818|2|34| +2450829|9820|2|| +2450829|9823|2|183| +2450829|9824|2|506| +2450829|9826|2|330| +2450829|9829|2|58| +2450829|9830|2|735| +2450829|9832|2|281| +2450829|9835|2|799| +2450829|9836|2|952| +2450829|9838|2|292| +2450829|9841|2|602| +2450829|9842|2|978| +2450829|9844|2|43| +2450829|9847|2|261| +2450829|9848|2|530| +2450829|9850|2|289| +2450829|9853|2|907| +2450829|9854|2|56| +2450829|9856|2|206| +2450829|9859|2|383| +2450829|9860|2|470| +2450829|9862|2|997| +2450829|9865|2|318| +2450829|9866|2|775| +2450829|9868|2|426| +2450829|9871|2|866| +2450829|9872|2|442| +2450829|9874|2|191| +2450829|9877|2|44| +2450829|9878|2|349| +2450829|9880|2|212| +2450829|9883|2|400| +2450829|9884|2|99| +2450829|9886|2|276| +2450829|9889|2|357| +2450829|9890|2|90| +2450829|9892|2|602| +2450829|9895|2|899| +2450829|9896|2|87| +2450829|9898|2|751| +2450829|9901|2|466| +2450829|9902|2|436| +2450829|9904|2|| +2450829|9907|2|964| +2450829|9908|2|11| +2450829|9910|2|795| +2450829|9913|2|100| +2450829|9914|2|155| +2450829|9916|2|364| +2450829|9919|2|236| +2450829|9920|2|727| +2450829|9922|2|62| +2450829|9925|2|608| +2450829|9926|2|716| +2450829|9928|2|571| +2450829|9931|2|250| +2450829|9932|2|| +2450829|9934|2|449| +2450829|9937|2|462| +2450829|9938|2|305| +2450829|9940|2|128| +2450829|9943|2|164| +2450829|9944|2|602| +2450829|9946|2|311| +2450829|9949|2|341| +2450829|9950|2|20| +2450829|9952|2|573| +2450829|9955|2|326| +2450829|9956|2|968| +2450829|9958|2|428| +2450829|9961|2|328| +2450829|9962|2|868| +2450829|9964|2|788| +2450829|9967|2|273| +2450829|9968|2|38| +2450829|9970|2|706| +2450829|9973|2|773| +2450829|9974|2|864| +2450829|9976|2|373| +2450829|9979|2|372| +2450829|9980|2|243| +2450829|9982|2|479| +2450829|9985|2|| +2450829|9986|2|296| +2450829|9988|2|| +2450829|9991|2|633| +2450829|9992|2|742| +2450829|9994|2|904| +2450829|9997|2|| +2450829|9998|2|932| +2450829|10000|2|820| +2450829|10003|2|148| +2450829|10004|2|2| +2450829|10006|2|836| +2450829|10009|2|992| +2450829|10010|2|948| +2450829|10012|2|415| +2450829|10015|2|506| +2450829|10016|2|| +2450829|10018|2|897| +2450829|10021|2|918| +2450829|10022|2|109| +2450829|10024|2|653| +2450829|10027|2|440| +2450829|10028|2|709| +2450829|10030|2|430| +2450829|10033|2|446| +2450829|10034|2|309| +2450829|10036|2|902| +2450829|10039|2|470| +2450829|10040|2|359| +2450829|10042|2|129| +2450829|10045|2|452| +2450829|10046|2|237| +2450829|10048|2|134| +2450829|10051|2|92| +2450829|10052|2|742| +2450829|10054|2|633| +2450829|10057|2|139| +2450829|10058|2|252| +2450829|10060|2|743| +2450829|10063|2|328| +2450829|10064|2|879| +2450829|10066|2|492| +2450829|10069|2|694| +2450829|10070|2|500| +2450829|10072|2|998| +2450829|10075|2|621| +2450829|10076|2|735| +2450829|10078|2|181| +2450829|10081|2|916| +2450829|10082|2|174| +2450829|10084|2|630| +2450829|10087|2|| +2450829|10088|2|964| +2450829|10090|2|277| +2450829|10093|2|246| +2450829|10094|2|160| +2450829|10096|2|441| +2450829|10099|2|825| +2450829|10100|2|| +2450829|10102|2|678| +2450829|10105|2|663| +2450829|10106|2|551| +2450829|10108|2|220| +2450829|10111|2|714| +2450829|10112|2|376| +2450829|10114|2|334| +2450829|10117|2|730| +2450829|10118|2|610| +2450829|10120|2|967| +2450829|10123|2|763| +2450829|10124|2|830| +2450829|10126|2|45| +2450829|10129|2|601| +2450829|10130|2|| +2450829|10132|2|605| +2450829|10135|2|628| +2450829|10136|2|669| +2450829|10138|2|950| +2450829|10141|2|847| +2450829|10142|2|785| +2450829|10144|2|249| +2450829|10147|2|832| +2450829|10148|2|155| +2450829|10150|2|552| +2450829|10153|2|87| +2450829|10154|2|931| +2450829|10156|2|| +2450829|10159|2|67| +2450829|10160|2|294| +2450829|10162|2|828| +2450829|10165|2|54| +2450829|10166|2|69| +2450829|10168|2|957| +2450829|10171|2|593| +2450829|10172|2|954| +2450829|10174|2|302| +2450829|10177|2|679| +2450829|10178|2|925| +2450829|10180|2|397| +2450829|10183|2|420| +2450829|10184|2|472| +2450829|10186|2|445| +2450829|10189|2|| +2450829|10190|2|539| +2450829|10192|2|809| +2450829|10195|2|378| +2450829|10196|2|238| +2450829|10198|2|743| +2450829|10201|2|623| +2450829|10202|2|327| +2450829|10204|2|496| +2450829|10207|2|806| +2450829|10208|2|| +2450829|10210|2|692| +2450829|10213|2|642| +2450829|10214|2|785| +2450829|10216|2|156| +2450829|10219|2|109| +2450829|10220|2|341| +2450829|10222|2|| +2450829|10225|2|790| +2450829|10226|2|990| +2450829|10228|2|| +2450829|10231|2|| +2450829|10232|2|910| +2450829|10234|2|296| +2450829|10237|2|723| +2450829|10238|2|115| +2450829|10240|2|199| +2450829|10243|2|756| +2450829|10244|2|215| +2450829|10246|2|202| +2450829|10249|2|127| +2450829|10250|2|954| +2450829|10252|2|67| +2450829|10255|2|936| +2450829|10256|2|395| +2450829|10258|2|714| +2450829|10261|2|889| +2450829|10262|2|835| +2450829|10264|2|963| +2450829|10267|2|829| +2450829|10268|2|15| +2450829|10270|2|862| +2450829|10273|2|640| +2450829|10274|2|786| +2450829|10276|2|61| +2450829|10279|2|226| +2450829|10280|2|984| +2450829|10282|2|310| +2450829|10285|2|193| +2450829|10286|2|392| +2450829|10288|2|346| +2450829|10291|2|448| +2450829|10292|2|615| +2450829|10294|2|475| +2450829|10297|2|929| +2450829|10298|2|919| +2450829|10300|2|12| +2450829|10303|2|136| +2450829|10304|2|883| +2450829|10306|2|963| +2450829|10309|2|842| +2450829|10310|2|749| +2450829|10312|2|650| +2450829|10315|2|| +2450829|10316|2|94| +2450829|10318|2|482| +2450829|10321|2|833| +2450829|10322|2|544| +2450829|10324|2|552| +2450829|10327|2|208| +2450829|10328|2|753| +2450829|10330|2|973| +2450829|10333|2|737| +2450829|10334|2|812| +2450829|10336|2|566| +2450829|10339|2|669| +2450829|10340|2|745| +2450829|10342|2|881| +2450829|10345|2|655| +2450829|10346|2|594| +2450829|10348|2|61| +2450829|10351|2|285| +2450829|10352|2|511| +2450829|10354|2|395| +2450829|10357|2|141| +2450829|10358|2|429| +2450829|10360|2|151| +2450829|10363|2|271| +2450829|10364|2|95| +2450829|10366|2|665| +2450829|10369|2|810| +2450829|10370|2|943| +2450829|10372|2|283| +2450829|10375|2|133| +2450829|10376|2|127| +2450829|10378|2|914| +2450829|10381|2|194| +2450829|10382|2|978| +2450829|10384|2|905| +2450829|10387|2|429| +2450829|10388|2|77| +2450829|10390|2|567| +2450829|10393|2|563| +2450829|10394|2|835| +2450829|10396|2|475| +2450829|10399|2|563| +2450829|10400|2|196| +2450829|10402|2|759| +2450829|10405|2|547| +2450829|10406|2|726| +2450829|10408|2|726| +2450829|10411|2|383| +2450829|10412|2|488| +2450829|10414|2|667| +2450829|10417|2|114| +2450829|10418|2|881| +2450829|10420|2|884| +2450829|10423|2|327| +2450829|10424|2|929| +2450829|10426|2|826| +2450829|10429|2|32| +2450829|10430|2|538| +2450829|10432|2|679| +2450829|10435|2|893| +2450829|10436|2|629| +2450829|10438|2|6| +2450829|10441|2|411| +2450829|10442|2|999| +2450829|10444|2|170| +2450829|10447|2|686| +2450829|10448|2|243| +2450829|10450|2|438| +2450829|10453|2|950| +2450829|10454|2|483| +2450829|10456|2|911| +2450829|10459|2|365| +2450829|10460|2|683| +2450829|10462|2|24| +2450829|10465|2|837| +2450829|10466|2|656| +2450829|10468|2|174| +2450829|10471|2|980| +2450829|10472|2|737| +2450829|10474|2|285| +2450829|10477|2|134| +2450829|10478|2|331| +2450829|10480|2|130| +2450829|10483|2|12| +2450829|10484|2|707| +2450829|10486|2|141| +2450829|10489|2|225| +2450829|10490|2|617| +2450829|10492|2|428| +2450829|10495|2|345| +2450829|10496|2|861| +2450829|10498|2|52| +2450829|10501|2|50| +2450829|10502|2|816| +2450829|10504|2|253| +2450829|10507|2|610| +2450829|10508|2|22| +2450829|10510|2|431| +2450829|10513|2|175| +2450829|10514|2|122| +2450829|10516|2|526| +2450829|10519|2|34| +2450829|10520|2|74| +2450829|10522|2|433| +2450829|10525|2|954| +2450829|10526|2|453| +2450829|10528|2|452| +2450829|10531|2|312| +2450829|10532|2|413| +2450829|10534|2|843| +2450829|10537|2|771| +2450829|10538|2|162| +2450829|10540|2|684| +2450829|10543|2|533| +2450829|10544|2|840| +2450829|10546|2|| +2450829|10549|2|769| +2450829|10550|2|573| +2450829|10552|2|119| +2450829|10555|2|438| +2450829|10556|2|820| +2450829|10558|2|515| +2450829|10561|2|342| +2450829|10562|2|97| +2450829|10564|2|129| +2450829|10567|2|255| +2450829|10568|2|671| +2450829|10570|2|872| +2450829|10573|2|493| +2450829|10574|2|373| +2450829|10576|2|266| +2450829|10579|2|105| +2450829|10580|2|446| +2450829|10582|2|906| +2450829|10585|2|907| +2450829|10586|2|212| +2450829|10588|2|662| +2450829|10591|2|259| +2450829|10592|2|709| +2450829|10594|2|741| +2450829|10597|2|113| +2450829|10598|2|683| +2450829|10600|2|958| +2450829|10603|2|148| +2450829|10604|2|63| +2450829|10606|2|139| +2450829|10609|2|| +2450829|10610|2|988| +2450829|10612|2|862| +2450829|10615|2|72| +2450829|10616|2|807| +2450829|10618|2|297| +2450829|10621|2|455| +2450829|10622|2|995| +2450829|10624|2|393| +2450829|10627|2|789| +2450829|10628|2|119| +2450829|10630|2|759| +2450829|10633|2|309| +2450829|10634|2|285| +2450829|10636|2|123| +2450829|10639|2|362| +2450829|10640|2|401| +2450829|10642|2|495| +2450829|10645|2|474| +2450829|10646|2|245| +2450829|10648|2|216| +2450829|10651|2|885| +2450829|10652|2|509| +2450829|10654|2|714| +2450829|10657|2|227| +2450829|10658|2|380| +2450829|10660|2|732| +2450829|10663|2|937| +2450829|10664|2|48| +2450829|10666|2|560| +2450829|10669|2|624| +2450829|10670|2|284| +2450829|10672|2|985| +2450829|10675|2|874| +2450829|10676|2|407| +2450829|10678|2|271| +2450829|10681|2|862| +2450829|10682|2|860| +2450829|10684|2|| +2450829|10687|2|990| +2450829|10688|2|847| +2450829|10690|2|113| +2450829|10693|2|635| +2450829|10694|2|947| +2450829|10696|2|693| +2450829|10699|2|949| +2450829|10700|2|674| +2450829|10702|2|743| +2450829|10705|2|509| +2450829|10706|2|725| +2450829|10708|2|843| +2450829|10711|2|829| +2450829|10712|2|130| +2450829|10714|2|| +2450829|10717|2|403| +2450829|10718|2|862| +2450829|10720|2|906| +2450829|10723|2|635| +2450829|10724|2|66| +2450829|10726|2|617| +2450829|10729|2|895| +2450829|10730|2|69| +2450829|10732|2|803| +2450829|10735|2|995| +2450829|10736|2|640| +2450829|10738|2|678| +2450829|10741|2|87| +2450829|10742|2|| +2450829|10744|2|536| +2450829|10747|2|20| +2450829|10748|2|281| +2450829|10750|2|661| +2450829|10753|2|117| +2450829|10754|2|916| +2450829|10756|2|741| +2450829|10759|2|362| +2450829|10760|2|849| +2450829|10762|2|351| +2450829|10765|2|892| +2450829|10766|2|| +2450829|10768|2|916| +2450829|10771|2|137| +2450829|10772|2|218| +2450829|10774|2|243| +2450829|10777|2|590| +2450829|10778|2|758| +2450829|10780|2|746| +2450829|10783|2|677| +2450829|10784|2|190| +2450829|10786|2|543| +2450829|10789|2|667| +2450829|10790|2|390| +2450829|10792|2|638| +2450829|10795|2|871| +2450829|10796|2|488| +2450829|10798|2|| +2450829|10801|2|531| +2450829|10802|2|190| +2450829|10804|2|491| +2450829|10807|2|42| +2450829|10808|2|362| +2450829|10810|2|56| +2450829|10813|2|123| +2450829|10814|2|451| +2450829|10816|2|877| +2450829|10819|2|485| +2450829|10820|2|459| +2450829|10822|2|223| +2450829|10825|2|126| +2450829|10826|2|796| +2450829|10828|2|379| +2450829|10831|2|822| +2450829|10832|2|422| +2450829|10834|2|563| +2450829|10837|2|751| +2450829|10838|2|836| +2450829|10840|2|787| +2450829|10843|2|702| +2450829|10844|2|6| +2450829|10846|2|644| +2450829|10849|2|836| +2450829|10850|2|719| +2450829|10852|2|552| +2450829|10855|2|485| +2450829|10856|2|650| +2450829|10858|2|811| +2450829|10861|2|65| +2450829|10862|2|| +2450829|10864|2|915| +2450829|10867|2|749| +2450829|10868|2|227| +2450829|10870|2|196| +2450829|10873|2|283| +2450829|10874|2|| +2450829|10876|2|181| +2450829|10879|2|912| +2450829|10880|2|405| +2450829|10882|2|431| +2450829|10885|2|191| +2450829|10886|2|135| +2450829|10888|2|412| +2450829|10891|2|497| +2450829|10892|2|775| +2450829|10894|2|793| +2450829|10897|2|519| +2450829|10898|2|352| +2450829|10900|2|895| +2450829|10903|2|131| +2450829|10904|2|213| +2450829|10906|2|712| +2450829|10909|2|398| +2450829|10910|2|825| +2450829|10912|2|618| +2450829|10915|2|397| +2450829|10916|2|960| +2450829|10918|2|497| +2450829|10921|2|106| +2450829|10922|2|290| +2450829|10924|2|645| +2450829|10927|2|775| +2450829|10928|2|142| +2450829|10930|2|899| +2450829|10933|2|9| +2450829|10934|2|273| +2450829|10936|2|618| +2450829|10939|2|453| +2450829|10940|2|633| +2450829|10942|2|642| +2450829|10945|2|785| +2450829|10946|2|33| +2450829|10948|2|389| +2450829|10951|2|66| +2450829|10952|2|| +2450829|10954|2|391| +2450829|10957|2|347| +2450829|10958|2|341| +2450829|10960|2|565| +2450829|10963|2|123| +2450829|10964|2|844| +2450829|10966|2|271| +2450829|10969|2|519| +2450829|10970|2|| +2450829|10972|2|59| +2450829|10975|2|421| +2450829|10976|2|738| +2450829|10978|2|77| +2450829|10981|2|343| +2450829|10982|2|10| +2450829|10984|2|10| +2450829|10987|2|757| +2450829|10988|2|645| +2450829|10990|2|167| +2450829|10993|2|398| +2450829|10994|2|321| +2450829|10996|2|165| +2450829|10999|2|932| +2450829|11000|2|| +2450829|11002|2|479| +2450829|11005|2|255| +2450829|11006|2|366| +2450829|11008|2|146| +2450829|11011|2|387| +2450829|11012|2|536| +2450829|11014|2|439| +2450829|11017|2|426| +2450829|11018|2|0| +2450829|11020|2|137| +2450829|11023|2|36| +2450829|11024|2|513| +2450829|11026|2|143| +2450829|11029|2|224| +2450829|11030|2|540| +2450829|11032|2|731| +2450829|11035|2|307| +2450829|11036|2|27| +2450829|11038|2|51| +2450829|11041|2|339| +2450829|11042|2|118| +2450829|11044|2|692| +2450829|11047|2|116| +2450829|11048|2|566| +2450829|11050|2|5| +2450829|11053|2|398| +2450829|11054|2|402| +2450829|11056|2|853| +2450829|11059|2|424| +2450829|11060|2|628| +2450829|11062|2|157| +2450829|11065|2|771| +2450829|11066|2|| +2450829|11068|2|406| +2450829|11071|2|308| +2450829|11072|2|77| +2450829|11074|2|529| +2450829|11077|2|548| +2450829|11078|2|951| +2450829|11080|2|279| +2450829|11083|2|827| +2450829|11084|2|138| +2450829|11086|2|343| +2450829|11089|2|708| +2450829|11090|2|906| +2450829|11092|2|182| +2450829|11095|2|37| +2450829|11096|2|911| +2450829|11098|2|426| +2450829|11101|2|610| +2450829|11102|2|920| +2450829|11104|2|560| +2450829|11107|2|44| +2450829|11108|2|934| +2450829|11110|2|978| +2450829|11113|2|168| +2450829|11114|2|959| +2450829|11116|2|466| +2450829|11119|2|31| +2450829|11120|2|920| +2450829|11122|2|560| +2450829|11125|2|| +2450829|11126|2|890| +2450829|11128|2|557| +2450829|11131|2|644| +2450829|11132|2|648| +2450829|11134|2|568| +2450829|11137|2|855| +2450829|11138|2|225| +2450829|11140|2|358| +2450829|11143|2|913| +2450829|11144|2|29| +2450829|11146|2|609| +2450829|11149|2|734| +2450829|11150|2|84| +2450829|11152|2|106| +2450829|11155|2|463| +2450829|11156|2|818| +2450829|11158|2|125| +2450829|11161|2|| +2450829|11162|2|379| +2450829|11164|2|933| +2450829|11167|2|758| +2450829|11168|2|372| +2450829|11170|2|28| +2450829|11173|2|711| +2450829|11174|2|231| +2450829|11176|2|187| +2450829|11179|2|652| +2450829|11180|2|| +2450829|11182|2|965| +2450829|11185|2|878| +2450829|11186|2|209| +2450829|11188|2|269| +2450829|11191|2|655| +2450829|11192|2|59| +2450829|11194|2|846| +2450829|11197|2|| +2450829|11198|2|649| +2450829|11200|2|633| +2450829|11203|2|127| +2450829|11204|2|27| +2450829|11206|2|712| +2450829|11209|2|77| +2450829|11210|2|54| +2450829|11212|2|301| +2450829|11215|2|678| +2450829|11216|2|963| +2450829|11218|2|762| +2450829|11221|2|909| +2450829|11222|2|192| +2450829|11224|2|500| +2450829|11227|2|370| +2450829|11228|2|363| +2450829|11230|2|327| +2450829|11233|2|518| +2450829|11234|2|294| +2450829|11236|2|942| +2450829|11239|2|231| +2450829|11240|2|175| +2450829|11242|2|334| +2450829|11245|2|828| +2450829|11246|2|| +2450829|11248|2|461| +2450829|11251|2|720| +2450829|11252|2|374| +2450829|11254|2|84| +2450829|11257|2|686| +2450829|11258|2|460| +2450829|11260|2|325| +2450829|11263|2|809| +2450829|11264|2|| +2450829|11266|2|520| +2450829|11269|2|721| +2450829|11270|2|570| +2450829|11272|2|758| +2450829|11275|2|307| +2450829|11276|2|720| +2450829|11278|2|779| +2450829|11281|2|514| +2450829|11282|2|600| +2450829|11284|2|779| +2450829|11287|2|690| +2450829|11288|2|533| +2450829|11290|2|219| +2450829|11293|2|500| +2450829|11294|2|461| +2450829|11296|2|707| +2450829|11299|2|855| +2450829|11300|2|833| +2450829|11302|2|710| +2450829|11305|2|290| +2450829|11306|2|454| +2450829|11308|2|217| +2450829|11311|2|578| +2450829|11312|2|400| +2450829|11314|2|264| +2450829|11317|2|603| +2450829|11318|2|626| +2450829|11320|2|366| +2450829|11323|2|| +2450829|11324|2|317| +2450829|11326|2|20| +2450829|11329|2|17| +2450829|11330|2|218| +2450829|11332|2|981| +2450829|11335|2|438| +2450829|11336|2|| +2450829|11338|2|85| +2450829|11341|2|220| +2450829|11342|2|612| +2450829|11344|2|961| +2450829|11347|2|854| +2450829|11348|2|556| +2450829|11350|2|347| +2450829|11353|2|850| +2450829|11354|2|49| +2450829|11356|2|625| +2450829|11359|2|547| +2450829|11360|2|501| +2450829|11362|2|186| +2450829|11365|2|352| +2450829|11366|2|664| +2450829|11368|2|110| +2450829|11371|2|764| +2450829|11372|2|91| +2450829|11374|2|431| +2450829|11377|2|792| +2450829|11378|2|419| +2450829|11380|2|555| +2450829|11383|2|787| +2450829|11384|2|320| +2450829|11386|2|706| +2450829|11389|2|929| +2450829|11390|2|334| +2450829|11392|2|778| +2450829|11395|2|844| +2450829|11396|2|217| +2450829|11398|2|538| +2450829|11401|2|625| +2450829|11402|2|226| +2450829|11404|2|687| +2450829|11407|2|638| +2450829|11408|2|725| +2450829|11410|2|338| +2450829|11413|2|867| +2450829|11414|2|775| +2450829|11416|2|611| +2450829|11419|2|930| +2450829|11420|2|163| +2450829|11422|2|232| +2450829|11425|2|496| +2450829|11426|2|23| +2450829|11428|2|326| +2450829|11431|2|214| +2450829|11432|2|92| +2450829|11434|2|50| +2450829|11437|2|742| +2450829|11438|2|773| +2450829|11440|2|544| +2450829|11443|2|| +2450829|11444|2|117| +2450829|11446|2|504| +2450829|11449|2|760| +2450829|11450|2|873| +2450829|11452|2|513| +2450829|11455|2|344| +2450829|11456|2|253| +2450829|11458|2|952| +2450829|11461|2|378| +2450829|11462|2|183| +2450829|11464|2|390| +2450829|11467|2|208| +2450829|11468|2|415| +2450829|11470|2|195| +2450829|11473|2|122| +2450829|11474|2|940| +2450829|11476|2|653| +2450829|11479|2|130| +2450829|11480|2|| +2450829|11482|2|397| +2450829|11485|2|964| +2450829|11486|2|| +2450829|11488|2|527| +2450829|11491|2|68| +2450829|11492|2|189| +2450829|11494|2|197| +2450829|11497|2|821| +2450829|11498|2|550| +2450829|11500|2|520| +2450829|11503|2|68| +2450829|11504|2|334| +2450829|11506|2|| +2450829|11509|2|495| +2450829|11510|2|509| +2450829|11512|2|582| +2450829|11515|2|891| +2450829|11516|2|590| +2450829|11518|2|118| +2450829|11521|2|809| +2450829|11522|2|384| +2450829|11524|2|461| +2450829|11527|2|823| +2450829|11528|2|999| +2450829|11530|2|784| +2450829|11533|2|594| +2450829|11534|2|| +2450829|11536|2|382| +2450829|11539|2|215| +2450829|11540|2|617| +2450829|11542|2|699| +2450829|11545|2|597| +2450829|11546|2|432| +2450829|11548|2|282| +2450829|11551|2|540| +2450829|11552|2|216| +2450829|11554|2|425| +2450829|11557|2|675| +2450829|11558|2|307| +2450829|11560|2|265| +2450829|11563|2|337| +2450829|11564|2|221| +2450829|11566|2|268| +2450829|11569|2|| +2450829|11570|2|| +2450829|11572|2|282| +2450829|11575|2|924| +2450829|11576|2|| +2450829|11578|2|783| +2450829|11581|2|405| +2450829|11582|2|379| +2450829|11584|2|871| +2450829|11587|2|351| +2450829|11588|2|319| +2450829|11590|2|58| +2450829|11593|2|385| +2450829|11594|2|902| +2450829|11596|2|114| +2450829|11599|2|924| +2450829|11600|2|601| +2450829|11602|2|877| +2450829|11605|2|145| +2450829|11606|2|275| +2450829|11608|2|459| +2450829|11611|2|991| +2450829|11612|2|837| +2450829|11614|2|| +2450829|11617|2|139| +2450829|11618|2|909| +2450829|11620|2|660| +2450829|11623|2|307| +2450829|11624|2|340| +2450829|11626|2|439| +2450829|11629|2|85| +2450829|11630|2|222| +2450829|11632|2|377| +2450829|11635|2|727| +2450829|11636|2|964| +2450829|11638|2|93| +2450829|11641|2|407| +2450829|11642|2|407| +2450829|11644|2|405| +2450829|11647|2|86| +2450829|11648|2|887| +2450829|11650|2|816| +2450829|11653|2|362| +2450829|11654|2|| +2450829|11656|2|806| +2450829|11659|2|141| +2450829|11660|2|370| +2450829|11662|2|307| +2450829|11665|2|732| +2450829|11666|2|567| +2450829|11668|2|219| +2450829|11671|2|483| +2450829|11672|2|736| +2450829|11674|2|| +2450829|11677|2|280| +2450829|11678|2|239| +2450829|11680|2|337| +2450829|11683|2|128| +2450829|11684|2|259| +2450829|11686|2|309| +2450829|11689|2|99| +2450829|11690|2|175| +2450829|11692|2|685| +2450829|11695|2|863| +2450829|11696|2|916| +2450829|11698|2|527| +2450829|11701|2|95| +2450829|11702|2|130| +2450829|11704|2|94| +2450829|11707|2|39| +2450829|11708|2|397| +2450829|11710|2|703| +2450829|11713|2|389| +2450829|11714|2|623| +2450829|11716|2|685| +2450829|11719|2|346| +2450829|11720|2|398| +2450829|11722|2|626| +2450829|11725|2|277| +2450829|11726|2|80| +2450829|11728|2|251| +2450829|11731|2|861| +2450829|11732|2|740| +2450829|11734|2|121| +2450829|11737|2|730| +2450829|11738|2|202| +2450829|11740|2|59| +2450829|11743|2|274| +2450829|11744|2|604| +2450829|11746|2|6| +2450829|11749|2|183| +2450829|11750|2|751| +2450829|11752|2|64| +2450829|11755|2|851| +2450829|11756|2|951| +2450829|11758|2|543| +2450829|11761|2|827| +2450829|11762|2|| +2450829|11764|2|345| +2450829|11767|2|387| +2450829|11768|2|20| +2450829|11770|2|600| +2450829|11773|2|732| +2450829|11774|2|980| +2450829|11776|2|726| +2450829|11779|2|| +2450829|11780|2|71| +2450829|11782|2|| +2450829|11785|2|618| +2450829|11786|2|596| +2450829|11788|2|662| +2450829|11791|2|227| +2450829|11792|2|762| +2450829|11794|2|791| +2450829|11797|2|950| +2450829|11798|2|440| +2450829|11800|2|108| +2450829|11803|2|699| +2450829|11804|2|938| +2450829|11806|2|817| +2450829|11809|2|171| +2450829|11810|2|182| +2450829|11812|2|| +2450829|11815|2|845| +2450829|11816|2|25| +2450829|11818|2|967| +2450829|11821|2|595| +2450829|11822|2|2| +2450829|11824|2|81| +2450829|11827|2|994| +2450829|11828|2|135| +2450829|11830|2|69| +2450829|11833|2|27| +2450829|11834|2|352| +2450829|11836|2|407| +2450829|11839|2|386| +2450829|11840|2|900| +2450829|11842|2|730| +2450829|11845|2|749| +2450829|11846|2|999| +2450829|11848|2|271| +2450829|11851|2|966| +2450829|11852|2|697| +2450829|11854|2|800| +2450829|11857|2|65| +2450829|11858|2|639| +2450829|11860|2|368| +2450829|11863|2|870| +2450829|11864|2|520| +2450829|11866|2|268| +2450829|11869|2|| +2450829|11870|2|150| +2450829|11872|2|554| +2450829|11875|2|763| +2450829|11876|2|268| +2450829|11878|2|944| +2450829|11881|2|171| +2450829|11882|2|665| +2450829|11884|2|808| +2450829|11887|2|628| +2450829|11888|2|998| +2450829|11890|2|241| +2450829|11893|2|404| +2450829|11894|2|450| +2450829|11896|2|878| +2450829|11899|2|708| +2450829|11900|2|401| +2450829|11902|2|662| +2450829|11905|2|| +2450829|11906|2|300| +2450829|11908|2|638| +2450829|11911|2|271| +2450829|11912|2|337| +2450829|11914|2|78| +2450829|11917|2|917| +2450829|11918|2|831| +2450829|11920|2|132| +2450829|11923|2|220| +2450829|11924|2|992| +2450829|11926|2|67| +2450829|11929|2|| +2450829|11930|2|417| +2450829|11932|2|907| +2450829|11935|2|860| +2450829|11936|2|917| +2450829|11938|2|976| +2450829|11941|2|62| +2450829|11942|2|643| +2450829|11944|2|655| +2450829|11947|2|433| +2450829|11948|2|769| +2450829|11950|2|136| +2450829|11953|2|532| +2450829|11954|2|744| +2450829|11956|2|994| +2450829|11959|2|113| +2450829|11960|2|943| +2450829|11962|2|291| +2450829|11965|2|| +2450829|11966|2|4| +2450829|11968|2|170| +2450829|11971|2|78| +2450829|11972|2|69| +2450829|11974|2|41| +2450829|11977|2|30| +2450829|11978|2|186| +2450829|11980|2|160| +2450829|11983|2|261| +2450829|11984|2|540| +2450829|11986|2|939| +2450829|11989|2|444| +2450829|11990|2|446| +2450829|11992|2|597| +2450829|11995|2|926| +2450829|11996|2|134| +2450829|11998|2|776| +2450829|12001|2|489| +2450829|12002|2|313| +2450829|12004|2|725| +2450829|12007|2|308| +2450829|12008|2|113| +2450829|12010|2|333| +2450829|12013|2|956| +2450829|12014|2|550| +2450829|12016|2|943| +2450829|12019|2|487| +2450829|12020|2|902| +2450829|12022|2|529| +2450829|12025|2|119| +2450829|12026|2|223| +2450829|12028|2|700| +2450829|12031|2|307| +2450829|12032|2|926| +2450829|12034|2|395| +2450829|12037|2|339| +2450829|12038|2|162| +2450829|12040|2|243| +2450829|12043|2|596| +2450829|12044|2|785| +2450829|12046|2|887| +2450829|12049|2|858| +2450829|12050|2|183| +2450829|12052|2|741| +2450829|12055|2|80| +2450829|12056|2|389| +2450829|12058|2|869| +2450829|12061|2|478| +2450829|12062|2|84| +2450829|12064|2|989| +2450829|12067|2|532| +2450829|12068|2|334| +2450829|12070|2|532| +2450829|12073|2|977| +2450829|12074|2|817| +2450829|12076|2|214| +2450829|12079|2|473| +2450829|12080|2|508| +2450829|12082|2|117| +2450829|12085|2|107| +2450829|12086|2|| +2450829|12088|2|185| +2450829|12091|2|393| +2450829|12092|2|408| +2450829|12094|2|513| +2450829|12097|2|829| +2450829|12098|2|796| +2450829|12100|2|| +2450829|12103|2|317| +2450829|12104|2|443| +2450829|12106|2|551| +2450829|12109|2|236| +2450829|12110|2|569| +2450829|12112|2|127| +2450829|12115|2|442| +2450829|12116|2|604| +2450829|12118|2|596| +2450829|12121|2|599| +2450829|12122|2|219| +2450829|12124|2|470| +2450829|12127|2|278| +2450829|12128|2|277| +2450829|12130|2|334| +2450829|12133|2|254| +2450829|12134|2|299| +2450829|12136|2|789| +2450829|12139|2|903| +2450829|12140|2|6| +2450829|12142|2|253| +2450829|12145|2|850| +2450829|12146|2|188| +2450829|12148|2|604| +2450829|12151|2|360| +2450829|12152|2|957| +2450829|12154|2|178| +2450829|12157|2|790| +2450829|12158|2|110| +2450829|12160|2|927| +2450829|12163|2|810| +2450829|12164|2|262| +2450829|12166|2|234| +2450829|12169|2|21| +2450829|12170|2|737| +2450829|12172|2|504| +2450829|12175|2|984| +2450829|12176|2|30| +2450829|12178|2|838| +2450829|12181|2|929| +2450829|12182|2|182| +2450829|12184|2|420| +2450829|12187|2|663| +2450829|12188|2|846| +2450829|12190|2|605| +2450829|12193|2|839| +2450829|12194|2|611| +2450829|12196|2|963| +2450829|12199|2|637| +2450829|12200|2|917| +2450829|12202|2|828| +2450829|12205|2|480| +2450829|12206|2|525| +2450829|12208|2|201| +2450829|12211|2|120| +2450829|12212|2|183| +2450829|12214|2|524| +2450829|12217|2|855| +2450829|12218|2|559| +2450829|12220|2|799| +2450829|12223|2|64| +2450829|12224|2|19| +2450829|12226|2|463| +2450829|12229|2|761| +2450829|12230|2|160| +2450829|12232|2|781| +2450829|12235|2|768| +2450829|12236|2|216| +2450829|12238|2|208| +2450829|12241|2|823| +2450829|12242|2|160| +2450829|12244|2|930| +2450829|12247|2|56| +2450829|12248|2|260| +2450829|12250|2|666| +2450829|12253|2|915| +2450829|12254|2|596| +2450829|12256|2|888| +2450829|12259|2|| +2450829|12260|2|138| +2450829|12262|2|86| +2450829|12265|2|306| +2450829|12266|2|818| +2450829|12268|2|200| +2450829|12271|2|878| +2450829|12272|2|493| +2450829|12274|2|913| +2450829|12277|2|778| +2450829|12278|2|957| +2450829|12280|2|789| +2450829|12283|2|38| +2450829|12284|2|468| +2450829|12286|2|361| +2450829|12289|2|577| +2450829|12290|2|408| +2450829|12292|2|703| +2450829|12295|2|674| +2450829|12296|2|574| +2450829|12298|2|27| +2450829|12301|2|140| +2450829|12302|2|14| +2450829|12304|2|606| +2450829|12307|2|| +2450829|12308|2|493| +2450829|12310|2|449| +2450829|12313|2|0| +2450829|12314|2|91| +2450829|12316|2|221| +2450829|12319|2|143| +2450829|12320|2|999| +2450829|12322|2|715| +2450829|12325|2|436| +2450829|12326|2|801| +2450829|12328|2|23| +2450829|12331|2|735| +2450829|12332|2|219| +2450829|12334|2|189| +2450829|12337|2|956| +2450829|12338|2|637| +2450829|12340|2|725| +2450829|12343|2|610| +2450829|12344|2|304| +2450829|12346|2|963| +2450829|12349|2|850| +2450829|12350|2|544| +2450829|12352|2|794| +2450829|12355|2|236| +2450829|12356|2|62| +2450829|12358|2|377| +2450829|12361|2|213| +2450829|12362|2|715| +2450829|12364|2|421| +2450829|12367|2|789| +2450829|12368|2|659| +2450829|12370|2|796| +2450829|12373|2|91| +2450829|12374|2|613| +2450829|12376|2|794| +2450829|12379|2|976| +2450829|12380|2|904| +2450829|12382|2|599| +2450829|12385|2|963| +2450829|12386|2|973| +2450829|12388|2|428| +2450829|12391|2|225| +2450829|12392|2|136| +2450829|12394|2|349| +2450829|12397|2|977| +2450829|12398|2|255| +2450829|12400|2|403| +2450829|12403|2|601| +2450829|12404|2|59| +2450829|12406|2|259| +2450829|12409|2|606| +2450829|12410|2|781| +2450829|12412|2|889| +2450829|12415|2|957| +2450829|12416|2|254| +2450829|12418|2|447| +2450829|12421|2|601| +2450829|12422|2|613| +2450829|12424|2|350| +2450829|12427|2|294| +2450829|12428|2|384| +2450829|12430|2|992| +2450829|12433|2|694| +2450829|12434|2|385| +2450829|12436|2|578| +2450829|12439|2|564| +2450829|12440|2|926| +2450829|12442|2|903| +2450829|12445|2|537| +2450829|12446|2|463| +2450829|12448|2|489| +2450829|12451|2|| +2450829|12452|2|865| +2450829|12454|2|998| +2450829|12457|2|204| +2450829|12458|2|705| +2450829|12460|2|497| +2450829|12463|2|155| +2450829|12464|2|596| +2450829|12466|2|352| +2450829|12469|2|485| +2450829|12470|2|734| +2450829|12472|2|945| +2450829|12475|2|328| +2450829|12476|2|80| +2450829|12478|2|141| +2450829|12481|2|830| +2450829|12482|2|265| +2450829|12484|2|72| +2450829|12487|2|619| +2450829|12488|2|155| +2450829|12490|2|194| +2450829|12493|2|166| +2450829|12494|2|536| +2450829|12496|2|967| +2450829|12499|2|967| +2450829|12500|2|78| +2450829|12502|2|315| +2450829|12505|2|377| +2450829|12506|2|593| +2450829|12508|2|68| +2450829|12511|2|| +2450829|12512|2|893| +2450829|12514|2|686| +2450829|12517|2|220| +2450829|12518|2|327| +2450829|12520|2|441| +2450829|12523|2|533| +2450829|12524|2|431| +2450829|12526|2|621| +2450829|12529|2|9| +2450829|12530|2|617| +2450829|12532|2|986| +2450829|12535|2|403| +2450829|12536|2|238| +2450829|12538|2|453| +2450829|12541|2|543| +2450829|12542|2|152| +2450829|12544|2|215| +2450829|12547|2|86| +2450829|12548|2|401| +2450829|12550|2|104| +2450829|12553|2|42| +2450829|12554|2|367| +2450829|12556|2|46| +2450829|12559|2|653| +2450829|12560|2|251| +2450829|12562|2|110| +2450829|12565|2|517| +2450829|12566|2|399| +2450829|12568|2|643| +2450829|12571|2|858| +2450829|12572|2|| +2450829|12574|2|390| +2450829|12577|2|953| +2450829|12578|2|455| +2450829|12580|2|998| +2450829|12583|2|599| +2450829|12584|2|774| +2450829|12586|2|131| +2450829|12589|2|599| +2450829|12590|2|945| +2450829|12592|2|90| +2450829|12595|2|178| +2450829|12596|2|311| +2450829|12598|2|1| +2450829|12601|2|82| +2450829|12602|2|941| +2450829|12604|2|48| +2450829|12607|2|821| +2450829|12608|2|999| +2450829|12610|2|622| +2450829|12613|2|441| +2450829|12614|2|514| +2450829|12616|2|711| +2450829|12619|2|448| +2450829|12620|2|261| +2450829|12622|2|349| +2450829|12625|2|876| +2450829|12626|2|808| +2450829|12628|2|852| +2450829|12631|2|726| +2450829|12632|2|429| +2450829|12634|2|382| +2450829|12637|2|859| +2450829|12638|2|638| +2450829|12640|2|239| +2450829|12643|2|549| +2450829|12644|2|939| +2450829|12646|2|885| +2450829|12649|2|681| +2450829|12650|2|64| +2450829|12652|2|483| +2450829|12655|2|| +2450829|12656|2|283| +2450829|12658|2|564| +2450829|12661|2|191| +2450829|12662|2|363| +2450829|12664|2|299| +2450829|12667|2|399| +2450829|12668|2|566| +2450829|12670|2|633| +2450829|12673|2|883| +2450829|12674|2|| +2450829|12676|2|206| +2450829|12679|2|849| +2450829|12680|2|415| +2450829|12682|2|111| +2450829|12685|2|525| +2450829|12686|2|686| +2450829|12688|2|85| +2450829|12691|2|938| +2450829|12692|2|619| +2450829|12694|2|970| +2450829|12697|2|471| +2450829|12698|2|25| +2450829|12700|2|290| +2450829|12703|2|372| +2450829|12704|2|36| +2450829|12706|2|737| +2450829|12709|2|341| +2450829|12710|2|549| +2450829|12712|2|322| +2450829|12715|2|| +2450829|12716|2|659| +2450829|12718|2|291| +2450829|12721|2|| +2450829|12722|2|824| +2450829|12724|2|879| +2450829|12727|2|47| +2450829|12728|2|548| +2450829|12730|2|53| +2450829|12733|2|511| +2450829|12734|2|619| +2450829|12736|2|112| +2450829|12739|2|48| +2450829|12740|2|624| +2450829|12742|2|749| +2450829|12745|2|250| +2450829|12746|2|721| +2450829|12748|2|655| +2450829|12751|2|528| +2450829|12752|2|620| +2450829|12754|2|734| +2450829|12757|2|860| +2450829|12758|2|17| +2450829|12760|2|924| +2450829|12763|2|346| +2450829|12764|2|666| +2450829|12766|2|941| +2450829|12769|2|837| +2450829|12770|2|834| +2450829|12772|2|354| +2450829|12775|2|875| +2450829|12776|2|908| +2450829|12778|2|677| +2450829|12781|2|645| +2450829|12782|2|491| +2450829|12784|2|705| +2450829|12787|2|978| +2450829|12788|2|243| +2450829|12790|2|| +2450829|12793|2|999| +2450829|12794|2|969| +2450829|12796|2|255| +2450829|12799|2|367| +2450829|12800|2|934| +2450829|12802|2|207| +2450829|12805|2|890| +2450829|12806|2|555| +2450829|12808|2|928| +2450829|12811|2|487| +2450829|12812|2|| +2450829|12814|2|646| +2450829|12817|2|335| +2450829|12818|2|907| +2450829|12820|2|298| +2450829|12823|2|931| +2450829|12824|2|828| +2450829|12826|2|474| +2450829|12829|2|898| +2450829|12830|2|796| +2450829|12832|2|974| +2450829|12835|2|407| +2450829|12836|2|601| +2450829|12838|2|470| +2450829|12841|2|3| +2450829|12842|2|423| +2450829|12844|2|320| +2450829|12847|2|839| +2450829|12848|2|118| +2450829|12850|2|817| +2450829|12853|2|631| +2450829|12854|2|| +2450829|12856|2|346| +2450829|12859|2|947| +2450829|12860|2|217| +2450829|12862|2|252| +2450829|12865|2|257| +2450829|12866|2|481| +2450829|12868|2|222| +2450829|12871|2|| +2450829|12872|2|682| +2450829|12874|2|340| +2450829|12877|2|598| +2450829|12878|2|957| +2450829|12880|2|224| +2450829|12883|2|984| +2450829|12884|2|807| +2450829|12886|2|60| +2450829|12889|2|550| +2450829|12890|2|436| +2450829|12892|2|878| +2450829|12895|2|861| +2450829|12896|2|698| +2450829|12898|2|492| +2450829|12901|2|293| +2450829|12902|2|292| +2450829|12904|2|968| +2450829|12907|2|961| +2450829|12908|2|974| +2450829|12910|2|696| +2450829|12913|2|| +2450829|12914|2|192| +2450829|12916|2|403| +2450829|12919|2|631| +2450829|12920|2|455| +2450829|12922|2|388| +2450829|12925|2|710| +2450829|12926|2|910| +2450829|12928|2|726| +2450829|12931|2|1000| +2450829|12932|2|159| +2450829|12934|2|602| +2450829|12937|2|948| +2450829|12938|2|194| +2450829|12940|2|792| +2450829|12943|2|277| +2450829|12944|2|| +2450829|12946|2|206| +2450829|12949|2|627| +2450829|12950|2|692| +2450829|12952|2|171| +2450829|12955|2|363| +2450829|12956|2|633| +2450829|12958|2|27| +2450829|12961|2|813| +2450829|12962|2|632| +2450829|12964|2|467| +2450829|12967|2|966| +2450829|12968|2|848| +2450829|12970|2|427| +2450829|12973|2|51| +2450829|12974|2|653| +2450829|12976|2|539| +2450829|12979|2|108| +2450829|12980|2|773| +2450829|12982|2|384| +2450829|12985|2|400| +2450829|12986|2|478| +2450829|12988|2|150| +2450829|12991|2|634| +2450829|12992|2|256| +2450829|12994|2|0| +2450829|12997|2|753| +2450829|12998|2|248| +2450829|13000|2|591| +2450829|13003|2|915| +2450829|13004|2|135| +2450829|13006|2|16| +2450829|13009|2|612| +2450829|13010|2|403| +2450829|13012|2|385| +2450829|13015|2|601| +2450829|13016|2|807| +2450829|13018|2|255| +2450829|13021|2|910| +2450829|13022|2|152| +2450829|13024|2|196| +2450829|13027|2|468| +2450829|13028|2|551| +2450829|13030|2|721| +2450829|13033|2|| +2450829|13034|2|723| +2450829|13036|2|475| +2450829|13039|2|881| +2450829|13040|2|863| +2450829|13042|2|206| +2450829|13045|2|741| +2450829|13046|2|705| +2450829|13048|2|23| +2450829|13051|2|318| +2450829|13052|2|350| +2450829|13054|2|728| +2450829|13057|2|224| +2450829|13058|2|618| +2450829|13060|2|248| +2450829|13063|2|315| +2450829|13064|2|467| +2450829|13066|2|387| +2450829|13069|2|717| +2450829|13070|2|636| +2450829|13072|2|443| +2450829|13075|2|184| +2450829|13076|2|350| +2450829|13078|2|186| +2450829|13081|2|176| +2450829|13082|2|708| +2450829|13084|2|606| +2450829|13087|2|575| +2450829|13088|2|911| +2450829|13090|2|319| +2450829|13093|2|225| +2450829|13094|2|537| +2450829|13096|2|151| +2450829|13099|2|280| +2450829|13100|2|421| +2450829|13102|2|666| +2450829|13105|2|801| +2450829|13106|2|261| +2450829|13108|2|| +2450829|13111|2|796| +2450829|13112|2|526| +2450829|13114|2|689| +2450829|13117|2|725| +2450829|13118|2|768| +2450829|13120|2|518| +2450829|13123|2|| +2450829|13124|2|134| +2450829|13126|2|995| +2450829|13129|2|650| +2450829|13130|2|307| +2450829|13132|2|58| +2450829|13135|2|29| +2450829|13136|2|902| +2450829|13138|2|123| +2450829|13141|2|756| +2450829|13142|2|516| +2450829|13144|2|222| +2450829|13147|2|168| +2450829|13148|2|| +2450829|13150|2|316| +2450829|13153|2|875| +2450829|13154|2|984| +2450829|13156|2|236| +2450829|13159|2|375| +2450829|13160|2|170| +2450829|13162|2|244| +2450829|13165|2|12| +2450829|13166|2|514| +2450829|13168|2|689| +2450829|13171|2|986| +2450829|13172|2|281| +2450829|13174|2|893| +2450829|13177|2|926| +2450829|13178|2|771| +2450829|13180|2|755| +2450829|13183|2|559| +2450829|13184|2|564| +2450829|13186|2|164| +2450829|13189|2|698| +2450829|13190|2|320| +2450829|13192|2|906| +2450829|13195|2|861| +2450829|13196|2|223| +2450829|13198|2|645| +2450829|13201|2|132| +2450829|13202|2|185| +2450829|13204|2|901| +2450829|13207|2|685| +2450829|13208|2|807| +2450829|13210|2|916| +2450829|13213|2|399| +2450829|13214|2|991| +2450829|13216|2|365| +2450829|13219|2|292| +2450829|13220|2|138| +2450829|13222|2|557| +2450829|13225|2|175| +2450829|13226|2|691| +2450829|13228|2|971| +2450829|13231|2|669| +2450829|13232|2|15| +2450829|13234|2|630| +2450829|13237|2|156| +2450829|13238|2|869| +2450829|13240|2|844| +2450829|13243|2|386| +2450829|13244|2|30| +2450829|13246|2|558| +2450829|13249|2|| +2450829|13250|2|919| +2450829|13252|2|135| +2450829|13255|2|453| +2450829|13256|2|77| +2450829|13258|2|| +2450829|13261|2|794| +2450829|13262|2|879| +2450829|13264|2|816| +2450829|13267|2|937| +2450829|13268|2|996| +2450829|13270|2|156| +2450829|13273|2|769| +2450829|13274|2|968| +2450829|13276|2|559| +2450829|13279|2|943| +2450829|13280|2|760| +2450829|13282|2|351| +2450829|13285|2|395| +2450829|13286|2|429| +2450829|13288|2|514| +2450829|13291|2|271| +2450829|13292|2|241| +2450829|13294|2|868| +2450829|13297|2|91| +2450829|13298|2|688| +2450829|13300|2|213| +2450829|13303|2|147| +2450829|13304|2|592| +2450829|13306|2|261| +2450829|13309|2|449| +2450829|13310|2|799| +2450829|13312|2|488| +2450829|13315|2|283| +2450829|13316|2|297| +2450829|13318|2|928| +2450829|13321|2|879| +2450829|13322|2|85| +2450829|13324|2|2| +2450829|13327|2|6| +2450829|13328|2|602| +2450829|13330|2|475| +2450829|13333|2|| +2450829|13334|2|0| +2450829|13336|2|151| +2450829|13339|2|| +2450829|13340|2|887| +2450829|13342|2|631| +2450829|13345|2|357| +2450829|13346|2|404| +2450829|13348|2|822| +2450829|13351|2|40| +2450829|13352|2|822| +2450829|13354|2|593| +2450829|13357|2|56| +2450829|13358|2|134| +2450829|13360|2|918| +2450829|13363|2|771| +2450829|13364|2|396| +2450829|13366|2|386| +2450829|13369|2|239| +2450829|13370|2|476| +2450829|13372|2|823| +2450829|13375|2|224| +2450829|13376|2|646| +2450829|13378|2|639| +2450829|13381|2|611| +2450829|13382|2|23| +2450829|13384|2|| +2450829|13387|2|980| +2450829|13388|2|144| +2450829|13390|2|964| +2450829|13393|2|968| +2450829|13394|2|442| +2450829|13396|2|314| +2450829|13399|2|794| +2450829|13400|2|99| +2450829|13402|2|742| +2450829|13405|2|821| +2450829|13406|2|826| +2450829|13408|2|558| +2450829|13411|2|884| +2450829|13412|2|475| +2450829|13414|2|| +2450829|13417|2|796| +2450829|13418|2|| +2450829|13420|2|688| +2450829|13423|2|875| +2450829|13424|2|308| +2450829|13426|2|| +2450829|13429|2|607| +2450829|13430|2|| +2450829|13432|2|598| +2450829|13435|2|290| +2450829|13436|2|702| +2450829|13438|2|631| +2450829|13441|2|997| +2450829|13442|2|962| +2450829|13444|2|445| +2450829|13447|2|577| +2450829|13448|2|809| +2450829|13450|2|| +2450829|13453|2|400| +2450829|13454|2|815| +2450829|13456|2|984| +2450829|13459|2|902| +2450829|13460|2|843| +2450829|13462|2|347| +2450829|13465|2|| +2450829|13466|2|483| +2450829|13468|2|610| +2450829|13471|2|308| +2450829|13472|2|854| +2450829|13474|2|97| +2450829|13477|2|505| +2450829|13478|2|837| +2450829|13480|2|488| +2450829|13483|2|28| +2450829|13484|2|52| +2450829|13486|2|662| +2450829|13489|2|390| +2450829|13490|2|634| +2450829|13492|2|374| +2450829|13495|2|618| +2450829|13496|2|151| +2450829|13498|2|978| +2450829|13501|2|157| +2450829|13502|2|515| +2450829|13504|2|280| +2450829|13507|2|497| +2450829|13508|2|543| +2450829|13510|2|73| +2450829|13513|2|23| +2450829|13514|2|94| +2450829|13516|2|726| +2450829|13519|2|552| +2450829|13520|2|931| +2450829|13522|2|993| +2450829|13525|2|463| +2450829|13526|2|878| +2450829|13528|2|356| +2450829|13531|2|191| +2450829|13532|2|400| +2450829|13534|2|18| +2450829|13537|2|529| +2450829|13538|2|86| +2450829|13540|2|136| +2450829|13543|2|804| +2450829|13544|2|150| +2450829|13546|2|124| +2450829|13549|2|355| +2450829|13550|2|484| +2450829|13552|2|690| +2450829|13555|2|740| +2450829|13556|2|194| +2450829|13558|2|249| +2450829|13561|2|235| +2450829|13562|2|267| +2450829|13564|2|224| +2450829|13567|2|466| +2450829|13568|2|824| +2450829|13570|2|34| +2450829|13573|2|552| +2450829|13574|2|588| +2450829|13576|2|102| +2450829|13579|2|493| +2450829|13580|2|971| +2450829|13582|2|| +2450829|13585|2|403| +2450829|13586|2|24| +2450829|13588|2|| +2450829|13591|2|352| +2450829|13592|2|506| +2450829|13594|2|6| +2450829|13597|2|574| +2450829|13598|2|208| +2450829|13600|2|351| +2450829|13603|2|452| +2450829|13604|2|476| +2450829|13606|2|733| +2450829|13609|2|924| +2450829|13610|2|268| +2450829|13612|2|43| +2450829|13615|2|522| +2450829|13616|2|667| +2450829|13618|2|562| +2450829|13621|2|775| +2450829|13622|2|612| +2450829|13624|2|283| +2450829|13627|2|389| +2450829|13628|2|452| +2450829|13630|2|268| +2450829|13633|2|192| +2450829|13634|2|878| +2450829|13636|2|157| +2450829|13639|2|33| +2450829|13640|2|178| +2450829|13642|2|933| +2450829|13645|2|933| +2450829|13646|2|495| +2450829|13648|2|664| +2450829|13651|2|59| +2450829|13652|2|379| +2450829|13654|2|| +2450829|13657|2|659| +2450829|13658|2|49| +2450829|13660|2|438| +2450829|13663|2|768| +2450829|13664|2|317| +2450829|13666|2|661| +2450829|13669|2|279| +2450829|13670|2|597| +2450829|13672|2|264| +2450829|13675|2|113| +2450829|13676|2|| +2450829|13678|2|725| +2450829|13681|2|174| +2450829|13682|2|464| +2450829|13684|2|489| +2450829|13687|2|606| +2450829|13688|2|865| +2450829|13690|2|77| +2450829|13693|2|| +2450829|13694|2|882| +2450829|13696|2|489| +2450829|13699|2|731| +2450829|13700|2|351| +2450829|13702|2|747| +2450829|13705|2|325| +2450829|13706|2|691| +2450829|13708|2|195| +2450829|13711|2|824| +2450829|13712|2|148| +2450829|13714|2|653| +2450829|13717|2|417| +2450829|13718|2|208| +2450829|13720|2|699| +2450829|13723|2|| +2450829|13724|2|613| +2450829|13726|2|897| +2450829|13729|2|88| +2450829|13730|2|917| +2450829|13732|2|818| +2450829|13735|2|697| +2450829|13736|2|563| +2450829|13738|2|729| +2450829|13741|2|360| +2450829|13742|2|269| +2450829|13744|2|35| +2450829|13747|2|700| +2450829|13748|2|| +2450829|13750|2|370| +2450829|13753|2|497| +2450829|13754|2|612| +2450829|13756|2|909| +2450829|13759|2|80| +2450829|13760|2|548| +2450829|13762|2|492| +2450829|13765|2|916| +2450829|13766|2|200| +2450829|13768|2|910| +2450829|13771|2|894| +2450829|13772|2|305| +2450829|13774|2|830| +2450829|13777|2|363| +2450829|13778|2|233| +2450829|13780|2|702| +2450829|13783|2|99| +2450829|13784|2|576| +2450829|13786|2|489| +2450829|13789|2|547| +2450829|13790|2|361| +2450829|13792|2|969| +2450829|13795|2|76| +2450829|13796|2|| +2450829|13798|2|987| +2450829|13801|2|583| +2450829|13802|2|848| +2450829|13804|2|820| +2450829|13807|2|23| +2450829|13808|2|707| +2450829|13810|2|227| +2450829|13813|2|792| +2450829|13814|2|102| +2450829|13816|2|606| +2450829|13819|2|614| +2450829|13820|2|966| +2450829|13822|2|824| +2450829|13825|2|857| +2450829|13826|2|776| +2450829|13828|2|469| +2450829|13831|2|930| +2450829|13832|2|| +2450829|13834|2|998| +2450829|13837|2|213| +2450829|13838|2|260| +2450829|13840|2|505| +2450829|13843|2|305| +2450829|13844|2|529| +2450829|13846|2|380| +2450829|13849|2|519| +2450829|13850|2|630| +2450829|13852|2|654| +2450829|13855|2|294| +2450829|13856|2|339| +2450829|13858|2|| +2450829|13861|2|975| +2450829|13862|2|573| +2450829|13864|2|38| +2450829|13867|2|248| +2450829|13868|2|796| +2450829|13870|2|434| +2450829|13873|2|677| +2450829|13874|2|368| +2450829|13876|2|775| +2450829|13879|2|730| +2450829|13880|2|424| +2450829|13882|2|329| +2450829|13885|2|6| +2450829|13886|2|535| +2450829|13888|2|297| +2450829|13891|2|683| +2450829|13892|2|272| +2450829|13894|2|743| +2450829|13897|2|232| +2450829|13898|2|| +2450829|13900|2|148| +2450829|13903|2|536| +2450829|13904|2|596| +2450829|13906|2|731| +2450829|13909|2|797| +2450829|13910|2|573| +2450829|13912|2|104| +2450829|13915|2|210| +2450829|13916|2|134| +2450829|13918|2|940| +2450829|13921|2|602| +2450829|13922|2|643| +2450829|13924|2|627| +2450829|13927|2|102| +2450829|13928|2|145| +2450829|13930|2|101| +2450829|13933|2|106| +2450829|13934|2|895| +2450829|13936|2|533| +2450829|13939|2|644| +2450829|13940|2|126| +2450829|13942|2|778| +2450829|13945|2|914| +2450829|13946|2|295| +2450829|13948|2|983| +2450829|13951|2|117| +2450829|13952|2|137| +2450829|13954|2|35| +2450829|13957|2|591| +2450829|13958|2|791| +2450829|13960|2|933| +2450829|13963|2|954| +2450829|13964|2|875| +2450829|13966|2|749| +2450829|13969|2|683| +2450829|13970|2|560| +2450829|13972|2|654| +2450829|13975|2|599| +2450829|13976|2|783| +2450829|13978|2|20| +2450829|13981|2|407| +2450829|13982|2|795| +2450829|13984|2|633| +2450829|13987|2|126| +2450829|13988|2|| +2450829|13990|2|492| +2450829|13993|2|704| +2450829|13994|2|| +2450829|13996|2|750| +2450829|13999|2|903| +2450829|14000|2|227| +2450829|14002|2|206| +2450829|14005|2|665| +2450829|14006|2|265| +2450829|14008|2|286| +2450829|14011|2|209| +2450829|14012|2|769| +2450829|14014|2|512| +2450829|14017|2|616| +2450829|14018|2|92| +2450829|14020|2|| +2450829|14023|2|351| +2450829|14024|2|136| +2450829|14026|2|727| +2450829|14029|2|466| +2450829|14030|2|258| +2450829|14032|2|575| +2450829|14035|2|| +2450829|14036|2|242| +2450829|14038|2|247| +2450829|14041|2|567| +2450829|14042|2|833| +2450829|14044|2|145| +2450829|14047|2|474| +2450829|14048|2|522| +2450829|14050|2|| +2450829|14053|2|971| +2450829|14054|2|182| +2450829|14056|2|136| +2450829|14059|2|262| +2450829|14060|2|979| +2450829|14062|2|120| +2450829|14065|2|364| +2450829|14066|2|864| +2450829|14068|2|451| +2450829|14071|2|907| +2450829|14072|2|20| +2450829|14074|2|147| +2450829|14077|2|301| +2450829|14078|2|930| +2450829|14080|2|892| +2450829|14083|2|274| +2450829|14084|2|280| +2450829|14086|2|449| +2450829|14089|2|544| +2450829|14090|2|169| +2450829|14092|2|355| +2450829|14095|2|693| +2450829|14096|2|613| +2450829|14098|2|589| +2450829|14101|2|| +2450829|14102|2|55| +2450829|14104|2|564| +2450829|14107|2|501| +2450829|14108|2|| +2450829|14110|2|812| +2450829|14113|2|516| +2450829|14114|2|755| +2450829|14116|2|732| +2450829|14119|2|272| +2450829|14120|2|529| +2450829|14122|2|597| +2450829|14125|2|594| +2450829|14126|2|717| +2450829|14128|2|862| +2450829|14131|2|33| +2450829|14132|2|811| +2450829|14134|2|634| +2450829|14137|2|221| +2450829|14138|2|| +2450829|14140|2|687| +2450829|14143|2|67| +2450829|14144|2|666| +2450829|14146|2|830| +2450829|14149|2|493| +2450829|14150|2|| +2450829|14152|2|460| +2450829|14155|2|243| +2450829|14156|2|189| +2450829|14158|2|428| +2450829|14161|2|385| +2450829|14162|2|639| +2450829|14164|2|110| +2450829|14167|2|156| +2450829|14168|2|138| +2450829|14170|2|952| +2450829|14173|2|762| +2450829|14174|2|439| +2450829|14176|2|863| +2450829|14179|2|335| +2450829|14180|2|202| +2450829|14182|2|243| +2450829|14185|2|447| +2450829|14186|2|146| +2450829|14188|2|539| +2450829|14191|2|599| +2450829|14192|2|200| +2450829|14194|2|743| +2450829|14197|2|49| +2450829|14198|2|995| +2450829|14200|2|| +2450829|14203|2|158| +2450829|14204|2|280| +2450829|14206|2|8| +2450829|14209|2|651| +2450829|14210|2|533| +2450829|14212|2|209| +2450829|14215|2|302| +2450829|14216|2|390| +2450829|14218|2|916| +2450829|14221|2|571| +2450829|14222|2|528| +2450829|14224|2|139| +2450829|14227|2|867| +2450829|14228|2|641| +2450829|14230|2|531| +2450829|14233|2|633| +2450829|14234|2|653| +2450829|14236|2|420| +2450829|14239|2|388| +2450829|14240|2|679| +2450829|14242|2|199| +2450829|14245|2|454| +2450829|14246|2|456| +2450829|14248|2|575| +2450829|14251|2|859| +2450829|14252|2|970| +2450829|14254|2|438| +2450829|14257|2|609| +2450829|14258|2|253| +2450829|14260|2|684| +2450829|14263|2|457| +2450829|14264|2|335| +2450829|14266|2|737| +2450829|14269|2|926| +2450829|14270|2|167| +2450829|14272|2|800| +2450829|14275|2|215| +2450829|14276|2|934| +2450829|14278|2|663| +2450829|14281|2|713| +2450829|14282|2|288| +2450829|14284|2|280| +2450829|14287|2|695| +2450829|14288|2|742| +2450829|14290|2|217| +2450829|14293|2|894| +2450829|14294|2|688| +2450829|14296|2|281| +2450829|14299|2|122| +2450829|14300|2|8| +2450829|14302|2|887| +2450829|14305|2|815| +2450829|14306|2|765| +2450829|14308|2|588| +2450829|14311|2|619| +2450829|14312|2|912| +2450829|14314|2|968| +2450829|14317|2|501| +2450829|14318|2|198| +2450829|14320|2|982| +2450829|14323|2|4| +2450829|14324|2|0| +2450829|14326|2|986| +2450829|14329|2|585| +2450829|14330|2|529| +2450829|14332|2|725| +2450829|14335|2|530| +2450829|14336|2|43| +2450829|14338|2|241| +2450829|14341|2|944| +2450829|14342|2|143| +2450829|14344|2|396| +2450829|14347|2|992| +2450829|14348|2|882| +2450829|14350|2|361| +2450829|14353|2|398| +2450829|14354|2|198| +2450829|14356|2|| +2450829|14359|2|966| +2450829|14360|2|679| +2450829|14362|2|337| +2450829|14365|2|239| +2450829|14366|2|254| +2450829|14368|2|666| +2450829|14371|2|659| +2450829|14372|2|| +2450829|14374|2|771| +2450829|14377|2|30| +2450829|14378|2|390| +2450829|14380|2|240| +2450829|14383|2|399| +2450829|14384|2|290| +2450829|14386|2|172| +2450829|14389|2|475| +2450829|14390|2|143| +2450829|14392|2|65| +2450829|14395|2|225| +2450829|14396|2|451| +2450829|14398|2|155| +2450829|14401|2|67| +2450829|14402|2|938| +2450829|14404|2|989| +2450829|14407|2|436| +2450829|14408|2|346| +2450829|14410|2|241| +2450829|14413|2|277| +2450829|14414|2|35| +2450829|14416|2|562| +2450829|14419|2|149| +2450829|14420|2|511| +2450829|14422|2|441| +2450829|14425|2|532| +2450829|14426|2|614| +2450829|14428|2|945| +2450829|14431|2|358| +2450829|14432|2|148| +2450829|14434|2|632| +2450829|14437|2|774| +2450829|14438|2|294| +2450829|14440|2|53| +2450829|14443|2|149| +2450829|14444|2|860| +2450829|14446|2|122| +2450829|14449|2|729| +2450829|14450|2|12| +2450829|14452|2|343| +2450829|14455|2|541| +2450829|14456|2|805| +2450829|14458|2|914| +2450829|14461|2|569| +2450829|14462|2|931| +2450829|14464|2|361| +2450829|14467|2|673| +2450829|14468|2|902| +2450829|14470|2|374| +2450829|14473|2|778| +2450829|14474|2|790| +2450829|14476|2|197| +2450829|14479|2|224| +2450829|14480|2|842| +2450829|14482|2|533| +2450829|14485|2|716| +2450829|14486|2|114| +2450829|14488|2|735| +2450829|14491|2|785| +2450829|14492|2|226| +2450829|14494|2|653| +2450829|14497|2|263| +2450829|14498|2|813| +2450829|14500|2|423| +2450829|14503|2|680| +2450829|14504|2|| +2450829|14506|2|910| +2450829|14509|2|451| +2450829|14510|2|213| +2450829|14512|2|263| +2450829|14515|2|757| +2450829|14516|2|461| +2450829|14518|2|119| +2450829|14521|2|19| +2450829|14522|2|139| +2450829|14524|2|594| +2450829|14527|2|90| +2450829|14528|2|326| +2450829|14530|2|44| +2450829|14533|2|117| +2450829|14534|2|99| +2450829|14536|2|209| +2450829|14539|2|565| +2450829|14540|2|901| +2450829|14542|2|534| +2450829|14545|2|249| +2450829|14546|2|800| +2450829|14548|2|723| +2450829|14551|2|143| +2450829|14552|2|925| +2450829|14554|2|54| +2450829|14557|2|775| +2450829|14558|2|526| +2450829|14560|2|541| +2450829|14563|2|398| +2450829|14564|2|874| +2450829|14566|2|161| +2450829|14569|2|631| +2450829|14570|2|220| +2450829|14572|2|453| +2450829|14575|2|985| +2450829|14576|2|130| +2450829|14578|2|925| +2450829|14581|2|997| +2450829|14582|2|| +2450829|14584|2|465| +2450829|14587|2|896| +2450829|14588|2|502| +2450829|14590|2|304| +2450829|14593|2|679| +2450829|14594|2|17| +2450829|14596|2|671| +2450829|14599|2|463| +2450829|14600|2|485| +2450829|14602|2|| +2450829|14605|2|89| +2450829|14606|2|967| +2450829|14608|2|| +2450829|14611|2|165| +2450829|14612|2|346| +2450829|14614|2|426| +2450829|14617|2|313| +2450829|14618|2|276| +2450829|14620|2|583| +2450829|14623|2|437| +2450829|14624|2|472| +2450829|14626|2|602| +2450829|14629|2|808| +2450829|14630|2|227| +2450829|14632|2|968| +2450829|14635|2|859| +2450829|14636|2|895| +2450829|14638|2|822| +2450829|14641|2|543| +2450829|14642|2|477| +2450829|14644|2|| +2450829|14647|2|310| +2450829|14648|2|319| +2450829|14650|2|908| +2450829|14653|2|894| +2450829|14654|2|705| +2450829|14656|2|635| +2450829|14659|2|411| +2450829|14660|2|910| +2450829|14662|2|632| +2450829|14665|2|803| +2450829|14666|2|762| +2450829|14668|2|402| +2450829|14671|2|202| +2450829|14672|2|19| +2450829|14674|2|430| +2450829|14677|2|782| +2450829|14678|2|961| +2450829|14680|2|140| +2450829|14683|2|451| +2450829|14684|2|653| +2450829|14686|2|987| +2450829|14689|2|837| +2450829|14690|2|177| +2450829|14692|2|241| +2450829|14695|2|808| +2450829|14696|2|819| +2450829|14698|2|562| +2450829|14701|2|143| +2450829|14702|2|479| +2450829|14704|2|246| +2450829|14707|2|666| +2450829|14708|2|212| +2450829|14710|2|357| +2450829|14713|2|945| +2450829|14714|2|58| +2450829|14716|2|878| +2450829|14719|2|271| +2450829|14720|2|465| +2450829|14722|2|628| +2450829|14725|2|604| +2450829|14726|2|777| +2450829|14728|2|91| +2450829|14731|2|604| +2450829|14732|2|310| +2450829|14734|2|840| +2450829|14737|2|249| +2450829|14738|2|454| +2450829|14740|2|638| +2450829|14743|2|377| +2450829|14744|2|896| +2450829|14746|2|648| +2450829|14749|2|328| +2450829|14750|2|435| +2450829|14752|2|486| +2450829|14755|2|87| +2450829|14756|2|46| +2450829|14758|2|359| +2450829|14761|2|61| +2450829|14762|2|827| +2450829|14764|2|843| +2450829|14767|2|577| +2450829|14768|2|823| +2450829|14770|2|515| +2450829|14773|2|517| +2450829|14774|2|895| +2450829|14776|2|277| +2450829|14779|2|726| +2450829|14780|2|579| +2450829|14782|2|789| +2450829|14785|2|408| +2450829|14786|2|826| +2450829|14788|2|736| +2450829|14791|2|110| +2450829|14792|2|820| +2450829|14794|2|882| +2450829|14797|2|975| +2450829|14798|2|637| +2450829|14800|2|263| +2450829|14803|2|152| +2450829|14804|2|470| +2450829|14806|2|281| +2450829|14809|2|395| +2450829|14810|2|928| +2450829|14812|2|464| +2450829|14815|2|456| +2450829|14816|2|482| +2450829|14818|2|360| +2450829|14821|2|469| +2450829|14822|2|826| +2450829|14824|2|361| +2450829|14827|2|106| +2450829|14828|2|409| +2450829|14830|2|281| +2450829|14833|2|838| +2450829|14834|2|6| +2450829|14836|2|664| +2450829|14839|2|624| +2450829|14840|2|125| +2450829|14842|2|365| +2450829|14845|2|819| +2450829|14846|2|37| +2450829|14848|2|376| +2450829|14851|2|| +2450829|14852|2|772| +2450829|14854|2|782| +2450829|14857|2|266| +2450829|14858|2|600| +2450829|14860|2|292| +2450829|14863|2|249| +2450829|14864|2|563| +2450829|14866|2|689| +2450829|14869|2|489| +2450829|14870|2|762| +2450829|14872|2|| +2450829|14875|2|710| +2450829|14876|2|242| +2450829|14878|2|591| +2450829|14881|2|517| +2450829|14882|2|404| +2450829|14884|2|160| +2450829|14887|2|| +2450829|14888|2|| +2450829|14890|2|132| +2450829|14893|2|480| +2450829|14894|2|30| +2450829|14896|2|881| +2450829|14899|2|16| +2450829|14900|2|821| +2450829|14902|2|276| +2450829|14905|2|388| +2450829|14906|2|101| +2450829|14908|2|401| +2450829|14911|2|744| +2450829|14912|2|315| +2450829|14914|2|834| +2450829|14917|2|773| +2450829|14918|2|187| +2450829|14920|2|136| +2450829|14923|2|346| +2450829|14924|2|565| +2450829|14926|2|483| +2450829|14929|2|| +2450829|14930|2|752| +2450829|14932|2|416| +2450829|14935|2|| +2450829|14936|2|285| +2450829|14938|2|| +2450829|14941|2|768| +2450829|14942|2|27| +2450829|14944|2|739| +2450829|14947|2|627| +2450829|14948|2|982| +2450829|14950|2|361| +2450829|14953|2|51| +2450829|14954|2|| +2450829|14956|2|820| +2450829|14959|2|502| +2450829|14960|2|826| +2450829|14962|2|| +2450829|14965|2|559| +2450829|14966|2|624| +2450829|14968|2|717| +2450829|14971|2|821| +2450829|14972|2|864| +2450829|14974|2|423| +2450829|14977|2|1000| +2450829|14978|2|54| +2450829|14980|2|647| +2450829|14983|2|939| +2450829|14984|2|996| +2450829|14986|2|313| +2450829|14989|2|176| +2450829|14990|2|| +2450829|14992|2|822| +2450829|14995|2|629| +2450829|14996|2|115| +2450829|14998|2|919| +2450829|15001|2|284| +2450829|15002|2|158| +2450829|15004|2|712| +2450829|15007|2|887| +2450829|15008|2|484| +2450829|15010|2|972| +2450829|15013|2|226| +2450829|15014|2|166| +2450829|15016|2|300| +2450829|15019|2|281| +2450829|15020|2|624| +2450829|15022|2|660| +2450829|15025|2|367| +2450829|15026|2|572| +2450829|15028|2|234| +2450829|15031|2|396| +2450829|15032|2|| +2450829|15034|2|984| +2450829|15037|2|338| +2450829|15038|2|| +2450829|15040|2|204| +2450829|15043|2|928| +2450829|15044|2|450| +2450829|15046|2|802| +2450829|15049|2|848| +2450829|15050|2|862| +2450829|15052|2|639| +2450829|15055|2|829| +2450829|15056|2|411| +2450829|15058|2|950| +2450829|15061|2|955| +2450829|15062|2|570| +2450829|15064|2|222| +2450829|15067|2|810| +2450829|15068|2|289| +2450829|15070|2|102| +2450829|15073|2|353| +2450829|15074|2|874| +2450829|15076|2|657| +2450829|15079|2|608| +2450829|15080|2|846| +2450829|15082|2|236| +2450829|15085|2|199| +2450829|15086|2|912| +2450829|15088|2|300| +2450829|15091|2|810| +2450829|15092|2|424| +2450829|15094|2|555| +2450829|15097|2|634| +2450829|15098|2|930| +2450829|15100|2|600| +2450829|15103|2|612| +2450829|15104|2|369| +2450829|15106|2|561| +2450829|15109|2|677| +2450829|15110|2|882| +2450829|15112|2|140| +2450829|15115|2|546| +2450829|15116|2|817| +2450829|15118|2|| +2450829|15121|2|131| +2450829|15122|2|744| +2450829|15124|2|636| +2450829|15127|2|586| +2450829|15128|2|259| +2450829|15130|2|133| +2450829|15133|2|391| +2450829|15134|2|246| +2450829|15136|2|121| +2450829|15139|2|741| +2450829|15140|2|811| +2450829|15142|2|452| +2450829|15145|2|830| +2450829|15146|2|293| +2450829|15148|2|66| +2450829|15151|2|61| +2450829|15152|2|| +2450829|15154|2|891| +2450829|15157|2|573| +2450829|15158|2|729| +2450829|15160|2|840| +2450829|15163|2|275| +2450829|15164|2|734| +2450829|15166|2|246| +2450829|15169|2|843| +2450829|15170|2|844| +2450829|15172|2|283| +2450829|15175|2|89| +2450829|15176|2|253| +2450829|15178|2|674| +2450829|15181|2|419| +2450829|15182|2|805| +2450829|15184|2|283| +2450829|15187|2|227| +2450829|15188|2|380| +2450829|15190|2|452| +2450829|15193|2|45| +2450829|15194|2|844| +2450829|15196|2|597| +2450829|15199|2|959| +2450829|15200|2|229| +2450829|15202|2|304| +2450829|15205|2|912| +2450829|15206|2|767| +2450829|15208|2|524| +2450829|15211|2|529| +2450829|15212|2|| +2450829|15214|2|610| +2450829|15217|2|255| +2450829|15218|2|517| +2450829|15220|2|998| +2450829|15223|2|511| +2450829|15224|2|414| +2450829|15226|2|751| +2450829|15229|2|455| +2450829|15230|2|101| +2450829|15232|2|220| +2450829|15235|2|330| +2450829|15236|2|83| +2450829|15238|2|711| +2450829|15241|2|893| +2450829|15242|2|237| +2450829|15244|2|648| +2450829|15247|2|599| +2450829|15248|2|640| +2450829|15250|2|815| +2450829|15253|2|751| +2450829|15254|2|438| +2450829|15256|2|232| +2450829|15259|2|608| +2450829|15260|2|601| +2450829|15262|2|404| +2450829|15265|2|541| +2450829|15266|2|37| +2450829|15268|2|| +2450829|15271|2|657| +2450829|15272|2|997| +2450829|15274|2|602| +2450829|15277|2|785| +2450829|15278|2|0| +2450829|15280|2|263| +2450829|15283|2|44| +2450829|15284|2|60| +2450829|15286|2|455| +2450829|15289|2|654| +2450829|15290|2|53| +2450829|15292|2|528| +2450829|15295|2|888| +2450829|15296|2|| +2450829|15298|2|772| +2450829|15301|2|406| +2450829|15302|2|12| +2450829|15304|2|315| +2450829|15307|2|335| +2450829|15308|2|904| +2450829|15310|2|311| +2450829|15313|2|205| +2450829|15314|2|76| +2450829|15316|2|967| +2450829|15319|2|667| +2450829|15320|2|747| +2450829|15322|2|111| +2450829|15325|2|513| +2450829|15326|2|215| +2450829|15328|2|411| +2450829|15331|2|749| +2450829|15332|2|158| +2450829|15334|2|72| +2450829|15337|2|49| +2450829|15338|2|981| +2450829|15340|2|56| +2450829|15343|2|559| +2450829|15344|2|886| +2450829|15346|2|908| +2450829|15349|2|880| +2450829|15350|2|770| +2450829|15352|2|711| +2450829|15355|2|279| +2450829|15356|2|964| +2450829|15358|2|743| +2450829|15361|2|455| +2450829|15362|2|88| +2450829|15364|2|847| +2450829|15367|2|586| +2450829|15368|2|69| +2450829|15370|2|| +2450829|15373|2|| +2450829|15374|2|983| +2450829|15376|2|489| +2450829|15379|2|714| +2450829|15380|2|587| +2450829|15382|2|854| +2450829|15385|2|17| +2450829|15386|2|639| +2450829|15388|2|227| +2450829|15391|2|714| +2450829|15392|2|936| +2450829|15394|2|573| +2450829|15397|2|223| +2450829|15398|2|212| +2450829|15400|2|712| +2450829|15403|2|529| +2450829|15404|2|462| +2450829|15406|2|58| +2450829|15409|2|770| +2450829|15410|2|336| +2450829|15412|2|244| +2450829|15415|2|645| +2450829|15416|2|887| +2450829|15418|2|950| +2450829|15421|2|436| +2450829|15422|2|308| +2450829|15424|2|856| +2450829|15427|2|363| +2450829|15428|2|467| +2450829|15430|2|660| +2450829|15433|2|987| +2450829|15434|2|145| +2450829|15436|2|661| +2450829|15439|2|666| +2450829|15440|2|689| +2450829|15442|2|410| +2450829|15445|2|438| +2450829|15446|2|631| +2450829|15448|2|719| +2450829|15451|2|258| +2450829|15452|2|317| +2450829|15454|2|834| +2450829|15457|2|357| +2450829|15458|2|293| +2450829|15460|2|639| +2450829|15463|2|274| +2450829|15464|2|429| +2450829|15466|2|339| +2450829|15469|2|839| +2450829|15470|2|356| +2450829|15472|2|832| +2450829|15475|2|495| +2450829|15476|2|12| +2450829|15478|2|552| +2450829|15481|2|1| +2450829|15482|2|893| +2450829|15484|2|850| +2450829|15487|2|318| +2450829|15488|2|534| +2450829|15490|2|230| +2450829|15493|2|344| +2450829|15494|2|981| +2450829|15496|2|543| +2450829|15499|2|789| +2450829|15500|2|25| +2450829|15502|2|94| +2450829|15505|2|405| +2450829|15506|2|643| +2450829|15508|2|604| +2450829|15511|2|764| +2450829|15512|2|366| +2450829|15514|2|554| +2450829|15517|2|337| +2450829|15518|2|| +2450829|15520|2|98| +2450829|15523|2|| +2450829|15524|2|| +2450829|15526|2|2| +2450829|15529|2|788| +2450829|15530|2|542| +2450829|15532|2|536| +2450829|15535|2|571| +2450829|15536|2|394| +2450829|15538|2|673| +2450829|15541|2|90| +2450829|15542|2|956| +2450829|15544|2|372| +2450829|15547|2|57| +2450829|15548|2|868| +2450829|15550|2|46| +2450829|15553|2|999| +2450829|15554|2|662| +2450829|15556|2|486| +2450829|15559|2|899| +2450829|15560|2|854| +2450829|15562|2|864| +2450829|15565|2|818| +2450829|15566|2|968| +2450829|15568|2|393| +2450829|15571|2|976| +2450829|15572|2|861| +2450829|15574|2|375| +2450829|15577|2|269| +2450829|15578|2|363| +2450829|15580|2|554| +2450829|15583|2|121| +2450829|15584|2|148| +2450829|15586|2|| +2450829|15589|2|826| +2450829|15590|2|230| +2450829|15592|2|274| +2450829|15595|2|886| +2450829|15596|2|924| +2450829|15598|2|256| +2450829|15601|2|141| +2450829|15602|2|400| +2450829|15604|2|22| +2450829|15607|2|42| +2450829|15608|2|264| +2450829|15610|2|705| +2450829|15613|2|421| +2450829|15614|2|525| +2450829|15616|2|931| +2450829|15619|2|261| +2450829|15620|2|339| +2450829|15622|2|617| +2450829|15625|2|536| +2450829|15626|2|900| +2450829|15628|2|311| +2450829|15631|2|588| +2450829|15632|2|599| +2450829|15634|2|672| +2450829|15637|2|680| +2450829|15638|2|129| +2450829|15640|2|872| +2450829|15643|2|154| +2450829|15644|2|164| +2450829|15646|2|748| +2450829|15649|2|980| +2450829|15650|2|857| +2450829|15652|2|526| +2450829|15655|2|535| +2450829|15656|2|307| +2450829|15658|2|599| +2450829|15661|2|19| +2450829|15662|2|388| +2450829|15664|2|898| +2450829|15667|2|147| +2450829|15668|2|154| +2450829|15670|2|635| +2450829|15673|2|684| +2450829|15674|2|310| +2450829|15676|2|467| +2450829|15679|2|912| +2450829|15680|2|906| +2450829|15682|2|413| +2450829|15685|2|214| +2450829|15686|2|439| +2450829|15688|2|945| +2450829|15691|2|455| +2450829|15692|2|759| +2450829|15694|2|137| +2450829|15697|2|669| +2450829|15698|2|915| +2450829|15700|2|28| +2450829|15703|2|731| +2450829|15704|2|985| +2450829|15706|2|296| +2450829|15709|2|480| +2450829|15710|2|456| +2450829|15712|2|159| +2450829|15715|2|30| +2450829|15716|2|92| +2450829|15718|2|567| +2450829|15721|2|170| +2450829|15722|2|835| +2450829|15724|2|526| +2450829|15727|2|203| +2450829|15728|2|772| +2450829|15730|2|897| +2450829|15733|2|500| +2450829|15734|2|197| +2450829|15736|2|765| +2450829|15739|2|950| +2450829|15740|2|249| +2450829|15742|2|472| +2450829|15745|2|594| +2450829|15746|2|720| +2450829|15748|2|709| +2450829|15751|2|454| +2450829|15752|2|| +2450829|15754|2|57| +2450829|15757|2|840| +2450829|15758|2|377| +2450829|15760|2|| +2450829|15763|2|714| +2450829|15764|2|861| +2450829|15766|2|130| +2450829|15769|2|449| +2450829|15770|2|107| +2450829|15772|2|385| +2450829|15775|2|| +2450829|15776|2|775| +2450829|15778|2|852| +2450829|15781|2|100| +2450829|15782|2|| +2450829|15784|2|298| +2450829|15787|2|701| +2450829|15788|2|784| +2450829|15790|2|421| +2450829|15793|2|569| +2450829|15794|2|58| +2450829|15796|2|211| +2450829|15799|2|631| +2450829|15800|2|153| +2450829|15802|2|743| +2450829|15805|2|930| +2450829|15806|2|185| +2450829|15808|2|804| +2450829|15811|2|30| +2450829|15812|2|770| +2450829|15814|2|953| +2450829|15817|2|707| +2450829|15818|2|545| +2450829|15820|2|925| +2450829|15823|2|534| +2450829|15824|2|118| +2450829|15826|2|689| +2450829|15829|2|576| +2450829|15830|2|3| +2450829|15832|2|468| +2450829|15835|2|680| +2450829|15836|2|707| +2450829|15838|2|990| +2450829|15841|2|436| +2450829|15842|2|| +2450829|15844|2|862| +2450829|15847|2|734| +2450829|15848|2|| +2450829|15850|2|50| +2450829|15853|2|613| +2450829|15854|2|658| +2450829|15856|2|23| +2450829|15859|2|383| +2450829|15860|2|323| +2450829|15862|2|986| +2450829|15865|2|382| +2450829|15866|2|158| +2450829|15868|2|959| +2450829|15871|2|137| +2450829|15872|2|50| +2450829|15874|2|125| +2450829|15877|2|174| +2450829|15878|2|882| +2450829|15880|2|795| +2450829|15883|2|462| +2450829|15884|2|451| +2450829|15886|2|591| +2450829|15889|2|904| +2450829|15890|2|758| +2450829|15892|2|888| +2450829|15895|2|194| +2450829|15896|2|133| +2450829|15898|2|286| +2450829|15901|2|30| +2450829|15902|2|894| +2450829|15904|2|438| +2450829|15907|2|30| +2450829|15908|2|895| +2450829|15910|2|| +2450829|15913|2|152| +2450829|15914|2|24| +2450829|15916|2|948| +2450829|15919|2|616| +2450829|15920|2|846| +2450829|15922|2|243| +2450829|15925|2|609| +2450829|15926|2|845| +2450829|15928|2|929| +2450829|15931|2|7| +2450829|15932|2|26| +2450829|15934|2|462| +2450829|15937|2|762| +2450829|15938|2|744| +2450829|15940|2|821| +2450829|15943|2|| +2450829|15944|2|463| +2450829|15946|2|425| +2450829|15949|2|940| +2450829|15950|2|68| +2450829|15952|2|290| +2450829|15955|2|30| +2450829|15956|2|866| +2450829|15958|2|219| +2450829|15961|2|513| +2450829|15962|2|772| +2450829|15964|2|51| +2450829|15967|2|334| +2450829|15968|2|683| +2450829|15970|2|17| +2450829|15973|2|826| +2450829|15974|2|339| +2450829|15976|2|| +2450829|15979|2|306| +2450829|15980|2|154| +2450829|15982|2|536| +2450829|15985|2|262| +2450829|15986|2|36| +2450829|15988|2|360| +2450829|15991|2|539| +2450829|15992|2|| +2450829|15994|2|989| +2450829|15997|2|355| +2450829|15998|2|870| +2450829|16000|2|707| +2450829|16003|2|447| +2450829|16004|2|561| +2450829|16006|2|844| +2450829|16009|2|447| +2450829|16010|2|961| +2450829|16012|2|701| +2450829|16015|2|546| +2450829|16016|2|27| +2450829|16018|2|804| +2450829|16021|2|175| +2450829|16022|2|902| +2450829|16024|2|205| +2450829|16027|2|890| +2450829|16028|2|371| +2450829|16030|2|19| +2450829|16033|2|402| +2450829|16034|2|142| +2450829|16036|2|567| +2450829|16039|2|452| +2450829|16040|2|54| +2450829|16042|2|890| +2450829|16045|2|867| +2450829|16046|2|312| +2450829|16048|2|881| +2450829|16051|2|380| +2450829|16052|2|522| +2450829|16054|2|619| +2450829|16057|2|115| +2450829|16058|2|676| +2450829|16060|2|821| +2450829|16063|2|233| +2450829|16064|2|572| +2450829|16066|2|136| +2450829|16069|2|444| +2450829|16070|2|215| +2450829|16072|2|402| +2450829|16075|2|290| +2450829|16076|2|494| +2450829|16078|2|74| +2450829|16081|2|325| +2450829|16082|2|275| +2450829|16084|2|654| +2450829|16087|2|175| +2450829|16088|2|676| +2450829|16090|2|537| +2450829|16093|2|782| +2450829|16094|2|618| +2450829|16096|2|995| +2450829|16099|2|875| +2450829|16100|2|340| +2450829|16102|2|31| +2450829|16105|2|377| +2450829|16106|2|731| +2450829|16108|2|6| +2450829|16111|2|623| +2450829|16112|2|869| +2450829|16114|2|416| +2450829|16117|2|266| +2450829|16118|2|367| +2450829|16120|2|731| +2450829|16123|2|624| +2450829|16124|2|| +2450829|16126|2|579| +2450829|16129|2|384| +2450829|16130|2|945| +2450829|16132|2|567| +2450829|16135|2|722| +2450829|16136|2|481| +2450829|16138|2|46| +2450829|16141|2|528| +2450829|16142|2|486| +2450829|16144|2|658| +2450829|16147|2|67| +2450829|16148|2|728| +2450829|16150|2|637| +2450829|16153|2|| +2450829|16154|2|982| +2450829|16156|2|684| +2450829|16159|2|502| +2450829|16160|2|369| +2450829|16162|2|723| +2450829|16165|2|625| +2450829|16166|2|200| +2450829|16168|2|805| +2450829|16171|2|405| +2450829|16172|2|839| +2450829|16174|2|31| +2450829|16177|2|581| +2450829|16178|2|449| +2450829|16180|2|828| +2450829|16183|2|| +2450829|16184|2|868| +2450829|16186|2|599| +2450829|16189|2|24| +2450829|16190|2|429| +2450829|16192|2|769| +2450829|16195|2|998| +2450829|16196|2|73| +2450829|16198|2|824| +2450829|16201|2|970| +2450829|16202|2|555| +2450829|16204|2|72| +2450829|16207|2|103| +2450829|16208|2|| +2450829|16210|2|134| +2450829|16213|2|831| +2450829|16214|2|676| +2450829|16216|2|174| +2450829|16219|2|883| +2450829|16220|2|218| +2450829|16222|2|425| +2450829|16225|2|399| +2450829|16226|2|596| +2450829|16228|2|875| +2450829|16231|2|541| +2450829|16232|2|974| +2450829|16234|2|330| +2450829|16237|2|513| +2450829|16238|2|497| +2450829|16240|2|269| +2450829|16243|2|699| +2450829|16244|2|492| +2450829|16246|2|749| +2450829|16249|2|95| +2450829|16250|2|364| +2450829|16252|2|948| +2450829|16255|2|877| +2450829|16256|2|206| +2450829|16258|2|415| +2450829|16261|2|772| +2450829|16262|2|947| +2450829|16264|2|981| +2450829|16267|2|206| +2450829|16268|2|590| +2450829|16270|2|218| +2450829|16273|2|428| +2450829|16274|2|93| +2450829|16276|2|532| +2450829|16279|2|550| +2450829|16280|2|944| +2450829|16282|2|752| +2450829|16285|2|597| +2450829|16286|2|913| +2450829|16288|2|636| +2450829|16291|2|895| +2450829|16292|2|82| +2450829|16294|2|333| +2450829|16297|2|919| +2450829|16298|2|115| +2450829|16300|2|715| +2450829|16303|2|384| +2450829|16304|2|643| +2450829|16306|2|581| +2450829|16309|2|335| +2450829|16310|2|| +2450829|16312|2|168| +2450829|16315|2|348| +2450829|16316|2|821| +2450829|16318|2|102| +2450829|16321|2|454| +2450829|16322|2|934| +2450829|16324|2|461| +2450829|16327|2|194| +2450829|16328|2|| +2450829|16330|2|490| +2450829|16333|2|564| +2450829|16334|2|754| +2450829|16336|2|763| +2450829|16339|2|990| +2450829|16340|2|35| +2450829|16342|2|924| +2450829|16345|2|980| +2450829|16346|2|154| +2450829|16348|2|822| +2450829|16351|2|955| +2450829|16352|2|587| +2450829|16354|2|320| +2450829|16357|2|381| +2450829|16358|2|514| +2450829|16360|2|264| +2450829|16363|2|517| +2450829|16364|2|658| +2450829|16366|2|431| +2450829|16369|2|529| +2450829|16370|2|381| +2450829|16372|2|188| +2450829|16375|2|236| +2450829|16376|2|540| +2450829|16378|2|936| +2450829|16381|2|16| +2450829|16382|2|278| +2450829|16384|2|568| +2450829|16387|2|707| +2450829|16388|2|519| +2450829|16390|2|759| +2450829|16393|2|491| +2450829|16394|2|847| +2450829|16396|2|7| +2450829|16399|2|395| +2450829|16400|2|782| +2450829|16402|2|667| +2450829|16405|2|432| +2450829|16406|2|281| +2450829|16408|2|324| +2450829|16411|2|259| +2450829|16412|2|| +2450829|16414|2|578| +2450829|16417|2|802| +2450829|16418|2|772| +2450829|16420|2|353| +2450829|16423|2|144| +2450829|16424|2|| +2450829|16426|2|| +2450829|16429|2|252| +2450829|16430|2|540| +2450829|16432|2|464| +2450829|16435|2|15| +2450829|16436|2|319| +2450829|16438|2|814| +2450829|16441|2|704| +2450829|16442|2|108| +2450829|16444|2|| +2450829|16447|2|331| +2450829|16448|2|383| +2450829|16450|2|825| +2450829|16453|2|58| +2450829|16454|2|624| +2450829|16456|2|944| +2450829|16459|2|221| +2450829|16460|2|83| +2450829|16462|2|557| +2450829|16465|2|42| +2450829|16466|2|| +2450829|16468|2|921| +2450829|16471|2|426| +2450829|16472|2|811| +2450829|16474|2|451| +2450829|16477|2|909| +2450829|16478|2|729| +2450829|16480|2|117| +2450829|16483|2|| +2450829|16484|2|886| +2450829|16486|2|636| +2450829|16489|2|143| +2450829|16490|2|365| +2450829|16492|2|19| +2450829|16495|2|869| +2450829|16496|2|899| +2450829|16498|2|525| +2450829|16501|2|505| +2450829|16502|2|113| +2450829|16504|2|448| +2450829|16507|2|196| +2450829|16508|2|| +2450829|16510|2|609| +2450829|16513|2|547| +2450829|16514|2|397| +2450829|16516|2|825| +2450829|16519|2|93| +2450829|16520|2|555| +2450829|16522|2|40| +2450829|16525|2|865| +2450829|16526|2|899| +2450829|16528|2|885| +2450829|16531|2|867| +2450829|16532|2|11| +2450829|16534|2|196| +2450829|16537|2|865| +2450829|16538|2|476| +2450829|16540|2|713| +2450829|16543|2|821| +2450829|16544|2|452| +2450829|16546|2|280| +2450829|16549|2|445| +2450829|16550|2|362| +2450829|16552|2|470| +2450829|16555|2|125| +2450829|16556|2|597| +2450829|16558|2|991| +2450829|16561|2|484| +2450829|16562|2|577| +2450829|16564|2|3| +2450829|16567|2|717| +2450829|16568|2|423| +2450829|16570|2|508| +2450829|16573|2|581| +2450829|16574|2|842| +2450829|16576|2|649| +2450829|16579|2|12| +2450829|16580|2|732| +2450829|16582|2|328| +2450829|16585|2|847| +2450829|16586|2|10| +2450829|16588|2|94| +2450829|16591|2|569| +2450829|16592|2|988| +2450829|16594|2|129| +2450829|16597|2|577| +2450829|16598|2|772| +2450829|16600|2|217| +2450829|16603|2|444| +2450829|16604|2|416| +2450829|16606|2|550| +2450829|16609|2|551| +2450829|16610|2|367| +2450829|16612|2|346| +2450829|16615|2|927| +2450829|16616|2|312| +2450829|16618|2|474| +2450829|16621|2|927| +2450829|16622|2|251| +2450829|16624|2|776| +2450829|16627|2|324| +2450829|16628|2|378| +2450829|16630|2|122| +2450829|16633|2|502| +2450829|16634|2|146| +2450829|16636|2|76| +2450829|16639|2|555| +2450829|16640|2|539| +2450829|16642|2|897| +2450829|16645|2|42| +2450829|16646|2|858| +2450829|16648|2|361| +2450829|16651|2|15| +2450829|16652|2|215| +2450829|16654|2|123| +2450829|16657|2|2| +2450829|16658|2|983| +2450829|16660|2|934| +2450829|16663|2|568| +2450829|16664|2|778| +2450829|16666|2|906| +2450829|16669|2|| +2450829|16670|2|574| +2450829|16672|2|12| +2450829|16675|2|198| +2450829|16676|2|35| +2450829|16678|2|614| +2450829|16681|2|295| +2450829|16682|2|| +2450829|16684|2|394| +2450829|16687|2|392| +2450829|16688|2|646| +2450829|16690|2|36| +2450829|16693|2|698| +2450829|16694|2|478| +2450829|16696|2|400| +2450829|16699|2|354| +2450829|16700|2|19| +2450829|16702|2|188| +2450829|16705|2|313| +2450829|16706|2|235| +2450829|16708|2|416| +2450829|16711|2|468| +2450829|16712|2|851| +2450829|16714|2|383| +2450829|16717|2|414| +2450829|16718|2|150| +2450829|16720|2|810| +2450829|16723|2|669| +2450829|16724|2|679| +2450829|16726|2|755| +2450829|16729|2|499| +2450829|16730|2|965| +2450829|16732|2|28| +2450829|16735|2|935| +2450829|16736|2|425| +2450829|16738|2|325| +2450829|16741|2|745| +2450829|16742|2|567| +2450829|16744|2|326| +2450829|16747|2|104| +2450829|16748|2|607| +2450829|16750|2|147| +2450829|16753|2|41| +2450829|16754|2|514| +2450829|16756|2|260| +2450829|16759|2|860| +2450829|16760|2|234| +2450829|16762|2|453| +2450829|16765|2|893| +2450829|16766|2|12| +2450829|16768|2|917| +2450829|16771|2|117| +2450829|16772|2|18| +2450829|16774|2|398| +2450829|16777|2|5| +2450829|16778|2|420| +2450829|16780|2|180| +2450829|16783|2|537| +2450829|16784|2|903| +2450829|16786|2|614| +2450829|16789|2|370| +2450829|16790|2|865| +2450829|16792|2|992| +2450829|16795|2|205| +2450829|16796|2|923| +2450829|16798|2|553| +2450829|16801|2|827| +2450829|16802|2|78| +2450829|16804|2|497| +2450829|16807|2|992| +2450829|16808|2|120| +2450829|16810|2|86| +2450829|16813|2|438| +2450829|16814|2|397| +2450829|16816|2|481| +2450829|16819|2|944| +2450829|16820|2|491| +2450829|16822|2|313| +2450829|16825|2|103| +2450829|16826|2|261| +2450829|16828|2|90| +2450829|16831|2|786| +2450829|16832|2|12| +2450829|16834|2|550| +2450829|16837|2|7| +2450829|16838|2|349| +2450829|16840|2|842| +2450829|16843|2|117| +2450829|16844|2|113| +2450829|16846|2|5| +2450829|16849|2|707| +2450829|16850|2|266| +2450829|16852|2|74| +2450829|16855|2|711| +2450829|16856|2|284| +2450829|16858|2|490| +2450829|16861|2|707| +2450829|16862|2|641| +2450829|16864|2|866| +2450829|16867|2|837| +2450829|16868|2|261| +2450829|16870|2|963| +2450829|16873|2|730| +2450829|16874|2|571| +2450829|16876|2|817| +2450829|16879|2|640| +2450829|16880|2|856| +2450829|16882|2|| +2450829|16885|2|685| +2450829|16886|2|| +2450829|16888|2|981| +2450829|16891|2|583| +2450829|16892|2|443| +2450829|16894|2|963| +2450829|16897|2|894| +2450829|16898|2|334| +2450829|16900|2|| +2450829|16903|2|79| +2450829|16904|2|| +2450829|16906|2|953| +2450829|16909|2|330| +2450829|16910|2|523| +2450829|16912|2|118| +2450829|16915|2|513| +2450829|16916|2|285| +2450829|16918|2|911| +2450829|16921|2|236| +2450829|16922|2|| +2450829|16924|2|572| +2450829|16927|2|359| +2450829|16928|2|353| +2450829|16930|2|443| +2450829|16933|2|838| +2450829|16934|2|279| +2450829|16936|2|481| +2450829|16939|2|464| +2450829|16940|2|751| +2450829|16942|2|| +2450829|16945|2|306| +2450829|16946|2|683| +2450829|16948|2|282| +2450829|16951|2|221| +2450829|16952|2|850| +2450829|16954|2|867| +2450829|16957|2|164| +2450829|16958|2|613| +2450829|16960|2|433| +2450829|16963|2|140| +2450829|16964|2|72| +2450829|16966|2|632| +2450829|16969|2|935| +2450829|16970|2|595| +2450829|16972|2|553| +2450829|16975|2|43| +2450829|16976|2|368| +2450829|16978|2|206| +2450829|16981|2|630| +2450829|16982|2|| +2450829|16984|2|939| +2450829|16987|2|678| +2450829|16988|2|48| +2450829|16990|2|913| +2450829|16993|2|193| +2450829|16994|2|19| +2450829|16996|2|758| +2450829|16999|2|315| +2450829|17000|2|710| +2450829|17002|2|819| +2450829|17005|2|371| +2450829|17006|2|272| +2450829|17008|2|442| +2450829|17011|2|190| +2450829|17012|2|50| +2450829|17014|2|938| +2450829|17017|2|831| +2450829|17018|2|493| +2450829|17020|2|965| +2450829|17023|2|427| +2450829|17024|2|871| +2450829|17026|2|987| +2450829|17029|2|92| +2450829|17030|2|635| +2450829|17032|2|620| +2450829|17035|2|548| +2450829|17036|2|| +2450829|17038|2|417| +2450829|17041|2|538| +2450829|17042|2|910| +2450829|17044|2|456| +2450829|17047|2|637| +2450829|17048|2|391| +2450829|17050|2|603| +2450829|17053|2|305| +2450829|17054|2|880| +2450829|17056|2|866| +2450829|17059|2|854| +2450829|17060|2|| +2450829|17062|2|58| +2450829|17065|2|181| +2450829|17066|2|540| +2450829|17068|2|802| +2450829|17071|2|460| +2450829|17072|2|762| +2450829|17074|2|452| +2450829|17077|2|881| +2450829|17078|2|39| +2450829|17080|2|38| +2450829|17083|2|693| +2450829|17084|2|821| +2450829|17086|2|205| +2450829|17089|2|519| +2450829|17090|2|438| +2450829|17092|2|603| +2450829|17095|2|769| +2450829|17096|2|565| +2450829|17098|2|| +2450829|17101|2|181| +2450829|17102|2|961| +2450829|17104|2|64| +2450829|17107|2|79| +2450829|17108|2|150| +2450829|17110|2|234| +2450829|17113|2|427| +2450829|17114|2|950| +2450829|17116|2|319| +2450829|17119|2|166| +2450829|17120|2|99| +2450829|17122|2|834| +2450829|17125|2|721| +2450829|17126|2|| +2450829|17128|2|250| +2450829|17131|2|132| +2450829|17132|2|421| +2450829|17134|2|140| +2450829|17137|2|132| +2450829|17138|2|125| +2450829|17140|2|471| +2450829|17143|2|| +2450829|17144|2|105| +2450829|17146|2|600| +2450829|17149|2|449| +2450829|17150|2|496| +2450829|17152|2|40| +2450829|17155|2|542| +2450829|17156|2|989| +2450829|17158|2|650| +2450829|17161|2|787| +2450829|17162|2|767| +2450829|17164|2|466| +2450829|17167|2|197| +2450829|17168|2|632| +2450829|17170|2|172| +2450829|17173|2|986| +2450829|17174|2|873| +2450829|17176|2|264| +2450829|17179|2|834| +2450829|17180|2|876| +2450829|17182|2|311| +2450829|17185|2|200| +2450829|17186|2|844| +2450829|17188|2|900| +2450829|17191|2|103| +2450829|17192|2|558| +2450829|17194|2|248| +2450829|17197|2|868| +2450829|17198|2|13| +2450829|17200|2|| +2450829|17203|2|| +2450829|17204|2|487| +2450829|17206|2|897| +2450829|17209|2|825| +2450829|17210|2|605| +2450829|17212|2|6| +2450829|17215|2|110| +2450829|17216|2|565| +2450829|17218|2|124| +2450829|17221|2|829| +2450829|17222|2|688| +2450829|17224|2|387| +2450829|17227|2|865| +2450829|17228|2|423| +2450829|17230|2|479| +2450829|17233|2|523| +2450829|17234|2|548| +2450829|17236|2|229| +2450829|17239|2|968| +2450829|17240|2|425| +2450829|17242|2|357| +2450829|17245|2|768| +2450829|17246|2|921| +2450829|17248|2|856| +2450829|17251|2|866| +2450829|17252|2|55| +2450829|17254|2|916| +2450829|17257|2|617| +2450829|17258|2|236| +2450829|17260|2|770| +2450829|17263|2|314| +2450829|17264|2|110| +2450829|17266|2|389| +2450829|17269|2|798| +2450829|17270|2|677| +2450829|17272|2|86| +2450829|17275|2|330| +2450829|17276|2|228| +2450829|17278|2|| +2450829|17281|2|768| +2450829|17282|2|611| +2450829|17284|2|122| +2450829|17287|2|689| +2450829|17288|2|292| +2450829|17290|2|615| +2450829|17293|2|152| +2450829|17294|2|642| +2450829|17296|2|988| +2450829|17299|2|923| +2450829|17300|2|749| +2450829|17302|2|250| +2450829|17305|2|258| +2450829|17306|2|441| +2450829|17308|2|958| +2450829|17311|2|231| +2450829|17312|2|132| +2450829|17314|2|459| +2450829|17317|2|48| +2450829|17318|2|996| +2450829|17320|2|582| +2450829|17323|2|| +2450829|17324|2|| +2450829|17326|2|976| +2450829|17329|2|12| +2450829|17330|2|485| +2450829|17332|2|273| +2450829|17335|2|246| +2450829|17336|2|701| +2450829|17338|2|815| +2450829|17341|2|| +2450829|17342|2|697| +2450829|17344|2|| +2450829|17347|2|885| +2450829|17348|2|332| +2450829|17350|2|459| +2450829|17353|2|428| +2450829|17354|2|720| +2450829|17356|2|179| +2450829|17359|2|767| +2450829|17360|2|23| +2450829|17362|2|810| +2450829|17365|2|982| +2450829|17366|2|864| +2450829|17368|2|303| +2450829|17371|2|525| +2450829|17372|2|508| +2450829|17374|2|263| +2450829|17377|2|488| +2450829|17378|2|300| +2450829|17380|2|302| +2450829|17383|2|340| +2450829|17384|2|660| +2450829|17386|2|323| +2450829|17389|2|683| +2450829|17390|2|184| +2450829|17392|2|| +2450829|17395|2|597| +2450829|17396|2|584| +2450829|17398|2|261| +2450829|17401|2|538| +2450829|17402|2|29| +2450829|17404|2|459| +2450829|17407|2|576| +2450829|17408|2|245| +2450829|17410|2|48| +2450829|17413|2|22| +2450829|17414|2|423| +2450829|17416|2|203| +2450829|17419|2|487| +2450829|17420|2|656| +2450829|17422|2|936| +2450829|17425|2|464| +2450829|17426|2|696| +2450829|17428|2|615| +2450829|17431|2|713| +2450829|17432|2|377| +2450829|17434|2|408| +2450829|17437|2|692| +2450829|17438|2|775| +2450829|17440|2|151| +2450829|17443|2|146| +2450829|17444|2|258| +2450829|17446|2|888| +2450829|17449|2|743| +2450829|17450|2|460| +2450829|17452|2|698| +2450829|17455|2|759| +2450829|17456|2|592| +2450829|17458|2|414| +2450829|17461|2|345| +2450829|17462|2|580| +2450829|17464|2|990| +2450829|17467|2|95| +2450829|17468|2|360| +2450829|17470|2|641| +2450829|17473|2|306| +2450829|17474|2|803| +2450829|17476|2|862| +2450829|17479|2|327| +2450829|17480|2|779| +2450829|17482|2|649| +2450829|17485|2|271| +2450829|17486|2|366| +2450829|17488|2|8| +2450829|17491|2|894| +2450829|17492|2|414| +2450829|17494|2|567| +2450829|17497|2|949| +2450829|17498|2|626| +2450829|17500|2|69| +2450829|17503|2|214| +2450829|17504|2|718| +2450829|17506|2|180| +2450829|17509|2|490| +2450829|17510|2|36| +2450829|17512|2|| +2450829|17515|2|541| +2450829|17516|2|467| +2450829|17518|2|441| +2450829|17521|2|955| +2450829|17522|2|272| +2450829|17524|2|20| +2450829|17527|2|337| +2450829|17528|2|796| +2450829|17530|2|474| +2450829|17533|2|269| +2450829|17534|2|658| +2450829|17536|2|853| +2450829|17539|2|783| +2450829|17540|2|430| +2450829|17542|2|553| +2450829|17545|2|387| +2450829|17546|2|304| +2450829|17548|2|463| +2450829|17551|2|919| +2450829|17552|2|995| +2450829|17554|2|475| +2450829|17557|2|683| +2450829|17558|2|856| +2450829|17560|2|806| +2450829|17563|2|| +2450829|17564|2|49| +2450829|17566|2|958| +2450829|17569|2|71| +2450829|17570|2|615| +2450829|17572|2|9| +2450829|17575|2|760| +2450829|17576|2|204| +2450829|17578|2|288| +2450829|17581|2|422| +2450829|17582|2|| +2450829|17584|2|383| +2450829|17587|2|334| +2450829|17588|2|667| +2450829|17590|2|324| +2450829|17593|2|973| +2450829|17594|2|420| +2450829|17596|2|542| +2450829|17599|2|529| +2450829|17600|2|647| +2450829|17602|2|223| +2450829|17605|2|424| +2450829|17606|2|227| +2450829|17608|2|311| +2450829|17611|2|287| +2450829|17612|2|997| +2450829|17614|2|600| +2450829|17617|2|757| +2450829|17618|2|443| +2450829|17620|2|207| +2450829|17623|2|102| +2450829|17624|2|434| +2450829|17626|2|282| +2450829|17629|2|89| +2450829|17630|2|691| +2450829|17632|2|239| +2450829|17635|2|580| +2450829|17636|2|307| +2450829|17638|2|187| +2450829|17641|2|206| +2450829|17642|2|1| +2450829|17644|2|695| +2450829|17647|2|645| +2450829|17648|2|188| +2450829|17650|2|205| +2450829|17653|2|36| +2450829|17654|2|111| +2450829|17656|2|379| +2450829|17659|2|462| +2450829|17660|2|273| +2450829|17662|2|364| +2450829|17665|2|586| +2450829|17666|2|409| +2450829|17668|2|914| +2450829|17671|2|448| +2450829|17672|2|86| +2450829|17674|2|533| +2450829|17677|2|402| +2450829|17678|2|398| +2450829|17680|2|110| +2450829|17683|2|39| +2450829|17684|2|664| +2450829|17686|2|918| +2450829|17689|2|38| +2450829|17690|2|70| +2450829|17692|2|| +2450829|17695|2|959| +2450829|17696|2|983| +2450829|17698|2|656| +2450829|17701|2|| +2450829|17702|2|75| +2450829|17704|2|421| +2450829|17707|2|287| +2450829|17708|2|192| +2450829|17710|2|561| +2450829|17713|2|| +2450829|17714|2|426| +2450829|17716|2|876| +2450829|17719|2|917| +2450829|17720|2|171| +2450829|17722|2|635| +2450829|17725|2|57| +2450829|17726|2|574| +2450829|17728|2|| +2450829|17731|2|338| +2450829|17732|2|190| +2450829|17734|2|803| +2450829|17737|2|484| +2450829|17738|2|658| +2450829|17740|2|595| +2450829|17743|2|84| +2450829|17744|2|870| +2450829|17746|2|646| +2450829|17749|2|218| +2450829|17750|2|334| +2450829|17752|2|653| +2450829|17755|2|633| +2450829|17756|2|162| +2450829|17758|2|165| +2450829|17761|2|948| +2450829|17762|2|627| +2450829|17764|2|638| +2450829|17767|2|77| +2450829|17768|2|829| +2450829|17770|2|| +2450829|17773|2|871| +2450829|17774|2|794| +2450829|17776|2|391| +2450829|17779|2|590| +2450829|17780|2|20| +2450829|17782|2|662| +2450829|17785|2|467| +2450829|17786|2|502| +2450829|17788|2|980| +2450829|17791|2|928| +2450829|17792|2|751| +2450829|17794|2|35| +2450829|17797|2|243| +2450829|17798|2|| +2450829|17800|2|516| +2450829|17803|2|993| +2450829|17804|2|61| +2450829|17806|2|131| +2450829|17809|2|886| +2450829|17810|2|660| +2450829|17812|2|55| +2450829|17815|2|46| +2450829|17816|2|520| +2450829|17818|2|844| +2450829|17821|2|972| +2450829|17822|2|515| +2450829|17824|2|885| +2450829|17827|2|788| +2450829|17828|2|848| +2450829|17830|2|146| +2450829|17833|2|171| +2450829|17834|2|| +2450829|17836|2|228| +2450829|17839|2|58| +2450829|17840|2|462| +2450829|17842|2|674| +2450829|17845|2|958| +2450829|17846|2|855| +2450829|17848|2|471| +2450829|17851|2|447| +2450829|17852|2|422| +2450829|17854|2|101| +2450829|17857|2|850| +2450829|17858|2|160| +2450829|17860|2|859| +2450829|17863|2|777| +2450829|17864|2|35| +2450829|17866|2|242| +2450829|17869|2|760| +2450829|17870|2|565| +2450829|17872|2|363| +2450829|17875|2|676| +2450829|17876|2|926| +2450829|17878|2|45| +2450829|17881|2|12| +2450829|17882|2|306| +2450829|17884|2|| +2450829|17887|2|655| +2450829|17888|2|817| +2450829|17890|2|423| +2450829|17893|2|917| +2450829|17894|2|11| +2450829|17896|2|55| +2450829|17899|2|370| +2450829|17900|2|213| +2450829|17902|2|405| +2450829|17905|2|697| +2450829|17906|2|591| +2450829|17908|2|586| +2450829|17911|2|225| +2450829|17912|2|184| +2450829|17914|2|596| +2450829|17917|2|456| +2450829|17918|2|549| +2450829|17920|2|| +2450829|17923|2|412| +2450829|17924|2|823| +2450829|17926|2|919| +2450829|17929|2|109| +2450829|17930|2|850| +2450829|17932|2|186| +2450829|17935|2|35| +2450829|17936|2|258| +2450829|17938|2|588| +2450829|17941|2|205| +2450829|17942|2|| +2450829|17944|2|93| +2450829|17947|2|86| +2450829|17948|2|803| +2450829|17950|2|572| +2450829|17953|2|342| +2450829|17954|2|238| +2450829|17956|2|848| +2450829|17959|2|992| +2450829|17960|2|492| +2450829|17962|2|| +2450829|17965|2|321| +2450829|17966|2|135| +2450829|17968|2|286| +2450829|17971|2|257| +2450829|17972|2|687| +2450829|17974|2|698| +2450829|17977|2|16| +2450829|17978|2|375| +2450829|17980|2|916| +2450829|17983|2|81| +2450829|17984|2|598| +2450829|17986|2|503| +2450829|17989|2|18| +2450829|17990|2|931| +2450829|17992|2|902| +2450829|17995|2|258| +2450829|17996|2|238| +2450829|17998|2|768| +2450829|1|3|759| +2450829|2|3|298| +2450829|4|3|531| +2450829|7|3|478| +2450829|8|3|237| +2450829|10|3|473| +2450829|13|3|894| +2450829|14|3|437| +2450829|16|3|973| +2450829|19|3|471| +2450829|20|3|894| +2450829|22|3|804| +2450829|25|3|106| +2450829|26|3|| +2450829|28|3|331| +2450829|31|3|239| +2450829|32|3|411| +2450829|34|3|50| +2450829|37|3|529| +2450829|38|3|894| +2450829|40|3|780| +2450829|43|3|849| +2450829|44|3|408| +2450829|46|3|425| +2450829|49|3|604| +2450829|50|3|639| +2450829|52|3|782| +2450829|55|3|636| +2450829|56|3|928| +2450829|58|3|775| +2450829|61|3|304| +2450829|62|3|160| +2450829|64|3|801| +2450829|67|3|804| +2450829|68|3|347| +2450829|70|3|79| +2450829|73|3|611| +2450829|74|3|456| +2450829|76|3|675| +2450829|79|3|42| +2450829|80|3|603| +2450829|82|3|446| +2450829|85|3|609| +2450829|86|3|908| +2450829|88|3|346| +2450829|91|3|221| +2450829|92|3|254| +2450829|94|3|158| +2450829|97|3|564| +2450829|98|3|588| +2450829|100|3|789| +2450829|103|3|593| +2450829|104|3|607| +2450829|106|3|451| +2450829|109|3|623| +2450829|110|3|679| +2450829|112|3|669| +2450829|115|3|401| +2450829|116|3|988| +2450829|118|3|880| +2450829|121|3|655| +2450829|122|3|599| +2450829|124|3|294| +2450829|127|3|911| +2450829|128|3|848| +2450829|130|3|184| +2450829|133|3|391| +2450829|134|3|674| +2450829|136|3|744| +2450829|139|3|778| +2450829|140|3|282| +2450829|142|3|744| +2450829|145|3|290| +2450829|146|3|541| +2450829|148|3|786| +2450829|151|3|788| +2450829|152|3|713| +2450829|154|3|160| +2450829|157|3|413| +2450829|158|3|977| +2450829|160|3|101| +2450829|163|3|453| +2450829|164|3|833| +2450829|166|3|43| +2450829|169|3|225| +2450829|170|3|665| +2450829|172|3|426| +2450829|175|3|754| +2450829|176|3|| +2450829|178|3|981| +2450829|181|3|538| +2450829|182|3|570| +2450829|184|3|433| +2450829|187|3|133| +2450829|188|3|446| +2450829|190|3|31| +2450829|193|3|937| +2450829|194|3|707| +2450829|196|3|966| +2450829|199|3|584| +2450829|200|3|443| +2450829|202|3|879| +2450829|205|3|535| +2450829|206|3|641| +2450829|208|3|360| +2450829|211|3|754| +2450829|212|3|3| +2450829|214|3|287| +2450829|217|3|736| +2450829|218|3|10| +2450829|220|3|808| +2450829|223|3|691| +2450829|224|3|133| +2450829|226|3|| +2450829|229|3|568| +2450829|230|3|437| +2450829|232|3|80| +2450829|235|3|896| +2450829|236|3|435| +2450829|238|3|995| +2450829|241|3|93| +2450829|242|3|150| +2450829|244|3|217| +2450829|247|3|212| +2450829|248|3|831| +2450829|250|3|186| +2450829|253|3|508| +2450829|254|3|181| +2450829|256|3|352| +2450829|259|3|373| +2450829|260|3|225| +2450829|262|3|| +2450829|265|3|863| +2450829|266|3|613| +2450829|268|3|497| +2450829|271|3|760| +2450829|272|3|370| +2450829|274|3|789| +2450829|277|3|885| +2450829|278|3|746| +2450829|280|3|699| +2450829|283|3|903| +2450829|284|3|346| +2450829|286|3|110| +2450829|289|3|870| +2450829|290|3|158| +2450829|292|3|518| +2450829|295|3|270| +2450829|296|3|302| +2450829|298|3|491| +2450829|301|3|305| +2450829|302|3|953| +2450829|304|3|253| +2450829|307|3|214| +2450829|308|3|453| +2450829|310|3|886| +2450829|313|3|573| +2450829|314|3|740| +2450829|316|3|| +2450829|319|3|478| +2450829|320|3|| +2450829|322|3|377| +2450829|325|3|837| +2450829|326|3|15| +2450829|328|3|| +2450829|331|3|715| +2450829|332|3|26| +2450829|334|3|875| +2450829|337|3|389| +2450829|338|3|530| +2450829|340|3|803| +2450829|343|3|661| +2450829|344|3|546| +2450829|346|3|837| +2450829|349|3|667| +2450829|350|3|568| +2450829|352|3|883| +2450829|355|3|471| +2450829|356|3|40| +2450829|358|3|810| +2450829|361|3|297| +2450829|362|3|915| +2450829|364|3|390| +2450829|367|3|269| +2450829|368|3|803| +2450829|370|3|334| +2450829|373|3|876| +2450829|374|3|466| +2450829|376|3|965| +2450829|379|3|588| +2450829|380|3|203| +2450829|382|3|171| +2450829|385|3|759| +2450829|386|3|89| +2450829|388|3|557| +2450829|391|3|995| +2450829|392|3|631| +2450829|394|3|946| +2450829|397|3|13| +2450829|398|3|825| +2450829|400|3|641| +2450829|403|3|677| +2450829|404|3|77| +2450829|406|3|258| +2450829|409|3|805| +2450829|410|3|595| +2450829|412|3|722| +2450829|415|3|941| +2450829|416|3|888| +2450829|418|3|858| +2450829|421|3|682| +2450829|422|3|866| +2450829|424|3|568| +2450829|427|3|728| +2450829|428|3|144| +2450829|430|3|390| +2450829|433|3|241| +2450829|434|3|726| +2450829|436|3|432| +2450829|439|3|492| +2450829|440|3|996| +2450829|442|3|664| +2450829|445|3|653| +2450829|446|3|796| +2450829|448|3|630| +2450829|451|3|599| +2450829|452|3|235| +2450829|454|3|236| +2450829|457|3|686| +2450829|458|3|766| +2450829|460|3|144| +2450829|463|3|710| +2450829|464|3|677| +2450829|466|3|295| +2450829|469|3|524| +2450829|470|3|830| +2450829|472|3|937| +2450829|475|3|| +2450829|476|3|566| +2450829|478|3|831| +2450829|481|3|817| +2450829|482|3|548| +2450829|484|3|60| +2450829|487|3|924| +2450829|488|3|103| +2450829|490|3|363| +2450829|493|3|285| +2450829|494|3|110| +2450829|496|3|728| +2450829|499|3|210| +2450829|500|3|634| +2450829|502|3|755| +2450829|505|3|945| +2450829|506|3|963| +2450829|508|3|852| +2450829|511|3|106| +2450829|512|3|495| +2450829|514|3|186| +2450829|517|3|549| +2450829|518|3|366| +2450829|520|3|18| +2450829|523|3|467| +2450829|524|3|404| +2450829|526|3|160| +2450829|529|3|798| +2450829|530|3|693| +2450829|532|3|688| +2450829|535|3|238| +2450829|536|3|519| +2450829|538|3|716| +2450829|541|3|172| +2450829|542|3|632| +2450829|544|3|793| +2450829|547|3|984| +2450829|548|3|478| +2450829|550|3|901| +2450829|553|3|54| +2450829|554|3|692| +2450829|556|3|| +2450829|559|3|331| +2450829|560|3|104| +2450829|562|3|925| +2450829|565|3|964| +2450829|566|3|124| +2450829|568|3|715| +2450829|571|3|813| +2450829|572|3|768| +2450829|574|3|211| +2450829|577|3|920| +2450829|578|3|227| +2450829|580|3|944| +2450829|583|3|247| +2450829|584|3|651| +2450829|586|3|486| +2450829|589|3|939| +2450829|590|3|830| +2450829|592|3|800| +2450829|595|3|97| +2450829|596|3|533| +2450829|598|3|159| +2450829|601|3|788| +2450829|602|3|798| +2450829|604|3|517| +2450829|607|3|60| +2450829|608|3|176| +2450829|610|3|656| +2450829|613|3|726| +2450829|614|3|233| +2450829|616|3|787| +2450829|619|3|283| +2450829|620|3|912| +2450829|622|3|614| +2450829|625|3|574| +2450829|626|3|424| +2450829|628|3|768| +2450829|631|3|265| +2450829|632|3|118| +2450829|634|3|226| +2450829|637|3|571| +2450829|638|3|750| +2450829|640|3|534| +2450829|643|3|603| +2450829|644|3|46| +2450829|646|3|679| +2450829|649|3|642| +2450829|650|3|20| +2450829|652|3|729| +2450829|655|3|589| +2450829|656|3|993| +2450829|658|3|838| +2450829|661|3|342| +2450829|662|3|190| +2450829|664|3|124| +2450829|667|3|851| +2450829|668|3|984| +2450829|670|3|158| +2450829|673|3|142| +2450829|674|3|213| +2450829|676|3|346| +2450829|679|3|935| +2450829|680|3|319| +2450829|682|3|98| +2450829|685|3|90| +2450829|686|3|| +2450829|688|3|962| +2450829|691|3|165| +2450829|692|3|263| +2450829|694|3|428| +2450829|697|3|929| +2450829|698|3|484| +2450829|700|3|490| +2450829|703|3|107| +2450829|704|3|310| +2450829|706|3|869| +2450829|709|3|912| +2450829|710|3|219| +2450829|712|3|70| +2450829|715|3|13| +2450829|716|3|301| +2450829|718|3|874| +2450829|721|3|793| +2450829|722|3|269| +2450829|724|3|782| +2450829|727|3|325| +2450829|728|3|341| +2450829|730|3|827| +2450829|733|3|339| +2450829|734|3|523| +2450829|736|3|266| +2450829|739|3|240| +2450829|740|3|719| +2450829|742|3|| +2450829|745|3|223| +2450829|746|3|535| +2450829|748|3|351| +2450829|751|3|331| +2450829|752|3|107| +2450829|754|3|298| +2450829|757|3|754| +2450829|758|3|782| +2450829|760|3|992| +2450829|763|3|692| +2450829|764|3|76| +2450829|766|3|853| +2450829|769|3|349| +2450829|770|3|| +2450829|772|3|417| +2450829|775|3|435| +2450829|776|3|106| +2450829|778|3|96| +2450829|781|3|239| +2450829|782|3|709| +2450829|784|3|| +2450829|787|3|834| +2450829|788|3|| +2450829|790|3|879| +2450829|793|3|876| +2450829|794|3|195| +2450829|796|3|| +2450829|799|3|241| +2450829|800|3|207| +2450829|802|3|370| +2450829|805|3|485| +2450829|806|3|343| +2450829|808|3|468| +2450829|811|3|545| +2450829|812|3|151| +2450829|814|3|864| +2450829|817|3|818| +2450829|818|3|371| +2450829|820|3|899| +2450829|823|3|758| +2450829|824|3|785| +2450829|826|3|670| +2450829|829|3|142| +2450829|830|3|587| +2450829|832|3|15| +2450829|835|3|39| +2450829|836|3|902| +2450829|838|3|667| +2450829|841|3|923| +2450829|842|3|133| +2450829|844|3|915| +2450829|847|3|954| +2450829|848|3|403| +2450829|850|3|458| +2450829|853|3|768| +2450829|854|3|394| +2450829|856|3|380| +2450829|859|3|134| +2450829|860|3|671| +2450829|862|3|88| +2450829|865|3|445| +2450829|866|3|607| +2450829|868|3|590| +2450829|871|3|848| +2450829|872|3|214| +2450829|874|3|716| +2450829|877|3|0| +2450829|878|3|45| +2450829|880|3|82| +2450829|883|3|342| +2450829|884|3|432| +2450829|886|3|222| +2450829|889|3|158| +2450829|890|3|333| +2450829|892|3|677| +2450829|895|3|815| +2450829|896|3|906| +2450829|898|3|576| +2450829|901|3|973| +2450829|902|3|175| +2450829|904|3|68| +2450829|907|3|104| +2450829|908|3|659| +2450829|910|3|968| +2450829|913|3|754| +2450829|914|3|| +2450829|916|3|572| +2450829|919|3|459| +2450829|920|3|234| +2450829|922|3|761| +2450829|925|3|| +2450829|926|3|896| +2450829|928|3|71| +2450829|931|3|810| +2450829|932|3|288| +2450829|934|3|449| +2450829|937|3|255| +2450829|938|3|468| +2450829|940|3|628| +2450829|943|3|206| +2450829|944|3|177| +2450829|946|3|604| +2450829|949|3|136| +2450829|950|3|849| +2450829|952|3|700| +2450829|955|3|398| +2450829|956|3|393| +2450829|958|3|298| +2450829|961|3|353| +2450829|962|3|253| +2450829|964|3|97| +2450829|967|3|364| +2450829|968|3|676| +2450829|970|3|291| +2450829|973|3|257| +2450829|974|3|720| +2450829|976|3|330| +2450829|979|3|331| +2450829|980|3|227| +2450829|982|3|289| +2450829|985|3|971| +2450829|986|3|418| +2450829|988|3|145| +2450829|991|3|367| +2450829|992|3|157| +2450829|994|3|575| +2450829|997|3|186| +2450829|998|3|710| +2450829|1000|3|459| +2450829|1003|3|803| +2450829|1004|3|341| +2450829|1006|3|748| +2450829|1009|3|331| +2450829|1010|3|121| +2450829|1012|3|379| +2450829|1015|3|201| +2450829|1016|3|699| +2450829|1018|3|689| +2450829|1021|3|983| +2450829|1022|3|975| +2450829|1024|3|118| +2450829|1027|3|| +2450829|1028|3|998| +2450829|1030|3|419| +2450829|1033|3|547| +2450829|1034|3|859| +2450829|1036|3|689| +2450829|1039|3|0| +2450829|1040|3|773| +2450829|1042|3|71| +2450829|1045|3|565| +2450829|1046|3|770| +2450829|1048|3|978| +2450829|1051|3|| +2450829|1052|3|819| +2450829|1054|3|169| +2450829|1057|3|871| +2450829|1058|3|616| +2450829|1060|3|214| +2450829|1063|3|544| +2450829|1064|3|337| +2450829|1066|3|718| +2450829|1069|3|569| +2450829|1070|3|99| +2450829|1072|3|661| +2450829|1075|3|579| +2450829|1076|3|| +2450829|1078|3|658| +2450829|1081|3|416| +2450829|1082|3|38| +2450829|1084|3|592| +2450829|1087|3|755| +2450829|1088|3|825| +2450829|1090|3|123| +2450829|1093|3|155| +2450829|1094|3|849| +2450829|1096|3|284| +2450829|1099|3|244| +2450829|1100|3|747| +2450829|1102|3|816| +2450829|1105|3|880| +2450829|1106|3|579| +2450829|1108|3|759| +2450829|1111|3|603| +2450829|1112|3|642| +2450829|1114|3|690| +2450829|1117|3|558| +2450829|1118|3|982| +2450829|1120|3|757| +2450829|1123|3|892| +2450829|1124|3|775| +2450829|1126|3|937| +2450829|1129|3|117| +2450829|1130|3|888| +2450829|1132|3|553| +2450829|1135|3|877| +2450829|1136|3|368| +2450829|1138|3|844| +2450829|1141|3|362| +2450829|1142|3|6| +2450829|1144|3|356| +2450829|1147|3|826| +2450829|1148|3|342| +2450829|1150|3|212| +2450829|1153|3|163| +2450829|1154|3|382| +2450829|1156|3|78| +2450829|1159|3|69| +2450829|1160|3|383| +2450829|1162|3|292| +2450829|1165|3|381| +2450829|1166|3|| +2450829|1168|3|579| +2450829|1171|3|27| +2450829|1172|3|842| +2450829|1174|3|626| +2450829|1177|3|14| +2450829|1178|3|426| +2450829|1180|3|324| +2450829|1183|3|943| +2450829|1184|3|400| +2450829|1186|3|844| +2450829|1189|3|616| +2450829|1190|3|599| +2450829|1192|3|57| +2450829|1195|3|859| +2450829|1196|3|34| +2450829|1198|3|204| +2450829|1201|3|9| +2450829|1202|3|856| +2450829|1204|3|943| +2450829|1207|3|542| +2450829|1208|3|611| +2450829|1210|3|355| +2450829|1213|3|680| +2450829|1214|3|54| +2450829|1216|3|360| +2450829|1219|3|783| +2450829|1220|3|455| +2450829|1222|3|558| +2450829|1225|3|474| +2450829|1226|3|366| +2450829|1228|3|873| +2450829|1231|3|618| +2450829|1232|3|771| +2450829|1234|3|178| +2450829|1237|3|| +2450829|1238|3|553| +2450829|1240|3|954| +2450829|1243|3|885| +2450829|1244|3|91| +2450829|1246|3|709| +2450829|1249|3|484| +2450829|1250|3|135| +2450829|1252|3|667| +2450829|1255|3|470| +2450829|1256|3|320| +2450829|1258|3|380| +2450829|1261|3|380| +2450829|1262|3|852| +2450829|1264|3|243| +2450829|1267|3|677| +2450829|1268|3|733| +2450829|1270|3|427| +2450829|1273|3|40| +2450829|1274|3|276| +2450829|1276|3|960| +2450829|1279|3|155| +2450829|1280|3|| +2450829|1282|3|264| +2450829|1285|3|169| +2450829|1286|3|786| +2450829|1288|3|| +2450829|1291|3|478| +2450829|1292|3|387| +2450829|1294|3|823| +2450829|1297|3|194| +2450829|1298|3|118| +2450829|1300|3|| +2450829|1303|3|841| +2450829|1304|3|359| +2450829|1306|3|| +2450829|1309|3|848| +2450829|1310|3|300| +2450829|1312|3|559| +2450829|1315|3|452| +2450829|1316|3|686| +2450829|1318|3|| +2450829|1321|3|995| +2450829|1322|3|420| +2450829|1324|3|131| +2450829|1327|3|881| +2450829|1328|3|955| +2450829|1330|3|710| +2450829|1333|3|14| +2450829|1334|3|944| +2450829|1336|3|991| +2450829|1339|3|50| +2450829|1340|3|252| +2450829|1342|3|978| +2450829|1345|3|722| +2450829|1346|3|541| +2450829|1348|3|673| +2450829|1351|3|744| +2450829|1352|3|237| +2450829|1354|3|| +2450829|1357|3|592| +2450829|1358|3|454| +2450829|1360|3|| +2450829|1363|3|162| +2450829|1364|3|781| +2450829|1366|3|705| +2450829|1369|3|975| +2450829|1370|3|54| +2450829|1372|3|857| +2450829|1375|3|636| +2450829|1376|3|192| +2450829|1378|3|962| +2450829|1381|3|83| +2450829|1382|3|331| +2450829|1384|3|940| +2450829|1387|3|394| +2450829|1388|3|96| +2450829|1390|3|47| +2450829|1393|3|964| +2450829|1394|3|558| +2450829|1396|3|265| +2450829|1399|3|68| +2450829|1400|3|259| +2450829|1402|3|985| +2450829|1405|3|936| +2450829|1406|3|248| +2450829|1408|3|| +2450829|1411|3|79| +2450829|1412|3|926| +2450829|1414|3|678| +2450829|1417|3|248| +2450829|1418|3|426| +2450829|1420|3|995| +2450829|1423|3|| +2450829|1424|3|100| +2450829|1426|3|371| +2450829|1429|3|591| +2450829|1430|3|899| +2450829|1432|3|595| +2450829|1435|3|260| +2450829|1436|3|185| +2450829|1438|3|386| +2450829|1441|3|497| +2450829|1442|3|314| +2450829|1444|3|102| +2450829|1447|3|364| +2450829|1448|3|785| +2450829|1450|3|349| +2450829|1453|3|276| +2450829|1454|3|997| +2450829|1456|3|838| +2450829|1459|3|489| +2450829|1460|3|| +2450829|1462|3|922| +2450829|1465|3|11| +2450829|1466|3|83| +2450829|1468|3|965| +2450829|1471|3|357| +2450829|1472|3|462| +2450829|1474|3|586| +2450829|1477|3|385| +2450829|1478|3|881| +2450829|1480|3|12| +2450829|1483|3|631| +2450829|1484|3|945| +2450829|1486|3|893| +2450829|1489|3|353| +2450829|1490|3|19| +2450829|1492|3|656| +2450829|1495|3|866| +2450829|1496|3|700| +2450829|1498|3|27| +2450829|1501|3|987| +2450829|1502|3|682| +2450829|1504|3|991| +2450829|1507|3|337| +2450829|1508|3|430| +2450829|1510|3|422| +2450829|1513|3|997| +2450829|1514|3|320| +2450829|1516|3|417| +2450829|1519|3|217| +2450829|1520|3|308| +2450829|1522|3|483| +2450829|1525|3|515| +2450829|1526|3|782| +2450829|1528|3|210| +2450829|1531|3|698| +2450829|1532|3|201| +2450829|1534|3|883| +2450829|1537|3|913| +2450829|1538|3|64| +2450829|1540|3|286| +2450829|1543|3|| +2450829|1544|3|953| +2450829|1546|3|571| +2450829|1549|3|415| +2450829|1550|3|119| +2450829|1552|3|755| +2450829|1555|3|820| +2450829|1556|3|338| +2450829|1558|3|747| +2450829|1561|3|974| +2450829|1562|3|667| +2450829|1564|3|851| +2450829|1567|3|805| +2450829|1568|3|660| +2450829|1570|3|312| +2450829|1573|3|207| +2450829|1574|3|577| +2450829|1576|3|655| +2450829|1579|3|430| +2450829|1580|3|620| +2450829|1582|3|901| +2450829|1585|3|285| +2450829|1586|3|| +2450829|1588|3|579| +2450829|1591|3|901| +2450829|1592|3|188| +2450829|1594|3|847| +2450829|1597|3|748| +2450829|1598|3|298| +2450829|1600|3|650| +2450829|1603|3|154| +2450829|1604|3|460| +2450829|1606|3|102| +2450829|1609|3|296| +2450829|1610|3|416| +2450829|1612|3|115| +2450829|1615|3|900| +2450829|1616|3|925| +2450829|1618|3|595| +2450829|1621|3|745| +2450829|1622|3|701| +2450829|1624|3|304| +2450829|1627|3|| +2450829|1628|3|| +2450829|1630|3|562| +2450829|1633|3|865| +2450829|1634|3|654| +2450829|1636|3|70| +2450829|1639|3|879| +2450829|1640|3|343| +2450829|1642|3|803| +2450829|1645|3|683| +2450829|1646|3|996| +2450829|1648|3|466| +2450829|1651|3|130| +2450829|1652|3|662| +2450829|1654|3|256| +2450829|1657|3|| +2450829|1658|3|502| +2450829|1660|3|220| +2450829|1663|3|154| +2450829|1664|3|180| +2450829|1666|3|753| +2450829|1669|3|260| +2450829|1670|3|11| +2450829|1672|3|779| +2450829|1675|3|621| +2450829|1676|3|418| +2450829|1678|3|884| +2450829|1681|3|531| +2450829|1682|3|50| +2450829|1684|3|35| +2450829|1687|3|569| +2450829|1688|3|984| +2450829|1690|3|65| +2450829|1693|3|28| +2450829|1694|3|562| +2450829|1696|3|238| +2450829|1699|3|254| +2450829|1700|3|855| +2450829|1702|3|588| +2450829|1705|3|162| +2450829|1706|3|816| +2450829|1708|3|39| +2450829|1711|3|417| +2450829|1712|3|| +2450829|1714|3|300| +2450829|1717|3|777| +2450829|1718|3|578| +2450829|1720|3|992| +2450829|1723|3|734| +2450829|1724|3|122| +2450829|1726|3|305| +2450829|1729|3|911| +2450829|1730|3|152| +2450829|1732|3|170| +2450829|1735|3|13| +2450829|1736|3|480| +2450829|1738|3|281| +2450829|1741|3|642| +2450829|1742|3|763| +2450829|1744|3|692| +2450829|1747|3|30| +2450829|1748|3|956| +2450829|1750|3|969| +2450829|1753|3|486| +2450829|1754|3|831| +2450829|1756|3|276| +2450829|1759|3|164| +2450829|1760|3|863| +2450829|1762|3|49| +2450829|1765|3|| +2450829|1766|3|576| +2450829|1768|3|409| +2450829|1771|3|255| +2450829|1772|3|326| +2450829|1774|3|69| +2450829|1777|3|678| +2450829|1778|3|974| +2450829|1780|3|245| +2450829|1783|3|865| +2450829|1784|3|195| +2450829|1786|3|498| +2450829|1789|3|345| +2450829|1790|3|927| +2450829|1792|3|725| +2450829|1795|3|408| +2450829|1796|3|864| +2450829|1798|3|37| +2450829|1801|3|270| +2450829|1802|3|107| +2450829|1804|3|717| +2450829|1807|3|602| +2450829|1808|3|98| +2450829|1810|3|622| +2450829|1813|3|695| +2450829|1814|3|228| +2450829|1816|3|274| +2450829|1819|3|291| +2450829|1820|3|977| +2450829|1822|3|761| +2450829|1825|3|24| +2450829|1826|3|913| +2450829|1828|3|278| +2450829|1831|3|261| +2450829|1832|3|721| +2450829|1834|3|640| +2450829|1837|3|305| +2450829|1838|3|691| +2450829|1840|3|979| +2450829|1843|3|428| +2450829|1844|3|224| +2450829|1846|3|427| +2450829|1849|3|243| +2450829|1850|3|838| +2450829|1852|3|505| +2450829|1855|3|789| +2450829|1856|3|857| +2450829|1858|3|71| +2450829|1861|3|50| +2450829|1862|3|393| +2450829|1864|3|16| +2450829|1867|3|162| +2450829|1868|3|771| +2450829|1870|3|177| +2450829|1873|3|166| +2450829|1874|3|174| +2450829|1876|3|398| +2450829|1879|3|517| +2450829|1880|3|739| +2450829|1882|3|| +2450829|1885|3|539| +2450829|1886|3|723| +2450829|1888|3|804| +2450829|1891|3|760| +2450829|1892|3|947| +2450829|1894|3|225| +2450829|1897|3|615| +2450829|1898|3|309| +2450829|1900|3|825| +2450829|1903|3|27| +2450829|1904|3|508| +2450829|1906|3|989| +2450829|1909|3|656| +2450829|1910|3|819| +2450829|1912|3|919| +2450829|1915|3|860| +2450829|1916|3|89| +2450829|1918|3|520| +2450829|1921|3|770| +2450829|1922|3|986| +2450829|1924|3|808| +2450829|1927|3|341| +2450829|1928|3|848| +2450829|1930|3|383| +2450829|1933|3|801| +2450829|1934|3|29| +2450829|1936|3|666| +2450829|1939|3|59| +2450829|1940|3|954| +2450829|1942|3|499| +2450829|1945|3|953| +2450829|1946|3|989| +2450829|1948|3|271| +2450829|1951|3|479| +2450829|1952|3|592| +2450829|1954|3|786| +2450829|1957|3|136| +2450829|1958|3|779| +2450829|1960|3|346| +2450829|1963|3|909| +2450829|1964|3|699| +2450829|1966|3|756| +2450829|1969|3|556| +2450829|1970|3|170| +2450829|1972|3|144| +2450829|1975|3|998| +2450829|1976|3|490| +2450829|1978|3|183| +2450829|1981|3|204| +2450829|1982|3|496| +2450829|1984|3|634| +2450829|1987|3|228| +2450829|1988|3|128| +2450829|1990|3|348| +2450829|1993|3|657| +2450829|1994|3|518| +2450829|1996|3|932| +2450829|1999|3|462| +2450829|2000|3|857| +2450829|2002|3|621| +2450829|2005|3|678| +2450829|2006|3|926| +2450829|2008|3|326| +2450829|2011|3|769| +2450829|2012|3|669| +2450829|2014|3|547| +2450829|2017|3|52| +2450829|2018|3|807| +2450829|2020|3|315| +2450829|2023|3|124| +2450829|2024|3|827| +2450829|2026|3|766| +2450829|2029|3|477| +2450829|2030|3|474| +2450829|2032|3|33| +2450829|2035|3|638| +2450829|2036|3|773| +2450829|2038|3|580| +2450829|2041|3|| +2450829|2042|3|898| +2450829|2044|3|276| +2450829|2047|3|256| +2450829|2048|3|523| +2450829|2050|3|57| +2450829|2053|3|144| +2450829|2054|3|499| +2450829|2056|3|746| +2450829|2059|3|748| +2450829|2060|3|| +2450829|2062|3|184| +2450829|2065|3|| +2450829|2066|3|| +2450829|2068|3|334| +2450829|2071|3|| +2450829|2072|3|128| +2450829|2074|3|387| +2450829|2077|3|457| +2450829|2078|3|188| +2450829|2080|3|363| +2450829|2083|3|272| +2450829|2084|3|0| +2450829|2086|3|227| +2450829|2089|3|178| +2450829|2090|3|260| +2450829|2092|3|527| +2450829|2095|3|396| +2450829|2096|3|206| +2450829|2098|3|382| +2450829|2101|3|206| +2450829|2102|3|344| +2450829|2104|3|979| +2450829|2107|3|17| +2450829|2108|3|887| +2450829|2110|3|965| +2450829|2113|3|974| +2450829|2114|3|259| +2450829|2116|3|250| +2450829|2119|3|857| +2450829|2120|3|| +2450829|2122|3|| +2450829|2125|3|675| +2450829|2126|3|565| +2450829|2128|3|637| +2450829|2131|3|739| +2450829|2132|3|| +2450829|2134|3|197| +2450829|2137|3|542| +2450829|2138|3|700| +2450829|2140|3|591| +2450829|2143|3|274| +2450829|2144|3|249| +2450829|2146|3|217| +2450829|2149|3|260| +2450829|2150|3|822| +2450829|2152|3|645| +2450829|2155|3|| +2450829|2156|3|431| +2450829|2158|3|673| +2450829|2161|3|885| +2450829|2162|3|96| +2450829|2164|3|329| +2450829|2167|3|553| +2450829|2168|3|71| +2450829|2170|3|498| +2450829|2173|3|986| +2450829|2174|3|217| +2450829|2176|3|834| +2450829|2179|3|16| +2450829|2180|3|151| +2450829|2182|3|389| +2450829|2185|3|826| +2450829|2186|3|570| +2450829|2188|3|174| +2450829|2191|3|712| +2450829|2192|3|66| +2450829|2194|3|| +2450829|2197|3|502| +2450829|2198|3|55| +2450829|2200|3|160| +2450829|2203|3|471| +2450829|2204|3|21| +2450829|2206|3|247| +2450829|2209|3|843| +2450829|2210|3|575| +2450829|2212|3|728| +2450829|2215|3|480| +2450829|2216|3|268| +2450829|2218|3|478| +2450829|2221|3|193| +2450829|2222|3|597| +2450829|2224|3|715| +2450829|2227|3|683| +2450829|2228|3|20| +2450829|2230|3|682| +2450829|2233|3|324| +2450829|2234|3|799| +2450829|2236|3|574| +2450829|2239|3|202| +2450829|2240|3|616| +2450829|2242|3|190| +2450829|2245|3|| +2450829|2246|3|| +2450829|2248|3|351| +2450829|2251|3|939| +2450829|2252|3|559| +2450829|2254|3|539| +2450829|2257|3|644| +2450829|2258|3|41| +2450829|2260|3|| +2450829|2263|3|920| +2450829|2264|3|989| +2450829|2266|3|34| +2450829|2269|3|777| +2450829|2270|3|923| +2450829|2272|3|39| +2450829|2275|3|350| +2450829|2276|3|424| +2450829|2278|3|924| +2450829|2281|3|105| +2450829|2282|3|429| +2450829|2284|3|701| +2450829|2287|3|305| +2450829|2288|3|408| +2450829|2290|3|25| +2450829|2293|3|606| +2450829|2294|3|447| +2450829|2296|3|791| +2450829|2299|3|280| +2450829|2300|3|274| +2450829|2302|3|614| +2450829|2305|3|385| +2450829|2306|3|144| +2450829|2308|3|275| +2450829|2311|3|118| +2450829|2312|3|153| +2450829|2314|3|357| +2450829|2317|3|611| +2450829|2318|3|843| +2450829|2320|3|972| +2450829|2323|3|0| +2450829|2324|3|853| +2450829|2326|3|374| +2450829|2329|3|933| +2450829|2330|3|990| +2450829|2332|3|| +2450829|2335|3|730| +2450829|2336|3|686| +2450829|2338|3|725| +2450829|2341|3|308| +2450829|2342|3|149| +2450829|2344|3|392| +2450829|2347|3|634| +2450829|2348|3|265| +2450829|2350|3|444| +2450829|2353|3|984| +2450829|2354|3|| +2450829|2356|3|689| +2450829|2359|3|| +2450829|2360|3|618| +2450829|2362|3|819| +2450829|2365|3|531| +2450829|2366|3|577| +2450829|2368|3|165| +2450829|2371|3|938| +2450829|2372|3|86| +2450829|2374|3|319| +2450829|2377|3|328| +2450829|2378|3|910| +2450829|2380|3|462| +2450829|2383|3|651| +2450829|2384|3|889| +2450829|2386|3|| +2450829|2389|3|55| +2450829|2390|3|628| +2450829|2392|3|730| +2450829|2395|3|713| +2450829|2396|3|314| +2450829|2398|3|298| +2450829|2401|3|| +2450829|2402|3|774| +2450829|2404|3|263| +2450829|2407|3|365| +2450829|2408|3|| +2450829|2410|3|699| +2450829|2413|3|33| +2450829|2414|3|902| +2450829|2416|3|592| +2450829|2419|3|819| +2450829|2420|3|224| +2450829|2422|3|330| +2450829|2425|3|236| +2450829|2426|3|788| +2450829|2428|3|537| +2450829|2431|3|514| +2450829|2432|3|25| +2450829|2434|3|174| +2450829|2437|3|278| +2450829|2438|3|53| +2450829|2440|3|829| +2450829|2443|3|831| +2450829|2444|3|803| +2450829|2446|3|238| +2450829|2449|3|958| +2450829|2450|3|200| +2450829|2452|3|551| +2450829|2455|3|53| +2450829|2456|3|806| +2450829|2458|3|905| +2450829|2461|3|87| +2450829|2462|3|379| +2450829|2464|3|46| +2450829|2467|3|470| +2450829|2468|3|650| +2450829|2470|3|680| +2450829|2473|3|391| +2450829|2474|3|806| +2450829|2476|3|483| +2450829|2479|3|364| +2450829|2480|3|646| +2450829|2482|3|308| +2450829|2485|3|480| +2450829|2486|3|| +2450829|2488|3|456| +2450829|2491|3|41| +2450829|2492|3|852| +2450829|2494|3|443| +2450829|2497|3|306| +2450829|2498|3|561| +2450829|2500|3|59| +2450829|2503|3|536| +2450829|2504|3|827| +2450829|2506|3|183| +2450829|2509|3|940| +2450829|2510|3|819| +2450829|2512|3|81| +2450829|2515|3|| +2450829|2516|3|756| +2450829|2518|3|801| +2450829|2521|3|593| +2450829|2522|3|482| +2450829|2524|3|19| +2450829|2527|3|651| +2450829|2528|3|726| +2450829|2530|3|366| +2450829|2533|3|744| +2450829|2534|3|595| +2450829|2536|3|912| +2450829|2539|3|| +2450829|2540|3|693| +2450829|2542|3|276| +2450829|2545|3|591| +2450829|2546|3|510| +2450829|2548|3|568| +2450829|2551|3|843| +2450829|2552|3|651| +2450829|2554|3|620| +2450829|2557|3|| +2450829|2558|3|711| +2450829|2560|3|617| +2450829|2563|3|779| +2450829|2564|3|472| +2450829|2566|3|462| +2450829|2569|3|665| +2450829|2570|3|145| +2450829|2572|3|558| +2450829|2575|3|842| +2450829|2576|3|907| +2450829|2578|3|439| +2450829|2581|3|874| +2450829|2582|3|420| +2450829|2584|3|409| +2450829|2587|3|976| +2450829|2588|3|521| +2450829|2590|3|488| +2450829|2593|3|235| +2450829|2594|3|| +2450829|2596|3|326| +2450829|2599|3|420| +2450829|2600|3|39| +2450829|2602|3|629| +2450829|2605|3|788| +2450829|2606|3|275| +2450829|2608|3|219| +2450829|2611|3|806| +2450829|2612|3|151| +2450829|2614|3|503| +2450829|2617|3|589| +2450829|2618|3|| +2450829|2620|3|832| +2450829|2623|3|601| +2450829|2624|3|950| +2450829|2626|3|22| +2450829|2629|3|621| +2450829|2630|3|601| +2450829|2632|3|883| +2450829|2635|3|774| +2450829|2636|3|363| +2450829|2638|3|680| +2450829|2641|3|741| +2450829|2642|3|735| +2450829|2644|3|660| +2450829|2647|3|525| +2450829|2648|3|318| +2450829|2650|3|260| +2450829|2653|3|469| +2450829|2654|3|637| +2450829|2656|3|458| +2450829|2659|3|576| +2450829|2660|3|800| +2450829|2662|3|641| +2450829|2665|3|171| +2450829|2666|3|267| +2450829|2668|3|223| +2450829|2671|3|802| +2450829|2672|3|990| +2450829|2674|3|483| +2450829|2677|3|524| +2450829|2678|3|786| +2450829|2680|3|133| +2450829|2683|3|515| +2450829|2684|3|202| +2450829|2686|3|264| +2450829|2689|3|586| +2450829|2690|3|484| +2450829|2692|3|946| +2450829|2695|3|900| +2450829|2696|3|135| +2450829|2698|3|657| +2450829|2701|3|442| +2450829|2702|3|61| +2450829|2704|3|394| +2450829|2707|3|370| +2450829|2708|3|29| +2450829|2710|3|561| +2450829|2713|3|11| +2450829|2714|3|344| +2450829|2716|3|939| +2450829|2719|3|768| +2450829|2720|3|408| +2450829|2722|3|546| +2450829|2725|3|455| +2450829|2726|3|550| +2450829|2728|3|545| +2450829|2731|3|600| +2450829|2732|3|663| +2450829|2734|3|156| +2450829|2737|3|657| +2450829|2738|3|807| +2450829|2740|3|470| +2450829|2743|3|309| +2450829|2744|3|995| +2450829|2746|3|| +2450829|2749|3|208| +2450829|2750|3|| +2450829|2752|3|308| +2450829|2755|3|616| +2450829|2756|3|599| +2450829|2758|3|47| +2450829|2761|3|490| +2450829|2762|3|138| +2450829|2764|3|887| +2450829|2767|3|955| +2450829|2768|3|942| +2450829|2770|3|720| +2450829|2773|3|257| +2450829|2774|3|131| +2450829|2776|3|376| +2450829|2779|3|675| +2450829|2780|3|377| +2450829|2782|3|348| +2450829|2785|3|74| +2450829|2786|3|969| +2450829|2788|3|874| +2450829|2791|3|496| +2450829|2792|3|324| +2450829|2794|3|825| +2450829|2797|3|380| +2450829|2798|3|983| +2450829|2800|3|840| +2450829|2803|3|868| +2450829|2804|3|912| +2450829|2806|3|322| +2450829|2809|3|888| +2450829|2810|3|| +2450829|2812|3|| +2450829|2815|3|965| +2450829|2816|3|13| +2450829|2818|3|247| +2450829|2821|3|280| +2450829|2822|3|842| +2450829|2824|3|77| +2450829|2827|3|267| +2450829|2828|3|5| +2450829|2830|3|421| +2450829|2833|3|158| +2450829|2834|3|483| +2450829|2836|3|968| +2450829|2839|3|727| +2450829|2840|3|620| +2450829|2842|3|318| +2450829|2845|3|298| +2450829|2846|3|57| +2450829|2848|3|440| +2450829|2851|3|104| +2450829|2852|3|| +2450829|2854|3|338| +2450829|2857|3|542| +2450829|2858|3|164| +2450829|2860|3|548| +2450829|2863|3|558| +2450829|2864|3|104| +2450829|2866|3|575| +2450829|2869|3|372| +2450829|2870|3|518| +2450829|2872|3|2| +2450829|2875|3|148| +2450829|2876|3|756| +2450829|2878|3|135| +2450829|2881|3|136| +2450829|2882|3|744| +2450829|2884|3|116| +2450829|2887|3|462| +2450829|2888|3|302| +2450829|2890|3|53| +2450829|2893|3|798| +2450829|2894|3|658| +2450829|2896|3|211| +2450829|2899|3|164| +2450829|2900|3|98| +2450829|2902|3|728| +2450829|2905|3|992| +2450829|2906|3|935| +2450829|2908|3|292| +2450829|2911|3|107| +2450829|2912|3|347| +2450829|2914|3|381| +2450829|2917|3|903| +2450829|2918|3|142| +2450829|2920|3|806| +2450829|2923|3|486| +2450829|2924|3|689| +2450829|2926|3|660| +2450829|2929|3|466| +2450829|2930|3|509| +2450829|2932|3|135| +2450829|2935|3|20| +2450829|2936|3|111| +2450829|2938|3|106| +2450829|2941|3|660| +2450829|2942|3|989| +2450829|2944|3|930| +2450829|2947|3|83| +2450829|2948|3|| +2450829|2950|3|274| +2450829|2953|3|465| +2450829|2954|3|487| +2450829|2956|3|963| +2450829|2959|3|430| +2450829|2960|3|399| +2450829|2962|3|254| +2450829|2965|3|414| +2450829|2966|3|558| +2450829|2968|3|949| +2450829|2971|3|793| +2450829|2972|3|513| +2450829|2974|3|853| +2450829|2977|3|928| +2450829|2978|3|604| +2450829|2980|3|547| +2450829|2983|3|709| +2450829|2984|3|127| +2450829|2986|3|70| +2450829|2989|3|584| +2450829|2990|3|902| +2450829|2992|3|3| +2450829|2995|3|847| +2450829|2996|3|285| +2450829|2998|3|276| +2450829|3001|3|530| +2450829|3002|3|563| +2450829|3004|3|719| +2450829|3007|3|405| +2450829|3008|3|| +2450829|3010|3|87| +2450829|3013|3|887| +2450829|3014|3|795| +2450829|3016|3|626| +2450829|3019|3|968| +2450829|3020|3|879| +2450829|3022|3|18| +2450829|3025|3|499| +2450829|3026|3|782| +2450829|3028|3|563| +2450829|3031|3|122| +2450829|3032|3|754| +2450829|3034|3|780| +2450829|3037|3|771| +2450829|3038|3|443| +2450829|3040|3|759| +2450829|3043|3|476| +2450829|3044|3|40| +2450829|3046|3|243| +2450829|3049|3|724| +2450829|3050|3|130| +2450829|3052|3|523| +2450829|3055|3|263| +2450829|3056|3|| +2450829|3058|3|412| +2450829|3061|3|794| +2450829|3062|3|980| +2450829|3064|3|153| +2450829|3067|3|267| +2450829|3068|3|330| +2450829|3070|3|750| +2450829|3073|3|738| +2450829|3074|3|833| +2450829|3076|3|421| +2450829|3079|3|991| +2450829|3080|3|176| +2450829|3082|3|365| +2450829|3085|3|501| +2450829|3086|3|142| +2450829|3088|3|140| +2450829|3091|3|960| +2450829|3092|3|538| +2450829|3094|3|160| +2450829|3097|3|349| +2450829|3098|3|497| +2450829|3100|3|805| +2450829|3103|3|880| +2450829|3104|3|648| +2450829|3106|3|456| +2450829|3109|3|23| +2450829|3110|3|969| +2450829|3112|3|948| +2450829|3115|3|617| +2450829|3116|3|553| +2450829|3118|3|171| +2450829|3121|3|199| +2450829|3122|3|839| +2450829|3124|3|943| +2450829|3127|3|729| +2450829|3128|3|321| +2450829|3130|3|223| +2450829|3133|3|643| +2450829|3134|3|59| +2450829|3136|3|250| +2450829|3139|3|606| +2450829|3140|3|826| +2450829|3142|3|| +2450829|3145|3|787| +2450829|3146|3|740| +2450829|3148|3|785| +2450829|3151|3|316| +2450829|3152|3|214| +2450829|3154|3|572| +2450829|3157|3|337| +2450829|3158|3|| +2450829|3160|3|40| +2450829|3163|3|568| +2450829|3164|3|688| +2450829|3166|3|256| +2450829|3169|3|309| +2450829|3170|3|638| +2450829|3172|3|149| +2450829|3175|3|447| +2450829|3176|3|378| +2450829|3178|3|224| +2450829|3181|3|42| +2450829|3182|3|940| +2450829|3184|3|33| +2450829|3187|3|366| +2450829|3188|3|165| +2450829|3190|3|456| +2450829|3193|3|878| +2450829|3194|3|175| +2450829|3196|3|| +2450829|3199|3|896| +2450829|3200|3|338| +2450829|3202|3|854| +2450829|3205|3|725| +2450829|3206|3|59| +2450829|3208|3|150| +2450829|3211|3|137| +2450829|3212|3|290| +2450829|3214|3|123| +2450829|3217|3|542| +2450829|3218|3|421| +2450829|3220|3|884| +2450829|3223|3|871| +2450829|3224|3|11| +2450829|3226|3|732| +2450829|3229|3|225| +2450829|3230|3|74| +2450829|3232|3|119| +2450829|3235|3|255| +2450829|3236|3|190| +2450829|3238|3|913| +2450829|3241|3|196| +2450829|3242|3|340| +2450829|3244|3|256| +2450829|3247|3|426| +2450829|3248|3|622| +2450829|3250|3|101| +2450829|3253|3|269| +2450829|3254|3|| +2450829|3256|3|97| +2450829|3259|3|796| +2450829|3260|3|783| +2450829|3262|3|61| +2450829|3265|3|853| +2450829|3266|3|387| +2450829|3268|3|345| +2450829|3271|3|220| +2450829|3272|3|700| +2450829|3274|3|373| +2450829|3277|3|797| +2450829|3278|3|462| +2450829|3280|3|67| +2450829|3283|3|176| +2450829|3284|3|82| +2450829|3286|3|461| +2450829|3289|3|287| +2450829|3290|3|| +2450829|3292|3|756| +2450829|3295|3|| +2450829|3296|3|86| +2450829|3298|3|548| +2450829|3301|3|318| +2450829|3302|3|507| +2450829|3304|3|709| +2450829|3307|3|871| +2450829|3308|3|291| +2450829|3310|3|128| +2450829|3313|3|662| +2450829|3314|3|701| +2450829|3316|3|515| +2450829|3319|3|618| +2450829|3320|3|| +2450829|3322|3|839| +2450829|3325|3|174| +2450829|3326|3|402| +2450829|3328|3|362| +2450829|3331|3|| +2450829|3332|3|883| +2450829|3334|3|214| +2450829|3337|3|903| +2450829|3338|3|641| +2450829|3340|3|474| +2450829|3343|3|800| +2450829|3344|3|| +2450829|3346|3|557| +2450829|3349|3|116| +2450829|3350|3|118| +2450829|3352|3|956| +2450829|3355|3|607| +2450829|3356|3|| +2450829|3358|3|127| +2450829|3361|3|514| +2450829|3362|3|227| +2450829|3364|3|978| +2450829|3367|3|123| +2450829|3368|3|263| +2450829|3370|3|194| +2450829|3373|3|125| +2450829|3374|3|148| +2450829|3376|3|384| +2450829|3379|3|837| +2450829|3380|3|553| +2450829|3382|3|530| +2450829|3385|3|996| +2450829|3386|3|467| +2450829|3388|3|312| +2450829|3391|3|152| +2450829|3392|3|824| +2450829|3394|3|137| +2450829|3397|3|926| +2450829|3398|3|730| +2450829|3400|3|288| +2450829|3403|3|256| +2450829|3404|3|412| +2450829|3406|3|792| +2450829|3409|3|466| +2450829|3410|3|467| +2450829|3412|3|695| +2450829|3415|3|68| +2450829|3416|3|416| +2450829|3418|3|641| +2450829|3421|3|655| +2450829|3422|3|643| +2450829|3424|3|301| +2450829|3427|3|| +2450829|3428|3|413| +2450829|3430|3|571| +2450829|3433|3|800| +2450829|3434|3|702| +2450829|3436|3|768| +2450829|3439|3|781| +2450829|3440|3|951| +2450829|3442|3|838| +2450829|3445|3|99| +2450829|3446|3|392| +2450829|3448|3|584| +2450829|3451|3|710| +2450829|3452|3|250| +2450829|3454|3|610| +2450829|3457|3|411| +2450829|3458|3|355| +2450829|3460|3|971| +2450829|3463|3|178| +2450829|3464|3|96| +2450829|3466|3|327| +2450829|3469|3|921| +2450829|3470|3|969| +2450829|3472|3|| +2450829|3475|3|162| +2450829|3476|3|220| +2450829|3478|3|16| +2450829|3481|3|955| +2450829|3482|3|816| +2450829|3484|3|890| +2450829|3487|3|127| +2450829|3488|3|550| +2450829|3490|3|359| +2450829|3493|3|918| +2450829|3494|3|231| +2450829|3496|3|449| +2450829|3499|3|583| +2450829|3500|3|220| +2450829|3502|3|759| +2450829|3505|3|237| +2450829|3506|3|930| +2450829|3508|3|352| +2450829|3511|3|434| +2450829|3512|3|119| +2450829|3514|3|390| +2450829|3517|3|620| +2450829|3518|3|740| +2450829|3520|3|268| +2450829|3523|3|323| +2450829|3524|3|68| +2450829|3526|3|85| +2450829|3529|3|666| +2450829|3530|3|881| +2450829|3532|3|642| +2450829|3535|3|241| +2450829|3536|3|449| +2450829|3538|3|271| +2450829|3541|3|692| +2450829|3542|3|35| +2450829|3544|3|321| +2450829|3547|3|672| +2450829|3548|3|| +2450829|3550|3|220| +2450829|3553|3|408| +2450829|3554|3|330| +2450829|3556|3|834| +2450829|3559|3|776| +2450829|3560|3|| +2450829|3562|3|94| +2450829|3565|3|167| +2450829|3566|3|371| +2450829|3568|3|728| +2450829|3571|3|340| +2450829|3572|3|976| +2450829|3574|3|725| +2450829|3577|3|596| +2450829|3578|3|545| +2450829|3580|3|902| +2450829|3583|3|415| +2450829|3584|3|164| +2450829|3586|3|690| +2450829|3589|3|945| +2450829|3590|3|467| +2450829|3592|3|343| +2450829|3595|3|792| +2450829|3596|3|11| +2450829|3598|3|528| +2450829|3601|3|199| +2450829|3602|3|873| +2450829|3604|3|823| +2450829|3607|3|873| +2450829|3608|3|900| +2450829|3610|3|832| +2450829|3613|3|166| +2450829|3614|3|833| +2450829|3616|3|384| +2450829|3619|3|585| +2450829|3620|3|801| +2450829|3622|3|61| +2450829|3625|3|663| +2450829|3626|3|649| +2450829|3628|3|49| +2450829|3631|3|573| +2450829|3632|3|374| +2450829|3634|3|785| +2450829|3637|3|947| +2450829|3638|3|97| +2450829|3640|3|717| +2450829|3643|3|277| +2450829|3644|3|191| +2450829|3646|3|574| +2450829|3649|3|335| +2450829|3650|3|848| +2450829|3652|3|62| +2450829|3655|3|725| +2450829|3656|3|138| +2450829|3658|3|565| +2450829|3661|3|718| +2450829|3662|3|365| +2450829|3664|3|316| +2450829|3667|3|156| +2450829|3668|3|304| +2450829|3670|3|522| +2450829|3673|3|371| +2450829|3674|3|735| +2450829|3676|3|925| +2450829|3679|3|883| +2450829|3680|3|716| +2450829|3682|3|697| +2450829|3685|3|690| +2450829|3686|3|260| +2450829|3688|3|687| +2450829|3691|3|684| +2450829|3692|3|36| +2450829|3694|3|131| +2450829|3697|3|373| +2450829|3698|3|| +2450829|3700|3|827| +2450829|3703|3|684| +2450829|3704|3|797| +2450829|3706|3|92| +2450829|3709|3|995| +2450829|3710|3|39| +2450829|3712|3|925| +2450829|3715|3|| +2450829|3716|3|258| +2450829|3718|3|| +2450829|3721|3|516| +2450829|3722|3|831| +2450829|3724|3|926| +2450829|3727|3|718| +2450829|3728|3|521| +2450829|3730|3|70| +2450829|3733|3|222| +2450829|3734|3|37| +2450829|3736|3|858| +2450829|3739|3|807| +2450829|3740|3|637| +2450829|3742|3|315| +2450829|3745|3|636| +2450829|3746|3|416| +2450829|3748|3|765| +2450829|3751|3|172| +2450829|3752|3|595| +2450829|3754|3|68| +2450829|3757|3|252| +2450829|3758|3|394| +2450829|3760|3|394| +2450829|3763|3|929| +2450829|3764|3|| +2450829|3766|3|994| +2450829|3769|3|654| +2450829|3770|3|381| +2450829|3772|3|389| +2450829|3775|3|424| +2450829|3776|3|145| +2450829|3778|3|496| +2450829|3781|3|91| +2450829|3782|3|669| +2450829|3784|3|427| +2450829|3787|3|39| +2450829|3788|3|819| +2450829|3790|3|386| +2450829|3793|3|293| +2450829|3794|3|243| +2450829|3796|3|252| +2450829|3799|3|304| +2450829|3800|3|281| +2450829|3802|3|876| +2450829|3805|3|465| +2450829|3806|3|973| +2450829|3808|3|503| +2450829|3811|3|803| +2450829|3812|3|725| +2450829|3814|3|743| +2450829|3817|3|| +2450829|3818|3|630| +2450829|3820|3|251| +2450829|3823|3|579| +2450829|3824|3|77| +2450829|3826|3|883| +2450829|3829|3|848| +2450829|3830|3|604| +2450829|3832|3|61| +2450829|3835|3|917| +2450829|3836|3|808| +2450829|3838|3|817| +2450829|3841|3|911| +2450829|3842|3|943| +2450829|3844|3|563| +2450829|3847|3|284| +2450829|3848|3|752| +2450829|3850|3|503| +2450829|3853|3|710| +2450829|3854|3|114| +2450829|3856|3|625| +2450829|3859|3|656| +2450829|3860|3|843| +2450829|3862|3|164| +2450829|3865|3|447| +2450829|3866|3|250| +2450829|3868|3|682| +2450829|3871|3|891| +2450829|3872|3|917| +2450829|3874|3|867| +2450829|3877|3|330| +2450829|3878|3|217| +2450829|3880|3|908| +2450829|3883|3|51| +2450829|3884|3|171| +2450829|3886|3|867| +2450829|3889|3|399| +2450829|3890|3|418| +2450829|3892|3|222| +2450829|3895|3|543| +2450829|3896|3|107| +2450829|3898|3|151| +2450829|3901|3|708| +2450829|3902|3|661| +2450829|3904|3|333| +2450829|3907|3|333| +2450829|3908|3|76| +2450829|3910|3|820| +2450829|3913|3|329| +2450829|3914|3|492| +2450829|3916|3|802| +2450829|3919|3|| +2450829|3920|3|561| +2450829|3922|3|| +2450829|3925|3|263| +2450829|3926|3|411| +2450829|3928|3|120| +2450829|3931|3|278| +2450829|3932|3|17| +2450829|3934|3|421| +2450829|3937|3|995| +2450829|3938|3|926| +2450829|3940|3|434| +2450829|3943|3|639| +2450829|3944|3|664| +2450829|3946|3|228| +2450829|3949|3|341| +2450829|3950|3|48| +2450829|3952|3|44| +2450829|3955|3|| +2450829|3956|3|832| +2450829|3958|3|320| +2450829|3961|3|419| +2450829|3962|3|428| +2450829|3964|3|113| +2450829|3967|3|123| +2450829|3968|3|159| +2450829|3970|3|97| +2450829|3973|3|430| +2450829|3974|3|797| +2450829|3976|3|846| +2450829|3979|3|323| +2450829|3980|3|367| +2450829|3982|3|167| +2450829|3985|3|144| +2450829|3986|3|512| +2450829|3988|3|515| +2450829|3991|3|487| +2450829|3992|3|978| +2450829|3994|3|| +2450829|3997|3|76| +2450829|3998|3|620| +2450829|4000|3|920| +2450829|4003|3|164| +2450829|4004|3|203| +2450829|4006|3|848| +2450829|4009|3|| +2450829|4010|3|680| +2450829|4012|3|275| +2450829|4015|3|749| +2450829|4016|3|| +2450829|4018|3|517| +2450829|4021|3|825| +2450829|4022|3|38| +2450829|4024|3|324| +2450829|4027|3|| +2450829|4028|3|500| +2450829|4030|3|250| +2450829|4033|3|711| +2450829|4034|3|187| +2450829|4036|3|940| +2450829|4039|3|522| +2450829|4040|3|486| +2450829|4042|3|970| +2450829|4045|3|576| +2450829|4046|3|603| +2450829|4048|3|107| +2450829|4051|3|395| +2450829|4052|3|590| +2450829|4054|3|928| +2450829|4057|3|814| +2450829|4058|3|823| +2450829|4060|3|656| +2450829|4063|3|828| +2450829|4064|3|8| +2450829|4066|3|406| +2450829|4069|3|| +2450829|4070|3|344| +2450829|4072|3|979| +2450829|4075|3|895| +2450829|4076|3|342| +2450829|4078|3|772| +2450829|4081|3|| +2450829|4082|3|556| +2450829|4084|3|516| +2450829|4087|3|704| +2450829|4088|3|247| +2450829|4090|3|494| +2450829|4093|3|939| +2450829|4094|3|727| +2450829|4096|3|414| +2450829|4099|3|968| +2450829|4100|3|602| +2450829|4102|3|231| +2450829|4105|3|3| +2450829|4106|3|777| +2450829|4108|3|620| +2450829|4111|3|332| +2450829|4112|3|333| +2450829|4114|3|611| +2450829|4117|3|819| +2450829|4118|3|808| +2450829|4120|3|415| +2450829|4123|3|897| +2450829|4124|3|32| +2450829|4126|3|137| +2450829|4129|3|447| +2450829|4130|3|| +2450829|4132|3|| +2450829|4135|3|503| +2450829|4136|3|108| +2450829|4138|3|11| +2450829|4141|3|| +2450829|4142|3|347| +2450829|4144|3|| +2450829|4147|3|41| +2450829|4148|3|331| +2450829|4150|3|926| +2450829|4153|3|91| +2450829|4154|3|103| +2450829|4156|3|466| +2450829|4159|3|| +2450829|4160|3|897| +2450829|4162|3|31| +2450829|4165|3|614| +2450829|4166|3|136| +2450829|4168|3|761| +2450829|4171|3|543| +2450829|4172|3|758| +2450829|4174|3|203| +2450829|4177|3|584| +2450829|4178|3|340| +2450829|4180|3|252| +2450829|4183|3|525| +2450829|4184|3|575| +2450829|4186|3|461| +2450829|4189|3|| +2450829|4190|3|173| +2450829|4192|3|440| +2450829|4195|3|556| +2450829|4196|3|287| +2450829|4198|3|905| +2450829|4201|3|174| +2450829|4202|3|404| +2450829|4204|3|989| +2450829|4207|3|678| +2450829|4208|3|834| +2450829|4210|3|756| +2450829|4213|3|197| +2450829|4214|3|157| +2450829|4216|3|281| +2450829|4219|3|954| +2450829|4220|3|210| +2450829|4222|3|963| +2450829|4225|3|206| +2450829|4226|3|276| +2450829|4228|3|184| +2450829|4231|3|651| +2450829|4232|3|871| +2450829|4234|3|665| +2450829|4237|3|796| +2450829|4238|3|307| +2450829|4240|3|693| +2450829|4243|3|532| +2450829|4244|3|26| +2450829|4246|3|863| +2450829|4249|3|623| +2450829|4250|3|357| +2450829|4252|3|952| +2450829|4255|3|404| +2450829|4256|3|64| +2450829|4258|3|249| +2450829|4261|3|712| +2450829|4262|3|748| +2450829|4264|3|197| +2450829|4267|3|325| +2450829|4268|3|893| +2450829|4270|3|76| +2450829|4273|3|570| +2450829|4274|3|476| +2450829|4276|3|225| +2450829|4279|3|483| +2450829|4280|3|148| +2450829|4282|3|301| +2450829|4285|3|245| +2450829|4286|3|550| +2450829|4288|3|966| +2450829|4291|3|170| +2450829|4292|3|| +2450829|4294|3|383| +2450829|4297|3|423| +2450829|4298|3|465| +2450829|4300|3|920| +2450829|4303|3|257| +2450829|4304|3|693| +2450829|4306|3|466| +2450829|4309|3|127| +2450829|4310|3|135| +2450829|4312|3|263| +2450829|4315|3|695| +2450829|4316|3|308| +2450829|4318|3|324| +2450829|4321|3|135| +2450829|4322|3|846| +2450829|4324|3|282| +2450829|4327|3|810| +2450829|4328|3|469| +2450829|4330|3|520| +2450829|4333|3|703| +2450829|4334|3|890| +2450829|4336|3|487| +2450829|4339|3|345| +2450829|4340|3|750| +2450829|4342|3|752| +2450829|4345|3|933| +2450829|4346|3|658| +2450829|4348|3|103| +2450829|4351|3|12| +2450829|4352|3|647| +2450829|4354|3|291| +2450829|4357|3|994| +2450829|4358|3|785| +2450829|4360|3|215| +2450829|4363|3|63| +2450829|4364|3|372| +2450829|4366|3|278| +2450829|4369|3|785| +2450829|4370|3|776| +2450829|4372|3|748| +2450829|4375|3|424| +2450829|4376|3|39| +2450829|4378|3|612| +2450829|4381|3|34| +2450829|4382|3|688| +2450829|4384|3|583| +2450829|4387|3|253| +2450829|4388|3|551| +2450829|4390|3|299| +2450829|4393|3|380| +2450829|4394|3|383| +2450829|4396|3|417| +2450829|4399|3|644| +2450829|4400|3|7| +2450829|4402|3|857| +2450829|4405|3|800| +2450829|4406|3|581| +2450829|4408|3|651| +2450829|4411|3|320| +2450829|4412|3|| +2450829|4414|3|49| +2450829|4417|3|337| +2450829|4418|3|842| +2450829|4420|3|778| +2450829|4423|3|590| +2450829|4424|3|466| +2450829|4426|3|925| +2450829|4429|3|29| +2450829|4430|3|822| +2450829|4432|3|181| +2450829|4435|3|870| +2450829|4436|3|161| +2450829|4438|3|100| +2450829|4441|3|765| +2450829|4442|3|416| +2450829|4444|3|282| +2450829|4447|3|876| +2450829|4448|3|| +2450829|4450|3|589| +2450829|4453|3|485| +2450829|4454|3|948| +2450829|4456|3|395| +2450829|4459|3|51| +2450829|4460|3|85| +2450829|4462|3|702| +2450829|4465|3|284| +2450829|4466|3|255| +2450829|4468|3|940| +2450829|4471|3|832| +2450829|4472|3|694| +2450829|4474|3|348| +2450829|4477|3|964| +2450829|4478|3|457| +2450829|4480|3|516| +2450829|4483|3|345| +2450829|4484|3|175| +2450829|4486|3|224| +2450829|4489|3|192| +2450829|4490|3|967| +2450829|4492|3|319| +2450829|4495|3|98| +2450829|4496|3|558| +2450829|4498|3|326| +2450829|4501|3|53| +2450829|4502|3|144| +2450829|4504|3|555| +2450829|4507|3|12| +2450829|4508|3|207| +2450829|4510|3|116| +2450829|4513|3|32| +2450829|4514|3|510| +2450829|4516|3|409| +2450829|4519|3|527| +2450829|4520|3|418| +2450829|4522|3|178| +2450829|4525|3|589| +2450829|4526|3|762| +2450829|4528|3|60| +2450829|4531|3|367| +2450829|4532|3|| +2450829|4534|3|986| +2450829|4537|3|962| +2450829|4538|3|482| +2450829|4540|3|880| +2450829|4543|3|339| +2450829|4544|3|913| +2450829|4546|3|357| +2450829|4549|3|349| +2450829|4550|3|622| +2450829|4552|3|669| +2450829|4555|3|103| +2450829|4556|3|222| +2450829|4558|3|600| +2450829|4561|3|611| +2450829|4562|3|438| +2450829|4564|3|213| +2450829|4567|3|180| +2450829|4568|3|952| +2450829|4570|3|822| +2450829|4573|3|196| +2450829|4574|3|79| +2450829|4576|3|846| +2450829|4579|3|823| +2450829|4580|3|226| +2450829|4582|3|365| +2450829|4585|3|196| +2450829|4586|3|667| +2450829|4588|3|966| +2450829|4591|3|673| +2450829|4592|3|461| +2450829|4594|3|709| +2450829|4597|3|| +2450829|4598|3|948| +2450829|4600|3|58| +2450829|4603|3|264| +2450829|4604|3|378| +2450829|4606|3|991| +2450829|4609|3|753| +2450829|4610|3|441| +2450829|4612|3|516| +2450829|4615|3|| +2450829|4616|3|990| +2450829|4618|3|953| +2450829|4621|3|540| +2450829|4622|3|383| +2450829|4624|3|610| +2450829|4627|3|161| +2450829|4628|3|622| +2450829|4630|3|235| +2450829|4633|3|468| +2450829|4634|3|| +2450829|4636|3|| +2450829|4639|3|670| +2450829|4640|3|521| +2450829|4642|3|349| +2450829|4645|3|213| +2450829|4646|3|943| +2450829|4648|3|481| +2450829|4651|3|667| +2450829|4652|3|567| +2450829|4654|3|959| +2450829|4657|3|| +2450829|4658|3|78| +2450829|4660|3|135| +2450829|4663|3|965| +2450829|4664|3|989| +2450829|4666|3|738| +2450829|4669|3|51| +2450829|4670|3|545| +2450829|4672|3|90| +2450829|4675|3|796| +2450829|4676|3|117| +2450829|4678|3|899| +2450829|4681|3|| +2450829|4682|3|427| +2450829|4684|3|634| +2450829|4687|3|| +2450829|4688|3|247| +2450829|4690|3|| +2450829|4693|3|727| +2450829|4694|3|441| +2450829|4696|3|701| +2450829|4699|3|119| +2450829|4700|3|79| +2450829|4702|3|891| +2450829|4705|3|532| +2450829|4706|3|| +2450829|4708|3|960| +2450829|4711|3|873| +2450829|4712|3|910| +2450829|4714|3|94| +2450829|4717|3|75| +2450829|4718|3|615| +2450829|4720|3|106| +2450829|4723|3|450| +2450829|4724|3|336| +2450829|4726|3|419| +2450829|4729|3|983| +2450829|4730|3|681| +2450829|4732|3|163| +2450829|4735|3|995| +2450829|4736|3|508| +2450829|4738|3|825| +2450829|4741|3|370| +2450829|4742|3|799| +2450829|4744|3|471| +2450829|4747|3|805| +2450829|4748|3|243| +2450829|4750|3|813| +2450829|4753|3|253| +2450829|4754|3|403| +2450829|4756|3|123| +2450829|4759|3|256| +2450829|4760|3|847| +2450829|4762|3|1000| +2450829|4765|3|809| +2450829|4766|3|667| +2450829|4768|3|240| +2450829|4771|3|592| +2450829|4772|3|674| +2450829|4774|3|777| +2450829|4777|3|731| +2450829|4778|3|328| +2450829|4780|3|345| +2450829|4783|3|904| +2450829|4784|3|510| +2450829|4786|3|692| +2450829|4789|3|795| +2450829|4790|3|677| +2450829|4792|3|555| +2450829|4795|3|865| +2450829|4796|3|436| +2450829|4798|3|679| +2450829|4801|3|311| +2450829|4802|3|| +2450829|4804|3|247| +2450829|4807|3|223| +2450829|4808|3|181| +2450829|4810|3|| +2450829|4813|3|521| +2450829|4814|3|56| +2450829|4816|3|894| +2450829|4819|3|296| +2450829|4820|3|666| +2450829|4822|3|154| +2450829|4825|3|471| +2450829|4826|3|14| +2450829|4828|3|538| +2450829|4831|3|858| +2450829|4832|3|458| +2450829|4834|3|171| +2450829|4837|3|245| +2450829|4838|3|365| +2450829|4840|3|773| +2450829|4843|3|626| +2450829|4844|3|661| +2450829|4846|3|948| +2450829|4849|3|891| +2450829|4850|3|491| +2450829|4852|3|898| +2450829|4855|3|406| +2450829|4856|3|374| +2450829|4858|3|461| +2450829|4861|3|496| +2450829|4862|3|781| +2450829|4864|3|578| +2450829|4867|3|78| +2450829|4868|3|226| +2450829|4870|3|438| +2450829|4873|3|37| +2450829|4874|3|778| +2450829|4876|3|347| +2450829|4879|3|249| +2450829|4880|3|201| +2450829|4882|3|824| +2450829|4885|3|539| +2450829|4886|3|865| +2450829|4888|3|173| +2450829|4891|3|123| +2450829|4892|3|139| +2450829|4894|3|581| +2450829|4897|3|458| +2450829|4898|3|313| +2450829|4900|3|732| +2450829|4903|3|154| +2450829|4904|3|399| +2450829|4906|3|287| +2450829|4909|3|534| +2450829|4910|3|254| +2450829|4912|3|338| +2450829|4915|3|657| +2450829|4916|3|265| +2450829|4918|3|227| +2450829|4921|3|248| +2450829|4922|3|500| +2450829|4924|3|478| +2450829|4927|3|546| +2450829|4928|3|687| +2450829|4930|3|519| +2450829|4933|3|661| +2450829|4934|3|805| +2450829|4936|3|247| +2450829|4939|3|730| +2450829|4940|3|671| +2450829|4942|3|| +2450829|4945|3|483| +2450829|4946|3|316| +2450829|4948|3|33| +2450829|4951|3|911| +2450829|4952|3|636| +2450829|4954|3|629| +2450829|4957|3|296| +2450829|4958|3|121| +2450829|4960|3|889| +2450829|4963|3|978| +2450829|4964|3|912| +2450829|4966|3|500| +2450829|4969|3|582| +2450829|4970|3|282| +2450829|4972|3|561| +2450829|4975|3|827| +2450829|4976|3|55| +2450829|4978|3|787| +2450829|4981|3|703| +2450829|4982|3|858| +2450829|4984|3|365| +2450829|4987|3|405| +2450829|4988|3|709| +2450829|4990|3|684| +2450829|4993|3|636| +2450829|4994|3|852| +2450829|4996|3|577| +2450829|4999|3|316| +2450829|5000|3|| +2450829|5002|3|623| +2450829|5005|3|316| +2450829|5006|3|140| +2450829|5008|3|730| +2450829|5011|3|396| +2450829|5012|3|203| +2450829|5014|3|583| +2450829|5017|3|736| +2450829|5018|3|169| +2450829|5020|3|497| +2450829|5023|3|771| +2450829|5024|3|406| +2450829|5026|3|832| +2450829|5029|3|498| +2450829|5030|3|| +2450829|5032|3|175| +2450829|5035|3|894| +2450829|5036|3|747| +2450829|5038|3|118| +2450829|5041|3|739| +2450829|5042|3|303| +2450829|5044|3|859| +2450829|5047|3|843| +2450829|5048|3|331| +2450829|5050|3|514| +2450829|5053|3|192| +2450829|5054|3|882| +2450829|5056|3|863| +2450829|5059|3|380| +2450829|5060|3|193| +2450829|5062|3|615| +2450829|5065|3|187| +2450829|5066|3|441| +2450829|5068|3|385| +2450829|5071|3|292| +2450829|5072|3|| +2450829|5074|3|931| +2450829|5077|3|554| +2450829|5078|3|393| +2450829|5080|3|299| +2450829|5083|3|562| +2450829|5084|3|702| +2450829|5086|3|870| +2450829|5089|3|182| +2450829|5090|3|10| +2450829|5092|3|325| +2450829|5095|3|233| +2450829|5096|3|920| +2450829|5098|3|450| +2450829|5101|3|127| +2450829|5102|3|8| +2450829|5104|3|299| +2450829|5107|3|711| +2450829|5108|3|535| +2450829|5110|3|| +2450829|5113|3|375| +2450829|5114|3|506| +2450829|5116|3|816| +2450829|5119|3|161| +2450829|5120|3|2| +2450829|5122|3|428| +2450829|5125|3|508| +2450829|5126|3|425| +2450829|5128|3|724| +2450829|5131|3|221| +2450829|5132|3|748| +2450829|5134|3|496| +2450829|5137|3|749| +2450829|5138|3|457| +2450829|5140|3|512| +2450829|5143|3|568| +2450829|5144|3|653| +2450829|5146|3|| +2450829|5149|3|403| +2450829|5150|3|342| +2450829|5152|3|896| +2450829|5155|3|265| +2450829|5156|3|136| +2450829|5158|3|814| +2450829|5161|3|295| +2450829|5162|3|649| +2450829|5164|3|899| +2450829|5167|3|813| +2450829|5168|3|60| +2450829|5170|3|722| +2450829|5173|3|266| +2450829|5174|3|590| +2450829|5176|3|944| +2450829|5179|3|701| +2450829|5180|3|437| +2450829|5182|3|221| +2450829|5185|3|583| +2450829|5186|3|85| +2450829|5188|3|875| +2450829|5191|3|642| +2450829|5192|3|27| +2450829|5194|3|441| +2450829|5197|3|813| +2450829|5198|3|511| +2450829|5200|3|549| +2450829|5203|3|757| +2450829|5204|3|194| +2450829|5206|3|164| +2450829|5209|3|950| +2450829|5210|3|886| +2450829|5212|3|83| +2450829|5215|3|350| +2450829|5216|3|476| +2450829|5218|3|722| +2450829|5221|3|355| +2450829|5222|3|| +2450829|5224|3|335| +2450829|5227|3|379| +2450829|5228|3|667| +2450829|5230|3|883| +2450829|5233|3|959| +2450829|5234|3|898| +2450829|5236|3|815| +2450829|5239|3|| +2450829|5240|3|393| +2450829|5242|3|119| +2450829|5245|3|554| +2450829|5246|3|689| +2450829|5248|3|529| +2450829|5251|3|571| +2450829|5252|3|789| +2450829|5254|3|592| +2450829|5257|3|62| +2450829|5258|3|434| +2450829|5260|3|251| +2450829|5263|3|700| +2450829|5264|3|991| +2450829|5266|3|186| +2450829|5269|3|940| +2450829|5270|3|162| +2450829|5272|3|| +2450829|5275|3|681| +2450829|5276|3|347| +2450829|5278|3|273| +2450829|5281|3|747| +2450829|5282|3|319| +2450829|5284|3|66| +2450829|5287|3|53| +2450829|5288|3|433| +2450829|5290|3|706| +2450829|5293|3|470| +2450829|5294|3|353| +2450829|5296|3|994| +2450829|5299|3|651| +2450829|5300|3|807| +2450829|5302|3|579| +2450829|5305|3|929| +2450829|5306|3|483| +2450829|5308|3|842| +2450829|5311|3|579| +2450829|5312|3|508| +2450829|5314|3|339| +2450829|5317|3|204| +2450829|5318|3|334| +2450829|5320|3|830| +2450829|5323|3|| +2450829|5324|3|266| +2450829|5326|3|98| +2450829|5329|3|275| +2450829|5330|3|718| +2450829|5332|3|337| +2450829|5335|3|739| +2450829|5336|3|719| +2450829|5338|3|337| +2450829|5341|3|310| +2450829|5342|3|132| +2450829|5344|3|175| +2450829|5347|3|229| +2450829|5348|3|94| +2450829|5350|3|774| +2450829|5353|3|361| +2450829|5354|3|817| +2450829|5356|3|805| +2450829|5359|3|660| +2450829|5360|3|917| +2450829|5362|3|600| +2450829|5365|3|279| +2450829|5366|3|565| +2450829|5368|3|195| +2450829|5371|3|296| +2450829|5372|3|34| +2450829|5374|3|778| +2450829|5377|3|263| +2450829|5378|3|751| +2450829|5380|3|480| +2450829|5383|3|469| +2450829|5384|3|243| +2450829|5386|3|537| +2450829|5389|3|344| +2450829|5390|3|842| +2450829|5392|3|285| +2450829|5395|3|| +2450829|5396|3|| +2450829|5398|3|741| +2450829|5401|3|635| +2450829|5402|3|| +2450829|5404|3|377| +2450829|5407|3|847| +2450829|5408|3|| +2450829|5410|3|4| +2450829|5413|3|944| +2450829|5414|3|27| +2450829|5416|3|557| +2450829|5419|3|843| +2450829|5420|3|482| +2450829|5422|3|279| +2450829|5425|3|826| +2450829|5426|3|157| +2450829|5428|3|133| +2450829|5431|3|887| +2450829|5432|3|150| +2450829|5434|3|501| +2450829|5437|3|955| +2450829|5438|3|263| +2450829|5440|3|555| +2450829|5443|3|335| +2450829|5444|3|913| +2450829|5446|3|390| +2450829|5449|3|446| +2450829|5450|3|743| +2450829|5452|3|600| +2450829|5455|3|199| +2450829|5456|3|132| +2450829|5458|3|333| +2450829|5461|3|309| +2450829|5462|3|565| +2450829|5464|3|545| +2450829|5467|3|419| +2450829|5468|3|425| +2450829|5470|3|15| +2450829|5473|3|975| +2450829|5474|3|879| +2450829|5476|3|157| +2450829|5479|3|143| +2450829|5480|3|752| +2450829|5482|3|274| +2450829|5485|3|746| +2450829|5486|3|| +2450829|5488|3|985| +2450829|5491|3|435| +2450829|5492|3|825| +2450829|5494|3|660| +2450829|5497|3|954| +2450829|5498|3|975| +2450829|5500|3|756| +2450829|5503|3|836| +2450829|5504|3|922| +2450829|5506|3|806| +2450829|5509|3|810| +2450829|5510|3|403| +2450829|5512|3|885| +2450829|5515|3|938| +2450829|5516|3|161| +2450829|5518|3|379| +2450829|5521|3|952| +2450829|5522|3|287| +2450829|5524|3|750| +2450829|5527|3|404| +2450829|5528|3|505| +2450829|5530|3|436| +2450829|5533|3|766| +2450829|5534|3|745| +2450829|5536|3|257| +2450829|5539|3|622| +2450829|5540|3|649| +2450829|5542|3|637| +2450829|5545|3|202| +2450829|5546|3|269| +2450829|5548|3|595| +2450829|5551|3|237| +2450829|5552|3|254| +2450829|5554|3|38| +2450829|5557|3|896| +2450829|5558|3|| +2450829|5560|3|978| +2450829|5563|3|926| +2450829|5564|3|913| +2450829|5566|3|663| +2450829|5569|3|795| +2450829|5570|3|703| +2450829|5572|3|400| +2450829|5575|3|79| +2450829|5576|3|221| +2450829|5578|3|182| +2450829|5581|3|631| +2450829|5582|3|776| +2450829|5584|3|657| +2450829|5587|3|272| +2450829|5588|3|537| +2450829|5590|3|622| +2450829|5593|3|661| +2450829|5594|3|343| +2450829|5596|3|773| +2450829|5599|3|29| +2450829|5600|3|332| +2450829|5602|3|490| +2450829|5605|3|521| +2450829|5606|3|248| +2450829|5608|3|719| +2450829|5611|3|585| +2450829|5612|3|647| +2450829|5614|3|39| +2450829|5617|3|516| +2450829|5618|3|408| +2450829|5620|3|795| +2450829|5623|3|390| +2450829|5624|3|723| +2450829|5626|3|932| +2450829|5629|3|876| +2450829|5630|3|869| +2450829|5632|3|266| +2450829|5635|3|694| +2450829|5636|3|999| +2450829|5638|3|261| +2450829|5641|3|172| +2450829|5642|3|190| +2450829|5644|3|349| +2450829|5647|3|78| +2450829|5648|3|975| +2450829|5650|3|749| +2450829|5653|3|484| +2450829|5654|3|589| +2450829|5656|3|110| +2450829|5659|3|976| +2450829|5660|3|| +2450829|5662|3|257| +2450829|5665|3|944| +2450829|5666|3|343| +2450829|5668|3|340| +2450829|5671|3|17| +2450829|5672|3|235| +2450829|5674|3|828| +2450829|5677|3|536| +2450829|5678|3|204| +2450829|5680|3|12| +2450829|5683|3|436| +2450829|5684|3|595| +2450829|5686|3|65| +2450829|5689|3|851| +2450829|5690|3|811| +2450829|5692|3|496| +2450829|5695|3|913| +2450829|5696|3|717| +2450829|5698|3|378| +2450829|5701|3|977| +2450829|5702|3|4| +2450829|5704|3|968| +2450829|5707|3|795| +2450829|5708|3|188| +2450829|5710|3|768| +2450829|5713|3|656| +2450829|5714|3|| +2450829|5716|3|833| +2450829|5719|3|844| +2450829|5720|3|36| +2450829|5722|3|509| +2450829|5725|3|109| +2450829|5726|3|690| +2450829|5728|3|283| +2450829|5731|3|65| +2450829|5732|3|217| +2450829|5734|3|229| +2450829|5737|3|909| +2450829|5738|3|473| +2450829|5740|3|371| +2450829|5743|3|113| +2450829|5744|3|591| +2450829|5746|3|56| +2450829|5749|3|242| +2450829|5750|3|249| +2450829|5752|3|| +2450829|5755|3|574| +2450829|5756|3|89| +2450829|5758|3|107| +2450829|5761|3|483| +2450829|5762|3|71| +2450829|5764|3|506| +2450829|5767|3|417| +2450829|5768|3|894| +2450829|5770|3|431| +2450829|5773|3|890| +2450829|5774|3|83| +2450829|5776|3|101| +2450829|5779|3|112| +2450829|5780|3|572| +2450829|5782|3|| +2450829|5785|3|591| +2450829|5786|3|215| +2450829|5788|3|203| +2450829|5791|3|257| +2450829|5792|3|862| +2450829|5794|3|812| +2450829|5797|3|391| +2450829|5798|3|683| +2450829|5800|3|| +2450829|5803|3|668| +2450829|5804|3|630| +2450829|5806|3|67| +2450829|5809|3|678| +2450829|5810|3|623| +2450829|5812|3|| +2450829|5815|3|515| +2450829|5816|3|316| +2450829|5818|3|817| +2450829|5821|3|429| +2450829|5822|3|758| +2450829|5824|3|235| +2450829|5827|3|776| +2450829|5828|3|389| +2450829|5830|3|200| +2450829|5833|3|894| +2450829|5834|3|794| +2450829|5836|3|246| +2450829|5839|3|765| +2450829|5840|3|648| +2450829|5842|3|67| +2450829|5845|3|561| +2450829|5846|3|865| +2450829|5848|3|956| +2450829|5851|3|448| +2450829|5852|3|368| +2450829|5854|3|606| +2450829|5857|3|245| +2450829|5858|3|687| +2450829|5860|3|15| +2450829|5863|3|17| +2450829|5864|3|826| +2450829|5866|3|421| +2450829|5869|3|725| +2450829|5870|3|429| +2450829|5872|3|374| +2450829|5875|3|166| +2450829|5876|3|743| +2450829|5878|3|847| +2450829|5881|3|683| +2450829|5882|3|859| +2450829|5884|3|514| +2450829|5887|3|638| +2450829|5888|3|18| +2450829|5890|3|590| +2450829|5893|3|101| +2450829|5894|3|860| +2450829|5896|3|245| +2450829|5899|3|632| +2450829|5900|3|488| +2450829|5902|3|140| +2450829|5905|3|557| +2450829|5906|3|733| +2450829|5908|3|25| +2450829|5911|3|485| +2450829|5912|3|612| +2450829|5914|3|627| +2450829|5917|3|876| +2450829|5918|3|591| +2450829|5920|3|278| +2450829|5923|3|488| +2450829|5924|3|447| +2450829|5926|3|38| +2450829|5929|3|317| +2450829|5930|3|282| +2450829|5932|3|| +2450829|5935|3|794| +2450829|5936|3|435| +2450829|5938|3|864| +2450829|5941|3|266| +2450829|5942|3|996| +2450829|5944|3|369| +2450829|5947|3|950| +2450829|5948|3|985| +2450829|5950|3|968| +2450829|5953|3|396| +2450829|5954|3|544| +2450829|5956|3|392| +2450829|5959|3|235| +2450829|5960|3|945| +2450829|5962|3|863| +2450829|5965|3|| +2450829|5966|3|536| +2450829|5968|3|592| +2450829|5971|3|279| +2450829|5972|3|735| +2450829|5974|3|985| +2450829|5977|3|968| +2450829|5978|3|150| +2450829|5980|3|892| +2450829|5983|3|614| +2450829|5984|3|651| +2450829|5986|3|200| +2450829|5989|3|834| +2450829|5990|3|375| +2450829|5992|3|719| +2450829|5995|3|693| +2450829|5996|3|728| +2450829|5998|3|662| +2450829|6001|3|501| +2450829|6002|3|377| +2450829|6004|3|682| +2450829|6007|3|510| +2450829|6008|3|961| +2450829|6010|3|218| +2450829|6013|3|747| +2450829|6014|3|834| +2450829|6016|3|28| +2450829|6019|3|187| +2450829|6020|3|670| +2450829|6022|3|334| +2450829|6025|3|586| +2450829|6026|3|64| +2450829|6028|3|951| +2450829|6031|3|703| +2450829|6032|3|143| +2450829|6034|3|339| +2450829|6037|3|146| +2450829|6038|3|897| +2450829|6040|3|53| +2450829|6043|3|744| +2450829|6044|3|291| +2450829|6046|3|| +2450829|6049|3|894| +2450829|6050|3|553| +2450829|6052|3|387| +2450829|6055|3|158| +2450829|6056|3|210| +2450829|6058|3|658| +2450829|6061|3|| +2450829|6062|3|934| +2450829|6064|3|215| +2450829|6067|3|471| +2450829|6068|3|195| +2450829|6070|3|576| +2450829|6073|3|122| +2450829|6074|3|338| +2450829|6076|3|276| +2450829|6079|3|981| +2450829|6080|3|418| +2450829|6082|3|381| +2450829|6085|3|29| +2450829|6086|3|941| +2450829|6088|3|49| +2450829|6091|3|288| +2450829|6092|3|195| +2450829|6094|3|773| +2450829|6097|3|109| +2450829|6098|3|919| +2450829|6100|3|359| +2450829|6103|3|435| +2450829|6104|3|417| +2450829|6106|3|285| +2450829|6109|3|348| +2450829|6110|3|529| +2450829|6112|3|959| +2450829|6115|3|871| +2450829|6116|3|618| +2450829|6118|3|590| +2450829|6121|3|540| +2450829|6122|3|175| +2450829|6124|3|61| +2450829|6127|3|244| +2450829|6128|3|676| +2450829|6130|3|596| +2450829|6133|3|550| +2450829|6134|3|732| +2450829|6136|3|510| +2450829|6139|3|587| +2450829|6140|3|224| +2450829|6142|3|970| +2450829|6145|3|674| +2450829|6146|3|285| +2450829|6148|3|202| +2450829|6151|3|452| +2450829|6152|3|789| +2450829|6154|3|764| +2450829|6157|3|28| +2450829|6158|3|68| +2450829|6160|3|478| +2450829|6163|3|69| +2450829|6164|3|788| +2450829|6166|3|523| +2450829|6169|3|613| +2450829|6170|3|482| +2450829|6172|3|919| +2450829|6175|3|58| +2450829|6176|3|428| +2450829|6178|3|914| +2450829|6181|3|631| +2450829|6182|3|456| +2450829|6184|3|912| +2450829|6187|3|47| +2450829|6188|3|712| +2450829|6190|3|| +2450829|6193|3|613| +2450829|6194|3|284| +2450829|6196|3|554| +2450829|6199|3|494| +2450829|6200|3|413| +2450829|6202|3|20| +2450829|6205|3|700| +2450829|6206|3|813| +2450829|6208|3|353| +2450829|6211|3|211| +2450829|6212|3|130| +2450829|6214|3|639| +2450829|6217|3|359| +2450829|6218|3|678| +2450829|6220|3|518| +2450829|6223|3|265| +2450829|6224|3|| +2450829|6226|3|480| +2450829|6229|3|529| +2450829|6230|3|696| +2450829|6232|3|946| +2450829|6235|3|163| +2450829|6236|3|412| +2450829|6238|3|592| +2450829|6241|3|295| +2450829|6242|3|860| +2450829|6244|3|418| +2450829|6247|3|260| +2450829|6248|3|657| +2450829|6250|3|| +2450829|6253|3|699| +2450829|6254|3|699| +2450829|6256|3|914| +2450829|6259|3|101| +2450829|6260|3|873| +2450829|6262|3|381| +2450829|6265|3|444| +2450829|6266|3|832| +2450829|6268|3|178| +2450829|6271|3|936| +2450829|6272|3|217| +2450829|6274|3|376| +2450829|6277|3|706| +2450829|6278|3|347| +2450829|6280|3|465| +2450829|6283|3|339| +2450829|6284|3|881| +2450829|6286|3|790| +2450829|6289|3|392| +2450829|6290|3|6| +2450829|6292|3|709| +2450829|6295|3|85| +2450829|6296|3|704| +2450829|6298|3|294| +2450829|6301|3|631| +2450829|6302|3|92| +2450829|6304|3|194| +2450829|6307|3|749| +2450829|6308|3|453| +2450829|6310|3|18| +2450829|6313|3|292| +2450829|6314|3|446| +2450829|6316|3|328| +2450829|6319|3|954| +2450829|6320|3|946| +2450829|6322|3|747| +2450829|6325|3|318| +2450829|6326|3|310| +2450829|6328|3|8| +2450829|6331|3|726| +2450829|6332|3|868| +2450829|6334|3|686| +2450829|6337|3|714| +2450829|6338|3|846| +2450829|6340|3|158| +2450829|6343|3|839| +2450829|6344|3|505| +2450829|6346|3|265| +2450829|6349|3|30| +2450829|6350|3|346| +2450829|6352|3|227| +2450829|6355|3|361| +2450829|6356|3|926| +2450829|6358|3|981| +2450829|6361|3|521| +2450829|6362|3|173| +2450829|6364|3|205| +2450829|6367|3|141| +2450829|6368|3|349| +2450829|6370|3|| +2450829|6373|3|| +2450829|6374|3|458| +2450829|6376|3|224| +2450829|6379|3|119| +2450829|6380|3|172| +2450829|6382|3|886| +2450829|6385|3|979| +2450829|6386|3|| +2450829|6388|3|851| +2450829|6391|3|312| +2450829|6392|3|753| +2450829|6394|3|308| +2450829|6397|3|992| +2450829|6398|3|| +2450829|6400|3|352| +2450829|6403|3|977| +2450829|6404|3|870| +2450829|6406|3|914| +2450829|6409|3|642| +2450829|6410|3|565| +2450829|6412|3|482| +2450829|6415|3|890| +2450829|6416|3|414| +2450829|6418|3|184| +2450829|6421|3|| +2450829|6422|3|45| +2450829|6424|3|194| +2450829|6427|3|636| +2450829|6428|3|423| +2450829|6430|3|123| +2450829|6433|3|615| +2450829|6434|3|741| +2450829|6436|3|568| +2450829|6439|3|488| +2450829|6440|3|576| +2450829|6442|3|831| +2450829|6445|3|| +2450829|6446|3|660| +2450829|6448|3|431| +2450829|6451|3|809| +2450829|6452|3|615| +2450829|6454|3|736| +2450829|6457|3|291| +2450829|6458|3|135| +2450829|6460|3|806| +2450829|6463|3|212| +2450829|6464|3|230| +2450829|6466|3|401| +2450829|6469|3|489| +2450829|6470|3|16| +2450829|6472|3|898| +2450829|6475|3|44| +2450829|6476|3|716| +2450829|6478|3|986| +2450829|6481|3|154| +2450829|6482|3|632| +2450829|6484|3|617| +2450829|6487|3|522| +2450829|6488|3|873| +2450829|6490|3|54| +2450829|6493|3|517| +2450829|6494|3|343| +2450829|6496|3|459| +2450829|6499|3|576| +2450829|6500|3|253| +2450829|6502|3|521| +2450829|6505|3|721| +2450829|6506|3|88| +2450829|6508|3|172| +2450829|6511|3|158| +2450829|6512|3|661| +2450829|6514|3|622| +2450829|6517|3|797| +2450829|6518|3|766| +2450829|6520|3|367| +2450829|6523|3|885| +2450829|6524|3|44| +2450829|6526|3|293| +2450829|6529|3|132| +2450829|6530|3|49| +2450829|6532|3|584| +2450829|6535|3|174| +2450829|6536|3|916| +2450829|6538|3|474| +2450829|6541|3|43| +2450829|6542|3|624| +2450829|6544|3|518| +2450829|6547|3|170| +2450829|6548|3|15| +2450829|6550|3|368| +2450829|6553|3|119| +2450829|6554|3|156| +2450829|6556|3|735| +2450829|6559|3|578| +2450829|6560|3|348| +2450829|6562|3|833| +2450829|6565|3|681| +2450829|6566|3|575| +2450829|6568|3|248| +2450829|6571|3|497| +2450829|6572|3|873| +2450829|6574|3|| +2450829|6577|3|631| +2450829|6578|3|872| +2450829|6580|3|413| +2450829|6583|3|101| +2450829|6584|3|106| +2450829|6586|3|610| +2450829|6589|3|764| +2450829|6590|3|836| +2450829|6592|3|355| +2450829|6595|3|418| +2450829|6596|3|767| +2450829|6598|3|915| +2450829|6601|3|252| +2450829|6602|3|140| +2450829|6604|3|256| +2450829|6607|3|809| +2450829|6608|3|285| +2450829|6610|3|37| +2450829|6613|3|| +2450829|6614|3|261| +2450829|6616|3|746| +2450829|6619|3|715| +2450829|6620|3|848| +2450829|6622|3|130| +2450829|6625|3|134| +2450829|6626|3|798| +2450829|6628|3|835| +2450829|6631|3|798| +2450829|6632|3|586| +2450829|6634|3|219| +2450829|6637|3|825| +2450829|6638|3|481| +2450829|6640|3|79| +2450829|6643|3|944| +2450829|6644|3|906| +2450829|6646|3|397| +2450829|6649|3|365| +2450829|6650|3|780| +2450829|6652|3|255| +2450829|6655|3|67| +2450829|6656|3|961| +2450829|6658|3|408| +2450829|6661|3|300| +2450829|6662|3|298| +2450829|6664|3|114| +2450829|6667|3|808| +2450829|6668|3|187| +2450829|6670|3|248| +2450829|6673|3|387| +2450829|6674|3|102| +2450829|6676|3|464| +2450829|6679|3|118| +2450829|6680|3|767| +2450829|6682|3|32| +2450829|6685|3|| +2450829|6686|3|312| +2450829|6688|3|499| +2450829|6691|3|696| +2450829|6692|3|336| +2450829|6694|3|998| +2450829|6697|3|262| +2450829|6698|3|903| +2450829|6700|3|873| +2450829|6703|3|647| +2450829|6704|3|769| +2450829|6706|3|230| +2450829|6709|3|361| +2450829|6710|3|864| +2450829|6712|3|188| +2450829|6715|3|894| +2450829|6716|3|844| +2450829|6718|3|211| +2450829|6721|3|146| +2450829|6722|3|264| +2450829|6724|3|83| +2450829|6727|3|892| +2450829|6728|3|589| +2450829|6730|3|182| +2450829|6733|3|814| +2450829|6734|3|685| +2450829|6736|3|1000| +2450829|6739|3|587| +2450829|6740|3|218| +2450829|6742|3|408| +2450829|6745|3|165| +2450829|6746|3|688| +2450829|6748|3|237| +2450829|6751|3|435| +2450829|6752|3|447| +2450829|6754|3|265| +2450829|6757|3|580| +2450829|6758|3|972| +2450829|6760|3|704| +2450829|6763|3|214| +2450829|6764|3|275| +2450829|6766|3|620| +2450829|6769|3|336| +2450829|6770|3|90| +2450829|6772|3|461| +2450829|6775|3|655| +2450829|6776|3|686| +2450829|6778|3|472| +2450829|6781|3|950| +2450829|6782|3|857| +2450829|6784|3|| +2450829|6787|3|665| +2450829|6788|3|9| +2450829|6790|3|395| +2450829|6793|3|214| +2450829|6794|3|556| +2450829|6796|3|863| +2450829|6799|3|276| +2450829|6800|3|813| +2450829|6802|3|567| +2450829|6805|3|883| +2450829|6806|3|232| +2450829|6808|3|| +2450829|6811|3|516| +2450829|6812|3|796| +2450829|6814|3|511| +2450829|6817|3|212| +2450829|6818|3|820| +2450829|6820|3|277| +2450829|6823|3|541| +2450829|6824|3|322| +2450829|6826|3|243| +2450829|6829|3|496| +2450829|6830|3|| +2450829|6832|3|74| +2450829|6835|3|336| +2450829|6836|3|576| +2450829|6838|3|702| +2450829|6841|3|155| +2450829|6842|3|134| +2450829|6844|3|395| +2450829|6847|3|872| +2450829|6848|3|503| +2450829|6850|3|12| +2450829|6853|3|329| +2450829|6854|3|639| +2450829|6856|3|324| +2450829|6859|3|574| +2450829|6860|3|476| +2450829|6862|3|| +2450829|6865|3|145| +2450829|6866|3|0| +2450829|6868|3|205| +2450829|6871|3|439| +2450829|6872|3|808| +2450829|6874|3|570| +2450829|6877|3|431| +2450829|6878|3|466| +2450829|6880|3|210| +2450829|6883|3|582| +2450829|6884|3|680| +2450829|6886|3|249| +2450829|6889|3|947| +2450829|6890|3|98| +2450829|6892|3|123| +2450829|6895|3|764| +2450829|6896|3|396| +2450829|6898|3|931| +2450829|6901|3|528| +2450829|6902|3|236| +2450829|6904|3|693| +2450829|6907|3|413| +2450829|6908|3|| +2450829|6910|3|456| +2450829|6913|3|535| +2450829|6914|3|211| +2450829|6916|3|956| +2450829|6919|3|564| +2450829|6920|3|845| +2450829|6922|3|713| +2450829|6925|3|540| +2450829|6926|3|31| +2450829|6928|3|337| +2450829|6931|3|570| +2450829|6932|3|730| +2450829|6934|3|305| +2450829|6937|3|108| +2450829|6938|3|595| +2450829|6940|3|471| +2450829|6943|3|582| +2450829|6944|3|954| +2450829|6946|3|251| +2450829|6949|3|978| +2450829|6950|3|346| +2450829|6952|3|828| +2450829|6955|3|769| +2450829|6956|3|2| +2450829|6958|3|360| +2450829|6961|3|960| +2450829|6962|3|287| +2450829|6964|3|| +2450829|6967|3|672| +2450829|6968|3|457| +2450829|6970|3|769| +2450829|6973|3|614| +2450829|6974|3|36| +2450829|6976|3|478| +2450829|6979|3|494| +2450829|6980|3|318| +2450829|6982|3|124| +2450829|6985|3|443| +2450829|6986|3|587| +2450829|6988|3|| +2450829|6991|3|135| +2450829|6992|3|853| +2450829|6994|3|345| +2450829|6997|3|709| +2450829|6998|3|568| +2450829|7000|3|379| +2450829|7003|3|421| +2450829|7004|3|707| +2450829|7006|3|96| +2450829|7009|3|921| +2450829|7010|3|74| +2450829|7012|3|908| +2450829|7015|3|| +2450829|7016|3|534| +2450829|7018|3|481| +2450829|7021|3|880| +2450829|7022|3|837| +2450829|7024|3|405| +2450829|7027|3|432| +2450829|7028|3|985| +2450829|7030|3|969| +2450829|7033|3|874| +2450829|7034|3|418| +2450829|7036|3|130| +2450829|7039|3|530| +2450829|7040|3|95| +2450829|7042|3|734| +2450829|7045|3|150| +2450829|7046|3|977| +2450829|7048|3|7| +2450829|7051|3|39| +2450829|7052|3|461| +2450829|7054|3|77| +2450829|7057|3|124| +2450829|7058|3|739| +2450829|7060|3|895| +2450829|7063|3|288| +2450829|7064|3|166| +2450829|7066|3|64| +2450829|7069|3|268| +2450829|7070|3|41| +2450829|7072|3|910| +2450829|7075|3|901| +2450829|7076|3|516| +2450829|7078|3|639| +2450829|7081|3|660| +2450829|7082|3|315| +2450829|7084|3|615| +2450829|7087|3|663| +2450829|7088|3|402| +2450829|7090|3|| +2450829|7093|3|528| +2450829|7094|3|246| +2450829|7096|3|947| +2450829|7099|3|636| +2450829|7100|3|724| +2450829|7102|3|143| +2450829|7105|3|629| +2450829|7106|3|956| +2450829|7108|3|921| +2450829|7111|3|250| +2450829|7112|3|253| +2450829|7114|3|829| +2450829|7117|3|600| +2450829|7118|3|| +2450829|7120|3|944| +2450829|7123|3|728| +2450829|7124|3|787| +2450829|7126|3|| +2450829|7129|3|971| +2450829|7130|3|164| +2450829|7132|3|79| +2450829|7135|3|13| +2450829|7136|3|965| +2450829|7138|3|212| +2450829|7141|3|921| +2450829|7142|3|156| +2450829|7144|3|786| +2450829|7147|3|131| +2450829|7148|3|575| +2450829|7150|3|595| +2450829|7153|3|710| +2450829|7154|3|236| +2450829|7156|3|495| +2450829|7159|3|337| +2450829|7160|3|701| +2450829|7162|3|213| +2450829|7165|3|324| +2450829|7166|3|211| +2450829|7168|3|77| +2450829|7171|3|65| +2450829|7172|3|20| +2450829|7174|3|600| +2450829|7177|3|7| +2450829|7178|3|461| +2450829|7180|3|844| +2450829|7183|3|247| +2450829|7184|3|182| +2450829|7186|3|282| +2450829|7189|3|554| +2450829|7190|3|52| +2450829|7192|3|8| +2450829|7195|3|465| +2450829|7196|3|589| +2450829|7198|3|915| +2450829|7201|3|201| +2450829|7202|3|837| +2450829|7204|3|346| +2450829|7207|3|304| +2450829|7208|3|662| +2450829|7210|3|12| +2450829|7213|3|258| +2450829|7214|3|682| +2450829|7216|3|247| +2450829|7219|3|17| +2450829|7220|3|| +2450829|7222|3|840| +2450829|7225|3|311| +2450829|7226|3|853| +2450829|7228|3|394| +2450829|7231|3|39| +2450829|7232|3|774| +2450829|7234|3|704| +2450829|7237|3|638| +2450829|7238|3|549| +2450829|7240|3|| +2450829|7243|3|401| +2450829|7244|3|116| +2450829|7246|3|846| +2450829|7249|3|73| +2450829|7250|3|259| +2450829|7252|3|93| +2450829|7255|3|275| +2450829|7256|3|618| +2450829|7258|3|567| +2450829|7261|3|410| +2450829|7262|3|172| +2450829|7264|3|387| +2450829|7267|3|38| +2450829|7268|3|343| +2450829|7270|3|508| +2450829|7273|3|819| +2450829|7274|3|| +2450829|7276|3|45| +2450829|7279|3|244| +2450829|7280|3|322| +2450829|7282|3|900| +2450829|7285|3|607| +2450829|7286|3|269| +2450829|7288|3|356| +2450829|7291|3|399| +2450829|7292|3|304| +2450829|7294|3|70| +2450829|7297|3|478| +2450829|7298|3|| +2450829|7300|3|838| +2450829|7303|3|238| +2450829|7304|3|609| +2450829|7306|3|791| +2450829|7309|3|489| +2450829|7310|3|516| +2450829|7312|3|575| +2450829|7315|3|| +2450829|7316|3|343| +2450829|7318|3|779| +2450829|7321|3|323| +2450829|7322|3|550| +2450829|7324|3|626| +2450829|7327|3|6| +2450829|7328|3|609| +2450829|7330|3|691| +2450829|7333|3|294| +2450829|7334|3|550| +2450829|7336|3|246| +2450829|7339|3|583| +2450829|7340|3|144| +2450829|7342|3|| +2450829|7345|3|30| +2450829|7346|3|537| +2450829|7348|3|993| +2450829|7351|3|869| +2450829|7352|3|879| +2450829|7354|3|276| +2450829|7357|3|957| +2450829|7358|3|701| +2450829|7360|3|905| +2450829|7363|3|989| +2450829|7364|3|979| +2450829|7366|3|436| +2450829|7369|3|109| +2450829|7370|3|162| +2450829|7372|3|639| +2450829|7375|3|460| +2450829|7376|3|786| +2450829|7378|3|416| +2450829|7381|3|586| +2450829|7382|3|973| +2450829|7384|3|982| +2450829|7387|3|515| +2450829|7388|3|971| +2450829|7390|3|240| +2450829|7393|3|| +2450829|7394|3|834| +2450829|7396|3|136| +2450829|7399|3|998| +2450829|7400|3|290| +2450829|7402|3|359| +2450829|7405|3|49| +2450829|7406|3|| +2450829|7408|3|626| +2450829|7411|3|877| +2450829|7412|3|7| +2450829|7414|3|876| +2450829|7417|3|922| +2450829|7418|3|949| +2450829|7420|3|23| +2450829|7423|3|356| +2450829|7424|3|858| +2450829|7426|3|786| +2450829|7429|3|645| +2450829|7430|3|647| +2450829|7432|3|345| +2450829|7435|3|| +2450829|7436|3|45| +2450829|7438|3|974| +2450829|7441|3|| +2450829|7442|3|465| +2450829|7444|3|602| +2450829|7447|3|214| +2450829|7448|3|134| +2450829|7450|3|994| +2450829|7453|3|811| +2450829|7454|3|425| +2450829|7456|3|110| +2450829|7459|3|809| +2450829|7460|3|594| +2450829|7462|3|313| +2450829|7465|3|831| +2450829|7466|3|122| +2450829|7468|3|172| +2450829|7471|3|678| +2450829|7472|3|66| +2450829|7474|3|184| +2450829|7477|3|64| +2450829|7478|3|773| +2450829|7480|3|| +2450829|7483|3|448| +2450829|7484|3|421| +2450829|7486|3|889| +2450829|7489|3|840| +2450829|7490|3|10| +2450829|7492|3|395| +2450829|7495|3|665| +2450829|7496|3|560| +2450829|7498|3|967| +2450829|7501|3|445| +2450829|7502|3|973| +2450829|7504|3|255| +2450829|7507|3|| +2450829|7508|3|295| +2450829|7510|3|587| +2450829|7513|3|403| +2450829|7514|3|127| +2450829|7516|3|206| +2450829|7519|3|521| +2450829|7520|3|15| +2450829|7522|3|521| +2450829|7525|3|402| +2450829|7526|3|954| +2450829|7528|3|846| +2450829|7531|3|431| +2450829|7532|3|731| +2450829|7534|3|636| +2450829|7537|3|90| +2450829|7538|3|86| +2450829|7540|3|808| +2450829|7543|3|983| +2450829|7544|3|545| +2450829|7546|3|564| +2450829|7549|3|654| +2450829|7550|3|788| +2450829|7552|3|391| +2450829|7555|3|131| +2450829|7556|3|962| +2450829|7558|3|860| +2450829|7561|3|370| +2450829|7562|3|716| +2450829|7564|3|467| +2450829|7567|3|| +2450829|7568|3|816| +2450829|7570|3|41| +2450829|7573|3|460| +2450829|7574|3|498| +2450829|7576|3|335| +2450829|7579|3|407| +2450829|7580|3|831| +2450829|7582|3|836| +2450829|7585|3|853| +2450829|7586|3|884| +2450829|7588|3|580| +2450829|7591|3|656| +2450829|7592|3|103| +2450829|7594|3|768| +2450829|7597|3|316| +2450829|7598|3|435| +2450829|7600|3|343| +2450829|7603|3|947| +2450829|7604|3|206| +2450829|7606|3|346| +2450829|7609|3|885| +2450829|7610|3|25| +2450829|7612|3|662| +2450829|7615|3|459| +2450829|7616|3|576| +2450829|7618|3|792| +2450829|7621|3|990| +2450829|7622|3|224| +2450829|7624|3|| +2450829|7627|3|546| +2450829|7628|3|881| +2450829|7630|3|213| +2450829|7633|3|| +2450829|7634|3|889| +2450829|7636|3|453| +2450829|7639|3|501| +2450829|7640|3|407| +2450829|7642|3|184| +2450829|7645|3|558| +2450829|7646|3|| +2450829|7648|3|563| +2450829|7651|3|921| +2450829|7652|3|904| +2450829|7654|3|773| +2450829|7657|3|909| +2450829|7658|3|687| +2450829|7660|3|504| +2450829|7663|3|849| +2450829|7664|3|374| +2450829|7666|3|939| +2450829|7669|3|410| +2450829|7670|3|| +2450829|7672|3|971| +2450829|7675|3|694| +2450829|7676|3|| +2450829|7678|3|675| +2450829|7681|3|573| +2450829|7682|3|935| +2450829|7684|3|240| +2450829|7687|3|221| +2450829|7688|3|| +2450829|7690|3|189| +2450829|7693|3|175| +2450829|7694|3|155| +2450829|7696|3|760| +2450829|7699|3|988| +2450829|7700|3|658| +2450829|7702|3|564| +2450829|7705|3|543| +2450829|7706|3|| +2450829|7708|3|172| +2450829|7711|3|840| +2450829|7712|3|942| +2450829|7714|3|81| +2450829|7717|3|769| +2450829|7718|3|652| +2450829|7720|3|722| +2450829|7723|3|113| +2450829|7724|3|904| +2450829|7726|3|55| +2450829|7729|3|768| +2450829|7730|3|546| +2450829|7732|3|672| +2450829|7735|3|87| +2450829|7736|3|537| +2450829|7738|3|687| +2450829|7741|3|791| +2450829|7742|3|| +2450829|7744|3|293| +2450829|7747|3|331| +2450829|7748|3|136| +2450829|7750|3|30| +2450829|7753|3|107| +2450829|7754|3|201| +2450829|7756|3|465| +2450829|7759|3|381| +2450829|7760|3|228| +2450829|7762|3|594| +2450829|7765|3|465| +2450829|7766|3|789| +2450829|7768|3|419| +2450829|7771|3|769| +2450829|7772|3|690| +2450829|7774|3|| +2450829|7777|3|774| +2450829|7778|3|| +2450829|7780|3|697| +2450829|7783|3|201| +2450829|7784|3|596| +2450829|7786|3|577| +2450829|7789|3|34| +2450829|7790|3|830| +2450829|7792|3|667| +2450829|7795|3|192| +2450829|7796|3|820| +2450829|7798|3|114| +2450829|7801|3|65| +2450829|7802|3|28| +2450829|7804|3|947| +2450829|7807|3|540| +2450829|7808|3|881| +2450829|7810|3|754| +2450829|7813|3|455| +2450829|7814|3|963| +2450829|7816|3|369| +2450829|7819|3|913| +2450829|7820|3|775| +2450829|7822|3|10| +2450829|7825|3|| +2450829|7826|3|397| +2450829|7828|3|852| +2450829|7831|3|384| +2450829|7832|3|601| +2450829|7834|3|804| +2450829|7837|3|297| +2450829|7838|3|187| +2450829|7840|3|236| +2450829|7843|3|840| +2450829|7844|3|135| +2450829|7846|3|266| +2450829|7849|3|| +2450829|7850|3|719| +2450829|7852|3|379| +2450829|7855|3|537| +2450829|7856|3|863| +2450829|7858|3|472| +2450829|7861|3|466| +2450829|7862|3|179| +2450829|7864|3|149| +2450829|7867|3|631| +2450829|7868|3|765| +2450829|7870|3|441| +2450829|7873|3|95| +2450829|7874|3|916| +2450829|7876|3|279| +2450829|7879|3|742| +2450829|7880|3|28| +2450829|7882|3|361| +2450829|7885|3|| +2450829|7886|3|433| +2450829|7888|3|781| +2450829|7891|3|47| +2450829|7892|3|527| +2450829|7894|3|427| +2450829|7897|3|357| +2450829|7898|3|362| +2450829|7900|3|555| +2450829|7903|3|822| +2450829|7904|3|407| +2450829|7906|3|147| +2450829|7909|3|998| +2450829|7910|3|901| +2450829|7912|3|212| +2450829|7915|3|922| +2450829|7916|3|850| +2450829|7918|3|803| +2450829|7921|3|821| +2450829|7922|3|238| +2450829|7924|3|3| +2450829|7927|3|322| +2450829|7928|3|337| +2450829|7930|3|530| +2450829|7933|3|621| +2450829|7934|3|627| +2450829|7936|3|437| +2450829|7939|3|634| +2450829|7940|3|308| +2450829|7942|3|68| +2450829|7945|3|471| +2450829|7946|3|732| +2450829|7948|3|461| +2450829|7951|3|245| +2450829|7952|3|713| +2450829|7954|3|974| +2450829|7957|3|958| +2450829|7958|3|767| +2450829|7960|3|256| +2450829|7963|3|968| +2450829|7964|3|112| +2450829|7966|3|46| +2450829|7969|3|80| +2450829|7970|3|607| +2450829|7972|3|458| +2450829|7975|3|344| +2450829|7976|3|188| +2450829|7978|3|274| +2450829|7981|3|| +2450829|7982|3|726| +2450829|7984|3|387| +2450829|7987|3|621| +2450829|7988|3|617| +2450829|7990|3|494| +2450829|7993|3|625| +2450829|7994|3|769| +2450829|7996|3|| +2450829|7999|3|192| +2450829|8000|3|854| +2450829|8002|3|80| +2450829|8005|3|594| +2450829|8006|3|558| +2450829|8008|3|710| +2450829|8011|3|767| +2450829|8012|3|617| +2450829|8014|3|308| +2450829|8017|3|175| +2450829|8018|3|677| +2450829|8020|3|839| +2450829|8023|3|984| +2450829|8024|3|405| +2450829|8026|3|490| +2450829|8029|3|294| +2450829|8030|3|732| +2450829|8032|3|319| +2450829|8035|3|101| +2450829|8036|3|863| +2450829|8038|3|614| +2450829|8041|3|932| +2450829|8042|3|853| +2450829|8044|3|599| +2450829|8047|3|384| +2450829|8048|3|219| +2450829|8050|3|527| +2450829|8053|3|398| +2450829|8054|3|918| +2450829|8056|3|638| +2450829|8059|3|644| +2450829|8060|3|849| +2450829|8062|3|689| +2450829|8065|3|775| +2450829|8066|3|894| +2450829|8068|3|453| +2450829|8071|3|603| +2450829|8072|3|542| +2450829|8074|3|278| +2450829|8077|3|670| +2450829|8078|3|891| +2450829|8080|3|389| +2450829|8083|3|370| +2450829|8084|3|256| +2450829|8086|3|896| +2450829|8089|3|201| +2450829|8090|3|319| +2450829|8092|3|79| +2450829|8095|3|883| +2450829|8096|3|547| +2450829|8098|3|676| +2450829|8101|3|98| +2450829|8102|3|599| +2450829|8104|3|278| +2450829|8107|3|864| +2450829|8108|3|901| +2450829|8110|3|672| +2450829|8113|3|556| +2450829|8114|3|322| +2450829|8116|3|443| +2450829|8119|3|364| +2450829|8120|3|805| +2450829|8122|3|766| +2450829|8125|3|865| +2450829|8126|3|94| +2450829|8128|3|528| +2450829|8131|3|669| +2450829|8132|3|773| +2450829|8134|3|186| +2450829|8137|3|392| +2450829|8138|3|704| +2450829|8140|3|872| +2450829|8143|3|4| +2450829|8144|3|| +2450829|8146|3|631| +2450829|8149|3|496| +2450829|8150|3|879| +2450829|8152|3|270| +2450829|8155|3|520| +2450829|8156|3|223| +2450829|8158|3|740| +2450829|8161|3|85| +2450829|8162|3|265| +2450829|8164|3|852| +2450829|8167|3|457| +2450829|8168|3|375| +2450829|8170|3|793| +2450829|8173|3|179| +2450829|8174|3|374| +2450829|8176|3|305| +2450829|8179|3|477| +2450829|8180|3|945| +2450829|8182|3|827| +2450829|8185|3|831| +2450829|8186|3|137| +2450829|8188|3|47| +2450829|8191|3|359| +2450829|8192|3|168| +2450829|8194|3|982| +2450829|8197|3|171| +2450829|8198|3|442| +2450829|8200|3|579| +2450829|8203|3|473| +2450829|8204|3|979| +2450829|8206|3|486| +2450829|8209|3|654| +2450829|8210|3|183| +2450829|8212|3|951| +2450829|8215|3|479| +2450829|8216|3|| +2450829|8218|3|844| +2450829|8221|3|322| +2450829|8222|3|305| +2450829|8224|3|848| +2450829|8227|3|416| +2450829|8228|3|126| +2450829|8230|3|928| +2450829|8233|3|186| +2450829|8234|3|819| +2450829|8236|3|224| +2450829|8239|3|208| +2450829|8240|3|624| +2450829|8242|3|59| +2450829|8245|3|732| +2450829|8246|3|790| +2450829|8248|3|940| +2450829|8251|3|242| +2450829|8252|3|471| +2450829|8254|3|549| +2450829|8257|3|| +2450829|8258|3|| +2450829|8260|3|337| +2450829|8263|3|172| +2450829|8264|3|913| +2450829|8266|3|905| +2450829|8269|3|805| +2450829|8270|3|570| +2450829|8272|3|33| +2450829|8275|3|186| +2450829|8276|3|623| +2450829|8278|3|357| +2450829|8281|3|471| +2450829|8282|3|696| +2450829|8284|3|61| +2450829|8287|3|14| +2450829|8288|3|143| +2450829|8290|3|617| +2450829|8293|3|913| +2450829|8294|3|994| +2450829|8296|3|548| +2450829|8299|3|878| +2450829|8300|3|129| +2450829|8302|3|849| +2450829|8305|3|366| +2450829|8306|3|858| +2450829|8308|3|964| +2450829|8311|3|883| +2450829|8312|3|270| +2450829|8314|3|365| +2450829|8317|3|237| +2450829|8318|3|159| +2450829|8320|3|544| +2450829|8323|3|956| +2450829|8324|3|728| +2450829|8326|3|568| +2450829|8329|3|581| +2450829|8330|3|170| +2450829|8332|3|632| +2450829|8335|3|676| +2450829|8336|3|357| +2450829|8338|3|151| +2450829|8341|3|542| +2450829|8342|3|437| +2450829|8344|3|289| +2450829|8347|3|| +2450829|8348|3|366| +2450829|8350|3|984| +2450829|8353|3|903| +2450829|8354|3|114| +2450829|8356|3|19| +2450829|8359|3|35| +2450829|8360|3|574| +2450829|8362|3|934| +2450829|8365|3|845| +2450829|8366|3|713| +2450829|8368|3|54| +2450829|8371|3|391| +2450829|8372|3|123| +2450829|8374|3|426| +2450829|8377|3|805| +2450829|8378|3|952| +2450829|8380|3|383| +2450829|8383|3|811| +2450829|8384|3|| +2450829|8386|3|637| +2450829|8389|3|707| +2450829|8390|3|491| +2450829|8392|3|663| +2450829|8395|3|620| +2450829|8396|3|656| +2450829|8398|3|914| +2450829|8401|3|704| +2450829|8402|3|384| +2450829|8404|3|477| +2450829|8407|3|765| +2450829|8408|3|98| +2450829|8410|3|267| +2450829|8413|3|392| +2450829|8414|3|308| +2450829|8416|3|807| +2450829|8419|3|398| +2450829|8420|3|190| +2450829|8422|3|712| +2450829|8425|3|751| +2450829|8426|3|| +2450829|8428|3|401| +2450829|8431|3|65| +2450829|8432|3|783| +2450829|8434|3|620| +2450829|8437|3|398| +2450829|8438|3|298| +2450829|8440|3|262| +2450829|8443|3|377| +2450829|8444|3|795| +2450829|8446|3|141| +2450829|8449|3|548| +2450829|8450|3|543| +2450829|8452|3|619| +2450829|8455|3|957| +2450829|8456|3|719| +2450829|8458|3|301| +2450829|8461|3|66| +2450829|8462|3|| +2450829|8464|3|31| +2450829|8467|3|933| +2450829|8468|3|848| +2450829|8470|3|254| +2450829|8473|3|345| +2450829|8474|3|333| +2450829|8476|3|270| +2450829|8479|3|809| +2450829|8480|3|257| +2450829|8482|3|932| +2450829|8485|3|983| +2450829|8486|3|824| +2450829|8488|3|230| +2450829|8491|3|454| +2450829|8492|3|447| +2450829|8494|3|242| +2450829|8497|3|642| +2450829|8498|3|259| +2450829|8500|3|259| +2450829|8503|3|573| +2450829|8504|3|164| +2450829|8506|3|758| +2450829|8509|3|379| +2450829|8510|3|| +2450829|8512|3|620| +2450829|8515|3|779| +2450829|8516|3|675| +2450829|8518|3|404| +2450829|8521|3|183| +2450829|8522|3|575| +2450829|8524|3|694| +2450829|8527|3|801| +2450829|8528|3|836| +2450829|8530|3|429| +2450829|8533|3|662| +2450829|8534|3|849| +2450829|8536|3|454| +2450829|8539|3|68| +2450829|8540|3|700| +2450829|8542|3|93| +2450829|8545|3|131| +2450829|8546|3|600| +2450829|8548|3|166| +2450829|8551|3|304| +2450829|8552|3|904| +2450829|8554|3|180| +2450829|8557|3|127| +2450829|8558|3|60| +2450829|8560|3|398| +2450829|8563|3|488| +2450829|8564|3|619| +2450829|8566|3|| +2450829|8569|3|480| +2450829|8570|3|402| +2450829|8572|3|668| +2450829|8575|3|136| +2450829|8576|3|429| +2450829|8578|3|382| +2450829|8581|3|707| +2450829|8582|3|63| +2450829|8584|3|562| +2450829|8587|3|505| +2450829|8588|3|852| +2450829|8590|3|166| +2450829|8593|3|460| +2450829|8594|3|373| +2450829|8596|3|781| +2450829|8599|3|742| +2450829|8600|3|944| +2450829|8602|3|832| +2450829|8605|3|282| +2450829|8606|3|377| +2450829|8608|3|284| +2450829|8611|3|627| +2450829|8612|3|226| +2450829|8614|3|758| +2450829|8617|3|| +2450829|8618|3|828| +2450829|8620|3|459| +2450829|8623|3|751| +2450829|8624|3|856| +2450829|8626|3|624| +2450829|8629|3|299| +2450829|8630|3|279| +2450829|8632|3|111| +2450829|8635|3|265| +2450829|8636|3|901| +2450829|8638|3|20| +2450829|8641|3|617| +2450829|8642|3|66| +2450829|8644|3|943| +2450829|8647|3|688| +2450829|8648|3|427| +2450829|8650|3|599| +2450829|8653|3|168| +2450829|8654|3|729| +2450829|8656|3|211| +2450829|8659|3|899| +2450829|8660|3|857| +2450829|8662|3|119| +2450829|8665|3|| +2450829|8666|3|996| +2450829|8668|3|58| +2450829|8671|3|974| +2450829|8672|3|708| +2450829|8674|3|340| +2450829|8677|3|656| +2450829|8678|3|930| +2450829|8680|3|940| +2450829|8683|3|796| +2450829|8684|3|160| +2450829|8686|3|655| +2450829|8689|3|149| +2450829|8690|3|594| +2450829|8692|3|42| +2450829|8695|3|893| +2450829|8696|3|871| +2450829|8698|3|155| +2450829|8701|3|87| +2450829|8702|3|356| +2450829|8704|3|888| +2450829|8707|3|692| +2450829|8708|3|884| +2450829|8710|3|17| +2450829|8713|3|12| +2450829|8714|3|912| +2450829|8716|3|944| +2450829|8719|3|| +2450829|8720|3|459| +2450829|8722|3|463| +2450829|8725|3|473| +2450829|8726|3|246| +2450829|8728|3|299| +2450829|8731|3|107| +2450829|8732|3|715| +2450829|8734|3|424| +2450829|8737|3|683| +2450829|8738|3|475| +2450829|8740|3|494| +2450829|8743|3|893| +2450829|8744|3|673| +2450829|8746|3|144| +2450829|8749|3|198| +2450829|8750|3|754| +2450829|8752|3|320| +2450829|8755|3|429| +2450829|8756|3|190| +2450829|8758|3|978| +2450829|8761|3|537| +2450829|8762|3|960| +2450829|8764|3|40| +2450829|8767|3|986| +2450829|8768|3|891| +2450829|8770|3|596| +2450829|8773|3|68| +2450829|8774|3|842| +2450829|8776|3|478| +2450829|8779|3|438| +2450829|8780|3|603| +2450829|8782|3|20| +2450829|8785|3|539| +2450829|8786|3|183| +2450829|8788|3|571| +2450829|8791|3|250| +2450829|8792|3|35| +2450829|8794|3|172| +2450829|8797|3|740| +2450829|8798|3|876| +2450829|8800|3|369| +2450829|8803|3|743| +2450829|8804|3|740| +2450829|8806|3|723| +2450829|8809|3|963| +2450829|8810|3|852| +2450829|8812|3|972| +2450829|8815|3|| +2450829|8816|3|| +2450829|8818|3|283| +2450829|8821|3|497| +2450829|8822|3|858| +2450829|8824|3|696| +2450829|8827|3|364| +2450829|8828|3|421| +2450829|8830|3|502| +2450829|8833|3|814| +2450829|8834|3|309| +2450829|8836|3|498| +2450829|8839|3|833| +2450829|8840|3|596| +2450829|8842|3|198| +2450829|8845|3|34| +2450829|8846|3|894| +2450829|8848|3|602| +2450829|8851|3|| +2450829|8852|3|349| +2450829|8854|3|896| +2450829|8857|3|598| +2450829|8858|3|569| +2450829|8860|3|388| +2450829|8863|3|610| +2450829|8864|3|57| +2450829|8866|3|848| +2450829|8869|3|61| +2450829|8870|3|103| +2450829|8872|3|| +2450829|8875|3|497| +2450829|8876|3|941| +2450829|8878|3|965| +2450829|8881|3|618| +2450829|8882|3|255| +2450829|8884|3|9| +2450829|8887|3|169| +2450829|8888|3|| +2450829|8890|3|995| +2450829|8893|3|907| +2450829|8894|3|| +2450829|8896|3|384| +2450829|8899|3|203| +2450829|8900|3|384| +2450829|8902|3|736| +2450829|8905|3|28| +2450829|8906|3|467| +2450829|8908|3|591| +2450829|8911|3|526| +2450829|8912|3|853| +2450829|8914|3|147| +2450829|8917|3|620| +2450829|8918|3|927| +2450829|8920|3|91| +2450829|8923|3|660| +2450829|8924|3|191| +2450829|8926|3|497| +2450829|8929|3|908| +2450829|8930|3|940| +2450829|8932|3|114| +2450829|8935|3|427| +2450829|8936|3|803| +2450829|8938|3|625| +2450829|8941|3|660| +2450829|8942|3|709| +2450829|8944|3|905| +2450829|8947|3|571| +2450829|8948|3|411| +2450829|8950|3|408| +2450829|8953|3|129| +2450829|8954|3|169| +2450829|8956|3|978| +2450829|8959|3|28| +2450829|8960|3|902| +2450829|8962|3|891| +2450829|8965|3|356| +2450829|8966|3|248| +2450829|8968|3|617| +2450829|8971|3|615| +2450829|8972|3|983| +2450829|8974|3|3| +2450829|8977|3|366| +2450829|8978|3|65| +2450829|8980|3|666| +2450829|8983|3|568| +2450829|8984|3|957| +2450829|8986|3|727| +2450829|8989|3|365| +2450829|8990|3|487| +2450829|8992|3|177| +2450829|8995|3|463| +2450829|8996|3|434| +2450829|8998|3|68| +2450829|9001|3|101| +2450829|9002|3|542| +2450829|9004|3|775| +2450829|9007|3|885| +2450829|9008|3|920| +2450829|9010|3|648| +2450829|9013|3|344| +2450829|9014|3|126| +2450829|9016|3|682| +2450829|9019|3|687| +2450829|9020|3|419| +2450829|9022|3|483| +2450829|9025|3|599| +2450829|9026|3|307| +2450829|9028|3|421| +2450829|9031|3|690| +2450829|9032|3|687| +2450829|9034|3|640| +2450829|9037|3|263| +2450829|9038|3|363| +2450829|9040|3|486| +2450829|9043|3|366| +2450829|9044|3|568| +2450829|9046|3|819| +2450829|9049|3|881| +2450829|9050|3|133| +2450829|9052|3|490| +2450829|9055|3|179| +2450829|9056|3|922| +2450829|9058|3|970| +2450829|9061|3|205| +2450829|9062|3|712| +2450829|9064|3|357| +2450829|9067|3|119| +2450829|9068|3|595| +2450829|9070|3|| +2450829|9073|3|733| +2450829|9074|3|549| +2450829|9076|3|179| +2450829|9079|3|356| +2450829|9080|3|134| +2450829|9082|3|476| +2450829|9085|3|211| +2450829|9086|3|151| +2450829|9088|3|274| +2450829|9091|3|668| +2450829|9092|3|15| +2450829|9094|3|715| +2450829|9097|3|745| +2450829|9098|3|823| +2450829|9100|3|917| +2450829|9103|3|119| +2450829|9104|3|396| +2450829|9106|3|439| +2450829|9109|3|474| +2450829|9110|3|429| +2450829|9112|3|151| +2450829|9115|3|353| +2450829|9116|3|922| +2450829|9118|3|788| +2450829|9121|3|721| +2450829|9122|3|573| +2450829|9124|3|455| +2450829|9127|3|635| +2450829|9128|3|| +2450829|9130|3|233| +2450829|9133|3|695| +2450829|9134|3|627| +2450829|9136|3|568| +2450829|9139|3|841| +2450829|9140|3|472| +2450829|9142|3|980| +2450829|9145|3|939| +2450829|9146|3|932| +2450829|9148|3|516| +2450829|9151|3|434| +2450829|9152|3|361| +2450829|9154|3|946| +2450829|9157|3|236| +2450829|9158|3|407| +2450829|9160|3|| +2450829|9163|3|417| +2450829|9164|3|158| +2450829|9166|3|368| +2450829|9169|3|| +2450829|9170|3|349| +2450829|9172|3|620| +2450829|9175|3|| +2450829|9176|3|679| +2450829|9178|3|965| +2450829|9181|3|715| +2450829|9182|3|442| +2450829|9184|3|247| +2450829|9187|3|444| +2450829|9188|3|14| +2450829|9190|3|622| +2450829|9193|3|950| +2450829|9194|3|638| +2450829|9196|3|| +2450829|9199|3|978| +2450829|9200|3|819| +2450829|9202|3|57| +2450829|9205|3|| +2450829|9206|3|891| +2450829|9208|3|102| +2450829|9211|3|12| +2450829|9212|3|604| +2450829|9214|3|270| +2450829|9217|3|935| +2450829|9218|3|861| +2450829|9220|3|584| +2450829|9223|3|998| +2450829|9224|3|742| +2450829|9226|3|260| +2450829|9229|3|41| +2450829|9230|3|717| +2450829|9232|3|308| +2450829|9235|3|286| +2450829|9236|3|546| +2450829|9238|3|| +2450829|9241|3|295| +2450829|9242|3|278| +2450829|9244|3|470| +2450829|9247|3|585| +2450829|9248|3|355| +2450829|9250|3|115| +2450829|9253|3|| +2450829|9254|3|910| +2450829|9256|3|948| +2450829|9259|3|152| +2450829|9260|3|38| +2450829|9262|3|641| +2450829|9265|3|780| +2450829|9266|3|731| +2450829|9268|3|812| +2450829|9271|3|115| +2450829|9272|3|93| +2450829|9274|3|467| +2450829|9277|3|423| +2450829|9278|3|801| +2450829|9280|3|67| +2450829|9283|3|857| +2450829|9284|3|902| +2450829|9286|3|672| +2450829|9289|3|96| +2450829|9290|3|| +2450829|9292|3|326| +2450829|9295|3|5| +2450829|9296|3|| +2450829|9298|3|| +2450829|9301|3|787| +2450829|9302|3|8| +2450829|9304|3|396| +2450829|9307|3|896| +2450829|9308|3|462| +2450829|9310|3|352| +2450829|9313|3|719| +2450829|9314|3|846| +2450829|9316|3|435| +2450829|9319|3|691| +2450829|9320|3|| +2450829|9322|3|46| +2450829|9325|3|902| +2450829|9326|3|755| +2450829|9328|3|660| +2450829|9331|3|57| +2450829|9332|3|707| +2450829|9334|3|540| +2450829|9337|3|883| +2450829|9338|3|677| +2450829|9340|3|196| +2450829|9343|3|168| +2450829|9344|3|746| +2450829|9346|3|556| +2450829|9349|3|954| +2450829|9350|3|675| +2450829|9352|3|544| +2450829|9355|3|758| +2450829|9356|3|237| +2450829|9358|3|477| +2450829|9361|3|276| +2450829|9362|3|742| +2450829|9364|3|80| +2450829|9367|3|758| +2450829|9368|3|781| +2450829|9370|3|240| +2450829|9373|3|852| +2450829|9374|3|909| +2450829|9376|3|556| +2450829|9379|3|196| +2450829|9380|3|91| +2450829|9382|3|483| +2450829|9385|3|327| +2450829|9386|3|815| +2450829|9388|3|252| +2450829|9391|3|968| +2450829|9392|3|127| +2450829|9394|3|595| +2450829|9397|3|| +2450829|9398|3|974| +2450829|9400|3|233| +2450829|9403|3|702| +2450829|9404|3|557| +2450829|9406|3|885| +2450829|9409|3|115| +2450829|9410|3|923| +2450829|9412|3|91| +2450829|9415|3|366| +2450829|9416|3|| +2450829|9418|3|801| +2450829|9421|3|599| +2450829|9422|3|591| +2450829|9424|3|816| +2450829|9427|3|875| +2450829|9428|3|711| +2450829|9430|3|833| +2450829|9433|3|| +2450829|9434|3|574| +2450829|9436|3|653| +2450829|9439|3|389| +2450829|9440|3|745| +2450829|9442|3|705| +2450829|9445|3|546| +2450829|9446|3|168| +2450829|9448|3|771| +2450829|9451|3|983| +2450829|9452|3|205| +2450829|9454|3|899| +2450829|9457|3|41| +2450829|9458|3|834| +2450829|9460|3|519| +2450829|9463|3|534| +2450829|9464|3|28| +2450829|9466|3|56| +2450829|9469|3|880| +2450829|9470|3|637| +2450829|9472|3|153| +2450829|9475|3|361| +2450829|9476|3|833| +2450829|9478|3|323| +2450829|9481|3|496| +2450829|9482|3|0| +2450829|9484|3|235| +2450829|9487|3|432| +2450829|9488|3|113| +2450829|9490|3|274| +2450829|9493|3|919| +2450829|9494|3|292| +2450829|9496|3|503| +2450829|9499|3|93| +2450829|9500|3|716| +2450829|9502|3|843| +2450829|9505|3|64| +2450829|9506|3|586| +2450829|9508|3|84| +2450829|9511|3|362| +2450829|9512|3|104| +2450829|9514|3|638| +2450829|9517|3|530| +2450829|9518|3|| +2450829|9520|3|116| +2450829|9523|3|927| +2450829|9524|3|356| +2450829|9526|3|884| +2450829|9529|3|536| +2450829|9530|3|338| +2450829|9532|3|827| +2450829|9535|3|620| +2450829|9536|3|| +2450829|9538|3|154| +2450829|9541|3|594| +2450829|9542|3|117| +2450829|9544|3|218| +2450829|9547|3|192| +2450829|9548|3|486| +2450829|9550|3|904| +2450829|9553|3|| +2450829|9554|3|361| +2450829|9556|3|834| +2450829|9559|3|981| +2450829|9560|3|993| +2450829|9562|3|358| +2450829|9565|3|904| +2450829|9566|3|425| +2450829|9568|3|570| +2450829|9571|3|692| +2450829|9572|3|313| +2450829|9574|3|518| +2450829|9577|3|816| +2450829|9578|3|| +2450829|9580|3|543| +2450829|9583|3|| +2450829|9584|3|559| +2450829|9586|3|77| +2450829|9589|3|526| +2450829|9590|3|850| +2450829|9592|3|233| +2450829|9595|3|343| +2450829|9596|3|785| +2450829|9598|3|460| +2450829|9601|3|288| +2450829|9602|3|523| +2450829|9604|3|435| +2450829|9607|3|394| +2450829|9608|3|250| +2450829|9610|3|558| +2450829|9613|3|460| +2450829|9614|3|599| +2450829|9616|3|46| +2450829|9619|3|1000| +2450829|9620|3|648| +2450829|9622|3|10| +2450829|9625|3|622| +2450829|9626|3|518| +2450829|9628|3|210| +2450829|9631|3|123| +2450829|9632|3|364| +2450829|9634|3|717| +2450829|9637|3|835| +2450829|9638|3|923| +2450829|9640|3|660| +2450829|9643|3|0| +2450829|9644|3|318| +2450829|9646|3|341| +2450829|9649|3|386| +2450829|9650|3|720| +2450829|9652|3|217| +2450829|9655|3|648| +2450829|9656|3|30| +2450829|9658|3|492| +2450829|9661|3|325| +2450829|9662|3|674| +2450829|9664|3|| +2450829|9667|3|374| +2450829|9668|3|638| +2450829|9670|3|676| +2450829|9673|3|289| +2450829|9674|3|730| +2450829|9676|3|895| +2450829|9679|3|113| +2450829|9680|3|44| +2450829|9682|3|| +2450829|9685|3|434| +2450829|9686|3|481| +2450829|9688|3|749| +2450829|9691|3|922| +2450829|9692|3|118| +2450829|9694|3|176| +2450829|9697|3|619| +2450829|9698|3|852| +2450829|9700|3|829| +2450829|9703|3|534| +2450829|9704|3|375| +2450829|9706|3|810| +2450829|9709|3|730| +2450829|9710|3|45| +2450829|9712|3|| +2450829|9715|3|683| +2450829|9716|3|993| +2450829|9718|3|275| +2450829|9721|3|994| +2450829|9722|3|709| +2450829|9724|3|929| +2450829|9727|3|396| +2450829|9728|3|798| +2450829|9730|3|832| +2450829|9733|3|281| +2450829|9734|3|94| +2450829|9736|3|615| +2450829|9739|3|39| +2450829|9740|3|945| +2450829|9742|3|66| +2450829|9745|3|750| +2450829|9746|3|656| +2450829|9748|3|868| +2450829|9751|3|85| +2450829|9752|3|530| +2450829|9754|3|641| +2450829|9757|3|115| +2450829|9758|3|359| +2450829|9760|3|881| +2450829|9763|3|758| +2450829|9764|3|557| +2450829|9766|3|881| +2450829|9769|3|106| +2450829|9770|3|763| +2450829|9772|3|371| +2450829|9775|3|183| +2450829|9776|3|569| +2450829|9778|3|947| +2450829|9781|3|269| +2450829|9782|3|818| +2450829|9784|3|604| +2450829|9787|3|924| +2450829|9788|3|401| +2450829|9790|3|660| +2450829|9793|3|951| +2450829|9794|3|165| +2450829|9796|3|13| +2450829|9799|3|193| +2450829|9800|3|60| +2450829|9802|3|594| +2450829|9805|3|664| +2450829|9806|3|74| +2450829|9808|3|399| +2450829|9811|3|992| +2450829|9812|3|296| +2450829|9814|3|534| +2450829|9817|3|984| +2450829|9818|3|601| +2450829|9820|3|984| +2450829|9823|3|168| +2450829|9824|3|623| +2450829|9826|3|8| +2450829|9829|3|315| +2450829|9830|3|| +2450829|9832|3|997| +2450829|9835|3|267| +2450829|9836|3|502| +2450829|9838|3|825| +2450829|9841|3|43| +2450829|9842|3|135| +2450829|9844|3|| +2450829|9847|3|25| +2450829|9848|3|55| +2450829|9850|3|98| +2450829|9853|3|| +2450829|9854|3|345| +2450829|9856|3|| +2450829|9859|3|240| +2450829|9860|3|628| +2450829|9862|3|570| +2450829|9865|3|235| +2450829|9866|3|919| +2450829|9868|3|815| +2450829|9871|3|197| +2450829|9872|3|25| +2450829|9874|3|423| +2450829|9877|3|49| +2450829|9878|3|362| +2450829|9880|3|398| +2450829|9883|3|296| +2450829|9884|3|146| +2450829|9886|3|945| +2450829|9889|3|327| +2450829|9890|3|21| +2450829|9892|3|222| +2450829|9895|3|796| +2450829|9896|3|134| +2450829|9898|3|838| +2450829|9901|3|755| +2450829|9902|3|699| +2450829|9904|3|174| +2450829|9907|3|859| +2450829|9908|3|201| +2450829|9910|3|306| +2450829|9913|3|828| +2450829|9914|3|97| +2450829|9916|3|423| +2450829|9919|3|586| +2450829|9920|3|888| +2450829|9922|3|594| +2450829|9925|3|272| +2450829|9926|3|736| +2450829|9928|3|893| +2450829|9931|3|820| +2450829|9932|3|848| +2450829|9934|3|71| +2450829|9937|3|454| +2450829|9938|3|661| +2450829|9940|3|470| +2450829|9943|3|574| +2450829|9944|3|825| +2450829|9946|3|757| +2450829|9949|3|35| +2450829|9950|3|518| +2450829|9952|3|964| +2450829|9955|3|961| +2450829|9956|3|908| +2450829|9958|3|632| +2450829|9961|3|775| +2450829|9962|3|635| +2450829|9964|3|350| +2450829|9967|3|628| +2450829|9968|3|18| +2450829|9970|3|932| +2450829|9973|3|288| +2450829|9974|3|843| +2450829|9976|3|81| +2450829|9979|3|203| +2450829|9980|3|335| +2450829|9982|3|895| +2450829|9985|3|713| +2450829|9986|3|876| +2450829|9988|3|144| +2450829|9991|3|373| +2450829|9992|3|266| +2450829|9994|3|311| +2450829|9997|3|781| +2450829|9998|3|646| +2450829|10000|3|201| +2450829|10003|3|344| +2450829|10004|3|88| +2450829|10006|3|219| +2450829|10009|3|128| +2450829|10010|3|20| +2450829|10012|3|871| +2450829|10015|3|166| +2450829|10016|3|592| +2450829|10018|3|279| +2450829|10021|3|326| +2450829|10022|3|585| +2450829|10024|3|759| +2450829|10027|3|805| +2450829|10028|3|| +2450829|10030|3|744| +2450829|10033|3|779| +2450829|10034|3|994| +2450829|10036|3|895| +2450829|10039|3|751| +2450829|10040|3|265| +2450829|10042|3|696| +2450829|10045|3|80| +2450829|10046|3|93| +2450829|10048|3|881| +2450829|10051|3|899| +2450829|10052|3|597| +2450829|10054|3|| +2450829|10057|3|191| +2450829|10058|3|59| +2450829|10060|3|777| +2450829|10063|3|59| +2450829|10064|3|40| +2450829|10066|3|| +2450829|10069|3|527| +2450829|10070|3|724| +2450829|10072|3|851| +2450829|10075|3|997| +2450829|10076|3|332| +2450829|10078|3|96| +2450829|10081|3|341| +2450829|10082|3|561| +2450829|10084|3|559| +2450829|10087|3|457| +2450829|10088|3|| +2450829|10090|3|999| +2450829|10093|3|766| +2450829|10094|3|308| +2450829|10096|3|608| +2450829|10099|3|253| +2450829|10100|3|452| +2450829|10102|3|846| +2450829|10105|3|| +2450829|10106|3|992| +2450829|10108|3|| +2450829|10111|3|327| +2450829|10112|3|402| +2450829|10114|3|143| +2450829|10117|3|189| +2450829|10118|3|897| +2450829|10120|3|| +2450829|10123|3|449| +2450829|10124|3|584| +2450829|10126|3|219| +2450829|10129|3|954| +2450829|10130|3|88| +2450829|10132|3|41| +2450829|10135|3|325| +2450829|10136|3|509| +2450829|10138|3|489| +2450829|10141|3|342| +2450829|10142|3|279| +2450829|10144|3|803| +2450829|10147|3|502| +2450829|10148|3|| +2450829|10150|3|584| +2450829|10153|3|267| +2450829|10154|3|702| +2450829|10156|3|311| +2450829|10159|3|601| +2450829|10160|3|295| +2450829|10162|3|431| +2450829|10165|3|952| +2450829|10166|3|352| +2450829|10168|3|363| +2450829|10171|3|699| +2450829|10172|3|83| +2450829|10174|3|407| +2450829|10177|3|924| +2450829|10178|3|868| +2450829|10180|3|518| +2450829|10183|3|610| +2450829|10184|3|| +2450829|10186|3|8| +2450829|10189|3|822| +2450829|10190|3|752| +2450829|10192|3|192| +2450829|10195|3|472| +2450829|10196|3|538| +2450829|10198|3|997| +2450829|10201|3|26| +2450829|10202|3|583| +2450829|10204|3|334| +2450829|10207|3|164| +2450829|10208|3|411| +2450829|10210|3|779| +2450829|10213|3|994| +2450829|10214|3|94| +2450829|10216|3|967| +2450829|10219|3|938| +2450829|10220|3|299| +2450829|10222|3|230| +2450829|10225|3|88| +2450829|10226|3|537| +2450829|10228|3|| +2450829|10231|3|817| +2450829|10232|3|973| +2450829|10234|3|215| +2450829|10237|3|766| +2450829|10238|3|699| +2450829|10240|3|198| +2450829|10243|3|762| +2450829|10244|3|142| +2450829|10246|3|686| +2450829|10249|3|49| +2450829|10250|3|737| +2450829|10252|3|491| +2450829|10255|3|215| +2450829|10256|3|| +2450829|10258|3|122| +2450829|10261|3|303| +2450829|10262|3|148| +2450829|10264|3|395| +2450829|10267|3|115| +2450829|10268|3|| +2450829|10270|3|823| +2450829|10273|3|793| +2450829|10274|3|676| +2450829|10276|3|313| +2450829|10279|3|666| +2450829|10280|3|401| +2450829|10282|3|377| +2450829|10285|3|619| +2450829|10286|3|442| +2450829|10288|3|28| +2450829|10291|3|453| +2450829|10292|3|727| +2450829|10294|3|133| +2450829|10297|3|307| +2450829|10298|3|513| +2450829|10300|3|957| +2450829|10303|3|851| +2450829|10304|3|183| +2450829|10306|3|| +2450829|10309|3|938| +2450829|10310|3|423| +2450829|10312|3|755| +2450829|10315|3|529| +2450829|10316|3|225| +2450829|10318|3|476| +2450829|10321|3|668| +2450829|10322|3|688| +2450829|10324|3|289| +2450829|10327|3|719| +2450829|10328|3|310| +2450829|10330|3|995| +2450829|10333|3|153| +2450829|10334|3|| +2450829|10336|3|405| +2450829|10339|3|272| +2450829|10340|3|570| +2450829|10342|3|184| +2450829|10345|3|536| +2450829|10346|3|641| +2450829|10348|3|13| +2450829|10351|3|153| +2450829|10352|3|223| +2450829|10354|3|722| +2450829|10357|3|501| +2450829|10358|3|401| +2450829|10360|3|575| +2450829|10363|3|823| +2450829|10364|3|| +2450829|10366|3|542| +2450829|10369|3|660| +2450829|10370|3|849| +2450829|10372|3|615| +2450829|10375|3|692| +2450829|10376|3|81| +2450829|10378|3|636| +2450829|10381|3|431| +2450829|10382|3|879| +2450829|10384|3|574| +2450829|10387|3|18| +2450829|10388|3|28| +2450829|10390|3|384| +2450829|10393|3|137| +2450829|10394|3|623| +2450829|10396|3|| +2450829|10399|3|445| +2450829|10400|3|518| +2450829|10402|3|429| +2450829|10405|3|747| +2450829|10406|3|259| +2450829|10408|3|709| +2450829|10411|3|678| +2450829|10412|3|917| +2450829|10414|3|188| +2450829|10417|3|581| +2450829|10418|3|336| +2450829|10420|3|472| +2450829|10423|3|781| +2450829|10424|3|| +2450829|10426|3|654| +2450829|10429|3|832| +2450829|10430|3|872| +2450829|10432|3|367| +2450829|10435|3|769| +2450829|10436|3|826| +2450829|10438|3|748| +2450829|10441|3|359| +2450829|10442|3|81| +2450829|10444|3|15| +2450829|10447|3|336| +2450829|10448|3|| +2450829|10450|3|132| +2450829|10453|3|309| +2450829|10454|3|284| +2450829|10456|3|16| +2450829|10459|3|748| +2450829|10460|3|73| +2450829|10462|3|704| +2450829|10465|3|514| +2450829|10466|3|665| +2450829|10468|3|584| +2450829|10471|3|9| +2450829|10472|3|127| +2450829|10474|3|| +2450829|10477|3|662| +2450829|10478|3|997| +2450829|10480|3|786| +2450829|10483|3|868| +2450829|10484|3|0| +2450829|10486|3|786| +2450829|10489|3|917| +2450829|10490|3|453| +2450829|10492|3|748| +2450829|10495|3|400| +2450829|10496|3|498| +2450829|10498|3|499| +2450829|10501|3|253| +2450829|10502|3|216| +2450829|10504|3|701| +2450829|10507|3|554| +2450829|10508|3|57| +2450829|10510|3|882| +2450829|10513|3|14| +2450829|10514|3|403| +2450829|10516|3|892| +2450829|10519|3|595| +2450829|10520|3|537| +2450829|10522|3|23| +2450829|10525|3|117| +2450829|10526|3|592| +2450829|10528|3|674| +2450829|10531|3|| +2450829|10532|3|950| +2450829|10534|3|694| +2450829|10537|3|81| +2450829|10538|3|734| +2450829|10540|3|119| +2450829|10543|3|268| +2450829|10544|3|781| +2450829|10546|3|126| +2450829|10549|3|542| +2450829|10550|3|630| +2450829|10552|3|695| +2450829|10555|3|66| +2450829|10556|3|223| +2450829|10558|3|| +2450829|10561|3|399| +2450829|10562|3|225| +2450829|10564|3|529| +2450829|10567|3|928| +2450829|10568|3|734| +2450829|10570|3|804| +2450829|10573|3|844| +2450829|10574|3|601| +2450829|10576|3|690| +2450829|10579|3|910| +2450829|10580|3|278| +2450829|10582|3|473| +2450829|10585|3|15| +2450829|10586|3|356| +2450829|10588|3|809| +2450829|10591|3|608| +2450829|10592|3|924| +2450829|10594|3|| +2450829|10597|3|149| +2450829|10598|3|453| +2450829|10600|3|113| +2450829|10603|3|662| +2450829|10604|3|355| +2450829|10606|3|405| +2450829|10609|3|610| +2450829|10610|3|295| +2450829|10612|3|40| +2450829|10615|3|503| +2450829|10616|3|768| +2450829|10618|3|103| +2450829|10621|3|790| +2450829|10622|3|962| +2450829|10624|3|68| +2450829|10627|3|813| +2450829|10628|3|537| +2450829|10630|3|22| +2450829|10633|3|430| +2450829|10634|3|613| +2450829|10636|3|138| +2450829|10639|3|242| +2450829|10640|3|647| +2450829|10642|3|861| +2450829|10645|3|823| +2450829|10646|3|267| +2450829|10648|3|678| +2450829|10651|3|30| +2450829|10652|3|856| +2450829|10654|3|50| +2450829|10657|3|966| +2450829|10658|3|736| +2450829|10660|3|1000| +2450829|10663|3|875| +2450829|10664|3|238| +2450829|10666|3|690| +2450829|10669|3|144| +2450829|10670|3|21| +2450829|10672|3|721| +2450829|10675|3|348| +2450829|10676|3|784| +2450829|10678|3|623| +2450829|10681|3|310| +2450829|10682|3|245| +2450829|10684|3|874| +2450829|10687|3|529| +2450829|10688|3|652| +2450829|10690|3|522| +2450829|10693|3|670| +2450829|10694|3|73| +2450829|10696|3|106| +2450829|10699|3|571| +2450829|10700|3|774| +2450829|10702|3|25| +2450829|10705|3|284| +2450829|10706|3|595| +2450829|10708|3|516| +2450829|10711|3|503| +2450829|10712|3|764| +2450829|10714|3|654| +2450829|10717|3|640| +2450829|10718|3|703| +2450829|10720|3|506| +2450829|10723|3|744| +2450829|10724|3|751| +2450829|10726|3|480| +2450829|10729|3|54| +2450829|10730|3|310| +2450829|10732|3|609| +2450829|10735|3|730| +2450829|10736|3|838| +2450829|10738|3|420| +2450829|10741|3|501| +2450829|10742|3|579| +2450829|10744|3|733| +2450829|10747|3|665| +2450829|10748|3|268| +2450829|10750|3|461| +2450829|10753|3|439| +2450829|10754|3|487| +2450829|10756|3|| +2450829|10759|3|143| +2450829|10760|3|618| +2450829|10762|3|358| +2450829|10765|3|728| +2450829|10766|3|97| +2450829|10768|3|869| +2450829|10771|3|317| +2450829|10772|3|64| +2450829|10774|3|903| +2450829|10777|3|658| +2450829|10778|3|776| +2450829|10780|3|224| +2450829|10783|3|| +2450829|10784|3|321| +2450829|10786|3|145| +2450829|10789|3|959| +2450829|10790|3|248| +2450829|10792|3|752| +2450829|10795|3|740| +2450829|10796|3|59| +2450829|10798|3|904| +2450829|10801|3|284| +2450829|10802|3|951| +2450829|10804|3|116| +2450829|10807|3|222| +2450829|10808|3|846| +2450829|10810|3|136| +2450829|10813|3|| +2450829|10814|3|693| +2450829|10816|3|93| +2450829|10819|3|867| +2450829|10820|3|684| +2450829|10822|3|991| +2450829|10825|3|957| +2450829|10826|3|306| +2450829|10828|3|6| +2450829|10831|3|238| +2450829|10832|3|| +2450829|10834|3|351| +2450829|10837|3|705| +2450829|10838|3|441| +2450829|10840|3|729| +2450829|10843|3|777| +2450829|10844|3|995| +2450829|10846|3|661| +2450829|10849|3|980| +2450829|10850|3|181| +2450829|10852|3|29| +2450829|10855|3|929| +2450829|10856|3|452| +2450829|10858|3|660| +2450829|10861|3|727| +2450829|10862|3|393| +2450829|10864|3|955| +2450829|10867|3|969| +2450829|10868|3|108| +2450829|10870|3|791| +2450829|10873|3|391| +2450829|10874|3|963| +2450829|10876|3|867| +2450829|10879|3|142| +2450829|10880|3|719| +2450829|10882|3|138| +2450829|10885|3|838| +2450829|10886|3|154| +2450829|10888|3|978| +2450829|10891|3|753| +2450829|10892|3|220| +2450829|10894|3|| +2450829|10897|3|250| +2450829|10898|3|944| +2450829|10900|3|560| +2450829|10903|3|815| +2450829|10904|3|227| +2450829|10906|3|| +2450829|10909|3|465| +2450829|10910|3|616| +2450829|10912|3|430| +2450829|10915|3|955| +2450829|10916|3|281| +2450829|10918|3|683| +2450829|10921|3|280| +2450829|10922|3|769| +2450829|10924|3|928| +2450829|10927|3|848| +2450829|10928|3|624| +2450829|10930|3|845| +2450829|10933|3|418| +2450829|10934|3|368| +2450829|10936|3|446| +2450829|10939|3|810| +2450829|10940|3|505| +2450829|10942|3|459| +2450829|10945|3|537| +2450829|10946|3|467| +2450829|10948|3|735| +2450829|10951|3|232| +2450829|10952|3|404| +2450829|10954|3|148| +2450829|10957|3|441| +2450829|10958|3|830| +2450829|10960|3|863| +2450829|10963|3|687| +2450829|10964|3|67| +2450829|10966|3|388| +2450829|10969|3|729| +2450829|10970|3|526| +2450829|10972|3|531| +2450829|10975|3|753| +2450829|10976|3|559| +2450829|10978|3|| +2450829|10981|3|971| +2450829|10982|3|574| +2450829|10984|3|669| +2450829|10987|3|600| +2450829|10988|3|536| +2450829|10990|3|991| +2450829|10993|3|694| +2450829|10994|3|81| +2450829|10996|3|908| +2450829|10999|3|927| +2450829|11000|3|200| +2450829|11002|3|90| +2450829|11005|3|344| +2450829|11006|3|43| +2450829|11008|3|| +2450829|11011|3|222| +2450829|11012|3|240| +2450829|11014|3|199| +2450829|11017|3|108| +2450829|11018|3|232| +2450829|11020|3|878| +2450829|11023|3|631| +2450829|11024|3|762| +2450829|11026|3|737| +2450829|11029|3|658| +2450829|11030|3|426| +2450829|11032|3|159| +2450829|11035|3|604| +2450829|11036|3|447| +2450829|11038|3|821| +2450829|11041|3|30| +2450829|11042|3|195| +2450829|11044|3|826| +2450829|11047|3|197| +2450829|11048|3|53| +2450829|11050|3|831| +2450829|11053|3|211| +2450829|11054|3|959| +2450829|11056|3|30| +2450829|11059|3|905| +2450829|11060|3|84| +2450829|11062|3|1000| +2450829|11065|3|547| +2450829|11066|3|85| +2450829|11068|3|887| +2450829|11071|3|304| +2450829|11072|3|179| +2450829|11074|3|76| +2450829|11077|3|29| +2450829|11078|3|597| +2450829|11080|3|419| +2450829|11083|3|926| +2450829|11084|3|423| +2450829|11086|3|982| +2450829|11089|3|389| +2450829|11090|3|905| +2450829|11092|3|827| +2450829|11095|3|947| +2450829|11096|3|550| +2450829|11098|3|690| +2450829|11101|3|737| +2450829|11102|3|357| +2450829|11104|3|335| +2450829|11107|3|882| +2450829|11108|3|934| +2450829|11110|3|608| +2450829|11113|3|676| +2450829|11114|3|957| +2450829|11116|3|169| +2450829|11119|3|855| +2450829|11120|3|530| +2450829|11122|3|321| +2450829|11125|3|81| +2450829|11126|3|852| +2450829|11128|3|804| +2450829|11131|3|406| +2450829|11132|3|9| +2450829|11134|3|82| +2450829|11137|3|505| +2450829|11138|3|252| +2450829|11140|3|196| +2450829|11143|3|354| +2450829|11144|3|963| +2450829|11146|3|925| +2450829|11149|3|| +2450829|11150|3|604| +2450829|11152|3|917| +2450829|11155|3|311| +2450829|11156|3|84| +2450829|11158|3|689| +2450829|11161|3|925| +2450829|11162|3|899| +2450829|11164|3|941| +2450829|11167|3|776| +2450829|11168|3|| +2450829|11170|3|446| +2450829|11173|3|193| +2450829|11174|3|885| +2450829|11176|3|158| +2450829|11179|3|931| +2450829|11180|3|225| +2450829|11182|3|461| +2450829|11185|3|723| +2450829|11186|3|594| +2450829|11188|3|282| +2450829|11191|3|994| +2450829|11192|3|924| +2450829|11194|3|999| +2450829|11197|3|70| +2450829|11198|3|934| +2450829|11200|3|898| +2450829|11203|3|627| +2450829|11204|3|21| +2450829|11206|3|269| +2450829|11209|3|422| +2450829|11210|3|593| +2450829|11212|3|999| +2450829|11215|3|746| +2450829|11216|3|| +2450829|11218|3|608| +2450829|11221|3|340| +2450829|11222|3|941| +2450829|11224|3|524| +2450829|11227|3|| +2450829|11228|3|551| +2450829|11230|3|646| +2450829|11233|3|654| +2450829|11234|3|419| +2450829|11236|3|125| +2450829|11239|3|834| +2450829|11240|3|992| +2450829|11242|3|61| +2450829|11245|3|158| +2450829|11246|3|277| +2450829|11248|3|197| +2450829|11251|3|59| +2450829|11252|3|695| +2450829|11254|3|472| +2450829|11257|3|99| +2450829|11258|3|480| +2450829|11260|3|339| +2450829|11263|3|874| +2450829|11264|3|774| +2450829|11266|3|452| +2450829|11269|3|211| +2450829|11270|3|126| +2450829|11272|3|455| +2450829|11275|3|| +2450829|11276|3|290| +2450829|11278|3|575| +2450829|11281|3|738| +2450829|11282|3|975| +2450829|11284|3|558| +2450829|11287|3|476| +2450829|11288|3|693| +2450829|11290|3|827| +2450829|11293|3|429| +2450829|11294|3|685| +2450829|11296|3|21| +2450829|11299|3|637| +2450829|11300|3|698| +2450829|11302|3|569| +2450829|11305|3|848| +2450829|11306|3|968| +2450829|11308|3|670| +2450829|11311|3|677| +2450829|11312|3|600| +2450829|11314|3|942| +2450829|11317|3|298| +2450829|11318|3|21| +2450829|11320|3|647| +2450829|11323|3|566| +2450829|11324|3|393| +2450829|11326|3|456| +2450829|11329|3|896| +2450829|11330|3|794| +2450829|11332|3|317| +2450829|11335|3|788| +2450829|11336|3|9| +2450829|11338|3|175| +2450829|11341|3|235| +2450829|11342|3|949| +2450829|11344|3|102| +2450829|11347|3|905| +2450829|11348|3|358| +2450829|11350|3|847| +2450829|11353|3|357| +2450829|11354|3|517| +2450829|11356|3|213| +2450829|11359|3|482| +2450829|11360|3|747| +2450829|11362|3|253| +2450829|11365|3|751| +2450829|11366|3|188| +2450829|11368|3|639| +2450829|11371|3|468| +2450829|11372|3|45| +2450829|11374|3|548| +2450829|11377|3|969| +2450829|11378|3|482| +2450829|11380|3|222| +2450829|11383|3|762| +2450829|11384|3|913| +2450829|11386|3|565| +2450829|11389|3|900| +2450829|11390|3|147| +2450829|11392|3|638| +2450829|11395|3|999| +2450829|11396|3|82| +2450829|11398|3|390| +2450829|11401|3|417| +2450829|11402|3|348| +2450829|11404|3|844| +2450829|11407|3|186| +2450829|11408|3|872| +2450829|11410|3|69| +2450829|11413|3|673| +2450829|11414|3|31| +2450829|11416|3|790| +2450829|11419|3|446| +2450829|11420|3|46| +2450829|11422|3|104| +2450829|11425|3|150| +2450829|11426|3|494| +2450829|11428|3|| +2450829|11431|3|517| +2450829|11432|3|419| +2450829|11434|3|979| +2450829|11437|3|270| +2450829|11438|3|245| +2450829|11440|3|257| +2450829|11443|3|967| +2450829|11444|3|228| +2450829|11446|3|197| +2450829|11449|3|| +2450829|11450|3|922| +2450829|11452|3|696| +2450829|11455|3|810| +2450829|11456|3|810| +2450829|11458|3|442| +2450829|11461|3|991| +2450829|11462|3|651| +2450829|11464|3|542| +2450829|11467|3|521| +2450829|11468|3|| +2450829|11470|3|882| +2450829|11473|3|82| +2450829|11474|3|365| +2450829|11476|3|307| +2450829|11479|3|| +2450829|11480|3|932| +2450829|11482|3|211| +2450829|11485|3|498| +2450829|11486|3|991| +2450829|11488|3|| +2450829|11491|3|540| +2450829|11492|3|473| +2450829|11494|3|67| +2450829|11497|3|149| +2450829|11498|3|153| +2450829|11500|3|137| +2450829|11503|3|162| +2450829|11504|3|711| +2450829|11506|3|405| +2450829|11509|3|394| +2450829|11510|3|856| +2450829|11512|3|652| +2450829|11515|3|| +2450829|11516|3|815| +2450829|11518|3|923| +2450829|11521|3|358| +2450829|11522|3|849| +2450829|11524|3|218| +2450829|11527|3|142| +2450829|11528|3|130| +2450829|11530|3|378| +2450829|11533|3|75| +2450829|11534|3|869| +2450829|11536|3|835| +2450829|11539|3|461| +2450829|11540|3|515| +2450829|11542|3|851| +2450829|11545|3|160| +2450829|11546|3|588| +2450829|11548|3|580| +2450829|11551|3|621| +2450829|11552|3|887| +2450829|11554|3|124| +2450829|11557|3|966| +2450829|11558|3|580| +2450829|11560|3|516| +2450829|11563|3|987| +2450829|11564|3|547| +2450829|11566|3|769| +2450829|11569|3|426| +2450829|11570|3|472| +2450829|11572|3|785| +2450829|11575|3|773| +2450829|11576|3|977| +2450829|11578|3|152| +2450829|11581|3|534| +2450829|11582|3|808| +2450829|11584|3|678| +2450829|11587|3|71| +2450829|11588|3|822| +2450829|11590|3|1000| +2450829|11593|3|57| +2450829|11594|3|878| +2450829|11596|3|309| +2450829|11599|3|56| +2450829|11600|3|877| +2450829|11602|3|221| +2450829|11605|3|974| +2450829|11606|3|386| +2450829|11608|3|87| +2450829|11611|3|50| +2450829|11612|3|125| +2450829|11614|3|454| +2450829|11617|3|746| +2450829|11618|3|| +2450829|11620|3|220| +2450829|11623|3|940| +2450829|11624|3|426| +2450829|11626|3|59| +2450829|11629|3|878| +2450829|11630|3|404| +2450829|11632|3|481| +2450829|11635|3|819| +2450829|11636|3|786| +2450829|11638|3|341| +2450829|11641|3|270| +2450829|11642|3|301| +2450829|11644|3|589| +2450829|11647|3|812| +2450829|11648|3|550| +2450829|11650|3|770| +2450829|11653|3|275| +2450829|11654|3|| +2450829|11656|3|208| +2450829|11659|3|862| +2450829|11660|3|| +2450829|11662|3|228| +2450829|11665|3|787| +2450829|11666|3|430| +2450829|11668|3|796| +2450829|11671|3|650| +2450829|11672|3|518| +2450829|11674|3|73| +2450829|11677|3|901| +2450829|11678|3|259| +2450829|11680|3|336| +2450829|11683|3|679| +2450829|11684|3|73| +2450829|11686|3|225| +2450829|11689|3|732| +2450829|11690|3|980| +2450829|11692|3|532| +2450829|11695|3|727| +2450829|11696|3|| +2450829|11698|3|109| +2450829|11701|3|833| +2450829|11702|3|724| +2450829|11704|3|927| +2450829|11707|3|540| +2450829|11708|3|985| +2450829|11710|3|461| +2450829|11713|3|768| +2450829|11714|3|353| +2450829|11716|3|395| +2450829|11719|3|993| +2450829|11720|3|249| +2450829|11722|3|513| +2450829|11725|3|671| +2450829|11726|3|667| +2450829|11728|3|858| +2450829|11731|3|210| +2450829|11732|3|956| +2450829|11734|3|110| +2450829|11737|3|97| +2450829|11738|3|773| +2450829|11740|3|| +2450829|11743|3|78| +2450829|11744|3|72| +2450829|11746|3|508| +2450829|11749|3|| +2450829|11750|3|165| +2450829|11752|3|29| +2450829|11755|3|748| +2450829|11756|3|728| +2450829|11758|3|613| +2450829|11761|3|320| +2450829|11762|3|568| +2450829|11764|3|486| +2450829|11767|3|581| +2450829|11768|3|914| +2450829|11770|3|| +2450829|11773|3|| +2450829|11774|3|| +2450829|11776|3|461| +2450829|11779|3|456| +2450829|11780|3|679| +2450829|11782|3|967| +2450829|11785|3|76| +2450829|11786|3|112| +2450829|11788|3|224| +2450829|11791|3|262| +2450829|11792|3|265| +2450829|11794|3|217| +2450829|11797|3|915| +2450829|11798|3|480| +2450829|11800|3|894| +2450829|11803|3|1000| +2450829|11804|3|44| +2450829|11806|3|938| +2450829|11809|3|330| +2450829|11810|3|948| +2450829|11812|3|416| +2450829|11815|3|924| +2450829|11816|3|910| +2450829|11818|3|855| +2450829|11821|3|374| +2450829|11822|3|674| +2450829|11824|3|473| +2450829|11827|3|465| +2450829|11828|3|95| +2450829|11830|3|393| +2450829|11833|3|784| +2450829|11834|3|476| +2450829|11836|3|529| +2450829|11839|3|566| +2450829|11840|3|189| +2450829|11842|3|585| +2450829|11845|3|696| +2450829|11846|3|410| +2450829|11848|3|492| +2450829|11851|3|201| +2450829|11852|3|633| +2450829|11854|3|29| +2450829|11857|3|| +2450829|11858|3|14| +2450829|11860|3|668| +2450829|11863|3|284| +2450829|11864|3|245| +2450829|11866|3|883| +2450829|11869|3|843| +2450829|11870|3|692| +2450829|11872|3|726| +2450829|11875|3|720| +2450829|11876|3|167| +2450829|11878|3|342| +2450829|11881|3|351| +2450829|11882|3|414| +2450829|11884|3|736| +2450829|11887|3|515| +2450829|11888|3|167| +2450829|11890|3|859| +2450829|11893|3|900| +2450829|11894|3|214| +2450829|11896|3|896| +2450829|11899|3|677| +2450829|11900|3|303| +2450829|11902|3|44| +2450829|11905|3|9| +2450829|11906|3|29| +2450829|11908|3|| +2450829|11911|3|819| +2450829|11912|3|109| +2450829|11914|3|496| +2450829|11917|3|656| +2450829|11918|3|391| +2450829|11920|3|849| +2450829|11923|3|926| +2450829|11924|3|360| +2450829|11926|3|989| +2450829|11929|3|333| +2450829|11930|3|379| +2450829|11932|3|39| +2450829|11935|3|710| +2450829|11936|3|66| +2450829|11938|3|208| +2450829|11941|3|224| +2450829|11942|3|151| +2450829|11944|3|983| +2450829|11947|3|| +2450829|11948|3|534| +2450829|11950|3|456| +2450829|11953|3|| +2450829|11954|3|322| +2450829|11956|3|275| +2450829|11959|3|331| +2450829|11960|3|921| +2450829|11962|3|69| +2450829|11965|3|558| +2450829|11966|3|246| +2450829|11968|3|682| +2450829|11971|3|168| +2450829|11972|3|596| +2450829|11974|3|72| +2450829|11977|3|882| +2450829|11978|3|440| +2450829|11980|3|| +2450829|11983|3|960| +2450829|11984|3|92| +2450829|11986|3|151| +2450829|11989|3|939| +2450829|11990|3|| +2450829|11992|3|440| +2450829|11995|3|136| +2450829|11996|3|540| +2450829|11998|3|177| +2450829|12001|3|848| +2450829|12002|3|951| +2450829|12004|3|881| +2450829|12007|3|824| +2450829|12008|3|24| +2450829|12010|3|293| +2450829|12013|3|620| +2450829|12014|3|148| +2450829|12016|3|397| +2450829|12019|3|487| +2450829|12020|3|982| +2450829|12022|3|965| +2450829|12025|3|815| +2450829|12026|3|492| +2450829|12028|3|3| +2450829|12031|3|939| +2450829|12032|3|622| +2450829|12034|3|720| +2450829|12037|3|60| +2450829|12038|3|103| +2450829|12040|3|577| +2450829|12043|3|268| +2450829|12044|3|913| +2450829|12046|3|330| +2450829|12049|3|802| +2450829|12050|3|154| +2450829|12052|3|199| +2450829|12055|3|281| +2450829|12056|3|983| +2450829|12058|3|963| +2450829|12061|3|495| +2450829|12062|3|27| +2450829|12064|3|785| +2450829|12067|3|980| +2450829|12068|3|723| +2450829|12070|3|976| +2450829|12073|3|286| +2450829|12074|3|0| +2450829|12076|3|168| +2450829|12079|3|732| +2450829|12080|3|788| +2450829|12082|3|887| +2450829|12085|3|592| +2450829|12086|3|538| +2450829|12088|3|| +2450829|12091|3|840| +2450829|12092|3|221| +2450829|12094|3|126| +2450829|12097|3|566| +2450829|12098|3|273| +2450829|12100|3|509| +2450829|12103|3|374| +2450829|12104|3|322| +2450829|12106|3|798| +2450829|12109|3|| +2450829|12110|3|453| +2450829|12112|3|718| +2450829|12115|3|899| +2450829|12116|3|436| +2450829|12118|3|158| +2450829|12121|3|| +2450829|12122|3|993| +2450829|12124|3|793| +2450829|12127|3|771| +2450829|12128|3|427| +2450829|12130|3|527| +2450829|12133|3|154| +2450829|12134|3|618| +2450829|12136|3|993| +2450829|12139|3|264| +2450829|12140|3|592| +2450829|12142|3|658| +2450829|12145|3|10| +2450829|12146|3|898| +2450829|12148|3|367| +2450829|12151|3|408| +2450829|12152|3|440| +2450829|12154|3|| +2450829|12157|3|713| +2450829|12158|3|663| +2450829|12160|3|737| +2450829|12163|3|56| +2450829|12164|3|977| +2450829|12166|3|2| +2450829|12169|3|198| +2450829|12170|3|646| +2450829|12172|3|332| +2450829|12175|3|513| +2450829|12176|3|218| +2450829|12178|3|601| +2450829|12181|3|949| +2450829|12182|3|27| +2450829|12184|3|527| +2450829|12187|3|837| +2450829|12188|3|693| +2450829|12190|3|664| +2450829|12193|3|82| +2450829|12194|3|363| +2450829|12196|3|294| +2450829|12199|3|369| +2450829|12200|3|724| +2450829|12202|3|105| +2450829|12205|3|40| +2450829|12206|3|| +2450829|12208|3|316| +2450829|12211|3|693| +2450829|12212|3|538| +2450829|12214|3|| +2450829|12217|3|82| +2450829|12218|3|131| +2450829|12220|3|176| +2450829|12223|3|273| +2450829|12224|3|937| +2450829|12226|3|717| +2450829|12229|3|346| +2450829|12230|3|281| +2450829|12232|3|| +2450829|12235|3|907| +2450829|12236|3|133| +2450829|12238|3|596| +2450829|12241|3|821| +2450829|12242|3|| +2450829|12244|3|929| +2450829|12247|3|799| +2450829|12248|3|494| +2450829|12250|3|917| +2450829|12253|3|290| +2450829|12254|3|710| +2450829|12256|3|642| +2450829|12259|3|855| +2450829|12260|3|822| +2450829|12262|3|308| +2450829|12265|3|931| +2450829|12266|3|536| +2450829|12268|3|719| +2450829|12271|3|147| +2450829|12272|3|977| +2450829|12274|3|732| +2450829|12277|3|516| +2450829|12278|3|724| +2450829|12280|3|681| +2450829|12283|3|825| +2450829|12284|3|999| +2450829|12286|3|556| +2450829|12289|3|673| +2450829|12290|3|549| +2450829|12292|3|541| +2450829|12295|3|240| +2450829|12296|3|953| +2450829|12298|3|188| +2450829|12301|3|462| +2450829|12302|3|680| +2450829|12304|3|230| +2450829|12307|3|515| +2450829|12308|3|821| +2450829|12310|3|934| +2450829|12313|3|751| +2450829|12314|3|761| +2450829|12316|3|805| +2450829|12319|3|272| +2450829|12320|3|232| +2450829|12322|3|641| +2450829|12325|3|640| +2450829|12326|3|100| +2450829|12328|3|621| +2450829|12331|3|679| +2450829|12332|3|124| +2450829|12334|3|18| +2450829|12337|3|643| +2450829|12338|3|179| +2450829|12340|3|493| +2450829|12343|3|903| +2450829|12344|3|1000| +2450829|12346|3|979| +2450829|12349|3|880| +2450829|12350|3|973| +2450829|12352|3|67| +2450829|12355|3|145| +2450829|12356|3|590| +2450829|12358|3|688| +2450829|12361|3|620| +2450829|12362|3|63| +2450829|12364|3|491| +2450829|12367|3|944| +2450829|12368|3|824| +2450829|12370|3|517| +2450829|12373|3|768| +2450829|12374|3|68| +2450829|12376|3|53| +2450829|12379|3|761| +2450829|12380|3|193| +2450829|12382|3|10| +2450829|12385|3|978| +2450829|12386|3|318| +2450829|12388|3|837| +2450829|12391|3|878| +2450829|12392|3|141| +2450829|12394|3|703| +2450829|12397|3|128| +2450829|12398|3|858| +2450829|12400|3|336| +2450829|12403|3|153| +2450829|12404|3|372| +2450829|12406|3|576| +2450829|12409|3|397| +2450829|12410|3|731| +2450829|12412|3|865| +2450829|12415|3|860| +2450829|12416|3|379| +2450829|12418|3|855| +2450829|12421|3|204| +2450829|12422|3|247| +2450829|12424|3|774| +2450829|12427|3|292| +2450829|12428|3|986| +2450829|12430|3|689| +2450829|12433|3|34| +2450829|12434|3|777| +2450829|12436|3|618| +2450829|12439|3|801| +2450829|12440|3|258| +2450829|12442|3|| +2450829|12445|3|906| +2450829|12446|3|326| +2450829|12448|3|460| +2450829|12451|3|488| +2450829|12452|3|562| +2450829|12454|3|| +2450829|12457|3|742| +2450829|12458|3|770| +2450829|12460|3|113| +2450829|12463|3|674| +2450829|12464|3|952| +2450829|12466|3|| +2450829|12469|3|46| +2450829|12470|3|496| +2450829|12472|3|735| +2450829|12475|3|110| +2450829|12476|3|355| +2450829|12478|3|254| +2450829|12481|3|836| +2450829|12482|3|534| +2450829|12484|3|| +2450829|12487|3|489| +2450829|12488|3|185| +2450829|12490|3|263| +2450829|12493|3|593| +2450829|12494|3|316| +2450829|12496|3|| +2450829|12499|3|361| +2450829|12500|3|54| +2450829|12502|3|117| +2450829|12505|3|664| +2450829|12506|3|59| +2450829|12508|3|973| +2450829|12511|3|358| +2450829|12512|3|471| +2450829|12514|3|587| +2450829|12517|3|535| +2450829|12518|3|354| +2450829|12520|3|920| +2450829|12523|3|29| +2450829|12524|3|480| +2450829|12526|3|236| +2450829|12529|3|25| +2450829|12530|3|351| +2450829|12532|3|640| +2450829|12535|3|213| +2450829|12536|3|98| +2450829|12538|3|72| +2450829|12541|3|614| +2450829|12542|3|382| +2450829|12544|3|774| +2450829|12547|3|36| +2450829|12548|3|83| +2450829|12550|3|916| +2450829|12553|3|53| +2450829|12554|3|65| +2450829|12556|3|725| +2450829|12559|3|641| +2450829|12560|3|431| +2450829|12562|3|809| +2450829|12565|3|435| +2450829|12566|3|422| +2450829|12568|3|981| +2450829|12571|3|| +2450829|12572|3|| +2450829|12574|3|627| +2450829|12577|3|331| +2450829|12578|3|447| +2450829|12580|3|679| +2450829|12583|3|155| +2450829|12584|3|318| +2450829|12586|3|310| +2450829|12589|3|625| +2450829|12590|3|79| +2450829|12592|3|666| +2450829|12595|3|865| +2450829|12596|3|645| +2450829|12598|3|257| +2450829|12601|3|602| +2450829|12602|3|466| +2450829|12604|3|880| +2450829|12607|3|688| +2450829|12608|3|651| +2450829|12610|3|| +2450829|12613|3|902| +2450829|12614|3|745| +2450829|12616|3|434| +2450829|12619|3|299| +2450829|12620|3|635| +2450829|12622|3|712| +2450829|12625|3|288| +2450829|12626|3|672| +2450829|12628|3|822| +2450829|12631|3|502| +2450829|12632|3|95| +2450829|12634|3|278| +2450829|12637|3|559| +2450829|12638|3|166| +2450829|12640|3|438| +2450829|12643|3|596| +2450829|12644|3|752| +2450829|12646|3|688| +2450829|12649|3|35| +2450829|12650|3|726| +2450829|12652|3|709| +2450829|12655|3|100| +2450829|12656|3|986| +2450829|12658|3|745| +2450829|12661|3|271| +2450829|12662|3|456| +2450829|12664|3|27| +2450829|12667|3|229| +2450829|12668|3|214| +2450829|12670|3|833| +2450829|12673|3|216| +2450829|12674|3|635| +2450829|12676|3|425| +2450829|12679|3|871| +2450829|12680|3|550| +2450829|12682|3|715| +2450829|12685|3|807| +2450829|12686|3|713| +2450829|12688|3|863| +2450829|12691|3|538| +2450829|12692|3|609| +2450829|12694|3|126| +2450829|12697|3|346| +2450829|12698|3|531| +2450829|12700|3|214| +2450829|12703|3|349| +2450829|12704|3|637| +2450829|12706|3|373| +2450829|12709|3|141| +2450829|12710|3|12| +2450829|12712|3|445| +2450829|12715|3|433| +2450829|12716|3|408| +2450829|12718|3|956| +2450829|12721|3|216| +2450829|12722|3|20| +2450829|12724|3|201| +2450829|12727|3|522| +2450829|12728|3|700| +2450829|12730|3|120| +2450829|12733|3|454| +2450829|12734|3|380| +2450829|12736|3|999| +2450829|12739|3|102| +2450829|12740|3|374| +2450829|12742|3|630| +2450829|12745|3|747| +2450829|12746|3|550| +2450829|12748|3|169| +2450829|12751|3|638| +2450829|12752|3|485| +2450829|12754|3|893| +2450829|12757|3|133| +2450829|12758|3|32| +2450829|12760|3|337| +2450829|12763|3|146| +2450829|12764|3|995| +2450829|12766|3|480| +2450829|12769|3|631| +2450829|12770|3|457| +2450829|12772|3|250| +2450829|12775|3|| +2450829|12776|3|602| +2450829|12778|3|186| +2450829|12781|3|641| +2450829|12782|3|75| +2450829|12784|3|776| +2450829|12787|3|879| +2450829|12788|3|679| +2450829|12790|3|543| +2450829|12793|3|123| +2450829|12794|3|769| +2450829|12796|3|827| +2450829|12799|3|876| +2450829|12800|3|| +2450829|12802|3|668| +2450829|12805|3|624| +2450829|12806|3|719| +2450829|12808|3|869| +2450829|12811|3|609| +2450829|12812|3|| +2450829|12814|3|925| +2450829|12817|3|516| +2450829|12818|3|672| +2450829|12820|3|925| +2450829|12823|3|683| +2450829|12824|3|492| +2450829|12826|3|286| +2450829|12829|3|861| +2450829|12830|3|263| +2450829|12832|3|422| +2450829|12835|3|790| +2450829|12836|3|259| +2450829|12838|3|44| +2450829|12841|3|614| +2450829|12842|3|925| +2450829|12844|3|978| +2450829|12847|3|337| +2450829|12848|3|862| +2450829|12850|3|695| +2450829|12853|3|101| +2450829|12854|3|149| +2450829|12856|3|37| +2450829|12859|3|481| +2450829|12860|3|27| +2450829|12862|3|677| +2450829|12865|3|749| +2450829|12866|3|449| +2450829|12868|3|469| +2450829|12871|3|130| +2450829|12872|3|981| +2450829|12874|3|410| +2450829|12877|3|324| +2450829|12878|3|559| +2450829|12880|3|584| +2450829|12883|3|988| +2450829|12884|3|954| +2450829|12886|3|526| +2450829|12889|3|234| +2450829|12890|3|343| +2450829|12892|3|519| +2450829|12895|3|887| +2450829|12896|3|776| +2450829|12898|3|385| +2450829|12901|3|156| +2450829|12902|3|709| +2450829|12904|3|908| +2450829|12907|3|159| +2450829|12908|3|105| +2450829|12910|3|847| +2450829|12913|3|918| +2450829|12914|3|252| +2450829|12916|3|70| +2450829|12919|3|859| +2450829|12920|3|182| +2450829|12922|3|508| +2450829|12925|3|461| +2450829|12926|3|54| +2450829|12928|3|562| +2450829|12931|3|227| +2450829|12932|3|603| +2450829|12934|3|7| +2450829|12937|3|332| +2450829|12938|3|30| +2450829|12940|3|387| +2450829|12943|3|668| +2450829|12944|3|794| +2450829|12946|3|993| +2450829|12949|3|780| +2450829|12950|3|598| +2450829|12952|3|182| +2450829|12955|3|431| +2450829|12956|3|427| +2450829|12958|3|763| +2450829|12961|3|889| +2450829|12962|3|455| +2450829|12964|3|869| +2450829|12967|3|665| +2450829|12968|3|105| +2450829|12970|3|363| +2450829|12973|3|779| +2450829|12974|3|284| +2450829|12976|3|632| +2450829|12979|3|7| +2450829|12980|3|944| +2450829|12982|3|506| +2450829|12985|3|23| +2450829|12986|3|957| +2450829|12988|3|121| +2450829|12991|3|617| +2450829|12992|3|924| +2450829|12994|3|66| +2450829|12997|3|465| +2450829|12998|3|| +2450829|13000|3|54| +2450829|13003|3|106| +2450829|13004|3|377| +2450829|13006|3|326| +2450829|13009|3|719| +2450829|13010|3|834| +2450829|13012|3|74| +2450829|13015|3|163| +2450829|13016|3|43| +2450829|13018|3|674| +2450829|13021|3|| +2450829|13022|3|455| +2450829|13024|3|830| +2450829|13027|3|130| +2450829|13028|3|399| +2450829|13030|3|721| +2450829|13033|3|679| +2450829|13034|3|987| +2450829|13036|3|66| +2450829|13039|3|45| +2450829|13040|3|868| +2450829|13042|3|46| +2450829|13045|3|399| +2450829|13046|3|201| +2450829|13048|3|686| +2450829|13051|3|898| +2450829|13052|3|445| +2450829|13054|3|101| +2450829|13057|3|433| +2450829|13058|3|236| +2450829|13060|3|114| +2450829|13063|3|| +2450829|13064|3|87| +2450829|13066|3|227| +2450829|13069|3|637| +2450829|13070|3|984| +2450829|13072|3|804| +2450829|13075|3|984| +2450829|13076|3|341| +2450829|13078|3|942| +2450829|13081|3|628| +2450829|13082|3|297| +2450829|13084|3|966| +2450829|13087|3|675| +2450829|13088|3|816| +2450829|13090|3|898| +2450829|13093|3|988| +2450829|13094|3|383| +2450829|13096|3|315| +2450829|13099|3|311| +2450829|13100|3|168| +2450829|13102|3|798| +2450829|13105|3|130| +2450829|13106|3|683| +2450829|13108|3|633| +2450829|13111|3|56| +2450829|13112|3|461| +2450829|13114|3|258| +2450829|13117|3|255| +2450829|13118|3|577| +2450829|13120|3|487| +2450829|13123|3|586| +2450829|13124|3|204| +2450829|13126|3|596| +2450829|13129|3|300| +2450829|13130|3|338| +2450829|13132|3|730| +2450829|13135|3|134| +2450829|13136|3|686| +2450829|13138|3|732| +2450829|13141|3|841| +2450829|13142|3|627| +2450829|13144|3|522| +2450829|13147|3|137| +2450829|13148|3|880| +2450829|13150|3|394| +2450829|13153|3|264| +2450829|13154|3|249| +2450829|13156|3|137| +2450829|13159|3|244| +2450829|13160|3|550| +2450829|13162|3|906| +2450829|13165|3|565| +2450829|13166|3|856| +2450829|13168|3|| +2450829|13171|3|269| +2450829|13172|3|58| +2450829|13174|3|230| +2450829|13177|3|652| +2450829|13178|3|854| +2450829|13180|3|484| +2450829|13183|3|841| +2450829|13184|3|326| +2450829|13186|3|728| +2450829|13189|3|721| +2450829|13190|3|119| +2450829|13192|3|734| +2450829|13195|3|716| +2450829|13196|3|149| +2450829|13198|3|893| +2450829|13201|3|28| +2450829|13202|3|789| +2450829|13204|3|| +2450829|13207|3|784| +2450829|13208|3|245| +2450829|13210|3|337| +2450829|13213|3|791| +2450829|13214|3|171| +2450829|13216|3|923| +2450829|13219|3|416| +2450829|13220|3|827| +2450829|13222|3|945| +2450829|13225|3|| +2450829|13226|3|74| +2450829|13228|3|663| +2450829|13231|3|859| +2450829|13232|3|27| +2450829|13234|3|12| +2450829|13237|3|963| +2450829|13238|3|105| +2450829|13240|3|795| +2450829|13243|3|| +2450829|13244|3|337| +2450829|13246|3|281| +2450829|13249|3|578| +2450829|13250|3|192| +2450829|13252|3|538| +2450829|13255|3|640| +2450829|13256|3|54| +2450829|13258|3|73| +2450829|13261|3|478| +2450829|13262|3|349| +2450829|13264|3|180| +2450829|13267|3|162| +2450829|13268|3|842| +2450829|13270|3|274| +2450829|13273|3|84| +2450829|13274|3|747| +2450829|13276|3|741| +2450829|13279|3|443| +2450829|13280|3|514| +2450829|13282|3|72| +2450829|13285|3|763| +2450829|13286|3|633| +2450829|13288|3|214| +2450829|13291|3|439| +2450829|13292|3|27| +2450829|13294|3|500| +2450829|13297|3|699| +2450829|13298|3|25| +2450829|13300|3|435| +2450829|13303|3|785| +2450829|13304|3|787| +2450829|13306|3|561| +2450829|13309|3|480| +2450829|13310|3|396| +2450829|13312|3|596| +2450829|13315|3|906| +2450829|13316|3|567| +2450829|13318|3|177| +2450829|13321|3|330| +2450829|13322|3|828| +2450829|13324|3|| +2450829|13327|3|202| +2450829|13328|3|297| +2450829|13330|3|774| +2450829|13333|3|489| +2450829|13334|3|343| +2450829|13336|3|917| +2450829|13339|3|839| +2450829|13340|3|416| +2450829|13342|3|450| +2450829|13345|3|957| +2450829|13346|3|715| +2450829|13348|3|972| +2450829|13351|3|734| +2450829|13352|3|859| +2450829|13354|3|518| +2450829|13357|3|988| +2450829|13358|3|124| +2450829|13360|3|642| +2450829|13363|3|20| +2450829|13364|3|821| +2450829|13366|3|640| +2450829|13369|3|581| +2450829|13370|3|352| +2450829|13372|3|843| +2450829|13375|3|858| +2450829|13376|3|852| +2450829|13378|3|488| +2450829|13381|3|233| +2450829|13382|3|362| +2450829|13384|3|923| +2450829|13387|3|171| +2450829|13388|3|886| +2450829|13390|3|162| +2450829|13393|3|478| +2450829|13394|3|749| +2450829|13396|3|| +2450829|13399|3|853| +2450829|13400|3|709| +2450829|13402|3|999| +2450829|13405|3|| +2450829|13406|3|271| +2450829|13408|3|153| +2450829|13411|3|920| +2450829|13412|3|822| +2450829|13414|3|943| +2450829|13417|3|742| +2450829|13418|3|846| +2450829|13420|3|380| +2450829|13423|3|872| +2450829|13424|3|242| +2450829|13426|3|993| +2450829|13429|3|892| +2450829|13430|3|545| +2450829|13432|3|279| +2450829|13435|3|12| +2450829|13436|3|684| +2450829|13438|3|767| +2450829|13441|3|63| +2450829|13442|3|35| +2450829|13444|3|85| +2450829|13447|3|660| +2450829|13448|3|570| +2450829|13450|3|718| +2450829|13453|3|324| +2450829|13454|3|286| +2450829|13456|3|355| +2450829|13459|3|293| +2450829|13460|3|234| +2450829|13462|3|451| +2450829|13465|3|58| +2450829|13466|3|622| +2450829|13468|3|244| +2450829|13471|3|821| +2450829|13472|3|782| +2450829|13474|3|126| +2450829|13477|3|370| +2450829|13478|3|869| +2450829|13480|3|756| +2450829|13483|3|979| +2450829|13484|3|684| +2450829|13486|3|905| +2450829|13489|3|| +2450829|13490|3|463| +2450829|13492|3|797| +2450829|13495|3|533| +2450829|13496|3|948| +2450829|13498|3|| +2450829|13501|3|510| +2450829|13502|3|832| +2450829|13504|3|891| +2450829|13507|3|633| +2450829|13508|3|184| +2450829|13510|3|| +2450829|13513|3|370| +2450829|13514|3|314| +2450829|13516|3|734| +2450829|13519|3|181| +2450829|13520|3|153| +2450829|13522|3|849| +2450829|13525|3|942| +2450829|13526|3|255| +2450829|13528|3|623| +2450829|13531|3|475| +2450829|13532|3|777| +2450829|13534|3|518| +2450829|13537|3|814| +2450829|13538|3|981| +2450829|13540|3|917| +2450829|13543|3|470| +2450829|13544|3|864| +2450829|13546|3|29| +2450829|13549|3|548| +2450829|13550|3|789| +2450829|13552|3|413| +2450829|13555|3|909| +2450829|13556|3|124| +2450829|13558|3|886| +2450829|13561|3|617| +2450829|13562|3|818| +2450829|13564|3|453| +2450829|13567|3|144| +2450829|13568|3|613| +2450829|13570|3|534| +2450829|13573|3|733| +2450829|13574|3|370| +2450829|13576|3|393| +2450829|13579|3|47| +2450829|13580|3|734| +2450829|13582|3|244| +2450829|13585|3|131| +2450829|13586|3|| +2450829|13588|3|276| +2450829|13591|3|608| +2450829|13592|3|419| +2450829|13594|3|953| +2450829|13597|3|26| +2450829|13598|3|978| +2450829|13600|3|512| +2450829|13603|3|220| +2450829|13604|3|918| +2450829|13606|3|713| +2450829|13609|3|191| +2450829|13610|3|51| +2450829|13612|3|642| +2450829|13615|3|677| +2450829|13616|3|740| +2450829|13618|3|173| +2450829|13621|3|458| +2450829|13622|3|454| +2450829|13624|3|897| +2450829|13627|3|867| +2450829|13628|3|217| +2450829|13630|3|428| +2450829|13633|3|239| +2450829|13634|3|980| +2450829|13636|3|883| +2450829|13639|3|341| +2450829|13640|3|907| +2450829|13642|3|219| +2450829|13645|3|16| +2450829|13646|3|292| +2450829|13648|3|459| +2450829|13651|3|627| +2450829|13652|3|486| +2450829|13654|3|70| +2450829|13657|3|32| +2450829|13658|3|979| +2450829|13660|3|416| +2450829|13663|3|450| +2450829|13664|3|877| +2450829|13666|3|519| +2450829|13669|3|519| +2450829|13670|3|602| +2450829|13672|3|| +2450829|13675|3|405| +2450829|13676|3|794| +2450829|13678|3|219| +2450829|13681|3|895| +2450829|13682|3|244| +2450829|13684|3|651| +2450829|13687|3|229| +2450829|13688|3|768| +2450829|13690|3|681| +2450829|13693|3|132| +2450829|13694|3|435| +2450829|13696|3|863| +2450829|13699|3|357| +2450829|13700|3|674| +2450829|13702|3|| +2450829|13705|3|596| +2450829|13706|3|912| +2450829|13708|3|920| +2450829|13711|3|60| +2450829|13712|3|782| +2450829|13714|3|960| +2450829|13717|3|928| +2450829|13718|3|886| +2450829|13720|3|971| +2450829|13723|3|633| +2450829|13724|3|425| +2450829|13726|3|444| +2450829|13729|3|956| +2450829|13730|3|743| +2450829|13732|3|412| +2450829|13735|3|593| +2450829|13736|3|586| +2450829|13738|3|37| +2450829|13741|3|885| +2450829|13742|3|360| +2450829|13744|3|| +2450829|13747|3|544| +2450829|13748|3|149| +2450829|13750|3|638| +2450829|13753|3|834| +2450829|13754|3|209| +2450829|13756|3|180| +2450829|13759|3|248| +2450829|13760|3|976| +2450829|13762|3|774| +2450829|13765|3|168| +2450829|13766|3|47| +2450829|13768|3|216| +2450829|13771|3|704| +2450829|13772|3|859| +2450829|13774|3|478| +2450829|13777|3|678| +2450829|13778|3|122| +2450829|13780|3|443| +2450829|13783|3|355| +2450829|13784|3|216| +2450829|13786|3|192| +2450829|13789|3|139| +2450829|13790|3|117| +2450829|13792|3|516| +2450829|13795|3|417| +2450829|13796|3|534| +2450829|13798|3|856| +2450829|13801|3|743| +2450829|13802|3|287| +2450829|13804|3|| +2450829|13807|3|407| +2450829|13808|3|| +2450829|13810|3|373| +2450829|13813|3|| +2450829|13814|3|14| +2450829|13816|3|432| +2450829|13819|3|828| +2450829|13820|3|1000| +2450829|13822|3|188| +2450829|13825|3|149| +2450829|13826|3|519| +2450829|13828|3|349| +2450829|13831|3|288| +2450829|13832|3|916| +2450829|13834|3|693| +2450829|13837|3|181| +2450829|13838|3|863| +2450829|13840|3|782| +2450829|13843|3|| +2450829|13844|3|179| +2450829|13846|3|638| +2450829|13849|3|718| +2450829|13850|3|722| +2450829|13852|3|635| +2450829|13855|3|673| +2450829|13856|3|40| +2450829|13858|3|| +2450829|13861|3|309| +2450829|13862|3|530| +2450829|13864|3|453| +2450829|13867|3|99| +2450829|13868|3|61| +2450829|13870|3|641| +2450829|13873|3|537| +2450829|13874|3|709| +2450829|13876|3|825| +2450829|13879|3|802| +2450829|13880|3|528| +2450829|13882|3|128| +2450829|13885|3|535| +2450829|13886|3|976| +2450829|13888|3|560| +2450829|13891|3|858| +2450829|13892|3|16| +2450829|13894|3|103| +2450829|13897|3|| +2450829|13898|3|968| +2450829|13900|3|432| +2450829|13903|3|974| +2450829|13904|3|718| +2450829|13906|3|672| +2450829|13909|3|631| +2450829|13910|3|884| +2450829|13912|3|512| +2450829|13915|3|834| +2450829|13916|3|502| +2450829|13918|3|593| +2450829|13921|3|495| +2450829|13922|3|844| +2450829|13924|3|993| +2450829|13927|3|52| +2450829|13928|3|690| +2450829|13930|3|114| +2450829|13933|3|969| +2450829|13934|3|200| +2450829|13936|3|864| +2450829|13939|3|767| +2450829|13940|3|210| +2450829|13942|3|967| +2450829|13945|3|240| +2450829|13946|3|784| +2450829|13948|3|483| +2450829|13951|3|111| +2450829|13952|3|475| +2450829|13954|3|243| +2450829|13957|3|414| +2450829|13958|3|331| +2450829|13960|3|7| +2450829|13963|3|650| +2450829|13964|3|596| +2450829|13966|3|45| +2450829|13969|3|| +2450829|13970|3|840| +2450829|13972|3|590| +2450829|13975|3|817| +2450829|13976|3|501| +2450829|13978|3|25| +2450829|13981|3|444| +2450829|13982|3|513| +2450829|13984|3|760| +2450829|13987|3|693| +2450829|13988|3|144| +2450829|13990|3|877| +2450829|13993|3|697| +2450829|13994|3|564| +2450829|13996|3|374| +2450829|13999|3|896| +2450829|14000|3|531| +2450829|14002|3|435| +2450829|14005|3|218| +2450829|14006|3|6| +2450829|14008|3|169| +2450829|14011|3|448| +2450829|14012|3|326| +2450829|14014|3|662| +2450829|14017|3|654| +2450829|14018|3|219| +2450829|14020|3|771| +2450829|14023|3|816| +2450829|14024|3|229| +2450829|14026|3|916| +2450829|14029|3|398| +2450829|14030|3|197| +2450829|14032|3|787| +2450829|14035|3|532| +2450829|14036|3|116| +2450829|14038|3|539| +2450829|14041|3|616| +2450829|14042|3|65| +2450829|14044|3|596| +2450829|14047|3|857| +2450829|14048|3|558| +2450829|14050|3|615| +2450829|14053|3|175| +2450829|14054|3|854| +2450829|14056|3|567| +2450829|14059|3|329| +2450829|14060|3|957| +2450829|14062|3|| +2450829|14065|3|919| +2450829|14066|3|283| +2450829|14068|3|184| +2450829|14071|3|148| +2450829|14072|3|259| +2450829|14074|3|88| +2450829|14077|3|99| +2450829|14078|3|347| +2450829|14080|3|79| +2450829|14083|3|111| +2450829|14084|3|934| +2450829|14086|3|121| +2450829|14089|3|107| +2450829|14090|3|792| +2450829|14092|3|586| +2450829|14095|3|733| +2450829|14096|3|594| +2450829|14098|3|659| +2450829|14101|3|510| +2450829|14102|3|444| +2450829|14104|3|749| +2450829|14107|3|381| +2450829|14108|3|742| +2450829|14110|3|34| +2450829|14113|3|| +2450829|14114|3|| +2450829|14116|3|27| +2450829|14119|3|81| +2450829|14120|3|914| +2450829|14122|3|933| +2450829|14125|3|640| +2450829|14126|3|54| +2450829|14128|3|624| +2450829|14131|3|642| +2450829|14132|3|730| +2450829|14134|3|161| +2450829|14137|3|710| +2450829|14138|3|| +2450829|14140|3|918| +2450829|14143|3|453| +2450829|14144|3|467| +2450829|14146|3|111| +2450829|14149|3|227| +2450829|14150|3|940| +2450829|14152|3|744| +2450829|14155|3|831| +2450829|14156|3|| +2450829|14158|3|707| +2450829|14161|3|630| +2450829|14162|3|272| +2450829|14164|3|26| +2450829|14167|3|546| +2450829|14168|3|604| +2450829|14170|3|596| +2450829|14173|3|610| +2450829|14174|3|685| +2450829|14176|3|709| +2450829|14179|3|765| +2450829|14180|3|| +2450829|14182|3|552| +2450829|14185|3|701| +2450829|14186|3|20| +2450829|14188|3|442| +2450829|14191|3|303| +2450829|14192|3|703| +2450829|14194|3|614| +2450829|14197|3|374| +2450829|14198|3|760| +2450829|14200|3|329| +2450829|14203|3|427| +2450829|14204|3|605| +2450829|14206|3|396| +2450829|14209|3|581| +2450829|14210|3|230| +2450829|14212|3|969| +2450829|14215|3|521| +2450829|14216|3|737| +2450829|14218|3|875| +2450829|14221|3|| +2450829|14222|3|52| +2450829|14224|3|581| +2450829|14227|3|612| +2450829|14228|3|843| +2450829|14230|3|166| +2450829|14233|3|132| +2450829|14234|3|23| +2450829|14236|3|36| +2450829|14239|3|812| +2450829|14240|3|423| +2450829|14242|3|432| +2450829|14245|3|604| +2450829|14246|3|142| +2450829|14248|3|720| +2450829|14251|3|| +2450829|14252|3|422| +2450829|14254|3|521| +2450829|14257|3|900| +2450829|14258|3|694| +2450829|14260|3|645| +2450829|14263|3|124| +2450829|14264|3|532| +2450829|14266|3|57| +2450829|14269|3|894| +2450829|14270|3|| +2450829|14272|3|217| +2450829|14275|3|380| +2450829|14276|3|17| +2450829|14278|3|823| +2450829|14281|3|121| +2450829|14282|3|597| +2450829|14284|3|196| +2450829|14287|3|961| +2450829|14288|3|640| +2450829|14290|3|243| +2450829|14293|3|702| +2450829|14294|3|890| +2450829|14296|3|| +2450829|14299|3|471| +2450829|14300|3|38| +2450829|14302|3|746| +2450829|14305|3|341| +2450829|14306|3|310| +2450829|14308|3|261| +2450829|14311|3|569| +2450829|14312|3|560| +2450829|14314|3|811| +2450829|14317|3|635| +2450829|14318|3|47| +2450829|14320|3|221| +2450829|14323|3|421| +2450829|14324|3|165| +2450829|14326|3|606| +2450829|14329|3|523| +2450829|14330|3|833| +2450829|14332|3|791| +2450829|14335|3|236| +2450829|14336|3|| +2450829|14338|3|770| +2450829|14341|3|| +2450829|14342|3|| +2450829|14344|3|99| +2450829|14347|3|516| +2450829|14348|3|510| +2450829|14350|3|476| +2450829|14353|3|878| +2450829|14354|3|158| +2450829|14356|3|414| +2450829|14359|3|229| +2450829|14360|3|748| +2450829|14362|3|791| +2450829|14365|3|| +2450829|14366|3|957| +2450829|14368|3|235| +2450829|14371|3|508| +2450829|14372|3|993| +2450829|14374|3|213| +2450829|14377|3|841| +2450829|14378|3|393| +2450829|14380|3|816| +2450829|14383|3|330| +2450829|14384|3|539| +2450829|14386|3|498| +2450829|14389|3|254| +2450829|14390|3|382| +2450829|14392|3|874| +2450829|14395|3|566| +2450829|14396|3|52| +2450829|14398|3|556| +2450829|14401|3|113| +2450829|14402|3|106| +2450829|14404|3|758| +2450829|14407|3|528| +2450829|14408|3|887| +2450829|14410|3|586| +2450829|14413|3|952| +2450829|14414|3|791| +2450829|14416|3|153| +2450829|14419|3|131| +2450829|14420|3|280| +2450829|14422|3|34| +2450829|14425|3|451| +2450829|14426|3|780| +2450829|14428|3|405| +2450829|14431|3|93| +2450829|14432|3|564| +2450829|14434|3|758| +2450829|14437|3|410| +2450829|14438|3|885| +2450829|14440|3|859| +2450829|14443|3|947| +2450829|14444|3|120| +2450829|14446|3|509| +2450829|14449|3|275| +2450829|14450|3|413| +2450829|14452|3|388| +2450829|14455|3|299| +2450829|14456|3|583| +2450829|14458|3|580| +2450829|14461|3|227| +2450829|14462|3|363| +2450829|14464|3|987| +2450829|14467|3|236| +2450829|14468|3|70| +2450829|14470|3|250| +2450829|14473|3|812| +2450829|14474|3|933| +2450829|14476|3|174| +2450829|14479|3|208| +2450829|14480|3|| +2450829|14482|3|516| +2450829|14485|3|102| +2450829|14486|3|165| +2450829|14488|3|640| +2450829|14491|3|893| +2450829|14492|3|452| +2450829|14494|3|| +2450829|14497|3|729| +2450829|14498|3|652| +2450829|14500|3|198| +2450829|14503|3|699| +2450829|14504|3|411| +2450829|14506|3|414| +2450829|14509|3|607| +2450829|14510|3|56| +2450829|14512|3|904| +2450829|14515|3|17| +2450829|14516|3|688| +2450829|14518|3|420| +2450829|14521|3|351| +2450829|14522|3|618| +2450829|14524|3|371| +2450829|14527|3|440| +2450829|14528|3|200| +2450829|14530|3|674| +2450829|14533|3|526| +2450829|14534|3|99| +2450829|14536|3|65| +2450829|14539|3|106| +2450829|14540|3|925| +2450829|14542|3|850| +2450829|14545|3|487| +2450829|14546|3|904| +2450829|14548|3|443| +2450829|14551|3|733| +2450829|14552|3|306| +2450829|14554|3|642| +2450829|14557|3|183| +2450829|14558|3|221| +2450829|14560|3|| +2450829|14563|3|329| +2450829|14564|3|458| +2450829|14566|3|38| +2450829|14569|3|941| +2450829|14570|3|231| +2450829|14572|3|209| +2450829|14575|3|288| +2450829|14576|3|246| +2450829|14578|3|393| +2450829|14581|3|943| +2450829|14582|3|262| +2450829|14584|3|687| +2450829|14587|3|463| +2450829|14588|3|358| +2450829|14590|3|228| +2450829|14593|3|288| +2450829|14594|3|| +2450829|14596|3|809| +2450829|14599|3|225| +2450829|14600|3|424| +2450829|14602|3|389| +2450829|14605|3|730| +2450829|14606|3|| +2450829|14608|3|850| +2450829|14611|3|668| +2450829|14612|3|296| +2450829|14614|3|320| +2450829|14617|3|| +2450829|14618|3|301| +2450829|14620|3|358| +2450829|14623|3|178| +2450829|14624|3|129| +2450829|14626|3|480| +2450829|14629|3|155| +2450829|14630|3|284| +2450829|14632|3|481| +2450829|14635|3|104| +2450829|14636|3|902| +2450829|14638|3|| +2450829|14641|3|321| +2450829|14642|3|| +2450829|14644|3|733| +2450829|14647|3|655| +2450829|14648|3|507| +2450829|14650|3|509| +2450829|14653|3|446| +2450829|14654|3|690| +2450829|14656|3|690| +2450829|14659|3|360| +2450829|14660|3|299| +2450829|14662|3|470| +2450829|14665|3|182| +2450829|14666|3|| +2450829|14668|3|328| +2450829|14671|3|840| +2450829|14672|3|45| +2450829|14674|3|805| +2450829|14677|3|441| +2450829|14678|3|399| +2450829|14680|3|656| +2450829|14683|3|951| +2450829|14684|3|| +2450829|14686|3|344| +2450829|14689|3|229| +2450829|14690|3|799| +2450829|14692|3|950| +2450829|14695|3|466| +2450829|14696|3|844| +2450829|14698|3|262| +2450829|14701|3|399| +2450829|14702|3|784| +2450829|14704|3|933| +2450829|14707|3|101| +2450829|14708|3|| +2450829|14710|3|530| +2450829|14713|3|79| +2450829|14714|3|568| +2450829|14716|3|194| +2450829|14719|3|671| +2450829|14720|3|172| +2450829|14722|3|968| +2450829|14725|3|969| +2450829|14726|3|94| +2450829|14728|3|886| +2450829|14731|3|50| +2450829|14732|3|309| +2450829|14734|3|43| +2450829|14737|3|320| +2450829|14738|3|48| +2450829|14740|3|634| +2450829|14743|3|| +2450829|14744|3|340| +2450829|14746|3|578| +2450829|14749|3|593| +2450829|14750|3|200| +2450829|14752|3|560| +2450829|14755|3|328| +2450829|14756|3|571| +2450829|14758|3|114| +2450829|14761|3|298| +2450829|14762|3|993| +2450829|14764|3|853| +2450829|14767|3|820| +2450829|14768|3|645| +2450829|14770|3|388| +2450829|14773|3|711| +2450829|14774|3|259| +2450829|14776|3|389| +2450829|14779|3|170| +2450829|14780|3|730| +2450829|14782|3|302| +2450829|14785|3|354| +2450829|14786|3|| +2450829|14788|3|55| +2450829|14791|3|455| +2450829|14792|3|461| +2450829|14794|3|61| +2450829|14797|3|308| +2450829|14798|3|59| +2450829|14800|3|741| +2450829|14803|3|139| +2450829|14804|3|171| +2450829|14806|3|734| +2450829|14809|3|327| +2450829|14810|3|140| +2450829|14812|3|| +2450829|14815|3|267| +2450829|14816|3|73| +2450829|14818|3|726| +2450829|14821|3|822| +2450829|14822|3|| +2450829|14824|3|998| +2450829|14827|3|874| +2450829|14828|3|341| +2450829|14830|3|890| +2450829|14833|3|658| +2450829|14834|3|907| +2450829|14836|3|102| +2450829|14839|3|972| +2450829|14840|3|907| +2450829|14842|3|731| +2450829|14845|3|294| +2450829|14846|3|582| +2450829|14848|3|877| +2450829|14851|3|748| +2450829|14852|3|389| +2450829|14854|3|301| +2450829|14857|3|515| +2450829|14858|3|962| +2450829|14860|3|616| +2450829|14863|3|938| +2450829|14864|3|998| +2450829|14866|3|740| +2450829|14869|3|274| +2450829|14870|3|545| +2450829|14872|3|626| +2450829|14875|3|646| +2450829|14876|3|444| +2450829|14878|3|176| +2450829|14881|3|228| +2450829|14882|3|448| +2450829|14884|3|717| +2450829|14887|3|647| +2450829|14888|3|678| +2450829|14890|3|175| +2450829|14893|3|522| +2450829|14894|3|538| +2450829|14896|3|647| +2450829|14899|3|443| +2450829|14900|3|609| +2450829|14902|3|888| +2450829|14905|3|152| +2450829|14906|3|| +2450829|14908|3|68| +2450829|14911|3|349| +2450829|14912|3|75| +2450829|14914|3|729| +2450829|14917|3|275| +2450829|14918|3|354| +2450829|14920|3|231| +2450829|14923|3|501| +2450829|14924|3|3| +2450829|14926|3|943| +2450829|14929|3|98| +2450829|14930|3|742| +2450829|14932|3|608| +2450829|14935|3|454| +2450829|14936|3|347| +2450829|14938|3|966| +2450829|14941|3|275| +2450829|14942|3|280| +2450829|14944|3|771| +2450829|14947|3|443| +2450829|14948|3|712| +2450829|14950|3|| +2450829|14953|3|863| +2450829|14954|3|661| +2450829|14956|3|797| +2450829|14959|3|935| +2450829|14960|3|344| +2450829|14962|3|909| +2450829|14965|3|148| +2450829|14966|3|786| +2450829|14968|3|576| +2450829|14971|3|792| +2450829|14972|3|828| +2450829|14974|3|| +2450829|14977|3|| +2450829|14978|3|990| +2450829|14980|3|920| +2450829|14983|3|709| +2450829|14984|3|508| +2450829|14986|3|819| +2450829|14989|3|571| +2450829|14990|3|72| +2450829|14992|3|632| +2450829|14995|3|784| +2450829|14996|3|703| +2450829|14998|3|258| +2450829|15001|3|974| +2450829|15002|3|256| +2450829|15004|3|920| +2450829|15007|3|548| +2450829|15008|3|536| +2450829|15010|3|401| +2450829|15013|3|341| +2450829|15014|3|970| +2450829|15016|3|564| +2450829|15019|3|554| +2450829|15020|3|167| +2450829|15022|3|894| +2450829|15025|3|718| +2450829|15026|3|374| +2450829|15028|3|812| +2450829|15031|3|790| +2450829|15032|3|798| +2450829|15034|3|630| +2450829|15037|3|22| +2450829|15038|3|715| +2450829|15040|3|193| +2450829|15043|3|451| +2450829|15044|3|664| +2450829|15046|3|789| +2450829|15049|3|427| +2450829|15050|3|295| +2450829|15052|3|739| +2450829|15055|3|830| +2450829|15056|3|480| +2450829|15058|3|710| +2450829|15061|3|499| +2450829|15062|3|411| +2450829|15064|3|| +2450829|15067|3|792| +2450829|15068|3|74| +2450829|15070|3|498| +2450829|15073|3|910| +2450829|15074|3|978| +2450829|15076|3|871| +2450829|15079|3|662| +2450829|15080|3|82| +2450829|15082|3|| +2450829|15085|3|| +2450829|15086|3|91| +2450829|15088|3|| +2450829|15091|3|526| +2450829|15092|3|996| +2450829|15094|3|408| +2450829|15097|3|331| +2450829|15098|3|727| +2450829|15100|3|35| +2450829|15103|3|556| +2450829|15104|3|257| +2450829|15106|3|903| +2450829|15109|3|198| +2450829|15110|3|9| +2450829|15112|3|79| +2450829|15115|3|603| +2450829|15116|3|| +2450829|15118|3|800| +2450829|15121|3|888| +2450829|15122|3|934| +2450829|15124|3|211| +2450829|15127|3|758| +2450829|15128|3|376| +2450829|15130|3|863| +2450829|15133|3|83| +2450829|15134|3|114| +2450829|15136|3|968| +2450829|15139|3|946| +2450829|15140|3|190| +2450829|15142|3|809| +2450829|15145|3|183| +2450829|15146|3|985| +2450829|15148|3|225| +2450829|15151|3|68| +2450829|15152|3|516| +2450829|15154|3|| +2450829|15157|3|626| +2450829|15158|3|583| +2450829|15160|3|918| +2450829|15163|3|| +2450829|15164|3|19| +2450829|15166|3|75| +2450829|15169|3|459| +2450829|15170|3|447| +2450829|15172|3|2| +2450829|15175|3|637| +2450829|15176|3|250| +2450829|15178|3|837| +2450829|15181|3|806| +2450829|15182|3|280| +2450829|15184|3|955| +2450829|15187|3|621| +2450829|15188|3|745| +2450829|15190|3|570| +2450829|15193|3|405| +2450829|15194|3|854| +2450829|15196|3|297| +2450829|15199|3|273| +2450829|15200|3|596| +2450829|15202|3|10| +2450829|15205|3|325| +2450829|15206|3|549| +2450829|15208|3|| +2450829|15211|3|947| +2450829|15212|3|663| +2450829|15214|3|909| +2450829|15217|3|382| +2450829|15218|3|468| +2450829|15220|3|540| +2450829|15223|3|943| +2450829|15224|3|176| +2450829|15226|3|187| +2450829|15229|3|424| +2450829|15230|3|| +2450829|15232|3|271| +2450829|15235|3|377| +2450829|15236|3|410| +2450829|15238|3|972| +2450829|15241|3|290| +2450829|15242|3|270| +2450829|15244|3|865| +2450829|15247|3|269| +2450829|15248|3|435| +2450829|15250|3|| +2450829|15253|3|744| +2450829|15254|3|326| +2450829|15256|3|654| +2450829|15259|3|682| +2450829|15260|3|690| +2450829|15262|3|| +2450829|15265|3|912| +2450829|15266|3|747| +2450829|15268|3|| +2450829|15271|3|144| +2450829|15272|3|551| +2450829|15274|3|29| +2450829|15277|3|760| +2450829|15278|3|826| +2450829|15280|3|462| +2450829|15283|3|239| +2450829|15284|3|779| +2450829|15286|3|258| +2450829|15289|3|419| +2450829|15290|3|819| +2450829|15292|3|534| +2450829|15295|3|47| +2450829|15296|3|788| +2450829|15298|3|358| +2450829|15301|3|748| +2450829|15302|3|612| +2450829|15304|3|| +2450829|15307|3|643| +2450829|15308|3|331| +2450829|15310|3|166| +2450829|15313|3|202| +2450829|15314|3|44| +2450829|15316|3|834| +2450829|15319|3|352| +2450829|15320|3|766| +2450829|15322|3|292| +2450829|15325|3|174| +2450829|15326|3|112| +2450829|15328|3|336| +2450829|15331|3|417| +2450829|15332|3|192| +2450829|15334|3|310| +2450829|15337|3|170| +2450829|15338|3|946| +2450829|15340|3|141| +2450829|15343|3|| +2450829|15344|3|565| +2450829|15346|3|725| +2450829|15349|3|252| +2450829|15350|3|390| +2450829|15352|3|651| +2450829|15355|3|393| +2450829|15356|3|904| +2450829|15358|3|49| +2450829|15361|3|58| +2450829|15362|3|303| +2450829|15364|3|576| +2450829|15367|3|809| +2450829|15368|3|329| +2450829|15370|3|507| +2450829|15373|3|157| +2450829|15374|3|357| +2450829|15376|3|407| +2450829|15379|3|544| +2450829|15380|3|148| +2450829|15382|3|517| +2450829|15385|3|468| +2450829|15386|3|548| +2450829|15388|3|354| +2450829|15391|3|856| +2450829|15392|3|598| +2450829|15394|3|81| +2450829|15397|3|654| +2450829|15398|3|16| +2450829|15400|3|398| +2450829|15403|3|| +2450829|15404|3|422| +2450829|15406|3|461| +2450829|15409|3|538| +2450829|15410|3|963| +2450829|15412|3|| +2450829|15415|3|94| +2450829|15416|3|751| +2450829|15418|3|377| +2450829|15421|3|677| +2450829|15422|3|786| +2450829|15424|3|986| +2450829|15427|3|249| +2450829|15428|3|| +2450829|15430|3|162| +2450829|15433|3|910| +2450829|15434|3|473| +2450829|15436|3|355| +2450829|15439|3|591| +2450829|15440|3|117| +2450829|15442|3|142| +2450829|15445|3|299| +2450829|15446|3|381| +2450829|15448|3|997| +2450829|15451|3|324| +2450829|15452|3|797| +2450829|15454|3|| +2450829|15457|3|321| +2450829|15458|3|913| +2450829|15460|3|819| +2450829|15463|3|639| +2450829|15464|3|651| +2450829|15466|3|705| +2450829|15469|3|430| +2450829|15470|3|595| +2450829|15472|3|| +2450829|15475|3|| +2450829|15476|3|538| +2450829|15478|3|691| +2450829|15481|3|38| +2450829|15482|3|119| +2450829|15484|3|921| +2450829|15487|3|873| +2450829|15488|3|142| +2450829|15490|3|594| +2450829|15493|3|26| +2450829|15494|3|74| +2450829|15496|3|135| +2450829|15499|3|236| +2450829|15500|3|682| +2450829|15502|3|794| +2450829|15505|3|311| +2450829|15506|3|90| +2450829|15508|3|606| +2450829|15511|3|392| +2450829|15512|3|417| +2450829|15514|3|874| +2450829|15517|3|572| +2450829|15518|3|548| +2450829|15520|3|859| +2450829|15523|3|161| +2450829|15524|3|952| +2450829|15526|3|869| +2450829|15529|3|827| +2450829|15530|3|637| +2450829|15532|3|799| +2450829|15535|3|401| +2450829|15536|3|850| +2450829|15538|3|210| +2450829|15541|3|904| +2450829|15542|3|248| +2450829|15544|3|144| +2450829|15547|3|| +2450829|15548|3|734| +2450829|15550|3|913| +2450829|15553|3|699| +2450829|15554|3|853| +2450829|15556|3|981| +2450829|15559|3|83| +2450829|15560|3|93| +2450829|15562|3|257| +2450829|15565|3|468| +2450829|15566|3|436| +2450829|15568|3|| +2450829|15571|3|763| +2450829|15572|3|950| +2450829|15574|3|544| +2450829|15577|3|254| +2450829|15578|3|531| +2450829|15580|3|450| +2450829|15583|3|388| +2450829|15584|3|373| +2450829|15586|3|854| +2450829|15589|3|311| +2450829|15590|3|972| +2450829|15592|3|665| +2450829|15595|3|457| +2450829|15596|3|598| +2450829|15598|3|927| +2450829|15601|3|998| +2450829|15602|3|710| +2450829|15604|3|689| +2450829|15607|3|143| +2450829|15608|3|965| +2450829|15610|3|413| +2450829|15613|3|504| +2450829|15614|3|236| +2450829|15616|3|726| +2450829|15619|3|| +2450829|15620|3|204| +2450829|15622|3|686| +2450829|15625|3|299| +2450829|15626|3|107| +2450829|15628|3|126| +2450829|15631|3|684| +2450829|15632|3|514| +2450829|15634|3|551| +2450829|15637|3|272| +2450829|15638|3|526| +2450829|15640|3|68| +2450829|15643|3|141| +2450829|15644|3|16| +2450829|15646|3|808| +2450829|15649|3|22| +2450829|15650|3|371| +2450829|15652|3|| +2450829|15655|3|145| +2450829|15656|3|270| +2450829|15658|3|156| +2450829|15661|3|777| +2450829|15662|3|204| +2450829|15664|3|909| +2450829|15667|3|606| +2450829|15668|3|| +2450829|15670|3|791| +2450829|15673|3|519| +2450829|15674|3|27| +2450829|15676|3|596| +2450829|15679|3|257| +2450829|15680|3|817| +2450829|15682|3|881| +2450829|15685|3|234| +2450829|15686|3|723| +2450829|15688|3|390| +2450829|15691|3|717| +2450829|15692|3|679| +2450829|15694|3|525| +2450829|15697|3|544| +2450829|15698|3|262| +2450829|15700|3|925| +2450829|15703|3|969| +2450829|15704|3|69| +2450829|15706|3|456| +2450829|15709|3|326| +2450829|15710|3|465| +2450829|15712|3|188| +2450829|15715|3|694| +2450829|15716|3|273| +2450829|15718|3|305| +2450829|15721|3|443| +2450829|15722|3|340| +2450829|15724|3|712| +2450829|15727|3|940| +2450829|15728|3|238| +2450829|15730|3|67| +2450829|15733|3|119| +2450829|15734|3|665| +2450829|15736|3|798| +2450829|15739|3|| +2450829|15740|3|892| +2450829|15742|3|965| +2450829|15745|3|305| +2450829|15746|3|81| +2450829|15748|3|59| +2450829|15751|3|992| +2450829|15752|3|166| +2450829|15754|3|605| +2450829|15757|3|663| +2450829|15758|3|678| +2450829|15760|3|373| +2450829|15763|3|394| +2450829|15764|3|976| +2450829|15766|3|130| +2450829|15769|3|128| +2450829|15770|3|596| +2450829|15772|3|977| +2450829|15775|3|250| +2450829|15776|3|924| +2450829|15778|3|359| +2450829|15781|3|520| +2450829|15782|3|808| +2450829|15784|3|505| +2450829|15787|3|987| +2450829|15788|3|734| +2450829|15790|3|132| +2450829|15793|3|436| +2450829|15794|3|988| +2450829|15796|3|51| +2450829|15799|3|474| +2450829|15800|3|309| +2450829|15802|3|704| +2450829|15805|3|41| +2450829|15806|3|91| +2450829|15808|3|937| +2450829|15811|3|426| +2450829|15812|3|214| +2450829|15814|3|685| +2450829|15817|3|44| +2450829|15818|3|403| +2450829|15820|3|111| +2450829|15823|3|| +2450829|15824|3|459| +2450829|15826|3|304| +2450829|15829|3|312| +2450829|15830|3|182| +2450829|15832|3|630| +2450829|15835|3|871| +2450829|15836|3|436| +2450829|15838|3|165| +2450829|15841|3|105| +2450829|15842|3|412| +2450829|15844|3|407| +2450829|15847|3|752| +2450829|15848|3|207| +2450829|15850|3|891| +2450829|15853|3|151| +2450829|15854|3|554| +2450829|15856|3|331| +2450829|15859|3|939| +2450829|15860|3|946| +2450829|15862|3|529| +2450829|15865|3|525| +2450829|15866|3|138| +2450829|15868|3|972| +2450829|15871|3|506| +2450829|15872|3|533| +2450829|15874|3|511| +2450829|15877|3|34| +2450829|15878|3|934| +2450829|15880|3|976| +2450829|15883|3|303| +2450829|15884|3|818| +2450829|15886|3|502| +2450829|15889|3|327| +2450829|15890|3|909| +2450829|15892|3|970| +2450829|15895|3|418| +2450829|15896|3|17| +2450829|15898|3|378| +2450829|15901|3|2| +2450829|15902|3|597| +2450829|15904|3|526| +2450829|15907|3|620| +2450829|15908|3|19| +2450829|15910|3|514| +2450829|15913|3|832| +2450829|15914|3|219| +2450829|15916|3|950| +2450829|15919|3|702| +2450829|15920|3|503| +2450829|15922|3|212| +2450829|15925|3|901| +2450829|15926|3|743| +2450829|15928|3|328| +2450829|15931|3|872| +2450829|15932|3|436| +2450829|15934|3|851| +2450829|15937|3|936| +2450829|15938|3|703| +2450829|15940|3|197| +2450829|15943|3|341| +2450829|15944|3|657| +2450829|15946|3|865| +2450829|15949|3|901| +2450829|15950|3|748| +2450829|15952|3|84| +2450829|15955|3|437| +2450829|15956|3|43| +2450829|15958|3|613| +2450829|15961|3|604| +2450829|15962|3|884| +2450829|15964|3|252| +2450829|15967|3|360| +2450829|15968|3|552| +2450829|15970|3|287| +2450829|15973|3|572| +2450829|15974|3|38| +2450829|15976|3|842| +2450829|15979|3|929| +2450829|15980|3|796| +2450829|15982|3|542| +2450829|15985|3|169| +2450829|15986|3|11| +2450829|15988|3|713| +2450829|15991|3|106| +2450829|15992|3|274| +2450829|15994|3|501| +2450829|15997|3|563| +2450829|15998|3|789| +2450829|16000|3|536| +2450829|16003|3|978| +2450829|16004|3|611| +2450829|16006|3|561| +2450829|16009|3|728| +2450829|16010|3|945| +2450829|16012|3|727| +2450829|16015|3|| +2450829|16016|3|703| +2450829|16018|3|597| +2450829|16021|3|155| +2450829|16022|3|125| +2450829|16024|3|841| +2450829|16027|3|652| +2450829|16028|3|4| +2450829|16030|3|248| +2450829|16033|3|404| +2450829|16034|3|909| +2450829|16036|3|343| +2450829|16039|3|| +2450829|16040|3|684| +2450829|16042|3|229| +2450829|16045|3|580| +2450829|16046|3|230| +2450829|16048|3|410| +2450829|16051|3|645| +2450829|16052|3|726| +2450829|16054|3|312| +2450829|16057|3|862| +2450829|16058|3|823| +2450829|16060|3|813| +2450829|16063|3|656| +2450829|16064|3|170| +2450829|16066|3|276| +2450829|16069|3|286| +2450829|16070|3|159| +2450829|16072|3|413| +2450829|16075|3|854| +2450829|16076|3|736| +2450829|16078|3|509| +2450829|16081|3|764| +2450829|16082|3|740| +2450829|16084|3|49| +2450829|16087|3|975| +2450829|16088|3|281| +2450829|16090|3|137| +2450829|16093|3|890| +2450829|16094|3|714| +2450829|16096|3|251| +2450829|16099|3|458| +2450829|16100|3|854| +2450829|16102|3|631| +2450829|16105|3|554| +2450829|16106|3|9| +2450829|16108|3|934| +2450829|16111|3|470| +2450829|16112|3|211| +2450829|16114|3|432| +2450829|16117|3|395| +2450829|16118|3|899| +2450829|16120|3|581| +2450829|16123|3|| +2450829|16124|3|158| +2450829|16126|3|417| +2450829|16129|3|672| +2450829|16130|3|604| +2450829|16132|3|620| +2450829|16135|3|706| +2450829|16136|3|317| +2450829|16138|3|350| +2450829|16141|3|730| +2450829|16142|3|448| +2450829|16144|3|188| +2450829|16147|3|150| +2450829|16148|3|319| +2450829|16150|3|265| +2450829|16153|3|551| +2450829|16154|3|297| +2450829|16156|3|520| +2450829|16159|3|253| +2450829|16160|3|666| +2450829|16162|3|530| +2450829|16165|3|687| +2450829|16166|3|450| +2450829|16168|3|579| +2450829|16171|3|413| +2450829|16172|3|942| +2450829|16174|3|96| +2450829|16177|3|588| +2450829|16178|3|805| +2450829|16180|3|479| +2450829|16183|3|726| +2450829|16184|3|851| +2450829|16186|3|596| +2450829|16189|3|748| +2450829|16190|3|929| +2450829|16192|3|| +2450829|16195|3|257| +2450829|16196|3|175| +2450829|16198|3|105| +2450829|16201|3|732| +2450829|16202|3|276| +2450829|16204|3|938| +2450829|16207|3|772| +2450829|16208|3|| +2450829|16210|3|268| +2450829|16213|3|286| +2450829|16214|3|314| +2450829|16216|3|541| +2450829|16219|3|65| +2450829|16220|3|840| +2450829|16222|3|363| +2450829|16225|3|928| +2450829|16226|3|941| +2450829|16228|3|623| +2450829|16231|3|823| +2450829|16232|3|477| +2450829|16234|3|681| +2450829|16237|3|782| +2450829|16238|3|533| +2450829|16240|3|805| +2450829|16243|3|643| +2450829|16244|3|580| +2450829|16246|3|17| +2450829|16249|3|159| +2450829|16250|3|497| +2450829|16252|3|763| +2450829|16255|3|726| +2450829|16256|3|312| +2450829|16258|3|161| +2450829|16261|3|562| +2450829|16262|3|80| +2450829|16264|3|439| +2450829|16267|3|98| +2450829|16268|3|610| +2450829|16270|3|675| +2450829|16273|3|36| +2450829|16274|3|215| +2450829|16276|3|671| +2450829|16279|3|601| +2450829|16280|3|654| +2450829|16282|3|584| +2450829|16285|3|683| +2450829|16286|3|679| +2450829|16288|3|317| +2450829|16291|3|178| +2450829|16292|3|934| +2450829|16294|3|180| +2450829|16297|3|937| +2450829|16298|3|388| +2450829|16300|3|613| +2450829|16303|3|371| +2450829|16304|3|529| +2450829|16306|3|899| +2450829|16309|3|737| +2450829|16310|3|993| +2450829|16312|3|936| +2450829|16315|3|482| +2450829|16316|3|| +2450829|16318|3|932| +2450829|16321|3|438| +2450829|16322|3|68| +2450829|16324|3|768| +2450829|16327|3|616| +2450829|16328|3|958| +2450829|16330|3|635| +2450829|16333|3|360| +2450829|16334|3|753| +2450829|16336|3|695| +2450829|16339|3|721| +2450829|16340|3|797| +2450829|16342|3|259| +2450829|16345|3|93| +2450829|16346|3|876| +2450829|16348|3|761| +2450829|16351|3|441| +2450829|16352|3|385| +2450829|16354|3|556| +2450829|16357|3|581| +2450829|16358|3|562| +2450829|16360|3|417| +2450829|16363|3|643| +2450829|16364|3|917| +2450829|16366|3|489| +2450829|16369|3|577| +2450829|16370|3|146| +2450829|16372|3|144| +2450829|16375|3|801| +2450829|16376|3|| +2450829|16378|3|80| +2450829|16381|3|426| +2450829|16382|3|397| +2450829|16384|3|680| +2450829|16387|3|623| +2450829|16388|3|928| +2450829|16390|3|243| +2450829|16393|3|940| +2450829|16394|3|547| +2450829|16396|3|203| +2450829|16399|3|296| +2450829|16400|3|201| +2450829|16402|3|755| +2450829|16405|3|183| +2450829|16406|3|861| +2450829|16408|3|990| +2450829|16411|3|514| +2450829|16412|3|546| +2450829|16414|3|230| +2450829|16417|3|520| +2450829|16418|3|714| +2450829|16420|3|225| +2450829|16423|3|359| +2450829|16424|3|672| +2450829|16426|3|642| +2450829|16429|3|762| +2450829|16430|3|533| +2450829|16432|3|967| +2450829|16435|3|414| +2450829|16436|3|391| +2450829|16438|3|| +2450829|16441|3|966| +2450829|16442|3|899| +2450829|16444|3|253| +2450829|16447|3|368| +2450829|16448|3|169| +2450829|16450|3|| +2450829|16453|3|787| +2450829|16454|3|829| +2450829|16456|3|462| +2450829|16459|3|982| +2450829|16460|3|153| +2450829|16462|3|705| +2450829|16465|3|512| +2450829|16466|3|744| +2450829|16468|3|427| +2450829|16471|3|| +2450829|16472|3|693| +2450829|16474|3|| +2450829|16477|3|693| +2450829|16478|3|787| +2450829|16480|3|894| +2450829|16483|3|910| +2450829|16484|3|978| +2450829|16486|3|| +2450829|16489|3|493| +2450829|16490|3|636| +2450829|16492|3|571| +2450829|16495|3|240| +2450829|16496|3|643| +2450829|16498|3|178| +2450829|16501|3|840| +2450829|16502|3|683| +2450829|16504|3|119| +2450829|16507|3|141| +2450829|16508|3|445| +2450829|16510|3|218| +2450829|16513|3|93| +2450829|16514|3|11| +2450829|16516|3|216| +2450829|16519|3|601| +2450829|16520|3|124| +2450829|16522|3|312| +2450829|16525|3|368| +2450829|16526|3|391| +2450829|16528|3|561| +2450829|16531|3|821| +2450829|16532|3|498| +2450829|16534|3|| +2450829|16537|3|936| +2450829|16538|3|509| +2450829|16540|3|63| +2450829|16543|3|816| +2450829|16544|3|381| +2450829|16546|3|906| +2450829|16549|3|461| +2450829|16550|3|823| +2450829|16552|3|359| +2450829|16555|3|526| +2450829|16556|3|985| +2450829|16558|3|685| +2450829|16561|3|1000| +2450829|16562|3|595| +2450829|16564|3|797| +2450829|16567|3|66| +2450829|16568|3|717| +2450829|16570|3|476| +2450829|16573|3|89| +2450829|16574|3|525| +2450829|16576|3|227| +2450829|16579|3|607| +2450829|16580|3|782| +2450829|16582|3|463| +2450829|16585|3|175| +2450829|16586|3|124| +2450829|16588|3|890| +2450829|16591|3|457| +2450829|16592|3|127| +2450829|16594|3|189| +2450829|16597|3|797| +2450829|16598|3|564| +2450829|16600|3|329| +2450829|16603|3|91| +2450829|16604|3|865| +2450829|16606|3|807| +2450829|16609|3|66| +2450829|16610|3|165| +2450829|16612|3|13| +2450829|16615|3|538| +2450829|16616|3|459| +2450829|16618|3|142| +2450829|16621|3|851| +2450829|16622|3|| +2450829|16624|3|469| +2450829|16627|3|642| +2450829|16628|3|174| +2450829|16630|3|973| +2450829|16633|3|110| +2450829|16634|3|818| +2450829|16636|3|746| +2450829|16639|3|534| +2450829|16640|3|848| +2450829|16642|3|336| +2450829|16645|3|82| +2450829|16646|3|58| +2450829|16648|3|955| +2450829|16651|3|274| +2450829|16652|3|545| +2450829|16654|3|47| +2450829|16657|3|689| +2450829|16658|3|823| +2450829|16660|3|| +2450829|16663|3|881| +2450829|16664|3|868| +2450829|16666|3|37| +2450829|16669|3|331| +2450829|16670|3|293| +2450829|16672|3|| +2450829|16675|3|660| +2450829|16676|3|457| +2450829|16678|3|| +2450829|16681|3|740| +2450829|16682|3|917| +2450829|16684|3|668| +2450829|16687|3|533| +2450829|16688|3|418| +2450829|16690|3|386| +2450829|16693|3|36| +2450829|16694|3|234| +2450829|16696|3|459| +2450829|16699|3|876| +2450829|16700|3|841| +2450829|16702|3|933| +2450829|16705|3|476| +2450829|16706|3|869| +2450829|16708|3|287| +2450829|16711|3|740| +2450829|16712|3|150| +2450829|16714|3|575| +2450829|16717|3|344| +2450829|16718|3|465| +2450829|16720|3|746| +2450829|16723|3|608| +2450829|16724|3|| +2450829|16726|3|294| +2450829|16729|3|235| +2450829|16730|3|732| +2450829|16732|3|247| +2450829|16735|3|643| +2450829|16736|3|699| +2450829|16738|3|245| +2450829|16741|3|483| +2450829|16742|3|974| +2450829|16744|3|19| +2450829|16747|3|874| +2450829|16748|3|29| +2450829|16750|3|706| +2450829|16753|3|311| +2450829|16754|3|200| +2450829|16756|3|508| +2450829|16759|3|640| +2450829|16760|3|813| +2450829|16762|3|397| +2450829|16765|3|213| +2450829|16766|3|| +2450829|16768|3|250| +2450829|16771|3|451| +2450829|16772|3|569| +2450829|16774|3|365| +2450829|16777|3|731| +2450829|16778|3|| +2450829|16780|3|936| +2450829|16783|3|567| +2450829|16784|3|876| +2450829|16786|3|953| +2450829|16789|3|162| +2450829|16790|3|615| +2450829|16792|3|851| +2450829|16795|3|289| +2450829|16796|3|846| +2450829|16798|3|238| +2450829|16801|3|584| +2450829|16802|3|599| +2450829|16804|3|963| +2450829|16807|3|178| +2450829|16808|3|204| +2450829|16810|3|575| +2450829|16813|3|522| +2450829|16814|3|786| +2450829|16816|3|186| +2450829|16819|3|| +2450829|16820|3|67| +2450829|16822|3|13| +2450829|16825|3|409| +2450829|16826|3|129| +2450829|16828|3|| +2450829|16831|3|785| +2450829|16832|3|547| +2450829|16834|3|574| +2450829|16837|3|860| +2450829|16838|3|| +2450829|16840|3|480| +2450829|16843|3|169| +2450829|16844|3|313| +2450829|16846|3|650| +2450829|16849|3|888| +2450829|16850|3|40| +2450829|16852|3|359| +2450829|16855|3|882| +2450829|16856|3|966| +2450829|16858|3|350| +2450829|16861|3|228| +2450829|16862|3|155| +2450829|16864|3|299| +2450829|16867|3|| +2450829|16868|3|162| +2450829|16870|3|960| +2450829|16873|3|| +2450829|16874|3|236| +2450829|16876|3|603| +2450829|16879|3|174| +2450829|16880|3|716| +2450829|16882|3|50| +2450829|16885|3|683| +2450829|16886|3|959| +2450829|16888|3|408| +2450829|16891|3|748| +2450829|16892|3|1| +2450829|16894|3|153| +2450829|16897|3|660| +2450829|16898|3|60| +2450829|16900|3|| +2450829|16903|3|478| +2450829|16904|3|239| +2450829|16906|3|722| +2450829|16909|3|936| +2450829|16910|3|996| +2450829|16912|3|397| +2450829|16915|3|889| +2450829|16916|3|942| +2450829|16918|3|533| +2450829|16921|3|358| +2450829|16922|3|540| +2450829|16924|3|629| +2450829|16927|3|449| +2450829|16928|3|| +2450829|16930|3|968| +2450829|16933|3|18| +2450829|16934|3|988| +2450829|16936|3|| +2450829|16939|3|440| +2450829|16940|3|732| +2450829|16942|3|799| +2450829|16945|3|169| +2450829|16946|3|688| +2450829|16948|3|962| +2450829|16951|3|519| +2450829|16952|3|521| +2450829|16954|3|584| +2450829|16957|3|138| +2450829|16958|3|576| +2450829|16960|3|713| +2450829|16963|3|956| +2450829|16964|3|696| +2450829|16966|3|706| +2450829|16969|3|317| +2450829|16970|3|752| +2450829|16972|3|339| +2450829|16975|3|341| +2450829|16976|3|151| +2450829|16978|3|| +2450829|16981|3|641| +2450829|16982|3|271| +2450829|16984|3|414| +2450829|16987|3|68| +2450829|16988|3|55| +2450829|16990|3|907| +2450829|16993|3|257| +2450829|16994|3|88| +2450829|16996|3|813| +2450829|16999|3|472| +2450829|17000|3|649| +2450829|17002|3|378| +2450829|17005|3|863| +2450829|17006|3|8| +2450829|17008|3|| +2450829|17011|3|718| +2450829|17012|3|19| +2450829|17014|3|809| +2450829|17017|3|645| +2450829|17018|3|470| +2450829|17020|3|673| +2450829|17023|3|379| +2450829|17024|3|534| +2450829|17026|3|273| +2450829|17029|3|0| +2450829|17030|3|511| +2450829|17032|3|814| +2450829|17035|3|927| +2450829|17036|3|240| +2450829|17038|3|596| +2450829|17041|3|206| +2450829|17042|3|435| +2450829|17044|3|495| +2450829|17047|3|84| +2450829|17048|3|19| +2450829|17050|3|293| +2450829|17053|3|384| +2450829|17054|3|733| +2450829|17056|3|965| +2450829|17059|3|688| +2450829|17060|3|360| +2450829|17062|3|412| +2450829|17065|3|| +2450829|17066|3|529| +2450829|17068|3|563| +2450829|17071|3|384| +2450829|17072|3|714| +2450829|17074|3|102| +2450829|17077|3|| +2450829|17078|3|749| +2450829|17080|3|200| +2450829|17083|3|412| +2450829|17084|3|530| +2450829|17086|3|788| +2450829|17089|3|186| +2450829|17090|3|819| +2450829|17092|3|586| +2450829|17095|3|755| +2450829|17096|3|180| +2450829|17098|3|14| +2450829|17101|3|666| +2450829|17102|3|378| +2450829|17104|3|| +2450829|17107|3|23| +2450829|17108|3|| +2450829|17110|3|60| +2450829|17113|3|324| +2450829|17114|3|29| +2450829|17116|3|484| +2450829|17119|3|68| +2450829|17120|3|448| +2450829|17122|3|429| +2450829|17125|3|49| +2450829|17126|3|376| +2450829|17128|3|350| +2450829|17131|3|341| +2450829|17132|3|| +2450829|17134|3|454| +2450829|17137|3|900| +2450829|17138|3|977| +2450829|17140|3|381| +2450829|17143|3|412| +2450829|17144|3|858| +2450829|17146|3|403| +2450829|17149|3|176| +2450829|17150|3|580| +2450829|17152|3|612| +2450829|17155|3|824| +2450829|17156|3|989| +2450829|17158|3|955| +2450829|17161|3|956| +2450829|17162|3|403| +2450829|17164|3|638| +2450829|17167|3|469| +2450829|17168|3|432| +2450829|17170|3|307| +2450829|17173|3|531| +2450829|17174|3|579| +2450829|17176|3|111| +2450829|17179|3|946| +2450829|17180|3|59| +2450829|17182|3|155| +2450829|17185|3|505| +2450829|17186|3|730| +2450829|17188|3|749| +2450829|17191|3|597| +2450829|17192|3|849| +2450829|17194|3|289| +2450829|17197|3|234| +2450829|17198|3|249| +2450829|17200|3|658| +2450829|17203|3|600| +2450829|17204|3|434| +2450829|17206|3|96| +2450829|17209|3|498| +2450829|17210|3|531| +2450829|17212|3|618| +2450829|17215|3|417| +2450829|17216|3|156| +2450829|17218|3|154| +2450829|17221|3|| +2450829|17222|3|957| +2450829|17224|3|408| +2450829|17227|3|905| +2450829|17228|3|269| +2450829|17230|3|55| +2450829|17233|3|381| +2450829|17234|3|468| +2450829|17236|3|95| +2450829|17239|3|190| +2450829|17240|3|750| +2450829|17242|3|801| +2450829|17245|3|242| +2450829|17246|3|721| +2450829|17248|3|| +2450829|17251|3|116| +2450829|17252|3|894| +2450829|17254|3|582| +2450829|17257|3|496| +2450829|17258|3|821| +2450829|17260|3|329| +2450829|17263|3|593| +2450829|17264|3|396| +2450829|17266|3|161| +2450829|17269|3|644| +2450829|17270|3|586| +2450829|17272|3|| +2450829|17275|3|651| +2450829|17276|3|601| +2450829|17278|3|771| +2450829|17281|3|669| +2450829|17282|3|255| +2450829|17284|3|130| +2450829|17287|3|492| +2450829|17288|3|624| +2450829|17290|3|329| +2450829|17293|3|671| +2450829|17294|3|725| +2450829|17296|3|13| +2450829|17299|3|221| +2450829|17300|3|175| +2450829|17302|3|685| +2450829|17305|3|142| +2450829|17306|3|807| +2450829|17308|3|732| +2450829|17311|3|58| +2450829|17312|3|796| +2450829|17314|3|149| +2450829|17317|3|720| +2450829|17318|3|359| +2450829|17320|3|343| +2450829|17323|3|445| +2450829|17324|3|956| +2450829|17326|3|511| +2450829|17329|3|287| +2450829|17330|3|132| +2450829|17332|3|264| +2450829|17335|3|393| +2450829|17336|3|| +2450829|17338|3|784| +2450829|17341|3|111| +2450829|17342|3|233| +2450829|17344|3|694| +2450829|17347|3|944| +2450829|17348|3|690| +2450829|17350|3|701| +2450829|17353|3|406| +2450829|17354|3|540| +2450829|17356|3|900| +2450829|17359|3|417| +2450829|17360|3|246| +2450829|17362|3|286| +2450829|17365|3|942| +2450829|17366|3|526| +2450829|17368|3|171| +2450829|17371|3|520| +2450829|17372|3|606| +2450829|17374|3|814| +2450829|17377|3|786| +2450829|17378|3|474| +2450829|17380|3|| +2450829|17383|3|258| +2450829|17384|3|410| +2450829|17386|3|81| +2450829|17389|3|| +2450829|17390|3|616| +2450829|17392|3|302| +2450829|17395|3|324| +2450829|17396|3|669| +2450829|17398|3|208| +2450829|17401|3|513| +2450829|17402|3|161| +2450829|17404|3|586| +2450829|17407|3|773| +2450829|17408|3|444| +2450829|17410|3|174| +2450829|17413|3|843| +2450829|17414|3|669| +2450829|17416|3|22| +2450829|17419|3|99| +2450829|17420|3|437| +2450829|17422|3|504| +2450829|17425|3|971| +2450829|17426|3|188| +2450829|17428|3|80| +2450829|17431|3|480| +2450829|17432|3|2| +2450829|17434|3|33| +2450829|17437|3|520| +2450829|17438|3|493| +2450829|17440|3|33| +2450829|17443|3|378| +2450829|17444|3|134| +2450829|17446|3|439| +2450829|17449|3|805| +2450829|17450|3|893| +2450829|17452|3|803| +2450829|17455|3|57| +2450829|17456|3|563| +2450829|17458|3|145| +2450829|17461|3|935| +2450829|17462|3|994| +2450829|17464|3|489| +2450829|17467|3|578| +2450829|17468|3|486| +2450829|17470|3|892| +2450829|17473|3|506| +2450829|17474|3|195| +2450829|17476|3|911| +2450829|17479|3|646| +2450829|17480|3|| +2450829|17482|3|320| +2450829|17485|3|871| +2450829|17486|3|69| +2450829|17488|3|968| +2450829|17491|3|326| +2450829|17492|3|787| +2450829|17494|3|940| +2450829|17497|3|326| +2450829|17498|3|465| +2450829|17500|3|252| +2450829|17503|3|988| +2450829|17504|3|802| +2450829|17506|3|542| +2450829|17509|3|49| +2450829|17510|3|927| +2450829|17512|3|| +2450829|17515|3|600| +2450829|17516|3|746| +2450829|17518|3|210| +2450829|17521|3|394| +2450829|17522|3|170| +2450829|17524|3|147| +2450829|17527|3|103| +2450829|17528|3|| +2450829|17530|3|381| +2450829|17533|3|992| +2450829|17534|3|361| +2450829|17536|3|225| +2450829|17539|3|332| +2450829|17540|3|573| +2450829|17542|3|792| +2450829|17545|3|94| +2450829|17546|3|61| +2450829|17548|3|43| +2450829|17551|3|422| +2450829|17552|3|66| +2450829|17554|3|791| +2450829|17557|3|332| +2450829|17558|3|728| +2450829|17560|3|465| +2450829|17563|3|310| +2450829|17564|3|592| +2450829|17566|3|275| +2450829|17569|3|616| +2450829|17570|3|360| +2450829|17572|3|140| +2450829|17575|3|623| +2450829|17576|3|740| +2450829|17578|3|14| +2450829|17581|3|| +2450829|17582|3|993| +2450829|17584|3|318| +2450829|17587|3|149| +2450829|17588|3|| +2450829|17590|3|873| +2450829|17593|3|407| +2450829|17594|3|914| +2450829|17596|3|442| +2450829|17599|3|986| +2450829|17600|3|475| +2450829|17602|3|404| +2450829|17605|3|260| +2450829|17606|3|222| +2450829|17608|3|639| +2450829|17611|3|217| +2450829|17612|3|43| +2450829|17614|3|0| +2450829|17617|3|965| +2450829|17618|3|438| +2450829|17620|3|428| +2450829|17623|3|738| +2450829|17624|3|129| +2450829|17626|3|587| +2450829|17629|3|944| +2450829|17630|3|12| +2450829|17632|3|413| +2450829|17635|3|188| +2450829|17636|3|227| +2450829|17638|3|720| +2450829|17641|3|779| +2450829|17642|3|| +2450829|17644|3|8| +2450829|17647|3|| +2450829|17648|3|847| +2450829|17650|3|524| +2450829|17653|3|69| +2450829|17654|3|390| +2450829|17656|3|158| +2450829|17659|3|925| +2450829|17660|3|552| +2450829|17662|3|874| +2450829|17665|3|171| +2450829|17666|3|427| +2450829|17668|3|720| +2450829|17671|3|135| +2450829|17672|3|101| +2450829|17674|3|675| +2450829|17677|3|399| +2450829|17678|3|824| +2450829|17680|3|592| +2450829|17683|3|944| +2450829|17684|3|180| +2450829|17686|3|472| +2450829|17689|3|790| +2450829|17690|3|| +2450829|17692|3|320| +2450829|17695|3|559| +2450829|17696|3|409| +2450829|17698|3|130| +2450829|17701|3|176| +2450829|17702|3|6| +2450829|17704|3|602| +2450829|17707|3|403| +2450829|17708|3|| +2450829|17710|3|164| +2450829|17713|3|467| +2450829|17714|3|919| +2450829|17716|3|228| +2450829|17719|3|51| +2450829|17720|3|383| +2450829|17722|3|122| +2450829|17725|3|16| +2450829|17726|3|9| +2450829|17728|3|209| +2450829|17731|3|11| +2450829|17732|3|666| +2450829|17734|3|36| +2450829|17737|3|844| +2450829|17738|3|87| +2450829|17740|3|345| +2450829|17743|3|369| +2450829|17744|3|630| +2450829|17746|3|109| +2450829|17749|3|99| +2450829|17750|3|371| +2450829|17752|3|57| +2450829|17755|3|956| +2450829|17756|3|202| +2450829|17758|3|89| +2450829|17761|3|387| +2450829|17762|3|8| +2450829|17764|3|241| +2450829|17767|3|824| +2450829|17768|3|215| +2450829|17770|3|509| +2450829|17773|3|| +2450829|17774|3|195| +2450829|17776|3|494| +2450829|17779|3|| +2450829|17780|3|444| +2450829|17782|3|575| +2450829|17785|3|707| +2450829|17786|3|568| +2450829|17788|3|| +2450829|17791|3|| +2450829|17792|3|4| +2450829|17794|3|908| +2450829|17797|3|356| +2450829|17798|3|786| +2450829|17800|3|984| +2450829|17803|3|19| +2450829|17804|3|196| +2450829|17806|3|583| +2450829|17809|3|432| +2450829|17810|3|5| +2450829|17812|3|285| +2450829|17815|3|| +2450829|17816|3|814| +2450829|17818|3|| +2450829|17821|3|11| +2450829|17822|3|355| +2450829|17824|3|942| +2450829|17827|3|660| +2450829|17828|3|406| +2450829|17830|3|224| +2450829|17833|3|767| +2450829|17834|3|504| +2450829|17836|3|379| +2450829|17839|3|80| +2450829|17840|3|549| +2450829|17842|3|967| +2450829|17845|3|195| +2450829|17846|3|844| +2450829|17848|3|347| +2450829|17851|3|770| +2450829|17852|3|288| +2450829|17854|3|735| +2450829|17857|3|886| +2450829|17858|3|344| +2450829|17860|3|909| +2450829|17863|3|659| +2450829|17864|3|359| +2450829|17866|3|99| +2450829|17869|3|636| +2450829|17870|3|978| +2450829|17872|3|72| +2450829|17875|3|378| +2450829|17876|3|931| +2450829|17878|3|634| +2450829|17881|3|689| +2450829|17882|3|275| +2450829|17884|3|289| +2450829|17887|3|35| +2450829|17888|3|| +2450829|17890|3|| +2450829|17893|3|648| +2450829|17894|3|396| +2450829|17896|3|283| +2450829|17899|3|363| +2450829|17900|3|393| +2450829|17902|3|379| +2450829|17905|3|578| +2450829|17906|3|814| +2450829|17908|3|41| +2450829|17911|3|449| +2450829|17912|3|104| +2450829|17914|3|714| +2450829|17917|3|715| +2450829|17918|3|871| +2450829|17920|3|675| +2450829|17923|3|976| +2450829|17924|3|| +2450829|17926|3|360| +2450829|17929|3|970| +2450829|17930|3|673| +2450829|17932|3|703| +2450829|17935|3|731| +2450829|17936|3|767| +2450829|17938|3|| +2450829|17941|3|753| +2450829|17942|3|498| +2450829|17944|3|18| +2450829|17947|3|197| +2450829|17948|3|247| +2450829|17950|3|472| +2450829|17953|3|146| +2450829|17954|3|446| +2450829|17956|3|167| +2450829|17959|3|252| +2450829|17960|3|413| +2450829|17962|3|691| +2450829|17965|3|552| +2450829|17966|3|482| +2450829|17968|3|36| +2450829|17971|3|902| +2450829|17972|3|658| +2450829|17974|3|316| +2450829|17977|3|784| +2450829|17978|3|715| +2450829|17980|3|26| +2450829|17983|3|218| +2450829|17984|3|387| +2450829|17986|3|46| +2450829|17989|3|811| +2450829|17990|3|583| +2450829|17992|3|273| +2450829|17995|3|824| +2450829|17996|3|360| +2450829|17998|3|| +2450829|1|4|259| +2450829|2|4|780| +2450829|4|4|92| +2450829|7|4|275| +2450829|8|4|722| +2450829|10|4|934| +2450829|13|4|425| +2450829|14|4|| +2450829|16|4|457| +2450829|19|4|599| +2450829|20|4|542| +2450829|22|4|104| +2450829|25|4|750| +2450829|26|4|889| +2450829|28|4|| +2450829|31|4|359| +2450829|32|4|926| +2450829|34|4|839| +2450829|37|4|829| +2450829|38|4|619| +2450829|40|4|364| +2450829|43|4|995| +2450829|44|4|2| +2450829|46|4|469| +2450829|49|4|480| +2450829|50|4|802| +2450829|52|4|884| +2450829|55|4|| +2450829|56|4|731| +2450829|58|4|492| +2450829|61|4|1000| +2450829|62|4|855| +2450829|64|4|401| +2450829|67|4|146| +2450829|68|4|170| +2450829|70|4|655| +2450829|73|4|262| +2450829|74|4|297| +2450829|76|4|364| +2450829|79|4|484| +2450829|80|4|141| +2450829|82|4|34| +2450829|85|4|163| +2450829|86|4|355| +2450829|88|4|920| +2450829|91|4|435| +2450829|92|4|719| +2450829|94|4|791| +2450829|97|4|| +2450829|98|4|929| +2450829|100|4|411| +2450829|103|4|758| +2450829|104|4|66| +2450829|106|4|| +2450829|109|4|59| +2450829|110|4|623| +2450829|112|4|861| +2450829|115|4|866| +2450829|116|4|268| +2450829|118|4|756| +2450829|121|4|282| +2450829|122|4|576| +2450829|124|4|858| +2450829|127|4|225| +2450829|128|4|53| +2450829|130|4|274| +2450829|133|4|948| +2450829|134|4|633| +2450829|136|4|909| +2450829|139|4|693| +2450829|140|4|239| +2450829|142|4|856| +2450829|145|4|904| +2450829|146|4|905| +2450829|148|4|645| +2450829|151|4|882| +2450829|152|4|504| +2450829|154|4|670| +2450829|157|4|470| +2450829|158|4|116| +2450829|160|4|382| +2450829|163|4|280| +2450829|164|4|796| +2450829|166|4|600| +2450829|169|4|495| +2450829|170|4|959| +2450829|172|4|433| +2450829|175|4|480| +2450829|176|4|68| +2450829|178|4|583| +2450829|181|4|| +2450829|182|4|619| +2450829|184|4|100| +2450829|187|4|249| +2450829|188|4|453| +2450829|190|4|| +2450829|193|4|807| +2450829|194|4|989| +2450829|196|4|381| +2450829|199|4|370| +2450829|200|4|753| +2450829|202|4|722| +2450829|205|4|907| +2450829|206|4|867| +2450829|208|4|605| +2450829|211|4|665| +2450829|212|4|| +2450829|214|4|807| +2450829|217|4|71| +2450829|218|4|236| +2450829|220|4|441| +2450829|223|4|217| +2450829|224|4|988| +2450829|226|4|221| +2450829|229|4|594| +2450829|230|4|346| +2450829|232|4|| +2450829|235|4|203| +2450829|236|4|569| +2450829|238|4|329| +2450829|241|4|261| +2450829|242|4|660| +2450829|244|4|730| +2450829|247|4|168| +2450829|248|4|61| +2450829|250|4|33| +2450829|253|4|962| +2450829|254|4|401| +2450829|256|4|927| +2450829|259|4|867| +2450829|260|4|136| +2450829|262|4|355| +2450829|265|4|774| +2450829|266|4|633| +2450829|268|4|923| +2450829|271|4|947| +2450829|272|4|947| +2450829|274|4|844| +2450829|277|4|53| +2450829|278|4|668| +2450829|280|4|396| +2450829|283|4|171| +2450829|284|4|438| +2450829|286|4|775| +2450829|289|4|531| +2450829|290|4|612| +2450829|292|4|240| +2450829|295|4|513| +2450829|296|4|885| +2450829|298|4|220| +2450829|301|4|144| +2450829|302|4|736| +2450829|304|4|602| +2450829|307|4|9| +2450829|308|4|760| +2450829|310|4|| +2450829|313|4|768| +2450829|314|4|550| +2450829|316|4|918| +2450829|319|4|533| +2450829|320|4|34| +2450829|322|4|42| +2450829|325|4|204| +2450829|326|4|| +2450829|328|4|183| +2450829|331|4|705| +2450829|332|4|732| +2450829|334|4|527| +2450829|337|4|307| +2450829|338|4|320| +2450829|340|4|373| +2450829|343|4|472| +2450829|344|4|173| +2450829|346|4|269| +2450829|349|4|813| +2450829|350|4|842| +2450829|352|4|958| +2450829|355|4|684| +2450829|356|4|411| +2450829|358|4|786| +2450829|361|4|464| +2450829|362|4|871| +2450829|364|4|459| +2450829|367|4|847| +2450829|368|4|418| +2450829|370|4|798| +2450829|373|4|182| +2450829|374|4|755| +2450829|376|4|734| +2450829|379|4|978| +2450829|380|4|357| +2450829|382|4|967| +2450829|385|4|716| +2450829|386|4|754| +2450829|388|4|187| +2450829|391|4|119| +2450829|392|4|433| +2450829|394|4|512| +2450829|397|4|290| +2450829|398|4|497| +2450829|400|4|572| +2450829|403|4|597| +2450829|404|4|901| +2450829|406|4|801| +2450829|409|4|379| +2450829|410|4|711| +2450829|412|4|112| +2450829|415|4|466| +2450829|416|4|594| +2450829|418|4|264| +2450829|421|4|| +2450829|422|4|749| +2450829|424|4|422| +2450829|427|4|188| +2450829|428|4|176| +2450829|430|4|765| +2450829|433|4|181| +2450829|434|4|601| +2450829|436|4|| +2450829|439|4|227| +2450829|440|4|242| +2450829|442|4|681| +2450829|445|4|717| +2450829|446|4|736| +2450829|448|4|654| +2450829|451|4|749| +2450829|452|4|605| +2450829|454|4|158| +2450829|457|4|256| +2450829|458|4|984| +2450829|460|4|688| +2450829|463|4|84| +2450829|464|4|978| +2450829|466|4|453| +2450829|469|4|16| +2450829|470|4|830| +2450829|472|4|874| +2450829|475|4|62| +2450829|476|4|300| +2450829|478|4|917| +2450829|481|4|37| +2450829|482|4|539| +2450829|484|4|550| +2450829|487|4|139| +2450829|488|4|778| +2450829|490|4|683| +2450829|493|4|319| +2450829|494|4|756| +2450829|496|4|479| +2450829|499|4|473| +2450829|500|4|795| +2450829|502|4|998| +2450829|505|4|470| +2450829|506|4|768| +2450829|508|4|165| +2450829|511|4|863| +2450829|512|4|827| +2450829|514|4|336| +2450829|517|4|156| +2450829|518|4|304| +2450829|520|4|683| +2450829|523|4|312| +2450829|524|4|413| +2450829|526|4|| +2450829|529|4|861| +2450829|530|4|973| +2450829|532|4|614| +2450829|535|4|598| +2450829|536|4|169| +2450829|538|4|319| +2450829|541|4|589| +2450829|542|4|615| +2450829|544|4|463| +2450829|547|4|145| +2450829|548|4|986| +2450829|550|4|773| +2450829|553|4|793| +2450829|554|4|875| +2450829|556|4|126| +2450829|559|4|128| +2450829|560|4|277| +2450829|562|4|310| +2450829|565|4|980| +2450829|566|4|795| +2450829|568|4|395| +2450829|571|4|937| +2450829|572|4|497| +2450829|574|4|801| +2450829|577|4|315| +2450829|578|4|665| +2450829|580|4|272| +2450829|583|4|798| +2450829|584|4|289| +2450829|586|4|371| +2450829|589|4|312| +2450829|590|4|989| +2450829|592|4|424| +2450829|595|4|419| +2450829|596|4|366| +2450829|598|4|317| +2450829|601|4|992| +2450829|602|4|918| +2450829|604|4|269| +2450829|607|4|24| +2450829|608|4|743| +2450829|610|4|388| +2450829|613|4|560| +2450829|614|4|17| +2450829|616|4|973| +2450829|619|4|205| +2450829|620|4|237| +2450829|622|4|86| +2450829|625|4|844| +2450829|626|4|521| +2450829|628|4|| +2450829|631|4|| +2450829|632|4|| +2450829|634|4|779| +2450829|637|4|24| +2450829|638|4|612| +2450829|640|4|957| +2450829|643|4|869| +2450829|644|4|| +2450829|646|4|777| +2450829|649|4|409| +2450829|650|4|998| +2450829|652|4|844| +2450829|655|4|| +2450829|656|4|148| +2450829|658|4|430| +2450829|661|4|981| +2450829|662|4|392| +2450829|664|4|959| +2450829|667|4|283| +2450829|668|4|921| +2450829|670|4|875| +2450829|673|4|333| +2450829|674|4|| +2450829|676|4|44| +2450829|679|4|417| +2450829|680|4|950| +2450829|682|4|874| +2450829|685|4|920| +2450829|686|4|56| +2450829|688|4|129| +2450829|691|4|414| +2450829|692|4|553| +2450829|694|4|57| +2450829|697|4|988| +2450829|698|4|453| +2450829|700|4|698| +2450829|703|4|691| +2450829|704|4|446| +2450829|706|4|248| +2450829|709|4|139| +2450829|710|4|376| +2450829|712|4|228| +2450829|715|4|115| +2450829|716|4|485| +2450829|718|4|944| +2450829|721|4|460| +2450829|722|4|139| +2450829|724|4|830| +2450829|727|4|496| +2450829|728|4|257| +2450829|730|4|338| +2450829|733|4|330| +2450829|734|4|| +2450829|736|4|151| +2450829|739|4|387| +2450829|740|4|812| +2450829|742|4|476| +2450829|745|4|586| +2450829|746|4|408| +2450829|748|4|729| +2450829|751|4|230| +2450829|752|4|506| +2450829|754|4|252| +2450829|757|4|708| +2450829|758|4|526| +2450829|760|4|754| +2450829|763|4|109| +2450829|764|4|567| +2450829|766|4|| +2450829|769|4|30| +2450829|770|4|446| +2450829|772|4|539| +2450829|775|4|413| +2450829|776|4|200| +2450829|778|4|210| +2450829|781|4|| +2450829|782|4|691| +2450829|784|4|325| +2450829|787|4|475| +2450829|788|4|113| +2450829|790|4|908| +2450829|793|4|549| +2450829|794|4|938| +2450829|796|4|793| +2450829|799|4|482| +2450829|800|4|718| +2450829|802|4|61| +2450829|805|4|751| +2450829|806|4|335| +2450829|808|4|429| +2450829|811|4|967| +2450829|812|4|161| +2450829|814|4|618| +2450829|817|4|0| +2450829|818|4|477| +2450829|820|4|711| +2450829|823|4|821| +2450829|824|4|142| +2450829|826|4|697| +2450829|829|4|329| +2450829|830|4|387| +2450829|832|4|561| +2450829|835|4|971| +2450829|836|4|861| +2450829|838|4|903| +2450829|841|4|877| +2450829|842|4|153| +2450829|844|4|273| +2450829|847|4|956| +2450829|848|4|723| +2450829|850|4|944| +2450829|853|4|262| +2450829|854|4|406| +2450829|856|4|174| +2450829|859|4|87| +2450829|860|4|669| +2450829|862|4|779| +2450829|865|4|796| +2450829|866|4|409| +2450829|868|4|687| +2450829|871|4|997| +2450829|872|4|780| +2450829|874|4|742| +2450829|877|4|264| +2450829|878|4|562| +2450829|880|4|669| +2450829|883|4|369| +2450829|884|4|673| +2450829|886|4|392| +2450829|889|4|783| +2450829|890|4|119| +2450829|892|4|500| +2450829|895|4|881| +2450829|896|4|807| +2450829|898|4|207| diff --git a/spark-connector/spark-common/src/test/resources/data/tpcds/item/item.dat b/spark-connector/spark-common/src/test/resources/data/tpcds/item/item.dat new file mode 100644 index 00000000000..0e977a132c7 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/data/tpcds/item/item.dat @@ -0,0 +1,18000 @@ +1|AAAAAAAABAAAAAAA|1997-10-27||Powers will not get influences. Electoral ports should show low, annual chains. Now young visitors may pose now however final pages. Bitterly right children suit increasing, leading el|27.02|23.23|5003002|exportischolar #2|3|pop|5|Music|52|ableanti|N/A|3663peru009490160959|spring|Tsp|Unknown|6|ought| +2|AAAAAAAACAAAAAAA|1997-10-27|2000-10-26|False opportunities would run alone with a views. Early approaches would show inc, european intentions; important, main passages shall know urban, |1.12|0.38|1001001|amalgamalg #1|1|dresses|1|Women|294|esen stable|petite|516steel060826230906|rosy|Bunch|Unknown|98|able| +3|AAAAAAAACAAAAAAA|2000-10-27||False opportunities would run alone with a views. Early approaches would show inc, european intentions; important, main passages shall know urban, |7.11|0.38|1001001|brandbrand #4|7|decor|7|Home|294|esen stable|N/A|516steel060826230906|sienna|Cup|Unknown|18|pri| +4|AAAAAAAAEAAAAAAA|1997-10-27|1999-10-27|Normal systems would join simply different theories. Full, new clothes may eat instead achievements. D|1.35|0.85|3002001|importoexporti #1|2|infants|3|Children|479|n stationese|extra large|610157moccasin018327|red|Tbl|Unknown|26|ese| +5|AAAAAAAAEAAAAAAA|1999-10-28|2001-10-26|Normal systems would join simply different theories. Full, new clothes may eat instead achievements. D|4.00|1.76|2002002|importoimporto #2|2|shirts|2|Men|220|barableable|petite|42214rosy28066558020|pink|Cup|Unknown|27|anti| +6|AAAAAAAAEAAAAAAA|2001-10-27||Normal systems would join simply different theories. Full, new clothes may eat instead achievements. D|0.85|1.76|2002002|exportiimporto #1|3|pants|2|Men|212|barableable|large|42214rosy28066558020|moccasin|Bundle|Unknown|6|cally| +7|AAAAAAAAHAAAAAAA|1997-10-27||Anxious accounts must catch also years. Revolutionary, large directors used to embrace then mo|9.94|6.75|3001002|amalgexporti #2|1|newborn|3|Children|214|eseoughtable|petite|6moccasin24027188872|spring|Tsp|Unknown|64|ation| +8|AAAAAAAAIAAAAAAA|1997-10-27|2000-10-26|F|2.76|0.85|3003001|exportiexporti #1|3|toddlers|3|Children|630|barprically|extra large|35123wheat3256343398|turquoise|Carton|Unknown|25|eing| +9|AAAAAAAAIAAAAAAA|2000-10-27||F|4.46|0.85|1004002|edu packamalg #2|3|swimwear|1|Women|630|barprically|medium|35123wheat3256343398|wheat|Tbl|Unknown|3|n st| +10|AAAAAAAAKAAAAAAA|1997-10-27|1999-10-27|Classical services go trousers. However great galleries might say needs. Assumptions change very in favour of the notes. Teeth woul|8.94|4.11|10008011|namelessunivamalg #11|8|scanners|10|Electronics|350|barantipri|N/A|8159007505thistle447|pale|Tsp|Unknown|34|barought| +11|AAAAAAAAKAAAAAAA|1999-10-28|2001-10-26|Correct, fo|54.87|4.11|10008011|edu packbrand #4|14|estate|6|Jewelry|625|antiablecally|N/A|snow1543775706017405|yellow|Bunch|Unknown|26|oughtought| +12|AAAAAAAAKAAAAAAA|2001-10-27||Corporate, important facilities claim trying, external sides. Elements used to expect home pr|6.54|4.11|10008011|corpnameless #3|14|furniture|7|Home|264|esecallyable|N/A|968467777sky92069287|royal|Tbl|Unknown|19|ableought| +13|AAAAAAAANAAAAAAA|1997-10-27||Hard, private departments spoil more quickly possible members; clear troops fail only needs. |8.76|7.62|6012006|importobrand #6|12|costume|6|Jewelry|167|ationcallyought|N/A|883208731996blue7862|olive|Bundle|Unknown|51|priought| +14|AAAAAAAAOAAAAAAA|1997-10-27|2000-10-26|Teachers carry by the children; old democrats enco|1.85|0.59|8007005|brandnameless #5|7|hockey|8|Sports|460|barcallyese|N/A|1144670162goldenrod2|red|Dram|Unknown|6|eseought| +15|AAAAAAAAOAAAAAAA|2000-10-27||Teachers carry by the children; old democrats enco|2.57|0.59|5002002|importoscholar #2|2|country|5|Music|86|barcallyese|N/A|1144670162goldenrod2|royal|Pound|Unknown|11|antiought| +16|AAAAAAAAABAAAAAA|1997-10-27|1999-10-27|Dominant, christian pp. may not raise|0.31|0.14|1002001|importoamalg #1|2|fragrances|1|Women|117|ationoughtought|large|36933056603steel7373|bisque|Lb|Unknown|23|callyought| +17|AAAAAAAAABAAAAAA|1999-10-28|2001-10-26|Dominant, christian pp. may not raise|6.49|0.14|1002001|amalgimporto #2|1|accessories|2|Men|117|ationoughtought|extra large|452645olive281530722|snow|Dram|Unknown|41|ationought| +18|AAAAAAAAABAAAAAA|2001-10-27||Twin, particular aspects will accept only on|0.87|0.48|1002001|importoamalg #1|2|fragrances|1|Women|117|ationoughtought|medium|452645olive281530722|rosy|Carton|Unknown|63|eingought| +19|AAAAAAAADBAAAAAA|1997-10-27||Political parents know right; perfec|10.61|4.77|2004002|edu packimporto #2|4|sports-apparel|2|Men|364|esecallypri|large|85seashell1303417084|smoke|Ton|Unknown|13|n stought| +20|AAAAAAAAEBAAAAAA|1997-10-27|2000-10-26|Legal, foreign days know losses; briefly equivalent arguments will expect today. New, front grounds look hot, other aspects. Actually national husbands show usually of course other stations; huge, k|29.35|18.78|1003001|exportiamalg #1|3|maternity|1|Women|995|antin stn st|extra large|97245417ivory0043452|tan|Gram|Unknown|21|barable| +21|AAAAAAAAEBAAAAAA|2000-10-27||Legal, foreign days know losses; briefly equivalent arguments will expect today. New, front grounds look hot, other aspects. Actually national husbands show usually of course other stations; huge, k|10.71|8.46|6016006|corpbrand #6|3|consignment|6|Jewelry|995|antin stn st|N/A|1050463678plum205437|sienna|Pallet|Unknown|4|oughtable| +22|AAAAAAAAGBAAAAAA|1997-10-27|1999-10-27|Members endure already near additional details. Ministers should ignore whole times. Bright, brief beliefs become highly bright men; o|7.11|3.27|10005006|scholarunivamalg #6|5|karoke|10|Electronics|169|n stcallyought|N/A|4900684033pink844758|sandy|Each|Unknown|40|ableable| +23|AAAAAAAAGBAAAAAA|1999-10-28|2001-10-26|Below long minutes make primarily by a months. Secure effects get much upo|3.73|1.41|10005006|scholarunivamalg #7|5|karoke|10|Electronics|260|n stcallyought|N/A|569seashell149755477|papaya|Bunch|Unknown|44|priable| +24|AAAAAAAAGBAAAAAA|2001-10-27||Below long minutes make primarily by a months. Secure effects get much upo|2.61|0.86|10005006|amalgamalg #1|1|dresses|1|Women|260|barcallyable|medium|569seashell149755477|lime|Ton|Unknown|10|eseable| +25|AAAAAAAAJBAAAAAA|1997-10-27||Major brothers must not mean true annual, aware workers. Regulations will let more mechanisms. English pictures take tha|75.28|50.43|1004002|edu packamalg #2|4|swimwear|1|Women|338|eingpripri|large|90papaya546284022999|purple|Bundle|Unknown|18|antiable| +26|AAAAAAAAKBAAAAAA|1997-10-27|2000-10-26|Hands cannot show just administrative studies. Powerful orders come fiscal thanks. Major, certain nurses work still relative pa|2.77|1.91|2004001|edu packimporto #1|4|sports-apparel|2|Men|26|callyable|petite|thistle3370503164308|spring|Box|Unknown|31|callyable| +27|AAAAAAAAKBAAAAAA|2000-10-27||New, only police shall start generally left, important holidays. Inte|8.57|1.91|2004001|importocorp #4|4|diamonds|6|Jewelry|734|esepriation|N/A|thistle3370503164308|orchid|Pound|Unknown|31|ationable| +28|AAAAAAAAMBAAAAAA|1997-10-27|1999-10-27|External, actual teache|0.15|0.07|3003001|exportiexporti #1|3|toddlers|3|Children|886|callyeingeing|medium|56715743614hot522158|turquoise|Dram|Unknown|32|eingable| +29|AAAAAAAAMBAAAAAA|1999-10-28|2001-10-26|Rational, electronic photographs worry. Other, similar pounds might enable suddenly middle, growing months. Military, desperate funds shall not know then also various germans. Heavy|1.04|0.07|3003001|edu packamalg #2|3|swimwear|1|Women|209|callyeingeing|large|56715743614hot522158|yellow|Carton|Unknown|35|n stable| +30|AAAAAAAAMBAAAAAA|2001-10-27||Systems loosen special, easy partners. Prime, noble windows condemn effective, nation|86.00|62.78|6008007|namelesscorp #7|8|mens watch|6|Jewelry|732|callyeingeing|N/A|spring82346236549375|gainsboro|Bundle|Unknown|32|barpri| +31|AAAAAAAAPBAAAAAA|1997-10-27||So new campaigns teach more straight early indians. International offices shake actual ministers. New, liable theories can see expenses. Nice, imperial teams wo|8.48|3.30|8015002|scholarmaxi #2|15|fishing|8|Sports|167|ationcallyought|N/A|7wheat78839382233684|lawn|Each|Unknown|98|oughtpri| +32|AAAAAAAAACAAAAAA|1997-10-27|2000-10-26|Different numbers might not visit; rights used to remember. Labour students must put as slowly possible children. Never|1.77|1.00|7004003|edu packbrand #3|4|curtains/drapes|7|Home|431|oughtpriese|N/A|royal348530153475607|turquoise|Dram|Unknown|7|ablepri| +33|AAAAAAAAACAAAAAA|2000-10-27||Sure available terms know just nice, human officials. Problems used to receive. United, cheap changes get better british houses. Full bars shift often important readers; inc|7.86|1.00|3003002|exportiexporti #2|4|toddlers|3|Children|154|eseantiought|medium|36972puff56515747175|plum|Carton|Unknown|51|pripri| +34|AAAAAAAACCAAAAAA|1997-10-27|1999-10-27|Victorian miles would reform as well relationships. However tired institutions will send. Dramatically likely rats catch in a teams. Years may spend nearly mass, high|2.07|1.49|10007001|brandunivamalg #1|7|personal|10|Electronics|553|priantianti|N/A|29171thistle61991621|honeydew|Pallet|Unknown|58|esepri| +35|AAAAAAAACCAAAAAA|1999-10-28|2001-10-26|Victorian miles would reform as well relationships. However tired institutions will send. Dramatically likely rats catch in a teams. Years may spend nearly mass, high|2.32|1.11|10007001|amalgmaxi #12|1|arts|9|Books|553|priantianti|N/A|khaki119648424197997|green|Tsp|Unknown|5|antipri| +36|AAAAAAAACCAAAAAA|2001-10-27||Victorian miles would reform as well relationships. However tired institutions will send. Dramatically likely rats catch in a teams. Years may spend nearly mass, high|91.42|34.73|1001001|amalgamalg #1|1|dresses|1|Women|553|priantianti|medium|khaki119648424197997|thistle|Case|Unknown|37|callypri| +37|AAAAAAAAFCAAAAAA|1997-10-27||Clearly due proceedings should compensate particular, main agreements. Heels may not deny later easily different spaces; vast, additional items might export fresh, only sentences. Patients expand |2.42|0.87|2001002|amalgimporto #2|1|accessories|2|Men|489|n steingese|petite|8violet4563425031455|rose|Bundle|Unknown|28|ationpri| +38|AAAAAAAAGCAAAAAA|1997-10-27|2000-10-26|Cheap problems can draw please over good stars. Either long children would pass similar, different metres; base, large surroundings |16.70|8.01|2004001|edu packimporto #1|4|sports-apparel|2|Men|933|priprin st|medium|5red3080638918015686|violet|Bunch|Unknown|36|eingpri| +39|AAAAAAAAGCAAAAAA|2000-10-27||Cheap problems can draw please over good stars. Either long children would pass similar, different metres; base, large surroundings |5.07|8.01|2004001|importomaxi #4|2|business|9|Books|368|eingcallypri|N/A|5red3080638918015686|sandy|Unknown|Unknown|34|n stpri| +40|AAAAAAAAICAAAAAA|1997-10-27|1999-10-27|Deep good activities should resist to a substances; that is beautiful businessmen like problems. Late huge meet|9.93|8.63|7008001|namelessbrand #1|8|lighting|7|Home|147|ationeseought|N/A|olive423496330709355|sandy|Pallet|Unknown|64|barese| +41|AAAAAAAAICAAAAAA|1999-10-28|2001-10-26|Deep good activities should resist to a substances; that is beautiful businessmen like problems. Late huge meet|4.40|8.63|7008001|exportiexporti #2|8|toddlers|3|Children|310|ationeseought|small|09031176427khaki3414|red|Bunch|Unknown|85|oughtese| +42|AAAAAAAAICAAAAAA|2001-10-27||Once straight tools migh|9.15|8.63|5003001|exportischolar #1|3|pop|5|Music|540|bareseanti|N/A|09031176427khaki3414|powder|Cup|Unknown|64|ableese| +43|AAAAAAAALCAAAAAA|1997-10-27||Thus present women should hear for a shares; leaders must come early; immediate men will want exactly young groups. Insects may ask narrow variations. New leaders should deal |6.08|4.86|9010008|univunivamalg #8|10|travel|9|Books|301|oughtbarpri|N/A|19orchid108200357725|firebrick|Pallet|Unknown|19|priese| +44|AAAAAAAAMCAAAAAA|1997-10-27|2000-10-26|Wide clear weeks join surely medical others; again dull effects stretch. Only, great needs may brin|5.20|1.71|5001001|amalgscholar #1|1|rock|5|Music|124|eseableought|N/A|salmon36671081354930|chocolate|Case|Unknown|59|eseese| +45|AAAAAAAAMCAAAAAA|2000-10-27||Serious, little suppliers object hot, new weeks. Votes shall prevail yet. Windows understand equal members. Average consequences go at least pales|0.37|1.71|5001001|namelesscorp #6|1|mens watch|6|Jewelry|459|eseableought|N/A|salmon36671081354930|khaki|N/A|Unknown|15|antiese| +46|AAAAAAAAOCAAAAAA|1997-10-27|1999-10-27|Emotional, nuclear farmers can understand also in t|98.66|85.83|6012007|importobrand #7|12|costume|6|Jewelry|252|ableantiable|N/A|115838salmon36529828|saddle|Tsp|Unknown|19|callyese| +47|AAAAAAAAOCAAAAAA|1999-10-28|2001-10-26|Emotional, nuclear farmers can understand also in t|3.00|1.74|6012007|amalgedu pack #2|1|womens|4|Shoes|99|ableantiable|economy|115838salmon36529828|peru|Dram|Unknown|5|ationese| +48|AAAAAAAAOCAAAAAA|2001-10-27||Emotional, nuclear farmers can understand also in t|1.45|1.74|5001001|amalgscholar #1|1|rock|5|Music|41|ableantiable|N/A|03879white9235299949|thistle|Dram|Unknown|21|eingese| +49|AAAAAAAABDAAAAAA|1997-10-27||Regional, indian casualties shall say of course legal relations. Endless, cultural animals dislike new, possible resources. Socialist, very plans wonder precisely available star|8.01|2.48|3004002|edu packexporti #2|4|school-uniforms|3|Children|443|prieseese|large|027315130steel656106|powder|Unknown|Unknown|56|n stese| +50|AAAAAAAACDAAAAAA|1997-10-27|2000-10-26|Activities cover |4.99|3.54|4004001|edu packedu pack #1|4|athletic|4|Shoes|140|bareseought|small|1446003050medium4139|papaya|Cup|Unknown|72|baranti| +51|AAAAAAAACDAAAAAA|2000-10-27||International, likely trees put carefully variations. Eventually silent observations might think better changes. Western, aggressive gardens protect furthermore stars. Yesterday adequate action|3.99|1.99|3003002|exportiexporti #2|3|toddlers|3|Children|140|bareseought|extra large|1446003050medium4139|sky|Gram|Unknown|33|oughtanti| +52|AAAAAAAAEDAAAAAA|1997-10-27|1999-10-27|Very private features may not like newspapers. Intently right groups try very enquiries; too good minutes want less limited employers. Books ought to provide accounta|2.99|2.15|3002001|importoexporti #1|2|infants|3|Children|865|anticallyeing|medium|moccasin133060463675|rose|Pound|Unknown|33|ableanti| +53|AAAAAAAAEDAAAAAA|1999-10-28|2001-10-26|Very private features may not like newspapers. Intently right groups try very enquiries; too good minutes want less limited employers. Books ought to provide accounta|8.26|2.47|3002001|univamalgamalg #3|10|memory|10|Electronics|865|anticallyeing|N/A|moccasin133060463675|peru|Case|Unknown|14|prianti| +54|AAAAAAAAEDAAAAAA|2001-10-27||Very private features may not like newspapers. Intently right groups try very enquiries; too good minutes want less limited employers. Books ought to provide accounta|37.84|20.05|1001001|amalgamalg #1|1|dresses|1|Women|865|anticallyeing|medium|turquoise29355882814|khaki|Ounce|Unknown|40|eseanti| +55|AAAAAAAAHDAAAAAA|1997-10-27||Applicable versions feel so silly peasants. Lessons make relatively for example medical leaves. Details might want criminal, id|0.99|0.79|1004002|edu packamalg #2|4|swimwear|1|Women|47|ationese|medium|7975726dodger7126990|ghost|Cup|Unknown|18|antianti| +56|AAAAAAAAIDAAAAAA|1997-10-27|2000-10-26|Events might shape successfully; recent flowers should trace alike hard questions. Small areas could not give easy, enthusiastic ends. Obvious |7.44|3.94|10011006|amalgamalgamalg #6|11|disk drives|10|Electronics|412|ableoughtese|N/A|45885navy00969807009|hot|Tsp|Unknown|5|callyanti| +57|AAAAAAAAIDAAAAAA|2000-10-27||Events might shape successfully; recent flowers should trace alike hard questions. Small areas could not give easy, enthusiastic ends. Obvious |4.58|3.94|10011006|edu packexporti #2|11|school-uniforms|3|Children|591|ableoughtese|medium|611purple39006426349|red|Dozen|Unknown|24|ationanti| +58|AAAAAAAAKDAAAAAA|1997-10-27|1999-10-27|Cuts may hold already; daughters can like exclusively pregnant, fresh police; actual,|0.87|0.71|4003001|exportiedu pack #1|3|kids|4|Shoes|187|ationeingought|small|602196716medium47751|blush|Pound|Unknown|4|einganti| +59|AAAAAAAAKDAAAAAA|1999-10-28|2001-10-26|Cuts may hold already; daughters can like exclusively pregnant, fresh police; actual,|38.82|15.13|3004002|edu packexporti #2|3|school-uniforms|3|Children|413|prioughtese|medium|602196716medium47751|sandy|Tsp|Unknown|57|n stanti| +60|AAAAAAAAKDAAAAAA|2001-10-27||Cuts may hold already; daughters can like exclusively pregnant, fresh police; actual,|4.87|15.13|3004002|maxicorp #5|9|womens watch|6|Jewelry|849|n steseeing|N/A|602196716medium47751|indian|Unknown|Unknown|19|barcally| +61|AAAAAAAANDAAAAAA|1997-10-27||Pictures see even healthy designs. Considerable, visual concessions last controversia|0.50|0.22|10006011|corpunivamalg #11|6|musical|10|Electronics|785|antieingation|N/A|83227powder377577506|gainsboro|Case|Unknown|22|oughtcally| +62|AAAAAAAAODAAAAAA|1997-10-27|2000-10-26|Either british authorities would receive slightly. Yesterday unique clothes work seldom old sales. Famous, possible programmes take reall|3.49|1.18|6004007|edu packcorp #7|4|bracelets|6|Jewelry|633|priprically|N/A|206thistle2954355697|violet|Bunch|Unknown|84|ablecally| +63|AAAAAAAAODAAAAAA|2000-10-27||Short, bitter heads like then events. Assessments may hide just. Wild european books should maintain yesterday useful natural workers. Late responsible estimates would |4.34|1.18|6004007|importoimporto #2|2|shirts|2|Men|633|priprically|medium|206thistle2954355697|red|Ton|Unknown|41|prically| +64|AAAAAAAAAEAAAAAA|1997-10-27|1999-10-27|Both new conditions ask acute, ashamed pupils. Short, poor fami|2.02|1.69|8009005|maxinameless #5|9|optics|8|Sports|322|ableablepri|N/A|2tan8009172776437677|maroon|Tbl|Unknown|14|esecally| +65|AAAAAAAAAEAAAAAA|1999-10-28|2001-10-26|Both new conditions ask acute, ashamed pupils. Short, poor fami|73.24|54.19|10008015|namelessunivamalg #15|8|scanners|10|Electronics|363|ableablepri|N/A|567702bisque83928674|blue|Bundle|Unknown|27|antically| +66|AAAAAAAAAEAAAAAA|2001-10-27||Conservatives exert ahead for a|6.67|54.19|10008015|maximaxi #7|8|science|9|Books|121|oughtableought|N/A|438189282firebrick54|purple|Bundle|Unknown|51|callycally| +67|AAAAAAAADEAAAAAA|1997-10-27||So fair schools must go problems. Children should not paint in a photographs. Great, late senten|1.47|0.83|9015008|scholarunivamalg #8|15|fiction|9|Books|285|antieingable|N/A|583spring53597495026|black|Ounce|Unknown|57|ationcally| +68|AAAAAAAAEEAAAAAA|1997-10-27|2000-10-26|Principles secure best. Relevant foods mislead there so prime relations. Quickly convenient times may think men. Interesting, other bodies w|6.93|3.11|2004001|edu packimporto #1|4|sports-apparel|2|Men|76|callyation|petite|16294729rose54248882|navy|Carton|Unknown|35|eingcally| +69|AAAAAAAAEEAAAAAA|2000-10-27||Modern, golden faces illustrate previously grounds. Acceptable, gold sides cannot mind modern plans; large thanks find highly only meals. Enormous, popular teachers|5.67|3.11|2004001|importoamalg #2|2|fragrances|1|Women|76|callyation|large|972807876turquoise30|pale|Gross|Unknown|28|n stcally| +70|AAAAAAAAGEAAAAAA|1997-10-27|1999-10-27|Factors sha|9.79|4.40|5004001|edu packscholar #1|4|classical|5|Music|445|antieseese|N/A|purple89249207875245|rose|Dram|Unknown|52|baration| +71|AAAAAAAAGEAAAAAA|1999-10-28|2001-10-26|General stars believe for example at a|3.21|2.53|9014012|edu packunivamalg #12|14|sports|9|Books|445|antieseese|N/A|purple89249207875245|lavender|Ounce|Unknown|26|oughtation| +72|AAAAAAAAGEAAAAAA|2001-10-27||Particu|5.31|2.53|1002001|importoamalg #1|2|fragrances|1|Women|445|antieseese|medium|5peru941765840464570|blush|Ton|Unknown|27|ableation| +73|AAAAAAAAJEAAAAAA|1997-10-27||Social, simple se|41.16|27.57|1003002|exportiamalg #2|3|maternity|1|Women|653|priantically|petite|03293333saddle019730|misty|Tbl|Unknown|3|priation| +74|AAAAAAAAKEAAAAAA|1997-10-27|2000-10-26|Clergy ought |3.91|1.44|2004001|edu packimporto #1|4|sports-apparel|2|Men|954|eseantin st|small|8341981838751coral30|papaya|Carton|Unknown|17|eseation| +75|AAAAAAAAKEAAAAAA|2000-10-27||Here distant classes would not prove therefore objectives. Public books improve of course politi|3.26|1.30|2004001|exportiamalgamalg #8|4|stereo|10|Electronics|954|eseantin st|N/A|965042tomato56650663|wheat|Tbl|Unknown|12|antiation| +76|AAAAAAAAMEAAAAAA|1997-10-27|1999-10-27|Around single relations clear heavily over a controls. Arms could leave signs. T|3.84|2.76|9010005|univunivamalg #5|10|travel|9|Books|46|callyese|N/A|140503351631sienna87|puff|Oz|Unknown|32|callyation| +77|AAAAAAAAMEAAAAAA|1999-10-28|2001-10-26|Around single relations clear heavily over a controls. Arms could leave signs. T|0.90|2.76|3003002|exportiexporti #2|3|toddlers|3|Children|46|callyese|small|140503351631sienna87|purple|Ounce|Unknown|88|ationation| +78|AAAAAAAAMEAAAAAA|2001-10-27||Around single relations clear heavily over a controls. Arms could leave signs. T|0.23|0.14|3003002|corpcorp #7|6|rings|6|Jewelry|515|callyese|N/A|85716779158spring390|royal|Lb|Unknown|70|eingation| +79|AAAAAAAAPEAAAAAA|1997-10-27||Video-taped, ch|2.87|1.37|10009012|maxiunivamalg #12|9|televisions|10|Electronics|385|antieingpri|N/A|sandy027498424299031|papaya|Oz|Unknown|2|n station| +80|AAAAAAAAAFAAAAAA|1997-10-27|2000-10-26|Particular, armed costs ought to spin certainly. Events get in the costs. Late, parliamentary foods shall not arrest there able men. Other, difficult officers hold high interests. Model shares dre|2.16|1.59|4003001|exportiedu pack #1|3|kids|4|Shoes|376|callyationpri|N/A|26985wheat7381826860|yellow|Bundle|Unknown|38|bareing| +81|AAAAAAAAAFAAAAAA|2000-10-27||Particular, armed costs ought to spin certainly. Events get in the costs. Late, parliamentary foods shall not arrest there able men. Other, difficult officers hold high interests. Model shares dre|3.65|3.13|4002002|importoedu pack #2|3|mens|4|Shoes|37|callyationpri|large|26985wheat7381826860|cyan|Box|Unknown|68|oughteing| +82|AAAAAAAACFAAAAAA|1997-10-27|1999-10-27|Seats could come confident, moder|1.43|1.17|4004001|edu packedu pack #1|4|athletic|4|Shoes|306|callybarpri|small|0340aquamarine369806|violet|Dram|Unknown|53|ableeing| +83|AAAAAAAACFAAAAAA|1999-10-28|2001-10-26|Children could not influence most flat, careful figures. Local, true stars could commend. As scottish blocks want still old benefits. Skills should not try precisely. Y|4.25|1.17|4004001|brandcorp #4|4|pendants|6|Jewelry|622|ableablecally|N/A|146048520linen957568|misty|Each|Unknown|5|prieing| +84|AAAAAAAACFAAAAAA|2001-10-27||Cultural, cultural contracts assume colours. Close legitimate flowers watch enough british reasons. Probably old years must set recently intensive words. Elderly factors go otherwise useless months.|1.41|1.26|2004001|edu packimporto #1|4|sports-apparel|2|Men|662|ablecallycally|large|8587plum853813979789|grey|Gross|Unknown|3|eseeing| +85|AAAAAAAAFFAAAAAA|1997-10-27||Again common customers should make hardly proposals. Close groups can prevent after a cars. Physical, human banks will not roll simply for instance agricultural hours. Northern centre|39.31|34.98|2003002|exportiimporto #2|3|pants|2|Men|156|callyantiought|small|859187711467pale4603|sky|Oz|Unknown|66|antieing| +86|AAAAAAAAGFAAAAAA|1997-10-27|2000-10-26|Parents like only new demands. Current players let charges. Specific, young towns realise however countr|0.87|0.74|6002007|importocorp #7|2|diamonds|6|Jewelry|985|antieingn st|N/A|642534525118salmon94|powder|Pallet|Unknown|2|callyeing| +87|AAAAAAAAGFAAAAAA|2000-10-27||Lives must rule better now sem|4.51|1.53|3001002|amalgexporti #2|2|newborn|3|Children|985|antieingn st|petite|66917821011ghost0861|sky|Bunch|Unknown|57|ationeing| +88|AAAAAAAAIFAAAAAA|1997-10-27|1999-10-27|Fears obtain still blind, active others. Forever numerous signs last even here civil strate|7.29|3.71|3001001|amalgexporti #1|1|newborn|3|Children|87|ationeing|large|97346sandy2527135842|snow|Gross|Unknown|24|eingeing| +89|AAAAAAAAIFAAAAAA|1999-10-28|2001-10-26|Sensitive requirements will not supply all intelligent effects. U|4.79|3.44|3001001|amalgimporto #2|1|accessories|2|Men|967|ationcallyn st|large|97346sandy2527135842|coral|Bundle|Unknown|5|n steing| +90|AAAAAAAAIFAAAAAA|2001-10-27||Both french meanings cannot tackle then|1.18|0.44|3001001|amalgscholar #1|1|rock|5|Music|907|ationbarn st|N/A|97346sandy2527135842|blue|Tsp|Unknown|35|barn st| +91|AAAAAAAALFAAAAAA|1997-10-27||True potatoes sound equal heads|0.64|0.28|10006007|corpunivamalg #7|6|musical|10|Electronics|176|callyationought|N/A|2925royal36510681005|lace|Lb|Unknown|74|oughtn st| +92|AAAAAAAAMFAAAAAA|1997-10-27|2000-10-26|Employers w|4.48|2.64|8009003|maxinameless #3|9|optics|8|Sports|547|ationeseanti|N/A|1906778941seashell97|indian|N/A|Unknown|12|ablen st| +93|AAAAAAAAMFAAAAAA|2000-10-27||Employers w|1.06|0.61|8015004|scholarmaxi #4|15|fishing|8|Sports|290|barn stable|N/A|082sienna17917347780|cyan|Lb|Unknown|90|prin st| +94|AAAAAAAAOFAAAAAA|1997-10-27|1999-10-27|Institutions ought to provide good agents; large, comfortable|0.11|0.06|2002001|importoimporto #1|2|shirts|2|Men|254|eseantiable|economy|46040088turquoise736|orange|Box|Unknown|33|esen st| +95|AAAAAAAAOFAAAAAA|1999-10-28|2001-10-26|Institutions ought to provide good agents; large, comfortable|2.55|1.30|5003002|exportischolar #2|3|pop|5|Music|276|eseantiable|N/A|414160524grey3616827|pale|Tbl|Unknown|95|antin st| +96|AAAAAAAAOFAAAAAA|2001-10-27||Periods provide as distinctive, little ingredients; therefore social boys must not leave slightl|4.19|1.30|6001001|amalgcorp #1|3|birdal|6|Jewelry|54|eseanti|N/A|20988791004papaya520|cream|Pound|Unknown|17|callyn st| +97|AAAAAAAABGAAAAAA|1997-10-27||Quite different services promote all the same. Private, marginal colleagues play of course similar, different girls. French, local girls reap here. Bad movies shorten relatively. Terms|57.09|32.54|9015002|scholarunivamalg #2|15|fiction|9|Books|275|antiationable|N/A|96goldenrod451188509|rose|Gross|Unknown|5|ationn st| +98|AAAAAAAACGAAAAAA|1997-10-27|2000-10-26|Opportunities clear there. Basic rules ask british locations. More financial visits construct other funds. Unk|3.16|2.18|8008009|namelessnameless #9|8|outdoor|8|Sports|302|ablebarpri|N/A|01882924064yellow241|medium|Ton|Unknown|24|eingn st| +99|AAAAAAAACGAAAAAA|2000-10-27||Opportunities clear there. Basic rules ask british locations. More financial visits construct other funds. Unk|4.81|2.18|8008009|scholarunivamalg #15|5|karoke|10|Electronics|302|ablebarpri|N/A|215235wheat677989979|salmon|Pound|Unknown|19|n stn st| +100|AAAAAAAAEGAAAAAA|1997-10-27|1999-10-27|British motives understand much heavy approaches. Officers keep. Significant systems operate characteristics.|2.06|0.63|5003001|exportischolar #1|3|pop|5|Music|462|ablecallyese|N/A|64492774682spring442|peru|N/A|Unknown|31|barbarought| +101|AAAAAAAAEGAAAAAA|1999-10-28|2001-10-26|More other feet think between a machines; just expensive days will open labour parties. Long changes join artificial, modern humans. Welsh historians go unfair, stra|4.51|4.01|1002002|importoamalg #2|3|fragrances|1|Women|816|ablecallyese|medium|yellow36581261689932|thistle|Case|Unknown|40|oughtbarought| +102|AAAAAAAAEGAAAAAA|2001-10-27||Formidable years will prove too. Factors used to match however now european spaces. Pupils look normally a|3.93|3.18|1004001|edu packamalg #1|4|swimwear|1|Women|816|callyoughteing|medium|9thistle261645945803|white|N/A|Unknown|24|ablebarought| +103|AAAAAAAAHGAAAAAA|1997-10-27||Fair, continental thanks see. Ideological stairs would not make local, civil employees. Light children would appear short away funny speakers. Little, other genes can dete|4.30|3.26|4001002|amalgedu pack #2|1|womens|4|Shoes|187|ationeingought|large|bisque72496427392367|wheat|Cup|Unknown|30|pribarought| +104|AAAAAAAAIGAAAAAA|1997-10-27|2000-10-26|Rapidly difficult films realize. Deep electronic parents calculate remaining affairs|2.33|1.25|9016009|corpunivamalg #9|16|mystery|9|Books|453|priantiese|N/A|57steel5257997933643|papaya|Ton|Unknown|4|esebarought| +105|AAAAAAAAIGAAAAAA|2000-10-27||Rapidly difficult films realize. Deep electronic parents calculate remaining affairs|9.21|5.34|6015002|scholarbrand #2|15|custom|6|Jewelry|452|priantiese|N/A|lace0157762062441759|lime|Pallet|Unknown|24|antibarought| +106|AAAAAAAAKGAAAAAA|1997-10-27|1999-10-27|European eyes could not establish workers. Then other members could not pres|0.58|0.22|10010005|univamalgamalg #5|10|memory|10|Electronics|307|ationbarpri|N/A|77620349purple966184|turquoise|Ounce|Unknown|93|callybarought| +107|AAAAAAAAKGAAAAAA|1999-10-28|2001-10-26|A little special changes would not turn certainly terms. Big, immediate earn|7.77|0.22|10010005|amalgmaxi #12|1|arts|9|Books|307|ationbarpri|N/A|7493lawn291455079127|peru|Bundle|Unknown|40|ationbarought| +108|AAAAAAAAKGAAAAAA|2001-10-27||A little special changes would not turn certainly terms. Big, immediate earn|6.38|5.55|10010005|edu packamalgamalg #7|1|automotive|10|Electronics|297|ationbarpri|N/A|401azure184427054517|powder|Tsp|Unknown|59|eingbarought| +109|AAAAAAAANGAAAAAA|1997-10-27||Rightly central others concentrate often mental s|9.46|6.24|7006010|corpbrand #10|6|rugs|7|Home|306|callybarpri|N/A|61dodger212717666342|lawn|Oz|Unknown|38|n stbarought| +110|AAAAAAAAOGAAAAAA|1997-10-27|2000-10-26|Programmes will retreat rich, late demands. English, particular|1.31|1.06|1001001|amalgamalg #1|1|dresses|1|Women|253|priantiable|petite|85914244513indian675|tan|Lb|Unknown|77|baroughtought| +111|AAAAAAAAOGAAAAAA|2000-10-27||No doubt general companies result properly systems. Companies may allow young shapes. Essentially interested colours shall |1.18|0.48|10009010|maxiunivamalg #10|9|televisions|10|Electronics|253|priantiable|N/A|192018goldenrod39351|seashell|Lb|Unknown|35|oughtoughtought| +112|AAAAAAAAAHAAAAAA|1997-10-27|1999-10-27|Complete shares ought to imp|0.18|0.15|4003001|exportiedu pack #1|3|kids|4|Shoes|303|pribarpri|extra large|6236light83622302998|salmon|Dram|Unknown|62|ableoughtought| +113|AAAAAAAAAHAAAAAA|1999-10-28|2001-10-26|Then ordina|3.81|3.31|4003001|importomaxi #4|3|guns|8|Sports|719|pribarpri|N/A|3097441603rose229570|pink|Gram|Unknown|12|prioughtought| +114|AAAAAAAAAHAAAAAA|2001-10-27||Woods might not come sometimes individual units; occasions used to keep. Significant units work|4.30|2.66|7016005|corpnameless #5|16|furniture|7|Home|202|pribarpri|N/A|3097441603rose229570|midnight|Lb|Unknown|1|eseoughtought| +115|AAAAAAAADHAAAAAA|1997-10-27||Enough apparent elements reverse actu|2.68|2.27|7016006|corpnameless #6|16|furniture|7|Home|67|ationcally|N/A|22456140014blanched1|rosy|Pound|Unknown|85|antioughtought| +116|AAAAAAAAEHAAAAAA|1997-10-27|2000-10-26|Never chronic yards must see home political, cautious practices. Stages see offen|9.39|3.19|6007005|brandcorp #5|7|pendants|6|Jewelry|231|oughtpriable|N/A|3683520508moccasin63|purple|Pallet|Unknown|34|callyoughtought| +117|AAAAAAAAEHAAAAAA|2000-10-27||Never chronic yards must see home political, cautious practices. Stages see offen|4.43|2.30|6007005|importoscholar #2|7|country|5|Music|231|oughtpriable|N/A|3683520508moccasin63|pale|Gross|Unknown|62|ationoughtought| +118|AAAAAAAAGHAAAAAA|1997-10-27|1999-10-27|Low, left communities send always so gothic operations. Too significant colours remove for a pounds. Eggs go scientific levels. Results sleep short, black partic|3.83|2.79|3001001|amalgexporti #1|1|newborn|3|Children|451|oughtantiese|small|683777798795sienna50|pale|Oz|Unknown|62|eingoughtought| +119|AAAAAAAAGHAAAAAA|1999-10-28|2001-10-26|Low, left communities send always so gothic operations. Too significant colours remove for a pounds. Eggs go scientific levels. Results sleep short, black partic|4.67|2.79|3001001|edu packimporto #2|1|sports-apparel|2|Men|186|oughtantiese|petite|683777798795sienna50|red|Bundle|Unknown|21|n stoughtought| +120|AAAAAAAAGHAAAAAA|2001-10-27||Basic facts might seem then dirty developments; institutions used t|4.56|2.32|3001001|importoedu pack #1|2|mens|4|Shoes|186|oughtantiese|petite|8601952peru191605660|slate|Gram|Unknown|30|barableought| +121|AAAAAAAAJHAAAAAA|1997-10-27||Alre|38.79|11.63|9013002|exportiunivamalg #2|13|self-help|9|Books|296|callyn stable|N/A|629852575spring16060|spring|Oz|Unknown|24|oughtableought| +122|AAAAAAAAKHAAAAAA|1997-10-27|2000-10-26|Generally french beds will ask amounts. Difficult, difficult workers would come once again in a resources. So inc|2.62|2.33|9006003|corpmaxi #3|6|parenting|9|Books|321|oughtablepri|N/A|turquoise42787458444|sienna|Ounce|Unknown|13|ableableought| +123|AAAAAAAAKHAAAAAA|2000-10-27||Original theories may not need more other, economic exports. Widely great things cannot concentrate over. Times could ask occasionally american |0.64|2.33|10012005|importoamalgamalg #5|12|monitors|10|Electronics|60|oughtablepri|N/A|498502yellow44224084|sandy|Ton|Unknown|26|priableought| +124|AAAAAAAAMHAAAAAA|1997-10-27|1999-10-27|Steadily other miles stop terms. Schemes can retire little, physical chairs; likely, statutory improvements stop essentially increased, co|9.03|4.69|2004001|edu packimporto #1|4|sports-apparel|2|Men|319|n stoughtpri|economy|14sky707696411757120|rose|Ounce|Unknown|54|eseableought| +125|AAAAAAAAMHAAAAAA|1999-10-28|2001-10-26|Steadily other miles stop terms. Schemes can retire little, physical chairs; likely, statutory improvements stop essentially increased, co|2.98|2.20|2004001|univbrand #6|10|jewelry boxes|6|Jewelry|168|eingcallyought|N/A|turquoise67308465535|peru|Tsp|Unknown|54|antiableought| +126|AAAAAAAAMHAAAAAA|2001-10-27||Words tell as commercial years. Medical studies shall cope attitudes; completely local others can think here winds. Seconds observe. New, fortunate services can lure to a pp.. Intensely|2.60|2.20|1003001|exportiamalg #1|10|maternity|1|Women|168|eingcallyought|small|turquoise67308465535|yellow|Bundle|Unknown|51|callyableought| +127|AAAAAAAAPHAAAAAA|1997-10-27||Private islands will complete large homes. Parts illustrate most in a operations; labour games could not use. Leaders feel. New groups shall not devote too pale characteristics. Mad thanks may not |3.66|1.13|7003008|exportibrand #8|3|kids|7|Home|417|ationoughtese|N/A|9154848890214sienna3|navy|Ton|Unknown|11|ationableought| +128|AAAAAAAAAIAAAAAA|1997-10-27|2000-10-26|Wrong limits could not accompany now perhaps lonely customers. Anxious, neighbouring principles might arise molecules. Useful, short nerves think advantages. Angry, parental prices fly t|4.06|3.20|7004009|edu packbrand #9|4|curtains/drapes|7|Home|234|esepriable|N/A|7359977floral8145107|pale|Gram|Unknown|23|eingableought| +129|AAAAAAAAAIAAAAAA|2000-10-27||Wrong limits could not accompany now perhaps lonely customers. Anxious, neighbouring principles might arise molecules. Useful, short nerves think advantages. Angry, parental prices fly t|8.98|4.49|10004011|edu packunivamalg #11|4|audio|10|Electronics|165|esepriable|N/A|4898smoke29906028892|ivory|Gram|Unknown|23|n stableought| +130|AAAAAAAACIAAAAAA|1997-10-27|1999-10-27|Really original police could not cope nearly. Trusts will give. Conventional, positive pool|1.70|1.00|8015001|scholarmaxi #1|15|fishing|8|Sports|349|n stesepri|N/A|6beige99731892899200|rosy|Gross|Unknown|65|barpriought| +131|AAAAAAAACIAAAAAA|1999-10-28|2001-10-26|Really original police could not cope nearly. Trusts will give. Conventional, positive pool|96.31|30.81|8015001|importoscholar #2|2|country|5|Music|349|n stesepri|N/A|6beige99731892899200|saddle|Ounce|Unknown|84|oughtpriought| +132|AAAAAAAACIAAAAAA|2001-10-27||Courses p|1.40|30.81|8015001|importoamalg #1|2|fragrances|1|Women|525|n stesepri|large|6beige99731892899200|moccasin|Bundle|Unknown|1|ablepriought| +133|AAAAAAAAFIAAAAAA|1997-10-27||Dead services would cut from a jobs. Personally positive women should not start excessively automatic children. New, leading waves reach thousands; |3.12|1.21|2001002|amalgimporto #2|1|accessories|2|Men|338|eingpripri|medium|675lime2191538141656|red|Dram|Unknown|50|pripriought| +134|AAAAAAAAGIAAAAAA|1997-10-27|2000-10-26|Exactly|3.56|2.52|4003001|exportiedu pack #1|3|kids|4|Shoes|329|n stablepri|large|10957355855purple718|papaya|Cup|Unknown|58|esepriought| +135|AAAAAAAAGIAAAAAA|2000-10-27||New terms help today actual, clear words. Just intelligent times want thousands; free, english responses fetch normally from a occasions. Instead unemplo|0.52|0.28|4003001|maxiunivamalg #17|3|televisions|10|Electronics|621|n stablepri|N/A|10957355855purple718|smoke|Pallet|Unknown|27|antipriought| +136|AAAAAAAAIIAAAAAA|1997-10-27|1999-10-27|Beautiful, late walls see less political, required women. Windows hope refugees. Industrial, present windows will consider so american, social pupils. Tomorrow male girl|3.16|1.04|4002001|importoedu pack #1|2|mens|4|Shoes|428|eingableese|extra large|violet69857900085047|snow|N/A|Unknown|4|callypriought| +137|AAAAAAAAIIAAAAAA|1999-10-28|2001-10-26|Better tough shares enter namely helpful, new police. Homes mean now such as a estates; sexual, guilty parties run widely present implications. Truly beautiful b|1.68|1.04|4002001|exportiunivamalg #6|13|self-help|9|Books|428|eingableese|N/A|violet69857900085047|orchid|Gram|Unknown|29|ationpriought| +138|AAAAAAAAIIAAAAAA|2001-10-27||Over effective members shall possess serious, local supporters. Then overseas representatives will say cultural, particular planes. Black relationships ob|3.07|2.60|4002001|edu packmaxi #7|4|entertainments|9|Books|37|eingableese|N/A|8pink559006702646158|green|Each|Unknown|3|eingpriought| +139|AAAAAAAALIAAAAAA|1997-10-27||Clothes can get also; home financial premises should not give proudly. Disabled, urgent tears would not run. Previous, electric schools shall qualify usefully real heads. Very, |2.99|1.49|9015008|scholarunivamalg #8|15|fiction|9|Books|384|eseeingpri|N/A|steel025649618513128|medium|Tsp|Unknown|20|n stpriought| +140|AAAAAAAAMIAAAAAA|1997-10-27|2000-10-26|Hard, simple officers should reflect as a whole females. Sole sides must cover straight|9.00|2.88|5002001|importoscholar #1|2|country|5|Music|286|callyeingable|N/A|757388306purple44645|green|Gross|Unknown|56|bareseought| +141|AAAAAAAAMIAAAAAA|2000-10-27||Hard, simple officers should reflect as a whole females. Sole sides must cover straight|7.78|4.20|9015010|scholarunivamalg #10|2|fiction|9|Books|286|callyeingable|N/A|757388306purple44645|hot|Gross|Unknown|5|oughteseought| +142|AAAAAAAAOIAAAAAA|1997-10-27|1999-10-27|Details might not produce a bit other colours. Police cannot win past. Factors could get from time to time emotions. Temporary, model successes would |3.77|2.82|3001001|amalgexporti #1|1|newborn|3|Children|793|prin station|small|seashell948166819527|yellow|Unknown|Unknown|2|ableeseought| +143|AAAAAAAAOIAAAAAA|1999-10-28|2001-10-26|Desperately general missiles ought to know here other far talks. New times could exercise too young months. Unpleasant, likely countries explain.|4.74|2.82|3001001|exportiunivamalg #8|3|dvd/vcr players|10|Electronics|51|oughtanti|N/A|orchid98617682924562|mint|Gross|Unknown|78|prieseought| +144|AAAAAAAAOIAAAAAA|2001-10-27||Facilities deflect fully special consumers. Various, legal elections train close|1.52|0.80|7007005|brandbrand #5|3|decor|7|Home|51|oughtanti|N/A|dark2806463212122477|tan|N/A|Unknown|52|eseeseought| +145|AAAAAAAABJAAAAAA|1997-10-27||Members begin together industrial, re|59.77|24.50|9013002|exportiunivamalg #2|13|self-help|9|Books|548|eingeseanti|N/A|398424622862lime0710|saddle|Each|Unknown|43|antieseought| +146|AAAAAAAACJAAAAAA|1997-10-27|2000-10-26|Day|9.67|8.02|5001001|amalgscholar #1|1|rock|5|Music|644|eseesecally|N/A|04220113258562rose09|royal|Unknown|Unknown|23|callyeseought| +147|AAAAAAAACJAAAAAA|2000-10-27||As national events can strike more full services. Obviously original representatives should like poor years. Relatively va|2.48|1.56|2002002|importoimporto #2|2|shirts|2|Men|9|eseesecally|extra large|04220113258562rose09|salmon|Gross|Unknown|34|ationeseought| +148|AAAAAAAAEJAAAAAA|1997-10-27|1999-10-27|Now federal women stay really. Hugely usual events should include at a children. Mutual observations hea|2.23|0.93|4004001|edu packedu pack #1|4|athletic|4|Shoes|627|ationablecally|petite|3822958purple5800198|purple|Box|Unknown|31|eingeseought| +149|AAAAAAAAEJAAAAAA|1999-10-28|2001-10-26|Now federal women stay really. Hugely usual events should include at a children. Mutual observations hea|8.49|5.17|1004002|edu packamalg #2|4|swimwear|1|Women|627|ationablecally|petite|114misty484305013524|honeydew|Oz|Unknown|84|n steseought| +150|AAAAAAAAEJAAAAAA|2001-10-27||Now federal women stay really. Hugely usual events should include at a children. Mutual observations hea|5.48|4.38|1004002|importoedu pack #1|4|mens|4|Shoes|627|ationablecally|small|2137560268saddle0903|salmon|Bundle|Unknown|1|barantiought| +151|AAAAAAAAHJAAAAAA|1997-10-27||Facilities could put backs. Good children ought to cut behind good numbers; vast, public campaigns used to develop again concerned partners. Widely follow|0.96|0.28|3004002|edu packexporti #2|4|school-uniforms|3|Children|327|ationablepri|small|866salmon34723399950|royal|Tsp|Unknown|3|oughtantiought| +152|AAAAAAAAIJAAAAAA|1997-10-27|2000-10-26|Private, present methods join others. Times try; private, level customers may make only continuously small obj|4.52|2.84|1002001|importoamalg #1|2|fragrances|1|Women|93|prin st|petite|01329slate7866063154|peru|Cup|Unknown|11|ableantiought| +153|AAAAAAAAIJAAAAAA|2000-10-27||Private, present methods join others. Times try; private, level customers may make only continuously small obj|60.71|2.84|1002001|edu packscholar #2|4|classical|5|Music|93|prin st|N/A|01329slate7866063154|papaya|Oz|Unknown|62|priantiought| +154|AAAAAAAAKJAAAAAA|1997-10-27|1999-10-27|Years house. White spor|8.25|7.09|4001001|amalgedu pack #1|1|womens|4|Shoes|103|pribarought|N/A|6steel85221242759043|powder|Ton|Unknown|15|eseantiought| +155|AAAAAAAAKJAAAAAA|1999-10-28|2001-10-26|Men ought to make words. Annual commu|1.47|7.09|4001001|importobrand #6|1|bedding|7|Home|12|ableought|N/A|77yellow409260944395|olive|Tbl|Unknown|22|antiantiought| +156|AAAAAAAAKJAAAAAA|2001-10-27||P|2.31|1.01|2003001|exportiimporto #1|1|pants|2|Men|12|ableought|petite|77yellow409260944395|slate|Pallet|Unknown|8|callyantiought| +157|AAAAAAAANJAAAAAA|1997-10-27||Young, white workers may not wreck british, statistical explanations. New complaints leave no longer only wide doors; shops beat new restrictions. Horses must not test by now anonym|2.21|0.86|9007002|brandmaxi #2|7|reference|9|Books|301|oughtbarpri|N/A|chocolate27270618988|sienna|Ounce|Unknown|23|ationantiought| +158|AAAAAAAAOJAAAAAA|1997-10-27|2000-10-26|General personnel should take by the pictures; personal, ol|9.17|4.40|7008009|namelessbrand #9|8|lighting|7|Home|656|callyantically|N/A|9purple4922510664060|royal|Bundle|Unknown|1|eingantiought| +159|AAAAAAAAOJAAAAAA|2000-10-27||General personnel should take by the pictures; personal, ol|9.45|5.57|2001002|amalgimporto #2|1|accessories|2|Men|2|callyantically|medium|9purple4922510664060|green|Bunch|Unknown|93|n stantiought| +160|AAAAAAAAAKAAAAAA|1997-10-27|1999-10-27|Desirable members will compare in a terms. Light friends shall record notably there continuous problems. Late, re|1.17|0.88|8013001|exportimaxi #1|13|sailing|8|Sports|207|ationbarable|N/A|41354metallic1918550|goldenrod|Gross|Unknown|50|barcallyought| +161|AAAAAAAAAKAAAAAA|1999-10-28|2001-10-26|Desirable members will compare in a terms. Light friends shall record notably there continuous problems. Late, re|3.74|1.68|1003002|exportiamalg #2|3|maternity|1|Women|207|ationbarable|large|41354metallic1918550|purple|Dram|Unknown|33|oughtcallyought| +162|AAAAAAAAAKAAAAAA|2001-10-27||Economic places must attack always too silly|1.79|1.25|4002001|importoedu pack #1|2|mens|4|Shoes|207|ationbarable|economy|251917727227snow2413|sienna|Pallet|Unknown|32|ablecallyought| +163|AAAAAAAADKAAAAAA|1997-10-27||Well alleged locations claim there critical babies. Current, peculiar needs could see for a animals. Inches cause junior, middle-class rates. Relationships count re|75.25|63.96|2004002|edu packimporto #2|4|sports-apparel|2|Men|168|eingcallyought|medium|3274olive04547276620|purple|Ton|Unknown|44|pricallyought| +164|AAAAAAAAEKAAAAAA|1997-10-27|2000-10-26|Contracts provide never main villages. Politicians should not enable only rooms. Also inner flowers will make so t|5.21|3.07|10016012|corpamalgamalg #12|16|wireless|10|Electronics|590|barn stanti|N/A|675973sienna77137603|turquoise|Cup|Unknown|52|esecallyought| +165|AAAAAAAAEKAAAAAA|2000-10-27||Contracts provide never main villages. Politicians should not enable only rooms. Also inner flowers will make so t|1.43|3.07|10016012|exportimaxi #6|13|sailing|8|Sports|46|callyese|N/A|66785157pink48052327|turquoise|N/A|Unknown|53|anticallyought| +166|AAAAAAAAGKAAAAAA|1997-10-27|1999-10-27|Difficulties might allow with a persons. Single, significant allegations matter |4.36|3.92|1002001|importoamalg #1|2|fragrances|1|Women|310|baroughtpri|medium|29968859ivory8916557|salmon|Gross|Unknown|30|callycallyought| +167|AAAAAAAAGKAAAAAA|1999-10-28|2001-10-26|Over great kids used to know even different, consistent homes. Further irish situations will claim. British acids examine here numerous expressions. Fine, big products can meet long|8.19|3.92|1002001|exporticorp #8|2|gold|6|Jewelry|310|baroughtpri|N/A|9white01458292336155|thistle|Gross|Unknown|6|ationcallyought| +168|AAAAAAAAGKAAAAAA|2001-10-27||Over great kids used to know even different, consistent homes. Further irish situations will claim. British acids examine here numerous expressions. Fine, big products can meet long|6.73|3.92|1001001|amalgamalg #1|2|dresses|1|Women|310|baroughtpri|small|4072099074376deep197|yellow|Dozen|Unknown|22|eingcallyought| +169|AAAAAAAAJKAAAAAA|1997-10-27||Lovely, social members would not follow either aware numbers; defences believe again over there poor applicatio|3.54|2.33|6011002|amalgbrand #2|11|semi-precious|6|Jewelry|238|eingpriable|N/A|98617450purple436508|slate|Each|Unknown|30|n stcallyought| +170|AAAAAAAAKKAAAAAA|1997-10-27|2000-10-26|Broad members see accurately guilty, public thanks; others meet close slowly sophisticated difficulties. Trees can search more large chains. |1.65|1.46|8004001|edu packnameless #1|4|camping|8|Sports|120|barableought|N/A|4127023489969steel40|sky|Bundle|Unknown|9|barationought| +171|AAAAAAAAKKAAAAAA|2000-10-27||Now social peasants proceed afterwards. Now unhappy police describe separately helpful figures. At least positive st|28.29|1.46|8004001|edu packscholar #2|4|classical|5|Music|65|barableought|N/A|4127023489969steel40|red|Ton|Unknown|14|oughtationought| +172|AAAAAAAAMKAAAAAA|1997-10-27|1999-10-27|Unacceptable, widespread towns may not block there about a records. Then |0.83|0.41|8007003|brandnameless #3|7|hockey|8|Sports|500|barbaranti|N/A|227943724898sky43420|spring|Cup|Unknown|13|ableationought| +173|AAAAAAAAMKAAAAAA|1999-10-28|2001-10-26|Unacceptable, widespread towns may not block there about a records. Then |1.75|0.41|8007003|amalgimporto #2|1|accessories|2|Men|500|barbaranti|medium|227943724898sky43420|yellow|Carton|Unknown|5|priationought| +174|AAAAAAAAMKAAAAAA|2001-10-27||Major, unknown hands might |0.65|0.41|10011005|amalgamalgamalg #5|1|disk drives|10|Electronics|187|barbaranti|N/A|61439655670steel4120|royal|Tbl|Unknown|49|eseationought| +175|AAAAAAAAPKAAAAAA|1997-10-27||Articles |1.26|1.10|5003002|exportischolar #2|3|pop|5|Music|162|ablecallyought|N/A|7374751569164saddle0|sienna|Carton|Unknown|96|antiationought| +176|AAAAAAAAALAAAAAA|1997-10-27|2000-10-26|Impossible, nearby bodies know much small, good officials. Sure ashamed problems show homes; exotic workers must |2.65|0.82|1004001|edu packamalg #1|4|swimwear|1|Women|555|antiantianti|small|64880479linen6337143|turquoise|Box|Unknown|21|callyationought| +177|AAAAAAAAALAAAAAA|2000-10-27||Appropriate, working-class lakes will bump similar members. Fully valuable hours exclu|5.14|0.82|1004001|exportischolar #2|3|pop|5|Music|210|baroughtable|N/A|2247rose749790060147|sandy|Pallet|Unknown|90|ationationought| +178|AAAAAAAACLAAAAAA|1997-10-27|1999-10-27|Civil, different employers ought to produce constantly on a arrangements. National, enormous observers secure as factors. Strong trends discuss as negative, major|2.40|1.34|6011003|amalgbrand #3|11|semi-precious|6|Jewelry|889|n steingeing|N/A|33338497dark80132200|yellow|Pallet|Unknown|22|eingationought| +179|AAAAAAAACLAAAAAA|1999-10-28|2001-10-26|Civil, different employers ought to produce constantly on a arrangements. National, enormous observers secure as factors. Strong trends discuss as negative, major|64.48|1.34|6011003|edu packscholar #2|4|classical|5|Music|451|oughtantiese|N/A|15482344rose71601051|sienna|Ounce|Unknown|3|n stationought| +180|AAAAAAAACLAAAAAA|2001-10-27||Civil, different employers ought to produce constantly on a arrangements. National, enormous observers secure as factors. Strong trends discuss as negative, major|1.71|1.34|6011003|edu packunivamalg #1|4|sports|9|Books|451|oughtantiese|N/A|15482344rose71601051|rose|Dozen|Unknown|12|bareingought| +181|AAAAAAAAFLAAAAAA|1997-10-27||||8.83|9007002|brandmaxi #2|||||395|antin stpri|||gainsboro||Unknown|44|oughteingought| +182|AAAAAAAAGLAAAAAA|1997-10-27|2000-10-26|Always full sources must assume now. Both annual grounds might |4.18|1.79|2001001|amalgimporto #1|1|accessories|2|Men|362|ablecallypri|petite|4470655798892sienna7|frosted|Ton|Unknown|38|ableeingought| +183|AAAAAAAAGLAAAAAA|2000-10-27||Always full sources must assume now. Both annual grounds might |1.71|1.21|2001001|importomaxi #4|1|guns|8|Sports|174|eseationought|N/A|4470655798892sienna7|violet|Gross|Unknown|88|prieingought| +184|AAAAAAAAILAAAAAA|1997-10-27|1999-10-27|Songs understand always in a components. Etc contemporary controls must plan together indeed dea|9.54|5.34|6009001|maxicorp #1|9|womens watch|6|Jewelry|134|esepriought|N/A|slate164703742328865|saddle|Ton|Unknown|38|eseeingought| +185|AAAAAAAAILAAAAAA|1999-10-28|2001-10-26|American women must achieve inevitably ethnic problems. Units send in a towns. Effectively traditional farms could not cause furthermore great stages. Wonderful machines|0.61|0.21|10002016|importounivamalg #16|2|camcorders|10|Electronics|482|esepriought|N/A|9700drab872791624666|steel|Ounce|Unknown|7|antieingought| +186|AAAAAAAAILAAAAAA|2001-10-27||American women must achieve inevitably ethnic problems. Units send in a towns. Effectively traditional farms could not cause furthermore great stages. Wonderful machines|5.49|2.58|5003001|exportischolar #1|3|pop|5|Music|755|antiantiation|N/A|88930037156lime16734|white|N/A|Unknown|32|callyeingought| +187|AAAAAAAALLAAAAAA|1997-10-27||Patients could learn then fund|0.79|0.66|8008008|namelessnameless #8|8|outdoor|8|Sports|357|ationantipri|N/A|58663296rosy34139329|red|Cup|Unknown|52|ationeingought| +188|AAAAAAAAMLAAAAAA|1997-10-27|2000-10-26|Central, principal men a|0.47|0.34|9015009|scholarunivamalg #9|15|fiction|9|Books|356|callyantipri|N/A|63676seashell2251108|peach|Oz|Unknown|30|eingeingought| +189|AAAAAAAAMLAAAAAA|2000-10-27||Central, principal men a|2.56|2.04|3003002|exportiexporti #2|15|toddlers|3|Children|976|callyantipri|medium|84067835920023slate9|ghost|Ounce|Unknown|18|n steingought| +190|AAAAAAAAOLAAAAAA|1997-10-27|1999-10-27|Concessions used to benefit there according to a duties; offic|0.56|0.33|1001001|amalgamalg #1|1|dresses|1|Women|46|callyese|extra large|9754006425189015hot8|snow|Lb|Unknown|61|barn stought| +191|AAAAAAAAOLAAAAAA|1999-10-28|2001-10-26|Earlier little men should not like. Demonstrations |7.98|0.33|9008012|namelessmaxi #12|1|romance|9|Books|71|callyese|N/A|66139473767red951496|medium|Tsp|Unknown|39|oughtn stought| +192|AAAAAAAAOLAAAAAA|2001-10-27||Earlier little men should not like. Demonstrations |7.27|0.33|9008012|exportiamalgamalg #6|1|stereo|10|Electronics|240|callyese|N/A|829182honeydew303134|white|N/A|Unknown|14|ablen stought| +193|AAAAAAAABMAAAAAA|1997-10-27||Young matches shall control about female years. Concerned, short sk|8.71|5.74|2003002|exportiimporto #2|3|pants|2|Men|315|antioughtpri|extra large|757241646950mint8541|linen|Dram|Unknown|43|prin stought| +194|AAAAAAAACMAAAAAA|1997-10-27|2000-10-26|Official candidates discount with a instruments. Very small colleagues might tear activities. Lines must cut as alone, inc policies. Relevant, used photographs shall pr|7.56|6.12|10005008|scholarunivamalg #8|5|karoke|10|Electronics|77|ationation|N/A|74002516navajo085083|spring|Dram|Unknown|48|esen stought| +195|AAAAAAAACMAAAAAA|2000-10-27||Official candidates discount with a instruments. Very small colleagues might tear activities. Lines must cut as alone, inc policies. Relevant, used photographs shall pr|0.64|6.12|1004002|edu packamalg #2|5|swimwear|1|Women|77|ationation|extra large|74002516navajo085083|pale|Gram|Unknown|22|antin stought| +196|AAAAAAAAEMAAAAAA|1997-10-27|1999-10-27|National citizens arrange totally ready, golden roads. Areas marry afraid, redundant pockets. Sections ought to know; feelings arise now revenues. A little great |3.56|1.13|2001001|amalgimporto #1|1|accessories|2|Men|959|n stantin st|petite|84808peach0512519858|tomato|Oz|Unknown|18|callyn stought| +197|AAAAAAAAEMAAAAAA|1999-10-28|2001-10-26|National citizens arrange totally ready, golden roads. Areas marry afraid, redundant pockets. Sections ought to know; feelings arise now revenues. A little great |8.75|1.13|4003002|exportiedu pack #2|3|kids|4|Shoes|959|n stantin st|extra large|5670yellow0545220029|sandy|Dram|Unknown|72|ationn stought| +198|AAAAAAAAEMAAAAAA|2001-10-27||National citizens arrange totally ready, golden roads. Areas marry afraid, redundant pockets. Sections ought to know; feelings arise now revenues. A little great |2.36|1.13|4003002|scholarcorp #7|5|earings|6|Jewelry|637|n stantin st|N/A|7tan3712622019776528|sandy|Tsp|Unknown|16|eingn stought| +199|AAAAAAAAHMAAAAAA|1997-10-27||Detectives must not assess international prisoners. Accidents amount further actions. Organis|2.18|1.67|3001002|amalgexporti #2|1|newborn|3|Children|534|eseprianti|large|72289240puff20224990|sandy|Each|Unknown|15|n stn stought| +200|AAAAAAAAIMAAAAAA|1997-10-27|2000-10-26|Discussions should not show so subsequently young stairs. Reasons would raise probably major sorts. Final, united attempts turn out of a parts. Rather small |3.46|1.86|1001001|amalgamalg #1|1|dresses|1|Women|40|barese|petite|134salmon13702979811|saddle|N/A|Unknown|36|barbarable| +201|AAAAAAAAIMAAAAAA|2000-10-27||Elsewhere free contacts would take well from a aspects. Trying, glad economies must begin equal emissions. Settlements used to sh|29.47|1.86|1001001|amalgscholar #2|1|rock|5|Music|40|barese|N/A|024570959751peru8082|white|Tbl|Unknown|3|oughtbarable| +202|AAAAAAAAKMAAAAAA|1997-10-27|1999-10-27|Plans consult interested, light boys. Selective, other problems create scientific, young parties. Sufficient speakers might not kiss too social, basic interests. Dual, other times s|0.19|0.11|9002011|importomaxi #11|2|business|9|Books|606|callybarcally|N/A|5680664776rose064391|tan|Carton|Unknown|8|ablebarable| +203|AAAAAAAAKMAAAAAA|1999-10-28|2001-10-26|Plans consult interested, light boys. Selective, other problems create scientific, young parties. Sufficient speakers might not kiss too social, basic interests. Dual, other times s|9.32|7.73|8010004|univmaxi #4|10|pools|8|Sports|606|callybarcally|N/A|16orchid959385650795|red|Tsp|Unknown|64|pribarable| +204|AAAAAAAAKMAAAAAA|2001-10-27||Legal fields choose subject miles; mutually profound markets avoid of course afterwards urban facts. Large boats set reall|0.76|0.44|8010004|exporticorp #5|3|gold|6|Jewelry|606|callybarcally|N/A|24262380725wheat3635|yellow|Ton|Unknown|88|esebarable| +205|AAAAAAAANMAAAAAA|1997-10-27||Over wide attacks agree i|7.30|5.98|9006002|corpmaxi #2|6|parenting|9|Books|555|antiantianti|N/A|1797605peach42656788|lemon|Each|Unknown|15|antibarable| +206|AAAAAAAAOMAAAAAA|1997-10-27|2000-10-26|Figures could not require. Joint feet should provide here realistic, serious characters. Brief friends could not avoid professionally british women. Altogether past sales will not p|4.22|3.67|4004001|edu packedu pack #1|4|athletic|4|Shoes|480|bareingese|medium|2270sienna9777537946|tan|Unknown|Unknown|49|callybarable| +207|AAAAAAAAOMAAAAAA|2000-10-27||Then wise terms cannot care no longer great ministers. Probably considerable prices provide shadows. New banks wo|3.14|3.67|10008004|namelessunivamalg #4|4|scanners|10|Electronics|179|bareingese|N/A|87500024salmon369387|spring|Tsp|Unknown|43|ationbarable| +208|AAAAAAAAANAAAAAA|1997-10-27|1999-10-27|Assets like as semantic children. At least accused elements shall force noisily years. So final payment|94.12|46.11|6009001|maxicorp #1|9|womens watch|6|Jewelry|231|oughtpriable|N/A|8821803328499pale740|cream|Dram|Unknown|20|eingbarable| +209|AAAAAAAAANAAAAAA|1999-10-28|2001-10-26|Cheerful daughters take on a shops. Far definite parties withdraw talks. Years quantify. Miraculously fast membe|5.62|3.48|2002002|importoimporto #2|9|shirts|2|Men|123|oughtpriable|petite|8821803328499pale740|thistle|Cup|Unknown|8|n stbarable| +210|AAAAAAAAANAAAAAA|2001-10-27||Beans should give; never happy|0.93|0.30|2002002|exportiamalg #1|9|maternity|1|Women|501|oughtpriable|small|8821803328499pale740|sky|Ton|Unknown|23|baroughtable| +211|AAAAAAAADNAAAAAA|1997-10-27||Equal police use then main solicitors. So small features accept very in a families. Relations breach either. Firms shall sell simply. Colleagues c|1.83|1.53|1002002|importoamalg #2|2|fragrances|1|Women|489|n steingese|large|9forest7812359600604|sienna|Lb|Unknown|8|oughtoughtable| +212|AAAAAAAAENAAAAAA|1997-10-27|2000-10-26|Forms end social tons. Level, only fruits establish alone more decisive differences. Also persistent wi|0.52|0.27|4002001|importoedu pack #1|2|mens|4|Shoes|328|eingablepri|petite|77416linen1913860202|blue|Ounce|Unknown|31|ableoughtable| +213|AAAAAAAAENAAAAAA|2000-10-27||Forms end social tons. Level, only fruits establish alone more decisive differences. Also persistent wi|0.10|0.27|5003002|exportischolar #2|3|pop|5|Music|328|eingablepri|N/A|169turquoise59304786|papaya|Each|Unknown|14|prioughtable| +214|AAAAAAAAGNAAAAAA|1997-10-27|1999-10-27|Largely fine activities complete professional, biological models. Likely, coming goals take ever unknown engines. Appropriate parts should join today liber|7.22|5.34|1003001|exportiamalg #1|3|maternity|1|Women|628|eingablecally|medium|3934382dodger2259382|tan|Cup|Unknown|64|eseoughtable| +215|AAAAAAAAGNAAAAAA|1999-10-28|2001-10-26|More short periods shall start simply united small devices. Authorities might let really much concerned subjects. Fat, only cases suggest j|9.77|5.34|1003001|amalgmaxi #6|11|archery|8|Sports|628|eingablecally|N/A|0027145403rose644589|peach|N/A|Unknown|24|antioughtable| +216|AAAAAAAAGNAAAAAA|2001-10-27||Royal women mind very equally sure prisons. Unlikely observations appreciate also into a pictures. Especially new children must not work|3.20|2.81|1003001|univbrand #1|10|jewelry boxes|6|Jewelry|298|eingablecally|N/A|96327lace57140766145|pale|Lb|Unknown|33|callyoughtable| +217|AAAAAAAAJNAAAAAA|1997-10-27||Different, expected readers offer in a changes. Pockets can make enough without the parents. Good, simple|3.09|1.32|1001002|amalgamalg #2|1|dresses|1|Women|130|barpriought|petite|30283blanched0306305|tan|Dram|Unknown|5|ationoughtable| +218|AAAAAAAAKNAAAAAA|1997-10-27|2000-10-26|So recent procedures ought to like mental pupils. Still young respects should perform now medium words. High cells will|0.21|0.17|4004001|edu packedu pack #1|4|athletic|4|Shoes|236|callypriable|medium|0458032992thistle030|violet|Carton|Unknown|21|eingoughtable| +219|AAAAAAAAKNAAAAAA|2000-10-27||Usually wonderful theories feel devices; electric views should know else. Lips expand in a numbers. Pieces can say. Major, english buildings will not|0.55|0.17|4004001|importoimporto #2|2|shirts|2|Men|236|callypriable|extra large|13806207burlywood178|red|Dram|Unknown|71|n stoughtable| +220|AAAAAAAAMNAAAAAA|1997-10-27|1999-10-27|Conservative areas ask there|1.58|0.96|3001001|amalgexporti #1|1|newborn|3|Children|47|ationese|N/A|52powder910392692673|olive|Unknown|Unknown|7|barableable| +221|AAAAAAAAMNAAAAAA|1999-10-28|2001-10-26|Increasing, major women will not combat so white, residential years. Also other forms take after every figures. Available, specific hills suffer bright, principal drawings. Intervi|6.03|0.96|3001001|edu packimporto #2|4|sports-apparel|2|Men|744|ationese|petite|52powder910392692673|bisque|Lb|Unknown|48|oughtableable| +222|AAAAAAAAMNAAAAAA|2001-10-27||Increasing, major women will not combat so white, residential years. Also other forms take after every figures. Available, specific hills suffer bright, principal drawings. Intervi|3.36|0.96|3001001|importoimporto #1|4|shirts|2|Men|599|ationese|petite|52powder910392692673|powder|Cup|Unknown|11|ableableable| +223|AAAAAAAAPNAAAAAA|1997-10-27||Regular, bad memories might|5.87|2.23|8008004|namelessnameless #4|8|outdoor|8|Sports|296|callyn stable|N/A|papaya58550929407290|saddle|Each|Unknown|76|priableable| +224|AAAAAAAAAOAAAAAA|1997-10-27|2000-10-26|Visitors could face very european, bottom officers. Alternative schools say more current women. Golden, s|1.67|0.65|1004001|edu packamalg #1|4|swimwear|1|Women|684|eseeingcally|medium|55165indian628143836|sienna|Dozen|Unknown|6|eseableable| +225|AAAAAAAAAOAAAAAA|2000-10-27||Large, strong issues increase moving, nuclear forces. Pp. pay barely experimental groups. Economic activities may grow initially areas. Mechanical|5.35|3.79|1004001|importomaxi #6|4|guns|8|Sports|628|eseeingcally|N/A|657113165yellow10813|sky|Each|Unknown|82|antiableable| +226|AAAAAAAACOAAAAAA|1997-10-27|1999-10-27|Probably general controls defend at least already powerful police. Particular supporters should not receive both residents. Secret procedures shall not take eyes. Desperat|3.19|2.52|6007003|brandcorp #3|7|pendants|6|Jewelry|52|ableanti|N/A|04salmon758173786010|purple|Tbl|Unknown|32|callyableable| +227|AAAAAAAACOAAAAAA|1999-10-28|2001-10-26|European fees shall not try so high, difficult symptoms. Too entire pp. disguise holidays. Thinly universal losses survive by the mo|10.44|3.34|6007003|corpunivamalg #12|16|mystery|9|Books|52|ableanti|N/A|04salmon758173786010|indian|Dozen|Unknown|43|ationableable| +228|AAAAAAAACOAAAAAA|2001-10-27||Particular effects will imply along a companie|1.51|3.34|2004001|edu packimporto #1|4|sports-apparel|2|Men|184|eseeingought|large|14209cornflower22193|salmon|Bunch|Unknown|83|eingableable| +229|AAAAAAAAFOAAAAAA|1997-10-27||Pink, continuous courts solve inevitably short future problems. Broad plans pass as a drawings. Only bad negotiations come|3.20|1.92|7005010|scholarbrand #10|5|blinds/shades|7|Home|223|priableable|N/A|6266415783white26017|orange|Gross|Unknown|61|n stableable| +230|AAAAAAAAGOAAAAAA|1997-10-27|2000-10-26|Subsequent, new studies can say. Familiar stairs smell equally in the patients. Days might exploit elsewhere double police. Women order only perha|8.08|6.22|5001001|amalgscholar #1|1|rock|5|Music|69|n stcally|N/A|02376976999329olive2|spring|Case|Unknown|42|barpriable| +231|AAAAAAAAGOAAAAAA|2000-10-27||Subsequent, new studies can say. Familiar stairs smell equally in the patients. Days might exploit elsewhere double police. Women order only perha|3.72|2.52|5001001|exportiamalg #2|1|maternity|1|Women|323|priablepri|medium|02376976999329olive2|yellow|Case|Unknown|67|oughtpriable| +232|AAAAAAAAIOAAAAAA|1997-10-27|1999-10-27|Earnestly russian computers approach happy days. Extreme, interesting blocks become sometimes high workers. F|6.93|5.61|10003012|exportiunivamalg #12|3|dvd/vcr players|10|Electronics|69|n stcally|N/A|8186powder0122252949|slate|Dozen|Unknown|34|ablepriable| +233|AAAAAAAAIOAAAAAA|1999-10-28|2001-10-26|Slower military faci|4.60|3.49|5004002|edu packscholar #2|3|classical|5|Music|69|n stcally|N/A|3098182plum647681885|pale|Cup|Unknown|15|pripriable| +234|AAAAAAAAIOAAAAAA|2001-10-27||Only industrial patients would not get small women. Special, long steps can control too hidden lists. Always top years need bizarre books. Chief, large problems cannot read from a profits|22.87|17.60|3004001|edu packexporti #1|3|school-uniforms|3|Children|66|callycally|extra large|3098182plum647681885|red|Case|Unknown|43|esepriable| +235|AAAAAAAALOAAAAAA|1997-10-27||Agents see companies. Weekly clergy might not enable always mere studies. Men throw possible relations. Then static rights wr|4.08|1.59|5001002|amalgscholar #2|1|rock|5|Music|787|ationeingation|N/A|88210042seashell0023|hot|Bundle|Unknown|35|antipriable| +236|AAAAAAAAMOAAAAAA|1997-10-27|2000-10-26|Sympathetically scottish things should take regularly on a programmes. Suitable, high stars could find above in a gains; wrong orders see for the speakers. English, grand groups shall not m|0.75|0.23|9008009|namelessmaxi #9|8|romance|9|Books|214|eseoughtable|N/A|70yellow905498868670|plum|Tsp|Unknown|31|callypriable| +237|AAAAAAAAMOAAAAAA|2000-10-27||Sympathetically scottish things should take regularly on a programmes. Suitable, high stars could find above in a gains; wrong orders see for the speakers. English, grand groups shall not m|3.03|1.69|3002002|importoexporti #2|2|infants|3|Children|214|eseoughtable|medium|411963lemon755338987|slate|Ounce|Unknown|34|ationpriable| +238|AAAAAAAAOOAAAAAA|1997-10-27|1999-10-27|High, japanese terms recapture far from tightly similar sections; widespread, romantic teeth shall sort so elabo|2.39|2.03|7014009|edu packnameless #9|14|glassware|7|Home|119|n stoughtought|N/A|7light11718169292149|rosy|Gross|Unknown|39|eingpriable| +239|AAAAAAAAOOAAAAAA|1999-10-28|2001-10-26|No longer national paren|84.29|2.03|9002012|importomaxi #12|14|business|9|Books|119|n stoughtought|N/A|3031996cornflower626|wheat|Oz|Unknown|51|n stpriable| +240|AAAAAAAAOOAAAAAA|2001-10-27||No longer national paren|95.10|85.59|8005001|scholarnameless #1|14|fitness|8|Sports|259|n stantiable|N/A|3031996cornflower626|maroon|Bunch|Unknown|26|bareseable| +241|AAAAAAAABPAAAAAA|1997-10-27||Other, public boys stay never various women. Intellectually divine effects draw v|6.32|5.30|1003002|exportiamalg #2|3|maternity|1|Women|186|callyeingought|large|00262puff78971986333|metallic|Each|Unknown|84|oughteseable| +242|AAAAAAAACPAAAAAA|1997-10-27|2000-10-26|Industrial, delighted sounds can kill further regional, personal vegetables; both real companies will experiment once minimum, overall leaders. Difficult, helpful supporters shoul|1.76|0.59|8001003|amalgnameless #3|1|athletic shoes|8|Sports|426|callyableese|N/A|3601995737rosy512874|orchid|Bundle|Unknown|17|ableeseable| +243|AAAAAAAACPAAAAAA|2000-10-27||Very precise periods know examples. Points should change inextricably eyes; walls fit suddenly senior parts. Main, palestinian tren|3.92|3.09|8001003|amalgimporto #2|1|accessories|2|Men|426|callyableese|small|3601995737rosy512874|lavender|Tbl|Unknown|62|prieseable| +244|AAAAAAAAEPAAAAAA|1997-10-27|1999-10-27|Little, inc qualities form facts. Common, late scholars mig|3.20|1.12|6002005|importocorp #5|2|diamonds|6|Jewelry|307|ationbarpri|N/A|367160281rose7379005|turquoise|Dram|Unknown|11|eseeseable| +245|AAAAAAAAEPAAAAAA|1999-10-28|2001-10-26|International, current lips used to forgive black components. Books need yet in a homes. Only whole pieces dry direct by a hours. Chinese, local changes should not understand further sole fortunes;|2.23|1.12|10012008|importoamalgamalg #8|2|monitors|10|Electronics|307|ationbarpri|N/A|75530904sky158511603|seashell|Ounce|Unknown|60|antieseable| +246|AAAAAAAAEPAAAAAA|2001-10-27||Available companies ought to distort utterly scottis|3.67|2.12|10012008|amalgbrand #7|2|semi-precious|6|Jewelry|307|ationbarpri|N/A|75530904sky158511603|tan|Bunch|Unknown|37|callyeseable| +247|AAAAAAAAHPAAAAAA|1997-10-27||Other boys see that amongst a words. Things used to suppose regulations. Sources e|7.85|2.59|10016010|corpamalgamalg #10|16|wireless|10|Electronics|729|n stableation|N/A|00394216orange366958|rose|Unknown|Unknown|62|ationeseable| +248|AAAAAAAAIPAAAAAA|1997-10-27|2000-10-26|Old streets would persuade also types. Still elderly reasons provide clean at a women. Cultural |5.38|4.41|10005011|scholarunivamalg #11|5|karoke|10|Electronics|224|eseableable|N/A|5salmon6226434803756|powder|Pound|Unknown|87|eingeseable| +249|AAAAAAAAIPAAAAAA|2000-10-27||Results shall reconcile so local, c|2.23|4.41|10005011|amalgscholar #2|1|rock|5|Music|1|ought|N/A|092smoke536051130494|puff|Cup|Unknown|4|n steseable| +250|AAAAAAAAKPAAAAAA|1997-10-27|1999-10-27|Effects ought|4.16|2.16|9010011|univunivamalg #11|10|travel|9|Books|353|priantipri|N/A|984olive300633578279|peach|Ton|Unknown|20|barantiable| +251|AAAAAAAAKPAAAAAA|1999-10-28|2001-10-26|Effects ought|2.33|2.16|8007002|brandnameless #2|7|hockey|8|Sports|353|priantipri|N/A|984olive300633578279|peach|Gross|Unknown|15|oughtantiable| +252|AAAAAAAAKPAAAAAA|2001-10-27||Effects ought|6.62|2.16|7012003|importonameless #3|7|paint|7|Home|353|priantipri|N/A|1062honeydew67027626|yellow|Each|Unknown|31|ableantiable| +253|AAAAAAAANPAAAAAA|1997-10-27||Now easy facilities provide yesterday nuclear effects. Women install however actually local propos|3.17|2.69|10012016|importoamalgamalg #16|12|monitors|10|Electronics|251|oughtantiable|N/A|839992358505lemon225|lemon|Carton|Unknown|33|priantiable| +254|AAAAAAAAOPAAAAAA|1997-10-27|2000-10-26|Areas might cry very whole occupations. Local minutes cannot attract more; social years might not maintain immediately left changes. In addition si|7.75|6.20|1004001|edu packamalg #1|4|swimwear|1|Women|128|eingableought|small|879seashell586609450|blue|Carton|Unknown|24|eseantiable| +255|AAAAAAAAOPAAAAAA|2000-10-27||Areas might cry very whole occupations. Local minutes cannot attract more; social years might not maintain immediately left changes. In addition si|2.33|6.20|1004001|exportiexporti #2|3|toddlers|3|Children|128|eingableought|petite|34pale53170605089834|ivory|Bundle|Unknown|8|antiantiable| +256|AAAAAAAAAABAAAAA|1997-10-27|1999-10-27|Always available requests defy ver|5.57|4.95|2004001|edu packimporto #1|4|sports-apparel|2|Men|234|esepriable|medium|41red155507640139571|navajo|Case|Unknown|23|callyantiable| +257|AAAAAAAAAABAAAAA|1999-10-28|2001-10-26|Always available requests defy ver|4.23|4.95|6013002|exportibrand #2|13|loose stones|6|Jewelry|294|esen stable|N/A|319241115299slate651|red|Unknown|Unknown|20|ationantiable| +258|AAAAAAAAAABAAAAA|2001-10-27||Obviously single friends may adopt really on the things. Rather secondary li|7.47|3.58|6013002|amalgexporti #1|1|newborn|3|Children|68|esen stable|medium|319241115299slate651|yellow|Box|Unknown|24|eingantiable| +259|AAAAAAAADABAAAAA|1997-10-27||Eyes might fire local laws; practical businessmen transport again essential players. Acts forgive then generally necessary standards; bills will c|3.88|3.22|6012004|importobrand #4|12|costume|6|Jewelry|723|priableation|N/A|235851red25618699233|olive|Bundle|Unknown|6|n stantiable| +260|AAAAAAAAEABAAAAA|1997-10-27|2000-10-26|Guidelines might spot at least organic, big artists. Detailed carers might debate olympi|8.91|7.03|5002001|importoscholar #1|2|country|5|Music|455|antiantiese|N/A|0845582565922violet7|white|Dram|Unknown|30|barcallyable| +261|AAAAAAAAEABAAAAA|2000-10-27||Guidelines might spot at least organic, big artists. Detailed carers might debate olympi|3.14|7.03|5002001|edu packunivamalg #7|2|audio|10|Electronics|185|antieingought|N/A|0845582565922violet7|plum|Tbl|Unknown|90|oughtcallyable| +262|AAAAAAAAGABAAAAA|1997-10-27|1999-10-27|Employees shall obtain later. So dead events learn further to a goods. Potential police write profoundly average directors. Young studies use above differences. Excell|5.27|2.58|2004001|edu packimporto #1|4|sports-apparel|2|Men|230|barpriable|large|6333lace143473078155|yellow|Box|Unknown|4|ablecallyable| +263|AAAAAAAAGABAAAAA|1999-10-28|2001-10-26|Employees shall obtain later. So dead events learn further to a goods. Potential police write profoundly average directors. Young studies use above differences. Excell|1.78|2.58|2004001|importoamalg #2|4|fragrances|1|Women|39|n stpri|large|5887saddle5097449732|olive|Bunch|Unknown|61|pricallyable| +264|AAAAAAAAGABAAAAA|2001-10-27||Now other hands coul|4.93|2.58|10014010|edu packamalgamalg #10|14|automotive|10|Electronics|133|n stpri|N/A|032670182indian71336|papaya|Carton|Unknown|40|esecallyable| +265|AAAAAAAAJABAAAAA|1997-10-27||Similar fingers could r|9.59|4.21|3001002|amalgexporti #2|1|newborn|3|Children|516|callyoughtanti|petite|5coral51548820574210|moccasin|Case|Unknown|22|anticallyable| +266|AAAAAAAAKABAAAAA|1997-10-27|2000-10-26|Groups may not find only for a |8.59|3.09|7013007|exportinameless #7|13|wallpaper|7|Home|655|antiantically|N/A|974seashell910073386|steel|Pallet|Unknown|58|callycallyable| +267|AAAAAAAAKABAAAAA|2000-10-27||Clearly new homes move. In order specialist men used to know yet large other users. New prices will see there about powerful things. Other expectations stan|3.19|1.75|2003002|exportiimporto #2|3|pants|2|Men|655|antiantically|large|627846427pink8073950|cornflower|Cup|Unknown|48|ationcallyable| +268|AAAAAAAAMABAAAAA|1997-10-27|1999-10-27|Small designs may not guide sure single things|3.73|2.38|9016005|corpunivamalg #5|16|mystery|9|Books|396|callyn stpri|N/A|4009723141488115red5|sandy|Case|Unknown|42|eingcallyable| +269|AAAAAAAAMABAAAAA|1999-10-28|2001-10-26|Small designs may not guide sure single things|7.67|2.38|3003002|exportiexporti #2|3|toddlers|3|Children|19|n stought|petite|4009723141488115red5|magenta|Carton|Unknown|6|n stcallyable| +270|AAAAAAAAMABAAAAA|2001-10-27||Again difficult things should start there; future, english offices make high, personal lives. Models |3.60|3.24|5001001|amalgscholar #1|1|rock|5|Music|75|antiation|N/A|4009723141488115red5|spring|Oz|Unknown|27|barationable| +271|AAAAAAAAPABAAAAA|1997-10-27||As academic officials woul|8.24|6.83|6006008|corpcorp #8|6|rings|6|Jewelry|617|ationoughtcally|N/A|70281007570tan428925|smoke|Cup|Unknown|55|oughtationable| +272|AAAAAAAAABBAAAAA|1997-10-27|2000-10-26|Ways share electronic benefits. Just effective groups repeat social relations. Always coming deaths would treat so ideas. Effective, grand patterns would hold more. Capable feet|1.71|0.73|7016003|corpnameless #3|16|furniture|7|Home|334|esepripri|N/A|29green5854310705548|beige|Carton|Unknown|25|ableationable| +273|AAAAAAAAABBAAAAA|2000-10-27||Ways share electronic benefits. Just effective groups repeat social relations. Always coming deaths would treat so ideas. Effective, grand patterns would hold more. Capable feet|1.09|0.73|8014004|edu packmaxi #4|14|tennis|8|Sports|659|n stantically|N/A|29green5854310705548|linen|Box|Unknown|12|priationable| +274|AAAAAAAACBBAAAAA|1997-10-27|1999-10-27|Guns provide changes. Ago new references used to accompany on the eyes. Forward supreme patients cannot ask real, spiritual channels. Interest|4.69|2.11|7016005|corpnameless #5|16|furniture|7|Home|24|eseable|N/A|33092941809lace45853|bisque|Tbl|Unknown|57|eseationable| +275|AAAAAAAACBBAAAAA|1999-10-28|2001-10-26|Future, relative combinations could see working, bright women. Appropriate conditions shift yet front accounts. Human, main prisoners permit. Years could concentrate human, high points. |2.87|2.11|4003002|exportiedu pack #2|3|kids|4|Shoes|337|ationpripri|extra large|33092941809lace45853|wheat|Tsp|Unknown|7|antiationable| +276|AAAAAAAACBBAAAAA|2001-10-27||Traditional laboratories kill brutally direct, alive police. Extra early assumptions respond delighted, ready chan|22.57|14.44|4003002|maximaxi #1|9|science|9|Books|49|ationpripri|N/A|33092941809lace45853|yellow|Bundle|Unknown|59|callyationable| +277|AAAAAAAAFBBAAAAA|1997-10-27||Short, black systems run. Delegates think up to a figures; employers will not|3.83|3.29|8006008|corpnameless #8|6|football|8|Sports|263|pricallyable|N/A|762dark9769961298169|pink|Dozen|Unknown|61|ationationable| +278|AAAAAAAAGBBAAAAA|1997-10-27|2000-10-26|Reports would influence windows. As grey relations make little national, wonderful folk. Even german personnel bring then. Rocks make far from the lips. So early men survive conseque|3.09|1.23|1002001|importoamalg #1|2|fragrances|1|Women|185|antieingought|extra large|17087260turquoise687|ghost|Tbl|Unknown|7|eingationable| +279|AAAAAAAAGBBAAAAA|2000-10-27||Frequently local years should see about to |78.10|1.23|1002001|corpbrand #8|2|consignment|6|Jewelry|185|antieingought|N/A|17087260turquoise687|khaki|Pound|Unknown|31|n stationable| +280|AAAAAAAAIBBAAAAA|1997-10-27|1999-10-27|Holidays let safely camps. Babies sm|1.94|1.24|1004001|edu packamalg #1|4|swimwear|1|Women|193|prin stought|large|8violet6417847423758|peru|Gross|Unknown|48|bareingable| +281|AAAAAAAAIBBAAAAA|1999-10-28|2001-10-26|Holidays let safely camps. Babies sm|6.58|5.06|1004001|importoedu pack #2|4|mens|4|Shoes|193|prin stought|small|96579605648seashell4|sienna|Bunch|Unknown|32|oughteingable| +282|AAAAAAAAIBBAAAAA|2001-10-27||Holidays let safely camps. Babies sm|9.01|5.06|7013003|exportinameless #3|13|wallpaper|7|Home|193|prin stought|N/A|286168813664slate039|burnished|Unknown|Unknown|21|ableeingable| +283|AAAAAAAALBBAAAAA|1997-10-27||New, labour players must start subsequently magnetic values. Dark problems laugh; accountants|9.13|4.56|7008004|namelessbrand #4|8|lighting|7|Home|2|able|N/A|89418417midnight3657|white|Pound|Unknown|26|prieingable| +284|AAAAAAAAMBBAAAAA|1997-10-27|2000-10-26|Now national objects shall not arise home rece|0.55|0.35|2001001|amalgimporto #1|1|accessories|2|Men|411|oughtoughtese|large|9335383387862violet3|seashell|Gram|Unknown|56|eseeingable| +285|AAAAAAAAMBBAAAAA|2000-10-27||Now national objects shall not arise home rece|1.62|1.36|2001001|brandmaxi #10|7|reference|9|Books|85|oughtoughtese|N/A|031570795176282puff1|tomato|Dram|Unknown|40|antieingable| +286|AAAAAAAAOBBAAAAA|1997-10-27|1999-10-27|Boundaries make however foreign days. Eventually new centres would not see well. Personally giant dreams represent services. Much perfect steps vis|1.21|0.65|8015007|scholarmaxi #7|15|fishing|8|Sports|158|eingantiought|N/A|96202757710seashell1|smoke|Cup|Unknown|36|callyeingable| +287|AAAAAAAAOBBAAAAA|1999-10-28|2001-10-26|Boundaries make however foreign days. Eventually new centres would not see well. Personally giant dreams represent services. Much perfect steps vis|2.68|0.65|8015007|corpmaxi #8|16|golf|8|Sports|516|eingantiought|N/A|96202757710seashell1|powder|Gram|Unknown|54|ationeingable| +288|AAAAAAAAOBBAAAAA|2001-10-27||Boundaries make however foreign days. Eventually new centres would not see well. Personally giant dreams represent services. Much perfect steps vis|3.60|0.65|3001001|amalgexporti #1|16|newborn|3|Children|557|eingantiought|small|96202757710seashell1|coral|Bundle|Unknown|26|eingeingable| +289|AAAAAAAABCBAAAAA|1997-10-27||Briefly foreign issues could play. Periods question fin|0.40|0.31|10001001|amalgunivamalg #1|1|cameras|10|Electronics|542|ableeseanti|N/A|8305205frosted981124|yellow|Dozen|Unknown|6|n steingable| +290|AAAAAAAACCBAAAAA|1997-10-27|2000-10-26|Eggs can let in a drivers. Various, black arrangements share so males. Typical, desirable pupils will expect as designs; especially direct claims might|4.58|1.37|1003001|exportiamalg #1|3|maternity|1|Women|117|ationoughtought|extra large|53960718559pale51050|peach|Ton|Unknown|12|barn stable| +291|AAAAAAAACCBAAAAA|2000-10-27||Eggs can let in a drivers. Various, black arrangements share so males. Typical, desirable pupils will expect as designs; especially direct claims might|1.92|1.37|1003001|edu packamalg #2|3|swimwear|1|Women|117|ationoughtought|petite|53960718559pale51050|misty|Tsp|Unknown|4|oughtn stable| +292|AAAAAAAAECBAAAAA|1997-10-27|1999-10-27|New, particular motives should not secure rules. Consistent users must|1.95|0.72|2003001|exportiimporto #1|3|pants|2|Men|788|eingeingation|large|3465890588773yellow0|pink|Pound|Unknown|25|ablen stable| +293|AAAAAAAAECBAAAAA|1999-10-28|2001-10-26|New, particular motives should not secure rules. Consistent users must|8.20|6.56|2003001|amalgexporti #2|1|newborn|3|Children|788|eingeingation|petite|3465890588773yellow0|papaya|Pound|Unknown|58|prin stable| +294|AAAAAAAAECBAAAAA|2001-10-27||New, particular motives should not secure rules. Consistent users must|6.13|2.26|2003001|exportiimporto #1|1|pants|2|Men|788|eingeingation|large|1393888290puff127353|tan|Dozen|Unknown|32|esen stable| +295|AAAAAAAAHCBAAAAA|1997-10-27||Available preparations ought to glimpse either from a prisoners; royal occasions camp at all cases. Short doctors like again symbolic children. Large, medical families crea|6.20|2.04|2002002|importoimporto #2|2|shirts|2|Men|183|prieingought|small|935968443832139rose8|puff|Ton|Unknown|75|antin stable| +296|AAAAAAAAICBAAAAA|1997-10-27|2000-10-26|Procedures can declare easy following positions. Men shall not comprise boring, active men. Far new magazines get. Other, happy p|4.30|3.31|3002001|importoexporti #1|2|infants|3|Children|441|oughteseese|medium|1852120716650peach70|sky|Bunch|Unknown|21|callyn stable| +297|AAAAAAAAICBAAAAA|2000-10-27||Legal, economic companies like thus in the eyes. Necessary others take to a areas. Silly, european teeth redress too american eyes. Legal, good pensions can get by a stands. E|5.35|3.31|4001002|amalgedu pack #2|2|womens|4|Shoes|441|oughteseese|large|59972peru45704554371|spring|Bunch|Unknown|32|ationn stable| +298|AAAAAAAAKCBAAAAA|1997-10-27|1999-10-27|Plants will build new, hot articles; students could cover high girls. Low services shall opt really businessmen. National, he|4.91|1.57|2004001|edu packimporto #1|4|sports-apparel|2|Men|15|antiought|medium|439cornflower7592388|purple|Case|Unknown|10|eingn stable| +299|AAAAAAAAKCBAAAAA|1999-10-28|2001-10-26|Plants will build new, hot articles; students could cover high girls. Low services shall opt really businessmen. National, he|0.26|1.57|2002002|importoimporto #2|4|shirts|2|Men|15|antiought|medium|hot36436602514455256|tomato|Unknown|Unknown|17|n stn stable| +300|AAAAAAAAKCBAAAAA|2001-10-27||Plants will build new, hot articles; students could cover high girls. Low services shall opt really businessmen. National, he|90.90|34.54|1003001|exportiamalg #1|4|maternity|1|Women|15|antiought|petite|523seashell329970040|green|Bundle|Unknown|64|barbarpri| +301|AAAAAAAANCBAAAAA|1997-10-27||As eastern pupils help new resources. Radical weeks should enable over circumstances; casual characteristics ought to manage yet anima|26.01|14.30|6007006|brandcorp #6|7|pendants|6|Jewelry|35|antipri|N/A|283350565698thistle7|peru|Dozen|Unknown|13|oughtbarpri| +302|AAAAAAAAOCBAAAAA|1997-10-27|2000-10-26|Soviet standards could not play never in a losses. Even lovely flats will not serve almost independent, social hours; most simple figures go huma|2.60|1.35|1004001|edu packamalg #1|4|swimwear|1|Women|286|callyeingable|medium|5tomato3854022030561|turquoise|Dram|Unknown|52|ablebarpri| +303|AAAAAAAAOCBAAAAA|2000-10-27||African markets could hit enough popular members. Black, widespread brothers keep quiet candidates; simple, special girls ought to want statutory instructions. Severe emp|4.38|2.14|1004001|scholarmaxi #4|4|history|9|Books|286|callyeingable|N/A|4609saddle1108631196|seashell|Bundle|Unknown|22|pribarpri| +304|AAAAAAAAADBAAAAA|1997-10-27|1999-10-27|Activities used to get firms. Positive resources would need too across a posts. Bad issues used to sit closely. Slightly true boundaries follow all model courses. Obvious methods attract little o|90.93|73.65|2003001|exportiimporto #1|3|pants|2|Men|143|prieseought|economy|228795928168green522|red|Dozen|Unknown|90|esebarpri| +305|AAAAAAAAADBAAAAA|1999-10-28|2001-10-26|Activities used to get firms. Positive resources would need too across a posts. Bad issues used to sit closely. Slightly true boundaries follow all model courses. Obvious methods attract little o|2.26|73.65|10011017|amalgamalgamalg #17|3|disk drives|10|Electronics|143|prieseought|N/A|743075royal751156817|sandy|Pallet|Unknown|3|antibarpri| +306|AAAAAAAAADBAAAAA|2001-10-27||Activities used to get firms. Positive resources would need too across a posts. Bad issues used to sit closely. Slightly true boundaries follow all model courses. Obvious methods attract little o|0.68|0.55|5004001|edu packscholar #1|4|classical|5|Music|143|prieseought|N/A|89778293violet502238|orchid|Lb|Unknown|43|callybarpri| +307|AAAAAAAADDBAAAAA|1997-10-27||Particularly current years cannot keep literally linguistic national opinions. Substantial, upper things could mention just from the workers. Organic years go never as cautious |9.48|8.15|5003002|exportischolar #2|3|pop|5|Music|88|eingeing|N/A|4246588saddle8225746|goldenrod|Oz|Unknown|7|ationbarpri| +308|AAAAAAAAEDBAAAAA|1997-10-27|2000-10-26|Other, still members ou|0.26|0.20|4003001|exportiedu pack #1|3|kids|4|Shoes|173|priationought|medium|93153154snow93437698|thistle|N/A|Unknown|60|eingbarpri| +309|AAAAAAAAEDBAAAAA|2000-10-27||Here tall shareholders must get|0.36|0.20|4003001|edu packimporto #2|4|sports-apparel|2|Men|301|oughtbarpri|medium|93153154snow93437698|metallic|Gram|Unknown|15|n stbarpri| +310|AAAAAAAAGDBAAAAA|1997-10-27|1999-10-27|Never impossible responses cannot register then interesting schools. Confident, useful requirements remember t|6.49|2.20|2004001|edu packimporto #1|4|sports-apparel|2|Men|264|esecallyable|medium|78orchid467210112384|rose|Carton|Unknown|24|baroughtpri| +311|AAAAAAAAGDBAAAAA|1999-10-28|2001-10-26|African minutes consider else critical, christian years. Normal, similar runs may not open even. Appearances shall not g|3.48|1.11|2004001|brandmaxi #12|4|reference|9|Books|264|esecallyable|N/A|chocolate18299585019|sienna|Oz|Unknown|58|oughtoughtpri| +312|AAAAAAAAGDBAAAAA|2001-10-27||African minutes consider else critical, christian years. Normal, similar runs may not open even. Appearances shall not g|9.50|5.22|2004001|edu packimporto #1|4|sports-apparel|2|Men|415|esecallyable|N/A|chocolate18299585019|slate|Gross|Unknown|10|ableoughtpri| +313|AAAAAAAAJDBAAAAA|1997-10-27||Schools check by a machines. Main units will involve hands. All large orders will not date so findings. Strongly brilliant roots reduce. Members bring behind a services. Surprisingly econom|98.44|67.92|6013002|exportibrand #2|13|loose stones|6|Jewelry|594|esen stanti|N/A|334black992910174063|purple|Dram|Unknown|9|prioughtpri| +314|AAAAAAAAKDBAAAAA|1997-10-27|2000-10-26|Distinct, sure years restart about open mere leaders. Mental, double families shall not die effecti|62.76|35.14|5002001|importoscholar #1|2|country|5|Music|111|oughtoughtought|N/A|56624705916moccasin1|beige|Dram|Unknown|11|eseoughtpri| +315|AAAAAAAAKDBAAAAA|2000-10-27||Distinct, sure years restart about open mere leaders. Mental, double families shall not die effecti|2.43|1.38|5002001|importoimporto #2|2|shirts|2|Men|111|oughtoughtought|small|violet82461888639517|rose|Ton|Unknown|100|antioughtpri| +316|AAAAAAAAMDBAAAAA|1997-10-27|1999-10-27|Clear|9.82|7.95|2003001|exportiimporto #1|3|pants|2|Men|177|ationationought|extra large|24582gainsboro043231|grey|Case|Unknown|15|callyoughtpri| +317|AAAAAAAAMDBAAAAA|1999-10-28|2001-10-26|Clear|5.46|7.95|10003012|exportiunivamalg #12|3|dvd/vcr players|10|Electronics|431|oughtpriese|N/A|294905882721thistle3|powder|Bunch|Unknown|5|ationoughtpri| +318|AAAAAAAAMDBAAAAA|2001-10-27||Clear|9.98|7.95|10003012|amalgamalgamalg #13|11|disk drives|10|Electronics|45|antiese|N/A|6970602612693saddle2|pale|Dozen|Unknown|88|eingoughtpri| +319|AAAAAAAAPDBAAAAA|1997-10-27||Cold, early wings mind like a columns. Women suffer; under new intervals come financial, level professionals. Countries shape. Of course international leg|0.45|0.33|8001010|amalgnameless #10|1|athletic shoes|8|Sports|453|priantiese|N/A|midnight338864388518|puff|Case|Unknown|63|n stoughtpri| +320|AAAAAAAAAEBAAAAA|1997-10-27|2000-10-26|Also general services could know brown colonies. Long, determined reasons can care|7.37|3.16|5004001|edu packscholar #1|4|classical|5|Music|409|n stbarese|N/A|05384750pale15935643|papaya|Gross|Unknown|26|barablepri| +321|AAAAAAAAAEBAAAAA|2000-10-27||Groups ought to control only neut|5.38|2.58|3003002|exportiexporti #2|3|toddlers|3|Children|889|n steingeing|N/A|05384750pale15935643|misty|Cup|Unknown|32|oughtablepri| +322|AAAAAAAACEBAAAAA|1997-10-27|1999-10-27|Grounds freeze previously then full rises. Familiar, big activities require as across a attitudes. Nuclear movies gain t|8.74|6.64|6015003|scholarbrand #3|15|custom|6|Jewelry|158|eingantiought|N/A|002lace0527448183262|almond|Oz|Unknown|17|ableablepri| +323|AAAAAAAACEBAAAAA|1999-10-28|2001-10-26|Somehow similar prisoners forget at a companies. Points matter; small, isolated studies travel necessarily that chri|5.72|6.64|4001002|amalgedu pack #2|1|womens|4|Shoes|810|eingantiought|medium|1navajo0970608667596|floral|Tbl|Unknown|32|priablepri| +324|AAAAAAAACEBAAAAA|2001-10-27||Somehow similar prisoners forget at a companies. Points matter; small, isolated studies travel necessarily that chri|0.83|0.31|4001002|importomaxi #1|2|business|9|Books|810|eingantiought|N/A|3656sandy94638381804|lime|Pallet|Unknown|72|eseablepri| +325|AAAAAAAAFEBAAAAA|1997-10-27||Matters outweigh again for the minutes; only international weeks should win backwards professional, dependent events; just slow shows create above a forests; full, subject improvements will not|0.76|0.66|2004002|edu packimporto #2|4|sports-apparel|2|Men|343|priesepri|medium|8798636014316yellow4|wheat|Unknown|Unknown|49|antiablepri| +326|AAAAAAAAGEBAAAAA|1997-10-27|2000-10-26|Social, black comments diminish|5.74|3.09|10014004|edu packamalgamalg #4|14|automotive|10|Electronics|26|callyable|N/A|19373salmon313559845|slate|Tbl|Unknown|60|callyablepri| +327|AAAAAAAAGEBAAAAA|2000-10-27||Rather c|44.99|35.09|10014004|namelesscorp #8|14|mens watch|6|Jewelry|26|callyable|N/A|79899royal4024043247|peru|Bundle|Unknown|22|ationablepri| +328|AAAAAAAAIEBAAAAA|1997-10-27|1999-10-27|Children wake ever long silent lawyers. Dramatic, wide hands stir here. Fresh bags preserve perhaps voluntary cl|6.76|5.67|1003001|exportiamalg #1|3|maternity|1|Women|555|antiantianti|medium|039663rosy8628423037|goldenrod|Oz|Unknown|78|eingablepri| +329|AAAAAAAAIEBAAAAA|1999-10-28|2001-10-26|Children wake ever long silent lawyers. Dramatic, wide hands stir here. Fresh bags preserve perhaps voluntary cl|0.65|5.67|2001002|amalgimporto #2|3|accessories|2|Men|13|antiantianti|medium|saddle10797969894459|red|Dozen|Unknown|11|n stablepri| +330|AAAAAAAAIEBAAAAA|2001-10-27||Big, national p|3.65|5.67|3003001|exportiexporti #1|3|toddlers|3|Children|13|priought|small|saddle10797969894459|plum|Pallet|Unknown|18|barpripri| +331|AAAAAAAALEBAAAAA|1997-10-27||Months may occur thus modern, inner patients. Political penalties put events. Forms used to win in particular; areas c|5.37|4.51|3001002|amalgexporti #2|1|newborn|3|Children|780|bareingation|medium|969448158986646tan50|tomato|N/A|Unknown|2|oughtpripri| +332|AAAAAAAAMEBAAAAA|1997-10-27|2000-10-26|Able issues find sure equal practices. Simply familiar patterns contain hours. |0.12|0.07|2001001|amalgimporto #1|1|accessories|2|Men|285|antieingable|N/A|079sky63295894306379|plum|Cup|Unknown|20|ablepripri| +333|AAAAAAAAMEBAAAAA|2000-10-27||Police start there as likely personnel; teams intervene only pieces; qualities cope. Highly industrial students matter to a |5.82|0.07|9009010|maximaxi #10|1|science|9|Books|503|pribaranti|N/A|8666seashell93279400|midnight|Gram|Unknown|18|pripripri| +334|AAAAAAAAOEBAAAAA|1997-10-27|1999-10-27|Men decide also white rates. Established positions draw at all ch|1.94|0.79|7015005|scholarnameless #5|15|tables|7|Home|329|n stablepri|N/A|smoke746380659562992|indian|Cup|Unknown|4|esepripri| +335|AAAAAAAAOEBAAAAA|1999-10-28|2001-10-26|Also difficult symptoms wait here runs. At present|1.10|0.82|9011012|amalgunivamalg #12|11|cooking|9|Books|329|n stablepri|N/A|smoke746380659562992|papaya|Carton|Unknown|24|antipripri| +336|AAAAAAAAOEBAAAAA|2001-10-27||Employers give barely rivers. Pro|3.23|0.82|2003001|exportiimporto #1|3|pants|2|Men|329|n stablepri|extra large|smoke746380659562992|turquoise|Dram|Unknown|11|callypripri| +337|AAAAAAAABFBAAAAA|1997-10-27||Sales cannot write commonly in a w|9.37|4.87|5004002|edu packscholar #2|4|classical|5|Music|369|n stcallypri|N/A|709326608321red11366|sky|Bunch|Unknown|64|ationpripri| +338|AAAAAAAACFBAAAAA|1997-10-27|2000-10-26|Measures get at a hopes. Clients complete arms. Front activities might bring sharp, medical areas. Fixed unions ought to limit later in the police. Workers allow soon dominant, physical meetings. N|3.51|1.54|4004001|edu packedu pack #1|4|athletic|4|Shoes|504|esebaranti|economy|101226947purple68839|cornsilk|Ton|Unknown|31|eingpripri| +339|AAAAAAAACFBAAAAA|2000-10-27||Measures get at a hopes. Clients complete arms. Front activities might bring sharp, medical areas. Fixed unions ought to limit later in the police. Workers allow soon dominant, physical meetings. N|4.73|1.54|8008010|namelessnameless #10|4|outdoor|8|Sports|167|ationcallyought|N/A|101226947purple68839|thistle|Bunch|Unknown|2|n stpripri| +340|AAAAAAAAEFBAAAAA|1997-10-27|1999-10-27|Important thousands look children. So|4.18|3.21|1004001|edu packamalg #1|4|swimwear|1|Women|32|ablepri|small|5lemon32588227874485|rosy|Gross|Unknown|14|baresepri| +341|AAAAAAAAEFBAAAAA|1999-10-28|2001-10-26|Systems purchase at all on a years. Major, prominent numbers ought to refuse of course broad systems. To|3.87|3.21|1004001|exportiunivamalg #6|13|self-help|9|Books|32|ablepri|N/A|5lemon32588227874485|violet|Each|Unknown|18|oughtesepri| +342|AAAAAAAAEFBAAAAA|2001-10-27||Systems purchase at all on a years. Major, prominent numbers ought to refuse of course broad systems. To|4.08|2.28|1004001|exportiunivamalg #3|13|dvd/vcr players|10|Electronics|32|ablepri|N/A|835gainsboro56466674|pale|Bunch|Unknown|23|ableesepri| +343|AAAAAAAAHFBAAAAA|1997-10-27||Old places avoid certain, typical hands; here original arms see in a ideas. Good|38.26|22.19|9007008|brandmaxi #8|7|reference|9|Books|614|eseoughtcally|N/A|1pale830779050741780|white|Case|Unknown|32|priesepri| +344|AAAAAAAAIFBAAAAA|1997-10-27|2000-10-26|Political, commercial expectations could say services. Royal, successful lists must not give areas. Feelings labour to a plans. Unique|8.32|4.57|5001001|amalgscholar #1|1|rock|5|Music|184|eseeingought|N/A|1seashell51407533385|rosy|Dozen|Unknown|27|eseesepri| +345|AAAAAAAAIFBAAAAA|2000-10-27||Ye|4.06|2.96|7014006|edu packnameless #6|14|glassware|7|Home|184|eseeingought|N/A|1seashell51407533385|peach|Unknown|Unknown|6|antiesepri| +346|AAAAAAAAKFBAAAAA|1997-10-27|1999-10-27|Disabled police require also guilty points. Upper drivers shall feed right social brothers. Elderly parties feel virtually. Others could reme|6.09|4.32|6009003|maxicorp #3|9|womens watch|6|Jewelry|172|ableationought|N/A|3smoke82712010565933|maroon|Unknown|Unknown|30|callyesepri| +347|AAAAAAAAKFBAAAAA|1999-10-28|2001-10-26|Disabled police require also guilty points. Upper drivers shall feed right social brothers. Elderly parties feel virtually. Others could reme|31.32|26.62|8005008|scholarnameless #8|5|fitness|8|Sports|137|ableationought|N/A|0red6848718347017127|rose|Tbl|Unknown|7|ationesepri| +348|AAAAAAAAKFBAAAAA|2001-10-27||Natural goals prevent less now alone others. Issues catch homes. Men write w|90.04|26.62|4001001|amalgedu pack #1|1|womens|4|Shoes|137|ableationought|extra large|58467866yellow993051|lawn|Box|Unknown|10|eingesepri| +349|AAAAAAAANFBAAAAA|1997-10-27||Championships might want always almost civil metres. At least great thousands commit. Wo|0.63|0.22|3004002|edu packexporti #2|4|school-uniforms|3|Children|958|eingantin st|extra large|690343orange51757700|thistle|Ounce|Unknown|2|n stesepri| +350|AAAAAAAAOFBAAAAA|1997-10-27|2000-10-26|Real, correct drinks deny carefully. Good subjects shall not contribute home highly mediterranean ideas; whole workers should affect by a dishes. Eyes can believe productive, total eyes. Databa|3.10|2.04|9016003|corpunivamalg #3|16|mystery|9|Books|746|callyeseation|N/A|55866893532789tan142|puff|Box|Unknown|72|barantipri| +351|AAAAAAAAOFBAAAAA|2000-10-27||Real, correct drinks deny carefully. Good subjects shall not contribute home highly mediterranean ideas; whole workers should affect by a dishes. Eyes can believe productive, total eyes. Databa|4.21|1.81|9016003|edu packunivamalg #12|4|audio|10|Electronics|182|ableeingought|N/A|55866893532789tan142|purple|N/A|Unknown|5|oughtantipri| +352|AAAAAAAAAGBAAAAA|1997-10-27|1999-10-27|Different, rare effects should not treat. Possible, able techniques shall show again arms. Requirements sh|0.97|0.85|6015001|scholarbrand #1|15|custom|6|Jewelry|284|eseeingable|N/A|993877577805630pale3|tan|Pallet|Unknown|31|ableantipri| +353|AAAAAAAAAGBAAAAA|1999-10-28|2001-10-26|Different, rare effects should not treat. Possible, able techniques shall show again arms. Requirements sh|4.42|2.21|7015004|scholarnameless #4|15|tables|7|Home|284|eseeingable|N/A|9slate13184970061872|seashell|Unknown|Unknown|25|priantipri| +354|AAAAAAAAAGBAAAAA|2001-10-27||Shadows can monitor techn|3.04|2.12|7003005|exportibrand #5|15|kids|7|Home|708|eseeingable|N/A|9slate13184970061872|magenta|Case|Unknown|44|eseantipri| +355|AAAAAAAADGBAAAAA|1997-10-27||Alive sources should drive upstairs interests. Elections shall meet severe, controversial lines. Central, full provisions determine as large officials; neatly select sources come; pregnant, wooden |8.62|3.79|5001002|amalgscholar #2|1|rock|5|Music|61|oughtcally|N/A|0purple1838414979942|lawn|Bundle|Unknown|49|antiantipri| +356|AAAAAAAAEGBAAAAA|1997-10-27|2000-10-26|So clear layers mean by the losses. Rules will seem now with a companies. Successful, corporate reasons might knock then final resources. C|7.82|6.25|7010007|univnameless #7|10|flatware|7|Home|361|oughtcallypri|N/A|73turquoise266313335|wheat|Pallet|Unknown|32|callyantipri| +357|AAAAAAAAEGBAAAAA|2000-10-27||Solid, preliminary weeks might import geographical, comparative sheets. Creditors should not pay possibly; children get even with a examples. Inadequate producers w|9.71|6.25|7010007|importoimporto #2|10|shirts|2|Men|456|oughtcallypri|large|8turquoise6768994616|slate|Each|Unknown|77|ationantipri| +358|AAAAAAAAGGBAAAAA|1997-10-27|1999-10-27|Patiently full features speculate heavily in the r|3.75|1.91|8012009|importomaxi #9|12|guns|8|Sports|240|bareseable|N/A|4409550turquoise1911|puff|Unknown|Unknown|3|eingantipri| +359|AAAAAAAAGGBAAAAA|1999-10-28|2001-10-26|Patiently full features speculate heavily in the r|4.94|1.91|2002002|importoimporto #2|12|shirts|2|Men|240|bareseable|petite|4409550turquoise1911|rose|Tbl|Unknown|19|n stantipri| +360|AAAAAAAAGGBAAAAA|2001-10-27||Certainly scottish applications shall send enough red, social grounds. Smooth |1.22|0.93|2002002|importoscholar #1|12|country|5|Music|240|bareseable|N/A|4409550turquoise1911|saddle|Cup|Unknown|61|barcallypri| +361|AAAAAAAAJGBAAAAA|1997-10-27||Strong tears should observe perhaps toxic recordings; accordingly new children turn on the mon|4.49|2.96|1002002|importoamalg #2|2|fragrances|1|Women|50|baranti|economy|30906mint30202923348|violet|Box|Unknown|8|oughtcallypri| +362|AAAAAAAAKGBAAAAA|1997-10-27|2000-10-26|Light moments cannot date following sy|5.60|4.92|9001003|amalgmaxi #3|1|arts|9|Books|64|esecally|N/A|86604248470sky818629|snow|Case|Unknown|21|ablecallypri| +363|AAAAAAAAKGBAAAAA|2000-10-27||Skills compromise responsible, conscious words. Totally tory faces used to take silently difficult required steps. Open intentions take by a rules.|4.46|3.79|9001003|amalgedu pack #2|1|womens|4|Shoes|64|esecally|petite|23142smoke8603747755|peru|Case|Unknown|23|pricallypri| +364|AAAAAAAAMGBAAAAA|1997-10-27|1999-10-27|Home certain acts adopt then new women. Statements reinforce thus mainly new rates. Real, other men must find. Late new children should not achieve years. Extr|8.58|6.34|9010005|univunivamalg #5|10|travel|9|Books|247|ationeseable|N/A|5903633868467slate25|sandy|Cup|Unknown|98|esecallypri| +365|AAAAAAAAMGBAAAAA|1999-10-28|2001-10-26|Home certain acts adopt then new women. Statements reinforce thus mainly new rates. Real, other men must find. Late new children should not achieve years. Extr|1.81|1.04|9010005|amalgscholar #2|1|rock|5|Music|411|oughtoughtese|N/A|5903633868467slate25|burlywood|Tbl|Unknown|44|anticallypri| +366|AAAAAAAAMGBAAAAA|2001-10-27||Entire, spiritual children implement purposes. Generally white schools deal just inc, theoretical years. Quite regular investors continue blue charges. Efficient, little hours send. Patients infor|3.69|1.54|5003001|exportischolar #1|3|pop|5|Music|411|oughtoughtese|N/A|721royal160887010702|pink|N/A|Unknown|26|callycallypri| +367|AAAAAAAAPGBAAAAA|1997-10-27||Excellent, real advantages would exist posts. Activities shall continue in a feet. Effects think only confidently local c|2.41|1.90|10001011|amalgunivamalg #11|1|cameras|10|Electronics|41|oughtese|N/A|139661661spring64224|khaki|Pallet|Unknown|39|ationcallypri| +368|AAAAAAAAAHBAAAAA|1997-10-27|2000-10-26|Social plans shall tell currently active demands. Bright, new perso|22.00|18.70|4003001|exportiedu pack #1|3|kids|4|Shoes|364|esecallypri|small|11560944yellow562058|goldenrod|Ounce|Unknown|63|eingcallypri| +369|AAAAAAAAAHBAAAAA|2000-10-27||Legal, grim findings|8.74|18.70|8009010|maxinameless #10|9|optics|8|Sports|585|antieinganti|N/A|4963050997687sky3040|tomato|Lb|Unknown|10|n stcallypri| +370|AAAAAAAACHBAAAAA|1997-10-27|1999-10-27|Persons would not tell talks; no doubt internal criteria see totally t|2.13|1.32|8007001|brandnameless #1|7|hockey|8|Sports|519|n stoughtanti|N/A|67185tan213520439951|snow|Gross|Unknown|31|barationpri| +371|AAAAAAAACHBAAAAA|1999-10-28|2001-10-26|Large, available pupils want bonds. New times will guess. Wrong genuine troops find necessarily difficult deposits. High children may not shout only with a shoulders. Hist|2.37|1.32|7010002|univnameless #2|7|flatware|7|Home|652|n stoughtanti|N/A|67185tan213520439951|blanched|Each|Unknown|22|oughtationpri| +372|AAAAAAAACHBAAAAA|2001-10-27||Other things refer central results. Sizes ought to free ready costs. Used, u|8.87|1.32|1001001|amalgamalg #1|1|dresses|1|Women|652|n stoughtanti|large|67185tan213520439951|saddle|Cup|Unknown|17|ableationpri| +373|AAAAAAAAFHBAAAAA|1997-10-27||Marvellous, useful pupils could ask remaining police; remarkable roots|13.01|11.31|1004002|edu packamalg #2|4|swimwear|1|Women|291|oughtn stable|medium|03902783pink08384973|pale|Box|Unknown|19|priationpri| +374|AAAAAAAAGHBAAAAA|1997-10-27|2000-10-26|Doubts could not think. Acres shall live tired ways. Obvious interests pay seldom severe images. Quick officials should no|8.82|4.67|7011005|amalgnameless #5|11|accent|7|Home|599|n stn stanti|N/A|snow5592168387360599|magenta|Unknown|Unknown|21|eseationpri| +375|AAAAAAAAGHBAAAAA|2000-10-27||Doubts could not think. Acres shall live tired ways. Obvious interests pay seldom severe images. Quick officials should no|1.28|1.08|10010002|univamalgamalg #2|10|memory|10|Electronics|21|n stn stanti|N/A|370490217683cream740|moccasin|Oz|Unknown|5|antiationpri| +376|AAAAAAAAIHBAAAAA|1997-10-27|1999-10-27|Real techniques intend by the fields; mothers interpret aut|4.10|1.88|3001001|amalgexporti #1|1|newborn|3|Children|475|antiationese|petite|2020091621657slate66|honeydew|Ounce|Unknown|25|callyationpri| +377|AAAAAAAAIHBAAAAA|1999-10-28|2001-10-26|Real techniques intend by the fields; mothers interpret aut|4.49|3.50|3001001|exportiimporto #2|3|pants|2|Men|539|n stprianti|medium|845450899108427lime4|lavender|Tsp|Unknown|43|ationationpri| +378|AAAAAAAAIHBAAAAA|2001-10-27||Real techniques intend by the fields; mothers interpret aut|1.05|3.50|3001001|corpunivamalg #5|6|musical|10|Electronics|301|oughtbarpri|N/A|845450899108427lime4|smoke|Case|Unknown|60|eingationpri| +379|AAAAAAAALHBAAAAA|1997-10-27||Sole, m|9.68|4.16|10013006|exportiamalgamalg #6|13|stereo|10|Electronics|149|n steseought|N/A|748591925966misty584|turquoise|Pallet|Unknown|20|n stationpri| +380|AAAAAAAAMHBAAAAA|1997-10-27|2000-10-26|Women begin. Workers ought to transform|2.36|1.08|5003001|exportischolar #1|3|pop|5|Music|361|oughtcallypri|N/A|4604512smoke46280454|sky|Dozen|Unknown|18|bareingpri| +381|AAAAAAAAMHBAAAAA|2000-10-27||Values operate nearer unknown years. Windows could deposit on a movies. Too large students adopt activities. Policies shall try dark, distinctive laws; later sexual investments d|5.19|4.20|5003001|amalgscholar #2|1|rock|5|Music|538|eingprianti|N/A|7840645470yellow2510|saddle|Unknown|Unknown|96|oughteingpri| +382|AAAAAAAAOHBAAAAA|1997-10-27|1999-10-27|Well empty components think furthermore romantic functions. Adjacent signs can show behind a pounds. Conscious contracts work never in the professionals. Only years could not say both small worke|7.32|3.14|1002001|importoamalg #1|2|fragrances|1|Women|3|pri|small|2454229sienna9307045|rose|Carton|Unknown|16|ableeingpri| +383|AAAAAAAAOHBAAAAA|1999-10-28|2001-10-26|New, real signs could not provide. Short soldiers mig|0.39|0.26|1002001|corpbrand #4|2|rugs|7|Home|3|pri|N/A|918365793thistle9671|mint|Tbl|Unknown|21|prieingpri| +384|AAAAAAAAOHBAAAAA|2001-10-27||New, real signs could not provide. Short soldiers mig|71.83|0.26|1002001|amalgedu pack #1|1|womens|4|Shoes|822|pri|extra large|918365793thistle9671|sienna|Each|Unknown|5|eseeingpri| +385|AAAAAAAABIBAAAAA|1997-10-27||Gradually new problems provide. Only complex years mention bitterly important guests. Sad times cannot belong merchants. Personal books could not i|5.21|4.16|3003002|exportiexporti #2|3|toddlers|3|Children|169|n stcallyought|medium|952061248878tan10619|hot|Bundle|Unknown|47|antieingpri| +386|AAAAAAAACIBAAAAA|1997-10-27|2000-10-26|Surely specific clubs must remember necessary, big duties. There final words learn practically standard lands. Private, clear companies must see|4.94|3.01|9005003|scholarmaxi #3|5|history|9|Books|605|antibarcally|N/A|9peach54880588951034|ivory|Bunch|Unknown|7|callyeingpri| +387|AAAAAAAACIBAAAAA|2000-10-27||Surely specific clubs must remember necessary, big duties. There final words learn practically standard lands. Private, clear companies must see|4.00|3.00|9005003|importoedu pack #2|5|mens|4|Shoes|605|antibarcally|medium|9peach54880588951034|slate|Cup|Unknown|24|ationeingpri| +388|AAAAAAAAEIBAAAAA|1997-10-27|1999-10-27|Then human details|0.82|0.50|9012005|importounivamalg #5|12|home repair|9|Books|929|n stablen st|N/A|5513366788864steel65|rose|Bunch|Unknown|20|eingeingpri| +389|AAAAAAAAEIBAAAAA|1999-10-28|2001-10-26|Well considerable princi|2.82|1.12|9012005|amalgunivamalg #16|12|cameras|10|Electronics|780|n stablen st|N/A|211smoke182293401788|dim|Gross|Unknown|36|n steingpri| +390|AAAAAAAAEIBAAAAA|2001-10-27||Well considerable princi|7.13|1.12|10016017|corpamalgamalg #17|12|wireless|10|Electronics|780|bareingation|N/A|894517455violet60482|hot|Ton|Unknown|18|barn stpri| +391|AAAAAAAAHIBAAAAA|1997-10-27||Suddenly quiet difficulties prove again actively new issues. Uniquely interested bases used to end enough contacts. Old, extraordina|7.66|4.59|5004002|edu packscholar #2|4|classical|5|Music|649|n stesecally|N/A|97906plum83663700362|tan|Ton|Unknown|12|oughtn stpri| +392|AAAAAAAAIIBAAAAA|1997-10-27|2000-10-26|Figures cannot make mere|1.64|0.83|5001001|amalgscholar #1|1|rock|5|Music|376|callyationpri|N/A|5070710721indian1995|rosy|N/A|Unknown|21|ablen stpri| +393|AAAAAAAAIIBAAAAA|2000-10-27||Figures cannot make mere|1.76|0.83|5001001|importoexporti #2|1|infants|3|Children|250|barantiable|petite|5070710721indian1995|peach|Ton|Unknown|37|prin stpri| +394|AAAAAAAAKIBAAAAA|1997-10-27|1999-10-27|Wholly dangerous schools say very local men. Dark methods forget other things; over welcome lists must allow for a men. Prime expla|7.61|6.62|3003001|exportiexporti #1|3|toddlers|3|Children|103|pribarought|medium|114022618467sandy005|peru|N/A|Unknown|66|esen stpri| +395|AAAAAAAAKIBAAAAA|1999-10-28|2001-10-26|Wholly dangerous schools say very local men. Dark methods forget other things; over welcome lists must allow for a men. Prime expla|64.97|6.62|6004004|edu packcorp #4|4|bracelets|6|Jewelry|103|pribarought|N/A|cornsilk068310017635|rose|Ton|Unknown|20|antin stpri| +396|AAAAAAAAKIBAAAAA|2001-10-27||Equally positive reasons should win civil worlds. Major women play sharply just parents. Men crawl for a members; debts might allow ev|0.61|6.62|6004004|amalgedu pack #1|1|womens|4|Shoes|103|pribarought|extra large|6842036630puff888587|peach|Case|Unknown|4|callyn stpri| +397|AAAAAAAANIBAAAAA|1997-10-27||Beneath decent wives write t|2.72|1.16|9001002|amalgmaxi #2|1|arts|9|Books|571|oughtationanti|N/A|8025633452269red0108|moccasin|Dozen|Unknown|17|ationn stpri| +398|AAAAAAAAOIBAAAAA|1997-10-27|2000-10-26|Purposes could give either nuclear fields. Initially extreme dreams go political, possible rights; existing shares should e|0.81|0.49|3004001|edu packexporti #1|4|school-uniforms|3|Children|862|ablecallyeing|large|3558235527slate61532|tan|Ounce|Unknown|79|eingn stpri| +399|AAAAAAAAOIBAAAAA|2000-10-27||Meals used to pull more. Cases can like more to a times. Local women ought to know much problems; russian, big persons cannot understand very papers. Fortunately raw observers op|6.48|0.49|3002002|importoexporti #2|2|infants|3|Children|845|antieseeing|medium|915473706turquoise42|peach|Tsp|Unknown|17|n stn stpri| +400|AAAAAAAAAJBAAAAA|1997-10-27|1999-10-27|Close, junior attitudes find. Internal, good areas promote necessary, economic revenues. Speakers can note much with a systems. Automatically central laws may work also from a p|1.24|0.84|4003001|exportiedu pack #1|3|kids|4|Shoes|88|eingeing|extra large|78283938smoke1853613|violet|Case|Unknown|10|barbarese| +401|AAAAAAAAAJBAAAAA|1999-10-28|2001-10-26|Inexorably criminal resources mention successful winners. Soft trends suppose|4.79|1.96|4003001|edu packimporto #2|4|sports-apparel|2|Men|88|eingeing|extra large|78283938smoke1853613|sandy|Oz|Unknown|30|oughtbarese| +402|AAAAAAAAAJBAAAAA|2001-10-27||Public, separate waters may travel very; eyes must work later to the fans. Well southern types could enter atlantic exa|9.01|1.96|8008003|namelessnameless #3|4|outdoor|8|Sports|88|eingeing|N/A|54370670648metallic2|yellow|Case|Unknown|15|ablebarese| +403|AAAAAAAADJBAAAAA|1997-10-27||National forms would not save very deliberate drugs. Cheaply particu|4.35|3.00|5002002|importoscholar #2|2|country|5|Music|792|ablen station|N/A|26white3881494347763|sky|Carton|Unknown|30|pribarese| +404|AAAAAAAAEJBAAAAA|1997-10-27|2000-10-26|Various hours wor|2.52|0.78|1001001|amalgamalg #1|1|dresses|1|Women|260|barcallyable|large|8741802245lawn580938|violet|Lb|Unknown|28|esebarese| +405|AAAAAAAAEJBAAAAA|2000-10-27||Various hours wor|1.50|0.55|4001002|amalgedu pack #2|1|womens|4|Shoes|185|barcallyable|extra large|831lavender008888402|tomato|Pound|Unknown|2|antibarese| +406|AAAAAAAAGJBAAAAA|1997-10-27|1999-10-27|Less different beans shall ask probably other american ends. Foreign times might begin well; children lose quite. Surely theoretical friends use usually faint, different affairs. Industria|3.91|3.47|10001016|amalgunivamalg #16|1|cameras|10|Electronics|55|antianti|N/A|9sandy48772482698258|tan|Ton|Unknown|11|callybarese| +407|AAAAAAAAGJBAAAAA|1999-10-28|2001-10-26|Less different beans shall ask probably other american ends. Foreign times might begin well; children lose quite. Surely theoretical friends use usually faint, different affairs. Industria|3.61|2.20|10001016|edu packscholar #2|1|classical|5|Music|948|antianti|N/A|7thistle287834117103|violet|Carton|Unknown|20|ationbarese| +408|AAAAAAAAGJBAAAAA|2001-10-27||Less different beans shall ask probably other american ends. Foreign times might begin well; children lose quite. Surely theoretical friends use usually faint, different affairs. Industria|2.70|1.16|10001016|exportibrand #9|1|kids|7|Home|566|callycallyanti|N/A|4puff858599653405826|aquamarine|Unknown|Unknown|62|eingbarese| +409|AAAAAAAAJJBAAAAA|1997-10-27||Finally odd factors see classes; laboratories might|58.51|48.56|3001002|amalgexporti #2|1|newborn|3|Children|747|ationeseation|economy|2ivory87524552129789|peru|Tsp|Unknown|33|n stbarese| +410|AAAAAAAAKJBAAAAA|1997-10-27|2000-10-26|Small kinds would recognize notably violent, labour years. Electronic days would not|0.90|0.34|6008003|namelesscorp #3|8|mens watch|6|Jewelry|567|ationcallyanti|N/A|6719033goldenrod1733|saddle|Each|Unknown|44|baroughtese| +411|AAAAAAAAKJBAAAAA|2000-10-27||Here representati|0.52|0.16|1004002|edu packamalg #2|4|swimwear|1|Women|452|ableantiese|medium|81604slate1859698906|navajo|Unknown|Unknown|5|oughtoughtese| +412|AAAAAAAAMJBAAAAA|1997-10-27|1999-10-27|Direct, basic tanks shall complain suddenly nearer clear governments. Ma|3.09|2.75|2003001|exportiimporto #1|3|pants|2|Men|261|oughtcallyable|large|8030purple9496998447|turquoise|Case|Unknown|42|ableoughtese| +413|AAAAAAAAMJBAAAAA|1999-10-28|2001-10-26|Statutory, growing proposals hope old, other crimes. Global others focus well excellent, differ|0.42|0.15|10013006|exportiamalgamalg #6|3|stereo|10|Electronics|200|oughtcallyable|N/A|6saddle0632303572287|hot|Cup|Unknown|22|prioughtese| +414|AAAAAAAAMJBAAAAA|2001-10-27||Early pairs should not explore in the signals. Rules get farmers. For example ill operations keep practic|8.27|0.15|10013006|exportiimporto #1|3|pants|2|Men|174|eseationought|extra large|6saddle0632303572287|royal|Pallet|Unknown|29|eseoughtese| +415|AAAAAAAAPJBAAAAA|1997-10-27||Recent points used to see socie|5.65|3.05|2003002|exportiimporto #2|3|pants|2|Men|321|oughtablepri|medium|7015honeydew79581521|spring|Carton|Unknown|88|antioughtese| +416|AAAAAAAAAKBAAAAA|1997-10-27|2000-10-26|Grateful, western applicants may apologise individually central groups. Findings can reduce large-scale, coming effects. Foreign chemicals should not reveal then far futur|7.87|6.05|10007009|brandunivamalg #9|7|personal|10|Electronics|288|eingeingable|N/A|8356smoke96150445596|orchid|Unknown|Unknown|25|callyoughtese| +417|AAAAAAAAAKBAAAAA|2000-10-27||Grateful, western applicants may apologise individually central groups. Findings can reduce large-scale, coming effects. Foreign chemicals should not reveal then far futur|5.80|6.05|5004002|edu packscholar #2|4|classical|5|Music|35|antipri|N/A|8356smoke96150445596|medium|Oz|Unknown|8|ationoughtese| +418|AAAAAAAACKBAAAAA|1997-10-27|1999-10-27|Thoroughly divine miles may not|3.35|1.00|10005011|scholarunivamalg #11|5|karoke|10|Electronics|229|n stableable|N/A|579698405white380493|floral|Ton|Unknown|33|eingoughtese| +419|AAAAAAAACKBAAAAA|1999-10-28|2001-10-26|Alternative plants know therefore by the tools. Much competitive firms should not identify in order anx|1.85|1.00|4004002|edu packedu pack #2|4|athletic|4|Shoes|229|n stableable|petite|579698405white380493|puff|Tbl|Unknown|12|n stoughtese| +420|AAAAAAAACKBAAAAA|2001-10-27||Flowers fetch good, unlikely h|13.56|1.00|4004002|amalgexporti #1|4|newborn|3|Children|393|n stableable|medium|74red134756258903405|misty|Tbl|Unknown|37|barableese| +421|AAAAAAAAFKBAAAAA|1997-10-27||Lists let as. Trees control moreover directly safe occasions; exactly legal personnel could not understand right, remaining bonds. Ever prepar|5.70|2.56|10005014|scholarunivamalg #14|5|karoke|10|Electronics|578|eingationanti|N/A|6210041851930tomato7|navajo|Ton|Unknown|66|oughtableese| +422|AAAAAAAAGKBAAAAA|1997-10-27|2000-10-26|As other folk can remain quickly methods. Easy, othe|1.87|1.08|7008003|namelessbrand #3|8|lighting|7|Home|207|ationbarable|N/A|05119271454876rose77|tan|Pound|Unknown|9|ableableese| +423|AAAAAAAAGKBAAAAA|2000-10-27||As other folk can remain quickly methods. Easy, othe|3.50|1.08|7008003|corpnameless #4|16|furniture|7|Home|699|n stn stcally|N/A|05119271454876rose77|thistle|Dram|Unknown|15|priableese| +424|AAAAAAAAIKBAAAAA|1997-10-27|1999-10-27|National, permanent processes help modestly crim|4.05|2.87|6015001|scholarbrand #1|15|custom|6|Jewelry|536|callyprianti|N/A|49833119brown6948742|seashell|N/A|Unknown|28|eseableese| +425|AAAAAAAAIKBAAAAA|1999-10-28|2001-10-26|Targets will give functional, continental cases. Documents will stand mistakenly short interested groups|94.88|2.87|6015001|exportiedu pack #2|3|kids|4|Shoes|537|ationprianti|N/A|49833119brown6948742|rose|N/A|Unknown|32|antiableese| +426|AAAAAAAAIKBAAAAA|2001-10-27||Targets will give functional, continental cases. Documents will stand mistakenly short interested groups|0.18|2.87|5003001|exportischolar #1|3|pop|5|Music|537|ationprianti|N/A|21095356957peach9087|midnight|Box|Unknown|1|callyableese| +427|AAAAAAAALKBAAAAA|1997-10-27||Merely professional rules like; photographs relieve so at the costs. New officers could provide yesterday wooden facilities. Great trees may understand else. New armies might gain nuclear, superb|4.45|2.09|3002002|importoexporti #2|2|infants|3|Children|47|ationese|small|11335743129slate7897|gainsboro|Gross|Unknown|44|ationableese| +428|AAAAAAAAMKBAAAAA|1997-10-27|2000-10-26|Territories may not give on a exercises. Customers indicate never just logical beings. Total years must not get still. Results help however. Local organizations continue much economic women. |2.19|1.59|3004001|edu packexporti #1|4|school-uniforms|3|Children|412|ableoughtese|medium|4236134271077lawn102|seashell|Case|Unknown|12|eingableese| +429|AAAAAAAAMKBAAAAA|2000-10-27||Titles must not collect with a years. Letters understand in a circumstances. As large teachers will not want lives. National concerns c|2.88|1.52|3004001|importounivamalg #10|4|home repair|9|Books|312|ableoughtese|N/A|4236134271077lawn102|yellow|N/A|Unknown|8|n stableese| +430|AAAAAAAAOKBAAAAA|1997-10-27|1999-10-27|Once sweet resources must talk effective, suspicious galleries. Citizens must not capitalise electronically further forward decisions. Developments would n|4.56|1.86|3003001|exportiexporti #1|3|toddlers|3|Children|741|oughteseation|medium|6041298spring6778583|purple|Dram|Unknown|25|barpriese| +431|AAAAAAAAOKBAAAAA|1999-10-28|2001-10-26|Once sweet resources must talk effective, suspicious galleries. Citizens must not capitalise electronically further forward decisions. Developments would n|2.86|1.31|6013008|exportibrand #8|13|loose stones|6|Jewelry|332|oughteseation|N/A|6041298spring6778583|plum|Cup|Unknown|88|oughtpriese| +432|AAAAAAAAOKBAAAAA|2001-10-27||Once sweet resources must talk effective, suspicious galleries. Citizens must not capitalise electronically further forward decisions. Developments would n|70.80|50.26|7006003|corpbrand #3|6|rugs|7|Home|295|oughteseation|N/A|6041298spring6778583|thistle|Each|Unknown|30|ablepriese| +433|AAAAAAAABLBAAAAA|1997-10-27||Able animals win even from the events; mass standards write both sm|7.40|3.47|1003002|exportiamalg #2|3|maternity|1|Women|348|eingesepri|medium|misty598726805352809|purple|Tbl|Unknown|46|pripriese| +434|AAAAAAAACLBAAAAA|1997-10-27|2000-10-26|Common, logical babies must take somehow general months. Costs drag. Big, british areas give dramatic, effective clot|3.00|1.83|9016003|corpunivamalg #3|16|mystery|9|Books|190|barn stought|N/A|7911smoke46488847861|smoke|Carton|Unknown|54|esepriese| +435|AAAAAAAACLBAAAAA|2000-10-27||Considerable, required developments might resign quite to the services. Alternative, public practitio|6.94|5.34|5001002|amalgscholar #2|1|rock|5|Music|190|barn stought|N/A|7911smoke46488847861|lavender|Box|Unknown|37|antipriese| +436|AAAAAAAAELBAAAAA|1997-10-27|1999-10-27|Valuable cells assist from a days; backs would take per a techniques. Dates ought to wake ste|5.53|2.98|6003005|exporticorp #5|3|gold|6|Jewelry|95|antin st|N/A|6056tan3860238191214|peach|Lb|Unknown|14|callypriese| +437|AAAAAAAAELBAAAAA|1999-10-28|2001-10-26|Valuable cells assist from a days; backs would take per a techniques. Dates ought to wake ste|82.37|2.98|8016008|corpmaxi #8|16|golf|8|Sports|95|antin st|N/A|6056tan3860238191214|thistle|Lb|Unknown|99|ationpriese| +438|AAAAAAAAELBAAAAA|2001-10-27||Valuable cells assist from a days; backs would take per a techniques. Dates ought to wake ste|31.15|2.98|8016008|edu packimporto #1|4|sports-apparel|2|Men|95|antin st|petite|6056tan3860238191214|dim|Box|Unknown|34|eingpriese| +439|AAAAAAAAHLBAAAAA|1997-10-27||Men worsen increasingly top studies. Too able patients take signs. Double terms cannot reduce suddenly lands. Full, square talks need anyway all just|1.80|1.38|3002002|importoexporti #2|2|infants|3|Children|597|ationn stanti|medium|7plum410077317317662|tomato|Gram|Unknown|8|n stpriese| +440|AAAAAAAAILBAAAAA|1997-10-27|2000-10-26|Always working matters may worry now important new officers. Faster likel|3.16|1.20|4003001|exportiedu pack #1|3|kids|4|Shoes|162|ablecallyought|extra large|503988023313slate707|peach|Case|Unknown|51|bareseese| +441|AAAAAAAAILBAAAAA|2000-10-27||Always working matters may worry now important new officers. Faster likel|6.44|1.20|5004002|edu packscholar #2|3|classical|5|Music|162|ablecallyought|N/A|503988023313slate707|seashell|Ton|Unknown|42|oughteseese| +442|AAAAAAAAKLBAAAAA|1997-10-27|1999-10-27|Extreme, detailed fortunes carry other, different colleges.|1.06|0.60|2003001|exportiimporto #1|3|pants|2|Men|500|barbaranti|large|73345146986moccasin2|violet|Tbl|Unknown|45|ableeseese| +443|AAAAAAAAKLBAAAAA|1999-10-28|2001-10-26|Extreme, detailed fortunes carry other, different colleges.|0.91|0.37|6009004|maxicorp #4|3|womens watch|6|Jewelry|500|barbaranti|N/A|8055172blanched03077|turquoise|Oz|Unknown|2|prieseese| +444|AAAAAAAAKLBAAAAA|2001-10-27||Extreme, detailed fortunes carry other, different colleges.|3.85|1.34|6009004|edu packimporto #1|4|sports-apparel|2|Men|69|n stcally|petite|28thistle45133059487|pink|Box|Unknown|57|eseeseese| +445|AAAAAAAANLBAAAAA|1997-10-27||Hard industrial issues hold internal, dramatic libraries. Almost total days know much such as a characteristics. Estimated voices rid on the consumers. Able users suggest single, human c|2.39|1.09|1002002|importoamalg #2|2|fragrances|1|Women|579|n stationanti|small|173897788spring95565|purple|Ounce|Unknown|53|antieseese| +446|AAAAAAAAOLBAAAAA|1997-10-27|2000-10-26|Activities say. Right lips resort current techniques. Regional, possible daughters might not present changes; students can notice ridiculous, l|2.96|2.10|3002001|importoexporti #1|2|infants|3|Children|313|prioughtpri|small|6831410960066khaki91|thistle|Unknown|Unknown|2|callyeseese| +447|AAAAAAAAOLBAAAAA|2000-10-27||Activities say. Right lips resort current techniques. Regional, possible daughters might not present changes; students can notice ridiculous, l|1.31|2.10|3002001|exportiedu pack #2|3|kids|4|Shoes|740|prioughtpri|extra large|6831410960066khaki91|saddle|N/A|Unknown|40|ationeseese| +448|AAAAAAAAAMBAAAAA|1997-10-27|1999-10-27|Especially atomic transactions explain angrily full, white relations. Ordinary conflicts should enforce for example. Trends w|1.93|1.33|2003001|exportiimporto #1|3|pants|2|Men|534|eseprianti|extra large|925603026seashell260|snow|Each|Unknown|19|eingeseese| +449|AAAAAAAAAMBAAAAA|1999-10-28|2001-10-26|Especially atomic transactions explain angrily full, white relations. Ordinary conflicts should enforce for example. Trends w|0.68|1.33|2003001|edu packunivamalg #8|4|audio|10|Electronics|534|eseprianti|N/A|664199tan05534254942|green|Box|Unknown|81|n steseese| +450|AAAAAAAAAMBAAAAA|2001-10-27||Far consistent regions might believe; teenage accounts ought t|4.59|1.79|2002001|importoimporto #1|4|shirts|2|Men|242|ableeseable|petite|664199tan05534254942|thistle|Carton|Unknown|20|barantiese| +451|AAAAAAAADMBAAAAA|1997-10-27||Matters conceal new chips. Together typical newspapers inspire approximately. Objectives perfect profitable, running things. Gastric coll|1.01|0.66|1003002|exportiamalg #2|3|maternity|1|Women|457|ationantiese|medium|954406spring13099976|olive|Tbl|Unknown|33|oughtantiese| +452|AAAAAAAAEMBAAAAA|1997-10-27|2000-10-26|Smooth ears take as well years. Optimistic numbers use now|9.57|5.07|5004001|edu packscholar #1|4|classical|5|Music|767|ationcallyation|N/A|42778109115sienna429|burlywood|Each|Unknown|28|ableantiese| +453|AAAAAAAAEMBAAAAA|2000-10-27||As fond rules explain again to a members. Also central doors balance. Other, significant arms could not ignore best for the women.|0.18|0.06|5004001|corpunivamalg #10|4|mystery|9|Books|767|ationcallyation|N/A|sky74652322066145098|midnight|Pallet|Unknown|5|priantiese| +454|AAAAAAAAGMBAAAAA|1997-10-27|1999-10-27|Average fathers might show anyway too previo|3.48|1.07|2002001|importoimporto #1|2|shirts|2|Men|145|antieseought|medium|95752249452pink31055|violet|Lb|Unknown|21|eseantiese| +455|AAAAAAAAGMBAAAAA|1999-10-28|2001-10-26|Average fathers might show anyway too previo|1.47|0.48|2002001|edu packscholar #2|4|classical|5|Music|514|antieseought|N/A|95752249452pink31055|red|Pound|Unknown|17|antiantiese| +456|AAAAAAAAGMBAAAAA|2001-10-27||Average fathers might show anyway too previo|81.39|36.62|2002001|importoamalg #1|4|fragrances|1|Women|514|antieseought|small|95752249452pink31055|tomato|Tbl|Unknown|72|callyantiese| +457|AAAAAAAAJMBAAAAA|1997-10-27||Ec|6.59|3.03|10015016|scholaramalgamalg #16|15|portable|10|Electronics|220|barableable|N/A|580peach606151411689|ghost|Tbl|Unknown|20|ationantiese| +458|AAAAAAAAKMBAAAAA|1997-10-27|2000-10-26|Perhaps silver points join in order easy interests. Likely, tender individuals ought to give still cigarettes. At least criminal steps could not make now really apparen|1.53|1.16|4004001|edu packedu pack #1|4|athletic|4|Shoes|356|callyantipri|medium|537183274865328lawn9|blanched|Dram|Unknown|88|eingantiese| +459|AAAAAAAAKMBAAAAA|2000-10-27||Perhaps silver points join in order easy interests. Likely, tender individuals ought to give still cigarettes. At least criminal steps could not make now really apparen|7.18|2.51|4004001|brandbrand #10|7|decor|7|Home|21|callyantipri|N/A|742202525052wheat252|linen|Cup|Unknown|18|n stantiese| +460|AAAAAAAAMMBAAAAA|1997-10-27|1999-10-27|Boys might not work yet then fast clothes. Simply large elements think in a factors. Royal charges happen at least on a children. Holy prospects think individu|8.88|7.81|7004001|edu packbrand #1|4|curtains/drapes|7|Home|697|ationn stcally|N/A|384drab7768884701620|pink|Cup|Unknown|11|barcallyese| +461|AAAAAAAAMMBAAAAA|1999-10-28|2001-10-26|Boys might not work yet then fast clothes. Simply large elements think in a factors. Royal charges happen at least on a children. Holy prospects think individu|1.69|7.81|7004001|maxinameless #2|9|optics|8|Sports|697|ationn stcally|N/A|seashell854301591973|salmon|Ton|Unknown|30|oughtcallyese| +462|AAAAAAAAMMBAAAAA||||5.94|||||athletic||Shoes||||22244926blanched9762|||Unknown|25|| +463|AAAAAAAAPMBAAAAA|1997-10-27||Exactly financial games may find effective, delight|8.79|4.65|9015008|scholarunivamalg #8|15|fiction|9|Books|600|barbarcally|N/A|5891pale752461542494|plum|Tbl|Unknown|33|pricallyese| +464|AAAAAAAAANBAAAAA|1997-10-27|2000-10-26|Rarely splendid books study better arguments. Then ot|6.51|2.92|10006006|corpunivamalg #6|6|musical|10|Electronics|438|eingpriese|N/A|8412417836417puff438|seashell|Gross|Unknown|3|esecallyese| +465|AAAAAAAAANBAAAAA|2000-10-27||Expenses should become heavily boys. Words make also fairly new improvements. Primitive, good parties used to discuss capable readers. Large, economic ques|7.23|2.92|10006006|exportiedu pack #2|3|kids|4|Shoes|438|eingpriese|extra large|98232535245628peru83|pink|Lb|Unknown|57|anticallyese| +466|AAAAAAAACNBAAAAA|1997-10-27|1999-10-27|Prime things shall not steer. Outside, early words hear high large |2.39|1.45|6007003|brandcorp #3|7|pendants|6|Jewelry|20|barable|N/A|9469194177317wheat82|pink|N/A|Unknown|1|callycallyese| +467|AAAAAAAACNBAAAAA|1999-10-28|2001-10-26|Almost distant supporters serve anyway now cheap parents. Reforms shall not think old, back clothes; rivers cannot |2.67|1.89|6007003|exportischolar #2|3|pop|5|Music|98|eingn st|N/A|9469194177317wheat82|turquoise|Bunch|Unknown|32|ationcallyese| +468|AAAAAAAACNBAAAAA|2001-10-27||Almost distant supporters serve anyway now cheap parents. Reforms shall not think old, back clothes; rivers cannot |3.35|1.27|7013009|exportinameless #9|3|wallpaper|7|Home|285|eingn st|N/A|4678162769499wheat99|azure|N/A|Unknown|97|eingcallyese| +469|AAAAAAAAFNBAAAAA|1997-10-27||Happy laws sit on the powers. Quickly convenient newspapers |0.16|0.09|7016010|corpnameless #10|16|furniture|7|Home|444|eseeseese|N/A|5237smoke06628202907|sandy|Ounce|Unknown|12|n stcallyese| +470|AAAAAAAAGNBAAAAA|1997-10-27|2000-10-26|Spiritual, black words resist enough old parents. Late expected problems go single, aware sources. British, sad |1.34|1.01|3002001|importoexporti #1|2|infants|3|Children|146|callyeseought|extra large|209spring09286098297|papaya|Oz|Unknown|32|barationese| +471|AAAAAAAAGNBAAAAA|2000-10-27||Spiritual, black words resist enough old parents. Late expected problems go single, aware sources. British, sad |9.69|7.84|3002001|edu packimporto #2|2|sports-apparel|2|Men|146|callyeseought|small|87light7818582179653|purple|Ton|Unknown|29|oughtationese| +472|AAAAAAAAINBAAAAA|1997-10-27|1999-10-27|Forward different degrees free very feet. Other houses must arise even. Husbands must not say also sciences; numerous, imperial arms get. Months |2.26|0.67|6016001|corpbrand #1|16|consignment|6|Jewelry|615|antioughtcally|N/A|0990turquoise9462865|salmon|Dozen|Unknown|16|ableationese| +473|AAAAAAAAINBAAAAA|1999-10-28|2001-10-26|Forward different degrees free very feet. Other houses must arise even. Husbands must not say also sciences; numerous, imperial arms get. Months |0.89|0.70|6003002|exporticorp #2|16|gold|6|Jewelry|615|antioughtcally|N/A|138139319cyan5828404|navajo|Unknown|Unknown|28|priationese| +474|AAAAAAAAINBAAAAA|2001-10-27||Forward different degrees free very feet. Other houses must arise even. Husbands must not say also sciences; numerous, imperial arms get. Months |5.00|0.70|6004003|edu packcorp #3|4|bracelets|6|Jewelry|615|antioughtcally|N/A|243518232forest94247|brown|Tsp|Unknown|17|eseationese| +475|AAAAAAAALNBAAAAA|1997-10-27||Areas must sho|29.85|14.32|2001002|amalgimporto #2|1|accessories|2|Men|85|antieing|medium|4150399836ivory44645|salmon|Ounce|Unknown|19|antiationese| +476|AAAAAAAAMNBAAAAA|1997-10-27|2000-10-26|Golden, international studies will not occur still old holes. Small lines tear possible weaknesses. Once reasonable actions buy formerly interested results. Unions must no|1.79|0.91|4004001|edu packedu pack #1|4|athletic|4|Shoes|318|eingoughtpri|medium|671snow7732200796544|orange|Pound|Unknown|89|callyationese| +477|AAAAAAAAMNBAAAAA|2000-10-27||Golden, international studies will not occur still old holes. Small lines tear possible weaknesses. Once reasonable actions buy formerly interested results. Unions must no|7.37|0.91|4004001|importoamalgamalg #2|4|monitors|10|Electronics|318|eingoughtpri|N/A|671snow7732200796544|sandy|Dram|Unknown|15|ationationese| +478|AAAAAAAAONBAAAAA|1997-10-27|1999-10-27|Top, new subjects could learn small things. Economic, open unions see public, careful tenants. Constant, popula|8.61|4.73|6003007|exporticorp #7|3|gold|6|Jewelry|743|prieseation|N/A|00turquoise939138765|slate|Unknown|Unknown|3|eingationese| +479|AAAAAAAAONBAAAAA|1999-10-28|2001-10-26|Top, new subjects could learn small things. Economic, open unions see public, careful tenants. Constant, popula|2.41|2.04|6002008|importocorp #8|3|diamonds|6|Jewelry|743|prieseation|N/A|00turquoise939138765|yellow|Bundle|Unknown|96|n stationese| +480|AAAAAAAAONBAAAAA|2001-10-27||Males may not swing ultimately as well accessible occupations; children might change only easy dimensions. Catholic arts can benefit sure total sentences. Pupils give centres. |1.98|2.04|6002001|importocorp #1|2|diamonds|6|Jewelry|743|prieseation|N/A|00turquoise939138765|white|Oz|Unknown|93|bareingese| +481|AAAAAAAABOBAAAAA|1997-10-27||Old, revolutionary eyes may not serve fully by|2.38|1.90|9008002|namelessmaxi #2|8|romance|9|Books|219|n stoughtable|N/A|573444233278022peru8|puff|Dram|Unknown|60|oughteingese| +482|AAAAAAAACOBAAAAA|1997-10-27|2000-10-26|Young, impossible illustration|1.79|0.66|4002001|importoedu pack #1|2|mens|4|Shoes|577|ationationanti|extra large|5120942754metallic53|khaki|Each|Unknown|22|ableeingese| +483|AAAAAAAACOBAAAAA|2000-10-27||Old, true governments decide ideal, overall classes. Fairly slight questions leave joint, front brothers. Likely jobs go for the regions. Personal, productive adults leave by a men. Ot|2.29|0.66|4002002|importoedu pack #2|2|mens|4|Shoes|255|ationationanti|N/A|5120942754metallic53|honeydew|Pound|Unknown|10|prieingese| +484|AAAAAAAAEOBAAAAA|1997-10-27|1999-10-27|Too old colonies see later sorry ideas. Pale, excessive parties might take other rivers. Immediately major politicians s|4.34|1.73|4002001|importoedu pack #1|2|mens|4|Shoes|300|barbarpri|medium|2rosy068945275237395|misty|Unknown|Unknown|45|eseeingese| +485|AAAAAAAAEOBAAAAA|1999-10-28|2001-10-26|Measurements clean in a men; clear representatives ought to link sorry, modern facts. Professional documents should not find clear sanctions; only, sound minds should not make now speciali|8.43|2.86|4002001|importocorp #6|2|diamonds|6|Jewelry|300|barbarpri|N/A|2rosy068945275237395|misty|Gram|Unknown|23|antieingese| +486|AAAAAAAAEOBAAAAA|2001-10-27||Measurements clean in a men; clear representatives ought to link sorry, modern facts. Professional documents should not find clear sanctions; only, sound minds should not make now speciali|2.86|2.86|4002001|scholarcorp #7|2|earings|6|Jewelry|636|callyprically|N/A|110706peach778322441|wheat|Lb|Unknown|12|callyeingese| +487|AAAAAAAAHOBAAAAA|1997-10-27||Ordinary, high strings should argue for instance monetary teams; bright, major problems will not make less annually unexpected years. Affairs must not maintain in vitro for a concentrations. Other, s|4.61|3.78|4001002|amalgedu pack #2|1|womens|4|Shoes|85|antieing|large|violet22719139070291|yellow|Dram|Unknown|4|ationeingese| +488|AAAAAAAAIOBAAAAA|1997-10-27|2000-10-26|Medical, extra ages used to draw possible holidays. Ever si|0.69|0.52|3004001|edu packexporti #1|4|school-uniforms|3|Children|256|callyantiable|medium|8355sandy81023683627|violet|Each|Unknown|51|eingeingese| +489|AAAAAAAAIOBAAAAA|2000-10-27||Dead, marginal holes shall feel; private conditions improve straight, french languages. General, concerned lips get. Artists should take scottish, attractive threats. Quickly f|6.72|4.50|3004001|scholarbrand #2|15|custom|6|Jewelry|186|callyeingought|N/A|8355sandy81023683627|dodger|Gram|Unknown|73|n steingese| +490|AAAAAAAAKOBAAAAA|1997-10-27|1999-10-27|Modern men would not ask girls. Often p|6.55|5.63|7012001|importonameless #1|12|paint|7|Home|189|n steingought|N/A|570196833759rosy1999|lawn|Cup|Unknown|80|barn stese| +491|AAAAAAAAKOBAAAAA|1999-10-28|2001-10-26|Modern men would not ask girls. Often p|4.00|2.28|10012016|importoamalgamalg #16|12|monitors|10|Electronics|36|callypri|N/A|09934015631white0814|honeydew|Each|Unknown|4|oughtn stese| +492|AAAAAAAAKOBAAAAA|2001-10-27||Modern men would not ask girls. Often p|3.54|2.28|5001001|amalgscholar #1|12|rock|5|Music|36|callypri|N/A|2811thistle833232139|plum|Pallet|Unknown|61|ablen stese| +493|AAAAAAAANOBAAAAA|1997-10-27||Much human forms repeat better now large seats. New fingers may |4.09|3.39|5004002|edu packscholar #2|4|classical|5|Music|300|barbarpri|N/A|3617690157377smoke19|white|Carton|Unknown|22|prin stese| +494|AAAAAAAAOOBAAAAA|1997-10-27|2000-10-26|Nasty, past questions can go busy, certain reasons. Basic societies participate for example royal costs. Dif|34.47|16.20|1003001|exportiamalg #1|3|maternity|1|Women|328|eingablepri|small|61206856682seashell2|tomato|Tsp|Unknown|49|esen stese| +495|AAAAAAAAOOBAAAAA|2000-10-27||Nasty, past questions can go busy, certain reasons. Basic societies participate for example royal costs. Dif|4.91|16.20|1003001|importoedu pack #2|3|mens|4|Shoes|328|eingablepri|petite|12731320460wheat6192|light|Tsp|Unknown|62|antin stese| +496|AAAAAAAAAPBAAAAA|1997-10-27|1999-10-27|Times may make rightly papers. Very roya|6.92|4.63|5002001|importoscholar #1|2|country|5|Music|64|esecally|N/A|deep1441188952752090|plum|Bundle|Unknown|43|callyn stese| +497|AAAAAAAAAPBAAAAA|1999-10-28|2001-10-26|Times may make rightly papers. Very roya|1.49|4.63|5002001|univamalgamalg #5|2|memory|10|Electronics|64|esecally|N/A|96chartreuse01493458|metallic|Tsp|Unknown|98|ationn stese| +498|AAAAAAAAAPBAAAAA|2001-10-27||Times may make rightly papers. Very roya|1.59|0.66|5002001|amalgedu pack #1|1|womens|4|Shoes|29|esecally|petite|96chartreuse01493458|powder|Ounce|Unknown|44|eingn stese| +499|AAAAAAAADPBAAAAA|1997-10-27||Dry houses drive. Popular cars cannot think well. Ltd. crops take just. Also public dogs could not use either free areas. So available magistrates might not mat|1.52|1.06|6011004|amalgbrand #4|11|semi-precious|6|Jewelry|9|n st|N/A|0014yellow4570989308|rose|Each|Unknown|8|n stn stese| +500|AAAAAAAAEPBAAAAA|1997-10-27|2000-10-26|Potential materials form|6.28|2.07|9007009|brandmaxi #9|7|reference|9|Books|178|eingationought|N/A|111wheat602909523598|sienna|Bunch|Unknown|13|barbaranti| +501|AAAAAAAAEPBAAAAA|2000-10-27||Potential materials form|8.10|2.07|5002002|importoscholar #2|2|country|5|Music|192|eingationought|N/A|61871509117524peru81|honeydew|Dozen|Unknown|7|oughtbaranti| +502|AAAAAAAAGPBAAAAA|1997-10-27|1999-10-27|Gre|4.04|2.26|10009010|maxiunivamalg #10|9|televisions|10|Electronics|192|ablen stought|N/A|3881831645559powder4|peach|Box|Unknown|57|ablebaranti| +503|AAAAAAAAGPBAAAAA|1999-10-28|2001-10-26|Gre|7.09|6.23|1003002|exportiamalg #2|9|maternity|1|Women|396|ablen stought|large|3881831645559powder4|wheat|Bunch|Unknown|65|pribaranti| +504|AAAAAAAAGPBAAAAA|2001-10-27||Gre|3.19|6.23|4001001|amalgedu pack #1|1|womens|4|Shoes|432|ablen stought|petite|3881831645559powder4|rosy|Dram|Unknown|3|esebaranti| +505|AAAAAAAAJPBAAAAA|1997-10-27||Wise employees must resist on|1.72|0.68|10009013|maxiunivamalg #13|9|televisions|10|Electronics|265|anticallyable|N/A|9ghost97040689829432|sky|Ton|Unknown|13|antibaranti| +506|AAAAAAAAKPBAAAAA|1997-10-27|2000-10-26|Simply scottish corporations join whole, practical concerns. Ma|6.27|2.44|7013007|exportinameless #7|13|wallpaper|7|Home|314|eseoughtpri|N/A|0377740521167medium8|tan|Tsp|Unknown|44|callybaranti| +507|AAAAAAAAKPBAAAAA|2000-10-27||Valid, aware details ought to ask to a aspirations. Differences will satisfy very original facts; afraid men could not sit bloody girls; decisions feel |9.44|5.38|7013007|scholarcorp #4|5|earings|6|Jewelry|415|eseoughtpri|N/A|0377740521167medium8|white|Dozen|Unknown|48|ationbaranti| +508|AAAAAAAAMPBAAAAA|1997-10-27|1999-10-27|Able, strong pictures understand especially. Similar years feed sometimes close, bri|2.94|2.08|9013005|exportiunivamalg #5|13|self-help|9|Books|265|anticallyable|N/A|70363salmon360524007|papaya|Dozen|Unknown|3|eingbaranti| +509|AAAAAAAAMPBAAAAA|1999-10-28|2001-10-26|Able, strong pictures understand especially. Similar years feed sometimes close, bri|0.62|2.08|6006006|corpcorp #6|13|rings|6|Jewelry|7|anticallyable|N/A|70363salmon360524007|tan|Dozen|Unknown|9|n stbaranti| +510|AAAAAAAAMPBAAAAA|2001-10-27||Able, strong pictures understand especially. Similar years feed sometimes close, bri|8.92|3.38|7010001|univnameless #1|13|flatware|7|Home|471|oughtationese|N/A|51puff47964612070504|rosy|Gram|Unknown|3|baroughtanti| +511|AAAAAAAAPPBAAAAA|1997-10-27||Sensitive, able meanings might not mean delicately options. Details tell even influential, appropriate structu|6.25|3.87|5002002|importoscholar #2|2|country|5|Music|19|n stought|N/A|123615038white780413|white|Ton|Unknown|59|oughtoughtanti| +512|AAAAAAAAAACAAAAA|1997-10-27|2000-10-26|Suitable, free others might run in a parts. Once material conditions might get real, top dogs. Key, |74.88|64.39|2002001|importoimporto #1|2|shirts|2|Men|150|barantiought|large|967529peru2368582484|steel|Pallet|Unknown|73|ableoughtanti| +513|AAAAAAAAAACAAAAA|2000-10-27||Magnetic years should not interpret then at the fingers. Bright stands could work yesterday supreme, reasonable details. Scientific, great lev|3.34|64.39|4001002|amalgedu pack #2|1|womens|4|Shoes|150|barantiought|medium|5038burlywood1202869|snow|Gross|Unknown|14|prioughtanti| +514|AAAAAAAACACAAAAA|1997-10-27|1999-10-27|Political men cannot get|2.38|1.61|2002001|importoimporto #1|2|shirts|2|Men|102|ablebarought|medium|134plum0449526057824|papaya|Dram|Unknown|26|eseoughtanti| +515|AAAAAAAACACAAAAA|1999-10-28|2001-10-26|Financial, domestic hours find perhaps then western consumers. Deep months would think authorities. Just |2.11|1.61|2002001|importomaxi #6|12|guns|8|Sports|308|eingbarpri|N/A|134plum0449526057824|peach|N/A|Unknown|2|antioughtanti| +516|AAAAAAAACACAAAAA|2001-10-27||Students celebrate furthermore previous candidates. Chief, various services introduce similar rates; aims would believe swiftly initial studies; phases should record|68.47|36.28|6003005|exporticorp #5|12|gold|6|Jewelry|659|eingbarpri|N/A|2044377chiffon991677|tan|Oz|Unknown|96|callyoughtanti| +517|AAAAAAAAFACAAAAA|1997-10-27||Colleges trust most other novels. New, prospective women want unex|0.81|0.24|6016006|corpbrand #6|16|consignment|6|Jewelry|509|n stbaranti|N/A|1307884red4957300719|smoke|Pallet|Unknown|3|ationoughtanti| +518|AAAAAAAAGACAAAAA|1997-10-27|2000-10-26|All traditional colours should ensure elsewhere recent, lucky pounds. Limited hours mean easier passengers. Now p|4.24|2.71|6003007|exporticorp #7|3|gold|6|Jewelry|985|antieingn st|N/A|32150643wheat4031096|spring|Tsp|Unknown|31|eingoughtanti| +519|AAAAAAAAGACAAAAA|2000-10-27||All traditional colours should ensure elsewhere recent, lucky pounds. Limited hours mean easier passengers. Now p|5.98|5.20|7002010|importobrand #10|2|bedding|7|Home|313|prioughtpri|N/A|85780turquoise223075|dodger|Bundle|Unknown|19|n stoughtanti| +520|AAAAAAAAIACAAAAA|1997-10-27|1999-10-27|Clearly chief relations shall find women. Alone shops will get forwards far corporate countries. Subject, new others ought to call over men. Good, secondary resear|3.73|1.52|6006001|corpcorp #1|6|rings|6|Jewelry|718|eingoughtation|N/A|6412907steel34938900|seashell|Ton|Unknown|6|barableanti| +521|AAAAAAAAIACAAAAA|1999-10-28|2001-10-26|Feet should take more popular, black qualities. Expensive, grey individuals jeopardise to a humans. Political operations concentrate once at a nights. Early po|0.71|1.52|2004002|edu packimporto #2|6|sports-apparel|2|Men|718|eingoughtation|medium|4297961944093rose674|misty|Box|Unknown|65|oughtableanti| +522|AAAAAAAAIACAAAAA|2001-10-27||Complex recommendations prefer more by a shares.|1.78|1.52|2004002|amalgamalgamalg #13|6|disk drives|10|Electronics|718|eingoughtation|N/A|4297961944093rose674|slate|Bundle|Unknown|21|ableableanti| +523|AAAAAAAALACAAAAA|1997-10-27||Very free names explain elsewhere. Alternatively innocent parts may not support as for a ways. Black organ|1.86|1.07|2002002|importoimporto #2|2|shirts|2|Men|824|eseableeing|medium|391chocolate47903129|lavender|Case|Unknown|40|priableanti| +524|AAAAAAAAMACAAAAA|1997-10-27|2000-10-26|Events may accept |5.42|3.63|5003001|exportischolar #1|3|pop|5|Music|212|ableoughtable|N/A|0898844smoke13076322|slate|Bundle|Unknown|46|eseableanti| +525|AAAAAAAAMACAAAAA|2000-10-27||Events may accept |6.50|3.63|5003001|amalgexporti #2|3|newborn|3|Children|212|ableoughtable|medium|797954716172medium61|orchid|Gram|Unknown|34|antiableanti| +526|AAAAAAAAOACAAAAA|1997-10-27|1999-10-27|Still violent officials promo|8.99|7.10|6010007|univbrand #7|10|jewelry boxes|6|Jewelry|17|ationought|N/A|44violet725909325768|powder|Cup|Unknown|15|callyableanti| +527|AAAAAAAAOACAAAAA|1999-10-28|2001-10-26|Ltd. fees ought to pass everyday numbers. Criminal reso|4.50|7.10|4002002|importoedu pack #2|10|mens|4|Shoes|88|ationought|extra large|001plum4057924645801|pale|Bundle|Unknown|3|ationableanti| +528|AAAAAAAAOACAAAAA|2001-10-27||Full owners may undertake tactics. However medical patterns take most official farmers. Outer, lively stars would not buy entire metres. Local, national rea|3.11|7.10|7002009|importobrand #9|10|bedding|7|Home|88|ationought|N/A|692rose6743924297527|lawn|Lb|Unknown|18|eingableanti| +529|AAAAAAAABBCAAAAA|1997-10-27||Flights concentrate ironically fundamental unions. Military, foll|5.19|4.67|3001002|amalgexporti #2|1|newborn|3|Children|19|n stought|petite|503yellow98756015845|brown|Case|Unknown|23|n stableanti| +530|AAAAAAAACBCAAAAA|1997-10-27|2000-10-26|Largely indian lessons summon today. Patterns could leave techniques. Expected children imagine detectives. Currently quick candidates know so as a|0.56|0.48|3001001|amalgexporti #1|1|newborn|3|Children|615|antioughtcally|economy|8sienna0398529051935|seashell|Gram|Unknown|20|barprianti| +531|AAAAAAAACBCAAAAA|2000-10-27||Pleasant, furious weapons obtain yet for the poems. Radically federal activities should receive parts. Relations repair. European|98.30|84.53|3001001|importoexporti #2|2|infants|3|Children|165|antioughtcally|extra large|8sienna0398529051935|violet|Each|Unknown|7|oughtprianti| +532|AAAAAAAAEBCAAAAA|1997-10-27|1999-10-27|Mass, running men could fight so; previous, present doctors used to ask. Other, important events grow always top, in|9.36|4.02|5001001|amalgscholar #1|1|rock|5|Music|758|eingantiation|N/A|365gainsboro43921612|wheat|Tbl|Unknown|30|ableprianti| +533|AAAAAAAAEBCAAAAA|1999-10-28|2001-10-26|Mass, running men could fight so; previous, present doctors used to ask. Other, important events grow always top, in|6.18|3.64|1002002|importoamalg #2|2|fragrances|1|Women|758|eingantiation|petite|07431763orchid262317|pink|Dram|Unknown|46|priprianti| +534|AAAAAAAAEBCAAAAA|2001-10-27||Remarkable results could play automatically clear, mature problems; there available years rise new years. Home healthy initiatives come at the trials. Poor problems must fly only hours|3.18|3.64|1002002|exportiexporti #1|3|toddlers|3|Children|758|eingantiation|medium|1007584353962white99|salmon|Each|Unknown|65|eseprianti| +535|AAAAAAAAHBCAAAAA|1997-10-27||Suddenly grey nations allow yesterday original ideas. Old institutions can want never able, main authors. Measures blame only costs. Honest, dangerous offices will dismiss only sure ports; |8.22|2.46|6004008|edu packcorp #8|4|bracelets|6|Jewelry|66|callycally|N/A|430sandy467434872647|peach|Tsp|Unknown|82|antiprianti| +536|AAAAAAAAIBCAAAAA|1997-10-27|2000-10-26|Exactl|2.65|1.00|2004001|edu packimporto #1|4|sports-apparel|2|Men|10|barought|petite|710955545smoke804269|tan|Unknown|Unknown|41|callyprianti| +537|AAAAAAAAIBCAAAAA|2000-10-27||Common, old modules express alone enterprises. British, other obligations provide criminal, old details. Complete eyes used to make inside radical t|6.17|2.22|9001010|amalgmaxi #10|4|arts|9|Books|268|barought|N/A|710955545smoke804269|peach|Carton|Unknown|60|ationprianti| +538|AAAAAAAAKBCAAAAA|1997-10-27|1999-10-27|Names could not ask appointments; difficu|3.23|2.84|6002003|importocorp #3|2|diamonds|6|Jewelry|119|n stoughtought|N/A|472470718332sandy463|tomato|Pound|Unknown|100|eingprianti| +539|AAAAAAAAKBCAAAAA|1999-10-28|2001-10-26|Names could not ask appointments; difficu|3.31|2.84|6002003|edu packmaxi #12|4|entertainments|9|Books|230|n stoughtought|N/A|831brown371979460755|slate|Carton|Unknown|61|n stprianti| +540|AAAAAAAAKBCAAAAA|2001-10-27||Names could not ask appointments; difficu|4.76|3.52|6002003|amalgbrand #1|4|bathroom|7|Home|27|n stoughtought|N/A|760466papaya46234734|peach|Lb|Unknown|13|bareseanti| +541|AAAAAAAANBCAAAAA|1997-10-27||Unlike, planned men guess marginal things. Marvellous conditions work long important clothes. Old, numerous symptoms realise skilfully. Visible, substantial police could not prepare records. Open |4.71|1.69|10003015|exportiunivamalg #15|3|dvd/vcr players|10|Electronics|89|n steing|N/A|43211124872pink97990|pink|N/A|Unknown|43|oughteseanti| +542|AAAAAAAAOBCAAAAA|1997-10-27|2000-10-26|Right backs want other, necessary lines. Lightly low men look already thanks. Avai|4.06|2.39|6007007|brandcorp #7|7|pendants|6|Jewelry|449|n steseese|N/A|6steel46577965778112|powder|Carton|Unknown|18|ableeseanti| +543|AAAAAAAAOBCAAAAA|2000-10-27||Right backs want other, necessary lines. Lightly low men look already thanks. Avai|7.03|5.62|6007007|importoamalg #2|7|fragrances|1|Women|449|n steseese|small|49872709839mint77297|hot|Tsp|Unknown|6|prieseanti| +544|AAAAAAAAACCAAAAA|1997-10-27|1999-10-27|Rural, modern affairs depend in a areas. Men kill only actually true imports. Here similar effects give able lights|7.14|4.35|2004001|edu packimporto #1|4|sports-apparel|2|Men|487|ationeingese|petite|88490817515saddle185|mint|Case|Unknown|26|eseeseanti| +545|AAAAAAAAACCAAAAA|1999-10-28|2001-10-26|Homes run political, real interests. Historic, f|2.20|1.43|3002002|importoexporti #2|2|infants|3|Children|288|eingeingable|small|07orchid701827327226|peach|Bunch|Unknown|52|antieseanti| +546|AAAAAAAAACCAAAAA|2001-10-27||Ties give safe classes. Subjects would extend exactly. Eastern, little imports differ places. Eyes call maybe concerned goods. General forests see also joint nurses. Alternative, metrop|3.74|1.43|8016007|corpmaxi #7|2|golf|8|Sports|288|eingeingable|N/A|07orchid701827327226|pink|Pallet|Unknown|30|callyeseanti| +547|AAAAAAAADCCAAAAA|1997-10-27||Psychological, economic leaders suppose hardly as cultural conditions. Regular, apparent schools must say again no polish matters. Yet short ways can give enough for the values. Prisoners est|3.54|2.08|3001002|amalgexporti #2|1|newborn|3|Children|478|eingationese|large|sky08536272495844045|spring|Bundle|Unknown|5|ationeseanti| +548|AAAAAAAAECCAAAAA|1997-10-27|2000-10-26|Boats confer never national, isolated posts. Modern res|3.40|2.58|10011005|amalgamalgamalg #5|11|disk drives|10|Electronics|499|n stn stese|N/A|708622808827powder99|midnight|Case|Unknown|31|eingeseanti| +549|AAAAAAAAECCAAAAA|2000-10-27||Very early vehicles tu|6.47|4.72|10011005|corpmaxi #10|6|parenting|9|Books|193|prin stought|N/A|97181588978khaki4991|peach|Carton|Unknown|14|n steseanti| +550|AAAAAAAAGCCAAAAA|1997-10-27|1999-10-27|Visible members defeat low in the sons. Final measures wish clear clouds. In order public years cannot find la|3.72|1.86|8008001|namelessnameless #1|8|outdoor|8|Sports|392|ablen stpri|N/A|53704smoke5438796329|saddle|Dram|Unknown|23|barantianti| +551|AAAAAAAAGCCAAAAA|1999-10-28|2001-10-26|Possible, responsible parents|7.91|1.86|8013002|exportimaxi #2|13|sailing|8|Sports|392|ablen stpri|N/A|1921313028black38021|wheat|Cup|Unknown|3|oughtantianti| +552|AAAAAAAAGCCAAAAA|2001-10-27||Possible, responsible parents|52.72|1.86|4004001|edu packedu pack #1|4|athletic|4|Shoes|263|ablen stpri|small|1921313028black38021|indian|Cup|Unknown|20|ableantianti| +553|AAAAAAAAJCCAAAAA|1997-10-27||Fair kids tell spots. Easily suitable records can wake more results. Areas face so.|0.87|0.68|1004002|edu packamalg #2|4|swimwear|1|Women|593|prin stanti|medium|873525062steel867865|red|Gram|Unknown|51|priantianti| +554|AAAAAAAAKCCAAAAA|1997-10-27|2000-10-26|Possible schools carry primarily dual rises; important meetings could continue other passengers. More scottish things might not fall orders. Right, unable expectati|4.44|2.39|9011003|amalgunivamalg #3|11|cooking|9|Books|207|ationbarable|N/A|ghost821708904982369|peru|Box|Unknown|21|eseantianti| +555|AAAAAAAAKCCAAAAA|2000-10-27||Possible schools carry primarily dual rises; important meetings could continue other passengers. More scottish things might not fall orders. Right, unable expectati|9.29|2.39|9011003|amalgamalg #2|1|dresses|1|Women|262|ationbarable|medium|ghost821708904982369|papaya|Cup|Unknown|1|antiantianti| +556|AAAAAAAAMCCAAAAA|1997-10-27|1999-10-27|Problems would use carefully chinese, good views. Joint, only officers study interesting areas. Ultimate dangers may work recently in a pictures. |3.95|3.43|4003001|exportiedu pack #1|3|kids|4|Shoes|311|oughtoughtpri|large|4989lace475962731989|rosy|Gram|Unknown|2|callyantianti| +557|AAAAAAAAMCCAAAAA|1999-10-28|2001-10-26|Problems would use carefully chinese, good views. Joint, only officers study interesting areas. Ultimate dangers may work recently in a pictures. |8.49|3.90|5001002|amalgscholar #2|1|rock|5|Music|794|oughtoughtpri|N/A|4989lace475962731989|orange|Each|Unknown|4|ationantianti| +558|AAAAAAAAMCCAAAAA|2001-10-27||Ages cannot relate most to a hours. Rough shareholders bear grey, front years; driver|0.13|0.05|4001001|amalgedu pack #1|1|womens|4|Shoes|609|oughtoughtpri|petite|4989lace475962731989|spring|Oz|Unknown|57|eingantianti| +559|AAAAAAAAPCCAAAAA|1997-10-27||Tall relationships may not determine upon a relations. Again popular children would base cold, old boundaries;|3.30|2.64|9010008|univunivamalg #8|10|travel|9|Books|66|callycally|N/A|5732412255white76381|pale|Gram|Unknown|99|n stantianti| +560|AAAAAAAAADCAAAAA|1997-10-27|2000-10-26|Nice knees help be|1.55|1.33|9016009|corpunivamalg #9|16|mystery|9|Books|523|priableanti|N/A|782951spring00486499|tan|Oz|Unknown|11|barcallyanti| +561|AAAAAAAAADCAAAAA|2000-10-27||Very european markets used to pl|4.34|3.38|9016009|exportiimporto #2|16|pants|2|Men|179|priableanti|medium|957sandy182660809955|spring|Dozen|Unknown|70|oughtcallyanti| +562|AAAAAAAACDCAAAAA|1997-10-27|1999-10-27|Young children ought to believe more dramatic houses. Committees would dream conditions. Simply co|2.71|2.22|3001001|amalgexporti #1|1|newborn|3|Children|300|barbarpri|small|594436741920sandy824|olive|Tsp|Unknown|22|ablecallyanti| +563|AAAAAAAACDCAAAAA|1999-10-28|2001-10-26|Issues must raise obviously reasons. Particles meet easily for the years. Problems say with the things. Sha|7.17|2.36|4002002|importoedu pack #2|2|mens|4|Shoes|300|barbarpri|extra large|594436741920sandy824|red|Tsp|Unknown|59|pricallyanti| +564|AAAAAAAACDCAAAAA|2001-10-27||Please main pages ought to k|4.60|3.58|2001001|amalgimporto #1|2|accessories|2|Men|300|barbarpri|N/A|594436741920sandy824|smoke|Pallet|Unknown|34|esecallyanti| +565|AAAAAAAAFDCAAAAA|1997-10-27||Kinds used to go. Then perfect ways blame rather than the shelves; sexual conditions use arguably controls. All pos|7.56|5.51|1001002|amalgamalg #2|1|dresses|1|Women|127|ationableought|medium|340003882seashell284|papaya|Each|Unknown|37|anticallyanti| +566|AAAAAAAAGDCAAAAA|1997-10-27|2000-10-26|Still final relationships ought to apply quite outdoor judicial relations. Inland homes provide hours. Unique eyes may claim however attractive pp.; ef|8.65|4.84|2002001|importoimporto #1|2|shirts|2|Men|273|priationable|small|03394491pink68973592|navajo|Each|Unknown|48|callycallyanti| +567|AAAAAAAAGDCAAAAA|2000-10-27||Still final relationships ought to apply quite outdoor judicial relations. Inland homes provide hours. Unique eyes may claim however attractive pp.; ef|4.61|2.95|2002001|importoimporto #2|2|shirts|2|Men|273|priationable|medium|36971firebrick614505|green|Each|Unknown|73|ationcallyanti| +568|AAAAAAAAIDCAAAAA|1997-10-27|1999-10-27|Major, old conflicts shall make yet experienced commentators. Active, other losses see. At least intense accidents will maintain more a|4.04|2.70|4003001|exportiedu pack #1|3|kids|4|Shoes|413|prioughtese|small|71gainsboro962251870|chiffon|Tsp|Unknown|9|eingcallyanti| +569|AAAAAAAAIDCAAAAA|1999-10-28|2001-10-26|Major, old conflicts shall make yet experienced commentators. Active, other losses see. At least intense accidents will maintain more a|9.08|4.26|4003001|namelessnameless #10|8|outdoor|8|Sports|85|antieing|N/A|71gainsboro962251870|sky|Box|Unknown|39|n stcallyanti| +570|AAAAAAAAIDCAAAAA|2001-10-27||Major, old conflicts shall make yet experienced commentators. Active, other losses see. At least intense accidents will maintain more a|3.42|4.26|8004001|edu packnameless #1|8|camping|8|Sports|869|antieing|N/A|51787155758sandy1494|ivory|Lb|Unknown|31|barationanti| +571|AAAAAAAALDCAAAAA|1997-10-27||Fingers apply. Legs poison high windows; widespread, male effects will educate. Patterns leave only essentially mass drivers. Seats launch ext|88.29|26.48|5001002|amalgscholar #2|1|rock|5|Music|243|prieseable|N/A|652lemon791038445113|plum|Ounce|Unknown|40|oughtationanti| +572|AAAAAAAAMDCAAAAA|1997-10-27|2000-10-26|Championships must visit now cells; shows give. Major, british positions ought to get eyes. New colleagues want objects. Upwards eco|2.97|2.13|1002001|importoamalg #1|2|fragrances|1|Women|59|n stanti|small|306sky12260561147661|smoke|Bunch|Unknown|76|ableationanti| +573|AAAAAAAAMDCAAAAA|2000-10-27||Most great times would bear then safe companies. Cultural, autonomous jobs must give enough environmental, video-taped girls. Parliamentary, glad minerals should not like i|3.93|1.49|7006004|corpbrand #4|2|rugs|7|Home|59|n stanti|N/A|306sky12260561147661|orchid|Tsp|Unknown|6|priationanti| +574|AAAAAAAAODCAAAAA|1997-10-27|1999-10-27|Visitors shall talk less just violent members. S|0.15|0.12|3003001|exportiexporti #1|3|toddlers|3|Children|495|antin stese|small|10610puff54288649087|mint|Cup|Unknown|15|eseationanti| +575|AAAAAAAAODCAAAAA|1999-10-28|2001-10-26|Visitors shall talk less just violent members. S|5.07|0.12|10012015|importoamalgamalg #15|12|monitors|10|Electronics|181|antin stese|N/A|62467914pink99245733|olive|Each|Unknown|12|antiationanti| +576|AAAAAAAAODCAAAAA|2001-10-27||Visitors shall talk less just violent members. S|4.19|0.12|10012015|exportischolar #1|12|pop|5|Music|50|antin stese|N/A|2426658999puff993916|steel|Ounce|Unknown|28|callyationanti| +577|AAAAAAAABECAAAAA|1997-10-27||Southern countries believe. Mentally different members view originally environmental eyes. Opinions express almost comfortable months. Local, underlying needs ima|2.02|0.86|5001002|amalgscholar #2|1|rock|5|Music|67|ationcally|N/A|4182peach02274427669|plum|Pound|Unknown|27|ationationanti| +578|AAAAAAAACECAAAAA|1997-10-27|2000-10-26|Human, possible rumours buy then both following sides; continuous hands use again in the writers; distinctive others increase afterwards wild s|4.92|4.03|9016003|corpunivamalg #3|16|mystery|9|Books|505|antibaranti|N/A|9493seashell80275696|white|Each|Unknown|28|eingationanti| +579|AAAAAAAACECAAAAA|2000-10-27||Human, possible rumours buy then both following sides; continuous hands use again in the writers; distinctive others increase afterwards wild s|3.07|4.03|9016003|edu packimporto #2|16|sports-apparel|2|Men|505|antibaranti|medium|9493seashell80275696|misty|Box|Unknown|62|n stationanti| +580|AAAAAAAAEECAAAAA|1997-10-27|1999-10-27|New, full rises drive however. Legs retain often in a women. Remaining, parliamentary miles may help nevertheless rather full pupils. Glad crimes might|3.98|1.83|10014003|edu packamalgamalg #3|14|automotive|10|Electronics|286|callyeingable|N/A|55537047peach8247615|burlywood|Cup|Unknown|66|bareinganti| +581|AAAAAAAAEECAAAAA|1999-10-28|2001-10-26|New, full rises drive however. Legs retain often in a women. Remaining, parliamentary miles may help nevertheless rather full pupils. Glad crimes might|7.17|3.87|10014003|scholarnameless #2|5|fitness|8|Sports|167|callyeingable|N/A|55537047peach8247615|orange|Cup|Unknown|62|oughteinganti| +582|AAAAAAAAEECAAAAA|2001-10-27||Subsequent, slight issues ask soft matters. Informal things assert achievements. Single, attractive bedrooms could not make then by a horses. Automatic s|1.02|3.87|6015007|scholarbrand #7|15|custom|6|Jewelry|279|n stationable|N/A|14peru79055030972507|thistle|Tsp|Unknown|28|ableeinganti| +583|AAAAAAAAHECAAAAA|1997-10-27||Public, typical features ough|5.15|1.90|1003002|exportiamalg #2|3|maternity|1|Women|115|antioughtought|large|7077589mint793434145|pink|Ounce|Unknown|30|prieinganti| +584|AAAAAAAAIECAAAAA|1997-10-27|2000-10-26|Quite top models laugh now scenes. Thanks characterize as minimal posts. Developers flush no longer in the prefer|0.16|0.04|4001001|amalgedu pack #1|1|womens|4|Shoes|294|esen stable|extra large|3867934grey271924876|yellow|Case|Unknown|33|eseeinganti| +585|AAAAAAAAIECAAAAA|2000-10-27||Quite top models laugh now scenes. Thanks characterize as minimal posts. Developers flush no longer in the prefer|8.00|5.28|4001001|scholarnameless #6|1|tables|7|Home|634|eseprically|N/A|3867934grey271924876|pale|Ton|Unknown|34|antieinganti| +586|AAAAAAAAKECAAAAA|1997-10-27|1999-10-27|High publishers can exclude certain stars. Too i|87.61|69.21|7006007|corpbrand #7|6|rugs|7|Home|269|n stcallyable|N/A|152174303221lemon250|lemon|Box|Unknown|56|callyeinganti| +587|AAAAAAAAKECAAAAA|1999-10-28|2001-10-26|Yet crucial words could take most effective, good activities; ke|3.48|1.21|7006007|edu packamalg #2|6|swimwear|1|Women|480|n stcallyable|large|2570843rosy502431501|sienna|Gram|Unknown|1|ationeinganti| +588|AAAAAAAAKECAAAAA|2001-10-27||Yet crucial words could take most effective, good activities; ke|9.84|5.31|10012011|importoamalgamalg #11|6|monitors|10|Electronics|480|n stcallyable|N/A|2570843rosy502431501|white|Cup|Unknown|92|eingeinganti| +589|AAAAAAAANECAAAAA|1997-10-27||Friends tell also still large students. Public personnel return. Local, recent hours could see n|4.84|3.63|1004002|edu packamalg #2|4|swimwear|1|Women|206|callybarable|small|6387slate79073984495|lawn|Lb|Unknown|32|n steinganti| +590|AAAAAAAAOECAAAAA|1997-10-27|2000-10-26|Restrictions boast patient eyes. Alone major rooms sho|8.86|3.80|10007013|brandunivamalg #13|7|personal|10|Electronics|342|ableesepri|N/A|96lemon3526506946574|purple|Ton|Unknown|29|barn stanti| +591|AAAAAAAAOECAAAAA|2000-10-27||Practical, given questions bury women; probably other points stress in the phases. Red, tiny purposes say. Valuable, conservative times shall add new, light cases. Studies like ago seri|55.95|3.80|10007013|edu packedu pack #2|7|athletic|4|Shoes|342|ableesepri|small|0758996262469rosy238|sky|Ton|Unknown|56|oughtn stanti| +592|AAAAAAAAAFCAAAAA|1997-10-27|1999-10-27|Mental, nice projects go enough for the bonds. Good lessons happen free paym|4.15|1.78|10010015|univamalgamalg #15|10|memory|10|Electronics|552|ableantianti|N/A|341901010salmon99331|powder|N/A|Unknown|49|ablen stanti| +593|AAAAAAAAAFCAAAAA|1999-10-28|2001-10-26|Mental, nice projects go enough for the bonds. Good lessons happen free paym|2.91|1.39|10010015|importoexporti #2|2|infants|3|Children|552|ableantianti|medium|341901010salmon99331|sky|Case|Unknown|31|prin stanti| +594|AAAAAAAAAFCAAAAA|2001-10-27||Mental, nice projects go enough for the bonds. Good lessons happen free paym|2.12|1.78|3002001|importoexporti #1|2|infants|3|Children|911|oughtoughtn st|medium|341901010salmon99331|chiffon|Ounce|Unknown|16|esen stanti| +595|AAAAAAAADFCAAAAA|1997-10-27||Areas can like traditional details. Sources hear well daily participants; cheap, russian jobs use with a feet. Just heavy relat|2.96|2.33|3004002|edu packexporti #2|4|school-uniforms|3|Children|470|barationese|medium|5wheat81646471412273|thistle|Each|Unknown|71|antin stanti| +596|AAAAAAAAEFCAAAAA|1997-10-27|2000-10-26|Eligible, stupid attitudes used to protect so. Alone, good sciences concentrate suddenly liable eyes. Revolutionary students should punch f|0.35|0.19|8011007|amalgmaxi #7|11|archery|8|Sports|196|callyn stought|N/A|863874turquoise57159|forest|Ounce|Unknown|5|callyn stanti| +597|AAAAAAAAEFCAAAAA|2000-10-27||Small, opening animals think. Again following tactics might provide however useful police. Hours would get now major, beautiful rewards. Readers appeal so fi|5.60|0.19|8011007|amalgedu pack #2|11|womens|4|Shoes|196|callyn stought|small|863874turquoise57159|medium|Ton|Unknown|18|ationn stanti| +598|AAAAAAAAGFCAAAAA|1997-10-27|1999-10-27|Departmen|2.09|1.19|9006011|corpmaxi #11|6|parenting|9|Books|69|n stcally|N/A|7red0264956424087297|olive|N/A|Unknown|49|eingn stanti| +599|AAAAAAAAGFCAAAAA|1999-10-28|2001-10-26|Federa|4.16|2.53|9006011|exportiunivamalg #5|3|dvd/vcr players|10|Electronics|69|n stcally|N/A|7red0264956424087297|thistle|Unknown|Unknown|5|n stn stanti| +600|AAAAAAAAGFCAAAAA|2001-10-27||Federa|0.31|0.10|9006011|edu packimporto #1|4|sports-apparel|2|Men|69|n stcally|large|7red0264956424087297|violet|Dozen|Unknown|51|barbarcally| +601|AAAAAAAAJFCAAAAA|1997-10-27||Times increase well as small arrangements. Present numbers lead police. Efficiently parliamen|0.50|0.33|5003002|exportischolar #2|3|pop|5|Music|100|barbarought|N/A|37365700628salmon405|royal|Tsp|Unknown|23|oughtbarcally| +602|AAAAAAAAKFCAAAAA|1997-10-27|2000-10-26|Nation|53.95|31.83|4004001|edu packedu pack #1|4|athletic|4|Shoes|16|callyought|medium|53713091857salmon835|goldenrod|Dozen|Unknown|34|ablebarcally| +603|AAAAAAAAKFCAAAAA|2000-10-27||Experienced, certain months ought to produce more new problems. Experiences worry somewhere shops; other years know really young objections. Particular, convention|5.80|31.83|4004002|edu packedu pack #2|4|athletic|4|Shoes|44|callyought|extra large|015458614turquoise81|yellow|Carton|Unknown|2|pribarcally| +604|AAAAAAAAMFCAAAAA|1997-10-27|1999-10-27|Followers may not believe maybe. Units feel sensitive, confide|8.66|5.19|2004001|edu packimporto #1|4|sports-apparel|2|Men|985|antieingn st|petite|6565215513magenta421|steel|Pound|Unknown|29|esebarcally| +605|AAAAAAAAMFCAAAAA|1999-10-28|2001-10-26|Years could move suddenly on the systems. Drawings used to go even final contacts; differences look today workers; bare, superior ci|0.10|0.07|4001002|amalgedu pack #2|4|womens|4|Shoes|985|antieingn st|small|68102moccasin0275403|saddle|N/A|Unknown|5|antibarcally| +606|AAAAAAAAMFCAAAAA|2001-10-27||Duties get real children. At all permanent organisations will not place especially social enquiries. Human, young days should arrest there active |2.07|1.09|4001002|edu packscholar #1|4|classical|5|Music|792|ablen station|N/A|68102moccasin0275403|rose|Cup|Unknown|30|callybarcally| +607|AAAAAAAAPFCAAAAA|1997-10-27||However common mistakes used to relate quickly officers. Courses will determine new, close types. Cells shall not counter mos|3.18|2.28|10010013|univamalgamalg #13|10|memory|10|Electronics|638|eingprically|N/A|18136140959peach8258|khaki|Dram|Unknown|1|ationbarcally| +608|AAAAAAAAAGCAAAAA|1997-10-27|2000-10-26|Old heroes ought to seek wildly glorious cultures. Prepared events might support inside. Factors should argue suitable cat|7.52|2.63|9014009|edu packunivamalg #9|14|sports|9|Books|550|barantianti|N/A|6396350pale227086104|midnight|Box|Unknown|24|eingbarcally| +609|AAAAAAAAAGCAAAAA|2000-10-27||Twice bad girls will make perfectly normal necessary doors; principles ta|6.78|4.27|6011002|amalgbrand #2|14|semi-precious|6|Jewelry|73|barantianti|N/A|6396350pale227086104|steel|Lb|Unknown|61|n stbarcally| +610|AAAAAAAACGCAAAAA|1997-10-27|1999-10-27|Knowingly unable trees may talk. Real, public places may not contain numerous years. Long different stones remain|2.74|0.95|4003001|exportiedu pack #1|3|kids|4|Shoes|322|ableablepri|small|papaya05302672076691|wheat|Gram|Unknown|32|baroughtcally| +611|AAAAAAAACGCAAAAA|1999-10-28|2001-10-26|Yet traditional institutions will not cut of course; in|7.62|5.86|1002002|importoamalg #2|2|fragrances|1|Women|322|ableablepri|small|43364puff53862702239|saddle|Dram|Unknown|11|oughtoughtcally| +612|AAAAAAAACGCAAAAA|2001-10-27||C|1.70|1.32|3001001|amalgexporti #1|2|newborn|3|Children|552|ableantianti|extra large|43364puff53862702239|plum|Case|Unknown|36|ableoughtcally| +613|AAAAAAAAFGCAAAAA|1997-10-27||At last involved stages look sharply allies. Ini|1.89|1.19|9013002|exportiunivamalg #2|13|self-help|9|Books|290|barn stable|N/A|7179069007184snow284|plum|Cup|Unknown|96|prioughtcally| +614|AAAAAAAAGGCAAAAA|1997-10-27|2000-10-26|Modes sha|8.65|3.97|4004001|edu packedu pack #1|4|athletic|4|Shoes|111|oughtoughtought|medium|24spring600356759158|wheat|Dozen|Unknown|25|eseoughtcally| +615|AAAAAAAAGGCAAAAA|2000-10-27||Later british glasses continue early head |2.89|3.97|3003002|exportiexporti #2|3|toddlers|3|Children|111|oughtoughtought|petite|24spring600356759158|salmon|Ton|Unknown|35|antioughtcally| +616|AAAAAAAAIGCAAAAA|1997-10-27|1999-10-27|Clearly various effects|9.34|3.45|3003001|exportiexporti #1|3|toddlers|3|Children|174|eseationought|N/A|28909603sandy5487322|seashell|Dozen|Unknown|28|callyoughtcally| +617|AAAAAAAAIGCAAAAA|1999-10-28|2001-10-26|Black, brilliant investors would limit; total doubts approach additional, big examples. Times may expand nicely children. Constantly well-known changes exist years. Particularly russian stories|8.20|6.80|3001002|amalgexporti #2|1|newborn|3|Children|681|eseationought|medium|papaya29585064557550|snow|Lb|Unknown|44|ationoughtcally| +618|AAAAAAAAIGCAAAAA|2001-10-27||Black, brilliant investors would limit; total doubts approach additional, big examples. Times may expand nicely children. Constantly well-known changes exist years. Particularly russian stories|1.72|6.80|3001002|univamalgamalg #7|10|memory|10|Electronics|79|n station|N/A|4968923732510maroon3|forest|N/A|Unknown|25|eingoughtcally| +619|AAAAAAAALGCAAAAA|1997-10-27||Voters cause already urban, formal children. Medieval shares must not spare human, crazy things; so public |9.27|3.98|7009010|maxibrand #10|9|mattresses|7|Home|496|callyn stese|N/A|766128pale9821609014|sienna|Each|Unknown|15|n stoughtcally| +620|AAAAAAAAMGCAAAAA|1997-10-27|2000-10-26|Excellent hands should go reall|52.90|39.67|6003005|exporticorp #5|3|gold|6|Jewelry|34|esepri|N/A|087saddle77009822441|puff|Tbl|Unknown|2|barablecally| +621|AAAAAAAAMGCAAAAA|2000-10-27||Excellent hands should go reall|2.62|39.67|6003005|importoamalg #2|2|fragrances|1|Women|34|esepri|medium|087saddle77009822441|sky|Case|Unknown|3|oughtablecally| +622|AAAAAAAAOGCAAAAA|1997-10-27|1999-10-27|Practical institutions introduce today losses. Great structures would place different eyes. Children make deaf years. More famous letters know only special, economic goods; then increased ser|3.49|1.91|6006007|corpcorp #7|6|rings|6|Jewelry|133|pripriought|N/A|5418899233735829sky1|snow|Pallet|Unknown|56|ableablecally| +623|AAAAAAAAOGCAAAAA|1999-10-28|2001-10-26|Practical institutions introduce today losses. Great structures would place different eyes. Children make deaf years. More famous letters know only special, economic goods; then increased ser|5.67|1.91|3002002|importoexporti #2|6|infants|3|Children|889|pripriought|large|5418899233735829sky1|peru|Ounce|Unknown|31|priablecally| +624|AAAAAAAAOGCAAAAA|2001-10-27||Procedures|8.77|1.91|3002002|edu packunivamalg #1|6|sports|9|Books|722|pripriought|N/A|5418899233735829sky1|maroon|Case|Unknown|45|eseablecally| +625|AAAAAAAABHCAAAAA|1997-10-27||Deep, various children convince already. Good, pregnant boys gain so girls. Long-term, large industries might pay much difficult approaches. Strategic, christian crimes serve uniformly new, strong mil|4.16|1.83|2002002|importoimporto #2|2|shirts|2|Men|41|oughtese|large|07912771834lawn05870|coral|Lb|Unknown|17|antiablecally| +626|AAAAAAAACHCAAAAA|1997-10-27|2000-10-26|Total, certain demands want in a men. Customs zer|3.78|2.15|5001001|amalgscholar #1|1|rock|5|Music|178|eingationought|N/A|78509577210tomato793|indian|N/A|Unknown|14|callyablecally| +627|AAAAAAAACHCAAAAA|2000-10-27||Actively successive eyes shall remain undoubtedly similar, right others. Patients might surrender also due, sig|1.08|0.97|2001002|amalgimporto #2|1|accessories|2|Men|321|eingationought|medium|35005884113159white7|dodger|Ton|Unknown|36|ationablecally| +628|AAAAAAAAEHCAAAAA|1997-10-27|1999-10-27|Relevant years develop apparent, english stages. Young eyes lift sometimes at the complaints; high, initial funds should not protect certainly full, ann|0.39|0.21|10014017|edu packamalgamalg #17|14|automotive|10|Electronics|334|esepripri|N/A|8plum798721994631510|peru|Ton|Unknown|77|eingablecally| +629|AAAAAAAAEHCAAAAA|1999-10-28|2001-10-26|Relevant years develop apparent, english stages. Young eyes lift sometimes at the complaints; high, initial funds should not protect certainly full, ann|4.45|0.21|10014017|edu packscholar #2|14|classical|5|Music|778|esepripri|N/A|107157942347orange44|royal|Case|Unknown|33|n stablecally| +630|AAAAAAAAEHCAAAAA|2001-10-27||Children cannot go perfectly able, unlike years. Values would not repay usually clearly good years. Bloody negotiatio|0.23|0.21|10014017|edu packmaxi #1|14|tennis|8|Sports|346|callyesepri|N/A|107157942347orange44|papaya|Unknown|Unknown|26|barprically| +631|AAAAAAAAHHCAAAAA|1997-10-27||Old, cultural workers ought to take both now everyday budgets. Nearer interesting hours could not assure very centuries|1.65|0.59|7002002|importobrand #2|2|bedding|7|Home|294|esen stable|N/A|66lavender2471213739|wheat|Case|Unknown|3|oughtprically| +632|AAAAAAAAIHCAAAAA|1997-10-27|2000-10-26|Democratic levels may attend in a books. Yesterday british industries could not experience almost. Various, fair years leave middle activities; more only thousands st|4.63|3.65|1003001|exportiamalg #1|3|maternity|1|Women|897|ationn steing|small|91peach5457743770309|misty|Ton|Unknown|22|ableprically| +633|AAAAAAAAIHCAAAAA|2000-10-27||Nations used to profit there lucky aspects. Lengthy individuals say often full members. Closely economic prices te|8.47|3.65|2004002|edu packimporto #2|3|sports-apparel|2|Men|604|ationn steing|large|91peach5457743770309|dim|Cup|Unknown|25|priprically| +634|AAAAAAAAKHCAAAAA|1997-10-27|1999-10-27|Always similar signs ask. Social, real knees ought to live apparently women. Also similar hands may think private governments. Fr|0.09|0.06|1004001|edu packamalg #1|4|swimwear|1|Women|764|esecallyation|medium|1sandy39463559830250|salmon|Each|Unknown|36|eseprically| +635|AAAAAAAAKHCAAAAA|1999-10-28|2001-10-26|Always similar signs ask. Social, real knees ought to live apparently women. Also similar hands may think private governments. Fr|1.59|0.81|1004001|exportiimporto #2|3|pants|2|Men|764|esecallyation|economy|1sandy39463559830250|metallic|Each|Unknown|3|antiprically| +636|AAAAAAAAKHCAAAAA|2001-10-27||Always similar signs ask. Social, real knees ought to live apparently women. Also similar hands may think private governments. Fr|4.87|0.81|3004001|edu packexporti #1|4|school-uniforms|3|Children|564|esecallyanti|large|408401071310white961|papaya|Pound|Unknown|2|callyprically| +637|AAAAAAAANHCAAAAA|1997-10-27||Unexpected employers shall not make much schools. Urban, following galleries keep only sure, demanding changes. Surprisingly married colours |9.10|6.91|1003002|exportiamalg #2|3|maternity|1|Women|594|esen stanti|extra large|163069wheat073280020|antique|N/A|Unknown|48|ationprically| +638|AAAAAAAAOHCAAAAA|1997-10-27|2000-10-26|Japanese, other children plot between a patients; large, old-fashioned groups must reverse easily in the ways; short nerves may emphasize better small|9.55|8.59|1003001|exportiamalg #1|3|maternity|1|Women|369|n stcallypri|large|0788183899334tomato2|khaki|Case|Unknown|22|eingprically| +639|AAAAAAAAOHCAAAAA|2000-10-27||Activities could not call for a journalists. Continental effects can cope also french classes. Legal teams should inspect problems. Farmers must review in the efforts. Black areas will come |4.22|8.59|2004002|edu packimporto #2|4|sports-apparel|2|Men|447|ationeseese|medium|0788183899334tomato2|sky|Tsp|Unknown|12|n stprically| +640|AAAAAAAAAICAAAAA|1997-10-27|1999-10-27|Urgent, simple cases may not help. Industrial, other pp. reverse as a schools. Asleep, free systems make then more available discussions. Soci|4.82|2.41|7013001|exportinameless #1|13|wallpaper|7|Home|456|callyantiese|N/A|66724015slate4227739|red|Bunch|Unknown|43|baresecally| +641|AAAAAAAAAICAAAAA|1999-10-28|2001-10-26|Only new hours must |3.76|2.41|7005002|scholarbrand #2|13|blinds/shades|7|Home|393|callyantiese|N/A|snow0478588997185983|rosy|Ton|Unknown|44|oughtesecally| +642|AAAAAAAAAICAAAAA|2001-10-27||Only new hours must |60.80|2.41|9008007|namelessmaxi #7|8|romance|9|Books|233|pripriable|N/A|98368828857434lemon9|azure|Box|Unknown|13|ableesecally| +643|AAAAAAAADICAAAAA|1997-10-27||Members become so poor peri|32.36|19.41|8016004|corpmaxi #4|16|golf|8|Sports|11|oughtought|N/A|43870706beige9683056|thistle|Case|Unknown|26|priesecally| +644|AAAAAAAAEICAAAAA|1997-10-27|2000-10-26|Cases include proudly without a columns. Solid, pre|2.42|1.83|9002009|importomaxi #9|2|business|9|Books|306|callybarpri|N/A|pale1146850355676228|seashell|Unknown|Unknown|18|eseesecally| +645|AAAAAAAAEICAAAAA|2000-10-27||Fast fundamental centres can agree etc about sure councils. So main details get. Impossible outcomes worry right critical opponents; cars shall drive to the|7.81|5.07|9002009|exportimaxi #6|13|sailing|8|Sports|242|ableeseable|N/A|pale1146850355676228|slate|Bunch|Unknown|40|antiesecally| +646|AAAAAAAAGICAAAAA|1997-10-27|1999-10-27|Domestic, reasonable activities lose closer; here super problems can mak|3.86|3.43|10006001|corpunivamalg #1|6|musical|10|Electronics|267|ationcallyable|N/A|086030725342rosy9705|spring|Dram|Unknown|95|callyesecally| +647|AAAAAAAAGICAAAAA|1999-10-28|2001-10-26|Domestic, reasonable activities lose closer; here super problems can mak|5.95|5.35|1003002|exportiamalg #2|6|maternity|1|Women|267|ationcallyable|medium|086030725342rosy9705|sienna|Dozen|Unknown|96|ationesecally| +648|AAAAAAAAGICAAAAA|2001-10-27||Domestic, reasonable activities lose closer; here super problems can mak|7.16|4.58|7015009|scholarnameless #9|6|tables|7|Home|267|ationcallyable|N/A|59083navy95563959360|sienna|Ton|Unknown|52|eingesecally| +649|AAAAAAAAJICAAAAA|1997-10-27||Similar, public interests need. Slowly final differe|3.87|1.70|6003002|exporticorp #2|3|gold|6|Jewelry|602|ablebarcally|N/A|499528104387wheat670|plum|Bundle|Unknown|18|n stesecally| +650|AAAAAAAAKICAAAAA|1997-10-27|2000-10-26|Easier economic con|0.34|0.22|4003001|exportiedu pack #1|3|kids|4|Shoes|453|priantiese|medium|0358894453663wheat54|plum|N/A|Unknown|28|barantically| +651|AAAAAAAAKICAAAAA|2000-10-27||Pictures develop only lexical months. Simple d|0.94|0.22|9010004|univunivamalg #4|3|travel|9|Books|948|priantiese|N/A|4476834177tan5359961|saddle|Bunch|Unknown|7|oughtantically| +652|AAAAAAAAMICAAAAA|1997-10-27|1999-10-27|Too prime years must fear most symptoms. Domestic, royal americans shall not check. Domestic, necessary rights enter in a years. Good, surprising crimes may not choose very thro|5.61|1.68|3001001|amalgexporti #1|1|newborn|3|Children|19|n stought|extra large|20spring253344535987|sky|Each|Unknown|30|ableantically| +653|AAAAAAAAMICAAAAA|1999-10-28|2001-10-26|Too prime years must fear most symptoms. Domestic, royal americans shall not check. Domestic, necessary rights enter in a years. Good, surprising crimes may not choose very thro|0.76|1.68|3001001|corpmaxi #6|6|parenting|9|Books|19|n stought|N/A|20spring253344535987|rose|Dram|Unknown|34|priantically| +654|AAAAAAAAMICAAAAA|2001-10-27||Still, different sons will save ago male, other decades; concerned functions prevent painfully important services. |7.67|1.68|9010007|univunivamalg #7|10|travel|9|Books|119|n stought|N/A|20spring253344535987|saddle|Tbl|Unknown|33|eseantically| +655|AAAAAAAAPICAAAAA|1997-10-27||Children would |2.05|1.08|2002002|importoimporto #2|2|shirts|2|Men|426|callyableese|N/A|spring03724762637202|smoke|Bunch|Unknown|7|antiantically| +656|AAAAAAAAAJCAAAAA|1997-10-27|2000-10-26|Previous pictures resent and so on |1.31|0.65|10015011|scholaramalgamalg #11|15|portable|10|Electronics|576|callyationanti|N/A|40118568545098linen3|saddle|Dozen|Unknown|28|callyantically| +657|AAAAAAAAAJCAAAAA|2000-10-27||Events stand certain, social friends; else prime jobs become more elements. Ordinary, teenage meals fly golden, dominant files. Then financial groups avoid partly from the cats; needs |1.44|0.57|1003002|exportiamalg #2|3|maternity|1|Women|576|callyationanti|extra large|62731snow49320890137|yellow|Box|Unknown|99|ationantically| +658|AAAAAAAACJCAAAAA|1997-10-27|1999-10-27|Also poor relations take equal, metropolitan courses. Peculiar, popular authorities lose|82.11|57.47|8013009|exportimaxi #9|13|sailing|8|Sports|503|pribaranti|N/A|614magenta0441373466|pale|Pallet|Unknown|11|eingantically| +659|AAAAAAAACJCAAAAA|1999-10-28|2001-10-26|Also poor relations take equal, metropolitan courses. Peculiar, popular authorities lose|0.85|57.47|6005004|scholarcorp #4|5|earings|6|Jewelry|138|pribaranti|N/A|614magenta0441373466|thistle|Gram|Unknown|14|n stantically| +660|AAAAAAAACJCAAAAA|2001-10-27||Totally bitter findings see from the streets. Difficult, dramatic prefere|35.09|57.47|4002001|importoedu pack #1|5|mens|4|Shoes|232|pribaranti|N/A|5678030turquoise6363|light|N/A|Unknown|11|barcallycally| +661|AAAAAAAAFJCAAAAA|1997-10-27||Further internal females might sit parents. Resources match|4.56|1.86|10014016|edu packamalgamalg #16|14|automotive|10|Electronics|141|oughteseought|N/A|pink6229832898534298|purple|Ton|Unknown|28|oughtcallycally| +662|AAAAAAAAGJCAAAAA|1997-10-27|2000-10-26|European roads see tomorrow then central causes. Friendly, molecular creatures sweep as the police. Simply waste techniques control directors. Early, increa|0.52|0.42|5002001|importoscholar #1|2|country|5|Music|102|ablebarought|N/A|42salmon901088413526|metallic|Bundle|Unknown|51|ablecallycally| +663|AAAAAAAAGJCAAAAA|2000-10-27||European roads see tomorrow then central causes. Friendly, molecular creatures sweep as the police. Simply waste techniques control directors. Early, increa|69.48|25.01|5002001|maxinameless #4|9|optics|8|Sports|102|ablebarought|N/A|592927royal756165780|tomato|Each|Unknown|4|pricallycally| +664|AAAAAAAAIJCAAAAA|1997-10-27|1999-10-27|Vulnerable circumstances move main, top women; given, top partners may make then religious documents. Trees ought to travel extra, inc images. Ne|0.29|0.25|6001001|amalgcorp #1|1|birdal|6|Jewelry|996|callyn stn st|N/A|542730smoke604087373|smoke|Carton|Unknown|2|esecallycally| +665|AAAAAAAAIJCAAAAA|1999-10-28|2001-10-26|Advanced strangers roll total imports. Directly assistant inches should not understa|3.13|0.25|8003006|exportinameless #6|1|basketball|8|Sports|810|callyn stn st|N/A|29334286turquoise253|turquoise|Bunch|Unknown|42|anticallycally| +666|AAAAAAAAIJCAAAAA|2001-10-27||General, powerful signals may not lead so actually other kids. Rather light leaders might find any longer obvious, back affairs. Origins may pay just difficulties.|5.41|3.78|5004001|edu packscholar #1|1|classical|5|Music|545|callyn stn st|N/A|29334286turquoise253|turquoise|Tbl|Unknown|17|callycallycally| +667|AAAAAAAALJCAAAAA|1997-10-27||Lexical areas would not start ago in the beings. Territorial districts must fall even economic|8.61|5.07|4002002|importoedu pack #2|2|mens|4|Shoes|37|ationpri|economy|2442519green46854672|white|Dram|Unknown|32|ationcallycally| +668|AAAAAAAAMJCAAAAA|1997-10-27|2000-10-26|Needs would go u|1.40|1.06|5003001|exportischolar #1|3|pop|5|Music|566|callycallyanti|N/A|80309146273046lawn18|red|Each|Unknown|96|eingcallycally| +669|AAAAAAAAMJCAAAAA|2000-10-27||Alone joint reports must not prove ahead scenes; suitable, free events take nearly. Results draw systems. Earlier given rules paint still. Close, serious facilities can become later too l|9.67|4.25|5003001|importoscholar #2|2|country|5|Music|311|callycallyanti|N/A|80309146273046lawn18|cyan|Oz|Unknown|77|n stcallycally| +670|AAAAAAAAOJCAAAAA|1997-10-27|1999-10-27|Earnings get far new races. Roughly thick cases should not meet over years. Attractive, social actions may not help systems. Nice markets know at a expectations; points becom|4.43|2.39|4001001|amalgedu pack #1|1|womens|4|Shoes|107|ationbarought|petite|896tomato34083078629|saddle|Case|Unknown|49|barationcally| +671|AAAAAAAAOJCAAAAA|1999-10-28|2001-10-26|Earnings get far new races. Roughly thick cases should not meet over years. Attractive, social actions may not help systems. Nice markets know at a expectations; points becom|9.53|2.39|4001001|exportischolar #2|1|pop|5|Music|32|ablepri|N/A|104730269papaya57543|white|Pallet|Unknown|38|oughtationcally| +672|AAAAAAAAOJCAAAAA|2001-10-27||Earnings get far new races. Roughly thick cases should not meet over years. Attractive, social actions may not help systems. Nice markets know at a expectations; points becom|0.83|2.39|4001001|corpnameless #3|1|furniture|7|Home|663|pricallycally|N/A|104730269papaya57543|violet|Gross|Unknown|24|ableationcally| +673|AAAAAAAABKCAAAAA|1997-10-27||Of course heavy persons get now implications. Phases show even. So old women develop; big, other jeans drive frantically official shots. Facts might disturb too new, gentle children. G|0.79|0.50|8009004|maxinameless #4|9|optics|8|Sports|258|eingantiable|N/A|sienna80141060566892|thistle|Tbl|Unknown|57|priationcally| +674|AAAAAAAACKCAAAAA|1997-10-27|2000-10-26|Very conventional quantities would turn on a clients.|4.49|2.46|4002001|importoedu pack #1|2|mens|4|Shoes|534|eseprianti|economy|410red33584771041109|steel|Box|Unknown|19|eseationcally| +675|AAAAAAAACKCAAAAA|2000-10-27||Very conventional quantities would turn on a clients.|1.37|0.57|4002001|scholarbrand #6|5|blinds/shades|7|Home|534|eseprianti|N/A|625144snow1620026342|misty|Each|Unknown|9|antiationcally| +676|AAAAAAAAEKCAAAAA|1997-10-27|1999-10-27|Proposal|1.65|0.64|3002001|importoexporti #1|2|infants|3|Children|813|prioughteing|large|88191891snow86642064|peach|Oz|Unknown|35|callyationcally| +677|AAAAAAAAEKCAAAAA|1999-10-28|2001-10-26|Children |1.16|0.64|3002001|maxiunivamalg #15|9|televisions|10|Electronics|316|callyoughtpri|N/A|931peru3479079225269|snow|Case|Unknown|25|ationationcally| +678|AAAAAAAAEKCAAAAA|2001-10-27||Alive, permanent forms kiss like a feelings. Conditions shall use within a figures; extra, happy leaves vote virtually in a options. Women can|23.22|9.05|3002001|edu packamalg #1|4|swimwear|1|Women|316|callyoughtpri|extra large|738732875910white759|salmon|Ounce|Unknown|30|eingationcally| +679|AAAAAAAAHKCAAAAA|1997-10-27||Eyes read clearly only good resources. Illustrations spend finally. Permanent, southern times would take economic da|2.97|1.45|6010008|univbrand #8|10|jewelry boxes|6|Jewelry|316|callyoughtpri|N/A|82372antique13301933|peach|Case|Unknown|73|n stationcally| +680|AAAAAAAAIKCAAAAA|1997-10-27|2000-10-26|All right instruments would ensure perhaps old years. Pale, given decisions prevent heavily significant, industrial authorities. U|9.05|7.05|4003001|exportiedu pack #1|3|kids|4|Shoes|57|ationanti|small|2349gainsboro5383118|khaki|Ounce|Unknown|36|bareingcally| +681|AAAAAAAAIKCAAAAA|2000-10-27||All right instruments would ensure perhaps old years. Pale, given decisions prevent heavily significant, industrial authorities. U|3.18|2.86|4003001|amalgexporti #2|1|newborn|3|Children|57|ationanti|small|0014orchid3016167652|slate|Oz|Unknown|49|oughteingcally| +682|AAAAAAAAKKCAAAAA|1997-10-27|1999-10-27|Workers give arguments. Potential lads might mean like a statements. Foreign presents should gain hostile, long others. Lega|1.46|0.93|2003001|exportiimporto #1|3|pants|2|Men|251|oughtantiable|extra large|slate442351022671268|sienna|Ounce|Unknown|9|ableeingcally| +683|AAAAAAAAKKCAAAAA|1999-10-28|2001-10-26|Workers give arguments. Potential lads might mean like a statements. Foreign presents should gain hostile, long others. Lega|92.53|0.93|7008004|namelessbrand #4|3|lighting|7|Home|344|eseesepri|N/A|4615119100413papaya2|antique|Bunch|Unknown|47|prieingcally| +684|AAAAAAAAKKCAAAAA|2001-10-27||Eyes make yet; alone, new words show for a points. As private parties stick elsewhere needs. Acutel|3.86|2.81|7008004|importoimporto #1|3|shirts|2|Men|214|eseesepri|extra large|4615119100413papaya2|royal|Dozen|Unknown|35|eseeingcally| +685|AAAAAAAANKCAAAAA|1997-10-27||Also top references ma|19.90|12.13|5002002|importoscholar #2|2|country|5|Music|100|barbarought|N/A|7397362670913white17|powder|Carton|Unknown|32|antieingcally| +686|AAAAAAAAOKCAAAAA|1997-10-27|2000-10-26|There welcome references must lower. Legal, broken houses may not note both large efforts; technical, agricultural patterns must not make strategic children. |2.33|1.30|9008003|namelessmaxi #3|8|romance|9|Books|8|eing|N/A|8971479409salmon6636|lace|Each|Unknown|43|callyeingcally| +687|AAAAAAAAOKCAAAAA|2000-10-27||There welcome references must lower. Legal, broken houses may not note both large efforts; technical, agricultural patterns must not make strategic children. |4.06|2.80|9008003|edu packbrand #8|14|estate|6|Jewelry|8|eing|N/A|8971479409salmon6636|rose|Unknown|Unknown|23|ationeingcally| +688|AAAAAAAAALCAAAAA|1997-10-27|1999-10-27|Now new goods get outstanding, surprising changes; almost new advertisements pay successful, concerned member|2.87|1.66|8014009|edu packmaxi #9|14|tennis|8|Sports|181|oughteingought|N/A|080014734thistle6338|goldenrod|Cup|Unknown|36|eingeingcally| +689|AAAAAAAAALCAAAAA|1999-10-28|2001-10-26|Supreme, minor years take forward needs. Concepts will not spend kind claims. Tired, retail proposals could not go etc round arts. Farmers could not leave continuous|2.07|1.66|8014009|amalgimporto #2|14|accessories|2|Men|181|oughteingought|petite|080014734thistle6338|olive|Oz|Unknown|27|n steingcally| +690|AAAAAAAAALCAAAAA|2001-10-27||Supreme, minor years take forward needs. Concepts will not spend kind claims. Tired, retail proposals could not go etc round arts. Farmers could not leave continuous|2.10|1.66|8014009|exportiedu pack #1|3|kids|4|Shoes|221|oughtableable|small|181106115smoke406038|sky|Tsp|Unknown|22|barn stcally| +691|AAAAAAAADLCAAAAA|1997-10-27||Recent, french conservatives cannot get somehow; decisions save accordingly happy thousands. Seriously good years argue then golden attacks. Just wide eyes drink underground likely, fin|0.09|0.03|8007002|brandnameless #2|7|hockey|8|Sports|104|esebarought|N/A|1457magenta883734948|papaya|Pound|Unknown|19|oughtn stcally| +692|AAAAAAAAELCAAAAA|1997-10-27|2000-10-26|Yards would not explore apparently with the attitudes. Difficult circumstances ought to work too to a interactions. Royal, slim states must produce |4.95|2.52|2002001|importoimporto #1|2|shirts|2|Men|233|pripriable|medium|450lace3292318822805|plum|Cup|Unknown|13|ablen stcally| +693|AAAAAAAAELCAAAAA|2000-10-27||Communities show nearly. All pa|6.47|2.52|7002004|importobrand #4|2|bedding|7|Home|86|pripriable|N/A|450lace3292318822805|sandy|Tbl|Unknown|4|prin stcally| +694|AAAAAAAAGLCAAAAA|1997-10-27|1999-10-27|Educational groups|0.70|0.53|8010005|univmaxi #5|10|pools|8|Sports|249|n steseable|N/A|73756764621466blush9|seashell|Pound|Unknown|23|esen stcally| +695|AAAAAAAAGLCAAAAA|1999-10-28|2001-10-26|Houses reflect so. Thereby sig|2.08|0.53|4002002|importoedu pack #2|2|mens|4|Shoes|249|n steseable|economy|73756764621466blush9|maroon|Ounce|Unknown|27|antin stcally| +696|AAAAAAAAGLCAAAAA|2001-10-27||Likely, false children please lengthy services. Absolutely statutory representations should get s|66.67|34.00|7002007|importobrand #7|2|bedding|7|Home|78|n steseable|N/A|867742235steel028826|sienna|N/A|Unknown|65|callyn stcally| +697|AAAAAAAAJLCAAAAA|1997-10-27||Public rights talk more twice new duties; political criteria ought to balance in the years. Things try then independent personnel. Likel|9.51|8.36|3003002|exportiexporti #2|3|toddlers|3|Children|79|n station|small|358080827pink6716235|papaya|Tbl|Unknown|36|ationn stcally| +698|AAAAAAAAKLCAAAAA|1997-10-27|2000-10-26|Disastrous, other concessions surprise heavy cars; now economic homes place; sudden, social results may get raw, just publications. Only awful condition|2.43|1.04|8004009|edu packnameless #9|4|camping|8|Sports|513|prioughtanti|N/A|2975383953123rosy373|lawn|Tsp|Unknown|44|eingn stcally| +699|AAAAAAAAKLCAAAAA|2000-10-27||Meetings used to rectify functions. Sharply broken businesses will not see over western efforts. British minutes m|9.39|6.47|9011004|amalgunivamalg #4|11|cooking|9|Books|513|prioughtanti|N/A|2975383953123rosy373|metallic|Pound|Unknown|5|n stn stcally| +700|AAAAAAAAMLCAAAAA|1997-10-27|1999-10-27|Members might surrender relatively now standard friends. Soviet thanks go either fortunate arrangements. Main manufacturers must try into a police. Almost difficult plans must |2.43|0.87|8008001|namelessnameless #1|8|outdoor|8|Sports|499|n stn stese|N/A|769359366317navajo07|salmon|Ounce|Unknown|25|barbaration| +701|AAAAAAAAMLCAAAAA|1999-10-28|2001-10-26|Members might surrender relatively now standard friends. Soviet thanks go either fortunate arrangements. Main manufacturers must try into a police. Almost difficult plans must |5.08|0.87|4002002|importoedu pack #2|8|mens|4|Shoes|16|callyought|small|769359366317navajo07|purple|Pound|Unknown|41|oughtbaration| +702|AAAAAAAAMLCAAAAA|2001-10-27||Members might surrender relatively now standard friends. Soviet thanks go either fortunate arrangements. Main manufacturers must try into a police. Almost difficult plans must |43.04|0.87|1003001|exportiamalg #1|8|maternity|1|Women|16|callyought|large|769359366317navajo07|maroon|Cup|Unknown|5|ablebaration| +703|AAAAAAAAPLCAAAAA|1997-10-27||Well important projects free too upper members. Additional, main emissions get with the officer|11.25|4.83|6012008|importobrand #8|12|costume|6|Jewelry|645|antiesecally|N/A|6wheat68000807268686|steel|Gram|Unknown|26|pribaration| +704|AAAAAAAAAMCAAAAA|1997-10-27|2000-10-26|Prime, racial rocks enable then reduced, wonderful num|7.43|5.27|10003008|exportiunivamalg #8|3|dvd/vcr players|10|Electronics|175|antiationought|N/A|6591607617slate99067|deep|Ounce|Unknown|43|esebaration| +705|AAAAAAAAAMCAAAAA|2000-10-27||Prime, racial rocks enable then reduced, wonderful num|7.90|5.27|10003008|namelessmaxi #10|8|romance|9|Books|175|antiationought|N/A|55orchid702654931461|slate|Ounce|Unknown|27|antibaration| +706|AAAAAAAACMCAAAAA|1997-10-27|1999-10-27|Times take particularly below new dogs; black, public expectations shall no|4.67|1.49|10001010|amalgunivamalg #10|1|cameras|10|Electronics|191|oughtn stought|N/A|56035lime42356381162|lemon|N/A|Unknown|47|callybaration| +707|AAAAAAAACMCAAAAA|1999-10-28|2001-10-26|Difficult, sorry women like there before a parties. Wholly sure lives would establish sure, top hands. Manufacturing, bad ad|0.78|1.49|3004002|edu packexporti #2|4|school-uniforms|3|Children|191|oughtn stought|economy|56035lime42356381162|sienna|Gross|Unknown|63|ationbaration| +708|AAAAAAAACMCAAAAA|2001-10-27||Absol|4.13|1.85|10008012|namelessunivamalg #12|4|scanners|10|Electronics|191|oughtn stought|N/A|56035lime42356381162|puff|Oz|Unknown|1|eingbaration| +709|AAAAAAAAFMCAAAAA|1997-10-27||There certain teams offer patients. British branches lay doubts; architects could remember always various thousands. Relationshi|0.24|0.18|5003002|exportischolar #2|3|pop|5|Music|417|ationoughtese|N/A|84peach4718574522150|plum|Tbl|Unknown|55|n stbaration| +710|AAAAAAAAGMCAAAAA|1997-10-27|2000-10-26|More important names induce; now similar standards will train correctly times. Ex|9.23|3.78|9003003|exportimaxi #3|3|computers|9|Books|28|eingable|N/A|82rose36098315995834|navajo|Gram|Unknown|57|baroughtation| +711|AAAAAAAAGMCAAAAA|2000-10-27||More important names induce; now similar standards will train correctly times. Ex|3.18|3.78|1003002|exportiamalg #2|3|maternity|1|Women|846|eingable|medium|82rose36098315995834|red|Box|Unknown|1|oughtoughtation| +712|AAAAAAAAIMCAAAAA|1997-10-27|1999-10-27|Offences would not slip sometimes strange questions; causal schemes say successfully by the moments; reasons focus foreign years. Merely red|9.44|6.04|2001001|amalgimporto #1|1|accessories|2|Men|326|callyablepri|medium|3moccasin10926838329|rosy|Bundle|Unknown|12|ableoughtation| +713|AAAAAAAAIMCAAAAA|1999-10-28|2001-10-26|Offences would not slip sometimes strange questions; causal schemes say successfully by the moments; reasons focus foreign years. Merely red|2.98|1.22|2002002|importoimporto #2|2|shirts|2|Men|296|callyn stable|extra large|3moccasin10926838329|plum|Dozen|Unknown|48|prioughtation| +714|AAAAAAAAIMCAAAAA|2001-10-27||Prev|0.91|1.22|4004001|edu packedu pack #1|2|athletic|4|Shoes|296|callyn stable|medium|50902951514indian548|steel|Ton|Unknown|31|eseoughtation| +715|AAAAAAAALMCAAAAA|1997-10-27||Scottish speakers support finally fundamental, severe days. Reasonably labour images would ease. Complete, different guards must not relieve structures; high, good windows will f|47.77|39.64|4003002|exportiedu pack #2|3|kids|4|Shoes|302|ablebarpri|large|10744ghost6155265315|sienna|Ton|Unknown|10|antioughtation| +716|AAAAAAAAMMCAAAAA|1997-10-27|2000-10-26|Never following relations describe; scientific, effective rates might let initially topics. Thorough, sufficient causes would find slowly at an conditions; as great stations meet rather. |3.87|3.28|10009003|maxiunivamalg #3|9|televisions|10|Electronics|389|n steingpri|N/A|041346sienna01252657|cornsilk|Gross|Unknown|8|callyoughtation| +717|AAAAAAAAMMCAAAAA|2000-10-27||Numbers arouse still during a characteristics. Poi|6.85|2.12|1001002|amalgamalg #2|9|dresses|1|Women|181|oughteingought|small|7561925449288wheat05|slate|Bunch|Unknown|24|ationoughtation| +718|AAAAAAAAOMCAAAAA|1997-10-27|1999-10-27|Cool details could not influence really disastrous, old songs; later specific areas used to arise by a books. Aware honours support. There fixed services happen plain days. Cri|45.47|20.46|10005005|scholarunivamalg #5|5|karoke|10|Electronics|133|pripriought|N/A|16tomato404466246619|rosy|Unknown|Unknown|14|eingoughtation| +719|AAAAAAAAOMCAAAAA|1999-10-28|2001-10-26|Special, new attacks leave here between the responsibilities. Approximately new months turn especially. Natural visitors respond str|4.36|20.46|10005005|brandnameless #10|5|hockey|8|Sports|133|pripriought|N/A|311135506555rosy2447|sienna|N/A|Unknown|48|n stoughtation| +720|AAAAAAAAOMCAAAAA|2001-10-27||Also general bodies prevent very large ways. Plans must tackle far for a children. Most final companies matter now previous sourc|4.85|20.46|9016001|corpunivamalg #1|5|mystery|9|Books|51|oughtanti|N/A|1779741530white73501|rosy|Dram|Unknown|2|barableation| +721|AAAAAAAABNCAAAAA|1997-10-27||Criminal companies may emerge sometimes children. Urban, other efforts dominate policies. Very right fans drive briti|9.67|5.12|7010002|univnameless #2|10|flatware|7|Home|131|oughtpriought|N/A|9187921909salmon7429|ivory|Tbl|Unknown|9|oughtableation| +722|AAAAAAAACNCAAAAA|1997-10-27|2000-10-26|Already other elements will not matter statistically others. Guns ex|3.38|1.45|8015003|scholarmaxi #3|15|fishing|8|Sports|289|n steingable|N/A|422peach113152202125|steel|Bundle|Unknown|59|ableableation| +723|AAAAAAAACNCAAAAA|2000-10-27||Particular governors call for the chairs. Other terms used to serve accurat|1.26|1.45|8015003|edu packmaxi #4|4|entertainments|9|Books|289|n steingable|N/A|422peach113152202125|hot|Case|Unknown|32|priableation| +724|AAAAAAAAENCAAAAA|1997-10-27|1999-10-27|Slightly large companies might not manage. Dist|4.21|1.59|6016005|corpbrand #5|16|consignment|6|Jewelry|315|antioughtpri|N/A|8084029khaki68278747|sandy|Cup|Unknown|37|eseableation| +725|AAAAAAAAENCAAAAA|1999-10-28|2001-10-26|Then mental sides should not shake to a colleagues. Particular, imaginative others must begin then|1.73|0.57|9012006|importounivamalg #6|16|home repair|9|Books|315|antioughtpri|N/A|492573149878plum0824|khaki|Bunch|Unknown|2|antiableation| +726|AAAAAAAAENCAAAAA|2001-10-27||Then mental sides should not shake to a colleagues. Particular, imaginative others must begin then|2.89|1.82|9012006|edu packimporto #1|16|sports-apparel|2|Men|266|antioughtpri|extra large|149175607176slate005|pale|Bundle|Unknown|11|callyableation| +727|AAAAAAAAHNCAAAAA|1997-10-27||Bottles take significantly popular, official seconds. Ever inadequate environments should go largely skills. Plans make males. There labour years get so different, oth|2.18|1.48|1003002|exportiamalg #2|3|maternity|1|Women|968|eingcallyn st|medium|33263286642midnight5|honeydew|Each|Unknown|26|ationableation| +728|AAAAAAAAINCAAAAA|1997-10-27|2000-10-26|Organisations would consider both whole pubs. Adults like; revolutionary, personal officers verify much visual officers; hidden rights would market working prayers. Trial|1.58|1.16|4004001|edu packedu pack #1|4|athletic|4|Shoes|23|priable|small|06073144876tomato416|goldenrod|N/A|Unknown|66|eingableation| +729|AAAAAAAAINCAAAAA|2000-10-27||Significant times used to service then hidden students; home other policies become above a services. Very|0.24|0.07|9004010|edu packmaxi #10|4|entertainments|9|Books|10|priable|N/A|06073144876tomato416|peru|Unknown|Unknown|11|n stableation| +730|AAAAAAAAKNCAAAAA|1997-10-27|1999-10-27|Other products follow for a eyes. Greatly normal models shall assess for example immediately easy lands. Impossible, foreign plans criticise however also free years; known, ruling pol|0.94|0.83|1002001|importoamalg #1|2|fragrances|1|Women|583|prieinganti|economy|27384290751pale09457|peach|Pound|Unknown|60|barpriation| +731|AAAAAAAAKNCAAAAA|1999-10-28|2001-10-26|Other products follow for a eyes. Greatly normal models shall assess for example immediately easy lands. Impossible, foreign plans criticise however also free years; known, ruling pol|9.47|0.83|1002001|amalgmaxi #12|1|arts|9|Books|583|prieinganti|N/A|27384290751pale09457|lawn|Gross|Unknown|2|oughtpriation| +732|AAAAAAAAKNCAAAAA|2001-10-27||French, current members join available reasons; committees know to a stations. Continually domestic improvements may look generally like the |1.20|0.83|7016003|corpnameless #3|16|furniture|7|Home|583|prieinganti|N/A|6061471thistle319951|thistle|Oz|Unknown|98|ablepriation| +733|AAAAAAAANNCAAAAA|1997-10-27||Particular, previous machi|1.40|0.82|8015004|scholarmaxi #4|15|fishing|8|Sports|721|oughtableation|N/A|4859825powder1161538|moccasin|Lb|Unknown|54|pripriation| +734|AAAAAAAAONCAAAAA|1997-10-27|2000-10-26|Prices wou|5.81|3.25|4003001|exportiedu pack #1|3|kids|4|Shoes|23|priable|small|8turquoise2955448303|ivory|Lb|Unknown|9|esepriation| +735|AAAAAAAAONCAAAAA|2000-10-27||Consumers pull essentially private records. Dear things may see yet birds. Divisions see; general, other publications shall stand there parts. Perhaps minor levels work coming, vi|2.46|3.25|4003001|importoedu pack #2|3|mens|4|Shoes|23|priable|small|8turquoise2955448303|sienna|Case|Unknown|23|antipriation| +736|AAAAAAAAAOCAAAAA|1997-10-27|1999-10-27|Truly rare w|1.63|0.74|5001001|amalgscholar #1|1|rock|5|Music|30|barpri|N/A|843941203saddle22225|purple|Each|Unknown|16|callypriation| +737|AAAAAAAAAOCAAAAA|1999-10-28|2001-10-26|Truly rare w|40.11|23.26|5001001|edu packunivamalg #6|1|sports|9|Books|30|barpri|N/A|843941203saddle22225|pale|Pound|Unknown|11|ationpriation| +738|AAAAAAAAAOCAAAAA|2001-10-27||Truly rare w|8.20|6.56|5001001|amalgscholar #1|1|rock|5|Music|30|barpri|N/A|986595579saddle19945|tan|Box|Unknown|4|eingpriation| +739|AAAAAAAADOCAAAAA|1997-10-27||Here remaining years might remember even marine major shelves. Lovely, low movements must find with a aspects. Opposite, reliable lengths cannot feed. English, maximum tim|75.80|56.09|2004002|edu packimporto #2|4|sports-apparel|2|Men|647|ationesecally|large|300312385920khaki308|medium|Each|Unknown|21|n stpriation| +740|AAAAAAAAEOCAAAAA|1997-10-27|2000-10-26|High cases cannot make that is unlike films; certain, social books ought to exchange like the measures. Organisations sit far members. Empty, international skills might make to a movements. |6.47|5.56|1003001|exportiamalg #1|3|maternity|1|Women|959|n stantin st|medium|930748455361747plum5|white|Pound|Unknown|57|bareseation| +741|AAAAAAAAEOCAAAAA|2000-10-27||Phenomena remind naturally. Lead|1.22|0.59|1003001|amalgunivamalg #10|11|cooking|9|Books|42|n stantin st|N/A|plum6284277666996326|puff|Bundle|Unknown|40|oughteseation| +742|AAAAAAAAGOCAAAAA|1997-10-27|1999-10-27|Historical, economic lights shall stand much big, odd proposals. Rather grateful branches ought to take. Northern, high miles must ask increasingly. Once chronic|4.37|2.31|9011011|amalgunivamalg #11|11|cooking|9|Books|281|oughteingable|N/A|191507315purple21697|salmon|Tsp|Unknown|12|ableeseation| +743|AAAAAAAAGOCAAAAA|1999-10-28|2001-10-26|Balls must not avoid today central problems. Yesterday existing friends must cling aspects. Principles find still also tory years. New, traditional|22.18|2.31|4004002|edu packedu pack #2|11|athletic|4|Shoes|741|oughteingable|extra large|1khaki51858519433395|sandy|Dram|Unknown|15|prieseation| +744|AAAAAAAAGOCAAAAA|2001-10-27||Effects differ surfaces; happy, economic aspects clear great parties. Much sophisticated calls should believe|4.41|2.86|4004002|exportiexporti #1|11|toddlers|3|Children|108|eingbarought|extra large|1khaki51858519433395|turquoise|Box|Unknown|42|eseeseation| +745|AAAAAAAAJOCAAAAA|1997-10-27||Millions give like the models. To|7.43|5.57|4001002|amalgedu pack #2|1|womens|4|Shoes|449|n steseese|small|4954659maroon5215481|powder|Ounce|Unknown|10|antieseation| +746|AAAAAAAAKOCAAAAA|1997-10-27|2000-10-26|National, superior costs act |7.31|4.02|3002001|importoexporti #1|2|infants|3|Children|301|oughtbarpri|medium|734053905919110puff5|plum|Lb|Unknown|58|callyeseation| +747|AAAAAAAAKOCAAAAA|2000-10-27||Other changes ought to mean from a troops. Fals|0.33|0.26|3002001|importomaxi #8|12|guns|8|Sports|301|oughtbarpri|N/A|28seashell4811261542|plum|Case|Unknown|11|ationeseation| +748|AAAAAAAAMOCAAAAA|1997-10-27|1999-10-27|Weeks will claim at a hands. Cuts meet smart, relevant lawyers. Enormous sides should |23.89|10.75|7016009|corpnameless #9|16|furniture|7|Home|118|eingoughtought|N/A|green704815986576567|plum|Tbl|Unknown|24|eingeseation| +749|AAAAAAAAMOCAAAAA|1999-10-28|2001-10-26|Different, certain premises bring satisfactory, short parties. Honest, conservative enemie|0.93|10.75|7016009|brandmaxi #6|7|reference|9|Books|118|eingoughtought|N/A|9snow295643471232748|dark|Gram|Unknown|100|n steseation| +750|AAAAAAAAMOCAAAAA|2001-10-27||Techniques ought to count mainly complex, real members. Really commercial signs think above emotional, conservative flights. Men ask relatively di|4.33|2.07|7016009|amalgamalg #1|7|dresses|1|Women|342|ableesepri|medium|934metallic870100140|purple|N/A|Unknown|23|barantiation| +751|AAAAAAAAPOCAAAAA|1997-10-27||Bishops could confirm; rates rot very pp.. Prisoners will want old countries. Too po|3.71|1.63|7005002|scholarbrand #2|5|blinds/shades|7|Home|115|antioughtought|N/A|89269942rosy34556580|pale|Carton|Unknown|23|oughtantiation| +752|AAAAAAAAAPCAAAAA|1997-10-27|2000-10-26|Trustees grow well thereby national attitudes. Social, excellent bacteria contain permanent gaps. Only dynamic uses ought to halt very long, bright men; japanese, distin|3.31|2.87|6003001|exporticorp #1|3|gold|6|Jewelry|276|callyationable|N/A|45801569366798wheat0|red|Box|Unknown|21|ableantiation| +753|AAAAAAAAAPCAAAAA|2000-10-27||New, personal groups could not say costs; there social issues would take undoubtedly days; domestic, large eyes plan more possible steps; great, new expe|4.10|2.87|6003001|edu packamalgamalg #6|3|automotive|10|Electronics|207|callyationable|N/A|89353518saddle753945|saddle|Ounce|Unknown|43|priantiation| +754|AAAAAAAACPCAAAAA|1997-10-27|1999-10-27|Only, s|0.31|0.19|9009011|maximaxi #11|9|science|9|Books|48|eingese|N/A|666925674166sandy303|pale|Pound|Unknown|33|eseantiation| +755|AAAAAAAACPCAAAAA|1999-10-28|2001-10-26|Only, s|5.45|0.19|9009011|amalgimporto #2|9|accessories|2|Men|48|eingese|small|412674sky04646579076|puff|Tbl|Unknown|20|antiantiation| +756|AAAAAAAACPCAAAAA|2001-10-27||Perhaps past preferences tell rather to a accounts. Very common feet can command never available final years; minutes expect recent, due employers. Altogether english shoes |9.84|0.19|9009011|importoamalgamalg #9|12|monitors|10|Electronics|48|eingese|N/A|1158974sienna7795556|tan|Tbl|Unknown|69|callyantiation| +757|AAAAAAAAFPCAAAAA|1997-10-27||Male, late p|4.92|3.19|1001002|amalgamalg #2|1|dresses|1|Women|141|oughteseought|extra large|salmon33429712347634|olive|Carton|Unknown|28|ationantiation| +758|AAAAAAAAGPCAAAAA|1997-10-27|2000-10-26|Everyday strings might wash for example about a pp.. Right physical specialists can break privately hungry journalists. Devices will not defend offices. Content, new projects should not se|2.40|1.99|5004001|edu packscholar #1|4|classical|5|Music|181|oughteingought|N/A|26peru94551104682596|orchid|Tsp|Unknown|33|eingantiation| +759|AAAAAAAAGPCAAAAA|2000-10-27||Colleges would use sales. Ag|2.99|1.99|5004001|importocorp #8|4|diamonds|6|Jewelry|181|oughteingought|N/A|837617ghost831945976|seashell|Tsp|Unknown|33|n stantiation| +760|AAAAAAAAIPCAAAAA|1997-10-27|1999-10-27|Likely, |0.80|0.64|1002001|importoamalg #1|2|fragrances|1|Women|9|n st|small|puff7091488908881250|plum|Box|Unknown|5|barcallyation| +761|AAAAAAAAIPCAAAAA|1999-10-28|2001-10-26|Likely, |1.96|0.64|7008002|namelessbrand #2|8|lighting|7|Home|312|n st|N/A|78sienna392232007183|violet|Cup|Unknown|20|oughtcallyation| +762|AAAAAAAAIPCAAAAA|2001-10-27||Likely, |6.85|0.64|7008002|scholarnameless #3|8|tables|7|Home|312|ableoughtpri|N/A|78sienna392232007183|navy|Dozen|Unknown|51|ablecallyation| +763|AAAAAAAALPCAAAAA|1997-10-27||Sorts might think full birds. New packages shall exceed sad arrangements. Problems cannot come together other employees.|1.54|0.61|7003004|exportibrand #4|3|kids|7|Home|288|eingeingable|N/A|1602062185202forest1|tan|Bunch|Unknown|6|pricallyation| +764|AAAAAAAAMPCAAAAA|1997-10-27|2000-10-26|Growing dogs can procure as total, valid teams. Continental, specific skills co|1.51|0.81|1002001|importoamalg #1|2|fragrances|1|Women|638|eingprically|small|29lavender3737541031|lavender|Ounce|Unknown|65|esecallyation| +765|AAAAAAAAMPCAAAAA|2000-10-27||Growing dogs can procure as total, valid teams. Continental, specific skills co|0.13|0.81|6007006|brandcorp #6|7|pendants|6|Jewelry|305|eingprically|N/A|27549purple597318969|lawn|Bundle|Unknown|7|anticallyation| +766|AAAAAAAAOPCAAAAA|1997-10-27|1999-10-27|Years lend upwards quickly subtle islands. Amazing, main counties must gain pounds. Changing, established variations can avoid rather defensive d|2.24|1.03|2002001|importoimporto #1|2|shirts|2|Men|859|n stantieing|medium|802529867747plum4428|lawn|N/A|Unknown|2|callycallyation| +767|AAAAAAAAOPCAAAAA|1999-10-28|2001-10-26|Activities should not meet royal reforms. There elected directors get other skills. Complaints shall procure agreements. Rati|7.67|6.36|2004002|edu packimporto #2|2|sports-apparel|2|Men|859|n stantieing|large|802529867747plum4428|pale|Lb|Unknown|43|ationcallyation| +768|AAAAAAAAOPCAAAAA|2001-10-27||Objectives ought to go too schools. Civil, political boundaries could become even whole approaches. Goods give|1.02|0.72|3001001|amalgexporti #1|1|newborn|3|Children|859|n stantieing|medium|802529867747plum4428|powder|Ton|Unknown|51|eingcallyation| +769|AAAAAAAABADAAAAA|1997-10-27||English, efficient scene|8.54|5.46|6010002|univbrand #2|10|jewelry boxes|6|Jewelry|596|callyn stanti|N/A|7986415spring3572029|saddle|Cup|Unknown|4|n stcallyation| +770|AAAAAAAACADAAAAA|1997-10-27|2000-10-26|Lawyers allow within the qualities. General, bloody regulations shall not save nea|7.93|3.40|3001001|amalgexporti #1|1|newborn|3|Children|370|barationpri|economy|8852034323sky0881133|moccasin|Tsp|Unknown|30|barationation| +771|AAAAAAAACADAAAAA|2000-10-27||Positive rules mean here low, green holidays. Also legal women make. Particularly simple reasons must list irritably. Foreign, large stars can take disputes. Theories study civil letters. Once hap|0.61|3.40|3001001|exportiunivamalg #7|1|dvd/vcr players|10|Electronics|105|antibarought|N/A|8852034323sky0881133|steel|N/A|Unknown|53|oughtationation| +772|AAAAAAAAEADAAAAA|1997-10-27|1999-10-27|So good choices accept good events; mean, effective birds remember away of course mixed vegetables. Requirements concede quite worth the steps. Heavy, big war|2.70|2.40|7016003|corpnameless #3|16|furniture|7|Home|250|barantiable|N/A|4044810335584688hot6|slate|Cup|Unknown|38|ableationation| +773|AAAAAAAAEADAAAAA|1999-10-28|2001-10-26|So good choices accept good events; mean, effective birds remember away of course mixed vegetables. Requirements concede quite worth the steps. Heavy, big war|6.16|5.29|7016003|amalgexporti #2|1|newborn|3|Children|473|priationese|petite|67224134301941plum25|sienna|Unknown|Unknown|31|priationation| +774|AAAAAAAAEADAAAAA|2001-10-27||So good choices accept good events; mean, effective birds remember away of course mixed vegetables. Requirements concede quite worth the steps. Heavy, big war|8.15|2.77|4001001|amalgedu pack #1|1|womens|4|Shoes|473|priationese|extra large|67224134301941plum25|blanched|Unknown|Unknown|34|eseationation| +775|AAAAAAAAHADAAAAA|1997-10-27||Other officers tell months. Also favorite enquiries might welcome in a women. Principles pay uncomfortably young, new others. Halfway immediate comments|2.87|0.88|1001002|amalgamalg #2|1|dresses|1|Women|626|callyablecally|medium|lace5348484830479818|turquoise|Pallet|Unknown|41|antiationation| +776|AAAAAAAAIADAAAAA|1997-10-27|2000-10-26|New, anonymous goals might paint now old authorities. Movements would remove then short generations. Violent, simple words must meet quite distant, open applications. Indian friends might not acquir|4.12|3.29|10003012|exportiunivamalg #12|3|dvd/vcr players|10|Electronics|428|eingableese|N/A|83558seashell2528465|pale|N/A|Unknown|14|callyationation| +777|AAAAAAAAIADAAAAA|2000-10-27||New, anonymous goals might paint now old authorities. Movements would remove then short generations. Violent, simple words must meet quite distant, open applications. Indian friends might not acquir|0.88|0.29|8002008|importonameless #8|3|baseball|8|Sports|645|eingableese|N/A|83558seashell2528465|grey|Oz|Unknown|41|ationationation| +778|AAAAAAAAKADAAAAA|1997-10-27|1999-10-27|Pupils invest especially boards. Lonely professionals like then readers; manufacturers lose clearly clergy. Temporary holes take aside in a users. Working, high heads identif|8.27|6.53|3002001|importoexporti #1|2|infants|3|Children|72|ableation|large|119543purple93829317|almond|Each|Unknown|36|eingationation| +779|AAAAAAAAKADAAAAA|1999-10-28|2001-10-26|Particular, low needs operate|4.66|6.53|3002001|edu packnameless #10|2|camping|8|Sports|386|ableation|N/A|3plum795057082720498|cyan|Tsp|Unknown|13|n stationation| +780|AAAAAAAAKADAAAAA|2001-10-27||||3.03||||televisions||Electronics||callyeingpri||90peach7971010775768|||Unknown|8|| +781|AAAAAAAANADAAAAA|1997-10-27||National, certain days help fully french tickets. Just heavy questions will find on a habits. Owners used to represent straight medical teams. V|0.20|0.11|1002002|importoamalg #2|2|fragrances|1|Women|288|eingeingable|large|39red225146199513947|lemon|Gross|Unknown|61|oughteingation| +782|AAAAAAAAOADAAAAA|1997-10-27|2000-10-26|Otherwise european things assist useful grounds. Big students provoke. Really certain twins ought to kee|4.67|4.15|1003001|exportiamalg #1|3|maternity|1|Women|6|cally|large|91397649saddle403023|ivory|Pallet|Unknown|19|ableeingation| +783|AAAAAAAAOADAAAAA|2000-10-27||Crazy, economic things ought to descend later. Arrangeme|5.08|4.15|3002002|importoexporti #2|3|infants|3|Children|6|cally|N/A|91397649saddle403023|misty|Pallet|Unknown|19|prieingation| +784|AAAAAAAAABDAAAAA|1997-10-27|1999-10-27|Goods mean so correct, legal systems. Just alternative banks tend then more concrete edges. Close, united chapters get only rus|1.06|0.93|7011005|amalgnameless #5|11|accent|7|Home|65|antically|N/A|09105262saddle672096|red|Cup|Unknown|89|eseeingation| +785|AAAAAAAAABDAAAAA|1999-10-28|2001-10-26|Even main fields would pinpoint. Basic teachers must seek equally. Seriously huge instructions grasp barely following ca|1.49|0.93|9008006|namelessmaxi #6|11|romance|9|Books|178|antically|N/A|252935814magenta4324|spring|Lb|Unknown|28|antieingation| +786|AAAAAAAAABDAAAAA|2001-10-27||Social teachers raise similarly from a characteristics. Able, real years study with a matters. Still severe homes share in a components. Lips shall not move limited, p|3.02|2.02|8013007|exportimaxi #7|13|sailing|8|Sports|178|antically|N/A|252935814magenta4324|midnight|Dozen|Unknown|10|callyeingation| +787|AAAAAAAADBDAAAAA|1997-10-27||British institutions all|9.28|5.47|3001002|amalgexporti #2|1|newborn|3|Children|68|eingcally|petite|39yellow713680706404|green|Dram|Unknown|50|ationeingation| +788|AAAAAAAAEBDAAAAA|1997-10-27|2000-10-26|Around happy methods sit possible, soft researchers. For example capitalist injuries might win well different, unemployed bits. Firms discuss as eit|46.84|14.05|1004001|edu packamalg #1|4|swimwear|1|Women|382|ableeingpri|petite|57580175291371grey61|violet|Tsp|Unknown|33|eingeingation| +789|AAAAAAAAEBDAAAAA|2000-10-27||Around happy methods sit possible, soft researchers. For example capitalist injuries might win well different, unemployed bits. Firms discuss as eit|8.06|3.46|1004001|importobrand #10|4|bedding|7|Home|562|ablecallyanti|N/A|57580175291371grey61|rose|Carton|Unknown|15|n steingation| +790|AAAAAAAAGBDAAAAA|1997-10-27|1999-10-27|Only, closed things ought to look stiff decades. Other problems would lay to a authors. Outer men will not matter adult, single women. Then great weeks ought to agree l|41.61|24.96|1004001|edu packamalg #1|4|swimwear|1|Women|326|callyablepri|small|174153938olive737037|navy|Dozen|Unknown|64|barn station| +791|AAAAAAAAGBDAAAAA|1999-10-28|2001-10-26|Constantly other windows assume seriously indeed other schemes. More desirable lights cou|7.67|2.30|4004002|edu packedu pack #2|4|athletic|4|Shoes|297|ationn stable|petite|174153938olive737037|smoke|Unknown|Unknown|23|oughtn station| +792|AAAAAAAAGBDAAAAA|2001-10-27||Constantly other windows assume seriously indeed other schemes. More desirable lights cou|8.87|5.23|4004002|scholarmaxi #3|4|fishing|8|Sports|297|ationn stable|N/A|174153938olive737037|sky|Lb|Unknown|3|ablen station| +793|AAAAAAAAJBDAAAAA|1997-10-27||Public, resulting tickets might carry double, firm times. Human pictur|7.09|4.82|2001002|amalgimporto #2|1|accessories|2|Men|97|ationn st|medium|076goldenrod11861575|misty|Box|Unknown|38|prin station| +794|AAAAAAAAKBDAAAAA|1997-10-27|2000-10-26|Never alternative eyes would not see final points; main, controversial studies supply|2.03|1.46|1001001|amalgamalg #1|1|dresses|1|Women|60|barcally|medium|9413violet3042942036|royal|Pound|Unknown|56|esen station| +795|AAAAAAAAKBDAAAAA|2000-10-27||Im|66.66|39.99|1001001|exportischolar #2|3|pop|5|Music|60|barcally|N/A|9413violet3042942036|powder|Case|Unknown|26|antin station| +796|AAAAAAAAMBDAAAAA|1997-10-27|1999-10-27|Partly large numbers could not tell hardly on the boys. Insects must ring ext|8.82|2.99|10004015|edu packunivamalg #15|4|audio|10|Electronics|263|pricallyable|N/A|cyan1241009288661776|mint|Gross|Unknown|27|callyn station| +797|AAAAAAAAMBDAAAAA|1999-10-28|2001-10-26|Partly large numbers could not tell hardly on the boys. Insects must ring ext|2.07|1.49|10004015|importoamalg #2|4|fragrances|1|Women|74|pricallyable|extra large|cyan1241009288661776|indian|Cup|Unknown|17|ationn station| +798|AAAAAAAAMBDAAAAA|2001-10-27||Others form suddenly|56.92|1.49|5001001|amalgscholar #1|1|rock|5|Music|358|pricallyable|N/A|cyan1241009288661776|chiffon|Ounce|Unknown|41|eingn station| +799|AAAAAAAAPBDAAAAA|1997-10-27||Clear, top associations can activate all national factors. Items could think sure skills. Fine, thin classes must not help simply only statutory|6.27|4.57|7007010|brandbrand #10|7|decor|7|Home|232|ablepriable|N/A|187889870papaya74731|lemon|Dram|Unknown|45|n stn station| +800|AAAAAAAAACDAAAAA|1997-10-27|2000-10-26|Extremely social homes could drive before sensitive relevant buildings. Noble characters provide always interesting sides. Functional reforms hide outside aware artists. States improve affairs|3.33|2.96|1004001|edu packamalg #1|4|swimwear|1|Women|93|prin st|petite|84370saddle248326998|thistle|Bunch|Unknown|2|barbareing| +801|AAAAAAAAACDAAAAA|2000-10-27||Extremely social homes could drive before sensitive relevant buildings. Noble characters provide always interesting sides. Functional reforms hide outside aware artists. States improve affairs|2.22|0.71|10016003|corpamalgamalg #3|4|wireless|10|Electronics|124|eseableought|N/A|08256789violet215736|red|N/A|Unknown|12|oughtbareing| +802|AAAAAAAACCDAAAAA|1997-10-27|1999-10-27|Hospitals know with a beaches. Following days could carry too runs. Local shares may not want completely for a statements. Here simple facts cannot get poor demands. Perfectly top hours make then |19.11|13.95|4003001|exportiedu pack #1|3|kids|4|Shoes|659|n stantically|large|4551251220tan4574555|powder|Tbl|Unknown|8|ablebareing| +803|AAAAAAAACCDAAAAA|1999-10-28|2001-10-26|Hospitals know with a beaches. Following days could carry too runs. Local shares may not want completely for a statements. Here simple facts cannot get poor demands. Perfectly top hours make then |0.58|0.20|10001005|amalgunivamalg #5|3|cameras|10|Electronics|224|eseableable|N/A|4551251220tan4574555|almond|Cup|Unknown|98|pribareing| +804|AAAAAAAACCDAAAAA|2001-10-27||Mean rates n|4.86|0.20|10001005|amalgedu pack #1|1|womens|4|Shoes|61|eseableable|small|2salmon0753699168608|powder|Ton|Unknown|9|esebareing| +805|AAAAAAAAFCDAAAAA|1997-10-27||Live, late activities feel principles. In|4.50|1.44|9008002|namelessmaxi #2|8|romance|9|Books|262|ablecallyable|N/A|puff1187971434393951|salmon|Box|Unknown|23|antibareing| +806|AAAAAAAAGCDAAAAA|1997-10-27|2000-10-26|Subsequently full views add still considerable changes. Extra names suffer conservatives. So odd hours work just real standard|2.01|1.18|7012007|importonameless #7|12|paint|7|Home|573|priationanti|N/A|49429233829seashell7|steel|Dozen|Unknown|3|callybareing| +807|AAAAAAAAGCDAAAAA|2000-10-27||Social services print quickly at a interviews. Players let; british disputes compose more blue matters. Studies could not stand finally safe enemies. Simple patterns may walk right, financial tanks. M|5.30|1.18|1001002|amalgamalg #2|1|dresses|1|Women|181|priationanti|medium|636606542862peach476|azure|Each|Unknown|43|ationbareing| +808|AAAAAAAAICDAAAAA|1997-10-27|1999-10-27|Other votes should hear rather|7.42|4.89|8005009|scholarnameless #9|5|fitness|8|Sports|321|oughtablepri|N/A|679408136sandy646004|powder|N/A|Unknown|47|eingbareing| +809|AAAAAAAAICDAAAAA|1999-10-28|2001-10-26|Front, entire hands could go so. New, passive points may think high reasonable gains. Important departments will tell always social, psychological gifts|2.55|1.27|8005009|corpnameless #10|16|furniture|7|Home|321|oughtablepri|N/A|679408136sandy646004|moccasin|Case|Unknown|37|n stbareing| +810|AAAAAAAAICDAAAAA|2001-10-27||For example known studies must identify actually easy royal titles. Certain, broad teachers light different, english structures. Weeks used to allow |3.61|1.33|9014007|edu packunivamalg #7|16|sports|9|Books|321|oughtablepri|N/A|679408136sandy646004|floral|Case|Unknown|26|baroughteing| +811|AAAAAAAALCDAAAAA|1997-10-27||Clearly actual places would supply apparently only rats.|4.34|2.64|9003008|exportimaxi #8|3|computers|9|Books|375|antiationpri|N/A|1turquoise6997994009|salmon|Dozen|Unknown|39|oughtoughteing| +812|AAAAAAAAMCDAAAAA|1997-10-27|2000-10-26|Other, british benefits begin over about the participants. Legal, short contracts receive for a procedures. Openly unlikely countries need both planes. Lines should not get very ago historical |9.51|7.60|7010003|univnameless #3|10|flatware|7|Home|74|eseation|N/A|466768peach606326716|slate|Carton|Unknown|48|ableoughteing| +813|AAAAAAAAMCDAAAAA|2000-10-27||Other, british benefits begin over about the participants. Legal, short contracts receive for a procedures. Openly unlikely countries need both planes. Lines should not get very ago historical |5.03|7.60|10005015|scholarunivamalg #15|5|karoke|10|Electronics|74|eseation|N/A|701554503762sienna55|indian|Carton|Unknown|50|prioughteing| +814|AAAAAAAAOCDAAAAA|1997-10-27|1999-10-27|Personal weeks ought to become late opposite ways. Together safe men frighten expected, southern studies. Guilty, real woods would not become great. Liable, other eyes see much. |7.44|4.01|3003001|exportiexporti #1|3|toddlers|3|Children|178|eingationought|medium|9860552papaya2758476|rose|Carton|Unknown|38|eseoughteing| +815|AAAAAAAAOCDAAAAA|1999-10-28|2001-10-26|Terms would avoid different circles. Special, alive factors might spring always eastern costs. Earnings feel british, related items. Famous systems ba|1.66|0.59|8004006|edu packnameless #6|4|camping|8|Sports|178|eingationought|N/A|9860552papaya2758476|hot|Pound|Unknown|75|antioughteing| +816|AAAAAAAAOCDAAAAA|2001-10-27||Arrangements live too for a men. Customers come so children. Directly canadian measures strengthen both in a systems. Now middle-class arts sit most in general unique groups. Black |2.23|0.59|8004006|edu packscholar #1|4|classical|5|Music|178|eingationought|N/A|9860552papaya2758476|wheat|Ounce|Unknown|54|callyoughteing| +817|AAAAAAAABDDAAAAA|1997-10-27||Men know later other pp.. Matters may campaign however different men. Formal, dependent rates deal in |0.37|0.30|6001002|amalgcorp #2|1|birdal|6|Jewelry|513|prioughtanti|N/A|thistle9587474142171|moccasin|Ton|Unknown|36|ationoughteing| +818|AAAAAAAACDDAAAAA|1997-10-27|2000-10-26|Arab roses used to keep both right, prime children; only young edges feel only little networks. Right elements get commonly parties. Foreve|7.51|3.52|3002001|importoexporti #1|2|infants|3|Children|319|n stoughtpri|large|01580529290lawn45378|salmon|Case|Unknown|3|eingoughteing| +819|AAAAAAAACDDAAAAA|2000-10-27||So small issues shall reach good in a wars. Fully genuine objectives will exper|2.83|3.52|10005004|scholarunivamalg #4|2|karoke|10|Electronics|79|n stoughtpri|N/A|01580529290lawn45378|rosy|Each|Unknown|47|n stoughteing| +820|AAAAAAAAEDDAAAAA|1997-10-27|1999-10-27|Slow soviet projects should know often expert systems. Normal investigations will receive most unaware problems; meanwhile recent seconds declare as for a grounds. Good, early preparations frame o|1.89|0.96|6010005|univbrand #5|10|jewelry boxes|6|Jewelry|48|eingese|N/A|steel839123192238119|blanched|Gram|Unknown|65|barableeing| +821|AAAAAAAAEDDAAAAA|1999-10-28|2001-10-26|Simply little days should come without a costs. Occasionally private negotiations choose please thinking emotions. Eyes ignore for instance external lights. Desira|0.32|0.25|6010005|univamalgamalg #6|10|memory|10|Electronics|28|eingese|N/A|095sandy804516793408|red|Bundle|Unknown|54|oughtableeing| +822|AAAAAAAAEDDAAAAA|2001-10-27||Then serious difficulties use|4.64|2.32|4003001|exportiedu pack #1|3|kids|4|Shoes|58|einganti|medium|095sandy804516793408|sandy|Pallet|Unknown|31|ableableeing| +823|AAAAAAAAHDDAAAAA|1997-10-27||Remote, japanese things would not need at all|45.99|26.67|9015008|scholarunivamalg #8|15|fiction|9|Books|543|prieseanti|N/A|1576powder6393462987|royal|Carton|Unknown|17|priableeing| +824|AAAAAAAAIDDAAAAA|1997-10-27|2000-10-26|Over standard lines become in particular by the things. Simple, existing devices see other proposals; interests might quit beh|2.47|1.55|1002001|importoamalg #1|2|fragrances|1|Women|246|callyeseable|large|265153369slate829688|powder|Dozen|Unknown|10|eseableeing| +825|AAAAAAAAIDDAAAAA|2000-10-27||Good ministers grow a little particular words. Additional plans ought to begin orders. Rare, free classes could fa|2.44|1.55|2004002|edu packimporto #2|4|sports-apparel|2|Men|246|callyeseable|large|272misty039453240704|bisque|Pallet|Unknown|3|antiableeing| +826|AAAAAAAAKDDAAAAA|1997-10-27|1999-10-27|African colleges wander already |3.24|2.91|6005003|scholarcorp #3|5|earings|6|Jewelry|535|antiprianti|N/A|3sky2545830119671126|khaki|Gram|Unknown|28|callyableeing| +827|AAAAAAAAKDDAAAAA|1999-10-28|2001-10-26|African colleges wander already |1.99|1.57|6005003|maxiunivamalg #12|9|televisions|10|Electronics|470|antiprianti|N/A|3sky2545830119671126|lace|Dozen|Unknown|12|ationableeing| +828|AAAAAAAAKDDAAAAA|2001-10-27||Changes m|0.56|0.35|1001001|amalgamalg #1|9|dresses|1|Women|112|antiprianti|medium|6835732384khaki05401|seashell|Tsp|Unknown|33|eingableeing| +829|AAAAAAAANDDAAAAA|1997-10-27||Plants should manage slowly on a managers. Trials could stop never also obvious awards; true, attractive controls determine psychiatric, bad relations. Keys follow. Positions coul|2.73|2.26|9015002|scholarunivamalg #2|15|fiction|9|Books|332|ablepripri|N/A|30royal5668572880571|steel|Tsp|Unknown|56|n stableeing| +830|AAAAAAAAODDAAAAA|1997-10-27|2000-10-26|Individuals cast artificially quite irish clients. Fundamentally left changes suggest also easy years. Only social numbers ought|6.62|5.42|5002001|importoscholar #1|2|country|5|Music|91|oughtn st|N/A|5pale665393496529332|salmon|Dram|Unknown|20|barprieing| +831|AAAAAAAAODDAAAAA|2000-10-27||Slightly comprehensive names could tell elegantly national images. Hundreds fly social, original blues. Local, engli|6.51|5.42|5002001|scholarunivamalg #4|2|fiction|9|Books|91|oughtn st|N/A|375gainsboro72612001|saddle|N/A|Unknown|30|oughtprieing| +832|AAAAAAAAAEDAAAAA|1997-10-27|1999-10-27|Today previous months address. Identical, appropriate details may remain at all final, small variations. So middle|7.16|4.43|7015003|scholarnameless #3|15|tables|7|Home|73|priation|N/A|664steel009862079596|purple|Pound|Unknown|62|ableprieing| +833|AAAAAAAAAEDAAAAA|1999-10-28|2001-10-26|Today previous months address. Identical, appropriate details may remain at all final, small variations. So middle|87.37|50.67|7015003|amalgimporto #2|1|accessories|2|Men|73|priation|small|664steel009862079596|maroon|Case|Unknown|68|priprieing| +834|AAAAAAAAAEDAAAAA|2001-10-27||Today previous months address. Identical, appropriate details may remain at all final, small variations. So middle|0.36|0.13|7015003|edu packimporto #1|1|sports-apparel|2|Men|73|priation|medium|019989479yellow42960|goldenrod|Pound|Unknown|58|eseprieing| +835|AAAAAAAADEDAAAAA|1997-10-27||Only likely practitioners pay simply. Solid horses must push shows. Foreign, furious pairs might not approach in a patients. Days sound shortly therefore local instructions. Under slim yea|5.52|2.09|8015006|scholarmaxi #6|15|fishing|8|Sports|752|ableantiation|N/A|43718564rosy31069255|pink|Carton|Unknown|37|antiprieing| +836|AAAAAAAAEEDAAAAA|1997-10-27|2000-10-26|Dead pages know mo|93.11|37.24|5003001|exportischolar #1|3|pop|5|Music|409|n stbarese|N/A|324973594peru7486242|red|Unknown|Unknown|17|callyprieing| +837|AAAAAAAAEEDAAAAA|2000-10-27||Dead pages know mo|3.63|2.25|5003001|amalgnameless #8|1|athletic shoes|8|Sports|96|n stbarese|N/A|95peru40399568442190|purple|Oz|Unknown|23|ationprieing| +838|AAAAAAAAGEDAAAAA|1997-10-27|1999-10-27|Linguistic, possible legs make very more brave|2.71|2.24|6006007|corpcorp #7|6|rings|6|Jewelry|420|barableese|N/A|160salmon86487227338|peach|Bunch|Unknown|3|eingprieing| +839|AAAAAAAAGEDAAAAA|1999-10-28|2001-10-26|Friends can believe usually french games. Young, scientific tanks might put still distinct centres. Single services |7.98|2.24|6014008|edu packbrand #8|6|estate|6|Jewelry|56|barableese|N/A|7231810711890papaya7|saddle|Tsp|Unknown|92|n stprieing| +840|AAAAAAAAGEDAAAAA|2001-10-27||Men can make human, thin communities. Victorian, sure solutions fit other, able publications. Geographical casualties can press now thereafter neighbouring services. Electr|8.11|2.24|8007001|brandnameless #1|6|hockey|8|Sports|241|barableese|N/A|7231810711890papaya7|wheat|Bunch|Unknown|67|bareseeing| +841|AAAAAAAAJEDAAAAA|1997-10-27||Local, social parties shall enjoy deep, hot items. Slim, different places shall laugh again. Already previous tracks ought to long whol|3.79|2.95|10002009|importounivamalg #9|2|camcorders|10|Electronics|466|callycallyese|N/A|663593779purple12060|wheat|Pallet|Unknown|44|oughteseeing| +842|AAAAAAAAKEDAAAAA|1997-10-27|2000-10-26|Welsh powers prevent automatically independent horses. Reduced |65.31|50.94|4004001|edu packedu pack #1|4|athletic|4|Shoes|383|prieingpri|petite|9271602yellow8853485|peru|Pallet|Unknown|61|ableeseeing| +843|AAAAAAAAKEDAAAAA|2000-10-27||Welsh powers prevent automatically independent horses. Reduced |2.08|1.80|4004002|edu packedu pack #2|4|athletic|4|Shoes|141|oughteseought|medium|58925125394sienna878|puff|Tsp|Unknown|16|prieseeing| +844|AAAAAAAAMEDAAAAA|1997-10-27|1999-10-27|Social, national goals apply oral pubs; old requirements retain incredibly old plants. Concepts gain rather dramatically m|0.38|0.23|5001001|amalgscholar #1|1|rock|5|Music|148|eingeseought|N/A|89477268164tan313921|spring|Bundle|Unknown|35|eseeseeing| +845|AAAAAAAAMEDAAAAA|1999-10-28|2001-10-26|Social, national goals apply oral pubs; old requirements retain incredibly old plants. Concepts gain rather dramatically m|6.48|3.04|5001001|importoimporto #2|2|shirts|2|Men|148|eingeseought|medium|89477268164tan313921|pink|Case|Unknown|61|antieseeing| +846|AAAAAAAAMEDAAAAA|2001-10-27||Social, national goals apply oral pubs; old requirements retain incredibly old plants. Concepts gain rather dramatically m|5.61|2.97|5001001|amalgscholar #1|1|rock|5|Music|148|eingeseought|N/A|89477268164tan313921|purple|Lb|Unknown|31|callyeseeing| +847|AAAAAAAAPEDAAAAA|1997-10-27||Detailed, solid customers deal. Decades mean across unique demands; cheerful teachers influence more psychological companies; more basic i|3.99|3.51|1002002|importoamalg #2|2|fragrances|1|Women|868|eingcallyeing|medium|889683pale9504259744|violet|Oz|Unknown|23|ationeseeing| +848|AAAAAAAAAFDAAAAA|1997-10-27|2000-10-26|New relations should get ideal shapes. Revolutionary settings forget however soviet institutions. Guests might disguise probably miners; immediate, local barriers destroy exactly pol|0.85|0.69|7016009|corpnameless #9|16|furniture|7|Home|212|ableoughtable|N/A|357salmon38372766127|yellow|Dram|Unknown|5|eingeseeing| +849|AAAAAAAAAFDAAAAA|2000-10-27||New relations should get ideal shapes. Revolutionary settings forget however soviet institutions. Guests might disguise probably miners; immediate, local barriers destroy exactly pol|6.23|0.69|3003002|exportiexporti #2|16|toddlers|3|Children|212|ableoughtable|extra large|416chiffon2944985526|thistle|Ton|Unknown|42|n steseeing| +850|AAAAAAAACFDAAAAA|1997-10-27|1999-10-27|Normally odd numbers give however pretty atomic methods. Cold|3.07|1.28|4003001|exportiedu pack #1|3|kids|4|Shoes|497|ationn stese|medium|21570520796692peru59|indian|Dozen|Unknown|83|barantieing| +851|AAAAAAAACFDAAAAA|1999-10-28|2001-10-26|Normally odd numbers give however pretty atomic methods. Cold|2.30|1.28|2004002|edu packimporto #2|4|sports-apparel|2|Men|497|ationn stese|medium|7chartreuse450807699|pale|Lb|Unknown|21|oughtantieing| +852|AAAAAAAACFDAAAAA|2001-10-27||Normally odd numbers give however pretty atomic methods. Cold|4.46|1.28|2004002|exportiamalg #1|4|maternity|1|Women|161|oughtcallyought|medium|7chartreuse450807699|honeydew|Pound|Unknown|29|ableantieing| +853|AAAAAAAAFFDAAAAA|1997-10-27||Circumstances break much to a hills. Royal defendants would realize still on a flames. Gothic, open businesses shall come. Narrow, other times make |8.35|7.51|10004004|edu packunivamalg #4|4|audio|10|Electronics|46|callyese|N/A|144798066tan18416826|yellow|Gram|Unknown|1|priantieing| +854|AAAAAAAAGFDAAAAA|1997-10-27|2000-10-26|Western, different stages will help sharp documents. Royal, serious classes develop sometimes high windows. Lips thi|0.93|0.66|4003001|exportiedu pack #1|3|kids|4|Shoes|484|eseeingese|medium|1182715white34423044|salmon|Gross|Unknown|18|eseantieing| +855|AAAAAAAAGFDAAAAA|2000-10-27||Western, different stages will help sharp documents. Royal, serious classes develop sometimes high windows. Lips thi|8.03|2.40|3002002|importoexporti #2|2|infants|3|Children|484|eseeingese|medium|1182715white34423044|salmon|Bundle|Unknown|25|antiantieing| +856|AAAAAAAAIFDAAAAA|1997-10-27|1999-10-27|Vulnerable, statistical photographs handle|7.98|6.38|5001001|amalgscholar #1|1|rock|5|Music|184|eseeingought|N/A|66368chartreuse88027|blush|Box|Unknown|18|callyantieing| +857|AAAAAAAAIFDAAAAA|1999-10-28|2001-10-26|Vulnerable, statistical photographs handle|3.64|6.38|5001001|importoamalgamalg #8|12|monitors|10|Electronics|17|eseeingought|N/A|116peru6516607233852|navy|Carton|Unknown|65|ationantieing| +858|AAAAAAAAIFDAAAAA|2001-10-27||Vulnerable, statistical photographs handle|7.69|6.38|2001001|amalgimporto #1|12|accessories|2|Men|17|ationought|medium|8017397267506wheat07|sandy|Dram|Unknown|31|eingantieing| +859|AAAAAAAALFDAAAAA|1997-10-27||Rural, difficult categories bring possible politicians. Dead, other sentences use often traditional, inappropriate efforts. Beautiful mate|5.82|1.80|4004002|edu packedu pack #2|4|athletic|4|Shoes|187|ationeingought|economy|88029931seashell3436|tomato|Unknown|Unknown|30|n stantieing| +860|AAAAAAAAMFDAAAAA|1997-10-27|2000-10-26|Other writers could mean. Facilities breed later. Creditors look in a knees. Scotti|3.85|1.84|5001001|amalgscholar #1|1|rock|5|Music|196|callyn stought|N/A|4893112slate21479424|moccasin|Gram|Unknown|57|barcallyeing| +861|AAAAAAAAMFDAAAAA|2000-10-27||Labour, natural teeth become for the paths; present, lesser criticisms refine with a files. Musical talks should recoup less; french methods may not start companies. La|1.85|1.84|6010006|univbrand #6|1|jewelry boxes|6|Jewelry|196|callyn stought|N/A|4893112slate21479424|honeydew|Dram|Unknown|23|oughtcallyeing| +862|AAAAAAAAOFDAAAAA|1997-10-27|1999-10-27|Candidates will hold. Chief ages used to choose with the foundations; horrible, complex rates meet|2.21|1.32|6009007|maxicorp #7|9|womens watch|6|Jewelry|44|eseese|N/A|47aquamarine62134677|lime|Bundle|Unknown|30|ablecallyeing| +863|AAAAAAAAOFDAAAAA|1999-10-28|2001-10-26|Candidates will hold. Chief ages used to choose with the foundations; horrible, complex rates meet|1.08|0.72|2003002|exportiimporto #2|9|pants|2|Men|44|eseese|petite|9gainsboro6386540807|slate|Dozen|Unknown|60|pricallyeing| +864|AAAAAAAAOFDAAAAA|2001-10-27||Candidates will hold. Chief ages used to choose with the foundations; horrible, complex rates meet|8.66|0.72|7007005|brandbrand #5|9|decor|7|Home|249|n steseable|N/A|9gainsboro6386540807|gainsboro|Bunch|Unknown|50|esecallyeing| +865|AAAAAAAABGDAAAAA|1997-10-27||Just c|3.26|1.07|5003002|exportischolar #2|3|pop|5|Music|538|eingprianti|N/A|7682950105green96153|yellow|Pound|Unknown|19|anticallyeing| +866|AAAAAAAACGDAAAAA|1997-10-27|2000-10-26|Results can drive in a minds. Sweet subsidies go particles. Even right schools tell almost to the books. Ultimately vague resources used to reject once with a tech|4.54|1.95|3004001|edu packexporti #1|4|school-uniforms|3|Children|361|oughtcallypri|N/A|89335607indian002292|sandy|Ounce|Unknown|74|callycallyeing| +867|AAAAAAAACGDAAAAA|2000-10-27||Results can drive in a minds. Sweet subsidies go particles. Even right schools tell almost to the books. Ultimately vague resources used to reject once with a tech|2.22|1.95|2002002|importoimporto #2|4|shirts|2|Men|361|oughtcallypri|medium|714047sienna36446866|lawn|Gram|Unknown|55|ationcallyeing| +868|AAAAAAAAEGDAAAAA|1997-10-27|1999-10-27|Things must work simply clear chips. Technical leaders used to see around following, major patterns; always other elements pay daily for a times. More adult co|9.07|8.07|5004001|edu packscholar #1|4|classical|5|Music|69|n stcally|N/A|9271376518yellow4209|tan|Box|Unknown|55|eingcallyeing| +869|AAAAAAAAEGDAAAAA|1999-10-28|2001-10-26|Bars can s|59.88|8.07|9009006|maximaxi #6|4|science|9|Books|69|n stcally|N/A|9271376518yellow4209|orange|Ton|Unknown|16|n stcallyeing| +870|AAAAAAAAEGDAAAAA|2001-10-27||Bars can s|7.47|3.51|9009006|exportiamalg #1|3|maternity|1|Women|69|n stcally|N/A|67slate1165784382936|almond|Carton|Unknown|65|barationeing| +871|AAAAAAAAHGDAAAAA|1997-10-27||Ever l|1.22|0.97|3003002|exportiexporti #2|3|toddlers|3|Children|162|ablecallyought|small|5750408315ivory88903|sky|Each|Unknown|18|oughtationeing| +872|AAAAAAAAIGDAAAAA|1997-10-27|2000-10-26|Traditional, modern grants may not think popular branches. Races assess british students. Classes become men. Marvellous, local details shall not admire there logical parents. Of course teenage |2.12|0.97|6003001|exporticorp #1|3|gold|6|Jewelry|201|oughtbarable|N/A|sandy075531493611538|violet|Pallet|Unknown|56|ableationeing| +873|AAAAAAAAIGDAAAAA|2000-10-27||Traditional, modern grants may not think popular branches. Races assess british students. Classes become men. Marvellous, local details shall not admire there logical parents. Of course teenage |64.34|0.97|6003001|exportiamalg #2|3|maternity|1|Women|143|prieseought|large|355salmon87205215190|orchid|Case|Unknown|11|priationeing| +874|AAAAAAAAKGDAAAAA|1997-10-27|1999-10-27|Agencies shall not ignore detailed costs. About small others cann|4.30|2.36|1001001|amalgamalg #1|1|dresses|1|Women|778|eingationation|medium|6752green39020488057|medium|Box|Unknown|34|eseationeing| +875|AAAAAAAAKGDAAAAA|1999-10-28|2001-10-26|Scenes make already correct years. Religious, difficult exports may think too absolute, likely things. Artifici|7.62|2.36|7006006|corpbrand #6|1|rugs|7|Home|778|eingationation|N/A|369161navy5172798464|salmon|Dram|Unknown|33|antiationeing| +876|AAAAAAAAKGDAAAAA|2001-10-27||Scenes make already correct years. Religious, difficult exports may think too absolute, likely things. Artifici|2.80|2.36|3001001|amalgexporti #1|1|newborn|3|Children|778|eingationation|economy|369161navy5172798464|lavender|Box|Unknown|87|callyationeing| +877|AAAAAAAANGDAAAAA|1997-10-27||Modest systems cannot spare competitive places. Able, innocent workers will not form still conflicts; black, olympic forces may jump ever eyes. Ov|1.09|0.53|5002002|importoscholar #2|2|country|5|Music|120|barableought|N/A|96180violet753782002|puff|Gross|Unknown|18|ationationeing| +878|AAAAAAAAOGDAAAAA|1997-10-27|2000-10-26|Women spend rocks. Similar children draw there scottish areas. Cases can take earlier previous statements. New, high stars intend warm tools. Implicit, digital details might take by a busin|4.40|2.86|10008012|namelessunivamalg #12|8|scanners|10|Electronics|523|priableanti|N/A|895164388623yellow69|pink|Case|Unknown|14|eingationeing| +879|AAAAAAAAOGDAAAAA|2000-10-27||Women spend rocks. Similar children draw there scottish areas. Cases can take earlier previous statements. New, high stars intend warm tools. Implicit, digital details might take by a busin|1.25|0.60|10008012|importoscholar #2|8|country|5|Music|523|priableanti|N/A|062334816451rosy5650|midnight|Case|Unknown|20|n stationeing| +880|AAAAAAAAAHDAAAAA|1997-10-27|1999-10-27|New others keep roughly polite engines. Male questions decide in the papers. Both other users may see today young, past decision|4.02|2.25|9010005|univunivamalg #5|10|travel|9|Books|10|barought|N/A|940turquoise11604800|tan|Bundle|Unknown|14|bareingeing| +881|AAAAAAAAAHDAAAAA|1999-10-28|2001-10-26|New others keep roughly polite engines. Male questions decide in the papers. Both other users may see today young, past decision|7.69|4.84|9010006|univunivamalg #6|10|travel|9|Books|869|n stcallyeing|N/A|940turquoise11604800|brown|Tbl|Unknown|22|oughteingeing| +882|AAAAAAAAAHDAAAAA|2001-10-27||New others keep roughly polite engines. Male questions decide in the papers. Both other users may see today young, past decision|30.55|9.16|10016016|corpamalgamalg #16|10|wireless|10|Electronics|287|n stcallyeing|N/A|710469620beige548296|rose|Lb|Unknown|33|ableeingeing| +883|AAAAAAAADHDAAAAA|1997-10-27||Words pursue very soon weekly governments. Small, soviet months should not say parents. Other buildings shall clean then other, large meal|6.29|2.95|6002004|importocorp #4|2|diamonds|6|Jewelry|97|ationn st|N/A|68721743663lavender7|sandy|Gross|Unknown|78|prieingeing| +884|AAAAAAAAEHDAAAAA|1997-10-27|2000-10-26|Civil brothers|65.85|21.07|10006001|corpunivamalg #1|6|musical|10|Electronics|566|callycallyanti|N/A|8855peach00185973673|papaya|Oz|Unknown|84|eseeingeing| +885|AAAAAAAAEHDAAAAA|2000-10-27||Arts will like chapters.|5.82|4.48|10006001|importoimporto #2|6|shirts|2|Men|566|callycallyanti|medium|61093234820violet223|chiffon|Tsp|Unknown|52|antieingeing| +886|AAAAAAAAGHDAAAAA|1997-10-27|1999-10-27|In common academic pupils know highly joint sites. Twin, safe methods introduce most possible others; times fall most effects. Highest parliamentary performances used |6.97|2.78|7005007|scholarbrand #7|5|blinds/shades|7|Home|110|baroughtought|N/A|2264320815snow401205|tomato|Case|Unknown|20|callyeingeing| +887|AAAAAAAAGHDAAAAA|1999-10-28|2001-10-26|In common academic pupils know highly joint sites. Twin, safe methods introduce most possible others; times fall most effects. Highest parliamentary performances used |4.70|2.78|7005007|exportiamalg #2|3|maternity|1|Women|110|baroughtought|medium|2264320815snow401205|cornflower|Pound|Unknown|66|ationeingeing| +888|AAAAAAAAGHDAAAAA|2001-10-27||Far dependent patients must not s|5.91|4.19|7005007|importobrand #9|2|bedding|7|Home|110|baroughtought|N/A|2264320815snow401205|powder|Ton|Unknown|67|eingeingeing| +889|AAAAAAAAJHDAAAAA|1997-10-27||Available, lucky groups spread entirely always similar societies; things shall not get at a hours. So closed birds can turn|2.03|0.75|5003002|exportischolar #2|3|pop|5|Music|87|ationeing|N/A|695229883065salmon46|puff|Lb|Unknown|59|n steingeing| +890|AAAAAAAAKHDAAAAA|1997-10-27|2000-10-26|Even chief games can present privately extra whole steps. Over old reasons ought to |4.30|1.41|6008003|namelesscorp #3|8|mens watch|6|Jewelry|253|priantiable|N/A|86735ivory8960580512|sandy|Bunch|Unknown|6|barn steing| +891|AAAAAAAAKHDAAAAA|2000-10-27||Great, extraordinary goals must not take later good, irrelevant centuries; obvious, bright daughters shall|1.81|1.41|6008003|importomaxi #4|2|business|9|Books|93|prin st|N/A|86735ivory8960580512|peru|Ounce|Unknown|21|oughtn steing| +892|AAAAAAAAMHDAAAAA|1997-10-27|1999-10-27|Classical, major quantities fit to a prices. Reasons would know. Cold, direct groups may not charge interested, urban conclusions. Quite lively voters provid|3.11|1.92|3002001|importoexporti #1|2|infants|3|Children|365|anticallypri|extra large|goldenrod47154545981|peru|Tbl|Unknown|49|ablen steing| +893|AAAAAAAAMHDAAAAA|1999-10-28|2001-10-26|Classical, major quantities fit to a prices. Reasons would know. Cold, direct groups may not charge interested, urban conclusions. Quite lively voters provid|1.55|1.92|4004002|edu packedu pack #2|4|athletic|4|Shoes|605|antibarcally|extra large|goldenrod47154545981|puff|Dram|Unknown|15|prin steing| +894|AAAAAAAAMHDAAAAA|2001-10-27||Other, good sales keep logical objectives. Surprising, roman councils must pa|7.39|1.92|4004002|amalgmaxi #7|1|arts|9|Books|767|ationcallyation|N/A|3525seashell35658422|sienna|Oz|Unknown|31|esen steing| +895|AAAAAAAAPHDAAAAA|1997-10-27||Easily new legs rain boats; lost, military comments accommodate very reasons. |1.37|0.93|6009008|maxicorp #8|9|womens watch|6|Jewelry|298|eingn stable|N/A|32292153530225rose12|tomato|Ounce|Unknown|32|antin steing| +896|AAAAAAAAAIDAAAAA|1997-10-27|2000-10-26|Systems would not send more faithfully easy ministers. Conditions penetrate vulnerable questions. Most regular parts create well german commentators. Odd difficulties mus|3.26|1.49|8004007|edu packnameless #7|4|camping|8|Sports|76|callyation|N/A|102860067784magenta3|plum|Pound|Unknown|53|callyn steing| +897|AAAAAAAAAIDAAAAA|2000-10-27||Even difficult ways publish different, other interests. Subjects prefer following, assistant men. Variables match then old planes. Proposed items open hitherto major groups. Likely, go|0.39|0.29|8004007|exportischolar #2|4|pop|5|Music|76|callyation|N/A|102860067784magenta3|pale|N/A|Unknown|54|ationn steing| +898|AAAAAAAACIDAAAAA|1997-10-27|1999-10-27|Millions might answer. Attractive rules might beat coloured volunteers. Scottis|3.51|1.93|9004011|edu packmaxi #11|4|entertainments|9|Books|340|baresepri|N/A|seashell912874805264|sky|Dram|Unknown|44|eingn steing| +899|AAAAAAAACIDAAAAA|1999-10-28|2001-10-26|Millions might answer. Attractive rules might beat coloured volunteers. Scottis|1.76|1.39|9011012|amalgunivamalg #12|4|cooking|9|Books|340|baresepri|N/A|seashell912874805264|yellow|Each|Unknown|24|n stn steing| +900|AAAAAAAACIDAAAAA|2001-10-27||Substantial skills ought to turn big, waste affairs. Days offer really positive, inc trees. Only pla|5.36|2.57|6009005|maxicorp #5|4|womens watch|6|Jewelry|340|baresepri|N/A|8860snow135817095781|pale|Carton|Unknown|30|barbarn st| +901|AAAAAAAAFIDAAAAA|1997-10-27||Personal services will say so little beds. Other, alt|25.40|20.32|3002002|importoexporti #2|2|infants|3|Children|371|oughtationpri|large|2608947645546plum067|powder|Ounce|Unknown|5|oughtbarn st| +902|AAAAAAAAGIDAAAAA|1997-10-27|2000-10-26|Alone firms obey furthermore serious children. Now relative customers must not soften slower small, inc men. Still wonderful authorities try|0.64|0.51|3003001|exportiexporti #1|3|toddlers|3|Children|48|eingese|N/A|3sky6837901316259604|peach|Pound|Unknown|100|ablebarn st| +903|AAAAAAAAGIDAAAAA|2000-10-27||Alone firms obey furthermore serious children. Now relative customers must not soften slower small, inc men. Still wonderful authorities try|1.91|0.51|3003001|importomaxi #4|3|guns|8|Sports|48|eingese|N/A|787011044cornflower5|royal|Dram|Unknown|70|pribarn st| +904|AAAAAAAAIIDAAAAA|1997-10-27|1999-10-27|Too nervous orders prevent further at a rocks. Good, right roads feel versus a questions. Furthermore dear visitors can raise no longer british national plants; duties ought to serve. Offic|3.30|1.35|8006005|corpnameless #5|6|football|8|Sports|602|ablebarcally|N/A|47861rose98087629271|sky|Gross|Unknown|23|esebarn st| +905|AAAAAAAAIIDAAAAA|1999-10-28|2001-10-26|Political acres advi|9.04|4.79|4003002|exportiedu pack #2|6|kids|4|Shoes|224|eseableable|extra large|47861rose98087629271|lawn|N/A|Unknown|24|antibarn st| +906|AAAAAAAAIIDAAAAA|2001-10-27||Margins can explore unlikely failures. Women mean|6.22|4.66|5004001|edu packscholar #1|4|classical|5|Music|224|eseableable|N/A|47861rose98087629271|yellow|Carton|Unknown|11|callybarn st| +907|AAAAAAAALIDAAAAA|1997-10-27||Largely substantial contracts facilitate. Yet full values can advise extremely plants. Men classify empty contacts. Private, common events can want more just central patients. Enti|1.55|0.48|9007008|brandmaxi #8|7|reference|9|Books|54|eseanti|N/A|04815040grey09723365|smoke|Pallet|Unknown|5|ationbarn st| +908|AAAAAAAAMIDAAAAA|1997-10-27|2000-10-26|Other, effective eyes please common estimates. Equally essenti|28.80|18.43|5002001|importoscholar #1|2|country|5|Music|289|n steingable|N/A|49972ivory4002482858|purple|Tbl|Unknown|61|eingbarn st| +909|AAAAAAAAMIDAAAAA|2000-10-27||Other, effective eyes please common estimates. Equally essenti|0.94|18.43|5002001|corpmaxi #10|16|golf|8|Sports|289|n steingable|N/A|49972ivory4002482858|orange|Each|Unknown|17|n stbarn st| +910|AAAAAAAAOIDAAAAA|1997-10-27|1999-10-27|Ju|8.33|7.41|4004001|edu packedu pack #1|4|athletic|4|Shoes|32|ablepri|petite|6456429742cream82683|salmon|Tbl|Unknown|17|baroughtn st| +911|AAAAAAAAOIDAAAAA|1999-10-28|2001-10-26|Ju|3.39|1.05|4004001|maxiunivamalg #11|9|televisions|10|Electronics|32|ablepri|N/A|813090136633631red21|purple|N/A|Unknown|13|oughtoughtn st| +912|AAAAAAAAOIDAAAAA|2001-10-27||Sharp procedures treat indeed; assessments address originally in a countries. Free, beautiful men will provide ultimately additional efforts; leading hours invest already conditions;|3.96|1.78|9002001|importomaxi #1|9|business|9|Books|32|ablepri|N/A|59474363orchid232751|hot|Pallet|Unknown|48|ableoughtn st| +913|AAAAAAAABJDAAAAA|1997-10-27||Particular countries will choose by a studies. Strongly technical images cut. Over other games might not hold so political, attractive figures. Right sales used to overcome wooden farms. Experiments m|2.15|1.54|4003002|exportiedu pack #2|3|kids|4|Shoes|816|callyoughteing|medium|4835199129blue108637|blanched|Bundle|Unknown|42|prioughtn st| +914|AAAAAAAACJDAAAAA|1997-10-27|2000-10-26|Particularly major balls need remarkably early errors. Swiss factors shall stay political payments. Quick schools increase the|4.83|1.49|6008003|namelesscorp #3|8|mens watch|6|Jewelry|211|oughtoughtable|N/A|677pale1665750632859|tan|Pound|Unknown|5|eseoughtn st| +915|AAAAAAAACJDAAAAA|2000-10-27||Particularly major balls need remarkably early errors. Swiss factors shall stay political payments. Quick schools increase the|1.94|1.59|6008003|edu packamalgamalg #15|14|automotive|10|Electronics|191|oughtn stought|N/A|677pale1665750632859|drab|Dram|Unknown|66|antioughtn st| +916|AAAAAAAAEJDAAAAA|1997-10-27|1999-10-27|Palestinian vegetables begin above; details shall head largely wild, symbolic answers. Late use|8.91|5.34|4004001|edu packedu pack #1|4|athletic|4|Shoes|218|eingoughtable|small|5560768tomato4404385|red|Oz|Unknown|31|callyoughtn st| +917|AAAAAAAAEJDAAAAA|1999-10-28|2001-10-26|Close years join so for example new programmes; colours shall get never turkish rights. Others |2.88|1.81|4004001|edu packexporti #2|4|school-uniforms|3|Children|117|eingoughtable|small|5560768tomato4404385|steel|Dram|Unknown|61|ationoughtn st| +918|AAAAAAAAEJDAAAAA|2001-10-27||Whole, public articles might enjoy direc|2.12|1.20|9004007|edu packmaxi #7|4|entertainments|9|Books|496|callyn stese|N/A|3804913419690wheat83|steel|Ounce|Unknown|38|eingoughtn st| +919|AAAAAAAAHJDAAAAA|1997-10-27||Perhaps economic dimensions help gains. Funny, traditional couples will depend silently |3.48|1.32|10003002|exportiunivamalg #2|3|dvd/vcr players|10|Electronics|477|ationationese|N/A|94171salmon485735957|turquoise|Case|Unknown|62|n stoughtn st| +920|AAAAAAAAIJDAAAAA|1997-10-27|2000-10-26|Political walls shall behave clients. Artistic, complex feet build then present, following names. Central, good director|4.67|2.80|3002001|importoexporti #1|2|infants|3|Children|36|callypri|economy|82sandy7304372715754|black|Box|Unknown|28|barablen st| +921|AAAAAAAAIJDAAAAA|2000-10-27||Political walls shall behave clients. Artistic, complex feet build then present, following names. Central, good director|0.75|0.48|3002001|univnameless #2|2|flatware|7|Home|279|callypri|N/A|82sandy7304372715754|olive|Pallet|Unknown|12|oughtablen st| +922|AAAAAAAAKJDAAAAA|1997-10-27|1999-10-27|Urgent, civil opportunities will document. Economic, steady engines examine already. Carefully clear arms strike now. Numerous principles would not promote |1.52|0.57|5004001|edu packscholar #1|4|classical|5|Music|849|n steseeing|N/A|2violet5789101757182|sienna|Case|Unknown|87|ableablen st| +923|AAAAAAAAKJDAAAAA|1999-10-28|2001-10-26|Agricultural, possible thousands would not fly low officials. Changes u|2.14|0.57|5004001|edu packnameless #4|4|camping|8|Sports|324|n steseeing|N/A|2violet5789101757182|papaya|Bundle|Unknown|21|priablen st| +924|AAAAAAAAKJDAAAAA|2001-10-27||Surely following patien|58.71|39.33|5004001|importobrand #5|2|bedding|7|Home|324|eseablepri|N/A|2violet5789101757182|purple|Unknown|Unknown|48|eseablen st| +925|AAAAAAAANJDAAAAA|1997-10-27||Sometimes bitter periods may get. Difficult, hot sources used to understand women. Patients may mention; awfu|3.87|2.70|5001002|amalgscholar #2|1|rock|5|Music|28|eingable|N/A|706997961913rosy1771|midnight|Tbl|Unknown|51|antiablen st| +926|AAAAAAAAOJDAAAAA|1997-10-27|2000-10-26|Reasonably relative children escape also maps. In addition brief debts find customs. Whole, essential ships gain less cool days. Easy, early processes would not t|4.59|2.79|2001001|amalgimporto #1|1|accessories|2|Men|110|baroughtought|petite|597899brown244435417|salmon|Ton|Unknown|12|callyablen st| +927|AAAAAAAAOJDAAAAA|2000-10-27||Reasonably relative children escape also maps. In addition brief debts find customs. Whole, essential ships gain less cool days. Easy, early processes would not t|0.60|0.30|10004010|edu packunivamalg #10|1|audio|10|Electronics|110|baroughtought|N/A|4273purple7889581775|white|Carton|Unknown|5|ationablen st| +928|AAAAAAAAAKDAAAAA|1997-10-27|1999-10-27|Political boys ought to return actually strong, ex|61.62|47.44|5002001|importoscholar #1|2|country|5|Music|631|oughtprically|N/A|146787snow6869627687|white|Ounce|Unknown|10|eingablen st| +929|AAAAAAAAAKDAAAAA|1999-10-28|2001-10-26|Political boys ought to return actually strong, ex|1.71|1.14|5002001|edu packscholar #2|4|classical|5|Music|306|callybarpri|N/A|06965986bisque459474|brown|Ton|Unknown|61|n stablen st| +930|AAAAAAAAAKDAAAAA|2001-10-27||Long ways used to want mysterious hours. Difficult audiences copy even new shelves. Possible, alone developments k|4.48|1.14|5002001|scholarnameless #1|5|fitness|8|Sports|258|callybarpri|N/A|7603031891629saddle2|wheat|Bundle|Unknown|10|barprin st| +931|AAAAAAAADKDAAAAA|1997-10-27||Patients run; very minutes enhance too there exceptional difficulties. Minor, afraid years will sleep simply strongly common po|91.44|33.83|6002004|importocorp #4|2|diamonds|6|Jewelry|842|ableeseeing|N/A|4seashell09990712184|gainsboro|Carton|Unknown|21|oughtprin st| +932|AAAAAAAAEKDAAAAA|1997-10-27|2000-10-26|Now short views cannot include. Real, northern interests may build. Fresh|1.78|0.76|7002003|importobrand #3|2|bedding|7|Home|274|eseationable|N/A|9346590383492papaya8|red|Carton|Unknown|70|ableprin st| +933|AAAAAAAAEKDAAAAA|2000-10-27||Now short views cannot include. Real, northern interests may build. Fresh|7.78|2.56|9008010|namelessmaxi #10|2|romance|9|Books|954|eseationable|N/A|41136433329sienna418|tan|Gross|Unknown|33|priprin st| +934|AAAAAAAAGKDAAAAA|1997-10-27|1999-10-27|More expensive men used to become most current offices. There royal areas shall not study particularly important, remain|0.46|0.21|7004005|edu packbrand #5|4|curtains/drapes|7|Home|386|callyeingpri|N/A|702975328snow7332362|smoke|Tsp|Unknown|29|eseprin st| +935|AAAAAAAAGKDAAAAA|1999-10-28|2001-10-26|Alive targets work nearly. Patients fall maybe by a elections. Considerable developments quit perhaps. Ligh|2.77|1.55|6004008|edu packcorp #8|4|bracelets|6|Jewelry|215|callyeingpri|N/A|702975328snow7332362|turquoise|Ton|Unknown|29|antiprin st| +936|AAAAAAAAGKDAAAAA|2001-10-27||Alive targets work nearly. Patients fall maybe by a elections. Considerable developments quit perhaps. Ligh|1.74|1.55|6004008|amalgunivamalg #2|1|cameras|10|Electronics|286|callyeingpri|N/A|56rosy21219400441204|purple|Ton|Unknown|10|callyprin st| +937|AAAAAAAAJKDAAAAA|1997-10-27||Yet equal pa|57.16|25.15|7013008|exportinameless #8|13|wallpaper|7|Home|361|oughtcallypri|N/A|229764snow2927035741|steel|Oz|Unknown|61|ationprin st| +938|AAAAAAAAKKDAAAAA|1997-10-27|2000-10-26|Risks launch short systems; exclusive|81.00|58.32|3003001|exportiexporti #1|3|toddlers|3|Children|469|n stcallyese|extra large|461green648126941974|rosy|N/A|Unknown|13|eingprin st| +939|AAAAAAAAKKDAAAAA|2000-10-27||Risks launch short systems; exclusive|2.97|58.32|3003001|amalgmaxi #10|11|archery|8|Sports|469|n stcallyese|N/A|161304yellow42553307|orange|Dozen|Unknown|1|n stprin st| +940|AAAAAAAAMKDAAAAA|1997-10-27|1999-10-27|Too big interests ought to go annually old real classes; little edges entail still electoral matters. Sure bonds make wonderful, central provisions. Internation|1.52|0.94|9004005|edu packmaxi #5|4|entertainments|9|Books|487|ationeingese|N/A|02694019689lawn30573|papaya|Each|Unknown|6|baresen st| +941|AAAAAAAAMKDAAAAA|1999-10-28|2001-10-26|||||edu packimporto #2|4|sports-apparel|2|Men||ationeingese|medium|||Tsp||51|oughtesen st| +942|AAAAAAAAMKDAAAAA|2001-10-27||Too big interests ought to go annually old real classes; little edges entail still electoral matters. Sure bonds make wonderful, central provisions. Internation|3.91|2.93|3001001|amalgexporti #1|1|newborn|3|Children|391|oughtn stpri|medium|02694019689lawn30573|peach|Case|Unknown|54|ableesen st| +943|AAAAAAAAPKDAAAAA|1997-10-27||Other, united fires meet in a researchers. Otherwise|0.69|0.57|3002002|importoexporti #2|2|infants|3|Children|265|anticallyable|medium|06356peru66747085533|misty|Box|Unknown|57|priesen st| +944|AAAAAAAAALDAAAAA|1997-10-27|2000-10-26|Conv|5.02|3.46|5002001|importoscholar #1|2|country|5|Music|862|ablecallyeing|N/A|39270073spring132933|red|Carton|Unknown|17|eseesen st| +945|AAAAAAAAALDAAAAA|2000-10-27||Likely states may not sound bars. Sick, left words make below now ideological p|2.09|3.46|5002001|exportiunivamalg #11|2|dvd/vcr players|10|Electronics|263|pricallyable|N/A|39270073spring132933|red|Pallet|Unknown|58|antiesen st| +946|AAAAAAAACLDAAAAA|1997-10-27|1999-10-27|Desperate activities increase likely judges. Standards may not make national, fatal courses. Soon european factories hear various cattle; possible rates |6.33|2.91|8010007|univmaxi #7|10|pools|8|Sports|67|ationcally|N/A|03290480731904ghost3|pale|Case|Unknown|51|callyesen st| +947|AAAAAAAACLDAAAAA|1999-10-28|2001-10-26|Desperate activities increase likely judges. Standards may not make national, fatal courses. Soon european factories hear various cattle; possible rates |0.26|0.20|7016008|corpnameless #8|10|furniture|7|Home|67|ationcally|N/A|1029pale006013426358|powder|Bunch|Unknown|4|ationesen st| +948|AAAAAAAACLDAAAAA|2001-10-27||Contracts could prove very only leading divisions. Main men would not mov|8.15|2.44|7016008|univmaxi #9|10|pools|8|Sports|302|ablebarpri|N/A|1029pale006013426358|magenta|Tsp|Unknown|34|eingesen st| +949|AAAAAAAAFLDAAAAA|1997-10-27||Languages shall shape by now ever vague categories. Things put possible groups. Also social cuts produce etc local officers. International, |3.19|1.65|10014015|edu packamalgamalg #15|14|automotive|10|Electronics|302|ablebarpri|N/A|5mint916134355538759|papaya|Gram|Unknown|25|n stesen st| +950|AAAAAAAAGLDAAAAA|1997-10-27|2000-10-26|Reports think above in a points. Shortly great songs sound typically young, easy hours. Daily orders kill usually male men; special, leg|2.62|2.35|6016007|corpbrand #7|16|consignment|6|Jewelry|282|ableeingable|N/A|900198573828948drab6|thistle|Ton|Unknown|82|barantin st| +951|AAAAAAAAGLDAAAAA|2000-10-27||Also able liabilities acquire perhaps as possible residents. Constantly common firms ease less. Possible, cer|3.12|2.35|6016007|amalgexporti #2|16|newborn|3|Children|282|ableeingable|medium|417136358805slate028|peru|Dram|Unknown|56|oughtantin st| +952|AAAAAAAAILDAAAAA|1997-10-27|1999-10-27|Proud, brown hours overcome high lines|85.66|27.41|2002001|importoimporto #1|2|shirts|2|Men|443|prieseese|large|7636142946mint614427|seashell|Bunch|Unknown|3|ableantin st| +953|AAAAAAAAILDAAAAA|1999-10-28|2001-10-26|Only samples clear great accounts. Slowly encouraging girls look national, future parameters. Women used to tell voices. Stores will see sincerely educational opportunities. Relati|1.01|27.41|5004002|edu packscholar #2|4|classical|5|Music|443|prieseese|N/A|491390goldenrod51816|sienna|N/A|Unknown|2|priantin st| +954|AAAAAAAAILDAAAAA|2001-10-27||Only samples clear great accounts. Slowly encouraging girls look national, future parameters. Women used to tell voices. Stores will see sincerely educational opportunities. Relati|1.70|1.24|3003001|exportiexporti #1|4|toddlers|3|Children|443|prieseese|large|0539614727dim0966136|pale|Dozen|Unknown|21|eseantin st| +955|AAAAAAAALLDAAAAA|1997-10-27||Personal days see large, important parents. Children |90.72|75.29|9008008|namelessmaxi #8|8|romance|9|Books|194|esen stought|N/A|turquoise35358947055|metallic|Dram|Unknown|27|antiantin st| +956|AAAAAAAAMLDAAAAA|1997-10-27|2000-10-26|Likely, representative hours will get up to public parents. Just political statements start. Just central children would find as for a studies. Regular, free cats att|4.55|2.45|1001001|amalgamalg #1|1|dresses|1|Women|13|priought|extra large|rosy4312210183088504|tan|Lb|Unknown|10|callyantin st| +957|AAAAAAAAMLDAAAAA|2000-10-27||Likely, representative hours will get up to public parents. Just political statements start. Just central children would find as for a studies. Regular, free cats att|3.76|1.54|7014008|edu packnameless #8|1|glassware|7|Home|13|priought|N/A|rosy4312210183088504|yellow|Bundle|Unknown|42|ationantin st| +958|AAAAAAAAOLDAAAAA|1997-10-27|1999-10-27|Sales know in a meanings. International, professional figures may get in a statement|0.48|0.22|7011009|amalgnameless #9|11|accent|7|Home|314|eseoughtpri|N/A|64433309865787ivory2|rose|Box|Unknown|6|eingantin st| +959|AAAAAAAAOLDAAAAA|1999-10-28|2001-10-26|Alternative bills would not attend different, generous months. Even great profits contribute necessarily particular, classical windows. Magistrates know colours. Social, blue pr|2.59|0.22|7011009|scholarnameless #10|5|fitness|8|Sports|381|oughteingpri|N/A|64433309865787ivory2|red|N/A|Unknown|7|n stantin st| +960|AAAAAAAAOLDAAAAA|2001-10-27||Courts stop. Of course deep months end nearly in a machines. Minimal, convenient angles tear wrong injuries. That is blue demands|22.89|7.32|7011009|exportimaxi #1|5|computers|9|Books|63|prically|N/A|9920334108tan7485528|orange|N/A|Unknown|23|barcallyn st| +961|AAAAAAAABMDAAAAA|1997-10-27||Male, human sales perform studies. Dependent, hostile services answer principles. Formal men show|8.66|2.68|2001002|amalgimporto #2|1|accessories|2|Men|327|ationablepri|medium|419705719frosted1973|turquoise|Dozen|Unknown|12|oughtcallyn st| +962|AAAAAAAACMDAAAAA|1997-10-27|2000-10-26|Commercial, popular processes give later now wooden facts. Black, outer purposes examine national, precise heels. Invisible times s|5.25|1.68|4001001|amalgedu pack #1|1|womens|4|Shoes|193|prin stought|medium|6576441753158lace489|spring|N/A|Unknown|26|ablecallyn st| +963|AAAAAAAACMDAAAAA|2000-10-27||Double, practical|4.25|1.68|7012004|importonameless #4|1|paint|7|Home|193|prin stought|N/A|6576441753158lace489|orchid|Lb|Unknown|41|pricallyn st| +964|AAAAAAAAEMDAAAAA|1997-10-27|1999-10-27|Dry troops may say far legal branches. Women remember for a bacteria. Poles can pass away stages. Grounds might not ask now famous ambitions. Only public dates need soon.|4.66|2.98|8008005|namelessnameless #5|8|outdoor|8|Sports|293|prin stable|N/A|5honeydew02723497867|rosy|Bunch|Unknown|26|esecallyn st| +965|AAAAAAAAEMDAAAAA|1999-10-28|2001-10-26|Possible, welsh rooms sing short procedures. |59.50|22.61|6010006|univbrand #6|8|jewelry boxes|6|Jewelry|293|prin stable|N/A|5honeydew02723497867|rose|Dram|Unknown|36|anticallyn st| +966|AAAAAAAAEMDAAAAA|2001-10-27||Topics choose minutes. Ideas choose then good dates. S|1.54|22.61|5004001|edu packscholar #1|4|classical|5|Music|293|prin stable|N/A|495white273132692209|peru|Unknown|Unknown|30|callycallyn st| +967|AAAAAAAAHMDAAAAA|1997-10-27||Open, practical ideas get. General patients can take terrible, labou|8.69|4.08|6001008|amalgcorp #8|1|birdal|6|Jewelry|273|priationable|N/A|35791934blue08435275|papaya|Ounce|Unknown|45|ationcallyn st| +968|AAAAAAAAIMDAAAAA|1997-10-27|2000-10-26|Soviet rooms can give however certain, deaf characteristics. Most marked parents would stamp costs; other, right part|0.37|0.30|1001001|amalgamalg #1|1|dresses|1|Women|197|ationn stought|extra large|231yellow34822666649|chiffon|Each|Unknown|53|eingcallyn st| +969|AAAAAAAAIMDAAAAA|2000-10-27||Soviet rooms can give however certain, deaf characteristics. Most marked parents would stamp costs; other, right part|0.36|0.23|5002002|importoscholar #2|1|country|5|Music|211|ationn stought|N/A|231yellow34822666649|lavender|Box|Unknown|64|n stcallyn st| +970|AAAAAAAAKMDAAAAA|1997-10-27|1999-10-27|Sure, dominant institutions will not search much brief times. Slowly dominant risks transform everywhere active eyes. Content, dual parents could ensure long publishers; necessary machines|9.53|5.62|10002002|importounivamalg #2|2|camcorders|10|Electronics|215|antioughtable|N/A|35gainsboro709444434|slate|Box|Unknown|16|barationn st| +971|AAAAAAAAKMDAAAAA|1999-10-28|2001-10-26|Sure, dominant institutions will not search much brief times. Slowly dominant risks transform everywhere active eyes. Content, dual parents could ensure long publishers; necessary machines|5.31|3.02|8006002|corpnameless #2|2|football|8|Sports|286|antioughtable|N/A|925302164violet56478|sky|Gross|Unknown|30|oughtationn st| +972|AAAAAAAAKMDAAAAA|2001-10-27||Sure, dominant institutions will not search much brief times. Slowly dominant risks transform everywhere active eyes. Content, dual parents could ensure long publishers; necessary machines|2.71|2.38|8006002|edu packnameless #3|2|glassware|7|Home|66|antioughtable|N/A|00274928papaya031917|peru|Tbl|Unknown|10|ableationn st| +973|AAAAAAAANMDAAAAA|1997-10-27||Major, spanish limits cover too in the group|2.03|1.15|7010004|univnameless #4|10|flatware|7|Home|331|oughtpripri|N/A|5699444990544papaya4|sandy|Oz|Unknown|1|priationn st| +974|AAAAAAAAOMDAAAAA|1997-10-27|2000-10-26|Together hot rights |4.99|3.14|9003003|exportimaxi #3|3|computers|9|Books|397|ationn stpri|N/A|3purple4617360893019|maroon|Dozen|Unknown|18|eseationn st| +975|AAAAAAAAOMDAAAAA|2000-10-27||Subject, external services control external, heavy workers. Materials could see already rights. Courts could not cost firstly taxes; difficult, proud times|88.02|79.21|9003003|namelessmaxi #4|3|romance|9|Books|397|ationn stpri|N/A|3purple4617360893019|green|Bunch|Unknown|12|antiationn st| +976|AAAAAAAAANDAAAAA|1997-10-27|1999-10-27|Beautiful areas know ever actually chief patterns. International, simple feelings like in a russians. National methods would not agree new, other practices; remote, small respects |7.13|3.99|7010007|univnameless #7|10|flatware|7|Home|537|ationprianti|N/A|3steel97717809692920|purple|Gross|Unknown|17|callyationn st| +977|AAAAAAAAANDAAAAA|1999-10-28|2001-10-26|Beautiful areas know ever actually chief patterns. International, simple feelings like in a russians. National methods would not agree new, other practices; remote, small respects |7.87|3.99|8012008|importomaxi #8|12|guns|8|Sports|537|ationprianti|N/A|4sandy15340052594504|royal|Gram|Unknown|4|ationationn st| +978|AAAAAAAAANDAAAAA|2001-10-27||Beautiful areas know ever actually chief patterns. International, simple feelings like in a russians. National methods would not agree new, other practices; remote, small respects |3.55|3.99|6012003|importobrand #3|12|costume|6|Jewelry|156|callyantiought|N/A|4sandy15340052594504|midnight|Tbl|Unknown|13|eingationn st| +979|AAAAAAAADNDAAAAA|1997-10-27||Boats return small, right words. Hours say generally hostile, hard firms. Other, bri|47.47|17.56|2004002|edu packimporto #2|4|sports-apparel|2|Men|155|antiantiought|extra large|802tan86016416599448|coral|Ton|Unknown|12|n stationn st| +980|AAAAAAAAENDAAAAA|1997-10-27|2000-10-26|Police can decline politically lately late costs. Profitable, poor ac|1.00|0.88|5001001|amalgscholar #1|1|rock|5|Music|379|n stationpri|N/A|4wheat76732885047844|royal|Carton|Unknown|50|bareingn st| +981|AAAAAAAAENDAAAAA|2000-10-27||Police can decline politically lately late costs. Profitable, poor ac|3.85|0.88|2004002|edu packimporto #2|1|sports-apparel|2|Men|379|n stationpri|extra large|4wheat76732885047844|salmon|Gram|Unknown|13|oughteingn st| +982|AAAAAAAAGNDAAAAA|1997-10-27|1999-10-27|Old times used to make p|6.73|4.03|2002001|importoimporto #1|2|shirts|2|Men|27|ationable|small|858259683210lace6438|saddle|Dram|Unknown|5|ableeingn st| +983|AAAAAAAAGNDAAAAA|1999-10-28|2001-10-26|Old times used to make p|4.28|4.03|2002001|edu packimporto #2|2|sports-apparel|2|Men|563|pricallyanti|medium|77867747058seashell5|pale|Lb|Unknown|100|prieingn st| +984|AAAAAAAAGNDAAAAA|2001-10-27||Old times used to make p|8.41|5.97|10005016|scholarunivamalg #16|5|karoke|10|Electronics|73|pricallyanti|N/A|77867747058seashell5|sienna|Each|Unknown|32|eseeingn st| +985|AAAAAAAAJNDAAAAA|1997-10-27||Questions would complete alone police; rounds would seem acceptable, valuable activities. Colleges shall fight special, royal properties. Stron|6.10|3.84|3002002|importoexporti #2|2|infants|3|Children|515|antioughtanti|small|6476880054indian4381|moccasin|Carton|Unknown|3|antieingn st| +986|AAAAAAAAKNDAAAAA|1997-10-27|2000-10-26|Attitudes could represent however tears; centres mix over morally slow ways. Model months compare then there single environments. Variables cannot find gentl|27.41|19.46|6012003|importobrand #3|12|costume|6|Jewelry|576|callyationanti|N/A|5khaki47759036680657|wheat|N/A|Unknown|4|callyeingn st| +987|AAAAAAAAKNDAAAAA|2000-10-27||Boys visit normally as a o|91.40|47.52|1004002|edu packamalg #2|4|swimwear|1|Women|576|callyationanti|small|5khaki47759036680657|black|Dozen|Unknown|28|ationeingn st| +988|AAAAAAAAMNDAAAAA|1997-10-27|1999-10-27|Successful, warm employers can show easily true, handsome brothers. Bad, great men return great, linguistic gardens. Both political tra|4.16|3.49|7013009|exportinameless #9|13|wallpaper|7|Home|134|esepriought|N/A|1065955568honeydew24|salmon|Dozen|Unknown|13|eingeingn st| +989|AAAAAAAAMNDAAAAA|1999-10-28|2001-10-26|Rigid, sudden sites borrow hence years; large developments believe here. Comfortable arrangements cut private affairs; areas study obvious, full sorts. Inc, good oblig|7.78|3.49|7013009|amalgbrand #10|13|bathroom|7|Home|134|esepriought|N/A|1065955568honeydew24|salmon|Dozen|Unknown|55|n steingn st| +990|AAAAAAAAMNDAAAAA|2001-10-27||Rigid, sudden sites borrow hence years; large developments believe here. Comfortable arrangements cut private affairs; areas study obvious, full sorts. Inc, good oblig|9.51|4.18|10006005|corpunivamalg #5|6|musical|10|Electronics|134|esepriought|N/A|93829662930violet885|wheat|Ounce|Unknown|58|barn stn st| +991|AAAAAAAAPNDAAAAA|1997-10-27||Late used patterns must solve only fiercely strategic claims. Fun readers allow academic contracts. Considerable|9.19|3.67|3003002|exportiexporti #2|3|toddlers|3|Children|44|eseese|medium|17848317355puff08001|mint|Dram|Unknown|36|oughtn stn st| +992|AAAAAAAAAODAAAAA|1997-10-27|2000-10-26|Secondary, economic pupils loo|3.68|2.98|9007009|brandmaxi #9|7|reference|9|Books|628|eingablecally|N/A|939000maroon97984085|smoke|N/A|Unknown|50|ablen stn st| +993|AAAAAAAAAODAAAAA|2000-10-27||Secondary, economic pupils loo|7.36|3.38|9007009|amalgexporti #2|1|newborn|3|Children|515|antioughtanti|large|1428961steel09376045|sky|Gram|Unknown|27|prin stn st| +994|AAAAAAAACODAAAAA|1997-10-27|1999-10-27|Smart matters get great other views. Usually wonderful factors put; huge projects supply particularly. Ultimate, following services will m|8.96|4.39|5001001|amalgscholar #1|1|rock|5|Music|450|barantiese|N/A|yellow92850746563485|pale|Pound|Unknown|28|esen stn st| +995|AAAAAAAACODAAAAA|1999-10-28|2001-10-26|Smart matters get great other views. Usually wonderful factors put; huge projects supply particularly. Ultimate, following services will m|1.63|4.39|5001001|exportinameless #6|1|wallpaper|7|Home|217|barantiese|N/A|yellow92850746563485|red|Box|Unknown|26|antin stn st| +996|AAAAAAAACODAAAAA|2001-10-27||Smart matters get great other views. Usually wonderful factors put; huge projects supply particularly. Ultimate, following services will m|87.27|4.39|5001001|exportiexporti #1|3|toddlers|3|Children|217|ationoughtable|large|yellow92850746563485|pink|Tsp|Unknown|50|callyn stn st| +997|AAAAAAAAFODAAAAA|1997-10-27||Isolated, good conditions ought to know triumphantly by the teams. Markets may go probably. Streets believe still new, equal individuals.|7.92|7.12|2003002|exportiimporto #2|3|pants|2|Men|328|eingablepri|extra large|05804royal6683640889|snow|Each|Unknown|11|ationn stn st| +998|AAAAAAAAGODAAAAA|1997-10-27|2000-10-26|Likely courses say even within a circumstances. Together local risks shall want then sympathetic, angry |2.35|1.55|3002001|importoexporti #1|2|infants|3|Children|63|prically|small|6449509sky2526556470|white|Ton|Unknown|33|eingn stn st| +999|AAAAAAAAGODAAAAA|2000-10-27||Maximum friends kiss other, quiet readers. Also single ho|5.96|4.29|3001002|amalgexporti #2|1|newborn|3|Children|63|prically|small|6449509sky2526556470|sienna|Tbl|Unknown|55|n stn stn st| +1000|AAAAAAAAIODAAAAA|1997-10-27|1999-10-27|Young, right players|7.27|3.92|2001001|amalgimporto #1|1|accessories|2|Men|433|pripriese|extra large|4255883406indian2338|seashell|Oz|Unknown|56|barbarbarought| +1001|AAAAAAAAIODAAAAA|1999-10-28|2001-10-26|Young, right players|0.90|0.67|2001001|edu packedu pack #2|1|athletic|4|Shoes|313|prioughtpri|medium|780salmon69484003964|sandy|Lb|Unknown|37|oughtbarbarought| +1002|AAAAAAAAIODAAAAA|2001-10-27||Young, right players|80.59|24.98|2001001|brandmaxi #7|7|reference|9|Books|16|prioughtpri|N/A|3639287046plum586830|violet|Ounce|Unknown|13|ablebarbarought| +1003|AAAAAAAALODAAAAA|1997-10-27||Sharp, poor rights make in the r|4.20|3.19|6007004|brandcorp #4|7|pendants|6|Jewelry|184|eseeingought|N/A|3231plum774271579552|thistle|Tsp|Unknown|45|pribarbarought| +1004|AAAAAAAAMODAAAAA|1997-10-27|2000-10-26|Critically green vegetables continue just men. White cases must take by a attitudes. Good, true costs explain over implicit shares. Commercial, following cells feel available crimes. Ini|0.23|0.11|9010009|univunivamalg #9|10|travel|9|Books|379|n stationpri|N/A|steel988173130823558|spring|Dram|Unknown|95|esebarbarought| +1005|AAAAAAAAMODAAAAA|2000-10-27||Critically green vegetables continue just men. White cases must take by a attitudes. Good, true costs explain over implicit shares. Commercial, following cells feel available crimes. Ini|4.68|1.87|9010009|amalgnameless #6|11|accent|7|Home|379|n stationpri|N/A|steel988173130823558|snow|Ton|Unknown|43|antibarbarought| +1006|AAAAAAAAOODAAAAA|1997-10-27|1999-10-27|Above ships can upset before public children; however sharp consumers may not see great pounds. Environme|6.00|2.64|8012007|importomaxi #7|12|guns|8|Sports|142|ableeseought|N/A|snow8539698287499100|blue|Bunch|Unknown|12|callybarbarought| +1007|AAAAAAAAOODAAAAA|1999-10-28|2001-10-26|Sure official findings cannot return gladly popular, true years. Attractive kinds resort other, late activities. Articles would perform in a re|2.48|2.64|6003008|exporticorp #8|12|gold|6|Jewelry|142|ableeseought|N/A|cream800218807236387|orange|Carton|Unknown|13|ationbarbarought| +1008|AAAAAAAAOODAAAAA|2001-10-27||Schools could like then facts. Main categories come particularly to a effects. Other, relative votes shall not know with a results. Russian n|8.07|6.05|10006006|corpunivamalg #6|12|musical|10|Electronics|394|esen stpri|N/A|10111peach9988649381|cyan|Dozen|Unknown|44|eingbarbarought| +1009|AAAAAAAABPDAAAAA|1997-10-27||Final differences reduce today costs. Only children ought to e|1.83|0.93|10007007|brandunivamalg #7|7|personal|10|Electronics|740|bareseation|N/A|1902lime711397019888|azure|Carton|Unknown|11|n stbarbarought| +1010|AAAAAAAACPDAAAAA|1997-10-27|2000-10-26|Only ideas might support children. Certainly local minds savour no doubt significant, financial teachers. Bright things will not lead subjects. Historical, male computer|5.79|2.60|5003001|exportischolar #1|3|pop|5|Music|326|callyablepri|N/A|5salmon8653237031606|wheat|Cup|Unknown|68|baroughtbarought| +1011|AAAAAAAACPDAAAAA|2000-10-27||Then heavy decisions used to indicate early companies. Players must not smooth above female products; corporate prices know so emotional, mass surveys. Standards sh|0.17|2.60|8001002|amalgnameless #2|1|athletic shoes|8|Sports|326|callyablepri|N/A|4tan2148771583086235|wheat|Dram|Unknown|19|oughtoughtbarought| +1012|AAAAAAAAEPDAAAAA|1997-10-27|1999-10-27|However straightforward principles used|94.89|76.86|4001001|amalgedu pack #1|1|womens|4|Shoes|37|ationpri|petite|3033peach71387836439|seashell|Case|Unknown|45|ableoughtbarought| +1013|AAAAAAAAEPDAAAAA|1999-10-28|2001-10-26|High, separate soldiers must not refer strong, new terms; b|1.47|76.86|4001001|amalgscholar #2|1|rock|5|Music|37|ationpri|N/A|6medium9333853473288|salmon|Bunch|Unknown|11|prioughtbarought| +1014|AAAAAAAAEPDAAAAA|2001-10-27||High, separate soldiers must not refer strong, new terms; b|7.78|76.86|3004001|edu packexporti #1|1|school-uniforms|3|Children|207|ationpri|medium|797560rose6068013658|drab|Lb|Unknown|6|eseoughtbarought| +1015|AAAAAAAAHPDAAAAA|1997-10-27||Afraid years suspend much building|6.81|2.31|1001002|amalgamalg #2|1|dresses|1|Women|545|antieseanti|extra large|99894237tan735493935|rose|Tbl|Unknown|1|antioughtbarought| +1016|AAAAAAAAIPDAAAAA|1997-10-27|2000-10-26|Australian, current gentlemen ought to go other drawings. Subjects go for a women. Level, personal lawyers turn aside so minimum recommendations. New places despair just peaceful refugee|4.79|3.25|4001001|amalgedu pack #1|1|womens|4|Shoes|844|eseeseeing|medium|504849739plum9167848|seashell|Pallet|Unknown|15|callyoughtbarought| +1017|AAAAAAAAIPDAAAAA|2000-10-27||Practical, local dreams go banks; various descriptions overcome differently important sen|4.34|3.25|4001001|importoedu pack #2|2|mens|4|Shoes|844|eseeseeing|small|504849739plum9167848|moccasin|Dozen|Unknown|31|ationoughtbarought| +1018|AAAAAAAAKPDAAAAA|1997-10-27|1999-10-27|Almost separate f|0.36|0.29|10015016|scholaramalgamalg #16|15|portable|10|Electronics|772|ableationation|N/A|82785712bisque506844|lemon|Tsp|Unknown|41|eingoughtbarought| +1019|AAAAAAAAKPDAAAAA|1999-10-28|2001-10-26|Prisoners shall not come jobs. Now other activities must present hot children. Fresh, light implications compare from a buildings. Always good years must |5.13|0.29|2002002|importoimporto #2|2|shirts|2|Men|183|prieingought|small|82785712bisque506844|spring|Cup|Unknown|29|n stoughtbarought| +1020|AAAAAAAAKPDAAAAA|2001-10-27||Prisoners shall not come jobs. Now other activities must present hot children. Fresh, light implications compare from a buildings. Always good years must |6.59|0.29|2002002|scholarmaxi #1|5|history|9|Books|183|prieingought|N/A|5294248violet2635895|sienna|Dozen|Unknown|28|barablebarought| +1021|AAAAAAAANPDAAAAA|1997-10-27||Years teach then at once right heads. Agencies will say by a suppliers. Most permanent blacks should not promote as other, legal eyes. Courses get so double, light flowers. Eastern, british times c|2.90|2.08|9008002|namelessmaxi #2|8|romance|9|Books|12|ableought|N/A|81889945518seashell1|red|Gross|Unknown|19|oughtablebarought| +1022|AAAAAAAAOPDAAAAA|1997-10-27|2000-10-26|Certain, regular ways will receive then now old areas. Comments give in a acts. Special, whole galleries know further possible, cool provi|0.16|0.08|2004001|edu packimporto #1|4|sports-apparel|2|Men|158|eingantiought|petite|140340white428921825|pale|Bunch|Unknown|11|ableablebarought| +1023|AAAAAAAAOPDAAAAA|2000-10-27||Certain, regular ways will receive then now old areas. Comments give in a acts. Special, whole galleries know further possible, cool provi|50.74|26.38|6002008|importocorp #8|2|diamonds|6|Jewelry|158|eingantiought|N/A|140340white428921825|olive|Ton|Unknown|18|priablebarought| +1024|AAAAAAAAAAEAAAAA|1997-10-27|1999-10-27|Individual, other things used to occur still. Conscious, popular songs begin provincial, free hours. Unhappy, brief sections use wit|5.65|3.78|4004001|edu packedu pack #1|4|athletic|4|Shoes|865|anticallyeing|medium|365262indian98522959|spring|Pound|Unknown|25|eseablebarought| +1025|AAAAAAAAAAEAAAAA|1999-10-28|2001-10-26|Productive techniques leave highly ago central sides. Chronic, local circumstances confuse clear questions; languages need from a problems. Narrow years exist for example so publ|5.07|3.78|6009002|maxicorp #2|9|womens watch|6|Jewelry|263|anticallyeing|N/A|0342slate79329611140|red|Bundle|Unknown|57|antiablebarought| +1026|AAAAAAAAAAEAAAAA|2001-10-27||Important, big items buy actually to a police. Just expensive rights may not go really part|86.80|78.12|10004007|edu packunivamalg #7|4|audio|10|Electronics|798|anticallyeing|N/A|0342slate79329611140|violet|Tsp|Unknown|33|callyablebarought| +1027|AAAAAAAADAEAAAAA|1997-10-27||Stupid errors must make for the books. So industrial rates used to exist simply; literary|3.84|2.61|6011004|amalgbrand #4|11|semi-precious|6|Jewelry|71|oughtation|N/A|9384926tan3823264470|peach|Ounce|Unknown|16|ationablebarought| +1028|AAAAAAAAEAEAAAAA|1997-10-27|2000-10-26|Here able guns would not win main, independent vessels. Pleasant, white pounds know recordings. At least possible characters see also for a races. British, clear |9.50|5.89|7014009|edu packnameless #9|14|glassware|7|Home|270|barationable|N/A|781177664plum0704508|thistle|Cup|Unknown|38|eingablebarought| +1029|AAAAAAAAEAEAAAAA|2000-10-27||Communications grab as. Never new minutes release unfair occ|1.09|5.89|7014009|amalgmaxi #10|1|arts|9|Books|270|barationable|N/A|781177664plum0704508|puff|Gross|Unknown|60|n stablebarought| +1030|AAAAAAAAGAEAAAAA|1997-10-27|1999-10-27|Rather able men set important, young hands. Never dangerous stages can see only here public fingers. Already unique police shall sleep certain styl|6.16|4.37|8001001|amalgnameless #1|1|athletic shoes|8|Sports|146|callyeseought|N/A|14royal0759010771421|medium|Carton|Unknown|4|barpribarought| +1031|AAAAAAAAGAEAAAAA|1999-10-28|2001-10-26|Rather able men set important, young hands. Never dangerous stages can see only here public fingers. Already unique police shall sleep certain styl|8.53|4.37|9004012|edu packmaxi #12|4|entertainments|9|Books|146|callyeseought|N/A|14royal0759010771421|royal|Pallet|Unknown|10|oughtpribarought| +1032|AAAAAAAAGAEAAAAA|2001-10-27||Rather able men set important, young hands. Never dangerous stages can see only here public fingers. Already unique police shall sleep certain styl|1.77|1.07|9015001|scholarunivamalg #1|4|fiction|9|Books|272|callyeseought|N/A|14royal0759010771421|peru|Tbl|Unknown|6|ablepribarought| +1033|AAAAAAAAJAEAAAAA|1997-10-27||Supplies can give. Overseas, keen attacks draw actually actual, lesser disputes. Probably wild ways aim actually economic, necessary speeches. Full |1.24|0.96|10013014|exportiamalgamalg #14|13|stereo|10|Electronics|568|eingcallyanti|N/A|45012sky541559218375|salmon|Dram|Unknown|10|pripribarought| +1034|AAAAAAAAKAEAAAAA|1997-10-27|2000-10-26|High needs live both christian budgets. Formidable feet shall not close more at the elections.|4.77|3.86|10007015|brandunivamalg #15|7|personal|10|Electronics|813|prioughteing|N/A|0891135795peru426788|smoke|Pound|Unknown|65|esepribarought| +1035|AAAAAAAAKAEAAAAA|2000-10-27||Open real cards might perform then directly public developments. A |4.33|3.86|2004002|edu packimporto #2|4|sports-apparel|2|Men|813|prioughteing|medium|3ghost22862818669842|violet|Case|Unknown|26|antipribarought| +1036|AAAAAAAAMAEAAAAA|1997-10-27|1999-10-27|Civil sites overlap conditions. More high interests send. Real, human cases provide straight enquiries. Months collect again just specifi|7.92|6.01|9009005|maximaxi #5|9|science|9|Books|306|callybarpri|N/A|7496papaya6653699061|snow|Ton|Unknown|25|callypribarought| +1037|AAAAAAAAMAEAAAAA|1999-10-28|2001-10-26|Popular boots say both rough, important police. Nations steer long|5.35|6.01|9009005|amalgunivamalg #1|1|cameras|10|Electronics|306|callybarpri|N/A|20775seashell4913696|white|Gross|Unknown|33|ationpribarought| +1038|AAAAAAAAMAEAAAAA|2001-10-27||Popular boots say both rough, important police. Nations steer long|7.73|4.79|9009005|edu packamalg #1|4|swimwear|1|Women|422|ableableese|large|moccasin904476293758|orange|Tbl|Unknown|95|eingpribarought| +1039|AAAAAAAAPAEAAAAA|1997-10-27||Patterns must achieve more always corporate problems. Windows like less usually senior cards. For example necessary colours should work fully av|1.07|0.62|3004002|edu packexporti #2|4|school-uniforms|3|Children|139|n stpriought|extra large|95612798metallic9844|wheat|Each|Unknown|27|n stpribarought| +1040|AAAAAAAAABEAAAAA|1997-10-27|2000-10-26|Now good legs find from the ideas. Available courts must risk eventually more complex strangers. Sections|8.76|6.13|7016001|corpnameless #1|16|furniture|7|Home|331|oughtpripri|N/A|290violet77860567386|royal|Gross|Unknown|40|baresebarought| +1041|AAAAAAAAABEAAAAA|2000-10-27||Now good legs find from the ideas. Available courts must risk eventually more complex strangers. Sections|9.01|6.13|2004002|edu packimporto #2|16|sports-apparel|2|Men|372|oughtpripri|large|15516steel6259142865|brown|Cup|Unknown|48|oughtesebarought| +1042|AAAAAAAACBEAAAAA|1997-10-27|1999-10-27|So electoral death|75.58|58.95|1001001|amalgamalg #1|1|dresses|1|Women|732|ablepriation|small|8246chocolate7147576|steel|Case|Unknown|96|ableesebarought| +1043|AAAAAAAACBEAAAAA|1999-10-28|2001-10-26|Items must record in a fans. Still spatial sections go empty chips. Wages may get old, high ey|9.10|58.95|9010012|univunivamalg #12|10|travel|9|Books|54|eseanti|N/A|5478990839037tomato9|indian|Pound|Unknown|18|priesebarought| +1044|AAAAAAAACBEAAAAA|2001-10-27||As sure |2.54|2.10|3004001|edu packexporti #1|10|school-uniforms|3|Children|354|eseantipri|small|5478990839037tomato9|peru|Carton|Unknown|25|eseesebarought| +1045|AAAAAAAAFBEAAAAA|1997-10-27||Absolutely old payments will b|6.40|3.52|1002002|importoamalg #2|2|fragrances|1|Women|240|bareseable|small|90656262798yellow006|midnight|Ton|Unknown|15|antiesebarought| +1046|AAAAAAAAGBEAAAAA|1997-10-27|2000-10-26|Later wonderful authorities must get famous terms. Articles shall vary to a shoulders. Exhibitions replace far good councillors. Feet can increase rarely later high sales. Open c|2.10|1.11|9012003|importounivamalg #3|12|home repair|9|Books|866|callycallyeing|N/A|6645574359sienna8381|seashell|Ounce|Unknown|29|callyesebarought| +1047|AAAAAAAAGBEAAAAA|2000-10-27||Later wonderful authorities must get famous terms. Articles shall vary to a shoulders. Exhibitions replace far good councillors. Feet can increase rarely later high sales. Open c|3.31|2.18|4001002|amalgedu pack #2|1|womens|4|Shoes|6|cally|medium|6645574359sienna8381|slate|Case|Unknown|45|ationesebarought| +1048|AAAAAAAAIBEAAAAA||1999-10-27||14.70|12.20||importoedu pack #1||||Shoes|159|n stantiought||9white02789225104416||Cup||16|eingesebarought| +1049|AAAAAAAAIBEAAAAA|1999-10-28|2001-10-26|Almost other shares go. Soft, old items used to understand. Typically complete names stop here correct enemies. |2.62|2.17|4002002|importoedu pack #2|2|mens|4|Shoes|159|n stantiought|extra large|14225682896indian053|yellow|Tbl|Unknown|38|n stesebarought| +1050|AAAAAAAAIBEAAAAA|2001-10-27||Almost other shares go. Soft, old items used to understand. Typically complete names stop here correct enemies. |9.58|2.17|7006001|corpbrand #1|6|rugs|7|Home|159|n stantiought|N/A|14225682896indian053|tomato|Gram|Unknown|15|barantibarought| +1051|AAAAAAAALBEAAAAA|1997-10-27||Proposed facilities might prefer. Pages can go appropriate, friendly titles. Doctors m|48.57|22.82|7008002|namelessbrand #2|8|lighting|7|Home|610|baroughtcally|N/A|65263yellow608068517|green|Dram|Unknown|7|oughtantibarought| +1052|AAAAAAAAMBEAAAAA|1997-10-27|2000-10-26|Clearly local bars put still. |0.69|0.31|7007003|brandbrand #3|7|decor|7|Home|911|oughtoughtn st|N/A|254287949532olive260|pale|Bunch|Unknown|17|ableantibarought| +1053|AAAAAAAAMBEAAAAA|2000-10-27||Clearly local bars put still. |8.74|4.80|8003004|exportinameless #4|3|basketball|8|Sports|168|oughtoughtn st|N/A|76337395medium550348|green|Unknown|Unknown|25|priantibarought| +1054|AAAAAAAAOBEAAAAA|1997-10-27|1999-10-27|Various, no|29.04|25.26|4003001|exportiedu pack #1|3|kids|4|Shoes|252|ableantiable|medium|04637320288848pink17|royal|Each|Unknown|22|eseantibarought| +1055|AAAAAAAAOBEAAAAA|1999-10-28|2001-10-26|Difficult, certain hands hear from the relations. Real targets know much medical managers; cultural centuries claim enterprises. Similar, large ga|8.44|7.59|4003001|scholarnameless #6|5|fitness|8|Sports|252|ableantiable|N/A|296457white480598149|spring|Cup|Unknown|24|antiantibarought| +1056|AAAAAAAAOBEAAAAA|2001-10-27||Difficult, certain hands hear from the relations. Real targets know much medical managers; cultural centuries claim enterprises. Similar, large ga|3.99|7.59|5001001|amalgscholar #1|1|rock|5|Music|634|eseprically|N/A|296457white480598149|salmon|Unknown|Unknown|25|callyantibarought| +1057|AAAAAAAABCEAAAAA|1997-10-27||New, private values will not lose badly relationships. Republics must bear; stations will settle much pretty, sufficient prob|2.01|1.36|6012002|importobrand #2|12|costume|6|Jewelry|764|esecallyation|N/A|02914948844084cyan10|navajo|Case|Unknown|50|ationantibarought| +1058|AAAAAAAACCEAAAAA|1997-10-27|2000-10-26|Still miserable views ought to remi|3.51|2.84|4004001|edu packedu pack #1|4|athletic|4|Shoes|188|eingeingought|small|21618169985smoke9132|white|Ounce|Unknown|1|eingantibarought| +1059|AAAAAAAACCEAAAAA|2000-10-27||Formal references affect again for a rights. So far minutes reveal still because of a priorities|8.55|2.84|8012010|importomaxi #10|12|guns|8|Sports|188|eingeingought|N/A|21618169985smoke9132|smoke|Pound|Unknown|3|n stantibarought| +1060|AAAAAAAAECEAAAAA|1997-10-27|1999-10-27|Apparently eligible centres provide in or|2.32|0.78|1003001|exportiamalg #1|3|maternity|1|Women|257|ationantiable|extra large|8127348733367royal45|ghost|Ounce|Unknown|33|barcallybarought| +1061|AAAAAAAAECEAAAAA|1999-10-28|2001-10-26|Apparently eligible centres provide in or|0.88|0.38|7005002|scholarbrand #2|3|blinds/shades|7|Home|257|ationantiable|N/A|8127348733367royal45|snow|Each|Unknown|52|oughtcallybarought| +1062|AAAAAAAAECEAAAAA|2001-10-27||Apparently eligible centres provide in or|4.44|0.38|7005002|scholarunivamalg #9|3|karoke|10|Electronics|171|oughtationought|N/A|8127348733367royal45|lime|Unknown|Unknown|62|ablecallybarought| +1063|AAAAAAAAHCEAAAAA|1997-10-27||Almost significant organisations ought to get downwards partly different pupils. Prices understand things. Chosen, only banks can upset only logic|3.04|1.09|1003002|exportiamalg #2|3|maternity|1|Women|840|bareseeing|medium|497449navajo27838303|yellow|Dozen|Unknown|39|pricallybarought| +1064|AAAAAAAAICEAAAAA|1997-10-27|2000-10-26|Agents show days. Economic councillors can get merely large con|14.21|4.97|2002001|importoimporto #1|2|shirts|2|Men|528|eingableanti|medium|3033951677tomato7349|pink|Ton|Unknown|3|esecallybarought| +1065|AAAAAAAAICEAAAAA|2000-10-27||Sim|4.33|4.97|6001002|amalgcorp #2|2|birdal|6|Jewelry|528|eingableanti|N/A|9124lemon23752720138|seashell|Unknown|Unknown|66|anticallybarought| +1066|AAAAAAAAKCEAAAAA|1997-10-27|1999-10-27|Relevant, subject courses will not deny certainly |1.36|0.55|10012013|importoamalgamalg #13|12|monitors|10|Electronics|46|callyese|N/A|39005284white4558095|peach|Gram|Unknown|45|callycallybarought| +1067|AAAAAAAAKCEAAAAA|1999-10-28|2001-10-26|Relevant, subject courses will not deny certainly |7.98|3.59|10012013|importounivamalg #14|2|camcorders|10|Electronics|269|n stcallyable|N/A|8153526886lemon05647|pink|Bundle|Unknown|37|ationcallybarought| +1068|AAAAAAAAKCEAAAAA|2001-10-27||Difficult windows respond to the firms. Similar members like all |6.75|3.59|10012013|exportischolar #1|2|pop|5|Music|249|n steseable|N/A|8153526886lemon05647|steel|Tbl|Unknown|19|eingcallybarought| +1069|AAAAAAAANCEAAAAA|1997-10-27||Yet private services question soon impossible protests; so social forms sha|4.55|3.73|1002002|importoamalg #2|2|fragrances|1|Women|167|ationcallyought|extra large|267392smoke960317841|slate|Dram|Unknown|56|n stcallybarought| +1070|AAAAAAAAOCEAAAAA|1997-10-27|2000-10-26|United estates experience better new, little proposals. Great weeks refer both beautiful women. Big, equal guards rest interesting, right students. Local, particu|0.18|0.07|2001001|amalgimporto #1|1|accessories|2|Men|107|ationbarought|extra large|86saddle429568519798|midnight|Pound|Unknown|22|barationbarought| +1071|AAAAAAAAOCEAAAAA|2000-10-27||United estates experience better new, little proposals. Great weeks refer both beautiful women. Big, equal guards rest interesting, right students. Local, particu|3.90|2.49|2001001|edu packexporti #2|1|school-uniforms|3|Children|107|ationbarought|small|86saddle429568519798|slate|Gross|Unknown|61|oughtationbarought| +1072|AAAAAAAAADEAAAAA|1997-10-27|1999-10-27|Days stick fairly big, integrated women. Much other fortunes ought to work so to the losses. Subsidies take |2.57|1.10|7001003|amalgbrand #3|1|bathroom|7|Home|387|ationeingpri|N/A|17605violet059170574|thistle|Unknown|Unknown|84|ableationbarought| +1073|AAAAAAAAADEAAAAA|1999-10-28|2001-10-26|Acres can cause here with a talks. Original, old notes may bring also basic, soviet patients. Curious, g|5.21|1.10|7008004|namelessbrand #4|8|lighting|7|Home|692|ationeingpri|N/A|17605violet059170574|azure|Unknown|Unknown|1|priationbarought| +1074|AAAAAAAAADEAAAAA|2001-10-27||Acres can cause here with a talks. Original, old notes may bring also basic, soviet patients. Curious, g|0.95|0.70|7008004|importoedu pack #1|2|mens|4|Shoes|342|ationeingpri|medium|17605violet059170574|dark|Case|Unknown|6|eseationbarought| +1075|AAAAAAAADDEAAAAA|1997-10-27||Main concentrations must not fix l|33.23|29.24|3003002|exportiexporti #2|3|toddlers|3|Children|233|pripriable|extra large|1058steel67348196657|lawn|Pound|Unknown|21|antiationbarought| +1076|AAAAAAAAEDEAAAAA|1997-10-27|2000-10-26|Marginal, top feet play also previously simple records. Months cannot imitate almost previous chains;|9.68|8.51|10014006|edu packamalgamalg #6|14|automotive|10|Electronics|145|antieseought|N/A|24479976spring352241|royal|Tbl|Unknown|64|callyationbarought| +1077|AAAAAAAAEDEAAAAA|2000-10-27||Marginal, top feet play also previously simple records. Months cannot imitate almost previous chains;|4.31|1.89|10016007|corpamalgamalg #7|16|wireless|10|Electronics|280|bareingable|N/A|69926turquoise507554|spring|Ounce|Unknown|55|ationationbarought| +1078|AAAAAAAAGDEAAAAA|1997-10-27|1999-10-27|United, dangerous plants commit only expected states; soft colleges issue in addition. Too visible clubs used to help strongly in a services|0.47|0.21|4004001|edu packedu pack #1|4|athletic|4|Shoes|183|prieingought|petite|8royal56896261477133|metallic|Oz|Unknown|7|eingationbarought| +1079|AAAAAAAAGDEAAAAA|1999-10-28|2001-10-26|Monthly hands feel here lexical customers. Countries ought to kill too better personal students; key window|3.96|0.21|4004001|namelessnameless #10|8|outdoor|8|Sports|416|prieingought|N/A|569605orange68149178|steel|Gross|Unknown|10|n stationbarought| +1080|AAAAAAAAGDEAAAAA|2001-10-27||Monthly hands feel here lexical customers. Countries ought to kill too better personal students; key window|7.11|0.21|4004001|edu packedu pack #1|8|athletic|4|Shoes|191|prieingought|extra large|171430632611steel569|pink|Ton|Unknown|14|bareingbarought| +1081|AAAAAAAAJDEAAAAA|1997-10-27||Economic, impressive thoughts will not neglect. Strong, serious records should capture o|8.11|7.13|8016002|corpmaxi #2|16|golf|8|Sports|129|n stableought|N/A|0486195194317red5173|steel|Bunch|Unknown|73|oughteingbarought| +1082|AAAAAAAAKDEAAAAA|1997-10-27|2000-10-26|Yet national bodies could answer on behalf of a hours. Features use later workers. Fortunes placa|6.46|2.45|9004003|edu packmaxi #3|4|entertainments|9|Books|59|n stanti|N/A|060225840puff7468957|snow|Tbl|Unknown|29|ableeingbarought| +1083|AAAAAAAAKDEAAAAA|2000-10-27||Yet national bodies could answer on behalf of a hours. Features use later workers. Fortunes placa|2.82|2.45|9004003|scholarbrand #4|15|custom|6|Jewelry|59|n stanti|N/A|925299986spring35099|burnished|Box|Unknown|37|prieingbarought| +1084|AAAAAAAAMDEAAAAA|1997-10-27|1999-10-27|Today british hills include p|0.52|0.28|8006005|corpnameless #5|6|football|8|Sports|295|antin stable|N/A|16390158royal3538163|rosy|Oz|Unknown|28|eseeingbarought| +1085|AAAAAAAAMDEAAAAA|1999-10-28|2001-10-26|Today british hills include p|3.32|0.28|8006005|brandcorp #6|7|pendants|6|Jewelry|295|antin stable|N/A|697283black008079569|almond|Ton|Unknown|37|antieingbarought| +1086|AAAAAAAAMDEAAAAA|2001-10-27||Charges could not attempt natural, gold rocks. Libraries will call wild sentences; other, relevant conferences can pray far upon a years. Essential passages look a|1.92|1.51|10005016|scholarunivamalg #16|7|karoke|10|Electronics|295|antin stable|N/A|697283black008079569|powder|Bundle|Unknown|96|callyeingbarought| +1087|AAAAAAAAPDEAAAAA|1997-10-27||Private, other years would create spontaneously pregnant services. Br|45.09|15.78|6004008|edu packcorp #8|4|bracelets|6|Jewelry|128|eingableought|N/A|66874white6619416914|bisque|Carton|Unknown|20|ationeingbarought| +1088|AAAAAAAAAEEAAAAA|1997-10-27|2000-10-26|Ordinary systems remain certainly elsewhere ge|7.15|3.36|3004001|edu packexporti #1|4|school-uniforms|3|Children|111|oughtoughtought|medium|lavender460273424521|pink|Gross|Unknown|96|eingeingbarought| +1089|AAAAAAAAAEEAAAAA|2000-10-27||Ordinary systems remain certainly elsewhere ge|3.80|3.36|3004001|edu packmaxi #10|14|tennis|8|Sports|186|callyeingought|N/A|8785696090magenta316|ivory|Bundle|Unknown|54|n steingbarought| +1090|AAAAAAAACEEAAAAA|1997-10-27|1999-10-27|Manufacturing, illegal areas ought to work by a seats. Brief, alone terms realise more forces. Telecommunications might not get in a classes. Police should see more |4.75|2.85|4003001|exportiedu pack #1|3|kids|4|Shoes|325|antiablepri|extra large|056616575947thistle4|salmon|Pound|Unknown|42|barn stbarought| +1091|AAAAAAAACEEAAAAA|1999-10-28|2001-10-26|Manufacturing, illegal areas ought to work by a seats. Brief, alone terms realise more forces. Telecommunications might not get in a classes. Police should see more |1.27|2.85|4003001|scholarmaxi #2|15|fishing|8|Sports|579|n stationanti|N/A|056616575947thistle4|snow|N/A|Unknown|17|oughtn stbarought| +1092|AAAAAAAACEEAAAAA|2001-10-27||Manufacturing, illegal areas ought to work by a seats. Brief, alone terms realise more forces. Telecommunications might not get in a classes. Police should see more |6.71|4.16|10006005|corpunivamalg #5|6|musical|10|Electronics|579|n stationanti|N/A|4707847527salmon5618|sky|Carton|Unknown|47|ablen stbarought| +1093|AAAAAAAAFEEAAAAA|1997-10-27||Capital, united feelings paint only things. Greatly financial economies should not pay somewhere soviet necessary armies; educational concepts mus|3.83|1.30|9003002|exportimaxi #2|3|computers|9|Books|643|priesecally|N/A|seashell184727810187|seashell|Each|Unknown|54|prin stbarought| +1094|AAAAAAAAGEEAAAAA|1997-10-27|2000-10-26|Below urban facilities look |0.52|0.23|6010007|univbrand #7|10|jewelry boxes|6|Jewelry|225|antiableable|N/A|994988246594wheat066|chartreuse|Lb|Unknown|72|esen stbarought| +1095|AAAAAAAAGEEAAAAA|2000-10-27||Hands think teams; young windows should agree better effective cha|2.32|1.39|9008004|namelessmaxi #4|8|romance|9|Books|220|barableable|N/A|994988246594wheat066|rose|Case|Unknown|42|antin stbarought| +1096|AAAAAAAAIEEAAAAA|1997-10-27|1999-10-27|Medical, old shops would not put very likely stones; clergy will maximise for good negotiations. Technically hard months leave just subsequent parents. Good, social banks shall not c|2.00|1.74|3004001|edu packexporti #1|4|school-uniforms|3|Children|119|n stoughtought|petite|924330217sienna41731|metallic|Oz|Unknown|8|callyn stbarought| +1097|AAAAAAAAIEEAAAAA|1999-10-28|2001-10-26|Medical, old shops would not put very likely stones; clergy will maximise for good negotiations. Technically hard months leave just subsequent parents. Good, social banks shall not c|6.40|1.74|9008006|namelessmaxi #6|8|romance|9|Books|119|n stoughtought|N/A|7055turquoise3714148|plum|Bunch|Unknown|5|ationn stbarought| +1098|AAAAAAAAIEEAAAAA|2001-10-27||Medical, old shops would not put very likely stones; clergy will maximise for good negotiations. Technically hard months leave just subsequent parents. Good, social banks shall not c|0.93|0.70|9008006|importomaxi #7|8|business|9|Books|465|n stoughtought|N/A|7055turquoise3714148|papaya|Tbl|Unknown|4|eingn stbarought| +1099|AAAAAAAALEEAAAAA|1997-10-27||Terms cannot enc|5.90|3.71|8007010|brandnameless #10|7|hockey|8|Sports|267|ationcallyable|N/A|2533saddle5416739167|papaya|Pallet|Unknown|34|n stn stbarought| +1100|AAAAAAAAMEEAAAAA|1997-10-27|2000-10-26|Long fresh shots shall believe both most electronic others. Other police could reverse at least available, late years. Sums must |0.66|0.55|5004001|edu packscholar #1|4|classical|5|Music|227|ationableable|N/A|616468thistle8513491|violet|Each|Unknown|40|barbaroughtought| +1101|AAAAAAAAMEEAAAAA|2000-10-27||Abstract, dead members look sorry child|0.81|0.68|6011006|amalgbrand #6|11|semi-precious|6|Jewelry|328|eingablepri|N/A|7017142985tan2515344|slate|Each|Unknown|30|oughtbaroughtought| +1102|AAAAAAAAOEEAAAAA|1997-10-27|1999-10-27|Temporary, hea|0.67|0.52|2004001|edu packimporto #1|4|sports-apparel|2|Men|291|oughtn stable|medium|2896458sandy64752575|papaya|Pound|Unknown|37|ablebaroughtought| +1103|AAAAAAAAOEEAAAAA|1999-10-28|2001-10-26|Willing, familiar institutions like. Western periods hold. Numbers make always golden terms. National roads will not per|2.40|0.52|4001002|amalgedu pack #2|1|womens|4|Shoes|291|oughtn stable|large|2896458sandy64752575|sky|Gross|Unknown|49|pribaroughtought| +1104|AAAAAAAAOEEAAAAA|2001-10-27||Inside base regulations defeat in full for a others; still normal payments work again famous members. Facilities should not remain so upper links. Arms would give from a|4.70|0.52|8013005|exportimaxi #5|13|sailing|8|Sports|291|oughtn stable|N/A|57lace76243365674195|smoke|Bundle|Unknown|21|esebaroughtought| +1105|AAAAAAAABFEAAAAA|1997-10-27||Letters make around elderly, clear supporters; sales can wish securely darling mechanisms. Authorities used to keep conditions. Political sports may matter also. Po|9.23|7.66|4002002|importoedu pack #2|2|mens|4|Shoes|529|n stableanti|extra large|464steel201931282283|grey|Ton|Unknown|11|antibaroughtought| +1106|AAAAAAAACFEAAAAA|1997-10-27|2000-10-26|Students can rule fast bad poor times. Other, short levels could see from a jobs. Old actions put as criteria; ideal, awful contacts shal|1.42|0.83|2003001|exportiimporto #1|3|pants|2|Men|151|oughtantiought|medium|978123494peach075549|sienna|Dram|Unknown|6|callybaroughtought| +1107|AAAAAAAACFEAAAAA|2000-10-27||Students can rule fast bad poor times. Other, short levels could see from a jobs. Old actions put as criteria; ideal, awful contacts shal|2.52|0.95|2003001|exportischolar #2|3|pop|5|Music|151|oughtantiought|N/A|002pink4846804652269|rosy|Gross|Unknown|67|ationbaroughtought| +1108|AAAAAAAAEFEAAAAA|1997-10-27|1999-10-27|Considerations might make of course clear services. Political, beautiful germans can become in a contents; famous areas r|2.15|0.92|9006005|corpmaxi #5|6|parenting|9|Books|523|priableanti|N/A|939115666goldenrod61|ghost|Ounce|Unknown|66|eingbaroughtought| +1109|AAAAAAAAEFEAAAAA|1999-10-28|2001-10-26|Sure, |0.74|0.92|9006005|importobrand #10|2|bedding|7|Home|523|priableanti|N/A|69959425seashell3484|purple|Tbl|Unknown|70|n stbaroughtought| +1110|AAAAAAAAEFEAAAAA|2001-10-27||Sure, |7.17|3.08|3002001|importoexporti #1|2|infants|3|Children|523|priableanti|medium|69959425seashell3484|white|Case|Unknown|27|baroughtoughtought| +1111|AAAAAAAAHFEAAAAA|1997-10-27||Fine, right provinces take pretty. Unknown, regular events notice fro|4.13|1.94|6011008|amalgbrand #8|11|semi-precious|6|Jewelry|72|ableation|N/A|314407maroon65555879|royal|Bunch|Unknown|4|oughtoughtoughtought| +1112|AAAAAAAAIFEAAAAA|1997-10-27|2000-10-26|Literary voters expect i|7.31|5.99|1004001|edu packamalg #1|4|swimwear|1|Women|188|eingeingought|extra large|saddle75510944133241|violet|Oz|Unknown|29|ableoughtoughtought| +1113|AAAAAAAAIFEAAAAA|2000-10-27||Literary voters expect i|9.76|5.99|5001002|amalgscholar #2|1|rock|5|Music|314|eingeingought|N/A|340223honeydew020949|puff|Tbl|Unknown|40|prioughtoughtought| +1114|AAAAAAAAKFEAAAAA|1997-10-27|1999-10-27|Available classes could draw linguistic politicians; extremely available times withdraw old changes; common systems would not look all particularly poor others. Free, e|8.59|3.00|5001001|amalgscholar #1|1|rock|5|Music|380|bareingpri|N/A|5685380steel68842332|blush|Ton|Unknown|31|eseoughtoughtought| +1115|AAAAAAAAKFEAAAAA|1999-10-28|2001-10-26|Available classes could draw linguistic politicians; extremely available times withdraw old changes; common systems would not look all particularly poor others. Free, e|71.43|60.71|5001001|brandunivamalg #11|1|personal|10|Electronics|380|bareingpri|N/A|53088101honeydew8403|thistle|Ounce|Unknown|1|antioughtoughtought| +1116|AAAAAAAAKFEAAAAA|2001-10-27||Available classes could draw linguistic politicians; extremely available times withdraw old changes; common systems would not look all particularly poor others. Free, e|65.21|60.71|9009001|maximaxi #1|9|science|9|Books|380|bareingpri|N/A|53088101honeydew8403|tomato|Ton|Unknown|11|callyoughtoughtought| +1117|AAAAAAAANFEAAAAA|1997-10-27||Top p|3.38|2.26|6005006|scholarcorp #6|5|earings|6|Jewelry|721|oughtableation|N/A|882293444purple32911|papaya|Unknown|Unknown|15|ationoughtoughtought| +1118|AAAAAAAAOFEAAAAA|1997-10-27|2000-10-26|Financial, unknown features could regard really. Desirable, hard glasses go fast friends. Political churches attempt; nearly required feelings will|2.34|1.49|8010009|univmaxi #9|10|pools|8|Sports|135|antipriought|N/A|5917740white82472339|lemon|Dram|Unknown|7|eingoughtoughtought| +1119|AAAAAAAAOFEAAAAA|2000-10-27||Past links must try generally late, unexpected groups.|1.05|1.49|8010009|corpnameless #10|16|furniture|7|Home|320|antipriought|N/A|17yellow604162916194|gainsboro|Case|Unknown|7|n stoughtoughtought| +1120|AAAAAAAAAGEAAAAA|1997-10-27|1999-10-27|Foreign conditions could not think scientists. Big, applicable jobs could not perform social, high profits. Even young orde|7.02|2.31|7004001|edu packbrand #1|4|curtains/drapes|7|Home|438|eingpriese|N/A|109737tomato84492021|pink|Box|Unknown|29|barableoughtought| +1121|AAAAAAAAAGEAAAAA|1999-10-28|2001-10-26|Ago recent|51.43|26.22|7004001|amalgedu pack #2|1|womens|4|Shoes|706|eingpriese|large|109737tomato84492021|indian|Box|Unknown|15|oughtableoughtought| +1122|AAAAAAAAAGEAAAAA|2001-10-27||Anyway active princ|8.11|6.73|9010007|univunivamalg #7|1|travel|9|Books|706|callybaration|N/A|497royal903582668448|snow|Bundle|Unknown|17|ableableoughtought| +1123|AAAAAAAADGEAAAAA|1997-10-27||Informal individuals give in the signs. Economic, moving reasons know to a areas. Meals will not show over subsequent examinations. Tory children provide revolutionary,|90.44|51.55|5003002|exportischolar #2|3|pop|5|Music|303|pribarpri|N/A|59366085wheat6312167|seashell|Dozen|Unknown|94|priableoughtought| +1124|AAAAAAAAEGEAAAAA|1997-10-27|2000-10-26|New, new signals would not take gently exactly brief games. Devices should feel always efforts. Obvious manage|0.22|0.17|4001001|amalgedu pack #1|1|womens|4|Shoes|651|oughtantically|medium|7097302834cornsilk80|tan|Lb|Unknown|41|eseableoughtought| +1125|AAAAAAAAEGEAAAAA|2000-10-27||More tiny practitioners could not help however mature, constitutional features. Single, basic officers must resist tomorrow consequently dependent contacts. Sovi|2.40|1.27|8008006|namelessnameless #6|1|outdoor|8|Sports|109|n stbarought|N/A|08rose84303031551853|navajo|Cup|Unknown|28|antiableoughtought| +1126|AAAAAAAAGGEAAAAA|1997-10-27|1999-10-27|Again ideal supporters determine never suspicious, british days. Industrial techniques stir states. Precisely net p|2.56|2.07|3003001|exportiexporti #1|3|toddlers|3|Children|252|ableantiable|economy|0326837597red4285044|ivory|N/A|Unknown|31|callyableoughtought| +1127|AAAAAAAAGGEAAAAA|1999-10-28|2001-10-26|Again ideal supporters determine never suspicious, british days. Industrial techniques stir states. Precisely net p|8.95|5.28|5001002|amalgscholar #2|1|rock|5|Music|252|ableantiable|N/A|62000tomato433392113|lime|Each|Unknown|19|ationableoughtought| +1128|AAAAAAAAGGEAAAAA|2001-10-27||Engines could not give open very experiments. Then ready ears assist also other, unlikely years; therein great systems will think linguistic, great pupils. Almost |4.96|5.28|5001001|amalgscholar #1|1|rock|5|Music|94|ableantiable|N/A|99207617165maroon119|plum|Dram|Unknown|14|eingableoughtought| +1129|AAAAAAAAJGEAAAAA|1997-10-27||Fully powerful qualities pinpoint thus movements. Domestic officers continue in a cases. Teachers shall introduce indeed other, good|0.65|0.35|9005002|scholarmaxi #2|5|history|9|Books|654|eseantically|N/A|545468651tomato79116|linen|Tsp|Unknown|58|n stableoughtought| +1130|AAAAAAAAKGEAAAAA|1997-10-27|2000-10-26|There possible efforts might bring yet brief, kind days. Oddly white dangers could like maximum things. Hours might |9.23|5.44|9013003|exportiunivamalg #3|13|self-help|9|Books|796|callyn station|N/A|54ghost9989892199785|wheat|Tsp|Unknown|50|barprioughtought| +1131|AAAAAAAAKGEAAAAA|2000-10-27||There possible efforts might bring yet brief, kind days. Oddly white dangers could like maximum things. Hours might |9.93|5.44|9013003|amalgscholar #2|1|rock|5|Music|796|callyn station|N/A|54ghost9989892199785|steel|Pound|Unknown|9|oughtprioughtought| +1132|AAAAAAAAMGEAAAAA|1997-10-27|1999-10-27|Responsible, great pr|2.88|1.95|10005011|scholarunivamalg #11|5|karoke|10|Electronics|153|priantiought|N/A|32green5493156877809|mint|Tbl|Unknown|84|ableprioughtought| +1133|AAAAAAAAMGEAAAAA|1999-10-28|2001-10-26|Responsible, great pr|0.79|0.56|1002002|importoamalg #2|5|fragrances|1|Women|153|priantiought|petite|32green5493156877809|khaki|Dozen|Unknown|41|priprioughtought| +1134|AAAAAAAAMGEAAAAA|2001-10-27||Efficient letters might buy about by the feet. Democratic, short suggestions play|8.25|0.56|9013007|exportiunivamalg #7|13|self-help|9|Books|153|priantiought|N/A|32green5493156877809|seashell|Pallet|Unknown|16|eseprioughtought| +1135|AAAAAAAAPGEAAAAA|1997-10-27||Northern lips must go permanently effective, ra|0.58|0.41|10005014|scholarunivamalg #14|5|karoke|10|Electronics|392|ablen stpri|N/A|6335600446026yellow5|cornflower|Oz|Unknown|43|antiprioughtought| +1136|AAAAAAAAAHEAAAAA|1997-10-27|2000-10-26|Pale children used to explain then traditional years. Quietly similar suggestions give. Evil others result j|1.69|0.96|6009001|maxicorp #1|9|womens watch|6|Jewelry|944|eseesen st|N/A|6705280sandy22789069|rose|Gross|Unknown|30|callyprioughtought| +1137|AAAAAAAAAHEAAAAA|2000-10-27||Pale children used to explain then traditional years. Quietly similar suggestions give. Evil others result j|3.64|0.96|6009001|scholarcorp #2|9|earings|6|Jewelry|944|eseesen st|N/A|6705280sandy22789069|saddle|Box|Unknown|7|ationprioughtought| +1138|AAAAAAAACHEAAAAA|1997-10-27|1999-10-27|At last political managers would get new, historic workers. Requirements seem loose shadows; activities carry favorite mothers; likely issues stand aside environmental, current funds; below|1.08|0.43|3003001|exportiexporti #1|3|toddlers|3|Children|155|antiantiought|small|610342red46498200513|pale|Tsp|Unknown|39|eingprioughtought| +1139|AAAAAAAACHEAAAAA|1999-10-28|2001-10-26|Thick directions might become overall employees. Specific years ought to think light, clear features. Strategies might not see well. Only important cases must |0.41|0.34|3003001|namelessbrand #10|8|lighting|7|Home|155|antiantiought|N/A|610342red46498200513|yellow|Bunch|Unknown|49|n stprioughtought| +1140|AAAAAAAACHEAAAAA|2001-10-27||Thick directions might become overall employees. Specific years ought to think light, clear features. Strategies might not see well. Only important cases must |4.24|0.34|2002001|importoimporto #1|8|shirts|2|Men|155|antiantiought|economy|610342red46498200513|orchid|Carton|Unknown|50|bareseoughtought| +1141|AAAAAAAAFHEAAAAA|1997-10-27||Cells stand past, recent users. As common victims must send big, handsome students. Finally asleep wives could ignore surely up to a reasons. Ears will not be|7.55|2.94|1002002|importoamalg #2|2|fragrances|1|Women|234|esepriable|N/A|817107steel628707016|violet|Dozen|Unknown|12|oughteseoughtought| +1142|AAAAAAAAGHEAAAAA|1997-10-27|2000-10-26|Principal, relative centres may hit sure yet brief neighbours; even high talks should not establish else classes. Economically remote networks |9.03|5.77|6012007|importobrand #7|12|costume|6|Jewelry|274|eseationable|N/A|78smoke1629990866486|mint|Carton|Unknown|31|ableeseoughtought| +1143|AAAAAAAAGHEAAAAA|2000-10-27||Subjects reflect only hardly successful laws. In particular following characteristics could acknowledge typical friends. Unanimo|11.11|5.77|6012007|exportischolar #2|3|pop|5|Music|274|eseationable|N/A|9metallic59088157569|white|Pallet|Unknown|62|prieseoughtought| +1144|AAAAAAAAIHEAAAAA|1997-10-27|1999-10-27|Parties register soon big goals. Arms shall not find also different qualities. Secondly blue thanks may use techniques. Grey, light critics facilitate |3.69|1.40|10014006|edu packamalgamalg #6|14|automotive|10|Electronics|504|esebaranti|N/A|grey8782160632327289|snow|Ounce|Unknown|25|eseeseoughtought| +1145|AAAAAAAAIHEAAAAA|1999-10-28|2001-10-26|Parties register soon big goals. Arms shall not find also different qualities. Secondly blue thanks may use techniques. Grey, light critics facilitate |3.11|1.40|10014006|exportiamalg #2|14|maternity|1|Women|510|esebaranti|medium|grey8782160632327289|maroon|Lb|Unknown|87|antieseoughtought| +1146|AAAAAAAAIHEAAAAA|2001-10-27||||3.03|10014006||14|maternity|||510|baroughtanti|extra large|3424374147241plum052|thistle|Dram|||callyeseoughtought| +1147|AAAAAAAALHEAAAAA|1997-10-27||Literary efforts could meet please in the terms. Difficult securities make aspects. Pressures support charming, british men. |0.75|0.38|1001002|amalgamalg #2|1|dresses|1|Women|312|ableoughtpri|petite|0930794goldenrod6531|khaki|Dozen|Unknown|27|ationeseoughtought| +1148|AAAAAAAAMHEAAAAA|1997-10-27|2000-10-26|Symbols complete then benefits. Very, different relations listen between a beings; temporary competitors discuss late left companies. Important, big methods advertise operational streets. |2.32|1.83|4004001|edu packedu pack #1|4|athletic|4|Shoes|262|ablecallyable|medium|4salmon3789135697117|thistle|Pallet|Unknown|3|eingeseoughtought| +1149|AAAAAAAAMHEAAAAA|2000-10-27||Symbols complete then benefits. Very, different relations listen between a beings; temporary competitors discuss late left companies. Important, big methods advertise operational streets. |4.19|1.83|2003002|exportiimporto #2|4|pants|2|Men|262|ablecallyable|medium|28344violet897371950|royal|Pound|Unknown|28|n steseoughtought| +1150|AAAAAAAAOHEAAAAA|1997-10-27|1999-10-27|Absolutely front men turn spatial hours. Good, free sales used to marry outside appropriate ships. Noble men sa|1.83|1.53|8006001|corpnameless #1|6|football|8|Sports|255|antiantiable|N/A|56635092133237smoke7|green|Ton|Unknown|44|barantioughtought| +1151|AAAAAAAAOHEAAAAA|1999-10-28|2001-10-26|Absolutely front men turn spatial hours. Good, free sales used to marry outside appropriate ships. Noble men sa|1.49|1.53|4001002|amalgedu pack #2|1|womens|4|Shoes|255|antiantiable|petite|86116645turquoise474|steel|Gross|Unknown|2|oughtantioughtought| +1152|AAAAAAAAOHEAAAAA|2001-10-27||Smart, potential countries used to keep without a users. Good, due communications can get poss|48.00|41.28|9016001|corpunivamalg #1|16|mystery|9|Books|255|antiantiable|N/A|86116645turquoise474|steel|Carton|Unknown|31|ableantioughtought| +1153|AAAAAAAABIEAAAAA|1997-10-27||Eyes help; residents train even essential classes. Concerned gardens make most black, genuine objects; atomic heads must interpret practically short social pai|9.98|4.69|6003002|exporticorp #2|3|gold|6|Jewelry|175|antiationought|N/A|88969010saddle566529|powder|Gram|Unknown|25|priantioughtought| +1154|AAAAAAAACIEAAAAA|1997-10-27|2000-10-26|Police|2.57|2.23|10013016|exportiamalgamalg #16|13|stereo|10|Electronics|227|ationableable|N/A|54639blanched1242985|white|Oz|Unknown|6|eseantioughtought| +1155|AAAAAAAACIEAAAAA|2000-10-27||Offers cannot ask for example in a members. As little competitors kil|3.30|2.23|3001002|amalgexporti #2|13|newborn|3|Children|227|ationableable|medium|54639blanched1242985|gainsboro|Pallet|Unknown|13|antiantioughtought| +1156|AAAAAAAAEIEAAAAA|1997-10-27|1999-10-27|Votes reach then public words; children pick. Too individual |0.98|0.73|6015005|scholarbrand #5|15|custom|6|Jewelry|343|priesepri|N/A|169saddle12786388749|deep|Pound|Unknown|57|callyantioughtought| +1157|AAAAAAAAEIEAAAAA|1999-10-28|2001-10-26|Broad writers ought to claim acceptable rooms. Badly unlikely costs take for a|3.88|0.73|7003008|exportibrand #8|15|kids|7|Home|497|priesepri|N/A|75246376lavender2495|sky|Oz|Unknown|19|ationantioughtought| +1158|AAAAAAAAEIEAAAAA|2001-10-27||Intervals see very also regional months. Therefore great values correct politically thinking periods. Front goods raise in a days. Real reactions would not expect hardly unknown departments. White|0.12|0.04|7003008|univmaxi #9|15|pools|8|Sports|28|priesepri|N/A|732lavender927679752|snow|Oz|Unknown|9|eingantioughtought| +1159|AAAAAAAAHIEAAAAA|1997-10-27||Classes share names; still|5.17|4.13|1001002|amalgamalg #2|1|dresses|1|Women|70|baration|petite|84571wheat8603932941|pink|Dozen|Unknown|31|n stantioughtought| +1160|AAAAAAAAIIEAAAAA|1997-10-27|2000-10-26|Publications could not judge; double deputies|6.30|2.39|1004001|edu packamalg #1|4|swimwear|1|Women|341|oughtesepri|small|14423720923navy38717|peru|Ounce|Unknown|30|barcallyoughtought| +1161|AAAAAAAAIIEAAAAA|2000-10-27||Significant, wrong patients marry rather in a accounts. Only, popular organs stand c|1.85|0.99|1004001|corpbrand #2|4|rugs|7|Home|341|oughtesepri|N/A|9tan3616178916219494|salmon|Bunch|Unknown|99|oughtcallyoughtought| +1162|AAAAAAAAKIEAAAAA|1997-10-27|1999-10-27|Women effect once in a bars. Good chips buy fast |9.52|5.90|4002001|importoedu pack #1|2|mens|4|Shoes|452|ableantiese|petite|9038royal37972544503|wheat|Oz|Unknown|17|ablecallyoughtought| +1163|AAAAAAAAKIEAAAAA|1999-10-28|2001-10-26|Essential, honest years pick english reasons. Ago personal bags solve often changes. Likely details ought to happen indee|7.50|3.75|8007004|brandnameless #4|7|hockey|8|Sports|143|ableantiese|N/A|4456029769red0432612|lawn|Carton|Unknown|8|pricallyoughtought| +1164|AAAAAAAAKIEAAAAA|2001-10-27||Essential, honest years pick english reasons. Ago personal bags solve often changes. Likely details ought to happen indee|0.38|3.75|3002001|importoexporti #1|2|infants|3|Children|143|ableantiese|petite|4456029769red0432612|lime|Pallet|Unknown|18|esecallyoughtought| +1165|AAAAAAAANIEAAAAA|1997-10-27||Very financial engineers believe at the teeth. Short, current influences will not carry main, appropriate eyes. Well likely books used to read. New sites might la|6.71|2.54|4002002|importoedu pack #2|2|mens|4|Shoes|112|ableoughtought|extra large|727435445207saddle50|tan|Ounce|Unknown|4|anticallyoughtought| +1166|AAAAAAAAOIEAAAAA|1997-10-27|2000-10-26|Occasional, main sites result often to the figures. Again diplomatic years could place about often large experts. Spanish parties se|3.38|2.77|2002001|importoimporto #1|2|shirts|2|Men|474|eseationese|petite|197012chocolate01732|metallic|Pound|Unknown|42|callycallyoughtought| +1167|AAAAAAAAOIEAAAAA|2000-10-27||Natural, wooden doors read already votes. Units must follow even unemployed, visible children; religious eyes retain more no|3.62|2.77|2002001|edu packbrand #8|2|curtains/drapes|7|Home|13|priought|N/A|292372618700lime9227|plum|Cup|Unknown|14|ationcallyoughtought| +1168|AAAAAAAAAJEAAAAA|1997-10-27|1999-10-27|Often open hours wait about; ever safe repor|8.66|4.93|2002001|importoimporto #1|2|shirts|2|Men|609|n stbarcally|medium|281243turquoise48753|turquoise|Tbl|Unknown|35|eingcallyoughtought| +1169|AAAAAAAAAJEAAAAA|1999-10-28|2001-10-26|Military shareholders refine consciously. New resources understand better bottom, local systems. Early, compulsory decisions will not produce wonderful, added|3.96|4.93|2002001|univbrand #2|2|jewelry boxes|6|Jewelry|609|n stbarcally|N/A|281243turquoise48753|sandy|Ounce|Unknown|23|n stcallyoughtought| +1170|AAAAAAAAAJEAAAAA|2001-10-27||Away mean rivers proceed also to no hours. Tiny terms might ch|3.79|4.93|10003015|exportiunivamalg #15|2|dvd/vcr players|10|Electronics|609|n stbarcally|N/A|281243turquoise48753|pale|Gram|Unknown|2|barationoughtought| +1171|AAAAAAAADJEAAAAA|1997-10-27||Specific, temporary goals take. Ideas might reduce economic authorities. Fundamentally external prayers matter really|84.79|32.22|9010008|univunivamalg #8|10|travel|9|Books|284|eseeingable|N/A|7032285903246khaki73|moccasin|Unknown|Unknown|13|oughtationoughtought| +1172|AAAAAAAAEJEAAAAA|1997-10-27|2000-10-26|Necessary years achieve only vegetables. Customs ask old, ruling poles. Easy arguments stop technical, industrial problems; steps slip much goals. So regular types dwell before also normal men. Due|6.77|5.21|3004001|edu packexporti #1|4|school-uniforms|3|Children|180|bareingought|medium|577195352puff3662593|wheat|Unknown|Unknown|30|ableationoughtought| +1173|AAAAAAAAEJEAAAAA|2000-10-27||Necessary years achieve only vegetables. Customs ask old, ruling poles. Easy arguments stop technical, industrial problems; steps slip much goals. So regular types dwell before also normal men. Due|4.21|5.21|3004001|exportiunivamalg #1|3|dvd/vcr players|10|Electronics|806|bareingought|N/A|20smoke5000277981489|rose|Tsp|Unknown|13|priationoughtought| +1174|AAAAAAAAGJEAAAAA|1997-10-27|1999-10-27|Actually strong reports |2.76|2.20|5003001|exportischolar #1|3|pop|5|Music|295|antin stable|N/A|88158156230089slate0|slate|Ounce|Unknown|22|eseationoughtought| +1175|AAAAAAAAGJEAAAAA|1999-10-28|2001-10-26|Concerned, cheerful candidates precede for the sales. Full, proposed costs give far moments. Sides used to look concerned classes. Parts ask ju|4.38|1.53|5003001|exportiamalg #2|3|maternity|1|Women|17|ationought|extra large|88158156230089slate0|pale|Pound|Unknown|12|antiationoughtought| +1176|AAAAAAAAGJEAAAAA|2001-10-27||Concerned, cheerful candidates precede for the sales. Full, proposed costs give far moments. Sides used to look concerned classes. Parts ask ju|1.50|1.53|5003001|corpbrand #1|16|consignment|6|Jewelry|215|antioughtable|N/A|9089484522555ivory42|khaki|Gram|Unknown|50|callyationoughtought| +1177|AAAAAAAAJJEAAAAA|1997-10-27||Only, particular claims go as recent universities. Sessions would intimidate particular, identical figures. Levels ought to want better senior functions. Cons|0.83|0.46|2002002|importoimporto #2|2|shirts|2|Men|151|oughtantiought|large|78267458056hot589754|spring|Tsp|Unknown|34|ationationoughtought| +1178|AAAAAAAAKJEAAAAA|1997-10-27|2000-10-26|Clearly conservative children could not moderate with a decisions. As good as important track|7.66|3.21|8016009|corpmaxi #9|16|golf|8|Sports|734|esepriation|N/A|745purple34801743823|tan|Tsp|Unknown|8|eingationoughtought| +1179|AAAAAAAAKJEAAAAA|2000-10-27||Clearly conservative children could not moderate with a decisions. As good as important track|6.99|3.21|1001002|amalgamalg #2|1|dresses|1|Women|734|esepriation|large|745purple34801743823|linen|Gross|Unknown|58|n stationoughtought| +1180|AAAAAAAAMJEAAAAA|1997-10-27|1999-10-27|Local pro|1.04|0.70|9003005|exportimaxi #5|3|computers|9|Books|756|callyantiation|N/A|117197531290sky04358|azure|Bundle|Unknown|4|bareingoughtought| +1181|AAAAAAAAMJEAAAAA|1999-10-28|2001-10-26|Easily scientific details listen much scottish books; women think completely so-called banks. Other members afford at a months. All young |1.37|0.70|9003005|corpmaxi #2|16|golf|8|Sports|461|callyantiation|N/A|117197531290sky04358|gainsboro|Dram|Unknown|20|oughteingoughtought| +1182|AAAAAAAAMJEAAAAA|2001-10-27||Years may dismiss gravely female mechanisms. Savings should make. High, small boots must contact tonight physical, important difficulties; previous words overcome in addition genes. Estates see person|5.68|0.70|5002001|importoscholar #1|2|country|5|Music|249|callyantiation|N/A|117197531290sky04358|puff|Gross|Unknown|23|ableeingoughtought| +1183|AAAAAAAAPJEAAAAA|1997-10-27||New prisons used to come sexual|2.39|1.81|3003002|exportiexporti #2|3|toddlers|3|Children|203|pribarable|small|58365wheat8287489521|puff|Dozen|Unknown|3|prieingoughtought| +1184|AAAAAAAAAKEAAAAA|1997-10-27|2000-10-26|Eggs assist light years. Components might give later broad mothers; bloo|5.70|4.04|10003012|exportiunivamalg #12|3|dvd/vcr players|10|Electronics|25|antiable|N/A|85369299484941grey94|rose|Ounce|Unknown|45|eseeingoughtought| +1185|AAAAAAAAAKEAAAAA|2000-10-27||Eggs assist light years. Components might give later broad mothers; bloo|3.95|3.31|10003012|edu packcorp #2|4|bracelets|6|Jewelry|95|antin st|N/A|0674109rosy405416099|purple|Each|Unknown|24|antieingoughtought| +1186|AAAAAAAACKEAAAAA|1997-10-27|1999-10-27|Legs appear eventually soci|35.27|22.92|9001011|amalgmaxi #11|1|arts|9|Books|612|ableoughtcally|N/A|83powder156418086730|dark|Case|Unknown|30|callyeingoughtought| +1187|AAAAAAAACKEAAAAA|1999-10-28|2001-10-26|Important, hard stories should get british, inner sentences. Bloody|4.90|3.33|9001011|scholarcorp #4|1|earings|6|Jewelry|612|ableoughtcally|N/A|8990633smoke70356719|ivory|N/A|Unknown|54|ationeingoughtought| +1188|AAAAAAAACKEAAAAA|2001-10-27||Important, hard stories should get british, inner sentences. Bloody|0.81|3.33|9001011|importoedu pack #1|1|mens|4|Shoes|612|ableoughtcally|petite|79steel8588660820579|powder|Pound|Unknown|12|eingeingoughtought| +1189|AAAAAAAAFKEAAAAA|1997-10-27||Children may close necessary dates. Short opportunities improve. Han|1.25|0.43|5002002|importoscholar #2|2|country|5|Music|298|eingn stable|N/A|992484violet37743948|steel|Ton|Unknown|68|n steingoughtought| +1190|AAAAAAAAGKEAAAAA|1997-10-27|2000-10-26|Farmers stay most in a police. Possible, expensive years prevail songs. Profits sustain other, prime parents. Further economic costs might not negotiate more acute tiny months. P|3.98|3.34|10009001|maxiunivamalg #1|9|televisions|10|Electronics|325|antiablepri|N/A|422tan53592822157455|snow|N/A|Unknown|19|barn stoughtought| +1191|AAAAAAAAGKEAAAAA|2000-10-27||General adults go new heads; schemes would represent new forces. Usua|20.69|3.34|6009008|maxicorp #8|9|womens watch|6|Jewelry|251|oughtantiable|N/A|361tan76640076560635|sky|Tsp|Unknown|17|oughtn stoughtought| +1192|AAAAAAAAIKEAAAAA|1997-10-27|1999-10-27|All |9.44|6.70|9002005|importomaxi #5|2|business|9|Books|507|ationbaranti|N/A|1739191rosy316697193|pale|Ton|Unknown|27|ablen stoughtought| +1193|AAAAAAAAIKEAAAAA|1999-10-28|2001-10-26|All |62.48|6.70|2001002|amalgimporto #2|2|accessories|2|Men|507|ationbaranti|medium|1739191rosy316697193|smoke|Tsp|Unknown|22|prin stoughtought| +1194|AAAAAAAAIKEAAAAA|2001-10-27||All |8.74|6.70|9004007|edu packmaxi #7|4|entertainments|9|Books|507|ationbaranti|N/A|1708549269royal69656|thistle|Case|Unknown|28|esen stoughtought| +1195|AAAAAAAALKEAAAAA|1997-10-27||Previous, large police can encourage much able, basic details. Elderly, small years would end more institutions. Cities could go perhaps soon large words. Feelings should apply sports. Initial na|2.49|1.61|3003002|exportiexporti #2|3|toddlers|3|Children|751|oughtantiation|medium|1493769865orchid9995|thistle|Unknown|Unknown|22|antin stoughtought| +1196|AAAAAAAAMKEAAAAA|1997-10-27|2000-10-26|Certain, military duties might work maybe exports. Here simple decisions will imagine also sufficient beds. Advances withdraw still more cold|9.80|4.60|1002001|importoamalg #1|2|fragrances|1|Women|100|barbarought|small|72923papaya848140183|rosy|Pound|Unknown|52|callyn stoughtought| +1197|AAAAAAAAMKEAAAAA|2000-10-27||Holy clouds learn weekends. Short, leading judges describe ideas. Months explain past children. Important signs should not take; economic incenti|9.06|5.97|9006010|corpmaxi #10|6|parenting|9|Books|100|barbarought|N/A|21grey76008785337118|wheat|Ton|Unknown|51|ationn stoughtought| +1198|AAAAAAAAOKEAAAAA|1997-10-27|1999-10-27|Roman lines talk children. Parties account exactly toward|4.28|2.99|8004009|edu packnameless #9|4|camping|8|Sports|292|ablen stable|N/A|1603476569175olive19|orange|Each|Unknown|41|eingn stoughtought| +1199|AAAAAAAAOKEAAAAA|1999-10-28|2001-10-26|Then available facilities should not want helpful architects. Merely right communications become. Ethnic, christian women could not pay. False panels test. Appropriate, available dishes |4.09|2.99|8004009|edu packscholar #2|4|classical|5|Music|90|ablen stable|N/A|1603476569175olive19|seashell|Case|Unknown|42|n stn stoughtought| +1200|AAAAAAAAOKEAAAAA|2001-10-27||Then available facilities should not want helpful architects. Merely right communications become. Ethnic, christian women could not pay. False panels test. Appropriate, available dishes |4.53|2.85|3001001|amalgexporti #1|4|newborn|3|Children|613|prioughtcally|extra large|1603476569175olive19|white|Pound|Unknown|25|barbarableought| +1201|AAAAAAAABLEAAAAA|1997-10-27||Rough, great men should tell again isolated plants. Firms ought to describe formerly children. Times mig|2.46|1.96|2002002|importoimporto #2|2|shirts|2|Men|429|n stableese|medium|528powder53005156530|pale|Bunch|Unknown|9|oughtbarableought| +1202|AAAAAAAACLEAAAAA|1997-10-27|2000-10-26|New requirements can implement at all fingers. Firm odds distribute countries; problems can fall in a cards. Public banks ought to close brave children|8.09|5.17|2004001|edu packimporto #1|4|sports-apparel|2|Men|206|callybarable|small|cornsilk939672966647|peru|Dram|Unknown|58|ablebarableought| +1203|AAAAAAAACLEAAAAA|2000-10-27||New requirements can implement at all fingers. Firm odds distribute countries; problems can fall in a cards. Public banks ought to close brave children|59.02|20.06|2004001|exportinameless #4|13|wallpaper|7|Home|14|callybarable|N/A|51536812turquoise548|salmon|Each|Unknown|100|pribarableought| +1204|AAAAAAAAELEAAAAA|1997-10-27|1999-10-27|Horizontal buyers tell about new cattle. In orde|0.23|0.19|4002001|importoedu pack #1|2|mens|4|Shoes|173|priationought|extra large|53781017448wheat8264|indian|Unknown|Unknown|14|esebarableought| +1205|AAAAAAAAELEAAAAA|1999-10-28|2001-10-26|Here statutory birds add metho|4.40|0.19|1001002|amalgamalg #2|1|dresses|1|Women|411|priationought|N/A|53781017448wheat8264|puff|Oz|Unknown|3|antibarableought| +1206|AAAAAAAAELEAAAAA|2001-10-27||Here statutory birds add metho|2.58|0.19|1001002|brandnameless #7|7|hockey|8|Sports|411|priationought|N/A|53781017448wheat8264|powder|Carton|Unknown|24|callybarableought| +1207|AAAAAAAAHLEAAAAA|1997-10-27||Serious, social teams could not take also other, blind designers. Clear groups would find ot|7.00|4.48|8007008|brandnameless #8|7|hockey|8|Sports|193|prin stought|N/A|52drab01036015465557|tomato|Tbl|Unknown|18|ationbarableought| +1208|AAAAAAAAILEAAAAA|1997-10-27|2000-10-26|Considerably similar rules avoid more; cases get against the situations. Beds must like large, limited approaches. Less unable groups could say. Speedily fiscal concerns pay too talks. Long nee|0.76|0.67|7012009|importonameless #9|12|paint|7|Home|269|n stcallyable|N/A|322pink0559142651794|green|Bunch|Unknown|7|eingbarableought| +1209|AAAAAAAAILEAAAAA|2000-10-27||Considerably similar rules avoid more; cases get against the situations. Beds must like large, limited approaches. Less unable groups could say. Speedily fiscal concerns pay too talks. Long nee|0.38|0.13|7012009|exportiamalg #2|12|maternity|1|Women|269|n stcallyable|large|322pink0559142651794|brown|Cup|Unknown|23|n stbarableought| +1210|AAAAAAAAKLEAAAAA|1997-10-27|1999-10-27|So political disciplines live so pure movements. New, fair lights used to expect much civil res|3.05|1.92|1003001|exportiamalg #1|3|maternity|1|Women|88|eingeing|small|14323smoke9387027134|puff|Gross|Unknown|2|baroughtableought| +1211|AAAAAAAAKLEAAAAA|1999-10-28|2001-10-26|Local clubs should not include raw shops. Nevertheless main acts might not come already abstract, different rounds. Material, complete|3.97|1.92|1003001|edu packmaxi #12|4|entertainments|9|Books|600|eingeing|N/A|14323smoke9387027134|dodger|Ounce|Unknown|43|oughtoughtableought| +1212|AAAAAAAAKLEAAAAA|2001-10-27||Teachers define personal movies. Careful, difficult surroundings might get ago high vehicles. Too social hospitals give primarily content, burning months. Par|9.49|6.73|1004001|edu packamalg #1|4|swimwear|1|Women|600|barbarcally|petite|14323smoke9387027134|rosy|N/A|Unknown|81|ableoughtableought| +1213|AAAAAAAANLEAAAAA|1997-10-27||Also new lives fly actually. Goals ought to allow soon. Old regulations should not refle|7.56|6.35|3001002|amalgexporti #2|1|newborn|3|Children|448|eingeseese|medium|43389rose41430544815|powder|Oz|Unknown|8|prioughtableought| +1214|AAAAAAAAOLEAAAAA|1997-10-27|2000-10-26|Physical accidents shut. Very sure schools get directly wide, proud budgets. Old children cannot pri|3.37|2.39|10015008|scholaramalgamalg #8|15|portable|10|Electronics|113|prioughtought|N/A|41787peach2595881534|sky|Tbl|Unknown|5|eseoughtableought| +1215|AAAAAAAAOLEAAAAA|2000-10-27||Physical accidents shut. Very sure schools get directly wide, proud budgets. Old children cannot pri|9.68|2.39|2004002|edu packimporto #2|4|sports-apparel|2|Men|264|esecallyable|economy|1110182322lace541115|lawn|Oz|Unknown|14|antioughtableought| +1216|AAAAAAAAAMEAAAAA|1997-10-27|1999-10-27|Very, true women eat. Left institutions may agree towards the kids; national, other terms open there then different prices; others settle however. Apparently normal |9.64|6.26|9005005|scholarmaxi #5|5|history|9|Books|418|eingoughtese|N/A|violet98445335625871|linen|Bunch|Unknown|64|callyoughtableought| +1217|AAAAAAAAAMEAAAAA|1999-10-28|2001-10-26|Very, true women eat. Left institutions may agree towards the kids; national, other terms open there then different prices; others settle however. Apparently normal |9.42|7.91|9005005|amalgscholar #2|1|rock|5|Music|656|callyantically|N/A|violet98445335625871|tan|Pallet|Unknown|41|ationoughtableought| +1218|AAAAAAAAAMEAAAAA|2001-10-27||Popular rates take likely, legal details; again different videos must know to a reserves. Social, evolutionary liabilities used to ask. Fine women might suit girls; ho|1.56|7.91|9005005|edu packbrand #3|1|estate|6|Jewelry|656|callyantically|N/A|violet98445335625871|sandy|Unknown|Unknown|23|eingoughtableought| +1219|AAAAAAAADMEAAAAA|1997-10-27||Groups decide therefore charges. Inc, easy months would give measures; jobs go with the signs. Broa|5.40|2.21|8012010|importomaxi #10|12|guns|8|Sports|3|pri|N/A|9151691410indian0795|cornflower|Gram|Unknown|46|n stoughtableought| +1220|AAAAAAAAEMEAAAAA|1997-10-27|2000-10-26|Technical, successful orders make. Past eyes shall love for a tasks. Tiny cities ought to catch. Unpleasant cells should write either female shares; frequently useful effects ought to implement |1.14|0.50|2003001|exportiimporto #1|3|pants|2|Men|380|bareingpri|small|7846840623green95600|slate|Cup|Unknown|31|barableableought| +1221|AAAAAAAAEMEAAAAA|2000-10-27||Concrete, long farmers hide alone worth the policies; slightly american officers must not give also most big businesses. Capitalist commentators used to save also children. Sounds ought to|2.72|0.50|2003001|exportiamalgamalg #15|3|stereo|10|Electronics|84|bareingpri|N/A|94wheat6987427391281|puff|Ton|Unknown|35|oughtableableought| +1222|AAAAAAAAGMEAAAAA|1997-10-27|1999-10-27|Recent neighbours ought to bring short rough wee|4.71|2.92|1001001|amalgamalg #1|1|dresses|1|Women|102|ablebarought|small|41237680pink04929568|powder|N/A|Unknown|61|ableableableought| +1223|AAAAAAAAGMEAAAAA|1999-10-28|2001-10-26|Styles may happen only so|4.05|1.66|1002002|importoamalg #2|1|fragrances|1|Women|102|ablebarought|medium|79278seashell3785323|indian|Bunch|Unknown|43|priableableought| +1224|AAAAAAAAGMEAAAAA|2001-10-27||Styles may happen only so|2.22|0.66|5003001|exportischolar #1|1|pop|5|Music|269|n stcallyable|N/A|52984951649pink12460|sky|Gross|Unknown|20|eseableableought| +1225|AAAAAAAAJMEAAAAA|1997-10-27||Nearby, only championships mix private costs. Ethical, old centuries abide |2.27|1.08|10012002|importoamalgamalg #2|12|monitors|10|Electronics|24|eseable|N/A|19261pink03383406886|sandy|Box|Unknown|55|antiableableought| +1226|AAAAAAAAKMEAAAAA|1997-10-27|2000-10-26|Rapidly chemical processes could imagine ironically true, large features. Groups will listen almost long-t|5.63|2.47|6012003|importobrand #3|12|costume|6|Jewelry|378|eingationpri|N/A|529769145210hot19776|yellow|N/A|Unknown|59|callyableableought| +1227|AAAAAAAAKMEAAAAA|2000-10-27||Rapidly chemical processes could imagine ironically true, large features. Groups will listen almost long-t|4.98|2.14|6012003|edu packexporti #2|4|school-uniforms|3|Children|378|eingationpri|medium|529769145210hot19776|medium|Ton|Unknown|59|ationableableought| +1228|AAAAAAAAMMEAAAAA|1997-10-27|1999-10-27|European, pleased shows reach alone patients. Sooner prime cases should raise duties. Reliable styles must clear fe|5.49|2.30|4004001|edu packedu pack #1|4|athletic|4|Shoes|641|oughtesecally|medium|1687turquoise2837062|ghost|Tbl|Unknown|43|eingableableought| +1229|AAAAAAAAMMEAAAAA|1999-10-28|2001-10-26|Aware players operate new, tough times.|9.57|6.60|5003002|exportischolar #2|4|pop|5|Music|214|oughtesecally|N/A|1687turquoise2837062|turquoise|Pallet|Unknown|28|n stableableought| +1230|AAAAAAAAMMEAAAAA|2001-10-27||Aware players operate new, tough times.|2.57|2.13|5003002|maxinameless #1|9|optics|8|Sports|214|oughtesecally|N/A|1687turquoise2837062|cornsilk|Case|Unknown|64|barpriableought| +1231|AAAAAAAAPMEAAAAA|1997-10-27||Too personal forces might not play really by a fees. Effective others|8.75|3.58|10014008|edu packamalgamalg #8|14|automotive|10|Electronics|377|ationationpri|N/A|006snow7930705676236|lime|Carton|Unknown|66|oughtpriableought| +1232|AAAAAAAAANEAAAAA|1997-10-27|2000-10-26|Temperatures reflect quite|0.90|0.45|8005003|scholarnameless #3|5|fitness|8|Sports|159|n stantiought|N/A|519490seashell847101|navy|Tbl|Unknown|7|ablepriableought| +1233|AAAAAAAAANEAAAAA|2000-10-27||Temperatures reflect quite|2.00|0.45|1001002|amalgamalg #2|5|dresses|1|Women|159|n stantiought|petite|519490seashell847101|lime|Dozen|Unknown|72|pripriableought| +1234|AAAAAAAACNEAAAAA|1997-10-27|1999-10-27|Horses last results. There thorough parents sail everywhere into a gua|3.45|2.55|8005005|scholarnameless #5|5|fitness|8|Sports|323|priablepri|N/A|0522lavender16368102|white|Pallet|Unknown|26|esepriableought| +1235|AAAAAAAACNEAAAAA|1999-10-28|2001-10-26|Horses last results. There thorough parents sail everywhere into a gua|6.48|4.66|8005005|exportischolar #2|5|pop|5|Music|188|priablepri|N/A|0522lavender16368102|honeydew|Tbl|Unknown|24|antipriableought| +1236|AAAAAAAACNEAAAAA|2001-10-27||Horses last results. There thorough parents sail everywhere into a gua|34.11|4.66|8005005|amalgimporto #1|5|accessories|2|Men|784|eseeingation|medium|73796709pink23330372|royal|Case|Unknown|50|callypriableought| +1237|AAAAAAAAFNEAAAAA|1997-10-27||Short, political campaigns establish much objectives; so main lights used to reach then. New options oust criminal proposals; external advisers share then char|3.78|2.23|10010014|univamalgamalg #14|10|memory|10|Electronics|227|ationableable|N/A|6773650pale371208580|pale|Case|Unknown|27|ationpriableought| +1238|AAAAAAAAGNEAAAAA|1997-10-27|2000-10-26|Parents induce from a views; informal amounts say jobs. Entries tell. Little, shallow profits cannot see easily chief clothes. Too commercial politicians might not |8.60|5.41|10005015|scholarunivamalg #15|5|karoke|10|Electronics|503|pribaranti|N/A|701papaya25978334971|peach|Lb|Unknown|56|eingpriableought| +1239|AAAAAAAAGNEAAAAA|2000-10-27||Answers confirm great women. English animals can prolong utterly new areas. Open technologies may know. Buildings appreciate to a tests. Unable studies shall mind. Howeve|9.61|5.41|10005015|edu packexporti #2|4|school-uniforms|3|Children|503|pribaranti|large|701papaya25978334971|spring|Gross|Unknown|15|n stpriableought| +1240|AAAAAAAAINEAAAAA|1997-10-27|1999-10-27|Perhaps busy institutions can appear now future, tall times. Secondary, warm problems would stimulate more |3.09|0.92|8009001|maxinameless #1|9|optics|8|Sports|822|ableableeing|N/A|98163906mint89269070|tan|Gram|Unknown|75|bareseableought| +1241|AAAAAAAAINEAAAAA|1999-10-28|2001-10-26|Perhaps busy institutions can appear now future, tall times. Secondary, warm problems would stimulate more |9.41|0.92|8009001|maxicorp #2|9|womens watch|6|Jewelry|822|ableableeing|N/A|98163906mint89269070|powder|Gross|Unknown|61|oughteseableought| +1242|AAAAAAAAINEAAAAA|2001-10-27||Perhaps busy institutions can appear now future, tall times. Secondary, warm problems would stimulate more |2.68|1.23|1002001|importoamalg #1|9|fragrances|1|Women|357|ableableeing|economy|78574621peach3677689|powder|Ton|Unknown|31|ableeseableought| +1243|AAAAAAAALNEAAAAA|1997-10-27||Much |92.12|58.95|10016003|corpamalgamalg #3|16|wireless|10|Electronics|369|n stcallypri|N/A|37rose59318811699434|sienna|Carton|Unknown|57|prieseableought| +1244|AAAAAAAAMNEAAAAA|1997-10-27|2000-10-26|Open special customs should need even daily regulations. Always enormous clients retire there important police. Military, complete po|7.05|4.23|3001001|amalgexporti #1|1|newborn|3|Children|128|eingableought|medium|22606648rosy36717992|moccasin|Carton|Unknown|35|eseeseableought| +1245|AAAAAAAAMNEAAAAA|2000-10-27||Always only customs call too to a passages. Secondary,|0.59|0.38|3001001|importoscholar #2|1|country|5|Music|128|eingableought|N/A|22606648rosy36717992|orange|Carton|Unknown|44|antieseableought| +1246|AAAAAAAAONEAAAAA|1997-10-27|1999-10-27|Social, young days guide presumably. Somehow old servants return so|2.18|1.94|7016007|corpnameless #7|16|furniture|7|Home|510|baroughtanti|N/A|98897816599lawn18743|pale|Cup|Unknown|24|callyeseableought| +1247|AAAAAAAAONEAAAAA|1999-10-28|2001-10-26|Social, young days guide presumably. Somehow old servants return so|17.32|9.69|5001002|amalgscholar #2|1|rock|5|Music|154|baroughtanti|N/A|98897816599lawn18743|salmon|Dozen|Unknown|99|ationeseableought| +1248|AAAAAAAAONEAAAAA|2001-10-27||Clear conditions will not produce schools. Everyday, educational institutions will love then individual times. Standard, possible times meet even so high profit|4.68|2.71|10009008|maxiunivamalg #8|1|televisions|10|Electronics|154|baroughtanti|N/A|98897816599lawn18743|green|Dozen|Unknown|75|eingeseableought| +1249|AAAAAAAABOEAAAAA|1997-10-27||Black areas suggest central groups. Only children would release then bad, supreme ideas. Final ears bear always |2.24|1.18|4004002|edu packedu pack #2|4|athletic|4|Shoes|98|eingn st|petite|medium07672942563395|powder|Each|Unknown|57|n steseableought| +1250|AAAAAAAACOEAAAAA|1997-10-27|2000-10-26|High, high allowances change gradually arts. Possible clients grant properly ways. Children should avert as in an machines. Relevant offences continu|9.70|6.20|2003001|exportiimporto #1|3|pants|2|Men|77|ationation|extra large|067418701106839plum6|smoke|Cup|Unknown|22|barantiableought| +1251|AAAAAAAACOEAAAAA|2000-10-27||High, high allowances change gradually arts. Possible clients grant properly ways. Children should avert as in an machines. Relevant offences continu|7.60|2.35|8005002|scholarnameless #2|5|fitness|8|Sports|77|ationation|N/A|94514682653spring410|orchid|Gross|Unknown|30|oughtantiableought| +1252|AAAAAAAAEOEAAAAA|1997-10-27|1999-10-27|Other phrases cannot invest only previously small views. As following pupils give of course selective citizens. Content, burning positions take through a factors|5.18|4.24|1003001|exportiamalg #1|3|maternity|1|Women|58|einganti|medium|8859018554095puff400|purple|Bunch|Unknown|88|ableantiableought| +1253|AAAAAAAAEOEAAAAA|1999-10-28|2001-10-26|Political angles kick regularly protective, british contributions. Democrats must not know only in a prop|2.39|4.24|1003002|exportiamalg #2|3|maternity|1|Women|308|eingbarpri|medium|8859018554095puff400|peru|Lb|Unknown|33|priantiableought| +1254|AAAAAAAAEOEAAAAA|2001-10-27||Images doubt sooner. Old employees would not say. Leaders cause later unusual, inner righ|91.27|4.24|1003002|scholarnameless #5|15|tables|7|Home|308|eingbarpri|N/A|904036powder12212901|royal|Pallet|Unknown|43|eseantiableought| +1255|AAAAAAAAHOEAAAAA|1997-10-27||Main, aware rights will not escape under the systems. Circumstances must introduce just as a children. Publ|1.46|0.90|7008006|namelessbrand #6|8|lighting|7|Home|15|antiought|N/A|039059445royal372444|royal|Case|Unknown|48|antiantiableought| +1256|AAAAAAAAIOEAAAAA|1997-10-27|2000-10-26|However modern companies ought to make industria|9.56|3.25|6007001|brandcorp #1|7|pendants|6|Jewelry|265|anticallyable|N/A|885476663snow3325425|saddle|Bunch|Unknown|2|callyantiableought| +1257|AAAAAAAAIOEAAAAA|2000-10-27||Front prisons cannot take successful women. Historians use additional, aware years; naturally local children |9.56|4.11|6014002|edu packbrand #2|14|estate|6|Jewelry|265|anticallyable|N/A|885476663snow3325425|lavender|Gram|Unknown|35|ationantiableought| +1258|AAAAAAAAKOEAAAAA|1997-10-27|1999-10-27|Important blues rise steady major ideas. Years must organize long books. More contemporary girls get for instance under a things. Proper experiments might carry so friendly, cold cards; da|0.46|0.20|10007001|brandunivamalg #1|7|personal|10|Electronics|68|eingcally|N/A|39124purple033145323|puff|Ounce|Unknown|23|eingantiableought| +1259|AAAAAAAAKOEAAAAA|1999-10-28|2001-10-26|Important blues rise steady major ideas. Years must organize long books. More contemporary girls get for instance under a things. Proper experiments might carry so friendly, cold cards; da|3.68|0.20|10007001|exportiamalg #2|7|maternity|1|Women|68|eingcally|large|39124purple033145323|turquoise|Ton|Unknown|83|n stantiableought| +1260|AAAAAAAAKOEAAAAA|2001-10-27||Important blues rise steady major ideas. Years must organize long books. More contemporary girls get for instance under a things. Proper experiments might carry so friendly, cold cards; da|6.34|0.20|10007001|scholarbrand #5|15|custom|6|Jewelry|68|eingcally|N/A|2yellow0123618831250|dodger|Unknown|Unknown|84|barcallyableought| +1261|AAAAAAAANOEAAAAA|1997-10-27||Mean, recent sequences throw separate, other eyes. Sudden, cold roots take just general relations. Advantages could meet. Then annual page|4.83|4.34|8006002|corpnameless #2|6|football|8|Sports|167|ationcallyought|N/A|111622381spring15528|sky|Pallet|Unknown|5|oughtcallyableought| +1262|AAAAAAAAOOEAAAAA|1997-10-27|2000-10-26|Companies reveal national reforms; kinds initiate in a languages. Positive miles ought to hesitate thick priorities. Large, cons|1.45|0.97|9004003|edu packmaxi #3|4|entertainments|9|Books|557|ationantianti|N/A|3aquamarine117015297|thistle|Tsp|Unknown|94|ablecallyableought| +1263|AAAAAAAAOOEAAAAA|2000-10-27||Companies reveal national reforms; kinds initiate in a languages. Positive miles ought to hesitate thick priorities. Large, cons|2.41|1.37|9004003|amalgexporti #2|4|newborn|3|Children|557|ationantianti|medium|364270352plum2579528|royal|Case|Unknown|18|pricallyableought| +1264|AAAAAAAAAPEAAAAA|1997-10-27|1999-10-27|Men would find above awards. Really true homes spend since cautious points. Essenti|0.57|0.17|8002005|importonameless #5|2|baseball|8|Sports|107|ationbarought|N/A|0343maroon0890545761|red|Case|Unknown|9|esecallyableought| +1265|AAAAAAAAAPEAAAAA|1999-10-28|2001-10-26|Men would find above awards. Really true homes spend since cautious points. Essenti|9.60|0.17|10016008|corpamalgamalg #8|2|wireless|10|Electronics|107|ationbarought|N/A|932364372094282rosy9|medium|Bunch|Unknown|14|anticallyableought| +1266|AAAAAAAAAPEAAAAA|2001-10-27||As minute illustrations go long into a levels. Families could not disagree fully please normal farmers. New pensioners permit primitive, modern pictures. Exercises ought to ask|6.05|5.32|10016008|edu packexporti #1|2|school-uniforms|3|Children|52|ableanti|medium|8477723158503papaya6|saddle|Lb|Unknown|13|callycallyableought| +1267|AAAAAAAADPEAAAAA|1997-10-27||Catholic years st|0.45|0.19|9009008|maximaxi #8|9|science|9|Books|305|antibarpri|N/A|31spring483871682489|peach|Box|Unknown|25|ationcallyableought| +1268|AAAAAAAAEPEAAAAA|1997-10-27|2000-10-26|Experiments would fall in a models. Sales cannot|0.44|0.16|1001001|amalgamalg #1|1|dresses|1|Women|585|antieinganti|small|6499116339lawn350386|sky|Lb|Unknown|44|eingcallyableought| +1269|AAAAAAAAEPEAAAAA|2000-10-27||Experiments would fall in a models. Sales cannot|7.09|3.11|5001002|amalgscholar #2|1|rock|5|Music|585|antieinganti|N/A|364548875tomato17892|violet|Unknown|Unknown|5|n stcallyableought| +1270|AAAAAAAAGPEAAAAA|1997-10-27|1999-10-27|Parts race joint teachers. Strong, potential group|7.17|4.23|3003001|exportiexporti #1|3|toddlers|3|Children|178|eingationought|medium|71745aquamarine12327|lavender|Dozen|Unknown|21|barationableought| +1271|AAAAAAAAGPEAAAAA|1999-10-28|2001-10-26|Parts race joint teachers. Strong, potential group|2.68|2.03|2002002|importoimporto #2|3|shirts|2|Men|777|eingationought|extra large|72169373powder890094|lavender|N/A|Unknown|38|oughtationableought| +1272|AAAAAAAAGPEAAAAA|2001-10-27||Teachers shall keep clear, delicate attacks. Enough potential mountains would go further chosen strong c|2.28|0.68|2002002|edu packnameless #3|14|glassware|7|Home|777|eingationought|N/A|72169373powder890094|chartreuse|Case|Unknown|33|ableationableought| +1273|AAAAAAAAJPEAAAAA|1997-10-27||Interesting, slim students will supervise by a councils. Wild, european sets convey com|0.75|0.27|4001002|amalgedu pack #2|1|womens|4|Shoes|80|bareing|medium|29765388ivory2819351|peru|Lb|Unknown|38|priationableought| +1274|AAAAAAAAKPEAAAAA|1997-10-27|2000-10-26|Occasional, blind railways would not find about only colonial players. Favorable, foreign centuries might like again normal stairs|3.21|1.09|5002001|importoscholar #1|2|country|5|Music|40|barese|N/A|papaya26444793881110|navajo|Case|Unknown|22|eseationableought| +1275|AAAAAAAAKPEAAAAA|2000-10-27||Strangely persistent colours would develop already able views. Religious, similar problems must get now modest keys. Members could lose; there evident s|8.12|6.33|5002001|brandmaxi #4|2|reference|9|Books|325|antiablepri|N/A|papaya26444793881110|tan|Oz|Unknown|55|antiationableought| +1276|AAAAAAAAMPEAAAAA|1997-10-27|1999-10-27|Relations marry in a attacks. Prime books ma|2.81|1.71|9008005|namelessmaxi #5|8|romance|9|Books|771|oughtationation|N/A|35207006745sandy6120|smoke|Tsp|Unknown|21|callyationableought| +1277|AAAAAAAAMPEAAAAA|1999-10-28|2001-10-26|Relations marry in a attacks. Prime books ma|2.73|2.42|6002006|importocorp #6|8|diamonds|6|Jewelry|771|oughtationation|N/A|2235rose753970134399|drab|Cup|Unknown|72|ationationableought| +1278|AAAAAAAAMPEAAAAA|2001-10-27||Relations marry in a attacks. Prime books ma|1.99|0.73|3001001|amalgexporti #1|8|newborn|3|Children|771|oughtationation|petite|2235rose753970134399|thistle|Dozen|Unknown|89|eingationableought| +1279|AAAAAAAAPPEAAAAA|1997-10-27||Else real homes will not start in common for a issues. Elderly, other players would learn; absolute years make even related managers. Almost economic wr|75.77|67.43|10013005|exportiamalgamalg #5|13|stereo|10|Electronics|702|ablebaration|N/A|5137603822796rosy561|midnight|Pound|Unknown|11|n stationableought| +1280|AAAAAAAAAAFAAAAA|1997-10-27|2000-10-26|Students make. New, common elections make in a conditions; now legal women halt exactly local mountains. Financial, traditional goods get still other parents. Oth|4.81|2.11|3001001|amalgexporti #1|1|newborn|3|Children|254|eseantiable|medium|10yellow363305346041|blush|Bunch|Unknown|30|bareingableought| +1281|AAAAAAAAAAFAAAAA|2000-10-27||General, new programmes suppress dangerously big kids. Still full members say buildings. Old, classic proceedings change long, helpful tons. Stra|4.05|3.28|4003002|exportiedu pack #2|3|kids|4|Shoes|54|eseanti|medium|079695088smoke791791|white|Pallet|Unknown|39|oughteingableought| +1282|AAAAAAAACAFAAAAA|1997-10-27|1999-10-27|Catholic, favorite interests may decide agents. Extraordinary office|29.09|18.03|7005003|scholarbrand #3|5|blinds/shades|7|Home|198|eingn stought|N/A|8451366tomato3211228|sandy|Ounce|Unknown|4|ableeingableought| +1283|AAAAAAAACAFAAAAA|1999-10-28|2001-10-26|Costs fire now. Leading, complete weeks know enough historical |2.42|18.03|1002002|importoamalg #2|2|fragrances|1|Women|46|callyese|economy|500042987339986sky80|burlywood|Lb|Unknown|16|prieingableought| +1284|AAAAAAAACAFAAAAA|2001-10-27||Much deep projects bat surely new, deep husbands. Only chief leaders kiss so clothes; schools |6.42|18.03|4003001|exportiedu pack #1|2|kids|4|Shoes|46|callyese|large|6sky6452044039025779|peru|Ounce|Unknown|44|eseeingableought| +1285|AAAAAAAAFAFAAAAA|1997-10-27||Large-scale, still minutes ought to hang |0.58|0.31|6012006|importobrand #6|12|costume|6|Jewelry|478|eingationese|N/A|42347207109091smoke4|almond|Bunch|Unknown|2|antieingableought| +1286|AAAAAAAAGAFAAAAA|1997-10-27|2000-10-26|Social characters follow too natural systems; charges must not catch please severely natural walls; enemies hook as powerfu|5.49|2.79|10009012|maxiunivamalg #12|9|televisions|10|Electronics|705|antibaration|N/A|71857726521navy26139|smoke|Dozen|Unknown|18|callyeingableought| +1287|AAAAAAAAGAFAAAAA|2000-10-27||Fans sleep here tough leaves. Fully free practitioners exert women. Ideal times might not use hence plan|6.08|2.79|9006004|corpmaxi #4|9|parenting|9|Books|705|antibaration|N/A|71857726521navy26139|floral|Gross|Unknown|31|ationeingableought| +1288|AAAAAAAAIAFAAAAA|1997-10-27|1999-10-27|Physical, old services shall not like physi|2.35|1.29|10014014|edu packamalgamalg #14|14|automotive|10|Electronics|391|oughtn stpri|N/A|papaya21709438376486|salmon|Carton|Unknown|12|eingeingableought| +1289|AAAAAAAAIAFAAAAA|1999-10-28|2001-10-26|Able responses might think in a sciences. Really other prices start far by a issues. Alone fatal females increase important authorities; jobs could know o|2.94|1.29|4001002|amalgedu pack #2|14|womens|4|Shoes|532|ableprianti|petite|papaya21709438376486|peru|Gross|Unknown|37|n steingableought| +1290|AAAAAAAAIAFAAAAA|2001-10-27||Able responses might think in a sciences. Really other prices start far by a issues. Alone fatal females increase important authorities; jobs could know o|3.78|3.40|7013001|exportinameless #1|13|wallpaper|7|Home|532|ableprianti|N/A|papaya21709438376486|maroon|Case|Unknown|11|barn stableought| +1291|AAAAAAAALAFAAAAA|1997-10-27||New plants bring however on a years. Economic, british needs go of course. Children shall not benefit. Dangerous, whole patients ought to let. Camps shall not seek merely modest hearts. Hands like|5.91|3.95|9005008|scholarmaxi #8|5|history|9|Books|112|ableoughtought|N/A|405072082burnished90|tomato|Carton|Unknown|27|oughtn stableought| +1292|AAAAAAAAMAFAAAAA|1997-10-27|2000-10-26|Still good processes might work instructions. Falls inspire long other, decent teachers. Hundreds cause also dear, local men. For example specialist programmes will |5.13|2.10|8015003|scholarmaxi #3|15|fishing|8|Sports|307|ationbarpri|N/A|1886173spring4929302|red|Each|Unknown|26|ablen stableought| +1293|AAAAAAAAMAFAAAAA|2000-10-27||Still good processes might work instructions. Falls inspire long other, decent teachers. Hundreds cause also dear, local men. For example specialist programmes will |4.61|2.10|4004002|edu packedu pack #2|15|athletic|4|Shoes|191|oughtn stought|medium|1886173spring4929302|wheat|Carton|Unknown|7|prin stableought| +1294|AAAAAAAAOAFAAAAA|1997-10-27|1999-10-27|Firstly other dishes see mainly successful points. Especially main pupils express lately other, r|95.00|36.10|10012003|importoamalgamalg #3|12|monitors|10|Electronics|331|oughtpripri|N/A|7281120634linen88846|gainsboro|Each|Unknown|19|esen stableought| +1295|AAAAAAAAOAFAAAAA|1999-10-28|2001-10-26|Firstly other dishes see mainly successful points. Especially main pupils express lately other, r|7.63|6.25|10012003|importonameless #6|12|paint|7|Home|331|oughtpripri|N/A|7281120634linen88846|saddle|Gross|Unknown|15|antin stableought| +1296|AAAAAAAAOAFAAAAA|2001-10-27||Firstly other dishes see mainly successful points. Especially main pupils express lately other, r|4.04|2.06|6005001|scholarcorp #1|5|earings|6|Jewelry|662|oughtpripri|N/A|672097782peach715907|maroon|Case|Unknown|60|callyn stableought| +1297|AAAAAAAABBFAAAAA|1997-10-27||Days cannot catch particularly exciting, given teachers; |2.05|0.90|4004002|edu packedu pack #2|4|athletic|4|Shoes|263|pricallyable|extra large|6613656851107rosy367|linen|Dram|Unknown|53|ationn stableought| +1298|AAAAAAAACBFAAAAA|1997-10-27|2000-10-26|Following, true criteria used to look pretty net days. Other plans need clearly. Dee|1.19|0.99|8003009|exportinameless #9|3|basketball|8|Sports|407|ationbarese|N/A|5418200pink988850911|saddle|Bunch|Unknown|29|eingn stableought| +1299|AAAAAAAACBFAAAAA|2000-10-27||Following, true criteria used to look pretty net days. Other plans need clearly. Dee|8.81|0.99|1002002|importoamalg #2|2|fragrances|1|Women|407|ationbarese|small|66powder458903231121|lace|Bundle|Unknown|38|n stn stableought| +1300|AAAAAAAAEBFAAAAA|1997-10-27|1999-10-27|Small companies develop vehemently. Past, great rights would get so ways. Soon national members achieve. Professional, stupid properties can tell m|99.89|73.91|9006005|corpmaxi #5|6|parenting|9|Books|176|callyationought|N/A|923rosy4515093315242|purple|Box|Unknown|39|barbarpriought| +1301|AAAAAAAAEBFAAAAA|1999-10-28|2001-10-26|Small companies develop vehemently. Past, great rights would get so ways. Soon national members achieve. Professional, stupid properties can tell m|8.16|73.91|8014002|edu packmaxi #2|14|tennis|8|Sports|254|callyationought|N/A|86lemon0413294118177|navajo|Oz|Unknown|13|oughtbarpriought| +1302|AAAAAAAAEBFAAAAA|2001-10-27||Small companies develop vehemently. Past, great rights would get so ways. Soon national members achieve. Professional, stupid properties can tell m|3.50|3.15|8004003|edu packnameless #3|14|camping|8|Sports|91|callyationought|N/A|075ghost236734446970|snow|Tbl|Unknown|30|ablebarpriought| +1303|AAAAAAAAHBFAAAAA|1997-10-27||Dollars prove everywhere o|7.89|4.02|7001004|amalgbrand #4|1|bathroom|7|Home|225|antiableable|N/A|76104691777096slate4|saddle|Carton|Unknown|22|pribarpriought| +1304|AAAAAAAAIBFAAAAA|1997-10-27|2000-10-26|Comprehensive reasons follow so from |8.00|4.88|2002001|importoimporto #1|2|shirts|2|Men|510|baroughtanti|medium|1977513sandy90023784|wheat|Tsp|Unknown|25|esebarpriought| +1305|AAAAAAAAIBFAAAAA|2000-10-27||Sales save from a weeks. Considerable,|0.23|4.88|2002002|importoimporto #2|2|shirts|2|Men|488|baroughtanti|petite|1977513sandy90023784|salmon|Gram|Unknown|46|antibarpriought| +1306|AAAAAAAAKBFAAAAA|1997-10-27|1999-10-27|Final, increasing cases see never wooden miles; top officials can matter keenly s|8.26|2.97|4004001|edu packedu pack #1|4|athletic|4|Shoes|945|antiesen st|medium|0104sandy26964944956|forest|Box|Unknown|34|callybarpriought| +1307|AAAAAAAAKBFAAAAA|1999-10-28|2001-10-26|Final, increasing cases see never wooden miles; top officials can matter keenly s|3.14|2.97|7007008|brandbrand #8|7|decor|7|Home|945|antiesen st|N/A|54530thistle87029994|tomato|Carton|Unknown|94|ationbarpriought| +1308|AAAAAAAAKBFAAAAA|2001-10-27||Final, increasing cases see never wooden miles; top officials can matter keenly s|2.52|1.81|7007008|edu packexporti #1|7|school-uniforms|3|Children|945|antiesen st|N/A|54530thistle87029994|navy|Tsp|Unknown|15|eingbarpriought| +1309|AAAAAAAANBFAAAAA|1997-10-27||Unhappy,|8.54|3.84|10003001|exportiunivamalg #1|3|dvd/vcr players|10|Electronics|46|callyese|N/A|6971puff409477879788|olive|Gram|Unknown|19|n stbarpriought| +1310|AAAAAAAAOBFAAAAA|1997-10-27|2000-10-26|Pupils can pay really excellent figures. Recent standards may adopt; days take after the words. Questions o|3.25|1.33|5004001|edu packscholar #1|4|classical|5|Music|434|esepriese|N/A|519237382568purple57|thistle|Cup|Unknown|33|baroughtpriought| +1311|AAAAAAAAOBFAAAAA|2000-10-27||Important colours used to get so future, usual journals. Annually vertical tables celebrate more. Especially military sounds mig|0.27|1.33|5004001|amalgedu pack #2|1|womens|4|Shoes|1|ought|large|8840116pink006876518|lavender|Tbl|Unknown|2|oughtoughtpriought| +1312|AAAAAAAAACFAAAAA|1997-10-27|1999-10-27|Distinct, mental legs must think fully sure, firm departments; both only circumstances could know significantly. Only, front processes see now between the others. Ago english ti|2.50|1.42|4002001|importoedu pack #1|2|mens|4|Shoes|623|priablecally|large|2057036turquoise1337|navy|Gross|Unknown|28|ableoughtpriought| +1313|AAAAAAAAACFAAAAA|1999-10-28|2001-10-26|Much chosen colours recognise among a rules. Days shall sleep just. Irish, late customers make aside for a versions. Other standards leave availabl|2.04|1.42|9009006|maximaxi #6|9|science|9|Books|313|priablecally|N/A|92midnight4719965450|midnight|Gram|Unknown|29|prioughtpriought| +1314|AAAAAAAAACFAAAAA|2001-10-27||Much chosen colours recognise among a rules. Days shall sleep just. Irish, late customers make aside for a versions. Other standards leave availabl|3.83|1.42|1002001|importoamalg #1|9|fragrances|1|Women|313|priablecally|large|92midnight4719965450|thistle|Oz|Unknown|12|eseoughtpriought| +1315|AAAAAAAADCFAAAAA|1997-10-27||Quick, sad situations might leave |6.79|4.48|8011006|amalgmaxi #6|11|archery|8|Sports|38|eingpri|N/A|95500violet268800266|seashell|Bundle|Unknown|16|antioughtpriought| +1316|AAAAAAAAECFAAAAA|1997-10-27|2000-10-26|Apparently dead factors make well monthly, european documents. Strong meetings may not understand |9.74|3.01|6001005|amalgcorp #5|1|birdal|6|Jewelry|878|eingationeing|N/A|sky97630691614041488|goldenrod|Dram|Unknown|26|callyoughtpriought| +1317|AAAAAAAAECFAAAAA|2000-10-27||Apparently dead factors make well monthly, european documents. Strong meetings may not understand |5.80|2.03|2001002|amalgimporto #2|1|accessories|2|Men|492|eingationeing|extra large|sky97630691614041488|royal|Lb|Unknown|47|ationoughtpriought| +1318|AAAAAAAAGCFAAAAA|1997-10-27|1999-10-27|Hard british units see so different communities.|8.17|7.18|7013009|exportinameless #9|13|wallpaper|7|Home|138|eingpriought|N/A|6138violet4854285631|red|Gross|Unknown|9|eingoughtpriought| +1319|AAAAAAAAGCFAAAAA|1999-10-28|2001-10-26|Hard british units see so different communities.|0.09|7.18|6009008|maxicorp #8|13|womens watch|6|Jewelry|138|eingpriought|N/A|8741940707salmon1965|pink|Bunch|Unknown|38|n stoughtpriought| +1320|AAAAAAAAGCFAAAAA|2001-10-27||Well old beaches take below minutes. Women find also; irish, complete things might operate about rural examples|24.48|7.18|10011012|amalgamalgamalg #12|11|disk drives|10|Electronics|566|callycallyanti|N/A|8741940707salmon1965|powder|Ounce|Unknown|84|barablepriought| +1321|AAAAAAAAJCFAAAAA|1997-10-27||Willingly sensible accounts tell directly big bodies. Concerned hours win also agricultural attacks. Variable ends might not ensure together hands. Public police used to come probably with a|84.32|27.82|8008002|namelessnameless #2|8|outdoor|8|Sports|373|priationpri|N/A|5rosy777217074172416|yellow|Each|Unknown|19|oughtablepriought| +1322|AAAAAAAAKCFAAAAA|1997-10-27|2000-10-26|High, clear characters might believe still. Weak others might reassure impossible weeks. Tiny months could not map earlier. Far,|3.70|1.11|2003001|exportiimporto #1|3|pants|2|Men|134|esepriought|medium|05649999720lavender6|white|Each|Unknown|6|ableablepriought| +1323|AAAAAAAAKCFAAAAA|2000-10-27||Little achievements sh|9.88|5.63|2003001|scholarbrand #4|3|blinds/shades|7|Home|289|n steingable|N/A|05649999720lavender6|almond|Case|Unknown|33|priablepriought| +1324|AAAAAAAAMCFAAAAA|1997-10-27|1999-10-27|Members suffer sound employees. Europe|4.22|3.16|10009016|maxiunivamalg #16|9|televisions|10|Electronics|928|eingablen st|N/A|9ivory55404501927660|moccasin|Carton|Unknown|3|eseablepriought| +1325|AAAAAAAAMCFAAAAA|1999-10-28|2001-10-26|Seats should put both. Actual, financial techniques need forward only similar cases. Automatic reasons used to go less than sure represen|2.87|2.29|1002002|importoamalg #2|9|fragrances|1|Women|928|eingablen st|large|0369994807159pale478|tomato|Tsp|Unknown|10|antiablepriought| +1326|AAAAAAAAMCFAAAAA|2001-10-27||Seats should put both. Actual, financial techniques need forward only similar cases. Automatic reasons used to go less than sure represen|8.34|2.29|4002001|importoedu pack #1|9|mens|4|Shoes|928|eingablen st|extra large|0369994807159pale478|rosy|Pallet|Unknown|5|callyablepriought| +1327|AAAAAAAAPCFAAAAA|1997-10-27||Flat patients die specific, pink losses. Palestinian thousands tolerate materially cuts. Bodies may not float senior, other factors. Pure experiments could survive too |7.34|6.23|7006008|corpbrand #8|6|rugs|7|Home|295|antin stable|N/A|7518541930948white57|plum|Lb|Unknown|30|ationablepriought| +1328|AAAAAAAAADFAAAAA|1997-10-27|2000-10-26|Satisfactory, technical shadows get. Lexical structures would not blame. Only hard |78.25|59.47|9002009|importomaxi #9|2|business|9|Books|168|eingcallyought|N/A|171574511aquamarine2|tan|Case|Unknown|12|eingablepriought| +1329|AAAAAAAAADFAAAAA|||Satisfactory, technical shadows get. Lexical structures would not blame. Only hard |0.63||||1||5||139||N/A||||||| +1330|AAAAAAAACDFAAAAA|1997-10-27|1999-10-27|Asleep, philos|4.18|2.54|9009011|maximaxi #11|9|science|9|Books|328|eingablepri|N/A|7salmon8451264629191|rose|Pound|Unknown|10|barpripriought| +1331|AAAAAAAACDFAAAAA|1999-10-28|2001-10-26|Red, light activities |1.58|2.54|6008004|namelesscorp #4|8|mens watch|6|Jewelry|328|eingablepri|N/A|7salmon8451264629191|sky|Carton|Unknown|38|oughtpripriought| +1332|AAAAAAAACDFAAAAA|2001-10-27||Traders should not like; also national sites need there successful areas. Old parties would expect home in a feet; national, fine forces assume. Members exist badly. Soft books used to put profess|2.62|2.54|6008004|maximaxi #1|8|science|9|Books|131|eingablepri|N/A|7salmon8451264629191|smoke|Oz|Unknown|40|ablepripriought| +1333|AAAAAAAAFDFAAAAA|1997-10-27||All right used men must demand. Visual companies take entirely inhabitants; forward common hands hear here local customers. So traditional questions shal|7.18|4.30|9012002|importounivamalg #2|12|home repair|9|Books|622|ableablecally|N/A|tomato22044281957751|plum|Tsp|Unknown|98|pripripriought| +1334|AAAAAAAAGDFAAAAA|1997-10-27|2000-10-26|Longer usual findings afford yet. As willing other|1.75|0.75|8008005|namelessnameless #5|8|outdoor|8|Sports|645|antiesecally|N/A|0481357023salmon1836|gainsboro|Ounce|Unknown|50|esepripriought| +1335|AAAAAAAAGDFAAAAA|2000-10-27||Longer usual findings afford yet. As willing other|2.57|2.15|8008005|amalgamalgamalg #10|11|disk drives|10|Electronics|181|antiesecally|N/A|0481357023salmon1836|antique|Unknown|Unknown|40|antipripriought| +1336|AAAAAAAAIDFAAAAA|1997-10-27|1999-10-27|New, ethnic firms can increase distinctive schools. So scottish children remember really good r|1.15|0.64|4001001|amalgedu pack #1|1|womens|4|Shoes|217|ationoughtable|large|590854chiffon4133295|slate|Lb|Unknown|17|callypripriought| +1337|AAAAAAAAIDFAAAAA|1999-10-28|2001-10-26|New, ethnic firms can increase distinctive schools. So scottish children remember really good r|6.75|2.43|4001001|corpnameless #8|6|football|8|Sports|217|ationoughtable|N/A|590854chiffon4133295|sienna|Tbl|Unknown|35|ationpripriought| +1338|AAAAAAAAIDFAAAAA|2001-10-27||New, ethnic firms can increase distinctive schools. So scottish children remember really good r|7.09|2.43|4004001|edu packedu pack #1|6|athletic|4|Shoes|627|ationablecally|N/A|281011chiffon0392842|sienna|Gross|Unknown|4|eingpripriought| +1339|AAAAAAAALDFAAAAA|1997-10-27||Hands may not allow only in a lands; linear, other pubs say; social, precise women identify for a patients. Preferences develop alone now rich motives. Ever good tas|3.68|2.68|9002008|importomaxi #8|2|business|9|Books|256|callyantiable|N/A|74296348713thistle53|peru|Cup|Unknown|40|n stpripriought| +1340|AAAAAAAAMDFAAAAA|1997-10-27|2000-10-26|Black, old things prove. Even rural businesses used to control really from the decisions; strange colle|1.79|1.28|7002001|importobrand #1|2|bedding|7|Home|233|pripriable|N/A|tan16939606784942023|saddle|Pound|Unknown|45|baresepriought| +1341|AAAAAAAAMDFAAAAA|2000-10-27||Full couples keep terribly; political, difficult rules may look; unusu|5.49|1.70|7002001|exportiexporti #2|3|toddlers|3|Children|233|pripriable|extra large|papaya47689544348902|honeydew|Pound|Unknown|4|oughtesepriought| +1342|AAAAAAAAODFAAAAA|1997-10-27|1999-10-27|Illegal, beautiful points know forward in a banks. Here good details should last today key doctors. Practical rooms cost responsible colonies; twice clear parents should thi|9.22|4.79|7010003|univnameless #3|10|flatware|7|Home|97|ationn st|N/A|2444820205pink418162|seashell|Lb|Unknown|12|ableesepriought| +1343|AAAAAAAAODFAAAAA|1999-10-28|2001-10-26|Communities could stop long cars; technical eyes extend sure|4.21|4.79|7010003|maxinameless #4|10|optics|8|Sports|97|ationn st|N/A|993933pale1480012447|mint|N/A|Unknown|56|priesepriought| +1344|AAAAAAAAODFAAAAA|2001-10-27||Vaguely running pp. ought to drink much tropical, local sides. Traditional details like personal perceptions. Poor experiments should |5.87|2.52|7010003|edu packedu pack #1|10|athletic|4|Shoes|97|ationn st|small|308837701salmon40542|olive|Unknown|Unknown|4|eseesepriought| +1345|AAAAAAAABEFAAAAA|1997-10-27||Purpo|34.69|24.28|4004002|edu packedu pack #2|4|athletic|4|Shoes|668|eingcallycally|medium|42803500993snow99511|ghost|Oz|Unknown|5|antiesepriought| +1346|AAAAAAAACEFAAAAA|1997-10-27|2000-10-26|Important years participate indeed. Hands make so. Great, environmental lives ought to exist so national, free|4.25|1.74|9015003|scholarunivamalg #3|15|fiction|9|Books|234|esepriable|N/A|1919556666medium0315|smoke|Carton|Unknown|15|callyesepriought| +1347|AAAAAAAACEFAAAAA|2000-10-27||Important years participate indeed. Hands make so. Great, environmental lives ought to exist so national, free|5.69|4.66|9013004|exportiunivamalg #4|13|self-help|9|Books|204|esebarable|N/A|1919556666medium0315|turquoise|Tbl|Unknown|3|ationesepriought| +1348|AAAAAAAAEEFAAAAA|1997-10-27|1999-10-27|Raw men produce thick practices. Machines come. Well increased changes ought to generate local, ordinary movement|3.69|1.73|4004001|edu packedu pack #1|4|athletic|4|Shoes|286|callyeingable|large|7174129911violet0691|turquoise|N/A|Unknown|5|eingesepriought| +1349|AAAAAAAAEEFAAAAA|1999-10-28|2001-10-26|Raw men produce thick practices. Machines come. Well increased changes ought to generate local, ordinary movement|8.78|1.73|6010006|univbrand #6|10|jewelry boxes|6|Jewelry|83|prieing|N/A|7174129911violet0691|peach|Tbl|Unknown|8|n stesepriought| +1350|AAAAAAAAEEFAAAAA|2001-10-27||Raw men produce thick practices. Machines come. Well increased changes ought to generate local, ordinary movement|6.24|3.43|6010006|importoamalg #1|2|fragrances|1|Women|957|ationantin st|large|3240871599277slate63|royal|Dozen|Unknown|52|barantipriought| +1351|AAAAAAAAHEFAAAAA|1997-10-27||Sure, common teams surrender again important, clear goals. Wives take happy, certain issues. Objects get quite. Representatives|2.01|0.76|5003002|exportischolar #2|3|pop|5|Music|27|ationable|N/A|654176926344694red07|rosy|Dram|Unknown|40|oughtantipriought| +1352|AAAAAAAAIEFAAAAA|1997-10-27|2000-10-26|About natural economie|8.10|2.43|1003001|exportiamalg #1|3|maternity|1|Women|213|prioughtable|N/A|20lace14440158644623|deep|Bunch|Unknown|31|ableantipriought| +1353|AAAAAAAAIEFAAAAA|2000-10-27||Thick centres ought to alter german stations. Occasionally british bodies say popular, final things. Happy, white contracts may not imagine also good doctor|5.23|2.43|9006010|corpmaxi #10|3|parenting|9|Books|99|prioughtable|N/A|08859609932salmon239|wheat|Gram|Unknown|5|priantipriought| +1354|AAAAAAAAKEFAAAAA|1997-10-27|1999-10-27|Social quantities shoul|0.75|0.23|7013005|exportinameless #5|13|wallpaper|7|Home|147|ationeseought|N/A|6674lawn202869777830|sienna|Cup|Unknown|37|eseantipriought| +1355|AAAAAAAAKEFAAAAA|1999-10-28|2001-10-26|Social quantities shoul|2.48|0.96|4002002|importoedu pack #2|13|mens|4|Shoes|147|ationeseought|small|6674lawn202869777830|pale|Cup|Unknown|18|antiantipriought| +1356|AAAAAAAAKEFAAAAA|2001-10-27||Difficulties can know urban, complex grants. Scottish hundreds sell for instance|3.59|2.36|4002001|importoedu pack #1|13|mens|4|Shoes|621|ationeseought|medium|1pink386818625415908|goldenrod|Oz|Unknown|34|callyantipriought| +1357|AAAAAAAANEFAAAAA|1997-10-27||Operations used to kill perhaps alwa|1.08|0.56|10010015|univamalgamalg #15|10|memory|10|Electronics|154|eseantiought|N/A|97sienna877643279336|slate|Oz|Unknown|9|ationantipriought| +1358|AAAAAAAAOEFAAAAA|1997-10-27|2000-10-26|Things used to go yet. Fingers will not keep desperate decades. Again forward|2.36|1.55|2004001|edu packimporto #1|4|sports-apparel|2|Men|126|callyableought|large|32rose34103672495948|saddle|Lb|Unknown|9|eingantipriought| +1359|AAAAAAAAOEFAAAAA|2000-10-27||Things used to go yet. Fingers will not keep desperate decades. Again forward|9.77|8.59|2004001|amalgamalg #2|1|dresses|1|Women|644|callyableought|petite|6201324steel83225509|red|Cup|Unknown|1|n stantipriought| +1360|AAAAAAAAAFFAAAAA|1997-10-27|1999-10-27|Local numbers may reap that in a shows. Useful, democratic methods effect under; resources catch. Wearily other roads ought to benef|1.54|1.26|5001001|amalgscholar #1|1|rock|5|Music|443|prieseese|N/A|07spring892482015364|orange|Cup|Unknown|20|barcallypriought| +1361|AAAAAAAAAFFAAAAA|1999-10-28|2001-10-26|Local numbers may reap that in a shows. Useful, democratic methods effect under; resources catch. Wearily other roads ought to benef|4.59|2.29|5001001|amalgscholar #2|1|rock|5|Music|443|prieseese|N/A|07spring892482015364|papaya|Carton|Unknown|58|oughtcallypriought| +1362|AAAAAAAAAFFAAAAA|2001-10-27||Local numbers may reap that in a shows. Useful, democratic methods effect under; resources catch. Wearily other roads ought to benef|6.51|2.29|5001001|edu packscholar #1|4|classical|5|Music|443|prieseese|N/A|58032247337tomato082|orchid|Bunch|Unknown|7|ablecallypriought| +1363|AAAAAAAADFFAAAAA|1997-10-27||Necessary, social bedrooms think so full poles; babies prove now. Profitable payments used to break there. Major, radical households |1.51|1.31|8009004|maxinameless #4|9|optics|8|Sports|582|ableeinganti|N/A|542678105tan98745896|aquamarine|N/A|Unknown|33|pricallypriought| +1364|AAAAAAAAEFFAAAAA|1997-10-27|2000-10-26|Totally pure styles would seek charges; values say. Normal, big activi|1.07|0.84|10009005|maxiunivamalg #5|9|televisions|10|Electronics|149|n steseought|N/A|758981rose6296291598|sienna|Each|Unknown|59|esecallypriought| +1365|AAAAAAAAEFFAAAAA|2000-10-27||Totally pure styles would seek charges; values say. Normal, big activi|6.87|0.84|10009005|scholarmaxi #10|9|history|9|Books|149|n steseought|N/A|758981rose6296291598|goldenrod|Case|Unknown|28|anticallypriought| +1366|AAAAAAAAGFFAAAAA|1997-10-27|1999-10-27|Likely, casual patients work most negotiations. Recent plans must |1.60|1.12|10016007|corpamalgamalg #7|16|wireless|10|Electronics|747|ationeseation|N/A|13592metallic0890662|red|Bundle|Unknown|38|callycallypriought| +1367|AAAAAAAAGFFAAAAA|1999-10-28|2001-10-26|Due contexts should press quickly on the negotia|3.62|1.12|10016007|edu packunivamalg #12|16|sports|9|Books|19|n stought|N/A|13592metallic0890662|snow|Ton|Unknown|11|ationcallypriought| +1368|AAAAAAAAGFFAAAAA|2001-10-27||Perhaps continuing users shall improve troubles;|2.98|1.12|10016007|edu packscholar #1|16|classical|5|Music|19|n stought|N/A|13592metallic0890662|peru|Each|Unknown|9|eingcallypriought| +1369|AAAAAAAAJFFAAAAA|1997-10-27||Exact jews make again regional times|0.82|0.27|7014010|edu packnameless #10|14|glassware|7|Home|46|callyese|N/A|8858583560sienna8532|sienna|Lb|Unknown|71|n stcallypriought| +1370|AAAAAAAAKFFAAAAA|1997-10-27|2000-10-26|Full-time, lovely miles employ home. Regular assets may not protect much for the relationships. So good guidelines may care small figures. Financial, happy parents call also much real op|51.70|29.46|8004001|edu packnameless #1|4|camping|8|Sports|481|oughteingese|N/A|magenta8260952640276|midnight|Lb|Unknown|29|barationpriought| +1371|AAAAAAAAKFFAAAAA|2000-10-27||Typical copies used to forget much fixed, crucial schools. Intimate, financial co|18.18|12.72|7008002|namelessbrand #2|4|lighting|7|Home|481|oughteingese|N/A|5476goldenrod9515900|metallic|N/A|Unknown|20|oughtationpriought| +1372|AAAAAAAAMFFAAAAA|1997-10-27|1999-10-27|Communities used to relocate clearly strange, new walls; european, rich championships make current depths. Sure studies may reflect only instinctively old forces. Foreign, diverse|8.22|4.11|9001005|amalgmaxi #5|1|arts|9|Books|183|prieingought|N/A|8574277567royal26939|thistle|Gram|Unknown|44|ableationpriought| +1373|AAAAAAAAMFFAAAAA|1999-10-28|2001-10-26|Communities used to relocate clearly strange, new walls; european, rich championships make current depths. Sure studies may reflect only instinctively old forces. Foreign, diverse|4.11|4.11|9001005|amalgunivamalg #14|1|cameras|10|Electronics|183|prieingought|N/A|8574277567royal26939|violet|Gross|Unknown|5|priationpriought| +1374|AAAAAAAAMFFAAAAA|2001-10-27||Communities used to relocate clearly strange, new walls; european, rich championships make current depths. Sure studies may reflect only instinctively old forces. Foreign, diverse|0.16|0.11|9001005|amalgscholar #1|1|rock|5|Music|183|prieingought|N/A|83164377rose55213154|rose|Ounce|Unknown|63|eseationpriought| +1375|AAAAAAAAPFFAAAAA|1997-10-27||Plain, long periods retrieve for a recommendation|7.56|6.72|1001002|amalgamalg #2|1|dresses|1|Women|21|oughtable|economy|4309348708554red1128|rose|Oz|Unknown|7|antiationpriought| +1376|AAAAAAAAAGFAAAAA|1997-10-27|2000-10-26|Military, practical factors could not compete progr|2.69|1.58|6012001|importobrand #1|12|costume|6|Jewelry|113|prioughtought|N/A|snow8893404317305605|puff|Dozen|Unknown|34|callyationpriought| +1377|AAAAAAAAAGFAAAAA|2000-10-27||Poor, western pains will not rule in a relations. Evidently complete stars would transfer just in a residents. Likely, old members work now extensive, slim leaves. Limited p|4.52|3.57|6012001|exportischolar #2|12|pop|5|Music|113|prioughtought|N/A|snow8893404317305605|red|Ton|Unknown|27|ationationpriought| +1378|AAAAAAAACGFAAAAA|1997-10-27|1999-10-27|Short pupils might open metres. Very pro|18.31|5.85|4004001|edu packedu pack #1|4|athletic|4|Shoes|629|n stablecally|medium|76678553737magenta43|red|Bundle|Unknown|99|eingationpriought| +1379|AAAAAAAACGFAAAAA|1999-10-28|2001-10-26|Short pupils might open metres. Very pro|9.29|5.85|4004001|exportiunivamalg #3|3|dvd/vcr players|10|Electronics|963|pricallyn st|N/A|847197860seashell257|dim|Carton|Unknown|3|n stationpriought| +1380|AAAAAAAACGFAAAAA|2001-10-27||Remote, severe merchants eat exclusively general, sil|6.66|5.99|3003001|exportiexporti #1|3|toddlers|3|Children|963|pricallyn st|N/A|847197860seashell257|yellow|Dram|Unknown|5|bareingpriought| +1381|AAAAAAAAFGFAAAAA|1997-10-27||Average teachers go single men. Now integrated women include however in order clean materials. British, white doors take more. Even true holes state substantially later other rats; downs|5.87|4.51|4002002|importoedu pack #2|2|mens|4|Shoes|181|oughteingought|large|4rosy608047435703382|snow|Bunch|Unknown|3|oughteingpriought| +1382|AAAAAAAAGGFAAAAA|1997-10-27|2000-10-26|Spare players will not remove fo|1.79|0.68|4001001|amalgedu pack #1|1|womens|4|Shoes|69|n stcally|small|626623004163wheat716|slate|Dozen|Unknown|84|ableeingpriought| +1383|AAAAAAAAGGFAAAAA|2000-10-27||Lives offer real|3.41|0.68|6009008|maxicorp #8|9|womens watch|6|Jewelry|401|oughtbarese|N/A|76277122saddle612436|orchid|N/A|Unknown|24|prieingpriought| +1384|AAAAAAAAIGFAAAAA|1997-10-27|1999-10-27|Eyes understand much small diseases. Individual opportunities dominate high directors. Ne|1.53|1.17|6005001|scholarcorp #1|5|earings|6|Jewelry|204|esebarable|N/A|4164751512970pale337|light|Pound|Unknown|1|eseeingpriought| +1385|AAAAAAAAIGFAAAAA|1999-10-28|2001-10-26|Eyes understand much small diseases. Individual opportunities dominate high directors. Ne|9.59|6.71|6005001|edu packscholar #2|5|classical|5|Music|204|esebarable|N/A|742snow0654489904696|tomato|Cup|Unknown|39|antieingpriought| +1386|AAAAAAAAIGFAAAAA|2001-10-27||Countries reassure together under a points; so other papers cannot pay very goals. Concerned markets must not harm easily local, |8.80|6.71|9014007|edu packunivamalg #7|14|sports|9|Books|242|ableeseable|N/A|742snow0654489904696|smoke|Case|Unknown|25|callyeingpriought| +1387|AAAAAAAALGFAAAAA|1997-10-27||Only european minutes should account just also pink options. Acute, national teeth supply then evil officials. Goals concentrate rather employers. Candi|97.23|46.67|6010004|univbrand #4|10|jewelry boxes|6|Jewelry|160|barcallyought|N/A|81142436345royal5169|white|Cup|Unknown|86|ationeingpriought| +1388|AAAAAAAAMGFAAAAA|1997-10-27|2000-10-26|Straight, immediate parents help more than reso|7.56|2.79|7012009|importonameless #9|12|paint|7|Home|169|n stcallyought|N/A|8614322salmon5562886|sandy|Bunch|Unknown|24|eingeingpriought| +1389|AAAAAAAAMGFAAAAA|2000-10-27||Straight, immediate parents help more than reso|8.52|2.79|7004010|edu packbrand #10|4|curtains/drapes|7|Home|810|baroughteing|N/A|3021758208royal26287|snow|Lb|Unknown|16|n steingpriought| +1390|AAAAAAAAOGFAAAAA|1997-10-27|1999-10-27|Real kids give rather lips. Pure, hungry sides might not resolve both impressive attacks; over large friends refuse. Guilty, sp|99.41|59.64|9003011|exportimaxi #11|3|computers|9|Books|890|barn steing|N/A|63085493orange397077|peru|Each|Unknown|55|barn stpriought| +1391|AAAAAAAAOGFAAAAA|1999-10-28|2001-10-26|Real kids give rather lips. Pure, hungry sides might not resolve both impressive attacks; over large friends refuse. Guilty, sp|2.93|59.64|9003011|exportischolar #2|3|pop|5|Music|443|barn steing|N/A|5042732296saddle4905|hot|Bunch|Unknown|61|oughtn stpriought| +1392|AAAAAAAAOGFAAAAA|2001-10-27||Tightly alleged results put moreover orange sentences. Enough new minutes|3.90|1.32|5003001|exportischolar #1|3|pop|5|Music|443|prieseese|N/A|5042732296saddle4905|plum|Oz|Unknown|9|ablen stpriought| +1393|AAAAAAAABHFAAAAA|1997-10-27||Big, strong women ought to help. Arts stretch more mostly outstanding supplies. Flowers stay soon appointed amounts. Further mental varieties ventur|2.59|0.90|2004002|edu packimporto #2|4|sports-apparel|2|Men|800|barbareing|extra large|3398navajo9424147860|thistle|Bunch|Unknown|11|prin stpriought| +1394|AAAAAAAACHFAAAAA|1997-10-27|2000-10-26|Sometimes new officers will not fancy unhappily just strong hours. Students give very. Formidable services should introduce beings. Traditions may not look instead pract|11.66|6.99|3003001|exportiexporti #1|3|toddlers|3|Children|162|ablecallyought|small|067720439magenta3701|rose|Ounce|Unknown|33|esen stpriought| +1395|AAAAAAAACHFAAAAA|2000-10-27||Sometimes new officers will not fancy unhappily just strong hours. Students give very. Formidable services should introduce beings. Traditions may not look instead pract|1.10|0.97|3003001|edu packimporto #2|3|sports-apparel|2|Men|358|ablecallyought|extra large|067720439magenta3701|spring|Box|Unknown|15|antin stpriought| +1396|AAAAAAAAEHFAAAAA|1997-10-27|1999-10-27|Actively fair matches will like even; brit|3.14|2.51|7003007|exportibrand #7|3|kids|7|Home|206|callybarable|N/A|031saddle31237238621|medium|Pallet|Unknown|21|callyn stpriought| +1397|AAAAAAAAEHFAAAAA|1999-10-28|2001-10-26|Shy offenders cannot find hence even semantic mountains. Cold leads f|4.08|2.20|10009004|maxiunivamalg #4|9|televisions|10|Electronics|179|callybarable|N/A|031saddle31237238621|drab|Cup|Unknown|12|ationn stpriought| +1398|AAAAAAAAEHFAAAAA|2001-10-27||Lovely bags absorb well important shops. Professional, common techniques will not live|6.92|2.20|3002001|importoexporti #1|2|infants|3|Children|118|eingoughtought|medium|607221039958orange91|linen|Bunch|Unknown|36|eingn stpriought| +1399|AAAAAAAAHHFAAAAA|1997-10-27||Pictures shall offer royal, british details. Just new issues will inflict pure functions. Options notice only; tightly local sides shall not make now ago other years. Successful wom|0.23|0.18|4002002|importoedu pack #2|2|mens|4|Shoes|726|callyableation|medium|99208481791222misty3|violet|Case|Unknown|30|n stn stpriought| +1400|AAAAAAAAIHFAAAAA|1997-10-27|2000-10-26|Significantly small suggestions will not come more new blue terms. Fundamentally previous soldiers understand alone huge contracts. Religious, professional miles must ap|4.64|1.57|9006009|corpmaxi #9|6|parenting|9|Books|626|callyablecally|N/A|5powder9324628289656|lemon|N/A|Unknown|13|barbareseought| +1401|AAAAAAAAIHFAAAAA|2000-10-27||Significantly small suggestions will not come more new blue terms. Fundamentally previous soldiers understand alone huge contracts. Religious, professional miles must ap|3.24|1.57|9006009|edu packexporti #2|6|school-uniforms|3|Children|131|callyablecally|large|9915132blush46510815|sky|N/A|Unknown|26|oughtbareseought| +1402|AAAAAAAAKHFAAAAA|1997-10-27|1999-10-27|Subsequent, serious gene|4.93|3.84|7016003|corpnameless #3|16|furniture|7|Home|453|priantiese|N/A|64lavender8845474318|sky|Bundle|Unknown|65|ablebareseought| +1403|AAAAAAAAKHFAAAAA|1999-10-28|2001-10-26|Cuts raise simply in a things. Lips can apply. Basic branches keep low chief difficulties. Products quit about groups. |8.11|7.29|2003002|exportiimporto #2|3|pants|2|Men|453|priantiese|medium|87106salmon038651589|rose|Bunch|Unknown|32|pribareseought| +1404|AAAAAAAAKHFAAAAA|2001-10-27||Cuts raise simply in a things. Lips can apply. Basic branches keep low chief difficulties. Products quit about groups. |0.64|0.57|2003002|exportischolar #1|3|pop|5|Music|38|priantiese|N/A|87106salmon038651589|sandy|Case|Unknown|65|esebareseought| +1405|AAAAAAAANHFAAAAA|1997-10-27||Further new men cannot see between the colours. Classic, small parents will raise current hands; very light policies see probably citi|8.58|4.89|6014006|edu packbrand #6|14|estate|6|Jewelry|329|n stablepri|N/A|632metallic004171130|rosy|Bundle|Unknown|25|antibareseought| +1406|AAAAAAAAOHFAAAAA|1997-10-27|2000-10-26|Bad, national minutes shall oppose else possible years. Comprehensive, particular dates will not repay even old eyes. Years drink liv|8.66|5.28|4001001|amalgedu pack #1|1|womens|4|Shoes|572|ableationanti|medium|541royal416025298540|peach|Tbl|Unknown|29|callybareseought| +1407|AAAAAAAAOHFAAAAA|2000-10-27||Bad, national minutes shall oppose else possible years. Comprehensive, particular dates will not repay even old eyes. Years drink liv|2.96|1.27|4001001|univmaxi #8|10|pools|8|Sports|572|ableationanti|N/A|541royal416025298540|plum|N/A|Unknown|56|ationbareseought| +1408|AAAAAAAAAIFAAAAA|1997-10-27|1999-10-27|Sure, successive titles know undoubtedly additional, hot centres; since different prospects would not tell into a ideas. Cattle will not play enough modern charges. Unnecessary, delighted me|1.77|1.52|2003001|exportiimporto #1|3|pants|2|Men|228|eingableable|large|gainsboro69806648427|white|Tsp|Unknown|26|eingbareseought| +1409|AAAAAAAAAIFAAAAA|1999-10-28|2001-10-26|Years would not drop local chains; sometimes chief investors may not bear then common times. Scottish, adult eyes show. Schools ought to write only more lesser men. Di|9.75|3.51|4003002|exportiedu pack #2|3|kids|4|Shoes|387|eingableable|medium|57256linen3177326978|spring|Box|Unknown|33|n stbareseought| +1410|AAAAAAAAAIFAAAAA|2001-10-27||Years would not drop local chains; sometimes chief investors may not bear then common times. Scottish, adult eyes show. Schools ought to write only more lesser men. Di|1.09|3.51|9014007|edu packunivamalg #7|14|sports|9|Books|22|eingableable|N/A|862304116700235lime9|pink|Ton|Unknown|63|baroughteseought| +1411|AAAAAAAADIFAAAAA|1997-10-27||Hot, royal elements could not kill just local affairs. Never labour links would not use well hard, unusual str|8.30|3.32|6012004|importobrand #4|12|costume|6|Jewelry|870|barationeing|N/A|3smoke14638214587383|smoke|Tbl|Unknown|35|oughtoughteseought| +1412|AAAAAAAAEIFAAAAA|1997-10-27|2000-10-26|New girls reach exactly; only additional students wil|3.94|2.75|8013003|exportimaxi #3|13|sailing|8|Sports|95|antin st|N/A|891633317medium35472|pale|Oz|Unknown|6|ableoughteseought| +1413|AAAAAAAAEIFAAAAA|2000-10-27||New girls reach exactly; only additional students wil|4.39|2.75|8013003|exportischolar #2|13|pop|5|Music|102|antin st|N/A|891633317medium35472|spring|Each|Unknown|49|prioughteseought| +1414|AAAAAAAAGIFAAAAA|1997-10-27|1999-10-27|Full, bad comp|5.79|3.12|5003001|exportischolar #1|3|pop|5|Music|270|barationable|N/A|342038244046papaya77|burnished|Tsp|Unknown|61|eseoughteseought| +1415|AAAAAAAAGIFAAAAA|1999-10-28|2001-10-26|Fully new decisions will move away on a times. British years might condemn so important manage|18.92|3.12|5003001|edu packbrand #6|4|curtains/drapes|7|Home|270|barationable|N/A|342038244046papaya77|honeydew|N/A|Unknown|19|antioughteseought| +1416|AAAAAAAAGIFAAAAA|2001-10-27||Fully new decisions will move away on a times. British years might condemn so important manage|47.95|26.85|5003001|brandbrand #7|4|decor|7|Home|232|ablepriable|N/A|970sienna00625191068|sienna|Pallet|Unknown|42|callyoughteseought| +1417|AAAAAAAAJIFAAAAA|1997-10-27||Movies look previously friends. Oddly responsible objects forget companies. Children want below available, long others. Cu|9.20|5.88|3001002|amalgexporti #2|1|newborn|3|Children|140|bareseought|medium|2737748spring1135747|orange|Dram|Unknown|20|ationoughteseought| +1418|AAAAAAAAKIFAAAAA|1997-10-27|2000-10-26|Bad, greek points would not leave now like a patients. Here national years forget entirely civil consequences. Sl|0.13|0.06|2001001|amalgimporto #1|1|accessories|2|Men|921|oughtablen st|small|408lace8183153339518|aquamarine|Ounce|Unknown|55|eingoughteseought| +1419|AAAAAAAAKIFAAAAA|2000-10-27||Bad, greek points would not leave now like a patients. Here national years forget entirely civil consequences. Sl|2.55|1.07|2001001|univunivamalg #4|1|travel|9|Books|2|able|N/A|408lace8183153339518|papaya|Unknown|Unknown|94|n stoughteseought| +1420|AAAAAAAAMIFAAAAA|1997-10-27|1999-10-27|Expensive, old references make so weeks. Acts climb only. Various minutes ought to attack. No longer different colours inc|4.71|2.11|1002001|importoamalg #1|2|fragrances|1|Women|250|barantiable|small|4dodger0496361723363|goldenrod|Oz|Unknown|20|barableeseought| +1421|AAAAAAAAMIFAAAAA|1999-10-28|2001-10-26|Expensive, old references make so weeks. Acts climb only. Various minutes ought to attack. No longer different colours inc|7.38|2.11|1002001|importoscholar #2|2|country|5|Music|250|barantiable|N/A|4dodger0496361723363|spring|Tbl|Unknown|7|oughtableeseought| +1422|AAAAAAAAMIFAAAAA|2001-10-27||Expensive, old references make so weeks. Acts climb only. Various minutes ought to attack. No longer different colours inc|3.72|2.11|3001001|amalgexporti #1|2|newborn|3|Children|335|barantiable|petite|2802932572901ghost24|sienna|Pallet|Unknown|59|ableableeseought| +1423|AAAAAAAAPIFAAAAA|1997-10-27||Very short foundations would work as. Daily comfortable shareholders take very instruments|4.72|1.88|8011004|amalgmaxi #4|11|archery|8|Sports|116|callyoughtought|N/A|lawn5620312783903769|saddle|Bunch|Unknown|9|priableeseought| +1424|AAAAAAAAAJFAAAAA|1997-10-27|2000-10-26|Normal cases call into a rates. Easy royal police cannot assert long records. Young, scottish exceptions kill more ce|2.50|1.65|9005009|scholarmaxi #9|5|history|9|Books|101|oughtbarought|N/A|2887015yellow4237007|dodger|Case|Unknown|32|eseableeseought| +1425|AAAAAAAAAJFAAAAA|2000-10-27||Western, ordinary problems may run for a years; thus |6.54|1.65|9005009|edu packexporti #2|5|school-uniforms|3|Children|101|oughtbarought|small|2887015yellow4237007|honeydew|Ounce|Unknown|24|antiableeseought| +1426|AAAAAAAACJFAAAAA|1997-10-27|1999-10-27|For example difficult issues com|2.38|2.04|10004016|edu packunivamalg #16|4|audio|10|Electronics|147|ationeseought|N/A|005gainsboro13896984|peach|Cup|Unknown|74|callyableeseought| +1427|AAAAAAAACJFAAAAA|1999-10-28|2001-10-26|Able, angry sources master. Minimum proceedings go more reasonable|2.98|1.54|10004016|exportimaxi #12|3|computers|9|Books|147|ationeseought|N/A|04146762salmon017581|yellow|Tbl|Unknown|11|ationableeseought| +1428|AAAAAAAACJFAAAAA|2001-10-27||Songs should |2.15|1.65|5003001|exportischolar #1|3|pop|5|Music|129|n stableought|N/A|96443lace52114646465|rose|Gross|Unknown|12|eingableeseought| +1429|AAAAAAAAFJFAAAAA|1997-10-27||Best alone women |1.45|0.76|5003002|exportischolar #2|3|pop|5|Music|411|oughtoughtese|N/A|008seashell730862738|dim|Bundle|Unknown|29|n stableeseought| +1430|AAAAAAAAGJFAAAAA|1997-10-27|2000-10-26|Directly different rounds express later; frequent, gentle intervals think justly by a dogs. Short-term, main mana|1.29|0.94|4002001|importoedu pack #1|2|mens|4|Shoes|287|ationeingable|small|3597red0352464672509|smoke|Bundle|Unknown|25|barprieseought| +1431|AAAAAAAAGJFAAAAA|2000-10-27||Directly different rounds express later; frequent, gentle intervals think justly by a dogs. Short-term, main mana|4.48|0.94|5004002|edu packscholar #2|4|classical|5|Music|47|ationese|N/A|80saddle509941019439|thistle|Bundle|Unknown|29|oughtprieseought| +1432|AAAAAAAAIJFAAAAA|1997-10-27|1999-10-27|Nice things would coincide still satisfactory students. Now oth|1.08|0.62|7005003|scholarbrand #3|5|blinds/shades|7|Home|303|pribarpri|N/A|8603200542440sky7533|floral|Dozen|Unknown|10|ableprieseought| +1433|AAAAAAAAIJFAAAAA|1999-10-28|2001-10-26|Proper lives may make; asleep years take pretty about a profits. Indeed primary questions draw |2.65|0.62|7005003|edu packamalg #2|5|swimwear|1|Women|303|pribarpri|medium|8603200542440sky7533|tomato|Dozen|Unknown|15|priprieseought| +1434|AAAAAAAAIJFAAAAA|2001-10-27||Industrial fields write with a resources. Items rain of course white traditions. Addresses ought to complete still to a ages; years ought |3.83|0.62|4002001|importoedu pack #1|2|mens|4|Shoes|303|pribarpri|small|8603200542440sky7533|chartreuse|Cup|Unknown|6|eseprieseought| +1435|AAAAAAAALJFAAAAA|1997-10-27||Homes want sincerely in a unions; together financia|6.67|3.80|10009008|maxiunivamalg #8|9|televisions|10|Electronics|328|eingablepri|N/A|4913384turquoise3179|tomato|Unknown|Unknown|86|antiprieseought| +1436|AAAAAAAAMJFAAAAA|1997-10-27|2000-10-26|Creditors will not put to|6.03|4.94|10002009|importounivamalg #9|2|camcorders|10|Electronics|285|antieingable|N/A|17108879783metallic9|yellow|Dozen|Unknown|52|callyprieseought| +1437|AAAAAAAAMJFAAAAA|2000-10-27||Whole ages eat in a attempts; stores read to a eyes. Demands should test yet the|4.66|4.94|10002009|exportiimporto #2|3|pants|2|Men|162|ablecallyought|medium|5217plum070262593426|metallic|Tsp|Unknown|9|ationprieseought| +1438|AAAAAAAAOJFAAAAA|1997-10-27|1999-10-27|Even sympathetic orders change alternative, available articles; girls can laugh girls; decisions ought to see just healthy, psychological times. Domestic pieces take sales|2.84|1.05|10007011|brandunivamalg #11|7|personal|10|Electronics|100|barbarought|N/A|84874490indian531278|green|Bunch|Unknown|48|eingprieseought| +1439|AAAAAAAAOJFAAAAA|1999-10-28|2001-10-26|Clinical patients lead quite personnel. Developments could like also home statistical effects. Other ministers should want in a ways. Soon soft disputes will include over bot|4.28|1.36|5001002|amalgscholar #2|1|rock|5|Music|100|barbarought|N/A|20253pink30119317472|smoke|N/A|Unknown|13|n stprieseought| +1440|AAAAAAAAOJFAAAAA|2001-10-27||Quickly social movements fix really quite constitutional conflicts. Courts find always kinds. Terms choose thousands. Maybe co|3.63|1.27|4003001|exportiedu pack #1|3|kids|4|Shoes|100|barbarought|medium|20253pink30119317472|violet|Pound|Unknown|17|bareseeseought| +1441|AAAAAAAABKFAAAAA|1997-10-27||Computers ought to get apart f|1.01|0.63|5001002|amalgscholar #2|1|rock|5|Music|481|oughteingese|N/A|8675562saddle1046760|forest|Carton|Unknown|55|oughteseeseought| +1442|AAAAAAAACKFAAAAA|1997-10-27|2000-10-26|Others |3.03|2.69|3004001|edu packexporti #1|4|school-uniforms|3|Children|60|barcally|large|4800480810sandy04811|plum|Pound|Unknown|28|ableeseeseought| +1443|AAAAAAAACKFAAAAA|2000-10-27||Sadly small flights can improve new facilities. Origins must get public, responsible contributions. Spectacular languages might not differentiate downs|4.62|2.69|3004001|amalgexporti #2|1|newborn|3|Children|60|barcally|large|4800480810sandy04811|spring|Dram|Unknown|24|prieseeseought| +1444|AAAAAAAAEKFAAAAA|1997-10-27|1999-10-27|Reports introduce likewise ill, individual schools. Busy balls must belong determined responses. However outstanding services used to interpret quite from the arrangements. C|0.14|0.05|8011005|amalgmaxi #5|11|archery|8|Sports|636|callyprically|N/A|2red9835436056443443|peru|Bunch|Unknown|16|eseeseeseought| +1445|AAAAAAAAEKFAAAAA|1999-10-28|2001-10-26|Appropriate, magnificent police believe all small, small systems. About gradual patients shall not like both mixed gifts. Strong, european categories demonstrate early. Reve|0.78|0.05|4004002|edu packedu pack #2|4|athletic|4|Shoes|23|callyprically|medium|2red9835436056443443|steel|Oz|Unknown|15|antieseeseought| +1446|AAAAAAAAEKFAAAAA|2001-10-27||Appropriate, magnificent police believe all small, small systems. About gradual patients shall not like both mixed gifts. Strong, european categories demonstrate early. Reve|7.70|0.05|3001001|amalgexporti #1|4|newborn|3|Children|23|priable|small|469778mint1120654286|navajo|Case|Unknown|19|callyeseeseought| +1447|AAAAAAAAHKFAAAAA|1997-10-27||Friends tell. Living times should no|4.43|1.63|7006008|corpbrand #8|6|rugs|7|Home|652|ableantically|N/A|095peach644191870177|orchid|Carton|Unknown|5|ationeseeseought| +1448|AAAAAAAAIKFAAAAA|1997-10-27|2000-10-26|More level stories take ther|4.93|3.89|1004001|edu packamalg #1|4|swimwear|1|Women|243|prieseable|small|17375652229papaya636|slate|Gram|Unknown|32|eingeseeseought| +1449|AAAAAAAAIKFAAAAA|2000-10-27||More level stories take ther|2.16|3.89|1004001|importoimporto #2|2|shirts|2|Men|243|prieseable|economy|99033pink83462019546|blue|Dram|Unknown|51|n steseeseought| +1450|AAAAAAAAKKFAAAAA|1997-10-27|1999-10-27|Bright, long contents must not plant fairly spatial miles. Usually young conditions will abandon overall months. Directors will learn clear|4.21|2.86|5001001|amalgscholar #1|1|rock|5|Music|14|eseought|N/A|31royal9563662327101|purple|Cup|Unknown|30|barantieseought| +1451|AAAAAAAAKKFAAAAA|1999-10-28|2001-10-26|Already advisory things seek more before awkward clothes. Customers offer social, quick needs. Differences mi|9.61|3.74|6010004|univbrand #4|10|jewelry boxes|6|Jewelry|14|eseought|N/A|38orchid783419275552|seashell|Tbl|Unknown|15|oughtantieseought| +1452|AAAAAAAAKKFAAAAA|2001-10-27||Already advisory things seek more before awkward clothes. Customers offer social, quick needs. Differences mi|3.04|3.74|6010004|maxicorp #5|9|womens watch|6|Jewelry|285|antieingable|N/A|38orchid783419275552|gainsboro|Pallet|Unknown|57|ableantieseought| +1453|AAAAAAAANKFAAAAA|1997-10-27||Girls replace far chosen factories; other adults see very with the taxes. Sure short sectors may relinquish so nasty women. More warm me|0.90|0.72|1004002|edu packamalg #2|4|swimwear|1|Women|338|eingpripri|small|426082orange41612943|turquoise|Lb|Unknown|3|priantieseought| +1454|AAAAAAAAOKFAAAAA|1997-10-27|2000-10-26|Extra, new authors picture limited laws. Normal |48.70|25.81|5002001|importoscholar #1|2|country|5|Music|20|barable|N/A|986673989white418751|rose|Tsp|Unknown|16|eseantieseought| +1455|AAAAAAAAOKFAAAAA|2000-10-27||Monetary benefits |5.49|25.81|5002001|amalgexporti #2|1|newborn|3|Children|20|barable|medium|20433slate1959569321|rosy|Dozen|Unknown|33|antiantieseought| +1456|AAAAAAAAALFAAAAA|1997-10-27|1999-10-27|Constant employees interfere from the rooms. Simply small awards would not relocate as well widespread minerals. Old, public schools would|5.85|4.73|8002007|importonameless #7|2|baseball|8|Sports|594|esen stanti|N/A|96pale01870210717232|peach|Case|Unknown|39|callyantieseought| +1457|AAAAAAAAALFAAAAA|1999-10-28|2001-10-26|Constant employees interfere from the rooms. Simply small awards would not relocate as well widespread minerals. Old, public schools would|3.09|4.73|8002007|univmaxi #8|2|pools|8|Sports|594|esen stanti|N/A|96pale01870210717232|orchid|Bundle|Unknown|69|ationantieseought| +1458|AAAAAAAAALFAAAAA|2001-10-27||Systems understand very continental numbers. Never contemporary problems decide too american, existing years. In order pure investors say open, clear voters; totally legal fields mus|7.90|4.81|8002007|namelessunivamalg #14|8|scanners|10|Electronics|594|esen stanti|N/A|96pale01870210717232|steel|Carton|Unknown|91|eingantieseought| +1459|AAAAAAAADLFAAAAA|1997-10-27||Also silent nurses find also fully mental priorities. Savings shall bring naturally silent e|3.04|2.18|8016010|corpmaxi #10|16|golf|8|Sports|328|eingablepri|N/A|05601997aquamarine66|beige|Pallet|Unknown|12|n stantieseought| +1460|AAAAAAAAELFAAAAA|1997-10-27|2000-10-26|Skills|4.18|1.42|6008005|namelesscorp #5|8|mens watch|6|Jewelry|654|eseantically|N/A|9266tomato1592462462|peru|Pound|Unknown|29|barcallyeseought| +1461|AAAAAAAAELFAAAAA|2000-10-27||Skills|2.63|1.42|6008005|exportiedu pack #2|8|kids|4|Shoes|596|eseantically|N/A|9266tomato1592462462|dim|Gram|Unknown|10|oughtcallyeseought| +1462|AAAAAAAAGLFAAAAA|1997-10-27|1999-10-27|Extremely level sources hear; months make less above common materials. Main, unpleasant parts allow workers. Foreign, yellow interests go teeth. Academic yards would not |2.84|1.87|7014003|edu packnameless #3|14|glassware|7|Home|209|n stbarable|N/A|31471white3455164625|linen|Carton|Unknown|18|ablecallyeseought| +1463|AAAAAAAAGLFAAAAA|1999-10-28|2001-10-26|Wrong, available organizations seem; developments may believe true, new countries; of course old profits must compete plainly. Pink proposals will not refuse highly across a arts. Explicitly com|96.82|34.85|3004002|edu packexporti #2|4|school-uniforms|3|Children|209|n stbarable|medium|939113snow3766883316|sky|Ounce|Unknown|47|pricallyeseought| +1464|AAAAAAAAGLFAAAAA|2001-10-27||Components can play oral countries. General feet harm sometimes ladies. Please obvious yards dream just however strong matters. Fresh companies give eyes. Local, still cir|4.25|2.93|3004002|edu packamalg #1|4|swimwear|1|Women|209|n stbarable|small|939113snow3766883316|beige|Box|Unknown|30|esecallyeseought| +1465|AAAAAAAAJLFAAAAA|1997-10-27||Economic, ready words appear more. New, good statements must not save high, emotional notes. Attractive classes carry maybe really unknown electio|8.94|3.48|10001004|amalgunivamalg #4|1|cameras|10|Electronics|57|ationanti|N/A|67716508147572slate4|sandy|Pound|Unknown|12|anticallyeseought| +1466|AAAAAAAAKLFAAAAA|1997-10-27|2000-10-26|Thus effective eyes would not achieve though. Similar folk cover so russian, great police. Again existing sides could not stand more a|0.82|0.50|2001001|amalgimporto #1|1|accessories|2|Men|198|eingn stought|medium|733088snow2483407855|gainsboro|Each|Unknown|6|callycallyeseought| +1467|AAAAAAAAKLFAAAAA|2000-10-27||Annual, pure thoughts cannot understand poor developers. Following things shall advise. Costs may turn thereby social, small years. New unions provide young colours. Important, loca|8.92|5.88|2001001|importomaxi #4|1|business|9|Books|495|antin stese|N/A|545901530435spring77|lime|Bunch|Unknown|56|ationcallyeseought| +1468|AAAAAAAAMLFAAAAA|1997-10-27|1999-10-27|Over national pupils look corporate, chronic children. Notes retain useful eyes. Nations take social rooms. Members find much medical characters. Too royal cases ought to strike also |2.66|1.59|6016005|corpbrand #5|16|consignment|6|Jewelry|272|ableationable|N/A|2peru089996135666209|lawn|Unknown|Unknown|24|eingcallyeseought| +1469|AAAAAAAAMLFAAAAA|1999-10-28|2001-10-26|Boys could save normally |3.63|1.59|6016005|corpunivamalg #8|6|musical|10|Electronics|173|priationought|N/A|312indian75455918488|coral|Oz|Unknown|12|n stcallyeseought| +1470|AAAAAAAAMLFAAAAA|2001-10-27||Balls must speak nearly. Local, new hors|8.18|5.48|6016005|edu packcorp #7|6|bracelets|6|Jewelry|176|priationought|N/A|3252327328111khaki51|white|N/A|Unknown|26|barationeseought| +1471|AAAAAAAAPLFAAAAA|1997-10-27||Discussions act widespread lovers. Easy, environmental subjects join from a years. Other, complete communities may not go in a organs. Special examp|9.40|4.04|6002008|importocorp #8|2|diamonds|6|Jewelry|720|barableation|N/A|32tomato506051992340|puff|Ton|Unknown|21|oughtationeseought| +1472|AAAAAAAAAMFAAAAA|1997-10-27|2000-10-26|High select countries will debate new, able workers. Peaceful, irish signs may come considerably always gentle factor|3.14|2.29|4001001|amalgedu pack #1|1|womens|4|Shoes|727|ationableation|extra large|343470snow4854168147|ivory|Ounce|Unknown|24|ableationeseought| +1473|AAAAAAAAAMFAAAAA|2000-10-27||High select countries will debate new, able workers. Peaceful, irish signs may come considerably always gentle factor|6.35|1.96|4001001|edu packamalg #2|4|swimwear|1|Women|727|ationableation|large|343470snow4854168147|sandy|Gram|Unknown|31|priationeseought| +1474|AAAAAAAACMFAAAAA|1997-10-27|1999-10-27|However remote members talk indeed no longer local costs. Irish plans shou|42.98|22.77|7004005|edu packbrand #5|4|curtains/drapes|7|Home|360|barcallypri|N/A|151383steel011587300|sky|Each|Unknown|3|eseationeseought| +1475|AAAAAAAACMFAAAAA|1999-10-28|2001-10-26|Still special camps progress direc|79.91|22.77|3004002|edu packexporti #2|4|school-uniforms|3|Children|360|barcallypri|large|33941863408471pink21|spring|Unknown|Unknown|82|antiationeseought| +1476|AAAAAAAACMFAAAAA|2001-10-27||Special, available issue|0.14|0.04|3004002|amalgedu pack #1|4|womens|4|Shoes|201|oughtbarable|extra large|6603124royal08255647|wheat|Oz|Unknown|10|callyationeseought| +1477|AAAAAAAAFMFAAAAA|1997-10-27||Normal investments would reduce indeed. Informal, apparent women cannot ensure now seriously structural fears. Previ|7.05|6.27|1004002|edu packamalg #2|4|swimwear|1|Women|131|oughtpriought|economy|37581saddle690296243|lawn|Gram|Unknown|31|ationationeseought| +1478|AAAAAAAAGMFAAAAA|1997-10-27|2000-10-26|Fields generate. Universities get honest, fixed locations. Possible requirements might not see ideas. Communications visit continuous others. Stor|1.76|1.53|8010009|univmaxi #9|10|pools|8|Sports|904|esebarn st|N/A|01434pink97432939029|salmon|Bunch|Unknown|42|eingationeseought| +1479|AAAAAAAAGMFAAAAA|2000-10-27||Fields generate. Universities get honest, fixed locations. Possible requirements might not see ideas. Communications visit continuous others. Stor|67.79|27.79|2004002|edu packimporto #2|4|sports-apparel|2|Men|904|esebarn st|petite|01434pink97432939029|dim|Tbl|Unknown|41|n stationeseought| +1480|AAAAAAAAIMFAAAAA|1997-10-27|1999-10-27|Wide, technical paren|6.64|3.98|3003001|exportiexporti #1|3|toddlers|3|Children|115|antioughtought|medium|rose8201652246286893|tan|Ton|Unknown|37|bareingeseought| +1481|AAAAAAAAIMFAAAAA|1999-10-28|2001-10-26|Wide, technical paren|1.91|3.98|3003001|corpbrand #2|3|rugs|7|Home|22|antioughtought|N/A|2564023015steel05932|purple|Bunch|Unknown|55|oughteingeseought| +1482|AAAAAAAAIMFAAAAA|2001-10-27||Wide, technical paren|2.79|3.98|10001004|amalgunivamalg #4|3|cameras|10|Electronics|22|ableable|N/A|618157580929486rosy4|sky|N/A|Unknown|68|ableeingeseought| +1483|AAAAAAAALMFAAAAA|1997-10-27|||0.74|0.56||exportischolar #2|3||5||||N/A|6966812883675906red9|||||prieingeseought| +1484|AAAAAAAAMMFAAAAA|1997-10-27|2000-10-26|Easily major passages should feel here in a events; far european police should not die sometimes local possible systems. Off|4.17|3.29|2004001|edu packimporto #1|4|sports-apparel|2|Men|315|antioughtpri|extra large|2899148133olive24860|rosy|Oz|Unknown|54|eseeingeseought| +1485|AAAAAAAAMMFAAAAA|2000-10-27||Easily major passages should feel here in a events; far european police should not die sometimes local possible systems. Off|4.07|3.29|2004001|amalgscholar #2|1|rock|5|Music|322|ableablepri|N/A|2899148133olive24860|tomato|Carton|Unknown|42|antieingeseought| +1486|AAAAAAAAOMFAAAAA|1997-10-27|1999-10-27|Now legal ministers disarm words. Users used to leave yesterday new windows. High other windows stick then. Light, big crews cause further. Very |3.24|1.23|5002001|importoscholar #1|2|country|5|Music|622|ableablecally|N/A|31944sienna335869990|sienna|Dram|Unknown|28|callyeingeseought| +1487|AAAAAAAAOMFAAAAA|1999-10-28|2001-10-26|Otherwise relevant bars stay animals. Identical, familiar methods keep more by a pages. Interpretations used to find with the homes. Young miles may not raise windows. Tenants shall not turn men. E|0.49|1.23|6007008|brandcorp #8|7|pendants|6|Jewelry|622|ableablecally|N/A|60slate2797310898856|orange|Carton|Unknown|13|ationeingeseought| +1488|AAAAAAAAOMFAAAAA|2001-10-27||Otherwise relevant bars stay animals. Identical, familiar methods keep more by a pages. Interpretations used to find with the homes. Young miles may not raise windows. Tenants shall not turn men. E|0.61|1.23|9005001|scholarmaxi #1|7|history|9|Books|622|ableablecally|N/A|60slate2797310898856|tomato|Case|Unknown|21|eingeingeseought| +1489|AAAAAAAABNFAAAAA|1997-10-27||Dominant policies will not continue territories. Entire services show already always national waters. Children might make still foreign, grey enterprises. Ancient factors must not mention p|1.97|1.02|4003002|exportiedu pack #2|3|kids|4|Shoes|611|oughtoughtcally|extra large|55wheat4796153860912|salmon|Pound|Unknown|49|n steingeseought| +1490|AAAAAAAACNFAAAAA|1997-10-27|2000-10-26|New, good results feel new programmes. Accounts should come for a words. Now important groups might ask hard v|1.00|0.34|10013012|exportiamalgamalg #12|13|stereo|10|Electronics|83|prieing|N/A|2689050699olive72910|royal|N/A|Unknown|76|barn steseought| +1491|AAAAAAAACNFAAAAA|2000-10-27||Only delicate times alter both rights. Tha|8.25|5.28|7002002|importobrand #2|2|bedding|7|Home|125|antiableought|N/A|352gainsboro40619495|aquamarine|Ounce|Unknown|27|oughtn steseought| +1492|AAAAAAAAENFAAAAA|1997-10-27|1999-10-27|Outstanding shows would produce all english hearts; deep, strange relations will help bars. At last available operations should not dry long alternative gl|1.51|1.23|9009005|maximaxi #5|9|science|9|Books|42|ableese|N/A|80308718royal2060432|seashell|Cup|Unknown|95|ablen steseought| +1493|AAAAAAAAENFAAAAA|1999-10-28|2001-10-26|Outstanding shows would produce all english hearts; deep, strange relations will help bars. At last available operations should not dry long alternative gl|3.49|1.23|5002002|importoscholar #2|2|country|5|Music|42|ableese|N/A|violet45652855370637|sandy|Case|Unknown|51|prin steseought| +1494|AAAAAAAAENFAAAAA|2001-10-27||Outstanding shows would produce all english hearts; deep, strange relations will help bars. At last available operations should not dry long alternative gl|6.37|5.03|5002002|exportiexporti #1|3|toddlers|3|Children|42|ableese|large|violet45652855370637|tan|Tsp|Unknown|65|esen steseought| +1495|AAAAAAAAHNFAAAAA|1997-10-27||Important, small areas will play often. Courses|7.85|6.43|5001002|amalgscholar #2|1|rock|5|Music|480|bareingese|N/A|40713755509slate1288|papaya|Cup|Unknown|75|antin steseought| +1496|AAAAAAAAINFAAAAA|1997-10-27|2000-10-26|Much trying boys play really seconds. Clear cases cannot stop only so social types. Areas see |5.48|2.30|7004007|edu packbrand #7|4|curtains/drapes|7|Home|248|eingeseable|N/A|904372018pink8413768|misty|Pound|Unknown|12|callyn steseought| +1497|AAAAAAAAINFAAAAA|2000-10-27||Sen|9.83|2.94|7004007|importomaxi #10|2|business|9|Books|248|eingeseable|N/A|268pale5014012503676|plum|Tbl|Unknown|55|ationn steseought| +1498|AAAAAAAAKNFAAAAA|1997-10-27|1999-10-27|Good, clear years absorb both short, ready feet. Federal, exotic novels lead then new pi|5.90|4.72|1002001|importoamalg #1|2|fragrances|1|Women|484|eseeingese|petite|35367midnight7535957|sandy|Pallet|Unknown|75|eingn steseought| +1499|AAAAAAAAKNFAAAAA|1999-10-28|2001-10-26|Therefore public attacks used to act approximately single subsequent hours. Valuable, wooden problems may not say |4.95|2.27|1002001|amalgimporto #2|1|accessories|2|Men|90|eseeingese|medium|3406397898plum936720|sky|Ton|Unknown|53|n stn steseought| +1500|AAAAAAAAKNFAAAAA|2001-10-27||Poor eyes form again proper, eligible prisoners. Areas must not|8.15|2.27|9003001|exportimaxi #1|3|computers|9|Books|90|eseeingese|N/A|6982732orchid0796175|white|Carton|Unknown|54|barbarantiought| +1501|AAAAAAAANNFAAAAA|1997-10-27||Old males bring directly keen settlements. Late plants may defend apart social, necessary chips. Beds make soon carefully small measures; clear cases must not say more pos|7.40|3.55|5003002|exportischolar #2|3|pop|5|Music|4|ese|N/A|8875890008blanched10|magenta|Carton|Unknown|6|oughtbarantiought| +1502|AAAAAAAAONFAAAAA|1997-10-27|2000-10-26|Appropriate categories know at least as good concepts. Dead days should praise in order wide|5.70|4.44|6007007|brandcorp #7|7|pendants|6|Jewelry|513|prioughtanti|N/A|47967133139rose58993|rosy|Carton|Unknown|92|ablebarantiought| +1503|AAAAAAAAONFAAAAA|2000-10-27||Large sons would remain main, insufficient designs. Effects celebrate serious details. Now foreign paintings give very sim|8.22|4.44|6007007|edu packedu pack #2|7|athletic|4|Shoes|152|ableantiought|medium|383588peach771057105|misty|Lb|Unknown|33|pribarantiought| +1504|AAAAAAAAAOFAAAAA|1997-10-27|1999-10-27|Great, contemp|4.31|2.15|1003001|exportiamalg #1|3|maternity|1|Women|639|n stprically|extra large|spring23133133495598|salmon|Cup|Unknown|21|esebarantiought| +1505|AAAAAAAAAOFAAAAA|1999-10-28|2001-10-26|Great, contemp|2.66|2.15|1003001|brandcorp #2|7|pendants|6|Jewelry|639|n stprically|N/A|spring23133133495598|peach|Dozen|Unknown|17|antibarantiought| +1506|AAAAAAAAAOFAAAAA|2001-10-27||Home black designs find names. Occup|40.10|2.15|1003001|exportiamalg #1|7|maternity|1|Women|639|n stprically|economy|spring23133133495598|snow|Dozen|Unknown|18|callybarantiought| +1507|AAAAAAAADOFAAAAA|1997-10-27||Both early efforts must dispose simply on a men. Real workshops say properly from a possibiliti|2.08|1.76|7005008|scholarbrand #8|5|blinds/shades|7|Home|276|callyationable|N/A|48889053salmon751057|papaya|Tsp|Unknown|51|ationbarantiought| +1508|AAAAAAAAEOFAAAAA|1997-10-27|2000-10-26|Economic, littl|66.66|20.66|1002001|importoamalg #1|2|fragrances|1|Women|546|callyeseanti|small|52peru34215939211716|royal|Ounce|Unknown|61|eingbarantiought| +1509|AAAAAAAAEOFAAAAA|2000-10-27||Economic, littl|6.15|20.66|1002001|edu packmaxi #10|4|entertainments|9|Books|546|callyeseanti|N/A|52peru34215939211716|steel|Each|Unknown|39|n stbarantiought| +1510|AAAAAAAAGOFAAAAA|1997-10-27|1999-10-27|Bre|8.21|3.69|5004001|edu packscholar #1|4|classical|5|Music|352|ableantipri|N/A|smoke471406641445203|saddle|Dram|Unknown|23|baroughtantiought| +1511|AAAAAAAAGOFAAAAA|1999-10-28|2001-10-26|Bre|4.89|3.69|9010012|univunivamalg #12|4|travel|9|Books|289|n steingable|N/A|9antique449872239637|tan|Box|Unknown|38|oughtoughtantiought| +1512|AAAAAAAAGOFAAAAA|2001-10-27||Bre|3.23|0.96|9010012|corpunivamalg #17|4|musical|10|Electronics|289|n steingable|N/A|9antique449872239637|smoke|Bundle|Unknown|33|ableoughtantiought| +1513|AAAAAAAAJOFAAAAA|1997-10-27||Grateful rates turn social, young answers. Visual miles |5.05|1.51|5004002|edu packscholar #2|4|classical|5|Music|111|oughtoughtought|N/A|20thistle13738845401|rose|Ton|Unknown|42|prioughtantiought| +1514|AAAAAAAAKOFAAAAA|1997-10-27|2000-10-26|Casual, original arms should work dramatically in a scholars; classes oug|1.96|1.25|1002001|importoamalg #1|2|fragrances|1|Women|860|barcallyeing|extra large|7199719618snow379055|sky|Oz|Unknown|28|eseoughtantiought| +1515|AAAAAAAAKOFAAAAA|2000-10-27||Natural, open nurs|4.74|1.25|1003002|exportiamalg #2|3|maternity|1|Women|461|oughtcallyese|large|7199719618snow379055|purple|Bundle|Unknown|61|antioughtantiought| +1516|AAAAAAAAMOFAAAAA|1997-10-27|1999-10-27|Political shares become then firmly english men. Hardly young police |1.89|0.90|7007007|brandbrand #7|7|decor|7|Home|158|eingantiought|N/A|tan40004597485688454|sky|Cup|Unknown|63|callyoughtantiought| +1517|AAAAAAAAMOFAAAAA|1999-10-28|2001-10-26|Political shares become then firmly english men. Hardly young police |85.05|0.90|3002002|importoexporti #2|2|infants|3|Children|158|eingantiought|medium|47465midnight3847213|gainsboro|Dozen|Unknown|68|ationoughtantiought| +1518|AAAAAAAAMOFAAAAA|2001-10-27||Downstairs certain forces live already. Even unlikely questions use even years. Middle customers argue impressive, good years. Enthusiastic studies ought to take afterwards clear pos|3.16|1.86|2003001|exportiimporto #1|2|pants|2|Men|158|eingantiought|small|893031orchid43362086|snow|Ounce|Unknown|20|eingoughtantiought| +1519|AAAAAAAAPOFAAAAA|1997-10-27||Brothers see bodies. Social, military emotions might know well still able officials. Ga|7.61|2.66|5002002|importoscholar #2|2|country|5|Music|498|eingn stese|N/A|47797949purple559501|plum|Tsp|Unknown|51|n stoughtantiought| +1520|AAAAAAAAAPFAAAAA|1997-10-27|2000-10-26|Similar, other creatures hold ultimate, theoretical expenses. English difficulties should turn only completely pos|4.23|3.46|10011008|amalgamalgamalg #8|11|disk drives|10|Electronics|118|eingoughtought|N/A|32822lavender7481969|khaki|Pallet|Unknown|13|barableantiought| +1521|AAAAAAAAAPFAAAAA|2000-10-27||There independent parts would determine so with a symptoms. Friends see also. Distant, right years would try nationally less obvious resources. New, effective dem|70.80|3.46|7002002|importobrand #2|11|bedding|7|Home|232|ablepriable|N/A|32822lavender7481969|powder|Ounce|Unknown|11|oughtableantiought| +1522|AAAAAAAACPFAAAAA|1997-10-27|1999-10-27|Days would not raise just similar governments; components need more large, other rises. Authorities produce. Scottish, available eggs help mere hundreds. Grim winners help private, good|0.10|0.08|10012010|importoamalgamalg #10|12|monitors|10|Electronics|140|bareseought|N/A|2431301678008medium0|white|Cup|Unknown|29|ableableantiought| +1523|AAAAAAAACPFAAAAA|1999-10-28|2001-10-26|Annual demands win ever for a things. Scientific, good policies shall think less dead, new instances. Youn|3.58|0.08|10012010|exportiimporto #2|12|pants|2|Men|140|bareseought|small|2431301678008medium0|violet|Ton|Unknown|27|priableantiought| +1524|AAAAAAAACPFAAAAA|2001-10-27||Annual demands win ever for a things. Scientific, good policies shall think less dead, new instances. Youn|6.13|0.08|10012010|corpunivamalg #1|12|mystery|9|Books|267|bareseought|N/A|2431301678008medium0|lawn|Dram|Unknown|12|eseableantiought| +1525|AAAAAAAAFPFAAAAA|1997-10-27||Questions used to look social technologies. As high women get indoors spec|4.01|1.96|8005006|scholarnameless #6|5|fitness|8|Sports|182|ableeingought|N/A|044290turquoise08999|sienna|Dozen|Unknown|4|antiableantiought| +1526|AAAAAAAAGPFAAAAA|1997-10-27|2000-10-26|Small, bare solicitors may take for |3.20|1.28|7011007|amalgnameless #7|11|accent|7|Home|444|eseeseese|N/A|63965128yellow998891|salmon|Lb|Unknown|26|callyableantiought| +1527|AAAAAAAAGPFAAAAA|2000-10-27||More aware responses find|2.96|1.28|7011007|univmaxi #8|11|pools|8|Sports|444|eseeseese|N/A|63965128yellow998891|steel|Dram|Unknown|54|ationableantiought| +1528|AAAAAAAAIPFAAAAA|1997-10-27|1999-10-27|Extra, public branches shall list rather international police. Little v|2.51|2.15|9009005|maximaxi #5|9|science|9|Books|318|eingoughtpri|N/A|631saddle29476964777|gainsboro|Case|Unknown|23|eingableantiought| +1529|AAAAAAAAIPFAAAAA|1999-10-28|2001-10-26|Only international officers operate as deep, local po|15.09|5.13|1004002|edu packamalg #2|9|swimwear|1|Women|633|eingoughtpri|small|0082620674smoke74950|white|Pound|Unknown|66|n stableantiought| +1530|AAAAAAAAIPFAAAAA|2001-10-27||Studies suggest families; small minutes would bring much. |1.60|5.13|1004002|importoscholar #1|2|country|5|Music|633|priprically|N/A|0846968sandy69790178|plum|Pound|Unknown|35|barpriantiought| +1531|AAAAAAAALPFAAAAA|1997-10-27||Engines create enough; enough pure companies will follow up to through a miles; similar, united things sho|2.20|0.68|3001002|amalgexporti #2|1|newborn|3|Children|372|ableationpri|large|87wheat5346477249087|seashell|Cup|Unknown|23|oughtpriantiought| +1532|AAAAAAAAMPFAAAAA|1997-10-27|2000-10-26|Clear artists stay so that is limited causes; innocent, unusual claims make to a horses. Concerns will see almost in a centres. Seriously great maste|79.19|45.13|7002003|importobrand #3|2|bedding|7|Home|647|ationesecally|N/A|007925136557626peru8|pale|Cup|Unknown|48|ablepriantiought| +1533|AAAAAAAAMPFAAAAA|2000-10-27||Provin|2.12|45.13|2004002|edu packimporto #2|2|sports-apparel|2|Men|545|antieseanti|N/A|4527983700sandy77462|smoke|Gross|Unknown|44|pripriantiought| +1534|AAAAAAAAOPFAAAAA|1997-10-27|1999-10-27|Enough labour days watch to a shops. Residents sharpen now scottish, complete expressions; time and again painful others shall not reduce for a enemies. Images visit bef|4.92|3.24|7004005|edu packbrand #5|4|curtains/drapes|7|Home|185|antieingought|N/A|42powder596420491417|hot|Box|Unknown|13|esepriantiought| +1535|AAAAAAAAOPFAAAAA|1999-10-28|2001-10-26|Home serious women think to a investors. Red feelings hear only acceptable stages. Other, ancient others m|7.45|3.94|8012006|importomaxi #6|4|guns|8|Sports|326|antieingought|N/A|42powder596420491417|sky|Oz|Unknown|91|antipriantiought| +1536|AAAAAAAAOPFAAAAA|2001-10-27||Syst|1.65|1.07|4001001|amalgedu pack #1|4|womens|4|Shoes|27|ationable|small|809219398798tan61983|gainsboro|Unknown|Unknown|25|callypriantiought| +1537|AAAAAAAABAGAAAAA|1997-10-27||Necessarily basic organisati|9.11|2.91|6015002|scholarbrand #2|15|custom|6|Jewelry|21|oughtable|N/A|1378spring2071448641|smoke|Unknown|Unknown|7|ationpriantiought| +1538|AAAAAAAACAGAAAAA|1997-10-27|2000-10-26|Customs should build slight schools. Dangers find loyal computers. Other holidays learn ago only human jobs. Special, scottish women|39.43|35.09|10004009|edu packunivamalg #9|4|audio|10|Electronics|559|n stantianti|N/A|82095plum12286877177|honeydew|Cup|Unknown|14|eingpriantiought| +1539|AAAAAAAACAGAAAAA|2000-10-27||Customs should build slight schools. Dangers find loyal computers. Other holidays learn ago only human jobs. Special, scottish women|9.23|35.09|10004009|edu packamalg #2|4|swimwear|1|Women|217|n stantianti|large|1036541630190puff795|red|Each|Unknown|14|n stpriantiought| +1540|AAAAAAAAEAGAAAAA|1997-10-27|1999-10-27|Dynamic, available memories may go abstract years; presumably high members stay faster industries. Offices give thus. Carers ought to pay well fields. Obvious|9.45|8.41|7011001|amalgnameless #1|11|accent|7|Home|495|antin stese|N/A|3368690989719violet6|rosy|Gross|Unknown|45|bareseantiought| +1541|AAAAAAAAEAGAAAAA|1999-10-28|2001-10-26|Dynamic, available memories may go abstract years; presumably high members stay faster industries. Offices give thus. Carers ought to pay well fields. Obvious|8.91|4.27|7011001|brandnameless #2|11|hockey|8|Sports|495|antin stese|N/A|chocolate57072260785|lemon|Dram|Unknown|17|oughteseantiought| +1542|AAAAAAAAEAGAAAAA|2001-10-27||Dynamic, available memories may go abstract years; presumably high members stay faster industries. Offices give thus. Carers ought to pay well fields. Obvious|24.19|4.27|7011001|namelesscorp #7|11|mens watch|6|Jewelry|37|ationpri|N/A|chocolate57072260785|medium|Case|Unknown|13|ableeseantiought| +1543|AAAAAAAAHAGAAAAA|1997-10-27||Black, successive interactions u|2.75|1.70|7009004|maxibrand #4|9|mattresses|7|Home|171|oughtationought|N/A|75640powder037280220|salmon|Gross|Unknown|57|prieseantiought| +1544|AAAAAAAAIAGAAAAA|1997-10-27|2000-10-26|Social wor|0.79|0.66|7004005|edu packbrand #5|4|curtains/drapes|7|Home|20|barable|N/A|9351322037forest7961|saddle|Bunch|Unknown|16|eseeseantiought| +1545|AAAAAAAAIAGAAAAA|2000-10-27||Natural, electoral savings may not welcome at least other, long officials. Different, advanced lips should buy. Policies should |5.96|2.20|7004005|scholarmaxi #10|4|history|9|Books|156|callyantiought|N/A|30418178white7769486|snow|Gross|Unknown|36|antieseantiought| +1546|AAAAAAAAKAGAAAAA|1997-10-27|1999-10-27|Trees could conceive overall liberal mechanisms. Full, firm examples may point; deliberately new pensions leave to a games. Economic homes grant almost services; plans might not c|4.52|2.80|10005017|scholarunivamalg #17|5|karoke|10|Electronics|7|ation|N/A|500447turquoise57613|gainsboro|Box|Unknown|4|callyeseantiought| +1547|AAAAAAAAKAGAAAAA|1999-10-28|2001-10-26|Pink, important thousands think well lines. Near special children agree all. Political things might not result only external fees. Yesterday p|6.76|2.80|10005017|exportiexporti #2|5|toddlers|3|Children|7|ation|large|500447turquoise57613|mint|Oz|Unknown|41|ationeseantiought| +1548|AAAAAAAAKAGAAAAA|2001-10-27||Pink, important thousands think well lines. Near special children agree all. Political things might not result only external fees. Yesterday p|4.01|2.80|10005017|exportiunivamalg #2|5|dvd/vcr players|10|Electronics|278|eingationable|N/A|500447turquoise57613|sky|Tsp|Unknown|15|eingeseantiought| +1549|AAAAAAAANAGAAAAA|1997-10-27||Managers may not come slightly possible occasions; naked, organisational goods could pull. Things put much little, experimental mistakes. Healthy, cruel hours acknowledge red doubts. Citie|7.24|3.18|7012010|importonameless #10|12|paint|7|Home|300|barbarpri|N/A|2652551lawn918126128|wheat|Ounce|Unknown|23|n steseantiought| +1550|AAAAAAAAOAGAAAAA|1997-10-27|2000-10-26|Very poor powers can cry vessels. Clear|8.95|3.58|3004001|edu packexporti #1|4|school-uniforms|3|Children|40|barese|medium|786red65437311056156|saddle|Ounce|Unknown|56|barantiantiought| +1551|AAAAAAAAOAGAAAAA|2000-10-27||Very poor powers can cry vessels. Clear|4.70|1.64|3004001|edu packimporto #2|4|sports-apparel|2|Men|40|barese|large|786red65437311056156|seashell|Case|Unknown|10|oughtantiantiought| +1552|AAAAAAAAABGAAAAA|1997-10-27|1999-10-27|Otherwise suitable products consider too technical techniques; common women spend quickly assessments; chemical habits develop more. Very universal processes determine gingerly; months may discover mo|4.64|3.15|7016003|corpnameless #3|16|furniture|7|Home|873|priationeing|N/A|21sienna704937286191|metallic|Ton|Unknown|55|ableantiantiought| +1553|AAAAAAAAABGAAAAA|1999-10-28|2001-10-26|Here alternative changes may produce maybe involved sums; purely wide groups take then. Civil poets may change as by a elections. Less pub|4.63|1.48|10009007|maxiunivamalg #7|9|televisions|10|Electronics|778|priationeing|N/A|444965716spring12143|royal|Unknown|Unknown|76|priantiantiought| +1554|AAAAAAAAABGAAAAA|2001-10-27||So correct thousands possess atti|2.16|1.66|10004008|edu packunivamalg #8|9|audio|10|Electronics|778|priationeing|N/A|4183156plum809935342|steel|Box|Unknown|5|eseantiantiought| +1555|AAAAAAAADBGAAAAA|1997-10-27||Useful, poor keys can make on a matters. Favorite, other degrees know here other lights. Intellec|4.32|3.15|8002006|importonameless #6|2|baseball|8|Sports|643|priesecally|N/A|932079081034pale4169|sky|Unknown|Unknown|5|antiantiantiought| +1556|AAAAAAAAEBGAAAAA|1997-10-27|2000-10-26|Police should not expect material, acceptable shares. Houses should not hold alread|6.97|5.22|7010007|univnameless #7|10|flatware|7|Home|111|oughtoughtought|N/A|4254145768purple0607|pink|Dram|Unknown|21|callyantiantiought| +1557|AAAAAAAAEBGAAAAA|2000-10-27||Police should not expect material, acceptable shares. Houses should not hold alread|4.53|2.49|5002002|importoscholar #2|2|country|5|Music|627|oughtoughtought|N/A|1190royal20948234577|maroon|Gross|Unknown|44|ationantiantiought| +1558|AAAAAAAAGBGAAAAA|1997-10-27|1999-10-27|Then social minutes will think nuclear, great contents. Opening men find else able amounts. National, diffe|2.47|2.19|6010007|univbrand #7|10|jewelry boxes|6|Jewelry|380|bareingpri|N/A|73lemon8937513864550|sky|Case|Unknown|40|eingantiantiought| +1559|AAAAAAAAGBGAAAAA|1999-10-28|2001-10-26|Companies improve international, male details; presidential members subscribe together perfect constraints. More distinct policies carry difficult grounds. Inwards beneficial element|6.78|2.19|6010007|importoamalg #2|10|fragrances|1|Women|380|bareingpri|small|653395868922245navy1|steel|Dozen|Unknown|35|n stantiantiought| +1560|AAAAAAAAGBGAAAAA|2001-10-27||Certain, personal writers must work fully other employers; blue, similar books claim on an nights. Yet p|1.03|0.78|4002001|importoedu pack #1|10|mens|4|Shoes|380|bareingpri|economy|9722202146spring8178|tomato|Ounce|Unknown|44|barcallyantiought| +1561|AAAAAAAAJBGAAAAA|1997-10-27||Reasonable authors cannot go relatively educational arms. Other, italian ships will not stop increasingly pp.. So civil employers |2.20|1.69|1001002|amalgamalg #2|1|dresses|1|Women|222|ableableable|medium|85peru78393856207553|sienna|Unknown|Unknown|82|oughtcallyantiought| +1562|AAAAAAAAKBGAAAAA|1997-10-27|2000-10-26|Also white pounds ought to join loc|0.79|0.70|6014003|edu packbrand #3|14|estate|6|Jewelry|275|antiationable|N/A|89377148salmon742641|papaya|Bundle|Unknown|40|ablecallyantiought| +1563|AAAAAAAAKBGAAAAA|2000-10-27||Also white pounds ought to join loc|1.74|1.42|3001002|amalgexporti #2|1|newborn|3|Children|275|antiationable|medium|89377148salmon742641|honeydew|Lb|Unknown|13|pricallyantiought| +1564|AAAAAAAAMBGAAAAA|1997-10-27|1999-10-27|Regulatory, financial words would obtain yet at a relatives. Also main lines should bel|3.33|2.09|9012005|importounivamalg #5|12|home repair|9|Books|62|ablecally|N/A|413196521996lemon997|honeydew|Unknown|Unknown|12|esecallyantiought| +1565|AAAAAAAAMBGAAAAA|1999-10-28|2001-10-26|Important studies would not sell far. Passive, usual prices ought to take consequently little functions; home similar provisions shall pass. St|60.25|2.09|8013006|exportimaxi #6|13|sailing|8|Sports|62|ablecally|N/A|93olive7622756127946|slate|Dram|Unknown|28|anticallyantiought| +1566|AAAAAAAAMBGAAAAA|2001-10-27||Important studies would not sell far. Passive, usual prices ought to take consequently little functions; home similar provisions shall pass. St|1.15|2.09|8013006|amalgbrand #7|13|semi-precious|6|Jewelry|863|pricallyeing|N/A|954683302tomato09848|tomato|N/A|Unknown|20|callycallyantiought| +1567|AAAAAAAAPBGAAAAA|1997-10-27||Individuals will think really recent minutes. Rightly political problems may not consider|0.58|0.33|8002008|importonameless #8|2|baseball|8|Sports|368|eingcallypri|N/A|39861tomato940373857|papaya|Tbl|Unknown|48|ationcallyantiought| +1568|AAAAAAAAACGAAAAA|1997-10-27|2000-10-26|High, jewish legs should not direct existing, current points. Opposite effects make women. Organisers suggest mo|6.86|2.33|2001001|amalgimporto #1|1|accessories|2|Men|732|ablepriation|small|82thistle00312632988|saddle|Bundle|Unknown|28|eingcallyantiought| +1569|AAAAAAAAACGAAAAA|2000-10-27||High, jewish legs should not direct existing, current points. Opposite effects make women. Organisers suggest mo|2.17|1.77|6010002|univbrand #2|1|jewelry boxes|6|Jewelry|25|ablepriation|N/A|82thistle00312632988|purple|N/A|Unknown|53|n stcallyantiought| +1570|AAAAAAAACCGAAAAA|1997-10-27|1999-10-27|Large assets trust even; individuals record formal, short t|7.78|3.96|8009001|maxinameless #1|9|optics|8|Sports|737|ationpriation|N/A|1987845pink883000983|peru|Bundle|Unknown|4|barationantiought| +1571|AAAAAAAACCGAAAAA|1999-10-28|2001-10-26|Large assets trust even; individuals record formal, short t|5.61|3.96|1003002|exportiamalg #2|9|maternity|1|Women|737|ationpriation|medium|317791midnight560713|lawn|Tbl|Unknown|21|oughtationantiought| +1572|AAAAAAAACCGAAAAA|2001-10-27||Large assets trust even; individuals record formal, short t|81.03|48.61|10011009|amalgamalgamalg #9|11|disk drives|10|Electronics|886|callyeingeing|N/A|317791midnight560713|royal|Carton|Unknown|9|ableationantiought| +1573|AAAAAAAAFCGAAAAA|1997-10-27||Absolutely|1.40|0.78|10016010|corpamalgamalg #10|16|wireless|10|Electronics|334|esepripri|N/A|02936023257maroon394|burnished|Tbl|Unknown|19|priationantiought| +1574|AAAAAAAAGCGAAAAA|1997-10-27|2000-10-26|Moments incur pa|6.14|1.84|2003001|exportiimporto #1|3|pants|2|Men|775|antiationation|small|974264772669steel864|violet|Oz|Unknown|73|eseationantiought| +1575|AAAAAAAAGCGAAAAA|2000-10-27||Other, high symptoms can write. Current, cold men tighten almost very industrial spirits. Main libraries enable la|4.43|3.63|5003002|exportischolar #2|3|pop|5|Music|532|antiationation|N/A|974264772669steel864|snow|Ton|Unknown|20|antiationantiought| +1576|AAAAAAAAICGAAAAA|1997-10-27|1999-10-27|Pretty, established folk continue sheets. Labour, soviet thanks might talk here managerial women. Increasing, alternative costs stop. Never absolute components used to like reall|4.48|3.80|10003013|exportiunivamalg #13|3|dvd/vcr players|10|Electronics|201|oughtbarable|N/A|3955002934040saddle7|turquoise|Bundle|Unknown|5|callyationantiought| +1577|AAAAAAAAICGAAAAA|1999-10-28|2001-10-26|Pretty, established folk continue sheets. Labour, soviet thanks might talk here managerial women. Increasing, alternative costs stop. Never absolute components used to like reall|2.58|2.08|9008006|namelessmaxi #6|8|romance|9|Books|198|oughtbarable|N/A|3955002934040saddle7|tan|Oz|Unknown|2|ationationantiought| +1578|AAAAAAAAICGAAAAA|2001-10-27||English hands might finance only months. Full, similar players used to result and so on irrelevant hours. Schools cannot purchase about now local bands; foreign bands make only for the other|2.26|2.08|9008006|edu packamalgamalg #15|14|automotive|10|Electronics|198|eingn stought|N/A|3955002934040saddle7|mint|Bundle|Unknown|52|eingationantiought| +1579|AAAAAAAALCGAAAAA|1997-10-27||R|3.18|2.41|7008010|namelessbrand #10|8|lighting|7|Home|406|callybarese|N/A|72royal2850804005125|misty|Dram|Unknown|3|n stationantiought| +1580|AAAAAAAAMCGAAAAA|1997-10-27|2000-10-26|In order african parties shall work already schools. Only political matters pretend most high applications; |4.07|2.36|3003001|exportiexporti #1|3|toddlers|3|Children|96|callyn st|medium|41681222olive4963389|seashell|Pound|Unknown|33|bareingantiought| +1581|AAAAAAAAMCGAAAAA|2000-10-27||In order african parties shall work already schools. Only political matters pretend most high applications; |46.35|2.36|3003001|scholarmaxi #2|3|fishing|8|Sports|96|callyn st|N/A|41681222olive4963389|orange|Bundle|Unknown|93|oughteingantiought| +1582|AAAAAAAAOCGAAAAA|1997-10-27|1999-10-27|Well different problems must not disrupt|8.69|4.43|9015011|scholarunivamalg #11|15|fiction|9|Books|53|prianti|N/A|881955749710734rose9|grey|Bundle|Unknown|47|ableeingantiought| +1583|AAAAAAAAOCGAAAAA|1999-10-28|2001-10-26|Comprehensive levels construct different instruments. Dangerous, important attempts listen however at the pensions. Old periods sh|0.34|0.27|9012012|importounivamalg #12|15|home repair|9|Books|268|eingcallyable|N/A|881955749710734rose9|tan|Ounce|Unknown|25|prieingantiought| +1584|AAAAAAAAOCGAAAAA|2001-10-27||Accurate deaths might starve today rigid comparisons. Prime, new studies react periods. Eyes|2.13|0.27|9012012|exportischolar #1|15|pop|5|Music|268|eingcallyable|N/A|881955749710734rose9|light|Pallet|Unknown|2|eseeingantiought| +1585|AAAAAAAABDGAAAAA|1997-10-27||Blue, financial opportunities could hope social humans. Lights must vote states. Then new companies make important, a|4.83|3.38|7012006|importonameless #6|12|paint|7|Home|88|eingeing|N/A|0536spring6927797121|olive|Bunch|Unknown|15|antieingantiought| +1586|AAAAAAAACDGAAAAA|1997-10-27|2000-10-26|Great wishes could discuss similar women. Here american jobs see choices; western, artistic|3.45|2.44|5002001|importoscholar #1|2|country|5|Music|601|oughtbarcally|N/A|129138058173591plum4|olive|Carton|Unknown|19|callyeingantiought| +1587|AAAAAAAACDGAAAAA|2000-10-27||Finally effective comments will not laugh over. Others will go legal chairs. Good dreams should not negotiate ever assets. Traditionally complete |3.62|2.44|3003002|exportiexporti #2|3|toddlers|3|Children|601|oughtbarcally|petite|624434602049786tan73|steel|Unknown|Unknown|12|ationeingantiought| +1588|AAAAAAAAEDGAAAAA|1997-10-27|1999-10-27|Only expected governments used to describe; institutions can make bad, industrial years. Decidedly basic enemies must not send shortly maybe like reports; clearly free systems used to order ital|2.45|1.34|8015009|scholarmaxi #9|15|fishing|8|Sports|854|eseantieing|N/A|082white520591595570|peach|Carton|Unknown|33|eingeingantiought| +1589|AAAAAAAAEDGAAAAA|1999-10-28|2001-10-26|Only expected governments used to describe; institutions can make bad, industrial years. Decidedly basic enemies must not send shortly maybe like reports; clearly free systems used to order ital|0.90|1.34|5001002|amalgscholar #2|1|rock|5|Music|38|eseantieing|N/A|082white520591595570|spring|Tsp|Unknown|24|n steingantiought| +1590|AAAAAAAAEDGAAAAA|2001-10-27||Most growing offers grasp thirdly appropriate looks; new women bother about big cards. Acute members may announce events. Already possible streets collaborate later lively circumstance|9.53|1.34|3002001|importoexporti #1|1|infants|3|Children|301|eseantieing|medium|082white520591595570|peach|N/A|Unknown|15|barn stantiought| +1591|AAAAAAAAHDGAAAAA|1997-10-27||Factors will not wish frequently; tables might notice already radically correct sales. Just different cards command high for the women. Concerns may want a|2.65|1.37|6001008|amalgcorp #8|1|birdal|6|Jewelry|114|eseoughtought|N/A|19934464243puff47827|hot|Case|Unknown|53|oughtn stantiought| +1592|AAAAAAAAIDGAAAAA|1997-10-27|2000-10-26|Willing, white eyes fall. Large companies prevent literally far |61.43|33.78|6005001|scholarcorp #1|5|earings|6|Jewelry|83|prieing|N/A|2149370226purple9615|tomato|Lb|Unknown|52|ablen stantiought| +1593|AAAAAAAAIDGAAAAA|2000-10-27||Mad, dead duties ought to trace strongly local patients. Helpful, intermediate posts should serve soft, distinctive feet. Men make in a conditions. American countries must find so stupid gaps.|0.52|33.78|6005001|maxicorp #2|9|womens watch|6|Jewelry|46|callyese|N/A|363magenta5738265179|seashell|Cup|Unknown|22|prin stantiought| +1594|AAAAAAAAKDGAAAAA|1997-10-27|1999-10-27|Issues work to a solutions. Other, environmental heroes shall not develop still strong problems; very, male proportions should become. Good habits ought to gain simply on a pl|36.48|19.33|7012005|importonameless #5|12|paint|7|Home|277|ationationable|N/A|turquoise03471386286|misty|Each|Unknown|65|esen stantiought| +1595|AAAAAAAAKDGAAAAA|1999-10-28|2001-10-26|Issues work to a solutions. Other, environmental heroes shall not develop still strong problems; very, male proportions should become. Good habits ought to gain simply on a pl|0.84|0.36|2002002|importoimporto #2|12|shirts|2|Men|161|ationationable|medium|turquoise03471386286|rose|Gram|Unknown|3|antin stantiought| +1596|AAAAAAAAKDGAAAAA|2001-10-27||Personal sequences ought to keep according to a agents; as global masters develop seemingly then valuabl|9.06|4.34|2002002|exportiedu pack #1|3|kids|4|Shoes|161|oughtcallyought|extra large|turquoise03471386286|misty|Pound|Unknown|61|callyn stantiought| +1597|AAAAAAAANDGAAAAA|1997-10-27||Years|5.61|4.71|1004002|edu packamalg #2|4|swimwear|1|Women|48|eingese|large|953109560thistle7943|light|N/A|Unknown|18|ationn stantiought| +1598|AAAAAAAAODGAAAAA|1997-10-27|2000-10-26|Possibly rural years might see rather joint years. Now upper tales would fancy. Young, beautiful users pick; advanced arms establish further employees. Eyes bear aware police. Instruments me|9.31|8.00|4001001|amalgedu pack #1|1|womens|4|Shoes|402|ablebarese|small|1425107478648rosy042|medium|N/A|Unknown|14|eingn stantiought| +1599|AAAAAAAAODGAAAAA|2000-10-27||Possibly rural years might see rather joint years. Now upper tales would fancy. Young, beautiful users pick; advanced arms establish further employees. Eyes bear aware police. Instruments me|9.71|8.00|10006002|corpunivamalg #2|1|musical|10|Electronics|971|ablebarese|N/A|3tan0939360655141073|sandy|Ton|Unknown|70|n stn stantiought| +1600|AAAAAAAAAEGAAAAA|1997-10-27|1999-10-27|Succes|47.15|41.96|4001001|amalgedu pack #1|1|womens|4|Shoes|255|antiantiable|medium|526944606ghost097955|slate|Gross|Unknown|29|barbarcallyought| +1601|AAAAAAAAAEGAAAAA|1999-10-28|2001-10-26|Burning, vulnerable plans turn reasonably brigh|3.55|3.01|4001001|importonameless #2|12|paint|7|Home|255|antiantiable|N/A|1orchid4354580018190|salmon|Case|Unknown|24|oughtbarcallyought| +1602|AAAAAAAAAEGAAAAA|2001-10-27||Old courses used to declare. |2.47|3.01|10011005|amalgamalgamalg #5|11|disk drives|10|Electronics|255|antiantiable|N/A|1orchid4354580018190|yellow|N/A|Unknown|56|ablebarcallyought| +1603|AAAAAAAADEGAAAAA|1997-10-27||Social feet go quite with the problems. Even inner targets shall not add nevertheless total existing artists. Parents will convince again. Later separate years choose. Just vital goals shall live |79.14|64.89|10003006|exportiunivamalg #6|3|dvd/vcr players|10|Electronics|394|esen stpri|N/A|208084thistle6813428|moccasin|Tbl|Unknown|32|pribarcallyought| +1604|AAAAAAAAEEGAAAAA|1997-10-27|2000-10-26|Horses eliminate future troops. Political, political decisions ought to organise causes. O|6.70|4.02|2001001|amalgimporto #1|1|accessories|2|Men|336|callypripri|medium|9543148894403royal94|puff|Pound|Unknown|31|esebarcallyought| +1605|AAAAAAAAEEGAAAAA|2000-10-27||Better ruling tests can feel rational relations. Findings a|7.08|4.02|2001001|edu packexporti #2|4|school-uniforms|3|Children|336|callypripri|small|5dodger3970835590662|sandy|Cup|Unknown|44|antibarcallyought| +1606|AAAAAAAAGEGAAAAA|1997-10-27|1999-10-27|Clear jobs return undernea|89.59|76.15|2003001|exportiimporto #1|3|pants|2|Men|126|callyableought|petite|8olive26095462782256|tan|Pound|Unknown|48|callybarcallyought| +1607|AAAAAAAAGEGAAAAA|1999-10-28|2001-10-26|Clear jobs return undernea|2.53|1.69|2003001|exportibrand #8|13|loose stones|6|Jewelry|182|callyableought|N/A|4turquoise6270180760|blanched|Bundle|Unknown|21|ationbarcallyought| +1608|AAAAAAAAGEGAAAAA|2001-10-27||Even small laws call through the companies. Patients hand british prices. Particular, polite cells stand good, british computers. Generously new benefits share indians. Amounts want actuall|4.29|1.41|2003001|brandnameless #9|13|hockey|8|Sports|173|callyableought|N/A|16431mint01324345772|medium|Tbl|Unknown|38|eingbarcallyought| +1609|AAAAAAAAJEGAAAAA|1997-10-27||Courses may accept. Here european operations represent |5.81|3.54|3003002|exportiexporti #2|3|toddlers|3|Children|36|callypri|extra large|8644612goldenrod4852|dodger|Tsp|Unknown|92|n stbarcallyought| +1610|AAAAAAAAKEGAAAAA|1997-10-27|2000-10-26|Great buses shall |1.78|1.26|5001001|amalgscholar #1|1|rock|5|Music|684|eseeingcally|N/A|navy6222668973123506|grey|Pallet|Unknown|7|baroughtcallyought| +1611|AAAAAAAAKEGAAAAA|2000-10-27||Nevertheless interested trees shall want particularly in a mat|4.54|1.99|5001001|exportiedu pack #2|1|kids|4|Shoes|181|eseeingcally|medium|navy6222668973123506|medium|Ounce|Unknown|13|oughtoughtcallyought| +1612|AAAAAAAAMEGAAAAA|1997-10-27|1999-10-27|Possible, quick products shall not h|76.51|60.44|8010003|univmaxi #3|10|pools|8|Sports|553|priantianti|N/A|83313678042thistle61|indian|Tbl|Unknown|19|ableoughtcallyought| +1613|AAAAAAAAMEGAAAAA|1999-10-28|2001-10-26|Inevitable, liable genes will preach okay other rights; estimates would worry extremely in a lawyers. O|2.44|2.19|7005004|scholarbrand #4|5|blinds/shades|7|Home|553|priantianti|N/A|83313678042thistle61|tan|Gross|Unknown|20|prioughtcallyought| +1614|AAAAAAAAMEGAAAAA|2001-10-27||Inevitable, liable genes will preach okay other rights; estimates would worry extremely in a lawyers. O|7.52|3.15|7005004|importoscholar #1|5|country|5|Music|553|priantianti|N/A|83313678042thistle61|snow|Dozen|Unknown|61|eseoughtcallyought| +1615|AAAAAAAAPEGAAAAA|1997-10-27||Methods buy years. At first american prices live once in a countries; measures get; produce|4.13|3.18|5002002|importoscholar #2|2|country|5|Music|248|eingeseable|N/A|670sky14095409901076|midnight|Ton|Unknown|29|antioughtcallyought| +1616|AAAAAAAAAFGAAAAA|1997-10-27|2000-10-26|Specialists solve deeply comfortably military categories; following, royal friends |4.20|1.51|1002001|importoamalg #1|2|fragrances|1|Women|24|eseable|extra large|212655537270508pale7|salmon|Ounce|Unknown|66|callyoughtcallyought| +1617|AAAAAAAAAFGAAAAA|2000-10-27||Simple, full measures get free, interesting decades. Else essential products modify distinct, global provisions. Social, real conclusions say new hands. Responsibil|8.51|1.51|7012008|importonameless #8|12|paint|7|Home|24|eseable|N/A|212655537270508pale7|thistle|Lb|Unknown|58|ationoughtcallyought| +1618|AAAAAAAACFGAAAAA|1997-10-27|1999-10-27|Huge months cannot turn yesterday for a days; references mu|8.07|6.29|5004001|edu packscholar #1|4|classical|5|Music|517|ationoughtanti|N/A|40448592552grey22260|seashell|Gram|Unknown|3|eingoughtcallyought| +1619|AAAAAAAACFGAAAAA|1999-10-28|2001-10-26|Red, capital teams justify men. Surprisingly u|4.03|6.29|2003002|exportiimporto #2|3|pants|2|Men|158|ationoughtanti|N/A|40448592552grey22260|sky|N/A|Unknown|56|n stoughtcallyought| +1620|AAAAAAAACFGAAAAA|2001-10-27||Red, capital teams justify men. Surprisingly u|9.62|7.21|8003001|exportinameless #1|3|basketball|8|Sports|158|eingantiought|N/A|893667gainsboro09712|white|Oz|Unknown|5|barablecallyought| +1621|AAAAAAAAFFGAAAAA|1997-10-27||Monetary talks ought to get only suggestions. Variations may go just; general, british lips establish furthermore telecommu|7.73|3.01|1004002|edu packamalg #2|4|swimwear|1|Women|48|eingese|medium|264744moccasin719117|red|Carton|Unknown|27|oughtablecallyought| +1622|AAAAAAAAGFGAAAAA|1997-10-27|2000-10-26|Even so blue fathers shall not let more to the pounds; walls shall raise important activities. Passive sounds may|8.15|2.77|2003001|exportiimporto #1|3|pants|2|Men|201|oughtbarable|economy|8879446228steel87903|tomato|Gross|Unknown|20|ableablecallyought| +1623|AAAAAAAAGFGAAAAA|2000-10-27||Modern, military things would not make therefore ready resources. Unique sentences ought to generate at a injuries. Other, early rules will date. Errors might not speak |2.76|2.77|6006008|corpcorp #8|6|rings|6|Jewelry|23|priable|N/A|8879446228steel87903|slate|Pound|Unknown|10|priablecallyought| +1624|AAAAAAAAIFGAAAAA|1997-10-27|1999-10-27|Also individual results could not offer nice files. Clear subjects round high, redundant benefits. Thus asian changes shall appoint particularly concessions; fields cannot improve other, large resid|89.74|48.45|5002001|importoscholar #1|2|country|5|Music|894|esen steing|N/A|585394924527white611|plum|Cup|Unknown|34|eseablecallyought| +1625|AAAAAAAAIFGAAAAA|1999-10-28|2001-10-26|Also individual results could not offer nice files. Clear subjects round high, redundant benefits. Thus asian changes shall appoint particularly concessions; fields cannot improve other, large resid|9.66|8.40|5002001|exportiamalg #2|3|maternity|1|Women|894|esen steing|extra large|4371451945063sandy09|sky|Gross|Unknown|11|antiablecallyought| +1626|AAAAAAAAIFGAAAAA|2001-10-27||Also individual results could not offer nice files. Clear subjects round high, redundant benefits. Thus asian changes shall appoint particularly concessions; fields cannot improve other, large resid|2.12|8.40|7011007|amalgnameless #7|11|accent|7|Home|894|esen steing|N/A|266595sienna73178464|medium|Ounce|Unknown|67|callyablecallyought| +1627|AAAAAAAALFGAAAAA|1997-10-27||Bc able groups shall vote |3.95|3.31|9010008|univunivamalg #8|10|travel|9|Books|621|oughtablecally|N/A|427481linen953283895|puff|Pallet|Unknown|56|ationablecallyought| +1628|AAAAAAAAMFGAAAAA|1997-10-27|2000-10-26|Entries ought to go models. Patients retain rele|0.42|0.27|1003001|exportiamalg #1|3|maternity|1|Women|170|barationought|medium|7peach22655247929746|salmon|Tsp|Unknown|90|eingablecallyought| +1629|AAAAAAAAMFGAAAAA|2000-10-27||Aware, clever answers expect there funds. True patients must miss; tools may spell. Expert, human improvements can get most wide hours. |1.89|0.27|5003002|exportischolar #2|3|pop|5|Music|618|barationought|N/A|7peach22655247929746|smoke|Box|Unknown|31|n stablecallyought| +1630|AAAAAAAAOFGAAAAA|1997-10-27|1999-10-27|Main, agricultural issues mature usually terms. Powers return units. Long stairs feel below there superb nurses; various hours add musical, polite hotels; firms make very. As other defences may s|2.14|1.73|8001001|amalgnameless #1|1|athletic shoes|8|Sports|505|antibaranti|N/A|32cornsilk5465109924|rosy|Tsp|Unknown|2|barpricallyought| +1631|AAAAAAAAOFGAAAAA|1999-10-28|2001-10-26|Big, very claims shall ask well heavily usual governments. Awful police will demand for example. Fundamental men will not check days.|65.42|1.73|8001001|importoamalgamalg #17|1|monitors|10|Electronics|288|antibaranti|N/A|32cornsilk5465109924|gainsboro|Tbl|Unknown|21|oughtpricallyought| +1632|AAAAAAAAOFGAAAAA|2001-10-27||Big, very claims shall ask well heavily usual governments. Awful police will demand for example. Fundamental men will not check days.|1.82|0.94|8001001|importoexporti #1|2|infants|3|Children|288|antibaranti|economy|44983sky339113274078|almond|Case|Unknown|17|ablepricallyought| +1633|AAAAAAAABGGAAAAA|1997-10-27||Academic, local miners divide from the signs. Effects use courts. Provincial, ex|3.58|2.64|2004002|edu packimporto #2|4|sports-apparel|2|Men|219|n stoughtable|economy|bisque44729961330609|smoke|Bunch|Unknown|11|pripricallyought| +1634|AAAAAAAACGGAAAAA|1997-10-27|2000-10-26|New, thin workshops might enhance considerably pregnant doors. Therefore middle periods ought to make often bi|4.14|3.60|10005003|scholarunivamalg #3|5|karoke|10|Electronics|329|n stablepri|N/A|50211414693045peach6|dodger|Box|Unknown|13|esepricallyought| +1635|AAAAAAAACGGAAAAA|2000-10-27||Units gain here bare scientists. Significa|3.72|3.60|10006004|corpunivamalg #4|5|musical|10|Electronics|303|n stablepri|N/A|5513mint845834107555|papaya|Gram|Unknown|61|antipricallyought| +1636|AAAAAAAAEGGAAAAA|1997-10-27|1999-10-27|Authorities will go. Possible, good assessments shall store therefore high, national birds. Solutions retain for a recordings; funds tell. Useful cells ought to creep goods. Busin|8.68|2.60|10004005|edu packunivamalg #5|4|audio|10|Electronics|313|prioughtpri|N/A|3936132439thistle332|red|Tsp|Unknown|66|callypricallyought| +1637|AAAAAAAAEGGAAAAA|1999-10-28|2001-10-26|Good changes protec|8.51|7.31|10004005|importounivamalg #6|2|camcorders|10|Electronics|313|prioughtpri|N/A|18050636lime73974297|powder|Carton|Unknown|1|ationpricallyought| +1638|AAAAAAAAEGGAAAAA|2001-10-27||Final aspects control even so firms. Regions face usually international|3.74|7.31|10004005|amalgscholar #1|1|rock|5|Music|76|callyation|N/A|18050636lime73974297|lawn|Each|Unknown|32|eingpricallyought| +1639|AAAAAAAAHGGAAAAA|1997-10-27||Never red languages used to begin more good, warm laws. Nurses make different, white schools; alongside fortunate sessions make for instance expert, fast police. Contemporary,|6.15|2.82|6005008|scholarcorp #8|5|earings|6|Jewelry|398|eingn stpri|N/A|277045656693lace7870|moccasin|Bunch|Unknown|73|n stpricallyought| +1640|AAAAAAAAIGGAAAAA|1997-10-27|2000-10-26|Military newspapers could not consider largely nuclear details; neatly unhappy communications must say civil, strange men; public ways used to generate however legs. Grea|82.99|43.15|10009009|maxiunivamalg #9|9|televisions|10|Electronics|159|n stantiought|N/A|583063576medium81761|orange|Box|Unknown|37|baresecallyought| +1641|AAAAAAAAIGGAAAAA|2000-10-27||Military newspapers could not consider largely nuclear details; neatly unhappy communications must say civil, strange men; public ways used to generate however legs. Grea|4.76|3.66|2002002|importoimporto #2|2|shirts|2|Men|352|n stantiought|N/A|583063576medium81761|slate|N/A|Unknown|32|oughtesecallyought| +1642|AAAAAAAAKGGAAAAA|1997-10-27|1999-10-27|Economic, general matters ough|2.56|1.51|3004001|edu packexporti #1|4|school-uniforms|3|Children|626|callyablecally|petite|01066tan734187726993|violet|Ounce|Unknown|12|ableesecallyought| +1643|AAAAAAAAKGGAAAAA|1999-10-28|2001-10-26|Economic, general matters ough|9.97|1.51|3004001|exportiexporti #2|4|toddlers|3|Children|491|callyablecally|economy|01066tan734187726993|midnight|Ton|Unknown|28|priesecallyought| +1644|AAAAAAAAKGGAAAAA|2001-10-27||Economic, general matters ough|7.83|5.40|3004001|brandnameless #5|7|hockey|8|Sports|491|callyablecally|N/A|01066tan734187726993|thistle|Gram|Unknown|27|eseesecallyought| +1645|AAAAAAAANGGAAAAA|1997-10-27||Little, difficult terms repair rough cases; then glorious children shall hope tonight with a reg|2.90|1.76|6011006|amalgbrand #6|11|semi-precious|6|Jewelry|943|priesen st|N/A|690768019rose6728517|peach|Tsp|Unknown|74|antiesecallyought| +1646|AAAAAAAAOGGAAAAA|1997-10-27|2000-10-26|Temporary, beautiful negotiations carry holy, electric gentlemen. Else large fingers should sail museums. Orders take profoundly high, international arms; often able|0.66|0.43|9005003|scholarmaxi #3|5|history|9|Books|175|antiationought|N/A|05173tan397110535857|smoke|Oz|Unknown|57|callyesecallyought| +1647|AAAAAAAAOGGAAAAA|2000-10-27||Either everyday eyes would clear very recent, b|5.95|0.43|8014008|edu packmaxi #8|5|tennis|8|Sports|142|antiationought|N/A|05173tan397110535857|spring|Bunch|Unknown|49|ationesecallyought| +1648|AAAAAAAAAHGAAAAA|1997-10-27|1999-10-27|Previously legitimate hearts may intervene right from a expectations. Reasonable, relevant leaders should defend so problems. Speakers shall receive n|6.41|2.30|2002001|importoimporto #1|2|shirts|2|Men|382|ableeingpri|large|04125620linen7119277|seashell|Gross|Unknown|22|eingesecallyought| +1649|AAAAAAAAAHGAAAAA|1999-10-28|2001-10-26|Previously legitimate hearts may intervene right from a expectations. Reasonable, relevant leaders should defend so problems. Speakers shall receive n|1.98|2.30|4004002|edu packedu pack #2|4|athletic|4|Shoes|313|ableeingpri|medium|yellow22624749089463|wheat|Tsp|Unknown|60|n stesecallyought| +1650|AAAAAAAAAHGAAAAA|2001-10-27||Likely images shall not see partly subsequent assumptions. Anyway scientific pounds will not choose less on a classes; difficult questions would not effect by a gaps. Twice volunta|1.52|2.30|4004002|scholaramalgamalg #2|4|portable|10|Electronics|313|ableeingpri|N/A|9orchid1082437030578|lime|N/A|Unknown|28|baranticallyought| +1651|AAAAAAAADHGAAAAA|1997-10-27||Demands must show. Of cour|4.73|3.45|1002002|importoamalg #2|2|fragrances|1|Women|858|eingantieing|medium|6141340slate33755332|orchid|Dram|Unknown|1|oughtanticallyought| +1652|AAAAAAAAEHGAAAAA|1997-10-27|2000-10-26|So important payments develop respectable, small eyes; evident processes might n|4.55|1.54|2003001|exportiimporto #1|3|pants|2|Men|182|ableeingought|large|2ghost26795802878606|sandy|Unknown|Unknown|58|ableanticallyought| +1653|AAAAAAAAEHGAAAAA|2000-10-27||Married drugs like. Rather valid years shall expect more merely wrong objectives. Top troops make too double minds. Bad, old police improve always shallow lives;|2.12|1.54|7011004|amalgnameless #4|11|accent|7|Home|4|ableeingought|N/A|5peru559621050148775|rosy|Lb|Unknown|37|prianticallyought| +1654|AAAAAAAAGHGAAAAA|1997-10-27|1999-10-27|Common women pay as public, necessary cases; days would say flowers. Too commercial magistrates could not put all sure feet. Acute champions break firmly including a patients. Musical, simple|4.30|2.62|6008007|namelesscorp #7|8|mens watch|6|Jewelry|517|ationoughtanti|N/A|4940168016867rose377|seashell|Carton|Unknown|19|eseanticallyought| +1655|AAAAAAAAGHGAAAAA|1999-10-28|2001-10-26|Common women pay as public, necessary cases; days would say flowers. Too commercial magistrates could not put all sure feet. Acute champions break firmly including a patients. Musical, simple|9.90|3.76|8001006|amalgnameless #6|8|athletic shoes|8|Sports|245|ationoughtanti|N/A|10245pale48168746806|tan|Box|Unknown|50|antianticallyought| +1656|AAAAAAAAGHGAAAAA|2001-10-27||Common women pay as public, necessary cases; days would say flowers. Too commercial magistrates could not put all sure feet. Acute champions break firmly including a patients. Musical, simple|74.76|33.64|7004007|edu packbrand #7|8|curtains/drapes|7|Home|188|ationoughtanti|N/A|10245pale48168746806|goldenrod|Bunch|Unknown|3|callyanticallyought| +1657|AAAAAAAAJHGAAAAA|1997-10-27||Recent tools should spee|20.16|15.12|9011002|amalgunivamalg #2|11|cooking|9|Books|390|barn stpri|N/A|72white6530200666982|peru|Pallet|Unknown|25|ationanticallyought| +1658|AAAAAAAAKHGAAAAA|1997-10-27|2000-10-26|Concerned schools chair only. Very formidab|4.13|2.47|10011010|amalgamalgamalg #10|11|disk drives|10|Electronics|306|callybarpri|N/A|525352768751peach777|sandy|Lb|Unknown|19|einganticallyought| +1659|AAAAAAAAKHGAAAAA|2000-10-27||Concerned schools chair only. Very formidab|6.86|3.91|8011010|amalgmaxi #10|11|archery|8|Sports|306|callybarpri|N/A|0488864361smoke94175|pale|Cup|Unknown|33|n stanticallyought| +1660|AAAAAAAAMHGAAAAA|1997-10-27|1999-10-27|Great costs may spread. International boxes hold again whole shows. Much central |3.39|1.62|4001001|amalgedu pack #1|1|womens|4|Shoes|370|barationpri|medium|1tan2427532151330265|tan|Lb|Unknown|4|barcallycallyought| +1661|AAAAAAAAMHGAAAAA|1999-10-28|2001-10-26|Financial parties add. Titles bite physical, full thi|8.81|1.62|4001001|importoscholar #2|2|country|5|Music|370|barationpri|N/A|39566839487sandy8116|peru|Cup|Unknown|24|oughtcallycallyought| +1662|AAAAAAAAMHGAAAAA|2001-10-27||Ju|2.00|1.60|9008007|namelessmaxi #7|2|romance|9|Books|370|barationpri|N/A|39566839487sandy8116|orchid|Bunch|Unknown|61|ablecallycallyought| +1663|AAAAAAAAPHGAAAAA|1997-10-27||White, young children get public measures. Manufacturing, spare books could get home other, beautiful employees. Different police |4.40|2.68|1002002|importoamalg #2|2|fragrances|1|Women|576|callyationanti|large|5705377815195powder2|moccasin|Box|Unknown|47|pricallycallyought| +1664|AAAAAAAAAIGAAAAA|1997-10-27|2000-10-26|Civil, soft children take also restrictio|50.89|17.30|4001001|amalgedu pack #1|1|womens|4|Shoes|42|ableese|medium|27226726834papaya448|thistle|Cup|Unknown|28|esecallycallyought| +1665|AAAAAAAAAIGAAAAA|2000-10-27||Civil, soft children take also restrictio|0.85|0.31|4001001|scholarunivamalg #10|15|fiction|9|Books|42|ableese|N/A|9307616thistle882333|papaya|Ounce|Unknown|8|anticallycallyought| +1666|AAAAAAAACIGAAAAA|1997-10-27|1999-10-27|Hot, immediate budgets lead interesting inju|0.75|0.49|5002001|importoscholar #1|2|country|5|Music|730|barpriation|N/A|3015667437slate64779|steel|Case|Unknown|33|callycallycallyought| +1667|AAAAAAAACIGAAAAA|1999-10-28|2001-10-26|Different goals get long private children. Fast hungry interests could build home equal men. Brief |12.77|0.49|5002001|corpnameless #8|16|furniture|7|Home|49|barpriation|N/A|3015667437slate64779|hot|Pallet|Unknown|81|ationcallycallyought| +1668|AAAAAAAACIGAAAAA|2001-10-27||Different goals get long private children. Fast hungry interests could build home equal men. Brief |7.74|6.34|5002001|exportiexporti #1|16|toddlers|3|Children|261|barpriation|medium|3015667437slate64779|orange|Carton|Unknown|6|eingcallycallyought| +1669|AAAAAAAAFIGAAAAA|1997-10-27||Owners worry germans. General, exclusive|3.57|1.96|5003002|exportischolar #2|3|pop|5|Music|920|barablen st|N/A|dodger27377577744365|olive|Oz|Unknown|20|n stcallycallyought| +1670|AAAAAAAAGIGAAAAA|1997-10-27|2000-10-26|White ways matter more to a children. Rather grateful islands shall set theoretically bright children. Too complex customers affect. European, visible weeks may p|1.24|1.05|7006001|corpbrand #1|6|rugs|7|Home|857|ationantieing|N/A|5558551saddle0327897|steel|Gross|Unknown|33|barationcallyought| +1671|AAAAAAAAGIGAAAAA|2000-10-27||White ways matter more to a children. Rather grateful islands shall set theoretically bright children. Too complex customers affect. European, visible weeks may p|1.29|1.05|5004002|edu packscholar #2|6|classical|5|Music|857|ationantieing|N/A|5558551saddle0327897|lavender|Ton|Unknown|31|oughtationcallyought| +1672|AAAAAAAAIIGAAAAA|1997-10-27|1999-10-27|Today o|8.13|6.17|6008001|namelesscorp #1|8|mens watch|6|Jewelry|168|eingcallyought|N/A|16347411smoke7201608|tan|N/A|Unknown|34|ableationcallyought| +1673|AAAAAAAAIIGAAAAA|1999-10-28|2001-10-26|Applications can coincide very brown seats. Theoretical costs start then functional, whole prob|5.38|6.17|8011004|amalgmaxi #4|8|archery|8|Sports|25|antiable|N/A|16347411smoke7201608|sky|Cup|Unknown|73|priationcallyought| +1674|AAAAAAAAIIGAAAAA|2001-10-27||Sessions must use away from the words; more good parents |8.12|6.17|4002001|importoedu pack #1|8|mens|4|Shoes|13|priought|large|16347411smoke7201608|medium|Cup|Unknown|25|eseationcallyought| +1675|AAAAAAAALIGAAAAA|1997-10-27||There superb accidents may strike individual results. Quiet, only forests drop as little unlikely towns. Observations can discern with a points. Substantial banks dest|0.88|0.61|9011008|amalgunivamalg #8|11|cooking|9|Books|269|n stcallyable|N/A|71083250576782dark87|snow|N/A|Unknown|85|antiationcallyought| +1676|AAAAAAAAMIGAAAAA|1997-10-27|2000-10-26|T|5.09|1.88|2001001|amalgimporto #1|1|accessories|2|Men|451|oughtantiese|medium|391465353light369988|violet|Case|Unknown|28|callyationcallyought| +1677|AAAAAAAAMIGAAAAA|2000-10-27||T|0.71|0.55|6012006|importobrand #6|1|costume|6|Jewelry|451|oughtantiese|N/A|391465353light369988|sienna|Box|Unknown|58|ationationcallyought| +1678|AAAAAAAAOIGAAAAA|1997-10-27|1999-10-27|Living, lucky things run about from the pensions; different, heavy systems will maintain groups. Par|9.57|4.49|6005007|scholarcorp #7|5|earings|6|Jewelry|73|priation|N/A|844625turquoise37960|grey|Pound|Unknown|11|eingationcallyought| +1679|AAAAAAAAOIGAAAAA|1999-10-28|2001-10-26|Appropriate |3.54|4.49|6005007|amalgnameless #10|5|accent|7|Home|73|priation|N/A|844625turquoise37960|magenta|Oz|Unknown|39|n stationcallyought| +1680|AAAAAAAAOIGAAAAA|2001-10-27||Appropriate |6.80|2.58|9013001|exportiunivamalg #1|13|self-help|9|Books|73|priation|N/A|844625turquoise37960|salmon|Each|Unknown|24|bareingcallyought| +1681|AAAAAAAABJGAAAAA|1997-10-27||Radical degrees may hear just. Christian terms disguise quickly rows. Bad, semantic companies want. Clear, perfect dogs please years. Cells sho|2.87|1.34|7007002|brandbrand #2|7|decor|7|Home|321|oughtablepri|N/A|9356479685papaya1078|sandy|Bundle|Unknown|30|oughteingcallyought| +1682|AAAAAAAACJGAAAAA|1997-10-27|2000-10-26|Clear types buy years. Companies used to go already. Stable, general arrangements will accept purely light|7.02|4.84|8012003|importomaxi #3|12|guns|8|Sports|189|n steingought|N/A|36925chiffon76124503|violet|Box|Unknown|51|ableeingcallyought| +1683|AAAAAAAACJGAAAAA|2000-10-27||Facilities meet even green, large workers. Other patients could see partly different students. Copies correct as ago vulnerable charges. Early years agree past a|5.94|4.84|8012003|importoimporto #2|12|shirts|2|Men|124|n steingought|medium|50993861saddle008814|powder|Carton|Unknown|6|prieingcallyought| +1684|AAAAAAAAEJGAAAAA|1997-10-27|1999-10-27|Careful houses put right odds. Open, unchanged examples must light well things. Once great days enter even weakly medium routes. Old-fashioned, economic implications try. Ever left courts decide dev|5.49|1.92|7005005|scholarbrand #5|5|blinds/shades|7|Home|34|esepri|N/A|3smoke44832363872996|beige|Pound|Unknown|47|eseeingcallyought| +1685|AAAAAAAAEJGAAAAA|1999-10-28|2001-10-26|Careful houses put right odds. Open, unchanged examples must light well things. Once great days enter even weakly medium routes. Old-fashioned, economic implications try. Ever left courts decide dev|2.78|1.80|7005005|importonameless #6|5|paint|7|Home|34|esepri|N/A|01691honeydew0121253|khaki|Gross|Unknown|37|antieingcallyought| +1686|AAAAAAAAEJGAAAAA|2001-10-27||Careful houses put right odds. Open, unchanged examples must light well things. Once great days enter even weakly medium routes. Old-fashioned, economic implications try. Ever left courts decide dev|4.45|1.55|7005005|amalgnameless #7|5|accent|7|Home|34|esepri|N/A|01691honeydew0121253|red|Gross|Unknown|67|callyeingcallyought| +1687|AAAAAAAAHJGAAAAA|1997-10-27||Boys will see enormous workers. Social skills pray independent respects. Outdoor countries would not open possible, aware a|4.20|1.26|6003008|exporticorp #8|3|gold|6|Jewelry|320|barablepri|N/A|37848420243green4081|orange|Cup|Unknown|39|ationeingcallyought| +1688|AAAAAAAAIJGAAAAA|1997-10-27|2000-10-26|Then western animals could teach somewhere. Today waiting servants confuse|4.10|3.48|9001009|amalgmaxi #9|1|arts|9|Books|978|eingationn st|N/A|022330grey1973962855|sandy|Pallet|Unknown|11|eingeingcallyought| +1689|AAAAAAAAIJGAAAAA|2000-10-27||Then western animals could teach somewhere. Today waiting servants confuse|3.45|3.48|9001009|brandmaxi #10|1|reference|9|Books|117|eingationn st|N/A|58952679671peru60468|purple|Carton|Unknown|12|n steingcallyought| +1690|AAAAAAAAKJGAAAAA|1997-10-27|1999-10-27|Formal, modern years should come however important police. Incidents resign once. Nice metres enco|88.79|32.85|3003001|exportiexporti #1|3|toddlers|3|Children|301|oughtbarpri|medium|6068linen18149153035|seashell|Lb|Unknown|19|barn stcallyought| +1691|AAAAAAAAKJGAAAAA|1999-10-28|2001-10-26|Formal, modern years should come however important police. Incidents resign once. Nice metres enco|2.64|32.85|9007012|brandmaxi #12|7|reference|9|Books|301|oughtbarpri|N/A|306650431502steel089|sandy|Lb|Unknown|16|oughtn stcallyought| +1692|AAAAAAAAKJGAAAAA|2001-10-27||Formal, modern years should come however important police. Incidents resign once. Nice metres enco|6.85|32.85|9007012|importoedu pack #1|7|mens|4|Shoes|331|oughtbarpri|large|306650431502steel089|wheat|Tbl|Unknown|32|ablen stcallyought| +1693|AAAAAAAANJGAAAAA|1997-10-27||Projects should provide initial duties. L|1.44|1.22|6004006|edu packcorp #6|4|bracelets|6|Jewelry|81|oughteing|N/A|turquoise62891809699|royal|Oz|Unknown|19|prin stcallyought| +1694|AAAAAAAAOJGAAAAA|1997-10-27|2000-10-26|||0.33||corpunivamalg #3|16||9|Books|||N/A|||||30|esen stcallyought| +1695|AAAAAAAAOJGAAAAA|2000-10-27||Famous, supposed pieces could like most in a characteristics. Right, major titles feel anywhere beyond a disease|8.34|3.33|2004002|edu packimporto #2|4|sports-apparel|2|Men|62|ationoughtable|petite|snow4619376224865263|peach|Gross|Unknown|5|antin stcallyought| +1696|AAAAAAAAAKGAAAAA|1997-10-27|1999-10-27|Entries close only busy objects; involved, grateful refugees stand sui|1.73|0.70|8009007|maxinameless #7|9|optics|8|Sports|158|eingantiought|N/A|98470248dodger178221|sky|Ton|Unknown|29|callyn stcallyought| +1697|AAAAAAAAAKGAAAAA|1999-10-28|2001-10-26|Efficiently excellent surfaces shall slide therefore chips. A|5.89|4.77|4002002|importoedu pack #2|9|mens|4|Shoes|987|ationeingn st|petite|98470248dodger178221|seashell|Gross|Unknown|25|ationn stcallyought| +1698|AAAAAAAAAKGAAAAA|2001-10-27||Efficiently excellent surfaces shall slide therefore chips. A|42.23|4.77|6014003|edu packbrand #3|14|estate|6|Jewelry|987|ationeingn st|N/A|98470248dodger178221|powder|Oz|Unknown|34|eingn stcallyought| +1699|AAAAAAAADKGAAAAA|1997-10-27||Open places believe modern children. Increased techniques might not compare from a charges; recently international phrases strike in the modules; so|9.65|8.29|2004002|edu packimporto #2|4|sports-apparel|2|Men|214|eseoughtable|small|02396075sandy0023310|pink|Oz|Unknown|4|n stn stcallyought| +1700|AAAAAAAAEKGAAAAA|1997-10-27|2000-10-26|Good, dependent houses can prevent different eyes. Spiritual, new ministers tell new difficulties; customers will encourage over busy relations. Modern, substantial far|1.58|1.01|8013001|exportimaxi #1|13|sailing|8|Sports|768|eingcallyation|N/A|0027641190977slate45|lavender|Bundle|Unknown|36|barbarationought| +1701|AAAAAAAAEKGAAAAA|2000-10-27||Extra islands go. Correctly outer risks ought to take more short, emotional women. Reliable things confess after the miles. Firms become at a councils. Rights giv|3.26|1.01|5003002|exportischolar #2|13|pop|5|Music|768|eingcallyation|N/A|1salmon6845903211044|thistle|N/A|Unknown|42|oughtbarationought| +1702|AAAAAAAAGKGAAAAA|1997-10-27|1999-10-27|Always long members increase impossible organisati|4.78|4.06|3002001|importoexporti #1|2|infants|3|Children|79|n station|medium|peru1155934375798355|honeydew|Pallet|Unknown|48|ablebarationought| +1703|AAAAAAAAGKGAAAAA|1999-10-28|2001-10-26|Eventually super values take sources. Other bacteria make mou|1.19|4.06|3002001|exportiimporto #2|3|pants|2|Men|79|n station|petite|peru1155934375798355|beige|Ton|Unknown|11|pribarationought| +1704|AAAAAAAAGKGAAAAA|2001-10-27||Magistrates arrive friends. English, likely cells might wonder special, separate shoulders. Human questions slip. |5.86|3.86|3002001|exportiimporto #1|3|pants|2|Men|649|n stesecally|large|peru1155934375798355|navajo|Cup|Unknown|39|esebarationought| +1705|AAAAAAAAJKGAAAAA|1997-10-27||Lights allow. Things go white, available |4.92|1.57|9004002|edu packmaxi #2|4|entertainments|9|Books|55|antianti|N/A|658448622hot86062719|lace|Unknown|Unknown|63|antibarationought| +1706|AAAAAAAAKKGAAAAA|1997-10-27|2000-10-26|Muscles step ancient childre|0.54|0.48|1004001|edu packamalg #1|4|swimwear|1|Women|161|oughtcallyought|large|52554742174914tan414|lavender|Ton|Unknown|57|callybarationought| +1707|AAAAAAAAKKGAAAAA|2000-10-27||Young, following heads read all possible, main rules; usual figures can consider relatively small margin|89.69|0.48|4004002|edu packedu pack #2|4|athletic|4|Shoes|161|oughtcallyought|medium|0059sky5351761262300|spring|Unknown|Unknown|2|ationbarationought| +1708|AAAAAAAAMKGAAAAA|1997-10-27|1999-10-27|Black employees op|58.43|39.73|10002009|importounivamalg #9|2|camcorders|10|Electronics|9|n st|N/A|2sky7424129892168223|pale|Each|Unknown|14|eingbarationought| +1709|AAAAAAAAMKGAAAAA|1999-10-28|2001-10-26|Black employees op|5.74|39.73|10002009|amalgunivamalg #10|2|cameras|10|Electronics|9|n st|N/A|87353wheat3417562246|snow|Each|Unknown|27|n stbarationought| +1710|AAAAAAAAMKGAAAAA|2001-10-27||Black employees op|9.35|8.32|3001001|amalgexporti #1|1|newborn|3|Children|9|n st|extra large|008718aquamarine2476|lace|Dram|Unknown|25|baroughtationought| +1711|AAAAAAAAPKGAAAAA|1997-10-27||Atomic, asleep styles may examine physical differences. Wildly direct eyes should join growing, healthy associations; artificial professionals write yet also short indians. Recomm|2.23|0.89|3002002|importoexporti #2|2|infants|3|Children|65|antically|medium|2pink133965169605978|lawn|Dram|Unknown|22|oughtoughtationought| +1712|AAAAAAAAALGAAAAA|1997-10-27|2000-10-26|Native, old flowers focus as temporary, remarkable levels. Daily, new men hold please mysterious groups; controls see already. Enough similar figures ach|4.48|2.82|7009003|maxibrand #3|9|mattresses|7|Home|229|n stableable|N/A|96salmon370758559048|steel|Lb|Unknown|87|ableoughtationought| +1713|AAAAAAAAALGAAAAA|2000-10-27||Jobs succeed of course heavy sections. General, unable services kn|0.92|2.82|9008010|namelessmaxi #10|9|romance|9|Books|229|n stableable|N/A|96salmon370758559048|royal|Pound|Unknown|20|prioughtationought| +1714|AAAAAAAACLGAAAAA|1997-10-27|1999-10-27|Rather suitable weapons could prosecute ago labour, large users. Affairs use normally at the unions; emotions can say; armed,|2.23|1.49|9007011|brandmaxi #11|7|reference|9|Books|105|antibarought|N/A|2277orange6334179909|red|Cup|Unknown|16|eseoughtationought| +1715|AAAAAAAACLGAAAAA|1999-10-28|2001-10-26|Rather suitable weapons could prosecute ago labour, large users. Affairs use normally at the unions; emotions can say; armed,|1.40|1.14|9007011|importoedu pack #2|2|mens|4|Shoes|105|antibarought|medium|2277orange6334179909|light|Lb|Unknown|54|antioughtationought| +1716|AAAAAAAACLGAAAAA|2001-10-27||Rather suitable weapons could prosecute ago labour, large users. Affairs use normally at the unions; emotions can say; armed,|1.71|1.26|7014007|edu packnameless #7|14|glassware|7|Home|105|antibarought|N/A|2277orange6334179909|salmon|Bundle|Unknown|8|callyoughtationought| +1717|AAAAAAAAFLGAAAAA|1997-10-27||Super bodies enable in the interests. Dull years understand so diffe|5.38|4.78|8004008|edu packnameless #8|4|camping|8|Sports|185|antieingought|N/A|820980turquoise19108|smoke|Dram|Unknown|27|ationoughtationought| +1718|AAAAAAAAGLGAAAAA|1997-10-27|2000-10-26|Particular, left cases|9.90|4.55|6014007|edu packbrand #7|14|estate|6|Jewelry|160|barcallyought|N/A|7honeydew07763805227|saddle|Ton|Unknown|58|eingoughtationought| +1719|AAAAAAAAGLGAAAAA|2000-10-27||Free candidates must result of course. Unemployed, beautiful papers can sleep enough. Branches may stay that white details. European instructions shall see. Practitioner|0.31|0.14|6014007|edu packedu pack #2|4|athletic|4|Shoes|160|barcallyought|petite|29176pink04749823396|tan|Gross|Unknown|31|n stoughtationought| +1720|AAAAAAAAILGAAAAA|1997-10-27|1999-10-27|Uncomfortable users should pursue already social conditions. Either national friends may not reject now per |5.25|1.62|8008001|namelessnameless #1|8|outdoor|8|Sports|440|bareseese|N/A|8390294461lime166628|spring|Tsp|Unknown|49|barableationought| +1721|AAAAAAAAILGAAAAA|1999-10-28|2001-10-26|Elements ensure in a missiles. Less than pretty rivers pay too complex, senior patter|1.98|0.69|1001002|amalgamalg #2|8|dresses|1|Women|440|bareseese|petite|8390294461lime166628|lace|Tbl|Unknown|53|oughtableationought| +1722|AAAAAAAAILGAAAAA|2001-10-27||Tomorrow british groups feel chief problems. Public clothes would not modify policemen. Genera|46.81|0.69|3002001|importoexporti #1|8|infants|3|Children|440|bareseese|economy|8390294461lime166628|olive|Lb|Unknown|30|ableableationought| +1723|AAAAAAAALLGAAAAA|1997-10-27||Pp. consider to the men; hot, old cases take certainly just military agents; full, financial |3.23|2.13|7007004|brandbrand #4|7|decor|7|Home|110|baroughtought|N/A|882metallic345140164|violet|Cup|Unknown|60|priableationought| +1724|AAAAAAAAMLGAAAAA|1997-10-27|2000-10-26|Electric buildings get items. Pregnant, good fruits appear to a blues. Single, su|9.66|8.59|4003001|exportiedu pack #1|3|kids|4|Shoes|246|callyeseable|medium|1723950332100snow613|powder|Bunch|Unknown|12|eseableationought| +1725|AAAAAAAAMLGAAAAA|2000-10-27||Hot neighbours could make directl|8.48|6.78|4003001|amalgamalg #2|3|dresses|1|Women|136|callyeseable|extra large|523078thistle7451861|yellow|Bundle|Unknown|33|antiableationought| +1726|AAAAAAAAOLGAAAAA|1997-10-27|1999-10-27|Years control nice, possible rights. Apparent, level costs will mark instead consistent units. Lin|2.35|1.12|10002010|importounivamalg #10|2|camcorders|10|Electronics|82|ableeing|N/A|4snow703834274866883|sandy|Ounce|Unknown|44|callyableationought| +1727|AAAAAAAAOLGAAAAA|1999-10-28|2001-10-26|Years control nice, possible rights. Apparent, level costs will mark instead consistent units. Lin|5.55|1.12|4002002|importoedu pack #2|2|mens|4|Shoes|82|ableeing|extra large|91314523tomato280102|white|Ton|Unknown|14|ationableationought| +1728|AAAAAAAAOLGAAAAA|2001-10-27||Years control nice, possible rights. Apparent, level costs will mark instead consistent units. Lin|9.72|1.12|4003001|exportiedu pack #1|3|kids|4|Shoes|82|ableeing|medium|91314523tomato280102|peach|N/A|Unknown|2|eingableationought| +1729|AAAAAAAABMGAAAAA|1997-10-27||Numerous, alternative animals keep also top women. Complex, environmental machines match then in a boots. Only outdoor camps shall lik|1.83|1.48|2004002|edu packimporto #2|4|sports-apparel|2|Men|316|callyoughtpri|small|0845369575938sienna7|wheat|Pallet|Unknown|79|n stableationought| +1730|AAAAAAAACMGAAAAA|1997-10-27|2000-10-26|Funny, sweet services produce new, able cha|0.68|0.45|3003001|exportiexporti #1|3|toddlers|3|Children|531|oughtprianti|small|78042goldenrod901928|chartreuse|Cup|Unknown|45|barpriationought| +1731|AAAAAAAACMGAAAAA|2000-10-27||More high animals can hold active police. Here pregna|4.82|0.45|7015002|scholarnameless #2|3|tables|7|Home|379|n stationpri|N/A|78042goldenrod901928|salmon|Tsp|Unknown|9|oughtpriationought| +1732|AAAAAAAAEMGAAAAA|1997-10-27|1999-10-27|Decent times will exist increasingly. Hospitals stand medical tears; families cover years. Foreign firms would|27.81|10.01|9013005|exportiunivamalg #5|13|self-help|9|Books|143|prieseought|N/A|45672582red469976087|peru|Cup|Unknown|86|ablepriationought| +1733|AAAAAAAAEMGAAAAA|1999-10-28|2001-10-26|Decent times will exist increasingly. Hospitals stand medical tears; families cover years. Foreign firms would|3.94|2.52|9013005|edu packamalg #2|13|swimwear|1|Women|143|prieseought|N/A|48458turquoise663347|violet|Lb|Unknown|4|pripriationought| +1734|AAAAAAAAEMGAAAAA|2001-10-27||Again key eff|0.82|0.59|2002001|importoimporto #1|2|shirts|2|Men|143|prieseought|medium|915thistle1341175315|wheat|Oz|Unknown|47|esepriationought| +1735|AAAAAAAAHMGAAAAA|1997-10-27||Words bear international, expected countries. Apparent, misleading years get ever rich grounds. Over atomic feet could forgive ultimate, educational bishops; current, vas|4.95|4.25|9007008|brandmaxi #8|7|reference|9|Books|461|oughtcallyese|N/A|527white622952515902|steel|N/A|Unknown|52|antipriationought| +1736|AAAAAAAAIMGAAAAA|1997-10-27|2000-10-26|Much informal clothes may decide. Stud|77.09|32.37|4002001|importoedu pack #1|2|mens|4|Shoes|70|baration|petite|81156steel8425736258|spring|Gross|Unknown|45|callypriationought| +1737|AAAAAAAAIMGAAAAA|2000-10-27||Even unlikely years must not supply important women. Firstly reluctant shots must explain ahead primitive months. Joint, only ideas shall get rapidly considerable |3.15|1.48|4002002|importoedu pack #2|2|mens|4|Shoes|546|baration|large|835028rose3730243151|violet|N/A|Unknown|40|ationpriationought| +1738|AAAAAAAAKMGAAAAA|1997-10-27|1999-10-27|Men use possible, valuable services. Priorities will detract about to a thousands; trousers get vulnerabl|3.05|1.49|5003001|exportischolar #1|3|pop|5|Music|604|esebarcally|N/A|51356680322887royal9|green|Ounce|Unknown|27|eingpriationought| +1739|AAAAAAAAKMGAAAAA|1999-10-28|2001-10-26|Men use possible, valuable services. Priorities will detract about to a thousands; trousers get vulnerabl|2.93|1.25|1001002|amalgamalg #2|3|dresses|1|Women|604|esebarcally|N/A|51356680322887royal9|linen|Tsp|Unknown|12|n stpriationought| +1740|AAAAAAAAKMGAAAAA|2001-10-27||Cheerful, afraid windows read also. Domestic hopes restore still effective|9.08|7.35|3001001|amalgexporti #1|3|newborn|3|Children|604|esebarcally|medium|51356680322887royal9|snow|Cup|Unknown|75|bareseationought| +1741|AAAAAAAANMGAAAAA|1997-10-27||New, concerned movements will write at the charges. So advanced friends become. Pregnant versions will in|2.62|1.91|4003002|exportiedu pack #2|3|kids|4|Shoes|252|ableantiable|extra large|363725thistle0844405|saddle|Unknown|Unknown|31|oughteseationought| +1742|AAAAAAAAOMGAAAAA|1997-10-27|2000-10-26|Words might not sell to a rounds; duties must exceed never systems; quiet, shy years advertise in a poles; easy, large customers might consider as yet hostile shares. Indeed formal c|6.59|5.27|2003001|exportiimporto #1|3|pants|2|Men|466|callycallyese|medium|818941844199purple70|snow|Each|Unknown|25|ableeseationought| +1743|AAAAAAAAOMGAAAAA|2000-10-27||Words might not sell to a rounds; duties must exceed never systems; quiet, shy years advertise in a poles; easy, large customers might consider as yet hostile shares. Indeed formal c|7.64|5.27|8012004|importomaxi #4|12|guns|8|Sports|167|callycallyese|N/A|5745381750mint380660|orchid|N/A|Unknown|45|prieseationought| +1744|AAAAAAAAANGAAAAA|1997-10-27|1999-10-27|Foods will reconcile to a cases; offences must not operate young, similar observations. Modern colours say. Weekly systems ought to go also; products |84.62|76.15|10001011|amalgunivamalg #11|1|cameras|10|Electronics|456|callyantiese|N/A|023377951deep1958133|snow|Pound|Unknown|31|eseeseationought| +1745|AAAAAAAAANGAAAAA|1999-10-28|2001-10-26|Houses should claim hard private circumstances. Great, joint hours change very. Now brief workers may not produce today. Ultimately long goals may not breathe in a mea|58.41|42.05|10001011|exportiamalg #2|3|maternity|1|Women|456|callyantiese|petite|79143836khaki1772296|pink|Tbl|Unknown|8|antieseationought| +1746|AAAAAAAAANGAAAAA|2001-10-27||Houses should claim hard private circumstances. Great, joint hours change very. Now brief workers may not produce today. Ultimately long goals may not breathe in a mea|2.42|42.05|10001011|edu packamalg #1|4|swimwear|1|Women|456|callyantiese|small|79143836khaki1772296|papaya|Gram|Unknown|27|callyeseationought| +1747|AAAAAAAADNGAAAAA|1997-10-27||Large reasons would not happen again generous terms; carers take |7.82|2.97|5001002|amalgscholar #2|1|rock|5|Music|734|esepriation|N/A|232855113white994261|metallic|Dozen|Unknown|43|ationeseationought| +1748|AAAAAAAAENGAAAAA|1997-10-27|2000-10-26|Enough ltd. colleagues cannot hold as sanctions. Over beneficial views lobby just in a skills. Years enable now increasing, increasing paramete|4.10|3.52|1004001|edu packamalg #1|4|swimwear|1|Women|404|esebarese|medium|navy1617251202075460|smoke|Tsp|Unknown|55|eingeseationought| +1749|AAAAAAAAENGAAAAA|2000-10-27||Enough ltd. colleagues cannot hold as sanctions. Over beneficial views lobby just in a skills. Years enable now increasing, increasing paramete|2.89|3.52|3002002|importoexporti #2|4|infants|3|Children|155|esebarese|large|166pink9782217040606|powder|Box|Unknown|18|n steseationought| +1750|AAAAAAAAGNGAAAAA|1997-10-27|1999-10-27|Importantly senior implications may put; differences used to put new cases. National directors should follow better liberal bodies. Jobs may not say channels. Able, dull |1.51|0.45|2004001|edu packimporto #1|4|sports-apparel|2|Men|26|callyable|extra large|858840ghost533554354|wheat|Pound|Unknown|15|barantiationought| +1751|AAAAAAAAGNGAAAAA|1999-10-28|2001-10-26|Increased, golden properties used to integrate over islamic losses. Funny, responsible chapters will support home members. So obvious criteria hunt namely new items. Wrong, pale|3.64|0.45|10003001|exportiunivamalg #1|3|dvd/vcr players|10|Electronics|26|callyable|N/A|7222honeydew10158691|olive|Ton|Unknown|84|oughtantiationought| +1752|AAAAAAAAGNGAAAAA|2001-10-27||Increased, golden properties used to integrate over islamic losses. Funny, responsible chapters will support home members. So obvious criteria hunt namely new items. Wrong, pale|3.18|0.45|10003001|univunivamalg #1|3|travel|9|Books|26|callyable|N/A|95448858718sienna097|wheat|Ounce|Unknown|5|ableantiationought| +1753|AAAAAAAAJNGAAAAA|1997-10-27||Boys measure else towns. Advertisements challenge just prominent, local areas; other, singl|4.49|3.18|7007004|brandbrand #4|7|decor|7|Home|63|prically|N/A|5032400tomato2366637|navy|Gross|Unknown|43|priantiationought| +1754|AAAAAAAAKNGAAAAA|1997-10-27|2000-10-26|Technologies produce for a streets. Geographical adults recognise for example raw accounts. Conservative, normal functi|2.60|1.74|10015004|scholaramalgamalg #4|15|portable|10|Electronics|545|antieseanti|N/A|5543819041papaya9935|sky|Bunch|Unknown|7|eseantiationought| +1755|AAAAAAAAKNGAAAAA|2000-10-27||Technologies produce for a streets. Geographical adults recognise for example raw accounts. Conservative, normal functi|6.78|5.89|9010004|univunivamalg #4|10|travel|9|Books|545|antieseanti|N/A|5543819041papaya9935|papaya|Unknown|Unknown|6|antiantiationought| +1756|AAAAAAAAMNGAAAAA|1997-10-27|1999-10-27|Nice organs exceed apart sure areas. Adult, islamic christians identify then with a countries. Happ|4.80|3.79|8011007|amalgmaxi #7|11|archery|8|Sports|147|ationeseought|N/A|91987smoke2770023882|indian|N/A|Unknown|15|callyantiationought| +1757|AAAAAAAAMNGAAAAA|1999-10-28|2001-10-26|More american objects will not come asleep cases. Independent unions might not meet available, secret lines. Steps would discourage. Clean, social factors might not cut political, domestic studies.|8.81|3.79|7007008|brandbrand #8|7|decor|7|Home|147|ationeseought|N/A|760686sienna40910446|tan|Bundle|Unknown|8|ationantiationought| +1758|AAAAAAAAMNGAAAAA|2001-10-27||More american objects will not come asleep cases. Independent unions might not meet available, secret lines. Steps would discourage. Clean, social factors might not cut political, domestic studies.|8.22|7.06|7007008|importoimporto #1|2|shirts|2|Men|390|ationeseought|small|760686sienna40910446|sky|Dozen|Unknown|1|eingantiationought| +1759|AAAAAAAAPNGAAAAA|1997-10-27||Unusually small programmes would lift recently social, small workshops. Offices s|1.73|0.64|9012008|importounivamalg #8|12|home repair|9|Books|474|eseationese|N/A|68595khaki4811461458|grey|Box|Unknown|64|n stantiationought| +1760|AAAAAAAAAOGAAAAA|1997-10-27|2000-10-26|Foreign, good things must get eyes. Low, thin members must rest. International looks allow. Senses should not touch. Limited, single backs would not walk opportunities; high|3.51|1.26|7002001|importobrand #1|2|bedding|7|Home|404|esebarese|N/A|5461689483midnight00|hot|Unknown|Unknown|16|barcallyationought| +1761|AAAAAAAAAOGAAAAA|2000-10-27||Proud, mutual tales may not notice more than prime, joint animals; minimum eyes may see just hundreds. Arms make old bars. Plates mind actually in an rights. Spontaneously annual|54.12|35.17|7002001|exportiamalgamalg #11|2|stereo|10|Electronics|11|oughtought|N/A|5461689483midnight00|sky|Dram|Unknown|39|oughtcallyationought| +1762|AAAAAAAACOGAAAAA|1997-10-27|1999-10-27|Public, sex|2.88|1.35|6014003|edu packbrand #3|14|estate|6|Jewelry|524|eseableanti|N/A|572614262metallic716|olive|Pound|Unknown|58|ablecallyationought| +1763|AAAAAAAACOGAAAAA|1999-10-28|2001-10-26|Public, sex|9.09|1.35|4004002|edu packedu pack #2|14|athletic|4|Shoes|524|eseableanti|large|572614262metallic716|mint|Dram|Unknown|33|pricallyationought| +1764|AAAAAAAACOGAAAAA|2001-10-27||At least new words used to join earlier select young areas. Resources should see women. |4.80|2.92|4004002|importocorp #5|2|diamonds|6|Jewelry|524|eseableanti|N/A|36490429619lemon0062|thistle|Dram|Unknown|3|esecallyationought| +1765|AAAAAAAAFOGAAAAA|1997-10-27||High years wait easily needs. Intense metals see procedures; a|4.51|3.83|6015006|scholarbrand #6|15|custom|6|Jewelry|509|n stbaranti|N/A|737514789388saddle15|saddle|Bunch|Unknown|27|anticallyationought| +1766|AAAAAAAAGOGAAAAA|1997-10-27|2000-10-26|Small interviews would not find more; solicitors may not throw logically. Increasing, small brothers take very sounds. Metres know here. Largely royal minutes cou|27.91|20.37|1004001|edu packamalg #1|4|swimwear|1|Women|23|priable|petite|3sky2658666235152232|slate|Ton|Unknown|2|callycallyationought| +1767|AAAAAAAAGOGAAAAA|2000-10-27||Nuclear, experi|8.20|20.37|1004001|importoimporto #2|4|shirts|2|Men|218|eingoughtable|petite|116872374slate910559|rose|Pallet|Unknown|19|ationcallyationought| +1768|AAAAAAAAIOGAAAAA|1997-10-27|1999-10-27|Possibly general transactions would keep po|5.77|4.38|1001001|amalgamalg #1|1|dresses|1|Women|721|oughtableation|large|1939824sandy58426870|peach|Box|Unknown|2|eingcallyationought| +1769|AAAAAAAAIOGAAAAA|1999-10-28|2001-10-26|Possibly general transactions would keep po|0.21|4.38|1001001|exportiimporto #2|3|pants|2|Men|608|eingbarcally|petite|467549213peach529701|sandy|Case|Unknown|17|n stcallyationought| +1770|AAAAAAAAIOGAAAAA|2001-10-27||Young boys ought to execute on a functions. Small plants give opening, current lands; injuries emerge shortly. Well huge police can insist. Huge boys shall try home only historical auth|4.02|3.13|5004001|edu packscholar #1|4|classical|5|Music|201|oughtbarable|N/A|6822353920magenta085|purple|Gram|Unknown|6|barationationought| +1771|AAAAAAAALOGAAAAA|1997-10-27||Drugs pay kids. Overall, necessary numbers should hurt profoundly common, likely changes. Present, unlikely changes can try rather from a documents. Sa|3.83|1.87|10006004|corpunivamalg #4|6|musical|10|Electronics|223|priableable|N/A|steel959720261346181|orange|Carton|Unknown|20|oughtationationought| +1772|AAAAAAAAMOGAAAAA|1997-10-27|2000-10-26|Other acts used to prove as forward, flat hundreds. Heavily single proposals guarantee actually american, theoretical solicito|0.57|0.50|3004001|edu packexporti #1|4|school-uniforms|3|Children|591|oughtn stanti|small|86942269violet543923|navy|N/A|Unknown|21|ableationationought| +1773|AAAAAAAAMOGAAAAA|2000-10-27||Other acts used to prove as forward, flat hundreds. Heavily single proposals guarantee actually american, theoretical solicito|2.87|0.50|5003002|exportischolar #2|4|pop|5|Music|805|antibareing|N/A|367362598tomato10150|puff|Tbl|Unknown|32|priationationought| +1774|AAAAAAAAOOGAAAAA|1997-10-27|1999-10-27|Long useful rates would buy hours. Willing, absolute teachers lay electronic fields. Years shall distinguish better only doubtful heads. As little affairs require perhaps ridiculous, enthusia|3.16|1.67|4002001|importoedu pack #1|2|mens|4|Shoes|337|ationpripri|small|006plum1853665998109|yellow|Lb|Unknown|12|eseationationought| +1775|AAAAAAAAOOGAAAAA|1999-10-28|2001-10-26|Long useful rates would buy hours. Willing, absolute teachers lay electronic fields. Years shall distinguish better only doubtful heads. As little affairs require perhaps ridiculous, enthusia|2.43|1.87|4002001|univbrand #8|2|jewelry boxes|6|Jewelry|337|ationpripri|N/A|127rose7616861003293|snow|Ounce|Unknown|15|antiationationought| +1776|AAAAAAAAOOGAAAAA|2001-10-27||Parties shall find. Perfect officers ought to like then other, relative questions. Famous seeds cannot receive equally. Different, ideal laws pay of course.|3.30|1.87|2002001|importoimporto #1|2|shirts|2|Men|337|ationpripri|large|127rose7616861003293|violet|Tsp|Unknown|2|callyationationought| +1777|AAAAAAAABPGAAAAA|1997-10-27||Now basic wo|0.40|0.33|6013002|exportibrand #2|13|loose stones|6|Jewelry|880|bareingeing|N/A|violet49552433332445|turquoise|Lb|Unknown|18|ationationationought| +1778|AAAAAAAACPGAAAAA|1997-10-27|2000-10-26|Economic cars support extremely important responsibilities. Better other teachers forgive together with a jobs. Educational, political times will recognise. Widely final |61.81|33.37|4002001|importoedu pack #1|2|mens|4|Shoes|115|antioughtought|medium|083743543725salmon86|sienna|N/A|Unknown|5|eingationationought| +1779|AAAAAAAACPGAAAAA|2000-10-27||Thousands can make virtually good theories. Large techniques may not climb however. New farmers represent previous men. Ever liberal damages shall not recognise |26.02|33.37|4002001|exportibrand #4|2|loose stones|6|Jewelry|115|antioughtought|N/A|083743543725salmon86|puff|Pound|Unknown|12|n stationationought| +1780|AAAAAAAAEPGAAAAA|1997-10-27|1999-10-27|Only white accounts ought to offer badly new, relative christians. Direct, obvious tensions go solutions. International, impossible impr|8.00|3.60|10001013|amalgunivamalg #13|1|cameras|10|Electronics|48|eingese|N/A|314red85990711964825|khaki|Carton|Unknown|29|bareingationought| +1781|AAAAAAAAEPGAAAAA|1999-10-28|2001-10-26|English theories experience only in charge of the office|5.22|3.60|2002002|importoimporto #2|2|shirts|2|Men|48|eingese|large|314red85990711964825|turquoise|Ounce|Unknown|62|oughteingationought| +1782|AAAAAAAAEPGAAAAA|2001-10-27||Active eyes might assure firm fingers. Extrem|3.82|3.60|2002002|amalgscholar #1|1|rock|5|Music|48|eingese|N/A|314red85990711964825|rose|Cup|Unknown|43|ableeingationought| +1783|AAAAAAAAHPGAAAAA|1997-10-27||British groups will not extend. Friends ought to divert views. Decisive clothes generate. Bl|3.43|2.50|3004002|edu packexporti #2|4|school-uniforms|3|Children|202|ablebarable|large|560801olive640826086|dim|Bunch|Unknown|26|prieingationought| +1784|AAAAAAAAIPGAAAAA|1997-10-27|2000-10-26|Qual|3.80|2.50|3004001|edu packexporti #1|4|school-uniforms|3|Children|359|n stantipri|N/A|280185822236yellow31|khaki|Ounce|Unknown|35|eseeingationought| +1785|AAAAAAAAIPGAAAAA|2000-10-27||Pr|7.60|2.50|3004001|exportiexporti #2|4|toddlers|3|Children|164|n stantipri|small|31773red257741745263|lawn|N/A|Unknown|5|antieingationought| +1786|AAAAAAAAKPGAAAAA|1997-10-27|1999-10-27|Just direct bills co-ordinate by a troops. Clothes belong old, essent|4.76|3.76|7007007|brandbrand #7|7|decor|7|Home|231|oughtpriable|N/A|96yellow036758963624|violet|Lb|Unknown|60|callyeingationought| +1787|AAAAAAAAKPGAAAAA|1999-10-28|2001-10-26|Just direct bills co-ordinate by a troops. Clothes belong old, essent|0.61|0.28|7007007|amalgamalg #2|7|dresses|1|Women|538|oughtpriable|medium|747blue2876977428246|red|Cup|Unknown|9|ationeingationought| +1788|AAAAAAAAKPGAAAAA|2001-10-27||Available, successful methods give free cases. Estimated, inner trees drink mad situations. Then h|3.66|2.08|7007007|edu packmaxi #9|7|tennis|8|Sports|18|eingought|N/A|54217597white9374328|violet|Box|Unknown|93|eingeingationought| +1789|AAAAAAAANPGAAAAA|1997-10-27||Nice charges live new, military men; red tonnes should seem islamic methods. |6.44|3.60|2002002|importoimporto #2|2|shirts|2|Men|257|ationantiable|small|3black67139949982182|peach|Lb|Unknown|5|n steingationought| +1790|AAAAAAAAOPGAAAAA|1997-10-27|2000-10-26|Arrangeme|12.63|10.60|3004001|edu packexporti #1|4|school-uniforms|3|Children|201|oughtbarable|large|3336078846steel18454|papaya|Unknown|Unknown|30|barn stationought| +1791|AAAAAAAAOPGAAAAA|2000-10-27||Arrangeme|1.49|10.60|3004001|edu packedu pack #2|4|athletic|4|Shoes|728|oughtbarable|petite|824687850984ghost420|sienna|Ton|Unknown|5|oughtn stationought| +1792|AAAAAAAAAAHAAAAA|1997-10-27|1999-10-27|Likely doctors give most. Awful problems att|2.16|1.38|8003003|exportinameless #3|3|basketball|8|Sports|68|eingcally|N/A|34117808sky609409345|wheat|Pallet|Unknown|18|ablen stationought| +1793|AAAAAAAAAAHAAAAA|1999-10-28|2001-10-26|Likely doctors give most. Awful problems att|4.00|2.92|5003002|exportischolar #2|3|pop|5|Music|461|eingcally|N/A|34117808sky609409345|peach|Dram|Unknown|68|prin stationought| +1794|AAAAAAAAAAHAAAAA|2001-10-27||Likely doctors give most. Awful problems att|3.65|2.92|5003002|importoimporto #1|2|shirts|2|Men|461|oughtcallyese|large|purple12334810398815|white|Box|Unknown|38|esen stationought| +1795|AAAAAAAADAHAAAAA|1997-10-27||Forces require more new examples. Also narrow students take files. Native, important objectives ought to release still legs. Difficulties might say mainly. Years|2.33|1.25|9009008|maximaxi #8|9|science|9|Books|453|priantiese|N/A|8731808tomato9792967|sienna|Bundle|Unknown|9|antin stationought| +1796|AAAAAAAAEAHAAAAA|1997-10-27|2000-10-26|Here unlikely advantages might go merely members. Immediately different pp. intrude. Different holidays|4.08|2.61|5002001|importoscholar #1|2|country|5|Music|198|eingn stought|N/A|7512628653939spring9|sandy|Each|Unknown|3|callyn stationought| +1797|AAAAAAAAEAHAAAAA|2000-10-27||American times may not prevent sufficient contents. Clubs used to st|18.28|8.77|5002001|importoamalg #2|2|fragrances|1|Women|198|eingn stought|N/A|9891wheat38520502293|plum|Oz|Unknown|17|ationn stationought| +1798|AAAAAAAAGAHAAAAA|1997-10-27|1999-10-27|Instead good proceedings would pay traditional subjects. Therefore industrial stones would convey. Essentially large nations|7.94|4.68|10007014|brandunivamalg #14|7|personal|10|Electronics|461|oughtcallyese|N/A|3084006592salmon6102|sky|Tbl|Unknown|24|eingn stationought| +1799|AAAAAAAAGAHAAAAA|1999-10-28|2001-10-26|Instead good proceedings would pay traditional subjects. Therefore industrial stones would convey. Essentially large nations|3.22|1.86|10007014|exportischolar #2|7|pop|5|Music|11|oughtcallyese|N/A|5949363547216peach68|slate|Cup|Unknown|2|n stn stationought| +1800|AAAAAAAAGAHAAAAA|2001-10-27||Negative classes may not pinpoint later peculiar, possible instruments. D|0.54|0.17|9001001|amalgmaxi #1|1|arts|9|Books|486|callyeingese|N/A|0honeydew03555390863|ivory|N/A|Unknown|73|barbareingought| +1801|AAAAAAAAJAHAAAAA|1997-10-27||Other, public activities fill there internal, forward cars. Consultants shall bel|2.31|1.87|7012002|importonameless #2|12|paint|7|Home|265|anticallyable|N/A|129753pink0660056537|seashell|Dozen|Unknown|82|oughtbareingought| +1802|AAAAAAAAKAHAAAAA|1997-10-27|2000-10-26|Long, fine teachers should not occur. Particular, monetary children organise in situ on to a tests; details ma|4.40|3.52|3002001|importoexporti #1|2|infants|3|Children|891|oughtn steing|large|1248526tomato2058894|purple|Ounce|Unknown|23|ablebareingought| +1803|AAAAAAAAKAHAAAAA|2000-10-27||Laws might not let local values. Rooms get firms. Also visual applications stop difficult, good plates. Officials provide with a years. However western stores introduce aware minds. |0.68|0.43|3002001|amalgedu pack #2|2|womens|4|Shoes|891|oughtn steing|petite|1248526tomato2058894|yellow|Carton|Unknown|22|pribareingought| +1804|AAAAAAAAMAHAAAAA|1997-10-27|1999-10-27|Interesting, inevitable regions continue most proposed ways; carefully classi|37.24|29.41|10003003|exportiunivamalg #3|3|dvd/vcr players|10|Electronics|502|ablebaranti|N/A|58red023981842256614|smoke|Lb|Unknown|9|esebareingought| +1805|AAAAAAAAMAHAAAAA|1999-10-28|2001-10-26|Blue, human years should not spend. Social designers will complain more electronic windo|1.28|29.41|2004002|edu packimporto #2|4|sports-apparel|2|Men|31|ablebaranti|petite|6066022441white64320|rosy|Bunch|Unknown|29|antibareingought| +1806|AAAAAAAAMAHAAAAA|2001-10-27||Blue, human years should not spend. Social designers will complain more electronic windo|4.69|3.89|2004002|edu packimporto #1|4|sports-apparel|2|Men|31|oughtpri|small|6066022441white64320|peru|Case|Unknown|11|callybareingought| +1807|AAAAAAAAPAHAAAAA|1997-10-27||Subjective views conduct to a children. Yet english officials go. Available candidates happe|1.14|0.77|2004002|edu packimporto #2|4|sports-apparel|2|Men|107|ationbarought|petite|127672orange04245424|saddle|Lb|Unknown|39|ationbareingought| +1808|AAAAAAAAABHAAAAA|1997-10-27|2000-10-26|Old sources pull later examples. Rich others ought to e|6.47|4.59|8012009|importomaxi #9|12|guns|8|Sports|41|oughtese|N/A|790654221lime3772433|red|Box|Unknown|84|eingbareingought| +1809|AAAAAAAAABHAAAAA|2000-10-27||Scottish, local |6.56|3.08|8012009|corpbrand #10|6|rugs|7|Home|74|oughtese|N/A|790654221lime3772433|moccasin|Ounce|Unknown|38|n stbareingought| +1810|AAAAAAAACBHAAAAA|1997-10-27|1999-10-27|Extra, lesser arms formulate as deaths. Important, |2.15|1.33|8009001|maxinameless #1|9|optics|8|Sports|433|pripriese|N/A|69656mint49821613083|seashell|Bunch|Unknown|88|baroughteingought| +1811|AAAAAAAACBHAAAAA|1999-10-28|2001-10-26|British, american parties say through a conditions; possible managers should put international, creative sides. Individual steps can see rather to no affairs. Philosophical meanings provide|2.39|0.93|4004002|edu packedu pack #2|4|athletic|4|Shoes|129|pripriese|extra large|80114720steel1326780|rose|Cup|Unknown|34|oughtoughteingought| +1812|AAAAAAAACBHAAAAA|2001-10-27||British, american parties say through a conditions; possible managers should put international, creative sides. Individual steps can see rather to no affairs. Philosophical meanings provide|4.37|0.93|4004002|edu packexporti #1|4|school-uniforms|3|Children|129|pripriese|small|299975539royal329876|slate|Bundle|Unknown|22|ableoughteingought| +1813|AAAAAAAAFBHAAAAA|1997-10-27||Free stati|4.69|3.51|4001002|amalgedu pack #2|1|womens|4|Shoes|388|eingeingpri|large|4smoke41768007864537|midnight|N/A|Unknown|85|prioughteingought| +1814|AAAAAAAAGBHAAAAA|1997-10-27|2000-10-26|Now political hills used to look channels. Memories keep much. Charges manage just ever holy criteri|3.07|1.44|8002005|importonameless #5|2|baseball|8|Sports|14|eseought|N/A|73923370029611steel9|pale|Pound|Unknown|68|eseoughteingought| +1815|AAAAAAAAGBHAAAAA|2000-10-27||Firm, social years see most funds. Large, s|2.55|1.65|8002005|importoamalg #2|2|fragrances|1|Women|317|ationoughtpri|small|73923370029611steel9|sandy|Box|Unknown|69|antioughteingought| +1816|AAAAAAAAIBHAAAAA|1997-10-27|1999-10-27|Distinct scenes find more off a forces. Sufficient, level authorities will benefit therefore from a championships. Following, common views make critics. Words se|5.12|3.02|1001001|amalgamalg #1|1|dresses|1|Women|106|callybarought|large|58417turquoise045917|gainsboro|Case|Unknown|14|callyoughteingought| +1817|AAAAAAAAIBHAAAAA|1999-10-28|2001-10-26|Only, agricultural words would accept under way; in addition d|90.41|3.02|2001002|amalgimporto #2|1|accessories|2|Men|569|n stcallyanti|large|3729seashell45596260|pink|Bunch|Unknown|57|ationoughteingought| +1818|AAAAAAAAIBHAAAAA|2001-10-27||Only, agricultural words would accept under way; in addition d|2.93|2.13|5002001|importoscholar #1|2|country|5|Music|226|callyableable|N/A|3729seashell45596260|peru|Tbl|Unknown|21|eingoughteingought| +1819|AAAAAAAALBHAAAAA|1997-10-27||Findings buy most afraid tests. Ostensibly chief organisms act over databases. Bitter, exciting parents lose just real years. Again slow countries concentrate with a children. Huge, new policies prod|1.43|0.44|4003002|exportiedu pack #2|3|kids|4|Shoes|87|ationeing|medium|5230purple2337307671|puff|Gram|Unknown|27|n stoughteingought| +1820|AAAAAAAAMBHAAAAA|1997-10-27|2000-10-26|False, medium memories stop halfway. Institutional, spiritual sales follow strongly small prospects; as political songs should not crea|3.75|3.37|10005002|scholarunivamalg #2|5|karoke|10|Electronics|561|oughtcallyanti|N/A|28367226693tan205645|salmon|Tbl|Unknown|9|barableeingought| +1821|AAAAAAAAMBHAAAAA|2000-10-27||Recent, black pupils could improve characters. Weeks imagine ways; separately alternative effects would see quickly other ne|3.27|3.37|2003002|exportiimporto #2|3|pants|2|Men|522|oughtcallyanti|small|28367226693tan205645|lace|Gross|Unknown|1|oughtableeingought| +1822|AAAAAAAAOBHAAAAA|1997-10-27|1999-10-27|Powers could meet however short-term, usual levels. Hands may travel again into a aims. Issues wa|3.89|1.67|6016007|corpbrand #7|16|consignment|6|Jewelry|85|antieing|N/A|15648472955snow59281|white|Bundle|Unknown|24|ableableeingought| +1823|AAAAAAAAOBHAAAAA|1999-10-28|2001-10-26|Powers could meet however short-term, usual levels. Hands may travel again into a aims. Issues wa|3.27|2.81|7002004|importobrand #4|2|bedding|7|Home|85|antieing|N/A|15648472955snow59281|rose|Oz|Unknown|23|priableeingought| +1824|AAAAAAAAOBHAAAAA|2001-10-27||Powers could meet however short-term, usual levels. Hands may travel again into a aims. Issues wa|8.41|2.81|5001001|amalgscholar #1|1|rock|5|Music|522|ableableanti|N/A|15648472955snow59281|lemon|Cup|Unknown|29|eseableeingought| +1825|AAAAAAAABCHAAAAA|1997-10-27||Companies describe nationally chief, close hands. For instance opposite brothers play fairly|2.98|1.78|6002002|importocorp #2|2|diamonds|6|Jewelry|352|ableantipri|N/A|32302turquoise956469|green|Oz|Unknown|20|antiableeingought| +1826|AAAAAAAACCHAAAAA|1997-10-27|2000-10-26|Empty, double classes know clearly prior to the eyes. So french eve|0.23|0.13|6006003|corpcorp #3|6|rings|6|Jewelry|37|ationpri|N/A|689ghost842315205281|tan|Carton|Unknown|24|callyableeingought| +1827|AAAAAAAACCHAAAAA|2000-10-27||Hours shall want only able, alone police. Indeed very animals stay automatically new years; angry members would go nearly sheer sets. Medical, atlantic children help more on the subjec|0.45|0.13|2001002|amalgimporto #2|1|accessories|2|Men|398|eingn stpri|medium|689ghost842315205281|lime|Unknown|Unknown|27|ationableeingought| +1828|AAAAAAAAECHAAAAA|1997-10-27|1999-10-27|Right, other definitions tap thin risks. Present, poor leaders would continue often other, good|7.30|4.89|5003001|exportischolar #1|3|pop|5|Music|344|eseesepri|N/A|lime3321756567750037|rose|Each|Unknown|50|eingableeingought| +1829|AAAAAAAAECHAAAAA|1999-10-28|2001-10-26|Right, other definitions tap thin risks. Present, poor leaders would continue often other, good|2.46|1.79|5003001|edu packbrand #6|14|estate|6|Jewelry|103|pribarought|N/A|lime3321756567750037|purple|Lb|Unknown|22|n stableeingought| +1830|AAAAAAAAECHAAAAA|2001-10-27||Right, other definitions tap thin risks. Present, poor leaders would continue often other, good|3.00|2.07|5003001|amalgexporti #1|14|newborn|3|Children|187|pribarought|medium|614597linen198419283|bisque|Unknown|Unknown|24|barprieingought| +1831|AAAAAAAAHCHAAAAA|1997-10-27||More important regulations shall know more loose, liberal women; new police could spare always in a thousands. Whole, crucial residents must sell ahead soviet homes. Powers ought to win progra|4.32|3.45|3002002|importoexporti #2|2|infants|3|Children|52|ableanti|medium|676879345goldenrod67|violet|Ton|Unknown|7|oughtprieingought| +1832|AAAAAAAAICHAAAAA|1997-10-27|2000-10-26|Equally warm conclusions cook often regional subjects. Reasonably cheerful readers slow men. Dangerous reactions hide. So curious contrac|1.64|0.50|6003001|exporticorp #1|3|gold|6|Jewelry|35|antipri|N/A|522220snow3868040375|goldenrod|Box|Unknown|39|ableprieingought| +1833|AAAAAAAAICHAAAAA|2000-10-27||Equally warm conclusions cook often regional subjects. Reasonably cheerful readers slow men. Dangerous reactions hide. So curious contrac|4.48|3.18|6003001|exportischolar #2|3|pop|5|Music|327|ationablepri|N/A|114417seashell586953|thistle|Cup|Unknown|11|priprieingought| +1834|AAAAAAAAKCHAAAAA|1997-10-27|1999-10-27|Weekly, possible letters take loudly over neighb|3.36|2.78|5003001|exportischolar #1|3|pop|5|Music|655|antiantically|N/A|3210973639pink840993|grey|Pound|Unknown|61|eseprieingought| +1835|AAAAAAAAKCHAAAAA|1999-10-28|2001-10-26|Weekly, possible letters take loudly over neighb|8.32|2.78|7007006|brandbrand #6|3|decor|7|Home|655|antiantically|N/A|3210973639pink840993|peach|Pallet|Unknown|11|antiprieingought| +1836|AAAAAAAAKCHAAAAA|2001-10-27||Weekly, possible letters take loudly over neighb|2.83|2.78|8004007|edu packnameless #7|3|camping|8|Sports|271|antiantically|N/A|3868391288937powder4|seashell|Tsp|Unknown|1|callyprieingought| +1837|AAAAAAAANCHAAAAA|1997-10-27||High, italian orders used to move only for a lights. British, daily doctors come. Therefore small fears buy. O|1.18|0.67|8001008|amalgnameless #8|1|athletic shoes|8|Sports|162|ablecallyought|N/A|779863018953512sky53|thistle|Pound|Unknown|53|ationprieingought| +1838|AAAAAAAAOCHAAAAA|1997-10-27|2000-10-26|English, public guards enter for a friends. Others must accept physically in a bodies. Sons may get by the years. Wide, possible versions examine at least different, econ|76.16|45.69|3002001|importoexporti #1|2|infants|3|Children|190|barn stought|medium|28619378olive6160949|pink|Bunch|Unknown|64|eingprieingought| +1839|AAAAAAAAOCHAAAAA|2000-10-27||English, public guards enter for a friends. Others must accept physically in a bodies. Sons may get by the years. Wide, possible versions examine at least different, econ|7.31|45.69|6001008|amalgcorp #8|2|birdal|6|Jewelry|190|barn stought|N/A|28619378olive6160949|slate|Box|Unknown|2|n stprieingought| +1840|AAAAAAAAADHAAAAA|1997-10-27|1999-10-27|Finally nasty facilities ask so. Requirements shall continue at |3.67|1.87|1004001|edu packamalg #1|4|swimwear|1|Women|267|ationcallyable|economy|4steel01232676249052|lawn|Box|Unknown|90|bareseeingought| +1841|AAAAAAAAADHAAAAA|1999-10-28|2001-10-26|Attractive youngsters arise so in the systems. Similar, large characteristics used to operate then. Working, daily papers used to supply bit|9.51|1.87|1004001|amalgexporti #2|4|newborn|3|Children|588|eingeinganti|medium|089430331347red92138|powder|Cup|Unknown|60|oughteseeingought| +1842|AAAAAAAAADHAAAAA|2001-10-27||Attractive youngsters arise so in the systems. Similar, large characteristics used to operate then. Working, daily papers used to supply bit|2.92|1.87|1004001|exportiamalg #1|3|maternity|1|Women|74|eseation|medium|089430331347red92138|royal|Ounce|Unknown|62|ableeseeingought| +1843|AAAAAAAADDHAAAAA|1997-10-27||Other, present styles think obviously beforehand advanced characteristics. Entirely civil pp. keep with a schools. Angrily outstanding pa|1.48|0.53|3002002|importoexporti #2|2|infants|3|Children|66|callycally|small|678009779352543hot11|sky|Lb|Unknown|32|prieseeingought| +1844|AAAAAAAAEDHAAAAA|1997-10-27|2000-10-26|Months cope new, significant guns. Following, other years might boil most instead elected tourists. Speci|4.32|1.64|4003001|exportiedu pack #1|3|kids|4|Shoes|608|eingbarcally|medium|pale0699477656233412|rosy|N/A|Unknown|10|eseeseeingought| +1845|AAAAAAAAEDHAAAAA|2000-10-27||Developers can buy inherently here other shar|4.86|1.64|4003001|edu packmaxi #10|4|entertainments|9|Books|985|antieingn st|N/A|pale0699477656233412|peach|N/A|Unknown|12|antieseeingought| +1846|AAAAAAAAGDHAAAAA|1997-10-27|1999-10-27|Very large men occupy then actual, considerable years. Likely, strange daughters produce meanwhile new views. Ministe|5.94|2.37|4001001|amalgedu pack #1|1|womens|4|Shoes|319|n stoughtpri|medium|orchid81490916386675|seashell|Ton|Unknown|61|callyeseeingought| +1847|AAAAAAAAGDHAAAAA|1999-10-28|2001-10-26|Very large men occupy then actual, considerable years. Likely, strange daughters produce meanwhile new views. Ministe|4.72|3.82|4001001|maxibrand #8|9|mattresses|7|Home|319|n stoughtpri|N/A|orchid81490916386675|puff|Dram|Unknown|26|ationeseeingought| +1848|AAAAAAAAGDHAAAAA|2001-10-27||Very large men occupy then actual, considerable years. Likely, strange daughters produce meanwhile new views. Ministe|81.80|3.82|2004001|edu packimporto #1|4|sports-apparel|2|Men|319|n stoughtpri|economy|57saddle925857273542|peach|Pallet|Unknown|29|eingeseeingought| +1849|AAAAAAAAJDHAAAAA|1997-10-27||Continued ideas reflect only still other prices. Actually historical weeks help low, appropriate companies; recent provisions widen du|2.16|1.79|9006002|corpmaxi #2|6|parenting|9|Books|137|ationpriought|N/A|541810767gainsboro82|seashell|Bundle|Unknown|91|n steseeingought| +1850|AAAAAAAAKDHAAAAA|1997-10-27|2000-10-26|National, wrong sources must rot. Cases take often for a words. Hours shall tell particularly popular nurses; special, serious gr|5.00|4.05|7010001|univnameless #1|10|flatware|7|Home|71|oughtation|N/A|333plum2043374862674|burnished|Gram|Unknown|4|barantieingought| +1851|AAAAAAAAKDHAAAAA|2000-10-27||National, wrong sources must rot. Cases take often for a words. Hours shall tell particularly popular nurses; special, serious gr|1.49|1.07|7010001|edu packscholar #2|4|classical|5|Music|71|oughtation|N/A|333plum2043374862674|cornflower|Each|Unknown|3|oughtantieingought| +1852|AAAAAAAAMDHAAAAA|1997-10-27|1999-10-27|Asleep rights continue over papers. Yesterday poor combinations ought to like votes. Hardly similar manufacturers used to see groups. Rel|65.51|44.54|7016003|corpnameless #3|16|furniture|7|Home|182|ableeingought|N/A|02179983106tan716358|light|Lb|Unknown|33|ableantieingought| +1853|AAAAAAAAMDHAAAAA|1999-10-28|2001-10-26|High, dominant sides turn. True duties ap|3.80|2.85|7016003|edu packamalg #2|4|swimwear|1|Women|182|ableeingought|medium|02179983106tan716358|forest|Box|Unknown|25|priantieingought| +1854|AAAAAAAAMDHAAAAA|2001-10-27||High, dominant sides turn. True duties ap|3.32|2.85|7016003|edu packexporti #1|4|school-uniforms|3|Children|182|ableeingought|N/A|9276antique001843484|olive|Pound|Unknown|20|eseantieingought| +1855|AAAAAAAAPDHAAAAA|1997-10-27||Difficult, specific days would hold long lakes. Wild flowers must not go here. British, horrible plates shall put big, wooden charges. Others cannot sail so central cou|5.08|3.30|2001002|amalgimporto #2|1|accessories|2|Men|321|oughtablepri|extra large|7moccasin54893934011|goldenrod|Pallet|Unknown|34|antiantieingought| +1856|AAAAAAAAAEHAAAAA|1997-10-27|2000-10-26|Precise, short-term minutes attribute able, little results. Powerful, serious services replace hardly constant bodies. Clients prove well in a hotels. High|2.94|0.97|6015001|scholarbrand #1|15|custom|6|Jewelry|206|callybarable|N/A|109112sandy962403032|metallic|Unknown|Unknown|31|callyantieingought| +1857|AAAAAAAAAEHAAAAA|2000-10-27||Precise, short-term minutes attribute able, little results. Powerful, serious services replace hardly constant bodies. Clients prove well in a hotels. High|1.19|1.02|1001002|amalgamalg #2|1|dresses|1|Women|821|callybarable|medium|4262105704powder3933|wheat|Bunch|Unknown|44|ationantieingought| +1858|AAAAAAAACEHAAAAA|1997-10-27|1999-10-27|National sea|29.68|17.80|7015009|scholarnameless #9|15|tables|7|Home|453|priantiese|N/A|773732blue5152485755|cyan|Dozen|Unknown|47|eingantieingought| +1859|AAAAAAAACEHAAAAA|1999-10-28|2001-10-26|National sea|3.23|1.19|6015004|scholarbrand #4|15|custom|6|Jewelry|288|eingeingable|N/A|82aquamarine48613794|navajo|Unknown|Unknown|19|n stantieingought| +1860|AAAAAAAACEHAAAAA|2001-10-27||National sea|3.33|1.79|6015004|importoedu pack #1|15|mens|4|Shoes|144|eingeingable|small|2navajo1086742095849|smoke|Cup|Unknown|54|barcallyeingought| +1861|AAAAAAAAFEHAAAAA|1997-10-27||Healthy forests could admit regulatory, single heads. S|1.53|0.71|4001002|amalgedu pack #2|1|womens|4|Shoes|102|ablebarought|small|3rosy988122346887254|indian|Case|Unknown|17|oughtcallyeingought| +1862|AAAAAAAAGEHAAAAA|1997-10-27|2000-10-26|Details design well with th|3.01|1.80|8001003|amalgnameless #3|1|athletic shoes|8|Sports|18|eingought|N/A|87papaya049556294335|sky|Oz|Unknown|55|ablecallyeingought| +1863|AAAAAAAAGEHAAAAA|2000-10-27||Exclusively impossible children make at the walls. Artists can win well elegantly other jobs. R|0.68|1.80|8001003|exportibrand #4|1|kids|7|Home|877|ationationeing|N/A|43215seashell8806273|lime|Pound|Unknown|11|pricallyeingought| +1864|AAAAAAAAIEHAAAAA|1997-10-27|1999-10-27|Players could master since. Wide impossible reactions must |2.24|1.32|6013001|exportibrand #1|13|loose stones|6|Jewelry|162|ablecallyought|N/A|1562934075664red8255|purple|Carton|Unknown|23|esecallyeingought| +1865|AAAAAAAAIEHAAAAA|1999-10-28|2001-10-26|Players could master since. Wide impossible reactions must |6.97|5.99|6013001|edu packedu pack #2|4|athletic|4|Shoes|329|ablecallyought|large|1562934075664red8255|plum|Box|Unknown|16|anticallyeingought| +1866|AAAAAAAAIEHAAAAA|2001-10-27||Players could master since. Wide impossible reactions must |2.26|1.46|5002001|importoscholar #1|2|country|5|Music|874|ablecallyought|N/A|4328443smoke95955422|peach|Cup|Unknown|22|callycallyeingought| +1867|AAAAAAAALEHAAAAA|1997-10-27||Men will take carefully too local schemes. Later british panels find perhaps wide, recent centuries. Forests will keep late regional years. Best special days might p|2.73|1.41|3001002|amalgexporti #2|1|newborn|3|Children|141|oughteseought|extra large|54283dodger150399841|snow|Carton|Unknown|34|ationcallyeingought| +1868|AAAAAAAAMEHAAAAA|1997-10-27|2000-10-26|So married arts must not land somewhat. Specific, long cases cover today existing, southern reasons; well substantial features would not sell b|0.86|0.77|8012009|importomaxi #9|12|guns|8|Sports|241|oughteseable|N/A|62225723sky532435539|puff|Each|Unknown|37|eingcallyeingought| +1869|AAAAAAAAMEHAAAAA|2000-10-27||So married arts must not land somewhat. Specific, long cases cover today existing, southern reasons; well substantial features would not sell b|60.31|0.77|8012009|importoedu pack #2|2|mens|4|Shoes|241|oughteseable|large|665750543823chiffon6|seashell|Cup|Unknown|59|n stcallyeingought| +1870|AAAAAAAAOEHAAAAA|1997-10-27|1999-10-27|Warm councils cou|2.84|1.64|2002001|importoimporto #1|2|shirts|2|Men|325|antiablepri|large|370chartreuse6223959|sky|Each|Unknown|50|barationeingought| +1871|AAAAAAAAOEHAAAAA|1999-10-28|2001-10-26|Most angry men think by a years. Even general facts set upon the experiences. Years could bear deals. American concepts take possible quantities; musical allowances ought to deserve often|0.42|1.64|2002001|importounivamalg #2|2|camcorders|10|Electronics|844|eseeseeing|N/A|370chartreuse6223959|midnight|Bunch|Unknown|51|oughtationeingought| +1872|AAAAAAAAOEHAAAAA|2001-10-27||Most angry men think by a years. Even general facts set upon the experiences. Years could bear deals. American concepts take possible quantities; musical allowances ought to deserve often|1.33|1.64|2002001|importoamalg #1|2|fragrances|1|Women|844|eseeseeing|small|914438289000purple01|rose|Dram|Unknown|100|ableationeingought| +1873|AAAAAAAABFHAAAAA|1997-10-27||Military, national problems run international, good tears. Actually similar rooms let social, international books. Enterprises should amount too con|0.28|0.16|2001002|amalgimporto #2|1|accessories|2|Men|221|oughtableable|medium|54midnight8216452774|medium|Box|Unknown|9|priationeingought| +1874|AAAAAAAACFHAAAAA|1997-10-27|2000-10-26|Asleep, regular month|0.91|0.72|7011005|amalgnameless #5|11|accent|7|Home|640|baresecally|N/A|9steel02802068078507|sienna|Cup|Unknown|27|eseationeingought| +1875|AAAAAAAACFHAAAAA|2000-10-27||Possibly civil stars should review in addition widely valuable letters; big visitors t|1.82|1.03|2001002|amalgimporto #2|1|accessories|2|Men|425|baresecally|large|9steel02802068078507|tomato|Dozen|Unknown|48|antiationeingought| +1876|AAAAAAAAEFHAAAAA|1997-10-27|1999-10-27|Ruling engines set under private, total re|21.71|12.59|1001001|amalgamalg #1|1|dresses|1|Women|1|ought|large|771664845250902lime4|rosy|N/A|Unknown|46|callyationeingought| +1877|AAAAAAAAEFHAAAAA|1999-10-28|2001-10-26|Frankly sufficient men use either ordinary findings; offices shoul|8.80|4.92|1001001|exportiimporto #2|3|pants|2|Men|211|ought|medium|771664845250902lime4|metallic|Bunch|Unknown|55|ationationeingought| +1878|AAAAAAAAEFHAAAAA|2001-10-27||Frankly sufficient men use either ordinary findings; offices shoul|6.98|2.23|1001001|edu packimporto #1|4|sports-apparel|2|Men|418|ought|medium|771664845250902lime4|frosted|Lb|Unknown|72|eingationeingought| +1879|AAAAAAAAHFHAAAAA|1997-10-27||Independent, main patients belong ver|4.72|1.93|1002002|importoamalg #2|2|fragrances|1|Women|201|oughtbarable|large|52853655seashell6740|rosy|Tbl|Unknown|40|n stationeingought| +1880|AAAAAAAAIFHAAAAA|1997-10-27|2000-10-26|Indeed old events would count yesterday big daily activities. Other passages ban very years|5.42|3.57|10014011|edu packamalgamalg #11|14|automotive|10|Electronics|218|eingoughtable|N/A|18grey09578252872418|sandy|Dozen|Unknown|96|bareingeingought| +1881|AAAAAAAAIFHAAAAA|2000-10-27||Systems will return |1.81|3.57|10014011|univbrand #2|10|jewelry boxes|6|Jewelry|179|eingoughtable|N/A|1532869359406167sky7|tomato|Tbl|Unknown|55|oughteingeingought| +1882|AAAAAAAAKFHAAAAA|1997-10-27|1999-10-27|Improvements direct inevitable, wrong arrangements. White songs must say in a centuries. Dark, narrow schools go always possible stories. Progressive, certain parts might sleep to the players. True,|1.43|1.02|3001001|amalgexporti #1|1|newborn|3|Children|267|ationcallyable|large|2578198078yellow1240|rosy|Ton|Unknown|25|ableeingeingought| +1883|AAAAAAAAKFHAAAAA|1999-10-28|2001-10-26|Improvements direct inevitable, wrong arrangements. White songs must say in a centuries. Dark, narrow schools go always possible stories. Progressive, certain parts might sleep to the players. True,|19.44|1.02|3001001|importomaxi #12|2|business|9|Books|267|ationcallyable|N/A|874lace4207066797477|white|Tsp|Unknown|24|prieingeingought| +1884|AAAAAAAAKFHAAAAA|2001-10-27||Improvements direct inevitable, wrong arrangements. White songs must say in a centuries. Dark, narrow schools go always possible stories. Progressive, certain parts might sleep to the players. True,|7.82|1.02|3001001|exportischolar #1|2|pop|5|Music|569|ationcallyable|N/A|05922tan844532271689|salmon|Unknown|Unknown|8|eseeingeingought| +1885|AAAAAAAANFHAAAAA|1997-10-27||Reluctant terms meet fingers. High long authors must offer local, new items. Political ways must help |1.05|0.67|5003002|exportischolar #2|3|pop|5|Music|202|ablebarable|N/A|thistle1717903613306|sienna|Oz|Unknown|31|antieingeingought| +1886|AAAAAAAAOFHAAAAA|1997-10-27|2000-10-26|Simple, ideal images ought to stand accid|7.19|2.51|9008003|namelessmaxi #3|8|romance|9|Books|267|ationcallyable|N/A|57418rosy48163439882|yellow|Tbl|Unknown|13|callyeingeingought| +1887|AAAAAAAAOFHAAAAA|2000-10-27||Instead historical signs say sales. Costs could ask just for a ways. Also obvious difficulties go a|0.79|2.51|3002002|importoexporti #2|2|infants|3|Children|409|n stbarese|large|679lace5940795632636|tan|Ton|Unknown|95|ationeingeingought| +1888|AAAAAAAAAGHAAAAA|1997-10-27|1999-10-27|Now rare strengths will not bear still essential public clothes. Good, regional effects must increase on a earnings. Criminal, central patterns could not |5.64|4.73|6012001|importobrand #1|12|costume|6|Jewelry|425|antiableese|N/A|7785752495541azure01|snow|Bunch|Unknown|55|eingeingeingought| +1889|AAAAAAAAAGHAAAAA|1999-10-28|2001-10-26|Etc maximum restrictions should use for a observers. Just flexible employers frighten at last other pupils; british reasons must not tend howev|41.66|26.24|4003002|exportiedu pack #2|3|kids|4|Shoes|317|ationoughtpri|large|21929014265864sky657|puff|Gross|Unknown|31|n steingeingought| +1890|AAAAAAAAAGHAAAAA|2001-10-27||Living students show; residents may realize about minutes. Available, separ|29.72|19.31|4003002|exportischolar #1|3|pop|5|Music|16|ationoughtpri|N/A|21929014265864sky657|tan|Gross|Unknown|21|barn steingought| +1891|AAAAAAAADGHAAAAA|1997-10-27||Current feelings c|1.37|0.61|2003002|exportiimporto #2|3|pants|2|Men|767|ationcallyation|small|372magenta1258861305|lace|Carton|Unknown|37|oughtn steingought| +1892|AAAAAAAAEGHAAAAA|1997-10-27|2000-10-26|Legal difficulties take wrong, controversial members. Men will not sing weapons. High conditions take fair. Proposals will not pay however bright stones. Free authorities shall use constan|9.49|7.49|6013005|exportibrand #5|13|loose stones|6|Jewelry|451|oughtantiese|N/A|25792005honeydew7140|rose|N/A|Unknown|65|ablen steingought| +1893|AAAAAAAAEGHAAAAA|2000-10-27||American, comparative buildings return separate enemies. Difficult flowers live only for a machines; likely, usual rules might love here germans. Agents encourage high at|9.72|7.49|6013005|importoexporti #2|2|infants|3|Children|966|oughtantiese|medium|05075885sandy6552089|sienna|Tsp|Unknown|39|prin steingought| +1894|AAAAAAAAGGHAAAAA|1997-10-27|1999-10-27|Crude men used to want versions. Immediately remarkable prisoners shall not see other m|3.23|1.03|1004001|edu packamalg #1|4|swimwear|1|Women|267|ationcallyable|small|225688491507white278|papaya|Case|Unknown|82|esen steingought| +1895|AAAAAAAAGGHAAAAA|1999-10-28|2001-10-26|Crude men used to want versions. Immediately remarkable prisoners shall not see other m|8.37|1.03|1004001|corpunivamalg #12|16|mystery|9|Books|436|ationcallyable|N/A|88261895719maroon044|royal|Oz|Unknown|24|antin steingought| +1896|AAAAAAAAGGHAAAAA|2001-10-27||Now precious types could compare unable, crucial occasions. General, quiet profits struggle on a houses. Cars go jus|4.50|1.03|1004001|exportiunivamalg #1|16|self-help|9|Books|685|antieingcally|N/A|88261895719maroon044|salmon|Carton|Unknown|45|callyn steingought| +1897|AAAAAAAAJGHAAAAA|1997-10-27||Church|3.90|3.39|5003002|exportischolar #2|3|pop|5|Music|162|ablecallyought|N/A|998thistle6567114297|wheat|Gram|Unknown|8|ationn steingought| +1898|AAAAAAAAKGHAAAAA|1997-10-27|2000-10-26|Public|0.50|0.43|4001001|amalgedu pack #1|1|womens|4|Shoes|270|barationable|petite|55601626657157red771|red|Gross|Unknown|93|eingn steingought| +1899|AAAAAAAAKGHAAAAA|2000-10-27||Public|32.09|26.95|4001001|amalgimporto #2|1|accessories|2|Men|270|barationable|medium|55601626657157red771|royal|Pound|Unknown|30|n stn steingought| +1900|AAAAAAAAMGHAAAAA|1997-10-27|1999-10-27|Economic shareholders need. App|7.14|2.28|10001014|amalgunivamalg #14|1|cameras|10|Electronics|230|barpriable|N/A|3rosy368154422736792|chocolate|Pallet|Unknown|6|barbarn stought| +1901|AAAAAAAAMGHAAAAA|1999-10-28|2001-10-26|Local things would not wish often difficult lives. Clear years may not allow fast, local vehicles. New females slide subsequently officers. Different fo|4.20|3.44|1004002|edu packamalg #2|4|swimwear|1|Women|230|barpriable|petite|9misty46748541614183|red|Ton|Unknown|26|oughtbarn stought| +1902|AAAAAAAAMGHAAAAA|2001-10-27||Local things would not wish often difficult lives. Clear years may not allow fast, local vehicles. New females slide subsequently officers. Different fo|1.79|3.44|9012007|importounivamalg #7|4|home repair|9|Books|230|barpriable|N/A|9misty46748541614183|indian|Box|Unknown|15|ablebarn stought| +1903|AAAAAAAAPGHAAAAA|1997-10-27||Rapid pro|3.11|1.52|10002017|importounivamalg #17|2|camcorders|10|Electronics|328|eingablepri|N/A|60948893586125peach7|thistle|Lb|Unknown|82|pribarn stought| +1904|AAAAAAAAAHHAAAAA|1997-10-27|2000-10-26|Human, friendly tools could become also regularly live pounds. Also individual vehicles work best about a places; sides ought to place widely since a hands. Women shall n|1.25|0.78|1004001|edu packamalg #1|4|swimwear|1|Women|639|n stprically|large|2023990367sienna4400|ghost|Case|Unknown|25|esebarn stought| +1905|AAAAAAAAAHHAAAAA|2000-10-27||Other seats replace repeatedly special elements. Volunteers could not speak more future friends; personal methods should affect for instan|4.40|3.96|1004001|amalgedu pack #2|4|womens|4|Shoes|96|n stprically|medium|608pale9304481560040|lavender|Case|Unknown|13|antibarn stought| +1906|AAAAAAAACHHAAAAA|1997-10-27|1999-10-27|Neat things might imagine for the families. Capable telecommunications meet acute stars. Standard parameters like loudly; months try later also good resources; jobs |1.05|0.92|3004001|edu packexporti #1|4|school-uniforms|3|Children|65|antically|extra large|sandy526202656876427|smoke|Oz|Unknown|14|callybarn stought| +1907|AAAAAAAACHHAAAAA|1999-10-28|2001-10-26|Neat things might imagine for the families. Capable telecommunications meet acute stars. Standard parameters like loudly; months try later also good resources; jobs |0.73|0.54|1002002|importoamalg #2|2|fragrances|1|Women|65|antically|N/A|631lime3851169140493|pale|Pallet|Unknown|83|ationbarn stought| +1908|AAAAAAAACHHAAAAA|2001-10-27||Big services shall not stay of course interviews. Only officers improve then|1.84|0.90|1002002|maxinameless #9|2|optics|8|Sports|222|ableableable|N/A|237675saddle25722620|puff|Box|Unknown|86|eingbarn stought| +1909|AAAAAAAAFHHAAAAA|1997-10-27||Badly small sentences must make important, previous weapons. Back, economic careers stand despite a differences. |0.40|0.21|4002002|importoedu pack #2|2|mens|4|Shoes|185|antieingought|petite|1556plum915134716366|pink|Dozen|Unknown|53|n stbarn stought| +1910|AAAAAAAAGHHAAAAA|1997-10-27|2000-10-26|Arguments play through a |2.51|1.73|1001001|amalgamalg #1|1|dresses|1|Women|872|ableationeing|medium|2546702190773pale489|gainsboro|Unknown|Unknown|30|baroughtn stought| +1911|AAAAAAAAGHHAAAAA|2000-10-27||Arguments play through a |6.12|2.20|1001001|scholaramalgamalg #8|15|portable|10|Electronics|872|ableationeing|N/A|2546702190773pale489|orchid|Tsp|Unknown|31|oughtoughtn stought| +1912|AAAAAAAAIHHAAAAA|1997-10-27|1999-10-27|Public standards should make maybe aspects. Values realize letters. Very unique symptoms act|2.47|0.79|6002001|importocorp #1|2|diamonds|6|Jewelry|95|antin st|N/A|lemon365983173140629|saddle|Carton|Unknown|91|ableoughtn stought| +1913|AAAAAAAAIHHAAAAA|1999-10-28|2001-10-26|Spanish, usual structures divide special, public se|1.32|0.84|6002001|exportiamalg #2|3|maternity|1|Women|175|antiationought|small|348342235rose8119540|powder|N/A|Unknown|2|prioughtn stought| +1914|AAAAAAAAIHHAAAAA|2001-10-27||Immediately american agents must assert always from a stones. National, young schools will need; foreign decisions show judicial courts. Y|0.92|0.80|7012005|importonameless #5|3|paint|7|Home|175|antiationought|N/A|348342235rose8119540|rosy|N/A|Unknown|75|eseoughtn stought| +1915|AAAAAAAALHHAAAAA|||||||amalgmaxi #6|11|archery|||33|pripri|||snow|Tbl|Unknown||antioughtn stought| +1916|AAAAAAAAMHHAAAAA|1997-10-27|2000-10-26|Comprehensive, awful perceptio|2.59|2.07|6004005|edu packcorp #5|4|bracelets|6|Jewelry|273|priationable|N/A|0436709735936grey378|snow|Tbl|Unknown|39|callyoughtn stought| +1917|AAAAAAAAMHHAAAAA|2000-10-27||Comprehensive, awful perceptio|80.53|2.07|6004005|amalgmaxi #8|4|archery|8|Sports|398|priationable|N/A|07573486662355mint94|medium|Each|Unknown|36|ationoughtn stought| +1918|AAAAAAAAOHHAAAAA|1997-10-27|1999-10-27|Private children used to stop really national, mate|2.82|1.43|9015011|scholarunivamalg #11|15|fiction|9|Books|365|anticallypri|N/A|231017powder18026655|plum|Case|Unknown|1|eingoughtn stought| +1919|AAAAAAAAOHHAAAAA|1999-10-28|2001-10-26|Private children used to stop really national, mate|1.58|1.43|9015011|namelessmaxi #12|15|romance|9|Books|365|anticallypri|N/A|542843585846489rose6|firebrick|Box|Unknown|3|n stoughtn stought| +1920|AAAAAAAAOHHAAAAA|2001-10-27||Private children used to stop really national, mate|4.91|3.14|9015011|namelesscorp #1|15|mens watch|6|Jewelry|365|anticallypri|N/A|542843585846489rose6|plum|Gram|Unknown|3|barablen stought| +1921|AAAAAAAABIHAAAAA|1997-10-27||Profitable documents would apply now. Literary, nuclear magazines might run general, proposed details. C|4.18|1.54|8016002|corpmaxi #2|16|golf|8|Sports|81|oughteing|N/A|5045purple9069617726|mint|Pound|Unknown|7|oughtablen stought| +1922|AAAAAAAACIHAAAAA|1997-10-27|2000-10-26|Considerable problems ought to clarify. Others might investigate e|3.51|2.84|5003001|exportischolar #1|3|pop|5|Music|542|ableeseanti|N/A|99158756556papaya785|salmon|Carton|Unknown|24|ableablen stought| +1923|AAAAAAAACIHAAAAA|2000-10-27||Strong examples can give quickly; again co|0.47|2.84|4002002|importoedu pack #2|2|mens|4|Shoes|298|ableeseanti|petite|0729thistle995542953|tomato|Lb|Unknown|18|priablen stought| +1924|AAAAAAAAEIHAAAAA|1997-10-27|1999-10-27|Carefully independent newspapers used to link increasingly outstanding healthy women. Important companies undertake most numerous figures. Pr|4.65|3.02|4002001|importoedu pack #1|2|mens|4|Shoes|664|esecallycally|large|43551179peru57903664|yellow|Box|Unknown|68|eseablen stought| +1925|AAAAAAAAEIHAAAAA|1999-10-28|2001-10-26|Carefully independent newspapers used to link increasingly outstanding healthy women. Important companies undertake most numerous figures. Pr|6.34|2.28|4002001|brandunivamalg #5|7|personal|10|Electronics|664|esecallycally|N/A|513681sky00949696001|sienna|Tsp|Unknown|28|antiablen stought| +1926|AAAAAAAAEIHAAAAA|2001-10-27||Critics respond successfully welcome, residential societies. Favorite results offer everyday wa|3.15|2.74|4002001|edu packcorp #7|7|bracelets|6|Jewelry|664|esecallycally|N/A|513681sky00949696001|ghost|Box|Unknown|21|callyablen stought| +1927|AAAAAAAAHIHAAAAA|1997-10-27||Reasons seem commonly related processes; h|1.64|0.98|3001002|amalgexporti #2|1|newborn|3|Children|52|ableanti|medium|776594spring88314120|sky|Bundle|Unknown|82|ationablen stought| +1928|AAAAAAAAIIHAAAAA|1997-10-27|2000-10-26|Expensive services ensur|2.88|2.36|9005009|scholarmaxi #9|5|history|9|Books|255|antiantiable|N/A|254898steel493716101|puff|Tbl|Unknown|8|eingablen stought| +1929|AAAAAAAAIIHAAAAA|2000-10-27||Expensive services ensur|4.17|3.37|9005009|scholarunivamalg #9|5|karoke|10|Electronics|255|antiantiable|N/A|254898steel493716101|lime|Ton|Unknown|78|n stablen stought| +1930|AAAAAAAAKIHAAAAA|1997-10-27|1999-10-27|Short-term instructions must plan eventually easily military women. Strange materials must look right more metropolitan rivers. Close ot|2.62|0.78|10009010|maxiunivamalg #10|9|televisions|10|Electronics|842|ableeseeing|N/A|3492seashell12861581|thistle|N/A|Unknown|89|barprin stought| +1931|AAAAAAAAKIHAAAAA|1999-10-28|2001-10-26|Short-term instructions must plan eventually easily military women. Strange materials must look right more metropolitan rivers. Close ot|7.08|5.09|10009010|edu packscholar #2|4|classical|5|Music|842|ableeseeing|N/A|211yellow37305777933|spring|Pallet|Unknown|19|oughtprin stought| +1932|AAAAAAAAKIHAAAAA|2001-10-27||Other, whole phrases must tell usually. Enough high legs form quite. Situ|44.09|26.45|10009010|edu packexporti #1|4|school-uniforms|3|Children|842|ableeseeing|small|211yellow37305777933|salmon|Dram|Unknown|50|ableprin stought| +1933|AAAAAAAANIHAAAAA|1997-10-27||Social shows appeal largely once more african clothes. Single, current groups feel somewhat courses. National aspects find minutes. Now real farmers would talk in a assu|4.89|2.05|7012004|importonameless #4|12|paint|7|Home|64|esecally|N/A|2621lime289826360037|papaya|Gross|Unknown|23|priprin stought| +1934|AAAAAAAAOIHAAAAA|1997-10-27|2000-10-26|Good, healthy buildings send following, european markets. Guilty, international suggestions cope in a runs. Streets save as please cool thousands. Administrative times use by the babies|7.18|5.74|6004007|edu packcorp #7|4|bracelets|6|Jewelry|194|esen stought|N/A|29lime10544742523283|khaki|Pound|Unknown|6|eseprin stought| +1935|AAAAAAAAOIHAAAAA|2000-10-27||Good, healthy buildings send following, european markets. Guilty, international suggestions cope in a runs. Streets save as please cool thousands. Administrative times use by the babies|8.99|5.74|3004002|edu packexporti #2|4|school-uniforms|3|Children|194|esen stought|extra large|507583royal044517637|salmon|Gross|Unknown|13|antiprin stought| +1936|AAAAAAAAAJHAAAAA|1997-10-27|1999-10-27|American, excellent police can kill for example vertical clothes.|7.18|3.23|3002001|importoexporti #1|2|infants|3|Children|137|ationpriought|small|826946968252red57201|rosy|Pallet|Unknown|32|callyprin stought| +1937|AAAAAAAAAJHAAAAA|1999-10-28|2001-10-26|American, excellent police can kill for example vertical clothes.|5.85|2.92|2002002|importoimporto #2|2|shirts|2|Men|137|ationpriought|petite|826946968252red57201|thistle|Ton|Unknown|13|ationprin stought| +1938|AAAAAAAAAJHAAAAA|2001-10-27||Very friendly conditions used to ensure rather sure clear alternatives. Grounds forecast most widesp|6.18|3.64|2002002|importoamalg #1|2|fragrances|1|Women|137|ationpriought|small|1619chocolate7951631|lime|Pound|Unknown|14|eingprin stought| +1939|AAAAAAAADJHAAAAA|1997-10-27||Immediate police face now on a cu|8.34|4.17|7007010|brandbrand #10|7|decor|7|Home|128|eingableought|N/A|73303slate8183365072|gainsboro|Oz|Unknown|13|n stprin stought| +1940|AAAAAAAAEJHAAAAA|1997-10-27|2000-10-26|Policies retain also on a claim|2.24|0.96|3004001|edu packexporti #1|4|school-uniforms|3|Children|612|ableoughtcally|small|50587490811saddle086|indian|Gram|Unknown|87|baresen stought| +1941|AAAAAAAAEJHAAAAA|2000-10-27||Local, thin interactions used to write tickets. Political su|1.81|0.96|3004001|edu packunivamalg #10|4|sports|9|Books|612|ableoughtcally|N/A|50587490811saddle086|yellow|Oz|Unknown|65|oughtesen stought| +1942|AAAAAAAAGJHAAAAA|1997-10-27|1999-10-27|Practical, related ideas cannot unravel ruling days. Late variations should keep so so complex scots; probable, pe|5.97|1.85|2002001|importoimporto #1|2|shirts|2|Men|669|n stcallycally|large|29hot637261022998951|grey|Pound|Unknown|26|ableesen stought| +1943|AAAAAAAAGJHAAAAA|1999-10-28|2001-10-26|Practical, related ideas cannot unravel ruling days. Late variations should keep so so complex scots; probable, pe|4.32|1.85|2002001|amalgscholar #2|1|rock|5|Music|209|n stcallycally|N/A|9256149tan1132110110|plum|Dozen|Unknown|2|priesen stought| +1944|AAAAAAAAGJHAAAAA|2001-10-27||Books must remember unlike, distinct buyers; specialist, labour attacks share. Multiple lines shall not find before a leaders. Soldiers dry yet at a markets. Really fit features focus e|4.20|1.51|10009007|maxiunivamalg #7|1|televisions|10|Electronics|545|n stcallycally|N/A|9256149tan1132110110|ivory|Tsp|Unknown|83|eseesen stought| +1945|AAAAAAAAJJHAAAAA|1997-10-27||Ye|5.99|5.33|3001002|amalgexporti #2|1|newborn|3|Children|159|n stantiought|extra large|592620322thistle2965|mint|Gross|Unknown|52|antiesen stought| +1946|AAAAAAAAKJHAAAAA|1997-10-27|2000-10-26|Initiatives expel wild. |2.90|1.85|5001001|amalgscholar #1|1|rock|5|Music|504|esebaranti|N/A|737wheat432948361997|royal|Cup|Unknown|31|callyesen stought| +1947|AAAAAAAAKJHAAAAA|2000-10-27||Initiatives expel wild. |9.14|1.85|5002002|importoscholar #2|2|country|5|Music|413|prioughtese|N/A|737wheat432948361997|powder|Box|Unknown|24|ationesen stought| +1948|AAAAAAAAMJHAAAAA|1997-10-27|1999-10-27|Officials calculate in the images. Military, olympic services throw apparently old photographs; exotic, wonderful children benefit|9.36|3.55|7009009|maxibrand #9|9|mattresses|7|Home|228|eingableable|N/A|51535287874lime90056|snow|Ounce|Unknown|88|eingesen stought| +1949|AAAAAAAAMJHAAAAA|1999-10-28|2001-10-26|Strengths must pay acceptable places. Campaigns must arrange in a applicants. Miles resolve curiously in a lips. Nowhere tiny men must de|3.32|2.52|1003002|exportiamalg #2|9|maternity|1|Women|228|eingableable|economy|51535287874lime90056|rosy|Ton|Unknown|24|n stesen stought| +1950|AAAAAAAAMJHAAAAA|2001-10-27||Bright, other scientists want considerably. Frequently prime schemes might isolate politically nice |88.22|66.16|9012007|importounivamalg #7|12|home repair|9|Books|228|eingableable|N/A|545297mint5340690144|indian|Bundle|Unknown|43|barantin stought| +1951|AAAAAAAAPJHAAAAA|1997-10-27||Necessary, yellow minds ought to go once more previous victims. Activities could not assume precisely specific, long-term years. Russian eyes mi|2.27|1.45|10013014|exportiamalgamalg #14|13|stereo|10|Electronics|236|callypriable|N/A|9881875658444salmon1|rosy|Pound|Unknown|88|oughtantin stought| +1952|AAAAAAAAAKHAAAAA|1997-10-27|2000-10-26|Key names make somewhere. Women develop moreover favorite, widespread needs; also new|6.76|2.43|8008003|namelessnameless #3|8|outdoor|8|Sports|496|callyn stese|N/A|55150yellow416778180|plum|Box|Unknown|64|ableantin stought| +1953|AAAAAAAAAKHAAAAA|2000-10-27||Key names make somewhere. Women develop moreover favorite, widespread needs; also new|6.95|5.97|5003002|exportischolar #2|3|pop|5|Music|188|callyn stese|N/A|55150yellow416778180|sienna|Oz|Unknown|9|priantin stought| +1954|AAAAAAAACKHAAAAA|1997-10-27|1999-10-27|Too old audiences ought to inspect. Times help then. Forces may not become. Strong fires must write. Then big shows shall not fix. Now multiple police challenge most complex|1.04|0.55|5004001|edu packscholar #1|4|classical|5|Music|552|ableantianti|N/A|9314682766pink839527|sandy|Dozen|Unknown|6|eseantin stought| +1955|AAAAAAAACKHAAAAA|1999-10-28|2001-10-26|Too old audiences ought to inspect. Times help then. Forces may not become. Strong fires must write. Then big shows shall not fix. Now multiple police challenge most complex|1.19|0.55|5004001|amalgexporti #2|1|newborn|3|Children|312|ableantianti|small|30773714137powder052|white|Box|Unknown|11|antiantin stought| +1956|AAAAAAAACKHAAAAA|2001-10-27||Great tenants find only straight new words. Black friend|4.80|0.55|3001001|amalgexporti #1|1|newborn|3|Children|312|ableantianti|medium|30773714137powder052|khaki|Carton|Unknown|38|callyantin stought| +1957|AAAAAAAAFKHAAAAA|1997-10-27||Manufacturing elections prefer affairs. Trends used to |2.76|1.35|8004008|edu packnameless #8|4|camping|8|Sports|355|antiantipri|N/A|9427860sandy00991316|peach|Case|Unknown|50|ationantin stought| +1958|AAAAAAAAGKHAAAAA|1997-10-27|2000-10-26|Ideas must display as remarkable trees. Organic offenders create yet under large writings. Legal, white programmes print badly typically specific teachers. There pos|0.42|0.18|2002001|importoimporto #1|2|shirts|2|Men|307|ationbarpri|small|53lime46416440173926|pale|Tsp|Unknown|10|eingantin stought| +1959|AAAAAAAAGKHAAAAA|2000-10-27||Ideas must display as remarkable trees. Organic offenders create yet under large writings. Legal, white programmes print badly typically specific teachers. There pos|7.99|7.19|2002001|exportimaxi #10|13|sailing|8|Sports|376|callyationpri|N/A|321925944puff7596465|metallic|Gram|Unknown|87|n stantin stought| +1960|AAAAAAAAIKHAAAAA|1997-10-27|1999-10-27|Pictures cannot get advantages. Roman, difficult issues shift easy. Guidelines rouse just all actual hours. Coherent, main days acknowledge forward previous |0.48|0.20|7015001|scholarnameless #1|15|tables|7|Home|88|eingeing|N/A|79380028493royal2540|white|Each|Unknown|17|barcallyn stought| +1961|AAAAAAAAIKHAAAAA|1999-10-28|2001-10-26|Pictures cannot get advantages. Roman, difficult issues shift easy. Guidelines rouse just all actual hours. Coherent, main days acknowledge forward previous |1.08|0.20|8006002|corpnameless #2|6|football|8|Sports|88|eingeing|N/A|757362365242olive728|light|Each|Unknown|19|oughtcallyn stought| +1962|AAAAAAAAIKHAAAAA|2001-10-27||High, adjacent details might not introduce thoroughly ideas. Subjects might lift desperately financial, profe|9.80|7.54|10014008|edu packamalgamalg #8|14|automotive|10|Electronics|88|eingeing|N/A|6006347sandy68497801|rose|Gross|Unknown|41|ablecallyn stought| +1963|AAAAAAAALKHAAAAA|1997-10-27||Past periods may not admit very white tickets; catholic, mo|8.13|4.39|4004002|edu packedu pack #2|4|athletic|4|Shoes|109|n stbarought|extra large|4turquoise5707363982|snow|Ounce|Unknown|32|pricallyn stought| +1964|AAAAAAAAMKHAAAAA|1997-10-27|2000-10-26|Unions shall see enough over true attitudes; of course full variable|8.90|7.83|9002009|importomaxi #9|2|business|9|Books|595|antin stanti|N/A|515356144chiffon9224|honeydew|Tbl|Unknown|64|esecallyn stought| +1965|AAAAAAAAMKHAAAAA|2000-10-27||Very english policies could take. Dogs allow better years. Black, cultural areas lead|88.24|46.76|9002009|exportiexporti #2|3|toddlers|3|Children|32|antin stanti|medium|515356144chiffon9224|rosy|Dozen|Unknown|8|anticallyn stought| +1966|AAAAAAAAOKHAAAAA|1997-10-27|1999-10-27|So british cases could not know hard. Grateful, single drugs should not get secondly international levels. Considerations used to connect governments. Exact men get at a patients. Yesterday good men s|19.51|15.80|8014007|edu packmaxi #7|14|tennis|8|Sports|350|barantipri|N/A|703plum6706592979852|pink|Box|Unknown|8|callycallyn stought| +1967|AAAAAAAAOKHAAAAA|1999-10-28|2001-10-26|Likely funds argue surprising women. Regions assess already comparative lips. Legs may not say new, important machines. Young germans move also national children. Questions will expect hig|7.42|15.80|3002002|importoexporti #2|14|infants|3|Children|350|barantipri|medium|703plum6706592979852|slate|Carton|Unknown|33|ationcallyn stought| +1968|AAAAAAAAOKHAAAAA|2001-10-27||Services experience elderly councillors. |4.61|3.64|3002002|corpmaxi #9|16|golf|8|Sports|286|barantipri|N/A|7puff852355083578907|thistle|Unknown|Unknown|6|eingcallyn stought| +1969|AAAAAAAABLHAAAAA|1997-10-27||Perhaps old sources disappear. Small, bright enterprises used to take by a systems. Local proteins could not try then. Blank, special colleges appear.|7.38|2.43|9007002|brandmaxi #2|7|reference|9|Books|471|oughtationese|N/A|0medium3441006852115|honeydew|Case|Unknown|48|n stcallyn stought| +1970|AAAAAAAACLHAAAAA|1997-10-27|2000-10-26|Huge, private situations ought to back by an marks. Girls can come also local,|7.03|5.06|8012001|importomaxi #1|12|guns|8|Sports|277|ationationable|N/A|76219284673grey74306|cyan|Oz|Unknown|8|barationn stought| +1971|AAAAAAAACLHAAAAA|2000-10-27||Plans shall pay more practitioners. Earlie|7.36|6.40|5004002|edu packscholar #2|12|classical|5|Music|277|ationationable|N/A|06525157417salmon002|white|Tsp|Unknown|3|oughtationn stought| +1972|AAAAAAAAELHAAAAA|1997-10-27|1999-10-27|Normal, important minutes compete years; able, foreign seats ought to act always new attacks. In general red programmes may crawl dealers. Extra losses ma|0.34|0.25|3002001|importoexporti #1|2|infants|3|Children|81|oughteing|large|5160674564grey803625|steel|Unknown|Unknown|24|ableationn stought| +1973|AAAAAAAAELHAAAAA|1999-10-28|2001-10-26|Normal, important minutes compete years; able, foreign seats ought to act always new attacks. In general red programmes may crawl dealers. Extra losses ma|4.66|0.25|3002001|importoscholar #2|2|country|5|Music|81|oughteing|N/A|5160674564grey803625|red|Carton|Unknown|9|priationn stought| +1974|AAAAAAAAELHAAAAA|2001-10-27||Normal, important minutes compete years; able, foreign seats ought to act always new attacks. In general red programmes may crawl dealers. Extra losses ma|8.62|0.25|3002001|amalgunivamalg #7|2|cooking|9|Books|703|oughteing|N/A|5160674564grey803625|slate|Case|Unknown|43|eseationn stought| +1975|AAAAAAAAHLHAAAAA|1997-10-27||Teachers should not maximise among a areas. Old police can say all right golden views; ideas must understand later. As usual paintings used to miss even at a charges. Systems pick. Home supre|1.15|0.47|4002002|importoedu pack #2|2|mens|4|Shoes|256|callyantiable|large|233567681482olive858|puff|Cup|Unknown|53|antiationn stought| +1976|AAAAAAAAILHAAAAA|1997-10-27|2000-10-26|Prime systems used to raise ever classical forms. Well sure preparations view tonight. Directly appr|6.39|4.60|4003001|exportiedu pack #1|3|kids|4|Shoes|196|callyn stought|large|8snow602910365813850|drab|Ounce|Unknown|44|callyationn stought| +1977|AAAAAAAAILHAAAAA|2000-10-27||Irish, wrong sums will not run including a universities. Only popular eyes can cut only just useful cells. Possible numbers give also hi|0.84|0.69|5002002|importoscholar #2|3|country|5|Music|196|callyn stought|N/A|8snow602910365813850|turquoise|Dram|Unknown|3|ationationn stought| +1978|AAAAAAAAKLHAAAAA|1997-10-27|1999-10-27|Never basic chairs should not discover able, federal animals. Li|4.03|2.78|1001001|amalgamalg #1|1|dresses|1|Women|228|eingableable|medium|20293sienna897240194|powder|Lb|Unknown|68|eingationn stought| +1979|AAAAAAAAKLHAAAAA|1999-10-28|2001-10-26|Never basic chairs should not discover able, federal animals. Li|6.58|2.17|1001001|importoimporto #2|2|shirts|2|Men|228|eingableable|medium|20293sienna897240194|blush|Ounce|Unknown|29|n stationn stought| +1980|AAAAAAAAKLHAAAAA|2001-10-27||Never basic chairs should not discover able, federal animals. Li|3.09|1.26|3004001|edu packexporti #1|4|school-uniforms|3|Children|228|eingableable|medium|30701961metallic9610|light|Tsp|Unknown|42|bareingn stought| +1981|AAAAAAAANLHAAAAA|1997-10-27||Heads might use deeper before a men. Liberal, major authorities must pay extremely broad owners. |0.12|0.04|8012002|importomaxi #2|12|guns|8|Sports|122|ableableought|N/A|903rosy1311339079017|lavender|Each|Unknown|20|oughteingn stought| +1982|AAAAAAAAOLHAAAAA|1997-10-27|2000-10-26|Serious schools consider then no longer other talks; shares would bring never gold tasks; goo|0.40|0.16|5003001|exportischolar #1|3|pop|5|Music|239|n stpriable|N/A|3796royal18333898871|rosy|Gross|Unknown|98|ableeingn stought| +1983|AAAAAAAAOLHAAAAA|2000-10-27||Serious schools consider then no longer other talks; shares would bring never gold tasks; goo|5.44|2.28|10006012|corpunivamalg #12|6|musical|10|Electronics|239|n stpriable|N/A|3796royal18333898871|pink|Carton|Unknown|67|prieingn stought| +1984|AAAAAAAAAMHAAAAA|1997-10-27|1999-10-27|Social, reduced rebels would not achieve very free ships. Selective|3.41|2.35|8009005|maxinameless #5|9|optics|8|Sports|286|callyeingable|N/A|purple44744632364981|steel|Tbl|Unknown|14|eseeingn stought| +1985|AAAAAAAAAMHAAAAA|1999-10-28|2001-10-26|Initial, essential areas take often legal demonstrations. Approximately correct members shine. Problems build|53.70|33.83|8009005|corpunivamalg #6|9|mystery|9|Books|132|ablepriought|N/A|4384papaya5296346405|red|Unknown|Unknown|60|antieingn stought| +1986|AAAAAAAAAMHAAAAA|2001-10-27||Initial, essential areas take often legal demonstrations. Approximately correct members shine. Problems build|6.05|33.83|8009005|amalgamalg #1|1|dresses|1|Women|44|eseese|small|4384papaya5296346405|violet|Each|Unknown|53|callyeingn stought| +1987|AAAAAAAADMHAAAAA|1997-10-27||Powers feel in the affairs. Hands might determine for the eyes. Young, elderly parameters would not pe|0.49|0.19|10011016|amalgamalgamalg #16|11|disk drives|10|Electronics|778|eingationation|N/A|4673055899280peru313|thistle|Dram|Unknown|50|ationeingn stought| +1988|AAAAAAAAEMHAAAAA|1997-10-27|2000-10-26|Boots recommend usually just local centres; c|7.56|3.40|9003009|exportimaxi #9|3|computers|9|Books|256|callyantiable|N/A|04449150882orchid307|tan|Bunch|Unknown|18|eingeingn stought| +1989|AAAAAAAAEMHAAAAA|2000-10-27||Schools can hold years. Payable, ethnic shops ought to find prote|5.04|3.40|9003009|amalgamalg #2|3|dresses|1|Women|593|prin stanti|extra large|04449150882orchid307|smoke|Ton|Unknown|13|n steingn stought| +1990|AAAAAAAAGMHAAAAA|1997-10-27|1999-10-27|Toys must not meet again goods. Comfortable, major members last days. Popular, double occupations may compensate researchers. Conditions |3.35|2.41|5004001|edu packscholar #1|4|classical|5|Music|29|n stable|N/A|66394879lace27668491|chocolate|Pallet|Unknown|66|barn stn stought| +1991|AAAAAAAAGMHAAAAA|1999-10-28|2001-10-26|Rural issues take cool; busy, necessary pools ought to fail forward other, national institutions. Elegant parents ab|0.35|2.41|5004001|importoedu pack #2|4|mens|4|Shoes|82|ableeing|extra large|11287yellow596784680|seashell|Tsp|Unknown|8|oughtn stn stought| +1992|AAAAAAAAGMHAAAAA|2001-10-27||Rural issues take cool; busy, necessary pools ought to fail forward other, national institutions. Elegant parents ab|4.24|2.41|5004001|amalgmaxi #3|11|archery|8|Sports|35|antipri|N/A|5medium1092053768872|sky|Box|Unknown|37|ablen stn stought| +1993|AAAAAAAAJMHAAAAA|1997-10-27||Eye|2.18|1.85|7006004|corpbrand #4|6|rugs|7|Home|307|ationbarpri|N/A|604thistle1238024184|wheat|Each|Unknown|24|prin stn stought| +1994|AAAAAAAAKMHAAAAA|1997-10-27|2000-10-26|Nations save further new complaints. Perfect things murder different odds. General firms will like also; fatal grounds lie however working sorts. However internal police should design|8.09|3.31|3001001|amalgexporti #1|1|newborn|3|Children|352|ableantipri|large|092firebrick89628774|slate|Ounce|Unknown|41|esen stn stought| +1995|AAAAAAAAKMHAAAAA|2000-10-27||Nations save further new complaints. Perfect things murder different odds. General firms will like also; fatal grounds lie however working sorts. However internal police should design|4.31|3.31|10012007|importoamalgamalg #7|12|monitors|10|Electronics|352|ableantipri|N/A|092firebrick89628774|white|Unknown|Unknown|16|antin stn stought| +1996|AAAAAAAAMMHAAAAA|1997-10-27|1999-10-27|Long provisions will keep ago necessary nurses. Again certain patients come tentatively dutch teachers. Modern, certain years assist only separate hours. Fundamental facilities mean much comple|0.18|0.12|8003007|exportinameless #7|3|basketball|8|Sports|85|antieing|N/A|3hot2777122538830688|turquoise|Pound|Unknown|8|callyn stn stought| +1997|AAAAAAAAMMHAAAAA|1999-10-28|2001-10-26|Then smart fires used to form. Defences shall not ease precious, interesting women. Terms might not expect long consequences. Mass years shall not need most obvious obligat|2.08|0.12|9004006|edu packmaxi #6|3|entertainments|9|Books|85|antieing|N/A|3hot2777122538830688|orchid|Case|Unknown|62|ationn stn stought| +1998|AAAAAAAAMMHAAAAA|2001-10-27||Economic, surprised specimens disappear. True teachers force even. Local, little figures go. Developments could not realize feet. Careful, likely grounds prod|4.21|2.10|4004001|edu packedu pack #1|3|athletic|4|Shoes|120|antieing|large|2mint824947135580642|green|Lb|Unknown|8|eingn stn stought| +1999|AAAAAAAAPMHAAAAA|1997-10-27||Right social years would fit indirectly creatures. Very suspicious words should not write particular, typical views. Rarely evident hours wish more lucky others. So racial loans imitate a|6.39|2.87|9006008|corpmaxi #8|6|parenting|9|Books|72|ableation|N/A|249711305mint5449317|yellow|Case|Unknown|48|n stn stn stought| +2000|AAAAAAAAANHAAAAA|1997-10-27|2000-10-26|Ago regional objects finish courts. Large, serio|9.51|8.08|4004001|edu packedu pack #1|4|athletic|4|Shoes|171|oughtationought|small|30598royal3229309249|sky|Pound|Unknown|22|barbarbarable| +2001|AAAAAAAAANHAAAAA|2000-10-27||Ago regional objects finish courts. Large, serio|0.75|8.08|4004001|edu packimporto #2|4|sports-apparel|2|Men|171|oughtationought|medium|687289413250peach054|powder|Gram|Unknown|24|oughtbarbarable| +2002|AAAAAAAACNHAAAAA|1997-10-27|1999-10-27|Political customs differ in a issues; public purposes dream. Old, other|1.25|0.62|1001001|amalgamalg #1|1|dresses|1|Women|261|oughtcallyable|medium|6648seashell87062287|beige|Gram|Unknown|5|ablebarbarable| +2003|AAAAAAAACNHAAAAA|1999-10-28|2001-10-26|Changes m|2.46|0.62|1001001|amalgamalgamalg #15|1|disk drives|10|Electronics|261|oughtcallyable|N/A|6648seashell87062287|tomato|Pound|Unknown|28|pribarbarable| +2004|AAAAAAAACNHAAAAA|2001-10-27||Changes m|0.69|0.55|1002001|importoamalg #1|1|fragrances|1|Women|261|oughtcallyable|small|27turquoise675035444|red|Gram|Unknown|43|esebarbarable| +2005|AAAAAAAAFNHAAAAA|1997-10-27||Liberal committees go weeks. Good problems want in a symbols. Other, reasonable brothers terminate th|8.78|4.65|5004002|edu packscholar #2|4|classical|5|Music|207|ationbarable|N/A|puff4475656912691642|spring|Gram|Unknown|18|antibarbarable| +2006|AAAAAAAAGNHAAAAA|1997-10-27|2000-10-26|White, familiar months agree fully. Significantly political services used to portray abruptly types. Simply ill funds used to damage real sales. Changes would occur more than great r|45.37|20.87|4001001|amalgedu pack #1|1|womens|4|Shoes|379|n stationpri|small|28536441turquoise438|pale|Unknown|Unknown|8|callybarbarable| +2007|AAAAAAAAGNHAAAAA|2000-10-27||Levels bring for the disciplines|77.55|30.24|2003002|exportiimporto #2|3|pants|2|Men|221|oughtableable|large|28536441turquoise438|puff|Gram|Unknown|40|ationbarbarable| +2008|AAAAAAAAINHAAAAA|1997-10-27|1999-10-27|Vast, successful effects could function young, public areas; settlements come automatically then collective skills. Sudden lines pull no longer happy facilities. Profits|6.95|3.89|2001001|amalgimporto #1|1|accessories|2|Men|157|ationantiought|medium|4836steel29406772982|slate|Unknown|Unknown|91|eingbarbarable| +2009|AAAAAAAAINHAAAAA|1999-10-28|2001-10-26|I|0.76|0.61|7007010|brandbrand #10|1|decor|7|Home|299|n stn stable|N/A|1473489pink389289917|indian|Unknown|Unknown|26|n stbarbarable| +2010|AAAAAAAAINHAAAAA|2001-10-27||Particularly public boots serve even british events. Rates shall not drive for instance guns. Hands should see. Eyes shall n|58.34|33.83|7007010|exportiexporti #1|3|toddlers|3|Children|515|n stn stable|N/A|02peach6862370921244|magenta|Ton|Unknown|65|baroughtbarable| +2011|AAAAAAAALNHAAAAA|1997-10-27||Well famous years ought to answer men. Defin|3.13|1.03|6007004|brandcorp #4|7|pendants|6|Jewelry|146|callyeseought|N/A|55498893salmon587446|puff|Gram|Unknown|18|oughtoughtbarable| +2012|AAAAAAAAMNHAAAAA|1997-10-27|2000-10-26|Democrats proceed; goods should cope settlements. Steps stand today to a terms. Resources get further. Institutions live there. Normal stories may|5.08|3.65|5002001|importoscholar #1|2|country|5|Music|282|ableeingable|N/A|505262792877672rose4|papaya|Tsp|Unknown|71|ableoughtbarable| +2013|AAAAAAAAMNHAAAAA|2000-10-27||Democrats proceed; goods should cope settlements. Steps stand today to a terms. Resources get further. Institutions live there. Normal stories may|1.70|1.39|5002001|edu packimporto #2|2|sports-apparel|2|Men|548|ableeingable|extra large|505262792877672rose4|peru|Each|Unknown|71|prioughtbarable| +2014|AAAAAAAAONHAAAAA|1997-10-27|1999-10-27|Constant individuals give so in a jobs. Quite given activities return too; as yet geographical figures investigate possibly. Public police prepare t|0.98|0.62|7015005|scholarnameless #5|15|tables|7|Home|286|callyeingable|N/A|2850656352powder0949|red|Tsp|Unknown|18|eseoughtbarable| +2015|AAAAAAAAONHAAAAA|1999-10-28|2001-10-26|Drivers distort industrial, secondary nations. |8.67|0.62|5003002|exportischolar #2|15|pop|5|Music|164|callyeingable|N/A|2850656352powder0949|lemon|Dram|Unknown|40|antioughtbarable| +2016|AAAAAAAAONHAAAAA|2001-10-27||Soldiers take school|1.17|0.57|2001001|amalgimporto #1|15|accessories|2|Men|164|callyeingable|medium|2850656352powder0949|thistle|Ton|Unknown|5|callyoughtbarable| +2017|AAAAAAAABOHAAAAA|1997-10-27||Vehicles restore most weeks; together serious members call also from a diseases. Then european remains regard perhaps personal pupils. Multiple miners should not |3.60|1.22|1001002|amalgamalg #2|1|dresses|1|Women|686|callyeingcally|medium|53tan915386640086295|light|Carton|Unknown|51|ationoughtbarable| +2018|AAAAAAAACOHAAAAA|1997-10-27|2000-10-26|Again parliamentary stocks may generate typically unnecessary external arrangements. Funds fight again sole, rural contributions. Public fires|6.21|2.35|9012003|importounivamalg #3|12|home repair|9|Books|110|baroughtought|N/A|236986606370008rose3|rosy|Dram|Unknown|6|eingoughtbarable| +2019|AAAAAAAACOHAAAAA|2000-10-27||Physically young stages would not escape clear times. Existing, absent parts ought to av|4.06|2.31|9012003|edu packamalg #2|4|swimwear|1|Women|110|baroughtought|petite|9881747289936steel45|indian|Unknown|Unknown|87|n stoughtbarable| +2020|AAAAAAAAEOHAAAAA|1997-10-27|1999-10-27|Then strong parents progress only overall areas. Also r|34.77|16.34|2002001|importoimporto #1|2|shirts|2|Men|126|callyableought|N/A|79671632wheat4318299|sienna|Box|Unknown|77|barablebarable| +2021|AAAAAAAAEOHAAAAA|1999-10-28|2001-10-26|Very important years may seem to a others. Enough critical carers equate too to a facilities.|3.02|16.34|2002001|exportiamalg #2|3|maternity|1|Women|651|callyableought|medium|012913pale0289577797|misty|Lb|Unknown|24|oughtablebarable| +2022|AAAAAAAAEOHAAAAA|2001-10-27||Very important years may seem to a others. Enough critical carers equate too to a facilities.|4.91|1.76|2002001|amalgscholar #1|3|rock|5|Music|433|callyableought|N/A|9powder2133567485776|saddle|Cup|Unknown|13|ableablebarable| +2023|AAAAAAAAHOHAAAAA|1997-10-27||Cheap bright rounds may not deny most likely heads. Men find; sports dominate qu|37.87|22.34|6015008|scholarbrand #8|15|custom|6|Jewelry|404|esebarese|N/A|3653001491sandy47923|plum|Tsp|Unknown|53|priablebarable| +2024|AAAAAAAAIOHAAAAA|1997-10-27|2000-10-26|Current prices could take safely fingers. Once other subjects appear then; key, following courses should not slow n|8.61|5.85|4002001|importoedu pack #1|2|mens|4|Shoes|534|eseprianti|extra large|61664170navy89561684|firebrick|Ounce|Unknown|7|eseablebarable| +2025|AAAAAAAAIOHAAAAA|2000-10-27||Workers must return below. Circumstances ought to get also customers. Now historical children ma|0.64|0.56|4002001|edu packmaxi #6|2|tennis|8|Sports|534|eseprianti|N/A|61664170navy89561684|honeydew|Ounce|Unknown|50|antiablebarable| +2026|AAAAAAAAKOHAAAAA|1997-10-27|1999-10-27|Violent instruments may not think quite magnetic children. Scientific, economic products ought to refuse so. Responsible, happy results run. All but close others see directly together with a deaths. V|88.03|57.21|6009003|maxicorp #3|9|womens watch|6|Jewelry|290|barn stable|N/A|07007150329530rose05|spring|Pallet|Unknown|1|callyablebarable| +2027|AAAAAAAAKOHAAAAA|1999-10-28|2001-10-26|Violent instruments may not think quite magnetic children. Scientific, economic products ought to refuse so. Responsible, happy results run. All but close others see directly together with a deaths. V|6.78|57.21|3003002|exportiexporti #2|3|toddlers|3|Children|26|callyable|large|34020seashell3613650|thistle|Bundle|Unknown|34|ationablebarable| +2028|AAAAAAAAKOHAAAAA|2001-10-27||Poor friends make in a duties. Figures know royal, personal members. Different, new games take effective, available pr|60.50|57.21|3003002|edu packscholar #1|4|classical|5|Music|216|callyable|N/A|26seashell3513270105|rose|Gross|Unknown|51|eingablebarable| +2029|AAAAAAAANOHAAAAA|||Goods will not comply for the facts; social, true girls ought to know now for a topics. Babies wo|||5003002||3||5|Music|169|n stcallyought|||wheat|Carton|Unknown|82|| +2030|AAAAAAAAOOHAAAAA|1997-10-27|2000-10-26|Always political centres seem now never modest sets. Major comments may search additional, other guards. National, appropriate relations buy very.|5.07|3.95|10006008|corpunivamalg #8|6|musical|10|Electronics|749|n steseation|N/A|9534turquoise5415424|sky|Bundle|Unknown|11|barpribarable| +2031|AAAAAAAAOOHAAAAA|2000-10-27||Always political centres seem now never modest sets. Major comments may search additional, other guards. National, appropriate relations buy very.|4.48|3.95|10006008|edu packexporti #2|6|school-uniforms|3|Children|776|n steseation|economy|9534turquoise5415424|orange|Ton|Unknown|43|oughtpribarable| +2032|AAAAAAAAAPHAAAAA|1997-10-27|1999-10-27|Proper, great processes hang mediterranean, main communities. Games might not entitle longer nuclear runs. Digital, regular trials keep. Able, primitive walls sleep never internal types. Different, |2.49|1.34|5003001|exportischolar #1|3|pop|5|Music|112|ableoughtought|N/A|09643030rose84971735|plum|Gross|Unknown|52|ablepribarable| +2033|AAAAAAAAAPHAAAAA|1999-10-28|2001-10-26|Nuclear instruments might cool briskly psychological friends. Christian penalties should not know in a centres. Shee|0.96|0.43|5003001|namelesscorp #2|3|mens watch|6|Jewelry|112|ableoughtought|N/A|259838brown200839054|linen|Dram|Unknown|5|pripribarable| +2034|AAAAAAAAAPHAAAAA|2001-10-27||Basic, late purposes shall believe often in a boats. Other, commercial lips crack finally experiences; also long structures must not understand unfortunately major, general weeks. Gr|4.43|1.86|5003001|maxiunivamalg #12|3|televisions|10|Electronics|112|ableoughtought|N/A|259838brown200839054|firebrick|Tbl|Unknown|9|esepribarable| +2035|AAAAAAAADPHAAAAA|1997-10-27||Too curious regions work. New drugs describe na|4.87|3.55|2003002|exportiimporto #2|3|pants|2|Men|615|antioughtcally|large|789807373808pale1897|rosy|N/A|Unknown|34|antipribarable| +2036|AAAAAAAAEPHAAAAA|1997-10-27|2000-10-26|Ago social details will gain mothers. Actively regional reports remain|0.14|0.05|9009009|maximaxi #9|9|science|9|Books|91|oughtn st|N/A|8451637powder6556156|seashell|Ton|Unknown|52|callypribarable| +2037|AAAAAAAAEPHAAAAA|2000-10-27||Ago social details will gain mothers. Actively regional reports remain|7.16|5.22|10003015|exportiunivamalg #15|9|dvd/vcr players|10|Electronics|270|oughtn st|N/A|561yellow68502476253|lime|Box|Unknown|34|ationpribarable| +2038|AAAAAAAAGPHAAAAA|1997-10-27|1999-10-27|Well single minutes imagine firmly commercial sites. Shortly local workshops sustain elsewhere great eyes. Costs ought to expe|5.52|2.37|1001001|amalgamalg #1|1|dresses|1|Women|286|callyeingable|petite|89coral0942541586795|turquoise|Ounce|Unknown|73|eingpribarable| +2039|AAAAAAAAGPHAAAAA|1999-10-28|2001-10-26|Away elderly problems ought to attend good, necessary colleagues; thousands continue at least only other factors|8.55|5.98|8001010|amalgnameless #10|1|athletic shoes|8|Sports|308|eingbarpri|N/A|1seashell49879062732|turquoise|Tbl|Unknown|12|n stpribarable| +2040|AAAAAAAAGPHAAAAA|2001-10-27||Scientifically other toys will achieve. Yesterday heavy friends must not represent ways. Large miles could not realize once more dreams. Names shall make in order heavily|0.61|5.98|8001010|edu packamalg #1|4|swimwear|1|Women|308|eingbarpri|extra large|1seashell49879062732|pale|Each|Unknown|25|baresebarable| +2041|AAAAAAAAJPHAAAAA|1997-10-27||Primarily lovely relations should reveal just. Other relationships imply further on a jeans. Nice, sad clients spell about willing materials. Followers shall reply scottish parts. Very obvious |9.40|7.61|6001002|amalgcorp #2|1|birdal|6|Jewelry|118|eingoughtought|N/A|56749650purple222561|powder|Tbl|Unknown|2|oughtesebarable| +2042|AAAAAAAAKPHAAAAA|1997-10-27|2000-10-26|Schools should want relatively|7.89|2.52|5004001|edu packscholar #1|4|classical|5|Music|108|eingbarought|N/A|829khaki980366267461|saddle|Bunch|Unknown|5|ableesebarable| +2043|AAAAAAAAKPHAAAAA|2000-10-27||Schools should want relatively|1.84|0.90|10004004|edu packunivamalg #4|4|audio|10|Electronics|505|eingbarought|N/A|802884123rosy5205953|pink|Pound|Unknown|28|priesebarable| +2044|AAAAAAAAMPHAAAAA|1997-10-27|1999-10-27|Payments appear forces. New proceedings pursue at least financial, current angles. Remarkable, main documents comply unusual, solid aspects. Wrong, just films ask different, l|9.49|4.65|7001005|amalgbrand #5|1|bathroom|7|Home|220|barableable|N/A|286743green015732927|linen|Each|Unknown|25|eseesebarable| +2045|AAAAAAAAMPHAAAAA|1999-10-28|2001-10-26|Payments appear forces. New proceedings pursue at least financial, current angles. Remarkable, main documents comply unusual, solid aspects. Wrong, just films ask different, l|4.26|4.65|4001002|amalgedu pack #2|1|womens|4|Shoes|934|eseprin st|extra large|286743green015732927|rose|N/A|Unknown|2|antiesebarable| +2046|AAAAAAAAMPHAAAAA|2001-10-27||Payments appear forces. New proceedings pursue at least financial, current angles. Remarkable, main documents comply unusual, solid aspects. Wrong, just films ask different, l|6.42|4.65|4001002|amalgexporti #1|1|newborn|3|Children|188|eseprin st|medium|286743green015732927|lawn|Each|Unknown|54|callyesebarable| +2047|AAAAAAAAPPHAAAAA|1997-10-27||Coming, small events sleep significantly. Applications can finish never earlier national students. Original shops hear police; mor|4.13|1.94|2002002|importoimporto #2|2|shirts|2|Men|223|priableable|small|4958568746584salmon5|powder|Ounce|Unknown|33|ationesebarable| +2048|AAAAAAAAAAIAAAAA|1997-10-27|2000-10-26|Years make particularly hundreds. So good subsidies shall not fall full, official skills; new, dead t|4.70|1.64|3004001|edu packexporti #1|4|school-uniforms|3|Children|75|antiation|extra large|6802798200863878sky1|khaki|Case|Unknown|25|eingesebarable| +2049|AAAAAAAAAAIAAAAA|2000-10-27||Teachers write available, public |8.61|1.64|3004001|exportiamalgamalg #10|13|stereo|10|Electronics|75|antiation|N/A|6802798200863878sky1|steel|Dozen|Unknown|5|n stesebarable| +2050|AAAAAAAACAIAAAAA|1997-10-27|1999-10-27|Days can need fully other refer|91.34|61.19|10014011|edu packamalgamalg #11|14|automotive|10|Electronics|589|n steinganti|N/A|768turquoise67768162|peru|Ounce|Unknown|15|barantibarable| +2051|AAAAAAAACAIAAAAA|1999-10-28|2001-10-26|Days can need fully other refer|1.93|1.69|10014011|scholarbrand #4|15|custom|6|Jewelry|589|n steinganti|N/A|324506471303pink6967|tan|Tbl|Unknown|46|oughtantibarable| +2052|AAAAAAAACAIAAAAA|2001-10-27||So male features put different, useful inches. English, identical cases would not catch normally. Yea|2.02|1.69|9012001|importounivamalg #1|15|home repair|9|Books|810|baroughteing|N/A|324506471303pink6967|green|Lb|Unknown|2|ableantibarable| +2053|AAAAAAAAFAIAAAAA|1997-10-27||Good, irrelevant ways know particular reports. Fears continue. Great re|89.85|37.73|3003002|exportiexporti #2|3|toddlers|3|Children|199|n stn stought|medium|180457sky12385502770|navy|Pallet|Unknown|28|priantibarable| +2054|AAAAAAAAGAIAAAAA|1997-10-27|2000-10-26|A little strict results play; widespread doors seem then necessary reasons. Communications explain unfortunately. Policies produce free plants. Minutes say. Tensions retur|79.33|28.55|1002001|importoamalg #1|2|fragrances|1|Women|231|oughtpriable|extra large|35707orchid054428407|papaya|Gram|Unknown|5|eseantibarable| +2055|AAAAAAAAGAIAAAAA|2000-10-27||Dull day|65.12|28.55|1002001|univunivamalg #4|10|travel|9|Books|238|eingpriable|N/A|35707orchid054428407|light|Tsp|Unknown|13|antiantibarable| +2056|AAAAAAAAIAIAAAAA|1997-10-27|1999-10-27|Aware, public materials can supply for a firms. Delicious sets should move hence in a kids. Nuclear, able sessions may |59.67|33.41|9013005|exportiunivamalg #5|13|self-help|9|Books|504|esebaranti|N/A|69violet254147582370|indian|Each|Unknown|5|callyantibarable| +2057|AAAAAAAAIAIAAAAA|1999-10-28|2001-10-26|Aware, public materials can supply for a firms. Delicious sets should move hence in a kids. Nuclear, able sessions may |5.90|2.12|9013005|maxinameless #8|9|optics|8|Sports|226|callyableable|N/A|snow1084193132375131|medium|Bundle|Unknown|30|ationantibarable| +2058|AAAAAAAAIAIAAAAA|2001-10-27||Aware, public materials can supply for a firms. Delicious sets should move hence in a kids. Nuclear, able sessions may |4.55|3.64|9013005|importoexporti #1|2|infants|3|Children|525|antiableanti|petite|snow1084193132375131|white|Bunch|Unknown|60|eingantibarable| +2059|AAAAAAAALAIAAAAA|1997-10-27||Holes continue. Deep human cases would not find goods. Anonymous, bright com|3.64|2.32|2004002|edu packimporto #2|4|sports-apparel|2|Men|458|eingantiese|large|3178612magenta846348|salmon|Gram|Unknown|45|n stantibarable| +2060|AAAAAAAAMAIAAAAA|1997-10-27|2000-10-26|Improvements must live|2.66|2.39|6010005|univbrand #5|10|jewelry boxes|6|Jewelry|288|eingeingable|N/A|64784598342770royal7|spring|Gross|Unknown|18|barcallybarable| +2061|AAAAAAAAMAIAAAAA|2000-10-27||Improvements must live|0.65|0.49|7015002|scholarnameless #2|10|tables|7|Home|288|eingeingable|N/A|064wheat345548047604|seashell|Pound|Unknown|6|oughtcallybarable| +2062|AAAAAAAAOAIAAAAA|1997-10-27|1999-10-27|Possible, concerned facilities would not show also most due opinions. Empty students maintain of course possible, particular years. |8.67|5.54|9006011|corpmaxi #11|6|parenting|9|Books|221|oughtableable|N/A|99380486530red506268|khaki|Tbl|Unknown|29|ablecallybarable| +2063|AAAAAAAAOAIAAAAA|1999-10-28|2001-10-26|Possible, concerned facilities would not show also most due opinions. Empty students maintain of course possible, particular years. |0.14|0.04|10009007|maxiunivamalg #7|6|televisions|10|Electronics|221|oughtableable|N/A|0107slate30715075994|puff|Oz|Unknown|49|pricallybarable| +2064|AAAAAAAAOAIAAAAA|2001-10-27||Courses touch also high families. Presidential,|4.83|0.04|10009007|maxibrand #5|9|mattresses|7|Home|663|oughtableable|N/A|5631612thistle747400|dodger|Lb|Unknown|21|esecallybarable| +2065|AAAAAAAABBIAAAAA|1997-10-27||As mixed months will start then on a players; systems could show less opinions. Groun|4.83|2.80|10011009|amalgamalgamalg #9|11|disk drives|10|Electronics|522|ableableanti|N/A|405939846pale5425233|steel|Oz|Unknown|23|anticallybarable| +2066|AAAAAAAACBIAAAAA|1997-10-27|2000-10-26|Simple friends take then available, modern countries. Operational bands find at all early governors. Big patients u|1.00|0.74|8006007|corpnameless #7|6|football|8|Sports|602|ablebarcally|N/A|4powder7512248758642|yellow|Ton|Unknown|7|callycallybarable| +2067|AAAAAAAACBIAAAAA|2000-10-27||Simple friends take then available, modern countries. Operational bands find at all early governors. Big patients u|7.67|5.13|10012011|importoamalgamalg #11|6|monitors|10|Electronics|244|eseeseable|N/A|4powder7512248758642|hot|Case|Unknown|98|ationcallybarable| +2068|AAAAAAAAEBIAAAAA|1997-10-27|1999-10-27|Properly british facilities must see from a debts. British, armed rivals buy known movements. Either unacceptable notes get more |9.37|3.74|4004001|edu packedu pack #1|4|athletic|4|Shoes|294|esen stable|extra large|586313398goldenrod55|plum|Bundle|Unknown|61|eingcallybarable| +2069|AAAAAAAAEBIAAAAA|1999-10-28|2001-10-26|Normal officers become especially at least natural ideas. Papers used to arrive women. Wrong, open books used to respond so|3.34|2.67|10004013|edu packunivamalg #13|4|audio|10|Electronics|294|esen stable|N/A|17533violet696605697|slate|Pound|Unknown|36|n stcallybarable| +2070|AAAAAAAAEBIAAAAA|2001-10-27||Christian, lively hearts illustrate actually in the techniques. Dependent, following loans will purchase; fees ensure w|3.75|2.43|5004001|edu packscholar #1|4|classical|5|Music|108|esen stable|N/A|17533violet696605697|thistle|Oz|Unknown|35|barationbarable| +2071|AAAAAAAAHBIAAAAA|1997-10-27||Instructions make semantic, different conditions; drastically other pieces protest still in a|9.16|4.58|1001002|amalgamalg #2|1|dresses|1|Women|279|n stationable|economy|4368058575puff176486|violet|Tsp|Unknown|39|oughtationbarable| +2072|AAAAAAAAIBIAAAAA|1997-10-27|2000-10-26|Neither holy directors would not stem excessively. Often absolute newspapers can command attractive, white ey|0.47|0.27|6005001|scholarcorp #1|5|earings|6|Jewelry|118|eingoughtought|N/A|8127491296760red6597|red|Cup|Unknown|6|ableationbarable| +2073|AAAAAAAAIBIAAAAA|2000-10-27||Marine children could not get directly weeks; quite social ministers receive simple, small items. General, physical methods see in the pri|0.59|0.35|6005001|scholarmaxi #4|15|fishing|8|Sports|118|eingoughtought|N/A|yellow38159689557655|pale|Gross|Unknown|28|priationbarable| +2074|AAAAAAAAKBIAAAAA|1997-10-27|1999-10-27|Services contain plans. There colourful prospects might not know now sides. Dual pupils can a|1.19|0.46|2001001|amalgimporto #1|1|accessories|2|Men|64|esecally|large|091936465saddle70527|magenta|Oz|Unknown|65|eseationbarable| +2075|AAAAAAAAKBIAAAAA|1999-10-28|2001-10-26|Services contain plans. There colourful prospects might not know now sides. Dual pupils can a|7.32|2.41|4001002|amalgedu pack #2|1|womens|4|Shoes|642|ableesecally|large|11450023puff97635453|papaya|N/A|Unknown|53|antiationbarable| +2076|AAAAAAAAKBIAAAAA|2001-10-27||British legs manufacture simply present years. Aims reduce free applications. British, final reasons dash british, clear years. Very wrong courts look just. Work|3.42|2.41|7009007|maxibrand #7|9|mattresses|7|Home|266|ableesecally|N/A|9175258971spring0107|turquoise|Bundle|Unknown|11|callyationbarable| +2077|AAAAAAAANBIAAAAA|1997-10-27||Empty records shall not set. Happy, large colonies|1.93|0.69|3004002|edu packexporti #2|4|school-uniforms|3|Children|131|oughtpriought|medium|6518mint523894876437|yellow|Tsp|Unknown|27|ationationbarable| +2078|AAAAAAAAOBIAAAAA|1997-10-27|2000-10-26|Gaps ought to tell no doubt. Meetings ought to follow also japanese studies. New, future patients dance new, level newspapers. Details used to carry. Quite mean terms must see once. Schools wil|0.37|0.25|5002001|importoscholar #1|2|country|5|Music|164|esecallyought|N/A|915918931708plum1130|papaya|Dram|Unknown|66|eingationbarable| +2079|AAAAAAAAOBIAAAAA|2000-10-27||Gaps ought to tell no doubt. Meetings ought to follow also japanese studies. New, future patients dance new, level newspapers. Details used to carry. Quite mean terms must see once. Schools wil|1.96|0.74|5002001|exportiexporti #2|2|toddlers|3|Children|23|esecallyought|large|1035581489280drab860|sky|Tbl|Unknown|8|n stationbarable| +2080|AAAAAAAAACIAAAAA|1997-10-27|1999-10-27|Original, active users might draw for a associatio|2.36|1.69|7002001|importobrand #1|2|bedding|7|Home|106|callybarought|N/A|77817snow82436452590|seashell|Box|Unknown|13|bareingbarable| +2081|AAAAAAAAACIAAAAA|1999-10-28|2001-10-26|At all free friends shall not give most at the changes; participants should not come sometimes strong, constant countries; animals enjoy already late basic costs; weeks might not |3.78|2.15|2002002|importoimporto #2|2|shirts|2|Men|106|callybarought|extra large|77817snow82436452590|snow|Carton|Unknown|6|oughteingbarable| +2082|AAAAAAAAACIAAAAA|2001-10-27||Interesting sales rest just beautiful, rich copies; tourists ought to beat much prime members. Together glad charges incorporate much at a hours. Disabled, |2.61|1.74|4004001|edu packedu pack #1|4|athletic|4|Shoes|24|callybarought|small|5121royal92997384264|dim|Box|Unknown|54|ableeingbarable| +2083|AAAAAAAADCIAAAAA|1997-10-27||However import|2.88|1.52|1003002|exportiamalg #2|3|maternity|1|Women|133|pripriought|large|4salmon9836570246937|peach|Gram|Unknown|16|prieingbarable| +2084|AAAAAAAAECIAAAAA|1997-10-27|2000-10-26|Unusual boundaries answer apart resources. Ethnic, tragic methods might not operate rather so rich representatives. Poor nurses go. Short systems keep highly anywhere central|3.23|2.42|4002001|importoedu pack #1|2|mens|4|Shoes|126|callyableought|large|170726804papaya11546|pink|Case|Unknown|24|eseeingbarable| +2085|AAAAAAAAECIAAAAA|2000-10-27||Unusual boundaries answer apart resources. Ethnic, tragic methods might not operate rather so rich representatives. Poor nurses go. Short systems keep highly anywhere central|4.34|2.42|4002001|edu packamalg #2|4|swimwear|1|Women|126|callyableought|large|008461tan47152788811|navajo|Cup|Unknown|38|antieingbarable| +2086|AAAAAAAAGCIAAAAA|1997-10-27|1999-10-27|Missiles accept more police; police take neighbours. Other patients support brilliantly. Creative, other dates ought to assess successes. A|3.78|2.94|4003001|exportiedu pack #1|3|kids|4|Shoes|263|pricallyable|large|98926097892rose56654|chocolate|Ounce|Unknown|56|callyeingbarable| +2087|AAAAAAAAGCIAAAAA|1999-10-28|2001-10-26|Missiles accept more police; police take neighbours. Other patients support brilliantly. Creative, other dates ought to assess successes. A|1.40|2.94|4002002|importoedu pack #2|3|mens|4|Shoes|330|pricallyable|large|98926097892rose56654|pale|Ounce|Unknown|11|ationeingbarable| +2088|AAAAAAAAGCIAAAAA|2001-10-27||More very boys get clearly relationships. In order professional results grant running, able margins. Grim materials build. Molecules drop courts; remar|0.94|2.94|5004001|edu packscholar #1|4|classical|5|Music|330|pricallyable|N/A|2296pale091368644610|plum|Oz|Unknown|28|eingeingbarable| +2089|AAAAAAAAJCIAAAAA|1997-10-27||Able women ought to attempt in the schools. Then clear lives will live suitable, gastric complaints. Months should say surely then sufficient doors. Genetic times sign from a options.|1.02|0.83|1002002|importoamalg #2|2|fragrances|1|Women|651|oughtantically|small|126066lavender104081|orange|Pallet|Unknown|23|n steingbarable| +2090|AAAAAAAAKCIAAAAA|1997-10-27|2000-10-26|Fairly sure thousands publish earlier different, na|72.37|36.18|1004001|edu packamalg #1|4|swimwear|1|Women|228|eingableable|large|01773732729metallic9|tan|Each|Unknown|13|barn stbarable| +2091|AAAAAAAAKCIAAAAA|2000-10-27||Worri|2.75|1.62|9014004|edu packunivamalg #4|14|sports|9|Books|263|pricallyable|N/A|77967863042red495052|frosted|Bunch|Unknown|1|oughtn stbarable| +2092|AAAAAAAAMCIAAAAA|1997-10-27|1999-10-27|Urgent styles could prefer marked cells. Legal, regional patients call there animals. Actu|53.01|38.69|1003001|exportiamalg #1|3|maternity|1|Women|291|oughtn stable|medium|56515804465sienna929|seashell|Ton|Unknown|42|ablen stbarable| +2093|AAAAAAAAMCIAAAAA|1999-10-28|2001-10-26|Urgent styles could prefer marked cells. Legal, regional patients call there animals. Actu|0.29|0.20|1003001|brandunivamalg #3|3|personal|10|Electronics|291|oughtn stable|N/A|56515804465sienna929|gainsboro|Lb|Unknown|4|prin stbarable| +2094|AAAAAAAAMCIAAAAA|2001-10-27||Urgent styles could prefer marked cells. Legal, regional patients call there animals. Actu|2.00|0.20|1003001|amalgamalg #1|3|dresses|1|Women|291|oughtn stable|medium|56515804465sienna929|sienna|Oz|Unknown|42|esen stbarable| +2095|AAAAAAAAPCIAAAAA|1997-10-27||Arrangements might not go on a lawyers. Too small legs may explain most officer|6.07|2.42|7010006|univnameless #6|10|flatware|7|Home|304|esebarpri|N/A|79719687699honeydew4|chiffon|Ounce|Unknown|27|antin stbarable| +2096|AAAAAAAAADIAAAAA|1997-10-27|2000-10-26|Lips see outside quickly protective systems. |4.65|2.51|8014007|edu packmaxi #7|14|tennis|8|Sports|22|ableable|N/A|03mint81034809331720|lemon|Dram|Unknown|22|callyn stbarable| +2097|AAAAAAAAADIAAAAA|2000-10-27||Expected borders may emphasize old, open components. Already painful shoes could recover in a proposals. Fierce taxes bring liberal causes. A|2.06|0.90|8014007|corpmaxi #8|14|golf|8|Sports|22|ableable|N/A|7097175magenta898148|orchid|Unknown|Unknown|2|ationn stbarable| +2098|AAAAAAAACDIAAAAA|1997-10-27|1999-10-27|Handsome, common ministers shall not find |7.12|5.69|9010011|univunivamalg #11|10|travel|9|Books|431|oughtpriese|N/A|8410354622856purple8|lemon|Oz|Unknown|15|eingn stbarable| +2099|AAAAAAAACDIAAAAA|1999-10-28|2001-10-26|S|3.63|5.69|10004009|edu packunivamalg #9|4|audio|10|Electronics|431|oughtpriese|N/A|8410354622856purple8|sky|Dozen|Unknown|21|n stn stbarable| +2100|AAAAAAAACDIAAAAA|2001-10-27||Times should describe also; even recen|0.82|5.69|10004009|amalgunivamalg #1|4|cooking|9|Books|192|oughtpriese|N/A|8410354622856purple8|navy|Bunch|Unknown|12|barbaroughtable| +2101|AAAAAAAAFDIAAAAA|1997-10-27||Events would not take only to the notes. Close weeks can compromise yesterday new, dramatic members; forces must catch experiments; words forget so; so m|9.23|5.07|9013002|exportiunivamalg #2|13|self-help|9|Books|59|n stanti|N/A|60698312saddle227802|dim|Dram|Unknown|37|oughtbaroughtable| +2102|AAAAAAAAGDIAAAAA|1997-10-27|2000-10-26|Major, short devices shall return principles. Remains dominate enough then high levels|95.32|43.84|3004001|edu packexporti #1|4|school-uniforms|3|Children|776|callyationation|extra large|6ivory62605408617260|plum|Dozen|Unknown|9|ablebaroughtable| +2103|AAAAAAAAGDIAAAAA|2000-10-27||Major, short devices shall return principles. Remains dominate enough then high levels|70.35|59.09|4003002|exportiedu pack #2|3|kids|4|Shoes|776|callyationation|medium|4rosy812288259943494|light|Dram|Unknown|5|pribaroughtable| +2104|AAAAAAAAIDIAAAAA|1997-10-27|1999-10-27|High, little circumstances ought t|4.87|2.62|10016014|corpamalgamalg #14|16|wireless|10|Electronics|482|ableeingese|N/A|827376479439459sky92|spring|Lb|Unknown|65|esebaroughtable| +2105|AAAAAAAAIDIAAAAA|1999-10-28|2001-10-26|High, little circumstances ought t|1.74|1.20|1004002|edu packamalg #2|16|swimwear|1|Women|482|ableeingese|large|1410945papaya8837210|navy|Tsp|Unknown|28|antibaroughtable| +2106|AAAAAAAAIDIAAAAA|2001-10-27||High, little circumstances ought t|1.45|1.20|1004002|scholarbrand #7|16|blinds/shades|7|Home|248|ableeingese|N/A|562775951metallic311|violet|Lb|Unknown|71|callybaroughtable| +2107|AAAAAAAALDIAAAAA|1997-10-27||Thin, good units may feel to a |1.09|0.32|5001002|amalgscholar #2|1|rock|5|Music|77|ationation|N/A|49631071953chiffon04|lime|Box|Unknown|25|ationbaroughtable| +2108|AAAAAAAAMDIAAAAA|1997-10-27|2000-10-26|Suitable ways tell below. Domestic, chronic ears start so. Outside, front tiles co|3.15|2.64|4001001|amalgedu pack #1|1|womens|4|Shoes|144|eseeseought|medium|1556snow244358897846|rose|Pound|Unknown|70|eingbaroughtable| +2109|AAAAAAAAMDIAAAAA|2000-10-27||Alone other animals must want just so quick pers|2.48|1.58|9016010|corpunivamalg #10|16|mystery|9|Books|144|eseeseought|N/A|79555288steel8665954|misty|Pallet|Unknown|44|n stbaroughtable| +2110|AAAAAAAAODIAAAAA|1997-10-27|1999-10-27|So terrible shops wriggle in the practices. Years |6.68|3.07|6006007|corpcorp #7|6|rings|6|Jewelry|451|oughtantiese|N/A|1082799448838lawn791|spring|Pallet|Unknown|49|baroughtoughtable| +2111|AAAAAAAAODIAAAAA|1999-10-28|2001-10-26|Bedrooms enter sideways also old vessels; powerful, free topics use ordinary, new brothers; even afraid lines can see most for example financial dollars. Local resources happen high |3.78|1.77|6006007|corpnameless #2|6|football|8|Sports|773|oughtantiese|N/A|08581924963049sky453|saddle|Bunch|Unknown|34|oughtoughtoughtable| +2112|AAAAAAAAODIAAAAA|2001-10-27||Then right loans may not read with a affairs. Else musical facilities sell dead stores. Central, necessary pounds give old, serious firms. New, public companies might help too|3.78|1.17|6006007|univnameless #3|6|flatware|7|Home|289|n steingable|N/A|08581924963049sky453|salmon|Gross|Unknown|55|ableoughtoughtable| +2113|AAAAAAAABEIAAAAA|1997-10-27||Originally federal implications continue always manufacturers. Ins|0.63|0.32|8013004|exportimaxi #4|13|sailing|8|Sports|580|bareinganti|N/A|66897399730040peru69|snow|Dozen|Unknown|24|prioughtoughtable| +2114|AAAAAAAACEIAAAAA|1997-10-27|2000-10-26|Solid, major partners tell about good letters; |7.76|3.18|4003001|exportiedu pack #1|3|kids|4|Shoes|124|eseableought|small|63969727pale03226964|royal|Each|Unknown|5|eseoughtoughtable| +2115|AAAAAAAACEIAAAAA|2000-10-27||Seemingly american areas tell final, far resources. Immediate areas issue to a steps; never sweet patterns look better keen individuals. Diff|1.90|0.62|4003001|edu packimporto #2|4|sports-apparel|2|Men|124|eseableought|extra large|63969727pale03226964|almond|Each|Unknown|54|antioughtoughtable| +2116|AAAAAAAAEEIAAAAA|1997-10-27|1999-10-27|Democratic, american days damage still employers. Able banks could suggest full-time elements; daughters care minister|2.04|0.95|9013005|exportiunivamalg #5|13|self-help|9|Books|297|ationn stable|N/A|9087062381000purple5|cyan|Lb|Unknown|30|callyoughtoughtable| +2117|AAAAAAAAEEIAAAAA|1999-10-28|2001-10-26|Issues might not lose very bold, different opportunities. Of course absolute areas might balance bottom mechanisms. A|0.75|0.38|9001006|amalgmaxi #6|1|arts|9|Books|18|ationn stable|N/A|2183648dodger8062568|red|Ounce|Unknown|13|ationoughtoughtable| +2118|AAAAAAAAEEIAAAAA|2001-10-27||Shows suppose. Usually magic places ought to aid therefore low occupations. British, english systems could take |7.82|4.14|9001006|importoscholar #1|2|country|5|Music|342|ableesepri|N/A|33215papaya488161153|khaki|Cup|Unknown|24|eingoughtoughtable| +2119|AAAAAAAAHEIAAAAA|1997-10-27||Impor|6.96|3.54|6007008|brandcorp #8|7|pendants|6|Jewelry|117|ationoughtought|N/A|209377789929wheat362|indian|Ton|Unknown|35|n stoughtoughtable| +2120|AAAAAAAAIEIAAAAA|1997-10-27|2000-10-26|Also welsh police may keep police. So territorial cases organ|3.26|2.73|2001001|amalgimporto #1|1|accessories|2|Men|370|barationpri|medium|4905568ghost13407001|tomato|Oz|Unknown|19|barableoughtable| +2121|AAAAAAAAIEIAAAAA|2000-10-27||Also welsh police may keep police. So territorial cases organ|3.26|1.43|5004002|edu packscholar #2|1|classical|5|Music|370|barationpri|N/A|4905568ghost13407001|sienna|Pound|Unknown|2|oughtableoughtable| +2122|AAAAAAAAKEIAAAAA|1997-10-27|1999-10-27|Parti|3.31|1.85|6013003|exportibrand #3|13|loose stones|6|Jewelry|845|antieseeing|N/A|5506878826saddle3366|wheat|Dram|Unknown|4|ableableoughtable| +2123|AAAAAAAAKEIAAAAA|1999-10-28|2001-10-26|Most necessary weekends should enter. Things forget how|3.65|2.62|6013003|amalgamalg #2|13|dresses|1|Women|845|antieseeing|economy|5506878826saddle3366|spring|Bundle|Unknown|11|priableoughtable| +2124|AAAAAAAAKEIAAAAA|2001-10-27||Most necessary weekends should enter. Things forget how|4.65|3.76|7008005|namelessbrand #5|8|lighting|7|Home|40|barese|N/A|878516837816wheat160|navy|Box|Unknown|3|eseableoughtable| +2125|AAAAAAAANEIAAAAA|1997-10-27||Free cells get like a teeth. Pe|4.05|1.53|6005006|scholarcorp #6|5|earings|6|Jewelry|270|barationable|N/A|3938235923dark617647|tan|Bunch|Unknown|8|antiableoughtable| +2126|AAAAAAAAOEIAAAAA|1997-10-27|2000-10-26|Different, prime hills hear. Right, raw organisers put fierce, concerned years.|2.42|1.66|8001007|amalgnameless #7|1|athletic shoes|8|Sports|274|eseationable|N/A|2654847561sky2640427|royal|Pound|Unknown|89|callyableoughtable| +2127|AAAAAAAAOEIAAAAA|2000-10-27||Silly goods could i|6.58|1.66|2002002|importoimporto #2|2|shirts|2|Men|5|eseationable|large|2654847561sky2640427|medium|Case|Unknown|17|ationableoughtable| +2128|AAAAAAAAAFIAAAAA|1997-10-27|1999-10-27|Inner members make at least complete soldiers. Other workers indicate sophisticated, natural roads. Parents look; national, consc|6.28|4.45|10015004|scholaramalgamalg #4|15|portable|10|Electronics|213|prioughtable|N/A|37942555rosy09769590|thistle|Oz|Unknown|1|eingableoughtable| +2129|AAAAAAAAAFIAAAAA|1999-10-28|2001-10-26|Inner members make at least complete soldiers. Other workers indicate sophisticated, natural roads. Parents look; national, consc|3.67|1.94|10015004|amalgimporto #2|15|accessories|2|Men|100|prioughtable|small|89violet534831651369|chiffon|Cup|Unknown|86|n stableoughtable| +2130|AAAAAAAAAFIAAAAA|2001-10-27||Inner members make at least complete soldiers. Other workers indicate sophisticated, natural roads. Parents look; national, consc|4.72|2.40|5003001|exportischolar #1|3|pop|5|Music|250|barantiable|N/A|89violet534831651369|honeydew|Dozen|Unknown|31|barprioughtable| +2131|AAAAAAAADFIAAAAA|1997-10-27||Institutions say quite by no means difficult i|7.37|5.38|3001002|amalgexporti #2|1|newborn|3|Children|464|esecallyese|large|8802128166peach55503|spring|Unknown|Unknown|56|oughtprioughtable| +2132|AAAAAAAAEFIAAAAA|1997-10-27|2000-10-26|Coloured, foreign words may expe|1.96|0.72|4001001|amalgedu pack #1|1|womens|4|Shoes|296|callyn stable|extra large|rosy5740392024351711|peach|Dram|Unknown|30|ableprioughtable| +2133|AAAAAAAAEFIAAAAA|2000-10-27||Coloured, foreign words may expe|2.63|2.15|4001001|importoedu pack #2|2|mens|4|Shoes|296|callyn stable|extra large|0seashell47788280565|pink|Gram|Unknown|22|priprioughtable| +2134|AAAAAAAAGFIAAAAA|1997-10-27|1999-10-27|Crucial tests must not come merely continuing variables. Interests used to take lengthy, easy efforts. Specific men admit. Relevant names shall not f|0.69|0.28|6007007|brandcorp #7|7|pendants|6|Jewelry|301|oughtbarpri|N/A|14920878200orange577|snow|Dram|Unknown|17|eseprioughtable| +2135|AAAAAAAAGFIAAAAA|1999-10-28|2001-10-26|Crucial tests must not come merely continuing variables. Interests used to take lengthy, easy efforts. Specific men admit. Relevant names shall not f|1.54|1.15|2002002|importoimporto #2|2|shirts|2|Men|512|oughtbarpri|extra large|381purple73967184666|medium|Bundle|Unknown|32|antiprioughtable| +2136|AAAAAAAAGFIAAAAA|2001-10-27||Enough far police must not swa|6.85|1.15|5002001|importoscholar #1|2|country|5|Music|512|ableoughtanti|N/A|381purple73967184666|antique|Pallet|Unknown|40|callyprioughtable| +2137|AAAAAAAAJFIAAAAA|1997-10-27||Serious, national years could return only remarkable cars. Capital words could not give normally. Excellent chann|3.74|1.27|5004002|edu packscholar #2|4|classical|5|Music|248|eingeseable|N/A|5284899purple2436782|plum|Case|Unknown|4|ationprioughtable| +2138|AAAAAAAAKFIAAAAA|1997-10-27|2000-10-26|Waves could suit. Irish, following degrees set still; years |1.59|0.76|3001001|amalgexporti #1|1|newborn|3|Children|29|n stable|medium|954492629042897blue2|slate|Dram|Unknown|13|eingprioughtable| +2139|AAAAAAAAKFIAAAAA|2000-10-27||Strange authorities look sheer, other doctors; entirely aware systems oppose ago also free errors. Bad cultural troops describe inevitably full specimens; important areas spend |8.16|3.42|3001001|exportiamalg #2|1|maternity|1|Women|284|n stable|N/A|954492629042897blue2|papaya|Tbl|Unknown|9|n stprioughtable| +2140|AAAAAAAAMFIAAAAA|1997-10-27|1999-10-27|Appropriate losse|9.96|3.08|4003001|exportiedu pack #1|3|kids|4|Shoes|8|eing|petite|metallic901783670516|mint|Cup|Unknown|72|bareseoughtable| +2141|AAAAAAAAMFIAAAAA|1999-10-28|2001-10-26|Villages would not rest; unfair reasons must not spend gently nuclear, final numbers. Political accounts should describe now young, useful response|3.33|3.08|4003001|importoexporti #2|2|infants|3|Children|8|eing|petite|62177984drab46448302|puff|Gram|Unknown|10|oughteseoughtable| +2142|AAAAAAAAMFIAAAAA|2001-10-27||Villages would not rest; unfair reasons must not spend gently nuclear, final numbers. Political accounts should describe now young, useful response|8.22|3.45|4003001|corpnameless #3|2|furniture|7|Home|8|eing|N/A|62177984drab46448302|steel|Gross|Unknown|66|ableeseoughtable| +2143|AAAAAAAAPFIAAAAA|1997-10-27||As substantial texts rush occasionally once sufficient thousands. Then right prices agree over. Almost important losses need almost sisters. Men |6.39|3.77|5003002|exportischolar #2|3|pop|5|Music|434|esepriese|N/A|66610176puff72533415|thistle|Dozen|Unknown|27|prieseoughtable| +2144|AAAAAAAAAGIAAAAA|1997-10-27|2000-10-26|Debts may react birds. Officials will establish e|2.48|1.33|7005005|scholarbrand #5|5|blinds/shades|7|Home|238|eingpriable|N/A|tan08955165274515674|pale|Tbl|Unknown|21|eseeseoughtable| +2145|AAAAAAAAAGIAAAAA|2000-10-27||Debts may react birds. Officials will establish e|9.47|1.33|4002002|importoedu pack #2|2|mens|4|Shoes|238|eingpriable|extra large|tan08955165274515674|spring|Gross|Unknown|51|antieseoughtable| +2146|AAAAAAAACGIAAAAA|1997-10-27|1999-10-27|Fresh, poor lives may work strong, sm|3.92|2.39|9014011|edu packunivamalg #11|14|sports|9|Books|749|n steseation|N/A|6124665066firebrick3|white|Cup|Unknown|9|callyeseoughtable| +2147|AAAAAAAACGIAAAAA|1999-10-28|2001-10-26|Fresh, poor lives may work strong, sm|0.33|2.39|3002002|importoexporti #2|2|infants|3|Children|233|pripriable|medium|6124665066firebrick3|metallic|Tbl|Unknown|64|ationeseoughtable| +2148|AAAAAAAACGIAAAAA|2001-10-27||Powerful communities can enable then behind the effects. Later lengthy animals could not avoid medical, industrial countries. Eye|2.55|1.70|3002002|brandcorp #5|7|pendants|6|Jewelry|51|oughtanti|N/A|70061392068140mint12|honeydew|Ounce|Unknown|80|eingeseoughtable| +2149|AAAAAAAAFGIAAAAA|1997-10-27||Just hungry members leave generally constant, multiple |4.54|2.81|2002002|importoimporto #2|2|shirts|2|Men|430|barpriese|large|554papaya07182107324|steel|Lb|Unknown|94|n steseoughtable| +2150|AAAAAAAAGGIAAAAA|1997-10-27|2000-10-26|Marvell|1.38|0.45|10008009|namelessunivamalg #9|8|scanners|10|Electronics|314|eseoughtpri|N/A|9green27643375441084|royal|N/A|Unknown|42|barantioughtable| +2151|AAAAAAAAGGIAAAAA|2000-10-27||Foreign men will build otherwise on a f|4.24|0.45|10008009|amalgunivamalg #10|1|cameras|10|Electronics|807|ationbareing|N/A|9green27643375441084|pale|Carton|Unknown|12|oughtantioughtable| +2152|AAAAAAAAIGIAAAAA|1997-10-27|1999-10-27|More new procedures can look no longer other,|82.09|57.46|5004001|edu packscholar #1|4|classical|5|Music|342|ableesepri|N/A|441216563tomato36440|salmon|Unknown|Unknown|65|ableantioughtable| +2153|AAAAAAAAIGIAAAAA|1999-10-28|2001-10-26|More new procedures can look no longer other,|8.98|6.19|8009004|maxinameless #4|9|optics|8|Sports|342|ableesepri|N/A|96776purple974566233|saddle|Ton|Unknown|55|priantioughtable| +2154|AAAAAAAAIGIAAAAA|2001-10-27||Also royal reports use just due yards. Children put of course. Also environmental men leave thus here intelligent ends. General studen|4.72|6.19|4002001|importoedu pack #1|2|mens|4|Shoes|851|ableesepri|medium|96776purple974566233|puff|Lb|Unknown|18|eseantioughtable| +2155|AAAAAAAALGIAAAAA|1997-10-27||Fully new leaders must exercise in addition. Whole, different lists should regulate to a parts. All historic negotiations might begin just then different places; |1.32|1.00|1001002|amalgamalg #2|1|dresses|1|Women|199|n stn stought|economy|2517779olive15389695|seashell|N/A|Unknown|50|antiantioughtable| +2156|AAAAAAAAMGIAAAAA|1997-10-27|2000-10-26|Chemicals should wait of course. Even good years shall not offer cool, |88.71|61.20|10016015|corpamalgamalg #15|16|wireless|10|Electronics|320|barablepri|N/A|43380ghost4157255734|powder|Box|Unknown|55|callyantioughtable| +2157|AAAAAAAAMGIAAAAA|2000-10-27||Chemicals should wait of course. Even good years shall not offer cool, |0.88|61.20|2003002|exportiimporto #2|3|pants|2|Men|225|antiableable|extra large|0193orchid6693353073|royal|Dozen|Unknown|10|ationantioughtable| +2158|AAAAAAAAOGIAAAAA|1997-10-27|1999-10-27|Difficulties focus with a women. Favorite, regul|0.97|0.59|1004001|edu packamalg #1|4|swimwear|1|Women|420|barableese|medium|936714645926633red65|turquoise|Tsp|Unknown|16|eingantioughtable| +2159|AAAAAAAAOGIAAAAA|1999-10-28|2001-10-26|Difficulties focus with a women. Favorite, regul|6.31|3.53|1004001|exportiunivamalg #1|4|dvd/vcr players|10|Electronics|224|barableese|N/A|936714645926633red65|papaya|Carton|Unknown|30|n stantioughtable| +2160|AAAAAAAAOGIAAAAA|2001-10-27||Henceforth historic years concentrate sharp, little readers. Again happy thanks force even for a boots. Minor, responsible governments may not see only guilty facts. There economic instrument|8.79|7.38|1004001|exportiunivamalg #2|4|dvd/vcr players|10|Electronics|490|barn stese|N/A|936714645926633red65|plum|Carton|Unknown|10|barcallyoughtable| +2161|AAAAAAAABHIAAAAA|1997-10-27||Clearly particular affairs must close patterns. Urban, proper women hear open words. About central arts ought to intervene anxiously in general main problems.|9.02|3.51|4001002|amalgedu pack #2|1|womens|4|Shoes|154|eseantiought|extra large|6149877559azure73009|seashell|Oz|Unknown|13|oughtcallyoughtable| +2162|AAAAAAAACHIAAAAA|1997-10-27|2000-10-26|Administrative messages should see miles. S|6.50|5.20|10016004|corpamalgamalg #4|16|wireless|10|Electronics|544|eseeseanti|N/A|1530269peru997988350|violet|Oz|Unknown|6|ablecallyoughtable| +2163|AAAAAAAACHIAAAAA|2000-10-27||Administrative messages should see miles. S|48.77|25.84|10016004|amalgscholar #2|1|rock|5|Music|89|n steing|N/A|697729743peach198506|cornflower|Box|Unknown|3|pricallyoughtable| +2164|AAAAAAAAEHIAAAAA|1997-10-27|1999-10-27|Safe, hot others write even anywhere hard movements. Inc, short views keep naturally radical women; criticisms find sooner. Now eager practices shall know thus unacceptable similar|0.21|0.14|6001005|amalgcorp #5|1|birdal|6|Jewelry|508|eingbaranti|N/A|1175deep777027735301|khaki|Ton|Unknown|87|esecallyoughtable| +2165|AAAAAAAAEHIAAAAA|1999-10-28|2001-10-26|Safe, hot others write even anywhere hard movements. Inc, short views keep naturally radical women; criticisms find sooner. Now eager practices shall know thus unacceptable similar|0.41|0.14|8004006|edu packnameless #6|1|camping|8|Sports|508|eingbaranti|N/A|1175deep777027735301|salmon|Ton|Unknown|26|anticallyoughtable| +2166|AAAAAAAAEHIAAAAA|2001-10-27||Experienced, new doubts see also however fine sides. More controversial times weaken mainly dark eyes. Best basic women would not satisfy once thus great appointments|1.96|0.14|8004006|maxicorp #7|1|womens watch|6|Jewelry|508|eingbaranti|N/A|4554140purple7366592|purple|Lb|Unknown|33|callycallyoughtable| +2167|AAAAAAAAHHIAAAAA|1997-10-27||Evenings go simply central conditions. Small, other characters must not sha|2.79|1.45|9016008|corpunivamalg #8|16|mystery|9|Books|281|oughteingable|N/A|093649926150indian91|spring|Gross|Unknown|31|ationcallyoughtable| +2168|AAAAAAAAIHIAAAAA|1997-10-27|2000-10-26|Economic things determine. However overseas teachers listen clearly by a types; signs telephone probably. Environmental ty|16.26|8.78|7001009|amalgbrand #9|1|bathroom|7|Home|406|callybarese|N/A|88952094thistle13515|light|Ton|Unknown|30|eingcallyoughtable| +2169|AAAAAAAAIHIAAAAA|2000-10-27||Alike dark benefits solve individuals. Occasional, famous services succeed possible, overall trials. Relatively young successes shall not|0.37|0.28|6008002|namelesscorp #2|8|mens watch|6|Jewelry|406|callybarese|N/A|84peru08731710820294|medium|Lb|Unknown|24|n stcallyoughtable| +2170|AAAAAAAAKHIAAAAA|1997-10-27|1999-10-27|Almost various customs shut so investors; changes will not ask|1.66|1.31|6003003|exporticorp #3|3|gold|6|Jewelry|599|n stn stanti|N/A|43lime12188306731129|peach|Bunch|Unknown|76|barationoughtable| +2171|AAAAAAAAKHIAAAAA|1999-10-28|2001-10-26|Elsewhere important goods give only british years. Effective, ill cases tell ever on a figures; romantic, western photographs used to know more general implicati|8.70|1.31|6003003|importoamalg #2|2|fragrances|1|Women|599|n stn stanti|extra large|43lime12188306731129|saddle|Each|Unknown|35|oughtationoughtable| +2172|AAAAAAAAKHIAAAAA|2001-10-27||Things light systematicall|7.57|1.31|8002003|importonameless #3|2|baseball|8|Sports|71|n stn stanti|N/A|07912934yellow656238|slate|Bunch|Unknown|21|ableationoughtable| +2173|AAAAAAAANHIAAAAA|1997-10-27||Simple guests leave british, skilled terms. Kind, little standards must suspect. Combinations may think like, distinguished inches. Artists beat awfully. Ide|1.68|0.53|4004002|edu packedu pack #2|4|athletic|4|Shoes|95|antin st|medium|741698554midnight100|moccasin|Tsp|Unknown|9|priationoughtable| +2174|AAAAAAAAOHIAAAAA|1997-10-27|2000-10-26|Special, internal books will turn even coherent losses. Even afraid regulations ought to admit then in a features. Other, deep demands |0.38|0.14|4002001|importoedu pack #1|2|mens|4|Shoes|176|callyationought|medium|5peru658543032734783|royal|Case|Unknown|50|eseationoughtable| +2175|AAAAAAAAOHIAAAAA|2000-10-27||Pains pretend only only little toys. Otherwise linear questions might question short|9.86|5.42|4002001|exportiexporti #2|2|toddlers|3|Children|201|callyationought|petite|5peru658543032734783|cyan|Carton|Unknown|12|antiationoughtable| +2176|AAAAAAAAAIIAAAAA|1997-10-27|1999-10-27|Enough future writings hear closer at a areas. However early farmers win. Only special mothers make then american programmes. Political boys avoid |5.75|3.04|10012001|importoamalgamalg #1|12|monitors|10|Electronics|32|ablepri|N/A|purple22515798827381|powder|Bundle|Unknown|55|callyationoughtable| +2177|AAAAAAAAAIIAAAAA|1999-10-28|2001-10-26|Enough future writings hear closer at a areas. However early farmers win. Only special mothers make then american programmes. Political boys avoid |70.78|33.97|10012001|edu packscholar #2|4|classical|5|Music|933|priprin st|N/A|purple22515798827381|blush|Dram|Unknown|25|ationationoughtable| +2178|AAAAAAAAAIIAAAAA|2001-10-27||Schools explain black aspects. Again monetary farmers should not pay strictly prime new institutions; apparent, certain institutions mus|4.42|33.97|6015003|scholarbrand #3|4|custom|6|Jewelry|933|priprin st|N/A|purple22515798827381|steel|Pallet|Unknown|18|eingationoughtable| +2179|AAAAAAAADIIAAAAA|1997-10-27||Members tell very new, final legs; expensive skills will break gradually also provincial differences. Legally red areas perform institutions. E|0.43|0.35|4001002|amalgedu pack #2|1|womens|4|Shoes|483|prieingese|small|tan96365563116018492|gainsboro|Tbl|Unknown|15|n stationoughtable| +2180|AAAAAAAAEIIAAAAA|1997-10-27|2000-10-26|Independent, other conclusions ought to die hands. Proposed, lovely days celebrate doubtless children. Correct, eastern kinds used to teach across social, gradual years; here seriou|41.55|36.97|7016001|corpnameless #1|16|furniture|7|Home|112|ableoughtought|N/A|56526085688wheat0584|puff|Pallet|Unknown|1|bareingoughtable| +2181|AAAAAAAAEIIAAAAA|2000-10-27||Difficult cells grow quite scottish institutions. Little, mere procedures clear through a issues. Poor novels kill pre|1.26|0.74|7016001|amalgexporti #2|1|newborn|3|Children|112|ableoughtought|small|56526085688wheat0584|slate|Gross|Unknown|60|oughteingoughtable| +2182|AAAAAAAAGIIAAAAA|1997-10-27|1999-10-27|Usually temporary classes can apply|3.20|1.79|8010003|univmaxi #3|10|pools|8|Sports|213|prioughtable|N/A|8302011724puff609992|royal|Lb|Unknown|26|ableeingoughtable| +2183|AAAAAAAAGIIAAAAA|1999-10-28|2001-10-26|Future prices may combine. Better possible edges would mon|2.80|1.79|10006008|corpunivamalg #8|10|musical|10|Electronics|213|prioughtable|N/A|75peach7503432651583|red|Ton|Unknown|33|prieingoughtable| +2184|AAAAAAAAGIIAAAAA|2001-10-27||Future prices may combine. Better possible edges would mon|3.42|1.79|10006008|edu packamalg #1|4|swimwear|1|Women|126|prioughtable|petite|75peach7503432651583|tan|N/A|Unknown|39|eseeingoughtable| +2185|AAAAAAAAJIIAAAAA|1997-10-27||Large boards give carefully just mixed services. Cases shall colle|0.65|0.44|6012002|importobrand #2|12|costume|6|Jewelry|207|ationbarable|N/A|447383429213salmon09|spring|Pound|Unknown|43|antieingoughtable| +2186|AAAAAAAAKIIAAAAA|1997-10-27|2000-10-26|For example new children shall take general jobs. British, proposed government|5.52|4.80|7002007|importobrand #7|2|bedding|7|Home|273|priationable|N/A|05797sky062835848376|sienna|Tsp|Unknown|33|callyeingoughtable| +2187|AAAAAAAAKIIAAAAA|2000-10-27||For example new children shall take general jobs. British, proposed government|9.73|4.28|6001004|amalgcorp #4|1|birdal|6|Jewelry|273|priationable|N/A|7658346887moccasin59|sky|Bunch|Unknown|15|ationeingoughtable| +2188|AAAAAAAAMIIAAAAA|1997-10-27|1999-10-27|Tonnes live levels. Public countries use yet police; at once natural words say; international shares communicate. Decent difficulties like free, professional personnel. So recent faces |3.84|1.15|2002001|importoimporto #1|2|shirts|2|Men|533|priprianti|large|03221179915222pink82|tomato|Carton|Unknown|37|eingeingoughtable| +2189|AAAAAAAAMIIAAAAA|1999-10-28|2001-10-26|Tonnes live levels. Public countries use yet police; at once natural words say; international shares communicate. Decent difficulties like free, professional personnel. So recent faces |2.08|1.15|2002001|scholarbrand #10|5|blinds/shades|7|Home|155|antiantiought|N/A|03221179915222pink82|rose|Pallet|Unknown|12|n steingoughtable| +2190|AAAAAAAAMIIAAAAA|2001-10-27||Lacking failures swallow like the processes; small, social figures see to a procedures. Regular, historical matters must lift again on a systems. Pale expens|0.37|1.15|5002001|importoscholar #1|2|country|5|Music|155|antiantiought|N/A|005029342366sky16279|peru|N/A|Unknown|34|barn stoughtable| +2191|AAAAAAAAPIIAAAAA|1997-10-27||Chinese rules must not provide very only courts. Interests ought to carry troubles; probably elderly stars take local homes. Then initial parents complain either central h|3.07|2.45|10011016|amalgamalgamalg #16|11|disk drives|10|Electronics|550|barantianti|N/A|130coral180419734225|light|Dozen|Unknown|63|oughtn stoughtable| +2192|AAAAAAAAAJIAAAAA|1997-10-27|2000-10-26|Joint, superior police would use through an restrictions. Buyers ought to contract generally in a efforts. Days cut also sure, frequent s|0.43|0.21|9001009|amalgmaxi #9|1|arts|9|Books|89|n steing|N/A|804580powder29560302|saddle|Dozen|Unknown|16|ablen stoughtable| +2193|AAAAAAAAAJIAAAAA|2000-10-27||Joint, superior police would use through an restrictions. Buyers ought to contract generally in a efforts. Days cut also sure, frequent s|45.20|40.22|9001009|edu packnameless #4|4|camping|8|Sports|16|callyought|N/A|804580powder29560302|purple|Bundle|Unknown|88|prin stoughtable| +2194|AAAAAAAACJIAAAAA|1997-10-27|1999-10-27|Prisoners must not end well. Hope|0.96|0.57|8004005|edu packnameless #5|4|camping|8|Sports|37|ationpri|N/A|7plum225903579203462|black|Pound|Unknown|10|esen stoughtable| +2195|AAAAAAAACJIAAAAA|1999-10-28|2001-10-26|Prisoners must not end well. Hope|20.80|12.68|7001006|amalgbrand #6|4|bathroom|7|Home|37|ationpri|N/A|7plum225903579203462|puff|Unknown|Unknown|32|antin stoughtable| +2196|AAAAAAAACJIAAAAA|2001-10-27||Prisoners must not end well. Hope|1.81|12.68|5001001|amalgscholar #1|1|rock|5|Music|37|ationpri|N/A|7plum225903579203462|midnight|Oz|Unknown|11|callyn stoughtable| +2197|AAAAAAAAFJIAAAAA|1997-10-27||Simple, desperate lines speak very; different, capit|3.58|2.57|2001002|amalgimporto #2|1|accessories|2|Men|119|n stoughtought|large|61907turquoise377235|royal|Cup|Unknown|14|ationn stoughtable| +2198|AAAAAAAAGJIAAAAA|1997-10-27|2000-10-26|Just modern pictures would put considerations. Like homes check hard, ethnic words. Then new books cannot flood here by the qualities; marks shall pay jobs. Huge, model environments ca|3.63|1.16|8008009|namelessnameless #9|8|outdoor|8|Sports|62|ablecally|N/A|8986papaya6262862145|purple|Pallet|Unknown|16|eingn stoughtable| +2199|AAAAAAAAGJIAAAAA|2000-10-27||Gas|29.12|1.16|8008009|importomaxi #10|12|guns|8|Sports|62|ablecally|N/A|771slate553205487289|turquoise|Gross|Unknown|31|n stn stoughtable| +2200|AAAAAAAAIJIAAAAA|1997-10-27|1999-10-27|Genuine factors should not address in a units; new windows find closely present, open shadows; centuries should |3.93|2.39|4003001|exportiedu pack #1|3|kids|4|Shoes|319|n stoughtpri|petite|78166384807300smoke8|aquamarine|Carton|Unknown|6|barbarableable| +2201|AAAAAAAAIJIAAAAA|1999-10-28|2001-10-26|Obligations stay efficiently |58.43|2.39|4003001|exportiedu pack #2|3|kids|4|Shoes|751|n stoughtpri|large|78166384807300smoke8|mint|Lb|Unknown|47|oughtbarableable| +2202|AAAAAAAAIJIAAAAA|2001-10-27||Correct, useful details go. Low, nuclear pupils will not answer reportedly disabled, environmental things. Due, electrical liabilities find among a lessons. Barely middl|1.96|1.13|8007003|brandnameless #3|3|hockey|8|Sports|236|n stoughtpri|N/A|33868pink80467760995|turquoise|N/A|Unknown|9|ablebarableable| +2203|AAAAAAAALJIAAAAA|1997-10-27||Healthy, new industries learn exclusively economic, automatic gaps. Foreigners develop ever just little children; surely yellow things distinguish in |5.97|3.22|3002002|importoexporti #2|2|infants|3|Children|728|eingableation|medium|5864943mint646768116|white|Gram|Unknown|25|pribarableable| +2204|AAAAAAAAMJIAAAAA|1997-10-27|2000-10-26|Social, evident earnings reach nearly dark schools. Else practical citizens cease days; ever other ambitions can catch good, inc services|32.03|25.30|3001001|amalgexporti #1|1|newborn|3|Children|898|eingn steing|medium|3664380plum387757320|tan|Pound|Unknown|65|esebarableable| +2205|AAAAAAAAMJIAAAAA|2000-10-27||Bad attempts form now high notes. Relevant classes can last great skills. Clever, harsh restricti|8.15|25.30|3001002|amalgexporti #2|1|newborn|3|Children|18|eingn steing|N/A|32733293violet309643|pink|Bunch|Unknown|1|antibarableable| +2206|AAAAAAAAOJIAAAAA|1997-10-27|1999-10-27|Redundant children will not replace at all useful hospitals; technical|1.32|0.97|9012011|importounivamalg #11|12|home repair|9|Books|121|oughtableought|N/A|50538steel5502321565|bisque|Bundle|Unknown|10|callybarableable| +2207|AAAAAAAAOJIAAAAA|1999-10-28|2001-10-26|Redundant children will not replace at all useful hospitals; technical|2.06|0.97|9012011|corpbrand #8|6|rugs|7|Home|121|oughtableought|N/A|50538steel5502321565|tomato|Cup|Unknown|5|ationbarableable| +2208|AAAAAAAAOJIAAAAA|2001-10-27||Duties tell already revolutionary countries. Funds visit already in a armies. Here small years trust quie|2.20|0.97|9013001|exportiunivamalg #1|13|self-help|9|Books|121|oughtableought|N/A|50538steel5502321565|honeydew|Oz|Unknown|2|eingbarableable| +2209|AAAAAAAABKIAAAAA|1997-10-27||More moral cases like by no means military, new savings. Other, unlike clients shall complete just then warm motives; grateful, inner hours think; very significant leaders play e|2.92|1.48|1004002|edu packamalg #2|4|swimwear|1|Women|427|ationableese|small|51125613grey62943795|steel|Oz|Unknown|18|n stbarableable| +2210|AAAAAAAACKIAAAAA|1997-10-27|2000-10-26|Also important gardens reflect above samples. Geographical protests date quite; brothers used to go pretty by a ma|0.99|0.29|9013003|exportiunivamalg #3|13|self-help|9|Books|111|oughtoughtought|N/A|401sandy493861022472|violet|Carton|Unknown|21|baroughtableable| +2211|AAAAAAAACKIAAAAA|2000-10-27||Precious ages will not come for example less old tickets. Emotions tell small countries. Ministers used to train more male pr|0.37|0.29|3003002|exportiexporti #2|3|toddlers|3|Children|94|esen st|small|91midnight4815543806|sandy|N/A|Unknown|2|oughtoughtableable| +2212|AAAAAAAAEKIAAAAA|1997-10-27|1999-10-27|Unlikely letters inhibit only jobs. Brightly hard procedures might eat mainly complex odd tories. Powers would not achieve too dem|2.51|1.95|8014003|edu packmaxi #3|14|tennis|8|Sports|551|oughtantianti|N/A|1532turquoise7143679|steel|Tbl|Unknown|17|ableoughtableable| +2213|AAAAAAAAEKIAAAAA|1999-10-28|2001-10-26|Right other savings extend automatically new, local novels. New plants must need legal conditions; st|7.37|1.95|8014003|edu packnameless #4|14|glassware|7|Home|551|oughtantianti|N/A|1532turquoise7143679|royal|Each|Unknown|84|prioughtableable| +2214|AAAAAAAAEKIAAAAA|2001-10-27||Years could not find so files. Across grey police cannot talk patients. Contracts make; often successful assets may not take accurate, late points. Arrangeme|80.21|42.51|1002001|importoamalg #1|14|fragrances|1|Women|551|oughtantianti|extra large|40969769yellow699075|papaya|Bundle|Unknown|9|eseoughtableable| +2215|AAAAAAAAHKIAAAAA|1997-10-27||Eastern criteria shall work natural homes. Nuclear tourists should not get ever too curious things. Sti|1.66|1.49|6013008|exportibrand #8|13|loose stones|6|Jewelry|531|oughtprianti|N/A|5wheat13881559158679|salmon|Tbl|Unknown|49|antioughtableable| +2216|AAAAAAAAIKIAAAAA|1997-10-27|2000-10-26|Visual e|7.68|5.06|3001001|amalgexporti #1|1|newborn|3|Children|53|prianti|petite|958seashell842390286|chartreuse|Unknown|Unknown|49|callyoughtableable| +2217|AAAAAAAAIKIAAAAA|2000-10-27||Groups include suddenly total, red kids. Examinations shall include estimated pp.. Years offer co|4.56|4.10|3001001|amalgamalg #2|1|dresses|1|Women|53|prianti|extra large|958seashell842390286|bisque|Bunch|Unknown|83|ationoughtableable| +2218|AAAAAAAAKKIAAAAA|1997-10-27|1999-10-27|Naked, popular schemes campaign then offices. Underlying shares may join|79.28|31.71|9001011|amalgmaxi #11|1|arts|9|Books|168|eingcallyought|N/A|46106654759saddle518|pink|Case|Unknown|41|eingoughtableable| +2219|AAAAAAAAKKIAAAAA|1999-10-28|2001-10-26|Only normal components cannot express electronic, huge features. Small, statutory men go. Risks cannot take and so forth messages. Others think tears. Home religious incomes examine|1.57|31.71|6008004|namelesscorp #4|1|mens watch|6|Jewelry|168|eingcallyought|N/A|46106654759saddle518|magenta|Gram|Unknown|38|n stoughtableable| +2220|AAAAAAAAKKIAAAAA|2001-10-27||Only normal components cannot express electronic, huge features. Small, statutory men go. Risks cannot take and so forth messages. Others think tears. Home religious incomes examine|5.24|31.71|7011001|amalgnameless #1|1|accent|7|Home|255|eingcallyought|N/A|6149603862tomato1522|sienna|Dram|Unknown|28|barableableable| +2221|AAAAAAAANKIAAAAA|1997-10-27||Methods used to perform eggs; now good years diversify only |8.37|5.60|8004002|edu packnameless #2|4|camping|8|Sports|46|callyese|N/A|9523472deep969275299|cornflower|Gross|Unknown|19|oughtableableable| +2222|AAAAAAAAOKIAAAAA|1997-10-27|2000-10-26|Just natural directors would limit. Necessarily ready parts spend more important refugees. Folk would not use events. Unacceptable dreams believe i|70.08|26.63|5003001|exportischolar #1|3|pop|5|Music|175|antiationought|N/A|rosy5872315100149673|slate|Tbl|Unknown|74|ableableableable| +2223|AAAAAAAAOKIAAAAA|2000-10-27||Just natural directors would limit. Necessarily ready parts spend more important refugees. Folk would not use events. Unacceptable dreams believe i|1.84|26.63|4003002|exportiedu pack #2|3|kids|4|Shoes|312|ableoughtpri|petite|0605111575221orchid3|pale|Cup|Unknown|47|priableableable| +2224|AAAAAAAAALIAAAAA|1997-10-27|1999-10-27|Effective, chinese areas follow roughly |3.22|1.77|10007015|brandunivamalg #15|7|personal|10|Electronics|511|oughtoughtanti|N/A|46650244violet764240|honeydew|Gram|Unknown|26|eseableableable| +2225|AAAAAAAAALIAAAAA|1999-10-28|2001-10-26|Only parties must choose in a police. Conventional, common men capture with a |0.27|0.15|4004002|edu packedu pack #2|4|athletic|4|Shoes|170|barationought|medium|656chiffon3640197857|snow|Carton|Unknown|1|antiableableable| +2226|AAAAAAAAALIAAAAA|2001-10-27||Certainly appropriate stages make nations. Safe, nuclear members used to happen as; fine, national charges should give about the lips. Industrial, liberal st|6.43|0.15|4004002|edu packamalgamalg #17|4|automotive|10|Electronics|180|barationought|N/A|656chiffon3640197857|saddle|Carton|Unknown|32|callyableableable| +2227|AAAAAAAADLIAAAAA|1997-10-27||Stupid ingredients must not raise adjacent hours. Large, white re|9.59|6.04|1004002|edu packamalg #2|4|swimwear|1|Women|24|eseable|medium|0250248495534snow158|turquoise|Each|Unknown|23|ationableableable| +2228|AAAAAAAAELIAAAAA|1997-10-27|2000-10-26|So heavy years may not laugh only votes. Ministers explain over. Left sides should bring then reluctant products. Elements would issue little now full societ|8.91|7.39|10001002|amalgunivamalg #2|1|cameras|10|Electronics|128|eingableought|N/A|1392493623533red9775|turquoise|Ton|Unknown|15|eingableableable| +2229|AAAAAAAAELIAAAAA|2000-10-27||So heavy years may not laugh only votes. Ministers explain over. Left sides should bring then reluctant products. Elements would issue little now full societ|6.74|2.42|8004010|edu packnameless #10|4|camping|8|Sports|128|eingableought|N/A|738852505cornsilk447|white|Dram|Unknown|62|n stableableable| +2230|AAAAAAAAGLIAAAAA|1997-10-27|1999-10-27|Subjects |0.09|0.04|6010007|univbrand #7|10|jewelry boxes|6|Jewelry|283|prieingable|N/A|56green4889375446102|slate|Lb|Unknown|32|barpriableable| +2231|AAAAAAAAGLIAAAAA|1999-10-28|2001-10-26|Subjects |4.77|2.67|10010005|univamalgamalg #5|10|memory|10|Electronics|283|prieingable|N/A|56green4889375446102|plum|Each|Unknown|11|oughtpriableable| +2232|AAAAAAAAGLIAAAAA|2001-10-27||Subjects |7.82|2.58|10010005|brandmaxi #1|7|reference|9|Books|283|prieingable|N/A|56green4889375446102|salmon|Lb|Unknown|57|ablepriableable| +2233|AAAAAAAAJLIAAAAA|1997-10-27||Foreign years should say at least to a firms. African, direct children become yesterday. Today heavy circumstances say ago likely childre|2.21|1.96|7006004|corpbrand #4|6|rugs|7|Home|504|esebaranti|N/A|806944747272125red61|floral|Bundle|Unknown|4|pripriableable| +2234|AAAAAAAAKLIAAAAA|1997-10-27|2000-10-26|Resources must show at least available, beautiful nations. So|9.91|3.76|4004001|edu packedu pack #1|4|athletic|4|Shoes|261|oughtcallyable|extra large|4869619100764medium8|slate|Carton|Unknown|50|esepriableable| +2235|AAAAAAAAKLIAAAAA|2000-10-27||Resources must show at least available, beautiful nations. So|94.81|3.76|5002002|importoscholar #2|2|country|5|Music|332|oughtcallyable|N/A|51551606almond179667|wheat|Pound|Unknown|25|antipriableable| +2236|AAAAAAAAMLIAAAAA|1997-10-27|1999-10-27|Urban, damp chang|1.51|1.08|10013010|exportiamalgamalg #10|13|stereo|10|Electronics|100|barbarought|N/A|48845792018199tan365|pink|Lb|Unknown|16|callypriableable| +2237|AAAAAAAAMLIAAAAA|1999-10-28|2001-10-26|Seconds would n|5.71|2.91|4003002|exportiedu pack #2|13|kids|4|Shoes|100|barbarought|extra large|4912peru319373676785|white|Carton|Unknown|25|ationpriableable| +2238|AAAAAAAAMLIAAAAA|2001-10-27||Contracts investigate trends. Supplies obtain formal, social parties. Foreign, free environments should understand times. Particular, experimental relations choose so rough police. Re|4.04|2.66|4003002|importoexporti #1|13|infants|3|Children|106|barbarought|small|yellow89829491246347|tan|Pallet|Unknown|47|eingpriableable| +2239|AAAAAAAAPLIAAAAA|1997-10-27||Real, fair sales used to lend much drawings. Tanks believe new, present minutes. Contemporary, lovely contributions happen stairs. Problems keep. However sha|1.13|0.96|7007010|brandbrand #10|7|decor|7|Home|148|eingeseought|N/A|38seashell1080812204|purple|Case|Unknown|20|n stpriableable| +2240|AAAAAAAAAMIAAAAA|1997-10-27|2000-10-26|Open signs may not move articles. Little spaces could welcome to a privileges. Able cells take great cells. Sales like eventually certain,|2.52|2.01|5002001|importoscholar #1|2|country|5|Music|248|eingeseable|N/A|8044844730rose894128|lime|Carton|Unknown|5|bareseableable| +2241|AAAAAAAAAMIAAAAA|2000-10-27||Buildings should cease so reasonable, western regulations. Ancient weeks join on a participants; national, wide meetings answer thus things; di|2.93|2.01|4004002|edu packedu pack #2|2|athletic|4|Shoes|518|eingoughtanti|large|8044844730rose894128|sandy|Box|Unknown|28|oughteseableable| +2242|AAAAAAAACMIAAAAA|1997-10-27|1999-10-27|Smooth stations make troubles. Dealers will sleep only regularly head possibilities. Never young days wish military asset|4.29|1.67|6009003|maxicorp #3|9|womens watch|6|Jewelry|90|barn st|N/A|8570powder4951686462|yellow|Each|Unknown|68|ableeseableable| +2243|AAAAAAAACMIAAAAA|1999-10-28|2001-10-26|Smooth stations make troubles. Dealers will sleep only regularly head possibilities. Never young days wish military asset|0.10|1.67|6009003|amalgnameless #4|9|athletic shoes|8|Sports|90|barn st|N/A|8570powder4951686462|cornflower|Dram|Unknown|5|prieseableable| +2244|AAAAAAAACMIAAAAA|2001-10-27||||||exportiunivamalg #1|13|self-help|9||||N/A|8570powder4951686462|||||| +2245|AAAAAAAAFMIAAAAA|1997-10-27||Metals might not reassure below particular, human measures. Patterns must not go very wide times; activities happen off a photographs. Sorry, beneficial industries try logically at the parents. Larg|9.93|5.75|2004002|edu packimporto #2|4|sports-apparel|2|Men|7|ation|medium|6875041466papaya9544|papaya|Dram|Unknown|3|antieseableable| +2246|AAAAAAAAGMIAAAAA|1997-10-27|2000-10-26|Ancient, joint cases sit then. Eyes regard especially green, necessary years. Fine cards point liberal blocks. Always fe|42.23|15.62|3003001|exportiexporti #1|3|toddlers|3|Children|577|ationationanti|medium|288tomato07855460319|honeydew|Lb|Unknown|22|callyeseableable| +2247|AAAAAAAAGMIAAAAA|2000-10-27||Ancient, joint cases sit then. Eyes regard especially green, necessary years. Fine cards point liberal blocks. Always fe|9.79|4.40|3003001|importoamalg #2|2|fragrances|1|Women|345|ationationanti|extra large|288tomato07855460319|pale|Tbl|Unknown|52|ationeseableable| +2248|AAAAAAAAIMIAAAAA|1997-10-27|1999-10-27|All generous children could start already to a years|0.45|0.22|6009001|maxicorp #1|9|womens watch|6|Jewelry|779|n stationation|N/A|5390976529honeydew00|navajo|N/A|Unknown|32|eingeseableable| +2249|AAAAAAAAIMIAAAAA|1999-10-28|2001-10-26|Angles can buy over logical houses. Regional, mixed others agree too jointly rapid elements. Men f|0.33|0.10|1001002|amalgamalg #2|9|dresses|1|Women|779|n stationation|medium|5390976529honeydew00|spring|Oz|Unknown|73|n steseableable| +2250|AAAAAAAAIMIAAAAA|2001-10-27||Regularly open cases will make tomorrow. Together white|4.52|0.10|6010003|univbrand #3|9|jewelry boxes|6|Jewelry|95|n stationation|N/A|5390976529honeydew00|saddle|Each|Unknown|14|barantiableable| +2251|AAAAAAAALMIAAAAA|1997-10-27||Then bad dishes like open at a workers. Accurate, cu|3.21|1.66|2002002|importoimporto #2|2|shirts|2|Men|507|ationbaranti|medium|3snow523285706962079|burlywood|N/A|Unknown|44|oughtantiableable| +2252|AAAAAAAAMMIAAAAA|1997-10-27|2000-10-26|Scenes should|8.25|4.78|7008003|namelessbrand #3|8|lighting|7|Home|329|n stablepri|N/A|235149899844spring51|violet|Gross|Unknown|21|ableantiableable| +2253|AAAAAAAAMMIAAAAA|2000-10-27||As fast words restore plants. Religious pages could try so gold, lonely visitors. Unusual weeks go otherwise. Waves shall l|31.92|12.76|6010006|univbrand #6|8|jewelry boxes|6|Jewelry|329|n stablepri|N/A|235149899844spring51|seashell|Dram|Unknown|23|priantiableable| +2254|AAAAAAAAOMIAAAAA|1997-10-27|1999-10-27|Local holidays give dominant, good chapters. Short terms will not meet. Other, available employees could say yesterday important hands; movements used to pay |7.51|2.92|4002001|importoedu pack #1|2|mens|4|Shoes|159|n stantiought|small|rosy5142243983576162|tomato|Lb|Unknown|68|eseantiableable| +2255|AAAAAAAAOMIAAAAA|1999-10-28|2001-10-26|Essential, keen groups ought to replace long eyes; desperate years would not help worth a ste|2.53|2.92|3002002|importoexporti #2|2|infants|3|Children|228|eingableable|economy|rosy5142243983576162|tomato|Dram|Unknown|13|antiantiableable| +2256|AAAAAAAAOMIAAAAA|2001-10-27||New workshops sleep never other, advanced samples. Expected plants mean also individual grants; social, free users ought to play maybe. A bit valid discussions should turn stars. Available times wo|0.23|2.92|2004001|edu packimporto #1|4|sports-apparel|2|Men|659|eingableable|economy|rosy5142243983576162|papaya|Pound|Unknown|55|callyantiableable| +2257|AAAAAAAABNIAAAAA|1997-10-27||Toge|3.49|2.05|2004002|edu packimporto #2|4|sports-apparel|2|Men|291|oughtn stable|large|8530680670misty46170|slate|Gram|Unknown|17|ationantiableable| +2258|AAAAAAAACNIAAAAA|1997-10-27|2000-10-26|Average hands may not present very else old years. Too pregnant members escape mechanisms; rather roug|2.00|1.04|4001001|amalgedu pack #1|1|womens|4|Shoes|109|n stbarought|medium|470753208yellow44579|lawn|Pallet|Unknown|62|eingantiableable| +2259|AAAAAAAACNIAAAAA|2000-10-27||Experimental, personal sides suit level, subj|0.46|0.17|4001001|importoimporto #2|2|shirts|2|Men|579|n stbarought|petite|470753208yellow44579|yellow|Tsp|Unknown|3|n stantiableable| +2260|AAAAAAAAENIAAAAA|1997-10-27|1999-10-27|Open enquiries could want. Examples investigate. Future policies will match once tory cigarettes. Hard examples should heal schools. Later top guests cannot scre|0.21|0.09|10014017|edu packamalgamalg #17|14|automotive|10|Electronics|436|callypriese|N/A|21powder130362587571|steel|Pallet|Unknown|33|barcallyableable| +2261|AAAAAAAAENIAAAAA|1999-10-28|2001-10-26|Prime, fit issues will make only. Never particular seconds think well critical studies. Also financial supporters bear. Short colours cost bad, angry experts. P|8.99|0.09|10014017|edu packexporti #2|4|school-uniforms|3|Children|436|callypriese|large|7385983433237spring4|tomato|Tbl|Unknown|9|oughtcallyableable| +2262|AAAAAAAAENIAAAAA|2001-10-27||Fierce, new feelings should raise times. |3.26|0.09|10014017|exportinameless #3|3|basketball|8|Sports|7|callypriese|N/A|7385983433237spring4|rosy|Box|Unknown|58|ablecallyableable| +2263|AAAAAAAAHNIAAAAA|1997-10-27||Unfair, likely authorities shall stop|9.82|6.18|10009003|maxiunivamalg #3|9|televisions|10|Electronics|566|callycallyanti|N/A|59336649purple891268|indian|Bunch|Unknown|85|pricallyableable| +2264|AAAAAAAAINIAAAAA|1997-10-27|2000-10-26|Civil observations tel|61.69|53.05|5004001|edu packscholar #1|4|classical|5|Music|191|oughtn stought|N/A|59snow42324462417410|thistle|Each|Unknown|52|esecallyableable| +2265|AAAAAAAAINIAAAAA|2000-10-27||Civil observations tel|4.46|53.05|3001002|amalgexporti #2|4|newborn|3|Children|191|oughtn stought|medium|59snow42324462417410|yellow|Ton|Unknown|8|anticallyableable| +2266|AAAAAAAAKNIAAAAA|1997-10-27|1999-10-27|Developments explode connections. Studie|1.33|0.43|2001001|amalgimporto #1|1|accessories|2|Men|253|priantiable|economy|1green68880746568829|moccasin|Gram|Unknown|51|callycallyableable| +2267|AAAAAAAAKNIAAAAA|1999-10-28|2001-10-26|Required areas result again. Again whole passengers let ugly views. Available, secondary sales meet otherwise just old lives. Ex|1.84|1.12|2001001|exportiamalg #2|1|maternity|1|Women|762|ablecallyation|large|1green68880746568829|rose|Lb|Unknown|26|ationcallyableable| +2268|AAAAAAAAKNIAAAAA|2001-10-27||Required areas result again. Again whole passengers let ugly views. Available, secondary sales meet otherwise just old lives. Ex|4.74|1.51|2001001|exportiimporto #1|1|pants|2|Men|452|ablecallyation|extra large|1green68880746568829|peach|Gross|Unknown|23|eingcallyableable| +2269|AAAAAAAANNIAAAAA|1997-10-27||Dear patients give again able directors. Modest terms think. For example assistant|1.89|1.60|9003002|exportimaxi #2|3|computers|9|Books|921|oughtablen st|N/A|4504041210040plum048|snow|Dozen|Unknown|82|n stcallyableable| +2270|AAAAAAAAONIAAAAA|1997-10-27|2000-10-26|Main problems seek seconds; originally impossible sales like successful, big women. Real twins emerge in order for example extra expectations. Togethe|2.23|0.98|1004001|edu packamalg #1|4|swimwear|1|Women|237|ationpriable|small|79saddle120617009559|thistle|Bunch|Unknown|31|barationableable| +2271|AAAAAAAAONIAAAAA|2000-10-27||National items sha|3.60|1.44|1004001|importomaxi #2|12|guns|8|Sports|451|ationpriable|N/A|2610steel65982596805|pink|Tbl|Unknown|14|oughtationableable| +2272|AAAAAAAAAOIAAAAA|1997-10-27|1999-10-27|New contributions manage now from a r|8.95|2.95|2001001|amalgimporto #1|1|accessories|2|Men|322|ableablepri|large|06yellow927227572776|sandy|Dram|Unknown|52|ableationableable| +2273|AAAAAAAAAOIAAAAA|1999-10-28|2001-10-26|Meanings endorse here. National, individual movements should make other children. Political, able |2.65|1.74|2001001|maxibrand #4|9|mattresses|7|Home|396|callyn stpri|N/A|06yellow927227572776|wheat|Box|Unknown|53|priationableable| +2274|AAAAAAAAAOIAAAAA|2001-10-27||Meanings endorse here. National, individual movements should make other children. Political, able |7.58|1.74|2001001|exportiunivamalg #14|9|dvd/vcr players|10|Electronics|100|callyn stpri|N/A|9821567pale538910982|pale|Pound|Unknown|11|eseationableable| +2275|AAAAAAAADOIAAAAA|1997-10-27||Particular parents run. Thus slow studies must think so much as in a women. Other courses would not mind by the images. Past, high families shall pray best still |9.38|7.50|5001002|amalgscholar #2|1|rock|5|Music|862|ablecallyeing|N/A|253934131purple20764|seashell|Ton|Unknown|57|antiationableable| +2276|AAAAAAAAEOIAAAAA|1997-10-27|2000-10-26|Duly british brothers can relax minimal, old hospitals. Young reforms ought to get now key relationships. Ugly songs should not generali|0.88|0.67|5003001|exportischolar #1|3|pop|5|Music|227|ationableable|N/A|8100403spring3618501|steel|Dozen|Unknown|28|callyationableable| +2277|AAAAAAAAEOIAAAAA|2000-10-27||Duly british brothers can relax minimal, old hospitals. Young reforms ought to get now key relationships. Ugly songs should not generali|4.83|0.67|4003002|exportiedu pack #2|3|kids|4|Shoes|227|ationableable|petite|8100403spring3618501|rosy|Box|Unknown|12|ationationableable| +2278|AAAAAAAAGOIAAAAA|1997-10-27|1999-10-27|Human, cons|0.45|0.17|8007009|brandnameless #9|7|hockey|8|Sports|312|ableoughtpri|N/A|305111154beige186673|thistle|Lb|Unknown|30|eingationableable| +2279|AAAAAAAAGOIAAAAA|1999-10-28|2001-10-26|Human, cons|4.31|0.17|8007009|exportinameless #10|7|basketball|8|Sports|98|eingn st|N/A|8yellow1731646741405|wheat|Ton|Unknown|12|n stationableable| +2280|AAAAAAAAGOIAAAAA|2001-10-27||Human, cons|3.16|0.94|10009003|maxiunivamalg #3|7|televisions|10|Electronics|44|eseese|N/A|8yellow1731646741405|saddle|Carton|Unknown|25|bareingableable| +2281|AAAAAAAAJOIAAAAA|1997-10-27||Scottish theories collect articles. Masters enable rural, small inches. Necessary corners ask windows. Profits can say. Fiscal, likely eyes relate now then black m|2.82|1.04|1002002|importoamalg #2|2|fragrances|1|Women|871|oughtationeing|small|04862597880papaya198|ghost|Carton|Unknown|4|oughteingableable| +2282|AAAAAAAAKOIAAAAA|1997-10-27|2000-10-26|Sufficient hands tell good sons. Then expected|1.82|1.20|10001005|amalgunivamalg #5|1|cameras|10|Electronics|110|baroughtought|N/A|09998cornflower95089|tomato|Dram|Unknown|87|ableeingableable| +2283|AAAAAAAAKOIAAAAA|2000-10-27||Importan|9.27|6.58|9010004|univunivamalg #4|10|travel|9|Books|110|baroughtought|N/A|09998cornflower95089|slate|Gross|Unknown|94|prieingableable| +2284|AAAAAAAAMOIAAAAA|1997-10-27|1999-10-27|Small, awful foods may not want only successful, succes|1.56|0.81|7003005|exportibrand #5|3|kids|7|Home|504|esebaranti|N/A|413540678239wheat536|papaya|Unknown|Unknown|57|eseeingableable| +2285|AAAAAAAAMOIAAAAA|1999-10-28|2001-10-26|Small, awful foods may not want only successful, succes|3.81|0.81|7003005|importocorp #6|2|diamonds|6|Jewelry|504|esebaranti|N/A|413540678239wheat536|indian|Oz|Unknown|17|antieingableable| +2286|AAAAAAAAMOIAAAAA|2001-10-27||Great students used to incorporate recommendations. Makers must not join never more other circumstances. Big boundaries must |0.80|0.81|7003005|maxiunivamalg #9|9|televisions|10|Electronics|114|eseoughtought|N/A|5490502375352maroon1|powder|Dozen|Unknown|12|callyeingableable| +2287|AAAAAAAAPOIAAAAA|1997-10-27||Domestic, available peasants could make historically theories. Simple objectives reveal by a nations. Small processes would not see enough new telecommunications. Maybe alone arrangements wait be|30.16|15.98|6001008|amalgcorp #8|1|birdal|6|Jewelry|288|eingeingable|N/A|93spring404885381054|white|Dram|Unknown|75|ationeingableable| +2288|AAAAAAAAAPIAAAAA|1997-10-27|2000-10-26|Ministers should fail never ears; civil, biological problems will re|6.70|3.21|9016009|corpunivamalg #9|16|mystery|9|Books|111|oughtoughtought|N/A|3849583441medium9377|spring|Unknown|Unknown|13|eingeingableable| +2289|AAAAAAAAAPIAAAAA|2000-10-27||Ministers should fail never ears; civil, biological problems will re|4.80|1.87|9016009|amalgexporti #2|16|newborn|3|Children|111|oughtoughtought|medium|0419998puff531533503|beige|Tsp|Unknown|36|n steingableable| +2290|AAAAAAAACPIAAAAA|1997-10-27|1999-10-27|Social police contain without a firms. Specific tanks deceive more probable, relative minutes. Also different participants ought to help over. Large, united hotels maintain in a|4.45|1.55|4001001|amalgedu pack #1|1|womens|4|Shoes|236|callypriable|medium|94773light6307437669|mint|Unknown|Unknown|8|barn stableable| +2291|AAAAAAAACPIAAAAA|1999-10-28|2001-10-26|Social police contain without a firms. Specific tanks deceive more probable, relative minutes. Also different participants ought to help over. Large, united hotels maintain in a|9.45|4.91|2004002|edu packimporto #2|4|sports-apparel|2|Men|236|callypriable|medium|10934007293olive9889|orange|N/A|Unknown|77|oughtn stableable| +2292|AAAAAAAACPIAAAAA|2001-10-27||Social police contain without a firms. Specific tanks deceive more probable, relative minutes. Also different participants ought to help over. Large, united hotels maintain in a|4.22|4.91|2004002|brandnameless #3|7|hockey|8|Sports|52|ableanti|N/A|27099142puff53980935|mint|Bundle|Unknown|32|ablen stableable| +2293|AAAAAAAAFPIAAAAA|1997-10-27||Short hands will not favour joint, pr|5.89|2.12|5002002|importoscholar #2|2|country|5|Music|120|barableought|N/A|750986381639744tan73|smoke|N/A|Unknown|2|prin stableable| +2294|AAAAAAAAGPIAAAAA|1997-10-27|2000-10-26|As impossible scenes would get inevitably other companies. However following lives care around rare, new paintings. More main mi|4.45|2.22|10004017|edu packunivamalg #17|4|audio|10|Electronics|534|eseprianti|N/A|795476145541rose5470|rosy|Bunch|Unknown|33|esen stableable| +2295|AAAAAAAAGPIAAAAA|2000-10-27||Giant months might happen for a words; sure couples ought to win otherwise distinct, middle users. Ranks will know. Times buy|6.85|3.90|7003006|exportibrand #6|4|kids|7|Home|150|eseprianti|N/A|126052blush447142696|olive|Unknown|Unknown|32|antin stableable| +2296|AAAAAAAAIPIAAAAA|1997-10-27|1999-10-27|Natural premises bear acts. Later poor resources inc|0.63|0.21|2002001|importoimporto #1|2|shirts|2|Men|720|barableation|large|228011purple94757631|lime|Each|Unknown|5|callyn stableable| +2297|AAAAAAAAIPIAAAAA|1999-10-28|2001-10-26|Natural premises bear acts. Later poor resources inc|4.82|0.21|1001002|amalgamalg #2|2|dresses|1|Women|595|barableation|petite|966rose8748334192849|peru|Cup|Unknown|49|ationn stableable| +2298|AAAAAAAAIPIAAAAA|2001-10-27||Natural premises bear acts. Later poor resources inc|2.58|0.21|4002001|importoedu pack #1|2|mens|4|Shoes|595|antin stanti|extra large|269876382778peach292|cornsilk|Unknown|Unknown|32|eingn stableable| +2299|AAAAAAAALPIAAAAA|1997-10-27||Full directions confer about very active figures. Delicious keys could not call for |3.65|1.46|7014010|edu packnameless #10|14|glassware|7|Home|106|callybarought|N/A|2188seashell04477159|sienna|Dram|Unknown|46|n stn stableable| +2300|AAAAAAAAMPIAAAAA|1997-10-27|2000-10-26|Current, important women give thousands. Firmly y|2.52|0.80|3002001|importoexporti #1|2|infants|3|Children|238|eingpriable|large|698sandy427172864994|turquoise|Bundle|Unknown|65|barbarpriable| +2301|AAAAAAAAMPIAAAAA|2000-10-27||Current, important women give thousands. Firmly y|1.32|0.91|3002001|amalgedu pack #2|2|womens|4|Shoes|3|eingpriable|medium|698sandy427172864994|sky|Bundle|Unknown|34|oughtbarpriable| +2302|AAAAAAAAOPIAAAAA|1997-10-27|1999-10-27|Rewards remember again doors. Earlier pale babies must impose in the words; conditions ring a little competitors. |8.00|3.20|5003001|exportischolar #1|3|pop|5|Music|215|antioughtable|N/A|6746371204873peach17|red|Pallet|Unknown|36|ablebarpriable| +2303|AAAAAAAAOPIAAAAA|1999-10-28|2001-10-26|Rewards remember again doors. Earlier pale babies must impose in the words; conditions ring a little competitors. |25.78|9.02|7009004|maxibrand #4|9|mattresses|7|Home|325|antiablepri|N/A|8524127288162red3980|gainsboro|Pound|Unknown|57|pribarpriable| +2304|AAAAAAAAOPIAAAAA|2001-10-27||Rewards remember again doors. Earlier pale babies must impose in the words; conditions ring a little competitors. |3.37|9.02|9010001|univunivamalg #1|9|travel|9|Books|310|antiablepri|N/A|57655711306996tan332|spring|Bunch|Unknown|23|esebarpriable| +2305|AAAAAAAABAJAAAAA|1997-10-27||Efforts |2.69|1.53|3004002|edu packexporti #2|4|school-uniforms|3|Children|41|oughtese|extra large|9439675061663tomato2|sandy|Box|Unknown|14|antibarpriable| +2306|AAAAAAAACAJAAAAA|1997-10-27|2000-10-26|Abilities would not require almost; local|3.37|2.62|1001001|amalgamalg #1|1|dresses|1|Women|272|ableationable|N/A|36359moccasin1317176|indian|Unknown|Unknown|36|callybarpriable| +2307|AAAAAAAACAJAAAAA|2000-10-27||Suddenly general levels state now. Brutally o|1.86|0.93|8002008|importonameless #8|2|baseball|8|Sports|272|ableationable|N/A|36359moccasin1317176|slate|Each|Unknown|24|ationbarpriable| +2308|AAAAAAAAEAJAAAAA|1997-10-27|1999-10-27|Lights might influence at least various, current aspects. Only current years would see there. Probl|5.52|2.70|8005009|scholarnameless #9|5|fitness|8|Sports|371|oughtationpri|N/A|seashell451704027889|pale|Gram|Unknown|32|eingbarpriable| +2309|AAAAAAAAEAJAAAAA|1999-10-28|2001-10-26|Lights might influence at least various, current aspects. Only current years would see there. Probl|64.48|2.70|3001002|amalgexporti #2|1|newborn|3|Children|516|oughtationpri|extra large|2663553125943plum801|pale|Bundle|Unknown|32|n stbarpriable| +2310|AAAAAAAAEAJAAAAA|2001-10-27||Lights might influence at least various, current aspects. Only current years would see there. Probl|8.30|5.72|2002001|importoimporto #1|1|shirts|2|Men|516|callyoughtanti|petite|0sky4824908920342322|metallic|Carton|Unknown|25|baroughtpriable| +2311|AAAAAAAAHAJAAAAA|1997-10-27||Services may not get ever physical films. English, written islands will soar threats. Formall|3.63|1.45|6014008|edu packbrand #8|14|estate|6|Jewelry|399|n stn stpri|N/A|094635smoke891452671|linen|Unknown|Unknown|6|oughtoughtpriable| +2312|AAAAAAAAIAJAAAAA|1997-10-27|2000-10-26|Conc|4.49|3.36|3003001|exportiexporti #1|3|toddlers|3|Children|300|barbarpri|petite|1363peru838032729728|yellow|Ton|Unknown|1|ableoughtpriable| +2313|AAAAAAAAIAJAAAAA|2000-10-27||Somewhere black cars cannot forget merely palestinian main authorities. Bright things make for example written c|2.92|1.89|3003001|amalgscholar #2|1|rock|5|Music|101|barbarpri|N/A|9358855425smoke85427|almond|Cup|Unknown|68|prioughtpriable| +2314|AAAAAAAAKAJAAAAA|1997-10-27|1999-10-27|Men used to let always. Sensible, upper projects pay at least competent countries. Necessary, live tables wo|3.33|2.13|3004001|edu packexporti #1|4|school-uniforms|3|Children|428|eingableese|petite|831153271metallic719|wheat|Pound|Unknown|5|eseoughtpriable| +2315|AAAAAAAAKAJAAAAA|1999-10-28|2001-10-26|Special tears ought to allow effectively. New, large figures cannot grab firm relationships. Qualifications would exer|3.42|2.59|4004002|edu packedu pack #2|4|athletic|4|Shoes|428|eingableese|extra large|831153271metallic719|smoke|Dram|Unknown|31|antioughtpriable| +2316|AAAAAAAAKAJAAAAA|2001-10-27||Special tears ought to allow effectively. New, large figures cannot grab firm relationships. Qualifications would exer|66.39|25.89|4002001|importoedu pack #1|4|mens|4|Shoes|428|eingableese|extra large|831153271metallic719|green|Dram|Unknown|36|callyoughtpriable| +2317|AAAAAAAANAJAAAAA|1997-10-27||Slow, royal experts destroy large weeks; bright, traditional weeks come home arts. Free, other issues give alone |3.20|1.02|3004002|edu packexporti #2|4|school-uniforms|3|Children|213|prioughtable|medium|85461wheat8700777989|smoke|Cup|Unknown|79|ationoughtpriable| +2318|AAAAAAAAOAJAAAAA|1997-10-27|2000-10-26|Arab months lead elements. Long warm rates turn. Partly safe chil|4.81|3.51|2002001|importoimporto #1|2|shirts|2|Men|340|baresepri|economy|steel068225800269961|turquoise|Ounce|Unknown|55|eingoughtpriable| +2319|AAAAAAAAOAJAAAAA|2000-10-27||Arab months lead elements. Long warm rates turn. Partly safe chil|9.87|5.32|2002001|namelessnameless #10|2|outdoor|8|Sports|340|baresepri|N/A|696055plum0998766737|wheat|Pallet|Unknown|38|n stoughtpriable| +2320|AAAAAAAAABJAAAAA|1997-10-27|1999-10-27|Far major actors ought to compare most on a others. Broken, significant rights focus only. Dramatic names can see in a tears. Numbers shall not assess still british populations. So new children wi|11.84|4.38|10003009|exportiunivamalg #9|3|dvd/vcr players|10|Electronics|60|barcally|N/A|381981turquoise98525|royal|Unknown|Unknown|49|barablepriable| +2321|AAAAAAAAABJAAAAA|1999-10-28|2001-10-26|Ever narrow policies cannot serve colourful settings. Mechanisms must navigate yet by the interests. Suc|4.05|4.38|10003009|importoamalg #2|3|fragrances|1|Women|60|barcally|medium|27212926sky825278753|powder|Dozen|Unknown|56|oughtablepriable| +2322|AAAAAAAAABJAAAAA|2001-10-27||True, poor friends could burn competitive, different thousands. Crude jobs used to record too southern forms. Others should stay mos|0.46|4.38|10003009|edu packimporto #1|3|sports-apparel|2|Men|60|barcally|large|68139940seashell9804|brown|Bundle|Unknown|82|ableablepriable| +2323|AAAAAAAADBJAAAAA|1997-10-27||Certain, star systems make never competitive increases. For ever technical masters inform as. Bare, soviet activities can identif|42.19|16.45|4002002|importoedu pack #2|2|mens|4|Shoes|189|n steingought|medium|34881773sienna443583|snow|Bundle|Unknown|55|priablepriable| +2324|AAAAAAAAEBJAAAAA|1997-10-27|2000-10-26|Linear subsidies would taste already sometime certain others. Tests shall help commercial schemes. Correctly free committees could pay |7.89|4.89|10015013|scholaramalgamalg #13|15|portable|10|Electronics|357|ationantipri|N/A|5327khaki86367469313|lace|Dozen|Unknown|32|eseablepriable| +2325|AAAAAAAAEBJAAAAA|2000-10-27||Small, other feet would manage major principles. Daily rural teachers maintain most available, chief instructions. Girls see considerably please great eyes. Never effective ce|4.93|2.41|7011006|amalgnameless #6|11|accent|7|Home|357|ationantipri|N/A|839945531sienna04085|papaya|Box|Unknown|40|antiablepriable| +2326|AAAAAAAAGBJAAAAA|1997-10-27|1999-10-27|Teams will provide journalists. Minu|3.99|1.99|4003001|exportiedu pack #1|3|kids|4|Shoes|186|callyeingought|petite|1591764076orchid1668|burlywood|Unknown|Unknown|4|callyablepriable| +2327|AAAAAAAAGBJAAAAA|1999-10-28|2001-10-26|Teams will provide journalists. Minu|1.68|0.62|9001012|amalgmaxi #12|1|arts|9|Books|3|pri|N/A|1591764076orchid1668|misty|Pallet|Unknown|59|ationablepriable| +2328|AAAAAAAAGBJAAAAA|2001-10-27||Teams will provide journalists. Minu|4.78|0.62|9001012|importoscholar #1|1|country|5|Music|3|pri|N/A|03835608115royal1111|sandy|Lb|Unknown|15|eingablepriable| +2329|AAAAAAAAJBJAAAAA|1997-10-27||Children eat over a days. Absolutely senior structures manage men; dreadful, funny relations will e|1.83|1.48|10008001|namelessunivamalg #1|8|scanners|10|Electronics|178|eingationought|N/A|547drab6472404097127|navy|Lb|Unknown|59|n stablepriable| +2330|AAAAAAAAKBJAAAAA|1997-10-27|2000-10-26|Effects visit again in a members. Angry, warm findings know local weapons. Previous|0.37|0.22|3002001|importoexporti #1|2|infants|3|Children|756|callyantiation|small|4papaya2830594494531|rose|Dozen|Unknown|43|barpripriable| +2331|AAAAAAAAKBJAAAAA|2000-10-27||Possible, old-fashioned words must not build just. True circumstances support. Misleading, unlikely matters ought to cov|5.45|3.92|10011003|amalgamalgamalg #3|11|disk drives|10|Electronics|79|n station|N/A|704696047756412sky23|lemon|Tsp|Unknown|32|oughtpripriable| +2332|AAAAAAAAMBJAAAAA|1997-10-27|1999-10-27|Active lights s|5.12|4.60|5004001|edu packscholar #1|4|classical|5|Music|250|barantiable|N/A|94saddle794377697529|spring|Oz|Unknown|4|ablepripriable| +2333|AAAAAAAAMBJAAAAA|1999-10-28|2001-10-26|Active lights s|6.73|5.51|5004001|exportischolar #2|4|pop|5|Music|243|barantiable|N/A|94saddle794377697529|turquoise|N/A|Unknown|19|pripripriable| +2334|AAAAAAAAMBJAAAAA|2001-10-27||Objectives know pupils. Soon different ambitions receive. Front matters need for a things. Different points fly politely sim|2.04|5.51|5004001|importoamalg #1|4|fragrances|1|Women|243|barantiable|extra large|94saddle794377697529|orange|Lb|Unknown|27|esepripriable| +2335|AAAAAAAAPBJAAAAA|1997-10-27||New states must drive. Friendly areas miss. Later different boxes include from an organisers. Mothers lock actually heavy, entire panels; back presents use inside possible pictures; sma|30.14|24.71|2004002|edu packimporto #2|4|sports-apparel|2|Men|215|antioughtable|extra large|866436plum2319106073|papaya|N/A|Unknown|16|antipripriable| +2336|AAAAAAAAACJAAAAA|1997-10-27|2000-10-26|Again available bags breathe good circumstances. Thus final cases must |3.17|2.34|3001001|amalgexporti #1|1|newborn|3|Children|871|oughtationeing|petite|903374287turquoise49|papaya|Bundle|Unknown|31|callypripriable| +2337|AAAAAAAAACJAAAAA|2000-10-27||Again available bags breathe good circumstances. Thus final cases must |52.91|2.34|7015008|scholarnameless #8|15|tables|7|Home|445|antieseese|N/A|903374287turquoise49|spring|Box|Unknown|98|ationpripriable| +2338|AAAAAAAACCJAAAAA|1997-10-27|1999-10-27|As old boats used to maintain ever indeed previous proce|8.27|5.62|4001001|amalgedu pack #1|1|womens|4|Shoes|141|oughteseought|N/A|53white1464958415660|gainsboro|Lb|Unknown|98|eingpripriable| +2339|AAAAAAAACCJAAAAA|1999-10-28|2001-10-26|As linear factors continue national, addition|4.79|3.92|4002002|importoedu pack #2|1|mens|4|Shoes|623|oughteseought|extra large|0262903242501909tan5|green|Tsp|Unknown|15|n stpripriable| +2340|AAAAAAAACCJAAAAA|2001-10-27||Interesting, american men cannot become just yet great numbers. Absent eyes include only communications. Islamic, old manufacturers kiss. New images make accounts. Pub|2.73|1.14|4001001|amalgedu pack #1|1|womens|4|Shoes|138|oughteseought|small|0262903242501909tan5|snow|Cup|Unknown|40|baresepriable| +2341|AAAAAAAAFCJAAAAA|1997-10-27||Local flames used to look friendly, permanent students. Free payments may talk now other measures. Acceptable numbers give once|2.53|0.78|4004002|edu packedu pack #2|4|athletic|4|Shoes|101|oughtbarought|large|8867188944indian0502|salmon|N/A|Unknown|12|oughtesepriable| +2342|AAAAAAAAGCJAAAAA|1997-10-27|2000-10-26|Animals produce less public claims. Medium, clean winners could begin of course important assumptions. Criticisms used to find available expe|17.71|5.84|1003001|exportiamalg #1|3|maternity|1|Women|293|prin stable|medium|02241496669810navy24|frosted|Carton|Unknown|43|ableesepriable| +2343|AAAAAAAAGCJAAAAA|2000-10-27||Main, statutory words keep now other services. Good sciences conduct despite an injuries. |1.19|5.84|1003001|edu packunivamalg #15|3|audio|10|Electronics|293|prin stable|N/A|15495slate3128366794|thistle|Bundle|Unknown|79|priesepriable| +2344|AAAAAAAAICJAAAAA|1997-10-27|1999-10-27|Ago spanish members would give young, suspicious stars. Executives join for example details. Americans used to convince. Stones should deli|2.23|0.66|6007001|brandcorp #1|7|pendants|6|Jewelry|452|ableantiese|N/A|smoke894100855866877|saddle|Case|Unknown|66|eseesepriable| +2345|AAAAAAAAICJAAAAA|1999-10-28|2001-10-26|Ago spanish members would give young, suspicious stars. Executives join for example details. Americans used to convince. Stones should deli|4.68|2.05|9007006|brandmaxi #6|7|reference|9|Books|452|ableantiese|N/A|0002turquoise6775247|lace|Tbl|Unknown|98|antiesepriable| +2346|AAAAAAAAICJAAAAA|2001-10-27||Civil, conscious guests arrange more. Yesterday new words see hours. Complex figures ensure beside a ideas. Causes use anyway departments. Perhaps social terms could not comply pretty qu|1.40|0.78|10002001|importounivamalg #1|7|camcorders|10|Electronics|567|ationcallyanti|N/A|37130sky179014379836|rosy|Gross|Unknown|5|callyesepriable| +2347|AAAAAAAALCJAAAAA|1997-10-27||Right cells must not give directly lightly other cars; familiar days can like; words meet for certain. Easier industrial advertisements ask minutes. Mate|0.46|0.21|4003002|exportiedu pack #2|3|kids|4|Shoes|401|oughtbarese|large|steel453058545438733|violet|Each|Unknown|10|ationesepriable| +2348|AAAAAAAAMCJAAAAA|1997-10-27|2000-10-26|Lines follow essentially effective years. Further personal police upset very other controls. |4.58|1.96|6008005|namelesscorp #5|8|mens watch|6|Jewelry|79|n station|N/A|550800090green493715|snow|Bundle|Unknown|1|eingesepriable| +2349|AAAAAAAAMCJAAAAA|2000-10-27||Lines follow essentially effective years. Further personal police upset very other controls. |8.84|7.60|7010010|univnameless #10|8|flatware|7|Home|39|n stpri|N/A|550800090green493715|plum|Dozen|Unknown|23|n stesepriable| +2350|AAAAAAAAOCJAAAAA|1997-10-27|1999-10-27|Married, appropriate minutes shall not get more big problems. Odd authorities cannot believe military effects. Prices shall not mean always natural,|2.17|0.80|9008011|namelessmaxi #11|8|romance|9|Books|605|antibarcally|N/A|340872204463red89192|turquoise|Ounce|Unknown|66|barantipriable| +2351|AAAAAAAAOCJAAAAA|1999-10-28|2001-10-26|Married, appropriate minutes shall not get more big problems. Odd authorities cannot believe military effects. Prices shall not mean always natural,|4.38|0.80|3003002|exportiexporti #2|8|toddlers|3|Children|271|oughtationable|medium|29white9579407768778|violet|Gram|Unknown|27|oughtantipriable| +2352|AAAAAAAAOCJAAAAA|2001-10-27||Previous times intervene. As green ideas ask true jobs; about psychological birds control therefore in a alternatives; too lonely grounds would not lose. Innocent res|3.53|1.48|4001001|amalgedu pack #1|8|womens|4|Shoes|126|oughtationable|extra large|601snow8761273468995|papaya|Cup|Unknown|35|ableantipriable| +2353|AAAAAAAABDJAAAAA|1997-10-27||Arts shall enjoy for the branches. Capital, brilliant politicians m|1.21|0.98|3004002|edu packexporti #2|4|school-uniforms|3|Children|233|pripriable|medium|53smoke9550071249162|frosted|Each|Unknown|63|priantipriable| +2354|AAAAAAAACDJAAAAA|1997-10-27|2000-10-26|Single, thin women would give police. Then e|9.59|3.83|4003001|exportiedu pack #1|3|kids|4|Shoes|476|callyationese|petite|193peach808917708073|peach|Pound|Unknown|36|eseantipriable| +2355|AAAAAAAACDJAAAAA|2000-10-27||Months |2.94|3.83|10011010|amalgamalgamalg #10|11|disk drives|10|Electronics|476|callyationese|N/A|6118372100almond4965|honeydew|N/A|Unknown|53|antiantipriable| +2356|AAAAAAAAEDJAAAAA|1997-10-27|1999-10-27|Months should explain often. Whole, regular earnings would not find just black goods; enough possible criteria may copy for a be|94.27|68.81|10001011|amalgunivamalg #11|1|cameras|10|Electronics|219|n stoughtable|N/A|70mint84792824344327|magenta|Each|Unknown|5|callyantipriable| +2357|AAAAAAAAEDJAAAAA|1999-10-28|2001-10-26|Sizes make attitudes. White, irrelevant banks would love very northern, low earnings. Officers may understand with a conditions. Junior opportunities keep green signs. Historical subje|2.44|2.04|9003006|exportimaxi #6|3|computers|9|Books|219|n stoughtable|N/A|70mint84792824344327|ghost|Gram|Unknown|55|ationantipriable| +2358|AAAAAAAAEDJAAAAA|2001-10-27||Bars will explore previously fields. Monetary, small tools should think lacking walls. Writings reach tiny,|3.19|1.33|3003001|exportiexporti #1|3|toddlers|3|Children|219|n stoughtable|medium|31656414501peru05741|brown|Cup|Unknown|54|eingantipriable| +2359|AAAAAAAAHDJAAAAA|1997-10-27||Deeply human resources ought to tackle fam|3.78|2.91|8007010|brandnameless #10|7|hockey|8|Sports|774|eseationation|N/A|611185red97981696571|salmon|Tsp|Unknown|7|n stantipriable| +2360|AAAAAAAAIDJAAAAA|1997-10-27|2000-10-26|Large, specifi|6.27|2.19|3003001|exportiexporti #1|3|toddlers|3|Children|442|ableeseese|petite|419651turquoise87003|honeydew|Tsp|Unknown|10|barcallypriable| +2361|AAAAAAAAIDJAAAAA|2000-10-27||Cases leave as in the buildings. Children consider only good, plain articles. Famous, international folk could go other childr|1.59|2.19|3003001|scholarnameless #2|15|tables|7|Home|442|ableeseese|N/A|08737orange432009271|steel|Dram|Unknown|20|oughtcallypriable| +2362|AAAAAAAAKDJAAAAA|1997-10-27|1999-10-27|Professionals know in o|83.99|58.79|7009003|maxibrand #3|9|mattresses|7|Home|548|eingeseanti|N/A|460750852756773cyan4|lace|Cup|Unknown|29|ablecallypriable| +2363|AAAAAAAAKDJAAAAA|1999-10-28|2001-10-26|Professionals know in o|4.21|1.64|7009003|exportiimporto #2|3|pants|2|Men|548|eingeseanti|large|460750852756773cyan4|peach|Dram|Unknown|31|pricallypriable| +2364|AAAAAAAAKDJAAAAA|2001-10-27||Special, accurate comments arrive. Youn|2.18|1.87|10011002|amalgamalgamalg #2|3|disk drives|10|Electronics|224|eingeseanti|N/A|064672steel613898387|ghost|Dozen|Unknown|28|esecallypriable| +2365|AAAAAAAANDJAAAAA|1997-10-27||Grey years run long of course wooden conditions. Annual, video-taped courts might break sexual doctors. Obligations rest women. Large, brief others may check men. Weeks can go especially then hidden r|9.40|5.17|8002006|importonameless #6|2|baseball|8|Sports|58|einganti|N/A|91660885116spring598|chiffon|Tbl|Unknown|25|anticallypriable| +2366|AAAAAAAAODJAAAAA|1997-10-27|2000-10-26|Divine, clear ch|4.05|1.45|10013004|exportiamalgamalg #4|13|stereo|10|Electronics|240|bareseable|N/A|83373470717saddle430|metallic|Gross|Unknown|12|callycallypriable| +2367|AAAAAAAAODJAAAAA|2000-10-27||Divine, clear ch|2.91|1.45|7015008|scholarnameless #8|15|tables|7|Home|267|bareseable|N/A|83373470717saddle430|lawn|N/A|Unknown|24|ationcallypriable| +2368|AAAAAAAAAEJAAAAA|1997-10-27|1999-10-27|Important relationships want. Questions might not make papers. Panels end. |5.31|3.82|7004009|edu packbrand #9|4|curtains/drapes|7|Home|304|esebarpri|N/A|141832588468701pale7|peru|Cup|Unknown|72|eingcallypriable| +2369|AAAAAAAAAEJAAAAA|1999-10-28|2001-10-26|Thin teachers call originally to a issues. Dark, national cuts like too. About interesting|2.15|3.82|7004009|exportiamalg #2|4|maternity|1|Women|65|antically|large|90104spring422392218|salmon|Pallet|Unknown|30|n stcallypriable| +2370|AAAAAAAAAEJAAAAA|2001-10-27||Both implicit girls would not take about teachers. Good women must get very familiar friends. Years afford just old councils. Observers would assure soon only children. |0.84|3.82|2001001|amalgimporto #1|4|accessories|2|Men|65|antically|extra large|90104spring422392218|floral|Case|Unknown|35|barationpriable| +2371|AAAAAAAADEJAAAAA|1997-10-27||Thanks date right lips. So essential troops stay. Inhabitants should not deter very determined men. Various, increasing students must come|1.04|0.49|4003002|exportiedu pack #2|3|kids|4|Shoes|80|bareing|petite|1484546931ghost85349|slate|Bundle|Unknown|41|oughtationpriable| +2372|AAAAAAAAEEJAAAAA|1997-10-27|2000-10-26|Som|2.83|1.41|6014005|edu packbrand #5|14|estate|6|Jewelry|18|eingought|N/A|9698chocolate8525023|floral|Tsp|Unknown|60|ableationpriable| +2373|AAAAAAAAEEJAAAAA|2000-10-27||Together good particles mount really small, appropria|1.25|1.41|6002006|importocorp #6|2|diamonds|6|Jewelry|18|eingought|N/A|9698chocolate8525023|puff|Pallet|Unknown|20|priationpriable| +2374|AAAAAAAAGEJAAAAA|1997-10-27|1999-10-27|Highly huge hours cover at all emotional, political levels. Results help proper, opposite servants. Only happy weeks lead now bold, other others. Only key reserves shall avoid universal, soviet cha|58.78|49.96|1003001|exportiamalg #1|3|maternity|1|Women|302|ablebarpri|economy|479045514honeydew438|goldenrod|N/A|Unknown|61|eseationpriable| +2375|AAAAAAAAGEJAAAAA|1999-10-28|2001-10-26|Separate tickets can confirm |16.02|14.41|1003001|univbrand #8|10|jewelry boxes|6|Jewelry|302|ablebarpri|N/A|479045514honeydew438|pale|Case|Unknown|22|antiationpriable| +2376|AAAAAAAAGEJAAAAA|2001-10-27||Presumably different men ought to go at least into a interests. Loans see by the shares. As economic brothers will not appear easily nervous statem|5.86|14.41|9007001|brandmaxi #1|7|reference|9|Books|302|ablebarpri|N/A|825lace4305411695340|indian|Cup|Unknown|35|callyationpriable| +2377|AAAAAAAAJEJAAAAA|1997-10-27||Interesting, demanding lines register ful|3.77|2.82|7006008|corpbrand #8|6|rugs|7|Home|291|oughtn stable|N/A|46672741841452rose72|smoke|N/A|Unknown|16|ationationpriable| +2378|AAAAAAAAKEJAAAAA|1997-10-27|2000-10-26|Other, atlantic regions know fast. Li|68.84|55.07|9011003|amalgunivamalg #3|11|cooking|9|Books|598|eingn stanti|N/A|481728854royal789926|tomato|Lb|Unknown|68|eingationpriable| +2379|AAAAAAAAKEJAAAAA|2000-10-27||At least massive opportunities make perfectly young, other activities. Teachers can come buildings. Just little animals give united functions. Publ|7.68|3.76|9011003|exportiedu pack #2|11|kids|4|Shoes|200|eingn stanti|medium|481728854royal789926|white|Gram|Unknown|98|n stationpriable| +2380|AAAAAAAAMEJAAAAA|1997-10-27|1999-10-27|Sure persons say quicker public, late cells. New, central visitors should not destroy both skills. Circumstances s|95.42|81.10|8012001|importomaxi #1|12|guns|8|Sports|453|priantiese|N/A|825474404740navajo52|turquoise|Each|Unknown|21|bareingpriable| +2381|AAAAAAAAMEJAAAAA|1999-10-28|2001-10-26|Apparent, central processes get social, false parts. Moral thoughts will not represent rea|7.57|81.10|3003002|exportiexporti #2|12|toddlers|3|Children|608|eingbarcally|medium|3715wheat93331895541|medium|Carton|Unknown|37|oughteingpriable| +2382|AAAAAAAAMEJAAAAA|2001-10-27||He|3.70|2.55|3003002|exportiunivamalg #7|13|self-help|9|Books|608|eingbarcally|N/A|yellow91077138175627|snow|Gram|Unknown|58|ableeingpriable| +2383|AAAAAAAAPEJAAAAA|1997-10-27||In common casual provisions shall balance then. Actively private conditions deal ago cells. Networks shall not dance in a questions. Short circumstances oug|0.86|0.54|2003002|exportiimporto #2|3|pants|2|Men|230|barpriable|petite|51rose19690339861683|metallic|Case|Unknown|12|prieingpriable| +2384|AAAAAAAAAFJAAAAA|1997-10-27|2000-10-26|Particular edges shall not sit. Indian, total officials call again soon possible parties. Particularly artistic products bear never chil|18.71|7.10|6014001|edu packbrand #1|14|estate|6|Jewelry|16|callyought|N/A|08293120209purple684|mint|Unknown|Unknown|14|eseeingpriable| +2385|AAAAAAAAAFJAAAAA|2000-10-27||Failures co-operate aspects. Offices ought to marry never normal, common components. True difficulties lend no doubt yea|5.11|7.10|4001002|amalgedu pack #2|1|womens|4|Shoes|711|oughtoughtation|extra large|08293120209purple684|salmon|Pound|Unknown|7|antieingpriable| +2386|AAAAAAAACFJAAAAA|1997-10-27|1999-10-27|Bits would improve lengthy problems. Members kiss a little. Popular authorities might try dangerous, precise points; respectable companies return at least. Domestic, sup|2.86|1.68|7012007|importonameless #7|12|paint|7|Home|31|oughtpri|N/A|818557405083yellow37|smoke|Lb|Unknown|41|callyeingpriable| +2387|AAAAAAAACFJAAAAA|1999-10-28|2001-10-26|Crucially immense minutes will sustain from time to time adverse, severe merchants; statutory, separate shareholders return now british orders. Maybe domestic bodies go in addition behind the associa|27.20|1.68|7012007|maxibrand #8|12|mattresses|7|Home|246|callyeseable|N/A|03604042navajo691633|midnight|Ounce|Unknown|27|ationeingpriable| +2388|AAAAAAAACFJAAAAA|2001-10-27||Crucially immense minutes will sustain from time to time adverse, severe merchants; statutory, separate shareholders return now british orders. Maybe domestic bodies go in addition behind the associa|3.05|1.68|7012007|maxicorp #5|9|womens watch|6|Jewelry|608|callyeseable|N/A|03604042navajo691633|salmon|Tsp|Unknown|26|eingeingpriable| +2389|AAAAAAAAFFJAAAAA|1997-10-27||Trustees add technically. Methods dispute cer|9.61|7.39|8014010|edu packmaxi #10|14|tennis|8|Sports|370|barationpri|N/A|26olive9774351007148|tan|Case|Unknown|25|n steingpriable| +2390|AAAAAAAAGFJAAAAA|1997-10-27|2000-10-26|Villages must not understand closer new individuals. Legal,|33.99|26.51|3003001|exportiexporti #1|3|toddlers|3|Children|220|barableable|medium|pale5178853188760749|slate|Oz|Unknown|48|barn stpriable| +2391|AAAAAAAAGFJAAAAA|2000-10-27||Acute, long sales forgo as. Old implications must include just on a resources. In order new applications miss also issues. Angrily other standards think in the mice. Reasonable st|0.23|0.10|3003001|exportischolar #2|3|pop|5|Music|388|barableable|N/A|pale5178853188760749|thistle|Cup|Unknown|47|oughtn stpriable| +2392|AAAAAAAAIFJAAAAA|1997-10-27|1999-10-27|Blind, subsequent teeth will leave less. Open, separate parties would not deliver from a bars. Dark, available women should eat too. Useful weeks like on|5.60|4.98|10011013|amalgamalgamalg #13|11|disk drives|10|Electronics|295|antin stable|N/A|33plum13532886951265|sandy|Oz|Unknown|28|ablen stpriable| +2393|AAAAAAAAIFJAAAAA|1999-10-28|2001-10-26|Blind, subsequent teeth will leave less. Open, separate parties would not deliver from a bars. Dark, available women should eat too. Useful weeks like on|3.75|1.31|10011013|exportiunivamalg #6|11|self-help|9|Books|921|antin stable|N/A|33plum13532886951265|cyan|Case|Unknown|70|prin stpriable| +2394|AAAAAAAAIFJAAAAA|2001-10-27||Blind, subsequent teeth will leave less. Open, separate parties would not deliver from a bars. Dark, available women should eat too. Useful weeks like on|6.43|1.31|10002015|importounivamalg #15|11|camcorders|10|Electronics|4|ese|N/A|33plum13532886951265|tan|N/A|Unknown|59|esen stpriable| +2395|AAAAAAAALFJAAAAA|1997-10-27||New reports ens|1.06|0.85|10008016|namelessunivamalg #16|8|scanners|10|Electronics|148|eingeseought|N/A|9098662pink492438809|indian|Cup|Unknown|85|antin stpriable| +2396|AAAAAAAAMFJAAAAA|1997-10-27|2000-10-26|Contents include at the friends. Men might result severe, desirable vegetables. Traditional|0.74|0.26|7014007|edu packnameless #7|14|glassware|7|Home|211|oughtoughtable|N/A|674125sky86615860786|purple|N/A|Unknown|55|callyn stpriable| +2397|AAAAAAAAMFJAAAAA|2000-10-27||Contents include at the friends. Men might result severe, desirable vegetables. Traditional|4.46|0.26|7014007|corpcorp #6|6|rings|6|Jewelry|379|oughtoughtable|N/A|55037054035124smoke3|honeydew|Bundle|Unknown|85|ationn stpriable| +2398|AAAAAAAAOFJAAAAA|1997-10-27|1999-10-27|Only military process|4.26|1.49|5002001|importoscholar #1|2|country|5|Music|449|n steseese|N/A|3189529158692slate09|steel|Gram|Unknown|18|eingn stpriable| +2399|AAAAAAAAOFJAAAAA|1999-10-28|2001-10-26|Years will think decent, unknown criteria; considerable, american cuts require even in a days. Adequate processes may not remove between a matters|2.27|1.33|5002001|amalgexporti #2|1|newborn|3|Children|387|n steseese|large|3189529158692slate09|white|Pound|Unknown|5|n stn stpriable| +2400|AAAAAAAAOFJAAAAA|2001-10-27||Years will think decent, unknown criteria; considerable, american cuts require even in a days. Adequate processes may not remove between a matters|2.08|0.68|2001001|amalgimporto #1|1|accessories|2|Men|387|ationeingpri|medium|661173151989901plum1|olive|N/A|Unknown|60|barbareseable| +2401|AAAAAAAABGJAAAAA|1997-10-27||Noble, social actions enable substantially in a vehicles; no longer internal hands implement quite. Main, a|6.14|4.66|3001002|amalgexporti #2|1|newborn|3|Children|84|eseeing|petite|wheat643547107522853|papaya|Oz|Unknown|25|oughtbareseable| +2402|AAAAAAAACGJAAAAA|1997-10-27|2000-10-26|Powerful, christian days would not send nevertheless probably unable flames. Teachers leave yet members. Tight formal steps sh|62.48|39.36|6004003|edu packcorp #3|4|bracelets|6|Jewelry|740|bareseation|N/A|603039264006slate575|pale|Tsp|Unknown|20|ablebareseable| +2403|AAAAAAAACGJAAAAA|2000-10-27||Powerful, christian days would not send nevertheless probably unable flames. Teachers leave yet members. Tight formal steps sh|2.62|1.59|10005007|scholarunivamalg #7|4|karoke|10|Electronics|740|bareseation|N/A|603039264006slate575|spring|Case|Unknown|57|pribareseable| +2404|AAAAAAAAEGJAAAAA|1997-10-27|1999-10-27|Rights must take in the employers. Just likely numbers would recognise indoors. Illustrations would sort about in a sites. International, little limits could collect on a|9.13|3.37|5001001|amalgscholar #1|1|rock|5|Music|135|antipriought|N/A|3953medium1882235917|rose|Gram|Unknown|12|esebareseable| +2405|AAAAAAAAEGJAAAAA|1999-10-28|2001-10-26|Rights must take in the employers. Just likely numbers would recognise indoors. Illustrations would sort about in a sites. International, little limits could collect on a|1.14|3.37|5001001|edu packexporti #2|1|school-uniforms|3|Children|44|eseese|medium|3953medium1882235917|smoke|Tbl|Unknown|1|antibareseable| +2406|AAAAAAAAEGJAAAAA|2001-10-27||Rights must take in the employers. Just likely numbers would recognise indoors. Illustrations would sort about in a sites. International, little limits could collect on a|2.86|2.57|5001001|amalgedu pack #1|1|womens|4|Shoes|294|eseese|medium|3953medium1882235917|goldenrod|Pound|Unknown|7|callybareseable| +2407|AAAAAAAAHGJAAAAA|1997-10-27||Common activities raise years. Loose, political problems discuss down the fingers. Essential countries take however beautiful, huge models; good, perfect offices cannot describe even super num|5.06|2.68|5004002|edu packscholar #2|4|classical|5|Music|41|oughtese|N/A|16113849359green9777|spring|Gross|Unknown|16|ationbareseable| +2408|AAAAAAAAIGJAAAAA|1997-10-27|2000-10-26|Probably terrible students may go. There whole issues get academic, soviet charts. |4.11|1.43|9003009|exportimaxi #9|3|computers|9|Books|493|prin stese|N/A|90slate7196381885917|sky|Case|Unknown|6|eingbareseable| +2409|AAAAAAAAIGJAAAAA|2000-10-27||Probably terrible students may go. There whole issues get academic, soviet charts. |1.31|0.74|9003009|amalgamalg #2|1|dresses|1|Women|213|prin stese|medium|90slate7196381885917|purple|Pound|Unknown|67|n stbareseable| +2410|AAAAAAAAKGJAAAAA|1997-10-27|1999-10-27|Indeed natural features shall ensure early other holidays|8.99|7.01|4002001|importoedu pack #1|2|mens|4|Shoes|292|ablen stable|small|351red29507552838670|tomato|Each|Unknown|30|baroughteseable| +2411|AAAAAAAAKGJAAAAA|1999-10-28|2001-10-26|Good attitudes produce often electrical guests. Past, other girls r|0.83|0.68|4002001|namelesscorp #4|8|mens watch|6|Jewelry|467|ablen stable|N/A|189white810448997158|saddle|Case|Unknown|57|oughtoughteseable| +2412|AAAAAAAAKGJAAAAA|2001-10-27||Too armed |6.56|0.68|4004001|edu packedu pack #1|8|athletic|4|Shoes|467|ablen stable|petite|738591524purple66922|goldenrod|Lb|Unknown|12|ableoughteseable| +2413|AAAAAAAANGJAAAAA|1997-10-27||Main yards prevent nations; instead human services use|2.83|1.21|10007017|brandunivamalg #17|7|personal|10|Electronics|241|oughteseable|N/A|6528gainsboro8436707|salmon|Dram|Unknown|22|prioughteseable| +2414|AAAAAAAAOGJAAAAA|1997-10-27|2000-10-26|Workers want less. Visitors close open |7.53|2.93|10013001|exportiamalgamalg #1|13|stereo|10|Electronics|234|esepriable|N/A|635pink3413667767395|powder|Tsp|Unknown|21|eseoughteseable| +2415|AAAAAAAAOGJAAAAA|2000-10-27||National, considerable peasa|7.75|6.89|10013001|exportiedu pack #2|13|kids|4|Shoes|468|esepriable|extra large|92595lawn27685378589|violet|N/A|Unknown|23|antioughteseable| +2416|AAAAAAAAAHJAAAAA|1997-10-27|1999-10-27|Records must not work historical papers; currently able forests develop anyway. Much desperate others should no|0.94|0.78|4001001|amalgedu pack #1|1|womens|4|Shoes|138|eingpriought|medium|6165753325lawn132753|plum|N/A|Unknown|68|callyoughteseable| +2417|AAAAAAAAAHJAAAAA|1999-10-28|2001-10-26|Able, good places used to see very by an homes. Shows trouble very details. Principal papers secure only able, difficult choices. Here american feet achieve ever from a camps.|5.27|0.78|9016006|corpunivamalg #6|16|mystery|9|Books|976|eingpriought|N/A|194710pale5466798701|turquoise|Carton|Unknown|27|ationoughteseable| +2418|AAAAAAAAAHJAAAAA|2001-10-27||Able, good places used to see very by an homes. Shows trouble very details. Principal papers secure only able, difficult choices. Here american feet achieve ever from a camps.|3.26|0.97|9016006|exportiimporto #1|3|pants|2|Men|665|eingpriought|extra large|194710pale5466798701|rosy|Dozen|Unknown|28|eingoughteseable| +2419|AAAAAAAADHJAAAAA|1997-10-27||Other, western grounds must save nervously up a boxes. Again local couples ought to fall again industrial boards. True, natural assets would advance extra hills. Underlying |0.49|0.31|7004010|edu packbrand #10|4|curtains/drapes|7|Home|148|eingeseought|N/A|5smoke80684781388213|dim|Oz|Unknown|47|n stoughteseable| +2420|AAAAAAAAEHJAAAAA|1997-10-27|2000-10-26|Hard roads seem prospective pp.. Distant years mi|3.88|2.63|7013001|exportinameless #1|13|wallpaper|7|Home|856|callyantieing|N/A|5221234073470sandy99|tan|Oz|Unknown|19|barableeseable| +2421|AAAAAAAAEHJAAAAA|2000-10-27||Stones can find to a operations. Practically tough farmers accept recently. Certainly high fans know graduall|4.96|1.53|7008002|namelessbrand #2|13|lighting|7|Home|484|eseeingese|N/A|5221234073470sandy99|light|Carton|Unknown|26|oughtableeseable| +2422|AAAAAAAAGHJAAAAA|1997-10-27|1999-10-27|Engines enter countries. Scottish students exist islands. Poor eyes can assume properly comfortable societies; continued, |7.87|4.87|4004001|edu packedu pack #1|4|athletic|4|Shoes|59|n stanti|small|6linen04306344086273|steel|Ton|Unknown|22|ableableeseable| +2423|AAAAAAAAGHJAAAAA|1999-10-28|2001-10-26|Bare taxes wil|1.73|4.87|8015004|scholarmaxi #4|15|fishing|8|Sports|119|n stanti|N/A|731767577441peru9913|tan|Oz|Unknown|58|priableeseable| +2424|AAAAAAAAGHJAAAAA|2001-10-27||Significant, pure forces ought to let usually newspapers; as recent members hear final paths. Girls might define. Hours pay very increased miles. Implications must commence|64.23|34.04|8003005|exportinameless #5|15|basketball|8|Sports|119|n stanti|N/A|saddle47800956257876|gainsboro|Pound|Unknown|76|eseableeseable| +2425|AAAAAAAAJHJAAAAA|1997-10-27||Techniques assume at least old children. Legal daughters happen only. Courses begin exec|8.14|6.51|3004002|edu packexporti #2|4|school-uniforms|3|Children|83|prieing|medium|667turquoise00562498|papaya|Pound|Unknown|7|antiableeseable| +2426|AAAAAAAAKHJAAAAA|1997-10-27|2000-10-26|Posts opt. Already potential grants cannot enable key women. More coastal flames should n|1.52|1.14|5004001|edu packscholar #1|4|classical|5|Music|91|oughtn st|N/A|271308699155light145|spring|Gram|Unknown|65|callyableeseable| +2427|AAAAAAAAKHJAAAAA|2000-10-27||Freshly ancient interactions may work. Formerly increasing hands used to burn large se|0.13|0.05|8002008|importonameless #8|4|baseball|8|Sports|91|oughtn st|N/A|271308699155light145|rose|Dram|Unknown|39|ationableeseable| +2428|AAAAAAAAMHJAAAAA|1997-10-27|1999-10-27|Often british structures realize individually during a miles; students talk effectively civil waves. Now national levels predict over true, only years. Girls canno|4.49|2.96|2003001|exportiimporto #1|3|pants|2|Men|904|esebarn st|petite|9443707619thistle731|orange|Gross|Unknown|47|eingableeseable| +2429|AAAAAAAAMHJAAAAA|1999-10-28|2001-10-26|Too western officers give less clear trends; nevertheless positive st|0.62|0.34|2003001|exportiedu pack #2|3|kids|4|Shoes|904|esebarn st|medium|82sandy8556904501415|purple|Tbl|Unknown|33|n stableeseable| +2430|AAAAAAAAMHJAAAAA|2001-10-27||Different, financial organisations shall give ideas; yesterday tough ends discuss also french, different levels. Large, democratic p|3.15|2.04|2003001|amalgamalg #1|3|dresses|1|Women|71|oughtation|petite|876floral44548449163|pink|N/A|Unknown|34|barprieseable| +2431|AAAAAAAAPHJAAAAA|1997-10-27||Possibly great customs suit close looks. Capable, frequent processes shall pass possible dangers; hard, private words act measures. Mysterious, acceptable fac|6.64|5.51|9002008|importomaxi #8|2|business|9|Books|63|prically|N/A|25069floral675506139|pale|Tbl|Unknown|5|oughtprieseable| +2432|AAAAAAAAAIJAAAAA|1997-10-27|2000-10-26|Complex, different boats pick only. Objectives assess on the bands; full, effective arts must mis|6.70|5.56|9009009|maximaxi #9|9|science|9|Books|14|eseought|N/A|turquoise67486418245|white|Tbl|Unknown|47|ableprieseable| +2433|AAAAAAAAAIJAAAAA|2000-10-27||Complex, different boats pick only. Objectives assess on the bands; full, effective arts must mis|7.98|5.56|10005003|scholarunivamalg #3|9|karoke|10|Electronics|14|eseought|N/A|6lavender45025745058|medium|Tsp|Unknown|6|priprieseable| +2434|AAAAAAAACIJAAAAA|1997-10-27|1999-10-27|High, independent benefits wait right trees. New, religious rivers buy regularly basic b|4.88|2.48|5001001|amalgscholar #1|1|rock|5|Music|130|barpriought|N/A|558peru0807759201462|navy|Carton|Unknown|63|eseprieseable| +2435|AAAAAAAACIJAAAAA|1999-10-28|2001-10-26|Dreadful, actual relations bar conditions. Usual, net lives must want chief, foreign hands. Units must see pure, vast victims. Main, specific children aid |2.45|2.48|5001001|exportiimporto #2|1|pants|2|Men|130|barpriought|medium|558peru0807759201462|yellow|Cup|Unknown|61|antiprieseable| +2436|AAAAAAAACIJAAAAA|2001-10-27||Dreadful, actual relations bar conditions. Usual, net lives must want chief, foreign hands. Units must see pure, vast victims. Main, specific children aid |7.90|2.60|6009005|maxicorp #5|1|womens watch|6|Jewelry|457|ationantiese|N/A|691347peru1282009503|sandy|Pound|Unknown|5|callyprieseable| +2437|AAAAAAAAFIJAAAAA|1997-10-27||Else competitive shares look alone, english years; large workers determine slowly as good ways; papers ought to dispose then to a years. High, royal lip|4.19|3.22|4004002|edu packedu pack #2|4|athletic|4|Shoes|121|oughtableought|extra large|8239673486orange7042|lawn|Tbl|Unknown|56|ationprieseable| +2438|AAAAAAAAGIJAAAAA|1997-10-27|2000-10-26|Techniques expand however activities. Clergy sustain young boys. Sufficient parts ask representatives; very poor years would slip at least low directors. Required estates join too. Pub|8.06|4.83|7014009|edu packnameless #9|14|glassware|7|Home|551|oughtantianti|N/A|1398448937580snow612|saddle|Pound|Unknown|5|eingprieseable| +2439|AAAAAAAAGIJAAAAA|2000-10-27||Techniques expand however activities. Clergy sustain young boys. Sufficient parts ask representatives; very poor years would slip at least low directors. Required estates join too. Pub|9.47|3.78|7001010|amalgbrand #10|14|bathroom|7|Home|551|oughtantianti|N/A|1398448937580snow612|steel|Ton|Unknown|2|n stprieseable| +2440|AAAAAAAAIIJAAAAA|1997-10-27|1999-10-27|Social group|2.88|1.09|5004001|edu packscholar #1|4|classical|5|Music|229|n stableable|N/A|4780tomato6849717944|tomato|Cup|Unknown|90|bareseeseable| +2441|AAAAAAAAIIJAAAAA|1999-10-28|2001-10-26|Social group|3.90|3.43|5004001|exportischolar #2|3|pop|5|Music|174|eseationought|N/A|257750604985908snow3|sienna|Carton|Unknown|33|oughteseeseable| +2442|AAAAAAAAIIJAAAAA|2001-10-27||Social group|71.58|44.37|5004001|corpmaxi #7|3|parenting|9|Books|79|eseationought|N/A|1magenta712815964916|yellow|Bundle|Unknown|12|ableeseeseable| +2443|AAAAAAAALIJAAAAA|1997-10-27||Here popular cards ring just firm benefit|8.08|6.94|7015004|scholarnameless #4|15|tables|7|Home|463|pricallyese|N/A|9351805943cream29324|lime|Pallet|Unknown|21|prieseeseable| +2444|AAAAAAAAMIJAAAAA|1997-10-27|2000-10-26|S|4.35|1.82|10008014|namelessunivamalg #14|8|scanners|10|Electronics|361|oughtcallypri|N/A|87283272710thistle67|yellow|Each|Unknown|21|eseeseeseable| +2445|AAAAAAAAMIJAAAAA|2000-10-27||False, concerned hearts may hear short, british|0.30|0.19|10008014|corpmaxi #6|8|golf|8|Sports|35|oughtcallypri|N/A|87283272710thistle67|tan|Ton|Unknown|37|antieseeseable| +2446|AAAAAAAAOIJAAAAA|1997-10-27|1999-10-27|Social, possible opportunities should not stop so still increased groups. Of course great men set usually back rights. Regulations put. Mag|3.95|3.00|8002007|importonameless #7|2|baseball|8|Sports|462|ablecallyese|N/A|679847809976violet06|yellow|Oz|Unknown|23|callyeseeseable| +2447|AAAAAAAAOIJAAAAA|1999-10-28|2001-10-26|Social, possible opportunities should not stop so still increased groups. Of course great men set usually back rights. Regulations put. Mag|3.62|2.78|7011008|amalgnameless #8|2|accent|7|Home|462|ablecallyese|N/A|68299951sienna115068|royal|N/A|Unknown|10|ationeseeseable| +2448|AAAAAAAAOIJAAAAA|2001-10-27||Social, possible opportunities should not stop so still increased groups. Of course great men set usually back rights. Regulations put. Mag|1.81|2.78|2002001|importoimporto #1|2|shirts|2|Men|799|ablecallyese|petite|68299951sienna115068|floral|Dozen|Unknown|13|eingeseeseable| +2449|AAAAAAAABJJAAAAA|1997-10-27||Very european writers ought to swim so efficient, proud opponents. Quickly medical si|3.61|2.27|9012002|importounivamalg #2|12|home repair|9|Books|446|callyeseese|N/A|779green338750334023|almond|Each|Unknown|2|n steseeseable| +2450|AAAAAAAACJJAAAAA|1997-10-27|2000-10-26|Criticisms would not think. Steps shall go previous, obvious jobs. Only current yo|12.06|6.99|7013001|exportinameless #1|13|wallpaper|7|Home|48|eingese|N/A|7270728887salmon8874|thistle|Bundle|Unknown|56|barantieseable| +2451|AAAAAAAACJJAAAAA|2000-10-27||Criticisms would not think. Steps shall go previous, obvious jobs. Only current yo|6.21|3.72|7013001|edu packmaxi #4|13|entertainments|9|Books|48|eingese|N/A|7270728887salmon8874|goldenrod|Bunch|Unknown|30|oughtantieseable| +2452|AAAAAAAAEJJAAAAA|1997-10-27|1999-10-27|Direct, unable problems could dry scarcely isolated, local strategies. Leading years must not follow |4.77|3.76|2004001|edu packimporto #1|4|sports-apparel|2|Men|543|prieseanti|large|9653salmon2454064195|tan|Gross|Unknown|31|ableantieseable| +2453|AAAAAAAAEJJAAAAA|1999-10-28|2001-10-26|Direct, unable problems could dry scarcely isolated, local strategies. Leading years must not follow |1.40|1.13|10003006|exportiunivamalg #6|4|dvd/vcr players|10|Electronics|543|prieseanti|N/A|9653salmon2454064195|slate|Tsp|Unknown|28|priantieseable| +2454|AAAAAAAAEJJAAAAA|2001-10-27||Direct, unable problems could dry scarcely isolated, local strategies. Leading years must not follow |2.26|1.51|8001005|amalgnameless #5|1|athletic shoes|8|Sports|543|prieseanti|N/A|9653salmon2454064195|metallic|Carton|Unknown|7|eseantieseable| +2455|AAAAAAAAHJJAAAAA|1997-10-27||Anxious investments shi|2.94|2.11|10014008|edu packamalgamalg #8|14|automotive|10|Electronics|682|ableeingcally|N/A|390seashell531959254|smoke|Gross|Unknown|36|antiantieseable| +2456|AAAAAAAAIJJAAAAA|1997-10-27|2000-10-26|Games should not build relevant letters. Excellent proceedings see periodically finally independent executives. Good, annual cases shall not reduce enough for a lips. Members get new, ni|17.68|12.19|6014001|edu packbrand #1|14|estate|6|Jewelry|282|ableeingable|N/A|saddle91606417579975|papaya|Gross|Unknown|44|callyantieseable| +2457|AAAAAAAAIJJAAAAA|2000-10-27||Besides realistic schools forget around now|2.22|0.88|6014001|importounivamalg #10|2|camcorders|10|Electronics|450|ableeingable|N/A|saddle91606417579975|rose|Gross|Unknown|35|ationantieseable| +2458|AAAAAAAAKJJAAAAA|1997-10-27|1999-10-27|Financial, hard laws meet cases. Generally special records could hope frien|3.29|2.79|10008011|namelessunivamalg #11|8|scanners|10|Electronics|228|eingableable|N/A|2006powder8741093520|puff|Carton|Unknown|64|eingantieseable| +2459|AAAAAAAAKJJAAAAA|1999-10-28|2001-10-26|Closer dependen|3.12|2.79|10008011|corpmaxi #12|6|parenting|9|Books|600|barbarcally|N/A|2006powder8741093520|peach|Oz|Unknown|4|n stantieseable| +2460|AAAAAAAAKJJAAAAA|2001-10-27||We|36.67|20.16|7014001|edu packnameless #1|14|glassware|7|Home|600|barbarcally|N/A|29600024858almond422|peru|Unknown|Unknown|33|barcallyeseable| +2461|AAAAAAAANJJAAAAA|1997-10-27||Little models cannot go obvious sub|9.57|5.55|4002002|importoedu pack #2|2|mens|4|Shoes|11|oughtought|N/A|8hot2514913242586564|rose|Lb|Unknown|22|oughtcallyeseable| +2462|AAAAAAAAOJJAAAAA|1997-10-27|2000-10-26|Suitable, major villages look. Different, political payments vote new, good clergy. Rep|2.71|2.35|5003001|exportischolar #1|3|pop|5|Music|247|ationeseable|N/A|63385833426navy06934|violet|Lb|Unknown|73|ablecallyeseable| +2463|AAAAAAAAOJJAAAAA|2000-10-27||Suitable, major villages look. Different, political payments vote new, good clergy. Rep|8.49|2.35|2002002|importoimporto #2|3|shirts|2|Men|247|ationeseable|medium|63385833426navy06934|seashell|Tbl|Unknown|92|pricallyeseable| +2464|AAAAAAAAAKJAAAAA|1997-10-27|1999-10-27|More than other contributions know also foreigners. Keen compa|4.84|4.35|2001001|amalgimporto #1|1|accessories|2|Men|22|ableable|medium|0095536steel00431830|seashell|Gross|Unknown|74|esecallyeseable| +2465|AAAAAAAAAKJAAAAA|1999-10-28|2001-10-26|Video-taped, original gardens visit at a interests. Ready tracks should not describe urban, foreign plans. Happy doors will know ago |4.13|4.35|9007006|brandmaxi #6|1|reference|9|Books|22|ableable|N/A|0095536steel00431830|tan|Tsp|Unknown|30|anticallyeseable| +2466|AAAAAAAAAKJAAAAA|2001-10-27||Provincial, digital materials ought to stay more; studies plunge later techniques; eventually academic proposals know officials; low teams get|0.94|4.35|1004001|edu packamalg #1|1|swimwear|1|Women|22|ableable|N/A|226962sienna72076777|thistle|Gram|Unknown|62|callycallyeseable| +2467|AAAAAAAADKJAAAAA|1997-10-27||White, normal notes get refugees. Potentially different leaders throw there much following words; fans review dogs. Unkn|4.04|1.73|10013003|exportiamalgamalg #3|13|stereo|10|Electronics|274|eseationable|N/A|9814637622384336red4|navajo|Gross|Unknown|90|ationcallyeseable| +2468|AAAAAAAAEKJAAAAA|1997-10-27|2000-10-26|More overall sides exist also new, economic republics. Dogs must enjoy possible rooms. Sales can meet suddenly special, fixed weeks. Children shall not let great, visible campaigns. Authoritie|39.49|19.35|4003001|exportiedu pack #1|3|kids|4|Shoes|150|barantiought|small|98peach0711389891287|grey|Gross|Unknown|70|eingcallyeseable| +2469|AAAAAAAAEKJAAAAA|2000-10-27||Parties shall give complex members. More existing relations sign new cattle. Reasons help sufficiently. Muscles may not see daily. Mothers can take only subjects. Lessons turn so with a polls. |4.63|1.43|2002002|importoimporto #2|3|shirts|2|Men|150|barantiought|medium|671spring37290963927|medium|Case|Unknown|29|n stcallyeseable| +2470|AAAAAAAAGKJAAAAA|1997-10-27|1999-10-27|Old things should not regulate. African walls could not say incidents. Great days keep always different women. Previous provisions may want|1.26|0.78|7010001|univnameless #1|10|flatware|7|Home|213|prioughtable|N/A|89950520676indian462|papaya|Dram|Unknown|62|barationeseable| +2471|AAAAAAAAGKJAAAAA|1999-10-28|2001-10-26|Outside extra days would portray then like, funny operations. Now very years can retain military holidays; bad, whole experts might affect. Critical r|1.87|1.36|7010001|edu packexporti #2|10|school-uniforms|3|Children|213|prioughtable|petite|sky36888526667591979|peru|Box|Unknown|17|oughtationeseable| +2472|AAAAAAAAGKJAAAAA|2001-10-27||Types keep in a rules; light, domestic sanctions enjoy neatly please british women. Words used to think in a examples. Above bare lectures shall cause. Cases use q|7.78|1.36|10011008|amalgamalgamalg #8|11|disk drives|10|Electronics|213|prioughtable|N/A|0579076tan5836770640|lace|Bundle|Unknown|33|ableationeseable| +2473|AAAAAAAAJKJAAAAA|1997-10-27||Young, other areas confirm already natural drawings. Recently boring friends ought to apply for the affairs. Banks can explore. High, happy sources might not smile too good bi|3.85|1.57|2001002|amalgimporto #2|1|accessories|2|Men|269|n stcallyable|extra large|119561salmon89827364|linen|Ounce|Unknown|17|priationeseable| +2474|AAAAAAAAKKJAAAAA|1997-10-27|2000-10-26|There different entries advertise usually too general groups; ministers would not make effective eyes; new, apparent ministers read so residential, expen|7.11|4.76|4004001|edu packedu pack #1|4|athletic|4|Shoes|171|oughtationought|small|92095127red917786197|yellow|Dram|Unknown|19|eseationeseable| +2475|AAAAAAAAKKJAAAAA|2000-10-27||There different entries advertise usually too general groups; ministers would not make effective eyes; new, apparent ministers read so residential, expen|30.26|21.78|4004001|brandbrand #6|7|decor|7|Home|302|ablebarpri|N/A|92095127red917786197|yellow|Dozen|Unknown|33|antiationeseable| +2476|AAAAAAAAMKJAAAAA|1997-10-27|1999-10-27|Arms consolidate late seats; as important r|6.97|5.36|1003001|exportiamalg #1|3|maternity|1|Women|39|n stpri|petite|131bisque92246578084|slate|Bunch|Unknown|84|callyationeseable| +2477|AAAAAAAAMKJAAAAA|1999-10-28|2001-10-26|Arms consolidate late seats; as important r|3.81|5.36|1003001|amalgscholar #2|3|rock|5|Music|623|priablecally|N/A|131bisque92246578084|honeydew|Unknown|Unknown|18|ationationeseable| +2478|AAAAAAAAMKJAAAAA|2001-10-27||Obviously certain hours ought to|1.29|0.55|1003001|corpbrand #9|3|rugs|7|Home|536|callyprianti|N/A|131bisque92246578084|lemon|Bunch|Unknown|49|eingationeseable| +2479|AAAAAAAAPKJAAAAA|1997-10-27||Junior, fundamental activities cannot hook actually special, moral subjects. Front years might not let. Grand armies shall examine later human children. Specific divisions could not overloo|95.05|66.53|1002002|importoamalg #2|2|fragrances|1|Women|293|prin stable|medium|6284601pink716998251|orange|Lb|Unknown|22|n stationeseable| +2480|AAAAAAAAALJAAAAA|1997-10-27|2000-10-26|Hostile days reach. Effective, huge men may not talk somehow. Almost hard cigarettes used to use|7.85|6.35|6002001|importocorp #1|2|diamonds|6|Jewelry|539|n stprianti|N/A|41935107955royal5874|navy|Gram|Unknown|37|bareingeseable| +2481|AAAAAAAAALJAAAAA|2000-10-27||Hostile days reach. Effective, huge men may not talk somehow. Almost hard cigarettes used to use|1.46|6.35|6002001|exportiamalg #2|2|maternity|1|Women|37|n stprianti|medium|41935107955royal5874|puff|Cup|Unknown|30|oughteingeseable| +2482|AAAAAAAACLJAAAAA|1997-10-27|1999-10-27|Institutions help shel|3.69|2.06|8007003|brandnameless #3|7|hockey|8|Sports|107|ationbarought|N/A|533121258snow6522743|peru|Lb|Unknown|42|ableeingeseable| +2483|AAAAAAAACLJAAAAA|1999-10-28|2001-10-26|Institutions help shel|17.36|2.06|8007003|scholarbrand #4|5|blinds/shades|7|Home|951|oughtantin st|N/A|75tomato331606214206|saddle|Bunch|Unknown|13|prieingeseable| +2484|AAAAAAAACLJAAAAA|2001-10-27||Institutions help shel|0.70|0.39|8007003|corpmaxi #5|5|golf|8|Sports|385|antieingpri|N/A|75tomato331606214206|royal|Oz|Unknown|78|eseeingeseable| +2485|AAAAAAAAFLJAAAAA|1997-10-27||Healthy children talk remaining principles. Windows compare ordinary girls; |5.56|2.33|6007006|brandcorp #6|7|pendants|6|Jewelry|297|ationn stable|N/A|9699indian0918168792|navy|Ounce|Unknown|32|antieingeseable| +2486|AAAAAAAAGLJAAAAA|1997-10-27|2000-10-26|Colourful walls play ever to a supporters; high sides will proceed suddenly animals. Separate dogs avoid en|0.91|0.80|1003001|exportiamalg #1|3|maternity|1|Women|525|antiableanti|large|3002551518pink953941|wheat|Gram|Unknown|1|callyeingeseable| +2487|AAAAAAAAGLJAAAAA|2000-10-27||Colourful walls play ever to a supporters; high sides will proceed suddenly animals. Separate dogs avoid en|4.78|0.80|10004006|edu packunivamalg #6|4|audio|10|Electronics|264|antiableanti|N/A|24019805cornflower49|black|Case|Unknown|23|ationeingeseable| +2488|AAAAAAAAILJAAAAA|1997-10-27|1999-10-27|Detailed companies may facilitate in the suggestions; scottish hopes lead more good shelves. Long, increased years drive perhaps elderly pressures; all good game|9.84|5.80|7003009|exportibrand #9|3|kids|7|Home|599|n stn stanti|N/A|86118green1488163462|royal|Ounce|Unknown|2|eingeingeseable| +2489|AAAAAAAAILJAAAAA|1999-10-28|2001-10-26|Rough users replace like a songs; readily so-called implications might |3.92|2.66|7003009|importoamalg #2|2|fragrances|1|Women|599|n stn stanti|medium|86118green1488163462|peru|N/A|Unknown|43|n steingeseable| +2490|AAAAAAAAILJAAAAA|2001-10-27||Rough users replace like a songs; readily so-called implications might |4.72|3.91|7003009|edu packamalg #1|2|swimwear|1|Women|50|n stn stanti|extra large|86118green1488163462|chocolate|Lb|Unknown|10|barn steseable| +2491|AAAAAAAALLJAAAAA|1997-10-27||Hot, full appointments reduce forward ever distinguished relationships. Under way soph|2.41|1.01|2002002|importoimporto #2|2|shirts|2|Men|174|eseationought|large|8peach35366482117174|midnight|Bundle|Unknown|32|oughtn steseable| +2492|AAAAAAAAMLJAAAAA|1997-10-27|2000-10-26|Industrial funds must stuff now weak men;|5.61|4.48|7003003|exportibrand #3|3|kids|7|Home|330|barpripri|N/A|frosted9962996289120|beige|Dozen|Unknown|98|ablen steseable| +2493|AAAAAAAAMLJAAAAA|2000-10-27||Western, successful services could see only general rates. More other police will influence years. Happy, new drugs must give local, small voters. Financial, formal women shall trust so yet fine l|5.23|4.48|3004002|edu packexporti #2|3|school-uniforms|3|Children|142|barpripri|petite|frosted9962996289120|puff|Pallet|Unknown|20|prin steseable| +2494|AAAAAAAAOLJAAAAA|1997-10-27|1999-10-27|Theoretically international tickets used to taste occasionally on a years. Regular techniques cannot no. Men shall show less strong, practical concentrations; cha|8.81|6.43|2001001|amalgimporto #1|1|accessories|2|Men|142|ableeseought|medium|8290sienna1435149455|puff|Case|Unknown|17|esen steseable| +2495|AAAAAAAAOLJAAAAA|1999-10-28|2001-10-26|Theoretically international tickets used to taste occasionally on a years. Regular techniques cannot no. Men shall show less strong, practical concentrations; cha|2.44|6.43|2001001|corpcorp #8|1|rings|6|Jewelry|431|oughtpriese|N/A|26912aquamarine58322|gainsboro|Ton|Unknown|10|antin steseable| +2496|AAAAAAAAOLJAAAAA|2001-10-27||Theoretically international tickets used to taste occasionally on a years. Regular techniques cannot no. Men shall show less strong, practical concentrations; cha|81.35|6.43|3003001|exportiexporti #1|1|toddlers|3|Children|431|oughtpriese|small|26912aquamarine58322|drab|Case|Unknown|22|callyn steseable| +2497|AAAAAAAABMJAAAAA|1997-10-27||Other, tight solicitors shall not win now clouds. There base drugs contain well by a workers; local churches expect usually applications; more open creditors should not improve even. The|2.66|1.11|8007008|brandnameless #8|7|hockey|8|Sports|69|n stcally|N/A|77775171106rose30117|violet|Tbl|Unknown|32|ationn steseable| +2498|AAAAAAAACMJAAAAA|1997-10-27|2000-10-26|Inner, slow children foster. Nice, political words get complaints. Individual, able opinions bite thus nasty materials. Active views used to join with a feet.|1.44|1.18|5003001|exportischolar #1|3|pop|5|Music|497|ationn stese|N/A|95navajo839203064445|navajo|Lb|Unknown|34|eingn steseable| +2499|AAAAAAAACMJAAAAA|2000-10-27||Games would not|2.97|1.18|8008010|namelessnameless #10|3|outdoor|8|Sports|495|antin stese|N/A|95navajo839203064445|snow|Carton|Unknown|18|n stn steseable| +2500|AAAAAAAAEMJAAAAA|1997-10-27|1999-10-27|Persons keep perhaps in a belief|0.73|0.35|6015005|scholarbrand #5|15|custom|6|Jewelry|642|ableesecally|N/A|00602788yellow971597|medium|Tbl|Unknown|82|barbarantiable| +2501|AAAAAAAAEMJAAAAA|1999-10-28|2001-10-26|Persons keep perhaps in a belief|4.82|3.18|6015005|corpmaxi #2|16|golf|8|Sports|69|ableesecally|N/A|00602788yellow971597|royal|Ounce|Unknown|51|oughtbarantiable| +2502|AAAAAAAAEMJAAAAA|2001-10-27||Persons keep perhaps in a belief|4.08|3.18|6015005|exportiedu pack #1|3|kids|4|Shoes|69|n stcally|medium|642465009red25639557|sandy|Ton|Unknown|54|ablebarantiable| +2503|AAAAAAAAHMJAAAAA|1997-10-27||Really soviet rights resign for a models. Quickly impossible injuries shall refer tory, obvious mothers; here american statements determine. Costs should n|1.95|1.44|2003002|exportiimporto #2|3|pants|2|Men|258|eingantiable|large|069401876571wheat212|firebrick|Tbl|Unknown|50|pribarantiable| +2504|AAAAAAAAIMJAAAAA|1997-10-27|2000-10-26|Royal, european steps discover just. Obligations might jump in a children. So ind|5.32|3.40|1002001|importoamalg #1|2|fragrances|1|Women|242|ableeseable|small|07203564214940snow97|smoke|Dram|Unknown|16|esebarantiable| +2505|AAAAAAAAIMJAAAAA|2000-10-27||Fun, statutor|2.97|1.12|1002001|importoimporto #2|2|shirts|2|Men|46|ableeseable|extra large|rosy0073409315831181|deep|Ounce|Unknown|24|antibarantiable| +2506|AAAAAAAAKMJAAAAA|1997-10-27|1999-10-27|Only familiar goods will not lie old losses; also progressive assets could spawn american seconds.|4.52|3.79|5003001|exportischolar #1|3|pop|5|Music|36|callypri|N/A|red08304704193020626|turquoise|Oz|Unknown|45|callybarantiable| +2507|AAAAAAAAKMJAAAAA|1999-10-28|2001-10-26|Only familiar goods will not lie old losses; also progressive assets could spawn american seconds.|7.91|3.71|5001002|amalgscholar #2|1|rock|5|Music|177|ationationought|N/A|645lavender700600632|royal|Oz|Unknown|98|ationbarantiable| +2508|AAAAAAAAKMJAAAAA|2001-10-27||Very useful studies appear raw, likely parents. Lost goods may pass actually different, similar problems; completely technical months can manipul|2.80|1.68|5001002|exportiimporto #1|1|pants|2|Men|177|ationationought|medium|645lavender700600632|lace|N/A|Unknown|4|eingbarantiable| +2509|AAAAAAAANMJAAAAA|1997-10-27||High members may not fulfil by a officials. Bishops may practise well to a bodies; both considerable problems would not make however organic important things. Particular, old companies must take |5.84|4.26|8016010|corpmaxi #10|16|golf|8|Sports|91|oughtn st|N/A|5514197249362yellow2|purple|Cup|Unknown|10|n stbarantiable| +2510|AAAAAAAAOMJAAAAA|1997-10-27|2000-10-26|At least appropriate types would select recordings. Dead, old balls help years. So practical factors can produce in order past, following |2.76|1.95|10015012|scholaramalgamalg #12|15|portable|10|Electronics|273|priationable|N/A|05425324739644khaki0|sienna|Lb|Unknown|30|baroughtantiable| +2511|AAAAAAAAOMJAAAAA|2000-10-27||At least appropriate types would select recordings. Dead, old balls help years. So practical factors can produce in order past, following |0.19|0.13|10015012|importobrand #8|15|costume|6|Jewelry|273|priationable|N/A|72182873762yellow273|tan|Case|Unknown|24|oughtoughtantiable| +2512|AAAAAAAAANJAAAAA|1997-10-27|1999-10-27|Years should draw else profitable, silent officials. Materials could not decide well new, likely studies. Long|1.58|1.39|5001001|amalgscholar #1|1|rock|5|Music|153|priantiought|N/A|1056red9138844484056|chartreuse|Gram|Unknown|20|ableoughtantiable| +2513|AAAAAAAAANJAAAAA|1999-10-28|2001-10-26|Widely young years might not assemble secondly nearly little profits. Boys live years; forwards head yea|0.73|1.39|2002002|importoimporto #2|1|shirts|2|Men|153|priantiought|large|00888403409smoke7087|red|Box|Unknown|5|prioughtantiable| +2514|AAAAAAAAANJAAAAA|2001-10-27||Widely young years might not assemble secondly nearly little profits. Boys live years; forwards head yea|1.70|1.24|10010016|univamalgamalg #16|10|memory|10|Electronics|153|priantiought|N/A|00888403409smoke7087|white|Oz|Unknown|11|eseoughtantiable| +2515|AAAAAAAADNJAAAAA|1997-10-27||Full volumes can show financial accountants. Geographically positive matters exist ready, excessive rooms. Old levels should beat books; only|2.85|1.22|7013006|exportinameless #6|13|wallpaper|7|Home|378|eingationpri|N/A|450015029rosy2654842|blanched|Carton|Unknown|48|antioughtantiable| +2516|AAAAAAAAENJAAAAA|1997-10-27|2000-10-26|Available pupils take necessarily good payments. Beneficial, detailed figures understand obviously with a lines. Commercial, poor changes used t|7.36|2.72|2002001|importoimporto #1|2|shirts|2|Men|159|n stantiought|small|6seashell59652758585|snow|Carton|Unknown|46|callyoughtantiable| +2517|AAAAAAAAENJAAAAA|2000-10-27||Girls will not delete in a applicants. Personal divisions loosen easy children; only tired talks lack |4.02|2.37|8009008|maxinameless #8|9|optics|8|Sports|273|priationable|N/A|89metallic2999364021|yellow|Gross|Unknown|6|ationoughtantiable| +2518|AAAAAAAAGNJAAAAA|1997-10-27|1999-10-27|Open, big trees sell orange women. Also only weeks must encourage well considerable books. Parties spend very. Laws review; funn|2.37|1.32|2003001|exportiimporto #1|3|pants|2|Men|478|eingationese|petite|049762427misty778507|turquoise|N/A|Unknown|45|eingoughtantiable| +2519|AAAAAAAAGNJAAAAA|1999-10-28|2001-10-26|Small friends might illustrate perhaps dead minutes. Similar areas say principles. Good, useful patients may vote |2.09|1.32|2003001|importoimporto #2|3|shirts|2|Men|478|eingationese|medium|049762427misty778507|pale|Bunch|Unknown|33|n stoughtantiable| +2520|AAAAAAAAGNJAAAAA|2001-10-27||Already successful police hold more free holidays. Alone, sensitive beds volunteer available poems. Interesting communities could fly. C|1.36|1.21|2003001|exportiamalgamalg #5|13|stereo|10|Electronics|177|eingationese|N/A|049762427misty778507|rosy|Pallet|Unknown|45|barableantiable| +2521|AAAAAAAAJNJAAAAA|1997-10-27||Places take rules. For example scientific buildings may not maintain notably developers. Prime, other heads limit marginal places. Good, part-tim|9.77|5.17|7002002|importobrand #2|2|bedding|7|Home|289|n steingable|N/A|94982850926honeydew2|tomato|Dozen|Unknown|42|oughtableantiable| +2522|AAAAAAAAKNJAAAAA|1997-10-27|2000-10-26|Windows shall not apply modern members. Trades ought to cry alone, modern trees. Friends can cal|4.84|3.87|5002001|importoscholar #1|2|country|5|Music|810|baroughteing|N/A|782384613714958pale2|steel|Dram|Unknown|96|ableableantiable| +2523|AAAAAAAAKNJAAAAA|2000-10-27||Windows shall not apply modern members. Trades ought to cry alone, modern trees. Friends can cal|29.99|15.29|8015004|scholarmaxi #4|15|fishing|8|Sports|119|n stoughtought|N/A|782384613714958pale2|seashell|Oz|Unknown|80|priableantiable| +2524|AAAAAAAAMNJAAAAA|1997-10-27|1999-10-27|American, liberal minerals may no|4.32|2.54|8004005|edu packnameless #5|4|camping|8|Sports|578|eingationanti|N/A|65667tan563167899642|salmon|Box|Unknown|23|eseableantiable| +2525|AAAAAAAAMNJAAAAA|1999-10-28|2001-10-26|Free years think together rates. New values cannot take tools; clean missiles must not perform as new stones. Times help to a directors; fresh, beautiful games scare actually low courts. All healthy |4.67|2.54|8004005|corpamalgamalg #10|16|wireless|10|Electronics|628|eingationanti|N/A|12899640moccasin6051|steel|Cup|Unknown|6|antiableantiable| +2526|AAAAAAAAMNJAAAAA|2001-10-27||Free years think together rates. New values cannot take tools; clean missiles must not perform as new stones. Times help to a directors; fresh, beautiful games scare actually low courts. All healthy |6.77|4.06|3002001|importoexporti #1|16|infants|3|Children|628|eingationanti|large|390272849675steel616|green|Each|Unknown|15|callyableantiable| +2527|AAAAAAAAPNJAAAAA|1997-10-27||Therefore common years look external, separate lines. Physical |8.18|3.35|3004002|edu packexporti #2|4|school-uniforms|3|Children|232|ablepriable|large|96451130687purple724|pale|Each|Unknown|6|ationableantiable| +2528|AAAAAAAAAOJAAAAA|1997-10-27|2000-10-26|Educational, nervous solicitors go and so on by the papers. Agricultural rates may not beat grateful ideas. Only ready records could bring also always important conclusions; so active sale|7.83|3.05|3001001|amalgexporti #1|1|newborn|3|Children|116|callyoughtought|small|7259896white72671396|snow|Ton|Unknown|22|eingableantiable| +2529|AAAAAAAAAOJAAAAA|2000-10-27||Educational, nervous solicitors go and so on by the papers. Agricultural rates may not beat grateful ideas. Only ready records could bring also always important conclusions; so active sale|13.79|7.30|3003002|exportiexporti #2|3|toddlers|3|Children|625|callyoughtought|medium|203378wheat087197464|hot|Cup|Unknown|5|n stableantiable| +2530|AAAAAAAACOJAAAAA|1997-10-27|1999-10-27|Beautiful judges relax. Practices like others. Current, free languages represent to a routes. Available firms use; expected, british |0.54|0.16|1002001|importoamalg #1|2|fragrances|1|Women|365|anticallypri|medium|796941167midnight000|plum|Lb|Unknown|7|barpriantiable| +2531|AAAAAAAACOJAAAAA|1999-10-28|2001-10-26|Beautiful judges relax. Practices like others. Current, free languages represent to a routes. Available firms use; expected, british |3.90|0.16|10005016|scholarunivamalg #16|2|karoke|10|Electronics|365|anticallypri|N/A|796941167midnight000|peach|Box|Unknown|54|oughtpriantiable| +2532|AAAAAAAACOJAAAAA|2001-10-27||Beautiful judges relax. Practices like others. Current, free languages represent to a routes. Available firms use; expected, british |7.20|2.16|10005016|importoexporti #1|2|infants|3|Children|365|anticallypri|petite|3860041973firebrick9|pink|Carton|Unknown|24|ablepriantiable| +2533|AAAAAAAAFOJAAAAA|1997-10-27||Corporate, fast hills give most changes. Late months move likely, full eyes. Employees develop around pensioners. Then numerous scholars would not charge by a earnings; pa|0.27|0.14|7005004|scholarbrand #4|5|blinds/shades|7|Home|641|oughtesecally|N/A|0138319peach14150314|snow|Ton|Unknown|8|pripriantiable| +2534|AAAAAAAAGOJAAAAA|1997-10-27|2000-10-26|Stro|9.35|8.13|6003007|exporticorp #7|3|gold|6|Jewelry|297|ationn stable|N/A|9misty57478113232791|indian|Oz|Unknown|48|esepriantiable| +2535|AAAAAAAAGOJAAAAA|2000-10-27||Position|75.22|8.13|6003007|corpmaxi #4|3|parenting|9|Books|595|ationn stable|N/A|8974738257slate71017|magenta|Box|Unknown|81|antipriantiable| +2536|AAAAAAAAIOJAAAAA|1997-10-27|1999-10-27|Mountains understand up a rights. Local sites will not break however up a circumstances; constant phrases come. Only, great branches might remain completely willing infl|2.43|2.04|6004001|edu packcorp #1|4|bracelets|6|Jewelry|5|anti|N/A|77905642188460steel5|wheat|Tsp|Unknown|64|callypriantiable| +2537|AAAAAAAAIOJAAAAA|1999-10-28|2001-10-26|Working, ill standards used to return with a suppliers. Rare, part-time services should not form. Rural groups might not find impatiently by a members. Examples apply roughly small rises|0.50|0.17|3004002|edu packexporti #2|4|school-uniforms|3|Children|936|anti|large|73156483seashell1548|wheat|Gross|Unknown|19|ationpriantiable| +2538|AAAAAAAAIOJAAAAA|2001-10-27||Working, ill standards used to return with a suppliers. Rare, part-time services should not form. Rural groups might not find impatiently by a members. Examples apply roughly small rises|3.59|0.17|3004002|exportiimporto #1|4|pants|2|Men|156|anti|medium|1889616730222light50|peach|Case|Unknown|25|eingpriantiable| +2539|AAAAAAAALOJAAAAA|1997-10-27||Young waters used to like straight substantial, rare writers. Legs make|3.92|1.52|5003002|exportischolar #2|3|pop|5|Music|113|prioughtought|N/A|3363398thistle000264|lemon|Dram|Unknown|14|n stpriantiable| +2540|AAAAAAAAMOJAAAAA|1997-10-27|2000-10-26|More married aspects replace eastern, creative institutions. Public, blue ideas will not reduce in a results. Sometimes underlying parameters use widely results. Id|4.15|2.44|2001001|amalgimporto #1|1|accessories|2|Men|151|oughtantiought|large|8511262302goldenrod9|smoke|Ton|Unknown|13|bareseantiable| +2541|AAAAAAAAMOJAAAAA|2000-10-27||More married aspects replace eastern, creative institutions. Public, blue ideas will not reduce in a results. Sometimes underlying parameters use widely results. Id|3.16|1.76|2001001|edu packunivamalg #10|1|sports|9|Books|151|oughtantiought|N/A|6425920509975violet7|spring|Oz|Unknown|66|oughteseantiable| +2542|AAAAAAAAOOJAAAAA|1997-10-27|1999-10-27|Basic sites rest|29.92|18.55|4002001|importoedu pack #1|2|mens|4|Shoes|351|oughtantipri|large|646029pink2702982368|red|Dozen|Unknown|19|ableeseantiable| +2543|AAAAAAAAOOJAAAAA|1999-10-28|2001-10-26|True, ugly plants may flow here single services. Parties record always resources. Thus little stairs play local, strong months. Circ|4.54|18.55|8001004|amalgnameless #4|2|athletic shoes|8|Sports|382|ableeingpri|N/A|royal147073049238333|snow|Oz|Unknown|32|prieseantiable| +2544|AAAAAAAAOOJAAAAA|2001-10-27||Practices must put more worlds; molecular, other policies will ignore. Proper soldiers could take bishops. National cases should see ad|0.19|18.55|8001004|exportiedu pack #1|3|kids|4|Shoes|382|ableeingpri|medium|421866195792682puff4|powder|Unknown|Unknown|50|eseeseantiable| +2545|AAAAAAAABPJAAAAA|1997-10-27||Applicatio|1.64|0.50|3002002|importoexporti #2|2|infants|3|Children|277|ationationable|economy|37330681yellow867786|puff|Dozen|Unknown|55|antieseantiable| +2546|AAAAAAAACPJAAAAA|1997-10-27|2000-10-26|Political, final arts appreciate available users. Here waiting memories can use long. For example current women communicate never budgets; sufficient, other rivals shall provide unhappy pupils. Rel|3.18|2.25|6007003|brandcorp #3|7|pendants|6|Jewelry|194|esen stought|N/A|1478257668seashell70|saddle|Ton|Unknown|66|callyeseantiable| +2547|AAAAAAAACPJAAAAA|2000-10-27||Days affect healthy, extra police. Bars can ask possibly however vict|2.48|1.66|6007003|brandbrand #8|7|decor|7|Home|196|callyn stought|N/A|693565981213183pink4|snow|Ton|Unknown|74|ationeseantiable| +2548|AAAAAAAAEPJAAAAA|1997-10-27|1999-10-27|Pages might s|8.85|6.19|6016005|corpbrand #5|16|consignment|6|Jewelry|19|n stought|N/A|062636257139pink0117|turquoise|Gross|Unknown|54|eingeseantiable| +2549|AAAAAAAAEPJAAAAA|1999-10-28|2001-10-26|Pages might s|4.12|6.19|6016005|amalgexporti #2|1|newborn|3|Children|199|n stn stought|medium|dodger53654482927679|peru|Gram|Unknown|15|n steseantiable| +2550|AAAAAAAAEPJAAAAA|2001-10-27||Pages might s|4.54|6.19|6016005|brandcorp #7|7|pendants|6|Jewelry|199|n stn stought|N/A|dodger53654482927679|plum|Bundle|Unknown|37|barantiantiable| +2551|AAAAAAAAHPJAAAAA|1997-10-27||Ago personal clients could sound already great, enthusiastic surveys. Namely acceptable winners cry; con|4.48|2.19|3004002|edu packexporti #2|4|school-uniforms|3|Children|80|bareing|medium|tan94080595434186523|slate|Each|Unknown|5|oughtantiantiable| +2552|AAAAAAAAIPJAAAAA|1997-10-27|2000-10-26|Countries put away indeed social services.|9.43|3.86|8016003|corpmaxi #3|16|golf|8|Sports|555|antiantianti|N/A|2131793violet2465974|peru|Lb|Unknown|76|ableantiantiable| +2553|AAAAAAAAIPJAAAAA|2000-10-27||Distant demonstrations sink demanding, regular beliefs. Men go comparable, old letters. Correct, large effects help ev|9.98|3.86|8016003|edu packimporto #2|4|sports-apparel|2|Men|23|priable|small|518009002014yellow64|yellow|Lb|Unknown|30|priantiantiable| +2554|AAAAAAAAKPJAAAAA|1997-10-27|1999-10-27|Sorry, fine men hit very future shoulders. Safely real eyes see so between a leaves. Also great functions would find completely. Parties cannot mean legally rapid, advanced cont|0.11|0.07|1002001|importoamalg #1|2|fragrances|1|Women|577|ationationanti|extra large|954926618640947sky63|turquoise|Pound|Unknown|6|eseantiantiable| +2555|AAAAAAAAKPJAAAAA|1999-10-28|2001-10-26|Pages used to understand right later right institut|3.41|0.07|7002006|importobrand #6|2|bedding|7|Home|575|antiationanti|N/A|954926618640947sky63|pale|Gram|Unknown|29|antiantiantiable| +2556|AAAAAAAAKPJAAAAA|2001-10-27||Ready officers press once again years. Particularly aware inches produce good nurses. Groups cut. Turkish, legal boundaries can hold probably now reasonable minutes|1.94|0.07|6009005|maxicorp #5|9|womens watch|6|Jewelry|575|antiationanti|N/A|5640203royal51358564|light|Unknown|Unknown|52|callyantiantiable| +2557|AAAAAAAANPJAAAAA|1997-10-27||International, important addresses earn now associations. Well vast developments encourage all in a cases. Social arms lose things. Strong shoulders will earn s|3.28|2.88|9013002|exportiunivamalg #2|13|self-help|9|Books|174|eseationought|N/A|579478steel625978219|peru|Pallet|Unknown|15|ationantiantiable| +2558|AAAAAAAAOPJAAAAA|1997-10-27|2000-10-26|Precious, likely patterns ought |1.69|0.54|1001001|amalgamalg #1|1|dresses|1|Women|190|barn stought|medium|79182531gainsboro656|pink|Case|Unknown|32|eingantiantiable| +2559|AAAAAAAAOPJAAAAA|2000-10-27||Accounts get once more independent words. Players would diminish yet various councils. Fiscal, important weeks kill common |1.16|0.58|2001002|amalgimporto #2|1|accessories|2|Men|190|barn stought|medium|9380lemon79725954277|turquoise|N/A|Unknown|54|n stantiantiable| +2560|AAAAAAAAAAKAAAAA|1997-10-27|1999-10-27|Small, political activities help great, bad policies. Therefore square features provide on a machines. Rules make over me|2.42|2.05|9001005|amalgmaxi #5|1|arts|9|Books|434|esepriese|N/A|04sky797115132545222|yellow|Gross|Unknown|52|barcallyantiable| +2561|AAAAAAAAAAKAAAAA|1999-10-28|2001-10-26|Definitions wait simple books. New, whole services follow effectively popular parts. Also giant thoughts speed small owners. True, social miles shall give very national p|71.25|39.90|4003002|exportiedu pack #2|1|kids|4|Shoes|434|esepriese|N/A|0tomato7238390909853|peach|Carton|Unknown|42|oughtcallyantiable| +2562|AAAAAAAAAAKAAAAA|2001-10-27||Definitions wait simple books. New, whole services follow effectively popular parts. Also giant thoughts speed small owners. True, social miles shall give very national p|1.93|0.84|2003001|exportiimporto #1|3|pants|2|Men|387|esepriese|medium|0tomato7238390909853|snow|Ton|Unknown|25|ablecallyantiable| +2563|AAAAAAAADAKAAAAA|1997-10-27||Following, other respects must not come new,|6.10|4.27|9012008|importounivamalg #8|12|home repair|9|Books|159|n stantiought|N/A|60peach5858368035851|steel|Box|Unknown|17|pricallyantiable| +2564|AAAAAAAAEAKAAAAA|1997-10-27|2000-10-26|Old, moving times might look available cuts. Rates come. Resources used to take also at a questions. Times move strong earnings. White, economic committees must marry. Progress|5.00|2.20|2003001|exportiimporto #1|3|pants|2|Men|108|eingbarought|petite|5267025462241peach17|thistle|Cup|Unknown|34|esecallyantiable| +2565|AAAAAAAAEAKAAAAA|2000-10-27||Old, moving times might look available cuts. Rates come. Resources used to take also at a questions. Times move strong earnings. White, economic committees must marry. Progress|0.16|2.20|3002002|importoexporti #2|2|infants|3|Children|155|antiantiought|medium|5267025462241peach17|tan|Bundle|Unknown|46|anticallyantiable| +2566|AAAAAAAAGAKAAAAA|1997-10-27|1999-10-27|Especially early girls glance however specific, relevant steps. Financial worlds telephone most dark gains. Warm, outdoor devices defend besides. Unions must not say narrow powers; individual ti|8.96|3.22|9001011|amalgmaxi #11|1|arts|9|Books|165|anticallyought|N/A|7seashell03976627392|royal|Ton|Unknown|31|callycallyantiable| +2567|AAAAAAAAGAKAAAAA|1999-10-28|2001-10-26|Especially early girls glance however specific, relevant steps. Financial worlds telephone most dark gains. Warm, outdoor devices defend besides. Unions must not say narrow powers; individual ti|9.05|3.22|9001011|exporticorp #8|1|gold|6|Jewelry|224|eseableable|N/A|41452733royal5069066|plum|Cup|Unknown|34|ationcallyantiable| +2568|AAAAAAAAGAKAAAAA|2001-10-27||Especially early girls glance however specific, relevant steps. Financial worlds telephone most dark gains. Warm, outdoor devices defend besides. Unions must not say narrow powers; individual ti|4.86|1.89|9001011|maxicorp #1|1|womens watch|6|Jewelry|739|n stpriation|N/A|41452733royal5069066|mint|Unknown|Unknown|70|eingcallyantiable| +2569|AAAAAAAAJAKAAAAA|1997-10-27||Sometimes foreign things might not follow for the most part with a events. Just political intentions cannot see very in a resources. So available feet look originally managers; distinct, accep|3.99|1.51|5002002|importoscholar #2|2|country|5|Music|259|n stantiable|N/A|9157692metallic17362|peach|Pound|Unknown|33|n stcallyantiable| +2570|AAAAAAAAKAKAAAAA|1997-10-27|2000-10-26|Still urban stages shall not take for a legs. Other, holy demands pay further young, positive numbers. A little criminal i|7.68|4.60|9002003|importomaxi #3|2|business|9|Books|286|callyeingable|N/A|27sky512626511609861|pink|Gross|Unknown|37|barationantiable| +2571|AAAAAAAAKAKAAAAA|2000-10-27||Still urban stages shall not take for a legs. Other, holy demands pay further young, positive numbers. A little criminal i|1.42|0.79|9002003|exportischolar #2|2|pop|5|Music|291|callyeingable|N/A|27sky512626511609861|hot|Each|Unknown|33|oughtationantiable| +2572|AAAAAAAAMAKAAAAA|1997-10-27|1999-10-27|However important children could expect sincerely by way of a potatoes. Even able cars suggest by the issues. Shoes would perform sincerely |4.86|3.01|8013003|exportimaxi #3|13|sailing|8|Sports|746|callyeseation|N/A|7068373499yellow6722|maroon|Tsp|Unknown|30|ableationantiable| +2573|AAAAAAAAMAKAAAAA|1999-10-28|2001-10-26|However important children could expect sincerely by way of a potatoes. Even able cars suggest by the issues. Shoes would perform sincerely |4.17|2.58|8010004|univmaxi #4|10|pools|8|Sports|43|callyeseation|N/A|40933026lace77776276|powder|Each|Unknown|87|priationantiable| +2574|AAAAAAAAMAKAAAAA|2001-10-27||Phases e|9.18|5.50|8010004|namelessnameless #5|8|outdoor|8|Sports|43|priese|N/A|40933026lace77776276|cornsilk|Bunch|Unknown|50|eseationantiable| +2575|AAAAAAAAPAKAAAAA|1997-10-27||Unique developments should guess once at the assumptions. Letters might not provide especially |4.38|2.01|7011006|amalgnameless #6|11|accent|7|Home|13|priought|N/A|1seashell77840923158|sandy|Gram|Unknown|18|antiationantiable| +2576|AAAAAAAAABKAAAAA|1997-10-27|2000-10-26|Other houses can go just much violent points. Local results might not refer about difficult, speci|0.36|0.13|4004001|edu packedu pack #1|4|athletic|4|Shoes|179|n stationought|petite|48372924yellow546585|olive|Dram|Unknown|33|callyationantiable| +2577|AAAAAAAAABKAAAAA|2000-10-27||Other houses can go just much violent points. Local results might not refer about difficult, speci|2.64|2.16|4004001|edu packamalg #2|4|swimwear|1|Women|676|n stationought|medium|6saddle9068211615709|snow|Bundle|Unknown|1|ationationantiable| +2578|AAAAAAAACBKAAAAA|1997-10-27|1999-10-27|Associations could go in a copies. Patterns settle horses. Indicators shall not pursue. Years find carefully particular flowers; fresh demands used to know most; later patient products|4.97|3.42|8002009|importonameless #9|2|baseball|8|Sports|275|antiationable|N/A|120puff9441959321459|powder|Tbl|Unknown|45|eingationantiable| +2579|AAAAAAAACBKAAAAA|1999-10-28|2001-10-26|Associations could go in a copies. Patterns settle horses. Indicators shall not pursue. Years find carefully particular flowers; fresh demands used to know most; later patient products|0.49|3.42|8002009|corpamalgamalg #13|16|wireless|10|Electronics|278|eingationable|N/A|120puff9441959321459|purple|Ounce|Unknown|4|n stationantiable| +2580|AAAAAAAACBKAAAAA|2001-10-27||Associations could go in a copies. Patterns settle horses. Indicators shall not pursue. Years find carefully particular flowers; fresh demands used to know most; later patient products|3.02|2.14|8002009|exportimaxi #1|16|sailing|8|Sports|278|eingationable|N/A|120puff9441959321459|sky|Pound|Unknown|23|bareingantiable| +2581|AAAAAAAAFBKAAAAA|1997-10-27||Here incredible activities prepare like a songs; occupational, black months score of course crops; royal authorities shall|8.68|3.55|1003002|exportiamalg #2|3|maternity|1|Women|186|callyeingought|small|848697971plum9322014|thistle|Bundle|Unknown|34|oughteingantiable| +2582|AAAAAAAAGBKAAAAA|1997-10-27|2000-10-26|Likely practitioners leave |1.21|1.06|10013016|exportiamalgamalg #16|13|stereo|10|Electronics|670|barationcally|N/A|9saddle3833112548920|royal|Pallet|Unknown|25|ableeingantiable| +2583|AAAAAAAAGBKAAAAA|2000-10-27||Creative students may test then almost cheap individuals. Females talk more tha|54.92|1.06|5001002|amalgscholar #2|1|rock|5|Music|670|barationcally|N/A|142624589sky14428053|midnight|Bundle|Unknown|3|prieingantiable| +2584|AAAAAAAAIBKAAAAA|1997-10-27|1999-10-27|Things take increasingly at a areas. Christian, common efforts afford under the teachers. Open, alone affairs join members; previous conditions live very |3.03|1.99|5001001|amalgscholar #1|1|rock|5|Music|188|eingeingought|N/A|29189552tomato913831|metallic|N/A|Unknown|62|eseeingantiable| +2585|AAAAAAAAIBKAAAAA|1999-10-28|2001-10-26|So effective animals used to live to|4.76|1.66|5001001|exportiimporto #2|3|pants|2|Men|188|eingeingought|small|29189552tomato913831|plum|Gram|Unknown|46|antieingantiable| +2586|AAAAAAAAIBKAAAAA|2001-10-27||Big, competitive questions write industries; remaining consumers make to every years. Permanent procedures would demonstrate uniquely often mo|0.45|1.66|5001001|importoimporto #1|2|shirts|2|Men|636|callyprically|economy|3dodger4248492596281|honeydew|Box|Unknown|13|callyeingantiable| +2587|AAAAAAAALBKAAAAA|1997-10-27||Conditions used to test so for a spirits; open, royal provisions might not look approximate|36.97|29.94|9011008|amalgunivamalg #8|11|cooking|9|Books|979|n stationn st|N/A|148876442goldenrod37|seashell|Case|Unknown|28|ationeingantiable| +2588|AAAAAAAAMBKAAAAA|1997-10-27|2000-10-26|Identical affairs stand things; repres|6.31|3.97|3003001|exportiexporti #1|3|toddlers|3|Children|427|ationableese|small|14sienna652672170328|rose|N/A|Unknown|16|eingeingantiable| +2589|AAAAAAAAMBKAAAAA|2000-10-27||Identical affairs stand things; repres|8.76|3.97|3003001|scholarmaxi #10|3|fishing|8|Sports|427|ationableese|N/A|14sienna652672170328|sky|Tsp|Unknown|50|n steingantiable| +2590|AAAAAAAAOBKAAAAA|1997-10-27|1999-10-27|Present, professional things ought to find. Intensive friends may mind at least questions; forward, old foll|26.35|18.44|10006007|corpunivamalg #7|6|musical|10|Electronics|725|antiableation|N/A|0216thistle292083151|slate|Lb|Unknown|58|barn stantiable| +2591|AAAAAAAAOBKAAAAA|1999-10-28|2001-10-26|Enough necessary powers can make nearly at a signals. Old-fashioned, whole officers say f|2.13|1.42|10006007|importoscholar #2|2|country|5|Music|725|antiableation|N/A|0216thistle292083151|metallic|Pound|Unknown|20|oughtn stantiable| +2592|AAAAAAAAOBKAAAAA|2001-10-27||Banks care no longer always other resources. Eager movements will add however with an leaders. Children believe for a banks. Around old areas used to learn finally related capital beds. Democ|2.06|1.42|9003001|exportimaxi #1|3|computers|9|Books|725|antiableation|N/A|400681924navajo14610|powder|N/A|Unknown|3|ablen stantiable| +2593|AAAAAAAABCKAAAAA|1997-10-27||Young views know then. Intact jews lack still. Too black planes resist p|7.59|5.46|1004002|edu packamalg #2|4|swimwear|1|Women|283|prieingable|petite|50260022192rosy06048|rosy|Tbl|Unknown|40|prin stantiable| +2594|AAAAAAAACCKAAAAA|1997-10-27|2000-10-26|Effective wives ought to adopt even golden sports; various shows cannot feel|3.70|1.36|7010005|univnameless #5|10|flatware|7|Home|12|ableought|N/A|66thistle17888996574|saddle|Dram|Unknown|20|esen stantiable| +2595|AAAAAAAACCKAAAAA|2000-10-27||New problems sho|0.12|1.36|7010005|exportimaxi #4|3|computers|9|Books|350|barantipri|N/A|0090872744lavender76|steel|Ton|Unknown|43|antin stantiable| +2596|AAAAAAAAECKAAAAA|1997-10-27|1999-10-27|Documents appear very low tickets; fine, single appeals would bring ever busy, vast parents; fields may choose |7.55|6.11|4003001|exportiedu pack #1|3|kids|4|Shoes|468|eingcallyese|large|5368945741khaki83536|burlywood|Box|Unknown|22|callyn stantiable| +2597|AAAAAAAAECKAAAAA|1999-10-28|2001-10-26|Upper, regular changes could teach as real, common exercises. Final, unique faces used to follow. Famous, total stre|0.67|6.11|4003001|edu packamalg #2|3|swimwear|1|Women|134|esepriought|N/A|36321335lawn59394984|peach|Bunch|Unknown|44|ationn stantiable| +2598|AAAAAAAAECKAAAAA|2001-10-27||For example free b|2.08|1.12|1001001|amalgamalg #1|1|dresses|1|Women|134|esepriought|small|244060526608papaya52|royal|Gross|Unknown|46|eingn stantiable| +2599|AAAAAAAAHCKAAAAA|1997-10-27||Industrial, remote members would suppose even on a references; doctors turn under the districts; simply current subjects involve small te|5.90|2.36|8001010|amalgnameless #10|1|athletic shoes|8|Sports|146|callyeseought|N/A|85lemon0250771742703|turquoise|Dram|Unknown|66|n stn stantiable| +2600|AAAAAAAAICKAAAAA|1997-10-27|2000-10-26|Now social arrangements publish men. Global, personal corners estimate together old things. There poor officers like perhaps french, available women. Natural, great answers must not contend ex|4.91|1.71|4001001|amalgedu pack #1|1|womens|4|Shoes|651|oughtantically|petite|0sienna7370469607688|tomato|Ton|Unknown|58|barbarcallyable| +2601|AAAAAAAAICKAAAAA|2000-10-27||Now social arrangements publish men. Global, personal corners estimate together old things. There poor officers like perhaps french, available women. Natural, great answers must not contend ex|0.28|1.71|3003002|exportiexporti #2|1|toddlers|3|Children|511|oughtantically|small|0sienna7370469607688|yellow|Dram|Unknown|22|oughtbarcallyable| +2602|AAAAAAAAKCKAAAAA|1997-10-27|1999-10-27|Precise estimates play then up to a teachers. Capable yards mean|2.56|1.04|6005003|scholarcorp #3|5|earings|6|Jewelry|377|ationationpri|N/A|930985134574rosy0067|puff|N/A|Unknown|5|ablebarcallyable| +2603|AAAAAAAAKCKAAAAA|1999-10-28|2001-10-26|Animals drill quickly by a cities. Contemporary, general miles move|0.53|1.04|9015012|scholarunivamalg #12|5|fiction|9|Books|377|ationationpri|N/A|930985134574rosy0067|midnight|Pound|Unknown|27|pribarcallyable| +2604|AAAAAAAAKCKAAAAA|2001-10-27||Minute children might |5.69|4.95|7006005|corpbrand #5|5|rugs|7|Home|346|ationationpri|N/A|11spring013985211160|smoke|Each|Unknown|8|esebarcallyable| +2605|AAAAAAAANCKAAAAA|1997-10-27||Green, different animals might delay mostly other, similar miles. Then tiny attempts take obviously very constant machines. Prime schools like again pe|4.58|2.06|8014006|edu packmaxi #6|14|tennis|8|Sports|689|n steingcally|N/A|2645020turquoise9029|royal|Tsp|Unknown|14|antibarcallyable| +2606|AAAAAAAAOCKAAAAA|1997-10-27|2000-10-26|Annual theories will not sleep particular colleagues. Inherent trees put partners. Other layers place there backs. Effects would know often for an guns. Certain, bitter|4.28|1.96|7011007|amalgnameless #7|11|accent|7|Home|535|antiprianti|N/A|417205turquoise20359|sky|Each|Unknown|27|callybarcallyable| +2607|AAAAAAAAOCKAAAAA|2000-10-27||Annual theories will not sleep particular colleagues. Inherent trees put partners. Other layers place there backs. Effects would know often for an guns. Certain, bitter|0.66|0.33|7011007|amalgexporti #2|1|newborn|3|Children|535|antiprianti|small|417205turquoise20359|salmon|Gram|Unknown|31|ationbarcallyable| +2608|AAAAAAAAADKAAAAA|1997-10-27|1999-10-27|So other|0.83|0.55|3002001|importoexporti #1|2|infants|3|Children|411|oughtoughtese|small|261olive285440650429|navajo|Oz|Unknown|39|eingbarcallyable| +2609|AAAAAAAAADKAAAAA|1999-10-28|2001-10-26|Original minutes sell studies. Very certain rocks|3.51|0.55|3002002|importoexporti #2|2|infants|3|Children|411|oughtoughtese|petite|261olive285440650429|peach|N/A|Unknown|50|n stbarcallyable| +2610|AAAAAAAAADKAAAAA|2001-10-27||Human, healthy institutions postpone then political, ltd. reasons. Strongly personal parties square less than. Rounds must connect ever in a features. D|1.48|1.28|1002001|importoamalg #1|2|fragrances|1|Women|411|oughtoughtese|petite|05562813443pale88976|red|Ounce|Unknown|20|baroughtcallyable| +2611|AAAAAAAADDKAAAAA|1997-10-27||Additional, special customers help fans. Poorly planned systems may not happen most far final forests. Nations will get really particular times. Bon|1.89|1.32|5003002|exportischolar #2|3|pop|5|Music|26|callyable|N/A|087069269973191snow9|metallic|Case|Unknown|52|oughtoughtcallyable| +2612|AAAAAAAAEDKAAAAA|1997-10-27|2000-10-26|Critical, old years get readers. Structures used to attract in a designs. Needs could not ascertain. Meetings want also. Problems may |40.37|17.35|10005012|scholarunivamalg #12|5|karoke|10|Electronics|284|eseeingable|N/A|916salmon93982621063|sandy|Unknown|Unknown|15|ableoughtcallyable| +2613|AAAAAAAAEDKAAAAA|2000-10-27||Critical, old years get readers. Structures used to attract in a designs. Needs could not ascertain. Meetings want also. Problems may |4.54|17.35|8001004|amalgnameless #4|1|athletic shoes|8|Sports|409|eseeingable|N/A|khaki363646170097852|thistle|Lb|Unknown|52|prioughtcallyable| +2614|AAAAAAAAGDKAAAAA|1997-10-27|1999-10-27|Usually upper authorities give fairly in the teachers. Good women might not provide purely remote objections. Beautiful assets mean ago;|1.28|0.93|10006014|corpunivamalg #14|6|musical|10|Electronics|82|ableeing|N/A|131996frosted9661924|smoke|Carton|Unknown|62|eseoughtcallyable| +2615|AAAAAAAAGDKAAAAA|1999-10-28|2001-10-26|As delighted deaths bring as a whole democratic quant|1.85|0.93|10006014|edu packnameless #6|4|camping|8|Sports|583|prieinganti|N/A|9thistle885145947391|pink|Bundle|Unknown|29|antioughtcallyable| +2616|AAAAAAAAGDKAAAAA|2001-10-27||Clubs relax already economic firms. Policies could show r|1.78|0.93|10006014|exportiimporto #1|4|pants|2|Men|291|prieinganti|medium|9thistle885145947391|hot|Oz|Unknown|77|callyoughtcallyable| +2617|AAAAAAAAJDKAAAAA|1997-10-27||Little colleagues ought to matter subsequently sharp practices. General, small metals pass on the circumstances. Special, social guns raise personally fresh|2.29|1.55|2002002|importoimporto #2|2|shirts|2|Men|158|eingantiought|extra large|6444641chocolate9847|yellow|Cup|Unknown|91|ationoughtcallyable| +2618|AAAAAAAAKDKAAAAA|1997-10-27|2000-10-26|Free, different divisions ought to see more whole terms. So substantial schools will measure others. British classes consider though dishes. Pupils mount. Ugly, economic schemes cannot erect |4.43|1.55|7001009|amalgbrand #9|1|bathroom|7|Home|263|pricallyable|N/A|982116799866powder34|violet|Ounce|Unknown|63|eingoughtcallyable| +2619|AAAAAAAAKDKAAAAA|2000-10-27||Please small engineers understand again now local children. White, contrary sides can hear great contracts. Still modern changes may not get alternative years. Over small po|4.40|1.55|7001009|exportiimporto #2|1|pants|2|Men|135|pricallyable|medium|766puff0230832860782|rose|Each|Unknown|8|n stoughtcallyable| +2620|AAAAAAAAMDKAAAAA|1997-10-27|1999-10-27|Rights shall not work constantly respectable years. Low impossible planes assist much economic, inadequate studies; musical notes provide still in a grounds|8.10|2.99|5002001|importoscholar #1|2|country|5|Music|376|callyationpri|N/A|25791pink79467295157|slate|Pound|Unknown|8|barablecallyable| +2621|AAAAAAAAMDKAAAAA|1999-10-28|2001-10-26|Rights shall not work constantly respectable years. Low impossible planes assist much economic, inadequate studies; musical notes provide still in a grounds|6.37|2.99|5002001|corpbrand #6|2|consignment|6|Jewelry|376|callyationpri|N/A|25791pink79467295157|peru|N/A|Unknown|99|oughtablecallyable| +2622|AAAAAAAAMDKAAAAA|2001-10-27||Rights shall not work constantly respectable years. Low impossible planes assist much economic, inadequate studies; musical notes provide still in a grounds|2.35|1.90|5002001|scholarmaxi #3|2|fishing|8|Sports|376|callyationpri|N/A|25791pink79467295157|pink|Carton|Unknown|13|ableablecallyable| +2623|AAAAAAAAPDKAAAAA|1997-10-27||Centuries will|1.83|0.84|1002002|importoamalg #2|2|fragrances|1|Women|79|n station|extra large|874970383083tomato51|peach|Dozen|Unknown|61|priablecallyable| +2624|AAAAAAAAAEKAAAAA|1997-10-27|2000-10-26|Involved, different questions show about final holidays. Recent theories used to make clinical hours; cases shall give on behalf of the principles. Ex|46.43|31.57|6014001|edu packbrand #1|14|estate|6|Jewelry|170|barationought|N/A|49358purple432224456|burnished|Dozen|Unknown|28|eseablecallyable| +2625|AAAAAAAAAEKAAAAA|2000-10-27||Involved, different questions show about final holidays. Recent theories used to make clinical hours; cases shall give on behalf of the principles. Ex|5.02|1.65|9016010|corpunivamalg #10|16|mystery|9|Books|140|bareseought|N/A|49358purple432224456|lawn|Tsp|Unknown|71|antiablecallyable| +2626|AAAAAAAACEKAAAAA|1997-10-27|1999-10-27|So basic defences would use regular numbers; about small stands can yield just forward apparent windows; problems get so never social babies. Good allowances ought to hear fingers. S|2.88|2.01|4003001|exportiedu pack #1|3|kids|4|Shoes|158|eingantiought|small|33805pale91083422504|slate|Pound|Unknown|64|callyablecallyable| +2627|AAAAAAAACEKAAAAA|1999-10-28|2001-10-26|Developments die young, small letters. Others love orders; impo|4.95|4.15|4003001|importobrand #8|3|bedding|7|Home|557|eingantiought|N/A|33805pale91083422504|indian|Case|Unknown|11|ationablecallyable| +2628|AAAAAAAACEKAAAAA|2001-10-27||Developments die young, small letters. Others love orders; impo|1.31|0.96|4003001|exportiedu pack #1|3|kids|4|Shoes|238|eingantiought|extra large|33805pale91083422504|violet|Pound|Unknown|52|eingablecallyable| +2629|AAAAAAAAFEKAAAAA|1997-10-27||Commonly recent victims make only things. Weapons may start. Additional, main shoes get also. Times could appear only to the steps; previous dimensions must make howe|8.31|3.40|6006006|corpcorp #6|6|rings|6|Jewelry|536|callyprianti|N/A|821752409102939puff3|saddle|Case|Unknown|10|n stablecallyable| +2630|AAAAAAAAGEKAAAAA|1997-10-27|2000-10-26|Other things get now. Quite eastern systems should not ask then new days; usual, good friends should work at a proposals. Highly pr|0.27|0.09|8004001|edu packnameless #1|4|camping|8|Sports|410|baroughtese|N/A|6736124227380tan7076|tan|Each|Unknown|5|barpricallyable| +2631|AAAAAAAAGEKAAAAA|2000-10-27||Other things get now. Quite eastern systems should not ask then new days; usual, good friends should work at a proposals. Highly pr|50.03|0.09|9014004|edu packunivamalg #4|14|sports|9|Books|748|eingeseation|N/A|17pale68664355207800|violet|Oz|Unknown|51|oughtpricallyable| +2632|AAAAAAAAIEKAAAAA|1997-10-27|1999-10-27|Independently soft players look very parts. Royal, poor groups beat often completely sure texts; national, used matches could deal only particular quantities. Pol|82.48|70.93|1004001|edu packamalg #1|4|swimwear|1|Women|377|ationationpri|petite|2982peru216115431141|peru|Bundle|Unknown|5|ablepricallyable| +2633|AAAAAAAAIEKAAAAA|1999-10-28|2001-10-26|Affairs could not accept eyes. Nearby, usual names might make suddenly black systems. Words should go only with a b|4.96|70.93|3003002|exportiexporti #2|4|toddlers|3|Children|377|ationationpri|extra large|2982peru216115431141|sky|Unknown|Unknown|11|pripricallyable| +2634|AAAAAAAAIEKAAAAA|2001-10-27||Affairs could not accept eyes. Nearby, usual names might make suddenly black systems. Words should go only with a b|8.32|70.93|3003002|corpamalgamalg #17|16|wireless|10|Electronics|18|eingought|N/A|085puff4779865449482|navajo|Ounce|Unknown|19|esepricallyable| +2635|AAAAAAAALEKAAAAA|1997-10-27||Communities know true, willing quantities; millions use|4.33|2.12|10006001|corpunivamalg #1|6|musical|10|Electronics|746|callyeseation|N/A|46896pale41422222934|deep|Tbl|Unknown|30|antipricallyable| +2636|AAAAAAAAMEKAAAAA|1997-10-27|2000-10-26|Public marks attract now between a personnel. Profits monitor police. Favorite, distinctive girls head ever. Men can let. Members co|5.75|5.00|6010005|univbrand #5|10|jewelry boxes|6|Jewelry|572|ableationanti|N/A|43seashell4508416373|slate|Ton|Unknown|7|callypricallyable| +2637|AAAAAAAAMEKAAAAA|2000-10-27||Public marks attract now between a personnel. Profits monitor police. Favorite, distinctive girls head ever. Men can let. Members co|80.94|43.70|6010005|importoscholar #2|10|country|5|Music|152|ableationanti|N/A|43seashell4508416373|navy|Tsp|Unknown|44|ationpricallyable| +2638|AAAAAAAAOEKAAAAA|1997-10-27|1999-10-27|Bad commentators should not happen; furious |0.55|0.19|9005011|scholarmaxi #11|5|history|9|Books|349|n stesepri|N/A|seashell147795076534|slate|Ounce|Unknown|8|eingpricallyable| +2639|AAAAAAAAOEKAAAAA|1999-10-28|2001-10-26|Bad commentators should not happen; furious |0.52|0.29|9005011|edu packexporti #2|4|school-uniforms|3|Children|164|esecallyought|extra large|seashell147795076534|sienna|Bundle|Unknown|49|n stpricallyable| +2640|AAAAAAAAOEKAAAAA|2001-10-27||Russian, liberal members could not change well sacred relations. Then fond issues will give. Primitive, critical years go so surprising supporters. |7.52|2.93|9005011|amalgmaxi #1|11|archery|8|Sports|118|esecallyought|N/A|795843turquoise17969|rose|Unknown|Unknown|40|baresecallyable| +2641|AAAAAAAABFKAAAAA|1997-10-27||Desperately prime vehicles will not remedy widely for once difficult operations. Distinct pla|3.18|1.30|8010002|univmaxi #2|10|pools|8|Sports|874|eseationeing|N/A|569rose9625688044383|tomato|Bunch|Unknown|38|oughtesecallyable| +2642|AAAAAAAACFKAAAAA|1997-10-27|2000-10-26|Normal ideas practise more. Late, particular cases may not pay rightly open, whole arms. Too cautious ways see useless, main arrangements; poor things hear straight top managers. Ch|0.60|0.52|8008003|namelessnameless #3|8|outdoor|8|Sports|20|barable|N/A|45262338gainsboro432|powder|N/A|Unknown|4|ableesecallyable| +2643|AAAAAAAACFKAAAAA|2000-10-27||Normal ideas practise more. Late, particular cases may not pay rightly open, whole arms. Too cautious ways see useless, main arrangements; poor things hear straight top managers. Ch|2.29|0.87|8008003|brandmaxi #4|7|reference|9|Books|20|barable|N/A|45262338gainsboro432|peach|Bunch|Unknown|42|priesecallyable| +2644|AAAAAAAAEFKAAAAA|1997-10-27|1999-10-27|Apart victorian clients might |56.42|47.95|3004001|edu packexporti #1|4|school-uniforms|3|Children|95|antin st|N/A|tan95849530248881606|seashell|Ton|Unknown|10|eseesecallyable| +2645|AAAAAAAAEFKAAAAA|1999-10-28|2001-10-26|Apart victorian clients might |9.56|7.07|3004001|exportiexporti #2|4|toddlers|3|Children|95|antin st|extra large|tan95849530248881606|orchid|Cup|Unknown|62|antiesecallyable| +2646|AAAAAAAAEFKAAAAA|2001-10-27||Votes could kill very minor relationships; arguments would guide aside examples; inches tackle points; colourful, external terms le|44.99|7.07|1001001|amalgamalg #1|1|dresses|1|Women|95|antin st|medium|tan95849530248881606|red|Each|Unknown|14|callyesecallyable| +2647|AAAAAAAAHFKAAAAA|1997-10-27||Also evolutionary weeks might realise profitable churches. Characteristics ought to go nevertheless. Ins|7.67|5.44|1003002|exportiamalg #2|3|maternity|1|Women|495|antin stese|medium|0392855002939sienna1|turquoise|Pallet|Unknown|3|ationesecallyable| +2648|AAAAAAAAIFKAAAAA|1997-10-27|2000-10-26|Likel|7.43|5.42|3002001|importoexporti #1|2|infants|3|Children|352|ableantipri|small|purple26963670610090|pink|Pallet|Unknown|60|eingesecallyable| +2649|AAAAAAAAIFKAAAAA|2000-10-27||Likel|4.44|3.15|3002001|corpamalgamalg #15|2|wireless|10|Electronics|432|ablepriese|N/A|purple26963670610090|sky|Bundle|Unknown|50|n stesecallyable| +2650|AAAAAAAAKFKAAAAA|1997-10-27|1999-10-27|White systems can obtain already. Recent factors could not get. Perfectly differ|4.85|3.34|1002001|importoamalg #1|2|fragrances|1|Women|272|ableationable|medium|77602powder445129196|turquoise|Each|Unknown|68|baranticallyable| +2651|AAAAAAAAKFKAAAAA|1999-10-28|2001-10-26|White systems can obtain already. Recent factors could not get. Perfectly differ|2.99|3.34|1002001|exportinameless #2|13|wallpaper|7|Home|272|ableationable|N/A|73262416823rose33582|orange|Pound|Unknown|20|oughtanticallyable| +2652|AAAAAAAAKFKAAAAA|2001-10-27||White systems can obtain already. Recent factors could not get. Perfectly differ|3.50|2.06|1002001|edu packimporto #1|4|sports-apparel|2|Men|208|ableationable|petite|78094wheat4285550423|papaya|Bundle|Unknown|25|ableanticallyable| +2653|AAAAAAAANFKAAAAA|1997-10-27||Proposed sources shall not disappear much also primary facilities; artists spring direct either british relationships. Sure pure arts leave well. Everyday, scottish benefits |4.94|4.19|4004002|edu packedu pack #2|4|athletic|4|Shoes|46|callyese|large|085181738magenta0276|khaki|Pound|Unknown|60|prianticallyable| +2654|AAAAAAAAOFKAAAAA|1997-10-27|2000-10-26|Direct ways topple inches. Complex years can take also across a words. Stupid, global implications would provide sure things. Russians take much for the spots. Indoors local men w|7.18|4.88|4003001|exportiedu pack #1|3|kids|4|Shoes|286|callyeingable|large|44peru34460874859460|papaya|Bunch|Unknown|44|eseanticallyable| +2655|AAAAAAAAOFKAAAAA|2000-10-27||Forthcoming, secondary plans would see then. Active masses swim low illegal frames. Guns can transfer at a schemes. |5.77|4.88|4003001|univnameless #6|3|flatware|7|Home|286|callyeingable|N/A|44peru34460874859460|turquoise|Unknown|Unknown|17|antianticallyable| +2656|AAAAAAAAAGKAAAAA|1997-10-27|1999-10-27|All realistic employees should attempt all only expert parties. Complete days cannot come as possible rules. Normal candidates would not pay there improved, o|5.65|3.72|5002001|importoscholar #1|2|country|5|Music|815|antioughteing|N/A|2692018tan8270559364|peach|Ton|Unknown|3|callyanticallyable| +2657|AAAAAAAAAGKAAAAA|1999-10-28|2001-10-26|British, wrong scales shall look. Everywhere modern shares ought to raise. Recent consumers make so ready years; stro|5.96|3.72|4001002|amalgedu pack #2|2|womens|4|Shoes|815|antioughteing|small|83971503green4754678|magenta|Pound|Unknown|51|ationanticallyable| +2658|AAAAAAAAAGKAAAAA|2001-10-27||British, wrong scales shall look. Everywhere modern shares ought to raise. Recent consumers make so ready years; stro|90.77|3.72|7003009|exportibrand #9|2|kids|7|Home|815|antioughteing|N/A|83971503green4754678|peach|Unknown|Unknown|56|einganticallyable| +2659|AAAAAAAADGKAAAAA|1997-10-27||However small values |1.49|0.93|9004008|edu packmaxi #8|4|entertainments|9|Books|454|eseantiese|N/A|2258903055orchid0389|royal|Each|Unknown|50|n stanticallyable| +2660|AAAAAAAAEGKAAAAA|1997-10-27|2000-10-26|Solutions might look much indian holidays. Already other lads sh|0.40|0.16|2002001|importoimporto #1|2|shirts|2|Men|353|priantipri|medium|1269600lavender57396|royal|N/A|Unknown|64|barcallycallyable| +2661|AAAAAAAAEGKAAAAA|2000-10-27||Popular, ordinary others know properly so northern months. Happily sweet even|4.16|3.28|2002001|amalgunivamalg #10|1|cameras|10|Electronics|353|priantipri|N/A|1269600lavender57396|spring|Oz|Unknown|100|oughtcallycallyable| +2662|AAAAAAAAGGKAAAAA|1997-10-27|1999-10-27|Together fresh men used to get hard, dependent operations. Sorry problems use for a co|5.21|1.71|1001001|amalgamalg #1|1|dresses|1|Women|87|ationeing|extra large|8465lawn612350269968|navajo|Tsp|Unknown|9|ablecallycallyable| +2663|AAAAAAAAGGKAAAAA|1999-10-28|2001-10-26|Physical others would not involve home ancient, possible shops; powers may not keep mo|6.82|5.59|4001002|amalgedu pack #2|1|womens|4|Shoes|87|ationeing|medium|7241018779209snow792|papaya|Gram|Unknown|18|pricallycallyable| +2664|AAAAAAAAGGKAAAAA|2001-10-27||Good, able parents used to retain also soon junior methods. Enquiries persuade even able, valuable police.|7.07|2.61|6005001|scholarcorp #1|5|earings|6|Jewelry|87|ationeing|N/A|7241018779209snow792|powder|Gross|Unknown|52|esecallycallyable| +2665|AAAAAAAAJGKAAAAA|1997-10-27||Exact powers keep so ago internal face|2.27|1.24|4004002|edu packedu pack #2|4|athletic|4|Shoes|655|antiantically|medium|19659lawn68526621372|lime|Ounce|Unknown|26|anticallycallyable| +2666|AAAAAAAAKGKAAAAA|1997-10-27|2000-10-26|Competitors may pin including the |0.82|0.26|9007003|brandmaxi #3|7|reference|9|Books|347|ationesepri|N/A|86427786081lime51313|ghost|Pound|Unknown|15|callycallycallyable| +2667|AAAAAAAAKGKAAAAA|2000-10-27||Great, local trees follow inadequate, high buildings. Since white cases would not tap much. Studies may not produce; too human terms can learn dep|0.70|0.26|1003002|exportiamalg #2|7|maternity|1|Women|347|ationesepri|small|86427786081lime51313|pink|Tbl|Unknown|3|ationcallycallyable| +2668|AAAAAAAAMGKAAAAA|1997-10-27|1999-10-27|Recommendations shall let at least racial individuals. Contemporary seats provide also foreign, known efforts. Dr|3.28|2.39|10012017|importoamalgamalg #17|12|monitors|10|Electronics|651|oughtantically|N/A|76082674177powder626|tan|Bundle|Unknown|17|eingcallycallyable| +2669|AAAAAAAAMGKAAAAA|1999-10-28|2001-10-26|Recommendations shall let at least racial individuals. Contemporary seats provide also foreign, known efforts. Dr|3.03|2.09|10012017|maximaxi #6|12|science|9|Books|311|oughtantically|N/A|76082674177powder626|smoke|Ounce|Unknown|23|n stcallycallyable| +2670|AAAAAAAAMGKAAAAA|2001-10-27||Recommendations shall let at least racial individuals. Contemporary seats provide also foreign, known efforts. Dr|8.86|2.09|10012017|exportischolar #1|3|pop|5|Music|311|oughtoughtpri|N/A|0588914295gainsboro2|sienna|Tsp|Unknown|24|barationcallyable| +2671|AAAAAAAAPGKAAAAA|1997-10-27||Appearances ought to commit dead, only days. Never new miners ought to come other, current designs. Just economic countries could pay|1.82|1.36|10009003|maxiunivamalg #3|9|televisions|10|Electronics|182|ableeingought|N/A|67811633salmon708961|puff|Ton|Unknown|21|oughtationcallyable| +2672|AAAAAAAAAHKAAAAA|1997-10-27|2000-10-26|Years cannot attract frantically. Low designers think with a subsidies. Clear, joint forests go especially simple, tiny complaints. Lists can run never strong|72.34|44.12|2002001|importoimporto #1|2|shirts|2|Men|370|barationpri|medium|tomato43239632772499|plum|Oz|Unknown|18|ableationcallyable| +2673|AAAAAAAAAHKAAAAA|2000-10-27||Years cannot attract frantically. Low designers think with a subsidies. Clear, joint forests go especially simple, tiny complaints. Lists can run never strong|8.01|44.12|2002001|univnameless #4|10|flatware|7|Home|370|barationpri|N/A|9914075080salmon9777|royal|Cup|Unknown|10|priationcallyable| +2674|AAAAAAAACHKAAAAA|1997-10-27|1999-10-27|Laws propose policies. Commercial, foreign restaurants could take. District|84.97|32.28|8001005|amalgnameless #5|1|athletic shoes|8|Sports|53|prianti|N/A|7378royal90455828349|black|Pallet|Unknown|84|eseationcallyable| +2675|AAAAAAAACHKAAAAA|1999-10-28|2001-10-26|Other, adverse times see newly possible police. Vital seconds could not poison now like a newspapers; years shall not travel prime, minor po|56.19|32.28|3004002|edu packexporti #2|4|school-uniforms|3|Children|53|prianti|small|488268peach729124883|peru|N/A|Unknown|21|antiationcallyable| +2676|AAAAAAAACHKAAAAA|2001-10-27||Other, adverse times see newly possible police. Vital seconds could not poison now like a newspapers; years shall not travel prime, minor po|99.02|32.67|8002007|importonameless #7|4|baseball|8|Sports|55|antianti|N/A|slate634937029218552|purple|Ton|Unknown|16|callyationcallyable| +2677|AAAAAAAAFHKAAAAA|1997-10-27||Good parents could find also also main characteristics; ships |2.53|1.36|10016009|corpamalgamalg #9|16|wireless|10|Electronics|865|anticallyeing|N/A|26thistle54667192220|sky|Case|Unknown|26|ationationcallyable| +2678|AAAAAAAAGHKAAAAA|1997-10-27|2000-10-26|Good spatial othe|6.71|4.36|7009009|maxibrand #9|9|mattresses|7|Home|25|antiable|N/A|24755847558568navy25|blush|Bundle|Unknown|18|eingationcallyable| +2679|AAAAAAAAGHKAAAAA|2000-10-27||Good spatial othe|2.98|4.36|1004002|edu packamalg #2|4|swimwear|1|Women|25|antiable|large|694448716054143red03|spring|Lb|Unknown|31|n stationcallyable| +2680|AAAAAAAAIHKAAAAA|1997-10-27|1999-10-27|Daughters should turn well poor words. General, young others go at present. Wet, desperate consequ|0.75|0.33|4001001|amalgedu pack #1|1|womens|4|Shoes|300|barbarpri|medium|00791649417slate2240|plum|Gram|Unknown|2|bareingcallyable| +2681|AAAAAAAAIHKAAAAA|1999-10-28|2001-10-26|Fast, effective skills used to communicate that is possible women. Right|5.88|0.33|6004002|edu packcorp #2|1|bracelets|6|Jewelry|300|barbarpri|N/A|00791649417slate2240|pale|Dozen|Unknown|1|oughteingcallyable| +2682|AAAAAAAAIHKAAAAA|2001-10-27||Central, dark prisoners |9.78|6.06|6004002|edu packedu pack #1|4|athletic|4|Shoes|203|barbarpri|medium|4yellow6963321185414|tan|Oz|Unknown|15|ableeingcallyable| +2683|AAAAAAAALHKAAAAA|1997-10-27||Homes will catch as solar records. Residential, sensible lives must acco|34.82|29.24|6001004|amalgcorp #4|1|birdal|6|Jewelry|852|ableantieing|N/A|297178gainsboro81660|red|Cup|Unknown|13|prieingcallyable| +2684|AAAAAAAAMHKAAAAA|1997-10-27|2000-10-26|Interpretations should not remember furiously professionals. Continuous, successful effects may remain wholly obligations; more than financial jobs might fulfil other, common s|2.44|0.85|4004001|edu packedu pack #1|4|athletic|4|Shoes|174|eseationought|large|2996143607purple5264|yellow|Ounce|Unknown|3|eseeingcallyable| +2685|AAAAAAAAMHKAAAAA|2000-10-27||Interpretations should not remember furiously professionals. Continuous, successful effects may remain wholly obligations; more than financial jobs might fulfil other, common s|4.70|0.85|4003002|exportiedu pack #2|4|kids|4|Shoes|279|eseationought|petite|seashell103231722524|papaya|Pallet|Unknown|22|antieingcallyable| +2686|AAAAAAAAOHKAAAAA|1997-10-27|1999-10-27|Sure young results may realize documents. Right other purposes may answer conventional times. Altogether appropriate rul|2.23|0.80|1002001|importoamalg #1|2|fragrances|1|Women|581|oughteinganti|large|8785royal88152921807|dim|Cup|Unknown|12|callyeingcallyable| +2687|AAAAAAAAOHKAAAAA|1999-10-28|2001-10-26|Sure young results may realize documents. Right other purposes may answer conventional times. Altogether appropriate rul|3.34|0.80|1002001|amalgamalg #2|1|dresses|1|Women|581|oughteinganti|medium|981757721015slate632|sky|Gram|Unknown|33|ationeingcallyable| +2688|AAAAAAAAOHKAAAAA|2001-10-27||Black, new votes give harsh, public feet. British changes discover original hands. Significant, active others go exceptionally very strong flowers. Othe|0.38|0.14|1002001|edu packedu pack #1|4|athletic|4|Shoes|896|oughteinganti|small|981757721015slate632|burlywood|Bunch|Unknown|41|eingeingcallyable| +2689|AAAAAAAABIKAAAAA|1997-10-27||Rough conservatives function easily views; modern, corresponding texts improve wide, faint experiments. Duties cannot support similarly pages. Shows should discuss apart scenes. Ye|34.30|23.32|9012002|importounivamalg #2|12|home repair|9|Books|92|ablen st|N/A|4783507395powder2471|pale|Tsp|Unknown|6|n steingcallyable| +2690|AAAAAAAACIKAAAAA|1997-10-27|2000-10-26|Therefore typical purposes decide cases. Academic offic|3.09|2.65|6007003|brandcorp #3|7|pendants|6|Jewelry|579|n stationanti|N/A|46229892718smoke8164|peach|Tsp|Unknown|19|barn stcallyable| +2691|AAAAAAAACIKAAAAA|2000-10-27||Therefore typical purposes decide cases. Academic offic|5.82|4.88|6007003|exportiexporti #2|7|toddlers|3|Children|579|n stationanti|medium|29737738932lavender2|salmon|Bundle|Unknown|14|oughtn stcallyable| +2692|AAAAAAAAEIKAAAAA|1997-10-27|1999-10-27|Northern, productive sides shall not renew local, informal pupils. Students may afford on the masters. Sufficiently whole degrees vi|4.38|2.14|10011007|amalgamalgamalg #7|11|disk drives|10|Electronics|583|prieinganti|N/A|9209644peru528835965|purple|Unknown|Unknown|5|ablen stcallyable| +2693|AAAAAAAAEIKAAAAA|1999-10-28|2001-10-26|Issues should set neatly. International, compl|7.18|2.14|10011007|amalgimporto #2|11|accessories|2|Men|599|prieinganti|N/A|9209644peru528835965|papaya|Bundle|Unknown|15|prin stcallyable| +2694|AAAAAAAAEIKAAAAA|2001-10-27||Issues should set neatly. International, compl|45.30|33.97|5004001|edu packscholar #1|4|classical|5|Music|254|eseantiable|N/A|9209644peru528835965|deep|Unknown|Unknown|32|esen stcallyable| +2695|AAAAAAAAHIKAAAAA|1997-10-27||Capitalist words could make easily for the patients. Again new schools could not visit just bonds; upper bases help poor, cheap goods. Either ri|9.65|7.23|6008008|namelesscorp #8|8|mens watch|6|Jewelry|656|callyantically|N/A|1khaki41000412541768|powder|Carton|Unknown|21|antin stcallyable| +2696|AAAAAAAAIIKAAAAA|1997-10-27|2000-10-26|So individual vehicles repeat then similar supporters. Serious, young par|0.31|0.24|4003001|exportiedu pack #1|3|kids|4|Shoes|234|esepriable|small|116steel444184282957|sienna|Each|Unknown|79|callyn stcallyable| +2697|AAAAAAAAIIKAAAAA|2000-10-27||Habitually alone products care also european police. Eyes would not admit so facilities. Odd, supreme documents will affect completely spanish shows. Currently small drugs may remain ab|3.10|0.24|4003001|amalgimporto #2|3|accessories|2|Men|160|esepriable|small|116steel444184282957|rose|Tbl|Unknown|8|ationn stcallyable| +2698|AAAAAAAAKIKAAAAA|1997-10-27|1999-10-27|Plain old foods cross to a factors. Global, attractive emotions would cause away however new crops. Small appeals ensure members. Times explain so so only reports. |4.01|1.36|8015009|scholarmaxi #9|15|fishing|8|Sports|144|eseeseought|N/A|papaya15920181238356|powder|Ounce|Unknown|6|eingn stcallyable| +2699|AAAAAAAAKIKAAAAA|1999-10-28|2001-10-26|Members would buy proposals. British sides might not help rates. Natural requirements might withdraw very similar industries; nearly other publications could happen suffi|1.58|0.53|8015009|amalgamalg #2|1|dresses|1|Women|302|ablebarpri|medium|639hot03748455091941|ghost|Oz|Unknown|19|n stn stcallyable| +2700|AAAAAAAAKIKAAAAA|2001-10-27||Beautiful responses view immediately in the grounds. Comparable, commercial others answer much dangerous meanings. Quite other problems investigate so. Most cla|0.48|0.53|8015009|maxibrand #1|9|mattresses|7|Home|488|ablebarpri|N/A|2674sky2978192177280|red|Bundle|Unknown|23|barbarationable| +2701|AAAAAAAANIKAAAAA|1997-10-27||Stocks achieve even good, level reports. Authorities would see after a ways; here male shows would|1.14|0.86|6007006|brandcorp #6|7|pendants|6|Jewelry|126|callyableought|N/A|2steel88935823447579|seashell|Bundle|Unknown|90|oughtbarationable| +2702|AAAAAAAAOIKAAAAA|1997-10-27|2000-10-26|Great, old things will back about however modern yards. Rather selective rows may not try presumably differences. Weapons used to read organizations; go|4.36|1.91|8013003|exportimaxi #3|13|sailing|8|Sports|561|oughtcallyanti|N/A|171525528236730red93|turquoise|Ounce|Unknown|7|ablebarationable| +2703|AAAAAAAAOIKAAAAA|2000-10-27||Dollars may live finally like a services. Short pensioners see frequent, new dangers. Red proportions come entirely early, vital delegates. Others |8.94|3.30|5004002|edu packscholar #2|13|classical|5|Music|561|oughtcallyanti|N/A|2711184360007sienna1|tomato|Lb|Unknown|18|pribarationable| +2704|AAAAAAAAAJKAAAAA|1997-10-27|1999-10-27|Terrible countries could take objects. National roots should not move companies. Females must not tick. Then ordinary cars go at worst for a reports. |8.80|3.16|8004005|edu packnameless #5|4|camping|8|Sports|332|ablepripri|N/A|95spring612084133542|yellow|Box|Unknown|14|esebarationable| +2705|AAAAAAAAAJKAAAAA|1999-10-28|2001-10-26|Principles know once important woods. Other households should say much also noble leads; special, impossible men take. Originally negative times guide just; now eligible patterns mature flat nat|6.63|3.97|10001003|amalgunivamalg #3|1|cameras|10|Electronics|332|ablepripri|N/A|5papaya9783517748286|peru|Gram|Unknown|5|antibarationable| +2706|AAAAAAAAAJKAAAAA|2001-10-27||Girls ought to get then new women. Much new groups could remind so for a institutions. Dogs must achieve then to a off|2.08|3.97|1001001|amalgamalg #1|1|dresses|1|Women|324|eseablepri|medium|25478993965chiffon04|light|Box|Unknown|8|callybarationable| +2707|AAAAAAAADJKAAAAA|1997-10-27||New, real events shall like most together only records. H|0.63|0.37|6007004|brandcorp #4|7|pendants|6|Jewelry|359|n stantipri|N/A|34760saddle702011542|violet|Tsp|Unknown|64|ationbarationable| +2708|AAAAAAAAEJKAAAAA|1997-10-27|2000-10-26|Also real details used to travel for the articles. A bit anonymous factors aim there early, clean standards. Normal grounds weaken in a times; significant wheels cannot help black studies. Right l|5.29|2.22|3003001|exportiexporti #1|3|toddlers|3|Children|183|prieingought|N/A|229578873917564rosy5|red|Ton|Unknown|2|eingbarationable| +2709|AAAAAAAAEJKAAAAA|2000-10-27||International, cold rights join much possible, corporate thing|1.55|2.22|7004010|edu packbrand #10|4|curtains/drapes|7|Home|99|prieingought|N/A|229578873917564rosy5|salmon|Cup|Unknown|57|n stbarationable| +2710|AAAAAAAAGJKAAAAA|1997-10-27|1999-10-27|Great, other times pick since to a farmers. Areas exist poor, married advantages. Other adults realize here principles. Other, interested eyes find services. Much effective feet ought|9.33|3.73|5004001|edu packscholar #1|4|classical|5|Music|101|oughtbarought|N/A|88840magenta60580134|puff|N/A|Unknown|9|baroughtationable| +2711|AAAAAAAAGJKAAAAA|1999-10-28|2001-10-26|Great, other times pick since to a farmers. Areas exist poor, married advantages. Other adults realize here principles. Other, interested eyes find services. Much effective feet ought|0.61|3.73|5004001|namelessnameless #2|8|outdoor|8|Sports|894|oughtbarought|N/A|yellow27419945068247|medium|Ton|Unknown|67|oughtoughtationable| +2712|AAAAAAAAGJKAAAAA|2001-10-27||Special jobs shall see then labour, healthy facilitie|5.91|4.31|5004001|edu packexporti #1|4|school-uniforms|3|Children|460|barcallyese|small|yellow27419945068247|metallic|Gross|Unknown|33|ableoughtationable| +2713|AAAAAAAAJJKAAAAA|1997-10-27||Elsewher|2.23|1.24|9015002|scholarunivamalg #2|15|fiction|9|Books|725|antiableation|N/A|8steel38282988902277|antique|Ton|Unknown|25|prioughtationable| +2714|AAAAAAAAKJKAAAAA|1997-10-27|2000-10-26|Certain pensions lay therefore. Then fair tears occur ago. Directors used to respect more others. Direct clothes must guarantee environmental traders. Later rich developments would know. Total, incre|9.90|7.12|7005005|scholarbrand #5|5|blinds/shades|7|Home|583|prieinganti|N/A|17766thistle26438272|maroon|Tsp|Unknown|10|eseoughtationable| +2715|AAAAAAAAKJKAAAAA|2000-10-27||Certain pensions lay therefore. Then fair tears occur ago. Directors used to respect more others. Direct clothes must guarantee environmental traders. Later rich developments would know. Total, incre|1.14|7.12|7005005|scholarmaxi #6|5|fishing|8|Sports|781|prieinganti|N/A|tomato98404264219603|white|Dozen|Unknown|13|antioughtationable| +2716|AAAAAAAAMJKAAAAA|1997-10-27|1999-10-27|Games would win at a systems. Ever possible assumptions ought to give bodies; blue ministers take similarly in addition dead boats. Warm possibilities place adequate councils. Unusual |94.10|62.10|2002001|importoimporto #1|2|shirts|2|Men|597|ationn stanti|medium|36191359grey29068472|royal|Dram|Unknown|5|callyoughtationable| +2717|AAAAAAAAMJKAAAAA|1999-10-28|2001-10-26|Games would win at a systems. Ever possible assumptions ought to give bodies; blue ministers take similarly in addition dead boats. Warm possibilities place adequate councils. Unusual |9.25|62.10|2002001|edu packbrand #6|2|estate|6|Jewelry|546|callyeseanti|N/A|35409343papaya043496|rosy|Pallet|Unknown|5|ationoughtationable| +2718|AAAAAAAAMJKAAAAA|2001-10-27||Games would win at a systems. Ever possible assumptions ought to give bodies; blue ministers take similarly in addition dead boats. Warm possibilities place adequate councils. Unusual |9.87|3.05|2002001|edu packimporto #1|4|sports-apparel|2|Men|749|n steseation|N/A|35409343papaya043496|violet|Unknown|Unknown|9|eingoughtationable| +2719|AAAAAAAAPJKAAAAA|1997-10-27||Frequently back times could not abide against a accounts;|9.08|7.17|3004002|edu packexporti #2|4|school-uniforms|3|Children|108|eingbarought|medium|74451977564peach8897|papaya|Oz|Unknown|48|n stoughtationable| +2720|AAAAAAAAAKKAAAAA|1997-10-27|2000-10-26|Pounds will leave even so occasional results. Simple, overseas parties could hear new applications. Children treat. Essential positions form nearly details. Hard available details make under succ|60.98|18.90|5001001|amalgscholar #1|1|rock|5|Music|257|ationantiable|N/A|78057grey68798663010|tomato|Ounce|Unknown|51|barableationable| +2721|AAAAAAAAAKKAAAAA|2000-10-27||So competitive criteria want polish effects. Later other variables could not know on|3.42|2.12|10012002|importoamalgamalg #2|1|monitors|10|Electronics|257|ationantiable|N/A|34456577rose27256397|yellow|Ton|Unknown|55|oughtableationable| +2722|AAAAAAAACKKAAAAA|1997-10-27|1999-10-27|Ago proposed hours accelerate. Urban stones used to find recent, interested delegates. Eyes rise. Always red leaders shall not put. |3.20|2.65|3001001|amalgexporti #1|1|newborn|3|Children|54|eseanti|small|5470thistle895257968|snow|Gross|Unknown|20|ableableationable| +2723|AAAAAAAACKKAAAAA||2001-10-26|Ago proposed hours accelerate. Urban stones used to find recent, interested delegates. Eyes rise. Always red leaders shall not put. |0.46||6005004||1||6|Jewelry||||67cornsilk5852258121||Pallet|Unknown||| +2724|AAAAAAAACKKAAAAA|2001-10-27||Ago proposed hours accelerate. Urban stones used to find recent, interested delegates. Eyes rise. Always red leaders shall not put. |4.55|2.73|1002001|importoamalg #1|2|fragrances|1|Women|732|ablepriation|medium|072237powder26310443|puff|Bunch|Unknown|44|eseableationable| +2725|AAAAAAAAFKKAAAAA|1997-10-27||Trees leap so extended moments. Possible adults might continue finally just civil years. By no means high hands must not think particularly values. |0.21|0.13|1002002|importoamalg #2|2|fragrances|1|Women|224|eseableable|medium|0746896056plum658817|maroon|Bunch|Unknown|50|antiableationable| +2726|AAAAAAAAGKKAAAAA|1997-10-27|2000-10-26|Only, civil sales make. More early years develop here realistic, important values. Able services would not expect rare mem|0.47|0.25|10006007|corpunivamalg #7|6|musical|10|Electronics|135|antipriought|N/A|8pale661911061359557|khaki|Bunch|Unknown|12|callyableationable| +2727|AAAAAAAAGKKAAAAA|2000-10-27||Only, civil sales make. More early years develop here realistic, important values. Able services would not expect rare mem|2.12|0.25|10006007|edu packamalg #2|6|swimwear|1|Women|22|antipriought|medium|8pale661911061359557|purple|Case|Unknown|31|ationableationable| +2728|AAAAAAAAIKKAAAAA|1997-10-27|1999-10-27|True prizes might not fight. Radically respective bedrooms take now in order other preferences. Like miles must explain forward available, early times. Too careful re|2.19|1.27|5001001|amalgscholar #1|1|rock|5|Music|32|ablepri|N/A|822seashell281825400|peach|Tbl|Unknown|40|eingableationable| +2729|AAAAAAAAIKKAAAAA|1999-10-28|2001-10-26|National, other concerns gain in a systems. European, bizarre developments panic different, future legs. Positive nations wou|3.80|1.27|5004002|edu packscholar #2|4|classical|5|Music|594|ablepri|N/A|white918882910786454|spring|Oz|Unknown|89|n stableationable| +2730|AAAAAAAAIKKAAAAA|2001-10-27||National, other concerns gain in a systems. European, bizarre developments panic different, future legs. Positive nations wou|2.53|1.41|6015003|scholarbrand #3|4|custom|6|Jewelry|594|ablepri|N/A|29goldenrod661369774|sandy|Unknown|Unknown|55|barpriationable| +2731|AAAAAAAALKKAAAAA|1997-10-27||Good, silver assumptions think much under a earnings. Average, ne|2.66|1.30|1002002|importoamalg #2|2|fragrances|1|Women|144|eseeseought|extra large|60frosted17880887967|snow|Dram|Unknown|49|oughtpriationable| +2732|AAAAAAAAMKKAAAAA|1997-10-27|2000-10-26|Foreign, new forms account arbitrary, excessive fears. Asleep, mass grounds cannot lik|2.65|2.38|8002003|importonameless #3|2|baseball|8|Sports|453|priantiese|N/A|028532613tomato57605|turquoise|Case|Unknown|11|ablepriationable| +2733|AAAAAAAAMKKAAAAA|2000-10-27||Foreign, new forms account arbitrary, excessive fears. Asleep, mass grounds cannot lik|0.40|2.38|8002003|edu packimporto #2|4|sports-apparel|2|Men|73|priation|small|028532613tomato57605|steel|Ounce|Unknown|6|pripriationable| +2734|AAAAAAAAOKKAAAAA|1997-10-27|1999-10-27|Beings |5.41|1.62|7010005|univnameless #5|10|flatware|7|Home|507|ationbaranti|N/A|6736296201pale244733|firebrick|Lb|Unknown|15|esepriationable| +2735|AAAAAAAAOKKAAAAA|1999-10-28|2001-10-26|Beings |2.93|1.62|7010005|corpmaxi #6|16|golf|8|Sports|507|ationbaranti|N/A|6736296201pale244733|magenta|Ton|Unknown|40|antipriationable| +2736|AAAAAAAAOKKAAAAA|2001-10-27||Beings |4.46|3.61|1003001|exportiamalg #1|16|maternity|1|Women|137|ationpriought|economy|1820117087016sandy29|royal|Cup|Unknown|26|callypriationable| +2737|AAAAAAAABLKAAAAA|1997-10-27||Rather high miles will not understand whole, federal fees. Serious, important needs become almost deaf reservations; important p|1.67|1.20|6010002|univbrand #2|10|jewelry boxes|6|Jewelry|102|ablebarought|N/A|80687707royal6665420|slate|Carton|Unknown|56|ationpriationable| +2738|AAAAAAAACLKAAAAA|1997-10-27|2000-10-26|Excellent users s|1.40|0.71|1003001|exportiamalg #1|3|maternity|1|Women|867|ationcallyeing|medium|8979076798964papaya2|papaya|Pound|Unknown|19|eingpriationable| +2739|AAAAAAAACLKAAAAA|2000-10-27||Excellent users s|0.36|0.28|1003001|edu packcorp #4|4|bracelets|6|Jewelry|664|esecallycally|N/A|704473359lime3752851|seashell|N/A|Unknown|44|n stpriationable| +2740|AAAAAAAAELKAAAAA|1997-10-27|1999-10-27|Advertisements care. |1.07|0.69|3003001|exportiexporti #1|3|toddlers|3|Children|505|antibaranti|small|8527058097rose371584|sandy|Carton|Unknown|30|bareseationable| +2741|AAAAAAAAELKAAAAA|1999-10-28|2001-10-26|Advertisements care. |5.98|0.69|3003001|exportiedu pack #2|3|kids|4|Shoes|505|antibaranti|medium|01sandy8042250937453|saddle|Tbl|Unknown|21|oughteseationable| +2742|AAAAAAAAELKAAAAA|2001-10-27||Old, useful lists will not answer in the ages; attacks plant only royal powers; cultural, impossible qualities pick to a exports. Common movements |3.26|1.20|1001001|amalgamalg #1|3|dresses|1|Women|505|antibaranti|small|067651842279papaya58|smoke|Case|Unknown|15|ableeseationable| +2743|AAAAAAAAHLKAAAAA|1997-10-27||New centuries seem too. Wide, possible fathers shall rise in addition in a homes. Parti|51.60|22.18|9016008|corpunivamalg #8|16|mystery|9|Books|243|prieseable|N/A|345indian48815061904|ivory|Tsp|Unknown|25|prieseationable| +2744|AAAAAAAAILKAAAAA|1997-10-27|2000-10-26|Prepared, very procedures use ago assumptions; indian, new words make neither; concrete houses protest in a humans. Now white procedures affect then much important cars; clear, superb |4.18|2.34|5004001|edu packscholar #1|4|classical|5|Music|389|n steingpri|N/A|2979978turquoise1864|cyan|Cup|Unknown|46|eseeseationable| +2745|AAAAAAAAILKAAAAA|2000-10-27||Old eyes sleep. Quite sophisticated members run below sole rates. Hot guidelines put papers; difficul|2.41|2.34|5004001|importoedu pack #2|2|mens|4|Shoes|389|n steingpri|economy|757linen322933617216|turquoise|Dozen|Unknown|58|antieseationable| +2746|AAAAAAAAKLKAAAAA|1997-10-27|1999-10-27|Only bizarre areas approach equally possible businesses. Prices used to take wrong women. About serious relations say relatively. Always large items could offset surprised, remaining agen|7.63|6.40|5001001|amalgscholar #1|1|rock|5|Music|11|oughtought|N/A|344saddle92884830514|lawn|Bunch|Unknown|6|callyeseationable| +2747|AAAAAAAAKLKAAAAA|1999-10-28|2001-10-26|Underlying |2.92|6.40|2004002|edu packimporto #2|4|sports-apparel|2|Men|85|oughtought|extra large|9bisque8552836811324|moccasin|Unknown|Unknown|17|ationeseationable| +2748|AAAAAAAAKLKAAAAA|2001-10-27||Underlying |1.31|0.65|8010009|univmaxi #9|10|pools|8|Sports|946|callyesen st|N/A|0thistle566537144745|saddle|Pound|Unknown|16|eingeseationable| +2749|AAAAAAAANLKAAAAA|1997-10-27||Sign|5.65|1.97|7012010|importonameless #10|12|paint|7|Home|448|eingeseese|N/A|mint7485798553431338|mint|Carton|Unknown|5|n steseationable| +2750|AAAAAAAAOLKAAAAA|1997-10-27|2000-10-26|Sports might not refute creditors. Possible, appropriate examples will not indicate corporate patients. Potential centuri|4.90|3.08|5004001|edu packscholar #1|4|classical|5|Music|759|n stantiation|N/A|peach109808959219266|wheat|N/A|Unknown|12|barantiationable| +2751|AAAAAAAAOLKAAAAA|2000-10-27||Central, new cuts used to create yesterday possible, other hands. Labour citizens make sometimes streets. Strong schools hit apparently interesting laws. About major things may ma|6.02|3.08|5004001|importobrand #2|2|bedding|7|Home|759|n stantiation|N/A|peach109808959219266|smoke|Gram|Unknown|62|oughtantiationable| +2752|AAAAAAAAAMKAAAAA|1997-10-27|1999-10-27|New, major factors feel probably. Anywhere old studies afford sure cheap refugees. Seri|1.60|0.67|2003001|exportiimporto #1|3|pants|2|Men|27|ationable|small|488284993010spring82|royal|Gram|Unknown|78|ableantiationable| +2753|AAAAAAAAAMKAAAAA|1999-10-28|2001-10-26|Numbers could muster |3.03|0.67|2003001|importoexporti #2|2|infants|3|Children|67|ationcally|small|488284993010spring82|purple|Case|Unknown|25|priantiationable| +2754|AAAAAAAAAMKAAAAA|2001-10-27||Numbers could muster |1.57|0.67|10013001|exportiamalgamalg #1|13|stereo|10|Electronics|67|ationcally|N/A|8572262coral46110918|rosy|Dozen|Unknown|38|eseantiationable| +2755|AAAAAAAADMKAAAAA|1997-10-27||Simply central steps used to climb also with a reasons. Familiar, chief schools want still about a peasants. Members regard new links. Real, normal programmes pay probably with the minerals. Sorts t|8.15|2.93|6003004|exporticorp #4|3|gold|6|Jewelry|469|n stcallyese|N/A|5papaya6111701059434|purple|Each|Unknown|30|antiantiationable| +2756|AAAAAAAAEMKAAAAA|1997-10-27|2000-10-26|Also apparent waves average. Of course large vehicles will repeat also to the links. Finally light figures develop numbers; simply new committees shal|7.14|3.21|10002003|importounivamalg #3|2|camcorders|10|Electronics|270|barationable|N/A|38277619074azure2842|papaya|N/A|Unknown|24|callyantiationable| +2757|AAAAAAAAEMKAAAAA|2000-10-27||Only new farms operate in every eyes. Primary tickets limit just particularly old others. By now short regulations force well. Minutes know families. Patterns attempt mea|8.80|6.42|10002003|exportiunivamalg #4|2|dvd/vcr players|10|Electronics|270|barationable|N/A|079sandy172457233533|tan|N/A|Unknown|9|ationantiationable| +2758|AAAAAAAAGMKAAAAA|1997-10-27|1999-10-27|Independent, other pools must expect still; either main consequences tell with a materials. Russian schools should explore new children. Almost trying pieces sit|3.95|1.58|5001001|amalgscholar #1|1|rock|5|Music|313|prioughtpri|N/A|9indian4987258349406|purple|Lb|Unknown|22|eingantiationable| +2759|AAAAAAAAGMKAAAAA|1999-10-28|2001-10-26|Regular implications should not go sections; maximum forms might see in|3.42|1.58|5001001|exportischolar #2|1|pop|5|Music|313|prioughtpri|N/A|577970turquoise89546|burlywood|Unknown|Unknown|7|n stantiationable| +2760|AAAAAAAAGMKAAAAA|2001-10-27||Generally small facilities might stay yet programmes. Right comparative companies could know still domestic dark categories.|6.05|2.60|5001001|amalgedu pack #1|1|womens|4|Shoes|313|prioughtpri|small|577970turquoise89546|spring|Tbl|Unknown|34|barcallyationable| +2761|AAAAAAAAJMKAAAAA|1997-10-27||Single, new steps run hence new, private horses. Base, liable areas must not pick once only parents. Other, national years give much sessions. Feet must stop simply |13.02|9.11|1003002|exportiamalg #2|3|maternity|1|Women|414|eseoughtese|medium|436962952tomato37369|yellow|Pallet|Unknown|100|oughtcallyationable| +2762|AAAAAAAAKMKAAAAA|1997-10-27|2000-10-26|Old tears like with a mammal|86.73|72.85|4001001|amalgedu pack #1|1|womens|4|Shoes|121|oughtableought|large|rosy5206636994425060|medium|Cup|Unknown|55|ablecallyationable| +2763|AAAAAAAAKMKAAAAA|2000-10-27||Civil researchers take at least in a words; groups bear quite amidst a principles. Boys shall mean here unable, aware populations. Jewish dates exploit jus|0.92|72.85|6010004|univbrand #4|1|jewelry boxes|6|Jewelry|121|oughtableought|N/A|38897343884604royal0|snow|Dram|Unknown|24|pricallyationable| +2764|AAAAAAAAMMKAAAAA|1997-10-27|1999-10-27|Widespread, mental authorities go less than new symptoms. |3.63|1.30|9016005|corpunivamalg #5|16|mystery|9|Books|232|ablepriable|N/A|404994134salmon03856|azure|Pallet|Unknown|15|esecallyationable| +2765|AAAAAAAAMMKAAAAA|1999-10-28|2001-10-26|Both new appro|3.39|1.30|2004002|edu packimporto #2|16|sports-apparel|2|Men|526|ablepriable|petite|295154snow3961454098|pink|Case|Unknown|19|anticallyationable| +2766|AAAAAAAAMMKAAAAA|2001-10-27||Southern cases ensure to a names. Executive techniques will not negotiate much available hotels. Now common gardens will let |5.07|2.38|2004002|exportiexporti #1|3|toddlers|3|Children|195|ablepriable|petite|seashell471505052214|yellow|Gross|Unknown|60|callycallyationable| +2767|AAAAAAAAPMKAAAAA|1997-10-27||Trying types could not follow oddly autonomous walls. Gmt different others will build maybe able parameters. Private, main dealers shall not watch unfortunately also different novel|2.78|0.83|9010008|univunivamalg #8|10|travel|9|Books|171|oughtationought|N/A|658762203170peru1779|papaya|Tbl|Unknown|25|ationcallyationable| +2768|AAAAAAAAANKAAAAA|1997-10-27|2000-10-26|Entire questions find too|6.20|4.40|1004001|edu packamalg #1|4|swimwear|1|Women|603|pribarcally|extra large|80256551tomato322148|turquoise|Pallet|Unknown|36|eingcallyationable| +2769|AAAAAAAAANKAAAAA|2000-10-27||Entire questions find too|8.37|4.40|10007016|brandunivamalg #16|7|personal|10|Electronics|603|pribarcally|N/A|02341powder862344706|metallic|Carton|Unknown|19|n stcallyationable| +2770|AAAAAAAACNKAAAAA|1997-10-27|1999-10-27|Probably new women should not enter differently. Rare, public letters take reasons. Councils receive similarly social minutes. Plants pr|6.67|4.93|8004001|edu packnameless #1|4|camping|8|Sports|17|ationought|N/A|845firebrick20326268|snow|Gross|Unknown|65|barationationable| +2771|AAAAAAAACNKAAAAA|1999-10-28|2001-10-26|Rather big companies shall contain barely royal solicitors. Numbers must not hear cultures;|77.12|4.93|8005002|scholarnameless #2|4|fitness|8|Sports|17|ationought|N/A|845firebrick20326268|saddle|Unknown|Unknown|65|oughtationationable| +2772|AAAAAAAACNKAAAAA|2001-10-27||Rather big companies shall contain barely royal solicitors. Numbers must not hear cultures;|2.62|4.93|2002001|importoimporto #1|4|shirts|2|Men|299|ationought|extra large|steel845716491529680|misty|Bunch|Unknown|82|ableationationable| +2773|AAAAAAAAFNKAAAAA|1997-10-27||Always determined parties go here early pounds. Now scientific patients give a little into a options. Different boxes end; in particular brown days come no doubt acros|2.73|1.69|3002002|importoexporti #2|2|infants|3|Children|33|pripri|medium|789seashell192434425|beige|Bunch|Unknown|29|priationationable| +2774|AAAAAAAAGNKAAAAA|1997-10-27|2000-10-26|Resources comply cheap, ready places. Different, other lights will pay well. Days assume more large courts. Recordings could not design also at the members. Yards can let still political others|73.05|32.87|9004003|edu packmaxi #3|4|entertainments|9|Books|563|pricallyanti|N/A|pale9730831983650428|orchid|Carton|Unknown|61|eseationationable| +2775|AAAAAAAAGNKAAAAA|2000-10-27||Resources comply cheap, ready places. Different, other lights will pay well. Days assume more large courts. Recordings could not design also at the members. Yards can let still political others|3.02|32.87|2001002|amalgimporto #2|1|accessories|2|Men|110|baroughtought|economy|1637881788midnight33|spring|Cup|Unknown|45|antiationationable| +2776|AAAAAAAAINKAAAAA|1997-10-27|1999-10-27|Global, heavy drawings work. Professionals believe certainly. Existing legs|5.47|2.46|10003006|exportiunivamalg #6|3|dvd/vcr players|10|Electronics|493|prin stese|N/A|11987789093drab36808|tomato|Unknown|Unknown|21|callyationationable| +2777|AAAAAAAAINKAAAAA|1999-10-28|2001-10-26|Common, commercial years may not learn then high pp.. So old colo|4.33|1.51|10003006|edu packmaxi #6|3|entertainments|9|Books|493|prin stese|N/A|11987789093drab36808|powder|Tbl|Unknown|39|ationationationable| +2778|AAAAAAAAINKAAAAA|2001-10-27||Scottish, chief stories give merely now fair words. So great walls could afford above. Far average acti|1.08|1.51|3004001|edu packexporti #1|4|school-uniforms|3|Children|493|prin stese|petite|11987789093drab36808|seashell|Cup|Unknown|18|eingationationable| +2779|AAAAAAAALNKAAAAA|1997-10-27||Fam|0.41|0.19|4001002|amalgedu pack #2|1|womens|4|Shoes|174|eseationought|medium|69sky168666642074334|powder|Bundle|Unknown|15|n stationationable| +2780|AAAAAAAAMNKAAAAA|1997-10-27|2000-10-26|Payments mean there at a spots. At all bottom hands implement predominantly to a conditions. Stones enrich twice important members. Mere|0.49|0.36|7005001|scholarbrand #1|5|blinds/shades|7|Home|534|eseprianti|N/A|sky70862160886353700|smoke|Ounce|Unknown|51|bareingationable| +2781|AAAAAAAAMNKAAAAA|2000-10-27||Never racial villages would look literally still active children; still so-called settings see seriously for the feet|2.35|0.36|6007006|brandcorp #6|7|pendants|6|Jewelry|534|eseprianti|N/A|68688powder150349292|lace|Dozen|Unknown|4|oughteingationable| +2782|AAAAAAAAONKAAAAA|1997-10-27|1999-10-27|New, difficult countries will not transfer even; troops say. Far forms might not lead more usual figures. Nowhere far systems produce practitioners. Due p|4.87|1.65|1004001|edu packamalg #1|4|swimwear|1|Women|375|antiationpri|extra large|423royal647800249630|sandy|Gross|Unknown|98|ableeingationable| +2783|AAAAAAAAONKAAAAA|1999-10-28|2001-10-26|New, difficult countries will not transfer even; troops say. Far forms might not lead more usual figures. Nowhere far systems produce practitioners. Due p|2.57|2.15|1004001|maxiunivamalg #13|4|televisions|10|Electronics|375|antiationpri|N/A|423royal647800249630|rosy|Carton|Unknown|19|prieingationable| +2784|AAAAAAAAONKAAAAA|2001-10-27||New, difficult countries will not transfer even; troops say. Far forms might not lead more usual figures. Nowhere far systems produce practitioners. Due p|4.40|2.20|2003001|exportiimporto #1|4|pants|2|Men|223|priableable|medium|11495721tomato852643|slate|Tsp|Unknown|36|eseeingationable| +2785|AAAAAAAABOKAAAAA|1997-10-27||Also general seconds use |9.56|4.68|4001002|amalgedu pack #2|1|womens|4|Shoes|127|ationableought|large|07232027657159royal3|pale|Bunch|Unknown|30|antieingationable| +2786|AAAAAAAACOKAAAAA|1997-10-27|2000-10-26|Ultimate, remarkable losses know characteristics. Young workers must stop precise symptoms. Mysterious women shall |9.84|8.85|6012003|importobrand #3|12|costume|6|Jewelry|48|eingese|N/A|02940029208royal1578|steel|Each|Unknown|96|callyeingationable| +2787|AAAAAAAACOKAAAAA|2000-10-27||Domestic losses doubt so feet. New,|2.28|8.85|6012003|maximaxi #4|9|science|9|Books|48|eingese|N/A|02940029208royal1578|salmon|Case|Unknown|21|ationeingationable| +2788|AAAAAAAAEOKAAAAA|1997-10-27|1999-10-27|Typically conservative organs must ensure big, mysterious differences; temporary, obvious payments used to say particularly to a messages; old, interesting securities can expect qualificat|3.14|2.44|3001001|amalgexporti #1|1|newborn|3|Children|30|barpri|economy|8003131759gainsboro1|yellow|Pallet|Unknown|7|eingeingationable| +2789|AAAAAAAAEOKAAAAA|1999-10-28|2001-10-26|Private, democratic arrangements can join automatically years. Words fall old authorities. Temporarily criminal groups ought to visit interesting, great sales; minutes enj|3.84|2.44|6016006|corpbrand #6|16|consignment|6|Jewelry|30|barpri|N/A|blush370580912464511|beige|Pound|Unknown|24|n steingationable| +2790|AAAAAAAAEOKAAAAA|2001-10-27||Eyes ensue. Courts find essentially|12.77|6.25|8008001|namelessnameless #1|16|outdoor|8|Sports|382|barpri|N/A|blush370580912464511|tomato|Dozen|Unknown|98|barn stationable| +2791|AAAAAAAAHOKAAAAA|1997-10-27||Colours ought to try potential, expensive parties. Excessive, possible pp. could lay perhaps awful bills. Only high stages wish ago from the years. Much french colleagues explain much shoulde|2.83|2.40|10005004|scholarunivamalg #4|5|karoke|10|Electronics|30|barpri|N/A|207612754538007puff7|rosy|Pound|Unknown|29|oughtn stationable| +2792|AAAAAAAAIOKAAAAA|1997-10-27|2000-10-26|Incomes become certainly only issues. Very strange scientists could forget indeed exper|0.46|0.40|6005001|scholarcorp #1|5|earings|6|Jewelry|481|oughteingese|N/A|7698398528117saddle2|sienna|Dozen|Unknown|50|ablen stationable| +2793|AAAAAAAAIOKAAAAA|2000-10-27||Increasingly continued changes shall turn now residential proceedings. Banks swim only in the questions. Required exceptions shall sense economic,|7.57|0.40|6005001|namelessunivamalg #6|5|scanners|10|Electronics|810|baroughteing|N/A|0135130pink152034332|navy|Unknown|Unknown|62|prin stationable| +2794|AAAAAAAAKOKAAAAA|1997-10-27|1999-10-27|Both mixed doc|3.59|3.15|3004001|edu packexporti #1|4|school-uniforms|3|Children|117|ationoughtought|medium|sky84790775183622988|thistle|Box|Unknown|77|esen stationable| +2795|AAAAAAAAKOKAAAAA|1999-10-28|2001-10-26|Both mixed doc|8.71|3.15|7012006|importonameless #6|4|paint|7|Home|593|prin stanti|N/A|934251638650white902|white|Case|Unknown|28|antin stationable| +2796|AAAAAAAAKOKAAAAA|2001-10-27||Constant provisions help direct small men. Exact preferences dry in a ways; good, other |8.65|3.15|7012006|importounivamalg #1|12|home repair|9|Books|593|prin stanti|N/A|934251638650white902|ghost|Ton|Unknown|14|callyn stationable| +2797|AAAAAAAANOKAAAAA|1997-10-27||So ethnic championships think totally soft, appropriate customers. Perfect, military enterprises used to reach away essential authorities. Stages|5.77|4.21|9015002|scholarunivamalg #2|15|fiction|9|Books|313|prioughtpri|N/A|272snow5510265330933|royal|Ton|Unknown|86|ationn stationable| +2798|AAAAAAAAOOKAAAAA|1997-10-27|2000-10-26|Perhaps hard lakes consolidate pr|0.77|0.36|4001001|amalgedu pack #1|1|womens|4|Shoes|110|baroughtought|extra large|623699thistle2318110|seashell|Lb|Unknown|4|eingn stationable| +2799|AAAAAAAAOOKAAAAA|2000-10-27||Perhaps hard lakes consolidate pr|0.09|0.07|4001001|exportimaxi #10|13|sailing|8|Sports|110|baroughtought|N/A|89829611801white0108|rosy|Cup|Unknown|21|n stn stationable| +2800|AAAAAAAAAPKAAAAA|1997-10-27|1999-10-27|Good democrats behave a little american, good homes. Clients press at all industrial homes. Other variables must not look very initiatives. Glad, traditional children shall exchange. Pe|5.42|2.05|8002001|importonameless #1|2|baseball|8|Sports|408|eingbarese|N/A|9123902firebrick7677|tan|Unknown|Unknown|57|barbareingable| +2801|AAAAAAAAAPKAAAAA|1999-10-28|2001-10-26|Good democrats behave a little american, good homes. Clients press at all industrial homes. Other variables must not look very initiatives. Glad, traditional children shall exchange. Pe|9.03|2.05|8006002|corpnameless #2|2|football|8|Sports|508|eingbaranti|N/A|9384594sandy76867898|powder|Unknown|Unknown|33|oughtbareingable| +2802|AAAAAAAAAPKAAAAA|2001-10-27||Good democrats behave a little american, good homes. Clients press at all industrial homes. Other variables must not look very initiatives. Glad, traditional children shall exchange. Pe|3.69|1.91|8006002|edu packunivamalg #15|4|audio|10|Electronics|508|eingbaranti|N/A|9384594sandy76867898|rose|Gram|Unknown|45|ablebareingable| +2803|AAAAAAAADPKAAAAA|1997-10-27||Surely wel|7.88|3.78|1004002|edu packamalg #2|4|swimwear|1|Women|490|barn stese|petite|41726743steel0664902|salmon|Oz|Unknown|15|pribareingable| +2804|AAAAAAAAEPKAAAAA|1997-10-27|2000-10-26|Here political studies give once at the qu|1.78|0.99|9001009|amalgmaxi #9|1|arts|9|Books|330|barpripri|N/A|11063538376760snow00|sky|Oz|Unknown|32|esebareingable| +2805|AAAAAAAAEPKAAAAA|2000-10-27||Here political studies give once at the qu|5.40|0.99|8004006|edu packnameless #6|1|camping|8|Sports|207|ationbarable|N/A|446puff1662363997876|steel|Bunch|Unknown|13|antibareingable| +2806|AAAAAAAAGPKAAAAA|1997-10-27|1999-10-27|Provisions give systems. Skills explain later small glasses. Then moving pages direct all right more genuine companies. Upstairs goo|3.22|1.25|10010002|univamalgamalg #2|10|memory|10|Electronics|128|eingableought|N/A|3412686medium0979776|lime|Box|Unknown|57|callybareingable| +2807|AAAAAAAAGPKAAAAA|1999-10-28|2001-10-26|Clothes might like specially quiet, high descriptions. Industrial figures navigate particular officers. Good, empty countries deal so western fingers. Long relatives will n|1.84|1.15|8015008|scholarmaxi #8|15|fishing|8|Sports|110|baroughtought|N/A|03smoke6542975919762|indian|Tbl|Unknown|28|ationbareingable| +2808|AAAAAAAAGPKAAAAA|2001-10-27||Clothes might like specially quiet, high descriptions. Industrial figures navigate particular officers. Good, empty countries deal so western fingers. Long relatives will n|4.80|1.15|5004001|edu packscholar #1|15|classical|5|Music|110|baroughtought|N/A|47128powder717943034|indian|Lb|Unknown|13|eingbareingable| +2809|AAAAAAAAJPKAAAAA|1997-10-27||Lengthy, sad parties grow into a times; red, small millions manage international abilities. Delicate, suitable boys cannot find more unhappy losses. Examinations would no|4.15|2.44|3004002|edu packexporti #2|4|school-uniforms|3|Children|338|eingpripri|medium|528sandy376477330995|green|Oz|Unknown|14|n stbareingable| +2810|AAAAAAAAKPKAAAAA|1997-10-27|2000-10-26|New patients could accept sooner most left styles. Closed, free troops push just t|0.64|0.35|4003001|exportiedu pack #1|3|kids|4|Shoes|272|ableationable|large|247303370304270hot52|papaya|Unknown|Unknown|63|baroughteingable| +2811|AAAAAAAAKPKAAAAA|2000-10-27||New patients could accept sooner most left styles. Closed, free troops push just t|4.95|0.35|5002002|importoscholar #2|2|country|5|Music|694|esen stcally|N/A|658764927337325snow5|magenta|Carton|Unknown|40|oughtoughteingable| +2812|AAAAAAAAMPKAAAAA|1997-10-27|1999-10-27|Reserves undertake so special standards. Across natural leaders ought to develop |6.00|1.80|3002001|importoexporti #1|2|infants|3|Children|922|ableablen st|petite|466302pink9094144810|sienna|Cup|Unknown|5|ableoughteingable| +2813|AAAAAAAAMPKAAAAA|1999-10-28|2001-10-26|Reserves undertake so special standards. Across natural leaders ought to develop |1.22|0.47|3002001|scholarbrand #6|15|custom|6|Jewelry|461|ableablen st|N/A|466302pink9094144810|rosy|Dozen|Unknown|4|prioughteingable| +2814|AAAAAAAAMPKAAAAA|2001-10-27||Voluntary, surprising banks suit deliberately. Wrong stocks gain stri|0.09|0.06|3002001|scholarbrand #5|15|blinds/shades|7|Home|461|oughtcallyese|N/A|4226khaki56560623271|pink|Case|Unknown|30|eseoughteingable| +2815|AAAAAAAAPPKAAAAA|1997-10-27||Further appropriate children keep dear clubs. Profitable, wooden representatives could diversify; civil, bright seats could not buy. Inner ingredients should not promote isolated, outside talks. F|18.08|14.28|2003002|exportiimporto #2|3|pants|2|Men|47|ationese|medium|4924547309purple9650|rosy|Ounce|Unknown|31|antioughteingable| +2816|AAAAAAAAAALAAAAA|1997-10-27|2000-10-26|Relations cannot question besides european conditions|1.32|1.01|7015007|scholarnameless #7|15|tables|7|Home|116|callyoughtought|N/A|898978papaya52276355|tomato|Gram|Unknown|66|callyoughteingable| +2817|AAAAAAAAAALAAAAA|2000-10-27||Most sophisticated musicians ought to get more real, special types. Cheap friends would take;|5.09|1.57|1003002|exportiamalg #2|3|maternity|1|Women|116|callyoughtought|extra large|97496454765943pale67|royal|Lb|Unknown|85|ationoughteingable| +2818|AAAAAAAACALAAAAA|1997-10-27|1999-10-27|Relations should influence merely normal reactions. Empty comments clean really fa|21.40|11.12|7014009|edu packnameless #9|14|glassware|7|Home|78|eingation|N/A|45255970156thistle69|goldenrod|Gross|Unknown|11|eingoughteingable| +2819|AAAAAAAACALAAAAA|1999-10-28|2001-10-26|Relations should influence merely normal reactions. Empty comments clean really fa|1.96|11.12|7014009|importounivamalg #15|14|camcorders|10|Electronics|78|eingation|N/A|55817olive5450761102|lace|Gross|Unknown|15|n stoughteingable| +2820|AAAAAAAACALAAAAA|2001-10-27||Relations should influence merely normal reactions. Empty comments clean really fa|8.84|11.12|7004001|edu packbrand #1|4|curtains/drapes|7|Home|78|eingation|N/A|6260057698lemon86485|saddle|Tsp|Unknown|19|barableeingable| +2821|AAAAAAAAFALAAAAA|1997-10-27||Certain, adequate times would not believe also particular authorities. Soon|5.54|3.26|10013017|exportiamalgamalg #17|13|stereo|10|Electronics|147|ationeseought|N/A|092985478149101peru2|yellow|Unknown|Unknown|29|oughtableeingable| +2822|AAAAAAAAGALAAAAA|1997-10-27|2000-10-26|Great, only pages might not contribute so; small components require on a films. Times find apparently. So traditional sources find conditions. Gro|3.40|1.42|9011003|amalgunivamalg #3|11|cooking|9|Books|714|eseoughtation|N/A|4933888tomato4244463|aquamarine|Cup|Unknown|25|ableableeingable| +2823|AAAAAAAAGALAAAAA|2000-10-27||Red, maximum comments shall not break maybe current services. New symptoms see in a tables. Prominent, ord|4.51|1.42|2003002|exportiimporto #2|11|pants|2|Men|41|eseoughtation|large|49605307snow19708810|peach|Dozen|Unknown|64|priableeingable| +2824|AAAAAAAAIALAAAAA|1997-10-27|1999-10-27|Always entire activities shall not grow all at a pieces. Running crowds ought to approach civ|4.30|1.33|6007001|brandcorp #1|7|pendants|6|Jewelry|566|callycallyanti|N/A|9553506puff112567633|midnight|Case|Unknown|25|eseableeingable| +2825|AAAAAAAAIALAAAAA|1999-10-28|2001-10-26|Always entire activities shall not grow all at a pieces. Running crowds ought to approach civ|63.92|44.10|6007001|amalgedu pack #2|1|womens|4|Shoes|35|antipri|medium|9553506puff112567633|navy|Carton|Unknown|49|antiableeingable| +2826|AAAAAAAAIALAAAAA|2001-10-27||Always entire activities shall not grow all at a pieces. Running crowds ought to approach civ|9.50|4.65|6007001|edu packedu pack #1|4|athletic|4|Shoes|35|antipri|extra large|9553506puff112567633|smoke|Each|Unknown|10|callyableeingable| +2827|AAAAAAAALALAAAAA|1997-10-27||Important months sing then remaining ways; national tears seem other, com|0.95|0.66|9012008|importounivamalg #8|12|home repair|9|Books|583|prieinganti|N/A|7salmon4068100769002|gainsboro|Oz|Unknown|27|ationableeingable| +2828|AAAAAAAAMALAAAAA|1997-10-27|2000-10-26|Eyes must not stay natural clothes. Proceedings must not analyse mechanical ideas. Old, alternative pupils mi|4.47|2.50|2001001|amalgimporto #1|1|accessories|2|Men|555|antiantianti|petite|123835464yellow29109|peru|Bunch|Unknown|11|eingableeingable| +2829|AAAAAAAAMALAAAAA|2000-10-27||C|79.06|2.50|2001001|exportiedu pack #2|1|kids|4|Shoes|555|antiantianti|petite|123835464yellow29109|red|Tsp|Unknown|18|n stableeingable| +2830|AAAAAAAAOALAAAAA|1997-10-27|1999-10-27|Growing, civil practices may commit wrongly. Different, marine visitors would let.|2.52|2.26|8009001|maxinameless #1|9|optics|8|Sports|134|esepriought|N/A|4smoke78950807773653|seashell|Pallet|Unknown|20|barprieingable| +2831|AAAAAAAAOALAAAAA|1999-10-28|2001-10-26|Different, giant powers will appear therefore great, happy sounds. Models could come only possible, average objects. Then liable |8.22|2.26|3002002|importoexporti #2|2|infants|3|Children|134|esepriought|small|4smoke78950807773653|rosy|Each|Unknown|17|oughtprieingable| +2832|AAAAAAAAOALAAAAA|2001-10-27||Different, giant powers will appear therefore great, happy sounds. Models could come only possible, average objects. Then liable |2.18|1.59|3002002|maxiunivamalg #11|9|televisions|10|Electronics|134|esepriought|N/A|4smoke78950807773653|rose|Dram|Unknown|5|ableprieingable| +2833|AAAAAAAABBLAAAAA|1997-10-27||Consequences |4.59|2.24|5004002|edu packscholar #2|4|classical|5|Music|532|ableprianti|N/A|780580967turquoise57|papaya|Gram|Unknown|60|priprieingable| +2834|AAAAAAAACBLAAAAA|1997-10-27|2000-10-26|Years ought to eat past a advances. Beautiful, equal companies come long artistic ambitions. Services resume int|3.36|2.11|8011005|amalgmaxi #5|11|archery|8|Sports|151|oughtantiought|N/A|533193lavender103342|pink|Carton|Unknown|9|eseprieingable| +2835|AAAAAAAACBLAAAAA|2000-10-27||Working managers might not put eyes. A|9.34|2.11|8011005|amalgamalg #2|11|dresses|1|Women|192|ablen stought|large|365674670yellow83332|ivory|Carton|Unknown|8|antiprieingable| +2836|AAAAAAAAEBLAAAAA|1997-10-27|1999-10-27|Much different services used to share successfully. Forests realise so even substantial thousands. Again young streets drink possible, legal citizens. Once|0.62|0.50|3002001|importoexporti #1|2|infants|3|Children|94|esen st|petite|4662879922694sienna7|lace|Unknown|Unknown|3|callyprieingable| +2837|AAAAAAAAEBLAAAAA|1999-10-28|2001-10-26|Much different services used to share successfully. Forests realise so even substantial thousands. Again young streets drink possible, legal citizens. Once|9.42|7.53|3002001|importoscholar #2|2|country|5|Music|253|priantiable|N/A|2322795thistle668589|peach|Pallet|Unknown|21|ationprieingable| +2838|AAAAAAAAEBLAAAAA|2001-10-27||Much different services used to share successfully. Forests realise so even substantial thousands. Again young streets drink possible, legal citizens. Once|7.06|7.53|8004009|edu packnameless #9|2|camping|8|Sports|203|pribarable|N/A|2322795thistle668589|wheat|Each|Unknown|40|eingprieingable| +2839|AAAAAAAAHBLAAAAA|1997-10-27||Safe, sufficient representatives stop perhaps places. Functional, initial senses work wild women. Wonderful, commercial things u|0.95|0.56|10007001|brandunivamalg #1|7|personal|10|Electronics|257|ationantiable|N/A|32tomato223918119922|purple|Dozen|Unknown|9|n stprieingable| +2840|AAAAAAAAIBLAAAAA|1997-10-27|2000-10-26|Hour|4.81|3.99|5001001|amalgscholar #1|1|rock|5|Music|192|ablen stought|N/A|55541rosy43534275852|grey|Ton|Unknown|8|bareseeingable| +2841|AAAAAAAAIBLAAAAA|2000-10-27||Average, final chains shall exclude commentators. Urgently likely records put recent difficulties. Good, overseas parents might not see; memories may not receive both. Familiar |4.71|3.99|7004002|edu packbrand #2|4|curtains/drapes|7|Home|192|ablen stought|N/A|55541rosy43534275852|violet|Oz|Unknown|11|oughteseeingable| +2842|AAAAAAAAKBLAAAAA|1997-10-27|1999-10-27|Long, useful authorities would not decide subject delegates. Labour, open thoughts may explore. Relatively adult sections pursue rema|18.18|11.99|4004001|edu packedu pack #1|4|athletic|4|Shoes|235|antipriable|medium|73181213506188pale74|tan|N/A|Unknown|21|ableeseeingable| +2843|AAAAAAAAKBLAAAAA|1999-10-28|2001-10-26|Urgent pieces say stiff effective |2.15|11.99|5002002|importoscholar #2|4|country|5|Music|235|antipriable|N/A|8101smoke85454666959|tan|Box|Unknown|16|prieseeingable| +2844|AAAAAAAAKBLAAAAA|2001-10-27||Still open degrees shall count. Just european women facili|1.09|11.99|6002005|importocorp #5|4|diamonds|6|Jewelry|235|antipriable|N/A|896410808turquoise66|turquoise|Pallet|Unknown|39|eseeseeingable| +2845|AAAAAAAANBLAAAAA|1997-10-27||Police|75.92|31.88|9009002|maximaxi #2|9|science|9|Books|835|antiprieing|N/A|655927194973grey8791|white|Pound|Unknown|33|antieseeingable| +2846|AAAAAAAAOBLAAAAA|1997-10-27|2000-10-26|Alone parental holes go y|1.75|0.89|1001001|amalgamalg #1|1|dresses|1|Women|239|n stpriable|economy|206734029168indian09|powder|Oz|Unknown|74|callyeseeingable| +2847|AAAAAAAAOBLAAAAA|2000-10-27||Studies control tonight on a millions. Available, local facts know. Roads back hard as well extensive |5.58|0.89|1001001|importoamalgamalg #9|12|monitors|10|Electronics|239|n stpriable|N/A|206734029168indian09|lavender|Dozen|Unknown|11|ationeseeingable| +2848|AAAAAAAAACLAAAAA|1997-10-27|1999-10-27|Large, unfair eyes try instead leaders; nev|7.85|4.31|8005009|scholarnameless #9|5|fitness|8|Sports|235|antipriable|N/A|68196red463578385512|rose|Carton|Unknown|62|eingeseeingable| +2849|AAAAAAAAACLAAAAA|1999-10-28|2001-10-26|Cigarettes used to throw never; communists take very by the circumstances. All political proposals might weave clearly future, level presents. Too successful cases would rea|2.96|1.33|8005009|brandbrand #10|5|decor|7|Home|235|antipriable|N/A|68196red463578385512|tan|Box|Unknown|34|n steseeingable| +2850|AAAAAAAAACLAAAAA|2001-10-27||Normal buildings look over stable cheeks. Only, handsome types would not pay then goods. Countries appear supposedly |1.65|1.46|8005009|importoimporto #1|2|shirts|2|Men|405|antipriable|medium|1593313295056dodger7|violet|Tsp|Unknown|15|barantieingable| +2851|AAAAAAAADCLAAAAA|1997-10-27||Political, top victims make again still private members. Other, t|4.16|1.99|4001002|amalgedu pack #2|1|womens|4|Shoes|626|callyablecally|medium|86turquoise437490014|olive|Box|Unknown|4|oughtantieingable| +2852|AAAAAAAAECLAAAAA|1997-10-27|2000-10-26|Words use up a documents. Collections may|3.67|2.27|7004003|edu packbrand #3|4|curtains/drapes|7|Home|186|callyeingought|N/A|thistle0879365067678|puff|Oz|Unknown|62|ableantieingable| +2853|AAAAAAAAECLAAAAA|2000-10-27||Total, chief discussions could elect to a directors; alone electric initiatives like also new claims. Far views go final, pleased |4.12|3.62|6005006|scholarcorp #6|4|earings|6|Jewelry|131|callyeingought|N/A|38703331475khaki2674|sky|Pound|Unknown|32|priantieingable| +2854|AAAAAAAAGCLAAAAA|1997-10-27|1999-10-27|Schools could change carefully then national courses. Vaguely capable others shall not say right arms. Goals know still products. Agencies would not drop ahead |57.12|38.84|9008011|namelessmaxi #11|8|romance|9|Books|238|eingpriable|N/A|0455610gainsboro9464|purple|Tsp|Unknown|24|eseantieingable| +2855|AAAAAAAAGCLAAAAA|1999-10-28|2001-10-26|Only pink changes see certainly characteristics. F|0.25|0.16|7002006|importobrand #6|8|bedding|7|Home|452|ableantiese|N/A|0455610gainsboro9464|brown|Gross|Unknown|80|antiantieingable| +2856|AAAAAAAAGCLAAAAA|2001-10-27||Only pink changes see certainly characteristics. F|4.59|0.16|4001001|amalgedu pack #1|8|womens|4|Shoes|452|ableantiese|large|3427994793midnight54|pink|Bunch|Unknown|12|callyantieingable| +2857|AAAAAAAAJCLAAAAA|1997-10-27||Towns show personally asleep, additional women. Lines should catch words. True words will hit full, new months; in order general teams shall lead. Fairly apparent leaders re|4.79|2.10|5004002|edu packscholar #2|4|classical|5|Music|405|antibarese|N/A|1beige11931979490732|white|Ounce|Unknown|57|ationantieingable| +2858|AAAAAAAAKCLAAAAA|1997-10-27|2000-10-26|In addition blue feet feel. Ever real prices endanger at last only dramatic p|6.89|4.27|8008009|namelessnameless #9|8|outdoor|8|Sports|114|eseoughtought|N/A|027693mint2173161956|smoke|Gross|Unknown|14|eingantieingable| +2859|AAAAAAAAKCLAAAAA|2000-10-27||In addition blue feet feel. Ever real prices endanger at last only dramatic p|84.31|41.31|9016004|corpunivamalg #4|8|mystery|9|Books|114|eseoughtought|N/A|027693mint2173161956|misty|Tbl|Unknown|31|n stantieingable| +2860|AAAAAAAAMCLAAAAA|1997-10-27|1999-10-27|Considerable, real colleagues change. Seriously american letters know high differently systematic lists. Promptly major studies worry. Emotional features look. Soon chinese pages arr|6.48|2.13|9014005|edu packunivamalg #5|14|sports|9|Books|147|ationeseought|N/A|109203479086saddle69|lawn|Cup|Unknown|23|barcallyeingable| +2861|AAAAAAAAMCLAAAAA|1999-10-28|2001-10-26|Wide visible consequences must not prevent to a priorities. Traditionally yellow pieces must twist. New, other securities borrow then final arguments. Young walls should go by|3.20|2.78|6015006|scholarbrand #6|14|custom|6|Jewelry|658|ationeseought|N/A|21133tan723660973464|papaya|Bundle|Unknown|34|oughtcallyeingable| +2862|AAAAAAAAMCLAAAAA|2001-10-27||Wide visible consequences must not prevent to a priorities. Traditionally yellow pieces must twist. New, other securities borrow then final arguments. Young walls should go by|5.23|2.14|1001001|amalgamalg #1|14|dresses|1|Women|67|ationeseought|small|47plum91387785983752|light|Dozen|Unknown|59|ablecallyeingable| +2863|AAAAAAAAPCLAAAAA|1997-10-27||Different shops will hear far strong, physical purposes. Ages should g|3.91|2.54|7014004|edu packnameless #4|14|glassware|7|Home|209|n stbarable|N/A|7511481898metallic63|steel|N/A|Unknown|15|pricallyeingable| +2864|AAAAAAAAADLAAAAA|1997-10-27|2000-10-26|Conditions set persistently fresh contacts. Special, only women sing as. Organisations regard nice kinds. Stupid, other parties mention gently future, warm birds. Internal sorts|0.21|0.13|4001001|amalgedu pack #1|1|womens|4|Shoes|126|callyableought|small|plum0198250678150762|royal|Gram|Unknown|34|esecallyeingable| +2865|AAAAAAAAADLAAAAA|2000-10-27||New, ready times shall not pass international studies; financial years cannot s|28.46|0.13|4003002|exportiedu pack #2|3|kids|4|Shoes|126|callyableought|petite|plum0198250678150762|green|Pound|Unknown|11|anticallyeingable| +2866|AAAAAAAACDLAAAAA|1997-10-27|1999-10-27|Over potential items would look departments. The|7.72|3.78|10009011|maxiunivamalg #11|9|televisions|10|Electronics|88|eingeing|N/A|962smoke228292825120|peach|Box|Unknown|45|callycallyeingable| +2867|AAAAAAAACDLAAAAA|1999-10-28|2001-10-26|Bodies get so important sales. Unfortunately event|3.31|2.01|10009011|importobrand #4|9|costume|6|Jewelry|88|eingeing|N/A|962smoke228292825120|orchid|Case|Unknown|62|ationcallyeingable| +2868|AAAAAAAACDLAAAAA|2001-10-27||Bodies get so important sales. Unfortunately event|2.02|2.01|9006001|corpmaxi #1|6|parenting|9|Books|235|eingeing|N/A|01yellow214709336662|magenta|Dram|Unknown|14|eingcallyeingable| +2869|AAAAAAAAFDLAAAAA|1997-10-27||Belatedly public backs might see overall, net buildings. Services must like strong pa|4.81|3.55|6011006|amalgbrand #6|11|semi-precious|6|Jewelry|662|ablecallycally|N/A|10716royal5211344116|lace|Box|Unknown|12|n stcallyeingable| +2870|AAAAAAAAGDLAAAAA|1997-10-27|2000-10-26|Poor, certain materials meet so time and again delicious attitudes. Original, growing thanks could sweep no doubt bad, matu|7.46|6.19|4004001|edu packedu pack #1|4|athletic|4|Shoes|660|barcallycally|large|1457620powder0360736|spring|Tsp|Unknown|21|barationeingable| +2871|AAAAAAAAGDLAAAAA|2000-10-27||Poor, certain materials meet so time and again delicious attitudes. Original, growing thanks could sweep no doubt bad, matu|0.67|6.19|4004001|edu packedu pack #2|4|athletic|4|Shoes|530|barprianti|large|1457620powder0360736|chartreuse|Dozen|Unknown|7|oughtationeingable| +2872|AAAAAAAAIDLAAAAA|1997-10-27|1999-10-27|Probably european characters ought to think then darling rivals. Obvious premises can study r|1.35|0.45|1004001|edu packamalg #1|4|swimwear|1|Women|195|antin stought|small|purple51164693247081|purple|Box|Unknown|20|ableationeingable| +2873|AAAAAAAAIDLAAAAA|1999-10-28|2001-10-26|Probably european characters ought to think then darling rivals. Obvious premises can study r|1.66|0.99|1004002|edu packamalg #2|4|swimwear|1|Women|195|antin stought|medium|650904690sandy821220|peru|Each|Unknown|64|priationeingable| +2874|AAAAAAAAIDLAAAAA|2001-10-27||Breasts move most more economic hours. Expensive, open chiefs must not sell no longer mines. Tory councils will not reach similarly great, soviet images. Evolutionary, sma|4.66|0.99|1004002|edu packmaxi #5|14|tennis|8|Sports|195|antin stought|N/A|650904690sandy821220|lavender|Lb|Unknown|55|eseationeingable| +2875|AAAAAAAALDLAAAAA|1997-10-27||Signs receive in a rates; active days see to a weeks. So judicia|0.22|0.12|4004002|edu packedu pack #2|4|athletic|4|Shoes|459|n stantiese|small|70white1982175900696|misty|Bundle|Unknown|80|antiationeingable| +2876|AAAAAAAAMDLAAAAA|1997-10-27|2000-10-26|British, similar voters need. More wrong policies shall not see however at a hours. Other, occupational agreements s|2.55|2.11|6010005|univbrand #5|10|jewelry boxes|6|Jewelry|245|antieseable|N/A|chartreuse2995333661|goldenrod|Pallet|Unknown|16|callyationeingable| +2877|AAAAAAAAMDLAAAAA|2000-10-27||Hours should dispose more various lines; favorable, adult associations used to see nearly concentrations. New, able stores cannot give monetary trusts. Ill miles finish flat sections. Instruction|1.76|0.89|6010005|exportibrand #6|13|loose stones|6|Jewelry|319|n stoughtpri|N/A|4933salmon0097159837|steel|Unknown|Unknown|29|ationationeingable| +2878|AAAAAAAAODLAAAAA|1997-10-27|1999-10-27|Considerably odd officials expect also vulnerable voters. More fine users get already very, reasonable males. Well urban|9.87|4.83|3003001|exportiexporti #1|3|toddlers|3|Children|306|callybarpri|large|62361591394001rose62|olive|Dram|Unknown|46|eingationeingable| +2879|AAAAAAAAODLAAAAA|1999-10-28|2001-10-26|Familiar women concentrate tonight national passages. Important, difficult hands must combat genetic connections; institutions discuss then large men. Economic values make then new shareholders. Di|56.57|4.83|3003001|scholarcorp #8|5|earings|6|Jewelry|306|callybarpri|N/A|6667198699269sandy77|turquoise|Each|Unknown|65|n stationeingable| +2880|AAAAAAAAODLAAAAA|2001-10-27||National thousands could not repay gold, middle-class threats. Fine, old levels deal even now european holders. Markets get writers; social premises rest terms. Hu|2.46|0.76|3003001|importoedu pack #1|2|mens|4|Shoes|146|callyeseought|large|53712276orange445739|sandy|N/A|Unknown|62|bareingeingable| +2881|AAAAAAAABELAAAAA|1997-10-27||Sometimes major magazines avoid pointedly young publications. Police may show pretty subsequently possible objects. Expr|4.46|1.51|2004002|edu packimporto #2|4|sports-apparel|2|Men|141|oughteseought|medium|515thistle3144627946|yellow|Unknown|Unknown|3|oughteingeingable| +2882|AAAAAAAACELAAAAA|1997-10-27|2000-10-26|Yet old instruments hire. About other |1.03|0.36|3003001|exportiexporti #1|3|toddlers|3|Children|620|barablecally|large|3162129600grey269818|dodger|Dozen|Unknown|16|ableeingeingable| +2883|AAAAAAAACELAAAAA|2000-10-27||Key, left kinds determine even |1.84|1.56|9006004|corpmaxi #4|6|parenting|9|Books|620|barablecally|N/A|3162129600grey269818|tan|Oz|Unknown|31|prieingeingable| +2884|AAAAAAAAEELAAAAA|1997-10-27|1999-10-27|Years want as a whole. Public eyes shall win against a books. Special minutes intensify stones. Alone, right fingers spring men. Ho|1.73|0.77|7003005|exportibrand #5|3|kids|7|Home|47|ationese|N/A|07105809168793sandy1|grey|Gram|Unknown|49|eseeingeingable| +2885|AAAAAAAAEELAAAAA|1999-10-28|2001-10-26|Particular, single boys ought to light over labour years. Red parts shall make reasons. Just popular users feed wide arts. Governors call strongly nuclear things. Precise, royal chapters mak|8.16|6.12|7003005|edu packcorp #6|4|bracelets|6|Jewelry|47|ationese|N/A|07105809168793sandy1|powder|Pound|Unknown|19|antieingeingable| +2886|AAAAAAAAEELAAAAA|2001-10-27||Unexpected courts disappear in a statement|1.43|6.12|2002001|importoimporto #1|2|shirts|2|Men|701|ationese|extra large|8071515seashell53187|seashell|Pallet|Unknown|31|callyeingeingable| +2887|AAAAAAAAHELAAAAA|1997-10-27||Absolute,|9.52|5.52|5002002|importoscholar #2|2|country|5|Music|374|eseationpri|N/A|91800418580tan637512|maroon|Cup|Unknown|30|ationeingeingable| +2888|AAAAAAAAIELAAAAA|1997-10-27|2000-10-26|Possible, capable men see enough in a friends. Similar forces satisfy effects. Problems might compare n|1.50|1.00|3003001|exportiexporti #1|3|toddlers|3|Children|798|eingn station|large|5wheat47941038645616|white|Oz|Unknown|42|eingeingeingable| +2889|AAAAAAAAIELAAAAA|2000-10-27||Possible, capable men see enough in a friends. Similar forces satisfy effects. Problems might compare n|3.97|3.17|10002017|importounivamalg #17|3|camcorders|10|Electronics|798|eingn station|N/A|5wheat47941038645616|green|Box|Unknown|42|n steingeingable| +2890|AAAAAAAAKELAAAAA|1997-10-27|1999-10-27|Long young markets know for a owners. Developments say partial players. Social employees label for the problems. Also overseas times legislate. Top servi|4.95|3.31|5001001|amalgscholar #1|1|rock|5|Music|328|eingablepri|N/A|783782497gainsboro08|papaya|Pallet|Unknown|38|barn steingable| +2891|AAAAAAAAKELAAAAA|1999-10-28|2001-10-26|Modern weeks will kill predominantly visual scientists; con|0.92|3.31|4003002|exportiedu pack #2|3|kids|4|Shoes|328|eingablepri|medium|783782497gainsboro08|sandy|Case|Unknown|50|oughtn steingable| +2892|AAAAAAAAKELAAAAA|2001-10-27||Modern weeks will kill predominantly visual scientists; con|1.75|3.31|4003002|importoamalgamalg #3|12|monitors|10|Electronics|419|eingablepri|N/A|79602600cornflower75|khaki|Bunch|Unknown|16|ablen steingable| +2893|AAAAAAAANELAAAAA|1997-10-27||Brief drugs grant limited, voluntary teachers. Voluntarily flexible maps provide invariably in a responses. Oth|1.91|1.24|6003006|exporticorp #6|3|gold|6|Jewelry|253|priantiable|N/A|896366761499pale1060|sandy|Cup|Unknown|32|prin steingable| +2894|AAAAAAAAOELAAAAA|1997-10-27|2000-10-26|Competitors improve obviously as political police. By now new prisoners may arrive by a strings. Natural, short-term associations reduce so new cha|7.55|2.94|7002005|importobrand #5|2|bedding|7|Home|971|oughtationn st|N/A|66459877gainsboro656|pink|Tbl|Unknown|66|esen steingable| +2895|AAAAAAAAOELAAAAA|2000-10-27||Much ideal feet live relatively by a doubts. Only values leave recently for a strings. Guidelines guide really ot|44.82|18.37|8013006|exportimaxi #6|13|sailing|8|Sports|400|oughtationn st|N/A|66459877gainsboro656|orchid|Cup|Unknown|57|antin steingable| +2896|AAAAAAAAAFLAAAAA|1997-10-27|1999-10-27|Creditors should make as commercial states. Artificial organs can wait as normal, little eyes. Alternative hands know sacred lads. Users may investigate now. Successful terms play practically|4.06|1.90|7013007|exportinameless #7|13|wallpaper|7|Home|972|ableationn st|N/A|563045186866402rose9|saddle|Dram|Unknown|29|callyn steingable| +2897|AAAAAAAAAFLAAAAA|1999-10-28|2001-10-26|Distant characters concern after the aspects; however serious markets face regularly parents|7.03|3.44|8002008|importonameless #8|2|baseball|8|Sports|620|ableationn st|N/A|9126hot8862134698833|orange|Ton|Unknown|51|ationn steingable| +2898|AAAAAAAAAFLAAAAA|2001-10-27||Distant characters concern after the aspects; however serious markets face regularly parents|3.09|3.44|1002001|importoamalg #1|2|fragrances|1|Women|620|barablecally|medium|6927azure97908757902|hot|Tsp|Unknown|1|eingn steingable| +2899|AAAAAAAADFLAAAAA|1997-10-27||Only so-called managers provide as well good units. Numerous others think more certain observers. Political, specified individuals resist enthusiastic, unexpected companies. Hard makers st|1.36|1.03|10010010|univamalgamalg #10|10|memory|10|Electronics|282|ableeingable|N/A|3728312450cream51405|sandy|Tsp|Unknown|22|n stn steingable| +2900|AAAAAAAAEFLAAAAA|1997-10-27|2000-10-26|Issues find with the principles. Months move sharply players. Available diseases develop almost necessary, large facilities; fast|4.17|1.54|6009005|maxicorp #5|9|womens watch|6|Jewelry|109|n stbarought|N/A|14yellow368480450034|slate|Tbl|Unknown|23|barbarn stable| +2901|AAAAAAAAEFLAAAAA|2000-10-27||Issues find with the principles. Months move sharply players. Available diseases develop almost necessary, large facilities; fast|2.73|1.54|5001002|amalgscholar #2|1|rock|5|Music|109|n stbarought|N/A|14yellow368480450034|ghost|Lb|Unknown|45|oughtbarn stable| +2902|AAAAAAAAGFLAAAAA|1997-10-27|1999-10-27|Then local sets fail now agencies. Complete, new players will understand foreign groups. Strong views show already enthusiastic lines. Still new r|84.42|75.13|5003001|exportischolar #1|3|pop|5|Music|947|ationesen st|N/A|303973794735peach226|maroon|Box|Unknown|16|ablebarn stable| +2903|AAAAAAAAGFLAAAAA|1999-10-28|2001-10-26|Carers will not require often prime notes. F|5.58|1.89|7010004|univnameless #4|10|flatware|7|Home|947|ationesen st|N/A|303973794735peach226|rose|Dram|Unknown|34|pribarn stable| +2904|AAAAAAAAGFLAAAAA|2001-10-27||Carers will not require often prime notes. F|7.13|4.34|7010004|importoamalgamalg #15|10|monitors|10|Electronics|519|n stoughtanti|N/A|303973794735peach226|rosy|Dozen|Unknown|30|esebarn stable| +2905|AAAAAAAAJFLAAAAA|1997-10-27||Increasingly political sorts will win. Male trees must interfere yet possible terms. Local sides disclose only others. Very countries find edges. |71.39|29.26|2002002|importoimporto #2|2|shirts|2|Men|304|esebarpri|extra large|5614180662448sky7184|sienna|Tbl|Unknown|21|antibarn stable| +2906|AAAAAAAAKFLAAAAA|1997-10-27|2000-10-26|Everyd|1.79|1.28|9005003|scholarmaxi #3|5|history|9|Books|573|priationanti|N/A|1543541473288salmon4|red|Oz|Unknown|30|callybarn stable| +2907|AAAAAAAAKFLAAAAA|2000-10-27||H|1.52|0.82|9005003|amalgexporti #2|1|newborn|3|Children|573|priationanti|small|2493light80532049855|rose|Bunch|Unknown|7|ationbarn stable| +2908|AAAAAAAAMFLAAAAA|1997-10-27|1999-10-27|Effective things want now addre|87.48|31.49|3002001|importoexporti #1|2|infants|3|Children|28|eingable|medium|265090purple68008290|turquoise|Gross|Unknown|24|eingbarn stable| +2909|AAAAAAAAMFLAAAAA|1999-10-28|2001-10-26|Sufficient windows understand. Yet young men accept so. Totally willing effects make simply simple activities. Presently regional grants may not establish forward, selective birds; superb ru|5.93|31.49|3002001|brandcorp #6|7|pendants|6|Jewelry|239|eingable|N/A|1710223tan0606544611|rosy|Unknown|Unknown|64|n stbarn stable| +2910|AAAAAAAAMFLAAAAA|2001-10-27||Sufficient windows understand. Yet young men accept so. Totally willing effects make simply simple activities. Presently regional grants may not establish forward, selective birds; superb ru|1.42|31.49|3002001|brandmaxi #7|7|reference|9|Books|239|n stpriable|N/A|1710223tan0606544611|white|Box|Unknown|50|baroughtn stable| +2911|AAAAAAAAPFLAAAAA|1997-10-27||Clear factors shall win more electric lectures. Commercial, left colonies would go too good rare children. However easy thanks should change difficult, brown girls.|8.39|6.46|4001002|amalgedu pack #2|1|womens|4|Shoes|552|ableantianti|medium|7896051759283416red4|papaya|Tsp|Unknown|22|oughtoughtn stable| +2912|AAAAAAAAAGLAAAAA|1997-10-27|2000-10-26|Too only cars see already thus necessary feet; full streets keep more. Empirical details must not forgive then months. Shoes could not remain enough critical payments. Goa|5.55|2.66|6012001|importobrand #1|12|costume|6|Jewelry|644|eseesecally|N/A|2570860royal73449284|turquoise|Tbl|Unknown|33|ableoughtn stable| +2913|AAAAAAAAAGLAAAAA|2000-10-27||Events match only even common wives; grey, other periods may measure however. Royal, little options answer dimensions; very other circumstances can question usually now final activities; corners must |6.63|2.66|10013007|exportiamalgamalg #7|13|stereo|10|Electronics|6|cally|N/A|015942403905472pink5|navy|Cup|Unknown|53|prioughtn stable| +2914|AAAAAAAACGLAAAAA|1997-10-27|1999-10-27|Now full events should rain right. Matters will not write obvious, unlikely perceptions. Sure services treat often over important pr|4.33|2.94|8011005|amalgmaxi #5|11|archery|8|Sports|95|antin st|N/A|3440wheat95776320771|saddle|Case|Unknown|50|eseoughtn stable| +2915|AAAAAAAACGLAAAAA|1999-10-28|2001-10-26|Now full events should rain right. Matters will not write obvious, unlikely perceptions. Sure services treat often over important pr|9.97|2.94|1002002|importoamalg #2|11|fragrances|1|Women|514|eseoughtanti|large|1thistle118395154676|spring|Carton|Unknown|87|antioughtn stable| +2916|AAAAAAAACGLAAAAA|2001-10-27||Now full events should rain right. Matters will not write obvious, unlikely perceptions. Sure services treat often over important pr|1.46|0.91|3004001|edu packexporti #1|4|school-uniforms|3|Children|514|eseoughtanti|small|0445383178peach73137|chartreuse|Tsp|Unknown|61|callyoughtn stable| +2917|AAAAAAAAFGLAAAAA|1997-10-27||Plans could not bloc|7.08|2.26|10005011|scholarunivamalg #11|5|karoke|10|Electronics|125|antiableought|N/A|84294olive1987505193|lavender|Carton|Unknown|75|ationoughtn stable| +2918|AAAAAAAAGGLAAAAA|1997-10-27|2000-10-26|Perfect scholars |1.86|1.65|10006012|corpunivamalg #12|6|musical|10|Electronics|232|ablepriable|N/A|82345747298saddle866|hot|Ton|Unknown|7|eingoughtn stable| +2919|AAAAAAAAGGLAAAAA|2000-10-27||Only good victims cover only visual clothes; here uncertain needs see rights; jobs emerge from the stairs; local studen|5.29|1.65|2003002|exportiimporto #2|6|pants|2|Men|232|ablepriable|large|82345747298saddle866|smoke|Case|Unknown|20|n stoughtn stable| +2920|AAAAAAAAIGLAAAAA|1997-10-27|1999-10-27|Fresh systems say below likely, important flow|0.98|0.34|1003001|exportiamalg #1|3|maternity|1|Women|26|callyable|medium|97misty2025372823284|rose|Pound|Unknown|16|barablen stable| +2921|AAAAAAAAIGLAAAAA|1999-10-28|2001-10-26|Feet would wear by a words. Only realistic children must ment|3.71|3.26|1003001|exportimaxi #2|13|sailing|8|Sports|65|antically|N/A|97misty2025372823284|moccasin|Gram|Unknown|11|oughtablen stable| +2922|AAAAAAAAIGLAAAAA|2001-10-27||Yet bitter decisions exploit present, reasonable terms. Serious, narrow visitors must influence by a years. Quickly very conventions used to stop high|4.05|3.26|1001001|amalgamalg #1|1|dresses|1|Women|419|antically|large|97misty2025372823284|ghost|Cup|Unknown|23|ableablen stable| +2923|AAAAAAAALGLAAAAA|1997-10-27||Central requi|3.41|1.80|1001002|amalgamalg #2|1|dresses|1|Women|809|n stbareing|large|2cornsilk56002785606|yellow|N/A|Unknown|18|priablen stable| +2924|AAAAAAAAMGLAAAAA|1997-10-27|2000-10-26|Real, free metres might not survive so for|6.82|2.72|10010001|univamalgamalg #1|10|memory|10|Electronics|199|n stn stought|N/A|44553171seashell7571|seashell|Oz|Unknown|29|eseablen stable| +2925|AAAAAAAAMGLAAAAA|2000-10-27||Little, full workers involve pretty in a rates. Economic years used to s|1.31|0.73|2002002|importoimporto #2|10|shirts|2|Men|842|ableeseeing|small|44553171seashell7571|olive|Box|Unknown|89|antiablen stable| +2926|AAAAAAAAOGLAAAAA|1997-10-27|1999-10-27|Short tensions may not wait ever times. Coastal, ou|0.24|0.07|6003007|exporticorp #7|3|gold|6|Jewelry|184|eseeingought|N/A|740117915sky70960411|tomato|Dozen|Unknown|59|callyablen stable| +2927|AAAAAAAAOGLAAAAA|1999-10-28|2001-10-26|Wooden eyes see small |4.07|2.68|4004002|edu packedu pack #2|4|athletic|4|Shoes|208|eseeingought|small|740117915sky70960411|salmon|Ton|Unknown|65|ationablen stable| +2928|AAAAAAAAOGLAAAAA|2001-10-27||Wooden eyes see small |2.87|2.68|5002001|importoscholar #1|4|country|5|Music|918|eingoughtn st|N/A|59308white3332331014|hot|Gram|Unknown|51|eingablen stable| +2929|AAAAAAAABHLAAAAA|1997-10-27||Patiently good teachers might get now funny, dead |0.73|0.54|2004002|edu packimporto #2|4|sports-apparel|2|Men|533|priprianti|petite|1134lime458069319146|rose|Lb|Unknown|18|n stablen stable| +2930|AAAAAAAACHLAAAAA|1997-10-27|2000-10-26|Sad profits get independently with a women. Discussions drive schools. Then basic beliefs find generally traditionally funny sectors. French, certain lawyers would see. Good, black nations promote ex|9.53|3.04|7014001|edu packnameless #1|14|glassware|7|Home|23|priable|N/A|77115749012389lemon6|dodger|N/A|Unknown|8|barprin stable| +2931|AAAAAAAACHLAAAAA|2000-10-27||Only hands behave miners. Materials must wish; great, british words matter about a women. Gods restore good large colleges. Vulnerable teachers contribute now a|8.56|2.82|7014001|edu packcorp #4|14|bracelets|6|Jewelry|23|priable|N/A|77115749012389lemon6|snow|Bunch|Unknown|36|oughtprin stable| +2932|AAAAAAAAEHLAAAAA|1997-10-27|1999-10-27|Pretty bloody countr|6.45|3.93|7005003|scholarbrand #3|5|blinds/shades|7|Home|256|callyantiable|N/A|922770319910rosy6654|powder|N/A|Unknown|49|ableprin stable| +2933|AAAAAAAAEHLAAAAA|1999-10-28|2001-10-26|Pretty bloody countr|4.69|3.93|7005003|amalgamalg #2|5|dresses|1|Women|256|callyantiable|petite|6lavender36274002212|khaki|Pound|Unknown|45|priprin stable| +2934|AAAAAAAAEHLAAAAA|2001-10-27||Pretty bloody countr|27.88|3.93|1001001|amalgamalg #1|1|dresses|1|Women|256|callyantiable|medium|6lavender36274002212|light|Pound|Unknown|18|eseprin stable| +2935|AAAAAAAAHHLAAAAA|1997-10-27||Considerab|1.00|0.62|6013008|exportibrand #8|13|loose stones|6|Jewelry|107|ationbarought|N/A|5683569goldenrod4517|yellow|Oz|Unknown|42|antiprin stable| +2936|AAAAAAAAIHLAAAAA|1997-10-27|2000-10-26|Royal eyes |0.68|0.37|10009013|maxiunivamalg #13|9|televisions|10|Electronics|182|ableeingought|N/A|79steel9786250941878|rosy|Bunch|Unknown|11|callyprin stable| +2937|AAAAAAAAIHLAAAAA|2000-10-27||Words could wait then good, defensive thanks. Properly internal girls might not arrest ago for a objectives; brief, national lists put significantly here national clubs; regular stars may not put|82.08|47.60|1004002|edu packamalg #2|4|swimwear|1|Women|358|eingantipri|large|79steel9786250941878|snow|Case|Unknown|15|ationprin stable| +2938|AAAAAAAAKHLAAAAA|1997-10-27|1999-10-27|Of course complex conditions make mainly about pleasant children. Grounds hear reluctantly comparatively easy women; notes generate about for a facilities. Years silence in|2.06|1.75|5004001|edu packscholar #1|4|classical|5|Music|45|antiese|N/A|629921plum6703074517|lavender|Bundle|Unknown|56|eingprin stable| +2939|AAAAAAAAKHLAAAAA|1999-10-28|2001-10-26|Of course complex conditions make mainly about pleasant children. Grounds hear reluctantly comparatively easy women; notes generate about for a facilities. Years silence in|2.13|1.75|3004002|edu packexporti #2|4|school-uniforms|3|Children|45|antiese|extra large|629921plum6703074517|thistle|Unknown|Unknown|66|n stprin stable| +2940|AAAAAAAAKHLAAAAA|2001-10-27||Of course complex conditions make mainly about pleasant children. Grounds hear reluctantly comparatively easy women; notes generate about for a facilities. Years silence in|0.37|1.75|1001001|amalgamalg #1|1|dresses|1|Women|135|antipriought|small|7653687violet0314663|seashell|Pallet|Unknown|7|baresen stable| +2941|AAAAAAAANHLAAAAA|1997-10-27||Warm areas shall agree automatically mostly original pieces. Past domestic approaches post|3.72|1.97|9010002|univunivamalg #2|10|travel|9|Books|44|eseese|N/A|5212525518980saddle4|magenta|Bundle|Unknown|52|oughtesen stable| +2942|AAAAAAAAOHLAAAAA|1997-10-27|2000-10-26|High tasks will not leave again by means of a ladies. Glasses can represent. British, soviet months pull r|3.58|2.79|6001007|amalgcorp #7|1|birdal|6|Jewelry|193|prin stought|N/A|7050747royal91274796|puff|Pound|Unknown|41|ableesen stable| +2943|AAAAAAAAOHLAAAAA|2000-10-27||Societies revise so urban hours. Things should treat at a firms. Fresh, growing |3.19|2.79|6001007|edu packscholar #2|1|classical|5|Music|335|prin stought|N/A|8133peach34827541792|rose|Bundle|Unknown|99|priesen stable| +2944|AAAAAAAAAILAAAAA|1997-10-27|1999-10-27|Hands respond quickly heavy armies. Firms must reduce into a numbers; personal, british figures transfer entirely logi|3.17|1.71|8014005|edu packmaxi #5|14|tennis|8|Sports|75|antiation|N/A|violet89564523641919|rosy|Cup|Unknown|1|eseesen stable| +2945|AAAAAAAAAILAAAAA|1999-10-28|2001-10-26|Hands respond quickly heavy armies. Firms must reduce into a numbers; personal, british figures transfer entirely logi|9.17|5.68|8014005|maximaxi #6|9|science|9|Books|75|antiation|N/A|violet89564523641919|slate|Gram|Unknown|5|antiesen stable| +2946|AAAAAAAAAILAAAAA|2001-10-27||Again live vehicles blame yesterday soft depths. Indeed successful nations would provid|9.05|6.51|8014005|importobrand #3|9|costume|6|Jewelry|75|antiation|N/A|24882621056718steel3|tan|N/A|Unknown|55|callyesen stable| +2947|AAAAAAAADILAAAAA|1997-10-27||Cold countries may set. Small years become general changes. Rats must acc|8.76|5.43|5002002|importoscholar #2|2|country|5|Music|563|pricallyanti|N/A|thistle2835340362713|goldenrod|Oz|Unknown|89|ationesen stable| +2948|AAAAAAAAEILAAAAA|1997-10-27|2000-10-26|Substantial problems claim certai|1.26|0.84|1001001|amalgamalg #1|1|dresses|1|Women|578|eingationanti|petite|1805puff564126026517|ivory|Carton|Unknown|54|eingesen stable| +2949|AAAAAAAAEILAAAAA|2000-10-27||Substantial problems claim certai|0.95|0.84|1001001|namelessbrand #10|8|lighting|7|Home|112|ableoughtought|N/A|1805puff564126026517|grey|Ounce|Unknown|55|n stesen stable| +2950|AAAAAAAAGILAAAAA|1997-10-27|1999-10-27|There political guidelines must rise actually small new roads. Temperatures should not cry new victims. Very possible cal|3.68|1.43|7010001|univnameless #1|10|flatware|7|Home|463|pricallyese|N/A|4774706645linen83780|steel|Each|Unknown|4|barantin stable| +2951|AAAAAAAAGILAAAAA|1999-10-28|2001-10-26|There political guidelines must rise actually small new roads. Temperatures should not cry new victims. Very possible cal|81.93|1.43|7010001|exportimaxi #12|3|computers|9|Books|463|pricallyese|N/A|712633153red81011878|khaki|Pallet|Unknown|35|oughtantin stable| +2952|AAAAAAAAGILAAAAA|2001-10-27||There political guidelines must rise actually small new roads. Temperatures should not cry new victims. Very possible cal|5.31|1.43|7010001|edu packscholar #1|3|classical|5|Music|463|pricallyese|N/A|712633153red81011878|salmon|Tsp|Unknown|3|ableantin stable| +2953|AAAAAAAAJILAAAAA|1997-10-27||Objectives ought to let in short short levels. Industries exist within a examples. Papers will come inevitably again other musicians. Possible, sexual parts rise very effective to|8.78|3.07|8008004|namelessnameless #4|8|outdoor|8|Sports|208|eingbarable|N/A|pink6830518997439109|puff|Lb|Unknown|2|priantin stable| +2954|AAAAAAAAKILAAAAA|1997-10-27|2000-10-26|Outdoor regulations keep concerns. Kin|1.52|1.14|7015005|scholarnameless #5|15|tables|7|Home|537|ationprianti|N/A|4478931479violet1654|seashell|Case|Unknown|10|eseantin stable| +2955|AAAAAAAAKILAAAAA|2000-10-27||H|8.62|1.14|6016004|corpbrand #4|16|consignment|6|Jewelry|537|ationprianti|N/A|4478931479violet1654|sienna|Case|Unknown|11|antiantin stable| +2956|AAAAAAAAMILAAAAA|1997-10-27|1999-10-27|Well religious positions grow other, unusual obligations. Barely concerned colours will feel also other personal artists. Perhaps old libraries can chase in|0.60|0.32|6014005|edu packbrand #5|14|estate|6|Jewelry|245|antieseable|N/A|769943323seashell263|steel|Tbl|Unknown|16|callyantin stable| +2957|AAAAAAAAMILAAAAA|1999-10-28|2001-10-26|Young, c|18.48|0.32|6014005|amalgimporto #2|1|accessories|2|Men|245|antieseable|petite|769943323seashell263|linen|Ton|Unknown|18|ationantin stable| +2958|AAAAAAAAMILAAAAA|2001-10-27||References shall reverse for a tensions. Then critical surveys pass tight present lives. Personal contexts will clear either finan|8.15|0.32|8012009|importomaxi #9|12|guns|8|Sports|245|antieseable|N/A|93234241938forest911|sienna|Ton|Unknown|23|eingantin stable| +2959|AAAAAAAAPILAAAAA|1997-10-27||Detailed si|0.88|0.36|8001010|amalgnameless #10|1|athletic shoes|8|Sports|662|ablecallycally|N/A|04570577092peru15014|papaya|Pound|Unknown|40|n stantin stable| +2960|AAAAAAAAAJLAAAAA|1997-10-27|2000-10-26|English suggestions say model driv|2.37|0.82|1002001|importoamalg #1|2|fragrances|1|Women|303|pribarpri|extra large|91537988floral713902|ghost|Gross|Unknown|64|barcallyn stable| +2961|AAAAAAAAAJLAAAAA|2000-10-27||Invariably new goods want small, necessary surveys|6.35|3.93|6007002|brandcorp #2|7|pendants|6|Jewelry|542|ableeseanti|N/A|52yellow332615780853|seashell|Tbl|Unknown|91|oughtcallyn stable| +2962|AAAAAAAACJLAAAAA|1997-10-27|1999-10-27|Very huge councils will not stay elected, outstanding criticisms. Comfortable, financial rivers ought to follow on a men; children may not g|2.63|0.99|9008011|namelessmaxi #11|8|romance|9|Books|235|antipriable|N/A|3762022864rosy621411|violet|Case|Unknown|4|ablecallyn stable| +2963|AAAAAAAACJLAAAAA|1999-10-28|2001-10-26|Very huge councils will not stay elected, outstanding criticisms. Comfortable, financial rivers ought to follow on a men; children may not g|1.10|0.99|1003002|exportiamalg #2|8|maternity|1|Women|235|antipriable|medium|07152820plum63754622|olive|Gram|Unknown|8|pricallyn stable| +2964|AAAAAAAACJLAAAAA|2001-10-27||Very huge councils will not stay elected, outstanding criticisms. Comfortable, financial rivers ought to follow on a men; children may not g|7.00|2.87|10016007|corpamalgamalg #7|16|wireless|10|Electronics|235|antipriable|N/A|07152820plum63754622|papaya|Tsp|Unknown|22|esecallyn stable| +2965|AAAAAAAAFJLAAAAA|1997-10-27||Far useful preparations see almost central authorities. Social, economic areas can improve to a pupils. Nearly long days live. Agents get activities. Mysterious di|6.44|3.86|1002002|importoamalg #2|2|fragrances|1|Women|373|priationpri|large|69944772736mint34272|tan|Bunch|Unknown|53|anticallyn stable| +2966|AAAAAAAAGJLAAAAA|1997-10-27|2000-10-26|Patients say less. English, public deaths remember however now responsible needs. Serious laws make soon sympathetic units. Cities come tired stations. Products bring freely close|9.16|5.67|6013007|exportibrand #7|13|loose stones|6|Jewelry|234|esepriable|N/A|7linen74127729110158|pink|Dozen|Unknown|18|callycallyn stable| +2967|AAAAAAAAGJLAAAAA|2000-10-27||Patients say less. English, public deaths remember however now responsible needs. Serious laws make soon sympathetic units. Cities come tired stations. Products bring freely close|3.67|2.23|1004002|edu packamalg #2|4|swimwear|1|Women|349|esepriable|small|36701331393631snow00|peach|Pallet|Unknown|13|ationcallyn stable| +2968|AAAAAAAAIJLAAAAA|1997-10-27|1999-10-27|Wonderful borders could not make more national, active plans. Purely magnetic experiences move. Just other leaves shall pay. Dead, accessible sectio|6.42|3.33|6001001|amalgcorp #1|1|birdal|6|Jewelry|60|barcally|N/A|37rose14107825281948|rose|Dram|Unknown|48|eingcallyn stable| +2969|AAAAAAAAIJLAAAAA|1999-10-28|2001-10-26|Very languages stop profoundly equal forms. General villages review truly; basic crowds may sit quite possibly old shareholde|4.86|3.33|3001002|amalgexporti #2|1|newborn|3|Children|947|barcally|medium|6pink177139973104908|spring|Unknown|Unknown|11|n stcallyn stable| +2970|AAAAAAAAIJLAAAAA|2001-10-27||Hence pregnant passengers must not fly young movements. Children could not deter effectively officers. Quite active posts may return national buildings. Assessments set all w|74.26|60.15|3004001|edu packexporti #1|4|school-uniforms|3|Children|437|barcally|medium|6pink177139973104908|rosy|Carton|Unknown|33|barationn stable| +2971|AAAAAAAALJLAAAAA|1997-10-27||Students shall want. Competitive parents can need. Big, kind children should relax also potential interviews. As available offenders must not continue too random, econo|8.70|3.21|9009008|maximaxi #8|9|science|9|Books|354|eseantipri|N/A|2144751red6196488398|pale|Dram|Unknown|11|oughtationn stable| +2972|AAAAAAAAMJLAAAAA|1997-10-27|2000-10-26|Pure, high rivers come solely. Features cannot find important, relative payments; personal years could not feel. Quite commercial women must not say quite. Black trees state profits. Officers|4.94|3.95|6011005|amalgbrand #5|11|semi-precious|6|Jewelry|154|eseantiought|N/A|94371122red680760349|goldenrod|Case|Unknown|52|ableationn stable| +2973|AAAAAAAAMJLAAAAA|2000-10-27||Still assistant movements ought to become there great premises. Free e|4.47|3.95|6011005|edu packnameless #4|11|camping|8|Sports|536|callyprianti|N/A|94371122red680760349|royal|Dozen|Unknown|62|priationn stable| +2974|AAAAAAAAOJLAAAAA|1997-10-27|1999-10-27|Severe, strategic millions get with a guards. At least full duties ought to show in the policies. Everywhere particular dates prepare. Newspapers might receive|7.32|5.63|6005007|scholarcorp #7|5|earings|6|Jewelry|321|oughtablepri|N/A|95549884158peach2189|royal|Tsp|Unknown|76|eseationn stable| +2975|AAAAAAAAOJLAAAAA|1999-10-28|2001-10-26|Just other studies must embark agencies. Future, flexible pounds might change both living degrees. So tiny children join; long democrats must not offer in p|2.37|5.63|6005007|exporticorp #8|3|gold|6|Jewelry|321|oughtablepri|N/A|95549884158peach2189|sienna|Gram|Unknown|34|antiationn stable| +2976|AAAAAAAAOJLAAAAA|2001-10-27||Interesting, white circumstances should leave soon in a statements; unnecessary, radical studen|4.83|5.63|6005007|importoexporti #1|3|infants|3|Children|244|eseeseable|economy|95549884158peach2189|purple|Box|Unknown|37|callyationn stable| +2977|AAAAAAAABKLAAAAA|1997-10-27||Private, tiny questions take. Arms will cause circa. Levels shall enforce by the years. Points may perpetuate conversely by a ways. Medical organisations see |2.70|2.29|4002002|importoedu pack #2|2|mens|4|Shoes|349|n stesepri|medium|7708825386violet8200|spring|Lb|Unknown|15|ationationn stable| +2978|AAAAAAAACKLAAAAA|1997-10-27|2000-10-26|Early peasants include scientists; running minutes could tell probably close profits. However asleep awards might retain for a errors. Pounds include hardly even large individuals. Seriously rough i|3.30|1.38|4003001|exportiedu pack #1|3|kids|4|Shoes|91|oughtn st|petite|steel219436570973424|hot|Carton|Unknown|1|eingationn stable| +2979|AAAAAAAACKLAAAAA|2000-10-27||Human contacts consider home. Increased steps provide badly natural gardens. Social fears get calls. New, white contents will make. Still white officials shall call cons|1.76|0.88|1004002|edu packamalg #2|4|swimwear|1|Women|424|eseableese|small|saddle57534171054009|turquoise|Gross|Unknown|15|n stationn stable| +2980|AAAAAAAAEKLAAAAA|1997-10-27|1999-10-27|Classes could interfere more little eyes. Ridiculous, private plants move often |1.16|0.77|4004001|edu packedu pack #1|4|athletic|4|Shoes|899|n stn steing|medium|295482salmon97321844|rosy|Each|Unknown|19|bareingn stable| +2981|AAAAAAAAEKLAAAAA|1999-10-28|2001-10-26|Figures could not stay long certain examples. Numbers can follow. Children could not buy now important individuals. Either outstanding years remove in general typical divis|3.96|1.62|4004001|edu packunivamalg #6|4|sports|9|Books|280|bareingable|N/A|295482salmon97321844|sienna|Pallet|Unknown|40|oughteingn stable| +2982|AAAAAAAAEKLAAAAA|2001-10-27||Sudden servants play short, close parents; specific, illegal records generate less more known countries. Twin, creative meanings must take eagerly. Now sorry cl|54.97|31.33|4004001|exportiimporto #1|4|pants|2|Men|105|antibarought|medium|06611254lemon1420818|peach|Lb|Unknown|73|ableeingn stable| +2983|AAAAAAAAHKLAAAAA|1997-10-27||Purel|4.62|1.47|9011008|amalgunivamalg #8|11|cooking|9|Books|309|n stbarpri|N/A|32linen2899537977429|yellow|Bundle|Unknown|16|prieingn stable| +2984|AAAAAAAAIKLAAAAA|1997-10-27|2000-10-26|Electrical drivers will save only social seconds. Large, other tests test cold, other instruments. Simple, similar tests read cases. Enthusiastic, happy cases should not keep almost indian, t|1.40|0.85|3002001|importoexporti #1|2|infants|3|Children|295|antin stable|medium|0drab772724185148179|pink|Dram|Unknown|16|eseeingn stable| +2985|AAAAAAAAIKLAAAAA|2000-10-27||Patterns could allow with a thanks. Reactions pass slightly ashamed companies. Old, forward experiments could note small achievements. Different, soviet lips might wish different, pub|79.32|69.80|3002001|edu packmaxi #10|2|entertainments|9|Books|295|antin stable|N/A|0drab772724185148179|moccasin|Tbl|Unknown|12|antieingn stable| +2986|AAAAAAAAKKLAAAAA|1997-10-27|1999-10-27|British, new drugs get by a |3.59|1.50|2004001|edu packimporto #1|4|sports-apparel|2|Men|25|antiable|petite|yellow15465619774701|turquoise|Case|Unknown|7|callyeingn stable| +2987|AAAAAAAAKKLAAAAA|1999-10-28|2001-10-26|Considerable, public sides live by the instruments. Early, live aff|5.42|3.19|2004001|scholaramalgamalg #13|15|portable|10|Electronics|25|antiable|N/A|yellow15465619774701|rosy|Bundle|Unknown|55|ationeingn stable| +2988|AAAAAAAAKKLAAAAA|2001-10-27||Considerable, public sides live by the instruments. Early, live aff|8.83|6.71|2004001|namelesscorp #5|8|mens watch|6|Jewelry|70|baration|N/A|yellow15465619774701|thistle|Oz|Unknown|1|eingeingn stable| +2989|AAAAAAAANKLAAAAA|1997-10-27||Claims may care in a topics. Crucial, local years should mean often political members. Short, possible memories would get with a patients; natural contracts wa|0.25|0.15|3003002|exportiexporti #2|3|toddlers|3|Children|130|barpriought|petite|148287354613royal421|rosy|Tbl|Unknown|48|n steingn stable| +2990|AAAAAAAAOKLAAAAA|1997-10-27|2000-10-26|Standards visit to a opportunities; just redundant trades ascertain painfully local demands. Contributions want voices. Slight, certain beliefs fill better traditional problems. Above p|4.73|3.68|10011016|amalgamalgamalg #16|11|disk drives|10|Electronics|291|oughtn stable|N/A|30482steel5845757470|tan|Dozen|Unknown|30|barn stn stable| +2991|AAAAAAAAOKLAAAAA|2000-10-27||Female shareholders will maintain only old, head tickets. Also human lines must not |3.19|3.68|10011016|univunivamalg #4|11|travel|9|Books|291|oughtn stable|N/A|30482steel5845757470|peru|Gram|Unknown|50|oughtn stn stable| +2992|AAAAAAAAALLAAAAA|1997-10-27|1999-10-27|Actions provide into a reasons. Representatives keep artists. Again anxious investi|3.07|1.22|2003001|exportiimporto #1|3|pants|2|Men|911|oughtoughtn st|small|961457720397smoke766|khaki|Pound|Unknown|59|ablen stn stable| +2993|AAAAAAAAALLAAAAA|1999-10-28|2001-10-26|Actions provide into a reasons. Representatives keep artists. Again anxious investi|2.14|1.28|7001004|amalgbrand #4|3|bathroom|7|Home|310|baroughtpri|N/A|hot63998041851597071|medium|Each|Unknown|42|prin stn stable| +2994|AAAAAAAAALLAAAAA|2001-10-27||Worthwhile, d|7.61|2.89|1001001|amalgamalg #1|1|dresses|1|Women|310|baroughtpri|petite|247violet46819061718|snow|Pound|Unknown|42|esen stn stable| +2995|AAAAAAAADLLAAAAA|1997-10-27||Complete, good women mind promptly socialist societies. Clearly national provisions might allow more special, single skills. Be|5.57|3.00|2004002|edu packimporto #2|4|sports-apparel|2|Men|701|oughtbaration|petite|943853cornflower4925|plum|Gross|Unknown|65|antin stn stable| +2996|AAAAAAAAELLAAAAA|1997-10-27|2000-10-26|Great, british standards must raise entire words. Also smooth doctors demonstrate for a m|91.99|68.07|1002001|importoamalg #1|2|fragrances|1|Women|657|ationantically|petite|62thistle52459145275|peru|Oz|Unknown|37|callyn stn stable| +2997|AAAAAAAAELLAAAAA|2000-10-27||Great, british standards must raise entire words. Also smooth doctors demonstrate for a m|2.48|68.07|1002001|corpmaxi #10|2|parenting|9|Books|657|ationantically|N/A|62thistle52459145275|orchid|Ton|Unknown|61|ationn stn stable| +2998|AAAAAAAAGLLAAAAA|1997-10-27|1999-10-27|Departments could make also there personal customers; possible, unhappy statements construct. Running, comprehensive volumes leave therefore equally financial strangers. Public, english |9.65|5.98|4003001|exportiedu pack #1|3|kids|4|Shoes|164|esecallyought|medium|42turquoise178784937|salmon|Pallet|Unknown|9|eingn stn stable| +2999|AAAAAAAAGLLAAAAA||2001-10-26|Departments could make also there personal customers; possible, unhappy statements construct. Running, comprehensive volumes leave therefore equally financial strangers. Public, english ||||||glassware||||esecallyought||42turquoise178784937||Carton|Unknown|85|| +3000|AAAAAAAAGLLAAAAA|2001-10-27||Yet full budgets will not record new councillors; specialist, free wives will work tiles. Collections contribute almost |4.60|5.98|6004001|edu packcorp #1|3|bracelets|6|Jewelry|388|eingeingpri|N/A|16163085snow05045036|thistle|N/A|Unknown|18|barbarbarpri| +3001|AAAAAAAAJLLAAAAA|1997-10-27||Far widespread babies |98.24|70.73|5002002|importoscholar #2|2|country|5|Music|69|n stcally|N/A|683714359puff9299502|honeydew|Box|Unknown|38|oughtbarbarpri| +3002|AAAAAAAAKLLAAAAA|1997-10-27|2000-10-26|Labour items press un|2.75|0.96|1004001|edu packamalg #1|4|swimwear|1|Women|222|ableableable|large|508975181073linen350|cornsilk|Pound|Unknown|8|ablebarbarpri| +3003|AAAAAAAAKLLAAAAA|2000-10-27||Almost grey counties would account terms. Public others ask clearly great sites. Colonial studies could not play physically other children. However |2.68|0.96|1004001|exportiamalg #2|3|maternity|1|Women|187|ationeingought|medium|508975181073linen350|spring|N/A|Unknown|42|pribarbarpri| +3004|AAAAAAAAMLLAAAAA|1997-10-27|1999-10-27|There high houses live only educational troops. Quickly marve|3.26|2.21|7009005|maxibrand #5|9|mattresses|7|Home|937|ationprin st|N/A|16529997884967rose92|orchid|Dozen|Unknown|10|esebarbarpri| +3005|AAAAAAAAMLLAAAAA|1999-10-28|2001-10-26|Lovers help just basic rights. At lea|6.53|2.08|2003002|exportiimporto #2|3|pants|2|Men|587|ationprin st|economy|322800white289407469|pale|Oz|Unknown|30|antibarbarpri| +3006|AAAAAAAAMLLAAAAA|2001-10-27||Military reserves ought to see british, full events; whole objectives would continue largely regional, independent clothes. Large, multiple par|2.08|1.39|4003001|exportiedu pack #1|3|kids|4|Shoes|345|ationprin st|N/A|7738507tomato8833117|magenta|Lb|Unknown|2|callybarbarpri| +3007|AAAAAAAAPLLAAAAA|1997-10-27||Only|3.96|1.74|7007008|brandbrand #8|7|decor|7|Home|16|callyought|N/A|09236319seashell0086|snow|Tsp|Unknown|33|ationbarbarpri| +3008|AAAAAAAAAMLAAAAA|1997-10-27|2000-10-26|However thick blocks could not learn also medical fingers. Like, european |8.39|7.21|3004001|edu packexporti #1|4|school-uniforms|3|Children|532|ableprianti|extra large|757191thistle0395115|lemon|Gross|Unknown|53|eingbarbarpri| +3009|AAAAAAAAAMLAAAAA|2000-10-27||However thick blocks could not learn also medical fingers. Like, european |3.14|7.21|1003002|exportiamalg #2|4|maternity|1|Women|532|ableprianti|large|757191thistle0395115|turquoise|Tsp|Unknown|52|n stbarbarpri| +3010|AAAAAAAACMLAAAAA|1997-10-27|1999-10-27|Years make otherwise others. Windows accept. Black, contemporary appointments study |2.21|1.37|7014001|edu packnameless #1|14|glassware|7|Home|329|n stablepri|N/A|9364640648224peach37|steel|Pound|Unknown|5|baroughtbarpri| +3011|AAAAAAAACMLAAAAA|1999-10-28|2001-10-26|Years make otherwise others. Windows accept. Black, contemporary appointments study |1.04|0.92|1003002|exportiamalg #2|3|maternity|1|Women|239|n stpriable|small|4611964magenta378859|peru|Pallet|Unknown|25|oughtoughtbarpri| +3012|AAAAAAAACMLAAAAA|2001-10-27||Years could not risk surely unusual cultu|3.12|0.92|1003002|importoscholar #1|2|country|5|Music|239|n stpriable|N/A|54039306598376white1|blush|Lb|Unknown|19|ableoughtbarpri| +3013|AAAAAAAAFMLAAAAA|1997-10-27||Old, financial hours could |0.62|0.47|10003005|exportiunivamalg #5|3|dvd/vcr players|10|Electronics|419|n stoughtese|N/A|53330338878olive2298|tomato|Each|Unknown|38|prioughtbarpri| +3014|AAAAAAAAGMLAAAAA|1997-10-27|2000-10-26|Hard, proper plans must make birds. Academic homes should recognise. Goods may not obtain well|4.72|3.87|7006005|corpbrand #5|6|rugs|7|Home|313|prioughtpri|N/A|97668286smoke3315835|dim|Dozen|Unknown|12|eseoughtbarpri| +3015|AAAAAAAAGMLAAAAA|2000-10-27||Agents calculate less in the guns. Semantic, individual banks make however u|1.02|3.87|9014004|edu packunivamalg #4|6|sports|9|Books|313|prioughtpri|N/A|74pale32173028039869|blush|Ounce|Unknown|23|antioughtbarpri| +3016|AAAAAAAAIMLAAAAA|1997-10-27|1999-10-27|Able||1.07|||4|sports-apparel|2|Men|272||medium|0salmon2571504350106|sienna||Unknown||| +3017|AAAAAAAAIMLAAAAA|1999-10-28|2001-10-26|Even united failures sample olympic, m|8.48|1.07|2004001|amalgamalg #2|4|dresses|1|Women|272|ableationable|medium|0salmon2571504350106|smoke|Tbl|Unknown|7|ationoughtbarpri| +3018|AAAAAAAAIMLAAAAA|2001-10-27||Even united failures sample olympic, m|3.51|2.98|2002001|importoimporto #1|2|shirts|2|Men|272|ableationable|large|ghost547171190214906|metallic|Pound|Unknown|25|eingoughtbarpri| +3019|AAAAAAAALMLAAAAA|1997-10-27||Mechanical, bad areas co|1.60|0.75|2004002|edu packimporto #2|4|sports-apparel|2|Men|280|bareingable|extra large|88470270116snow82157|spring|Each|Unknown|16|n stoughtbarpri| +3020|AAAAAAAAMMLAAAAA|1997-10-27|2000-10-26|Reductions must make in order yet underlying partners. Streets|5.98|3.22|2001001|amalgimporto #1|1|accessories|2|Men|685|antieingcally|petite|7555029702913papaya3|sandy|Pound|Unknown|7|barablebarpri| +3021|AAAAAAAAMMLAAAAA|2000-10-27||Reductions must make in order yet underlying partners. Streets|5.17|2.74|2004002|edu packimporto #2|4|sports-apparel|2|Men|685|antieingcally|extra large|7555029702913papaya3|lavender|Cup|Unknown|9|oughtablebarpri| +3022|AAAAAAAAOMLAAAAA|1997-10-27|1999-10-27|Disabled relations express doubtfully common hours; very inappropriate ideas make bad, light theorie|28.84|19.32|9007011|brandmaxi #11|7|reference|9|Books|163|pricallyought|N/A|992108990518thistle4|violet|Oz|Unknown|73|ableablebarpri| +3023|AAAAAAAAOMLAAAAA|1999-10-28|2001-10-26|Disabled relations express doubtfully common hours; very inappropriate ideas make bad, light theorie|0.86|19.32|9007011|exportiedu pack #2|7|kids|4|Shoes|163|pricallyought|large|992939570656salmon14|metallic|Lb|Unknown|25|priablebarpri| +3024|AAAAAAAAOMLAAAAA|2001-10-27||Disabled relations express doubtfully common hours; very inappropriate ideas make bad, light theorie|87.30|54.12|9007011|edu packedu pack #1|4|athletic|4|Shoes|163|pricallyought|petite|992939570656salmon14|sienna|Tsp|Unknown|63|eseablebarpri| +3025|AAAAAAAABNLAAAAA|1997-10-27||Conscious, irish marks continue as so internal cells. Faint ships |1.27|0.53|3002002|importoexporti #2|2|infants|3|Children|345|antiesepri|large|680sienna80884776361|powder|Pound|Unknown|40|antiablebarpri| +3026|AAAAAAAACNLAAAAA|1997-10-27|2000-10-26|Active abilities depend smoothly by a |2.40|1.44|3002001|importoexporti #1|2|infants|3|Children|350|barantipri|petite|0755028998pink067196|pink|Ounce|Unknown|16|callyablebarpri| +3027|AAAAAAAACNLAAAAA|2000-10-27||Active abilities depend smoothly by a |0.22|0.08|3002001|brandnameless #8|2|hockey|8|Sports|350|barantipri|N/A|611368261529plum6622|honeydew|Ounce|Unknown|12|ationablebarpri| +3028|AAAAAAAAENLAAAAA|1997-10-27|1999-10-27|Adverse prayers promote open, main limitations. Women cou|4.08|3.26|7009009|maxibrand #9|9|mattresses|7|Home|10|barought|N/A|pale3800557037295102|puff|Cup|Unknown|8|eingablebarpri| +3029|AAAAAAAAENLAAAAA|1999-10-28|2001-10-26|Adverse prayers promote open, main limitations. Women cou|4.46|3.26|7005010|scholarbrand #10|9|blinds/shades|7|Home|10|barought|N/A|0056slate68973833926|midnight|Bunch|Unknown|65|n stablebarpri| +3030|AAAAAAAAENLAAAAA|2001-10-27||Later successful men can provide. At least other visitors may not run of course other, true lines. Less than afraid sports expand suddenly ethnic writings. Pictures search. Short progra|19.63|12.95|5002001|importoscholar #1|9|country|5|Music|10|barought|N/A|394590820057deep9831|tan|Pound|Unknown|28|barpribarpri| +3031|AAAAAAAAHNLAAAAA|1997-10-27||Local, national students sue seldom. Boys used to dismiss to the movements. Considerable, civil workers must clear eventually associated games. Helpful, practical hours shout.|37.48|14.24|1001002|amalgamalg #2|1|dresses|1|Women|647|ationesecally|large|52midnight8099175005|turquoise|Oz|Unknown|11|oughtpribarpri| +3032|AAAAAAAAINLAAAAA|1997-10-27|2000-10-26|Common colonies tell british, regular me|4.31|3.87|9012009|importounivamalg #9|12|home repair|9|Books|290|barn stable|N/A|66857832158pink66421|pink|Bunch|Unknown|65|ablepribarpri| +3033|AAAAAAAAINLAAAAA|2000-10-27||Suitable sheets take miserable factors. Later political memories used to study stages. Corporate, other scots appoint. Elements explore desirable, delicate titles. Of|6.92|5.88|9012009|importoexporti #2|12|infants|3|Children|519|barn stable|petite|61289625386salmon726|salmon|Case|Unknown|63|pripribarpri| +3034|AAAAAAAAKNLAAAAA|1997-10-27|1999-10-27|Following questions might take foreign boots. Finally white boundaries mu|1.68|1.36|8002005|importonameless #5|2|baseball|8|Sports|209|n stbarable|N/A|7665509999500lime541|powder|N/A|Unknown|95|esepribarpri| +3035|AAAAAAAAKNLAAAAA|1999-10-28|2001-10-26|However political programs would consider in a times. Areas will respond then for the grounds; still little arts would not lie only principally chief |9.96|1.36|8002005|scholarmaxi #12|5|history|9|Books|200|n stbarable|N/A|07152saddle976125350|rosy|Tsp|Unknown|42|antipribarpri| +3036|AAAAAAAAKNLAAAAA|2001-10-27||Common needs would not want daughters. Markets use mountains; administrative, entire costs hang basic|8.01|1.36|8002005|exportiimporto #1|3|pants|2|Men|200|n stbarable|medium|33416716orange960543|tan|Tsp|Unknown|91|callypribarpri| +3037|AAAAAAAANNLAAAAA|1997-10-27||Great|6.29|5.28|3002002|importoexporti #2|2|infants|3|Children|388|eingeingpri|petite|480033551508saddle38|lemon|Tsp|Unknown|24|ationpribarpri| +3038|AAAAAAAAONLAAAAA|1997-10-27|2000-10-26|Too public churches shall say importa|24.17|16.43|4002001|importoedu pack #1|2|mens|4|Shoes|273|priationable|medium|074946azure548858381|orange|Tbl|Unknown|9|eingpribarpri| +3039|AAAAAAAAONLAAAAA|2000-10-27||Too public churches shall say importa|8.35|16.43|2002002|importoimporto #2|2|shirts|2|Men|273|priationable|medium|074946azure548858381|puff|Pound|Unknown|30|n stpribarpri| +3040|AAAAAAAAAOLAAAAA|1997-10-27|1999-10-27|Near educational cases shall become big hotels. Periods should not|5.92|2.84|9011005|amalgunivamalg #5|11|cooking|9|Books|454|eseantiese|N/A|68914298010thistle08|wheat|Bunch|Unknown|46|baresebarpri| +3041|AAAAAAAAAOLAAAAA|1999-10-28|2001-10-26|Owners find normally so persistent doctors. National, vast lessons shall hang now con|5.38|2.84|4003002|exportiedu pack #2|11|kids|4|Shoes|182|ableeingought|N/A|9236171rosy937323150|sky|Gross|Unknown|32|oughtesebarpri| +3042|AAAAAAAAAOLAAAAA|2001-10-27||Rare officers might touch arms. Whole pictures serve modern flames; industrial hours expect once cool, concerned operators; bands find |5.03|1.60|4003002|exportiamalg #1|11|maternity|1|Women|665|anticallycally|petite|9236171rosy937323150|smoke|Lb|Unknown|63|ableesebarpri| +3043|AAAAAAAADOLAAAAA|1997-10-27||New r|2.10|1.19|2002002|importoimporto #2|2|shirts|2|Men|50|baranti|small|05927679steel6502176|violet|Unknown|Unknown|39|priesebarpri| +3044|AAAAAAAAEOLAAAAA|1997-10-27|2000-10-26|Annual, responsible players show reluctantly. Basic, early girls shall provide old years. Quick pages may not demand worth a transactions. Scientists form fully unable relationships.|4.44|1.77|5002001|importoscholar #1|2|country|5|Music|139|n stpriought|N/A|63542almond526376656|lace|Bunch|Unknown|87|eseesebarpri| +3045|AAAAAAAAEOLAAAAA|2000-10-27||More horrible legs follow very worried centres. Large, warm parts get surprisingly for a shares. Able thoughts should sleep. Long-term, enormous c|68.71|1.77|5002001|scholarunivamalg #3|5|karoke|10|Electronics|319|n stpriought|N/A|63542almond526376656|plum|Bundle|Unknown|10|antiesebarpri| +3046|AAAAAAAAGOLAAAAA|1997-10-27|1999-10-27|Rich managers used to proceed; therefore conservative models used to sell with a needs. Royal reasons ought to need cha|2.34|1.82|8015007|scholarmaxi #7|15|fishing|8|Sports|987|ationeingn st|N/A|90593honeydew2058351|rosy|Cup|Unknown|18|callyesebarpri| +3047|AAAAAAAAGOLAAAAA|1999-10-28|2001-10-26|Given costs date mont|6.57|5.19|7003008|exportibrand #8|15|kids|7|Home|462|ationeingn st|N/A|90593honeydew2058351|white|Bundle|Unknown|16|ationesebarpri| +3048|AAAAAAAAGOLAAAAA|2001-10-27||Earnings make moments. Secondary elections might not take thousands. Hardly great conclusions form seriously gentle preparations. Countries support hundreds; fields miss new functions. Adequate,|85.11|40.00|7003008|amalgscholar #1|1|rock|5|Music|462|ationeingn st|N/A|56169735094164snow02|red|Oz|Unknown|45|eingesebarpri| +3049|AAAAAAAAJOLAAAAA|1997-10-27||Communist, yellow objectives might come ever for a investors. Highly general users come again highest red councils. Problems welcome relatively therefore low units. Broadly close fac|3.04|1.55|4003002|exportiedu pack #2|3|kids|4|Shoes|453|priantiese|extra large|843260230goldenrod91|salmon|Pallet|Unknown|30|n stesebarpri| +3050|AAAAAAAAKOLAAAAA|1997-10-27|2000-10-26|Independent govern|3.77|2.07|6006003|corpcorp #3|6|rings|6|Jewelry|246|callyeseable|N/A|41178tomato828250815|pink|Lb|Unknown|29|barantibarpri| +3051|AAAAAAAAKOLAAAAA|2000-10-27||Independent govern|2.69|1.07|9003004|exportimaxi #4|6|computers|9|Books|246|callyeseable|N/A|41178tomato828250815|lemon|Gross|Unknown|25|oughtantibarpri| +3052|AAAAAAAAMOLAAAAA|1997-10-27|1999-10-27|Far, senior holidays boost however real troubles. Comprehensive women ask both anxious police; economic, financial products must land highly tiny times. Major, old years accept |5.41|4.86|1004001|edu packamalg #1|4|swimwear|1|Women|16|callyought|small|308320012turquoise37|pale|Ounce|Unknown|24|ableantibarpri| +3053|AAAAAAAAMOLAAAAA|1999-10-28|2001-10-26|Far, senior holidays boost however real troubles. Comprehensive women ask both anxious police; economic, financial products must land highly tiny times. Major, old years accept |1.40|0.84|1004001|amalgcorp #6|1|birdal|6|Jewelry|16|callyought|N/A|40437166light3114471|royal|Box|Unknown|50|priantibarpri| +3054|AAAAAAAAMOLAAAAA|2001-10-27||Far, senior holidays boost however real troubles. Comprehensive women ask both anxious police; economic, financial products must land highly tiny times. Major, old years accept |3.50|0.84|1004001|amalgscholar #1|1|rock|5|Music|468|eingcallyese|N/A|40437166light3114471|slate|Cup|Unknown|22|eseantibarpri| +3055|AAAAAAAAPOLAAAAA|1997-10-27||Different others say so today just terms. Occasional n|1.71|0.92|10010013|univamalgamalg #13|10|memory|10|Electronics|618|eingoughtcally|N/A|904424chartreuse7814|peru|Unknown|Unknown|16|antiantibarpri| +3056|AAAAAAAAAPLAAAAA|1997-10-27|2000-10-26|Rich, small ministers respect anything but national, religious points. Homes examine necessarily annual courses. Eyes talk perfectly. F|1.04|0.67|6006001|corpcorp #1|6|rings|6|Jewelry|268|eingcallyable|N/A|4pink599794408998986|deep|Unknown|Unknown|7|callyantibarpri| +3057|AAAAAAAAAPLAAAAA|2000-10-27||Away present children see perhaps recently political qualities. Black, faint proceedings interpret single men|4.12|3.66|8003008|exportinameless #8|6|basketball|8|Sports|268|eingcallyable|N/A|4pink599794408998986|white|Gross|Unknown|14|ationantibarpri| +3058|AAAAAAAACPLAAAAA|1997-10-27|1999-10-27|Particular resources keep variables. Twice old guns say social, particular decisions. Old poems suit fo|4.69|3.93|5004001|edu packscholar #1|4|classical|5|Music|565|anticallyanti|N/A|5109588110powder5288|olive|Tbl|Unknown|5|eingantibarpri| +3059|AAAAAAAACPLAAAAA|1999-10-28|2001-10-26|Artists can start usually canadian relatives. Foreign, great homes can go rapidly sometimes southern types. Persons will feed quite. Practical, |3.07|3.93|9014012|edu packunivamalg #12|4|sports|9|Books|660|anticallyanti|N/A|5109588110powder5288|spring|Case|Unknown|63|n stantibarpri| +3060|AAAAAAAACPLAAAAA|2001-10-27||Artists can start usually canadian relatives. Foreign, great homes can go rapidly sometimes southern types. Persons will feed quite. Practical, |8.60|6.79|9014012|edu packedu pack #1|4|athletic|4|Shoes|660|anticallyanti|large|80thistle36915345382|firebrick|Bunch|Unknown|23|barcallybarpri| +3061|AAAAAAAAFPLAAAAA|1997-10-27||General matters can imply then in terms of a operations. Eas|3.00|1.68|5001002|amalgscholar #2|1|rock|5|Music|176|callyationought|N/A|2545366357323wheat52|turquoise|Dozen|Unknown|20|oughtcallybarpri| +3062|AAAAAAAAGPLAAAAA|1997-10-27|2000-10-26|Various powers could negotiate no longer; english, other americans shall hurt uncertain, available years. Var|8.69|3.21|3002001|importoexporti #1|2|infants|3|Children|169|n stcallyought|extra large|goldenrod07094182183|mint|Tsp|Unknown|9|ablecallybarpri| +3063|AAAAAAAAGPLAAAAA|2000-10-27||Various powers could negotiate no longer; english, other americans shall hurt uncertain, available years. Var|3.28|1.41|3003002|exportiexporti #2|2|toddlers|3|Children|169|n stcallyought|small|654051049slate934512|pink|Tbl|Unknown|26|pricallybarpri| +3064|AAAAAAAAIPLAAAAA|1997-10-27|1999-10-27|Direct, related feet can think|86.38|42.32|4004001|edu packedu pack #1|4|athletic|4|Shoes|147|ationeseought|extra large|34309387527lemon5289|tan|Cup|Unknown|58|esecallybarpri| +3065|AAAAAAAAIPLAAAAA|1999-10-28|2001-10-26|Prices used to examine clearly home effective members. In short bad sums may not make chief prices. Duties hear commercial, mental indi|8.51|4.34|4004001|amalgedu pack #2|4|womens|4|Shoes|588|ationeseought|medium|34309387527lemon5289|cornsilk|Unknown|Unknown|55|anticallybarpri| +3066|AAAAAAAAIPLAAAAA|2001-10-27||Prices used to examine clearly home effective members. In short bad sums may not make chief prices. Duties hear commercial, mental indi|85.34|43.52|4004001|importoimporto #1|4|shirts|2|Men|158|ationeseought|medium|34309387527lemon5289|orange|Box|Unknown|25|callycallybarpri| +3067|AAAAAAAALPLAAAAA|1997-10-27||Methods shall take. Actually original daughters might not move. Top, insufficient sciences deliver quite by a voters. As combined le|76.02|32.68|3003002|exportiexporti #2|3|toddlers|3|Children|564|esecallyanti|small|papaya63375665566900|tan|Gross|Unknown|17|ationcallybarpri| +3068|AAAAAAAAMPLAAAAA|1997-10-27|2000-10-26|Discussions would not call designs. Labour reasons handle conservative, mean level|9.53|3.62|4001001|amalgedu pack #1|1|womens|4|Shoes|27|ationable|large|431280374888violet24|green|Pallet|Unknown|7|eingcallybarpri| +3069|AAAAAAAAMPLAAAAA|2000-10-27||Wet, supreme courts buy bishops|1.59|3.62|3002002|importoexporti #2|1|infants|3|Children|611|oughtoughtcally|large|0311blanched44502789|peru|Oz|Unknown|26|n stcallybarpri| +3070|AAAAAAAAOPLAAAAA|1997-10-27|1999-10-27|Active plants need necessary, widespread roads. Best back visits hold regularly fresh friend|0.32|0.25|3001001|amalgexporti #1|1|newborn|3|Children|548|eingeseanti|N/A|625150600240sienna84|plum|Pallet|Unknown|13|barationbarpri| +3071|AAAAAAAAOPLAAAAA|1999-10-28|2001-10-26|Active plants need necessary, widespread roads. Best back visits hold regularly fresh friend|41.73|0.25|3001001|exportiamalgamalg #12|13|stereo|10|Electronics|374|eseationpri|N/A|625150600240sienna84|purple|Unknown|Unknown|16|oughtationbarpri| +3072|AAAAAAAAOPLAAAAA|2001-10-27||Customs can see incidents; good cards should not miss again as the practices. Full fans tell soon busy cheeks. Cards come certain facts. Relevant, new walls cannot play largely in a structur|2.02|0.94|3001001|amalgbrand #3|1|bathroom|7|Home|374|eseationpri|N/A|625150600240sienna84|wheat|Tbl|Unknown|23|ableationbarpri| +3073|AAAAAAAABAMAAAAA|1997-10-27||Courts vary new, chinese weeks. B|84.72|56.76|8004004|edu packnameless #4|4|camping|8|Sports|673|priationcally|N/A|79316430799025lime94|smoke|Lb|Unknown|59|priationbarpri| +3074|AAAAAAAACAMAAAAA|1997-10-27|2000-10-26|High trustees would establish. Bands can shoot ago individual members. Also respective sides know proposed, full forms. Parents might happen consequently. Much good us|8.40|5.71|10010015|univamalgamalg #15|10|memory|10|Electronics|297|ationn stable|N/A|169189384334indian58|lime|Bundle|Unknown|15|eseationbarpri| +3075|AAAAAAAACAMAAAAA|2000-10-27||High trustees would establish. Bands can shoot ago individual members. Also respective sides know proposed, full forms. Parents might happen consequently. Much good us|1.91|1.03|3003002|exportiexporti #2|10|toddlers|3|Children|849|n steseeing|large|86387048violet453867|pale|Gram|Unknown|1|antiationbarpri| +3076|AAAAAAAAEAMAAAAA|1997-10-27|1999-10-27|Rates see quite into a terms. Gastric, good plants used to scrape to a hands. High abilities sell secondly years. Trying, british practices need long, small situations. Indeed s|1.24|0.59|4003001|exportiedu pack #1|3|kids|4|Shoes|202|ablebarable|economy|09311261083798brown2|peru|Bundle|Unknown|79|callyationbarpri| +3077|AAAAAAAAEAMAAAAA|1999-10-28|2001-10-26|Rates see quite into a terms. Gastric, good plants used to scrape to a hands. High abilities sell secondly years. Trying, british practices need long, small situations. Indeed s|2.71|2.14|4003001|namelesscorp #6|8|mens watch|6|Jewelry|564|ablebarable|N/A|09311261083798brown2|powder|Tbl|Unknown|44|ationationbarpri| +3078|AAAAAAAAEAMAAAAA|2001-10-27||Rates see quite into a terms. Gastric, good plants used to scrape to a hands. High abilities sell secondly years. Trying, british practices need long, small situations. Indeed s|1.71|0.92|10001002|amalgunivamalg #2|1|cameras|10|Electronics|564|esecallyanti|N/A|26475693002073pale48|spring|Each|Unknown|48|eingationbarpri| +3079|AAAAAAAAHAMAAAAA|1997-10-27||S|4.91|3.73|2003002|exportiimporto #2|3|pants|2|Men|119|n stoughtought|petite|283783939695royal108|thistle|Dozen|Unknown|1|n stationbarpri| +3080|AAAAAAAAIAMAAAAA|1997-10-27|2000-10-26|Alone walls mus|2.00|1.28|9003009|exportimaxi #9|3|computers|9|Books|18|eingought|N/A|604481739409peach672|plum|Carton|Unknown|83|bareingbarpri| +3081|AAAAAAAAIAMAAAAA|2000-10-27||Alone walls mus|1.67|0.91|9003009|exportiexporti #2|3|toddlers|3|Children|622|eingought|medium|604481739409peach672|papaya|Bunch|Unknown|8|oughteingbarpri| +3082|AAAAAAAAKAMAAAAA|1997-10-27|1999-10-27|So popular men tend there possible versions. Key, t|3.37|1.88|4003001|exportiedu pack #1|3|kids|4|Shoes|366|callycallypri|economy|131670313068938sky39|navy|Unknown|Unknown|19|ableeingbarpri| +3083|AAAAAAAAKAMAAAAA|1999-10-28|2001-10-26|So popular men tend there possible versions. Key, t|1.18|0.84|6007004|brandcorp #4|7|pendants|6|Jewelry|7|callycallypri|N/A|101216860626rosy8723|pale|Carton|Unknown|16|prieingbarpri| +3084|AAAAAAAAKAMAAAAA|2001-10-27||So popular men tend there possible versions. Key, t|0.19|0.84|6007004|importoscholar #1|7|country|5|Music|29|n stable|N/A|101216860626rosy8723|sienna|Dozen|Unknown|2|eseeingbarpri| +3085|AAAAAAAANAMAAAAA|1997-10-27||Always common pie|2.76|2.26|1002002|importoamalg #2|2|fragrances|1|Women|173|priationought|large|20845017370wheat7224|red|Pallet|Unknown|12|antieingbarpri| +3086|AAAAAAAAOAMAAAAA|1997-10-27|2000-10-26|Professional interests cannot accept necessarily. Settlements cook cheap h|1.98|1.76|8014007|edu packmaxi #7|14|tennis|8|Sports|631|oughtprically|N/A|617peru7092798458677|rosy|Tsp|Unknown|6|callyeingbarpri| +3087|AAAAAAAAOAMAAAAA|2000-10-27||Professional interests cannot accept necessarily. Settlements cook cheap h|0.09|0.04|8014007|scholarbrand #8|14|custom|6|Jewelry|631|oughtprically|N/A|617peru7092798458677|rosy|Tsp|Unknown|60|ationeingbarpri| +3088|AAAAAAAAABMAAAAA|1997-10-27|1999-10-27|Sexual yards hang rapidly enough constant opponents. Accounts should control; balls see years. Economic scales divide then by a farm|8.21|5.25|3003001|exportiexporti #1|3|toddlers|3|Children|51|oughtanti|petite|888052000394rosy2243|khaki|Pallet|Unknown|18|eingeingbarpri| +3089|AAAAAAAAABMAAAAA|1999-10-28|2001-10-26|Sexual yards hang rapidly enough constant opponents. Accounts should control; balls see years. Economic scales divide then by a farm|2.74|1.78|1001002|amalgamalg #2|3|dresses|1|Women|300|barbarpri|medium|86159020violet012979|rosy|Tsp|Unknown|19|n steingbarpri| +3090|AAAAAAAAABMAAAAA|2001-10-27||Sexual yards hang rapidly enough constant opponents. Accounts should control; balls see years. Economic scales divide then by a farm|16.34|13.23|1001002|amalgedu pack #1|3|womens|4|Shoes|514|eseoughtanti|petite|86159020violet012979|tomato|Gram|Unknown|15|barn stbarpri| +3091|AAAAAAAADBMAAAAA|1997-10-27||So right intentions work authorities. Certain others could lie then external goals. Characters should see; almost likely o|5.24|2.51|8005002|scholarnameless #2|5|fitness|8|Sports|324|eseablepri|N/A|176smoke476499518978|puff|Pallet|Unknown|44|oughtn stbarpri| +3092|AAAAAAAAEBMAAAAA|1997-10-27|2000-10-26|Always orange cases should insist exciting fears. Overseas things will see. Bedrooms shall not define just in a books. Informal, close eyes could benefit awards; now curio|1.73|0.86|4001001|amalgedu pack #1|1|womens|4|Shoes|139|n stpriought|N/A|05992deep42685413248|wheat|Gram|Unknown|20|ablen stbarpri| +3093|AAAAAAAAEBMAAAAA|2000-10-27||Always orange cases should insist exciting fears. Overseas things will see. Bedrooms shall not define just in a books. Informal, close eyes could benefit awards; now curio|4.65|0.86|4001001|edu packexporti #2|1|school-uniforms|3|Children|366|n stpriought|small|45510481424slate5587|moccasin|Each|Unknown|46|prin stbarpri| +3094|AAAAAAAAGBMAAAAA|1997-10-27|1999-10-27|Services may|2.27|1.13|6008007|namelesscorp #7|8|mens watch|6|Jewelry|317|ationoughtpri|N/A|3powder1592063050655|peach|Box|Unknown|11|esen stbarpri| +3095|AAAAAAAAGBMAAAAA|1999-10-28|2001-10-26|Services may|6.15|2.82|9003012|exportimaxi #12|8|computers|9|Books|317|ationoughtpri|N/A|3powder1592063050655|peru|Box|Unknown|31|antin stbarpri| +3096|AAAAAAAAGBMAAAAA|2001-10-27||So close causes could not pay already subsequent parties; political |3.54|2.82|3001001|amalgexporti #1|1|newborn|3|Children|126|callyableought|extra large|400tomato45568175172|steel|Unknown|Unknown|8|callyn stbarpri| +3097|AAAAAAAAJBMAAAAA|1997-10-27||Old glasses know other, new details. Black months result. Others might evoke again less parliamentary countries. Families rise for a businesses. Poles share |3.67|2.45|3004002|edu packexporti #2|4|school-uniforms|3|Children|387|ationeingpri|medium|56purple506987514546|thistle|Gross|Unknown|52|ationn stbarpri| +3098|AAAAAAAAKBMAAAAA|1997-10-27|2000-10-26|Activities want with the walls. Faint, main minutes stand as activiti|0.54|0.42|3001001|amalgexporti #1|1|newborn|3|Children|159|n stantiought|medium|90817172thistle51196|snow|Carton|Unknown|29|eingn stbarpri| +3099|AAAAAAAAKBMAAAAA|2000-10-27||Activities want with the walls. Faint, main minutes stand as activiti|4.37|0.42|10013006|exportiamalgamalg #6|13|stereo|10|Electronics|159|n stantiought|N/A|207seashell831789723|sienna|Cup|Unknown|16|n stn stbarpri| +3100|AAAAAAAAMBMAAAAA|1997-10-27|1999-10-27|Levels ought to know late pensions; often blue years say clearly marginal, late months. Agricultural, large rooms compare only very standard studies; attractive, sma|1.75|1.50|5001001|amalgscholar #1|1|rock|5|Music|100|barbarought|N/A|8361118gainsboro8465|slate|Pallet|Unknown|5|barbaroughtpri| +3101|AAAAAAAAMBMAAAAA|1999-10-28|2001-10-26|Immediately other fields stand then by a matters. Both big men can go as yet away important needs. Big traditions will arrive very about the experiments. Interesting, relevant claims affect rather fin|4.89|1.50|4004002|edu packedu pack #2|1|athletic|4|Shoes|354|barbarought|medium|8361118gainsboro8465|midnight|Each|Unknown|89|oughtbaroughtpri| +3102|AAAAAAAAMBMAAAAA|2001-10-27||Early major pieces cannot take also again surpris|5.20|1.50|5004001|edu packscholar #1|1|classical|5|Music|313|barbarought|N/A|9526chocolate8542601|purple|Carton|Unknown|28|ablebaroughtpri| +3103|AAAAAAAAPBMAAAAA|1997-10-27||Yea|1.36|0.92|7011004|amalgnameless #4|11|accent|7|Home|580|bareinganti|N/A|0salmon8002989681141|puff|Each|Unknown|12|pribaroughtpri| +3104|AAAAAAAAACMAAAAA|1997-10-27|2000-10-26|Else old brothers eat as jointly necessar|3.91|3.08|2002001|importoimporto #1|2|shirts|2|Men|134|esepriought|medium|96552391916238green3|indian|Gram|Unknown|8|esebaroughtpri| +3105|AAAAAAAAACMAAAAA|2000-10-27||For instance similar theories ought to go at a schemes. Always political children may not hold enough blind regular hands. Minutes will know with a trades;|5.75|3.08|6002002|importocorp #2|2|diamonds|6|Jewelry|559|n stantianti|N/A|4196385steel84506322|orchid|Unknown|Unknown|41|antibaroughtpri| +3106|AAAAAAAACCMAAAAA|1997-10-27|1999-10-27|Illegal processes need social, true wives. Australian, new deputies |4.45|1.55|10006013|corpunivamalg #13|6|musical|10|Electronics|315|antioughtpri|N/A|3001348636350plum363|smoke|Dozen|Unknown|80|callybaroughtpri| +3107|AAAAAAAACCMAAAAA|1999-10-28|2001-10-26|Illegal processes need social, true wives. Australian, new deputies |7.10|3.33|10006013|amalgedu pack #2|6|womens|4|Shoes|315|antioughtpri|economy|3001348636350plum363|cornsilk|Ounce|Unknown|59|ationbaroughtpri| +3108|AAAAAAAACCMAAAAA|2001-10-27||Aspects remove etc right parties. Social police formulate as. Cheerful, grey profits would state crucial books; |0.65|3.33|10006013|amalgmaxi #1|6|arts|9|Books|315|antioughtpri|N/A|3001348636350plum363|olive|Gross|Unknown|61|eingbaroughtpri| +3109|AAAAAAAAFCMAAAAA|1997-10-27||At all liberal countries cannot sum overnight full houses. At least new films release as a cups. Conce|7.39|6.35|3001002|amalgexporti #2|1|newborn|3|Children|759|n stantiation|petite|4290047974royal80743|moccasin|Carton|Unknown|11|n stbaroughtpri| +3110|AAAAAAAAGCMAAAAA|1997-10-27|2000-10-26|For |0.31|0.19|1001001|amalgamalg #1|1|dresses|1|Women|365|anticallypri|medium|8568829steel95609076|slate|Ounce|Unknown|22|baroughtoughtpri| +3111|AAAAAAAAGCMAAAAA|2000-10-27||For |42.12|0.19|3002002|importoexporti #2|2|infants|3|Children|365|anticallypri|medium|8568829steel95609076|spring|Case|Unknown|65|oughtoughtoughtpri| +3112|AAAAAAAAICMAAAAA|1997-10-27|1999-10-27|Then other hands conflict never early terms. Active, direct ministers shall determine. Ethnic procedures lose just. Large feet try later parents. Far likely minutes |1.46|0.48|4002001|importoedu pack #1|2|mens|4|Shoes|36|callypri|medium|345yellow76446060680|honeydew|Ton|Unknown|63|ableoughtoughtpri| +3113|AAAAAAAAICMAAAAA|1999-10-28|2001-10-26|Wee|8.55|0.48|2002002|importoimporto #2|2|shirts|2|Men|36|callypri|small|011peach414843309925|sandy|N/A|Unknown|14|prioughtoughtpri| +3114|AAAAAAAAICMAAAAA|2001-10-27||Wee|1.93|0.48|9011007|amalgunivamalg #7|11|cooking|9|Books|36|callypri|N/A|navajo01832356671232|purple|Carton|Unknown|30|eseoughtoughtpri| +3115|AAAAAAAALCMAAAAA|1997-10-27||Human, bad girls shall combine then such as the proposals. Sometimes enormous cus|87.01|54.81|6010004|univbrand #4|10|jewelry boxes|6|Jewelry|391|oughtn stpri|N/A|7010green86745384494|white|Bunch|Unknown|20|antioughtoughtpri| +3116|AAAAAAAAMCMAAAAA|1997-10-27|2000-10-26|Attempts feel; extre|3.41|2.69|5003001|exportischolar #1|3|pop|5|Music|224|eseableable|N/A|3265659478040rosy322|wheat|Pallet|Unknown|8|callyoughtoughtpri| +3117|AAAAAAAAMCMAAAAA|2000-10-27||Attempts feel; extre|9.20|3.95|8012008|importomaxi #8|3|guns|8|Sports|246|eseableable|N/A|3265659478040rosy322|olive|Bunch|Unknown|64|ationoughtoughtpri| +3118|AAAAAAAAOCMAAAAA|1997-10-27|1999-10-27|Possible artists think in a forms. Central, free rights should take outside weeks. Fine reports will find away. Upper, statistical cars appreciate out of a papers. Overseas words may wait only cases; |1.60|0.56|6016007|corpbrand #7|16|consignment|6|Jewelry|754|eseantiation|N/A|smoke373483227608686|green|Pallet|Unknown|20|eingoughtoughtpri| +3119|AAAAAAAAOCMAAAAA|1999-10-28|2001-10-26|Le|6.47|0.56|2004002|edu packimporto #2|16|sports-apparel|2|Men|231|eseantiation|large|48303412powder422342|pale|Dozen|Unknown|39|n stoughtoughtpri| +3120|AAAAAAAAOCMAAAAA|2001-10-27||Unique programmes must not stuff on a privileges; ideas ought to continue however however financial theories. Full stars could overwhelm exactly. Terribly reasonable patients kn|1.01|0.56|2004002|importoimporto #1|2|shirts|2|Men|231|oughtpriable|medium|48303412powder422342|snow|Ounce|Unknown|9|barableoughtpri| +3121|AAAAAAAABDMAAAAA|1997-10-27||Brief, main shareholders give. Sites help higher. Grey resources roll usually clear, true corners. Balls claim. Pe|4.27|3.15|4001002|amalgedu pack #2|1|womens|4|Shoes|306|callybarpri|medium|415693641036291pink2|rose|Bunch|Unknown|27|oughtableoughtpri| +3122|AAAAAAAACDMAAAAA|1997-10-27|2000-10-26|Successful varieties would not discuss points. Short lovely models must not organise faster. Successive, familiar officers use|9.70|5.91|10002012|importounivamalg #12|2|camcorders|10|Electronics|968|eingcallyn st|N/A|628700944718yellow82|hot|Cup|Unknown|1|ableableoughtpri| +3123|AAAAAAAACDMAAAAA|2000-10-27||Successful varieties would not discuss points. Short lovely models must not organise faster. Successive, familiar officers use|7.60|4.40|9008004|namelessmaxi #4|8|romance|9|Books|968|eingcallyn st|N/A|867444773powder16024|tomato|Box|Unknown|11|priableoughtpri| +3124|AAAAAAAAEDMAAAAA|1997-10-27|1999-10-27|Books understand. Principles produce just at a premises. Years |44.48|34.24|9003005|exportimaxi #5|3|computers|9|Books|636|callyprically|N/A|36459877184spring831|turquoise|Pallet|Unknown|2|eseableoughtpri| +3125|AAAAAAAAEDMAAAAA|1999-10-28|2001-10-26|Books understand. Principles produce just at a premises. Years |56.43|34.24|9003005|amalgamalg #2|3|dresses|1|Women|636|callyprically|small|2440khaki86552455240|sky|Oz|Unknown|33|antiableoughtpri| +3126|AAAAAAAAEDMAAAAA|2001-10-27||Publications shall not assume home u|4.50|34.24|9003005|exportinameless #7|3|basketball|8|Sports|636|callyprically|N/A|2440khaki86552455240|smoke|Bunch|Unknown|27|callyableoughtpri| +3127|AAAAAAAAHDMAAAAA|1997-10-27||Capable, alleged families mean long english ter|0.47|0.23|10014017|edu packamalgamalg #17|14|automotive|10|Electronics|263|pricallyable|N/A|53404942677rose58048|snow|Pallet|Unknown|23|ationableoughtpri| +3128|AAAAAAAAIDMAAAAA|1997-10-27|2000-10-26|Major lines establish too conditions. Softly rural teachers ought to offend essential circumstances. Main, large officers get und|5.55|2.55|5003001|exportischolar #1|3|pop|5|Music|166|callycallyought|N/A|095856163204tomato60|plum|Bundle|Unknown|11|eingableoughtpri| +3129|AAAAAAAAIDMAAAAA|2000-10-27||Crucial restaurants make for a children. Too united results begin effectively results. Others feel only in a collections. Aware pp. go simply. Neighbours mig|7.53|3.16|5003001|univmaxi #10|3|pools|8|Sports|166|callycallyought|N/A|249863994942yellow72|cornflower|Bundle|Unknown|3|n stableoughtpri| +3130|AAAAAAAAKDMAAAAA|1997-10-27|1999-10-27|Rapid, short authoritie|6.68|4.34|1002001|importoamalg #1|2|fragrances|1|Women|249|n steseable|medium|76656272172sandy5666|orchid|Case|Unknown|22|barprioughtpri| +3131|AAAAAAAAKDMAAAAA|1999-10-28|2001-10-26|Simply previous goals should contain below to a members. Chief, suitable visitors talk automatically famous facilities. Specifically good communities will select here only distinctive bases. G|8.20|7.29|1002001|exportibrand #2|3|kids|7|Home|360|n steseable|N/A|76656272172sandy5666|thistle|Pound|Unknown|7|oughtprioughtpri| +3132|AAAAAAAAKDMAAAAA|2001-10-27||Simply previous goals should contain below to a members. Chief, suitable visitors talk automatically famous facilities. Specifically good communities will select here only distinctive bases. G|0.64|7.29|3001001|amalgexporti #1|3|newborn|3|Children|208|n steseable|small|423174121sky54925954|papaya|Lb|Unknown|48|ableprioughtpri| +3133|AAAAAAAANDMAAAAA|1997-10-27||Only, main forces happen at least. Decisions remain hard. Words pin available, only victims; crucial animals must remain now n|0.15|0.08|5002002|importoscholar #2|2|country|5|Music|563|pricallyanti|N/A|2277121583yellow3147|mint|Each|Unknown|9|priprioughtpri| +3134|AAAAAAAAODMAAAAA|1997-10-27|2000-10-26|Years used to affect so upper families; se|3.00|1.68|5003001|exportischolar #1|3|pop|5|Music|323|priablepri|N/A|180royal569883429393|red|Gross|Unknown|1|eseprioughtpri| +3135|AAAAAAAAODMAAAAA|2000-10-27||Unique, old levels can stare empty teachers.|4.96|1.68|9016004|corpunivamalg #4|16|mystery|9|Books|584|eseeinganti|N/A|180royal569883429393|royal|Carton|Unknown|89|antiprioughtpri| +3136|AAAAAAAAAEMAAAAA|1997-10-27|1999-10-27|Results should search so middle, jewish services. Ago long points shall use usually various stores. Possible, old polls recover initially contracts; all medical parents join then negative pages|1.16|0.88|8009007|maxinameless #7|9|optics|8|Sports|660|barcallycally|N/A|3violet3298392521150|spring|Oz|Unknown|3|callyprioughtpri| +3137|AAAAAAAAAEMAAAAA|1999-10-28|2001-10-26|Results should search so middle, jewish services. Ago long points shall use usually various stores. Possible, old polls recover initially contracts; all medical parents join then negative pages|2.19|0.88|8009007|scholarunivamalg #6|15|fiction|9|Books|660|barcallycally|N/A|784989293tan31353161|peach|Bunch|Unknown|17|ationprioughtpri| +3138|AAAAAAAAAEMAAAAA|2001-10-27||Results should search so middle, jewish services. Ago long points shall use usually various stores. Possible, old polls recover initially contracts; all medical parents join then negative pages|3.12|0.88|8009007|edu packedu pack #1|15|athletic|4|Shoes|660|barcallycally|extra large|784989293tan31353161|saddle|Pallet|Unknown|9|eingprioughtpri| +3139|AAAAAAAADEMAAAAA|1997-10-27||Total, only women co|0.36|0.22|6010004|univbrand #4|10|jewelry boxes|6|Jewelry|615|antioughtcally|N/A|239metallic754571302|tan|Tsp|Unknown|68|n stprioughtpri| +3140|AAAAAAAAEEMAAAAA|1997-10-27|2000-10-26|Still fair relations ought to testify very over a discussions. Thoughts add so on a teachers; extensive, particular foods ought to d|83.19|27.45|5002001|importoscholar #1|2|country|5|Music|601|oughtbarcally|N/A|38502burlywood262461|salmon|Oz|Unknown|88|bareseoughtpri| +3141|AAAAAAAAEEMAAAAA|2000-10-27||Tradition|1.16|27.45|5002001|importoscholar #2|2|country|5|Music|601|oughtbarcally|N/A|48lavender3780851546|tomato|Pound|Unknown|13|oughteseoughtpri| +3142|AAAAAAAAGEMAAAAA|1997-10-27|1999-10-27|Rooms decide hardly successful, central r|1.11|0.79|8011003|amalgmaxi #3|11|archery|8|Sports|552|ableantianti|N/A|55124431491tan948569|rose|Oz|Unknown|29|ableeseoughtpri| +3143|AAAAAAAAGEMAAAAA|1999-10-28|2001-10-26|Rooms decide hardly successful, central r|3.74|0.79|9008012|namelessmaxi #12|8|romance|9|Books|552|ableantianti|N/A|55124431491tan948569|indian|Ton|Unknown|45|prieseoughtpri| +3144|AAAAAAAAGEMAAAAA|2001-10-27||Rooms decide hardly successful, central r|7.27|5.16|9008012|exportinameless #5|13|wallpaper|7|Home|116|callyoughtought|N/A|55124431491tan948569|khaki|Cup|Unknown|73|eseeseoughtpri| +3145|AAAAAAAAJEMAAAAA|1997-10-27||Prisons introduce explicitly hours. Men must get pretty together existing holidays. Really marvellous operations can come|4.49|1.75|6006002|corpcorp #2|6|rings|6|Jewelry|468|eingcallyese|N/A|86712618554916khaki4|sky|Oz|Unknown|23|antieseoughtpri| +3146|AAAAAAAAKEMAAAAA|1997-10-27|2000-10-26|Appropriate disputes shall not strike effectively at a parents. Then ill strategies must submit of course brilli|3.23|1.58|7007007|brandbrand #7|7|decor|7|Home|246|callyeseable|N/A|896593838948sienna61|violet|Cup|Unknown|10|callyeseoughtpri| +3147|AAAAAAAAKEMAAAAA|2000-10-27||Personal, green year|9.42|1.58|5002002|importoscholar #2|2|country|5|Music|285|antieingable|N/A|896593838948sienna61|white|Unknown|Unknown|44|ationeseoughtpri| +3148|AAAAAAAAMEMAAAAA|1997-10-27|1999-10-27|For example n|6.84|3.48|4003001|exportiedu pack #1|3|kids|4|Shoes|200|barbarable|large|83075689663rose81141|snow|Carton|Unknown|29|eingeseoughtpri| +3149|AAAAAAAAMEMAAAAA|1999-10-28|2001-10-26|Moving, obvious interests will deny chief, international activitie|8.17|3.48|7012010|importonameless #10|12|paint|7|Home|67|ationcally|N/A|83075689663rose81141|thistle|N/A|Unknown|21|n steseoughtpri| +3150|AAAAAAAAMEMAAAAA|2001-10-27||Moving, obvious interests will deny chief, international activitie|13.76|6.60|6008007|namelesscorp #7|12|mens watch|6|Jewelry|248|ationcally|N/A|83075689663rose81141|rose|Lb|Unknown|21|barantioughtpri| +3151|AAAAAAAAPEMAAAAA|1997-10-27||Firm nurses evolve briefly individual connections; of course safe minutes come white laws. Windows used to identify. Written trees ask eventually with a governments. Churches focus clearly|1.04|0.58|6006008|corpcorp #8|6|rings|6|Jewelry|375|antiationpri|N/A|574524398148brown586|ivory|Carton|Unknown|52|oughtantioughtpri| +3152|AAAAAAAAAFMAAAAA|1997-10-27|2000-10-26|More noble eyes could not form slightly almost previous troops. Organisational sciences used to deal yesterday enough british ope|3.96|1.62|1002001|importoamalg #1|2|fragrances|1|Women|384|eseeingpri|large|1193792419345powder8|lavender|Gram|Unknown|3|ableantioughtpri| +3153|AAAAAAAAAFMAAAAA|2000-10-27||More noble eyes could not form slightly almost previous troops. Organisational sciences used to deal yesterday enough british ope|7.49|3.81|1002001|scholarmaxi #4|15|fishing|8|Sports|528|eseeingpri|N/A|1193792419345powder8|peach|Ton|Unknown|39|priantioughtpri| +3154|AAAAAAAACFMAAAAA|1997-10-27|1999-10-27|Appropriate savings approach. Good charges gain. Primary tourists take pretty employees. Following, average arguments ought to matter possibly like women; specialist, black days us|2.97|2.49|7007005|brandbrand #5|7|decor|7|Home|327|ationablepri|N/A|213303slate979701400|sienna|Box|Unknown|13|eseantioughtpri| +3155|AAAAAAAACFMAAAAA|1999-10-28|2001-10-26|On board blue friends form for a months. New str|4.17|1.70|7007005|brandmaxi #12|7|reference|9|Books|322|ationablepri|N/A|213303slate979701400|sandy|Ounce|Unknown|79|antiantioughtpri| +3156|AAAAAAAACFMAAAAA|2001-10-27||Articles must not sta|7.04|1.70|1004001|edu packamalg #1|7|swimwear|1|Women|765|ationablepri|extra large|213303slate979701400|rosy|Box|Unknown|1|callyantioughtpri| +3157|AAAAAAAAFFMAAAAA|1997-10-27||Courses might search quite such as a scots. Women ought to borro|9.57|5.55|10002013|importounivamalg #13|2|camcorders|10|Electronics|277|ationationable|N/A|741651salmon21572293|turquoise|Ounce|Unknown|21|ationantioughtpri| +3158|AAAAAAAAGFMAAAAA|1997-10-27|2000-10-26|Just others should find tempo|0.36|0.25|6013007|exportibrand #7|13|loose stones|6|Jewelry|430|barpriese|N/A|81273354634sienna276|slate|Carton|Unknown|53|eingantioughtpri| +3159|AAAAAAAAGFMAAAAA|2000-10-27||Sensible, sorry stations ought to make home great developments. More strong stations might impress readily high civil advertisements. Studies might build legally appropriate|2.25|0.25|6013007|edu packunivamalg #15|13|audio|10|Electronics|430|barpriese|N/A|73496247tomato748512|ghost|Dozen|Unknown|53|n stantioughtpri| +3160|AAAAAAAAIFMAAAAA|1997-10-27|1999-10-27|As public farmers will wait pages. Waste, small consequences prepare also rocks. Characters must not attend quickly early individua|5.05|3.58|6015001|scholarbrand #1|15|custom|6|Jewelry|116|callyoughtought|N/A|purple57309647278010|slate|Unknown|Unknown|15|barcallyoughtpri| +3161|AAAAAAAAIFMAAAAA|1999-10-28|2001-10-26|As public farmers will wait pages. Waste, small consequences prepare also rocks. Characters must not attend quickly early individua|9.33|7.65|2003002|exportiimporto #2|15|pants|2|Men|566|callycallyanti|medium|purple57309647278010|seashell|Gross|Unknown|13|oughtcallyoughtpri| +3162|AAAAAAAAIFMAAAAA|2001-10-27||As public farmers will wait pages. Waste, small consequences prepare also rocks. Characters must not attend quickly early individua|3.17|7.65|2003001|exportiimporto #1|15|pants|2|Men|566|callycallyanti|large|purple57309647278010|tan|Bunch|Unknown|69|ablecallyoughtpri| +3163|AAAAAAAALFMAAAAA|1997-10-27||Specific, h|1.82|0.94|6011004|amalgbrand #4|11|semi-precious|6|Jewelry|364|esecallypri|N/A|919peach546303248083|hot|Carton|Unknown|30|pricallyoughtpri| +3164|AAAAAAAAMFMAAAAA|1997-10-27|2000-10-26|Legal tasks could keep somewhat black experiences. Groups would expect characters. Also steep concerns might cost for a volunteers. W|2.70|1.43|9004009|edu packmaxi #9|4|entertainments|9|Books|94|esen st|N/A|26950865596purple745|lawn|Gram|Unknown|51|esecallyoughtpri| +3165|AAAAAAAAMFMAAAAA|2000-10-27||Knees might chan|4.83|1.43|8014006|edu packmaxi #6|14|tennis|8|Sports|94|esen st|N/A|9peru831213707539965|turquoise|Oz|Unknown|74|anticallyoughtpri| +3166|AAAAAAAAOFMAAAAA|1997-10-27|1999-10-27|Brown states read responsible, s|4.81|2.21|7009007|maxibrand #7|9|mattresses|7|Home|460|barcallyese|N/A|2419789steel88225500|yellow|Carton|Unknown|26|callycallyoughtpri| +3167|AAAAAAAAOFMAAAAA|1999-10-28|2001-10-26|Brown states read responsible, s|0.31|2.21|9016012|corpunivamalg #12|9|mystery|9|Books|143|barcallyese|N/A|2419789steel88225500|dim|Gross|Unknown|59|ationcallyoughtpri| +3168|AAAAAAAAOFMAAAAA|2001-10-27||Brown states read responsible, s|4.09|1.22|9016012|exportiimporto #1|9|pants|2|Men|143|barcallyese|small|2419789steel88225500|rose|Each|Unknown|4|eingcallyoughtpri| +3169|AAAAAAAABGMAAAAA|1997-10-27||Far, alive figures protect often entire quantities. Reasons run nights. Now follo|1.57|1.36|6005002|scholarcorp #2|5|earings|6|Jewelry|147|ationeseought|N/A|4990057873598yellow8|peach|Each|Unknown|23|n stcallyoughtpri| +3170|AAAAAAAACGMAAAAA|1997-10-27|2000-10-26|Experimental, unable decisions could see at least ministers. However available windows prevent other, important images. Different cases|7.50|5.10|1003001|exportiamalg #1|3|maternity|1|Women|261|oughtcallyable|extra large|09232beige0529917232|thistle|Lb|Unknown|27|barationoughtpri| +3171|AAAAAAAACGMAAAAA|2000-10-27||Controversial police cannot avoid for example by a bits. Intermediate, potential lines help also more mental passengers; companies will hold later inner, manageria|3.78|5.10|1003001|maxiunivamalg #10|9|televisions|10|Electronics|238|oughtcallyable|N/A|09232beige0529917232|indian|Dozen|Unknown|26|oughtationoughtpri| +3172|AAAAAAAAEGMAAAAA|1997-10-27|1999-10-27|Strange, different photographs put all. Well other parties occur towards a championships. Female families take again high farms. Public mat|9.86|3.25|8010003|univmaxi #3|10|pools|8|Sports|596|callyn stanti|N/A|91270946411853pale42|aquamarine|Unknown|Unknown|76|ableationoughtpri| +3173|AAAAAAAAEGMAAAAA|1999-10-28|2001-10-26|New phases look; now numerous boots find. Circumstances look straight howe|1.38|3.25|8010003|edu packunivamalg #12|10|audio|10|Electronics|596|callyn stanti|N/A|7420354smoke80852733|papaya|N/A|Unknown|74|priationoughtpri| +3174|AAAAAAAAEGMAAAAA|2001-10-27||Only present operations may establish successful tourists. Samples would flourish novel words. Still true outcomes stop considerable agreements. As black roads decide ago serious, safe places. Scotti|5.18|3.25|8002005|importonameless #5|10|baseball|8|Sports|393|prin stpri|N/A|7420354smoke80852733|white|Unknown|Unknown|28|eseationoughtpri| +3175|AAAAAAAAHGMAAAAA|1997-10-27||Variables arrange hostile democrats. Original habits know as certain horses. Firm, technical pupils must see also never other|9.17|6.96|9012008|importounivamalg #8|12|home repair|9|Books|879|n stationeing|N/A|4526639653917rose446|light|Ton|Unknown|31|antiationoughtpri| +3176|AAAAAAAAIGMAAAAA|1997-10-27|2000-10-26|Nuclear, inc goods ought to taste as then fresh eyebrows. Subsidies look. Free trees o|4.65|2.37|1001001|amalgamalg #1|1|dresses|1|Women|346|callyesepri|extra large|511414482peach428081|thistle|Ton|Unknown|8|callyationoughtpri| +3177|AAAAAAAAIGMAAAAA|2000-10-27||Narrow issues attend adult, public papers. Old, severe legs must n|5.01|2.37|1001001|exportiimporto #2|1|pants|2|Men|33|callyesepri|medium|06steel4160236158477|tan|N/A|Unknown|96|ationationoughtpri| +3178|AAAAAAAAKGMAAAAA|1997-10-27|1999-10-27|Services earn later facts. Cases might work; engines can think. Easily major publications ought to gain very details; better ordinary funct|6.55|2.55|4001001|amalgedu pack #1|1|womens|4|Shoes|167|ationcallyought|small|26730192504034pale89|orchid|Ton|Unknown|7|eingationoughtpri| +3179|AAAAAAAAKGMAAAAA|1999-10-28|2001-10-26|Services earn later facts. Cases might work; engines can think. Easily major publications ought to gain very details; better ordinary funct|7.05|4.58|6016004|corpbrand #4|16|consignment|6|Jewelry|167|ationcallyought|N/A|7299229spring2606497|khaki|Bunch|Unknown|5|n stationoughtpri| +3180|AAAAAAAAKGMAAAAA|2001-10-27||Companies should describe in the hours; left, organisational grounds would take always about the|2.50|1.12|9007001|brandmaxi #1|16|reference|9|Books|167|ationcallyought|N/A|7299229spring2606497|royal|Tsp|Unknown|52|bareingoughtpri| +3181|AAAAAAAANGMAAAAA|1997-10-27||Necessarily independent rights defend relatively in a governments. Involved, sad |2.65|1.56|2002002|importoimporto #2|2|shirts|2|Men|805|antibareing|extra large|7067700893573maroon1|slate|Carton|Unknown|13|oughteingoughtpri| +3182|AAAAAAAAOGMAAAAA|1997-10-27|2000-10-26|Rich, deep types go. Safe premises differ particul|5.55|2.27|7003003|exportibrand #3|3|kids|7|Home|83|prieing|N/A|55655blush2192136073|tomato|Pallet|Unknown|18|ableeingoughtpri| +3183|AAAAAAAAOGMAAAAA|2000-10-27||Demonstrations shall not think often different hands. Goods could suit most national indians. Especially clinical |67.97|33.98|4004002|edu packedu pack #2|3|athletic|4|Shoes|83|prieing|petite|55655blush2192136073|turquoise|Bunch|Unknown|24|prieingoughtpri| +3184|AAAAAAAAAHMAAAAA|1997-10-27|1999-10-27|Round managers take processes. Primary, particular courses used to hold sacred cases. C|4.13|2.84|5004001|edu packscholar #1|4|classical|5|Music|183|prieingought|N/A|56565826sienna835729|yellow|Ton|Unknown|10|eseeingoughtpri| +3185|AAAAAAAAAHMAAAAA|1999-10-28|2001-10-26|Round managers take processes. Primary, particular courses used to hold sacred cases. C|6.79|2.84|5004001|scholarmaxi #6|4|history|9|Books|95|antin st|N/A|88686lavender4137662|peach|Ounce|Unknown|58|antieingoughtpri| +3186|AAAAAAAAAHMAAAAA|2001-10-27||Round managers take processes. Primary, particular courses used to hold sacred cases. C|7.96|2.84|1001001|amalgamalg #1|1|dresses|1|Women|95|antin st|petite|9490298metallic52300|ghost|Bundle|Unknown|33|callyeingoughtpri| +3187|AAAAAAAADHMAAAAA|1997-10-27||English parameters may not spring late a|31.90|14.03|10010009|univamalgamalg #9|10|memory|10|Electronics|463|pricallyese|N/A|9peru439856308879954|cornflower|Unknown|Unknown|48|ationeingoughtpri| +3188|AAAAAAAAEHMAAAAA|1997-10-27|2000-10-26|Properly young things would tell comparatively deep, beaut|0.55|0.42|8008009|namelessnameless #9|8|outdoor|8|Sports|268|eingcallyable|N/A|02284814sandy8310962|navy|Tsp|Unknown|38|eingeingoughtpri| +3189|AAAAAAAAEHMAAAAA|2000-10-27||Properly young things would tell comparatively deep, beaut|0.37|0.42|7016010|corpnameless #10|8|furniture|7|Home|268|eingcallyable|N/A|02284814sandy8310962|rosy|Dram|Unknown|100|n steingoughtpri| +3190|AAAAAAAAGHMAAAAA|1997-10-27|1999-10-27|Wives generate just about. Senior, afraid cattle should mention classical odds. As|6.50|4.42|10009012|maxiunivamalg #12|9|televisions|10|Electronics|12|ableought|N/A|04851medium767730026|blush|Tbl|Unknown|100|barn stoughtpri| +3191|AAAAAAAAGHMAAAAA|1999-10-28|2001-10-26|Years get eastern cities. General colleagues might not respond meanwhile for the drivers. Others would not get as especially rapid attitudes; white, new members ought to prev|2.83|4.42|10009012|edu packamalg #2|9|swimwear|1|Women|194|esen stought|N/A|04851medium767730026|medium|Dozen|Unknown|13|oughtn stoughtpri| +3192|AAAAAAAAGHMAAAAA|2001-10-27||Meals act then in a houses. Levels see with a citizens. Ideas alter minimum, specific figures. Financial, pretty rates must not tuck sometimes continuous, local police. No doubt late areas |2.61|4.42|10009012|exportiunivamalg #1|13|self-help|9|Books|194|esen stought|N/A|04851medium767730026|pink|Case|Unknown|62|ablen stoughtpri| +3193|AAAAAAAAJHMAAAAA|1997-10-27||Old men rise too busy projects. Terms shall not speak assu|4.34|3.90|2003002|exportiimporto #2|3|pants|2|Men|46|callyese|petite|77342384291thistle45|steel|Oz|Unknown|16|prin stoughtpri| +3194|AAAAAAAAKHMAAAAA|1997-10-27|2000-10-26|Mad relationships know essentially little books. Statemen|0.76|0.38|8010005|univmaxi #5|10|pools|8|Sports|548|eingeseanti|N/A|8521salmon9240135963|floral|Tbl|Unknown|4|esen stoughtpri| +3195|AAAAAAAAKHMAAAAA|2000-10-27||Conventional, islamic prices would not go about |7.80|0.38|8010005|exportiunivamalg #17|10|dvd/vcr players|10|Electronics|548|eingeseanti|N/A|8521salmon9240135963|thistle|Each|Unknown|24|antin stoughtpri| +3196|AAAAAAAAMHMAAAAA|1997-10-27|1999-10-27|Real, other chiefs may not participate then frequent wives. Names provide figures. Right full workers used to withstand; later complex systems appear|8.03|6.42|7004007|edu packbrand #7|4|curtains/drapes|7|Home|288|eingeingable|N/A|116552467yellow00701|orange|Lb|Unknown|20|callyn stoughtpri| +3197|AAAAAAAAMHMAAAAA|1999-10-28|2001-10-26|British, original members must not remind later likely reasons. Here nice records mark now awa|27.79|6.42|6007006|brandcorp #6|4|pendants|6|Jewelry|288|eingeingable|N/A|116552467yellow00701|snow|Cup|Unknown|35|ationn stoughtpri| +3198|AAAAAAAAMHMAAAAA|2001-10-27||Rea|3.87|6.42|10002003|importounivamalg #3|2|camcorders|10|Electronics|275|antiationable|N/A|116552467yellow00701|slate|Cup|Unknown|62|eingn stoughtpri| +3199|AAAAAAAAPHMAAAAA|1997-10-27||As available women may test british, other strategies. Interior estates will feel now living police. As strategic documents ought to take just far bright |97.44|73.08|4002002|importoedu pack #2|2|mens|4|Shoes|594|esen stanti|extra large|026283436193red45404|smoke|Each|Unknown|19|n stn stoughtpri| +3200|AAAAAAAAAIMAAAAA|1997-10-27|2000-10-26|Machines might let nowhere effects. Major,|3.24|2.49|4001001|amalgedu pack #1|1|womens|4|Shoes|664|esecallycally|large|5479741528sienna7083|coral|N/A|Unknown|2|barbarablepri| +3201|AAAAAAAAAIMAAAAA|2000-10-27||Machines might let nowhere effects. Major,|1.60|0.73|5003002|exportischolar #2|3|pop|5|Music|135|esecallycally|N/A|287944sandy321047587|pink|Box|Unknown|6|oughtbarablepri| +3202|AAAAAAAACIMAAAAA|1997-10-27|1999-10-27|Traditional pressures ought to speak too other days. Perhaps terrible relatives might become. About formal police appreciate at best ever|7.91|3.24|6016003|corpbrand #3|16|consignment|6|Jewelry|137|ationpriought|N/A|83192yellow563974045|pink|Dram|Unknown|15|ablebarablepri| +3203|AAAAAAAACIMAAAAA|1999-10-28|2001-10-26|Of course big countries withdraw collectively so big instruments. Never cold developments harbour with the shows. Further head events remember on a others. Standard tears receive increased co|0.59|0.40|6016003|edu packimporto #2|4|sports-apparel|2|Men|54|eseanti|extra large|0wheat66542216342691|rose|Case|Unknown|58|pribarablepri| +3204|AAAAAAAACIMAAAAA|2001-10-27||Of course big countries withdraw collectively so big instruments. Never cold developments harbour with the shows. Further head events remember on a others. Standard tears receive increased co|2.74|0.84|10004009|edu packunivamalg #9|4|audio|10|Electronics|44|eseese|N/A|90508turquoise205207|wheat|Bunch|Unknown|65|esebarablepri| +3205|AAAAAAAAFIMAAAAA|1997-10-27||Elsewhere warm auth|4.88|1.75|5003002|exportischolar #2|3|pop|5|Music|11|oughtought|N/A|93710523689316peru88|smoke|Each|Unknown|44|antibarablepri| +3206|AAAAAAAAGIMAAAAA|1997-10-27|2000-10-26|Of course commercial uses look rapidly historical societies. Writers make just high|3.82|1.33|5003001|exportischolar #1|3|pop|5|Music|148|eingeseought|N/A|96dim449235528539370|linen|Unknown|Unknown|28|callybarablepri| +3207|AAAAAAAAGIMAAAAA|2000-10-27||Greatly payable experts could compare weeks; leading notes will not wait poss|4.93|1.33|9006004|corpmaxi #4|6|parenting|9|Books|148|eingeseought|N/A|653062008yellow20298|red|Ton|Unknown|12|ationbarablepri| +3208|AAAAAAAAIIMAAAAA|1997-10-27|1999-10-27|Nearly high characters can secure significant, local years. Clouds prevent for the feelings. Always rare events sit suddenly little arts. Yet bad article|4.19|3.18|3002001|importoexporti #1|2|infants|3|Children|217|ationoughtable|extra large|6946906salmon6486506|rosy|Pallet|Unknown|42|eingbarablepri| +3209|AAAAAAAAIIMAAAAA|1999-10-28|2001-10-26|True m|9.46|3.18|3002001|corpnameless #10|16|furniture|7|Home|23|ationoughtable|N/A|891863firebrick47666|pink|Unknown|Unknown|18|n stbarablepri| +3210|AAAAAAAAIIMAAAAA|2001-10-27||True m|0.79|0.23|3002001|scholarmaxi #1|15|fishing|8|Sports|351|oughtantipri|N/A|891863firebrick47666|smoke|Bundle|Unknown|11|baroughtablepri| +3211|AAAAAAAALIMAAAAA|1997-10-27||Months get due in the revenues. Only important parties walk civil, respective vehicles; cultural courses would not count commercial, labour actions; major politicians shall come hopefully r|1.68|0.65|8001002|amalgnameless #2|1|athletic shoes|8|Sports|752|ableantiation|N/A|7178071797plum276905|thistle|Dozen|Unknown|20|oughtoughtablepri| +3212|AAAAAAAAMIMAAAAA||2000-10-26|||||namelessnameless #3|8|outdoor|8||404|esebarese||1678298462038linen43|tomato|Tbl|Unknown|40|| +3213|AAAAAAAAMIMAAAAA|2000-10-27||High bacteria produce somewhere goods; international documents can go abroad new operations. Home famous letters should respond on the demands. Heavy, experimental authorities shall not d|9.01|5.49|8008003|importoimporto #2|8|shirts|2|Men|404|esebarese|small|1678298462038linen43|turquoise|N/A|Unknown|69|prioughtablepri| +3214|AAAAAAAAOIMAAAAA|1997-10-27|1999-10-27|Typical, short circumstances see at present home other negotiations. Left, white plants ought to allow primary figures.|0.95|0.62|2003001|exportiimporto #1|3|pants|2|Men|207|ationbarable|medium|purple83022073518270|saddle|Gram|Unknown|55|eseoughtablepri| +3215|AAAAAAAAOIMAAAAA|1999-10-28|2001-10-26|Below wide items ought to require very; processes could fall badly general cu|68.42|0.62|2003001|namelessunivamalg #3|8|scanners|10|Electronics|207|ationbarable|N/A|purple83022073518270|blanched|Cup|Unknown|86|antioughtablepri| +3216|AAAAAAAAOIMAAAAA|2001-10-27||Below wide items ought to require very; processes could fall badly general cu|8.17|4.33|2003001|importobrand #1|8|costume|6|Jewelry|5|ationbarable|N/A|1934medium8205486690|saddle|Ton|Unknown|60|callyoughtablepri| +3217|AAAAAAAABJMAAAAA|1997-10-27||Small dogs meet. Correc|0.89|0.28|5003002|exportischolar #2|3|pop|5|Music|43|priese|N/A|honeydew896467814416|white|Tsp|Unknown|5|ationoughtablepri| +3218|AAAAAAAACJMAAAAA|1997-10-27|2000-10-26|Evenly royal employees want safely royal, exact rights. Hours give. Immediate, important members used to build really also black years. African, continuous changes add legal ye|4.37|1.92|4003001|exportiedu pack #1|3|kids|4|Shoes|77|ationation|medium|4rose525446471864568|linen|Oz|Unknown|3|eingoughtablepri| +3219|AAAAAAAACJMAAAAA|2000-10-27||Evenly royal employees want safely royal, exact rights. Hours give. Immediate, important members used to build really also black years. African, continuous changes add legal ye|7.65|1.92|4004002|edu packedu pack #2|4|athletic|4|Shoes|293|ationation|extra large|4rose525446471864568|orchid|Bunch|Unknown|50|n stoughtablepri| +3220|AAAAAAAAEJMAAAAA|1997-10-27|1999-10-27|Kinds mean never different weeks. Likely areas ask perhaps. Beautiful rights may not celebrate working-c|3.81|1.60|7014001|edu packnameless #1|14|glassware|7|Home|132|ablepriought|N/A|822571552678wheat655|sienna|Tsp|Unknown|48|barableablepri| +3221|AAAAAAAAEJMAAAAA|1999-10-28|2001-10-26|Female, similar refugees open especially great attitudes. Then satisfied photographs mark delicately healthy plans. Formal troops will lodge highest fresh eyes. Reli|2.85|2.53|7007002|brandbrand #2|14|decor|7|Home|132|ablepriought|N/A|8gainsboro5541952065|grey|Carton|Unknown|5|oughtableablepri| +3222|AAAAAAAAEJMAAAAA|2001-10-27||Female, similar refugees open especially great attitudes. Then satisfied photographs mark delicately healthy plans. Formal troops will lodge highest fresh eyes. Reli|6.83|2.53|7007002|scholarnameless #3|5|fitness|8|Sports|132|ablepriought|N/A|61093sandy2147320720|linen|Each|Unknown|58|ableableablepri| +3223|AAAAAAAAHJMAAAAA|1997-10-27||Sharp brief preferences cannot know overall levels. Joint, good feet visit probably. Players will not get small stars|1.91|1.47|8013004|exportimaxi #4|13|sailing|8|Sports|27|ationable|N/A|0789984889965yellow9|sienna|Carton|Unknown|47|priableablepri| +3224|AAAAAAAAIJMAAAAA|1997-10-27|2000-10-26|Little, healthy options ask around magistrates. Quite e|0.80|0.68|3004001|edu packexporti #1|4|school-uniforms|3|Children|143|prieseought|small|5574996293plum077333|smoke|Cup|Unknown|18|eseableablepri| +3225|AAAAAAAAIJMAAAAA|2000-10-27||Prices avoid. Neither new directors cover very resources; mutual techniques would |1.59|0.68|5001002|amalgscholar #2|4|rock|5|Music|143|prieseought|N/A|5574996293plum077333|steel|Gram|Unknown|34|antiableablepri| +3226|AAAAAAAAKJMAAAAA|1997-10-27|1999-10-27|Imaginative games distinguish ambitio|2.46|1.03|7008007|namelessbrand #7|8|lighting|7|Home|774|eseationation|N/A|3rose100738477404805|steel|Cup|Unknown|84|callyableablepri| +3227|AAAAAAAAKJMAAAAA|1999-10-28|2001-10-26|Imaginative games distinguish ambitio|3.16|1.03|8007008|brandnameless #8|8|hockey|8|Sports|8|eing|N/A|3rose100738477404805|rosy|Unknown|Unknown|20|ationableablepri| +3228|AAAAAAAAKJMAAAAA|2001-10-27||Imaginative games distinguish ambitio|8.86|1.03|8007008|exportiimporto #1|8|pants|2|Men|297|eing|medium|07290127peru18333453|metallic|Each|Unknown|30|eingableablepri| +3229|AAAAAAAANJMAAAAA|1997-10-27||Minutes find by a others. Then new firms|3.93|3.45|7002010|importobrand #10|2|bedding|7|Home|44|eseese|N/A|52619529pink99792676|peru|Each|Unknown|33|n stableablepri| +3230|AAAAAAAAOJMAAAAA|1997-10-27|2000-10-26|Only central books trust yea|2.06|1.17|1003001|exportiamalg #1|3|maternity|1|Women|88|eingeing|medium|7turquoise9682200212|plum|Pound|Unknown|48|barpriablepri| +3231|AAAAAAAAOJMAAAAA|2000-10-27||Projects may show for a guns. Views sing persons. Often joint things|8.47|1.17|1003001|importoimporto #2|2|shirts|2|Men|88|eingeing|large|041wheat513266717232|seashell|Ounce|Unknown|23|oughtpriablepri| +3232|AAAAAAAAAKMAAAAA|1997-10-27|1999-10-27|Proper things ought to come sometime|3.56|2.91|7014003|edu packnameless #3|14|glassware|7|Home|146|callyeseought|N/A|605031spring11241731|royal|Unknown|Unknown|3|ablepriablepri| +3233|AAAAAAAAAKMAAAAA|1999-10-28|2001-10-26|Proper things ought to come sometime|4.32|3.36|7014003|exportiexporti #2|3|toddlers|3|Children|146|callyeseought|medium|0815033210080puff368|lemon|Pallet|Unknown|92|pripriablepri| +3234|AAAAAAAAAKMAAAAA|2001-10-27||Proper things ought to come sometime|0.38|3.36|7014003|importoamalg #1|2|fragrances|1|Women|65|antically|economy|0815033210080puff368|steel|Tbl|Unknown|31|esepriablepri| +3235|AAAAAAAADKMAAAAA|1997-10-27||Now va|5.44|2.33|1003002|exportiamalg #2|3|maternity|1|Women|698|eingn stcally|medium|43512695wheat0942153|sandy|Ton|Unknown|1|antipriablepri| +3236|AAAAAAAAEKMAAAAA|1997-10-27|2000-10-26|Christian years wo|1.63|0.71|4002001|importoedu pack #1|2|mens|4|Shoes|185|antieingought|medium|2461slate45167440931|tan|Gross|Unknown|18|callypriablepri| +3237|AAAAAAAAEKMAAAAA|2000-10-27||Fine others make rapid departments; large, canadian goals seem also by a matters. Soviet, national friends may know long, hig|8.70|0.71|5004002|edu packscholar #2|2|classical|5|Music|185|antieingought|N/A|708630puff3989174229|royal|Lb|Unknown|5|ationpriablepri| +3238|AAAAAAAAGKMAAAAA|1997-10-27|1999-10-27|Officers prove more often substantial fans. Loans would approach forward tired men. Variable, different alternatives used to |35.02|18.91|3004001|edu packexporti #1|4|school-uniforms|3|Children|649|n stesecally|large|11497616spring150934|hot|N/A|Unknown|12|eingpriablepri| +3239|AAAAAAAAGKMAAAAA|1999-10-28|2001-10-26|Officers prove more often substantial fans. Loans would approach forward tired men. Variable, different alternatives used to |0.41|0.25|3004001|scholarnameless #10|4|tables|7|Home|215|antioughtable|N/A|11497616spring150934|saddle|Pallet|Unknown|46|n stpriablepri| +3240|AAAAAAAAGKMAAAAA|2001-10-27||International, initial provisions shall follow problems. Forward necessary others will use thereby interesting international companies. Working friends could say flowers. Away roug|4.82|3.47|3004001|amalgedu pack #1|4|womens|4|Shoes|841|antioughtable|petite|27113216steel1738287|gainsboro|Gross|Unknown|50|bareseablepri| +3241|AAAAAAAAJKMAAAAA|1997-10-27||English, overseas lives used to move again similar sentences. Sites can view always. Able, essential incom|4.37|1.79|8009002|maxinameless #2|9|optics|8|Sports|291|oughtn stable|N/A|26688301569cream5464|peru|Gross|Unknown|33|oughteseablepri| +3242|AAAAAAAAKKMAAAAA|1997-10-27|2000-10-26|Possible ages rely from a settlements. Frequently accurate proposals expand quickly budgets. Virtually other years appear on a managers. European, small reasons replace fam|8.89|5.42|10004013|edu packunivamalg #13|4|audio|10|Electronics|11|oughtought|N/A|87261609394deep60748|almond|Dram|Unknown|4|ableeseablepri| +3243|AAAAAAAAKKMAAAAA|2000-10-27||Perhaps single operations shall reform historical, increased others. Fair, real kinds try entirely respectively separate others. Precise, royal claims must kill otherwise forces. Women must not pick|0.10|0.06|2004002|edu packimporto #2|4|sports-apparel|2|Men|484|eseeingese|medium|87261609394deep60748|peach|Cup|Unknown|63|prieseablepri| +3244|AAAAAAAAMKMAAAAA|1997-10-27|1999-10-27|Cheap, ill schools earn countries; russian strategies might look beliefs. Super police investigate m|1.05|0.93|7016005|corpnameless #5|16|furniture|7|Home|267|ationcallyable|N/A|275200455655papaya16|seashell|Carton|Unknown|4|eseeseablepri| +3245|AAAAAAAAMKMAAAAA|1999-10-28|2001-10-26|Cheap, ill schools earn countries; russian strategies might look beliefs. Super police investigate m|2.11|0.93|7016005|edu packexporti #2|16|school-uniforms|3|Children|972|ableationn st|large|39767papaya823629694|almond|Tbl|Unknown|27|antieseablepri| +3246|AAAAAAAAMKMAAAAA|2001-10-27||Cheap, ill schools earn countries; russian strategies might look beliefs. Super police investigate m|3.97|0.93|9015007|scholarunivamalg #7|15|fiction|9|Books|711|ableationn st|N/A|39767papaya823629694|steel|Box|Unknown|23|callyeseablepri| +3247|AAAAAAAAPKMAAAAA|1997-10-27||Again silver aspects clarify there onto a servants. Users wonder very|2.05|1.41|1004002|edu packamalg #2|4|swimwear|1|Women|125|antiableought|small|485429348305rose9259|rosy|Cup|Unknown|99|ationeseablepri| +3248|AAAAAAAAALMAAAAA|1997-10-27|2000-10-26|Back, social names gather known experiences. Tough problems shall gain. Powerful, far stones cou|3.50|3.15|9007009|brandmaxi #9|7|reference|9|Books|257|ationantiable|N/A|84770838802149tan271|hot|Each|Unknown|5|eingeseablepri| +3249|AAAAAAAAALMAAAAA|2000-10-27||Only vast prices should give great, na|5.70|2.05|9007009|edu packcorp #2|7|bracelets|6|Jewelry|241|oughteseable|N/A|84770838802149tan271|forest|Gram|Unknown|37|n steseablepri| +3250|AAAAAAAACLMAAAAA|1997-10-27|1999-10-27|Underlying, particular records make; previous societies should not come limits. Well clear legs leave still good full boys; typical, unlikely buildings would come already polit|5.43|2.98|4004001|edu packedu pack #1|4|athletic|4|Shoes|251|oughtantiable|medium|24697831284006smoke5|saddle|Case|Unknown|55|barantiablepri| +3251|AAAAAAAACLMAAAAA|1999-10-28|2001-10-26|Very new points may see invariably natu|1.89|2.98|4004001|edu packedu pack #2|4|athletic|4|Shoes|251|oughtantiable|petite|24697831284006smoke5|metallic|Unknown|Unknown|66|oughtantiablepri| +3252|AAAAAAAACLMAAAAA|2001-10-27||Others may not market then back companies. So international activities will reflect over a members. Late, practical attacks used to generate tragically. Hours include in|3.69|2.98|4004001|exportinameless #3|4|basketball|8|Sports|952|ableantin st|N/A|24697831284006smoke5|violet|Pallet|Unknown|10|ableantiablepri| +3253|AAAAAAAAFLMAAAAA|1997-10-27||Video-taped, independent doubts see rates. Employees would not give elsewhere welsh words. Strong, happy sanctions cannot take within the options; eyes look just diffe|2.57|0.82|1001002|amalgamalg #2|1|dresses|1|Women|201|oughtbarable|medium|56969sienna351380972|lime|Box|Unknown|24|priantiablepri| +3254|AAAAAAAAGLMAAAAA|1997-10-27|2000-10-26|Comprehensive kinds may c|9.43|4.24|9005003|scholarmaxi #3|5|history|9|Books|91|oughtn st|N/A|353sky28364398119455|tomato|Gross|Unknown|16|eseantiablepri| +3255|AAAAAAAAGLMAAAAA|2000-10-27||Comprehensive kinds may c|0.36|0.24|9005003|scholarunivamalg #4|15|fiction|9|Books|91|oughtn st|N/A|353sky28364398119455|seashell|Bunch|Unknown|69|antiantiablepri| +3256|AAAAAAAAILMAAAAA|1997-10-27|1999-10-27|Leading, healthy schools understand only different, other rumours. Ever technical signs meet very political months. Firm symptoms keep physical, independent lines. Later identical subjects |4.25|1.99|2002001|importoimporto #1|2|shirts|2|Men|61|oughtcally|economy|3120476purple9038968|dark|Oz|Unknown|97|callyantiablepri| +3257|AAAAAAAAILMAAAAA|1999-10-28|2001-10-26|Leading, healthy schools understand only different, other rumours. Ever technical signs meet very political months. Firm symptoms keep physical, independent lines. Later identical subjects |0.47|0.15|5002002|importoscholar #2|2|country|5|Music|204|oughtcally|N/A|3120476purple9038968|dodger|Gram|Unknown|10|ationantiablepri| +3258|AAAAAAAAILMAAAAA|2001-10-27||Leading, healthy schools understand only different, other rumours. Ever technical signs meet very political months. Firm symptoms keep physical, independent lines. Later identical subjects |9.36|0.15|5002002|edu packimporto #1|4|sports-apparel|2|Men|204|esebarable|small|3120476purple9038968|sandy|Gram|Unknown|51|eingantiablepri| +3259|AAAAAAAALLMAAAAA|1997-10-27||Nights go most mere, foreign colleagu|2.96|1.09|8005010|scholarnameless #10|5|fitness|8|Sports|225|antiableable|N/A|18634gainsboro799818|sienna|Gram|Unknown|57|n stantiablepri| +3260|AAAAAAAAMLMAAAAA|1997-10-27|2000-10-26|Various, key mines get institutions. |4.19|2.22|8006001|corpnameless #1|6|football|8|Sports|72|ableation|N/A|9583pale119181251953|yellow|Gross|Unknown|17|barcallyablepri| +3261|AAAAAAAAMLMAAAAA|2000-10-27||Various, key mines get institutions. |1.88|2.22|8008002|namelessnameless #2|6|outdoor|8|Sports|72|ableation|N/A|199694302tan52134887|slate|Tsp|Unknown|23|oughtcallyablepri| +3262|AAAAAAAAOLMAAAAA|1997-10-27|1999-10-27|Then european wages must get shareholders. Positions say much before the metres; circumstances must not see readily black revenues. Developing|6.64|4.58|4004001|edu packedu pack #1|4|athletic|4|Shoes|81|oughteing|extra large|272270aquamarine7449|medium|Gross|Unknown|69|ablecallyablepri| +3263|AAAAAAAAOLMAAAAA|1999-10-28|2001-10-26|Particular le|1.77|4.58|4004001|amalgmaxi #4|4|archery|8|Sports|550|barantianti|N/A|272270aquamarine7449|sandy|Dram|Unknown|52|pricallyablepri| +3264|AAAAAAAAOLMAAAAA|2001-10-27||Particular le|4.10|4.58|4004001|amalgmaxi #1|4|arts|9|Books|550|barantianti|N/A|272270aquamarine7449|antique|Unknown|Unknown|66|esecallyablepri| +3265|AAAAAAAABMMAAAAA|1997-10-27||Horses will not give. Historical writers shall land here dry, influential assets. Even crucial definitions should pay backwards situations. Never other forces find importan|0.56|0.47|7015006|scholarnameless #6|15|tables|7|Home|457|ationantiese|N/A|83605saddle727372172|ivory|Dozen|Unknown|29|anticallyablepri| +3266|AAAAAAAACMMAAAAA|1997-10-27|2000-10-26|Standards might not halt crazy seconds. Affairs may keep remaining, central ships. All nati|9.78|3.03|3002001|importoexporti #1|2|infants|3|Children|123|priableought|medium|674975092tomato59363|tan|Tbl|Unknown|15|callycallyablepri| +3267|AAAAAAAACMMAAAAA|2000-10-27||Rooms may not carry towards a problems. Human members must not climb popular, empty words; patien|5.43|4.56|4004002|edu packedu pack #2|4|athletic|4|Shoes|1|ought|large|674975092tomato59363|peach|Pallet|Unknown|9|ationcallyablepri| +3268|AAAAAAAAEMMAAAAA|1997-10-27|1999-10-27|Years take at least national projects. Other things go here worth a ideas. Perhaps political countries monitor more for good dependent ch|3.72|1.41|8010009|univmaxi #9|10|pools|8|Sports|420|barableese|N/A|plum8852462167465641|magenta|Ounce|Unknown|25|eingcallyablepri| +3269|AAAAAAAAEMMAAAAA|1999-10-28|2001-10-26|Years take at least national projects. Other things go here worth a ideas. Perhaps political countries monitor more for good dependent ch|34.92|13.96|8010009|scholarmaxi #6|5|history|9|Books|68|barableese|N/A|1247208461spring9456|smoke|Tbl|Unknown|34|n stcallyablepri| +3270|AAAAAAAAEMMAAAAA|2001-10-27||Years take at least national projects. Other things go here worth a ideas. Perhaps political countries monitor more for good dependent ch|1.69|0.54|8010009|amalgedu pack #1|5|womens|4|Shoes|211|oughtoughtable|small|380573pale2607546825|salmon|Carton|Unknown|7|barationablepri| +3271|AAAAAAAAHMMAAAAA|1997-10-27||Appropriate, high businesses bind young, social championships. Boxes present. Unfortunately single carers should heed primarily vertically|2.88|1.67|4003002|exportiedu pack #2|3|kids|4|Shoes|219|n stoughtable|medium|51003plum11458436936|snow|Bundle|Unknown|54|oughtationablepri| +3272|AAAAAAAAIMMAAAAA|1997-10-27|2000-10-26|Decisions get quickly with a notes. Anywhere private journals can get. Front, secondary hands could see yet national i|3.67|2.64|6002001|importocorp #1|2|diamonds|6|Jewelry|280|bareingable|N/A|42wheat8716314086508|lace|N/A|Unknown|9|ableationablepri| +3273|AAAAAAAAIMMAAAAA|2000-10-27||Decisions get quickly with a notes. Anywhere private journals can get. Front, secondary hands could see yet national i|0.36|2.64|6002001|importobrand #2|12|costume|6|Jewelry|144|bareingable|N/A|42wheat8716314086508|yellow|Each|Unknown|24|priationablepri| +3274|AAAAAAAAKMMAAAAA|1997-10-27|1999-10-27|Specific walls go conversely russian women. Correctly fair priorities track to a lives. Complete memorie|2.22|1.11|8016005|corpmaxi #5|16|golf|8|Sports|590|barn stanti|N/A|95861188tan980046110|green|Gram|Unknown|29|eseationablepri| +3275|AAAAAAAAKMMAAAAA|1999-10-28|2001-10-26|Agents will not follow convincingly neatly different beliefs. Public shadows put even separate, tall associations. Hills would vote direct conventional|4.15|3.36|9001012|amalgmaxi #12|1|arts|9|Books|590|barn stanti|N/A|95861188tan980046110|powder|Dozen|Unknown|16|antiationablepri| +3276|AAAAAAAAKMMAAAAA|2001-10-27||Agents will not follow convincingly neatly different beliefs. Public shadows put even separate, tall associations. Hills would vote direct conventional|9.09|3.36|7016007|corpnameless #7|16|furniture|7|Home|211|barn stanti|N/A|95861188tan980046110|brown|Dozen|Unknown|75|callyationablepri| +3277|AAAAAAAANMMAAAAA|1997-10-27||Alone working-class dates open from a issues. Most european concessions will not tell personal areas; central, poor officials might not act |4.68|4.16|6003006|exporticorp #6|3|gold|6|Jewelry|33|pripri|N/A|thistle7440470506576|floral|Case|Unknown|5|ationationablepri| +3278|AAAAAAAAOMMAAAAA|1997-10-27|2000-10-26|Special objectives shall appear patients. Exciting minutes shall attend. Proud, british details learn comm|0.39|0.25|2003001|exportiimporto #1|3|pants|2|Men|546|callyeseanti|petite|469200tan17018622347|honeydew|Pallet|Unknown|25|eingationablepri| +3279|AAAAAAAAOMMAAAAA|2000-10-27||Special objectives shall appear patients. Exciting minutes shall attend. Proud, british details learn comm|9.62|0.25|2003001|exportischolar #2|3|pop|5|Music|546|callyeseanti|N/A|469200tan17018622347|wheat|Ton|Unknown|92|n stationablepri| +3280|AAAAAAAAANMAAAAA|1997-10-27|1999-10-27|Questions take. Patient,|3.85|2.34|10012017|importoamalgamalg #17|12|monitors|10|Electronics|525|antiableanti|N/A|641624005209puff4836|purple|Pound|Unknown|1|bareingablepri| +3281|AAAAAAAAANMAAAAA|1999-10-28|2001-10-26|Questions take. Patient,|7.71|2.34|3003002|exportiexporti #2|12|toddlers|3|Children|820|antiableanti|large|404323839380pink3165|powder|N/A|Unknown|62|oughteingablepri| +3282|AAAAAAAAANMAAAAA|2001-10-27||Questions take. Patient,|4.74|3.83|4003001|exportiedu pack #1|12|kids|4|Shoes|820|antiableanti|medium|404323839380pink3165|rose|Tsp|Unknown|43|ableeingablepri| +3283|AAAAAAAADNMAAAAA|1997-10-27||Groups will not tell sales. Again harsh mice |2.25|1.68|5001002|amalgscholar #2|1|rock|5|Music|78|eingation|N/A|50white7552436890485|midnight|Tsp|Unknown|18|prieingablepri| +3284|AAAAAAAAENMAAAAA|1997-10-27|2000-10-26|Gratefully male kids see at first odd principles. New, brown processes buy just areas. Conditions find much with a|4.67|2.10|2004001|edu packimporto #1|4|sports-apparel|2|Men|369|n stcallypri|large|8197634780thistle140|wheat|Tsp|Unknown|69|eseeingablepri| +3285|AAAAAAAAENMAAAAA|2000-10-27||Gratefully male kids see at first odd principles. New, brown processes buy just areas. Conditions find much with a|7.37|4.86|3002002|importoexporti #2|4|infants|3|Children|876|callyationeing|medium|8197634780thistle140|smoke|Dram|Unknown|22|antieingablepri| +3286|AAAAAAAAGNMAAAAA|1997-10-27|1999-10-27|As different|0.31|0.25|8013007|exportimaxi #7|13|sailing|8|Sports|77|ationation|N/A|beige428693014310581|sienna|Cup|Unknown|15|callyeingablepri| +3287|AAAAAAAAGNMAAAAA|1999-10-28|2001-10-26|Tired, light owners generate then by the rights. Formidable pr|2.00|0.25|2004002|edu packimporto #2|13|sports-apparel|2|Men|19|n stought|petite|beige428693014310581|salmon|Case|Unknown|66|ationeingablepri| +3288|AAAAAAAAGNMAAAAA|2001-10-27||Tired, light owners generate then by the rights. Formidable pr|1.17|0.25|10014008|edu packamalgamalg #8|14|automotive|10|Electronics|19|n stought|N/A|4348644710534red1537|red|Dozen|Unknown|14|eingeingablepri| +3289|AAAAAAAAJNMAAAAA|1997-10-27||Also international ministers shall know. Muscles bet governments. Soviet, different things believe average nurses; largely full offences shall keep then gen|70.93|21.27|1003002|exportiamalg #2|3|maternity|1|Women|221|oughtableable|medium|10337817189601pale21|turquoise|Dozen|Unknown|13|n steingablepri| +3290|AAAAAAAAKNMAAAAA|1997-10-27|2000-10-26|Solicitors could come for a values. Enough small hospi|8.12|3.57|1001001|amalgamalg #1|1|dresses|1|Women|66|callycally|medium|4899royal68076166923|navy|Ton|Unknown|58|barn stablepri| +3291|AAAAAAAAKNMAAAAA|2000-10-27||Major, internal manufacturers might follow. Occasionally unaware prices must live well in the times; big, other children come fears. Minutes would grant very somewhat ci|2.95|2.30|1001001|exportiamalg #2|1|maternity|1|Women|66|callycally|large|4899royal68076166923|azure|Gram|Unknown|65|oughtn stablepri| +3292|AAAAAAAAMNMAAAAA|1997-10-27|1999-10-27|Sure eggs cope too central forces. Faces make indeed hot thousands. Perfectly new pupils ought to encourage already both old eff|2.69|1.23|4002001|importoedu pack #1|2|mens|4|Shoes|62|ablecally|medium|8186676tan4733131542|misty|Cup|Unknown|14|ablen stablepri| +3293|AAAAAAAAMNMAAAAA|1999-10-28|2001-10-26|Combined uses shall not apply strongly upper studies. Soft, poor interests try simply rational areas. Fo|1.45|1.23|9010006|univunivamalg #6|2|travel|9|Books|213|ablecally|N/A|46white8975290566371|violet|Gross|Unknown|92|prin stablepri| +3294|AAAAAAAAMNMAAAAA|2001-10-27||Important customers say ago straightforward trustees. Direct points deliver horizontally unfortunately strong others. Concerns might chart then good wor|19.08|14.31|2002001|importoimporto #1|2|shirts|2|Men|213|ablecally|medium|46white8975290566371|lavender|Bunch|Unknown|3|esen stablepri| +3295|AAAAAAAAPNMAAAAA|1997-10-27||Further excessive reactions will provide quickly types. Lucky colleagues seem for a |8.47|3.21|9010008|univunivamalg #8|10|travel|9|Books|317|ationoughtpri|N/A|papaya84948257112695|grey|Carton|Unknown|3|antin stablepri| +3296|AAAAAAAAAOMAAAAA|1997-10-27|2000-10-26|Competent women must work substantially in a months. Standard, united police will hide at a women. Specialist activities create directly. About beautiful terms can tell at first incidents. More i|6.29|4.21|2001001|amalgimporto #1|1|accessories|2|Men|941|oughtesen st|large|1sandy81117787268680|royal|Cup|Unknown|35|callyn stablepri| +3297|AAAAAAAAAOMAAAAA|2000-10-27||Competent women must work substantially in a months. Standard, united police will hide at a women. Specialist activities create directly. About beautiful terms can tell at first incidents. More i|2.96|4.21|10001017|amalgunivamalg #17|1|cameras|10|Electronics|645|oughtesen st|N/A|1sandy81117787268680|peach|Oz|Unknown|58|ationn stablepri| +3298|AAAAAAAACOMAAAAA|1997-10-27|1999-10-27|Reluctantly experimental kinds must not become. Addresses remember electoral inhabitants. Keen, big paintings might not provide. Hap|1.32|0.72|2003001|exportiimporto #1|3|pants|2|Men|181|oughteingought|petite|57849gainsboro807840|sandy|Ounce|Unknown|19|eingn stablepri| +3299|AAAAAAAACOMAAAAA|1999-10-28|2001-10-26|Pregnant, important times use at a times. Lim|0.68|0.72|1002002|importoamalg #2|3|fragrances|1|Women|247|ationeseable|large|6pale850467433688670|thistle|Unknown|Unknown|11|n stn stablepri| +3300|AAAAAAAACOMAAAAA|2001-10-27||Soft plans select just cautiously daily thousands. Early, old germans find never. Already fo|8.74|2.88|1002002|edu packscholar #1|4|classical|5|Music|48|eingese|N/A|0838thistle512851096|thistle|Bundle|Unknown|1|barbarpripri| +3301|AAAAAAAAFOMAAAAA|1997-10-27||Nights must not hear suddenly patients; times show also employees. Different,|5.34|2.24|1003002|exportiamalg #2|3|maternity|1|Women|450|barantiese|small|35cyan23043050701081|peach|Tbl|Unknown|71|oughtbarpripri| +3302|AAAAAAAAGOMAAAAA|1997-10-27|2000-10-26|Mothers used to offer then private agencies. Figures give over. Feet might not collect difficult, military areas. Black|1.95|1.26|10001005|amalgunivamalg #5|1|cameras|10|Electronics|709|n stbaration|N/A|67516666777447light7|violet|Tsp|Unknown|66|ablebarpripri| +3303|AAAAAAAAGOMAAAAA|2000-10-27||Eventually other opponents withdraw meanwhile. Little churches run soon with a children; patients could not ensure away personal workers; fans shall offer certainly within the n|2.13|1.26|5001002|amalgscholar #2|1|rock|5|Music|709|n stbaration|N/A|573767796royal736575|smoke|Gram|Unknown|59|pribarpripri| +3304|AAAAAAAAIOMAAAAA|1997-10-27|1999-10-27|Monetary, western ideas may work as; determined, level events ought to explain in addition significant, annual pains; modern, semantic families could not make stones. Difficult, wooden of|4.43|3.10|1001001|amalgamalg #1|1|dresses|1|Women|314|eseoughtpri|petite|501823487687sky86348|sky|Dozen|Unknown|36|esebarpripri| +3305|AAAAAAAAIOMAAAAA|1999-10-28|2001-10-26|Dry forces might see. Designs make very prime cities. Cheap teachers will find firms. Classic, modest pupils worry finally in the explanations; exactly politic|5.08|1.87|1001001|scholaramalgamalg #8|15|portable|10|Electronics|314|eseoughtpri|N/A|8978933antique299976|turquoise|Gram|Unknown|27|antibarpripri| +3306|AAAAAAAAIOMAAAAA|2001-10-27||Dry forces might see. Designs make very prime cities. Cheap teachers will find firms. Classic, modest pupils worry finally in the explanations; exactly politic|8.81|7.04|7009007|maxibrand #7|9|mattresses|7|Home|314|eseoughtpri|N/A|8978933antique299976|plum|Bundle|Unknown|24|callybarpripri| +3307|AAAAAAAALOMAAAAA|1997-10-27||Developments produce joint, little points. So palestinian children research finally more agricultural estates. Appropriate miles may cost ministers; dirty terms mig|6.72|5.44|2002002|importoimporto #2|2|shirts|2|Men|947|ationesen st|large|451503958747rosy2935|forest|Cup|Unknown|62|ationbarpripri| +3308|AAAAAAAAMOMAAAAA|1997-10-27|2000-10-26|Political pockets stay|3.32|2.19|6013005|exportibrand #5|13|loose stones|6|Jewelry|316|callyoughtpri|N/A|637rose5624823456908|turquoise|Gram|Unknown|66|eingbarpripri| +3309|AAAAAAAAMOMAAAAA|2000-10-27||Political pockets stay|5.98|2.19|3003002|exportiexporti #2|13|toddlers|3|Children|464|callyoughtpri|small|637rose5624823456908|white|Bundle|Unknown|53|n stbarpripri| +3310|AAAAAAAAOOMAAAAA|1997-10-27|1999-10-27|Leaders guard generally police. Democratic witnesses may see efficiently questions. Clear, modern maps should not settle special, small elements. Final, public workers would not lose caref|3.54|1.77|7012001|importonameless #1|12|paint|7|Home|813|prioughteing|N/A|707568911magenta0157|sienna|Oz|Unknown|18|baroughtpripri| +3311|AAAAAAAAOOMAAAAA|1999-10-28|2001-10-26|Leaders guard generally police. Democratic witnesses may see efficiently questions. Clear, modern maps should not settle special, small elements. Final, public workers would not lose caref|3.22|2.73|7012001|importonameless #2|12|paint|7|Home|813|prioughteing|N/A|707568911magenta0157|wheat|Dram|Unknown|5|oughtoughtpripri| +3312|AAAAAAAAOOMAAAAA|2001-10-27||Leaders guard generally police. Democratic witnesses may see efficiently questions. Clear, modern maps should not settle special, small elements. Final, public workers would not lose caref|0.61|2.73|7012001|corpcorp #1|12|rings|6|Jewelry|813|prioughteing|N/A|4797076purple2327495|spring|Pound|Unknown|10|ableoughtpripri| +3313|AAAAAAAABPMAAAAA|1997-10-27||Liberal authorities could not read in a journalists. Modules meet young lines. Please important devices allow further even equal measures; there important mome|0.66|0.48|3001002|amalgexporti #2|1|newborn|3|Children|553|priantianti|large|lawn5888629000828190|papaya|Oz|Unknown|72|prioughtpripri| +3314|AAAAAAAACPMAAAAA|1997-10-27|2000-10-26|Different, clear payments present seats. Perfect, great obligations might disrupt more victorian titles. Chemicals could come too from a customers. Then serious profits|2.46|0.95|9015003|scholarunivamalg #3|15|fiction|9|Books|993|prin stn st|N/A|483333085saddle89922|pink|Pound|Unknown|29|eseoughtpripri| +3315|AAAAAAAACPMAAAAA|2000-10-27||Years can dar|43.27|31.58|4001002|amalgedu pack #2|1|womens|4|Shoes|31|oughtpri|extra large|1638550284801yellow4|midnight|Gross|Unknown|33|antioughtpripri| +3316|AAAAAAAAEPMAAAAA|1997-10-27|1999-10-27|Significantly s|7.24|3.76|1004001|edu packamalg #1|4|swimwear|1|Women|617|ationoughtcally|small|2088222964601puff439|steel|Cup|Unknown|47|callyoughtpripri| +3317|AAAAAAAAEPMAAAAA|1999-10-28|2001-10-26|Either sophisticated countries worry enough part-time years. Standards could combine further gently major states. Finally decent expressions may smell even likely, blue men. Rational h|0.35|0.27|1003002|exportiamalg #2|3|maternity|1|Women|617|ationoughtcally|medium|2088222964601puff439|ghost|Each|Unknown|24|ationoughtpripri| +3318|AAAAAAAAEPMAAAAA|2001-10-27||Either sophisticated countries worry enough part-time years. Standards could combine further gently major states. Finally decent expressions may smell even likely, blue men. Rational h|1.29|0.83|4003001|exportiedu pack #1|3|kids|4|Shoes|617|ationoughtcally|medium|2088222964601puff439|sienna|Bundle|Unknown|15|eingoughtpripri| +3319|AAAAAAAAHPMAAAAA|1997-10-27||Aware, popular elements used to examine red enemies. Able, adequate countries go minutes; young,|4.74|2.03|4003002|exportiedu pack #2|3|kids|4|Shoes|421|oughtableese|petite|76715766pink17706466|peach|Ounce|Unknown|55|n stoughtpripri| +3320|AAAAAAAAIPMAAAAA|1997-10-27|2000-10-26|Safe savings may not use wonderfully after the organizations. Variable, present vis|56.43|42.32|5003001|exportischolar #1|3|pop|5|Music|886|callyeingeing|N/A|37683997snow09530146|brown|Dozen|Unknown|28|barablepripri| +3321|AAAAAAAAIPMAAAAA|2000-10-27||Then major techniques might forget gently. Prospects encourage |7.33|3.29|8016002|corpmaxi #2|16|golf|8|Sports|25|callyeingeing|N/A|12921326tomato942081|papaya|Box|Unknown|5|oughtablepripri| +3322|AAAAAAAAKPMAAAAA|1997-10-27|1999-10-27|Only british streets occur rural, english cats. Federal rooms could agree always through a doors. Now tall populations grow in a ideas. Changes can see invariably able, livin|2.31|1.06|2001001|amalgimporto #1|1|accessories|2|Men|122|ableableought|N/A|7395307142plum056309|peach|Lb|Unknown|55|ableablepripri| +3323|AAAAAAAAKPMAAAAA|1999-10-28|2001-10-26|High, free feet would show roughly full, inappropriate descriptions. Considerable, ready organisations lower particularly bare, black requests; other conditi|8.08|1.06|8006004|corpnameless #4|1|football|8|Sports|631|oughtprically|N/A|8364tan8587913769333|ghost|Lb|Unknown|40|priablepripri| +3324|AAAAAAAAKPMAAAAA|2001-10-27||High, free feet would show roughly full, inappropriate descriptions. Considerable, ready organisations lower particularly bare, black requests; other conditi|6.17|3.33|8006004|exportischolar #1|3|pop|5|Music|631|oughtprically|N/A|8364tan8587913769333|sandy|Lb|Unknown|16|eseablepripri| +3325|AAAAAAAANPMAAAAA|1997-10-27||Perhaps other claims might not stay in an levels. Positive, clear |3.18|2.48|2004002|edu packimporto #2|4|sports-apparel|2|Men|183|prieingought|large|28981752318papaya794|papaya|Unknown|Unknown|9|antiablepripri| +3326|AAAAAAAAOPMAAAAA|1997-10-27|2000-10-26|Selective years may dispense especially dual schools. Carefully concrete tan|52.25|19.85|9008003|namelessmaxi #3|8|romance|9|Books|118|eingoughtought|N/A|1339119pale656836256|salmon|Case|Unknown|5|callyablepripri| +3327|AAAAAAAAOPMAAAAA|2000-10-27||Selective years may dispense especially dual schools. Carefully concrete tan|3.88|2.28|10004013|edu packunivamalg #13|8|audio|10|Electronics|210|eingoughtought|N/A|44339035puff28130303|red|Ton|Unknown|18|ationablepripri| +3328|AAAAAAAAAANAAAAA|1997-10-27|1999-10-27|Less decent machines ought to withdraw special, real differences. British variations might no|9.08|2.90|4002001|importoedu pack #1|2|mens|4|Shoes|995|antin stn st|large|21midnight1414184044|slate|Bundle|Unknown|6|eingablepripri| +3329|AAAAAAAAAANAAAAA|1999-10-28|2001-10-26|Wom|4.48|2.90|3001002|amalgexporti #2|1|newborn|3|Children|995|antin stn st|large|759045linen526989586|powder|Oz|Unknown|9|n stablepripri| +3330|AAAAAAAAAANAAAAA|2001-10-27||Wom|2.21|1.54|3001002|exportinameless #1|13|wallpaper|7|Home|821|antin stn st|N/A|56024196931snow25563|turquoise|Bunch|Unknown|30|barpripripri| +3331|AAAAAAAADANAAAAA|1997-10-27||Fixed, able books write extraordinarily figures. Walls would not guarantee|1.94|0.62|7011002|amalgnameless #2|11|accent|7|Home|264|esecallyable|N/A|3661536wheat08642279|light|Dram|Unknown|39|oughtpripripri| +3332|AAAAAAAAEANAAAAA|1997-10-27|2000-10-26|Suddenly single businesses earn no doubt separate metres. Terms practise now cats. Key, significant companies meet ever main substances. Co|1.86|0.63|5003001|exportischolar #1|3|pop|5|Music|236|callypriable|N/A|222956snow0249058780|salmon|Gram|Unknown|13|ablepripripri| +3333|AAAAAAAAEANAAAAA|2000-10-27||Suddenly single businesses earn no doubt separate metres. Terms practise now cats. Key, significant companies meet ever main substances. Co|15.53|9.16|4001002|amalgedu pack #2|3|womens|4|Shoes|236|callypriable|medium|222956snow0249058780|steel|Oz|Unknown|28|pripripripri| +3334|AAAAAAAAGANAAAAA|1997-10-27|1999-10-27|Brown, united systems take very. Costs shall go almost to a houses. Social feelings ought to expect very. Grey, hot pr|5.85|2.98|3004001|edu packexporti #1|4|school-uniforms|3|Children|12|ableought|small|911941318712white249|white|Pound|Unknown|94|esepripripri| +3335|AAAAAAAAGANAAAAA|1999-10-28|2001-10-26|Brown, united systems take very. Costs shall go almost to a houses. Social feelings ought to expect very. Grey, hot pr|0.87|0.67|2003002|exportiimporto #2|4|pants|2|Men|12|ableought|medium|499438dodger19669872|slate|Ton|Unknown|24|antipripripri| +3336|AAAAAAAAGANAAAAA|2001-10-27||All right severe interests can clear both merely local animals. Useful damages may tell again human activities; needs say lat|8.80|3.43|3001001|amalgexporti #1|1|newborn|3|Children|82|ableeing|large|573820180018purple48|violet|Bunch|Unknown|32|callypripripri| +3337|AAAAAAAAJANAAAAA|1997-10-27||Outside advanced girls will become sadly senior marked t|0.54|0.38|1002002|importoamalg #2|2|fragrances|1|Women|917|ationoughtn st|extra large|0963848misty19265931|spring|Ounce|Unknown|4|ationpripripri| +3338|AAAAAAAAKANAAAAA|1997-10-27|2000-10-26|Fat colleagues ide|2.48|1.61|3004001|edu packexporti #1|4|school-uniforms|3|Children|357|ationantipri|small|1073seashell33672249|linen|Tbl|Unknown|19|eingpripripri| +3339|AAAAAAAAKANAAAAA|2000-10-27||Fat colleagues ide|8.40|1.61|3004001|brandnameless #10|4|hockey|8|Sports|357|ationantipri|N/A|1073seashell33672249|powder|Bundle|Unknown|28|n stpripripri| +3340|AAAAAAAAMANAAAAA|1997-10-27|1999-10-27|Heads fail only serious li|2.40|1.60|8006001|corpnameless #1|6|football|8|Sports|432|ablepriese|N/A|1148sky6839971377211|cyan|Box|Unknown|61|baresepripri| +3341|AAAAAAAAMANAAAAA|1999-10-28|2001-10-26|Happy, regular |50.06|1.60|7002002|importobrand #2|6|bedding|7|Home|432|ablepriese|N/A|4561698orchid4542335|peru|Bundle|Unknown|24|oughtesepripri| +3342|AAAAAAAAMANAAAAA|2001-10-27||National, colourful girls see effectively modern years. Birds obtain possibly in a enemies. Practical, dry mothers may spring normally immediate contexts. Instructions u|1.79|1.60|7002002|edu packedu pack #1|6|athletic|4|Shoes|110|baroughtought|extra large|4561698orchid4542335|sky|Tbl|Unknown|33|ableesepripri| +3343|AAAAAAAAPANAAAAA|1997-10-27||Relatively other towns|8.02|2.64|6013008|exportibrand #8|13|loose stones|6|Jewelry|288|eingeingable|N/A|81627seashell5853329|salmon|Ounce|Unknown|20|priesepripri| +3344|AAAAAAAAABNAAAAA|1997-10-27|2000-10-26|Motives shall inform current, potential contracts. Natural, official centres spend more than here free libraries. Poor, other possibilities want behind a knees. Still st|2.41|1.75|7009005|maxibrand #5|9|mattresses|7|Home|917|ationoughtn st|N/A|911765152snow7465302|pale|Bunch|Unknown|28|eseesepripri| +3345|AAAAAAAAABNAAAAA|2000-10-27||Motives shall inform current, potential contracts. Natural, official centres spend more than here free libraries. Poor, other possibilities want behind a knees. Still st|3.84|1.75|7009005|amalgscholar #2|9|rock|5|Music|917|ationoughtn st|N/A|911765152snow7465302|red|Ounce|Unknown|26|antiesepripri| +3346|AAAAAAAACBNAAAAA|1997-10-27|1999-10-27|Artists press. Years practice yesterday. |1.49|1.11|2004001|edu packimporto #1|4|sports-apparel|2|Men|390|barn stpri|medium|480dim27700845399173|peach|Case|Unknown|15|callyesepripri| +3347|AAAAAAAACBNAAAAA|1999-10-28|2001-10-26|Artists press. Years practice yesterday. |7.35|1.11|2004001|importoimporto #2|4|shirts|2|Men|878|eingationeing|N/A|480dim27700845399173|metallic|Lb|Unknown|36|ationesepripri| +3348|AAAAAAAACBNAAAAA|2001-10-27||Here soviet authorities would secure almost stupid good attacks. Dirty degrees will live over poor, lexical pp.. Provincial, ugly documents sh|5.65|1.11|6008005|namelesscorp #5|8|mens watch|6|Jewelry|878|eingationeing|N/A|khaki446235134701463|blush|Unknown|Unknown|55|eingesepripri| +3349|AAAAAAAAFBNAAAAA|1997-10-27||Shares might handle just throughout a sources|0.21|0.16|6009006|maxicorp #6|9|womens watch|6|Jewelry|630|barprically|N/A|powder00773651074603|peru|Dram|Unknown|5|n stesepripri| +3350|AAAAAAAAGBNAAAAA||2000-10-26|Big, complete degrees alleviate tomorrow elsewhere ||||corpcorp #7||||Jewelry|10|||99924468353709lime44||||17|barantipripri| +3351|AAAAAAAAGBNAAAAA|2000-10-27||Big, complete degrees alleviate tomorrow elsewhere |5.09|3.35|6006007|amalgimporto #2|6|accessories|2|Men|60|barcally|medium|99924468353709lime44|puff|Bunch|Unknown|9|oughtantipripri| +3352|AAAAAAAAIBNAAAAA|1997-10-27|1999-10-27|European, huma|2.20|0.79|2004001|edu packimporto #1|4|sports-apparel|2|Men|239|n stpriable|medium|8violet1541573102922|tomato|Unknown|Unknown|13|ableantipripri| +3353|AAAAAAAAIBNAAAAA|1999-10-28|2001-10-26|European, huma|5.21|4.58|9007006|brandmaxi #6|4|reference|9|Books|239|n stpriable|N/A|8violet1541573102922|slate|Tsp|Unknown|64|priantipripri| +3354|AAAAAAAAIBNAAAAA|2001-10-27||European, huma|2.62|1.75|9003007|exportimaxi #7|4|computers|9|Books|159|n stantiought|N/A|516356366rosy6287431|green|Pound|Unknown|6|eseantipripri| +3355|AAAAAAAALBNAAAAA|1997-10-27||Main games may move enough following, poor organisers; female flowers pretend hardly quantities; excessive drugs will|0.37|0.15|3003002|exportiexporti #2|3|toddlers|3|Children|122|ableableought|petite|86287695525257grey08|tan|Bundle|Unknown|20|antiantipripri| +3356|AAAAAAAAMBNAAAAA|1997-10-27|2000-10-26|Computers used to eat. Pink flowers maintain|91.00|64.61|4003001|exportiedu pack #1|3|kids|4|Shoes|416|callyoughtese|medium|00613739356567snow60|slate|N/A|Unknown|60|callyantipripri| +3357|AAAAAAAAMBNAAAAA|2000-10-27||Clearly financial scales stick simultaneously only other changes. Still unaware groups must make soon indirect, mutual rights. Lovely years give possible women. Well s|54.15|25.45|5002002|importoscholar #2|2|country|5|Music|114|callyoughtese|N/A|991745435119navy8642|gainsboro|Tsp|Unknown|18|ationantipripri| +3358|AAAAAAAAOBNAAAAA|1997-10-27|1999-10-27|Never lucky windows go mature aspects. Studies might run subsequently; likely, industrial facilities should not carve sufficient eyes; early, english benefits invi|1.41|1.22|7009009|maxibrand #9|9|mattresses|7|Home|294|esen stable|N/A|08lawn38271308771279|seashell|Gross|Unknown|5|eingantipripri| +3359|AAAAAAAAOBNAAAAA|1999-10-28|2001-10-26|New women will reassure courses. Rare, profound years must forgive single, sure pp.. Available homes wo|5.86|1.87|7009009|exportibrand #10|3|kids|7|Home|294|esen stable|N/A|9141310049213tan1970|tan|Ounce|Unknown|1|n stantipripri| +3360|AAAAAAAAOBNAAAAA|2001-10-27||New women will reassure courses. Rare, profound years must forgive single, sure pp.. Available homes wo|27.34|21.32|5004001|edu packscholar #1|4|classical|5|Music|294|esen stable|N/A|9141310049213tan1970|lemon|Dozen|Unknown|28|barcallypripri| +3361|AAAAAAAABCNAAAAA|1997-10-27||Even academic ages return different, sorry grounds. Even happy causes offer offices. Organic, remaining dreams go well around a plants. Once successful child|70.40|45.76|3003002|exportiexporti #2|3|toddlers|3|Children|79|n station|N/A|00244619615259tan807|slate|Dozen|Unknown|56|oughtcallypripri| +3362|AAAAAAAACCNAAAAA|1997-10-27|2000-10-26|Exotic, other patients retaliate too still bad sources. Little, italian villages support then foreign, similar women; statements see there responsible, able m|6.88|5.64|1001001|amalgamalg #1|1|dresses|1|Women|347|ationesepri|N/A|1404tan7322445292784|powder|Box|Unknown|28|ablecallypripri| +3363|AAAAAAAACCNAAAAA|2000-10-27||Names must not say however new issues. Nearly super years|2.88|5.64|1001001|edu packimporto #2|4|sports-apparel|2|Men|347|ationesepri|extra large|866891679830810red72|seashell|N/A|Unknown|91|pricallypripri| +3364|AAAAAAAAECNAAAAA|1997-10-27|1999-10-27|Simple countries meet less clothes. Doubts could take also clear, nece|6.95|5.14|10005016|scholarunivamalg #16|5|karoke|10|Electronics|245|antieseable|N/A|peach090555735312833|powder|Box|Unknown|50|esecallypripri| +3365|AAAAAAAAECNAAAAA|1999-10-28|2001-10-26|Rates indicate minor counci|51.01|24.48|10005016|importoedu pack #2|5|mens|4|Shoes|245|antieseable|medium|peach090555735312833|blush|N/A|Unknown|41|anticallypripri| +3366|AAAAAAAAECNAAAAA|2001-10-27||Events should no|8.69|6.34|10005016|importoscholar #1|2|country|5|Music|245|antieseable|N/A|peach090555735312833|seashell|Gross|Unknown|8|callycallypripri| +3367|AAAAAAAAHCNAAAAA|1997-10-27||Properly unable movements share even in a speakers. Frames may look partl|0.88|0.73|6003008|exporticorp #8|3|gold|6|Jewelry|43|priese|N/A|1753236183drab933147|thistle|Lb|Unknown|2|ationcallypripri| +3368|AAAAAAAAICNAAAAA|1997-10-27|2000-10-26|Capital shoulders live vari|56.18|30.33|9010009|univunivamalg #9|10|travel|9|Books|201|oughtbarable|N/A|5790729025495lime915|royal|Gross|Unknown|14|eingcallypripri| +3369|AAAAAAAAICNAAAAA|2000-10-27||Capital shoulders live vari|4.38|2.14|9010009|exportimaxi #10|10|computers|9|Books|316|callyoughtpri|N/A|5790729025495lime915|puff|Pallet|Unknown|42|n stcallypripri| +3370|AAAAAAAAKCNAAAAA|1997-10-27|1999-10-27|Immediate, fast rows must not take successful, right sons. As sign|9.44|4.43|5004001|edu packscholar #1|4|classical|5|Music|198|eingn stought|N/A|063764428tan62035930|peach|Oz|Unknown|32|barationpripri| +3371|AAAAAAAAKCNAAAAA|1999-10-28|2001-10-26|Immediate, fast rows must not take successful, right sons. As sign|2.14|4.43|3002002|importoexporti #2|4|infants|3|Children|345|eingn stought|petite|063764428tan62035930|lavender|Each|Unknown|29|oughtationpripri| +3372|AAAAAAAAKCNAAAAA|2001-10-27||Immediate, fast rows must not take successful, right sons. As sign|16.17|4.43|10006007|corpunivamalg #7|6|musical|10|Electronics|345|antiesepri|N/A|1673026389996powder2|yellow|Oz|Unknown|26|ableationpripri| +3373|AAAAAAAANCNAAAAA|1997-10-27||Super, aware taxes used to expect. Available, active falls provide. Awful hands may play ever|7.90|3.87|9008002|namelessmaxi #2|8|romance|9|Books|107|ationbarought|N/A|48547594045204mint84|royal|Bundle|Unknown|66|priationpripri| +3374|AAAAAAAAOCNAAAAA|1997-10-27|2000-10-26|Fie|4.71|1.88|10016009|corpamalgamalg #9|16|wireless|10|Electronics|518|eingoughtanti|N/A|0960909816786royal38|seashell|Carton|Unknown|30|eseationpripri| +3375|AAAAAAAAOCNAAAAA|2000-10-27||Committees shall build sure metres. Good, local banks may allow together eastern implications. Rigid, old rates ensure different, political courses. Wide, dead pieces last both today clinical |1.57|1.88|10016009|brandnameless #6|16|hockey|8|Sports|518|eingoughtanti|N/A|0960909816786royal38|royal|Tsp|Unknown|60|antiationpripri| +3376|AAAAAAAAADNAAAAA|1997-10-27|1999-10-27|Patients will step as able, new patients. Interesting gaps tell economic, respective meals. Relatively natural duties may not read most different indust|2.82|1.18|10012011|importoamalgamalg #11|12|monitors|10|Electronics|251|oughtantiable|N/A|364575831479floral98|pink|Cup|Unknown|1|callyationpripri| +3377|AAAAAAAAADNAAAAA|1999-10-28|2001-10-26|Hours may not allow just in a eyes. New, chief pubs find much italian, different forces. Governments discuss requirements. Tropical, open businesses get also later significant pounds; sites provi|6.67|1.18|10012011|edu packexporti #2|12|school-uniforms|3|Children|251|oughtantiable|medium|364575831479floral98|honeydew|Case|Unknown|27|ationationpripri| +3378|AAAAAAAAADNAAAAA|2001-10-27||Hours may not allow just in a eyes. New, chief pubs find much italian, different forces. Governments discuss requirements. Tropical, open businesses get also later significant pounds; sites provi|74.85|1.18|10012011|namelesscorp #3|8|mens watch|6|Jewelry|231|oughtantiable|N/A|369152royal521590789|tomato|Dozen|Unknown|24|eingationpripri| +3379|AAAAAAAADDNAAAAA|1997-10-27||Small, local shoulders could ensure just here teenage men; clearly increased exchanges may read a little attractive, silen|6.43|4.30|2001002|amalgimporto #2|1|accessories|2|Men|644|eseesecally|large|ivory457370549352452|seashell|Ounce|Unknown|19|n stationpripri| +3380|AAAAAAAAEDNAAAAA|1997-10-27|2000-10-26|Categories ought to read also on a questions. Small years bring tonight between the holes. Growing, total artists think too for a values; french winds|2.08|0.99|7015001|scholarnameless #1|15|tables|7|Home|597|ationn stanti|N/A|4226124719tan9202152|snow|Dozen|Unknown|27|bareingpripri| +3381|AAAAAAAAEDNAAAAA|2000-10-27||Large, executive authorities may widen worse children. Good floors go absolute charges. Steady loose days ensure appropriate, annual merchants; early long m|2.97|0.99|6015006|scholarbrand #6|15|custom|6|Jewelry|578|eingationanti|N/A|4226124719tan9202152|wheat|Pound|Unknown|95|oughteingpripri| +3382|AAAAAAAAGDNAAAAA|1997-10-27|1999-10-27|Public levels will not build then alone goods; rivals create rapidly feet. Below serious|0.30|0.19|3002001|importoexporti #1|2|infants|3|Children|207|ationbarable|large|24plum58554711003701|snow|Case|Unknown|1|ableeingpripri| +3383|AAAAAAAAGDNAAAAA|1999-10-28|2001-10-26|Related, local purposes make domestic, other records. Maximum drivers can refine less in the scholars. Private charges open however d|2.19|1.55|3002001|corpamalgamalg #1|2|wireless|10|Electronics|595|antin stanti|N/A|24plum58554711003701|lemon|Cup|Unknown|16|prieingpripri| +3384|AAAAAAAAGDNAAAAA|2001-10-27||Long, great parties|2.06|0.65|2003001|exportiimporto #1|2|pants|2|Men|454|eseantiese|petite|24plum58554711003701|tan|Dozen|Unknown|51|eseeingpripri| +3385|AAAAAAAAJDNAAAAA|1997-10-27||Devices might not survive at the goods. Genuine, necessary days make just |0.75|0.67|4003002|exportiedu pack #2|3|kids|4|Shoes|182|ableeingought|small|79985cornsilk0195125|tomato|Tsp|Unknown|72|antieingpripri| +3386|AAAAAAAAKDNAAAAA|1997-10-27|2000-10-26|Above american meanings shall get with a troops. Essentially physical professionals outweigh more. Obviou|7.74|3.40|6007003|brandcorp #3|7|pendants|6|Jewelry|310|baroughtpri|N/A|44403579601powder968|salmon|Box|Unknown|54|callyeingpripri| +3387|AAAAAAAAKDNAAAAA|2000-10-27||Above american meanings shall get with a troops. Essentially physical professionals outweigh more. Obviou|5.35|2.35|9014004|edu packunivamalg #4|7|sports|9|Books|321|oughtablepri|N/A|29045667285puff91566|puff|Tsp|Unknown|3|ationeingpripri| +3388|AAAAAAAAMDNAAAAA|1997-10-27|1999-10-27|Angles pro|9.09|2.81|8005009|scholarnameless #9|5|fitness|8|Sports|31|oughtpri|N/A|151079749wheat815980|violet|Ounce|Unknown|40|eingeingpripri| +3389|AAAAAAAAMDNAAAAA|1999-10-28|2001-10-26|Angles pro|8.55|2.81|5001002|amalgscholar #2|5|rock|5|Music|31|oughtpri|N/A|008620powder53355431|sky|Dozen|Unknown|22|n steingpripri| +3390|AAAAAAAAMDNAAAAA|2001-10-27||Effects can move even most famous tests; especially dark children see all difficult, linguistic months. Marginal, main areas could check educational changes. Simil|28.36|2.81|5001002|namelessnameless #1|8|outdoor|8|Sports|84|eseeing|N/A|008620powder53355431|tomato|Pallet|Unknown|2|barn stpripri| +3391|AAAAAAAAPDNAAAAA|1997-10-27||Local, unlikely bits sign completely. Areas feel only manufacturing legs. Amounts must go personal, very things; areas could take clo|5.20|4.10|9002008|importomaxi #8|2|business|9|Books|202|ablebarable|N/A|2905230303127puff333|tomato|Box|Unknown|7|oughtn stpripri| +3392|AAAAAAAAAENAAAAA|1997-10-27|2000-10-26|Different, busy lines see most |4.95|2.52|6008001|namelesscorp #1|8|mens watch|6|Jewelry|258|eingantiable|N/A|83sienna979906880454|honeydew|Dram|Unknown|10|ablen stpripri| +3393|AAAAAAAAAENAAAAA|2000-10-27||Different, busy lines see most |3.40|2.52|6008001|univbrand #2|8|jewelry boxes|6|Jewelry|195|antin stought|N/A|83sienna979906880454|moccasin|Bundle|Unknown|15|prin stpripri| +3394|AAAAAAAACENAAAAA|1997-10-27|1999-10-27|Urban generations lay sorry routes. Angry activities shall face however. Original words prove to the lights.|1.90|1.40|1001001|amalgamalg #1|1|dresses|1|Women|554|eseantianti|medium|74turquoise298595401|rosy|Pound|Unknown|19|esen stpripri| +3395|AAAAAAAACENAAAAA|1999-10-28|2001-10-26|Urban generations lay sorry routes. Angry activities shall face however. Original words prove to the lights.|8.60|1.40|2002002|importoimporto #2|2|shirts|2|Men|554|eseantianti|medium|74turquoise298595401|peru|Oz|Unknown|53|antin stpripri| +3396|AAAAAAAACENAAAAA|2001-10-27||Urban generations lay sorry routes. Angry activities shall face however. Original words prove to the lights.|4.94|3.26|2002002|edu packimporto #1|2|sports-apparel|2|Men|554|eseantianti|extra large|74turquoise298595401|wheat|Pallet|Unknown|30|callyn stpripri| +3397|AAAAAAAAFENAAAAA|1997-10-27||Thick, single subjects wait also. Often popular places could steer as supreme, able cities. Up|0.16|0.07|8010008|univmaxi #8|10|pools|8|Sports|700|barbaration|N/A|97606042974ghost5969|white|Pallet|Unknown|24|ationn stpripri| +3398|AAAAAAAAGENAAAAA|1997-10-27|2000-10-26|Genera|2.84|1.76|9003003|exportimaxi #3|3|computers|9|Books|92|ablen st|N/A|85563chartreuse90406|sandy|Cup|Unknown|47|eingn stpripri| +3399|AAAAAAAAGENAAAAA|2000-10-27||Improvements might use |0.97|0.47|9003003|exportiexporti #2|3|toddlers|3|Children|951|ablen st|large|85563chartreuse90406|saddle|Pallet|Unknown|54|n stn stpripri| +3400|AAAAAAAAIENAAAAA|1997-10-27|1999-10-27|Old others may not ask there operational cases; fast, new bars go over|7.25|5.22|10013001|exportiamalgamalg #1|13|stereo|10|Electronics|262|ablecallyable|N/A|04619151365155sky401|thistle|Case|Unknown|7|barbaresepri| +3401|AAAAAAAAIENAAAAA|1999-10-28|2001-10-26|Old others may not ask there operational cases; fast, new bars go over|5.30|5.22|7008002|namelessbrand #2|8|lighting|7|Home|127|ationableought|N/A|04619151365155sky401|steel|Dozen|Unknown|26|oughtbaresepri| +3402|AAAAAAAAIENAAAAA|2001-10-27||Old others may not ask there operational cases; fast, new bars go over|9.14|5.22|7008002|scholarbrand #3|8|blinds/shades|7|Home|617|ationableought|N/A|33seashell2815006090|linen|Bunch|Unknown|20|ablebaresepri| +3403|AAAAAAAALENAAAAA|1997-10-27||Progressive projects construct only ever only groups. Private responses allow even different, new animals; fellow, realistic i|5.77|4.09|4003002|exportiedu pack #2|3|kids|4|Shoes|173|priationought|medium|865452248208sandy439|burnished|N/A|Unknown|25|pribaresepri| +3404|AAAAAAAAMENAAAAA|1997-10-27|2000-10-26|Companies find financially substances. National, enormous conclusions might object here firms; exactly different jobs should complete; practices encourage really months. Necessary, previous recor|5.63|3.71|10012005|importoamalgamalg #5|12|monitors|10|Electronics|84|eseeing|N/A|172violet43143119695|blush|Oz|Unknown|60|esebaresepri| +3405|AAAAAAAAMENAAAAA|2000-10-27||Fairly english collections ought to rely |4.50|3.71|1001002|amalgamalg #2|1|dresses|1|Women|561|eseeing|medium|9794557seashell44397|tan|Cup|Unknown|15|antibaresepri| +3406|AAAAAAAAOENAAAAA|1997-10-27|1999-10-27|Companies will render only in the prices. Medium, australian others would not decide certain institutions; possible paintings may approach c|3.08|2.58|8003007|exportinameless #7|3|basketball|8|Sports|340|baresepri|N/A|11989992849392royal5|violet|Gram|Unknown|64|callybaresepri| +3407|AAAAAAAAOENAAAAA|1999-10-28|2001-10-26|Companies will render only in the prices. Medium, australian others would not decide certain institutions; possible paintings may approach c|3.52|1.40|8003007|edu packnameless #8|4|camping|8|Sports|553|priantianti|N/A|seashell844325362210|rosy|Oz|Unknown|71|ationbaresepri| +3408|AAAAAAAAOENAAAAA|2001-10-27||Able councillors would not show times. Other, big signals mus|8.09|1.40|7014009|edu packnameless #9|4|glassware|7|Home|553|priantianti|N/A|7034289741pale265739|hot|Tbl|Unknown|23|eingbaresepri| +3409|AAAAAAAABFNAAAAA|1997-10-27|||1.10|||namelessunivamalg #10|8|scanners|||689|||54682931177301snow44|light|||51|n stbaresepri| +3410|AAAAAAAACFNAAAAA|1997-10-27|2000-10-26|Slim, new banks give hence cups; single, likely periods feel difficult features. Other, original procedures play children. Low procedures will allow formerly statem|1.88|1.37|9012003|importounivamalg #3|12|home repair|9|Books|114|eseoughtought|N/A|1126037940orchid3608|yellow|Pound|Unknown|18|baroughtesepri| +3411|AAAAAAAACFNAAAAA|2000-10-27||Left professionals speak. Great, expected manufacturers get; civil inst|1.59|1.37|7014002|edu packnameless #2|14|glassware|7|Home|114|eseoughtought|N/A|05white4949832588263|slate|Box|Unknown|58|oughtoughtesepri| +3412|AAAAAAAAEFNAAAAA|1997-10-27|1999-10-27|However underlying particles achieve eventually southern, recent resources. Maybe mediterranean roles could let. Like, remarkable banks shall control great miles. Diplomatic ways used to forget more |4.77|1.57|3001001|amalgexporti #1|1|newborn|3|Children|608|eingbarcally|medium|326333peach296080186|green|Gram|Unknown|24|ableoughtesepri| +3413|AAAAAAAAEFNAAAAA|1999-10-28|2001-10-26|Large girls should buy under deep, long windows. Breasts prefer. Patients build. Wealthy newspapers read more just speakers; sure attitudes m|2.13|0.78|2001002|amalgimporto #2|1|accessories|2|Men|608|eingbarcally|medium|28211953401106peach5|white|Tsp|Unknown|2|prioughtesepri| +3414|AAAAAAAAEFNAAAAA|2001-10-27||Social, social others must happen most in a flames. Main, tory women guarantee for a procedures. Individual techni|3.12|0.93|2001002|exportiimporto #1|1|pants|2|Men|608|eingbarcally|petite|507plum5500969197312|sandy|Box|Unknown|12|eseoughtesepri| +3415|AAAAAAAAHFNAAAAA|1997-10-27||Afraid, new contexts contemplate ago for the buildings. Narrow, young games m|39.02|18.33|6007008|brandcorp #8|7|pendants|6|Jewelry|364|esecallypri|N/A|20pale46230148114850|pink|Oz|Unknown|30|antioughtesepri| +3416|AAAAAAAAIFNAAAAA|1997-10-27|2000-10-26|Acute, important performances afford. New, nuclear men used to assess again small results.|10.11|5.86|9006009|corpmaxi #9|6|parenting|9|Books|167|ationcallyought|N/A|37025604264tan988300|moccasin|Ounce|Unknown|14|callyoughtesepri| +3417|AAAAAAAAIFNAAAAA|2000-10-27||Requirements must not submit prime populations; other, miserable thousands like however persistent ti|4.29|5.86|9006009|brandunivamalg #1|7|personal|10|Electronics|334|esepripri|N/A|54679308013pale35416|seashell|Dozen|Unknown|55|ationoughtesepri| +3418|AAAAAAAAKFNAAAAA|1997-10-27|1999-10-27|New details cannot resort there red, soft females. Words assess really past students; issues may mean suddenly questions. Terms cope con|7.28|5.16|5002001|importoscholar #1|2|country|5|Music|425|antiableese|N/A|046sienna38858085216|orchid|Tsp|Unknown|42|eingoughtesepri| +3419|AAAAAAAAKFNAAAAA|1999-10-28|2001-10-26|New details cannot resort there red, soft females. Words assess really past students; issues may mean suddenly questions. Terms cope con|2.12|1.84|10012003|importoamalgamalg #3|2|monitors|10|Electronics|86|antiableese|N/A|655354606red58567147|navajo|Bundle|Unknown|15|n stoughtesepri| +3420|AAAAAAAAKFNAAAAA|2001-10-27||Sexual readers comment. Simply able boys might express largely other single months. Full meetings could not require. Often female beliefs will not|7.65|1.84|10012003|amalgscholar #1|1|rock|5|Music|162|ablecallyought|N/A|655354606red58567147|papaya|Bundle|Unknown|6|barableesepri| +3421|AAAAAAAANFNAAAAA|1997-10-27||Once again additional units might try therefore certain details. Rather white times should not |4.13|2.93|3004002|edu packexporti #2|4|school-uniforms|3|Children|686|callyeingcally|large|5757895330sandy77879|turquoise|Each|Unknown|60|oughtableesepri| +3422|AAAAAAAAOFNAAAAA|1997-10-27|2000-10-26|Favorite, economic eyes suggest d|4.86|4.03|1001001|amalgamalg #1|1|dresses|1|Women|156|callyantiought|small|851lemon065892270157|orchid|Dozen|Unknown|12|ableableesepri| +3423|AAAAAAAAOFNAAAAA|2000-10-27||Favorite, economic eyes suggest d|79.25|54.68|1001001|importoamalgamalg #7|1|monitors|10|Electronics|156|callyantiought|N/A|693656244almond49593|gainsboro|Carton|Unknown|36|priableesepri| +3424|AAAAAAAAAGNAAAAA|1997-10-27|1999-10-27|Large, green details used to answer quite difficult games. Indeed other components may take good sites. Public victims thi|9.77|3.32|1001001|amalgamalg #1|1|dresses|1|Women|68|eingcally|medium|saddle10973892251881|puff|Bundle|Unknown|1|eseableesepri| +3425|AAAAAAAAAGNAAAAA|1999-10-28|2001-10-26|Bodies lift more national contexts; parental arms get strict incidents. |4.77|3.32|10008009|namelessunivamalg #9|1|scanners|10|Electronics|132|ablepriought|N/A|395tomato51683960705|lemon|Dozen|Unknown|32|antiableesepri| +3426|AAAAAAAAAGNAAAAA|2001-10-27||Bodies lift more national contexts; parental arms get strict incidents. |5.45|2.12|3003001|exportiexporti #1|1|toddlers|3|Children|132|ablepriought|petite|395tomato51683960705|yellow|Lb|Unknown|19|callyableesepri| +3427|AAAAAAAADGNAAAAA|1997-10-27||Foreign studies ought to make inevitable, other ways. Great games sw|2.02|0.94|2004002|edu packimporto #2|4|sports-apparel|2|Men|40|barese|medium|873peach915640218543|papaya|Unknown|Unknown|24|ationableesepri| +3428|AAAAAAAAEGNAAAAA|1997-10-27|2000-10-26|Rich legs use so. Other pages avoid even stones. Members shall support central rivers. Strategies would not apply weapons. Large, |4.95|2.82|2004001|edu packimporto #1|4|sports-apparel|2|Men|147|ationeseought|petite|8982almond2765182707|ivory|Ton|Unknown|2|eingableesepri| +3429|AAAAAAAAEGNAAAAA|2000-10-27||Rich legs use so. Other pages avoid even stones. Members shall support central rivers. Strategies would not apply weapons. Large, |8.24|2.63|2004001|importoscholar #2|2|country|5|Music|40|ationeseought|N/A|salmon87964546980613|snow|Pallet|Unknown|3|n stableesepri| +3430|AAAAAAAAGGNAAAAA|1997-10-27|1999-10-27|As great roads wave systems. Elections will obtain still silent areas. Responsible, contemporary writers may not take |9.77|5.37|1002001|importoamalg #1|2|fragrances|1|Women|112|ableoughtought|N/A|785wheat047679640642|rosy|Pound|Unknown|66|barpriesepri| +3431|AAAAAAAAGGNAAAAA|1999-10-28|2001-10-26|As great roads wave systems. Elections will obtain still silent areas. Responsible, contemporary writers may not take |1.45|0.85|9010012|univunivamalg #12|10|travel|9|Books|140|bareseought|N/A|1azure82831137780597|snow|Gram|Unknown|23|oughtpriesepri| +3432|AAAAAAAAGGNAAAAA|2001-10-27||As great roads wave systems. Elections will obtain still silent areas. Responsible, contemporary writers may not take |7.22|0.85|4004001|edu packedu pack #1|10|athletic|4|Shoes|447|ationeseese|medium|61816ivory1509879149|saddle|Ounce|Unknown|13|ablepriesepri| +3433|AAAAAAAAJGNAAAAA|1997-10-27||Just senior recommendations serve important, other payments. Centres make orange images. As important tons guarantee extremely just massive letters. Supposed, western guests mig|2.64|2.19|4004002|edu packedu pack #2|4|athletic|4|Shoes|218|eingoughtable|large|49224frosted31196213|lawn|Box|Unknown|19|pripriesepri| +3434|AAAAAAAAKGNAAAAA|1997-10-27|2000-10-26|Facilities shall receive also national scores. Military, international appeals understand then. Just|70.59|46.58|5004001|edu packscholar #1|4|classical|5|Music|110|baroughtought|N/A|7265295488plum932415|peach|Bunch|Unknown|97|esepriesepri| +3435|AAAAAAAAKGNAAAAA|2000-10-27||Facilities shall receive also national scores. Military, international appeals understand then. Just|5.81|3.42|5001002|amalgscholar #2|1|rock|5|Music|110|baroughtought|N/A|7265295488plum932415|plum|Dram|Unknown|10|antipriesepri| +3436|AAAAAAAAMGNAAAAA|1997-10-27|1999-10-27|Options shall afford excessive, human states. Available groups hold only tales. Estimated beliefs appreciate for every critics. However environm|3.03|2.30|10008003|namelessunivamalg #3|8|scanners|10|Electronics|299|n stn stable|N/A|9966439211purple6752|spring|Oz|Unknown|20|callypriesepri| +3437|AAAAAAAAMGNAAAAA|1999-10-28|2001-10-26|Options shall afford excessive, human states. Available groups hold only tales. Estimated beliefs appreciate for every critics. However environm|9.43|2.30|3001002|amalgexporti #2|8|newborn|3|Children|279|n stationable|small|9966439211purple6752|tan|Cup|Unknown|27|ationpriesepri| +3438|AAAAAAAAMGNAAAAA|2001-10-27||Criminal designs think right other exports. Video-taped actions indicate later normal topics. Pieces could fall much good remains; acre|0.45|2.30|3001002|amalgexporti #1|1|newborn|3|Children|427|n stationable|medium|528786005pale1995447|lavender|Case|Unknown|14|eingpriesepri| +3439|AAAAAAAAPGNAAAAA|1997-10-27||Partly available qualificat|0.96|0.37|9013008|exportiunivamalg #8|13|self-help|9|Books|418|eingoughtese|N/A|papaya16775301230625|seashell|Lb|Unknown|27|n stpriesepri| +3440|AAAAAAAAAHNAAAAA|1997-10-27|2000-10-26|Moreover social skills may go more long responses. Following eve|7.54|5.50|7002001|importobrand #1|2|bedding|7|Home|317|ationoughtpri|N/A|20519796maroon385379|royal|Gram|Unknown|25|bareseesepri| +3441|AAAAAAAAAHNAAAAA|2000-10-27||Recent, different events read quite outside things. Royal, current bases used to say much available companies. Of course direct cars shall not win moving results. Total, welcome f|27.63|5.50|7002001|importounivamalg #8|2|camcorders|10|Electronics|317|ationoughtpri|N/A|20519796maroon385379|wheat|Cup|Unknown|32|oughteseesepri| +3442|AAAAAAAACHNAAAAA|1997-10-27|1999-10-27|Long, cruel men should leave economic, key arguments. Extensive, avai|1.06|0.48|6006003|corpcorp #3|6|rings|6|Jewelry|260|barcallyable|N/A|823red00061361475500|tomato|Carton|Unknown|25|ableeseesepri| +3443|AAAAAAAACHNAAAAA|1999-10-28|2001-10-26|Beneficial trends obtain|4.38|2.49|6006003|edu packimporto #2|6|sports-apparel|2|Men|260|barcallyable|medium|342563389peach536920|spring|Box|Unknown|11|prieseesepri| +3444|AAAAAAAACHNAAAAA|2001-10-27||Beneficial trends obtain|6.97|6.27|6006003|corpunivamalg #11|6|musical|10|Electronics|651|oughtantically|N/A|883973230041251tan71|tomato|Pallet|Unknown|97|eseeseesepri| +3445|AAAAAAAAFHNAAAAA|1997-10-27||Common children could leave nearly so labour officers. Just decisive orders distinguish essentially regul|7.60|4.18|4002002|importoedu pack #2|2|mens|4|Shoes|473|priationese|medium|sandy894144031999132|linen|Box|Unknown|57|antieseesepri| +3446|AAAAAAAAGHNAAAAA|1997-10-27|2000-10-26|Atomic pp. might disappear as. Figures discuss men. Specific, local rivers might replace eyes. Safe cars take final services; old troops|6.29|3.83|7002007|importobrand #7|2|bedding|7|Home|20|barable|N/A|172834555634ghost246|salmon|Bundle|Unknown|2|callyeseesepri| +3447|AAAAAAAAGHNAAAAA|2000-10-27||Days like soft records. Suitable plans smile by a studies. Royal proposals should not |99.89|58.93|7002008|importobrand #8|2|bedding|7|Home|159|n stantiought|N/A|789248slate341635169|pink|Carton|Unknown|10|ationeseesepri| +3448|AAAAAAAAIHNAAAAA|1997-10-27|1999-10-27|Losses must spawn foreign players. Passengers can clear here low residents. Ready, bottom women ought to manage r|2.04|0.77|8006009|corpnameless #9|6|football|8|Sports|83|prieing|N/A|2461659551744papaya7|lavender|Pound|Unknown|7|eingeseesepri| +3449|AAAAAAAAIHNAAAAA|1999-10-28|2001-10-26|Once again recent prem|0.83|0.62|8006009|importonameless #10|6|baseball|8|Sports|126|prieing|N/A|2461659551744papaya7|pale|Gross|Unknown|23|n steseesepri| +3450|AAAAAAAAIHNAAAAA|2001-10-27||Once again recent prem|2.51|1.60|8006009|importoamalg #1|6|fragrances|1|Women|610|prieing|large|82998gainsboro712063|violet|N/A|Unknown|28|barantiesepri| +3451|AAAAAAAALHNAAAAA|1997-10-27||Other opportunities deter together measures. Women must show simply|0.19|0.16|6009004|maxicorp #4|9|womens watch|6|Jewelry|179|n stationought|N/A|68steel5100963044650|frosted|Lb|Unknown|17|oughtantiesepri| +3452|AAAAAAAAMHNAAAAA|1997-10-27|2000-10-26|So strong eyes offer proper, dead men. Stones plan so keen p|5.28|2.85|1003001|exportiamalg #1|3|maternity|1|Women|904|esebarn st|petite|397289pink0025331896|spring|Tsp|Unknown|65|ableantiesepri| +3453|AAAAAAAAMHNAAAAA|2000-10-27||So strong eyes offer proper, dead men. Stones plan so keen p|0.77|0.30|5002002|importoscholar #2|3|country|5|Music|993|prin stn st|N/A|397289pink0025331896|indian|Dozen|Unknown|13|priantiesepri| +3454|AAAAAAAAOHNAAAAA|1997-10-27|1999-10-27|German charges destroy later s|6.78|3.52|7013005|exportinameless #5|13|wallpaper|7|Home|324|eseablepri|N/A|9258royal16626178984|tomato|Unknown|Unknown|23|eseantiesepri| +3455|AAAAAAAAOHNAAAAA|1999-10-28|2001-10-26|German charges destroy later s|9.67|4.35|2001002|amalgimporto #2|1|accessories|2|Men|324|eseablepri|small|9258royal16626178984|sienna|Ounce|Unknown|16|antiantiesepri| +3456|AAAAAAAAOHNAAAAA|2001-10-27||German charges destroy later s|0.58|4.35|10010006|univamalgamalg #6|10|memory|10|Electronics|324|eseablepri|N/A|9258royal16626178984|gainsboro|Gram|Unknown|44|callyantiesepri| +3457|AAAAAAAABINAAAAA|1997-10-27||Below invisi|9.59|7.57|9011002|amalgunivamalg #2|11|cooking|9|Books|505|antibaranti|N/A|49247magenta63586562|sandy|Box|Unknown|62|ationantiesepri| +3458|AAAAAAAACINAAAAA|1997-10-27|2000-10-26|Payme|9.54|3.81|4002001|importoedu pack #1|2|mens|4|Shoes|937|ationprin st|economy|164941violet52281157|rose|Bunch|Unknown|4|eingantiesepri| +3459|AAAAAAAACINAAAAA|2000-10-27||Payme|0.77|3.81|7003010|exportibrand #10|2|kids|7|Home|937|ationprin st|N/A|164941violet52281157|sandy|Gram|Unknown|2|n stantiesepri| +3460|AAAAAAAAEINAAAAA|1997-10-27|1999-10-27|Waves would despair again adjacent terms. Shops can mean away in a unions. Regular plants flourish so other grounds. Officers shall ensure perhaps supposed officers. British, political m|27.95|17.04|5004001|edu packscholar #1|4|classical|5|Music|287|ationeingable|N/A|56lawn63480919277749|azure|Tsp|Unknown|23|barcallyesepri| +3461|AAAAAAAAEINAAAAA|1999-10-28|2001-10-26|Waves would despair again adjacent terms. Shops can mean away in a unions. Regular plants flourish so other grounds. Officers shall ensure perhaps supposed officers. British, political m|1.91|17.04|5004001|importoscholar #2|4|country|5|Music|58|einganti|N/A|41228212tomato585397|turquoise|Tbl|Unknown|26|oughtcallyesepri| +3462|AAAAAAAAEINAAAAA|2001-10-27||Waves would despair again adjacent terms. Shops can mean away in a unions. Regular plants flourish so other grounds. Officers shall ensure perhaps supposed officers. British, political m|2.63|2.18|5004001|importoexporti #1|2|infants|3|Children|224|einganti|large|41228212tomato585397|salmon|Tbl|Unknown|40|ablecallyesepri| +3463|AAAAAAAAHINAAAAA|1997-10-27||Great applications cannot agree better possible classes. Then big implications plan so british, practical interviews. Hours used to get totally common, familiar camps. Then local m|2.93|2.05|5002002|importoscholar #2|2|country|5|Music|279|n stationable|N/A|1322062867136puff683|beige|Carton|Unknown|14|pricallyesepri| +3464|AAAAAAAAIINAAAAA|1997-10-27|2000-10-26|Inland, royal areas make far by a officers. Helpful p|91.95|54.25|8010005|univmaxi #5|10|pools|8|Sports|895|antin steing|N/A|5233348turquoise7832|peach|Dram|Unknown|5|esecallyesepri| +3465|AAAAAAAAIINAAAAA|2000-10-27||As young offences might see more run|0.73|0.24|5004002|edu packscholar #2|10|classical|5|Music|895|antin steing|N/A|5233348turquoise7832|navy|Gross|Unknown|13|anticallyesepri| +3466|AAAAAAAAKINAAAAA|1997-10-27|1999-10-27|Poor windows bite again in a areas. Breast|3.43|1.85|6001003|amalgcorp #3|1|birdal|6|Jewelry|139|n stpriought|N/A|spring52930894893215|turquoise|Case|Unknown|2|callycallyesepri| +3467|AAAAAAAAKINAAAAA|1999-10-28|2001-10-26|Less interested publications shall put services; services like by the judges. About new persons provide too advisory pieces. Now single mus|5.08|1.85|6002004|importocorp #4|2|diamonds|6|Jewelry|388|eingeingpri|N/A|82638135tan092359566|tan|Bunch|Unknown|14|ationcallyesepri| +3468|AAAAAAAAKINAAAAA|2001-10-27||Over rare systems could scream accounts. Lac|1.95|0.79|6002004|edu packexporti #1|4|school-uniforms|3|Children|388|eingeingpri|petite|477644576058tomato32|royal|Ounce|Unknown|4|eingcallyesepri| +3469|AAAAAAAANINAAAAA|1997-10-27||Potential, unknown surfaces will not comment indeed only only clothes. Magistrates move only, elderly jobs. Similar curtains work; open walls may propose now joint systems. Common, mass drivers find s|4.32|2.85|2002002|importoimporto #2|2|shirts|2|Men|528|eingableanti|large|68664669aquamarine64|lawn|Box|Unknown|19|n stcallyesepri| +3470|AAAAAAAAOINAAAAA|1997-10-27|2000-10-26|Therefore able arts tell yellow sites. Affairs buy ethnic, central efforts. Activities wish. Just socialist feet acknowledge more operational grounds. Perhaps obvious vehicles shall find clear |3.98|3.22|5001001|amalgscholar #1|1|rock|5|Music|536|callyprianti|N/A|99816241912646peru65|sienna|Case|Unknown|28|barationesepri| +3471|AAAAAAAAOINAAAAA|2000-10-27||Proper, private women ought to deduce just again developing bishops. Sensible, successful weeks join ready, rough police. Certain areas used to sing more now close complaints. Whole, semantic ter|0.51|0.41|4002002|importoedu pack #2|1|mens|4|Shoes|536|callyprianti|petite|pale1048644518885753|rosy|Box|Unknown|22|oughtationesepri| +3472|AAAAAAAAAJNAAAAA|1997-10-27|1999-10-27|Individual personnel might check at least afraid, primary parts. Scientific years choose parliamentary, english products; designs cannot benefit all. Able, standard des|9.85|6.59|10004005|edu packunivamalg #5|4|audio|10|Electronics|990|barn stn st|N/A|5909128566snow627111|salmon|Bunch|Unknown|26|ableationesepri| +3473|AAAAAAAAAJNAAAAA|1999-10-28|2001-10-26|Visual, popular assets require fully with a prizes. Certainly high decisions fill surprising, classical orders. Commercial men might cut certainly capable tim|7.80|6.31|10004005|univnameless #4|10|flatware|7|Home|990|barn stn st|N/A|544drab7274469152377|tan|Unknown|Unknown|37|priationesepri| +3474|AAAAAAAAAJNAAAAA|2001-10-27||Too ethnic items lie to |2.00|6.31|10004005|corpunivamalg #7|16|mystery|9|Books|990|barn stn st|N/A|11211811320093plum53|peru|Gross|Unknown|30|eseationesepri| +3475|AAAAAAAADJNAAAAA|1997-10-27||So expert streets can back until the than|6.21|1.86|10013008|exportiamalgamalg #8|13|stereo|10|Electronics|110|baroughtought|N/A|1668957salmon7437060|wheat|Box|Unknown|10|antiationesepri| +3476|AAAAAAAAEJNAAAAA|1997-10-27|2000-10-26|Communications should know. Namely mere floors shall not land more ne|3.50|1.29|6002005|importocorp #5|2|diamonds|6|Jewelry|26|callyable|N/A|3purple6909254755981|wheat|Box|Unknown|79|callyationesepri| +3477|AAAAAAAAEJNAAAAA|2000-10-27||Final men may improve to the|4.04|1.29|3001002|amalgexporti #2|1|newborn|3|Children|26|callyable|petite|0812377181110white48|yellow|Each|Unknown|58|ationationesepri| +3478|AAAAAAAAGJNAAAAA|1997-10-27|1999-10-27|Wild agents shall not put ago. So important associations shall regard also into a patients. Happy materials think as friends. Added assumptions pick. Goo|1.04|0.58|3001001|amalgexporti #1|1|newborn|3|Children|764|esecallyation|economy|5816059sienna9131206|smoke|Tsp|Unknown|66|eingationesepri| +3479|AAAAAAAAGJNAAAAA|1999-10-28|2001-10-26|Wild agents shall not put ago. So important associations shall regard also into a patients. Happy materials think as friends. Added assumptions pick. Goo|3.65|1.75|3001001|namelessmaxi #12|8|romance|9|Books|270|barationable|N/A|5816059sienna9131206|snow|Gross|Unknown|29|n stationesepri| +3480|AAAAAAAAGJNAAAAA|2001-10-27||Original, late individuals mind however main rights. Accounts cannot belong then possible tables. Also active businesses would stress. Joint interests stop |2.88|1.75|7015001|scholarnameless #1|8|tables|7|Home|270|barationable|N/A|62826026640672puff96|tomato|Carton|Unknown|23|bareingesepri| +3481|AAAAAAAAJJNAAAAA|1997-10-27||Supreme years cannot get continually difficulties; fun kinds yield somewhat still old concepts. Different, appropriate hours ma|3.44|2.06|6008002|namelesscorp #2|8|mens watch|6|Jewelry|528|eingableanti|N/A|3426792azure49553652|orange|Bunch|Unknown|45|oughteingesepri| +3482|AAAAAAAAKJNAAAAA|1997-10-27|2000-10-26|Pp. should build white circumstances. Institutions cannot rest hardly. Minimum, personal goals will experi|2.86|2.23|9004003|edu packmaxi #3|4|entertainments|9|Books|160|barcallyought|N/A|47699134slate1793608|powder|Dozen|Unknown|58|ableeingesepri| +3483|AAAAAAAAKJNAAAAA|2000-10-27||Often new communities sell ready, particular pieces. Profits shall use. Around direct effects might not live less pr|7.55|2.23|9004004|edu packmaxi #4|4|entertainments|9|Books|334|esepripri|N/A|0sienna2700400834034|spring|Ounce|Unknown|15|prieingesepri| +3484|AAAAAAAAMJNAAAAA|1997-10-27|1999-10-27|Ways ought to use so armed, straight operators; inc, only techniques must distinguish never usual authorities. Moral projects show however. Goods will take new, physical cultures. Sufficient|9.15|2.92|8012005|importomaxi #5|12|guns|8|Sports|449|n steseese|N/A|044270502269pink2598|peru|Bunch|Unknown|39|eseeingesepri| +3485|AAAAAAAAMJNAAAAA|1999-10-28|2001-10-26|Ways ought to use so armed, straight operators; inc, only techniques must distinguish never usual authorities. Moral projects show however. Goods will take new, physical cultures. Sufficient|0.53|0.42|8012005|edu packbrand #6|14|estate|6|Jewelry|137|n steseese|N/A|044270502269pink2598|rosy|Each|Unknown|11|antieingesepri| +3486|AAAAAAAAMJNAAAAA|2001-10-27||Ways ought to use so armed, straight operators; inc, only techniques must distinguish never usual authorities. Moral projects show however. Goods will take new, physical cultures. Sufficient|1.10|0.63|5001001|amalgscholar #1|14|rock|5|Music|122|ableableought|N/A|044270502269pink2598|rose|Ounce|Unknown|59|callyeingesepri| +3487|AAAAAAAAPJNAAAAA|1997-10-27||Ultimately historical roots could not sell. British boys would fill in the light of a communications. Big, wild structures must get simply diffe|9.89|3.65|6001008|amalgcorp #8|1|birdal|6|Jewelry|318|eingoughtpri|N/A|8steel29215160630776|steel|Each|Unknown|56|ationeingesepri| +3488|AAAAAAAAAKNAAAAA|1997-10-27|2000-10-26|Crucial, safe rights used to endure also nuclear days. Able members reply|6.79|2.44|7001009|amalgbrand #9|1|bathroom|7|Home|107|ationbarought|N/A|977707almond09284529|papaya|Dozen|Unknown|49|eingeingesepri| +3489|AAAAAAAAAKNAAAAA|2000-10-27||Crucial, safe rights used to endure also nuclear days. Able members reply|12.53|11.27|7001009|importoexporti #2|1|infants|3|Children|107|ationbarought|extra large|9312786995139919tan2|pale|Bundle|Unknown|44|n steingesepri| +3490|AAAAAAAACKNAAAAA|1997-10-27|1999-10-27|Everyday, statutory muscles|3.86|2.39|1002001|importoamalg #1|2|fragrances|1|Women|249|n steseable|medium|0598783759snow753373|peru|Pallet|Unknown|54|barn stesepri| +3491|AAAAAAAACKNAAAAA|1999-10-28|2001-10-26|International, capital comments used to think relatively. Anxious things ensure violently that national findings. At least full surveys would seek.|7.41|4.74|1002001|scholarmaxi #12|5|history|9|Books|249|n steseable|N/A|0598783759snow753373|plum|Lb|Unknown|17|oughtn stesepri| +3492|AAAAAAAACKNAAAAA|2001-10-27||Far distant days might not take there low employees. Condit|4.37|4.74|1002001|importoexporti #1|5|infants|3|Children|443|n steseable|extra large|9781481924puff720923|purple|Unknown|Unknown|15|ablen stesepri| +3493|AAAAAAAAFKNAAAAA|1997-10-27||Possible letters co|6.54|4.25|5004002|edu packscholar #2|4|classical|5|Music|283|prieingable|N/A|7963sky5300353883904|midnight|Unknown|Unknown|10|prin stesepri| +3494|AAAAAAAAGKNAAAAA|1997-10-27|2000-10-26|Top children used to forget nuclear, initial words. For example overall regions shall not fix under inland bad women. Visible matters deny large, single |73.62|59.63|3004001|edu packexporti #1|4|school-uniforms|3|Children|236|callypriable|extra large|09798438moccasin4179|yellow|Tsp|Unknown|12|esen stesepri| +3495|AAAAAAAAGKNAAAAA|2000-10-27||Top children used to forget nuclear, initial words. For example overall regions shall not fix under inland bad women. Visible matters deny large, single |0.70|0.54|3004001|edu packcorp #8|4|bracelets|6|Jewelry|236|callypriable|N/A|09798438moccasin4179|rosy|Gross|Unknown|60|antin stesepri| +3496|AAAAAAAAIKNAAAAA|1997-10-27|1999-10-27|Careful activities shall drop yet personnel. Useful, difficult offices should bear relatively on a positions. Nevertheless|5.08|3.30|5001001|amalgscholar #1|1|rock|5|Music|881|oughteingeing|N/A|55misty6367248627860|sienna|Tbl|Unknown|55|callyn stesepri| +3497|AAAAAAAAIKNAAAAA|1999-10-28|2001-10-26|Careful activities shall drop yet personnel. Useful, difficult offices should bear relatively on a positions. Nevertheless|8.97|3.76|5001001|exportibrand #8|3|kids|7|Home|881|oughteingeing|N/A|62904sky664450654911|maroon|Tsp|Unknown|41|ationn stesepri| +3498|AAAAAAAAIKNAAAAA|2001-10-27||Careful activities shall drop yet personnel. Useful, difficult offices should bear relatively on a positions. Nevertheless|0.43|0.18|2003001|exportiimporto #1|3|pants|2|Men|881|oughteingeing|small|52902white6712485374|tan|Ounce|Unknown|96|eingn stesepri| +3499|AAAAAAAALKNAAAAA|1997-10-27||Political, widespread buses take so impossible voices. Buildings give adequately pas|3.06|1.04|7015010|scholarnameless #10|15|tables|7|Home|137|ationpriought|N/A|23242852saddle760520|rose|Gram|Unknown|20|n stn stesepri| +3500|AAAAAAAAMKNAAAAA|1997-10-27|2000-10-26|Current sheets impose through a times. Political, great councils sound sensitive, civil probl|0.22|0.11|5003001|exportischolar #1|3|pop|5|Music|99|n stn st|N/A|96570304256medium703|steel|Pallet|Unknown|17|barbarantipri| +3501|AAAAAAAAMKNAAAAA|2000-10-27||Already private diseases ought to lick distinctly sports. Also blind attempts could not fail m|9.43|0.11|5003001|importoimporto #2|3|shirts|2|Men|99|n stn st|small|33171gainsboro600945|snow|Dram|Unknown|64|oughtbarantipri| +3502|AAAAAAAAOKNAAAAA|1997-10-27|1999-10-27|Special, spare women let. There social americans secure apparently good women. Anxious sides would co-ordinate yet. Houses may marry da|72.54|47.87|5004001|edu packscholar #1|4|classical|5|Music|269|n stcallyable|N/A|30905190powder170706|sienna|Box|Unknown|21|ablebarantipri| +3503|AAAAAAAAOKNAAAAA|1999-10-28|2001-10-26|Special, spare women let. There social americans secure apparently good women. Anxious sides would co-ordinate yet. Houses may marry da|1.23|47.87|2002002|importoimporto #2|4|shirts|2|Men|25|n stcallyable|extra large|2331934122502red2450|thistle|Unknown|Unknown|10|pribarantipri| +3504|AAAAAAAAOKNAAAAA|2001-10-27||Similar, right pupils may|1.99|47.87|7002005|importobrand #5|2|bedding|7|Home|25|antiable|N/A|2331934122502red2450|sienna|Gram|Unknown|3|esebarantipri| +3505|AAAAAAAABLNAAAAA|1997-10-27||Assets would take. Then great fingers develop en|7.78|6.37|8016006|corpmaxi #6|16|golf|8|Sports|44|eseese|N/A|3668896170775peach87|lime|Cup|Unknown|3|antibarantipri| +3506|AAAAAAAACLNAAAAA|1997-10-27|2000-10-26|At all public areas object |75.67|34.80|9005003|scholarmaxi #3|5|history|9|Books|462|ablecallyese|N/A|99069677linen3193108|spring|Gram|Unknown|6|callybarantipri| +3507|AAAAAAAACLNAAAAA|2000-10-27||At all public areas object |1.63|34.80|6005004|scholarcorp #4|5|earings|6|Jewelry|462|ablecallyese|N/A|99069677linen3193108|peach|Carton|Unknown|91|ationbarantipri| +3508|AAAAAAAAELNAAAAA|1997-10-27|1999-10-27|Appropriate, american floors |84.90|37.35|3004001|edu packexporti #1|4|school-uniforms|3|Children|983|prieingn st|extra large|9024547salmon6760790|royal|Dram|Unknown|93|eingbarantipri| +3509|AAAAAAAAELNAAAAA|1999-10-28|2001-10-26|Appropriate, american floors |3.73|37.35|3004001|brandmaxi #6|4|reference|9|Books|983|prieingn st|N/A|9024547salmon6760790|ivory|Dram|Unknown|25|n stbarantipri| +3510|AAAAAAAAELNAAAAA|2001-10-27||Accused, conventional plants must result just western details. Yet major surroundings think in the activities. Amounts advise less base,|9.63|8.37|7004001|edu packbrand #1|4|curtains/drapes|7|Home|983|prieingn st|N/A|22misty2798838977710|plum|Dram|Unknown|7|baroughtantipri| +3511|AAAAAAAAHLNAAAAA|1997-10-27||Already recent teachers must receive from the d|2.18|1.06|6015008|scholarbrand #8|15|custom|6|Jewelry|557|ationantianti|N/A|54232549slate9717581|royal|Case|Unknown|41|oughtoughtantipri| +3512|AAAAAAAAILNAAAAA|1997-10-27|2000-10-26|Models used to keep bad|6.93|6.02|2001001|amalgimporto #1|1|accessories|2|Men|574|eseationanti|medium|65958651190powder899|snow|Gross|Unknown|59|ableoughtantipri| +3513|AAAAAAAAILNAAAAA|2000-10-27||Here electronic children should rise somehow public resources. Yet female came|4.58|1.83|3004002|edu packexporti #2|4|school-uniforms|3|Children|65|antically|small|65958651190powder899|navajo|Each|Unknown|40|prioughtantipri| +3514|AAAAAAAAKLNAAAAA|1997-10-27|1999-10-27|Days come to a books. Natural, yellow beds allow economic shares. Back, german days might think animals. Jobs mention green, busy words. Continuing, persistent acti|5.19|3.37|7013005|exportinameless #5|13|wallpaper|7|Home|504|esebaranti|N/A|259126olive509621452|royal|Cup|Unknown|64|eseoughtantipri| +3515|AAAAAAAAKLNAAAAA|1999-10-28|2001-10-26|Days come to a books. Natural, yellow beds allow economic shares. Back, german days might think animals. Jobs mention green, busy words. Continuing, persistent acti|6.59|3.82|7013005|edu packscholar #2|4|classical|5|Music|476|callyationese|N/A|16222373081rose92000|turquoise|Gross|Unknown|48|antioughtantipri| +3516|AAAAAAAAKLNAAAAA|2001-10-27||Days come to a books. Natural, yellow beds allow economic shares. Back, german days might think animals. Jobs mention green, busy words. Continuing, persistent acti|0.42|3.82|7013005|importounivamalg #15|2|camcorders|10|Electronics|476|callyationese|N/A|32005139honeydew8839|pale|Bundle|Unknown|18|callyoughtantipri| +3517|AAAAAAAANLNAAAAA|1997-10-27||Low eyes conduct precise women. Capable toys may |4.60|3.22|3004002|edu packexporti #2|4|school-uniforms|3|Children|158|eingantiought|medium|8889160lace882976088|snow|N/A|Unknown|87|ationoughtantipri| +3518|AAAAAAAAOLNAAAAA|1997-10-27|2000-10-26|Ways ought to come vice versa capable claims. Commer|90.51|52.49|5001001|amalgscholar #1|1|rock|5|Music|457|ationantiese|N/A|986honeydew468635810|medium|Each|Unknown|22|eingoughtantipri| +3519|AAAAAAAAOLNAAAAA|2000-10-27||New, equal soldiers understand weeks. That economic services tell already good programmes. Anywhere regional children used to transfer already large, positive passengers; days g|9.65|52.49|5001001|amalgcorp #8|1|birdal|6|Jewelry|457|ationantiese|N/A|907purple52833414826|turquoise|Bunch|Unknown|62|n stoughtantipri| +3520|AAAAAAAAAMNAAAAA|1997-10-27|1999-10-27|Results pass other, excellent bombs; very human ways arise other pensioners. Contracts go most with a lips|5.76|4.89|2002001|importoimporto #1|2|shirts|2|Men|373|priationpri|large|4540378papaya1113116|lavender|Gram|Unknown|48|barableantipri| +3521|AAAAAAAAAMNAAAAA|1999-10-28|2001-10-26|Results pass other, excellent bombs; very human ways arise other pensioners. Contracts go most with a lips|9.24|4.89|2002001|scholaramalgamalg #3|2|portable|10|Electronics|373|priationpri|N/A|powder45647736725958|slate|Tbl|Unknown|34|oughtableantipri| +3522|AAAAAAAAAMNAAAAA|2001-10-27||Friends sell special failures. Members make new sources. Times accept just important proportions. Activities need distinctive line|0.34|0.19|2002001|exportiamalg #1|3|maternity|1|Women|373|priationpri|economy|powder45647736725958|steel|Each|Unknown|9|ableableantipri| +3523|AAAAAAAADMNAAAAA|1997-10-27||Nec|8.74|7.60|10005005|scholarunivamalg #5|5|karoke|10|Electronics|174|eseationought|N/A|356673985368wheat575|lavender|Pallet|Unknown|34|priableantipri| +3524|AAAAAAAAEMNAAAAA|1997-10-27|2000-10-26|More local cities market as; numerous exercises rescue conditions. Cold weeks shall get well religious, english jeans; so economic services worry days. Then new routes carry very clie|4.41|1.85|8010005|univmaxi #5|10|pools|8|Sports|19|n stought|N/A|419296580violet82687|orchid|Tbl|Unknown|49|eseableantipri| +3525|AAAAAAAAEMNAAAAA|2000-10-27||Eyes draw years. Elections cannot oppose more by a cases. Local cattle face then different decades. Safely small relationships might develop quite available years; distin|6.55|1.85|8010005|importonameless #6|2|baseball|8|Sports|19|n stought|N/A|3wheat64213853426416|cyan|Tbl|Unknown|12|antiableantipri| +3526|AAAAAAAAGMNAAAAA|1997-10-27|1999-10-27|Dangerous, short hospitals should help. Rates identify according to a protests. Entirely golden changes make for the fears. Other, electronic shar|6.68|5.21|5001001|amalgscholar #1|1|rock|5|Music|128|eingableought|N/A|01537139789sky451400|ivory|Bunch|Unknown|13|callyableantipri| +3527|AAAAAAAAGMNAAAAA|1999-10-28|2001-10-26|Islands might lend as well in the skills. Equal, high users scream on to a expectations. Lights could not decide children. N|94.46|58.56|1003002|exportiamalg #2|1|maternity|1|Women|582|ableeinganti|N/A|01537139789sky451400|ivory|Dram|Unknown|34|ationableantipri| +3528|AAAAAAAAGMNAAAAA|2001-10-27||Old, particular factors must not name by a ladies. Components can pay perhaps other si|4.32|2.24|9003001|exportimaxi #1|3|computers|9|Books|582|ableeinganti|N/A|01537139789sky451400|tan|Tbl|Unknown|61|eingableantipri| +3529|AAAAAAAAJMNAAAAA|1997-10-27||Apart australian neighbours would not satisfy never. Short programs would not operate properties; only fair claims seem right on the decisions. Networks|2.43|1.77|10006011|corpunivamalg #11|6|musical|10|Electronics|328|eingablepri|N/A|93415333yellow417910|papaya|N/A|Unknown|8|n stableantipri| +3530|AAAAAAAAKMNAAAAA|1997-10-27|2000-10-26|Thousands cannot add only. Intermediate, strong areas come middle days. Charges know only, material techniques. Normal, practical respects p|8.78|4.21|3001001|amalgexporti #1|1|newborn|3|Children|306|callybarpri|petite|598792820432tomato85|purple|Tsp|Unknown|70|barpriantipri| +3531|AAAAAAAAKMNAAAAA|2000-10-27||Natural, important purposes must anticipate |3.40|2.95|4004002|edu packedu pack #2|4|athletic|4|Shoes|221|callybarpri|large|598792820432tomato85|gainsboro|Case|Unknown|43|oughtpriantipri| +3532|AAAAAAAAMMNAAAAA|1997-10-27|1999-10-27|Passengers will not specify proudly regular, working year|1.12|0.54|6012005|importobrand #5|12|costume|6|Jewelry|451|oughtantiese|N/A|8672232712926white44|spring|Dozen|Unknown|21|ablepriantipri| +3533|AAAAAAAAMMNAAAAA|1999-10-28|2001-10-26|Passengers will not specify proudly regular, working year|6.25|4.12|8010004|univmaxi #4|12|pools|8|Sports|315|oughtantiese|N/A|8672232712926white44|khaki|Each|Unknown|66|pripriantipri| +3534|AAAAAAAAMMNAAAAA|2001-10-27||Again real rights ride however small, fine men; as prime words can r|9.27|4.12|8010004|importoamalg #1|2|fragrances|1|Women|315|antioughtpri|large|618304752spring14100|salmon|Box|Unknown|69|esepriantipri| +3535|AAAAAAAAPMNAAAAA|1997-10-27||So other dates must take services. Alternatively industrial issues would go earlier young germans. Materials change at least in a numbers; women will investigate top or|3.37|2.62|1004002|edu packamalg #2|4|swimwear|1|Women|491|oughtn stese|large|01055pink51993937334|turquoise|N/A|Unknown|20|antipriantipri| +3536|AAAAAAAAANNAAAAA|1997-10-27|2000-10-26|Aware, forthcoming students miss markedly healthy weekends; types should become oppo|19.29|14.27|5003001|exportischolar #1|3|pop|5|Music|454|eseantiese|N/A|2turquoise2282970551|thistle|Gross|Unknown|66|callypriantipri| +3537|AAAAAAAAANNAAAAA|2000-10-27||Aware, forthcoming students miss markedly healthy weekends; types should become oppo|1.09|14.27|1004002|edu packamalg #2|3|swimwear|1|Women|454|eseantiese|economy|2turquoise2282970551|violet|Box|Unknown|24|ationpriantipri| +3538|AAAAAAAACNNAAAAA|1997-10-27|1999-10-27|Ports could search at least; similar trends tell usually poor targets. Possible scenes sound away. Policies suggest so|2.10|0.84|2004001|edu packimporto #1|4|sports-apparel|2|Men|190|barn stought|small|573659637spring78110|rosy|Ounce|Unknown|93|eingpriantipri| +3539|AAAAAAAACNNAAAAA|1999-10-28|2001-10-26|Recent journals make simply british, able shows. Classes will pass elsewhere general, certain animals. Unnecessarily various lights could rest less than regu|4.61|3.64|1001002|amalgamalg #2|4|dresses|1|Women|262|ablecallyable|N/A|774050227356698sky26|cyan|Unknown|Unknown|33|n stpriantipri| +3540|AAAAAAAACNNAAAAA|2001-10-27||Readers show later. Excessive, fresh managers used to work perfect, labour games|1.22|0.67|4002001|importoedu pack #1|4|mens|4|Shoes|262|ablecallyable|medium|774050227356698sky26|thistle|Bundle|Unknown|95|bareseantipri| +3541|AAAAAAAAFNNAAAAA|1997-10-27||Inland branches shall provide only available plants. Now available faces answer. Minutes could offer with a others. Forth bizarre dangers search welcome, b|1.86|0.61|8009002|maxinameless #2|9|optics|8|Sports|619|n stoughtcally|N/A|9677saddle9639557711|violet|Lb|Unknown|18|oughteseantipri| +3542|AAAAAAAAGNNAAAAA|1997-10-27|2000-10-26|Unlikely, possible grounds cannot get totally gracefully light companies; parliamentary, romantic levels aim often never so-called priorities. Hot, possible items share operations. A|7.77|6.75|8005003|scholarnameless #3|5|fitness|8|Sports|257|ationantiable|N/A|28533148rose25462902|wheat|Case|Unknown|57|ableeseantipri| +3543|AAAAAAAAGNNAAAAA|2000-10-27||Unlikely, possible grounds cannot get totally gracefully light companies; parliamentary, romantic levels aim often never so-called priorities. Hot, possible items share operations. A|9.03|6.75|2001002|amalgimporto #2|1|accessories|2|Men|662|ablecallycally|extra large|chocolate06454972510|puff|Lb|Unknown|20|prieseantipri| +3544|AAAAAAAAINNAAAAA|1997-10-27|1999-10-27|Experi|77.60|29.48|5003001|exportischolar #1|3|pop|5|Music|177|ationationought|N/A|2352wheat34377400774|ghost|Case|Unknown|67|eseeseantipri| +3545|AAAAAAAAINNAAAAA|1999-10-28|2001-10-26|Also significant lives make errors. More eastern cuts could exchange to a meetings; other, old years used to suggest as new, original weapons. Useful, month|2.22|1.15|5003001|amalgedu pack #2|1|womens|4|Shoes|374|ationationought|extra large|2352wheat34377400774|saddle|Pound|Unknown|11|antieseantipri| +3546|AAAAAAAAINNAAAAA|2001-10-27||Big thing|9.44|1.15|5003001|namelessmaxi #7|1|romance|9|Books|374|ationationought|N/A|5947192cream65561188|seashell|Gross|Unknown|25|callyeseantipri| +3547|AAAAAAAALNNAAAAA|1997-10-27||Skilled views call expensive officers. Widespread officials shall taste; again major circumstances play still following holes. Low skills develop directly ave|7.44|4.68|1004002|edu packamalg #2|4|swimwear|1|Women|656|callyantically|extra large|9185metallic88630138|peru|Gross|Unknown|28|ationeseantipri| +3548|AAAAAAAAMNNAAAAA|1997-10-27|2000-10-26|Rightly responsible documents laugh other candidates. Educational times hide specific, famous elections. Styles cannot go to the sides|0.70|0.49|9006009|corpmaxi #9|6|parenting|9|Books|1|ought|N/A|3orange3686127924707|thistle|Gram|Unknown|58|eingeseantipri| +3549|AAAAAAAAMNNAAAAA|2000-10-27||Rightly responsible documents laugh other candidates. Educational times hide specific, famous elections. Styles cannot go to the sides|3.81|3.31|9006009|importonameless #10|6|paint|7|Home|1|ought|N/A|419grey0503768140194|red|Pound|Unknown|23|n steseantipri| +3550|AAAAAAAAONNAAAAA|1997-10-27|1999-10-27|Secondary members may try so steady years. Previous, confident conclusions change only virtually great beings. Youngsters shall suspect so from |2.13|0.66|2002001|importoimporto #1|2|shirts|2|Men|277|ationationable|medium|83539993rosy57678182|red|Tbl|Unknown|12|barantiantipri| +3551|AAAAAAAAONNAAAAA|1999-10-28|2001-10-26|Secondary members may try so steady years. Previous, confident conclusions change only virtually great beings. Youngsters shall suspect so from |4.47|1.83|9015012|scholarunivamalg #12|15|fiction|9|Books|277|ationationable|N/A|83539993rosy57678182|tomato|Carton|Unknown|45|oughtantiantipri| +3552|AAAAAAAAONNAAAAA|2001-10-27||Secondary members may try so steady years. Previous, confident conclusions change only virtually great beings. Youngsters shall suspect so from |8.78|2.72|9015012|exportiedu pack #1|3|kids|4|Shoes|277|ationationable|medium|83539993rosy57678182|ivory|Unknown|Unknown|8|ableantiantipri| +3553|AAAAAAAABONAAAAA|1997-10-27||Gains cannot cross colourful, long individuals. Drily red difficulties may not say to a plans. Very different cases ta|1.60|1.08|9011002|amalgunivamalg #2|11|cooking|9|Books|372|ableationpri|N/A|8053511snow957816534|sandy|Lb|Unknown|4|priantiantipri| +3554|AAAAAAAACONAAAAA|1997-10-27|2000-10-26|Perfectly proud members may a|6.60|3.76|2004001|edu packimporto #1|4|sports-apparel|2|Men|267|ationcallyable|medium|3675wheat92601262851|red|Unknown|Unknown|77|eseantiantipri| +3555|AAAAAAAACONAAAAA|2000-10-27||Perfectly proud members may a|21.01|7.35|2004001|importoamalg #2|4|fragrances|1|Women|312|ationcallyable|extra large|6350white53744841793|spring|Tbl|Unknown|4|antiantiantipri| +3556|AAAAAAAAEONAAAAA|1997-10-27|1999-10-27|Direct lights save too in a kinds. Absent vehicles |8.40|6.80|10013004|exportiamalgamalg #4|13|stereo|10|Electronics|309|n stbarpri|N/A|141767gainsboro65150|sky|N/A|Unknown|31|callyantiantipri| +3557|AAAAAAAAEONAAAAA|1999-10-28|2001-10-26|Direct lights save too in a kinds. Absent vehicles |1.01|0.68|10013004|amalgnameless #8|13|accent|7|Home|508|n stbarpri|N/A|81442steel2154272321|pink|Tsp|Unknown|14|ationantiantipri| +3558|AAAAAAAAEONAAAAA|2001-10-27||Direct lights save too in a kinds. Absent vehicles |76.05|0.68|7005009|scholarbrand #9|5|blinds/shades|7|Home|179|n stbarpri|N/A|7210509tomato1107052|cornsilk|Tbl|Unknown|18|eingantiantipri| +3559|AAAAAAAAHONAAAAA|1997-10-27||Prominent benefits should not try so birds. Tories may sit recently bright police. Earlier normal subjects know then jewish, nice units. Tiny, superior elements show br|8.56|5.22|3003002|exportiexporti #2|3|toddlers|3|Children|262|ablecallyable|economy|510176863430thistle8|dim|N/A|Unknown|2|n stantiantipri| +3560|AAAAAAAAIONAAAAA|1997-10-27|2000-10-26|Most physical windows could emphasise always only free reports. Current citie|84.44|55.73|10006008|corpunivamalg #8|6|musical|10|Electronics|359|n stantipri|N/A|895702789white345547|forest|Each|Unknown|7|barcallyantipri| +3561|AAAAAAAAIONAAAAA|2000-10-27||Nurses must keep. Formal, independent thousands seek fears; past fatal meetings apply |5.45|3.27|10006008|importonameless #2|2|baseball|8|Sports|359|n stantipri|N/A|5954296265lime483418|olive|Cup|Unknown|44|oughtcallyantipri| +3562|AAAAAAAAKONAAAAA|1997-10-27|1999-10-27|New feet could not cope almost interests. Old, written groups should behave. Detailed, good letters bear hundreds. Hour|0.78|0.29|4004001|edu packedu pack #1|4|athletic|4|Shoes|159|n stantiought|small|583895630lavender972|sandy|Gram|Unknown|5|ablecallyantipri| +3563|AAAAAAAAKONAAAAA|1999-10-28|2001-10-26|New feet could not cope almost interests. Old, written groups should behave. Detailed, good letters bear hundreds. Hour|9.36|7.86|10010011|univamalgamalg #11|10|memory|10|Electronics|159|n stantiought|N/A|83828549481purple680|purple|Unknown|Unknown|2|pricallyantipri| +3564|AAAAAAAAKONAAAAA|2001-10-27||Long, contemporary cars discuss secondly. Basic aspects recognise notes. Current, organic samples use critical days. Areas|0.57|0.26|5004001|edu packscholar #1|10|classical|5|Music|516|callyoughtanti|N/A|411265yellow94007897|navajo|Pallet|Unknown|64|esecallyantipri| +3565|AAAAAAAANONAAAAA|1997-10-27||More sheer feet will make increasingly communities.|4.78|2.62|3004002|edu packexporti #2|4|school-uniforms|3|Children|733|pripriation|medium|3605625spring1173016|red|Ton|Unknown|52|anticallyantipri| +3566|AAAAAAAAOONAAAAA|1997-10-27|2000-10-26|Hands would not provide in a features. Subsequent vehic|3.67|2.71|6011007|amalgbrand #7|11|semi-precious|6|Jewelry|913|prioughtn st|N/A|519navy5595137606045|pink|Bundle|Unknown|42|callycallyantipri| +3567|AAAAAAAAOONAAAAA|2000-10-27||Hands would not provide in a features. Subsequent vehic|2.04|2.71|6011007|exportimaxi #4|11|computers|9|Books|913|prioughtn st|N/A|519navy5595137606045|drab|Cup|Unknown|5|ationcallyantipri| +3568|AAAAAAAAAPNAAAAA|1997-10-27|1999-10-27|Full in|6.73|5.11|5004001|edu packscholar #1|4|classical|5|Music|69|n stcally|N/A|1seashell36328936329|royal|Tbl|Unknown|40|eingcallyantipri| +3569|AAAAAAAAAPNAAAAA|1999-10-28|2001-10-26|Full in|0.60|5.11|5004001|amalgamalg #2|4|dresses|1|Women|69|n stcally|large|663375557maroon06640|pale|Carton|Unknown|57|n stcallyantipri| +3570|AAAAAAAAAPNAAAAA|2001-10-27||Full in|4.55|5.11|6013003|exportibrand #3|13|loose stones|6|Jewelry|62|ablecally|N/A|663375557maroon06640|lawn|Gross|Unknown|66|barationantipri| +3571|AAAAAAAADPNAAAAA|1997-10-27||Central, other hands will agree especially crucial differences|4.49|3.00|5003002|exportischolar #2|3|pop|5|Music|866|callycallyeing|N/A|019hot53019398729370|steel|Each|Unknown|63|oughtationantipri| +3572|AAAAAAAAEPNAAAAA|1997-10-27|2000-10-26|Else essential negotiations understand. Yet short options would carry largely other, special rights. Directly serious letters should change. Telecommunications migh|9.88|3.06|1001001|amalgamalg #1|1|dresses|1|Women|538|eingprianti|medium|22142167red007696281|dim|Dram|Unknown|14|ableationantipri| +3573|AAAAAAAAEPNAAAAA|2000-10-27||New, real potatoes may stand able heroes. |11.81|3.06|7009004|maxibrand #4|9|mattresses|7|Home|299|n stn stable|N/A|6floral1639029269197|misty|Bunch|Unknown|14|priationantipri| +3574|AAAAAAAAGPNAAAAA|1997-10-27|1999-10-27|Just, furious laws help. Details used to write a little. Also atomic terms let too to a priorities. Too m|9.58|2.96|4003001|exportiedu pack #1|3|kids|4|Shoes|502|ablebaranti|petite|111707676933sky69595|olive|Box|Unknown|31|eseationantipri| +3575|AAAAAAAAGPNAAAAA|1999-10-28|2001-10-26|Just, furious laws help. Details used to write a little. Also atomic terms let too to a priorities. Too m|4.76|3.09|4003001|univamalgamalg #6|3|memory|10|Electronics|461|ablebaranti|N/A|111707676933sky69595|blush|Tsp|Unknown|75|antiationantipri| +3576|AAAAAAAAGPNAAAAA|2001-10-27||Almost capable cities will lose as to t|1.07|0.36|4003001|edu packamalg #1|3|swimwear|1|Women|411|oughtoughtese|extra large|111707676933sky69595|tan|Ounce|Unknown|2|callyationantipri| +3577|AAAAAAAAJPNAAAAA|1997-10-27||Sources miss. Natural, communist telecommunications can descr|4.05|3.64|3004002|edu packexporti #2|4|school-uniforms|3|Children|229|n stableable|large|27ivory9966590683832|yellow|Gram|Unknown|38|ationationantipri| +3578|AAAAAAAAKPNAAAAA|1997-10-27|2000-10-26|Mechanical councils make of course about geograp|1.98|0.91|8011009|amalgmaxi #9|11|archery|8|Sports|634|eseprically|N/A|373518018503royal388|seashell|Carton|Unknown|13|eingationantipri| +3579|AAAAAAAAKPNAAAAA|2000-10-27||Extensive children face only subsequently rur|5.40|0.91|6001004|amalgcorp #4|1|birdal|6|Jewelry|155|antiantiought|N/A|28382071179084hot318|thistle|Gross|Unknown|6|n stationantipri| +3580|AAAAAAAAMPNAAAAA|1997-10-27|1999-10-27|Close administrative elections find at least courses; hearts provide nowhere on a legs; other, only points rotate more poor mines. Ori|2.42|1.25|5001001|amalgscholar #1|1|rock|5|Music|292|ablen stable|N/A|317886453203266sky56|powder|Bundle|Unknown|22|bareingantipri| +3581|AAAAAAAAMPNAAAAA|1999-10-28|2001-10-26|Close administrative elections find at least courses; hearts provide nowhere on a legs; other, only points rotate more poor mines. Ori|1.52|0.63|10004012|edu packunivamalg #12|1|audio|10|Electronics|292|ablen stable|N/A|317886453203266sky56|peach|Bunch|Unknown|1|oughteingantipri| +3582|AAAAAAAAMPNAAAAA|2001-10-27||Likely banks transfer tickets; overall, new seats will not cover dark weeks; poor, high hands ignore obviously economic difficul|2.20|0.63|10004012|importonameless #3|12|paint|7|Home|151|ablen stable|N/A|975285575ghost745094|saddle|Cup|Unknown|75|ableeingantipri| +3583|AAAAAAAAPPNAAAAA|1997-10-27||Future students put|2.93|0.90|3003002|exportiexporti #2|3|toddlers|3|Children|59|n stanti|extra large|251708red65333214140|steel|Ounce|Unknown|46|prieingantipri| +3584|AAAAAAAAAAOAAAAA|1997-10-27|2000-10-26|Blue days shall not run therefore american hands. Companies might take particularly years. Related hands avoid premises. Love|3.72|1.71|6013001|exportibrand #1|13|loose stones|6|Jewelry|596|callyn stanti|N/A|080281230smoke526534|sienna|Ounce|Unknown|11|eseeingantipri| +3585|AAAAAAAAAAOAAAAA|2000-10-27||Blue days shall not run therefore american hands. Companies might take particularly years. Related hands avoid premises. Love|5.22|1.71|8010006|univmaxi #6|13|pools|8|Sports|596|callyn stanti|N/A|080281230smoke526534|white|Ton|Unknown|72|antieingantipri| +3586|AAAAAAAACAOAAAAA|1997-10-27|1999-10-27|Equal, lucky unions allow strong surroundings. Together yellow women love once by|5.72|3.94|4003001|exportiedu pack #1|3|kids|4|Shoes|620|barablecally|petite|9607576moccasin69726|peach|Box|Unknown|2|callyeingantipri| +3587|AAAAAAAACAOAAAAA|1999-10-28|2001-10-26|Equal, lucky unions allow strong surroundings. Together yellow women love once by|0.85|3.94|2001002|amalgimporto #2|3|accessories|2|Men|139|n stpriought|medium|19indian599360658492|thistle|Cup|Unknown|13|ationeingantipri| +3588|AAAAAAAACAOAAAAA|2001-10-27||Voices stic|4.40|3.08|2001002|edu packscholar #1|3|classical|5|Music|528|eingableanti|N/A|19indian599360658492|spring|Oz|Unknown|21|eingeingantipri| +3589|AAAAAAAAFAOAAAAA|1997-10-27||International wars may make that is into a issues. Windows should not a|0.60|0.43|2003002|exportiimporto #2|3|pants|2|Men|267|ationcallyable|small|8974776511771sienna2|tan|Gram|Unknown|11|n steingantipri| +3590|AAAAAAAAGAOAAAAA|1997-10-27|2000-10-26|Only impossible words should not talk faintly forms. Economic companies could become really rough practices. Very philosophical heads used to show. Weak requests discover too for a moments. Political,|8.52|4.85|7012001|importonameless #1|12|paint|7|Home|357|ationantipri|N/A|698293255thistle0326|rosy|Gross|Unknown|52|barn stantipri| +3591|AAAAAAAAGAOAAAAA|2000-10-27||Crowds must not come through the parents. Hours display then easily strict meetings. Steep, key activities remember more probl|1.75|4.85|7012001|maxibrand #2|9|mattresses|7|Home|357|ationantipri|N/A|072209701seashell842|rose|Tsp|Unknown|100|oughtn stantipri| +3592|AAAAAAAAIAOAAAAA|1997-10-27|1999-10-27|Professional circumstances could live else others. Symptoms can see very leaves. Just personal institutions used to go. Capable workers used to play then able police.|2.40|1.36|9001005|amalgmaxi #5|1|arts|9|Books|737|ationpriation|N/A|6319255812676peach70|puff|N/A|Unknown|14|ablen stantipri| +3593|AAAAAAAAIAOAAAAA|1999-10-28|2001-10-26|Healthy, annual boys need british classes. Also cognitive groups could not like yet important materials; western women ignore|7.44|1.36|9001005|corpamalgamalg #7|16|wireless|10|Electronics|737|ationpriation|N/A|6319255812676peach70|navy|Gross|Unknown|57|prin stantipri| +3594|AAAAAAAAIAOAAAAA|2001-10-27||Russians would match as huge hours. Trees must not imitate here. Only, rational points remember very other members; quick,|5.20|3.12|9001005|exportiamalg #1|16|maternity|1|Women|314|ationpriation|extra large|3313lemon00689904440|royal|Case|Unknown|70|esen stantipri| +3595|AAAAAAAALAOAAAAA|1997-10-27||Often steep times telephone less british applications. Tory, favorite sales can work there historical benefits; beautiful, local nights cou|2.41|1.08|6014004|edu packbrand #4|14|estate|6|Jewelry|490|barn stese|N/A|5218cornsilk85792693|papaya|Pound|Unknown|49|antin stantipri| +3596|AAAAAAAAMAOAAAAA|1997-10-27|2000-10-26|French systems happe|4.94|1.87|4001001|amalgedu pack #1|1|womens|4|Shoes|566|callycallyanti|extra large|sienna67725649128275|steel|Ounce|Unknown|55|callyn stantipri| +3597|AAAAAAAAMAOAAAAA|2000-10-27||Minor, new police would not smoke both; fields must not grasp equally. Subsequently natural others add usually without a years; large cards ensure well in the times. Hot levels ought|3.62|1.41|4001001|edu packmaxi #8|1|tennis|8|Sports|417|callycallyanti|N/A|057snow1744429159751|dim|Ounce|Unknown|29|ationn stantipri| +3598|AAAAAAAAOAOAAAAA|1997-10-27|1999-10-27|Conservative, difficult fu|3.67|1.98|6014007|edu packbrand #7|14|estate|6|Jewelry|319|n stoughtpri|N/A|239sky67870522594563|sienna|Box|Unknown|5|eingn stantipri| +3599|AAAAAAAAOAOAAAAA|1999-10-28|2001-10-26|Conservative, difficult fu|4.15|1.98|1002002|importoamalg #2|2|fragrances|1|Women|319|n stoughtpri|small|239sky67870522594563|navy|Box|Unknown|23|n stn stantipri| +3600|AAAAAAAAOAOAAAAA|2001-10-27||Conservative, difficult fu|4.63|1.98|9015001|scholarunivamalg #1|2|fiction|9|Books|313|n stoughtpri|N/A|42navy35026608030950|plum|Case|Unknown|50|barbarcallypri| +3601|AAAAAAAABBOAAAAA|1997-10-27||Likely, conventional families could know men; c|3.64|2.54|10015015|scholaramalgamalg #15|15|portable|10|Electronics|316|callyoughtpri|N/A|spring08809180628018|pink|Unknown|Unknown|48|oughtbarcallypri| +3602|AAAAAAAACBOAAAAA|1997-10-27|2000-10-26|Hands used to trust democratic, green attitudes. Negotiations will take products;|0.25|0.12|8006003|corpnameless #3|6|football|8|Sports|331|oughtpripri|N/A|4royal30038696056828|honeydew|Ounce|Unknown|30|ablebarcallypri| +3603|AAAAAAAACBOAAAAA|2000-10-27||Recently thick communities return more precise weaknesses. Even clean degrees make pleasant, holy proposals. Materials s|1.41|0.49|8006003|namelessbrand #4|6|lighting|7|Home|331|oughtpripri|N/A|4royal30038696056828|sienna|Bunch|Unknown|12|pribarcallypri| +3604|AAAAAAAAEBOAAAAA|1997-10-27|1999-10-27|Values may include suddenly in a forms. Still general years shall describe attempts. Professional, informal years may not slow together turkish models. D|3.42|2.87|2002001|importoimporto #1|2|shirts|2|Men|48|eingese|petite|04221614591921pink03|sky|Gross|Unknown|23|esebarcallypri| +3605|AAAAAAAAEBOAAAAA|1999-10-28|2001-10-26|Values may include suddenly in a forms. Still general years shall describe attempts. Professional, informal years may not slow together turkish models. D|1.85|1.27|9016006|corpunivamalg #6|2|mystery|9|Books|264|eingese|N/A|04221614591921pink03|green|Ounce|Unknown|25|antibarcallypri| +3606|AAAAAAAAEBOAAAAA|2001-10-27||Values may include suddenly in a forms. Still general years shall describe attempts. Professional, informal years may not slow together turkish models. D|7.98|1.27|9016006|importoedu pack #1|2|mens|4|Shoes|493|prin stese|extra large|04221614591921pink03|dark|Ton|Unknown|5|callybarcallypri| +3607|AAAAAAAAHBOAAAAA|1997-10-27||Books study later brief communications. Arguments must not conclude home occasionally|8.47|3.30|2004002|edu packimporto #2|4|sports-apparel|2|Men|895|antin steing|petite|86682200violet029209|honeydew|Gram|Unknown|18|ationbarcallypri| +3608|AAAAAAAAIBOAAAAA|1997-10-27|2000-10-26|Universities implement much well financial links. Furthermore tender children maximise still to a her|8.02|6.81|6010001|univbrand #1|10|jewelry boxes|6|Jewelry|35|antipri|N/A|019589wheat855161049|tan|Dozen|Unknown|18|eingbarcallypri| +3609|AAAAAAAAIBOAAAAA|2000-10-27||Universities implement much well financial links. Furthermore tender children maximise still to a her|4.09|6.81|10016006|corpamalgamalg #6|10|wireless|10|Electronics|35|antipri|N/A|641wheat578260033361|peru|Gross|Unknown|20|n stbarcallypri| +3610|AAAAAAAAKBOAAAAA|1997-10-27|1999-10-27|Extraordinary churches increase thereby little orders. Measu|3.41|2.52|7016001|corpnameless #1|16|furniture|7|Home|522|ableableanti|N/A|0061353808salmon3140|turquoise|Oz|Unknown|30|baroughtcallypri| +3611|AAAAAAAAKBOAAAAA|1999-10-28|2001-10-26|Extraordinary churches increase thereby little orders. Measu|6.62|2.52|6011004|amalgbrand #4|11|semi-precious|6|Jewelry|74|eseation|N/A|1chiffon358607243041|wheat|Lb|Unknown|37|oughtoughtcallypri| +3612|AAAAAAAAKBOAAAAA|2001-10-27||However small |5.19|2.52|6011004|amalgscholar #1|11|rock|5|Music|276|eseation|N/A|1chiffon358607243041|medium|Bunch|Unknown|25|ableoughtcallypri| +3613|AAAAAAAANBOAAAAA|1997-10-27||Small, international points might not cry readily fine consequences. Thorough, back flow|28.77|19.27|3002002|importoexporti #2|2|infants|3|Children|26|callyable|large|3302697papaya9497132|red|Bunch|Unknown|6|prioughtcallypri| +3614|AAAAAAAAOBOAAAAA|1997-10-27|2000-10-26|Jeans would concentrate too from the kinds. Plans look gro|0.12|0.06|10010011|univamalgamalg #11|10|memory|10|Electronics|534|eseprianti|N/A|5smoke64187732502082|mint|Tbl|Unknown|9|eseoughtcallypri| +3615|AAAAAAAAOBOAAAAA|2000-10-27||National factories can service occasionally. Just early practitioners keep sometimes linear tables; necessarily principal countries ought to send completely old, good hearts; |1.16|0.06|10010011|edu packimporto #2|4|sports-apparel|2|Men|23|priable|medium|5smoke64187732502082|tan|Pallet|Unknown|17|antioughtcallypri| +3616|AAAAAAAAACOAAAAA|1997-10-27|1999-10-27|Fields begin previously; camps will not assess years; all good figures sta|0.09|0.04|1001001|amalgamalg #1|1|dresses|1|Women|144|eseeseought|N/A|1096607honeydew81696|navy|Bundle|Unknown|25|callyoughtcallypri| +3617|AAAAAAAAACOAAAAA|1999-10-28|2001-10-26|Fields begin previously; camps will not assess years; all good figures sta|7.09|0.04|6010002|univbrand #2|10|jewelry boxes|6|Jewelry|144|eseeseought|N/A|33715sandy9945098150|smoke|Ton|Unknown|39|ationoughtcallypri| +3618|AAAAAAAAACOAAAAA|2001-10-27||Capable arms should identify subjects. Present eyes could not attribute northwards. Final, posit|88.45|30.95|6010002|amalgedu pack #1|10|womens|4|Shoes|134|eseeseought|medium|053261saddle88098136|sienna|Carton|Unknown|33|eingoughtcallypri| +3619|AAAAAAAADCOAAAAA|1997-10-27||Silly acres shall belong alike following, similar pairs. Respectively lucky newspapers shall dare. Also labour requirements can leave; pounds used to stay even only solicitors. Silver systems may de|75.74|61.34|9004008|edu packmaxi #8|4|entertainments|9|Books|379|n stationpri|N/A|8493916183blanched01|rosy|Bundle|Unknown|3|n stoughtcallypri| +3620|AAAAAAAAECOAAAAA|1997-10-27|2000-10-26|Regulations would live parents. Departments shall not want. Standards must not cost difficult authorities. Young, international levels achieve nicely for a participants. Probably busy|43.29|34.19|8016001|corpmaxi #1|16|golf|8|Sports|33|pripri|N/A|6932498641943snow586|tan|Ounce|Unknown|11|barablecallypri| +3621|AAAAAAAAECOAAAAA|2000-10-27||Regulations would live parents. Departments shall not want. Standards must not cost difficult authorities. Young, international levels achieve nicely for a participants. Probably busy|9.48|34.19|8016001|amalgimporto #2|16|accessories|2|Men|87|ationeing|medium|648371599093292peru9|royal|Pallet|Unknown|48|oughtablecallypri| +3622|AAAAAAAAGCOAAAAA|1997-10-27|1999-10-27|Exclusively exclusive heads should admit remotely to the days. Large funds|7.46|2.90|5003001|exportischolar #1|3|pop|5|Music|306|callybarpri|N/A|9017834555royal28628|dim|Bunch|Unknown|8|ableablecallypri| +3623|AAAAAAAAGCOAAAAA|1999-10-28|2001-10-26|For example anxious men pay please rare wives. So able historians want journalists. Technically appropriate shares draw very almost bad diffe|9.62|4.90|7013004|exportinameless #4|3|wallpaper|7|Home|237|ationpriable|N/A|722859035866241rose5|slate|Case|Unknown|20|priablecallypri| +3624|AAAAAAAAGCOAAAAA|2001-10-27||Religious sons must let from time to time simple fears. A bi|0.82|0.63|7013004|corpcorp #1|3|rings|6|Jewelry|232|ationpriable|N/A|70peach5400160160173|lavender|Each|Unknown|22|eseablecallypri| +3625|AAAAAAAAJCOAAAAA|1997-10-27||Ways must indicate below running payments. Exotic views may|8.53|6.22|2001002|amalgimporto #2|1|accessories|2|Men|198|eingn stought|medium|5papaya1317893639503|midnight|Lb|Unknown|33|antiablecallypri| +3626|AAAAAAAAKCOAAAAA|1997-10-27|2000-10-26|Immediate, mixed hospitals become; bad, clear rates cut still for a units; independently existing weeks in|39.82|35.83|8008007|namelessnameless #7|8|outdoor|8|Sports|107|ationbarought|N/A|510powder83305399804|violet|Tsp|Unknown|35|callyablecallypri| +3627|AAAAAAAAKCOAAAAA|2000-10-27||Forces exercise at the facts. Around young circumstances go up to in a systems. Once more distinctive parts shall r|4.73|35.83|8008007|amalgscholar #2|8|rock|5|Music|61|ationbarought|N/A|2763563417sienna5035|plum|Pound|Unknown|57|ationablecallypri| +3628|AAAAAAAAMCOAAAAA|1997-10-27|1999-10-27|Services must live current problems. Reasonable, japanese transactions will increase only so live bodies. As equivalent accounts contact little, possible acc|9.11|6.28|6015005|scholarbrand #5|15|custom|6|Jewelry|495|antin stese|N/A|1957773730smoke14892|peach|Cup|Unknown|39|eingablecallypri| +3629|AAAAAAAAMCOAAAAA|1999-10-28|2001-10-26|Services must live current problems. Reasonable, japanese transactions will increase only so live bodies. As equivalent accounts contact little, possible acc|3.74|2.35|6015005|amalgamalg #2|15|dresses|1|Women|259|antin stese|small|1957773730smoke14892|plum|Cup|Unknown|59|n stablecallypri| +3630|AAAAAAAAMCOAAAAA|2001-10-27||Services must live current problems. Reasonable, japanese transactions will increase only so live bodies. As equivalent accounts contact little, possible acc|2.95|2.35|6015005|corpamalgamalg #10|16|wireless|10|Electronics|259|n stantiable|N/A|125907454burlywood94|maroon|Oz|Unknown|66|barpricallypri| +3631|AAAAAAAAPCOAAAAA|1997-10-27||Employees pay ahead comme|93.19|63.36|8009002|maxinameless #2|9|optics|8|Sports|305|antibarpri|N/A|sienna00387827251337|saddle|Pound|Unknown|12|oughtpricallypri| +3632|AAAAAAAAADOAAAAA|1997-10-27|2000-10-26|Only detailed memories can tackle free, good members. For example artistic women bec|4.37|2.70|7010003|univnameless #3|10|flatware|7|Home|288|eingeingable|N/A|6353386honeydew52752|sienna|Dram|Unknown|19|ablepricallypri| +3633|AAAAAAAAADOAAAAA|2000-10-27||Basic, sorry losses shall not attack personally busy holidays; brief|0.82|2.70|7010003|amalgedu pack #2|10|womens|4|Shoes|544|eingeingable|extra large|662151653peach629380|navajo|Gram|Unknown|44|pripricallypri| +3634|AAAAAAAACDOAAAAA|1997-10-27|1999-10-27|So sorry operations could make probably mutual red relations. Clear, early supporters gain past newspapers. Other source|3.58|2.61|6014003|edu packbrand #3|14|estate|6|Jewelry|568|eingcallyanti|N/A|674859pale7619534563|yellow|Box|Unknown|48|esepricallypri| +3635|AAAAAAAACDOAAAAA|1999-10-28|2001-10-26|Unions must not prevent again smart edges; social, hard men ought to apply merely free phenomena. Adequate requirements cannot describe very to the agencies. Only indian candidates may take in the c|0.41|2.61|6014003|importocorp #4|2|diamonds|6|Jewelry|568|eingcallyanti|N/A|674859pale7619534563|blush|Lb|Unknown|18|antipricallypri| +3636|AAAAAAAACDOAAAAA|2001-10-27||Unions must not prevent again smart edges; social, hard men ought to apply merely free phenomena. Adequate requirements cannot describe very to the agencies. Only indian candidates may take in the c|5.12|2.61|6014003|importoedu pack #1|2|mens|4|Shoes|568|eingcallyanti|medium|674859pale7619534563|pale|Pound|Unknown|75|callypricallypri| +3637|AAAAAAAAFDOAAAAA|1997-10-27||Resources must simulate in the functi|8.45|7.52|10009017|maxiunivamalg #17|9|televisions|10|Electronics|766|callycallyation|N/A|5deep088514243996786|papaya|Case|Unknown|51|ationpricallypri| +3638|AAAAAAAAGDOAAAAA|1997-10-27|2000-10-26|Organic girls could give messages. Germans might not take still up to a clients. Services ought to assume random demands. Due, irish modules used |2.04|0.63|5002001|importoscholar #1|2|country|5|Music|164|esecallyought|N/A|98295794017965slate0|light|Carton|Unknown|7|eingpricallypri| +3639|AAAAAAAAGDOAAAAA|2000-10-27||Organic girls could give messages. Germans might not take still up to a clients. Services ought to assume random demands. Due, irish modules used |1.15|0.63|9012004|importounivamalg #4|2|home repair|9|Books|22|ableable|N/A|98295794017965slate0|papaya|Carton|Unknown|58|n stpricallypri| +3640|AAAAAAAAIDOAAAAA|1997-10-27|1999-10-27|Holes know well at first happy heads. Sounds would say roots. More high individuals refuse distinct penalties. Others might account too ordinary children.|2.93|1.61|4002001|importoedu pack #1|2|mens|4|Shoes|148|eingeseought|extra large|6546572firebrick2192|navajo|Lb|Unknown|24|baresecallypri| +3641|AAAAAAAAIDOAAAAA|1999-10-28|2001-10-26|Holes know well at first happy heads. Sounds would say roots. More high individuals refuse distinct penalties. Others might account too ordinary children.|8.32|5.32|4002001|exportiamalg #2|3|maternity|1|Women|315|eingeseought|extra large|spring99574029481120|sky|Ton|Unknown|4|oughtesecallypri| +3642|AAAAAAAAIDOAAAAA|2001-10-27||Holes know well at first happy heads. Sounds would say roots. More high individuals refuse distinct penalties. Others might account too ordinary children.|2.37|5.32|4002001|namelessunivamalg #5|3|scanners|10|Electronics|229|eingeseought|N/A|spring99574029481120|steel|N/A|Unknown|90|ableesecallypri| +3643|AAAAAAAALDOAAAAA|1997-10-27||Correct, clinical years may know during a assumptions. Long prayers will not slip utterly again political contributions; daily players lift clearly. Extensive unions would avoid even u|1.65|0.72|3002002|importoexporti #2|2|infants|3|Children|126|callyableought|medium|3816535powder8163350|lace|Pallet|Unknown|65|priesecallypri| +3644|AAAAAAAAMDOAAAAA|1997-10-27|2000-10-26|Products meet types. Themes |0.16|0.07|5004001|edu packscholar #1|4|classical|5|Music|99|n stn st|N/A|7523147209cyan692993|turquoise|Box|Unknown|45|eseesecallypri| +3645|AAAAAAAAMDOAAAAA|2000-10-27||Products meet types. Themes |0.73|0.07|7010006|univnameless #6|10|flatware|7|Home|563|pricallyanti|N/A|7523147209cyan692993|khaki|Unknown|Unknown|60|antiesecallypri| +3646|AAAAAAAAODOAAAAA|1997-10-27|1999-10-27|Great, dreadful uses create prices. Different, personal rocks used to live; broken, conventional claim|1.91|0.89|4001001|amalgedu pack #1|1|womens|4|Shoes|290|barn stable|medium|790pale6645322452732|green|Box|Unknown|11|callyesecallypri| +3647|AAAAAAAAODOAAAAA|1999-10-28|2001-10-26|Great, dreadful uses create prices. Different, personal rocks used to live; broken, conventional claim|4.46|3.21|7016008|corpnameless #8|16|furniture|7|Home|290|barn stable|N/A|144438rose8612659407|yellow|Each|Unknown|65|ationesecallypri| +3648|AAAAAAAAODOAAAAA|2001-10-27||Real, passive costs complete more to a efforts. Much bare notes act predictably polls. Conclusions shall reconsider on the minutes. Likely notes might not happ|1.36|1.10|7016008|maxibrand #9|16|mattresses|7|Home|232|ablepriable|N/A|64113650royal2197417|honeydew|Oz|Unknown|60|eingesecallypri| +3649|AAAAAAAABEOAAAAA|1997-10-27||Vast, social days explain just. Young, necessary galleries might continue particular reports. Eyes wander. Seasons used to race at a eyes. Either other conclusions should see still. Great, indiv|2.90|0.98|5004002|edu packscholar #2|4|classical|5|Music|249|n steseable|N/A|4901211404honeydew74|yellow|Cup|Unknown|49|n stesecallypri| +3650|AAAAAAAACEOAAAAA|1997-10-27|2000-10-26|Minutes go very out of a pockets. Days ought to wish videos. For|6.14|3.56|10015013|scholaramalgamalg #13|15|portable|10|Electronics|33|pripri|N/A|97596653120mint10367|goldenrod|Ounce|Unknown|30|baranticallypri| +3651|AAAAAAAACEOAAAAA|2000-10-27||Ago careful peasants cou|5.15|3.56|10015013|scholarbrand #4|15|custom|6|Jewelry|33|pripri|N/A|8metallic43614105965|white|Cup|Unknown|18|oughtanticallypri| +3652|AAAAAAAAEEOAAAAA|1997-10-27|1999-10-27|Main,|8.94|7.86|6012005|importobrand #5|12|costume|6|Jewelry|248|eingeseable|N/A|2739912435salmon8540|thistle|Ounce|Unknown|46|ableanticallypri| +3653|AAAAAAAAEEOAAAAA|1999-10-28|2001-10-26|Main,|74.52|7.86|10005016|scholarunivamalg #16|5|karoke|10|Electronics|176|callyationought|N/A|28522saddle084933703|white|Tsp|Unknown|3|prianticallypri| +3654|AAAAAAAAEEOAAAAA|2001-10-27||Services shall comprehend in|2.97|2.34|10005016|amalgunivamalg #17|1|cameras|10|Electronics|243|callyationought|N/A|28522saddle084933703|tan|Carton|Unknown|24|eseanticallypri| +3655|AAAAAAAAHEOAAAAA|1997-10-27||Most native photograph|3.89|3.26|4004002|edu packedu pack #2|4|athletic|4|Shoes|54|eseanti|small|7356491867575lace698|powder|Dozen|Unknown|4|antianticallypri| +3656|AAAAAAAAIEOAAAAA|1997-10-27|2000-10-26|Dark, judicial votes hunt trains. For|3.26|1.27|6016001|corpbrand #1|16|consignment|6|Jewelry|466|callycallyese|N/A|356slate464767360736|saddle|Lb|Unknown|86|callyanticallypri| +3657|AAAAAAAAIEOAAAAA|2000-10-27||Dark, judicial votes hunt trains. For|4.72|1.27|6016001|importoamalg #2|2|fragrances|1|Women|466|callycallyese|N/A|25spring663761330893|rosy|Gram|Unknown|36|ationanticallypri| +3658|AAAAAAAAKEOAAAAA|1997-10-27|1999-10-27|Immensely|2.69|1.37|2001001|amalgimporto #1|1|accessories|2|Men|584|eseeinganti|petite|663slate212867906336|orchid|N/A|Unknown|20|einganticallypri| +3659|AAAAAAAAKEOAAAAA|1999-10-28|2001-10-26|Animals prove different records. Willingly strange forests create altogether for the attempts. Old, special resources must end|7.10|5.68|10014005|edu packamalgamalg #5|1|automotive|10|Electronics|584|eseeinganti|N/A|663slate212867906336|pink|Box|Unknown|36|n stanticallypri| +3660|AAAAAAAAKEOAAAAA|2001-10-27||Animals prove different records. Willingly strange forests create altogether for the attempts. Old, special resources must end|7.90|4.97|8008001|namelessnameless #1|1|outdoor|8|Sports|584|eseeinganti|N/A|663slate212867906336|medium|Lb|Unknown|50|barcallycallypri| +3661|AAAAAAAANEOAAAAA|1997-10-27||Working dangers must follow british, wealthy governments. Possible magistrates ought to mean old, major facilities. Contents int|3.42|1.09|9015002|scholarunivamalg #2|15|fiction|9|Books|195|antin stought|N/A|4847234rosy636110631|rosy|Dozen|Unknown|27|oughtcallycallypri| +3662|AAAAAAAAOEOAAAAA|1997-10-27|2000-10-26|Hard, close others should reflect. More high hands should like so political constraints. Patients need with a names. Even sweet men encounter sure tender, evil problems. Else |1.57|1.22|2002001|importoimporto #1|2|shirts|2|Men|229|n stableable|medium|095lavender363616216|steel|Ton|Unknown|8|ablecallycallypri| +3663|AAAAAAAAOEOAAAAA|2000-10-27||Great visitors understand more as to the galleries. Never twin members create none the less within a proposals. Women must not use very investors; so great numbers expect almost; more silent principle|1.66|0.59|2002001|edu packscholar #2|2|classical|5|Music|259|n stantiable|N/A|69538959lime49719069|purple|Each|Unknown|23|pricallycallypri| +3664|AAAAAAAAAFOAAAAA|1997-10-27|1999-10-27|National, upper principles fill relatives. Components make. Rarely unusual plants might not deal|1.57|0.64|1001001|amalgamalg #1|1|dresses|1|Women|829|n stableeing|medium|0881metallic45178535|purple|Ton|Unknown|28|esecallycallypri| +3665|AAAAAAAAAFOAAAAA|1999-10-28|2001-10-26|National, upper principles fill relatives. Components make. Rarely unusual plants might not deal|7.27|0.64|1001001|amalgmaxi #6|1|arts|9|Books|256|n stableeing|N/A|0881metallic45178535|orchid|Dram|Unknown|51|anticallycallypri| +3666|AAAAAAAAAFOAAAAA|2001-10-27||National, upper principles fill relatives. Components make. Rarely unusual plants might not deal|1.42|0.64|1001001|exportimaxi #7|1|sailing|8|Sports|256|callyantiable|N/A|0881metallic45178535|ghost|Oz|Unknown|49|callycallycallypri| +3667|AAAAAAAADFOAAAAA|1997-10-27||Tiny tensions used to find secondary, careful inches; important forms ask gently. Similar words think additional, positive discussio|0.45|0.19|4001002|amalgedu pack #2|1|womens|4|Shoes|74|eseation|small|5peach71025104044718|rosy|Each|Unknown|15|ationcallycallypri| +3668|AAAAAAAAEFOAAAAA|1997-10-27|2000-10-26|Parts come there from a horses. Else brown hours ought to move probably out of a eyes. Considerable, potential trees visit both internal russians. Other, alone circumstances find extensive words.|3.10|2.32|5004001|edu packscholar #1|4|classical|5|Music|145|antieseought|N/A|0633112988224sky2904|midnight|Oz|Unknown|60|eingcallycallypri| +3669|AAAAAAAAEFOAAAAA|2000-10-27||Adequate advantages can win as a whole women. Also small partners tell only g|7.46|2.32|5004001|exportischolar #2|4|pop|5|Music|223|antieseought|N/A|0633112988224sky2904|sky|Cup|Unknown|30|n stcallycallypri| +3670|AAAAAAAAGFOAAAAA|1997-10-27|1999-10-27|British arms could injure cheap various leaders. Marked interests qualify really red strengths. Badly valid machines would tell anyway now big arrangements. Constitutional, odd assets can help ang|0.38|0.30|6007007|brandcorp #7|7|pendants|6|Jewelry|147|ationeseought|N/A|413874571wheat159720|plum|Gross|Unknown|2|barationcallypri| +3671|AAAAAAAAGFOAAAAA|1999-10-28|2001-10-26|Brief, old officials remove gradually; social, ashamed |8.39|2.93|5001002|amalgscholar #2|1|rock|5|Music|18|eingought|N/A|seashell347459400899|powder|Cup|Unknown|64|oughtationcallypri| +3672|AAAAAAAAGFOAAAAA|2001-10-27||Current ambitions shall reduce clear widely subsequent ha|4.44|2.08|9013001|exportiunivamalg #1|1|self-help|9|Books|18|eingought|N/A|914651863slate756675|frosted|Gram|Unknown|30|ableationcallypri| +3673|AAAAAAAAJFOAAAAA|1997-10-27||Unable, great investigations must no perhaps adverse emotions. For example perfect prices feel then other, initial reactions. Cases predict champions; british, liable questions react reall|4.61|3.27|4002002|importoedu pack #2|2|mens|4|Shoes|247|ationeseable|small|6981linen40277864177|hot|Bundle|Unknown|40|priationcallypri| +3674|AAAAAAAAKFOAAAAA|1997-10-27|2000-10-26|Offices should enforce to a roles. Confident teachers help right ways; selective purposes reveal toys; |3.72|3.08|3004001|edu packexporti #1|4|school-uniforms|3|Children|197|ationn stought|N/A|26powder996278029480|yellow|Each|Unknown|37|eseationcallypri| +3675|AAAAAAAAKFOAAAAA|2000-10-27||Issues force. Working, great years wish economic, deliberate delegates. Certain, boring pou|9.72|6.02|3004001|edu packexporti #2|4|school-uniforms|3|Children|819|n stoughteing|small|9770843774829sky2128|beige|Unknown|Unknown|35|antiationcallypri| +3676|AAAAAAAAMFOAAAAA|1997-10-27|1999-10-27|Weeks spend on a pp.. Important clients contemplate rarely more other assumptions; gold, other procedures would not accept officially from a ci|4.74|1.42|5001001|amalgscholar #1|1|rock|5|Music|149|n steseought|N/A|4890sandy49043637920|smoke|Oz|Unknown|32|callyationcallypri| +3677|AAAAAAAAMFOAAAAA|1999-10-28|2001-10-26|Weeks spend on a pp.. Important clients contemplate rarely more other assumptions; gold, other procedures would not accept officially from a ci|57.84|19.66|9010006|univunivamalg #6|10|travel|9|Books|149|n steseought|N/A|4890sandy49043637920|seashell|Dram|Unknown|25|ationationcallypri| +3678|AAAAAAAAMFOAAAAA|2001-10-27||Atomic, clean workers accompany again new skills; things will not assume better. Members should choose ahead comparisons. D|8.73|7.50|3002001|importoexporti #1|10|infants|3|Children|505|n steseought|large|242wheat005154327764|puff|Gram|Unknown|15|eingationcallypri| +3679|AAAAAAAAPFOAAAAA|1997-10-27||Less western communities make nearer customs; now potential speakers would get separate, unchanged homes. Conditions help elderly, high votes. Souther|8.65|2.76|8002010|importonameless #10|2|baseball|8|Sports|630|barprically|N/A|780866519416dim50983|papaya|Pound|Unknown|28|n stationcallypri| +3680|AAAAAAAAAGOAAAAA|1997-10-27|2000-10-26|Then positive unions used |8.27|6.78|8002001|importonameless #1|2|baseball|8|Sports|268|eingcallyable|N/A|antique4112147944820|violet|Box|Unknown|46|bareingcallypri| +3681|AAAAAAAAAGOAAAAA|2000-10-27||Twice blue girls meet largely difficult, apparent groups. Used affairs would suggest professional details. Badly major weeks start more black services; books establish; old questions might carry ita|3.78|2.26|8002001|exportiunivamalg #10|13|self-help|9|Books|268|eingcallyable|N/A|antique4112147944820|snow|Ounce|Unknown|29|oughteingcallypri| +3682|AAAAAAAACGOAAAAA|1997-10-27|1999-10-27|The|0.99|0.33|3002001|importoexporti #1|2|infants|3|Children|130|barpriought|small|9034094984drab273509|gainsboro|Dram|Unknown|90|ableeingcallypri| +3683|AAAAAAAACGOAAAAA|1999-10-28|2001-10-26|The|6.41|4.99|10014012|edu packamalgamalg #12|2|automotive|10|Electronics|191|oughtn stought|N/A|9034094984drab273509|purple|Box|Unknown|55|prieingcallypri| +3684|AAAAAAAACGOAAAAA|2001-10-27||The|19.32|7.92|10014012|amalgmaxi #1|1|arts|9|Books|191|oughtn stought|N/A|18674991539steel5245|tomato|Tsp|Unknown|19|eseeingcallypri| +3685|AAAAAAAAFGOAAAAA|1997-10-27||Today possible men must not initiate previous bodies. Big hours shall gain there japanese pairs. Direct hands can find recent, national books. Good new weaknesses change firm|4.01|3.00|4004002|edu packedu pack #2|4|athletic|4|Shoes|491|oughtn stese|extra large|99392peru29734377164|steel|Tbl|Unknown|29|antieingcallypri| +3686|AAAAAAAAGGOAAAAA|1997-10-27|2000-10-26|Following, accurate matters shall see persistently increased procedures. Others could mind inside special, alone combinations. Concerned units think ofte|4.59|3.85|4003001|exportiedu pack #1|3|kids|4|Shoes|116|callyoughtought|medium|897621lemon171180040|yellow|N/A|Unknown|53|callyeingcallypri| +3687|AAAAAAAAGGOAAAAA|2000-10-27||Following, accurate matters shall see persistently increased procedures. Others could mind inside special, alone combinations. Concerned units think ofte|3.40|1.70|5004002|edu packscholar #2|4|classical|5|Music|173|priationought|N/A|81salmon112635868561|slate|Lb|Unknown|53|ationeingcallypri| +3688|AAAAAAAAIGOAAAAA|1997-10-27|1999-10-27|Technical centuries shall exercise less secret ra|1.73|1.28|3004001|edu packexporti #1|4|school-uniforms|3|Children|96|callyn st|extra large|wheat805101171315581|pink|Gram|Unknown|40|eingeingcallypri| +3689|AAAAAAAAIGOAAAAA|1999-10-28|2001-10-26|Technical centuries shall exercise less secret ra|6.05|5.38|5001002|amalgscholar #2|4|rock|5|Music|96|callyn st|N/A|107plum5073135742297|smoke|Ton|Unknown|5|n steingcallypri| +3690|AAAAAAAAIGOAAAAA|2001-10-27||Technical centuries shall exercise less secret ra|4.03|3.22|1004001|edu packamalg #1|4|swimwear|1|Women|96|callyn st|petite|goldenrod16787881747|salmon|Bunch|Unknown|10|barn stcallypri| +3691|AAAAAAAALGOAAAAA|1997-10-27||Obligations should provide more annual, sole stars. Obviously available unions receive there. Other wages must ruin much progressively new shares. Christian, c|3.76|3.04|8015002|scholarmaxi #2|15|fishing|8|Sports|18|eingought|N/A|20919metallic6776686|royal|Gram|Unknown|21|oughtn stcallypri| +3692|AAAAAAAAMGOAAAAA|1997-10-27|2000-10-26|Real arrangements mus|8.29|4.80|1002001|importoamalg #1|2|fragrances|1|Women|256|callyantiable|medium|01850881425961sandy7|green|Gross|Unknown|7|ablen stcallypri| +3693|AAAAAAAAMGOAAAAA|2000-10-27||Real arrangements mus|31.46|4.80|1002001|importoimporto #2|2|shirts|2|Men|293|callyantiable|large|1381342920843snow915|puff|Bundle|Unknown|4|prin stcallypri| +3694|AAAAAAAAOGOAAAAA|1997-10-27|1999-10-27|So dependent buildings provide; medical, expensive tools say years. Minor scales listen tomorrow in a teachers. Other, other childre|3.72|1.67|7015005|scholarnameless #5|15|tables|7|Home|236|callypriable|N/A|22004622468spring207|sky|Bunch|Unknown|3|esen stcallypri| +3695|AAAAAAAAOGOAAAAA|1999-10-28|2001-10-26|So dependent buildings provide; medical, expensive tools say years. Minor scales listen tomorrow in a teachers. Other, other childre|6.69|1.67|10005007|scholarunivamalg #7|5|karoke|10|Electronics|173|priationought|N/A|102427snow4652159462|sky|Case|Unknown|56|antin stcallypri| +3696|AAAAAAAAOGOAAAAA|2001-10-27||Anyway|1.20|0.84|8008007|namelessnameless #7|5|outdoor|8|Sports|378|eingationpri|N/A|497350202navajo82872|grey|Box|Unknown|66|callyn stcallypri| +3697|AAAAAAAABHOAAAAA|1997-10-27||So certain things cannot fall all past local doubts. Ever able leads may return as important, lengthy legs. Evil politicians will not make so. Potential fa|7.67|2.45|4001002|amalgedu pack #2|1|womens|4|Shoes|331|oughtpripri|medium|71516434black7805655|medium|Dozen|Unknown|90|ationn stcallypri| +3698|AAAAAAAACHOAAAAA|1997-10-27|2000-10-26|Again known |0.26|0.17|8001009|amalgnameless #9|1|athletic shoes|8|Sports|689|n steingcally|N/A|523619671chiffon0553|lavender|Pound|Unknown|22|eingn stcallypri| +3699|AAAAAAAACHOAAAAA|2000-10-27||Again known |6.83|2.18|2001002|amalgimporto #2|1|accessories|2|Men|689|n steingcally|extra large|24519079puff72900114|midnight|Ounce|Unknown|30|n stn stcallypri| +3700|AAAAAAAAEHOAAAAA|1997-10-27|1999-10-27|Clear laws believe just skills. Special games could tear never conservative alternatives. Usually aware ways must give obviously more famous metres. Hot, cu|1.80|0.81|3001001|amalgexporti #1|1|newborn|3|Children|337|ationpripri|extra large|68721605purple522960|slate|Gross|Unknown|9|barbarationpri| +3701|AAAAAAAAEHOAAAAA|1999-10-28|2001-10-26|Royal, british farmers deceive over results. Al|0.22|0.81|3001001|amalgedu pack #2|1|womens|4|Shoes|215|antioughtable|medium|68721605purple522960|rose|Gram|Unknown|21|oughtbarationpri| +3702|AAAAAAAAEHOAAAAA|2001-10-27||Royal, british farmers deceive over results. Al|4.29|2.23|8009003|maxinameless #3|1|optics|8|Sports|215|antioughtable|N/A|8799peru274569877134|peach|Ton|Unknown|26|ablebarationpri| +3703|AAAAAAAAHHOAAAAA|1997-10-27||Just red students should respond so econ|0.86|0.43|5003002|exportischolar #2|3|pop|5|Music|305|antibarpri|N/A|732798815papaya64314|papaya|Oz|Unknown|1|pribarationpri| +3704|AAAAAAAAIHOAAAAA|1997-10-27|2000-10-26|Quickly clear attitudes vote purely atomic senses; poor, concerned patterns achieve almost bright, european skills. Foreign, socialist individuals should not permit very just |8.94|4.47|7011005|amalgnameless #5|11|accent|7|Home|468|eingcallyese|N/A|navajo65032888420925|steel|Unknown|Unknown|44|esebarationpri| +3705|AAAAAAAAIHOAAAAA|2000-10-27||Still new women|6.42|1.92|7011005|importoexporti #2|11|infants|3|Children|18|eingcallyese|large|navajo65032888420925|peru|Ton|Unknown|28|antibarationpri| +3706|AAAAAAAAKHOAAAAA|1997-10-27|1999-10-27|Now lengthy problems face profits; outer, outstanding buildings shall dictate sharply classes. Forms should accept enough hot, possible women. Able, le|6.28|3.76|1002001|importoamalg #1|2|fragrances|1|Women|796|callyn station|medium|1slate41933721663761|turquoise|Bundle|Unknown|22|callybarationpri| +3707|AAAAAAAAKHOAAAAA|1999-10-28|2001-10-26|Now lengthy problems face profits; outer, outstanding buildings shall dictate sharply classes. Forms should accept enough hot, possible women. Able, le|2.09|3.76|1002001|exportiimporto #2|2|pants|2|Men|796|callyn station|medium|8318362672pink535054|slate|Lb|Unknown|12|ationbarationpri| +3708|AAAAAAAAKHOAAAAA|2001-10-27||Now lengthy problems face profits; outer, outstanding buildings shall dictate sharply classes. Forms should accept enough hot, possible women. Able, le|3.24|1.91|1002001|importoexporti #1|2|infants|3|Children|796|callyn station|petite|73230700sandy9700910|white|Pallet|Unknown|19|eingbarationpri| +3709|AAAAAAAANHOAAAAA|1997-10-27||Very little points |1.81|0.83|1001002|amalgamalg #2|1|dresses|1|Women|141|oughteseought|medium|43121peach5209277378|rose|Unknown|Unknown|6|n stbarationpri| +3710|AAAAAAAAOHOAAAAA|1997-10-27|2000-10-26|Shareholders hope suddenly new honours. Good children lie obviously strategic, final services. Others wake easily interested, right |4.32|1.98|5001001|amalgscholar #1|1|rock|5|Music|332|ablepripri|N/A|1439427wheat00327921|gainsboro|Gram|Unknown|31|baroughtationpri| +3711|AAAAAAAAOHOAAAAA|2000-10-27||Researchers can make today most white reports. As nice thanks indicate here with a flights; necessary, sufficient w|2.17|1.80|5001001|exportibrand #8|13|loose stones|6|Jewelry|332|ablepripri|N/A|116036violet01688171|puff|Gross|Unknown|22|oughtoughtationpri| +3712|AAAAAAAAAIOAAAAA|1997-10-27|1999-10-27|Procee|3.76|1.20|6009001|maxicorp #1|9|womens watch|6|Jewelry|784|eseeingation|N/A|406724cornsilk985468|puff|Tsp|Unknown|26|ableoughtationpri| +3713|AAAAAAAAAIOAAAAA|1999-10-28|2001-10-26|Shares would appoint into a interests. Current systems tell briefly spanish, ready models. Then available remains believe quickly established, organic feet. Levels might |7.76|3.41|10009008|maxiunivamalg #8|9|televisions|10|Electronics|784|eseeingation|N/A|2sandy13234705935376|salmon|Bunch|Unknown|39|prioughtationpri| +3714|AAAAAAAAAIOAAAAA|2001-10-27||Then ready movements keep here. Accidents cannot turn obviously strong trustees. Good, proper cities may tell minutes. Visitors might not preserve ok. Particularly definite assessments might |9.90|3.41|7015005|scholarnameless #5|9|tables|7|Home|784|eseeingation|N/A|7496tomato7883731360|puff|Cup|Unknown|33|eseoughtationpri| +3715|AAAAAAAADIOAAAAA|1997-10-27||More medical boundaries used to leave apart with a others. Designs like big, various countries. Small states keep always negotiations. Then easy techniques s|5.75|4.71|3002002|importoexporti #2|2|infants|3|Children|315|antioughtpri|petite|steel651927977817499|smoke|Cup|Unknown|41|antioughtationpri| +3716|AAAAAAAAEIOAAAAA|1997-10-27|2000-10-26|Old views explain much physical, grey groups. Employees would refer immediately subsequently real neighbours. Private, sacred banks cannot prevent restrictions. Good buildings rectify best minute|3.22|1.80|1004001|edu packamalg #1|4|swimwear|1|Women|894|esen steing|small|63075spring826943992|ivory|Box|Unknown|47|callyoughtationpri| +3717|AAAAAAAAEIOAAAAA|2000-10-27||Old views explain much physical, grey groups. Employees would refer immediately subsequently real neighbours. Private, sacred banks cannot prevent restrictions. Good buildings rectify best minute|8.31|5.73|1004001|exportiamalg #2|3|maternity|1|Women|117|ationoughtought|petite|spring56559042085629|tan|N/A|Unknown|64|ationoughtationpri| +3718|AAAAAAAAGIOAAAAA|1997-10-27|1999-10-27|Fellow, entire organisers may not wear much old, initial criticisms. Girls rely other, main|4.67|2.33|2002001|importoimporto #1|2|shirts|2|Men|212|ableoughtable|medium|3090612pale657062602|plum|Case|Unknown|7|eingoughtationpri| +3719|AAAAAAAAGIOAAAAA|1999-10-28|2001-10-26|Fellow, entire organisers may not wear much old, initial criticisms. Girls rely other, main|6.04|1.87|7007010|brandbrand #10|7|decor|7|Home|212|ableoughtable|N/A|3090612pale657062602|thistle|Ounce|Unknown|4|n stoughtationpri| +3720|AAAAAAAAGIOAAAAA|2001-10-27||Fellow, entire organisers may not wear much old, initial criticisms. Girls rely other, main|72.17|1.87|7007010|exportiamalg #1|7|maternity|1|Women|328|ableoughtable|medium|3090612pale657062602|spring|Pound|Unknown|69|barableationpri| +3721|AAAAAAAAJIOAAAAA|1997-10-27||Effective doctors make curiously small animals. Projects would not establish scottish problems. Reasons can read prope|3.23|2.16|2001002|amalgimporto #2|1|accessories|2|Men|246|callyeseable|large|4543710563tomato4721|smoke|Each|Unknown|34|oughtableationpri| +3722|AAAAAAAAKIOAAAAA|1997-10-27|2000-10-26|Western, successful levels |5.31|3.29|7004003|edu packbrand #3|4|curtains/drapes|7|Home|656|callyantically|N/A|16402820powder424038|yellow|Pallet|Unknown|12|ableableationpri| +3723|AAAAAAAAKIOAAAAA|2000-10-27||Previous experts will ask there debts. Methods need quite at the teachers. Troops attribute mildly. Active years train newspapers. Due cups would fight as very superb remains. |79.23|31.69|7004003|maxicorp #4|9|womens watch|6|Jewelry|656|callyantically|N/A|seashell924616767626|rose|N/A|Unknown|26|priableationpri| +3724|AAAAAAAAMIOAAAAA|1997-10-27|1999-10-27|Best dep|4.50|3.69|10009002|maxiunivamalg #2|9|televisions|10|Electronics|71|oughtation|N/A|2479542581881yellow7|seashell|Cup|Unknown|67|eseableationpri| +3725|AAAAAAAAMIOAAAAA|1999-10-28|2001-10-26|Social, soft forms must deal a bit into a hours. Middle details would increase originally australian others. Somewhere single sounds must recognise then. Councillors will|42.04|34.47|2003002|exportiimporto #2|9|pants|2|Men|71|oughtation|economy|628snow1778294593105|wheat|Bundle|Unknown|61|antiableationpri| +3726|AAAAAAAAMIOAAAAA|2001-10-27||Social, soft forms must deal a bit into a hours. Middle details would increase originally australian others. Somewhere single sounds must recognise then. Councillors will|2.18|0.95|3003001|exportiexporti #1|3|toddlers|3|Children|393|prin stpri|medium|628snow1778294593105|tan|Gram|Unknown|55|callyableationpri| +3727|AAAAAAAAPIOAAAAA|1997-10-27||Private, complicated schools avoid still able problems. Now private environments can fit real, high hands. Companies shall catch nominally years; never disabled universities read also fr|0.28|0.08|1004002|edu packamalg #2|4|swimwear|1|Women|396|callyn stpri|medium|31956695coral8892227|papaya|Ton|Unknown|40|ationableationpri| +3728|AAAAAAAAAJOAAAAA|1997-10-27|2000-10-26|Prime qualities see then around a differences; exports apply often great officials; steadily certain problems agree econom|3.26|2.18|3001001|amalgexporti #1|1|newborn|3|Children|437|ationpriese|economy|976snow1126058322097|tomato|Dram|Unknown|22|eingableationpri| +3729|AAAAAAAAAJOAAAAA|2000-10-27||Prime qualities see then around a differences; exports apply often great officials; steadily certain problems agree econom|6.02|2.18|8009010|maxinameless #10|9|optics|8|Sports|437|ationpriese|N/A|47382854193purple568|sky|Each|Unknown|84|n stableationpri| +3730|AAAAAAAACJOAAAAA|1997-10-27|1999-10-27|Hours must carry virtually new seats. Polish, happy affairs might get. Originally warm libraries operate real patients. Then soft scie|28.04|25.23|4004001|edu packedu pack #1|4|athletic|4|Shoes|16|callyought|medium|6370051910puff571475|purple|Carton|Unknown|22|barpriationpri| +3731|AAAAAAAACJOAAAAA|1999-10-28|2001-10-26|Economic, like lengths should not say. Organisations used to mark also symbolic, right heads. Human, poor governments prevent available, beautiful routes; common places might hire at a companies. |8.78|6.49|4004001|edu packamalg #2|4|swimwear|1|Women|197|callyought|medium|9sky0551711114088437|rose|Pound|Unknown|25|oughtpriationpri| +3732|AAAAAAAACJOAAAAA|2001-10-27||White classes may not demand|1.16|6.49|9008001|namelessmaxi #1|8|romance|9|Books|647|callyought|N/A|6053269376456tan2800|slate|Carton|Unknown|28|ablepriationpri| +3733|AAAAAAAAFJOAAAAA|1997-10-27||Familiar chara|5.85|1.98|2001002|amalgimporto #2|1|accessories|2|Men|471|oughtationese|economy|10065033pink80250678|white|Unknown|Unknown|2|pripriationpri| +3734|AAAAAAAAGJOAAAAA|1997-10-27|2000-10-26|Bright areas will let; famous conditions say halfway |0.81|0.55|2003001|exportiimporto #1|3|pants|2|Men|30|barpri|small|5259282344385dodger0|frosted|Ton|Unknown|28|esepriationpri| +3735|AAAAAAAAGJOAAAAA|2000-10-27||Private workers act for a records. Hundreds would tell opponents. Present eyes blame. Groups could suffer. Plants might see p|2.66|0.55|5003002|exportischolar #2|3|pop|5|Music|94|esen st|N/A|5259282344385dodger0|spring|Gram|Unknown|12|antipriationpri| +3736|AAAAAAAAIJOAAAAA|1997-10-27|1999-10-27|New exports shall not rule |7.69|2.92|2003001|exportiimporto #1|3|pants|2|Men|600|barbarcally|medium|532888728887peru5780|red|Case|Unknown|53|callypriationpri| +3737|AAAAAAAAIJOAAAAA|1999-10-28|2001-10-26|Here massive ranks feel already. Sexual ea|0.40|0.24|9015006|scholarunivamalg #6|15|fiction|9|Books|600|barbarcally|N/A|532888728887peru5780|orchid|Pallet|Unknown|34|ationpriationpri| +3738|AAAAAAAAIJOAAAAA|2001-10-27||Here massive ranks feel already. Sexual ea|9.26|0.24|9015006|exportiedu pack #1|3|kids|4|Shoes|6|barbarcally|petite|54sky378344656010154|yellow|Pound|Unknown|7|eingpriationpri| +3739|AAAAAAAALJOAAAAA|1997-10-27||Buildings would not get with a tools. Current, united elections |0.82|0.73|8016010|corpmaxi #10|16|golf|8|Sports|501|oughtbaranti|N/A|seashell185666891770|almond|Dozen|Unknown|35|n stpriationpri| +3740|AAAAAAAAMJOAAAAA|1997-10-27|2000-10-26|Only late sales act vitally. Lonely authorities will not complement then personal messages. Financial, present customers shall think lo|0.71|0.24|3003001|exportiexporti #1|3|toddlers|3|Children|436|callypriese|petite|7turquoise2241541967|sandy|Oz|Unknown|48|bareseationpri| +3741|AAAAAAAAMJOAAAAA|2000-10-27||Already hot politicians feel apart new directions. Usual notes nee|3.24|1.19|3003001|brandbrand #2|3|decor|7|Home|125|callypriese|N/A|52060533787wheat6611|cornflower|Pallet|Unknown|48|oughteseationpri| +3742|AAAAAAAAOJOAAAAA|1997-10-27|1999-10-27|Sensible countries catch honestly centres; shoulders will display light, old examinations. Too long women|53.17|32.96|1001001|amalgamalg #1|1|dresses|1|Women|89|n steing|large|1489navy439765875997|plum|Tsp|Unknown|8|ableeseationpri| +3743|AAAAAAAAOJOAAAAA|1999-10-28|2001-10-26|Upstairs legal institutions ought to back doctors. Always dark eyes can die resid|18.74|32.96|5003002|exportischolar #2|1|pop|5|Music|89|n steing|N/A|9782454980indian4620|floral|Bunch|Unknown|43|prieseationpri| +3744|AAAAAAAAOJOAAAAA|2001-10-27||Western conditions expect quickly in relation to a areas. Weeks could make within a individuals. Rights start loud. Prices can replace visits. Wages halt h|0.30|32.96|5003002|importoscholar #1|1|country|5|Music|89|n steing|N/A|9782454980indian4620|tomato|Bundle|Unknown|16|eseeseationpri| +3745|AAAAAAAABKOAAAAA|1997-10-27||Familiar eyes cook well services. Stairs s|9.75|5.94|3002002|importoexporti #2|2|infants|3|Children|136|callypriought|large|puff9563776258367960|seashell|Pallet|Unknown|3|antieseationpri| +3746|AAAAAAAACKOAAAAA|1997-10-27|2000-10-26|Active, easy rounds manage now with a birds. Previous pieces could develo|3.96|1.74|10012007|importoamalgamalg #7|12|monitors|10|Electronics|139|n stpriought|N/A|spring66179623397676|tan|Gram|Unknown|91|callyeseationpri| +3747|AAAAAAAACKOAAAAA|2000-10-27||Active, easy rounds manage now with a birds. Previous pieces could develo|3.71|1.18|2002002|importoimporto #2|2|shirts|2|Men|139|n stpriought|extra large|spring66179623397676|seashell|Bundle|Unknown|18|ationeseationpri| +3748|AAAAAAAAEKOAAAAA|1997-10-27|1999-10-27|Major values might give ways. Then unaware services ought to want ac|3.17|1.64|3001001|amalgexporti #1|1|newborn|3|Children|533|priprianti|medium|4271926457996red1412|lawn|Gross|Unknown|5|eingeseationpri| +3749|AAAAAAAAEKOAAAAA|1999-10-28|2001-10-26|Major values might give ways. Then unaware services ought to want ac|7.37|2.57|3001001|scholarcorp #6|1|earings|6|Jewelry|321|priprianti|N/A|35485tan238919009312|wheat|Tsp|Unknown|38|n steseationpri| +3750|AAAAAAAAEKOAAAAA|2001-10-27||Overall unaware remains like appearances. Jewish gardens cannot keep children. New prayers will enable raw, significant years. Afterwards minute standards write late variable|4.19|2.57|3001001|edu packscholar #1|4|classical|5|Music|377|ationationpri|N/A|35485tan238919009312|thistle|Ounce|Unknown|97|barantiationpri| +3751|AAAAAAAAHKOAAAAA|1997-10-27||Physically natural times used to improve models. Significantly close years ought to build ahead linguistic habi|0.27|0.15|9013008|exportiunivamalg #8|13|self-help|9|Books|95|antin st|N/A|2snow677146363739327|ivory|Pound|Unknown|63|oughtantiationpri| +3752|AAAAAAAAIKOAAAAA|1997-10-27|2000-10-26|Certainly modern charges apologize precisely administrative, historic communities. Different women may not govern so. Yellow colleagues might not come hardly magnificent, a|3.96|2.85|10003013|exportiunivamalg #13|3|dvd/vcr players|10|Electronics|519|n stoughtanti|N/A|297920974708plum3319|powder|Gram|Unknown|24|ableantiationpri| +3753|AAAAAAAAIKOAAAAA|2000-10-27||Certainly modern charges apologize precisely administrative, historic communities. Different women may not govern so. Yellow colleagues might not come hardly magnificent, a|6.89|2.85|3004002|edu packexporti #2|4|school-uniforms|3|Children|87|n stoughtanti|medium|297920974708plum3319|snow|Case|Unknown|26|priantiationpri| +3754|AAAAAAAAKKOAAAAA|1997-10-27|1999-10-27|Large novels ought to carry|4.18|1.83|1003001|exportiamalg #1|3|maternity|1|Women|66|callycally|petite|2325774linen82278706|salmon|Ton|Unknown|26|eseantiationpri| +3755|AAAAAAAAKKOAAAAA|1999-10-28|2001-10-26|Large novels ought to carry|1.89|1.83|1003001|importoedu pack #2|2|mens|4|Shoes|41|callycally|medium|2325774linen82278706|sandy|Dozen|Unknown|29|antiantiationpri| +3756|AAAAAAAAKKOAAAAA|2001-10-27||New members like |7.09|1.83|1003001|edu packedu pack #1|2|athletic|4|Shoes|41|callycally|extra large|2325774linen82278706|pale|Box|Unknown|6|callyantiationpri| +3757|AAAAAAAANKOAAAAA|1997-10-27||Definitely visual animals end able applications. Successive, orange years shall leave gradually by the efforts. Standards cannot read at the officers. Small children tell soldiers|4.10|1.31|3001002|amalgexporti #2|1|newborn|3|Children|224|eseableable|large|7293908096navajo3571|turquoise|Dram|Unknown|61|ationantiationpri| +3758|AAAAAAAAOKOAAAAA|1997-10-27|2000-10-26|Heads get thus difficult supporters; big develop|0.87|0.53|8001009|amalgnameless #9|1|athletic shoes|8|Sports|309|n stbarpri|N/A|540748254075199rose1|peach|Pound|Unknown|43|eingantiationpri| +3759|AAAAAAAAOKOAAAAA|2000-10-27||External, so-called ages answer later services. Brave diseases shall not raise sometimes old parents. Branches drop considerable, soviet home|0.47|0.53|8001009|importoamalg #2|2|fragrances|1|Women|309|n stbarpri|large|540748254075199rose1|pale|N/A|Unknown|8|n stantiationpri| +3760|AAAAAAAAALOAAAAA|1997-10-27|1999-10-27|Teams would lead now through a eggs. Explanations think good, alone questions; liberal, religious plans alter then. True sports reduce eagerly racial, direct t|2.73|1.69|9015005|scholarunivamalg #5|15|fiction|9|Books|650|barantically|N/A|cyan0019965837207340|chiffon|Bundle|Unknown|11|barcallyationpri| +3761|AAAAAAAAALOAAAAA|1999-10-28|2001-10-26|Teams would lead now through a eggs. Explanations think good, alone questions; liberal, religious plans alter then. True sports reduce eagerly racial, direct t|8.43|5.05|9015005|exportibrand #2|15|loose stones|6|Jewelry|650|barantically|N/A|081302slate931872624|wheat|Gross|Unknown|89|oughtcallyationpri| +3762|AAAAAAAAALOAAAAA|2001-10-27||Other, bad countries may make |1.38|0.91|9015005|exporticorp #3|3|gold|6|Jewelry|471|barantically|N/A|081302slate931872624|saddle|Gross|Unknown|8|ablecallyationpri| +3763|AAAAAAAADLOAAAAA|1997-10-27||Likely advantages might save ago black hours. Thousands offer good organisation|1.88|0.67|2001002|amalgimporto #2|1|accessories|2|Men|666|callycallycally|extra large|72spring861351903047|rose|Case|Unknown|64|pricallyationpri| +3764|AAAAAAAAELOAAAAA|1997-10-27|2000-10-26|Educational, cool implications will not measure between the sales. Minutes find ago other, religious patients. Particularly direct numbers should keep various, prime forms; extreme, professio|7.73|3.16|1002001|importoamalg #1|2|fragrances|1|Women|390|barn stpri|N/A|02660351391260coral7|tomato|Oz|Unknown|90|esecallyationpri| +3765|AAAAAAAAELOAAAAA|2000-10-27||Educational, cool implications will not measure between the sales. Minutes find ago other, religious patients. Particularly direct numbers should keep various, prime forms; extreme, professio|4.67|3.16|5002002|importoscholar #2|2|country|5|Music|390|barn stpri|N/A|02660351391260coral7|forest|Dozen|Unknown|38|anticallyationpri| +3766|AAAAAAAAGLOAAAAA|1997-10-27|1999-10-27|Old, professional neighbours should continue as. Co|1.88|1.10|8010007|univmaxi #7|10|pools|8|Sports|114|eseoughtought|N/A|849663misty980853242|sky|N/A|Unknown|89|callycallyationpri| +3767|AAAAAAAAGLOAAAAA|1999-10-28|2001-10-26|Old, professional neighbours should continue as. Co|4.51|1.10|8010007|amalgmaxi #12|1|arts|9|Books|114|eseoughtought|N/A|849663misty980853242|snow|N/A|Unknown|16|ationcallyationpri| +3768|AAAAAAAAGLOAAAAA|2001-10-27||Old, professional neighbours should continue as. Co|4.62|2.77|1003001|exportiamalg #1|3|maternity|1|Women|114|eseoughtought|medium|849663misty980853242|red|Ounce|Unknown|32|eingcallyationpri| +3769|AAAAAAAAJLOAAAAA|1997-10-27||Dangerous relations believe about birds; western officials start assistant, high nights.|1.55|1.13|4002002|importoedu pack #2|2|mens|4|Shoes|256|callyantiable|large|moccasin378238638737|red|Oz|Unknown|47|n stcallyationpri| +3770|AAAAAAAAKLOAAAAA|1997-10-27|2000-10-26|Well environmental weeks could look so hands. Recent ministers tell businesses. Expressions act whole arts. Very military letters might bring fo|7.44|4.38|3001001|amalgexporti #1|1|newborn|3|Children|663|pricallycally|medium|498375936837red67346|peach|Bunch|Unknown|1|barationationpri| +3771|AAAAAAAAKLOAAAAA|2000-10-27||Real authorities rush. Competent expectations take there|5.88|4.38|5002002|importoscholar #2|1|country|5|Music|275|pricallycally|N/A|6210827saddle5281585|pale|Unknown|Unknown|31|oughtationationpri| +3772|AAAAAAAAMLOAAAAA|1997-10-27|1999-10-27|Critical services may m|4.51|3.60|1002001|importoamalg #1|2|fragrances|1|Women|142|ableeseought|small|73powder298321585463|tomato|Bundle|Unknown|21|ableationationpri| +3773|AAAAAAAAMLOAAAAA|1999-10-28|2001-10-26|Critical services may m|2.82|2.08|10007017|brandunivamalg #17|7|personal|10|Electronics|501|oughtbaranti|N/A|74497lavender1043035|seashell|Tsp|Unknown|26|priationationpri| +3774|AAAAAAAAMLOAAAAA|2001-10-27||Upper studies suggest sh|1.79|2.08|10007017|exportiimporto #1|7|pants|2|Men|501|oughtbaranti|medium|puff5534479228870403|violet|Bundle|Unknown|22|eseationationpri| +3775|AAAAAAAAPLOAAAAA|1997-10-27||Regional sets may call then much social securities; gentlemen must launch so further national women.|2.46|1.45|8012006|importomaxi #6|12|guns|8|Sports|462|ablecallyese|N/A|0mint963673718064582|slate|Bundle|Unknown|24|antiationationpri| +3776|AAAAAAAAAMOAAAAA|1997-10-27|2000-10-26|Follow|9.98|5.28|8009007|maxinameless #7|9|optics|8|Sports|208|eingbarable|N/A|895861893rosy7842475|pale|Bundle|Unknown|38|callyationationpri| +3777|AAAAAAAAAMOAAAAA|2000-10-27||Follow|44.57|5.28|8009007|corpamalgamalg #4|16|wireless|10|Electronics|208|eingbarable|N/A|2puff090178503255541|black|Ounce|Unknown|2|ationationationpri| +3778|AAAAAAAACMOAAAAA|1997-10-27|1999-10-27|Solid patients used to enjoy in a actions. Systems s|3.30|2.57|6013003|exportibrand #3|13|loose stones|6|Jewelry|372|ableationpri|N/A|4564878133502white29|powder|Ounce|Unknown|12|eingationationpri| +3779|AAAAAAAACMOAAAAA|1999-10-28|2001-10-26|Solid patients used to enjoy in a actions. Systems s|2.23|1.13|6013003|edu packexporti #2|4|school-uniforms|3|Children|276|ableationpri|small|4564878133502white29|smoke|Box|Unknown|32|n stationationpri| +3780|AAAAAAAACMOAAAAA|2001-10-27||Private, hostile times es|4.93|1.13|3004001|edu packexporti #1|4|school-uniforms|3|Children|21|oughtable|medium|4564878133502white29|ghost|Bunch|Unknown|11|bareingationpri| +3781|AAAAAAAAFMOAAAAA|1997-10-27||Especially public police give weeks. Low hard things acquire similarly firm times. Heels retain artificial, faint women. National goods would give consequently to a notes. Pictures would not finan|45.01|19.80|6015006|scholarbrand #6|15|custom|6|Jewelry|347|ationesepri|N/A|41503snow62845510821|cyan|Pallet|Unknown|36|oughteingationpri| +3782|AAAAAAAAGMOAAAAA|1997-10-27|2000-10-26|Valuable costs must know as into the weekends. Necessary years see feet. Things may not help men. Actively odd needs sit today times; too warm measures move. More still month|4.44|2.35|1004001|edu packamalg #1|4|swimwear|1|Women|667|ationcallycally|medium|3thistle848711245752|medium|Oz|Unknown|38|ableeingationpri| +3783|AAAAAAAAGMOAAAAA|2000-10-27||Then slow directors find also suitable, historic parts. Right new sequences care; maximum, other ministers would approach here like a sales. Anywa|5.44|2.35|10010010|univamalgamalg #10|10|memory|10|Electronics|548|eingeseanti|N/A|3thistle848711245752|peach|Case|Unknown|19|prieingationpri| +3784|AAAAAAAAIMOAAAAA|1997-10-27|1999-10-27|Just live roads bother firmly future parts. Sexual times distinguish; wages s|0.97|0.50|8002005|importonameless #5|2|baseball|8|Sports|986|callyeingn st|N/A|026515641pink6173594|goldenrod|Tsp|Unknown|14|eseeingationpri| +3785|AAAAAAAAIMOAAAAA|1999-10-28|2001-10-26|Accidents used to convince at a occasions; recent areas should improve never for the f|4.09|2.86|5004002|edu packscholar #2|2|classical|5|Music|986|callyeingn st|N/A|6003344502142violet8|olive|Lb|Unknown|3|antieingationpri| +3786|AAAAAAAAIMOAAAAA|2001-10-27||Ministers shake only changes; major factors feed during a hours; following, painful arguments would not become much again free ages. Purely |7.16|2.43|5004002|amalgexporti #1|2|newborn|3|Children|110|baroughtought|medium|1008989snow429986852|pink|Tsp|Unknown|79|callyeingationpri| +3787|AAAAAAAALMOAAAAA|1997-10-27||Important officers shall not prove symbol|52.24|47.01|1003002|exportiamalg #2|3|maternity|1|Women|327|ationablepri|economy|8670113maroon8527342|cornsilk|Dozen|Unknown|56|ationeingationpri| +3788|AAAAAAAAMMOAAAAA|1997-10-27|2000-10-26|Ever legal links improve owners; just sole acts breed ever hot, foreign doors. Bad books introduce. Obvious offices will not learn importa|9.21|5.34|4004001|edu packedu pack #1|4|athletic|4|Shoes|308|eingbarpri|medium|817993270874yellow53|salmon|Bundle|Unknown|34|eingeingationpri| +3789|AAAAAAAAMMOAAAAA|2000-10-27||Questions ensure very white, big beans. Orders provide quite disastrous activities. Wide, darling |7.07|2.89|4004001|amalgexporti #2|1|newborn|3|Children|308|eingbarpri|extra large|817993270874yellow53|ghost|Dram|Unknown|7|n steingationpri| +3790|AAAAAAAAOMOAAAAA|1997-10-27|1999-10-27|Patie|3.99|2.87|9006011|corpmaxi #11|6|parenting|9|Books|137|ationpriought|N/A|1756451739salmon7073|papaya|Gram|Unknown|69|barn stationpri| +3791|AAAAAAAAOMOAAAAA|1999-10-28|2001-10-26|Soviet, white items used to miss. Also traditional techniques might not enable. Supreme, dependent weeks will deal. Guilty fingers might handle reasons. Often big rules shall not brush cold friend|4.55|3.36|8002002|importonameless #2|2|baseball|8|Sports|165|ationpriought|N/A|1756451739salmon7073|pink|Cup|Unknown|15|oughtn stationpri| +3792|AAAAAAAAOMOAAAAA|2001-10-27||Royal questions might not need; separate eyes can respond academic minister|2.30|1.47|7008003|namelessbrand #3|8|lighting|7|Home|165|anticallyought|N/A|1756451739salmon7073|misty|Cup|Unknown|12|ablen stationpri| +3793|AAAAAAAABNOAAAAA|1997-10-27||National points satisfy difficu|0.82|0.72|10006003|corpunivamalg #3|6|musical|10|Electronics|533|priprianti|N/A|459hot67723906685629|papaya|Pound|Unknown|45|prin stationpri| +3794|AAAAAAAACNOAAAAA|1997-10-27|2000-10-26|Terms will happen today after a arguments. Most physical flowers doubt just. Other authorities would like still|4.15|2.61|9004003|edu packmaxi #3|4|entertainments|9|Books|589|n steinganti|N/A|powder33206868686124|rose|Gross|Unknown|31|esen stationpri| +3795|AAAAAAAACNOAAAAA|2000-10-27||New, double pictures stop never just |7.44|4.46|9004003|amalgnameless #6|11|accent|7|Home|589|n steinganti|N/A|powder33206868686124|yellow|Bunch|Unknown|41|antin stationpri| +3796|AAAAAAAAENOAAAAA|1997-10-27|1999-10-27|National methods may satisfy grateful days; current, private machines compete all black factors. Less full horses could become then; sure main men prepare forwa|3.01|2.25|3003001|exportiexporti #1|3|toddlers|3|Children|439|n stpriese|large|65782121seashell4182|sandy|Each|Unknown|50|callyn stationpri| +3797|AAAAAAAAENOAAAAA|1999-10-28|2001-10-26|Parliamentary, primary contents can work slig|1.52|0.54|5001002|amalgscholar #2|3|rock|5|Music|638|eingprically|N/A|4674rose742665682738|mint|Ton|Unknown|48|ationn stationpri| +3798|AAAAAAAAENOAAAAA|2001-10-27||Parliamentary, primary contents can work slig|3.47|0.54|5001002|brandcorp #7|7|pendants|6|Jewelry|6|cally|N/A|goldenrod19442613058|light|Box|Unknown|46|eingn stationpri| +3799|AAAAAAAAHNOAAAAA|1997-10-27||Individuals sa|64.97|25.98|2002002|importoimporto #2|2|shirts|2|Men|132|ablepriought|petite|4chocolate3988477019|honeydew|Unknown|Unknown|9|n stn stationpri| +3800|AAAAAAAAINOAAAAA|1997-10-27|2000-10-26|Further possible reasons develop. Alternative, short men might justify from the months. More active miles should not answer to a gaps. Liberal measures should help more so black nu|6.90|3.72|3002001|importoexporti #1|2|infants|3|Children|277|ationationable|N/A|59357lime51088635288|honeydew|Bundle|Unknown|57|barbareingpri| +3801|AAAAAAAAINOAAAAA|2000-10-27||Further possible reasons develop. Alternative, short men might justify from the months. More active miles should not answer to a gaps. Liberal measures should help more so black nu|0.69|3.72|2002002|importoimporto #2|2|shirts|2|Men|277|ationationable|petite|4640183honeydew82951|yellow|Carton|Unknown|5|oughtbareingpri| +3802|AAAAAAAAKNOAAAAA|1997-10-27|1999-10-27|Golden foundations buy elsewhere areas. Numerous prices achieve then hard, difficult users. Main dreams ought to plant fortunately fore|13.58|8.82|9014011|edu packunivamalg #11|14|sports|9|Books|116|callyoughtought|N/A|054212199797sky00893|cyan|Ton|Unknown|9|ablebareingpri| +3803|AAAAAAAAKNOAAAAA|1999-10-28|2001-10-26|Golden foundations buy elsewhere areas. Numerous prices achieve then hard, difficult users. Main dreams ought to plant fortunately fore|2.13|8.82|5004002|edu packscholar #2|4|classical|5|Music|47|callyoughtought|N/A|46817smoke8851920064|puff|Cup|Unknown|38|pribareingpri| +3804|AAAAAAAAKNOAAAAA|2001-10-27||Regardless gree|4.97|2.98|5004002|edu packamalgamalg #14|14|automotive|10|Electronics|47|callyoughtought|N/A|46817smoke8851920064|puff|Pallet|Unknown|14|esebareingpri| +3805|AAAAAAAANNOAAAAA|1997-10-27||Details give quickly; |78.18|61.76|9013002|exportiunivamalg #2|13|self-help|9|Books|251|oughtantiable|N/A|0015259pink867606875|light|Each|Unknown|25|antibareingpri| +3806|AAAAAAAAONOAAAAA|1997-10-27|2000-10-26|Issues should quote worth a children. All social years stand men. Problems consider to a errors. Old groups cost permanently; pink, normal goods consider. Particularly oth|6.23|3.23|9009003|maximaxi #3|9|science|9|Books|657|ationantically|N/A|15409orchid210722177|plum|Oz|Unknown|47|callybareingpri| +3807|AAAAAAAAONOAAAAA|2000-10-27||Political island|0.36|0.22|3001002|amalgexporti #2|9|newborn|3|Children|509|ationantically|extra large|15409orchid210722177|saddle|Pound|Unknown|32|ationbareingpri| +3808|AAAAAAAAAOOAAAAA|1997-10-27|1999-10-27|Small ways get usually then physical processes; important ministers will not perform else over a features. Relations like years. New, elegant holes should roll soviet, social plan|4.37|3.67|7012009|importonameless #9|12|paint|7|Home|31|oughtpri|N/A|812287rosy3392506414|pale|Pallet|Unknown|7|eingbareingpri| +3809|AAAAAAAAAOOAAAAA|1999-10-28|2001-10-26|Small ways get usually then physical processes; important ministers will not perform else over a features. Relations like years. New, elegant holes should roll soviet, social plan|8.44|3.67|9006006|corpmaxi #6|12|parenting|9|Books|9|oughtpri|N/A|7871193sandy20737626|saddle|Pound|Unknown|56|n stbareingpri| +3810|AAAAAAAAAOOAAAAA|2001-10-27||Small ways get usually then physical processes; important ministers will not perform else over a features. Relations like years. New, elegant holes should roll soviet, social plan|48.66|40.38|8005001|scholarnameless #1|12|fitness|8|Sports|9|oughtpri|N/A|913moccasin474109614|salmon|Tbl|Unknown|23|baroughteingpri| +3811|AAAAAAAADOOAAAAA|1997-10-27||Imperial, terrible windows commit exactly new premises; now various days can distract often. Poor rates produce good foods. Available, lab|2.33|0.69|7011002|amalgnameless #2|11|accent|7|Home|300|barbarpri|N/A|57653427511154sky244|papaya|Dozen|Unknown|27|oughtoughteingpri| +3812|AAAAAAAAEOOAAAAA|1997-10-27|2000-10-26|Weeks follow also following ministers; fat procedures used to encourage then clothes. Different paintings can cover talks. Still new minutes ensure again effects. Too extra waves move|4.95|2.62|8005003|scholarnameless #3|5|fitness|8|Sports|456|callyantiese|N/A|36847forest790472832|plum|Box|Unknown|20|ableoughteingpri| +3813|AAAAAAAAEOOAAAAA|2000-10-27||All right single men must not predict to the organisa|9.45|2.62|5002002|importoscholar #2|2|country|5|Music|188|eingeingought|N/A|54416peach7418655054|peach|Pallet|Unknown|65|prioughteingpri| +3814|AAAAAAAAGOOAAAAA|1997-10-27|1999-10-27|Really deaf masters will mobilize so even curious stairs. Clearly early championships shall give much. Successful years should set more in a opera|1.87|0.63|5003001|exportischolar #1|3|pop|5|Music|189|n steingought|N/A|8418powder4301313950|purple|Gross|Unknown|34|eseoughteingpri| +3815|AAAAAAAAGOOAAAAA|1999-10-28|2001-10-26|Really deaf masters will mobilize so even curious stairs. Clearly early championships shall give much. Successful years should set more in a opera|3.66|0.63|10016008|corpamalgamalg #8|16|wireless|10|Electronics|269|n stcallyable|N/A|582862966white180791|powder|Pound|Unknown|59|antioughteingpri| +3816|AAAAAAAAGOOAAAAA|2001-10-27||Apparently active costs must not remain for instance large, |1.98|1.30|10016008|univbrand #1|16|jewelry boxes|6|Jewelry|834|eseprieing|N/A|cream217249152049345|peru|Ounce|Unknown|59|callyoughteingpri| +3817|AAAAAAAAJOOAAAAA|1997-10-27||Only classical groups c|9.44|5.38|3001002|amalgexporti #2|1|newborn|3|Children|219|n stoughtable|large|5948080835609orange1|goldenrod|N/A|Unknown|83|ationoughteingpri| +3818|AAAAAAAAKOOAAAAA|1997-10-27|2000-10-26|S|79.64|32.65|6013003|exportibrand #3|13|loose stones|6|Jewelry|311|oughtoughtpri|N/A|9664796510233slate88|sienna|Unknown|Unknown|25|eingoughteingpri| +3819|AAAAAAAAKOOAAAAA|2000-10-27||S|1.13|32.65|9003004|exportimaxi #4|3|computers|9|Books|311|oughtoughtpri|N/A|068820267538491snow8|tomato|Pallet|Unknown|18|n stoughteingpri| +3820|AAAAAAAAMOOAAAAA|1997-10-27|1999-10-27|Now natural cases must not protect so old friends. Possible, helpful buildings keep. Aspects explain complete eyes; much wrong hours seem just police. Polls should instruct. Unnecessary champio|3.27|2.15|1002001|importoamalg #1|2|fragrances|1|Women|503|pribaranti|extra large|2175977901purple5932|medium|Lb|Unknown|10|barableeingpri| +3821|AAAAAAAAMOOAAAAA|1999-10-28|2001-10-26|Little, important newspapers use with a m|15.91|2.15|1002001|amalgscholar #2|1|rock|5|Music|503|pribaranti|N/A|457459peru7421056218|tomato|Unknown|Unknown|52|oughtableeingpri| +3822|AAAAAAAAMOOAAAAA|2001-10-27||Little, important newspapers use with a m|1.48|1.15|8003003|exportinameless #3|1|basketball|8|Sports|487|ationeingese|N/A|24876149910rosy05071|moccasin|Each|Unknown|10|ableableeingpri| +3823|AAAAAAAAPOOAAAAA|1997-10-27||Together popular organisers bear usually major stars; immediately difficult pupils might happen then senses. Electrical rocks wea|12.20|6.46|6014008|edu packbrand #8|14|estate|6|Jewelry|165|anticallyought|N/A|91029511slate0173569|rose|Tbl|Unknown|12|priableeingpri| +3824|AAAAAAAAAPOAAAAA|1997-10-27|2000-10-26|A little average flames ought to break old, unique men. Things select often red, economic others. Hands will lift sufficiently; german, proper sections worry perhaps for the po|1.79|1.32|8014005|edu packmaxi #5|14|tennis|8|Sports|373|priationpri|N/A|8yellow2381289250166|wheat|Dram|Unknown|16|eseableeingpri| +3825|AAAAAAAAAPOAAAAA|2000-10-27||Executives|4.55|2.27|2001002|amalgimporto #2|1|accessories|2|Men|373|priationpri|medium|106450920548peach081|ivory|Lb|Unknown|65|antiableeingpri| +3826|AAAAAAAACPOAAAAA|1997-10-27|1999-10-27|Heavy, common authorities must think. Long st|13.11|8.52|3003001|exportiexporti #1|3|toddlers|3|Children|349|n stesepri|economy|46324556salmon818832|sandy|Ton|Unknown|8|callyableeingpri| +3827|AAAAAAAACPOAAAAA|1999-10-28|2001-10-26|Heavy, common authorities must think. Long st|1.52|8.52|3003001|edu packscholar #2|4|classical|5|Music|349|n stesepri|N/A|7984988971576900tan0|steel|Tsp|Unknown|12|ationableeingpri| +3828|AAAAAAAACPOAAAAA|2001-10-27||Criminal, legal hands leave again top, special copies; other feet take oft|9.11|8.52|9002001|importomaxi #1|4|business|9|Books|349|n stesepri|N/A|7984988971576900tan0|pale|Oz|Unknown|29|eingableeingpri| +3829|AAAAAAAAFPOAAAAA|1997-10-27||Certain, clear parties lead most about a volumes. Difficult, asian children should catch; pro|4.56|3.92|8004010|edu packnameless #10|4|camping|8|Sports|575|antiationanti|N/A|04580447031purple524|wheat|Each|Unknown|39|n stableeingpri| +3830|AAAAAAAAGPOAAAAA|1997-10-27|2000-10-26|Really possible rules obtain only w|8.47|6.94|1003001|exportiamalg #1|3|maternity|1|Women|211|oughtoughtable|medium|808052420spring03660|ghost|Unknown|Unknown|1|barprieingpri| +3831|AAAAAAAAGPOAAAAA|2000-10-27||Really possible rules obtain only w|4.45|2.58|8011002|amalgmaxi #2|3|archery|8|Sports|185|oughtoughtable|N/A|808052420spring03660|powder|Ton|Unknown|11|oughtprieingpri| +3832|AAAAAAAAIPOAAAAA|1997-10-27|1999-10-27|Comprehensive plans must plan even in a rules. Intermittently good children can form notions. Negative, likely sectors open even devices. Invisible,|6.21|3.60|7014003|edu packnameless #3|14|glassware|7|Home|285|antieingable|N/A|477923796036300rosy3|yellow|Carton|Unknown|21|ableprieingpri| +3833|AAAAAAAAIPOAAAAA|1999-10-28|2001-10-26|Victorian, eager books find as opening, english voters; religious, |2.25|1.44|7014003|edu packmaxi #4|14|tennis|8|Sports|64|antieingable|N/A|811046sienna34397552|salmon|Ton|Unknown|16|priprieingpri| +3834|AAAAAAAAIPOAAAAA|2001-10-27||Victorian, eager books find as opening, english voters; religious, |4.08|1.44|7014003|amalgunivamalg #10|14|cameras|10|Electronics|390|antieingable|N/A|492658985972saddle61|puff|Oz|Unknown|39|eseprieingpri| +3835|AAAAAAAALPOAAAAA|1997-10-27||Open services may |5.85|2.10|6003004|exporticorp #4|3|gold|6|Jewelry|560|barcallyanti|N/A|589186285wheat261703|lime|Bundle|Unknown|95|antiprieingpri| +3836|AAAAAAAAMPOAAAAA|1997-10-27|2000-10-26|Late other courts start now mountains; only previous styles keep political, small matters; always new years witness so; internal, equal relationships |3.81|1.86|2004001|edu packimporto #1|4|sports-apparel|2|Men|447|ationeseese|N/A|89690212pink00566545|yellow|Box|Unknown|56|callyprieingpri| +3837|AAAAAAAAMPOAAAAA|2000-10-27||Old, other stories shall borrow. Soft, other things shall not blame; relationships encourage large, social points. Local movies would look rather particularly english theme|0.39|1.86|7014008|edu packnameless #8|4|glassware|7|Home|447|ationeseese|N/A|89690212pink00566545|wheat|Carton|Unknown|10|ationprieingpri| +3838|AAAAAAAAOPOAAAAA|1997-10-27|1999-10-27|Objections used to sell others. Level places co-ordinate quite to the drawings. |2.83|1.81|1002001|importoamalg #1|2|fragrances|1|Women|932|ableprin st|petite|2859084012slate98350|sienna|Bundle|Unknown|24|eingprieingpri| +3839|AAAAAAAAOPOAAAAA|1999-10-28|2001-10-26|Objections used to sell others. Level places co-ordinate quite to the drawings. |1.22|1.81|5004002|edu packscholar #2|2|classical|5|Music|320|barablepri|N/A|2859084012slate98350|chartreuse|N/A|Unknown|36|n stprieingpri| +3840|AAAAAAAAOPOAAAAA|2001-10-27||Specialist days may break gold, unknown risks. Basic, stupid interests must place in a featur|68.58|1.81|5004002|edu packscholar #1|4|classical|5|Music|320|barablepri|N/A|2859084012slate98350|metallic|Pallet|Unknown|47|bareseeingpri| +3841|AAAAAAAABAPAAAAA|1997-10-27||Rates see today things. Records would realize|4.51|3.56|3004002|edu packexporti #2|4|school-uniforms|3|Children|438|eingpriese|large|03rose77940172914901|moccasin|Gram|Unknown|22|oughteseeingpri| +3842|AAAAAAAACAPAAAAA|1997-10-27|2000-10-26|Western types shall make according|7.34|5.72|2003001|exportiimporto #1|3|pants|2|Men|728|eingableation|small|7253069wheat08872405|misty|Cup|Unknown|15|ableeseeingpri| +3843|AAAAAAAACAPAAAAA|2000-10-27||Problems put popular resources. Homes provide also to a years. Important poets shall not move there with the paintings. So clinical effects used to support in a shares. Othe|8.43|5.72|2003001|amalgexporti #2|3|newborn|3|Children|353|priantipri|petite|18turquoise577088793|sienna|Dram|Unknown|81|prieseeingpri| +3844|AAAAAAAAEAPAAAAA|1997-10-27|1999-10-27|Just distinct children think individuals; popular arguments develop here cautious methods; appropriate children might beat. Proper, empirical hundreds fall oth|4.01|3.44|9003005|exportimaxi #5|3|computers|9|Books|254|eseantiable|N/A|6631rosy372829004625|peru|Unknown|Unknown|31|eseeseeingpri| +3845|AAAAAAAAEAPAAAAA|1999-10-28|2001-10-26|Particularly beautiful months balance afraid, uniq|1.14|3.44|9003005|importounivamalg #4|3|camcorders|10|Electronics|254|eseantiable|N/A|6631rosy372829004625|chocolate|Case|Unknown|6|antieseeingpri| +3846|AAAAAAAAEAPAAAAA|2001-10-27||Particularly beautiful months balance afraid, uniq|6.45|3.44|9003005|exportinameless #7|3|basketball|8|Sports|254|eseantiable|N/A|6631rosy372829004625|rose|Bunch|Unknown|24|callyeseeingpri| +3847|AAAAAAAAHAPAAAAA|1997-10-27||Even apparent goals make over up to excellent environments. Figures used to alter anywhere. Here slight agreements match. Statutory, old men get old personnel. Effects manage |0.20|0.17|2003002|exportiimporto #2|3|pants|2|Men|53|prianti|large|3270spring0213741083|cyan|Lb|Unknown|21|ationeseeingpri| +3848|AAAAAAAAIAPAAAAA|1997-10-27|2000-10-26|Large, daily results qualify women. Pp. support also. Growing, perm|0.29|0.14|8005009|scholarnameless #9|5|fitness|8|Sports|810|baroughteing|N/A|999moccasin532233370|tan|Ton|Unknown|52|eingeseeingpri| +3849|AAAAAAAAIAPAAAAA|2000-10-27||Global, poor influences differ away from the weaknesses. Pre|69.94|0.14|8005009|amalgscholar #2|1|rock|5|Music|582|ableeinganti|N/A|0318697turquoise1442|plum|Gram|Unknown|17|n steseeingpri| +3850|AAAAAAAAKAPAAAAA|1997-10-27|1999-10-27|Very prime teachers exercise really different, only jobs. Rights might not maintain in brief sectors. Convincing, bright tables leave clubs; grand courses cannot respond very. Really northe|4.77|3.33|4001001|amalgedu pack #1|1|womens|4|Shoes|613|prioughtcally|petite|royal716316762366867|peru|Each|Unknown|7|barantieingpri| +3851|AAAAAAAAKAPAAAAA|1999-10-28|2001-10-26|Very prime teachers exercise really different, only jobs. Rights might not maintain in brief sectors. Convincing, bright tables leave clubs; grand courses cannot respond very. Really northe|2.43|1.62|3001002|amalgexporti #2|1|newborn|3|Children|613|prioughtcally|medium|83828666cornsilk3617|pale|Tbl|Unknown|25|oughtantieingpri| +3852|AAAAAAAAKAPAAAAA|2001-10-27||As jus|6.53|2.41|5003001|exportischolar #1|1|pop|5|Music|71|prioughtcally|N/A|37836273906886puff11|lawn|Tbl|Unknown|4|ableantieingpri| +3853|AAAAAAAANAPAAAAA|1997-10-27||Gently other places qualify rational matches. Definitely supreme circles answer corporate, notable pictures. Generous, strategic orders ought to address public days. Employees answer perh|4.95|3.96|7015004|scholarnameless #4|15|tables|7|Home|605|antibarcally|N/A|2435404pink387186436|sienna|Bundle|Unknown|73|priantieingpri| +3854|AAAAAAAAOAPAAAAA|1997-10-27|2000-10-26|Players require only services. Figures reflect really candidates. Yet recent candidates will mean general, above coins. International houses could train in general dishes. Simply|9.66|4.34|8001005|amalgnameless #5|1|athletic shoes|8|Sports|976|callyationn st|N/A|7573025papaya5492413|puff|Dram|Unknown|81|eseantieingpri| +3855|AAAAAAAAOAPAAAAA|2000-10-27||Weekly, important writers may not expect naturally buildings. Laboratories go so common properties. Original categories man|89.65|4.34|8001005|namelessnameless #6|8|outdoor|8|Sports|464|esecallyese|N/A|7573025papaya5492413|sandy|Tbl|Unknown|45|antiantieingpri| +3856|AAAAAAAAABPAAAAA|1997-10-27|1999-10-27|Particularly professional women may not tell never present, distant times. Current, only weeks could hurry quite appropriate months. Little attacks waste carefully never politi|1.82|1.51|9011005|amalgunivamalg #5|11|cooking|9|Books|88|eingeing|N/A|92125232425saddle746|red|Tbl|Unknown|27|callyantieingpri| +3857|AAAAAAAAABPAAAAA|1999-10-28|2001-10-26|Eyes see rightly lucky homes. Legal, hidden practices might chat suddenly at a contracts. Facts could not put true problems;|5.33|4.58|9011005|corpmaxi #6|6|parenting|9|Books|88|eingeing|N/A|11465plum27107035648|violet|Unknown|Unknown|58|ationantieingpri| +3858|AAAAAAAAABPAAAAA|2001-10-27||Eyes see rightly lucky homes. Legal, hidden practices might chat suddenly at a contracts. Facts could not put true problems;|2.45|4.58|9011005|exportiedu pack #1|6|kids|4|Shoes|88|eingeing|petite|20969floral753753194|wheat|Case|Unknown|62|eingantieingpri| +3859|AAAAAAAADBPAAAAA|1997-10-27||Never international aspects could get just students. Quietl|4.30|3.35|5004002|edu packscholar #2|4|classical|5|Music|521|oughtableanti|N/A|5blanched96744589987|dim|Cup|Unknown|2|n stantieingpri| +3860|AAAAAAAAEBPAAAAA|1997-10-27|2000-10-26|Strange, big bodies beat behind high rates. Marginal, blank homes cannot begin int|0.28|0.09|5001001|amalgscholar #1|1|rock|5|Music|272|ableationable|N/A|yellow84696499632347|powder|Pound|Unknown|25|barcallyeingpri| +3861|AAAAAAAAEBPAAAAA|2000-10-27||Elections will use physical ar|4.04|0.09|6015006|scholarbrand #6|1|custom|6|Jewelry|851|oughtantieing|N/A|yellow84696499632347|sienna|Carton|Unknown|29|oughtcallyeingpri| +3862|AAAAAAAAGBPAAAAA|1997-10-27|1999-10-27|Sudden lights provide both possible films; again electronic entries could keep widely from the men. Then able women cannot uphold thorou|2.84|1.10|3002001|importoexporti #1|2|infants|3|Children|513|prioughtanti|medium|261721460peach631041|steel|Gram|Unknown|41|ablecallyeingpri| +3863|AAAAAAAAGBPAAAAA|1999-10-28|2001-10-26|Sudden lights provide both possible films; again electronic entries could keep widely from the men. Then able women cannot uphold thorou|6.58|1.10|3002001|importoimporto #2|2|shirts|2|Men|513|prioughtanti|medium|13powder988011180126|orange|Dram|Unknown|28|pricallyeingpri| +3864|AAAAAAAAGBPAAAAA|2001-10-27||Competitors develop on the islands. Wise mothers would obtain as s|0.59|1.10|3002001|maxibrand #5|2|mattresses|7|Home|434|prioughtanti|N/A|2214864burlywood1360|violet|Bundle|Unknown|21|esecallyeingpri| +3865|AAAAAAAAJBPAAAAA|1997-10-27||Once again general students would not get things. Perhaps armed legs c|3.30|2.40|1001002|amalgamalg #2|1|dresses|1|Women|185|antieingought|medium|3647609741883pink538|tan|Unknown|Unknown|7|anticallyeingpri| +3866|AAAAAAAAKBPAAAAA|1997-10-27|2000-10-26|Heavy, political rivers get so in a forms. Jobs stop very. Ju|9.52|3.04|10012008|importoamalgamalg #8|12|monitors|10|Electronics|692|ablen stcally|N/A|506purple12863704524|moccasin|Gross|Unknown|2|callycallyeingpri| +3867|AAAAAAAAKBPAAAAA|2000-10-27||Heavy, political rivers get so in a forms. Jobs stop very. Ju|6.12|3.04|10012008|amalgamalgamalg #9|12|disk drives|10|Electronics|692|ablen stcally|N/A|506purple12863704524|salmon|Dozen|Unknown|12|ationcallyeingpri| +3868|AAAAAAAAMBPAAAAA|1997-10-27|1999-10-27|Chemicals must see exactly tired corners; studies should offend evenly available, separate con|5.08|3.55|5001001|amalgscholar #1|1|rock|5|Music|341|oughtesepri|N/A|291028318sky18598263|gainsboro|Cup|Unknown|17|eingcallyeingpri| +3869|AAAAAAAAMBPAAAAA|1999-10-28|2001-10-26|Young, molecular assets punch quite hard, grey friends. Pale, responsible thanks reply again different, prospective indicators. Secretly corr|4.19|2.72|1002002|importoamalg #2|2|fragrances|1|Women|227|ationableable|N/A|0318347289wheat48738|ivory|Ton|Unknown|83|n stcallyeingpri| +3870|AAAAAAAAMBPAAAAA|2001-10-27||Young, molecular assets punch quite hard, grey friends. Pale, responsible thanks reply again different, prospective indicators. Secretly corr|7.83|2.72|1002002|namelessnameless #1|2|outdoor|8|Sports|227|ationableable|N/A|82548010sky672594050|orange|Ton|Unknown|98|barationeingpri| +3871|AAAAAAAAPBPAAAAA|1997-10-27||Wrong countries see countries; lengths will see possible sc|3.38|1.48|9014008|edu packunivamalg #8|14|sports|9|Books|406|callybarese|N/A|17471851048052wheat8|tomato|Tsp|Unknown|19|oughtationeingpri| +3872|AAAAAAAAACPAAAAA|1997-10-27|2000-10-26|Wels|0.98|0.55|1002001|importoamalg #1|2|fragrances|1|Women|70|baration|medium|6204142081olive80548|yellow|N/A|Unknown|27|ableationeingpri| +3873|AAAAAAAAACPAAAAA|2000-10-27||Inc members light. Mental, national ways rent yet pr|3.81|1.25|10011015|amalgamalgamalg #15|2|disk drives|10|Electronics|70|baration|N/A|6204142081olive80548|khaki|Tsp|Unknown|33|priationeingpri| +3874|AAAAAAAACCPAAAAA|1997-10-27|1999-10-27|Still old sides keep really save for a police. Big, foreign things enable. Other children illustrate distinct, distingui|0.46|0.20|7004005|edu packbrand #5|4|curtains/drapes|7|Home|408|eingbarese|N/A|9908405573117peach31|grey|Box|Unknown|8|eseationeingpri| +3875|AAAAAAAACCPAAAAA|1999-10-28|2001-10-26|Still old sides keep really save for a police. Big, foreign things enable. Other children illustrate distinct, distingui|0.46|0.26|7004005|scholarnameless #6|4|fitness|8|Sports|408|eingbarese|N/A|0navy881512687190892|slate|Gross|Unknown|16|antiationeingpri| +3876|AAAAAAAACCPAAAAA|2001-10-27||Terribly primitive questions add there levels. Possible, black changes play under trains. Quite tiny studies consider|4.93|2.07|10008001|namelessunivamalg #1|8|scanners|10|Electronics|25|eingbarese|N/A|0navy881512687190892|orange|Carton|Unknown|7|callyationeingpri| +3877|AAAAAAAAFCPAAAAA|1997-10-27||Historical, broad letters see; perhaps local activities will cure fair remote, effective schools. Customers used to shed prayers. Other guests will not give too old activi|1.06|0.42|1003002|exportiamalg #2|3|maternity|1|Women|978|eingationn st|small|80659354peach4362724|tan|Tsp|Unknown|4|ationationeingpri| +3878|AAAAAAAAGCPAAAAA|1997-10-27|2000-10-26|Costs receive. British teachers evolve mentally only, new words. Good tickets give |0.12|0.06|3004001|edu packexporti #1|4|school-uniforms|3|Children|86|callyeing|large|5799634639989wheat82|tan|N/A|Unknown|2|eingationeingpri| +3879|AAAAAAAAGCPAAAAA|2000-10-27||Costs receive. British teachers evolve mentally only, new words. Good tickets give |3.72|0.06|4004002|edu packedu pack #2|4|athletic|4|Shoes|724|eseableation|large|358984turquoise25369|violet|N/A|Unknown|8|n stationeingpri| +3880|AAAAAAAAICPAAAAA|1997-10-27|1999-10-27|Up |7.52|3.98|6012001|importobrand #1|12|costume|6|Jewelry|204|esebarable|N/A|182518036grey8690782|rosy|Carton|Unknown|94|bareingeingpri| +3881|AAAAAAAAICPAAAAA|1999-10-28|2001-10-26|Up |1.55|3.98|6012001|amalgbrand #2|1|bathroom|7|Home|204|esebarable|N/A|182518036grey8690782|spring|Cup|Unknown|42|oughteingeingpri| +3882|AAAAAAAAICPAAAAA|2001-10-27||Records exercise then small friends. True wages would ask other answers. Birds will not seek really far legal networks. Benefits change here equal, civil windows. Ways wish cheap. Files|2.79|3.98|10002007|importounivamalg #7|2|camcorders|10|Electronics|435|antipriese|N/A|182518036grey8690782|peach|Ounce|Unknown|30|ableeingeingpri| +3883|AAAAAAAALCPAAAAA|1997-10-27||Always small values will love important markets. Likely, hard links used to kill much philosophical, extensive supporters. A|3.70|2.10|7015004|scholarnameless #4|15|tables|7|Home|240|bareseable|N/A|05313450papaya335123|violet|Gross|Unknown|11|prieingeingpri| +3884|AAAAAAAAMCPAAAAA|1997-10-27|2000-10-26|Proudly personal costs cut very able, big rights. Sufficient years will promote nearly achievements. Nations woul|0.93|0.34|4002001|importoedu pack #1|2|mens|4|Shoes|647|ationesecally|extra large|997417787906spring99|violet|Pallet|Unknown|24|eseeingeingpri| +3885|AAAAAAAAMCPAAAAA|2000-10-27||Comparatively absolute copies could get cars; national, particular classes will determine until a organs. Slightl|3.02|0.34|4002001|maxicorp #6|2|womens watch|6|Jewelry|84|eseeing|N/A|9117218639royal45159|wheat|N/A|Unknown|60|antieingeingpri| +3886|AAAAAAAAOCPAAAAA|1997-10-27|1999-10-27|Main lives can know further electrical, leading properties. Figures waste a little thoughts. Pp. reduce now real twins. Future, final lawyers will serve r|16.50|7.09|10013011|exportiamalgamalg #11|13|stereo|10|Electronics|157|ationantiought|N/A|3278156hot4470921579|turquoise|Dram|Unknown|31|callyeingeingpri| +3887|AAAAAAAAOCPAAAAA|1999-10-28|2001-10-26|Never certain operators shall look just demanding, novel masters. Royal documents control even academic bodies. Downstairs following trees might not take too downst|2.45|7.09|10013011|corpmaxi #8|16|golf|8|Sports|264|ationantiought|N/A|9974snow379795379031|spring|Tbl|Unknown|64|ationeingeingpri| +3888|AAAAAAAAOCPAAAAA|2001-10-27||Never certain operators shall look just demanding, novel masters. Royal documents control even academic bodies. Downstairs following trees might not take too downst|6.42|7.09|5004001|edu packscholar #1|4|classical|5|Music|470|ationantiought|N/A|9974snow379795379031|wheat|Box|Unknown|13|eingeingeingpri| +3889|AAAAAAAABDPAAAAA|1997-10-27||Keen years fight much. Concerned, vital kings get downstairs new, worthy millions. Else full gam|2.95|2.21|9015002|scholarunivamalg #2|15|fiction|9|Books|538|eingprianti|N/A|83smoke4388574923275|royal|Bunch|Unknown|67|n steingeingpri| +3890|AAAAAAAACDPAAAAA|1997-10-27|2000-10-26|Old, immediate months see especially different leaders. Other, pale charges influence even english, middle-class others; pregnant, wrong eyes help by way of the activ|3.61|2.41|9010003|univunivamalg #3|10|travel|9|Books|79|n station|N/A|25pale78198007290788|violet|Oz|Unknown|14|barn steingpri| +3891|AAAAAAAACDPAAAAA|2000-10-27||Old, immediate months see especially different leaders. Other, pale charges influence even english, middle-class others; pregnant, wrong eyes help by way of the activ|99.99|71.99|9010003|corpmaxi #4|10|parenting|9|Books|79|n station|N/A|25pale78198007290788|linen|Bundle|Unknown|14|oughtn steingpri| +3892|AAAAAAAAEDPAAAAA|1997-10-27|1999-10-27|Yellow things make governments; royal |9.34|4.48|10016017|corpamalgamalg #17|16|wireless|10|Electronics|609|n stbarcally|N/A|9927063031745slate71|peach|Ounce|Unknown|32|ablen steingpri| +3893|AAAAAAAAEDPAAAAA|1999-10-28|2001-10-26|Yellow things make governments; royal |3.25|1.46|10016017|edu packamalg #2|4|swimwear|1|Women|609|n stbarcally|petite|9snow138991925309625|lawn|Ton|Unknown|15|prin steingpri| +3894|AAAAAAAAEDPAAAAA|2001-10-27||Stations could emerge left, horizontal boards. Social days|4.62|3.69|10016017|scholarunivamalg #7|4|fiction|9|Books|609|n stbarcally|N/A|9snow138991925309625|lemon|Lb|Unknown|55|esen steingpri| +3895|AAAAAAAAHDPAAAAA|1997-10-27||Early, possible forces release long dirty |6.26|5.50|8016006|corpmaxi #6|16|golf|8|Sports|602|ablebarcally|N/A|4214544peach01354211|antique|Each|Unknown|49|antin steingpri| +3896|AAAAAAAAIDPAAAAA|1997-10-27|2000-10-26|Total organisations celebrate hard benefits. Points could not c|8.43|6.40|4004001|edu packedu pack #1|4|athletic|4|Shoes|370|barationpri|small|647410lavender558288|puff|Bundle|Unknown|12|callyn steingpri| +3897|AAAAAAAAIDPAAAAA|2000-10-27||Existing changes take even favorite, english industries. As public consequences find upwards months. Regional aspects need recently with a systems. Now essent|5.89|6.40|8012008|importomaxi #8|4|guns|8|Sports|370|barationpri|N/A|5powder0778913969925|turquoise|Cup|Unknown|32|ationn steingpri| +3898|AAAAAAAAKDPAAAAA|1997-10-27|1999-10-27|Backwards royal assets get plans; countries used to swing then. Most strange|1.84|0.82|4003001|exportiedu pack #1|3|kids|4|Shoes|190|barn stought|N/A|869073121166steel553|lavender|Box|Unknown|27|eingn steingpri| +3899|AAAAAAAAKDPAAAAA|1999-10-28|2001-10-26|Just pregnant walls mean in a components. Silly, international figures find also steadily married women. Days might try in a hearts. Aspects will not help never. Fascinating, public standards stand he|0.31|0.27|4003001|edu packimporto #2|4|sports-apparel|2|Men|371|oughtationpri|small|304491278768maroon06|salmon|Gram|Unknown|57|n stn steingpri| +3900|AAAAAAAAKDPAAAAA|2001-10-27||Years investigate seats. Scientific detectives should listen today. Relat|8.44|0.27|4001001|amalgedu pack #1|4|womens|4|Shoes|371|oughtationpri|medium|304491278768maroon06|peru|Carton|Unknown|18|barbarn stpri| +3901|AAAAAAAANDPAAAAA|1997-10-27||Very special others smile rather. Tools might decide other times. Wages may fit almost. Black relations would come on|0.98|0.45|7012002|importonameless #2|12|paint|7|Home|642|ableesecally|N/A|291ghost103919182582|puff|Carton|Unknown|23|oughtbarn stpri| +3902|AAAAAAAAODPAAAAA|1997-10-27|2000-10-26|In particular long-term masses may remove sometimes in a results. New ranks|5.94|4.27|8009003|maxinameless #3|9|optics|8|Sports|54|eseanti|N/A|5020298red2300377010|turquoise|Unknown|Unknown|28|ablebarn stpri| +3903|AAAAAAAAODPAAAAA|2000-10-27||Only artistic ages recover programmes; experienced parents ought to free local gif|8.44|5.57|4003002|exportiedu pack #2|9|kids|4|Shoes|205|antibarable|medium|5020298red2300377010|tan|N/A|Unknown|49|pribarn stpri| +3904|AAAAAAAAAEPAAAAA|1997-10-27|1999-10-27|Causes|3.57|2.71|8015005|scholarmaxi #5|15|fishing|8|Sports|313|prioughtpri|N/A|503rosy7632298021369|linen|Pallet|Unknown|89|esebarn stpri| +3905|AAAAAAAAAEPAAAAA|1999-10-28|2001-10-26|Causes|6.66|2.71|8015005|edu packamalg #2|4|swimwear|1|Women|313|prioughtpri|extra large|503rosy7632298021369|rose|Dram|Unknown|47|antibarn stpri| +3906|AAAAAAAAAEPAAAAA|2001-10-27||Causes|2.11|2.71|8015005|exportimaxi #7|4|computers|9|Books|600|barbarcally|N/A|503rosy7632298021369|smoke|Unknown|Unknown|16|callybarn stpri| +3907|AAAAAAAADEPAAAAA|1997-10-27||Really other strengths used to drive then by a needs; pupils might not conflict of course now elderly mice|9.42|4.52|6009004|maxicorp #4|9|womens watch|6|Jewelry|874|eseationeing|N/A|2184807511548powder9|violet|Gram|Unknown|5|ationbarn stpri| +3908|AAAAAAAAEEPAAAAA|1997-10-27|2000-10-26|Easy, new days lock highly historians; families can go military powers. Perhaps alone places used to try orthodox enemies. Political, silent|8.21|2.95|5001001|amalgscholar #1|1|rock|5|Music|379|n stationpri|N/A|086304274982spring71|rosy|Tsp|Unknown|22|eingbarn stpri| +3909|AAAAAAAAEEPAAAAA|2000-10-27||Common, important relationships note at all sensitive pa|1.82|2.95|4004002|edu packedu pack #2|4|athletic|4|Shoes|379|n stationpri|medium|086304274982spring71|goldenrod|Bunch|Unknown|19|n stbarn stpri| +3910|AAAAAAAAGEPAAAAA|1997-10-27|1999-10-27|Social men build also national, key parents; boys may take particularly here lost reasons. Opportunities used to i|56.67|31.16|7008001|namelessbrand #1|8|lighting|7|Home|260|barcallyable|N/A|87papaya742635527562|navy|Case|Unknown|55|baroughtn stpri| +3911|AAAAAAAAGEPAAAAA|1999-10-28|2001-10-26|Social men build also national, key parents; boys may take particularly here lost reasons. Opportunities used to i|1.99|0.91|7008001|importoamalg #2|8|fragrances|1|Women|337|barcallyable|medium|92110mint78084612986|linen|Oz|Unknown|32|oughtoughtn stpri| +3912|AAAAAAAAGEPAAAAA|2001-10-27||Anyway japanese applications may strike rationally jobs; national memories avoid projects. Most true windows pay partly cultural steps. For example practical units|39.26|0.91|4002001|importoedu pack #1|2|mens|4|Shoes|337|barcallyable|medium|92110mint78084612986|linen|Gram|Unknown|16|ableoughtn stpri| +3913|AAAAAAAAJEPAAAAA|1997-10-27||Adults reply hence outside weak lives. Chief beliefs put hence open police. Concrete, private conservatives result in a securities. Suddenly dear findings can stop completely during |7.13|4.42|2002002|importoimporto #2|2|shirts|2|Men|996|callyn stn st|medium|6866110992693powder8|puff|Cup|Unknown|9|prioughtn stpri| +3914|AAAAAAAAKEPAAAAA|1997-10-27|2000-10-26|Times must produce. Also special areas change affairs. Legs join at a dreams. Guests make eastern, fine goals; political children should shift again immediate years. Attempts used to stop |1.23|0.89|5001001|amalgscholar #1|1|rock|5|Music|78|eingation|N/A|171055734papaya12589|plum|Case|Unknown|76|eseoughtn stpri| +3915|AAAAAAAAKEPAAAAA|2000-10-27||Times must produce. Also special areas change affairs. Legs join at a dreams. Guests make eastern, fine goals; political children should shift again immediate years. Attempts used to stop |2.88|0.89|5001001|importoscholar #2|1|country|5|Music|365|eingation|N/A|171055734papaya12589|orange|Lb|Unknown|12|antioughtn stpri| +3916|AAAAAAAAMEPAAAAA|1997-10-27|1999-10-27|Also political flowers confirm now then close companies. Just past arguments should produce additional, outer ingredients. Tories should see. Goods |8.28|6.95|5004001|edu packscholar #1|4|classical|5|Music|40|barese|N/A|976951medium78478565|royal|Gram|Unknown|13|callyoughtn stpri| +3917|AAAAAAAAMEPAAAAA|1999-10-28|2001-10-26|Also political flowers confirm now then close companies. Just past arguments should produce additional, outer ingredients. Tories should see. Goods |5.48|1.69|10015008|scholaramalgamalg #8|15|portable|10|Electronics|233|barese|N/A|492906692101peru2738|azure|Oz|Unknown|70|ationoughtn stpri| +3918|AAAAAAAAMEPAAAAA|2001-10-27||Traditional officials establish. Unions supervise as well. Most high structures could not fear well more victorian standards. Slim years used to continue ill,|3.77|1.69|1003001|exportiamalg #1|3|maternity|1|Women|233|pripriable|large|727peru3853582147928|white|Ton|Unknown|57|eingoughtn stpri| +3919|AAAAAAAAPEPAAAAA|1997-10-27||Concerned strategies would know |2.41|1.80|1004002|edu packamalg #2|4|swimwear|1|Women|345|antiesepri|extra large|30855973122saddle113|puff|Each|Unknown|7|n stoughtn stpri| +3920|AAAAAAAAAFPAAAAA|1997-10-27|2000-10-26|Miles could produce probably seconds; small officials will build islands. Natural specialists s|8.45|3.97|8009001|maxinameless #1|9|optics|8|Sports|224|eseableable|N/A|00370013984042smoke6|wheat|Ounce|Unknown|24|barablen stpri| +3921|AAAAAAAAAFPAAAAA|2000-10-27||Most available things give so hard refugees. Alone years ought to turn proportions. Calls know. Necessary, unnecessary da|6.23|3.05|2001002|amalgimporto #2|9|accessories|2|Men|191|oughtn stought|N/A|42989544576670steel0|spring|Tsp|Unknown|4|oughtablen stpri| +3922|AAAAAAAACFPAAAAA|1997-10-27|1999-10-27|Already great boots walk actually present masses. Following, common ports win sort of little consequences; shoulders imply more. Chinese characteristics would hear|5.22|3.39|6016003|corpbrand #3|16|consignment|6|Jewelry|394|esen stpri|N/A|salmon05350290670536|white|Ounce|Unknown|1|ableablen stpri| +3923|AAAAAAAACFPAAAAA|1999-10-28|2001-10-26|Married supporters must interpret instead minutes. Political eyes may put for ex|4.44|3.39|10011014|amalgamalgamalg #14|16|disk drives|10|Electronics|119|esen stpri|N/A|salmon05350290670536|lawn|Gross|Unknown|43|priablen stpri| +3924|AAAAAAAACFPAAAAA|2001-10-27||Months keep so. Strong women use never. Far politicians represent like head dangers. Children might not get much stairs. Thing|3.52|3.39|9007001|brandmaxi #1|16|reference|9|Books|119|esen stpri|N/A|51559456peru89405545|tomato|Pound|Unknown|21|eseablen stpri| +3925|AAAAAAAAFFPAAAAA|1997-10-27||More natural feet should assume ever due, certain problems. Large offic|3.94|2.28|8010006|univmaxi #6|10|pools|8|Sports|285|antieingable|N/A|7247178244rose009614|white|Unknown|Unknown|34|antiablen stpri| +3926|AAAAAAAAGFPAAAAA|1997-10-27|2000-10-26|Profits deliver. Even possible guidelines ought to cry new teeth; necessary events will hear quickly counties. Pocket|7.31|4.09|7016007|corpnameless #7|16|furniture|7|Home|109|n stbarought|N/A|7068132wheat69763495|white|Cup|Unknown|15|callyablen stpri| +3927|AAAAAAAAGFPAAAAA|2000-10-27||Profits deliver. Even possible guidelines ought to cry new teeth; necessary events will hear quickly counties. Pocket|1.63|4.09|7015008|scholarnameless #8|15|tables|7|Home|109|n stbarought|N/A|3wheat15682316049006|snow|Pound|Unknown|4|ationablen stpri| +3928|AAAAAAAAIFPAAAAA|1997-10-27|1999-10-27|Sacred deaths represent at the foods. Bedrooms run procedures. Unknown, british difficulties used to demand less on the measures. Written, only years cou|1.47|1.07|5004001|edu packscholar #1|4|classical|5|Music|661|oughtcallycally|N/A|374764cornflower3152|pink|Oz|Unknown|26|eingablen stpri| +3929|AAAAAAAAIFPAAAAA|1999-10-28|2001-10-26|English champions blame to a jobs. Skills might throw fully historians. Soft countries could not afford else on the subjects; available books will not |2.54|2.10|1004002|edu packamalg #2|4|swimwear|1|Women|287|oughtcallycally|N/A|4960408lawn998346911|dark|Gross|Unknown|30|n stablen stpri| +3930|AAAAAAAAIFPAAAAA|2001-10-27||High old men should give really from the terms. Though suspicious days lead home developing |10.07|2.10|1004002|edu packedu pack #1|4|athletic|4|Shoes|594|esen stanti|medium|8001999747178wheat71|medium|Case|Unknown|54|barprin stpri| +3931|AAAAAAAALFPAAAAA|1997-10-27||Potatoes illustrate away prices. Central men need in the guns; different, material v|4.10|3.69|6004004|edu packcorp #4|4|bracelets|6|Jewelry|633|priprically|N/A|51540pink61477368002|slate|Carton|Unknown|24|oughtprin stpri| +3932|AAAAAAAAMFPAAAAA|1997-10-27|2000-10-26|English, different topics mean too. Long, lucky residents discover also away from a victims. Lakes restrict. Certain, standard banks belong tiny centres. E|0.70|0.43|1004001|edu packamalg #1|4|swimwear|1|Women|147|ationeseought|small|2685636289382white25|pale|Ounce|Unknown|17|ableprin stpri| +3933|AAAAAAAAMFPAAAAA|2000-10-27||English, different topics mean too. Long, lucky residents discover also away from a victims. Lakes restrict. Certain, standard banks belong tiny centres. E|9.04|7.59|9015010|scholarunivamalg #10|4|fiction|9|Books|57|ationanti|N/A|2685636289382white25|tan|Gross|Unknown|10|priprin stpri| +3934|AAAAAAAAOFPAAAAA|1997-10-27|1999-10-27|Increasing, serious materials shall not slow horrible elements. Horizontal topics avoid less figures|4.59|1.46|1001001|amalgamalg #1|1|dresses|1|Women|406|callybarese|small|466367966464orchid56|puff|Gross|Unknown|50|eseprin stpri| +3935|AAAAAAAAOFPAAAAA|1999-10-28|2001-10-26|Increasing, serious materials shall not slow horrible elements. Horizontal topics avoid less figures|4.18|3.59|1001001|exportibrand #6|1|kids|7|Home|277|callybarese|N/A|466367966464orchid56|white|Carton|Unknown|9|antiprin stpri| +3936|AAAAAAAAOFPAAAAA|2001-10-27||Increasing, serious materials shall not slow horrible elements. Horizontal topics avoid less figures|6.61|2.11|1001001|brandunivamalg #10|7|personal|10|Electronics|277|ationationable|N/A|466367966464orchid56|wheat|Tbl|Unknown|32|callyprin stpri| +3937|AAAAAAAABGPAAAAA|1997-10-27||Ame|8.59|7.21|6006002|corpcorp #2|6|rings|6|Jewelry|514|eseoughtanti|N/A|3772353559rose289731|saddle|Oz|Unknown|31|ationprin stpri| +3938|AAAAAAAACGPAAAAA|1997-10-27|2000-10-26|Innocent babies m|0.87|0.40|10005012|scholarunivamalg #12|5|karoke|10|Electronics|325|antiablepri|N/A|6616thistle164948961|peru|Case|Unknown|58|eingprin stpri| +3939|AAAAAAAACGPAAAAA|2000-10-27||Innocent babies m|4.59|3.07|5001002|amalgscholar #2|5|rock|5|Music|325|antiablepri|N/A|brown436868430051627|salmon|Carton|Unknown|41|n stprin stpri| +3940|AAAAAAAAEGPAAAAA|1997-10-27|1999-10-27|Living kids use by far by a prizes; social, other efforts accept risks. Further high rivals set office|19.51|17.16|6013005|exportibrand #5|13|loose stones|6|Jewelry|346|callyesepri|N/A|18958133lemon3374554|sky|Pound|Unknown|9|baresen stpri| +3941|AAAAAAAAEGPAAAAA|1999-10-28|2001-10-26|Candidates make conventional centres; advisory, subject parts can miss correct, small times. Words think else other, urban products. Gra|4.70|1.55|6002006|importocorp #6|2|diamonds|6|Jewelry|346|callyesepri|N/A|742royal498587243520|slate|Ounce|Unknown|30|oughtesen stpri| +3942|AAAAAAAAEGPAAAAA|2001-10-27||Candidates make conventional centres; advisory, subject parts can miss correct, small times. Words think else other, urban products. Gra|7.33|2.41|6001007|amalgcorp #7|1|birdal|6|Jewelry|885|callyesepri|N/A|742royal498587243520|tan|Oz|Unknown|26|ableesen stpri| +3943|AAAAAAAAHGPAAAAA|1997-10-27||Historic, level to|29.30|20.21|9016008|corpunivamalg #8|16|mystery|9|Books|654|eseantically|N/A|6334saddle7228030780|metallic|Tbl|Unknown|54|priesen stpri| +3944|AAAAAAAAIGPAAAAA|1997-10-27|2000-10-26|Happy procedures will make flat, single teachers. Coloured, economic concepts|4.08|2.93|9005009|scholarmaxi #9|5|history|9|Books|260|barcallyable|N/A|23ivory9169260239668|snow|Bunch|Unknown|5|eseesen stpri| +3945|AAAAAAAAIGPAAAAA|2000-10-27||Fun members provide all right huge, main items. Poor areas would not let already overseas, charming plans. Able, other eyes spend anyway boundaries. Monthly lessons shall go also good |0.77|0.27|1002002|importoamalg #2|5|fragrances|1|Women|260|barcallyable|extra large|78921seashell6020512|misty|Bunch|Unknown|32|antiesen stpri| +3946|AAAAAAAAKGPAAAAA|1997-10-27|1999-10-27|There chief conditions get therefore eyes. Significant, small ideas use at a deposits. New, minor minerals shall not drive |49.69|43.23|8005007|scholarnameless #7|5|fitness|8|Sports|280|bareingable|N/A|1723087771steel16346|smoke|Bundle|Unknown|22|callyesen stpri| +3947|AAAAAAAAKGPAAAAA|1999-10-28|2001-10-26|Creditors should fall; possible years must feel only sure posts. Young shops know rapidly permanent, prime doctors; minutes ought|4.07|3.58|2002002|importoimporto #2|5|shirts|2|Men|280|bareingable|large|1723087771steel16346|puff|N/A|Unknown|23|ationesen stpri| +3948|AAAAAAAAKGPAAAAA|2001-10-27||Only, early proposals might not identify even soft, single services. Especially common results can sell perhaps. Times check else nearly single eyes. Social cases boast from |2.23|3.58|2002002|edu packamalg #1|5|swimwear|1|Women|280|bareingable|petite|18450992094649olive0|smoke|Dram|Unknown|54|eingesen stpri| +3949|AAAAAAAANGPAAAAA|1997-10-27||Particular, new defences ought to defer modern studies. Methods ought to plant |6.46|4.13|8001010|amalgnameless #10|1|athletic shoes|8|Sports|98|eingn st|N/A|2coral22447572307978|thistle|Cup|Unknown|90|n stesen stpri| +3950|AAAAAAAAOGPAAAAA|1997-10-27|2000-10-26|Well planned problems use more in a origins; main, senior sons enter right, substantial faces. Typical, other measures must counteract so minutes; yet |1.28|0.98|8016001|corpmaxi #1|16|golf|8|Sports|635|antiprically|N/A|burnished77160586376|puff|Case|Unknown|30|barantin stpri| +3951|AAAAAAAAOGPAAAAA|2000-10-27||Intact, temp|6.32|0.98|8016001|edu packedu pack #2|4|athletic|4|Shoes|463|antiprically|large|papaya83459345085462|wheat|Bunch|Unknown|28|oughtantin stpri| +3952|AAAAAAAAAHPAAAAA|1997-10-27|1999-10-27|Surveys shall not ne|4.61|3.41|9011005|amalgunivamalg #5|11|cooking|9|Books|384|eseeingpri|N/A|390095422438white655|puff|Ton|Unknown|89|ableantin stpri| +3953|AAAAAAAAAHPAAAAA|1999-10-28|2001-10-26|Surveys shall not ne|9.42|3.41|9011005|importoedu pack #2|11|mens|4|Shoes|663|eseeingpri|large|8231639moccasin80156|green|Bundle|Unknown|21|priantin stpri| +3954|AAAAAAAAAHPAAAAA|2001-10-27||Merchants see complex rocks. Due, ethnic thousands go precise, complete lines. Popular others conflict at the pieces. Alive, permanent|2.42|1.76|9011005|edu packcorp #3|11|bracelets|6|Jewelry|968|eingcallyn st|N/A|8231639moccasin80156|white|Gram|Unknown|3|eseantin stpri| +3955|AAAAAAAADHPAAAAA|1997-10-27||Hundreds will not stop great years. Methods ought to last vaguely plants. |1.35|1.13|7003006|exportibrand #6|3|kids|7|Home|233|pripriable|N/A|600619rosy5976646904|wheat|Carton|Unknown|16|antiantin stpri| +3956|AAAAAAAAEHPAAAAA|1997-10-27|2000-10-26|Particular, possible signs tend lips. Economic centres should get; surprising, poor flats used to clinch similar terms. Agricultural words ought t|1.33|1.17|2002001|importoimporto #1|2|shirts|2|Men|255|antiantiable|large|3766purple3623784141|rose|Ton|Unknown|6|callyantin stpri| +3957|AAAAAAAAEHPAAAAA|2000-10-27||Honest, great areas may not catch. Individual boards should compete very secondly follo|9.91|1.17|5004002|edu packscholar #2|4|classical|5|Music|577|ationationanti|N/A|3766purple3623784141|forest|Ton|Unknown|22|ationantin stpri| +3958|AAAAAAAAGHPAAAAA|1997-10-27|1999-10-27|Large, pure employees locate ever dead relationships. Good fingers shall suffer only on a things; economic elections occ|0.93|0.71|3004001|edu packexporti #1|4|school-uniforms|3|Children|30|barpri|medium|3924779almond7697753|puff|N/A|Unknown|89|eingantin stpri| +3959|AAAAAAAAGHPAAAAA|1999-10-28|2001-10-26|Pains get. Tonnes must consider rare, other owners. Personally high comparisons should develop small, proper patients. In particular hard visitors m|2.22|0.71|3004001|importoimporto #2|2|shirts|2|Men|30|barpri|large|3924779almond7697753|tomato|Bunch|Unknown|43|n stantin stpri| +3960|AAAAAAAAGHPAAAAA|2001-10-27||Outside simple materials regard with a boys. Amounts should start even social, u|8.60|0.71|4002001|importoedu pack #1|2|mens|4|Shoes|30|barpri|extra large|3924779almond7697753|peach|Carton|Unknown|25|barcallyn stpri| +3961|AAAAAAAAJHPAAAAA|1997-10-27||Serious crimes must finance yesterday strong fathers. Ever sure years would not say. International, new att|0.92|0.68|10015001|scholaramalgamalg #1|15|portable|10|Electronics|834|eseprieing|N/A|132270thistle9822806|coral|Box|Unknown|21|oughtcallyn stpri| +3962|AAAAAAAAKHPAAAAA|1997-10-27|2000-10-26|Necessary, important years can co-operate naturally properties. Sharp insects may see french, only things. Below significant winds|1.57|1.38|8016003|corpmaxi #3|16|golf|8|Sports|304|esebarpri|N/A|82278462517yellow228|sandy|Each|Unknown|60|ablecallyn stpri| +3963|AAAAAAAAKHPAAAAA|2000-10-27||Old funds express previously. Linear women move. Both female principles ought to establish yet wide others. Patients lose so concepts. Only industrial jou|2.87|1.92|4001002|amalgedu pack #2|16|womens|4|Shoes|304|esebarpri|petite|82278462517yellow228|purple|Each|Unknown|35|pricallyn stpri| +3964|AAAAAAAAMHPAAAAA|1997-10-27|1999-10-27|Available centres go in a ears. Arrangements cannot stay expectations. French buildings used to use now ago ex|9.81|3.92|8011005|amalgmaxi #5|11|archery|8|Sports|316|callyoughtpri|N/A|purple32525401183038|steel|Ounce|Unknown|9|esecallyn stpri| +3965|AAAAAAAAMHPAAAAA|1999-10-28|2001-10-26|Available centres go in a ears. Arrangements cannot stay expectations. French buildings used to use now ago ex|6.66|2.93|8011005|edu packexporti #2|4|school-uniforms|3|Children|316|callyoughtpri|medium|9989729835peach60492|seashell|Carton|Unknown|25|anticallyn stpri| +3966|AAAAAAAAMHPAAAAA|2001-10-27||Available centres go in a ears. Arrangements cannot stay expectations. French buildings used to use now ago ex|8.96|7.52|8011005|amalgscholar #1|4|rock|5|Music|316|callyoughtpri|N/A|696snow1843645368436|saddle|Pallet|Unknown|7|callycallyn stpri| +3967|AAAAAAAAPHPAAAAA|1997-10-27||Variable, cruel countries must not find skills. Significantl|3.11|1.02|8015008|scholarmaxi #8|15|fishing|8|Sports|964|esecallyn st|N/A|58852659518steel0991|smoke|Gross|Unknown|44|ationcallyn stpri| +3968|AAAAAAAAAIPAAAAA|1997-10-27|2000-10-26|Children could see again ahead of the patients. Nice terms ought to get high materials. Odd others would enable at a others. Together common subjects shall not interrupt other, brig|6.74|2.83|10010008|univamalgamalg #8|10|memory|10|Electronics|405|antibarese|N/A|62violet683543880415|plum|Bunch|Unknown|12|eingcallyn stpri| +3969|AAAAAAAAAIPAAAAA|2000-10-27||Children could see again ahead of the patients. Nice terms ought to get high materials. Odd others would enable at a others. Together common subjects shall not interrupt other, brig|4.90|2.94|10010008|scholaramalgamalg #9|15|portable|10|Electronics|612|antibarese|N/A|62violet683543880415|seashell|Carton|Unknown|29|n stcallyn stpri| +3970|AAAAAAAACIPAAAAA|1997-10-27|1999-10-27|Multiple stars give in general for a panels. Real schools |44.12|38.82|9010011|univunivamalg #11|10|travel|9|Books|560|barcallyanti|N/A|1909161973goldenrod5|lavender|Unknown|Unknown|23|barationn stpri| +3971|AAAAAAAACIPAAAAA|1999-10-28|2001-10-26|Multiple stars give in general for a panels. Real schools |5.53|4.70|8013002|exportimaxi #2|10|sailing|8|Sports|560|barcallyanti|N/A|58127257grey88621094|steel|N/A|Unknown|8|oughtationn stpri| +3972|AAAAAAAACIPAAAAA|2001-10-27||Multiple stars give in general for a panels. Real schools |24.22|13.07|9003001|exportimaxi #1|3|computers|9|Books|560|barcallyanti|N/A|79224681spring473664|almond|Ounce|Unknown|48|ableationn stpri| +3973|AAAAAAAAFIPAAAAA|1997-10-27||Away able bills should not like private, other universities. Years meet sharply fat, middle lips. Nuclear acts might buy western names; libraries examine also. Single, powerful problems win |4.04|2.74|1001002|amalgamalg #2|1|dresses|1|Women|253|priantiable|medium|10254203269midnight5|spring|Each|Unknown|32|priationn stpri| +3974|AAAAAAAAGIPAAAAA|1997-10-27|2000-10-26|Terms would look women. Long-term, remote arms see more willing w|2.19|1.75|4003001|exportiedu pack #1|3|kids|4|Shoes|508|eingbaranti|N/A|477thistle9915072899|peach|Ounce|Unknown|27|eseationn stpri| +3975|AAAAAAAAGIPAAAAA|||Terms would look women. Long-term, remote arms see more willing w|75.43|1.75||scholarnameless #6|5||8|Sports|492||||yellow|Cup|||antiationn stpri| +3976|AAAAAAAAIIPAAAAA|1997-10-27|1999-10-27|Attractive, private levels used to run too european, sexua|75.24|51.16|1004001|edu packamalg #1|4|swimwear|1|Women|411|oughtoughtese|extra large|royal023646913064506|turquoise|Tbl|Unknown|44|callyationn stpri| +3977|AAAAAAAAIIPAAAAA|1999-10-28|2001-10-26|Attractive, private levels used to run too european, sexua|5.29|51.16|1004001|edu packimporto #2|4|sports-apparel|2|Men|103|oughtoughtese|medium|88675spring705444723|wheat|Each|Unknown|21|ationationn stpri| +3978|AAAAAAAAIIPAAAAA|2001-10-27||Attractive, private levels used to run too european, sexua|8.11|51.16|1004001|exportinameless #9|13|wallpaper|7|Home|82|oughtoughtese|N/A|88675spring705444723|almond|Ton|Unknown|9|eingationn stpri| +3979|AAAAAAAALIPAAAAA|1997-10-27||Imaginative, old areas may own happy items. Types make in a historians. Western s|0.34|0.11|8001010|amalgnameless #10|1|athletic shoes|8|Sports|465|anticallyese|N/A|211plum4083783157154|steel|Bundle|Unknown|3|n stationn stpri| +3980|AAAAAAAAMIPAAAAA|1997-10-27|2000-10-26|Unlikely ends look inside members. Small, heavy women would lay however armies. New thanks make rather |4.19|3.18|5001001|amalgscholar #1|1|rock|5|Music|466|callycallyese|N/A|6224262690880pale527|goldenrod|Case|Unknown|43|bareingn stpri| +3981|AAAAAAAAMIPAAAAA|2000-10-27||Unlikely ends look inside members. Small, heavy women would lay however armies. New thanks make rather |4.54|1.36|5001001|corpamalgamalg #4|1|wireless|10|Electronics|466|callycallyese|N/A|6224262690880pale527|saddle|Ounce|Unknown|14|oughteingn stpri| +3982|AAAAAAAAOIPAAAAA|1997-10-27|1999-10-27|Minutes ought to carry about a operations. Also hidden terms zero approximately please local borders. Traditionally harsh pieces could not remain too patt|2.59|1.55|7016003|corpnameless #3|16|furniture|7|Home|860|barcallyeing|N/A|44rose61779437206064|yellow|Ton|Unknown|36|ableeingn stpri| +3983|AAAAAAAAOIPAAAAA|1999-10-28|2001-10-26|Minutes ought to carry about a operations. Also hidden terms zero approximately please local borders. Traditionally harsh pieces could not remain too patt|5.76|1.55|4001002|amalgedu pack #2|1|womens|4|Shoes|860|barcallyeing|small|44rose61779437206064|sandy|Each|Unknown|78|prieingn stpri| +3984|AAAAAAAAOIPAAAAA|2001-10-27||White, long advantages produce regions. Only things go due, detailed months; lin|8.42|1.55|4001002|edu packamalg #1|1|swimwear|1|Women|860|barcallyeing|extra large|177093788993tan71116|tomato|Oz|Unknown|27|eseeingn stpri| +3985|AAAAAAAABJPAAAAA|1997-10-27||Soon intermediate needs should increase more feet. Useful participants enable; much |77.28|50.23|8007006|brandnameless #6|7|hockey|8|Sports|596|callyn stanti|N/A|7559735890172tomato0|papaya|Box|Unknown|12|antieingn stpri| +3986|AAAAAAAACJPAAAAA|1997-10-27|2000-10-26|Corp|0.89|0.54|3002001|importoexporti #1|2|infants|3|Children|585|antieinganti|petite|75046720008spring103|hot|Ton|Unknown|27|callyeingn stpri| +3987|AAAAAAAACJPAAAAA|2000-10-27||Broken fingers used to get always industrial relationships. Big needs jump for a residents. Outside, individual deaths might make just sexual parents. Departments could seem. Over |90.17|64.92|3002001|amalgscholar #2|2|rock|5|Music|585|antieinganti|N/A|85327499842royal8135|medium|Dozen|Unknown|42|ationeingn stpri| +3988|AAAAAAAAEJPAAAAA|1997-10-27|1999-10-27|Political, natural pools used to abandon then english, previous days. Old patients see concerned, long authorities. About internal visitors would hold sadly. Phases shall not argue relat|6.01|3.78|1002001|importoamalg #1|2|fragrances|1|Women|121|oughtableought|small|112696violet87416741|rose|Ton|Unknown|86|eingeingn stpri| +3989|AAAAAAAAEJPAAAAA|1999-10-28|2001-10-26|Political, natural pools used to abandon then english, previous days. Old patients see concerned, long authorities. About internal visitors would hold sadly. Phases shall not argue relat|0.48|3.78|10006012|corpunivamalg #12|6|musical|10|Electronics|305|oughtableought|N/A|112696violet87416741|yellow|N/A|Unknown|30|n steingn stpri| +3990|AAAAAAAAEJPAAAAA|2001-10-27||Political, natural pools used to abandon then english, previous days. Old patients see concerned, long authorities. About internal visitors would hold sadly. Phases shall not argue relat|3.44|3.78|10006012|brandbrand #1|6|decor|7|Home|305|antibarpri|N/A|112696violet87416741|powder|Ounce|Unknown|24|barn stn stpri| +3991|AAAAAAAAHJPAAAAA|1997-10-27||Secret, |3.18|2.22|6001008|amalgcorp #8|1|birdal|6|Jewelry|108|eingbarought|N/A|5666905714830khaki91|turquoise|Dozen|Unknown|66|oughtn stn stpri| +3992|AAAAAAAAIJPAAAAA|1997-10-27|2000-10-26|Ways fall politically markets. |0.84|0.66|5004001|edu packscholar #1|4|classical|5|Music|433|pripriese|N/A|0spring5930309040833|wheat|Ton|Unknown|26|ablen stn stpri| +3993|AAAAAAAAIJPAAAAA|2000-10-27||Loc|0.77|0.39|3001002|amalgexporti #2|4|newborn|3|Children|996|callyn stn st|large|0spring5930309040833|pale|Unknown|Unknown|41|prin stn stpri| +3994|AAAAAAAAKJPAAAAA|1997-10-27|1999-10-27|Much special advantages disturb then. Only old-fashioned others cost sympathetic, sick facts. Qu|47.83|33.95|6003003|exporticorp #3|3|gold|6|Jewelry|699|n stn stcally|N/A|4294salmon2101266275|pink|Cup|Unknown|46|esen stn stpri| +3995|AAAAAAAAKJPAAAAA|1999-10-28|2001-10-26|Much special advantages disturb then. Only old-fashioned others cost sympathetic, sick facts. Qu|4.68|33.95|6003003|univunivamalg #12|10|travel|9|Books|699|n stn stcally|N/A|9245734thistle218787|smoke|Unknown|Unknown|28|antin stn stpri| +3996|AAAAAAAAKJPAAAAA|2001-10-27||Remote voices can succeed later actions. Errors extend well poor matters|4.15|33.95|3001001|amalgexporti #1|10|newborn|3|Children|699|n stn stcally|small|9245734thistle218787|wheat|Each|Unknown|47|callyn stn stpri| +3997|AAAAAAAANJPAAAAA|1997-10-27||Apparent visitors might make more particular communities; heavy, mass effects retain then foreign, active bags. Never existing memb|8.29|5.22|10007003|brandunivamalg #3|7|personal|10|Electronics|59|n stanti|N/A|667spring55897392799|spring|Bundle|Unknown|74|ationn stn stpri| +3998|AAAAAAAAOJPAAAAA|1997-10-27|2000-10-26|Minute, private relations would break for a things. Broad schools affect in the cars. More asleep|3.74|1.19|3001001|amalgexporti #1|1|newborn|3|Children|614|eseoughtcally|medium|921olive986078662791|royal|Tsp|Unknown|66|eingn stn stpri| +3999|AAAAAAAAOJPAAAAA|2000-10-27||Foll|6.92|5.60|10012005|importoamalgamalg #5|12|monitors|10|Electronics|614|eseoughtcally|N/A|921olive986078662791|floral|N/A|Unknown|30|n stn stn stpri| +4000|AAAAAAAAAKPAAAAA|1997-10-27|1999-10-27|Supposed wives wo|90.56|57.95|4004001|edu packedu pack #1|4|athletic|4|Shoes|460|barcallyese|medium|2787974146yellow3891|pink|Bundle|Unknown|20|barbarbarese| +4001|AAAAAAAAAKPAAAAA|1999-10-28|2001-10-26|Old, continental orders remain with a poems. Activities should let more closely human b|5.18|57.95|4004001|corpcorp #2|4|rings|6|Jewelry|68|barcallyese|N/A|2787974146yellow3891|salmon|Dozen|Unknown|3|oughtbarbarese| +4002|AAAAAAAAAKPAAAAA|2001-10-27||Old, continental orders remain with a poems. Activities should let more closely human b|3.43|2.09|7016003|corpnameless #3|16|furniture|7|Home|68|eingcally|N/A|2787974146yellow3891|spring|Box|Unknown|53|ablebarbarese| +4003|AAAAAAAADKPAAAAA|1997-10-27||Reall|1.50|0.82|5002002|importoscholar #2|2|country|5|Music|510|baroughtanti|N/A|90381972rose01094975|rosy|Cup|Unknown|75|pribarbarese| +4004|AAAAAAAAEKPAAAAA|1997-10-27|2000-10-26|So good eggs obtain schools. Late, great windows might not start other, free ideas. Stations can zero especially com|0.26|0.11|6013005|exportibrand #5|13|loose stones|6|Jewelry|200|barbarable|N/A|peach078626381933962|seashell|Gross|Unknown|91|esebarbarese| +4005|AAAAAAAAEKPAAAAA|2000-10-27||Men must not perform thus real customers. Public, large years get at last possible partners. Even other children could love excellent plants. Only new years will no|2.54|0.11|6013005|exportinameless #6|13|basketball|8|Sports|125|antiableought|N/A|peach078626381933962|thistle|Bundle|Unknown|4|antibarbarese| +4006|AAAAAAAAGKPAAAAA|1997-10-27|1999-10-27|Central, long members must swallow also women. Federal, major benefi|1.93|0.81|6004007|edu packcorp #7|4|bracelets|6|Jewelry|131|oughtpriought|N/A|471757787sandy420510|yellow|Each|Unknown|27|callybarbarese| +4007|AAAAAAAAGKPAAAAA|1999-10-28|2001-10-26|Central, long members must swallow also women. Federal, major benefi|4.72|0.81|6004007|exporticorp #8|3|gold|6|Jewelry|131|oughtpriought|N/A|471757787sandy420510|violet|Pound|Unknown|35|ationbarbarese| +4008|AAAAAAAAGKPAAAAA|2001-10-27||Central, long members must swallow also women. Federal, major benefi|58.00|40.60|6001001|amalgcorp #1|3|birdal|6|Jewelry|437|ationpriese|N/A|53honeydew5962068580|yellow|Each|Unknown|11|eingbarbarese| +4009|AAAAAAAAJKPAAAAA|1997-10-27||Yet good applications can mature here colours. Fo|9.62|6.83|2004002|edu packimporto #2|4|sports-apparel|2|Men|855|antiantieing|small|76061purple673706017|tan|Tbl|Unknown|26|n stbarbarese| +4010|AAAAAAAAKKPAAAAA|1997-10-27|2000-10-26|Eyes remember countries. Sources prepare downwards in a pages. Coloured years catch. Apart relevant fingers would prepare so to a days. Level activities would not review more ser|2.18|1.19|1001001|amalgamalg #1|1|dresses|1|Women|146|callyeseought|large|231773128yellow78530|linen|Lb|Unknown|1|baroughtbarese| +4011|AAAAAAAAKKPAAAAA|2000-10-27||Well new walls would give visual, industrial adults. Tropical, primitive expectations challenge; efforts talk too in a mice. National shoes|76.29|1.19|7014002|edu packnameless #2|14|glassware|7|Home|24|callyeseought|N/A|541snow9221385294046|slate|Pound|Unknown|17|oughtoughtbarese| +4012|AAAAAAAAMKPAAAAA|1997-10-27|1999-10-27|Blue findings used to assess by a relatives. International, important qualities shall stay spanish, active roses; solid villages will stand in order certain members.|96.43|69.42|9007005|brandmaxi #5|7|reference|9|Books|3|pri|N/A|8878745428390salmon5|sky|Pound|Unknown|5|ableoughtbarese| +4013|AAAAAAAAMKPAAAAA|1999-10-28|2001-10-26|Blue findings used to assess by a relatives. International, important qualities shall stay spanish, active roses; solid villages will stand in order certain members.|4.09|2.45|6014006|edu packbrand #6|7|estate|6|Jewelry|31|pri|N/A|8878745428390salmon5|royal|Ton|Unknown|41|prioughtbarese| +4014|AAAAAAAAMKPAAAAA|2001-10-27||Royal clients walk precisely bitter negotiations; then|1.98|1.18|6014006|edu packedu pack #1|4|athletic|4|Shoes|31|oughtpri|small|518395407almond29975|pink|N/A|Unknown|71|eseoughtbarese| +4015|AAAAAAAAPKPAAAAA|1997-10-27||Techniques sink very thinking examples. Still innocent spirits face eventually little products. Video-taped reports exceed far processes. New org|2.42|1.69|9012008|importounivamalg #8|12|home repair|9|Books|244|eseeseable|N/A|97132265547tomato581|coral|N/A|Unknown|14|antioughtbarese| +4016|AAAAAAAAALPAAAAA|1997-10-27|2000-10-26|In part wooden sides ought to diversify sites. White, green forests penetrate cult|5.41|3.94|2002001|importoimporto #1|2|shirts|2|Men|273|priationable|petite|1peach43974286906126|khaki|Unknown|Unknown|62|callyoughtbarese| +4017|AAAAAAAAALPAAAAA|2000-10-27||In part wooden sides ought to diversify sites. White, green forests penetrate cult|1.85|1.62|2002001|namelessunivamalg #6|8|scanners|10|Electronics|87|ationeing|N/A|09416958622038wheat9|slate|Bunch|Unknown|35|ationoughtbarese| +4018|AAAAAAAACLPAAAAA|1997-10-27|1999-10-27|Used, simple circumstances could not harness in a increases. Available tools follow vulnerable, international men. Open arrangements spend especiall|3.00|2.37|3004001|edu packexporti #1|4|school-uniforms|3|Children|284|eseeingable|medium|637393pink0680083777|blue|Dram|Unknown|39|eingoughtbarese| +4019|AAAAAAAACLPAAAAA|1999-10-28|2001-10-26|Used, simple circumstances could not harness in a increases. Available tools follow vulnerable, international men. Open arrangements spend especiall|0.12|2.37|3004001|exportinameless #10|3|basketball|8|Sports|446|callyeseese|N/A|0464slate69079110427|tomato|Each|Unknown|27|n stoughtbarese| +4020|AAAAAAAACLPAAAAA|2001-10-27||St|8.26|7.43|3004001|scholarmaxi #1|3|fishing|8|Sports|397|ationn stpri|N/A|0464slate69079110427|pink|Tbl|Unknown|63|barablebarese| +4021|AAAAAAAAFLPAAAAA|1997-10-27||Children could not see between a revenues. Elderly, annual contracts could not believe particularly as single problems. Democratic, human benefits appoint sometimes. Steep, nasty places |6.25|4.06|7006002|corpbrand #2|6|rugs|7|Home|513|prioughtanti|N/A|67snow70780810729696|white|Gram|Unknown|9|oughtablebarese| +4022|AAAAAAAAGLPAAAAA|1997-10-27|2000-10-26|Direct, small candidates shall not designate even important, attractive stones. Full, crucial women must talk only able, big talks; reservations make t|4.19|2.72|3003001|exportiexporti #1|3|toddlers|3|Children|280|bareingable|small|87032thistle08718378|turquoise|Box|Unknown|42|ableablebarese| +4023|AAAAAAAAGLPAAAAA|2000-10-27||Only faces will live however at a hours. Objectively old arms p|41.82|22.58|3003001|exportibrand #4|3|kids|7|Home|65|bareingable|N/A|9153816101915royal19|light|Dram|Unknown|16|priablebarese| +4024|AAAAAAAAILPAAAAA|1997-10-27|1999-10-27|Subsequently straight deve|8.31|6.64|5004001|edu packscholar #1|4|classical|5|Music|270|barationable|N/A|8894103927317white05|medium|Pallet|Unknown|60|eseablebarese| +4025|AAAAAAAAILPAAAAA|1999-10-28|2001-10-26|Subsequently straight deve|8.64|6.64|4003002|exportiedu pack #2|3|kids|4|Shoes|270|barationable|small|8894103927317white05|white|Bundle|Unknown|15|antiablebarese| +4026|AAAAAAAAILPAAAAA|2001-10-27||Minds shall|3.92|2.03|4003002|brandmaxi #7|7|reference|9|Books|853|priantieing|N/A|115tan26229850593869|lawn|Bundle|Unknown|18|callyablebarese| +4027|AAAAAAAALLPAAAAA|1997-10-27||Rapid, physical lips must think other, exclusive parts. Enough elegant results build. Just right wishes ought to join go|7.79|2.41|9014008|edu packunivamalg #8|14|sports|9|Books|904|esebarn st|N/A|260659305navajo99028|metallic|Cup|Unknown|6|ationablebarese| +4028|AAAAAAAAMLPAAAAA|1997-10-27|2000-10-26|Initial pieces happen profoundly great observers; since temporary police get in a months. Lengthy, likely parts identify disabled values; likely women listen full, local se|53.23|44.18|1002001|importoamalg #1|2|fragrances|1|Women|442|ableeseese|large|4691595782white53594|tomato|Dram|Unknown|1|eingablebarese| +4029|AAAAAAAAMLPAAAAA|2000-10-27||Initial pieces happen profoundly great observers; since temporary police get in a months. Lengthy, likely parts identify disabled values; likely women listen full, local se|1.09|0.42|1002001|edu packmaxi #10|2|entertainments|9|Books|7|ableeseese|N/A|11seashell6475559974|rosy|Bunch|Unknown|97|n stablebarese| +4030|AAAAAAAAOLPAAAAA|1997-10-27|1999-10-27|Past professionals refer openly into the factories. Free, subjective proceedings make for example senior, important conservatives. Sites suspe|4.13|2.93|8001001|amalgnameless #1|1|athletic shoes|8|Sports|302|ablebarpri|N/A|445245657seashell840|goldenrod|Lb|Unknown|16|barpribarese| +4031|AAAAAAAAOLPAAAAA|1999-10-28|2001-10-26|Needs might protect reductions. Political dogs will want to the colleagues. New events|2.55|2.93|8001001|scholarbrand #2|1|blinds/shades|7|Home|363|pricallypri|N/A|338506797royal918331|orchid|Oz|Unknown|19|oughtpribarese| +4032|AAAAAAAAOLPAAAAA|2001-10-27||Needs might protect reductions. Political dogs will want to the colleagues. New events|45.05|2.93|8001001|amalgcorp #1|1|birdal|6|Jewelry|363|pricallypri|N/A|338506797royal918331|slate|Carton|Unknown|50|ablepribarese| +4033|AAAAAAAABMPAAAAA|1997-10-27||General weeks like great, old families. Agricultural lips go with a influences. Regional, organic shoulders afford really m|1.76|0.54|6004002|edu packcorp #2|4|bracelets|6|Jewelry|43|priese|N/A|3586549burlywood5133|plum|Each|Unknown|42|pripribarese| +4034|AAAAAAAACMPAAAAA|1997-10-27|2000-10-26|New jews would not accept normally at the authorities. Forward integrated processes should find today. Ago possible americans shield |6.25|2.87|8010005|univmaxi #5|10|pools|8|Sports|123|priableought|N/A|78489365814509steel6|slate|Bundle|Unknown|52|esepribarese| +4035|AAAAAAAACMPAAAAA|2000-10-27||New jews would not accept normally at the authorities. Forward integrated processes should find today. Ago possible americans shield |5.62|2.87|2001002|amalgimporto #2|10|accessories|2|Men|460|priableought|petite|78489365814509steel6|maroon|Bunch|Unknown|3|antipribarese| +4036|AAAAAAAAEMPAAAAA|1997-10-27|1999-10-27|Cold areas spend. Random, other children recognize now a|2.25|1.80|3002001|importoexporti #1|2|infants|3|Children|319|n stoughtpri|large|097167320560snow3628|gainsboro|Case|Unknown|64|callypribarese| +4037|AAAAAAAAEMPAAAAA|1999-10-28|2001-10-26|Cold areas spend. Random, other children recognize now a|3.12|2.65|3002001|exportiamalg #2|2|maternity|1|Women|300|barbarpri|petite|097167320560snow3628|wheat|Carton|Unknown|20|ationpribarese| +4038|AAAAAAAAEMPAAAAA|2001-10-27||Cold areas spend. Random, other children recognize now a|3.34|2.07|3002001|corpunivamalg #7|2|mystery|9|Books|247|ationeseable|N/A|4943051391886yellow9|sandy|Tbl|Unknown|83|eingpribarese| +4039|AAAAAAAAHMPAAAAA|1997-10-27||Ready, technical activities attempt all. However certain artists admit. Mere, local teachers will return and so on beside a exhibitions. Fr|1.05|0.50|8005010|scholarnameless #10|5|fitness|8|Sports|532|ableprianti|N/A|2980pink241989268636|rosy|Tbl|Unknown|10|n stpribarese| +4040|AAAAAAAAIMPAAAAA|1997-10-27|2000-10-26|Only keys impose also thanks. New, new cells ought to form most existing spirits. Complete days |5.34|3.20|6011001|amalgbrand #1|11|semi-precious|6|Jewelry|19|n stought|N/A|1215063sandy92704131|purple|Unknown|Unknown|4|baresebarese| +4041|AAAAAAAAIMPAAAAA|2000-10-27||Structures count further then american children. Even fine |83.98|51.22|6011001|edu packedu pack #2|4|athletic|4|Shoes|19|n stought|small|1215063sandy92704131|papaya|Ounce|Unknown|60|oughtesebarese| +4042|AAAAAAAAKMPAAAAA|1997-10-27|1999-10-27|However direct directions u|0.54|0.17|3004001|edu packexporti #1|4|school-uniforms|3|Children|662|ablecallycally|medium|5puff621213606099433|purple|Unknown|Unknown|15|ableesebarese| +4043|AAAAAAAAKMPAAAAA|1999-10-28|2001-10-26|Central, magnetic sanctions may try both other voices. A little concrete shar|2.96|2.13|3004001|exportischolar #2|3|pop|5|Music|662|ablecallycally|N/A|5puff621213606099433|violet|Pallet|Unknown|30|priesebarese| +4044|AAAAAAAAKMPAAAAA|2001-10-27||Others m|3.28|1.67|4004001|edu packedu pack #1|3|athletic|4|Shoes|662|ablecallycally|extra large|linen091023006546049|slate|Unknown|Unknown|5|eseesebarese| +4045|AAAAAAAANMPAAAAA|1997-10-27||Comprehensive pages p|2.68|2.35|2004002|edu packimporto #2|4|sports-apparel|2|Men|208|eingbarable|medium|44chiffon15832072032|spring|Unknown|Unknown|10|antiesebarese| +4046|AAAAAAAAOMPAAAAA|1997-10-27|2000-10-26|Considerable tears reverse consciously points. Grey pounds should not go little even different price|1.11|0.92|6007007|brandcorp #7|7|pendants|6|Jewelry|74|eseation|N/A|22651162664peru78980|salmon|Ounce|Unknown|5|callyesebarese| +4047|AAAAAAAAOMPAAAAA|2000-10-27||Considerable tears reverse consciously points. Grey pounds should not go little even different price|1.80|0.92|5002002|importoscholar #2|7|country|5|Music|50|eseation|N/A|22651162664peru78980|drab|Oz|Unknown|50|ationesebarese| +4048|AAAAAAAAANPAAAAA|1997-10-27|1999-10-27|Sentences support. Sufficient, average officials know fully asian, existing styles. Hopes wi|6.14|2.21|6010001|univbrand #1|10|jewelry boxes|6|Jewelry|22|ableable|N/A|12639543475salmon987|dark|Bunch|Unknown|50|eingesebarese| +4049|AAAAAAAAANPAAAAA|1999-10-28|2001-10-26|Sentences support. Sufficient, average officials know fully asian, existing styles. Hopes wi|4.11|1.68|6010001|brandmaxi #6|10|reference|9|Books|218|eingoughtable|N/A|12639543475salmon987|drab|Oz|Unknown|48|n stesebarese| +4050|AAAAAAAAANPAAAAA|2001-10-27||Sentences support. Sufficient, average officials know fully asian, existing styles. Hopes wi|8.21|7.22|4003001|exportiedu pack #1|10|kids|4|Shoes|585|antieinganti|petite|steel523473218091424|navajo|Unknown|Unknown|28|barantibarese| +4051|AAAAAAAADNPAAAAA|1997-10-27||Professional cars start alternative days. Services shall give rather quicker high developments. For instance coming holes would not consi|0.65|0.52|5003002|exportischolar #2|3|pop|5|Music|225|antiableable|N/A|8390292honeydew56182|pale|Gross|Unknown|5|oughtantibarese| +4052|AAAAAAAAENPAAAAA|1997-10-27|2000-10-26|Eventual, little patients make demonstrations. Please left books can escape greek hands. Years shall not lift also loudly developing friends. Poor projects hear mos|4.83|1.69|8013003|exportimaxi #3|13|sailing|8|Sports|520|barableanti|N/A|80sandy4953516711028|tan|Bunch|Unknown|11|ableantibarese| +4053|AAAAAAAAENPAAAAA|2000-10-27||Banks tackle home communities. Both particular tests used to travel. Real techniques ought to involve to|1.68|0.87|8013003|exportinameless #4|13|wallpaper|7|Home|520|barableanti|N/A|smoke537256771192483|white|Bundle|Unknown|18|priantibarese| +4054|AAAAAAAAGNPAAAAA|1997-10-27|1999-10-27|Yet whole consequences could not point just to every protests. Pure, running agents prevent clearly to the businesses. New demands would ta|0.70|0.25|6015007|scholarbrand #7|15|custom|6|Jewelry|56|callyanti|N/A|576038884thistle5609|olive|Gross|Unknown|93|eseantibarese| +4055|AAAAAAAAGNPAAAAA|1999-10-28|2001-10-26|Yet whole consequences could not point just to every protests. Pure, running agents prevent clearly to the businesses. New demands would ta|8.02|0.25|5003002|exportischolar #2|3|pop|5|Music|152|callyanti|N/A|576038884thistle5609|white|Carton|Unknown|21|antiantibarese| +4056|AAAAAAAAGNPAAAAA|2001-10-27||Yet whole consequences could not point just to every protests. Pure, running agents prevent clearly to the businesses. New demands would ta|1.81|1.35|5003002|exportiamalg #1|3|maternity|1|Women|152|callyanti|small|576038884thistle5609|salmon|Carton|Unknown|28|callyantibarese| +4057|AAAAAAAAJNPAAAAA|1997-10-27||Golden years ought to influence gen|0.81|0.34|5004002|edu packscholar #2|4|classical|5|Music|283|prieingable|N/A|16601558231192puff97|navy|Pallet|Unknown|2|ationantibarese| +4058|AAAAAAAAKNPAAAAA|1997-10-27|2000-10-26|Other, short-term arrangements shall pay other, known times. Only coming men may answer for the strategies. Good thanks will not want actually general industries. Brothers should not win further.|9.53|4.19|4002001|importoedu pack #1|2|mens|4|Shoes|4|ese|large|57724251762824slate2|seashell|Gram|Unknown|66|eingantibarese| +4059|AAAAAAAAKNPAAAAA|2000-10-27||Other, short-term arrangements shall pay other, known times. Only coming men may answer for the strategies. Good thanks will not want actually general industries. Brothers should not win further.|0.25|0.14|3001002|amalgexporti #2|2|newborn|3|Children|660|barcallycally|extra large|0171980orchid0397517|spring|Oz|Unknown|28|n stantibarese| +4060|AAAAAAAAMNPAAAAA|1997-10-27|1999-10-27|Managers could not bring; considerably right engineers vary secondary contracts. Hours ask white books. Real clothes redu|8.72|5.14|6009005|maxicorp #5|9|womens watch|6|Jewelry|55|antianti|N/A|7813917566896469tan6|linen|Bundle|Unknown|19|barcallybarese| +4061|AAAAAAAAMNPAAAAA|1999-10-28|2001-10-26|Desirable efforts can get to the years. As potential allegations will not claim also. Expenses make; th|8.81|4.40|3004002|edu packexporti #2|4|school-uniforms|3|Children|497|antianti|small|46149073788white5074|powder|Dram|Unknown|11|oughtcallybarese| +4062|AAAAAAAAMNPAAAAA|2001-10-27||Other, fellow residents form on a types. Wooden, nuclear numbers will believe due to a negotiations. Deliberately full services can criticise etc so important courts; both supreme revenues fulfil ne|5.63|4.40|3004002|importoedu pack #1|2|mens|4|Shoes|497|ationn stese|economy|46149073788white5074|red|N/A|Unknown|67|ablecallybarese| +4063|AAAAAAAAPNPAAAAA|1997-10-27||Significant schools ought to aris|3.07|0.92|3002002|importoexporti #2|2|infants|3|Children|216|callyoughtable|large|11210302214072navy48|smoke|Cup|Unknown|73|pricallybarese| +4064|AAAAAAAAAOPAAAAA|1997-10-27|2000-10-26|Awkward, natural loans increase with a nurses. Yesterday old clergy must conceal sound champions. Others act otherwise only unknown st|4.11|3.12|6004001|edu packcorp #1|4|bracelets|6|Jewelry|188|eingeingought|N/A|14386290713purple468|turquoise|Ton|Unknown|29|esecallybarese| +4065|AAAAAAAAAOPAAAAA|2000-10-27||Awkward, natural loans increase with a nurses. Yesterday old clergy must conceal sound champions. Others act otherwise only unknown st|4.26|3.12|6004001|importoamalg #2|2|fragrances|1|Women|216|callyoughtable|small|14386290713purple468|rose|Carton|Unknown|8|anticallybarese| +4066|AAAAAAAACOPAAAAA|1997-10-27|1999-10-27|Faces try. Really appropriate hours investigate useful, blue organisations. Round others can greet|0.84|0.70|2002001|importoimporto #1|2|shirts|2|Men|132|ablepriought|large|21904733562almond400|tomato|N/A|Unknown|2|callycallybarese| +4067|AAAAAAAACOPAAAAA|1999-10-28|2001-10-26|Evil |0.86|0.43|5004002|edu packscholar #2|4|classical|5|Music|921|ablepriought|N/A|21904733562almond400|cornflower|Cup|Unknown|9|ationcallybarese| +4068|AAAAAAAACOPAAAAA|2001-10-27||Equations get subsequently total skills. Now due features know domestic families. So subjecti|2.87|0.94|9001001|amalgmaxi #1|4|arts|9|Books|921|oughtablen st|N/A|22893921lavender9082|indian|Unknown|Unknown|12|eingcallybarese| +4069|AAAAAAAAFOPAAAAA|1997-10-27||Moments use above local studies. More ordinary columns point now considerable services. At|1.46|1.00|9009002|maximaxi #2|9|science|9|Books|261|oughtcallyable|N/A|4944943spring9995354|saddle|N/A|Unknown|96|n stcallybarese| +4070|AAAAAAAAGOPAAAAA|1997-10-27|2000-10-26|Particular, arab cases shall like less current, different names. Computers start for the changes. Scottish, trying exercises operate marks; long, supreme miners may ro|31.25|28.12|4004001|edu packedu pack #1|4|athletic|4|Shoes|107|ationbarought|extra large|7013968171cream55620|smoke|Dram|Unknown|52|barationbarese| +4071|AAAAAAAAGOPAAAAA|2000-10-27||Particular, arab cases shall like less current, different names. Computers start for the changes. Scottish, trying exercises operate marks; long, supreme miners may ro|2.11|28.12|1004002|edu packamalg #2|4|swimwear|1|Women|470|ationbarought|medium|7013968171cream55620|linen|Pallet|Unknown|56|oughtationbarese| +4072|AAAAAAAAIOPAAAAA|1997-10-27|1999-10-27|Steps might not take. Reduced tactics could cut incidentally to a circumstances. Conservati|2.84|2.07|4004001|edu packedu pack #1|4|athletic|4|Shoes|764|esecallyation|extra large|288660593415spring03|papaya|Dozen|Unknown|2|ableationbarese| +4073|AAAAAAAAIOPAAAAA|1999-10-28|2001-10-26|Features should direct. Me|0.16|2.07|4004001|exportiedu pack #2|3|kids|4|Shoes|240|bareseable|extra large|4308028576tan7358889|lawn|Dozen|Unknown|8|priationbarese| +4074|AAAAAAAAIOPAAAAA|2001-10-27||Simply good measurements must appear hard discussions. Large|4.67|2.07|5001001|amalgscholar #1|3|rock|5|Music|161|oughtcallyought|N/A|4308028576tan7358889|rosy|N/A|Unknown|39|eseationbarese| +4075|AAAAAAAALOPAAAAA|1997-10-27||Then physical he|57.20|22.88|1001002|amalgamalg #2|1|dresses|1|Women|252|ableantiable|large|364289344cornflower0|seashell|Ounce|Unknown|1|antiationbarese| +4076|AAAAAAAAMOPAAAAA|1997-10-27|2000-10-26|Geographical, obvious conditions leave rather successful, new feelings. Here present friends would stop. New, positive terms shou|5.69|3.24|7007007|brandbrand #7|7|decor|7|Home|555|antiantianti|N/A|479brown616287354021|turquoise|Tbl|Unknown|27|callyationbarese| +4077|AAAAAAAAMOPAAAAA|2000-10-27||Keen players should go houses; military populations may not find especially on a americans. Vulnerable cars shall ask policies; requirements provide occasionally blue fri|10.38|3.24|1004002|edu packamalg #2|7|swimwear|1|Women|555|antiantianti|small|78399tan690672391192|wheat|Pallet|Unknown|26|ationationbarese| +4078|AAAAAAAAOOPAAAAA|1997-10-27|1999-10-27|International, british events break other rates. Good, spanish elements cut immediately annual lines. Members would a|0.66|0.43|1002001|importoamalg #1|2|fragrances|1|Women|186|callyeingought|medium|4364rosy728433957966|pink|Bunch|Unknown|2|eingationbarese| +4079|AAAAAAAAOOPAAAAA|1999-10-28|2001-10-26|International, british events break other rates. Good, spanish elements cut immediately annual lines. Members would a|0.97|0.55|1002001|importoamalg #2|2|fragrances|1|Women|208|callyeingought|small|2936sienna8833460975|lemon|Carton|Unknown|24|n stationbarese| +4080|AAAAAAAAOOPAAAAA|2001-10-27||Extremely american artists shall improve under the streets. Right, new systems beat far waters. Particularly large strangers migh|8.15|5.86|1002001|amalgcorp #1|1|birdal|6|Jewelry|208|eingbarable|N/A|2936sienna8833460975|honeydew|Pallet|Unknown|54|bareingbarese| +4081|AAAAAAAABPPAAAAA|1997-10-27||National, major years mean forever extreme, internal details. Special names blame so advanced tons; respons|8.86|3.89|10003002|exportiunivamalg #2|3|dvd/vcr players|10|Electronics|452|ableantiese|N/A|89028890purple719894|thistle|Gross|Unknown|26|oughteingbarese| +4082|AAAAAAAACPPAAAAA|1997-10-27|2000-10-26|Clear, strong patients happen far|6.82|4.91|1001001|amalgamalg #1|1|dresses|1|Women|669|n stcallycally|large|98832990666411plum58|plum|Unknown|Unknown|29|ableeingbarese| +4083|AAAAAAAACPPAAAAA|2000-10-27||Religious, local clients enjoy. Expensive, level accounts drink other results. Together other demands become real japanese toys. Large, cruel speakers attract recently a|9.45|4.91|1002002|importoamalg #2|1|fragrances|1|Women|669|n stcallycally|small|violet09144337344031|light|Ton|Unknown|57|prieingbarese| +4084|AAAAAAAAEPPAAAAA|1997-10-27|1999-10-27|English|0.28|0.23|2003001|exportiimporto #1|3|pants|2|Men|869|n stcallyeing|N/A|8rose780677236830792|white|Each|Unknown|62|eseeingbarese| +4085|AAAAAAAAEPPAAAAA|1999-10-28|2001-10-26|English|3.44|2.75|2003001|edu packamalg #2|3|swimwear|1|Women|869|n stcallyeing|medium|569365568ghost350058|slate|N/A|Unknown|82|antieingbarese| +4086|AAAAAAAAEPPAAAAA|2001-10-27||Especially sweet hours shall ask even. Never high prices may take beans. Points would join rather prime, old answers. Now real relationships used to spread more comparable inc times. Neve|0.23|0.15|2003001|amalgedu pack #1|3|womens|4|Shoes|869|n stcallyeing|extra large|782483257hot00407122|white|Each|Unknown|46|callyeingbarese| +4087|AAAAAAAAHPPAAAAA|1997-10-27||Crucial, other eyes must bow|2.16|1.25|5004002|edu packscholar #2|4|classical|5|Music|326|callyablepri|N/A|844064snow1446153910|royal|Gross|Unknown|33|ationeingbarese| +4088|AAAAAAAAIPPAAAAA|1997-10-27|2000-10-26|Companies change fixed, toxic men. Levels would mean much military studies; then available terms used to believe all dutch facilities. Rural gra|1.55|0.96|1002001|importoamalg #1|2|fragrances|1|Women|580|bareinganti|large|983652tan78852361323|royal|Unknown|Unknown|14|eingeingbarese| +4089|AAAAAAAAIPPAAAAA|2000-10-27||Patients win. About new weeks wait quite dishes. Scientific, cultural players take never new, present structures. British tories used to see homes. C|1.16|0.45|8002010|importonameless #10|2|baseball|8|Sports|60|barcally|N/A|06medium117065099455|thistle|Each|Unknown|4|n steingbarese| +4090|AAAAAAAAKPPAAAAA|1997-10-27|1999-10-27|Young hands report. Children would bre|4.09|2.08|8016001|corpmaxi #1|16|golf|8|Sports|205|antibarable|N/A|76134469steel8791095|olive|N/A|Unknown|46|barn stbarese| +4091|AAAAAAAAKPPAAAAA|1999-10-28|2001-10-26|Young, dead towns derive other, possible speeches. Altogether empty facts will begin from the facts. New, new relationships date there. Interesting, proper dates make on a effects. Never criminal year|4.39|2.63|4004002|edu packedu pack #2|16|athletic|4|Shoes|205|antibarable|extra large|76134469steel8791095|honeydew|Tbl|Unknown|83|oughtn stbarese| +4092|AAAAAAAAKPPAAAAA|2001-10-27||Unpleasant contracts offer there acids. Boxes would know able, inherent hours. Operatio|7.80|4.44|10011013|amalgamalgamalg #13|16|disk drives|10|Electronics|205|antibarable|N/A|60090830677789steel6|smoke|Each|Unknown|8|ablen stbarese| +4093|AAAAAAAANPPAAAAA|1997-10-27||Tears|2.49|2.21|7011004|amalgnameless #4|11|accent|7|Home|499|n stn stese|N/A|7218lime218446093470|yellow|Unknown|Unknown|42|prin stbarese| +4094|AAAAAAAAOPPAAAAA|1997-10-27|2000-10-26|Particular samples may not stop with a beliefs. Still di|0.80|0.72|10008015|namelessunivamalg #15|8|scanners|10|Electronics|18|eingought|N/A|238324733021206plum3|pink|Bunch|Unknown|13|esen stbarese| +4095|AAAAAAAAOPPAAAAA|2000-10-27||Weeks volunteer buildings. Beautifully medical banks would tell old legs. Norma|85.39|0.72|10008015|corpamalgamalg #16|8|wireless|10|Electronics|18|eingought|N/A|521953981347purple92|white|Cup|Unknown|76|antin stbarese| +4096|AAAAAAAAAAABAAAA|1997-10-27|1999-10-27|Find|6.79|2.91|6005001|scholarcorp #1|5|earings|6|Jewelry|397|ationn stpri|N/A|06489840714pale89271|white|Ounce|Unknown|28|callyn stbarese| +4097|AAAAAAAAAAABAAAA|1999-10-28|2001-10-26|Find|7.63|3.43|1001002|amalgamalg #2|1|dresses|1|Women|30|ationn stpri|small|07sienna976194257987|salmon|Dozen|Unknown|63|ationn stbarese| +4098|AAAAAAAAAAABAAAA|2001-10-27||Details show accordingly up to a adults. Schools could want drugs. Events go vigorously new decisions. Successful hours shall tell at the estimates. Women wish good teachers. Welsh, critica|1.47|1.14|1001002|importounivamalg #2|2|camcorders|10|Electronics|30|ationn stpri|N/A|snow1783475795097902|seashell|Oz|Unknown|72|eingn stbarese| +4099|AAAAAAAADAABAAAA||||||1004002||4||1|Women|273||medium||puff|Ton|Unknown|70|| +4100|AAAAAAAAEAABAAAA|1997-10-27|2000-10-26|Customs conform nearly hot bones; british, low types would impose completely in the agreem|1.74|0.88|8011001|amalgmaxi #1|11|archery|8|Sports|380|bareingpri|N/A|63166145070navy67848|turquoise|Dozen|Unknown|8|barbaroughtese| +4101|AAAAAAAAEAABAAAA|2000-10-27||Customs conform nearly hot bones; british, low types would impose completely in the agreem|3.07|0.88|8011001|importocorp #6|2|diamonds|6|Jewelry|51|bareingpri|N/A|63166145070navy67848|saddle|Lb|Unknown|7|oughtbaroughtese| +4102|AAAAAAAAGAABAAAA|1997-10-27|1999-10-27|Most great losses convert together in a terms.|9.19|7.81|10006006|corpunivamalg #6|6|musical|10|Electronics|161|oughtcallyought|N/A|74552211631953plum93|ivory|Case|Unknown|48|ablebaroughtese| +4103|AAAAAAAAGAABAAAA|1999-10-28|2001-10-26|Most great losses convert together in a terms.|2.07|1.65|10006006|importomaxi #12|6|business|9|Books|161|oughtcallyought|N/A|74552211631953plum93|plum|Tsp|Unknown|20|pribaroughtese| +4104|AAAAAAAAGAABAAAA|2001-10-27||Most great losses convert together in a terms.|1.92|1.65|7002005|importobrand #5|2|bedding|7|Home|161|oughtcallyought|N/A|12rose04857386298601|seashell|Dram|Unknown|54|esebaroughtese| +4105|AAAAAAAAJAABAAAA|1997-10-27||Basic, good students try secret consequences. Poor pilots could check a little national cases. Piec|1.66|1.42|2004002|edu packimporto #2|4|sports-apparel|2|Men|191|oughtn stought|petite|833394167pink1935896|lime|Box|Unknown|55|antibaroughtese| +4106|AAAAAAAAKAABAAAA|1997-10-27|2000-10-26|Men send substantially high republics. Tiny beings serve yesterday|3.04|2.03|6013003|exportibrand #3|13|loose stones|6|Jewelry|16|callyought|N/A|632misty415809273259|maroon|Gram|Unknown|8|callybaroughtese| +4107|AAAAAAAAKAABAAAA|2000-10-27||Men send substantially high republics. Tiny beings serve yesterday|3.75|2.28|2001002|amalgimporto #2|13|accessories|2|Men|16|callyought|medium|95white4171841471742|red|Gross|Unknown|20|ationbaroughtese| +4108|AAAAAAAAMAABAAAA|1997-10-27|1999-10-27|Increased times might know neither campaigns; probably written foods used to reduce then circles. Easte|0.37|0.20|5002001|importoscholar #1|2|country|5|Music|568|eingcallyanti|N/A|3064357350burlywood5|purple|Dram|Unknown|22|eingbaroughtese| +4109|AAAAAAAAMAABAAAA|1999-10-28|2001-10-26|Increased times might know neither campaigns; probably written foods used to reduce then circles. Easte|7.93|0.20|5001002|amalgscholar #2|2|rock|5|Music|568|eingcallyanti|N/A|3064357350burlywood5|maroon|Lb|Unknown|25|n stbaroughtese| +4110|AAAAAAAAMAABAAAA|2001-10-27||Pure, daily miles leave also new questions. Social results feel nearly due to a sons. Frequent, desi|3.43|0.20|7007001|brandbrand #1|7|decor|7|Home|58|einganti|N/A|72322812361spring418|turquoise|Ton|Unknown|66|baroughtoughtese| +4111|AAAAAAAAPAABAAAA|1997-10-27||Urban, substantial problems salvage actually degrees. Firmly different women ought to allow impossible, painful cases. Neatly giant forms know even english kids. Severe christians shall undergo pre|3.89|1.63|6012008|importobrand #8|12|costume|6|Jewelry|80|bareing|N/A|2428light20923914110|rosy|Oz|Unknown|25|oughtoughtoughtese| +4112|AAAAAAAAABABAAAA|1997-10-27|2000-10-26|Racial, sufficient difficulties support fully shops. Especially simple players must not measure famous, good affairs. European rumours begin again basic, national we|4.39|1.31|2004001|edu packimporto #1|4|sports-apparel|2|Men|136|callypriought|medium|9393655sandy75133031|rose|Dozen|Unknown|17|ableoughtoughtese| +4113|AAAAAAAAABABAAAA|2000-10-27||Black rounds|0.13|1.31|9016010|corpunivamalg #10|16|mystery|9|Books|898|eingn steing|N/A|orchid92535283194941|coral|Ounce|Unknown|25|prioughtoughtese| +4114|AAAAAAAACBABAAAA|1997-10-27|1999-10-27|Total, hard relationships can get there partly definite circumstances. Children must supp|7.41|3.55|10012001|importoamalgamalg #1|12|monitors|10|Electronics|46|callyese|N/A|00588802055807peach8|pale|Tbl|Unknown|6|eseoughtoughtese| +4115|AAAAAAAACBABAAAA|1999-10-28|2001-10-26|Total, hard relationships can get there partly definite circumstances. Children must supp|2.88|3.55|2002002|importoimporto #2|12|shirts|2|Men|216|callyese|small|00588802055807peach8|pink|Pallet|Unknown|48|antioughtoughtese| +4116|AAAAAAAACBABAAAA|2001-10-27||Goods worry above in a businesses. Items rea|2.28|3.55|2002002|amalgexporti #1|1|newborn|3|Children|216|callyoughtable|petite|00588802055807peach8|lace|Dozen|Unknown|28|callyoughtoughtese| +4117|AAAAAAAAFBABAAAA|1997-10-27||Experimental users know even extremely small aspects. Regular |2.85|1.88|8016008|corpmaxi #8|16|golf|8|Sports|113|prioughtought|N/A|881002puff3124167341|sienna|Each|Unknown|25|ationoughtoughtese| +4118|AAAAAAAAGBABAAAA|1997-10-27|2000-10-26|Members should arrest nuclear levels. Aware, early products must get letters. Good, very aspects will deco|7.88|6.06|3001001|amalgexporti #1|1|newborn|3|Children|977|ationationn st|medium|3602642745slate03348|snow|Tsp|Unknown|65|eingoughtoughtese| +4119|AAAAAAAAGBABAAAA|2000-10-27||Perhaps civil marks will not listen only between the affairs. Years reduce primarily at a women. Western, low packages might offer for a departments. Still poor compan|5.73|4.69|3001001|importoscholar #2|1|country|5|Music|977|ationationn st|N/A|salmon36416910989181|white|Dozen|Unknown|30|n stoughtoughtese| +4120|AAAAAAAAIBABAAAA|1997-10-27|1999-10-27|Levels assist. Other, growing points may deal clear numbers. Waiting, wide goods o|6.64|5.97|5002001|importoscholar #1|2|country|5|Music|354|eseantipri|N/A|408935978royal781892|peach|Gram|Unknown|50|barableoughtese| +4121|AAAAAAAAIBABAAAA|1999-10-28|2001-10-26|Levels assist. Other, growing points may deal clear numbers. Waiting, wide goods o|2.41|0.77|5002001|exportiexporti #2|2|toddlers|3|Children|354|eseantipri|extra large|408935978royal781892|sienna|Box|Unknown|14|oughtableoughtese| +4122|AAAAAAAAIBABAAAA|2001-10-27||Neither coloured lives go in particular new things. Unable, usual men provide shops; services wait in vain. Functions should understand similarly. Workers think about comprehensi|3.96|0.77|10003009|exportiunivamalg #9|2|dvd/vcr players|10|Electronics|354|eseantipri|N/A|408935978royal781892|puff|Box|Unknown|67|ableableoughtese| +4123|AAAAAAAALBABAAAA|1997-10-27||Social, grand services appear already sounds. Later national positions ought to grow available hours. Offenders ca|8.02|3.92|7003004|exportibrand #4|3|kids|7|Home|143|prieseought|N/A|544821thistle2334576|tan|Box|Unknown|53|priableoughtese| +4124|AAAAAAAAMBABAAAA|1997-10-27|2000-10-26|Observations used to lead th|5.75|1.78|6009005|maxicorp #5|9|womens watch|6|Jewelry|431|oughtpriese|N/A|6905215lawn761080192|pale|Ounce|Unknown|16|eseableoughtese| +4125|AAAAAAAAMBABAAAA|2000-10-27||Observations used to lead th|2.53|1.78|6009005|importoamalg #2|9|fragrances|1|Women|59|n stanti|small|6905215lawn761080192|tan|Bunch|Unknown|10|antiableoughtese| +4126|AAAAAAAAOBABAAAA|1997-10-27|1999-10-27|Surely spanish systems issue. Live, working nurses get hard negative eyes. Then top comments help so leaders. Forever milita|8.84|3.97|5003001|exportischolar #1|3|pop|5|Music|52|ableanti|N/A|184violet03309909822|peru|Gram|Unknown|17|callyableoughtese| +4127|AAAAAAAAOBABAAAA|1999-10-28|2001-10-26|Negative, general goods might not go. A|7.76|3.97|4002002|importoedu pack #2|3|mens|4|Shoes|171|ableanti|petite|89767350green9711258|smoke|Tbl|Unknown|65|ationableoughtese| +4128|AAAAAAAAOBABAAAA|2001-10-27||Strong, common readers meet above future, cultural arrangements. Also appropriate results extend certainly excellent decisions. Legal notes say never in a lines. Members stay over follo|9.20|3.97|4002002|amalgamalg #1|1|dresses|1|Women|519|ableanti|large|89767350green9711258|snow|Gross|Unknown|9|eingableoughtese| +4129|AAAAAAAABCABAAAA|1997-10-27||Minutes achieve however for a allies. Areas pay apparently alive officers|3.28|1.34|4001002|amalgedu pack #2|1|womens|4|Shoes|620|barablecally|extra large|64123378snow86567640|indian|Box|Unknown|32|n stableoughtese| +4130|AAAAAAAACCABAAAA|1997-10-27|2000-10-26|Alone rises mus|1.09|0.98|8014001|edu packmaxi #1|14|tennis|8|Sports|96|callyn st|N/A|779802476767orchid62|rose|Gram|Unknown|67|barprioughtese| +4131|AAAAAAAACCABAAAA|2000-10-27||Extra opponents say rats. Years may use under way|0.20|0.06|2004002|edu packimporto #2|4|sports-apparel|2|Men|96|callyn st|petite|salmon02843348005695|lace|Gram|Unknown|8|oughtprioughtese| +4132|AAAAAAAAECABAAAA|1997-10-27|1999-10-27|Healthy officials mind thin|7.68|4.22|6001005|amalgcorp #5|1|birdal|6|Jewelry|138|eingpriought|N/A|5964rosy759350759480|cornsilk|Case|Unknown|31|ableprioughtese| +4133|AAAAAAAAECABAAAA|1999-10-28|2001-10-26|Healthy officials mind thin|1.23|0.60|6001005|exportiamalg #2|3|maternity|1|Women|400|eingpriought|economy|5964rosy759350759480|pale|Ton|Unknown|14|priprioughtese| +4134|AAAAAAAAECABAAAA|2001-10-27||Healthy officials mind thin|0.46|0.21|1003001|exportiamalg #1|3|maternity|1|Women|400|barbarese|small|41550361781slate2276|rosy|Each|Unknown|29|eseprioughtese| +4135|AAAAAAAAHCABAAAA|1997-10-27||English, good democrats split for a states. Acceptable, likely eyes used to find anyway features. At least full taxes surprise for the sales.|0.97|0.53|4003002|exportiedu pack #2|3|kids|4|Shoes|340|baresepri|medium|447069179mint4696846|smoke|Ton|Unknown|7|antiprioughtese| +4136|AAAAAAAAICABAAAA|1997-10-27|2000-10-26|International, increased journalists transfer probably outside a resources. Together strong procedur|5.96|4.94|5001001|amalgscholar #1|1|rock|5|Music|147|ationeseought|N/A|7131pale121272996154|thistle|Cup|Unknown|44|callyprioughtese| +4137|AAAAAAAAICABAAAA|2000-10-27||Also constant programmes used to take only; white governments gain full years; afraid, special contacts could not read suddenly. Then victorian officials must not underline more in the is|6.08|4.94|3003002|exportiexporti #2|3|toddlers|3|Children|147|ationeseought|large|818metallic180333223|spring|Bunch|Unknown|15|ationprioughtese| +4138|AAAAAAAAKCABAAAA|1997-10-27|1999-10-27|Then likely minutes try organisms. Welsh modules continue grimly different stan|4.74|2.41|3002001|importoexporti #1|2|infants|3|Children|973|priationn st|medium|208082honeydew843268|turquoise|Tbl|Unknown|100|eingprioughtese| +4139|AAAAAAAAKCABAAAA|1999-10-28|2001-10-26|Tear|8.25|2.41|3002001|amalgimporto #2|2|accessories|2|Men|543|prieseanti|large|466808840orchid85642|smoke|Pallet|Unknown|46|n stprioughtese| +4140|AAAAAAAAKCABAAAA|2001-10-27||Tear|7.21|4.47|10013010|exportiamalgamalg #10|2|stereo|10|Electronics|543|prieseanti|N/A|451violet33820797203|slate|Carton|Unknown|6|bareseoughtese| +4141|AAAAAAAANCABAAAA|1997-10-27||Indeed british rewards could teach to a eye|3.74|2.87|1001002|amalgamalg #2|1|dresses|1|Women|70|baration|extra large|91354423679539peru97|pale|Case|Unknown|30|oughteseoughtese| +4142|AAAAAAAAOCABAAAA|1997-10-27|2000-10-26|Systems look. Much new times may not derive only. Running laws perform only important, normal posts. Surely working-class researchers lead well by the thanks. Chief homes work loc|0.93|0.76|3002001|importoexporti #1|2|infants|3|Children|322|ableablepri|medium|6powder7709172025669|wheat|Gram|Unknown|17|ableeseoughtese| +4143|AAAAAAAAOCABAAAA|2000-10-27||Systems look. Much new times may not derive only. Running laws perform only important, normal posts. Surely working-class researchers lead well by the thanks. Chief homes work loc|2.82|0.76|9004004|edu packmaxi #4|2|entertainments|9|Books|322|ableablepri|N/A|6powder7709172025669|peru|N/A|Unknown|18|prieseoughtese| +4144|AAAAAAAAADABAAAA|1997-10-27|1999-10-27|New differences strike familiar, ridiculous birds. Badly helpful ministers mean educational organizations. |0.55|0.29|3003001|exportiexporti #1|3|toddlers|3|Children|709|n stbaration|extra large|64237powder994850815|papaya|Ton|Unknown|38|eseeseoughtese| +4145|AAAAAAAAADABAAAA|1999-10-28|2001-10-26|New differences strike familiar, ridiculous birds. Badly helpful ministers mean educational organizations. |8.21|0.29|10015015|scholaramalgamalg #15|15|portable|10|Electronics|480|n stbaration|N/A|64237powder994850815|papaya|Unknown|Unknown|33|antieseoughtese| +4146|AAAAAAAAADABAAAA|2001-10-27||New differences strike familiar, ridiculous birds. Badly helpful ministers mean educational organizations. |3.65|0.29|6007003|brandcorp #3|7|pendants|6|Jewelry|480|n stbaration|N/A|64237powder994850815|tan|Bunch|Unknown|34|callyeseoughtese| +4147|AAAAAAAADDABAAAA|1997-10-27||English, real women can use. Italian tools get for the things; criminal, permanent women cannot make. New estates can go. Now great men might break even home squar|1.72|1.23|6013004|exportibrand #4|13|loose stones|6|Jewelry|173|priationought|N/A|8115786403848yellow5|seashell|Cup|Unknown|10|ationeseoughtese| +4148|AAAAAAAAEDABAAAA|1997-10-27|2000-10-26|Times last only as southern policies. Financial, spatial charts must become alw|4.40|2.24|5002001|importoscholar #1|2|country|5|Music|159|n stantiought|N/A|95963slate8737086344|seashell|Gram|Unknown|32|eingeseoughtese| +4149|AAAAAAAAEDABAAAA|2000-10-27||Times last only as southern policies. Financial, spatial charts must become alw|3.93|2.94|5002001|corpbrand #10|2|rugs|7|Home|242|ableeseable|N/A|95963slate8737086344|bisque|Case|Unknown|69|n steseoughtese| +4150|AAAAAAAAGDABAAAA|1997-10-27|1999-10-27|Women will run by the clients. Quarters proceed. Small ideas appear now vague affairs. Additional, social examples can go establis|3.11|1.24|6003007|exporticorp #7|3|gold|6|Jewelry|273|priationable|N/A|tomato60896280923263|steel|Tsp|Unknown|78|barantioughtese| +4151|AAAAAAAAGDABAAAA|1999-10-28|2001-10-26|As economic operations can date teams. Personnel could get alre|4.46|1.24|5002002|importoscholar #2|3|country|5|Music|273|priationable|N/A|46061663441powder950|medium|Dozen|Unknown|30|oughtantioughtese| +4152|AAAAAAAAGDABAAAA|2001-10-27||As economic operations can date teams. Personnel could get alre|4.23|1.24|10012005|importoamalgamalg #5|3|monitors|10|Electronics|398|eingn stpri|N/A|46061663441powder950|indian|Cup|Unknown|3|ableantioughtese| +4153|AAAAAAAAJDABAAAA|1997-10-27||Unique children force essentially fundamental eyes. Acids try rather strictly domestic houses. Even definite weeks used to s|3.51|1.47|1004002|edu packamalg #2|4|swimwear|1|Women|125|antiableought|medium|6wheat78893055775076|firebrick|Carton|Unknown|29|priantioughtese| +4154|AAAAAAAAKDABAAAA|1997-10-27|2000-10-26|Social eyes might complete at least customs. Very grea|7.73|4.71|7004005|edu packbrand #5|4|curtains/drapes|7|Home|311|oughtoughtpri|N/A|123moccasin014759847|turquoise|Tbl|Unknown|30|eseantioughtese| +4155|AAAAAAAAKDABAAAA|2000-10-27||Social eyes might complete at least customs. Very grea|9.13|4.71|10012008|importoamalgamalg #8|4|monitors|10|Electronics|311|oughtoughtpri|N/A|123moccasin014759847|misty|Tbl|Unknown|32|antiantioughtese| +4156|AAAAAAAAMDABAAAA|1997-10-27|1999-10-27|Military leaders will not mean to a values. Situations might lack. Foreign purposes pl|1.73|0.88|4001001|amalgedu pack #1|1|womens|4|Shoes|438|eingpriese|petite|952453372180peach566|tomato|Each|Unknown|15|callyantioughtese| +4157|AAAAAAAAMDABAAAA|1999-10-28|2001-10-26|Military leaders will not mean to a values. Situations might lack. Foreign purposes pl|2.96|0.88|4001001|edu packscholar #2|1|classical|5|Music|438|eingpriese|N/A|946950rosy3115275280|navajo|Each|Unknown|18|ationantioughtese| +4158|AAAAAAAAMDABAAAA|2001-10-27||Military leaders will not mean to a values. Situations might lack. Foreign purposes pl|6.22|0.88|10005011|scholarunivamalg #11|5|karoke|10|Electronics|438|eingpriese|N/A|531858232mint6002836|steel|Each|Unknown|4|eingantioughtese| +4159|AAAAAAAAPDABAAAA|1997-10-27||Big, obvious commitments must provide also; strange women will collect |7.69|2.53|3002002|importoexporti #2|2|infants|3|Children|130|barpriought|petite|peach468102206521110|seashell|Bundle|Unknown|20|n stantioughtese| +4160|AAAAAAAAAEABAAAA|1997-10-27|2000-10-26|Tropical adults give ready, absent estimates. Available, special horses find human types. Great effects handle obviously christian instit|4.84|2.80|2002001|importoimporto #1|2|shirts|2|Men|770|barationation|petite|7115387978851purple1|khaki|N/A|Unknown|47|barcallyoughtese| +4161|AAAAAAAAAEABAAAA|2000-10-27||Houses believe however. Sources control less on a demands. Processes integrate regional, mathematical stages. Fresh arms must not trade. Other hands o|3.94|2.91|3003002|exportiexporti #2|2|toddlers|3|Children|64|esecally|medium|7115387978851purple1|thistle|Each|Unknown|14|oughtcallyoughtese| +4162|AAAAAAAACEABAAAA|1997-10-27|1999-10-27|Important values shall say|1.94|0.95|7016003|corpnameless #3|16|furniture|7|Home|232|ablepriable|N/A|940126466indian64548|rose|Ton|Unknown|21|ablecallyoughtese| +4163|AAAAAAAACEABAAAA|1999-10-28|2001-10-26|Soon low problems find only. Local, familiar factors may ease much for a days. Exercises know eventually very left years. Senior courts suppose permanent, close months. Fond, local talks used to def|1.71|1.19|7016003|edu packscholar #2|4|classical|5|Music|232|ablepriable|N/A|957purple69514575689|purple|Carton|Unknown|48|pricallyoughtese| +4164|AAAAAAAACEABAAAA|2001-10-27||National times should take more full women. Large markets may not |6.66|4.72|7016003|importomaxi #1|2|business|9|Books|440|ablepriable|N/A|594puff8051271171940|medium|Oz|Unknown|84|esecallyoughtese| +4165|AAAAAAAAFEABAAAA|1997-10-27||Environmental days will dodge there electoral issues. Disputes requir|0.43|0.21|4002002|importoedu pack #2|2|mens|4|Shoes|623|priablecally|large|434971088572violet38|peru|Pound|Unknown|9|anticallyoughtese| +4166|AAAAAAAAGEABAAAA|1997-10-27|2000-10-26|Similar years should not attribute anyway now combined streets; important, convenient others represent moreover. Appropriate trousers provide more communications. Cultural comments would e|3.01|1.71|7007007|brandbrand #7|7|decor|7|Home|295|antin stable|N/A|841262medium89480814|saddle|Ton|Unknown|17|callycallyoughtese| +4167|AAAAAAAAGEABAAAA|2000-10-27||Similar years should not attribute anyway now combined streets; important, convenient others represent moreover. Appropriate trousers provide more communications. Cultural comments would e|7.84|3.52|6011008|amalgbrand #8|11|semi-precious|6|Jewelry|295|antin stable|N/A|4red7225032044318908|powder|Bunch|Unknown|14|ationcallyoughtese| +4168|AAAAAAAAIEABAAAA|1997-10-27|1999-10-27|Then real weeks pay sharp, new procedures; personal occasi|5.54|4.15|10014004|edu packamalgamalg #4|14|automotive|10|Electronics|260|barcallyable|N/A|46984364red483234977|sky|Tbl|Unknown|46|eingcallyoughtese| +4169|AAAAAAAAIEABAAAA|1999-10-28|2001-10-26|Then real weeks pay sharp, new procedures; personal occasi|9.72|3.40|10010005|univamalgamalg #5|14|memory|10|Electronics|11|oughtought|N/A|46984364red483234977|medium|Each|Unknown|47|n stcallyoughtese| +4170|AAAAAAAAIEABAAAA|2001-10-27||Then real weeks pay sharp, new procedures; personal occasi|8.33|3.40|6008003|namelesscorp #3|8|mens watch|6|Jewelry|11|oughtought|N/A|46984364red483234977|pale|Tsp|Unknown|76|barationoughtese| +4171|AAAAAAAALEABAAAA|1997-10-27||Always black matters say together on the explanations. Central problems get. Intervals favour severely disastrous reserves. Talks must retain scottish, fundamental years; other, fine |7.19|3.30|9009008|maximaxi #8|9|science|9|Books|958|eingantin st|N/A|4182213saddle0765196|rose|Oz|Unknown|32|oughtationoughtese| +4172|AAAAAAAAMEABAAAA|1997-10-27|2000-10-26|Tiny conditions may not clear about wonderful leaders. New, british miles may like outside even lega|57.26|29.77|7010003|univnameless #3|10|flatware|7|Home|139|n stpriought|N/A|923005791papaya92120|smoke|Pallet|Unknown|10|ableationoughtese| +4173|AAAAAAAAMEABAAAA|2000-10-27||Tiny conditions may not clear about wonderful leaders. New, british miles may like outside even lega|7.80|29.77|7010003|importoimporto #2|2|shirts|2|Men|72|n stpriought|N/A|923005791papaya92120|burlywood|Pallet|Unknown|35|priationoughtese| +4174|AAAAAAAAOEABAAAA|1997-10-27|1999-10-27|Demonstrations shall miss exact, labour thanks. Nuclear, rapid issues undermine vital provinces. Political, dark deals may get problems. Authori|5.36|4.55|7010005|univnameless #5|10|flatware|7|Home|226|callyableable|N/A|9255716saddle6239010|puff|Carton|Unknown|40|eseationoughtese| +4175|AAAAAAAAOEABAAAA|1999-10-28|2001-10-26|Inc, natural animals get bones. Willing beliefs know later final sizes. Cups will thi|4.94|4.55|7010005|exportiamalg #2|3|maternity|1|Women|311|oughtoughtpri|small|9255716saddle6239010|rosy|Ounce|Unknown|81|antiationoughtese| +4176|AAAAAAAAOEABAAAA|2001-10-27||Possible, young efforts could implement only key friends. Inevitably cle|2.44|4.55|1004001|edu packamalg #1|4|swimwear|1|Women|311|oughtoughtpri|medium|9255716saddle6239010|salmon|Each|Unknown|47|callyationoughtese| +4177|AAAAAAAABFABAAAA|1997-10-27||Windows flow just magnetic terms. Branches would possess |4.33|2.81|9010002|univunivamalg #2|10|travel|9|Books|606|callybarcally|N/A|327450522942773pale9|plum|Ounce|Unknown|16|ationationoughtese| +4178|AAAAAAAACFABAAAA|1997-10-27|2000-10-26|Fine, living women wait aside with the patients. Rarely arbitrary books should know already. Expenses will consider vigorously reports. Houses get there particular, local institutions. Really certain|7.88|5.35|8009009|maxinameless #9|9|optics|8|Sports|871|oughtationeing|N/A|5461steel31954474584|rose|Tsp|Unknown|36|eingationoughtese| +4179|AAAAAAAACFABAAAA|2000-10-27||Fine, living women wait aside with the patients. Rarely arbitrary books should know already. Expenses will consider vigorously reports. Houses get there particular, local institutions. Really certain|83.72|5.35|8009009|namelessbrand #10|8|lighting|7|Home|188|oughtationeing|N/A|72thistle90717076283|sky|Gross|Unknown|23|n stationoughtese| +4180|AAAAAAAAEFABAAAA|1997-10-27|1999-10-27|Similar, full sentences balance and so on in a systems; relationships benefit extremely at all possible bodies. Photographs might design part-time, urgent affair|0.74|0.57|10005016|scholarunivamalg #16|5|karoke|10|Electronics|484|eseeingese|N/A|6324165953118puff410|rose|Lb|Unknown|24|bareingoughtese| +4181|AAAAAAAAEFABAAAA|1999-10-28|2001-10-26|Similar, full sentences balance and so on in a systems; relationships benefit extremely at all possible bodies. Photographs might design part-time, urgent affair|2.71|0.57|10003017|exportiunivamalg #17|3|dvd/vcr players|10|Electronics|484|eseeingese|N/A|905967855sandy030927|puff|Gross|Unknown|60|oughteingoughtese| +4182|AAAAAAAAEFABAAAA|2001-10-27||Glad meanings would not boost as at a |3.43|3.05|9016007|corpunivamalg #7|16|mystery|9|Books|48|eingese|N/A|905967855sandy030927|rosy|Gram|Unknown|65|ableeingoughtese| +4183|AAAAAAAAHFABAAAA|1997-10-27||Comparable, fast rates must live for example current, specialist scores. Girls may not figure heavy products. Following items l|2.44|1.65|10003002|exportiunivamalg #2|3|dvd/vcr players|10|Electronics|41|oughtese|N/A|11413645turquoise344|hot|Cup|Unknown|38|prieingoughtese| +4184|AAAAAAAAIFABAAAA|1997-10-27|2000-10-26|Men shall tolerate easily too keen children. Relevant, full-time leaves cannot say presumably from the gods. Large, careful subjects sit pro|7.63|5.18|9004009|edu packmaxi #9|4|entertainments|9|Books|188|eingeingought|N/A|171280medium60046838|sandy|Cup|Unknown|42|eseeingoughtese| +4185|AAAAAAAAIFABAAAA|2000-10-27||Men shall tolerate easily too keen children. Relevant, full-time leaves cannot say presumably from the gods. Large, careful subjects sit pro|1.17|5.18|9004009|namelessunivamalg #4|8|scanners|10|Electronics|188|eingeingought|N/A|171280medium60046838|tomato|Each|Unknown|17|antieingoughtese| +4186|AAAAAAAAKFABAAAA|1997-10-27|1999-10-27|Significantly simple rules could face especially lively, popular employers. Days catc|1.96|1.23|7002007|importobrand #7|2|bedding|7|Home|571|oughtationanti|N/A|6373989234turquoise2|peru|Box|Unknown|74|callyeingoughtese| +4187|AAAAAAAAKFABAAAA|1999-10-28|2001-10-26|Yet powerful women leave gradually prices. Afraid instructions burn german, immediate lines. Special, industrial towns stand. Only royal parties put for a patterns. Ways think literar|2.55|1.58|7002007|namelesscorp #4|8|mens watch|6|Jewelry|763|oughtationanti|N/A|46275101477rosy60455|wheat|Bundle|Unknown|20|ationeingoughtese| +4188|AAAAAAAAKFABAAAA|2001-10-27||Regional, elected schools give yet in a benefits. General neg|1.17|1.58|5002001|importoscholar #1|2|country|5|Music|763|oughtationanti|N/A|46275101477rosy60455|papaya|Dozen|Unknown|10|eingeingoughtese| +4189|AAAAAAAANFABAAAA|1997-10-27||Processes may know. Cups might like sooner straight thousands. Alone strong concerns sho|4.77|4.19|3003002|exportiexporti #2|3|toddlers|3|Children|133|pripriought|petite|65397lemon8521467395|rose|Gross|Unknown|91|n steingoughtese| +4190|AAAAAAAAOFABAAAA|1997-10-27|2000-10-26|Implicati|2.46|1.77|8009001|maxinameless #1|9|optics|8|Sports|143|prieseought|N/A|3864580626364slate82|saddle|Case|Unknown|47|barn stoughtese| +4191|AAAAAAAAOFABAAAA|2000-10-27||Implicati|3.69|1.77|7009002|maxibrand #2|9|mattresses|7|Home|296|callyn stable|N/A|553yellow36795852751|purple|Each|Unknown|8|oughtn stoughtese| +4192|AAAAAAAAAGABAAAA|1997-10-27|1999-10-27|Patients escape by a instructions. Conv|5.31|2.54|10012011|importoamalgamalg #11|12|monitors|10|Electronics|623|priablecally|N/A|288943341568792rosy2|gainsboro|Ton|Unknown|30|ablen stoughtese| +4193|AAAAAAAAAGABAAAA|1999-10-28|2001-10-26|Badly difficult intervals should not get today new followers. Standards examine. Young, actual departments might knock special, national assumptions. Successful wi|0.29|0.21|10012011|amalgmaxi #6|12|arts|9|Books|623|priablecally|N/A|288943341568792rosy2|spring|N/A|Unknown|34|prin stoughtese| +4194|AAAAAAAAAGABAAAA|2001-10-27||Badly difficult intervals should not get today new followers. Standards examine. Young, actual departments might knock special, national assumptions. Successful wi|0.20|0.12|2002001|importoimporto #1|12|shirts|2|Men|623|priablecally|petite|288943341568792rosy2|drab|Cup|Unknown|41|esen stoughtese| +4195|AAAAAAAADGABAAAA|1997-10-27||Early dark trains may monitor never fairly leading men. National, large changes shall help different, international women. Expressions may dispense large, strict bacteria. Parts like c|7.94|3.81|9012008|importounivamalg #8|12|home repair|9|Books|174|eseationought|N/A|436156772466252hot55|puff|Tsp|Unknown|98|antin stoughtese| +4196|AAAAAAAAEGABAAAA|1997-10-27|2000-10-26|Sophisticated operations may advise continuing childre|5.44|2.55|5002001|importoscholar #1|2|country|5|Music|97|ationn st|N/A|6065048royal85651889|maroon|Cup|Unknown|7|callyn stoughtese| +4197|AAAAAAAAEGABAAAA|2000-10-27||Real, main streets would not like a little in a links. Certainly likely pp. would remember forward reportedly regulatory restrictions. Nuclear, due servants might not take properti|1.27|2.55|6012006|importobrand #6|2|costume|6|Jewelry|181|ationn st|N/A|6065048royal85651889|sky|Dram|Unknown|66|ationn stoughtese| +4198|AAAAAAAAGGABAAAA|1997-10-27|1999-10-27|Pairs get strong, good schools. Bloody, different priorities need very. Years may change mo|8.43|7.16|1002001|importoamalg #1|2|fragrances|1|Women|876|callyationeing|extra large|705350501peru1131760|pink|Case|Unknown|7|eingn stoughtese| +4199|AAAAAAAAGGABAAAA|1999-10-28|2001-10-26|Already posi|4.34|7.16|1002001|importounivamalg #1|2|camcorders|10|Electronics|10|callyationeing|N/A|705350501peru1131760|dodger|Case|Unknown|1|n stn stoughtese| +4200|AAAAAAAAGGABAAAA|2001-10-27||Already posi|26.71|7.16|1002001|brandunivamalg #2|7|personal|10|Electronics|300|callyationeing|N/A|705350501peru1131760|green|Gram|Unknown|7|barbarableese| +4201|AAAAAAAAJGABAAAA|1997-10-27||Ambitious views may |2.88|1.52|1004002|edu packamalg #2|4|swimwear|1|Women|968|eingcallyn st|extra large|505965546light485644|steel|Dozen|Unknown|64|oughtbarableese| +4202|AAAAAAAAKGABAAAA|1997-10-27|2000-10-26|Pale authorities look so successfully simple meals. Constant, fair babies meet the|2.27|1.56|1001001|amalgamalg #1|1|dresses|1|Women|71|oughtation|petite|rose9505468772679473|turquoise|Dram|Unknown|92|ablebarableese| +4203|AAAAAAAAKGABAAAA|2000-10-27||Faciliti|4.69|1.56|6011004|amalgbrand #4|1|semi-precious|6|Jewelry|71|oughtation|N/A|rose9505468772679473|khaki|Ounce|Unknown|62|pribarableese| +4204|AAAAAAAAMGABAAAA|1997-10-27|1999-10-27|Minutes must offer general, excellent signs. Groups afford just as wild needs. Abroad mild guidelines |69.92|44.74|6005005|scholarcorp #5|5|earings|6|Jewelry|93|prin st|N/A|5pink701141784774495|peach|Tsp|Unknown|68|esebarableese| +4205|AAAAAAAAMGABAAAA|1999-10-28|2001-10-26|Minutes must offer general, excellent signs. Groups afford just as wild needs. Abroad mild guidelines |0.54|44.74|6005005|amalgnameless #6|5|athletic shoes|8|Sports|238|prin st|N/A|5pink701141784774495|sky|Cup|Unknown|4|antibarableese| +4206|AAAAAAAAMGABAAAA|2001-10-27||Minutes must offer general, excellent signs. Groups afford just as wild needs. Abroad mild guidelines |1.05|0.54|6005005|amalgscholar #1|1|rock|5|Music|284|eseeingable|N/A|27654254840sky665034|purple|Lb|Unknown|9|callybarableese| +4207|AAAAAAAAPGABAAAA|1997-10-27||Women can provide new, thin memories. Conditions avoid useful, wrong lips; always royal times find. More private planes may talk again defences. Patients used to create unable letters. Main, spir|7.40|4.29|1001002|amalgamalg #2|1|dresses|1|Women|10|barought|medium|8282white85805953248|turquoise|Unknown|Unknown|90|ationbarableese| +4208|AAAAAAAAAHABAAAA|1997-10-27|2000-10-26|Smart committees ought to get really implications. I|0.75|0.67|3001001|amalgexporti #1|1|newborn|3|Children|41|oughtese|petite|1026958384hot0564147|ghost|N/A|Unknown|8|eingbarableese| +4209|AAAAAAAAAHABAAAA|2000-10-27||Smart committees ought to get really implications. I|5.58|0.67|2003002|exportiimporto #2|3|pants|2|Men|115|antioughtought|petite|47ivory0148408634620|royal|Carton|Unknown|62|n stbarableese| +4210|AAAAAAAACHABAAAA|1997-10-27|1999-10-27|Aware, fair |0.72|0.54|5001001|amalgscholar #1|1|rock|5|Music|777|ationationation|N/A|8991sienna9542400006|indian|Dram|Unknown|85|baroughtableese| +4211|AAAAAAAACHABAAAA|1999-10-28|2001-10-26|Aware, fair |8.05|0.54|5001001|scholarnameless #2|1|tables|7|Home|777|ationationation|N/A|8991sienna9542400006|cornflower|Cup|Unknown|24|oughtoughtableese| +4212|AAAAAAAACHABAAAA|2001-10-27||Aware, fair |5.62|4.66|6001005|amalgcorp #5|1|birdal|6|Jewelry|258|eingantiable|N/A|8991sienna9542400006|snow|Pallet|Unknown|7|ableoughtableese| +4213|AAAAAAAAFHABAAAA|1997-10-27||Presently contemporary facts could not get. Only shows get only unpleasant years; also rational negotiations marry w|3.60|2.59|5001002|amalgscholar #2|1|rock|5|Music|3|pri|N/A|312433495157pink3402|puff|Carton|Unknown|32|prioughtableese| +4214|AAAAAAAAGHABAAAA|1997-10-27|2000-10-26|Other fans become very for a eggs. Fresh, white cattle adjust good men. Con|4.04|3.23|6006007|corpcorp #7|6|rings|6|Jewelry|285|antieingable|N/A|6141white32594556315|rosy|Oz|Unknown|55|eseoughtableese| +4215|AAAAAAAAGHABAAAA|2000-10-27||Other fans become very for a eggs. Fresh, white cattle adjust good men. Con|8.96|3.23|4001002|amalgedu pack #2|1|womens|4|Shoes|12|ableought|petite|5royal50687441892650|pale|Unknown|Unknown|64|antioughtableese| +4216|AAAAAAAAIHABAAAA|1997-10-27|1999-10-27|Levels identify however suddenly social circumstances. Advertisements|4.25|2.50|10005001|scholarunivamalg #1|5|karoke|10|Electronics|311|oughtoughtpri|N/A|3900840spring2380789|steel|Tsp|Unknown|74|callyoughtableese| +4217|AAAAAAAAIHABAAAA|1999-10-28|2001-10-26|Levels identify however suddenly social circumstances. Advertisements|3.39|3.01|10005001|edu packedu pack #2|5|athletic|4|Shoes|134|esepriought|medium|3900840spring2380789|tomato|Pound|Unknown|7|ationoughtableese| +4218|AAAAAAAAIHABAAAA|2001-10-27||Levels identify however suddenly social circumstances. Advertisements|4.35|3.01|10005001|edu packunivamalg #7|5|sports|9|Books|858|eingantieing|N/A|856870679papaya55380|aquamarine|Case|Unknown|15|eingoughtableese| +4219|AAAAAAAALHABAAAA|1997-10-27||Almost low provisions suggest to|6.66|5.72|8008010|namelessnameless #10|8|outdoor|8|Sports|11|oughtought|N/A|0925996turquoise5849|wheat|Each|Unknown|53|n stoughtableese| +4220|AAAAAAAAMHABAAAA|1997-10-27|2000-10-26|Features might not get as pounds. Names should indicate ages. Police used to see ele|2.79|1.17|9006009|corpmaxi #9|6|parenting|9|Books|264|esecallyable|N/A|0986866719pale521068|chiffon|Case|Unknown|69|barableableese| +4221|AAAAAAAAMHABAAAA|2000-10-27||American, other circumstances tackle also with the facts. Passive|0.71|0.39|9006009|importomaxi #10|6|business|9|Books|264|esecallyable|N/A|44176326418rose04097|smoke|Carton|Unknown|82|oughtableableese| +4222|AAAAAAAAOHABAAAA|1997-10-27|1999-10-27|Other, willing materials could take ever external terms. Texts mean steady. Confident banks settle later national, foreign hours. Police will|4.20|2.14|7012003|importonameless #3|12|paint|7|Home|505|antibaranti|N/A|0427830786312plum970|misty|Box|Unknown|1|ableableableese| +4223|AAAAAAAAOHABAAAA|1999-10-28|2001-10-26|Other, willing materials could take ever external terms. Texts mean steady. Confident banks settle later national, foreign hours. Police will|21.07|2.14|7012003|maxinameless #4|9|optics|8|Sports|944|antibaranti|N/A|0427830786312plum970|powder|Tsp|Unknown|48|priableableese| +4224|AAAAAAAAOHABAAAA|2001-10-27||Too white police meet terribly at a countries; however other tons u|8.53|2.14|6010001|univbrand #1|10|jewelry boxes|6|Jewelry|539|antibaranti|N/A|01018903468542wheat2|light|Tbl|Unknown|14|eseableableese| +4225|AAAAAAAABIABAAAA|1997-10-27||Children ma|6.80|4.89|8009006|maxinameless #6|9|optics|8|Sports|310|baroughtpri|N/A|16170metallic1202570|steel|N/A|Unknown|46|antiableableese| +4226|AAAAAAAACIABAAAA|1997-10-27|2000-10-26|Real changes allow really. Current tasks regret nearly only numerous friends. Then public qualifications b|8.32|4.90|3004001|edu packexporti #1|4|school-uniforms|3|Children|900|barbarn st|medium|651smoke578727025575|thistle|Pound|Unknown|27|callyableableese| +4227|AAAAAAAACIABAAAA|2000-10-27||Given authorities should lead even so. Little, new childre|8.50|4.90|3004001|exportibrand #4|4|loose stones|6|Jewelry|900|barbarn st|N/A|34921343seashell7827|saddle|Gram|Unknown|36|ationableableese| +4228|AAAAAAAAEIABAAAA|1997-10-27|1999-10-27|Really foreign workers overcome asleep, young decades. Drugs may tell children; labour, real wages ev|4.24|2.96|8015009|scholarmaxi #9|15|fishing|8|Sports|360|barcallypri|N/A|4667000858plum529511|saddle|N/A|Unknown|54|eingableableese| +4229|AAAAAAAAEIABAAAA|1999-10-28|2001-10-26|New sections pay actually more different parents. Labour pupils shall grow else conveniently normal re|20.82|10.82|6006006|corpcorp #6|15|rings|6|Jewelry|360|barcallypri|N/A|4667000858plum529511|red|Gross|Unknown|3|n stableableese| +4230|AAAAAAAAEIABAAAA|2001-10-27||Mainly political difficulties vote again political, alternative pupils. Christian |9.93|6.85|6003007|exporticorp #7|15|gold|6|Jewelry|360|barcallypri|N/A|4667000858plum529511|purple|Tsp|Unknown|59|barpriableese| +4231|AAAAAAAAHIABAAAA|1997-10-27||However democratic families know then written examinations. Courts take now civil, free conditions. So ca|7.37|3.02|5002002|importoscholar #2|2|country|5|Music|236|callypriable|N/A|42480808357hot853982|pink|Unknown|Unknown|15|oughtpriableese| +4232|AAAAAAAAIIABAAAA|1997-10-27|2000-10-26|Religious, new movements learn successive magistrates. Comfortable, |2.01|1.00|7016003|corpnameless #3|16|furniture|7|Home|300|barbarpri|N/A|sandy483591568483266|indian|Bundle|Unknown|60|ablepriableese| +4233|AAAAAAAAIIABAAAA|2000-10-27||Religious, new movements learn successive magistrates. Comfortable, |3.41|1.00|1004002|edu packamalg #2|16|swimwear|1|Women|191|oughtn stought|large|sandy483591568483266|slate|Lb|Unknown|77|pripriableese| +4234|AAAAAAAAKIABAAAA|1997-10-27|1999-10-27|Responsible, spanish jobs tell of course in a months. Back demands will not work firmly as a nights. Words must not dry. As large days would commit ordinary, mental ideas. Dead, hot details |3.94|2.48|4003001|exportiedu pack #1|3|kids|4|Shoes|303|pribarpri|large|3905775peach74671344|tomato|Tsp|Unknown|12|esepriableese| +4235|AAAAAAAAKIABAAAA|1999-10-28|2001-10-26|Full, mean places might not account there never certain numbers. Able, model trials would |2.64|2.48|4003001|corpmaxi #6|16|golf|8|Sports|329|pribarpri|N/A|3905775peach74671344|snow|Cup|Unknown|11|antipriableese| +4236|AAAAAAAAKIABAAAA|2001-10-27||Yellow visitors must go informal techniques. Unnecessary months like high welsh, public lips; black, right workshops happen original differences|2.90|2.49|4004001|edu packedu pack #1|4|athletic|4|Shoes|302|pribarpri|petite|3905775peach74671344|rosy|Cup|Unknown|7|callypriableese| +4237|AAAAAAAANIABAAAA|1997-10-27||Prospective, other jeans must set short old women. |1.46|0.94|9007002|brandmaxi #2|7|reference|9|Books|623|priablecally|N/A|0chartreuse890629414|tan|Cup|Unknown|64|ationpriableese| +4238|AAAAAAAAOIABAAAA|1997-10-27|2000-10-26|Payments understand in a trends. Only average meetings evolve therefore religious years. Urgently old proceedings could|4.23|3.08|10013006|exportiamalgamalg #6|13|stereo|10|Electronics|13|priought|N/A|49042ivory6919827436|sandy|Carton|Unknown|7|eingpriableese| +4239|AAAAAAAAOIABAAAA|2000-10-27||Payments understand in a trends. Only average meetings evolve therefore religious years. Urgently old proceedings could|3.98|3.10|1004002|edu packamalg #2|4|swimwear|1|Women|13|priought|extra large|49042ivory6919827436|peru|Dram|Unknown|41|n stpriableese| +4240|AAAAAAAAAJABAAAA|1997-10-27|1999-10-27|Politic|5.13|1.69|3002001|importoexporti #1|2|infants|3|Children|22|ableable|medium|0866saddle8008125091|pale|Bundle|Unknown|2|bareseableese| +4241|AAAAAAAAAJABAAAA|1999-10-28|2001-10-26|Social, useful services become to a days. Spiritual, green opportunities shall pin still global trees. Objects forget chemicals. Frequently strange parties will show like.|0.31|1.69|10004009|edu packunivamalg #9|4|audio|10|Electronics|22|ableable|N/A|14793857888tomato175|spring|Carton|Unknown|11|oughteseableese| +4242|AAAAAAAAAJABAAAA|2001-10-27||Social, useful services become to a days. Spiritual, green opportunities shall pin still global trees. Objects forget chemicals. Frequently strange parties will show like.|4.27|3.24|10004009|maxinameless #3|9|optics|8|Sports|277|ationationable|N/A|14793857888tomato175|wheat|Ton|Unknown|41|ableeseableese| +4243|AAAAAAAADJABAAAA|1997-10-27||Difficult, public games know slightly right proceedings. Days may say attractiv|2.94|2.14|4002002|importoedu pack #2|2|mens|4|Shoes|232|ablepriable|small|113312990864sky95371|peru|Pound|Unknown|9|prieseableese| +4244|AAAAAAAAEJABAAAA|1997-10-27|2000-10-26|Communities make. Similar, open procedures could get for example during the youngsters. Healthy methods should not tell contemporary, inappropriate cha|1.47|0.89|1001001|amalgamalg #1|1|dresses|1|Women|478|eingationese|medium|972755white635996381|maroon|Cup|Unknown|6|eseeseableese| +4245|AAAAAAAAEJABAAAA|2000-10-27||Communities make. Similar, open procedures could get for example during the youngsters. Healthy methods should not tell contemporary, inappropriate cha|35.29|0.89|1001001|exportiexporti #2|3|toddlers|3|Children|478|eingationese|large|06791lawn43453124523|plum|Pound|Unknown|14|antieseableese| +4246|AAAAAAAAGJABAAAA|1997-10-27|1999-10-27|However roya|9.44|7.45|10013014|exportiamalgamalg #14|13|stereo|10|Electronics|420|barableese|N/A|4peach16689193424916|slate|Gross|Unknown|11|callyeseableese| +4247|AAAAAAAAGJABAAAA|1999-10-28|2001-10-26|Russian, coming methods must not evolve valid, relative boards; immediately nat|1.30|7.45|4002002|importoedu pack #2|13|mens|4|Shoes|420|barableese|petite|4peach16689193424916|seashell|Tbl|Unknown|16|ationeseableese| +4248|AAAAAAAAGJABAAAA|2001-10-27||Russian, coming methods must not evolve valid, relative boards; immediately nat|1.77|1.25|2004001|edu packimporto #1|13|sports-apparel|2|Men|95|antin st|petite|4peach16689193424916|powder|Gross|Unknown|95|eingeseableese| +4249|AAAAAAAAJJABAAAA|1997-10-27||Enough complete words go supreme authorities. Attitudes might run players. Respons|83.01|71.38|10006017|corpunivamalg #17|6|musical|10|Electronics|149|n steseought|N/A|7095639752537saddle1|salmon|Gram|Unknown|52|n steseableese| +4250|AAAAAAAAKJABAAAA|1997-10-27|2000-10-26|Authorities help notes. Almost significant goods look really as a farms. Flat schools pay closely m|9.63|4.33|10002001|importounivamalg #1|2|camcorders|10|Electronics|57|ationanti|N/A|928552279turquoise17|navy|Carton|Unknown|88|barantiableese| +4251|AAAAAAAAKJABAAAA|2000-10-27||Small payments cannot terminate much in a prices. Normal, other conservatives may specify human, alone observers. Significant clients will like inevitable goods; problems will not e|0.64|0.32|10002001|amalgexporti #2|2|newborn|3|Children|57|ationanti|small|sandy829147584348272|khaki|N/A|Unknown|61|oughtantiableese| +4252|AAAAAAAAMJABAAAA|1997-10-27|1999-10-27|Hardly mad councils might draw quantities. Y|2.35|1.38|5002001|importoscholar #1|2|country|5|Music|144|eseeseought|N/A|55401329peru00361458|salmon|Dozen|Unknown|49|ableantiableese| +4253|AAAAAAAAMJABAAAA|1999-10-28|2001-10-26|Local goods bite perhaps years. Dark feet may not call exactly from a shapes; high sums succeed; most creative estates use now delighted habits. Too useless charge|7.74|2.47|1001002|amalgamalg #2|2|dresses|1|Women|144|eseeseought|large|3ivory46998087269431|papaya|Oz|Unknown|88|priantiableese| +4254|AAAAAAAAMJABAAAA|2001-10-27||Local goods bite perhaps years. Dark feet may not call exactly from a shapes; high sums succeed; most creative estates use now delighted habits. Too useless charge|0.11|2.47|1001002|scholarunivamalg #5|5|karoke|10|Electronics|208|eingbarable|N/A|9541304snow120835697|indian|Carton|Unknown|30|eseantiableese| +4255|AAAAAAAAPJABAAAA|1997-10-27||Private copies should not want with the banks. No longer new polic|1.28|1.01|10013006|exportiamalgamalg #6|13|stereo|10|Electronics|461|oughtcallyese|N/A|goldenrod27543672636|saddle|Gross|Unknown|8|antiantiableese| +4256|AAAAAAAAAKABAAAA|1997-10-27|2000-10-26|Facilities can form too natural plants. Words can identify long free, concerned police. Popular patterns find par|9.50|7.88|3003001|exportiexporti #1|3|toddlers|3|Children|220|barableable|medium|9307650329087sky7497|midnight|Oz|Unknown|7|callyantiableese| +4257|AAAAAAAAAKABAAAA|2000-10-27||Stages would not see then of course large lands; p|1.73|7.88|3003001|importoexporti #2|2|infants|3|Children|220|barableable|medium|069016sandy983990004|salmon|Tbl|Unknown|62|ationantiableese| +4258|AAAAAAAACKABAAAA|1997-10-27|1999-10-27|Determined roads might lea|2.31|1.06|8012009|importomaxi #9|12|guns|8|Sports|365|anticallypri|N/A|3700873violet6850858|thistle|Box|Unknown|55|eingantiableese| +4259|AAAAAAAACKABAAAA|1999-10-28|2001-10-26|Excessive, common details might not fulfil live with the contents. Fields used to show generally with a years. Therefore prope|0.66|0.54|8012009|scholarunivamalg #12|15|fiction|9|Books|365|anticallypri|N/A|3700873violet6850858|spring|N/A|Unknown|66|n stantiableese| +4260|AAAAAAAACKABAAAA|2001-10-27||Wild, financial rates know well dramatically old positions; possible eyebrows could feel top, relevant girls. Unique systems review never small|1.98|1.58|8012009|edu packunivamalg #11|4|audio|10|Electronics|626|callyablecally|N/A|3700873violet6850858|peach|Each|Unknown|61|barcallyableese| +4261|AAAAAAAAFKABAAAA|1997-10-27||Mostly natural brothers would follow probably. Old dangers will not drag. Slightly acceptable voices could not recom|9.81|7.84|4002002|importoedu pack #2|2|mens|4|Shoes|631|oughtprically|medium|741804saddle04349277|papaya|Tsp|Unknown|16|oughtcallyableese| +4262|AAAAAAAAGKABAAAA|1997-10-27|2000-10-26|Never special sentences look small aspects. Eng|4.85|2.13|9010003|univunivamalg #3|10|travel|9|Books|271|oughtationable|N/A|086816puff8605200430|lavender|N/A|Unknown|46|ablecallyableese| +4263|AAAAAAAAGKABAAAA|2000-10-27||Never special sentences look small aspects. Eng|7.36|2.79|9010003|edu packedu pack #2|10|athletic|4|Shoes|188|oughtationable|medium|7466saddle0267663792|peru|Case|Unknown|52|pricallyableese| +4264|AAAAAAAAIKABAAAA|1997-10-27|1999-10-27|Worthy groups would hurt after a systems. Panels can use|8.09|4.53|10013015|exportiamalgamalg #15|13|stereo|10|Electronics|394|esen stpri|N/A|steel311124606712491|linen|Dram|Unknown|55|esecallyableese| +4265|AAAAAAAAIKABAAAA|1999-10-28|2001-10-26|Worthy groups would hurt after a systems. Panels can use|1.67|0.55|4003002|exportiedu pack #2|3|kids|4|Shoes|241|esen stpri|medium|steel311124606712491|ghost|Dram|Unknown|15|anticallyableese| +4266|AAAAAAAAIKABAAAA|2001-10-27||So environmental cells might not remind. Then regula|3.58|0.55|1002001|importoamalg #1|3|fragrances|1|Women|713|prioughtation|large|0yellow5064292370244|sienna|Each|Unknown|3|callycallyableese| +4267|AAAAAAAALKABAAAA|1997-10-27||Good christians decide too satisfied hands. Glasses might not spend never main feelings. Other, precious parents will record far true combinations|70.55|31.04|4001002|amalgedu pack #2|1|womens|4|Shoes|594|esen stanti|extra large|033231floral48813092|steel|Unknown|Unknown|37|ationcallyableese| +4268|AAAAAAAAMKABAAAA|1997-10-27|2000-10-26|Encouraging wines used to include quite grounds|5.01|2.55|2004001|edu packimporto #1|4|sports-apparel|2|Men|26|callyable|economy|317909chartreuse7006|steel|Oz|Unknown|54|eingcallyableese| +4269|AAAAAAAAMKABAAAA|2000-10-27||Encouraging wines used to include quite grounds|6.05|2.55|10002003|importounivamalg #3|2|camcorders|10|Electronics|26|callyable|N/A|14moccasin7893314277|thistle|Ounce|Unknown|76|n stcallyableese| +4270|AAAAAAAAOKABAAAA|1997-10-27|1999-10-27|Ever long elements used to obtain equ|5.88|4.82|8004001|edu packnameless #1|4|camping|8|Sports|401|oughtbarese|N/A|86861papaya697415097|wheat|Tbl|Unknown|16|barationableese| +4271|AAAAAAAAOKABAAAA|1999-10-28|2001-10-26|Ever long elements used to obtain equ|9.92|4.82|8004001|amalgexporti #2|4|newborn|3|Children|179|n stationought|medium|310floral99961497771|snow|Bundle|Unknown|12|oughtationableese| +4272|AAAAAAAAOKABAAAA|2001-10-27||Ever long elements used to obtain equ|6.78|5.49|10010006|univamalgamalg #6|4|memory|10|Electronics|635|antiprically|N/A|310floral99961497771|purple|Pound|Unknown|13|ableationableese| +4273|AAAAAAAABLABAAAA|1997-10-27||Years include away whole, necessary ties; perhaps careful police reinforce especially full, evil readers; social, young conditions shall not la|7.32|2.92|4004002|edu packedu pack #2|4|athletic|4|Shoes|566|callycallyanti|large|steel014573065354893|azure|Each|Unknown|5|priationableese| +4274|AAAAAAAACLABAAAA|1997-10-27|2000-10-26|Minor, obvi|1.53|0.88|9008003|namelessmaxi #3|8|romance|9|Books|91|oughtn st|N/A|62medium482360250166|peach|Ounce|Unknown|27|eseationableese| +4275|AAAAAAAACLABAAAA|2000-10-27||Minor, obvi|1.53|0.88|8005006|scholarnameless #6|8|fitness|8|Sports|91|oughtn st|N/A|62medium482360250166|peru|Bunch|Unknown|11|antiationableese| +4276|AAAAAAAAELABAAAA|1997-10-27|1999-10-27|Only lucky parts may accept at all eyes; questions |0.57|0.40|6003005|exporticorp #5|3|gold|6|Jewelry|182|ableeingought|N/A|7slate72247493972555|pink|Ounce|Unknown|2|callyationableese| +4277|AAAAAAAAELABAAAA|1999-10-28|2001-10-26|Particles must not leave times. Gently careful|8.16|3.18|6007006|brandcorp #6|3|pendants|6|Jewelry|182|ableeingought|N/A|7slate72247493972555|peru|Each|Unknown|10|ationationableese| +4278|AAAAAAAAELABAAAA|2001-10-27||Simply natural universities would appreciate just. Existin|1.12|3.18|10011012|amalgamalgamalg #12|11|disk drives|10|Electronics|182|ableeingought|N/A|13805seashell0516261|medium|Bunch|Unknown|52|eingationableese| +4279|AAAAAAAAHLABAAAA|1997-10-27||Shares should work best in a clothes. Commonly strong criteria think almost; programmes fall about things. Hours rest towards a months. Calls would sell certainly. United eyes must |2.54|1.27|10011013|amalgamalgamalg #13|11|disk drives|10|Electronics|551|oughtantianti|N/A|288872404763navy6397|pink|Oz|Unknown|8|n stationableese| +4280|AAAAAAAAILABAAAA|1997-10-27|2000-10-26|Base standards must not discuss ever just years. Human times used to derive especially therefore deaf programmes. Brit|97.72|69.38|1004001|edu packamalg #1|4|swimwear|1|Women|380|bareingpri|extra large|rose9877788868638588|lemon|Carton|Unknown|8|bareingableese| +4281|AAAAAAAAILABAAAA|2000-10-27||Base standards must not discuss ever just years. Human times used to derive especially therefore deaf programmes. Brit|8.81|69.38|10006015|corpunivamalg #15|4|musical|10|Electronics|290|bareingpri|N/A|rose9877788868638588|sienna|Bunch|Unknown|24|oughteingableese| +4282|AAAAAAAAKLABAAAA|1997-10-27|1999-10-27|Other teachers come different, special pieces. Bri|84.51|57.46|5004001|edu packscholar #1|4|classical|5|Music|197|ationn stought|N/A|5623037medium4731996|dim|Pallet|Unknown|39|ableeingableese| +4283|AAAAAAAAKLABAAAA|1999-10-28|2001-10-26|Other teachers come different, special pieces. Bri|67.20|57.46|4004002|edu packedu pack #2|4|athletic|4|Shoes|236|callypriable|medium|5623037medium4731996|saddle|Oz|Unknown|3|prieingableese| +4284|AAAAAAAAKLABAAAA|2001-10-27||Other teachers come different, special pieces. Bri|3.88|57.46|8007005|brandnameless #5|4|hockey|8|Sports|36|callypriable|N/A|39298880398misty9019|forest|Carton|Unknown|100|eseeingableese| +4285|AAAAAAAANLABAAAA|1997-10-27||Reports ought to contract arab, full friends; very impressive executives find there kings. Backwards local exp|4.05|2.83|1001002|amalgamalg #2|1|dresses|1|Women|292|ablen stable|medium|purple42179445360562|ghost|Lb|Unknown|22|antieingableese| +4286|AAAAAAAAOLABAAAA|1997-10-27|2000-10-26|Pairs sack forward democratic banks. Already full factors should know for example essential eyes. Bad considerati|5.47|1.96|2002001|importoimporto #1|2|shirts|2|Men|891|oughtn steing|extra large|7303598288535khaki87|medium|Case|Unknown|47|callyeingableese| +4287|AAAAAAAAOLABAAAA|2000-10-27||Fingers will treat sharply |9.86|1.96|8001008|amalgnameless #8|1|athletic shoes|8|Sports|338|eingpripri|N/A|7303598288535khaki87|papaya|Gross|Unknown|3|ationeingableese| +4288|AAAAAAAAAMABAAAA|1997-10-27|1999-10-27|Private, dead dealers benefit always average, various changes. Married problems know so equal premises. Political, fine processes wo|8.95|7.33|6004001|edu packcorp #1|4|bracelets|6|Jewelry|56|callyanti|N/A|5360449916452pale327|rosy|Bunch|Unknown|67|eingeingableese| +4289|AAAAAAAAAMABAAAA|1999-10-28|2001-10-26|Semantic, following rights mean creative, agricultural strategie|4.24|3.47|10013006|exportiamalgamalg #6|4|stereo|10|Electronics|56|callyanti|N/A|373papaya71340592111|puff|Lb|Unknown|45|n steingableese| +4290|AAAAAAAAAMABAAAA|2001-10-27||Semantic, following rights mean creative, agricultural strategie|1.24|3.47|10013006|brandbrand #1|4|decor|7|Home|56|callyanti|N/A|99351powder011091567|green|Tsp|Unknown|7|barn stableese| +4291|AAAAAAAADMABAAAA|1997-10-27||Modules will not conclude firm members. Base, various groups keep either industrial, cultural years. Subject, political links|48.62|16.04|2004002|edu packimporto #2|4|sports-apparel|2|Men|912|ableoughtn st|petite|8peach59913835352657|ivory|Unknown|Unknown|8|oughtn stableese| +4292|AAAAAAAAEMABAAAA|1997-10-27|2000-10-26|More original questions might weave very on behalf of the events. Economic standards go at a sheets. Around recent patterns see then actively massive hands. New, social women will|6.61|5.94|7004003|edu packbrand #3|4|curtains/drapes|7|Home|961|oughtcallyn st|N/A|1royal09068741246505|purple|Case|Unknown|9|ablen stableese| +4293|AAAAAAAAEMABAAAA|2000-10-27||Usual principles should write out of a banks. Hands must no|1.43|0.48|1003002|exportiamalg #2|4|maternity|1|Women|961|oughtcallyn st|extra large|7731068458violet5937|puff|Ton|Unknown|41|prin stableese| +4294|AAAAAAAAGMABAAAA|1997-10-27|1999-10-27|Then possible devices can conclude. Important drugs should stop much; ot|1.09|0.83|9014011|edu packunivamalg #11|14|sports|9|Books|87|ationeing|N/A|935998spring55374107|beige|Dram|Unknown|4|esen stableese| +4295|AAAAAAAAGMABAAAA|1999-10-28|2001-10-26|Ways would generalise so home regional systems. Other officers provide outside high, great camps. Other, particular feet give normally. Ever direct conditions ou|7.11|0.83|3003002|exportiexporti #2|14|toddlers|3|Children|296|ationeing|medium|935998spring55374107|pale|Ounce|Unknown|32|antin stableese| +4296|AAAAAAAAGMABAAAA|2001-10-27||Ways would generalise so home regional systems. Other officers provide outside high, great camps. Other, particular feet give normally. Ever direct conditions ou|8.61|6.88|3003002|edu packbrand #1|14|estate|6|Jewelry|296|ationeing|N/A|23458296070017peru06|steel|Carton|Unknown|28|callyn stableese| +4297|AAAAAAAAJMABAAAA|1997-10-27||Long parties shall speak specifically so daily forms; already eventual reasons put on the pp.. American cars could arrive before. Hours must result other workers. Underly|2.31|1.77|3003002|exportiexporti #2|3|toddlers|3|Children|226|callyableable|medium|21999252magenta53051|thistle|N/A|Unknown|49|ationn stableese| +4298|AAAAAAAAKMABAAAA|1997-10-27|2000-10-26|Pretty, part|2.90|1.59|8008009|namelessnameless #9|8|outdoor|8|Sports|317|ationoughtpri|N/A|9757654951901puff053|tan|Box|Unknown|73|eingn stableese| +4299|AAAAAAAAKMABAAAA|2000-10-27||Pretty, part|2.77|1.59|8008009|amalgexporti #2|8|newborn|3|Children|317|ationoughtpri|extra large|9757654951901puff053|pale|Unknown|Unknown|30|n stn stableese| +4300|AAAAAAAAMMABAAAA|1997-10-27|1999-10-27|Stupid years should not modify loose. Special, sure developments may|5.81|4.47|6009005|maxicorp #5|9|womens watch|6|Jewelry|963|pricallyn st|N/A|81611rosy38180306986|peach|Case|Unknown|16|barbarpriese| +4301|AAAAAAAAMMABAAAA|1999-10-28|2001-10-26|Stupid years should not modify loose. Special, sure developments may|1.88|1.22|7005002|scholarbrand #2|5|blinds/shades|7|Home|963|pricallyn st|N/A|81611rosy38180306986|pale|Bundle|Unknown|26|oughtbarpriese| +4302|AAAAAAAAMMABAAAA|2001-10-27||Rises should mould well modern, long colours. True families could argue rich, increased patients. Hurriedly european tales will not prepare. Late diseases used to pick with the friends. Items mi|8.67|1.22|7005002|amalgscholar #1|5|rock|5|Music|47|pricallyn st|N/A|81611rosy38180306986|puff|Dozen|Unknown|26|ablebarpriese| +4303|AAAAAAAAPMABAAAA|1997-10-27||So young experiences tell graphically. Today reliable students describe possible, young men; charges can play very more certain other|3.34|2.23|5004002|edu packscholar #2|4|classical|5|Music|123|priableought|N/A|91326lawn39886090468|moccasin|Cup|Unknown|42|pribarpriese| +4304|AAAAAAAAANABAAAA|1997-10-27|2000-10-26|Signs would repeat enough economic, annual books. |67.01|34.84|7012005|importonameless #5|12|paint|7|Home|176|callyationought|N/A|papaya89322087397117|steel|Box|Unknown|31|esebarpriese| +4305|AAAAAAAAANABAAAA|2000-10-27||Now small others will raise. Traditional premises see still immediate businesses. Silent produ|1.61|0.66|7012005|exportiexporti #2|3|toddlers|3|Children|176|callyationought|small|9724cornsilk40482108|sandy|Gross|Unknown|26|antibarpriese| +4306|AAAAAAAACNABAAAA|1997-10-27|1999-10-27|Surprisingly economic chapters accept tired,|0.52|0.19|5004001|edu packscholar #1|4|classical|5|Music|455|antiantiese|N/A|9615805509074201tan0|snow|N/A|Unknown|30|callybarpriese| +4307|AAAAAAAACNABAAAA|1999-10-28|2001-10-26|Small, busy schools shall not want big effects. Smart, potential pensions exceed else other years. Facts ought to adhere more advisers. Police must not bring prizes. Women should re|1.43|0.64|5003002|exportischolar #2|3|pop|5|Music|47|ationese|N/A|53332papaya088611358|khaki|Box|Unknown|18|ationbarpriese| +4308|AAAAAAAACNABAAAA|2001-10-27||Small, busy schools shall not want big effects. Smart, potential pensions exceed else other years. Facts ought to adhere more advisers. Police must not bring prizes. Women should re|4.67|0.64|5003002|exportiimporto #1|3|pants|2|Men|584|ationese|large|yellow53818160734352|sienna|Ounce|Unknown|58|eingbarpriese| +4309|AAAAAAAAFNABAAAA|1997-10-27||Disabled, southern chapters might not go possibly new activities. Interesting, basic hors|2.88|2.10|5002002|importoscholar #2|2|country|5|Music|261|oughtcallyable|N/A|9278559145sandy97519|violet|Carton|Unknown|21|n stbarpriese| +4310|AAAAAAAAGNABAAAA|1997-10-27|2000-10-26|Essential, great values belie|5.07|2.48|3001001|amalgexporti #1|1|newborn|3|Children|282|ableeingable|medium|863216596779orchid29|tomato|Lb|Unknown|15|baroughtpriese| +4311|AAAAAAAAGNABAAAA|2000-10-27||Personal, wealthy occasions fulfil. Physical, distant instructions may not put. Sure, private pages choose unusual, appropriate minutes. Boys can avoi|0.50|0.32|3001001|importoscholar #2|1|country|5|Music|282|ableeingable|N/A|863216596779orchid29|cornflower|Carton|Unknown|96|oughtoughtpriese| +4312|AAAAAAAAINABAAAA|1997-10-27|1999-10-27|Flexible towns shall not take simply ever proposed times. Other, short features raise services. Conside|2.07|1.34|8003003|exportinameless #3|3|basketball|8|Sports|970|barationn st|N/A|20red890333120464854|snow|Bunch|Unknown|18|ableoughtpriese| +4313|AAAAAAAAINABAAAA|1999-10-28|2001-10-26|Flexible towns shall not take simply ever proposed times. Other, short features raise services. Conside|7.49|1.34|7007004|brandbrand #4|7|decor|7|Home|348|eingesepri|N/A|20red890333120464854|thistle|Oz|Unknown|29|prioughtpriese| +4314|AAAAAAAAINABAAAA|2001-10-27||Just mutual workers|0.81|0.55|10014014|edu packamalgamalg #14|7|automotive|10|Electronics|246|eingesepri|N/A|puff0573497473553816|spring|Cup|Unknown|6|eseoughtpriese| +4315|AAAAAAAALNABAAAA|1997-10-27||Faint ways would not monitor just related families. Feet could see.|3.29|2.00|7012006|importonameless #6|12|paint|7|Home|352|ableantipri|N/A|521129050476royal060|green|Bundle|Unknown|52|antioughtpriese| +4316|AAAAAAAAMNABAAAA|1997-10-27|2000-10-26|Major applications must not sit. Video-taped minutes might not know only for a songs. Still loyal casualties used to make always from a wome|3.85|2.77|1003001|exportiamalg #1|3|maternity|1|Women|534|eseprianti|petite|25123061300plum68275|lace|Dozen|Unknown|36|callyoughtpriese| +4317|AAAAAAAAMNABAAAA|2000-10-27||Major applications must not sit. Video-taped minutes might not know only for a songs. Still loyal casualties used to make always from a wome|1.25|2.77|7003008|exportibrand #8|3|kids|7|Home|987|eseprianti|N/A|8antique344383544416|pale|Cup|Unknown|12|ationoughtpriese| +4318|AAAAAAAAONABAAAA|1997-10-27|1999-10-27|Thus main stories could ask enough brief hours. Open sites ought to make finally even different|3.49|1.43|5001001|amalgscholar #1|1|rock|5|Music|266|callycallyable|N/A|773584515orange20963|seashell|Oz|Unknown|28|eingoughtpriese| +4319|AAAAAAAAONABAAAA|1999-10-28|2001-10-26|Married measures let distinctly |7.09|1.43|1002002|importoamalg #2|2|fragrances|1|Women|266|callycallyable|petite|21118wheat4808511090|tan|Dram|Unknown|75|n stoughtpriese| +4320|AAAAAAAAONABAAAA|2001-10-27||Plants may reinforce following experiments; else other schools play other principle|3.06|1.43|1002002|namelessunivamalg #3|8|scanners|10|Electronics|266|callycallyable|N/A|43101018navajo584495|hot|Unknown|Unknown|11|barablepriese| +4321|AAAAAAAABOABAAAA|1997-10-27||Standards live still with a clergy. Goods may hate issues. Processes would not see only. Of course yellow towns follow quite beautiful ministers. P|0.29|0.21|3003002|exportiexporti #2|3|toddlers|3|Children|259|n stantiable|petite|95slate7234336538960|white|Unknown|Unknown|67|oughtablepriese| +4322|AAAAAAAACOABAAAA|1997-10-27|2000-10-26|Solicitors get. High, multiple boards shall jump generally.|2.34|1.82|10012005|importoamalgamalg #5|12|monitors|10|Electronics|79|n station|N/A|537582860saddle45401|pale|Bundle|Unknown|58|ableablepriese| +4323|AAAAAAAACOABAAAA|2000-10-27||Patients reveal wrong. So molecu|4.41|1.49|10012005|exportiamalgamalg #6|12|stereo|10|Electronics|79|n station|N/A|537582860saddle45401|smoke|Bundle|Unknown|54|priablepriese| +4324|AAAAAAAAEOABAAAA|1997-10-27|1999-10-27|Rapidly eventual problems can make. Final shareh|3.97|1.90|6010005|univbrand #5|10|jewelry boxes|6|Jewelry|119|n stoughtought|N/A|94100604salmon817650|pink|Dozen|Unknown|49|eseablepriese| +4325|AAAAAAAAEOABAAAA|1999-10-28|2001-10-26|Rapidly eventual problems can make. Final shareh|5.27|1.90|6010006|univbrand #6|10|jewelry boxes|6|Jewelry|649|n stesecally|N/A|40white9191290429889|royal|Oz|Unknown|25|antiablepriese| +4326|AAAAAAAAEOABAAAA|2001-10-27||Implicit costs mean negative effects; successful, sensitive trees allow there metropolitan difficul|3.40|1.90|6010006|importocorp #7|10|diamonds|6|Jewelry|299|n stn stable|N/A|592992194849rosy9221|rosy|Tbl|Unknown|18|callyablepriese| +4327|AAAAAAAAHOABAAAA|1997-10-27||Near medium rates take very. Rocks promote. Senior institutions will define more in a titles. Wr|3.39|1.32|10011010|amalgamalgamalg #10|11|disk drives|10|Electronics|157|ationantiought|N/A|461717dim70733401204|steel|Dozen|Unknown|96|ationablepriese| +4328|AAAAAAAAIOABAAAA|1997-10-27|2000-10-26|Months get also good, geographical years. Units may tell s|2.88|1.35|1004001|edu packamalg #1|4|swimwear|1|Women|188|eingeingought|extra large|7796876lemon10606981|pink|Oz|Unknown|17|eingablepriese| +4329|AAAAAAAAIOABAAAA|2000-10-27||Annual, genuine pp. see. Academic, inc products would help up front relative, traditional actions; companies change reliable difficulties. Tough resources return months; there able governmen|1.32|0.62|1004001|edu packexporti #2|4|school-uniforms|3|Children|583|eingeingought|medium|7796876lemon10606981|moccasin|Oz|Unknown|81|n stablepriese| +4330|AAAAAAAAKOABAAAA|1997-10-27|1999-10-27|Months go indian books. National, royal hands must care huge losses; attitudes support immediately great, developing cells. Complex times will sit certainly visitors. Afraid seeds attribute over gl|4.39|1.79|7011001|amalgnameless #1|11|accent|7|Home|36|callypri|N/A|53935229165302pink97|spring|Case|Unknown|71|barpripriese| +4331|AAAAAAAAKOABAAAA|1999-10-28|2001-10-26|Months go indian books. National, royal hands must care huge losses; attitudes support immediately great, developing cells. Complex times will sit certainly visitors. Afraid seeds attribute over gl|3.39|1.32|7011001|exportiimporto #2|3|pants|2|Men|538|callypri|medium|53935229165302pink97|yellow|Dram|Unknown|32|oughtpripriese| +4332|AAAAAAAAKOABAAAA|2001-10-27||Chemicals activate also lawyers; experimental, other developments cover with an factors. Actual decisions |1.75|1.06|7011001|namelessbrand #3|8|lighting|7|Home|538|eingprianti|N/A|41911379614867cyan11|turquoise|Unknown|Unknown|26|ablepripriese| +4333|AAAAAAAANOABAAAA|1997-10-27||Political years become more special, public neighbours. Very main doctors cannot wait in a systems. Pr|3.69|1.40|5004002|edu packscholar #2|4|classical|5|Music|718|eingoughtation|N/A|9473912peru509324046|misty|Dram|Unknown|29|pripripriese| +4334|AAAAAAAAOOABAAAA|1997-10-27|2000-10-26|Almost possible attempts ought to check very difficult ways. Legs used to arrange almost outsid|88.12|71.37|5003001|exportischolar #1|3|pop|5|Music|704|esebaration|N/A|25seashell5594972318|saddle|Bunch|Unknown|42|esepripriese| +4335|AAAAAAAAOOABAAAA|2000-10-27||Other sisters ought to un|8.96|6.89|5003001|importoexporti #2|2|infants|3|Children|704|esebaration|large|4295704964035maroon7|smoke|Bunch|Unknown|94|antipripriese| +4336|AAAAAAAAAPABAAAA|1997-10-27|1999-10-27|Bro|0.87|0.29|5001001|amalgscholar #1|1|rock|5|Music|315|antioughtpri|N/A|3126389818958sandy46|hot|Pound|Unknown|37|callypripriese| +4337|AAAAAAAAAPABAAAA|1999-10-28|2001-10-26|Bro|1.79|0.93|5001001|exportinameless #8|1|basketball|8|Sports|94|antioughtpri|N/A|3126389818958sandy46|slate|Cup|Unknown|1|ationpripriese| +4338|AAAAAAAAAPABAAAA|2001-10-27||Bro|2.03|1.15|5001001|edu packedu pack #1|4|athletic|4|Shoes|186|callyeingought|economy|1846043hot7184267116|seashell|Cup|Unknown|1|eingpripriese| +4339|AAAAAAAADPABAAAA|1997-10-27||European, incredible years might manage relatively in a trust|1.02|0.39|4002002|importoedu pack #2|2|mens|4|Shoes|451|oughtantiese|small|0077334white41131125|gainsboro|Unknown|Unknown|33|n stpripriese| +4340|AAAAAAAAEPABAAAA|1997-10-27|2000-10-26|Military workers can help. Long firm members split like a horses. Sufficient, adequate concessions interview obvious, mutual uses. Domestic seconds sho|95.02|59.86|10007006|brandunivamalg #6|7|personal|10|Electronics|260|barcallyable|N/A|74726695159saddle001|spring|Cup|Unknown|34|baresepriese| +4341|AAAAAAAAEPABAAAA|2000-10-27||Negative stages write now english, national parties. Odd, high shadows can think only weeks. Principles could stop massive keys. Left, powerf|0.88|0.48|10007006|edu packscholar #2|4|classical|5|Music|298|eingn stable|N/A|papaya50488305891314|thistle|Carton|Unknown|49|oughtesepriese| +4342|AAAAAAAAGPABAAAA|1997-10-27|1999-10-27|So islamic names used to force. Really good affairs used to walk actively british, other wings. Important classes equate simply inches. |2.45|2.03|9010011|univunivamalg #11|10|travel|9|Books|448|eingeseese|N/A|05linen0536617863065|misty|Dram|Unknown|56|ableesepriese| +4343|AAAAAAAAGPABAAAA|1999-10-28|2001-10-26|So islamic names used to force. Really good affairs used to walk actively british, other wings. Important classes equate simply inches. |1.33|0.99|4002002|importoedu pack #2|2|mens|4|Shoes|448|eingeseese|medium|5911778668lawn053072|white|Pound|Unknown|25|priesepriese| +4344|AAAAAAAAGPABAAAA|2001-10-27||So islamic names used to force. Really good affairs used to walk actively british, other wings. Important classes equate simply inches. |9.35|0.99|4002002|maxiunivamalg #10|2|televisions|10|Electronics|448|eingeseese|N/A|5911778668lawn053072|gainsboro|Oz|Unknown|60|eseesepriese| +4345|AAAAAAAAJPABAAAA|1997-10-27||Small, marked museums ought to validate. Ready circles disclose ahead on a months;|1.95|0.83|7004006|edu packbrand #6|4|curtains/drapes|7|Home|121|oughtableought|N/A|4882142147cream80877|peru|Tsp|Unknown|13|antiesepriese| +4346|AAAAAAAAKPABAAAA|1997-10-27|2000-10-26|Companies must not use especially other sentences. Just roman years benefit particular effects. Sometimes only factors imitate groups. Big processes would not require public, particular banks. |1.75|0.80|9004003|edu packmaxi #3|4|entertainments|9|Books|312|ableoughtpri|N/A|553rose6238706333924|pink|Tsp|Unknown|33|callyesepriese| +4347|AAAAAAAAKPABAAAA|2000-10-27||Companies must not use especially other sentences. Just roman years benefit particular effects. Sometimes only factors imitate groups. Big processes would not require public, particular banks. |8.02|2.72|5003002|exportischolar #2|4|pop|5|Music|218|ableoughtpri|N/A|553rose6238706333924|violet|Unknown|Unknown|18|ationesepriese| +4348|AAAAAAAAMPABAAAA|1997-10-27|1999-10-27|Original, recent armies must not back firms. Physical, valid women shall consider young, interested animals. British, new responses shall become brilliantly references. Outstanding, due cases sh|1.72|0.75|8001009|amalgnameless #9|1|athletic shoes|8|Sports|128|eingableought|N/A|97medium660515039299|seashell|Cup|Unknown|4|eingesepriese| +4349|AAAAAAAAMPABAAAA|1999-10-28|2001-10-26|Opinion|7.78|0.75|8001009|importoexporti #2|1|infants|3|Children|128|eingableought|small|0042988718825saddle0|gainsboro|Gram|Unknown|12|n stesepriese| +4350|AAAAAAAAMPABAAAA|2001-10-27||Findings move still usually special owners. Certain objections ought to start with a parents. Serious conditions shall smile less little farmers. Names must not ensure more than pos|7.45|0.75|10013016|exportiamalgamalg #16|13|stereo|10|Electronics|128|eingableought|N/A|8214915violet1008050|rosy|Ton|Unknown|38|barantipriese| +4351|AAAAAAAAPPABAAAA|1997-10-27||Ju|7.46|3.73|3004002|edu packexporti #2|4|school-uniforms|3|Children|231|oughtpriable|large|19777metallic3927108|mint|Gross|Unknown|24|oughtantipriese| +4352|AAAAAAAAAABBAAAA|1997-10-27|2000-10-26|Positions see lovely months. Proper patterns may not appeal in a c|5.21|4.32|2001001|amalgimporto #1|1|accessories|2|Men|636|callyprically|medium|203477812849magenta3|honeydew|Gram|Unknown|73|ableantipriese| +4353|AAAAAAAAAABBAAAA|2000-10-27||Recent discussi|5.37|2.14|2001001|importoscholar #2|1|country|5|Music|636|callyprically|N/A|203477812849magenta3|sky|Unknown|Unknown|10|priantipriese| +4354|AAAAAAAACABBAAAA|1997-10-27|1999-10-27|Simply great controls may get all upon a roads. Dramatically past women result on a cards. Warm principles cannot trap; at least secondary stan|8.95|6.80|3001001|amalgexporti #1|1|newborn|3|Children|209|n stbarable|extra large|3254saddle9518869903|turquoise|Ton|Unknown|86|eseantipriese| +4355|AAAAAAAACABBAAAA|1999-10-28|2001-10-26|Simply great controls may get all upon a roads. Dramatically past women result on a cards. Warm principles cannot trap; at least secondary stan|4.25|6.80|1003002|exportiamalg #2|1|maternity|1|Women|209|n stbarable|small|3254saddle9518869903|spring|Tsp|Unknown|22|antiantipriese| +4356|AAAAAAAACABBAAAA|2001-10-27||Sexual, local routes close whole, other opportunities. Affairs shall see especially init|7.25|6.80|5004001|edu packscholar #1|1|classical|5|Music|934|eseprin st|N/A|3254saddle9518869903|hot|Dram|Unknown|15|callyantipriese| +4357|AAAAAAAAFABBAAAA|1997-10-27||Easy times drive certainly poor officers. Common, previous bones retrieve however investigations; scottish men should provide. Possible methods must authorise. Mo|5.58|3.18|5004002|edu packscholar #2|4|classical|5|Music|18|eingought|N/A|96327713sandy8709746|rose|Case|Unknown|33|ationantipriese| +4358|AAAAAAAAGABBAAAA|1997-10-27|2000-10-26|Lives may convey fair, popular industries; sure main records will take please with a restrictions. Illegally tough rights might not return never at the waters. Sensitive standards could take completel|2.68|1.42|9004003|edu packmaxi #3|4|entertainments|9|Books|110|baroughtought|N/A|072khaki216749817666|rosy|Carton|Unknown|15|eingantipriese| +4359|AAAAAAAAGABBAAAA|2000-10-27||Lives may convey fair, popular industries; sure main records will take please with a restrictions. Illegally tough rights might not return never at the waters. Sensitive standards could take completel|49.59|40.66|9004003|importoscholar #2|4|country|5|Music|363|baroughtought|N/A|072khaki216749817666|salmon|Tbl|Unknown|14|n stantipriese| +4360|AAAAAAAAIABBAAAA|1997-10-27|1999-10-27|Pregnant, secondary years choose about above courses. Angry cells should not vary ever. American, normal details would aim swiss corners. Traditionally classical balls might not face never only|9.37|7.02|4001001|amalgedu pack #1|1|womens|4|Shoes|690|barn stcally|petite|9422017130431steel82|turquoise|Ounce|Unknown|18|barcallypriese| +4361|AAAAAAAAIABBAAAA|1999-10-28|2001-10-26|Pregnant, secondary years choose about above courses. Angry cells should not vary ever. American, normal details would aim swiss corners. Traditionally classical balls might not face never only|1.08|0.34|5001002|amalgscholar #2|1|rock|5|Music|690|barn stcally|N/A|9422017130431steel82|peru|Pallet|Unknown|23|oughtcallypriese| +4362|AAAAAAAAIABBAAAA|2001-10-27||Pregnant, secondary years choose about above courses. Angry cells should not vary ever. American, normal details would aim swiss corners. Traditionally classical balls might not face never only|5.25|0.34|5001002|amalgedu pack #1|1|womens|4|Shoes|319|n stoughtpri|petite|00242029yellow874107|pale|Carton|Unknown|59|ablecallypriese| +4363|AAAAAAAALABBAAAA|1997-10-27||Massive funds shall not learn very tests. Industrial, flat things would not speak because of the girls. Only bloody cells will not devise men. National, o|3.47|1.21|10009012|maxiunivamalg #12|9|televisions|10|Electronics|963|pricallyn st|N/A|9970turquoise7718376|rosy|Unknown|Unknown|20|pricallypriese| +4364|AAAAAAAAMABBAAAA|1997-10-27|2000-10-26|Light authorities melt therefore so real associations. Fortunes should loosen most only royal |7.08|3.46|7015005|scholarnameless #5|15|tables|7|Home|504|esebaranti|N/A|2chiffon372676522813|light|Ounce|Unknown|59|esecallypriese| +4365|AAAAAAAAMABBAAAA|2000-10-27||Labour, simple groups apply then; brilliant neighbours ought to respond at the customers. White, public powers could mak|4.46|3.46|3003002|exportiexporti #2|3|toddlers|3|Children|153|priantiought|petite|908803542peru5710342|beige|Tsp|Unknown|28|anticallypriese| +4366|AAAAAAAAOABBAAAA|1997-10-27|1999-10-27|Outdoor, increasing regions shall buy more interesting friends. Children find instead monetary, regular lessons. Popular, various games emphasise all likely women. Arguably single i|0.76|0.40|3002001|importoexporti #1|2|infants|3|Children|187|ationeingought|extra large|84seashell7106217131|cornsilk|Box|Unknown|17|callycallypriese| +4367|AAAAAAAAOABBAAAA|1999-10-28|2001-10-26|As a whole full months should initiate as good as knowingly immense reasons. Views say eyes; keen, equivalent houses let clearly. Ne|2.63|0.40|3002001|edu packunivamalg #12|2|sports|9|Books|97|ationn st|N/A|2saddle3270436400260|orange|Carton|Unknown|59|ationcallypriese| +4368|AAAAAAAAOABBAAAA|2001-10-27||As a whole full months should initiate as good as knowingly immense reasons. Views say eyes; keen, equivalent houses let clearly. Ne|8.53|7.16|1002001|importoamalg #1|2|fragrances|1|Women|258|eingantiable|medium|2saddle3270436400260|violet|Ounce|Unknown|17|eingcallypriese| +4369|AAAAAAAABBBBAAAA|1997-10-27||Much religious groups add groups. |0.97|0.84|5004002|edu packscholar #2|4|classical|5|Music|20|barable|N/A|253205824949coral470|tomato|Box|Unknown|24|n stcallypriese| +4370|AAAAAAAACBBBAAAA|1997-10-27|2000-10-26|Arms will pay scottish, wrong users. Already accurate factors would take over abl|1.87|1.17|7009001|maxibrand #1|9|mattresses|7|Home|453|priantiese|N/A|35710gainsboro075658|chiffon|Gram|Unknown|41|barationpriese| +4371|AAAAAAAACBBBAAAA|2000-10-27||Arms will pay scottish, wrong users. Already accurate factors would take over abl|1.37|1.17|7009001|amalgamalg #2|9|dresses|1|Women|453|priantiese|petite|violet83002072927312|purple|Lb|Unknown|67|oughtationpriese| +4372|AAAAAAAAEBBBAAAA|1997-10-27|1999-10-27|Essential, successive lives show already; writers create any lon|4.49|1.84|6008005|namelesscorp #5|8|mens watch|6|Jewelry|285|antieingable|N/A|12612spring212368836|violet|Cup|Unknown|66|ableationpriese| +4373|AAAAAAAAEBBBAAAA|1999-10-28|2001-10-26|Comments discriminate massive, central questions. Times should survive at a students. American, political terms might satisfy sti|0.65|0.43|6008005|amalgimporto #2|1|accessories|2|Men|295|antin stable|medium|229pink1634463329962|goldenrod|Dram|Unknown|18|priationpriese| +4374|AAAAAAAAEBBBAAAA|2001-10-27||Comments discriminate massive, central questions. Times should survive at a students. American, political terms might satisfy sti|3.81|0.43|9008007|namelessmaxi #7|1|romance|9|Books|656|antin stable|N/A|229pink1634463329962|plum|Gross|Unknown|14|eseationpriese| +4375|AAAAAAAAHBBBAAAA|1997-10-27||Perhaps major females make increasingly new trousers. Terms would stay other girls. Precisely differe|3.81|1.63|7003006|exportibrand #6|3|kids|7|Home|307|ationbarpri|N/A|81105125puff91749683|gainsboro|Tbl|Unknown|97|antiationpriese| +4376|AAAAAAAAIBBBAAAA|1997-10-27|2000-10-26|Services pick thus|6.87|2.81|4003001|exportiedu pack #1|3|kids|4|Shoes|92|ablen st|petite|40366tan139955656237|olive|N/A|Unknown|41|callyationpriese| +4377|AAAAAAAAIBBBAAAA|2000-10-27||Still successful sales look old, silent words; general, high sources might help british, po|3.78|1.62|9005010|scholarmaxi #10|5|history|9|Books|92|ablen st|N/A|tan31476920650415927|sky|Bunch|Unknown|18|ationationpriese| +4378|AAAAAAAAKBBBAAAA|1997-10-27|1999-10-27|Extensive, thin records respect more; dimensi|4.78|3.91|10005010|scholarunivamalg #10|5|karoke|10|Electronics|165|anticallyought|N/A|57513524418272sandy8|ivory|Tsp|Unknown|71|eingationpriese| +4379|AAAAAAAAKBBBAAAA|1999-10-28|2001-10-26|Main children tell. Recently white plans offer often in a ambitions. Barely mere arguments might not use still grand difficulties; valid exampl|4.99|2.69|10005010|importoedu pack #2|5|mens|4|Shoes|505|anticallyought|small|57513524418272sandy8|turquoise|Pallet|Unknown|12|n stationpriese| +4380|AAAAAAAAKBBBAAAA|2001-10-27||Main children tell. Recently white plans offer often in a ambitions. Barely mere arguments might not use still grand difficulties; valid exampl|36.74|26.45|7001001|amalgbrand #1|1|bathroom|7|Home|95|antin st|N/A|57513524418272sandy8|red|Bundle|Unknown|19|bareingpriese| +4381|AAAAAAAANBBBAAAA|1997-10-27||Used shots see so organic children; parliamentary friends used to get already as holy years. New, super profits could wonder now top projects. Clear, basic cars woul|2.90|1.39|1003002|exportiamalg #2|3|maternity|1|Women|30|barpri|medium|6953716pink095592023|sandy|Tbl|Unknown|39|oughteingpriese| +4382|AAAAAAAAOBBBAAAA|1997-10-27|2000-10-26|Main meetings can burst certain, parliamentary heroes. Much happy journals learn|2.61|1.72|9011003|amalgunivamalg #3|11|cooking|9|Books|91|oughtn st|N/A|7215183pink326373094|lime|Gross|Unknown|30|ableeingpriese| +4383|AAAAAAAAOBBBAAAA|2000-10-27||Main meetings can burst certain, parliamentary heroes. Much happy journals learn|8.36|5.60|9011003|edu packimporto #2|4|sports-apparel|2|Men|143|oughtn st|petite|756187spring94132499|wheat|Dozen|Unknown|37|prieingpriese| +4384|AAAAAAAAACBBAAAA|1997-10-27|1999-10-27|Severe characteristics enter top, individual teachers. Elderly homes may speak relations. Here senior others get determined, prime sizes. Palestinian feelings work today|3.20|1.47|8010005|univmaxi #5|10|pools|8|Sports|527|ationableanti|N/A|3267543tan2651192598|metallic|Each|Unknown|21|eseeingpriese| +4385|AAAAAAAAACBBAAAA|1999-10-28|2001-10-26|Strategic, english cars can leave social women. Royal men escape already to the years; irish, ro|4.71|1.47|8010005|exportiamalgamalg #17|13|stereo|10|Electronics|375|antiationpri|N/A|3267543tan2651192598|pale|Case|Unknown|14|antieingpriese| +4386|AAAAAAAAACBBAAAA|2001-10-27||Strategic, english cars can leave social women. Royal men escape already to the years; irish, ro|5.34|1.47|6011003|amalgbrand #3|13|semi-precious|6|Jewelry|375|antiationpri|N/A|5287915602880royal27|ivory|Oz|Unknown|23|callyeingpriese| +4387|AAAAAAAADCBBAAAA|1997-10-27||Critical, necessary decisions might get in short. Teams get always in the metres. Years would keep quite warm years. Questions must try right, weak days. Private, apparent skills know sole, terrible |3.40|2.92|5003002|exportischolar #2|3|pop|5|Music|252|ableantiable|N/A|tan88514591621738477|magenta|Oz|Unknown|52|ationeingpriese| +4388|AAAAAAAAECBBAAAA|1997-10-27|2000-10-26|Customs can say still contracts; arrangements organise so. Good, formal patients face even. Collea|0.63|0.28|2002001|importoimporto #1|2|shirts|2|Men|390|barn stpri|extra large|2131203601556tan9654|forest|Case|Unknown|6|eingeingpriese| +4389|AAAAAAAAECBBAAAA|2000-10-27||Senior children would face different, public areas. Almost personal activities get probably experiences. Mechanical, wild sa|7.90|0.28|8010010|univmaxi #10|2|pools|8|Sports|390|barn stpri|N/A|2131203601556tan9654|turquoise|Oz|Unknown|44|n steingpriese| +4390|AAAAAAAAGCBBAAAA|1997-10-27|1999-10-27|Differences no just widely appropriate kids. Groups end books. Therefore local recommendations can like early buildings. Pupils will lend on a clothes. Small shapes may indicate relatively; ag|7.07|2.96|3002001|importoexporti #1|2|infants|3|Children|226|callyableable|large|29208144red451184257|saddle|Pallet|Unknown|10|barn stpriese| +4391|AAAAAAAAGCBBAAAA|1999-10-28|2001-10-26|Differences no just widely appropriate kids. Groups end books. Therefore local recommendations can like early buildings. Pupils will lend on a clothes. Small shapes may indicate relatively; ag|5.13|1.79|3002001|corpunivamalg #6|6|musical|10|Electronics|395|callyableable|N/A|3460513974489slate95|orange|Oz|Unknown|74|oughtn stpriese| +4392|AAAAAAAAGCBBAAAA|2001-10-27||Spiritual miles would not quell always less urban profess|0.65|1.79|3002001|edu packamalg #1|6|swimwear|1|Women|181|oughteingought|large|3460513974489slate95|midnight|Gram|Unknown|19|ablen stpriese| +4393|AAAAAAAAJCBBAAAA|1997-10-27||Video-taped, unchanged days assess thus organisms. Funny, ready feet will read anyway dramatically high contracts. For certain overseas eggs will tal|37.14|17.08|1004002|edu packamalg #2|4|swimwear|1|Women|231|oughtpriable|medium|57585sienna024168653|chocolate|Unknown|Unknown|44|prin stpriese| +4394|AAAAAAAAKCBBAAAA|1997-10-27|2000-10-26|Families shall tell together under a problems; windows say yet clear, eligible responses. There good millions will pay later on the pieces; never positive members go |7.83|3.75|5004001|edu packscholar #1|4|classical|5|Music|595|antin stanti|N/A|53709202seashell2852|rosy|Carton|Unknown|8|esen stpriese| +4395|AAAAAAAAKCBBAAAA|2000-10-27||Males might ring particularly in a courts; english,|0.45|3.75|10011010|amalgamalgamalg #10|4|disk drives|10|Electronics|362|ablecallypri|N/A|21534971puff77066423|pale|Ounce|Unknown|98|antin stpriese| +4396|AAAAAAAAMCBBAAAA|1997-10-27|1999-10-27|Major, musical increases sort home. Unpleasant, tall effects release with a railways. P|4.15|2.28|4002001|importoedu pack #1|2|mens|4|Shoes|997|ationn stn st|petite|034506982moccasin404|orange|Ounce|Unknown|81|callyn stpriese| +4397|AAAAAAAAMCBBAAAA|1999-10-28|2001-10-26|Please major students could shift fully in a councils; still bright eyes should change. Better fixed earnings must find fiercely individual minut|2.14|2.28|7013008|exportinameless #8|2|wallpaper|7|Home|65|antically|N/A|7476grey547335139904|red|Bundle|Unknown|49|ationn stpriese| +4398|AAAAAAAAMCBBAAAA|2001-10-27||Please major students could shift fully in a councils; still bright eyes should change. Better fixed earnings must find fiercely individual minut|0.85|0.58|9016007|corpunivamalg #7|16|mystery|9|Books|65|antically|N/A|326411322plum3947521|sky|Pallet|Unknown|4|eingn stpriese| +4399|AAAAAAAAPCBBAAAA|1997-10-27||Social, resulting branches mi|7.52|3.98|8013010|exportimaxi #10|13|sailing|8|Sports|898|eingn steing|N/A|85174737070khaki8595|khaki|Tbl|Unknown|95|n stn stpriese| +4400|AAAAAAAAADBBAAAA|1997-10-27|2000-10-26|Supreme, hard problems might gain. Dull, constant babies will make deep other, poor futures. Good, late proportions root late joint men. Demands may start pretty. Patterns|0.79|0.63|2004001|edu packimporto #1|4|sports-apparel|2|Men|661|oughtcallycally|medium|sandy224019776855221|pale|Dram|Unknown|23|barbareseese| +4401|AAAAAAAAADBBAAAA|2000-10-27||Supreme, hard problems might gain. Dull, constant babies will make deep other, poor futures. Good, late proportions root late joint men. Demands may start pretty. Patterns|6.76|3.78|9013010|exportiunivamalg #10|4|self-help|9|Books|661|oughtcallycally|N/A|sandy224019776855221|tan|N/A|Unknown|100|oughtbareseese| +4402|AAAAAAAACDBBAAAA|1997-10-27|1999-10-27|Commercial, radical tenants ought to go once on a methods. Upper|0.51|0.38|8009003|maxinameless #3|9|optics|8|Sports|549|n steseanti|N/A|sky52392150469113923|sandy|Ounce|Unknown|58|ablebareseese| +4403|AAAAAAAACDBBAAAA|1999-10-28|2001-10-26|Products shall examine by a experts. Local, particular questions may suspect exceptions. Relations make legal, other activities; relative, ac|2.58|1.00|8009003|edu packexporti #2|9|school-uniforms|3|Children|401|oughtbarese|medium|sky52392150469113923|peach|Tbl|Unknown|17|pribareseese| +4404|AAAAAAAACDBBAAAA|2001-10-27||Products shall examine by a experts. Local, particular questions may suspect exceptions. Relations make legal, other activities; relative, ac|5.92|1.00|1004001|edu packamalg #1|4|swimwear|1|Women|401|oughtbarese|large|sky52392150469113923|yellow|Dozen|Unknown|57|esebareseese| +4405|AAAAAAAAFDBBAAAA|1997-10-27||Doctors act easy. Topics could exist contracts. Cameras lig|2.34|1.61|10002003|importounivamalg #3|2|camcorders|10|Electronics|536|callyprianti|N/A|9white43180759271586|metallic|Gram|Unknown|57|antibareseese| +4406|AAAAAAAAGDBBAAAA|1997-10-27|2000-10-26|Wrongly single restrictions run so whole, international problems. Often possible equations may clarify common, bare cattle. Important profits set for example female, grey paintings. Existing|0.38|0.21|6015007|scholarbrand #7|15|custom|6|Jewelry|769|n stcallyation|N/A|428281385703lawn8277|rose|Bundle|Unknown|66|callybareseese| +4407|AAAAAAAAGDBBAAAA|2000-10-27||So fascin|3.99|0.21|5002002|importoscholar #2|15|country|5|Music|769|n stcallyation|N/A|428281385703lawn8277|moccasin|Ton|Unknown|17|ationbareseese| +4408|AAAAAAAAIDBBAAAA|1997-10-27|1999-10-27|Corresponding, labour ch|2.69|2.04|1003001|exportiamalg #1|3|maternity|1|Women|537|ationprianti|large|2453911278685peach24|gainsboro|Gross|Unknown|19|eingbareseese| +4409|AAAAAAAAIDBBAAAA|1999-10-28|2001-10-26|Corresponding, labour ch|4.33|2.90|1003001|exportiunivamalg #6|3|self-help|9|Books|434|esepriese|N/A|2453911278685peach24|black|Unknown|Unknown|10|n stbareseese| +4410|AAAAAAAAIDBBAAAA|2001-10-27||Corresponding, labour ch|2.19|0.65|1003001|scholaramalgamalg #8|3|portable|10|Electronics|493|esepriese|N/A|41927590247wheat9427|royal|Unknown|Unknown|75|baroughteseese| +4411|AAAAAAAALDBBAAAA|1997-10-27||Mines should talk outside trees. Regular eyes encourage with an victims. Civil functions try actions. Movies fit secretly for a regions. Whole, imperial customs forget|7.44|4.16|9003008|exportimaxi #8|3|computers|9|Books|174|eseationought|N/A|6408408603233red4555|lavender|Tbl|Unknown|17|oughtoughteseese| +4412|AAAAAAAAMDBBAAAA|1997-10-27|2000-10-26|Completely upper clients achieve western fees. Small areas must get traditions. Folk can deal however|1.28|0.69|8009003|maxinameless #3|9|optics|8|Sports|214|eseoughtable|N/A|puff6973440483992284|spring|Ounce|Unknown|33|ableoughteseese| +4413|AAAAAAAAMDBBAAAA|2000-10-27||Black moments stop more subsequent circumstances; major, good minutes take finally fine, necessary sets. Particularly large changes stop just on a topics. Skills might shelter least finally |3.35|0.69|1003002|exportiamalg #2|3|maternity|1|Women|214|eseoughtable|medium|puff6973440483992284|gainsboro|Cup|Unknown|14|prioughteseese| +4414|AAAAAAAAODBBAAAA|1997-10-27|1999-10-27|Unaware results cannot bring. Flowers go immediately other patients. Heavy, sur|1.55|0.55|5003001|exportischolar #1|3|pop|5|Music|431|oughtpriese|N/A|4589smoke23031301182|royal|Pound|Unknown|31|eseoughteseese| +4415|AAAAAAAAODBBAAAA|1999-10-28|2001-10-26|Large, human others must get quite free, s|7.45|0.55|3003002|exportiexporti #2|3|toddlers|3|Children|304|oughtpriese|large|6040571134slate97788|rosy|Bunch|Unknown|20|antioughteseese| +4416|AAAAAAAAODBBAAAA|2001-10-27||Large, human others must get quite free, s|1.91|0.55|3003002|edu packexporti #1|3|school-uniforms|3|Children|304|esebarpri|large|48089993thistle36512|pale|Pound|Unknown|80|callyoughteseese| +4417|AAAAAAAABEBBAAAA|1997-10-27||Moreover new forces shall waive most honest, hard boundaries. Foreign, social facts may not head under values; old concentrations must not make from a numbers. Difficult victims pray open, female q|3.23|1.22|10015015|scholaramalgamalg #15|15|portable|10|Electronics|22|ableable|N/A|27634554644856sky305|salmon|Dozen|Unknown|3|ationoughteseese| +4418|AAAAAAAACEBBAAAA|1997-10-27|2000-10-26|Able villages enforce present holes; users will win increasingly wrong forces. Skilled, economic objectives used to notice. International, new theories go only however economic|73.95|35.49|9016003|corpunivamalg #3|16|mystery|9|Books|290|barn stable|N/A|728567578royal808446|snow|Dozen|Unknown|3|eingoughteseese| +4419|AAAAAAAACEBBAAAA|2000-10-27||Sufficient mothers prevent that for the eyes; painful purposes would catch surprisingly. Substantially different troops shall tell correct aspects. High weeks s|22.39|13.43|9016003|edu packamalgamalg #17|14|automotive|10|Electronics|290|barn stable|N/A|16309633peru68299682|lavender|Gross|Unknown|83|n stoughteseese| +4420|AAAAAAAAEEBBAAAA|1997-10-27|1999-10-27|Noble, general d|9.34|6.16|7012001|importonameless #1|12|paint|7|Home|81|oughteing|N/A|844metallic492214835|yellow|Oz|Unknown|58|barableeseese| +4421|AAAAAAAAEEBBAAAA|1999-10-28|2001-10-26|Magnetic quarters could say inches. Difficult, great issues will conform indeed bloody, alone questions. Linguistic, eastern fields will not go generally. Forests require. Wide, new books|7.16|6.16|7012001|amalgamalgamalg #2|11|disk drives|10|Electronics|277|oughteing|N/A|229430658850royal247|rose|Case|Unknown|1|oughtableeseese| +4422|AAAAAAAAEEBBAAAA|2001-10-27||Magnetic quarters could say inches. Difficult, great issues will conform indeed bloody, alone questions. Linguistic, eastern fields will not go generally. Forests require. Wide, new books|4.25|1.65|5002001|importoscholar #1|2|country|5|Music|277|ationationable|N/A|229430658850royal247|tomato|Unknown|Unknown|6|ableableeseese| +4423|AAAAAAAAHEBBAAAA|1997-10-27||Long, satisfactory goods take certainly. Really valid servants may not know still old, strange women. Apparent, rid|5.64|3.04|5003002|exportischolar #2|3|pop|5|Music|50|baranti|N/A|0white89845145209495|saddle|Unknown|Unknown|8|priableeseese| +4424|AAAAAAAAIEBBAAAA|1997-10-27|2000-10-26|Subtle, inc sorts wear costs; briefly importa|1.60|0.59|8012005|importomaxi #5|12|guns|8|Sports|372|ableationpri|N/A|2sky9941409005438909|royal|Cup|Unknown|25|eseableeseese| +4425|AAAAAAAAIEBBAAAA|2000-10-27||Subtle, inc sorts wear costs; briefly importa|4.11|0.59|8012005|exportiexporti #2|12|toddlers|3|Children|372|ableationpri|small|10353183261navajo547|turquoise|Ounce|Unknown|2|antiableeseese| +4426|AAAAAAAAKEBBAAAA|1997-10-27|1999-10-27|Internal, pleasant statements might plan again better other d|1.59|0.68|3002001|importoexporti #1|2|infants|3|Children|248|eingeseable|extra large|16189gainsboro172560|rose|Each|Unknown|49|callyableeseese| +4427|AAAAAAAAKEBBAAAA|1999-10-28|2001-10-26|Internal, pleasant statements might plan again better other d|0.10|0.05|3002001|amalgscholar #2|1|rock|5|Music|248|eingeseable|N/A|16189gainsboro172560|violet|Dram|Unknown|23|ationableeseese| +4428|AAAAAAAAKEBBAAAA|2001-10-27||Internal, pleasant statements might plan again better other d|8.49|0.05|3004001|edu packexporti #1|1|school-uniforms|3|Children|248|eingeseable|large|16189gainsboro172560|rose|Pallet|Unknown|23|eingableeseese| +4429|AAAAAAAANEBBAAAA|1997-10-27||Major, physical parents would keep the|5.58|3.79|2001002|amalgimporto #2|1|accessories|2|Men|148|eingeseought|medium|058845873395353peru1|pale|Bunch|Unknown|23|n stableeseese| +4430|AAAAAAAAOEBBAAAA|1997-10-27|2000-10-26|Differences last long audiences. Deep, i|6.90|5.31|6003007|exporticorp #7|3|gold|6|Jewelry|418|eingoughtese|N/A|40802linen0775520484|ivory|Dozen|Unknown|10|barprieseese| +4431|AAAAAAAAOEBBAAAA|2000-10-27||Differences last long audiences. Deep, i|3.82|2.02|10013012|exportiamalgamalg #12|3|stereo|10|Electronics|418|eingoughtese|N/A|0721488605910875sky9|powder|Tsp|Unknown|9|oughtprieseese| +4432|AAAAAAAAAFBBAAAA|1997-10-27|1999-10-27|Different mechanisms can|5.35|2.62|6005001|scholarcorp #1|5|earings|6|Jewelry|251|oughtantiable|N/A|126maroon82705064193|burlywood|Gross|Unknown|7|ableprieseese| +4433|AAAAAAAAAFBBAAAA|1999-10-28|2001-10-26|Cats win fiercely urban rises. Most central passages make feelings. Other, rare services know now sometimes signif|2.08|1.70|2001002|amalgimporto #2|1|accessories|2|Men|251|oughtantiable|medium|126maroon82705064193|peru|N/A|Unknown|19|priprieseese| +4434|AAAAAAAAAFBBAAAA|2001-10-27||Delicious teeth sound either below black weeks. Positive men let just total cities. Keenly possible vegetables get aware, black shares. Military, profitable animals should recoup beautif|5.48|2.95|2001002|corpamalgamalg #15|16|wireless|10|Electronics|284|eseeingable|N/A|2smoke47511355205697|rose|Oz|Unknown|51|eseprieseese| +4435|AAAAAAAADFBBAAAA|1997-10-27||Social, popular leaves could not ca|2.61|2.21|9004008|edu packmaxi #8|4|entertainments|9|Books|926|callyablen st|N/A|07yellow770162300099|thistle|Oz|Unknown|90|antiprieseese| +4436|AAAAAAAAEFBBAAAA|1997-10-27|2000-10-26|Units might not tell probably properly english managers. Nevertheless royal difficulties will go in a machines; big, large families remain again immediate, new months; old, important typ|8.59|5.75|3003001|exportiexporti #1|3|toddlers|3|Children|148|eingeseought|petite|slate270720585803541|papaya|Box|Unknown|11|callyprieseese| +4437|AAAAAAAAEFBBAAAA|2000-10-27||Units might not tell probably properly english managers. Nevertheless royal difficulties will go in a machines; big, large families remain again immediate, new months; old, important typ|12.40|9.79|7007008|brandbrand #8|7|decor|7|Home|148|eingeseought|N/A|502058goldenrod89695|pale|Bunch|Unknown|12|ationprieseese| +4438|AAAAAAAAGFBBAAAA|1997-10-27|1999-10-27|Conservative lives used to close also direct notions. More human structure|0.73|0.62|5004001|edu packscholar #1|4|classical|5|Music|403|pribarese|N/A|4425561354492peach79|violet|Ounce|Unknown|51|eingprieseese| +4439|AAAAAAAAGFBBAAAA|1999-10-28|2001-10-26|German years go then months. Generally new parents shall try anyway always little proceedings. Comparable, like results get in a difficulties. Good terms might remember only residential|9.07|5.80|10014003|edu packamalgamalg #3|4|automotive|10|Electronics|341|pribarese|N/A|4425561354492peach79|linen|Case|Unknown|20|n stprieseese| +4440|AAAAAAAAGFBBAAAA|2001-10-27||German years go then months. Generally new parents shall try anyway always little proceedings. Comparable, like results get in a difficulties. Good terms might remember only residential|3.92|5.80|10014003|amalgnameless #1|1|athletic shoes|8|Sports|341|oughtesepri|N/A|890567422sandy547035|wheat|Tbl|Unknown|54|bareseeseese| +4441|AAAAAAAAJFBBAAAA|1997-10-27||Political, common bones see competent, economic parents. Fairly wide students might not organise rapidly with a magistrates. Great, open trees should infer originally once other homes; actuall|9.08|6.99|10014005|edu packamalgamalg #5|14|automotive|10|Electronics|655|antiantically|N/A|chartreuse4437008117|tomato|Tbl|Unknown|55|oughteseeseese| +4442|AAAAAAAAKFBBAAAA|1997-10-27|2000-10-26|Pre|4.59|3.94|5004001|edu packscholar #1|4|classical|5|Music|52|ableanti|N/A|234083tan42553328002|turquoise|Gram|Unknown|32|ableeseeseese| +4443|AAAAAAAAKFBBAAAA|2000-10-27||All national hours see quite in spite of a agencies. Old individuals see often suddenly large women; other, other powers could repair live over the |4.95|3.94|5004001|amalgamalg #2|4|dresses|1|Women|423|priableese|extra large|234083tan42553328002|powder|Pallet|Unknown|1|prieseeseese| +4444|AAAAAAAAMFBBAAAA|1997-10-27|1999-10-27|Social standards feel then|4.71|3.39|3004001|edu packexporti #1|4|school-uniforms|3|Children|598|eingn stanti|medium|95053537899815white3|pale|Gross|Unknown|9|eseeseeseese| +4445|AAAAAAAAMFBBAAAA|1999-10-28|2001-10-26|Social standards feel then|1.66|0.49|3001002|amalgexporti #2|1|newborn|3|Children|468|eingcallyese|large|358307676thistle0355|cornflower|Dozen|Unknown|60|antieseeseese| +4446|AAAAAAAAMFBBAAAA|2001-10-27||Social standards feel then|3.47|1.80|3001002|exportiimporto #1|1|pants|2|Men|468|eingcallyese|small|57936531sky112544425|azure|Gross|Unknown|47|callyeseeseese| +4447|AAAAAAAAPFBBAAAA|1997-10-27||More open effects kill |1.22|0.62|10005011|scholarunivamalg #11|5|karoke|10|Electronics|136|callypriought|N/A|8119salmon1960060707|tan|N/A|Unknown|30|ationeseeseese| +4448|AAAAAAAAAGBBAAAA|1997-10-27|2000-10-26|Special fingers might pay sorry workers. National, polite patients may like in a de|9.08|7.17|10008012|namelessunivamalg #12|8|scanners|10|Electronics|168|eingcallyought|N/A|641sandy996293128037|chartreuse|Box|Unknown|63|eingeseeseese| +4449|AAAAAAAAAGBBAAAA|2000-10-27||Top performances shall not think almost. Apparent, new rights may open carefully potential ages. Really national copies would pose around notes. Deeply big words set ethnic, ed|9.12|7.02|10008012|edu packscholar #2|4|classical|5|Music|168|eingcallyought|N/A|642goldenrod89926732|pale|Unknown|Unknown|57|n steseeseese| +4450|AAAAAAAACGBBAAAA|1997-10-27|1999-10-27|Above warm issues assume in particular from the events. Sites would not come women. Large controls go grim, sudden men. Infor|9.52|4.37|8004001|edu packnameless #1|4|camping|8|Sports|620|barablecally|N/A|704684051magenta9335|turquoise|Gross|Unknown|3|barantieseese| +4451|AAAAAAAACGBBAAAA|1999-10-28|2001-10-26|Female, angry users help hardly operations. Potentially married troubles must make all in a laws. Inwards relevant views sh|2.18|1.46|8004001|importounivamalg #15|4|camcorders|10|Electronics|134|barablecally|N/A|704684051magenta9335|purple|Each|Unknown|53|oughtantieseese| +4452|AAAAAAAACGBBAAAA|2001-10-27||Satisfactory, inten|1.31|1.46|8004001|amalgimporto #1|1|accessories|2|Men|232|barablecally|medium|purple17366650988777|salmon|Tbl|Unknown|2|ableantieseese| +4453|AAAAAAAAFGBBAAAA|1997-10-27||So inc clients may tell as. Mothers could point points. Increasing, alone gifts |1.23|0.60|7016004|corpnameless #4|16|furniture|7|Home|570|barationanti|N/A|76458burlywood654950|dim|Dram|Unknown|54|priantieseese| +4454|AAAAAAAAGGBBAAAA|1997-10-27|2000-10-26|G|6.69|5.01|10013001|exportiamalgamalg #1|13|stereo|10|Electronics|475|antiationese|N/A|1116136yellow0233919|puff|N/A|Unknown|26|eseantieseese| +4455|AAAAAAAAGGBBAAAA|2000-10-27||Patients notice h|66.02|23.10|10013001|importoscholar #2|2|country|5|Music|475|antiationese|N/A|1116136yellow0233919|steel|Ounce|Unknown|14|antiantieseese| +4456|AAAAAAAAIGBBAAAA|1997-10-27|1999-10-27|Patients must sanction however examples. Electronic, executive patients may indicate at least american studies. Children might not give worldwide administ|61.27|41.05|9016005|corpunivamalg #5|16|mystery|9|Books|311|oughtoughtpri|N/A|7486tan7037408371769|rose|Gross|Unknown|22|callyantieseese| +4457|AAAAAAAAIGBBAAAA|1999-10-28|2001-10-26|Shots would go from a posts; then wild|1.23|41.05|9016005|corpmaxi #6|16|parenting|9|Books|311|oughtoughtpri|N/A|7486tan7037408371769|puff|Dozen|Unknown|32|ationantieseese| +4458|AAAAAAAAIGBBAAAA|2001-10-27||Shots would go from a posts; then wild|0.32|41.05|9016005|importoamalgamalg #5|16|monitors|10|Electronics|60|barcally|N/A|7486tan7037408371769|misty|Pallet|Unknown|31|eingantieseese| +4459|AAAAAAAALGBBAAAA|1997-10-27||Modern records retain about there civil plans. Social bodies survive. Great, living losses bother late, coherent others. About british sports ought to use cautiously from |1.94|1.53|9002008|importomaxi #8|2|business|9|Books|59|n stanti|N/A|3603white46824753678|thistle|Dram|Unknown|30|n stantieseese| +4460|AAAAAAAAMGBBAAAA|1997-10-27|2000-10-26|Inner, encouraging features should sue here to a terms. Patients will seem all slight members. Complex banks take apparently games. Able, irish patients used|7.27|2.61|9015009|scholarunivamalg #9|15|fiction|9|Books|244|eseeseable|N/A|238dim46903639527198|puff|Bunch|Unknown|16|barcallyeseese| +4461|AAAAAAAAMGBBAAAA|2000-10-27||Inner, encouraging features should sue here to a terms. Patients will seem all slight members. Complex banks take apparently games. Able, irish patients used|1.97|0.80|7012002|importonameless #2|12|paint|7|Home|124|eseeseable|N/A|0960mint328534197067|royal|Carton|Unknown|32|oughtcallyeseese| +4462|AAAAAAAAOGBBAAAA|1997-10-27|1999-10-27|Common, big performances look often directly dead offices; inst|0.09|0.04|10004009|edu packunivamalg #9|4|audio|10|Electronics|328|eingablepri|N/A|4742575sienna1621954|smoke|Dozen|Unknown|30|ablecallyeseese| +4463|AAAAAAAAOGBBAAAA|1999-10-28|2001-10-26|Symbolic, able operations should meet primary conditions. So ne|0.73|0.04|2001002|amalgimporto #2|1|accessories|2|Men|61|eingablepri|medium|4742575sienna1621954|tan|Box|Unknown|47|pricallyeseese| +4464|AAAAAAAAOGBBAAAA|2001-10-27||Old, official citizens may not help about changes. However good attitudes cope. Truly technical exchanges face well special, psychological years. Central, po|5.58|0.04|2004001|edu packimporto #1|4|sports-apparel|2|Men|948|eingesen st|large|4742575sienna1621954|mint|Unknown|Unknown|66|esecallyeseese| +4465|AAAAAAAABHBBAAAA|1997-10-27||Divine, physical teachers |9.87|3.94|7014006|edu packnameless #6|14|glassware|7|Home|649|n stesecally|N/A|0rose868484205857368|sandy|Bunch|Unknown|7|anticallyeseese| +4466|AAAAAAAACHBBAAAA|1997-10-27|2000-10-26|Relatively sad accidents happen secondary, other sons; organisatio|3.19|1.02|9009003|maximaxi #3|9|science|9|Books|440|bareseese|N/A|987678914salmon12977|saddle|Bundle|Unknown|69|callycallyeseese| +4467|AAAAAAAACHBBAAAA|2000-10-27||Tiny differences used to trace offenders. Particularly irish parents shout shoes. Less real communicat|1.62|1.03|4003002|exportiedu pack #2|3|kids|4|Shoes|440|bareseese|large|76229376877misty7587|peach|Each|Unknown|50|ationcallyeseese| +4468|AAAAAAAAEHBBAAAA|1997-10-27|1999-10-27|New, guilty reports could supply much right human men. Conventional males would not complain high, entire interests. Quantities provide hom|2.68|2.38|5004001|edu packscholar #1|4|classical|5|Music|149|n steseought|N/A|24317063124pale82676|mint|Carton|Unknown|9|eingcallyeseese| +4469|AAAAAAAAEHBBAAAA|1999-10-28|2001-10-26|New, guilty reports could supply much right human men. Conventional males would not complain high, entire interests. Quantities provide hom|6.66|4.86|5004001|scholarmaxi #6|5|history|9|Books|549|n steseought|N/A|24317063124pale82676|steel|Box|Unknown|21|n stcallyeseese| +4470|AAAAAAAAEHBBAAAA|2001-10-27||Often symbolic results would not submit newspapers. Points must not live again chinese, long modules. Bits would demonstrate so lucky boats. Then othe|0.42|0.15|4003001|exportiedu pack #1|3|kids|4|Shoes|123|n steseought|medium|24317063124pale82676|seashell|Tsp|Unknown|5|barationeseese| +4471|AAAAAAAAHHBBAAAA|1997-10-27||Indians ought to sign nowhere small sides. Overhead theoretical managers breed only weak, dead years. Cases could face fully including a walls. Low,|7.88|2.36|1003002|exportiamalg #2|3|maternity|1|Women|478|eingationese|large|647022white858898876|blanched|Bunch|Unknown|31|oughtationeseese| +4472|AAAAAAAAIHBBAAAA|1997-10-27|2000-10-26|Specialist, other affairs should show more with a commitments. Big, electronic conseq|7.95|6.51|5004001|edu packscholar #1|4|classical|5|Music|576|callyationanti|N/A|48753steel8444544041|peach|Bundle|Unknown|74|ableationeseese| +4473|AAAAAAAAIHBBAAAA|2000-10-27||Specialist, other affairs should show more with a commitments. Big, electronic conseq|0.94|6.51|7011004|amalgnameless #4|11|accent|7|Home|576|callyationanti|N/A|48753steel8444544041|tan|Pound|Unknown|24|priationeseese| +4474|AAAAAAAAKHBBAAAA|1997-10-27|1999-10-27|Specific, slow notes prevent now then oral parts. Serious, curren|3.17|1.68|7005005|scholarbrand #5|5|blinds/shades|7|Home|217|ationoughtable|N/A|971723922184orchid93|turquoise|Tbl|Unknown|32|eseationeseese| +4475|AAAAAAAAKHBBAAAA|1999-10-28|2001-10-26|Busy, available departments ought to weaken etc apparent reports. Smooth, successful others could last somewhere. Days make t|9.20|1.68|4003002|exportiedu pack #2|3|kids|4|Shoes|803|pribareing|petite|891463medium98588655|spring|Unknown|Unknown|72|antiationeseese| +4476|AAAAAAAAKHBBAAAA|2001-10-27||Busy, available departments ought to weaken etc apparent reports. Smooth, successful others could last somewhere. Days make t|2.53|1.06|4003002|maxibrand #7|3|mattresses|7|Home|803|pribareing|N/A|1213thistle162919641|steel|Each|Unknown|5|callyationeseese| +4477|AAAAAAAANHBBAAAA|1997-10-27||Direct, concerned users discuss pretty only great lines. Well late men add oddly electronic, personal refugees. Working, critical hours let a|4.06|2.96|4002002|importoedu pack #2|2|mens|4|Shoes|467|ationcallyese|N/A|073747329317black232|pale|Ton|Unknown|58|ationationeseese| +4478|AAAAAAAAOHBBAAAA|1997-10-27|2000-10-26|Grounds could not advise sophisticated, economic members. Firm roads regard home|7.17|2.86|7008009|namelessbrand #9|8|lighting|7|Home|540|bareseanti|N/A|0710727566330pale460|thistle|Oz|Unknown|8|eingationeseese| +4479|AAAAAAAAOHBBAAAA|2000-10-27||Grounds could not advise sophisticated, economic members. Firm roads regard home|71.84|2.86|7008009|exportiamalg #2|3|maternity|1|Women|309|n stbarpri|large|49navajo929513023562|peru|Box|Unknown|21|n stationeseese| +4480|AAAAAAAAAIBBAAAA|1997-10-27|1999-10-27|Capital, private calls may like. New males shall not make really from a store|36.21|32.58|3001001|amalgexporti #1|1|newborn|3|Children|732|ablepriation|small|yellow74713976186241|honeydew|Case|Unknown|47|bareingeseese| +4481|AAAAAAAAAIBBAAAA|1999-10-28|2001-10-26|Popular, mad proposals could handle outside enough working times. Largely foreign powers believe even members. Just tender shapes find never |2.97|2.49|7001002|amalgbrand #2|1|bathroom|7|Home|732|ablepriation|N/A|yellow74713976186241|black|Oz|Unknown|25|oughteingeseese| +4482|AAAAAAAAAIBBAAAA|2001-10-27||Popular, mad proposals could handle outside enough working times. Largely foreign powers believe even members. Just tender shapes find never |1.52|2.49|7001002|importoexporti #1|2|infants|3|Children|866|callycallyeing|extra large|yellow74713976186241|snow|Dozen|Unknown|9|ableeingeseese| +4483|AAAAAAAADIBBAAAA|1997-10-27||Results shall seem only careful ministers; areas could force about at th|0.74|0.45|5002002|importoscholar #2|2|country|5|Music|323|priablepri|N/A|563455937877white953|slate|Gross|Unknown|33|prieingeseese| +4484|AAAAAAAAEIBBAAAA|1997-10-27|2000-10-26|Long, german references experience numerous, roman sciences. Sales must appoint initial, remaining females; offices reduce at a agencies. Willing accounts might die. Throughout monthly police le|3.92|1.25|3002001|importoexporti #1|2|infants|3|Children|475|antiationese|medium|7459915steel24814237|honeydew|Pound|Unknown|48|eseeingeseese| +4485|AAAAAAAAEIBBAAAA|2000-10-27||Now possi|2.39|1.69|3002001|edu packimporto #2|2|sports-apparel|2|Men|475|antiationese|large|7459915steel24814237|red|Tbl|Unknown|3|antieingeseese| +4486|AAAAAAAAGIBBAAAA|1997-10-27|1999-10-27|Intentionally poor generations can benefit alone. Royal, new regions must see gloomily important, nearby functions. Unemployed, personal efforts reduce completely from a terms|1.53|0.78|6012007|importobrand #7|12|costume|6|Jewelry|116|callyoughtought|N/A|92188465steel8746091|peru|Lb|Unknown|5|callyeingeseese| +4487|AAAAAAAAGIBBAAAA|1999-10-28|2001-10-26|Intentionally poor generations can benefit alone. Royal, new regions must see gloomily important, nearby functions. Unemployed, personal efforts reduce completely from a terms|5.35|3.10|6012007|edu packbrand #8|14|estate|6|Jewelry|116|callyoughtought|N/A|92188465steel8746091|peru|Case|Unknown|4|ationeingeseese| +4488|AAAAAAAAGIBBAAAA|2001-10-27||Never economic cases should suggest significantly free, concerned allegations. As mad me|6.99|3.10|6012007|exportiimporto #1|3|pants|2|Men|116|callyoughtought|petite|92188465steel8746091|peru|Oz|Unknown|19|eingeingeseese| +4489|AAAAAAAAJIBBAAAA|1997-10-27||Also sufficient teachers shall go continually with the margins. Important conditions direct more major rights. Everywhere big pages c|7.99|5.27|6004002|edu packcorp #2|4|bracelets|6|Jewelry|297|ationn stable|N/A|1199tomato5437570838|royal|Cup|Unknown|32|n steingeseese| +4490|AAAAAAAAKIBBAAAA|1997-10-27|2000-10-26|As financial organisations reflect that is main, diplomatic words. Finally german relationships must watch still; lengthy years contact consistently light, equal methods. Current|3.52|1.83|3001001|amalgexporti #1|1|newborn|3|Children|70|baration|small|81563rosy58383971990|green|Oz|Unknown|4|barn steseese| +4491|AAAAAAAAKIBBAAAA|2000-10-27||As financial organisations reflect that is main, diplomatic words. Finally german relationships must watch still; lengthy years contact consistently light, equal methods. Current|4.34|1.90|3001001|scholaramalgamalg #4|1|portable|10|Electronics|70|baration|N/A|921470979297592peru0|sienna|Case|Unknown|26|oughtn steseese| +4492|AAAAAAAAMIBBAAAA|1997-10-27|1999-10-27|Most growing terms move extensive, heavy agents. Also |9.51|8.17|5001001|amalgscholar #1|1|rock|5|Music|442|ableeseese|N/A|8555166816468powder4|steel|Ton|Unknown|86|ablen steseese| +4493|AAAAAAAAMIBBAAAA|1999-10-28|2001-10-26|Fatal rooms can take finally p|16.48|5.76|7012004|importonameless #4|1|paint|7|Home|442|ableeseese|N/A|8555166816468powder4|chartreuse|Oz|Unknown|18|prin steseese| +4494|AAAAAAAAMIBBAAAA|2001-10-27||Chosen parties make reasonably. Feet speak still other, chos|1.10|0.55|3001001|amalgexporti #1|1|newborn|3|Children|442|ableeseese|small|625229steel274705715|purple|Ounce|Unknown|20|esen steseese| +4495|AAAAAAAAPIBBAAAA|1997-10-27||Colours produce heavily to a passengers. Available, unique studies kill popular, true fields. Full, r|6.54|4.64|2003002|exportiimporto #2|3|pants|2|Men|657|ationantically|medium|1426324706780spring7|peach|Ounce|Unknown|37|antin steseese| +4496|AAAAAAAAAJBBAAAA|1997-10-27|2000-10-26|Concepts used to understand even national sides. Exceptional, physical house|34.34|18.54|1003001|exportiamalg #1|3|maternity|1|Women|975|antiationn st|extra large|magenta2387328499592|powder|Gross|Unknown|57|callyn steseese| +4497|AAAAAAAAAJBBAAAA|2000-10-27||Heavy laws bring international, dramatic generations. Recently royal roads remember regulations. Much long costs ignore thus large children. British, assistant persons would occur in s|4.80|1.63|2003002|exportiimporto #2|3|pants|2|Men|975|antiationn st|extra large|magenta2387328499592|thistle|Tsp|Unknown|16|ationn steseese| +4498|AAAAAAAACJBBAAAA|1997-10-27|1999-10-27|Armed colleges must not join old firms; appropriate, warm magistrates know by a companies. Secrets shall get years; acceptable, technical things put annually daily, familiar parents. Notable rooms m|2.40|1.80|10008011|namelessunivamalg #11|8|scanners|10|Electronics|62|ablecally|N/A|038white558381484013|snow|Ton|Unknown|58|eingn steseese| +4499|AAAAAAAACJBBAAAA|1999-10-28|2001-10-26|Armed colleges must not join old firms; appropriate, warm magistrates know by a companies. Secrets shall get years; acceptable, technical things put annually daily, familiar parents. Notable rooms m|9.55|1.80|10008011|edu packbrand #10|4|curtains/drapes|7|Home|574|ablecally|N/A|0170476634377brown45|ghost|N/A|Unknown|29|n stn steseese| +4500|AAAAAAAACJBBAAAA|2001-10-27||Armed colleges must not join old firms; appropriate, warm magistrates know by a companies. Secrets shall get years; acceptable, technical things put annually daily, familiar parents. Notable rooms m|9.81|1.80|4003001|exportiedu pack #1|4|kids|4|Shoes|574|eseationanti|economy|0170476634377brown45|sienna|Gram|Unknown|61|barbarantiese| +4501|AAAAAAAAFJBBAAAA|1997-10-27||Controversial representatives believe places; open jobs unload only subsequently empty officers. Main, long models mention general students. Beings must use also to a |2.08|1.12|4003002|exportiedu pack #2|3|kids|4|Shoes|207|ationbarable|economy|5213steel25262509317|light|Unknown|Unknown|7|oughtbarantiese| +4502|AAAAAAAAGJBBAAAA|1997-10-27|2000-10-26|Ot|2.67|1.14|10001015|amalgunivamalg #15|1|cameras|10|Electronics|285|antieingable|N/A|712142682sandy353599|tan|Dozen|Unknown|27|ablebarantiese| +4503|AAAAAAAAGJBBAAAA|2000-10-27||Ot|5.83|3.67|7012004|importonameless #4|12|paint|7|Home|197|ationn stought|N/A|712142682sandy353599|purple|N/A|Unknown|59|pribarantiese| +4504|AAAAAAAAIJBBAAAA|1997-10-27|1999-10-27|Ships achieve as old, considerable members. New, key characters could not play n|0.98|0.47|9016005|corpunivamalg #5|16|mystery|9|Books|297|ationn stable|N/A|87705839330smoke7983|floral|Tsp|Unknown|25|esebarantiese| +4505|AAAAAAAAIJBBAAAA|1999-10-28|2001-10-26|Costs retain favorable oth|0.95|0.72|3003002|exportiexporti #2|3|toddlers|3|Children|297|ationn stable|small|87705839330smoke7983|powder|Gram|Unknown|27|antibarantiese| +4506|AAAAAAAAIJBBAAAA|2001-10-27||Then working problems could improve before a eyes. Almost dead points refer to a feet. Also practical metres sugge|1.96|0.80|5004001|edu packscholar #1|4|classical|5|Music|297|ationn stable|N/A|45051130667puff43663|ghost|Tsp|Unknown|64|callybarantiese| +4507|AAAAAAAALJBBAAAA|1997-10-27||Parents would undertake. Things can go materials. Different, mechanical tasks make there windows. Further high books explore often foreign, technical owners; increased matters get togeth|2.36|1.55|4003002|exportiedu pack #2|3|kids|4|Shoes|283|prieingable|large|379812066wheat673790|white|Case|Unknown|24|ationbarantiese| +4508|AAAAAAAAMJBBAAAA|1997-10-27|2000-10-26|Raw guns might march much experiences. Professional, strong characteristics need s|4.04|3.43|8003009|exportinameless #9|3|basketball|8|Sports|11|oughtought|N/A|papaya76590285083199|sienna|Box|Unknown|32|eingbarantiese| +4509|AAAAAAAAMJBBAAAA|2000-10-27||Social months would think; also common cattle lower no by the stages. Jews can mean economic, handsome lakes. Indicators tell more. Grateful hospitals ask seriou|4.45|3.43|4002002|importoedu pack #2|3|mens|4|Shoes|260|oughtought|petite|papaya76590285083199|puff|Each|Unknown|37|n stbarantiese| +4510|AAAAAAAAOJBBAAAA|1997-10-27|1999-10-27|Communists avoid however executives; types|1.51|1.17|5002001|importoscholar #1|2|country|5|Music|87|ationeing|N/A|5415535528590rosy694|puff|Unknown|Unknown|39|baroughtantiese| +4511|AAAAAAAAOJBBAAAA|1999-10-28|2001-10-26|Traditional, new rates answer to a t|3.71|2.67|5002001|scholarbrand #8|15|custom|6|Jewelry|598|ationeing|N/A|81salmon228571914422|tan|N/A|Unknown|18|oughtoughtantiese| +4512|AAAAAAAAOJBBAAAA|2001-10-27||Traditional, new rates answer to a t|0.33|0.23|3004001|edu packexporti #1|4|school-uniforms|3|Children|94|ationeing|extra large|81salmon228571914422|ivory|Each|Unknown|3|ableoughtantiese| +4513|AAAAAAAABKBBAAAA|1997-10-27||Obvious, concerned risks identify so. Single, valid hills could restore policies; eyes can get still. Large sales should bring still primary, main|66.30|53.04|8001004|amalgnameless #4|1|athletic shoes|8|Sports|243|prieseable|N/A|37867577744spring558|peru|Bundle|Unknown|35|prioughtantiese| +4514|AAAAAAAACKBBAAAA|1997-10-27|2000-10-26|Chemical dogs could not identify. Absolute |2.40|1.68|5001001|amalgscholar #1|1|rock|5|Music|747|ationeseation|N/A|17748blue09810671778|cornsilk|Box|Unknown|56|eseoughtantiese| +4515|AAAAAAAACKBBAAAA|2000-10-27||Chemical dogs could not identify. Absolute |9.74|1.68|5001001|amalgamalg #2|1|dresses|1|Women|747|ationeseation|medium|3008deep605477893739|lime|Box|Unknown|11|antioughtantiese| +4516|AAAAAAAAEKBBAAAA|1997-10-27|1999-10-27|Funds wander months. Advisory forms meet finally; complaints shall please to a roads. Often presen|3.58|2.32|8009007|maxinameless #7|9|optics|8|Sports|31|oughtpri|N/A|09831981linen1769874|misty|Carton|Unknown|26|callyoughtantiese| +4517|AAAAAAAAEKBBAAAA|1999-10-28|2001-10-26|Funds wander months. Advisory forms meet finally; complaints shall please to a roads. Often presen|1.71|2.32|8009007|importoedu pack #2|2|mens|4|Shoes|31|oughtpri|medium|47violet063986294269|spring|Lb|Unknown|31|ationoughtantiese| +4518|AAAAAAAAEKBBAAAA|2001-10-27||Funds wander months. Advisory forms meet finally; complaints shall please to a roads. Often presen|2.57|1.64|2001001|amalgimporto #1|2|accessories|2|Men|160|barcallyought|petite|4766184121snow440067|misty|Dozen|Unknown|2|eingoughtantiese| +4519|AAAAAAAAHKBBAAAA|1997-10-27||Hard sudden aspects shall not commemorate about a functions. Western, british cases see here churches. Stairs a|4.43|2.70|8008010|namelessnameless #10|8|outdoor|8|Sports|120|barableought|N/A|69403670papaya668731|peach|Gross|Unknown|85|n stoughtantiese| +4520|AAAAAAAAIKBBAAAA|1997-10-27|2000-10-26|Involved, jewish sanctions fill very regulatory miles. Drinks used to get below; sequences act very. For example lost prices must shake now normal, odd |6.20|3.22|4001001|amalgedu pack #1|1|womens|4|Shoes|189|n steingought|economy|9735pink000735236202|peru|Pound|Unknown|53|barableantiese| +4521|AAAAAAAAIKBBAAAA|2000-10-27||Speakers can combine new, formal firms. Similar studies might not think clearly big inc corners. Workers used to increase even used members. International, very years |1.89|0.64|4001001|importoimporto #2|2|shirts|2|Men|574|eseationanti|medium|9735pink000735236202|sienna|Cup|Unknown|12|oughtableantiese| +4522|AAAAAAAAKKBBAAAA|1997-10-27|1999-10-27|Days ought to fall definitely hard principles. Social limits may demonstrate here. Faintly electoral documents secure careful, ancient women|3.11|2.45|7011003|amalgnameless #3|11|accent|7|Home|168|eingcallyought|N/A|30314168brown1719684|honeydew|Cup|Unknown|61|ableableantiese| +4523|AAAAAAAAKKBBAAAA|1999-10-28|2001-10-26|Closed fears ask rules. Best accessible mammals should help cost|0.22|2.45|3003002|exportiexporti #2|3|toddlers|3|Children|160|eingcallyought|petite|1517459rosy139131622|spring|Gram|Unknown|19|priableantiese| +4524|AAAAAAAAKKBBAAAA|2001-10-27||Closed fears ask rules. Best accessible mammals should help cost|73.18|27.07|3003002|importoimporto #1|3|shirts|2|Men|187|ationeingought|medium|1517459rosy139131622|ghost|Box|Unknown|58|eseableantiese| +4525|AAAAAAAANKBBAAAA|1997-10-27||New developers used to receive newly funds. Important, absolute hours could construct so christian, bad services. Thin eyes may sell individual|8.05|3.30|4001002|amalgedu pack #2|1|womens|4|Shoes|332|ablepripri|small|211linen964998727153|maroon|Cup|Unknown|11|antiableantiese| +4526|AAAAAAAAOKBBAAAA|1997-10-27|2000-10-26|Votes can relieve then key sales; social, new proc|8.03|6.66|9003003|exportimaxi #3|3|computers|9|Books|307|ationbarpri|N/A|2551212932navajo7501|grey|Dram|Unknown|33|callyableantiese| +4527|AAAAAAAAOKBBAAAA|2000-10-27||Mental entries must not enable rarely cheeks. Other areas stop a|7.86|6.66|5004002|edu packscholar #2|4|classical|5|Music|402|ationbarpri|N/A|2551212932navajo7501|ivory|Dram|Unknown|17|ationableantiese| +4528|AAAAAAAAALBBAAAA|1997-10-27|1999-10-27|Relatively immediate products must not suppose so; accounts might experience still subject, difficult bits; employees avoid certainly worldwide sa|5.03|2.16|2004001|edu packimporto #1|4|sports-apparel|2|Men|974|eseationn st|petite|3651327192718tan2097|spring|Carton|Unknown|49|eingableantiese| +4529|AAAAAAAAALBBAAAA|1999-10-28|2001-10-26|Impossible, labour firms bring there domestic parents. |3.58|1.10|3003002|exportiexporti #2|3|toddlers|3|Children|974|eseationn st|petite|3651327192718tan2097|misty|Tsp|Unknown|51|n stableantiese| +4530|AAAAAAAAALBBAAAA|2001-10-27||States know low police. Direct restaurants take most. Expectations might not determine almost. Social interviews love economic, modern programmes. Different groups will get in a professionals. Defen|1.67|0.70|3003002|brandunivamalg #9|3|personal|10|Electronics|210|baroughtable|N/A|tomato32486898344421|pale|Carton|Unknown|55|barpriantiese| +4531|AAAAAAAADLBBAAAA|1997-10-27||Old-fashioned doctors must not bring generally. British rats serve skilled brothers. Wrong women will look definite conditions. Then vita|9.68|7.84|8010002|univmaxi #2|10|pools|8|Sports|234|esepriable|N/A|63324slate2079740175|puff|Gram|Unknown|9|oughtpriantiese| +4532|AAAAAAAAELBBAAAA|1997-10-27|2000-10-26|Free notes cannot ensure temporary things. Etc presidential purposes must not red|0.94|0.68|8015003|scholarmaxi #3|15|fishing|8|Sports|74|eseation|N/A|671029437551forest83|white|Box|Unknown|49|ablepriantiese| +4533|AAAAAAAAELBBAAAA|2000-10-27||Blues think yet interests. Rather other hundre|5.40|0.68|8015003|importoimporto #2|15|shirts|2|Men|58|eseation|medium|671029437551forest83|spring|Gram|Unknown|18|pripriantiese| +4534|AAAAAAAAGLBBAAAA|1997-10-27|1999-10-27|Numbers face correctly; events read trees. Recently normal results ought to hope; never useful eyes build just parties. Black, green clients hear as poor weeks. Too differe|4.75|1.42|10016013|corpamalgamalg #13|16|wireless|10|Electronics|60|barcally|N/A|80730323yellow334538|peru|Bundle|Unknown|23|esepriantiese| +4535|AAAAAAAAGLBBAAAA|1999-10-28|2001-10-26|Straight examples influence un|9.04|1.42|9003012|exportimaxi #12|3|computers|9|Books|60|barcally|N/A|80730323yellow334538|rose|Unknown|Unknown|18|antipriantiese| +4536|AAAAAAAAGLBBAAAA|2001-10-27||Straight examples influence un|2.68|0.80|1001001|amalgamalg #1|3|dresses|1|Women|222|barcally|extra large|1480270950769yellow6|tomato|Carton|Unknown|1|callypriantiese| +4537|AAAAAAAAJLBBAAAA|1997-10-27||Courts pay far american towns; more greek circumstances prevent so to a cars; sports read importantly also public lights. Strings grow short large, interesting interests. About good|7.06|2.75|7004008|edu packbrand #8|4|curtains/drapes|7|Home|246|callyeseable|N/A|547sandy236471673002|turquoise|Dozen|Unknown|26|ationpriantiese| +4538|AAAAAAAAKLBBAAAA|1997-10-27|2000-10-26|Questions would succeed never remains. Early host|0.79|0.65|8001009|amalgnameless #9|1|athletic shoes|8|Sports|37|ationpri|N/A|93spring364701034809|navajo|Dram|Unknown|33|eingpriantiese| +4539|AAAAAAAAKLBBAAAA|2000-10-27||Questions would succeed never remains. Early host|9.35|0.65|8014010|edu packmaxi #10|1|tennis|8|Sports|37|ationpri|N/A|93spring364701034809|medium|Dram|Unknown|5|n stpriantiese| +4540|AAAAAAAAMLBBAAAA|1997-10-27|1999-10-27|Years compensate gold,|4.23|3.21|7006001|corpbrand #1|6|rugs|7|Home|315|antioughtpri|N/A|8grey713045943405949|sky|Dram|Unknown|47|bareseantiese| +4541|AAAAAAAAMLBBAAAA|1999-10-28|2001-10-26|Years compensate gold,|0.47|3.21|6009006|maxicorp #6|9|womens watch|6|Jewelry|315|antioughtpri|N/A|8grey713045943405949|thistle|Ton|Unknown|54|oughteseantiese| +4542|AAAAAAAAMLBBAAAA|2001-10-27||Basic, religious individuals would safeguard away unique designers; able hours can see. Rare, considerable things would eat later initial belie|2.13|3.21|6009006|exportiunivamalg #4|3|dvd/vcr players|10|Electronics|516|antioughtpri|N/A|8grey713045943405949|royal|Carton|Unknown|25|ableeseantiese| +4543|AAAAAAAAPLBBAAAA|1997-10-27||Central years say more nights; reliable, inc years dismiss ideally englis|89.18|33.88|2004002|edu packimporto #2|4|sports-apparel|2|Men|481|oughteingese|medium|34lime96092325353397|steel|Gross|Unknown|7|prieseantiese| +4544|AAAAAAAAAMBBAAAA|1997-10-27|2000-10-26|Educational months reduce straight different attitudes. Quite serious facts could believe usually realistic ways. Then human cats touch quite moving lips. Strong, collective schools mig|1.97|1.12|4001001|amalgedu pack #1|1|womens|4|Shoes|534|eseprianti|medium|47slate1806413284717|sandy|Pound|Unknown|29|eseeseantiese| +4545|AAAAAAAAAMBBAAAA|2000-10-27||Educational months reduce straight different attitudes. Quite serious facts could believe usually realistic ways. Then human cats touch quite moving lips. Strong, collective schools mig|9.83|1.12|10005007|scholarunivamalg #7|1|karoke|10|Electronics|534|eseprianti|N/A|758371smoke731581080|salmon|Gross|Unknown|11|antieseantiese| +4546|AAAAAAAACMBBAAAA|1997-10-27|1999-10-27|Unfortunately secondary walls lead ever democrats; also nuclear requirements expose various, financial services. English changes should not cause conferences. Reports must not realise fully in |4.93|2.16|4002001|importoedu pack #1|2|mens|4|Shoes|645|antiesecally|petite|6steel37621819420417|smoke|Box|Unknown|57|callyeseantiese| +4547|AAAAAAAACMBBAAAA|1999-10-28|2001-10-26|Unfortunately secondary walls lead ever democrats; also nuclear requirements expose various, financial services. English changes should not cause conferences. Reports must not realise fully in |5.40|2.26|8006008|corpnameless #8|2|football|8|Sports|645|antiesecally|N/A|277949413058583mint1|pale|Oz|Unknown|6|ationeseantiese| +4548|AAAAAAAACMBBAAAA|2001-10-27||Unfortunately secondary walls lead ever democrats; also nuclear requirements expose various, financial services. English changes should not cause conferences. Reports must not realise fully in |3.34|1.77|8009009|maxinameless #9|9|optics|8|Sports|645|antiesecally|N/A|49120552345801tan982|white|Gross|Unknown|90|eingeseantiese| +4549|AAAAAAAAFMBBAAAA|1997-10-27||There bitter walls could provide now in a units. Awkward offices reveal ever really main dogs. Consequently poor figures give also big friends. Fol|0.09|0.06|4003002|exportiedu pack #2|3|kids|4|Shoes|367|ationcallypri|N/A|6008682spring4463303|seashell|Tbl|Unknown|5|n steseantiese| +4550|AAAAAAAAGMBBAAAA|1997-10-27|2000-10-26|Pensions used to meet in the words. Very african obligati|0.54|0.27|3003001|exportiexporti #1|3|toddlers|3|Children|560|barcallyanti|medium|608557red14704709613|red|Bunch|Unknown|2|barantiantiese| +4551|AAAAAAAAGMBBAAAA|2000-10-27||Pensions used to meet in the words. Very african obligati|8.23|5.59|3003001|exportiamalg #2|3|maternity|1|Women|560|barcallyanti|large|232240grey1207414724|turquoise|Bundle|Unknown|57|oughtantiantiese| +4552|AAAAAAAAIMBBAAAA|1997-10-27|1999-10-27|Single hours ought to say. Sources would contribute civil rivers. Good, central patients put too to the spirits. Sho|4.99|2.69|7015003|scholarnameless #3|15|tables|7|Home|132|ablepriought|N/A|5626311lemon25475949|puff|Gram|Unknown|22|ableantiantiese| +4553|AAAAAAAAIMBBAAAA|1999-10-28|2001-10-26|Single hours ought to say. Sources would contribute civil rivers. Good, central patients put too to the spirits. Sho|3.27|2.91|7015003|importomaxi #6|2|business|9|Books|978|ablepriought|N/A|5626311lemon25475949|powder|Oz|Unknown|24|priantiantiese| +4554|AAAAAAAAIMBBAAAA|2001-10-27||Single hours ought to say. Sources would contribute civil rivers. Good, central patients put too to the spirits. Sho|0.47|2.91|7015003|edu packamalg #1|4|swimwear|1|Women|978|eingationn st|medium|wheat684343186986332|seashell|Bunch|Unknown|18|eseantiantiese| +4555|AAAAAAAALMBBAAAA|1997-10-27||Medical policies expand open companies. Genetic, light years provide wide at least real equations. Everyday dogs vary formally simila|4.27|3.24|4001002|amalgedu pack #2|1|womens|4|Shoes|751|oughtantiation|large|rose2758376372765434|sandy|Tbl|Unknown|92|antiantiantiese| +4556|AAAAAAAAMMBBAAAA|1997-10-27|2000-10-26|Fresh plans appear ever; at least surprised colleagues contact for a facilities. Techniques gain too powerful channels. Blue stars shall not pursue usually underlying powe|2.36|1.67|1002001|importoamalg #1|2|fragrances|1|Women|167|ationcallyought|petite|pink5447282887349850|lawn|Pallet|Unknown|26|callyantiantiese| +4557|AAAAAAAAMMBBAAAA|2000-10-27||Fresh plans appear ever; at least surprised colleagues contact for a facilities. Techniques gain too powerful channels. Blue stars shall not pursue usually underlying powe|3.22|2.38|6002006|importocorp #6|2|diamonds|6|Jewelry|516|ationcallyought|N/A|pink5447282887349850|orange|Gram|Unknown|25|ationantiantiese| +4558|AAAAAAAAOMBBAAAA|1997-10-27|1999-10-27|Statistically warm resources keep too up to a p|6.39|3.25|9007011|brandmaxi #11|7|reference|9|Books|139|n stpriought|N/A|579307640570steel270|papaya|Pound|Unknown|61|eingantiantiese| +4559|AAAAAAAAOMBBAAAA|1999-10-28|2001-10-26|Statistically warm resources keep too up to a p|2.92|0.96|1002002|importoamalg #2|7|fragrances|1|Women|222|n stpriought|large|579307640570steel270|thistle|Pallet|Unknown|28|n stantiantiese| +4560|AAAAAAAAOMBBAAAA|2001-10-27||Whole, difficult police serve agricultural, left|2.51|0.96|9008001|namelessmaxi #1|7|romance|9|Books|477|n stpriought|N/A|94834302872tomato034|steel|Ounce|Unknown|7|barcallyantiese| +4561|AAAAAAAABNBBAAAA|1997-10-27||Numerous, young rooms will not keep expensive fields; probably british coun|1.63|0.68|2001002|amalgimporto #2|1|accessories|2|Men|77|ationation|petite|0532373413snow099589|pink|Bundle|Unknown|22|oughtcallyantiese| +4562|AAAAAAAACNBBAAAA|1997-10-27|2000-10-26|Flexible ships mean originally main bodies. Days contribute also invisible instruments. Underlying areas feel by a issues. Pieces used to fall however clean numbers; visual cases use|1.23|0.67|3004001|edu packexporti #1|4|school-uniforms|3|Children|481|oughteingese|medium|601255navy6099664014|ivory|Dram|Unknown|4|ablecallyantiese| +4563|AAAAAAAACNBBAAAA|2000-10-27||Forms must not say only through the cells. Developments must not contain with a wages. Differences might not look all soft,|5.12|3.32|10001008|amalgunivamalg #8|4|cameras|10|Electronics|1|oughteingese|N/A|601255navy6099664014|light|Bundle|Unknown|30|pricallyantiese| +4564|AAAAAAAAENBBAAAA|1997-10-27|1999-10-27|Quick, regular results would keep tomorrow; prisons lie. White, financial numbers would build now to a relationships. Japanese, hot limits set front components. Legs influence limi|5.25|2.04|8005005|scholarnameless #5|5|fitness|8|Sports|396|callyn stpri|N/A|11powder288496114265|smoke|Oz|Unknown|4|esecallyantiese| +4565|AAAAAAAAENBBAAAA|1999-10-28|2001-10-26|Quick, regular results would keep tomorrow; prisons lie. White, financial numbers would build now to a relationships. Japanese, hot limits set front components. Legs influence limi|0.88|2.04|8005005|exportibrand #6|13|loose stones|6|Jewelry|886|callyeingeing|N/A|06335turquoise669169|papaya|Gross|Unknown|7|anticallyantiese| +4566|AAAAAAAAENBBAAAA|2001-10-27||Quick, regular results would keep tomorrow; prisons lie. White, financial numbers would build now to a relationships. Japanese, hot limits set front components. Legs influence limi|0.81|0.35|6010007|univbrand #7|10|jewelry boxes|6|Jewelry|886|callyeingeing|N/A|06335turquoise669169|steel|Oz|Unknown|56|callycallyantiese| +4567|AAAAAAAAHNBBAAAA|1997-10-27||Corporate, nucl|8.99|3.23|8001008|amalgnameless #8|1|athletic shoes|8|Sports|321|oughtablepri|N/A|68706140798313pale74|sandy|Case|Unknown|39|ationcallyantiese| +4568|AAAAAAAAINBBAAAA|1997-10-27|2000-10-26|Only national standards could yield |83.15|28.27|3001001|amalgexporti #1|1|newborn|3|Children|73|priation|economy|282267609thistle3081|gainsboro|N/A|Unknown|32|eingcallyantiese| +4569|AAAAAAAAINBBAAAA|2000-10-27||Interesting, other processes must dispel. Still ugly dishes can think straight. Main missiles must pose theories. Sexual, level weeks should resolve for a facili|5.41|2.48|7008010|namelessbrand #10|1|lighting|7|Home|403|pribarese|N/A|73980482834rosy69959|floral|Case|Unknown|60|n stcallyantiese| +4570|AAAAAAAAKNBBAAAA|1997-10-27|1999-10-27|Friends send central, canadian beds. Wholly new organisations save thus heads. Complete students will com|4.68|2.43|8007001|brandnameless #1|7|hockey|8|Sports|481|oughteingese|N/A|lavender263047588607|mint|Pallet|Unknown|21|barationantiese| +4571|AAAAAAAAKNBBAAAA|1999-10-28|2001-10-26|Friends send central, canadian beds. Wholly new organisations save thus heads. Complete students will com|2.08|1.70|8007001|amalgbrand #4|11|semi-precious|6|Jewelry|481|oughteingese|N/A|lavender263047588607|tomato|Each|Unknown|35|oughtationantiese| +4572|AAAAAAAAKNBBAAAA|2001-10-27||Friends send central, canadian beds. Wholly new organisations save thus heads. Complete students will com|4.22|2.06|7005003|scholarbrand #3|5|blinds/shades|7|Home|481|oughteingese|N/A|0229978693moccasin01|papaya|Lb|Unknown|6|ableationantiese| +4573|AAAAAAAANNBBAAAA|1997-10-27||Constant, soft policies shelter so houses. Poor, other problems enter also proceedings; as real films clear no|7.61|3.72|1003002|exportiamalg #2|3|maternity|1|Women|503|pribaranti|medium|2222seashell42533247|powder|Dram|Unknown|65|priationantiese| +4574|AAAAAAAAONBBAAAA|1997-10-27|2000-10-26|Sorry trends must evacuate only. Inter|87.60|77.08|10011002|amalgamalgamalg #2|11|disk drives|10|Electronics|153|priantiought|N/A|734493powder30700304|sienna|Tsp|Unknown|9|eseationantiese| +4575|AAAAAAAAONBBAAAA|2000-10-27||Sorry trends must evacuate only. Inter|89.46|72.46|10011002|exportimaxi #4|11|computers|9|Books|153|priantiought|N/A|734493powder30700304|linen|Tbl|Unknown|25|antiationantiese| +4576|AAAAAAAAAOBBAAAA|1997-10-27|1999-10-27|Normal authorities understand more small expenses; copies |77.78|32.66|7007007|brandbrand #7|7|decor|7|Home|378|eingationpri|N/A|1738317steel91850285|red|Gram|Unknown|62|callyationantiese| +4577|AAAAAAAAAOBBAAAA|1999-10-28|2001-10-26|Ambitious machines can bring in a voters; automatically corporate contacts see other, local sessions. Intermediate, helpful branches take perfectly blacks. Els|1.49|0.90|10008005|namelessunivamalg #5|8|scanners|10|Electronics|409|n stbarese|N/A|7145powder0317245302|dark|Gross|Unknown|23|ationationantiese| +4578|AAAAAAAAAOBBAAAA|2001-10-27||Other, brief objections prove with a individuals. Commentators record just expensive families. Domestic, dead papers spend. Only good lives find so men. Certain, famous sales will find|31.28|0.90|10008005|amalgimporto #1|1|accessories|2|Men|78|n stbarese|medium|7145powder0317245302|metallic|Dram|Unknown|18|eingationantiese| +4579|AAAAAAAADOBBAAAA|1997-10-27||Tight definite videos shall not go in a ma|2.50|0.82|7011010|amalgnameless #10|11|accent|7|Home|329|n stablepri|N/A|4spring2817152262865|moccasin|Carton|Unknown|23|n stationantiese| +4580|AAAAAAAAEOBBAAAA|1997-10-27|2000-10-26|Interesting, environmental judge|3.93|1.17|2003001|exportiimporto #1|3|pants|2|Men|147|ationeseought|large|2061160118maroon1164|moccasin|Bunch|Unknown|11|bareingantiese| +4581|AAAAAAAAEOBBAAAA|2000-10-27||Interesting, environmental judge|4.14|1.17|4001002|amalgedu pack #2|1|womens|4|Shoes|361|oughtcallypri|large|2061160118maroon1164|white|Cup|Unknown|16|oughteingantiese| +4582|AAAAAAAAGOBBAAAA|1997-10-27|1999-10-27|Quick, other areas could not get especially customers. Just isolated problems may not sneak |8.24|3.13|6007007|brandcorp #7|7|pendants|6|Jewelry|75|antiation|N/A|008217751thistle4373|tan|Lb|Unknown|17|ableeingantiese| +4583|AAAAAAAAGOBBAAAA|1999-10-28|2001-10-26|Quick, other areas could not get especially customers. Just isolated problems may not sneak |3.72|3.13|6007007|edu packedu pack #2|7|athletic|4|Shoes|472|antiation|large|008217751thistle4373|white|Oz|Unknown|28|prieingantiese| +4584|AAAAAAAAGOBBAAAA|2001-10-27||Efficient, young photographs eat still territorial, educational reports. Financial, small bodies continue black, difficult days. Equal poli|7.71|3.13|2003001|exportiimporto #1|7|pants|2|Men|162|ablecallyought|large|008217751thistle4373|navy|Ounce|Unknown|23|eseeingantiese| +4585|AAAAAAAAJOBBAAAA|1997-10-27||Labour, mutual numbers impress useful bills. Narrow, import|3.93|3.02|2002002|importoimporto #2|2|shirts|2|Men|306|callybarpri|large|3801salmon0728799302|seashell|Bundle|Unknown|14|antieingantiese| +4586|AAAAAAAAKOBBAAAA|1997-10-27|2000-10-26|Sources replace however typical, significant|84.77|66.96|5001001|amalgscholar #1|1|rock|5|Music|532|ableprianti|N/A|897568201618saddle93|sandy|Tbl|Unknown|22|callyeingantiese| +4587|AAAAAAAAKOBBAAAA|2000-10-27||Large, optimistic lines used to compete only other, true changes. Sufficient, general rooms expec|5.86|4.92|5001001|amalgamalg #2|1|dresses|1|Women|532|ableprianti|medium|897568201618saddle93|sandy|Box|Unknown|18|ationeingantiese| +4588|AAAAAAAAMOBBAAAA|1997-10-27|1999-10-27|Patients determine political cases. Legislative organisations sell odds. Maybe soviet judges should appear fo|7.01|3.15|2004001|edu packimporto #1|4|sports-apparel|2|Men|89|n steing|medium|528874432plum4335868|rosy|Dram|Unknown|58|eingeingantiese| +4589|AAAAAAAAMOBBAAAA|1999-10-28|2001-10-26|Natural, exact rights conceal now lacking cells. Probably conventional activities make also opportunities. Main, other states should read|1.22|0.75|2004001|amalgnameless #10|1|athletic shoes|8|Sports|956|callyantin st|N/A|6378648990455peru131|wheat|Ton|Unknown|23|n steingantiese| +4590|AAAAAAAAMOBBAAAA|2001-10-27||Natural, exact rights conceal now lacking cells. Probably conventional activities make also opportunities. Main, other states should read|1.65|0.75|9015007|scholarunivamalg #7|15|fiction|9|Books|231|oughtpriable|N/A|6378648990455peru131|papaya|Pallet|Unknown|31|barn stantiese| +4591|AAAAAAAAPOBBAAAA|1997-10-27||Already european mothers ought to impose big ever fixed parents. Dominant groups say even. Here basic weeks set as winners. Modern, young prayers release very environ|7.48|3.14|7003002|exportibrand #2|3|kids|7|Home|756|callyantiation|N/A|41798maroon753496922|lace|Dram|Unknown|18|oughtn stantiese| +4592|AAAAAAAAAPBBAAAA|1997-10-27|2000-10-26|More different attempts replace. Changes look shoes. E|2.47|1.40|3001001|amalgexporti #1|1|newborn|3|Children|174|eseationought|medium|8503779907845salmon5|metallic|Lb|Unknown|66|ablen stantiese| +4593|AAAAAAAAAPBBAAAA|2000-10-27||Years project much between a councils. Afterwards public practices shall like a little there protective feet. African, clear materials apply. Al|4.15|2.57|1003002|exportiamalg #2|1|maternity|1|Women|174|eseationought|extra large|68809825thistle00586|pink|Case|Unknown|37|prin stantiese| +4594|AAAAAAAACPBBAAAA|1997-10-27|1999-10-27|R|8.39|2.68|6016003|corpbrand #3|16|consignment|6|Jewelry|615|antioughtcally|N/A|honeydew984600337829|slate|Cup|Unknown|11|esen stantiese| +4595|AAAAAAAACPBBAAAA|1999-10-28|2001-10-26|R|9.99|3.79|6016003|amalgunivamalg #6|1|cameras|10|Electronics|974|eseationn st|N/A|0papaya4954272672317|yellow|Dozen|Unknown|23|antin stantiese| +4596|AAAAAAAACPBBAAAA|2001-10-27||R|9.03|4.69|3004001|edu packexporti #1|1|school-uniforms|3|Children|137|eseationn st|small|39609maroon365471293|peru|Ton|Unknown|9|callyn stantiese| +4597|AAAAAAAAFPBBAAAA|1997-10-27||Negotiations must not drive slightly relative studies. Clear, new conditions get. Particular miles would res|3.95|1.93|1001002|amalgamalg #2|1|dresses|1|Women|649|n stesecally|medium|426057972047rose0899|salmon|Bundle|Unknown|47|ationn stantiese| +4598|AAAAAAAAGPBBAAAA|1997-10-27|2000-10-26|Numbers shall not cut almost. Just nice customers show only indian, |2.03|0.64|5003001|exportischolar #1|3|pop|5|Music|912|ableoughtn st|N/A|6753ivory89854659013|purple|Dozen|Unknown|37|eingn stantiese| +4599|AAAAAAAAGPBBAAAA|2000-10-27||Economic, parliamentary signs explore to a employers. Allegations can send over years. Friendly, |84.08|0.64|5003001|amalgamalg #2|1|dresses|1|Women|432|ableoughtn st|N/A|6753ivory89854659013|peru|N/A|Unknown|23|n stn stantiese| +4600|AAAAAAAAIPBBAAAA|1997-10-27|1999-10-27|Likely, residential efforts talk actual, close teachers. Other hundreds come rapidly as possible things. Good operations shall set fiercely. Great, upper difficulties become immediate|7.15|2.28|7012001|importonameless #1|12|paint|7|Home|585|antieinganti|N/A|163410red40250772247|slate|Unknown|Unknown|48|barbarcallyese| +4601|AAAAAAAAIPBBAAAA|1999-10-28|2001-10-26|Likely, residential efforts talk actual, close teachers. Other hundreds come rapidly as possible things. Good operations shall set fiercely. Great, upper difficulties become immediate|18.68|11.39|1003002|exportiamalg #2|12|maternity|1|Women|585|antieinganti|petite|72peru03112318158938|chocolate|Lb|Unknown|73|oughtbarcallyese| +4602|AAAAAAAAIPBBAAAA|2001-10-27||Nice, distant kinds employ then. Colourful, managerial|1.42|11.39|1002001|importoamalg #1|2|fragrances|1|Women|585|antieinganti|large|tomato42939608215661|pale|Bundle|Unknown|55|ablebarcallyese| +4603|AAAAAAAALPBBAAAA|1997-10-27||Very naval fears cannot get so as children; holes shall acknowledge virtually in a losses.|8.32|3.41|3004002|edu packexporti #2|4|school-uniforms|3|Children|646|callyesecally|medium|9764peru796775493850|rose|Lb|Unknown|21|pribarcallyese| +4604|AAAAAAAAMPBBAAAA|1997-10-27|2000-10-26|Still senior sons go far since a types. High, welsh appearances move at all sudden trials. Most sexual sources take open, small terms. Provincial|6.54|3.79|3003001|exportiexporti #1|3|toddlers|3|Children|234|esepriable|small|7202105747861pink200|purple|Tsp|Unknown|10|esebarcallyese| +4605|AAAAAAAAMPBBAAAA|2000-10-27||Terms would require in a campaigns. Brothers would borrow both with a wages. British companies must grow operators; english companies run. Premis|2.16|1.53|7005006|scholarbrand #6|3|blinds/shades|7|Home|234|esepriable|N/A|7202105747861pink200|navajo|Gram|Unknown|11|antibarcallyese| +4606|AAAAAAAAOPBBAAAA|1997-10-27|1999-10-27|Distant projects eat accounts. Sens|6.07|2.85|1002001|importoamalg #1|2|fragrances|1|Women|296|callyn stable|medium|53315272magenta92555|tomato|Oz|Unknown|30|callybarcallyese| +4607|AAAAAAAAOPBBAAAA|1999-10-28|2001-10-26|Black hills last full accounts. Months make overnight for a rules; alone large masters might not lower as effective previous hours. Religious names explain far from out of the reports. Le|4.99|2.74|1002001|scholarmaxi #12|2|history|9|Books|296|callyn stable|N/A|53315272magenta92555|rosy|Box|Unknown|62|ationbarcallyese| +4608|AAAAAAAAOPBBAAAA|2001-10-27||Recent, growing lips speak english structures. Complex, possible bits shall take. P|3.95|2.74|1002001|amalgscholar #1|2|rock|5|Music|296|callyn stable|N/A|53315272magenta92555|snow|Carton|Unknown|63|eingbarcallyese| +4609|AAAAAAAABACBAAAA|1997-10-27||Now financial measures shall see thereby lives; appropriate costs target normally new, limited children. Different children resist such as the victi|2.51|0.82|5002002|importoscholar #2|2|country|5|Music|28|eingable|N/A|1994202saddle8326596|rose|Gross|Unknown|19|n stbarcallyese| +4610|AAAAAAAACACBAAAA|1997-10-27|2000-10-26|Traditional, concerned cases say more now tough minutes. New pictures stop by a letters. Shareholders cannot teach over average, physical memor|8.53|4.26|8004001|edu packnameless #1|4|camping|8|Sports|474|eseationese|N/A|205220405531spring12|tomato|Unknown|Unknown|34|baroughtcallyese| +4611|AAAAAAAACACBAAAA|2000-10-27||Traditional, concerned cases say more now tough minutes. New pictures stop by a letters. Shareholders cannot teach over average, physical memor|2.31|4.26|8004001|namelessunivamalg #5|8|scanners|10|Electronics|474|eseationese|N/A|772273253067227lace2|navy|Unknown|Unknown|33|oughtoughtcallyese| +4612|AAAAAAAAEACBAAAA|1997-10-27|1999-10-27|Awkward, typical places must not check only workers; bright parties restore studies. Men show about finally interested programmes; certainly economic soldiers meet there |1.67|0.65|4004001|edu packedu pack #1|4|athletic|4|Shoes|82|ableeing|medium|19035597317927cyan14|grey|Dozen|Unknown|25|ableoughtcallyese| +4613|AAAAAAAAEACBAAAA|1999-10-28|2001-10-26|Awkward, typical places must not check only workers; bright parties restore studies. Men show about finally interested programmes; certainly economic soldiers meet there |1.72|1.51|4004001|amalgedu pack #2|1|womens|4|Shoes|82|ableeing|small|39gainsboro202094753|purple|Ton|Unknown|19|prioughtcallyese| +4614|AAAAAAAAEACBAAAA|2001-10-27||Awkward, typical places must not check only workers; bright parties restore studies. Men show about finally interested programmes; certainly economic soldiers meet there |2.79|1.51|4004001|amalgimporto #1|1|accessories|2|Men|82|ableeing|N/A|754201936royal008977|spring|Oz|Unknown|21|eseoughtcallyese| +4615|AAAAAAAAHACBAAAA|1997-10-27||Institutions ought to need projects. As possible citizens used to like here british male estates. Long, essential exceptions must win national, original outcomes; correspondi|3.58|2.29|7016006|corpnameless #6|16|furniture|7|Home|851|oughtantieing|N/A|7299786papaya7052746|brown|Each|Unknown|26|antioughtcallyese| +4616|AAAAAAAAIACBAAAA|1997-10-27|2000-10-26|Deaths provide corresponding expenses. Days must publish. Mental, private ma|1.77|0.67|9013009|exportiunivamalg #9|13|self-help|9|Books|783|prieingation|N/A|83356919red416812268|light|Gross|Unknown|52|callyoughtcallyese| +4617|AAAAAAAAIACBAAAA|2000-10-27||Deaths provide corresponding expenses. Days must publish. Mental, private ma|9.10|4.73|9013009|brandunivamalg #11|13|personal|10|Electronics|783|prieingation|N/A|818006158668606lime7|navajo|Bunch|Unknown|60|ationoughtcallyese| +4618|AAAAAAAAKACBAAAA|1997-10-27|1999-10-27|Light acts prepare later copies; technical, just departments would see almost possibl|8.76|2.62|9014011|edu packunivamalg #11|14|sports|9|Books|576|callyationanti|N/A|4490940336786grey846|tomato|Dozen|Unknown|61|eingoughtcallyese| +4619|AAAAAAAAKACBAAAA|1999-10-28|2001-10-26|Light acts prepare later copies; technical, just departments would see almost possibl|4.77|4.05|9014011|amalgamalg #2|1|dresses|1|Women|576|callyationanti|N/A|4490940336786grey846|forest|Case|Unknown|20|n stoughtcallyese| +4620|AAAAAAAAKACBAAAA|2001-10-27||Modest scenes write on a events|6.63|4.05|9014011|univbrand #5|10|jewelry boxes|6|Jewelry|576|callyationanti|N/A|30189332pink73086454|peru|Case|Unknown|59|barablecallyese| +4621|AAAAAAAANACBAAAA|1997-10-27||New teachers might demand never assets. Deeply bright ministers make generally never prime imports. Odd writings step common readers; talks take young, r|2.95|2.56|9002002|importomaxi #2|2|business|9|Books|86|callyeing|N/A|8lime421942684381437|tan|Oz|Unknown|34|oughtablecallyese| +4622|AAAAAAAAOACBAAAA|1997-10-27|2000-10-26|Competitive cities can begin partly responsibilities. New things would not raise dual seeds. Now primary comment|1.46|0.99|6011007|amalgbrand #7|11|semi-precious|6|Jewelry|47|ationese|N/A|77136214808thistle23|royal|Dozen|Unknown|24|ableablecallyese| +4623|AAAAAAAAOACBAAAA|2000-10-27||Profits must pay there revolutionary months. Serious, sudden experiences might go. Just appropriate police buy both adverse, costly holidays. Children must drive. German, single |9.85|3.34|6011007|edu packnameless #4|14|glassware|7|Home|47|ationese|N/A|77136214808thistle23|almond|Gross|Unknown|33|priablecallyese| +4624|AAAAAAAAABCBAAAA|1997-10-27|1999-10-27|Kings decide too signs. Usual, democratic cases must look teeth; so canadian children look. Free ideas would |2.80|1.34|1004001|edu packamalg #1|4|swimwear|1|Women|663|pricallycally|medium|77813sienna374927256|midnight|Lb|Unknown|31|eseablecallyese| +4625|AAAAAAAAABCBAAAA|1999-10-28|2001-10-26|Stars ensure never main schools. Enterprises must n|3.85|3.42|1004001|exportiunivamalg #2|3|dvd/vcr players|10|Electronics|70|baration|N/A|8623004red2910415591|red|Ton|Unknown|41|antiablecallyese| +4626|AAAAAAAAABCBAAAA|2001-10-27||Successive communities will meet. Other, past objects respond such as a times. Prospects can start lightly by a e|6.51|2.34|1004001|edu packimporto #1|3|sports-apparel|2|Men|70|baration|large|139684256wheat581977|hot|Ton|Unknown|7|callyablecallyese| +4627|AAAAAAAADBCBAAAA|1997-10-27||Major, great arms ought to engage tomorrow unlike, original words. Materials must perceive classes. However great readers see now meanwhile known p|2.54|1.06|6015004|scholarbrand #4|15|custom|6|Jewelry|126|callyableought|N/A|87143lawn43478393346|goldenrod|N/A|Unknown|4|ationablecallyese| +4628|AAAAAAAAEBCBAAAA|1997-10-27|2000-10-26|Darling children might direct together similar effects. Able, certain fat|3.66|1.09|6014005|edu packbrand #5|14|estate|6|Jewelry|77|ationation|N/A|470769954414powder18|spring|Bunch|Unknown|45|eingablecallyese| +4629|AAAAAAAAEBCBAAAA|2000-10-27||Italian, potential recommendations carry economic, living children; really other conditions grow fine sons. Different, various roads let|1.52|0.97|10009006|maxiunivamalg #6|14|televisions|10|Electronics|77|ationation|N/A|470769954414powder18|almond|Ton|Unknown|18|n stablecallyese| +4630|AAAAAAAAGBCBAAAA|1997-10-27|1999-10-27|European authors may belong apparently. Friends suit subsequently possible miles. Interesting organisations might send. Years put however angry, large stars; only healthy areas can send in a cen|4.63|3.61|1004001|edu packamalg #1|4|swimwear|1|Women|350|barantipri|medium|79437090violet791839|sandy|Cup|Unknown|58|barpricallyese| +4631|AAAAAAAAGBCBAAAA|1999-10-28|2001-10-26|Prime clouds should describe very little, good lights. Economic|0.11|0.04|1004001|edu packcorp #8|4|bracelets|6|Jewelry|350|barantipri|N/A|10640327850yellow143|blue|N/A|Unknown|58|oughtpricallyese| +4632|AAAAAAAAGBCBAAAA|2001-10-27||Present feet take more able cases. Then useful preferences rehearse together with the institutions. At all following things shall hope all economic different cells; even good|2.40|0.04|1004001|importoamalg #1|2|fragrances|1|Women|350|barantipri|extra large|10640327850yellow143|maroon|Box|Unknown|8|ablepricallyese| +4633|AAAAAAAAJBCBAAAA|1997-10-27||Payments work certainly deep proteins; now other reports used to attempt to a matters.|91.49|61.29|8010004|univmaxi #4|10|pools|8|Sports|499|n stn stese|N/A|53421turquoise221139|snow|Cup|Unknown|10|pripricallyese| +4634|AAAAAAAAKBCBAAAA|1997-10-27|2000-10-26|Other, sorry countries must help rather teachers. Specific, sensitive police will feel by a ministers; new terms build indeed months. Black i|6.07|3.88|7010005|univnameless #5|10|flatware|7|Home|330|barpripri|N/A|59592702thistle01325|purple|Pound|Unknown|20|esepricallyese| +4635|AAAAAAAAKBCBAAAA|2000-10-27||No longer other german|8.34|3.91|4002002|importoedu pack #2|2|mens|4|Shoes|330|barpripri|large|59592702thistle01325|tomato|Lb|Unknown|15|antipricallyese| +4636|AAAAAAAAMBCBAAAA|1997-10-27|1999-10-27|New votes should pay immediately material costs. Members may assess highly gra|1.16|0.81|4002001|importoedu pack #1|2|mens|4|Shoes|18|eingought|small|4967971323403white53|lace|Gross|Unknown|33|callypricallyese| +4637|AAAAAAAAMBCBAAAA|1999-10-28|2001-10-26|New votes should pay immediately material costs. Members may assess highly gra|1.30|0.74|4002001|exportiexporti #2|2|toddlers|3|Children|330|barpripri|petite|4967971323403white53|seashell|Pound|Unknown|4|ationpricallyese| +4638|AAAAAAAAMBCBAAAA|2001-10-27||Clearly practical winners can escap|3.11|0.74|4002001|importoamalg #1|2|fragrances|1|Women|234|barpripri|small|4967971323403white53|sky|Unknown|Unknown|3|eingpricallyese| +4639|AAAAAAAAPBCBAAAA|1997-10-27||Circumstances happen communications. Young pairs need of course workers. Answers will go. E|4.12|1.77|4001002|amalgedu pack #2|1|womens|4|Shoes|215|antioughtable|extra large|1223olive06651265776|peru|Tbl|Unknown|58|n stpricallyese| +4640|AAAAAAAAACCBAAAA|1997-10-27|2000-10-26|Different, magic others may take federal sources. Powerful, complete visitors|5.07|3.44|1002001|importoamalg #1|2|fragrances|1|Women|219|n stoughtable|medium|193886117026sienna89|yellow|Bundle|Unknown|55|baresecallyese| +4641|AAAAAAAAACCBAAAA|2000-10-27||Actual|1.80|3.44|1002001|namelessnameless #2|2|outdoor|8|Sports|393|prin stpri|N/A|193886117026sienna89|sandy|Tbl|Unknown|46|oughtesecallyese| +4642|AAAAAAAACCCBAAAA|1997-10-27|1999-10-27||4.05||3001001|amalgexporti #1||||Children|171||petite||sandy|||45|| +4643|AAAAAAAACCCBAAAA|1999-10-28|2001-10-26|Expensi|0.12|3.11|3001001|univnameless #4|1|flatware|7|Home|171|oughtationought|N/A|2397misty49003303136|lemon|Lb|Unknown|14|priesecallyese| +4644|AAAAAAAACCCBAAAA|2001-10-27||Expensi|8.57|5.05|2002001|importoimporto #1|1|shirts|2|Men|171|oughtationought|extra large|8604778664014steel07|gainsboro|N/A|Unknown|64|eseesecallyese| +4645|AAAAAAAAFCCBAAAA|1997-10-27||Gently sexual poems will need just different, subject negotiations. Christians survive crudely; local, unlikely forms used to make so full deci|2.58|1.49|4001002|amalgedu pack #2|1|womens|4|Shoes|87|ationeing|small|4387181492466yellow7|navajo|Carton|Unknown|91|antiesecallyese| +4646|AAAAAAAAGCCBAAAA|1997-10-27|2000-10-26|Excellent, practical police shall |4.08|2.97|4001001|amalgedu pack #1|1|womens|4|Shoes|311|oughtoughtpri|N/A|9saddle5070004922664|slate|Cup|Unknown|8|callyesecallyese| +4647|AAAAAAAAGCCBAAAA|2000-10-27||Excellent, practical police shall |1.62|2.97|4001002|amalgedu pack #2|1|womens|4|Shoes|563|oughtoughtpri|large|92009894gainsboro138|tomato|Bunch|Unknown|21|ationesecallyese| +4648|AAAAAAAAICCBAAAA|1997-10-27|1999-10-27|Other c|3.99|1.95|5001001|amalgscholar #1|1|rock|5|Music|565|anticallyanti|N/A|17248smoke3240476290|peach|N/A|Unknown|57|eingesecallyese| +4649|AAAAAAAAICCBAAAA|1999-10-28|2001-10-26|Partly worthwhile fans see french, interested hours. Together|4.75|1.95|3001002|amalgexporti #2|1|newborn|3|Children|656|anticallyanti|extra large|864806992salmon34860|red|Gram|Unknown|82|n stesecallyese| +4650|AAAAAAAAICCBAAAA|2001-10-27||Different artists sign forwar|2.36|1.95|3001002|amalgbrand #1|1|bathroom|7|Home|656|anticallyanti|N/A|996305yellow23381427|peach|Dram|Unknown|33|baranticallyese| +4651|AAAAAAAALCCBAAAA|1997-10-27||Likely opportunities used to exercise quiet, present children. Early, limited reasons mean also small types. Possible cases will not stop inevitably major, safe eyebrows. Also economic|8.65|4.84|8010002|univmaxi #2|10|pools|8|Sports|630|barprically|N/A|6395609572puff283230|sandy|Dram|Unknown|47|oughtanticallyese| +4652|AAAAAAAAMCCBAAAA|1997-10-27|2000-10-26|Genuine papers shall wake however enough|2.97|2.52|10013012|exportiamalgamalg #12|13|stereo|10|Electronics|453|priantiese|N/A|4271692602269snow885|turquoise|Bundle|Unknown|50|ableanticallyese| +4653|AAAAAAAAMCCBAAAA|2000-10-27||Doors need generally plants; a bit experimental forces see here only ready operators; so-called hands can work already between a types. Particularly male schools would not ask more na|9.81|4.51|10001013|amalgunivamalg #13|1|cameras|10|Electronics|383|prieingpri|N/A|4271692602269snow885|papaya|Dram|Unknown|9|prianticallyese| +4654|AAAAAAAAOCCBAAAA|1997-10-27|1999-10-27|Considerable, jewish months start personal, awful institutions. So new applications must absorb only complete, great patients. Fundamental miles would not find. Typ|8.54|6.57|3001001|amalgexporti #1|1|newborn|3|Children|450|barantiese|medium|moccasin039334484688|smoke|Bundle|Unknown|56|eseanticallyese| +4655|AAAAAAAAOCCBAAAA|1999-10-28|2001-10-26|Amounts must not choose never to|91.94|6.57|3001001|exportiedu pack #2|3|kids|4|Shoes|450|barantiese|medium|moccasin039334484688|slate|Gram|Unknown|22|antianticallyese| +4656|AAAAAAAAOCCBAAAA|2001-10-27||Successful, h|0.96|6.57|3001001|amalgimporto #1|1|accessories|2|Men|450|barantiese|extra large|030powder55406685385|white|N/A|Unknown|22|callyanticallyese| +4657|AAAAAAAABDCBAAAA|1997-10-27||New women avoid about silver, precise meetings; necessary, human troops could think underground also large nurses. Inadequate|3.81|2.62|6005002|scholarcorp #2|5|earings|6|Jewelry|164|esecallyought|N/A|066451939789030hot77|chartreuse|Bunch|Unknown|7|ationanticallyese| +4658|AAAAAAAACDCBAAAA|1997-10-27|2000-10-26|Fully international days e|20.44|13.28|5004001|edu packscholar #1|4|classical|5|Music|61|oughtcally|N/A|97518olive1500998826|spring|Carton|Unknown|98|einganticallyese| +4659|AAAAAAAACDCBAAAA|2000-10-27||Others will not kn|2.68|13.28|8010010|univmaxi #10|10|pools|8|Sports|402|ablebarese|N/A|97518olive1500998826|sky|Pound|Unknown|23|n stanticallyese| +4660|AAAAAAAAEDCBAAAA|1997-10-27|1999-10-27|Main problems proceed then |7.57|3.25|7014001|edu packnameless #1|14|glassware|7|Home|404|esebarese|N/A|9600900517193purple8|snow|Tsp|Unknown|20|barcallycallyese| +4661|AAAAAAAAEDCBAAAA|1999-10-28|2001-10-26|Main problems proceed then |8.72|5.84|7014001|univmaxi #2|10|pools|8|Sports|241|oughteseable|N/A|1622purple0939897044|wheat|Ton|Unknown|32|oughtcallycallyese| +4662|AAAAAAAAEDCBAAAA|2001-10-27||Extra, mental circumstances could not appear yards. Best tall ages should make desperately high, real girls. Other, structural things go so political, rough themes. Copies resist buyers. |3.88|2.79|2002001|importoimporto #1|2|shirts|2|Men|263|pricallyable|small|1622purple0939897044|royal|Gross|Unknown|2|ablecallycallyese| +4663|AAAAAAAAHDCBAAAA|1997-10-27||Charges will not insist broken aspects. Questio|2.27|1.15|4004002|edu packedu pack #2|4|athletic|4|Shoes|149|n steseought|large|rose2710523302100606|salmon|Ounce|Unknown|2|pricallycallyese| +4664|AAAAAAAAIDCBAAAA|1997-10-27|2000-10-26|Powerful, ab|7.17|3.22|2003001|exportiimporto #1|3|pants|2|Men|188|eingeingought|small|4pale619820005941737|pale|Tbl|Unknown|31|esecallycallyese| +4665|AAAAAAAAIDCBAAAA|2000-10-27||Powerful, ab|2.23|1.31|2002002|importoimporto #2|3|shirts|2|Men|188|eingeingought|small|43936638seashell4302|sky|Unknown|Unknown|23|anticallycallyese| +4666|AAAAAAAAKDCBAAAA|1997-10-27|1999-10-27|Days escape libera|5.27|4.63|10001009|amalgunivamalg #9|1|cameras|10|Electronics|142|ableeseought|N/A|5620tan1520296017373|salmon|Ounce|Unknown|7|callycallycallyese| +4667|AAAAAAAAKDCBAAAA|1999-10-28|2001-10-26|Weeks could set even judi|9.43|4.63|10013010|exportiamalgamalg #10|1|stereo|10|Electronics|142|ableeseought|N/A|5620tan1520296017373|powder|Box|Unknown|10|ationcallycallyese| +4668|AAAAAAAAKDCBAAAA|2001-10-27||Also divine issues would not encourage extremely i|4.21|4.63|5001001|amalgscholar #1|1|rock|5|Music|142|ableeseought|N/A|73smoke7883662814686|snow|Lb|Unknown|16|eingcallycallyese| +4669|AAAAAAAANDCBAAAA|1997-10-27||Remote, temporary contents know to a scholars; stubbornly normal details lead outside in the authors. New changes must care others. New, glad sanctions like later immediate sides. Studies|2.14|1.54|3003002|exportiexporti #2|3|toddlers|3|Children|274|eseationable|medium|8573245misty65573058|puff|Dram|Unknown|11|n stcallycallyese| +4670|AAAAAAAAODCBAAAA|1997-10-27|2000-10-26|New, clear objects survive far vital standards; various solutions ought to require enough just weak goods. Raw, old arch|6.61|2.84|7015001|scholarnameless #1|15|tables|7|Home|34|esepri|N/A|53304764247saddle586|tomato|Carton|Unknown|21|barationcallyese| +4671|AAAAAAAAODCBAAAA|2000-10-27||Socialist, serious trees may go attractive questions. Artistic clubs get different, simple troops. Sports could |15.52|2.84|10010014|univamalgamalg #14|10|memory|10|Electronics|34|esepri|N/A|3603842red1275628478|rosy|Tsp|Unknown|32|oughtationcallyese| +4672|AAAAAAAAAECBAAAA|1997-10-27|1999-10-27|Lucky experiments ought to sell services. Substances protect only; vessels give si|3.83|2.68|6008001|namelesscorp #1|8|mens watch|6|Jewelry|496|callyn stese|N/A|53232978indian560635|royal|Dram|Unknown|10|ableationcallyese| +4673|AAAAAAAAAECBAAAA|1999-10-28|2001-10-26|Lucky experiments ought to sell services. Substances protect only; vessels give si|5.59|2.68|6008001|importounivamalg #6|8|home repair|9|Books|992|ablen stn st|N/A|5626793mint129339460|tan|Tbl|Unknown|30|priationcallyese| +4674|AAAAAAAAAECBAAAA|2001-10-27||Lucky experiments ought to sell services. Substances protect only; vessels give si|3.71|2.68|6008001|amalgmaxi #5|11|archery|8|Sports|166|callycallyought|N/A|5626793mint129339460|lawn|Tsp|Unknown|22|eseationcallyese| +4675|AAAAAAAADECBAAAA|1997-10-27||Chemical, little examples vanish for a fragments. Joint examples pa|26.37|12.92|1001002|amalgamalg #2|1|dresses|1|Women|19|n stought|N/A|5royal90482649036343|smoke|Gram|Unknown|10|antiationcallyese| +4676|AAAAAAAAEECBAAAA|1997-10-27|2000-10-26|Relati|4.82|1.44|6010005|univbrand #5|10|jewelry boxes|6|Jewelry|29|n stable|N/A|227835827486sandy186|chartreuse|Gram|Unknown|1|callyationcallyese| +4677|AAAAAAAAEECBAAAA|2000-10-27||Relati|5.60|1.44|6010005|edu packamalg #2|10|swimwear|1|Women|29|n stable|petite|227835827486sandy186|green|Carton|Unknown|26|ationationcallyese| +4678|AAAAAAAAGECBAAAA|1997-10-27|1999-10-27|However dead stars shall not think lately only ordinary dates. Day|9.88|5.03|9005011|scholarmaxi #11|5|history|9|Books|14|eseought|N/A|833119227smoke516194|yellow|Pallet|Unknown|14|eingationcallyese| +4679|AAAAAAAAGECBAAAA|1999-10-28|2001-10-26|However dead stars shall not think lately only ordinary dates. Day|0.32|5.03|7013010|exportinameless #10|5|wallpaper|7|Home|476|eseought|N/A|833119227smoke516194|lace|Oz|Unknown|28|n stationcallyese| +4680|AAAAAAAAGECBAAAA|2001-10-27||However dead stars shall not think lately only ordinary dates. Day|95.39|67.72|4001001|amalgedu pack #1|5|womens|4|Shoes|3|pri|economy|833119227smoke516194|moccasin|Bunch|Unknown|22|bareingcallyese| +4681|AAAAAAAAJECBAAAA|1997-10-27||Free, personal results find easily also equal tears. Necessary, l|49.73|43.76|8011002|amalgmaxi #2|11|archery|8|Sports|587|ationeinganti|N/A|6325186702lavender71|wheat|Case|Unknown|39|oughteingcallyese| +4682|AAAAAAAAKECBAAAA|1997-10-27|2000-10-26|Expensive observations might accommodate even local specialists. Detailed publications break. Still operational terms will win so women; easy |29.03|18.57|1002001|importoamalg #1|2|fragrances|1|Women|278|eingationable|medium|23tan024562677651950|turquoise|Pound|Unknown|14|ableeingcallyese| +4683|AAAAAAAAKECBAAAA|2000-10-27||Expensive observations might accommodate even local specialists. Detailed publications break. Still operational terms will win so women; easy |2.88|2.41|1001002|amalgamalg #2|2|dresses|1|Women|278|eingationable|petite|23tan024562677651950|maroon|Bundle|Unknown|24|prieingcallyese| +4684|AAAAAAAAMECBAAAA|1997-10-27|1999-10-27|Provisions go too. Sad others contain italian branches. Keys k|2.08|1.66|9006005|corpmaxi #5|6|parenting|9|Books|839|n stprieing|N/A|5281132572117mint197|sienna|Unknown|Unknown|7|eseeingcallyese| +4685|AAAAAAAAMECBAAAA|1999-10-28|2001-10-26|Provisions go too. Sad others contain italian branches. Keys k|68.06|1.66|5004002|edu packscholar #2|6|classical|5|Music|839|n stprieing|N/A|5281132572117mint197|pink|Pound|Unknown|62|antieingcallyese| +4686|AAAAAAAAMECBAAAA|2001-10-27||European, western players get co|2.76|2.23|6010007|univbrand #7|10|jewelry boxes|6|Jewelry|188|n stprieing|N/A|27306medium149225280|magenta|Unknown|Unknown|81|callyeingcallyese| +4687|AAAAAAAAPECBAAAA|1997-10-27||Sides will not make very working influences. Assistant clothes carry quite benefits. Available part|25.23|22.45|9007008|brandmaxi #8|7|reference|9|Books|340|baresepri|N/A|7745377firebrick4000|bisque|Gross|Unknown|42|ationeingcallyese| +4688|AAAAAAAAAFCBAAAA|1997-10-27|2000-10-26|Then social things can intend once |3.19|2.20|1002001|importoamalg #1|2|fragrances|1|Women|155|antiantiought|medium|2490253931turquoise7|violet|Bundle|Unknown|64|eingeingcallyese| +4689|AAAAAAAAAFCBAAAA|2000-10-27||Then social things can intend once |1.04|2.20|1002001|edu packexporti #2|4|school-uniforms|3|Children|155|antiantiought|economy|00deep60071754923746|thistle|Carton|Unknown|19|n steingcallyese| +4690|AAAAAAAACFCBAAAA|1997-10-27|1999-10-27|Uniq|3.60|1.33|3004001|edu packexporti #1|4|school-uniforms|3|Children|182|ableeingought|large|1007363800183sienna7|spring|Tbl|Unknown|13|barn stcallyese| +4691|AAAAAAAACFCBAAAA|1999-10-28|2001-10-26|Uniq|77.90|57.64|5004002|edu packscholar #2|4|classical|5|Music|182|ableeingought|N/A|1007363800183sienna7|sandy|Lb|Unknown|26|oughtn stcallyese| +4692|AAAAAAAACFCBAAAA|2001-10-27||As economic demands cannot see. Easily vital programs mean legal, e|7.95|6.28|4001001|amalgedu pack #1|1|womens|4|Shoes|182|ableeingought|small|1007363800183sienna7|lemon|Unknown|Unknown|25|ablen stcallyese| +4693|AAAAAAAAFFCBAAAA|1997-10-27||Early, strong disputes tackle less than open, original sales. |1.43|1.11|3004002|edu packexporti #2|4|school-uniforms|3|Children|140|bareseought|large|4638417594441lime577|seashell|Oz|Unknown|26|prin stcallyese| +4694|AAAAAAAAGFCBAAAA|1997-10-27|2000-10-26|Difficulties prevent better small, mere operations. States provide now from a advantages; prime, ultimate thanks must set how|8.28|5.79|2003001|exportiimporto #1|3|pants|2|Men|231|oughtpriable|large|0495794mint838982794|rosy|Ton|Unknown|21|esen stcallyese| +4695|AAAAAAAAGFCBAAAA|2000-10-27||Ladies will not give deeper australian farmers. Contents know immediate, local years. Routine, separate friends fall either racial, northe|2.34|1.38|2003001|edu packamalg #2|4|swimwear|1|Women|538|eingprianti|medium|0495794mint838982794|powder|Pallet|Unknown|62|antin stcallyese| +4696|AAAAAAAAIFCBAAAA|1997-10-27|1999-10-27|Public things stumble above worlds. Technical kinds shall get at present scientists. V|4.71|2.07|10011005|amalgamalgamalg #5|11|disk drives|10|Electronics|285|antieingable|N/A|38pink59967128989615|saddle|Case|Unknown|62|callyn stcallyese| +4697|AAAAAAAAIFCBAAAA|1999-10-28|2001-10-26|Able artists love all classical, roman runs. British childre|9.93|8.44|10011005|edu packedu pack #2|11|athletic|4|Shoes|285|antieingable|medium|22907631thistle93367|black|Bunch|Unknown|5|ationn stcallyese| +4698|AAAAAAAAIFCBAAAA|2001-10-27||Able artists love all classical, roman runs. British childre|2.02|8.44|10016007|corpamalgamalg #7|11|wireless|10|Electronics|120|antieingable|N/A|25346529frosted25724|violet|Tbl|Unknown|32|eingn stcallyese| +4699|AAAAAAAALFCBAAAA|1997-10-27||Engines should go again as a years; british masters m|8.41|5.97|2004002|edu packimporto #2|4|sports-apparel|2|Men|129|n stableought|medium|93949989894685pale33|seashell|Carton|Unknown|20|n stn stcallyese| +4700|AAAAAAAAMFCBAAAA|1997-10-27|2000-10-26|Rich models believe to the concepts. In particular contemporary prisoners come on the questions; about willing sources used to afford below once additional males|2.45|0.73|5003001|exportischolar #1|3|pop|5|Music|587|ationeinganti|N/A|422042763961ivory510|violet|Cup|Unknown|25|barbarationese| +4701|AAAAAAAAMFCBAAAA|2000-10-27||Rich models believe to the concepts. In particular contemporary prisoners come on the questions; about willing sources used to afford below once additional males|3.70|0.73|3001002|amalgexporti #2|3|newborn|3|Children|587|ationeinganti|small|422042763961ivory510|lavender|Carton|Unknown|19|oughtbarationese| +4702|AAAAAAAAOFCBAAAA|1997-10-27|1999-10-27|Detailed friends assess from a trees. Common, royal windows might not think alone in a years. Proper colleagues would pick of course. So protective years play financial experiments. Enough major |6.21|3.10|1003001|exportiamalg #1|3|maternity|1|Women|748|eingeseation|extra large|3105377910002sky3470|goldenrod|Tsp|Unknown|4|ablebarationese| +4703|AAAAAAAAOFCBAAAA|1999-10-28|2001-10-26|Great chemicals would drain with a resources. Layers see better per a bags. Natural forces discover relatively japanese, |4.45|2.31|1003001|importoscholar #2|3|country|5|Music|240|bareseable|N/A|82612royal3963050274|saddle|Unknown|Unknown|88|pribarationese| +4704|AAAAAAAAOFCBAAAA|2001-10-27||Great chemicals would drain with a resources. Layers see better per a bags. Natural forces discover relatively japanese, |1.40|0.92|10010013|univamalgamalg #13|10|memory|10|Electronics|240|bareseable|N/A|82612royal3963050274|salmon|Ounce|Unknown|57|esebarationese| +4705|AAAAAAAABGCBAAAA|1997-10-27||Later deep offender|48.67|42.82|3004002|edu packexporti #2|4|school-uniforms|3|Children|38|eingpri|medium|59741yellow767381712|rosy|Ton|Unknown|15|antibarationese| +4706|AAAAAAAACGCBAAAA|1997-10-27|2000-10-26|Wonderful, int|5.94|4.75|8016007|corpmaxi #7|16|golf|8|Sports|293|prin stable|N/A|7purple8190591161038|smoke|Each|Unknown|45|callybarationese| +4707|AAAAAAAACGCBAAAA|2000-10-27||Wonderful, int|5.54|4.76|3004002|edu packexporti #2|16|school-uniforms|3|Children|293|prin stable|large|24268312896754rosy61|grey|Tbl|Unknown|12|ationbarationese| +4708|AAAAAAAAEGCBAAAA|1997-10-27|1999-10-27|Very long engines may clarify. Other principles could confirm merely good lovers; s|63.15|56.83|7004009|edu packbrand #9|4|curtains/drapes|7|Home|978|eingationn st|N/A|87053olive1344414569|magenta|Pound|Unknown|1|eingbarationese| +4709|AAAAAAAAEGCBAAAA|1999-10-28|2001-10-26|Very long engines may clarify. Other principles could confirm merely good lovers; s|2.70|2.29|5002002|importoscholar #2|2|country|5|Music|978|eingationn st|N/A|87053olive1344414569|violet|Lb|Unknown|19|n stbarationese| +4710|AAAAAAAAEGCBAAAA|2001-10-27||Processes|1.86|0.74|5002002|edu packedu pack #1|4|athletic|4|Shoes|978|eingationn st|extra large|3609747spring1944448|slate|Tsp|Unknown|20|baroughtationese| +4711|AAAAAAAAHGCBAAAA|1997-10-27||Very blank arguments remember in a magistrates. Great, black adults consider more real, decisive countries. Away f|2.86|2.43|4001002|amalgedu pack #2|1|womens|4|Shoes|641|oughtesecally|medium|475933plum5295012908|red|Unknown|Unknown|87|oughtoughtationese| +4712|AAAAAAAAIGCBAAAA|1997-10-27|2000-10-26|German answers like. Cultural, general characters go always into a hands. Necessary, impossible girls make. Slow, attractive regulations must|2.26|1.87|4002001|importoedu pack #1|2|mens|4|Shoes|47|ationese|medium|20299372014107peach4|wheat|Carton|Unknown|37|ableoughtationese| +4713|AAAAAAAAIGCBAAAA|2000-10-27||Positive measures must say|4.14|1.87|4002001|scholarbrand #2|2|custom|6|Jewelry|572|ableationanti|N/A|49puff29127463835925|rose|Pound|Unknown|16|prioughtationese| +4714|AAAAAAAAKGCBAAAA|1997-10-27|1999-10-27|Normal, mental machines take. Real,|4.25|3.69|7004005|edu packbrand #5|4|curtains/drapes|7|Home|339|n stpripri|N/A|coral715744225828764|sky|Oz|Unknown|37|eseoughtationese| +4715|AAAAAAAAKGCBAAAA||2001-10-26|Always industrial systems should not believe of course hard fingers. Either overall drawings appear increa|4.44|1.82||amalgscholar #2|||5|Music||eingantipri|N/A|1216899linen22829136|||Unknown|26|antioughtationese| +4716|AAAAAAAAKGCBAAAA|2001-10-27||Also occasional discussions can cover real beans. Thoroughly economic operators give still. Sources could not assist impossible problems. Considerable, difficult hotels|1.81|0.61|2002001|importoimporto #1|2|shirts|2|Men|358|eingantipri|large|1216899linen22829136|pale|Ton|Unknown|30|callyoughtationese| +4717|AAAAAAAANGCBAAAA|1997-10-27||Following categories give. Light requirements come in short short, linguistic tasks. Single, only securities say. Journals ought to know inappropriate intentions.|3.35|1.97|5002002|importoscholar #2|2|country|5|Music|467|ationcallyese|N/A|010powder25480154001|royal|Dram|Unknown|21|ationoughtationese| +4718|AAAAAAAAOGCBAAAA|1997-10-27|2000-10-26|Stages choose ever to the companies. Certain, national issues respond also reports. International, alive pupils get associated, conscious difficulties. High interests marry very high hands. There far |7.68|5.91|8008009|namelessnameless #9|8|outdoor|8|Sports|485|antieingese|N/A|55lace93445642614689|moccasin|Unknown|Unknown|14|eingoughtationese| +4719|AAAAAAAAOGCBAAAA|2000-10-27||Stages choose ever to the companies. Certain, national issues respond also reports. International, alive pupils get associated, conscious difficulties. High interests marry very high hands. There far |34.07|21.80|6006008|corpcorp #8|6|rings|6|Jewelry|149|n steseought|N/A|55lace93445642614689|powder|Ton|Unknown|6|n stoughtationese| +4720|AAAAAAAAAHCBAAAA|1997-10-27|1999-10-27|Conventional areas might create vaguely somewhere|3.91|1.91|1001001|amalgamalg #1|1|dresses|1|Women|61|oughtcally|petite|44160735255754light3|slate|Bunch|Unknown|6|barableationese| +4721|AAAAAAAAAHCBAAAA|1999-10-28|2001-10-26|Hundreds may not mean else girls. Seriously small birds could deny mutually. Ways get real inches. Long-term, old hours must establish now wi|2.19|1.91|1001001|importoscholar #2|2|country|5|Music|61|oughtcally|N/A|673631888royal249006|orange|Each|Unknown|66|oughtableationese| +4722|AAAAAAAAAHCBAAAA|2001-10-27||Hundreds may not mean else girls. Seriously small birds could deny mutually. Ways get real inches. Long-term, old hours must establish now wi|2.22|0.99|1001001|namelesscorp #3|2|mens watch|6|Jewelry|27|ationable|N/A|673631888royal249006|powder|Pallet|Unknown|76|ableableationese| +4723|AAAAAAAADHCBAAAA|1997-10-27||Other, tired years create tomorrow by a fingers. Cells could not snap. Modern, light waters take details; often residential agents prepare further right walls; presents supply to a|52.93|40.22|4004002|edu packedu pack #2|4|athletic|4|Shoes|531|oughtprianti|large|7white21302467115490|snow|Oz|Unknown|38|priableationese| +4724|AAAAAAAAEHCBAAAA|1997-10-27|2000-10-26|Devices know also so normal waters. Labour times say. Teachers tell |0.26|0.11|8012005|importomaxi #5|12|guns|8|Sports|293|prin stable|N/A|965drab7994285886810|linen|Pallet|Unknown|97|eseableationese| +4725|AAAAAAAAEHCBAAAA|2000-10-27||Families like obviously by the chains. In particular usual songs incorporate again in a conditions. Senior, common passengers help lazily; special, due men ca|4.67|0.11|8012005|scholaramalgamalg #17|15|portable|10|Electronics|293|prin stable|N/A|965drab7994285886810|rosy|Box|Unknown|33|antiableationese| +4726|AAAAAAAAGHCBAAAA|1997-10-27|1999-10-27|Hundreds would meet regardless german, foreign scien|9.77|5.17|9003011|exportimaxi #11|3|computers|9|Books|895|antin steing|N/A|58sandy1284203716347|saddle|Gram|Unknown|19|callyableationese| +4727|AAAAAAAAGHCBAAAA|1999-10-28|2001-10-26|Hundreds would meet regardless german, foreign scien|8.32|5.17|7001008|amalgbrand #8|3|bathroom|7|Home|240|antin steing|N/A|58sandy1284203716347|grey|Oz|Unknown|47|ationableationese| +4728|AAAAAAAAGHCBAAAA|2001-10-27||Hundreds would meet regardless german, foreign scien|1.98|0.85|7001008|amalgedu pack #1|3|womens|4|Shoes|240|antin steing|petite|58sandy1284203716347|maroon|Ton|Unknown|23|eingableationese| +4729|AAAAAAAAJHCBAAAA|1997-10-27||Yet left techniques phone very about other years. Costly, social examples estimate otherwise only western institutions. C|0.62|0.28|5001002|amalgscholar #2|1|rock|5|Music|57|ationanti|N/A|30874884064midnight4|puff|Pound|Unknown|41|n stableationese| +4730|AAAAAAAAKHCBAAAA|1997-10-27|2000-10-26|Technical standards might tackle. Excellent, similar women go correct, major companies; mutual, poor perceptions ought to make n|8.76|7.70|6006003|corpcorp #3|6|rings|6|Jewelry|133|pripriought|N/A|99773686473medium748|snow|Bundle|Unknown|23|barpriationese| +4731|AAAAAAAAKHCBAAAA|2000-10-27||Technical standards might tackle. Excellent, similar women go correct, major companies; mutual, poor perceptions ought to make n|4.78|7.70|6006003|importoscholar #2|6|country|5|Music|133|pripriought|N/A|99773686473medium748|moccasin|Carton|Unknown|67|oughtpriationese| +4732|AAAAAAAAMHCBAAAA|1997-10-27|1999-10-27|Problems protect followers. Particular, particular controls can consider later wide, high risks. Bars would consider always social markets. New instructions may sit initial terms; farm|6.45|4.32|7006003|corpbrand #3|6|rugs|7|Home|298|eingn stable|N/A|7tan2115980127304196|khaki|Pound|Unknown|59|ablepriationese| +4733|AAAAAAAAMHCBAAAA|1999-10-28|2001-10-26|Illegal, late carers should build totally processes; illegally simple cells shou|1.61|4.32|6004006|edu packcorp #6|6|bracelets|6|Jewelry|251|eingn stable|N/A|7tan2115980127304196|papaya|N/A|Unknown|16|pripriationese| +4734|AAAAAAAAMHCBAAAA|2001-10-27||Independent techniques leav|58.48|4.32|6004006|edu packamalg #1|4|swimwear|1|Women|251|eingn stable|medium|7tan2115980127304196|peru|Each|Unknown|7|esepriationese| +4735|AAAAAAAAPHCBAAAA|1997-10-27||Local, national judges should design hous|4.03|2.65|3002002|importoexporti #2|2|infants|3|Children|411|oughtoughtese|large|73peach5268233139270|pink|Dozen|Unknown|5|antipriationese| +4736|AAAAAAAAAICBAAAA|1997-10-27|2000-10-26|Necessary roads agree with the girls; measures should not end |4.65|2.41|2002001|importoimporto #1|2|shirts|2|Men|316|callyoughtpri|medium|5006374187076tomato1|tomato|Case|Unknown|95|callypriationese| +4737|AAAAAAAAAICBAAAA|2000-10-27||Programmes talk with a costs. Too involved assets try for example successful cities. Arms must look increasingly. Only, long details say now. Logical, victorian ide|1.61|1.40|2002001|importoamalg #2|2|fragrances|1|Women|35|antipri|medium|25538olive6447319086|yellow|Gram|Unknown|51|ationpriationese| +4738|AAAAAAAACICBAAAA|1997-10-27|1999-10-27|Free, major examples may not tell so final keys. Quick terms should read. Reasons teach. S|4.39|3.95|3002001|importoexporti #1|2|infants|3|Children|313|prioughtpri|petite|124peru5757865205134|thistle|Tsp|Unknown|29|eingpriationese| +4739|AAAAAAAACICBAAAA|1999-10-28|2001-10-26|Patterns let however real facilities. Naturally single variations used to save initially. Operations leave later t|7.04|3.95|3002001|brandmaxi #12|2|reference|9|Books|114|prioughtpri|N/A|124peru5757865205134|magenta|Tbl|Unknown|2|n stpriationese| +4740|AAAAAAAACICBAAAA|2001-10-27||Patterns let however real facilities. Naturally single variations used to save initially. Operations leave later t|2.15|3.95|3002001|amalgexporti #1|1|newborn|3|Children|665|anticallycally|petite|7royal79145585775607|saddle|Carton|Unknown|2|bareseationese| +4741|AAAAAAAAFICBAAAA|1997-10-27||Inevitable, local risks emphasize c|3.52|2.71|8007002|brandnameless #2|7|hockey|8|Sports|256|callyantiable|N/A|388982white199750017|papaya|Gram|Unknown|43|oughteseationese| +4742|AAAAAAAAGICBAAAA|1997-10-27|2000-10-26|Centres must announce soon briefly personal efforts. Agricultural, worthwhile elections wi|1.48|0.54|5003001|exportischolar #1|3|pop|5|Music|113|prioughtought|N/A|837blue7850160802423|firebrick|Bundle|Unknown|25|ableeseationese| +4743|AAAAAAAAGICBAAAA|2000-10-27||States must know ago for the police. Labour modules gain also following consumers. Environmental, ancient points take explici|1.92|0.86|5003001|edu packunivamalg #4|14|sports|9|Books|588|prioughtought|N/A|837blue7850160802423|violet|Oz|Unknown|15|prieseationese| +4744|AAAAAAAAIICBAAAA|1997-10-27|1999-10-27|Able, alternative police shall not give so other complaints. There complex |9.61|5.86|4002001|importoedu pack #1|2|mens|4|Shoes|82|ableeing|small|7smoke42145404607260|aquamarine|Bunch|Unknown|45|eseeseationese| +4745|AAAAAAAAIICBAAAA|1999-10-28|2001-10-26|Able, alternative police shall not give so other complaints. There complex |1.31|5.86|4002001|edu packimporto #2|4|sports-apparel|2|Men|560|barcallyanti|medium|8572090540antique502|puff|Tbl|Unknown|63|antieseationese| +4746|AAAAAAAAIICBAAAA|2001-10-27||Firms should not prevent. Then elaborate days think gently never running tears. Ways ensure upstairs. Unfair securities|2.57|1.13|9013007|exportiunivamalg #7|4|self-help|9|Books|560|barcallyanti|N/A|8572090540antique502|snow|Gram|Unknown|22|callyeseationese| +4747|AAAAAAAALICBAAAA|1997-10-27||Wide, firm offices may signify yet eligible periods. Terms compensate empty, new circumstances; negotiations used to make then major users. True, aggressive l|9.90|7.52|8010008|univmaxi #8|10|pools|8|Sports|143|prieseought|N/A|30smoke3771145980286|bisque|Bundle|Unknown|3|ationeseationese| +4748|AAAAAAAAMICBAAAA|1997-10-27|2000-10-26|In addition appropriate methods es|1.37|0.90|5004001|edu packscholar #1|4|classical|5|Music|141|oughteseought|N/A|991636lime9676680770|indian|Ounce|Unknown|11|eingeseationese| +4749|AAAAAAAAMICBAAAA|2000-10-27||In addition appropriate methods es|6.99|0.90|5004001|importobrand #6|4|costume|6|Jewelry|141|oughteseought|N/A|991636lime9676680770|rosy|Tsp|Unknown|11|n steseationese| +4750|AAAAAAAAOICBAAAA|1997-10-27|1999-10-27|Groups can call; children should d|2.38|1.57|10004008|edu packunivamalg #8|4|audio|10|Electronics|593|prin stanti|N/A|9225670329025rosy627|red|Dozen|Unknown|6|barantiationese| +4751|AAAAAAAAOICBAAAA|1999-10-28|2001-10-26|Different, special classes follow enough. Urgent, different police see then new birds. |1.57|1.57|10004008|edu packamalg #2|4|swimwear|1|Women|593|prin stanti|large|4072702351467peach53|puff|Tsp|Unknown|26|oughtantiationese| +4752|AAAAAAAAOICBAAAA|2001-10-27||Different, special classes follow enough. Urgent, different police see then new birds. |9.39|3.94|10008010|namelessunivamalg #10|4|scanners|10|Electronics|283|prin stanti|N/A|276papaya04049287923|sienna|Ounce|Unknown|72|ableantiationese| +4753|AAAAAAAABJCBAAAA|1997-10-27||Absolute ears shall expect hence important workers. Locations ought to infer even to the governors; wonderful results fashion; real, democratic |6.08|3.58|1001002|amalgamalg #2|1|dresses|1|Women|529|n stableanti|large|yellow35842791716279|sienna|Dozen|Unknown|60|priantiationese| +4754|AAAAAAAACJCBAAAA|1997-10-27|2000-10-26|English activities explain old principles. Years make other, little governors; able materials shrink grimly by the wishes. Wide months prevent so in a adults. Functions cannot ask blind events. St|1.00|0.77|8014005|edu packmaxi #5|14|tennis|8|Sports|397|ationn stpri|N/A|01papaya019806106722|turquoise|Each|Unknown|26|eseantiationese| +4755|AAAAAAAACJCBAAAA|2000-10-27||Later private strengths would not bother jobs; similar islands used to know more substantial years. Specialist words fetch perhaps to a authorities. Things review. Great terms will w|4.11|0.77|4004002|edu packedu pack #2|14|athletic|4|Shoes|416|callyoughtese|extra large|01papaya019806106722|floral|Cup|Unknown|46|antiantiationese| +4756|AAAAAAAAEJCBAAAA|1997-10-27|1999-10-27|Young, whi|5.38|1.93|9012005|importounivamalg #5|12|home repair|9|Books|231|oughtpriable|N/A|93867064121sienna439|plum|Bundle|Unknown|11|callyantiationese| +4757|AAAAAAAAEJCBAAAA|1999-10-28|2001-10-26|Ago unhappy countries mak|8.98|1.93|9012005|exportiimporto #2|12|pants|2|Men|231|oughtpriable|medium|01057384749728smoke2|turquoise|Ton|Unknown|87|ationantiationese| +4758|AAAAAAAAEJCBAAAA|2001-10-27||Ago unhappy countries mak|0.74|1.93|9012005|univbrand #7|10|jewelry boxes|6|Jewelry|231|oughtpriable|N/A|916642577royal260569|plum|Tbl|Unknown|11|eingantiationese| +4759|AAAAAAAAHJCBAAAA|1997-10-27||Systematically poor girls solve so available, general witnesses; windows could compete generally early men. Final eyes shall know|2.55|2.21|2003002|exportiimporto #2|3|pants|2|Men|272|ableationable|petite|89924197467red152775|saddle|N/A|Unknown|90|n stantiationese| +4760|AAAAAAAAIJCBAAAA|1997-10-27|2000-10-26|Possible, political men may hang. Civil women can celebrate only central author|9.49|4.93|10016001|corpamalgamalg #1|16|wireless|10|Electronics|445|antieseese|N/A|1188salmon5667697443|seashell|Box|Unknown|29|barcallyationese| +4761|AAAAAAAAIJCBAAAA|2000-10-27||Difficult, likely inst|4.54|2.40|1002002|importoamalg #2|2|fragrances|1|Women|506|antieseese|small|1188salmon5667697443|spring|Lb|Unknown|72|oughtcallyationese| +4762|AAAAAAAAKJCBAAAA|1997-10-27|1999-10-27|Still existing judges expect eventually ethnic schools. Films prevent chronic millions. Foreign photographs used to use more back individuals. Missiles become just happy, arti|8.38|6.70|4004001|edu packedu pack #1|4|athletic|4|Shoes|227|ationableable|medium|418297blue2581102690|sandy|Lb|Unknown|41|ablecallyationese| +4763|AAAAAAAAKJCBAAAA|1999-10-28|2001-10-26|Still existing judges expect eventually ethnic schools. Films prevent chronic millions. Foreign photographs used to use more back individuals. Missiles become just happy, arti|0.72|6.70|4004001|importoedu pack #2|2|mens|4|Shoes|721|ationableable|extra large|7688288papaya6008767|puff|Case|Unknown|3|pricallyationese| +4764|AAAAAAAAKJCBAAAA|2001-10-27||Still existing judges expect eventually ethnic schools. Films prevent chronic millions. Foreign photographs used to use more back individuals. Missiles become just happy, arti|2.78|6.70|6004005|edu packcorp #5|2|bracelets|6|Jewelry|721|oughtableation|N/A|7688288papaya6008767|white|Unknown|Unknown|49|esecallyationese| +4765|AAAAAAAANJCBAAAA|1997-10-27||Experts retaliate more good, foreign workers. Now surprised forests like. Very conscious weaknesses like m|4.79|3.97|5004002|edu packscholar #2|4|classical|5|Music|239|n stpriable|N/A|112780820091545peru6|slate|Tbl|Unknown|6|anticallyationese| +4766|AAAAAAAAOJCBAAAA|1997-10-27|2000-10-26|Traditional, small |2.31|1.20|8007007|brandnameless #7|7|hockey|8|Sports|59|n stanti|N/A|magenta6102663638338|rose|Dozen|Unknown|11|callycallyationese| +4767|AAAAAAAAOJCBAAAA|2000-10-27||Present, successful features should not search otherwise confident times. Usual, central teachers ought to borrow. Real addresses disappoint only. Generally clear structures o|2.95|1.23|8007007|importoscholar #2|7|country|5|Music|59|n stanti|N/A|4913248674salmon9661|pale|Ton|Unknown|49|ationcallyationese| +4768|AAAAAAAAAKCBAAAA|1997-10-27|1999-10-27|Main, new workers find always. A little fellow comm|71.28|36.35|2002001|importoimporto #1|2|shirts|2|Men|72|ableation|medium|24209tan530634196150|papaya|Dozen|Unknown|10|eingcallyationese| +4769|AAAAAAAAAKCBAAAA|1999-10-28|2001-10-26|Female elections double on the years. Stones strengthen supposed shares. Ec|4.06|36.35|10001010|amalgunivamalg #10|1|cameras|10|Electronics|72|ableation|N/A|5471698131297pink707|chartreuse|Tsp|Unknown|4|n stcallyationese| +4770|AAAAAAAAAKCBAAAA|2001-10-27||Female elections double on the years. Stones strengthen supposed shares. Ec|88.85|36.35|3004001|edu packexporti #1|4|school-uniforms|3|Children|124|eseableought|N/A|379867902purple92934|red|Pallet|Unknown|7|barationationese| +4771|AAAAAAAADKCBAAAA|1997-10-27||Social, important guns shall ask b|0.13|0.08|3003002|exportiexporti #2|3|toddlers|3|Children|105|antibarought|N/A|033525497703877rose7|sienna|Unknown|Unknown|20|oughtationationese| +4772|AAAAAAAAEKCBAAAA|1997-10-27|2000-10-26|Especially new sets should not deepen natural, mass interests. Other, proper industries confirm never. Frequent sentences should move only ga|3.49|1.67|10010013|univamalgamalg #13|10|memory|10|Electronics|656|callyantically|N/A|7chiffon243269731693|spring|Oz|Unknown|8|ableationationese| +4773|AAAAAAAAEKCBAAAA|2000-10-27||Remarks must scrap usually problems. Physical police can consult well always large matters. Neighbouring, french courts could build dema|1.51|1.31|1003002|exportiamalg #2|10|maternity|1|Women|53|callyantically|medium|7chiffon243269731693|lace|Tbl|Unknown|4|priationationese| +4774|AAAAAAAAGKCBAAAA|1997-10-27|1999-10-27|Upper points teach good other, blind troops. More confident personnel work about apart big songs. Unusual, |5.96|4.35|4001001|amalgedu pack #1|1|womens|4|Shoes|850|barantieing|medium|6burnished9793424426|peach|N/A|Unknown|3|eseationationese| +4775|AAAAAAAAGKCBAAAA|1999-10-28|2001-10-26|Upper points teach good other, blind troops. More confident personnel work about apart big songs. Unusual, |5.30|4.35|9010012|univunivamalg #12|1|travel|9|Books|593|prin stanti|N/A|6burnished9793424426|papaya|Cup|Unknown|35|antiationationese| +4776|AAAAAAAAGKCBAAAA|2001-10-27||Upper points teach good other, blind troops. More confident personnel work about apart big songs. Unusual, |2.81|4.35|2002001|importoimporto #1|2|shirts|2|Men|267|prin stanti|petite|6burnished9793424426|turquoise|Bundle|Unknown|23|callyationationese| +4777|AAAAAAAAJKCBAAAA|1997-10-27||Things might not provide in a features. Countries break seldom critical, social issues. Main, |25.82|8.26|4001002|amalgedu pack #2|1|womens|4|Shoes|360|barcallypri|medium|911905557610369rose8|lemon|Gram|Unknown|53|ationationationese| +4778|AAAAAAAAKKCBAAAA|1997-10-27|2000-10-26|Later human topics may not make well young questions; firm, good rivers sustain even. Usually weak foods kill companies. Other, brief directors |1.90|1.65|6005003|scholarcorp #3|5|earings|6|Jewelry|322|ableablepri|N/A|17849yellow405046053|seashell|Dozen|Unknown|12|eingationationese| +4779|AAAAAAAAKKCBAAAA|2000-10-27||Later human topics may not make well young questions; firm, good rivers sustain even. Usually weak foods kill companies. Other, brief directors |7.68|6.22|6005003|edu packscholar #2|5|classical|5|Music|322|ableablepri|N/A|38681navy07627908380|lavender|Gross|Unknown|59|n stationationese| +4780|AAAAAAAAMKCBAAAA|1997-10-27|1999-10-27|Schools reduce gradually quick, ordinary defences. Pictures would discourage too other feet. Hours play already over the members. Still simi|6.99|2.23|6016005|corpbrand #5|16|consignment|6|Jewelry|451|oughtantiese|N/A|4057990070231851tan6|smoke|Oz|Unknown|12|bareingationese| +4781|AAAAAAAAMKCBAAAA|1999-10-28|2001-10-26|So very difficulties may say approximately new, rec|3.79|2.23|4002002|importoedu pack #2|16|mens|4|Shoes|138|oughtantiese|medium|4057990070231851tan6|drab|Each|Unknown|52|oughteingationese| +4782|AAAAAAAAMKCBAAAA|2001-10-27||Again past offices recover. Invisible, distant years pay emphatically really modern versions. Later other songs mount to a feet|2.30|2.23|4002002|scholarnameless #3|16|tables|7|Home|236|oughtantiese|N/A|515574seashell680492|thistle|Box|Unknown|10|ableeingationese| +4783|AAAAAAAAPKCBAAAA|1997-10-27||Growing, different minutes agree actually in accordance with a units. Necessary powers make even. Brown, high names would not say; sales must no|1.22|1.01|8013004|exportimaxi #4|13|sailing|8|Sports|44|eseese|N/A|39moccasin9802647830|royal|Box|Unknown|12|prieingationese| +4784|AAAAAAAAALCBAAAA|1997-10-27|2000-10-26|Separ|8.63|6.29|10015008|scholaramalgamalg #8|15|portable|10|Electronics|516|callyoughtanti|N/A|hot05243457989859680|linen|Pound|Unknown|5|eseeingationese| +4785|AAAAAAAAALCBAAAA|2000-10-27||Bad, different systems shall forget neither in the forests; social stairs would not set western, organic words; flat arms should not provide even years. Rare, commercial men travel ab|0.30|6.29|10015008|namelessbrand #6|8|lighting|7|Home|516|callyoughtanti|N/A|84332837snow65407158|spring|Bunch|Unknown|28|antieingationese| +4786|AAAAAAAACLCBAAAA|1997-10-27|1999-10-27|Old inches may not become just. T|3.53|1.62|9013011|exportiunivamalg #11|13|self-help|9|Books|405|antibarese|N/A|77saddle171468071463|yellow|Pound|Unknown|68|callyeingationese| +4787|AAAAAAAACLCBAAAA|1999-10-28|2001-10-26|Old inches may not become just. T|1.69|1.62|6013004|exportibrand #4|13|loose stones|6|Jewelry|334|antibarese|N/A|932084988063ghost966|snow|Bunch|Unknown|40|ationeingationese| +4788|AAAAAAAACLCBAAAA|2001-10-27||Old inches may not become just. T|2.27|1.62|6013004|brandbrand #9|13|decor|7|Home|384|eseeingpri|N/A|385846625366371dark3|metallic|Ton|Unknown|33|eingeingationese| +4789|AAAAAAAAFLCBAAAA|1997-10-27||Northern words used to overthrow spiritual, additional feet. Still |8.83|7.68|10015013|scholaramalgamalg #13|15|portable|10|Electronics|65|antically|N/A|612278862turquoise74|peru|Case|Unknown|1|n steingationese| +4790|AAAAAAAAGLCBAAAA|1997-10-27|2000-10-26|Acceptable hands might feel kinds. Plans should not help quietly capable, musical funds. Male va|34.75|13.55|3003001|exportiexporti #1|3|toddlers|3|Children|557|ationantianti|medium|486459peach508405138|gainsboro|Carton|Unknown|25|barn stationese| +4791|AAAAAAAAGLCBAAAA|2000-10-27||Right|3.63|2.32|4001002|amalgedu pack #2|1|womens|4|Shoes|557|ationantianti|extra large|486459peach508405138|peru|Box|Unknown|63|oughtn stationese| +4792|AAAAAAAAILCBAAAA|1997-10-27|1999-10-27|Strangers pay still only cer|1.27|0.91|10002016|importounivamalg #16|2|camcorders|10|Electronics|290|barn stable|N/A|124627papaya90747230|snow|Ton|Unknown|65|ablen stationese| +4793|AAAAAAAAILCBAAAA|1999-10-28|2001-10-26|Strangers pay still only cer|3.33|0.91|10002016|edu packamalg #2|4|swimwear|1|Women|290|barn stable|large|124627papaya90747230|rose|Lb|Unknown|44|prin stationese| +4794|AAAAAAAAILCBAAAA|2001-10-27||Answers could know clearly in the conclusions; here academic cells will|2.83|1.21|4004001|edu packedu pack #1|4|athletic|4|Shoes|826|barn stable|large|124627papaya90747230|white|Cup|Unknown|93|esen stationese| +4795|AAAAAAAALLCBAAAA|1997-10-27||To and fro public arguments must not come then awkwardly joint men; boring members would not let current|5.99|1.91|3002002|importoexporti #2|2|infants|3|Children|333|pripripri|extra large|7107416hot2492855175|red|Tsp|Unknown|33|antin stationese| +4796|AAAAAAAAMLCBAAAA|1997-10-27|2000-10-26|Eyes will appeal in a performances; normal developments would not take then. Electric reasons rise only even european legs. Police mig|1.43|1.18|2001001|amalgimporto #1|1|accessories|2|Men|6|cally|petite|5176772137orange7916|thistle|Pallet|Unknown|33|callyn stationese| +4797|AAAAAAAAMLCBAAAA|2000-10-27||Eyes will appeal in a performances; normal developments would not take then. Electric reasons rise only even european legs. Police mig|8.98|1.18|2001001|corpnameless #8|1|furniture|7|Home|387|cally|N/A|5176772137orange7916|saddle|Unknown|Unknown|28|ationn stationese| +4798|AAAAAAAAOLCBAAAA|1997-10-27|1999-10-27|Good, dynamic details make authorities; proceedings interrupt nowhere commercial, statistical animals. Procedures try subject jobs. Tears can endanger eggs. Probably nervous feat|1.93|1.58|5003001|exportischolar #1|3|pop|5|Music|359|n stantipri|N/A|8680538297peach23845|snow|Lb|Unknown|35|eingn stationese| +4799|AAAAAAAAOLCBAAAA|1999-10-28|2001-10-26|Good, dynamic details make authorities; proceedings interrupt nowhere commercial, statistical animals. Procedures try subject jobs. Tears can endanger eggs. Probably nervous feat|2.90|1.58|5003001|exportimaxi #12|3|computers|9|Books|359|n stantipri|N/A|4897962navajo0670272|peach|Lb|Unknown|2|n stn stationese| +4800|AAAAAAAAOLCBAAAA|2001-10-27||Good, dynamic details make authorities; proceedings interrupt nowhere commercial, statistical animals. Procedures try subject jobs. Tears can endanger eggs. Probably nervous feat|2.36|1.79|5004001|edu packscholar #1|4|classical|5|Music|359|n stantipri|N/A|11wheat7064588861146|brown|Case|Unknown|88|barbareingese| +4801|AAAAAAAABMCBAAAA|1997-10-27||Royal performances could ask for example in light of a children. Categories cut here. Strong, able models should not break in a terms; chief, open appearances will not di|2.27|0.74|1002002|importoamalg #2|2|fragrances|1|Women|125|antiableought|extra large|0saddle0199981592023|saddle|Bundle|Unknown|1|oughtbareingese| +4802|AAAAAAAACMCBAAAA|1997-10-27|2000-10-26|Again other changes woul|0.52|0.35|7008003|namelessbrand #3|8|lighting|7|Home|498|eingn stese|N/A|7926purple6939937769|navajo|Ton|Unknown|64|ablebareingese| +4803|AAAAAAAACMCBAAAA|2000-10-27||Again other changes woul|85.27|34.10|6012004|importobrand #4|12|costume|6|Jewelry|498|eingn stese|N/A|289547878571papaya26|hot|Oz|Unknown|29|pribareingese| +4804|AAAAAAAAEMCBAAAA|1997-10-27|1999-10-27|O|92.60|34.26|8008005|namelessnameless #5|8|outdoor|8|Sports|422|ableableese|N/A|75291288plum38520801|peru|Each|Unknown|66|esebareingese| +4805|AAAAAAAAEMCBAAAA|1999-10-28|2001-10-26|O|8.88|34.26|3002002|importoexporti #2|8|infants|3|Children|604|ableableese|extra large|75291288plum38520801|sandy|Unknown|Unknown|19|antibareingese| +4806|AAAAAAAAEMCBAAAA|2001-10-27||O|8.53|34.26|3002002|scholaramalgamalg #13|15|portable|10|Electronics|604|esebarcally|N/A|198burnished41326000|red|Gross|Unknown|13|callybareingese| +4807|AAAAAAAAHMCBAAAA|1997-10-27||S|8.96|7.43|6001008|amalgcorp #8|1|birdal|6|Jewelry|525|antiableanti|N/A|0629092almond4776923|steel|N/A|Unknown|30|ationbareingese| +4808|AAAAAAAAIMCBAAAA|1997-10-27|2000-10-26|Standards can forget countries; lengthy streets used to lead. Other engineers could obtain o|1.92|1.65|5004001|edu packscholar #1|4|classical|5|Music|5|anti|N/A|5sienna5310456441475|violet|N/A|Unknown|63|eingbareingese| +4809|AAAAAAAAIMCBAAAA|2000-10-27||Standards can forget countries; lengthy streets used to lead. Other engineers could obtain o|7.11|3.62|5004001|exporticorp #2|3|gold|6|Jewelry|56|anti|N/A|8916linen51862092310|thistle|Tsp|Unknown|20|n stbareingese| +4810|AAAAAAAAKMCBAAAA|1997-10-27|1999-10-27|Various, historic writers sign european, dramatic loans. Strange creatures get soon important, available techniques. Important years shall not know into an days. Here|1.68|1.10|8011001|amalgmaxi #1|11|archery|8|Sports|25|antiable|N/A|877248678577yellow84|rosy|Unknown|Unknown|39|baroughteingese| +4811|AAAAAAAAKMCBAAAA|1999-10-28|2001-10-26|Then current women would scream here american floors; wholly final feet can live onto a minerals. Women find months. Great, poor services may not confirm into a children; underlying|4.81|3.65|6002004|importocorp #4|11|diamonds|6|Jewelry|330|antiable|N/A|877248678577yellow84|purple|Cup|Unknown|27|oughtoughteingese| +4812|AAAAAAAAKMCBAAAA|2001-10-27||Fully western lists might consider social, dire|7.55|3.65|3003001|exportiexporti #1|3|toddlers|3|Children|86|antiable|small|877248678577yellow84|white|Bunch|Unknown|30|ableoughteingese| +4813|AAAAAAAANMCBAAAA|1997-10-27||Policies compensate more long eyes. Terrible, single res|6.60|4.68|9005002|scholarmaxi #2|5|history|9|Books|226|callyableable|N/A|3974212257299yellow0|rosy|Case|Unknown|3|prioughteingese| +4814|AAAAAAAAOMCBAAAA|1997-10-27|2000-10-26|Similar men should hope things. Numbers might not opt now organisers. Just false offers determine judges.|2.00|1.32|8006005|corpnameless #5|6|football|8|Sports|203|pribarable|N/A|610royal472028063759|green|Pallet|Unknown|45|eseoughteingese| +4815|AAAAAAAAOMCBAAAA|2000-10-27||Together good term|15.62|1.32|10016005|corpamalgamalg #5|16|wireless|10|Electronics|203|pribarable|N/A|7837800452608tomato8|lawn|Pound|Unknown|82|antioughteingese| +4816|AAAAAAAAANCBAAAA|1997-10-27|1999-10-27|||||exportischolar #1|3|pop||Music||priableable||9244pale964460291186|saddle||||| +4817|AAAAAAAAANCBAAAA|1999-10-28|2001-10-26|Previous issues will print much. Hastily odd councillors try. Other investments could discuss tories; options will apply good lists. Near british days cut finally appropriate representatives. Lef|4.50|3.19|5003001|exporticorp #2|3|gold|6|Jewelry|366|callycallypri|N/A|1pale581336617346026|slate|Box|Unknown|18|ationoughteingese| +4818|AAAAAAAAANCBAAAA|2001-10-27||Possible trees visit too signals. Other, international versions may know; figures avoid yet small unit|98.91|42.53|7012009|importonameless #9|3|paint|7|Home|366|callycallypri|N/A|26189072251273olive0|tan|Ounce|Unknown|15|eingoughteingese| +4819|AAAAAAAADNCBAAAA|1997-10-27||Serious studies reduce inadeq|55.87|17.87|5003002|exportischolar #2|3|pop|5|Music|100|barbarought|N/A|7olive66732773303125|magenta|Tsp|Unknown|28|n stoughteingese| +4820|AAAAAAAAENCBAAAA|1997-10-27|2000-10-26|Other, suitable professionals could not tell then to a men. Investments cannot tell. Groups might not go; now angry circumstances will become still b|4.69|1.92|1002001|importoamalg #1|2|fragrances|1|Women|310|baroughtpri|medium|98573snow97140465317|purple|Bunch|Unknown|33|barableeingese| +4821|AAAAAAAAENCBAAAA|2000-10-27||Enthusiastically likely children might use probably increased, proper children. Losses should wish now. Stories make. Mental suggestions levy hardly. |1.54|1.92|1002001|amalgimporto #2|1|accessories|2|Men|310|baroughtpri|large|98573snow97140465317|slate|Bunch|Unknown|65|oughtableeingese| +4822|AAAAAAAAGNCBAAAA|1997-10-27|1999-10-27|Complex dreams must wait forward local levels; huge rights move alm|4.10|2.29|4001001|amalgedu pack #1|1|womens|4|Shoes|264|esecallyable|petite|3563903939thistle413|smoke|Tbl|Unknown|4|ableableeingese| +4823|AAAAAAAAGNCBAAAA|1999-10-28|2001-10-26|Complex dreams must wait forward local levels; huge rights move alm|27.33|22.95|9011012|amalgunivamalg #12|11|cooking|9|Books|264|esecallyable|N/A|sandy078658793086243|steel|Each|Unknown|64|priableeingese| +4824|AAAAAAAAGNCBAAAA|2001-10-27||Gre|6.93|2.70|10010014|univamalgamalg #14|11|memory|10|Electronics|259|esecallyable|N/A|050378173sandy506528|white|Lb|Unknown|28|eseableeingese| +4825|AAAAAAAAJNCBAAAA|1997-10-27||All b|1.54|1.10|4003002|exportiedu pack #2|3|kids|4|Shoes|983|prieingn st|N/A|1766859305tan1287172|black|Lb|Unknown|32|antiableeingese| +4826|AAAAAAAAKNCBAAAA|1997-10-27|2000-10-26|Senior, daily organizations provide objectives. Probably serious investors ought to carry soon other old sons. Basic services could not make re|7.13|2.21|1002001|importoamalg #1|2|fragrances|1|Women|632|ableprically|petite|27329papaya362694578|navajo|Bunch|Unknown|14|callyableeingese| +4827|AAAAAAAAKNCBAAAA|2000-10-27||Senior, daily organizations provide objectives. Probably serious investors ought to carry soon other old sons. Basic services could not make re|4.45|2.21|1002001|amalgunivamalg #17|1|cameras|10|Electronics|632|ableprically|N/A|27329papaya362694578|tan|Ton|Unknown|53|ationableeingese| +4828|AAAAAAAAMNCBAAAA|1997-10-27|1999-10-27|British years|8.64|5.09|10016001|corpamalgamalg #1|16|wireless|10|Electronics|224|eseableable|N/A|27487532016wheat7012|thistle|N/A|Unknown|12|eingableeingese| +4829|AAAAAAAAMNCBAAAA|1999-10-28|2001-10-26|Big years set minutes. Indeed royal questions lead elaborate tonnes. Primitive words start physically economic standards. Toys shall not deal necessarily. Great, ad|2.91|5.09|2001002|amalgimporto #2|16|accessories|2|Men|646|callyesecally|medium|27487532016wheat7012|drab|Carton|Unknown|21|n stableeingese| +4830|AAAAAAAAMNCBAAAA|2001-10-27||Big years set minutes. Indeed royal questions lead elaborate tonnes. Primitive words start physically economic standards. Toys shall not deal necessarily. Great, ad|6.52|5.09|2001002|exportiedu pack #1|16|kids|4|Shoes|533|priprianti|large|27487532016wheat7012|azure|N/A|Unknown|17|barprieingese| +4831|AAAAAAAAPNCBAAAA|1997-10-27||Relatively unable cultures shall sell changes. Relative, accessible initiatives go indeed main areas. Responsible, statut|0.99|0.88|4004002|edu packedu pack #2|4|athletic|4|Shoes|269|n stcallyable|petite|9024632606rose258560|yellow|Oz|Unknown|65|oughtprieingese| +4832|AAAAAAAAAOCBAAAA|1997-10-27|2000-10-26|Actual arms must enable finally national, public times; stones aim other tensions. Often clean incentives produce on an|2.99|2.57|8004003|edu packnameless #3|4|camping|8|Sports|147|ationeseought|N/A|4thistle290276736612|drab|Bunch|Unknown|15|ableprieingese| +4833|AAAAAAAAAOCBAAAA|2000-10-27||Fundamental years should make still with a organizations. Areas look of course elements. Public instances shoot. Terrible examples take true flowers; serious newspapers become excellent, vulnerable pl|70.70|2.57|8004003|edu packimporto #2|4|sports-apparel|2|Men|849|ationeseought|medium|4thistle290276736612|turquoise|Unknown|Unknown|57|priprieingese| +4834|AAAAAAAACOCBAAAA|1997-10-27|1999-10-27|Environmental, obvious songs say nearly among a measures. Stars deliver above regional, normal guidelines; excellent, actual competitors can support today rare details. Actually considerable|1.95|0.99|3004001|edu packexporti #1|4|school-uniforms|3|Children|221|oughtableable|large|578017khaki813656969|thistle|Dozen|Unknown|13|eseprieingese| +4835|AAAAAAAACOCBAAAA|1999-10-28|2001-10-26|Environmental, obvious songs say nearly among a measures. Stars deliver above regional, normal guidelines; excellent, actual competitors can support today rare details. Actually considerable|4.28|0.99|1004002|edu packamalg #2|4|swimwear|1|Women|221|oughtableable|medium|578017khaki813656969|thistle|Ton|Unknown|30|antiprieingese| +4836|AAAAAAAACOCBAAAA|2001-10-27||Still famous types could use almost to the patients. Conven|3.68|0.99|5003001|exportischolar #1|3|pop|5|Music|221|oughtableable|N/A|578017khaki813656969|peru|Dram|Unknown|66|callyprieingese| +4837|AAAAAAAAFOCBAAAA|1997-10-27||However new|2.06|0.96|9004002|edu packmaxi #2|4|entertainments|9|Books|136|callypriought|N/A|63927015indian563651|sienna|Dozen|Unknown|31|ationprieingese| +4838|AAAAAAAAGOCBAAAA|1997-10-27|2000-10-26|Maybe odd cattle will not want that private relations. Here tory prisoners cannot think possible|0.61|0.23|1002001|importoamalg #1|2|fragrances|1|Women|146|callyeseought|medium|5saddle6637926818277|spring|Ton|Unknown|19|eingprieingese| +4839|AAAAAAAAGOCBAAAA|2000-10-27||Maybe odd cattle will not want that private relations. Here tory prisoners cannot think possible|1.20|1.00|1004002|edu packamalg #2|2|swimwear|1|Women|215|callyeseought|medium|5saddle6637926818277|moccasin|Ounce|Unknown|99|n stprieingese| +4840|AAAAAAAAIOCBAAAA|1997-10-27|1999-10-27|Just bitter topics build at a parents. Falls must not|7.25|2.90|10004013|edu packunivamalg #13|4|audio|10|Electronics|223|priableable|N/A|62917152101wheat5575|snow|Ounce|Unknown|19|bareseeingese| +4841|AAAAAAAAIOCBAAAA|1999-10-28|2001-10-26|Just bitter topics build at a parents. Falls must not|3.16|2.90|10004013|corpnameless #2|4|football|8|Sports|49|n stese|N/A|62917152101wheat5575|moccasin|Dozen|Unknown|47|oughteseeingese| +4842|AAAAAAAAIOCBAAAA|2001-10-27||Incidentally existing customers must read sad words; royal parents remember specifically great, difficult times. Temporary, satisfactory materials roll in a stat|1.61|1.17|10004013|importoexporti #1|4|infants|3|Children|528|n stese|small|62917152101wheat5575|puff|Box|Unknown|88|ableeseeingese| +4843|AAAAAAAALOCBAAAA|1997-10-27||Measurements would accept then so poor troubles. Tears should carry necessary sciences. Large, social toys claim general voices. Critical countries will not restore funny advantages. As wel|3.89|3.03|8003004|exportinameless #4|3|basketball|8|Sports|264|esecallyable|N/A|63582indian391667549|rosy|Cup|Unknown|32|prieseeingese| +4844|AAAAAAAAMOCBAAAA|1997-10-27|2000-10-26|Capitalist, independent communities think further. Prime, domestic years wash o|2.98|2.59|10013017|exportiamalgamalg #17|13|stereo|10|Electronics|298|eingn stable|N/A|7937663honeydew71652|honeydew|Tsp|Unknown|33|eseeseeingese| +4845|AAAAAAAAMOCBAAAA|2000-10-27||Other, old windows will|33.06|2.59|10010001|univamalgamalg #1|10|memory|10|Electronics|298|eingn stable|N/A|7violet9441552269232|smoke|Pallet|Unknown|23|antieseeingese| +4846|AAAAAAAAOOCBAAAA|1997-10-27|1999-10-27|Young, academic variables advertise so; also able skills would not know more clear requests. Inter alia furious experiences may no|4.88|1.90|2003001|exportiimporto #1|3|pants|2|Men|288|eingeingable|petite|77875493519481peach0|thistle|Ounce|Unknown|56|callyeseeingese| +4847|AAAAAAAAOOCBAAAA|1999-10-28|2001-10-26|Young, academic variables advertise so; also able skills would not know more clear requests. Inter alia furious experiences may no|2.93|2.34|5003002|exportischolar #2|3|pop|5|Music|324|eingeingable|N/A|77875493519481peach0|seashell|Tbl|Unknown|21|ationeseeingese| +4848|AAAAAAAAOOCBAAAA|2001-10-27||Young, academic variables advertise so; also able skills would not know more clear requests. Inter alia furious experiences may no|1.57|2.34|8013009|exportimaxi #9|3|sailing|8|Sports|216|callyoughtable|N/A|119salmon28847475558|purple|Dram|Unknown|26|eingeseeingese| +4849|AAAAAAAABPCBAAAA|1997-10-27||Sure socia|1.78|1.17|7015010|scholarnameless #10|15|tables|7|Home|111|oughtoughtought|N/A|3693red9693188307213|lace|Cup|Unknown|86|n steseeingese| +4850|AAAAAAAACPCBAAAA|1997-10-27|2000-10-26|Gentle surveys cannot take hard releva|41.50|15.77|2003001|exportiimporto #1|3|pants|2|Men|160|barcallyought|economy|525slate366480135113|pale|Bundle|Unknown|7|barantieingese| +4851|AAAAAAAACPCBAAAA|2000-10-27||All confident affairs attend for a clubs. Far desirable shoulders commend apparently easy arguments. Houses keep in addition sorry children. Small hand|87.59|70.07|2003001|edu packamalg #2|3|swimwear|1|Women|160|barcallyought|large|525slate366480135113|cyan|Gram|Unknown|7|oughtantieingese| +4852|AAAAAAAAEPCBAAAA|1997-10-27|1999-10-27|Patient services will find also developing, social developers. Othe|0.55|0.45|9013005|exportiunivamalg #5|13|self-help|9|Books|362|ablecallypri|N/A|440tan97513859990786|red|Tbl|Unknown|8|ableantieingese| +4853|AAAAAAAAEPCBAAAA|1999-10-28|2001-10-26|Patient services will find also developing, social developers. Othe|0.88|0.29|3002002|importoexporti #2|2|infants|3|Children|543|ablecallypri|economy|471738609sienna15680|misty|Carton|Unknown|7|priantieingese| +4854|AAAAAAAAEPCBAAAA|2001-10-27||Excellent, political lovers reflect both. Also good tories could |1.55|0.29|3002001|importoexporti #1|2|infants|3|Children|543|ablecallypri|small|947676355903navajo04|violet|Unknown|Unknown|28|eseantieingese| +4855|AAAAAAAAHPCBAAAA|1997-10-27||Wet problems can meet etc by a rules; good, great applications would not attract early ideas. Right, nice members will afford quite popular others. Afraid lists must not wea|8.45|7.18|10006011|corpunivamalg #11|6|musical|10|Electronics|235|antipriable|N/A|linen783353285631954|olive|Each|Unknown|60|antiantieingese| +4856|AAAAAAAAIPCBAAAA|1997-10-27|2000-10-26|Sizes cannot press now. Free, real earnings roll tragically hard grants; also increased farmers used to examine factors. Long cars will not succeed respectively alone sales. Re|75.24|66.96|1003001|exportiamalg #1|3|maternity|1|Women|263|pricallyable|petite|58084798coral8871944|pale|Case|Unknown|18|callyantieingese| +4857|AAAAAAAAIPCBAAAA|2000-10-27||Able quarters open. Poor arms could eat for a potatoes. Quietly litt|9.27|66.96|4003002|exportiedu pack #2|3|kids|4|Shoes|68|pricallyable|petite|1436827118478spring8|peru|Box|Unknown|52|ationantieingese| +4858|AAAAAAAAKPCBAAAA|1997-10-27|1999-10-27|Together valid methods must limit; mild, american policemen |5.82|3.43|8003009|exportinameless #9|3|basketball|8|Sports|354|eseantipri|N/A|2731455yellow5251945|pale|Tbl|Unknown|26|eingantieingese| +4859|AAAAAAAAKPCBAAAA|1999-10-28|2001-10-26|Particular, easy policies give concerned, fresh n|3.02|2.56|8003009|edu packamalg #2|4|swimwear|1|Women|354|eseantipri|extra large|6454765535lime422803|royal|Ounce|Unknown|56|n stantieingese| +4860|AAAAAAAAKPCBAAAA|2001-10-27||Again holy activities|1.92|0.99|8003009|edu packnameless #1|4|glassware|7|Home|354|eseantipri|N/A|844775505353674tan52|salmon|Dram|Unknown|21|barcallyeingese| +4861|AAAAAAAANPCBAAAA|1997-10-27||Continuing, different areas could not manufacture alone also fatal services. Else final trains should know conversely in common double houses; pleasant, other activities may make rarely hote|1.48|0.84|1003002|exportiamalg #2|3|maternity|1|Women|543|prieseanti|petite|90959562ghost4869215|turquoise|Bundle|Unknown|61|oughtcallyeingese| +4862|AAAAAAAAOPCBAAAA|1997-10-27|2000-10-26|Today social roles could not understand more single men. True persons can receive rather extended eyes. Much other years flourish also together with the |1.92|1.40|3004001|edu packexporti #1|4|school-uniforms|3|Children|950|barantin st|medium|464navajo03601908822|peach|Case|Unknown|20|ablecallyeingese| +4863|AAAAAAAAOPCBAAAA|2000-10-27||Today social roles could not understand more single men. True persons can receive rather extended eyes. Much other years flourish also together with the |2.72|1.40|5004002|edu packscholar #2|4|classical|5|Music|20|barable|N/A|8206steel18460523271|rose|Dozen|Unknown|93|pricallyeingese| +4864|AAAAAAAAAADBAAAA|1997-10-27|1999-10-27|Sons could find out of the proceedings. Jobs give educational, young bars. Doors go sometimes at least public mountains. Texts oust theref|4.52|1.76|6004001|edu packcorp #1|4|bracelets|6|Jewelry|311|oughtoughtpri|N/A|11331pale51894005991|navy|Dozen|Unknown|5|esecallyeingese| +4865|AAAAAAAAAADBAAAA|1999-10-28|2001-10-26|Types provide so. Of course communist papers advise later contemporary transactions. |7.54|1.76|6004001|exportibrand #6|3|kids|7|Home|772|oughtoughtpri|N/A|11331pale51894005991|snow|Bunch|Unknown|29|anticallyeingese| +4866|AAAAAAAAAADBAAAA|2001-10-27||Types provide so. Of course communist papers advise later contemporary transactions. |8.71|1.76|6004001|edu packimporto #1|4|sports-apparel|2|Men|772|oughtoughtpri|petite|11331pale51894005991|purple|Gram|Unknown|10|callycallyeingese| +4867|AAAAAAAADADBAAAA|1997-10-27||A|3.72|2.26|7014008|edu packnameless #8|14|glassware|7|Home|444|eseeseese|N/A|0736485723pale278772|spring|Each|Unknown|24|ationcallyeingese| +4868|AAAAAAAAEADBAAAA|1997-10-27|2000-10-26|Far british products could like mutually. Also other|0.19|0.10|1004001|edu packamalg #1|4|swimwear|1|Women|15|antiought|small|0528856537525violet8|steel|N/A|Unknown|78|eingcallyeingese| +4869|AAAAAAAAEADBAAAA|2000-10-27||Far british products could like mutually. Also other|1.83|0.82|1004001|corpmaxi #10|4|parenting|9|Books|484|antiought|N/A|8869188slate27593233|moccasin|Gross|Unknown|26|n stcallyeingese| +4870|AAAAAAAAGADBAAAA|1997-10-27|1999-10-27|Ways could not use well about scottish years. Kilometres suit legs. Long, pale times smell negative, fine elections. At all single eyes would answer unusual, cultural con|9.14|6.76|3003001|exportiexporti #1|3|toddlers|3|Children|921|oughtablen st|large|62moccasin9372925744|sandy|Bundle|Unknown|4|barationeingese| +4871|AAAAAAAAGADBAAAA|1999-10-28|2001-10-26|Ways could not use well about scottish years. Kilometres suit legs. Long, pale times smell negative, fine elections. At all single eyes would answer unusual, cultural con|5.14|6.76|9005012|scholarmaxi #12|5|history|9|Books|921|oughtablen st|N/A|37orange693645434817|peach|Bunch|Unknown|19|oughtationeingese| +4872|AAAAAAAAGADBAAAA|2001-10-27||Simply working thanks must set newly cards. Residents drop to a options. Political conclusions shall park there|7.00|2.38|6012001|importobrand #1|12|costume|6|Jewelry|921|oughtablen st|N/A|67186smoke2619231634|pink|Unknown|Unknown|10|ableationeingese| +4873|AAAAAAAAJADBAAAA|1997-10-27||Modern women cannot produce police. All l|4.75|2.51|1001002|amalgamalg #2|1|dresses|1|Women|189|n steingought|small|94869182950607cream8|steel|Unknown|Unknown|15|priationeingese| +4874|AAAAAAAAKADBAAAA|1997-10-27|2000-10-26|Days let then|4.08|2.32|1004001|edu packamalg #1|4|swimwear|1|Women|278|eingationable|medium|220737459683maroon13|papaya|Ton|Unknown|24|eseationeingese| +4875|AAAAAAAAKADBAAAA|2000-10-27||Tickets could not share great rights. Well central hands avoid best financial arguments. Problems should work employers. Also particular languages would turn backwards hills. Recent standards m|6.86|2.32|1004001|importoscholar #2|2|country|5|Music|278|eingationable|N/A|220737459683maroon13|red|Tbl|Unknown|11|antiationeingese| +4876|AAAAAAAAMADBAAAA|1997-10-27|1999-10-27|Hundreds would sleep twice beautiful individuals. Kind of real mines record just full, good benefits; important, medieval stairs write cuts; black, difficult women teach hastily together legal|5.65|1.92|10011015|amalgamalgamalg #15|11|disk drives|10|Electronics|439|n stpriese|N/A|09183105868281pink52|olive|Bunch|Unknown|31|callyationeingese| +4877|AAAAAAAAMADBAAAA|1999-10-28|2001-10-26|Hundreds would sleep twice beautiful individuals. Kind of real mines record just full, good benefits; important, medieval stairs write cuts; black, difficult women teach hastily together legal|1.87|1.92|2001002|amalgimporto #2|1|accessories|2|Men|439|n stpriese|medium|09183105868281pink52|tomato|Each|Unknown|26|ationationeingese| +4878|AAAAAAAAMADBAAAA|2001-10-27||European, distinctive weeks would not work public, small museums. Local properties ought to appoint securities; odd, commercial cars might not move mainly economic arms; back contributions like ne|2.65|1.92|2001002|exportiedu pack #1|1|kids|4|Shoes|245|n stpriese|extra large|33yellow679325556331|puff|Bundle|Unknown|52|eingationeingese| +4879|AAAAAAAAPADBAAAA|1997-10-27||Ears must get almost by a centre|3.86|2.89|8008010|namelessnameless #10|8|outdoor|8|Sports|509|n stbaranti|N/A|292074sandy509097968|red|Bundle|Unknown|20|n stationeingese| +4880|AAAAAAAAABDBAAAA|1997-10-27|2000-10-26|Long, overseas symbols must not for|8.44|5.14|8015001|scholarmaxi #1|15|fishing|8|Sports|393|prin stpri|N/A|740833green516897769|purple|Dozen|Unknown|31|bareingeingese| +4881|AAAAAAAAABDBAAAA|2000-10-27||Social, hot females change constantly so personal legs. Suddenly common writings may feel months. So actual prisoners cannot give in a policies. Free offices catch a lit|48.57|22.82|10004003|edu packunivamalg #3|15|audio|10|Electronics|393|prin stpri|N/A|spring51003512926583|seashell|Ton|Unknown|33|oughteingeingese| +4882|AAAAAAAACBDBAAAA|1997-10-27|1999-10-27|Twice recent conditions inform agai|6.04|2.05|8010003|univmaxi #3|10|pools|8|Sports|401|oughtbarese|N/A|8956895251smoke41235|gainsboro|Gross|Unknown|40|ableeingeingese| +4883|AAAAAAAACBDBAAAA|1999-10-28|2001-10-26|Pol|7.88|2.05|8010003|namelessunivamalg #5|10|scanners|10|Electronics|401|oughtbarese|N/A|8956895251smoke41235|khaki|Bundle|Unknown|1|prieingeingese| +4884|AAAAAAAACBDBAAAA|2001-10-27||Pol|1.70|0.76|4003001|exportiedu pack #1|10|kids|4|Shoes|396|callyn stpri|medium|8956895251smoke41235|rosy|Pallet|Unknown|13|eseeingeingese| +4885|AAAAAAAAFBDBAAAA|1997-10-27||Fascinating, |4.19|3.47|6015006|scholarbrand #6|15|custom|6|Jewelry|127|ationableought|N/A|507violet82524476216|pale|Dram|Unknown|32|antieingeingese| +4886|AAAAAAAAGBDBAAAA|1997-10-27|2000-10-26|Sweet securities see a little in short large shareholders; already reasonable hands use|1.11|0.63|8015007|scholarmaxi #7|15|fishing|8|Sports|2|able|N/A|grey6114199918580636|peru|Pound|Unknown|77|callyeingeingese| +4887|AAAAAAAAGBDBAAAA|2000-10-27||Sweet securities see a little in short large shareholders; already reasonable hands use|7.74|0.63|5004002|edu packscholar #2|4|classical|5|Music|477|ationationese|N/A|grey6114199918580636|grey|Unknown|Unknown|29|ationeingeingese| +4888|AAAAAAAAIBDBAAAA|1997-10-27|1999-10-27|As previous bodies would maintain too in a images. Suddenly determined implications shall deal western members. Fundamental pages sit with the americans. Exceptio|3.40|2.24|4003001|exportiedu pack #1|3|kids|4|Shoes|54|eseanti|small|3576980790281pale581|rosy|Pound|Unknown|14|eingeingeingese| +4889|AAAAAAAAIBDBAAAA|1999-10-28|2001-10-26|As previous bodies would maintain too in a images. Suddenly determined implications shall deal western members. Fundamental pages sit with the americans. Exceptio|2.89|0.98|3004002|edu packexporti #2|4|school-uniforms|3|Children|54|eseanti|small|3576980790281pale581|midnight|Dozen|Unknown|2|n steingeingese| +4890|AAAAAAAAIBDBAAAA|2001-10-27||As previous bodies would maintain too in a images. Suddenly determined implications shall deal western members. Fundamental pages sit with the americans. Exceptio|4.03|2.01|10002012|importounivamalg #12|2|camcorders|10|Electronics|70|baration|N/A|3576980790281pale581|navy|Cup|Unknown|12|barn steingese| +4891|AAAAAAAALBDBAAAA|1997-10-27||Fo|1.39|1.22|7003002|exportibrand #2|3|kids|7|Home|272|ableationable|N/A|801goldenrod01231835|antique|Ton|Unknown|44|oughtn steingese| +4892|AAAAAAAAMBDBAAAA|1997-10-27|2000-10-26|Obvious, other police can defend. Specific, exciting seats might make on a organisms. Agencies revert indoors educational services. Case|6.30|4.91|2004001|edu packimporto #1|4|sports-apparel|2|Men|263|pricallyable|medium|719240612sandy319437|orchid|Dram|Unknown|6|ablen steingese| +4893|AAAAAAAAMBDBAAAA|2000-10-27||Old g|3.26|1.30|5002002|importoscholar #2|4|country|5|Music|986|pricallyable|N/A|719240612sandy319437|salmon|Bundle|Unknown|20|prin steingese| +4894|AAAAAAAAOBDBAAAA|1997-10-27|1999-10-27|Patterns|0.47|0.28|1004001|edu packamalg #1|4|swimwear|1|Women|215|antioughtable|small|96219turquoise346216|spring|Tbl|Unknown|67|esen steingese| +4895|AAAAAAAAOBDBAAAA|1999-10-28|2001-10-26|Pleasant, big conclusions look in conjunction with the ages. Bodies would not rest always to the institutions. Main, soviet professionals must call open friendly visitors. Reports decide for a clo|2.36|0.28|1004001|edu packunivamalg #12|14|sports|9|Books|96|antioughtable|N/A|96219turquoise346216|slate|Tbl|Unknown|4|antin steingese| +4896|AAAAAAAAOBDBAAAA|2001-10-27||Pleasant, big conclusions look in conjunction with the ages. Bodies would not rest always to the institutions. Main, soviet professionals must call open friendly visitors. Reports decide for a clo|0.63|0.28|1004001|corpcorp #1|14|rings|6|Jewelry|41|antioughtable|N/A|96219turquoise346216|plum|N/A|Unknown|29|callyn steingese| +4897|AAAAAAAABCDBAAAA|1997-10-27||Once firm events ought to speak applications. Labour subjects could not work f|4.81|2.78|5003002|exportischolar #2|3|pop|5|Music|212|ableoughtable|N/A|341350114973911mint2|seashell|Pound|Unknown|12|ationn steingese| +4898|AAAAAAAACCDBAAAA|1997-10-27|2000-10-26|Eyes can call wide, blue arts. Eyes take still like a places. Now naked beings should think recently groups. Generous stories reveal central results. |1.57|1.16|3001001|amalgexporti #1|1|newborn|3|Children|531|oughtprianti|large|tomato34854559299454|steel|N/A|Unknown|13|eingn steingese| +4899|AAAAAAAACCDBAAAA|2000-10-27||Alone british problems must get social, similar criticisms. Constitutional victims should tell. Views shall not explain anxiously essential, corporate ends. Good, good companies must hear|7.11|1.16|3002002|importoexporti #2|2|infants|3|Children|531|oughtprianti|extra large|tomato34854559299454|tomato|Gross|Unknown|29|n stn steingese| +4900|AAAAAAAAECDBAAAA|1997-10-27|1999-10-27|Main employees must provide growing, national politicians. Finally prime values see just in a circumstances. Various awards would like then dry, co|3.82|3.43|10010005|univamalgamalg #5|10|memory|10|Electronics|222|ableableable|N/A|8829074511yellow6743|purple|Box|Unknown|39|barbarn stese| +4901|AAAAAAAAECDBAAAA|1999-10-28|2001-10-26|Main employees must provide growing, national politicians. Finally prime values see just in a circumstances. Various awards would like then dry, co|9.05|5.15|10010005|univamalgamalg #6|10|memory|10|Electronics|222|ableableable|N/A|503005deep7765450166|chiffon|N/A|Unknown|23|oughtbarn stese| +4902|AAAAAAAAECDBAAAA|2001-10-27||Claims might not go advisory thousands; advisers must see now. Sweet organizations kill definitely true, surprising models|9.88|5.15|9007007|brandmaxi #7|7|reference|9|Books|222|ableableable|N/A|5409455navajo5348367|wheat|Ton|Unknown|31|ablebarn stese| +4903|AAAAAAAAHCDBAAAA|1997-10-27||Fresh, mixed conditions may n|9.52|3.23|6009008|maxicorp #8|9|womens watch|6|Jewelry|843|prieseeing|N/A|78236920202hot980592|sky|Cup|Unknown|25|pribarn stese| +4904|AAAAAAAAICDBAAAA|1997-10-27|2000-10-26|Average, above sentences should not care home years. Reactions come unfortunately full, capable sessions; dom|0.61|0.31|7004005|edu packbrand #5|4|curtains/drapes|7|Home|713|prioughtation|N/A|19710779092royal0758|purple|Each|Unknown|47|esebarn stese| +4905|AAAAAAAAICDBAAAA|2000-10-27||Average, above sentences should not care home years. Reactions come unfortunately full, capable sessions; dom|7.41|0.31|7004005|exportiamalg #2|3|maternity|1|Women|713|prioughtation|large|6945518910980snow469|tomato|Pound|Unknown|13|antibarn stese| +4906|AAAAAAAAKCDBAAAA|1997-10-27|1999-10-27|Often red years see good humans. Defe|5.74|3.04|2001001|amalgimporto #1|1|accessories|2|Men|304|esebarpri|petite|583074683992spring37|lawn|Oz|Unknown|25|callybarn stese| +4907|AAAAAAAAKCDBAAAA|1999-10-28|2001-10-26|Often red years see good humans. Defe|1.44|1.15|1003002|exportiamalg #2|3|maternity|1|Women|188|esebarpri|petite|418798909151powder71|yellow|Oz|Unknown|52|ationbarn stese| +4908|AAAAAAAAKCDBAAAA|2001-10-27||Red agreements cannot get eventually. Only, particular teachers benefit more spiritual symptoms. Women feel equally advanc|3.06|1.07|1003002|exportiedu pack #1|3|kids|4|Shoes|349|esebarpri|medium|418798909151powder71|linen|Gram|Unknown|18|eingbarn stese| +4909|AAAAAAAANCDBAAAA|1997-10-27||Women lose unwittingly new organisers. Absolute men reduce. Centuries must not|4.37|1.79|6015006|scholarbrand #6|15|custom|6|Jewelry|121|oughtableought|N/A|1131slate52310789546|green|Unknown|Unknown|36|n stbarn stese| +4910|AAAAAAAAOCDBAAAA|1997-10-27|2000-10-26|Well late friends reply correctly early new bene|0.80|0.67|3003001|exportiexporti #1|3|toddlers|3|Children|397|ationn stpri|small|420metallic411180326|slate|Ton|Unknown|5|baroughtn stese| +4911|AAAAAAAAOCDBAAAA|2000-10-27||Local, likely routes quit indeed only northern appearances. Sorry, prospective mothers replace tonight with a respects. Polis|1.28|0.67|5003002|exportischolar #2|3|pop|5|Music|397|ationn stpri|N/A|420metallic411180326|chartreuse|Case|Unknown|21|oughtoughtn stese| +4912|AAAAAAAAADDBAAAA|1997-10-27|1999-10-27|Numbers take serious, christian lips. Blue objects flow only quite immediate countr|7.65|3.36|9016005|corpunivamalg #5|16|mystery|9|Books|967|ationcallyn st|N/A|sienna92656687345131|sandy|Lb|Unknown|27|ableoughtn stese| +4913|AAAAAAAAADDBAAAA|1999-10-28|2001-10-26|Effective, inde|7.99|7.19|7002004|importobrand #4|16|bedding|7|Home|282|ationcallyn st|N/A|sienna92656687345131|light|Oz|Unknown|41|prioughtn stese| +4914|AAAAAAAAADDBAAAA|2001-10-27||Similar, wrong forces enable in the benefits. Somewhat different hands round windows. Keen roles cannot grow pretty strict, irish children. European pupil|5.10|1.98|7002004|amalgmaxi #5|11|archery|8|Sports|47|ationese|N/A|sienna92656687345131|ghost|Dram|Unknown|98|eseoughtn stese| +4915|AAAAAAAADDDBAAAA|1997-10-27||Either other thanks|4.44|1.86|2003002|exportiimporto #2|3|pants|2|Men|657|ationantically|economy|0709221ivory22213003|salmon|Pallet|Unknown|48|antioughtn stese| +4916|AAAAAAAAEDDBAAAA|1997-10-27|2000-10-26|Chemicals circumvent only other police. Leading, strong groups make respectively gently great events. Immediat|1.97|1.69|9013009|exportiunivamalg #9|13|self-help|9|Books|202|ablebarable|N/A|7pale122397121320763|indian|Carton|Unknown|64|callyoughtn stese| +4917|AAAAAAAAEDDBAAAA|2000-10-27||Late safe themes work even. About typical farmers shall look overseas into a problems. As superior requirements can take ever cheap officers. Conservative effects may keep both helpful agents|3.78|1.69|5003002|exportischolar #2|3|pop|5|Music|202|ablebarable|N/A|62saddle934614815356|mint|Tbl|Unknown|3|ationoughtn stese| +4918|AAAAAAAAGDDBAAAA|1997-10-27|1999-10-27|Prizes should not overlap steady in a hundreds; british possibilities could penetrate more in a programmes. Mechanical diff|6.28|2.19|10007006|brandunivamalg #6|7|personal|10|Electronics|170|barationought|N/A|3539papaya7209311605|cyan|Case|Unknown|46|eingoughtn stese| +4919|AAAAAAAAGDDBAAAA|1999-10-28|2001-10-26|Prizes should not overlap steady in a hundreds; british possibilities could penetrate more in a programmes. Mechanical diff|9.53|6.86|9013012|exportiunivamalg #12|7|self-help|9|Books|222|barationought|N/A|3539papaya7209311605|smoke|Pallet|Unknown|5|n stoughtn stese| +4920|AAAAAAAAGDDBAAAA|2001-10-27||Prizes should not overlap steady in a hundreds; british possibilities could penetrate more in a programmes. Mechanical diff|5.82|6.86|5004001|edu packscholar #1|7|classical|5|Music|608|eingbarcally|N/A|28837127peach1509170|violet|Tbl|Unknown|38|barablen stese| +4921|AAAAAAAAJDDBAAAA|1997-10-27||Basically amazing years would avoid statutory, abl|1.29|0.55|4002002|importoedu pack #2|2|mens|4|Shoes|402|ablebarese|economy|036salmon82767644558|plum|N/A|Unknown|5|oughtablen stese| +4922|AAAAAAAAKDDBAAAA|1997-10-27|2000-10-26|Only aware particles get at a prices. Isolated, high schools want only to a grounds. Real remains will not combat just about silly powers. So able ways should say natural games. Again old rates talk. |94.21|73.48|6001003|amalgcorp #3|1|birdal|6|Jewelry|499|n stn stese|N/A|4053violet3257443457|navajo|Lb|Unknown|38|ableablen stese| +4923|AAAAAAAAKDDBAAAA|2000-10-27||Able, supposed year|4.76|73.48|6001003|exportiexporti #2|3|toddlers|3|Children|109|n stn stese|extra large|4053violet3257443457|salmon|Oz|Unknown|68|priablen stese| +4924|AAAAAAAAMDDBAAAA|1997-10-27|1999-10-27|Motives may not avoid animals; comparative contents must make in a customers. Similar women chase also interests. I|1.06|0.38|7003005|exportibrand #5|3|kids|7|Home|548|eingeseanti|N/A|8sienna4368720521713|slate|Carton|Unknown|24|eseablen stese| +4925|AAAAAAAAMDDBAAAA|1999-10-28|2001-10-26|Special, realistic friends will comply. Opening concentrations could not look too major, soviet proceedings. National children might get following, hig|9.56|0.38|7003005|exportiedu pack #2|3|kids|4|Shoes|548|eingeseanti|medium|8sienna4368720521713|dark|Pallet|Unknown|27|antiablen stese| +4926|AAAAAAAAMDDBAAAA|2001-10-27||Agents know su|41.71|0.38|7003005|exportiamalg #1|3|maternity|1|Women|314|eseoughtpri|medium|8sienna4368720521713|misty|Dozen|Unknown|66|callyablen stese| +4927|AAAAAAAAPDDBAAAA|1997-10-27||Systematic, early pairs shop again only due chiefs. Only, following pilots reflect relationships. Of course moral years change either; junior, past thousands |4.52|1.49|1004002|edu packamalg #2|4|swimwear|1|Women|357|ationantipri|medium|6847240cornflower124|puff|Carton|Unknown|3|ationablen stese| +4928|AAAAAAAAAEDBAAAA|1997-10-27|2000-10-26|Special arguments may pay. Vitally boring others bring fully blue, right circles. Countries would not crawl labour others. More able restrictions may not predict before new, other trousers. Definitel|9.58|4.11|3002001|importoexporti #1|2|infants|3|Children|34|esepri|extra large|280166840smoke151471|olive|Gram|Unknown|57|eingablen stese| +4929|AAAAAAAAAEDBAAAA|2000-10-27||States sit dearly soviet, tory problems. Historical eyes nod to the candidates; important goals could draw home. Early volumes kill also visitors. Wise, public s|1.59|0.62|3002001|edu packamalg #2|4|swimwear|1|Women|250|barantiable|medium|08415sky258950108470|lemon|Cup|Unknown|12|n stablen stese| +4930|AAAAAAAACEDBAAAA|1997-10-27|1999-10-27|Impossible rights shall release there strong, single matters. Average, sorry clients ap|2.71|2.43|5003001|exportischolar #1|3|pop|5|Music|239|n stpriable|N/A|32914850violet376351|rosy|Dozen|Unknown|58|barprin stese| +4931|AAAAAAAACEDBAAAA|1999-10-28|2001-10-26|Numbers consider in order german workers. Poo|1.68|2.43|2003002|exportiimporto #2|3|pants|2|Men|538|n stpriable|medium|32914850violet376351|royal|Oz|Unknown|3|oughtprin stese| +4932|AAAAAAAACEDBAAAA|2001-10-27||Here vulnerable countries shall not open historic weapons; brothers would swel|8.84|4.68|2003002|importoexporti #1|2|infants|3|Children|538|eingprianti|medium|32914850violet376351|cyan|Tsp|Unknown|61|ableprin stese| +4933|AAAAAAAAFEDBAAAA|1997-10-27||Personal students would lie middle-class, comparable women. Heroes wreck similarly. Almost specialist p|2.18|1.89|6015006|scholarbrand #6|15|custom|6|Jewelry|294|esen stable|N/A|4rosy354012853369036|saddle|N/A|Unknown|7|priprin stese| +4934|AAAAAAAAGEDBAAAA|1997-10-27|2000-10-26|Systems submit often priests. Publications shall close high friendly instruments. Levels look white countries. Human, close weeks say never civil, small collections. Tory, tr|8.58|3.77|8016005|corpmaxi #5|16|golf|8|Sports|281|oughteingable|N/A|3673482764590violet8|smoke|Tbl|Unknown|42|eseprin stese| +4935|AAAAAAAAGEDBAAAA|2000-10-27||Systems submit often priests. Publications shall close high friendly instruments. Levels look white countries. Human, close weeks say never civil, small collections. Tory, tr|1.69|0.74|7007006|brandbrand #6|7|decor|7|Home|70|oughteingable|N/A|3673482764590violet8|rose|Unknown|Unknown|55|antiprin stese| +4936|AAAAAAAAIEDBAAAA|1997-10-27|1999-10-27|Questions can dry almost together northern prop|0.64|0.30|7004007|edu packbrand #7|4|curtains/drapes|7|Home|368|eingcallypri|N/A|7641479sienna1480449|tan|Tsp|Unknown|61|callyprin stese| +4937|AAAAAAAAIEDBAAAA|1999-10-28|2001-10-26|Economic seconds must receive great images. Savings believe|7.54|2.56|10006008|corpunivamalg #8|6|musical|10|Electronics|368|eingcallypri|N/A|2344peru526911779658|orange|Case|Unknown|12|ationprin stese| +4938|AAAAAAAAIEDBAAAA|2001-10-27||Economic seconds must receive great images. Savings believe|1.82|2.56|10006008|amalgscholar #1|6|rock|5|Music|368|eingcallypri|N/A|peru8821104715763745|turquoise|Ounce|Unknown|18|eingprin stese| +4939|AAAAAAAALEDBAAAA|1997-10-27||Only, alone lips hear also fine, social standards; economies see as; single cultures suffer honestly remarkable tasks. Existing elections worry small, western months. Principal tiles may influence |2.97|1.24|6014004|edu packbrand #4|14|estate|6|Jewelry|376|callyationpri|N/A|5171545809458papaya8|sandy|Tsp|Unknown|4|n stprin stese| +4940|AAAAAAAAMEDBAAAA|1997-10-27|2000-10-26|Genetic days see in the months. New standards shall experiment general patients. Men ask in order far differences. Parts look more new, aware properties. Wrong, unk|1.90|1.59|4001001|amalgedu pack #1|1|womens|4|Shoes|781|oughteingation|extra large|952120596peach072564|royal|Bunch|Unknown|46|baresen stese| +4941|AAAAAAAAMEDBAAAA|2000-10-27||Genetic days see in the months. New standards shall experiment general patients. Men ask in order far differences. Parts look more new, aware properties. Wrong, unk|0.44|0.33|4001002|amalgedu pack #2|1|womens|4|Shoes|781|oughteingation|petite|9777218violet9185584|tomato|Bunch|Unknown|20|oughtesen stese| +4942|AAAAAAAAOEDBAAAA|1997-10-27|1999-10-27|Arrangements bet now just spiritual councils. Hard kids unravel still. Decisions supplement also only, original ideas; back, other difficulties stop international busines|0.52|0.36|10002013|importounivamalg #13|2|camcorders|10|Electronics|532|ableprianti|N/A|120168070thistle2645|tomato|Case|Unknown|5|ableesen stese| +4943|AAAAAAAAOEDBAAAA|1999-10-28|2001-10-26|Local, able premises arise flat, blind plates. More young programmes control only as yet other criteria; higher large a|2.59|1.39|10002013|exportiamalg #2|2|maternity|1|Women|81|ableprianti|extra large|671182302302000sky50|sky|Bunch|Unknown|45|priesen stese| +4944|AAAAAAAAOEDBAAAA|2001-10-27||Local, able premises arise flat, blind plates. More young programmes control only as yet other criteria; higher large a|3.42|1.60|4004001|edu packedu pack #1|2|athletic|4|Shoes|283|prieingable|economy|468seashell689413646|olive|Dram|Unknown|31|eseesen stese| +4945|AAAAAAAABFDBAAAA|1997-10-27||Southern, extraordinary telecommunications set now fine materials. White years lose later in a movements. Much familiar companies ought to combine too only fiscal|5.00|3.55|6011002|amalgbrand #2|11|semi-precious|6|Jewelry|51|oughtanti|N/A|6033101yellow2184064|turquoise|Gram|Unknown|65|antiesen stese| +4946|AAAAAAAACFDBAAAA|1997-10-27|2000-10-26|National months express truly in the votes; competent, liberal costs include international, red complaints. All essenti|33.51|17.42|4001001|amalgedu pack #1|1|womens|4|Shoes|592|ablen stanti|medium|2633875208454tan6205|navajo|Pound|Unknown|15|callyesen stese| +4947|AAAAAAAACFDBAAAA|2000-10-27||National months express truly in the votes; competent, liberal costs include international, red complaints. All essenti|12.28|7.85|4001001|edu packexporti #2|1|school-uniforms|3|Children|592|ablen stanti|large|9khaki89925647221727|white|Dram|Unknown|34|ationesen stese| +4948|AAAAAAAAEFDBAAAA|1997-10-27|1999-10-27|Councils sort good, firm negot|8.19|7.28|8003009|exportinameless #9|3|basketball|8|Sports|400|barbarese|N/A|12725lemon8140599545|spring|Box|Unknown|13|eingesen stese| +4949|AAAAAAAAEFDBAAAA|1999-10-28|2001-10-26|Councils sort good, firm negot|4.89|1.80|1002002|importoamalg #2|2|fragrances|1|Women|478|eingationese|N/A|2902109630seashell80|rosy|Gross|Unknown|59|n stesen stese| +4950|AAAAAAAAEFDBAAAA|2001-10-27||Councils sort good, firm negot|1.01|1.80|5004001|edu packscholar #1|4|classical|5|Music|122|ableableought|N/A|28158069987304white4|puff|Bunch|Unknown|17|barantin stese| +4951|AAAAAAAAHFDBAAAA|1997-10-27||Losses hide |1.65|0.89|8009002|maxinameless #2|9|optics|8|Sports|379|n stationpri|N/A|6161803orchid7815984|seashell|Lb|Unknown|53|oughtantin stese| +4952|AAAAAAAAIFDBAAAA|1997-10-27|2000-10-26|Then public workers ought to believe accordingly great social days; miles hold just northern components. Probably|0.15|0.10|2004001|edu packimporto #1|4|sports-apparel|2|Men|166|callycallyought|medium|55920660823white9209|grey|N/A|Unknown|7|ableantin stese| +4953|AAAAAAAAIFDBAAAA|2000-10-27||Then public workers ought to believe accordingly great social days; miles hold just northern components. Probably|6.65|5.18|1004002|edu packamalg #2|4|swimwear|1|Women|302|ablebarpri|small|55920660823white9209|violet|Cup|Unknown|40|priantin stese| +4954|AAAAAAAAKFDBAAAA|1997-10-27|1999-10-27|Obligations meet then hence general jour|3.87|3.21|2002001|importoimporto #1|2|shirts|2|Men|386|callyeingpri|large|98348438119801puff47|peru|Dozen|Unknown|29|eseantin stese| +4955|AAAAAAAAKFDBAAAA|1999-10-28|2001-10-26|Obligations meet then hence general jour|2.41|3.21|2002001|maxibrand #6|9|mattresses|7|Home|386|callyeingpri|N/A|98348438119801puff47|puff|Carton|Unknown|49|antiantin stese| +4956|AAAAAAAAKFDBAAAA|2001-10-27||Jewish seconds stay courts; black, other members help less deaths; then political areas go formally wi|5.23|3.21|2002001|exportiimporto #1|9|pants|2|Men|386|callyeingpri|petite|312firebrick00231500|deep|Dram|Unknown|55|callyantin stese| +4957|AAAAAAAANFDBAAAA|1997-10-27||Scientific orders shall head also movements. Else huge eyes ought to talk yards. Good, accessible eyes used to suppose modern expectations. Rare feet used to comply net s|9.18|6.15|5004002|edu packscholar #2|4|classical|5|Music|6|cally|N/A|099151095607royal922|puff|Unknown|Unknown|49|ationantin stese| +4958|AAAAAAAAOFDBAAAA|1997-10-27|2000-10-26|Relatively various nations iron middle, artistic miles; potential forces should strike problems. Eggs sell soon young conditi|9.98|6.08|10016012|corpamalgamalg #12|16|wireless|10|Electronics|292|ablen stable|N/A|6092salmon5372752342|spring|Gross|Unknown|55|eingantin stese| +4959|AAAAAAAAOFDBAAAA|2000-10-27||Local, above others may not enable creative, long details. New women reckon too overall lives. Toys would enjoy only in a courts. Major, familiar windows keep now by a documents.|1.95|6.08|10016012|exportiexporti #2|3|toddlers|3|Children|292|ablen stable|medium|6092salmon5372752342|orchid|Case|Unknown|13|n stantin stese| +4960|AAAAAAAAAGDBAAAA|1997-10-27|1999-10-27|Live single numbers know now favor|7.95|7.15|1004001|edu packamalg #1|4|swimwear|1|Women|261|oughtcallyable|large|477864685808tan98922|spring|Dozen|Unknown|26|barcallyn stese| +4961|AAAAAAAAAGDBAAAA|1999-10-28|2001-10-26|Live single numbers know now favor|0.39|7.15|1004001|amalgamalg #2|4|dresses|1|Women|17|oughtcallyable|petite|163goldenrod55282661|rosy|Case|Unknown|38|oughtcallyn stese| +4962|AAAAAAAAAGDBAAAA|2001-10-27||Chief leaves should not make thus likely communities. Artistic, new women raise however young tourists. Main bombs try really high, new pp.; stiffly difficult miles spend main, dutch plates. Surp|67.46|43.17|1003001|exportiamalg #1|4|maternity|1|Women|17|oughtcallyable|medium|163goldenrod55282661|rose|Case|Unknown|51|ablecallyn stese| +4963|AAAAAAAADGDBAAAA|1997-10-27||Dogs catch hot words. Outside expressions ask quite current needs. There democratic complaints should back loudly in a crowds. Amazing, large materials care very highly anxious years; both industria|2.91|0.98|7012004|importonameless #4|12|paint|7|Home|989|n steingn st|N/A|2714096352khaki13783|purple|Tsp|Unknown|53|pricallyn stese| +4964|AAAAAAAAEGDBAAAA|1997-10-27|2000-10-26|English, american yards shall cause thic|3.01|1.74|4004001|edu packedu pack #1|4|athletic|4|Shoes|343|priesepri|small|8328slate25272329778|salmon|Carton|Unknown|2|esecallyn stese| +4965|AAAAAAAAEGDBAAAA|2000-10-27||English, american yards shall cause thic|4.24|3.77|2004002|edu packimporto #2|4|sports-apparel|2|Men|343|priesepri|petite|36125495brown2135514|seashell|Dram|Unknown|35|anticallyn stese| +4966|AAAAAAAAGGDBAAAA|1997-10-27|1999-10-27|Still members disclose then common methods. Pools c|1.01|0.81|6013007|exportibrand #7|13|loose stones|6|Jewelry|145|antieseought|N/A|1716951815962yellow6|yellow|Pound|Unknown|6|callycallyn stese| +4967|AAAAAAAAGGDBAAAA|1999-10-28|2001-10-26|Internal, acceptable blocks must run in a girls; independent developments chase just types. General, practical cells can occur so next to a gardens. Common patients take estimated,|5.32|2.44|6013007|edu packimporto #2|4|sports-apparel|2|Men|409|antieseought|large|tan52599893304499353|misty|Cup|Unknown|19|ationcallyn stese| +4968|AAAAAAAAGGDBAAAA|2001-10-27||Local governors tell provisions. Women will|7.42|6.30|10001005|amalgunivamalg #5|4|cameras|10|Electronics|409|antieseought|N/A|66658203284236sky226|plum|Tbl|Unknown|23|eingcallyn stese| +4969|AAAAAAAAJGDBAAAA|1997-10-27||Atlanti|4.65|1.95|10005006|scholarunivamalg #6|5|karoke|10|Electronics|263|pricallyable|N/A|237689453492lime8951|metallic|Dram|Unknown|7|n stcallyn stese| +4970|AAAAAAAAKGDBAAAA|1997-10-27|2000-10-26|Consistent, post-war attitudes try again. Yet interested islands might not reduce offices. Available, academic days care low visitors. Vehicles used to take massive, united efforts|2.76|1.57|2002001|importoimporto #1|2|shirts|2|Men|969|n stcallyn st|large|3453473991849pink921|black|Ton|Unknown|13|barationn stese| +4971|AAAAAAAAKGDBAAAA|2000-10-27||Sanctions would not suppose|7.25|4.06|4001002|amalgedu pack #2|1|womens|4|Shoes|83|n stcallyn st|medium|7pale797981861181956|sky|Gross|Unknown|67|oughtationn stese| +4972|AAAAAAAAMGDBAAAA|1997-10-27|1999-10-27|Sessions could not feel now huge businesses. Late, original readers|9.81|5.59|3001001|amalgexporti #1|1|newborn|3|Children|92|ablen st|small|634464660471113red70|saddle|N/A|Unknown|53|ableationn stese| +4973|AAAAAAAAMGDBAAAA|1999-10-28|2001-10-26|Huge cards know then grounds. Weekends tell. Councillors w|0.52|0.32|7006004|corpbrand #4|1|rugs|7|Home|655|ablen st|N/A|63313191385wheat2984|thistle|Box|Unknown|5|priationn stese| +4974|AAAAAAAAMGDBAAAA|2001-10-27||Huge cards know then grounds. Weekends tell. Councillors w|2.88|1.35|7006004|exportiamalgamalg #11|13|stereo|10|Electronics|655|ablen st|N/A|5675maroon7228848009|puff|Dram|Unknown|18|eseationn stese| +4975|AAAAAAAAPGDBAAAA|1997-10-27||Certainly remaining flowers can wonder then just significant papers; places secure below as a bombs. Other, domestic members must allow very polite thi|0.60|0.28|9002008|importomaxi #8|2|business|9|Books|603|pribarcally|N/A|337rose3163980333473|moccasin|Ton|Unknown|95|antiationn stese| +4976|AAAAAAAAAHDBAAAA|1997-10-27|2000-10-26|Vast, low years might find for instance |2.67|1.84|8013007|exportimaxi #7|13|sailing|8|Sports|605|antibarcally|N/A|81puff48229620048415|thistle|Box|Unknown|28|callyationn stese| +4977|AAAAAAAAAHDBAAAA|2000-10-27||Vast, low years might find for instance |2.00|1.84|8013007|exportiimporto #2|3|pants|2|Men|605|antibarcally|small|81puff48229620048415|antique|Case|Unknown|8|ationationn stese| +4978|AAAAAAAACHDBAAAA|1997-10-27|1999-10-27|Reasonably direct interests turn. Certainly existing |1.86|0.89|5002001|importoscholar #1|2|country|5|Music|552|ableantianti|N/A|violet24688075250187|smoke|Ton|Unknown|5|eingationn stese| +4979|AAAAAAAACHDBAAAA|1999-10-28|2001-10-26|Reasonably direct interests turn. Certainly existing |42.80|0.89|4004002|edu packedu pack #2|4|athletic|4|Shoes|772|ableationation|large|4837733pale437521462|saddle|Pound|Unknown|27|n stationn stese| +4980|AAAAAAAACHDBAAAA|2001-10-27||Reasonably direct interests turn. Certainly existing |77.14|0.89|4004002|importoamalg #1|4|fragrances|1|Women|772|ableationation|large|4837733pale437521462|white|Pound|Unknown|50|bareingn stese| +4981|AAAAAAAAFHDBAAAA|1997-10-27||Then christian rules should take here new choices; hard, pale changes avoid sections. Now main metres can need necessarily in spite of a stories; late colours keep now into the charts. Seque|3.59|1.14|7001002|amalgbrand #2|1|bathroom|7|Home|122|ableableought|N/A|4738227343pink105206|sandy|Tbl|Unknown|8|oughteingn stese| +4982|AAAAAAAAGHDBAAAA|1997-10-27|2000-10-26|Even royal packages stop in a minutes. Possible purposes|8.13|3.73|7014003|edu packnameless #3|14|glassware|7|Home|547|ationeseanti|N/A|00861559peach6310265|snow|Carton|Unknown|23|ableeingn stese| +4983|AAAAAAAAGHDBAAAA|2000-10-27||Tiny ways might rest exactly. Different requirements build. Political, military users matter still large plans; also spare tests might express in the doctors. Successful|0.24|0.14|7014003|amalgscholar #2|1|rock|5|Music|547|ationeseanti|N/A|565beige726125091409|burlywood|Pallet|Unknown|42|prieingn stese| +4984|AAAAAAAAIHDBAAAA|1997-10-27|1999-10-27|Other fal|8.43|4.29|3004001|edu packexporti #1|4|school-uniforms|3|Children|59|n stanti|medium|40753slate9800078763|slate|Pound|Unknown|6|eseeingn stese| +4985|AAAAAAAAIHDBAAAA|1999-10-28|2001-10-26|Small trees will come really at the nations. Areas say easy in the|6.55|4.29|3004001|edu packmaxi #6|4|entertainments|9|Books|160|n stanti|N/A|56white8789361797433|papaya|Pallet|Unknown|13|antieingn stese| +4986|AAAAAAAAIHDBAAAA|2001-10-27||Minor nations suggest more model rights; recent names come surely angry films. Figures might b|5.32|2.71|3004001|exportiedu pack #1|4|kids|4|Shoes|160|n stanti|large|56white8789361797433|turquoise|Pound|Unknown|34|callyeingn stese| +4987|AAAAAAAALHDBAAAA|1997-10-27||Thoughts obtain countri|3.35|2.98|1004002|edu packamalg #2|4|swimwear|1|Women|721|oughtableation|economy|342768946slate810710|blush|Cup|Unknown|28|ationeingn stese| +4988|AAAAAAAAMHDBAAAA|1997-10-27|2000-10-26|Colleges may know closer in a seeds. Conditions fail higher dangerous fears. Changes answer. Selective, sad weeks can register just circumstances. Today gastric publishers can get by a procedures. |9.05|5.88|8007009|brandnameless #9|7|hockey|8|Sports|148|eingeseought|N/A|397006008seashell881|pink|Ounce|Unknown|56|eingeingn stese| +4989|AAAAAAAAMHDBAAAA|2000-10-27||Colleges may know closer in a seeds. Conditions fail higher dangerous fears. Changes answer. Selective, sad weeks can register just circumstances. Today gastric publishers can get by a procedures. |2.63|0.86|1003002|exportiamalg #2|7|maternity|1|Women|148|eingeseought|medium|15590spring754513794|orange|Bunch|Unknown|29|n steingn stese| +4990|AAAAAAAAOHDBAAAA|1997-10-27|1999-10-27|High, happy funds would not change more minutes; ancient representations ca|4.12|2.51|7007001|brandbrand #1|7|decor|7|Home|127|ationableought|N/A|420454sienna12171404|tan|Dozen|Unknown|9|barn stn stese| +4991|AAAAAAAAOHDBAAAA|1999-10-28|2001-10-26|Ever domestic ways might fill wild methods. Eyes must not want then. Different, considerable forms would not tell already corporate, de|7.30|||edu packamalg #2|||1||624||petite|420454sienna12171404|||||oughtn stn stese| +4992|AAAAAAAAOHDBAAAA|2001-10-27||Ever domestic ways might fill wild methods. Eyes must not want then. Different, considerable forms would not tell already corporate, de|3.54|1.41|1004002|exportiamalg #1|7|maternity|1|Women|624|eseablecally|large|81royal6210157720070|wheat|Gross|Unknown|60|ablen stn stese| +4993|AAAAAAAABIDBAAAA|1997-10-27||Workers let pr|1.17|0.95|7003004|exportibrand #4|3|kids|7|Home|303|pribarpri|N/A|3drab771796234928573|smoke|Ton|Unknown|48|prin stn stese| +4994|AAAAAAAACIDBAAAA|1997-10-27|2000-10-26|Also public names should choose spanish, local things; slow aims use elderly bills. Workers might choose home local windows. Weekly, relative|2.52|1.99|2003001|exportiimporto #1|3|pants|2|Men|247|ationeseable|medium|26376568456maroon250|hot|Dozen|Unknown|17|esen stn stese| +4995|AAAAAAAACIDBAAAA|2000-10-27||Also public names should choose spanish, local things; slow aims use elderly bills. Workers might choose home local windows. Weekly, relative|9.11|1.99|4002002|importoedu pack #2|2|mens|4|Shoes|247|ationeseable|petite|36880frosted51392451|lime|Tsp|Unknown|26|antin stn stese| +4996|AAAAAAAAEIDBAAAA|1997-10-27|1999-10-27|Ever british schools advise as personal machines. Properties explain then. Unable, dirty materials close inside to the streets. That is other books would not give links. Unions ought to guess publ|1.67|1.48|6005005|scholarcorp #5|5|earings|6|Jewelry|415|antioughtese|N/A|925622883metallic065|ghost|Case|Unknown|55|callyn stn stese| +4997|AAAAAAAAEIDBAAAA|1999-10-28|2001-10-26|Ever british schools advise as personal machines. Properties explain then. Unable, dirty materials close inside to the streets. That is other books would not give links. Unions ought to guess publ|0.14|0.07|4001002|amalgedu pack #2|1|womens|4|Shoes|415|antioughtese|medium|551439650577rosy3960|seashell|Oz|Unknown|4|ationn stn stese| +4998|AAAAAAAAEIDBAAAA|2001-10-27||Ever british schools advise as personal machines. Properties explain then. Unable, dirty materials close inside to the streets. That is other books would not give links. Unions ought to guess publ|6.02|0.07|10003001|exportiunivamalg #1|3|dvd/vcr players|10|Electronics|415|antioughtese|N/A|551439650577rosy3960|turquoise|Tbl|Unknown|20|eingn stn stese| +4999|AAAAAAAAHIDBAAAA|1997-10-27||Structural, possible organisations can experience too relations. High stars guarantee hence small issues. T|0.39|0.15|4003002|exportiedu pack #2|3|kids|4|Shoes|335|antipripri|medium|49612misty8584535594|pale|Case|Unknown|17|n stn stn stese| +5000|AAAAAAAAIIDBAAAA|1997-10-27|2000-10-26|Alternatives must handle badly. Strong instruments facilitate already enthusiastically public institutions. Red districts could not make merely irish, available prisoners|2.62|1.72|10005003|scholarunivamalg #3|5|karoke|10|Electronics|337|ationpripri|N/A|2995purple1395310765|lace|Ounce|Unknown|8|barbarbaranti| +5001|AAAAAAAAIIDBAAAA|2000-10-27||Free years should not solve then thi|0.88|1.72|10005003|amalgscholar #2|5|rock|5|Music|337|ationpripri|N/A|2995purple1395310765|tan|Bundle|Unknown|1|oughtbarbaranti| +5002|AAAAAAAAKIDBAAAA|1997-10-27|1999-10-27|Holidays may attract local days. Low, sympathetic teachers might not provide especially resources. Soviet matt|2.12|1.10|7002003|importobrand #3|2|bedding|7|Home|154|eseantiought|N/A|3551wheat64658627078|peru|Gram|Unknown|25|ablebarbaranti| +5003|AAAAAAAAKIDBAAAA|1999-10-28|2001-10-26|Holidays may attract local days. Low, sympathetic teachers might not provide especially resources. Soviet matt|1.04|1.10|7002003|amalgscholar #2|1|rock|5|Music|154|eseantiought|N/A|3551wheat64658627078|wheat|Dram|Unknown|22|pribarbaranti| +5004|AAAAAAAAKIDBAAAA|2001-10-27||Holidays may attract local days. Low, sympathetic teachers might not provide especially resources. Soviet matt|4.97|4.32|7002003|exportinameless #5|1|wallpaper|7|Home|154|eseantiought|N/A|7137royal96730557453|yellow|Gross|Unknown|13|esebarbaranti| +5005|AAAAAAAANIDBAAAA|1997-10-27||Other tonnes test; sufficient indians tak|9.38|8.34|4004002|edu packedu pack #2|4|athletic|4|Shoes|162|ablecallyought|medium|372043171323purple81|mint|Oz|Unknown|9|antibarbaranti| +5006|AAAAAAAAOIDBAAAA|1997-10-27|2000-10-26|Local findings should give local quarters. Perfect, other museums run clearly famous images. Courses believe soft|1.77|0.53|9013003|exportiunivamalg #3|13|self-help|9|Books|585|antieinganti|N/A|56forest782174516865|yellow|Lb|Unknown|57|callybarbaranti| +5007|AAAAAAAAOIDBAAAA|2000-10-27||High records think networks. Critics know to a others. Direct, great concepts achieve somewhat catholic questions. New, indian losses used to undertake only new years. Big facts used to define|8.95|5.72|7002008|importobrand #8|2|bedding|7|Home|78|eingation|N/A|25811306538406red294|yellow|Each|Unknown|42|ationbarbaranti| +5008|AAAAAAAAAJDBAAAA|1997-10-27|1999-10-27|Black, particular months should make deep children. Open standards reopen over at a policies. Dangerous contents might mean on a streets. Very general cars need so into a practitioners; members ensu|83.43|55.06|8010009|univmaxi #9|10|pools|8|Sports|499|n stn stese|N/A|31279949860159ivory8|peru|Ounce|Unknown|58|eingbarbaranti| +5009|AAAAAAAAAJDBAAAA|1999-10-28|2001-10-26|Other states may fix otherwise. More major e|3.28|1.11|1001002|amalgamalg #2|1|dresses|1|Women|976|callyationn st|medium|31279949860159ivory8|tan|Unknown|Unknown|29|n stbarbaranti| +5010|AAAAAAAAAJDBAAAA|2001-10-27||Other states may fix otherwise. More major e|4.82|3.47|2003001|exportiimporto #1|1|pants|2|Men|976|callyationn st|small|269turquoise83347102|seashell|Pound|Unknown|3|baroughtbaranti| +5011|AAAAAAAADJDBAAAA|1997-10-27||Only rich places ignore tonight forever tall patients; really official pati|2.23|1.49|6014004|edu packbrand #4|14|estate|6|Jewelry|40|barese|N/A|4seashell90889911761|tan|Gram|Unknown|52|oughtoughtbaranti| +5012|AAAAAAAAEJDBAAAA|1997-10-27|2000-10-26|Orange services identify already. Parents look only. All physical representatives get alternatively real,|5.26|2.84|1003001|exportiamalg #1|3|maternity|1|Women|522|ableableanti|large|09494462299208rosy97|powder|Tsp|Unknown|60|ableoughtbaranti| +5013|AAAAAAAAEJDBAAAA|2000-10-27||Very precise features place ahead hardly big transactions. Hands end worse after a facilities. Full, young countries plan talks. Shares he|2.68|2.41|1003001|brandunivamalg #16|7|personal|10|Electronics|522|ableableanti|N/A|09494462299208rosy97|ivory|Pallet|Unknown|19|prioughtbaranti| +5014|AAAAAAAAGJDBAAAA|1997-10-27|1999-10-27|Plants will not assist then expe|5.18|2.95|3004001|edu packexporti #1|4|school-uniforms|3|Children|762|ablecallyation|large|349grey0073101528587|mint|N/A|Unknown|53|eseoughtbaranti| +5015|AAAAAAAAGJDBAAAA|1999-10-28|2001-10-26|Plants will not assist then expe|8.26|2.72|4001002|amalgedu pack #2|1|womens|4|Shoes|762|ablecallyation|medium|349grey0073101528587|spring|Gross|Unknown|42|antioughtbaranti| +5016|AAAAAAAAGJDBAAAA|2001-10-27||Plants will not assist then expe|9.88|2.72|1002001|importoamalg #1|1|fragrances|1|Women|204|esebarable|small|530715364835rose0334|sienna|Pallet|Unknown|67|callyoughtbaranti| +5017|AAAAAAAAJJDBAAAA|1997-10-27||Problems compete with a sets. Interesting, automatic pounds tell complete hills. |1.20|1.06|9001002|amalgmaxi #2|1|arts|9|Books|812|ableoughteing|N/A|875peach247630015011|steel|Tbl|Unknown|3|ationoughtbaranti| +5018|AAAAAAAAKJDBAAAA|1997-10-27|2000-10-26|Educational margins continue new, american eyes. As new police establish and so on therefore right children. Relu|4.11|1.80|5002001|importoscholar #1|2|country|5|Music|151|oughtantiought|N/A|777448078302purple78|sienna|Gram|Unknown|86|eingoughtbaranti| +5019|AAAAAAAAKJDBAAAA|2000-10-27||Models may provide central, theoretical years; ideological, conventional problems prevent fairly at the affairs; together essential rooms go secondly ideas. Sudden c|7.53|4.74|5002001|importobrand #10|2|bedding|7|Home|151|oughtantiought|N/A|777448078302purple78|light|Tsp|Unknown|20|n stoughtbaranti| +5020|AAAAAAAAMJDBAAAA|1997-10-27|1999-10-27|Always wrong scho|0.14|0.05|1002001|importoamalg #1|2|fragrances|1|Women|395|antin stpri|medium|81445755662pale56502|chartreuse|Ton|Unknown|90|barablebaranti| +5021|AAAAAAAAMJDBAAAA|1999-10-28|2001-10-26|Always wrong scho|1.97|0.05|1002001|importoexporti #2|2|infants|3|Children|268|antin stpri|large|27257213azure5670302|goldenrod|Carton|Unknown|27|oughtablebaranti| +5022|AAAAAAAAMJDBAAAA|2001-10-27||Steps could not maintain both there international stones. Readily technical groups shave also decisive years. Special, unusual women coul|9.52|7.90|6011007|amalgbrand #7|2|semi-precious|6|Jewelry|268|eingcallyable|N/A|27257213azure5670302|seashell|Bunch|Unknown|30|ableablebaranti| +5023|AAAAAAAAPJDBAAAA|1997-10-27||Sure falls will not visit. Merely clear eyes will not finance no doubt major clear miles. Cells intro|2.89|1.79|2003002|exportiimporto #2|3|pants|2|Men|487|ationeingese|petite|52752peru68681704325|plum|Tsp|Unknown|15|priablebaranti| +5024|AAAAAAAAAKDBAAAA|1997-10-27|2000-10-26|Discussions could spend somewhere likely rights. Personal things end typic|3.46|2.04|8015005|scholarmaxi #5|15|fishing|8|Sports|239|n stpriable|N/A|0green67251780458292|sky|Each|Unknown|4|eseablebaranti| +5025|AAAAAAAAAKDBAAAA|2000-10-27||Different, capable |7.16|3.50|8015005|exportischolar #2|15|pop|5|Music|239|n stpriable|N/A|0green67251780458292|magenta|Oz|Unknown|17|antiablebaranti| +5026|AAAAAAAACKDBAAAA|1997-10-27|1999-10-27|Positions can win increasingly entire units. Unions used to exclude fairly afraid fans. National fields appear also ways. Great lips print new teachers. Constant, primary deaths expect a little |3.82|2.36|9001011|amalgmaxi #11|1|arts|9|Books|187|ationeingought|N/A|964sky29739420533170|bisque|Bundle|Unknown|68|callyablebaranti| +5027|AAAAAAAACKDBAAAA|1999-10-28|2001-10-26|Positions can win increasingly entire units. Unions used to exclude fairly afraid fans. National fields appear also ways. Great lips print new teachers. Constant, primary deaths expect a little |6.18|2.96|9001011|scholarbrand #8|5|blinds/shades|7|Home|187|ationeingought|N/A|215813767blue8400067|smoke|Ton|Unknown|34|ationablebaranti| +5028|AAAAAAAACKDBAAAA|2001-10-27||Then high components bring as unnecessary laws; |4.69|2.96|2003001|exportiimporto #1|3|pants|2|Men|10|ationeingought|N/A|215813767blue8400067|royal|Dozen|Unknown|5|eingablebaranti| +5029|AAAAAAAAFKDBAAAA|1997-10-27||Considerable institutions say more sound jobs. Emotional moves seem religious allegations; flowers ask about from the terms. Police shall put suddenly big yards. Affairs stop |3.75|2.21|8003010|exportinameless #10|3|basketball|8|Sports|277|ationationable|N/A|793453283112peru7058|grey|Dram|Unknown|18|n stablebaranti| +5030|AAAAAAAAGKDBAAAA||2000-10-26|||2.39|8014001||14|tennis|8||289|n steingable||burlywood01426043083||||93|barpribaranti| +5031|AAAAAAAAGKDBAAAA|2000-10-27||In addition industrial flowers act sometime|3.75|1.42|1004002|edu packamalg #2|14|swimwear|1|Women|627|n steingable|small|45341142magenta06281|wheat|Box|Unknown|48|oughtpribaranti| +5032|AAAAAAAAIKDBAAAA|1997-10-27|1999-10-27|Really new orders contribute; other animals may mark more everyday women. Exclusive, tragic ministers occur besides bonds. Areas keep british, minor c|2.13|1.57|6014001|edu packbrand #1|14|estate|6|Jewelry|400|barbarese|N/A|281976salmon53675691|sienna|Box|Unknown|46|ablepribaranti| +5033|AAAAAAAAIKDBAAAA|1999-10-28|2001-10-26|Really new orders contribute; other animals may mark more everyday women. Exclusive, tragic ministers occur besides bonds. Areas keep british, minor c|4.79|1.48|6014001|namelessbrand #4|14|lighting|7|Home|575|antiationanti|N/A|281976salmon53675691|chartreuse|Carton|Unknown|11|pripribaranti| +5034|AAAAAAAAIKDBAAAA|2001-10-27||Mental, able children might resolve far military, p|5.46|1.48|6014001|importoedu pack #1|14|mens|4|Shoes|596|callyn stanti|large|281976salmon53675691|powder|Box|Unknown|7|esepribaranti| +5035|AAAAAAAALKDBAAAA|1997-10-27||Members would plant main, human vehicles; technical tests should not make. Short, happy worlds could guess from a sales; views write quite able tears. Available enquiries help respects. Years seem.|0.54|0.36|3003002|exportiexporti #2|3|toddlers|3|Children|186|callyeingought|medium|linen061346970736303|rose|Bundle|Unknown|56|antipribaranti| +5036|AAAAAAAAMKDBAAAA|1997-10-27|2000-10-26|So ultimate forces frame over political, similar children; appeals shall not feel enough ways. Coloured, popular assets mig|0.39|0.30|5004001|edu packscholar #1|4|classical|5|Music|191|oughtn stought|N/A|3287432steel34594270|ghost|Pallet|Unknown|8|callypribaranti| +5037|AAAAAAAAMKDBAAAA|2000-10-27||So ultimate forces frame over political, similar children; appeals shall not feel enough ways. Coloured, popular assets mig|9.78|0.30|5004001|importoamalg #2|4|fragrances|1|Women|447|ationeseese|small|3287432steel34594270|blanched|Oz|Unknown|2|ationpribaranti| +5038|AAAAAAAAOKDBAAAA|1997-10-27|1999-10-27|More alone needs could cut. Key, hard items say even just wet police. Too consistent rivers maintain as main lives. Different areas think very. Equal, ab|4.88|1.85|3003001|exportiexporti #1|3|toddlers|3|Children|56|callyanti|small|504275823tan15098450|smoke|Gross|Unknown|6|eingpribaranti| +5039|AAAAAAAAOKDBAAAA|1999-10-28|2001-10-26|Arms mus|2.46|0.86|3003001|namelessmaxi #12|3|romance|9|Books|176|callyanti|N/A|97672plum43504634176|turquoise|Pound|Unknown|22|n stpribaranti| +5040|AAAAAAAAOKDBAAAA|2001-10-27||Arms mus|1.26|0.86|2002001|importoimporto #1|3|shirts|2|Men|176|callyanti|petite|97672plum43504634176|khaki|Dozen|Unknown|92|baresebaranti| +5041|AAAAAAAABLDBAAAA|1997-10-27||Ministers may recognize local problems. As a whole similar eyes meet very long-term tea|3.43|2.05|8002002|importonameless #2|2|baseball|8|Sports|771|oughtationation|N/A|5765823133887medium4|pale|Bunch|Unknown|16|oughtesebaranti| +5042|AAAAAAAACLDBAAAA|1997-10-27|2000-10-26|Little poor markets wriggle commonly roughly strategic times. Able securities can handle involuntarily thus other rates; then famous pri|2.21|1.03|8012003|importomaxi #3|12|guns|8|Sports|469|n stcallyese|N/A|4157turquoise5779572|plum|Oz|Unknown|27|ableesebaranti| +5043|AAAAAAAACLDBAAAA|2000-10-27||Concerned studies confess per the days; different paintings might go europea|1.14|1.03|8012003|amalgedu pack #2|12|womens|4|Shoes|330|n stcallyese|large|4304676086purple2973|slate|Case|Unknown|44|priesebaranti| +5044|AAAAAAAAELDBAAAA|1997-10-27|1999-10-27|Managers hold voluntarily standard, different days. Desirable, able networks could not stop for a girls. There low groups must persuade fairly defensive, only pe|38.93|11.67|3004001|edu packexporti #1|4|school-uniforms|3|Children|238|eingpriable|petite|36121papaya666963986|powder|Ounce|Unknown|87|eseesebaranti| +5045|AAAAAAAAELDBAAAA|1999-10-28|2001-10-26|Managers hold voluntarily standard, different days. Desirable, able networks could not stop for a girls. There low groups must persuade fairly defensive, only pe|1.93|11.67|8006006|corpnameless #6|4|football|8|Sports|238|eingpriable|N/A|13peru24653858762589|mint|Pound|Unknown|9|antiesebaranti| +5046|AAAAAAAAELDBAAAA|2001-10-27||Important, other h|0.49|11.67|8006006|amalgimporto #1|1|accessories|2|Men|238|eingpriable|small|wheat814211381124701|navajo|Box|Unknown|35|callyesebaranti| +5047|AAAAAAAAHLDBAAAA|1997-10-27||Now certain habits place quite european indians. Western grounds achieve almost. Relations help. Once comfortable taxes ought to start afterwards charges. Complicated, major provisi|4.96|2.67|3004002|edu packexporti #2|4|school-uniforms|3|Children|176|callyationought|small|73goldenrod663506307|smoke|Gram|Unknown|50|ationesebaranti| +5048|AAAAAAAAILDBAAAA|1997-10-27|2000-10-26|Raw classes arrive forward chapters. Of cour|42.52|26.36|10001017|amalgunivamalg #17|1|cameras|10|Electronics|855|antiantieing|N/A|651purple40184716041|powder|Carton|Unknown|99|eingesebaranti| +5049|AAAAAAAAILDBAAAA|2000-10-27||Raw classes arrive forward chapters. Of cour|2.55|26.36|8015010|scholarmaxi #10|1|fishing|8|Sports|855|antiantieing|N/A|651purple40184716041|red|Cup|Unknown|2|n stesebaranti| +5050|AAAAAAAAKLDBAAAA|1997-10-27|1999-10-27|Specified banks close characters. Long sections stop unduly burning teachers. Leading, certain colonies could not live determined forces. Legs say. Administrative clothes say only personal |0.91|0.47|8013001|exportimaxi #1|13|sailing|8|Sports|17|ationought|N/A|11901709537khaki4975|misty|N/A|Unknown|29|barantibaranti| +5051|AAAAAAAAKLDBAAAA|1999-10-28|2001-10-26|Specified banks close characters. Long sections stop unduly burning teachers. Leading, certain colonies could not live determined forces. Legs say. Administrative clothes say only personal |6.95|2.36|8013001|importounivamalg #12|13|home repair|9|Books|17|ationought|N/A|006998sky35776353021|slate|Oz|Unknown|50|oughtantibaranti| +5052|AAAAAAAAKLDBAAAA|2001-10-27||Specified banks close characters. Long sections stop unduly burning teachers. Leading, certain colonies could not live determined forces. Legs say. Administrative clothes say only personal |3.94|1.61|8013001|namelessmaxi #1|13|romance|9|Books|17|ationought|N/A|8turquoise1969007322|sienna|Unknown|Unknown|59|ableantibaranti| +5053|AAAAAAAANLDBAAAA|1997-10-27||Nights ought to see in a operations; whole, ordinary leaders ensure again plants. Revolutionary, rural years can say public others; sensitive, important r|3.74|2.87|2002002|importoimporto #2|2|shirts|2|Men|242|ableeseable|small|857021seashell484513|sandy|Unknown|Unknown|7|priantibaranti| +5054|AAAAAAAAOLDBAAAA|1997-10-27|2000-10-26|Special chee|49.12|39.78|9005003|scholarmaxi #3|5|history|9|Books|153|priantiought|N/A|0230895660372puff700|rosy|Ounce|Unknown|66|eseantibaranti| +5055|AAAAAAAAOLDBAAAA|2000-10-27||Special chee|2.93|39.78|7007006|brandbrand #6|5|decor|7|Home|153|priantiought|N/A|89106313174saddle129|red|N/A|Unknown|71|antiantibaranti| +5056|AAAAAAAAAMDBAAAA|1997-10-27|1999-10-27|Months would not consider especially emotions. Different, major differences would keep forward possible, conv|5.46|3.38|10013008|exportiamalgamalg #8|13|stereo|10|Electronics|96|callyn st|N/A|50778229450peach4596|sandy|Tsp|Unknown|13|callyantibaranti| +5057|AAAAAAAAAMDBAAAA|1999-10-28|2001-10-26|Months would not consider especially emotions. Different, major differences would keep forward possible, conv|6.22|3.38|10013008|maxicorp #2|9|womens watch|6|Jewelry|185|antieingought|N/A|50778229450peach4596|red|Dozen|Unknown|11|ationantibaranti| +5058|AAAAAAAAAMDBAAAA|2001-10-27||Months would not consider especially emotions. Different, major differences would keep forward possible, conv|6.52|2.86|4004001|edu packedu pack #1|9|athletic|4|Shoes|185|antieingought|medium|567295purple62042677|pink|Pallet|Unknown|22|eingantibaranti| +5059|AAAAAAAADMDBAAAA|1997-10-27||Hot negotiations make also common, selective men. About safe publishers will tend just international duties. Full men should go american servic|1.25|0.86|1002002|importoamalg #2|2|fragrances|1|Women|150|barantiought|large|68489yellow005955027|navy|Pound|Unknown|55|n stantibaranti| +5060|AAAAAAAAEMDBAAAA|1997-10-27|2000-10-26|Parents think real, previous minutes. Regional organs expect there red numbers. |0.29|0.19|7013001|exportinameless #1|13|wallpaper|7|Home|320|barablepri|N/A|471773322249545lawn7|slate|Carton|Unknown|24|barcallybaranti| +5061|AAAAAAAAEMDBAAAA|2000-10-27||Parents think real, previous minutes. Regional organs expect there red numbers. |6.58|0.19|7013001|exportimaxi #2|13|sailing|8|Sports|320|barablepri|N/A|34497905wheat7910189|mint|Unknown|Unknown|35|oughtcallybaranti| +5062|AAAAAAAAGMDBAAAA|1997-10-27|1999-10-27|Times fire for a videos. Whole, different diffi|48.60|30.61|1002001|importoamalg #1|2|fragrances|1|Women|477|ationationese|medium|0804742038saddle9315|red|Lb|Unknown|59|ablecallybaranti| +5063|AAAAAAAAGMDBAAAA|1999-10-28|2001-10-26|Times fire for a videos. Whole, different diffi|6.93|30.61|1002001|amalgamalg #2|2|dresses|1|Women|477|ationationese|extra large|65slate1112702582044|grey|Tbl|Unknown|64|pricallybaranti| +5064|AAAAAAAAGMDBAAAA|2001-10-27||Anonymous, female men will shrink later from the teeth. Afraid, concerned pieces may not develop largely subsequent, western appeals; directions forget anyway to a questions. |69.14|30.61|3002001|importoexporti #1|2|infants|3|Children|477|ationationese|medium|65slate1112702582044|red|Each|Unknown|46|esecallybaranti| +5065|AAAAAAAAJMDBAAAA|1997-10-27||Material, voluntary glasses get always on a patients. Happy prices used |1.71|0.73|2004002|edu packimporto #2|4|sports-apparel|2|Men|17|ationought|medium|85813869682066peru92|lime|Pound|Unknown|61|anticallybaranti| +5066|AAAAAAAAKMDBAAAA|1997-10-27|2000-10-26|Basic lectures can achieve. Separate purposes might help. N|7.20|4.53|2002001|importoimporto #1|2|shirts|2|Men|338|eingpripri|petite|82518086saddle570132|seashell|Bunch|Unknown|41|callycallybaranti| +5067|AAAAAAAAKMDBAAAA|2000-10-27||Children could not use. Separate clients used to marry only; public cities help over proposed, following nations. Artistic sys|0.71|0.23|3004002|edu packexporti #2|2|school-uniforms|3|Children|338|eingpripri|large|6905195727smoke83236|snow|Pallet|Unknown|42|ationcallybaranti| +5068|AAAAAAAAMMDBAAAA|1997-10-27|1999-10-27|Methods secure commentators. Once full-time co|5.73|4.35|9004005|edu packmaxi #5|4|entertainments|9|Books|19|n stought|N/A|635873482sandy047829|grey|Ton|Unknown|49|eingcallybaranti| +5069|AAAAAAAAMMDBAAAA|1999-10-28|2001-10-26|Methods secure commentators. Once full-time co|2.05|1.33|9004005|edu packedu pack #2|4|athletic|4|Shoes|665|n stought|petite|635873482sandy047829|wheat|Cup|Unknown|83|n stcallybaranti| +5070|AAAAAAAAMMDBAAAA|2001-10-27||Philosophical, living shops accept requirements. Numbers must seem again. High, fresh skills should not receive in|2.11|1.33|9004005|scholarmaxi #7|5|history|9|Books|665|anticallycally|N/A|0814salmon0124227441|rose|Each|Unknown|39|barationbaranti| +5071|AAAAAAAAPMDBAAAA|1997-10-27||Other, critical groups win here songs. Rich sentences expose in the numbers. Eu|1.99|1.75|6003008|exporticorp #8|3|gold|6|Jewelry|339|n stpripri|N/A|03125khaki0842022098|purple|Dram|Unknown|27|oughtationbaranti| +5072|AAAAAAAAANDBAAAA|1997-10-27|2000-10-26|Weeks create sometimes with the problems. International qua|2.36|1.79|8011003|amalgmaxi #3|11|archery|8|Sports|696|callyn stcally|N/A|58973goldenrod127124|navy|Pound|Unknown|37|ableationbaranti| +5073|AAAAAAAAANDBAAAA|2000-10-27||Weeks create sometimes with the problems. International qua|1.82|1.79|7010004|univnameless #4|11|flatware|7|Home|912|callyn stcally|N/A|036340908214light133|sienna|Bundle|Unknown|22|priationbaranti| +5074|AAAAAAAACNDBAAAA|1997-10-27|1999-10-27|Silent stan|8.78|2.72|2004001|edu packimporto #1|4|sports-apparel|2|Men|168|eingcallyought|medium|80785121281misty1047|smoke|Pound|Unknown|31|eseationbaranti| +5075|AAAAAAAACNDBAAAA|1999-10-28|2001-10-26|Silent stan|1.94|0.64|10005010|scholarunivamalg #10|4|karoke|10|Electronics|168|eingcallyought|N/A|80785121281misty1047|thistle|Lb|Unknown|11|antiationbaranti| +5076|AAAAAAAACNDBAAAA|2001-10-27||Rich, cheerful affairs realise simply e|9.55|4.77|3002001|importoexporti #1|4|infants|3|Children|168|eingcallyought|medium|80785121281misty1047|pale|Bunch|Unknown|40|callyationbaranti| +5077|AAAAAAAAFNDBAAAA|1997-10-27||Mass ideas add also other mechanisms. Original authorities shall not recor|4.39|2.37|1003002|exportiamalg #2|3|maternity|1|Women|276|callyationable|extra large|9125064918753indian4|purple|Pallet|Unknown|42|ationationbaranti| +5078|AAAAAAAAGNDBAAAA|1997-10-27|2000-10-26|Even existing readers receive as. Ago small trees look sometimes. Probably linguistic tanks work all right; far simple |2.94|0.94|6011007|amalgbrand #7|11|semi-precious|6|Jewelry|284|eseeingable|N/A|55sky799826340222802|medium|Pound|Unknown|7|eingationbaranti| +5079|AAAAAAAAGNDBAAAA|2000-10-27||A l|99.71|0.94|6011007|scholarunivamalg #14|11|karoke|10|Electronics|206|callybarable|N/A|5610204570617pink063|orange|Pallet|Unknown|25|n stationbaranti| +5080|AAAAAAAAINDBAAAA|1997-10-27|1999-10-27|Horizontal, red partners could escape toys. Available, right years develop as. Male requirements rise weeks; later special a|4.21|1.93|1003001|exportiamalg #1|3|maternity|1|Women|574|eseationanti|large|green400488140185719|white|Pallet|Unknown|38|bareingbaranti| +5081|AAAAAAAAINDBAAAA|1999-10-28|2001-10-26|Easily hot interests su|4.59|3.58|10009016|maxiunivamalg #16|9|televisions|10|Electronics|251|eseationanti|N/A|green400488140185719|seashell|Each|Unknown|54|oughteingbaranti| +5082|AAAAAAAAINDBAAAA|2001-10-27||Below immediate homes could not write c|0.81|3.58|10009016|edu packexporti #1|4|school-uniforms|3|Children|106|callybarought|medium|5350587pink268094144|khaki|Unknown|Unknown|31|ableeingbaranti| +5083|AAAAAAAALNDBAAAA|1997-10-27||Months kill. Sorts used to build best up to subtle depths; clearly modern passengers show really bold boards; about metropolitan towns|12.70|6.09|6015004|scholarbrand #4|15|custom|6|Jewelry|646|callyesecally|N/A|88058822purple155610|ivory|Lb|Unknown|12|prieingbaranti| +5084|AAAAAAAAMNDBAAAA|1997-10-27|2000-10-26|Good, social contents used|1.64|1.41|4001001|amalgedu pack #1|1|womens|4|Shoes|272|ableationable|medium|717593536orchid04085|papaya|Gross|Unknown|20|eseeingbaranti| +5085|AAAAAAAAMNDBAAAA|2000-10-27||Good, social contents used|1.81|0.97|4001001|exportischolar #2|1|pop|5|Music|272|ableationable|N/A|717593536orchid04085|snow|N/A|Unknown|60|antieingbaranti| +5086|AAAAAAAAONDBAAAA|1997-10-27|1999-10-27|Simply old languages lead coldly daughters; conditions can turn probably as so-called parties; virtually possible systems commemorate towns. Social, integrated refugees would not gai|2.65|1.82|6002007|importocorp #7|2|diamonds|6|Jewelry|95|antin st|N/A|6490180658purple1663|smoke|Dozen|Unknown|4|callyeingbaranti| +5087|AAAAAAAAONDBAAAA|1999-10-28|2001-10-26|Free vegetables shall not carry really able agree|8.12|3.65|8014008|edu packmaxi #8|2|tennis|8|Sports|269|antin st|N/A|29043violet138715903|rose|Cup|Unknown|94|ationeingbaranti| +5088|AAAAAAAAONDBAAAA|2001-10-27||Free vegetables shall not carry really able agree|4.66|3.65|1001001|amalgamalg #1|2|dresses|1|Women|269|n stcallyable|medium|29043violet138715903|rosy|N/A|Unknown|8|eingeingbaranti| +5089|AAAAAAAABODBAAAA|1997-10-27||Governments shall light just. Mediterranean, russian differences would adjust perhaps methods. Holes answer largely commercially optimistic fees. Available houses used to help later scotti|1.89|0.77|9012002|importounivamalg #2|12|home repair|9|Books|75|antiation|N/A|86cornflower35530195|metallic|Tsp|Unknown|84|n steingbaranti| +5090|AAAAAAAACODBAAAA|1997-10-27|2000-10-26|Flexible men assist in a cases. So other government|6.16|2.52|6015003|scholarbrand #3|15|custom|6|Jewelry|403|pribarese|N/A|1steel10338534759333|saddle|Pound|Unknown|98|barn stbaranti| +5091|AAAAAAAACODBAAAA|2000-10-27||Generally other countries relish now conservative days. Colleges can need firm, sure shareholders. |4.92|2.52|6015003|amalgunivamalg #9|15|cameras|10|Electronics|403|pribarese|N/A|1steel10338534759333|puff|Cup|Unknown|51|oughtn stbaranti| +5092|AAAAAAAAEODBAAAA|1997-10-27|1999-10-27|Here particular years could not accept even. Ideal, lesser sciences take plainly regular hands. Routinely vulnerable names might find very right lives. Long circumstances used to raise act|7.76|6.36|8010003|univmaxi #3|10|pools|8|Sports|314|eseoughtpri|N/A|324844003567smoke807|powder|Each|Unknown|7|ablen stbaranti| +5093|AAAAAAAAEODBAAAA|1999-10-28|2001-10-26|Here particular years could not accept even. Ideal, lesser sciences take plainly regular hands. Routinely vulnerable names might find very right lives. Long circumstances used to raise act|7.33|2.49|3001002|amalgexporti #2|1|newborn|3|Children|90|eseoughtpri|large|324844003567smoke807|turquoise|Each|Unknown|48|prin stbaranti| +5094|AAAAAAAAEODBAAAA|2001-10-27||Also primary rates could |6.25|2.49|10006012|corpunivamalg #12|1|musical|10|Electronics|90|eseoughtpri|N/A|23653106red081328711|white|Bunch|Unknown|17|esen stbaranti| +5095|AAAAAAAAHODBAAAA|1997-10-27||Square, black institutions could change for example eventually other customers. Leaders must not fire toge|1.87|1.34|8009006|maxinameless #6|9|optics|8|Sports|168|eingcallyought|N/A|913297914298206pale0|linen|Cup|Unknown|16|antin stbaranti| +5096|AAAAAAAAIODBAAAA|1997-10-27|2000-10-26|New funds work for a coins. Pupils must drift more than difficult communications. Lovely, new centuries must understand seriously points. At last large associations can maintain literally in|1.05|0.78|10008014|namelessunivamalg #14|8|scanners|10|Electronics|539|n stprianti|N/A|6379305442904tomato9|tomato|Ounce|Unknown|12|callyn stbaranti| +5097|AAAAAAAAIODBAAAA|2000-10-27||Special managers know powerful groups. Independent publications induce angrily personal |0.50|0.78|6009002|maxicorp #2|9|womens watch|6|Jewelry|539|n stprianti|N/A|6379305442904tomato9|linen|Cup|Unknown|5|ationn stbaranti| +5098|AAAAAAAAKODBAAAA|1997-10-27|1999-10-27|Times make at least almost able girls; animals ought to admit sure, poor hours. Communities start curtains. Words would not break so cities. European, single pensioners shall provid|9.12|4.37|10006016|corpunivamalg #16|6|musical|10|Electronics|161|oughtcallyought|N/A|369381saddle05613859|saddle|Tbl|Unknown|6|eingn stbaranti| +5099|AAAAAAAAKODBAAAA|1999-10-28|2001-10-26|Ever aggressive colleges would shine levels. Local interests keep. Changes may not maintain middle, important days. Littl|31.40|12.56|6005004|scholarcorp #4|6|earings|6|Jewelry|161|oughtcallyought|N/A|572619004rosy3465660|rosy|Dozen|Unknown|71|n stn stbaranti| +5100|AAAAAAAAKODBAAAA|2001-10-27||Little rights may not buy even films. Thus large-scale days used to bring away |3.43|12.56|6005004|importoscholar #1|6|country|5|Music|594|oughtcallyought|N/A|52986391misty0164057|rose|Pallet|Unknown|3|barbaroughtanti| +5101|AAAAAAAANODBAAAA|1997-10-27||Furthermore low parents used to reach. Young years can rest completely busy woods. Formal, inadequ|2.17|1.86|8012002|importomaxi #2|12|guns|8|Sports|80|bareing|N/A|0moccasin88442162225|rosy|N/A|Unknown|51|oughtbaroughtanti| +5102|AAAAAAAAOODBAAAA|1997-10-27|2000-10-26|Naturally narrow standards might get tomorrow unknown, fu|9.56|3.25|6010007|univbrand #7|10|jewelry boxes|6|Jewelry|69|n stcally|N/A|15786996metallic9136|blush|Cup|Unknown|14|ablebaroughtanti| +5103|AAAAAAAAOODBAAAA|2000-10-27||Naturally narrow standards might get tomorrow unknown, fu|9.51|6.08|2003002|exportiimporto #2|3|pants|2|Men|69|n stcally|large|15786996metallic9136|puff|Bunch|Unknown|18|pribaroughtanti| +5104|AAAAAAAAAPDBAAAA|1997-10-27|1999-10-27|As general expenses manipulate traditionally pregnant facts. Centres enable absolutely. Right girls can see massively by a pubs. Aware a|0.35|0.12|2001001|amalgimporto #1|1|accessories|2|Men|479|n stationese|large|8411443840red7578382|pink|Gram|Unknown|35|esebaroughtanti| +5105|AAAAAAAAAPDBAAAA|1999-10-28|2001-10-26|As happy results might supply often able women; faces approach. Principles could think so to a members. Even yellow contributions shall continue quite exc|8.92|0.12|8003006|exportinameless #6|3|basketball|8|Sports|158|eingantiought|N/A|8411443840red7578382|blanched|Carton|Unknown|20|antibaroughtanti| +5106|AAAAAAAAAPDBAAAA|2001-10-27||So original languages shall help more black, various interests. Various, future lights will meet forwards; still likely funds cannot specify for example wide, national benefits; weeks|4.07|0.12|8003006|amalgamalg #1|1|dresses|1|Women|158|eingantiought|petite|8411443840red7578382|steel|Dram|Unknown|19|callybaroughtanti| +5107|AAAAAAAADPDBAAAA|1997-10-27||Opinions can allow firmly interests. Vague conferences know more public benefits. Still prime curtains apply along with a cases. Ha|5.99|3.47|3004002|edu packexporti #2|4|school-uniforms|3|Children|139|n stpriought|large|orange60032346398170|cornsilk|Bundle|Unknown|93|ationbaroughtanti| +5108|AAAAAAAAEPDBAAAA|1997-10-27|2000-10-26|Endless, free contexts lie so times. Usually other children will not admit sensitive members. Expensive,|4.52|3.29|1002001|importoamalg #1|2|fragrances|1|Women|899|n stn steing|small|5983072038820tan6201|goldenrod|N/A|Unknown|2|eingbaroughtanti| +5109|AAAAAAAAEPDBAAAA|2000-10-27||Endless, free contexts lie so times. Usually other children will not admit sensitive members. Expensive,|9.16|3.29|1004002|edu packamalg #2|4|swimwear|1|Women|148|eingeseought|medium|5983072038820tan6201|puff|Pallet|Unknown|29|n stbaroughtanti| +5110|AAAAAAAAGPDBAAAA|1997-10-27|1999-10-27|Personnel need actually|33.93|20.35|7014001|edu packnameless #1|14|glassware|7|Home|696|callyn stcally|N/A|419orange08225972174|indian|Bunch|Unknown|60|baroughtoughtanti| +5111|AAAAAAAAGPDBAAAA|1999-10-28|2001-10-26|European meetings serve most still emotional days. General, royal grounds may not assist profoundly economic references. Possible, double yards see clearly. Positions enlarge completely biological sk|4.76|2.76|7014001|exportischolar #2|3|pop|5|Music|696|callyn stcally|N/A|9746030884232grey804|linen|Tsp|Unknown|14|oughtoughtoughtanti| +5112|AAAAAAAAGPDBAAAA|2001-10-27||European meetings serve most still emotional days. General, royal grounds may not assist profoundly economic references. Possible, double yards see clearly. Positions enlarge completely biological sk|9.79|6.16|7014001|corpmaxi #3|3|golf|8|Sports|696|callyn stcally|N/A|9746030884232grey804|navajo|Pallet|Unknown|6|ableoughtoughtanti| +5113|AAAAAAAAJPDBAAAA|1997-10-27||Only costs exist also towards a suppliers. Men could tell any lo|5.62|2.02|2002002|importoimporto #2|2|shirts|2|Men|826|callyableeing|medium|756583orange38163713|spring|Tsp|Unknown|62|prioughtoughtanti| +5114|AAAAAAAAKPDBAAAA|1997-10-27|2000-10-26|Methods see open terms. Thorough problems return even at a functions. Aware years would seek here very social schools. Final, new groups get undoubtedly suitable women. Local,|1.53|1.20|10003015|exportiunivamalg #15|3|dvd/vcr players|10|Electronics|312|ableoughtpri|N/A|orange20575739617148|green|Unknown|Unknown|12|eseoughtoughtanti| +5115|AAAAAAAAKPDBAAAA|2000-10-27||Methods see open terms. Thorough problems return even at a functions. Aware years would seek here very social schools. Final, new groups get undoubtedly suitable women. Local,|2.50|1.65|8008006|namelessnameless #6|8|outdoor|8|Sports|312|ableoughtpri|N/A|726thistle4824732915|salmon|Dozen|Unknown|37|antioughtoughtanti| +5116|AAAAAAAAMPDBAAAA|1997-10-27|1999-10-27|Organizations should get. Workers wait resources; officers |4.57|2.51|4001001|amalgedu pack #1|1|womens|4|Shoes|395|antin stpri|N/A|492680983linen758439|snow|Pound|Unknown|9|callyoughtoughtanti| +5117|AAAAAAAAMPDBAAAA|1999-10-28|2001-10-26|Safe demonstrations might not sell ever. Special, likely tr|4.34|2.51|4001001|amalgbrand #6|11|semi-precious|6|Jewelry|542|ableeseanti|N/A|492680983linen758439|gainsboro|Ton|Unknown|17|ationoughtoughtanti| +5118|AAAAAAAAMPDBAAAA|2001-10-27||Due, complete points could continue terms. Areas understand right generally chemical talks. Clear process|26.64|2.51|8008009|namelessnameless #9|8|outdoor|8|Sports|137|ableeseanti|N/A|701sky79915417753948|violet|Cup|Unknown|99|eingoughtoughtanti| +5119|AAAAAAAAPPDBAAAA|1997-10-27||Mainly superior results bat fast efforts; categories may not supply so just local areas. Priorities would eat easily prime secrets. Blind, ma|2.47|2.12|2003002|exportiimporto #2|3|pants|2|Men|35|antipri|medium|15puff22132963668301|lime|N/A|Unknown|45|n stoughtoughtanti| +5120|AAAAAAAAAAEBAAAA|1997-10-27|2000-10-26|New items might take sure clergy. Only, |3.12|1.31|10016004|corpamalgamalg #4|16|wireless|10|Electronics|590|barn stanti|N/A|304972227047spring28|peru|Cup|Unknown|32|barableoughtanti| +5121|AAAAAAAAAAEBAAAA|2000-10-27||New items might take sure clergy. Only, |7.78|1.31|6014002|edu packbrand #2|16|estate|6|Jewelry|590|barn stanti|N/A|304972227047spring28|misty|Carton|Unknown|90|oughtableoughtanti| +5122|AAAAAAAACAEBAAAA|1997-10-27|1999-10-27|Below continuing managers should play simple types. Points provide direct, inevitable degrees. For sure valuable links afford furiously privately religious |1.74|1.49|7012003|importonameless #3|12|paint|7|Home|556|callyantianti|N/A|spring73963657835467|grey|Oz|Unknown|56|ableableoughtanti| +5123|AAAAAAAACAEBAAAA|1999-10-28|2001-10-26|Quite public|7.57|1.49|4003002|exportiedu pack #2|12|kids|4|Shoes|556|callyantianti|petite|78wheat0515945669442|pink|Lb|Unknown|33|priableoughtanti| +5124|AAAAAAAACAEBAAAA|2001-10-27||Quite public|2.86|1.43|10007008|brandunivamalg #8|12|personal|10|Electronics|556|callyantianti|N/A|5692012038969wheat91|rose|Lb|Unknown|12|eseableoughtanti| +5125|AAAAAAAAFAEBAAAA|1997-10-27||Voluntary neighbours would measure directly upper, labour governments. Women work us|3.18|2.54|6004006|edu packcorp #6|4|bracelets|6|Jewelry|238|eingpriable|N/A|3529royal71603533325|hot|Gram|Unknown|50|antiableoughtanti| +5126|AAAAAAAAGAEBAAAA|1997-10-27|2000-10-26|Sizes use painfully advantages. Regular, similar hands would not ask seldom more great fingers. Of course wide creditors may engage exactly complicated inevitable e|0.29|0.14|4004001|edu packedu pack #1|4|athletic|4|Shoes|281|oughteingable|medium|07peach2137078485510|powder|Pound|Unknown|41|callyableoughtanti| +5127|AAAAAAAAGAEBAAAA|2000-10-27||Sizes use painfully advantages. Regular, similar hands would not ask seldom more great fingers. Of course wide creditors may engage exactly complicated inevitable e|4.89|0.14|6007008|brandcorp #8|4|pendants|6|Jewelry|281|oughteingable|N/A|07peach2137078485510|goldenrod|Gram|Unknown|43|ationableoughtanti| +5128|AAAAAAAAIAEBAAAA|1997-10-27|1999-10-27|Big, great subjects would meet in a subjects. Acts record actually. Cases shall not look. So rural sales |3.11|1.30|3002001|importoexporti #1|2|infants|3|Children|553|priantianti|medium|42125459712slate4833|honeydew|Dozen|Unknown|14|eingableoughtanti| +5129|AAAAAAAAIAEBAAAA|1999-10-28|2001-10-26|That local texts want in a rights. Doubts might not change private, double years; clear children take even sure generations. International, good costs know happy, particular findings. Duties can unde|76.30|1.30|2002002|importoimporto #2|2|shirts|2|Men|500|priantianti|small|955527sandy413926295|thistle|Gross|Unknown|30|n stableoughtanti| +5130|AAAAAAAAIAEBAAAA|2001-10-27||That local texts want in a rights. Doubts might not change private, double years; clear children take even sure generations. International, good costs know happy, particular findings. Duties can unde|2.72|1.30|2002002|exportiamalg #1|3|maternity|1|Women|500|priantianti|small|955527sandy413926295|royal|Case|Unknown|14|barprioughtanti| +5131|AAAAAAAALAEBAAAA|1997-10-27||Unfair, current packages might keep then hard central firms. Workers keep possibilities; still social sounds help other things. Sad lights unload past lines|3.30|2.97|5002002|importoscholar #2|2|country|5|Music|611|oughtoughtcally|N/A|9132powder6745462325|slate|Tsp|Unknown|11|oughtprioughtanti| +5132|AAAAAAAAMAEBAAAA|1997-10-27|2000-10-26|Never special children try boys. Especially unable lines may not participate. Women set fundamental terms. Other, american trousers would not send still pp.. |3.50|2.10|5002001|importoscholar #1|2|country|5|Music|585|antieinganti|N/A|437952204seashell938|peru|Oz|Unknown|18|ableprioughtanti| +5133|AAAAAAAAMAEBAAAA|2000-10-27||Never special children try boys. Especially unable lines may not participate. Women set fundamental terms. Other, american trousers would not send still pp.. |1.64|1.21|5002002|importoscholar #2|2|country|5|Music|585|antieinganti|N/A|437952204seashell938|salmon|Ounce|Unknown|9|priprioughtanti| +5134|AAAAAAAAOAEBAAAA|1997-10-27|1999-10-27|Large, dirty problems might discourage sure for a committees. In general large letters shall come both different psychia|3.35|1.97|6011007|amalgbrand #7|11|semi-precious|6|Jewelry|440|bareseese|N/A|49ivory8380615759488|plum|Tbl|Unknown|20|eseprioughtanti| +5135|AAAAAAAAOAEBAAAA|1999-10-28|2001-10-26|Large, dirty problems might discourage sure for a committees. In general large letters shall come both different psychia|2.50|1.97|10002002|importounivamalg #2|2|camcorders|10|Electronics|3|bareseese|N/A|49ivory8380615759488|pale|Oz|Unknown|25|antiprioughtanti| +5136|AAAAAAAAOAEBAAAA|2001-10-27||Large, dirty problems might discourage sure for a committees. In general large letters shall come both different psychia|3.61|1.91|10002002|importoimporto #1|2|shirts|2|Men|3|bareseese|small|31seashell4904913646|peach|Tbl|Unknown|56|callyprioughtanti| +5137|AAAAAAAABBEBAAAA|1997-10-27||Most social degrees may integrate simply on a systems. Separate, huge values ought to make really tr|9.79|7.44|1004002|edu packamalg #2|4|swimwear|1|Women|159|n stantiought|large|549964140powder39336|lace|Pallet|Unknown|15|ationprioughtanti| +5138|AAAAAAAACBEBAAAA|1997-10-27|2000-10-26|Only long forms ought to stem exciting, ready companies. Activities may wish extraordinary entries. Please sorry banks can sing; known women can like always true units. Keen, upp|3.74|1.27|4003001|exportiedu pack #1|3|kids|4|Shoes|348|eingesepri|large|899018royal847097543|papaya|Unknown|Unknown|18|eingprioughtanti| +5139|AAAAAAAACBEBAAAA|2000-10-27||Police walk friends. Special changes take military, other costs. Matters must ask especially cold, single characters. Yet happy days enter easier a|1.32|0.93|2004002|edu packimporto #2|4|sports-apparel|2|Men|130|eingesepri|medium|876808327peach885004|purple|Carton|Unknown|40|n stprioughtanti| +5140|AAAAAAAAEBEBAAAA|1997-10-27|1999-10-27|Part-time details should not hear most in a miners. Things expand never areas. Large things work ver|2.99|0.89|1003001|exportiamalg #1|3|maternity|1|Women|642|ableesecally|medium|91046142052641lace32|grey|Gram|Unknown|33|bareseoughtanti| +5141|AAAAAAAAEBEBAAAA|1999-10-28|2001-10-26|Unknown, electronic organisations might know spare, extreme rewards. Appeals produce fun boys. Important years would not supp|5.87|0.89|1004002|edu packamalg #2|3|swimwear|1|Women|651|ableesecally|medium|29misty6578570801647|slate|Cup|Unknown|32|oughteseoughtanti| +5142|AAAAAAAAEBEBAAAA|2001-10-27||Hot colleagues buy at last inner, sufficient confere|77.47|23.24|7011003|amalgnameless #3|11|accent|7|Home|651|ableesecally|N/A|29misty6578570801647|puff|Tsp|Unknown|20|ableeseoughtanti| +5143|AAAAAAAAHBEBAAAA|1997-10-27||Forever strong arguments make main bills. Hard national thoughts might get commonly in a problems. Combined, lovely industries|1.17|0.79|5004002|edu packscholar #2|4|classical|5|Music|92|ablen st|N/A|2purple1403101514592|peru|Lb|Unknown|5|prieseoughtanti| +5144|AAAAAAAAIBEBAAAA|1997-10-27|2000-10-26|Prioritie|26.84|22.00|10004011|edu packunivamalg #11|4|audio|10|Electronics|135|antipriought|N/A|3purple7888596494182|salmon|Dram|Unknown|29|eseeseoughtanti| +5145|AAAAAAAAIBEBAAAA|2000-10-27||Prioritie|6.28|22.00|7014006|edu packnameless #6|14|glassware|7|Home|102|antipriought|N/A|3purple7888596494182|tan|Bunch|Unknown|62|antieseoughtanti| +5146|AAAAAAAAKBEBAAAA|1997-10-27|1999-10-27|Real foundations shall not read everyday friends. Big, human projects walk extremely targets. Previous surfaces may suggest well cells. Very previou|1.03|0.77|2002001|importoimporto #1|2|shirts|2|Men|190|barn stought|small|02royal3202726128395|plum|Tbl|Unknown|27|callyeseoughtanti| +5147|AAAAAAAAKBEBAAAA|1999-10-28|2001-10-26||5.12|2.15||amalgbrand #4|2|semi-precious|||190||N/A||sandy|Ton|||ationeseoughtanti| +5148|AAAAAAAAKBEBAAAA|2001-10-27||Royal friends enjoy single, fair times. National, wide authorities may mark now. Good banks prevent pure, english deaths. Major forests must organize. Free, private |2.32|2.15|6011004|exportiedu pack #1|2|kids|4|Shoes|190|barn stought|petite|24384coral5008205748|steel|Ounce|Unknown|61|eingeseoughtanti| +5149|AAAAAAAANBEBAAAA|1997-10-27||Social, mi|2.25|1.98|9013002|exportiunivamalg #2|13|self-help|9|Books|144|eseeseought|N/A|82navajo168585788456|royal|Bunch|Unknown|19|n steseoughtanti| +5150|AAAAAAAAOBEBAAAA|1997-10-27|2000-10-26|Independent, similar places reflect little final, available pressures. National, political words should deal much competent, ne|7.22|3.17|1004001|edu packamalg #1|4|swimwear|1|Women|41|oughtese|medium|30puff12619688077380|red|Bundle|Unknown|2|barantioughtanti| +5151|AAAAAAAAOBEBAAAA|2000-10-27||Independent, similar places reflect little final, available pressures. National, political words should deal much competent, ne|1.22|0.97|1002002|importoamalg #2|2|fragrances|1|Women|41|oughtese|petite|30puff12619688077380|chiffon|Tbl|Unknown|28|oughtantioughtanti| +5152|AAAAAAAAACEBAAAA|1997-10-27|1999-10-27|Prese|15.17|8.04|9002005|importomaxi #5|2|business|9|Books|619|n stoughtcally|N/A|2352002149sky1208090|papaya|Each|Unknown|92|ableantioughtanti| +5153|AAAAAAAAACEBAAAA|1999-10-28|2001-10-26|Prese|6.56|8.04|9002005|corpunivamalg #6|2|mystery|9|Books|87|n stoughtcally|N/A|2352002149sky1208090|linen|Ton|Unknown|26|priantioughtanti| +5154|AAAAAAAAACEBAAAA|2001-10-27||Prese|1.84|8.04|6007003|brandcorp #3|7|pendants|6|Jewelry|87|ationeing|N/A|5629269almond7290062|pink|Ton|Unknown|56|eseantioughtanti| +5155|AAAAAAAADCEBAAAA|1997-10-27||Yet electoral systems used to give just free troops. Occasions think necessarily. British towns think dead offers. Natural, social days drive nevertheless.|0.29|0.26|8008006|namelessnameless #6|8|outdoor|8|Sports|82|ableeing|N/A|192810salmon02088213|navajo|Ton|Unknown|35|antiantioughtanti| +5156|AAAAAAAAECEBAAAA|1997-10-27|2000-10-26|Forms should pursue really. Shops govern european, final situations; suitable, nuclear years colour; yards make all alternative qualities. Readers used to help europe|5.14|3.18|8010007|univmaxi #7|10|pools|8|Sports|426|callyableese|N/A|53burlywood562254727|lemon|Ounce|Unknown|39|callyantioughtanti| +5157|AAAAAAAAECEBAAAA|2000-10-27||Previous appearances will not make everywhere teams. Modern, famous photographs could not try secondly social folk. Pa|36.63|3.18|8010007|amalgimporto #2|1|accessories|2|Men|426|callyableese|economy|554yellow31991569080|orange|Dozen|Unknown|17|ationantioughtanti| +5158|AAAAAAAAGCEBAAAA|1997-10-27|1999-10-27|Reasons experience only to a rules; then major governors ought to wish forms. So casual manufa|7.66|3.90|4003001|exportiedu pack #1|3|kids|4|Shoes|246|callyeseable|large|37731621582seashell9|saddle|Case|Unknown|30|eingantioughtanti| +5159|AAAAAAAAGCEBAAAA|1999-10-28|2001-10-26|Reasons experience only to a rules; then major governors ought to wish forms. So casual manufa|2.95|3.90|4003001|amalgbrand #10|3|bathroom|7|Home|700|barbaration|N/A|37731621582seashell9|lace|Bundle|Unknown|93|n stantioughtanti| +5160|AAAAAAAAGCEBAAAA|2001-10-27||Spanish women remain equally unexpectedly fund|0.35|0.25|4003001|importoimporto #1|3|shirts|2|Men|700|barbaration|petite|5misty99547518884120|metallic|Oz|Unknown|25|barcallyoughtanti| +5161|AAAAAAAAJCEBAAAA|1997-10-27||Actual, natural areas know. Everyday things love very issues. Crimes remain always days. Active systems remember then. Dreams might tell from the shadows. Leading votes enable personal, ent|0.87|0.62|8010002|univmaxi #2|10|pools|8|Sports|226|callyableable|N/A|98890gainsboro025575|beige|Pound|Unknown|11|oughtcallyoughtanti| +5162|AAAAAAAAKCEBAAAA|1997-10-27|2000-10-26|Simple others repres|3.34|2.07|7010003|univnameless #3|10|flatware|7|Home|203|pribarable|N/A|984223147snow9996067|sandy|Bundle|Unknown|7|ablecallyoughtanti| +5163|AAAAAAAAKCEBAAAA|2000-10-27||Simple others repres|8.11|2.07|10011013|amalgamalgamalg #13|10|disk drives|10|Electronics|203|pribarable|N/A|932346068196smoke585|yellow|Bundle|Unknown|24|pricallyoughtanti| +5164|AAAAAAAAMCEBAAAA|1997-10-27|1999-10-27|Nowhere sure shops ought to constitute by a conditions. Apparent hands shall not fit slightly general men. Oth|3.59|3.08|5001001|amalgscholar #1|1|rock|5|Music|979|n stationn st|N/A|12512400steel6609561|black|Box|Unknown|100|esecallyoughtanti| +5165|AAAAAAAAMCEBAAAA|1999-10-28|2001-10-26|Nowhere sure shops ought to constitute by a conditions. Apparent hands shall not fit slightly general men. Oth|1.60|0.80|9004006|edu packmaxi #6|1|entertainments|9|Books|645|antiesecally|N/A|12512400steel6609561|light|Pound|Unknown|26|anticallyoughtanti| +5166|AAAAAAAAMCEBAAAA|2001-10-27||Nowhere sure shops ought to constitute by a conditions. Apparent hands shall not fit slightly general men. Oth|0.46|0.80|9004006|amalgexporti #1|1|newborn|3|Children|423|priableese|extra large|12512400steel6609561|rose|Oz|Unknown|11|callycallyoughtanti| +5167|AAAAAAAAPCEBAAAA|1997-10-27||Th|0.34|0.16|9015008|scholarunivamalg #8|15|fiction|9|Books|407|ationbarese|N/A|peru4088360614418760|puff|Gross|Unknown|9|ationcallyoughtanti| +5168|AAAAAAAAADEBAAAA|1997-10-27|2000-10-26|Already vocational holders like always further official deputies. Ac|3.85|2.77|8005009|scholarnameless #9|5|fitness|8|Sports|15|antiought|N/A|392pale1858543352701|puff|Box|Unknown|53|eingcallyoughtanti| +5169|AAAAAAAAADEBAAAA|2000-10-27||Already vocational holders like always further official deputies. Ac|4.11|2.79|8005009|amalgmaxi #10|5|archery|8|Sports|15|antiought|N/A|392pale1858543352701|metallic|Unknown|Unknown|70|n stcallyoughtanti| +5170|AAAAAAAACDEBAAAA|1997-10-27|1999-10-27|Late different pains would face broken, sound incomes. Certain reasons would tell years. Children used to boost rem|7.66|4.90|4003001|exportiedu pack #1|3|kids|4|Shoes|5|anti|medium|22789758purple022923|puff|Gross|Unknown|47|barationoughtanti| +5171|AAAAAAAACDEBAAAA|1999-10-28|2001-10-26|Late different pains would face broken, sound incomes. Certain reasons would tell years. Children used to boost rem|9.09|2.81|9007012|brandmaxi #12|7|reference|9|Books|5|anti|N/A|22789758purple022923|snow|N/A|Unknown|58|oughtationoughtanti| +5172|AAAAAAAACDEBAAAA|2001-10-27||Late different pains would face broken, sound incomes. Certain reasons would tell years. Children used to boost rem|3.88|2.91|9007012|importoedu pack #1|2|mens|4|Shoes|5|anti|large|520plum8689129727213|red|Box|Unknown|55|ableationoughtanti| +5173|AAAAAAAAFDEBAAAA|1997-10-27||Movements may bear still little questions. Steps come now students. Good friends report in|5.78|2.42|2001002|amalgimporto #2|1|accessories|2|Men|296|callyn stable|petite|974844890182lime1470|salmon|Each|Unknown|33|priationoughtanti| +5174|AAAAAAAAGDEBAAAA|1997-10-27|2000-10-26|Probably british interests could not arrange considerable sources; newspapers speak aback by a negotiations. |4.16|1.28|9012003|importounivamalg #3|12|home repair|9|Books|139|n stpriought|N/A|6seashell27894045623|metallic|Unknown|Unknown|5|eseationoughtanti| +5175|AAAAAAAAGDEBAAAA|2000-10-27||Possible programmes adapt here problems. Often video-taped units will not hide particularly rich women; |2.27|1.28|9012003|importoamalg #2|12|fragrances|1|Women|96|n stpriought|extra large|6seashell27894045623|rosy|Bunch|Unknown|33|antiationoughtanti| +5176|AAAAAAAAIDEBAAAA|1997-10-27|1999-10-27|Banks think very large,|4.97|1.78|7015007|scholarnameless #7|15|tables|7|Home|866|callycallyeing|N/A|281rose7958977084339|tomato|Dram|Unknown|25|callyationoughtanti| +5177|AAAAAAAAIDEBAAAA|1999-10-28|2001-10-26|Banks think very large,|4.79|1.78|7003008|exportibrand #8|15|kids|7|Home|866|callycallyeing|N/A|0269731509peru494460|peach|Pound|Unknown|55|ationationoughtanti| +5178|AAAAAAAAIDEBAAAA|2001-10-27||Banks think very large,|61.19|30.59|2002001|importoimporto #1|15|shirts|2|Men|303|pribarpri|small|858546floral72448776|rose|Dram|Unknown|11|eingationoughtanti| +5179|AAAAAAAALDEBAAAA|1997-10-27||New requirements can increase more than for example increasing leaves. Operational, simple hea|78.09|51.53|8003010|exportinameless #10|3|basketball|8|Sports|470|barationese|N/A|159310618lemon322934|slate|Case|Unknown|4|n stationoughtanti| +5180|AAAAAAAAMDEBAAAA|1997-10-27|2000-10-26|Voluntarily comparable classes offer officials. Probably true mechanisms fit national frames. Fair miles will eradicate b|1.22|0.37|1001001|amalgamalg #1|1|dresses|1|Women|405|antibarese|large|29sky902085858983194|aquamarine|Tbl|Unknown|85|bareingoughtanti| +5181|AAAAAAAAMDEBAAAA|2000-10-27||Current, central points may propose friends. Stupid, legal members marry costs. Short, new services become certainly regardless|4.82|3.51|1001001|univamalgamalg #14|1|memory|10|Electronics|657|ationantically|N/A|8189701982487slate17|salmon|Ton|Unknown|29|oughteingoughtanti| +5182|AAAAAAAAODEBAAAA|1997-10-27|1999-10-27|Shareholders encounter at once complex products. Real years might not get workers. Nevertheless medical proceedings should |64.88|48.01|2003001|exportiimporto #1|3|pants|2|Men|138|eingpriought|medium|4207710370ivory83722|honeydew|Box|Unknown|20|ableeingoughtanti| +5183|AAAAAAAAODEBAAAA|1999-10-28|2001-10-26|Shareholders encounter at once complex products. Real years might not get workers. Nevertheless medical proceedings should |3.65|48.01|2003001|exportinameless #4|3|basketball|8|Sports|269|eingpriought|N/A|337703986puff5532080|wheat|Each|Unknown|57|prieingoughtanti| +5184|AAAAAAAAODEBAAAA|2001-10-27||Broken, electrical pupils tell cleverly. Figures would not go literary, annual systems. Courses discuss. |2.93|48.01|5003001|exportischolar #1|3|pop|5|Music|936|callyprin st|N/A|886947333spring12340|navajo|Pallet|Unknown|94|eseeingoughtanti| +5185|AAAAAAAABEEBAAAA|1997-10-27||Real, essential things administer so joint, standard things; most main pupils act also in a grant|2.31|0.76|5001002|amalgscholar #2|1|rock|5|Music|181|oughteingought|N/A|red73561604178960974|saddle|Dram|Unknown|1|antieingoughtanti| +5186|AAAAAAAACEEBAAAA|1997-10-27|2000-10-26|Expectations adopt decent creatures. Only efficient features could evoke nearly down a officials. Just urban stars could stick lakes. Then empty jobs should not encourage ever |8.12|6.49|8010007|univmaxi #7|10|pools|8|Sports|249|n steseable|N/A|3013royal39262581559|navy|Lb|Unknown|87|callyeingoughtanti| +5187|AAAAAAAACEEBAAAA|2000-10-27||Great, sorry figures ought to use. Materials may provide sometimes hands. Similar chiefs may advertise left jobs. Men supervise far distinctive products. Private|4.61|6.49|3002002|importoexporti #2|10|infants|3|Children|288|eingeingable|medium|3013royal39262581559|honeydew|Box|Unknown|8|ationeingoughtanti| +5188|AAAAAAAAEEEBAAAA|1997-10-27|1999-10-27|Golden, local co|46.00|28.52|5003001|exportischolar #1|3|pop|5|Music|491|oughtn stese|N/A|42169043firebrick750|salmon|Dozen|Unknown|7|eingeingoughtanti| +5189|AAAAAAAAEEEBAAAA|1999-10-28|2001-10-26|Potential pil|0.31|0.26|5003001|importoexporti #2|2|infants|3|Children|491|oughtn stese|extra large|42169043firebrick750|frosted|Dram|Unknown|9|n steingoughtanti| +5190|AAAAAAAAEEEBAAAA|2001-10-27||Poles will say far likely, total ideas. Likely, black things could draft now small, clear seats. African friends ma|8.11|0.26|1004001|edu packamalg #1|4|swimwear|1|Women|470|barationese|medium|42169043firebrick750|red|Pallet|Unknown|26|barn stoughtanti| +5191|AAAAAAAAHEEBAAAA|1997-10-27||Still new differences ask |1.42|0.56|7013002|exportinameless #2|13|wallpaper|7|Home|62|ablecally|N/A|6416dim5948681701473|navajo|Each|Unknown|17|oughtn stoughtanti| +5192|AAAAAAAAIEEBAAAA|1997-10-27|2000-10-26|New interests feel home for the experiences. Services call numerous actions; ch|7.82|3.75|8016003|corpmaxi #3|16|golf|8|Sports|100|barbarought|N/A|44turquoise676857940|peach|Ton|Unknown|16|ablen stoughtanti| +5193|AAAAAAAAIEEBAAAA|2000-10-27||New interests feel home for the experiences. Services call numerous actions; ch|4.54|3.75|8016003|exportiimporto #2|16|pants|2|Men|100|barbarought|extra large|44turquoise676857940|royal|Cup|Unknown|43|prin stoughtanti| +5194|AAAAAAAAKEEBAAAA|1997-10-27|1999-10-27|Minutes run. Fast united words must allow partly high contents; neither good games could not get of course other principles; at present confident preparations get violentl|1.88|0.92|4003001|exportiedu pack #1|3|kids|4|Shoes|20|barable|medium|peach072406292445402|seashell|Dram|Unknown|14|esen stoughtanti| +5195|AAAAAAAAKEEBAAAA|1999-10-28|2001-10-26|Minutes run. Fast united words must allow partly high contents; neither good games could not get of course other principles; at present confident preparations get violentl|2.31|0.92|4003001|amalgexporti #2|1|newborn|3|Children|267|barable|petite|42482585396pale41578|grey|Ounce|Unknown|20|antin stoughtanti| +5196|AAAAAAAAKEEBAAAA|2001-10-27||Minutes run. Fast united words must allow partly high contents; neither good games could not get of course other principles; at present confident preparations get violentl|2.96|1.59|1004001|edu packamalg #1|4|swimwear|1|Women|454|barable|petite|42482585396pale41578|pale|Dram|Unknown|8|callyn stoughtanti| +5197|AAAAAAAANEEBAAAA|1997-10-27||Difficulties stand. Workers may wish s|2.31|0.80|3002002|importoexporti #2|2|infants|3|Children|926|callyablen st|extra large|2steel89802286685409|slate|Oz|Unknown|13|ationn stoughtanti| +5198|AAAAAAAAOEEBAAAA|1997-10-27|2000-10-26|There remarkable inhabitants cannot make inappropriate, basic tour|4.90|4.41|1002001|importoamalg #1|2|fragrances|1|Women|743|prieseation|large|36741midnight9609491|seashell|Unknown|Unknown|8|eingn stoughtanti| +5199|AAAAAAAAOEEBAAAA|2000-10-27||There remarkable inhabitants cannot make inappropriate, basic tour|5.46|2.83|1002001|importoexporti #2|2|infants|3|Children|92|prieseation|petite|36741midnight9609491|moccasin|Carton|Unknown|17|n stn stoughtanti| +5200|AAAAAAAAAFEBAAAA|1997-10-27|1999-10-27|Dry, bad elections should not work modern, clear regions. As good as violent signs will follow customers. Even casual years make clearly large magazines. Buildings want very roman, concerned branche|5.78|3.64|1003001|exportiamalg #1|3|maternity|1|Women|625|antiablecally|medium|30870linen4044317985|medium|Dram|Unknown|61|barbarableanti| +5201|AAAAAAAAAFEBAAAA|1999-10-28|2001-10-26|Collections shall not adju|3.01|3.64|4002002|importoedu pack #2|3|mens|4|Shoes|625|antiablecally|extra large|1446018violet5380733|deep|Tsp|Unknown|92|oughtbarableanti| +5202|AAAAAAAAAFEBAAAA|2001-10-27||Too biological wives overlap on a facts. Sur|9.58|3.64|4002002|exportiedu pack #1|3|kids|4|Shoes|944|antiablecally|small|deep8387416401075190|purple|Dozen|Unknown|23|ablebarableanti| +5203|AAAAAAAADFEBAAAA|1997-10-27||Professional eyes listen. Yet beautiful charges might drive roughly. Audiences play less cases. Existing, initial others should not help; left, partial tools ought to work partly there wrong person|4.82|1.97|7014004|edu packnameless #4|14|glassware|7|Home|275|antiationable|N/A|505002844722peru3539|salmon|Gram|Unknown|62|pribarableanti| +5204|AAAAAAAAEFEBAAAA|1997-10-27|2000-10-26|Old, personal difficulties shall not exist much terrible governments; in addition likely parties might not go probably wonderful, model uses. Christian, usual influences would tell mo|4.95|1.93|7007005|brandbrand #5|7|decor|7|Home|272|ableationable|N/A|2navy551110099757287|midnight|Gross|Unknown|53|esebarableanti| +5205|AAAAAAAAEFEBAAAA|2000-10-27||Musical, select animals acquire fixed ends. Foreign, grand preparations consider dangerous troops. Religious, leading childre|4.82|1.93|1003002|exportiamalg #2|7|maternity|1|Women|272|ableationable|extra large|1seashell01767800572|rosy|Bundle|Unknown|56|antibarableanti| +5206|AAAAAAAAGFEBAAAA|1997-10-27|1999-10-27|Ago rural mice must read new minutes. More safe levels step into a names. Walls conceive sensitive, old voices. Then cu|6.76|5.81|8008007|namelessnameless #7|8|outdoor|8|Sports|296|callyn stable|N/A|8318turquoise8176996|royal|Dozen|Unknown|57|callybarableanti| +5207|AAAAAAAAGFEBAAAA|1999-10-28|2001-10-26|Light, impossible directors must prote|2.48|1.98|1001002|amalgamalg #2|8|dresses|1|Women|153|callyn stable|medium|8318turquoise8176996|lawn|Ton|Unknown|56|ationbarableanti| +5208|AAAAAAAAGFEBAAAA|2001-10-27||Techniques shall come more other, english heads. Odd, young sites play merely qualities. New, parliamentary seconds f|4.20|1.98|1001001|amalgamalg #1|8|dresses|1|Women|226|callyn stable|economy|8318turquoise8176996|green|Ton|Unknown|92|eingbarableanti| +5209|AAAAAAAAJFEBAAAA|1997-10-27||Too necessary dreams should not co|3.75|2.77|9004002|edu packmaxi #2|4|entertainments|9|Books|761|oughtcallyation|N/A|554119778065saddle98|spring|Case|Unknown|66|n stbarableanti| +5210|AAAAAAAAKFEBAAAA|1997-10-27|2000-10-26|Again important minutes used to work key, essential details; formal clouds should not learn l|7.65|5.50|1004001|edu packamalg #1|4|swimwear|1|Women|628|eingablecally|petite|1815890205papaya5392|thistle|N/A|Unknown|41|baroughtableanti| +5211|AAAAAAAAKFEBAAAA|2000-10-27||Mainly complete equations use alternative things; open involved affairs develop almost effective machines. Federal words could not keep please as a subjects. Seconds will not see in a names. Cl|3.55|2.34|1004001|amalgamalgamalg #10|4|disk drives|10|Electronics|628|eingablecally|N/A|purple26966067895219|royal|Lb|Unknown|13|oughtoughtableanti| +5212|AAAAAAAAMFEBAAAA|1997-10-27|1999-10-27|Large, social reforms ought to lead hitherto well preliminary sides. Dead, environmental studies write sometimes. Mountains would argue so main laws. Either major pieces o|75.70|53.74|2004001|edu packimporto #1|4|sports-apparel|2|Men|660|barcallycally|small|82957spring817796937|thistle|Case|Unknown|10|ableoughtableanti| +5213|AAAAAAAAMFEBAAAA|1999-10-28|2001-10-26|Large, social reforms ought to lead hitherto well preliminary sides. Dead, environmental studies write sometimes. Mountains would argue so main laws. Either major pieces o|2.59|1.42|9010006|univunivamalg #6|10|travel|9|Books|660|barcallycally|N/A|1sienna7085883395761|thistle|Box|Unknown|3|prioughtableanti| +5214|AAAAAAAAMFEBAAAA|2001-10-27||Hopes m|6.48|1.42|1001001|amalgamalg #1|10|dresses|1|Women|660|barcallycally|extra large|1sienna7085883395761|red|Unknown|Unknown|58|eseoughtableanti| +5215|AAAAAAAAPFEBAAAA|1997-10-27||Poor, large reforms must give general months. Executive, old parts must want economic investigations. Still, other girls assist almost publications. Classes mean wi|63.66|49.01|8010006|univmaxi #6|10|pools|8|Sports|68|eingcally|N/A|220241035113212pink3|tan|Bundle|Unknown|96|antioughtableanti| +5216|AAAAAAAAAGEBAAAA|1997-10-27|2000-10-26|Now concerned efforts sue main, future instruments. Hardly deaf days may |2.74|1.06|10013015|exportiamalgamalg #15|13|stereo|10|Electronics|580|bareinganti|N/A|692228198599491rose7|slate|Tsp|Unknown|57|callyoughtableanti| +5217|AAAAAAAAAGEBAAAA|2000-10-27||Now concerned efforts sue main, future instruments. Hardly deaf days may |63.04|49.80|10013015|exporticorp #2|3|gold|6|Jewelry|580|bareinganti|N/A|692228198599491rose7|sienna|Box|Unknown|37|ationoughtableanti| +5218|AAAAAAAACGEBAAAA|1997-10-27|1999-10-27|Simply white u|6.49|2.98|3004001|edu packexporti #1|4|school-uniforms|3|Children|572|ableationanti|medium|74653859411sienna512|ivory|Cup|Unknown|64|eingoughtableanti| +5219|AAAAAAAACGEBAAAA|1999-10-28|2001-10-26|Simply white u|2.55|1.04|8003010|exportinameless #10|3|basketball|8|Sports|572|ableationanti|N/A|32sandy2687466682686|white|Gross|Unknown|30|n stoughtableanti| +5220|AAAAAAAACGEBAAAA|2001-10-27||Scientific studies read in ad|6.55|1.04|5001001|amalgscholar #1|3|rock|5|Music|128|eingableought|N/A|21607327midnight8608|seashell|Cup|Unknown|31|barableableanti| +5221|AAAAAAAAFGEBAAAA|1997-10-27||Chairs store much major owners. Long-term, civil profits rise mor|6.87|4.46|8008002|namelessnameless #2|8|outdoor|8|Sports|212|ableoughtable|N/A|28126826snow09090503|papaya|Tbl|Unknown|15|oughtableableanti| +5222|AAAAAAAAGGEBAAAA|1997-10-27|2000-10-26|Largely small arguments could make female, foreign titles. Ready, |2.77|1.24|7015003|scholarnameless #3|15|tables|7|Home|66|callycally|N/A|582pink2682257843558|steel|Each|Unknown|13|ableableableanti| +5223|AAAAAAAAGGEBAAAA|2000-10-27||Levels can go per a projects. Only, clear questions may control more parti|1.59|0.96|1001002|amalgamalg #2|1|dresses|1|Women|440|bareseese|extra large|582pink2682257843558|medium|Dozen|Unknown|5|priableableanti| +5224|AAAAAAAAIGEBAAAA|1997-10-27|1999-10-27|Proposed members would cut dangerously different years. Corresponding, special hundreds get so able, horrible teachers. As social do|5.87|4.75|8002005|importonameless #5|2|baseball|8|Sports|25|antiable|N/A|3895280tomato3017254|royal|Box|Unknown|22|eseableableanti| +5225|AAAAAAAAIGEBAAAA|1999-10-28|2001-10-26|Earlier hard authors worsen away successful readers. Technical, old ideas increase then. Glad, real colleagues should train eligibl|56.31|45.61|8002005|amalgscholar #2|1|rock|5|Music|25|antiable|N/A|3895280tomato3017254|saddle|Case|Unknown|34|antiableableanti| +5226|AAAAAAAAIGEBAAAA|2001-10-27||Earlier hard authors worsen away successful readers. Technical, old ideas increase then. Glad, real colleagues should train eligibl|4.62|2.17|4002001|importoedu pack #1|1|mens|4|Shoes|25|antiable|large|43173navy02310973619|navy|Pound|Unknown|5|callyableableanti| +5227|AAAAAAAALGEBAAAA|1997-10-27||Times should create subject years; again slow candidates build; recent effects think br|1.92|0.59|6012004|importobrand #4|12|costume|6|Jewelry|31|oughtpri|N/A|4papaya8240480654875|snow|Ounce|Unknown|34|ationableableanti| +5228|AAAAAAAAMGEBAAAA|1997-10-27|2000-10-26|Dramatic, cold eyes may tell too available, other lists. Still successful units get in order single dogs. Low infl|0.58|0.38|10003010|exportiunivamalg #10|3|dvd/vcr players|10|Electronics|478|eingationese|N/A|25965maroon685374712|maroon|N/A|Unknown|49|eingableableanti| +5229|AAAAAAAAMGEBAAAA|2000-10-27||Dramatic, cold eyes may tell too available, other lists. Still successful units get in order single dogs. Low infl|92.92|0.38|10003010|corpamalgamalg #11|16|wireless|10|Electronics|478|eingationese|N/A|25965maroon685374712|thistle|Each|Unknown|63|n stableableanti| +5230|AAAAAAAAOGEBAAAA|1997-10-27|1999-10-27|Free businessmen cause too basic, nice ideas. Great paintings used to advise now clothes; feelings shall occur just positive, assistant others. L|5.85|4.62|9013011|exportiunivamalg #11|13|self-help|9|Books|177|ationationought|N/A|13078603252517pink84|sandy|Unknown|Unknown|78|barpriableanti| +5231|AAAAAAAAOGEBAAAA|1999-10-28|2001-10-26|Free businessmen cause too basic, nice ideas. Great paintings used to advise now clothes; feelings shall occur just positive, assistant others. L|4.90|4.62|2002002|importoimporto #2|13|shirts|2|Men|177|ationationought|medium|13078603252517pink84|chartreuse|Box|Unknown|65|oughtpriableanti| +5232|AAAAAAAAOGEBAAAA|2001-10-27||Free businessmen cause too basic, nice ideas. Great paintings used to advise now clothes; feelings shall occur just positive, assistant others. L|6.32|4.48|5004001|edu packscholar #1|4|classical|5|Music|177|ationationought|N/A|13078603252517pink84|saddle|Ton|Unknown|7|ablepriableanti| +5233|AAAAAAAABHEBAAAA|1997-10-27||Previous things may not account; pounds need ultimately products. Owners must use that pleased, different opportunities. Typically diffi|0.20|0.16|6009002|maxicorp #2|9|womens watch|6|Jewelry|121|oughtableought|N/A|seashell285856101473|coral|Tbl|Unknown|37|pripriableanti| +5234|AAAAAAAACHEBAAAA|1997-10-27|2000-10-26|Bonds could rid so french options. Related, personal rules might not contact probably old, increased ages. Nights help certainly less dirty views. Forward alone respondents must not take legs. |7.20|4.60|8016005|corpmaxi #5|16|golf|8|Sports|359|n stantipri|N/A|47583237860coral9069|purple|Dram|Unknown|32|esepriableanti| +5235|AAAAAAAACHEBAAAA|2000-10-27||Capitalist, corporate colleagues contact to no things.|3.88|1.16|5004002|edu packscholar #2|4|classical|5|Music|61|n stantipri|N/A|28red182311475978192|spring|Dozen|Unknown|29|antipriableanti| +5236|AAAAAAAAEHEBAAAA|1997-10-27|1999-10-27|Vulnerable, poor requirements might not remember certainly foreign factors. Excellent days make indeed. Considerable theori|1.71|0.82|8011007|amalgmaxi #7|11|archery|8|Sports|572|ableationanti|N/A|seashell749232753032|plum|Gross|Unknown|11|callypriableanti| +5237|AAAAAAAAEHEBAAAA|1999-10-28|2001-10-26|Costs set always at a villages. Employees will show in a worlds. Special, effective animals make russians; of course isolated needs believe so too difficult scientists. Reasons might inclu|1.96|0.64|6010006|univbrand #6|10|jewelry boxes|6|Jewelry|634|ableationanti|N/A|seashell749232753032|wheat|Case|Unknown|10|ationpriableanti| +5238|AAAAAAAAEHEBAAAA|2001-10-27||Remote, wrong options ought to circumvent at least to the boats. Aspects avoid extensive days. Comfortab|6.49|0.64|3004001|edu packexporti #1|10|school-uniforms|3|Children|37|ationpri|small|409black556998154072|brown|Bunch|Unknown|36|eingpriableanti| +5239|AAAAAAAAHHEBAAAA|1997-10-27||Others come in addition voluntary issues. Nations shall not speak even social, educational results; old moments might laugh. Comparisons cost safe, middle problems. Right waves res|7.97|3.42|8008010|namelessnameless #10|8|outdoor|8|Sports|863|pricallyeing|N/A|3389pale045287010968|white|Pallet|Unknown|23|n stpriableanti| +5240|AAAAAAAAIHEBAAAA|1997-10-27|2000-10-26|Ways might not sense. Merely alone sentences may check payments; developments used to help ruthlessly between the republics. Usually|4.95|1.73|4002001|importoedu pack #1|2|mens|4|Shoes|76|callyation|petite|011212puff3957867627|orchid|Dram|Unknown|4|bareseableanti| +5241|AAAAAAAAIHEBAAAA|2000-10-27||Religious, possible months can like in a supporters. Females will not go ahead female parents; too royal experiences ought to |0.60|0.28|4002001|amalgamalg #2|2|dresses|1|Women|367|ationcallypri|petite|011212puff3957867627|antique|Pound|Unknown|5|oughteseableanti| +5242|AAAAAAAAKHEBAAAA|1997-10-27|1999-10-27|New, busy years think potentially to a lights. Much apparent individuals find still other places. Speakers could |4.76|3.04|9006011|corpmaxi #11|6|parenting|9|Books|553|priantianti|N/A|177900166peru6300322|spring|Bundle|Unknown|63|ableeseableanti| +5243|AAAAAAAAKHEBAAAA|1999-10-28|2001-10-26|New, busy years think potentially to a lights. Much apparent individuals find still other places. Speakers could |0.31|3.04|9015012|scholarunivamalg #12|15|fiction|9|Books|553|priantianti|N/A|604369sandy310810835|cornflower|Ton|Unknown|35|prieseableanti| +5244|AAAAAAAAKHEBAAAA|2001-10-27||Cases stop around suggestions. New, grey participants acknowledge better maybe military consequences; too statistical volumes cannot listen only underlying, scientific months. Advisory, spare techni|0.89|3.04|9015012|edu packimporto #1|4|sports-apparel|2|Men|5|anti|large|604369sandy310810835|seashell|Lb|Unknown|27|eseeseableanti| +5245|AAAAAAAANHEBAAAA|1997-10-27||Arts differ here; able, interesting movements could keep swiss, harsh books. Particularly specified conditions pay long-term, separate forces. Late, typical tw|9.81|3.33|1004002|edu packamalg #2|4|swimwear|1|Women|583|prieinganti|extra large|57945708blanched3060|sienna|Each|Unknown|3|antieseableanti| +5246|AAAAAAAAOHEBAAAA|1997-10-27|2000-10-26|Badly assistant pictures order best blue jobs. Budgets allow moreover gold, other purposes; workers undermine. Fe|0.80|0.41|8015007|scholarmaxi #7|15|fishing|8|Sports|995|antin stn st|N/A|9767049762555sienna1|snow|Gross|Unknown|30|callyeseableanti| +5247|AAAAAAAAOHEBAAAA|2000-10-27||Laboratories can assess much happy children. Very bloody characteristics might not use clean to a techniques. Temporary gates learn sometimes on a directions. Black, russian streets see indeed. El|3.67|1.17|8015007|importoedu pack #2|2|mens|4|Shoes|995|antin stn st|large|30930820966black2910|saddle|Each|Unknown|5|ationeseableanti| +5248|AAAAAAAAAIEBAAAA|1997-10-27|1999-10-27|More than small councils might not go also i|0.91|0.36|8015009|scholarmaxi #9|15|fishing|8|Sports|710|baroughtation|N/A|0white04934982831187|tomato|Box|Unknown|58|eingeseableanti| +5249|AAAAAAAAAIEBAAAA|1999-10-28|2001-10-26|More than small councils might not go also i|86.87|0.36|1003002|exportiamalg #2|3|maternity|1|Women|710|baroughtation|medium|white295616108585335|dim|Dozen|Unknown|59|n steseableanti| +5250|AAAAAAAAAIEBAAAA|2001-10-27||Then casual locations win now responsible departments; monetary strangers catch perhaps public, rare men. Method|1.04|0.86|1003002|edu packexporti #1|3|school-uniforms|3|Children|126|callyableought|large|white295616108585335|rose|Box|Unknown|18|barantiableanti| +5251|AAAAAAAADIEBAAAA|1997-10-27||Quite english programmes plead specialists. Major drawings used to know as a votes. Also civil varieties produce partly services. Useful, large allegations sleep changes. Unique, simila|6.76|5.54|10003016|exportiunivamalg #16|3|dvd/vcr players|10|Electronics|15|antiought|N/A|055869885342royal299|pale|Bunch|Unknown|30|oughtantiableanti| +5252|AAAAAAAAEIEBAAAA|1997-10-27|2000-10-26|Illegal, clear documents must pay in a brothers. Long characters make. Meetings should assure pages. Women remain as in a dreams. Police might expect thus at a women. Just sharp|56.88|48.91|6003005|exporticorp #5|3|gold|6|Jewelry|300|barbarpri|N/A|7112592089peach74653|tan|Gram|Unknown|66|ableantiableanti| +5253|AAAAAAAAEIEBAAAA|2000-10-27||Illegal, clear documents must pay in a brothers. Long characters make. Meetings should assure pages. Women remain as in a dreams. Police might expect thus at a women. Just sharp|1.08|0.91|6003005|edu packbrand #4|4|curtains/drapes|7|Home|300|barbarpri|N/A|6spring9631535231292|peru|Gram|Unknown|9|priantiableanti| +5254|AAAAAAAAGIEBAAAA|1997-10-27|1999-10-27|Unknown indians may wind still|88.12|67.85|8002005|importonameless #5|2|baseball|8|Sports|195|antin stought|N/A|33244999312floral880|salmon|Ton|Unknown|15|eseantiableanti| +5255|AAAAAAAAGIEBAAAA|1999-10-28|2001-10-26|Fingers mean much social jobs; never profess|8.27|3.63|8002005|amalgamalg #2|2|dresses|1|Women|195|antin stought|medium|33244999312floral880|snow|Bundle|Unknown|9|antiantiableanti| +5256|AAAAAAAAGIEBAAAA|2001-10-27||Services return; guidelines start great circumstances. F|8.12|5.68|6012001|importobrand #1|12|costume|6|Jewelry|72|antin stought|N/A|91sienna054833470253|snow|Gross|Unknown|11|callyantiableanti| +5257|AAAAAAAAJIEBAAAA|1997-10-27||Homes stand actively identical reasons. Downstairs confident weapons may not knock. Laws should not speculate int|35.98|21.22|5004002|edu packscholar #2|4|classical|5|Music|421|oughtableese|N/A|5131684934908almond7|smoke|Box|Unknown|18|ationantiableanti| +5258|AAAAAAAAKIEBAAAA|1997-10-27|2000-10-26|National, mental cars would not agree from a times; for example possible streets discuss |0.87|0.44|3001001|amalgexporti #1|1|newborn|3|Children|322|ableablepri|medium|59825281051rose99332|turquoise|Ton|Unknown|51|eingantiableanti| +5259|AAAAAAAAKIEBAAAA|2000-10-27||Alone men put wide, recent minutes. Elderly days could see just expensive, sure changes. Interests will tel|5.39|1.72|3001001|exportischolar #2|3|pop|5|Music|322|ableablepri|N/A|59825281051rose99332|mint|Tbl|Unknown|39|n stantiableanti| +5260|AAAAAAAAMIEBAAAA|1997-10-27|1999-10-27|Shy, young areas would return indeed obvious entries. Following, major villages require for the circumstances. Accordingly safe minutes specify. Serious|5.29|3.96|8012001|importomaxi #1|12|guns|8|Sports|253|priantiable|N/A|34sienna056889040969|tomato|Each|Unknown|92|barcallyableanti| +5261|AAAAAAAAMIEBAAAA|1999-10-28|2001-10-26|Shy, young areas would return indeed obvious entries. Following, major villages require for the circumstances. Accordingly safe minutes specify. Serious|7.36|4.34|8012001|edu packexporti #2|4|school-uniforms|3|Children|253|priantiable|economy|plum8185791281034478|tomato|Case|Unknown|12|oughtcallyableanti| +5262|AAAAAAAAMIEBAAAA|2001-10-27||Entirely similar concessions reduce so ready teachers. True years break by no means books. Reasons say. Tough sums make hence ever normal women; statis|9.17|6.87|5003001|exportischolar #1|3|pop|5|Music|253|priantiable|N/A|4spring9456394353458|spring|Unknown|Unknown|25|ablecallyableanti| +5263|AAAAAAAAPIEBAAAA|1997-10-27||Main cheeks must put |0.45|0.19|9001008|amalgmaxi #8|1|arts|9|Books|164|esecallyought|N/A|982maroon58697339466|metallic|Tsp|Unknown|59|pricallyableanti| +5264|AAAAAAAAAJEBAAAA|1997-10-27|2000-10-26|Supposedly young friends show only common steps. Well li|60.66|49.74|8002005|importonameless #5|2|baseball|8|Sports|417|ationoughtese|N/A|grey4402083793976839|rosy|Dozen|Unknown|11|esecallyableanti| +5265|AAAAAAAAAJEBAAAA|2000-10-27||Entire, good others could address around. Human, important years come colonial, financial errors. Cars may assist; islands may not chann|6.71|49.74|2003002|exportiimporto #2|2|pants|2|Men|644|eseesecally|medium|580689397thistle3480|slate|N/A|Unknown|6|anticallyableanti| +5266|AAAAAAAACJEBAAAA|1997-10-27|1999-10-27|Professional friends will discover never irish goods. Miserable males heal obviously dur|0.72|0.28|5001001|amalgscholar #1|1|rock|5|Music|621|oughtablecally|N/A|smoke455998442882715|lime|Unknown|Unknown|36|callycallyableanti| +5267|AAAAAAAACJEBAAAA|1999-10-28|2001-10-26|Professional friends will discover never irish goods. Miserable males heal obviously dur|5.58|4.46|9004012|edu packmaxi #12|4|entertainments|9|Books|621|oughtablecally|N/A|93521moccasin3259583|spring|Pound|Unknown|14|ationcallyableanti| +5268|AAAAAAAACJEBAAAA|2001-10-27||Originally high observations might permit both old sources; days should not buy very real months. Open favorable forms should like from a fingers. Suppliers experience later. Im|51.02|4.46|9004012|scholarunivamalg #1|4|fiction|9|Books|621|oughtablecally|N/A|93521moccasin3259583|steel|Bunch|Unknown|23|eingcallyableanti| +5269|AAAAAAAAFJEBAAAA|1997-10-27||Good opponents like thus years. Great attitudes must not give smooth, free authorities. Rather rare |9.23|7.10|10014017|edu packamalgamalg #17|14|automotive|10|Electronics|544|eseeseanti|N/A|11944505saddle814191|blue|Case|Unknown|16|n stcallyableanti| +5270|AAAAAAAAGJEBAAAA|1997-10-27|2000-10-26|Main, new areas eliminate eventually effectively royal facts; eligible, royal sources might reject nearly political seconds. Communities|0.95|0.73|2001001|amalgimporto #1|1|accessories|2|Men|155|antiantiought|extra large|817059768500slate068|rose|Dozen|Unknown|79|barationableanti| +5271|AAAAAAAAGJEBAAAA|2000-10-27||Main, new areas eliminate eventually effectively royal facts; eligible, royal sources might reject nearly political seconds. Communities|9.41|7.43|2001001|edu packnameless #2|14|glassware|7|Home|155|antiantiought|N/A|4571red7556844248536|linen|Lb|Unknown|66|oughtationableanti| +5272|AAAAAAAAIJEBAAAA|1997-10-27|1999-10-27|Extra copies must check advanced, satisfied ages. Healthy, colonial actions used to give again appropriate services; objections listen to a feel|7.23|4.91|1001001|amalgamalg #1|1|dresses|1|Women|328|eingablepri|large|51914735slate0720065|smoke|Ton|Unknown|21|ableationableanti| +5273|AAAAAAAAIJEBAAAA|1999-10-28|2001-10-26|Extra copies must check advanced, satisfied ages. Healthy, colonial actions used to give again appropriate services; objections listen to a feel|2.50|4.91|1001001|edu packedu pack #2|4|athletic|4|Shoes|328|eingablepri|extra large|51914735slate0720065|aquamarine|Carton|Unknown|34|priationableanti| +5274|AAAAAAAAIJEBAAAA|2001-10-27||Members can learn however aside new others. Firm years could present eastwards fair constitutional hours. Various attitudes describe very professional |4.07|4.91|1001001|edu packbrand #3|4|estate|6|Jewelry|667|ationcallycally|N/A|51914735slate0720065|violet|Cup|Unknown|41|eseationableanti| +5275|AAAAAAAALJEBAAAA|1997-10-27||Different agreements must not hear then in a concerns. Sections drift consciously; likely times cannot work at least in a bodies. Southern differences cause on a |9.27|7.78|3003002|exportiexporti #2|3|toddlers|3|Children|139|n stpriought|small|9042811powder8063460|honeydew|Ounce|Unknown|57|antiationableanti| +5276|AAAAAAAAMJEBAAAA|1997-10-27|2000-10-26|Permanently following units keep then different, preliminary heads. Still legal pounds must suppor|6.55|5.43|4003001|exportiedu pack #1|3|kids|4|Shoes|332|ablepripri|extra large|96505315425826rose10|pale|Bundle|Unknown|55|callyationableanti| +5277|AAAAAAAAMJEBAAAA|2000-10-27||Permanently following units keep then different, preliminary heads. Still legal pounds must suppor|0.93|0.76|6008006|namelesscorp #6|3|mens watch|6|Jewelry|172|ableationought|N/A|0856119092544green74|peru|Cup|Unknown|86|ationationableanti| +5278|AAAAAAAAOJEBAAAA|1997-10-27|1999-10-27|Equal rules provide for a areas. So future visitors used to walk luckil|0.95|0.64|1004001|edu packamalg #1|4|swimwear|1|Women|290|barn stable|extra large|5928831blush10944554|sienna|Case|Unknown|23|eingationableanti| +5279|AAAAAAAAOJEBAAAA|1999-10-28|2001-10-26|Doctors can think primary, easy components. Sufficiently married letters should say new, direct states. There young sites may not reach more prime, comprehensive reasons. Shar|9.40|3.29|8005010|scholarnameless #10|5|fitness|8|Sports|290|barn stable|N/A|rose4348834959676453|purple|Lb|Unknown|18|n stationableanti| +5280|AAAAAAAAOJEBAAAA|2001-10-27||Doctors can think primary, easy components. Sufficiently married letters should say new, direct states. There young sites may not reach more prime, comprehensive reasons. Shar|7.97|3.29|3001001|amalgexporti #1|1|newborn|3|Children|319|n stoughtpri|petite|rose4348834959676453|sandy|Cup|Unknown|68|bareingableanti| +5281|AAAAAAAABKEBAAAA|1997-10-27||Good, single pupils should not combine prisoners; a.d. strong shelves mean now p|0.83|0.47|8010002|univmaxi #2|10|pools|8|Sports|548|eingeseanti|N/A|7854972048firebrick9|turquoise|Dram|Unknown|20|oughteingableanti| +5282|AAAAAAAACKEBAAAA|1997-10-27|2000-10-26|Doubtful, urban humans make simply for a needs. Able hours take somewhere in a leaves. Other,|9.33|4.57|3004001|edu packexporti #1|4|school-uniforms|3|Children|279|n stationable|large|76951183azure9836887|thistle|Lb|Unknown|69|ableeingableanti| +5283|AAAAAAAACKEBAAAA|2000-10-27||Restrictions help very; now cle|1.04|4.57|3004001|amalgedu pack #2|4|womens|4|Shoes|279|n stationable|extra large|76951183azure9836887|rosy|Lb|Unknown|39|prieingableanti| +5284|AAAAAAAAEKEBAAAA|1997-10-27|1999-10-27|Now major trac|2.41|1.51|5003001|exportischolar #1|3|pop|5|Music|147|ationeseought|N/A|813807148sienna17191|sienna|Each|Unknown|23|eseeingableanti| +5285|AAAAAAAAEKEBAAAA|1999-10-28|2001-10-26|Now major trac|1.48|1.51|5003001|amalgscholar #2|1|rock|5|Music|147|ationeseought|N/A|813807148sienna17191|ivory|Each|Unknown|38|antieingableanti| +5286|AAAAAAAAEKEBAAAA|2001-10-27||Far chinese speakers must not visit strategies.|9.82|6.57|5003001|namelessnameless #7|1|outdoor|8|Sports|147|ationeseought|N/A|813807148sienna17191|steel|Carton|Unknown|18|callyeingableanti| +5287|AAAAAAAAHKEBAAAA|1997-10-27||Central ships will not muster necessary buildings. Activities request years.|0.26|0.11|3003002|exportiexporti #2|3|toddlers|3|Children|219|n stoughtable|large|1431807328lime677962|violet|Unknown|Unknown|83|ationeingableanti| +5288|AAAAAAAAIKEBAAAA|1997-10-27|2000-10-26|More running months ought to estab|1.24|0.81|7004009|edu packbrand #9|4|curtains/drapes|7|Home|7|ation|N/A|23570692steel4953399|navy|Unknown|Unknown|29|eingeingableanti| +5289|AAAAAAAAIKEBAAAA|2000-10-27||Needs might master too in the relations. Temporarily ready roads shall become new, regular words; diverse, other visitors concentrate small, local firms; special, political months m|3.27|1.53|9002010|importomaxi #10|4|business|9|Books|553|ation|N/A|powder48480254652819|spring|Ton|Unknown|6|n steingableanti| +5290|AAAAAAAAKKEBAAAA|1997-10-27|1999-10-27|Economic, normal voters find particular, new steps. Areas ought to live japanese programmes. Years could seem directly. About |1.19|0.36|5002001|importoscholar #1|2|country|5|Music|399|n stn stpri|N/A|523khaki942649340145|lace|Bunch|Unknown|84|barn stableanti| +5291|AAAAAAAAKKEBAAAA|1999-10-28|2001-10-26|Economic, normal voters find particular, new steps. Areas ought to live japanese programmes. Years could seem directly. About |0.57|0.41|5002001|importomaxi #2|2|guns|8|Sports|236|n stn stpri|N/A|523khaki942649340145|indian|Box|Unknown|25|oughtn stableanti| +5292|AAAAAAAAKKEBAAAA|2001-10-27||Particular concepts make at an shapes; sole, national jobs kiss how|7.99|0.41|5002001|importoscholar #1|2|country|5|Music|236|callypriable|N/A|0plum321960540795973|powder|Lb|Unknown|22|ablen stableanti| +5293|AAAAAAAANKEBAAAA|1997-10-27||Long running backs might not bring ve|6.54|4.90|10001007|amalgunivamalg #7|1|cameras|10|Electronics|151|oughtantiought|N/A|1179601447powder4869|wheat|Each|Unknown|65|prin stableanti| +5294|AAAAAAAAOKEBAAAA|1997-10-27|2000-10-26|Other organisations imagine often therefore stable blues; horses might grasp things. Talks should not let apparently growing authorities. Factors rescue local objections. Probably wild trustees woul|8.38|3.77|8006005|corpnameless #5|6|football|8|Sports|142|ableeseought|N/A|718902746cornsilk869|light|Bunch|Unknown|38|esen stableanti| +5295|AAAAAAAAOKEBAAAA|2000-10-27||Other organisations imagine often therefore stable blues; horses might grasp things. Talks should not let apparently growing authorities. Factors rescue local objections. Probably wild trustees woul|0.50|3.77|2003002|exportiimporto #2|6|pants|2|Men|142|ableeseought|medium|03steel0124593936115|peach|Ounce|Unknown|66|antin stableanti| +5296|AAAAAAAAALEBAAAA|1997-10-27|1999-10-27|Financial trees will include all administrative colleges. Local windows will go happy hundreds; other details must not press h|0.97|0.51|4002001|importoedu pack #1|2|mens|4|Shoes|315|antioughtpri|medium|554112038664spring94|antique|Carton|Unknown|57|callyn stableanti| +5297|AAAAAAAAALEBAAAA|1999-10-28|2001-10-26|Financial trees will include all administrative colleges. Local windows will go happy hundreds; other details must not press h|1.55|0.51|4002001|amalgunivamalg #6|11|cooking|9|Books|229|antioughtpri|N/A|46209ghost6635093210|hot|Oz|Unknown|21|ationn stableanti| +5298|AAAAAAAAALEBAAAA|2001-10-27||Financial trees will include all administrative colleges. Local windows will go happy hundreds; other details must not press h|3.12|0.51|4002001|edu packmaxi #9|14|tennis|8|Sports|229|antioughtpri|N/A|46209ghost6635093210|papaya|Tsp|Unknown|27|eingn stableanti| +5299|AAAAAAAADLEBAAAA|1997-10-27||Responsibilities require ships. Women ought to accept as to the pp.; huge children could hold wonderful, wil|0.66|0.28|9008008|namelessmaxi #8|8|romance|9|Books|535|antiprianti|N/A|66947844tan287840901|rosy|Tsp|Unknown|30|n stn stableanti| +5300|AAAAAAAAELEBAAAA|1997-10-27|2000-10-26|Roughly recent thanks change seldom with a generations. New, neighbouring effects must come personal years. Records can save importan|3.31|1.82|10007014|brandunivamalg #14|7|personal|10|Electronics|745|antieseation|N/A|8051564783886sienna4|rosy|Bundle|Unknown|33|barbarprianti| +5301|AAAAAAAAELEBAAAA|2000-10-27||Roughly recent thanks change seldom with a generations. New, neighbouring effects must come personal years. Records can save importan|2.71|1.82|3001002|amalgexporti #2|7|newborn|3|Children|66|callycally|medium|9091366734bisque9133|powder|Tsp|Unknown|26|oughtbarprianti| +5302|AAAAAAAAGLEBAAAA|1997-10-27|1999-10-27|Only students might not discove|4.73|1.98|3001001|amalgexporti #1|1|newborn|3|Children|510|baroughtanti|extra large|2649tan0507752359885|blanched|Box|Unknown|28|ablebarprianti| +5303|AAAAAAAAGLEBAAAA|1999-10-28|2001-10-26|Stars should talk less than with a gains. Lives must give with |1.07|0.80|3001001|importoedu pack #2|2|mens|4|Shoes|510|baroughtanti|petite|2649tan0507752359885|tomato|Gram|Unknown|15|pribarprianti| +5304|AAAAAAAAGLEBAAAA|2001-10-27||Stars should talk less than with a gains. Lives must give with |5.81|0.80|5003001|exportischolar #1|2|pop|5|Music|13|baroughtanti|N/A|2649tan0507752359885|spring|Ton|Unknown|27|esebarprianti| +5305|AAAAAAAAJLEBAAAA|1997-10-27||Particular victims get slightly hospitals. Spiritual comments express sooner. Courts reduce new, electric eyes. English eyes could not ensure later on a files; central members make rural managers. Ne|1.99|1.77|4003002|exportiedu pack #2|3|kids|4|Shoes|487|ationeingese|small|steel198171862240821|tan|Unknown|Unknown|55|antibarprianti| +5306|AAAAAAAAKLEBAAAA|1997-10-27|2000-10-26|Financ|2.70|1.89|1004001|edu packamalg #1|4|swimwear|1|Women|41|oughtese|extra large|22smoke0465575696187|bisque|Box|Unknown|18|callybarprianti| +5307|AAAAAAAAKLEBAAAA|2000-10-27||Financ|3.43|1.47|10004004|edu packunivamalg #4|4|audio|10|Electronics|41|oughtese|N/A|22smoke0465575696187|purple|Tbl|Unknown|57|ationbarprianti| +5308|AAAAAAAAMLEBAAAA|1997-10-27|1999-10-27|Large, good flights facilitate nearly proposals; british conditions try. Futures lie beautifully laws. Then local resources install numbers. Also jewis|0.56|0.49|3003001|exportiexporti #1|3|toddlers|3|Children|5|anti|petite|65940bisque249481730|pale|Ounce|Unknown|37|eingbarprianti| +5309|AAAAAAAAMLEBAAAA|1999-10-28|2001-10-26|Dates pray from a courses. Goals will come yet regularly divine teams.|6.82|0.49|3003001|amalgexporti #2|1|newborn|3|Children|5|anti|medium|428689snow2121903644|rose|Dozen|Unknown|8|n stbarprianti| +5310|AAAAAAAAMLEBAAAA|2001-10-27||Dates pray from a courses. Goals will come yet regularly divine teams.|2.75|1.07|3003001|exportiexporti #1|1|toddlers|3|Children|439|anti|petite|78760630salmon604377|white|N/A|Unknown|2|baroughtprianti| +5311|AAAAAAAAPLEBAAAA|1997-10-27||British, usual stars can enable dead re|2.86|1.40|5002002|importoscholar #2|2|country|5|Music|184|eseeingought|N/A|9cyan631256620000402|slate|Ounce|Unknown|58|oughtoughtprianti| +5312|AAAAAAAAAMEBAAAA|1997-10-27|2000-10-26|Only lights may talk more other relatives; direct services buy with a houses. Ago annual worker|3.24|1.19|6011001|amalgbrand #1|11|semi-precious|6|Jewelry|283|prieingable|N/A|86misty4539565077592|medium|Each|Unknown|31|ableoughtprianti| +5313|AAAAAAAAAMEBAAAA|2000-10-27||Only lights may talk more other relatives; direct services buy with a houses. Ago annual worker|0.38|1.19|2003002|exportiimporto #2|11|pants|2|Men|283|prieingable|medium|86misty4539565077592|smoke|Bunch|Unknown|12|prioughtprianti| +5314|AAAAAAAACMEBAAAA|1997-10-27|1999-10-27|High practices resist for the performances. Competent, general relations can make round managers; low, advanced origins come however; minimum, large-scale divi|4.86|1.74|6007003|brandcorp #3|7|pendants|6|Jewelry|578|eingationanti|N/A|58995221202tomato117|papaya|Dozen|Unknown|52|eseoughtprianti| +5315|AAAAAAAACMEBAAAA|1999-10-28|2001-10-26|Never wooden talks reflect never however new negotiations. Technical, other miles receive please single, individual variables. All fine duties|7.58|5.38|6007003|amalgamalg #2|7|dresses|1|Women|578|eingationanti|extra large|58995221202tomato117|antique|Tbl|Unknown|5|antioughtprianti| +5316|AAAAAAAACMEBAAAA|2001-10-27||Member|0.97|0.51|8008007|namelessnameless #7|7|outdoor|8|Sports|687|ationeingcally|N/A|21198883slate2719701|papaya|Lb|Unknown|18|callyoughtprianti| +5317|AAAAAAAAFMEBAAAA|1997-10-27||Feet can let |2.72|2.33|10002014|importounivamalg #14|2|camcorders|10|Electronics|51|oughtanti|N/A|699111410872rosy7212|orange|Dozen|Unknown|61|ationoughtprianti| +5318|AAAAAAAAGMEBAAAA|1997-10-27|2000-10-26|So new questions survive free fields. Alone true documen|4.63|2.40|2001001|amalgimporto #1|1|accessories|2|Men|486|callyeingese|large|6656royal03957770497|royal|Gram|Unknown|40|eingoughtprianti| +5319|AAAAAAAAGMEBAAAA|2000-10-27||So new questions survive free fields. Alone true documen|1.28|0.89|2001002|amalgimporto #2|1|accessories|2|Men|213|prioughtable|medium|6656royal03957770497|beige|Lb|Unknown|17|n stoughtprianti| +5320|AAAAAAAAIMEBAAAA|1997-10-27|1999-10-27|Messages may prevent now streets. Efficient, crucial years might not mind actually from a sons. Electronic requirements might not limit permanently then objective members. Lo|3.79|1.47|3003001|exportiexporti #1|3|toddlers|3|Children|789|n steingation|extra large|82376947502463royal2|red|Bundle|Unknown|16|barableprianti| +5321|AAAAAAAAIMEBAAAA|1999-10-28|2001-10-26|Messages may prevent now streets. Efficient, crucial years might not mind actually from a sons. Electronic requirements might not limit permanently then objective members. Lo|1.74|1.47|3003001|edu packscholar #2|4|classical|5|Music|160|barcallyought|N/A|82376947502463royal2|lime|Gross|Unknown|48|oughtableprianti| +5322|AAAAAAAAIMEBAAAA|2001-10-27||Messages may prevent now streets. Efficient, crucial years might not mind actually from a sons. Electronic requirements might not limit permanently then objective members. Lo|2.95|1.94|10015002|scholaramalgamalg #2|4|portable|10|Electronics|160|barcallyought|N/A|96878174248smoke5534|puff|Carton|Unknown|36|ableableprianti| +5323|AAAAAAAALMEBAAAA|1997-10-27||Sophisticated, national others come often others. Different churches may not keep at least human words. Usually daily shoes shall not steer purely trains. Blue ears shall not s|0.70|0.44|1001002|amalgamalg #2|1|dresses|1|Women|168|eingcallyought|extra large|5436120727magenta794|moccasin|Carton|Unknown|4|priableprianti| +5324|AAAAAAAAMMEBAAAA|1997-10-27|2000-10-26|Common men would create perhaps dates. Unfortunately intermediate women di|68.04|26.53|5002001|importoscholar #1|2|country|5|Music|62|ablecally|N/A|3169684634purple9226|peru|Tsp|Unknown|52|eseableprianti| +5325|AAAAAAAAMMEBAAAA|2000-10-27||Common men would create perhaps dates. Unfortunately intermediate women di|0.55|26.53|5002001|exportiexporti #2|2|toddlers|3|Children|28|eingable|medium|178peru0295661352495|steel|Bundle|Unknown|29|antiableprianti| +5326|AAAAAAAAOMEBAAAA|1997-10-27|1999-10-27|Students cannot teach only shares. Common, logical results might not |0.32|0.11|9010011|univunivamalg #11|10|travel|9|Books|387|ationeingpri|N/A|5635554883610lime839|yellow|Ounce|Unknown|98|callyableprianti| +5327|AAAAAAAAOMEBAAAA|1999-10-28|2001-10-26|Satisfactory, clear futures influe|8.92|0.11|9010011|amalgamalg #2|1|dresses|1|Women|387|ationeingpri|small|5635554883610lime839|wheat|Bundle|Unknown|8|ationableprianti| +5328|AAAAAAAAOMEBAAAA|2001-10-27||Satisfactory, clear futures influe|6.85|4.17|9010011|edu packunivamalg #1|14|sports|9|Books|133|ationeingpri|N/A|5635554883610lime839|moccasin|Pound|Unknown|10|eingableprianti| +5329|AAAAAAAABNEBAAAA|1997-10-27||Low, concerned children see types. Drugs may bring probably there tradi|9.24|4.25|10007009|brandunivamalg #9|7|personal|10|Electronics|708|eingbaration|N/A|477620579slate672811|spring|Carton|Unknown|26|n stableprianti| +5330|AAAAAAAACNEBAAAA|1997-10-27|2000-10-26|Initially dangerous things might not vote ago great, decisive proteins. Curre|1.31|0.39|4003001|exportiedu pack #1|3|kids|4|Shoes|543|prieseanti|petite|63685papaya780351109|sandy|Carton|Unknown|44|barpriprianti| +5331|AAAAAAAACNEBAAAA|2000-10-27||Initially dangerous things might not vote ago great, decisive proteins. Curre|2.47|1.55|4003001|exportischolar #2|3|pop|5|Music|543|prieseanti|N/A|953141smoke386317849|sienna|Carton|Unknown|61|oughtpriprianti| +5332|AAAAAAAAENEBAAAA|1997-10-27|1999-10-27|Germans advise articles. Daily likely questions may not depend nevertheless; private, existing females might try. Social, ready schemes can start over a moves; old, intellectu|4.53|2.89|6009005|maxicorp #5|9|womens watch|6|Jewelry|550|barantianti|N/A|violet98062590023584|smoke|Dozen|Unknown|30|ablepriprianti| +5333|AAAAAAAAENEBAAAA|1999-10-28|2001-10-26|Germans advise articles. Daily likely questions may not depend nevertheless; private, existing females might try. Social, ready schemes can start over a moves; old, intellectu|3.22|2.89|6009005|scholarnameless #4|5|fitness|8|Sports|66|callycally|N/A|violet98062590023584|chiffon|Tbl|Unknown|28|pripriprianti| +5334|AAAAAAAAENEBAAAA|2001-10-27||Germans advise articles. Daily likely questions may not depend nevertheless; private, existing females might try. Social, ready schemes can start over a moves; old, intellectu|7.44|2.89|6009005|exportiimporto #1|5|pants|2|Men|192|callycally|extra large|violet98062590023584|medium|Box|Unknown|6|esepriprianti| +5335|AAAAAAAAHNEBAAAA|1997-10-27||Cultural, critical descriptions shall get hands. Lips afford unknown benefits. Due layers move yes|1.34|0.76|8008006|namelessnameless #6|8|outdoor|8|Sports|199|n stn stought|N/A|59264769navajo166840|pale|Oz|Unknown|9|antipriprianti| +5336|AAAAAAAAINEBAAAA|1997-10-27|2000-10-26|Cold, federal banks shall not look simply a|85.47|31.62|4004001|edu packedu pack #1|4|athletic|4|Shoes|369|n stcallypri|large|186031turquoise65807|almond|Box|Unknown|56|callypriprianti| +5337|AAAAAAAAINEBAAAA|2000-10-27||Cold, federal banks shall not look simply a|6.90|5.79|2004002|edu packimporto #2|4|sports-apparel|2|Men|617|n stcallypri|large|3532959837855plum487|sienna|Oz|Unknown|18|ationpriprianti| +5338|AAAAAAAAKNEBAAAA|1997-10-27|1999-10-27|Dynamic rates should ban therefore willing, unable rights. Scientists used to elucidate short small, french trains. Statements could not surprise ever with a rooms; later goo|7.33|2.49|4004001|edu packedu pack #1|4|athletic|4|Shoes|206|callybarable|large|688500magenta1376649|purple|Case|Unknown|13|eingpriprianti| +5339|AAAAAAAAKNEBAAAA|1999-10-28|2001-10-26|Dynamic rates should ban therefore willing, unable rights. Scientists used to elucidate short small, french trains. Statements could not surprise ever with a rooms; later goo|4.32|2.49|4004001|exportischolar #2|4|pop|5|Music|206|callybarable|N/A|688500magenta1376649|navy|Tbl|Unknown|13|n stpriprianti| +5340|AAAAAAAAKNEBAAAA|2001-10-27||Administrative decisions ought to play rarely old casualties. Levels meet nearly of cou|0.43|0.18|4004001|exportiamalgamalg #3|4|stereo|10|Electronics|206|callybarable|N/A|688500magenta1376649|salmon|Gram|Unknown|18|bareseprianti| +5341|AAAAAAAANNEBAAAA|1997-10-27||Terrible aspects shall open hard violent muscles. Namely internal incentives used to meet in a jobs. Eyes seem visible institutions. Depths go on a trees; european things turn. Various laws |1.26|0.52|6015006|scholarbrand #6|15|custom|6|Jewelry|542|ableeseanti|N/A|9438puff882968580713|slate|Oz|Unknown|24|oughteseprianti| +5342|AAAAAAAAONEBAAAA|1997-10-27|2000-10-26|Doubtful years control very benefits. Beautiful, surprised spirits could integrate international, causal techniques. Lists will accept for example |0.96|0.67|2003001|exportiimporto #1|3|pants|2|Men|649|n stesecally|medium|528101719885saddle81|ivory|Ounce|Unknown|14|ableeseprianti| +5343|AAAAAAAAONEBAAAA|2000-10-27||Doubtful years control very benefits. Beautiful, surprised spirits could integrate international, causal techniques. Lists will accept for example |31.85|28.66|2003001|edu packscholar #2|3|classical|5|Music|977|n stesecally|N/A|528101719885saddle81|khaki|Pallet|Unknown|56|prieseprianti| +5344|AAAAAAAAAOEBAAAA|1997-10-27|1999-10-27|Leads will use into a houses. Inc goals hear among a boards. Atlantic patients carry. Available acids mind afterwards then elderly trains|0.76|0.48|5003001|exportischolar #1|3|pop|5|Music|592|ablen stanti|N/A|749lace9336628617429|snow|Lb|Unknown|19|eseeseprianti| +5345|AAAAAAAAAOEBAAAA|1999-10-28|2001-10-26|Leads will use into a houses. Inc goals hear among a boards. Atlantic patients carry. Available acids mind afterwards then elderly trains|4.65|0.48|5003001|exportiimporto #2|3|pants|2|Men|68|eingcally|medium|767645cream486366969|sky|Box|Unknown|18|antieseprianti| +5346|AAAAAAAAAOEBAAAA|2001-10-27||Main, appointed parties must not ban also compl|3.36|2.25|5003001|importoexporti #1|2|infants|3|Children|313|prioughtpri|large|54083216rose43038422|goldenrod|Bunch|Unknown|56|callyeseprianti| +5347|AAAAAAAADOEBAAAA|1997-10-27||Manufacturers cannot think more positive copies. Seats explain in a doctors. Env|8.14|3.82|8002008|importonameless #8|2|baseball|8|Sports|203|pribarable|N/A|1820452746khaki12190|spring|Ounce|Unknown|24|ationeseprianti| +5348|AAAAAAAAEOEBAAAA|1997-10-27|2000-10-26|Fairly different lights can strike already vast, british minutes. So labour followers disturb plus a groups. Scottish, su|4.81|3.22|2003001|exportiimporto #1|3|pants|2|Men|404|esebarese|small|pale0092149954938757|peru|Unknown|Unknown|13|eingeseprianti| +5349|AAAAAAAAEOEBAAAA|2000-10-27||Feet could n|8.88|4.44|2003001|importoamalg #2|2|fragrances|1|Women|404|esebarese|N/A|8698182500antique111|snow|Cup|Unknown|41|n steseprianti| +5350|AAAAAAAAGOEBAAAA|1997-10-27|1999-10-27|Needs take blind, dramatic users; huge, american modules set so labour policies. Other rates might hold more realistic, private decisions. Practices see. New markets speak once again very old standard|2.55|1.02|6006007|corpcorp #7|6|rings|6|Jewelry|980|bareingn st|N/A|437405purple43317534|spring|Bundle|Unknown|4|barantiprianti| +5351|AAAAAAAAGOEBAAAA|1999-10-28|2001-10-26|Needs take blind, dramatic users; huge, american modules set so labour policies. Other rates might hold more realistic, private decisions. Practices see. New markets speak once again very old standard|1.74|1.02|6006007|edu packedu pack #2|4|athletic|4|Shoes|11|bareingn st|petite|437405purple43317534|white|Box|Unknown|49|oughtantiprianti| +5352|AAAAAAAAGOEBAAAA|2001-10-27||Crucial, technical coins should care friends. Very other sounds continue. Times break correct, consistent sha|2.26|1.02|6006007|exportischolar #1|4|pop|5|Music|357|ationantipri|N/A|84137251plum02427856|pale|Dozen|Unknown|30|ableantiprianti| +5353|AAAAAAAAJOEBAAAA|1997-10-27||Adverse things would create|5.51|4.95|5003002|exportischolar #2|3|pop|5|Music|758|eingantiation|N/A|9354394tan0462075651|thistle|Dram|Unknown|24|priantiprianti| +5354|AAAAAAAAKOEBAAAA|1997-10-27|2000-10-26|Years earn negative, great reasons; too long feet become during a failures. Also anxious hours used to follow doors. Finely rural needs find violent, brig|1.52|1.04|8004005|edu packnameless #5|4|camping|8|Sports|398|eingn stpri|N/A|snow7365624253548189|powder|N/A|Unknown|34|eseantiprianti| +5355|AAAAAAAAKOEBAAAA|2000-10-27||Silent legs exclude safely. Cli|0.76|1.04|8004005|edu packmaxi #6|14|tennis|8|Sports|594|esen stanti|N/A|snow7365624253548189|hot|Unknown|Unknown|16|antiantiprianti| +5356|AAAAAAAAMOEBAAAA|1997-10-27|1999-10-27|Effortlessly economic wives would keep. Economic costs may not recognise dramatically also general goals. Other savings may explain in a situations|0.18|0.11|1001001|amalgamalg #1|1|dresses|1|Women|186|callyeingought|N/A|222674796forest48418|violet|N/A|Unknown|63|callyantiprianti| +5357|AAAAAAAAMOEBAAAA|1999-10-28|2001-10-26|Hours can seem then terms. Existing, true workers form. Very common depths could not put truly again common classes. Bodies |2.36|0.87|3002002|importoexporti #2|1|infants|3|Children|186|callyeingought|medium|9582181192016navajo1|plum|N/A|Unknown|3|ationantiprianti| +5358|AAAAAAAAMOEBAAAA|2001-10-27||Hours can seem then terms. Existing, true workers form. Very common depths could not put truly again common classes. Bodies |8.27|0.87|7014009|edu packnameless #9|1|glassware|7|Home|186|callyeingought|N/A|6098pale739705488183|peach|Box|Unknown|70|eingantiprianti| +5359|AAAAAAAAPOEBAAAA|1997-10-27||Subjects shall blame personally new sources; big managers sit formally in a fingers. Cars aid very only |0.98|0.47|2001002|amalgimporto #2|1|accessories|2|Men|329|n stablepri|extra large|086powder93789445802|turquoise|Case|Unknown|1|n stantiprianti| +5360|AAAAAAAAAPEBAAAA|1997-10-27|2000-10-26|Words think as the police. Only companies shall speak anyway sure, present pairs. Small days may not beat short-term things. Well constant|3.13|2.28|9015009|scholarunivamalg #9|15|fiction|9|Books|171|oughtationought|N/A|9953236488629spring2|mint|Carton|Unknown|27|barcallyprianti| +5361|AAAAAAAAAPEBAAAA|2000-10-27||Words think as the police. Only companies shall speak anyway sure, present pairs. Small days may not beat short-term things. Well constant|7.00|2.87|9015009|exportischolar #2|15|pop|5|Music|129|oughtationought|N/A|9953236488629spring2|antique|Gram|Unknown|18|oughtcallyprianti| +5362|AAAAAAAACPEBAAAA|1997-10-27|1999-10-27|Questions build firm accounts; revolutionary, cold groups imagine over supreme parties. Bold horses get also inner, clear concessions. Happy, criminal habits like local, tight women. Black facilities|0.66|0.21|10014008|edu packamalgamalg #8|14|automotive|10|Electronics|741|oughteseation|N/A|64142743sky855014921|rose|Gram|Unknown|56|ablecallyprianti| +5363|AAAAAAAACPEBAAAA|1999-10-28|2001-10-26|Questions build firm accounts; revolutionary, cold groups imagine over supreme parties. Bold horses get also inner, clear concessions. Happy, criminal habits like local, tight women. Black facilities|4.85|0.21|10014009|edu packamalgamalg #9|14|automotive|10|Electronics|116|oughteseation|N/A|457860133293red19345|wheat|Each|Unknown|24|pricallyprianti| +5364|AAAAAAAACPEBAAAA|2001-10-27||Sure weapons should realize. Economies might make. Sharp, guilty colleagues see elements. More complex years shall not reve|4.93|0.21|5002001|importoscholar #1|14|country|5|Music|116|callyoughtought|N/A|652blush817367861540|spring|Ounce|Unknown|35|esecallyprianti| +5365|AAAAAAAAFPEBAAAA|1997-10-27||Soci|1.08|0.48|1001002|amalgamalg #2|1|dresses|1|Women|49|n stese|large|981475slate846794780|sky|Lb|Unknown|2|anticallyprianti| +5366|AAAAAAAAGPEBAAAA|1997-10-27|2000-10-26|Proposals might sum rightly complex, social eyes. Over basic sy|4.41|1.58|4001001|amalgedu pack #1|1|womens|4|Shoes|417|ationoughtese|extra large|81145779rose62237459|royal|Ton|Unknown|58|callycallyprianti| +5367|AAAAAAAAGPEBAAAA|2000-10-27||Rates might see in a seconds. Always equal sheets can go moreo|4.41|1.58|10003013|exportiunivamalg #13|3|dvd/vcr players|10|Electronics|417|ationoughtese|N/A|841royal247585971358|thistle|Bunch|Unknown|11|ationcallyprianti| +5368|AAAAAAAAIPEBAAAA|1997-10-27|1999-10-27|Today e|3.24|2.81|4002001|importoedu pack #1|2|mens|4|Shoes|39|n stpri|extra large|0389222207204ghost67|saddle|Carton|Unknown|28|eingcallyprianti| +5369|AAAAAAAAIPEBAAAA|1999-10-28|2001-10-26|Today e|1.56|2.81|4001002|amalgedu pack #2|1|womens|4|Shoes|39|n stpri|large|6midnight13056585872|ivory|Pallet|Unknown|48|n stcallyprianti| +5370|AAAAAAAAIPEBAAAA|2001-10-27||Today e|0.26|0.13|4001002|importonameless #1|1|baseball|8|Sports|39|n stpri|N/A|6midnight13056585872|rosy|Ounce|Unknown|4|barationprianti| +5371|AAAAAAAALPEBAAAA|1997-10-27||Great, active gard|49.90|24.95|3004002|edu packexporti #2|4|school-uniforms|3|Children|122|ableableought|economy|0purple8157735632113|firebrick|Cup|Unknown|52|oughtationprianti| +5372|AAAAAAAAMPEBAAAA|1997-10-27|2000-10-26|Annual, strong managers should renew often vehicles. Sophisticated, forthcoming times note high lar|9.46|3.02|10005001|scholarunivamalg #1|5|karoke|10|Electronics|475|antiationese|N/A|943pale9168885940646|papaya|Tbl|Unknown|24|ableationprianti| +5373|AAAAAAAAMPEBAAAA|2000-10-27||Annual, strong managers should renew often vehicles. Sophisticated, forthcoming times note high lar|1.28|0.87|10005001|namelesscorp #6|5|mens watch|6|Jewelry|475|antiationese|N/A|6984889378olive81309|slate|Gross|Unknown|63|priationprianti| +5374|AAAAAAAAOPEBAAAA|1997-10-27|1999-10-27|Nonetheless united materials talk individuals; inc, effec|5.48|2.74|7002005|importobrand #5|2|bedding|7|Home|517|ationoughtanti|N/A|77goldenrod577331818|papaya|Tsp|Unknown|49|eseationprianti| +5375|AAAAAAAAOPEBAAAA|1999-10-28|2001-10-26|Nonetheless united materials talk individuals; inc, effec|8.80|2.74|7010006|univnameless #6|2|flatware|7|Home|20|barable|N/A|purple33200923064309|tomato|N/A|Unknown|26|antiationprianti| +5376|AAAAAAAAOPEBAAAA|2001-10-27||Nonetheless united materials talk individuals; inc, effec|0.60|0.54|7010006|amalgedu pack #1|1|womens|4|Shoes|328|barable|large|purple33200923064309|seashell|Each|Unknown|62|callyationprianti| +5377|AAAAAAAABAFBAAAA|1997-10-27||Rates allow particularly certain heroes.|4.98|4.18|1002002|importoamalg #2|2|fragrances|1|Women|171|oughtationought|medium|0077579950078green99|pink|Pound|Unknown|72|ationationprianti| +5378|AAAAAAAACAFBAAAA|1997-10-27|2000-10-26|Attractive organizations will turn other, effective patients. Social, christian mo|3.31|2.41|4002001|importoedu pack #1|2|mens|4|Shoes|64|esecally|large|819seashell913074897|salmon|Cup|Unknown|3|eingationprianti| +5379|AAAAAAAACAFBAAAA|2000-10-27||Attractive organizations will turn other, effective patients. Social, christian mo|3.10|2.51|3004002|edu packexporti #2|2|school-uniforms|3|Children|280|esecally|small|27292630yellow164766|cornsilk|Tbl|Unknown|68|n stationprianti| +5380|AAAAAAAAEAFBAAAA|1997-10-27|1999-10-27|Eastern, considerable words keep surely spanish, occasional animals; so roman years export quickly ever likely |4.01|2.04|4002001|importoedu pack #1|2|mens|4|Shoes|707|ationbaration|medium|6733lavender03484102|spring|Oz|Unknown|49|bareingprianti| +5381|AAAAAAAAEAFBAAAA|1999-10-28|2001-10-26|Enough unlikely |8.87|2.04|6004006|edu packcorp #6|4|bracelets|6|Jewelry|181|ationbaration|N/A|8903230frosted695895|metallic|Carton|Unknown|59|oughteingprianti| +5382|AAAAAAAAEAFBAAAA|2001-10-27||There old materials call financial, hard fees. Huge, difficult numbers can comply however about a lives. Nuclear, commercial others may improve various, angry ministers. Then|4.95|4.45|7006003|corpbrand #3|4|rugs|7|Home|674|ationbaration|N/A|52224340883918tan071|peach|Pallet|Unknown|37|ableeingprianti| +5383|AAAAAAAAHAFBAAAA|1997-10-27||Ultimate, normal shareholders shall bu|9.07|3.80|7007004|brandbrand #4|7|decor|7|Home|521|oughtableanti|N/A|71704668448485pale50|turquoise|Unknown|Unknown|11|prieingprianti| +5384|AAAAAAAAIAFBAAAA|1997-10-27|2000-10-26|Easily exciting rights increase under values; here british boundaries display special fears. Tory, adjacent instruments drop. Authorities could |9.91|5.64|6012001|importobrand #1|12|costume|6|Jewelry|109|n stbarought|N/A|315salmon37751057692|pink|Pallet|Unknown|25|eseeingprianti| +5385|AAAAAAAAIAFBAAAA|2000-10-27||New, miserable forces secure social, likely friends. Excellent children begin recently available margins. Jobs know better able film|3.35|1.20|8001006|amalgnameless #6|1|athletic shoes|8|Sports|626|callyablecally|N/A|0sienna7232854475028|yellow|Dram|Unknown|58|antieingprianti| +5386|AAAAAAAAKAFBAAAA|1997-10-27|1999-10-27|Possible, white matters may overcome twice distinct projects. Digital shares will like silent loans. Difficult, other children cannot know goa|0.46|0.40|7008007|namelessbrand #7|8|lighting|7|Home|744|eseeseation|N/A|0483828burlywood9935|seashell|Cup|Unknown|61|callyeingprianti| +5387|AAAAAAAAKAFBAAAA|1999-10-28|2001-10-26|Possible, white matters may overcome twice distinct projects. Digital shares will like silent loans. Difficult, other children cannot know goa|4.69|0.40|7008007|amalgimporto #2|1|accessories|2|Men|744|eseeseation|petite|6cornsilk80660021393|linen|Gross|Unknown|29|ationeingprianti| +5388|AAAAAAAAKAFBAAAA|2001-10-27||Possible, white matters may overcome twice distinct projects. Digital shares will like silent loans. Difficult, other children cannot know goa|4.55|2.27|7008007|edu packcorp #5|4|bracelets|6|Jewelry|741|eseeseation|N/A|71277994928sandy4309|violet|Tsp|Unknown|69|eingeingprianti| +5389|AAAAAAAANAFBAAAA|1997-10-27||Temporary, national figures like straight at a sentences. Re|3.46|1.28|3003002|exportiexporti #2|3|toddlers|3|Children|159|n stantiought|medium|medium74535980696444|maroon|Cup|Unknown|25|n steingprianti| +5390|AAAAAAAAOAFBAAAA|1997-10-27|2000-10-26|Upper windows can hurt high, able corners. Applicants shrink once trying trees. About other hands settle too other eyes. Suddenly major d|0.31|0.17|7002001|importobrand #1|2|bedding|7|Home|7|ation|N/A|0441532827848tomato0|lime|Bundle|Unknown|62|barn stprianti| +5391|AAAAAAAAOAFBAAAA|2000-10-27||Main services mus|7.88|0.17|7002001|amalgexporti #2|2|newborn|3|Children|7|ation|petite|0441532827848tomato0|hot|Dozen|Unknown|19|oughtn stprianti| +5392|AAAAAAAAABFBAAAA|1997-10-27|1999-10-27|Hard dogs should operate then different years; once dependent ideas |3.50|1.89|2004001|edu packimporto #1|4|sports-apparel|2|Men|108|eingbarought|medium|70174956001010sky721|lemon|Unknown|Unknown|27|ablen stprianti| +5393|AAAAAAAAABFBAAAA|1999-10-28|2001-10-26|Cards redress howev|2.80|1.89|2004001|importoscholar #2|4|country|5|Music|108|eingbarought|N/A|70174956001010sky721|peru|Ton|Unknown|12|prin stprianti| +5394|AAAAAAAAABFBAAAA|2001-10-27||Years will set kinds. Also great police visit pregnant|3.24|2.81|2004001|corpbrand #5|4|rugs|7|Home|108|eingbarought|N/A|70174956001010sky721|sky|Dozen|Unknown|24|esen stprianti| +5395|AAAAAAAADBFBAAAA|1997-10-27||Members shall not notice drastically in a standards. Concerned yea|3.22|0.96|7002006|importobrand #6|2|bedding|7|Home|168|eingcallyought|N/A|89453yellow463635463|rosy|Cup|Unknown|59|antin stprianti| +5396|AAAAAAAAEBFBAAAA|1997-10-27|2000-10-26|Reduced, new persons must support journalists. Projects involve actually anonymous, conscious references. |0.77|0.48|7008007|namelessbrand #7|8|lighting|7|Home|125|antiableought|N/A|75111120243smoke7143|navajo|Oz|Unknown|99|callyn stprianti| +5397|AAAAAAAAEBFBAAAA|2000-10-27||Simply old services think famous friends. Discussions should develop most. Rather outstanding signs would hear only about a shops; eyes attend qu|8.78|3.16|7008007|exportiexporti #2|3|toddlers|3|Children|596|antiableought|extra large|75111120243smoke7143|smoke|Bunch|Unknown|22|ationn stprianti| +5398|AAAAAAAAGBFBAAAA|1997-10-27|1999-10-27|Free pp. think rather to the shoulders. Original rates wil|3.71|2.85|8012009|importomaxi #9|12|guns|8|Sports|264|esecallyable|N/A|sky38248395067380995|smoke|Ounce|Unknown|5|eingn stprianti| +5399|AAAAAAAAGBFBAAAA|1999-10-28|2001-10-26|Free pp. think rather to the shoulders. Original rates wil|4.98|3.93|8012010|importomaxi #10|12|guns|8|Sports|217|ationoughtable|N/A|144245025647sky22018|tan|Bundle|Unknown|32|n stn stprianti| +5400|AAAAAAAAGBFBAAAA|2001-10-27||Prepared terms |2.78|3.93|6009001|maxicorp #1|12|womens watch|6|Jewelry|38|eingpri|N/A|144245025647sky22018|linen|Ounce|Unknown|22|barbareseanti| +5401|AAAAAAAAJBFBAAAA|1997-10-27||Too productive points would leave material ministers. Public, objective elections loosen no longer children; political, central movements speak |9.42|6.12|9011002|amalgunivamalg #2|11|cooking|9|Books|42|ableese|N/A|6sandy45315135534416|seashell|Tbl|Unknown|14|oughtbareseanti| +5402|AAAAAAAAKBFBAAAA|1997-10-27|2000-10-26|Already desperate weeks think also little weeks. Original, clear chiefs could not add about meetings. Women could not take effects. Intelligent, social others draw aware, normal thanks. Unl|2.73|2.21|1002001|importoamalg #1|2|fragrances|1|Women|186|callyeingought|extra large|moccasin800365838178|rosy|Each|Unknown|98|ablebareseanti| +5403|AAAAAAAAKBFBAAAA|2000-10-27||Only independent dogs need blues. Here foreign off|3.27|2.21|2001002|amalgimporto #2|2|accessories|2|Men|130|callyeingought|medium|476348265531papaya84|navajo|Pallet|Unknown|35|pribareseanti| +5404|AAAAAAAAMBFBAAAA|1997-10-27|1999-10-27|Beautiful ways try from the years. Famous, simple judges could not remain certainly rather black holidays; fixed, logical laboratories might ask as important problems. Nig|69.90|34.95|2002001|importoimporto #1|2|shirts|2|Men|457|ationantiese|medium|504776019linen701377|rose|Ton|Unknown|23|esebareseanti| +5405|AAAAAAAAMBFBAAAA|1999-10-28|2001-10-26|Women say. Values carry again theories. Original germans assess more for the grounds. Also modern emotions jump hardly details. Comparatively just eyes will discover there al|1.29|0.73|2002001|edu packscholar #2|2|classical|5|Music|457|ationantiese|N/A|323906889peach121421|salmon|Ton|Unknown|38|antibareseanti| +5406|AAAAAAAAMBFBAAAA|2001-10-27||Daily, inner journalists mean formally stages. Northern, other children see. Men see innocently english, secret pounds; defendants play just standar|0.88|0.73|2002001|scholarbrand #7|15|custom|6|Jewelry|293|ationantiese|N/A|323906889peach121421|orchid|Case|Unknown|4|callybareseanti| +5407|AAAAAAAAPBFBAAAA|1997-10-27||Consider|3.14|1.88|1002002|importoamalg #2|2|fragrances|1|Women|201|oughtbarable|petite|8218574568spring5695|puff|Oz|Unknown|22|ationbareseanti| +5408|AAAAAAAAACFBAAAA|1997-10-27|2000-10-26|Possible, wide opinions shall not believe really large dimensions. Less historic awards arrange|4.43|2.79|6013001|exportibrand #1|13|loose stones|6|Jewelry|158|eingantiought|N/A|780red42803321821876|peach|Cup|Unknown|45|eingbareseanti| +5409|AAAAAAAAACFBAAAA|2000-10-27||Already dark structures shall appoint only often irrelevant improvements. Short early scenes see however lone|8.27|3.80|6005002|scholarcorp #2|13|earings|6|Jewelry|158|eingantiought|N/A|266194plum2061530994|smoke|Cup|Unknown|21|n stbareseanti| +5410|AAAAAAAACCFBAAAA|1997-10-27|1999-10-27|Young fields might intervene str|3.57|2.35|4004001|edu packedu pack #1|4|athletic|4|Shoes|251|oughtantiable|large|7076769misty33756918|metallic|Gram|Unknown|36|baroughteseanti| +5411|AAAAAAAACCFBAAAA|1999-10-28|2001-10-26|Young fields might intervene str|3.26|2.02|2001002|amalgimporto #2|1|accessories|2|Men|831|oughtantiable|large|7076769misty33756918|peach|N/A|Unknown|70|oughtoughteseanti| +5412|AAAAAAAACCFBAAAA|2001-10-27||Certain, solid agencies read fresh, crucial articles. African presents used to kick living, sure computers. Possible instructions should emphasise only authorities. So big households used |9.58|7.28|2001002|edu packamalg #1|1|swimwear|1|Women|831|oughtantiable|small|6magenta943580196225|tan|Gram|Unknown|57|ableoughteseanti| +5413|AAAAAAAAFCFBAAAA|1997-10-27||Golden estates meet as yet hands. About solid proteins used to tell. Once causal boots imagine frequently new elections; flexible, other ways find re|9.76|3.70|9001002|amalgmaxi #2|1|arts|9|Books|519|n stoughtanti|N/A|5rose539678792335533|steel|Pallet|Unknown|17|prioughteseanti| +5414|AAAAAAAAGCFBAAAA|1997-10-27|2000-10-26|Pounds come even funds. Certain, minor blacks take politely m|1.44|0.46|3004001|edu packexporti #1|4|school-uniforms|3|Children|197|ationn stought|medium|44592salmon208874735|rosy|Bundle|Unknown|38|eseoughteseanti| +5415|AAAAAAAAGCFBAAAA|2000-10-27||Libraries should not sign forever sometimes international weapons. Technical, important patterns might tell gently towards a measures. Cells must go in a perfo|2.80|0.46|3004001|exporticorp #8|3|gold|6|Jewelry|188|eingeingought|N/A|2606519964saddle1024|royal|Dram|Unknown|69|antioughteseanti| +5416|AAAAAAAAICFBAAAA|1997-10-27|1999-10-27|Improved stars turn quickly benefits. Children bear only normal women. Also curious levels |37.00|33.30|3004001|edu packexporti #1|4|school-uniforms|3|Children|863|pricallyeing|extra large|3306789puff763950485|metallic|Oz|Unknown|2|callyoughteseanti| +5417|AAAAAAAAICFBAAAA|1999-10-28|2001-10-26|Improved stars turn quickly benefits. Children bear only normal women. Also curious levels |8.98|4.57|3004001|edu packscholar #2|4|classical|5|Music|177|ationationought|N/A|3306789puff763950485|magenta|Carton|Unknown|1|ationoughteseanti| +5418|AAAAAAAAICFBAAAA|2001-10-27||Improved stars turn quickly benefits. Children bear only normal women. Also curious levels |1.84|4.57|3004001|corpnameless #9|4|football|8|Sports|177|ationationought|N/A|703128150752tomato95|black|Lb|Unknown|17|eingoughteseanti| +5419|AAAAAAAALCFBAAAA|1997-10-27||English |3.87|3.44|9007008|brandmaxi #8|7|reference|9|Books|766|callycallyation|N/A|42340428979lime72059|powder|Cup|Unknown|10|n stoughteseanti| +5420|AAAAAAAAMCFBAAAA|1997-10-27|2000-10-26|Rural phenomena would not sound more than key, moral rates. Also subsequent aspec|6.50|3.96|4003001|exportiedu pack #1|3|kids|4|Shoes|30|barpri|petite|6285steel59146420834|almond|Bundle|Unknown|8|barableeseanti| +5421|AAAAAAAAMCFBAAAA|2000-10-27||Rural phenomena would not sound more than key, moral rates. Also subsequent aspec|14.96|3.96|4003001|importoimporto #2|3|shirts|2|Men|30|barpri|extra large|6285steel59146420834|hot|Each|Unknown|64|oughtableeseanti| +5422|AAAAAAAAOCFBAAAA|1997-10-27|1999-10-27|Amounts feel as parents. Loud old assumptions can end no longer friendly p|3.64|2.62|9011011|amalgunivamalg #11|11|cooking|9|Books|32|ablepri|N/A|09metallic9396562511|antique|Bunch|Unknown|15|ableableeseanti| +5423|AAAAAAAAOCFBAAAA|1999-10-28|2001-10-26|Amounts feel as parents. Loud old assumptions can end no longer friendly p|4.15|2.62|10005001|scholarunivamalg #1|11|karoke|10|Electronics|386|callyeingpri|N/A|65968626burnished779|lavender|Pound|Unknown|49|priableeseanti| +5424|AAAAAAAAOCFBAAAA|2001-10-27||Characteristics used to like too in favour of the churches. Notes might not occur. Married, final categories ought to see now like, electoral arts. Growing conventions ough|0.20|0.14|4002001|importoedu pack #1|2|mens|4|Shoes|157|callyeingpri|medium|789866983169steel854|royal|Gross|Unknown|58|eseableeseanti| +5425|AAAAAAAABDFBAAAA|1997-10-27||Coming, big children shall give italian, whole figures. Effectively new changes want |0.24|0.15|1002002|importoamalg #2|2|fragrances|1|Women|922|ableablen st|small|rosy2210724642198506|smoke|Pound|Unknown|39|antiableeseanti| +5426|AAAAAAAACDFBAAAA|1997-10-27|2000-10-26|Great deputies take from a managers. Bishops get softly in a houses. References shall form especially silly, very courses. Too new worlds|4.13|2.39|6010003|univbrand #3|10|jewelry boxes|6|Jewelry|202|ablebarable|N/A|870blanched403759429|rose|Cup|Unknown|6|callyableeseanti| +5427|AAAAAAAACDFBAAAA|2000-10-27||Precise, literary teachers live easy official pound|7.70|2.39|4002002|importoedu pack #2|10|mens|4|Shoes|202|ablebarable|petite|870blanched403759429|turquoise|Cup|Unknown|4|ationableeseanti| +5428|AAAAAAAAEDFBAAAA|1997-10-27|1999-10-27|Persons think really there primitive purposes. Standards tell hands; complete, final schools trust quite obviously conservative presents. National, other pa|2.57|2.10|2001001|amalgimporto #1|1|accessories|2|Men|214|eseoughtable|petite|569thistle4994510044|drab|Tsp|Unknown|70|eingableeseanti| +5429|AAAAAAAAEDFBAAAA|1999-10-28|2001-10-26|Persons think really there primitive purposes. Standards tell hands; complete, final schools trust quite obviously conservative presents. National, other pa|7.09|2.10|2001001|edu packimporto #2|1|sports-apparel|2|Men|214|eseoughtable|large|4682peru678595073801|misty|Pound|Unknown|13|n stableeseanti| +5430|AAAAAAAAEDFBAAAA|2001-10-27||Simple lives vote especially international leaders. Female times accept already; fashionable, running powers might not g|9.33|3.63|2001001|amalgamalg #1|1|dresses|1|Women|214|eseoughtable|large|4682peru678595073801|white|Pallet|Unknown|63|barprieseanti| +5431|AAAAAAAAHDFBAAAA|1997-10-27||Available operations should go as walls; optimistic, dangerous others succeed helplessly dyna|2.19|1.05|4001002|amalgedu pack #2|1|womens|4|Shoes|865|anticallyeing|extra large|8seashell50012408102|magenta|Bunch|Unknown|30|oughtprieseanti| +5432|AAAAAAAAIDFBAAAA|1997-10-27|2000-10-26|Too successive affairs ought to know. Obvious women|6.01|3.72|8003003|exportinameless #3|3|basketball|8|Sports|261|oughtcallyable|N/A|58674087952278smoke7|lemon|Tbl|Unknown|57|ableprieseanti| +5433|AAAAAAAAIDFBAAAA|2000-10-27||Front, british occasions could provide anywhere years. Lucky margins may get longer to the agencies; shallow levels should see as available exercises. Inner sites should not need now. Debts boot o|3.42|2.90|8003003|amalgamalg #2|1|dresses|1|Women|261|oughtcallyable|petite|1021157000611snow503|sandy|Dozen|Unknown|51|priprieseanti| +5434|AAAAAAAAKDFBAAAA|1997-10-27|1999-10-27|At once difficult hands will get there at least great men. Odd, important jobs cause still for the years; downstream retail branche|4.11|2.75|2002001|importoimporto #1|2|shirts|2|Men|178|eingationought|medium|0metallic25784849972|peach|Ton|Unknown|31|eseprieseanti| +5435|AAAAAAAAKDFBAAAA|1999-10-28|2001-10-26|Particular parents must recognise swiss, comprehensive gods. Exclusively necessary decisions would not believe just precious projects. Always old councillors shall leave. Overseas, serious rel|9.26|2.75|2002001|importoamalgamalg #13|12|monitors|10|Electronics|53|prianti|N/A|0metallic25784849972|pink|Tsp|Unknown|24|antiprieseanti| +5436|AAAAAAAAKDFBAAAA|2001-10-27||Particular parents must recognise swiss, comprehensive gods. Exclusively necessary decisions would not believe just precious projects. Always old councillors shall leave. Overseas, serious rel|6.33|2.21|2002001|importobrand #7|12|bedding|7|Home|53|prianti|N/A|0metallic25784849972|spring|Unknown|Unknown|7|callyprieseanti| +5437|AAAAAAAANDFBAAAA|1997-10-27||Negotiations could not know true effects. Rich visitors will think inc, foreign lists. Significantly only elements flourish already; companies remember habits. Difficult, occupational |8.37|7.53|8001008|amalgnameless #8|1|athletic shoes|8|Sports|293|prin stable|N/A|4251goldenrod9481316|saddle|Case|Unknown|7|ationprieseanti| +5438|AAAAAAAAODFBAAAA|1997-10-27|2000-10-26|Available savings can find. Western, old fa|3.14|1.06|5003001|exportischolar #1|3|pop|5|Music|270|barationable|N/A|06158158992plum79341|tomato|Carton|Unknown|35|eingprieseanti| +5439|AAAAAAAAODFBAAAA|2000-10-27||Excellent, only issues simplify for example. Fro|1.04|1.06|5003001|corpbrand #8|3|consignment|6|Jewelry|270|barationable|N/A|06158158992plum79341|violet|Ton|Unknown|24|n stprieseanti| +5440|AAAAAAAAAEFBAAAA|1997-10-27|1999-10-27|Available facilities stop free, american branches. British, religious decisions may stop |1.12|0.72|7014001|edu packnameless #1|14|glassware|7|Home|342|ableesepri|N/A|67232861784purple012|purple|Tsp|Unknown|38|bareseeseanti| +5441|AAAAAAAAAEFBAAAA|1999-10-28|2001-10-26|Available facilities stop free, american branches. British, religious decisions may stop |39.57|0.72|7014001|namelessmaxi #6|8|romance|9|Books|207|ableesepri|N/A|031pink6292964352899|royal|Pallet|Unknown|15|oughteseeseanti| +5442|AAAAAAAAAEFBAAAA|2001-10-27||Interesting police ought to react yet americans. Only western prices shall not exercise methods. Forms might ignore for a members. Priorities could benefit |5.38|0.72|10015003|scholaramalgamalg #3|8|portable|10|Electronics|207|ationbarable|N/A|99070046873indian696|violet|Box|Unknown|17|ableeseeseanti| +5443|AAAAAAAADEFBAAAA|1997-10-27||Successful, specific effects shall get the|0.41|0.25|10013004|exportiamalgamalg #4|13|stereo|10|Electronics|156|callyantiought|N/A|4881274198steel11625|linen|Pallet|Unknown|50|prieseeseanti| +5444|AAAAAAAAEEFBAAAA|1997-10-27|2000-10-26|Even early demands must stop spontaneously clear rooms. Bottom, useful males could know all on the teachers; just men will want classes. Large, odd minutes understand in a |2.32|2.08|1003001|exportiamalg #1|3|maternity|1|Women|307|ationbarpri|medium|86432lime34615246475|white|Unknown|Unknown|77|eseeseeseanti| +5445|AAAAAAAAEEFBAAAA|2000-10-27||Even early demands must stop spontaneously clear rooms. Bottom, useful males could know all on the teachers; just men will want classes. Large, odd minutes understand in a |3.07|1.32|3002002|importoexporti #2|3|infants|3|Children|307|ationbarpri|small|86432lime34615246475|pink|Gross|Unknown|14|antieseeseanti| +5446|AAAAAAAAGEFBAAAA|1997-10-27|1999-10-27|Moreover little chemicals take long in a words. Evil months amend worse times. Nevertheless good leaders might stop honestly priv|7.32|2.26|5001001|amalgscholar #1|1|rock|5|Music|337|ationpripri|N/A|7107maroon6027901906|magenta|Pound|Unknown|12|callyeseeseanti| +5447|AAAAAAAAGEFBAAAA|1999-10-28|2001-10-26|Ideas call true, ethi|4.18|2.26|5001001|exportibrand #8|1|kids|7|Home|337|ationpripri|N/A|68070metallic0833263|powder|Each|Unknown|75|ationeseeseanti| +5448|AAAAAAAAGEFBAAAA|2001-10-27||Ideas call true, ethi|1.58|0.64|7009009|maxibrand #9|1|mattresses|7|Home|530|ationpripri|N/A|0973057926moccasin52|midnight|Pallet|Unknown|55|eingeseeseanti| +5449|AAAAAAAAJEFBAAAA|1997-10-27||Times clean plants; academic, ancient speeches get other authorities; subjective needs may not ta|4.35|2.39|4002002|importoedu pack #2|2|mens|4|Shoes|389|n steingpri|medium|492chartreuse9648169|burlywood|Oz|Unknown|56|n steseeseanti| +5450|AAAAAAAAKEFBAAAA|1997-10-27|2000-10-26|Privat|2.29|1.64|1004001|edu packamalg #1|4|swimwear|1|Women|136|callypriought|petite|69687peach3756845625|yellow|Tbl|Unknown|40|barantieseanti| +5451|AAAAAAAAKEFBAAAA|2000-10-27||Privat|5.99|1.64|1002002|importoamalg #2|2|fragrances|1|Women|136|callypriought|large|69687peach3756845625|puff|Tbl|Unknown|28|oughtantieseanti| +5452|AAAAAAAAMEFBAAAA|1997-10-27|1999-10-27|Themes consider. Strings could not rotate above, small houses. Sure, consta|1.89|1.30|5001001|amalgscholar #1|1|rock|5|Music|594|esen stanti|N/A|45878688584dim226645|spring|Bunch|Unknown|28|ableantieseanti| +5453|AAAAAAAAMEFBAAAA|1999-10-28|2001-10-26|Effectively excellent quarters create but late, beautiful games. Good proceedings used to play yet other, full minutes; controls limit especially. Special, wonderf|82.06|45.13|5001001|scholaramalgamalg #14|1|portable|10|Electronics|195|esen stanti|N/A|45878688584dim226645|slate|Ton|Unknown|10|priantieseanti| +5454|AAAAAAAAMEFBAAAA|2001-10-27||Lips pass minds. Clea|1.86|45.13|10008015|namelessunivamalg #15|8|scanners|10|Electronics|195|esen stanti|N/A|45878688584dim226645|sienna|Tsp|Unknown|77|eseantieseanti| +5455|AAAAAAAAPEFBAAAA|1997-10-27||Tears present total duties. Minutes may not m|5.27|3.47|8013006|exportimaxi #6|13|sailing|8|Sports|575|antiationanti|N/A|13salmon970300327055|red|Case|Unknown|14|antiantieseanti| +5456|AAAAAAAAAFFBAAAA|1997-10-27|2000-10-26|English, automatic firms s|5.09|3.61|3001001|amalgexporti #1|1|newborn|3|Children|225|antiableable|economy|991318407117papaya39|pink|Each|Unknown|21|callyantieseanti| +5457|AAAAAAAAAFFBAAAA|2000-10-27||English, automatic firms s|1.93|3.61|8009008|maxinameless #8|9|optics|8|Sports|225|antiableable|N/A|turquoise96029472934|goldenrod|Each|Unknown|2|ationantieseanti| +5458|AAAAAAAACFFBAAAA|1997-10-27|1999-10-27|Main, ltd. flames continue firmly. European spirits used to endure true with a features. Others tell never moral, normal writers. Li|0.77|0.39|9015011|scholarunivamalg #11|15|fiction|9|Books|945|antiesen st|N/A|6626086peach67967610|spring|Gram|Unknown|52|eingantieseanti| +5459|AAAAAAAACFFBAAAA|1999-10-28|2001-10-26|Broad, external societies put. Either parental measurements look very future responses. Above local mountains eat. Houses react elsewhere legal guns. Technologies oug|3.39|0.39|4003002|exportiedu pack #2|15|kids|4|Shoes|571|oughtationanti|N/A|509121red92293715536|turquoise|Box|Unknown|36|n stantieseanti| +5460|AAAAAAAACFFBAAAA|2001-10-27||Broad, external societies put. Either parental measurements look very future responses. Above local mountains eat. Houses react elsewhere legal guns. Technologies oug|0.33|0.20|4003002|exportimaxi #1|13|sailing|8|Sports|824|eseableeing|N/A|509121red92293715536|firebrick|Carton|Unknown|84|barcallyeseanti| +5461|AAAAAAAAFFFBAAAA|1997-10-27||Aware rooms bring there patently good home|0.82|0.73|6009006|maxicorp #6|9|womens watch|6|Jewelry|348|eingesepri|N/A|0563848wheat33463755|chartreuse|Gram|Unknown|91|oughtcallyeseanti| +5462|AAAAAAAAGFFBAAAA|1997-10-27|2000-10-26|Badly black animals must not get creative, national changes. Eastern adults claim at la|3.04|2.70|10010006|univamalgamalg #6|10|memory|10|Electronics|191|oughtn stought|N/A|rose9305336358839166|turquoise|Pallet|Unknown|62|ablecallyeseanti| +5463|AAAAAAAAGFFBAAAA|2000-10-27||Badly black animals must not get creative, national changes. Eastern adults claim at la|8.59|2.70|10010006|corpcorp #8|10|rings|6|Jewelry|191|oughtn stought|N/A|rose9305336358839166|peach|Each|Unknown|18|pricallyeseanti| +5464|AAAAAAAAIFFBAAAA|1997-10-27|1999-10-27|Publishers accept under in a minutes. Terms ensure pounds. |2.80|1.23|8004005|edu packnameless #5|4|camping|8|Sports|172|ableationought|N/A|02tan169945112062877|white|Bundle|Unknown|7|esecallyeseanti| +5465|AAAAAAAAIFFBAAAA|1999-10-28|2001-10-26|Publishers accept under in a minutes. Terms ensure pounds. |2.37|1.80|8004005|corpbrand #6|6|rugs|7|Home|172|ableationought|N/A|02tan169945112062877|moccasin|Bunch|Unknown|18|anticallyeseanti| +5466|AAAAAAAAIFFBAAAA|2001-10-27||Publishers accept under in a minutes. Terms ensure pounds. |1.84|0.95|4001001|amalgedu pack #1|6|womens|4|Shoes|172|ableationought|large|9313824066705violet1|white|Ton|Unknown|20|callycallyeseanti| +5467|AAAAAAAALFFBAAAA|1997-10-27||Remarkable, pos|1.43|1.14|2003002|exportiimporto #2|3|pants|2|Men|20|barable|N/A|8grey686940985907675|white|Bunch|Unknown|18|ationcallyeseanti| +5468|AAAAAAAAMFFBAAAA|1997-10-27|2000-10-26|Relative reactions begin completely today shy proposals. United, good feelings should get nearly |1.82|1.29|7009009|maxibrand #9|9|mattresses|7|Home|306|callybarpri|N/A|2826burnished2919811|peru|Pallet|Unknown|6|eingcallyeseanti| +5469|AAAAAAAAMFFBAAAA|2000-10-27||Past years used to maintain however required, soviet factors. Costs may not divert short families. Areas take impor|1.58|0.94|7009009|amalgimporto #2|1|accessories|2|Men|306|callybarpri|large|2826burnished2919811|tan|Pound|Unknown|43|n stcallyeseanti| +5470|AAAAAAAAOFFBAAAA|1997-10-27|1999-10-27|Even inner rules must suspend traditional years; able, environmental police keep then in the steps; certain, old consequences may limit for a players. Ass|6.25|2.37|1004001|edu packamalg #1|4|swimwear|1|Women|313|prioughtpri|medium|387867894476ghost229|pale|Dram|Unknown|29|barationeseanti| +5471|AAAAAAAAOFFBAAAA|1999-10-28|2001-10-26|Anc|3.94|2.37|1004001|importoamalg #2|2|fragrances|1|Women|313|prioughtpri|medium|387867894476ghost229|seashell|Ounce|Unknown|74|oughtationeseanti| +5472|AAAAAAAAOFFBAAAA|2001-10-27||Current gods might avoid white sheets. Basic islands will not carry proper efforts. Good, a|5.77|2.53|1004001|scholarunivamalg #16|5|karoke|10|Electronics|313|prioughtpri|N/A|7spring3700071783585|magenta|Gram|Unknown|31|ableationeseanti| +5473|AAAAAAAABGFBAAAA|1997-10-27||Too scientific letters could not depend more; instead national attitudes read less magnificent politici|4.01|2.36|8010004|univmaxi #4|10|pools|8|Sports|648|eingesecally|N/A|930885807rose3825768|spring|Gram|Unknown|27|priationeseanti| +5474|AAAAAAAACGFBAAAA|1997-10-27|2000-10-26|Requirements can understand much national, reduced |2.86|2.40|6008003|namelesscorp #3|8|mens watch|6|Jewelry|656|callyantically|N/A|689756342929sienna20|spring|Gross|Unknown|14|eseationeseanti| +5475|AAAAAAAACGFBAAAA|2000-10-27||British skills b|9.66|4.73|6008003|importonameless #6|8|paint|7|Home|656|callyantically|N/A|619305sky09182850266|smoke|N/A|Unknown|76|antiationeseanti| +5476|AAAAAAAAEGFBAAAA|1997-10-27|1999-10-27|Social terms can go so there whole beds. More symp|3.55|1.59|1004001|edu packamalg #1|4|swimwear|1|Women|1|ought|N/A|2108sandy05175754278|pale|Ton|Unknown|79|callyationeseanti| +5477|AAAAAAAAEGFBAAAA|1999-10-28|2001-10-26|Records may not say to a fingers. Insta|86.28|36.23|1004001|amalgnameless #8|4|athletic shoes|8|Sports|1|ought|N/A|36433wheat2658483193|salmon|Bundle|Unknown|55|ationationeseanti| +5478|AAAAAAAAEGFBAAAA|2001-10-27||Clear gentlemen will not work already appropriately american experiences. Messages carry parents; ostensibly young officers preserve lite|4.51|36.23|1004001|amalgamalg #1|1|dresses|1|Women|1|ought|small|36433wheat2658483193|orchid|Gram|Unknown|37|eingationeseanti| +5479|AAAAAAAAHGFBAAAA|1997-10-27||Magnetic eyes hold exotic, high points. Now particular losses give again other, previous services. Social, natural co|3.23|2.03|5004002|edu packscholar #2|4|classical|5|Music|862|ablecallyeing|N/A|2287225wheat42842242|salmon|Case|Unknown|38|n stationeseanti| +5480|AAAAAAAAIGFBAAAA|1997-10-27|2000-10-26|Actually ethnic authorities used to qualify just |3.61|2.70|5004001|edu packscholar #1|4|classical|5|Music|39|n stpri|N/A|1753410832red0265785|deep|Pound|Unknown|21|bareingeseanti| +5481|AAAAAAAAIGFBAAAA|2000-10-27||Actually ethnic authorities used to qualify just |1.53|1.07|1004002|edu packamalg #2|4|swimwear|1|Women|39|n stpri|medium|rose5753089086016852|pale|Unknown|Unknown|12|oughteingeseanti| +5482|AAAAAAAAKGFBAAAA|1997-10-27|1999-10-27|Boundaries will take almost familiar loans. Below public services shall keep early schools. Issues sti|7.45|4.54|7010003|univnameless #3|10|flatware|7|Home|205|antibarable|N/A|49656533964wheat3145|slate|Bunch|Unknown|32|ableeingeseanti| +5483|AAAAAAAAKGFBAAAA|1999-10-28|2001-10-26|Much rough symptoms pass new, funny sites. Rules can prove remarkably proc|0.46|4.54|7010003|brandcorp #4|10|pendants|6|Jewelry|205|antibarable|N/A|lace2205219291000622|seashell|Oz|Unknown|8|prieingeseanti| +5484|AAAAAAAAKGFBAAAA|2001-10-27||Interests should not allow no longer hours; as reasonable levels must clear then firms. Therefore international things will not find aside short, joint techno|2.45|4.54|7010003|importobrand #5|10|bedding|7|Home|190|antibarable|N/A|lace2205219291000622|tan|Unknown|Unknown|50|eseeingeseanti| +5485|AAAAAAAANGFBAAAA|1997-10-27||Small, surprising girls speak with a thousands. Important agreements approach statistical figures|1.92|1.61|5003002|exportischolar #2|3|pop|5|Music|395|antin stpri|N/A|482turquoise73674923|navy|Pallet|Unknown|60|antieingeseanti| +5486|AAAAAAAAOGFBAAAA|1997-10-27|2000-10-26|Perhaps young problems shoot well powerful schools. Possibilities risk parliamentary, local guidelines. Mild things refuse only still secret patterns. Great, aware women |3.76|2.48|7006007|corpbrand #7|6|rugs|7|Home|925|antiablen st|N/A|52pale37226751916225|slate|Dozen|Unknown|17|callyeingeseanti| +5487|AAAAAAAAOGFBAAAA|2000-10-27||Extended others lay pro|4.93|2.31|7006007|amalgamalg #2|6|dresses|1|Women|925|antiablen st|petite|2241810lawn261775620|puff|Each|Unknown|42|ationeingeseanti| +5488|AAAAAAAAAHFBAAAA|1997-10-27|1999-10-27|Dead, great states let together practitioners. New liabilities migrate very social things. Little, tired foods might not spin also pregnant services; officers deal. Home adverse languages cou|2.87|2.32|7006009|corpbrand #9|6|rugs|7|Home|325|antiablepri|N/A|1pink828259575400062|gainsboro|Oz|Unknown|56|eingeingeseanti| +5489|AAAAAAAAAHFBAAAA|1999-10-28|2001-10-26|Enough recent communists mind. Similar hands should join central, various books. Alone police grant good, subject words. T|4.81|2.32|1002002|importoamalg #2|6|fragrances|1|Women|325|antiablepri|petite|1pink828259575400062|papaya|Bunch|Unknown|25|n steingeseanti| +5490|AAAAAAAAAHFBAAAA|2001-10-27||A little other children will lack. Ends see. Payable, common things might not know only helpful, familiar times. Never old |3.31|1.52|1002002|amalgimporto #1|1|accessories|2|Men|325|antiablepri|large|misty783004650319463|indian|Ounce|Unknown|6|barn steseanti| +5491|AAAAAAAADHFBAAAA|1997-10-27||Accused, black forms would not obtain eventually for a groups. |5.68|2.55|7004002|edu packbrand #2|4|curtains/drapes|7|Home|377|ationationpri|N/A|3678311301yellow6237|seashell|Pound|Unknown|42|oughtn steseanti| +5492|AAAAAAAAEHFBAAAA|1997-10-27|2000-10-26|Strong, social authors speak fully still lucky results. Colonial groups used to satisfy ever open stages; words begin also about a patients. Chronic, noble allegations used to insist |7.24|3.62|7008003|namelessbrand #3|8|lighting|7|Home|154|eseantiought|N/A|273973honeydew779420|olive|Case|Unknown|16|ablen steseanti| +5493|AAAAAAAAEHFBAAAA|2000-10-27||Other clubs will feel high guests. Appropriate things could not follow forward easy, supreme schools; immensely open terms must not see since with a expectations. Settin|6.05|4.84|7008003|exportibrand #4|8|kids|7|Home|154|eseantiought|N/A|273973honeydew779420|yellow|Each|Unknown|75|prin steseanti| +5494|AAAAAAAAGHFBAAAA|1997-10-27|1999-10-27|Days appear hard on a years. Here electrical brothers ignore again difficulties. Positions could fall across darling allowanc|1.60|0.86|1004001|edu packamalg #1|4|swimwear|1|Women|505|antibaranti|petite|787634578deep2815331|powder|Bundle|Unknown|57|esen steseanti| +5495|AAAAAAAAGHFBAAAA|1999-10-28|2001-10-26|Capital words ought to come rightly years. Main, nice feet co|5.22|0.86|1004001|edu packmaxi #12|4|entertainments|9|Books|505|antibaranti|N/A|787634578deep2815331|violet|Gram|Unknown|46|antin steseanti| +5496|AAAAAAAAGHFBAAAA|2001-10-27||In order able countries seem more than on a chemicals; addition|2.92|1.57|1004001|edu packamalgamalg #6|4|automotive|10|Electronics|238|eingpriable|N/A|1394violet5198734052|wheat|Carton|Unknown|56|callyn steseanti| +5497|AAAAAAAAJHFBAAAA|1997-10-27||Margins must admit possible, main |3.86|2.39|3003002|exportiexporti #2|3|toddlers|3|Children|162|ablecallyought|petite|74117066tomato958029|lawn|Case|Unknown|15|ationn steseanti| +5498|AAAAAAAAKHFBAAAA|1997-10-27|2000-10-26|Tenants cope against the guns. Ever particular fears explain numerous players. Agencies give early economic securities. National probl|3.78|3.06|9014003|edu packunivamalg #3|14|sports|9|Books|294|esen stable|N/A|65616royal6187297540|magenta|Unknown|Unknown|95|eingn steseanti| +5499|AAAAAAAAKHFBAAAA|2000-10-27||Original teachers will not think instead various children. Strong, numero|2.34|1.33|9014003|scholarunivamalg #9|5|karoke|10|Electronics|294|esen stable|N/A|65616royal6187297540|goldenrod|Tsp|Unknown|1|n stn steseanti| +5500|AAAAAAAAMHFBAAAA|1997-10-27|1999-10-27|Determined g|4.55|2.82|4003001|exportiedu pack #1|3|kids|4|Shoes|480|bareingese|small|1115453840tomato4967|papaya|Each|Unknown|13|barbarantianti| +5501|AAAAAAAAMHFBAAAA|1999-10-28|2001-10-26|Determined g|8.25|3.21|4003001|edu packunivamalg #6|14|sports|9|Books|278|bareingese|N/A|1115453840tomato4967|tan|Bunch|Unknown|10|oughtbarantianti| +5502|AAAAAAAAMHFBAAAA|2001-10-27||Determined g|3.67|1.98|4003001|amalgexporti #1|14|newborn|3|Children|278|eingationable|large|4474snow050445017241|tan|Carton|Unknown|20|ablebarantianti| +5503|AAAAAAAAPHFBAAAA|1997-10-27||Later military shoulders secure increasingly clear poems; suitable values cannot forget even boxes; available, new conditions happen yet with a developments. Already |16.29|12.21|1002002|importoamalg #2|2|fragrances|1|Women|844|eseeseeing|N/A|0956pink528854345659|sienna|Oz|Unknown|33|pribarantianti| +5504|AAAAAAAAAIFBAAAA|1997-10-27|2000-10-26|As available agents must not form there square, useful earnings. At once busy fingers construct anyway. Severe, conservative parents should not serve alternative, specialist subjects. Mus|8.61|3.27|6010001|univbrand #1|10|jewelry boxes|6|Jewelry|507|ationbaranti|N/A|096946955139sandy136|sandy|Cup|Unknown|3|esebarantianti| +5505|AAAAAAAAAIFBAAAA|2000-10-27||As available agents must not form there square, useful earnings. At once busy fingers construct anyway. Severe, conservative parents should not serve alternative, specialist subjects. Mus|1.25|1.01|6010001|exportiamalg #2|10|maternity|1|Women|507|ationbaranti|extra large|6127peach39524312201|salmon|Lb|Unknown|21|antibarantianti| +5506|AAAAAAAACIFBAAAA|1997-10-27|1999-10-27|Free proced|5.97|4.41|8009007|maxinameless #7|9|optics|8|Sports|146|callyeseought|N/A|00472203snow14727419|plum|Pound|Unknown|8|callybarantianti| +5507|AAAAAAAACIFBAAAA|1999-10-28|2001-10-26|Free proced|0.30|4.41|4002002|importoedu pack #2|2|mens|4|Shoes|146|callyeseought|small|00472203snow14727419|salmon|N/A|Unknown|91|ationbarantianti| +5508|AAAAAAAACIFBAAAA|2001-10-27||Private parents will understand likely, particular items. Often interesting policies ought to predict physical expenses;|6.62|4.41|4002002|importoexporti #1|2|infants|3|Children|146|callyeseought|small|433285yellow08504683|red|Oz|Unknown|30|eingbarantianti| +5509|AAAAAAAAFIFBAAAA|1997-10-27||Therefore remarkable results want later unlikely, great form|7.39|3.54|5001002|amalgscholar #2|1|rock|5|Music|127|ationableought|N/A|6coral10746841986436|red|Pound|Unknown|13|n stbarantianti| +5510|AAAAAAAAGIFBAAAA|1997-10-27|2000-10-26|Slow years would not win|0.60|0.48|2001001|amalgimporto #1|1|accessories|2|Men|363|pricallypri|large|84688lawn69871793219|moccasin|Unknown|Unknown|93|baroughtantianti| +5511|AAAAAAAAGIFBAAAA|2000-10-27||All glad keys like as hours. Dual, sound properties should sign at a russians. Sudden, available clients thrive sometimes relations. Partly friendly farmers could not wa|8.22|0.48|2001001|edu packimporto #2|4|sports-apparel|2|Men|616|callyoughtcally|small|84688lawn69871793219|pink|Gross|Unknown|8|oughtoughtantianti| +5512|AAAAAAAAIIFBAAAA|1997-10-27|1999-10-27|Equal areas show. Police admit below overseas, educational levels. Trees leave circumstances. Technological organisations would go by the margins. Available police would not appea|6.91|4.90|7010003|univnameless #3|10|flatware|7|Home|164|esecallyought|N/A|82494820085674royal7|papaya|Oz|Unknown|31|ableoughtantianti| +5513|AAAAAAAAIIFBAAAA|1999-10-28|2001-10-26|Equal areas show. Police admit below overseas, educational levels. Trees leave circumstances. Technological organisations would go by the margins. Available police would not appea|4.38|4.90|7010003|edu packexporti #2|4|school-uniforms|3|Children|308|esecallyought|economy|3950orange7817984009|tan|Gram|Unknown|55|prioughtantianti| +5514|AAAAAAAAIIFBAAAA|2001-10-27||Most future characters keep holidays; different, similar engines learn certainly late due trusts. Bad changes acquire now. British faces must propose slowly|4.79|1.91|7010003|amalgexporti #1|4|newborn|3|Children|308|eingbarpri|medium|3950orange7817984009|burlywood|Cup|Unknown|5|eseoughtantianti| +5515|AAAAAAAALIFBAAAA|1997-10-27||Still tough unions must refuse especially services. Authorities play only. Main, nati|6.81|5.17|8006006|corpnameless #6|6|football|8|Sports|316|callyoughtpri|N/A|937129997146830pink1|plum|Oz|Unknown|55|antioughtantianti| +5516|AAAAAAAAMIFBAAAA|1997-10-27|2000-10-26|Calls used to eradicate here national, old knees. Able, english opinions afford concepts. Vital, commercial cigar|6.82|2.38|8011007|amalgmaxi #7|11|archery|8|Sports|211|oughtoughtable|N/A|6196313242rosy778156|saddle|N/A|Unknown|20|callyoughtantianti| +5517|AAAAAAAAMIFBAAAA|2000-10-27||Calls used to eradicate here national, old knees. Able, english opinions afford concepts. Vital, commercial cigar|5.38|3.76|8011007|exportiamalg #2|3|maternity|1|Women|57|oughtoughtable|large|9162362red3429113541|tomato|Unknown|Unknown|54|ationoughtantianti| +5518|AAAAAAAAOIFBAAAA|1997-10-27|1999-10-27|Members shall not help increa|3.55|3.08|8005009|scholarnameless #9|5|fitness|8|Sports|215|antioughtable|N/A|6goldenrod3237673459|salmon|Box|Unknown|16|eingoughtantianti| +5519|AAAAAAAAOIFBAAAA|1999-10-28|2001-10-26|Members shall not help increa|0.59|0.38|8005009|edu packedu pack #2|4|athletic|4|Shoes|125|antioughtable|medium|6goldenrod3237673459|antique|Ounce|Unknown|2|n stoughtantianti| +5520|AAAAAAAAOIFBAAAA|2001-10-27||More new units eliminate only. Other, interesting indians target. Already chemical shelves must act just prisoner|0.16|0.38|3003001|exportiexporti #1|4|toddlers|3|Children|455|antiantiese|N/A|6goldenrod3237673459|plum|Bundle|Unknown|1|barableantianti| +5521|AAAAAAAABJFBAAAA|1997-10-27||Ltd. years hold dreadful men. Really immediate years shall learn under a shares. Just domestic patients could climb easy great elderly results. Sales will hand really more poor b|7.22|5.70|3002002|importoexporti #2|2|infants|3|Children|8|eing|medium|8powder0756290096043|pale|Gram|Unknown|10|oughtableantianti| +5522|AAAAAAAACJFBAAAA|1997-10-27|2000-10-26|In general female areas may not understand different, italian words; here total practices like briefly monetary, domesti|0.87|0.38|5003001|exportischolar #1|3|pop|5|Music|345|antiesepri|N/A|108689844violet29310|goldenrod|Carton|Unknown|14|ableableantianti| +5523|AAAAAAAACJFBAAAA|2000-10-27||Perfectly additional children leave usually. Dead fires must not live. Employers use highly in the children. Brothers used to perceive seldom members. Public, final yea|8.29|0.38|6010004|univbrand #4|10|jewelry boxes|6|Jewelry|345|antiesepri|N/A|425metallic386417636|spring|Pallet|Unknown|30|priableantianti| +5524|AAAAAAAAEJFBAAAA|1997-10-27|1999-10-27|Old, complex police shall contain simultaneously great clouds. Formal cases give public sections.|16.25|11.70|3001001|amalgexporti #1|1|newborn|3|Children|266|callycallyable|petite|72635680saddle777294|hot|Bunch|Unknown|10|eseableantianti| +5525|AAAAAAAAEJFBAAAA|1999-10-28|2001-10-26|Companies may not identify preci|2.77|11.70|3001001|exportischolar #2|1|pop|5|Music|266|callycallyable|N/A|72635680saddle777294|puff|Oz|Unknown|21|antiableantianti| +5526|AAAAAAAAEJFBAAAA|2001-10-27||Companies may not identify preci|1.94|11.70|8015007|scholarmaxi #7|15|fishing|8|Sports|266|callycallyable|N/A|8frosted382977423853|violet|N/A|Unknown|19|callyableantianti| +5527|AAAAAAAAHJFBAAAA|1997-10-27||Boldly successful results must serve at least most medical fields. Als|7.89|3.62|2003002|exportiimporto #2|3|pants|2|Men|958|eingantin st|small|0225turquoise6867659|sienna|Bunch|Unknown|19|ationableantianti| +5528|AAAAAAAAIJFBAAAA|1997-10-27|2000-10-26|Once more parliamentary sizes separate fairly executive movements. Positive years would get there open units; left governments used to show new police. |2.74|0.87|7001009|amalgbrand #9|1|bathroom|7|Home|155|antiantiought|N/A|52626919536066khaki4|sky|Ounce|Unknown|16|eingableantianti| +5529|AAAAAAAAIJFBAAAA|2000-10-27||Internal prizes believe nearby; relations |6.73|0.87|7001009|exportinameless #10|1|basketball|8|Sports|326|callyablepri|N/A|52626919536066khaki4|salmon|Tsp|Unknown|11|n stableantianti| +5530|AAAAAAAAKJFBAAAA|1997-10-27|1999-10-27|Thin p|8.58|2.65|5004001|edu packscholar #1|4|classical|5|Music|349|n stesepri|N/A|6355790278plum067369|magenta|Bundle|Unknown|1|barpriantianti| +5531|AAAAAAAAKJFBAAAA|1999-10-28|2001-10-26|Thin p|74.30|46.06|5004001|importoimporto #2|4|shirts|2|Men|943|priesen st|economy|50gainsboro441241154|rosy|Ounce|Unknown|55|oughtpriantianti| +5532|AAAAAAAAKJFBAAAA|2001-10-27||Main ships see mor|50.66|46.06|5004001|importoimporto #1|4|shirts|2|Men|943|priesen st|small|29441887985sandy4267|olive|Dozen|Unknown|80|ablepriantianti| +5533|AAAAAAAANJFBAAAA|1997-10-27||Also small months could leave; emotional subs|3.18|2.16|2001002|amalgimporto #2|1|accessories|2|Men|40|barese|small|236560seashell797791|tomato|Unknown|Unknown|21|pripriantianti| +5534|AAAAAAAAOJFBAAAA|1997-10-27|2000-10-26|Accessible, ang|2.00|0.88|10008010|namelessunivamalg #10|8|scanners|10|Electronics|479|n stationese|N/A|940slate309532756739|spring|Cup|Unknown|18|esepriantianti| +5535|AAAAAAAAOJFBAAAA|2000-10-27||Original, french gains bring enough. Main principles dispel especially generally total limits; uncomfortably late sides must expect tomorrow for a cuts; now early rates ought to ring n|33.18|0.88|2003002|exportiimporto #2|8|pants|2|Men|479|n stationese|extra large|2764650tan0467578043|smoke|Pallet|Unknown|61|antipriantianti| +5536|AAAAAAAAAKFBAAAA|1997-10-27|1999-10-27|Positive, similar cases might hold together single statements. Stupid dates could not believe very small lines. Quick, nuclear questi|1.13|0.75|5001001|amalgscholar #1|1|rock|5|Music|289|n steingable|N/A|7smoke62882365302881|metallic|Ton|Unknown|53|callypriantianti| +5537|AAAAAAAAAKFBAAAA|1999-10-28|2001-10-26|Hands lower less under a years. Nuclear plans surprise particularly high, other letters. Bird|4.36|2.22|5001001|univamalgamalg #13|10|memory|10|Electronics|289|n steingable|N/A|3482009steel67645615|pale|Dozen|Unknown|49|ationpriantianti| +5538|AAAAAAAAAKFBAAAA|2001-10-27||More social implications believe perhaps enough positive girls. Available, formal feet s|0.41|0.27|1004001|edu packamalg #1|10|swimwear|1|Women|289|n steingable|large|6953light22859417082|medium|Unknown|Unknown|46|eingpriantianti| +5539|AAAAAAAADKFBAAAA|1997-10-27||Far, true regions might agree standard prices. Sources tell perhaps much furious forms. Separate years cry competent, severe matches. Dangers should like slightly able women; t|4.96|3.42|3004002|edu packexporti #2|4|school-uniforms|3|Children|568|eingcallyanti|large|41107rose73702102524|plum|Lb|Unknown|16|n stpriantianti| +5540|AAAAAAAAEKFBAAAA|1997-10-27|2000-10-26|Other others provide simple descriptions. |76.52|34.43|9008009|namelessmaxi #9|8|romance|9|Books|548|eingeseanti|N/A|71789lawn82218952920|yellow|Ounce|Unknown|25|bareseantianti| +5541|AAAAAAAAEKFBAAAA|2000-10-27||Other others provide simple descriptions. |3.54|34.43|9008009|exportiedu pack #2|8|kids|4|Shoes|548|eingeseanti|petite|71789lawn82218952920|burnished|Tsp|Unknown|55|oughteseantianti| +5542|AAAAAAAAGKFBAAAA|1997-10-27|1999-10-27|Children would not |1.93|0.65|3001001|amalgexporti #1|1|newborn|3|Children|342|ableesepri|large|slate981088142692829|navy|Bundle|Unknown|50|ableeseantianti| +5543|AAAAAAAAGKFBAAAA|1999-10-28|2001-10-26|Children would not |3.31|0.65|3002002|importoexporti #2|1|infants|3|Children|132|ableesepri|small|40665tan335915288701|green|Gram|Unknown|89|prieseantianti| +5544|AAAAAAAAGKFBAAAA|2001-10-27||Usually new words fly days; |2.84|1.42|2003001|exportiimporto #1|3|pants|2|Men|112|ableoughtought|medium|40665tan335915288701|sky|Gram|Unknown|53|eseeseantianti| +5545|AAAAAAAAJKFBAAAA|1997-10-27||Events stop a little. Northern, white walls welcome at all businesses. Governors must see from a |1.96|1.15|9012002|importounivamalg #2|12|home repair|9|Books|431|oughtpriese|N/A|609213yellow32185858|papaya|Ton|Unknown|24|antieseantianti| +5546|AAAAAAAAKKFBAAAA|1997-10-27|2000-10-26|Dry forces tell. More fat arguments relate here at random physical entries. Social, sure windows w|0.84|0.50|6014003|edu packbrand #3|14|estate|6|Jewelry|892|ablen steing|N/A|20582ivory3877924893|tan|Carton|Unknown|13|callyeseantianti| +5547|AAAAAAAAKKFBAAAA|2000-10-27||Dry forces tell. More fat arguments relate here at random physical entries. Social, sure windows w|2.12|0.50|9006004|corpmaxi #4|14|parenting|9|Books|215|antioughtable|N/A|20582ivory3877924893|yellow|Ounce|Unknown|17|ationeseantianti| +5548|AAAAAAAAMKFBAAAA|1997-10-27|1999-10-27|Well common members keep medical, ancient muscles; goods reclaim only diffi|0.36|0.27|5003001|exportischolar #1|3|pop|5|Music|702|ablebaration|N/A|6242583446powder1320|brown|Pound|Unknown|66|eingeseantianti| +5549|AAAAAAAAMKFBAAAA|1999-10-28|2001-10-26|Potential amounts study greatly alone damages. Material, following expenses look since a tiles. Double ways should cope great, medical roads. |57.70|25.38|8001010|amalgnameless #10|3|athletic shoes|8|Sports|16|callyought|N/A|6242583446powder1320|papaya|Each|Unknown|27|n steseantianti| +5550|AAAAAAAAMKFBAAAA|2001-10-27||Shy, small stocks write other co|2.72|1.71|9012007|importounivamalg #7|3|home repair|9|Books|63|prically|N/A|1087896440292salmon9|rosy|Unknown|Unknown|50|barantiantianti| +5551|AAAAAAAAPKFBAAAA|1997-10-27||Able calls will see far stores; national eyes shall stand among a owners. Long, heavy patients prevent occasionally practical, level sections. Certainly specified regulations could |2.08|1.68|8006002|corpnameless #2|6|football|8|Sports|168|eingcallyought|N/A|1428994red2771826066|salmon|Tbl|Unknown|40|oughtantiantianti| +5552|AAAAAAAAALFBAAAA|1997-10-27|2000-10-26|Beaches make |0.44|0.39|9006009|corpmaxi #9|6|parenting|9|Books|257|ationantiable|N/A|11993453017powder456|maroon|Case|Unknown|69|ableantiantianti| +5553|AAAAAAAAALFBAAAA|2000-10-27||Beaches make |7.95|3.10|9006009|edu packcorp #2|6|bracelets|6|Jewelry|257|ationantiable|N/A|200196376030016lace5|lawn|Ounce|Unknown|2|priantiantianti| +5554|AAAAAAAACLFBAAAA|1997-10-27|1999-10-27|Buildings could not keep even changes. Increasing poems ought to make independent, healthy teams. Long, equal experiments expect of course dirty yards; continued, standard eyes must not r|1.96|0.90|4001001|amalgedu pack #1|1|womens|4|Shoes|584|eseeinganti|medium|23tomato241099216051|pink|Gross|Unknown|44|eseantiantianti| +5555|AAAAAAAACLFBAAAA|1999-10-28|2001-10-26|Unable difficulties find possibly foreign responsibilit|89.42|0.90|4001001|importonameless #6|12|paint|7|Home|839|eseeinganti|N/A|23tomato241099216051|tan|Tbl|Unknown|27|antiantiantianti| +5556|AAAAAAAACLFBAAAA|2001-10-27||Constitutional, voluntary provisions move over there for a years. Comprehensive, liberal banks see only physical, annual decad|6.87|0.90|4001001|edu packnameless #7|14|glassware|7|Home|275|eseeinganti|N/A|9666papaya7485796085|spring|N/A|Unknown|55|callyantiantianti| +5557|AAAAAAAAFLFBAAAA|1997-10-27||Conditions ride others. Everywhere other years should not point like a bonds. Best lexical others used to run small days. Written tears start. Whole, sexu|4.42|2.87|1001002|amalgamalg #2|1|dresses|1|Women|640|baresecally|medium|orchid00474603274786|maroon|Cup|Unknown|65|ationantiantianti| +5558|AAAAAAAAGLFBAAAA|1997-10-27|2000-10-26|Claims choose rarely too armed differences. Personal, wise goods build ill n|1.06|0.64|7001009|amalgbrand #9|1|bathroom|7|Home|360|barcallypri|N/A|5796032885tomato6587|snow|Dozen|Unknown|55|eingantiantianti| +5559|AAAAAAAAGLFBAAAA|2000-10-27||Reasonable, single countries might know. Bodies could pay young, important reason|4.80|0.64|7001009|exportiamalgamalg #1|1|stereo|10|Electronics|360|barcallypri|N/A|5796032885tomato6587|turquoise|Ounce|Unknown|28|n stantiantianti| +5560|AAAAAAAAILFBAAAA|1997-10-27|1999-10-27|Therefore complete problems manage from a regions. Single actions ring bare departments. Together deep services may judge almost to the days; crucial things p|4.19|1.50|2004001|edu packimporto #1|4|sports-apparel|2|Men|214|eseoughtable|medium|90701810pink90339024|thistle|Dram|Unknown|10|barcallyantianti| +5561|AAAAAAAAILFBAAAA|1999-10-28|2001-10-26|Therefore complete problems manage from a regions. Single actions ring bare departments. Together deep services may judge almost to the days; crucial things p|0.91|0.38|5003002|exportischolar #2|4|pop|5|Music|177|ationationought|N/A|90701810pink90339024|lime|Cup|Unknown|28|oughtcallyantianti| +5562|AAAAAAAAILFBAAAA|2001-10-27||Therefore complete problems manage from a regions. Single actions ring bare departments. Together deep services may judge almost to the days; crucial things p|46.30|14.81|5003002|exportinameless #3|4|wallpaper|7|Home|177|ationationought|N/A|90701810pink90339024|smoke|Pallet|Unknown|30|ablecallyantianti| +5563|AAAAAAAALLFBAAAA|1997-10-27||Issues used to incorporate in order with a substances. Already simple generations can result in a adv|0.68|0.23|1001002|amalgamalg #2|1|dresses|1|Women|717|ationoughtation|large|2555navajo2539302215|wheat|Oz|Unknown|48|pricallyantianti| +5564|AAAAAAAAMLFBAAAA||2000-10-26|||4.93|||||6|Jewelry|502|ablebaranti|N/A||tomato||||| +5565|AAAAAAAAMLFBAAAA|2000-10-27||O|7.93|5.70|6013005|namelesscorp #6|13|mens watch|6|Jewelry|502|ablebaranti|N/A|2burlywood2666609234|hot|Pallet|Unknown|4|anticallyantianti| +5566|AAAAAAAAOLFBAAAA|1997-10-27|1999-10-27|More effective pounds govern later distinctly solar hands; relationships encourage about full sessions. Perso|6.90|5.93|2003001|exportiimporto #1|3|pants|2|Men|143|prieseought|medium|yellow92939674830318|powder|Pallet|Unknown|48|callycallyantianti| +5567|AAAAAAAAOLFBAAAA|1999-10-28|2001-10-26|Politi|61.61|5.93|2003001|edu packamalgamalg #9|3|automotive|10|Electronics|143|prieseought|N/A|1775937moccasin33573|slate|Gross|Unknown|95|ationcallyantianti| +5568|AAAAAAAAOLFBAAAA|2001-10-27||Environmental, junior eyes meet about at an advantages. Old circles display more. Affairs can service much rural gardens. Currently constant changes ought to locate. Then possible rig|6.77|2.16|3003001|exportiexporti #1|3|toddlers|3|Children|143|prieseought|medium|056160295navy8187922|snow|Bundle|Unknown|30|eingcallyantianti| +5569|AAAAAAAABMFBAAAA|1997-10-27||High, local centres ought to kill of course falsely english districts. Stages may not work just si|1.71|0.63|10006011|corpunivamalg #11|6|musical|10|Electronics|407|ationbarese|N/A|734349570steel417338|salmon|Gram|Unknown|46|n stcallyantianti| +5570|AAAAAAAACMFBAAAA|1997-10-27|2000-10-26|Museums addre|5.20|4.36|8014001|edu packmaxi #1|14|tennis|8|Sports|301|oughtbarpri|N/A|sienna07857427036382|yellow|N/A|Unknown|21|barationantianti| +5571|AAAAAAAACMFBAAAA|2000-10-27||Separate children ought to answer in a leaves. Parameters shall not give against a responses; free, typical programmes investigate however possible votes. Only, new doors t|1.22|0.80|7001002|amalgbrand #2|14|bathroom|7|Home|292|oughtbarpri|N/A|sienna07857427036382|pale|Unknown|Unknown|19|oughtationantianti| +5572|AAAAAAAAEMFBAAAA|1997-10-27|1999-10-27|Demanding, final patients attack inc|2.23|1.71|3001001|amalgexporti #1|1|newborn|3|Children|324|eseablepri|large|63145698tomato128768|sienna|N/A|Unknown|9|ableationantianti| +5573|AAAAAAAAEMFBAAAA|1999-10-28|2001-10-26|Demanding, final patients attack inc|8.23|1.71|2001002|amalgimporto #2|1|accessories|2|Men|67|ationcally|small|63145698tomato128768|wheat|Case|Unknown|29|priationantianti| +5574|AAAAAAAAEMFBAAAA|2001-10-27||Authorities should not appear even. Years respond rarely. Friendly problems can secure dramatic, f|3.52|1.71|2001002|importonameless #5|1|baseball|8|Sports|579|ationcally|N/A|63145698tomato128768|spring|Lb|Unknown|74|eseationantianti| +5575|AAAAAAAAHMFBAAAA|1997-10-27||Running, intense things improve sure members. Permanent, certain leaders seal decisions.|1.73|0.95|8001006|amalgnameless #6|1|athletic shoes|8|Sports|805|antibareing|N/A|219lace7175307136751|cornsilk|Case|Unknown|65|antiationantianti| +5576|AAAAAAAAIMFBAAAA|1997-10-27|2000-10-26|Below poor troops tell. Effective, single tourists attend usually beforehand difficult machines. Women examine workers. Particular, serious muscles should remedy ever; matters can act from the impr|4.58|1.37|5004001|edu packscholar #1|4|classical|5|Music|14|eseought|N/A|960plum9276294096943|olive|Tbl|Unknown|11|callyationantianti| +5577|AAAAAAAAIMFBAAAA|2000-10-27||Below poor troops tell. Effective, single tourists attend usually beforehand difficult machines. Women examine workers. Particular, serious muscles should remedy ever; matters can act from the impr|9.84|7.67|5004001|maxibrand #8|4|mattresses|7|Home|320|eseought|N/A|47922764808medium151|thistle|Dozen|Unknown|85|ationationantianti| +5578|AAAAAAAAKMFBAAAA|1997-10-27|1999-10-27|Arms m|7.41|5.92|10012003|importoamalgamalg #3|12|monitors|10|Electronics|449|n steseese|N/A|934297729454dim22454|plum|Cup|Unknown|4|eingationantianti| +5579|AAAAAAAAKMFBAAAA|1999-10-28|2001-10-26|Arms m|9.50|5.92|10001004|amalgunivamalg #4|1|cameras|10|Electronics|116|callyoughtought|N/A|305moccasin049677993|powder|Tbl|Unknown|56|n stationantianti| +5580|AAAAAAAAKMFBAAAA|2001-10-27||Simple, wrong counties should not believe very duties. Almost conventional uses suit wild electoral, early islands. Direct associations may appreciate elsewhere classes. Right|9.69|5.92|10001004|amalgscholar #1|1|rock|5|Music|116|callyoughtought|N/A|790942860308powder54|pink|Each|Unknown|24|bareingantianti| +5581|AAAAAAAANMFBAAAA|1997-10-27||Constraints would achieve new authorities. Miserable|10.87|7.50|5004002|edu packscholar #2|4|classical|5|Music|273|priationable|N/A|605rosy4973245154284|wheat|Each|Unknown|66|oughteingantianti| +5582|AAAAAAAAOMFBAAAA|1997-10-27|2000-10-26|Major, typical exercises used to share at least. At last priva|3.42|1.74|2004001|edu packimporto #1|4|sports-apparel|2|Men|639|n stprically|large|332ghost913942025947|rosy|Box|Unknown|21|ableeingantianti| +5583|AAAAAAAAOMFBAAAA|2000-10-27||Major, typical exercises used to share at least. At last priva|66.07|1.74|2004001|corpunivamalg #8|4|musical|10|Electronics|639|n stprically|N/A|332ghost913942025947|sienna|Each|Unknown|64|prieingantianti| +5584|AAAAAAAAANFBAAAA|1997-10-27|1999-10-27|Enough central jobs offer much similar, large rules. Full, safe minutes would not d|1.63|1.14|3004001|edu packexporti #1|4|school-uniforms|3|Children|46|callyese|extra large|1785white33669279897|lawn|Dram|Unknown|78|eseeingantianti| +5585|AAAAAAAAANFBAAAA|1999-10-28|2001-10-26|Enough central jobs offer much similar, large rules. Full, safe minutes would not d|9.11|1.14|3004001|importounivamalg #10|2|camcorders|10|Electronics|46|callyese|N/A|1785white33669279897|slate|Oz|Unknown|90|antieingantianti| +5586|AAAAAAAAANFBAAAA|2001-10-27||Enough central jobs offer much similar, large rules. Full, safe minutes would not d|8.85|5.57|3004001|amalgimporto #1|1|accessories|2|Men|589|n steinganti|small|1785white33669279897|lime|Tsp|Unknown|6|callyeingantianti| +5587|AAAAAAAADNFBAAAA|1997-10-27||Fixed men could seek sometimes diff|8.22|5.91|4004002|edu packedu pack #2|4|athletic|4|Shoes|525|antiableanti|large|868steel990247489336|antique|Unknown|Unknown|20|ationeingantianti| +5588|AAAAAAAAENFBAAAA|1997-10-27|2000-10-26|British, general thousands cannot look so. Years contradict elsewhere again local paths. Upward tropical resources see connections. Colours can require home. Questions exist individual|0.99|0.74|10011013|amalgamalgamalg #13|11|disk drives|10|Electronics|123|priableought|N/A|34192923cream9142623|tomato|Lb|Unknown|54|eingeingantianti| +5589|AAAAAAAAENFBAAAA|2000-10-27||Remarkable, narrow standards should recognize well sure, physical years. Wages must not embrace only af|7.33|6.15|10011013|importounivamalg #10|11|home repair|9|Books|123|priableought|N/A|7white50528246169443|smoke|Unknown|Unknown|15|n steingantianti| +5590|AAAAAAAAGNFBAAAA|1997-10-27|1999-10-27|Other arms take years. Valuable ef|1.14|0.59|2001001|amalgimporto #1|1|accessories|2|Men|568|eingcallyanti|small|9124930115slate03787|rose|Dram|Unknown|30|barn stantianti| +5591|AAAAAAAAGNFBAAAA|1999-10-28|2001-10-26|Cases see officers. Monthly, confident expectations keep. Too possible regulations see however important producers. Forces bear possible, weekly others|8.52|0.59|2001001|importoimporto #2|1|shirts|2|Men|568|eingcallyanti|medium|80283powder894448493|rose|Lb|Unknown|1|oughtn stantianti| +5592|AAAAAAAAGNFBAAAA|2001-10-27||Cases see officers. Monthly, confident expectations keep. Too possible regulations see however important producers. Forces bear possible, weekly others|0.88|0.55|2001001|exportinameless #3|3|basketball|8|Sports|46|eingcallyanti|N/A|80283powder894448493|khaki|Dozen|Unknown|16|ablen stantianti| +5593|AAAAAAAAJNFBAAAA|1997-10-27||Instead certain attempts would fit even medical natural rates. Aware, critical newspapers say wit|71.58|60.84|9006002|corpmaxi #2|6|parenting|9|Books|903|pribarn st|N/A|1537thistle313443172|peach|Gram|Unknown|30|prin stantianti| +5594|AAAAAAAAKNFBAAAA|1997-10-27|2000-10-26|Publi|1.56|0.53|9015003|scholarunivamalg #3|15|fiction|9|Books|470|barationese|N/A|098royal708352087697|powder|Gross|Unknown|24|esen stantianti| +5595|AAAAAAAAKNFBAAAA|2000-10-27||Publi|3.54|2.79|9015003|exportiedu pack #2|15|kids|4|Shoes|172|barationese|petite|098royal708352087697|smoke|Gross|Unknown|5|antin stantianti| +5596|AAAAAAAAMNFBAAAA|1997-10-27|1999-10-27|Level workers may not wait almost; sons come quite long circumstances; olympic streets alter for example u|9.40|7.23|6003005|exporticorp #5|3|gold|6|Jewelry|146|callyeseought|N/A|saddle91446845563460|puff|Each|Unknown|74|callyn stantianti| +5597|AAAAAAAAMNFBAAAA|1999-10-28|2001-10-26|Level workers may not wait almost; sons come quite long circumstances; olympic streets alter for example u|1.28|1.12|6015006|scholarbrand #6|3|custom|6|Jewelry|598|eingn stanti|N/A|377238141royal798949|wheat|Carton|Unknown|56|ationn stantianti| +5598|AAAAAAAAMNFBAAAA|||Level workers may not wait almost; sons come quite long circumstances; olympic streets alter for example u||66.85||edu packedu pack #1|4|||||eingn stanti|small|||||77|| +5599|AAAAAAAAPNFBAAAA|1997-10-27||Available arms will compromise however neighbours. Stable mothers might sing carefully. Little, permanent others result apart from a values. Other, numerous rights would walk privi|3.61|1.37|3001002|amalgexporti #2|1|newborn|3|Children|186|callyeingought|medium|2magenta494761922426|antique|Carton|Unknown|12|n stn stantianti| +5600|AAAAAAAAAOFBAAAA|1997-10-27|2000-10-26|Also independent documents can answer approximately. Negotiations drop never. Similar, likely panels take parents. Ordinary, financial requirements could not match short, international p|3.95|1.77|9005009|scholarmaxi #9|5|history|9|Books|379|n stationpri|N/A|7278medium2747080973|wheat|Unknown|Unknown|41|barbarcallyanti| +5601|AAAAAAAAAOFBAAAA|2000-10-27||Long services risk mostly for a lists. Doctors benefit no doubt odds. Main records may not help soon. Geographical, available findings used to jump most polish ce|37.08|25.58|9005009|importoedu pack #2|2|mens|4|Shoes|379|n stationpri|medium|338254spring42797614|papaya|Unknown|Unknown|72|oughtbarcallyanti| +5602|AAAAAAAACOFBAAAA|1997-10-27|1999-10-27|British, corporate years used to land all poor sequences. Lights ought to get wide real, everyday performances. Ears know essentially. C|5.45|2.88|9004011|edu packmaxi #11|4|entertainments|9|Books|256|callyantiable|N/A|5177675steel97460605|turquoise|Bundle|Unknown|89|ablebarcallyanti| +5603|AAAAAAAACOFBAAAA|1999-10-28|2001-10-26|Scientific demonstrations reach even; futures can know just valid courts. Forward crucial prices can help peculiar, common taxes. Common cells might trust by now work|4.73|2.88|9004011|importomaxi #12|2|business|9|Books|256|callyantiable|N/A|8296293violet5503513|sky|Carton|Unknown|5|pribarcallyanti| +5604|AAAAAAAACOFBAAAA|2001-10-27||Scientific demonstrations reach even; futures can know just valid courts. Forward crucial prices can help peculiar, common taxes. Common cells might trust by now work|57.22|2.88|10015012|scholaramalgamalg #12|15|portable|10|Electronics|185|antieingought|N/A|622hot39056962004834|sandy|Cup|Unknown|95|esebarcallyanti| +5605|AAAAAAAAFOFBAAAA|1997-10-27||Very rare achievements could not say like the systems; rapid cells may not see conferences. R|0.41|0.28|7013006|exportinameless #6|13|wallpaper|7|Home|546|callyeseanti|N/A|05203025spring373398|turquoise|Pallet|Unknown|70|antibarcallyanti| +5606|AAAAAAAAGOFBAAAA|1997-10-27|2000-10-26|Industrial dollars communicate margins. Negative, financial others might make also |2.00|0.80|2002001|importoimporto #1|2|shirts|2|Men|794|esen station|extra large|71645003348azure2643|seashell|Gram|Unknown|1|callybarcallyanti| +5607|AAAAAAAAGOFBAAAA|2000-10-27||Industrial dollars communicate margins. Negative, financial others might make also |6.23|1.86|2002001|exportischolar #2|2|pop|5|Music|794|esen station|N/A|77midnight1626038754|magenta|Tbl|Unknown|33|ationbarcallyanti| +5608|AAAAAAAAIOFBAAAA|1997-10-27|1999-10-27|Different area|9.18|5.59|6011001|amalgbrand #1|11|semi-precious|6|Jewelry|740|bareseation|N/A|43540rose41189334520|chiffon|Ton|Unknown|22|eingbarcallyanti| +5609|AAAAAAAAIOFBAAAA|1999-10-28|2001-10-26|Social, british interests feel. Aloud only conditions can miss bodies. Local, open numbers break further. Victorian claims would not greet then briti|0.47|0.19|3002002|importoexporti #2|2|infants|3|Children|469|n stcallyese|medium|429908misty071318407|plum|Gross|Unknown|10|n stbarcallyanti| +5610|AAAAAAAAIOFBAAAA|2001-10-27||Though available things shall bother equally rough, good files; local plans used to rely houses. Other, huge reports may n|9.76|4.39|3002002|amalgedu pack #1|2|womens|4|Shoes|46|callyese|N/A|9847200snow158133638|rosy|Carton|Unknown|12|baroughtcallyanti| +5611|AAAAAAAALOFBAAAA|1997-10-27||Black others should provide very in a systems. Overall whole animals will not learn secret, different agencies. Techniques used to borrow pu|4.81|3.46|7013002|exportinameless #2|13|wallpaper|7|Home|414|eseoughtese|N/A|1866201256spring4853|wheat|Bunch|Unknown|83|oughtoughtcallyanti| +5612|AAAAAAAAMOFBAAAA|1997-10-27|2000-10-26|Different, small fields receive far years. Unemployed mechanisms jump virtually active others. |3.91|1.48|3002001|importoexporti #1|2|infants|3|Children|640|baresecally|extra large|7831691053smoke29520|linen|Lb|Unknown|23|ableoughtcallyanti| +5613|AAAAAAAAMOFBAAAA|2000-10-27||Different, small fields receive far years. Unemployed mechanisms jump virtually active others. |6.25|1.48|3002001|corpmaxi #4|16|golf|8|Sports|640|baresecally|N/A|84332lime00991714280|sandy|Tbl|Unknown|15|prioughtcallyanti| +5614|AAAAAAAAOOFBAAAA|1997-10-27|1999-10-27|Left, crucial countries should create regardless. Doubts convert just|4.60|1.93|6005007|scholarcorp #7|5|earings|6|Jewelry|276|callyationable|N/A|7sienna0779657729785|navajo|Dozen|Unknown|70|eseoughtcallyanti| +5615|AAAAAAAAOOFBAAAA|1999-10-28|2001-10-26|Left, crucial countries should create regardless. Doubts convert just|1.41|0.66|1003002|exportiamalg #2|3|maternity|1|Women|276|callyationable|medium|7sienna0779657729785|rose|Tsp|Unknown|20|antioughtcallyanti| +5616|AAAAAAAAOOFBAAAA|2001-10-27||Left, crucial countries should create regardless. Doubts convert just|2.23|0.66|4003001|exportiedu pack #1|3|kids|4|Shoes|552|callyationable|petite|7sienna0779657729785|ivory|Case|Unknown|93|callyoughtcallyanti| +5617|AAAAAAAABPFBAAAA|1997-10-27||New, novel practices may guard ever with a years. Camps could not |59.09|40.18|10003008|exportiunivamalg #8|3|dvd/vcr players|10|Electronics|436|callypriese|N/A|743spring96163948322|seashell|Each|Unknown|9|ationoughtcallyanti| +5618|AAAAAAAACPFBAAAA|1997-10-27|2000-10-26|Movements could not result still historians. Net systems give tomorrow different lists. E|7.49|2.77|5003001|exportischolar #1|3|pop|5|Music|496|callyn stese|N/A|5138713smoke40838469|rose|Tbl|Unknown|11|eingoughtcallyanti| +5619|AAAAAAAACPFBAAAA|2000-10-27||Movements could not result still historians. Net systems give tomorrow different lists. E|6.54|4.77|5003001|importoedu pack #2|2|mens|4|Shoes|88|callyn stese|extra large|1973moccasin09857912|navy|Pallet|Unknown|17|n stoughtcallyanti| +5620|AAAAAAAAEPFBAAAA|1997-10-27|1999-10-27|Seats ought to consult tools. Far strong hundreds include children. Concessions sho|8.96|6.45|8012001|importomaxi #1|12|guns|8|Sports|116|callyoughtought|N/A|208659smoke756075584|sandy|Unknown|Unknown|65|barablecallyanti| +5621|AAAAAAAAEPFBAAAA|1999-10-28|2001-10-26|Spiritual, cold jobs contain recent windows. Active, annual children create other miles. Suffi|5.85|2.98|8012001|edu packedu pack #2|4|athletic|4|Shoes|116|callyoughtought|large|208659smoke756075584|powder|Tsp|Unknown|31|oughtablecallyanti| +5622|AAAAAAAAEPFBAAAA|2001-10-27||Spiritual, cold jobs contain recent windows. Active, annual children create other miles. Suffi|3.15|2.98|8012003|importomaxi #3|12|guns|8|Sports|116|callyoughtought|N/A|79395979593820snow44|ghost|Pallet|Unknown|57|ableablecallyanti| +5623|AAAAAAAAHPFBAAAA|1997-10-27||Things may use quickly at a changes; local, late powers shall think much; plans expect there academic results. Reservations tell there stars. Hours apply at|3.58|1.64|9007008|brandmaxi #8|7|reference|9|Books|122|ableableought|N/A|63714448755384olive8|plum|Dozen|Unknown|19|priablecallyanti| +5624|AAAAAAAAIPFBAAAA|1997-10-27|2000-10-26|Yesterday splendid authorities refuse at once late moments. Available lips could result old vehicles. Issues shall see due cases. Other, standard equations would go simultaneously effects; democratic |1.31|0.52|9010009|univunivamalg #9|10|travel|9|Books|897|ationn steing|N/A|672467001royal229747|royal|Lb|Unknown|69|eseablecallyanti| +5625|AAAAAAAAIPFBAAAA|2000-10-27||Different, social trees should not act perhaps then good years. Great injuries feel fresh masses. Potenti|9.52|0.52|7012006|importonameless #6|12|paint|7|Home|897|ationn steing|N/A|672467001royal229747|salmon|Gram|Unknown|4|antiablecallyanti| +5626|AAAAAAAAKPFBAAAA|1997-10-27|1999-10-27|Young, outstanding services will not apply compulsory, dirty decisions. Presidential, spanish cities might like well top, red miles. More soci|9.94|6.06|2004001|edu packimporto #1|4|sports-apparel|2|Men|80|bareing|medium|70001048orange982053|peach|Carton|Unknown|16|callyablecallyanti| +5627|AAAAAAAAKPFBAAAA|1999-10-28|2001-10-26|Young, outstanding services will not apply compulsory, dirty decisions. Presidential, spanish cities might like well top, red miles. More soci|3.52|6.06|6010004|univbrand #4|10|jewelry boxes|6|Jewelry|898|bareing|N/A|70001048orange982053|peru|Unknown|Unknown|6|ationablecallyanti| +5628|AAAAAAAAKPFBAAAA|2001-10-27||Broadly happy things will bring also truly other systems. Often french definitions hesitate actually silver systems. Committees speak sometimes bloody, easy names. Friends could avoid. Pr|1.49|1.10|6010004|amalgedu pack #1|10|womens|4|Shoes|898|eingn steing|large|1puff225348254060043|moccasin|Tbl|Unknown|27|eingablecallyanti| +5629|AAAAAAAANPFBAAAA|1997-10-27||Good, vulnerable worlds could take recently actually estimated agents. Unusual ideas work else sentences. More wide fortunes may embrace even black difficult tasks. Deep,|6.59|3.42|7016010|corpnameless #10|16|furniture|7|Home|263|pricallyable|N/A|0410papaya6943028952|peru|Lb|Unknown|33|n stablecallyanti| +5630|AAAAAAAAOPFBAAAA|1997-10-27|2000-10-26|Authors explain therefore with a|8.58|7.20|3003001|exportiexporti #1|3|toddlers|3|Children|26|callyable|small|246618706625papaya63|olive|Oz|Unknown|10|barpricallyanti| +5631|AAAAAAAAOPFBAAAA|2000-10-27||Authors explain therefore with a|5.99|4.61|4004002|edu packedu pack #2|4|athletic|4|Shoes|95|callyable|petite|246618706625papaya63|medium|Each|Unknown|11|oughtpricallyanti| +5632|AAAAAAAAAAGBAAAA|1997-10-27|1999-10-27|Biological moments mean cold suggestions. True stages give better long-term, busy areas. Ties ask now. Bad figures kiss. Hard, legal sales act only signals. Lives may not pretend. Leading, posi|1.56|0.48|7011003|amalgnameless #3|11|accent|7|Home|643|priesecally|N/A|13940928pink47616357|salmon|Gram|Unknown|38|ablepricallyanti| +5633|AAAAAAAAAAGBAAAA|1999-10-28|2001-10-26|Full, public nations finance howev|6.05|0.48|8010004|univmaxi #4|10|pools|8|Sports|369|priesecally|N/A|307chiffon1567273370|tomato|Bundle|Unknown|18|pripricallyanti| +5634|AAAAAAAAAAGBAAAA|2001-10-27||Warm weapons may claim brown, intense countries. Recent|0.88|0.48|4002001|importoedu pack #1|10|mens|4|Shoes|279|priesecally|large|307chiffon1567273370|navy|N/A|Unknown|100|esepricallyanti| +5635|AAAAAAAADAGBAAAA|1997-10-27||Powerful architects may require gentle patients. Rather following mountains seem. Secret, southern goods hesitate concerns. Very substantial girls name still concerning a children. Subsequen|2.65|1.53|10013009|exportiamalgamalg #9|13|stereo|10|Electronics|417|ationoughtese|N/A|01625655sienna267420|snow|Case|Unknown|21|antipricallyanti| +5636|AAAAAAAAEAGBAAAA|1997-10-27|2000-10-26|Carefully irish numbers may not get therefore prime, available services; implications would improve bitterly else direct companies. Impressive, alone ships respond consistent|4.03|2.78|2004001|edu packimporto #1|4|sports-apparel|2|Men|489|n steingese|medium|52821801360061rose19|sandy|Tsp|Unknown|23|callypricallyanti| +5637|AAAAAAAAEAGBAAAA|2000-10-27||Carefully irish numbers may not get therefore prime, available services; implications would improve bitterly else direct companies. Impressive, alone ships respond consistent|3.87|2.55|1001002|amalgamalg #2|1|dresses|1|Women|489|n steingese|small|684944984682powder69|thistle|Dozen|Unknown|54|ationpricallyanti| +5638|AAAAAAAAGAGBAAAA|1997-10-27|1999-10-27|Successful, successful authorities should respond that is only scottish jobs. Normal, costly jobs detect throu|0.70|0.44|2001001|amalgimporto #1|1|accessories|2|Men|878|eingationeing|petite|7seashell57878497206|steel|Carton|Unknown|13|eingpricallyanti| +5639|AAAAAAAAGAGBAAAA|1999-10-28|2001-10-26|In addition parliamentary factors impose away main |6.84|4.78|9010012|univunivamalg #12|1|travel|9|Books|152|eingationeing|N/A|7seashell57878497206|khaki|Box|Unknown|11|n stpricallyanti| +5640|AAAAAAAAGAGBAAAA|2001-10-27||In addition parliamentary factors impose away main |0.54|0.41|9010012|amalgamalgamalg #14|1|disk drives|10|Electronics|152|eingationeing|N/A|1orange0309681372558|violet|Oz|Unknown|6|baresecallyanti| +5641|AAAAAAAAJAGBAAAA|1997-10-27||Long times put quite. Others appoint memories. Particular, weekly feet could not become select, normal criteria. Dang|3.06|0.97|1002002|importoamalg #2|2|fragrances|1|Women|329|n stablepri|petite|1301saddle8967780234|green|Dram|Unknown|11|oughtesecallyanti| +5642|AAAAAAAAKAGBAAAA|1997-10-27|2000-10-26|Dear feet ought to wear never home other stages. Systems get other, high flowers; late, protective sales want of course horses. Divisions shake as more certain problems. Recently other years ac|4.69|1.40|4002001|importoedu pack #1|2|mens|4|Shoes|95|antin st|economy|900480320tomato15681|turquoise|Lb|Unknown|42|ableesecallyanti| +5643|AAAAAAAAKAGBAAAA|2000-10-27||Dear feet ought to wear never home other stages. Systems get other, high flowers; late, protective sales want of course horses. Divisions shake as more certain problems. Recently other years ac|0.75|0.53|6008004|namelesscorp #4|2|mens watch|6|Jewelry|95|antin st|N/A|900480320tomato15681|tomato|Ounce|Unknown|96|priesecallyanti| +5644|AAAAAAAAMAGBAAAA|1997-10-27|1999-10-27|Full observations might not undertake high. Councils should not bear years. Complex circumstances mean for long statistical, empty years|8.29|7.04|7014005|edu packnameless #5|14|glassware|7|Home|580|bareinganti|N/A|deep8878704326997560|papaya|Carton|Unknown|56|eseesecallyanti| +5645|AAAAAAAAMAGBAAAA|1999-10-28|2001-10-26|Other pictures take still in the conclusions; well competitive e|69.33|7.04|7014005|importobrand #6|14|bedding|7|Home|3|pri|N/A|30260180588499pink55|yellow|Case|Unknown|32|antiesecallyanti| +5646|AAAAAAAAMAGBAAAA|2001-10-27||Other pictures take still in the conclusions; well competitive e|2.09|1.83|7014005|scholarunivamalg #3|5|karoke|10|Electronics|3|pri|N/A|30260180588499pink55|wheat|Bunch|Unknown|45|callyesecallyanti| +5647|AAAAAAAAPAGBAAAA|1997-10-27||Legislative women could accept oddly gates; low, initial laws may continue there examples. Little, close organisations would not join alw|2.25|1.59|7005008|scholarbrand #8|5|blinds/shades|7|Home|382|ableeingpri|N/A|5229459493slate94967|violet|Pallet|Unknown|16|ationesecallyanti| +5648|AAAAAAAAABGBAAAA|1997-10-27|2000-10-26|True, preliminary s|8.97|4.30|3004001|edu packexporti #1|4|school-uniforms|3|Children|970|barationn st|large|7peach76224612160680|pale|Dozen|Unknown|47|eingesecallyanti| +5649|AAAAAAAAABGBAAAA|2000-10-27||Wide prime workers ought to prepare only creative physical appointments. Interested, formal years keep there only available problems. Ligh|3.41|4.30|3004001|exportiexporti #2|4|toddlers|3|Children|606|barationn st|extra large|965ghost354328272475|royal|Pallet|Unknown|41|n stesecallyanti| +5650|AAAAAAAACBGBAAAA|1997-10-27|1999-10-27|Just comm|5.33|2.18|4001001|amalgedu pack #1|1|womens|4|Shoes|73|priation|small|0222902301rosy833160|white|Each|Unknown|26|baranticallyanti| +5651|AAAAAAAACBGBAAAA|1999-10-28|2001-10-26|Services shall live by no means classic subject|4.61|2.18|9003012|exportimaxi #12|1|computers|9|Books|109|n stbarought|N/A|0222902301rosy833160|turquoise|Carton|Unknown|11|oughtanticallyanti| +5652|AAAAAAAACBGBAAAA|2001-10-27||Secure colonies could not note very, chief rights; there bitter forces might find badly other large f|6.18|2.18|9003012|scholarnameless #3|15|tables|7|Home|155|antiantiought|N/A|0222902301rosy833160|sandy|Pound|Unknown|19|ableanticallyanti| +5653|AAAAAAAAFBGBAAAA|1997-10-27||Wars must prosper good models. Beautiful players deal never cold weaknesses. Specifically daily skills get less great producers; clothes must wish even difficult shares. On|6.78|2.91|10007010|brandunivamalg #10|7|personal|10|Electronics|151|oughtantiought|N/A|1540pale850487926084|slate|Unknown|Unknown|45|prianticallyanti| +5654|AAAAAAAAGBGBAAAA|1997-10-27|2000-10-26|Somehow revolutionary sh|7.10|3.05|9011003|amalgunivamalg #3|11|cooking|9|Books|117|ationoughtought|N/A|3193757pale556828511|green|Oz|Unknown|13|eseanticallyanti| +5655|AAAAAAAAGBGBAAAA|2000-10-27||Somehow revolutionary sh|9.67|6.28|3001002|amalgexporti #2|1|newborn|3|Children|117|ationoughtought|medium|5navajo2261541457365|yellow|Carton|Unknown|35|antianticallyanti| +5656|AAAAAAAAIBGBAAAA|1997-10-27|1999-10-27|Particular departments draw never most stupid shoulders. Lonely areas see again high, british units; sure, english seats might round arguments. Running, interesting weeks ought to handle |95.36|76.28|8010007|univmaxi #7|10|pools|8|Sports|40|barese|N/A|992305chocolate36322|white|Lb|Unknown|50|callyanticallyanti| +5657|AAAAAAAAIBGBAAAA|1999-10-28|2001-10-26|Particular departments draw never most stupid shoulders. Lonely areas see again high, british units; sure, english seats might round arguments. Running, interesting weeks ought to handle |57.77|51.41|8010007|exportiedu pack #2|3|kids|4|Shoes|316|callyoughtpri|small|992305chocolate36322|grey|Case|Unknown|88|ationanticallyanti| +5658|AAAAAAAAIBGBAAAA|2001-10-27||Particular departments draw never most stupid shoulders. Lonely areas see again high, british units; sure, english seats might round arguments. Running, interesting weeks ought to handle |1.81|1.21|4002001|importoedu pack #1|2|mens|4|Shoes|316|callyoughtpri|large|992305chocolate36322|plum|Case|Unknown|78|einganticallyanti| +5659|AAAAAAAALBGBAAAA|1997-10-27||Remaining votes distinguish ahead minimal ser|90.32|74.96|1001002|amalgamalg #2|1|dresses|1|Women|432|ablepriese|medium|8073yellow9080536937|dark|Bundle|Unknown|20|n stanticallyanti| +5660|AAAAAAAAMBGBAAAA|1997-10-27|2000-10-26|Significant, dynamic hands could point major, urban hands. However large others could throw following, previous years. In order clear view|4.49|2.78|6013005|exportibrand #5|13|loose stones|6|Jewelry|513|prioughtanti|N/A|71532611028yellow429|tomato|Dozen|Unknown|9|barcallycallyanti| +5661|AAAAAAAAMBGBAAAA|2000-10-27||Cultural women used to include well over here dull opportunities. Thin, other stars see so for a par|2.59|1.03|6013005|importounivamalg #10|12|home repair|9|Books|513|prioughtanti|N/A|pale7102472066632305|sky|Carton|Unknown|9|oughtcallycallyanti| +5662|AAAAAAAAOBGBAAAA|1997-10-27|1999-10-27|Additional, related fruits want previous, protective details. Sexu|0.09|0.06|6003007|exporticorp #7|3|gold|6|Jewelry|148|eingeseought|N/A|871465rosy5845894867|plum|Carton|Unknown|85|ablecallycallyanti| +5663|AAAAAAAAOBGBAAAA|1999-10-28|2001-10-26|Additional, related fruits want previous, protective details. Sexu|4.50|0.06|6003007|exportischolar #2|3|pop|5|Music|148|eingeseought|N/A|871465rosy5845894867|blush|Lb|Unknown|65|pricallycallyanti| +5664|AAAAAAAAOBGBAAAA|2001-10-27||Very expert months should wave available, joint areas; better distant duties live|2.57|2.03|6003007|importoamalg #1|2|fragrances|1|Women|148|eingeseought|extra large|9935985turquoise1946|red|Cup|Unknown|52|esecallycallyanti| +5665|AAAAAAAABCGBAAAA|1997-10-27||Frankly tory miles might make extremely new properties; either big pictures must not return therefore in a cities. Perhaps effective assessments emerge parliamentary opponents. Probably external purpo|7.68|5.91|8013006|exportimaxi #6|13|sailing|8|Sports|195|antin stought|N/A|1sienna2040860465045|salmon|Tsp|Unknown|10|anticallycallyanti| +5666|AAAAAAAACCGBAAAA|1997-10-27|2000-10-26|Absent, various difficulties go excellent, hungry grounds. Then theoretical |3.76|2.55|6007003|brandcorp #3|7|pendants|6|Jewelry|256|callyantiable|N/A|60smoke0100162119879|puff|Tbl|Unknown|98|callycallycallyanti| +5667|AAAAAAAACCGBAAAA|2000-10-27||Absent, various difficulties go excellent, hungry grounds. Then theoretical |1.08|0.35|7008008|namelessbrand #8|7|lighting|7|Home|256|callyantiable|N/A|256974350indian63592|turquoise|Gram|Unknown|53|ationcallycallyanti| +5668|AAAAAAAAECGBAAAA|1997-10-27|1999-10-27|Lit|4.85|3.54|5003001|exportischolar #1|3|pop|5|Music|626|callyablecally|N/A|786567082orange87954|red|Carton|Unknown|23|eingcallycallyanti| +5669|AAAAAAAAECGBAAAA|1999-10-28|2001-10-26|Lit|3.62|2.82|1004002|edu packamalg #2|4|swimwear|1|Women|656|callyantically|medium|786567082orange87954|honeydew|Dram|Unknown|77|n stcallycallyanti| +5670|AAAAAAAAECGBAAAA|2001-10-27||Lit|7.66|2.82|1004002|importoamalg #1|2|fragrances|1|Women|656|callyantically|extra large|786567082orange87954|rose|Pound|Unknown|21|barationcallyanti| +5671|AAAAAAAAHCGBAAAA|1997-10-27||High, rigid members go subjects. Current, physical lights can shake still from the programmes. Eyes might not |3.40|1.05|3002002|importoexporti #2|2|infants|3|Children|320|barablepri|small|7456008253907khaki84|goldenrod|Dram|Unknown|53|oughtationcallyanti| +5672|AAAAAAAAICGBAAAA|1997-10-27|2000-10-26|Long, heavy police live; civil drawings face suddenly increased, new clothes. Problems run british, cultural sanctions. General, great cases shall see lessons; otherwise middle causes use|6.26|4.63|5001001|amalgscholar #1|1|rock|5|Music|317|ationoughtpri|N/A|306powder20061055996|pale|Cup|Unknown|12|ableationcallyanti| +5673|AAAAAAAAICGBAAAA|2000-10-27||Long, heavy police live; civil drawings face suddenly increased, new clothes. Problems run british, cultural sanctions. General, great cases shall see lessons; otherwise middle causes use|41.30|4.63|5001001|edu packnameless #4|14|glassware|7|Home|116|callyoughtought|N/A|306powder20061055996|royal|Gross|Unknown|27|priationcallyanti| +5674|AAAAAAAAKCGBAAAA|1997-10-27|1999-10-27|Other sides will matter as strong tables. Trousers should keep only common polls. Exactly young heads used to drag popular years. Hours work nationa|5.70|2.56|3002001|importoexporti #1|2|infants|3|Children|631|oughtprically|large|27710753salmon131958|snow|Bundle|Unknown|6|eseationcallyanti| +5675|AAAAAAAAKCGBAAAA|1999-10-28|2001-10-26|Men save largely statutory, practical eyes. Wooden, additional men enquire perhaps so right years. National children answer separately fierce protests; young, reasona|3.17|1.61|3002001|importocorp #4|2|diamonds|6|Jewelry|631|oughtprically|N/A|27710753salmon131958|deep|Ounce|Unknown|56|antiationcallyanti| +5676|AAAAAAAAKCGBAAAA|2001-10-27||As maximum individuals bother solely years. Hostile products could not deserve. Previous|0.37|0.21|1004001|edu packamalg #1|2|swimwear|1|Women|181|oughtprically|large|957674239027182lace8|turquoise|Gross|Unknown|9|callyationcallyanti| +5677|AAAAAAAANCGBAAAA|1997-10-27||Here serious ministers would devise in particular adjacent to a weeks. Standard shares will call much bad ideas. Countries should expand. Slightly peculiar examinations wo|0.70|0.21|10006017|corpunivamalg #17|6|musical|10|Electronics|617|ationoughtcally|N/A|194338186457yellow88|smoke|Unknown|Unknown|28|ationationcallyanti| +5678|AAAAAAAAOCGBAAAA|1997-10-27|2000-10-26|Outdoor, white buildings must not keep concerning the sessions. So annual minerals stress clear to a voters; also cool thanks raise there evident, young aspects. Others must appeal e|4.28|2.95|1001001|amalgamalg #1|1|dresses|1|Women|332|ablepripri|large|91035433320magenta95|powder|Carton|Unknown|4|eingationcallyanti| +5679|AAAAAAAAOCGBAAAA|2000-10-27||About total honours carry now never emotio|0.76|2.95|1001001|exportimaxi #4|3|computers|9|Books|332|ablepripri|N/A|91035433320magenta95|ghost|Oz|Unknown|19|n stationcallyanti| +5680|AAAAAAAAADGBAAAA|1997-10-27|1999-10-27|Forces can live mostly. Again indian stars ought to establish just. So british y|6.35|5.14|7016001|corpnameless #1|16|furniture|7|Home|270|barationable|N/A|8227maroon3723951181|turquoise|Each|Unknown|30|bareingcallyanti| +5681|AAAAAAAAADGBAAAA|1999-10-28|2001-10-26|Possible parties find acros|1.26|0.52|7016001|edu packunivamalg #6|16|sports|9|Books|270|barationable|N/A|1887magenta978641011|black|Dozen|Unknown|14|oughteingcallyanti| +5682|AAAAAAAAADGBAAAA|2001-10-27||Independent hours might begin too large days; dangerous minutes see houses. Good members get interests. Other policies respond paradoxi|8.48|6.27|7016001|scholarcorp #3|5|earings|6|Jewelry|270|barationable|N/A|25609732430614hot771|yellow|Lb|Unknown|51|ableeingcallyanti| +5683|AAAAAAAADDGBAAAA|1997-10-27||Almost new charges prove necessary provinces. Days lose almost|4.20|3.61|7009004|maxibrand #4|9|mattresses|7|Home|23|priable|N/A|39332863455264pale92|cornflower|Each|Unknown|26|prieingcallyanti| +5684|AAAAAAAAEDGBAAAA|1997-10-27|2000-10-26|Subsequent events get international trousers; children show. Different, statutory birds would involve alone black features. Incomes will not offer wit|32.88|18.74|6012005|importobrand #5|12|costume|6|Jewelry|238|eingpriable|N/A|170129444purple85587|sienna|Cup|Unknown|33|eseeingcallyanti| +5685|AAAAAAAAEDGBAAAA|2000-10-27||Men may think local disciplines. Deeply cold difficulties wind soviet, particular leaves. Soft standards enjoy. Now different ways last however recent, p|4.11|2.21|8016006|corpmaxi #6|16|golf|8|Sports|340|eingpriable|N/A|170129444purple85587|spring|Box|Unknown|69|antieingcallyanti| +5686|AAAAAAAAGDGBAAAA|1997-10-27|1999-10-27|Tonnes shall limit neat, likel|7.43|6.53|10003009|exportiunivamalg #9|3|dvd/vcr players|10|Electronics|174|eseationought|N/A|80thistle91361563001|sky|Each|Unknown|32|callyeingcallyanti| +5687|AAAAAAAAGDGBAAAA|1999-10-28|2001-10-26|Traditional wages may not feel really nervous homes. Hard levels rest now european governments. Intimate, assistant sections can think criminal rights. Attitudes break controls. Unique|2.40|6.53|10003009|amalgmaxi #8|3|archery|8|Sports|174|eseationought|N/A|93honeydew7466722199|seashell|Dram|Unknown|49|ationeingcallyanti| +5688|AAAAAAAAGDGBAAAA|2001-10-27||Vehicles should not raise things. Years would not measure early public successes. Proces|3.70|1.55|6007001|brandcorp #1|7|pendants|6|Jewelry|174|eseationought|N/A|998746468813grey2609|hot|Carton|Unknown|17|eingeingcallyanti| +5689|AAAAAAAAJDGBAAAA|1997-10-27||Military rounds come months. Young crops see fine degrees. Points used to use economic calculations. Better great wings conclude; somewhat american cases see now big days. Still |7.77|6.21|4002002|importoedu pack #2|2|mens|4|Shoes|154|eseantiought|medium|561701sky83296802871|violet|Dram|Unknown|2|n steingcallyanti| +5690|AAAAAAAAKDGBAAAA|1997-10-27|2000-10-26|Possibly environmental links must hurt faster bright, cultural lovers. Rooms could|2.09|0.89|7001001|amalgbrand #1|1|bathroom|7|Home|174|eseationought|N/A|7811340snow137734288|sky|Carton|Unknown|24|barn stcallyanti| +5691|AAAAAAAAKDGBAAAA|2000-10-27||Possibly environmental links must hurt faster bright, cultural lovers. Rooms could|0.90|0.67|7001001|scholarmaxi #2|15|fishing|8|Sports|174|eseationought|N/A|7811340snow137734288|smoke|Oz|Unknown|76|oughtn stcallyanti| +5692|AAAAAAAAMDGBAAAA|1997-10-27|1999-10-27|Accounts used to matter crucially. More than useful ha|8.72|4.01|9015005|scholarunivamalg #5|15|fiction|9|Books|107|ationbarought|N/A|873807blanched657824|peach|Unknown|Unknown|11|ablen stcallyanti| +5693|AAAAAAAAMDGBAAAA|1999-10-28|2001-10-26|Accounts used to matter crucially. More than useful ha|2.73|2.29|9015005|namelessunivamalg #16|8|scanners|10|Electronics|107|ationbarought|N/A|4642961906132spring3|lime|Carton|Unknown|59|prin stcallyanti| +5694|AAAAAAAAMDGBAAAA|2001-10-27||Accounts used to matter crucially. More than useful ha|1.51|2.29|9015005|edu packbrand #7|14|estate|6|Jewelry|107|ationbarought|N/A|8746788601white03446|seashell|Cup|Unknown|27|esen stcallyanti| +5695|AAAAAAAAPDGBAAAA|1997-10-27||Mistakes preserve there impossible, new customers. Also french vegetables ought to decide possible others. Just young girls administer individual disputes. Extensive, |7.59|5.00|7002006|importobrand #6|2|bedding|7|Home|157|ationantiought|N/A|678chocolate70774355|plum|Gram|Unknown|3|antin stcallyanti| +5696|AAAAAAAAAEGBAAAA|1997-10-27|2000-10-26|Leaves register important observers. Genuine authorities ought to fire then standard, heavy wives; sure significant shadows gain high. Mental, great seats work other, low resources. Busy, scot|9.67|8.02|7009007|maxibrand #7|9|mattresses|7|Home|443|prieseese|N/A|9002turquoise2484194|lawn|Tbl|Unknown|19|callyn stcallyanti| +5697|AAAAAAAAAEGBAAAA|2000-10-27||Respectively everyday stations obscure with a examples. Glad resources want right pu|3.76|3.23|1001002|amalgamalg #2|1|dresses|1|Women|196|prieseese|economy|9002turquoise2484194|sandy|Dram|Unknown|77|ationn stcallyanti| +5698|AAAAAAAACEGBAAAA|1997-10-27|1999-10-27|Identical employers will displace free reasons. Bad, likely refugees must display rare, busy steps. Now conventional sides see yesterday prime bedrooms. O|3.57|2.78|3001001|amalgexporti #1|1|newborn|3|Children|845|antieseeing|large|67220412392059royal4|brown|Oz|Unknown|49|eingn stcallyanti| +5699|AAAAAAAACEGBAAAA|1999-10-28|2001-10-26|Identical employers will displace free reasons. Bad, likely refugees must display rare, busy steps. Now conventional sides see yesterday prime bedrooms. O|2.87|1.63|3001001|maxiunivamalg #5|9|televisions|10|Electronics|5|antieseeing|N/A|865beige084027352880|steel|N/A|Unknown|18|n stn stcallyanti| +5700|AAAAAAAACEGBAAAA|2001-10-27||Sounds would strike less monetary players. Families could consider originally; pregnant cats would underst|3.08|1.63|3001001|exportiexporti #1|9|toddlers|3|Children|5|antieseeing|medium|865beige084027352880|saddle|Oz|Unknown|30|barbarationanti| +5701|AAAAAAAAFEGBAAAA|1997-10-27||Always coloured birds cou|9.28|6.21|8003002|exportinameless #2|3|basketball|8|Sports|9|n st|N/A|4117581708grey534850|navy|Bunch|Unknown|17|oughtbarationanti| +5702|AAAAAAAAGEGBAAAA|1997-10-27|2000-10-26|Only efficient hands may get always namely good questions. Social projects carry already usual councils. Legal, dear kids will like then |6.65|2.06|4004001|edu packedu pack #1|4|athletic|4|Shoes|558|eingantianti|extra large|580914270yellow35398|thistle|Tsp|Unknown|16|ablebarationanti| +5703|AAAAAAAAGEGBAAAA|2000-10-27||Only efficient hands may get always namely good questions. Social projects carry already usual councils. Legal, dear kids will like then |4.27|2.06|1004002|edu packamalg #2|4|swimwear|1|Women|115|antioughtought|extra large|74604pink02571660383|salmon|Unknown|Unknown|32|pribarationanti| +5704|AAAAAAAAIEGBAAAA|1997-10-27|1999-10-27|Roman, conventional dogs used to help even tightly english pictures. Clear days|0.35|0.29|5003001|exportischolar #1|3|pop|5|Music|328|eingablepri|N/A|3yellow2558797170357|puff|Lb|Unknown|77|esebarationanti| +5705|AAAAAAAAIEGBAAAA|1999-10-28|2001-10-26|Roman, conventional dogs used to help even tightly english pictures. Clear days|1.85|0.77|6002002|importocorp #2|2|diamonds|6|Jewelry|665|eingablepri|N/A|6154818lawn205287808|mint|Box|Unknown|66|antibarationanti| +5706|AAAAAAAAIEGBAAAA|2001-10-27||Roman, conventional dogs used to help even tightly english pictures. Clear days|4.83|4.05|9015007|scholarunivamalg #7|15|fiction|9|Books|462|ablecallyese|N/A|60654ghost4407309693|salmon|Case|Unknown|42|callybarationanti| +5707|AAAAAAAALEGBAAAA|1997-10-27||Scientific, usefu|0.91|0.77|6011004|amalgbrand #4|11|semi-precious|6|Jewelry|175|antiationought|N/A|spring70928475271040|blue|Bunch|Unknown|33|ationbarationanti| +5708|AAAAAAAAMEGBAAAA|1997-10-27|2000-10-26|Annual democrats create only emissions. Huge teachers could tour there ways. There british plans make. New, inadequate authorities may not handle like a records.|6.49|4.21|8006009|corpnameless #9|6|football|8|Sports|213|prioughtable|N/A|97pink78205056894754|lawn|Pallet|Unknown|24|eingbarationanti| +5709|AAAAAAAAMEGBAAAA|2000-10-27||Legal eyes should not notice rapidly perfect eyes. Recently brown sheets leave too steps. There personal needs cannot start now re|9.63|7.02|8006009|amalgedu pack #2|6|womens|4|Shoes|213|prioughtable|large|84193honeydew6787483|yellow|Each|Unknown|27|n stbarationanti| +5710|AAAAAAAAOEGBAAAA|1997-10-27|1999-10-27|Current sections could not vary small others|4.20|3.02|2002001|importoimporto #1|2|shirts|2|Men|36|callypri|small|55267rosy26998503704|turquoise|Oz|Unknown|6|baroughtationanti| +5711|AAAAAAAAOEGBAAAA|1999-10-28|2001-10-26|Current sections could not vary small others|4.91|3.02|10016017|corpamalgamalg #17|16|wireless|10|Electronics|662|callypri|N/A|880502167204pink6033|tan|N/A|Unknown|30|oughtoughtationanti| +5712|AAAAAAAAOEGBAAAA|2001-10-27||Current sections could not vary small others|1.97|0.88|2001001|amalgimporto #1|16|accessories|2|Men|662|ablecallycally|medium|738849923581tomato35|slate|Ounce|Unknown|19|ableoughtationanti| +5713|AAAAAAAABFGBAAAA|1997-10-27||Sympathetic, ready buses bump however specific buil|3.24|2.55|8002004|importonameless #4|2|baseball|8|Sports|896|callyn steing|N/A|728944061hot93740926|almond|Tsp|Unknown|9|prioughtationanti| +5714|AAAAAAAACFGBAAAA|1997-10-27|2000-10-26|Reduced, rational stories ought to mean dangerous children. Quickly true |2.88|0.89|1004001|edu packamalg #1|4|swimwear|1|Women|527|ationableanti|medium|07708574160azure9820|moccasin|Ounce|Unknown|5|eseoughtationanti| +5715|AAAAAAAACFGBAAAA|2000-10-27||Reduced, rational stories ought to mean dangerous children. Quickly true |2.85|1.82|1004001|edu packexporti #2|4|school-uniforms|3|Children|664|esecallycally|petite|07708574160azure9820|purple|Pallet|Unknown|5|antioughtationanti| +5716|AAAAAAAAEFGBAAAA|1997-10-27|1999-10-27|Small, easy drawings reve|4.18|2.25|5003001|exportischolar #1|3|pop|5|Music|636|callyprically|N/A|54341641798peach2658|drab|Unknown|Unknown|83|callyoughtationanti| +5717|AAAAAAAAEFGBAAAA|1999-10-28|2001-10-26|Small, easy drawings reve|3.67|2.25|5003001|edu packamalg #2|4|swimwear|1|Women|301|oughtbarpri|small|54341641798peach2658|navajo|Box|Unknown|30|ationoughtationanti| +5718|AAAAAAAAEFGBAAAA|2001-10-27||Rounds tell. Able, famous visitors ought to sell absolute, small days. Female, dark guns d|3.74|1.45|5003001|importounivamalg #7|12|home repair|9|Books|455|antiantiese|N/A|54341641798peach2658|burlywood|Ton|Unknown|47|eingoughtationanti| +5719|AAAAAAAAHFGBAAAA|1997-10-27||Good, helpful men close. Please difficult lakes should waste very conservative, labour |3.13|0.97|6014008|edu packbrand #8|14|estate|6|Jewelry|237|ationpriable|N/A|27371papaya622344771|antique|N/A|Unknown|23|n stoughtationanti| +5720|AAAAAAAAIFGBAAAA|1997-10-27|2000-10-26|Just sheer others support of course then vital eggs. Polls used to distinguish easily complex circumstan|1.59|0.77|8013001|exportimaxi #1|13|sailing|8|Sports|503|pribaranti|N/A|334832425lawn3621973|light|Oz|Unknown|80|barableationanti| +5721|AAAAAAAAIFGBAAAA|2000-10-27||Just sheer others support of course then vital eggs. Polls used to distinguish easily complex circumstan|0.90|0.47|2001002|amalgimporto #2|13|accessories|2|Men|484|pribaranti|small|334832425lawn3621973|puff|Box|Unknown|3|oughtableationanti| +5722|AAAAAAAAKFGBAAAA|1997-10-27|1999-10-27|Odd, political authorities will lie really other, social occupations. Here final firms catch more secondary, empirical sports. Bef|8.17|2.85|10004011|edu packunivamalg #11|4|audio|10|Electronics|224|eseableable|N/A|66277800976591slate2|medium|Oz|Unknown|3|ableableationanti| +5723|AAAAAAAAKFGBAAAA|1999-10-28|2001-10-26|Irrelevant differences stifle too industrial, blind affairs. Pounds will prevent free years. Years smile services. Only decades might express for example in a women. Once again |5.59|4.86|8006004|corpnameless #4|6|football|8|Sports|288|eseableable|N/A|66277800976591slate2|purple|Ounce|Unknown|18|priableationanti| +5724|AAAAAAAAKFGBAAAA|2001-10-27||Irrelevant differences stifle too industrial, blind affairs. Pounds will prevent free years. Years smile services. Only decades might express for example in a women. Once again |7.03|4.86|7010005|univnameless #5|6|flatware|7|Home|120|barableought|N/A|66277800976591slate2|sienna|Dram|Unknown|19|eseableationanti| +5725|AAAAAAAANFGBAAAA|1997-10-27||Compatible products come there new, physical individuals. International conditions could kill there more total grounds. Long, colourful cuts cannot cry just products. Years ought to go s|33.02|26.74|5004002|edu packscholar #2|4|classical|5|Music|168|eingcallyought|N/A|sandy229171431372795|orchid|Bunch|Unknown|8|antiableationanti| +5726|AAAAAAAAOFGBAAAA|1997-10-27|2000-10-26|Facts can highlig|4.20|2.22|1001001|amalgamalg #1|1|dresses|1|Women|282|ableeingable|medium|584814salmon71344370|salmon|Tsp|Unknown|52|callyableationanti| +5727|AAAAAAAAOFGBAAAA|2000-10-27||Facts can highlig|1.18|1.06|9002004|importomaxi #4|1|business|9|Books|282|ableeingable|N/A|584814salmon71344370|peru|Gram|Unknown|21|ationableationanti| +5728|AAAAAAAAAGGBAAAA|1997-10-27|1999-10-27|Totally individual patients examine. New, appropriate things lik|2.49|0.87|9013005|exportiunivamalg #5|13|self-help|9|Books|82|ableeing|N/A|1924453yellow0890927|saddle|Pallet|Unknown|53|eingableationanti| +5729|AAAAAAAAAGGBAAAA|1999-10-28|2001-10-26|Combined fing|8.54|0.87|9013005|edu packamalg #2|13|swimwear|1|Women|458|eingantiese|medium|1924453yellow0890927|rose|Gram|Unknown|18|n stableationanti| +5730|AAAAAAAAAGGBAAAA|2001-10-27||Remains might go even |96.30|0.87|9013005|maxibrand #1|13|mattresses|7|Home|458|eingantiese|N/A|1924453yellow0890927|violet|N/A|Unknown|21|barpriationanti| +5731|AAAAAAAADGGBAAAA|1997-10-27||Forward students can involve there aware lawyers. Scientifically costly achievements could involve sta|1.09|0.46|8013002|exportimaxi #2|13|sailing|8|Sports|589|n steinganti|N/A|turquoise34872633268|salmon|Cup|Unknown|40|oughtpriationanti| +5732|AAAAAAAAEGGBAAAA|1997-10-27|2000-10-26|Enough average men keep conditions. Smooth magistrates kill only increasingly labour numbers. Numbers beat for a positions. Villages could make yet except for a thoughts. Little, cold prices think; d|1.41|0.74|9015009|scholarunivamalg #9|15|fiction|9|Books|398|eingn stpri|N/A|6851149443steel04518|steel|Ton|Unknown|1|ablepriationanti| +5733|AAAAAAAAEGGBAAAA|2000-10-27||Enough average men keep conditions. Smooth magistrates kill only increasingly labour numbers. Numbers beat for a positions. Villages could make yet except for a thoughts. Little, cold prices think; d|6.67|0.74|5003002|exportischolar #2|3|pop|5|Music|398|eingn stpri|N/A|20573pale60842333445|papaya|Gram|Unknown|20|pripriationanti| +5734|AAAAAAAAGGGBAAAA|1997-10-27|1999-10-27|Ideas shall learn always. Armies should go as yet growing, correct years. Now unlikely weeks become bad english years. Gently vital bone|0.12|0.04|6010007|univbrand #7|10|jewelry boxes|6|Jewelry|394|esen stpri|N/A|747442054330868plum1|peru|N/A|Unknown|53|esepriationanti| +5735|AAAAAAAAGGGBAAAA|1999-10-28|2001-10-26|Wrong farmers can decide well preliminary women; evidently short times shall not improve more dependent sides. Chapters gl|3.23|2.68|6010007|univbrand #8|10|jewelry boxes|6|Jewelry|394|esen stpri|N/A|68978202418linen5024|peru|Gram|Unknown|71|antipriationanti| +5736|AAAAAAAAGGGBAAAA|2001-10-27||Solemnly current proposals wait organizations; months approach to a directors. Bad, fine years locate prisons. Internal, fundamental things provide in the light of a times. Social leaders i|7.12|2.20|1004001|edu packamalg #1|4|swimwear|1|Women|144|eseeseought|small|843669peach087642236|salmon|Pound|Unknown|37|callypriationanti| +5737|AAAAAAAAJGGBAAAA|1997-10-27||Yet average schemes help therefore else confident taxes. Facts used to slip black studies. Northern gardens forget. Poor, high users might not show now with a things. Little plants view at last hig|96.49|37.63|4002002|importoedu pack #2|2|mens|4|Shoes|507|ationbaranti|extra large|338013890963440dark8|purple|Each|Unknown|14|ationpriationanti| +5738|AAAAAAAAKGGBAAAA|1997-10-27|2000-10-26|Sudden, eligible workers pour french, capable d|4.53|2.31|2002001|importoimporto #1|2|shirts|2|Men|479|n stationese|N/A|0894741261navy721199|sandy|Tbl|Unknown|21|eingpriationanti| +5739|AAAAAAAAKGGBAAAA|2000-10-27||Sudden, eligible workers pour french, capable d|7.84|2.31|8002010|importonameless #10|2|baseball|8|Sports|479|n stationese|N/A|2wheat45280883878474|rose|Pallet|Unknown|36|n stpriationanti| +5740|AAAAAAAAMGGBAAAA|1997-10-27|1999-10-27|Certain, criminal loans used to ensure lightly following pages. Subjects a|7.85|6.75|2001001|amalgimporto #1|1|accessories|2|Men|212|ableoughtable|extra large|808591steel660010316|tan|Each|Unknown|3|bareseationanti| +5741|AAAAAAAAMGGBAAAA|1999-10-28|2001-10-26|Certain, criminal loans used to ensure lightly following pages. Subjects a|0.71|0.48|6007006|brandcorp #6|7|pendants|6|Jewelry|212|ableoughtable|N/A|808591steel660010316|spring|Carton|Unknown|8|oughteseationanti| +5742|AAAAAAAAMGGBAAAA|2001-10-27||Ever unable police will know immediately pieces. Then central|2.30|0.48|10009014|maxiunivamalg #14|9|televisions|10|Electronics|210|ableoughtable|N/A|44529047265sandy5518|saddle|Ounce|Unknown|12|ableeseationanti| +5743|AAAAAAAAPGGBAAAA|1997-10-27||Pictures ought to run. Bad, public workers pr|24.80|21.82|8001004|amalgnameless #4|1|athletic shoes|8|Sports|603|pribarcally|N/A|8832200439khaki81028|ivory|Carton|Unknown|18|prieseationanti| +5744|AAAAAAAAAHGBAAAA|1997-10-27|2000-10-26|As political activities go never after a years; pleased, important schools release specific, legal |3.02|2.14|2004001|edu packimporto #1|4|sports-apparel|2|Men|291|oughtn stable|small|71521579metallic1521|white|Tbl|Unknown|9|eseeseationanti| +5745|AAAAAAAAAHGBAAAA|2000-10-27||As political activities go never after a years; pleased, important schools release specific, legal |6.87|4.87|6011002|amalgbrand #2|4|semi-precious|6|Jewelry|338|eingpripri|N/A|976pink8616484361856|sienna|Pound|Unknown|20|antieseationanti| +5746|AAAAAAAACHGBAAAA|1997-10-27|1999-10-27|Suitable, new be|2.69|1.26|7016007|corpnameless #7|16|furniture|7|Home|452|ableantiese|N/A|447129671305grey9631|goldenrod|Cup|Unknown|37|callyeseationanti| +5747|AAAAAAAACHGBAAAA|1999-10-28|2001-10-26|Years shall see immediately; farmers may not go only, real r|85.10|1.26|7016007|importoscholar #2|16|country|5|Music|452|ableantiese|N/A|29627honeydew9605737|snow|Dozen|Unknown|20|ationeseationanti| +5748|AAAAAAAACHGBAAAA|2001-10-27||Social theories may prevent barely together secret objects. Areas keep ago. Educational blues use. Regional, aware terms look generally there new days. Homes would bear generall|0.57|0.39|1002001|importoamalg #1|2|fragrances|1|Women|445|ableantiese|medium|29627honeydew9605737|plum|Each|Unknown|86|eingeseationanti| +5749|AAAAAAAAFHGBAAAA|1997-10-27||Strategic jobs move then also secondary lands; short, nervous conditions will not adopt; councils shed sales. As industrial societies must cut clear consequences. Communist, informal ye|5.46|1.63|6002006|importocorp #6|2|diamonds|6|Jewelry|844|eseeseeing|N/A|489maroon93616638379|linen|Tbl|Unknown|29|n steseationanti| +5750|AAAAAAAAGHGBAAAA|1997-10-27|2000-10-26|Ago foreign writings leave; even considerable artists let fully then minimal workers. Clear ministers keep. Specifically dry men increase central tests. Living, alternative meanings ought|9.33|6.99|5004001|edu packscholar #1|4|classical|5|Music|438|eingpriese|N/A|61797126695indian173|pink|Tsp|Unknown|6|barantiationanti| +5751|AAAAAAAAGHGBAAAA|2000-10-27||General, human countries might test certain, small soldiers; senior, positive sounds must not become ashore for a expressions. Cells make there at the acciden|0.96|6.99|5004001|namelessnameless #2|8|outdoor|8|Sports|438|eingpriese|N/A|534white524344464015|tan|Unknown|Unknown|65|oughtantiationanti| +5752|AAAAAAAAIHGBAAAA|1997-10-27|1999-10-27|New minutes will visit small expenses. Obviously normal losses w|9.04|5.51|3001001|amalgexporti #1|1|newborn|3|Children|29|n stable|medium|3738622757green99414|rosy|Each|Unknown|45|ableantiationanti| +5753|AAAAAAAAIHGBAAAA|1999-10-28|2001-10-26|New minutes will visit small expenses. Obviously normal losses w|2.64|5.51|3001001|amalgexporti #2|1|newborn|3|Children|29|n stable|large|burnished53354829785|smoke|Cup|Unknown|66|priantiationanti| +5754|AAAAAAAAIHGBAAAA|2001-10-27||New minutes will visit small expenses. Obviously normal losses w|1.72|5.51|9003007|exportimaxi #7|1|computers|9|Books|409|n stable|N/A|burnished53354829785|hot|Cup|Unknown|14|eseantiationanti| +5755|AAAAAAAALHGBAAAA|1997-10-27||Big products blame ever electoral libraries. Final, musical women drop more video-taped devel|2.44|1.46|6007004|brandcorp #4|7|pendants|6|Jewelry|398|eingn stpri|N/A|turquoise80927186443|snow|Oz|Unknown|27|antiantiationanti| +5756|AAAAAAAAMHGBAAAA|1997-10-27|2000-10-26|Correct, personal years will draw above spiritual, reasonable parents. Things shall go very to a fee|3.19|2.71|4002001|importoedu pack #1|2|mens|4|Shoes|189|n steingought|medium|sky53473397511808988|sandy|Tsp|Unknown|48|callyantiationanti| +5757|AAAAAAAAMHGBAAAA|2000-10-27||Fortunate, average yards will keep live so new eyes. So|4.35|2.71|4002001|importoimporto #2|2|shirts|2|Men|189|n steingought|extra large|sky53473397511808988|sienna|Carton|Unknown|65|ationantiationanti| +5758|AAAAAAAAOHGBAAAA|1997-10-27|1999-10-27|Late, perfect questions run criminal, wrong indians. Even popular answers close too surely little businesses. Quietly exact results sell with the days|2.90|1.10|5004001|edu packscholar #1|4|classical|5|Music|55|antianti|N/A|0259734peach11779760|violet|Dram|Unknown|24|eingantiationanti| +5759|AAAAAAAAOHGBAAAA|1999-10-28|2001-10-26|Better exceptional others should feel maybe. Possible details would not forget even. Now different groups correlate cri|3.40|1.80|6004008|edu packcorp #8|4|bracelets|6|Jewelry|898|antianti|N/A|0259734peach11779760|wheat|Dozen|Unknown|57|n stantiationanti| +5760|AAAAAAAAOHGBAAAA|2001-10-27||Better exceptional others should feel maybe. Possible details would not forget even. Now different groups correlate cri|1.39|1.80|6004008|importomaxi #1|2|business|9|Books|317|antianti|N/A|9621254784metallic53|steel|Pound|Unknown|54|barcallyationanti| +5761|AAAAAAAABIGBAAAA|1997-10-27||Simple|1.46|0.80|3004002|edu packexporti #2|4|school-uniforms|3|Children|235|antipriable|large|2096270olive36774390|lime|N/A|Unknown|9|oughtcallyationanti| +5762|AAAAAAAACIGBAAAA|1997-10-27|2000-10-26|At least short chi|7.14|4.64|5002001|importoscholar #1|2|country|5|Music|131|oughtpriought|N/A|91005095722819lime22|sienna|Tsp|Unknown|64|ablecallyationanti| +5763|AAAAAAAACIGBAAAA|2000-10-27||Only times can ide|6.04|5.07|4002002|importoedu pack #2|2|mens|4|Shoes|131|oughtpriought|medium|91005095722819lime22|peru|Case|Unknown|6|pricallyationanti| +5764|AAAAAAAAEIGBAAAA|1997-10-27|1999-10-27|Classes w|40.35|21.38|2001001|amalgimporto #1|1|accessories|2|Men|223|priableable|large|3steel73979417452940|snow|Dram|Unknown|30|esecallyationanti| +5765|AAAAAAAAEIGBAAAA|1999-10-28|2001-10-26|In order modern effects say too organisational birds. Real operators shall not use on a committees. Legal, far documents s|4.01|2.92|2001001|exportinameless #6|1|wallpaper|7|Home|223|priableable|N/A|3steel73979417452940|blue|Unknown|Unknown|77|anticallyationanti| +5766|AAAAAAAAEIGBAAAA|2001-10-27||In order modern effects say too organisational birds. Real operators shall not use on a committees. Legal, far documents s|1.10|2.92|8011007|amalgmaxi #7|11|archery|8|Sports|321|oughtablepri|N/A|1345270seashell81867|blue|Oz|Unknown|8|callycallyationanti| +5767|AAAAAAAAHIGBAAAA|1997-10-27||Elections establish again quick years. Right groups might accompany urg|7.62|6.47|5001002|amalgscholar #2|1|rock|5|Music|262|ablecallyable|N/A|86614767papaya855214|steel|Lb|Unknown|85|ationcallyationanti| +5768|AAAAAAAAIIGBAAAA|1997-10-27|2000-10-26|Inc notes go earlier discussions. Pockets grow more goals. Diplom|1.06|0.66|3003001|exportiexporti #1|3|toddlers|3|Children|210|baroughtable|small|67833855192312olive2|plum|Ounce|Unknown|39|eingcallyationanti| +5769|AAAAAAAAIIGBAAAA|2000-10-27||Official, monetary cells settle seriously low elections. Inst|4.55|2.04|3002002|importoexporti #2|3|infants|3|Children|210|baroughtable|extra large|56361peach3344523902|saddle|Bunch|Unknown|18|n stcallyationanti| +5770|AAAAAAAAKIGBAAAA|1997-10-27|1999-10-27|Still suit|9.35|6.63|4001001|amalgedu pack #1|1|womens|4|Shoes|132|ablepriought|large|97076173turquoise585|pale|Carton|Unknown|23|barationationanti| +5771|AAAAAAAAKIGBAAAA|1999-10-28|2001-10-26|Still suit|2.54|1.90|7002002|importobrand #2|2|bedding|7|Home|65|antically|N/A|330368navajo71015060|metallic|Lb|Unknown|31|oughtationationanti| +5772|AAAAAAAAKIGBAAAA|2001-10-27|||0.69|0.57|4002001|importoedu pack #1||mens|4|Shoes|65|antically|small|27749medium213508920|royal|Dozen|||ableationationanti| +5773|AAAAAAAANIGBAAAA|1997-10-27||Criteria say evenly beyond a terms. Terrible, full seeds cannot need slightly just revolutionary papers. Psychological friends take elsewhere. Elsewhere|2.36|1.03|4002002|importoedu pack #2|2|mens|4|Shoes|88|eingeing|medium|14742595480074lemon8|turquoise|Ounce|Unknown|19|priationationanti| +5774|AAAAAAAAOIGBAAAA|1997-10-27|2000-10-26|Important lette|2.03|1.31|3003001|exportiexporti #1|3|toddlers|3|Children|779|n stationation|petite|480570honeydew650040|violet|Tbl|Unknown|22|eseationationanti| +5775|AAAAAAAAOIGBAAAA|2000-10-27||Financial, firm parents listen rough, other facilities. Common, bad miles could not recall therefore then central streets. |9.00|7.83|3003001|exportiimporto #2|3|pants|2|Men|779|n stationation|extra large|480570honeydew650040|papaya|Dram|Unknown|35|antiationationanti| +5776|AAAAAAAAAJGBAAAA|1997-10-27|1999-10-27|Added, similar grounds spend also concrete terms. Fellow, mass|2.27|0.70|2001001|amalgimporto #1|1|accessories|2|Men|256|callyantiable|petite|95peru85616843651469|pink|Tsp|Unknown|95|callyationationanti| +5777|AAAAAAAAAJGBAAAA|1999-10-28|2001-10-26|Extra, heavy years ought to love only effectively particular years. Working affairs believe most into a acres. Senior pupils ring. Small, |7.81|0.70|10002015|importounivamalg #15|1|camcorders|10|Electronics|256|callyantiable|N/A|2598473moccasin42350|tan|Ton|Unknown|33|ationationationanti| +5778|AAAAAAAAAJGBAAAA|2001-10-27||Also right miles oppose currently systems. Needs sit yesterday. Never sl|58.19|18.62|10002015|exportischolar #1|1|pop|5|Music|256|callyantiable|N/A|1metallic00099400166|cornsilk|Tsp|Unknown|12|eingationationanti| +5779|AAAAAAAADJGBAAAA|1997-10-27||Economic, quiet charts go then. European, famous benefits shall name. Courts flour|0.80|0.43|2004002|edu packimporto #2|4|sports-apparel|2|Men|280|bareingable|petite|06251809powder942131|ivory|Tbl|Unknown|79|n stationationanti| +5780|AAAAAAAAEJGBAAAA|1997-10-27|2000-10-26|Huge workers must not show for a members. Intentions pay never aware, basic children. Stairs cope relentlessly. Traditional, pol|2.67|1.06|7012001|importonameless #1|12|paint|7|Home|234|esepriable|N/A|9turquoise4631385126|seashell|Bunch|Unknown|84|bareingationanti| +5781|AAAAAAAAEJGBAAAA|2000-10-27||Huge workers must not show for a members. Intentions pay never aware, basic children. Stairs cope relentlessly. Traditional, pol|8.99|1.06|7012001|scholarbrand #6|12|custom|6|Jewelry|318|eingoughtpri|N/A|9turquoise4631385126|peach|Ounce|Unknown|97|oughteingationanti| +5782|AAAAAAAAGJGBAAAA|1997-10-27|1999-10-27|Modern, average faces describe equally particular men; meanwhile professional gifts must consolidate richly red considerable studies. Great, success|6.14|2.08|2004001|edu packimporto #1|4|sports-apparel|2|Men|981|oughteingn st|small|978891700351purple25|pink|Each|Unknown|50|ableeingationanti| +5783|AAAAAAAAGJGBAAAA|1999-10-28|2001-10-26|Men wait away easy, head years. Only, capitalist artis|9.86|2.08|9002012|importomaxi #12|2|business|9|Books|466|callycallyese|N/A|4765374papaya1661757|navajo|Dozen|Unknown|33|prieingationanti| +5784|AAAAAAAAGJGBAAAA|2001-10-27||Men wait away easy, head years. Only, capitalist artis|3.03|1.45|10002005|importounivamalg #5|2|camcorders|10|Electronics|520|barableanti|N/A|4765374papaya1661757|red|Dozen|Unknown|30|eseeingationanti| +5785|AAAAAAAAJJGBAAAA|1997-10-27||Strangers gain officially enough labour problems. Overall systems may not help below lives. Heroes find just apparently generous couple|7.15|3.78|9003002|exportimaxi #2|3|computers|9|Books|315|antioughtpri|N/A|068black346195158580|lime|Ton|Unknown|5|antieingationanti| +5786|AAAAAAAAKJGBAAAA|1997-10-27|2000-10-26|International, successful bodies build. Just national refugees see necessary books. Worried, young demands shall help new courts. Around new operations cannot identify. Modest, top kinds take real d|4.76|1.47|2004001|edu packimporto #1|4|sports-apparel|2|Men|38|eingpri|medium|7341372tomato1552278|grey|N/A|Unknown|27|callyeingationanti| +5787|AAAAAAAAKJGBAAAA|2000-10-27||International, successful bodies build. Just national refugees see necessary books. Worried, young demands shall help new courts. Around new operations cannot identify. Modest, top kinds take real d|15.38|6.92|2004001|amalgscholar #2|1|rock|5|Music|38|eingpri|N/A|78601317yellow427801|salmon|Ton|Unknown|35|ationeingationanti| +5788|AAAAAAAAMJGBAAAA|1997-10-27|1999-10-27|Traditional, fine conditions come just young, dull months. Objections identify consciously without a officers. Real pro|0.13|0.07|1004001|edu packamalg #1|4|swimwear|1|Women|340|baresepri|large|6969752756tan3978599|spring|Bundle|Unknown|49|eingeingationanti| +5789|AAAAAAAAMJGBAAAA|1999-10-28|2001-10-26|Traditional, fine conditions come just young, dull months. Objections identify consciously without a officers. Real pro|6.66|3.19|1004001|exportinameless #10|4|wallpaper|7|Home|301|baresepri|N/A|6969752756tan3978599|white|Gram|Unknown|93|n steingationanti| +5790|AAAAAAAAMJGBAAAA|2001-10-27||Today fresh years keep sharply exercises. Different, industrial |7.30|5.98|6003007|exporticorp #7|4|gold|6|Jewelry|301|oughtbarpri|N/A|6969752756tan3978599|spring|Carton|Unknown|31|barn stationanti| +5791|AAAAAAAAPJGBAAAA|1997-10-27||Of course short cups start on a tasks. Both capital years draw. Sometimes good questions could not lose up a trees. Units used to get great right uni|7.37|3.24|1003002|exportiamalg #2|3|maternity|1|Women|2|able|petite|746rosy9946120141778|navajo|Cup|Unknown|90|oughtn stationanti| +5792|AAAAAAAAAKGBAAAA|1997-10-27|2000-10-26|Lives used to keep therefore along a companie|3.08|1.63|10011013|amalgamalgamalg #13|11|disk drives|10|Electronics|592|ablen stanti|N/A|1609456lace727716987|lemon|Dozen|Unknown|15|ablen stationanti| +5793|AAAAAAAAAKGBAAAA|2000-10-27||Lives used to keep therefore along a companie|2.81|1.82|10011013|namelessunivamalg #14|8|scanners|10|Electronics|218|eingoughtable|N/A|09862rose86260767584|aquamarine|Carton|Unknown|65|prin stationanti| +5794|AAAAAAAACKGBAAAA|1997-10-27|1999-10-27|Chronic polici|3.30|2.97|2003001|exportiimporto #1|3|pants|2|Men|81|oughteing|large|4midnight51360721413|turquoise|Ounce|Unknown|19|esen stationanti| +5795|AAAAAAAACKGBAAAA|1999-10-28|2001-10-26|Rare, well-known matters think foreign causes; different studies may not sa|0.56|2.97|10012016|importoamalgamalg #16|3|monitors|10|Electronics|653|priantically|N/A|4midnight51360721413|papaya|Box|Unknown|25|antin stationanti| +5796|AAAAAAAACKGBAAAA|2001-10-27||Rare, well-known matters think foreign causes; different studies may not sa|56.73|26.09|10012016|scholarnameless #7|15|tables|7|Home|653|priantically|N/A|19pink77623000722427|olive|Box|Unknown|52|callyn stationanti| +5797|AAAAAAAAFKGBAAAA|1997-10-27||Social policies experience as immense, other organizations. New products will ensure other allowances. Good|5.07|4.30|7010008|univnameless #8|10|flatware|7|Home|84|eseeing|N/A|6823pale621256376628|snow|Gross|Unknown|77|ationn stationanti| +5798|AAAAAAAAGKGBAAAA|1997-10-27|2000-10-26|Then dark professionals die in a men. Tender, appropriate statements include with a feelings. Members forget on an forms. Slowly new passengers may remember very; also precious teams see also con|5.68|3.46|3004001|edu packexporti #1|4|school-uniforms|3|Children|545|antieseanti|medium|03548rosy80337248269|plum|Unknown|Unknown|4|eingn stationanti| +5799|AAAAAAAAGKGBAAAA|2000-10-27||Particularly executive presents will not hold sentences. Quite major rates ask indeed. At present public standards disappear about. Ordinary, se|3.17|1.68|3004001|brandmaxi #4|4|reference|9|Books|545|antieseanti|N/A|03548rosy80337248269|sky|Dram|Unknown|29|n stn stationanti| +5800|AAAAAAAAIKGBAAAA|1997-10-27|1999-10-27|Hot, military things|3.20|2.52|1004001|edu packamalg #1|4|swimwear|1|Women|136|callypriought|large|105208742yellow12134|spring|Gross|Unknown|52|barbareinganti| +5801|AAAAAAAAIKGBAAAA|1999-10-28|2001-10-26|Available strategie|3.88|2.52|7012002|importonameless #2|12|paint|7|Home|136|callypriought|N/A|2978558282627slate84|plum|Gross|Unknown|17|oughtbareinganti| +5802|AAAAAAAAIKGBAAAA|2001-10-27||Available strategie|7.88|5.04|7012002|brandunivamalg #6|7|personal|10|Electronics|513|callypriought|N/A|4919violet4579643945|indian|Each|Unknown|63|ablebareinganti| +5803|AAAAAAAALKGBAAAA|1997-10-27||Confident metres look materials; gothic, other years should extend also; immediately primary consequences th|0.43|0.35|1002002|importoamalg #2|2|fragrances|1|Women|267|ationcallyable|petite|peach129077567203612|rose|Pound|Unknown|20|pribareinganti| +5804|AAAAAAAAMKGBAAAA|1997-10-27|2000-10-26|So white republics squeeze however new days; effectively whole minutes cannot give more never alternative years. Natural changes would disc|1.23|0.66|7003005|exportibrand #5|3|kids|7|Home|323|priablepri|N/A|723peru7485369350690|salmon|Pallet|Unknown|11|esebareinganti| +5805|AAAAAAAAMKGBAAAA|2000-10-27||So white republics squeeze however new days; effectively whole minutes cannot give more never alternative years. Natural changes would disc|0.75|0.66|7003005|exportischolar #2|3|pop|5|Music|323|priablepri|N/A|723peru7485369350690|sandy|Box|Unknown|33|antibareinganti| +5806|AAAAAAAAOKGBAAAA|1997-10-27|1999-10-27|Total, various theories can mean that is too religious men. Administrative men m|4.99|1.69|7016007|corpnameless #7|16|furniture|7|Home|568|eingcallyanti|N/A|puff8915246129570897|grey|Gram|Unknown|85|callybareinganti| +5807|AAAAAAAAOKGBAAAA|1999-10-28|2001-10-26|Total, various theories can mean that is too religious men. Administrative men m|1.57|0.87|7016007|amalgexporti #2|16|newborn|3|Children|53|eingcallyanti|large|puff8915246129570897|wheat|Each|Unknown|98|ationbareinganti| +5808|AAAAAAAAOKGBAAAA|2001-10-27||Total, various theories can mean that is too religious men. Administrative men m|94.40|0.87|7016007|brandunivamalg #12|16|personal|10|Electronics|53|eingcallyanti|N/A|13295192612010white4|spring|Oz|Unknown|35|eingbareinganti| +5809|AAAAAAAABLGBAAAA|1997-10-27||Simple, ec|7.35|5.14|9014002|edu packunivamalg #2|14|sports|9|Books|62|ablecally|N/A|942118635slate178072|moccasin|Oz|Unknown|66|n stbareinganti| +5810|AAAAAAAACLGBAAAA|1997-10-27|2000-10-26|Really excellent consequences shall not know then white conscious generations. British, immediate owners reflect in a others. Local parents should become approximately medical animals; personal fe|3.77|3.24|10002014|importounivamalg #14|2|camcorders|10|Electronics|205|antibarable|N/A|82322796344snow52409|sandy|Carton|Unknown|33|baroughteinganti| +5811|AAAAAAAACLGBAAAA|2000-10-27||Really excellent consequences shall not know then white conscious generations. British, immediate owners reflect in a others. Local parents should become approximately medical animals; personal fe|1.82|3.24|4002002|importoedu pack #2|2|mens|4|Shoes|205|antibarable|medium|659189344076puff1260|orange|Pound|Unknown|11|oughtoughteinganti| +5812|AAAAAAAAELGBAAAA|1997-10-27|1999-10-27|About international concentrations could avoid then alone apparent activities; inadequate, mediterranean days get eve|6.63|2.91|8010003|univmaxi #3|10|pools|8|Sports|31|oughtpri|N/A|chartreuse3218293028|turquoise|Ounce|Unknown|30|ableoughteinganti| +5813|AAAAAAAAELGBAAAA|1999-10-28|2001-10-26|New, fat questions will think into the children. Teachers meet at all well real years. Collections put only metres. Properties offer essential |2.71|2.08|4002002|importoedu pack #2|10|mens|4|Shoes|31|oughtpri|large|63066258navajo684762|yellow|N/A|Unknown|5|prioughteinganti| +5814|AAAAAAAAELGBAAAA|2001-10-27||New, fat questions will think into the children. Teachers meet at all well real years. Collections put only metres. Properties offer essential |9.50|6.74|6009007|maxicorp #7|10|womens watch|6|Jewelry|133|oughtpri|N/A|63066258navajo684762|rosy|Gross|Unknown|50|eseoughteinganti| +5815|AAAAAAAAHLGBAAAA|1997-10-27||Efforts might not suggest proceedings; featur|0.17|0.09|6009008|maxicorp #8|9|womens watch|6|Jewelry|33|pripri|N/A|9045rose366437059528|thistle|Pound|Unknown|49|antioughteinganti| +5816|AAAAAAAAILGBAAAA|1997-10-27|2000-10-26|Offices would dare then |4.39|3.55|7005007|scholarbrand #7|5|blinds/shades|7|Home|278|eingationable|N/A|0045smoke05352593836|peach|Oz|Unknown|21|callyoughteinganti| +5817|AAAAAAAAILGBAAAA|2000-10-27||Radical feet may come slightly now large patients. Below close publications make assessments; other, different ends matter different tories. Women lead strong, single men; |22.80|3.55|7005007|edu packexporti #2|5|school-uniforms|3|Children|490|eingationable|large|56934969yellow853045|tomato|Tsp|Unknown|8|ationoughteinganti| +5818|AAAAAAAAKLGBAAAA|1997-10-27|1999-10-27|Foreign, lucky components must reduce t|6.01|3.90|8013009|exportimaxi #9|13|sailing|8|Sports|571|oughtationanti|N/A|48356727059navy93337|drab|Oz|Unknown|30|eingoughteinganti| +5819|AAAAAAAAKLGBAAAA|1999-10-28|2001-10-26|Foreign, lucky components must reduce t|5.10|4.28|8013009|importoimporto #2|2|shirts|2|Men|104|esebarought|large|48356727059navy93337|forest|Bunch|Unknown|13|n stoughteinganti| +5820|AAAAAAAAKLGBAAAA|2001-10-27||Foreign, lucky components must reduce t|4.40|4.28|3002001|importoexporti #1|2|infants|3|Children|431|oughtpriese|large|7violet9033978757914|tan|Ounce|Unknown|1|barableeinganti| +5821|AAAAAAAANLGBAAAA|1997-10-27||Commercial tickets include before other waves. Very|1.24|1.06|10007008|brandunivamalg #8|7|personal|10|Electronics|157|ationantiought|N/A|063435212204papaya76|salmon|Cup|Unknown|32|oughtableeinganti| +5822|AAAAAAAAOLGBAAAA|1997-10-27|2000-10-26|Difficulties wou|6.61|2.31|6012007|importobrand #7|12|costume|6|Jewelry|334|esepripri|N/A|6salmon4520327352736|tan|Cup|Unknown|58|ableableeinganti| +5823|AAAAAAAAOLGBAAAA|2000-10-27||Far social prizes can come o|2.68|1.50|10016010|corpamalgamalg #10|16|wireless|10|Electronics|456|callyantiese|N/A|6salmon4520327352736|chartreuse|Gram|Unknown|4|priableeinganti| +5824|AAAAAAAAAMGBAAAA|1997-10-27|1999-10-27|Managerial, obvi|5.59|4.30|4004001|edu packedu pack #1|4|athletic|4|Shoes|53|prianti|petite|02800slate8869656759|sienna|Box|Unknown|63|eseableeinganti| +5825|AAAAAAAAAMGBAAAA|1999-10-28|2001-10-26|Points want long, equal sports. Normally|1.55|4.30|5003002|exportischolar #2|3|pop|5|Music|139|n stpriought|N/A|royal709568892576698|powder|Ounce|Unknown|8|antiableeinganti| +5826|AAAAAAAAAMGBAAAA|2001-10-27||Indian, american branches ask then in a feelings. New, recent banks should|0.27|4.30|5003002|corpcorp #3|6|rings|6|Jewelry|139|n stpriought|N/A|280687peru9289474264|puff|Oz|Unknown|20|callyableeinganti| +5827|AAAAAAAADMGBAAAA|1997-10-27||Offenders would not make now. Necessary, forward rights abandon never. Big pupils ensure today o|4.69|2.76|2004002|edu packimporto #2|4|sports-apparel|2|Men|147|ationeseought|petite|877843973dim46723685|white|Unknown|Unknown|72|ationableeinganti| +5828|AAAAAAAAEMGBAAAA|1997-10-27|2000-10-26|Local agencies wish members. New year|2.85|1.25|8006009|corpnameless #9|6|football|8|Sports|313|prioughtpri|N/A|4smoke16235200991144|lime|Each|Unknown|61|eingableeinganti| +5829|AAAAAAAAEMGBAAAA|2000-10-27||Eyes could believe only great, standard customers. Historical lines may smile only resources. Head, old merchants could persuade including the records. Running books buy quickly sides. Interesting|1.52|1.25|8006009|amalgedu pack #2|6|womens|4|Shoes|269|n stcallyable|small|4smoke16235200991144|plum|Gross|Unknown|50|n stableeinganti| +5830|AAAAAAAAGMGBAAAA|1997-10-27|1999-10-27|Teachers must not persuade high opportunities. Capable, neighbouring parents follow now to a days. Rather healthy letters must n|7.37|6.63|6002007|importocorp #7|2|diamonds|6|Jewelry|125|antiableought|N/A|46273sienna024801830|blanched|Tbl|Unknown|27|barprieinganti| +5831|AAAAAAAAGMGBAAAA|1999-10-28|2001-10-26|Winners can start very. Women stay unlikely names; double words ought to buy straight in a fathers. Both good bodies will attend. National others must not get reasonably in a conditions. Times |52.04|40.59|4003002|exportiedu pack #2|2|kids|4|Shoes|92|antiableought|petite|plum1001823428038709|light|Gram|Unknown|30|oughtprieinganti| +5832|AAAAAAAAGMGBAAAA|2001-10-27||Winners can start very. Women stay unlikely names; double words ought to buy straight in a fathers. Both good bodies will attend. National others must not get reasonably in a conditions. Times |2.56|1.81|4003002|corpbrand #1|16|consignment|6|Jewelry|403|antiableought|N/A|plum1001823428038709|saddle|Box|Unknown|23|ableprieinganti| +5833|AAAAAAAAJMGBAAAA|1997-10-27||Complete, head ways entail additional books; social letters drive perfect ends. Supporters should undermine therefore relat|4.15|3.69|9003002|exportimaxi #2|3|computers|9|Books|238|eingpriable|N/A|29494431light7540995|navy|Lb|Unknown|22|priprieinganti| +5834|AAAAAAAAKMGBAAAA|1997-10-27|2000-10-26|Big, full efforts may allocate long often new members. Inhabitants believe views. Little, level circumstances expand for a provisions. Criminal studies used to ra|0.66|0.22|4002001|importoedu pack #1|2|mens|4|Shoes|565|anticallyanti|petite|903451398honeydew872|powder|Box|Unknown|24|eseprieinganti| +5835|AAAAAAAAKMGBAAAA|2000-10-27||Big, full efforts may allocate long often new members. Inhabitants believe views. Little, level circumstances expand for a provisions. Criminal studies used to ra|2.01|1.40|4003002|exportiedu pack #2|2|kids|4|Shoes|565|anticallyanti|small|85023851547sky463481|moccasin|Each|Unknown|2|antiprieinganti| +5836|AAAAAAAAMMGBAAAA|1997-10-27|1999-10-27|New, living banks used to reduce soon open moves. Required, poor attacks would slow aware, full|3.47|1.94|2004001|edu packimporto #1|4|sports-apparel|2|Men|94|esen st|medium|8172757350799purple9|white|N/A|Unknown|1|callyprieinganti| +5837|AAAAAAAAMMGBAAAA|1999-10-28|2001-10-26|Once following respondents visit quite lips. Classical peasan|7.60|6.68|10010007|univamalgamalg #7|10|memory|10|Electronics|148|esen st|N/A|plum2714617761655517|sky|Carton|Unknown|20|ationprieinganti| +5838|AAAAAAAAMMGBAAAA|2001-10-27||Doubtful, inadequate sales consider equa|0.10|6.68|5004001|edu packscholar #1|4|classical|5|Music|146|esen st|N/A|plum2714617761655517|dodger|N/A|Unknown|9|eingprieinganti| +5839|AAAAAAAAPMGBAAAA|1997-10-27||Countries can create at least never progressive policies. Kind, common beans might find quite internal mothers. High loans should suit evident feet; differences prevent more necessary|4.56|3.78|5001002|amalgscholar #2|1|rock|5|Music|151|oughtantiought|N/A|643399royal088731660|rose|Dram|Unknown|13|n stprieinganti| +5840|AAAAAAAAANGBAAAA|1997-10-27|2000-10-26|Even national users put well methods. Delighted, good pp. will put authorities. Current waters say just consistent users. We|1.55|0.82|1001001|amalgamalg #1|1|dresses|1|Women|240|bareseable|petite|887677089maroon67867|salmon|Cup|Unknown|13|bareseeinganti| +5841|AAAAAAAAANGBAAAA|2000-10-27||Directions can get northern, jewish imports. Bright, various heads would think necessarily human quantities. Physical fees expect; far abilities|4.83|0.82|1001001|importoimporto #2|1|shirts|2|Men|240|bareseable|extra large|887677089maroon67867|royal|Lb|Unknown|15|oughteseeinganti| +5842|AAAAAAAACNGBAAAA|1997-10-27|1999-10-27|Much new musicians may not lean; winds keep very companies. True, certain priests smile. Other departments must s|5.15|2.47|3002001|importoexporti #1|2|infants|3|Children|131|oughtpriought|extra large|96dark52166117206816|purple|N/A|Unknown|45|ableeseeinganti| +5843|AAAAAAAACNGBAAAA|1999-10-28|2001-10-26|Much new musicians may not lean; winds keep very companies. True, certain priests smile. Other departments must s|1.44|2.47|3002001|exportiamalg #2|3|maternity|1|Women|131|oughtpriought|medium|96dark52166117206816|papaya|N/A|Unknown|26|prieseeinganti| +5844|AAAAAAAACNGBAAAA|2001-10-27||Much new musicians may not lean; winds keep very companies. True, certain priests smile. Other departments must s|5.32|2.47|3001001|amalgexporti #1|3|newborn|3|Children|208|oughtpriought|medium|95762645131linen8011|turquoise|Carton|Unknown|20|eseeseeinganti| +5845|AAAAAAAAFNGBAAAA|1997-10-27||Usually social eyes can get available, coastal things. Better social talks should accept as unique measures. Extreme|6.00|4.44|2002002|importoimporto #2|2|shirts|2|Men|47|ationese|large|78140218912754tan962|turquoise|Case|Unknown|46|antieseeinganti| +5846|AAAAAAAAGNGBAAAA|1997-10-27|2000-10-26|Brilliant, massive prisons take still national others. Only northern guidelines go right by the lips. General, spiritual walls shall reach in a languages. British nations eat substantial polici|3.42|1.67|9003003|exportimaxi #3|3|computers|9|Books|848|eingeseeing|N/A|saddle80913994360776|rose|N/A|Unknown|39|callyeseeinganti| +5847|AAAAAAAAGNGBAAAA|2000-10-27||Brilliant, massive prisons take still national others. Only northern guidelines go right by the lips. General, spiritual walls shall reach in a languages. British nations eat substantial polici|0.35|1.67|9003003|edu packamalg #2|3|swimwear|1|Women|848|eingeseeing|N/A|81361indian739542802|tan|Ounce|Unknown|14|ationeseeinganti| +5848|AAAAAAAAINGBAAAA|1997-10-27|1999-10-27|Earnings mind well foreign laboratories. Major letters get with the cases. Complete sides would get. Immediate, living pupils allow hands. Only regional services join ho|5.60|4.59|8005009|scholarnameless #9|5|fitness|8|Sports|490|barn stese|N/A|951868988092spring38|slate|Pallet|Unknown|52|eingeseeinganti| +5849|AAAAAAAAINGBAAAA|1999-10-28|2001-10-26|Sensibly wide spots put. Delighted interests see at an views. Typically unable partie|2.21|4.59|8005009|corpnameless #10|16|furniture|7|Home|490|barn stese|N/A|62thistle26257724778|lemon|Gross|Unknown|1|n steseeinganti| +5850|AAAAAAAAINGBAAAA|2001-10-27||Sensibly wide spots put. Delighted interests see at an views. Typically unable partie|4.43|4.59|8005009|exportiedu pack #1|3|kids|4|Shoes|490|barn stese|medium|8aquamarine997437011|papaya|Ton|Unknown|45|barantieinganti| +5851|AAAAAAAALNGBAAAA|1997-10-27||Levels will l|3.87|1.39|9007008|brandmaxi #8|7|reference|9|Books|594|esen stanti|N/A|sienna81771218210123|antique|Dram|Unknown|10|oughtantieinganti| +5852|AAAAAAAAMNGBAAAA|1997-10-27|2000-10-26|Less broad prisoners ought to help only to a parents. However particula|3.04|0.97|3002001|importoexporti #1|2|infants|3|Children|157|ationantiought|medium|76203195718white8508|royal|Each|Unknown|49|ableantieinganti| +5853|AAAAAAAAMNGBAAAA|2000-10-27||Personal, british yards read weak elections. Long crimes ought to hear. Very difficult applications offer on a others. Prime, sure points win by a|9.10|7.37|3002001|importoamalg #2|2|fragrances|1|Women|157|ationantiought|medium|76203195718white8508|plum|Ounce|Unknown|16|priantieinganti| +5854|AAAAAAAAONGBAAAA|1997-10-27|1999-10-27|Loans realise requirements. Full contracts will replace even sorry, ideal explanations. Crazy, major researc|9.46|5.48|9010011|univunivamalg #11|10|travel|9|Books|451|oughtantiese|N/A|269566yellow89754971|peru|Tsp|Unknown|73|eseantieinganti| +5855|AAAAAAAAONGBAAAA|1999-10-28|2001-10-26|Loans realise requirements. Full contracts will replace even sorry, ideal explanations. Crazy, major researc|2.46|5.48|9010011|edu packimporto #2|4|sports-apparel|2|Men|451|oughtantiese|N/A|269566yellow89754971|pink|Each|Unknown|28|antiantieinganti| +5856|AAAAAAAAONGBAAAA|2001-10-27||Tall, black products come a|1.06|5.48|5001001|amalgscholar #1|1|rock|5|Music|451|oughtantiese|N/A|269566yellow89754971|smoke|N/A|Unknown|31|callyantieinganti| +5857|AAAAAAAABOGBAAAA|1997-10-27||Then regular errors take that. Services get so. Both extraordinar|8.64|2.59|10015010|scholaramalgamalg #10|15|portable|10|Electronics|799|n stn station|N/A|2720316steel98148435|smoke|Ton|Unknown|6|ationantieinganti| +5858|AAAAAAAACOGBAAAA|1997-10-27|2000-10-26|Facil|8.46|2.96|2002001|importoimporto #1|2|shirts|2|Men|176|callyationought|petite|74898124074634pale05|sky|Cup|Unknown|9|eingantieinganti| +5859|AAAAAAAACOGBAAAA|2000-10-27||Married children used to obtain eastern, close abiliti|2.79|1.84|8006010|corpnameless #10|2|football|8|Sports|169|callyationought|N/A|067806honeydew341177|metallic|Gross|Unknown|33|n stantieinganti| +5860|AAAAAAAAEOGBAAAA|1997-10-27|1999-10-27|Then appropriate times|5.79|4.92|3004001|edu packexporti #1|4|school-uniforms|3|Children|176|callyationought|medium|6803477honeydew73603|violet|Tbl|Unknown|18|barcallyeinganti| +5861|AAAAAAAAEOGBAAAA|1999-10-28|2001-10-26|Then appropriate times|1.59|0.87|3004001|amalgbrand #6|4|semi-precious|6|Jewelry|501|oughtbaranti|N/A|6803477honeydew73603|black|N/A|Unknown|22|oughtcallyeinganti| +5862|AAAAAAAAEOGBAAAA|2001-10-27||Then appropriate times|3.97|0.87|3004001|corpunivamalg #15|4|musical|10|Electronics|501|oughtbaranti|N/A|6803477honeydew73603|coral|Bundle|Unknown|9|ablecallyeinganti| +5863|AAAAAAAAHOGBAAAA|1997-10-27||Standards may open both op|2.90|1.21|8012004|importomaxi #4|12|guns|8|Sports|187|ationeingought|N/A|97metallic2141683663|metallic|Pallet|Unknown|94|pricallyeinganti| +5864|AAAAAAAAIOGBAAAA|1997-10-27|2000-10-26|Even sexual operations could not bring internal, present bits; proper, safe cattle murder easily right paintings. Strong rules eat however labour numbers. Male brothers call so classes. Emissions must|2.12|0.69|2002001|importoimporto #1|2|shirts|2|Men|736|callypriation|large|4plum729386316014565|pink|Ounce|Unknown|20|esecallyeinganti| +5865|AAAAAAAAIOGBAAAA|2000-10-27||Even sexual operations could not bring internal, present bits; proper, safe cattle murder easily right paintings. Strong rules eat however labour numbers. Male brothers call so classes. Emissions must|3.52|2.85|4003002|exportiedu pack #2|2|kids|4|Shoes|286|callypriation|extra large|4plum729386316014565|turquoise|Oz|Unknown|47|anticallyeinganti| +5866|AAAAAAAAKOGBAAAA|1997-10-27|1999-10-27|Inadequate feet see wo|4.01|2.20|10003002|exportiunivamalg #2|3|dvd/vcr players|10|Electronics|603|pribarcally|N/A|4turquoise9438049365|white|Bunch|Unknown|36|callycallyeinganti| +5867|AAAAAAAAKOGBAAAA|1999-10-28|2001-10-26|Then strict movements could form in a problems.|3.10|1.58|7014008|edu packnameless #8|14|glassware|7|Home|990|pribarcally|N/A|653467986258thistle1|yellow|Oz|Unknown|19|ationcallyeinganti| +5868|AAAAAAAAKOGBAAAA|2001-10-27||Then strict movements could form in a problems.|9.72|1.58|7014008|importoimporto #1|2|shirts|2|Men|990|barn stn st|petite|653467986258thistle1|wheat|Bundle|Unknown|54|eingcallyeinganti| +5869|AAAAAAAANOGBAAAA|1997-10-27||Old, formidable charges force full, catholic clothes. Other arms may not perceive less ordinary periods. Gmt little authorities can |49.77|25.88|10002005|importounivamalg #5|2|camcorders|10|Electronics|477|ationationese|N/A|690sky58926493266071|sky|Pallet|Unknown|39|n stcallyeinganti| +5870|AAAAAAAAOOGBAAAA|1997-10-27|2000-10-26|Confident teeth give natural, dark directions. Complete, english members shall feel most. Then generous pp. |36.92|33.22|8012001|importomaxi #1|12|guns|8|Sports|408|eingbarese|N/A|36905208924puff38118|magenta|Unknown|Unknown|61|barationeinganti| +5871|AAAAAAAAOOGBAAAA|2000-10-27||Times drift steps. Respective, powerful reasons offer. Organic, large properties pull along a magazines. Leaders feed. Still new standards might not think too smooth general hospital|2.86|33.22|8012001|amalgimporto #2|1|accessories|2|Men|642|eingbarese|large|1636505879tan0004303|lawn|Unknown|Unknown|56|oughtationeinganti| +5872|AAAAAAAAAPGBAAAA|1997-10-27|1999-10-27|Local, ol|7.11|2.34|3004001|edu packexporti #1|4|school-uniforms|3|Children|81|oughteing|petite|13096759739sky556951|plum|Ounce|Unknown|44|ableationeinganti| +5873|AAAAAAAAAPGBAAAA|1999-10-28|2001-10-26|Local, ol|0.10|2.34|3004001|importoimporto #2|2|shirts|2|Men|81|oughteing|large|221162pale8356161366|navy|Tbl|Unknown|20|priationeinganti| +5874|AAAAAAAAAPGBAAAA|2001-10-27||Local, ol|4.82|2.34|8012005|importomaxi #5|2|guns|8|Sports|556|oughteing|N/A|6186638steel91785288|sandy|Dram|Unknown|39|eseationeinganti| +5875|AAAAAAAADPGBAAAA|1997-10-27||Likely, specific thousands can enjoy common, important forces. Green, useful regions shall continue losses. Leading, reasonable children develop at least. Good, long materials might not improve p|1.97|1.47|||13|||||ablepri|N/A|2585629645105royal64|papaya||Unknown||antiationeinganti| +5876|AAAAAAAAEPGBAAAA|1997-10-27|2000-10-26|Sick, contemporary stores extend useful, wrong circumstances. Men used to make all more encouraging accounts. Valuable members ought|2.36|2.02|2002001|importoimporto #1|2|shirts|2|Men|44|eseese|large|5807437tomato4855133|goldenrod|Gram|Unknown|72|callyationeinganti| +5877|AAAAAAAAEPGBAAAA|2000-10-27||Main, early |3.43|2.60|1003002|exportiamalg #2|2|maternity|1|Women|48|eseese|large|2324red0558250061035|tan|Pallet|Unknown|21|ationationeinganti| +5878|AAAAAAAAGPGBAAAA|1997-10-27|1999-10-27|Medical, strong doctors say impossible, bad cars. Fair, old communities will see too together with a patterns; projects support large months; offices get especial|6.74|3.16|4002001|importoedu pack #1|2|mens|4|Shoes|765|anticallyation|petite|97spring014301185615|royal|Case|Unknown|14|eingationeinganti| +5879|AAAAAAAAGPGBAAAA|1999-10-28|2001-10-26|Medical, strong doctors say impossible, bad cars. Fair, old communities will see too together with a patterns; projects support large months; offices get especial|5.60|3.16|7014010|edu packnameless #10|2|glassware|7|Home|765|anticallyation|N/A|97spring014301185615|plum|Pound|Unknown|14|n stationeinganti| +5880|AAAAAAAAGPGBAAAA|2001-10-27||National clubs finance approx improvements. Concerned days might not take forces. Dry, american rules know enough banks. Hours ought to remember never stars; police e|9.96|4.08|9009001|maximaxi #1|2|science|9|Books|765|anticallyation|N/A|97spring014301185615|almond|Oz|Unknown|18|bareingeinganti| +5881|AAAAAAAAJPGBAAAA|1997-10-27||Economic, assistant unions play formerly all lacking times. Free operations suggest beautiful, old facilities. Odd, wide findings keep intentionally far ty|9.82|5.99|4004002|edu packedu pack #2|4|athletic|4|Shoes|287|ationeingable|medium|79sandy6613903539412|steel|Dram|Unknown|71|oughteingeinganti| +5882|AAAAAAAAKPGBAAAA|1997-10-27|2000-10-26|Judges will not counter clean, chief statements. Relatives see;|4.10|3.11|5002001|importoscholar #1|2|country|5|Music|704|esebaration|N/A|86962red017562532586|violet|Bunch|Unknown|17|ableeingeinganti| +5883|AAAAAAAAKPGBAAAA|2000-10-27||Amazing, victorian materials let immensely years; well happy systems must not see undoubtedly within a days. Major, physical parents must imply small, different parents. Easy teeth me|9.42|3.11|5002001|importomaxi #4|2|business|9|Books|189|n steingought|N/A|41116papaya216563413|tomato|Tbl|Unknown|3|prieingeinganti| +5884|AAAAAAAAMPGBAAAA|1997-10-27|1999-10-27|Phases would sell scarcely. Seats work here secret variations. Reports order no|35.49|11.35|7008005|namelessbrand #5|8|lighting|7|Home|375|antiationpri|N/A|738241slate403539926|misty|Case|Unknown|17|eseeingeinganti| +5885|AAAAAAAAMPGBAAAA|1999-10-28|2001-10-26|Phases would sell scarcely. Seats work here secret variations. Reports order no|8.07|11.35|7008005|corpcorp #6|8|rings|6|Jewelry|806|callybareing|N/A|738241slate403539926|floral|Ton|Unknown|9|antieingeinganti| +5886|AAAAAAAAMPGBAAAA|2001-10-27||Phases would sell scarcely. Seats work here secret variations. Reports order no|0.36|11.35|7008005|amalgamalg #1|1|dresses|1|Women|806|callybareing|small|738241slate403539926|firebrick|Tbl|Unknown|4|callyeingeinganti| +5887|AAAAAAAAPPGBAAAA|1997-10-27||Much rapid expenses used to terminate. Lands used to achieve just about a seeds. Powers indic|6.96|2.22|8015008|scholarmaxi #8|15|fishing|8|Sports|570|barationanti|N/A|68454slate2753676838|gainsboro|Bundle|Unknown|20|ationeingeinganti| +5888|AAAAAAAAAAHBAAAA|1997-10-27|2000-10-26|Od|6.56|3.28|7005009|scholarbrand #9|5|blinds/shades|7|Home|99|n stn st|N/A|41161023907tan750432|powder|Gross|Unknown|31|eingeingeinganti| +5889|AAAAAAAAAAHBAAAA|2000-10-27||Successful, familiar questions should tea|7.72|2.77|7005009|exportiimporto #2|5|pants|2|Men|518|n stn st|medium|41161023907tan750432|mint|Dram|Unknown|43|n steingeinganti| +5890|AAAAAAAACAHBAAAA|1997-10-27|1999-10-27|Otherwise great policies would find indoors wrong, responsible photographs. Weeks may lose villages. Fine bits will derive actually systems. Now silver r|43.77|35.01|5001001|amalgscholar #1|1|rock|5|Music|523|priableanti|N/A|68902908white5777512|purple|Unknown|Unknown|14|barn steinganti| +5891|AAAAAAAACAHBAAAA|1999-10-28|2001-10-26|Overwhelmingly british boots increase ago women. Good activities might stand either changes|8.33|35.01|5001001|corpcorp #4|1|rings|6|Jewelry|523|priableanti|N/A|0055648425199119tan5|black|Tsp|Unknown|50|oughtn steinganti| +5892|AAAAAAAACAHBAAAA|2001-10-27||Liberal factors rank effectively drawings. Following, married combinations should depend b|0.90|35.01|1003001|exportiamalg #1|1|maternity|1|Women|302|ablebarpri|medium|462248627puff0382189|seashell|Lb|Unknown|18|ablen steinganti| +5893|AAAAAAAAFAHBAAAA|1997-10-27||Awful, hostile arrangements ought to surprise little, very studies. New numbers might mind almost guests. Units may speak |3.95|1.54|3003002|exportiexporti #2|3|toddlers|3|Children|164|esecallyought|medium|195green949465456752|pink|Lb|Unknown|76|prin steinganti| +5894|AAAAAAAAGAHBAAAA|1997-10-27|2000-10-26|Please open guidelines could not change above others; more p|4.17|3.62|6003007|exporticorp #7|3|gold|6|Jewelry|511|oughtoughtanti|N/A|60159517560388white8|purple|Carton|Unknown|44|esen steinganti| +5895|AAAAAAAAGAHBAAAA|2000-10-27||Please open guidelines could not change above others; more p|3.73|1.86|6003007|importoedu pack #2|3|mens|4|Shoes|303|oughtoughtanti|medium|60159517560388white8|pink|Unknown|Unknown|24|antin steinganti| +5896|AAAAAAAAIAHBAAAA|1997-10-27|1999-10-27|Sentences|2.88|2.07|10007015|brandunivamalg #15|7|personal|10|Electronics|16|callyought|N/A|1075492598peru766770|black|Bundle|Unknown|2|callyn steinganti| +5897|AAAAAAAAIAHBAAAA|1999-10-28|2001-10-26|Sentences|6.27|4.95|10007015|maxibrand #8|9|mattresses|7|Home|16|callyought|N/A|1075492598peru766770|violet|N/A|Unknown|24|ationn steinganti| +5898|AAAAAAAAIAHBAAAA|2001-10-27||Sentences|4.74|4.95|10007015|amalgcorp #3|1|birdal|6|Jewelry|315|callyought|N/A|55pink42577812060851|metallic|Ounce|Unknown|93|eingn steinganti| +5899|AAAAAAAALAHBAAAA|1997-10-27||Tickets will participate miles. English children shall occur. Surroundings create in order again level flowers; cases get constantly in addition different|4.44|1.33|10010001|univamalgamalg #1|10|memory|10|Electronics|31|oughtpri|N/A|4438khaki55261523698|peru|Dram|Unknown|57|n stn steinganti| +5900|AAAAAAAAMAHBAAAA|1997-10-27|2000-10-26|Warm workers should supply now unlike years. Tot|78.55|33.77|5003001|exportischolar #1|3|pop|5|Music|472|ableationese|N/A|115866666pale3474764|tan|Lb|Unknown|49|barbarn stanti| +5901|AAAAAAAAMAHBAAAA|2000-10-27||Warm workers should supply now unlike years. Tot|81.74|33.77|2002002|importoimporto #2|2|shirts|2|Men|169|n stcallyought|large|115866666pale3474764|royal|Ounce|Unknown|4|oughtbarn stanti| +5902|AAAAAAAAOAHBAAAA|1997-10-27|1999-10-27|Approx electrical events could not want exciting relations; years can seem afte|8.58|6.43|2003001|exportiimporto #1|3|pants|2|Men|262|ablecallyable|small|827274turquoise21243|yellow|Oz|Unknown|5|ablebarn stanti| +5903|AAAAAAAAOAHBAAAA|1999-10-28|2001-10-26|Approx electrical events could not want exciting relations; years can seem afte|8.60|6.43|2003001|brandunivamalg #5|7|personal|10|Electronics|262|ablecallyable|N/A|8169766107817light23|gainsboro|Unknown|Unknown|52|pribarn stanti| +5904|AAAAAAAAOAHBAAAA|2001-10-27||Clouds alter gastric activities. Free, n|4.46|6.43|4003001|exportiedu pack #1|3|kids|4|Shoes|349|ablecallyable|medium|8169766107817light23|peru|Box|Unknown|9|esebarn stanti| +5905|AAAAAAAABBHBAAAA|1997-10-27||Local circumstances will match; numerous, representative signs travel also home local computers. Annual features could not preserve about by a delegates. Too male assets r|7.88|5.12|1003002|exportiamalg #2|3|maternity|1|Women|577|ationationanti|medium|99289742lime99299037|plum|Tsp|Unknown|1|antibarn stanti| +5906|AAAAAAAACBHBAAAA|1997-10-27|2000-10-26|Even necessary fees would ta|2.39|1.98|10015008|scholaramalgamalg #8|15|portable|10|Electronics|511|oughtoughtanti|N/A|979antique7756907525|lemon|Dram|Unknown|15|callybarn stanti| +5907|AAAAAAAACBHBAAAA|2000-10-27||Chief, particular forces sit. Practical hours ring political workers. Very acute costs shall say at least now federal others. Early parties can get |9.73|1.98|5003002|exportischolar #2|3|pop|5|Music|199|n stn stought|N/A|979antique7756907525|peru|Cup|Unknown|46|ationbarn stanti| +5908|AAAAAAAAEBHBAAAA|1997-10-27|1999-10-27|Always old incentives mitigate again late, political loans. Wines believe only international workers. Thin|2.75|2.09|5004001|edu packscholar #1|4|classical|5|Music|164|esecallyought|N/A|7589133482489steel33|pale|Dram|Unknown|81|eingbarn stanti| +5909|AAAAAAAAEBHBAAAA|1999-10-28|2001-10-26|Old muscles might become commonly mass opportunities. Hills wait only positive, profitable tables;|5.42|4.87|5004001|amalgexporti #2|4|newborn|3|Children|164|esecallyought|small|63578922tan275600371|green|Oz|Unknown|22|n stbarn stanti| +5910|AAAAAAAAEBHBAAAA|2001-10-27||National, da|9.50|7.03|10015012|scholaramalgamalg #12|4|portable|10|Electronics|164|esecallyought|N/A|63578922tan275600371|olive|Unknown|Unknown|3|baroughtn stanti| +5911|AAAAAAAAHBHBAAAA|1997-10-27||Failures measu|4.15|2.78|5004002|edu packscholar #2|4|classical|5|Music|236|callypriable|N/A|3992071pale521022209|yellow|Bunch|Unknown|11|oughtoughtn stanti| +5912|AAAAAAAAIBHBAAAA|1997-10-27|2000-10-26|Please colourful goals ask now mild bones. Low active stairs conduct hands. Only chief subjects would continue others. Remaining goals would drink. Markets may think. Dangerous leade|47.70|25.75|10011014|amalgamalgamalg #14|11|disk drives|10|Electronics|191|oughtn stought|N/A|3682234812425orange8|spring|Bundle|Unknown|58|ableoughtn stanti| +5913|AAAAAAAAIBHBAAAA|2000-10-27||Please colourful goals ask now mild bones. Low active stairs conduct hands. Only chief subjects would continue others. Remaining goals would drink. Markets may think. Dangerous leade|7.61|2.28|10011014|amalgedu pack #2|1|womens|4|Shoes|383|oughtn stought|extra large|3682234812425orange8|honeydew|Ton|Unknown|7|prioughtn stanti| +5914|AAAAAAAAKBHBAAAA|1997-10-27|1999-10-27|National stages get only eager forms. Most bad eyes will not get by no m|2.86|1.37|8003005|exportinameless #5|3|basketball|8|Sports|716|callyoughtation|N/A|36879672red098741450|royal|Ounce|Unknown|21|eseoughtn stanti| +5915|AAAAAAAAKBHBAAAA|1999-10-28|2001-10-26|Marginal, precio|3.65|1.37|8003005|scholarnameless #6|3|fitness|8|Sports|565|anticallyanti|N/A|36879672red098741450|turquoise|Case|Unknown|10|antioughtn stanti| +5916|AAAAAAAAKBHBAAAA|2001-10-27||Once more commercial guns pour nevertheless months. Experiences discuss particularly southern negotiations; correctly applicable ideas could not comment in the hands. Fa|1.06|0.86|8003005|importoscholar #1|2|country|5|Music|565|anticallyanti|N/A|36879672red098741450|linen|Unknown|Unknown|29|callyoughtn stanti| +5917|AAAAAAAANBHBAAAA|1997-10-27||Women may speak upon a |3.17|1.55|4002002|importoedu pack #2|2|mens|4|Shoes|835|antiprieing|medium|059823614pale4719546|pink|Case|Unknown|86|ationoughtn stanti| +5918|AAAAAAAAOBHBAAAA|1997-10-27|2000-10-26|Teachers achieve again sheer women; able, good matters ought to see consequently well different men. Forms work so high kilometres. Incredibly variable respondents must not stick merely common|7.18|3.01|6010007|univbrand #7|10|jewelry boxes|6|Jewelry|68|eingcally|N/A|9light60627745064969|turquoise|Case|Unknown|21|eingoughtn stanti| +5919|AAAAAAAAOBHBAAAA|2000-10-27||Programmes present also available, perfect schools. Chief police may see then. Sufficient, fair governments would direct never in no lights. Initial, new walls |5.56|3.01|8007010|brandnameless #10|10|hockey|8|Sports|142|eingcally|N/A|9light60627745064969|navajo|Carton|Unknown|30|n stoughtn stanti| +5920|AAAAAAAAACHBAAAA|1997-10-27|1999-10-27|Jobs provide quite. Major increases should not soften clear, select shoes. Final months take computers. Military years draw other friends. Too alone firms believe conditions. Signs give fixed |1.98|1.56|10007005|brandunivamalg #5|7|personal|10|Electronics|635|antiprically|N/A|75018indian017469683|navy|Bundle|Unknown|12|barablen stanti| +5921|AAAAAAAAACHBAAAA|1999-10-28|2001-10-26|Jobs provide quite. Major increases should not soften clear, select shoes. Final months take computers. Military years draw other friends. Too alone firms believe conditions. Signs give fixed |6.94|1.56|6007002|brandcorp #2|7|pendants|6|Jewelry|428|eingableese|N/A|75018indian017469683|sienna|Lb|Unknown|63|oughtablen stanti| +5922|AAAAAAAAACHBAAAA|2001-10-27||Jobs provide quite. Major increases should not soften clear, select shoes. Final months take computers. Military years draw other friends. Too alone firms believe conditions. Signs give fixed |3.03|1.56|6015003|scholarbrand #3|15|custom|6|Jewelry|428|eingableese|N/A|75018indian017469683|red|Gram|Unknown|24|ableablen stanti| +5923|AAAAAAAADCHBAAAA|1997-10-27||Usually dark centres should not give |1.68|0.90|4001002|amalgedu pack #2|1|womens|4|Shoes|216|callyoughtable|economy|506gainsboro66418965|plum|Gram|Unknown|24|priablen stanti| +5924|AAAAAAAAECHBAAAA|1997-10-27|2000-10-26|Numbers must tick. New, pleased methods should not meet very bags. Reasonable, assistant officials will raise small tories. Significant, environmental duties expose possibly for a prizes. On|3.66|1.46|6002005|importocorp #5|2|diamonds|6|Jewelry|315|antioughtpri|N/A|156738318753860pink9|grey|Pallet|Unknown|6|eseablen stanti| +5925|AAAAAAAAECHBAAAA|2000-10-27||Numbers must tick. New, pleased methods should not meet very bags. Reasonable, assistant officials will raise small tories. Significant, environmental duties expose possibly for a prizes. On|4.65|3.62|8002006|importonameless #6|2|baseball|8|Sports|213|prioughtable|N/A|156738318753860pink9|slate|Ounce|Unknown|20|antiablen stanti| +5926|AAAAAAAAGCHBAAAA|1997-10-27|1999-10-27|Here other tensions come subsequent terms. Too different friends discuss goods. Always long eyes act outside in a projects. Easy solid years sell most single champions; monthly, specific fee|3.42|2.35|5002001|importoscholar #1|2|country|5|Music|552|ableantianti|N/A|381tomato37312445584|khaki|N/A|Unknown|26|callyablen stanti| +5927|AAAAAAAAGCHBAAAA|1999-10-28|2001-10-26|Here other tensions come subsequent terms. Too different friends discuss goods. Always long eyes act outside in a projects. Easy solid years sell most single champions; monthly, specific fee|8.13|3.65|7005008|scholarbrand #8|5|blinds/shades|7|Home|18|eingought|N/A|381tomato37312445584|sienna|Cup|Unknown|99|ationablen stanti| +5928|AAAAAAAAGCHBAAAA|2001-10-27||Here other tensions come subsequent terms. Too different friends discuss goods. Always long eyes act outside in a projects. Easy solid years sell most single champions; monthly, specific fee|4.77|3.65|2002001|importoimporto #1|2|shirts|2|Men|781|oughteingation|medium|381tomato37312445584|peach|Gram|Unknown|21|eingablen stanti| +5929|AAAAAAAAJCHBAAAA|1997-10-27||Clear advantages see. Phases think good, long-term attempts. Urgently good futures would not supply softly for a fees; other, appropriate records should create now major children. B|3.31|2.28|2003002|exportiimporto #2|3|pants|2|Men|12|ableought|large|83945lime47185655326|dim|Dram|Unknown|61|n stablen stanti| +5930|AAAAAAAAKCHBAAAA|1997-10-27|2000-10-26|Welsh members must ac|5.22|2.66|1003001|exportiamalg #1|3|maternity|1|Women|415|antioughtese|extra large|804rosy0359429417148|royal|Tbl|Unknown|56|barprin stanti| +5931|AAAAAAAAKCHBAAAA|2000-10-27||However broken pp. slip rightly as good un|9.10|2.66|2003002|exportiimporto #2|3|pants|2|Men|342|ableesepri|medium|5596627425333peach43|maroon|Unknown|Unknown|13|oughtprin stanti| +5932|AAAAAAAAMCHBAAAA|1997-10-27|1999-10-27|Ago low signs cannot account only successfully available solutions. Medical, overseas terms s|1.95|1.63|8009003|maxinameless #3|9|optics|8|Sports|148|eingeseought|N/A|6273994223274380sky3|navajo|Tsp|Unknown|19|ableprin stanti| +5933|AAAAAAAAMCHBAAAA|1999-10-28|2001-10-26|Nuclear, limited police could not find most below fundamental months; bombs could indicate tall, ready hours; human, global rights re|1.51|1.32|8009003|exportischolar #2|3|pop|5|Music|148|eingeseought|N/A|6273994223274380sky3|linen|Pallet|Unknown|34|priprin stanti| +5934|AAAAAAAAMCHBAAAA|2001-10-27||Nuclear, limited police could not find most below fundamental months; bombs could indicate tall, ready hours; human, global rights re|2.09|0.94|8009003|edu packamalg #1|3|swimwear|1|Women|267|ationcallyable|large|2380728215medium4336|slate|Unknown|Unknown|62|eseprin stanti| +5935|AAAAAAAAPCHBAAAA|1997-10-27||Strong memb|6.63|2.58|8012006|importomaxi #6|12|guns|8|Sports|297|ationn stable|N/A|73948523107seashell9|thistle|Pound|Unknown|21|antiprin stanti| +5936|AAAAAAAAADHBAAAA|1997-10-27|2000-10-26|Ears can think on the pockets. Necessary, essential men used to earn only small developments. Ill, nuclear teeth must welcome brief, equal clubs. Officials shall necessitate so ancient|9.18|8.07|3004001|edu packexporti #1|4|school-uniforms|3|Children|518|eingoughtanti|N/A|87olive1222457518500|azure|Pound|Unknown|15|callyprin stanti| +5937|AAAAAAAAADHBAAAA|2000-10-27||Ears can think on the pockets. Necessary, essential men used to earn only small developments. Ill, nuclear teeth must welcome brief, equal clubs. Officials shall necessitate so ancient|2.11|0.90|3004001|scholarmaxi #8|15|fishing|8|Sports|622|eingoughtanti|N/A|87olive1222457518500|rose|Bundle|Unknown|33|ationprin stanti| +5938|AAAAAAAACDHBAAAA|1997-10-27|1999-10-27|Dogs may make just local boats. Public, industrial pp. may not provide r|5.60|2.40|4004001|edu packedu pack #1|4|athletic|4|Shoes|112|ableoughtought|small|939031chartreuse6752|snow|Cup|Unknown|17|eingprin stanti| +5939|AAAAAAAACDHBAAAA|1999-10-28|2001-10-26|Dogs may make just local boats. Public, industrial pp. may not provide r|30.54|25.65|7001010|amalgbrand #10|1|bathroom|7|Home|112|ableoughtought|N/A|30251044224936wheat6|plum|Tsp|Unknown|50|n stprin stanti| +5940|AAAAAAAACDHBAAAA|2001-10-27||Dogs may make just local boats. Public, industrial pp. may not provide r|2.12|1.50|7001010|amalgmaxi #1|11|archery|8|Sports|715|ableoughtought|N/A|30251044224936wheat6|burlywood|Box|Unknown|10|baresen stanti| +5941|AAAAAAAAFDHBAAAA|1997-10-27||Ages see both to an supporters. Creative sides will not make always. Groups grow therefore expensive talks. Apparent citizens survive across new, single minutes; previous, dark rivers qualify.|7.04|5.98|9006002|corpmaxi #2|6|parenting|9|Books|47|ationese|N/A|294951783royal657268|sandy|Cup|Unknown|5|oughtesen stanti| +5942|AAAAAAAAGDHBAAAA|1997-10-27|2000-10-26|Tonight english schools result presumably afraid, keen parts. Potential villages must accommodate in a fig|81.95|46.71|5004001|edu packscholar #1|4|classical|5|Music|436|callypriese|N/A|5dark347143031551758|slate|Ounce|Unknown|33|ableesen stanti| +5943|AAAAAAAAGDHBAAAA|2000-10-27||Workshops can tend completely competitive officers. Personal, certain troops cannot let soon for a goals; hours cause; waves justify. Still other systems might comment on the|2.94|1.35|5004001|importocorp #8|2|diamonds|6|Jewelry|436|callypriese|N/A|52989violet982319237|tan|Unknown|Unknown|29|priesen stanti| +5944|AAAAAAAAIDHBAAAA|1997-10-27|1999-10-27|Precise techniques must not take. Orange, illegal issues pay more new issues. Only positive rights might not fast perhaps difficult hot agents. Police |2.21|1.08|10005012|scholarunivamalg #12|5|karoke|10|Electronics|21|oughtable|N/A|80green3348407640391|purple|Dram|Unknown|13|eseesen stanti| +5945|AAAAAAAAIDHBAAAA|1999-10-28|2001-10-26|Over severe men m|7.47|5.90|10005012|scholarunivamalg #6|15|fiction|9|Books|21|oughtable|N/A|80green3348407640391|orchid|Case|Unknown|2|antiesen stanti| +5946|AAAAAAAAIDHBAAAA|2001-10-27||Civil shareholders include supposedly political shows. Responsible, new knees deliver hard excellent values. Positive lines permit muc|2.56|5.90|10005012|amalgscholar #1|15|rock|5|Music|21|oughtable|N/A|80green3348407640391|white|Bundle|Unknown|8|callyesen stanti| +5947|AAAAAAAALDHBAAAA|1997-10-27||Clear, administrative stages mind hundreds; shares could imply here; further important suppliers should perceive |4.90|1.66|3002002|importoexporti #2|2|infants|3|Children|43|priese|petite|4443plum352193143257|lace|Tsp|Unknown|9|ationesen stanti| +5948|AAAAAAAAMDHBAAAA|1997-10-27|2000-10-26|Generally slight services used to reach earnings. Almost other schemes may not carry someti|0.39|0.12|6008005|namelesscorp #5|8|mens watch|6|Jewelry|243|prieseable|N/A|1saddle4961637707857|goldenrod|Gross|Unknown|9|eingesen stanti| +5949|AAAAAAAAMDHBAAAA|2000-10-27||Good, safe conditions cannot recover governments. More concerned levels defend still big periods. Much basic exports will not express too about loca|4.05|3.19|5001002|amalgscholar #2|1|rock|5|Music|409|n stbarese|N/A|12511895238193puff57|sky|Pallet|Unknown|4|n stesen stanti| +5950|AAAAAAAAODHBAAAA|1997-10-27|1999-10-27|Right, linguistic interests like outside. Dogs could call newly at a years. Hard products could contemplate other, centra|67.47|42.50|4004001|edu packedu pack #1|4|athletic|4|Shoes|88|eingeing|small|82708006frosted47246|indian|Case|Unknown|17|barantin stanti| +5951|AAAAAAAAODHBAAAA|1999-10-28|2001-10-26|Children must consider private, arab interests. Unusual, concerned arms go |0.09|42.50|4004001|importoscholar #2|4|country|5|Music|559|n stantianti|N/A|7324250burlywood7406|pink|Box|Unknown|13|oughtantin stanti| +5952|AAAAAAAAODHBAAAA|2001-10-27||Complete problems sell at least customs. Most evolutionary fires could indulge only to a minutes. Efficient minutes monitor then local consequenc|2.57|1.28|1001001|amalgamalg #1|1|dresses|1|Women|189|n stantianti|medium|7324250burlywood7406|thistle|Oz|Unknown|44|ableantin stanti| +5953|AAAAAAAABEHBAAAA|1997-10-27||Steep, labour clubs achieve less hands; often great towns mean tall, new maps. Conditions occur following men. Costs should coordinate; objectives know modest details. Child|2.13|1.42|9007002|brandmaxi #2|7|reference|9|Books|207|ationbarable|N/A|470352purple03253752|goldenrod|Bunch|Unknown|70|priantin stanti| +5954|AAAAAAAACEHBAAAA|1997-10-27|2000-10-26|International, other possibilities might remain reliably far british doors. Good plants will not encourage forwards sometimes great pieces. Wrong, c|0.85|0.71|8005005|scholarnameless #5|5|fitness|8|Sports|371|oughtationpri|N/A|866682903128505pale4|sienna|N/A|Unknown|32|eseantin stanti| +5955|AAAAAAAACEHBAAAA|2000-10-27||Programmes shall fix. Types would get still neither real rates. Head, general makers should give public, advanced changes. Women might hang outside live, careful |1.99|1.27|3002002|importoexporti #2|5|infants|3|Children|916|oughtationpri|small|866682903128505pale4|wheat|Pallet|Unknown|30|antiantin stanti| +5956|AAAAAAAAEEHBAAAA|1997-10-27|1999-10-27|Generally communist workers ought to speak to a quantities. Male, english decades take. Explanations retain comparatively large, enormous patterns. Mediterranean budget|5.73|4.87|9006005|corpmaxi #5|6|parenting|9|Books|378|eingationpri|N/A|4531052light77777517|lime|Ton|Unknown|38|callyantin stanti| +5957|AAAAAAAAEEHBAAAA|1999-10-28|2001-10-26|Generally communist workers ought to speak to a quantities. Male, english decades take. Explanations retain comparatively large, enormous patterns. Mediterranean budget|0.48|4.87|9006005|exportiedu pack #2|6|kids|4|Shoes|378|eingationpri|petite|03655tomato406863136|indian|N/A|Unknown|40|ationantin stanti| +5958|AAAAAAAAEEHBAAAA|2001-10-27||New benefits oppose events. Low|5.67|4.87|9006005|maxibrand #9|9|mattresses|7|Home|497|ationn stese|N/A|418796585925957peru9|aquamarine|Cup|Unknown|34|eingantin stanti| +5959|AAAAAAAAHEHBAAAA|1997-10-27||Payments make imperial sources. Gmt left pensions would not come moreover new public terms; certain teachers may rest finally; certain flowers used to look. Friendly friends must conv|3.86|1.77|9010008|univunivamalg #8|10|travel|9|Books|192|ablen stought|N/A|tomato91789760783913|white|Case|Unknown|49|n stantin stanti| +5960|AAAAAAAAIEHBAAAA|1997-10-27|2000-10-26|Just great elements prepare again sharp, able influences. Boys will relate great, single questions; difficult, com|0.79|0.40|5003001|exportischolar #1|3|pop|5|Music|448|eingeseese|N/A|steel561047166226111|snow|Carton|Unknown|18|barcallyn stanti| +5961|AAAAAAAAIEHBAAAA|2000-10-27||Just great elements prepare again sharp, able influences. Boys will relate great, single questions; difficult, com|1.54|0.90|5003001|edu packedu pack #2|3|athletic|4|Shoes|317|ationoughtpri|petite|1smoke45901216877661|thistle|Tsp|Unknown|89|oughtcallyn stanti| +5962|AAAAAAAAKEHBAAAA|1997-10-27|1999-10-27|Crowds could belong more far characters. Afraid, major hands may|2.76|1.84|5002001|importoscholar #1|2|country|5|Music|12|ableought|N/A|6tan6160276598402213|slate|Unknown|Unknown|25|ablecallyn stanti| +5963|AAAAAAAAKEHBAAAA|1999-10-28|2001-10-26|Very net plans shall kill then medical, medical artists. Usual, general emotions remember please contributions. So profitable skills shall not inher|5.04|1.84|10004014|edu packunivamalg #14|2|audio|10|Electronics|12|ableought|N/A|6tan6160276598402213|salmon|Cup|Unknown|3|pricallyn stanti| +5964|AAAAAAAAKEHBAAAA|2001-10-27||Only interested values say at|7.80|1.84|2004001|edu packimporto #1|4|sports-apparel|2|Men|191|ableought|large|6tan6160276598402213|mint|Dram|Unknown|33|esecallyn stanti| +5965|AAAAAAAANEHBAAAA|1997-10-27||Too sharp rates look recently secondly international arrangements. Events survive thus main numbers. Police shall ask home in the others. Groups shall sense somewhat to a method|35.22|31.69|2003002|exportiimporto #2|3|pants|2|Men|209|n stbarable|medium|8goldenrod7089304883|thistle|Dram|Unknown|26|anticallyn stanti| +5966|AAAAAAAAOEHBAAAA|1997-10-27|2000-10-26|Different characteristics would leave short, complex situations; inner relations assure poor experiences;|2.53|2.15|1002001|importoamalg #1|2|fragrances|1|Women|148|eingeseought|small|9541cornflower218332|maroon|Unknown|Unknown|15|callycallyn stanti| +5967|AAAAAAAAOEHBAAAA|2000-10-27||Statutory, extra men matter to a conditions. European words can move then necessarily international leaves. So asleep words should anger. Already individual points find now human|6.35|3.17|5004002|edu packscholar #2|2|classical|5|Music|148|eingeseought|N/A|floral48097214121629|saddle|Case|Unknown|30|ationcallyn stanti| +5968|AAAAAAAAAFHBAAAA|1997-10-27|1999-10-27|Only necessary occasions subdue similarly for example political metres. Values shut then countries. Loudly basic profits would arise mentally apparent rooms; eyes may know anywhere views. Approx fu|4.10|3.32|7015009|scholarnameless #9|15|tables|7|Home|312|ableoughtpri|N/A|5812892728766pink338|saddle|N/A|Unknown|44|eingcallyn stanti| +5969|AAAAAAAAAFHBAAAA|1999-10-28|2001-10-26|Only necessary occasions subdue similarly for example political metres. Values shut then countries. Loudly basic profits would arise mentally apparent rooms; eyes may know anywhere views. Approx fu|4.68|2.38|7015009|amalgimporto #2|1|accessories|2|Men|143|ableoughtpri|N/A|683452wheat049365518|goldenrod|Dram|Unknown|3|n stcallyn stanti| +5970|AAAAAAAAAFHBAAAA|2001-10-27||Only necessary occasions subdue similarly for example political metres. Values shut then countries. Loudly basic profits would arise mentally apparent rooms; eyes may know anywhere views. Approx fu|4.25|2.38|6013003|exportibrand #3|13|loose stones|6|Jewelry|143|prieseought|N/A|06483907607spring553|peru|Dozen|Unknown|47|barationn stanti| +5971|AAAAAAAADFHBAAAA|1997-10-27||Good, inherent wings possess especially others; |2.33|1.72|10013005|exportiamalgamalg #5|13|stereo|10|Electronics|177|ationationought|N/A|violet53956230522477|wheat|Tbl|Unknown|14|oughtationn stanti| +5972|AAAAAAAAEFHBAAAA|1997-10-27|2000-10-26|Components edit however with a nurses. National words shall not stress still equal forms. Western writers would restrict only buildings. More intense historians gather authorities. Real, importa|6.47|3.75|2004001|edu packimporto #1|4|sports-apparel|2|Men|487|ationeingese|petite|62316716yellow812110|papaya|Carton|Unknown|45|ableationn stanti| +5973|AAAAAAAAEFHBAAAA|2000-10-27||Components edit however with a nurses. National words shall not stress still equal forms. Western writers would restrict only buildings. More intense historians gather authorities. Real, importa|4.51|3.75|7005004|scholarbrand #4|5|blinds/shades|7|Home|455|ationeingese|N/A|62316716yellow812110|plum|Box|Unknown|11|priationn stanti| +5974|AAAAAAAAGFHBAAAA|1997-10-27|1999-10-27|Contemporary occasions provide she|1.75|0.94|9001011|amalgmaxi #11|1|arts|9|Books|283|prieingable|N/A|8312997023744salmon0|royal|Lb|Unknown|46|eseationn stanti| +5975|AAAAAAAAGFHBAAAA|1999-10-28|2001-10-26|As secondary politicians must safeguard. Particular, light teams will avoid about. Evident, responsible approaches refer also appear|3.92|3.13|1003002|exportiamalg #2|1|maternity|1|Women|283|prieingable|petite|8312997023744salmon0|pink|Oz|Unknown|43|antiationn stanti| +5976|AAAAAAAAGFHBAAAA|2001-10-27||As secondary politicians must safeguard. Particular, light teams will avoid about. Evident, responsible approaches refer also appear|6.81|3.13|9004001|edu packmaxi #1|4|entertainments|9|Books|283|prieingable|N/A|8312997023744salmon0|lace|Oz|Unknown|36|callyationn stanti| +5977|AAAAAAAAJFHBAAAA|1997-10-27||Public passages ought to intervene long extra members. Finally direct poun|2.87|1.52|2004002|edu packimporto #2|4|sports-apparel|2|Men|725|antiableation|extra large|8667689wheat43018884|sienna|Case|Unknown|96|ationationn stanti| +5978|AAAAAAAAKFHBAAAA|1997-10-27|2000-10-26|Offences would not tell ideas. Required neighbours would create previously. Human processes become suddenly specific casualties; things used to propose closely. Substantial views may claim |8.16|4.89|9012003|importounivamalg #3|12|home repair|9|Books|578|eingationanti|N/A|27turquoise098683430|orchid|Ton|Unknown|26|eingationn stanti| +5979|AAAAAAAAKFHBAAAA|2000-10-27||Offences would not tell ideas. Required neighbours would create previously. Human processes become suddenly specific casualties; things used to propose closely. Substantial views may claim |7.18|4.89|5002002|importoscholar #2|2|country|5|Music|497|eingationanti|N/A|64434264650223snow38|steel|Box|Unknown|67|n stationn stanti| +5980|AAAAAAAAMFHBAAAA|1997-10-27|1999-10-27|So famous documents cannot put substantially. Natural, wide measurements will not make national, sufficient users. Quiet figures|0.18|0.13|7012001|importonameless #1|12|paint|7|Home|950|barantin st|N/A|1226turquoise3624412|rosy|Dram|Unknown|48|bareingn stanti| +5981|AAAAAAAAMFHBAAAA|1999-10-28|2001-10-26|So famous documents cannot put substantially. Natural, wide measurements will not make national, sufficient users. Quiet figures|2.80|1.73|7012001|univnameless #2|10|flatware|7|Home|879|n stationeing|N/A|1226turquoise3624412|yellow|Box|Unknown|47|oughteingn stanti| +5982|AAAAAAAAMFHBAAAA|2001-10-27||Available, extensive others feel ba|7.58|2.50|10012016|importoamalgamalg #16|12|monitors|10|Electronics|879|n stationeing|N/A|788061red39926624304|spring|Gram|Unknown|16|ableeingn stanti| +5983|AAAAAAAAPFHBAAAA|1997-10-27||Pregnant, additional hours collect genuine quantities. Other, great members will see totally. Functions wake. National, new courses can keep papers. Only western standards can hit also con|1.36|0.54|1002002|importoamalg #2|2|fragrances|1|Women|74|eseation|large|89808624341sandy1772|sky|Tbl|Unknown|5|prieingn stanti| +5984|AAAAAAAAAGHBAAAA|1997-10-27|2000-10-26|Very minor pairs would end particularly free bad houses. Regulations will learn; as superb hopes suit further here economic weeks. Only monthly leaves could emulate at las|1.50|0.88|2002001|importoimporto #1|2|shirts|2|Men|399|n stn stpri|medium|3162327thistle381113|thistle|Gross|Unknown|6|eseeingn stanti| +5985|AAAAAAAAAGHBAAAA|2000-10-27||So extensive children will make only deaf french functions. Lists demand sweetly harder royal students. Natural, special lines mi|5.23|0.88|2002001|exportinameless #6|2|wallpaper|7|Home|399|n stn stpri|N/A|3162327thistle381113|tan|Pallet|Unknown|16|antieingn stanti| +5986|AAAAAAAACGHBAAAA|1997-10-27|1999-10-27|Other foods w|1.39|1.16|9014011|edu packunivamalg #11|14|sports|9|Books|185|antieingought|N/A|692251459365spring95|peach|Dram|Unknown|55|callyeingn stanti| +5987|AAAAAAAACGHBAAAA|1999-10-28|2001-10-26|Super, ment|8.04|6.19|10013004|exportiamalgamalg #4|14|stereo|10|Electronics|583|prieinganti|N/A|71293493654smoke9705|spring|Lb|Unknown|20|ationeingn stanti| +5988|AAAAAAAACGHBAAAA|2001-10-27||Ever full trusts may make common, dry occasions; years should satisfy from the areas; later able plants should not lose perhaps; integrated, particular skills used to exist alone sufficient,|4.92|3.59|10013004|edu packimporto #1|4|sports-apparel|2|Men|16|callyought|N/A|81095135slate7396955|peach|Tbl|Unknown|28|eingeingn stanti| +5989|AAAAAAAAFGHBAAAA|1997-10-27||Tenants could meet over joint students. Aware, hard males come ordinary, nuclear clothes. National, wes|0.52|0.24|6003006|exporticorp #6|3|gold|6|Jewelry|371|oughtationpri|N/A|691990moccasin143185|wheat|Ounce|Unknown|28|n steingn stanti| +5990|AAAAAAAAGGHBAAAA|1997-10-27|2000-10-26|Only difficult children permit also. Ends must up|3.77|1.88|7004001|edu packbrand #1|4|curtains/drapes|7|Home|644|eseesecally|N/A|579rose9284086993898|pink|Dram|Unknown|56|barn stn stanti| +5991|AAAAAAAAGGHBAAAA|2000-10-27||Only difficult children permit also. Ends must up|3.54|2.26|7004001|edu packscholar #2|4|classical|5|Music|706|eseesecally|N/A|579rose9284086993898|lace|Dozen|Unknown|79|oughtn stn stanti| +5992|AAAAAAAAIGHBAAAA|1997-10-27|1999-10-27|Even different networks cool however|1.64|1.23|5003001|exportischolar #1|3|pop|5|Music|61|oughtcally|N/A|182086512877sky25230|slate|Bunch|Unknown|95|ablen stn stanti| +5993|AAAAAAAAIGHBAAAA|1999-10-28|2001-10-26|Close, previ|1.91|1.45|6012002|importobrand #2|12|costume|6|Jewelry|471|oughtcally|N/A|835228507704plum8665|slate|N/A|Unknown|16|prin stn stanti| +5994|AAAAAAAAIGHBAAAA|2001-10-27||Close, previ|8.88|7.01|10015011|scholaramalgamalg #11|15|portable|10|Electronics|63|prically|N/A|835228507704plum8665|khaki|Case|Unknown|14|esen stn stanti| +5995|AAAAAAAALGHBAAAA|1997-10-27||Real, european |71.44|29.29|1001002|amalgamalg #2|1|dresses|1|Women|25|antiable|extra large|61467327770181navy46|sky|Box|Unknown|20|antin stn stanti| +5996|AAAAAAAAMGHBAAAA|1997-10-27|2000-10-26|Total, bad lines shall not bring in a weeks; healthy, pub|9.14|2.92|9013009|exportiunivamalg #9|13|self-help|9|Books|158|eingantiought|N/A|lawn5530519319754747|powder|Lb|Unknown|26|callyn stn stanti| +5997|AAAAAAAAMGHBAAAA|2000-10-27||Total, bad lines shall not bring in a weeks; healthy, pub|3.94|3.38|9013009|importoscholar #2|13|country|5|Music|158|eingantiought|N/A|277295012894thistle1|rosy|Gram|Unknown|33|ationn stn stanti| +5998|AAAAAAAAOGHBAAAA|1997-10-27|1999-10-27|Scientific, various times|6.78|5.28|3003001|exportiexporti #1|3|toddlers|3|Children|383|prieingpri|medium|thistle6564909985384|chocolate|Tsp|Unknown|10|eingn stn stanti| +5999|AAAAAAAAOGHBAAAA|1999-10-28|2001-10-26|Scientific, various times|84.94|5.28|7003010|exportibrand #10|3|kids|7|Home|210|prieingpri|N/A|thistle6564909985384|puff|Each|Unknown|20|n stn stn stanti| +6000|AAAAAAAAOGHBAAAA|2001-10-27||Scientific, various times|82.53|24.75|7003010|edu packamalg #1|4|swimwear|1|Women|210|baroughtable|medium|6sky0609648976969169|metallic|N/A|Unknown|55|barbarbarcally| +6001|AAAAAAAABHHBAAAA|1997-10-27||New, australian marks move. However loyal things collect commonly. |0.56|0.28|4004002|edu packedu pack #2|4|athletic|4|Shoes|656|callyantically|medium|479455758pale8877150|smoke|Cup|Unknown|22|oughtbarbarcally| +6002|AAAAAAAACHHBAAAA|1997-10-27|2000-10-26|Private lines come plans. Experimental hospitals will not amuse. Of course major runs |5.17|2.17|5004001|edu packscholar #1|4|classical|5|Music|441|oughteseese|N/A|5tan1449367638057035|puff|Gram|Unknown|26|ablebarbarcally| +6003|AAAAAAAACHHBAAAA|2000-10-27||At|4.51|2.17|5004001|scholarcorp #4|4|earings|6|Jewelry|441|oughteseese|N/A|54559thistle91182857|medium|Dram|Unknown|8|pribarbarcally| +6004|AAAAAAAAEHHBAAAA|1997-10-27|1999-10-27|Prepared, necessary others will let above for a stocks. Clearly new studies know. Final, social doubts worry certainly conclusions. Essential, severe attitudes respond sufficiently |8.82|6.79|9004005|edu packmaxi #5|4|entertainments|9|Books|246|callyeseable|N/A|078090938712839navy7|turquoise|Dozen|Unknown|14|esebarbarcally| +6005|AAAAAAAAEHHBAAAA|1999-10-28|2001-10-26|Prepared, necessary others will let above for a stocks. Clearly new studies know. Final, social doubts worry certainly conclusions. Essential, severe attitudes respond sufficiently |0.92|0.57|9004005|importoexporti #2|2|infants|3|Children|158|callyeseable|large|0362honeydew45022704|magenta|Bunch|Unknown|19|antibarbarcally| +6006|AAAAAAAAEHHBAAAA|2001-10-27||Stupid, yellow applications can know less fiscal, united notes. Police accept local, patient hours. Lines spoil. Eastern things |1.96|0.57|2003001|exportiimporto #1|2|pants|2|Men|158|eingantiought|petite|0362honeydew45022704|smoke|Gross|Unknown|30|callybarbarcally| +6007|AAAAAAAAHHHBAAAA|1997-10-27||Most male concessions start forth|10.94|9.62|1001002|amalgamalg #2|1|dresses|1|Women|784|eseeingation|large|255373394saddle07865|bisque|Dozen|Unknown|56|ationbarbarcally| +6008|AAAAAAAAIHHBAAAA|1997-10-27|2000-10-26|Just agricultural years may not talk. Superior, national units will not understand now looks; fresh, soft values trust. Partners ought to discredit methods. Gothic, |8.39|6.04|8007009|brandnameless #9|7|hockey|8|Sports|322|ableablepri|N/A|98689thistle90268685|violet|Bundle|Unknown|3|eingbarbarcally| +6009|AAAAAAAAIHHBAAAA|2000-10-27||Just agricultural years may not talk. Superior, national units will not understand now looks; fresh, soft values trust. Partners ought to discredit methods. Gothic, |8.29|4.64|8007009|maximaxi #10|9|science|9|Books|322|ableablepri|N/A|lace9523263450870555|plum|Ton|Unknown|61|n stbarbarcally| +6010|AAAAAAAAKHHBAAAA|1997-10-27|1999-10-27|Home detailed figures will ask just to a neg|2.29|1.48|6011003|amalgbrand #3|11|semi-precious|6|Jewelry|303|pribarpri|N/A|260836navajo71589538|peru|Cup|Unknown|31|baroughtbarcally| +6011|AAAAAAAAKHHBAAAA|1999-10-28|2001-10-26|Deeply easy advisers shall think for the miles. Complete pensioners can create. Theories give too to a hills. Guidelines love junio|4.99|3.64|6011003|brandcorp #4|7|pendants|6|Jewelry|303|pribarpri|N/A|260836navajo71589538|thistle|Tbl|Unknown|19|oughtoughtbarcally| +6012|AAAAAAAAKHHBAAAA|2001-10-27||Po|1.84|0.73|6011003|exportiimporto #1|7|pants|2|Men|303|pribarpri|extra large|564118921purple26353|spring|Dozen|Unknown|36|ableoughtbarcally| +6013|AAAAAAAANHHBAAAA|1997-10-27||Also f|3.46|2.49|6007006|brandcorp #6|7|pendants|6|Jewelry|116|callyoughtought|N/A|502332287magenta6453|chocolate|Ton|Unknown|17|prioughtbarcally| +6014|AAAAAAAAOHHBAAAA|1997-10-27|2000-10-26|Subjects sit only usually financial drugs; either joint months eat at a changes. Unpleasant gardens gain sad, new values. Articles give similarly ideally strange others. As responsible c|6.71|2.48|9012003|importounivamalg #3|12|home repair|9|Books|579|n stationanti|N/A|1violet8086436460723|lawn|Oz|Unknown|13|eseoughtbarcally| +6015|AAAAAAAAOHHBAAAA|2000-10-27||Objectives could get to a assets. Notions mean completely. Great centres reflect into a cattle. Then provincial eyes shall not|3.21|2.48|9012003|maximaxi #4|12|science|9|Books|579|n stationanti|N/A|94769166680yellow355|sienna|Case|Unknown|83|antioughtbarcally| +6016|AAAAAAAAAIHBAAAA|1997-10-27|1999-10-27|Valuable, young ways make at all years. Mo|3.67|1.98|9009005|maximaxi #5|9|science|9|Books|598|eingn stanti|N/A|2319plum522968431951|sienna|Dram|Unknown|15|callyoughtbarcally| +6017|AAAAAAAAAIHBAAAA|1999-10-28|2001-10-26|Sales will not know genuinely apparent, blue studies. Spare attempts can turn safely strange foods. International tears help only uses. Studies notify quickly only th|7.95|1.98|4003002|exportiedu pack #2|3|kids|4|Shoes|598|eingn stanti|large|2319plum522968431951|red|Oz|Unknown|8|ationoughtbarcally| +6018|AAAAAAAAAIHBAAAA|2001-10-27||Sales will not know genuinely apparent, blue studies. Spare attempts can turn safely strange foods. International tears help only uses. Studies notify quickly only th|1.90|1.98|4003002|importoedu pack #1|2|mens|4|Shoes|979|eingn stanti|medium|90077528rosy76447539|mint|Cup|Unknown|33|eingoughtbarcally| +6019|AAAAAAAADIHBAAAA|1997-10-27||Various, standard sources shall not manage never in a levels. Sound, existing factors would make either around the awards. Home domestic stars reach swiftly local, peaceful vegetables.|2.58|2.03|2001002|amalgimporto #2|1|accessories|2|Men|970|barationn st|economy|0763208214155705tan7|chartreuse|Bundle|Unknown|65|n stoughtbarcally| +6020|AAAAAAAAEIHBAAAA|1997-10-27|2000-10-26|Indian, old classes must not see always years. Military, proud hands allocate pupils. Generally funny assumpti|27.12|16.81|2001001|amalgimporto #1|1|accessories|2|Men|124|eseableought|large|55676320peach0694561|maroon|Oz|Unknown|25|barablebarcally| +6021|AAAAAAAAEIHBAAAA|2000-10-27||Indian, old classes must not see always years. Military, proud hands allocate pupils. Generally funny assumpti|1.69|1.36|2001001|amalgbrand #2|1|bathroom|7|Home|179|n stationought|N/A|55676320peach0694561|steel|Cup|Unknown|59|oughtablebarcally| +6022|AAAAAAAAGIHBAAAA|1997-10-27|1999-10-27|Right joint uses cannot provide less labour, final windows. Ori|5.83|3.73|9008011|namelessmaxi #11|8|romance|9|Books|202|ablebarable|N/A|42gainsboro321799545|turquoise|Unknown|Unknown|88|ableablebarcally| +6023|AAAAAAAAGIHBAAAA|1999-10-28|2001-10-26|Major difficulties will confess also visua|1.89|1.39|2001002|amalgimporto #2|8|accessories|2|Men|202|ablebarable|large|42gainsboro321799545|salmon|Bunch|Unknown|45|priablebarcally| +6024|AAAAAAAAGIHBAAAA|2001-10-27||Great, rational neighbours could not build honestly difficulties. Top, automatic states should provide years. Only light aims|8.77|1.39|2001002|edu packedu pack #1|4|athletic|4|Shoes|202|ablebarable|extra large|945884navy3202979120|saddle|Pound|Unknown|31|eseablebarcally| +6025|AAAAAAAAJIHBAAAA|1997-10-27||Large occasions will dance more great objections. Certain firms recommend quite in a areas. Much intimate goals sound so patient relations. True notio|9.80|4.50|4001002|amalgedu pack #2|1|womens|4|Shoes|412|ableoughtese|small|696966salmon44160771|spring|N/A|Unknown|5|antiablebarcally| +6026|AAAAAAAAKIHBAAAA|1997-10-27|2000-10-26|Days make now parents. True women deal divisions. Available, recent areas should help effectively important matters. Businesses tell pr|4.22|1.94|2004001|edu packimporto #1|4|sports-apparel|2|Men|427|ationableese|medium|836lavender598165901|saddle|Each|Unknown|56|callyablebarcally| +6027|AAAAAAAAKIHBAAAA|2000-10-27||Days make now parents. True women deal divisions. Available, recent areas should help effectively important matters. Businesses tell pr|7.08|3.89|5001002|amalgscholar #2|4|rock|5|Music|427|ationableese|N/A|86582wheat2759773367|smoke|Case|Unknown|30|ationablebarcally| +6028|AAAAAAAAMIHBAAAA|1997-10-27|1999-10-27|Grateful, elaborate matters like. Final cards operate cold families; impossib|1.40|0.49|5001001|amalgscholar #1|1|rock|5|Music|30|barpri|N/A|2516144309saddle6984|tan|Dozen|Unknown|10|eingablebarcally| +6029|AAAAAAAAMIHBAAAA|1999-10-28|2001-10-26|Grateful, elaborate matters like. Final cards operate cold families; impossib|1.39|0.49|10003012|exportiunivamalg #12|3|dvd/vcr players|10|Electronics|101|barpri|N/A|2516144309saddle6984|purple|Bunch|Unknown|77|n stablebarcally| +6030|AAAAAAAAMIHBAAAA|2001-10-27||Thus private hospitals could gain undoubte|2.49|0.49|10003012|importoscholar #1|3|country|5|Music|101|barpri|N/A|peach044538206079326|cyan|Ounce|Unknown|66|barpribarcally| +6031|AAAAAAAAPIHBAAAA|1997-10-27||Below voluntary results should become with a teachers. Constant movements might not develop again in the plans.|3.90|3.35|2003002|exportiimporto #2|3|pants|2|Men|455|antiantiese|petite|92945443312431white2|thistle|Dram|Unknown|6|oughtpribarcally| +6032|AAAAAAAAAJHBAAAA|1997-10-27|2000-10-26|Of course special proposals support certainly social objects. Very urgent tests |1.51|1.14|2002001|importoimporto #1|2|shirts|2|Men|225|antiableable|petite|1snow518495719167387|rose|Pound|Unknown|64|ablepribarcally| +6033|AAAAAAAAAJHBAAAA|2000-10-27||Gastric games insist suddenly in connection with a politicians; employees mean then arms; s|7.35|4.85|10013016|exportiamalgamalg #16|13|stereo|10|Electronics|83|antiableable|N/A|02912110puff22406444|powder|Tsp|Unknown|22|pripribarcally| +6034|AAAAAAAACJHBAAAA|1997-10-27|1999-10-27|Good children shall pass fairly free, current cards. German tactics know|1.13|0.48|9005011|scholarmaxi #11|5|history|9|Books|144|eseeseought|N/A|17597336142691red890|linen|Gross|Unknown|5|esepribarcally| +6035|AAAAAAAACJHBAAAA|1999-10-28|2001-10-26|Good children shall pass fairly free, current cards. German tactics know|0.36|0.13|9005011|exportiimporto #2|3|pants|2|Men|265|eseeseought|medium|6216royal12479564777|sky|N/A|Unknown|42|antipribarcally| +6036|AAAAAAAACJHBAAAA|2001-10-27||Good children shall pass fairly free, current cards. German tactics know|0.58|0.13|4004001|edu packedu pack #1|3|athletic|4|Shoes|265|eseeseought|petite|6216royal12479564777|puff|Tsp|Unknown|51|callypribarcally| +6037|AAAAAAAAFJHBAAAA|1997-10-27||British services will compare greatly very early words. Key, high personn|17.81|13.00|5004002|edu packscholar #2|4|classical|5|Music|381|oughteingpri|N/A|5398papaya9351743604|snow|Dram|Unknown|48|ationpribarcally| +6038|AAAAAAAAGJHBAAAA|1997-10-27|2000-10-26|Lead|2.43|1.40|3004001|edu packexporti #1|4|school-uniforms|3|Children|269|n stcallyable|small|6278073168014forest1|sky|Tsp|Unknown|18|eingpribarcally| +6039|AAAAAAAAGJHBAAAA|2000-10-27||Lead|4.52|1.40|3004001|amalgunivamalg #5|1|cameras|10|Electronics|216|callyoughtable|N/A|479730678cornflower3|blue|Dram|Unknown|54|n stpribarcally| +6040|AAAAAAAAIJHBAAAA|1997-10-27|1999-10-27|Italian, european executives ma|3.08|2.24|2002001|importoimporto #1|2|shirts|2|Men|660|barcallycally|small|664plum8109869413401|gainsboro|Cup|Unknown|91|baresebarcally| +6041|AAAAAAAAIJHBAAAA|1999-10-28|2001-10-26|Small collections save at present international boxes. Efforts could not know less than months. Da|8.12|3.89|2002001|exportiimporto #2|3|pants|2|Men|744|barcallycally|N/A|9848450596tan1773829|yellow|Pallet|Unknown|3|oughtesebarcally| +6042|AAAAAAAAIJHBAAAA|2001-10-27||Small collections save at present international boxes. Efforts could not know less than months. Da|0.57|3.89|2002001|namelessbrand #3|8|lighting|7|Home|176|callyationought|N/A|63799135126821plum67|rose|Box|Unknown|49|ableesebarcally| +6043|AAAAAAAALJHBAAAA|1997-10-27||Standards could not exploit total communities; extraordinary, young laws go there. Boys must not|1.65|0.77|7004004|edu packbrand #4|4|curtains/drapes|7|Home|134|esepriought|N/A|2187380730617snow199|royal|Carton|Unknown|14|priesebarcally| +6044|AAAAAAAAMJHBAAAA|1997-10-27|2000-10-26|Very other statements ease ways. Provisions occur. Terms arrive only rural sons. Foreign estates provide outside in a patients. Select, formal cases shout near liberal th|5.15|2.42|2002001|importoimporto #1|2|shirts|2|Men|551|oughtantianti|medium|sienna78613667141397|seashell|Box|Unknown|14|eseesebarcally| +6045|AAAAAAAAMJHBAAAA|2000-10-27||Pictures organize shar|6.39|2.42|2002001|exportiimporto #2|3|pants|2|Men|567|ationcallyanti|small|peach316584990629045|plum|Pallet|Unknown|30|antiesebarcally| +6046|AAAAAAAAOJHBAAAA|1997-10-27|1999-10-27|Techniques must protect simply recordings. Final, wide arms touch somewhere patients. Excellent, broad plans think in a areas. Simple, main links stand as days. Actually young yards ought to spend |2.82|1.07|5004001|edu packscholar #1|4|classical|5|Music|246|callyeseable|N/A|9898798633521saddle5|peach|Pallet|Unknown|23|callyesebarcally| +6047|AAAAAAAAOJHBAAAA|1999-10-28|2001-10-26|Techniques must protect simply recordings. Final, wide arms touch somewhere patients. Excellent, broad plans think in a areas. Simple, main links stand as days. Actually young yards ought to spend |4.44|1.07|5004001|amalgexporti #2|4|newborn|3|Children|601|oughtbarcally|petite|2825299516slate96397|wheat|Box|Unknown|16|ationesebarcally| +6048|AAAAAAAAOJHBAAAA|2001-10-27||Techniques must protect simply recordings. Final, wide arms touch somewhere patients. Excellent, broad plans think in a areas. Simple, main links stand as days. Actually young yards ought to spend |3.27|1.07|5004001|brandnameless #9|7|hockey|8|Sports|552|oughtbarcally|N/A|513rose8289014991568|powder|Gram|Unknown|13|eingesebarcally| +6049|AAAAAAAABKHBAAAA|1997-10-27||Big, other animals get; tiny, active products shall take most both long facilities. Sexual, high councillors spare particular books. Already cognitive plans shall look in a |3.23|2.87|4004002|edu packedu pack #2|4|athletic|4|Shoes|57|ationanti|economy|49spring118730991782|seashell|Oz|Unknown|26|n stesebarcally| +6050|AAAAAAAACKHBAAAA|1997-10-27|2000-10-26|Now british prices include so. Usually real centres will not know strikingly. Nice pages would become under a males. Strong experts insert ever royal, co|4.34|2.34|1001001|amalgamalg #1|1|dresses|1|Women|798|eingn station|extra large|58goldenrod496684131|peach|Gram|Unknown|62|barantibarcally| +6051|AAAAAAAACKHBAAAA|2000-10-27||Now british prices include so. Usually real centres will not know strikingly. Nice pages would become under a males. Strong experts insert ever royal, co|6.32|2.21|1002002|importoamalg #2|1|fragrances|1|Women|295|eingn station|medium|391741142961papaya52|white|Each|Unknown|19|oughtantibarcally| +6052|AAAAAAAAEKHBAAAA|1997-10-27|1999-10-27|Always smart princip|9.79|8.71|4002001|importoedu pack #1|2|mens|4|Shoes|237|ationpriable|extra large|pale5777583505799980|sienna|Pallet|Unknown|51|ableantibarcally| +6053|AAAAAAAAEKHBAAAA|1999-10-28|2001-10-26|Schools may encourage s|1.57|8.71|8010004|univmaxi #4|10|pools|8|Sports|237|ationpriable|N/A|pale5777583505799980|saddle|Bunch|Unknown|63|priantibarcally| +6054|AAAAAAAAEKHBAAAA|2001-10-27||Left boys get wages.|0.12|8.71|8010004|edu packamalg #1|4|swimwear|1|Women|162|ationpriable|medium|579639seashell891773|lavender|Pallet|Unknown|18|eseantibarcally| +6055|AAAAAAAAHKHBAAAA|1997-10-27||Clean w|1.35|1.02|4003002|exportiedu pack #2|3|kids|4|Shoes|295|antin stable|economy|211307powder03003130|pale|Each|Unknown|29|antiantibarcally| +6056|AAAAAAAAIKHBAAAA|1997-10-27|2000-10-26|Even so pale beings could not live active studies. Severely criminal variations may not choose yet ingre|2.71|2.35|1001001|amalgamalg #1|1|dresses|1|Women|27|ationable|extra large|80510053409180slate9|powder|Lb|Unknown|44|callyantibarcally| +6057|AAAAAAAAIKHBAAAA|2000-10-27||Even so pale beings could not live active studies. Severely criminal variations may not choose yet ingre|1.81|2.35|3004002|edu packexporti #2|1|school-uniforms|3|Children|27|ationable|small|80510053409180slate9|ghost|Gross|Unknown|21|ationantibarcally| +6058|AAAAAAAAKKHBAAAA|1997-10-27|1999-10-27|Trousers|99.11|32.70|3003001|exportiexporti #1|3|toddlers|3|Children|29|n stable|extra large|7952739873steel03501|olive|Box|Unknown|55|eingantibarcally| +6059|AAAAAAAAKKHBAAAA|1999-10-28|2001-10-26|Trousers|1.87|1.42|3003001|exportiedu pack #2|3|kids|4|Shoes|81|n stable|medium|65576211645402pink96|salmon|Pallet|Unknown|27|n stantibarcally| +6060|AAAAAAAAKKHBAAAA|2001-10-27||Trousers|9.42|1.42|3003001|edu packamalgamalg #9|14|automotive|10|Electronics|86|callyeing|N/A|65576211645402pink96|goldenrod|Cup|Unknown|21|barcallybarcally| +6061|AAAAAAAANKHBAAAA|1997-10-27||Young figures should go onl|9.27|4.63|8016002|corpmaxi #2|16|golf|8|Sports|73|priation|N/A|56sienna113581592949|rosy|Case|Unknown|21|oughtcallybarcally| +6062|AAAAAAAAOKHBAAAA|1997-10-27|2000-10-26|Significant fingers may translate with a transactions. Parties know only. Databases give always never natural laws; years might not see then european tasks. Songs involve ca|5.63|2.98|3004001|edu packexporti #1|4|school-uniforms|3|Children|67|ationcally|extra large|31822291058tomato060|salmon|Bundle|Unknown|40|ablecallybarcally| +6063|AAAAAAAAOKHBAAAA|2000-10-27||Moving writings lose later students; now regional pairs may not provide then. Late, rare premises go well young, existing seats. Years start even for a pupils. Formal pressures admit |1.38|2.98|3004001|amalgimporto #2|1|accessories|2|Men|174|eseationought|small|31822291058tomato060|tan|Carton|Unknown|53|pricallybarcally| +6064|AAAAAAAAALHBAAAA|1997-10-27|1999-10-27|Over concerned paymen|8.97|3.13|10015013|scholaramalgamalg #13|15|portable|10|Electronics|187|ationeingought|N/A|3799496smoke58083128|salmon|Tsp|Unknown|7|esecallybarcally| +6065|AAAAAAAAALHBAAAA|1999-10-28|2001-10-26|Over concerned paymen|4.50|2.11|4002002|importoedu pack #2|2|mens|4|Shoes|187|ationeingought|large|3799496smoke58083128|sienna|Dram|Unknown|62|anticallybarcally| +6066|AAAAAAAAALHBAAAA|2001-10-27||Other cit|0.16|2.11|7007007|brandbrand #7|7|decor|7|Home|187|ationeingought|N/A|3799496smoke58083128|purple|Dram|Unknown|27|callycallybarcally| +6067|AAAAAAAADLHBAAAA|1997-10-27||Illegal, specific buildings must not take. Tests cease s|0.52|0.30|5004002|edu packscholar #2|4|classical|5|Music|955|antiantin st|N/A|8355621047983smoke11|snow|Ounce|Unknown|6|ationcallybarcally| +6068|AAAAAAAAELHBAAAA|1997-10-27|2000-10-26|Eyes could not|4.34|1.64|9010009|univunivamalg #9|10|travel|9|Books|212|ableoughtable|N/A|0salmon3575331679625|salmon|Each|Unknown|19|eingcallybarcally| +6069|AAAAAAAAELHBAAAA|2000-10-27||Eyes could not|8.86|6.73|9010009|brandmaxi #10|7|reference|9|Books|231|oughtpriable|N/A|0salmon3575331679625|lace|Pound|Unknown|26|n stcallybarcally| +6070|AAAAAAAAGLHBAAAA|1997-10-27|1999-10-27|Irish sentences make never to a ti|2.92|1.19|6009007|maxicorp #7|9|womens watch|6|Jewelry|230|barpriable|N/A|388562683366white166|misty|Dram|Unknown|21|barationbarcally| +6071|AAAAAAAAGLHBAAAA|1999-10-28|2001-10-26|Vigorously anonymous women follow; european pupils leap mass, pleasant characteristics.|30.51|1.19|6011008|amalgbrand #8|11|semi-precious|6|Jewelry|230|barpriable|N/A|060449172614yellow21|violet|Cup|Unknown|47|oughtationbarcally| +6072|AAAAAAAAGLHBAAAA|2001-10-27||Vigorously anonymous women follow; european pupils leap mass, pleasant characteristics.|9.36|4.68|2002001|importoimporto #1|2|shirts|2|Men|230|barpriable|petite|884683803702509snow6|navy|Dozen|Unknown|26|ableationbarcally| +6073|AAAAAAAAJLHBAAAA|1997-10-27||Inc, little sections will make now native, greek ways; greek, good children live most so new pupils. |17.40|13.57|6006002|corpcorp #2|6|rings|6|Jewelry|243|prieseable|N/A|1102784519885purple2|navajo|Pallet|Unknown|28|priationbarcally| +6074|AAAAAAAAKLHBAAAA|1997-10-27|2000-10-26|Types support already forms. So appropriate substances must not control perhaps nervous young years. Communist services must go decisive, conside|5.43|4.45|9002003|importomaxi #3|2|business|9|Books|385|antieingpri|N/A|242039273383khaki174|steel|Pallet|Unknown|25|eseationbarcally| +6075|AAAAAAAAKLHBAAAA|2000-10-27||Types support already forms. So appropriate substances must not control perhaps nervous young years. Communist services must go decisive, conside|1.65|4.45|9013004|exportiunivamalg #4|2|self-help|9|Books|385|antieingpri|N/A|242039273383khaki174|light|Pallet|Unknown|61|antiationbarcally| +6076|AAAAAAAAMLHBAAAA|1997-10-27|1999-10-27|Valu|1.87|1.53|9008005|namelessmaxi #5|8|romance|9|Books|78|eingation|N/A|46198484832navy42452|turquoise|Pallet|Unknown|1|callyationbarcally| +6077|AAAAAAAAMLHBAAAA|1999-10-28|2001-10-26|Soviet, tired answers would call now negative plans. Able, social years accept low fine sales. New, new women could disce|1.80|1.53|4004002|edu packedu pack #2|8|athletic|4|Shoes|233|eingation|extra large|5414511seashell29111|medium|Ton|Unknown|23|ationationbarcally| +6078|AAAAAAAAMLHBAAAA|2001-10-27||Rare, superb sites may identify in an voices. Men hurt all right small, isolated trees. Previous employees imagine also right sides. Poor heads turn simply serious votes. Decisive goals would s|3.54|1.53|10002010|importounivamalg #10|8|camcorders|10|Electronics|424|eingation|N/A|5414511seashell29111|honeydew|Cup|Unknown|67|eingationbarcally| +6079|AAAAAAAAPLHBAAAA|1997-10-27||Distinct areas develop; supposedly free pupils used to show only in a organisations. Applications await so. Companies ought to get firms; successfully slow cases beco|0.99|0.67|5001002|amalgscholar #2|1|rock|5|Music|615|antioughtcally|N/A|1646555174349indian2|turquoise|Tbl|Unknown|15|n stationbarcally| +6080|AAAAAAAAAMHBAAAA|1997-10-27|2000-10-26|Parliamentary guests could not convey real chiefs; integrated, full responsibilities take later then important categories. T|0.41|0.13|1004001|edu packamalg #1|4|swimwear|1|Women|856|callyantieing|medium|13606146152saddle347|sandy|Case|Unknown|19|bareingbarcally| +6081|AAAAAAAAAMHBAAAA|2000-10-27||Healthy, special times tell just no doubt single objects; southern, other issues look years. Difficult, minor minds should smell reasonably. Changes know only likely cases. Businessme|97.66|0.13|4002002|importoedu pack #2|4|mens|4|Shoes|856|callyantieing|medium|48490026179569royal2|rosy|Carton|Unknown|53|oughteingbarcally| +6082|AAAAAAAACMHBAAAA|1997-10-27|1999-10-27|Images would maintain for a things. Waiting amounts wander more in the actions. More fat traditions bear formally great, old hands; generally upper consultants may |0.15|0.06|2002001|importoimporto #1|2|shirts|2|Men|163|pricallyought|petite|69827918peru29676052|white|N/A|Unknown|24|ableeingbarcally| +6083|AAAAAAAACMHBAAAA|1999-10-28|2001-10-26|Sure, socia|5.88|4.17|2002001|importoscholar #2|2|country|5|Music|584|pricallyought|N/A|5493836puff759447308|steel|Bunch|Unknown|37|prieingbarcally| +6084|AAAAAAAACMHBAAAA|2001-10-27||Super, increased governments stay now students. Major signs remember even by a projects. Songs make public, necessary practitioners. Structures can cha|2.82|4.17|3004001|edu packexporti #1|2|school-uniforms|3|Children|634|eseprically|medium|5493836puff759447308|sky|Box|Unknown|78|eseeingbarcally| +6085|AAAAAAAAFMHBAAAA|1997-10-27||Real, social markets used to regulate independent members. Different teeth shall p|67.52|27.00|1003002|exportiamalg #2|3|maternity|1|Women|483|prieingese|medium|8664340077066papaya0|thistle|Dozen|Unknown|8|antieingbarcally| +6086|AAAAAAAAGMHBAAAA|1997-10-27|2000-10-26|Onc|2.64|2.13|1004001|edu packamalg #1|4|swimwear|1|Women|201|oughtbarable|medium|98822721869purple189|sandy|Gram|Unknown|77|callyeingbarcally| +6087|AAAAAAAAGMHBAAAA|2000-10-27||Measures ride linguistic, islamic schools. Strong women incur only boundaries. So wrong attitudes want otherwise special, easy women. Cool publications should organise problems. Poets get more prev|5.74|2.64|1004001|edu packscholar #2|4|classical|5|Music|54|eseanti|N/A|0266703630black83449|hot|Ton|Unknown|54|ationeingbarcally| +6088|AAAAAAAAIMHBAAAA|1997-10-27|1999-10-27|British, quick friends might make early good min|2.17|0.80|8016009|corpmaxi #9|16|golf|8|Sports|71|oughtation|N/A|7826606496524cyan179|snow|Oz|Unknown|3|eingeingbarcally| +6089|AAAAAAAAIMHBAAAA|1999-10-28|2001-10-26|British, quick friends might make early good min|6.95|2.43|7001010|amalgbrand #10|1|bathroom|7|Home|107|oughtation|N/A|427547002683sky40769|saddle|N/A|Unknown|25|n steingbarcally| +6090|AAAAAAAAIMHBAAAA|2001-10-27||Others might improve on every explanations. Consis|4.79|2.05|7001010|exportinameless #1|1|basketball|8|Sports|91|oughtn st|N/A|58532sky096770881032|papaya|Dozen|Unknown|11|barn stbarcally| +6091|AAAAAAAALMHBAAAA|1997-10-27||Private, public standards would take standards; outside knees may complement more abroad possible|17.26|9.83|3001002|amalgexporti #2|1|newborn|3|Children|512|ableoughtanti|extra large|0426385174ghost81351|white|Pound|Unknown|16|oughtn stbarcally| +6092|AAAAAAAAMMHBAAAA|1997-10-27|2000-10-26|Losses use high. Clearly small customers must not lead there only remarkable suggestions. Much old policies receive then convincing affairs. Silent, british years may not take now |0.72|0.56|4004001|edu packedu pack #1|4|athletic|4|Shoes|591|oughtn stanti|small|815278333658plum0270|frosted|Each|Unknown|9|ablen stbarcally| +6093|AAAAAAAAMMHBAAAA|2000-10-27||Religious days say away good p|6.92|0.56|8001004|amalgnameless #4|4|athletic shoes|8|Sports|591|oughtn stanti|N/A|6373561737tan1672359|rosy|Case|Unknown|32|prin stbarcally| +6094|AAAAAAAAOMHBAAAA|1997-10-27|1999-10-27|Usual services help also gothic fields; temporary, christian police contract only with a instances. Overall |0.70|0.35|6014007|edu packbrand #7|14|estate|6|Jewelry|160|barcallyought|N/A|23539plum21411811839|snow|Bunch|Unknown|6|esen stbarcally| +6095|AAAAAAAAOMHBAAAA|1999-10-28|2001-10-26|Usual services help also gothic fields; temporary, christian police contract only with a instances. Overall |9.94|0.35|3001002|amalgexporti #2|1|newborn|3|Children|399|n stn stpri|medium|02601340azure7298775|hot|Each|Unknown|42|antin stbarcally| +6096|AAAAAAAAOMHBAAAA|2001-10-27||Usual services help also gothic fields; temporary, christian police contract only with a instances. Overall |2.10|0.35|9011001|amalgunivamalg #1|1|cooking|9|Books|399|n stn stpri|N/A|02601340azure7298775|papaya|N/A|Unknown|72|callyn stbarcally| +6097|AAAAAAAABNHBAAAA|1997-10-27||Now lovely arms may notice nowadays important rights; difficult, only hundreds shall remain new years. Readers would know away different produ|1.27|0.85|6001002|amalgcorp #2|1|birdal|6|Jewelry|594|esen stanti|N/A|8921232222violet3849|navajo|Pallet|Unknown|48|ationn stbarcally| +6098|AAAAAAAACNHBAAAA|1997-10-27|2000-10-26|Simple friends receive finally from the times. Famous, |2.20|1.71|1001001|amalgamalg #1|1|dresses|1|Women|230|barpriable|extra large|6804560285gainsboro0|papaya|Gross|Unknown|62|eingn stbarcally| +6099|AAAAAAAACNHBAAAA|2000-10-27||Simple friends receive finally from the times. Famous, |1.87|1.36|1001001|amalgedu pack #2|1|womens|4|Shoes|230|barpriable|petite|6804560285gainsboro0|olive|Gram|Unknown|71|n stn stbarcally| +6100|AAAAAAAAENHBAAAA|1997-10-27|1999-10-27|Good, low h|3.65|1.13|10002015|importounivamalg #15|2|camcorders|10|Electronics|71|oughtation|N/A|493279papaya03928996|misty|Case|Unknown|27|barbaroughtcally| +6101|AAAAAAAAENHBAAAA|1999-10-28|2001-10-26|Too key programmes will say tomorrow|7.65|5.12|6004006|edu packcorp #6|4|bracelets|6|Jewelry|71|oughtation|N/A|07331658189peru67045|black|Pallet|Unknown|60|oughtbaroughtcally| +6102|AAAAAAAAENHBAAAA|2001-10-27||Effective shops shall bear also electronic, different activities. Red, difficult women seem justifiably in a weeks. Different citizens ought to note perfectly for a hands. Large, musical conc|35.24|5.12|6004006|amalgimporto #1|1|accessories|2|Men|605|oughtation|economy|43205sandy3024683703|lace|Each|Unknown|1|ablebaroughtcally| +6103|AAAAAAAAHNHBAAAA|1997-10-27||Views should cultivate even ambitious, in|1.58|1.21|8016004|corpmaxi #4|16|golf|8|Sports|168|eingcallyought|N/A|94turquoise772166967|orchid|Dozen|Unknown|6|pribaroughtcally| +6104|AAAAAAAAINHBAAAA|1997-10-27|2000-10-26|Minutes return too political times. Then social terms will say thus accurate, ad|8.82|4.05|2002001|importoimporto #1|2|shirts|2|Men|146|callyeseought|large|429048871smoke355598|slate|Oz|Unknown|6|esebaroughtcally| +6105|AAAAAAAAINHBAAAA|2000-10-27||Minutes return too political times. Then social terms will say thus accurate, ad|3.21|1.92|3004002|edu packexporti #2|2|school-uniforms|3|Children|513|prioughtanti|medium|429048871smoke355598|bisque|Dozen|Unknown|7|antibaroughtcally| +6106|AAAAAAAAKNHBAAAA|1997-10-27|1999-10-27|Once difficult families ought to give around legal, british models. Likely agents cannot say often for a applica|57.18|26.87|10003004|exportiunivamalg #4|3|dvd/vcr players|10|Electronics|482|ableeingese|N/A|404honeydew635846587|navajo|Dozen|Unknown|1|callybaroughtcally| +6107|AAAAAAAAKNHBAAAA|1999-10-28|2001-10-26|Employees used to take of course needs. Cultural effects ought to see highly. Scottish companies would enforce currently to the plants. Modern eyes will not sit famous, concerned shares. Fr|0.84|0.32|6002004|importocorp #4|2|diamonds|6|Jewelry|255|ableeingese|N/A|404honeydew635846587|sienna|Pallet|Unknown|14|ationbaroughtcally| +6108|AAAAAAAAKNHBAAAA|2001-10-27||Employees used to take of course needs. Cultural effects ought to see highly. Scottish companies would enforce currently to the plants. Modern eyes will not sit famous, concerned shares. Fr|3.79|0.32|6002004|edu packamalg #1|2|swimwear|1|Women|314|ableeingese|medium|2snow775804464533338|thistle|Tbl|Unknown|20|eingbaroughtcally| +6109|AAAAAAAANNHBAAAA|1997-10-27||Old, casual cards appear large, industrial areas. There chinese profits receive well safe words. Contemporary centuries consider particularly|9.83|4.12|9005002|scholarmaxi #2|5|history|9|Books|46|callyese|N/A|52461157papaya983442|light|Ounce|Unknown|44|n stbaroughtcally| +6110|AAAAAAAAONHBAAAA|1997-10-27|2000-10-26|Houses will find exactly; political women want internationally materials. Religious areas recei|2.38|0.85|2002001|importoimporto #1|2|shirts|2|Men|100|barbarought|small|38304681642wheat1122|powder|Lb|Unknown|8|baroughtoughtcally| +6111|AAAAAAAAONHBAAAA|2000-10-27||Houses will find exactly; political women want internationally materials. Religious areas recei|7.97|0.85|5003002|exportischolar #2|2|pop|5|Music|100|barbarought|N/A|38304681642wheat1122|peru|Case|Unknown|10|oughtoughtoughtcally| +6112|AAAAAAAAAOHBAAAA|1997-10-27|1999-10-27|Opponents file on a initiatives. Ready, professional lips shall not work under way about a things. Ugly plants highlight wel|4.08|2.16|7009003|maxibrand #3|9|mattresses|7|Home|629|n stablecally|N/A|5031magenta004583592|dodger|Tsp|Unknown|60|ableoughtoughtcally| +6113|AAAAAAAAAOHBAAAA|1999-10-28|2001-10-26|Specific, basic debts take relevant, commo|29.12|2.16|2002002|importoimporto #2|9|shirts|2|Men|356|n stablecally|medium|5031magenta004583592|smoke|Pallet|Unknown|48|prioughtoughtcally| +6114|AAAAAAAAAOHBAAAA|2001-10-27||Specific, basic debts take relevant, commo|5.14|4.11|2002002|exportischolar #1|3|pop|5|Music|356|callyantipri|N/A|383533smoke237735300|snow|Case|Unknown|8|eseoughtoughtcally| +6115|AAAAAAAADOHBAAAA|1997-10-27||Special, other rig|4.34|2.38|8004006|edu packnameless #6|4|camping|8|Sports|7|ation|N/A|20713399823lavender4|yellow|Gram|Unknown|46|antioughtoughtcally| +6116|AAAAAAAAEOHBAAAA|1997-10-27|2000-10-26|Main teachers walk about far available sentences. General kids will happen often various boys. Inc premises shall not get constantly administrative days. No|2.95|2.00|10001014|amalgunivamalg #14|1|cameras|10|Electronics|4|ese|N/A|640116415165pink0445|violet|Bundle|Unknown|61|callyoughtoughtcally| +6117|AAAAAAAAEOHBAAAA|2000-10-27||Volumes report well black books. Right little towns take things. Funds go well taxes. Suddenly electric reasons apply spectacular, short goals. Hostile problems should admit methods; insuffici|4.30|3.48|10001014|exportinameless #8|1|wallpaper|7|Home|4|ese|N/A|640116415165pink0445|deep|Bundle|Unknown|47|ationoughtoughtcally| +6118|AAAAAAAAGOHBAAAA|1997-10-27|1999-10-27|Central times need severely there continental matters. Old statements cannot influence late. Empty, due patients can include. Hot years will get in the men|8.34|4.75|4004001|edu packedu pack #1|4|athletic|4|Shoes|217|ationoughtable|extra large|08157lavender9866610|salmon|Ounce|Unknown|16|eingoughtoughtcally| +6119|AAAAAAAAGOHBAAAA|1999-10-28|2001-10-26|Sudden lives used to take even in a hands. Patients get. Other, important opportunities could think at a resources. Benefits decide later prominent, correct pp.. Measures use pupils. Assets cou|0.25|0.08|4004001|edu packamalg #2|4|swimwear|1|Women|62|ablecally|large|08157lavender9866610|royal|Gross|Unknown|99|n stoughtoughtcally| +6120|AAAAAAAAGOHBAAAA|2001-10-27||Sudden lives used to take even in a hands. Patients get. Other, important opportunities could think at a resources. Benefits decide later prominent, correct pp.. Measures use pupils. Assets cou|4.54|0.08|6010001|univbrand #1|10|jewelry boxes|6|Jewelry|164|ablecally|N/A|08157lavender9866610|papaya|Pallet|Unknown|29|barableoughtcally| +6121|AAAAAAAAJOHBAAAA|1997-10-27||Posts use revolutionary, important ideas; very common years used to remember at least left, easy arguments. More able boys |0.59|0.45|1001002|amalgamalg #2|1|dresses|1|Women|263|pricallyable|medium|9243741142859slate80|sienna|Ounce|Unknown|31|oughtableoughtcally| +6122|AAAAAAAAKOHBAAAA|1997-10-27|2000-10-26|Defiantly positive parts work only already global connections. Political, historical pages estimate appr|7.84|4.23|8004003|edu packnameless #3|4|camping|8|Sports|634|eseprically|N/A|68397204916167hot974|pale|Ton|Unknown|3|ableableoughtcally| +6123|AAAAAAAAKOHBAAAA|2000-10-27||Defiantly positive parts work only already global connections. Political, historical pages estimate appr|25.52|4.23|10011004|amalgamalgamalg #4|11|disk drives|10|Electronics|607|ationbarcally|N/A|68397204916167hot974|medium|Lb|Unknown|52|priableoughtcally| +6124|AAAAAAAAMOHBAAAA|1997-10-27|1999-10-27|Fortunately past rules mind respectively appropriate losses. Men must develop above the sources. Mere values lis|2.02|0.66|9003005|exportimaxi #5|3|computers|9|Books|145|antieseought|N/A|19salmon902633599394|goldenrod|Box|Unknown|34|eseableoughtcally| +6125|AAAAAAAAMOHBAAAA|1999-10-28|2001-10-26|Really warm achievements|4.30|0.66|10015006|scholaramalgamalg #6|3|portable|10|Electronics|280|bareingable|N/A|19salmon902633599394|papaya|Carton|Unknown|73|antiableoughtcally| +6126|AAAAAAAAMOHBAAAA|2001-10-27||Scottish, natural ends feel slightly new, fine stud|1.82|0.66|10015006|univmaxi #7|10|pools|8|Sports|280|bareingable|N/A|97406921386590royal7|white|Cup|Unknown|42|callyableoughtcally| +6127|AAAAAAAAPOHBAAAA|1997-10-27||Afraid years must not leave more national, young managers. Somewhat complex prisoners apply now certainly consistent fans. Today f|41.49|14.10|1001002|amalgamalg #2|1|dresses|1|Women|786|callyeingation|extra large|7377610light59566089|moccasin|Pallet|Unknown|19|ationableoughtcally| +6128|AAAAAAAAAPHBAAAA|1997-10-27|2000-10-26|Suitable, historical workers sign too always different boxes. Good, unique lessons remain facilities; increasingly old persons find very nervous hills; small provi|8.00|5.76|8002009|importonameless #9|2|baseball|8|Sports|186|callyeingought|N/A|64263857peach7273534|rose|Box|Unknown|1|eingableoughtcally| +6129|AAAAAAAAAPHBAAAA|2000-10-27||Suitable, historical workers sign too always different boxes. Good, unique lessons remain facilities; increasingly old persons find very nervous hills; small provi|3.19|2.23|4001002|amalgedu pack #2|2|womens|4|Shoes|186|callyeingought|medium|64263857peach7273534|rosy|Ton|Unknown|31|n stableoughtcally| +6130|AAAAAAAACPHBAAAA|1997-10-27|1999-10-27|Nearby unaccep|13.09|4.05|6007003|brandcorp #3|7|pendants|6|Jewelry|186|callyeingought|N/A|white431344861142048|violet|Dram|Unknown|6|barprioughtcally| +6131|AAAAAAAACPHBAAAA|1999-10-28|2001-10-26|Nearby unaccep|3.04|4.05|6007003|importoimporto #2|7|shirts|2|Men|418|callyeingought|large|white431344861142048|ivory|Each|Unknown|5|oughtprioughtcally| +6132|AAAAAAAACPHBAAAA|2001-10-27||Nearby unaccep|21.22|4.05|6007003|univamalgamalg #13|7|memory|10|Electronics|444|callyeingought|N/A|8620midnight29957294|misty|Bunch|Unknown|6|ableprioughtcally| +6133|AAAAAAAAFPHBAAAA|1997-10-27||Other, royal parents might not proceed professional, similar transacti|5.17|3.82|8001004|amalgnameless #4|1|athletic shoes|8|Sports|224|eseableable|N/A|17pink42571352952116|saddle|Unknown|Unknown|45|priprioughtcally| +6134|AAAAAAAAGPHBAAAA|1997-10-27|2000-10-26|Prime, secondary systems |91.03|41.87|8005005|scholarnameless #5|5|fitness|8|Sports|184|eseeingought|N/A|6964gainsboro7335614|ivory|Cup|Unknown|62|eseprioughtcally| +6135|AAAAAAAAGPHBAAAA|2000-10-27||Prime, secondary systems |1.44|1.23|9013004|exportiunivamalg #4|5|self-help|9|Books|184|eseeingought|N/A|6964gainsboro7335614|tan|Tbl|Unknown|11|antiprioughtcally| +6136|AAAAAAAAIPHBAAAA|1997-10-27|1999-10-27|Narrow eyes shall affect in a goods. Addit|0.45|0.34|9012005|importounivamalg #5|12|home repair|9|Books|230|barpriable|N/A|0644plum029780043759|pink|N/A|Unknown|16|callyprioughtcally| +6137|AAAAAAAAIPHBAAAA|1999-10-28|2001-10-26|American, interested arts hide. No doubt human time|1.67|0.73|5004002|edu packscholar #2|12|classical|5|Music|230|barpriable|N/A|maroon09596826230576|thistle|Lb|Unknown|7|ationprioughtcally| +6138|AAAAAAAAIPHBAAAA|2001-10-27||American, interested arts hide. No doubt human time|8.26|5.94|7015009|scholarnameless #9|12|tables|7|Home|230|barpriable|N/A|690651turquoise75796|tan|Unknown|Unknown|42|eingprioughtcally| +6139|AAAAAAAALPHBAAAA|1997-10-27||Young problems generate now for example growing terms. Alone societies discuss; conclusions want between the departments. Workers must include perhaps ordinary complaints.|0.56|0.30|2001002|amalgimporto #2|1|accessories|2|Men|292|ablen stable|medium|16papaya850470318678|smoke|Gross|Unknown|50|n stprioughtcally| +6140|AAAAAAAAMPHBAAAA|1997-10-27|2000-10-26|Also individual arguments appear sufficiently levels; appropriate, economic churches know as insects. Settings take at least directly urgent conditions; then initial flowers must catc|4.61|3.22|5002001|importoscholar #1|2|country|5|Music|507|ationbaranti|N/A|purple40310668924403|turquoise|Each|Unknown|21|bareseoughtcally| +6141|AAAAAAAAMPHBAAAA|2000-10-27||Forms used to handle alone to a stores. Players would help now about average principles; indians may fill strong subjects. Fiscal, great details will not tell so eligible shapes|6.66|3.22|2001002|amalgimporto #2|2|accessories|2|Men|131|oughtpriought|economy|purple40310668924403|pink|Tsp|Unknown|41|oughteseoughtcally| +6142|AAAAAAAAOPHBAAAA|1997-10-27|1999-10-27|Later actual connections debate also clear, desperate men. Most awkward d|7.52|5.79|2003001|exportiimporto #1|3|pants|2|Men|315|antioughtpri|economy|4moccasin28276642945|spring|Oz|Unknown|32|ableeseoughtcally| +6143|AAAAAAAAOPHBAAAA|1999-10-28|2001-10-26|Later actual connections debate also clear, desperate men. Most awkward d|2.25|5.79|10015007|scholaramalgamalg #7|3|portable|10|Electronics|315|antioughtpri|N/A|4moccasin28276642945|floral|Gross|Unknown|34|prieseoughtcally| +6144|AAAAAAAAOPHBAAAA|2001-10-27||Separate cases come eventually soft countries. Subjects welcome nasty, other parties. Strong arms might not speak very local languages; desperate,|0.37|5.79|1002001|importoamalg #1|2|fragrances|1|Women|315|antioughtpri|small|4moccasin28276642945|lavender|Unknown|Unknown|1|eseeseoughtcally| +6145|AAAAAAAABAIBAAAA|||||||||dresses|||335|antipripri||8218653270honeydew72||Pound|Unknown||antieseoughtcally| +6146|AAAAAAAACAIBAAAA|1997-10-27|2000-10-26|Mediterranean laws used to come ago in the stages. Thousands approach home. Real views will not fend other, old books. Political, small limits could not go industries. Ordinary trousers re|64.92|25.96|2001001|amalgimporto #1|1|accessories|2|Men|140|bareseought|petite|4sandy97541249940282|tomato|Bunch|Unknown|26|callyeseoughtcally| +6147|AAAAAAAACAIBAAAA|2000-10-27||Flowers affect again eq|9.98|5.48|2001001|amalgnameless #8|1|athletic shoes|8|Sports|175|antiationought|N/A|4773saddle2125768632|rosy|Case|Unknown|98|ationeseoughtcally| +6148|AAAAAAAAEAIBAAAA|1997-10-27|1999-10-27|New demands could not assess in a boots. True, involved jobs realize in a members. Recently p|74.80|45.62|6011005|amalgbrand #5|11|semi-precious|6|Jewelry|172|ableationought|N/A|29799rosy69868739542|thistle|N/A|Unknown|1|eingeseoughtcally| +6149|AAAAAAAAEAIBAAAA|1999-10-28|2001-10-26|However fashionable officials know to a yea|0.44|45.62|5004002|edu packscholar #2|4|classical|5|Music|143|prieseought|N/A|29799rosy69868739542|dim|Pallet|Unknown|20|n steseoughtcally| +6150|AAAAAAAAEAIBAAAA|2001-10-27||However fashionable officials know to a yea|2.11|0.75|5004002|corpnameless #1|16|furniture|7|Home|820|prieseought|N/A|105178014556orange15|violet|Gross|Unknown|14|barantioughtcally| +6151|AAAAAAAAHAIBAAAA|1997-10-27||Classical, disciplinary a|5.01|3.80|10002015|importounivamalg #15|2|camcorders|10|Electronics|120|barableought|N/A|218879peru8239738030|slate|Ton|Unknown|5|oughtantioughtcally| +6152|AAAAAAAAIAIBAAAA|1997-10-27|2000-10-26|In particular public parents must submit long right, administrative systems. Voters make never thin, |4.05|1.37|10012016|importoamalgamalg #16|12|monitors|10|Electronics|965|anticallyn st|N/A|459056902slate153771|antique|Gross|Unknown|46|ableantioughtcally| +6153|AAAAAAAAIAIBAAAA|2000-10-27||Sorts must say about open, other details. Charges must look already on a offenders. Great, marvellous words used to see public, famous provisions; just other cells mus|0.43|1.37|10012016|edu packexporti #2|4|school-uniforms|3|Children|965|anticallyn st|medium|459056902slate153771|pink|Case|Unknown|18|priantioughtcally| +6154|AAAAAAAAKAIBAAAA|1997-10-27|1999-10-27|Successful, important men mean finally more than comfortable others; just, other children cope whole pupils. Terms should not worry italian, sophi|6.34|4.37|5003001|exportischolar #1|3|pop|5|Music|630|barprically|N/A|43875salmon338443667|spring|N/A|Unknown|13|eseantioughtcally| +6155|AAAAAAAAKAIBAAAA|1999-10-28|2001-10-26|Successful, important men mean finally more than comfortable others; just, other children cope whole pupils. Terms should not worry italian, sophi|8.36|4.76|2001002|amalgimporto #2|3|accessories|2|Men|406|barprically|petite|43875salmon338443667|spring|Each|Unknown|24|antiantioughtcally| +6156|AAAAAAAAKAIBAAAA|2001-10-27||Identical, |3.23|4.76|2001002|importonameless #7|3|baseball|8|Sports|688|eingeingcally|N/A|14571324164880cream1|red|Box|Unknown|62|callyantioughtcally| +6157|AAAAAAAANAIBAAAA|1997-10-27||Fond sentences must add in a documents. Also in|11.59|7.06|8013008|exportimaxi #8|13|sailing|8|Sports|471|oughtationese|N/A|202748gainsboro37168|tomato|Dozen|Unknown|40|ationantioughtcally| +6158|AAAAAAAAOAIBAAAA|1997-10-27|2000-10-26|Terms kiss now to a names. Bottles may not make also new, certain problems. Pregnant, special traditions would not capture purely. Definitely large others|2.70|1.32|7008009|namelessbrand #9|8|lighting|7|Home|150|barantiought|N/A|7linen24843894005123|smoke|Ounce|Unknown|64|eingantioughtcally| +6159|AAAAAAAAOAIBAAAA|2000-10-27||Too similar adults su|2.66|1.32|7008009|edu packbrand #10|8|curtains/drapes|7|Home|150|barantiought|N/A|7linen24843894005123|rosy|Gram|Unknown|30|n stantioughtcally| +6160|AAAAAAAAABIBAAAA|1997-10-27|1999-10-27|Healthy causes worry hardly useful matters; deep, mixed employers used to facilitate we|2.51|1.70|1001001|amalgamalg #1|1|dresses|1|Women|119|n stoughtought|medium|7391papaya6231660993|honeydew|Dram|Unknown|11|barcallyoughtcally| +6161|AAAAAAAAABIBAAAA|1999-10-28|2001-10-26|Healthy causes worry hardly useful matters; deep, mixed employers used to facilitate we|7.97|1.70|5003002|exportischolar #2|1|pop|5|Music|626|n stoughtought|N/A|7391papaya6231660993|pink|Pallet|Unknown|49|oughtcallyoughtcally| +6162|AAAAAAAAABIBAAAA|2001-10-27||Healthy causes worry hardly useful matters; deep, mixed employers used to facilitate we|2.65|1.48|5003002|edu packamalg #1|1|swimwear|1|Women|41|oughtese|medium|7391papaya6231660993|seashell|Bundle|Unknown|21|ablecallyoughtcally| +6163|AAAAAAAADBIBAAAA|1997-10-27||Jud|8.27|5.37|6015004|scholarbrand #4|15|custom|6|Jewelry|369|n stcallypri|N/A|45417656403793light4|sandy|Cup|Unknown|11|pricallyoughtcally| +6164|AAAAAAAAEBIBAAAA|1997-10-27|2000-10-26|Properties might follow muc|1.82|1.63|8004005|edu packnameless #5|4|camping|8|Sports|295|antin stable|N/A|2seashell13675801011|deep|Lb|Unknown|55|esecallyoughtcally| +6165|AAAAAAAAEBIBAAAA|2000-10-27||Properties might follow muc|93.04|79.08|8004005|exportimaxi #6|13|sailing|8|Sports|116|antin stable|N/A|2seashell13675801011|saddle|Lb|Unknown|66|anticallyoughtcally| +6166|AAAAAAAAGBIBAAAA|1997-10-27|1999-10-27|Problems shall end now perhaps warm students. Rare, various students stand expres|63.26|53.77|5001001|amalgscholar #1|1|rock|5|Music|48|eingese|N/A|5888277rosy894360178|indian|Ton|Unknown|4|callycallyoughtcally| +6167|AAAAAAAAGBIBAAAA|1999-10-28|2001-10-26|Problems shall end now perhaps warm students. Rare, various students stand expres|5.90|2.00|3001002|amalgexporti #2|1|newborn|3|Children|892|eingese|extra large|5888277rosy894360178|maroon|Tbl|Unknown|90|ationcallyoughtcally| +6168|AAAAAAAAGBIBAAAA|2001-10-27||Problems shall end now perhaps warm students. Rare, various students stand expres|1.08|2.00|3001002|exportiamalg #1|3|maternity|1|Women|924|eingese|small|4952424seashell44352|violet|Bunch|Unknown|24|eingcallyoughtcally| +6169|AAAAAAAAJBIBAAAA|1997-10-27||Heavy windows can claim difficulties; supreme responses might say there in a needs. Ago full kids stop scientific accounts. Mean institutions know|1.49|0.52|4002002|importoedu pack #2|2|mens|4|Shoes|962|ablecallyn st|large|36790754pink09360207|peru|Carton|Unknown|6|n stcallyoughtcally| +6170|AAAAAAAAKBIBAAAA|1997-10-27|2000-10-26|Automatic, fine seconds stop possible, liberal plans; brilliant processes cannot cost studies. Theories can belong well n|7.64|5.11|1004001|edu packamalg #1|4|swimwear|1|Women|569|n stcallyanti|medium|3138319lemon40879616|turquoise|Bundle|Unknown|7|barationoughtcally| +6171|AAAAAAAAKBIBAAAA|2000-10-27||Proud sets may not understand yet heavy periods. Years get together written, endless differences. Still practical |87.24|5.11|1004001|edu packedu pack #2|4|athletic|4|Shoes|569|n stcallyanti|medium|305198043236saddle10|pink|Lb|Unknown|9|oughtationoughtcally| +6172|AAAAAAAAMBIBAAAA|1997-10-27|1999-10-27|Plain experts cannot paint even. Elections call policies|4.10|1.84|10015002|scholaramalgamalg #2|15|portable|10|Electronics|136|callypriought|N/A|14879669khaki1364461|peru|Lb|Unknown|5|ableationoughtcally| +6173|AAAAAAAAMBIBAAAA|1999-10-28|2001-10-26|Slowly dear times learn then. Moreover free priorities might wrap once again from a cr|1.84|1.10|7001004|amalgbrand #4|15|bathroom|7|Home|136|callypriought|N/A|14879669khaki1364461|pale|Bundle|Unknown|38|priationoughtcally| +6174|AAAAAAAAMBIBAAAA|2001-10-27||Slowly dear times learn then. Moreover free priorities might wrap once again from a cr|2.41|0.81|10002004|importounivamalg #4|15|camcorders|10|Electronics|552|ableantianti|N/A|444126360096salmon07|purple|Case|Unknown|20|eseationoughtcally| +6175|AAAAAAAAPBIBAAAA|1997-10-27||Special workers produce even only, international parties. Cultures score from a colou|1.95|1.65|4004002|edu packedu pack #2|4|athletic|4|Shoes|574|eseationanti|medium|759621354812peach183|orange|Gram|Unknown|6|antiationoughtcally| +6176|AAAAAAAAACIBAAAA|1997-10-27|2000-10-26|Yet essential facts constitute following races. Grou|4.08|3.46|6005001|scholarcorp #1|5|earings|6|Jewelry|123|priableought|N/A|23189906518514hot628|mint|Cup|Unknown|45|callyationoughtcally| +6177|AAAAAAAAACIBAAAA|2000-10-27||Particularly large elections pose sadly direct, international words. Too possible traders could forget as. Social requirements think always |1.24|3.46|3003002|exportiexporti #2|3|toddlers|3|Children|123|priableought|large|23189906518514hot628|tan|Bunch|Unknown|5|ationationoughtcally| +6178|AAAAAAAACCIBAAAA|1997-10-27|1999-10-27|Sensitive colleagues encourage and so on only prime circumstances. Rapidly final colours help therefore always active years. Different years w|34.50|18.97|2001001|amalgimporto #1|1|accessories|2|Men|251|oughtantiable|extra large|4lemon88978395350965|thistle|Each|Unknown|43|eingationoughtcally| +6179|AAAAAAAACCIBAAAA|1999-10-28|2001-10-26|Sensitive colleagues encourage and so on only prime circumstances. Rapidly final colours help therefore always active years. Different years w|2.86|18.97|2001001|importoscholar #2|2|country|5|Music|251|oughtantiable|N/A|903402047051pale3040|royal|Case|Unknown|33|n stationoughtcally| +6180|AAAAAAAACCIBAAAA|2001-10-27||Sensitive colleagues encourage and so on only prime circumstances. Rapidly final colours help therefore always active years. Different years w|1.70|18.97|2001001|edu packmaxi #1|2|tennis|8|Sports|93|oughtantiable|N/A|903402047051pale3040|mint|Bundle|Unknown|12|bareingoughtcally| +6181|AAAAAAAAFCIBAAAA|1997-10-27||Jobs last again. Great, free men compare successful copies. Right students could get just high offenders. Specific, clear students could plead like a decisions. Little|2.64|1.66|5002002|importoscholar #2|2|country|5|Music|187|ationeingought|N/A|952white314297443620|thistle|Bunch|Unknown|9|oughteingoughtcally| +6182|AAAAAAAAGCIBAAAA|1997-10-27|2000-10-26|Widely likely firms will compromise constantly true young settings. Early, uncomfortable areas could panic. All olympic premises achieve even. Now islamic funds ought to emerge so only aware b|4.77|1.57|8003003|exportinameless #3|3|basketball|8|Sports|476|callyationese|N/A|314245850sienna05426|indian|Dozen|Unknown|21|ableeingoughtcally| +6183|AAAAAAAAGCIBAAAA|2000-10-27||Widely likely firms will compromise constantly true young settings. Early, uncomfortable areas could panic. All olympic premises achieve even. Now islamic funds ought to emerge so only aware b|1.42|0.56|4004002|edu packedu pack #2|3|athletic|4|Shoes|476|callyationese|petite|314245850sienna05426|peach|Case|Unknown|81|prieingoughtcally| +6184|AAAAAAAAICIBAAAA|1997-10-27|1999-10-27|Never regional years may get absently greatly red services. Dangerously fascinating profits must return very hands. Unlikely, |3.84|1.95|7016005|corpnameless #5|16|furniture|7|Home|53|prianti|N/A|7807734steel82474138|honeydew|Dram|Unknown|44|eseeingoughtcally| +6185|AAAAAAAAICIBAAAA|1999-10-28|2001-10-26||1.41||7016005|edu packamalg #2||swimwear|||900||medium|7807734steel82474138|||||antieingoughtcally| +6186|AAAAAAAAICIBAAAA|2001-10-27||Never regional years may get absently greatly red services. Dangerously fascinating profits must return very hands. Unlikely, |5.00|1.95|8014007|edu packmaxi #7|4|tennis|8|Sports|900|barbarn st|N/A|53792916turquoise375|wheat|Gram|Unknown|64|callyeingoughtcally| +6187|AAAAAAAALCIBAAAA|1997-10-27||Beautiful feelings work so well-known heels. Then hard clients take very different samples. Rather criminal stars might remove with the ideas. Problems should not help we|4.56|2.14|1002002|importoamalg #2|2|fragrances|1|Women|474|eseationese|medium|7green51651098334858|red|Pound|Unknown|54|ationeingoughtcally| +6188|AAAAAAAAMCIBAAAA|1997-10-27|2000-10-26|Small decisions shall not arise never low, concerned members. Cups appear extremely legal effects. Common|4.60|3.40|3004001|edu packexporti #1|4|school-uniforms|3|Children|273|priationable|medium|3633514796white05939|honeydew|Case|Unknown|26|eingeingoughtcally| +6189|AAAAAAAAMCIBAAAA|2000-10-27||Local ways make slightly to a guard|0.99|0.33|7008010|namelessbrand #10|8|lighting|7|Home|312|ableoughtpri|N/A|6060634290739powder1|azure|Lb|Unknown|1|n steingoughtcally| +6190|AAAAAAAAOCIBAAAA|1997-10-27|1999-10-27|Irish, hard recordings cannot make overnight then whole games. Frequently front workers would not keep constant, educational rivers. Faces must take under to a cuts. Inc seed|4.97|1.59|7015001|scholarnameless #1|15|tables|7|Home|832|ableprieing|N/A|rose7902487843752098|burlywood|Tsp|Unknown|26|barn stoughtcally| +6191|AAAAAAAAOCIBAAAA|1999-10-28|2001-10-26|Irish, hard recordings cannot make overnight then whole games. Frequently front workers would not keep constant, educational rivers. Faces must take under to a cuts. Inc seed|81.64|51.43|7015001|namelessbrand #2|8|lighting|7|Home|832|ableprieing|N/A|rose7902487843752098|slate|Gross|Unknown|33|oughtn stoughtcally| +6192|AAAAAAAAOCIBAAAA|2001-10-27||Irish, hard recordings cannot make overnight then whole games. Frequently front workers would not keep constant, educational rivers. Faces must take under to a cuts. Inc seed|1.18|51.43|7015001|scholarunivamalg #1|8|fiction|9|Books|832|ableprieing|N/A|264snow8997649919644|smoke|Ton|Unknown|48|ablen stoughtcally| +6193|AAAAAAAABDIBAAAA|1997-10-27||Fixed cars pull sometimes for a seats. Arms cover extensively because of a words. Financial ways know in|2.90|0.95|10006006|corpunivamalg #6|6|musical|10|Electronics|851|oughtantieing|N/A|264bisque15277617135|black|Ounce|Unknown|21|prin stoughtcally| +6194|AAAAAAAACDIBAAAA|1997-10-27|2000-10-26|Small results would go colours; sexual agencies ought to assure moreover unique premises; then complex provisions use often normal windows. Better educational girls should not believe however struct|9.78|8.41|9002003|importomaxi #3|2|business|9|Books|171|oughtationought|N/A|800misty987368609141|puff|Bunch|Unknown|27|esen stoughtcally| +6195|AAAAAAAACDIBAAAA|2000-10-27||Small results would go colours; sexual agencies ought to assure moreover unique premises; then complex provisions use often normal windows. Better educational girls should not believe however struct|0.55|8.41|9002003|exportiedu pack #2|2|kids|4|Shoes|171|oughtationought|medium|800misty987368609141|turquoise|Dram|Unknown|66|antin stoughtcally| +6196|AAAAAAAAEDIBAAAA|1997-10-27|1999-10-27|Weapons keep only scient|0.50|0.25|1001001|amalgamalg #1|1|dresses|1|Women|257|ationantiable|medium|88758wheat1483213246|steel|Dozen|Unknown|8|callyn stoughtcally| +6197|AAAAAAAAEDIBAAAA|1999-10-28|2001-10-26|Electric, difficult schemes retain much new, senior feelings. Difficult, fierce advantages can pay eyes; religious,|32.44|27.24|4004002|edu packedu pack #2|1|athletic|4|Shoes|257|ationantiable|extra large|88758wheat1483213246|turquoise|Bundle|Unknown|27|ationn stoughtcally| +6198|AAAAAAAAEDIBAAAA|2001-10-27||Electric, difficult schemes retain much new, senior feelings. Difficult, fierce advantages can pay eyes; religious,|3.40|27.24|5002001|importoscholar #1|1|country|5|Music|166|ationantiable|N/A|09109885chiffon54262|moccasin|Bundle|Unknown|25|eingn stoughtcally| +6199|AAAAAAAAHDIBAAAA|1997-10-27||Books give simply again technical terms. Fun deaths must not take below carefully true sons. Expensive arts could receive just about leaves. Central, payable reform|0.86|0.54|9011008|amalgunivamalg #8|11|cooking|9|Books|451|oughtantiese|N/A|91rosy98470970228721|slate|Tsp|Unknown|2|n stn stoughtcally| +6200|AAAAAAAAIDIBAAAA|1997-10-27|2000-10-26|Great, central provisions may not see on a habits. Possible, administrative figures shall dry here yet tory categories; standards stand twice. Responsible miners report on|2.35|1.85|7012001|importonameless #1|12|paint|7|Home|211|oughtoughtable|N/A|1061966184peach98062|honeydew|Ounce|Unknown|42|barbarablecally| +6201|AAAAAAAAIDIBAAAA|2000-10-27||Basic, proud seats must pick mo|7.55|3.17|7012001|importoscholar #2|12|country|5|Music|540|bareseanti|N/A|2253sienna9131170648|slate|Each|Unknown|2|oughtbarablecally| +6202|AAAAAAAAKDIBAAAA|1997-10-27|1999-10-27|Isolated styles follow in ge|1.91|0.91|6016003|corpbrand #3|16|consignment|6|Jewelry|239|n stpriable|N/A|912068817407668pink5|lavender|Cup|Unknown|57|ablebarablecally| +6203|AAAAAAAAKDIBAAAA|1999-10-28|2001-10-26|Isolated styles follow in ge|94.27|0.91|6016003|amalgimporto #2|16|accessories|2|Men|239|n stpriable|small|7896049thistle899352|peach|Cup|Unknown|67|pribarablecally| +6204|AAAAAAAAKDIBAAAA|2001-10-27||Isolated styles follow in ge|1.48|1.03|1003001|exportiamalg #1|16|maternity|1|Women|545|n stpriable|medium|8529603747664olive23|saddle|N/A|Unknown|50|esebarablecally| +6205|AAAAAAAANDIBAAAA|1997-10-27||Men should consider so breasts. Probably american functions teach. Bonds shall enable much extended members. Lines think to a prayers. Years could use; classes will think oft|2.91|0.90|4003002|exportiedu pack #2|3|kids|4|Shoes|317|ationoughtpri|N/A|7751forest8887312802|peru|Carton|Unknown|24|antibarablecally| +6206|AAAAAAAAODIBAAAA|1997-10-27|2000-10-26|Total, new changes used to go just workers. Used, public tears ought to apply. Affairs get for instance by a techniques. Mistakes may not talk eyes. Adequate, necessary towns could not mind yesterd|4.23|3.63|6007007|brandcorp #7|7|pendants|6|Jewelry|416|callyoughtese|N/A|632818880115linen027|sky|Box|Unknown|2|callybarablecally| +6207|AAAAAAAAODIBAAAA|2000-10-27||Total, new changes used to go just workers. Used, public tears ought to apply. Affairs get for instance by a techniques. Mistakes may not talk eyes. Adequate, necessary towns could not mind yesterd|9.81|3.63|6007007|exporticorp #8|3|gold|6|Jewelry|250|callyoughtese|N/A|632818880115linen027|sky|Lb|Unknown|2|ationbarablecally| +6208|AAAAAAAAAEIBAAAA|1997-10-27|1999-10-27|Greek prizes confirm soon. Very moves give as yet. Decisio|0.23|0.20|1003001|exportiamalg #1|3|maternity|1|Women|7|ation|small|grey0974592843840201|yellow|Carton|Unknown|9|eingbarablecally| +6209|AAAAAAAAAEIBAAAA|1999-10-28|2001-10-26|Separate residents could live potentially. Colours clear late; old groups threaten forwards inc, british hours. Growing policies learn at a |7.50|5.92|5002002|importoscholar #2|2|country|5|Music|180|ation|N/A|grey0974592843840201|lime|Ton|Unknown|45|n stbarablecally| +6210|AAAAAAAAAEIBAAAA|2001-10-27||Separate residents could live potentially. Colours clear late; old groups threaten forwards inc, british hours. Growing policies learn at a |6.32|3.98|5002002|importoedu pack #1|2|mens|4|Shoes|180|bareingought|large|grey0974592843840201|peach|Each|Unknown|53|baroughtablecally| +6211|AAAAAAAADEIBAAAA|1997-10-27||Then attractive practices establish also at a issues; more independent records can inject even weak confidential bands. General parts will come culturally national standards.|8.90|5.51|9011008|amalgunivamalg #8|11|cooking|9|Books|77|ationation|N/A|356964499244smoke471|slate|Dozen|Unknown|66|oughtoughtablecally| +6212|AAAAAAAAEEIBAAAA|1997-10-27|2000-10-26|Unnecessary types intervene little close ages. Reasons find accordingly however whole resources; birds join fl|2.46|1.03|7015003|scholarnameless #3|15|tables|7|Home|146|callyeseought|N/A|941920turquoise06613|royal|Ton|Unknown|5|ableoughtablecally| +6213|AAAAAAAAEEIBAAAA|2000-10-27||Unnecessary types intervene little close ages. Reasons find accordingly however whole resources; birds join fl|38.27|1.03|10002009|importounivamalg #9|15|camcorders|10|Electronics|617|callyeseought|N/A|7steel40680900049449|salmon|Carton|Unknown|29|prioughtablecally| +6214|AAAAAAAAGEIBAAAA|1997-10-27|1999-10-27|European, new feelings may tell for a things. New eyes may manage really sexual, rural skills. Low events inc|3.66|3.29|10002010|importounivamalg #10|2|camcorders|10|Electronics|103|pribarought|N/A|81370pale90111894686|peach|Gross|Unknown|92|eseoughtablecally| +6215|AAAAAAAAGEIBAAAA|1999-10-28|2001-10-26|Kids should not spare originally british newspapers. Jobs co|7.58|3.29|10002010|maxinameless #6|9|optics|8|Sports|44|eseese|N/A|81370pale90111894686|purple|Gram|Unknown|3|antioughtablecally| +6216|AAAAAAAAGEIBAAAA|2001-10-27||Kids should not spare originally british newspapers. Jobs co|6.93|4.50|10002010|edu packedu pack #1|9|athletic|4|Shoes|44|eseese|extra large|1937823751141sienna2|yellow|Ton|Unknown|18|callyoughtablecally| +6217|AAAAAAAAJEIBAAAA|1997-10-27||Away red parties say no doubt; often rapid surfaces must form almost buildings. All scottish sessions shall make|8.96|6.72|5003002|exportischolar #2|3|pop|5|Music|213|prioughtable|N/A|04643998papaya746515|royal|Box|Unknown|14|ationoughtablecally| +6218|AAAAAAAAKEIBAAAA|1997-10-27|2000-10-26|Main sections may buy particular trains. Things show naturally recent levels. Necessary pounds will give at least large families. Certain persons used to depend possibly safe regions. Fina|5.99|4.37|3001001|amalgexporti #1|1|newborn|3|Children|365|anticallypri|petite|1423083359086white27|olive|Gram|Unknown|29|eingoughtablecally| +6219|AAAAAAAAKEIBAAAA|2000-10-27||Main sections may buy particular trains. Things show naturally recent levels. Necessary pounds will give at least large families. Certain persons used to depend possibly safe regions. Fina|2.71|1.00|8016010|corpmaxi #10|1|golf|8|Sports|365|anticallypri|N/A|339609470seashell166|sky|Each|Unknown|48|n stoughtablecally| +6220|AAAAAAAAMEIBAAAA|1997-10-27|1999-10-27|Sometimes organisational days could give legally. National towns shall work assistant tools. Downstairs local c|5.74|4.76|4004001|edu packedu pack #1|4|athletic|4|Shoes|386|callyeingpri|extra large|438turquoise71656930|sienna|Case|Unknown|21|barableablecally| +6221|AAAAAAAAMEIBAAAA|1999-10-28|2001-10-26|Sometimes organisational days could give legally. National towns shall work assistant tools. Downstairs local c|9.22|4.76|4004001|importounivamalg #17|4|camcorders|10|Electronics|556|callyeingpri|N/A|438turquoise71656930|turquoise|Gross|Unknown|6|oughtableablecally| +6222|AAAAAAAAMEIBAAAA|2001-10-27||Police achieve quite; hard, administrative fingers include years. Irish, excellent trees issue upstream able mothers. Stores used to like advertisements. Services mediate perhap|9.80|4.76|4004001|importomaxi #7|4|business|9|Books|556|callyeingpri|N/A|5184601050maroon0717|smoke|Lb|Unknown|19|ableableablecally| +6223|AAAAAAAAPEIBAAAA|1997-10-27||Quiet, political chemicals shall think to a shapes. Considerable patients will speak completely then special students. Dear, local men sit busy hopes. Too correct letters used to underst|1.70|0.66|1002002|importoamalg #2|2|fragrances|1|Women|604|esebarcally|economy|07250366326229misty2|deep|Gross|Unknown|56|priableablecally| +6224|AAAAAAAAAFIBAAAA|1997-10-27|2000-10-26|Different, fresh structures used to mean big schools; small, opposite findings drag |6.99|3.84|9005009|scholarmaxi #9|5|history|9|Books|853|priantieing|N/A|30774456707lawn47721|turquoise|Case|Unknown|16|eseableablecally| +6225|AAAAAAAAAFIBAAAA|2000-10-27||Different, fresh structures used to mean big schools; small, opposite findings drag |2.00|0.68|9005009|brandmaxi #10|7|reference|9|Books|853|priantieing|N/A|60815524225spring494|sienna|Carton|Unknown|69|antiableablecally| +6226|AAAAAAAACFIBAAAA|1997-10-27|1999-10-27|Fears might pay|8.69|3.56|2001001|amalgimporto #1|1|accessories|2|Men|20|barable|medium|8698chocolate6310940|snow|Carton|Unknown|28|callyableablecally| +6227|AAAAAAAACFIBAAAA|1999-10-28|2001-10-26|Relevant, relative materials apply so at a bars. Always long years may not develop beyond active far businesses. Existing kings camp military relationships; profits review; realistic,|2.07|0.62|2001001|edu packnameless #8|1|camping|8|Sports|20|barable|N/A|233yellow65568194357|saddle|Cup|Unknown|30|ationableablecally| +6228|AAAAAAAACFIBAAAA|2001-10-27||Relevant, relative materials apply so at a bars. Always long years may not develop beyond active far businesses. Existing kings camp military relationships; profits review; realistic,|9.36|0.62|5004001|edu packscholar #1|1|classical|5|Music|448|eingeseese|N/A|233yellow65568194357|sky|Dozen|Unknown|50|eingableablecally| +6229|AAAAAAAAFFIBAAAA|1997-10-27||Levels shield wonderful feelings. Red children cannot come mainl|7.88|2.75|1004002|edu packamalg #2|4|swimwear|1|Women|30|barpri|small|28gainsboro519162941|peach|Ton|Unknown|10|n stableablecally| +6230|AAAAAAAAGFIBAAAA|1997-10-27|2000-10-26|Regional, standard followers exercise as recent, different facts. Discussions bear early men; now good instruments might not admit just better red cuts.|4.68|3.41|8008001|namelessnameless #1|8|outdoor|8|Sports|577|ationationanti|N/A|553661925415purple72|tomato|Case|Unknown|64|barpriablecally| +6231|AAAAAAAAGFIBAAAA|2000-10-27||Alter|3.51|2.77|8008001|scholarbrand #8|15|custom|6|Jewelry|577|ationationanti|N/A|90317020lime85091191|tomato|Carton|Unknown|3|oughtpriablecally| +6232|AAAAAAAAIFIBAAAA|1997-10-27|1999-10-27|Patient, silly efforts pay at a hero|9.63|5.58|6009001|maxicorp #1|9|womens watch|6|Jewelry|424|eseableese|N/A|32672493419puff27769|salmon|Each|Unknown|22|ablepriablecally| +6233|AAAAAAAAIFIBAAAA|1999-10-28|2001-10-26|Islamic, average states cannot want then to a visitors. Finally normal questions get. Secret, good requirements will pursue m|1.90|5.58|6009001|importoamalg #2|2|fragrances|1|Women|424|eseableese|extra large|32672493419puff27769|pink|Gram|Unknown|78|pripriablecally| +6234|AAAAAAAAIFIBAAAA|2001-10-27||Islamic, average states cannot want then to a visitors. Finally normal questions get. Secret, good requirements will pursue m|1.86|5.58|2002001|importoimporto #1|2|shirts|2|Men|424|eseableese|extra large|32672493419puff27769|honeydew|Box|Unknown|46|esepriablecally| +6235|AAAAAAAALFIBAAAA|1997-10-27||Other implications w|0.86|0.61|3001002|amalgexporti #2|1|newborn|3|Children|332|ablepripri|small|262901spring40628249|indian|Pound|Unknown|55|antipriablecally| +6236|AAAAAAAAMFIBAAAA|1997-10-27|2000-10-26|Main parents realise places. Radical days could not seem decisions. As well nati|8.77|4.12|1001001|amalgamalg #1|1|dresses|1|Women|333|pripripri|medium|10736886159white2523|maroon|Ounce|Unknown|1|callypriablecally| +6237|AAAAAAAAMFIBAAAA|2000-10-27||Main parents realise places. Radical days could not seem decisions. As well nati|9.90|4.12|1001001|exportiamalg #2|3|maternity|1|Women|85|antieing|large|0450402523papaya1989|seashell|Ton|Unknown|45|ationpriablecally| +6238|AAAAAAAAOFIBAAAA|1997-10-27|1999-10-27|Shallow, vocational efforts used to give very part-time programmes. Only months ought to know; participants will not take then even natural events. Influences take al|7.44|2.97|8007009|brandnameless #9|7|hockey|8|Sports|34|esepri|N/A|1940188047salmon4899|plum|Tsp|Unknown|8|eingpriablecally| +6239|AAAAAAAAOFIBAAAA|1999-10-28|2001-10-26|Shallow, vocational efforts used to give very part-time programmes. Only months ought to know; participants will not take then even natural events. Influences take al|4.92|3.59|8010010|univmaxi #10|7|pools|8|Sports|34|esepri|N/A|1940188047salmon4899|pale|Dram|Unknown|26|n stpriablecally| +6240|AAAAAAAAOFIBAAAA|2001-10-27||Shallow, vocational efforts used to give very part-time programmes. Only months ought to know; participants will not take then even natural events. Influences take al|1.06|3.59|6004001|edu packcorp #1|7|bracelets|6|Jewelry|231|oughtpriable|N/A|1940188047salmon4899|red|Bundle|Unknown|39|bareseablecally| +6241|AAAAAAAABGIBAAAA|1997-10-27||Very services would tell. Model weeks fight most. Public, rough experiments un|5.65|4.40|10015003|scholaramalgamalg #3|15|portable|10|Electronics|511|oughtoughtanti|N/A|8lime846417555332833|sandy|Box|Unknown|29|oughteseablecally| +6242|AAAAAAAACGIBAAAA|1997-10-27|2000-10-26|Indeed glad hands should survive significantly. Senior, good clubs get once old beds. Co|1.67|0.71|10006004|corpunivamalg #4|6|musical|10|Electronics|188|eingeingought|N/A|139125062turquoise83|wheat|Case|Unknown|94|ableeseablecally| +6243|AAAAAAAACGIBAAAA|2000-10-27||Indeed glad hands should survive significantly. Senior, good clubs get once old beds. Co|4.65|0.71|1004002|edu packamalg #2|4|swimwear|1|Women|174|eingeingought|extra large|139125062turquoise83|orange|Bundle|Unknown|5|prieseablecally| +6244|AAAAAAAAEGIBAAAA|1997-10-27|1999-10-27|Phenomena argue on a sessions; almost left feet make perhaps. Ago separate members define so equal countries. Fiscal,|4.14|3.47|4003001|exportiedu pack #1|3|kids|4|Shoes|253|priantiable|petite|9422maroon1857384156|antique|Bundle|Unknown|61|eseeseablecally| +6245|AAAAAAAAEGIBAAAA|1999-10-28|2001-10-26|Phenomena argue on a sessions; almost left feet make perhaps. Ago separate members define so equal countries. Fiscal,|1.92|3.47|6008006|namelesscorp #6|3|mens watch|6|Jewelry|253|priantiable|N/A|173slate156261420690|snow|Box|Unknown|25|antieseablecally| +6246|AAAAAAAAEGIBAAAA|2001-10-27||Phenomena argue on a sessions; almost left feet make perhaps. Ago separate members define so equal countries. Fiscal,|7.56|3.47|2003001|exportiimporto #1|3|pants|2|Men|253|priantiable|small|salmon32201627143356|wheat|Pallet|Unknown|32|callyeseablecally| +6247|AAAAAAAAHGIBAAAA|1997-10-27||Normally welsh sports s|1.33|0.86|4002002|importoedu pack #2|2|mens|4|Shoes|620|barablecally|economy|7139555gainsboro9516|saddle|Dozen|Unknown|28|ationeseablecally| +6248|AAAAAAAAIGIBAAAA|1997-10-27|2000-10-26|Arms stand very significant changes. Effects look. Women compensate well notable details; immediately public hotels know more effective years. C|7.76|2.48|10004010|edu packunivamalg #10|4|audio|10|Electronics|661|oughtcallycally|N/A|steel503715760852697|snow|Pallet|Unknown|12|eingeseablecally| +6249|AAAAAAAAIGIBAAAA|2000-10-27||Americans broadcast dangerous agents. Political, silent miners ought to tell already countries; plain, new departments will develop later then de|2.49|2.48|10004010|importoimporto #2|2|shirts|2|Men|78|oughtcallycally|small|steel503715760852697|gainsboro|Unknown|Unknown|8|n steseablecally| +6250|AAAAAAAAKGIBAAAA|1997-10-27|1999-10-27|Green terms publish angry, healthy dog|2.61|1.09|10010012|univamalgamalg #12|10|memory|10|Electronics|640|baresecally|N/A|657630295sienna87167|sienna|Cup|Unknown|51|barantiablecally| +6251|AAAAAAAAKGIBAAAA|1999-10-28|2001-10-26|Green terms publish angry, healthy dog|1.59|0.47|2004002|edu packimporto #2|4|sports-apparel|2|Men|640|baresecally|medium|657630295sienna87167|powder|Each|Unknown|5|oughtantiablecally| +6252|AAAAAAAAKGIBAAAA|2001-10-27||Particular ma|5.89|0.47|5003001|exportischolar #1|4|pop|5|Music|640|baresecally|N/A|657630295sienna87167|peach|Case|Unknown|45|ableantiablecally| +6253|AAAAAAAANGIBAAAA|1997-10-27||There single police define mainly extraordinary, violent eyes. Files ought to see certainly responsible others; then comp|1.39|0.87|1003002|exportiamalg #2|3|maternity|1|Women|83|prieing|extra large|453881833145331plum5|slate|Pallet|Unknown|1|priantiablecally| +6254|AAAAAAAAOGIBAAAA|1997-10-27|2000-10-26|Social democrats begin more inside the results. Important, particular minutes make in front of the relations. |52.52|24.68|9007003|brandmaxi #3|7|reference|9|Books|359|n stantipri|N/A|1slate61694334975834|yellow|Gram|Unknown|4|eseantiablecally| +6255|AAAAAAAAOGIBAAAA|2000-10-27||Firm weeks give ju|9.21|4.88|9007003|corpbrand #6|7|rugs|7|Home|359|n stantipri|N/A|9sienna4691175199786|rose|Lb|Unknown|21|antiantiablecally| +6256|AAAAAAAAAHIBAAAA|1997-10-27|1999-10-27|Silly assessments affirm critical, important years; features want else. Odd levels will not spell more indeed flat others. Written, accurate decis|66.93|30.11|6003001|exporticorp #1|3|gold|6|Jewelry|157|ationantiought|N/A|851427963439yellow61|indian|Pound|Unknown|32|callyantiablecally| +6257|AAAAAAAAAHIBAAAA|1999-10-28|2001-10-26|Silly assessments affirm critical, important years; features want else. Odd levels will not spell more indeed flat others. Written, accurate decis|1.77|30.11|6003001|exportiamalg #2|3|maternity|1|Women|157|ationantiought|extra large|851427963439yellow61|purple|Dozen|Unknown|24|ationantiablecally| +6258|AAAAAAAAAHIBAAAA|2001-10-27||Silly assessments affirm critical, important years; features want else. Odd levels will not spell more indeed flat others. Written, accurate decis|94.18|77.22|6003001|importonameless #9|2|baseball|8|Sports|198|ationantiought|N/A|071504398241901rose5|sienna|Ounce|Unknown|11|eingantiablecally| +6259|AAAAAAAADHIBAAAA|1997-10-27||Eyes must not sound. Classes take. Best pleased methods provi|0.17|0.06|9005008|scholarmaxi #8|5|history|9|Books|557|ationantianti|N/A|58262155seashell1859|light|Tbl|Unknown|46|n stantiablecally| +6260|AAAAAAAAEHIBAAAA|1997-10-27|2000-10-26|Now corporate weeks control significantly all american seconds. Higher |3.81|2.17|2004001|edu packimporto #1|4|sports-apparel|2|Men|538|eingprianti|medium|954725409red89875016|yellow|Oz|Unknown|18|barcallyablecally| +6261|AAAAAAAAEHIBAAAA|2000-10-27||As regional times could exist now. Physical, appropriate objectives cannot combine residential, different matters. Absolute months form; almost positive years cannot obtain |1.44|2.17|2004001|scholarbrand #2|5|blinds/shades|7|Home|538|eingprianti|N/A|951952416turquoise64|puff|Ton|Unknown|64|oughtcallyablecally| +6262|AAAAAAAAGHIBAAAA|1997-10-27|1999-10-27|Young subjects could bring necessarily; things protect for a employers. |4.35|3.65|8001003|amalgnameless #3|1|athletic shoes|8|Sports|445|antieseese|N/A|1940074pale115153222|turquoise|Bundle|Unknown|46|ablecallyablecally| +6263|AAAAAAAAGHIBAAAA|1999-10-28|2001-10-26|Issues clear senior, minor towns. Huge women should accept feet. Very full standards bring normal, sm|0.63|0.37|2002002|importoimporto #2|1|shirts|2|Men|445|antieseese|extra large|1940074pale115153222|royal|Ton|Unknown|48|pricallyablecally| +6264|AAAAAAAAGHIBAAAA|2001-10-27||Issues clear senior, minor towns. Huge women should accept feet. Very full standards bring normal, sm|2.73|0.37|4001001|amalgedu pack #1|1|womens|4|Shoes|445|antieseese|medium|83592055magenta67589|thistle|Cup|Unknown|10|esecallyablecally| +6265|AAAAAAAAJHIBAAAA|1997-10-27||Ever impressive sounds shall not decide long cards. Readers accept still w|2.46|1.72|8001006|amalgnameless #6|1|athletic shoes|8|Sports|75|antiation|N/A|22430145251847tan987|plum|Tsp|Unknown|13|anticallyablecally| +6266|AAAAAAAAKHIBAAAA|1997-10-27|2000-10-26|Fundamental, other studies buy formerly from a services. Psyc|2.63|2.36|9010003|univunivamalg #3|10|travel|9|Books|161|oughtcallyought|N/A|937890puff6316255104|snow|Ton|Unknown|56|callycallyablecally| +6267|AAAAAAAAKHIBAAAA|2000-10-27||Fundamental, other studies buy formerly from a services. Psyc|6.46|2.36|9010003|edu packcorp #4|10|bracelets|6|Jewelry|161|oughtcallyought|N/A|937890puff6316255104|rose|Bunch|Unknown|63|ationcallyablecally| +6268|AAAAAAAAMHIBAAAA|1997-10-27|1999-10-27|Labour, happy rates stop details. Purposes say small, dead times; tickets will act hopefully yesterday considerable products. Competitive others stay with an purposes. Always personal guns might ri|2.78|1.66|9011005|amalgunivamalg #5|11|cooking|9|Books|749|n steseation|N/A|56peach6403086182930|puff|Tsp|Unknown|20|eingcallyablecally| +6269|AAAAAAAAMHIBAAAA|1999-10-28|2001-10-26|Labour, happy rates stop details. Purposes say small, dead times; tickets will act hopefully yesterday considerable products. Competitive others stay with an purposes. Always personal guns might ri|2.12|1.66|1004002|edu packamalg #2|11|swimwear|1|Women|749|n steseation|large|592483rosy9934768118|rosy|Tsp|Unknown|2|n stcallyablecally| +6270|AAAAAAAAMHIBAAAA|2001-10-27||New, tradition|2.85|1.66|1004002|brandnameless #1|11|hockey|8|Sports|615|n steseation|N/A|592483rosy9934768118|smoke|N/A|Unknown|62|barationablecally| +6271|AAAAAAAAPHIBAAAA|1997-10-27||Suddenly toxic trials indicate tender, light shares. |5.02|2.96|9012008|importounivamalg #8|12|home repair|9|Books|249|n steseable|N/A|4119navy553982208318|olive|Tsp|Unknown|60|oughtationablecally| +6272|AAAAAAAAAIIBAAAA|1997-10-27|2000-10-26|Necessary, unique programmes announce very. Proper arms used to mean under a generations. Political sorts could not cancel to a |4.92|3.29|4004001|edu packedu pack #1|4|athletic|4|Shoes|816|callyoughteing|medium|68568548slate8120891|slate|Lb|Unknown|4|ableationablecally| +6273|AAAAAAAAAIIBAAAA|2000-10-27||Necessary, unique programmes announce very. Proper arms used to mean under a generations. Political sorts could not cancel to a |1.98|0.99|3002002|importoexporti #2|4|infants|3|Children|471|oughtationese|economy|68568548slate8120891|magenta|Cup|Unknown|61|priationablecally| +6274|AAAAAAAACIIBAAAA|1997-10-27|1999-10-27|English, white costs feel furth|0.88|0.38|3002001|importoexporti #1|2|infants|3|Children|258|eingantiable|economy|olive187093086954265|hot|Tsp|Unknown|41|eseationablecally| +6275|AAAAAAAACIIBAAAA|1999-10-28|2001-10-26|Dark matters ought to eat still early comments. Simple, only careers would increase. Participants should think |43.62|0.38|7014006|edu packnameless #6|2|glassware|7|Home|12|ableought|N/A|olive187093086954265|yellow|Bundle|Unknown|41|antiationablecally| +6276|AAAAAAAACIIBAAAA|2001-10-27||Large, common secrets build relatives.|7.40|6.36|6011005|amalgbrand #5|11|semi-precious|6|Jewelry|12|ableought|N/A|27049621sienna274088|slate|Gross|Unknown|18|callyationablecally| +6277|AAAAAAAAFIIBAAAA|1997-10-27||Facts finish other passengers. Similar societies live personally. Visitors would manage light, new rocks; parts can brin|8.20|2.87|8016008|corpmaxi #8|16|golf|8|Sports|292|ablen stable|N/A|472692128sandy375359|slate|Pound|Unknown|25|ationationablecally| +6278|AAAAAAAAGIIBAAAA|1997-10-27|2000-10-26|Children would take decisive, high leaders; keys must not return local, extra components. Forms contend hi|1.49|0.64|1003001|exportiamalg #1|3|maternity|1|Women|228|eingableable|extra large|333pale2449615077663|misty|Bundle|Unknown|22|eingationablecally| +6279|AAAAAAAAGIIBAAAA|2000-10-27||Good, marvellous women affect services. Perhaps long representatives identify dry figures; political days ought to get. Universal, co|1.34|0.42|10005007|scholarunivamalg #7|3|karoke|10|Electronics|47|eingableable|N/A|plum2525536873049883|chartreuse|Tbl|Unknown|7|n stationablecally| +6280|AAAAAAAAIIIBAAAA|1997-10-27|1999-10-27|Supplies give much common males; methods turn ways; common, useful users may operate financially by the teachers; weeks complete in general. National, good neighbours should not pursue |0.67|0.43|8005001|scholarnameless #1|5|fitness|8|Sports|210|baroughtable|N/A|70909lace68950098352|tomato|Gram|Unknown|21|bareingablecally| +6281|AAAAAAAAIIIBAAAA|1999-10-28|2001-10-26|Supplies give much common males; methods turn ways; common, useful users may operate financially by the teachers; weeks complete in general. National, good neighbours should not pursue |2.02|0.43|8005001|edu packedu pack #2|4|athletic|4|Shoes|210|baroughtable|petite|70909lace68950098352|rosy|Unknown|Unknown|59|oughteingablecally| +6282|AAAAAAAAIIIBAAAA|2001-10-27||Available, open abilities would locate local, moral hearts. R|8.23|6.58|1003001|exportiamalg #1|3|maternity|1|Women|210|baroughtable|petite|70909lace68950098352|sienna|Bundle|Unknown|30|ableeingablecally| +6283|AAAAAAAALIIBAAAA|1997-10-27||Expressions could support yesterday widespread comments. Also level features used to cook to a places. Materials gather. Better different members reflect more than systems. Here|0.65|0.50|4002002|importoedu pack #2|2|mens|4|Shoes|308|eingbarpri|medium|219828062660sienna46|sandy|Oz|Unknown|14|prieingablecally| +6284|AAAAAAAAMIIBAAAA|1997-10-27|2000-10-26|Evil, public managers will take mutual days. Loyal wings might not approve quite interested pressures; financial pairs could work lat|2.09|1.48|1003001|exportiamalg #1|3|maternity|1|Women|327|ationablepri|medium|6673724243purple7970|royal|Ounce|Unknown|33|eseeingablecally| +6285|AAAAAAAAMIIBAAAA|2000-10-27||Persistent centres ought to go rights. Old, digital tears tell exciting refugees. Appropriate contracts will m|9.49|8.54|10014013|edu packamalgamalg #13|3|automotive|10|Electronics|327|ationablepri|N/A|003211817puff2188697|mint|Box|Unknown|12|antieingablecally| +6286|AAAAAAAAOIIBAAAA|1997-10-27|1999-10-27|Also acute signs may not address members. Elsewhere straightforward words limit hands. Separately scotti|1.09|0.37|10010014|univamalgamalg #14|10|memory|10|Electronics|256|callyantiable|N/A|3206726drab158967235|peach|Pound|Unknown|68|callyeingablecally| +6287|AAAAAAAAOIIBAAAA|1999-10-28|2001-10-26|Too scottish laws tea|0.61|0.37|7007008|brandbrand #8|7|decor|7|Home|74|callyantiable|N/A|3206726drab158967235|red|Dozen|Unknown|88|ationeingablecally| +6288|AAAAAAAAOIIBAAAA|2001-10-27||Too scottish laws tea|6.45|2.25|7007008|scholarnameless #9|7|fitness|8|Sports|74|callyantiable|N/A|99146420violet939787|sky|Case|Unknown|65|eingeingablecally| +6289|AAAAAAAABJIBAAAA|1997-10-27||Video-taped functions mentio|0.95|0.69|4004002|edu packedu pack #2|4|athletic|4|Shoes|647|ationesecally|petite|99sky744346866980766|tan|Box|Unknown|23|n steingablecally| +6290|AAAAAAAACJIBAAAA|1997-10-27|2000-10-26|Independent, constant prices smoke; homes might form now accounts. Other ranks could matter again outside the honours. Close, religious methods apply |4.55|3.91|8016001|corpmaxi #1|16|golf|8|Sports|585|antieinganti|N/A|69999rose11142388727|white|Gram|Unknown|10|barn stablecally| +6291|AAAAAAAACJIBAAAA|2000-10-27||Medite|0.38|3.91|8016001|univmaxi #2|10|pools|8|Sports|585|antieinganti|N/A|69999rose11142388727|chiffon|Ton|Unknown|4|oughtn stablecally| +6292|AAAAAAAAEJIBAAAA|1997-10-27|1999-10-27|H|1.94|1.33|3001001|amalgexporti #1|1|newborn|3|Children|563|pricallyanti|small|351violet82806018045|navy|Dozen|Unknown|69|ablen stablecally| +6293|AAAAAAAAEJIBAAAA|1999-10-28|2001-10-26|Futures manage acts; much criminal contracts ought to go relatively new, young years; quite christian copies may get. Fresh houses bury enough; twin, nice ends will not agree specific individual|1.46|0.78|10001004|amalgunivamalg #4|1|cameras|10|Electronics|133|pripriought|N/A|351violet82806018045|royal|Tbl|Unknown|10|prin stablecally| +6294|AAAAAAAAEJIBAAAA|2001-10-27||Futures manage acts; much criminal contracts ought to go relatively new, young years; quite christian copies may get. Fresh houses bury enough; twin, nice ends will not agree specific individual|2.58|0.78|10006005|corpunivamalg #5|1|musical|10|Electronics|225|antiableable|N/A|red53851449272413794|purple|Ounce|Unknown|44|esen stablecally| +6295|AAAAAAAAHJIBAAAA|1997-10-27||Systems could go drugs. Forces say more; wings shall not tell too relatively small scientists. Then mad blues flow. Complete, tremendous officers would not explain indeed years. Exc|9.66|4.25|7016006|corpnameless #6|16|furniture|7|Home|300|barbarpri|N/A|5798spring3699073841|orange|Ounce|Unknown|26|antin stablecally| +6296|AAAAAAAAIJIBAAAA|1997-10-27|2000-10-26|New, total organizations call at a aspects. Rates go often details. Local, magic services choose much with a police. Authorities push for a windows. Lovers must believe currently ltd. |28.77|21.28|7009007|maxibrand #7|9|mattresses|7|Home|443|prieseese|N/A|4048652429034grey989|royal|Ounce|Unknown|84|callyn stablecally| +6297|AAAAAAAAIJIBAAAA|2000-10-27||New, total organizations call at a aspects. Rates go often details. Local, magic services choose much with a police. Authorities push for a windows. Lovers must believe currently ltd. |2.20|0.77|7009007|edu packbrand #8|9|curtains/drapes|7|Home|443|prieseese|N/A|099053purple82878549|pale|Lb|Unknown|12|ationn stablecally| +6298|AAAAAAAAKJIBAAAA|1997-10-27|1999-10-27|Free characteristics could satisfy once workers. English, external fears keep ideas. Wages see by a markets. Really foreign carers help clubs. Real, popular specimens ena|2.87|2.52|5003001|exportischolar #1|3|pop|5|Music|167|ationcallyought|N/A|91powder466289886737|magenta|Bundle|Unknown|26|eingn stablecally| +6299|AAAAAAAAKJIBAAAA|1999-10-28|2001-10-26|Future plants might stop small services. Together brief songs could tur|2.25|2.52|3004002|edu packexporti #2|3|school-uniforms|3|Children|358|eingantipri|medium|91powder466289886737|tan|Unknown|Unknown|45|n stn stablecally| +6300|AAAAAAAAKJIBAAAA|2001-10-27||Due, united roots participate fairly young, main inhabitants; soon fiscal members may support nearly careful blind assets. Islamic contributions must belie|7.08|2.52|2001001|amalgimporto #1|3|accessories|2|Men|358|eingantipri|petite|78395turquoise861717|spring|Oz|Unknown|63|barbarprically| +6301|AAAAAAAANJIBAAAA|1997-10-27||Wonderful, clear years used to manage free, natural children. Ahead personal frames allow else in a estates; long, old theories shoul|0.11|0.06|10011012|amalgamalgamalg #12|11|disk drives|10|Electronics|349|n stesepri|N/A|18878789641saddle342|lace|Lb|Unknown|64|oughtbarprically| +6302|AAAAAAAAOJIBAAAA|1997-10-27|2000-10-26|Holy users teach yet differences. Only prime agencies ought to establish actually voluntary emotions. Front workers could desert as only current powers; practitioners respect. Lists will no|2.48|2.20|3001001|amalgexporti #1|1|newborn|3|Children|363|pricallypri|petite|pale3097576692236320|midnight|Pound|Unknown|30|ablebarprically| +6303|AAAAAAAAOJIBAAAA|2000-10-27||Serious participants could not land too strong, natural premises. Continuous costs keep logical t|4.06|2.72|3001001|edu packmaxi #4|1|tennis|8|Sports|363|pricallypri|N/A|pale3097576692236320|tan|Cup|Unknown|8|pribarprically| +6304|AAAAAAAAAKIBAAAA|1997-10-27|1999-10-27|Long independent elections used to work all right new, main elements; directly effective hospitals shall produce payments. Only controversia|2.53|2.22|7011005|amalgnameless #5|11|accent|7|Home|290|barn stable|N/A|0973367salmon4642839|navy|Ton|Unknown|50|esebarprically| +6305|AAAAAAAAAKIBAAAA|1999-10-28|2001-10-26|Difficult, poor eyes end enthusiastically. Too universal details would not reassure slightly in a colours. As well individual institutions regulate attempts. |83.09|54.83|7011005|scholarnameless #6|11|tables|7|Home|290|barn stable|N/A|408smoke313909936294|red|Lb|Unknown|48|antibarprically| +6306|AAAAAAAAAKIBAAAA|2001-10-27||Activities carry users. Charming moments resist months. Minor, lon|1.32|0.62|7011005|amalgunivamalg #17|1|cameras|10|Electronics|290|barn stable|N/A|408smoke313909936294|sandy|Carton|Unknown|13|callybarprically| +6307|AAAAAAAADKIBAAAA|1997-10-27||Muscles end obviously other sources. Major links prevent both to a lines. Devices might produce only different conferences. Favorite candidates a|4.86|2.91|8004008|edu packnameless #8|4|camping|8|Sports|48|eingese|N/A|957chiffon5616339493|sandy|Case|Unknown|6|ationbarprically| +6308|AAAAAAAAEKIBAAAA|1997-10-27|2000-10-26|Responsible demands can establish again. Well excellent kilometres set much. Bad, |7.67|6.51|4002001|importoedu pack #1|2|mens|4|Shoes|101|oughtbarought|extra large|053911750olive106417|metallic|Oz|Unknown|8|eingbarprically| +6309|AAAAAAAAEKIBAAAA|2000-10-27||Responsible demands can establish again. Well excellent kilometres set much. Bad, |9.76|8.78|10015003|scholaramalgamalg #3|15|portable|10|Electronics|101|oughtbarought|N/A|1rose726260317285106|royal|Ton|Unknown|1|n stbarprically| +6310|AAAAAAAAGKIBAAAA|1997-10-27|1999-10-27|Human, additional services used to use |0.76|0.26|6008007|namelesscorp #7|8|mens watch|6|Jewelry|821|oughtableeing|N/A|770678863357yellow94|lavender|N/A|Unknown|60|baroughtprically| +6311|AAAAAAAAGKIBAAAA|1999-10-28|2001-10-26|Human, additional services used to use |2.31|0.26|6008007|edu packscholar #2|8|classical|5|Music|62|ablecally|N/A|454619steel141804616|blush|Ton|Unknown|32|oughtoughtprically| +6312|AAAAAAAAGKIBAAAA|2001-10-27||Boring, general musicians tell. Responsibilities would not allow in a players; local cheeks cannot care e|7.60|0.26|6008007|amalgamalg #1|1|dresses|1|Women|604|ablecally|extra large|454619steel141804616|pink|Pallet|Unknown|48|ableoughtprically| +6313|AAAAAAAAJKIBAAAA|1997-10-27||Comments may not form. Similar clothes cannot know even through a kids; surprising, adjacent matters upset namely standards. Especially new words make. Immediately wooden reasons read to a findi|9.57|4.49|7008004|namelessbrand #4|8|lighting|7|Home|351|oughtantipri|N/A|3265169drab678542139|sandy|Cup|Unknown|87|prioughtprically| +6314|AAAAAAAAKKIBAAAA|1997-10-27|2000-10-26|Black visitors hurt so increasingly greek publishers; trustees put. Perfect circles will cost about poor travellers. Above critical shar|7.05|4.30|5003001|exportischolar #1|3|pop|5|Music|295|antin stable|N/A|1042975white19753315|magenta|Tbl|Unknown|19|eseoughtprically| +6315|AAAAAAAAKKIBAAAA|2000-10-27||Black visitors hurt so increasingly greek publishers; trustees put. Perfect circles will cost about poor travellers. Above critical shar|4.36|1.78|8012006|importomaxi #6|3|guns|8|Sports|295|antin stable|N/A|1042975white19753315|spring|N/A|Unknown|32|antioughtprically| +6316|AAAAAAAAMKIBAAAA|1997-10-27|1999-10-27|Anywhere high resources sha|7.58|3.10|5003001|exportischolar #1|3|pop|5|Music|166|callycallyought|N/A|23991purple319724743|saddle|N/A|Unknown|60|callyoughtprically| +6317|AAAAAAAAMKIBAAAA|1999-10-28|2001-10-26|Good, concerned funds require indeed. Symptoms can reduce away actually true orga|5.46|3.10|2002002|importoimporto #2|2|shirts|2|Men|166|callycallyought|small|077158484200slate712|purple|Ounce|Unknown|2|ationoughtprically| +6318|AAAAAAAAMKIBAAAA|2001-10-27||National criteria would not increase proportions; able, specific nations should not take invariably human areas. Important periods specify|1.43|0.57|2002002|edu packimporto #1|2|sports-apparel|2|Men|166|callycallyought|economy|8960124gainsboro2242|light|Tsp|Unknown|6|eingoughtprically| +6319|AAAAAAAAPKIBAAAA|1997-10-27||Other, level members send immediately to the services. Base, simple consequences should not visit particularly international, strong seconds; false, le|92.96|52.98|4001002|amalgedu pack #2|1|womens|4|Shoes|160|barcallyought|large|8burnished8102383368|sky|Ton|Unknown|31|n stoughtprically| +6320|AAAAAAAAALIBAAAA|1997-10-27|2000-10-26|Knees could no suitable, wild enemies. Always international personnel cl|4.81|3.60|2001001|amalgimporto #1|1|accessories|2|Men|129|n stableought|extra large|70077689tomato051351|maroon|Lb|Unknown|7|barableprically| +6321|AAAAAAAAALIBAAAA|2000-10-27||Then effective quantities should not select that somewhere complex flights; together actual directors force opposite, other products. Major, long cars |7.81|3.60|2001001|edu packscholar #2|4|classical|5|Music|129|n stableought|N/A|5647887602smoke48638|slate|Tsp|Unknown|2|oughtableprically| +6322|AAAAAAAACLIBAAAA|1997-10-27|1999-10-27|Close,|4.04|2.74|8013003|exportimaxi #3|13|sailing|8|Sports|205|antibarable|N/A|2807665645891snow393|rose|N/A|Unknown|14|ableableprically| +6323|AAAAAAAACLIBAAAA|1999-10-28|2001-10-26|Close,|0.70|2.74|8013003|brandnameless #4|7|hockey|8|Sports|272|ableationable|N/A|2807665645891snow393|wheat|Gross|Unknown|6|priableprically| +6324|AAAAAAAACLIBAAAA|2001-10-27||Probably miserable children form new, other videos. However mental terms must try years. Adequately full speakers may buy obviousl|2.57|2.74|5003001|exportischolar #1|7|pop|5|Music|419|ableationable|N/A|24560814729cyan18017|peach|Bundle|Unknown|5|eseableprically| +6325|AAAAAAAAFLIBAAAA|1997-10-27||Limits shall imagine conversely in the members. Commercial, national facts might give automatically soviet, bright communications; utterly short|0.92|0.44|3002002|importoexporti #2|2|infants|3|Children|142|ableeseought|N/A|43964powder268093654|indian|Pallet|Unknown|21|antiableprically| +6326|AAAAAAAAGLIBAAAA|1997-10-27|2000-10-26|Significant, different ministers must produce. Big things should not take normal months. Months could not |9.07|4.17|1004001|edu packamalg #1|4|swimwear|1|Women|916|callyoughtn st|N/A|2882linen19755265285|red|Tbl|Unknown|33|callyableprically| +6327|AAAAAAAAGLIBAAAA|2000-10-27||Significant, different ministers must produce. Big things should not take normal months. Months could not |3.16|4.17|1004001|importoexporti #2|2|infants|3|Children|916|callyoughtn st|medium|2882linen19755265285|green|Bunch|Unknown|6|ationableprically| +6328|AAAAAAAAILIBAAAA|1997-10-27|1999-10-27|Letters bring that is to say primarily local lines; true, necessary metres can talk more regional, regular years; losses spo|4.42|1.98|7004009|edu packbrand #9|4|curtains/drapes|7|Home|503|pribaranti|N/A|681773771571steel336|seashell|Each|Unknown|11|eingableprically| +6329|AAAAAAAAILIBAAAA|1999-10-28|2001-10-26|Letters bring that is to say primarily local lines; true, necessary metres can talk more regional, regular years; losses spo|0.69|1.98|7004009|amalgexporti #2|1|newborn|3|Children|503|pribaranti|large|681773771571steel336|orchid|Cup|Unknown|50|n stableprically| +6330|AAAAAAAAILIBAAAA|2001-10-27||Local rises check simply following, similar businesses. Alternative problems shall make absolutely original points; indirectly uncomfortable women should not give also wes|3.33|1.03|7004009|exportischolar #1|1|pop|5|Music|814|eseoughteing|N/A|4286tan5834940028156|salmon|Pallet|Unknown|38|barpriprically| +6331|AAAAAAAALLIBAAAA|1997-10-27||Internal companies could not repeat committees. Outside relatives keep leaders; economic, important girls kno|5.34|4.37|6007004|brandcorp #4|7|pendants|6|Jewelry|515|antioughtanti|N/A|19150981yellow040546|rosy|Ounce|Unknown|92|oughtpriprically| +6332|AAAAAAAAMLIBAAAA|1997-10-27|2000-10-26|High relationships improve. Names should not grip also on the problems. Future, ready hands will rot. Activities might not risk well right increases. Sudden, great circumst|0.57|0.27|8011003|amalgmaxi #3|11|archery|8|Sports|90|barn st|N/A|88312182seashell0791|papaya|Cup|Unknown|26|ablepriprically| +6333|AAAAAAAAMLIBAAAA|2000-10-27||Domestic poems issue often possible, deep doctors. New, wide boys deliver well in a |1.35|0.52|3003002|exportiexporti #2|11|toddlers|3|Children|82|barn st|extra large|6385powder9153449092|snow|Tsp|Unknown|3|pripriprically| +6334|AAAAAAAAOLIBAAAA|1997-10-27|1999-10-27|Progressive, ethnic activities ought to provide fortunate rese|8.28|3.22|10010011|univamalgamalg #11|10|memory|10|Electronics|971|oughtationn st|N/A|884706920navy6420308|saddle|Unknown|Unknown|66|esepriprically| +6335|AAAAAAAAOLIBAAAA|1999-10-28|2001-10-26|Ordinary, financial farmers will not agree. |2.23|3.22|8010006|univmaxi #6|10|pools|8|Sports|971|oughtationn st|N/A|74818371indian267321|cornsilk|Tsp|Unknown|79|antipriprically| +6336|AAAAAAAAOLIBAAAA|2001-10-27||Ordinary, financial farmers will not agree. |2.20|1.49|8010006|brandunivamalg #13|7|personal|10|Electronics|195|antin stought|N/A|377901757249923peru9|smoke|Tsp|Unknown|48|callypriprically| +6337|AAAAAAAABMIBAAAA|1997-10-27||Full, relevant manufacturers should open human, low charges. But far eyes take on a prisoners; politically normal doctors will join mostly incidents; national, pale |7.21|4.54|8005008|scholarnameless #8|5|fitness|8|Sports|184|eseeingought|N/A|167175569khaki842629|snow|Gross|Unknown|1|ationpriprically| +6338|AAAAAAAACMIBAAAA|1997-10-27|2000-10-26|Professionals may introduce properly; powerful sons would wait how|1.72|0.96|5001001|amalgscholar #1|1|rock|5|Music|484|eseeingese|N/A|4papaya2556225405232|rose|Bunch|Unknown|25|eingpriprically| +6339|AAAAAAAACMIBAAAA|2000-10-27||Professionals may introduce properly; powerful sons would wait how|0.13|0.96|5001001|maxicorp #4|9|womens watch|6|Jewelry|484|eseeingese|N/A|26517784024red203617|peach|Oz|Unknown|18|n stpriprically| +6340|AAAAAAAAEMIBAAAA|1997-10-27|1999-10-27|Principal emissions get structures. Large, happy districts can cover too muscles; sometimes great sales enable|5.08|3.09|6005005|scholarcorp #5|5|earings|6|Jewelry|465|anticallyese|N/A|1red7594548180984166|sky|Dram|Unknown|44|bareseprically| +6341|AAAAAAAAEMIBAAAA|1999-10-28|2001-10-26|Small workers browse in a areas. Only unacceptable |2.62|3.09|6005005|namelessnameless #2|8|outdoor|8|Sports|465|anticallyese|N/A|9salmon4003849639853|light|Dram|Unknown|3|oughteseprically| +6342|AAAAAAAAEMIBAAAA|2001-10-27||Small workers browse in a areas. Only unacceptable |3.30|0.99|5001001|amalgscholar #1|1|rock|5|Music|465|anticallyese|N/A|9salmon4003849639853|powder|Ounce|Unknown|37|ableeseprically| +6343|AAAAAAAAHMIBAAAA|1997-10-27||Other, pleased views might get corporate groups. Impatiently efficient texts show productive groups. Supposed, public politicians enter for a shares. Armies underst|9.15|6.67|4002002|importoedu pack #2|2|mens|4|Shoes|478|eingationese|petite|8844391steel36181810|sienna|Dram|Unknown|22|prieseprically| +6344|AAAAAAAAIMIBAAAA|1997-10-27|2000-10-26|Access|0.66|0.41|4003001|exportiedu pack #1|3|kids|4|Shoes|197|ationn stought|extra large|268944918444powder30|tan|N/A|Unknown|13|eseeseprically| +6345|AAAAAAAAIMIBAAAA|2000-10-27||Independent areas may look hardly plants. Ago rich years might not need. Features laugh now spiritual houses. Students might live aside maybe mad lads. Marvellous r|6.08|5.47|9016010|corpunivamalg #10|3|mystery|9|Books|197|ationn stought|N/A|292634wheat478647473|grey|Gram|Unknown|58|antieseprically| +6346|AAAAAAAAKMIBAAAA|1997-10-27|1999-10-27|Old, dull officers will not tread never for a students. Quickly effective farmers deal under. Fresh kids regain into a ears; new, popular conventions disregard as yet men. Private miles sha|9.13|6.48|5004001|edu packscholar #1|4|classical|5|Music|85|antieing|N/A|691744steel794512927|peach|Lb|Unknown|24|callyeseprically| +6347|AAAAAAAAKMIBAAAA|1999-10-28|2001-10-26|Crucial mountains see to a processes. Almost specific periods might think only. Never other walls make so from a ways. Major, narrow variations may notice out of|86.51|6.48|8007008|brandnameless #8|4|hockey|8|Sports|85|antieing|N/A|8343drab321733185446|turquoise|Gross|Unknown|88|ationeseprically| +6348|AAAAAAAAKMIBAAAA|2001-10-27||Men would afford so small objects. Keys ap|1.92|1.42|9014001|edu packunivamalg #1|14|sports|9|Books|301|oughtbarpri|N/A|8343drab321733185446|yellow|Each|Unknown|61|eingeseprically| +6349|AAAAAAAANMIBAAAA|1997-10-27||Exotic rights could not commit here persistent |3.07|1.10|8007010|brandnameless #10|7|hockey|8|Sports|46|callyese|N/A|8638636574tan3620783|smoke|Pound|Unknown|31|n steseprically| +6350|AAAAAAAAOMIBAAAA|1997-10-27|2000-10-26|Bad files make below bad occasions. Local days grow now for a years. Only royal years should look again correct fears. Creatures seem new conditions. Trials keep. Branches wa|9.13|7.66|7003001|exportibrand #1|3|kids|7|Home|837|ationprieing|N/A|123350royal851333399|tan|Ounce|Unknown|32|barantiprically| +6351|AAAAAAAAOMIBAAAA|2000-10-27||Bad files make below bad occasions. Local days grow now for a years. Only royal years should look again correct fears. Creatures seem new conditions. Trials keep. Branches wa|0.19|7.66|4003002|exportiedu pack #2|3|kids|4|Shoes|882|ationprieing|large|786664958tomato52581|plum|Ton|Unknown|10|oughtantiprically| +6352|AAAAAAAAANIBAAAA|1997-10-27|1999-10-27|Of course national games cannot produce obviously public complex women. So likely years accept magistrates. Good topics feel enough almost crucial pp.. Well resulting children sho|7.42|3.19|3001001|amalgexporti #1|1|newborn|3|Children|398|eingn stpri|petite|491024100439white928|ghost|Pallet|Unknown|61|ableantiprically| +6353|AAAAAAAAANIBAAAA|1999-10-28|2001-10-26|By now old years may not avoid again foreign previous eyes.|2.74|2.30|1001002|amalgamalg #2|1|dresses|1|Women|398|eingn stpri|petite|79white2773770001998|lavender|Oz|Unknown|9|priantiprically| +6354|AAAAAAAAANIBAAAA|2001-10-27||By now old years may not avoid again foreign previous eyes.|3.29|2.30|1001002|importonameless #5|1|paint|7|Home|398|eingn stpri|N/A|79white2773770001998|frosted|Carton|Unknown|26|eseantiprically| +6355|AAAAAAAADNIBAAAA|1997-10-27||Just presidential revenues exist home profitable, similar charges. Managers used to wonder there|1.55|1.05|6008004|namelesscorp #4|8|mens watch|6|Jewelry|631|oughtprically|N/A|3236snow527669332961|thistle|Carton|Unknown|9|antiantiprically| +6356|AAAAAAAAENIBAAAA|1997-10-27|2000-10-26|Strong volumes shall not dismiss regional packages. Parties might drop sharply about a employees. In|3.47|2.60|2001001|amalgimporto #1|1|accessories|2|Men|379|n stationpri|N/A|501374645025433lime4|aquamarine|Gross|Unknown|87|callyantiprically| +6357|AAAAAAAAENIBAAAA|2000-10-27||Strong volumes shall not dismiss regional packages. Parties might drop sharply about a employees. In|2.26|1.33|1004002|edu packamalg #2|4|swimwear|1|Women|379|n stationpri|large|501374645025433lime4|saddle|Bunch|Unknown|2|ationantiprically| +6358|AAAAAAAAGNIBAAAA|1997-10-27|1999-10-27|Pounds shall look for ever bitterly european things; more technical lengths find never with a reasons. Signs change. Familiar, safe others would not take ne|4.66|1.49|1003001|exportiamalg #1|3|maternity|1|Women|160|barcallyought|medium|saddle83721814578871|white|Tsp|Unknown|10|eingantiprically| +6359|AAAAAAAAGNIBAAAA|1999-10-28|2001-10-26|Pounds shall look for ever bitterly european things; more technical lengths find never with a reasons. Signs change. Familiar, safe others would not take ne|0.22|1.49|6008008|namelesscorp #8|3|mens watch|6|Jewelry|780|barcallyought|N/A|9733946papaya5092562|blush|Unknown|Unknown|68|n stantiprically| +6360|AAAAAAAAGNIBAAAA|2001-10-27||Pounds shall look for ever bitterly european things; more technical lengths find never with a reasons. Signs change. Familiar, safe others would not take ne|8.69|7.21|9003001|exportimaxi #1|3|computers|9|Books|780|barcallyought|N/A|9733946papaya5092562|saddle|Ounce|Unknown|27|barcallyprically| +6361|AAAAAAAAJNIBAAAA|1997-10-27||Useful miles remain of course hopes. Maybe gross features get unable countries. Difficult events get. Usually other circumstances continue als|7.12|5.41|2004002|edu packimporto #2|4|sports-apparel|2|Men|33|pripri|medium|546570navy8861105646|peru|Lb|Unknown|1|oughtcallyprically| +6362|AAAAAAAAKNIBAAAA|1997-10-27|2000-10-26|Subjects will read too. Reduced, identical patients like through a animals. At least unable c|0.12|0.07|9011003|amalgunivamalg #3|11|cooking|9|Books|10|barought|N/A|888535571turquoise87|spring|Box|Unknown|28|ablecallyprically| +6363|AAAAAAAAKNIBAAAA|2000-10-27||Subjects will read too. Reduced, identical patients like through a animals. At least unable c|4.90|0.07|6006004|corpcorp #4|11|rings|6|Jewelry|68|eingcally|N/A|salmon75817895006397|blue|Pound|Unknown|6|pricallyprically| +6364|AAAAAAAAMNIBAAAA|1997-10-27|1999-10-27|There general companies work even. Channels may not say easier things. Thereafter hot agents increase only years; reservations |7.80|3.66|8005005|scholarnameless #5|5|fitness|8|Sports|134|esepriought|N/A|9misty09473071023919|tan|Bundle|Unknown|19|esecallyprically| +6365|AAAAAAAAMNIBAAAA|1999-10-28|2001-10-26|There general companies work even. Channels may not say easier things. Thereafter hot agents increase only years; reservations |1.14|3.66|6004006|edu packcorp #6|4|bracelets|6|Jewelry|134|esepriought|N/A|9misty09473071023919|lawn|Carton|Unknown|5|anticallyprically| +6366|AAAAAAAAMNIBAAAA|2001-10-27||Quite other structures believe often from the parts. More guilty studies live regardless appropriate writers. Likely results should get difficult subjects; merely|6.57|3.41|6004006|corpbrand #7|4|consignment|6|Jewelry|134|esepriought|N/A|9misty09473071023919|papaya|Unknown|Unknown|28|callycallyprically| +6367|AAAAAAAAPNIBAAAA|1997-10-27||Very good prisoners go against a rules. |3.20|1.12|9004008|edu packmaxi #8|4|entertainments|9|Books|256|callyantiable|N/A|295245380220smoke262|khaki|Pallet|Unknown|38|ationcallyprically| +6368|AAAAAAAAAOIBAAAA|1997-10-27|2000-10-26|More national figures believe clearly dif|1.20|0.70|8005009|scholarnameless #9|5|fitness|8|Sports|328|eingablepri|N/A|12779603455633sky879|violet|Pound|Unknown|10|eingcallyprically| +6369|AAAAAAAAAOIBAAAA|2000-10-27||More national figures believe clearly dif|1.96|0.70|8005009|exportibrand #2|13|loose stones|6|Jewelry|328|eingablepri|N/A|12779603455633sky879|pink|Ton|Unknown|14|n stcallyprically| +6370|AAAAAAAACOIBAAAA|1997-10-27|1999-10-27|Plans promote false, emotional months. Wide, extraordinary hands provide just possible yards. Signs generate still friends. Both important m|9.04|3.97|10007013|brandunivamalg #13|7|personal|10|Electronics|358|eingantipri|N/A|94lace76392039988768|orchid|Ounce|Unknown|55|barationprically| +6371|AAAAAAAACOIBAAAA|1999-10-28|2001-10-26|Plans promote false, emotional months. Wide, extraordinary hands provide just possible yards. Signs generate still friends. Both important m|0.33|3.97|10007013|edu packamalg #2|7|swimwear|1|Women|358|eingantipri|large|94lace76392039988768|papaya|Cup|Unknown|26|oughtationprically| +6372|AAAAAAAACOIBAAAA|2001-10-27||Plans promote false, emotional months. Wide, extraordinary hands provide just possible yards. Signs generate still friends. Both important m|1.85|3.97|10007013|edu packcorp #5|4|bracelets|6|Jewelry|358|eingantipri|N/A|974388684white015026|pink|Carton|Unknown|35|ableationprically| +6373|AAAAAAAAFOIBAAAA|1997-10-27||Windows shall last; only deep thousands stay hence now poli|3.14|1.41|4004002|edu packedu pack #2|4|athletic|4|Shoes|189|n steingought|medium|304smoke027798298522|smoke|Each|Unknown|50|priationprically| +6374|AAAAAAAAGOIBAAAA|1997-10-27|2000-10-26|Possible, safe paintings would comment most dramatically possible years. Physical, formidable accounts take already by a records; etc applicable schools should succeed p|1.06|0.41|1003001|exportiamalg #1|3|maternity|1|Women|17|ationought|medium|9sandy73257847100729|snow|Pound|Unknown|18|eseationprically| +6375|AAAAAAAAGOIBAAAA|2000-10-27||Possible, safe paintings would comment most dramatically possible years. Physical, formidable accounts take already by a records; etc applicable schools should succeed p|2.72|1.71|1003001|amalgexporti #2|3|newborn|3|Children|332|ationought|medium|9sandy73257847100729|sienna|Lb|Unknown|10|antiationprically| +6376|AAAAAAAAIOIBAAAA|1997-10-27|1999-10-27|Small others will provide probably complete countries. Only effective pensions violate on board maybe left animals. Old r|2.55|1.60|10009002|maxiunivamalg #2|9|televisions|10|Electronics|910|baroughtn st|N/A|3715213381134smoke68|pale|Bunch|Unknown|6|callyationprically| +6377|AAAAAAAAIOIBAAAA|1999-10-28|2001-10-26|Small others will provide probably complete countries. Only effective pensions violate on board maybe left animals. Old r|7.05|2.32|6010002|univbrand #2|9|jewelry boxes|6|Jewelry|282|ableeingable|N/A|6901376132020purple1|orange|Pallet|Unknown|3|ationationprically| +6378|AAAAAAAAIOIBAAAA|2001-10-27||Small others will provide probably complete countries. Only effective pensions violate on board maybe left animals. Old r|0.90|2.32|6015003|scholarbrand #3|15|custom|6|Jewelry|282|ableeingable|N/A|6901376132020purple1|yellow|Tsp|Unknown|58|eingationprically| +6379|AAAAAAAALOIBAAAA|1997-10-27||Available, personal relations would decline rad|5.36|2.57|8001010|amalgnameless #10|1|athletic shoes|8|Sports|602|ablebarcally|N/A|39458sienna547652815|wheat|Dram|Unknown|65|n stationprically| +6380|AAAAAAAAMOIBAAAA|1997-10-27|2000-10-26|Effective poems spread radically wise dreams. Open irish instruments may hit. Then tory pupils relax relatively too american ministers. Accura|5.99|4.91|2001001|amalgimporto #1|1|accessories|2|Men|652|ableantically|petite|lace4254025386196558|purple|Dram|Unknown|63|bareingprically| +6381|AAAAAAAAMOIBAAAA|2000-10-27||Effective poems spread radically wise dreams. Open irish instruments may hit. Then tory pupils relax relatively too american ministers. Accura|5.05|3.38|5004002|edu packscholar #2|1|classical|5|Music|190|barn stought|N/A|02244588511520puff37|tomato|Bunch|Unknown|68|oughteingprically| +6382|AAAAAAAAOOIBAAAA|1997-10-27|1999-10-27|Final, final children know on a securities. Succe|1.73|0.81|9014011|edu packunivamalg #11|14|sports|9|Books|587|ationeinganti|N/A|54178460389hot730333|wheat|Gross|Unknown|2|ableeingprically| +6383|AAAAAAAAOOIBAAAA|1999-10-28|2001-10-26|Final, final children know on a securities. Succe|0.48|0.16|9014011|exportischolar #2|3|pop|5|Music|164|esecallyought|N/A|54178460389hot730333|plum|Case|Unknown|13|prieingprically| +6384|AAAAAAAAOOIBAAAA|2001-10-27||Brave, domestic groups look sooner in a years. Always main conservatives minimise never extra daughters. Implicit, powerful application|0.96|0.68|9014011|corpbrand #1|3|consignment|6|Jewelry|164|esecallyought|N/A|80215399359283peach6|maroon|Case|Unknown|30|eseeingprically| +6385|AAAAAAAABPIBAAAA|1997-10-27||Books can answer always old, old husbands. Quickly outstanding homes |3.15|1.07|5002002|importoscholar #2|2|country|5|Music|206|callybarable|N/A|381tan10806862937315|rose|Case|Unknown|59|antieingprically| +6386|AAAAAAAACPIBAAAA|1997-10-27|2000-10-26|New, pure children canno|3.51|1.71|4002001|importoedu pack #1|2|mens|4|Shoes|939|n stprin st|medium|8832886rosy460033057|slate|Each|Unknown|94|callyeingprically| +6387|AAAAAAAACPIBAAAA|2000-10-27||Important, annual women collect in part full, variou|5.54|1.71|4002001|exportiimporto #2|3|pants|2|Men|715|n stprin st|medium|71163755041528wheat1|papaya|N/A|Unknown|33|ationeingprically| +6388|AAAAAAAAEPIBAAAA|1997-10-27|1999-10-27|Available|2.19|0.72|7007009|brandbrand #9|7|decor|7|Home|569|n stcallyanti|N/A|75628962021470white1|thistle|Pound|Unknown|5|eingeingprically| +6389|AAAAAAAAEPIBAAAA|1999-10-28|2001-10-26|Available|3.72|0.72|7007009|exporticorp #6|7|gold|6|Jewelry|569|n stcallyanti|N/A|75628962021470white1|slate|Carton|Unknown|28|n steingprically| +6390|AAAAAAAAEPIBAAAA|2001-10-27||Sole words will not fix studies. Efficient changes sell about. Ordinary, attractive hours comply of course avail|3.13|2.22|7007009|importoamalg #1|2|fragrances|1|Women|88|eingeing|extra large|74736puff01169247197|almond|Case|Unknown|21|barn stprically| +6391|AAAAAAAAHPIBAAAA|1997-10-27||English, effective children teach reluctantly popular, sad successes. Heroes must not sing both unchange|7.49|5.39|7016002|corpnameless #2|16|furniture|7|Home|17|ationought|N/A|4646048papaya8076138|sienna|Carton|Unknown|10|oughtn stprically| +6392|AAAAAAAAIPIBAAAA|1997-10-27|2000-10-26|Local operations ought to affect regular, coherent increases. Empirical witnesses could|3.37|2.52|4003001|exportiedu pack #1|3|kids|4|Shoes|113|prioughtought|small|586535sienna61942551|violet|Dozen|Unknown|4|ablen stprically| +6393|AAAAAAAAIPIBAAAA|2000-10-27||Local operations ought to affect regular, coherent increases. Empirical witnesses could|0.73|2.52|4003001|edu packunivamalg #10|3|sports|9|Books|991|oughtn stn st|N/A|18tan778792213553307|sienna|Pallet|Unknown|49|prin stprically| +6394|AAAAAAAAKPIBAAAA|1997-10-27|1999-10-27|Ago high russians might regulate more than internal american women. Special students hear completely duties. Bizarre accounts will establish mostly for a models. Various strangers will license |0.65|0.40|10006003|corpunivamalg #3|6|musical|10|Electronics|137|ationpriought|N/A|lime0295415262115101|snow|Pound|Unknown|44|esen stprically| +6395|AAAAAAAAKPIBAAAA|1999-10-28|2001-10-26|Ago high russians might regulate more than internal american women. Special students hear completely duties. Bizarre accounts will establish mostly for a models. Various strangers will license |13.18|11.07|10006003|importoscholar #2|6|country|5|Music|137|ationpriought|N/A|0830227seashell54785|red|Bundle|Unknown|6|antin stprically| +6396|AAAAAAAAKPIBAAAA|2001-10-27||Products need to the streets. Top, br|53.96|27.51|2001001|amalgimporto #1|1|accessories|2|Men|68|eingcally|large|0830227seashell54785|blush|Tbl|Unknown|7|callyn stprically| +6397|AAAAAAAANPIBAAAA|1997-10-27||Ordinary n|5.56|4.00|2001002|amalgimporto #2|1|accessories|2|Men|308|eingbarpri|medium|sky97674524233953564|salmon|Unknown|Unknown|50|ationn stprically| +6398|AAAAAAAAOPIBAAAA|1997-10-27|2000-10-26|So red letters call properties. Both soviet organisations render little years. High days keep far possi|30.39|24.61|8001009|amalgnameless #9|1|athletic shoes|8|Sports|394|esen stpri|N/A|473smoke344508666214|yellow|Unknown|Unknown|22|eingn stprically| +6399|AAAAAAAAOPIBAAAA|2000-10-27||So red letters call properties. Both soviet organisations render little years. High days keep far possi|80.39|51.44|5003002|exportischolar #2|3|pop|5|Music|138|eingpriought|N/A|011159447grey7992755|medium|Lb|Unknown|13|n stn stprically| +6400|AAAAAAAAAAJBAAAA|1997-10-27|1999-10-27|Important, other hands used to shout with a activities. Groups would transfer only. Extreme children crack too in a standards. Serious records ought to please; labour years may show in the limits. Bea|6.78|3.66|3003001|exportiexporti #1|3|toddlers|3|Children|651|oughtantically|large|661pale3220811198771|seashell|Unknown|Unknown|22|barbaresecally| +6401|AAAAAAAAAAJBAAAA|1999-10-28|2001-10-26|Important, other hands used to shout with a activities. Groups would transfer only. Extreme children crack too in a standards. Serious records ought to please; labour years may show in the limits. Bea|0.98|3.66|7014002|edu packnameless #2|14|glassware|7|Home|651|oughtantically|N/A|697422644pink7984841|papaya|Pound|Unknown|53|oughtbaresecally| +6402|AAAAAAAAAAJBAAAA|2001-10-27||Physical terms make only on a questions. Offices will not make feet. Tests would not seem never oral, little women; fine, british communities could leave blue families. Now different views beco|8.58|7.03|1002001|importoamalg #1|2|fragrances|1|Women|651|oughtantically|small|697422644pink7984841|rose|Each|Unknown|33|ablebaresecally| +6403|AAAAAAAADAJBAAAA|1997-10-27||Y|1.95|0.89|3003002|exportiexporti #2|3|toddlers|3|Children|227|ationableable|medium|241155724smoke311657|indian|Dozen|Unknown|51|pribaresecally| +6404|AAAAAAAAEAJBAAAA|1997-10-27|2000-10-26|Years face fully necessary laboratories. American effects suffer down the tests. Voluntary groups m|8.83|5.73|2001001|amalgimporto #1|1|accessories|2|Men|53|prianti|small|484498487plum8363896|salmon|Carton|Unknown|62|esebaresecally| +6405|AAAAAAAAEAJBAAAA|2000-10-27||Years face fully necessary laboratories. American effects suffer down the tests. Voluntary groups m|6.40|5.73|2001001|scholarmaxi #10|1|history|9|Books|53|prianti|N/A|67spring693743974563|ghost|Gram|Unknown|12|antibaresecally| +6406|AAAAAAAAGAJBAAAA|1997-10-27|1999-10-27|Equal, british difficulties shall pick available, aware times. Families ought to find very in a unions. Workers should make late|5.40|3.72|4004001|edu packedu pack #1|4|athletic|4|Shoes|551|oughtantianti|N/A|934peach226736419790|orchid|Pallet|Unknown|18|callybaresecally| +6407|AAAAAAAAGAJBAAAA|1999-10-28|2001-10-26|Generally new meals can think more so major men. Persistent paths retain even open, yellow practices. Soft, full talks work other styles. Almo|1.33|3.72|4004001|maxinameless #8|4|optics|8|Sports|87|oughtantianti|N/A|934peach226736419790|plum|Tbl|Unknown|1|ationbaresecally| +6408|AAAAAAAAGAJBAAAA|2001-10-27||Generally new meals can think more so major men. Persistent paths retain even open, yellow practices. Soft, full talks work other styles. Almo|2.35|3.72|4004001|amalgexporti #1|4|newborn|3|Children|164|esecallyought|medium|plum4630931616622176|forest|Ounce|Unknown|31|eingbaresecally| +6409|AAAAAAAAJAJBAAAA|1997-10-27||Times might describe democratic shares. Dead, particular associations leave about free factors. Red minutes should seem explicitly commonly video-taped situations. Sometimes efficient others enable i|4.76|3.42|5002002|importoscholar #2|2|country|5|Music|359|n stantipri|N/A|94301556680salmon072|slate|Carton|Unknown|20|n stbaresecally| +6410|AAAAAAAAKAJBAAAA|1997-10-27|2000-10-26|Flexible architects may give new, key customers; little feet could abolish that; almost available reasons will not destroy following views. Easy children might forget w|1.44|1.12|5001001|amalgscholar #1|1|rock|5|Music|609|n stbarcally|N/A|3orchid0601529223707|peach|Cup|Unknown|34|baroughtesecally| +6411|AAAAAAAAKAJBAAAA|2000-10-27||Flexible architects may give new, key customers; little feet could abolish that; almost available reasons will not destroy following views. Easy children might forget w|0.78|0.29|5001001|edu packamalg #2|4|swimwear|1|Women|58|einganti|petite|3orchid0601529223707|salmon|Ton|Unknown|88|oughtoughtesecally| +6412|AAAAAAAAMAJBAAAA|1997-10-27|1999-10-27|English pupils wake thus states. Others find for a workers; final principles sound false obvious organisations. Contemporary, firm clubs proceed almost |2.44|2.12|1001001|amalgamalg #1|1|dresses|1|Women|216|callyoughtable|petite|39007sienna529019243|midnight|Pallet|Unknown|10|ableoughtesecally| +6413|AAAAAAAAMAJBAAAA|1999-10-28|2001-10-26|English pupils wake thus states. Others find for a workers; final principles sound false obvious organisations. Contemporary, firm clubs proceed almost |4.38|2.71|5002002|importoscholar #2|1|country|5|Music|392|ablen stpri|N/A|39007sienna529019243|misty|Unknown|Unknown|88|prioughtesecally| +6414|AAAAAAAAMAJBAAAA|2001-10-27||English pupils wake thus states. Others find for a workers; final principles sound false obvious organisations. Contemporary, firm clubs proceed almost |2.22|2.71|9007007|brandmaxi #7|7|reference|9|Books|392|ablen stpri|N/A|39007sienna529019243|steel|Tsp|Unknown|9|eseoughtesecally| +6415|AAAAAAAAPAJBAAAA|1997-10-27||Years ought to take new, central topics; public youngsters would not say often british actions. Less financial possibilities can make surely studies. Stores l|5.18|1.60|3001002|amalgexporti #2|1|newborn|3|Children|476|callyationese|large|8salmon6680230335778|papaya|Bundle|Unknown|28|antioughtesecally| +6416|AAAAAAAAABJBAAAA|1997-10-27|2000-10-26|Privileges can suggest hard decisions. Critics bear badly muscles; new, funny floors shall not like as difficult techniques; areas go often men. Blocks make as|7.94|3.09|8005007|scholarnameless #7|5|fitness|8|Sports|288|eingeingable|N/A|3556beige72156669333|lavender|Ounce|Unknown|62|callyoughtesecally| +6417|AAAAAAAAABJBAAAA|2000-10-27||Privileges can suggest hard decisions. Critics bear badly muscles; new, funny floors shall not like as difficult techniques; areas go often men. Blocks make as|4.10|2.09|9013010|exportiunivamalg #10|13|self-help|9|Books|288|eingeingable|N/A|4693724890361light66|thistle|Pound|Unknown|21|ationoughtesecally| +6418|AAAAAAAACBJBAAAA|1997-10-27|1999-10-27|Perhaps original police get ahead blues. Just comprehensive supporters might occupy financial, real details. Sudden suggestions enter almost about a holidays. Valuable,|1.27|0.63|3003001|exportiexporti #1|3|toddlers|3|Children|888|eingeingeing|small|66775813643rosy64365|peru|Unknown|Unknown|84|eingoughtesecally| +6419|AAAAAAAACBJBAAAA|1999-10-28|2001-10-26|Perhaps original police get ahead blues. Just comprehensive supporters might occupy financial, real details. Sudden suggestions enter almost about a holidays. Valuable,|1.87|0.63|3003001|amalgamalg #2|1|dresses|1|Women|126|callyableought|medium|9971572red7405604009|indian|Pound|Unknown|30|n stoughtesecally| +6420|AAAAAAAACBJBAAAA|2001-10-27||Perhaps original police get ahead blues. Just comprehensive supporters might occupy financial, real details. Sudden suggestions enter almost about a holidays. Valuable,|3.29|2.03|3003001|edu packscholar #1|4|classical|5|Music|242|callyableought|N/A|9971572red7405604009|indian|Pound|Unknown|94|barableesecally| +6421|AAAAAAAAFBJBAAAA|1997-10-27||Home contemporary places work. Growing banks may leave clearly special, beautiful ot|3.70|1.36|9010002|univunivamalg #2|10|travel|9|Books|351|oughtantipri|N/A|0841125569359papaya4|turquoise|Unknown|Unknown|33|oughtableesecally| +6422|AAAAAAAAGBJBAAAA|1997-10-27|2000-10-26|Now funny guidelines see prepared, logical seconds. Consequences must not go nearly shares. Teachers may not play. Acids cannot discern unnecessary days. |4.74|1.70|2001001|amalgimporto #1|1|accessories|2|Men|466|callycallyese|medium|6turquoise1907212986|dim|Gram|Unknown|11|ableableesecally| +6423|AAAAAAAAGBJBAAAA|2000-10-27||Now funny guidelines see prepared, logical seconds. Consequences must not go nearly shares. Teachers may not play. Acids cannot discern unnecessary days. |7.76|5.89|9012004|importounivamalg #4|1|home repair|9|Books|466|callycallyese|N/A|6turquoise1907212986|sienna|Box|Unknown|18|priableesecally| +6424|AAAAAAAAIBJBAAAA|1997-10-27|1999-10-27|Better wide yards clarify due afraid roads. Skilled sets should not distinguish simply well|7.76|5.50|1001001|amalgamalg #1|1|dresses|1|Women|438|eingpriese|large|38yellow211822996663|black|Ounce|Unknown|47|eseableesecally| +6425|AAAAAAAAIBJBAAAA|1999-10-28|2001-10-26|Better wide yards clarify due afraid roads. Skilled sets should not distinguish simply well|1.96|0.99|2002002|importoimporto #2|1|shirts|2|Men|141|oughteseought|small|38yellow211822996663|peru|Box|Unknown|48|antiableesecally| +6426|AAAAAAAAIBJBAAAA|2001-10-27||Better wide yards clarify due afraid roads. Skilled sets should not distinguish simply well|33.57|26.52|1004001|edu packamalg #1|1|swimwear|1|Women|120|barableought|petite|630chartreuse1959162|papaya|Case|Unknown|65|callyableesecally| +6427|AAAAAAAALBJBAAAA|1997-10-27||Children need also positive, british costs. Easy days cannot need. Obviously |0.17|0.10|5004002|edu packscholar #2|4|classical|5|Music|729|n stableation|N/A|0141burlywood3068285|sky|Tsp|Unknown|53|ationableesecally| +6428|AAAAAAAAMBJBAAAA|1997-10-27|2000-10-26|Other, good posts could lure. Beautiful, dead names leave so ago parental others. Curious democrats refer now; good yards sign. Accordingly mere mistakes identify perhaps. Limitations admit all|2.03|1.82|4001001|amalgedu pack #1|1|womens|4|Shoes|341|oughtesepri|extra large|30turquoise001788191|papaya|Dram|Unknown|1|eingableesecally| +6429|AAAAAAAAMBJBAAAA|2000-10-27||Other, good posts could lure. Beautiful, dead names leave so ago parental others. Curious democrats refer now; good yards sign. Accordingly mere mistakes identify perhaps. Limitations admit all|6.67|5.06|8005010|scholarnameless #10|1|fitness|8|Sports|341|oughtesepri|N/A|30turquoise001788191|pale|Gram|Unknown|59|n stableesecally| +6430|AAAAAAAAOBJBAAAA|1997-10-27|1999-10-27|Prospects might call slowly. Others sha|4.01|3.00|6007007|brandcorp #7|7|pendants|6|Jewelry|303|pribarpri|N/A|6sandy80253457611030|tomato|Ounce|Unknown|11|barpriesecally| +6431|AAAAAAAAOBJBAAAA|1999-10-28|2001-10-26|Prospects might call slowly. Others sha|4.98|3.00|6007007|amalgnameless #2|11|accent|7|Home|303|pribarpri|N/A|6sandy80253457611030|hot|Unknown|Unknown|43|oughtpriesecally| +6432|AAAAAAAAOBJBAAAA|2001-10-27||Prospects might call slowly. Others sha|5.57|3.00|6007007|importoimporto #1|11|shirts|2|Men|303|pribarpri|medium|6sandy80253457611030|snow|Pallet|Unknown|33|ablepriesecally| +6433|AAAAAAAABCJBAAAA|1997-10-27||Regions arrest too modern, clever hours. Obvious miles might buy good, light stars; really digital cells will find. Necessary students track rather. Bad, local observers take causes. Academic, sh|3.16|2.08|2002002|importoimporto #2|2|shirts|2|Men|145|antieseought|medium|1136926misty10481326|pale|Lb|Unknown|86|pripriesecally| +6434|AAAAAAAACCJBAAAA|1997-10-27|2000-10-26|Ancient origins could apply houses; proper needs settle so mostly obvious hands. Completely persona|4.82|2.26|5001001|amalgscholar #1|1|rock|5|Music|423|priableese|N/A|803honeydew418908465|hot|Oz|Unknown|30|esepriesecally| +6435|AAAAAAAACCJBAAAA|2000-10-27||Standard, economic times must discuss silently helpful grounds. Very local meetings develop to a types. Back, principal lovers stay immediatel|4.67|2.28|8016006|corpmaxi #6|1|golf|8|Sports|154|eseantiought|N/A|3355white66463162467|navajo|N/A|Unknown|8|antipriesecally| +6436|AAAAAAAAECJBAAAA|1997-10-27|1999-10-27|More ultimate chapters would not find even present late ingredients. Excellent things buy already in a boys. Sales may purchase so |4.07|3.17|1003001|exportiamalg #1|3|maternity|1|Women|120|barableought|small|2132tomato7829382151|orchid|Ounce|Unknown|22|callypriesecally| +6437|AAAAAAAAECJBAAAA|1999-10-28|2001-10-26|More ultimate chapters would not find even present late ingredients. Excellent things buy already in a boys. Sales may purchase so |1.84|1.10|1003001|exportiedu pack #2|3|kids|4|Shoes|120|barableought|large|471powder20440867800|rosy|Oz|Unknown|49|ationpriesecally| +6438|AAAAAAAAECJBAAAA|2001-10-27||Very sexual fears can dominate often services. Initial conditions must look members; special law|2.43|1.57|4002001|importoedu pack #1|2|mens|4|Shoes|942|ableesen st|extra large|471powder20440867800|slate|Oz|Unknown|27|eingpriesecally| +6439|AAAAAAAAHCJBAAAA|1997-10-27||Particularly particular managers provide provisions. Long men must not know instead accused deputies. Highly left deaths would want different students|3.76|3.00|2002002|importoimporto #2|2|shirts|2|Men|159|n stantiought|medium|171483614lime0236314|salmon|Bunch|Unknown|59|n stpriesecally| +6440|AAAAAAAAICJBAAAA|1997-10-27|2000-10-26|Parents understand certainly villages. Women will call gradually like english churches. Local, real regulations may not like. Deep, unique years will see individual centuries. Parents shall hea|8.35|6.68|5001001|amalgscholar #1|1|rock|5|Music|860|barcallyeing|N/A|6029275598031peach49|pink|Bundle|Unknown|10|bareseesecally| +6441|AAAAAAAAICJBAAAA|2000-10-27||Parents understand certainly villages. Women will call gradually like english churches. Local, real regulations may not like. Deep, unique years will see individual centuries. Parents shall hea|6.79|3.93|6015002|scholarbrand #2|15|custom|6|Jewelry|173|priationought|N/A|2114620641487violet9|midnight|Cup|Unknown|24|oughteseesecally| +6442|AAAAAAAAKCJBAAAA|1997-10-27|1999-10-27|Entire, expert methods ought to occur simply with|0.17|0.07|10007017|brandunivamalg #17|7|personal|10|Electronics|173|priationought|N/A|7lime438506018877665|tan|Cup|Unknown|43|ableeseesecally| +6443|AAAAAAAAKCJBAAAA|1999-10-28|2001-10-26|Forever medical years must leave together. Almost high points ensure bars. Large children could not take later by a companies. Shar|3.58|0.07|10007017|importoimporto #2|7|shirts|2|Men|72|ableation|small|7lime438506018877665|sky|Case|Unknown|3|prieseesecally| +6444|AAAAAAAAKCJBAAAA|2001-10-27||Very total men save strange, golden parents. Workers go meanwhile other objectives. Common titles answer brief travellers. Conditions might|5.89|2.88|3003001|exportiexporti #1|7|toddlers|3|Children|72|ableation|medium|11steel5228157751138|sandy|N/A|Unknown|58|eseeseesecally| +6445|AAAAAAAANCJBAAAA|1997-10-27||Owners involve already avai|3.67|2.86|6013006|exportibrand #6|13|loose stones|6|Jewelry|561|oughtcallyanti|N/A|sienna13173610403160|salmon|Oz|Unknown|17|antieseesecally| +6446|AAAAAAAAOCJBAAAA|1997-10-27|2000-10-26|Resources might benefit yesterday relations. Urban boats demonstrate main, following sales. Materials accept therefore thoughts. Short, particular paymen|8.95|7.96|9005003|scholarmaxi #3|5|history|9|Books|473|priationese|N/A|midnight831047755905|purple|Oz|Unknown|23|callyeseesecally| +6447|AAAAAAAAOCJBAAAA|2000-10-27||Resources might benefit yesterday relations. Urban boats demonstrate main, following sales. Materials accept therefore thoughts. Short, particular paymen|2.61|2.24|6001008|amalgcorp #8|5|birdal|6|Jewelry|101|priationese|N/A|midnight831047755905|olive|Oz|Unknown|17|ationeseesecally| +6448|AAAAAAAAADJBAAAA|1997-10-27|1999-10-27|Now small men may not come more neither annual others. Special things adjust further too black places. Economic, |5.15|4.01|3003001|exportiexporti #1|3|toddlers|3|Children|53|prianti|N/A|839555goldenrod48532|papaya|Cup|Unknown|1|eingeseesecally| +6449|AAAAAAAAADJBAAAA|1999-10-28|2001-10-26|Also criminal players minimise front, large|7.82|5.16|8004010|edu packnameless #10|3|camping|8|Sports|53|prianti|N/A|62773089788papaya220|tan|N/A|Unknown|35|n steseesecally| +6450|AAAAAAAAADJBAAAA|2001-10-27||Green operations must not cross always able gi|4.11|5.16|8009001|maxinameless #1|9|optics|8|Sports|266|prianti|N/A|62773089788papaya220|pale|Pallet|Unknown|19|barantiesecally| +6451|AAAAAAAADDJBAAAA|1997-10-27||International, capital forces give even english males. Downstairs urban resources bear especially regions. Residential, temporary ears know then; alone concerns proceed there national b|4.96|3.67|6016004|corpbrand #4|16|consignment|6|Jewelry|214|eseoughtable|N/A|6037245990615white70|saddle|Box|Unknown|41|oughtantiesecally| +6452|AAAAAAAAEDJBAAAA|1997-10-27|2000-10-26|Nuclear cards cannot use. Straight generations hear suddenly. Special charts live seriously directors; either technological offices might not begin more thus double cards. Growing, red entries c|65.88|28.98|7004003|edu packbrand #3|4|curtains/drapes|7|Home|454|eseantiese|N/A|613880945680rose0963|rose|N/A|Unknown|43|ableantiesecally| +6453|AAAAAAAAEDJBAAAA|2000-10-27||Nuclear cards cannot use. Straight generations hear suddenly. Special charts live seriously directors; either technological offices might not begin more thus double cards. Growing, red entries c|8.10|28.98|7004003|importoexporti #2|2|infants|3|Children|454|eseantiese|small|smoke007067424148301|sky|Dozen|Unknown|25|priantiesecally| +6454|AAAAAAAAGDJBAAAA|1997-10-27|1999-10-27|Cultural tories go workers. Usual articles can need again unions; acceptable, private miles might not organise internal years. Facts take. Even extensive books like for a righ|8.11|6.97|2001001|amalgimporto #1|1|accessories|2|Men|125|antiableought|large|65532642turquoise130|peach|Each|Unknown|83|eseantiesecally| +6455|AAAAAAAAGDJBAAAA|1999-10-28|2001-10-26|Early, brilliant vehicles sha|2.21|1.45|2001001|corpmaxi #6|1|golf|8|Sports|754|antiableought|N/A|65532642turquoise130|sky|Ounce|Unknown|41|antiantiesecally| +6456|AAAAAAAAGDJBAAAA|2001-10-27||Just silly cards come effects. Now local experiences cannot understand only for the groups. Characters think here slim applications; brief, used items will take immediately for a nov|0.81|0.44|2001001|edu packamalg #1|4|swimwear|1|Women|754|antiableought|economy|742aquamarine9272281|turquoise|Cup|Unknown|16|callyantiesecally| +6457|AAAAAAAAJDJBAAAA|1997-10-27||Domestic, physical characteristics channel on a practitioners. Far from impo|2.73|2.34|5003002|exportischolar #2|3|pop|5|Music|573|priationanti|N/A|6293312474forest1600|sandy|Oz|Unknown|18|ationantiesecally| +6458|AAAAAAAAKDJBAAAA|1997-10-27|2000-10-26|Concerned, reasonable parameters would not replace groups; temporarily clear values ought to defend safe sorts. Tears may not go in a items; readers will ref|58.21|50.06|5004001|edu packscholar #1|4|classical|5|Music|121|oughtableought|N/A|salmon25326413246410|salmon|Ounce|Unknown|2|eingantiesecally| +6459|AAAAAAAAKDJBAAAA|2000-10-27||Concerned, reasonable parameters would not replace groups; temporarily clear values ought to defend safe sorts. Tears may not go in a items; readers will ref|1.06|0.68|5004002|edu packscholar #2|4|classical|5|Music|121|oughtableought|N/A|9868khaki20164011112|magenta|Bundle|Unknown|27|n stantiesecally| +6460|AAAAAAAAMDJBAAAA|1997-10-27|1999-10-27|Likely, huge workers should leave proposal|9.67|5.41|10003001|exportiunivamalg #1|3|dvd/vcr players|10|Electronics|79|n station|N/A|1magenta746782546610|steel|Gross|Unknown|65|barcallyesecally| +6461|AAAAAAAAMDJBAAAA|1999-10-28|2001-10-26|Likely, huge workers should leave proposal|1.32|5.41|5002002|importoscholar #2|2|country|5|Music|79|n station|N/A|1magenta746782546610|lemon|Pallet|Unknown|62|oughtcallyesecally| +6462|AAAAAAAAMDJBAAAA|2001-10-27||Likely, huge workers should leave proposal|2.52|5.41|5002002|importonameless #3|2|baseball|8|Sports|79|n station|N/A|874548273papaya75881|rosy|Each|Unknown|12|ablecallyesecally| +6463|AAAAAAAAPDJBAAAA|1997-10-27||Important, working terms love t|1.30|0.46|6016008|corpbrand #8|16|consignment|6|Jewelry|246|callyeseable|N/A|6olive41593548790149|almond|Oz|Unknown|53|pricallyesecally| +6464|AAAAAAAAAEJBAAAA|1997-10-27|2000-10-26|European deals should suppress then full boots; then dead prayers must emphasize just; children will feel high satisfactory troops. Elections overcome as well busy years. |79.77|65.41|9014009|edu packunivamalg #9|14|sports|9|Books|418|eingoughtese|N/A|35896902486steel5722|powder|N/A|Unknown|79|esecallyesecally| +6465|AAAAAAAAAEJBAAAA|2000-10-27||Large sheets write also particular, functional paintings; prisoners assume yet new feet. Never extra systems will trigger either then silent days; songs afford bad dive|4.63|65.41|9014009|scholarnameless #6|14|tables|7|Home|273|priationable|N/A|35896902486steel5722|peru|Tsp|Unknown|11|anticallyesecally| +6466|AAAAAAAACEJBAAAA|1997-10-27|1999-10-27|Teenage, active condi|3.04|2.24|2004001|edu packimporto #1|4|sports-apparel|2|Men|12|ableought|medium|plum8556321981589048|green|Gram|Unknown|98|callycallyesecally| +6467|AAAAAAAACEJBAAAA|1999-10-28|2001-10-26|Abroad new others see as usual other, regular circumstances; similar, other words must not offer yesterday medical struc|9.90|7.92|2004001|edu packunivamalg #12|4|sports|9|Books|12|ableought|N/A|06maroon586810500180|yellow|N/A|Unknown|10|ationcallyesecally| +6468|AAAAAAAACEJBAAAA|2001-10-27||Abroad new others see as usual other, regular circumstances; similar, other words must not offer yesterday medical struc|5.25|2.20|2004001|edu packamalg #1|4|swimwear|1|Women|226|ableought|medium|4430432papaya6980872|pale|Bundle|Unknown|19|eingcallyesecally| +6469|AAAAAAAAFEJBAAAA|1997-10-27||Standards note. Pictures reassure only small beliefs; clear, short vehicles take small, old books. Also other groups serve rather than the children. Potentiall|4.80|3.88|6015006|scholarbrand #6|15|custom|6|Jewelry|238|eingpriable|N/A|09303purple778201976|plum|Case|Unknown|1|n stcallyesecally| +6470|AAAAAAAAGEJBAAAA|1997-10-27|2000-10-26|Endless, small hills cope again as ready forces. Ideal windows would not repeat so interested shoes. Really interesting stars suppress functional, local farmers. Leaves obtai|9.02|6.49|9015003|scholarunivamalg #3|15|fiction|9|Books|256|callyantiable|N/A|40bisque643206127444|steel|Tsp|Unknown|9|barationesecally| +6471|AAAAAAAAGEJBAAAA|2000-10-27||Previous, crude players monitor psychologically clearly various users.|0.13|0.10|4001002|amalgedu pack #2|1|womens|4|Shoes|256|callyantiable|extra large|40bisque643206127444|burnished|Tsp|Unknown|99|oughtationesecally| +6472|AAAAAAAAIEJBAAAA|1997-10-27|1999-10-27|Securities might lie only national hands. Spatial businesses enquire women. Vital records stop ill; below correct children |8.26|5.45|7011003|amalgnameless #3|11|accent|7|Home|178|eingationought|N/A|85060555986drab10125|honeydew|Gram|Unknown|13|ableationesecally| +6473|AAAAAAAAIEJBAAAA|1999-10-28|2001-10-26|Securities might lie only national hands. Spatial businesses enquire women. Vital records stop ill; below correct children |0.83|5.45|9003006|exportimaxi #6|3|computers|9|Books|178|eingationought|N/A|85060555986drab10125|lime|Dozen|Unknown|78|priationesecally| +6474|AAAAAAAAIEJBAAAA|2001-10-27||Securities might lie only national hands. Spatial businesses enquire women. Vital records stop ill; below correct children |6.18|5.45|9003006|importoedu pack #1|3|mens|4|Shoes|178|eingationought|medium|87gainsboro824284757|pale|Carton|Unknown|71|eseationesecally| +6475|AAAAAAAALEJBAAAA|1997-10-27||Times shall not scramble frequently on a foreigners. Players see then effective companies. Unemployed penalties must stop in the countries; courses make just extra, famous difficulties. Whole, tiny pa|1.58|0.66|5001002|amalgscholar #2|1|rock|5|Music|286|callyeingable|N/A|2329sandy36666880302|turquoise|Case|Unknown|30|antiationesecally| +6476|AAAAAAAAMEJBAAAA|1997-10-27|2000-10-26|Appoin|8.41|2.52|1004001|edu packamalg #1|4|swimwear|1|Women|826|callyableeing|medium|67031thistle57184968|navajo|Bundle|Unknown|12|callyationesecally| +6477|AAAAAAAAMEJBAAAA|2000-10-27||Appoin|2.49|2.52|1004001|scholarnameless #8|15|tables|7|Home|826|callyableeing|N/A|67031thistle57184968|rose|Case|Unknown|6|ationationesecally| +6478|AAAAAAAAOEJBAAAA||1999-10-27|Inevitable paintings shall forget more front, mass reasons. Good records u||0.20||||mattresses|||||||plum|Bundle|Unknown|5|| +6479|AAAAAAAAOEJBAAAA|1999-10-28|2001-10-26|Hills house also about central months. Magazines stand at all. Links shall n|1.17|0.20|7009009|edu packimporto #2|9|sports-apparel|2|Men|532|ableprianti|small|1478917tan1150387757|papaya|Gram|Unknown|19|n stationesecally| +6480|AAAAAAAAOEJBAAAA|2001-10-27||Hills house also about central months. Magazines stand at all. Links shall n|3.50|0.20|7009009|edu packamalg #1|9|swimwear|1|Women|532|ableprianti|petite|1478917tan1150387757|rose|Gross|Unknown|90|bareingesecally| +6481|AAAAAAAABFJBAAAA|1997-10-27||Stages i|5.37|3.16|4001002|amalgedu pack #2|1|womens|4|Shoes|65|antically|petite|1turquoise0862841391|tomato|Pallet|Unknown|28|oughteingesecally| +6482|AAAAAAAACFJBAAAA|1997-10-27|2000-10-26|Developments ought to preserve out of a branches. Progressive, popular amounts must not fulfil hence sensit|2.46|1.42|5001001|amalgscholar #1|1|rock|5|Music|30|barpri|N/A|589192487royal680888|yellow|Box|Unknown|4|ableeingesecally| +6483|AAAAAAAACFJBAAAA|2000-10-27||Developments ought to preserve out of a branches. Progressive, popular amounts must not fulfil hence sensit|4.04|1.42|5001001|exportiedu pack #2|1|kids|4|Shoes|405|barpri|N/A|7200maroon6534265481|violet|Dram|Unknown|31|prieingesecally| +6484|AAAAAAAAEFJBAAAA|1997-10-27|1999-10-27|Ultimate stands will like bright, possible tickets. Lips hesitate. Miners could not see today drinks. Only unions must convert very written movements. Senior options must not generate |2.76|1.38|10006008|corpunivamalg #8|6|musical|10|Electronics|315|antioughtpri|N/A|200726seashell557017|indian|Ton|Unknown|53|eseeingesecally| +6485|AAAAAAAAEFJBAAAA|1999-10-28|2001-10-26|Ultimate stands will like bright, possible tickets. Lips hesitate. Miners could not see today drinks. Only unions must convert very written movements. Senior options must not generate |5.29|1.38|9002006|importomaxi #6|6|business|9|Books|94|esen st|N/A|03545247747774smoke8|saddle|Case|Unknown|34|antieingesecally| +6486|AAAAAAAAEFJBAAAA|2001-10-27||Special, thin details possess very free changes. Briefly democratic sol|6.34|5.57|9002006|scholarmaxi #7|5|history|9|Books|247|esen st|N/A|1753767423947navajo4|powder|Tbl|Unknown|18|callyeingesecally| +6487|AAAAAAAAHFJBAAAA|1997-10-27||Smooth, existing teachers st|2.53|1.59|10013011|exportiamalgamalg #11|13|stereo|10|Electronics|187|ationeingought|N/A|29880157716azure0566|peach|Each|Unknown|10|ationeingesecally| +6488|AAAAAAAAIFJBAAAA|1997-10-27|2000-10-26|Letters can tell roman parties. Local, outside papers shall hear as big years. Grey, dark articles would get then small, d|2.03|0.85|1004001|edu packamalg #1|4|swimwear|1|Women|114|eseoughtought|medium|763626slate579870213|white|Box|Unknown|41|eingeingesecally| +6489|AAAAAAAAIFJBAAAA|2000-10-27||Vast, open views can beat still without the governments. Tasks could not become merely personal parents. Once vital places think earlier. Se|0.33|0.85|6015002|scholarbrand #2|4|custom|6|Jewelry|114|eseoughtought|N/A|763626slate579870213|puff|Tsp|Unknown|18|n steingesecally| +6490|AAAAAAAAKFJBAAAA|1997-10-27|1999-10-27|Open interests would not use good, light times. Most english wives complement most shares; high f|9.37|5.05|2001001|amalgimporto #1|1|accessories|2|Men|143|prieseought|extra large|0321846063035wheat56|misty|Gram|Unknown|76|barn stesecally| +6491|AAAAAAAAKFJBAAAA|1999-10-28|2001-10-26|Open interests would not use good, light times. Most english wives complement most shares; high f|3.25|1.26|8015002|scholarmaxi #2|1|fishing|8|Sports|143|prieseought|N/A|0321846063035wheat56|powder|Bunch|Unknown|54|oughtn stesecally| +6492|AAAAAAAAKFJBAAAA|2001-10-27||Fears may not become. Clear years must not allow males. Dear friends prepare also even military objects; stiffly |7.56|4.08|2001001|amalgimporto #1|1|accessories|2|Men|143|prieseought|small|92112020159367misty5|honeydew|Tbl|Unknown|9|ablen stesecally| +6493|AAAAAAAANFJBAAAA|1997-10-27||Only old doors shall wear again. Earlier high minerals might not tell better persona|16.62|5.31|9003002|exportimaxi #2|3|computers|9|Books|1000|barbarbarought|N/A|9975572955beige97058|powder|Oz|Unknown|43|prin stesecally| +6494|AAAAAAAAOFJBAAAA|1997-10-27|2000-10-26|Reasonable, political grants know organic qualifications|0.28|0.15|2002001|importoimporto #1|2|shirts|2|Men|215|antioughtable|petite|56602060442saddle349|cyan|Case|Unknown|48|esen stesecally| +6495|AAAAAAAAOFJBAAAA|2000-10-27||Statutory, strange industr|2.81|2.13|1003002|exportiamalg #2|3|maternity|1|Women|215|antioughtable|petite|56602060442saddle349|sienna|Bundle|Unknown|19|antin stesecally| +6496|AAAAAAAAAGJBAAAA|1997-10-27|1999-10-27|Sophisticated stories ought to ignore particularly pregnant railways; pregnant, absolute events take now tomorrow applicable applications. Central others miss unique, able managers. Better smal|6.65|2.19|2001001|amalgimporto #1|1|accessories|2|Men|298|eingn stable|extra large|magenta2747809831456|violet|Lb|Unknown|43|callyn stesecally| +6497|AAAAAAAAAGJBAAAA|1999-10-28|2001-10-26|Simply powerful powers work limits; simple, fond years could become too liberal eyes. About distinct birds cannot come n|1.75|2.19|3001002|amalgexporti #2|1|newborn|3|Children|208|eingbarable|large|3729bisque3036510112|tan|Pallet|Unknown|54|ationn stesecally| +6498|AAAAAAAAAGJBAAAA|2001-10-27||Simply powerful powers work limits; simple, fond years could become too liberal eyes. About distinct birds cannot come n|7.97|2.19|3001002|importobrand #3|12|costume|6|Jewelry|208|eingbarable|N/A|325943papaya79271516|lemon|Box|Unknown|52|eingn stesecally| +6499|AAAAAAAADGJBAAAA|1997-10-27||Sure companies secure to and fro unnecessa|2.84|0.85|8015010|scholarmaxi #10|15|fishing|8|Sports|561|oughtcallyanti|N/A|0960697347peach07406|salmon|Ton|Unknown|22|n stn stesecally| +6500|AAAAAAAAEGJBAAAA|1997-10-27|2000-10-26|Again small deaths could flou|6.40|3.64|5003001|exportischolar #1|3|pop|5|Music|491|oughtn stese|N/A|05026967turquoise358|pink|Ton|Unknown|44|barbarantically| +6501|AAAAAAAAEGJBAAAA|2000-10-27||Clients would win at all public arms. Fresh, great plans could get. Findings sit th|0.75|3.64|9003010|exportimaxi #10|3|computers|9|Books|425|antiableese|N/A|05026967turquoise358|papaya|Dram|Unknown|13|oughtbarantically| +6502|AAAAAAAAGGJBAAAA|1997-10-27|1999-10-27|Authorities satisfy exactly conditions. Odd words might rule only only elderly words. Possible weekends reject sometimes. Particularly safe members could follow that is magic houses. Soon compa|6.33|5.00|1001001|amalgamalg #1|1|dresses|1|Women|152|ableantiought|extra large|609252drab5373047513|red|Cup|Unknown|65|ablebarantically| +6503|AAAAAAAAGGJBAAAA|1999-10-28|2001-10-26|Authorities satisfy exactly conditions. Odd words might rule only only elderly words. Possible weekends reject sometimes. Particularly safe members could follow that is magic houses. Soon compa|4.56|5.00|4004002|edu packedu pack #2|1|athletic|4|Shoes|152|ableantiought|petite|6645854hot9619480846|royal|Case|Unknown|91|pribarantically| +6504|AAAAAAAAGGJBAAAA|2001-10-27||Authorities satisfy exactly conditions. Odd words might rule only only elderly words. Possible weekends reject sometimes. Particularly safe members could follow that is magic houses. Soon compa|74.07|5.00|8011005|amalgmaxi #5|1|archery|8|Sports|152|ableantiought|N/A|527409517248peru5853|indian|Unknown|Unknown|11|esebarantically| +6505|AAAAAAAAJGJBAAAA|||Urban, powerful concepts should discern efficiently others. Far other activities should not see politically politica|4.42||3001002|amalgexporti #2|1|||Children|114|||0820260228olive96303|sky|Lb|Unknown|58|antibarantically| +6506|AAAAAAAAKGJBAAAA|1997-10-27|2000-10-26|Easy values demonstrate late fair, hostile children. In order similar pairs must say long with the tales. Fully just officers cannot raise lines; days care polite,|67.80|56.27|2004001|edu packimporto #1|4|sports-apparel|2|Men|83|prieing|large|689795red40590043861|peru|Carton|Unknown|49|callybarantically| +6507|AAAAAAAAKGJBAAAA|2000-10-27||Easy values demonstrate late fair, hostile children. In order similar pairs must say long with the tales. Fully just officers cannot raise lines; days care polite,|5.68|56.27|2004002|edu packimporto #2|4|sports-apparel|2|Men|1|ought|large|8frosted215091180380|chartreuse|Pound|Unknown|29|ationbarantically| +6508|AAAAAAAAMGJBAAAA|1997-10-27|1999-10-27|Twice real affairs will cause chiefly. Industrial, private words differentiate still popular, worried schemes. Wide, british bits collect traditiona|0.84|0.60|4004001|edu packedu pack #1|4|athletic|4|Shoes|41|oughtese|small|924887753purple31610|violet|Case|Unknown|9|eingbarantically| +6509|AAAAAAAAMGJBAAAA|1999-10-28|2001-10-26|Twice real affairs will cause chiefly. Industrial, private words differentiate still popular, worried schemes. Wide, british bits collect traditiona|6.92|4.42|4004001|univmaxi #10|10|pools|8|Sports|41|oughtese|N/A|6seashell94458210450|yellow|Carton|Unknown|17|n stbarantically| +6510|AAAAAAAAMGJBAAAA|2001-10-27||Twice real affairs will cause chiefly. Industrial, private words differentiate still popular, worried schemes. Wide, british bits collect traditiona|4.05|4.42|4004001|namelessbrand #1|8|lighting|7|Home|544|oughtese|N/A|4796pink341873040150|deep|N/A|Unknown|26|baroughtantically| +6511|AAAAAAAAPGJBAAAA|1997-10-27||For example silver voters escape here delightful, other injuries. Light countries would fear by now young, new men. Underlying ch|3.55|1.10|5003002|exportischolar #2|3|pop|5|Music|81|oughteing|N/A|3266577678turquoise6|papaya|Gram|Unknown|17|oughtoughtantically| +6512|AAAAAAAAAHJBAAAA|1997-10-27|2000-10-26|International patterns may expect with a|1.29|0.82|4002001|importoedu pack #1|2|mens|4|Shoes|780|bareingation|medium|74salmon013251726306|grey|Pound|Unknown|23|ableoughtantically| +6513|AAAAAAAAAHJBAAAA|2000-10-27||Players cannot look houses. All other ways think here centres. Companies should deserve always various|2.61|0.82|4002001|importoimporto #2|2|shirts|2|Men|780|bareingation|extra large|8703514219powder2317|lace|Carton|Unknown|2|prioughtantically| +6514|AAAAAAAACHJBAAAA|1997-10-27|1999-10-27|Others will think forward. Social, particular unions might employ then leading, long-term needs. Deaths may convince. Ever gre|5.08|3.35|2004001|edu packimporto #1|4|sports-apparel|2|Men|243|prieseable|medium|530016salmon75563009|papaya|Lb|Unknown|9|eseoughtantically| +6515|AAAAAAAACHJBAAAA|1999-10-28|2001-10-26|Others will think forward. Social, particular unions might employ then leading, long-term needs. Deaths may convince. Ever gre|4.11|1.56|2004001|corpnameless #6|6|football|8|Sports|155|prieseable|N/A|5360violet1068192100|chocolate|Gram|Unknown|37|antioughtantically| +6516|AAAAAAAACHJBAAAA|2001-10-27||Allies might not say autho|9.04|1.56|2001001|amalgimporto #1|6|accessories|2|Men|155|antiantiought|small|5360violet1068192100|tan|Dram|Unknown|7|callyoughtantically| +6517|AAAAAAAAFHJBAAAA|1997-10-27||Only other books survive inside to a players. Others will sell social roots. Small, common proble|2.52|1.03|3003002|exportiexporti #2|3|toddlers|3|Children|196|callyn stought|large|7795553729618purple1|pink|Bundle|Unknown|31|ationoughtantically| +6518|AAAAAAAAGHJBAAAA|1997-10-27|2000-10-26|Central feet could not find obviously colourful, french awards. Right able holes see immediately green|42.58|34.48|10009008|maxiunivamalg #8|9|televisions|10|Electronics|639|n stprically|N/A|95122879sandy2976852|seashell|Each|Unknown|16|eingoughtantically| +6519|AAAAAAAAGHJBAAAA|2000-10-27||Central feet could not find obviously colourful, french awards. Right able holes see immediately green|8.09|34.48|10009008|exportiexporti #2|9|toddlers|3|Children|639|n stprically|medium|95122879sandy2976852|dark|Tbl|Unknown|4|n stoughtantically| +6520|AAAAAAAAIHJBAAAA|1997-10-27|1999-10-27|Necessary funds need |2.68|1.71|10008010|namelessunivamalg #10|8|scanners|10|Electronics|649|n stesecally|N/A|034burnished07897870|royal|Gram|Unknown|50|barableantically| +6521|AAAAAAAAIHJBAAAA|1999-10-28|2001-10-26|Appeals can lose more as appropriate organisers. Relative, back buildings like notably. Tiny authorities boost more also elegant questi|1.56|0.95|10008010|scholarmaxi #6|8|history|9|Books|501|oughtbaranti|N/A|1puff688669273479601|white|Bundle|Unknown|59|oughtableantically| +6522|AAAAAAAAIHJBAAAA|2001-10-27||Appeals can lose more as appropriate organisers. Relative, back buildings like notably. Tiny authorities boost more also elegant questi|6.05|2.35|7014003|edu packnameless #3|8|glassware|7|Home|501|oughtbaranti|N/A|1puff688669273479601|peach|Gram|Unknown|32|ableableantically| +6523|AAAAAAAALHJBAAAA|1997-10-27||All international functions can improve. Visitors take possible companies. Important eyes shall not follow at a sports. Glad p|4.31|3.74|4001002|amalgedu pack #2|1|womens|4|Shoes|152|ableantiought|large|044987943164pink3752|sky|Ounce|Unknown|76|priableantically| +6524|AAAAAAAAMHJBAAAA|1997-10-27|2000-10-26|Widespread comp|9.91|2.97|1004001|edu packamalg #1|4|swimwear|1|Women|244|eseeseable|large|04275967violet781748|navajo|Dozen|Unknown|14|eseableantically| +6525|AAAAAAAAMHJBAAAA|2000-10-27||Lives position relatively at a grants. Great, whole differences try. Terrible injuries|9.40|7.42|3003002|exportiexporti #2|3|toddlers|3|Children|244|eseeseable|large|04275967violet781748|salmon|Ounce|Unknown|47|antiableantically| +6526|AAAAAAAAOHJBAAAA|1997-10-27|1999-10-27|Fair years ensure likely, elected proceedings. Utterly asian products used to get light bird|6.54|2.41|10008016|namelessunivamalg #16|8|scanners|10|Electronics|543|prieseanti|N/A|234576868644tan14397|papaya|Pallet|Unknown|7|callyableantically| +6527|AAAAAAAAOHJBAAAA|1999-10-28|2001-10-26|Fair years ensure likely, elected proceedings. Utterly asian products used to get light bird|0.83|0.36|8005008|scholarnameless #8|5|fitness|8|Sports|543|prieseanti|N/A|234576868644tan14397|plum|Dram|Unknown|12|ationableantically| +6528|AAAAAAAAOHJBAAAA|2001-10-27||Fair years ensure likely, elected proceedings. Utterly asian products used to get light bird|6.08|0.36|9007001|brandmaxi #1|5|reference|9|Books|543|prieseanti|N/A|234576868644tan14397|lace|Case|Unknown|62|eingableantically| +6529|AAAAAAAABIJBAAAA|1997-10-27||New, good places used to gain various qualities. Rivals make probably other, wide trials; roads ought to prompt males. Women must not form immediate, back situations. Different, local applications c|0.99|0.89|6007002|brandcorp #2|7|pendants|6|Jewelry|96|callyn st|N/A|9391878wheat00861477|royal|Case|Unknown|59|n stableantically| +6530|AAAAAAAACIJBAAAA|1997-10-27|2000-10-26|Principles distract apart at a roles. Middle forces should die there in a imports. Fol|1.96|1.37|4004001|edu packedu pack #1|4|athletic|4|Shoes|906|callybarn st|small|00752122495red865859|powder|Case|Unknown|95|barpriantically| +6531|AAAAAAAACIJBAAAA|2000-10-27||Principles distract apart at a roles. Middle forces should die there in a imports. Fol|1.43|0.52|6013004|exportibrand #4|4|loose stones|6|Jewelry|906|callybarn st|N/A|00752122495red865859|yellow|Each|Unknown|22|oughtpriantically| +6532|AAAAAAAAEIJBAAAA|1997-10-27|1999-10-27|Areas|40.20|12.46|10004005|edu packunivamalg #5|4|audio|10|Electronics|329|n stablepri|N/A|52hot788049211291203|black|N/A|Unknown|5|ablepriantically| +6533|AAAAAAAAEIJBAAAA|1999-10-28|2001-10-26|Countries used to portray about political, sure mammals. Managers find thus senior years. Other ranks can limit proceedings; difficult leaders must depend ill. Large, commo|9.37|12.46|10004005|edu packexporti #2|4|school-uniforms|3|Children|629|n stablepri|small|444291657plum9226644|medium|Gram|Unknown|76|pripriantically| +6534|AAAAAAAAEIJBAAAA|2001-10-27||Quickly new events aid more. Front, historical combinations throw much full friends; about complete parties remember therefore yesterday special expectations; should|5.13|4.46|10004005|amalgimporto #1|1|accessories|2|Men|629|n stablecally|small|444291657plum9226644|puff|Unknown|Unknown|3|esepriantically| +6535|AAAAAAAAHIJBAAAA|1997-10-27||Features can get; fiscal, important considerations must claim then wrong bodies; various houses postpone yet spirits. Provincial, complete managers a|0.55|0.33|8006006|corpnameless #6|6|football|8|Sports|645|antiesecally|N/A|02317royal6909224425|lavender|Ton|Unknown|27|antipriantically| +6536|AAAAAAAAIIJBAAAA|1997-10-27|2000-10-26|Already unexpected relations must investigate sooner new fair |26.55|20.70|9005009|scholarmaxi #9|5|history|9|Books|104|esebarought|N/A|7884246peru614288993|linen|Carton|Unknown|81|callypriantically| +6537|AAAAAAAAIIJBAAAA|2000-10-27||So appropriate details tell onwards dead, available muscles. Massive things ask tonight thick collections. Americans compete even extreme, international societies; liberal|0.86|0.26|9005009|namelessbrand #8|8|lighting|7|Home|74|esebarought|N/A|7884246peru614288993|orange|Lb|Unknown|7|ationpriantically| +6538|AAAAAAAAKIJBAAAA|1997-10-27|1999-10-27|Excellent, specific peri|5.79|4.86|4004001|edu packedu pack #1|4|athletic|4|Shoes|414|eseoughtese|small|71214815royal7459854|pink|Ounce|Unknown|26|eingpriantically| +6539|AAAAAAAAKIJBAAAA|1999-10-28|2001-10-26|Excellent, specific peri|1.38|1.06|7009010|maxibrand #10|4|mattresses|7|Home|500|barbaranti|N/A|812firebrick68095151|saddle|Dozen|Unknown|21|n stpriantically| +6540|AAAAAAAAKIJBAAAA|2001-10-27||Excellent, specific peri|4.13|2.68|9003001|exportimaxi #1|3|computers|9|Books|167|ationcallyought|N/A|72091707papaya156205|steel|Each|Unknown|59|bareseantically| +6541|AAAAAAAANIJBAAAA|1997-10-27||Serious, necessary parts ought to accommodate all results. Even national patterns should break as doors|0.50|0.41|10007014|brandunivamalg #14|7|personal|10|Electronics|58|einganti|N/A|9290red8879024577351|purple|Carton|Unknown|42|oughteseantically| +6542|AAAAAAAAOIJBAAAA|1997-10-27|2000-10-26|New results stop beyond at a cuts. Natural powers find probably very constant characteristics. Members ought to convince still a bit semantic applications. Profession|3.12|2.15|4004001|edu packedu pack #1|4|athletic|4|Shoes|712|ableoughtation|medium|251909turquoise25268|sky|Case|Unknown|29|ableeseantically| +6543|AAAAAAAAOIJBAAAA|2000-10-27||Temporary plants may not help by the cards. Perhaps important dimensions give also rare, shared opportunities; other programme|8.62|2.15|4004001|edu packmaxi #4|4|entertainments|9|Books|216|ableoughtation|N/A|251909turquoise25268|moccasin|Pound|Unknown|50|prieseantically| +6544|AAAAAAAAAJJBAAAA|1997-10-27|1999-10-27|Suddenly additional workers might restore for example. Remaining, passive goods might imagine neat issues; miserable, long services would not form yet. In full various priv|2.63|1.44|10004017|edu packunivamalg #17|4|audio|10|Electronics|594|esen stanti|N/A|014340058goldenrod58|linen|Ton|Unknown|31|eseeseantically| +6545|AAAAAAAAAJJBAAAA|1999-10-28|2001-10-26|Suddenly additional workers might restore for example. Remaining, passive goods might imagine neat issues; miserable, long services would not form yet. In full various priv|4.56|3.64|8012006|importomaxi #6|4|guns|8|Sports|103|pribarought|N/A|014340058goldenrod58|smoke|Each|Unknown|32|antieseantically| +6546|AAAAAAAAAJJBAAAA|2001-10-27||Suddenly additional workers might restore for example. Remaining, passive goods might imagine neat issues; miserable, long services would not form yet. In full various priv|3.47|3.08|1002001|importoamalg #1|4|fragrances|1|Women|678|eingationcally|small|014340058goldenrod58|orchid|Each|Unknown|6|callyeseantically| +6547|AAAAAAAADJJBAAAA|1997-10-27||Social shops should provide soon na|4.56|2.87|1003002|exportiamalg #2|3|maternity|1|Women|309|n stbarpri|small|91346927334cream6405|sandy|Gram|Unknown|6|ationeseantically| +6548|AAAAAAAAEJJBAAAA|1997-10-27|2000-10-26|Most lovely procedures could learn once very distinct steps. Effects could claim however democratic bonds. Extremely other problems could explain much very leading changes. Legal, general doors belong|3.66|1.93|10016004|corpamalgamalg #4|16|wireless|10|Electronics|253|priantiable|N/A|206638881813lace0335|antique|Ton|Unknown|49|eingeseantically| +6549|AAAAAAAAEJJBAAAA|2000-10-27||Else conventional women shall not preven|9.24|1.93|10016004|amalgunivamalg #5|1|cameras|10|Electronics|339|priantiable|N/A|206638881813lace0335|navy|Tsp|Unknown|20|n steseantically| +6550|AAAAAAAAGJJBAAAA|1997-10-27|1999-10-27|I|73.49|39.68|8005001|scholarnameless #1|5|fitness|8|Sports|210|baroughtable|N/A|81131560498coral4415|sienna|Dozen|Unknown|12|barantiantically| +6551|AAAAAAAAGJJBAAAA|1999-10-28|2001-10-26|Difficult ideas should go badly observers. Much able premises im|22.77|39.68|6013008|exportibrand #8|5|loose stones|6|Jewelry|210|baroughtable|N/A|81131560498coral4415|snow|Box|Unknown|79|oughtantiantically| +6552|AAAAAAAAGJJBAAAA|2001-10-27||Difficult ideas should go badly observers. Much able premises im|4.01|39.68|2004001|edu packimporto #1|5|sports-apparel|2|Men|210|baroughtable|extra large|3265525903sienna7991|saddle|Box|Unknown|15|ableantiantically| +6553|AAAAAAAAJJJBAAAA|1997-10-27||Frien|8.05|6.68|1004002|edu packamalg #2|4|swimwear|1|Women|302|ablebarpri|large|3lemon04775634340399|green|Lb|Unknown|1|priantiantically| +6554|AAAAAAAAKJJBAAAA|1997-10-27|2000-10-26|Social, white branches shall not think grounds. Less upper men rejoin. Measures take nevertheless very costs; green characteristics let materials. High patie|4.80|2.11|1002001|importoamalg #1|2|fragrances|1|Women|194|esen stought|extra large|153698violet74196439|smoke|Pallet|Unknown|95|eseantiantically| +6555|AAAAAAAAKJJBAAAA|2000-10-27||Social, white branches shall not think grounds. Less upper men rejoin. Measures take nevertheless very costs; green characteristics let materials. High patie|2.22|1.15|1002001|corpbrand #4|2|consignment|6|Jewelry|540|esen stought|N/A|153698violet74196439|pale|Ton|Unknown|8|antiantiantically| +6556|AAAAAAAAMJJBAAAA|1997-10-27|1999-10-27|Just little machines used to maintain else. Improvements call right daily children. Human, i|1.17|0.93|8009007|maxinameless #7|9|optics|8|Sports|320|barablepri|N/A|31041729987yellow938|chartreuse|Dozen|Unknown|5|callyantiantically| +6557|AAAAAAAAMJJBAAAA|1999-10-28|2001-10-26|Particular, english institutions complete only eyes. Steps lik|1.98|0.93|10007013|brandunivamalg #13|9|personal|10|Electronics|594|barablepri|N/A|5157386ivory01306484|sandy|Unknown|Unknown|28|ationantiantically| +6558|AAAAAAAAMJJBAAAA|2001-10-27||Secondary profits could exhibit also on a videos. Daughters must use full, male cases. As influential managers like joint, strange churches. Clear, complete ideas will happen disciplinary|0.71|0.31|10007013|exportiedu pack #1|3|kids|4|Shoes|805|barablepri|medium|5157386ivory01306484|pale|Oz|Unknown|99|eingantiantically| +6559|AAAAAAAAPJJBAAAA|1997-10-27||Following, crucial students might claim medical, average margins|7.00|5.74|8007010|brandnameless #10|7|hockey|8|Sports|212|ableoughtable|N/A|09white9884550109134|cornsilk|Unknown|Unknown|40|n stantiantically| +6560|AAAAAAAAAKJBAAAA|1997-10-27|2000-10-26|Social, back times might not call. Capable men go therefore at the banks. Officially hot actions show very. Whole writers ought to get. Over crude levels wo|0.94|0.66|7009001|maxibrand #1|9|mattresses|7|Home|1|ought|N/A|52821945890magenta18|wheat|Box|Unknown|66|barcallyantically| +6561|AAAAAAAAAKJBAAAA|2000-10-27||Social, back times might not call. Capable men go therefore at the banks. Officially hot actions show very. Whole writers ought to get. Over crude levels wo|2.79|1.98|7007002|brandbrand #2|7|decor|7|Home|1|ought|N/A|52821945890magenta18|steel|Ton|Unknown|10|oughtcallyantically| +6562|AAAAAAAACKJBAAAA|1997-10-27|1999-10-27|Just respectable tables can win parti|1.79|1.16|6006003|corpcorp #3|6|rings|6|Jewelry|664|esecallycally|N/A|turquoise13634076879|orange|Ton|Unknown|64|ablecallyantically| +6563|AAAAAAAACKJBAAAA|1999-10-28|2001-10-26|Just respectable tables can win parti|1.74|1.16|6006003|scholarbrand #4|6|blinds/shades|7|Home|186|esecallycally|N/A|turquoise13634076879|tan|Gross|Unknown|9|pricallyantically| +6564|AAAAAAAACKJBAAAA|2001-10-27||Just respectable tables can win parti|86.67|1.16|9012001|importounivamalg #1|12|home repair|9|Books|186|callyeingought|N/A|turquoise13634076879|sandy|Lb|Unknown|15|esecallyantically| +6565|AAAAAAAAFKJBAAAA|1997-10-27||Ethnic employers must play too popular, full a|7.68|4.07|5002002|importoscholar #2|2|country|5|Music|995|antin stn st|N/A|20066snow77773694528|blush|Unknown|Unknown|70|anticallyantically| +6566|AAAAAAAAGKJBAAAA|1997-10-27|2000-10-26|Economic, available days will not know instantly au|5.39|2.58|3003001|exportiexporti #1|3|toddlers|3|Children|327|ationablepri|large|955685turquoise20444|thistle|Each|Unknown|23|callycallyantically| +6567|AAAAAAAAGKJBAAAA|2000-10-27||Aware, wide dreams become on a trees. Principal, hungry drugs used to need passengers. Aside clear workers take. Certainly possible champions can finance. Sudden, important of|9.70|2.58|5002002|importoscholar #2|3|country|5|Music|577|ationationanti|N/A|955685turquoise20444|chocolate|Gram|Unknown|93|ationcallyantically| +6568|AAAAAAAAIKJBAAAA|1997-10-27|1999-10-27|New firms shall|30.82|27.42|3001001|amalgexporti #1|1|newborn|3|Children|94|esen st|extra large|737006tan84370490182|salmon|Ounce|Unknown|17|eingcallyantically| +6569|AAAAAAAAIKJBAAAA|1999-10-28|2001-10-26|Forwards different yards might reflect perhaps unfair woo|68.85|53.01|3001001|exportiamalgamalg #8|13|stereo|10|Electronics|94|esen st|N/A|737006tan84370490182|khaki|Pallet|Unknown|93|n stcallyantically| +6570|AAAAAAAAIKJBAAAA|2001-10-27||Forwards different yards might reflect perhaps unfair woo|5.16|53.01|5003001|exportischolar #1|3|pop|5|Music|498|eingn stese|N/A|737006tan84370490182|papaya|Box|Unknown|13|barationantically| +6571|AAAAAAAALKJBAAAA|1997-10-27||Obviously a|3.35|1.60|10012010|importoamalgamalg #10|12|monitors|10|Electronics|9|n st|N/A|859917red43949799097|plum|Tsp|Unknown|3|oughtationantically| +6572|AAAAAAAAMKJBAAAA||2000-10-26|As sure women fall proposals. Entire, loc|1.91|1.14||edu packexporti #1|4||||||extra large|khaki669398103518817|||Unknown|11|| +6573|AAAAAAAAMKJBAAAA|2000-10-27||Naked costs tell so financial bodies. Farmers make years. Royal, tiny products used to hear de|1.89|1.14|3004001|importoamalg #2|4|fragrances|1|Women|445|antieseese|petite|563214016592royal839|pale|Case|Unknown|77|priationantically| +6574|AAAAAAAAOKJBAAAA|1997-10-27|1999-10-27|Only rapid powers used to translate voluntary, angry degrees. As new backs would not know subsequently other tasks. Tight capital teams used to go perhaps essential, pos|4.12|2.22|9009011|maximaxi #11|9|science|9|Books|189|n steingought|N/A|steel996302568897638|spring|Dozen|Unknown|66|eseationantically| +6575|AAAAAAAAOKJBAAAA|1999-10-28|2001-10-26|Only rapid powers used to translate voluntary, angry degrees. As new backs would not know subsequently other tasks. Tight capital teams used to go perhaps essential, pos|5.56|2.22|9009011|amalgedu pack #2|9|womens|4|Shoes|210|baroughtable|medium|934761897papaya65238|khaki|N/A|Unknown|17|antiationantically| +6576|AAAAAAAAOKJBAAAA|2001-10-27||Rational skills die enormous, national rates; cars would not allay all. Eyes avoid brief, tall places. Old, young circumstances shall use decisions. Short courts deny |8.60|4.21|9008001|namelessmaxi #1|9|romance|9|Books|666|baroughtable|N/A|934761897papaya65238|red|Pallet|Unknown|88|callyationantically| +6577|AAAAAAAABLJBAAAA|1997-10-27||Accidents pick about the windows. Injuries understand in a parties. Sexual glasses must recover now. Major roles shall lose boxes. Again true courses mak|9.49|3.79|2002002|importoimporto #2|2|shirts|2|Men|114|eseoughtought|petite|purple97572462533281|sienna|Carton|Unknown|3|ationationantically| +6578|AAAAAAAACLJBAAAA|1997-10-27|2000-10-26|Books may not transport. Industries explain that no longer female elections. Indian w|97.82|72.38|6004003|edu packcorp #3|4|bracelets|6|Jewelry|341|oughtesepri|N/A|5999sienna9002796138|snow|Box|Unknown|11|eingationantically| +6579|AAAAAAAACLJBAAAA|2000-10-27||Fre|0.23|0.13|4001002|amalgedu pack #2|4|womens|4|Shoes|394|oughtesepri|medium|5999sienna9002796138|white|Dozen|Unknown|27|n stationantically| +6580|AAAAAAAAELJBAAAA|1997-10-27|1999-10-27|Columns might lead only for a problems. Financial shoulders belong; industrial, new miners must carry very dangerous activities; sometimes national fathers could change|6.11|2.62|8005001|scholarnameless #1|5|fitness|8|Sports|303|pribarpri|N/A|3774075467962lemon93|turquoise|Bunch|Unknown|27|bareingantically| +6581|AAAAAAAAELJBAAAA|1999-10-28|2001-10-26|Presents will refute rural emotions. Special years ask surprisingly to a feet; also just numbers may att|4.22|3.29|8005001|scholarcorp #6|5|earings|6|Jewelry|303|pribarpri|N/A|3774075467962lemon93|saddle|Gram|Unknown|65|oughteingantically| +6582|AAAAAAAAELJBAAAA|2001-10-27||Presents will refute rural emotions. Special years ask surprisingly to a feet; also just numbers may att|2.38|3.29|3002001|importoexporti #1|5|infants|3|Children|164|esecallyought|extra large|8144951purple0358122|green|Bundle|Unknown|34|ableeingantically| +6583|AAAAAAAAHLJBAAAA|1997-10-27||Much conservative factors used to build then hard, due contexts. Just left eyes shall earn quickly. Excellent months work for the hours. Voluntary circles|35.53|29.48|2003002|exportiimporto #2|3|pants|2|Men|207|ationbarable|large|3080pink714928562944|peach|Ton|Unknown|9|prieingantically| +6584|AAAAAAAAILJBAAAA|1997-10-27|2000-10-26|Arab, exceptional eyes sound then at a pictures. Points shall sustain. So thick terms could take long also civil cheeks. As well scotti|1.32|0.89|1001001|amalgamalg #1|1|dresses|1|Women|218|eingoughtable|extra large|5102554963791powder8|blush|Dozen|Unknown|18|eseeingantically| +6585|AAAAAAAAILJBAAAA|2000-10-27||Arab, exceptional eyes sound then at a pictures. Points shall sustain. So thick terms could take long also civil cheeks. As well scotti|2.77|0.89|9006010|corpmaxi #10|1|parenting|9|Books|190|barn stought|N/A|796037151992plum9499|tan|Each|Unknown|44|antieingantically| +6586|AAAAAAAAKLJBAAAA|1997-10-27|1999-10-27|Candidates address comparatively modern steps; securities should affect apparently full children. Long colleg|29.83|22.67|2004001|edu packimporto #1|4|sports-apparel|2|Men|461|oughtcallyese|medium|527889010metallic167|hot|Gross|Unknown|35|callyeingantically| +6587|AAAAAAAAKLJBAAAA|1999-10-28|2001-10-26|Candidates address comparatively modern steps; securities should affect apparently full children. Long colleg|9.71|22.67|5004002|edu packscholar #2|4|classical|5|Music|461|oughtcallyese|N/A|527889010metallic167|papaya|Ton|Unknown|64|ationeingantically| +6588|AAAAAAAAKLJBAAAA|2001-10-27||Particular, collective times cannot win. Original contexts might not find completely apparently c|9.69|22.67|5004002|edu packexporti #1|4|school-uniforms|3|Children|461|oughtcallyese|petite|958345632599salmon71|yellow|Case|Unknown|41|eingeingantically| +6589|AAAAAAAANLJBAAAA|1997-10-27||Hours can work afraid, unique arms. Natural, other costs demonstrate depending o|4.60|1.93|5001002|amalgscholar #2|1|rock|5|Music|507|ationbaranti|N/A|686313powder15668529|wheat|Lb|Unknown|43|n steingantically| +6590|AAAAAAAAOLJBAAAA|1997-10-27|2000-10-26|Well |1.68|0.62|2002001|importoimporto #1|2|shirts|2|Men|131|oughtpriought|small|tan86163084865733021|hot|Tbl|Unknown|74|barn stantically| +6591|AAAAAAAAOLJBAAAA|2000-10-27||Days used to sit probably at once mere defendants. Decisions change international talks. Friendly, f|4.55|0.62|9014004|edu packunivamalg #4|14|sports|9|Books|270|barationable|N/A|852718035powder45595|red|N/A|Unknown|92|oughtn stantically| +6592|AAAAAAAAAMJBAAAA|1997-10-27|1999-10-27|Flowers will look respectable negotiations. Standards see and so on social men. Points could play in the steps|8.23|5.84|9005005|scholarmaxi #5|5|history|9|Books|193|prin stought|N/A|17657376spring568229|pink|Gram|Unknown|23|ablen stantically| +6593|AAAAAAAAAMJBAAAA|1999-10-28|2001-10-26|Police consist other, superb roads. Devices must enjoy to a expectations. Free, modern skill|7.90|5.92|9009006|maximaxi #6|5|science|9|Books|193|prin stought|N/A|9192330963rosy401867|snow|Unknown|Unknown|87|prin stantically| +6594|AAAAAAAAAMJBAAAA|2001-10-27||Police consist other, superb roads. Devices must enjoy to a expectations. Free, modern skill|4.15|2.07|9009006|namelessunivamalg #16|5|scanners|10|Electronics|193|prin stought|N/A|9192330963rosy401867|sandy|Tsp|Unknown|34|esen stantically| +6595|AAAAAAAADMJBAAAA|1997-10-27||Anywhere fascinating articles should take however functional goods; growing, detailed problems think political, required things. Projects formulate somewhere key, political differences|4.85|1.45|2004002|edu packimporto #2|4|sports-apparel|2|Men|102|ablebarought|medium|3586580444865medium0|sandy|Gram|Unknown|26|antin stantically| +6596|AAAAAAAAEMJBAAAA|1997-10-27|2000-10-26|Common, capitalist measures appoint words. Gentle lists house. Matters form inside gentle stories. Forward various kin|8.56|6.42|3001001|amalgexporti #1|1|newborn|3|Children|47|ationese|large|5pink662361352849921|plum|Pound|Unknown|9|callyn stantically| +6597|AAAAAAAAEMJBAAAA|2000-10-27||Common, capitalist measures appoint words. Gentle lists house. Matters form inside gentle stories. Forward various kin|0.37|6.42|3001001|edu packedu pack #2|1|athletic|4|Shoes|47|ationese|petite|699088021479peach561|wheat|Pound|Unknown|13|ationn stantically| +6598|AAAAAAAAGMJBAAAA|1997-10-27|1999-10-27|Electronic, protective ties cannot install temporarily opportunities. Likely experiments see so implicit patie|1.08|0.34|7007009|brandbrand #9|7|decor|7|Home|854|eseantieing|N/A|617steel621936286481|pale|Tsp|Unknown|58|eingn stantically| +6599|AAAAAAAAGMJBAAAA|1999-10-28|2001-10-26|Issue|9.33|6.06|1002002|importoamalg #2|2|fragrances|1|Women|854|eseantieing|medium|3679pale595732138289|turquoise|Lb|Unknown|99|n stn stantically| +6600|AAAAAAAAGMJBAAAA|2001-10-27||Eno|2.84|1.87|3002001|importoexporti #1|2|infants|3|Children|854|eseantieing|petite|3679pale595732138289|azure|Lb|Unknown|95|barbarcallycally| +6601|AAAAAAAAJMJBAAAA|1997-10-27||Fina|3.14|1.35|5004002|edu packscholar #2|4|classical|5|Music|228|eingableable|N/A|25sienna177945265477|purple|Gross|Unknown|18|oughtbarcallycally| +6602|AAAAAAAAKMJBAAAA|1997-10-27|2000-10-26|Now large measures should not remember still on a parents; different, other children prevent again on a policie|0.80|0.39|1001001|amalgamalg #1|1|dresses|1|Women|57|ationanti|large|499819417goldenrod97|indian|Lb|Unknown|33|ablebarcallycally| +6603|AAAAAAAAKMJBAAAA|2000-10-27||Deaf te|5.48|1.86|3004002|edu packexporti #2|1|school-uniforms|3|Children|159|ationanti|economy|507honeydew024769530|lawn|N/A|Unknown|23|pribarcallycally| +6604|AAAAAAAAMMJBAAAA|1997-10-27|1999-10-27|Actual, japanese successes ought to put. Studies shall make out of a observers. Public, dangerous ideas must stop blue, soft men. Shy, relevant pounds feel surprisingly old criteria; interested yea|2.89|2.51|8011005|amalgmaxi #5|11|archery|8|Sports|684|eseeingcally|N/A|332947483medium00561|turquoise|Tbl|Unknown|24|esebarcallycally| +6605|AAAAAAAAMMJBAAAA|1999-10-28|2001-10-26|Actual, japanese successes ought to put. Studies shall make out of a observers. Public, dangerous ideas must stop blue, soft men. Shy, relevant pounds feel surprisingly old criteria; interested yea|7.80|6.94|6010006|univbrand #6|11|jewelry boxes|6|Jewelry|684|eseeingcally|N/A|038699369416white368|thistle|Box|Unknown|99|antibarcallycally| +6606|AAAAAAAAMMJBAAAA|2001-10-27||Actual, japanese successes ought to put. Studies shall make out of a observers. Public, dangerous ideas must stop blue, soft men. Shy, relevant pounds feel surprisingly old criteria; interested yea|4.33|6.94|1004001|edu packamalg #1|4|swimwear|1|Women|153|priantiought|medium|7536675832963mint415|tan|Dram|Unknown|60|callybarcallycally| +6607|AAAAAAAAPMJBAAAA|1997-10-27||Immediately smart studies confront different, scottish walls. Never other refugees will know as; types could turn only economic, goo|5.04|3.17|2001002|amalgimporto #2|1|accessories|2|Men|261|oughtcallyable|small|07071670155papaya378|thistle|Ounce|Unknown|30|ationbarcallycally| +6608|AAAAAAAAANJBAAAA|1997-10-27|2000-10-26|New writers understand final restaura|4.74|3.22|7002009|importobrand #9|2|bedding|7|Home|125|antiableought|N/A|6red2825657185268205|sandy|Oz|Unknown|26|eingbarcallycally| +6609|AAAAAAAAANJBAAAA|2000-10-27||Extra reports us|4.31|2.49|5003002|exportischolar #2|2|pop|5|Music|475|antiableought|N/A|8063steel53623208522|smoke|Dram|Unknown|15|n stbarcallycally| +6610|AAAAAAAACNJBAAAA|1997-10-27|1999-10-27|Resources contain |4.30|2.19|10006015|corpunivamalg #15|6|musical|10|Electronics|170|barationought|N/A|71042blanched6888841|turquoise|Pound|Unknown|27|baroughtcallycally| +6611|AAAAAAAACNJBAAAA|1999-10-28|2001-10-26|Local, deep thoughts accept finally only attractive regulations. Single residents check final things. Most forward individuals feel theoretical, average sets. Onc|1.51|0.90|10006015|scholarunivamalg #12|15|fiction|9|Books|170|barationought|N/A|chiffon3063139796386|sandy|Ton|Unknown|13|oughtoughtcallycally| +6612|AAAAAAAACNJBAAAA|2001-10-27||Steady, prime issues date lonely, good friends. Red words used to demonstrate slightly areas. Labour pupils ought to become even purposes. Evolutionar|2.97|2.43|1003001|exportiamalg #1|3|maternity|1|Women|170|barationought|large|63109240saddle305918|pale|Lb|Unknown|41|ableoughtcallycally| +6613|AAAAAAAAFNJBAAAA|1997-10-27||Women take ironically things. Pounds|8.63|5.52|6011006|amalgbrand #6|11|semi-precious|6|Jewelry|224|eseableable|N/A|1959slate10596779501|smoke|Each|Unknown|28|prioughtcallycally| +6614|AAAAAAAAGNJBAAAA|1997-10-27|2000-10-26|Economic meanings |7.78|4.97|10010002|univamalgamalg #2|10|memory|10|Electronics|82|ableeing|N/A|2187lawn438299036228|wheat|Tbl|Unknown|23|eseoughtcallycally| +6615|AAAAAAAAGNJBAAAA|2000-10-27||Very whole students cut fully dogs. Major, new hands could suit. Advisers may lead. Then controversial policies help probably current, overseas obligations. Above serious banks ca|3.71|4.97|5001002|amalgscholar #2|1|rock|5|Music|398|eingn stpri|N/A|2187lawn438299036228|magenta|Gram|Unknown|58|antioughtcallycally| +6616|AAAAAAAAINJBAAAA|1997-10-27|1999-10-27|Young, working horses see mentally |1.27|0.50|9007005|brandmaxi #5|7|reference|9|Books|275|antiationable|N/A|turquoise59746654703|papaya|Ton|Unknown|47|callyoughtcallycally| +6617|AAAAAAAAINJBAAAA|1999-10-28|2001-10-26|Young, working horses see mentally |6.94|4.02|6012002|importobrand #2|7|costume|6|Jewelry|275|antiationable|N/A|turquoise59746654703|orange|Bunch|Unknown|12|ationoughtcallycally| +6618|AAAAAAAAINJBAAAA|2001-10-27||Young, working horses see mentally |12.34|4.02|6012002|importoscholar #1|2|country|5|Music|275|antiationable|N/A|turquoise59746654703|wheat|Oz|Unknown|62|eingoughtcallycally| +6619|AAAAAAAALNJBAAAA|1997-10-27||Fair stations win again substantial|9.22|7.92|3001002|amalgexporti #2|1|newborn|3|Children|445|antieseese|large|98490678lace96687175|pink|Pound|Unknown|67|n stoughtcallycally| +6620|AAAAAAAAMNJBAAAA|1997-10-27|2000-10-26|Universities obey moments. Extraordinary, actual scots ought to give english materials; yet private abilities need so new developments. Radically|3.66|2.23|9014009|edu packunivamalg #9|14|sports|9|Books|296|callyn stable|N/A|5pale614437036947473|smoke|Each|Unknown|24|barablecallycally| +6621|AAAAAAAAMNJBAAAA|2000-10-27||As average countries say again others. Figures shall eat never intensive lines. Grounds enter recently future parts; then disciplinary problems shall exert also gifts. Details smile often; likely|2.06|2.23|9014009|importoscholar #2|2|country|5|Music|296|callyn stable|N/A|5pale614437036947473|tomato|N/A|Unknown|15|oughtablecallycally| +6622|AAAAAAAAONJBAAAA|1997-10-27|1999-10-27|Past appropriate children become even overall, political men. Major, special papers accept only socia|2.28|1.71|5004001|edu packscholar #1|4|classical|5|Music|282|ableeingable|N/A|434blanched779044632|light|Cup|Unknown|20|ableablecallycally| +6623|AAAAAAAAONJBAAAA|1999-10-28|2001-10-26|Past appropriate children become even overall, political men. Major, special papers accept only socia|8.57|5.65|5004001|edu packmaxi #4|14|tennis|8|Sports|135|antipriought|N/A|434blanched779044632|lime|Lb|Unknown|49|priablecallycally| +6624|AAAAAAAAONJBAAAA|2001-10-27||Even critical police shall rally with a things. Black, wild conseq|5.00|2.75|5004001|brandmaxi #1|7|reference|9|Books|168|antipriought|N/A|7328625smoke43409747|misty|Pallet|Unknown|64|eseablecallycally| +6625|AAAAAAAABOJBAAAA|1997-10-27||Strange, social rooms point alternatively in an tracks. Elegantly russian vehicles can tell; long ministers should want now mou|30.29|10.29|8010006|univmaxi #6|10|pools|8|Sports|298|eingn stable|N/A|4851450263plum500036|smoke|Pallet|Unknown|14|antiablecallycally| +6626|AAAAAAAACOJBAAAA|1997-10-27|2000-10-26|Months shall touch too regular arms. Representatives will not fly silly wages; single, proper circumstances used to measure political, small tools. As isolated stones might mean e|8.67|2.60|5003001|exportischolar #1|3|pop|5|Music|246|callyeseable|N/A|832108503goldenrod56|salmon|Cup|Unknown|34|callyablecallycally| +6627|AAAAAAAACOJBAAAA|2000-10-27||Months shall touch too regular arms. Representatives will not fly silly wages; single, proper circumstances used to measure political, small tools. As isolated stones might mean e|9.01|2.70|7008008|namelessbrand #8|8|lighting|7|Home|246|callyeseable|N/A|832108503goldenrod56|antique|Tbl|Unknown|95|ationablecallycally| +6628|AAAAAAAAEOJBAAAA|1997-10-27|1999-10-27|Local, french intervals should care over terrible buildi|6.03|3.55|4001001|amalgedu pack #1|1|womens|4|Shoes|154|eseantiought|large|805351saddle43709546|lavender|Ton|Unknown|28|eingablecallycally| +6629|AAAAAAAAEOJBAAAA|1999-10-28|2001-10-26|Political, dependent boys visit most preferences. Shares work as in a rebels. Outstanding, scot|3.73|3.55|3003002|exportiexporti #2|1|toddlers|3|Children|77|eseantiought|medium|805351saddle43709546|powder|N/A|Unknown|42|n stablecallycally| +6630|AAAAAAAAEOJBAAAA|2001-10-27||Political, dependent boys visit most preferences. Shares work as in a rebels. Outstanding, scot|4.66|2.09|3003002|edu packamalg #1|4|swimwear|1|Women|183|eseantiought|medium|805351saddle43709546|beige|Case|Unknown|3|barpricallycally| +6631|AAAAAAAAHOJBAAAA|1997-10-27||W|3.29|2.20|7013002|exportinameless #2|13|wallpaper|7|Home|431|oughtpriese|N/A|266572wheat212017344|yellow|Ton|Unknown|10|oughtpricallycally| +6632|AAAAAAAAIOJBAAAA|1997-10-27|2000-10-26|Good, closed languages include b|6.42|4.75|9013009|exportiunivamalg #9|13|self-help|9|Books|406|callybarese|N/A|382588472209pale0324|khaki|Bundle|Unknown|1|ablepricallycally| +6633|AAAAAAAAIOJBAAAA|2000-10-27||Good, closed languages include b|1.25|4.75|9013009|amalgmaxi #4|13|archery|8|Sports|406|callybarese|N/A|382588472209pale0324|indian|Tbl|Unknown|17|pripricallycally| +6634|AAAAAAAAKOJBAAAA|1997-10-27|1999-10-27|Previous, general schools move both future, official authorities. Still young windows used to help too international actual views. Gentlemen promote much clearly beautiful organisms; mile|5.50|2.36|7012005|importonameless #5|12|paint|7|Home|494|esen stese|N/A|6140421102211tomato1|smoke|Tsp|Unknown|64|esepricallycally| +6635|AAAAAAAAKOJBAAAA|1999-10-28|2001-10-26|Previous, general schools move both future, official authorities. Still young windows used to help too international actual views. Gentlemen promote much clearly beautiful organisms; mile|6.15|2.36|7007006|brandbrand #6|12|decor|7|Home|494|esen stese|N/A|8444090gainsboro3390|sandy|Gram|Unknown|7|antipricallycally| +6636|AAAAAAAAKOJBAAAA|2001-10-27||Previous, general schools move both future, official authorities. Still young windows used to help too international actual views. Gentlemen promote much clearly beautiful organisms; mile|1.90|2.36|7007006|exportiedu pack #1|3|kids|4|Shoes|494|esen stese|large|blush860091654352637|honeydew|Unknown|Unknown|50|callypricallycally| +6637|AAAAAAAANOJBAAAA|1997-10-27||Growing, small aims might begin |2.75|1.15|9003002|exportimaxi #2|3|computers|9|Books|234|esepriable|N/A|5758blush72181488368|royal|Unknown|Unknown|6|ationpricallycally| +6638|AAAAAAAAOOJBAAAA|1997-10-27|2000-10-26|Long, soft sports dig again true services. Obvious|6.06|1.99|1002001|importoamalg #1|2|fragrances|1|Women|8|eing|petite|8069201608red9509654|tomato|Pallet|Unknown|9|eingpricallycally| +6639|AAAAAAAAOOJBAAAA|2000-10-27||Rights will donate additionally. Else other cattle must produce flat years. Only large funds ought to get. Key, whole quarters oug|97.40|85.71|1002001|scholarnameless #10|2|tables|7|Home|285|eing|N/A|10021sienna347218615|sienna|Pound|Unknown|47|n stpricallycally| +6640|AAAAAAAAAPJBAAAA|1997-10-27|1999-10-27|Nearly good centres could give brightly strange years. Southern, gener|6.27|2.25|5002001|importoscholar #1|2|country|5|Music|59|n stanti|N/A|186550468868thistle1|lawn|Gram|Unknown|15|baresecallycally| +6641|AAAAAAAAAPJBAAAA|1999-10-28|2001-10-26|Grounds watch linear problems. More easy differences will take indeed complete sources; also new dema|1.10|2.25|7016002|corpnameless #2|16|furniture|7|Home|175|n stanti|N/A|186550468868thistle1|wheat|Unknown|Unknown|18|oughtesecallycally| +6642|AAAAAAAAAPJBAAAA|2001-10-27||Grounds watch linear problems. More easy differences will take indeed complete sources; also new dema|5.49|2.25|8014003|edu packmaxi #3|14|tennis|8|Sports|684|n stanti|N/A|90454257775rosy32300|grey|Gram|Unknown|41|ableesecallycally| +6643|AAAAAAAADPJBAAAA|1997-10-27||Human years improve broadly poli|3.93|3.02|9016008|corpunivamalg #8|16|mystery|9|Books|134|esepriought|N/A|steel669167181780896|papaya|Pallet|Unknown|3|priesecallycally| +6644|AAAAAAAAEPJBAAAA|1997-10-27|2000-10-26|Most difficult documents know so legal colleagues. Premier, hard rights ought to use male, other photographs. Warm expenses could not think such as a cases; sophisticated, long sites ought to fe|9.52|7.23|2004001|edu packimporto #1|4|sports-apparel|2|Men|444|eseeseese|small|4190peru031162795148|olive|Cup|Unknown|32|eseesecallycally| +6645|AAAAAAAAEPJBAAAA|2000-10-27||Most difficult documents know so legal colleagues. Premier, hard rights ought to use male, other photographs. Warm expenses could not think such as a cases; sophisticated, long sites ought to fe|1.18|7.23|10011016|amalgamalgamalg #16|4|disk drives|10|Electronics|83|eseeseese|N/A|61yellow215440915732|tomato|Oz|Unknown|65|antiesecallycally| +6646|AAAAAAAAGPJBAAAA|1997-10-27|1999-10-27|Friends would marry. Red, overseas activities meet interests; immediate schools must not look families. More criminal prod|4.82|1.87|2001001|amalgimporto #1|1|accessories|2|Men|822|ableableeing|N/A|859013515507olive753|ivory|Gross|Unknown|27|callyesecallycally| +6647|AAAAAAAAGPJBAAAA|1999-10-28|2001-10-26|Friends would marry. Red, overseas activities meet interests; immediate schools must not look families. More criminal prod|6.66|5.12|9008012|namelessmaxi #12|8|romance|9|Books|822|ableableeing|N/A|4927771gainsboro4472|smoke|Carton|Unknown|30|ationesecallycally| +6648|AAAAAAAAGPJBAAAA|2001-10-27||Friends would marry. Red, overseas activities meet interests; immediate schools must not look families. More criminal prod|9.60|5.12|9008012|exportimaxi #1|8|computers|9|Books|822|ableableeing|N/A|4927771gainsboro4472|navy|Tsp|Unknown|98|eingesecallycally| +6649|AAAAAAAAJPJBAAAA|1997-10-27||Accurate institutions shall avoid also relative, broken cases. Effective, special citizens could answer there in a parties. Fre|9.59|3.54|7012010|importonameless #10|12|paint|7|Home|58|einganti|N/A|764893sky37204102451|wheat|N/A|Unknown|54|n stesecallycally| +6650|AAAAAAAAKPJBAAAA|1997-10-27|2000-10-26|Jobs must not let really anyway good systems. Partially ho|6.80|2.92|5002001|importoscholar #1|2|country|5|Music|541|oughteseanti|N/A|808017wheat161409081|orange|Cup|Unknown|54|baranticallycally| +6651|AAAAAAAAKPJBAAAA|2000-10-27||About musical years survive sort of; often consid|2.81|2.92|7013002|exportinameless #2|13|wallpaper|7|Home|229|n stableable|N/A|808017wheat161409081|slate|Ton|Unknown|63|oughtanticallycally| +6652|AAAAAAAAMPJBAAAA|1997-10-27|1999-10-27|Present, dangerous courts might send |1.93|0.61|7001003|amalgbrand #3|1|bathroom|7|Home|870|barationeing|N/A|56894snow81597171354|olive|Box|Unknown|68|ableanticallycally| +6653|AAAAAAAAMPJBAAAA|1999-10-28|2001-10-26|Present, dangerous courts might send |0.48|0.24|7001003|univamalgamalg #7|10|memory|10|Electronics|870|barationeing|N/A|8turquoise0520820933|violet|Cup|Unknown|61|prianticallycally| +6654|AAAAAAAAMPJBAAAA|2001-10-27||Present, dangerous courts might send |1.86|0.98|7001003|exportimaxi #7|10|computers|9|Books|275|barationeing|N/A|8turquoise0520820933|powder|Ounce|Unknown|12|eseanticallycally| +6655|AAAAAAAAPPJBAAAA|1997-10-27||Great numbers could not save with a children. So-called, good times use later for a sounds; never common years murder for example economic images. Inhabit|3.09|1.91|3002002|importoexporti #2|2|infants|3|Children|136|callypriought|extra large|45chartreuse96897242|sky|N/A|Unknown|44|antianticallycally| +6656|AAAAAAAAAAKBAAAA|1997-10-27|2000-10-26|Voluntary, single speakers ought to see often final, other problems. Economic, new acc|55.36|16.60|2004001|edu packimporto #1|4|sports-apparel|2|Men|809|n stbareing|extra large|88lemon2354500614572|blue|N/A|Unknown|47|callyanticallycally| +6657|AAAAAAAAAAKBAAAA|2000-10-27||Presents deter; no longer lively others realize just roman |1.17|0.93|2004001|amalgamalg #2|1|dresses|1|Women|546|n stbareing|extra large|88lemon2354500614572|thistle|Oz|Unknown|49|ationanticallycally| +6658|AAAAAAAACAKBAAAA|1997-10-27|1999-10-27|Strange friends ought to like in a holidays. Large cases offer consequently tropical, lost rights. National sides choose forward. High citizens enjoy strong, white experim|0.87|0.35|3002001|importoexporti #1|2|infants|3|Children|317|ationoughtpri|medium|259714028slate241186|violet|Tsp|Unknown|2|einganticallycally| +6659|AAAAAAAACAKBAAAA|1999-10-28|2001-10-26|Strange friends ought to like in a holidays. Large cases offer consequently tropical, lost rights. National sides choose forward. High citizens enjoy strong, white experim|8.04|3.69|2004002|edu packimporto #2|4|sports-apparel|2|Men|308|ationoughtpri|petite|259714028slate241186|slate|Oz|Unknown|24|n stanticallycally| +6660|AAAAAAAACAKBAAAA|2001-10-27||Strange friends ought to like in a holidays. Large cases offer consequently tropical, lost rights. National sides choose forward. High citizens enjoy strong, white experim|35.73|3.69|6007005|brandcorp #5|4|pendants|6|Jewelry|308|ationoughtpri|N/A|427010turquoise08757|royal|Lb|Unknown|2|barcallycallycally| +6661|AAAAAAAAFAKBAAAA|1997-10-27||Democratic hours initiate often; meanwhile prime years might move also dreadful, other cl|1.13|0.45|8005002|scholarnameless #2|5|fitness|8|Sports|83|prieing|N/A|9steel39385584518116|salmon|Gram|Unknown|12|oughtcallycallycally| +6662|AAAAAAAAGAKBAAAA|1997-10-27|2000-10-26|Small babies must get. Women drive individuals|8.65|6.92|9008003|namelessmaxi #3|8|romance|9|Books|76|callyation|N/A|13peach4286029340182|steel|Bunch|Unknown|61|ablecallycallycally| +6663|AAAAAAAAGAKBAAAA|2000-10-27||Too inadequate panels will achieve very too similar applications. Best big offices say also orange moments. Conditions might exercise suddenly similar children; soon due boys will make from a r|6.45|4.77|9008003|edu packexporti #2|8|school-uniforms|3|Children|197|ationn stought|extra large|13peach4286029340182|gainsboro|Bunch|Unknown|42|pricallycallycally| +6664|AAAAAAAAIAKBAAAA|1997-10-27|1999-10-27|Families avoid indian hills. Lists bring exactly pregnant, free managers. Social, overall bones may prolong again ancient, whole days. Therefore alive years provide then unfair cour|9.41|5.83|8004005|edu packnameless #5|4|camping|8|Sports|365|anticallypri|N/A|9sienna8821837218552|royal|Ton|Unknown|54|esecallycallycally| +6665|AAAAAAAAIAKBAAAA|1999-10-28|2001-10-26|Families avoid indian hills. Lists bring exactly pregnant, free managers. Social, overall bones may prolong again ancient, whole days. Therefore alive years provide then unfair cour|7.11|5.83|5003002|exportischolar #2|3|pop|5|Music|365|anticallypri|N/A|152716269metallic811|sandy|Lb|Unknown|25|anticallycallycally| +6666|AAAAAAAAIAKBAAAA|2001-10-27||Families avoid indian hills. Lists bring exactly pregnant, free managers. Social, overall bones may prolong again ancient, whole days. Therefore alive years provide then unfair cour|1.11|0.46|5003002|maxiunivamalg #3|3|televisions|10|Electronics|365|anticallypri|N/A|152716269metallic811|metallic|Tsp|Unknown|41|callycallycallycally| +6667|AAAAAAAALAKBAAAA|1997-10-27||Only aware standards belong fully social, heavy aims. No doubt great dif|3.02|1.38|5002002|importoscholar #2|2|country|5|Music|22|ableable|N/A|orange29360547736338|purple|Tbl|Unknown|20|ationcallycallycally| +6668|AAAAAAAAMAKBAAAA|1997-10-27|2000-10-26|Really ita|6.36|1.90|2001001|amalgimporto #1|1|accessories|2|Men|477|ationationese|large|0780511329947red9014|wheat|Gross|Unknown|23|eingcallycallycally| +6669|AAAAAAAAMAKBAAAA|2000-10-27||More new legs ought to insist. Al|89.91|38.66|6012006|importobrand #6|12|costume|6|Jewelry|334|ationationese|N/A|0154751orange2183935|saddle|Ounce|Unknown|13|n stcallycallycally| +6670|AAAAAAAAOAKBAAAA|1997-10-27|1999-10-27|Temperatures might not know by far; rocks should li|6.97|2.92|5001001|amalgscholar #1|1|rock|5|Music|252|ableantiable|N/A|03050639690708red288|wheat|Box|Unknown|25|barationcallycally| +6671|AAAAAAAAOAKBAAAA|1999-10-28|2001-10-26|Ugly, long men should keep by a rules. Particular|7.00|5.60|5001001|edu packamalg #2|4|swimwear|1|Women|252|ableantiable|N/A|03050639690708red288|yellow|Cup|Unknown|15|oughtationcallycally| +6672|AAAAAAAAOAKBAAAA|2001-10-27||Ugly, long men should keep by a rules. Particular|6.34|2.47|5001001|amalgmaxi #1|4|arts|9|Books|350|barantipri|N/A|272drab4514818732573|yellow|Oz|Unknown|33|ableationcallycally| +6673|AAAAAAAABBKBAAAA|1997-10-27||Children reduce usually in particular different sports. White, initial effects scrap however fur|4.46|2.09|6008002|namelesscorp #2|8|mens watch|6|Jewelry|328|eingablepri|N/A|610803brown446738533|metallic|Each|Unknown|7|priationcallycally| +6674|AAAAAAAACBKBAAAA|1997-10-27|2000-10-26|For certain excellent points take neighbo|12.26|9.31|6007003|brandcorp #3|7|pendants|6|Jewelry|505|antibaranti|N/A|84859764007medium759|saddle|Oz|Unknown|32|eseationcallycally| +6675|AAAAAAAACBKBAAAA|2000-10-27||About other rights weaken opening, likely representations. Obviously sp|3.00|9.31|2001002|amalgimporto #2|1|accessories|2|Men|505|antibaranti|medium|84859764007medium759|peru|Tsp|Unknown|13|antiationcallycally| +6676|AAAAAAAAEBKBAAAA|1997-10-27|1999-10-27|Then other fees must grow sometimes. Metals cannot dare fair letters. Large, tired sentences attack main, different dimensions. Years can wonder now small opportunities|4.66|2.70|3001001|amalgexporti #1|1|newborn|3|Children|662|ablecallycally|small|450ivory221640190105|smoke|Unknown|Unknown|41|callyationcallycally| +6677|AAAAAAAAEBKBAAAA|1999-10-28|2001-10-26|Then other fees must grow sometimes. Metals cannot dare fair letters. Large, tired sentences attack main, different dimensions. Years can wonder now small opportunities|7.12|2.70|7008008|namelessbrand #8|1|lighting|7|Home|513|prioughtanti|N/A|450ivory221640190105|steel|Ton|Unknown|24|ationationcallycally| +6678|AAAAAAAAEBKBAAAA|2001-10-27||Then other fees must grow sometimes. Metals cannot dare fair letters. Large, tired sentences attack main, different dimensions. Years can wonder now small opportunities|1.64|0.49|7008008|importoamalg #1|1|fragrances|1|Women|513|prioughtanti|extra large|450ivory221640190105|papaya|Gram|Unknown|31|eingationcallycally| +6679|AAAAAAAAHBKBAAAA|1997-10-27||Communities might not seize services. Personal, wooden millions disrupt over hardly early countries. Ancient, common qualifications can asses|1.96|1.64|5004002|edu packscholar #2|4|classical|5|Music|18|eingought|N/A|9237047562ivory09277|snow|Gross|Unknown|62|n stationcallycally| +6680|AAAAAAAAIBKBAAAA|1997-10-27|2000-10-26|Free women ought to contact new committees. Powerful laws must like then ideal hopes; human, separate allegations will provide educational years. Tiles could try maybe holy, free figures. |5.54|2.27|5004001|edu packscholar #1|4|classical|5|Music|553|priantianti|N/A|3224hot6022081622305|slate|Bundle|Unknown|30|bareingcallycally| +6681|AAAAAAAAIBKBAAAA|2000-10-27||Local years might enter recently even similar efforts. Key men open merely russian bodies. Circumstances lov|56.57|34.50|4001002|amalgedu pack #2|1|womens|4|Shoes|249|n steseable|extra large|59seashell1971346767|gainsboro|Lb|Unknown|13|oughteingcallycally| +6682|AAAAAAAAKBKBAAAA|1997-10-27|1999-10-27|Paintings cut institutions. Different, tiny ch|0.20|0.14|10001002|amalgunivamalg #2|1|cameras|10|Electronics|185|antieingought|N/A|seashell548727331973|tomato|Case|Unknown|13|ableeingcallycally| +6683|AAAAAAAAKBKBAAAA|1999-10-28|2001-10-26|Paintings cut institutions. Different, tiny ch|2.39|1.52|10001002|importoscholar #2|1|country|5|Music|185|antieingought|N/A|seashell548727331973|plum|Case|Unknown|39|prieingcallycally| +6684|AAAAAAAAKBKBAAAA|2001-10-27||Valuable cells might not continue internal, other years. Reserves become thin, other patients|9.79|4.40|5003001|exportischolar #1|1|pop|5|Music|301|antieingought|N/A|seashell548727331973|thistle|Lb|Unknown|9|eseeingcallycally| +6685|AAAAAAAANBKBAAAA|1997-10-27||Gold, real be|0.28|0.24|2004002|edu packimporto #2|4|sports-apparel|2|Men|483|prieingese|large|502sienna96755648648|rosy|Pound|Unknown|34|antieingcallycally| +6686|AAAAAAAAOBKBAAAA|1997-10-27|2000-10-26|Fellow, great costs may see elderly, similar months. National, public operations ignore finally. Regulations may return badly close, sophisticated schools. Northern materials |0.37|0.26|8003007|exportinameless #7|3|basketball|8|Sports|25|antiable|N/A|1461362royal74557559|tan|Oz|Unknown|17|callyeingcallycally| +6687|AAAAAAAAOBKBAAAA|2000-10-27||Italian, complete eyes must want clearly in a rights. Light services become please. Policies may go artists. Possible awards think now revo|69.87|21.65|9003004|exportimaxi #4|3|computers|9|Books|25|antiable|N/A|1461362royal74557559|puff|Bunch|Unknown|98|ationeingcallycally| +6688|AAAAAAAAACKBAAAA|1997-10-27|1999-10-27|Clinical, inc initiatives make specially according to a activities.|6.92|5.53|9001005|amalgmaxi #5|1|arts|9|Books|201|oughtbarable|N/A|45339puff36928410229|plum|Dram|Unknown|21|eingeingcallycally| +6689|AAAAAAAAACKBAAAA|1999-10-28|2001-10-26|Clinical, inc initiatives make specially according to a activities.|0.84|5.53|1004002|edu packamalg #2|1|swimwear|1|Women|84|eseeing|extra large|1629pink412947224706|goldenrod|Cup|Unknown|8|n steingcallycally| +6690|AAAAAAAAACKBAAAA|2001-10-27||Clinical, inc initiatives make specially according to a activities.|7.69|6.15|6003003|exporticorp #3|3|gold|6|Jewelry|84|eseeing|N/A|77lavender1318920569|turquoise|Box|Unknown|20|barn stcallycally| +6691|AAAAAAAADCKBAAAA|1997-10-27||Expressions record medical, british humans; early other opportunities confess just. Military us|1.99|0.67|6008004|namelesscorp #4|8|mens watch|6|Jewelry|272|ableationable|N/A|320thistle1882559300|pale|Carton|Unknown|15|oughtn stcallycally| +6692|AAAAAAAAECKBAAAA|1997-10-27|2000-10-26|Likely, complex users help very as immediate ways. Available farmers feel afterwards sometimes tiny times. |2.72|1.52|1002001|importoamalg #1|2|fragrances|1|Women|394|esen stpri|large|5peach10741552721227|lawn|Lb|Unknown|29|ablen stcallycally| +6693|AAAAAAAAECKBAAAA|2000-10-27||So detailed others should command please. Other, poor types will not provide apparently temporary, western poems; fun professionals might go however despite the pupils. Consultants expect|2.60|2.10|5002002|importoscholar #2|2|country|5|Music|7|ation|N/A|5peach10741552721227|white|Lb|Unknown|6|prin stcallycally| +6694|AAAAAAAAGCKBAAAA|1997-10-27|1999-10-27|Reluctant, public priests shall prepare in a years; applications prejudice just unlike|1.89|0.98|3002001|importoexporti #1|2|infants|3|Children|98|eingn st|petite|9081036916rosy789904|slate|Cup|Unknown|88|esen stcallycally| +6695|AAAAAAAAGCKBAAAA|1999-10-28|2001-10-26|Reluctant, public priests shall prepare in a years; applications prejudice just unlike|4.02|0.98|6006008|corpcorp #8|2|rings|6|Jewelry|106|eingn st|N/A|623876883beige864914|sandy|N/A|Unknown|61|antin stcallycally| +6696|AAAAAAAAGCKBAAAA|2001-10-27||Reluctant, public priests shall prepare in a years; applications prejudice just unlike|84.03|0.98|7012007|importonameless #7|12|paint|7|Home|67|eingn st|N/A|46677spring395101872|metallic|Gross|Unknown|55|callyn stcallycally| +6697|AAAAAAAAJCKBAAAA|1997-10-27||Well traditional governments want always in a points. Children sing then subseque|0.13|0.04|7013008|exportinameless #8|13|wallpaper|7|Home|2|able|N/A|8lemon60020506275238|salmon|Bundle|Unknown|36|ationn stcallycally| +6698|AAAAAAAAKCKBAAAA|1997-10-27|2000-10-26|Still dangerous bacteria could not match forms; warm days must enable recent, certain disciplines. Events should not take there. Subsequent, external views represent pretty ne|4.99|4.04|3003001|exportiexporti #1|3|toddlers|3|Children|268|eingcallyable|large|334193737352puff0076|ivory|Dram|Unknown|25|eingn stcallycally| +6699|AAAAAAAAKCKBAAAA|2000-10-27||Other proposa|7.20|4.04|3003001|namelesscorp #4|8|mens watch|6|Jewelry|229|eingcallyable|N/A|077631889yellow97213|turquoise|N/A|Unknown|76|n stn stcallycally| +6700|AAAAAAAAMCKBAAAA|1997-10-27|1999-10-27|Able hands help however inevitable policemen. Far true matters will not forgive never hands. Absolutely french events stop now. Well able procedures unde|0.14|0.04|1003001|exportiamalg #1|3|maternity|1|Women|220|barableable|medium|416299230goldenrod35|sienna|Tsp|Unknown|39|barbarationcally| +6701|AAAAAAAAMCKBAAAA|1999-10-28|2001-10-26|So old-fashioned places may arrange for no individuals. Volunteers exercise of course. Secondary, serious homes revive comparative councils. Very, european numbers m|2.93|0.04|1003001|amalgnameless #2|3|athletic shoes|8|Sports|220|barableable|N/A|416299230goldenrod35|seashell|Cup|Unknown|36|oughtbarationcally| +6702|AAAAAAAAMCKBAAAA|2001-10-27||Also following times can strengthen already weeks. About firm arrangements can thank also highly honest benefi|1.65|1.28|1003001|brandcorp #7|3|pendants|6|Jewelry|220|barableable|N/A|32521puff47914394299|peru|Each|Unknown|34|ablebarationcally| +6703|AAAAAAAAPCKBAAAA|1997-10-27||Possible months could record aspects. Royal, possible values start however other services. Very private things pr|4.47|2.50|6011008|amalgbrand #8|11|semi-precious|6|Jewelry|294|esen stable|N/A|64ghost9042866321996|wheat|Gram|Unknown|42|pribarationcally| +6704|AAAAAAAAADKBAAAA|1997-10-27|2000-10-26|Things used to help together more large aims. Democrats tell b|1.37|0.57|2003001|exportiimporto #1|3|pants|2|Men|116|callyoughtought|large|4170goldenrod4832172|purple|Cup|Unknown|56|esebarationcally| +6705|AAAAAAAAADKBAAAA|2000-10-27||Things used to help together more large aims. Democrats tell b|5.95|5.05|2003001|corpnameless #6|16|furniture|7|Home|160|barcallyought|N/A|4170goldenrod4832172|purple|Bundle|Unknown|38|antibarationcally| +6706|AAAAAAAACDKBAAAA|1997-10-27|1999-10-27|Unique, commercial discussions mark then social, top states; organizations will not hit never still traditional programmes. Social, afraid papers ought to meet english egg|2.98|1.72|9011011|amalgunivamalg #11|11|cooking|9|Books|79|n station|N/A|28puff40943712263312|pale|Oz|Unknown|19|callybarationcally| +6707|AAAAAAAACDKBAAAA|1999-10-28|2001-10-26|Unique, commercial discussions mark then social, top states; organizations will not hit never still traditional programmes. Social, afraid papers ought to meet english egg|5.88|4.23|6015004|scholarbrand #4|11|custom|6|Jewelry|79|n station|N/A|371spring89428844812|peach|Bunch|Unknown|23|ationbarationcally| +6708|AAAAAAAACDKBAAAA|2001-10-27||Clearly continuous cases overtake commonly areas. Protests might join purely in a trials. Always concerned ambitions accept british, real feet. Dependent days resist fin|4.71|4.23|6005005|scholarcorp #5|5|earings|6|Jewelry|74|n station|N/A|6860199004800grey703|tan|Tbl|Unknown|4|eingbarationcally| +6709|AAAAAAAAFDKBAAAA|1997-10-27||Clear, other proposals m|8.98|5.47|10003012|exportiunivamalg #12|3|dvd/vcr players|10|Electronics|1000|barbarbarought|N/A|3467669sandy11701271|hot|Tsp|Unknown|12|n stbarationcally| +6710|AAAAAAAAGDKBAAAA|1997-10-27|2000-10-26|However pleasant years should imitate as impossible, new districts. Urgent, major residen|8.51|4.59|7007001|brandbrand #1|7|decor|7|Home|133|pripriought|N/A|6545912912962lemon53|royal|Pound|Unknown|66|baroughtationcally| +6711|AAAAAAAAGDKBAAAA|2000-10-27||Important, french names see then in the ladies. Disciplines put. Big cups could strip mostly other meetings. Long times go necessary democrats. Worthy, middle gifts shall make only popular hab|9.28|3.15|1002002|importoamalg #2|2|fragrances|1|Women|133|pripriought|economy|6545912912962lemon53|turquoise|Gross|Unknown|25|oughtoughtationcally| +6712|AAAAAAAAIDKBAAAA|1997-10-27|1999-10-27|Strong, new girls might clear in the police. Schools would help there and so on english stations. Advisory, low friends cannot motivate jointly. Civil groups would not step |1.97|0.70|2003001|exportiimporto #1|3|pants|2|Men|284|eseeingable|large|0peach53244726986133|metallic|Dram|Unknown|78|ableoughtationcally| +6713|AAAAAAAAIDKBAAAA|1999-10-28|2001-10-26|Functions think long rigid boxes. Accurate, certain minutes calm. Equal backs expect regularly. Local, apparent fingers would leave with the hours. Managers en|1.53|0.70|1002002|importoamalg #2|3|fragrances|1|Women|83|prieing|extra large|0peach53244726986133|peru|Dram|Unknown|18|prioughtationcally| +6714|AAAAAAAAIDKBAAAA|2001-10-27||Functions think long rigid boxes. Accurate, certain minutes calm. Equal backs expect regularly. Local, apparent fingers would leave with the hours. Managers en|4.39|0.70|8004005|edu packnameless #5|3|camping|8|Sports|388|eingeingpri|N/A|498chartreuse3249994|honeydew|Lb|Unknown|22|eseoughtationcally| +6715|AAAAAAAALDKBAAAA|1997-10-27||Main, aware affairs ought to move also skills. Brothers fall so to a cities. Particularly other games give applications. Absolute, dif|0.22|0.16|10012001|importoamalgamalg #1|12|monitors|10|Electronics|430|barpriese|N/A|37269papaya025641951|burlywood|Oz|Unknown|4|antioughtationcally| +6716|AAAAAAAAMDKBAAAA|1997-10-27|2000-10-26|Sites can burst forwards. Neither economic photographs can start across a standards. Still weak lovers|7.07|3.60|10011002|amalgamalgamalg #2|11|disk drives|10|Electronics|112|ableoughtought|N/A|7purple6072907825557|indian|Lb|Unknown|8|callyoughtationcally| +6717|AAAAAAAAMDKBAAAA|2000-10-27||Sites can burst forwards. Neither economic photographs can start across a standards. Still weak lovers|7.23|6.36|10011002|edu packedu pack #2|11|athletic|4|Shoes|56|callyanti|small|7purple6072907825557|maroon|Dozen|Unknown|92|ationoughtationcally| +6718|AAAAAAAAODKBAAAA|1997-10-27|1999-10-27|Brief years sound neither at a payments. P|6.85|5.89|8004009|edu packnameless #9|4|camping|8|Sports|486|callyeingese|N/A|25323orange606772899|olive|Gross|Unknown|12|eingoughtationcally| +6719|AAAAAAAAODKBAAAA|1999-10-28|2001-10-26|Little, equal supporters shall not fit categories. Visual inhabitants come definitely books. Ge|6.71|4.89|7011010|amalgnameless #10|11|accent|7|Home|655|antiantically|N/A|21961032192246navy20|spring|Gram|Unknown|50|n stoughtationcally| +6720|AAAAAAAAODKBAAAA|2001-10-27||Little, equal supporters shall not fit categories. Visual inhabitants come definitely books. Ge|1.29|1.12|7011010|importoexporti #1|11|infants|3|Children|655|antiantically|extra large|9226507272red8441893|purple|Carton|Unknown|31|barableationcally| +6721|AAAAAAAABEKBAAAA|1997-10-27||Weapons wo|87.45|66.46|9002002|importomaxi #2|2|business|9|Books|315|antioughtpri|N/A|9lawn247458277410176|salmon|Tsp|Unknown|65|oughtableationcally| +6722|AAAAAAAACEKBAAAA|1997-10-27|2000-10-26|English, familiar details may|35.26|29.26|8013003|exportimaxi #3|13|sailing|8|Sports|398|eingn stpri|N/A|385violet09806511472|indian|Pallet|Unknown|20|ableableationcally| +6723|AAAAAAAACEKBAAAA|2000-10-27||English, familiar details may|6.98|29.26|8013003|edu packimporto #2|13|sports-apparel|2|Men|398|eingn stpri|petite|433387saddle34703620|yellow|Gross|Unknown|8|priableationcally| +6724|AAAAAAAAEEKBAAAA|1997-10-27|1999-10-27|Strategic, new principles shall ask young techniques. Full projects will refine main, red questions; mixed, fine levels develop current houses. A|5.48|1.69|5004001|edu packscholar #1|4|classical|5|Music|43|priese|N/A|2013243024saddle0660|peru|Bunch|Unknown|47|eseableationcally| +6725|AAAAAAAAEEKBAAAA|1999-10-28|2001-10-26|Strategic, new principles shall ask young techniques. Full projects will refine main, red questions; mixed, fine levels develop current houses. A|3.27|1.69|5004001|amalgimporto #2|1|accessories|2|Men|43|priese|medium|03617pale73484921271|wheat|Dozen|Unknown|16|antiableationcally| +6726|AAAAAAAAEEKBAAAA|2001-10-27||Again pleased problems can rec|7.13|4.91|5004001|exportischolar #1|3|pop|5|Music|43|priese|N/A|03617pale73484921271|lawn|Carton|Unknown|16|callyableationcally| +6727|AAAAAAAAHEKBAAAA|1997-10-27||Poor circumstances shall not put obviously trousers. Masters view totally on a machines. Rarely favorite off|4.46|3.92|6006008|corpcorp #8|6|rings|6|Jewelry|83|prieing|N/A|57877008394purple064|misty|N/A|Unknown|32|ationableationcally| +6728|AAAAAAAAIEKBAAAA|1997-10-27|2000-10-26|Equally high germans will kill. Legs ought to breathe alone ob|6.34|3.93|6005001|scholarcorp #1|5|earings|6|Jewelry|264|esecallyable|N/A|3451546734lace564565|slate|Ounce|Unknown|22|eingableationcally| +6729|AAAAAAAAIEKBAAAA|2000-10-27||Areas become only over other months. Young windows bring early, open days. Often colourful points would say trees; working dogs should consider for the |2.25|0.85|6005001|edu packimporto #2|5|sports-apparel|2|Men|128|esecallyable|medium|40100sandy2162435100|medium|Cup|Unknown|29|n stableationcally| +6730|AAAAAAAAKEKBAAAA|1997-10-27|1999-10-27|Authorities shall not recommend late observations. Dead decisions take also used trusts. Too black varieties look more small, european sold|5.44|3.15|4003001|exportiedu pack #1|3|kids|4|Shoes|640|baresecally|large|031415041dodger71967|sienna|N/A|Unknown|8|barpriationcally| +6731|AAAAAAAAKEKBAAAA|1999-10-28|2001-10-26|Sharp rates concern. Representative, prospective parts test. For example inc feet used to stay bloody shops. Per|2.96|3.15|1004002|edu packamalg #2|4|swimwear|1|Women|640|baresecally|petite|01815072649290green9|medium|Carton|Unknown|8|oughtpriationcally| +6732|AAAAAAAAKEKBAAAA|2001-10-27||Here industrial stories understand particularl|4.08|1.75|7006003|corpbrand #3|6|rugs|7|Home|51|baresecally|N/A|7811336430603sienna4|saddle|N/A|Unknown|39|ablepriationcally| +6733|AAAAAAAANEKBAAAA|1997-10-27||Hardly continental possibilities might proceed most for a values. Then following groups face. Loud other patients will approach only. Current practices will say nice, productive languages. Reportedly|0.78|0.56|7008004|namelessbrand #4|8|lighting|7|Home|343|priesepri|N/A|10105492548941rosy26|moccasin|Carton|Unknown|64|pripriationcally| +6734|AAAAAAAAOEKBAAAA|1997-10-27|2000-10-26|Necessary, warm standards can come basic colours. Powerful, social books influence more from the communities. Local chemicals may not sue. Never ad|4.97|3.97|6015007|scholarbrand #7|15|custom|6|Jewelry|454|eseantiese|N/A|3542385green43702122|white|Bundle|Unknown|22|esepriationcally| +6735|AAAAAAAAOEKBAAAA|2000-10-27||Necessary, warm standards can come basic colours. Powerful, social books influence more from the communities. Local chemicals may not sue. Never ad|3.95|1.46|6011008|amalgbrand #8|15|semi-precious|6|Jewelry|454|eseantiese|N/A|pale0311401763281672|yellow|Tbl|Unknown|8|antipriationcally| +6736|AAAAAAAAAFKBAAAA|1997-10-27|1999-10-27|Types cannot help so famous patterns. Minut|2.49|1.51|2004001|edu packimporto #1|4|sports-apparel|2|Men|679|n stationcally|medium|3610pale332915761081|deep|Ton|Unknown|32|callypriationcally| +6737|AAAAAAAAAFKBAAAA|1999-10-28|2001-10-26|Human courts apply more etc strong men. Democratic, level parties ought to find; economic factors accept only great solutions; efforts might include case|6.70|1.51|4004002|edu packedu pack #2|4|athletic|4|Shoes|679|n stationcally|medium|3610pale332915761081|papaya|Pound|Unknown|18|ationpriationcally| +6738|AAAAAAAAAFKBAAAA|2001-10-27||Western phases may not consult together once considerable others. Fine, overall conclusions could not undertake musicians. Often joint words maintain strong|2.10|1.51|6007003|brandcorp #3|4|pendants|6|Jewelry|679|n stationcally|N/A|782895945misty635891|hot|Bunch|Unknown|42|eingpriationcally| +6739|AAAAAAAADFKBAAAA|1997-10-27||Young, following parameters provide too clear customers. Possible, maximum services fall always new feelings. Scottish, communist projects benefit |1.47|1.05|7008010|namelessbrand #10|8|lighting|7|Home|142|ableeseought|N/A|0600546sienna9424443|papaya|Ounce|Unknown|22|n stpriationcally| +6740|AAAAAAAAEFKBAAAA|1997-10-27|2000-10-26|New knees apply. Other activities wait only. Students will not meet especi|2.81|1.60|5001001|amalgscholar #1|1|rock|5|Music|76|callyation|N/A|6586saddle2740329542|peach|Dozen|Unknown|80|bareseationcally| +6741|AAAAAAAAEFKBAAAA|2000-10-27||Human, available patients might not search now formal, normal features; wrong, weak weekends could develop for a players. Optimistic positions used to leave much pr|3.28|1.60|8001002|amalgnameless #2|1|athletic shoes|8|Sports|113|callyation|N/A|11firebrick924672404|sky|Tsp|Unknown|19|oughteseationcally| +6742|AAAAAAAAGFKBAAAA|1997-10-27|1999-10-27|Only young colleges will get nearly more major structures. Pink colleagues will employ fairly wooden objections. Leaders react around powers. Accounts might manifest northern varieties; im|3.15|2.67|10001011|amalgunivamalg #11|1|cameras|10|Electronics|415|antioughtese|N/A|26lavender7475671227|pink|Lb|Unknown|8|ableeseationcally| +6743|AAAAAAAAGFKBAAAA|1999-10-28|2001-10-26|Only young colleges will get nearly more major structures. Pink colleagues will employ fairly wooden objections. Leaders react around powers. Accounts might manifest northern varieties; im|1.16|0.37|10001011|brandnameless #4|7|hockey|8|Sports|221|antioughtese|N/A|26lavender7475671227|smoke|Dram|Unknown|79|prieseationcally| +6744|AAAAAAAAGFKBAAAA|2001-10-27||Other|9.49|5.31|7013005|exportinameless #5|13|wallpaper|7|Home|458|eingantiese|N/A|hot41924762560789316|salmon|Bunch|Unknown|28|eseeseationcally| +6745|AAAAAAAAJFKBAAAA|1997-10-27||Meanwhile wet products ascerta|5.40|2.59|9011002|amalgunivamalg #2|11|cooking|9|Books|362|ablecallypri|N/A|75910046996tomato010|saddle|Box|Unknown|24|antieseationcally| +6746|AAAAAAAAKFKBAAAA|1997-10-27|2000-10-26|Important childre|9.84|2.95|7012007|importonameless #7|12|paint|7|Home|449|n steseese|N/A|56honeydew1506587959|slate|Pallet|Unknown|7|callyeseationcally| +6747|AAAAAAAAKFKBAAAA|2000-10-27||New cats used to continue suddenly true wages. Goods would apply already professional, old aspects. Both essential users use workers. Trees remember on the customs. |2.82|2.95|7012007|edu packexporti #2|12|school-uniforms|3|Children|449|n steseese|medium|56honeydew1506587959|snow|Bundle|Unknown|15|ationeseationcally| +6748|AAAAAAAAMFKBAAAA|1997-10-27|1999-10-27|Most modern concentrations may direct e|0.56|0.45|9006005|corpmaxi #5|6|parenting|9|Books|144|eseeseought|N/A|9204946red8324242552|green|Tsp|Unknown|65|eingeseationcally| +6749|AAAAAAAAMFKBAAAA|1999-10-28|2001-10-26|About regular doors spread always never new days. Numbers install |2.14|1.49|1001002|amalgamalg #2|6|dresses|1|Women|28|eingable|medium|9204946red8324242552|grey|Tbl|Unknown|33|n steseationcally| +6750|AAAAAAAAMFKBAAAA|2001-10-27||About regular doors spread always never new days. Numbers install |4.31|1.49|1001002|edu packimporto #1|6|sports-apparel|2|Men|28|eingable|small|9204946red8324242552|orange|Unknown|Unknown|17|barantiationcally| +6751|AAAAAAAAPFKBAAAA|1997-10-27||Happy, other limits could|1.22|0.43|4001002|amalgedu pack #2|1|womens|4|Shoes|191|oughtn stought|extra large|71765043958rosy45952|firebrick|Ounce|Unknown|61|oughtantiationcally| +6752|AAAAAAAAAGKBAAAA|1997-10-27|2000-10-26|Problems will solve too priests. Forward sufficie|3.12|1.68|2002001|importoimporto #1|2|shirts|2|Men|310|baroughtpri|large|270771plum5638259129|seashell|Lb|Unknown|32|ableantiationcally| +6753|AAAAAAAAAGKBAAAA|2000-10-27||Problems will solve too priests. Forward sufficie|0.88|1.68|2002001|corpamalgamalg #5|2|wireless|10|Electronics|130|barpriought|N/A|270771plum5638259129|peru|Lb|Unknown|7|priantiationcally| +6754|AAAAAAAACGKBAAAA|1997-10-27|1999-10-27|Dry things cannot apply also. Earlier free lines continue very diff|7.88|5.98|1004001|edu packamalg #1|4|swimwear|1|Women|655|antiantically|economy|314764553seashell173|spring|Gram|Unknown|2|eseantiationcally| +6755|AAAAAAAACGKBAAAA|1999-10-28|2001-10-26|Ver|3.03|2.45|10002007|importounivamalg #7|4|camcorders|10|Electronics|655|antiantically|N/A|314764553seashell173|sandy|Ounce|Unknown|7|antiantiationcally| +6756|AAAAAAAACGKBAAAA|2001-10-27||Ver|7.58|2.45|1004001|edu packamalg #1|4|swimwear|1|Women|24|antiantically|extra large|73672340tomato218712|coral|Bunch|Unknown|39|callyantiationcally| +6757|AAAAAAAAFGKBAAAA|1997-10-27||Perhaps original notes |0.75|0.34|7016008|corpnameless #8|16|furniture|7|Home|195|antin stought|N/A|68163079090steel9193|misty|Dram|Unknown|20|ationantiationcally| +6758|AAAAAAAAGGKBAAAA|1997-10-27|2000-10-26|Therefore safe tec|5.97|3.10|9014003|edu packunivamalg #3|14|sports|9|Books|297|ationn stable|N/A|465687seashell706106|royal|Ton|Unknown|46|eingantiationcally| +6759|AAAAAAAAGGKBAAAA|2000-10-27||Important children vote so. Answers apply nevertheless free problems. Years hope easily united police. |0.29|0.09|9014003|exportibrand #10|3|kids|7|Home|986|ationn stable|N/A|3wheat46542939776684|spring|Pound|Unknown|15|n stantiationcally| +6760|AAAAAAAAIGKBAAAA|1997-10-27|1999-10-27|Likely elements could occur beyond the edges. Positive performances will flee apparently for a eyes. Gradually |2.63|2.34|10005012|scholarunivamalg #12|5|karoke|10|Electronics|317|ationoughtpri|N/A|03899121salmon297580|mint|Dram|Unknown|91|barcallyationcally| +6761|AAAAAAAAIGKBAAAA|1999-10-28|2001-10-26|Official, vast problems would telephone moderately natural, only heads; both great participants would not boost clear, only principles. Virtually national mi|3.80|2.34|10006013|corpunivamalg #13|6|musical|10|Electronics|317|ationoughtpri|N/A|03899121salmon297580|white|Bunch|Unknown|4|oughtcallyationcally| +6762|AAAAAAAAIGKBAAAA|2001-10-27||Workers could fit further. Games judge here effective, special areas. Only goods ought to |6.36|2.34|10014014|edu packamalgamalg #14|6|automotive|10|Electronics|317|ationoughtpri|N/A|26501881656611snow87|purple|Dozen|Unknown|68|ablecallyationcally| +6763|AAAAAAAALGKBAAAA|1997-10-27||Ext|33.33|26.33|10016015|corpamalgamalg #15|16|wireless|10|Electronics|745|antieseation|N/A|538puff7944033159846|white|Ton|Unknown|23|pricallyationcally| +6764|AAAAAAAAMGKBAAAA|1997-10-27|2000-10-26|Recent, complex supporters could not earn clearly significant counties; light goods cannot overcome drivers. Levels would maintain just already poor features. Other obser|13.37|7.62|7012005|importonameless #5|12|paint|7|Home|236|callypriable|N/A|7428rose819168146811|purple|Bunch|Unknown|9|esecallyationcally| +6765|AAAAAAAAMGKBAAAA|2000-10-27||Possible records |80.65|36.29|7012005|edu packunivamalg #10|14|sports|9|Books|437|ationpriese|N/A|786steel991577051183|white|Dram|Unknown|13|anticallyationcally| +6766|AAAAAAAAOGKBAAAA|1997-10-27|1999-10-27|Groups shall not want then; little lexical interpretations will regard police. English, classical plans produce still. Consid|2.35|1.62|3004001|edu packexporti #1|4|school-uniforms|3|Children|574|eseationanti|medium|1726362spring0475505|snow|Unknown|Unknown|63|callycallyationcally| +6767|AAAAAAAAOGKBAAAA|1999-10-28|2001-10-26|Groups shall not want then; little lexical interpretations will regard police. English, classical plans produce still. Consid|4.80|1.62|3004001|scholarbrand #8|4|blinds/shades|7|Home|109|n stbarought|N/A|33565302yellow893306|tomato|Ounce|Unknown|31|ationcallyationcally| +6768|AAAAAAAAOGKBAAAA|2001-10-27||Local women relax under a forces; financial lives remark electronic sales; undoubtedly young carers can go to the patients. Opportunities may not leap sales|0.12|1.62|10013003|exportiamalgamalg #3|13|stereo|10|Electronics|109|n stbarought|N/A|33565302yellow893306|lime|Bundle|Unknown|30|eingcallyationcally| +6769|AAAAAAAABHKBAAAA|1997-10-27||Positive, irish guidelines dine old times. Patient, interesting police e|4.12|1.52|9010002|univunivamalg #2|10|travel|9|Books|32|ablepri|N/A|9780259192honeydew23|rose|Ounce|Unknown|14|n stcallyationcally| +6770|AAAAAAAACHKBAAAA|1997-10-27|2000-10-26|Great, old firms watch tasks; things constrain. Senior deve|4.67|2.89|9010003|univunivamalg #3|10|travel|9|Books|40|barese|N/A|1328094367sienna2226|smoke|Box|Unknown|47|barationationcally| +6771|AAAAAAAACHKBAAAA|2000-10-27||Experimental shares shall not make into the memories. Schemes should think only, external men; widely significant varieties will hold genetic responses. High clear years want unexpe|8.26|5.86|7013002|exportinameless #2|10|wallpaper|7|Home|225|barese|N/A|1328094367sienna2226|slate|Bunch|Unknown|58|oughtationationcally| +6772|AAAAAAAAEHKBAAAA|1997-10-27|1999-10-27|Extensive circumstances consider already russian discussions. Both open problems try in an charts; wa|6.89|6.13|8012003|importomaxi #3|12|guns|8|Sports|641|oughtesecally|N/A|8981409063rose960903|cream|Gram|Unknown|6|ableationationcally| +6773|AAAAAAAAEHKBAAAA|1999-10-28|2001-10-26|Wet, chief units explain too buildings. Local guns could arise most variations; peoples would|4.04|2.58|1003002|exportiamalg #2|3|maternity|1|Women|432|ablepriese|petite|827080blue6142617112|burnished|Carton|Unknown|66|priationationcally| +6774|AAAAAAAAEHKBAAAA|2001-10-27||Wet, chief units explain too buildings. Local guns could arise most variations; peoples would|1.66|2.58|1003002|scholaramalgamalg #9|3|portable|10|Electronics|432|ablepriese|N/A|1054295powder3935044|yellow|Bunch|Unknown|9|eseationationcally| +6775|AAAAAAAAHHKBAAAA|1997-10-27||Tonight recent girls develop gently past powers|6.24|2.24|1004002|edu packamalg #2|4|swimwear|1|Women|58|einganti|medium|825126690puff9255639|peru|Dozen|Unknown|40|antiationationcally| +6776|AAAAAAAAIHKBAAAA|1997-10-27|2000-10-26|Very historic arms may happen even able exis|9.19|5.14|9001009|amalgmaxi #9|1|arts|9|Books|124|eseableought|N/A|213441steel051222851|seashell|Oz|Unknown|17|callyationationcally| +6777|AAAAAAAAIHKBAAAA|2000-10-27||Recent, small horses arrange here british, significant factors. Needs would not subscribe equal, ambitious days. Furious, forthcoming clothes may try at random straight post-war year|1.72|0.60|9001009|exportibrand #8|3|kids|7|Home|124|eseableought|N/A|213441steel051222851|purple|Bundle|Unknown|16|ationationationcally| +6778|AAAAAAAAKHKBAAAA|1997-10-27|1999-10-27|Beliefs pull families. Specific, new flames should exist quite relatively wonderful restrictions; rather empty officers used to prepare very mai|3.50|2.48|1001001|amalgamalg #1|1|dresses|1|Women|272|ableationable|extra large|359purple93747081809|smoke|Case|Unknown|65|eingationationcally| +6779|AAAAAAAAKHKBAAAA|1999-10-28|2001-10-26|Possible persons give radically sensitive prices. Great, standard birds teach elderly, unemployed|9.56|4.01|2001002|amalgimporto #2|1|accessories|2|Men|272|ableationable|petite|359purple93747081809|pink|Box|Unknown|37|n stationationcally| +6780|AAAAAAAAKHKBAAAA|2001-10-27||Military users shall not come instead parents. S|94.88|78.75|2001002|exportiimporto #1|1|pants|2|Men|222|ableableable|petite|359purple93747081809|misty|Gram|Unknown|19|bareingationcally| +6781|AAAAAAAANHKBAAAA|1997-10-27||Initial sales consider interesting fami|2.01|1.62|3002002|importoexporti #2|2|infants|3|Children|51|oughtanti|large|495303606olive532576|peru|Carton|Unknown|29|oughteingationcally| +6782|AAAAAAAAOHKBAAAA|1997-10-27|2000-10-26|Well efficient schools will include indeed areas. Maybe wrong years can like early |80.48|24.14|9007003|brandmaxi #3|7|reference|9|Books|253|priantiable|N/A|84638607slate7697948|steel|Pallet|Unknown|10|ableeingationcally| +6783|AAAAAAAAOHKBAAAA|2000-10-27||Boundaries should correspond tomorrow |4.69|3.61|2001002|amalgimporto #2|1|accessories|2|Men|579|priantiable|medium|84638607slate7697948|plum|Case|Unknown|24|prieingationcally| +6784|AAAAAAAAAIKBAAAA|1997-10-27|1999-10-27|Samples must counter national, wrong letters. Trees might not threaten around in a patients. Directors ought to try. Module|12.10|9.68|6003001|exporticorp #1|3|gold|6|Jewelry|315|antioughtpri|N/A|60557313691sandy0496|navy|Ounce|Unknown|38|eseeingationcally| +6785|AAAAAAAAAIKBAAAA|1999-10-28|2001-10-26|Samples must counter national, wrong letters. Trees might not threaten around in a patients. Directors ought to try. Module|4.91|9.68|6003001|edu packbrand #2|14|estate|6|Jewelry|129|antioughtpri|N/A|00139512peru41707198|smoke|Tbl|Unknown|95|antieingationcally| +6786|AAAAAAAAAIKBAAAA|2001-10-27||Samples must counter national, wrong letters. Trees might not threaten around in a patients. Directors ought to try. Module|1.21|0.77|6003001|maxiunivamalg #4|9|televisions|10|Electronics|129|n stableought|N/A|197sienna73371463806|purple|Dram|Unknown|22|callyeingationcally| +6787|AAAAAAAADIKBAAAA|1997-10-27||Rural, strong dollars can go in a students; nice restrictions leave afield spectacular, royal experts; decisions ought to defend about early effective pp.; russian, national relations shall deli|9.64|8.48|9007008|brandmaxi #8|7|reference|9|Books|152|ableantiought|N/A|76889079puff12359653|maroon|Ounce|Unknown|23|ationeingationcally| +6788|AAAAAAAAEIKBAAAA|1997-10-27|2000-10-26|Gold, effective positions matter grey areas; independently upper offices should use perhaps in a numbers; generally specialist centres should t|2.92|2.27|2001001|amalgimporto #1|1|accessories|2|Men|219|n stoughtable|medium|8772903665papaya8148|plum|Tsp|Unknown|8|eingeingationcally| +6789|AAAAAAAAEIKBAAAA|2000-10-27||Basic farms will locate too at the relations. Present, moving workers extract habitually even so final brothers. Separate payment|3.96|2.27|2001001|brandunivamalg #7|1|personal|10|Electronics|201|n stoughtable|N/A|8772903665papaya8148|sky|Gross|Unknown|14|n steingationcally| +6790|AAAAAAAAGIKBAAAA|1997-10-27|1999-10-27|Moreover new items will not cause cases. Competitive, common figures issue further. Holes raise finally elections. New, open var|4.84|2.03|10005008|scholarunivamalg #8|5|karoke|10|Electronics|828|eingableeing|N/A|2596601spring8371896|turquoise|Bundle|Unknown|59|barn stationcally| +6791|AAAAAAAAGIKBAAAA|1999-10-28|2001-10-26|Moreover new items will not cause cases. Competitive, common figures issue further. Holes raise finally elections. New, open var|3.51|1.89|10005008|importoedu pack #2|2|mens|4|Shoes|398|eingn stpri|economy|2596601spring8371896|spring|Cup|Unknown|11|oughtn stationcally| +6792|AAAAAAAAGIKBAAAA|2001-10-27||Fat prices make her|0.86|0.64|8009003|maxinameless #3|9|optics|8|Sports|398|eingn stpri|N/A|2596601spring8371896|honeydew|Ounce|Unknown|21|ablen stationcally| +6793|AAAAAAAAJIKBAAAA|1997-10-27||Primary years put so british others. Good lips exhibit. Commercial sco|7.74|3.63|7011004|amalgnameless #4|11|accent|7|Home|592|ablen stanti|N/A|9slate43463827587448|ghost|Gram|Unknown|17|prin stationcally| +6794|AAAAAAAAKIKBAAAA|1997-10-27|2000-10-26|Large, small interests used to combat high tears. Weak movies exhibit experimental, lengthy companies. Huge contracts set too. Nearly small lives save now. New, nuclear managers cannot|76.06|54.00|10007012|brandunivamalg #12|7|personal|10|Electronics|280|bareingable|N/A|9784787423spring9643|slate|Carton|Unknown|50|esen stationcally| +6795|AAAAAAAAKIKBAAAA|2000-10-27||Limitations shall see for the workers. And so on thin tools stimulate tropical, ultimate directors. Adult orders worry exactly well local children. Magic, huge un|67.17|54.00|10007012|amalgimporto #2|1|accessories|2|Men|280|bareingable|medium|9784787423spring9643|sandy|Pallet|Unknown|2|antin stationcally| +6796|AAAAAAAAMIKBAAAA|1997-10-27|1999-10-27|Children seem ever essential gaps; details make brightly. Ways must think annually clients. Late solid insects may not quantify more. Well|3.71|2.00|10004014|edu packunivamalg #14|4|audio|10|Electronics|173|priationought|N/A|5221630653853salmon1|smoke|Tsp|Unknown|60|callyn stationcally| +6797|AAAAAAAAMIKBAAAA|1999-10-28|2001-10-26|T|7.10|2.00|10004014|exportischolar #2|3|pop|5|Music|173|priationought|N/A|1664068royal85318050|peru|Pound|Unknown|28|ationn stationcally| +6798|AAAAAAAAMIKBAAAA|2001-10-27||Admirably different hands affect soon. |6.24|2.43|6011007|amalgbrand #7|3|semi-precious|6|Jewelry|598|priationought|N/A|999464seashell882503|dodger|Unknown|Unknown|34|eingn stationcally| +6799|AAAAAAAAPIKBAAAA|1997-10-27||Horses hide less total, musical islands; here entire directors must know more than by a lives. Tables can present in a hills. Gently other securities will not|2.66|2.26|8006010|corpnameless #10|6|football|8|Sports|228|eingableable|N/A|81cream0449419941198|beige|Bunch|Unknown|44|n stn stationcally| +6800|AAAAAAAAAJKBAAAA|1997-10-27|2000-10-26|Opposite, new days boast only more mixed employees; historical, experimental relations used to screw sometimes to a friends. Figures remind subsequent rules. Houses fly properly little sections|1.10|0.39|3001001|amalgexporti #1|1|newborn|3|Children|878|eingationeing|medium|03729688480419snow51|metallic|Unknown|Unknown|29|barbareingcally| +6801|AAAAAAAAAJKBAAAA|2000-10-27||Dead, new imports must bear just services. Even bloody feet ought to proceed things. Moreover coloured candidates shall talk considerable types. Nearby advisory agreements taste local penal|10.81|0.39|3001001|edu packnameless #2|1|glassware|7|Home|174|eingationeing|N/A|045120406511puff0899|saddle|Unknown|Unknown|18|oughtbareingcally| +6802|AAAAAAAACJKBAAAA|1997-10-27|1999-10-27|Comparabl|6.68|3.40|1003001|exportiamalg #1|3|maternity|1|Women|586|callyeinganti|extra large|81237934plum60728875|violet|Box|Unknown|18|ablebareingcally| +6803|AAAAAAAACJKBAAAA|1999-10-28|2001-10-26|Comparabl|2.39|3.40|1003001|scholarnameless #4|3|tables|7|Home|586|callyeinganti|N/A|81237934plum60728875|pink|Ounce|Unknown|54|pribareingcally| +6804|AAAAAAAACJKBAAAA|2001-10-27||Tools will meet just cases. Standard lights win now visitors. Nuclear dishes find hard, modern thousands; neighbours select at l|3.30|1.48|1003001|edu packnameless #5|3|glassware|7|Home|572|callyeinganti|N/A|00062428orange823528|plum|Cup|Unknown|31|esebareingcally| +6805|AAAAAAAAFJKBAAAA|1997-10-27||Roads clear. General exceptions give capable, free times; patients ought |4.41|3.61|9005002|scholarmaxi #2|5|history|9|Books|565|anticallyanti|N/A|6179706237386light13|smoke|Bundle|Unknown|54|antibareingcally| +6806|AAAAAAAAGJKBAAAA|1997-10-27|2000-10-26|Forward angry dogs suggest inc, necessary representatives. Concerned, |2.81|2.07|1004001|edu packamalg #1|4|swimwear|1|Women|199|n stn stought|large|794tan19471837701736|tomato|Each|Unknown|79|callybareingcally| +6807|AAAAAAAAGJKBAAAA|2000-10-27||Forward angry dogs suggest inc, necessary representatives. Concerned, |6.06|2.07|4002002|importoedu pack #2|2|mens|4|Shoes|199|n stn stought|small|3044145425320yellow8|indian|N/A|Unknown|52|ationbareingcally| +6808|AAAAAAAAIJKBAAAA|1997-10-27|1999-10-27|Cultural months carry. Categories will not ensure already national glasses. Researchers will not move only industries. Rich, rigid texts live by a girls. Proud, front views|5.42|4.28|9010005|univunivamalg #5|10|travel|9|Books|522|ableableanti|N/A|78336939402tan167812|puff|Box|Unknown|17|eingbareingcally| +6809|AAAAAAAAIJKBAAAA|1999-10-28|2001-10-26|Links lend nearly english women. Black, high questions identify early cont|1.65|1.13|4002002|importoedu pack #2|2|mens|4|Shoes|226|ableableanti|extra large|91046orchid089358942|aquamarine|Unknown|Unknown|99|n stbareingcally| +6810|AAAAAAAAIJKBAAAA|2001-10-27||More international trustees cost. Origins can go indeed hot eyes; varieties must not evaluate |9.38|4.31|4002002|exportiamalgamalg #11|2|stereo|10|Electronics|226|ableableanti|N/A|24323903617powder461|white|Ton|Unknown|8|baroughteingcally| +6811|AAAAAAAALJKBAAAA|1997-10-27||Particularly relevant masses used to need for the reasons. Together large agencies establish still women. More than traditional companies may not treat no doubt large naked organizations. Black, q|8.43|5.39|7006002|corpbrand #2|6|rugs|7|Home|397|ationn stpri|N/A|yellow97669389506333|smoke|Gross|Unknown|69|oughtoughteingcally| +6812|AAAAAAAAMJKBAAAA|1997-10-27|2000-10-26|References may not move deep on a sites. Almost other files can try quite welsh camps. Internal, certain bonds must remain never for ever immediate lin|2.95|1.00|8004003|edu packnameless #3|4|camping|8|Sports|216|callyoughtable|N/A|rose8495142675074438|sandy|Gram|Unknown|45|ableoughteingcally| +6813|AAAAAAAAMJKBAAAA|2000-10-27||Foundations raise always national cars. Further international friends help basically in no values. Police used to know very. Ready, e|59.70|35.82|8004003|amalgnameless #4|1|athletic shoes|8|Sports|216|callyoughtable|N/A|2279snow767951615657|thistle|Unknown|Unknown|49|prioughteingcally| +6814|AAAAAAAAOJKBAAAA|1997-10-27|1999-10-27|Excellent, labour problems will like minutes; main |1.85|1.60|3004001|edu packexporti #1|4|school-uniforms|3|Children|259|n stantiable|small|8868honeydew89388290|pale|Box|Unknown|5|eseoughteingcally| +6815|AAAAAAAAOJKBAAAA|1999-10-28|2001-10-26|Sophisticated, great cars remember infrequently great, identical women. Subject services see. Domestic, abstra|0.42|1.60|3001002|amalgexporti #2|1|newborn|3|Children|259|n stantiable|large|2034345purple9471672|powder|Case|Unknown|3|antioughteingcally| +6816|AAAAAAAAOJKBAAAA|2001-10-27||Sorry, pink newspapers understand even. Defensive, powerful members enable further. Public men should tell attractive forces. Characteristics must expect services. O|3.62|3.00|8002007|importonameless #7|2|baseball|8|Sports|259|n stantiable|N/A|85159379maroon651571|puff|Pound|Unknown|46|callyoughteingcally| +6817|AAAAAAAABKKBAAAA|1997-10-27||Large schools find just available groups. Authors keep far. Fierce sites can ensure other police. Traditional, following complaints shall see there popular clients. Serious initiatives add of cour|3.62|3.22|5004002|edu packscholar #2|4|classical|5|Music|514|eseoughtanti|N/A|81124papaya951607569|violet|N/A|Unknown|46|ationoughteingcally| +6818|AAAAAAAACKKBAAAA|1997-10-27|2000-10-26|Considerable, small tools used to bring above top improvements. Most full parts conclude often for the subjects|1.54|1.35|1001001|amalgamalg #1|1|dresses|1|Women|595|antin stanti|extra large|724829956518snow7957|snow|Dram|Unknown|59|eingoughteingcally| +6819|AAAAAAAACKKBAAAA|2000-10-27||Considerable, small tools used to bring above top improvements. Most full parts conclude often for the subjects|9.72|1.35|1001001|amalgimporto #2|1|accessories|2|Men|595|antin stanti|small|724829956518snow7957|maroon|Bundle|Unknown|22|n stoughteingcally| +6820|AAAAAAAAEKKBAAAA|1997-10-27|1999-10-27|Only single|1.98|1.36|9014005|edu packunivamalg #5|14|sports|9|Books|239|n stpriable|N/A|18seashell6692929770|peach|Ton|Unknown|5|barableeingcally| +6821|AAAAAAAAEKKBAAAA|1999-10-28|2001-10-26|Unnecessary mechanisms can reach. Wrong, ready lips |8.59|6.35|9014005|namelessbrand #2|14|lighting|7|Home|626|callyablecally|N/A|1647337lime951327357|misty|Bundle|Unknown|7|oughtableeingcally| +6822|AAAAAAAAEKKBAAAA|2001-10-27||Unnecessary mechanisms can reach. Wrong, ready lips |9.79|6.35|5003001|exportischolar #1|14|pop|5|Music|626|callyablecally|N/A|1647337lime951327357|purple|N/A|Unknown|30|ableableeingcally| +6823|AAAAAAAAHKKBAAAA|1997-10-27||Separate flowers agree most likely points. Overseas funds used to weaken only effective brothers. Industrial events must not hear colonial aspect|2.14|1.51|8010004|univmaxi #4|10|pools|8|Sports|343|priesepri|N/A|642pale5579098719016|smoke|Ton|Unknown|2|priableeingcally| +6824|AAAAAAAAIKKBAAAA|1997-10-27|2000-10-26|Particularly important effects ought |5.33|3.46|3002001|importoexporti #1|2|infants|3|Children|513|prioughtanti|economy|42843tomato336790837|sandy|Gross|Unknown|1|eseableeingcally| +6825|AAAAAAAAIKKBAAAA|2000-10-27||Particularly important effects ought |3.00|3.46|3002001|exportiedu pack #2|3|kids|4|Shoes|513|prioughtanti|small|47272thistle58125545|spring|Gram|Unknown|57|antiableeingcally| +6826|AAAAAAAAKKKBAAAA|1997-10-27|1999-10-27|Complete residents buy virtually weak times. Also effective clubs used to ask straight other i|1.42|0.42|3001001|amalgexporti #1|1|newborn|3|Children|239|n stpriable|small|725281211674floral74|peru|Dozen|Unknown|28|callyableeingcally| +6827|AAAAAAAAKKKBAAAA|1999-10-28|2001-10-26|Only professional unions would find clean sales. Privileges maintain both|4.26|1.83|8016008|corpmaxi #8|1|golf|8|Sports|140|bareseought|N/A|96611tomato147355719|bisque|Bundle|Unknown|9|ationableeingcally| +6828|AAAAAAAAKKKBAAAA|2001-10-27||Carefully catholic surfaces sell too little figures; right upper men shall step therefore utterly great comp|1.06|1.83|8016008|namelesscorp #5|1|mens watch|6|Jewelry|410|bareseought|N/A|96611tomato147355719|turquoise|Case|Unknown|93|eingableeingcally| +6829|AAAAAAAANKKBAAAA|1997-10-27||Now usual others shall express again books. Inevitable sales cannot take good. Significantly long words finish continuous, good duties. Countries can run in a branches; even s|6.03|3.01|9007002|brandmaxi #2|7|reference|9|Books|964|esecallyn st|N/A|801325546tomato30411|peru|Case|Unknown|9|n stableeingcally| +6830|AAAAAAAAOKKBAAAA|1997-10-27|2000-10-26|Weeks may not seem at least standard policemen. Together genetic be|3.66|2.81|5003001|exportischolar #1|3|pop|5|Music|137|ationpriought|N/A|smoke927169652380228|rose|Cup|Unknown|27|barprieingcally| +6831|AAAAAAAAOKKBAAAA|2000-10-27||Weeks may not seem at least standard policemen. Together genetic be|3.76|2.93|5003001|univunivamalg #4|10|travel|9|Books|50|ationpriought|N/A|smoke927169652380228|rose|Carton|Unknown|36|oughtprieingcally| +6832|AAAAAAAAALKBAAAA|1997-10-27|1999-10-27|Schools get for long local, green generations. Well casual walls hold at least to the colours. Social faces cannot cheat still for a relatives. Stupid, light homes m|6.85|3.90|1001001|amalgamalg #1|1|dresses|1|Women|931|oughtprin st|large|107945155154sienna70|coral|Carton|Unknown|88|ableprieingcally| +6833|AAAAAAAAALKBAAAA|1999-10-28|2001-10-26|Schools get for long local, green generations. Well casual walls hold at least to the colours. Social faces cannot cheat still for a relatives. Stupid, light homes m|5.41|3.30|5004002|edu packscholar #2|1|classical|5|Music|931|oughtprin st|N/A|107945155154sienna70|powder|Dram|Unknown|6|priprieingcally| +6834|AAAAAAAAALKBAAAA|2001-10-27||Schools get for long local, green generations. Well casual walls hold at least to the colours. Social faces cannot cheat still for a relatives. Stupid, light homes m|3.56|3.30|5004002|scholarunivamalg #7|15|fiction|9|Books|931|oughtprin st|N/A|107945155154sienna70|turquoise|Dozen|Unknown|31|eseprieingcally| +6835|AAAAAAAADLKBAAAA|1997-10-27||Military, royal ideas leave especially soviet parties. |6.81|3.33|1001002|amalgamalg #2|1|dresses|1|Women|174|eseationought|extra large|955magenta7041230428|sky|Ton|Unknown|53|antiprieingcally| +6836|AAAAAAAAELKBAAAA|1997-10-27|2000-10-26|Technically likely armies would increase at a twins. |4.06|1.42|6003005|exporticorp #5|3|gold|6|Jewelry|177|ationationought|N/A|27835720031papaya684|olive|Tbl|Unknown|53|callyprieingcally| +6837|AAAAAAAAELKBAAAA|2000-10-27||Just old minutes will see less than again black women. Horizontal, positive properties alter useful buildings. Movements must watch in the combinations. Rooms might state however british women. Corre|9.92|1.42|6015006|scholarbrand #6|3|custom|6|Jewelry|177|ationationought|N/A|27835720031papaya684|steel|Each|Unknown|11|ationprieingcally| +6838|AAAAAAAAGLKBAAAA|1997-10-27|1999-10-27|Actually correct presents will fill else double, useful operations. African, external penalties will|62.88|24.52|3003001|exportiexporti #1|3|toddlers|3|Children|204|esebarable|extra large|90646492rose47563250|steel|Ton|Unknown|15|eingprieingcally| +6839|AAAAAAAAGLKBAAAA|1999-10-28|2001-10-26|High communities would form widely certain point|6.67|24.52|3003001|scholarcorp #8|3|earings|6|Jewelry|624|esebarable|N/A|2572yellow0458912635|peru|Dozen|Unknown|31|n stprieingcally| +6840|AAAAAAAAGLKBAAAA|2001-10-27||High communities would form widely certain point|29.97|10.48|9008001|namelessmaxi #1|3|romance|9|Books|624|esebarable|N/A|2572yellow0458912635|purple|Carton|Unknown|13|bareseeingcally| +6841|AAAAAAAAJLKBAAAA|1997-10-27||Scholars promote little for|4.52|1.35|2003002|exportiimporto #2|3|pants|2|Men|792|ablen station|petite|823181rosy1747963746|rosy|Ounce|Unknown|26|oughteseeingcally| +6842|AAAAAAAAKLKBAAAA|1997-10-27|2000-10-26|Very wrong marks would like in particular new, african quantities; local barriers return. Things used to see. Dead clients must not say studies. There good studies start appropriat|4.54|3.08|9009003|maximaxi #3|9|science|9|Books|375|antiationpri|N/A|01325486515maroon762|pink|Bunch|Unknown|9|ableeseeingcally| +6843|AAAAAAAAKLKBAAAA|2000-10-27||Designers may describe closely little artists. Clear, modern partners depend really new children. Commercial bars rebuild as difficult wheels; industrial matt|6.77|4.94|9009003|importoamalg #2|9|fragrances|1|Women|244|eseeseable|extra large|2648823744318powder4|pale|Bundle|Unknown|56|prieseeingcally| +6844|AAAAAAAAMLKBAAAA|1997-10-27|1999-10-27|Implicit, indian |0.68|0.46|7012005|importonameless #5|12|paint|7|Home|646|callyesecally|N/A|5970333497plum609534|lawn|Bundle|Unknown|39|eseeseeingcally| +6845|AAAAAAAAMLKBAAAA|1999-10-28|2001-10-26|Popular men like. Common, small managers ought to meet in the women; annual|8.12|0.46|1001002|amalgamalg #2|1|dresses|1|Women|646|callyesecally|medium|5970333497plum609534|goldenrod|Gram|Unknown|21|antieseeingcally| +6846|AAAAAAAAMLKBAAAA|2001-10-27||Immediate, labour doubts come with the writers. Leading others join at all double laws. Mild, single feelings monitor probably little boys. Yet w|2.59|0.46|10011013|amalgamalgamalg #13|1|disk drives|10|Electronics|263|pricallyable|N/A|82075995slate8868945|pale|Lb|Unknown|49|callyeseeingcally| +6847|AAAAAAAAPLKBAAAA|1997-10-27||Then powerful eggs reclaim only full worlds; thus key values suit others. Very open effects|3.99|3.03|10008014|namelessunivamalg #14|8|scanners|10|Electronics|78|eingation|N/A|01783tomato901731140|peru|Bunch|Unknown|23|ationeseeingcally| +6848|AAAAAAAAAMKBAAAA|1997-10-27|2000-10-26|Scottish, broken pupils must not wait high just terms. International, european miles might think; areas may get true feet. Certain authorities can eliminate|3.89|2.13|9012009|importounivamalg #9|12|home repair|9|Books|468|eingcallyese|N/A|4245539peach75333893|thistle|Box|Unknown|15|eingeseeingcally| +6849|AAAAAAAAAMKBAAAA|2000-10-27||Remarkable powers |0.93|0.59|9012009|amalgunivamalg #16|12|cameras|10|Electronics|468|eingcallyese|N/A|4245539peach75333893|seashell|Tsp|Unknown|16|n steseeingcally| +6850|AAAAAAAACMKBAAAA|1997-10-27|1999-10-27|Previous, unusual pounds could concentrate short by the articles. For example possible|8.04|3.45|8007001|brandnameless #1|7|hockey|8|Sports|140|bareseought|N/A|833318585079plum7959|spring|Oz|Unknown|19|barantieingcally| +6851|AAAAAAAACMKBAAAA|1999-10-28|2001-10-26|Previous, unusual pounds could concentrate short by the articles. For example possible|7.53|3.61|5002002|importoscholar #2|2|country|5|Music|140|bareseought|N/A|833318585079plum7959|frosted|Dozen|Unknown|4|oughtantieingcally| +6852|AAAAAAAACMKBAAAA|2001-10-27||Tears meet provincial regulations; proper areas used to advise much maximum publications. Available things bear. Th|5.83|4.60|10003002|exportiunivamalg #2|3|dvd/vcr players|10|Electronics|140|bareseought|N/A|833318585079plum7959|sienna|Case|Unknown|54|ableantieingcally| +6853|AAAAAAAAFMKBAAAA|1997-10-27||Seriously keen contents sit never forces. Other days shall not drive foreign, crazy creditors. Scarcely australian teeth consider. Eff|6.93|2.07|6013006|exportibrand #6|13|loose stones|6|Jewelry|467|ationcallyese|N/A|515rosy6623089816479|saddle|Dozen|Unknown|17|priantieingcally| +6854|AAAAAAAAGMKBAAAA|1997-10-27|2000-10-26|Hard inappropriate consequences would reproduce quite. Pers|4.84|1.93|6016007|corpbrand #7|16|consignment|6|Jewelry|648|eingesecally|N/A|21254949orchid908364|wheat|Tsp|Unknown|46|eseantieingcally| +6855|AAAAAAAAGMKBAAAA|2000-10-27||For example old issues change critical universities. Alone, important speeches live police. Groups achieve. Poor, surprised priorities produce for a problems. Rich, po|1.46|1.93|7016006|corpnameless #6|16|furniture|7|Home|648|eingesecally|N/A|21254949orchid908364|pink|Dozen|Unknown|31|antiantieingcally| +6856|AAAAAAAAIMKBAAAA|1997-10-27|1999-10-27|Doors shall upset there. Early, main revenues visit tonnes; domestic, mere reasons may last around a yards. More new lives find whi|29.89|12.25|6009001|maxicorp #1|9|womens watch|6|Jewelry|66|callycally|N/A|5673119615pink414193|snow|Case|Unknown|8|callyantieingcally| +6857|AAAAAAAAIMKBAAAA|1999-10-28|2001-10-26|Doors shall upset there. Early, main revenues visit tonnes; domestic, mere reasons may last around a yards. More new lives find whi|8.07|4.43|6009001|exportischolar #2|9|pop|5|Music|216|callycally|N/A|936815392pale3883542|grey|Tsp|Unknown|51|ationantieingcally| +6858|AAAAAAAAIMKBAAAA|2001-10-27||Misleading, slow conferences would speak really specialist police|8.31|7.06|6009001|maxinameless #9|9|optics|8|Sports|648|eingesecally|N/A|7984781lavender05436|rosy|Case|Unknown|40|eingantieingcally| +6859|AAAAAAAALMKBAAAA|1997-10-27||Political, french streets used to introduce just labour |1.59|1.38|10004009|edu packunivamalg #9|4|audio|10|Electronics|569|n stcallyanti|N/A|640532642548steel011|mint|Pound|Unknown|31|n stantieingcally| +6860|AAAAAAAAMMKBAAAA|1997-10-27|2000-10-26|More than rich ideas will mean further in a|7.56|5.89|2002001|importoimporto #1|2|shirts|2|Men|737|ationpriation|large|4maroon4116344870368|magenta|Gram|Unknown|63|barcallyeingcally| +6861|AAAAAAAAMMKBAAAA|2000-10-27||Human, public cigarettes would feel authorities. Obv|2.64|5.89|2002001|corpunivamalg #11|2|musical|10|Electronics|262|ationpriation|N/A|9lime901781354768239|royal|Tsp|Unknown|9|oughtcallyeingcally| +6862|AAAAAAAAOMKBAAAA|1997-10-27|1999-10-27|Adequate things reassure unknown legs. Old, possible bishops shall locate else during a companies; bitter, alone |3.98|1.43|5004001|edu packscholar #1|4|classical|5|Music|101|oughtbarought|N/A|9106471327492sky7675|spring|Ton|Unknown|26|ablecallyeingcally| +6863|AAAAAAAAOMKBAAAA|1999-10-28|2001-10-26|Adequate things reassure unknown legs. Old, possible bishops shall locate else during a companies; bitter, alone |2.85|1.43|5004001|edu packamalg #2|4|swimwear|1|Women|101|oughtbarought|medium|7342449473peach09982|floral|Gross|Unknown|75|pricallyeingcally| +6864|AAAAAAAAOMKBAAAA|2001-10-27||Great, original ambitions accelerate to a concerns. No longer female hours withdraw certainly wages. M|0.19|0.09|5004001|importoexporti #1|4|infants|3|Children|101|oughtbarought|medium|6white20310895317236|honeydew|Dozen|Unknown|11|esecallyeingcally| +6865|AAAAAAAABNKBAAAA|1997-10-27||Minutes can expect outside strong, alternative developers. Proper movemen|7.15|5.21|7016006|corpnameless #6|16|furniture|7|Home|185|antieingought|N/A|97421gainsboro448354|metallic|Dram|Unknown|2|anticallyeingcally| +6866|AAAAAAAACNKBAAAA|1997-10-27|2000-10-26|European, happy homes shall not share. Double calls can cover just in order regular developments; inevitable rooms ought to promise according to a eyes. Normal attempts grow only, complex goods|8.03|4.57|7010007|univnameless #7|10|flatware|7|Home|68|eingcally|N/A|630565burlywood05104|snow|Each|Unknown|83|callycallyeingcally| +6867|AAAAAAAACNKBAAAA|2000-10-27||European, happy homes shall not share. Double calls can cover just in order regular developments; inevitable rooms ought to promise according to a eyes. Normal attempts grow only, complex goods|1.14|0.51|2004002|edu packimporto #2|4|sports-apparel|2|Men|68|eingcally|large|630565burlywood05104|metallic|Gram|Unknown|31|ationcallyeingcally| +6868|AAAAAAAAENKBAAAA|1997-10-27|1999-10-27|Views investigate previously significant points; cu|90.62|37.15|3004001|edu packexporti #1|4|school-uniforms|3|Children|257|ationantiable|medium|63652tomato432103651|almond|Gross|Unknown|22|eingcallyeingcally| +6869|AAAAAAAAENKBAAAA|1999-10-28|2001-10-26|Human, expected results shall move names; detailed, env|0.30|37.15|9014006|edu packunivamalg #6|4|sports|9|Books|259|ationantiable|N/A|63652tomato432103651|sky|Unknown|Unknown|91|n stcallyeingcally| +6870|AAAAAAAAENKBAAAA|2001-10-27||Human, expected results shall move names; detailed, env|87.93|37.15|8002001|importonameless #1|2|baseball|8|Sports|259|n stantiable|N/A|63132800blanched9749|white|Tsp|Unknown|58|barationeingcally| +6871|AAAAAAAAHNKBAAAA|1997-10-27||Main guests satisfy for a patterns. Particularly technical issues ought to know as others. Authoritie|0.68|0.21|4004002|edu packedu pack #2|4|athletic|4|Shoes|994|esen stn st|medium|160315peach824410232|spring|Tbl|Unknown|60|oughtationeingcally| +6872|AAAAAAAAINKBAAAA|1997-10-27|2000-10-26|Largely similar wings feature ago. Po|3.28|1.01|2001001|amalgimporto #1|1|accessories|2|Men|214|eseoughtable|large|57857541329steel2190|tan|Unknown|Unknown|57|ableationeingcally| +6873|AAAAAAAAINKBAAAA|2000-10-27||Largely similar wings feature ago. Po|4.59|1.01|2001001|exportischolar #2|3|pop|5|Music|214|eseoughtable|N/A|sienna35925719860567|sandy|Dram|Unknown|29|priationeingcally| +6874|AAAAAAAAKNKBAAAA|1997-10-27|1999-10-27|Of course large structures describe. Used factors would know commercial benefits. Then appropriate circumstances should not know so new terms; ev|2.18|0.65|8013005|exportimaxi #5|13|sailing|8|Sports|202|ablebarable|N/A|7208plum470084199631|maroon|Ton|Unknown|15|eseationeingcally| +6875|AAAAAAAAKNKBAAAA|1999-10-28|2001-10-26|Of course large structures describe. Used factors would know commercial benefits. Then appropriate circumstances should not know so new terms; ev|0.63|0.22|8013005|importomaxi #12|13|business|9|Books|67|ablebarable|N/A|430goldenrod30485517|turquoise|N/A|Unknown|7|antiationeingcally| +6876|AAAAAAAAKNKBAAAA|2001-10-27||As quick officers can hear seriously thin, eventual speeches; northern, cognitive factories |6.89|0.22|8013005|amalgedu pack #1|13|womens|4|Shoes|175|antiationought|medium|89359pink87639533088|navy|Bundle|Unknown|8|callyationeingcally| +6877|AAAAAAAANNKBAAAA|1997-10-27||Empty, remarka|9.76|3.61|8006008|corpnameless #8|6|football|8|Sports|307|ationbarpri|N/A|10878014mint41192157|red|Pallet|Unknown|28|ationationeingcally| +6878|AAAAAAAAONKBAAAA|1997-10-27|2000-10-26|Variable years use|6.93|5.40|10004011|edu packunivamalg #11|4|audio|10|Electronics|312|ableoughtpri|N/A|64salmon386411187047|lemon|Cup|Unknown|4|eingationeingcally| +6879|AAAAAAAAONKBAAAA|2000-10-27||Variable years use|2.91|5.40|8001010|amalgnameless #10|1|athletic shoes|8|Sports|100|ableoughtpri|N/A|64salmon386411187047|peach|Carton|Unknown|7|n stationeingcally| +6880|AAAAAAAAAOKBAAAA|1997-10-27|1999-10-27|Amounts go perhaps double settlements. Sufficient, inc years will not attribute little from a others; simply foreign officials should not care vil|3.42|1.12|5001001|amalgscholar #1|1|rock|5|Music|273|priationable|N/A|6042794655indian0282|saddle|Dozen|Unknown|16|bareingeingcally| +6881|AAAAAAAAAOKBAAAA|1999-10-28|2001-10-26|Late small walls can drive enormous pages. New bits should not make women. Husbands used to fetch here differen|4.42|1.81|1003002|exportiamalg #2|3|maternity|1|Women|153|priantiought|small|6042794655indian0282|peru|Bunch|Unknown|14|oughteingeingcally| +6882|AAAAAAAAAOKBAAAA|2001-10-27||Also alone arrangements ough|9.32|7.73|7007003|brandbrand #3|7|decor|7|Home|454|priantiought|N/A|6042794655indian0282|purple|Dram|Unknown|24|ableeingeingcally| +6883|AAAAAAAADOKBAAAA|1997-10-27||Much inner companies could not look nowadays managerial actual detectives. Great days |5.84|3.79|9015008|scholarunivamalg #8|15|fiction|9|Books|276|callyationable|N/A|39174lemon0668077304|blue|Gross|Unknown|15|prieingeingcally| +6884|AAAAAAAAEOKBAAAA|1997-10-27|2000-10-26|Also female signs may leave other, successful birds; highly available objectives used to walk. Ready, short men celebrate likely situations. Rights can get. Complete projects would help ju|3.73|2.61|4003001|exportiedu pack #1|3|kids|4|Shoes|45|antiese|medium|98419300400salmon474|turquoise|N/A|Unknown|35|eseeingeingcally| +6885|AAAAAAAAEOKBAAAA|2000-10-27||International casualties hit so. Chief, sexual prisoners incorporate competitive, major days. Hardly redundant animals diminish here tradi|0.60|2.61|4003001|corpmaxi #10|6|parenting|9|Books|356|antiese|N/A|98419300400salmon474|steel|Dozen|Unknown|29|antieingeingcally| +6886|AAAAAAAAGOKBAAAA|1997-10-27|1999-10-27|Othe|60.94|28.03|9001011|amalgmaxi #11|1|arts|9|Books|992|ablen stn st|N/A|82cornflower81901891|rosy|Pound|Unknown|24|callyeingeingcally| +6887|AAAAAAAAGOKBAAAA|1999-10-28|2001-10-26|Warm connections can tell ago premier, special prices. Winners form by the schemes; british poems involve once more by a results. As impossible prices plan rightly groups; o|1.57|28.03|9001011|amalgunivamalg #3|1|cameras|10|Electronics|262|ablecallyable|N/A|82cornflower81901891|purple|Ounce|Unknown|32|ationeingeingcally| +6888|AAAAAAAAGOKBAAAA|2001-10-27||Warm connections can tell ago premier, special prices. Winners form by the schemes; british poems involve once more by a results. As impossible prices plan rightly groups; o|0.28|28.03|7008009|namelessbrand #9|8|lighting|7|Home|621|oughtablecally|N/A|82cornflower81901891|rosy|Carton|Unknown|37|eingeingeingcally| +6889|AAAAAAAAJOKBAAAA|1997-10-27||Other, difficult towns believe ever good flowers. Extensive, useful strategies should not take. Supposed regions would |8.82|5.46|5004002|edu packscholar #2|4|classical|5|Music|329|n stablepri|N/A|41smoke9756814934999|firebrick|Dram|Unknown|3|n steingeingcally| +6890|AAAAAAAAKOKBAAAA|1997-10-27|2000-10-26|Costs should give ago perfect men. Years believe; tr|2.40|1.12|6007003|brandcorp #3|7|pendants|6|Jewelry|75|antiation|N/A|73673413881smoke6369|yellow|Cup|Unknown|13|barn steingcally| +6891|AAAAAAAAKOKBAAAA|2000-10-27||Agricultural, cultural newspapers take long different views. American|5.03|1.12|6007003|exportimaxi #2|13|sailing|8|Sports|75|antiation|N/A|73673413881smoke6369|tan|Tsp|Unknown|24|oughtn steingcally| +6892|AAAAAAAAMOKBAAAA|1997-10-27|1999-10-27|Suspicious days may not make now in a institutions. In o|2.98|1.04|1004001|edu packamalg #1|4|swimwear|1|Women|266|callycallyable|extra large|3wheat69307582977415|royal|Dram|Unknown|14|ablen steingcally| +6893|AAAAAAAAMOKBAAAA|1999-10-28|2001-10-26|Suspicious days may not make now in a institutions. In o|1.65|1.04|1004001|edu packamalg #2|4|swimwear|1|Women|70|baration|extra large|3wheat69307582977415|snow|Gross|Unknown|36|prin steingcally| +6894|AAAAAAAAMOKBAAAA|2001-10-27||Important homes know sophisticated, full others. Young eyes shall not want technical officers. Candidates attend recently in a names. Precisely good probl|3.55|1.04|1004001|edu packmaxi #5|14|tennis|8|Sports|70|baration|N/A|3wheat69307582977415|purple|Bundle|Unknown|7|esen steingcally| +6895|AAAAAAAAPOKBAAAA|1997-10-27||Chemical, elegant influences should pray certainly with a mo|6.10|3.59|7001006|amalgbrand #6|1|bathroom|7|Home|552|ableantianti|N/A|708lawn7232480498126|puff|Oz|Unknown|80|antin steingcally| +6896|AAAAAAAAAPKBAAAA|1997-10-27|2000-10-26|Wide guards come. Objective, labour industries ca|9.29|6.13|6016001|corpbrand #1|16|consignment|6|Jewelry|71|oughtation|N/A|307364868077142mint5|antique|N/A|Unknown|65|callyn steingcally| +6897|AAAAAAAAAPKBAAAA|2000-10-27||Interestingly high children would apply relevant gaps. Largely |4.75|6.13|2001002|amalgimporto #2|1|accessories|2|Men|166|callycallyought|extra large|7731654140snow562179|pale|N/A|Unknown|16|ationn steingcally| +6898|AAAAAAAACPKBAAAA|1997-10-27|1999-10-27|Years say from a deaths. Polite jeans see standards. Parties check elderly mice. Long young values would disguise before|9.58|6.51|7008009|namelessbrand #9|8|lighting|7|Home|429|n stableese|N/A|1868smoke11686936337|slate|Pallet|Unknown|2|eingn steingcally| +6899|AAAAAAAACPKBAAAA|1999-10-28|2001-10-26|Years say from a deaths. Polite jeans see standards. Parties check elderly mice. Long young values would disguise before|4.64|6.51|7008009|amalgscholar #2|1|rock|5|Music|429|n stableese|N/A|308983801315frosted6|peru|Tsp|Unknown|15|n stn steingcally| +6900|AAAAAAAACPKBAAAA|2001-10-27||Years say from a deaths. Polite jeans see standards. Parties check elderly mice. Long young values would disguise before|1.58|1.16|2002001|importoimporto #1|2|shirts|2|Men|429|n stableese|medium|308983801315frosted6|pale|Lb|Unknown|20|barbarn stcally| +6901|AAAAAAAAFPKBAAAA|1997-10-27||Perfectly coming moments used to rely industrial things. Private, other fig|0.65|0.42|7016002|corpnameless #2|16|furniture|7|Home|61|oughtcally|N/A|398344723deep0719460|pink|Tbl|Unknown|22|oughtbarn stcally| +6902|AAAAAAAAGPKBAAAA|1997-10-27|2000-10-26|Other, typical senses could not make most times. Considerations may not survive smooth bodies. Rational visits make here brief, wrong statements. Home following issues might |5.23|4.18|6004007|edu packcorp #7|4|bracelets|6|Jewelry|281|oughteingable|N/A|53701422ghost1290143|violet|Oz|Unknown|29|ablebarn stcally| +6903|AAAAAAAAGPKBAAAA|2000-10-27||Critics protect quietly just |2.06|1.68|6004007|univnameless #4|4|flatware|7|Home|444|oughteingable|N/A|53701422ghost1290143|red|N/A|Unknown|63|pribarn stcally| +6904|AAAAAAAAIPKBAAAA|1997-10-27|1999-10-27|Fine minds would not ask usually securities. Immediate, natural classes come personally angles. White years shall appear important, material aspects; simply general years organize al|5.66|4.98|8014005|edu packmaxi #5|14|tennis|8|Sports|581|oughteinganti|N/A|089235938253127puff7|pink|Box|Unknown|50|esebarn stcally| +6905|AAAAAAAAIPKBAAAA|1999-10-28|2001-10-26|Fine minds would not ask usually securities. Immediate, natural classes come personally angles. White years shall appear important, material aspects; simply general years organize al|1.15|0.60|5004002|edu packscholar #2|4|classical|5|Music|581|oughteinganti|N/A|240navy7402686014548|tomato|Bunch|Unknown|53|antibarn stcally| +6906|AAAAAAAAIPKBAAAA|2001-10-27||Small costs occur across. Children suggest too single, other kids. Also other practitioners will sell hence pretty, international layers. Other travellers will find; traditiona|4.05|2.87|3004001|edu packexporti #1|4|school-uniforms|3|Children|581|oughteinganti|small|8098365767215saddle5|peach|Each|Unknown|2|callybarn stcally| +6907|AAAAAAAALPKBAAAA|1997-10-27||Inappropriate, chief systems would not help in a offices; dangerous proportions might ins|3.08|1.38|7002008|importobrand #8|2|bedding|7|Home|150|barantiought|N/A|4730peru882836661963|spring|Unknown|Unknown|20|ationbarn stcally| +6908|AAAAAAAAMPKBAAAA|1997-10-27|2000-10-26|Countries may tell major, dangerous rules. French offers make here at a terms. Less new doctors go patients. Level countries may not examine also large teachers; once scientific men coul|8.61|5.16|9013009|exportiunivamalg #9|13|self-help|9|Books|97|ationn st|N/A|7002978495665spring9|plum|Carton|Unknown|12|eingbarn stcally| +6909|AAAAAAAAMPKBAAAA|2000-10-27||Countries may tell major, dangerous rules. French offers make here at a terms. Less new doctors go patients. Level countries may not examine also large teachers; once scientific men coul|73.03|57.69|9013009|amalgunivamalg #8|1|cameras|10|Electronics|53|prianti|N/A|86366016200mint21589|sandy|Ton|Unknown|21|n stbarn stcally| +6910|AAAAAAAAOPKBAAAA|1997-10-27|1999-10-27|Social, english measures|6.54|2.02|8015001|scholarmaxi #1|15|fishing|8|Sports|964|esecallyn st|N/A|61239540puff88719287|lemon|Dozen|Unknown|6|baroughtn stcally| +6911|AAAAAAAAOPKBAAAA|1999-10-28|2001-10-26|Social, english measures|6.16|4.43|7013002|exportinameless #2|13|wallpaper|7|Home|678|eingationcally|N/A|61239540puff88719287|plum|Ounce|Unknown|90|oughtoughtn stcally| +6912|AAAAAAAAOPKBAAAA|2001-10-27||Social, english measures|4.22|2.40|9013001|exportiunivamalg #1|13|self-help|9|Books|422|ableableese|N/A|04451snow50785767681|sienna|N/A|Unknown|5|ableoughtn stcally| +6913|AAAAAAAABALBAAAA|1997-10-27||Now foreign goods think already various times. Slow years shall not assert short. Unable, new woods will know big, genetic rights. Relative islands must express quite since a attitudes.|99.42|44.73|4001002|amalgedu pack #2|1|womens|4|Shoes|149|n steseought|extra large|637315840deep1090858|pink|Ounce|Unknown|3|prioughtn stcally| +6914|AAAAAAAACALBAAAA|1997-10-27|2000-10-26|Days protect most bad hours; at present empty things ought to speak usually further able profits. Feelings may know. Clients will not help|2.79|2.00|4002001|importoedu pack #1|2|mens|4|Shoes|712|ableoughtation|large|4thistle151104160811|red|Cup|Unknown|14|eseoughtn stcally| +6915|AAAAAAAACALBAAAA|2000-10-27||Poor movements exist keys; armed, red events go also. Seq|93.81|31.89|4002001|importoamalg #2|2|fragrances|1|Women|41|ableoughtation|medium|063019984155pink2640|lime|Lb|Unknown|21|antioughtn stcally| +6916|AAAAAAAAEALBAAAA|1997-10-27|1999-10-27|Worldwide united items put again seriously fr|8.40|7.30|1003001|exportiamalg #1|3|maternity|1|Women|52|ableanti|petite|142563731919sandy201|royal|Tbl|Unknown|38|callyoughtn stcally| +6917|AAAAAAAAEALBAAAA|1999-10-28|2001-10-26|Blind, available years ought to forget political films. Afraid days specify such as a lips. Employees may not look never enough average feet. Large, other attacks will not take regul|19.53|16.60|9004006|edu packmaxi #6|4|entertainments|9|Books|52|ableanti|N/A|46001066smoke7981270|smoke|Pallet|Unknown|12|ationoughtn stcally| +6918|AAAAAAAAEALBAAAA|2001-10-27||Parties pass either conditions. Neighbouring, dry miles should begin australian trades. Real publishers observe elsewhere likely skills; appropriate funds will not feel from a situations. Barely m|0.28|0.21|9004006|edu packimporto #1|4|sports-apparel|2|Men|52|ableanti|medium|46001066smoke7981270|brown|Oz|Unknown|71|eingoughtn stcally| +6919|AAAAAAAAHALBAAAA|1997-10-27||Occasionally united figures must not |5.89|3.76|4001002|amalgedu pack #2|1|womens|4|Shoes|500|barbaranti|small|08powder117753469141|salmon|Oz|Unknown|1|n stoughtn stcally| +6920|AAAAAAAAIALBAAAA|1997-10-27|2000-10-26|Extreme, responsible students ought|1.63|0.65|10001002|amalgunivamalg #2|1|cameras|10|Electronics|770|barationation|N/A|7064639230573yellow7|puff|Ounce|Unknown|25|barablen stcally| +6921|AAAAAAAAIALBAAAA|2000-10-27||Extreme, responsible students ought|1.94|0.65|10001002|edu packscholar #2|1|classical|5|Music|770|barationation|N/A|7064639230573yellow7|lawn|Dram|Unknown|40|oughtablen stcally| +6922|AAAAAAAAKALBAAAA|1997-10-27|1999-10-27|Shoulders talk a little essential kinds. Stories make for a months. Most competitive areas think away also global tools. Real differences like also over a device|3.09|2.71|7015003|scholarnameless #3|15|tables|7|Home|599|n stn stanti|N/A|16206372855231steel8|spring|Tbl|Unknown|9|ableablen stcally| +6923|AAAAAAAAKALBAAAA|1999-10-28|2001-10-26|Shoulders talk a little essential kinds. Stories make for a months. Most competitive areas think away also global tools. Real differences like also over a device|52.95|2.71|7015003|namelessmaxi #12|15|romance|9|Books|153|priantiought|N/A|16206372855231steel8|mint|Bunch|Unknown|56|priablen stcally| +6924|AAAAAAAAKALBAAAA|2001-10-27||Continuing, determined words |1.56|2.71|7015003|brandnameless #5|7|hockey|8|Sports|153|priantiought|N/A|863955tan32184857979|pale|Tbl|Unknown|7|eseablen stcally| +6925|AAAAAAAANALBAAAA|1997-10-27||Unlikely, assist|5.98|2.09|6010006|univbrand #6|10|jewelry boxes|6|Jewelry|649|n stesecally|N/A|9442534138645peru388|navy|Ounce|Unknown|33|antiablen stcally| +6926|AAAAAAAAOALBAAAA|1997-10-27|2000-10-26|Breasts might not know other details. Coming, national matters change slowly important f|3.20|2.88|6009007|maxicorp #7|9|womens watch|6|Jewelry|82|ableeing|N/A|26447honeydew6550580|papaya|Carton|Unknown|29|callyablen stcally| +6927|AAAAAAAAOALBAAAA|2000-10-27||Altogether simple grounds ought to|9.45|2.88|6009007|importoimporto #2|9|shirts|2|Men|16|callyought|extra large|745088maroon46497217|wheat|Tbl|Unknown|40|ationablen stcally| +6928|AAAAAAAAABLBAAAA|1997-10-27|1999-10-27|Similar, different weapons used to see common feelings; good strang|5.48|2.13|10011010|amalgamalgamalg #10|11|disk drives|10|Electronics|387|ationeingpri|N/A|76244884plum52042580|slate|Box|Unknown|40|eingablen stcally| +6929|AAAAAAAAABLBAAAA|1999-10-28|2001-10-26|Tired, parliamentary standards provide associated, ha|6.74|5.25|6009002|maxicorp #2|9|womens watch|6|Jewelry|387|ationeingpri|N/A|76244884plum52042580|cornsilk|Dram|Unknown|68|n stablen stcally| +6930|AAAAAAAAABLBAAAA|2001-10-27||Tired, parliamentary standards provide associated, ha|7.34|5.25|5002001|importoscholar #1|9|country|5|Music|148|eingeseought|N/A|76244884plum52042580|puff|Case|Unknown|30|barprin stcally| +6931|AAAAAAAADBLBAAAA|1997-10-27||British, accurate objects move.|7.59|4.70|7004002|edu packbrand #2|4|curtains/drapes|7|Home|8|eing|N/A|574872156peach396792|plum|Bunch|Unknown|31|oughtprin stcally| +6932|AAAAAAAAEBLBAAAA|1997-10-27|2000-10-26|As pri|3.16|2.62|10007014|brandunivamalg #14|7|personal|10|Electronics|47|ationese|N/A|72008magenta66747788|turquoise|Case|Unknown|33|ableprin stcally| +6933|AAAAAAAAEBLBAAAA|2000-10-27||As pri|9.43|6.88|10007014|importoscholar #2|2|country|5|Music|4|ationese|N/A|6629382saddle8201351|grey|Lb|Unknown|23|priprin stcally| +6934|AAAAAAAAGBLBAAAA|1997-10-27|1999-10-27|Good, excellent forms may not date however empty, full elements. Here political moves find at the weapons. Charming, simple problems try. Popular, personal dates demons|4.77|2.38|3003001|exportiexporti #1|3|toddlers|3|Children|70|baration|large|peru8075089201537363|hot|Pound|Unknown|46|eseprin stcally| +6935|AAAAAAAAGBLBAAAA|1999-10-28|2001-10-26|Normal points can seem increasingly northern eyes. General, limit|4.62|2.44|3003001|edu packnameless #6|14|glassware|7|Home|70|baration|N/A|peru8075089201537363|plum|Bundle|Unknown|80|antiprin stcally| +6936|AAAAAAAAGBLBAAAA|2001-10-27||Large, adjacent processes may not resort probably by a shareholders. Libraries|3.78|2.44|3003001|corpbrand #1|16|consignment|6|Jewelry|70|baration|N/A|1293044forest0768216|ghost|Unknown|Unknown|20|callyprin stcally| +6937|AAAAAAAAJBLBAAAA|1997-10-27||Things should take sometimes detailed, famous features. Bod|0.70|0.41|3001002|amalgexporti #2|1|newborn|3|Children|277|ationationable|petite|98727869rose59420020|snow|Box|Unknown|65|ationprin stcally| +6938|AAAAAAAAKBLBAAAA|1997-10-27|2000-10-26|Essential, common women cannot accommodate fun, main aspects. Hours would not make at a children. Kids used to cover once with a flats. Even large sons mi|48.40|25.16|10008003|namelessunivamalg #3|8|scanners|10|Electronics|787|ationeingation|N/A|44982777thistle08549|moccasin|Pallet|Unknown|26|eingprin stcally| +6939|AAAAAAAAKBLBAAAA|2000-10-27||Early, familiar users should not dare. Social, ordinary patients start again indeed specific |46.25|40.70|10008003|importoscholar #2|8|country|5|Music|787|ationeingation|N/A|44982777thistle08549|lace|Unknown|Unknown|19|n stprin stcally| +6940|AAAAAAAAMBLBAAAA|1997-10-27|1999-10-27|Studies rus|0.79|0.44|10001005|amalgunivamalg #5|1|cameras|10|Electronics|310|baroughtpri|N/A|powder33845920805208|moccasin|Bunch|Unknown|27|baresen stcally| +6941|AAAAAAAAMBLBAAAA|1999-10-28|2001-10-26|Studies rus|4.49|0.44|10001005|exportiamalg #2|3|maternity|1|Women|560|baroughtpri|large|45088pink52359469715|papaya|Bundle|Unknown|41|oughtesen stcally| +6942|AAAAAAAAMBLBAAAA|2001-10-27||Studies rus|54.64|19.67|10001005|amalgbrand #7|11|semi-precious|6|Jewelry|117|ationoughtought|N/A|45088pink52359469715|powder|Dozen|Unknown|99|ableesen stcally| +6943|AAAAAAAAPBLBAAAA|1997-10-27||Rights could live schools. German developments give things; social profits discover. Professional sources form. |10.22|5.11|6012008|importobrand #8|12|costume|6|Jewelry|220|barableable|N/A|67191068512thistle55|saddle|Unknown|Unknown|31|priesen stcally| +6944|AAAAAAAAACLBAAAA|1997-10-27|2000-10-26|Actual, grey hands giv|5.67|3.85|8011005|amalgmaxi #5|11|archery|8|Sports|584|eseeinganti|N/A|87587500orchid209426|cornsilk|Bunch|Unknown|10|eseesen stcally| +6945|AAAAAAAAACLBAAAA|2000-10-27||Democratic, evident customers bring earlier ha|7.50|6.45|5001002|amalgscholar #2|1|rock|5|Music|132|eseeinganti|N/A|87587500orchid209426|tomato|Unknown|Unknown|7|antiesen stcally| +6946|AAAAAAAACCLBAAAA|1997-10-27|1999-10-27|Real, trying prices control. Usually social priests confirm groups; significant days bear indeed other rules. More small colours could allow bette|6.77|3.38|4001001|amalgedu pack #1|1|womens|4|Shoes|100|barbarought|small|88808midnight5516684|misty|Oz|Unknown|5|callyesen stcally| +6947|AAAAAAAACCLBAAAA|1999-10-28|2001-10-26|Real, trying prices control. Usually social priests confirm groups; significant days bear indeed other rules. More small colours could allow bette|2.92|3.38|6001004|amalgcorp #4|1|birdal|6|Jewelry|100|barbarought|N/A|7linen94579041316830|rose|Pound|Unknown|29|ationesen stcally| +6948|AAAAAAAACCLBAAAA|2001-10-27||Real, trying prices control. Usually social priests confirm groups; significant days bear indeed other rules. More small colours could allow bette|4.38|3.38|6001004|importoamalg #1|1|fragrances|1|Women|100|barbarought|large|7linen94579041316830|salmon|Tsp|Unknown|2|eingesen stcally| +6949|AAAAAAAAFCLBAAAA|1997-10-27||Lines say just various problems. Aware, unique practices may think quite lakes. Only dramatic consequences give hard so black eyes. Tired, warm nurses will emphasize wonderful, shor|1.50|1.32|3002002|importoexporti #2|2|infants|3|Children|578|eingationanti|small|12467blush3314616602|metallic|Ounce|Unknown|19|n stesen stcally| +6950|AAAAAAAAGCLBAAAA|1997-10-27|2000-10-26|Sciences make here british, human laws. Possible effects used to feel usually from a years. Rough, japanese guidelines should ask |4.13|2.23|10010015|univamalgamalg #15|10|memory|10|Electronics|22|ableable|N/A|585127pink6494582632|violet|Oz|Unknown|55|barantin stcally| +6951|AAAAAAAAGCLBAAAA|2000-10-27||Sciences make here british, human laws. Possible effects used to feel usually from a years. Rough, japanese guidelines should ask |39.35|27.54|10003016|exportiunivamalg #16|3|dvd/vcr players|10|Electronics|22|ableable|N/A|585127pink6494582632|lemon|Tsp|Unknown|56|oughtantin stcally| +6952|AAAAAAAAICLBAAAA|1997-10-27|1999-10-27|Special words say little supreme, bare chapte|2.98|2.47|9008005|namelessmaxi #5|8|romance|9|Books|100|barbarought|N/A|9920411975tomato4173|puff|N/A|Unknown|56|ableantin stcally| +6953|AAAAAAAAICLBAAAA|1999-10-28|2001-10-26|Special words say little supreme, bare chapte|0.51|0.18|10005001|scholarunivamalg #1|5|karoke|10|Electronics|100|barbarought|N/A|9920411975tomato4173|ghost|Box|Unknown|14|priantin stcally| +6954|AAAAAAAAICLBAAAA|2001-10-27||Appropriate feet write right new conditions. Sensible sites cannot care events. Fingers retain certain runs. Carefully good conventions sign much eventual companies. Reg|3.57|1.42|3002001|importoexporti #1|5|infants|3|Children|100|barbarought|petite|9920411975tomato4173|moccasin|Dozen|Unknown|12|eseantin stcally| +6955|AAAAAAAALCLBAAAA|1997-10-27||Quickly vital descriptions drink almost gardens. Green hands used to assist with a projects. Exactly crazy statements should try concerned results. Courses open just in a causes. Differ|6.13|5.45|9010008|univunivamalg #8|10|travel|9|Books|106|callybarought|N/A|07669turquoise445157|turquoise|Dozen|Unknown|46|antiantin stcally| +6956|AAAAAAAAMCLBAAAA|1997-10-27|2000-10-26|Slow financial feet should not delay able, significant services. Old, possible users render then only final years. Agreem|9.51|5.99|1002001|importoamalg #1|2|fragrances|1|Women|504|esebaranti|extra large|8230pale280565596155|white|Gross|Unknown|37|callyantin stcally| +6957|AAAAAAAAMCLBAAAA|2000-10-27||Terms come in full around critical opponents. Too legislative holidays |1.09|0.62|8012008|importomaxi #8|12|guns|8|Sports|64|esecally|N/A|402459928seashell896|lavender|Pallet|Unknown|26|ationantin stcally| +6958|AAAAAAAAOCLBAAAA|1997-10-27|1999-10-27|Markedly modern thoughts could safeguard questions. Considerably pure levels would contro|0.39|0.35|6014007|edu packbrand #7|14|estate|6|Jewelry|418|eingoughtese|N/A|901194018sandy223189|maroon|Cup|Unknown|23|eingantin stcally| +6959|AAAAAAAAOCLBAAAA|1999-10-28|2001-10-26|Markedly modern thoughts could safeguard questions. Considerably pure levels would contro|0.24|0.35|6014007|amalgunivamalg #7|14|cameras|10|Electronics|418|eingoughtese|N/A|5956534704sky7464728|purple|Ounce|Unknown|46|n stantin stcally| +6960|AAAAAAAAOCLBAAAA|2001-10-27||References telephone boots. Unaware clients produce directly brown operators. Weeks|1.02|0.34|7007001|brandbrand #1|7|decor|7|Home|418|eingoughtese|N/A|423147898419papaya26|tan|Pallet|Unknown|12|barcallyn stcally| +6961|AAAAAAAABDLBAAAA|1997-10-27||Similar regu|4.57|3.10|5004002|edu packscholar #2|4|classical|5|Music|446|callyeseese|N/A|1wheat83814505341861|gainsboro|Each|Unknown|35|oughtcallyn stcally| +6962|AAAAAAAACDLBAAAA|1997-10-27|2000-10-26|Troops should not hit less organic students. Legal, female feelings present front periods. |4.95|3.96|3002001|importoexporti #1|2|infants|3|Children|597|ationn stanti|medium|60chartreuse17457423|medium|Cup|Unknown|59|ablecallyn stcally| +6963|AAAAAAAACDLBAAAA|2000-10-27||Exp|8.41|4.96|7013004|exportinameless #4|2|wallpaper|7|Home|597|ationn stanti|N/A|60chartreuse17457423|almond|Unknown|Unknown|94|pricallyn stcally| +6964|AAAAAAAAEDLBAAAA|1997-10-27|1999-10-27|Women take even reasonable causes; physical, medium buildings contain great operations. Ever other nights pin|75.25|63.96|7010005|univnameless #5|10|flatware|7|Home|582|ableeinganti|N/A|698puff1431524104664|smoke|Gross|Unknown|44|esecallyn stcally| +6965|AAAAAAAAEDLBAAAA|1999-10-28|2001-10-26|Private purposes stroke only current years; japanese payments used to come thus from a words. Areas wo|8.22|3.69|7010005|edu packexporti #2|4|school-uniforms|3|Children|232|ableeinganti|petite|698puff1431524104664|lawn|Pallet|Unknown|66|anticallyn stcally| +6966|AAAAAAAAEDLBAAAA|2001-10-27||Primary, hungry e|0.40|3.69|10006014|corpunivamalg #14|6|musical|10|Electronics|440|bareseese|N/A|698puff1431524104664|wheat|Dozen|Unknown|3|callycallyn stcally| +6967|AAAAAAAAHDLBAAAA|1997-10-27||Sweet points involve enough. Frequently flexible accounts attract quietly to the features. Hence primitive farmers must not perform with a blocks. Just excessive fields kick further simple, high citi|3.61|1.26|6012008|importobrand #8|12|costume|6|Jewelry|199|n stn stought|N/A|556595430goldenrod87|peru|Carton|Unknown|72|ationcallyn stcally| +6968|AAAAAAAAIDLBAAAA|1997-10-27|2000-10-26|Lines should like explicitly religious instruments. Fun buildings treat always officia|0.21|0.11|10012016|importoamalgamalg #16|12|monitors|10|Electronics|27|ationable|N/A|aquamarine2380128582|snow|Bundle|Unknown|9|eingcallyn stcally| +6969|AAAAAAAAIDLBAAAA|2000-10-27||Professional children develop inc cases. Situations can pay. Past, competitive issues understand other, substantial subjects; for long canadian kinds ought to eat for example theoretical, inte|3.30|1.22|10012016|amalgimporto #2|12|accessories|2|Men|167|ationcallyought|extra large|1694tomato3755277016|seashell|Bunch|Unknown|1|n stcallyn stcally| +6970|AAAAAAAAKDLBAAAA|1997-10-27|1999-10-27|Exact charges cannot result local conditions. Available savings shall put quite correctly local users. Ever positive groups move; els|2.56|1.99|5004001|edu packscholar #1|4|classical|5|Music|87|ationeing|N/A|15316451521921sandy2|tomato|Gram|Unknown|22|barationn stcally| +6971|AAAAAAAAKDLBAAAA|1999-10-28|2001-10-26|Exact charges cannot result local conditions. Available savings shall put quite correctly local users. Ever positive groups move; els|3.27|1.99|9009012|maximaxi #12|4|science|9|Books|87|ationeing|N/A|1898powder5370834691|yellow|Cup|Unknown|57|oughtationn stcally| +6972|AAAAAAAAKDLBAAAA|2001-10-27||Constitutional ingredients deliver. Marked cases shall intend of course reasons. Smart, real definitions make above in a women; medical principles used to live to the resources. Others soften soo|4.36|1.99|2003001|exportiimporto #1|3|pants|2|Men|87|ationeing|small|90sandy7046870636711|pink|Unknown|Unknown|27|ableationn stcally| +6973|AAAAAAAANDLBAAAA|1997-10-27||White posts find all on a attempts. Very formidable governments may not provoke then effective, average things. Foreig|33.28|14.31|4003002|exportiedu pack #2|3|kids|4|Shoes|239|n stpriable|large|6280799433smoke80816|yellow|Tbl|Unknown|3|priationn stcally| +6974|AAAAAAAAODLBAAAA|1997-10-27|2000-10-26|Parents may get also. Schools may finish softly additional sides; guns walk thus clear, military procedures. Urban trees walk g|1.55|0.75|2004001|edu packimporto #1|4|sports-apparel|2|Men|289|n steingable|petite|5174152papaya4805926|peru|Cup|Unknown|45|eseationn stcally| +6975|AAAAAAAAODLBAAAA|2000-10-27||Approaches ask personal s|9.30|0.75|2004001|amalgexporti #2|4|newborn|3|Children|289|n steingable|large|5174152papaya4805926|salmon|Ounce|Unknown|18|antiationn stcally| +6976|AAAAAAAAAELBAAAA|1997-10-27|1999-10-27|Minutes end in a members. Other, cultural visitors could say other, available children. Individual arms consider over boxes. Clearly voluntary sisters allow huge women. There small alternative|9.63|6.74|10012007|importoamalgamalg #7|12|monitors|10|Electronics|309|n stbarpri|N/A|70845powder368236190|tan|Bundle|Unknown|21|callyationn stcally| +6977|AAAAAAAAAELBAAAA|1999-10-28|2001-10-26|Constant, recent systems shall not want ministers. Somehow attractive ideas would assume never for a customers. Therefore local practitioners|0.85|0.26|10012007|amalgexporti #2|12|newborn|3|Children|435|n stbarpri|medium|561136681snow1901089|wheat|Dram|Unknown|27|ationationn stcally| +6978|AAAAAAAAAELBAAAA|2001-10-27||Really patient resources ought to handle plainly chinese, marked questions. Urban, white products allow. Conditions reach too in a offices. English, old things would make as; various price|7.86|2.82|10012007|exporticorp #3|12|gold|6|Jewelry|435|n stbarpri|N/A|7792214493metallic42|white|Oz|Unknown|1|eingationn stcally| +6979|AAAAAAAADELBAAAA|1997-10-27||Quickly hungry bills ought to cope errors. Professional pp. pay americans. Days allow. Ver|0.36|0.31|7008010|namelessbrand #10|8|lighting|7|Home|278|eingationable|N/A|4877592660peru588471|azure|Tsp|Unknown|71|n stationn stcally| +6980|AAAAAAAAEELBAAAA|1997-10-27|2000-10-26|Nice attempts will involve never grand sheets. Soviet, possible houses allow sensitive needs. Effects cause neverth|3.13|1.31|2004001|edu packimporto #1|4|sports-apparel|2|Men|943|priesen st|medium|3goldenrod9807400218|sky|Cup|Unknown|23|bareingn stcally| +6981|AAAAAAAAEELBAAAA|2000-10-27||Nice attempts will involve never grand sheets. Soviet, possible houses allow sensitive needs. Effects cause neverth|42.30|1.31|2004001|amalgexporti #2|1|newborn|3|Children|943|priesen st|small|43white5789034734086|lawn|Box|Unknown|27|oughteingn stcally| +6982|AAAAAAAAGELBAAAA|1997-10-27|1999-10-27|Increased to|1.61|1.40|3003001|exportiexporti #1|3|toddlers|3|Children|172|ableationought|large|135749944175lime6783|papaya|Pound|Unknown|28|ableeingn stcally| +6983|AAAAAAAAGELBAAAA|1999-10-28|2001-10-26|Increased to|0.37|1.40|3003001|importoedu pack #2|3|mens|4|Shoes|172|ableationought|small|135749944175lime6783|plum|Lb|Unknown|31|prieingn stcally| +6984|AAAAAAAAGELBAAAA|2001-10-27||Increased to|2.99|2.45|6012001|importobrand #1|12|costume|6|Jewelry|172|ableationought|N/A|41900450orchid187297|rose|Dozen|Unknown|51|eseeingn stcally| +6985|AAAAAAAAJELBAAAA|1997-10-27||Other securities used to prevent again over domestic components; easy, detailed women would ans|4.58|2.74|4003002|exportiedu pack #2|3|kids|4|Shoes|523|priableanti|medium|461793pale5397523312|mint|Tsp|Unknown|17|antieingn stcally| +6986|AAAAAAAAKELBAAAA|1997-10-27|2000-10-26|However fair pressures realise twice walls. Days bring both. Dreadful syste|17.28|9.67|9004003|edu packmaxi #3|4|entertainments|9|Books|410|baroughtese|N/A|1157almond2091427171|royal|Oz|Unknown|26|callyeingn stcally| +6987|AAAAAAAAKELBAAAA|2000-10-27||Universal, other observations shall test most men. Organisational matters should not take personal, electric millions. Slow, central men afford in a times. Slim, voluntary devices might sat|6.36|9.67|7010008|univnameless #8|4|flatware|7|Home|267|ationcallyable|N/A|97488803tomato319296|tan|N/A|Unknown|39|ationeingn stcally| +6988|AAAAAAAAMELBAAAA|1997-10-27|1999-10-27|Clean, particular individuals might go frequently difficult police. Increasingly indirect studies manage. So international directions|6.37|2.42|3001001|amalgexporti #1|1|newborn|3|Children|24|eseable|petite|21461828wheat7267035|turquoise|Box|Unknown|8|eingeingn stcally| +6989|AAAAAAAAMELBAAAA|1999-10-28|2001-10-26|However fair countries may enjoy; voluntary exports may not grow almost current conferences; roman restaurants sha|8.90|2.42|8005010|scholarnameless #10|1|fitness|8|Sports|133|pripriought|N/A|21461828wheat7267035|navy|Pallet|Unknown|30|n steingn stcally| +6990|AAAAAAAAMELBAAAA|2001-10-27||However fair countries may enjoy; voluntary exports may not grow almost current conferences; roman restaurants sha|15.61|10.92|4004001|edu packedu pack #1|4|athletic|4|Shoes|133|pripriought|medium|302saddle32917232866|smoke|Gross|Unknown|19|barn stn stcally| +6991|AAAAAAAAPELBAAAA|1997-10-27||Popular, overall members take etc relations. Colleges|3.36|1.34|1003002|exportiamalg #2|3|maternity|1|Women|324|eseablepri|medium|6443504lime260732030|hot|Box|Unknown|6|oughtn stn stcally| +6992|AAAAAAAAAFLBAAAA|1997-10-27|2000-10-26|Blue flowers may not stand fresh name|7.37|5.08|2003001|exportiimporto #1|3|pants|2|Men|27|ationable|medium|8179630950230salmon0|steel|Lb|Unknown|36|ablen stn stcally| +6993|AAAAAAAAAFLBAAAA|2000-10-27||Certainly possible statements ought to give there features. Almost cruel eyes may not point colleges. Hard hours free carefully in support of the points. Large cases may qualify yesterda|2.67|1.78|6007002|brandcorp #2|3|pendants|6|Jewelry|27|ationable|N/A|8179630950230salmon0|almond|Unknown|Unknown|100|prin stn stcally| +6994|AAAAAAAACFLBAAAA|1997-10-27|1999-10-27|Never voluntary eggs would not add from the others. Clear schools go occasionally good, canadian critics; fairly simple var|3.78|3.25|6002003|importocorp #3|2|diamonds|6|Jewelry|858|eingantieing|N/A|3909730502693pink333|peru|Carton|Unknown|17|esen stn stcally| +6995|AAAAAAAACFLBAAAA|1999-10-28|2001-10-26|So whole studies move important, unknown groups. Married pieces revive. Healthy, classical girls watch loudly reasonable win|2.54|1.24|1002002|importoamalg #2|2|fragrances|1|Women|459|eingantieing|large|684871028hot52432084|magenta|Case|Unknown|18|antin stn stcally| +6996|AAAAAAAACFLBAAAA|2001-10-27||Fun, hard items like for a attacks. Irish, deep connections get as ready guards. Now flat children stop problems. Easily oth|5.56|1.24|1002002|namelessbrand #7|2|lighting|7|Home|459|eingantieing|N/A|48615922white7993450|pale|Lb|Unknown|1|callyn stn stcally| +6997|AAAAAAAAFFLBAAAA|1997-10-27||Technica|5.48|4.00|2001002|amalgimporto #2|1|accessories|2|Men|146|callyeseought|N/A|033620cornflower1087|goldenrod|Pallet|Unknown|26|ationn stn stcally| +6998|AAAAAAAAGFLBAAAA|1997-10-27|2000-10-26|Young, old problems would produce left horses. Then sharp interests might stay once bad, continuous effects. Great planes might arise quite on a detectives. Pupi|60.68|44.29|1004001|edu packamalg #1|4|swimwear|1|Women|166|callycallyought|large|99901088white1974680|powder|Dozen|Unknown|28|eingn stn stcally| +6999|AAAAAAAAGFLBAAAA|2000-10-27||Young, old problems would produce left horses. Then sharp interests might stay once bad, continuous effects. Great planes might arise quite on a detectives. Pupi|9.67|4.44|1004001|edu packamalg #2|4|swimwear|1|Women|166|callycallyought|large|99901088white1974680|moccasin|Ounce|Unknown|6|n stn stn stcally| +7000|AAAAAAAAIFLBAAAA|1997-10-27|1999-10-27|Personal schools find additionally later international victims. As positive ch|4.45|1.82|1002001|importoamalg #1|2|fragrances|1|Women|75|antiation|medium|904520purple85241781|thistle|Ounce|Unknown|32|barbarbaration| +7001|AAAAAAAAIFLBAAAA|1999-10-28|2001-10-26|Underlying children help academic times. Ago different|7.43|6.38|1002001|maximaxi #6|2|science|9|Books|75|antiation|N/A|897995violet57730054|saddle|Cup|Unknown|41|oughtbarbaration| +7002|AAAAAAAAIFLBAAAA|2001-10-27||Vitally equal problems might find later; important stars should get as wooden windows; exports cannot stand then other, wise rates. Relationships must |6.93|5.61|1002001|importoamalg #1|2|fragrances|1|Women|152|ableantiought|economy|226106722065peach731|sienna|Gross|Unknown|49|ablebarbaration| +7003|AAAAAAAALFLBAAAA|1997-10-27||Finally reasonable lengths could use recent goods; quite orthodox services could not run efficiently really major sides. Ethical members should take also by the researchers. Sex|7.42|4.74|4002002|importoedu pack #2|2|mens|4|Shoes|98|eingn st|extra large|85pale77262218624536|ivory|Each|Unknown|49|pribarbaration| +7004|AAAAAAAAMFLBAAAA|1997-10-27|2000-10-26|Doctors give enough; only evident discussions cannot help a bit readily new men. |0.10|0.04|3004001|edu packexporti #1|4|school-uniforms|3|Children|533|priprianti|petite|92186cornsilk5954920|green|Each|Unknown|85|esebarbaration| +7005|AAAAAAAAMFLBAAAA|2000-10-27||Just young points concentrate naturally i|0.82|0.66|3004001|amalgedu pack #2|4|womens|4|Shoes|533|priprianti|petite|92186cornsilk5954920|drab|Case|Unknown|8|antibarbaration| +7006|AAAAAAAAOFLBAAAA|1997-10-27|1999-10-27|Equally brief figures used to come never regular sui|0.28|0.24|3001001|amalgexporti #1|1|newborn|3|Children|593|prin stanti|medium|04midnight1195178240|slate|N/A|Unknown|55|callybarbaration| +7007|AAAAAAAAOFLBAAAA|1999-10-28|2001-10-26|Equally brief figures used to come never regular sui|5.49|0.24|9006012|corpmaxi #12|6|parenting|9|Books|593|prin stanti|N/A|04midnight1195178240|rosy|Gross|Unknown|15|ationbarbaration| +7008|AAAAAAAAOFLBAAAA|2001-10-27||Decisions look in a months. Little changes reproduce nearly against a colleagues. Black, rig|8.89|0.24|2002001|importoimporto #1|2|shirts|2|Men|169|prin stanti|medium|4825tan1642390455416|peru|Bundle|Unknown|45|eingbarbaration| +7009|AAAAAAAABGLBAAAA|1997-10-27||Months make just authorities. Accurate, original sports disagree recently in a rebels. Items sound also. Short years must not develop once suitable chains; |9.45|7.37|2004002|edu packimporto #2|4|sports-apparel|2|Men|258|eingantiable|large|red14709942213323931|sky|Tbl|Unknown|9|n stbarbaration| +7010|AAAAAAAACGLBAAAA|1997-10-27|2000-10-26|Dollars put unnecessarily extra details. Appropriate men ought to put; realistic findings will not permit. Video-taped things house feature|0.97|0.72|5002001|importoscholar #1|2|country|5|Music|58|einganti|N/A|490deep4523998020541|slate|N/A|Unknown|36|baroughtbaration| +7011|AAAAAAAACGLBAAAA|2000-10-27||Polls should not submit slightly even various profits. Accounts achieve just small, relative copies. As written things view except a stairs; moments look final practices. |19.73|15.58|1004002|edu packamalg #2|4|swimwear|1|Women|277|ationationable|medium|490deep4523998020541|royal|Case|Unknown|6|oughtoughtbaration| +7012|AAAAAAAAEGLBAAAA|1997-10-27|1999-10-27|Offenders might grant; others could not make too alone mental women. More different pupils know never o|55.56|32.22|5004001|edu packscholar #1|4|classical|5|Music|105|antibarought|N/A|690059050065khaki096|peru|Bunch|Unknown|30|ableoughtbaration| +7013|AAAAAAAAEGLBAAAA|1999-10-28|2001-10-26|Offenders might grant; others could not make too alone mental women. More different pupils know never o|0.70|0.39|5004001|exportimaxi #6|3|computers|9|Books|105|antibarought|N/A|6906065875sky7986826|smoke|Pound|Unknown|30|prioughtbaration| +7014|AAAAAAAAEGLBAAAA|2001-10-27||Girls join again into a ages; boards produce. Recent, mean hours should depend maybe symbolic subjects. Sales see good factors; things|1.08|0.72|1001001|amalgamalg #1|3|dresses|1|Women|984|antibarought|large|38988364honeydew3223|thistle|Pallet|Unknown|32|eseoughtbaration| +7015|AAAAAAAAHGLBAAAA|1997-10-27||New alternatives point from a partners. Leaders would not put pp.; practices wear only to the areas. Most previous cattle know very local families. A little ex|9.16|4.03|4002002|importoedu pack #2|2|mens|4|Shoes|573|priationanti|medium|0239673003dark282527|thistle|Lb|Unknown|19|antioughtbaration| +7016|AAAAAAAAIGLBAAAA|1997-10-27|2000-10-26|Tory trains tell only old, good supporters. Patterns like only nations. Still important proportions respond as foreign, professional categories. Studies might develop onwards certain|9.41|4.32|7009007|maxibrand #7|9|mattresses|7|Home|584|eseeinganti|N/A|37807115560saddle030|purple|Dram|Unknown|46|callyoughtbaration| +7017|AAAAAAAAIGLBAAAA|2000-10-27||Tory trains tell only old, good supporters. Patterns like only nations. Still important proportions respond as foreign, professional categories. Studies might develop onwards certain|3.04|2.52|7002008|importobrand #8|2|bedding|7|Home|584|eseeinganti|N/A|37807115560saddle030|red|Ounce|Unknown|48|ationoughtbaration| +7018|AAAAAAAAKGLBAAAA|1997-10-27|1999-10-27|Clear approaches should take alone daughters. Complex, small materials provide also by a groups. Americans discuss so. Cons|3.34|2.83|9006011|corpmaxi #11|6|parenting|9|Books|727|ationableation|N/A|807190379orchid54118|plum|Gram|Unknown|12|eingoughtbaration| +7019|AAAAAAAAKGLBAAAA|1999-10-28|2001-10-26|Too actual chips will happen. Social, international years say only essential, hig|7.96|5.17|9006011|importonameless #10|6|baseball|8|Sports|866|ationableation|N/A|807190379orchid54118|turquoise|Gram|Unknown|40|n stoughtbaration| +7020|AAAAAAAAKGLBAAAA|2001-10-27||Too actual chips will happen. Social, international years say only essential, hig|8.51|5.17|9006011|scholarmaxi #1|15|fishing|8|Sports|866|ationableation|N/A|807190379orchid54118|rose|Ton|Unknown|76|barablebaration| +7021|AAAAAAAANGLBAAAA|1997-10-27||Kinds used to suffice decisively. Poor versions believe actually well good readers. Even labour ties could know further days. More final stars encourage again dr|4.85|3.00|3003002|exportiexporti #2|3|toddlers|3|Children|77|ationation|small|395444950979salmon96|tomato|Pound|Unknown|32|oughtablebaration| +7022|AAAAAAAAOGLBAAAA|1997-10-27|2000-10-26|Too supreme refugees will invade also of course little teeth. Entirely popular schemes may see else less positive memories. Wives may inquire well processes. Available, true parties|6.43|3.02|8009003|maxinameless #3|9|optics|8|Sports|93|prin st|N/A|827646496994176plum4|salmon|Dram|Unknown|30|ableablebaration| +7023|AAAAAAAAOGLBAAAA|2000-10-27||Too supreme refugees will invade also of course little teeth. Entirely popular schemes may see else less positive memories. Wives may inquire well processes. Available, true parties|3.70|1.33|8009003|edu packscholar #2|4|classical|5|Music|589|n steinganti|N/A|827646496994176plum4|purple|Dozen|Unknown|2|priablebaration| +7024|AAAAAAAAAHLBAAAA|1997-10-27|1999-10-27|Extended farmers return personal, free sections. Inc, bright years amend black compan|8.93|8.03|4001001|amalgedu pack #1|1|womens|4|Shoes|180|bareingought|petite|035224543187plum5400|lawn|Cup|Unknown|66|eseablebaration| +7025|AAAAAAAAAHLBAAAA|1999-10-28|2001-10-26|Like, scottish artists should think earlier sexual, resulting years. Soviet, national women should lead now angry costs. Other leve|74.03|50.34|1003002|exportiamalg #2|1|maternity|1|Women|180|bareingought|extra large|035224543187plum5400|mint|Tsp|Unknown|15|antiablebaration| +7026|AAAAAAAAAHLBAAAA|2001-10-27||Like, scottish artists should think earlier sexual, resulting years. Soviet, national women should lead now angry costs. Other leve|0.99|0.85|1003002|edu packnameless #7|1|camping|8|Sports|150|barantiought|N/A|035224543187plum5400|saddle|Pound|Unknown|28|callyablebaration| +7027|AAAAAAAADHLBAAAA|1997-10-27||Trousers understand patients. Less royal associa|63.39|22.82|4002002|importoedu pack #2|2|mens|4|Shoes|357|ationantipri|medium|887243452460plum1205|green|Bundle|Unknown|4|ationablebaration| +7028|AAAAAAAAEHLBAAAA|1997-10-27|2000-10-26|Western, local cars comply as at a lips. Good, brown lads could prove alone, early committees. Patterns must sign ev|1.22|0.42|4002001|importoedu pack #1|2|mens|4|Shoes|107|ationbarought|medium|1049950385805yellow5|white|Case|Unknown|50|eingablebaration| +7029|AAAAAAAAEHLBAAAA|2000-10-27||Western, local cars comply as at a lips. Good, brown lads could prove alone, early committees. Patterns must sign ev|4.71|3.57|4002001|importoamalg #2|2|fragrances|1|Women|185|antieingought|small|1049950385805yellow5|plum|Gross|Unknown|52|n stablebaration| +7030|AAAAAAAAGHLBAAAA|1997-10-27|1999-10-27|Beautiful, old examples might not involve in a resources; moreover national islands may not enhance too famous autonomous functions; rates would work home straig|8.88|4.88|3004001|edu packexporti #1|4|school-uniforms|3|Children|445|antieseese|petite|327773859929misty057|peach|Carton|Unknown|2|barpribaration| +7031|AAAAAAAAGHLBAAAA|1999-10-28|2001-10-26|Likely, european pro|1.71|0.76|10010011|univamalgamalg #11|4|memory|10|Electronics|445|antieseese|N/A|327773859929misty057|magenta|Dozen|Unknown|54|oughtpribaration| +7032|AAAAAAAAGHLBAAAA|2001-10-27||Likely, european pro|81.16|0.76|2002001|importoimporto #1|2|shirts|2|Men|163|antieseese|N/A|327773859929misty057|purple|Bunch|Unknown|44|ablepribaration| +7033|AAAAAAAAJHLBAAAA|1997-10-27||Low temperatures would not browse houses; cultural stations cann|6.76|4.25|5002002|importoscholar #2|2|country|5|Music|574|eseationanti|N/A|3smoke07569341432519|peach|Pound|Unknown|14|pripribaration| +7034|AAAAAAAAKHLBAAAA|1997-10-27|2000-10-26|Thirdly classical others travel; details write|2.15|1.05|4002001|importoedu pack #1|2|mens|4|Shoes|751|oughtantiation|medium|8837208navajo5402058|peach|Each|Unknown|9|esepribaration| +7035|AAAAAAAAKHLBAAAA|2000-10-27||Odds will not get diseases. Others might mean very |7.12|1.05|5004002|edu packscholar #2|4|classical|5|Music|751|oughtantiation|N/A|1767189879044white03|navy|Unknown|Unknown|87|antipribaration| +7036|AAAAAAAAMHLBAAAA|1997-10-27|1999-10-27|Generations used to enable duly blue, heavy tests. Points may send net children. Acute, similar res|1.14|0.91|1003001|exportiamalg #1|3|maternity|1|Women|72|ableation|medium|5067220607white57028|sandy|Dram|Unknown|50|callypribaration| +7037|AAAAAAAAMHLBAAAA|1999-10-28|2001-10-26|Generations used to enable duly blue, heavy tests. Points may send net children. Acute, similar res|5.93|4.74|6012006|importobrand #6|3|costume|6|Jewelry|72|ableation|N/A|5067220607white57028|powder|Cup|Unknown|44|ationpribaration| +7038|AAAAAAAAMHLBAAAA|2001-10-27||Humans produce also frequently male options. General parts work separate, remaining models. V|3.32|4.74|6012006|importoimporto #1|2|shirts|2|Men|72|ableation|small|5067220607white57028|violet|Ounce|Unknown|41|eingpribaration| +7039|AAAAAAAAPHLBAAAA|1997-10-27||Too white boys must appear alike rural months. Ago agricultural documents may not find nowadays r|5.74|4.01|8002010|importonameless #10|2|baseball|8|Sports|321|oughtablepri|N/A|890916947pale5643930|pink|Cup|Unknown|3|n stpribaration| +7040|AAAAAAAAAILBAAAA|1997-10-27|2000-10-26|Important functions can offer rather items. Christian ears preserve therefore additional, new foods. Now whole men make only black,|2.76|1.29|7007001|brandbrand #1|7|decor|7|Home|173|priationought|N/A|31512plum69631794082|burlywood|Ton|Unknown|4|baresebaration| +7041|AAAAAAAAAILBAAAA|2000-10-27||Good ideas shall go furthermore schem|2.64|1.29|7007001|importounivamalg #10|12|home repair|9|Books|221|priationought|N/A|31512plum69631794082|magenta|Ounce|Unknown|35|oughtesebaration| +7042|AAAAAAAACILBAAAA|1997-10-27|1999-10-27|Fruits ensure well possible mark|66.58|56.59|1003001|exportiamalg #1|3|maternity|1|Women|373|priationpri|medium|5514535smoke56034632|sienna|Oz|Unknown|27|ableesebaration| +7043|AAAAAAAACILBAAAA|1999-10-28|2001-10-26|N|73.62|48.58|4003002|exportiedu pack #2|3|kids|4|Shoes|530|barprianti|medium|5514535smoke56034632|sandy|Pallet|Unknown|10|priesebaration| +7044|AAAAAAAACILBAAAA|2001-10-27||N|1.97|48.58|5004001|edu packscholar #1|3|classical|5|Music|57|barprianti|N/A|5514535smoke56034632|lawn|Pallet|Unknown|1|eseesebaration| +7045|AAAAAAAAFILBAAAA|1997-10-27||Thus front journals will not go hot, available sentences. Fun buyers could force plus a nee|5.66|3.45|2002002|importoimporto #2|2|shirts|2|Men|324|eseablepri|large|75sky335257447560939|white|Bundle|Unknown|94|antiesebaration| +7046|AAAAAAAAGILBAAAA|1997-10-27|2000-10-26|Appropriate, demanding points shall proceed therefore difficulties. Ideas shall not manage supreme proportions. Nerves represent eyes. Fair boys may prevent too; services can get today parts. O|4.52|3.79|6008007|namelesscorp #7|8|mens watch|6|Jewelry|576|callyationanti|N/A|67542452grey23741144|royal|Unknown|Unknown|44|callyesebaration| +7047|AAAAAAAAGILBAAAA|2000-10-27||Partly social interests used to remember at least true, different standards. Clean, independent dates ease services; high, signif|6.86|5.07|6008007|edu packexporti #2|4|school-uniforms|3|Children|576|callyationanti|medium|84049601powder321421|puff|Bundle|Unknown|40|ationesebaration| +7048|AAAAAAAAIILBAAAA|1997-10-27|1999-10-27|Then other metres could perm|0.27|0.12|4001001|amalgedu pack #1|1|womens|4|Shoes|12|ableought|petite|058537080cornsilk144|rosy|Unknown|Unknown|5|eingesebaration| +7049|AAAAAAAAIILBAAAA|1999-10-28|2001-10-26|American, o|4.06|1.50|4001001|edu packedu pack #2|1|athletic|4|Shoes|12|ableought|extra large|3047smoke14118680568|plum|Bunch|Unknown|24|n stesebaration| +7050|AAAAAAAAIILBAAAA|2001-10-27||American, o|2.72|1.50|4001001|edu packbrand #1|1|curtains/drapes|7|Home|140|bareseought|N/A|99986ghost8653208289|turquoise|Each|Unknown|70|barantibaration| +7051|AAAAAAAALILBAAAA|1997-10-27||Emissions ban industrial eyes; warm, old drinks know enough young, soft things|0.34|0.17|5004002|edu packscholar #2|4|classical|5|Music|602|ablebarcally|N/A|2824347943papaya6532|wheat|Gram|Unknown|4|oughtantibaration| +7052|AAAAAAAAMILBAAAA|1997-10-27|2000-10-26|Only intere|31.16|20.56|5004001|edu packscholar #1|4|classical|5|Music|43|priese|N/A|666906sienna32250546|rose|Dram|Unknown|37|ableantibaration| +7053|AAAAAAAAMILBAAAA|2000-10-27||Only intere|9.10|20.56|5004001|exportimaxi #10|3|computers|9|Books|43|priese|N/A|steel407639927142420|purple|Bunch|Unknown|91|priantibaration| +7054|AAAAAAAAOILBAAAA|1997-10-27|1999-10-27|Very able relations obtain so|1.79|1.48|10001017|amalgunivamalg #17|1|cameras|10|Electronics|102|ablebarought|N/A|467241266381slate309|chartreuse|Bundle|Unknown|39|eseantibaration| +7055|AAAAAAAAOILBAAAA|1999-10-28|2001-10-26|Very able relations obtain so|5.58|1.48|9008012|namelessmaxi #12|8|romance|9|Books|540|bareseanti|N/A|saddle15696172752139|sky|Gross|Unknown|52|antiantibaration| +7056|AAAAAAAAOILBAAAA|2001-10-27||Very able relations obtain so|4.38|1.48|6007001|brandcorp #1|8|pendants|6|Jewelry|29|n stable|N/A|3178814731antique924|papaya|Cup|Unknown|63|callyantibaration| +7057|AAAAAAAABJLBAAAA|1997-10-27||Marginal expectations will manage significantly months. Hardly friendly points oug|14.94|6.72|8001008|amalgnameless #8|1|athletic shoes|8|Sports|11|oughtought|N/A|185white890202361394|royal|Each|Unknown|24|ationantibaration| +7058|AAAAAAAACJLBAAAA|1997-10-27|2000-10-26|Old, easy courses get into a years; ago routine reports carry most public girls; logically heavy practitioners must not make within a women. Central, unhappy s|9.17|5.68|2004001|edu packimporto #1|4|sports-apparel|2|Men|459|n stantiese|petite|0173spring1036986974|peru|Carton|Unknown|14|eingantibaration| +7059|AAAAAAAACJLBAAAA|2000-10-27||Narrow, friendly holders launch always up the minutes. Of course voluntary boots must forgo living shapes. Records migh|0.47|0.24|2004001|importoedu pack #2|4|mens|4|Shoes|459|n stantiese|medium|0173spring1036986974|smoke|Gross|Unknown|42|n stantibaration| +7060|AAAAAAAAEJLBAAAA|1997-10-27|1999-10-27|Then open operators should help inside the tr|4.04|3.23|1002001|importoamalg #1|2|fragrances|1|Women|23|priable|small|232943061377royal431|slate|Tsp|Unknown|21|barcallybaration| +7061|AAAAAAAAEJLBAAAA|1999-10-28|2001-10-26|Then open operators should help inside the tr|3.58|3.23|5001002|amalgscholar #2|2|rock|5|Music|23|priable|N/A|232943061377royal431|peru|Pallet|Unknown|36|oughtcallybaration| +7062|AAAAAAAAEJLBAAAA|2001-10-27||Then open operators should help inside the tr|3.26|3.23|5001002|edu packimporto #1|4|sports-apparel|2|Men|79|n station|large|232943061377royal431|orchid|Unknown|Unknown|30|ablecallybaration| +7063|AAAAAAAAHJLBAAAA|1997-10-27||Even poor women come much acceptable heads. Then similar trees live much circumstances. Then legal hours may walk eastern, simple cases; respectable|6.41|2.37|7004004|edu packbrand #4|4|curtains/drapes|7|Home|107|ationbarought|N/A|7359438puff225605876|grey|Box|Unknown|26|pricallybaration| +7064|AAAAAAAAIJLBAAAA|1997-10-27|2000-10-26|Light cases used to prevent always co|37.58|13.52|7004005|edu packbrand #5|4|curtains/drapes|7|Home|258|eingantiable|N/A|23chiffon85001131992|saddle|Each|Unknown|20|esecallybaration| +7065|AAAAAAAAIJLBAAAA|2000-10-27||Economic, useless studies pour much great eyes; at last anonymous regions get therefore as the changes. Places continue aggressively then social millions. E|5.12|13.52|7004005|amalgimporto #2|1|accessories|2|Men|258|eingantiable|medium|1steel73546617237360|lemon|Gram|Unknown|22|anticallybaration| +7066|AAAAAAAAKJLBAAAA|1997-10-27|1999-10-27|New, large duties th|33.58|16.11|10004012|edu packunivamalg #12|4|audio|10|Electronics|838|eingprieing|N/A|40180plum09091804464|seashell|Cup|Unknown|26|callycallybaration| +7067|AAAAAAAAKJLBAAAA|1999-10-28|2001-10-26|New, large duties th|6.27|16.11|10004012|amalgscholar #2|1|rock|5|Music|838|eingprieing|N/A|40180plum09091804464|papaya|N/A|Unknown|47|ationcallybaration| +7068|AAAAAAAAKJLBAAAA|2001-10-27||New, large duties th|0.47|0.16|10016014|corpamalgamalg #14|1|wireless|10|Electronics|120|eingprieing|N/A|40180plum09091804464|saddle|Tbl|Unknown|25|eingcallybaration| +7069|AAAAAAAANJLBAAAA|1997-10-27||Contemporary, actual followers eat notoriously close white orders. More familiar daughters expect sometimes often endless concentrations. Physical, military mov|3.42|2.18|2001002|amalgimporto #2|1|accessories|2|Men|153|priantiought|medium|487tan60075346936359|purple|Tbl|Unknown|100|n stcallybaration| +7070|AAAAAAAAOJLBAAAA|1997-10-27|2000-10-26|English, final telecom|4.34|2.86|2002001|importoimporto #1|2|shirts|2|Men|193|prin stought|medium|17khaki4350843313259|pale|Unknown|Unknown|5|barationbaration| +7071|AAAAAAAAOJLBAAAA|2000-10-27||Places record again; actual readers seem durin|3.19|2.86|2002001|edu packedu pack #2|2|athletic|4|Shoes|359|n stantipri|medium|992176wheat199977092|tan|Dozen|Unknown|30|oughtationbaration| +7072|AAAAAAAAAKLBAAAA|1997-10-27|1999-10-27|Assets might operate there local, sophisticated investors; poets should not get well domestic, sorry disputes. Voluntary bedrooms make at least professional states. Later vi|0.18|0.10|4001001|amalgedu pack #1|1|womens|4|Shoes|708|eingbaration|extra large|5412867765537898hot6|metallic|Each|Unknown|30|ableationbaration| +7073|AAAAAAAAAKLBAAAA|1999-10-28|2001-10-26|Assets might operate there local, sophisticated investors; poets should not get well domestic, sorry disputes. Voluntary bedrooms make at least professional states. Later vi|13.67|4.51|4001001|edu packexporti #2|1|school-uniforms|3|Children|708|eingbaration|large|8124powder8011649860|pale|Gross|Unknown|32|priationbaration| +7074|AAAAAAAAAKLBAAAA|2001-10-27||Assets might operate there local, sophisticated investors; poets should not get well domestic, sorry disputes. Voluntary bedrooms make at least professional states. Later vi|1.54|4.51|9001007|amalgmaxi #7|1|arts|9|Books|10|barought|N/A|1purple3910932839643|indian|Lb|Unknown|33|eseationbaration| +7075|AAAAAAAADKLBAAAA|1997-10-27||Public, legal languages might get easier easily regular towns. Very different children fulfil virtually tiles. Everyday, fresh numbers look only large, sole companies|9.11|3.00|8012006|importomaxi #6|12|guns|8|Sports|70|baration|N/A|675796300salmon17740|medium|Pallet|Unknown|38|antiationbaration| +7076|AAAAAAAAEKLBAAAA|1997-10-27|2000-10-26|Square, big pictures may |9.60|4.60|1002001|importoamalg #1|2|fragrances|1|Women|323|priablepri|extra large|3753rose652766312376|tomato|Ounce|Unknown|11|callyationbaration| +7077|AAAAAAAAEKLBAAAA|2000-10-27||Other emotions might rest much total clothes; nevertheless clear images fight directly odd, |2.23|4.60|9013010|exportiunivamalg #10|13|self-help|9|Books|323|priablepri|N/A|21437520thistle05339|slate|Oz|Unknown|12|ationationbaration| +7078|AAAAAAAAGKLBAAAA|1997-10-27|1999-10-27|Comfortable experiments hit defensive implications. Bad resources would not heal central, national twins. Kind, modern thoughts shall ensure home short|92.12|71.85|4002001|importoedu pack #1|2|mens|4|Shoes|55|antianti|N/A|809278213605451pale5|light|Dozen|Unknown|53|eingationbaration| +7079|AAAAAAAAGKLBAAAA|1999-10-28|2001-10-26|Comfortable experiments hit defensive implications. Bad resources would not heal central, national twins. Kind, modern thoughts shall ensure home short|5.62|3.59|4002001|exportiamalg #2|2|maternity|1|Women|182|antianti|petite|809278213605451pale5|peach|Carton|Unknown|42|n stationbaration| +7080|AAAAAAAAGKLBAAAA|2001-10-27||Comfortable experiments hit defensive implications. Bad resources would not heal central, national twins. Kind, modern thoughts shall ensure home short|9.61|3.59|6016001|corpbrand #1|2|consignment|6|Jewelry|453|priantiese|N/A|0384615orchid8924492|tan|Ton|Unknown|4|bareingbaration| +7081|AAAAAAAAJKLBAAAA|1997-10-27||Young, smart dogs vote ever; needs replace; homes must marry just on a residents; |1.32|0.89|7007002|brandbrand #2|7|decor|7|Home|227|ationableable|N/A|282683857840pale8519|indian|Pound|Unknown|6|oughteingbaration| +7082|AAAAAAAAKKLBAAAA|1997-10-27|2000-10-26|Good, wonderful lawyers stay other, awful concepts. Very acceptable states shall not say above by a trees; level foods could take at the purposes; hard ways will appear conventional rooms. Too |8.83|2.82|5003001|exportischolar #1|3|pop|5|Music|606|callybarcally|N/A|3090717steel77734545|yellow|Tbl|Unknown|13|ableeingbaration| +7083|AAAAAAAAKKLBAAAA|2000-10-27||Sadly famous knees used to write also in addition to a techniques. Great, de|1.11|2.82|10004012|edu packunivamalg #12|3|audio|10|Electronics|332|ablepripri|N/A|3090717steel77734545|green|Ton|Unknown|3|prieingbaration| +7084|AAAAAAAAMKLBAAAA|1997-10-27|1999-10-27|Efforts for|1.30|0.63|2002001|importoimporto #1|2|shirts|2|Men|282|ableeingable|medium|4chartreuse105241455|lime|Pallet|Unknown|4|eseeingbaration| +7085|AAAAAAAAMKLBAAAA|1999-10-28|2001-10-26|Efforts for|2.61|0.63|1002002|importoamalg #2|2|fragrances|1|Women|282|ableeingable|medium|339914blanched893854|peach|Tbl|Unknown|29|antieingbaration| +7086|AAAAAAAAMKLBAAAA|2001-10-27||At least static points work in order to the exceptions. Slow, other in|6.60|5.54|1002002|amalgamalg #1|2|dresses|1|Women|282|ableeingable|medium|4625metallic04968041|midnight|Pallet|Unknown|16|callyeingbaration| +7087|AAAAAAAAPKLBAAAA|1997-10-27||Other, exceptional things would buy so good, only refugees. Games fit nevertheless by a changes. Equally environmental studies destroy for the time bei|9.55|6.20|10001016|amalgunivamalg #16|1|cameras|10|Electronics|247|ationeseable|N/A|002296468903powder11|smoke|Case|Unknown|4|ationeingbaration| +7088|AAAAAAAAALLBAAAA|1997-10-27|2000-10-26|Following, different points will not attempt to a acres. Relationships pass on a representatives. Good towns might explain further down a areas. Flexible revenues would not pray purely cultural guns|7.17|2.15|4003001|exportiedu pack #1|3|kids|4|Shoes|478|eingationese|petite|2960425143893yellow7|smoke|Bunch|Unknown|32|eingeingbaration| +7089|AAAAAAAAALLBAAAA|2000-10-27||Following, different points will not attempt to a acres. Relationships pass on a representatives. Good towns might explain further down a areas. Flexible revenues would not pray purely cultural guns|4.05|2.30|4003001|namelessunivamalg #1|8|scanners|10|Electronics|424|eseableese|N/A|2960425143893yellow7|lawn|Gram|Unknown|6|n steingbaration| +7090|AAAAAAAACLLBAAAA|1997-10-27|1999-10-27|Good, black groups used to cover here with a laws. Vehicles talk. Just vocational travellers used to illustrate right, final thoughts; capital books can explain up a plants. Lightly new propos|4.66|3.86|1004001|edu packamalg #1|4|swimwear|1|Women|72|ableation|large|3769082690679green46|almond|Bundle|Unknown|45|barn stbaration| +7091|AAAAAAAACLLBAAAA|1999-10-28|2001-10-26|Good, black groups used to cover here with a laws. Vehicles talk. Just vocational travellers used to illustrate right, final thoughts; capital books can explain up a plants. Lightly new propos|1.06|0.60|5002002|importoscholar #2|2|country|5|Music|35|antipri|N/A|3769082690679green46|white|Pound|Unknown|11|oughtn stbaration| +7092|AAAAAAAACLLBAAAA|2001-10-27||Good, black groups used to cover here with a laws. Vehicles talk. Just vocational travellers used to illustrate right, final thoughts; capital books can explain up a plants. Lightly new propos|3.63|2.68|2001001|amalgimporto #1|1|accessories|2|Men|35|antipri|extra large|3769082690679green46|steel|Pallet|Unknown|66|ablen stbaration| +7093|AAAAAAAAFLLBAAAA|1997-10-27||Open, special levels cannot shut of course at a interests. Much main months alleviate married arms. Months produce drinks. Worlds find now twice other studies|4.35|1.82|7009004|maxibrand #4|9|mattresses|7|Home|206|callybarable|N/A|972397829lavender092|smoke|Bundle|Unknown|55|prin stbaration| +7094|AAAAAAAAGLLBAAAA|1997-10-27|2000-10-26|Extremely small flowers trust as children. Organisations might increase good hours; even obvious soldiers paint; considerable changes will not pick at the times. Reasons could supplement effectivel|39.43|25.62|2004001|edu packimporto #1|4|sports-apparel|2|Men|572|ableationanti|petite|998honeydew366510287|metallic|Each|Unknown|87|esen stbaration| +7095|AAAAAAAAGLLBAAAA|2000-10-27||Extremely small flowers trust as children. Organisations might increase good hours; even obvious soldiers paint; considerable changes will not pick at the times. Reasons could supplement effectivel|4.20|3.52|6005008|scholarcorp #8|4|earings|6|Jewelry|572|ableationanti|N/A|027chiffon8737487288|spring|Bundle|Unknown|6|antin stbaration| +7096|AAAAAAAAILLBAAAA|1997-10-27|1999-10-27|Numbers organize running, immediate competitors. Extensive g|5.21|2.29|1003001|exportiamalg #1|3|maternity|1|Women|269|n stcallyable|medium|3015746sky8382328629|snow|N/A|Unknown|37|callyn stbaration| +7097|AAAAAAAAILLBAAAA|1999-10-28|2001-10-26|Numbers organize running, immediate competitors. Extensive g|0.37|2.29|1003001|importoexporti #2|3|infants|3|Children|303|pribarpri|small|8284238powder7557035|rosy|Dram|Unknown|58|ationn stbaration| +7098|AAAAAAAAILLBAAAA|2001-10-27||So spanish followers would go about parents. Members confirm internally normally chosen facilities. T|9.86|4.63|3001001|amalgexporti #1|1|newborn|3|Children|303|pribarpri|medium|80382048997orange316|rosy|Dram|Unknown|19|eingn stbaration| +7099|AAAAAAAALLLBAAAA|1997-10-27||Fellow teeth could invest about compan|0.35|0.10|1004002|edu packamalg #2|4|swimwear|1|Women|47|ationese|petite|29796721salmon261905|light|Box|Unknown|30|n stn stbaration| +7100|AAAAAAAAMLLBAAAA|1997-10-27|2000-10-26|Possibly regulatory women establish usually international lives. Soviet points watch between a individuals; enough english sales must win soon different |91.10|41.90|5001001|amalgscholar #1|1|rock|5|Music|896|callyn steing|N/A|75turquoise031654051|deep|Pound|Unknown|13|barbaroughtation| +7101|AAAAAAAAMLLBAAAA|2000-10-27||Possibly regulatory women establish usually international lives. Soviet points watch between a individuals; enough english sales must win soon different |2.81|41.90|5001001|exportiedu pack #2|3|kids|4|Shoes|119|n stoughtought|N/A|75turquoise031654051|midnight|Gram|Unknown|6|oughtbaroughtation| +7102|AAAAAAAAOLLBAAAA|1997-10-27|1999-10-27|Centres must protect. Pro|3.79|1.28|5001001|amalgscholar #1|1|rock|5|Music|119|n stoughtought|N/A|2156689229788rose165|antique|Lb|Unknown|15|ablebaroughtation| +7103|AAAAAAAAOLLBAAAA|1999-10-28|2001-10-26|Reservations |0.44|0.23|5001001|exportiedu pack #2|3|kids|4|Shoes|317|ationoughtpri|medium|2156689229788rose165|puff|Ton|Unknown|26|pribaroughtation| +7104|AAAAAAAAOLLBAAAA|2001-10-27||Reservations |4.30|2.79|5004001|edu packscholar #1|3|classical|5|Music|317|ationoughtpri|N/A|2156689229788rose165|green|Pound|Unknown|26|esebaroughtation| +7105|AAAAAAAABMLBAAAA|1997-10-27||Factors could stimulate always. Public, local reactions might bring very. Sufficien|3.49|2.89|9015002|scholarunivamalg #2|15|fiction|9|Books|37|ationpri|N/A|9130084578076salmon5|thistle|Pallet|Unknown|22|antibaroughtation| +7106|AAAAAAAACMLBAAAA|1997-10-27|2000-10-26|Arrangements might ensure please countries. Anywhere genuine objectives will not resist jobs. Even wrong gods must privatise. Gardens trust before a emissions. Indeed internal movem|4.15|3.44|3003001|exportiexporti #1|3|toddlers|3|Children|688|eingeingcally|extra large|1115590022pink248620|lime|Ton|Unknown|81|callybaroughtation| +7107|AAAAAAAACMLBAAAA|2000-10-27||Even favorite minds make however patients. Final, national arm|0.12|0.08|7006008|corpbrand #8|3|rugs|7|Home|688|eingeingcally|N/A|8040969061dark579083|pink|Oz|Unknown|20|ationbaroughtation| +7108|AAAAAAAAEMLBAAAA|1997-10-27|1999-10-27|American, |1.06|0.58|4004001|edu packedu pack #1|4|athletic|4|Shoes|183|prieingought|small|68122410559tan496338|mint|Dram|Unknown|4|eingbaroughtation| +7109|AAAAAAAAEMLBAAAA|1999-10-28|2001-10-26|Both powerful solutions state consequently major horses. Trees perform quite in a associations. Chief, blue places make electoral, odd conditions. Departments convey go|0.87|0.29|3001002|amalgexporti #2|4|newborn|3|Children|112|prieingought|medium|68122410559tan496338|rosy|Cup|Unknown|40|n stbaroughtation| +7110|AAAAAAAAEMLBAAAA|2001-10-27||Full, early roots win perhaps divisions. Affairs shall not speak exactly. Hills ensure upstairs happy races. Right, successful pieces can l|5.37|0.29|3001002|corpnameless #1|6|football|8|Sports|112|prieingought|N/A|29695844840843steel5|grey|Dram|Unknown|25|baroughtoughtation| +7111|AAAAAAAAHMLBAAAA|1997-10-27||Outer, small standards correct quite unemployed quantities. Femal|2.26|1.64|10011006|amalgamalgamalg #6|11|disk drives|10|Electronics|966|callycallyn st|N/A|64826699363606slate5|turquoise|Tbl|Unknown|12|oughtoughtoughtation| +7112|AAAAAAAAIMLBAAAA|1997-10-27|2000-10-26|Industrial, good years shall hold besides things. Now leading powers might bet.|0.48|0.30|5002001|importoscholar #1|2|country|5|Music|566|callycallyanti|N/A|330356859saddle01558|honeydew|Case|Unknown|5|ableoughtoughtation| +7113|AAAAAAAAIMLBAAAA|2000-10-27||As well other sources enjoy further economic journalists. Oral, vertical conditions must recommend broadly banks. International officials expect in the audiences. Mostly major shops shar|3.45|0.30|5002001|exportiedu pack #2|2|kids|4|Shoes|566|callycallyanti|extra large|330356859saddle01558|cornsilk|Bundle|Unknown|17|prioughtoughtation| +7114|AAAAAAAAKMLBAAAA|1997-10-27|1999-10-27|Still mere hours used to comment under middle left miners. Available conservatives look far|36.21|21.36|5002001|importoscholar #1|2|country|5|Music|630|barprically|N/A|4972728757355yellow5|chocolate|Case|Unknown|20|eseoughtoughtation| +7115|AAAAAAAAKMLBAAAA|1999-10-28|2001-10-26|Small, particular details |3.74|1.79|5002001|amalgbrand #6|2|bathroom|7|Home|630|barprically|N/A|737553037smoke272845|powder|Bunch|Unknown|73|antioughtoughtation| +7116|AAAAAAAAKMLBAAAA|2001-10-27||Well wide settings start now special guns. Results start steps. Han|6.74|5.86|5002001|univunivamalg #1|10|travel|9|Books|117|ationoughtought|N/A|737553037smoke272845|smoke|Cup|Unknown|36|callyoughtoughtation| +7117|AAAAAAAANMLBAAAA|1997-10-27||Friends study cheerfully |6.20|4.40|1001002|amalgamalg #2|1|dresses|1|Women|268|eingcallyable|small|dodger82757730828225|thistle|Pound|Unknown|7|ationoughtoughtation| +7118|AAAAAAAAOMLBAAAA|1997-10-27|2000-10-26|Decisions get later overall, local women. Later direct results could seek finally large camps; other, main parts see lonely, central tho|0.71|0.63|5003001|exportischolar #1|3|pop|5|Music|583|prieinganti|N/A|864196160374papaya91|pink|Each|Unknown|58|eingoughtoughtation| +7119|AAAAAAAAOMLBAAAA|2000-10-27||Still british situations pick often successful vessels; early, different patterns isolate still more likely lawyers. C|2.73|0.63|5003001|edu packedu pack #2|3|athletic|4|Shoes|583|prieinganti|large|52639salmon488418509|rosy|Gross|Unknown|1|n stoughtoughtation| +7120|AAAAAAAAANLBAAAA|1997-10-27|1999-10-27|Commercial schemes evaluate however records. Also narrow stories survey always current, american resources; social details should take about only|6.63|4.24|1001001|amalgamalg #1|1|dresses|1|Women|161|oughtcallyought|petite|671371128royal249692|red|N/A|Unknown|44|barableoughtation| +7121|AAAAAAAAANLBAAAA|1999-10-28|2001-10-26|Commercial schemes evaluate however records. Also narrow stories survey always current, american resources; social details should take about only|7.18|4.24|4001002|amalgedu pack #2|1|womens|4|Shoes|161|oughtcallyought|economy|671371128royal249692|snow|Ton|Unknown|9|oughtableoughtation| +7122|AAAAAAAAANLBAAAA|2001-10-27||Commercial schemes evaluate however records. Also narrow stories survey always current, american resources; social details should take about only|5.84|4.24|4001002|brandbrand #3|1|decor|7|Home|153|oughtcallyought|N/A|07tomato826179320777|honeydew|Tbl|Unknown|10|ableableoughtation| +7123|AAAAAAAADNLBAAAA|1997-10-27||Small samples find doubtless foreign individuals. Quite scottish benefits will cause arrangements. Now desirable faces need always|1.28|0.52|4004002|edu packedu pack #2|4|athletic|4|Shoes|697|ationn stcally|large|57833854065pink20749|mint|Dram|Unknown|40|priableoughtation| +7124|AAAAAAAAENLBAAAA|1997-10-27|2000-10-26|Charges used to launch hard major adul|52.66|33.17|3003001|exportiexporti #1|3|toddlers|3|Children|335|antipripri|petite|288594light596046785|purple|Dram|Unknown|26|eseableoughtation| +7125|AAAAAAAAENLBAAAA|2000-10-27||Highly unknown questions know particularly comparisons. Estimated, relevant years tell. Persons take criminal councils. Particularly public pains can recov|1.58|1.10|2001002|amalgimporto #2|3|accessories|2|Men|502|antipripri|large|24073rose31217239635|spring|Pound|Unknown|38|antiableoughtation| +7126|AAAAAAAAGNLBAAAA|1997-10-27|1999-10-27|Very left numbers must change habits; as parliamentary surroundings come new efforts. So unable days should not respect |7.95|6.75|5002001|importoscholar #1|2|country|5|Music|42|ableese|N/A|9639614847royal34071|azure|Gram|Unknown|25|callyableoughtation| +7127|AAAAAAAAGNLBAAAA|1999-10-28|2001-10-26|Poles must bring in the patients. Other bodie|4.04|2.98|5002002|importoscholar #2|2|country|5|Music|188|eingeingought|N/A|9639614847royal34071|saddle|Ton|Unknown|9|ationableoughtation| +7128|AAAAAAAAGNLBAAAA|2001-10-27||Poles must bring in the patients. Other bodie|1.36|2.98|3004001|edu packexporti #1|2|school-uniforms|3|Children|115|antioughtought|medium|96914turquoise108338|spring|Dram|Unknown|42|eingableoughtation| +7129|AAAAAAAAJNLBAAAA|1997-10-27||Suddenly likely parents could need then estimated parents. Final, important terms can understand more other feel|0.96|0.81|5003002|exportischolar #2|3|pop|5|Music|425|antiableese|N/A|87chocolate864253327|papaya|Tbl|Unknown|27|n stableoughtation| +7130|AAAAAAAAKNLBAAAA|1997-10-27|2000-10-26|Sisters go seemingly tall, special fragments; straightforward grounds make all|7.67|5.67|9016003|corpunivamalg #3|16|mystery|9|Books|206|callybarable|N/A|51762194377101navy18|green|Tbl|Unknown|1|barprioughtation| +7131|AAAAAAAAKNLBAAAA|2000-10-27||Sisters go seemingly tall, special fragments; straightforward grounds make all|17.49|5.67|1001002|amalgamalg #2|16|dresses|1|Women|130|callybarable|medium|06369316720364azure7|lemon|Ton|Unknown|21|oughtprioughtation| +7132|AAAAAAAAMNLBAAAA|1997-10-27|1999-10-27|Specific, medium strings co|4.80|3.69|9010005|univunivamalg #5|10|travel|9|Books|652|ableantically|N/A|413frosted0472017128|sienna|Pallet|Unknown|56|ableprioughtation| +7133|AAAAAAAAMNLBAAAA|1999-10-28|2001-10-26|Royal patterns might measure male terms. High, other members would not need only things. Conditions shall want agricultural steps. Large, genuine days take now ruling, public appeals. Here |7.97|6.21|1004002|edu packamalg #2|4|swimwear|1|Women|652|ableantically|large|413frosted0472017128|white|Case|Unknown|3|priprioughtation| +7134|AAAAAAAAMNLBAAAA|2001-10-27||Other, huge nerves see just. Feet go much public, detailed fears. |0.32|6.21|8003005|exportinameless #5|4|basketball|8|Sports|180|bareingought|N/A|413frosted0472017128|almond|Dram|Unknown|57|eseprioughtation| +7135|AAAAAAAAPNLBAAAA|1997-10-27||National members sue inner tasks. Other, dark windows sleep notably burning arrangements. Lightly industrial ships may recognise alone a|0.13|0.11|9005008|scholarmaxi #8|5|history|9|Books|72|ableation|N/A|3668016899thistle749|wheat|N/A|Unknown|20|antiprioughtation| +7136|AAAAAAAAAOLBAAAA|1997-10-27|2000-10-26|Willing, political police might decrease principal, weak months. Dangerous prizes would unders|3.26|2.41|3003001|exportiexporti #1|3|toddlers|3|Children|153|priantiought|large|6009224846916wheat65|metallic|Box|Unknown|35|callyprioughtation| +7137|AAAAAAAAAOLBAAAA|2000-10-27||Willing, political police might decrease principal, weak months. Dangerous prizes would unders|3.27|1.47|3003001|importonameless #8|2|baseball|8|Sports|337|ationpripri|N/A|6009224846916wheat65|smoke|Oz|Unknown|1|ationprioughtation| +7138|AAAAAAAACOLBAAAA|1997-10-27|1999-10-27|Earnings must go aware minutes. Economic judges can work intensive bars. Obvious, past others will know over ordinary chil|5.52|4.19|1001001|amalgamalg #1|1|dresses|1|Women|602|ablebarcally|extra large|439935318khaki479922|burnished|Oz|Unknown|27|eingprioughtation| +7139|AAAAAAAACOLBAAAA|1999-10-28|2001-10-26|Vehicles will go. Ways will retreat courts. Years cannot enable especially days. Wings might lay soviet videos|75.18|51.12|1001001|corpunivamalg #12|1|mystery|9|Books|483|ablebarcally|N/A|439935318khaki479922|purple|Box|Unknown|82|n stprioughtation| +7140|AAAAAAAACOLBAAAA|2001-10-27||Vehicles will go. Ways will retreat courts. Years cannot enable especially days. Wings might lay soviet videos|7.15|4.50|1001001|exportiexporti #1|1|toddlers|3|Children|132|ablepriought|N/A|439935318khaki479922|slate|Case|Unknown|25|bareseoughtation| +7141|AAAAAAAAFOLBAAAA|1997-10-27||So other issues might protect late private friends; still mental suggestions establish in a drugs. Various d|2.15|0.81|7004002|edu packbrand #2|4|curtains/drapes|7|Home|97|ationn st|N/A|032612seashell815375|turquoise|N/A|Unknown|36|oughteseoughtation| +7142|AAAAAAAAGOLBAAAA|1997-10-27|2000-10-26|Intellectual feet take especially for the hours. Quite secondary mothers might produce german, hostile results. Men decide awards. There various |9.77|8.59|10008003|namelessunivamalg #3|8|scanners|10|Electronics|253|priantiable|N/A|490orchid77507079844|green|Pound|Unknown|15|ableeseoughtation| +7143|AAAAAAAAGOLBAAAA|2000-10-27||Intellectual feet take especially for the hours. Quite secondary mothers might produce german, hostile results. Men decide awards. There various |0.81|8.59|10008003|maxicorp #8|9|womens watch|6|Jewelry|471|oughtationese|N/A|5rose366348930354047|seashell|Each|Unknown|66|prieseoughtation| +7144|AAAAAAAAIOLBAAAA|1997-10-27|1999-10-27|Essential, other policies make across a peasants. Then light miles grow regularly future, full groups. Logical, accurate symptoms give tremendously. Also particular conditions could not c|6.21|3.29|10009005|maxiunivamalg #5|9|televisions|10|Electronics|834|eseprieing|N/A|7236840rosy403803203|violet|Gross|Unknown|29|eseeseoughtation| +7145|AAAAAAAAIOLBAAAA|1999-10-28|2001-10-26|Right modest visitors pay consequently to a terms. So psycholog|9.86|3.29|10009005|importoscholar #2|2|country|5|Music|834|eseprieing|N/A|7236840rosy403803203|pink|Each|Unknown|69|antieseoughtation| +7146|AAAAAAAAIOLBAAAA|2001-10-27||Both other telecommunications develop there; of course major months provide much. Cups shall not f|1.28|0.90|6016003|corpbrand #3|16|consignment|6|Jewelry|834|eseprieing|N/A|7236840rosy403803203|peru|Dram|Unknown|33|callyeseoughtation| +7147|AAAAAAAALOLBAAAA|1997-10-27||Various, royal reactions may fish sometimes special, political applications. Away special specimens would not wander professional, ta|8.08|5.41|2004002|edu packimporto #2|4|sports-apparel|2|Men|8|eing|N/A|2600176382wheat81637|steel|Unknown|Unknown|31|ationeseoughtation| +7148|AAAAAAAAMOLBAAAA|1997-10-27|2000-10-26|New, general recordings ought to meet later states. Really joint months can clear; tragic, fun funds seem relatively. Policies should not listen. Ther|1.10|0.58|10007009|brandunivamalg #9|7|personal|10|Electronics|96|callyn st|N/A|50thistle18131730420|salmon|Lb|Unknown|59|eingeseoughtation| +7149|AAAAAAAAMOLBAAAA|2000-10-27||New, general recordings ought to meet later states. Really joint months can clear; tragic, fun funds seem relatively. Policies should not listen. Ther|7.95|0.58|10007009|exportiamalg #2|3|maternity|1|Women|96|callyn st|large|1sandy03647278733782|white|Tsp|Unknown|95|n steseoughtation| +7150|AAAAAAAAOOLBAAAA|1997-10-27|1999-10-27|Judicial scientists shall carry too legal, effective women. Available prices must get. Sexual, future activities shall see early models; tiny, available|4.80|3.36|4001001|amalgedu pack #1|1|womens|4|Shoes|233|pripriable|extra large|001308436rosy8375597|rose|N/A|Unknown|33|barantioughtation| +7151|AAAAAAAAOOLBAAAA|1999-10-28|2001-10-26|Judicial scientists shall carry too legal, effective women. Available prices must get. Sexual, future activities shall see early models; tiny, available|9.12|6.74|3002002|importoexporti #2|1|infants|3|Children|233|pripriable|large|4667235612941spring4|white|Dozen|Unknown|9|oughtantioughtation| +7152|AAAAAAAAOOLBAAAA|2001-10-27||Judicial scientists shall carry too legal, effective women. Available prices must get. Sexual, future activities shall see early models; tiny, available|1.71|0.78|3002002|maxibrand #3|9|mattresses|7|Home|250|pripriable|N/A|4667235612941spring4|powder|Gram|Unknown|7|ableantioughtation| +7153|AAAAAAAABPLBAAAA|1997-10-27||Conflicts must bring cold, bol|2.77|1.27|4002002|importoedu pack #2|2|mens|4|Shoes|716|callyoughtation|small|2341071437sky6507537|peach|Carton|Unknown|5|priantioughtation| +7154|AAAAAAAACPLBAAAA|1997-10-27|2000-10-26|Most other events take adequate conditions. Never unable books like then essential words. Centuries cannot place years. Political, aware numbers may change also states. Nuclear, possible vo|0.10|0.04|5001001|amalgscholar #1|1|rock|5|Music|678|eingationcally|N/A|244027629sienna56313|wheat|Case|Unknown|65|eseantioughtation| +7155|AAAAAAAACPLBAAAA|2000-10-27||Areas give happily scientific, explicit musicians; happy, high years suit o|7.69|0.04|10016016|corpamalgamalg #16|16|wireless|10|Electronics|41|eingationcally|N/A|violet05998705636169|purple|Pallet|Unknown|27|antiantioughtation| +7156|AAAAAAAAEPLBAAAA|1997-10-27|1999-10-27|New charges can save at least in a decisions. Big, public computers survive again. Initial systems may not get cases; new levels land. Ra|9.95|6.36|5002001|importoscholar #1|2|country|5|Music|142|ableeseought|N/A|peach804302433205106|blanched|Tbl|Unknown|79|callyantioughtation| +7157|AAAAAAAAEPLBAAAA|1999-10-28|2001-10-26|New charges can save at least in a decisions. Big, public computers survive again. Initial systems may not get cases; new levels land. Ra|81.00|68.85|5002001|maxiunivamalg #1|9|televisions|10|Electronics|142|ableeseought|N/A|peach804302433205106|plum|Cup|Unknown|73|ationantioughtation| +7158|AAAAAAAAEPLBAAAA|2001-10-27||Light hands interrupt from the pap|1.63|1.27|1001001|amalgamalg #1|1|dresses|1|Women|560|barcallyanti|large|18823987349pink51780|smoke|Pound|Unknown|6|eingantioughtation| +7159|AAAAAAAAHPLBAAAA|1997-10-27||Laws should not justify then british hours. Close, chief sons continue also children. French, bitter relationships comprehend simultaneously from a police. Goa|3.71|2.00|9004008|edu packmaxi #8|4|entertainments|9|Books|738|eingpriation|N/A|92112785420256pale84|wheat|Dram|Unknown|50|n stantioughtation| +7160|AAAAAAAAIPLBAAAA|1997-10-27|2000-10-26|And so on hot trends pick really even initial concerns. Arrang|16.14|5.97|7005001|scholarbrand #1|5|blinds/shades|7|Home|148|eingeseought|N/A|569883323antique3078|salmon|Gross|Unknown|41|barcallyoughtation| +7161|AAAAAAAAIPLBAAAA|2000-10-27||And so on hot trends pick really even initial concerns. Arrang|6.37|5.47|5004002|edu packscholar #2|4|classical|5|Music|148|eingeseought|N/A|434850plum1213227188|white|Cup|Unknown|19|oughtcallyoughtation| +7162|AAAAAAAAKPLBAAAA|1997-10-27|1999-10-27|Labour, illegal friends must study here later red situations. Good titles exercise very. Social effects should see perhaps main pair|2.01|1.74|4004001|edu packedu pack #1|4|athletic|4|Shoes|539|n stprianti|large|3225794peach27260421|turquoise|Carton|Unknown|45|ablecallyoughtation| +7163|AAAAAAAAKPLBAAAA|1999-10-28|2001-10-26|Furthermore organic bones may lift mainly good years; well interesting s|2.69|1.07|2004002|edu packimporto #2|4|sports-apparel|2|Men|766|n stprianti|petite|7463985254olive70864|lawn|Lb|Unknown|62|pricallyoughtation| +7164|AAAAAAAAKPLBAAAA|2001-10-27||Furthermore organic bones may lift mainly good years; well interesting s|4.98|1.07|2004002|exportischolar #1|3|pop|5|Music|766|n stprianti|N/A|7463985254olive70864|yellow|Box|Unknown|87|esecallyoughtation| +7165|AAAAAAAANPLBAAAA|1997-10-27||However prof|1.15|0.77|3002002|importoexporti #2|2|infants|3|Children|11|oughtought|small|7494899snow406025898|seashell|Bunch|Unknown|5|anticallyoughtation| +7166|AAAAAAAAOPLBAAAA|1997-10-27|2000-10-26|Other views give around capital fields. Laws ask so full|3.69|3.21|1002001|importoamalg #1|2|fragrances|1|Women|440|bareseese|petite|7093158192902wheat73|tomato|Case|Unknown|21|callycallyoughtation| +7167|AAAAAAAAOPLBAAAA|2000-10-27||Other views give around capital fields. Laws ask so full|8.64|3.21|10014011|edu packamalgamalg #11|2|automotive|10|Electronics|18|bareseese|N/A|7093158192902wheat73|forest|Ounce|Unknown|61|ationcallyoughtation| +7168|AAAAAAAAAAMBAAAA|1997-10-27|1999-10-27|Major, concerned journalists go sure facilities. Junior, widespread changes visit technical, royal arrangeme|7.01|3.22|3002001|importoexporti #1|2|infants|3|Children|626|callyablecally|medium|yellow59211779642537|burlywood|Unknown|Unknown|1|eingcallyoughtation| +7169|AAAAAAAAAAMBAAAA|1999-10-28|2001-10-26|Major, concerned journalists go sure facilities. Junior, widespread changes visit technical, royal arrangeme|4.09|3.22|3002001|namelesscorp #2|2|mens watch|6|Jewelry|626|callyablecally|N/A|352526193puff5554697|almond|Dram|Unknown|96|n stcallyoughtation| +7170|AAAAAAAAAAMBAAAA|2001-10-27||Major, concerned journalists go sure facilities. Junior, widespread changes visit technical, royal arrangeme|4.16|3.22|3002001|edu packexporti #1|4|school-uniforms|3|Children|626|callyablecally|petite|352526193puff5554697|thistle|Lb|Unknown|5|barationoughtation| +7171|AAAAAAAADAMBAAAA|1997-10-27||Fine intervals must spend often on the hours; able problems will speak good, single divisions. Capable houses fail for a divisions. So male notes might not improve. White, important|3.10|0.96|10004015|edu packunivamalg #15|4|audio|10|Electronics|142|ableeseought|N/A|179peach715361084391|tan|Bundle|Unknown|14|oughtationoughtation| +7172|AAAAAAAAEAMBAAAA|1997-10-27|2000-10-26|Personal users may make behind a units; very other questions feed still studies. Informal lives grow. Good, young officers could get possibly problems. More clear weeks continu|8.02|5.85|8011003|amalgmaxi #3|11|archery|8|Sports|455|antiantiese|N/A|smoke178404672245319|firebrick|Tbl|Unknown|57|ableationoughtation| +7173|AAAAAAAAEAMBAAAA|2000-10-27||Wrong, average centuries will know as formidable forms. Wishes go with the results. Here likely hours would discuss full, high authorities. General, nucl|4.80|2.11|8011003|edu packnameless #4|4|camping|8|Sports|522|antiantiese|N/A|smoke178404672245319|snow|Box|Unknown|7|priationoughtation| +7174|AAAAAAAAGAMBAAAA|1997-10-27|1999-10-27|Full lines must know in general firm goods. Dangerous, economic members would try quite in a homes. Bi|9.72|6.90|10006001|corpunivamalg #1|6|musical|10|Electronics|339|n stpripri|N/A|093850wheat009086768|turquoise|Oz|Unknown|58|eseationoughtation| +7175|AAAAAAAAGAMBAAAA|1999-10-28|2001-10-26|Forwards religious cards deny never average, mass powers; big, particular critics prevent reforms.|1.00|0.41|2003002|exportiimporto #2|3|pants|2|Men|339|n stpripri|medium|78779steel2066542041|thistle|Ounce|Unknown|51|antiationoughtation| +7176|AAAAAAAAGAMBAAAA|2001-10-27||Northern, small rates end more standard cities. Young, fatal functions admi|1.66|0.41|2003002|exportinameless #7|3|basketball|8|Sports|142|ableeseought|N/A|78779steel2066542041|salmon|Lb|Unknown|63|callyationoughtation| +7177|AAAAAAAAJAMBAAAA|1997-10-27||Continuing, classical jews decide. Probably able sources used to modify most. At least whole costs operate fashio|0.74|0.23|1003002|exportiamalg #2|3|maternity|1|Women|245|antieseable|large|3278purple4560058048|purple|Tsp|Unknown|33|ationationoughtation| +7178|AAAAAAAAKAMBAAAA|1997-10-27|2000-10-26|Today current things cannot ensure well sorry, possible pupils. |0.32|0.17|4001001|amalgedu pack #1|1|womens|4|Shoes|286|callyeingable|medium|018093863879peru1874|navajo|Pound|Unknown|13|eingationoughtation| +7179|AAAAAAAAKAMBAAAA|2000-10-27||Other, other sales pay thus uncertain sections; economic, awful mountains might resist regular songs. Of course m|9.86|7.00|4001001|brandmaxi #4|7|reference|9|Books|286|callyeingable|N/A|975469blue9318364391|steel|N/A|Unknown|13|n stationoughtation| +7180|AAAAAAAAMAMBAAAA|1997-10-27|1999-10-27|Wild little results start sensitive, bare services. Big studies shall see with a men. Only services look|2.75|1.62|5003001|exportischolar #1|3|pop|5|Music|300|barbarpri|N/A|59turquoise509414283|turquoise|Dram|Unknown|20|bareingoughtation| +7181|AAAAAAAAMAMBAAAA|1999-10-28|2001-10-26|Wild little results start sensitive, bare services. Big studies shall see with a men. Only services look|1.02|0.44|4004002|edu packedu pack #2|3|athletic|4|Shoes|300|barbarpri|large|60thistle82978261544|peach|Tbl|Unknown|6|oughteingoughtation| +7182|AAAAAAAAMAMBAAAA|2001-10-27||Wild little results start sensitive, bare services. Big studies shall see with a men. Only services look|2.05|1.45|8010003|univmaxi #3|3|pools|8|Sports|536|callyprianti|N/A|60thistle82978261544|puff|Case|Unknown|15|ableeingoughtation| +7183|AAAAAAAAPAMBAAAA|1997-10-27||Live|7.97|6.85|6008008|namelesscorp #8|8|mens watch|6|Jewelry|725|antiableation|N/A|14800379856609slate0|misty|Ton|Unknown|54|prieingoughtation| +7184|AAAAAAAAABMBAAAA|1997-10-27|2000-10-26|Situations retain; units might sit operations; girls shall make. Ca|3.16|2.43|9002009|importomaxi #9|2|business|9|Books|68|eingcally|N/A|710468044296rose2043|spring|Case|Unknown|13|eseeingoughtation| +7185|AAAAAAAAABMBAAAA|2000-10-27||Situations retain; units might sit operations; girls shall make. Ca|9.14|7.95|9002009|scholarmaxi #6|15|fishing|8|Sports|373|eingcally|N/A|87310449purple520162|olive|Cup|Unknown|21|antieingoughtation| +7186|AAAAAAAACBMBAAAA|1997-10-27|1999-10-27|Ca|0.93|0.27|8013007|exportimaxi #7|13|sailing|8|Sports|214|eseoughtable|N/A|5408433644sienna6399|green|Lb|Unknown|1|callyeingoughtation| +7187|AAAAAAAACBMBAAAA|1999-10-28|2001-10-26|Ca|1.98|0.27|8013007|amalgcorp #4|13|birdal|6|Jewelry|214|eseoughtable|N/A|5408433644sienna6399|metallic|Tsp|Unknown|36|ationeingoughtation| +7188|AAAAAAAACBMBAAAA|2001-10-27||Ca|4.57|3.79|10011015|amalgamalgamalg #15|11|disk drives|10|Electronics|904|esebarn st|N/A|437416053183saddle75|snow|Carton|Unknown|83|eingeingoughtation| +7189|AAAAAAAAFBMBAAAA|1997-10-27||Years light glasses. Contemporary members might detect even drawings. Private instructions ought to expect well main streets. Children will say well; usually young members ought to ensure enough. |4.78|2.19|9001002|amalgmaxi #2|1|arts|9|Books|311|oughtoughtpri|N/A|97464676spring180073|metallic|Oz|Unknown|17|n steingoughtation| +7190|AAAAAAAAGBMBAAAA|1997-10-27|2000-10-26|Only, guilty changes ought to remember just different specimens. Hap|0.24|0.13|7007001|brandbrand #1|7|decor|7|Home|574|eseationanti|N/A|grey7883026537599557|spring|Tbl|Unknown|9|barn stoughtation| +7191|AAAAAAAAGBMBAAAA|2000-10-27||Local friends think like the devices. Only, sc|3.09|0.13|8002002|importonameless #2|7|baseball|8|Sports|287|ationeingable|N/A|grey7883026537599557|lime|Dram|Unknown|42|oughtn stoughtation| +7192|AAAAAAAAIBMBAAAA|1997-10-27|1999-10-27|Again vital details must not think users; thus total cattle sound central, particular churches; gentle, local materials could appreciate warm, high manufacturers. Qualifications allo|9.23|7.66|7006003|corpbrand #3|6|rugs|7|Home|468|eingcallyese|N/A|61078858magenta38906|steel|Pallet|Unknown|23|ablen stoughtation| +7193|AAAAAAAAIBMBAAAA|1999-10-28|2001-10-26|Small, small fears must not help leaves. Tory, grey|2.62|7.66|7006003|corpamalgamalg #3|16|wireless|10|Electronics|240|bareseable|N/A|61078858magenta38906|thistle|Each|Unknown|12|prin stoughtation| +7194|AAAAAAAAIBMBAAAA|2001-10-27||Other, due acts make then after a |3.32|2.52|2003001|exportiimporto #1|16|pants|2|Men|240|bareseable|medium|0665ivory78305759100|seashell|Bundle|Unknown|28|esen stoughtation| +7195|AAAAAAAALBMBAAAA|1997-10-27||Clothes dry now at a banks. Weapons leave. Today separate materials will obtain normally years. Good, annual men speed aside ready |5.41|4.70|10010005|univamalgamalg #5|10|memory|10|Electronics|684|eseeingcally|N/A|4424sandy30606456694|powder|Dram|Unknown|8|antin stoughtation| +7196|AAAAAAAAMBMBAAAA|1997-10-27|2000-10-26|Revolutionary rules help abroad in a details. Only, new studies get hidden, special ends. B|5.98|1.85|7006007|corpbrand #7|6|rugs|7|Home|620|barablecally|N/A|740506409peach312504|purple|Oz|Unknown|33|callyn stoughtation| +7197|AAAAAAAAMBMBAAAA|2000-10-27||Coming forms should not imagine objections. Social photographs|0.10|0.04|7006007|edu packimporto #2|6|sports-apparel|2|Men|620|barablecally|small|08104009516slate8835|yellow|Box|Unknown|45|ationn stoughtation| +7198|AAAAAAAAOBMBAAAA|1997-10-27|1999-10-27|Together minor co|2.37|1.94|8015009|scholarmaxi #9|15|fishing|8|Sports|266|callycallyable|N/A|385medium19719820348|turquoise|Tbl|Unknown|66|eingn stoughtation| +7199|AAAAAAAAOBMBAAAA|1999-10-28|2001-10-26|Events could cope simply bad roles. High days improve; french, right miles find clearly suddenly necessary organisers. White difficulties get reports. Preferences must form very sensible|96.42|79.06|1002002|importoamalg #2|15|fragrances|1|Women|241|oughteseable|large|998152532007820peru9|peru|N/A|Unknown|28|n stn stoughtation| +7200|AAAAAAAAOBMBAAAA|2001-10-27||Inevitably free eyes cannot ratify positive papers. Honours might pursue away houses; public, big events shall hear subjective, interesting companies. |6.38|79.06|7014001|edu packnameless #1|15|glassware|7|Home|241|oughteseable|N/A|8042light03809669145|khaki|Cup|Unknown|4|barbarableation| +7201|AAAAAAAABCMBAAAA|1997-10-27||||1.16|2004002||4|sports-apparel|2|||callyn stable|petite|11458512449tan376776||||2|| +7202|AAAAAAAACCMBAAAA|1997-10-27|2000-10-26|Therefore his|0.78|0.49|6007003|brandcorp #3|7|pendants|6|Jewelry|46|callyese|N/A|373907peach669515277|olive|Case|Unknown|30|ablebarableation| +7203|AAAAAAAACCMBAAAA|2000-10-27||Yet technical grounds protect more common republics; others prevent at least more trying attitudes. American members shou|2.61|2.03|6007003|amalgscholar #2|1|rock|5|Music|46|callyese|N/A|2777545673923papaya3|peach|N/A|Unknown|31|pribarableation| +7204|AAAAAAAAECMBAAAA|1997-10-27|1999-10-27|A|6.73|4.17|7008005|namelessbrand #5|8|lighting|7|Home|470|barationese|N/A|784809turquoise77193|tomato|Dozen|Unknown|65|esebarableation| +7205|AAAAAAAAECMBAAAA|1999-10-28|2001-10-26|A|6.61|4.17|4004002|edu packedu pack #2|8|athletic|4|Shoes|470|barationese|large|15342990680peru10137|khaki|Tbl|Unknown|31|antibarableation| +7206|AAAAAAAAECMBAAAA|2001-10-27||For example|46.82|29.02|4001001|amalgedu pack #1|1|womens|4|Shoes|470|barationese|small|15342990680peru10137|green|Gram|Unknown|1|callybarableation| +7207|AAAAAAAAHCMBAAAA|1997-10-27||Then widespread tourists make besides nation|2.44|0.97|4002002|importoedu pack #2|2|mens|4|Shoes|630|barprically|medium|0purple8472543605451|peru|Dram|Unknown|50|ationbarableation| +7208|AAAAAAAAICMBAAAA|1997-10-27|2000-10-26|Neatly hard theories turn by the females. Only fresh facilities should feed nicely. Simi|74.30|35.66|9013009|exportiunivamalg #9|13|self-help|9|Books|784|eseeingation|N/A|990702673640007lawn5|red|Case|Unknown|39|eingbarableation| +7209|AAAAAAAAICMBAAAA|2000-10-27||Front, imperial lines will not slow small laws. New countries get low. R|4.23|3.13|9013009|corpnameless #10|13|football|8|Sports|79|n station|N/A|5457967558turquoise0|red|Gram|Unknown|1|n stbarableation| +7210|AAAAAAAAKCMBAAAA|1997-10-27|1999-10-27|Already recent features ought to shed as so severe critics; coherent, new weapons ought to think to a accidents. Answers follow necessary, mental activities. Nuclear minutes run so no longer long clot|0.35|0.22|3003001|exportiexporti #1|3|toddlers|3|Children|234|esepriable|small|67792ivory6243481645|turquoise|Pallet|Unknown|1|baroughtableation| +7211|AAAAAAAAKCMBAAAA|1999-10-28|2001-10-26|Guidelines will not accept visitors; huge pieces must not rule though quite blue wines; always loose responsibilities|3.97|0.22|6015004|scholarbrand #4|15|custom|6|Jewelry|234|esepriable|N/A|348868turquoise83380|spring|Dram|Unknown|22|oughtoughtableation| +7212|AAAAAAAAKCMBAAAA|2001-10-27||Scientific bills occur. Options could identify only. Aware seats ou|0.35|0.22|1004001|edu packamalg #1|4|swimwear|1|Women|511|oughtoughtanti|petite|348868turquoise83380|wheat|Each|Unknown|29|ableoughtableation| +7213|AAAAAAAANCMBAAAA|1997-10-27||Remaining children love then from a gu|7.81|4.84|5001002|amalgscholar #2|1|rock|5|Music|28|eingable|N/A|731yellow02190650431|steel|Dozen|Unknown|37|prioughtableation| +7214|AAAAAAAAOCMBAAAA|1997-10-27|2000-10-26|Others navigate projects. Democratic, experimental margins ought to tell often personal, current reasons. Ph|17.35|10.23|8014005|edu packmaxi #5|14|tennis|8|Sports|247|ationeseable|N/A|43papaya355255226996|misty|Ounce|Unknown|93|eseoughtableation| +7215|AAAAAAAAOCMBAAAA|2000-10-27||Others navigate projects. Democratic, experimental margins ought to tell often personal, current reasons. Ph|95.04|10.23|7015006|scholarnameless #6|14|tables|7|Home|271|oughtationable|N/A|43papaya355255226996|khaki|Gross|Unknown|53|antioughtableation| +7216|AAAAAAAAADMBAAAA|1997-10-27|1999-10-27|Lovely eyes make nearly on a products; thin games ought to find additional, financial merchants; partly other men forget below. Periods shall take just. Here|8.25|4.45|2003001|exportiimporto #1|3|pants|2|Men|42|ableese|petite|78820231green4500269|blanched|Ton|Unknown|61|callyoughtableation| +7217|AAAAAAAAADMBAAAA|1999-10-28|2001-10-26|Lovely eyes make nearly on a products; thin games ought to find additional, financial merchants; partly other men forget below. Periods shall take just. Here|7.75|4.45|7009008|maxibrand #8|3|mattresses|7|Home|42|ableese|N/A|27857metallic8170948|sandy|Bundle|Unknown|5|ationoughtableation| +7218|AAAAAAAAADMBAAAA|2001-10-27||Ultimate, jewish eyes steal at a things. Constant lads would run just members|5.10|4.45|8014009|edu packmaxi #9|3|tennis|8|Sports|91|ableese|N/A|5869light71227471778|red|Gram|Unknown|27|eingoughtableation| +7219|AAAAAAAADDMBAAAA|1997-10-27||Effective, isolated comments might say essential, possible parents; recent years used to make years. Parents tell so from the centur|7.03|4.00|6012004|importobrand #4|12|costume|6|Jewelry|170|barationought|N/A|57193192violet607524|salmon|Tbl|Unknown|27|n stoughtableation| +7220|AAAAAAAAEDMBAAAA|1997-10-27|2000-10-26|Servants happen unpleasant stages. Able others die here real, late tensions. Even concerned s|4.05|1.29|10008013|namelessunivamalg #13|8|scanners|10|Electronics|509|n stbaranti|N/A|5midnight63908630943|dodger|Tsp|Unknown|9|barableableation| +7221|AAAAAAAAEDMBAAAA|2000-10-27||Single plans must gather now specific words. Appropriate, li|5.11|4.19|10008013|corpunivamalg #10|8|mystery|9|Books|450|n stbaranti|N/A|5midnight63908630943|pale|Ounce|Unknown|6|oughtableableation| +7222|AAAAAAAAGDMBAAAA|1997-10-27|1999-10-27|Good, main trees may forget enough in a hills; changes will ensure also notably certain banks. Possible, urgent passages ought to go statements. New, subtle photographs may maintain yet like|2.49|2.04|1001001|amalgamalg #1|1|dresses|1|Women|135|antipriought|small|30deep29822812318277|sandy|Bundle|Unknown|7|ableableableation| +7223|AAAAAAAAGDMBAAAA|1999-10-28|2001-10-26|Games might not give late old amounts. Leadin|0.33|0.24|1002002|importoamalg #2|1|fragrances|1|Women|135|antipriought|medium|30deep29822812318277|salmon|Ton|Unknown|48|priableableation| +7224|AAAAAAAAGDMBAAAA|2001-10-27||Games might not give late old amounts. Leadin|0.39|0.33|1002002|corpbrand #1|1|consignment|6|Jewelry|301|oughtbarpri|N/A|30deep29822812318277|thistle|N/A|Unknown|30|eseableableation| +7225|AAAAAAAAJDMBAAAA|1997-10-27||Judges sit enormous bodies. Always social versions shall not tell around special, new negotiations. Short, interest|3.34|2.60|4001002|amalgedu pack #2|1|womens|4|Shoes|68|eingcally|petite|347thistle4743999196|ghost|Pound|Unknown|48|antiableableation| +7226|AAAAAAAAKDMBAAAA|1997-10-27|2000-10-26|Welsh, electoral points shall fix more approximately possible claims. T|2.83|1.55|9016003|corpunivamalg #3|16|mystery|9|Books|461|oughtcallyese|N/A|656903256peach351046|sky|N/A|Unknown|62|callyableableation| +7227|AAAAAAAAKDMBAAAA|2000-10-27||Proud candidates go slightly slightly other conditions. Good ages shall not receive with a wea|1.93|0.73|9016003|brandcorp #4|16|pendants|6|Jewelry|207|oughtcallyese|N/A|82745peru17967105840|pale|Each|Unknown|59|ationableableation| +7228|AAAAAAAAMDMBAAAA|1997-10-27|1999-10-27|Easier ashamed implications will care. Exceptional men must not enjoy social, rural deposits. Upw|3.79|2.72|7002009|importobrand #9|2|bedding|7|Home|39|n stpri|N/A|6866seashell64309381|slate|Ounce|Unknown|8|eingableableation| +7229|AAAAAAAAMDMBAAAA|1999-10-28|2001-10-26|Only buildings think critica|4.15|2.72|7002009|corpcorp #6|2|rings|6|Jewelry|39|n stpri|N/A|90turquoise134169301|spring|Bundle|Unknown|11|n stableableation| +7230|AAAAAAAAMDMBAAAA|2001-10-27||American charges see frequently foreign, true details. Claims shall reduce able, political difficulties. Able, western principles tour fully available, light c|1.52|0.56|9012007|importounivamalg #7|12|home repair|9|Books|465|anticallyese|N/A|28837tan497778053029|green|Bunch|Unknown|58|barpriableation| +7231|AAAAAAAAPDMBAAAA|1997-10-27||Definitely jewish scientists give fiercely rapidly lucky clergy. Differently |0.50|0.28|4002002|importoedu pack #2|2|mens|4|Shoes|25|antiable|medium|756424425709051sky75|puff|Box|Unknown|54|oughtpriableation| +7232|AAAAAAAAAEMBAAAA|1997-10-27|2000-10-26|Only able governments complete other problems. Here significant profits shall not make sc|9.40|3.85|6011001|amalgbrand #1|11|semi-precious|6|Jewelry|245|antieseable|N/A|purple35012161286589|turquoise|Cup|Unknown|16|ablepriableation| +7233|AAAAAAAAAEMBAAAA|2000-10-27||Surely accurate departments knock later at |2.72|3.85|6011001|amalgexporti #2|11|newborn|3|Children|245|antieseable|medium|purple35012161286589|snow|Bunch|Unknown|45|pripriableation| +7234|AAAAAAAACEMBAAAA|1997-10-27|1999-10-27|Parents would concede over particular months. Modern, useful sports shall not say prime, western hills. Recently small implications would not write certain flats. Primary, pot|1.35|0.70|8011005|amalgmaxi #5|11|archery|8|Sports|816|callyoughteing|N/A|89578926sienna300954|steel|Case|Unknown|5|esepriableation| +7235|AAAAAAAACEMBAAAA|1999-10-28|2001-10-26|Automatically new terms give enough small thoughts. Yesterday other requirem|9.81|0.70|8004006|edu packnameless #6|11|camping|8|Sports|835|antiprieing|N/A|89578926sienna300954|orange|Ounce|Unknown|64|antipriableation| +7236|AAAAAAAACEMBAAAA|2001-10-27||Automatically new terms give enough small thoughts. Yesterday other requirem|14.21|0.70|8005007|scholarnameless #7|11|fitness|8|Sports|90|antiprieing|N/A|89578926sienna300954|tan|Lb|Unknown|6|callypriableation| +7237|AAAAAAAAFEMBAAAA|1997-10-27||Natural hands will see almost simple, alone seconds. Regulations shall impress white,|99.85|63.90|8014008|edu packmaxi #8|14|tennis|8|Sports|362|ablecallypri|N/A|3175083royal34299232|cornflower|Pallet|Unknown|10|ationpriableation| +7238|AAAAAAAAGEMBAAAA|1997-10-27|2000-10-26|Full rights go to the p|64.77|57.64|1004001|edu packamalg #1|4|swimwear|1|Women|462|ablecallyese|petite|143237peach634980845|sandy|Ton|Unknown|7|eingpriableation| +7239|AAAAAAAAGEMBAAAA|2000-10-27||Required, essential owners try about similar years. Claims will show particularly main talks. Men m|2.22|57.64|1004001|edu packamalg #2|4|swimwear|1|Women|462|ablecallyese|large|143237peach634980845|seashell|Lb|Unknown|53|n stpriableation| +7240|AAAAAAAAIEMBAAAA|1997-10-27|1999-10-27|However great lines might not finance of course fine example|2.86|2.14|5003001|exportischolar #1|3|pop|5|Music|538|eingprianti|N/A|8snow907613539688490|midnight|Pallet|Unknown|45|bareseableation| +7241|AAAAAAAAIEMBAAAA|1999-10-28|2001-10-26|Nice, friendly police used to attend more classical, left details. Good, welsh rounds might care formally; formal, attractive police go extremely in a bo|2.45|2.14|8015002|scholarmaxi #2|15|fishing|8|Sports|538|eingprianti|N/A|3684pink792611282969|green|Carton|Unknown|20|oughteseableation| +7242|AAAAAAAAIEMBAAAA|2001-10-27||Similar cases tell once particular children. New visitors eat soon here bad ideas; farmers ought to impose vehicles. Popular arts repeat then finally strict tracks. Odd, personal systems|1.37|0.57|8015002|edu packexporti #1|4|school-uniforms|3|Children|538|eingprianti|petite|399497turquoise02980|mint|Bunch|Unknown|1|ableeseableation| +7243|AAAAAAAALEMBAAAA|1997-10-27||Parents could not bring n|19.86|8.73|5003002|exportischolar #2|3|pop|5|Music|126|callyableought|N/A|34745sandy8102740362|plum|Gram|Unknown|29|prieseableation| +7244|AAAAAAAAMEMBAAAA||2000-10-26|General, likely points come really numerous accounts. Big operations admit. Then dependent partners used to take instantly for a contracts. Tests stay never. O|1.15|||||womens|||||||pale|Ton|Unknown||| +7245|AAAAAAAAMEMBAAAA|2000-10-27||General, likely points come really numerous accounts. Big operations admit. Then dependent partners used to take instantly for a contracts. Tests stay never. O|0.18|0.47|3004002|edu packexporti #2|1|school-uniforms|3|Children|243|ationableanti|large|8548392140moccasin80|royal|Bunch|Unknown|5|antieseableation| +7246|AAAAAAAAOEMBAAAA|1997-10-27|1999-10-27|Politically unknown types ought to raise again fa|2.63|0.89|5001001|amalgscholar #1|1|rock|5|Music|251|oughtantiable|N/A|649639541black797827|lace|Carton|Unknown|3|callyeseableation| +7247|AAAAAAAAOEMBAAAA|1999-10-28|2001-10-26|Very special effects shall make most sides. Precise processes use|1.67|0.89|5001001|corpnameless #8|1|football|8|Sports|251|oughtantiable|N/A|649639541black797827|navy|Tsp|Unknown|41|ationeseableation| +7248|AAAAAAAAOEMBAAAA|2001-10-27||Very special effects shall make most sides. Precise processes use|3.93|0.89|4001001|amalgedu pack #1|1|womens|4|Shoes|311|oughtantiable|large|rose2921676914486638|royal|Dram|Unknown|64|eingeseableation| +7249|AAAAAAAABFMBAAAA|1997-10-27||Guidelines design ago from a protests. America|1.38|0.59|7006010|corpbrand #10|6|rugs|7|Home|248|eingeseable|N/A|76364dim454504134450|spring|Ounce|Unknown|76|n steseableation| +7250|AAAAAAAACFMBAAAA|1997-10-27|2000-10-26|Minutes survive social, constitutional provinces. Insects control. Bitterly other tenants would realise requirements. Open wome|7.50|5.10|4004001|edu packedu pack #1|4|athletic|4|Shoes|787|ationeingation|small|479361156380sienna15|seashell|Bunch|Unknown|63|barantiableation| +7251|AAAAAAAACFMBAAAA|2000-10-27||Mathematical areas replace ever large, common studies. Detailed, legal vegetables overwhelm there shared studies. Ultimate, natural publishers will not enjoy totally stude|3.85|5.10|7005002|scholarbrand #2|5|blinds/shades|7|Home|176|ationeingation|N/A|1269825magenta985327|smoke|Bundle|Unknown|51|oughtantiableation| +7252|AAAAAAAAEFMBAAAA|1997-10-27|1999-10-27|Precious police phone full elements. Temporary friends seem sorry opportunities. Historical, impressive |8.38|4.52|1001001|amalgamalg #1|1|dresses|1|Women|220|barableable|medium|53517peach6446893149|wheat|Unknown|Unknown|18|ableantiableation| +7253|AAAAAAAAEFMBAAAA|1999-10-28|2001-10-26|Little, brown|76.96|48.48|4001002|amalgedu pack #2|1|womens|4|Shoes|787|barableable|petite|53517peach6446893149|honeydew|Cup|Unknown|37|priantiableation| +7254|AAAAAAAAEFMBAAAA|2001-10-27||Weeks might not lead so appropriate days. Carefully commercial heads compensa|2.59|48.48|4001002|amalgedu pack #1|1|womens|4|Shoes|115|barableable|economy|46709saddle184345465|beige|Pound|Unknown|15|eseantiableation| +7255|AAAAAAAAHFMBAAAA|1997-10-27||Jobs crack at all public countries. Possible, traditional initiatives will make british problems. Fiscal years call extremely just growing pupils. Soon dutch voices|86.11|52.52|10016014|corpamalgamalg #14|16|wireless|10|Electronics|694|esen stcally|N/A|84steel1342825899472|seashell|Lb|Unknown|16|antiantiableation| +7256|AAAAAAAAIFMBAAAA|1997-10-27|2000-10-26|Terms use rather likely, severe com|3.68|1.21|10009015|maxiunivamalg #15|9|televisions|10|Electronics|662|ablecallycally|N/A|95885033ivory0240132|sienna|Carton|Unknown|1|callyantiableation| +7257|AAAAAAAAIFMBAAAA|2000-10-27||Troops would not reflect never in a measurements; friends will want then. Foreign farmers might gain about to the efforts. |1.88|1.21|2004002|edu packimporto #2|9|sports-apparel|2|Men|662|ablecallycally|small|95885033ivory0240132|sienna|Ounce|Unknown|43|ationantiableation| +7258|AAAAAAAAKFMBAAAA|1997-10-27|1999-10-27|Other, elaborate organisations throw for a communists. Prime, dead programmes secure ready, glad beds. Main, big animals dry. Secondary months study quickly global troops. Situ|9.94|8.74|7016009|corpnameless #9|16|furniture|7|Home|232|ablepriable|N/A|18827659330926royal0|white|Tbl|Unknown|12|eingantiableation| +7259|AAAAAAAAKFMBAAAA|1999-10-28|2001-10-26|Other, elaborate organisations throw for a communists. Prime, dead programmes secure ready, glad beds. Main, big animals dry. Secondary months study quickly global troops. Situ|1.24|0.63|7016009|univnameless #10|10|flatware|7|Home|591|ablepriable|N/A|18827659330926royal0|red|Tsp|Unknown|1|n stantiableation| +7260|AAAAAAAAKFMBAAAA|2001-10-27||Other, elaborate organisations throw for a communists. Prime, dead programmes secure ready, glad beds. Main, big animals dry. Secondary months study quickly global troops. Situ|0.23|0.18|2004001|edu packimporto #1|10|sports-apparel|2|Men|591|oughtn stanti|large|18827659330926royal0|khaki|Pound|Unknown|94|barcallyableation| +7261|AAAAAAAANFMBAAAA|1997-10-27||Usually desperat|1.51|0.60|7009002|maxibrand #2|9|mattresses|7|Home|255|antiantiable|N/A|3244799903royal82675|magenta|Dram|Unknown|56|oughtcallyableation| +7262|AAAAAAAAOFMBAAAA|1997-10-27|2000-10-26|Red, enormous women used to wait tonight suitable feet. Taxes seem clearly studies. White hours should no|7.25|4.13|1004001|edu packamalg #1|4|swimwear|1|Women|43|priese|medium|793934482968245red02|snow|Unknown|Unknown|42|ablecallyableation| +7263|AAAAAAAAOFMBAAAA|2000-10-27||Other, nuclear years should not meet also free assets|3.38|2.77|1004001|importounivamalg #5|2|camcorders|10|Electronics|43|priese|N/A|7353397599indian7798|slate|Case|Unknown|1|pricallyableation| +7264|AAAAAAAAAGMBAAAA|1997-10-27|1999-10-27|True regulations may not restore with a magistrates. Critical results take once white, large prisoners; political,|1.54|1.27|9009005|maximaxi #5|9|science|9|Books|347|ationesepri|N/A|859112901941magenta5|seashell|N/A|Unknown|10|esecallyableation| +7265|AAAAAAAAAGMBAAAA|1999-10-28|2001-10-26|True regulations may not restore with a magistrates. Critical results take once white, large prisoners; political,|0.27|0.14|9009005|amalgunivamalg #6|11|cooking|9|Books|650|barantically|N/A|696931706663steel524|salmon|Tbl|Unknown|55|anticallyableation| +7266|AAAAAAAAAGMBAAAA|2001-10-27||True regulations may not restore with a magistrates. Critical results take once white, large prisoners; political,|4.13|2.60|6007003|brandcorp #3|11|pendants|6|Jewelry|650|barantically|N/A|04maroon835944740124|moccasin|Gram|Unknown|20|callycallyableation| +7267|AAAAAAAADGMBAAAA|1997-10-27||Again human powers should|0.69|0.54|6016004|corpbrand #4|16|consignment|6|Jewelry|228|eingableable|N/A|rose2504136454007821|spring|Box|Unknown|33|ationcallyableation| +7268|AAAAAAAAEGMBAAAA|1997-10-27|2000-10-26|There residential days bring for a years. Wron|3.14|1.53|6001005|amalgcorp #5|1|birdal|6|Jewelry|926|callyablen st|N/A|332615656violet39169|midnight|Bundle|Unknown|91|eingcallyableation| +7269|AAAAAAAAEGMBAAAA|2000-10-27||There residential days bring for a years. Wron|8.01|1.53|6001005|exportiamalg #2|1|maternity|1|Women|334|callyablen st|small|34763smoke8609486467|peru|Carton|Unknown|89|n stcallyableation| +7270|AAAAAAAAGGMBAAAA|1997-10-27|1999-10-27|Close publishers ought to tear fond, classical games. Even available uses like shelves. Then white states fit also. Far from difficult |0.69|0.22|4004001|edu packedu pack #1|4|athletic|4|Shoes|615|antioughtcally|small|055sky53424301357542|sienna|N/A|Unknown|19|barationableation| +7271|AAAAAAAAGGMBAAAA|1999-10-28|2001-10-26|Other processes cannot tell about hands; knees will give main, good inhabitants; aside general agencies ought to e|0.71|0.41|7010002|univnameless #2|4|flatware|7|Home|247|ationeseable|N/A|9313936404890smoke42|orchid|Case|Unknown|6|oughtationableation| +7272|AAAAAAAAGGMBAAAA|2001-10-27||Other processes cannot tell about hands; knees will give main, good inhabitants; aside general agencies ought to e|3.74|0.41|8015003|scholarmaxi #3|15|fishing|8|Sports|465|ationeseable|N/A|9313936404890smoke42|violet|Tsp|Unknown|63|ableationableation| +7273|AAAAAAAAJGMBAAAA|1997-10-27||Unique, ready patients should fit upon a periods. Russian participants used to pay very. Classes can speci|3.22|2.57|10016015|corpamalgamalg #15|16|wireless|10|Electronics|48|eingese|N/A|43918917041salmon111|powder|Each|Unknown|55|priationableation| +7274|AAAAAAAAKGMBAAAA|1997-10-27|2000-10-26|Now interested centres might obey yet objectives. Schools finish proposed, worthwhile areas. Simple, wide account|55.70|38.99|8001005|amalgnameless #5|1|athletic shoes|8|Sports|262|ablecallyable|N/A|40peru20652769387015|white|Bundle|Unknown|19|eseationableation| +7275|AAAAAAAAKGMBAAAA|2000-10-27||Now interested centres might obey yet objectives. Schools finish proposed, worthwhile areas. Simple, wide account|5.14|38.99|7011006|amalgnameless #6|1|accent|7|Home|502|ablecallyable|N/A|413411peru9018524832|sky|Box|Unknown|11|antiationableation| +7276|AAAAAAAAMGMBAAAA|1997-10-27|1999-10-27|Unusual, clinical guards may distort about symbols. Threats will seek for instance public matters. Buildings might find too new char|8.62|5.51|3002001|importoexporti #1|2|infants|3|Children|282|ableeingable|large|53pale20914789022845|smoke|Bundle|Unknown|12|callyationableation| +7277|AAAAAAAAMGMBAAAA|1999-10-28|2001-10-26|Unusual, clinical guards may distort about symbols. Threats will seek for instance public matters. Buildings might find too new char|8.78|5.51|3002001|scholarnameless #8|2|tables|7|Home|131|ableeingable|N/A|535754hot15202835986|medium|Bundle|Unknown|47|ationationableation| +7278|AAAAAAAAMGMBAAAA|2001-10-27||Managers|4.71|5.51|2003001|exportiimporto #1|3|pants|2|Men|72|ableation|large|535754hot15202835986|plum|Tbl|Unknown|45|eingationableation| +7279|AAAAAAAAPGMBAAAA|1997-10-27||Financial judges enter soon major appeals. Far social men use doubts. Results would not relate so. Classes used to mak|1.73|0.69|4001002|amalgedu pack #2|1|womens|4|Shoes|476|callyationese|medium|99328418693navajo494|navajo|Each|Unknown|95|n stationableation| +7280|AAAAAAAAAHMBAAAA|1997-10-27|2000-10-26|At least visual pounds could succeed ago just normal eyes. Convincing hospitals may not enjoy about for the|8.24|7.00|1001001|amalgamalg #1|1|dresses|1|Women|41|oughtese|medium|purple17637265137337|rose|Tbl|Unknown|2|bareingableation| +7281|AAAAAAAAAHMBAAAA|2000-10-27||At least visual pounds could succeed ago just normal eyes. Convincing hospitals may not enjoy about for the|9.84|7.00|8001002|amalgnameless #2|1|athletic shoes|8|Sports|638|eingprically|N/A|purple17637265137337|violet|Pound|Unknown|23|oughteingableation| +7282|AAAAAAAACHMBAAAA|1997-10-27|1999-10-27|Male, silent women must outline over international children. Local mice should mean |9.43|8.48|3003001|exportiexporti #1|3|toddlers|3|Children|290|barn stable|economy|37880214433puff20976|tomato|Ounce|Unknown|4|ableeingableation| +7283|AAAAAAAACHMBAAAA|1999-10-28|2001-10-26|Male, silent women must outline over international children. Local mice should mean |21.65|9.30|9005012|scholarmaxi #12|5|history|9|Books|205|antibarable|N/A|84sienna849679020624|salmon|Dozen|Unknown|11|prieingableation| +7284|AAAAAAAACHMBAAAA|2001-10-27||Male, silent women must outline over international children. Local mice should mean |75.22|9.30|9005012|importoexporti #1|5|infants|3|Children|396|callyn stpri|petite|723turquoise76978397|olive|Case|Unknown|25|eseeingableation| +7285|AAAAAAAAFHMBAAAA|1997-10-27||New, simple women may meet ago other ears. Either unknown thoughts cannot manage relatively. Other machines join as in a clouds. Men must answer police; casually|8.63|4.74|4002002|importoedu pack #2|2|mens|4|Shoes|933|priprin st|medium|681834580332coral226|mint|Cup|Unknown|9|antieingableation| +7286|AAAAAAAAGHMBAAAA|1997-10-27|2000-10-26|Main, nervous preferences find certainly constant reasons. Open, primary boys zero rats|1.78|0.78|7014007|edu packnameless #7|14|glassware|7|Home|388|eingeingpri|N/A|615407smoke124564949|sandy|Gross|Unknown|10|callyeingableation| +7287|AAAAAAAAGHMBAAAA|2000-10-27||Main, nervous preferences find certainly constant reasons. Open, primary boys zero rats|2.21|1.56|7014007|importoimporto #2|2|shirts|2|Men|388|eingeingpri|petite|19235peru13532347005|sandy|Dram|Unknown|19|ationeingableation| +7288|AAAAAAAAIHMBAAAA|1997-10-27|1999-10-27|Able considerations may end even; financial, useful courts ought to dominate however. Daily, concerned interests should come clean, local services. More unlikely tea|2.92|2.45|4003001|exportiedu pack #1|3|kids|4|Shoes|754|eseantiation|extra large|29983746violet767276|slate|Gram|Unknown|56|eingeingableation| +7289|AAAAAAAAIHMBAAAA|1999-10-28|2001-10-26|Able considerations may end even; financial, useful courts ought to dominate however. Daily, concerned interests should come clean, local services. More unlikely tea|7.85|2.45|4003001|importoscholar #2|2|country|5|Music|156|callyantiought|N/A|29983746violet767276|thistle|Carton|Unknown|9|n steingableation| +7290|AAAAAAAAIHMBAAAA|2001-10-27||Able considerations may end even; financial, useful courts ought to dominate however. Daily, concerned interests should come clean, local services. More unlikely tea|7.03|3.16|4003001|importoimporto #1|2|shirts|2|Men|228|eingableable|extra large|29983746violet767276|metallic|Carton|Unknown|6|barn stableation| +7291|AAAAAAAALHMBAAAA|1997-10-27||Professional, political days would beat human, only relationships. Difficulties pull more lucky members; high places look about a outcomes. Weeks pay really |58.49|25.15|4001002|amalgedu pack #2|1|womens|4|Shoes|334|esepripri|extra large|804440348721powder30|orange|Pound|Unknown|32|oughtn stableation| +7292|AAAAAAAAMHMBAAAA|1997-10-27|2000-10-26|Only supplies might remember again. Forces agree thus of course human areas. Budgets should pay similar, local procedures. Following, able things help elderly, american volu|3.98|3.58|9005009|scholarmaxi #9|5|history|9|Books|356|callyantipri|N/A|39055thistle39372157|snow|Case|Unknown|49|ablen stableation| +7293|AAAAAAAAMHMBAAAA|2000-10-27||Only supplies might remember again. Forces agree thus of course human areas. Budgets should pay similar, local procedures. Following, able things help elderly, american volu|0.09|3.58|5003002|exportischolar #2|3|pop|5|Music|356|callyantipri|N/A|39055thistle39372157|peru|Pallet|Unknown|5|prin stableation| +7294|AAAAAAAAOHMBAAAA|1997-10-27|1999-10-27|At last enthusiastic units make; very formal goods apply somewhat running years; re|34.87|24.06|8001005|amalgnameless #5|1|athletic shoes|8|Sports|85|antieing|N/A|25yellow728149426893|lace|Pallet|Unknown|18|esen stableation| +7295|AAAAAAAAOHMBAAAA|1999-10-28|2001-10-26|Payable, young teams know nicely on a policies; sharp, wise firms come suddenly young cars. Always working bonds might know for example white factors;|3.45|24.06|8001005|edu packbrand #6|4|curtains/drapes|7|Home|293|prin stable|N/A|25yellow728149426893|yellow|Ton|Unknown|18|antin stableation| +7296|AAAAAAAAOHMBAAAA|2001-10-27||Deaths eliminate different, individual twins; high estim|0.77|24.06|8001005|edu packbrand #1|14|estate|6|Jewelry|293|prin stable|N/A|25yellow728149426893|misty|Oz|Unknown|30|callyn stableation| +7297|AAAAAAAABIMBAAAA|1997-10-27||Supposed provisions exercise past figures. Now old theories shall not relax ultimately to a clothes. Jobs |2.85|1.53|2001002|amalgimporto #2|1|accessories|2|Men|89|n steing|extra large|9royal59219679238632|rosy|Box|Unknown|10|ationn stableation| +7298|AAAAAAAACIMBAAAA|1997-10-27|2000-10-26|Great, famous feet ought to exten|1.73|0.67|10002006|importounivamalg #6|2|camcorders|10|Electronics|316|callyoughtpri|N/A|880436213honeydew358|plum|Bundle|Unknown|37|eingn stableation| +7299|AAAAAAAACIMBAAAA|2000-10-27||Great, famous feet ought to exten|1.12|0.41|10002006|importoexporti #2|2|infants|3|Children|316|callyoughtpri|extra large|880436213honeydew358|linen|Unknown|Unknown|52|n stn stableation| +7300|AAAAAAAAEIMBAAAA|1997-10-27|1999-10-27|Carefully ordinary offices could not hope still true vast letters. Workers may not happen fortunately resulting, communist prices. Now good titles survive immensely to a pic|3.35|2.91|3002001|importoexporti #1|2|infants|3|Children|726|callyableation|medium|88258409yellow414930|spring|Oz|Unknown|27|barbarpriation| +7301|AAAAAAAAEIMBAAAA|1999-10-28|2001-10-26|Carefully ordinary offices could not hope still true vast letters. Workers may not happen fortunately resulting, communist prices. Now good titles survive immensely to a pic|5.50|2.91|8011002|amalgmaxi #2|2|archery|8|Sports|295|antin stable|N/A|88258409yellow414930|sky|Lb|Unknown|39|oughtbarpriation| +7302|AAAAAAAAEIMBAAAA|2001-10-27||Silver children will speak then exceptional times. Men live extensive measures. Now social agreements used to make usually common coloured consequences. Oral authorities might|72.95|2.91|8001003|amalgnameless #3|1|athletic shoes|8|Sports|225|antiableable|N/A|powder30960989870522|tomato|Tsp|Unknown|11|ablebarpriation| +7303|AAAAAAAAHIMBAAAA|1997-10-27||Perhaps british beings cannot attract model principl|4.56|1.91|6005008|scholarcorp #8|5|earings|6|Jewelry|799|n stn station|N/A|961426rose2433634224|hot|Dram|Unknown|6|pribarpriation| +7304|AAAAAAAAIIMBAAAA|1997-10-27|2000-10-26|Numbers answer apparent, naked colours. Brown decisions could find possible members. Years agree around joint, huge authorities|7.02|3.08|10013012|exportiamalgamalg #12|13|stereo|10|Electronics|115|antioughtought|N/A|359turquoise20550022|turquoise|Dozen|Unknown|17|esebarpriation| +7305|AAAAAAAAIIMBAAAA|2000-10-27||Numbers answer apparent, naked colours. Brown decisions could find possible members. Years agree around joint, huge authorities|4.84|3.67|10015013|scholaramalgamalg #13|13|portable|10|Electronics|954|eseantin st|N/A|359turquoise20550022|black|Oz|Unknown|55|antibarpriation| +7306|AAAAAAAAKIMBAAAA|1997-10-27|1999-10-27|Precisely weak powers could require sometimes important subjects. Rus|4.90|2.89|6007003|brandcorp #3|7|pendants|6|Jewelry|148|eingeseought|N/A|30687honeydew2401144|slate|Pallet|Unknown|23|callybarpriation| +7307|AAAAAAAAKIMBAAAA|1999-10-28|2001-10-26|Precisely weak powers could require sometimes important subjects. Rus|0.95|2.89|6007003|amalgmaxi #8|11|archery|8|Sports|148|eingeseought|N/A|7338216seashell09516|thistle|Dozen|Unknown|63|ationbarpriation| +7308|AAAAAAAAKIMBAAAA|2001-10-27||Officials belong for a options; da|9.11|6.55|10006016|corpunivamalg #16|11|musical|10|Electronics|222|ableableable|N/A|08grey04855733760751|purple|Lb|Unknown|65|eingbarpriation| +7309|AAAAAAAANIMBAAAA|1997-10-27||Bizarre relationships should change also today british proposals. Computers position more persistent stan|32.92|22.38|6014006|edu packbrand #6|14|estate|6|Jewelry|607|ationbarcally|N/A|2758224peru857449452|red|Ounce|Unknown|23|n stbarpriation| +7310|AAAAAAAAOIMBAAAA|1997-10-27|2000-10-26|All |1.99|1.01|7013001|exportinameless #1|13|wallpaper|7|Home|18|eingought|N/A|7739orange6548733796|seashell|Gross|Unknown|8|baroughtpriation| +7311|AAAAAAAAOIMBAAAA|2000-10-27||C|2.18|0.67|7013001|edu packscholar #2|4|classical|5|Music|350|barantipri|N/A|8446162242720slate71|sandy|Bunch|Unknown|33|oughtoughtpriation| +7312|AAAAAAAAAJMBAAAA|1997-10-27|1999-10-27|Efficiently running hopes prevent both general, significant con|8.02|4.57|5001001|amalgscholar #1|1|rock|5|Music|59|n stanti|N/A|3518450836991papaya3|wheat|Dram|Unknown|29|ableoughtpriation| +7313|AAAAAAAAAJMBAAAA|1999-10-28|2001-10-26|National, rare waters bring currently external, bad figures. Female standards would match that is in a dogs. Shoes strengthen loyal, late factories. Stairs shall see |4.37|2.66|9013006|exportiunivamalg #6|1|self-help|9|Books|59|n stanti|N/A|3518450836991papaya3|seashell|Ton|Unknown|30|prioughtpriation| +7314|AAAAAAAAAJMBAAAA|2001-10-27||National, rare waters bring currently external, bad figures. Female standards would match that is in a dogs. Shoes strengthen loyal, late factories. Stairs shall see |2.29|1.07|9013006|exportiexporti #1|3|toddlers|3|Children|495|antin stese|petite|3518450836991papaya3|snow|Pallet|Unknown|40|eseoughtpriation| +7315|AAAAAAAADJMBAAAA|1997-10-27||Young categories look grossly so colourfu|3.36|2.18|7002006|importobrand #6|2|bedding|7|Home|583|prieinganti|N/A|92356tomato296828672|yellow|Ounce|Unknown|13|antioughtpriation| +7316|AAAAAAAAEJMBAAAA|1997-10-27|2000-10-26|Local, physical diseases shall walk sometimes always national performances. Substances shall pass once more. Wrong, novel decisions trust and so on wee|9.94|4.17|2003001|exportiimporto #1|3|pants|2|Men|505|antibaranti|petite|92311246428yellow203|saddle|Ounce|Unknown|12|callyoughtpriation| +7317|AAAAAAAAEJMBAAAA|2000-10-27||Local, physical diseases shall walk sometimes always national performances. Substances shall pass once more. Wrong, novel decisions trust and so on wee|50.08|31.04|2003001|amalgscholar #2|3|rock|5|Music|36|antibaranti|N/A|92311246428yellow203|thistle|Unknown|Unknown|12|ationoughtpriation| +7318|AAAAAAAAGJMBAAAA|1997-10-27|1999-10-27|Holidays divorce nearly british years. Toxic, negative figures join for a men. Provisions show either answers; necessary, american organisations|69.08|41.44|6016007|corpbrand #7|16|consignment|6|Jewelry|327|ationablepri|N/A|01469987178440sky790|ghost|Ton|Unknown|53|eingoughtpriation| +7319|AAAAAAAAGJMBAAAA|1999-10-28|2001-10-26|Holidays divorce nearly british years. Toxic, negative figures join for a men. Provisions show either answers; necessary, american organisations|0.90|0.70|5004002|edu packscholar #2|4|classical|5|Music|991|oughtn stn st|N/A|01469987178440sky790|orange|Box|Unknown|13|n stoughtpriation| +7320|AAAAAAAAGJMBAAAA|2001-10-27||Very monetary wages may refer already lo|2.16|1.20|1002001|importoamalg #1|2|fragrances|1|Women|991|oughtn stn st|medium|01469987178440sky790|plum|N/A|Unknown|39|barablepriation| +7321|AAAAAAAAJJMBAAAA|1997-10-27||Grateful activities act too available products; |0.78|0.52|6013002|exportibrand #2|13|loose stones|6|Jewelry|230|barpriable|N/A|079926835wheat115574|seashell|Gram|Unknown|59|oughtablepriation| +7322|AAAAAAAAKJMBAAAA|1997-10-27|2000-10-26|Available charges stay then aspects; dramatic dealers g|4.71|2.44|10015013|scholaramalgamalg #13|15|portable|10|Electronics|8|eing|N/A|511459rose7469689378|purple|Oz|Unknown|35|ableablepriation| +7323|AAAAAAAAKJMBAAAA|2000-10-27||Available charges stay then aspects; dramatic dealers g|3.88|2.44|10015013|amalgedu pack #2|1|womens|4|Shoes|15|antiought|petite|511459rose7469689378|rose|Pallet|Unknown|14|priablepriation| +7324|AAAAAAAAMJMBAAAA|1997-10-27|1999-10-27|More just hands shall not identify however long problems. Characteristics will say now sufficient, early layers; difficult tools could not broaden interesting merchants; ready drugs should appear deta|0.56|0.29|4003001|exportiedu pack #1|3|kids|4|Shoes|292|ablen stable|petite|42042sienna430551116|lawn|Dozen|Unknown|54|eseablepriation| +7325|AAAAAAAAMJMBAAAA|1999-10-28|2001-10-26|Traditional changes could see again sexually initial years. Officers sit animals. Plans must not find most local, possible difficulties. Fiscal, residential costs would see also. Marginal arrangements|0.57|0.29|6001006|amalgcorp #6|3|birdal|6|Jewelry|636|callyprically|N/A|42042sienna430551116|peru|Cup|Unknown|6|antiablepriation| +7326|AAAAAAAAMJMBAAAA|2001-10-27||Traditional changes could see again sexually initial years. Officers sit animals. Plans must not find most local, possible difficulties. Fiscal, residential costs would see also. Marginal arrangements|0.71|0.40|2004001|edu packimporto #1|3|sports-apparel|2|Men|254|eseantiable|petite|749783664611peru1054|royal|Cup|Unknown|29|callyablepriation| +7327|AAAAAAAAPJMBAAAA|1997-10-27||Sensible bases bother up a priorities; tiny hands make importantly effective options. Notable, supreme ra|7.42|4.15|4002002|importoedu pack #2|2|mens|4|Shoes|254|eseantiable|medium|3tan5781627877827101|black|Tbl|Unknown|23|ationablepriation| +7328|AAAAAAAAAKMBAAAA|1997-10-27|2000-10-26|Wild scientific cri|9.52|8.28|4003001|exportiedu pack #1|3|kids|4|Shoes|525|antiableanti|small|86powder933243251989|puff|Pallet|Unknown|50|eingablepriation| +7329|AAAAAAAAAKMBAAAA|2000-10-27||Wild scientific cri|1.71|8.28|4003001|amalgexporti #2|1|newborn|3|Children|525|antiableanti|medium|1510161739linen12316|peach|Gross|Unknown|15|n stablepriation| +7330|AAAAAAAACKMBAAAA|1997-10-27|1999-10-27|Poets take at least on a sales. Other negotiations must become secondly. Reasonable words learn as|2.49|1.66|5003001|exportischolar #1|3|pop|5|Music|335|antipripri|N/A|rose2540412460051680|red|Carton|Unknown|1|barpripriation| +7331|AAAAAAAACKMBAAAA|1999-10-28|2001-10-26|Poets take at least on a sales. Other negotiations must become secondly. Reasonable words learn as|8.47|1.66|3002002|importoexporti #2|2|infants|3|Children|335|antipripri|small|rose2540412460051680|papaya|Tbl|Unknown|25|oughtpripriation| +7332|AAAAAAAACKMBAAAA|2001-10-27||Alone, overall patients would take anyhow good, surprising emissions. Well significan|2.68|0.83|3002002|scholarunivamalg #1|15|fiction|9|Books|250|barantiable|N/A|89337navajo169041277|orange|Box|Unknown|32|ablepripriation| +7333|AAAAAAAAFKMBAAAA|1997-10-27||Labour doubts bring somewhat. Women need retail groups. Practical, individual months show environmental, indian events|2.71|0.97|5002002|importoscholar #2|2|country|5|Music|19|n stought|N/A|571spring17441152226|moccasin|Gram|Unknown|1|pripripriation| +7334|AAAAAAAAGKMBAAAA|1997-10-27|2000-10-26|Front markets ought to reach very academic ways. Then possible words open entirely public products. Softly origin|4.07|3.54|9007003|brandmaxi #3|7|reference|9|Books|280|bareingable|N/A|586lawn5198658014505|red|Ton|Unknown|4|esepripriation| +7335|AAAAAAAAGKMBAAAA|2000-10-27||Front markets ought to reach very academic ways. Then possible words open entirely public products. Softly origin|4.37|3.54|2002002|importoimporto #2|2|shirts|2|Men|280|bareingable|large|586lawn5198658014505|light|Gross|Unknown|16|antipripriation| +7336|AAAAAAAAIKMBAAAA|1997-10-27|1999-10-27|Police stay normal details. Late unemployed items shall not recall. Yards tear important boo|38.52|34.28|2004001|edu packimporto #1|4|sports-apparel|2|Men|793|prin station|medium|4979613804253rose505|firebrick|Dram|Unknown|20|callypripriation| +7337|AAAAAAAAIKMBAAAA|1999-10-28|2001-10-26|Police stay normal details. Late unemployed items shall not recall. Yards tear important boo|2.28|34.28|2004001|exportiamalgamalg #11|13|stereo|10|Electronics|793|prin station|N/A|4979613804253rose505|blue|Gross|Unknown|78|ationpripriation| +7338|AAAAAAAAIKMBAAAA|2001-10-27||Poli|0.40|0.18|2004001|amalgimporto #1|13|accessories|2|Men|456|callyantiese|economy|103ivory925733006377|smoke|Oz|Unknown|4|eingpripriation| +7339|AAAAAAAALKMBAAAA|1997-10-27||Then young ideas leave. Alternative details must answer previously pp.. Of course key |1.40|1.06|4002002|importoedu pack #2|2|mens|4|Shoes|475|antiationese|small|3456steel69666478206|mint|Ounce|Unknown|13|n stpripriation| +7340|AAAAAAAAMKMBAAAA|1997-10-27|2000-10-26|Economic ways reach really at the models. Scientists might draw even major markets. Daily o|7.07|2.96|7007001|brandbrand #1|7|decor|7|Home|497|ationn stese|N/A|64tomato744843969033|green|Lb|Unknown|54|baresepriation| +7341|AAAAAAAAMKMBAAAA|2000-10-27||Just short|6.17|2.96|6004006|edu packcorp #6|4|bracelets|6|Jewelry|497|ationn stese|N/A|5sky9047126855704861|powder|N/A|Unknown|5|oughtesepriation| +7342|AAAAAAAAOKMBAAAA|1997-10-27|1999-10-27|Free, british places used to review yet. Famous colleagues give early.|8.91|7.48|3001001|amalgexporti #1|1|newborn|3|Children|90|barn st|N/A|71581plum69740593743|tan|Dozen|Unknown|10|ableesepriation| +7343|AAAAAAAAOKMBAAAA|1999-10-28|2001-10-26|Free, british places used to review yet. Famous colleagues give early.|6.66|2.33|3001001|amalgbrand #4|1|bathroom|7|Home|83|barn st|N/A|71581plum69740593743|sandy|Cup|Unknown|3|priesepriation| +7344|AAAAAAAAOKMBAAAA|2001-10-27||Precisely complete shots mention complete, following parts. Silent, specialist agents follow too national, easy humans. New, normal sales go more. Financial indi|0.17|2.33|3001001|exportiedu pack #1|3|kids|4|Shoes|10|barn st|medium|71581plum69740593743|white|Pallet|Unknown|11|eseesepriation| +7345|AAAAAAAABLMBAAAA|1997-10-27||External improvements effect so tough words. Great roads cause quickly popular, black stories. Clearly white members might ask enough details. Min|31.74|26.34|9003002|exportimaxi #2|3|computers|9|Books|149|n steseought|N/A|55144508indian377818|tan|Dozen|Unknown|46|antiesepriation| +7346|AAAAAAAACLMBAAAA|1997-10-27|2000-10-26|Teachers will respond both only vital changes. In order dull needs reduce there on a borders. However combined cars must achieve so responsible international c|3.96|2.73|3002001|importoexporti #1|2|infants|3|Children|157|ationantiought|petite|39metallic8377080007|dark|Dram|Unknown|1|callyesepriation| +7347|AAAAAAAACLMBAAAA|2000-10-27||Dark, civil wings admit so foreign, national languages. Mathematical, social thoughts shall not go; resources dare shortly spirits. In order statutory |2.64|1.63|3002001|exporticorp #4|3|gold|6|Jewelry|209|n stbarable|N/A|9382thistle338584673|purple|Dozen|Unknown|44|ationesepriation| +7348|AAAAAAAAELMBAAAA|1997-10-27|1999-10-27|British items land stil|1.47|0.66|10012005|importoamalgamalg #5|12|monitors|10|Electronics|272|ableationable|N/A|960212131red80093938|thistle|Dram|Unknown|41|eingesepriation| +7349|AAAAAAAAELMBAAAA|1999-10-28|2001-10-26|Ancient, direct advantages may not need economic v|2.41|0.77|1001002|amalgamalg #2|12|dresses|1|Women|272|ableationable|petite|960212131red80093938|green|Cup|Unknown|49|n stesepriation| +7350|AAAAAAAAELMBAAAA|2001-10-27||Always large steps like today with a patients. Sufficient records let; operations might provide below. High, human stations say military, mental birds. Sp|1.23|0.77|9003007|exportimaxi #7|3|computers|9|Books|272|ableationable|N/A|4turquoise5439385972|sienna|Gram|Unknown|66|barantipriation| +7351|AAAAAAAAHLMBAAAA|1997-10-27||Able, private products buy merely external females. Women |2.39|1.50|5003002|exportischolar #2|3|pop|5|Music|229|n stableable|N/A|6264sky1624201923251|wheat|Box|Unknown|61|oughtantipriation| +7352|AAAAAAAAILMBAAAA|1997-10-27|2000-10-26|Decisions would maximise actually patients. Then present days shall emerge dark, common birds. New possibilities give other arguments. Historians can protect to |98.13|84.39|10015009|scholaramalgamalg #9|15|portable|10|Electronics|273|priationable|N/A|90291smoke0090656950|medium|Tsp|Unknown|38|ableantipriation| +7353|AAAAAAAAILMBAAAA|2000-10-27||Black, sure families could not help in a seats. English pages entitle; lengthy, full tactics should provide substantially contents. Wooden, soft skills defeat together part|9.80|3.33|10015009|amalgscholar #2|1|rock|5|Music|103|priationable|N/A|90291smoke0090656950|slate|Tbl|Unknown|3|priantipriation| +7354|AAAAAAAAKLMBAAAA|1997-10-27|1999-10-27|Glad, routine times will not serve closely. Alone, other reports search most useful, political criteria. Times shall see. Long, opposite adults used to become differen|5.93|5.21|2002001|importoimporto #1|2|shirts|2|Men|7|ation|extra large|6487royal47981447546|violet|Bundle|Unknown|62|eseantipriation| +7355|AAAAAAAAKLMBAAAA|1999-10-28|2001-10-26|Glad, routine times will not serve closely. Alone, other reports search most useful, political criteria. Times shall see. Long, opposite adults used to become differen|6.06|3.63|2002001|corpnameless #6|16|furniture|7|Home|544|ation|N/A|66717515918white2363|rose|Bundle|Unknown|22|antiantipriation| +7356|AAAAAAAAKLMBAAAA|2001-10-27||Sacred effects call to and fro short|5.16|3.63|7001007|amalgbrand #7|1|bathroom|7|Home|614|ation|N/A|845rosy6476048881353|yellow|Ounce|Unknown|1|callyantipriation| +7357|AAAAAAAANLMBAAAA|1997-10-27||Here low|1.91|0.82|6004006|edu packcorp #6|4|bracelets|6|Jewelry|688|eingeingcally|N/A|6navy426612384445697|papaya|Bunch|Unknown|48|ationantipriation| +7358|AAAAAAAAOLMBAAAA|1997-10-27|2000-10-26|The|8.67|4.33|4003001|exportiedu pack #1|3|kids|4|Shoes|160|barcallyought|medium|97602260steel4179549|papaya|Unknown|Unknown|1|eingantipriation| +7359|AAAAAAAAOLMBAAAA|2000-10-27||Cigarettes used to ra|22.21|15.10|1001002|amalgamalg #2|3|dresses|1|Women|8|eing|large|97602260steel4179549|navajo|Unknown|Unknown|55|n stantipriation| +7360|AAAAAAAAAMMBAAAA|1997-10-27|1999-10-27|Administrative, british troops target further cases. Grand, clear children draw as foreign visits. Other couples die well levels. Other towns say on a policies. Parents decide in a arms. Abroad|2.79|1.22|10001017|amalgunivamalg #17|1|cameras|10|Electronics|121|oughtableought|N/A|lavender226969903225|sienna|Gross|Unknown|79|barcallypriation| +7361|AAAAAAAAAMMBAAAA|1999-10-28|2001-10-26|Products might provide. |2.60|0.91|10001017|exportiamalg #2|3|maternity|1|Women|121|oughtableought|large|lavender226969903225|yellow|Oz|Unknown|22|oughtcallypriation| +7362|AAAAAAAAAMMBAAAA|2001-10-27||Little movements can rise yesterday houses. Rich e|7.50|0.91|10001017|edu packamalg #1|4|swimwear|1|Women|527|ationableanti|small|43purple132494752718|cyan|Ton|Unknown|90|ablecallypriation| +7363|AAAAAAAADMMBAAAA|1997-10-27||Old, single sites come more rich targets. Never cen|7.13|2.21|1002002|importoamalg #2|2|fragrances|1|Women|374|eseationpri|extra large|04526thistle91068699|pink|Bundle|Unknown|46|pricallypriation| +7364|AAAAAAAAEMMBAAAA|1997-10-27|2000-10-26|Foreign, other wines compensate simply. Entirely required days can support experienced, superior children; customers may move. Lov|5.76|2.59|7012005|importonameless #5|12|paint|7|Home|773|priationation|N/A|0941892386835tan2083|spring|Pound|Unknown|33|esecallypriation| +7365|AAAAAAAAEMMBAAAA|2000-10-27||Securities incorporate most gradual, other programmes. Regulations go better cases. Directors eat most creditors. So new arrangements see shapes. Merely r|1.13|2.59|9004010|edu packmaxi #10|12|entertainments|9|Books|773|priationation|N/A|green817182479239600|salmon|Case|Unknown|12|anticallypriation| +7366|AAAAAAAAGMMBAAAA||1999-10-27|||1.74|6011007|amalgbrand #7|11||6|Jewelry|986|callyeingn st|N/A|83570053tomato658535|||Unknown||| +7367|AAAAAAAAGMMBAAAA|1999-10-28|2001-10-26|Accidents may not recall. Here new systems should not wait ago on the suggestions. Also other magistrates ought to market darkly hot, inc minutes. Intense, good hotels shall not make more bold, marrie|22.05|1.74|6011007|exportischolar #2|11|pop|5|Music|986|callyeingn st|N/A|royal749561604031796|violet|Ounce|Unknown|62|ationcallypriation| +7368|AAAAAAAAGMMBAAAA|2001-10-27||S|8.85|1.74|6007001|brandcorp #1|11|pendants|6|Jewelry|554|callyeingn st|N/A|20059670755977peach2|indian|Gram|Unknown|29|eingcallypriation| +7369|AAAAAAAAJMMBAAAA|1997-10-27||Different, social ideas ought to enjoy. No impo|1.66|0.76|3003002|exportiexporti #2|3|toddlers|3|Children|268|eingcallyable|medium|31266813peru81951772|puff|Gram|Unknown|52|n stcallypriation| +7370|AAAAAAAAKMMBAAAA|1997-10-27|2000-10-26|Even big demonstrations visit electronic bodies. Only lives ought to grind again early, wet cars. Dangerous children k|4.11|2.42|5004001|edu packscholar #1|4|classical|5|Music|438|eingpriese|N/A|23386682936sandy6800|cyan|Lb|Unknown|50|barationpriation| +7371|AAAAAAAAKMMBAAAA|2000-10-27||Even big demonstrations visit electronic bodies. Only lives ought to grind again early, wet cars. Dangerous children k|5.20|2.86|5004001|exportiimporto #2|4|pants|2|Men|175|eingpriese|economy|23386682936sandy6800|snow|Tsp|Unknown|42|oughtationpriation| +7372|AAAAAAAAMMMBAAAA|1997-10-27|1999-10-27|Inner,|7.38|5.09|4002001|importoedu pack #1|2|mens|4|Shoes|485|antieingese|extra large|89600213saddle898182|rosy|Ounce|Unknown|59|ableationpriation| +7373|AAAAAAAAMMMBAAAA|1999-10-28|2001-10-26|Inner,|1.45|0.62|4002001|univbrand #6|10|jewelry boxes|6|Jewelry|485|antieingese|N/A|115888585navy7408846|pale|Dozen|Unknown|76|priationpriation| +7374|AAAAAAAAMMMBAAAA|2001-10-27||Industri|83.54|0.62|3003001|exportiexporti #1|10|toddlers|3|Children|485|antieingese|medium|6718315508ghost31464|papaya|Bundle|Unknown|8|eseationpriation| +7375|AAAAAAAAPMMBAAAA|1997-10-27||Important, tall responsibilities may not operate rather exact, empty folk. Numbers dump political teachers. L|7.70|6.62|7002006|importobrand #6|2|bedding|7|Home|138|eingpriought|N/A|41119white1900778858|wheat|Pound|Unknown|26|antiationpriation| +7376|AAAAAAAAANMBAAAA|1997-10-27|2000-10-26|Quite terrible changes can lead then particular responsibilities. Too concerned arts exist to a views.|0.65|0.38|4004001|edu packedu pack #1|4|athletic|4|Shoes|492|ablen stese|medium|97sandy4749192096775|misty|Gram|Unknown|35|callyationpriation| +7377|AAAAAAAAANMBAAAA|2000-10-27||Quite terrible changes can lead then particular responsibilities. Too concerned arts exist to a views.|3.22|0.38|4004001|exportimaxi #8|4|sailing|8|Sports|642|ablen stese|N/A|664462beige812788988|lace|Box|Unknown|25|ationationpriation| +7378|AAAAAAAACNMBAAAA|1997-10-27|1999-10-27|Accused men cannot increase e|6.54|2.55|5003001|exportischolar #1|3|pop|5|Music|128|eingableought|N/A|389949antique7301699|sky|Oz|Unknown|9|eingationpriation| +7379|AAAAAAAACNMBAAAA|1999-10-28|2001-10-26|British, prime soldiers might come. Statutory, wet mountains watch so long-term, private sys|5.19|2.54|5003001|amalgunivamalg #2|1|cameras|10|Electronics|128|eingableought|N/A|415299902910slate550|sandy|N/A|Unknown|50|n stationpriation| +7380|AAAAAAAACNMBAAAA|2001-10-27||Royal lessons give there with a patients. Original, suitab|8.35|5.76|2001001|amalgimporto #1|1|accessories|2|Men|524|eseableanti|petite|415299902910slate550|honeydew|Pound|Unknown|55|bareingpriation| +7381|AAAAAAAAFNMBAAAA|1997-10-27||There minor parties would not run new instit|2.86|1.05|3004002|edu packexporti #2|4|school-uniforms|3|Children|73|priation|small|8702purple9037199430|smoke|Tsp|Unknown|61|oughteingpriation| +7382|AAAAAAAAGNMBAAAA|1997-10-27|2000-10-26|Later anxious detectives might not see. Only bonds improve even interests. Other, common bands go here rural sections. Relative daughters m|47.10|14.60|9008003|namelessmaxi #3|8|romance|9|Books|64|esecally|N/A|878red73756594462105|moccasin|Carton|Unknown|2|ableeingpriation| +7383|AAAAAAAAGNMBAAAA|2000-10-27||Later anxious detectives might not see. Only bonds improve even interests. Other, common bands go here rural sections. Relative daughters m|2.24|1.65|9008003|importoedu pack #2|8|mens|4|Shoes|64|esecally|large|101288indian99499031|smoke|Ounce|Unknown|56|prieingpriation| +7384|AAAAAAAAINMBAAAA|1997-10-27|1999-10-27|Expensive workers should not say accurately old ideas. Later arab types will last still reforms. Ev|1.29|1.04|7014005|edu packnameless #5|14|glassware|7|Home|142|ableeseought|N/A|4602709ghost43653250|white|Cup|Unknown|7|eseeingpriation| +7385|AAAAAAAAINMBAAAA|1999-10-28|2001-10-26|Minutes give. British, inte|6.95|1.04|7014005|importoamalg #2|2|fragrances|1|Women|142|ableeseought|petite|025181saddle44230675|pink|N/A|Unknown|2|antieingpriation| +7386|AAAAAAAAINMBAAAA|2001-10-27||Common, real sizes ensure especially fi|2.81|1.04|6001003|amalgcorp #3|1|birdal|6|Jewelry|142|ableeseought|N/A|025181saddle44230675|red|Oz|Unknown|98|callyeingpriation| +7387|AAAAAAAALNMBAAAA|1997-10-27||Different, low groups might not continue. Only heavy methods try as huge fears; instead civil steps su|1.68|1.12|7013008|exportinameless #8|13|wallpaper|7|Home|116|callyoughtought|N/A|21spring531034234004|antique|Gross|Unknown|14|ationeingpriation| +7388|AAAAAAAAMNMBAAAA|1997-10-27|2000-10-26|Long, elderly candidates steal. Methods maintain often necessary, video-taped breasts. Complex rates make conventions. Sure final damages must |4.38|2.23|5001001|amalgscholar #1|1|rock|5|Music|351|oughtantipri|N/A|7482157moccasin37942|turquoise|Cup|Unknown|55|eingeingpriation| +7389|AAAAAAAAMNMBAAAA|2000-10-27||Long, elderly candidates steal. Methods maintain often necessary, video-taped breasts. Complex rates make conventions. Sure final damages must |4.87|2.23|5001001|brandcorp #6|1|pendants|6|Jewelry|351|oughtantipri|N/A|7482157moccasin37942|tan|Tsp|Unknown|32|n steingpriation| +7390|AAAAAAAAONMBAAAA|1997-10-27|1999-10-27|Reasonable supporters move innocently studies; dangerous, private workshops would not collect fair so|2.24|1.34|10013013|exportiamalgamalg #13|13|stereo|10|Electronics|669|n stcallycally|N/A|7indian9631998735020|saddle|Dram|Unknown|14|barn stpriation| +7391|AAAAAAAAONMBAAAA|1999-10-28|2001-10-26|Reasonable supporters move innocently studies; dangerous, private workshops would not collect fair so|1.50|0.85|3001002|amalgexporti #2|1|newborn|3|Children|432|n stcallycally|large|7961472seashell19506|spring|N/A|Unknown|55|oughtn stpriation| +7392|AAAAAAAAONMBAAAA|2001-10-27||Reasonable supporters move innocently studies; dangerous, private workshops would not collect fair so|42.61|0.85|8001003|amalgnameless #3|1|athletic shoes|8|Sports|29|n stable|N/A|7961472seashell19506|metallic|Tbl|Unknown|30|ablen stpriation| +7393|AAAAAAAABOMBAAAA|1997-10-27||Common are|1.31|0.83|8008004|namelessnameless #4|8|outdoor|8|Sports|271|oughtationable|N/A|814saddle95947104401|snow|Each|Unknown|12|prin stpriation| +7394|AAAAAAAACOMBAAAA|1997-10-27|2000-10-26|European, surprising fears might transmit instructions. Often ugly drugs could not go all right. So particular products would not start mater|9.82|6.57|2001001|amalgimporto #1|1|accessories|2|Men|231|oughtpriable|medium|792293768yellow35794|royal|Pound|Unknown|10|esen stpriation| +7395|AAAAAAAACOMBAAAA|2000-10-27||Academic systems can give generally victorian knees. Straightforward, specific reasons might act on the guards. Of course recent men avoid perhaps|6.04|6.57|2001001|amalgmaxi #6|1|archery|8|Sports|231|oughtpriable|N/A|792293768yellow35794|powder|Gross|Unknown|20|antin stpriation| +7396|AAAAAAAAEOMBAAAA|1997-10-27|1999-10-27|Corporate, only hopes used to anger in general foods; present, roman talks will apply effec|4.27|3.03|8002007|importonameless #7|2|baseball|8|Sports|274|eseationable|N/A|169984695683lemon209|lavender|Gram|Unknown|91|callyn stpriation| +7397|AAAAAAAAEOMBAAAA|1999-10-28|2001-10-26|New experiences should not protect too. Services change. Possible arms could turn a little i|3.10|3.03|8002007|exportibrand #8|2|kids|7|Home|464|esecallyese|N/A|169984695683lemon209|papaya|Gram|Unknown|4|ationn stpriation| +7398|AAAAAAAAEOMBAAAA|2001-10-27||New experiences should not protect too. Services change. Possible arms could turn a little i|7.69|3.03|8002007|scholarmaxi #9|2|fishing|8|Sports|464|esecallyese|N/A|183721576779steel044|sky|Pallet|Unknown|26|eingn stpriation| +7399|AAAAAAAAHOMBAAAA|1997-10-27||Companies reassure on a researchers. Most anonymous factors ought to |0.66|0.44|10010005|univamalgamalg #5|10|memory|10|Electronics|292|ablen stable|N/A|9877210pale427142821|spring|Pallet|Unknown|30|n stn stpriation| +7400|AAAAAAAAIOMBAAAA|1997-10-27|2000-10-26|Investors negotiate rare|3.74|2.73|6002001|importocorp #1|2|diamonds|6|Jewelry|896|callyn steing|N/A|71272963120snow96724|yellow|Gross|Unknown|22|barbareseation| +7401|AAAAAAAAIOMBAAAA|2000-10-27||Investors negotiate rare|2.48|2.73|6002001|namelessunivamalg #7|8|scanners|10|Electronics|896|callyn steing|N/A|71272963120snow96724|snow|Ounce|Unknown|82|oughtbareseation| +7402|AAAAAAAAKOMBAAAA|1997-10-27|1999-10-27|Clearly like sales used to add simply contemporary, other males. Major, fashionable ages control particularly talks. Ot|4.01|1.80|5001001|amalgscholar #1|1|rock|5|Music|143|prieseought|N/A|82wheat0878196843119|plum|Oz|Unknown|2|ablebareseation| +7403|AAAAAAAAKOMBAAAA|1999-10-28|2001-10-26|Earlier mutual patients can want then onwards national persons; directly femal|2.03|1.72|5001001|amalgmaxi #4|1|archery|8|Sports|143|prieseought|N/A|82wheat0878196843119|smoke|Pound|Unknown|19|pribareseation| +7404|AAAAAAAAKOMBAAAA|2001-10-27||Earlier mutual patients can want then onwards national persons; directly femal|1.44|0.73|5001001|amalgexporti #1|1|newborn|3|Children|143|prieseought|medium|82wheat0878196843119|tan|Gram|Unknown|89|esebareseation| +7405|AAAAAAAANOMBAAAA|1997-10-27||Today italian things shall not discuss also again other thousands. New materials shall help|1.53|0.70|7006006|corpbrand #6|6|rugs|7|Home|79|n station|N/A|8657cornflower743797|slate|Dozen|Unknown|32|antibareseation| +7406|AAAAAAAAOOMBAAAA|1997-10-27|2000-10-26|Wings m|1.73|1.47|1001001|amalgamalg #1|1|dresses|1|Women|174|eseationought|medium|87197803674thistle88|goldenrod|Case|Unknown|78|callybareseation| +7407|AAAAAAAAOOMBAAAA|2000-10-27||So single parents find rooms. Corporate, used difficulties believe only solid degrees. Once dead fortunes would locate. Subsequent, strange decades ough|1.95|1.47|3001002|amalgexporti #2|1|newborn|3|Children|348|eseationought|large|powder99079266013321|salmon|Tbl|Unknown|83|ationbareseation| +7408|AAAAAAAAAPMBAAAA|1997-10-27|1999-10-27|Other, common needs could document hitherto hands; private, short consumers stand places. Things wish slow absent men|2.51|0.82|8011009|amalgmaxi #9|11|archery|8|Sports|8|eing|N/A|007smoke758094520928|green|Carton|Unknown|57|eingbareseation| +7409|AAAAAAAAAPMBAAAA|1999-10-28|2001-10-26|Very expenses would not wi|6.24|0.82|8011009|exporticorp #2|11|gold|6|Jewelry|146|eing|N/A|007smoke758094520928|moccasin|Bundle|Unknown|20|n stbareseation| +7410|AAAAAAAAAPMBAAAA|2001-10-27||Very expenses would not wi|8.83|5.73|8011009|corpamalgamalg #16|11|wireless|10|Electronics|63|eing|N/A|007smoke758094520928|royal|Lb|Unknown|43|baroughteseation| +7411|AAAAAAAADPMBAAAA|1997-10-27||Things know alone letters. Flights should tend even jewish fees. Civil plans could not cry also social days; other losses might not pay walls; still able signs should not remove too human |0.90|0.79|10014017|edu packamalgamalg #17|14|automotive|10|Electronics|184|eseeingought|N/A|712366turquoise67213|red|Bundle|Unknown|33|oughtoughteseation| +7412|AAAAAAAAEPMBAAAA|1997-10-27|2000-10-26|Strange do|9.47|4.82|3003001|exportiexporti #1|3|toddlers|3|Children|171|oughtationought|large|8112010sienna5344100|midnight|Oz|Unknown|49|ableoughteseation| +7413|AAAAAAAAEPMBAAAA|2000-10-27||Strange do|2.14|4.82|3003001|importonameless #4|3|baseball|8|Sports|171|oughtationought|N/A|8112010sienna5344100|tan|Case|Unknown|7|prioughteseation| +7414|AAAAAAAAGPMBAAAA|1997-10-27|1999-10-27|Mad groups help alone, famous times. Severe countries would not become current relationships. Married, african championships used to run clearly by a men. Members draw in a groups; accessib|1.74|1.51|6008007|namelesscorp #7|8|mens watch|6|Jewelry|592|ablen stanti|N/A|11seashell9963880119|royal|Unknown|Unknown|4|eseoughteseation| +7415|AAAAAAAAGPMBAAAA|1999-10-28|2001-10-26|Mad groups help alone, famous times. Severe countries would not become current relationships. Married, african championships used to run clearly by a men. Members draw in a groups; accessib|0.72|0.30|9011012|amalgunivamalg #12|11|cooking|9|Books|203|ablen stanti|N/A|02542165427peach5725|midnight|Case|Unknown|51|antioughteseation| +7416|AAAAAAAAGPMBAAAA|2001-10-27||Hands finish easy really likely police. Important rates shall not look wor|51.32|0.30|2004001|edu packimporto #1|4|sports-apparel|2|Men|203|ablen stanti|medium|1974756388violet1756|red|Carton|Unknown|29|callyoughteseation| +7417|AAAAAAAAJPMBAAAA|1997-10-27||States used to walk important, tough devices. Now local ideas should learn present features; special, small patients pay lightly french women. Thus enormous years buy open classes. Inevitably eco|0.53|0.34|10012006|importoamalgamalg #6|12|monitors|10|Electronics|162|ablecallyought|N/A|6711smoke17431118860|medium|Cup|Unknown|8|ationoughteseation| +7418|AAAAAAAAKPMBAAAA|1997-10-27|2000-10-26|There weak stones can belong clos|6.18|4.75|10006007|corpunivamalg #7|6|musical|10|Electronics|22|ableable|N/A|21615388376papaya842|ghost|Tbl|Unknown|33|eingoughteseation| +7419|AAAAAAAAKPMBAAAA|2000-10-27||As afraid benefits should buy assumptions. Dependent systems may pay also uncomfortable teeth. Pl|7.44|2.30|8005010|scholarnameless #10|5|fitness|8|Sports|615|antioughtcally|N/A|21615388376papaya842|cream|Case|Unknown|65|n stoughteseation| +7420|AAAAAAAAMPMBAAAA|1997-10-27|1999-10-27|Championships meet preliminary years. Decisions ask there much political patients. |5.57|2.45|10002009|importounivamalg #9|2|camcorders|10|Electronics|826|callyableeing|N/A|391611lemon956277961|deep|Box|Unknown|14|barableeseation| +7421|AAAAAAAAMPMBAAAA|1999-10-28|2001-10-26|Championships meet preliminary years. Decisions ask there much political patients. |65.35|37.90|10002009|edu packexporti #2|2|school-uniforms|3|Children|826|callyableeing|petite|0sandy52289192604979|sky|Gross|Unknown|29|oughtableeseation| +7422|AAAAAAAAMPMBAAAA||||||10002009||2||1|||callyableeing|extra large||papaya|Tbl|Unknown|20|ableableeseation| +7423|AAAAAAAAPPMBAAAA|1997-10-27||Companies could not make popular, strong paren|72.68|58.87|3001002|amalgexporti #2|1|newborn|3|Children|461|oughtcallyese|extra large|309241136lavender856|peru|Pound|Unknown|11|priableeseation| +7424|AAAAAAAAAANBAAAA|1997-10-27|2000-10-26|New winds maintain still with a experts. Natur|0.48|0.21|1002001|importoamalg #1|2|fragrances|1|Women|132|ablepriought|medium|royal241456759500272|gainsboro|Ton|Unknown|34|eseableeseation| +7425|AAAAAAAAAANBAAAA|2000-10-27||Technical, christian arrangements might record quite raw, urban colleagues. Secondary concepts would continue flowers. Never western deals used to gain by a varieties. Players m|7.26|3.92|10007014|brandunivamalg #14|7|personal|10|Electronics|132|ablepriought|N/A|royal241456759500272|bisque|Box|Unknown|43|antiableeseation| +7426|AAAAAAAACANBAAAA|1997-10-27|1999-10-27|Months cannot lead never unlikely problems. Special characteristics ought to borrow over banks. Patients make only. Networks might not want things. At least bad qualities would not gi|4.71|2.11|8007007|brandnameless #7|7|hockey|8|Sports|5|anti|N/A|2cornsilk35645447504|salmon|Bunch|Unknown|55|callyableeseation| +7427|AAAAAAAACANBAAAA|1999-10-28|2001-10-26|Months cannot lead never unlikely problems. Special characteristics ought to borrow over banks. Patients make only. Networks might not want things. At least bad qualities would not gi|3.91|3.32|7002008|importobrand #8|7|bedding|7|Home|5|anti|N/A|5723959honeydew26305|tomato|Gram|Unknown|58|ationableeseation| +7428|AAAAAAAACANBAAAA|2001-10-27||Months cannot lead never unlikely problems. Special characteristics ought to borrow over banks. Patients make only. Networks might not want things. At least bad qualities would not gi|1.06|3.32|7002008|amalgbrand #5|11|semi-precious|6|Jewelry|5|anti|N/A|465170096slate827796|goldenrod|Cup|Unknown|18|eingableeseation| +7429|AAAAAAAAFANBAAAA|1997-10-27||There only decisions take really royal, joint words. Too public copies must not invent so-called, important aspects. Human, positive organisations would view more male phrases. Relations must n|4.20|2.10|8016010|corpmaxi #10|16|golf|8|Sports|284|eseeingable|N/A|996965lawn4808629763|spring|Cup|Unknown|47|n stableeseation| +7430|AAAAAAAAGANBAAAA|1997-10-27|2000-10-26|Static events act video-taped, pure arms. Heavy, living characteristics release even fa|1.62|1.24|6012007|importobrand #7|12|costume|6|Jewelry|166|callycallyought|N/A|072373395401saddle44|ivory|Each|Unknown|17|barprieseation| +7431|AAAAAAAAGANBAAAA|2000-10-27||Over fond times ask. Yet remote days dry waves. United, only planes ought to answer later significant, competitiv|77.04|36.97|10015003|scholaramalgamalg #3|15|portable|10|Electronics|166|callycallyought|N/A|7167630thistle927823|slate|Oz|Unknown|21|oughtprieseation| +7432|AAAAAAAAIANBAAAA|1997-10-27|1999-10-27|Ordinary, left peasants find socially only, similar others. Dirty, pleased beliefs should use|3.09|1.66|6013001|exportibrand #1|13|loose stones|6|Jewelry|333|pripripri|N/A|6542731spring9352804|sky|Pound|Unknown|39|ableprieseation| +7433|AAAAAAAAIANBAAAA|1999-10-28|2001-10-26|Ordinary, left peasants find socially only, similar others. Dirty, pleased beliefs should use|4.02|1.66|6006002|corpcorp #2|6|rings|6|Jewelry|346|pripripri|N/A|8pink480057256760419|pale|Pallet|Unknown|52|priprieseation| +7434|AAAAAAAAIANBAAAA|2001-10-27||Historic, soviet explanati|8.54|1.66|6006002|namelessnameless #5|8|outdoor|8|Sports|346|pripripri|N/A|9011500white80217234|wheat|Lb|Unknown|59|eseprieseation| +7435|AAAAAAAALANBAAAA|1997-10-27||Chief objects look teachers. Already empi|1.13|0.63|9005008|scholarmaxi #8|5|history|9|Books|141|oughteseought|N/A|papaya33490362819841|wheat|Bunch|Unknown|25|antiprieseation| +7436|AAAAAAAAMANBAAAA|1997-10-27|2000-10-26|Students would take; better expected matters clear then private streets. Holy studies might not indicate in the books. Full, acceptable boo|72.59|24.68|7010007|univnameless #7|10|flatware|7|Home|181|oughteingought|N/A|royal742475788555498|smoke|Bunch|Unknown|22|callyprieseation| +7437|AAAAAAAAMANBAAAA|2000-10-27||Dates shall repair well social governments. Other, english taxes struggle then years. Signs lose most separate areas. Efficient children go years. Long weeks refer sharp. Names make t|1.96|24.68|4003002|exportiedu pack #2|10|kids|4|Shoes|181|oughteingought|small|01410tomato335439254|grey|Gram|Unknown|1|ationprieseation| +7438|AAAAAAAAOANBAAAA|1997-10-27|1999-10-27|Real departments help away. Easy, local |1.24|1.10|2004001|edu packimporto #1|4|sports-apparel|2|Men|167|ationcallyought|small|9yellow4427957924960|white|Lb|Unknown|14|eingprieseation| +7439|AAAAAAAAOANBAAAA|1999-10-28|2001-10-26|Real departments help away. Easy, local |4.30|2.36|2004001|edu packbrand #10|4|curtains/drapes|7|Home|167|ationcallyought|N/A|9yellow4427957924960|purple|Gram|Unknown|13|n stprieseation| +7440|AAAAAAAAOANBAAAA|2001-10-27||Real departments help away. Easy, local |5.62|2.36|2004001|amalgamalg #1|4|dresses|1|Women|1|ationcallyought|medium|9yellow4427957924960|violet|Each|Unknown|16|bareseeseation| +7441|AAAAAAAABBNBAAAA|1997-10-27||Goods dismantle really modern, polish letters. Rich, immense questions could match as a trees. Forms must not smell now strong, br|0.43|0.35|2004002|edu packimporto #2|4|sports-apparel|2|Men|220|barableable|petite|247241258783571pale4|snow|Pallet|Unknown|44|oughteseeseation| +7442|AAAAAAAACBNBAAAA|1997-10-27|2000-10-26|German systems give sides; helpful goals must know local, modest husbands. Initiatives watch quite fundamental years; also associated pounds take recent, major pr|4.65|2.83|3001001|amalgexporti #1|1|newborn|3|Children|110|baroughtought|small|6226756blue163844158|orange|Dram|Unknown|45|ableeseeseation| +7443|AAAAAAAACBNBAAAA|2000-10-27||Rates cause most comparisons. Forwards quiet scientists help applic|4.62|3.00|3001001|amalgamalgamalg #15|11|disk drives|10|Electronics|37|baroughtought|N/A|0686khaki88101194723|sandy|Bunch|Unknown|36|prieseeseation| +7444|AAAAAAAAEBNBAAAA|1997-10-27|1999-10-27|Wel|3.07|2.57|4001001|amalgedu pack #1|1|womens|4|Shoes|118|eingoughtought|medium|3190761seashell09705|grey|Pallet|Unknown|30|eseeseeseation| +7445|AAAAAAAAEBNBAAAA|1999-10-28|2001-10-26|Heavy, |7.24|2.57|10005017|scholarunivamalg #17|5|karoke|10|Electronics|118|eingoughtought|N/A|531midnight367714007|royal|Ounce|Unknown|40|antieseeseation| +7446|AAAAAAAAEBNBAAAA|2001-10-27||At least particular years cut rapidly on the officers. Full sentences should continue more for a reasons|6.14|4.35|10005017|exportiamalg #1|3|maternity|1|Women|53|prianti|small|854838773855314lace7|peach|Carton|Unknown|18|callyeseeseation| +7447|AAAAAAAAHBNBAAAA|1997-10-27||Benefits shout only normal, private songs. Stairs order imperial individuals.|1.68|1.47|1002002|importoamalg #2|2|fragrances|1|Women|283|prieingable|medium|536turquoise56152796|steel|Case|Unknown|62|ationeseeseation| +7448|AAAAAAAAIBNBAAAA|1997-10-27|2000-10-26|Possible, old words prepare parents. Possible prices go too to the children. Over good authorities involve also. Most beautiful profits might divide too severe poor id|6.11|4.52|1004001|edu packamalg #1|4|swimwear|1|Women|276|callyationable|medium|9062peru907260087544|pale|Gross|Unknown|33|eingeseeseation| +7449|AAAAAAAAIBNBAAAA|2000-10-27||Below low computers raise like|0.24|0.10|1004001|exportinameless #10|3|basketball|8|Sports|279|callyationable|N/A|9062peru907260087544|gainsboro|Pound|Unknown|31|n steseeseation| +7450|AAAAAAAAKBNBAAAA|1997-10-27|1999-10-27|Funds get. Simple, local children might show manufacturers; structures emerge later either novel agencies. Brilliant, other patients cannot ring projects. Then upper aims |1.65|0.74|3001001|amalgexporti #1|1|newborn|3|Children|390|barn stpri|petite|1wheat39609849270092|lace|Cup|Unknown|53|barantieseation| +7451|AAAAAAAAKBNBAAAA|1999-10-28|2001-10-26|Funds get. Simple, local children might show manufacturers; structures emerge later either novel agencies. Brilliant, other patients cannot ring projects. Then upper aims |8.79|0.74|3001001|exportiedu pack #2|3|kids|4|Shoes|27|barn stpri|medium|1wheat39609849270092|saddle|Dram|Unknown|73|oughtantieseation| +7452|AAAAAAAAKBNBAAAA|2001-10-27||Funds get. Simple, local children might show manufacturers; structures emerge later either novel agencies. Brilliant, other patients cannot ring projects. Then upper aims |0.88|0.35|9004001|edu packmaxi #1|3|entertainments|9|Books|27|barn stpri|N/A|1wheat39609849270092|burnished|Oz|Unknown|6|ableantieseation| +7453|AAAAAAAANBNBAAAA|1997-10-27||Signals will not dig for the women. Re|2.36|2.10|3002002|importoexporti #2|2|infants|3|Children|244|eseeseable|large|40590344512smoke8558|blush|Tbl|Unknown|64|priantieseation| +7454|AAAAAAAAOBNBAAAA|1997-10-27|2000-10-26|Regular, dark duties used to stare again. Also stupid voices produce thus here|1.38|0.88|5002001|importoscholar #1|2|country|5|Music|680|bareingcally|N/A|6845915260689lace773|royal|Ton|Unknown|46|eseantieseation| +7455|AAAAAAAAOBNBAAAA|2000-10-27||Regular, dark duties used to stare again. Also stupid voices produce thus here|0.56|0.88|5002001|amalgexporti #2|1|newborn|3|Children|101|bareingcally|medium|6845915260689lace773|violet|N/A|Unknown|57|antiantieseation| +7456|AAAAAAAAACNBAAAA|1997-10-27|1999-10-27|Now apparent forces can|31.75|13.65|4002001|importoedu pack #1|2|mens|4|Shoes|22|ableable|small|11893289443spring986|yellow|Dozen|Unknown|24|callyantieseation| +7457|AAAAAAAAACNBAAAA|1999-10-28|2001-10-26|Now apparent forces can|3.18|13.65|9009006|maximaxi #6|2|science|9|Books|601|oughtbarcally|N/A|11893289443spring986|puff|Each|Unknown|23|ationantieseation| +7458|AAAAAAAAACNBAAAA|2001-10-27||Gold|5.31|13.65|7013009|exportinameless #9|2|wallpaper|7|Home|601|oughtbarcally|N/A|783plum1554299975145|wheat|Tbl|Unknown|36|eingantieseation| +7459|AAAAAAAADCNBAAAA|1997-10-27||Writers would decrease however in a problems. Elsewhere standard areas |8.82|6.35|8004010|edu packnameless #10|4|camping|8|Sports|137|ationpriought|N/A|3grey096046654935936|white|Ton|Unknown|29|n stantieseation| +7460|AAAAAAAAECNBAAAA|1997-10-27|2000-10-26|International houses |7.70|3.92|6003005|exporticorp #5|3|gold|6|Jewelry|42|ableese|N/A|004powder38639618463|tomato|Dozen|Unknown|37|barcallyeseation| +7461|AAAAAAAAECNBAAAA|2000-10-27||Pointedly cheap rivers break then as soft decades. Fing|0.48|3.92|6003005|univamalgamalg #16|10|memory|10|Electronics|173|priationought|N/A|0169111purple0885439|pale|Dram|Unknown|11|oughtcallyeseation| +7462|AAAAAAAAGCNBAAAA|1997-10-27|1999-10-27|Emotional sources can use on a countr|6.82|3.61|4002001|importoedu pack #1|2|mens|4|Shoes|545|antieseanti|small|7816044ghost22632434|rosy|Tsp|Unknown|24|ablecallyeseation| +7463|AAAAAAAAGCNBAAAA|1999-10-28|2001-10-26|Emotional sources can use on a countr|3.27|1.73|4002001|importomaxi #12|2|business|9|Books|545|antieseanti|N/A|7816044ghost22632434|steel|Gram|Unknown|91|pricallyeseation| +7464|AAAAAAAAGCNBAAAA|2001-10-27||Emotional sources can use on a countr|3.10|2.54|4002001|maxicorp #1|2|womens watch|6|Jewelry|545|antieseanti|N/A|231lavender778036090|blue|Tbl|Unknown|71|esecallyeseation| +7465|AAAAAAAAJCNBAAAA|1997-10-27||Here future lessons should note critical origins. Forces compete with a falls. Recent, administrative w|8.02|3.84|2004002|edu packimporto #2|4|sports-apparel|2|Men|150|barantiought|large|9336074244peach55715|red|N/A|Unknown|44|anticallyeseation| +7466|AAAAAAAAKCNBAAAA|1997-10-27|2000-10-26|Almost unable supporters go others. Empty parties enter no lo|2.31|1.27|7003007|exportibrand #7|3|kids|7|Home|234|esepriable|N/A|76midnight2559508286|indian|Case|Unknown|31|callycallyeseation| +7467|AAAAAAAAKCNBAAAA|2000-10-27||Also chief events need with the students. Relations could fill very in a trees. More clear readers k|93.60|43.05|6012004|importobrand #4|3|costume|6|Jewelry|761|esepriable|N/A|917514violet51039212|papaya|Oz|Unknown|2|ationcallyeseation| +7468|AAAAAAAAMCNBAAAA|1997-10-27|1999-10-27|Well working companies will sell metropolitan, running interests. Right relative children might refer even christian miners. Stages can analyse yards. Always afraid features will express |6.73|2.89|8011009|amalgmaxi #9|11|archery|8|Sports|16|callyought|N/A|71891131677ghost8023|maroon|Box|Unknown|57|eingcallyeseation| +7469|AAAAAAAAMCNBAAAA|1999-10-28|2001-10-26|Poor, lucky years look ever vari|0.45|0.28|9008006|namelessmaxi #6|11|romance|9|Books|16|callyought|N/A|71891131677ghost8023|steel|Gross|Unknown|38|n stcallyeseation| +7470|AAAAAAAAMCNBAAAA|2001-10-27||Famili|5.80|0.28|8013001|exportimaxi #1|11|sailing|8|Sports|162|callyought|N/A|8377sandy27749022365|peach|Oz|Unknown|68|barationeseation| +7471|AAAAAAAAPCNBAAAA|1997-10-27||Areas may clea|2.32|1.11|7012002|importonameless #2|12|paint|7|Home|198|eingn stought|N/A|974peru1565233440429|red|Gross|Unknown|49|oughtationeseation| +7472|AAAAAAAAADNBAAAA|1997-10-27|2000-10-26|Net, regional lawyers would construct well different, different tools. Soon free meals distinguish pretty, sweet services. Horizontal contributions help. Again big supplies replace conc|3.03|1.45|7011003|amalgnameless #3|11|accent|7|Home|641|oughtesecally|N/A|3225292metallic27667|smoke|Case|Unknown|40|ableationeseation| +7473|AAAAAAAAADNBAAAA|2000-10-27||Net, regional lawyers would construct well different, different tools. Soon free meals distinguish pretty, sweet services. Horizontal contributions help. Again big supplies replace conc|7.01|5.95|7011003|univmaxi #4|10|pools|8|Sports|234|oughtesecally|N/A|3225292metallic27667|purple|Case|Unknown|3|priationeseation| +7474|AAAAAAAACDNBAAAA|1997-10-27|1999-10-27|Rational, sof|1.64|1.11|8012005|importomaxi #5|12|guns|8|Sports|257|ationantiable|N/A|43379grey58899654570|rosy|Bundle|Unknown|57|eseationeseation| +7475|AAAAAAAACDNBAAAA|1999-10-28|2001-10-26|Rational, sof|0.66|0.47|8003006|exportinameless #6|12|basketball|8|Sports|647|ationesecally|N/A|yellow57856820939171|steel|Box|Unknown|29|antiationeseation| +7476|AAAAAAAACDNBAAAA|2001-10-27||Different champions could not continue far possible areas. Often important conclusions might not revive then houses. Central women must consider ve|9.28|8.25|10003014|exportiunivamalg #14|3|dvd/vcr players|10|Electronics|647|ationesecally|N/A|3876998942spring8620|thistle|Gross|Unknown|42|callyationeseation| +7477|AAAAAAAAFDNBAAAA|1997-10-27||Large, necessary companies make delib|1.37|1.21|7005008|scholarbrand #8|5|blinds/shades|7|Home|913|prioughtn st|N/A|76374793azure9965853|yellow|Carton|Unknown|12|ationationeseation| +7478|AAAAAAAAGDNBAAAA|1997-10-27|2000-10-26|Towns compete longer. Sizes get. Alone, ready edges pass at once. Walls see windows. Minutes remain gently military games. Possible, white relations fade. Ideas c|7.33|6.37|2002001|importoimporto #1|2|shirts|2|Men|97|ationn st|medium|8513154253464hot7707|red|Gross|Unknown|7|eingationeseation| +7479|AAAAAAAAGDNBAAAA|2000-10-27||Towns compete longer. Sizes get. Alone, ready edges pass at once. Walls see windows. Minutes remain gently military games. Possible, white relations fade. Ideas c|2.43|6.37|2002001|edu packexporti #2|4|school-uniforms|3|Children|979|ationn st|extra large|8513154253464hot7707|wheat|Ounce|Unknown|31|n stationeseation| +7480|AAAAAAAAIDNBAAAA|1997-10-27|1999-10-27|Expensive reasons shall not carry hardly ri|4.59|1.46|7005001|scholarbrand #1|5|blinds/shades|7|Home|281|oughteingable|N/A|156143snow6317889193|white|Gross|Unknown|74|bareingeseation| +7481|AAAAAAAAIDNBAAAA|1999-10-28|2001-10-26|Expensive reasons shall not carry hardly ri|7.71|6.55|2002002|importoimporto #2|5|shirts|2|Men|262|ablecallyable|extra large|53535peach0003002670|yellow|Ton|Unknown|45|oughteingeseation| +7482|AAAAAAAAIDNBAAAA|2001-10-27||Yet significant spirits could play much able options. Also small systems may demonstrate. Effective, natural lights shall set. Remains see only. Main changes would contr|96.12|38.44|3003001|exportiexporti #1|3|toddlers|3|Children|150|barantiought|large|53535peach0003002670|rose|Tbl|Unknown|23|ableeingeseation| +7483|AAAAAAAALDNBAAAA|1997-10-27||Rates shall release. Ranks buy only possible months. Wonderful, easy persons would practise particularly between a times. National benefits|5.70|2.50|2004002|edu packimporto #2|4|sports-apparel|2|Men|24|eseable|medium|758misty744574309568|royal|Lb|Unknown|90|prieingeseation| +7484|AAAAAAAAMDNBAAAA|1997-10-27|2000-10-26|Other forests may secure long with a shelves. Animals would not help. Maybe distant courts go official days. Molecules shall motivate in a regions. Young, aware years might mean yet le|0.59|0.21|5002001|importoscholar #1|2|country|5|Music|145|antieseought|N/A|55864679lime04129447|turquoise|Case|Unknown|14|eseeingeseation| +7485|AAAAAAAAMDNBAAAA|2000-10-27||Small parents will not suspect. Prices should observe. S|9.16|3.75|6004006|edu packcorp #6|2|bracelets|6|Jewelry|941|antieseought|N/A|yellow90447611874549|seashell|Bundle|Unknown|7|antieingeseation| +7486|AAAAAAAAODNBAAAA|1997-10-27|1999-10-27|Regulations will tell eventually extra pounds|0.62|0.31|9011011|amalgunivamalg #11|11|cooking|9|Books|9|n st|N/A|4330171salmon5666655|purple|Pallet|Unknown|85|callyeingeseation| +7487|AAAAAAAAODNBAAAA|1999-10-28|2001-10-26|Regulations will tell eventually extra pounds|4.96|3.76|9011011|namelessunivamalg #8|11|scanners|10|Electronics|503|pribaranti|N/A|4330171salmon5666655|royal|Case|Unknown|20|ationeingeseation| +7488|AAAAAAAAODNBAAAA|2001-10-27||Present, angry wheels can speak criteria. Funds shall know tall, diplomatic criteria. Stars might tempt then long markets. Shares|0.09|3.76|9011011|edu packmaxi #9|11|tennis|8|Sports|503|pribaranti|N/A|4330171salmon5666655|pink|Oz|Unknown|61|eingeingeseation| +7489|AAAAAAAABENBAAAA|1997-10-27||Strong others will shout then ancient sums. Simple, mathematical millions hear free,|2.84|1.84|5002002|importoscholar #2|2|country|5|Music|57|ationanti|N/A|979thistle6569778934|seashell|Unknown|Unknown|13|n steingeseation| +7490|AAAAAAAACENBAAAA|1997-10-27|2000-10-26|Months ought to challenge softly hours. Different firms give in a volumes. Features may explain very increased doctors. Only se|4.44|2.22|5001001|amalgscholar #1|1|rock|5|Music|105|antibarought|N/A|tomato78462605335766|puff|Each|Unknown|16|barn steseation| +7491|AAAAAAAACENBAAAA|2000-10-27||Central, fair countries see thousands. Aspects build also for a children|14.86|9.65|5001001|amalgbrand #4|11|semi-precious|6|Jewelry|105|antibarought|N/A|tomato78462605335766|navy|Carton|Unknown|9|oughtn steseation| +7492|AAAAAAAAEENBAAAA|1997-10-27|1999-10-27|Rounds ought to ask doubtful c|4.72|4.20|7009003|maxibrand #3|9|mattresses|7|Home|764|esecallyation|N/A|8steel48680584821900|salmon|Carton|Unknown|54|ablen steseation| +7493|AAAAAAAAEENBAAAA|1999-10-28|2001-10-26|Rounds ought to ask doubtful c|8.50|7.22|7009003|corpnameless #4|6|football|8|Sports|764|esecallyation|N/A|8steel48680584821900|turquoise|Tsp|Unknown|7|prin steseation| +7494|AAAAAAAAEENBAAAA|2001-10-27||Rounds ought to ask doubtful c|4.44|7.22|4004001|edu packedu pack #1|4|athletic|4|Shoes|78|esecallyation|large|8steel48680584821900|cream|N/A|Unknown|18|esen steseation| +7495|AAAAAAAAHENBAAAA|1997-10-27||Everywhere concerned arms will not see tight exciting, real enemies; old, sudden types may prove yet in a developments. Terms will not appreciate so great situations. National, ne|2.48|1.51|1004002|edu packamalg #2|4|swimwear|1|Women|169|n stcallyought|extra large|red13705920025279890|white|Oz|Unknown|16|antin steseation| +7496|AAAAAAAAIENBAAAA|1997-10-27|2000-10-26|Experiences help by a goods. Black prices live all sure, technological employers. Short, clever breasts play old memories. Strong refugees tell basically feet. Things earn in a persons.|6.52|2.08|9009009|maximaxi #9|9|science|9|Books|973|priationn st|N/A|5284031564plum323465|red|Oz|Unknown|66|callyn steseation| +7497|AAAAAAAAIENBAAAA|2000-10-27||Economic, concerned food|93.76|2.08|3001002|amalgexporti #2|1|newborn|3|Children|973|priationn st|extra large|pale8737634113798955|red|Cup|Unknown|17|ationn steseation| +7498|AAAAAAAAKENBAAAA|1997-10-27|1999-10-27|Securities go. Aware, negative facilities preserve accidentally. Ahead vast sisters can bargain however up a feet. Sick leaders would favour schools. Most appropriate eyes engage no doubt still |0.84|0.56|4002001|importoedu pack #1|2|mens|4|Shoes|954|eseantin st|medium|042682014034spring91|medium|Pound|Unknown|51|eingn steseation| +7499|AAAAAAAAKENBAAAA|1999-10-28|2001-10-26|Securities go. Aware, negative facilities preserve accidentally. Ahead vast sisters can bargain however up a feet. Sick leaders would favour schools. Most appropriate eyes engage no doubt still |71.69|27.24|1002002|importoamalg #2|2|fragrances|1|Women|954|eseantin st|medium|042682014034spring91|slate|Each|Unknown|13|n stn steseation| +7500|AAAAAAAAKENBAAAA|2001-10-27||Personal visitors work states. Values take other police. Organisations stand major peasants. |0.54|0.28|1002002|edu packamalg #1|4|swimwear|1|Women|954|eseantin st|medium|042682014034spring91|red|Oz|Unknown|5|barbarantiation| +7501|AAAAAAAANENBAAAA|1997-10-27||Coming, western ladies will train difficult, small examples. Very small models find recent,|9.39|3.38|4001002|amalgedu pack #2|1|womens|4|Shoes|419|n stoughtese|N/A|08236700838062navy56|wheat|Bunch|Unknown|30|oughtbarantiation| +7502|AAAAAAAAOENBAAAA|1997-10-27|2000-10-26|Annually seco|83.66|61.07|1001001|amalgamalg #1|1|dresses|1|Women|262|ablecallyable|medium|7249974098cornsilk01|light|Pound|Unknown|28|ablebarantiation| +7503|AAAAAAAAOENBAAAA|2000-10-27||Annually seco|9.30|61.07|9008004|namelessmaxi #4|8|romance|9|Books|204|esebarable|N/A|7249974098cornsilk01|peru|Bunch|Unknown|2|pribarantiation| +7504|AAAAAAAAAFNBAAAA|1997-10-27|1999-10-27|Minutes accept often servants. Prime, other ways see away from a trousers; simply current conditio|3.93|1.25|5004001|edu packscholar #1|4|classical|5|Music|143|prieseought|N/A|5deep345582124722616|salmon|Dram|Unknown|37|esebarantiation| +7505|AAAAAAAAAFNBAAAA|1999-10-28|2001-10-26|Jobs might not carry that is to say prior to the schools. Sole, cheerful types must not give very. Inland links should vary before a schemes. Vital directors reduce particularly after the companie|0.11|1.25|5004001|edu packedu pack #2|4|athletic|4|Shoes|143|prieseought|medium|5deep345582124722616|slate|Lb|Unknown|25|antibarantiation| +7506|AAAAAAAAAFNBAAAA|2001-10-27||Jobs might not carry that is to say prior to the schools. Sole, cheerful types must not give very. Inland links should vary before a schemes. Vital directors reduce particularly after the companie|5.64|2.93|2002001|importoimporto #1|4|shirts|2|Men|241|oughteseable|medium|890620652691smoke830|snow|Ton|Unknown|26|callybarantiation| +7507|AAAAAAAADFNBAAAA|1997-10-27||Over wonderful pounds see still in a enterprises; useful families follow strong conditions. Full o|0.56|0.26|10014011|edu packamalgamalg #11|14|automotive|10|Electronics|167|ationcallyought|N/A|5649821579papaya2642|papaya|Pound|Unknown|93|ationbarantiation| +7508|AAAAAAAAEFNBAAAA|1997-10-27|2000-10-26|Radical advantages should not develop agents. Originally gastric groups claim maybe pale police. Almost popular concepts might step instead. Financial varieties should st|24.00|12.72|5004001|edu packscholar #1|4|classical|5|Music|185|antieingought|N/A|497779royal172953528|plum|Each|Unknown|12|eingbarantiation| +7509|AAAAAAAAEFNBAAAA|2000-10-27||Radical advantages should not develop agents. Originally gastric groups claim maybe pale police. Almost popular concepts might step instead. Financial varieties should st|4.57|4.11|5004001|scholarunivamalg #10|4|fiction|9|Books|185|antieingought|N/A|62938698417violet189|pale|Pound|Unknown|30|n stbarantiation| +7510|AAAAAAAAGFNBAAAA|1997-10-27|1999-10-27|Great, free relations distinguish|2.63|0.81|2002001|importoimporto #1|2|shirts|2|Men|719|n stoughtation|medium|447royal545361111274|sienna|Dram|Unknown|47|baroughtantiation| +7511|AAAAAAAAGFNBAAAA|1999-10-28|2001-10-26|Great, free relations distinguish|5.51|0.81|2002001|importoscholar #2|2|country|5|Music|3|pri|N/A|447royal545361111274|navajo|Carton|Unknown|33|oughtoughtantiation| +7512|AAAAAAAAGFNBAAAA|2001-10-27||Barely white humans shall scrape industrial movements; equal, pure affairs cannot watch. Strong acts ought to get less from a parents. Great, great workers must see|3.20|0.81|3004001|edu packexporti #1|4|school-uniforms|3|Children|577|pri|small|447royal545361111274|purple|Gram|Unknown|42|ableoughtantiation| +7513|AAAAAAAAJFNBAAAA|1997-10-27||Old, wooden negotiations would not protect never only human methods; maps may not warn at a sentences. Never different females discus|62.80|54.63|3002002|importoexporti #2|2|infants|3|Children|644|eseesecally|small|7530761thistle864590|saddle|Unknown|Unknown|22|prioughtantiation| +7514|AAAAAAAAKFNBAAAA|1997-10-27|2000-10-26|Goods should understand. More available mistakes mig|8.52|6.39|6008003|namelesscorp #3|8|mens watch|6|Jewelry|144|eseeseought|N/A|1244258678white49876|floral|Dram|Unknown|64|eseoughtantiation| +7515|AAAAAAAAKFNBAAAA|2000-10-27||Large eyes think remarkably initial tra|4.71|6.39|6008003|amalgexporti #2|1|newborn|3|Children|651|eseeseought|small|1244258678white49876|dark|Box|Unknown|4|antioughtantiation| +7516|AAAAAAAAMFNBAAAA|1997-10-27|1999-10-27|Perfect, good regulations convince tall eyes; thick marks fall careers. Poor, other police might not force now for a industries. Good, crucial solutions will predict parties; schemes |2.84|1.36|5003001|exportischolar #1|3|pop|5|Music|84|eseeing|N/A|078061555300pale3109|rose|Bunch|Unknown|47|callyoughtantiation| +7517|AAAAAAAAMFNBAAAA|1999-10-28|2001-10-26|Teams write well; pink standards approve. Then appropriate changes could not say very other great members. Islamic girls approve funds. Sources may meet socially themes.|0.75|1.36|5003001|exportiamalg #2|3|maternity|1|Women|84|eseeing|economy|078061555300pale3109|floral|Carton|Unknown|21|ationoughtantiation| +7518|AAAAAAAAMFNBAAAA|2001-10-27||Cheerful, right|3.58|2.14|6004007|edu packcorp #7|3|bracelets|6|Jewelry|84|eseeing|N/A|078061555300pale3109|plum|Dram|Unknown|65|eingoughtantiation| +7519|AAAAAAAAPFNBAAAA|1997-10-27||Long classes buy more subtle, professional problems. Now special thoughts follow cases. Just sig|1.73|1.34|8016010|corpmaxi #10|16|golf|8|Sports|597|ationn stanti|N/A|092350hot84096511737|magenta|Cup|Unknown|22|n stoughtantiation| +7520|AAAAAAAAAGNBAAAA|1997-10-27|2000-10-26|Institutions will get; values would go eventually worried chapters. Opposite at|75.91|49.34|9014009|edu packunivamalg #9|14|sports|9|Books|565|anticallyanti|N/A|8755525peach89935548|smoke|Dozen|Unknown|12|barableantiation| +7521|AAAAAAAAAGNBAAAA|2000-10-27||Early countries might say now electrical, eastern awards. Bright, american children shall lead enough proportions. Before old days might dip. Numbers o|72.93|43.75|9014009|exportischolar #2|3|pop|5|Music|565|anticallyanti|N/A|8755525peach89935548|puff|N/A|Unknown|2|oughtableantiation| +7522|AAAAAAAACGNBAAAA|1997-10-27|1999-10-27|Social problems will convince. Already poor members broaden firmly possible, perfect bodies. Serious, potential church|1.19|0.41|4001001|amalgedu pack #1|1|womens|4|Shoes|443|prieseese|medium|24693saddle569839232|orchid|Each|Unknown|52|ableableantiation| +7523|AAAAAAAACGNBAAAA|1999-10-28|2001-10-26|Stable others get that affairs; short modes listen already also royal periods. Various cattle benefit to the sciences. Visitors |4.45|4.00|10005010|scholarunivamalg #10|5|karoke|10|Electronics|156|prieseese|N/A|01163rose79210316709|sienna|Dozen|Unknown|53|priableantiation| +7524|AAAAAAAACGNBAAAA|2001-10-27||Stable others get that affairs; short modes listen already also royal periods. Various cattle benefit to the sciences. Visitors |2.91|2.18|2001001|amalgimporto #1|5|accessories|2|Men|252|prieseese|economy|174409954040steel990|steel|Gross|Unknown|65|eseableantiation| +7525|AAAAAAAAFGNBAAAA|1997-10-27||Policies shall not cause early negotiations. Red topics can recall as by a engineers. Excellent, german boundaries say only at a products. Possible groups receive simply central, far occasions. |14.07|6.47|4004002|edu packedu pack #2|4|athletic|4|Shoes|83|prieing|medium|881657729slate341266|peach|Box|Unknown|53|antiableantiation| +7526|AAAAAAAAGGNBAAAA|1997-10-27|2000-10-26|Slow patterns would step still part-time |3.35|2.44|7015007|scholarnameless #7|15|tables|7|Home|683|prieingcally|N/A|340white843353862694|pink|Oz|Unknown|31|callyableantiation| +7527|AAAAAAAAGGNBAAAA|2000-10-27||Students must care later. Social, blue men break changes. Serious events should leave inevitably good scientists. More clear police feel more patients|4.94|2.51|7015007|importoedu pack #2|15|mens|4|Shoes|206|callybarable|medium|1276777honeydew99047|sandy|Pallet|Unknown|7|ationableantiation| +7528|AAAAAAAAIGNBAAAA|1997-10-27|1999-10-27|Suitable streets choke in a members. Features offer yesterday oral members. Relations mention very waiting, italian ey|4.94|3.26|5004001|edu packscholar #1|4|classical|5|Music|9|n st|N/A|9103589891183snow110|puff|Gram|Unknown|54|eingableantiation| +7529|AAAAAAAAIGNBAAAA|1999-10-28|2001-10-26|National soldiers will not see usually. Common children ought to begin especially grounds. Only past changes will not scan hence just alleged polici|97.68|3.26|7010010|univnameless #10|4|flatware|7|Home|9|n st|N/A|9103589891183snow110|turquoise|Each|Unknown|1|n stableantiation| +7530|AAAAAAAAIGNBAAAA|2001-10-27||National soldiers will not see usually. Common children ought to begin especially grounds. Only past changes will not scan hence just alleged polici|4.71|2.44|7010010|exportiimporto #1|3|pants|2|Men|36|callypri|petite|20rose66466992244601|slate|Pallet|Unknown|48|barpriantiation| +7531|AAAAAAAALGNBAAAA|1997-10-27||Straig|46.34|35.68|8001002|amalgnameless #2|1|athletic shoes|8|Sports|418|eingoughtese|N/A|53705tomato420516339|maroon|Lb|Unknown|15|oughtpriantiation| +7532|AAAAAAAAMGNBAAAA|1997-10-27|2000-10-26|Agencies affect in common mountains. Clear eyes could work today models; cars get i|8.68|4.25|8009003|maxinameless #3|9|optics|8|Sports|143|prieseought|N/A|1252568rosy240201288|sandy|Oz|Unknown|23|ablepriantiation| +7533|AAAAAAAAMGNBAAAA|2000-10-27||Helpful affairs attend certain limits. Obvious, informal emotions can need only genuine, educational years. British |8.50|4.25|8009003|amalgscholar #2|1|rock|5|Music|443|prieseese|N/A|70salmon592615836195|peru|Oz|Unknown|14|pripriantiation| +7534|AAAAAAAAOGNBAAAA|1997-10-27|1999-10-27|Controls cause forward unable, other restrictions. As local details fall foreign names. Often proposed expectations m|5.95|3.39|4004001|edu packedu pack #1|4|athletic|4|Shoes|357|ationantipri|small|26308383turquoise346|thistle|Pound|Unknown|56|esepriantiation| +7535|AAAAAAAAOGNBAAAA|1999-10-28|2001-10-26|Controls cause forward unable, other restrictions. As local details fall foreign names. Often proposed expectations m|3.41|1.43|8015006|scholarmaxi #6|4|fishing|8|Sports|249|n steseable|N/A|9557tan7124165050350|green|Lb|Unknown|39|antipriantiation| +7536|AAAAAAAAOGNBAAAA|2001-10-27||Dark, white fingers love even reliable objectives. Democratic circumstances walk consc|78.71|25.97|3001001|amalgexporti #1|4|newborn|3|Children|192|n steseable|petite|9557tan7124165050350|honeydew|Dozen|Unknown|43|callypriantiation| +7537|AAAAAAAABHNBAAAA|1997-10-27||Corporate, inner plans could leave only for a prices. Already final|2.53|2.04|9015002|scholarunivamalg #2|15|fiction|9|Books|303|pribarpri|N/A|white286379160451361|moccasin|Bundle|Unknown|10|ationpriantiation| +7538|AAAAAAAACHNBAAAA|1997-10-27|2000-10-26|National, happy secrets make below standards; brief programmes look to a pages. Huge, various thousands could disclose possibly most surprising considerations. Goods will think|1.58|0.75|5003001|exportischolar #1|3|pop|5|Music|436|callypriese|N/A|7798sienna0408317083|hot|Lb|Unknown|6|eingpriantiation| +7539|AAAAAAAACHNBAAAA|2000-10-27||Conditions would impose effective, unlikely insects. Children must not return so annual, optimistic systems. Ready, different cities will maintain secondly difficult|1.42|0.75|5003001|maxiunivamalg #9|3|televisions|10|Electronics|436|callypriese|N/A|6722maroon9381767364|indian|Lb|Unknown|48|n stpriantiation| +7540|AAAAAAAAEHNBAAAA|1997-10-27|1999-10-27|Top, british hopes can appreciate as operations. Most clear songs stimulate initially. Farmers get as during a feet. Just direct|0.44|0.24|3002001|importoexporti #1|2|infants|3|Children|87|ationeing|petite|3341227841312tomato6|dark|Cup|Unknown|22|bareseantiation| +7541|AAAAAAAAEHNBAAAA|1999-10-28|2001-10-26|Top, british hopes can appreciate as operations. Most clear songs stimulate initially. Farmers get as during a feet. Just direct|2.77|1.44|3002001|maximaxi #6|9|science|9|Books|87|ationeing|N/A|3341227841312tomato6|pink|Tsp|Unknown|15|oughteseantiation| +7542|AAAAAAAAEHNBAAAA|2001-10-27||Top, british hopes can appreciate as operations. Most clear songs stimulate initially. Farmers get as during a feet. Just direct|3.90|2.06|3002001|amalgamalg #1|9|dresses|1|Women|87|ationeing|medium|3341227841312tomato6|lawn|Tsp|Unknown|66|ableeseantiation| +7543|AAAAAAAAHHNBAAAA|1997-10-27||Operatio|8.19|3.52|10006013|corpunivamalg #13|6|musical|10|Electronics|340|baresepri|N/A|peach463410301373277|thistle|Bunch|Unknown|17|prieseantiation| +7544|AAAAAAAAIHNBAAAA|1997-10-27|2000-10-26|Crucial citizens use. Presumably false issues need tools. Unhappy, important plans get|0.55|0.17|10003014|exportiunivamalg #14|3|dvd/vcr players|10|Electronics|195|antin stought|N/A|6685tan3757482872456|lawn|Dram|Unknown|85|eseeseantiation| +7545|AAAAAAAAIHNBAAAA|2000-10-27||Crucial citizens use. Presumably false issues need tools. Unhappy, important plans get|0.51|0.32|5004002|edu packscholar #2|4|classical|5|Music|195|antin stought|N/A|6685tan3757482872456|orchid|Tbl|Unknown|53|antieseantiation| +7546|AAAAAAAAKHNBAAAA|1997-10-27|1999-10-27|Services must feel double, yellow examples. Anywhere|0.77|0.41|5003001|exportischolar #1|3|pop|5|Music|242|ableeseable|N/A|351912690929286mint9|saddle|Gross|Unknown|30|callyeseantiation| +7547|AAAAAAAAKHNBAAAA|1999-10-28|2001-10-26|Services must feel double, yellow examples. Anywhere|2.50|0.41|5003001|importoedu pack #2|3|mens|4|Shoes|213|prioughtable|extra large|7smoke79302450916510|rose|Oz|Unknown|84|ationeseantiation| +7548|AAAAAAAAKHNBAAAA|2001-10-27||So national conditions prevent stil|8.69|0.41|5003001|amalgscholar #1|3|rock|5|Music|213|prioughtable|N/A|peru3918731591633706|rosy|Lb|Unknown|57|eingeseantiation| +7549|AAAAAAAANHNBAAAA|1997-10-27||Fast, national preferences know in no problems. Applications should develop old, likely eyes. Regulations may not ensure light workers. Long cases|4.96|4.36|8010010|univmaxi #10|10|pools|8|Sports|512|ableoughtanti|N/A|249white225405892302|pale|Unknown|Unknown|25|n steseantiation| +7550|AAAAAAAAOHNBAAAA|1997-10-27|2000-10-26|Eyes could not find join|1.23|1.00|10010003|univamalgamalg #3|10|memory|10|Electronics|678|eingationcally|N/A|66838785384517tan585|medium|Carton|Unknown|75|barantiantiation| +7551|AAAAAAAAOHNBAAAA|2000-10-27||Visual points trace then splendid computers. Clo|3.11|1.11|6008008|namelesscorp #8|10|mens watch|6|Jewelry|368|eingcallypri|N/A|09889449290sandy5810|lime|Dram|Unknown|16|oughtantiantiation| +7552|AAAAAAAAAINBAAAA|1997-10-27|1999-10-27|Overall|4.35|2.13|5004001|edu packscholar #1|4|classical|5|Music|531|oughtprianti|N/A|99253798323192smoke4|snow|Tsp|Unknown|61|ableantiantiation| +7553|AAAAAAAAAINBAAAA|1999-10-28|2001-10-26|Overall|2.69|2.13|5004001|importoimporto #2|4|shirts|2|Men|140|oughtprianti|medium|99253798323192smoke4|sienna|Gross|Unknown|17|priantiantiation| +7554|AAAAAAAAAINBAAAA|2001-10-27||Overall|9.82|2.13|5004001|importoamalgamalg #7|4|monitors|10|Electronics|246|oughtprianti|N/A|18157seashell8872638|hot|Pound|Unknown|81|eseantiantiation| +7555|AAAAAAAADINBAAAA|1997-10-27||Moreover other samples may not cause impressive funds. Also cle|4.34|3.77|5002002|importoscholar #2|2|country|5|Music|140|bareseought|N/A|747075smoke536947502|moccasin|Each|Unknown|65|antiantiantiation| +7556|AAAAAAAAEINBAAAA|1997-10-27|2000-10-26|Chemical, other times dismiss for the events. Middle, attractive opportunities lead virtually away american years. Full services involve left reasons. Now white times could not seem despite a meeting|7.97|3.98|5001001|amalgscholar #1|1|rock|5|Music|221|oughtableable|N/A|96319858021orange738|salmon|Dozen|Unknown|22|callyantiantiation| +7557|AAAAAAAAEINBAAAA|2000-10-27||Fundamental plans would get very british homes; new, main rates know top, good borders. Today conscious years can make easily everyd|26.98|15.91|3002002|importoexporti #2|1|infants|3|Children|75|antiation|large|7511sandy89158934546|green|Pound|Unknown|24|ationantiantiation| +7558|AAAAAAAAGINBAAAA|1997-10-27|1999-10-27|Much funny candidates smell by a weeks. Forms know please for a classes. There important la|1.74|1.33|7003009|exportibrand #9|3|kids|7|Home|261|oughtcallyable|N/A|06745045626purple393|seashell|Oz|Unknown|35|eingantiantiation| +7559|AAAAAAAAGINBAAAA|1999-10-28|2001-10-26|Very russian years love also racial, independent lips. For example new men give however. Relations would not survive surprised materials. Aroun|8.76|7.27|7003009|edu packscholar #2|4|classical|5|Music|236|callypriable|N/A|6344seashell75037132|moccasin|Bunch|Unknown|8|n stantiantiation| +7560|AAAAAAAAGINBAAAA|2001-10-27||Later good brothers might not meet now too possible charges. Necessary numbers refrain therefore immediately possible rooms. Well constant words s|1.39|7.27|7003009|exportibrand #1|3|kids|7|Home|236|callypriable|N/A|6344seashell75037132|sandy|Oz|Unknown|23|barcallyantiation| +7561|AAAAAAAAJINBAAAA|1997-10-27||That central men know independent authorities. Just new rights can make only such as a companies. Studies can stay a|9.89|3.16|7002002|importobrand #2|2|bedding|7|Home|201|oughtbarable|N/A|3176287961tomato2269|papaya|Ounce|Unknown|10|oughtcallyantiation| +7562|AAAAAAAAKINBAAAA|1997-10-27|2000-10-26|Often minor paintings would achieve highly direct, single details; quite mental customers might like before political financial devices; howe|8.77|3.85|2004001|edu packimporto #1|4|sports-apparel|2|Men|188|eingeingought|medium|208ghost713898099531|hot|Dram|Unknown|26|ablecallyantiation| +7563|AAAAAAAAKINBAAAA|2000-10-27||Accused areas pick quite generally quick relations; rules prevent. Now busy debts meet port|5.05|2.37|7011004|amalgnameless #4|4|accent|7|Home|188|eingeingought|N/A|88117370207509sandy0|pink|Case|Unknown|79|pricallyantiation| +7564|AAAAAAAAMINBAAAA|1997-10-27|1999-10-27|Together similar products chall|4.62|1.38|4004001|edu packedu pack #1|4|athletic|4|Shoes|104|esebarought|petite|42royal3450248272911|tomato|Pallet|Unknown|40|esecallyantiation| +7565|AAAAAAAAMINBAAAA|1999-10-28|2001-10-26|Urban, small payments should not get. Current, uncomfortable legs could talk only simply good courses. Military, c|0.85|0.67|4004001|corpunivamalg #1|6|musical|10|Electronics|104|esebarought|N/A|787233steel246113560|pale|Unknown|Unknown|54|anticallyantiation| +7566|AAAAAAAAMINBAAAA|2001-10-27||Then high pains should not invade so. Good, pure copies would take steadily flowers. Hours resist. Small trends could not shoot. Then other christians must form yest|0.93|0.79|10013002|exportiamalgamalg #2|13|stereo|10|Electronics|104|esebarought|N/A|787233steel246113560|turquoise|Case|Unknown|40|callycallyantiation| +7567|AAAAAAAAPINBAAAA|1997-10-27||New managers believe about conventional, famous women. Direct, excellent borders must snap pairs. Alone national miles contain more armed expenses; much young eggs must agree then dogs; firms us|7.22|6.42|4004002|edu packedu pack #2|4|athletic|4|Shoes|326|callyablepri|petite|219240yellow95788998|plum|Ounce|Unknown|49|ationcallyantiation| +7568|AAAAAAAAAJNBAAAA|1997-10-27|2000-10-26|Poor appeals may |3.32|1.26|1002001|importoamalg #1|2|fragrances|1|Women|244|eseeseable|large|50white6111372575751|peach|N/A|Unknown|21|eingcallyantiation| +7569|AAAAAAAAAJNBAAAA|2000-10-27||Early, imperial jobs must not leave in a days. Softly statutory effects should press very. Dollars would mark. Old months will su|0.56|0.44|5003002|exportischolar #2|3|pop|5|Music|244|eseeseable|N/A|50white6111372575751|metallic|Gram|Unknown|15|n stcallyantiation| +7570|AAAAAAAACJNBAAAA|1997-10-27|1999-10-27|Final poin|1.38|0.64|3001001|amalgexporti #1|1|newborn|3|Children|199|n stn stought|large|1indian6217344391191|rose|Gram|Unknown|76|barationantiation| +7571|AAAAAAAACJNBAAAA|1999-10-28|2001-10-26|Brothers begin at the finge|3.86|3.01|3001001|brandmaxi #12|1|reference|9|Books|199|n stn stought|N/A|1indian6217344391191|wheat|Lb|Unknown|13|oughtationantiation| +7572|AAAAAAAACJNBAAAA|2001-10-27||A little major citizens ought to tolerate also. Styles want occasionally. Sho|6.24|3.01|5001001|amalgscholar #1|1|rock|5|Music|104|esebarought|N/A|1indian6217344391191|sky|Box|Unknown|10|ableationantiation| +7573|AAAAAAAAFJNBAAAA|1997-10-27||Growing, left horses burn. Hungry talks confirm largely causes. Existing schools used to claim well only new levels. Women may revive small areas; arms w|2.28|1.98|5003002|exportischolar #2|3|pop|5|Music|172|ableationought|N/A|12860322475plum19109|turquoise|Gross|Unknown|7|priationantiation| +7574|AAAAAAAAGJNBAAAA|1997-10-27|2000-10-26|White facts tell narrowly however rich prices. Specifically rich costs should wear certain details. |1.65|0.66|1001001|amalgamalg #1|1|dresses|1|Women|34|esepri|small|86371017185tan071843|pale|Ton|Unknown|33|eseationantiation| +7575|AAAAAAAAGJNBAAAA|2000-10-27||Only primary modules must arrest attitudes. Also public projects shall join for a cha|7.46|4.69|4001002|amalgedu pack #2|1|womens|4|Shoes|587|esepri|small|2738300035642misty29|salmon|Dram|Unknown|33|antiationantiation| +7576|AAAAAAAAIJNBAAAA|1997-10-27|1999-10-27|Equal situations write very in the tears. Long representative |4.24|3.60|8002007|importonameless #7|2|baseball|8|Sports|470|barationese|N/A|59644059steel3704050|thistle|Oz|Unknown|18|callyationantiation| +7577|AAAAAAAAIJNBAAAA|1999-10-28|2001-10-26|Equal situations write very in the tears. Long representative |2.08|1.83|8002007|importoscholar #2|2|country|5|Music|470|barationese|N/A|59644059steel3704050|red|Pallet|Unknown|3|ationationantiation| +7578|AAAAAAAAIJNBAAAA|2001-10-27||Rules must work there. Over we|3.48|1.83|3003001|exportiexporti #1|2|toddlers|3|Children|691|oughtn stcally|small|3431720steel15069359|powder|Dozen|Unknown|55|eingationantiation| +7579|AAAAAAAALJNBAAAA|1997-10-27||More than british years inhi|90.64|46.22|10001015|amalgunivamalg #15|1|cameras|10|Electronics|82|ableeing|N/A|47sandy2526132615791|turquoise|Lb|Unknown|64|n stationantiation| +7580|AAAAAAAAMJNBAAAA|1997-10-27|2000-10-26|Forces should not like too; only sufficient facts might not think vast police. Here occasional weeks see exte|2.47|1.77|5004001|edu packscholar #1|4|classical|5|Music|303|pribarpri|N/A|40445seashell5567791|peru|Bunch|Unknown|8|bareingantiation| +7581|AAAAAAAAMJNBAAAA|2000-10-27||Forces should not like too; only sufficient facts might not think vast police. Here occasional weeks see exte|5.58|3.73|9001010|amalgmaxi #10|1|arts|9|Books|303|pribarpri|N/A|40445seashell5567791|salmon|Pallet|Unknown|29|oughteingantiation| +7582|AAAAAAAAOJNBAAAA|1997-10-27|1999-10-27|Even potential procedures continue legal, novel commitments; assets posse|5.45|1.74|3004001|edu packexporti #1|4|school-uniforms|3|Children|540|bareseanti|medium|7707281325tomato8214|pink|Lb|Unknown|38|ableeingantiation| +7583|AAAAAAAAOJNBAAAA|1999-10-28|2001-10-26|Again new costs extend closely capable patients. Medical doctors cannot use thus increased payments. Notes construct yet in a banks. Earnings should not sa|9.99|1.74|3002002|importoexporti #2|4|infants|3|Children|540|bareseanti|medium|7707281325tomato8214|wheat|Cup|Unknown|43|prieingantiation| +7584|AAAAAAAAOJNBAAAA|2001-10-27||Again new costs extend closely capable patients. Medical doctors cannot use thus increased payments. Notes construct yet in a banks. Earnings should not sa|9.46|1.74|7006005|corpbrand #5|6|rugs|7|Home|191|bareseanti|N/A|40129plum75781051939|thistle|Ton|Unknown|79|eseeingantiation| +7585|AAAAAAAABKNBAAAA|1997-10-27||Operations decide lo|7.61|6.46|10013004|exportiamalgamalg #4|13|stereo|10|Electronics|231|oughtpriable|N/A|7maroon9709569506309|salmon|Box|Unknown|6|antieingantiation| +7586|AAAAAAAACKNBAAAA|1997-10-27|2000-10-26|Today labour characters used to like quite black difficult papers; ages catch low, common matters. Sick judges might make both opposite seeds. Public, foreign proceedings must not rescue of c|3.30|1.71|9007003|brandmaxi #3|7|reference|9|Books|590|barn stanti|N/A|sienna85975188248335|sienna|Carton|Unknown|22|callyeingantiation| +7587|AAAAAAAACKNBAAAA|2000-10-27||Today labour characters used to like quite black difficult papers; ages catch low, common matters. Sick judges might make both opposite seeds. Public, foreign proceedings must not rescue of c|6.04|1.71|9007003|edu packunivamalg #4|14|sports|9|Books|590|barn stanti|N/A|sienna85975188248335|puff|Unknown|Unknown|30|ationeingantiation| +7588|AAAAAAAAEKNBAAAA|1997-10-27|1999-10-27|Scientific, different sides bring major, h|3.54|1.16|8004009|edu packnameless #9|4|camping|8|Sports|428|eingableese|N/A|315914513spring47473|linen|Unknown|Unknown|28|eingeingantiation| +7589|AAAAAAAAEKNBAAAA|1999-10-28|2001-10-26|Golden, financial generations should mind however instead yellow dates; british faces should not squeeze then; firm, important drivers know passages. Reasonable, severe hands me|2.38|1.76|8004009|amalgedu pack #2|1|womens|4|Shoes|4|ese|small|315914513spring47473|burlywood|Cup|Unknown|43|n steingantiation| +7590|AAAAAAAAEKNBAAAA|2001-10-27||Golden, financial generations should mind however instead yellow dates; british faces should not squeeze then; firm, important drivers know passages. Reasonable, severe hands me|4.61|3.78|3001001|amalgexporti #1|1|newborn|3|Children|4|ese|small|9plum997159335109702|grey|Bundle|Unknown|33|barn stantiation| +7591|AAAAAAAAHKNBAAAA|1997-10-27||Fine, white men vary almost natural, running companies. Obviously lost colleagues stop alike. White, impor|6.80|3.12|2003002|exportiimporto #2|3|pants|2|Men|613|prioughtcally|economy|582cornflower3829053|steel|Oz|Unknown|93|oughtn stantiation| +7592|AAAAAAAAIKNBAAAA|1997-10-27|2000-10-26|Chief payments used to decorate |5.08|4.16|7004003|edu packbrand #3|4|curtains/drapes|7|Home|114|eseoughtought|N/A|204rose5462699371252|antique|Dram|Unknown|16|ablen stantiation| +7593|AAAAAAAAIKNBAAAA|2000-10-27||Maximum variations detail almost working e|4.59|4.16|7004003|univunivamalg #10|10|travel|9|Books|274|eseationable|N/A|204rose5462699371252|slate|Pallet|Unknown|46|prin stantiation| +7594|AAAAAAAAKKNBAAAA|1997-10-27|1999-10-27|Old pp. mind. Full dogs should say northern strangers. Patterns provide comprehensive, public powers. Men would not know completely expensive conventions|3.44|1.41|10006013|corpunivamalg #13|6|musical|10|Electronics|204|esebarable|N/A|orange83015987895258|turquoise|Tbl|Unknown|21|esen stantiation| +7595|AAAAAAAAKKNBAAAA|1999-10-28|2001-10-26|Old pp. mind. Full dogs should say northern strangers. Patterns provide comprehensive, public powers. Men would not know completely expensive conventions|1.62|1.15|10008014|namelessunivamalg #14|6|scanners|10|Electronics|204|esebarable|N/A|8729228638138lime040|plum|N/A|Unknown|28|antin stantiation| +7596|AAAAAAAAKKNBAAAA|2001-10-27||Old pp. mind. Full dogs should say northern strangers. Patterns provide comprehensive, public powers. Men would not know completely expensive conventions|1.48|1.33|4003001|exportiedu pack #1|3|kids|4|Shoes|146|callyeseought|petite|8729228638138lime040|seashell|Dozen|Unknown|1|callyn stantiation| +7597|AAAAAAAANKNBAAAA|1997-10-27||Evolutionar|21.03|6.51|10010016|univamalgamalg #16|10|memory|10|Electronics|443|prieseese|N/A|52368721169025slate4|powder|Each|Unknown|56|ationn stantiation| +7598|AAAAAAAAOKNBAAAA|1997-10-27|2000-10-26|Quiet, small questions might see for a scales. Literary, national questions stay|4.98|4.38|5002001|importoscholar #1|2|country|5|Music|86|callyeing|N/A|029rose0409989720310|slate|Lb|Unknown|5|eingn stantiation| +7599|AAAAAAAAOKNBAAAA|2000-10-27||Quiet, small questions might see for a scales. Literary, national questions stay|2.21|4.38|5002002|importoscholar #2|2|country|5|Music|291|oughtn stable|N/A|388278373moccasin811|peach|Gross|Unknown|4|n stn stantiation| +7600|AAAAAAAAALNBAAAA|1997-10-27|1999-10-27|Sexual procedures should run emotions. Names may help. So scottish minutes consider initially high services. Together young millions complete sets. Old employees|1.94|1.62|8002001|importonameless #1|2|baseball|8|Sports|6|cally|N/A|385796820278antique0|white|Unknown|Unknown|19|barbarcallyation| +7601|AAAAAAAAALNBAAAA|1999-10-28|2001-10-26|Sexual procedures should run emotions. Names may help. So scottish minutes consider initially high services. Together young millions complete sets. Old employees|1.17|0.90|8002001|namelessbrand #2|8|lighting|7|Home|261|cally|N/A|4357cornflower059542|plum|Carton|Unknown|23|oughtbarcallyation| +7602|AAAAAAAAALNBAAAA|2001-10-27||Sexual procedures should run emotions. Names may help. So scottish minutes consider initially high services. Together young millions complete sets. Old employees|5.84|0.90|8002001|amalgexporti #1|8|newborn|3|Children|822|cally|extra large|38493362387241sandy4|seashell|Case|Unknown|21|ablebarcallyation| +7603|AAAAAAAADLNBAAAA|1997-10-27||Areas shall not follow achievements. Certain, political meetings hear just hor|0.13|0.11|4003002|exportiedu pack #2|3|kids|4|Shoes|630|barprically|extra large|7peru735826113332568|white|Carton|Unknown|59|pribarcallyation| +7604|AAAAAAAAELNBAAAA|1997-10-27|2000-10-26|Recently other times tell still here guilty workers. S|3.06|2.47|3004001|edu packexporti #1|4|school-uniforms|3|Children|357|ationantipri|medium|0986969231green29330|steel|Bundle|Unknown|7|esebarcallyation| +7605|AAAAAAAAELNBAAAA|2000-10-27||Recently other times tell still here guilty workers. S|9.52|2.47|3004002|edu packexporti #2|4|school-uniforms|3|Children|3|ationantipri|large|0986969231green29330|ghost|Tbl|Unknown|73|antibarcallyation| +7606|AAAAAAAAGLNBAAAA|1997-10-27|1999-10-27|Positions practice even obviously human p|65.94|39.56|1002001|importoamalg #1|2|fragrances|1|Women|457|ationantiese|small|9293185501sandy32827|spring|Gram|Unknown|27|callybarcallyation| +7607|AAAAAAAAGLNBAAAA|1999-10-28|2001-10-26|Positions practice even obviously human p|77.67|39.56|1002001|scholarunivamalg #9|5|karoke|10|Electronics|295|ationantiese|N/A|9293185501sandy32827|smoke|Carton|Unknown|99|ationbarcallyation| +7608|AAAAAAAAGLNBAAAA|2001-10-27||Positions practice even obviously human p|1.35|0.48|6001001|amalgcorp #1|5|birdal|6|Jewelry|551|ationantiese|N/A|9293185501sandy32827|steel|Dram|Unknown|34|eingbarcallyation| +7609|AAAAAAAAJLNBAAAA|1997-10-27||Subjects may think on a times. New, back services will keep along a runs; trees engage financial models; again limited men might join certainly. R|4.12|1.64|9006002|corpmaxi #2|6|parenting|9|Books|499|n stn stese|N/A|87886997613purple189|olive|Bundle|Unknown|27|n stbarcallyation| +7610|AAAAAAAAKLNBAAAA|1997-10-27|2000-10-26|Old centres tour. Big, traditional restauran|2.10|1.28|6012003|importobrand #3|12|costume|6|Jewelry|308|eingbarpri|N/A|09680935811321slate1|smoke|Pallet|Unknown|47|baroughtcallyation| +7611|AAAAAAAAKLNBAAAA|2000-10-27||Old centres tour. Big, traditional restauran|0.99|0.59|6002004|importocorp #4|2|diamonds|6|Jewelry|173|eingbarpri|N/A|09680935811321slate1|red|Case|Unknown|33|oughtoughtcallyation| +7612|AAAAAAAAMLNBAAAA|1997-10-27|1999-10-27|Perfectly spanish costs must say markedly big national horses. Different, moving men should iron equally black countries. Lawyers turn similarly. |36.89|16.60|1003001|exportiamalg #1|3|maternity|1|Women|540|bareseanti|small|9431699739339sky4149|violet|Dram|Unknown|44|ableoughtcallyation| +7613|AAAAAAAAMLNBAAAA|1999-10-28|2001-10-26|Perfectly spanish costs must say markedly big national horses. Different, moving men should iron equally black countries. Lawyers turn similarly. |3.65|16.60|10014015|edu packamalgamalg #15|14|automotive|10|Electronics|82|bareseanti|N/A|71017736994033puff36|orchid|Dram|Unknown|5|prioughtcallyation| +7614|AAAAAAAAMLNBAAAA|2001-10-27||Perfectly spanish costs must say markedly big national horses. Different, moving men should iron equally black countries. Lawyers turn similarly. |3.20|2.33|10014015|edu packcorp #7|14|bracelets|6|Jewelry|82|ableeing|N/A|71017736994033puff36|sky|Ton|Unknown|48|eseoughtcallyation| +7615|AAAAAAAAPLNBAAAA|1997-10-27||Forthcoming results destroy initially sorry groups. There numerous women glance great, strange animals. Famous, good fields clear original, early attemp|2.94|2.46|1004002|edu packamalg #2|4|swimwear|1|Women|335|antipripri|small|2076475snow213792971|orange|Dozen|Unknown|29|antioughtcallyation| +7616|AAAAAAAAAMNBAAAA|1997-10-27|2000-10-26|Local, direct times can go also. American lines mention further calculations. Russian devices advise sources. Political initiatives may learn just new machines. |3.42|2.39|9015009|scholarunivamalg #9|15|fiction|9|Books|450|barantiese|N/A|1630moccasin24823139|smoke|Oz|Unknown|1|callyoughtcallyation| +7617|AAAAAAAAAMNBAAAA|2000-10-27||Prepared bodies cannot achieve durin|5.57|3.50|9015009|importoimporto #2|15|shirts|2|Men|458|barantiese|medium|80498119759powder407|powder|Dozen|Unknown|9|ationoughtcallyation| +7618|AAAAAAAACMNBAAAA|1997-10-27|1999-10-27|Churches would want harder large aspects. Points turn small banks. Gates find yes|53.29|18.11|2004001|edu packimporto #1|4|sports-apparel|2|Men|297|ationn stable|small|710rose5631089845022|deep|Ton|Unknown|12|eingoughtcallyation| +7619|AAAAAAAACMNBAAAA|1999-10-28|2001-10-26|Churches would want harder large aspects. Points turn small banks. Gates find yes|3.71|18.11|2003002|exportiimporto #2|3|pants|2|Men|297|ationn stable|extra large|4wheat37343065372389|midnight|N/A|Unknown|28|n stoughtcallyation| +7620|AAAAAAAACMNBAAAA|2001-10-27||More bright churches provide hard techniques; well european times charge against a feelings. Difficult, vulnerabl|9.74|5.06|8001001|amalgnameless #1|1|athletic shoes|8|Sports|297|ationn stable|N/A|9361696877sienna7373|peach|Cup|Unknown|66|barablecallyation| +7621|AAAAAAAAFMNBAAAA|1997-10-27||So informal words ought to sanction full, inc problems. Different figures should prove new, other years. Forms ought to announce as total characteristics. Particular men must not provide ve|76.62|24.51|5001002|amalgscholar #2|1|rock|5|Music|110|baroughtought|N/A|41turquoise465000411|papaya|Pallet|Unknown|55|oughtablecallyation| +7622|AAAAAAAAGMNBAAAA|1997-10-27|2000-10-26|Stones must not serve along a shoulders; direct traditional bombs impro|8.61|3.09|10011007|amalgamalgamalg #7|11|disk drives|10|Electronics|214|eseoughtable|N/A|53696037rosy77780993|yellow|Each|Unknown|27|ableablecallyation| +7623|AAAAAAAAGMNBAAAA|2000-10-27||Stones must not serve along a shoulders; direct traditional bombs impro|93.73|3.09|3003002|exportiexporti #2|3|toddlers|3|Children|214|eseoughtable|petite|1001444011239smoke50|sky|Cup|Unknown|54|priablecallyation| +7624|AAAAAAAAIMNBAAAA|1997-10-27|1999-10-27|Black, necessary acts will claim over. Just painful lines prove national, detailed activiti|4.78|3.53|9005005|scholarmaxi #5|5|history|9|Books|633|priprically|N/A|9456925002893spring3|moccasin|Pallet|Unknown|23|eseablecallyation| +7625|AAAAAAAAIMNBAAAA|1999-10-28|2001-10-26|Also alone objectives could challenge for a questions. International, dark prices can set ground|4.02|3.53|9005005|edu packedu pack #2|4|athletic|4|Shoes|633|priprically|medium|9456925002893spring3|puff|Lb|Unknown|68|antiablecallyation| +7626|AAAAAAAAIMNBAAAA|2001-10-27||Improvements must not need also american pages. Initiatives used to execute british |4.05|3.53|9005005|edu packunivamalg #11|4|audio|10|Electronics|633|priprically|N/A|3133moccasin75720992|seashell|Dram|Unknown|5|callyablecallyation| +7627|AAAAAAAALMNBAAAA|1997-10-27||Less than strong minutes can see. At last frequent farms try too soviet, cogn|0.84|0.47|1003002|exportiamalg #2|3|maternity|1|Women|114|eseoughtought|medium|09691177902cream0133|plum|Oz|Unknown|11|ationablecallyation| +7628|AAAAAAAAMMNBAAAA|1997-10-27|2000-10-26|Years|0.97|0.38|7001009|amalgbrand #9|1|bathroom|7|Home|121|oughtableought|N/A|8095221313white67382|saddle|Gross|Unknown|80|eingablecallyation| +7629|AAAAAAAAMMNBAAAA|2000-10-27||Simple practitioners support obviously. Yesterday busy workers emerge indeed facilities. Quite similar errors press probabl|1.88|0.94|7001009|edu packimporto #2|1|sports-apparel|2|Men|121|oughtableought|large|8095221313white67382|linen|Ton|Unknown|12|n stablecallyation| +7630|AAAAAAAAOMNBAAAA|1997-10-27|1999-10-27|Open,|45.02|13.50|4001001|amalgedu pack #1|1|womens|4|Shoes|517|ationoughtanti|extra large|43253402khaki5427105|plum|Each|Unknown|60|barpricallyation| +7631|AAAAAAAAOMNBAAAA|1999-10-28|2001-10-26|Open,|61.16|46.48|10011016|amalgamalgamalg #16|11|disk drives|10|Electronics|813|prioughteing|N/A|58883456450antique02|sandy|Tbl|Unknown|9|oughtpricallyation| +7632|AAAAAAAAOMNBAAAA|2001-10-27||Policies must not walk probably now other systems. American ports would join later conventional circumstances. Necessarily little risks could trap subsequently past, other mo|6.65|3.85|10011016|maxicorp #1|9|womens watch|6|Jewelry|233|prioughteing|N/A|58883456450antique02|puff|Bunch|Unknown|42|ablepricallyation| +7633|AAAAAAAABNNBAAAA|1997-10-27||For example used comments could conduct still. Tab|0.36|0.12|9008002|namelessmaxi #2|8|romance|9|Books|217|ationoughtable|N/A|33726472449papaya144|grey|Bundle|Unknown|5|pripricallyation| +7634|AAAAAAAACNNBAAAA|1997-10-27|2000-10-26|Young participants remove aside. Then extreme preferences may not remember never pol|4.18|2.17|3004001|edu packexporti #1|4|school-uniforms|3|Children|263|pricallyable|extra large|seashell858550835943|powder|Pallet|Unknown|5|esepricallyation| +7635|AAAAAAAACNNBAAAA|2000-10-27||Young participants remove aside. Then extreme preferences may not remember never pol|2.31|0.90|7011006|amalgnameless #6|4|accent|7|Home|263|pricallyable|N/A|seashell858550835943|gainsboro|Pound|Unknown|5|antipricallyation| +7636|AAAAAAAAENNBAAAA|1997-10-27|1999-10-27|Individuals can inspire others. Established, additional methods lend pupils. Still clear weeks could need. Ordinary conditions shall |2.48|1.26|5003001|exportischolar #1|3|pop|5|Music|155|antiantiought|N/A|54991057metallic7128|dim|Cup|Unknown|1|callypricallyation| +7637|AAAAAAAAENNBAAAA|1999-10-28|2001-10-26|Tragic miles sustain ago familiar, major substances. Different teachers expect regional, red gains. Christian eyes follow to a responses. Modern years take|3.28|1.83|5003001|scholarbrand #6|3|custom|6|Jewelry|155|antiantiought|N/A|54991057metallic7128|violet|Gram|Unknown|7|ationpricallyation| +7638|AAAAAAAAENNBAAAA|2001-10-27||Tragic miles sustain ago familiar, major substances. Different teachers expect regional, red gains. Christian eyes follow to a responses. Modern years take|4.12|2.43|5003001|edu packcorp #7|4|bracelets|6|Jewelry|639|n stprically|N/A|54991057metallic7128|salmon|Pound|Unknown|15|eingpricallyation| +7639|AAAAAAAAHNNBAAAA|1997-10-27||Centuries must envisage already things. Officials take both for a sectors. Exact tears may not restore only rich inches; difficulties could speak physical families|3.97|2.30|7015010|scholarnameless #10|15|tables|7|Home|277|ationationable|N/A|sienna29233506572388|purple|Pound|Unknown|69|n stpricallyation| +7640|AAAAAAAAINNBAAAA|1997-10-27|2000-10-26|Mad, overall patients may not keep then; pounds used to allow freshly foreign, western changes. Critical, fresh consequences should |2.83|0.90|9010009|univunivamalg #9|10|travel|9|Books|196|callyn stought|N/A|93835412indian995092|grey|Oz|Unknown|63|baresecallyation| +7641|AAAAAAAAINNBAAAA|2000-10-27||Mad, overall patients may not keep then; pounds used to allow freshly foreign, western changes. Critical, fresh consequences should |6.75|0.90|9010009|edu packamalg #2|4|swimwear|1|Women|196|callyn stought|extra large|93835412indian995092|maroon|Pallet|Unknown|5|oughtesecallyation| +7642|AAAAAAAAKNNBAAAA|1997-10-27|1999-10-27|Subsequently public types safeguard years. Wide emotions may find soon live jobs. Expected eyes can help absolutely human floors. Even short problems get with a prefer|0.57|0.21|5003001|exportischolar #1|3|pop|5|Music|60|barcally|N/A|976812wheat978012358|salmon|Pallet|Unknown|65|ableesecallyation| +7643|AAAAAAAAKNNBAAAA|1999-10-28|2001-10-26|Others could not come some|8.51|4.76|7004004|edu packbrand #4|4|curtains/drapes|7|Home|60|barcally|N/A|5turquoise0188318146|lace|Lb|Unknown|59|priesecallyation| +7644|AAAAAAAAKNNBAAAA|2001-10-27||Co|0.71|0.61|7004004|corpmaxi #1|6|parenting|9|Books|84|eseeing|N/A|38khaki4748739804612|saddle|Cup|Unknown|39|eseesecallyation| +7645|AAAAAAAANNNBAAAA|1997-10-27||Hills stimulate together heroes. Fundamental, following relations join particularly times. Political acts might notice. Concer|7.16|3.50|9013002|exportiunivamalg #2|13|self-help|9|Books|770|barationation|N/A|57854565violet532001|cornsilk|Lb|Unknown|14|antiesecallyation| +7646|AAAAAAAAONNBAAAA|1997-10-27|2000-10-26|Countries shall draw. Particular lights go similarly in brief average players; rare developments light mildly real, bizarre parents. Possible parents require seldom gates. National,|9.95|8.25|5004001|edu packscholar #1|4|classical|5|Music|166|callycallyought|N/A|041sienna13448571544|wheat|Box|Unknown|41|callyesecallyation| +7647|AAAAAAAAONNBAAAA|2000-10-27||Regularly international families spend natural, rich duties. Ever necessary sense|3.53|2.92|6015008|scholarbrand #8|15|custom|6|Jewelry|166|callycallyought|N/A|041sienna13448571544|rose|Dram|Unknown|35|ationesecallyation| +7648|AAAAAAAAAONBAAAA|1997-10-27|1999-10-27|Over identical centuries might make then native conflicts; teams co-operate reluctantly should|32.58|29.32|7011009|amalgnameless #9|11|accent|7|Home|444|eseeseese|N/A|0837684299smoke23934|seashell|Tsp|Unknown|28|eingesecallyation| +7649|AAAAAAAAAONBAAAA|1999-10-28|2001-10-26|So-called steps would not see very original ideas. Changes may suggest dramatically also expected hours. Just molecular days may not take careful|6.93|29.32|5003002|exportischolar #2|11|pop|5|Music|444|eseeseese|N/A|53859sky984811346171|sandy|Ton|Unknown|64|n stesecallyation| +7650|AAAAAAAAAONBAAAA|2001-10-27||New, old rooms can produce friends. Increased consequences will stay only groups. Major parents would not consider. Unable, new publishers might prevent there steep, labour women.|68.13|29.32|5003002|importomaxi #1|12|guns|8|Sports|444|eseeseese|N/A|honeydew077852734492|pale|Tbl|Unknown|69|baranticallyation| +7651|AAAAAAAADONBAAAA|1997-10-27||Necessary trees shall not cause parliamentary, re|0.74|0.48|7012002|importonameless #2|12|paint|7|Home|121|oughtableought|N/A|14010458hot359037097|violet|Dozen|Unknown|35|oughtanticallyation| +7652|AAAAAAAAEONBAAAA|1997-10-27|2000-10-26|Either sudden centuries will not grant even historica|4.55|2.77|9013009|exportiunivamalg #9|13|self-help|9|Books|581|oughteinganti|N/A|41747607567668smoke1|puff|Case|Unknown|27|ableanticallyation| +7653|AAAAAAAAEONBAAAA|2000-10-27||Either sudden centuries will not grant even historica|0.15|0.08|6016006|corpbrand #6|13|consignment|6|Jewelry|581|oughteinganti|N/A|6376020pink786012278|pale|Tsp|Unknown|57|prianticallyation| +7654|AAAAAAAAGONBAAAA|1997-10-27|1999-10-27|Ever top offers might struggle far, automatic men. Long-term, long goods dare however; new, other gr|2.30|1.97|9002011|importomaxi #11|2|business|9|Books|776|callyationation|N/A|4502409grey499188488|blue|Box|Unknown|52|eseanticallyation| +7655|AAAAAAAAGONBAAAA|1999-10-28|2001-10-26|Ever top offers might struggle far, automatic men. Long-term, long goods dare however; new, other gr|1.12|0.85|9002011|exportiamalg #2|2|maternity|1|Women|274|callyationation|petite|4502409grey499188488|saddle|Ounce|Unknown|33|antianticallyation| +7656|AAAAAAAAGONBAAAA|2001-10-27||Ever top offers might struggle far, automatic men. Long-term, long goods dare however; new, other gr|5.83|0.85|9002011|importomaxi #1|2|business|9|Books|274|eseationable|N/A|61778171275340royal7|spring|Carton|Unknown|24|callyanticallyation| +7657|AAAAAAAAJONBAAAA|1997-10-27||Just quick heroes put; i|3.21|1.66|2002002|importoimporto #2|2|shirts|2|Men|602|ablebarcally|small|98pale99608521308036|lavender|Dozen|Unknown|63|ationanticallyation| +7658|AAAAAAAAKONBAAAA|1997-10-27|2000-10-26|Splendid off|1.82|0.70|7015009|scholarnameless #9|15|tables|7|Home|610|baroughtcally|N/A|98papaya978473007854|yellow|Each|Unknown|30|einganticallyation| +7659|AAAAAAAAKONBAAAA|2000-10-27||Splendid off|0.42|0.31|5002002|importoscholar #2|15|country|5|Music|191|oughtn stought|N/A|98papaya978473007854|thistle|Each|Unknown|21|n stanticallyation| +7660|AAAAAAAAMONBAAAA|1997-10-27|1999-10-27|Voices shall remain appropriate details. Temporary|1.39|0.90|10002011|importounivamalg #11|2|camcorders|10|Electronics|416|callyoughtese|N/A|044224khaki102089364|rosy|Gross|Unknown|29|barcallycallyation| +7661|AAAAAAAAMONBAAAA|1999-10-28|2001-10-26|Helpful matches should say better. Centres retire about. Civil, interested d|1.49|0.90|10002011|edu packimporto #2|2|sports-apparel|2|Men|395|callyoughtese|small|8turquoise8182917981|cyan|N/A|Unknown|22|oughtcallycallyation| +7662|AAAAAAAAMONBAAAA|2001-10-27||Helpful matches should say better. Centres retire about. Civil, interested d|7.80|0.90|6016007|corpbrand #7|2|consignment|6|Jewelry|69|callyoughtese|N/A|3578895sky0419787146|tomato|Tbl|Unknown|56|ablecallycallyation| +7663|AAAAAAAAPONBAAAA|1997-10-27||Difficult, necessary teams will not avoid apart. Allies may lo|3.20|1.34|4004002|edu packedu pack #2|4|athletic|4|Shoes|246|callyeseable|large|9843turquoise6041472|royal|Case|Unknown|28|pricallycallyation| +7664|AAAAAAAAAPNBAAAA|1997-10-27|2000-10-26|Islamic others should not avoid for the most part wild, go|9.30|6.13|2004001|edu packimporto #1|4|sports-apparel|2|Men|314|eseoughtpri|large|678323608277304puff3|blanched|Bunch|Unknown|86|esecallycallyation| +7665|AAAAAAAAAPNBAAAA|2000-10-27||Figures steal nuclear, able detectives. Primarily french days could prescribe long. Mod|1.81|6.13|2004001|univamalgamalg #16|10|memory|10|Electronics|314|eseoughtpri|N/A|91991598yellow581445|steel|Each|Unknown|45|anticallycallyation| +7666|AAAAAAAACPNBAAAA|1997-10-27|1999-10-27|Able results eat as terms. Explanations shall not name. Educational fields postpone proudly less fresh results. Strange sales judge increases. Only terrible persons will produce|6.01|1.86|1002001|importoamalg #1|2|fragrances|1|Women|273|priationable|medium|0993red2285180687984|slate|Gross|Unknown|71|callycallycallyation| +7667|AAAAAAAACPNBAAAA|1999-10-28|2001-10-26|Massive police destroy probably form|1.74|0.67|1002001|edu packexporti #2|4|school-uniforms|3|Children|273|priationable|large|25752021381saddle910|seashell|Lb|Unknown|20|ationcallycallyation| +7668|AAAAAAAACPNBAAAA|2001-10-27||Massive police destroy probably form|3.76|1.12|8012009|importomaxi #9|12|guns|8|Sports|273|priationable|N/A|778073gainsboro14167|red|Dozen|Unknown|30|eingcallycallyation| +7669|AAAAAAAAFPNBAAAA|1997-10-27||Fortunes continue certainly obvious, new feet; dangerous candidates shall reveal others; temporary tensions own like a sports. More tall rights should follow. Items l|4.04|1.77|6014006|edu packbrand #6|14|estate|6|Jewelry|348|eingesepri|N/A|7937771orchid5436630|puff|Cup|Unknown|16|n stcallycallyation| +7670|AAAAAAAAGPNBAAAA|1997-10-27|2000-10-26|Months make stands. Still |35.26|23.62|10002004|importounivamalg #4|2|camcorders|10|Electronics|15|antiought|N/A|0297019149powder4601|indian|Lb|Unknown|33|barationcallyation| +7671|AAAAAAAAGPNBAAAA|2000-10-27||Secondary requests will go yet good characters. Early applications prefer large experiments. Popular |44.61|25.87|10002004|exportiimporto #2|3|pants|2|Men|15|antiought|small|9871322ivory47154515|red|Gram|Unknown|33|oughtationcallyation| +7672|AAAAAAAAIPNBAAAA|1997-10-27|1999-10-27|Different shares shall last even words. Contracts make on a others. Far from awful colleagues know right years. Names know in a letters. High varieties ought to undergo successful, immed|8.97|6.54|8007003|brandnameless #3|7|hockey|8|Sports|309|n stbarpri|N/A|78583steel8900822115|purple|Pound|Unknown|7|ableationcallyation| +7673|AAAAAAAAIPNBAAAA|1999-10-28|2001-10-26|Different shares shall last even words. Contracts make on a others. Far from awful colleagues know right years. Names know in a letters. High varieties ought to undergo successful, immed|5.94|2.25|10004007|edu packunivamalg #7|4|audio|10|Electronics|65|n stbarpri|N/A|64882919625rose38301|pale|Gram|Unknown|6|priationcallyation| +7674|AAAAAAAAIPNBAAAA|2001-10-27||Different shares shall last even words. Contracts make on a others. Far from awful colleagues know right years. Names know in a letters. High varieties ought to undergo successful, immed|2.19|2.25|10004007|scholarunivamalg #8|5|karoke|10|Electronics|87|n stbarpri|N/A|64882919625rose38301|linen|Dram|Unknown|17|eseationcallyation| +7675|AAAAAAAALPNBAAAA|1997-10-27||All bad figures get brown games. Circumstances cannot understand just. Bio|1.30|0.83|2003002|exportiimporto #2|3|pants|2|Men|123|priableought|large|06828599638salmon565|purple|Lb|Unknown|16|antiationcallyation| +7676|AAAAAAAAMPNBAAAA|1997-10-27|2000-10-26|Democrats will make more. Aware, other |1.98|0.63|3003001|exportiexporti #1|3|toddlers|3|Children|252|ableantiable|medium|8097808209387saddle7|peach|Tbl|Unknown|15|callyationcallyation| +7677|AAAAAAAAMPNBAAAA|2000-10-27||American, early markets shake; labour intentions shall propose federal, main beans. Un|3.49|1.32|3003001|exportiamalg #2|3|maternity|1|Women|252|ableantiable|medium|8097808209387saddle7|deep|Tbl|Unknown|7|ationationcallyation| +7678|AAAAAAAAOPNBAAAA|1997-10-27|1999-10-27|Special, eligible c|2.03|1.40|7004009|edu packbrand #9|4|curtains/drapes|7|Home|310|baroughtpri|N/A|37570836690002red462|lace|Tsp|Unknown|44|eingationcallyation| +7679|AAAAAAAAOPNBAAAA|1999-10-28|2001-10-26|Pensions may not shift still out of a companies. Today potential implications ought to overcome both on a discussions. Lucky, new occasions |0.65|1.40|2002002|importoimporto #2|2|shirts|2|Men|310|baroughtpri|N/A|50powder853733098603|mint|Unknown|Unknown|48|n stationcallyation| +7680|AAAAAAAAOPNBAAAA|2001-10-27||Pensions may not shift still out of a companies. Today potential implications ought to overcome both on a discussions. Lucky, new occasions |8.71|6.53|2004001|edu packimporto #1|2|sports-apparel|2|Men|310|baroughtpri|small|0097705103turquoise2|violet|Dozen|Unknown|33|bareingcallyation| +7681|AAAAAAAABAOBAAAA|1997-10-27||New, red occasions make here from |2.15|0.94|10001015|amalgunivamalg #15|1|cameras|10|Electronics|501|oughtbaranti|N/A|38888067253135snow62|black|Tbl|Unknown|7|oughteingcallyation| +7682|AAAAAAAACAOBAAAA|1997-10-27|2000-10-26|Political policemen will|2.60|1.63|1001001|amalgamalg #1|1|dresses|1|Women|172|ableationought|medium|3wheat42400199680024|lace|Unknown|Unknown|25|ableeingcallyation| +7683|AAAAAAAACAOBAAAA|2000-10-27||Sick, massive streets ought to acknow|4.72|1.63|1001001|scholarmaxi #4|5|history|9|Books|172|ableationought|N/A|3wheat42400199680024|orange|N/A|Unknown|7|prieingcallyation| +7684|AAAAAAAAEAOBAAAA|1997-10-27|1999-10-27|Likely terms shall not act ben|1.56|1.21|6006005|corpcorp #5|6|rings|6|Jewelry|540|bareseanti|N/A|108647633salmon73960|misty|Box|Unknown|17|eseeingcallyation| +7685|AAAAAAAAEAOBAAAA|1999-10-28|2001-10-26|Likely terms shall not act ben|2.97|1.93|9009006|maximaxi #6|6|science|9|Books|540|bareseanti|N/A|108647633salmon73960|red|Pallet|Unknown|31|antieingcallyation| +7686|AAAAAAAAEAOBAAAA|2001-10-27||Likely terms shall not act ben|4.35|1.93|5001001|amalgscholar #1|6|rock|5|Music|540|bareseanti|N/A|108647633salmon73960|green|Gross|Unknown|31|callyeingcallyation| +7687|AAAAAAAAHAOBAAAA|1997-10-27||Associations may not soothe less increasing costs. Foreign years should find sure parties. Always new minutes cease children. Certainly ma|9.20|4.32|6009008|maxicorp #8|9|womens watch|6|Jewelry|254|eseantiable|N/A|4violet5868722689142|thistle|Carton|Unknown|28|ationeingcallyation| +7688|AAAAAAAAIAOBAAAA|1997-10-27|2000-10-26|Cheap women like there small deputies. International heads form. Wives stop absolute, overall efforts. Ma|0.29|0.24|5004001|edu packscholar #1|4|classical|5|Music|284|eseeingable|N/A|522450562193wheat845|thistle|Case|Unknown|33|eingeingcallyation| +7689|AAAAAAAAIAOBAAAA|2000-10-27||Cheap women like there small deputies. International heads form. Wives stop absolute, overall efforts. Ma|2.00|1.66|5004001|exportiexporti #2|3|toddlers|3|Children|110|eseeingable|medium|80277461144steel2255|cornsilk|Tsp|Unknown|12|n steingcallyation| +7690|AAAAAAAAKAOBAAAA|1997-10-27|1999-10-27|Ther|7.06|4.16|6013003|exportibrand #3|13|loose stones|6|Jewelry|324|eseablepri|N/A|seashell260320616258|papaya|Bundle|Unknown|12|barn stcallyation| +7691|AAAAAAAAKAOBAAAA|1999-10-28|2001-10-26|Probably general concentrations cannot reach in the institutions. Old centres may ask yesterday average elections. Foreigners look over royal plants. Rich phenomena shall|7.47|2.91|6013003|importoedu pack #2|13|mens|4|Shoes|324|eseablepri|small|6navajo6407457963592|peru|Bunch|Unknown|25|oughtn stcallyation| +7692|AAAAAAAAKAOBAAAA|2001-10-27||Probably general concentrations cannot reach in the institutions. Old centres may ask yesterday average elections. Foreigners look over royal plants. Rich phenomena shall|3.97|1.54|6013003|amalgbrand #3|1|bathroom|7|Home|91|oughtn st|N/A|6navajo6407457963592|pale|Box|Unknown|34|ablen stcallyation| +7693|AAAAAAAANAOBAAAA|1997-10-27||In general high russians sound easily police. Organisers can produce just off|35.14|24.24|9012002|importounivamalg #2|12|home repair|9|Books|181|oughteingought|N/A|275367746smoke905493|red|N/A|Unknown|32|prin stcallyation| +7694|AAAAAAAAOAOBAAAA|1997-10-27|2000-10-26|From time to time historical services attract minerals. Buildings create so|4.04|1.85|2001001|amalgimporto #1|1|accessories|2|Men|515|antioughtanti|petite|463misty142368899292|turquoise|Carton|Unknown|28|esen stcallyation| +7695|AAAAAAAAOAOBAAAA|2000-10-27||From time to time historical services attract minerals. Buildings create so|52.62|39.99|5004002|edu packscholar #2|1|classical|5|Music|515|antioughtanti|N/A|463misty142368899292|tomato|N/A|Unknown|7|antin stcallyation| +7696|AAAAAAAAABOBAAAA|1997-10-27|1999-10-27|Dangerous, low |4.26|3.70|6009001|maxicorp #1|9|womens watch|6|Jewelry|166|callycallyought|N/A|sky27728582703462992|aquamarine|Gram|Unknown|43|callyn stcallyation| +7697|AAAAAAAAABOBAAAA|1999-10-28|2001-10-26|Here united prospects stop carefully. Only english participants benefit new possibilities. |0.78|0.44|9010006|univunivamalg #6|10|travel|9|Books|227|ationableable|N/A|370521floral38206497|royal|Box|Unknown|27|ationn stcallyation| +7698|AAAAAAAAABOBAAAA|2001-10-27||Shoulders provide. New, hard pupils will not describe regardless white months. Medical files appeal just with a hours. Old line|7.58|0.44|9010006|amalgexporti #1|1|newborn|3|Children|731|oughtpriation|medium|370521floral38206497|papaya|Gram|Unknown|57|eingn stcallyation| +7699|AAAAAAAADBOBAAAA|1997-10-27||Notions shall say major journals; economic standards make at once old requirements. So corporate numbers ask now in a images; surely closed feelings m|1.80|1.36|9007008|brandmaxi #8|7|reference|9|Books|492|ablen stese|N/A|salmon64376820420869|plum|Tsp|Unknown|17|n stn stcallyation| +7700|AAAAAAAAEBOBAAAA|1997-10-27|2000-10-26|Overseas, permanent clothes should treat just left, darling liabilities. However main others let later colours. Model ministers slip brave, official gl|18.48|16.63|2003001|exportiimporto #1|3|pants|2|Men|591|oughtn stanti|extra large|13slate4554079962098|violet|Box|Unknown|16|barbarationation| +7701|AAAAAAAAEBOBAAAA|2000-10-27||Overseas, permanent clothes should treat just left, darling liabilities. However main others let later colours. Model ministers slip brave, official gl|7.81|5.31|3004002|edu packexporti #2|4|school-uniforms|3|Children|192|ablen stought|medium|13slate4554079962098|sienna|Cup|Unknown|12|oughtbarationation| +7702|AAAAAAAAGBOBAAAA|1997-10-27|1999-10-27|Religious, industrial rules will become still solely major |4.01|2.80|8001003|amalgnameless #3|1|athletic shoes|8|Sports|288|eingeingable|N/A|419206ivory530939314|spring|Each|Unknown|42|ablebarationation| +7703|AAAAAAAAGBOBAAAA|1999-10-28|2001-10-26|Thanks may know outdoor, special eyes. Cultural, y|20.34|8.94|4003002|exportiedu pack #2|3|kids|4|Shoes|288|eingeingable|large|2210873437806red5323|papaya|Bunch|Unknown|26|pribarationation| +7704|AAAAAAAAGBOBAAAA|2001-10-27||Then dominant things should complete smoothly into a developments. Generally straight sentences might move in short videos. Corporate, extensive in|2.37|1.37|4003002|exportiamalg #1|3|maternity|1|Women|288|eingeingable|large|7260976273334navy295|medium|Lb|Unknown|26|esebarationation| +7705|AAAAAAAAJBOBAAAA|1997-10-27||Prime, light |1.31|1.07|6010002|univbrand #2|10|jewelry boxes|6|Jewelry|619|n stoughtcally|N/A|74879095116seashell3|red|Ounce|Unknown|12|antibarationation| +7706|AAAAAAAAKBOBAAAA|1997-10-27|2000-10-26|Fond heads assign of course defences. Local, simple products should oppose sure old levels; male, accessible ships make just indian roads. Models will compose anyway natural steps. Yards see |9.63|5.68|3003001|exportiexporti #1|3|toddlers|3|Children|253|priantiable|petite|8coral61995310400584|medium|Bunch|Unknown|24|callybarationation| +7707|AAAAAAAAKBOBAAAA|2000-10-27||Further rich interests may deal fo|2.57|0.79|3003001|exportiexporti #2|3|toddlers|3|Children|214|priantiable|large|8coral61995310400584|tan|Dram|Unknown|27|ationbarationation| +7708|AAAAAAAAMBOBAAAA|1997-10-27|1999-10-27|Details could argue; high sales should not |3.55|2.48|9013005|exportiunivamalg #5|13|self-help|9|Books|55|antianti|N/A|9897640chartreuse997|seashell|Case|Unknown|45|eingbarationation| +7709|AAAAAAAAMBOBAAAA|1999-10-28|2001-10-26|Extremely huge friends must not understand more into a questions. Businesses would lay more. Deliberately psychiatric fragments reinforce to a consumers; different, general studie|0.46|0.27|7003010|exportibrand #10|13|kids|7|Home|55|antianti|N/A|94tomato309921909059|goldenrod|Gram|Unknown|26|n stbarationation| +7710|AAAAAAAAMBOBAAAA|2001-10-27||Free trees shall not see scarcely free positions. Homes should undermine assumptions; powerful students could make. Days must not speculate complete companies. Confidential police join red patient|4.16|0.27|7003010|edu packexporti #1|13|school-uniforms|3|Children|55|antianti|medium|013318094096purple89|rosy|Pallet|Unknown|17|baroughtationation| +7711|AAAAAAAAPBOBAAAA|1997-10-27||Personal, controversial features fight goals. Better other levels keep along a institutions. Internal, conscious conventions might expect short years. Small, |6.86|3.22|10012011|importoamalgamalg #11|12|monitors|10|Electronics|169|n stcallyought|N/A|405433220honeydew706|brown|Pallet|Unknown|16|oughtoughtationation| +7712|AAAAAAAAACOBAAAA|1997-10-27|2000-10-26|Problems could make seconds; humans guess ever there top pa|1.47|1.26|6007001|brandcorp #1|7|pendants|6|Jewelry|890|barn steing|N/A|48946misty4875100490|lime|Dram|Unknown|33|ableoughtationation| +7713|AAAAAAAAACOBAAAA|2000-10-27||Problems could make seconds; humans guess ever there top pa|9.46|1.26|8013004|exportimaxi #4|7|sailing|8|Sports|148|barn steing|N/A|plum0311028496949252|orange|Unknown|Unknown|10|prioughtationation| +7714|AAAAAAAACCOBAAAA|1997-10-27|1999-10-27|Railways use british, dark limits. Always overseas eyes know now disastrous, individual meetings; local values would persuade e|2.15|1.93|4004001|edu packedu pack #1|4|athletic|4|Shoes|492|ablen stese|small|4739red2795940842088|linen|Dram|Unknown|31|eseoughtationation| +7715|AAAAAAAACCOBAAAA|1999-10-28|2001-10-26|Railways use british, dark limits. Always overseas eyes know now disastrous, individual meetings; local values would persuade e|2.64|1.93|3003002|exportiexporti #2|4|toddlers|3|Children|401|oughtbarese|N/A|4739red2795940842088|burnished|Unknown|Unknown|6|antioughtationation| +7716|AAAAAAAACCOBAAAA|2001-10-27||Years encourage. There low plans may come there surpr|5.45|1.74|9011001|amalgunivamalg #1|11|cooking|9|Books|240|bareseable|N/A|4739red2795940842088|burnished|Each|Unknown|9|callyoughtationation| +7717|AAAAAAAAFCOBAAAA|1997-10-27||Hours should join far. Members used to set already aw|9.32|6.52|7015008|scholarnameless #8|15|tables|7|Home|286|callyeingable|N/A|3691933807ghost15864|medium|Unknown|Unknown|33|ationoughtationation| +7718|AAAAAAAAGCOBAAAA|1997-10-27|2000-10-26|Vital colleagues allow very letters; recent, dramatic resources tell very thousands. Royal, sexual aspirations will earn almost on a legs. We|4.05|1.98|9012003|importounivamalg #3|12|home repair|9|Books|136|callypriought|N/A|1420powder1976791129|sky|Tbl|Unknown|14|eingoughtationation| +7719|AAAAAAAAGCOBAAAA|2000-10-27||Vital colleagues allow very letters; recent, dramatic resources tell very thousands. Royal, sexual aspirations will earn almost on a legs. We|2.92|1.57|9014004|edu packunivamalg #4|12|sports|9|Books|423|priableese|N/A|4tomato9358015648698|plum|Gram|Unknown|82|n stoughtationation| +7720|AAAAAAAAICOBAAAA|1997-10-27|1999-10-27|Misleading mech|0.47|0.28|10006003|corpunivamalg #3|6|musical|10|Electronics|145|antieseought|N/A|89609967thistle24689|peru|Unknown|Unknown|12|barableationation| +7721|AAAAAAAAICOBAAAA|1999-10-28|2001-10-26|Misleading mech|4.07|1.79|5004002|edu packscholar #2|6|classical|5|Music|145|antieseought|N/A|89609967thistle24689|magenta|Dram|Unknown|95|oughtableationation| +7722|AAAAAAAAICOBAAAA|2001-10-27||Misleading mech|8.27|4.71|6002003|importocorp #3|6|diamonds|6|Jewelry|145|antieseought|N/A|89609967thistle24689|violet|Pound|Unknown|33|ableableationation| +7723|AAAAAAAALCOBAAAA|1997-10-27||Cases cannot resign indeed. New types used to prejudice often industrial votes. Honest|9.69|7.26|9008008|namelessmaxi #8|8|romance|9|Books|413|prioughtese|N/A|666594thistle6443678|rosy|Oz|Unknown|94|priableationation| +7724|AAAAAAAAMCOBAAAA|1997-10-27|2000-10-26|Emotional, useful policies come on a pairs. Exactly educational occasions may pay gently much basic services. Fixed, other proteins test ago major officials. Much economic years could not go mil|2.47|1.72|1004001|edu packamalg #1|4|swimwear|1|Women|553|priantianti|medium|357saddle67514082397|slate|Dram|Unknown|1|eseableationation| +7725|AAAAAAAAMCOBAAAA|2000-10-27||Full feet will not ensure birds. Voices know especially. Men can go features. Rural ways might not allow possible persons. Necessarily eastern incentives used to release open. Proper relationship|97.25|1.72|1004001|amalgscholar #2|1|rock|5|Music|911|oughtoughtn st|N/A|357saddle67514082397|yellow|Oz|Unknown|14|antiableationation| +7726|AAAAAAAAOCOBAAAA|1997-10-27|1999-10-27|Cars may supply quietly head diff|26.20|17.81|1001001|amalgamalg #1|1|dresses|1|Women|189|n steingought|extra large|830910blush265410477|yellow|Dram|Unknown|76|callyableationation| +7727|AAAAAAAAOCOBAAAA|1999-10-28|2001-10-26|Birds could help too to a thanks. Costs build hungry, main details. Public times may know more tha|9.46|17.81|1001001|scholarunivamalg #12|15|fiction|9|Books|189|n steingought|N/A|830910blush265410477|spring|Ounce|Unknown|49|ationableationation| +7728|AAAAAAAAOCOBAAAA|2001-10-27||Maybe ex|58.04|52.23|1002001|importoamalg #1|15|fragrances|1|Women|332|ablepripri|petite|830910blush265410477|sky|Dram|Unknown|98|eingableationation| +7729|AAAAAAAABDOBAAAA|1997-10-27||Times fall buildings. Causal yards will not survive over at the|11.60|8.93|7010010|univnameless #10|10|flatware|7|Home|306|callybarpri|N/A|3343156606slate08558|turquoise|Dram|Unknown|32|n stableationation| +7730|AAAAAAAACDOBAAAA|1997-10-27|2000-10-26|British, possible solicitors fall still in a indians. Perfect names should not cost still. Redundant, mild opponents give just military specialists. Here great |0.10|0.04|7001001|amalgbrand #1|1|bathroom|7|Home|174|eseationought|N/A|427889270549puff9728|hot|Carton|Unknown|35|barpriationation| +7731|AAAAAAAACDOBAAAA|2000-10-27||Also top members call perfectly. Individual, high materials get also most excessive years. Friends spend soon equivalent clubs. Grey arms will not take understandably senior cases. |3.44|0.04|8010002|univmaxi #2|1|pools|8|Sports|570|eseationought|N/A|780605861560782puff0|salmon|Oz|Unknown|5|oughtpriationation| +7732|AAAAAAAAEDOBAAAA|1997-10-27|1999-10-27||7.77||||2|country|5|Music|281||||lavender|Ounce|||| +7733|AAAAAAAAEDOBAAAA|1999-10-28|2001-10-26|Principally main prayers drive quickly. Various, added prisoners will not see open cal|12.02|5.20|4003002|exportiedu pack #2|2|kids|4|Shoes|279|n stationable|medium|69400purple751427268|royal|Unknown|Unknown|5|pripriationation| +7734|AAAAAAAAEDOBAAAA|2001-10-27||Alarmingly local measures ought to go oddly professional convenient liabilities. At least previous women use as foreign children; also re|1.70|0.86|4003002|exportibrand #5|3|kids|7|Home|498|eingn stese|N/A|816gainsboro93741026|plum|Lb|Unknown|8|esepriationation| +7735|AAAAAAAAHDOBAAAA|1997-10-27||Rights will try useful, intermediate thousands. Main aspirations keep there bright, possible lives. Problems render however significant, strange func|5.08|1.82|8007006|brandnameless #6|7|hockey|8|Sports|21|oughtable|N/A|665019629spring36330|medium|Oz|Unknown|78|antipriationation| +7736|AAAAAAAAIDOBAAAA|1997-10-27|2000-10-26|Gifts shall perform examinations. Mainly female factors could define architects. Efficient functions invite vast, important friends. Local, various groups must think then. |0.61|0.18|9012009|importounivamalg #9|12|home repair|9|Books|178|eingationought|N/A|80navy58753809730473|dim|Ton|Unknown|13|callypriationation| +7737|AAAAAAAAIDOBAAAA|2000-10-27||Gifts shall perform examinations. Mainly female factors could define architects. Efficient functions invite vast, important friends. Local, various groups must think then. |0.42|0.26|9012009|importounivamalg #10|12|home repair|9|Books|178|eingationought|N/A|80navy58753809730473|rosy|Tsp|Unknown|15|ationpriationation| +7738|AAAAAAAAKDOBAAAA|1997-10-27|1999-10-27|Just dead blocks cou|1.67|1.43|9015011|scholarunivamalg #11|15|fiction|9|Books|443|prieseese|N/A|358384peru6842985262|rosy|Ton|Unknown|34|eingpriationation| +7739|AAAAAAAAKDOBAAAA|1999-10-28|2001-10-26|Just dead blocks cou|4.37|1.43|9015011|corpcorp #4|6|rings|6|Jewelry|443|prieseese|N/A|1136691129452sandy03|pale|Carton|Unknown|23|n stpriationation| +7740|AAAAAAAAKDOBAAAA|2001-10-27||British, present reasons ought to operate sales; political, real scores will not need hardly years. Good, rich imports top elsewhere singl|1.43|0.64|2003001|exportiimporto #1|6|pants|2|Men|443|prieseese|N/A|1136691129452sandy03|lawn|Each|Unknown|32|bareseationation| +7741|AAAAAAAANDOBAAAA|1997-10-27||Limited, sharp hours look available proportions. Especially public ties object basic reductions; institutional sales apply; preferably territorial pp. used to pr|9.88|6.02|9008002|namelessmaxi #2|8|romance|9|Books|441|oughteseese|N/A|69649265528steel1416|misty|Ounce|Unknown|12|oughteseationation| +7742|AAAAAAAAODOBAAAA|1997-10-27|2000-10-26|Services shall make just never average rights; actual, high walls manufacture. Human, italian wars obtain then l|9.76|6.83|9009003|maximaxi #3|9|science|9|Books|129|n stableought|N/A|7127681469691purple8|hot|Unknown|Unknown|59|ableeseationation| +7743|AAAAAAAAODOBAAAA|2000-10-27||Forward aware services shall buy at a properties. Newspapers might drain free, shy penalties. More public parties should not decide at last. Results can get in a circumstances. Jobs wil|8.62|6.29|9009003|exportischolar #2|9|pop|5|Music|129|n stableought|N/A|7127681469691purple8|pale|Pallet|Unknown|14|prieseationation| +7744|AAAAAAAAAEOBAAAA|1997-10-27|1999-10-27|Open eyes could lower tomorrow specialist groups. Other depar|3.35|1.87|6002001|importocorp #1|2|diamonds|6|Jewelry|254|eseantiable|N/A|puff3559288653018642|indian|Dozen|Unknown|17|eseeseationation| +7745|AAAAAAAAAEOBAAAA|1999-10-28|2001-10-26|Best simple doctors may look on a individuals. Areas play policies. Officials make too. |7.94|1.87|6002001|exportiexporti #2|2|toddlers|3|Children|198|eingn stought|medium|621882180236purple41|puff|Oz|Unknown|45|antieseationation| +7746|AAAAAAAAAEOBAAAA|2001-10-27||Best simple doctors may look on a individuals. Areas play policies. Officials make too. |0.92|1.87|4002001|importoedu pack #1|2|mens|4|Shoes|198|eingn stought|medium|621882180236purple41|turquoise|Gram|Unknown|58|callyeseationation| +7747|AAAAAAAADEOBAAAA|1997-10-27||More high sciences contract still weak questions; years must think please so small numbers. Officers shall keep difficult, appropriate officials; major resources |5.32|4.04|6003004|exporticorp #4|3|gold|6|Jewelry|224|eseableable|N/A|34526saddle446167548|yellow|Dozen|Unknown|43|ationeseationation| +7748|AAAAAAAAEEOBAAAA|1997-10-27|2000-10-26|Years must find ultimately down an parents. National, literary|2.95|1.71|4002001|importoedu pack #1|2|mens|4|Shoes|658|eingantically|medium|49295211red785725652|wheat|Box|Unknown|40|eingeseationation| +7749|AAAAAAAAEEOBAAAA|2000-10-27||Intentions might not use pict|0.18|1.71|8007010|brandnameless #10|7|hockey|8|Sports|658|eingantically|N/A|49295211red785725652|slate|Ounce|Unknown|7|n steseationation| +7750|AAAAAAAAGEOBAAAA|1997-10-27|1999-10-27|Too usual techniques would not know so relevant techniques. However other sons get more corporate examples. Always large tanks lay for example. Still short processes sho|0.82|0.73|7011001|amalgnameless #1|11|accent|7|Home|199|n stn stought|N/A|73478740058seashell8|snow|Gross|Unknown|29|barantiationation| +7751|AAAAAAAAGEOBAAAA|1999-10-28|2001-10-26|Police would not end ea|2.55|1.40|7011001|importoscholar #2|2|country|5|Music|199|n stn stought|N/A|1551781369428388red8|violet|Pallet|Unknown|48|oughtantiationation| +7752|AAAAAAAAGEOBAAAA|2001-10-27||Police would not end ea|7.33|1.40|7011001|importoscholar #1|2|country|5|Music|199|n stn stought|N/A|1551781369428388red8|saddle|Box|Unknown|6|ableantiationation| +7753|AAAAAAAAJEOBAAAA|1997-10-27||Together cold languages use|2.51|2.08|2002002|importoimporto #2|2|shirts|2|Men|319|n stoughtpri|small|791576honeydew840580|linen|Lb|Unknown|74|priantiationation| +7754|AAAAAAAAKEOBAAAA|1997-10-27|2000-10-26|British, creative beliefs go potential, main patients. Scientists forget so female, honest streets; mode|0.09|0.03|5002001|importoscholar #1|2|country|5|Music|19|n stought|N/A|110144530674wheat577|rosy|Dozen|Unknown|28|eseantiationation| +7755|AAAAAAAAKEOBAAAA|2000-10-27||Members worry as in a cards; really full parents speak. Then teenage cattle own altogether minor |0.72|0.03|10010004|univamalgamalg #4|2|memory|10|Electronics|132|n stought|N/A|8purple9457714732340|thistle|Carton|Unknown|56|antiantiationation| +7756|AAAAAAAAMEOBAAAA|1997-10-27|1999-10-27|Powerful stan|1.35|0.68|3001001|amalgexporti #1|1|newborn|3|Children|570|barationanti|petite|377693774446791rosy5|forest|Tbl|Unknown|44|callyantiationation| +7757|AAAAAAAAMEOBAAAA|1999-10-28|2001-10-26|Social, simple g|7.53|3.31|3001001|importoscholar #2|1|country|5|Music|570|barationanti|N/A|goldenrod42347099828|grey|Bundle|Unknown|11|ationantiationation| +7758|AAAAAAAAMEOBAAAA|2001-10-27||Only back causes can watch even profitable services. Slowly secure troops achieve too plain weeks. Swiftly present years die into a glasses. Now common rig|8.91|3.31|3001001|namelessnameless #9|8|outdoor|8|Sports|173|priationought|N/A|goldenrod42347099828|sky|Bunch|Unknown|33|eingantiationation| +7759|AAAAAAAAPEOBAAAA|1997-10-27||Best american railways throw in order video-taped, large firms. Now large courses should take enou|4.53|1.99|3002002|importoexporti #2|2|infants|3|Children|581|oughteinganti|small|9883455028920sienna0|ivory|Ounce|Unknown|8|n stantiationation| +7760|AAAAAAAAAFOBAAAA|1997-10-27|2000-10-26|Lips can allow to a costs; finally existing characters ought to share then once new members. Yards see available, adult languages. More high plants c|8.64|2.93|6015001|scholarbrand #1|15|custom|6|Jewelry|107|ationbarought|N/A|55green6941910307082|chocolate|Each|Unknown|17|barcallyationation| +7761|AAAAAAAAAFOBAAAA|2000-10-27||Perhaps young trends should provide shortly to a houses; particular, pure events reduce. Crucial securities should force only also intense criteria. Needs would r|1.58|2.93|6015001|importounivamalg #10|15|camcorders|10|Electronics|899|n stn steing|N/A|55green6941910307082|rose|Cup|Unknown|38|oughtcallyationation| +7762|AAAAAAAACFOBAAAA|1997-10-27|1999-10-27|Specimens enjoy exactly other areas. Names mean just in a operati|63.63|25.45|7016003|corpnameless #3|16|furniture|7|Home|94|esen st|N/A|9696975823peach48993|steel|Dozen|Unknown|12|ablecallyationation| +7763|AAAAAAAACFOBAAAA|1999-10-28|2001-10-26|Clear, popular|80.98|64.78|7016003|importoexporti #2|16|infants|3|Children|90|esen st|small|229233sandy454627932|almond|Ounce|Unknown|70|pricallyationation| +7764|AAAAAAAACFOBAAAA|2001-10-27||Clear, popular|4.06|64.78|10009013|maxiunivamalg #13|9|televisions|10|Electronics|225|esen st|N/A|59585white7309728079|pale|Gram|Unknown|31|esecallyationation| +7765|AAAAAAAAFFOBAAAA|1997-10-27||Serious, political elements shall not ensure all but separate newspapers; implicitly different|8.12|3.49|1003002|exportiamalg #2|3|maternity|1|Women|21|oughtable|small|53powder840633643355|rose|N/A|Unknown|2|anticallyationation| +7766|AAAAAAAAGFOBAAAA|1997-10-27|2000-10-26|Really long walls should try only easy old holes. Young, accessible bars would see straight other candidat|1.96|0.96|1002001|importoamalg #1|2|fragrances|1|Women|345|antiesepri|small|818530056392yellow63|salmon|Ounce|Unknown|16|callycallyationation| +7767|AAAAAAAAGFOBAAAA|2000-10-27||Really long walls should try only easy old holes. Young, accessible bars would see straight other candidat|4.81|1.82|4001002|amalgedu pack #2|1|womens|4|Shoes|327|ationablepri|medium|818530056392yellow63|steel|Oz|Unknown|29|ationcallyationation| +7768|AAAAAAAAIFOBAAAA|1997-10-27|1999-10-27|Passive, heavy lawyers would confront yet. Popular, extended figures should say moving decisions. Final, australian difficulties co|8.97|4.03|2003001|exportiimporto #1|3|pants|2|Men|329|n stablepri|extra large|611851pink1613308158|tan|Cup|Unknown|79|eingcallyationation| +7769|AAAAAAAAIFOBAAAA|1999-10-28|2001-10-26|Passive, heavy lawyers would confront yet. Popular, extended figures should say moving decisions. Final, australian difficulties co|5.68|4.03|2002002|importoimporto #2|2|shirts|2|Men|187|ationeingought|small|8powder0613135204492|papaya|Gram|Unknown|21|n stcallyationation| +7770|AAAAAAAAIFOBAAAA|2001-10-27||However economic tanks contradict; ti|9.71|5.24|2002002|amalgedu pack #1|2|womens|4|Shoes|187|ationeingought|medium|7594816969sky6675362|red|Dozen|Unknown|14|barationationation| +7771|AAAAAAAALFOBAAAA|1997-10-27||About british reasons will draw occasionally practitioners. New attempts shall display in private private, major magazines. Questions dare on a losses. As american children take upwards good symptom|72.70|29.80|8012002|importomaxi #2|12|guns|8|Sports|431|oughtpriese|N/A|6213442frosted094898|papaya|Cup|Unknown|73|oughtationationation| +7772|AAAAAAAAMFOBAAAA|1997-10-27|2000-10-26|Reliable, free miles may speak dates. Managers explain else. Alone short police raise up to periods. Books can invest serious months. Thinking, followi|6.59|4.01|9013009|exportiunivamalg #9|13|self-help|9|Books|169|n stcallyought|N/A|7101593706misty24250|rose|Tsp|Unknown|28|ableationationation| +7773|AAAAAAAAMFOBAAAA|2000-10-27||Reliable, free miles may speak dates. Managers explain else. Alone short police raise up to periods. Books can invest serious months. Thinking, followi|1.36|0.89|1002002|importoamalg #2|13|fragrances|1|Women|566|callycallyanti|medium|7101593706misty24250|dark|Each|Unknown|38|priationationation| +7774|AAAAAAAAOFOBAAAA|1997-10-27|1999-10-27|Concerned, small activities must seem also times. Already international firms used to maintain into a standards. |4.68|3.46|8001005|amalgnameless #5|1|athletic shoes|8|Sports|278|eingationable|N/A|2841671chiffon369824|yellow|Gram|Unknown|82|eseationationation| +7775|AAAAAAAAOFOBAAAA|1999-10-28|2001-10-26|Concerned, small activities must seem also times. Already international firms used to maintain into a standards. |7.83|3.46|8001005|edu packexporti #2|4|school-uniforms|3|Children|423|priableese|small|2841671chiffon369824|peach|Cup|Unknown|52|antiationationation| +7776|AAAAAAAAOFOBAAAA|2001-10-27||Concerned, small activities must seem also times. Already international firms used to maintain into a standards. |3.88|3.46|8001005|corpbrand #7|6|rugs|7|Home|206|priableese|N/A|83rose69586524160455|turquoise|Carton|Unknown|50|callyationationation| +7777|AAAAAAAABGOBAAAA|1997-10-27||Japanese, long students may help very; there partial bombs must assess; intentions cannot execute most certain children; indeed necessary a|5.36|3.43|9004002|edu packmaxi #2|4|entertainments|9|Books|253|priantiable|N/A|596489800372powder14|green|Gross|Unknown|17|ationationationation| +7778|AAAAAAAACGOBAAAA|1997-10-27|2000-10-26|Awkward, poor points cannot weigh plants. Single, reasonable players may not go around scottish products. Then presidential years suffer clubs. Problems would attrac|4.15|2.32|8014009|edu packmaxi #9|14|tennis|8|Sports|214|eseoughtable|N/A|946sienna26401523629|white|Lb|Unknown|8|eingationationation| +7779|AAAAAAAACGOBAAAA|2000-10-27||Awkward, poor points cannot weigh plants. Single, reasonable players may not go around scottish products. Then presidential years suffer clubs. Problems would attrac|2.84|2.32|8014009|importoimporto #2|2|shirts|2|Men|214|eseoughtable|petite|27481117436sky763831|midnight|Pallet|Unknown|51|n stationationation| +7780|AAAAAAAAEGOBAAAA|1997-10-27|1999-10-27|Maj|50.24|33.15|4002001|importoedu pack #1|2|mens|4|Shoes|535|antiprianti|petite|14500966166lawn36242|lavender|Gross|Unknown|30|bareingationation| +7781|AAAAAAAAEGOBAAAA|1999-10-28|2001-10-26|Maj|8.05|33.15|4002001|corpnameless #2|2|furniture|7|Home|535|antiprianti|N/A|43901525papaya910748|slate|Each|Unknown|3|oughteingationation| +7782|AAAAAAAAEGOBAAAA|2001-10-27||Maj|2.64|1.66|1004001|edu packamalg #1|4|swimwear|1|Women|535|antiprianti|small|47026indian051620705|plum|Unknown|Unknown|58|ableeingationation| +7783|AAAAAAAAHGOBAAAA|1997-10-27||Royal things think that clearly free prayers. Temporary errors used to collect catholic, colourful pains. Eggs turn instead units. Even separate farms say soon to a considerati|9.91|7.92|7014004|edu packnameless #4|14|glassware|7|Home|325|antiablepri|N/A|167505300234khaki285|royal|Pallet|Unknown|53|prieingationation| +7784|AAAAAAAAIGOBAAAA|1997-10-27|2000-10-26|Unable, central streets move as new men. Wet, r|9.62|6.63|8010005|univmaxi #5|10|pools|8|Sports|38|eingpri|N/A|sky52332110873321393|powder|Oz|Unknown|32|eseeingationation| +7785|AAAAAAAAIGOBAAAA|2000-10-27||Always public options set ready tears. Public journalists make structures. Here private accountants sound classic visitors; far, other users may suffer there; emotional unio|3.73|2.01|10011017|amalgamalgamalg #17|11|disk drives|10|Electronics|372|eingpri|N/A|6860528peach30216846|metallic|Unknown|Unknown|25|antieingationation| +7786|AAAAAAAAKGOBAAAA|1997-10-27|1999-10-27|New, dangerous yards will go too considerable schools; away tough accounts should form living communists. Patients become unnecessarily clothes. |4.34|3.55|4004001|edu packedu pack #1|4|athletic|4|Shoes|304|esebarpri|N/A|900smoke936326402733|blanched|Each|Unknown|49|callyeingationation| +7787|AAAAAAAAKGOBAAAA|1999-10-28|2001-10-26|Awkwardly unable accountants must end other surveys. Gro|0.31|3.55|4004001|edu packamalg #2|4|swimwear|1|Women|112|ableoughtought|small|6maroon1032894975450|snow|Each|Unknown|15|ationeingationation| +7788|AAAAAAAAKGOBAAAA|2001-10-27||Still communist generatio|3.79|3.55|9016001|corpunivamalg #1|4|mystery|9|Books|112|ableoughtought|N/A|6maroon1032894975450|rosy|Pallet|Unknown|15|eingeingationation| +7789|AAAAAAAANGOBAAAA|1997-10-27||Concerned representatives could see. Always open plans reach over police. Short, reasonab|8.57|7.11|5002002|importoscholar #2|2|country|5|Music|609|n stbarcally|N/A|3791551662627sandy86|violet|N/A|Unknown|62|n steingationation| +7790|AAAAAAAAOGOBAAAA|1997-10-27|2000-10-26|Here great drawings put please affairs. Just impossible minutes shall get other fortunes. Fierce, good parents could sort quite outside a dogs. Workers dem|4.06|1.42|3001001|amalgexporti #1|1|newborn|3|Children|135|antipriought|medium|36884592442535smoke8|peach|Ounce|Unknown|22|barn stationation| +7791|AAAAAAAAOGOBAAAA|2000-10-27||Here great drawings put please affairs. Just impossible minutes shall get other fortunes. Fierce, good parents could sort quite outside a dogs. Workers dem|4.91|1.42|8012002|importomaxi #2|1|guns|8|Sports|176|antipriought|N/A|36884592442535smoke8|thistle|Dram|Unknown|20|oughtn stationation| +7792|AAAAAAAAAHOBAAAA|1997-10-27|1999-10-27|Still german machines should take absolute businesses. Less than |11.39|8.54|6009001|maxicorp #1|9|womens watch|6|Jewelry|132|ablepriought|N/A|55442643849899royal7|snow|Gram|Unknown|38|ablen stationation| +7793|AAAAAAAAAHOBAAAA|1999-10-28|2001-10-26|Fine women must not protect originally for a names; as dark materials shall pre|4.46|8.54|7015004|scholarnameless #4|15|tables|7|Home|132|ablepriought|N/A|55442643849899royal7|ghost|Dozen|Unknown|60|prin stationation| +7794|AAAAAAAAAHOBAAAA|2001-10-27||Wooden adults ought to construct as open rates. New, silver arms join far below normal books. Eith|0.78|8.54|7012005|importonameless #5|12|paint|7|Home|132|ablepriought|N/A|55442643849899royal7|purple|Cup|Unknown|5|esen stationation| +7795|AAAAAAAADHOBAAAA|1997-10-27||Suitable games win personal classes. High duties might show disabled developments. Southern, imperial doubts call sure entirely extra hands. Pub|8.91|6.50|6001004|amalgcorp #4|1|birdal|6|Jewelry|384|eseeingpri|N/A|929946drab6576692501|indian|Each|Unknown|13|antin stationation| +7796|AAAAAAAAEHOBAAAA|1997-10-27|2000-10-26|Already lucky tales can |2.85|2.30|6008005|namelesscorp #5|8|mens watch|6|Jewelry|954|eseantin st|N/A|2275puff738326910718|powder|Carton|Unknown|51|callyn stationation| +7797|AAAAAAAAEHOBAAAA|2000-10-27||Employees grow even small sections. Old thoughts would engage eminently significan|50.27|2.30|6008005|importoscholar #2|8|country|5|Music|954|eseantin st|N/A|100031lavender081288|forest|Pallet|Unknown|70|ationn stationation| +7798|AAAAAAAAGHOBAAAA|1997-10-27|1999-10-27|Fully existing proceedings could not tak|8.66|3.20|9001011|amalgmaxi #11|1|arts|9|Books|481|oughteingese|N/A|73green7202323237990|lawn|Carton|Unknown|89|eingn stationation| +7799|AAAAAAAAGHOBAAAA|1999-10-28|2001-10-26|Fully existing proceedings could not tak|9.27|6.76|9001011|exportischolar #2|1|pop|5|Music|481|oughteingese|N/A|6785919200light88007|spring|Lb|Unknown|22|n stn stationation| +7800|AAAAAAAAGHOBAAAA|2001-10-27||Neighbouring, aware strengths f|53.30|6.76|2003001|exportiimporto #1|1|pants|2|Men|166|callycallyought|extra large|6785919200light88007|ivory|Oz|Unknown|91|barbareingation| +7801|AAAAAAAAJHOBAAAA|1997-10-27||Combined, broad years could not make still better good users. Services provide yet into the functions. Now international policies like across long, econ|7.79|6.30|6007002|brandcorp #2|7|pendants|6|Jewelry|288|eingeingable|N/A|72628spring102540809|steel|Cup|Unknown|5|oughtbareingation| +7802|AAAAAAAAKHOBAAAA|1997-10-27|2000-10-26|Then good students should put only functional figures. Equal years ought to secure. And so on certain legs must not provide similar, current children. New skills|1.52|1.17|9010003|univunivamalg #3|10|travel|9|Books|594|esen stanti|N/A|6759445155946peru025|smoke|Ounce|Unknown|57|ablebareingation| +7803|AAAAAAAAKHOBAAAA|2000-10-27||Then good students should put only functional figures. Equal years ought to secure. And so on certain legs must not provide similar, current children. New skills|0.41|1.17|9010003|exportinameless #4|13|wallpaper|7|Home|594|esen stanti|N/A|535473119chartreuse4|sienna|Gross|Unknown|64|pribareingation| +7804|AAAAAAAAMHOBAAAA|1997-10-27|1999-10-27|Social, positive responsibilities should not form well. Rates paint exclusive terms. Reports endeavour sure, british horses; possible typ|6.73|5.51|5001001|amalgscholar #1|1|rock|5|Music|891|oughtn steing|N/A|7502953859snow474923|powder|Case|Unknown|41|esebareingation| +7805|AAAAAAAAMHOBAAAA|1999-10-28|2001-10-26|Social, positive responsibilities should not form well. Rates paint exclusive terms. Reports endeavour sure, british horses; possible typ|0.94|5.51|5001001|importobrand #6|1|costume|6|Jewelry|57|ationanti|N/A|7502953859snow474923|peru|Bunch|Unknown|42|antibareingation| +7806|AAAAAAAAMHOBAAAA|2001-10-27||Social, positive responsibilities should not form well. Rates paint exclusive terms. Reports endeavour sure, british horses; possible typ|0.48|5.51|5001001|exportiimporto #1|1|pants|2|Men|57|ationanti|medium|7502953859snow474923|bisque|Dozen|Unknown|55|callybareingation| +7807|AAAAAAAAPHOBAAAA|1997-10-27||Ethnic, female p|3.90|2.06|2002002|importoimporto #2|2|shirts|2|Men|328|eingablepri|small|7751807641068orchid2|light|Oz|Unknown|24|ationbareingation| +7808|AAAAAAAAAIOBAAAA|1997-10-27|2000-10-26|Dead|4.21|2.10|2004001|edu packimporto #1|4|sports-apparel|2|Men|384|eseeingpri|petite|413maroon12773054504|light|Box|Unknown|30|eingbareingation| +7809|AAAAAAAAAIOBAAAA|2000-10-27||Memories shall acquire parents. Alrea|5.36|2.10|2004001|exportibrand #10|3|kids|7|Home|47|eseeingpri|N/A|413maroon12773054504|ghost|Bunch|Unknown|4|n stbareingation| +7810|AAAAAAAACIOBAAAA|1997-10-27|1999-10-27|New, atomic terms could not overlap probably now changing views; urban, definite problems may answer too person|35.98|19.42|3002001|importoexporti #1|2|infants|3|Children|288|eingeingable|extra large|595463801930white556|ivory|Pallet|Unknown|52|baroughteingation| +7811|AAAAAAAACIOBAAAA|1999-10-28|2001-10-26|Sure, agricultura|3.50|19.42|4001002|amalgedu pack #2|1|womens|4|Shoes|288|eingeingable|medium|595463801930white556|green|Ounce|Unknown|62|oughtoughteingation| +7812|AAAAAAAACIOBAAAA|2001-10-27||Sure, agricultura|63.71|19.42|10006010|corpunivamalg #10|6|musical|10|Electronics|288|eingeingable|N/A|595463801930white556|misty|Box|Unknown|20|ableoughteingation| +7813|AAAAAAAAFIOBAAAA|1997-10-27||Basic, clear agents believe with a products. There lost pages rely then left things. Young, significant sales know still at the margins. Future, total decisions could fill during the notes. Fundam|1.13|0.65|3002002|importoexporti #2|2|infants|3|Children|706|callybaration|medium|6137964871rose595173|misty|Box|Unknown|35|prioughteingation| +7814|AAAAAAAAGIOBAAAA|1997-10-27|2000-10-26|Central, likely eyes sit finally proposals. Impossible, regular facilities win alone existing, private years. Sensible, |1.52|0.71|6001007|amalgcorp #7|1|birdal|6|Jewelry|789|n steingation|N/A|4200718olive71360674|slate|Dram|Unknown|66|eseoughteingation| +7815|AAAAAAAAGIOBAAAA|2000-10-27||Simply cheap sessions build figures. Local performances find often please likely hours. |3.40|0.71|9016004|corpunivamalg #4|1|mystery|9|Books|256|n steingation|N/A|0633991496892tomato2|seashell|Gross|Unknown|33|antioughteingation| +7816|AAAAAAAAIIOBAAAA|1997-10-27|1999-10-27|Marvellous, high hands for|6.07|1.94|9005005|scholarmaxi #5|5|history|9|Books|121|oughtableought|N/A|216325103red50356968|papaya|Oz|Unknown|71|callyoughteingation| +7817|AAAAAAAAIIOBAAAA|1999-10-28|2001-10-26|Suddenly specific men overcome equally as british sections. Growing wings used to try most mate|3.50|1.94|9005005|exportiunivamalg #6|5|self-help|9|Books|121|oughtableought|N/A|216325103red50356968|olive|Each|Unknown|72|ationoughteingation| +7818|AAAAAAAAIIOBAAAA|2001-10-27||Alone secret children prove socially routine competitors. |3.35|1.94|9005005|corpcorp #3|6|rings|6|Jewelry|121|oughtableought|N/A|62thistle17401409882|floral|N/A|Unknown|11|eingoughteingation| +7819|AAAAAAAALIOBAAAA|1997-10-27||So no|73.22|53.45|9007008|brandmaxi #8|7|reference|9|Books|502|ablebaranti|N/A|409midnight264149290|lemon|Unknown|Unknown|52|n stoughteingation| +7820|AAAAAAAAMIOBAAAA|1997-10-27|2000-10-26|Precise, rich things ought to enforce therefore powers. Assets get less with a males. Too info|0.40|0.21|6010005|univbrand #5|10|jewelry boxes|6|Jewelry|271|oughtationable|N/A|98974893092yellow729|slate|Bundle|Unknown|8|barableeingation| +7821|AAAAAAAAMIOBAAAA|2000-10-27||For example shallow babies clear different classes. Weeks prosecute well local readers. Original women could not discount once more other writers; supreme, good parties |2.89|2.45|10010002|univamalgamalg #2|10|memory|10|Electronics|278|oughtationable|N/A|98974893092yellow729|plum|Case|Unknown|12|oughtableeingation| +7822|AAAAAAAAOIOBAAAA|1997-10-27|1999-10-27|There pale members try a little cheap feet. Golden, o|65.21|53.47|9011011|amalgunivamalg #11|11|cooking|9|Books|179|n stationought|N/A|5slate75950486082278|violet|Pound|Unknown|50|ableableeingation| +7823|AAAAAAAAOIOBAAAA|1999-10-28|2001-10-26|There pale members try a little cheap feet. Golden, o|0.93|53.47|3001002|amalgexporti #2|1|newborn|3|Children|179|n stationought|large|5slate75950486082278|sky|Lb|Unknown|18|priableeingation| +7824|AAAAAAAAOIOBAAAA|2001-10-27||Diverse numbers used to seem furthermore complete, new words. Citizens will park cells. As whole skills might look now at the times. Relations sort spirits. Parents give to a candidate|7.06|53.47|3001002|amalgunivamalg #5|1|cameras|10|Electronics|179|n stationought|N/A|49222navy02705359745|saddle|Carton|Unknown|56|eseableeingation| +7825|AAAAAAAABJOBAAAA|1997-10-27||Annual, normal arguments should exercise thus times. White resources ought to cope|2.19|0.78|3001002|amalgexporti #2|1|newborn|3|Children|96|callyn st|N/A|121534sky17891743821|steel|Cup|Unknown|10|antiableeingation| +7826|AAAAAAAACJOBAAAA|1997-10-27|2000-10-26|Else local pairs shall lose by a goods. Simple measures|7.15|3.64|5003001|exportischolar #1|3|pop|5|Music|343|priesepri|N/A|7papaya4766124924697|moccasin|Each|Unknown|23|callyableeingation| +7827|AAAAAAAACJOBAAAA|2000-10-27||Else local pairs shall lose by a goods. Simple measures|3.85|3.64|5003001|exportischolar #2|3|pop|5|Music|343|priesepri|N/A|695yellow27751459301|powder|Ton|Unknown|50|ationableeingation| +7828|AAAAAAAAEJOBAAAA|1997-10-27|1999-10-27|Suddenly concerned employees provide complete variations. Great, judicial months may block about probable names. Legal, old goals learn now observations; elections could mean as available spirit|7.52|5.56|1001001|amalgamalg #1|1|dresses|1|Women|69|n stcally|medium|10843631149puff86710|yellow|Case|Unknown|27|eingableeingation| +7829|AAAAAAAAEJOBAAAA|1999-10-28|2001-10-26|Similar, big sources would mount categories. Modern rate|8.60|4.47|1001001|importoscholar #2|1|country|5|Music|69|n stcally|N/A|10843631149puff86710|papaya|Lb|Unknown|12|n stableeingation| +7830|AAAAAAAAEJOBAAAA|2001-10-27||Operations provide far social parties. Basic lists must handle still activities. Needs would not look then wonderful cases. Much social places could reach |3.78|3.17|9014007|edu packunivamalg #7|14|sports|9|Books|69|n stcally|N/A|10843631149puff86710|lemon|Each|Unknown|79|barprieingation| +7831|AAAAAAAAHJOBAAAA|1997-10-27||Highly local li|9.81|7.16|8014002|edu packmaxi #2|14|tennis|8|Sports|186|callyeingought|N/A|23turquoise378271354|peru|Tbl|Unknown|72|oughtprieingation| +7832|AAAAAAAAIJOBAAAA|1997-10-27|2000-10-26|Others burn. Systems should increase for instance others. Supposedly|4.79|1.48|3004001|edu packexporti #1|4|school-uniforms|3|Children|566|callycallyanti|large|52455sienna951793211|orchid|Oz|Unknown|31|ableprieingation| +7833|AAAAAAAAIJOBAAAA|2000-10-27||National groups choose unions. Years might work even old troops. American, short sources may smile trying ways. Necessary, long-term things could not play better on a groups. Existing steps require wi|4.09|1.39|8005004|scholarnameless #4|4|fitness|8|Sports|566|callycallyanti|N/A|662604drab0706373453|lace|Cup|Unknown|40|priprieingation| +7834|AAAAAAAAKJOBAAAA|1997-10-27|1999-10-27|Specific sales infer. New police die lightly games. Years make so sure images. Students talk else. Never |4.04|3.03|5004001|edu packscholar #1|4|classical|5|Music|38|eingpri|N/A|46rosy23134349809690|purple|Box|Unknown|39|eseprieingation| +7835|AAAAAAAAKJOBAAAA|1999-10-28|2001-10-26|Top, s|2.34|3.03|1003002|exportiamalg #2|3|maternity|1|Women|307|eingpri|extra large|7peru974828792052026|hot|Pound|Unknown|5|antiprieingation| +7836|AAAAAAAAKJOBAAAA|2001-10-27||Alleged, normal ideas could ask. Only light glasses see shared, new ways. Directors cannot labour just |2.71|3.03|1003002|exportischolar #1|3|pop|5|Music|307|ationbarpri|N/A|7peru974828792052026|powder|Carton|Unknown|43|callyprieingation| +7837|AAAAAAAANJOBAAAA|1997-10-27||Young, long letters know men. New, labour results concentrate overseas more known details. Effects rob; severe, high horses suspect relations; thick,|4.13|2.18|5002002|importoscholar #2|2|country|5|Music|758|eingantiation|N/A|21513912956553pale16|antique|Ounce|Unknown|37|ationprieingation| +7838|AAAAAAAAOJOBAAAA|1997-10-27|2000-10-26|Soft feet must facilitate often classes; children include british, able services. Friends obtain funny, great orders. Single goods search at a offices. Electoral years o|9.73|3.50|3002001|importoexporti #1|2|infants|3|Children|660|barcallycally|medium|898tan93038779603723|plum|Ton|Unknown|18|eingprieingation| +7839|AAAAAAAAOJOBAAAA|2000-10-27||Environmental hours should add completely alone scientists; centres appear more now young eyes. Goods join only yards. Inherent, strong texts sh|7.95|3.33|3002001|amalgamalgamalg #3|11|disk drives|10|Electronics|660|barcallycally|N/A|898tan93038779603723|yellow|Ton|Unknown|21|n stprieingation| +7840|AAAAAAAAAKOBAAAA|1997-10-27|1999-10-27|Wings deal. Free restrictions think t|3.49|1.71|8015001|scholarmaxi #1|15|fishing|8|Sports|167|ationcallyought|N/A|6363910thistle367067|grey|Pallet|Unknown|31|bareseeingation| +7841|AAAAAAAAAKOBAAAA|1999-10-28|2001-10-26|Wings deal. Free restrictions think t|5.46|4.75|8012002|importomaxi #2|12|guns|8|Sports|598|ationcallyought|N/A|6363910thistle367067|grey|Gram|Unknown|72|oughteseeingation| +7842|AAAAAAAAAKOBAAAA|2001-10-27||Adequate, pure town|9.65|4.75|8012002|scholarnameless #3|5|fitness|8|Sports|702|ationcallyought|N/A|98748443white0076655|violet|Carton|Unknown|48|ableeseeingation| +7843|AAAAAAAADKOBAAAA|1997-10-27||Surprising, strong effects may win more high, civil elements. New concentrations might cover articles; forms used to face courts. Departments shall assign for long inadequate details; suspiciously m|7.66|5.43|6014004|edu packbrand #4|14|estate|6|Jewelry|36|callypri|N/A|01748915yellow083668|plum|Dozen|Unknown|3|prieseeingation| +7844|AAAAAAAAEKOBAAAA|1997-10-27|2000-10-26|British animals ensure bitter, great doctors. A|0.94|0.82|10003008|exportiunivamalg #8|3|dvd/vcr players|10|Electronics|479|n stationese|N/A|87499389spring835151|tan|Bunch|Unknown|92|eseeseeingation| +7845|AAAAAAAAEKOBAAAA|2000-10-27||Real trees mean even for |2.73|2.10|10003008|amalgexporti #2|1|newborn|3|Children|479|n stationese|medium|87499389spring835151|midnight|Ton|Unknown|79|antieseeingation| +7846|AAAAAAAAGKOBAAAA|1997-10-27|1999-10-27|Feat|0.26|0.16|1003001|exportiamalg #1|3|maternity|1|Women|265|anticallyable|petite|429purple39720759734|lime|Ton|Unknown|27|callyeseeingation| +7847|AAAAAAAAGKOBAAAA|1999-10-28|2001-10-26|Feat|9.15|6.40|1003001|edu packamalg #2|4|swimwear|1|Women|265|anticallyable|small|872832257677olive750|violet|Bundle|Unknown|6|ationeseeingation| +7848|AAAAAAAAGKOBAAAA|2001-10-27||Subtle things must not follow for example. Mainly pale weeks cannot arrange ever up to a times. Electoral techn|1.08|0.58|1003001|exportiamalg #1|3|maternity|1|Women|265|anticallyable|medium|872832257677olive750|powder|N/A|Unknown|3|eingeseeingation| +7849|AAAAAAAAJKOBAAAA|1997-10-27||Big, similar lines will give states. Other, whole functions keep carefully. Customers cannot change especially wide origins. Planned police will not |3.05|2.16|7004010|edu packbrand #10|4|curtains/drapes|7|Home|115|antioughtought|N/A|14636001408midnight5|puff|Ton|Unknown|25|n steseeingation| +7850|AAAAAAAAKKOBAAAA|1997-10-27|2000-10-26|Hours woul|2.11|1.81|7009001|maxibrand #1|9|mattresses|7|Home|30|barpri|N/A|443314448green428540|pink|Carton|Unknown|53|barantieingation| +7851|AAAAAAAAKKOBAAAA|2000-10-27||Hours woul|1.18|1.81|7009001|corpcorp #4|9|rings|6|Jewelry|575|antiationanti|N/A|443314448green428540|saddle|Gram|Unknown|18|oughtantieingation| +7852|AAAAAAAAMKOBAAAA|1997-10-27|1999-10-27|New, western men can fix also in the firms. Parallel major protests say various, soft designers. Actually solar communities keep at a soldiers. Large clothes would show historical, objective wom|2.55|0.94|2001001|amalgimporto #1|1|accessories|2|Men|188|eingeingought|economy|1yellow7730044668160|yellow|Dram|Unknown|23|ableantieingation| +7853|AAAAAAAAMKOBAAAA|1999-10-28|2001-10-26|Others could go both friends; main goods cannot adopt. Hot machines must achieve as well apparently legal girls. Etc central years confer directly. Children |2.68|0.94|2001001|exportibrand #6|1|loose stones|6|Jewelry|125|eingeingought|N/A|23003midnight4432948|thistle|Oz|Unknown|37|priantieingation| +7854|AAAAAAAAMKOBAAAA|2001-10-27||Others could go both friends; main goods cannot adopt. Hot machines must achieve as well apparently legal girls. Etc central years confer directly. Children |9.54|7.44|2004001|edu packimporto #1|4|sports-apparel|2|Men|125|antiableought|extra large|967891774turquoise63|orange|Carton|Unknown|16|eseantieingation| +7855|AAAAAAAAPKOBAAAA|1997-10-27||Yet whole dealers p|3.63|3.23|9003008|exportimaxi #8|3|computers|9|Books|323|priablepri|N/A|605657violet09530916|red|Oz|Unknown|52|antiantieingation| +7856|AAAAAAAAALOBAAAA|1997-10-27|2000-10-26|Thoughts shall care religious employers; explicitly roman times work environmental girls. Recent heads see l|32.91|9.87|1004001|edu packamalg #1|4|swimwear|1|Women|196|callyn stought|medium|637491348honeydew635|saddle|Tsp|Unknown|35|callyantieingation| +7857|AAAAAAAAALOBAAAA|2000-10-27||Applicants will comment most complete arms; only northern men c|2.98|9.87|1004001|namelessnameless #8|4|outdoor|8|Sports|29|n stable|N/A|637491348honeydew635|salmon|Bundle|Unknown|48|ationantieingation| +7858|AAAAAAAACLOBAAAA|1997-10-27|1999-10-27|Years will include more apparent wives. Police may speak else additional aspects. Researchers move almost french changes. Eyes will recall surely. Still social teachers turn by a rec|8.48|5.68|10006005|corpunivamalg #5|6|musical|10|Electronics|137|ationpriought|N/A|30133609367spring161|purple|Gross|Unknown|25|eingantieingation| +7859|AAAAAAAACLOBAAAA|1999-10-28|2001-10-26|Small glasses would not reckon churches; medical investments used to know now in the papers; primary runs split however local knees. Best african notes start too irrelevant, tender days; th|1.06|5.68|10006006|corpunivamalg #6|6|musical|10|Electronics|72|ationpriought|N/A|30133609367spring161|thistle|N/A|Unknown|8|n stantieingation| +7860|AAAAAAAACLOBAAAA|2001-10-27||Rates should feel simply military, different friends. Natural disputes might tie then from a levels. Hot streets stay ke|9.17|5.68|10006006|exportimaxi #1|13|sailing|8|Sports|257|ationpriought|N/A|30133609367spring161|royal|Gross|Unknown|24|barcallyeingation| +7861|AAAAAAAAFLOBAAAA|1997-10-27||Then main numbers become sufficiently in the ports. Amazing, small positions see then sound arms. Metres would not get simply recently actual securities. Basically tired numbers shall get so |9.04|4.06|1004002|edu packamalg #2|4|swimwear|1|Women|5|anti|extra large|3681ivory18675479214|grey|Each|Unknown|24|oughtcallyeingation| +7862|AAAAAAAAGLOBAAAA|1997-10-27|2000-10-26|Finally h|3.12|1.80|4003001|exportiedu pack #1|3|kids|4|Shoes|447|ationeseese|small|988olive593313518252|maroon|Oz|Unknown|14|ablecallyeingation| +7863|AAAAAAAAGLOBAAAA|2000-10-27||Finally h|3.99|3.07|4003001|maximaxi #4|9|science|9|Books|447|ationeseese|N/A|4298618477pink663946|papaya|Ounce|Unknown|32|pricallyeingation| +7864|AAAAAAAAILOBAAAA|1997-10-27|1999-10-27|Long agricultural communications can get absolutely applications. Following p|0.80|0.49|6004001|edu packcorp #1|4|bracelets|6|Jewelry|372|ableationpri|N/A|65601716red708502074|rosy|Gross|Unknown|44|esecallyeingation| +7865|AAAAAAAAILOBAAAA|1999-10-28|2001-10-26|Concentrations ought to resign also true, great foreigners. Nations relish therefore by a criticisms.|9.73|0.49|3004002|edu packexporti #2|4|school-uniforms|3|Children|795|ableationpri|medium|65601716red708502074|pale|Pallet|Unknown|39|anticallyeingation| +7866|AAAAAAAAILOBAAAA|2001-10-27||Concentrations ought to resign also true, great foreigners. Nations relish therefore by a criticisms.|9.60|7.00|3004002|amalgcorp #3|1|birdal|6|Jewelry|795|ableationpri|N/A|87ivory5826600288633|lime|Case|Unknown|32|callycallyeingation| +7867|AAAAAAAALLOBAAAA|1997-10-27||Obvious problems may find |4.50|1.80|9010008|univunivamalg #8|10|travel|9|Books|65|antically|N/A|1199666342powder3541|lavender|Carton|Unknown|49|ationcallyeingation| +7868|AAAAAAAAMLOBAAAA|1997-10-27|2000-10-26|Passages provide differences; so effective streets used to mount then. Literary, mass women might go beautiful boys|0.54|0.24|3003001|exportiexporti #1|3|toddlers|3|Children|816|callyoughteing|small|022520metallic653072|ivory|Gross|Unknown|31|eingcallyeingation| +7869|AAAAAAAAMLOBAAAA|2000-10-27||Statutory, poor rights keep. Long labour areas would not fix further. Across ready streets try previously likely citizens. Paths sneak mad centuries. Commercial contr|9.87|0.24|6011006|amalgbrand #6|3|semi-precious|6|Jewelry|816|callyoughteing|N/A|022520metallic653072|yellow|Dram|Unknown|33|n stcallyeingation| +7870|AAAAAAAAOLOBAAAA|1997-10-27|1999-10-27|Just good unions used to return especially.|1.11|0.77|6003007|exporticorp #7|3|gold|6|Jewelry|163|pricallyought|N/A|4white64047680503194|burnished|Cup|Unknown|57|barationeingation| +7871|AAAAAAAAOLOBAAAA|1999-10-28|2001-10-26|Final books will mind similarly in a vehicles. Perhaps democratic contents arrive also by an fears. Then brilliant women might afford. New, standard letters offer specifically. Foreign, partic|64.09|0.77|6003007|exportiamalg #2|3|maternity|1|Women|163|pricallyought|extra large|4white64047680503194|orange|Dozen|Unknown|60|oughtationeingation| +7872|AAAAAAAAOLOBAAAA|2001-10-27||Final books will mind similarly in a vehicles. Perhaps democratic contents arrive also by an fears. Then brilliant women might afford. New, standard letters offer specifically. Foreign, partic|9.28|0.77|7002003|importobrand #3|2|bedding|7|Home|163|pricallyought|N/A|4white64047680503194|purple|Pallet|Unknown|7|ableationeingation| +7873|AAAAAAAABMOBAAAA|1997-10-27||Reasons feel other, literary details. Nearby, typical years believe to th|2.93|2.43|6009002|maxicorp #2|9|womens watch|6|Jewelry|526|callyableanti|N/A|610royal947910961468|puff|Unknown|Unknown|57|priationeingation| +7874|AAAAAAAACMOBAAAA|1997-10-27|2000-10-26|Techniques cannot put on a sisters; useful injuries will encourage also sufficiently other sizes. Signs rescu|6.04|4.28|10012004|importoamalgamalg #4|12|monitors|10|Electronics|120|barableought|N/A|862926steel658076529|peru|Ounce|Unknown|57|eseationeingation| +7875|AAAAAAAACMOBAAAA|2000-10-27||Techniques cannot put on a sisters; useful injuries will encourage also sufficiently other sizes. Signs rescu|0.11|0.07|10012004|amalgamalg #2|1|dresses|1|Women|120|barableought|medium|862926steel658076529|rosy|Carton|Unknown|27|antiationeingation| +7876|AAAAAAAAEMOBAAAA|1997-10-27|1999-10-27|Suspicious|4.26|1.95|9016005|corpunivamalg #5|16|mystery|9|Books|315|antioughtpri|N/A|99peach3551390344231|pink|Each|Unknown|41|callyationeingation| +7877|AAAAAAAAEMOBAAAA|1999-10-28|2001-10-26|Passionately individual practitioners relax just alongside a businessme|9.88|1.95|7004008|edu packbrand #8|4|curtains/drapes|7|Home|315|antioughtpri|N/A|99peach3551390344231|royal|Lb|Unknown|20|ationationeingation| +7878|AAAAAAAAEMOBAAAA|2001-10-27||Passionately individual practitioners relax just alongside a businessme|1.14|1.95|7004008|maxiunivamalg #8|9|televisions|10|Electronics|644|antioughtpri|N/A|99peach3551390344231|black|Bunch|Unknown|54|eingationeingation| +7879|AAAAAAAAHMOBAAAA|1997-10-27||Big, huge payments used to send however. Exactly long systems must not expose never english circumstances. Just national teeth find only. Close, great panels meet already windows. Quickly hostile qu|0.16|0.12|10016009|corpamalgamalg #9|16|wireless|10|Electronics|170|barationought|N/A|46steel4388355084309|violet|Carton|Unknown|51|n stationeingation| +7880|AAAAAAAAIMOBAAAA|1997-10-27|2000-10-26|Easily dry communities meet much harsh tears. Heavy minutes damage members. Industrial securiti|6.81|4.08|9013009|exportiunivamalg #9|13|self-help|9|Books|208|eingbarable|N/A|60925145391light0351|turquoise|Tsp|Unknown|16|bareingeingation| +7881|AAAAAAAAIMOBAAAA|2000-10-27||Documents may improve active |4.35|2.52|2002002|importoimporto #2|13|shirts|2|Men|573|priationanti|medium|15169746puff06882671|yellow|Lb|Unknown|63|oughteingeingation| +7882|AAAAAAAAKMOBAAAA|1997-10-27|1999-10-27|Nevertheless trying police can behave more cognitive, awful classes. Foreign rivers might make easily others. Dogs may score with a diffe|6.45|3.35|3001001|amalgexporti #1|1|newborn|3|Children|215|antioughtable|medium|534975279572khaki109|violet|Ton|Unknown|4|ableeingeingation| +7883|AAAAAAAAKMOBAAAA|1999-10-28|2001-10-26|Nevertheless trying police can behave more cognitive, awful classes. Foreign rivers might make easily others. Dogs may score with a diffe|2.32|3.35|10007013|brandunivamalg #13|1|personal|10|Electronics|371|oughtationpri|N/A|604pale0648842574318|snow|Gram|Unknown|14|prieingeingation| +7884|AAAAAAAAKMOBAAAA|2001-10-27||Heavily certain arms act as main, provincial visitors. Absent, real eyes show then english hands. In particular american|82.00|3.35|10007013|exportimaxi #1|1|computers|9|Books|371|oughtationpri|N/A|385141056puff6383410|lawn|Case|Unknown|52|eseeingeingation| +7885|AAAAAAAANMOBAAAA|1997-10-27||Much wide funds mus|1.01|0.53|3004002|edu packexporti #2|4|school-uniforms|3|Children|113|prioughtought|medium|170630587sky69767844|steel|Case|Unknown|46|antieingeingation| +7886|AAAAAAAAOMOBAAAA|1997-10-27|2000-10-26|Please big pr|8.52|2.64|4004001|edu packedu pack #1|4|athletic|4|Shoes|328|eingablepri|petite|5217093986bisque2766|sky|Dozen|Unknown|36|callyeingeingation| +7887|AAAAAAAAOMOBAAAA|2000-10-27||Horrible, young journalists listen well certain prices. Now concerned perceptions believe in the students. Bits might buy however traditions. Horses an|41.38|24.00|4001002|amalgedu pack #2|4|womens|4|Shoes|328|eingablepri|petite|336069182324purple45|salmon|Tsp|Unknown|21|ationeingeingation| +7888|AAAAAAAAANOBAAAA|1997-10-27|1999-10-27|Wo|6.52|4.43|6007001|brandcorp #1|7|pendants|6|Jewelry|232|ablepriable|N/A|peru4718631369947241|turquoise|Dozen|Unknown|2|eingeingeingation| +7889|AAAAAAAAANOBAAAA|1999-10-28|2001-10-26|Wo|2.10|1.17|6006002|corpcorp #2|6|rings|6|Jewelry|232|ablepriable|N/A|0orchid2347863581809|wheat|Carton|Unknown|31|n steingeingation| +7890|AAAAAAAAANOBAAAA|2001-10-27||Wo|5.66|2.03|6006002|scholarbrand #1|5|blinds/shades|7|Home|102|ablepriable|N/A|8266thistle105031043|yellow|Oz|Unknown|26|barn steingation| +7891|AAAAAAAADNOBAAAA|1997-10-27||Never certain bars name then naked, little schools. Forever single leaves distinguish; activities wait far final women. Ways should not arise. Close, big hours believ|7.81|3.51|4002002|importoedu pack #2|2|mens|4|Shoes|54|eseanti|medium|6456turquoise3454674|puff|Bunch|Unknown|24|oughtn steingation| +7892|AAAAAAAAENOBAAAA|1997-10-27|2000-10-26|Suitable characteristics block quite right parents. Full women should not store also t|92.56|57.38|3002001|importoexporti #1|2|infants|3|Children|116|callyoughtought|medium|1453powder2175150528|rose|Case|Unknown|59|ablen steingation| +7893|AAAAAAAAENOBAAAA|2000-10-27||Suitable characteristics block quite right parents. Full women should not store also t|4.12|2.96|3002001|amalgamalgamalg #6|11|disk drives|10|Electronics|116|callyoughtought|N/A|tomato38122539431480|seashell|Case|Unknown|32|prin steingation| +7894|AAAAAAAAGNOBAAAA|1997-10-27|1999-10-27|Colleges cannot create quickly great relations; significant methods pour as educational, constant po|5.95|4.64|9015011|scholarunivamalg #11|15|fiction|9|Books|224|eseableable|N/A|7148944488987violet0|smoke|Gross|Unknown|39|esen steingation| +7895|AAAAAAAAGNOBAAAA|1999-10-28|2001-10-26|Recent, personal systems may wrap running effects. Good, new memories ought to like so aspects. British, high streets should read more in a things. New, aware subj|3.71|4.64|9015011|exportiamalg #2|3|maternity|1|Women|224|eseableable|medium|245turquoise44041256|papaya|Bundle|Unknown|86|antin steingation| +7896|AAAAAAAAGNOBAAAA|2001-10-27||Recent, personal systems may wrap running effects. Good, new memories ought to like so aspects. British, high streets should read more in a things. New, aware subj|6.95|4.64|9015011|importoedu pack #1|2|mens|4|Shoes|224|eseableable|large|6midnight12828176280|purple|Gram|Unknown|42|callyn steingation| +7897|AAAAAAAAJNOBAAAA|1997-10-27||National attitudes must get. Words could learn there officials. Teachers dream only tales. Stories despair guests. |9.68|4.16|10011010|amalgamalgamalg #10|11|disk drives|10|Electronics|90|barn st|N/A|6368720914moccasin16|steel|Tbl|Unknown|62|ationn steingation| +7898|AAAAAAAAKNOBAAAA|1997-10-27|2000-10-26|Nar|4.37|2.75|6005003|scholarcorp #3|5|earings|6|Jewelry|60|barcally|N/A|6336683salmon1144854|sandy|Case|Unknown|51|eingn steingation| +7899|AAAAAAAAKNOBAAAA|2000-10-27||Late ordinary enterprises try religious levels. Natural sales would se|8.86|4.69|6005003|importoimporto #2|5|shirts|2|Men|193|barcally|extra large|6336683salmon1144854|floral|Ton|Unknown|77|n stn steingation| +7900|AAAAAAAAMNOBAAAA|1997-10-27|1999-10-27|Scientific, young creditors might see for the alternativ|6.98|2.79|9003005|exportimaxi #5|3|computers|9|Books|304|esebarpri|N/A|157rosy5586730577689|spring|Gram|Unknown|21|barbarn station| +7901|AAAAAAAAMNOBAAAA|1999-10-28|2001-10-26|Scientific, young creditors might see for the alternativ|6.61|2.79|3001002|amalgexporti #2|3|newborn|3|Children|304|esebarpri|N/A|7677638smoke08097229|dim|Carton|Unknown|3|oughtbarn station| +7902|AAAAAAAAMNOBAAAA|2001-10-27||So technological laws know no longer about international weeks. Authorities might not pick again loan|3.51|2.45|2002001|importoimporto #1|3|shirts|2|Men|251|esebarpri|small|7677638smoke08097229|chocolate|Dozen|Unknown|28|ablebarn station| +7903|AAAAAAAAPNOBAAAA|1997-10-27||Usual presents shall know here; then gross twins will use. Only wooden races walk joint times. Magnificent, america|9.87|7.69|1002002|importoamalg #2|2|fragrances|1|Women|266|callycallyable|large|98sandy2812701044244|indian|Dozen|Unknown|81|pribarn station| +7904|AAAAAAAAAOOBAAAA|1997-10-27|2000-10-26|Right, respectable values cannot remember seriously considerable, national contacts. Most statistical hours speak across mental, democratic ex|3.31|1.72|2002001|importoimporto #1|2|shirts|2|Men|144|eseeseought|medium|8178923607598puff747|peru|Oz|Unknown|95|esebarn station| +7905|AAAAAAAAAOOBAAAA|2000-10-27||Right, respectable values cannot remember seriously considerable, national contacts. Most statistical hours speak across mental, democratic ex|6.12|2.99|2002001|scholarbrand #2|15|custom|6|Jewelry|66|callycally|N/A|355plum4185963591500|maroon|Oz|Unknown|96|antibarn station| +7906|AAAAAAAACOOBAAAA|1997-10-27|1999-10-27|Dear, sexual objects might not pay. Often proper parts deal so meth|6.63|4.90|3002001|importoexporti #1|2|infants|3|Children|186|callyeingought|large|0honeydew29485113551|turquoise|Dram|Unknown|24|callybarn station| +7907|AAAAAAAACOOBAAAA|1999-10-28|2001-10-26|Stiff old products mean; etc basic requirements shall eliminate proposed comparisons. Still little years reach royal services. Rathe|2.58|1.67|7001008|amalgbrand #8|1|bathroom|7|Home|186|callyeingought|N/A|94623gainsboro835688|moccasin|Box|Unknown|20|ationbarn station| +7908|AAAAAAAACOOBAAAA|2001-10-27||Stiff old products mean; etc basic requirements shall eliminate proposed comparisons. Still little years reach royal services. Rathe|4.00|1.67|9003001|exportimaxi #1|1|computers|9|Books|186|callyeingought|N/A|94623gainsboro835688|slate|Pound|Unknown|3|eingbarn station| +7909|AAAAAAAAFOOBAAAA|1997-10-27||English, soviet subjects explode even. Countries stand then on a techniques. However increased concepts introduce now agricultural parts. Open s|0.74|0.32|4003002|exportiedu pack #2|3|kids|4|Shoes|60|barcally|extra large|01944sandy4298124807|honeydew|Case|Unknown|48|n stbarn station| +7910|AAAAAAAAGOOBAAAA|1997-10-27|2000-10-26|Move|5.46|2.73|1003001|exportiamalg #1|3|maternity|1|Women|797|ationn station|petite|6614375peru086321327|spring|Ton|Unknown|46|baroughtn station| +7911|AAAAAAAAGOOBAAAA|2000-10-27||Options discover freely in a policies; national, annual contracts would know. Whole, irish eyes would find urgent systems; bank|1.56|2.73|1003001|edu packscholar #2|3|classical|5|Music|175|ationn station|N/A|6614375peru086321327|sky|Bunch|Unknown|49|oughtoughtn station| +7912|AAAAAAAAIOOBAAAA|1997-10-27|1999-10-27|Super trends maximise warmly in a minutes; hardly european relationships may free us|8.77|6.75|10010008|univamalgamalg #8|10|memory|10|Electronics|442|ableeseese|N/A|8377834040spring9014|rose|Tsp|Unknown|14|ableoughtn station| +7913|AAAAAAAAIOOBAAAA|1999-10-28|2001-10-26|Parts|6.06|1.81|6015002|scholarbrand #2|10|custom|6|Jewelry|337|ationpripri|N/A|8377834040spring9014|goldenrod|Box|Unknown|20|prioughtn station| +7914|AAAAAAAAIOOBAAAA|2001-10-27||Parts|3.45|1.82|6015002|maxibrand #5|10|mattresses|7|Home|12|ableought|N/A|8377834040spring9014|peru|Bunch|Unknown|7|eseoughtn station| +7915|AAAAAAAALOOBAAAA|1997-10-27||Potential values ought to clear apart. Alarmingly like groups can board more unusual part|2.91|1.89|7013006|exportinameless #6|13|wallpaper|7|Home|212|ableoughtable|N/A|449661669265356pink2|snow|Pound|Unknown|52|antioughtn station| +7916|AAAAAAAAMOOBAAAA|1997-10-27|2000-10-26|Linguistic sports keep in a children; specific, poor children should play cards. Political systems go typically short gardens; only, possible un|4.26|3.70|6007005|brandcorp #5|7|pendants|6|Jewelry|380|bareingpri|N/A|92268777lace00000391|floral|Lb|Unknown|4|callyoughtn station| +7917|AAAAAAAAMOOBAAAA|2000-10-27||As considerable univers|0.84|0.74|6007005|importoedu pack #2|7|mens|4|Shoes|286|callyeingable|small|olive678771796136669|violet|Gram|Unknown|48|ationoughtn station| +7918|AAAAAAAAOOOBAAAA|1997-10-27|1999-10-27|Patients gain less temporary words. Certainly spe|4.35|2.52|1001001|amalgamalg #1|1|dresses|1|Women|384|eseeingpri|medium|409935072indian67486|salmon|Lb|Unknown|69|eingoughtn station| +7919|AAAAAAAAOOOBAAAA|1999-10-28|2001-10-26|Similar, foreign eyes should send over a states. Opportunities occur then necessary, iris|2.90|2.52|1001001|edu packimporto #2|4|sports-apparel|2|Men|384|eseeingpri|extra large|409935072indian67486|pink|Gram|Unknown|25|n stoughtn station| +7920|AAAAAAAAOOOBAAAA|2001-10-27||Similar, foreign eyes should send over a states. Opportunities occur then necessary, iris|4.63|1.62|7011001|amalgnameless #1|11|accent|7|Home|441|oughteseese|N/A|floral04306781050856|yellow|Gram|Unknown|7|barablen station| +7921|AAAAAAAABPOBAAAA|1997-10-27||Big qualities might go members. Companies wear either good, foreign years. Organisms shall see with a records. However narrow proposals must not hire other relations; mainly unknown children would con|2.73|2.23|1001002|amalgamalg #2|1|dresses|1|Women|47|ationese|extra large|33smoke2852443563530|turquoise|Cup|Unknown|19|oughtablen station| +7922|AAAAAAAACPOBAAAA|1997-10-27|2000-10-26|Units used to assess; old consequences suppose old, joint others. Mice could not show meanwhile close officials. Faster old parties s|0.83|0.46|8001003|amalgnameless #3|1|athletic shoes|8|Sports|474|eseationese|N/A|2090sky6562991960578|orange|Oz|Unknown|4|ableablen station| +7923|AAAAAAAACPOBAAAA|2000-10-27||Very new women depend also i|0.11|0.03|8001003|importoexporti #2|2|infants|3|Children|137|ationpriought|medium|89152sandy3234122329|lawn|Case|Unknown|39|priablen station| +7924|AAAAAAAAEPOBAAAA|1997-10-27|1999-10-27|High c|3.79|1.78|2001001|amalgimporto #1|1|accessories|2|Men|983|prieingn st|small|898567433649993lace1|lime|Ton|Unknown|51|eseablen station| +7925|AAAAAAAAEPOBAAAA|1999-10-28|2001-10-26|High c|6.39|1.78|2001001|scholarunivamalg #6|1|fiction|9|Books|108|prieingn st|N/A|5551957283khaki91750|violet|Cup|Unknown|50|antiablen station| +7926|AAAAAAAAEPOBAAAA|2001-10-27||Soon typical offices must not let common pictures. Young, fair services need later aware sales. French pupils can regist|9.60|4.70|2002001|importoimporto #1|2|shirts|2|Men|108|prieingn st|small|5551957283khaki91750|maroon|Ton|Unknown|85|callyablen station| +7927|AAAAAAAAHPOBAAAA|1997-10-27||Participants break the|9.54|8.20|3004002|edu packexporti #2|4|school-uniforms|3|Children|502|ablebaranti|small|84649yellow285323785|beige|Case|Unknown|66|ationablen station| +7928|AAAAAAAAIPOBAAAA|1997-10-27|2000-10-26|Molecules bear early affairs. Plans obscure efficiently. Police can keep silently new countries. Democratic, head years change min|2.62|1.88|7003009|exportibrand #9|3|kids|7|Home|128|eingableought|N/A|snow6154136600902018|pale|Case|Unknown|13|eingablen station| +7929|AAAAAAAAIPOBAAAA|2000-10-27||Too yellow objections could discern long homes. Basic demonstrations may not equip slightly universal, other sheets.|5.19|1.88|7011010|amalgnameless #10|11|accent|7|Home|128|eingableought|N/A|snow6154136600902018|red|Dozen|Unknown|26|n stablen station| +7930|AAAAAAAAKPOBAAAA|1997-10-27|1999-10-27|Public visitors might think however private companies. Corporate, final damages need good, other fires. British guests tell as round schools; extraordinary, military y|7.65|4.05|9008011|namelessmaxi #11|8|romance|9|Books|517|ationoughtanti|N/A|7382288186051537sky6|powder|Tbl|Unknown|4|barprin station| +7931|AAAAAAAAKPOBAAAA|1999-10-28|2001-10-26|Public visitors might think however private companies. Corporate, final damages need good, other fires. British guests tell as round schools; extraordinary, military y|2.67|1.12|7012002|importonameless #2|12|paint|7|Home|517|ationoughtanti|N/A|7saddle8790067283462|sky|Pound|Unknown|50|oughtprin station| +7932|AAAAAAAAKPOBAAAA|2001-10-27||Phrases think too tall, dramatic schemes. Public, minor interests would not work now arab, divine properties. Later new gains must not ensure relations. Front pilots migh|2.40|1.12|2002001|importoimporto #1|2|shirts|2|Men|517|ationoughtanti|N/A|7saddle8790067283462|steel|Tsp|Unknown|3|ableprin station| +7933|AAAAAAAANPOBAAAA|1997-10-27||Columns sleep more organizations|1.38|0.95|10005012|scholarunivamalg #12|5|karoke|10|Electronics|223|priableable|N/A|57389metallic4175035|gainsboro|Gram|Unknown|17|priprin station| +7934|AAAAAAAAOPOBAAAA|1997-10-27|2000-10-26|Duties must look. Right due de|33.81|19.27|6012007|importobrand #7|12|costume|6|Jewelry|689|n steingcally|N/A|1sandy26016869863478|sky|Carton|Unknown|21|eseprin station| +7935|AAAAAAAAOPOBAAAA|2000-10-27||Duties must look. Right due de|0.13|0.04|8006006|corpnameless #6|6|football|8|Sports|695|n steingcally|N/A|66midnight8547328561|royal|Tsp|Unknown|57|antiprin station| +7936|AAAAAAAAAAPBAAAA|1997-10-27|1999-10-27|At |3.68|2.64|8002007|importonameless #7|2|baseball|8|Sports|479|n stationese|N/A|32firebrick673282450|sky|Bunch|Unknown|36|callyprin station| +7937|AAAAAAAAAAPBAAAA|1999-10-28|2001-10-26|At |2.36|1.06|8011008|amalgmaxi #8|11|archery|8|Sports|151|oughtantiought|N/A|32firebrick673282450|cyan|Unknown|Unknown|40|ationprin station| +7938|AAAAAAAAAAPBAAAA|2001-10-27||Continuous, final wages ought to appoint clothes; kids cool physic|71.76|1.06|8011009|amalgmaxi #9|11|archery|8|Sports|151|oughtantiought|N/A|21237639518383sandy2|salmon|Gram|Unknown|34|eingprin station| +7939|AAAAAAAADAPBAAAA|1997-10-27||Important branches live that previous missiles. Fresh, immediate parties must compete far tiny pictures. Then relative companies obtain home generally right problems. Also exis|2.47|1.53|2002002|importoimporto #2|2|shirts|2|Men|414|eseoughtese|medium|94beige2797041182809|plum|Ounce|Unknown|49|n stprin station| +7940|AAAAAAAAEAPBAAAA|1997-10-27|2000-10-26|Never christian pains may seem high resources. Never distinguished hundreds might refer now hard, di|6.15|5.41|4003001|exportiedu pack #1|3|kids|4|Shoes|499|n stn stese|medium|5866782756109778sky4|peach|Each|Unknown|32|baresen station| +7941|AAAAAAAAEAPBAAAA|2000-10-27||Major, bright |2.50|0.87|7014002|edu packnameless #2|14|glassware|7|Home|419|n stn stese|N/A|09303545375mint74216|moccasin|Gross|Unknown|15|oughtesen station| +7942|AAAAAAAAGAPBAAAA|1997-10-27|1999-10-27|Less|5.99|4.07|1001001|amalgamalg #1|1|dresses|1|Women|291|oughtn stable|medium|2318611858tomato5042|yellow|Cup|Unknown|10|ableesen station| +7943|AAAAAAAAGAPBAAAA|1999-10-28|2001-10-26|Less|5.72|2.05|1001001|exportiedu pack #2|1|kids|4|Shoes|205|antibarable|medium|15579steel4623830698|ghost|Tbl|Unknown|87|priesen station| +7944|AAAAAAAAGAPBAAAA|2001-10-27||Less|7.76|4.42|1001001|maxicorp #1|1|womens watch|6|Jewelry|205|antibarable|N/A|15579steel4623830698|puff|Box|Unknown|8|eseesen station| +7945|AAAAAAAAJAPBAAAA|1997-10-27||Circumstances pull yet. Little writers improve sure other criteria. Now particular rates believe. Cards|2.78|1.00|1001002|amalgamalg #2|1|dresses|1|Women|63|prically|petite|3993947slate82003018|saddle|Tsp|Unknown|6|antiesen station| +7946|AAAAAAAAKAPBAAAA|1997-10-27|2000-10-26|Away other hopes must leave central children. Parts like enough educational, major nations; wonderfully great terms can make finally light, overall systems.|0.32|0.27|1002001|importoamalg #1|2|fragrances|1|Women|649|n stesecally|large|54169indian394193872|plum|Dozen|Unknown|56|callyesen station| +7947|AAAAAAAAKAPBAAAA|2000-10-27||Separate daughters examine sometimes surely common councils. Items sit constantly other, silver colonies. Red, interested difficulties shou|3.24|0.27|8002008|importonameless #8|2|baseball|8|Sports|649|n stesecally|N/A|54169indian394193872|seashell|Tbl|Unknown|22|ationesen station| +7948|AAAAAAAAMAPBAAAA|1997-10-27|1999-10-27|Pe|5.19|2.64|4002001|importoedu pack #1|2|mens|4|Shoes|130|barpriought|large|4658041peru154783186|royal|Dram|Unknown|22|eingesen station| +7949|AAAAAAAAMAPBAAAA|1999-10-28|2001-10-26|Pe|0.59|2.64|4002001|amalgedu pack #2|1|womens|4|Shoes|130|barpriought|large|77277356purple336416|light|Dram|Unknown|36|n stesen station| +7950|AAAAAAAAMAPBAAAA|2001-10-27||Friends soar affairs. Ago changing sessions make much awful, whole units. More famous cultures would see ec|0.11|0.06|3002001|importoexporti #1|2|infants|3|Children|996|callyn stn st|medium|steel950064271491223|grey|Carton|Unknown|77|barantin station| +7951|AAAAAAAAPAPBAAAA|1997-10-27||Seconds should tolerate certainly large stairs. Large, foreign months shall pa|0.94|0.35|7007002|brandbrand #2|7|decor|7|Home|446|callyeseese|N/A|smoke729806926888665|purple|Case|Unknown|26|oughtantin station| +7952|AAAAAAAAABPBAAAA|1997-10-27|2000-10-26|By now new rules follow here short proceedings. Low winners ought to look still fast k|45.27|23.99|7013003|exportinameless #3|13|wallpaper|7|Home|115|antioughtought|N/A|832134137lace7875106|rose|Pallet|Unknown|89|ableantin station| +7953|AAAAAAAAABPBAAAA|2000-10-27||By now new rules follow here short proceedings. Low winners ought to look still fast k|4.04|23.99|1003002|exportiamalg #2|13|maternity|1|Women|115|antioughtought|petite|7538496624409rose264|wheat|Each|Unknown|74|priantin station| +7954|AAAAAAAACBPBAAAA|1997-10-27|1999-10-27|Clearly warm differences cannot depart strongly; even jewis|2.11|0.94|5003001|exportischolar #1|3|pop|5|Music|121|oughtableought|N/A|96022thistle29738452|honeydew|N/A|Unknown|51|eseantin station| +7955|AAAAAAAACBPBAAAA|1999-10-28|2001-10-26|Never black members used to come now services; industrial, implic|1.17|0.94|5003001|importonameless #6|12|paint|7|Home|121|oughtableought|N/A|96022thistle29738452|khaki|Bunch|Unknown|35|antiantin station| +7956|AAAAAAAACBPBAAAA|2001-10-27||Here extra worlds feel later useful events. Much other clothes direct by a markets. Different, british individuals wait quite individual households. Warm examples ought to handle around|0.34|0.11|6001005|amalgcorp #5|12|birdal|6|Jewelry|300|oughtableought|N/A|66248465006ghost5586|linen|Bunch|Unknown|36|callyantin station| +7957|AAAAAAAAFBPBAAAA|1997-10-27||Red, good players would not make considerable countries. Statutory, political gains used to install. Waiting elements might keep additional, ultimate effects. French years could not carry very |4.49|1.57|10007002|brandunivamalg #2|7|personal|10|Electronics|235|antipriable|N/A|116653peach666054065|honeydew|Cup|Unknown|29|ationantin station| +7958|AAAAAAAAGBPBAAAA|1997-10-27|2000-10-26|Changes cannot unload often new nations. Civil ambitions cannot look. As simple members ge|4.82|1.59|2002001|importoimporto #1|2|shirts|2|Men|334|esepripri|large|5013621030218powder9|slate|Ton|Unknown|1|eingantin station| +7959|AAAAAAAAGBPBAAAA|2000-10-27||Changes cannot unload often new nations. Civil ambitions cannot look. As simple members ge|5.91|3.60|2002001|exportischolar #2|2|pop|5|Music|981|esepripri|N/A|769916207423rose1222|lime|Ton|Unknown|69|n stantin station| +7960|AAAAAAAAIBPBAAAA|1997-10-27|1999-10-27|Reliable tests may suppose very almost significant drugs. Perhaps different aims result on the arms. More fun things dismantle just during a babies. Nearby numbers bend very final ne|3.34|1.77|2004001|edu packimporto #1|4|sports-apparel|2|Men|195|antin stought|medium|2105plum136909254259|saddle|Bundle|Unknown|14|barcallyn station| +7961|AAAAAAAAIBPBAAAA|1999-10-28|2001-10-26|External mothers used to know adult, sad boxes. So unlikely artists forget more than with a topics. Econo|8.52|5.62|2004001|exportinameless #2|4|basketball|8|Sports|195|antin stought|N/A|13500759royal7879169|coral|Cup|Unknown|20|oughtcallyn station| +7962|AAAAAAAAIBPBAAAA|2001-10-27||Here major years should come growing interests. Long bars wi|0.73|0.30|2004001|edu packedu pack #1|4|athletic|4|Shoes|195|antin stought|large|13500759royal7879169|powder|Tbl|Unknown|66|ablecallyn station| +7963|AAAAAAAALBPBAAAA|1997-10-27||Alternative rates shall provide marvellous, good scenes. Joint, big issues might rise extra as the structures. As clear things make able difficulties. Local exceptions collect quickl|3.59|1.86|3004002|edu packexporti #2|4|school-uniforms|3|Children|121|oughtableought|medium|822152086papaya29647|aquamarine|Unknown|Unknown|22|pricallyn station| +7964|AAAAAAAAMBPBAAAA|1997-10-27|2000-10-26|Animals will encounter other, young policies. Essential, useful changes li|8.64|5.09|7013005|exportinameless #5|13|wallpaper|7|Home|374|eseationpri|N/A|6512720thistle008379|purple|Box|Unknown|9|esecallyn station| +7965|AAAAAAAAMBPBAAAA|2000-10-27||Major phases secure tenants. Mo|68.42|56.78|7013005|importoimporto #2|13|shirts|2|Men|274|eseationable|small|6512720thistle008379|lawn|Oz|Unknown|29|anticallyn station| +7966|AAAAAAAAOBPBAAAA|1997-10-27|1999-10-27|Northwards necessary patients preserve; labour times could m|7.52|6.24|3004001|edu packexporti #1|4|school-uniforms|3|Children|293|prin stable|extra large|2642powder8789738494|thistle|Tbl|Unknown|14|callycallyn station| +7967|AAAAAAAAOBPBAAAA|1999-10-28|2001-10-26|Applicants might not hear to the di|4.92|6.24|3004001|amalgexporti #2|1|newborn|3|Children|293|prin stable|extra large|2642powder8789738494|pink|Case|Unknown|5|ationcallyn station| +7968|AAAAAAAAOBPBAAAA|2001-10-27||Unpleasant animals say there. Delighted countries would attend again instead musical companies. Professional decisions no too like interests. Different, mixed directors must see often pound|1.21|6.24|6008001|namelesscorp #1|8|mens watch|6|Jewelry|560|prin stable|N/A|2642powder8789738494|thistle|Pallet|Unknown|1|eingcallyn station| +7969|AAAAAAAABCPBAAAA|1997-10-27||Equal, human roads break hard topics. So political feet should fail away relative publications. Final, industrial areas may leave however by a police. Realistica|30.09|24.67|9015002|scholarunivamalg #2|15|fiction|9|Books|654|eseantically|N/A|31tomato869031123703|green|Pallet|Unknown|60|n stcallyn station| +7970|AAAAAAAACCPBAAAA|1997-10-27|2000-10-26|Much correct benefits might allow in the teachers. Official, external states can pur|9.06|6.88|9013003|exportiunivamalg #3|13|self-help|9|Books|14|eseought|N/A|02538firebrick067924|sky|Dozen|Unknown|71|barationn station| +7971|AAAAAAAACCPBAAAA|2000-10-27||Then great representatives wear after a months; real, full members might not run eithe|1.45|6.88|7013002|exportinameless #2|13|wallpaper|7|Home|14|eseought|N/A|822804678chocolate48|purple|Gross|Unknown|15|oughtationn station| +7972|AAAAAAAAECPBAAAA|1997-10-27|1999-10-27|Alone, following police will not expect mentally clothes. Dramatic, american weeks will not leap so central images. Costs remedy below black, easy letters. Parties ought to come more for a |17.66|14.48|9011005|amalgunivamalg #5|11|cooking|9|Books|226|callyableable|N/A|9400966162046lace234|olive|Tsp|Unknown|48|ableationn station| +7973|AAAAAAAAECPBAAAA|1999-10-28|2001-10-26|Deputies want well corporate characteristics. Successive, wide mil|5.87|4.46|9011005|exportiexporti #2|3|toddlers|3|Children|226|callyableable|large|9400966162046lace234|papaya|Dram|Unknown|48|priationn station| +7974|AAAAAAAAECPBAAAA|2001-10-27||Worse social states recognize with a studi|1.54|4.46|9011005|exportibrand #7|3|loose stones|6|Jewelry|323|callyableable|N/A|9400966162046lace234|puff|Bundle|Unknown|35|eseationn station| +7975|AAAAAAAAHCPBAAAA|1997-10-27||Causes could not encourage between a teachers. Too low sums would not get necessarily friends. Novel hands avoid indeed meetings. Recent, certain students may not know as years. Serious|3.47|2.15|10010003|univamalgamalg #3|10|memory|10|Electronics|248|eingeseable|N/A|5174845metallic51553|chiffon|Ton|Unknown|26|antiationn station| +7976|AAAAAAAAICPBAAAA|1997-10-27|2000-10-26|Well northern countries trust in charge of a others. Rural, black courts w|5.99|3.65|2003001|exportiimporto #1|3|pants|2|Men|330|barpripri|small|0775851763830steel09|papaya|Cup|Unknown|37|callyationn station| +7977|AAAAAAAAICPBAAAA|2000-10-27||Widespread things might remind then double volumes. Economic reasons could tell too new, foreign tenants. Open places m|5.15|3.65|2003001|importoscholar #2|3|country|5|Music|330|barpripri|N/A|3963lavender03939794|violet|Gram|Unknown|28|ationationn station| +7978|AAAAAAAAKCPBAAAA|1997-10-27|1999-10-27|Days used to define for example contents; men will point students. As flexible researchers should not employ like residents. Most environmental products|3.14|2.79|5001001|amalgscholar #1|1|rock|5|Music|190|barn stought|N/A|149980363azure267433|maroon|Oz|Unknown|28|eingationn station| +7979|AAAAAAAAKCPBAAAA|1999-10-28|2001-10-26|Across possible colleagues know even years. Much bloody families shall wor|6.27|2.79|6016004|corpbrand #4|1|consignment|6|Jewelry|163|pricallyought|N/A|2756lace388116626103|yellow|Carton|Unknown|29|n stationn station| +7980|AAAAAAAAKCPBAAAA|2001-10-27||Across possible colleagues know even years. Much bloody families shall wor|0.85|2.79|6016004|corpnameless #1|16|furniture|7|Home|163|pricallyought|N/A|964plum2780298546227|ghost|Each|Unknown|59|bareingn station| +7981|AAAAAAAANCPBAAAA|1997-10-27||Average, mean unions include. Cold ways shall work particularly from no rights. Already crucial agencies get very professional days. Perhaps huge methods rule financially awful arms. Strong vehicl|7.97|4.46|8013002|exportimaxi #2|13|sailing|8|Sports|758|eingantiation|N/A|25slate3772985578714|snow|Ounce|Unknown|59|oughteingn station| +7982|AAAAAAAAOCPBAAAA|1997-10-27|2000-10-26|Local, new |2.16|1.29|5004001|edu packscholar #1|4|classical|5|Music|517|ationoughtanti|N/A|5spring9581076423335|rose|Pallet|Unknown|23|ableeingn station| +7983|AAAAAAAAOCPBAAAA|2000-10-27||Local, new |78.51|69.08|3004002|edu packexporti #2|4|school-uniforms|3|Children|302|ationoughtanti|extra large|1022849407white02395|violet|Box|Unknown|14|prieingn station| +7984|AAAAAAAAADPBAAAA|1997-10-27|1999-10-27|Then specific supporters know years. Flowers start deliberately lucky dealers. Much english trades want. Errors vary from a years. Only absolute women might lower material centres. White, civil j|6.98|2.58|9012005|importounivamalg #5|12|home repair|9|Books|83|prieing|N/A|636650seashell971680|snow|Case|Unknown|24|eseeingn station| +7985|AAAAAAAAADPBAAAA|1999-10-28|2001-10-26|Then specific supporters know years. Flowers start deliberately lucky dealers. Much english trades want. Errors vary from a years. Only absolute women might lower material centres. White, civil j|0.48|0.34|7012006|importonameless #6|12|paint|7|Home|202|prieing|N/A|1drab012295278412085|tan|Dram|Unknown|27|antieingn station| +7986|AAAAAAAAADPBAAAA|2001-10-27||Members will not uphold still throughout a forms. Fun relations lift. Available, d|1.69|0.57|7012006|importoedu pack #1|12|mens|4|Shoes|202|ablebarable|large|6356hot4120362828289|slate|Unknown|Unknown|44|callyeingn station| +7987|AAAAAAAADDPBAAAA|1997-10-27||Permanently good days progress really alternative plans. Small, sexual techniques ret|9.85|7.09|8004008|edu packnameless #8|4|camping|8|Sports|287|ationeingable|N/A|05614seashell9632366|spring|Oz|Unknown|38|ationeingn station| +7988|AAAAAAAAEDPBAAAA|1997-10-27|2000-10-26|Days shall reassure ahead. Signs come in a parties; fees e|0.89|0.76|6002005|importocorp #5|2|diamonds|6|Jewelry|144|eseeseought|N/A|1tan7572064702996523|sienna|Ton|Unknown|58|eingeingn station| +7989|AAAAAAAAEDPBAAAA|2000-10-27||Forth true experts w|8.22|5.34|3004002|edu packexporti #2|2|school-uniforms|3|Children|144|eseeseought|large|1tan7572064702996523|papaya|Oz|Unknown|90|n steingn station| +7990|AAAAAAAAGDPBAAAA|1997-10-27|1999-10-27|Certain years form ready, single cases. Sometimes certain members must afford doubtless records. National unions play highly; now good re|1.64|1.16|3003001|exportiexporti #1|3|toddlers|3|Children|252|ableantiable|small|20511217073salmon667|royal|Lb|Unknown|65|barn stn station| +7991|AAAAAAAAGDPBAAAA|1999-10-28|2001-10-26|Ultimately economic officers must not think quickly about unpleasant|4.53|1.16|9011012|amalgunivamalg #12|3|cooking|9|Books|252|ableantiable|N/A|20511217073salmon667|spring|Case|Unknown|28|oughtn stn station| +7992|AAAAAAAAGDPBAAAA|2001-10-27||Ultimately economic officers must not think quickly about unpleasant|4.17|1.16|4001001|amalgedu pack #1|3|womens|4|Shoes|252|ableantiable|medium|20511217073salmon667|olive|Bunch|Unknown|12|ablen stn station| +7993|AAAAAAAAJDPBAAAA|1997-10-27||Vehicles can phone in the resources. Past walls take considerably as mere prices; blue, wrong women would not offer in a nations. Public territories come mor|2.95|1.26|6011002|amalgbrand #2|11|semi-precious|6|Jewelry|584|eseeinganti|N/A|9249489625white64026|grey|Unknown|Unknown|10|prin stn station| +7994|AAAAAAAAKDPBAAAA|1997-10-27|2000-10-26|Stations work enough fellow difficulties. Spanish conditions discuss also things. Levels shall n|3.15|2.74|2002001|importoimporto #1|2|shirts|2|Men|157|ationantiought|large|0puff149164906054477|blanched|Case|Unknown|28|esen stn station| +7995|AAAAAAAAKDPBAAAA|2000-10-27||Stations work enough fellow difficulties. Spanish conditions discuss also things. Levels shall n|8.03|2.74|2002001|namelessnameless #6|2|outdoor|8|Sports|213|prioughtable|N/A|11626978582sienna738|cornflower|Each|Unknown|11|antin stn station| +7996|AAAAAAAAMDPBAAAA|1997-10-27|1999-10-27|Also personal or|0.14|0.08|7005007|scholarbrand #7|5|blinds/shades|7|Home|496|callyn stese|N/A|378cornflower4124207|sienna|Pallet|Unknown|40|callyn stn station| +7997|AAAAAAAAMDPBAAAA|1999-10-28|2001-10-26|Also personal or|6.88|0.08|7005007|importoamalg #2|5|fragrances|1|Women|496|callyn stese|large|0732057767navy671754|olive|Carton|Unknown|2|ationn stn station| +7998|AAAAAAAAMDPBAAAA|2001-10-27||Major, general funds explain there; scientific, social years help again independent, dead stones; ol|2.89|0.08|7005007|edu packamalg #1|5|swimwear|1|Women|473|callyn stese|medium|3677358sandy38960999|papaya|Dram|Unknown|55|eingn stn station| +7999|AAAAAAAAPDPBAAAA|1997-10-27||All future weeks would mean then main car|0.42|0.33|3004002|edu packexporti #2|4|school-uniforms|3|Children|266|callycallyable|petite|41846790antique32048|deep|Gram|Unknown|38|n stn stn station| +8000|AAAAAAAAAEPBAAAA|1997-10-27|2000-10-26|Long key contracts must make. Other others look just highly open things; stars would n|7.49|5.39|2002001|importoimporto #1|2|shirts|2|Men|103|pribarought|extra large|07567navy34526569752|beige|Gross|Unknown|29|barbarbareing| +8001|AAAAAAAAAEPBAAAA|2000-10-27||Rises should cover never perhaps international sides|4.75|3.65|9014010|edu packunivamalg #10|2|sports|9|Books|727|ationableation|N/A|158842928018olive354|pink|Carton|Unknown|76|oughtbarbareing| +8002|AAAAAAAACEPBAAAA|1997-10-27|1999-10-27|Long, married artists would see negative feelings. Emot|1.73|0.51|9002011|importomaxi #11|2|business|9|Books|317|ationoughtpri|N/A|52olive0590749648922|peach|Oz|Unknown|6|ablebarbareing| +8003|AAAAAAAACEPBAAAA|1999-10-28|2001-10-26|Very national transactions can breathe difficult, new men. British hundreds say fresh, local problems; small numbers ought to make here on a funds; hig|9.22|6.63|7011004|amalgnameless #4|11|accent|7|Home|370|barationpri|N/A|52olive0590749648922|pale|Carton|Unknown|3|pribarbareing| +8004|AAAAAAAACEPBAAAA|2001-10-27||Others could ensure available supporters. At random social changes win books. States support at a studies. Afraid, orthodox changes advise average points. Bodies could deliver far in short small i|0.76|0.57|7011004|importoedu pack #1|2|mens|4|Shoes|98|eingn st|small|52olive0590749648922|deep|Bundle|Unknown|19|esebarbareing| +8005|AAAAAAAAFEPBAAAA|1997-10-27||Golden, african protests rest there key paintings. Sole, national tears will recover to a companies. Good hopes exhibit quite troops. Often different attacks matter|4.24|2.28|4003002|exportiedu pack #2|3|kids|4|Shoes|51|oughtanti|medium|748959415821sandy659|medium|Pallet|Unknown|33|antibarbareing| +8006|AAAAAAAAGEPBAAAA|1997-10-27|2000-10-26|Financial months turn finally by a women. Practical, unfair |0.88|0.33|6008007|namelesscorp #7|8|mens watch|6|Jewelry|597|ationn stanti|N/A|5388419444219blush87|wheat|Tbl|Unknown|4|callybarbareing| +8007|AAAAAAAAGEPBAAAA|2000-10-27||Financial months turn finally by a women. Practical, unfair |6.10|4.27|7016008|corpnameless #8|8|furniture|7|Home|397|ationn stpri|N/A|6793190017navajo9991|honeydew|Dram|Unknown|13|ationbarbareing| +8008|AAAAAAAAIEPBAAAA|1997-10-27|1999-10-27|Things compromise la|60.74|44.34|7013009|exportinameless #9|13|wallpaper|7|Home|17|ationought|N/A|0461531salmon6425433|lawn|Each|Unknown|4|eingbarbareing| +8009|AAAAAAAAIEPBAAAA|1999-10-28|2001-10-26|Things compromise la|0.73|44.34|8005010|scholarnameless #10|13|fitness|8|Sports|206|ationought|N/A|911103234seashell170|plum|Unknown|Unknown|9|n stbarbareing| +8010|AAAAAAAAIEPBAAAA|2001-10-27||Things compromise la|75.77|40.91|1002001|importoamalg #1|2|fragrances|1|Women|203|ationought|medium|911103234seashell170|pink|Carton|Unknown|10|baroughtbareing| +8011|AAAAAAAALEPBAAAA|1997-10-27||Especially correct courts en|2.92|2.54|9016008|corpunivamalg #8|16|mystery|9|Books|340|baresepri|N/A|679007white988512271|white|Bundle|Unknown|2|oughtoughtbareing| +8012|AAAAAAAAMEPBAAAA|1997-10-27|2000-10-26|Good days trace able, immediate arrangements. Great countries agree also times. Different regions leave then as well very parents. Heavy, important months make more present lists. Police pass new |5.19|2.54|10003006|exportiunivamalg #6|3|dvd/vcr players|10|Electronics|106|callybarought|N/A|11643263salmon141052|linen|Tsp|Unknown|52|ableoughtbareing| +8013|AAAAAAAAMEPBAAAA|2000-10-27||Good days trace able, immediate arrangements. Great countries agree also times. Different regions leave then as well very parents. Heavy, important months make more present lists. Police pass new |6.55|2.54|10003006|amalgamalg #2|1|dresses|1|Women|106|callybarought|medium|5wheat35834195867145|antique|Each|Unknown|18|prioughtbareing| +8014|AAAAAAAAOEPBAAAA|1997-10-27|1999-10-27|Large, local requirements might not tr|6.57|3.87|10001008|amalgunivamalg #8|1|cameras|10|Electronics|415|antioughtese|N/A|8500496093230yellow4|thistle|Tsp|Unknown|79|eseoughtbareing| +8015|AAAAAAAAOEPBAAAA|1999-10-28|2001-10-26|Large, local requirements might not tr|1.50|3.87|2004002|edu packimporto #2|1|sports-apparel|2|Men|462|antioughtese|medium|2201851sandy61175579|hot|Dozen|Unknown|3|antioughtbareing| +8016|AAAAAAAAOEPBAAAA|2001-10-27||Large, local requirements might not tr|48.18|29.87|10004010|edu packunivamalg #10|4|audio|10|Electronics|566|antioughtese|N/A|9858midnight94552034|sandy|Tsp|Unknown|29|callyoughtbareing| +8017|AAAAAAAABFPBAAAA|1997-10-27||Global, financial lives make incentives. Patient results could reject more than now basic machines. Old roa|0.85|0.29|2002002|importoimporto #2|2|shirts|2|Men|86|callyeing|large|28203276572191pink33|papaya|Case|Unknown|62|ationoughtbareing| +8018|AAAAAAAACFPBAAAA|1997-10-27|2000-10-26|Babies ought to take yesterday. Females will pretend often neigh|9.78|7.23|7003009|exportibrand #9|3|kids|7|Home|502|ablebaranti|N/A|56849224powder265690|white|Unknown|Unknown|69|eingoughtbareing| +8019|AAAAAAAACFPBAAAA|2000-10-27||Independent stages run european, rich plans. Special, left agents develop children. Bl|4.15|1.95|7012010|importonameless #10|3|paint|7|Home|123|ablebaranti|N/A|56849224powder265690|gainsboro|Lb|Unknown|28|n stoughtbareing| +8020|AAAAAAAAEFPBAAAA|1997-10-27|1999-10-27|Particularly internal times could not achieve as yet indeed english phases. Good windows can become technically personal firms. Details need well for a miles. N|1.16|0.58|9010005|univunivamalg #5|10|travel|9|Books|458|eingantiese|N/A|24480036royal6766817|peach|Box|Unknown|31|barablebareing| +8021|AAAAAAAAEFPBAAAA|1999-10-28|2001-10-26|Meals drag premises. Recent, great girls take either. Applications may attract trends. Reliable legs would go trees. Small attacks used to remember t|4.77|0.58|9010005|exportischolar #2|10|pop|5|Music|458|eingantiese|N/A|24480036royal6766817|yellow|Box|Unknown|6|oughtablebareing| +8022|AAAAAAAAEFPBAAAA|2001-10-27||General, special visits can know|2.47|1.16|2004001|edu packimporto #1|4|sports-apparel|2|Men|323|priablepri|large|8127370400237sienna4|slate|Bundle|Unknown|79|ableablebareing| +8023|AAAAAAAAHFPBAAAA|1997-10-27||Straightforward, technical jews shall not happen well employ|2.31|1.03|10010017|univamalgamalg #17|10|memory|10|Electronics|171|oughtationought|N/A|152spring95190565968|peru|Pound|Unknown|11|priablebareing| +8024|AAAAAAAAIFPBAAAA|1997-10-27|2000-10-26|Entirely running managers can regard in a changes; sing|6.81|3.88|10014001|edu packamalgamalg #1|14|automotive|10|Electronics|621|oughtablecally|N/A|23forest255358398354|seashell|Each|Unknown|68|eseablebareing| +8025|AAAAAAAAIFPBAAAA|2000-10-27||Entirely running managers can regard in a changes; sing|0.31|3.88|4001002|amalgedu pack #2|1|womens|4|Shoes|952|oughtablecally|small|136857sienna17584912|white|Pound|Unknown|91|antiablebareing| +8026|AAAAAAAAKFPBAAAA|1997-10-27|1999-10-27|Poor, weak compa|1.11|0.97|2002001|importoimporto #1|2|shirts|2|Men|101|oughtbarought|small|7477553261orange5510|floral|Carton|Unknown|68|callyablebareing| +8027|AAAAAAAAKFPBAAAA|1999-10-28|2001-10-26|Poor, weak compa|1.10|0.97|2002001|importoamalg #2|2|fragrances|1|Women|101|oughtbarought|small|7477553261orange5510|puff|Tbl|Unknown|27|ationablebareing| +8028|AAAAAAAAKFPBAAAA|2001-10-27||Immense, unable women will set persons. Ultimately late ideas happen here sharp, political politicians. Small wheels shall use american, physical varieties. New functions should en|6.09|0.97|2002001|scholarmaxi #9|15|fishing|8|Sports|101|oughtbarought|N/A|steel209538261836638|violet|Carton|Unknown|41|eingablebareing| +8029|AAAAAAAANFPBAAAA|1997-10-27||Systems read previously|9.20|7.45|4004002|edu packedu pack #2|4|athletic|4|Shoes|763|pricallyation|medium|3241957415174spring1|slate|Pallet|Unknown|65|n stablebareing| +8030|AAAAAAAAOFPBAAAA|1997-10-27|2000-10-26|Other, bad features may mean. Pol|1.00|0.38|2003001|exportiimporto #1|3|pants|2|Men|190|barn stought|extra large|orange02625037601353|slate|N/A|Unknown|6|barpribareing| +8031|AAAAAAAAOFPBAAAA|2000-10-27||However negative shares achieve instead afraid clouds. Great businesses make absolute officials; growing views shall not follow ancient, suitable procedures; local others ma|1.69|1.04|7011002|amalgnameless #2|11|accent|7|Home|678|barn stought|N/A|orange02625037601353|misty|Bundle|Unknown|4|oughtpribareing| +8032|AAAAAAAAAGPBAAAA|1997-10-27|1999-10-27|Married types take. Excellent|3.32|1.39|4003001|exportiedu pack #1|3|kids|4|Shoes|240|bareseable|extra large|6902103258026tan2888|peach|Pallet|Unknown|3|ablepribareing| +8033|AAAAAAAAAGPBAAAA|1999-10-28|2001-10-26|Rights sense once gastric streets. Popular, normal animals resist also now certain settings. Easy restaurants perc|7.74|4.95|4003001|edu packimporto #2|4|sports-apparel|2|Men|240|bareseable|large|6902103258026tan2888|purple|Carton|Unknown|20|pripribareing| +8034|AAAAAAAAAGPBAAAA|2001-10-27||However other tools make. Moreover young countries will buy recent interests. Local authori|3.79|3.29|2002001|importoimporto #1|4|shirts|2|Men|240|bareseable|medium|6turquoise7056647337|spring|Gram|Unknown|4|esepribareing| +8035|AAAAAAAADGPBAAAA|1997-10-27||Little, multiple systems will not step already objects. Other, foreign careers push. Super, certain accidents matter. Direct, local countries can happen for a operations. O|6.03|4.76|1002002|importoamalg #2|2|fragrances|1|Women|448|eingeseese|extra large|545163979smoke917968|moccasin|Tbl|Unknown|19|antipribareing| +8036|AAAAAAAAEGPBAAAA|1997-10-27|2000-10-26|Only right publications exist very in a tables; difficult steps care currently. Generally basic successes will find commonly. Companies|0.73|0.34|3004001|edu packexporti #1|4|school-uniforms|3|Children|500|barbaranti|economy|94777pink14534627225|wheat|Tbl|Unknown|79|callypribareing| +8037|AAAAAAAAEGPBAAAA|2000-10-27||Areas determine shares; physical books may li|2.21|0.34|3004001|importocorp #6|2|diamonds|6|Jewelry|350|barbaranti|N/A|94777pink14534627225|lemon|Pallet|Unknown|95|ationpribareing| +8038|AAAAAAAAGGPBAAAA|1997-10-27|1999-10-27|Common, standard conditions can|2.68|1.68|1002001|importoamalg #1|2|fragrances|1|Women|455|antiantiese|large|324navy7104602772130|tomato|Each|Unknown|28|eingpribareing| +8039|AAAAAAAAGGPBAAAA|1999-10-28|2001-10-26|Contexts can get as small representations; computers say. Very distinctive teachers cannot make national children|3.04|1.68|9005012|scholarmaxi #12|2|history|9|Books|494|antiantiese|N/A|8white62551232164544|hot|Unknown|Unknown|3|n stpribareing| +8040|AAAAAAAAGGPBAAAA|2001-10-27||Often nervous questions would not exhibit equally to a areas. No evident devices must provide considera|4.59|1.74|7011001|amalgnameless #1|2|accent|7|Home|494|esen stese|N/A|8white62551232164544|tan|Carton|Unknown|4|baresebareing| +8041|AAAAAAAAJGPBAAAA|1997-10-27||Able, main parties think really. Resources arrive only independent, old representations. Small, double advantages |2.38|1.83|8001002|amalgnameless #2|1|athletic shoes|8|Sports|363|pricallypri|N/A|997305543puff5507880|ivory|Pallet|Unknown|34|oughtesebareing| +8042|AAAAAAAAKGPBAAAA|1997-10-27|2000-10-26|Again appropriate months could give young activities. Particularly alternative arms could not believe black, growing patterns. Mathematical, public candidates ought to see even only cheap ser|51.46|19.55|7010003|univnameless #3|10|flatware|7|Home|198|eingn stought|N/A|7072552640puff564269|seashell|Bunch|Unknown|1|ableesebareing| +8043|AAAAAAAAKGPBAAAA|2000-10-27||Again appropriate months could give young activities. Particularly alternative arms could not believe black, growing patterns. Mathematical, public candidates ought to see even only cheap ser|9.56|19.55|5004002|edu packscholar #2|10|classical|5|Music|167|eingn stought|N/A|7543orange0164690461|pale|Gross|Unknown|47|priesebareing| +8044|AAAAAAAAMGPBAAAA|1997-10-27|1999-10-27|Descriptions ought to defeat as prepared applicants. |2.21|1.85|5001001|amalgscholar #1|1|rock|5|Music|210|baroughtable|N/A|677760178025violet95|slate|Oz|Unknown|38|eseesebareing| +8045|AAAAAAAAMGPBAAAA|1999-10-28|2001-10-26|Descriptions ought to defeat as prepared applicants. |1.69|0.99|5001001|corpbrand #6|1|rugs|7|Home|210|baroughtable|N/A|62spring894178922778|drab|Unknown|Unknown|19|antiesebareing| +8046|AAAAAAAAMGPBAAAA|2001-10-27||Descriptions ought to defeat as prepared applicants. |0.60|0.41|2002001|importoimporto #1|1|shirts|2|Men|210|baroughtable|medium|7578282851sandy09782|linen|Dram|Unknown|23|callyesebareing| +8047|AAAAAAAAPGPBAAAA|1997-10-27||Women could head then even old tenants. Almost causal points can watch differently mental, previous cases. |2.25|1.53|9014008|edu packunivamalg #8|14|sports|9|Books|606|callybarcally|N/A|864122916gainsboro25|royal|Pallet|Unknown|8|ationesebareing| +8048|AAAAAAAAAHPBAAAA|1997-10-27|2000-10-26|Windows retain true records. Hot tables should look necessa|3.71|2.74|1001001|amalgamalg #1|1|dresses|1|Women|198|eingn stought|large|8yellow2237441657079|peru|Tbl|Unknown|5|eingesebareing| +8049|AAAAAAAAAHPBAAAA|2000-10-27||Windows retain true records. Hot tables should look necessa|6.99|2.74|1001001|edu packcorp #2|1|bracelets|6|Jewelry|691|oughtn stcally|N/A|8yellow2237441657079|puff|Ounce|Unknown|27|n stesebareing| +8050|AAAAAAAACHPBAAAA|1997-10-27|1999-10-27|Strong, necessary deals ought to ask decisions. Current articles get wide lett|5.58|3.68|4001001|amalgedu pack #1|1|womens|4|Shoes|331|oughtpripri|small|4652476796113dark548|lime|Bundle|Unknown|9|barantibareing| +8051|AAAAAAAACHPBAAAA|1999-10-28|2001-10-26|Strong, necessary deals ought to ask decisions. Current articles get wide lett|9.64|3.68|1003002|exportiamalg #2|1|maternity|1|Women|25|antiable|medium|4652476796113dark548|ghost|Pallet|Unknown|45|oughtantibareing| +8052|AAAAAAAACHPBAAAA|2001-10-27||Monetary, local others can sound also sadly living issues; new, long systems spend. Musical, differen|4.49|1.34|1003002|maxicorp #5|1|womens watch|6|Jewelry|178|eingationought|N/A|4652476796113dark548|thistle|Dozen|Unknown|19|ableantibareing| +8053|AAAAAAAAFHPBAAAA|1997-10-27||Good, white rivers leave only. Just chosen tiles enter v|3.37|2.02|8013004|exportimaxi #4|13|sailing|8|Sports|89|n steing|N/A|68207thistle77379304|violet|Pound|Unknown|37|priantibareing| +8054|AAAAAAAAGHPBAAAA|1997-10-27|2000-10-26|Rights shall let late as a proposals. Large, indirect police can join in an expectations. Real, attractive courts sound as both early candidates. Considerably following men approve so-called, contempo|1.85|1.40|8005005|scholarnameless #5|5|fitness|8|Sports|937|ationprin st|N/A|258506mint3974534885|salmon|Lb|Unknown|53|eseantibareing| +8055|AAAAAAAAGHPBAAAA|2000-10-27||Black, recent mothers should not allow very consistent cells. Labour, simple arms could not lay in general only main reports. Sure, chief trends look double parents. Small, local metres sha|4.97|1.40|6015008|scholarbrand #8|15|custom|6|Jewelry|937|ationprin st|N/A|258506mint3974534885|drab|Bunch|Unknown|45|antiantibareing| +8056|AAAAAAAAIHPBAAAA|1997-10-27|1999-10-27|Principal ambitions encourage |4.71|1.88|6005001|scholarcorp #1|5|earings|6|Jewelry|153|priantiought|N/A|9586701508347tomato4|rose|Pound|Unknown|27|callyantibareing| +8057|AAAAAAAAIHPBAAAA|1999-10-28|2001-10-26|Independent, f|7.23|1.88|6005001|exportiunivamalg #17|3|dvd/vcr players|10|Electronics|153|priantiought|N/A|490817887lavender316|ghost|Lb|Unknown|97|ationantibareing| +8058|AAAAAAAAIHPBAAAA|2001-10-27||Social, human workers can know existing fields. Systems conceal on a risks; now cheap ideas can say particularly far live|7.21|1.88|6005001|importoedu pack #1|3|mens|4|Shoes|153|priantiought|large|490817887lavender316|thistle|Bunch|Unknown|9|eingantibareing| +8059|AAAAAAAALHPBAAAA|1997-10-27||Convincing, perfect trees like powerfully. Months|8.94|6.34|1001002|amalgamalg #2|1|dresses|1|Women|178|eingationought|small|637754moccasin301704|brown|Ton|Unknown|53|n stantibareing| +8060|AAAAAAAAMHPBAAAA|1997-10-27|2000-10-26|Positive companies guide ordinary, important exchanges. Reasonable personnel seem above f|2.29|0.77|10016003|corpamalgamalg #3|16|wireless|10|Electronics|794|esen station|N/A|5441707605slate45551|blue|Unknown|Unknown|84|barcallybareing| +8061|AAAAAAAAMHPBAAAA|2000-10-27||Competitive games used to delay from a representations. Low, regional dogs leave short social lips. Currently jewish letters |2.49|0.97|10016003|univmaxi #2|10|pools|8|Sports|794|esen station|N/A|9178975349indian6927|linen|Cup|Unknown|14|oughtcallybareing| +8062|AAAAAAAAOHPBAAAA|1997-10-27|1999-10-27|Real lips cannot jail magistrates. Thoughts stand gravely with an ideas. American patterns will not make later. Primary years used to see persons; directors let |2.67|1.92|6015007|scholarbrand #7|15|custom|6|Jewelry|141|oughteseought|N/A|291slate764169315307|turquoise|Box|Unknown|2|ablecallybareing| +8063|AAAAAAAAOHPBAAAA|1999-10-28|2001-10-26|International, small communities shall supply also modern rules; high, different forces handle then at the players; fine allowances could adopt distinctive, easy costs. Short, new employers can agree|3.36|1.92|6015007|corpunivamalg #6|6|musical|10|Electronics|311|oughtoughtpri|N/A|6667105277orange5443|thistle|Pound|Unknown|44|pricallybareing| +8064|AAAAAAAAOHPBAAAA|2001-10-27||International, small communities shall supply also modern rules; high, different forces handle then at the players; fine allowances could adopt distinctive, easy costs. Short, new employers can agree|0.57|0.46|6015007|amalgimporto #1|1|accessories|2|Men|32|ablepri|medium|6667105277orange5443|indian|Oz|Unknown|7|esecallybareing| +8065|AAAAAAAABIPBAAAA|1997-10-27||Ultimate, other objects might not install good |2.57|2.21|9002002|importomaxi #2|2|business|9|Books|583|prieinganti|N/A|724725puff5919674109|thistle|Each|Unknown|60|anticallybareing| +8066|AAAAAAAACIPBAAAA|1997-10-27|2000-10-26|Working, single students join as still close relationships; clear states might stand of course|1.88|1.46|6010003|univbrand #3|10|jewelry boxes|6|Jewelry|652|ableantically|N/A|8151olive96119765340|pink|Unknown|Unknown|62|callycallybareing| +8067|AAAAAAAACIPBAAAA|2000-10-27||Practices would lead. Public meanings refer more used examples. Other, dangerous pupils would start now tables. Details could m|7.14|6.14|10012010|importoamalgamalg #10|10|monitors|10|Electronics|652|ableantically|N/A|026ghost370767648349|snow|Pound|Unknown|11|ationcallybareing| +8068|AAAAAAAAEIPBAAAA|1997-10-27|1999-10-27|Again sad units control. Ashore respectable earnings proceed always things; wages summon physical rates. Easier exciting books continue too men. As global nurses will signify professio|2.40|1.08|5001001|amalgscholar #1|1|rock|5|Music|332|ablepripri|N/A|981076sienna39721042|plum|Case|Unknown|4|eingcallybareing| +8069|AAAAAAAAEIPBAAAA|1999-10-28|2001-10-26|Again sad units control. Ashore respectable earnings proceed always things; wages summon physical rates. Easier exciting books continue too men. As global nurses will signify professio|1.02|1.08|5001001|importoamalgamalg #12|1|monitors|10|Electronics|332|ablepripri|N/A|35950938snow49163679|burnished|Carton|Unknown|29|n stcallybareing| +8070|AAAAAAAAEIPBAAAA|2001-10-27||Lesser, northern machines tackle here social patients. Titles serve of course for a children. Famous activities ought to s|1.22|1.08|5001001|amalgscholar #1|1|rock|5|Music|332|ablepripri|N/A|35950938snow49163679|puff|Carton|Unknown|3|barationbareing| +8071|AAAAAAAAHIPBAAAA|1997-10-27||Yet small groups feature earnings. Young engines would try t|0.75|0.56|9009008|maximaxi #8|9|science|9|Books|165|anticallyought|N/A|7seashell63007369397|puff|Pallet|Unknown|61|oughtationbareing| +8072|AAAAAAAAIIPBAAAA|1997-10-27|2000-10-26|True changes shall show mostly exceptional women. British, immediate things shall go regularly new ambitions; equal, new walls should not control then to a contracts. Shortly m|86.13|36.17|10008015|namelessunivamalg #15|8|scanners|10|Electronics|576|callyationanti|N/A|034goldenrod40934587|olive|Ton|Unknown|18|ableationbareing| +8073|AAAAAAAAIIPBAAAA|2000-10-27||True changes shall show mostly exceptional women. British, immediate things shall go regularly new ambitions; equal, new walls should not control then to a contracts. Shortly m|88.06|29.94|10004016|edu packunivamalg #16|8|audio|10|Electronics|576|callyationanti|N/A|034goldenrod40934587|ivory|Tbl|Unknown|20|priationbareing| +8074|AAAAAAAAKIPBAAAA|1997-10-27|1999-10-27|Special shops cut there; right, white vessels might avoid etc. Issues should not indicate; social, thick unions say sometimes important, other arrang|0.77|0.53|2003001|exportiimporto #1|3|pants|2|Men|33|pripri|large|582128531plum4160005|misty|Gram|Unknown|34|eseationbareing| +8075|AAAAAAAAKIPBAAAA|1999-10-28|2001-10-26|Special shops cut there; right, white vessels might avoid etc. Issues should not indicate; social, thick unions say sometimes important, other arrang|4.64|1.99|7010006|univnameless #6|3|flatware|7|Home|33|pripri|N/A|5655metallic55664833|peach|Gross|Unknown|10|antiationbareing| +8076|AAAAAAAAKIPBAAAA|2001-10-27||Minimal, individual women report little distinguished, small|9.59|5.08|7010006|exportiexporti #1|3|toddlers|3|Children|33|pripri|medium|papaya36107594359833|goldenrod|Dozen|Unknown|31|callyationbareing| +8077|AAAAAAAANIPBAAAA|1997-10-27||Ways find always in a jobs|2.47|1.82|1001002|amalgamalg #2|1|dresses|1|Women|561|oughtcallyanti|large|1thistle842484736716|steel|Cup|Unknown|12|ationationbareing| +8078|AAAAAAAAOIPBAAAA|1997-10-27|2000-10-26|Studies shall not advise new lives. Lines like blind, other regions. Future, sorry clients might not |1.87|1.43|6013007|exportibrand #7|13|loose stones|6|Jewelry|170|barationought|N/A|22salmon654411338450|pink|Tbl|Unknown|2|eingationbareing| +8079|AAAAAAAAOIPBAAAA|2000-10-27||Artists may get new sales. English experiments may provide human, manufacturing individuals. Details could take regulati|4.99|1.43|3004002|edu packexporti #2|4|school-uniforms|3|Children|170|barationought|extra large|336794gainsboro78976|sky|Bunch|Unknown|43|n stationbareing| +8080|AAAAAAAAAJPBAAAA|1997-10-27|1999-10-27|Social plans achieve secondary, bad bag|0.72|0.53|2003001|exportiimporto #1|3|pants|2|Men|357|ationantipri|extra large|2513846papaya1276331|wheat|Ounce|Unknown|33|bareingbareing| +8081|AAAAAAAAAJPBAAAA|1999-10-28|2001-10-26|Social plans achieve secondary, bad bag|2.76|1.26|2003001|amalgimporto #2|3|accessories|2|Men|357|ationantipri|small|2513846papaya1276331|red|Dram|Unknown|4|oughteingbareing| +8082|AAAAAAAAAJPBAAAA|2001-10-27||Inadequate aspirations would give now beautiful authorities. Products go enough other, alone years. Here able suggestions give collective|3.76|2.93|8004003|edu packnameless #3|4|camping|8|Sports|357|ationantipri|N/A|2513846papaya1276331|dim|Box|Unknown|20|ableeingbareing| +8083|AAAAAAAADJPBAAAA|1997-10-27||Flat, important teachers know highly foreign, great authors. Names might set as to a situations. Clean policies can buy orders. Emotional, individual pr|2.92|1.66|4001002|amalgedu pack #2|1|womens|4|Shoes|109|n stbarought|medium|286000red21973498336|snow|Gross|Unknown|28|prieingbareing| +8084|AAAAAAAAEJPBAAAA|1997-10-27|2000-10-26|Libraries sell great friends. Characteristics will write too. Fully right homes take very neither immediate targets. Children maintain more feet. Too australian terms claim only, broken ga|6.93|3.04|2002001|importoimporto #1|2|shirts|2|Men|300|barbarpri|N/A|51789496138351coral3|ghost|Dozen|Unknown|13|eseeingbareing| +8085|AAAAAAAAEJPBAAAA|2000-10-27||Specific, continuo|7.21|3.04|2002001|corpcorp #6|6|rings|6|Jewelry|300|barbarpri|N/A|1350778457ghost93051|rosy|Gram|Unknown|54|antieingbareing| +8086|AAAAAAAAGJPBAAAA|1997-10-27|1999-10-27|Significant films use swiftly; royal members should attract yet skills. Right stones c|0.30|0.12|2002001|importoimporto #1|2|shirts|2|Men|53|prianti|extra large|550441551337violet42|purple|Carton|Unknown|20|callyeingbareing| +8087|AAAAAAAAGJPBAAAA|1999-10-28|2001-10-26|Afterwards left paintings may not understand quiet accounts. Others will run far. Possible, b|3.27|0.12|2002001|amalgamalg #2|2|dresses|1|Women|363|prianti|petite|483701saddle30948062|violet|Each|Unknown|6|ationeingbareing| +8088|AAAAAAAAGJPBAAAA|2001-10-27||Afterwards left paintings may not understand quiet accounts. Others will run far. Possible, b|41.93|14.25|8005009|scholarnameless #9|5|fitness|8|Sports|363|prianti|N/A|350thistle1439606856|light|Bunch|Unknown|48|eingeingbareing| +8089|AAAAAAAAJJPBAAAA|1997-10-27||Cheap, british members cannot wait totally given procedures. Years could make aside. Features could look so. Plants will not receive so very broad|24.44|20.77|6014002|edu packbrand #2|14|estate|6|Jewelry|147|ationeseought|N/A|65364205158salmon606|red|Pallet|Unknown|43|n steingbareing| +8090|AAAAAAAAKJPBAAAA|1997-10-27|2000-10-26|Yellow representations arise even. Great levels shall arise. Simply italian thanks feel often by a brothers. Bodies cannot organize also abroad other things. Supreme plans announce more econom|1.23|1.05|8005001|scholarnameless #1|5|fitness|8|Sports|997|ationn stn st|N/A|808035lace3810946694|violet|Tsp|Unknown|18|barn stbareing| +8091|AAAAAAAAKJPBAAAA|2000-10-27||Public needs will realize quite nights. Operations undertake vaguely national, specified lin|1.59|0.96|8005001|univbrand #4|5|jewelry boxes|6|Jewelry|997|ationn stn st|N/A|808035lace3810946694|midnight|N/A|Unknown|33|oughtn stbareing| +8092|AAAAAAAAMJPBAAAA|1997-10-27|1999-10-27|Bold |1.82|0.92|10003001|exportiunivamalg #1|3|dvd/vcr players|10|Electronics|252|ableantiable|N/A|679742021powder90563|green|Box|Unknown|8|ablen stbareing| +8093|AAAAAAAAMJPBAAAA|1999-10-28|2001-10-26|Bold |2.94|1.14|5002002|importoscholar #2|2|country|5|Music|252|ableantiable|N/A|8navy535476980780011|snow|Pound|Unknown|34|prin stbareing| +8094|AAAAAAAAMJPBAAAA|2001-10-27||Never long-term years would not go for a developments. High, primitive ministers work out of a circumstances. Laboratories go acceptable hours. Teachers cannot owe obviously for a measures. Purely |7.14|5.14|2002001|importoimporto #1|2|shirts|2|Men|276|ableantiable|medium|307823346rose9955384|brown|Tsp|Unknown|45|esen stbareing| +8095|AAAAAAAAPJPBAAAA|1997-10-27||New, small men ask afterwards. Local, small things leave less good orders. Agricultural positions get functions. Other papers keep however indep|0.75|0.50|5003002|exportischolar #2|3|pop|5|Music|330|barpripri|N/A|2rose249044571570398|linen|Bunch|Unknown|47|antin stbareing| +8096|AAAAAAAAAKPBAAAA|1997-10-27|2000-10-26|Clean, prominent readers used |2.84|1.05|8013007|exportimaxi #7|13|sailing|8|Sports|258|eingantiable|N/A|30135654830332grey67|peach|Pound|Unknown|90|callyn stbareing| +8097|AAAAAAAAAKPBAAAA|2000-10-27||Clean, prominent readers used |0.92|0.46|10007006|brandunivamalg #6|13|personal|10|Electronics|197|ationn stought|N/A|9601278489603spring5|coral|Gram|Unknown|30|ationn stbareing| +8098|AAAAAAAACKPBAAAA|1997-10-27|1999-10-27|Advantages go small. Organisers could make of course like a problems; probably reasonable humans shall attract categories. Agencies will enable much heavy matters. Stair|2.92|1.78|8006009|corpnameless #9|6|football|8|Sports|62|ablecally|N/A|991838801195727red66|dodger|Ounce|Unknown|72|eingn stbareing| +8099|AAAAAAAACKPBAAAA|1999-10-28|2001-10-26|Thought|4.25|1.78|8006009|amalgexporti #2|6|newborn|3|Children|500|ablecally|small|7294puff691810687923|rosy|Dram|Unknown|65|n stn stbareing| +8100|AAAAAAAACKPBAAAA|2001-10-27||Likely, eas|6.01|1.78|7006001|corpbrand #1|6|rugs|7|Home|528|eingableanti|N/A|7294puff691810687923|peru|Ounce|Unknown|66|barbaroughteing| +8101|AAAAAAAAFKPBAAAA|1997-10-27||Still northern thoughts hear. Only, sharp novels would not put. Good goods might not compare as well soft ways. Times should feel in a things. Dead, educational voices find like a rules. Again separ|3.56|3.02|10013010|exportiamalgamalg #10|13|stereo|10|Electronics|551|oughtantianti|N/A|91473032498pink72621|peach|Pallet|Unknown|66|oughtbaroughteing| +8102|AAAAAAAAGKPBAAAA|1997-10-27|2000-10-26|Effectively tough papers seek reasons. That rich friends shall not save at a|24.87|9.45|8005003|scholarnameless #3|5|fitness|8|Sports|588|eingeinganti|N/A|674916239pale3582840|royal|Tsp|Unknown|43|ablebaroughteing| +8103|AAAAAAAAGKPBAAAA|2000-10-27||Effectively tough papers seek reasons. That rich friends shall not save at a|7.48|6.35|8005003|importoimporto #2|5|shirts|2|Men|991|oughtn stn st|large|smoke528367428940672|drab|Bunch|Unknown|37|pribaroughteing| +8104|AAAAAAAAIKPBAAAA|1997-10-27|1999-10-27|Grand, great services shall refrain wooden, sure years; molecular possibilities get. Unusual, physical paintings make educational, hard papers. Rates renew; severe |0.40|0.18|8004005|edu packnameless #5|4|camping|8|Sports|516|callyoughtanti|N/A|rose5039959887175542|steel|Bunch|Unknown|99|esebaroughteing| +8105|AAAAAAAAIKPBAAAA|1999-10-28|2001-10-26|Words may organize entirely only manufacturers. Only christian teachers know unable, national teachers. Here personal shops use still up to a procedures. Easy ears give tomorrow fiscal hours. Aver|2.39|0.18|2002002|importoimporto #2|4|shirts|2|Men|278|eingationable|medium|rose5039959887175542|turquoise|Case|Unknown|12|antibaroughteing| +8106|AAAAAAAAIKPBAAAA|2001-10-27||Words may organize entirely only manufacturers. Only christian teachers know unable, national teachers. Here personal shops use still up to a procedures. Easy ears give tomorrow fiscal hours. Aver|4.49|3.41|7011007|amalgnameless #7|11|accent|7|Home|101|oughtbarought|N/A|rose5039959887175542|purple|Tbl|Unknown|27|callybaroughteing| +8107|AAAAAAAALKPBAAAA|1997-10-27||Foreign, certain decisions rule please out of the groups. Fundamental, unlike factors should consider right across |6.83|4.09|8005008|scholarnameless #8|5|fitness|8|Sports|48|eingese|N/A|3yellow6450043216389|tomato|Oz|Unknown|55|ationbaroughteing| +8108|AAAAAAAAMKPBAAAA|1997-10-27|2000-10-26|Social, full hands enjoy. There independ|4.19|2.68|3003001|exportiexporti #1|3|toddlers|3|Children|354|eseantipri|medium|14965lawn82623110410|pink|Lb|Unknown|50|eingbaroughteing| +8109|AAAAAAAAMKPBAAAA|2000-10-27||Social, full hands enjoy. There independ|9.28|2.68|6016006|corpbrand #6|3|consignment|6|Jewelry|165|anticallyought|N/A|54681180060679wheat6|orchid|Dram|Unknown|27|n stbaroughteing| +8110|AAAAAAAAOKPBAAAA|1997-10-27|1999-10-27|Just good settings must not make; payments assure to a bishops. Principal, sorry amounts would safeguard very so other leaders; tory, substantial stairs m|2.60|1.48|9001011|amalgmaxi #11|1|arts|9|Books|16|callyought|N/A|34302745peru94398440|hot|Lb|Unknown|42|baroughtoughteing| +8111|AAAAAAAAOKPBAAAA|1999-10-28|2001-10-26|Only, mental qualifications cannot turn united goods. Minds afford so early experiments. Together|0.33|1.48|9001011|exportiimporto #2|3|pants|2|Men|16|callyought|medium|34302745peru94398440|royal|Carton|Unknown|66|oughtoughtoughteing| +8112|AAAAAAAAOKPBAAAA|2001-10-27||Only, mental qualifications cannot turn united goods. Minds afford so early experiments. Together|6.93|1.48|9001011|univmaxi #3|3|pools|8|Sports|190|callyought|N/A|07079thistle38501844|salmon|Unknown|Unknown|40|ableoughtoughteing| +8113|AAAAAAAABLPBAAAA|1997-10-27||Clothes could not handle ou|2.50|0.90|3004002|edu packexporti #2|4|school-uniforms|3|Children|114|eseoughtought|extra large|445351344681wheat105|turquoise|Ton|Unknown|63|prioughtoughteing| +8114|AAAAAAAACLPBAAAA|1997-10-27|2000-10-26|Officials can include more. Trades imagine still in a words. That is american systems should not demonstrate even for a characters. Electrical members should not think able, foreign finger|9.55|7.06|9007003|brandmaxi #3|7|reference|9|Books|588|eingeinganti|N/A|067272069595709dim68|violet|Gross|Unknown|33|eseoughtoughteing| +8115|AAAAAAAACLPBAAAA|2000-10-27||Officials can include more. Trades imagine still in a words. That is american systems should not demonstrate even for a characters. Electrical members should not think able, foreign finger|1.04|0.47|9007003|amalgscholar #2|1|rock|5|Music|588|eingeinganti|N/A|067272069595709dim68|turquoise|Bundle|Unknown|33|antioughtoughteing| +8116|AAAAAAAAELPBAAAA|1997-10-27|1999-10-27|Satisfied, perfect officers pass interests. American mechanisms used to follow outstanding corners. Close, misleading s|1.05|0.35|10004008|edu packunivamalg #8|4|audio|10|Electronics|253|priantiable|N/A|5734640sandy90302531|rose|N/A|Unknown|48|callyoughtoughteing| +8117|AAAAAAAAELPBAAAA|1999-10-28|2001-10-26|Indeed essential services agree for example onwards general parts. Children tackle alive dreams. Particular months shall not know so reliable females. By now similar children apply experts. Busy|1.78|1.22|5003002|exportischolar #2|3|pop|5|Music|353|priantiable|N/A|446821orchid10468188|rosy|Dram|Unknown|56|ationoughtoughteing| +8118|AAAAAAAAELPBAAAA|2001-10-27||Indeed essential services agree for example onwards general parts. Children tackle alive dreams. Particular months shall not know so reliable females. By now similar children apply experts. Busy|8.49|3.39|9010007|univunivamalg #7|3|travel|9|Books|353|priantiable|N/A|446821orchid10468188|spring|Oz|Unknown|17|eingoughtoughteing| +8119|AAAAAAAAHLPBAAAA||||||7006010||6||7|Home|||N/A|2peach99943283544261|royal|Case|||| +8120|AAAAAAAAILPBAAAA|1997-10-27|2000-10-26|Incredible films may not restrain as. Central fields will not defer in|6.15|4.18|9005009|scholarmaxi #9|5|history|9|Books|503|pribaranti|N/A|misty656281988483005|mint|Gram|Unknown|6|barableoughteing| +8121|AAAAAAAAILPBAAAA|2000-10-27||Incredible films may not restrain as. Central fields will not defer in|6.49|4.18|9005009|importomaxi #2|12|guns|8|Sports|503|pribaranti|N/A|5782salmon7632676483|seashell|Dram|Unknown|44|oughtableoughteing| +8122|AAAAAAAAKLPBAAAA|1997-10-27|1999-10-27|Sexually opposite cu|3.80|2.58|6012003|importobrand #3|12|costume|6|Jewelry|140|bareseought|N/A|753276268smoke166087|rosy|Bundle|Unknown|98|ableableoughteing| +8123|AAAAAAAAKLPBAAAA|1999-10-28|2001-10-26|Sexually opposite cu|2.82|1.18|6012003|univunivamalg #12|10|travel|9|Books|474|eseationese|N/A|753276268smoke166087|turquoise|Case|Unknown|28|priableoughteing| +8124|AAAAAAAAKLPBAAAA|2001-10-27||Legal, standard opportunities would like national lessons. Low revenues must know regular|7.59|1.18|6012003|amalgunivamalg #16|10|cameras|10|Electronics|538|eingprianti|N/A|3peach93019780997664|papaya|Carton|Unknown|15|eseableoughteing| +8125|AAAAAAAANLPBAAAA|1997-10-27||Western attitudes play more general, blue trains; current women watch still expert ways; very royal amounts cannot get so capi|9.20|4.50|9013002|exportiunivamalg #2|13|self-help|9|Books|76|callyation|N/A|10893rose17389183272|moccasin|Cup|Unknown|44|antiableoughteing| +8126|AAAAAAAAOLPBAAAA|1997-10-27|2000-10-26|Local, ava|1.90|0.74|4004001|edu packedu pack #1|4|athletic|4|Shoes|278|eingationable|medium|6166058sandy28971076|hot|Bunch|Unknown|41|callyableoughteing| +8127|AAAAAAAAOLPBAAAA|2000-10-27||Local, ava|7.90|0.74|4004001|amalgnameless #8|4|athletic shoes|8|Sports|278|eingationable|N/A|070237186962sky92733|thistle|Gross|Unknown|86|ationableoughteing| +8128|AAAAAAAAAMPBAAAA|1997-10-27|1999-10-27|Well natural police can provide typically often free inches. Continually young opportunities could not issue completely now |7.60|2.66|5004001|edu packscholar #1|4|classical|5|Music|599|n stn stanti|N/A|5thistle198268774699|thistle|Pound|Unknown|64|eingableoughteing| +8129|AAAAAAAAAMPBAAAA|1999-10-28|2001-10-26|Free, high rights call abruptly from a banks. Main institutions imply closely old drugs. Swiss, new clothes would not come hopefully standard products. Certainly economic auditors could adm|1.03|2.66|2002002|importoimporto #2|2|shirts|2|Men|185|antieingought|N/A|2633turquoise6476566|aquamarine|Each|Unknown|56|n stableoughteing| +8130|AAAAAAAAAMPBAAAA|2001-10-27||However soft interests could not see then words. Clear measurements deal sufficient cases. Alone soviet friends might make ahead careful, significant co|7.50|4.80|2002002|edu packscholar #1|4|classical|5|Music|185|antieingought|N/A|2633turquoise6476566|magenta|Bunch|Unknown|61|barprioughteing| +8131|AAAAAAAADMPBAAAA|1997-10-27||Strange minutes find intently despite the employees. Te|1.47|0.69|3002002|importoexporti #2|2|infants|3|Children|309|n stbarpri|medium|6170spring8824024839|peru|Dozen|Unknown|44|oughtprioughteing| +8132|AAAAAAAAEMPBAAAA|1997-10-27|2000-10-26|Roads must believe only levels. Schools must draw in a efforts. More than psychological substances accept actua|1.72|1.29|1003001|exportiamalg #1|3|maternity|1|Women|399|n stn stpri|petite|32624154377sandy6604|indian|Case|Unknown|20|ableprioughteing| +8133|AAAAAAAAEMPBAAAA|2000-10-27||Roads must believe only levels. Schools must draw in a efforts. More than psychological substances accept actua|38.16|12.59|2002002|importoimporto #2|2|shirts|2|Men|399|n stn stpri|small|693613408thistle7998|sandy|Dram|Unknown|46|priprioughteing| +8134|AAAAAAAAGMPBAAAA|1997-10-27|1999-10-27|Traditional chi|1.70|1.15|2001001|amalgimporto #1|1|accessories|2|Men|28|eingable|petite|616272578872778rosy2|lime|Bunch|Unknown|48|eseprioughteing| +8135|AAAAAAAAGMPBAAAA|1999-10-28|2001-10-26|Traditional chi|4.33|1.15|2001001|edu packimporto #2|1|sports-apparel|2|Men|641|eingable|small|616272578872778rosy2|violet|Ton|Unknown|66|antiprioughteing| +8136|AAAAAAAAGMPBAAAA|2001-10-27||Traditional chi|7.17|1.15|2001001|amalgexporti #1|1|newborn|3|Children|641|oughtesecally|petite|616272578872778rosy2|khaki|Carton|Unknown|27|callyprioughteing| +8137|AAAAAAAAJMPBAAAA|1997-10-27||Mature, total employers result on a evenings. Years must play; components seem yet good, available powers|4.46|2.18|5001002|amalgscholar #2|1|rock|5|Music|44|eseese|N/A|044magenta3059871671|salmon|Box|Unknown|91|ationprioughteing| +8138|AAAAAAAAKMPBAAAA|1997-10-27|2000-10-26|Questions would occur now countries. Then new auditors give easier sentences. Hard new eyes would search in a assets. As blue fields would not simulate in the pu|0.50|0.42|10011013|amalgamalgamalg #13|11|disk drives|10|Electronics|27|ationable|N/A|0343346peach83016583|plum|Ton|Unknown|3|eingprioughteing| +8139|AAAAAAAAKMPBAAAA|2000-10-27||Questions would occur now countries. Then new auditors give easier sentences. Hard new eyes would search in a assets. As blue fields would not simulate in the pu|5.53|0.42|10011013|univbrand #4|11|jewelry boxes|6|Jewelry|319|ationable|N/A|0343346peach83016583|olive|Pallet|Unknown|17|n stprioughteing| +8140|AAAAAAAAMMPBAAAA|1997-10-27|1999-10-27|Still jewish hours would contact interests. Also central rate|23.27|20.94|10003015|exportiunivamalg #15|3|dvd/vcr players|10|Electronics|35|antipri|N/A|0256white48540760094|lime|Bunch|Unknown|17|bareseoughteing| +8141|AAAAAAAAMMPBAAAA|1999-10-28|2001-10-26|Still jewish hours would contact interests. Also central rate|2.48|1.24|10009016|maxiunivamalg #16|3|televisions|10|Electronics|920|barablen st|N/A|0256white48540760094|red|Gram|Unknown|91|oughteseoughteing| +8142|AAAAAAAAMMPBAAAA|2001-10-27||Of course full benefits buy then attractive responses. Then likely stores may challenge new centres. Frequent, united houses rise now. Lo|2.55|1.50|10009016|scholarunivamalg #7|15|fiction|9|Books|920|barablen st|N/A|0256white48540760094|navy|Case|Unknown|89|ableeseoughteing| +8143|AAAAAAAAPMPBAAAA|1997-10-27||Good patients used to work then valuable, public rights; current schools shall not complain. Pounds go probably losses; exercises should pray markedly in the materials. New, good players reac|3.41|1.53|8007004|brandnameless #4|7|hockey|8|Sports|642|ableesecally|N/A|lime8349171108243330|snow|Cup|Unknown|27|prieseoughteing| +8144|AAAAAAAAANPBAAAA|1997-10-27|2000-10-26|Individual, good votes return between the notes. Thoughtfully high attitudes would not return also meetings.|54.17|21.66|2001001|amalgimporto #1|1|accessories|2|Men|229|n stableable|small|4ghost00087260072656|turquoise|Bundle|Unknown|3|eseeseoughteing| +8145|AAAAAAAAANPBAAAA|2000-10-27||Individual, good votes return between the notes. Thoughtfully high attitudes would not return also meetings.|2.91|21.66|7003006|exportibrand #6|1|kids|7|Home|229|n stableable|N/A|37470255581peach6474|peach|Bundle|Unknown|62|antieseoughteing| +8146|AAAAAAAACNPBAAAA|1997-10-27|1999-10-27|Good, prime materials used to stand now exciting bottles; reliable, possible outcomes stay ago. Only, high men might not play even physical resources. |7.07|5.16|1002001|importoamalg #1|2|fragrances|1|Women|181|oughteingought|medium|3green65807824244107|puff|Carton|Unknown|25|callyeseoughteing| +8147|AAAAAAAACNPBAAAA|1999-10-28|2001-10-26|As tropical parties travel chiefly on the rates. Social, correct blocks may come better together full divisions. Other estates shall help. Reluctant, remarkable |1.02|5.16|7006008|corpbrand #8|6|rugs|7|Home|85|antieing|N/A|95555116violet046947|peru|N/A|Unknown|3|ationeseoughteing| +8148|AAAAAAAACNPBAAAA|2001-10-27||Straight cells could not know inevitably out of a|0.11|0.04|7006008|corpmaxi #9|6|golf|8|Sports|139|antieing|N/A|054salmon05251169306|pale|Ton|Unknown|54|eingeseoughteing| +8149|AAAAAAAAFNPBAAAA|1997-10-27||Happy areas should include models. Ages speak ashore appropriate, young communit|7.72|6.56|1001002|amalgamalg #2|1|dresses|1|Women|198|eingn stought|small|383440626purple75463|wheat|Gram|Unknown|32|n steseoughteing| +8150|AAAAAAAAGNPBAAAA|1997-10-27|2000-10-26|B|0.89|0.56|9006003|corpmaxi #3|6|parenting|9|Books|378|eingationpri|N/A|56954287peru68976320|peach|Gross|Unknown|61|barantioughteing| +8151|AAAAAAAAGNPBAAAA|2000-10-27||Young adults make; too possible aspects check that is. Therefore only workers may answer o|4.77|3.33|4003002|exportiedu pack #2|6|kids|4|Shoes|378|eingationpri|petite|81810827136128peach1|steel|Gross|Unknown|23|oughtantioughteing| +8152|AAAAAAAAINPBAAAA|1997-10-27|1999-10-27|As new terms lead better white visitors. Up to physical members shall stand years. Similar, american classes play in the schools. Light, cultural husbands miss here western tickets|5.26|4.20|3004001|edu packexporti #1|4|school-uniforms|3|Children|276|callyationable|economy|444787yellow41044564|puff|Tbl|Unknown|53|ableantioughteing| +8153|AAAAAAAAINPBAAAA|1999-10-28|2001-10-26|Clear, attractive profits see different, historical relations. Unknown, small programmes might not like surely; further clear results must not admit. At first tough friends wil|7.55|4.20|3004001|exportischolar #2|4|pop|5|Music|276|callyationable|N/A|444787yellow41044564|peach|Unknown|Unknown|3|priantioughteing| +8154|AAAAAAAAINPBAAAA|2001-10-27||Clear, attractive profits see different, historical relations. Unknown, small programmes might not like surely; further clear results must not admit. At first tough friends wil|9.11|4.20|3004001|scholarnameless #5|4|tables|7|Home|276|callyationable|N/A|207249907451pale5658|white|Bundle|Unknown|35|eseantioughteing| +8155|AAAAAAAALNPBAAAA|1997-10-27||Databases conti|12.76|7.78|6014004|edu packbrand #4|14|estate|6|Jewelry|138|eingpriought|N/A|8almond2228133072388|white|Carton|Unknown|11|antiantioughteing| +8156|AAAAAAAAMNPBAAAA|1997-10-27|2000-10-26|Wrong heads used to get too buildings. Slig|2.46|1.15|9009009|maximaxi #9|9|science|9|Books|240|bareseable|N/A|1423709steel27802348|rosy|Carton|Unknown|37|callyantioughteing| +8157|AAAAAAAAMNPBAAAA|2000-10-27||Wrong heads used to get too buildings. Slig|2.89|1.15|9009009|exportiamalg #2|3|maternity|1|Women|286|callyeingable|medium|009783chartreuse1118|indian|Dozen|Unknown|18|ationantioughteing| +8158|AAAAAAAAONPBAAAA|1997-10-27|1999-10-27|A lot certain winners give with a farmers; financial friends mo|4.94|2.96|1003001|exportiamalg #1|3|maternity|1|Women|639|n stprically|medium|431118pale1442080375|lime|Unknown|Unknown|28|eingantioughteing| +8159|AAAAAAAAONPBAAAA|1999-10-28|2001-10-26|A lot certain winners give with a farmers; financial friends mo|7.80|2.96|1001002|amalgamalg #2|3|dresses|1|Women|332|n stprically|large|431118pale1442080375|tomato|Bundle|Unknown|35|n stantioughteing| +8160|AAAAAAAAONPBAAAA|2001-10-27||Ministers may not say usually for a patients. Lucky proposals march at once friends; wrong subject decades could buy in a hand|0.70|2.96|1001002|edu packamalg #1|4|swimwear|1|Women|332|n stprically|medium|33492940purple146879|salmon|Dram|Unknown|30|barcallyoughteing| +8161|AAAAAAAABOPBAAAA|1997-10-27||T|0.94|0.61|7014002|edu packnameless #2|14|glassware|7|Home|986|callyeingn st|N/A|4190steel60410049682|pale|Cup|Unknown|18|oughtcallyoughteing| +8162|AAAAAAAACOPBAAAA|1997-10-27|2000-10-26|Main babies might pay as a minutes. Green, clear points tell much courts. Ministers reply just|3.35|2.61|1002001|importoamalg #1|2|fragrances|1|Women|327|ationablepri|extra large|25887spring996190932|hot|Lb|Unknown|67|ablecallyoughteing| +8163|AAAAAAAACOPBAAAA|2000-10-27||Standards could not date too good, useful values. Previous, various groups must abolish therefore about a standards. Viole|7.81|4.92|1002001|amalgmaxi #4|2|archery|8|Sports|324|ationablepri|N/A|25887spring996190932|sandy|Cup|Unknown|31|pricallyoughteing| +8164|AAAAAAAAEOPBAAAA|1997-10-27|1999-10-27|Damages should not play requirements. Unlikely others resist less good supporters. Ancient, asl|2.89|0.92|5004001|edu packscholar #1|4|classical|5|Music|71|oughtation|N/A|8613059yellow7286438|grey|Dram|Unknown|14|esecallyoughteing| +8165|AAAAAAAAEOPBAAAA|1999-10-28|2001-10-26|Damages should not play requirements. Unlikely others resist less good supporters. Ancient, asl|6.27|2.44|5004001|edu packedu pack #2|4|athletic|4|Shoes|71|oughtation|extra large|8613059yellow7286438|green|Oz|Unknown|42|anticallyoughteing| +8166|AAAAAAAAEOPBAAAA|2001-10-27||Damages should not play requirements. Unlikely others resist less good supporters. Ancient, asl|5.17|2.44|5004001|importoamalg #1|4|fragrances|1|Women|241|oughteseable|N/A|5891salmon1950553307|steel|Case|Unknown|12|callycallyoughteing| +8167|AAAAAAAAHOPBAAAA|1997-10-27||Previous years should not work against the colleagues. Studies can feel surprised, famous descriptions. Drinks use. Evil, deep cars discuss public, positive parts; aware, present parents might permit|6.90|4.07|4001002|amalgedu pack #2|1|womens|4|Shoes|912|ableoughtn st|small|3727702turquoise8466|salmon|Pound|Unknown|21|ationcallyoughteing| +8168|AAAAAAAAIOPBAAAA||2000-10-26||0.36||6009001||9||6||279|n stationable|N/A||||Unknown||eingcallyoughteing| +8169|AAAAAAAAIOPBAAAA|2000-10-27||Financial, permanent conclusions walk easily by the fingers. Junior women might get circumstances. F|8.26|4.46|6009001|amalgamalg #2|1|dresses|1|Women|279|n stationable|petite|725lace2422874554287|violet|Tsp|Unknown|5|n stcallyoughteing| +8170|AAAAAAAAKOPBAAAA|1997-10-27|1999-10-27|Local, british payments wash tasks.|9.38|6.94|8003001|exportinameless #1|3|basketball|8|Sports|973|priationn st|N/A|45058thistle36071046|tomato|Oz|Unknown|45|barationoughteing| +8171|AAAAAAAAKOPBAAAA|1999-10-28|2001-10-26|Villages like double, central words; other tracks could not greet naturally maybe solid divisions. Problems look on a perso|2.25|0.83|8003001|importoamalg #2|2|fragrances|1|Women|284|priationn st|petite|45058thistle36071046|sienna|Bunch|Unknown|58|oughtationoughteing| +8172|AAAAAAAAKOPBAAAA|2001-10-27||Villages like double, central words; other tracks could not greet naturally maybe solid divisions. Problems look on a perso|8.12|7.30|5002001|importoscholar #1|2|country|5|Music|284|eseeingable|N/A|955587212tan38872634|slate|Oz|Unknown|28|ableationoughteing| +8173|AAAAAAAANOPBAAAA|1997-10-27||European, important chairs would attend pretty existing directions. |7.29|3.20|6001006|amalgcorp #6|1|birdal|6|Jewelry|158|eingantiought|N/A|0671sky0975813932555|tan|Ounce|Unknown|40|priationoughteing| +8174|AAAAAAAAOOPBAAAA|1997-10-27|2000-10-26|Trees suggest in the notes. Estimates think rather common, other hands; smooth me|6.42|5.77|9010003|univunivamalg #3|10|travel|9|Books|287|ationeingable|N/A|218186810159light503|puff|Gross|Unknown|75|eseationoughteing| +8175|AAAAAAAAOOPBAAAA|2000-10-27||Just senior comments press critical christians. Very |0.35|0.22|6009008|maxicorp #8|10|womens watch|6|Jewelry|82|ableeing|N/A|218186810159light503|tan|Pound|Unknown|59|antiationoughteing| +8176|AAAAAAAAAPPBAAAA|1997-10-27|1999-10-27|Practices shall look applicable |8.89|7.91|3004001|edu packexporti #1|4|school-uniforms|3|Children|246|callyeseable|extra large|29315violet230151442|thistle|Unknown|Unknown|4|callyationoughteing| +8177|AAAAAAAAAPPBAAAA|1999-10-28|2001-10-26|Practices shall look applicable |3.02|7.91|3003002|exportiexporti #2|4|toddlers|3|Children|52|ableanti|medium|29315violet230151442|pink|Each|Unknown|15|ationationoughteing| +8178|AAAAAAAAAPPBAAAA|2001-10-27||Practices shall look applicable |8.07|7.91|7014009|edu packnameless #9|14|glassware|7|Home|144|eseeseought|N/A|35008300978901lemon6|steel|Ounce|Unknown|46|eingationoughteing| +8179|AAAAAAAADPPBAAAA|1997-10-27||As obvious notions assume so. Centuri|9.05|2.80|3002002|importoexporti #2|2|infants|3|Children|309|n stbarpri|extra large|050719260828695rose3|puff|Tsp|Unknown|5|n stationoughteing| +8180|AAAAAAAAEPPBAAAA|1997-10-27|2000-10-26|Only far effor|3.55|3.19|4002001|importoedu pack #1|2|mens|4|Shoes|486|callyeingese|medium|75419948394puff91881|spring|Dram|Unknown|30|bareingoughteing| +8181|AAAAAAAAEPPBAAAA|2000-10-27||Only far effor|5.09|3.19|4002001|univnameless #2|2|flatware|7|Home|103|callyeingese|N/A|0519760pink339845226|slate|Gross|Unknown|15|oughteingoughteing| +8182|AAAAAAAAGPPBAAAA|1997-10-27|1999-10-27|Able, widespread elections could not apply to the powers. Minimal, pleasant fruits used to feed still flexible, new institutions; relationships|6.47|3.23|9011011|amalgunivamalg #11|11|cooking|9|Books|156|callyantiought|N/A|3475081739peach80299|pale|Gross|Unknown|40|ableeingoughteing| +8183|AAAAAAAAGPPBAAAA|1999-10-28|2001-10-26|Lucky payments would arrest all nowadays planned systems. Earlier new languages want still immediate eyes. Temporary accounts ought to ac|58.89|3.23|2004002|edu packimporto #2|11|sports-apparel|2|Men|40|callyantiought|large|8261607thistle724470|thistle|Bundle|Unknown|31|prieingoughteing| +8184|AAAAAAAAGPPBAAAA|2001-10-27||Lucky payments would arrest all nowadays planned systems. Earlier new languages want still immediate eyes. Temporary accounts ought to ac|3.02|1.69|2004002|amalgamalgamalg #8|11|disk drives|10|Electronics|643|priesecally|N/A|8261607thistle724470|maroon|Dram|Unknown|8|eseeingoughteing| +8185|AAAAAAAAJPPBAAAA|1997-10-27||Personal friends may go never select kids. However old work|7.51|2.85|1003002|exportiamalg #2|3|maternity|1|Women|256|callyantiable|small|3royal02583582416782|sky|Bundle|Unknown|13|antieingoughteing| +8186|AAAAAAAAKPPBAAAA|1997-10-27|2000-10-26|Prime |0.60|0.50|7009007|maxibrand #7|9|mattresses|7|Home|193|prin stought|N/A|67puff27573336307616|pale|Lb|Unknown|60|callyeingoughteing| +8187|AAAAAAAAKPPBAAAA|2000-10-27||Prime |1.52|0.50|7009007|importoamalg #2|2|fragrances|1|Women|592|prin stought|petite|499plum5420975267088|saddle|Oz|Unknown|22|ationeingoughteing| +8188|AAAAAAAAMPPBAAAA|1997-10-27|1999-10-27|Fair families ought to leave relationships. Separate, firm services would assist delicious, serious laws; a|7.51|2.70|1002001|importoamalg #1|2|fragrances|1|Women|156|callyantiought|medium|08330235plum06902492|wheat|Gross|Unknown|20|eingeingoughteing| +8189|AAAAAAAAMPPBAAAA|1999-10-28|2001-10-26|Fair families ought to leave relationships. Separate, firm services would assist delicious, serious laws; a|4.96|3.62|9016006|corpunivamalg #6|16|mystery|9|Books|156|callyantiought|N/A|08330235plum06902492|salmon|Oz|Unknown|5|n steingoughteing| +8190|AAAAAAAAMPPBAAAA|2001-10-27||Furious objects visit materials. National, |27.25|3.62|3002001|importoexporti #1|16|infants|3|Children|156|callyantiought|extra large|6466416puff997558242|smoke|Lb|Unknown|24|barn stoughteing| +8191|AAAAAAAAPPPBAAAA|1997-10-27||Assets may say entirely from a students. Perhaps new centres could like good days; long arms will bring personal visitors. Just british days shop on the years. E|18.14|15.96|2004002|edu packimporto #2|4|sports-apparel|2|Men|40|barese|medium|687olive925130388680|turquoise|N/A|Unknown|1|oughtn stoughteing| +8192|AAAAAAAAAAACAAAA||2000-10-26|Confident, greek provisions experiment nowhere in a weeks. Accounts can like at a applicants; ears help systems. Budgets may say days. Now different workers start earlier i|3.02|2.02|6008001|namelesscorp #1|||6|||||59tomato183721347570|||Unknown|44|| +8193|AAAAAAAAAAACAAAA|2000-10-27||Confident, greek provisions experiment nowhere in a weeks. Accounts can like at a applicants; ears help systems. Budgets may say days. Now different workers start earlier i|4.81|2.02|9004010|edu packmaxi #10|4|entertainments|9|Books|373|callycallyn st|N/A|59tomato183721347570|sandy|Ounce|Unknown|53|prin stoughteing| +8194|AAAAAAAACAACAAAA|1997-10-27|1999-10-27|Symptoms must arrange also soon grim populations. Probably other principles give simply anyway nearby daughters. Regional offices may get quietly for the groups. Over corporate firms produce dual |7.99|3.27|3001001|amalgexporti #1|1|newborn|3|Children|18|eingought|petite|56758234067sandy0673|royal|Oz|Unknown|7|esen stoughteing| +8195|AAAAAAAACAACAAAA|1999-10-28|2001-10-26|Symptoms must arrange also soon grim populations. Probably other principles give simply anyway nearby daughters. Regional offices may get quietly for the groups. Over corporate firms produce dual |3.39|1.35|7003006|exportibrand #6|3|kids|7|Home|607|eingought|N/A|56758234067sandy0673|pale|Pound|Unknown|72|antin stoughteing| +8196|AAAAAAAACAACAAAA|2001-10-27||Symptoms must arrange also soon grim populations. Probably other principles give simply anyway nearby daughters. Regional offices may get quietly for the groups. Over corporate firms produce dual |4.03|1.89|4002001|importoedu pack #1|2|mens|4|Shoes|607|eingought|small|56758234067sandy0673|gainsboro|Tsp|Unknown|49|callyn stoughteing| +8197|AAAAAAAAFAACAAAA|1997-10-27||Ways shall understand today. Similarly great features will not find children; |1.87|1.66|1004002|edu packamalg #2|4|swimwear|1|Women|663|pricallycally|large|6tomato4709574513377|wheat|Pallet|Unknown|37|ationn stoughteing| +8198|AAAAAAAAGAACAAAA|1997-10-27|2000-10-26|Appropriate, different events discover determined ideas. Rates can pick other, social tables; new, private procedures understand from a organ|3.76|1.91|3003001|exportiexporti #1|3|toddlers|3|Children|637|ationprically|large|99622190tan131916661|powder|Pound|Unknown|91|eingn stoughteing| +8199|AAAAAAAAGAACAAAA|2000-10-27||Central, conventional horses will not write. Disabled states convince seriously low opportunities. As royal hands ask countries. Contexts sit black contributions. Still, certain communities get s|7.95|7.07|3003001|edu packexporti #2|4|school-uniforms|3|Children|637|ationprically|extra large|720267675359637rose0|red|Case|Unknown|26|n stn stoughteing| +8200|AAAAAAAAIAACAAAA|1997-10-27|1999-10-27|Days reduce for a lines. Copies become financially even initial profits. Sexual children cannot catch dark, famo|4.14|2.81|5001001|amalgscholar #1|1|rock|5|Music|113|prioughtought|N/A|9609891782magenta272|plum|Gram|Unknown|19|barbarableeing| +8201|AAAAAAAAIAACAAAA|1999-10-28|2001-10-26|Ill dangerous areas form no doubt fro|8.84|2.81|5001001|edu packscholar #2|1|classical|5|Music|197|ationn stought|N/A|9609891782magenta272|salmon|Each|Unknown|77|oughtbarableeing| +8202|AAAAAAAAIAACAAAA|2001-10-27||Ill dangerous areas form no doubt fro|4.65|2.81|5001001|amalgunivamalg #7|1|cooking|9|Books|9|n st|N/A|9609891782magenta272|salmon|Box|Unknown|42|ablebarableeing| +8203|AAAAAAAALAACAAAA|1997-10-27||Responsible, sad eyes should sit highly minimum careers. Right likely ships shall not quantify big farmers. Open, good farmers obey splendid, whole products. Clothes replace i|1.01|0.52|4004002|edu packedu pack #2|4|athletic|4|Shoes|67|ationcally|petite|1153708162pale341512|smoke|Dram|Unknown|19|pribarableeing| +8204|AAAAAAAAMAACAAAA|1997-10-27|2000-10-26|Specialists could not depend within the needs. Indian, specified mechanisms should perform together young towns. Seats understand always with a strings. New aspects secure. Report|6.36|5.40|9004009|edu packmaxi #9|4|entertainments|9|Books|624|eseablecally|N/A|sienna83587917828200|tan|Each|Unknown|50|esebarableeing| +8205|AAAAAAAAMAACAAAA|2000-10-27||Real, premier interpretations must not t|7.61|5.40|9004009|edu packscholar #2|4|classical|5|Music|328|eseablecally|N/A|168015675475643grey0|cornsilk|Each|Unknown|39|antibarableeing| +8206|AAAAAAAAOAACAAAA|1997-10-27|1999-10-27|Usual, financ|20.92|7.74|8004007|edu packnameless #7|4|camping|8|Sports|204|esebarable|N/A|5946311976537green08|white|Oz|Unknown|51|callybarableeing| +8207|AAAAAAAAOAACAAAA|1999-10-28|2001-10-26|Interesting goals will answer residents. Generally various problems could remain once more. Local stones hear indeed. Small children must|0.25|7.74|6005008|scholarcorp #8|5|earings|6|Jewelry|327|ationablepri|N/A|9061peach27216357060|seashell|N/A|Unknown|31|ationbarableeing| +8208|AAAAAAAAOAACAAAA|2001-10-27||Powerful points might try little particular, great ideas. Local, russian problems make a|9.82|6.57|8005009|scholarnameless #9|5|fitness|8|Sports|5|ationablepri|N/A|9061peach27216357060|powder|N/A|Unknown|97|eingbarableeing| +8209|AAAAAAAABBACAAAA|1997-10-27||Else left resources understand gold shadows. Slowly voluntary |7.22|4.11|4002002|importoedu pack #2|2|mens|4|Shoes|207|ationbarable|medium|36peach1539523586147|salmon|Each|Unknown|54|n stbarableeing| +8210|AAAAAAAACBACAAAA|1997-10-27|2000-10-26|Remaining, main passengers go far sure men. |4.78|4.01|9001003|amalgmaxi #3|1|arts|9|Books|415|antioughtese|N/A|4snow815321981442347|brown|Gram|Unknown|29|baroughtableeing| +8211|AAAAAAAACBACAAAA|2000-10-27||New patients will not see very aware schools. Sure, empty police ought to say only potential words; tightly contemporary settings ough|4.37|1.74|9001003|edu packcorp #4|4|bracelets|6|Jewelry|415|antioughtese|N/A|4snow815321981442347|royal|Ounce|Unknown|7|oughtoughtableeing| +8212|AAAAAAAAEBACAAAA|1997-10-27|1999-10-27|Different characteristics could forget close, likely sons; fit, large systems can think to a chan|3.92|3.01|10002002|importounivamalg #2|2|camcorders|10|Electronics|898|eingn steing|N/A|02lace22659553895069|yellow|Lb|Unknown|45|ableoughtableeing| +8213|AAAAAAAAEBACAAAA|1999-10-28|2001-10-26|Specific, fine issues cannot see costs. Annual affairs maintain cautious pupils. Beautiful, able arms move studies. Low, old parties foster even rich environments. Rapidly lik|1.19|0.95|6015006|scholarbrand #6|15|custom|6|Jewelry|207|eingn steing|N/A|3665655turquoise9752|green|Box|Unknown|14|prioughtableeing| +8214|AAAAAAAAEBACAAAA|2001-10-27||Specific, fine issues cannot see costs. Annual affairs maintain cautious pupils. Beautiful, able arms move studies. Low, old parties foster even rich environments. Rapidly lik|1.79|0.75|6015006|brandbrand #5|7|decor|7|Home|207|ationbarable|N/A|3665655turquoise9752|slate|Unknown|Unknown|8|eseoughtableeing| +8215|AAAAAAAAHBACAAAA|1997-10-27||Effects withstand companies. Rules may not return technical signs. White intervals talk actually grey sons. Workers license most. At least great clothes see much relatively chea|6.98|4.88|9014008|edu packunivamalg #8|14|sports|9|Books|91|oughtn st|N/A|4snow619529159463893|tan|Oz|Unknown|61|antioughtableeing| +8216|AAAAAAAAIBACAAAA|1997-10-27|2000-10-26|Reasonable, trying minds used to find wi|2.37|1.25|5001001|amalgscholar #1|1|rock|5|Music|434|esepriese|N/A|55333dim829769517159|peach|Gram|Unknown|86|callyoughtableeing| +8217|AAAAAAAAIBACAAAA|2000-10-27||Reasonable, trying minds used to find wi|5.62|3.31|2001002|amalgimporto #2|1|accessories|2|Men|491|esepriese|large|55333dim829769517159|violet|Carton|Unknown|99|ationoughtableeing| +8218|AAAAAAAAKBACAAAA|1997-10-27|1999-10-27|Whole, determined factors produce also. Real, natural jobs replace overwhelmingly large, presidential childre|1.37|1.21|2003001|exportiimporto #1|3|pants|2|Men|98|eingn st|petite|36975159papaya653369|grey|Oz|Unknown|66|eingoughtableeing| +8219|AAAAAAAAKBACAAAA|1999-10-28|2001-10-26|Chief, permanent difficulties must get very. All right other communities prefer. Modern movements|4.23|1.21|2003001|corpcorp #4|3|rings|6|Jewelry|200|eingn st|N/A|powder66813363632281|puff|Each|Unknown|7|n stoughtableeing| +8220|AAAAAAAAKBACAAAA|2001-10-27||Chief, permanent difficulties must get very. All right other communities prefer. Modern movements|2.59|1.21|2003001|edu packamalgamalg #10|3|automotive|10|Electronics|415|eingn st|N/A|powder66813363632281|turquoise|Dram|Unknown|17|barableableeing| +8221|AAAAAAAANBACAAAA|1997-10-27||Difficulties evaluate much bad firms. Pub|2.60|1.97|5004002|edu packscholar #2|4|classical|5|Music|150|barantiought|N/A|66287tomato148305757|steel|Box|Unknown|73|oughtableableeing| +8222|AAAAAAAAOBACAAAA|1997-10-27|2000-10-26|Blue areas may not go inc temperatures. Sole, responsible standards follow females. Different, lit|6.71|2.61|8010003|univmaxi #3|10|pools|8|Sports|279|n stationable|N/A|03505985615puff60847|grey|Bunch|Unknown|21|ableableableeing| +8223|AAAAAAAAOBACAAAA|2000-10-27||Blue areas may not go inc temperatures. Sole, responsible standards follow females. Different, lit|7.00|2.61|4002002|importoedu pack #2|2|mens|4|Shoes|374|n stationable|medium|03505985615puff60847|mint|Tsp|Unknown|71|priableableeing| +8224|AAAAAAAAACACAAAA|1997-10-27|1999-10-27|Rich, quick e|5.72|4.34|5001001|amalgscholar #1|1|rock|5|Music|642|ableesecally|N/A|234819thistle2035777|seashell|Tbl|Unknown|23|eseableableeing| +8225|AAAAAAAAACACAAAA|1999-10-28|2001-10-26|Rich, quick e|6.28|4.34|5004002|edu packscholar #2|1|classical|5|Music|642|ableesecally|N/A|234819thistle2035777|steel|Dozen|Unknown|55|antiableableeing| +8226|AAAAAAAAACACAAAA|2001-10-27||Light, british banks force too next to the services. Systems know widely to a deaths. Single rights order for the cuts. Scales specify too heavy germans; old questions must not bel|4.43|3.36|10002016|importounivamalg #16|2|camcorders|10|Electronics|642|ableesecally|N/A|291595113284saddle28|blush|Dram|Unknown|2|callyableableeing| +8227|AAAAAAAADCACAAAA|1997-10-27||Japanese beliefs go then sources. Candidates may serve above to |4.94|1.53|1001002|amalgamalg #2|1|dresses|1|Women|397|ationn stpri|medium|thistle2616471419131|tomato|Each|Unknown|11|ationableableeing| +8228|AAAAAAAAECACAAAA|1997-10-27|2000-10-26|Comments must not offer; valuable, annual centres shoul|9.51|7.89|8002009|importonameless #9|2|baseball|8|Sports|636|callyprically|N/A|5061snow440233905806|smoke|Ounce|Unknown|75|eingableableeing| +8229|AAAAAAAAECACAAAA|2000-10-27||Else medical discussions need. Socialist police may hear usually canadian buyers. Necessarily english symptoms block. External goods cannot claim once soon big females; close ki|4.65|7.89|8016010|corpmaxi #10|16|golf|8|Sports|636|callyprically|N/A|5061snow440233905806|peach|Unknown|Unknown|41|n stableableeing| +8230|AAAAAAAAGCACAAAA|1997-10-27|1999-10-27|Unlikely theories can believe popular pieces. New, pleased|59.78|27.49|3003001|exportiexporti #1|3|toddlers|3|Children|11|oughtought|medium|79126570698orange008|sky|Ton|Unknown|66|barpriableeing| +8231|AAAAAAAAGCACAAAA|1999-10-28|2001-10-26|Unlikely theories can believe popular pieces. New, pleased|79.62|26.27|3003001|maxibrand #2|3|mattresses|7|Home|11|oughtought|N/A|79126570698orange008|lace|Cup|Unknown|78|oughtpriableeing| +8232|AAAAAAAAGCACAAAA|2001-10-27||Unlikely theories can believe popular pieces. New, pleased|1.04|0.68|3003001|importoimporto #1|2|shirts|2|Men|345|oughtought|small|9khaki20587714536759|purple|Box|Unknown|15|ablepriableeing| +8233|AAAAAAAAJCACAAAA|1997-10-27||New, red savings could justify to the principles; even exact years ought to win so. Records ens|39.61|35.64|7006004|corpbrand #4|6|rugs|7|Home|13|priought|N/A|54violet800339763319|dim|Case|Unknown|49|pripriableeing| +8234|AAAAAAAAKCACAAAA|1997-10-27|2000-10-26|Hundreds might excuse heavy countries. So new attacks might integrate strong, light terms. L|9.64|5.88|4001001|amalgedu pack #1|1|womens|4|Shoes|256|callyantiable|petite|0452honeydew22713235|slate|Box|Unknown|35|esepriableeing| +8235|AAAAAAAAKCACAAAA|2000-10-27||Ways make especially central advantages. Letters must keep well special bags. Specific populations used to examine then. Remarkably following questions should improve clinical |3.73|5.88|4001001|importoamalgamalg #8|1|monitors|10|Electronics|375|antiationpri|N/A|0452honeydew22713235|orange|Bunch|Unknown|82|antipriableeing| +8236|AAAAAAAAMCACAAAA|1997-10-27|1999-10-27|Critical, far groups should go police; well only institutions mean labour agents. Thin birds result there |3.34|1.70|5004001|edu packscholar #1|4|classical|5|Music|133|pripriought|N/A|256852681799ivory669|lemon|Tbl|Unknown|55|callypriableeing| +8237|AAAAAAAAMCACAAAA|1999-10-28|2001-10-26|Feet want far experienced, other devices. Long lines meet also other feet; other, only operations help together brief needs. Parties hold so firmly you|8.48|2.71|9005006|scholarmaxi #6|5|history|9|Books|133|pripriought|N/A|tomato31384929971223|khaki|Dozen|Unknown|63|ationpriableeing| +8238|AAAAAAAAMCACAAAA|2001-10-27||Feet want far experienced, other devices. Long lines meet also other feet; other, only operations help together brief needs. Parties hold so firmly you|4.50|2.38|9005006|amalgimporto #1|5|accessories|2|Men|133|pripriought|petite|tomato31384929971223|turquoise|Pound|Unknown|72|eingpriableeing| +8239|AAAAAAAAPCACAAAA|1997-10-27||Designs manipulate managers. Orga|2.70|0.91|5001002|amalgscholar #2|1|rock|5|Music|142|ableeseought|N/A|104227444puff6462186|pale|Tsp|Unknown|28|n stpriableeing| +8240|AAAAAAAAADACAAAA|1997-10-27|2000-10-26|Rich, nuclear ideas bear however religious words. More weak books should locate else. Neighbours look on a things. Good consumers take inside. Whit|8.39|6.62|6009001|maxicorp #1|9|womens watch|6|Jewelry|97|ationn st|N/A|82604903248seashell3|saddle|Pallet|Unknown|48|bareseableeing| +8241|AAAAAAAAADACAAAA|2000-10-27||Indus|3.91|1.48|6009001|importoscholar #2|2|country|5|Music|97|ationn st|N/A|33357891842linen3299|ivory|Carton|Unknown|86|oughteseableeing| +8242|AAAAAAAACDACAAAA|1997-10-27|1999-10-27|Females used |5.61|4.60|1002001|importoamalg #1|2|fragrances|1|Women|298|eingn stable|medium|918982008papaya31244|papaya|Ton|Unknown|1|ableeseableeing| +8243|AAAAAAAACDACAAAA|1999-10-28|2001-10-26|Hours increase even rich, irish wheels. Central others shall get more on the ways. Wonderful, able sounds shall want then high chapters. Apparent, appropriate questions think shares. P|3.36|4.60|1002001|brandbrand #4|7|decor|7|Home|298|eingn stable|N/A|9630423721green77722|spring|Each|Unknown|23|prieseableeing| +8244|AAAAAAAACDACAAAA|2001-10-27||Friends must sense live official trees. Existing germans decide views. Even difficult animals can say then old, public doctors. Prog|8.34|4.25|1002001|importonameless #5|2|baseball|8|Sports|298|eingn stable|N/A|69277998324sienna839|sienna|Oz|Unknown|31|eseeseableeing| +8245|AAAAAAAAFDACAAAA|1997-10-27||Essential memories continue dreams; average places administer respons|4.50|1.80|8015006|scholarmaxi #6|15|fishing|8|Sports|614|eseoughtcally|N/A|118rose9819225624010|peach|N/A|Unknown|17|antieseableeing| +8246|AAAAAAAAGDACAAAA|1997-10-27|2000-10-26|Children convince just to the miners. Prime, considerable components shall explain policies; single tasks know just all the same long shares; known police shall not draw co|7.20|6.19|10011002|amalgamalgamalg #2|11|disk drives|10|Electronics|243|prieseable|N/A|535116082694ghost718|lemon|Bunch|Unknown|85|callyeseableeing| +8247|AAAAAAAAGDACAAAA|2000-10-27||Children convince just to the miners. Prime, considerable components shall explain policies; single tasks know just all the same long shares; known police shall not draw co|3.25|6.19|9003004|exportimaxi #4|3|computers|9|Books|243|prieseable|N/A|8powder7943520846187|thistle|Carton|Unknown|57|ationeseableeing| +8248|AAAAAAAAIDACAAAA|1997-10-27|1999-10-27|Even splendid scots might not venture of course above new drawings. Companies occur french fields. Golden|7.72|5.79|10014004|edu packamalgamalg #4|14|automotive|10|Electronics|25|antiable|N/A|40366678732slate4732|spring|N/A|Unknown|23|eingeseableeing| +8249|AAAAAAAAIDACAAAA|1999-10-28|2001-10-26|Small, possible materials comfort even recent flowers. Important, vast claims kill; catholic, necessary systems could put fundamentally. Only duties see also little, late months. Only ce|6.40|5.79|10014004|corpmaxi #6|14|parenting|9|Books|411|antiable|N/A|415855indian24265690|indian|Gross|Unknown|5|n steseableeing| +8250|AAAAAAAAIDACAAAA|2001-10-27||Hands ought to become of course really different examples; free proposals shall consider previously results. Little questions take even so outer accidents. Again imperial years win to a|9.79|3.52|3004001|edu packexporti #1|4|school-uniforms|3|Children|5|anti|large|9378sky3246812044840|spring|N/A|Unknown|33|barantiableeing| +8251|AAAAAAAALDACAAAA|1997-10-27||Otherwise local relations would fly between a women. Whole costs make even from the types|0.62|0.31|9008008|namelessmaxi #8|8|romance|9|Books|82|ableeing|N/A|518094moccasin656311|linen|Unknown|Unknown|28|oughtantiableeing| +8252|AAAAAAAAMDACAAAA|1997-10-27|2000-10-26|Due, beautiful classes matter virtually however quick pupils. New, wide years might let to a terms. Real, gradual children shall not die hard cases. Values canno|9.06|5.34|3003001|exportiexporti #1|3|toddlers|3|Children|116|callyoughtought|large|1peach40825080322553|snow|Pallet|Unknown|28|ableantiableeing| +8253|AAAAAAAAMDACAAAA|2000-10-27||Due, beautiful classes matter virtually however quick pupils. New, wide years might let to a terms. Real, gradual children shall not die hard cases. Values canno|1.80|5.34|6013006|exportibrand #6|3|loose stones|6|Jewelry|130|callyoughtought|N/A|1peach40825080322553|lime|Carton|Unknown|95|priantiableeing| +8254|AAAAAAAAODACAAAA|1997-10-27|1999-10-27|However wild beliefs|3.91|3.24|7011005|amalgnameless #5|11|accent|7|Home|198|eingn stought|N/A|9297navy320405388213|purple|Unknown|Unknown|4|eseantiableeing| +8255|AAAAAAAAODACAAAA|1999-10-28|2001-10-26|However wild beliefs|8.60|3.24|7011005|edu packcorp #8|11|bracelets|6|Jewelry|28|eingable|N/A|9297navy320405388213|saddle|Tbl|Unknown|48|antiantiableeing| +8256|AAAAAAAAODACAAAA|2001-10-27||Beautiful, dark deaths ma|5.70|2.39|7011005|brandmaxi #1|7|reference|9|Books|49|n stese|N/A|7412seashell24016402|saddle|Bundle|Unknown|48|callyantiableeing| +8257|AAAAAAAABEACAAAA|1997-10-27||Good police shall add films. Most right signs ought to make better by the worke|6.48|4.66|10005013|scholarunivamalg #13|5|karoke|10|Electronics|596|callyn stanti|N/A|792goldenrod22056648|pink|Carton|Unknown|27|ationantiableeing| +8258|AAAAAAAACEACAAAA|1997-10-27|2000-10-26|Other, direct letters ought to make from a ways. British, large men could not work a|0.48|0.31|9002003|importomaxi #3|2|business|9|Books|125|antiableought|N/A|03358507319425beige0|cornsilk|Box|Unknown|10|eingantiableeing| +8259|AAAAAAAACEACAAAA|2000-10-27||Off|0.93|0.31|5001002|amalgscholar #2|1|rock|5|Music|483|prieingese|N/A|68moccasin8495519079|forest|Carton|Unknown|91|n stantiableeing| +8260|AAAAAAAAEEACAAAA|1997-10-27|1999-10-27|Fun, new beds would sleep widely; political, political years play. Eyes might not trace |1.04|0.54|3001001|amalgexporti #1|1|newborn|3|Children|220|barableable|large|51navy96518558471167|misty|Dram|Unknown|20|barcallyableeing| +8261|AAAAAAAAEEACAAAA|1999-10-28|2001-10-26|Happy, public problems explore old-fashioned, recent marks. Able companies will not imagine accordingly able grounds; british, digital respondents|5.79|2.89|3001001|edu packimporto #2|1|sports-apparel|2|Men|185|antieingought|petite|51navy96518558471167|blue|Oz|Unknown|43|oughtcallyableeing| +8262|AAAAAAAAEEACAAAA|2001-10-27||Also prime applications used to see closely. Easy women need in a problems. Key children become; cities stop by a techniques. French, high groups will|5.32|2.97|3001001|corpbrand #7|16|consignment|6|Jewelry|570|antieingought|N/A|51navy96518558471167|purple|Cup|Unknown|4|ablecallyableeing| +8263|AAAAAAAAHEACAAAA|1997-10-27||Physical, common others could not pronounce then banks. Good women perpetuate only in a minutes. Great systems explain never maybe pu|1.56|1.13|2001002|amalgimporto #2|1|accessories|2|Men|218|eingoughtable|medium|9goldenrod5574942702|plum|Case|Unknown|23|pricallyableeing| +8264|AAAAAAAAIEACAAAA|1997-10-27|2000-10-26|Also outside areas may not hear. More distant members can love other pounds. Surprising boundaries must say enough separate, delicious evenings; social, other feet would approach. Important |0.75|0.31|6008001|namelesscorp #1|8|mens watch|6|Jewelry|952|ableantin st|N/A|5767purple6375139118|purple|Oz|Unknown|27|esecallyableeing| +8265|AAAAAAAAIEACAAAA|2000-10-27||Also outside areas may not hear. More distant members can love other pounds. Surprising boundaries must say enough separate, delicious evenings; social, other feet would approach. Important |3.86|0.31|6008001|importoedu pack #2|8|mens|4|Shoes|952|ableantin st|extra large|5767purple6375139118|rosy|Ounce|Unknown|4|anticallyableeing| +8266|AAAAAAAAKEACAAAA|1997-10-27|1999-10-27|Both civil difficulties make weekly eyes; big benefits reach at a messages. Either civil deaths declare. Running prisons rea|13.76|4.81|3003001|exportiexporti #1|3|toddlers|3|Children|297|ationn stable|large|27072599325625snow30|pink|Lb|Unknown|36|callycallyableeing| +8267|AAAAAAAAKEACAAAA|1999-10-28|2001-10-26|Pure, slight names can see eith|1.58|0.90|6012004|importobrand #4|3|costume|6|Jewelry|297|ationn stable|N/A|27072599325625snow30|gainsboro|Tbl|Unknown|16|ationcallyableeing| +8268|AAAAAAAAKEACAAAA|2001-10-27||Di|4.01|1.72|6012004|edu packnameless #9|3|glassware|7|Home|297|ationn stable|N/A|6722518699862peru333|indian|Box|Unknown|75|eingcallyableeing| +8269|AAAAAAAANEACAAAA|1997-10-27||Specific, economic cells will find so e|47.12|16.49|5002002|importoscholar #2|2|country|5|Music|197|ationn stought|N/A|5642011sandy91493691|puff|Ton|Unknown|64|n stcallyableeing| +8270|AAAAAAAAOEACAAAA|1997-10-27|2000-10-26||||||3||2||325|antiablepri||3seashell03417988011||Cup|Unknown||barationableeing| +8271|AAAAAAAAOEACAAAA|2000-10-27||Partial applications would ensure forever with a members; young parents should take perhaps. Difficult roles|1.90|0.83|6005008|scholarcorp #8|3|earings|6|Jewelry|93|antiablepri|N/A|5752306961pink361833|pale|Case|Unknown|35|oughtationableeing| +8272|AAAAAAAAAFACAAAA|1997-10-27|1999-10-27|Excellent, major years make widely regular arms. Very old patients must represent to the sales. Regular women can g|82.77|27.31|2002001|importoimporto #1|2|shirts|2|Men|601|oughtbarcally|petite|2440203metallic74679|steel|Gram|Unknown|24|ableationableeing| +8273|AAAAAAAAAFACAAAA|1999-10-28|2001-10-26|Silent, other cells used t|1.05|0.59|6014002|edu packbrand #2|2|estate|6|Jewelry|212|ableoughtable|N/A|2440203metallic74679|yellow|Dozen|Unknown|33|priationableeing| +8274|AAAAAAAAAFACAAAA|2001-10-27||Widely ridiculous movements might play. Active, blind styles join particularly present farms. So hard min|9.10|0.59|10016013|corpamalgamalg #13|16|wireless|10|Electronics|212|ableoughtable|N/A|2440203metallic74679|sky|Oz|Unknown|12|eseationableeing| +8275|AAAAAAAADFACAAAA|1997-10-27||Able, full elections see then for instance real categories. Constitutional trades would go around merchants. Once appointed individuals shall take|2.12|1.25|5001002|amalgscholar #2|1|rock|5|Music|331|oughtpripri|N/A|619708444649steel888|spring|Dozen|Unknown|25|antiationableeing| +8276|AAAAAAAAEFACAAAA|1997-10-27|2000-10-26|Full-time, british democrats minimise about. Successful accidents used to respect once normally political notes; paint|6.97|2.78|1004001|edu packamalg #1|4|swimwear|1|Women|159|n stantiought|large|266618824rose9392577|hot|Bunch|Unknown|84|callyationableeing| +8277|AAAAAAAAEFACAAAA|2000-10-27||Full-time, british democrats minimise about. Successful accidents used to respect once normally political notes; paint|3.73|1.64|2002002|importoimporto #2|4|shirts|2|Men|109|n stbarought|large|33623218991373red041|bisque|Unknown|Unknown|32|ationationableeing| +8278|AAAAAAAAGFACAAAA|1997-10-27|1999-10-27|Particular benefits show only. Human, dead facts used to make boundaries. Likely, irish proposals walk during a societies. Eggs may look e|8.10|4.61|3002001|importoexporti #1|2|infants|3|Children|266|callycallyable|large|green226075353405126|lemon|Gross|Unknown|75|eingationableeing| +8279|AAAAAAAAGFACAAAA|1999-10-28|2001-10-26|Differences include yet dramatically difficult |6.24|4.61|3002001|edu packamalgamalg #1|2|automotive|10|Electronics|266|callycallyable|N/A|green226075353405126|saddle|Tsp|Unknown|18|n stationableeing| +8280|AAAAAAAAGFACAAAA|2001-10-27||Differences include yet dramatically difficult |0.79|0.41|8015001|scholarmaxi #1|2|fishing|8|Sports|141|callycallyable|N/A|green226075353405126|yellow|Lb|Unknown|51|bareingableeing| +8281|AAAAAAAAJFACAAAA|1997-10-27||Mistakes prove slowly most big companies. Eggs make even in a relations. Heavily little crops reach in a procedures. New, nuclear deposits reduce even of|4.93|2.85|9016002|corpunivamalg #2|16|mystery|9|Books|179|n stationought|N/A|47958red271873066584|steel|Carton|Unknown|24|oughteingableeing| +8282|AAAAAAAAKFACAAAA|1997-10-27|2000-10-26|Racial times give silent eggs. Political areas can see also. Here basic criticisms avoid unfortunately dirty months. Chief, spanish men know|2.69|1.15|2004001|edu packimporto #1|4|sports-apparel|2|Men|18|eingought|large|41679157812pink70360|salmon|Bunch|Unknown|14|ableeingableeing| +8283|AAAAAAAAKFACAAAA|2000-10-27||Fashionable concentrations close. |2.77|2.24|2004001|corpnameless #4|4|football|8|Sports|18|eingought|N/A|41679157812pink70360|snow|Gram|Unknown|21|prieingableeing| +8284|AAAAAAAAMFACAAAA|1997-10-27|1999-10-27|Clear subjects kiss always silver proje|9.97|3.29|8005005|scholarnameless #5|5|fitness|8|Sports|228|eingableable|N/A|7145510094442powder9|pink|Pallet|Unknown|1|eseeingableeing| +8285|AAAAAAAAMFACAAAA|1999-10-28|2001-10-26|Normal value|6.88|3.29|1002002|importoamalg #2|2|fragrances|1|Women|695|antin stcally|large|7145510094442powder9|lavender|Case|Unknown|52|antieingableeing| +8286|AAAAAAAAMFACAAAA|2001-10-27||Normal value|8.07|5.32|10009008|maxiunivamalg #8|9|televisions|10|Electronics|77|ationation|N/A|7145510094442powder9|smoke|Bundle|Unknown|1|callyeingableeing| +8287|AAAAAAAAPFACAAAA|1997-10-27||Twice part-time songs will end certainly free charges. Schools would make particularly terms; more fresh services change too. Books may secure in order artists; students should look right tired at|5.32|2.71|9009008|maximaxi #8|9|science|9|Books|823|priableeing|N/A|75pale06379811918512|smoke|Gross|Unknown|68|ationeingableeing| +8288|AAAAAAAAAGACAAAA|1997-10-27|2000-10-26|At least different intervals shall throw slowly phases. Small jobs can meet. Previous, new photographs try on a others; problems invest makers. Much different things sustain low ca|3.27|1.11|6003001|exporticorp #1|3|gold|6|Jewelry|616|callyoughtcally|N/A|2825ivory85957317362|snow|N/A|Unknown|50|eingeingableeing| +8289|AAAAAAAAAGACAAAA|2000-10-27||Friends meet most; immediat|5.58|3.29|6003001|amalgamalg #2|1|dresses|1|Women|529|n stableanti|medium|2825ivory85957317362|papaya|Dozen|Unknown|49|n steingableeing| +8290|AAAAAAAACGACAAAA|1997-10-27|1999-10-27|Public, extensive plans must not proceed severely with the requirements. Legal, junior matters should receive; subsequent, intense words see in the ideas; major resources need long effe|70.53|35.97|5001001|amalgscholar #1|1|rock|5|Music|216|callyoughtable|N/A|461sky47811626109931|powder|Pallet|Unknown|1|barn stableeing| +8291|AAAAAAAACGACAAAA|1999-10-28|2001-10-26|Public, extensive plans must not proceed severely with the requirements. Legal, junior matters should receive; subsequent, intense words see in the ideas; major resources need long effe|6.61|35.97|4003002|exportiedu pack #2|3|kids|4|Shoes|21|oughtable|extra large|54047salmon790623663|pale|Ounce|Unknown|10|oughtn stableeing| +8292|AAAAAAAACGACAAAA|2001-10-27||Public, extensive plans must not proceed severely with the requirements. Legal, junior matters should receive; subsequent, intense words see in the ideas; major resources need long effe|9.25|35.97|4003002|edu packexporti #1|3|school-uniforms|3|Children|404|oughtable|small|54047salmon790623663|thistle|Each|Unknown|16|ablen stableeing| +8293|AAAAAAAAFGACAAAA|1997-10-27||Fe|8.45|5.23|9016002|corpunivamalg #2|16|mystery|9|Books|129|n stableought|N/A|7964241318orange9819|yellow|Lb|Unknown|5|prin stableeing| +8294|AAAAAAAAGGACAAAA|1997-10-27|2000-10-26|C|8.64|2.93|2001001|amalgimporto #1|1|accessories|2|Men|372|ableationpri|medium|82puff05454759948940|violet|N/A|Unknown|51|esen stableeing| +8295|AAAAAAAAGGACAAAA|2000-10-27||C|30.93|2.93|9005004|scholarmaxi #4|5|history|9|Books|372|ableationpri|N/A|38737745green2020974|tan|Bundle|Unknown|93|antin stableeing| +8296|AAAAAAAAIGACAAAA|1997-10-27|1999-10-27|Waves may check very so various subsidies. Only political scales may refrain also. Panels |4.61|2.44|5003001|exportischolar #1|3|pop|5|Music|44|eseese|N/A|979747cornsilk954248|wheat|Cup|Unknown|28|callyn stableeing| +8297|AAAAAAAAIGACAAAA|1999-10-28|2001-10-26|Additional remarks can see themes. Short, soft techniques interpret at once single implications. Doctors can tend wide accurately following subjects. Approaches ought to deliver almost other terms. Le|5.18|2.44|3004002|edu packexporti #2|4|school-uniforms|3|Children|44|eseese|medium|4965092017020saddle6|rosy|Case|Unknown|1|ationn stableeing| +8298|AAAAAAAAIGACAAAA|2001-10-27||Additional remarks can see themes. Short, soft techniques interpret at once single implications. Doctors can tend wide accurately following subjects. Approaches ought to deliver almost other terms. Le|6.31|2.44|1001001|amalgamalg #1|1|dresses|1|Women|425|eseese|large|036434822650navajo09|grey|Tbl|Unknown|10|eingn stableeing| +8299|AAAAAAAALGACAAAA|1997-10-27||Hurriedly strong women may change legs. Useful recordings take recently cautious, numerous opportunities. Shares should secure probably simple, other approaches. Things would file; for|4.06|2.51|5004002|edu packscholar #2|4|classical|5|Music|115|antioughtought|N/A|white466704142867680|purple|Pound|Unknown|40|n stn stableeing| +8300|AAAAAAAAMGACAAAA|1997-10-27|2000-10-26|Early qualities shall not get of course there personal children. Significant intervals could no|7.16|5.87|5002001|importoscholar #1|2|country|5|Music|290|barn stable|N/A|61784048193444tan515|turquoise|Ounce|Unknown|22|barbarprieing| +8301|AAAAAAAAMGACAAAA|2000-10-27||Early qualities shall not get of course there personal children. Significant intervals could no|62.48|5.87|5002001|amalgscholar #2|1|rock|5|Music|52|ableanti|N/A|646445654486692pink4|smoke|Dozen|Unknown|18|oughtbarprieing| +8302|AAAAAAAAOGACAAAA|1997-10-27|1999-10-27|Prices might say open, local agreements. Pu|0.74|0.26|3004001|edu packexporti #1|4|school-uniforms|3|Children|147|ationeseought|N/A|54787527orange719070|sienna|Case|Unknown|78|ablebarprieing| +8303|AAAAAAAAOGACAAAA|1999-10-28|2001-10-26|Prices might say open, local agreements. Pu|4.72|0.26|7015004|scholarnameless #4|15|tables|7|Home|497|ationeseought|N/A|141979673sandy198821|rosy|Carton|Unknown|9|pribarprieing| +8304|AAAAAAAAOGACAAAA|2001-10-27||British items would not mention so messages. Immediately unpleasant months can get recently for example broken areas. Most interesting guests clear big wheels. Together national ty|8.06|5.23|7015004|edu packimporto #1|15|sports-apparel|2|Men|497|ationeseought|extra large|05264535purple437441|rosy|N/A|Unknown|77|esebarprieing| +8305|AAAAAAAABHACAAAA|1997-10-27||Red numbers visualize major, difficult universities. Traditional, british members must get completely related, advisory children. Especially concerned problems must not take perhaps ne|6.37|2.42|2003002|exportiimporto #2|3|pants|2|Men|732|ablepriation|medium|513612plum1156354196|slate|Pallet|Unknown|31|antibarprieing| +8306|AAAAAAAACHACAAAA|1997-10-27|2000-10-26|Chief, medical centres may lead ready, national cours|3.69|1.58|5004001|edu packscholar #1|4|classical|5|Music|489|n steingese|N/A|706white602842573072|wheat|Oz|Unknown|40|callybarprieing| +8307|AAAAAAAACHACAAAA|2000-10-27||Main, goo|0.24|0.12|6013004|exportibrand #4|13|loose stones|6|Jewelry|489|n steingese|N/A|825maroon81655153891|thistle|Dozen|Unknown|32|ationbarprieing| +8308|AAAAAAAAEHACAAAA|1997-10-27|1999-10-27|New authorities shall mount early important, prime children; straight economic ways ride over direct ideas. Behind white states treat so main countries. Traditional, |2.31|0.71|2002001|importoimporto #1|2|shirts|2|Men|8|eing|medium|989284502385olive503|sandy|Dram|Unknown|20|eingbarprieing| +8309|AAAAAAAAEHACAAAA|1999-10-28|2001-10-26|More expected databases will not add unusually. Industrial, large parents will n|6.75|0.71|2002001|importobrand #10|2|bedding|7|Home|8|eing|N/A|yellow24702695299178|pink|Gram|Unknown|20|n stbarprieing| +8310|AAAAAAAAEHACAAAA|2001-10-27||Sick problems may leave naturally here central notes; clear texts monitor extremely for a islands. Sites cook however neutral, leading women. Countries shall not expect. Carers used to g|21.85|10.26|2002001|edu packexporti #1|4|school-uniforms|3|Children|8|eing|petite|425365207022royal030|green|Gram|Unknown|1|baroughtprieing| +8311|AAAAAAAAHHACAAAA|1997-10-27||Also constant men would not go already often only rivers. Local powers would com|0.22|0.17|10012016|importoamalgamalg #16|12|monitors|10|Electronics|227|ationableable|N/A|5654061206smoke28926|saddle|Oz|Unknown|46|oughtoughtprieing| +8312|AAAAAAAAIHACAAAA|1997-10-27|2000-10-26|Only lovely electio|44.33|28.81|8014003|edu packmaxi #3|14|tennis|8|Sports|262|ablecallyable|N/A|737341slate946017326|steel|Bundle|Unknown|2|ableoughtprieing| +8313|AAAAAAAAIHACAAAA|2000-10-27||Right pounds must restore perfectly middle waters. However young approaches push late nervous, only scots. Seriously high families sell badly so complex years. Low, future parts must not fill. Q|3.32|28.81|8014003|amalgcorp #2|14|birdal|6|Jewelry|262|ablecallyable|N/A|373335669977almond75|tan|Tsp|Unknown|57|prioughtprieing| +8314|AAAAAAAAKHACAAAA|1997-10-27|1999-10-27|Terrible, national events can see still recent, technical eyes. High terms will work early flowers. Only empty events reach now e|5.77|4.15|3004001|edu packexporti #1|4|school-uniforms|3|Children|601|oughtbarcally|medium|783snow3067793407456|rosy|Gram|Unknown|1|eseoughtprieing| +8315|AAAAAAAAKHACAAAA|1999-10-28|2001-10-26|Great, acceptable governments expect recently complicated plants. Additional emotions can take. Inside fresh lines can en|3.75|1.23|2003002|exportiimporto #2|3|pants|2|Men|639|oughtbarcally|large|moccasin892502293078|ivory|Pound|Unknown|66|antioughtprieing| +8316|AAAAAAAAKHACAAAA|2001-10-27||Great, acceptable governments expect recently complicated plants. Additional emotions can take. Inside fresh lines can en|5.40|3.61|8002007|importonameless #7|3|baseball|8|Sports|639|n stprically|N/A|40833thistle96208838|wheat|Oz|Unknown|23|callyoughtprieing| +8317|AAAAAAAANHACAAAA|1997-10-27||Pages might not gain again pounds. |9.57|7.46|5004002|edu packscholar #2|4|classical|5|Music|42|ableese|N/A|light895025686169471|papaya|Gross|Unknown|31|ationoughtprieing| +8318|AAAAAAAAOHACAAAA|1997-10-27|2000-10-26|Careful clubs register always si|28.17|20.00|1001001|amalgamalg #1|1|dresses|1|Women|297|ationn stable|small|345107376slate221368|linen|Lb|Unknown|20|eingoughtprieing| +8319|AAAAAAAAOHACAAAA|2000-10-27||Careful clubs register always si|4.74|20.00|6013008|exportibrand #8|1|loose stones|6|Jewelry|147|ationeseought|N/A|345107376slate221368|puff|Tbl|Unknown|65|n stoughtprieing| +8320|AAAAAAAAAIACAAAA|1997-10-27|1999-10-27|Households ask little. Exactly usual levels interfere. Pure, dark claims provide. Clever men shall soar yesterday red photographs. Bare drinks would|8.50|6.88|1002001|importoamalg #1|2|fragrances|1|Women|303|pribarpri|small|312842white555567975|navy|Cup|Unknown|15|barableprieing| +8321|AAAAAAAAAIACAAAA|1999-10-28|2001-10-26|Tenants will not see properly to the actions. Seconds might get for a findings; difficulties shall knock more than states. Previously common women would not realise economic, solid d|4.67|2.19|1002001|exportiimporto #2|3|pants|2|Men|178|eingationought|small|312842white555567975|powder|Ton|Unknown|57|oughtableprieing| +8322|AAAAAAAAAIACAAAA|2001-10-27||Old, left quantities must understand normal, male charges. Raw committees employ very special, possible members. Sure other companies investigate politicians. Matters matter tog|3.64|2.19|1002001|amalgscholar #1|1|rock|5|Music|178|eingationought|N/A|7753red2470035971645|spring|Ounce|Unknown|27|ableableprieing| +8323|AAAAAAAADIACAAAA|1997-10-27||Professional services can resume in common alone parlia|6.20|4.52|4001002|amalgedu pack #2|1|womens|4|Shoes|89|n steing|medium|52872navy00036882246|snow|Pallet|Unknown|27|priableprieing| +8324|AAAAAAAAEIACAAAA|1997-10-27|2000-10-26|Other, slim days try |6.22|2.30|7001005|amalgbrand #5|1|bathroom|7|Home|103|pribarought|N/A|2slate78223897622079|slate|Bundle|Unknown|4|eseableprieing| +8325|AAAAAAAAEIACAAAA|2000-10-27||Other, slim days try |7.36|6.18|7001005|scholarunivamalg #13|5|karoke|10|Electronics|103|pribarought|N/A|2slate78223897622079|ghost|Gross|Unknown|12|antiableprieing| +8326|AAAAAAAAGIACAAAA|1997-10-27|1999-10-27|Regular motives gain only just public societies. Upwards little men live issues. Changes may not like elderly ministers. Observations ought to see|5.06|3.79|4002001|importoedu pack #1|2|mens|4|Shoes|388|eingeingpri|petite|85411864sky365174169|burnished|Cup|Unknown|79|callyableprieing| +8327|AAAAAAAAGIACAAAA|1999-10-28|2001-10-26|Regular motives gain only just public societies. Upwards little men live issues. Changes may not like elderly ministers. Observations ought to see|3.78|3.79|2003002|exportiimporto #2|2|pants|2|Men|186|callyeingought|economy|91520plum09854787461|midnight|Oz|Unknown|33|ationableprieing| +8328|AAAAAAAAGIACAAAA|2001-10-27||Countries develop very by |4.80|3.79|4001001|amalgedu pack #1|1|womens|4|Shoes|374|eseationpri|small|0767808snow443603848|spring|Pallet|Unknown|80|eingableprieing| +8329|AAAAAAAAJIACAAAA|1997-10-27||Current, certain clubs supply white, old proceedings. Safe brothers must not imagine especially.|9.86|8.47|3002002|importoexporti #2|2|infants|3|Children|627|ationablecally|small|16401892438199pale46|peach|Unknown|Unknown|64|n stableprieing| +8330|AAAAAAAAKIACAAAA|1997-10-27|2000-10-26|Then different matters shall not dare legally british pupils. Detailed, royal chapters must not mention quite in the sites. Costs take reasonably remote students. Systems return only now interesting |2.55|2.21|8004001|edu packnameless #1|4|camping|8|Sports|332|ablepripri|N/A|0991757hot4053065024|orchid|Gross|Unknown|58|barpriprieing| +8331|AAAAAAAAKIACAAAA|2000-10-27||Then different matters shall not dare legally british pupils. Detailed, royal chapters must not mention quite in the sites. Costs take reasonably remote students. Systems return only now interesting |3.71|2.21|8004001|amalgbrand #2|1|bathroom|7|Home|193|prin stought|N/A|0991757hot4053065024|thistle|Dram|Unknown|67|oughtpriprieing| +8332|AAAAAAAAMIACAAAA|1997-10-27|1999-10-27|Simple, powerful efforts may like |4.81|2.88|8006003|corpnameless #3|6|football|8|Sports|265|anticallyable|N/A|544172wheat673303390|sienna|Carton|Unknown|33|ablepriprieing| +8333|AAAAAAAAMIACAAAA|1999-10-28|2001-10-26|Simple, powerful efforts may like |3.92|2.07|8006003|importoedu pack #2|2|mens|4|Shoes|265|anticallyable|petite|544172wheat673303390|plum|Case|Unknown|2|pripriprieing| +8334|AAAAAAAAMIACAAAA|2001-10-27||Also main boats cannot find also a little nati|17.87|11.61|8006003|scholarunivamalg #5|2|karoke|10|Electronics|265|anticallyable|N/A|544172wheat673303390|puff|Unknown|Unknown|59|esepriprieing| +8335|AAAAAAAAPIACAAAA|1997-10-27||Rather olympic friends give very successful attitudes. For instance busy men secure normally implications; new, noble values would imply other, wonderful persons. All classical children sign an|89.27|27.67|10004006|edu packunivamalg #6|4|audio|10|Electronics|266|callycallyable|N/A|72thistle59513104725|dodger|Dram|Unknown|46|antipriprieing| +8336|AAAAAAAAAJACAAAA|1997-10-27|2000-10-26|Occupational, linear circumstances must work never as jewish winners. Regional, statistical clubs boost especially at least actual clergy. Member|1.39|0.51|2004001|edu packimporto #1|4|sports-apparel|2|Men|629|n stablecally|large|669khaki387477491836|snow|Cup|Unknown|25|callypriprieing| +8337|AAAAAAAAAJACAAAA|2000-10-27||Occupational, linear circumstances must work never as jewish winners. Regional, statistical clubs boost especially at least actual clergy. Member|4.48|0.51|2002002|importoimporto #2|2|shirts|2|Men|629|n stablecally|petite|219steel737811611866|frosted|Dram|Unknown|47|ationpriprieing| +8338|AAAAAAAACJACAAAA|1997-10-27|1999-10-27|Even administrative parties should spend customs. Mothers can make sometimes now model governments. National, full dogs know notably both common chil|0.39|0.21|8015009|scholarmaxi #9|15|fishing|8|Sports|565|anticallyanti|N/A|07olive5309640708466|spring|Tsp|Unknown|39|eingpriprieing| +8339|AAAAAAAACJACAAAA|1999-10-28|2001-10-26|Still |2.53|0.21|8015009|namelesscorp #4|8|mens watch|6|Jewelry|45|antiese|N/A|4461222peru989854808|wheat|Case|Unknown|26|n stpriprieing| +8340|AAAAAAAACJACAAAA|2001-10-27||Complex, labour companies stay little exhibitions. Words will not turn at least weeks. Excellent things sleep far never various businesses. Probably real babies used to stress as distinct professi|3.13|1.97|8007001|brandnameless #1|8|hockey|8|Sports|45|antiese|N/A|4461222peru989854808|rosy|Each|Unknown|63|bareseprieing| +8341|AAAAAAAAFJACAAAA|1997-10-27||Distinct proposal|4.79|2.44|10006012|corpunivamalg #12|6|musical|10|Electronics|166|callycallyought|N/A|7papaya8749677319668|turquoise|Gross|Unknown|8|oughteseprieing| +8342|AAAAAAAAGJACAAAA|1997-10-27|2000-10-26|More psychiatric standards make at all other, changing customers. M|3.70|1.88|3001001|amalgexporti #1|1|newborn|3|Children|40|barese|small|1khaki82911060142383|wheat|N/A|Unknown|46|ableeseprieing| +8343|AAAAAAAAGJACAAAA|2000-10-27||Recent, financial fields recommend for the blues. Marine o|4.06|2.07|3001001|importoexporti #2|2|infants|3|Children|241|barese|small|0353797310277peach11|red|Oz|Unknown|58|prieseprieing| +8344|AAAAAAAAIJACAAAA|1997-10-27|1999-10-27|Aims play already|39.50|23.30|6004001|edu packcorp #1|4|bracelets|6|Jewelry|216|callyoughtable|N/A|3011409purple8224346|seashell|Gram|Unknown|10|eseeseprieing| +8345|AAAAAAAAIJACAAAA|1999-10-28|2001-10-26|Aims play already|52.50|23.30|6004001|amalgimporto #2|4|accessories|2|Men|330|barpripri|N/A|85761209001saddle629|thistle|Oz|Unknown|22|antieseprieing| +8346|AAAAAAAAIJACAAAA|2001-10-27||Just soft lists correct public collections. Total, sure bonds would not take very words. Powerful minutes suit never spar|4.22|23.30|6004001|univamalgamalg #17|10|memory|10|Electronics|234|barpripri|N/A|9red9031953964436361|red|Tsp|Unknown|65|callyeseprieing| +8347|AAAAAAAALJACAAAA|1997-10-27||More co|0.39|0.22|5003002|exportischolar #2|3|pop|5|Music|368|eingcallypri|N/A|457orange17754461433|bisque|Bunch|Unknown|29|ationeseprieing| +8348|AAAAAAAAMJACAAAA|1997-10-27|2000-10-26|Narrowly other successes eat so. Only psychiatric systems develop hence. Engine|3.62|1.41|10003002|exportiunivamalg #2|3|dvd/vcr players|10|Electronics|255|antiantiable|N/A|pink1230359678516664|tomato|Gram|Unknown|42|eingeseprieing| +8349|AAAAAAAAMJACAAAA|2000-10-27||Narrowly other successes eat so. Only psychiatric systems develop hence. Engine|1.10|1.41|10003002|exportiunivamalg #3|3|dvd/vcr players|10|Electronics|145|antieseought|N/A|3070220mint334393649|puff|Pound|Unknown|60|n steseprieing| +8350|AAAAAAAAOJACAAAA|1997-10-27|1999-10-27|New, certain conditions use. Appropriate, good miles lift ne|8.68|5.98|9008011|namelessmaxi #11|8|romance|9|Books|308|eingbarpri|N/A|4529navajo5609900336|sienna|Pallet|Unknown|56|barantiprieing| +8351|AAAAAAAAOJACAAAA|1999-10-28|2001-10-26|Small, pretty bishops may not continue. Articles shall not set. Very special stages would not judge warm, old relations. Present allies cannot argue |9.00|5.98|9008011|edu packedu pack #2|8|athletic|4|Shoes|248|eingbarpri|petite|4529navajo5609900336|indian|Ton|Unknown|48|oughtantiprieing| +8352|AAAAAAAAOJACAAAA|2001-10-27||Free patients might not change really travellers. Days i|1.12|5.98|9008011|scholarmaxi #3|8|fishing|8|Sports|248|eingeseable|N/A|4529navajo5609900336|saddle|N/A|Unknown|39|ableantiprieing| +8353|AAAAAAAABKACAAAA|1997-10-27||Total pp. accept with a questions; able, generous a|5.25|1.99|9002002|importomaxi #2|2|business|9|Books|129|n stableought|N/A|47seashell6089243823|royal|Bundle|Unknown|27|priantiprieing| +8354|AAAAAAAACKACAAAA|1997-10-27|2000-10-26|Small, other parents cannot carry. Major police may believe for example. Hurriedly other styles develop then vertical refugees. Days respect commerc|7.29|4.66|10009008|maxiunivamalg #8|9|televisions|10|Electronics|624|eseablecally|N/A|1640180thistle202767|indian|Gram|Unknown|37|eseantiprieing| +8355|AAAAAAAACKACAAAA|2000-10-27||Upwards professional numbers shall not reveal detailed needs. Ho|3.46|2.97|8005006|scholarnameless #6|5|fitness|8|Sports|624|eseablecally|N/A|6065033lime534123662|powder|Cup|Unknown|10|antiantiprieing| +8356|AAAAAAAAEKACAAAA|1997-10-27|1999-10-27|Workers recognize including a temperatures. International towns shall agree afterwa|5.36|4.71|6015005|scholarbrand #5|15|custom|6|Jewelry|163|pricallyought|N/A|8199639266seashell10|ghost|Bunch|Unknown|6|callyantiprieing| +8357|AAAAAAAAEKACAAAA|1999-10-28|2001-10-26|Organic, small needs |3.30|1.51|6015005|importoexporti #2|2|infants|3|Children|365|anticallypri|small|3slate29119846934321|saddle|Box|Unknown|59|ationantiprieing| +8358|AAAAAAAAEKACAAAA|2001-10-27||Popular associations could speak. Services assist into a laws. Successful, defensive relationships cannot suggest never consistently wro|9.37|7.68|6015005|amalgamalg #1|2|dresses|1|Women|448|anticallypri|medium|3slate29119846934321|rosy|Oz|Unknown|91|eingantiprieing| +8359|AAAAAAAAHKACAAAA|1997-10-27||Powers shall produce never at a views. There special lives shall take positions. Emotional, following goods could open units; now un|4.96|3.17|10015013|scholaramalgamalg #13|15|portable|10|Electronics|577|ationationanti|N/A|08530981032361ivory5|white|Box|Unknown|25|n stantiprieing| +8360|AAAAAAAAIKACAAAA|1997-10-27|2000-10-26|Inches meet with a issu|4.98|2.09|5002001|importoscholar #1|2|country|5|Music|644|eseesecally|N/A|987627208774524peru0|light|Bunch|Unknown|2|barcallyprieing| +8361|AAAAAAAAIKACAAAA|2000-10-27||Matters master above. Confidential miles listen|5.61|2.46|5002001|exportiedu pack #2|3|kids|4|Shoes|570|barationanti|medium|987627208774524peru0|yellow|Unknown|Unknown|17|oughtcallyprieing| +8362|AAAAAAAAKKACAAAA|1997-10-27|1999-10-27|Minutes should ensure available millions. Bars reduc|2.69|2.04|2002001|importoimporto #1|2|shirts|2|Men|176|callyationought|medium|198640248215238red47|steel|Ounce|Unknown|34|ablecallyprieing| +8363|AAAAAAAAKKACAAAA|1999-10-28|2001-10-26|Minutes should ensure available millions. Bars reduc|0.12|0.06|2002001|importoedu pack #2|2|mens|4|Shoes|176|callyationought|large|721grey2002934021621|snow|Ounce|Unknown|4|pricallyprieing| +8364|AAAAAAAAKKACAAAA|2001-10-27||Minutes should ensure available millions. Bars reduc|4.50|0.06|2002001|importoexporti #1|2|infants|3|Children|145|antieseought|large|07227pale36130610789|seashell|Box|Unknown|51|esecallyprieing| +8365|AAAAAAAANKACAAAA|1997-10-27||Experiences go too relevant members. More large eyes get so ordinary shoulders. Still only manufacturers |19.13|13.96|5002002|importoscholar #2|2|country|5|Music|276|callyationable|N/A|3250linen97224065669|salmon|Case|Unknown|10|anticallyprieing| +8366|AAAAAAAAOKACAAAA|1997-10-27|2000-10-26|Grand firms sell sooner probably single months; generally typical copies occupy perhaps never large activities. Over unus|96.83|79.40|2002001|importoimporto #1|2|shirts|2|Men|125|antiableought|small|810846426powder51720|thistle|Each|Unknown|72|callycallyprieing| +8367|AAAAAAAAOKACAAAA|2000-10-27||Grand firms sell sooner probably single months; generally typical copies occupy perhaps never large activities. Over unus|2.54|0.88|1002002|importoamalg #2|2|fragrances|1|Women|355|antiableought|large|5saddle1500263640930|papaya|Cup|Unknown|2|ationcallyprieing| +8368|AAAAAAAAALACAAAA|1997-10-27|1999-10-27|Net, sensitive interests become. Communities shall test now metropolitan, favorable accounts; economic voices could elect wide drivers. Closely old-fashioned patients open differences. However spec|8.67|5.54|4001001|amalgedu pack #1|1|womens|4|Shoes|352|ableantipri|small|5270056139067pale610|lime|Dram|Unknown|47|eingcallyprieing| +8369|AAAAAAAAALACAAAA|1999-10-28|2001-10-26|Net, sensitive interests become. Communities shall test now metropolitan, favorable accounts; economic voices could elect wide drivers. Closely old-fashioned patients open differences. However spec|2.94|5.54|9012006|importounivamalg #6|12|home repair|9|Books|166|ableantipri|N/A|saddle85190188442174|wheat|Tsp|Unknown|43|n stcallyprieing| +8370|AAAAAAAAALACAAAA|2001-10-27||Companies make home via a problems; political, domestic issues arrest. Pounds might not retire effectively like supplies. Often clean regulations feel ostensibly police. General, democratic m|0.75|0.55|6003003|exporticorp #3|3|gold|6|Jewelry|152|ableantiought|N/A|191798645607wheat546|navajo|Bunch|Unknown|9|barationprieing| +8371|AAAAAAAADLACAAAA|1997-10-27||Big problems apply members. Faces must launch particul|3.33|2.89|2002002|importoimporto #2|2|shirts|2|Men|301|oughtbarpri|medium|90914129saddle506506|salmon|Pallet|Unknown|7|oughtationprieing| +8372|AAAAAAAAELACAAAA|1997-10-27|2000-10-26|Recent patients fill. Expensive studies used to feed recently mixed tears. Still simple relations exercise soon dull familie|6.45|2.58|1001001|amalgamalg #1|1|dresses|1|Women|1|ought|medium|96213554purple800271|sky|Pallet|Unknown|45|ableationprieing| +8373|AAAAAAAAELACAAAA|2000-10-27||Recent patients fill. Expensive studies used to feed recently mixed tears. Still simple relations exercise soon dull familie|7.30|6.49|1001001|maximaxi #10|1|science|9|Books|183|ought|N/A|purple49977086103635|saddle|Tsp|Unknown|38|priationprieing| +8374|AAAAAAAAGLACAAAA|1997-10-27|1999-10-27|Significant, gastric features share legally black families. Councils cover primarily national numbers. Strong notions ignore never only policies. Therefore l|4.39|3.55|4002001|importoedu pack #1|2|mens|4|Shoes|598|eingn stanti|medium|80836287tomato217619|moccasin|Box|Unknown|3|eseationprieing| +8375|AAAAAAAAGLACAAAA|1999-10-28|2001-10-26|Names act on a prisoners. Obvious, different words|2.30|1.67|3003002|exportiexporti #2|2|toddlers|3|Children|149|n steseought|extra large|84572546snow40971360|seashell|Dozen|Unknown|33|antiationprieing| +8376|AAAAAAAAGLACAAAA|2001-10-27||Mean years raise of course minimal disciplines. Duties get f|0.85|1.67|8004007|edu packnameless #7|4|camping|8|Sports|122|ableableought|N/A|84572546snow40971360|papaya|Bundle|Unknown|49|callyationprieing| +8377|AAAAAAAAJLACAAAA|1997-10-27||Well tiny gove|4.74|2.84|7004008|edu packbrand #8|4|curtains/drapes|7|Home|271|oughtationable|N/A|8174974984rosy201034|papaya|Box|Unknown|38|ationationprieing| +8378|AAAAAAAAKLACAAAA|1997-10-27|2000-10-26|Economic elements can expose however. Social organisations can use ea|2.38|0.97|7013009|exportinameless #9|13|wallpaper|7|Home|497|ationn stese|N/A|999saddle69981993145|linen|Tbl|Unknown|24|eingationprieing| +8379|AAAAAAAAKLACAAAA|2000-10-27||Economic elements can expose however. Social organisations can use ea|5.03|0.97|7013009|scholarbrand #10|5|blinds/shades|7|Home|497|ationn stese|N/A|46791896591913sky996|magenta|N/A|Unknown|33|n stationprieing| +8380|AAAAAAAAMLACAAAA|1997-10-27|1999-10-27|Years finance. Laws should not hear as hands. Al|52.42|23.06|10007017|brandunivamalg #17|7|personal|10|Electronics|29|n stable|N/A|091796khaki193096187|magenta|Case|Unknown|14|bareingprieing| +8381|AAAAAAAAMLACAAAA|1999-10-28|2001-10-26|Available preferences must trust now errors. Large publications would grow that is. Underlying, financial jobs shall develop medical, other crimes. Easily eventual skills|3.01|23.06|10007017|edu packedu pack #2|4|athletic|4|Shoes|295|antin stable|medium|091796khaki193096187|powder|Bunch|Unknown|58|oughteingprieing| +8382|AAAAAAAAMLACAAAA|2001-10-27||Available preferences must trust now errors. Large publications would grow that is. Underlying, financial jobs shall develop medical, other crimes. Easily eventual skills|9.69|6.49|8007003|brandnameless #3|7|hockey|8|Sports|299|antin stable|N/A|091796khaki193096187|tan|Pallet|Unknown|4|ableeingprieing| +8383|AAAAAAAAPLACAAAA|1997-10-27||Tanks need urban, busy areas. Owners want. Pretty years must trim afterwards entries; full, british results should|3.78|2.23|4004002|edu packedu pack #2|4|athletic|4|Shoes|277|ationationable|small|405purple96437673666|sky|Carton|Unknown|19|prieingprieing| +8384|AAAAAAAAAMACAAAA|1997-10-27|2000-10-26|Best national participants forget. Usually clear efforts can operate on|2.20|1.14|9011009|amalgunivamalg #9|11|cooking|9|Books|295|antin stable|N/A|57701675283purple701|smoke|Unknown|Unknown|91|eseeingprieing| +8385|AAAAAAAAAMACAAAA|2000-10-27||Candidates will not use never important delegates; typical types go ahead surely sorry s|4.51|1.14|9011010|amalgunivamalg #10|11|cooking|9|Books|254|antin stable|N/A|149825957powder47813|peru|Pound|Unknown|89|antieingprieing| +8386|AAAAAAAACMACAAAA|1997-10-27|1999-10-27|Circles say on the pairs. Political police support already level flats. Once more normal students may offend under a advantage|6.62|4.23|6007003|brandcorp #3|7|pendants|6|Jewelry|435|antipriese|N/A|589wheat947823419219|saddle|Dozen|Unknown|23|callyeingprieing| +8387|AAAAAAAACMACAAAA|1999-10-28|2001-10-26|Circles say on the pairs. Political police support already level flats. Once more normal students may offend under a advantage|6.44|4.23|6007003|exportiimporto #2|3|pants|2|Men|435|antipriese|large|589wheat947823419219|pink|Carton|Unknown|31|ationeingprieing| +8388|AAAAAAAACMACAAAA|2001-10-27||Fair houses add already russians. Quickly roman weeks remember again however free appeals; so supreme children can consider up t|1.75|4.23|7015009|scholarnameless #9|15|tables|7|Home|435|antipriese|N/A|589wheat947823419219|navajo|Gram|Unknown|20|eingeingprieing| +8389|AAAAAAAAFMACAAAA|1997-10-27||Others remain. Current men care now below a months. Partners should not wish entirely relevant clouds. Western, inland servants could not show symbols. Requests could make all unique|4.47|2.86|5003002|exportischolar #2|3|pop|5|Music|531|oughtprianti|N/A|012peru1767260630629|goldenrod|Pallet|Unknown|8|n steingprieing| +8390|AAAAAAAAGMACAAAA|1997-10-27|2000-10-26|Real minds shall|5.95|3.15|7010001|univnameless #1|10|flatware|7|Home|554|eseantianti|N/A|5406055steel23069722|seashell|Pallet|Unknown|64|barn stprieing| +8391|AAAAAAAAGMACAAAA|2000-10-27||Real minds shall|2.00|3.15|2002002|importoimporto #2|10|shirts|2|Men|60|eseantianti|large|8233252933linen54360|puff|Carton|Unknown|25|oughtn stprieing| +8392|AAAAAAAAIMACAAAA|1997-10-27|1999-10-27|Elements can face still interesting cases. Other, major colours kick even weeks; programmes shall result duly without a events. Square, little pa|4.50|4.00|2004001|edu packimporto #1|4|sports-apparel|2|Men|649|n stesecally|medium|6618sandy13993024533|linen|Case|Unknown|49|ablen stprieing| +8393|AAAAAAAAIMACAAAA|1999-10-28|2001-10-26|Fresh, direct steps must shrink already speci|3.93|2.24|2004001|amalgimporto #2|4|accessories|2|Men|577|ationationanti|medium|6618sandy13993024533|sandy|Bunch|Unknown|11|prin stprieing| +8394|AAAAAAAAIMACAAAA|2001-10-27||Far future steps start men. Forces create well types. Rapid, bad doctor|6.64|5.57|4001001|amalgedu pack #1|1|womens|4|Shoes|577|ationationanti|medium|48122817063plum41745|smoke|Box|Unknown|37|esen stprieing| +8395|AAAAAAAALMACAAAA|1997-10-27||Low examples may change tory, sacred elect|1.45|0.68|5004002|edu packscholar #2|4|classical|5|Music|808|eingbareing|N/A|872913firebrick41830|red|Cup|Unknown|95|antin stprieing| +8396|AAAAAAAAMMACAAAA|1997-10-27|2000-10-26|Alone different glasses could stop so numerous steps. Logical, social stages shall not afford letters.|1.39|0.59|5003001|exportischolar #1|3|pop|5|Music|887|ationeingeing|N/A|4674bisque7134455298|indian|Ton|Unknown|7|callyn stprieing| +8397|AAAAAAAAMMACAAAA|2000-10-27||Managers discuss for the revenues; prices used to want earlier dependent women. Considerable, unde|81.58|62.00|8004008|edu packnameless #8|3|camping|8|Sports|887|ationeingeing|N/A|4674bisque7134455298|sandy|Unknown|Unknown|12|ationn stprieing| +8398|AAAAAAAAOMACAAAA|1997-10-27|1999-10-27|So large borders must determine detailed missiles. Naval days should not allow components. Financial laws cost home the|9.79|7.73|8015009|scholarmaxi #9|15|fishing|8|Sports|240|bareseable|N/A|754225powder88114092|mint|Gross|Unknown|37|eingn stprieing| +8399|AAAAAAAAOMACAAAA|1999-10-28|2001-10-26|So large borders must determine detailed missiles. Naval days should not allow components. Financial laws cost home the|48.29|17.86|8015009|edu packnameless #10|15|glassware|7|Home|240|bareseable|N/A|754225powder88114092|magenta|N/A|Unknown|65|n stn stprieing| +8400|AAAAAAAAOMACAAAA|2001-10-27||So large borders must determine detailed missiles. Naval days should not allow components. Financial laws cost home the|5.38|17.86|2004001|edu packimporto #1|4|sports-apparel|2|Men|241|bareseable|small|754225powder88114092|light|Dram|Unknown|90|barbareseeing| +8401|AAAAAAAABNACAAAA|1997-10-27||Perhaps confident exercises hear less key children. Most numerous computers judge best slowly financial feelings. Long, current savings influence always criticall|4.82|4.14|5004002|edu packscholar #2|4|classical|5|Music|454|eseantiese|N/A|violet45816873268122|plum|Cup|Unknown|15|oughtbareseeing| +8402|AAAAAAAACNACAAAA|1997-10-27|2000-10-26|Commercial thousands look for a|2.86|1.91|10008005|namelessunivamalg #5|8|scanners|10|Electronics|134|esepriought|N/A|750smoke484151943573|slate|Bundle|Unknown|13|ablebareseeing| +8403|AAAAAAAACNACAAAA|2000-10-27||Commercial thousands look for a|8.27|3.39|3002002|importoexporti #2|8|infants|3|Children|462|ablecallyese|medium|14yellow921102910565|peach|Dozen|Unknown|4|pribareseeing| +8404|AAAAAAAAENACAAAA|1997-10-27|1999-10-27|Reliable firms fly. More new bases understand here on a powers. Measurements ought to know quite findings. Early southern views must consider other children. Good, growing needs stic|0.15|0.10|7011005|amalgnameless #5|11|accent|7|Home|57|ationanti|N/A|37115chocolate332150|snow|Ounce|Unknown|63|esebareseeing| +8405|AAAAAAAAENACAAAA|1999-10-28|2001-10-26|Reliable firms fly. More new bases understand here on a powers. Measurements ought to know quite findings. Early southern views must consider other children. Good, growing needs stic|1.06|0.38|3002002|importoexporti #2|2|infants|3|Children|625|antiablecally|large|65070992360papaya697|midnight|Dozen|Unknown|21|antibareseeing| +8406|AAAAAAAAENACAAAA|2001-10-27||Reliable firms fly. More new bases understand here on a powers. Measurements ought to know quite findings. Early southern views must consider other children. Good, growing needs stic|0.97|0.80|3002002|amalgimporto #1|2|accessories|2|Men|80|bareing|large|65070992360papaya697|smoke|Cup|Unknown|5|callybareseeing| +8407|AAAAAAAAHNACAAAA|1997-10-27||Numbers discuss all still, very firms. Necessary, strange guar|0.87|0.31|2002002|importoimporto #2|2|shirts|2|Men|24|eseable|extra large|9380521891042salmon3|mint|Each|Unknown|25|ationbareseeing| +8408|AAAAAAAAINACAAAA|1997-10-27|2000-10-26|Then harsh visitors limit irish, scottish photographs. Reasons should|0.69|0.35|5003001|exportischolar #1|3|pop|5|Music|235|antipriable|N/A|529111414960thistle7|rose|Gross|Unknown|46|eingbareseeing| +8409|AAAAAAAAINACAAAA|2000-10-27||New, russian circumstances make so however local premises. Other |1.72|1.42|5003001|scholarbrand #2|3|custom|6|Jewelry|651|antipriable|N/A|6057rosy175097512073|sienna|N/A|Unknown|64|n stbareseeing| +8410|AAAAAAAAKNACAAAA|1997-10-27|1999-10-27|Usual|8.63|4.83|5003001|exportischolar #1|3|pop|5|Music|342|ableesepri|N/A|9327goldenrod2663197|red|Gross|Unknown|1|baroughteseeing| +8411|AAAAAAAAKNACAAAA|1999-10-28|2001-10-26|Usual|4.22|1.81|1001002|amalgamalg #2|1|dresses|1|Women|135|ableesepri|large|64391782870072slate2|sienna|Dram|Unknown|2|oughtoughteseeing| +8412|AAAAAAAAKNACAAAA|2001-10-27||Processes shall suppor|9.57|4.78|9014001|edu packunivamalg #1|1|sports|9|Books|317|ableesepri|N/A|64391782870072slate2|honeydew|Gram|Unknown|27|ableoughteseeing| +8413|AAAAAAAANNACAAAA|1997-10-27||Long, good regions shall make under institutional societies. Disciplinary, unique clubs shall calm only more awkward females. Theories come hardly inappropriate issues;|1.67|1.40|7014004|edu packnameless #4|14|glassware|7|Home|331|oughtpripri|N/A|3193sky7639511045862|pink|Box|Unknown|10|prioughteseeing| +8414|AAAAAAAAONACAAAA|1997-10-27|2000-10-26|Special records should not organise practitioners. Too unable stories should not think very |3.92|2.66|6015007|scholarbrand #7|15|custom|6|Jewelry|552|ableantianti|N/A|996515634grey3110875|violet|Bundle|Unknown|14|eseoughteseeing| +8415|AAAAAAAAONACAAAA|2000-10-27||Political, special moments think inside usual types. However comprehensive parts give years; exercises can cut at all troops. Certain needs inform with a surfaces.|0.75|0.36|4004002|edu packedu pack #2|4|athletic|4|Shoes|552|ableantianti|small|996515634grey3110875|spring|N/A|Unknown|19|antioughteseeing| +8416|AAAAAAAAAOACAAAA|1997-10-27|1999-10-27|Alive effects must post over social months. New, early provisions shall not find aga|2.49|1.24|4004001|edu packedu pack #1|4|athletic|4|Shoes|289|n steingable|petite|8947pink851389696579|salmon|Box|Unknown|50|callyoughteseeing| +8417|AAAAAAAAAOACAAAA|1999-10-28|2001-10-26|Usual, large roles boost also to a units. Probably other sides may commit v|4.13|1.81|4004001|scholaramalgamalg #3|15|portable|10|Electronics|34|n steingable|N/A|4salmon8406587415246|smoke|Gross|Unknown|38|ationoughteseeing| +8418|AAAAAAAAAOACAAAA|2001-10-27||Legal, outer children might hear seriously eyes. Hitherto basic officers would not apply. Then |8.35|6.09|7001009|amalgbrand #9|15|bathroom|7|Home|31|oughtpri|N/A|4salmon8406587415246|salmon|Cup|Unknown|36|eingoughteseeing| +8419|AAAAAAAADOACAAAA|1997-10-27||Pounds |0.16|0.05|1001002|amalgamalg #2|1|dresses|1|Women|305|antibarpri|extra large|03716454gainsboro313|tomato|Each|Unknown|40|n stoughteseeing| +8420|AAAAAAAAEOACAAAA|1997-10-27|2000-10-26|O|9.52|5.61|3004001|edu packexporti #1|4|school-uniforms|3|Children|101|oughtbarought|petite|461purple96820378066|rosy|Case|Unknown|40|barableeseeing| +8421|AAAAAAAAEOACAAAA|2000-10-27||O|9.17|4.49|3004001|exportiedu pack #2|4|kids|4|Shoes|201|oughtbarought|economy|7282114008851snow330|midnight|Dozen|Unknown|93|oughtableeseeing| +8422|AAAAAAAAGOACAAAA|1997-10-27|1999-10-27|Just dead championships enable of course available systems. Easier physical systems must say. Electric, likely pages replace however natural feet. Criticisms annoy more positi|1.10|0.70|2003001|exportiimporto #1|3|pants|2|Men|246|callyeseable|extra large|537tan34793367880104|red|Gross|Unknown|6|ableableeseeing| +8423|AAAAAAAAGOACAAAA|1999-10-28|2001-10-26|Just dead championships enable of course available systems. Easier physical systems must say. Electric, likely pages replace however natural feet. Criticisms annoy more positi|6.73|3.70|2003001|corpcorp #8|6|rings|6|Jewelry|246|callyeseable|N/A|537tan34793367880104|sky|Ounce|Unknown|64|priableeseeing| +8424|AAAAAAAAGOACAAAA|2001-10-27||Today national ideas will not produce sure against the problems. Perfectly existing particles might tell civil years. Proposed, fixed things shall overthrow e|5.90|4.30|9014001|edu packunivamalg #1|6|sports|9|Books|246|callyeseable|N/A|597364686mint2973995|salmon|Pallet|Unknown|7|eseableeseeing| +8425|AAAAAAAAJOACAAAA|1997-10-27||Very small rights throw virtuall|4.57|3.15|3004002|edu packexporti #2|4|school-uniforms|3|Children|315|antioughtpri|extra large|980tomato72057340557|yellow|Tbl|Unknown|63|antiableeseeing| +8426|AAAAAAAAKOACAAAA|1997-10-27|2000-10-26|Local, left occasions provide for a patients; years ought to know under an workers. Churches maintain sufficient taxes. Almost unchanged visitors will observe simply in the affairs. P|6.66|2.19|1002001|importoamalg #1|2|fragrances|1|Women|554|eseantianti|medium|576152151517038rosy4|thistle|Tbl|Unknown|44|callyableeseeing| +8427|AAAAAAAAKOACAAAA|2000-10-27||Usual books could make big ears. Dark leaders shall remember however. Annual, spanish things should pa|2.23|2.19|5002002|importoscholar #2|2|country|5|Music|633|eseantianti|N/A|576152151517038rosy4|hot|N/A|Unknown|61|ationableeseeing| +8428|AAAAAAAAMOACAAAA|1997-10-27|1999-10-27|Minor expenses enjoy then with a programmes; french, appropriate years lend still visible, crude hotels. Quick patient|9.11|4.73|2004001|edu packimporto #1|4|sports-apparel|2|Men|44|eseese|medium|6moccasin84841557610|light|Pallet|Unknown|19|eingableeseeing| +8429|AAAAAAAAMOACAAAA|1999-10-28|2001-10-26|Minor expenses enjoy then with a programmes; french, appropriate years lend still visible, crude hotels. Quick patient|7.93|4.73|2004001|corpnameless #10|4|furniture|7|Home|314|eseese|N/A|7spring4317921868719|tomato|Ton|Unknown|63|n stableeseeing| +8430|AAAAAAAAMOACAAAA|2001-10-27||Minor expenses enjoy then with a programmes; french, appropriate years lend still visible, crude hotels. Quick patient|4.11|1.35|2004001|brandcorp #7|4|pendants|6|Jewelry|314|eseese|N/A|159325124tomato95794|tan|Gram|Unknown|21|barprieseeing| +8431|AAAAAAAAPOACAAAA|1997-10-27||Major deaths swing later books; particularly expected problems give. High, high tools must see big areas. Major, informal passengers devise; windows cannot think further nice doors. Small|4.56|2.28|9015008|scholarunivamalg #8|15|fiction|9|Books|180|bareingought|N/A|4827509rose685321475|tomato|Lb|Unknown|93|oughtprieseeing| +8432|AAAAAAAAAPACAAAA|1997-10-27|2000-10-26|Good, physical events should bu|3.35|1.00|8015003|scholarmaxi #3|15|fishing|8|Sports|664|esecallycally|N/A|2040steel70370008379|pink|Dozen|Unknown|56|ableprieseeing| +8433|AAAAAAAAAPACAAAA|2000-10-27||Rates could start on the women; still alternative governments l|7.94|1.00|8015003|exportiexporti #2|3|toddlers|3|Children|152|ableantiought|extra large|93005202royal6409259|sky|Box|Unknown|21|priprieseeing| +8434|AAAAAAAACPACAAAA|1997-10-27|1999-10-27|British, lost winds will make again by a kinds. Little, medical difficulties succeed social, relevant rights. Workshops will find forward little|3.75|2.55|10015003|scholaramalgamalg #3|15|portable|10|Electronics|783|prieingation|N/A|322485274mint7689610|hot|Box|Unknown|30|eseprieseeing| +8435|AAAAAAAACPACAAAA|1999-10-28|2001-10-26|British, lost winds will make again by a kinds. Little, medical difficulties succeed social, relevant rights. Workshops will find forward little|3.45|2.55|10015003|exportiexporti #2|3|toddlers|3|Children|698|eingn stcally|extra large|322485274mint7689610|yellow|Carton|Unknown|44|antiprieseeing| +8436|AAAAAAAACPACAAAA|2001-10-27||British, lost winds will make again by a kinds. Little, medical difficulties succeed social, relevant rights. Workshops will find forward little|3.01|2.40|7004007|edu packbrand #7|4|curtains/drapes|7|Home|286|eingn stcally|N/A|322485274mint7689610|green|Ton|Unknown|38|callyprieseeing| +8437|AAAAAAAAFPACAAAA|1997-10-27||Still personal products publish periods. Arrangements would seek separately outside a events. Minutes may complete aut|4.88|2.24|2003002|exportiimporto #2|3|pants|2|Men|124|eseableought|large|01497205sienna732993|white|Pallet|Unknown|24|ationprieseeing| +8438|AAAAAAAAGPACAAAA|1997-10-27|2000-10-26|Trends wo|1.34|0.61|10001007|amalgunivamalg #7|1|cameras|10|Electronics|871|oughtationeing|N/A|5760red3495721962198|chocolate|Ounce|Unknown|90|eingprieseeing| +8439|AAAAAAAAGPACAAAA|2000-10-27||Trends wo|3.86|2.74|10001007|exportiimporto #2|3|pants|2|Men|24|oughtationeing|extra large|3213571170479tomato1|grey|Ton|Unknown|26|n stprieseeing| +8440|AAAAAAAAIPACAAAA|1997-10-27|1999-10-27|Local, fresh cuts give partners. Indep|2.55|2.14|2003001|exportiimporto #1|3|pants|2|Men|166|callycallyought|petite|49501176olive3966160|royal|Tsp|Unknown|79|bareseeseeing| +8441|AAAAAAAAIPACAAAA|1999-10-28|2001-10-26|There potential prices lose already shares. Good instruments could point generally small, successful friends. Quite dead offences fool almost about a fruits. Extreme ships jus|4.53|2.14|1004002|edu packamalg #2|3|swimwear|1|Women|166|callycallyought|small|49501176olive3966160|turquoise|Box|Unknown|63|oughteseeseeing| +8442|AAAAAAAAIPACAAAA|2001-10-27||There potential prices lose already shares. Good instruments could point generally small, successful friends. Quite dead offences fool almost about a fruits. Extreme ships jus|14.11|12.27|1004002|edu packunivamalg #7|3|sports|9|Books|356|callycallyought|N/A|49501176olive3966160|powder|N/A|Unknown|8|ableeseeseeing| +8443|AAAAAAAALPACAAAA|1997-10-27||New, selective values might go now famous, ri|1.84|1.38|10008012|namelessunivamalg #12|8|scanners|10|Electronics|49|n stese|N/A|6598854maroon6044568|tan|Unknown|Unknown|65|prieseeseeing| +8444|AAAAAAAAMPACAAAA|1997-10-27|2000-10-26|Restaurants fulfil then medical girls; civil|7.30|6.13|3004001|edu packexporti #1|4|school-uniforms|3|Children|21|oughtable|large|408812336074dark5429|snow|Lb|Unknown|29|eseeseeseeing| +8445|AAAAAAAAMPACAAAA|2000-10-27||Restaurants fulfil then medical girls; civil|9.53|8.19|4003002|exportiedu pack #2|4|kids|4|Shoes|199|oughtable|large|408812336074dark5429|orchid|Bunch|Unknown|58|antieseeseeing| +8446|AAAAAAAAOPACAAAA|1997-10-27|1999-10-27|Old, good types could not stay strongly strange matters. Mild pensions might convene softly human studies. Today northern times ensure exactly. Marginally preliminary|3.90|3.00|4001001|amalgedu pack #1|1|womens|4|Shoes|276|callyationable|extra large|37561841595purple408|turquoise|Ton|Unknown|19|callyeseeseeing| +8447|AAAAAAAAOPACAAAA|1999-10-28|2001-10-26|Old, good types could not stay strongly strange matters. Mild pensions might convene softly human studies. Today northern times ensure exactly. Marginally preliminary|3.95|3.00|4001001|exportiamalg #2|3|maternity|1|Women|276|callyationable|medium|7764236374powder8036|gainsboro|Case|Unknown|24|ationeseeseeing| +8448|AAAAAAAAOPACAAAA|2001-10-27||Old, good types could not stay strongly strange matters. Mild pensions might convene softly human studies. Today northern times ensure exactly. Marginally preliminary|74.13|3.00|4001001|amalgamalg #1|3|dresses|1|Women|276|callyationable|large|7764236374powder8036|pink|N/A|Unknown|30|eingeseeseeing| +8449|AAAAAAAABABCAAAA|1997-10-27||Companies start only healthy police. |0.68|0.41|6013002|exportibrand #2|13|loose stones|6|Jewelry|224|eseableable|N/A|medium44055831735497|lace|Cup|Unknown|4|n steseeseeing| +8450|AAAAAAAACABCAAAA|1997-10-27|2000-10-26|Powerful, romantic decisions warm animals. |5.24|1.88|1003001|exportiamalg #1|3|maternity|1|Women|382|ableeingpri|extra large|6810332puff337182167|peru|Bunch|Unknown|18|barantieseeing| +8451|AAAAAAAACABCAAAA|2000-10-27||Different arrangements sound traditional, serious conclusions; prime, warm thanks used to tak|1.90|1.88|1003001|exportibrand #4|3|loose stones|6|Jewelry|382|ableeingpri|N/A|6810332puff337182167|yellow|Case|Unknown|49|oughtantieseeing| +8452|AAAAAAAAEABCAAAA|1997-10-27|1999-10-27|Parliamentary, prime appointments must worry full, unnecessary courts. Years help well sure, general resources. Skills should go base objectives. Eastern planes will reveal com|9.70|6.49|2003001|exportiimporto #1|3|pants|2|Men|908|eingbarn st|small|46tan418915027800508|yellow|Lb|Unknown|74|ableantieseeing| +8453|AAAAAAAAEABCAAAA|1999-10-28|2001-10-26|Young, current margins must seem effortlessly old representatives. Processes can add then large, vo|1.57|6.49|3004002|edu packexporti #2|4|school-uniforms|3|Children|908|eingbarn st|medium|46tan418915027800508|blue|Carton|Unknown|68|priantieseeing| +8454|AAAAAAAAEABCAAAA|2001-10-27||Young, current margins must seem effortlessly old representatives. Processes can add then large, vo|9.75|8.77|3004002|importoedu pack #1|4|mens|4|Shoes|501|eingbarn st|small|46tan418915027800508|orange|Bundle|Unknown|10|eseantieseeing| +8455|AAAAAAAAHABCAAAA|1997-10-27||Major, political games walk then other matters. Sure applications should|1.53|0.73|3003002|exportiexporti #2|3|toddlers|3|Children|54|eseanti|extra large|8saddle1287982852458|plum|Dozen|Unknown|90|antiantieseeing| +8456|AAAAAAAAIABCAAAA|1997-10-27|2000-10-26|Moves keep badly principal churches. Descriptions could suppose quite. Therefore huge tools prevent final, natural books; dark measures find g|4.22|3.71|6006001|corpcorp #1|6|rings|6|Jewelry|123|priableought|N/A|36papaya825388346623|pink|Lb|Unknown|6|callyantieseeing| +8457|AAAAAAAAIABCAAAA|2000-10-27||Royal, various deals may move enough. Christian, safe patterns used to blame modern sides. New, frequent reservations get for a communities; patients might turn educational things. Critical leaves |6.95|6.18|6006001|namelessbrand #8|8|lighting|7|Home|123|priableought|N/A|36papaya825388346623|olive|Pallet|Unknown|47|ationantieseeing| +8458|AAAAAAAAKABCAAAA|1997-10-27|1999-10-27|Calmly contrary measures should happen now funny things. Southern conditions may benefit actually other solicitors. Days ease. Young, kind thoughts can spend suffic|2.06|1.64|10016010|corpamalgamalg #10|16|wireless|10|Electronics|118|eingoughtought|N/A|8926orange6428740582|thistle|Bundle|Unknown|96|eingantieseeing| +8459|AAAAAAAAKABCAAAA|1999-10-28|2001-10-26|Usually greek sides cover also studies; high, immediate crews withdraw. |0.28|0.24|10016010|corpunivamalg #11|6|musical|10|Electronics|118|eingoughtought|N/A|8926orange6428740582|thistle|Lb|Unknown|78|n stantieseeing| +8460|AAAAAAAAKABCAAAA|2001-10-27||New, agricultural issues should talk healthy poems. Too ordinary hands shall not get also with|1.31|0.24|3002001|importoexporti #1|6|infants|3|Children|118|eingoughtought|extra large|18569064083cream1676|dim|Cup|Unknown|18|barcallyeseeing| +8461|AAAAAAAANABCAAAA|1997-10-27||Democrats follow mostly available,|0.59|0.47|7003002|exportibrand #2|3|kids|7|Home|456|callyantiese|N/A|95161298chocolate950|saddle|Ton|Unknown|61|oughtcallyeseeing| +8462|AAAAAAAAOABCAAAA|1997-10-27|2000-10-26|Loud independent plans used to shape under events. Meanwhile practical emotions examine subject children. Very, likely beliefs a|0.55|0.46|6013007|exportibrand #7|13|loose stones|6|Jewelry|143|prieseought|N/A|74427707turquoise634|green|Case|Unknown|100|ablecallyeseeing| +8463|AAAAAAAAOABCAAAA|2000-10-27||Difficult schools could allow big, specific charts. Services will rise however relevant residents. Also private numbers resist soon at a conclusions. Direct leaders ought to wait for example redu|87.18|0.46|3003002|exportiexporti #2|13|toddlers|3|Children|143|prieseought|economy|74427707turquoise634|powder|Pound|Unknown|10|pricallyeseeing| +8464|AAAAAAAAABBCAAAA|1997-10-27|1999-10-27|Warm edges round terms. Then key services undertake far small stars. Lights find before general causes. Great, other tables obstruct just changes. Already common m|3.29|1.54|5003001|exportischolar #1|3|pop|5|Music|131|oughtpriought|N/A|75649416448sky450727|tomato|Tbl|Unknown|43|esecallyeseeing| +8465|AAAAAAAAABBCAAAA|1999-10-28|2001-10-26|Feet use never bare, smooth symptoms; today available charges provide possible margins. German symptoms score here pages. Once again royal count|2.37|1.54|5003001|edu packbrand #2|14|estate|6|Jewelry|131|oughtpriought|N/A|0yellow5662099855989|pale|Bundle|Unknown|37|anticallyeseeing| +8466|AAAAAAAAABBCAAAA|2001-10-27||Colourful, c|9.44|5.56|7005007|scholarbrand #7|5|blinds/shades|7|Home|209|oughtpriought|N/A|9336748555slate41892|tan|Gross|Unknown|23|callycallyeseeing| +8467|AAAAAAAADBBCAAAA|1997-10-27||Important developers establish natural, british pieces. Visible, gentle sheets shall act roles; real, political days will want to the things. Mild, heavy applications find. Sensible problems used|4.94|3.21|3002002|importoexporti #2|2|infants|3|Children|742|ableeseation|medium|012983003652mint2422|lemon|Pallet|Unknown|56|ationcallyeseeing| +8468|AAAAAAAAEBBCAAAA|1997-10-27|2000-10-26|Nowadays single parties look variable, common children. Back de|0.34|0.11|4003001|exportiedu pack #1|3|kids|4|Shoes|575|antiationanti|N/A|233navy7763721960839|lemon|Oz|Unknown|6|eingcallyeseeing| +8469|AAAAAAAAEBBCAAAA|2000-10-27||Nowadays single parties look variable, common children. Back de|1.82|0.67|4003001|univbrand #6|3|jewelry boxes|6|Jewelry|321|antiationanti|N/A|738778808655mint4612|red|Tbl|Unknown|55|n stcallyeseeing| +8470|AAAAAAAAGBBCAAAA|1997-10-27|1999-10-27|Forward prepared homes face also so small applications|7.35|2.49|6001007|amalgcorp #7|1|birdal|6|Jewelry|520|barableanti|N/A|846610861550plum5973|royal|Dozen|Unknown|21|barationeseeing| +8471|AAAAAAAAGBBCAAAA|1999-10-28|2001-10-26|Points explain in a policies; benefits may trust as it were wooden, costly copies. Increasingl|7.57|3.25|6001007|amalgexporti #2|1|newborn|3|Children|520|barableanti|medium|649643855012737peru4|royal|Ounce|Unknown|24|oughtationeseeing| +8472|AAAAAAAAGBBCAAAA|2001-10-27||Rich, possible groups move activities. Problems must know further near usual colleagues; national books sort brown, delicious witnesses. Alone patients lower also i|2.87|1.43|3002001|importoexporti #1|1|infants|3|Children|520|barableanti|medium|649643855012737peru4|peru|Pound|Unknown|37|ableationeseeing| +8473|AAAAAAAAJBBCAAAA|1997-10-27||As huge types see conservative, main years. As it were separate groups may not lift soon on a studies. Dependent, due aspects shall count just environmental, national years. Damages shall allow |77.79|65.34|2002002|importoimporto #2|2|shirts|2|Men|181|oughteingought|extra large|741226966lawn5565311|peach|Dram|Unknown|27|priationeseeing| +8474|AAAAAAAAKBBCAAAA|1997-10-27|2000-10-26|Ever possible bags ought to consider ashore to the activities. Environmental boys tak|5.50|2.80|6003003|exporticorp #3|3|gold|6|Jewelry|198|eingn stought|N/A|8585thistle159839076|puff|Unknown|Unknown|3|eseationeseeing| +8475|AAAAAAAAKBBCAAAA|2000-10-27||Massive, happy tasks might change generally usefully evident libraries. So different pupils make whole, parti|1.56|2.80|3001002|amalgexporti #2|1|newborn|3|Children|77|ationation|petite|8610463steel43414127|sky|Pallet|Unknown|40|antiationeseeing| +8476|AAAAAAAAMBBCAAAA|1997-10-27|1999-10-27|Women appear for example things. Sure strengths go recent flights. Walls see more expen|0.72|0.29|3001001|amalgexporti #1|1|newborn|3|Children|615|antioughtcally|large|844887177913slate696|rosy|Ounce|Unknown|68|callyationeseeing| +8477|AAAAAAAAMBBCAAAA|1999-10-28|2001-10-26|Women appear for example things. Sure strengths go recent flights. Walls see more expen|4.54|2.95|3001001|exportiedu pack #2|1|kids|4|Shoes|615|antioughtcally|petite|844887177913slate696|wheat|Dram|Unknown|13|ationationeseeing| +8478|AAAAAAAAMBBCAAAA|2001-10-27||Main assets see tonight at a days. Sorry, dutch standards can exist marine months. Cities provide on the votes. Now different students fight as well conditi|4.19|3.68|1002001|importoamalg #1|2|fragrances|1|Women|615|antioughtcally|large|844887177913slate696|chiffon|Bundle|Unknown|28|eingationeseeing| +8479|AAAAAAAAPBBCAAAA|1997-10-27||Detailed, free organisers shall not pay more etc good falls. Double, full-time wages use immensely final arguments; finally |5.31|2.17|5002002|importoscholar #2|2|country|5|Music|602|ablebarcally|N/A|456711671spring81921|spring|N/A|Unknown|1|n stationeseeing| +8480|AAAAAAAAACBCAAAA|1997-10-27|2000-10-26|Old ports may not help frames; lucky, young standards can prevent quickly; pretty tender animals used to ta|1.60|0.64|6002001|importocorp #1|2|diamonds|6|Jewelry|211|oughtoughtable|N/A|142561gainsboro86797|powder|Pallet|Unknown|36|bareingeseeing| +8481|AAAAAAAAACBCAAAA|2000-10-27||Old ports may not help frames; lucky, young standards can prevent quickly; pretty tender animals used to ta|87.87|0.64|6002001|exportinameless #2|2|wallpaper|7|Home|211|oughtoughtable|N/A|142561gainsboro86797|seashell|Pallet|Unknown|38|oughteingeseeing| +8482|AAAAAAAACCBCAAAA|1997-10-27|1999-10-27|No longer asleep eyes ought to seem in a proceedings. Agencies slip much black clothes. Still big ways should pass recently very, solar authors; thus small countries send thick, splendid |7.15|5.43|2003001|exportiimporto #1|3|pants|2|Men|701|oughtbaration|large|93615473288189rose07|rose|Pallet|Unknown|33|ableeingeseeing| +8483|AAAAAAAACCBCAAAA|1999-10-28|2001-10-26|No longer asleep eyes ought to seem in a proceedings. Agencies slip much black clothes. Still big ways should pass recently very, solar authors; thus small countries send thick, splendid |3.84|2.61|9007012|brandmaxi #12|3|reference|9|Books|701|oughtbaration|N/A|93615473288189rose07|chartreuse|Carton|Unknown|16|prieingeseeing| +8484|AAAAAAAACCBCAAAA|2001-10-27||No longer asleep eyes ought to seem in a proceedings. Agencies slip much black clothes. Still big ways should pass recently very, solar authors; thus small countries send thick, splendid |2.43|2.61|9007012|namelessbrand #5|8|lighting|7|Home|701|oughtbaration|N/A|431metallic420978175|cyan|Oz|Unknown|50|eseeingeseeing| +8485|AAAAAAAAFCBCAAAA|1997-10-27||Powerful days see most potential, successful days. Existing men react profits. Mistakes point across a policies. Easy writers fall over to a strengths. Active, serious readers will tell now free, |9.00|5.67|6014006|edu packbrand #6|14|estate|6|Jewelry|201|oughtbarable|N/A|67thistle30018522608|wheat|N/A|Unknown|25|antieingeseeing| +8486|AAAAAAAAGCBCAAAA|1997-10-27|2000-10-26|Attacks should lend companies. Just general concentrations used to meet often with the walls. Public proposals ought to stir far public lips. Familiar, unlikely characteristics could require ev|8.79|5.01|3002001|importoexporti #1|2|infants|3|Children|9|n st|small|12196171yellow872678|saddle|Bunch|Unknown|15|callyeingeseeing| +8487|AAAAAAAAGCBCAAAA|2000-10-27||Attacks should lend companies. Just general concentrations used to meet often with the walls. Public proposals ought to stir far public lips. Familiar, unlikely characteristics could require ev|8.13|6.09|8015008|scholarmaxi #8|2|fishing|8|Sports|253|n st|N/A|279219298cyan6640056|royal|Bundle|Unknown|30|ationeingeseeing| +8488|AAAAAAAAICBCAAAA|1997-10-27|1999-10-27|Domestic millions used to place often able wings. About english words form. Potential paren|0.17|0.08|7015009|scholarnameless #9|15|tables|7|Home|86|callyeing|N/A|7675473327lawn749607|salmon|Box|Unknown|33|eingeingeseeing| +8489|AAAAAAAAICBCAAAA|1999-10-28|2001-10-26|Domestic millions used to place often able wings. About english words form. Potential paren|5.80|1.91|7015009|importoexporti #2|2|infants|3|Children|357|ationantipri|small|7675473327lawn749607|rosy|Oz|Unknown|49|n steingeseeing| +8490|AAAAAAAAICBCAAAA|2001-10-27||Overseas, short kilometres shall not include at least only limited operations; however similar words ensure shared, keen days. Huge attitudes come most. Obligations could make theref|4.93|3.99|7015009|edu packexporti #1|4|school-uniforms|3|Children|358|ationantipri|medium|7675473327lawn749607|puff|N/A|Unknown|4|barn steseeing| +8491|AAAAAAAALCBCAAAA|1997-10-27||Western elements shall not remember in the unions. Cruel assessments show again important teachers. Later real pp. engage boring hands. Earli|6.67|2.80|8016002|corpmaxi #2|16|golf|8|Sports|556|callyantianti|N/A|84543712004sandy7361|steel|Oz|Unknown|16|oughtn steseeing| +8492|AAAAAAAAMCBCAAAA|1997-10-27|2000-10-26|American, far marks consider early comments. Carefully various recordings see brief patients; hours bring local calls. Often various scenes capitalise coming, other a|53.43|20.83|8014003|edu packmaxi #3|14|tennis|8|Sports|219|n stoughtable|N/A|542681168422linen663|pink|Bundle|Unknown|31|ablen steseeing| +8493|AAAAAAAAMCBCAAAA|2000-10-27||American, far marks consider early comments. Carefully various recordings see brief patients; hours bring local calls. Often various scenes capitalise coming, other a|9.23|3.59|8014003|exportiunivamalg #11|3|dvd/vcr players|10|Electronics|219|n stoughtable|N/A|991red69556175907455|brown|Box|Unknown|8|prin steseeing| +8494|AAAAAAAAOCBCAAAA|1997-10-27|1999-10-27|Totally new stories describe beyond a hands; more different men see over cases. Serious, other cases spoil concerned teeth. Traditional movies run as. General, other cells|4.34|1.73|1001001|amalgamalg #1|1|dresses|1|Women|382|ableeingpri|extra large|coral949691643152479|saddle|Tsp|Unknown|28|esen steseeing| +8495|AAAAAAAAOCBCAAAA|1999-10-28|2001-10-26|Totally new stories describe beyond a hands; more different men see over cases. Serious, other cases spoil concerned teeth. Traditional movies run as. General, other cells|94.83|79.65|1001001|exportiedu pack #2|3|kids|4|Shoes|382|ableeingpri|medium|343463dim71537217874|sky|Dram|Unknown|19|antin steseeing| +8496|AAAAAAAAOCBCAAAA|2001-10-27||Right, acceptable mammals would look a little. Exciting others ought to exercise either already ordinary borders. Available m|2.39|79.65|10012014|importoamalgamalg #14|12|monitors|10|Electronics|382|ableeingpri|N/A|343463dim71537217874|sienna|Cup|Unknown|11|callyn steseeing| +8497|AAAAAAAABDBCAAAA|1997-10-27||Citizens leave clearly through a minutes. Only bare women should explain real eyes. Sales curb also wooden, successful visitors. Differences explain certainly earnings. More economic conditions us|5.14|3.39|10016015|corpamalgamalg #15|16|wireless|10|Electronics|148|eingeseought|N/A|0049205778454purple8|hot|Tbl|Unknown|33|ationn steseeing| +8498|AAAAAAAACDBCAAAA|1997-10-27|2000-10-26|Tiny years could run too above tough volumes. New germans must not leave as possible sales; inj|1.22|0.39|9002003|importomaxi #3|2|business|9|Books|586|callyeinganti|N/A|smoke442682918970985|goldenrod|Dozen|Unknown|38|eingn steseeing| +8499|AAAAAAAACDBCAAAA|2000-10-27||Tiny years could run too above tough volumes. New germans must not leave as possible sales; inj|0.90|0.39|9002003|univnameless #10|2|flatware|7|Home|479|callyeinganti|N/A|62455peru90974108292|papaya|Pallet|Unknown|43|n stn steseeing| +8500|AAAAAAAAEDBCAAAA|1997-10-27|1999-10-27|New, e|6.38|3.57|6011005|amalgbrand #5|11|semi-precious|6|Jewelry|707|ationbaration|N/A|powder77415856431289|smoke|Dozen|Unknown|12|barbarantieing| +8501|AAAAAAAAEDBCAAAA|1999-10-28|2001-10-26|English obligations used to know. Nuclear profits must sp|5.44|4.24|6011005|exportimaxi #2|11|sailing|8|Sports|707|ationbaration|N/A|powder77415856431289|goldenrod|Bunch|Unknown|9|oughtbarantieing| +8502|AAAAAAAAEDBCAAAA|2001-10-27||English obligations used to know. Nuclear profits must sp|6.24|4.74|7014003|edu packnameless #3|14|glassware|7|Home|707|ationbaration|N/A|5247160beige18460673|smoke|Carton|Unknown|68|ablebarantieing| +8503|AAAAAAAAHDBCAAAA|1997-10-27||Increased, occasional events shall not reveal. Local, popular positions will not need adequate me|1.77|0.74|4004002|edu packedu pack #2|4|athletic|4|Shoes|331|oughtpripri|medium|022royal481230307234|lavender|Pallet|Unknown|29|pribarantieing| +8504|AAAAAAAAIDBCAAAA|1997-10-27|2000-10-26|Autonomous, short patterns may not comply tests. So significant ladie|4.69|2.11|1003001|exportiamalg #1|3|maternity|1|Women|55|antianti|petite|55029goldenrod812323|sky|Bunch|Unknown|14|esebarantieing| +8505|AAAAAAAAIDBCAAAA|2000-10-27||Due, social keys would not play long|7.92|2.11|8014006|edu packmaxi #6|3|tennis|8|Sports|55|antianti|N/A|353646041seashell279|steel|N/A|Unknown|20|antibarantieing| +8506|AAAAAAAAKDBCAAAA|1997-10-27|1999-10-27|Following, free pounds might not suit enough years. Gradual objectives shall not reflect changes. Public, final followers used to kick a little british, pro|6.09|5.48|5004001|edu packscholar #1|4|classical|5|Music|630|barprically|N/A|882111indian95825080|purple|Gram|Unknown|20|callybarantieing| +8507|AAAAAAAAKDBCAAAA|1999-10-28|2001-10-26|Governments c|1.77|5.48|10014008|edu packamalgamalg #8|14|automotive|10|Electronics|87|ationeing|N/A|indian90981442748833|tan|Ton|Unknown|29|ationbarantieing| +8508|AAAAAAAAKDBCAAAA|2001-10-27||Governments c|3.04|2.15|10014008|exportinameless #9|14|wallpaper|7|Home|87|ationeing|N/A|indian90981442748833|yellow|Case|Unknown|36|eingbarantieing| +8509|AAAAAAAANDBCAAAA|1997-10-27||Again judicial colours may blame fully british strange groups. Rules shall cover probably participants. W|5.63|3.43|8003010|exportinameless #10|3|basketball|8|Sports|33|pripri|N/A|9900557floral1233215|puff|Dozen|Unknown|74|n stbarantieing| +8510|AAAAAAAAODBCAAAA|1997-10-27|2000-10-26|Others see plain even social disciplines. Locati|1.55|1.13|5003001|exportischolar #1|3|pop|5|Music|264|esecallyable|N/A|06096indian277761865|metallic|Carton|Unknown|8|baroughtantieing| +8511|AAAAAAAAODBCAAAA|2000-10-27||Exactly legal banks ought to account properly multiple, toxic fathers. Urgent systems answer relevant, remarkable participants. Primarily sure applic|2.01|1.20|8016002|corpmaxi #2|16|golf|8|Sports|496|esecallyable|N/A|06096indian277761865|black|Bunch|Unknown|52|oughtoughtantieing| +8512|AAAAAAAAAEBCAAAA|1997-10-27|1999-10-27|Members make|0.83|0.45|5001001|amalgscholar #1|1|rock|5|Music|82|ableeing|N/A|61642173776violet683|lemon|Bundle|Unknown|43|ableoughtantieing| +8513|AAAAAAAAAEBCAAAA|1999-10-28|2001-10-26|Recent, armed honours escape reluctantly now dead phenomena. Arts cannot speak northern prices. Ministers go. Old stars eat sometimes exactly formal years. Ra|4.76|0.45|9016006|corpunivamalg #6|16|mystery|9|Books|82|ableeing|N/A|280593097390tan55416|tan|Box|Unknown|49|prioughtantieing| +8514|AAAAAAAAAEBCAAAA|2001-10-27||Efforts offer already. British beings must not remain better in th|63.76|49.73|9016006|exportinameless #5|13|wallpaper|7|Home|129|ableeing|N/A|280593097390tan55416|midnight|Each|Unknown|41|eseoughtantieing| +8515|AAAAAAAADEBCAAAA|1997-10-27||Scenes attract wooden drugs; mai|2.05|1.47|8003006|exportinameless #6|3|basketball|8|Sports|525|antiableanti|N/A|108096257turquoise82|bisque|Bunch|Unknown|59|antioughtantieing| +8516|AAAAAAAAEEBCAAAA|1997-10-27|2000-10-26|Then soft proposals identify paintings. Even russian tonnes might prevent authorities. There video-taped miles could set imp|4.16|2.20|10002017|importounivamalg #17|2|camcorders|10|Electronics|102|ablebarought|N/A|652893716976plum8588|thistle|Case|Unknown|3|callyoughtantieing| +8517|AAAAAAAAEEBCAAAA|2000-10-27||Powerful cases happen; est|5.74|2.20|10002017|edu packscholar #2|2|classical|5|Music|191|oughtn stought|N/A|652893716976plum8588|purple|Unknown|Unknown|13|ationoughtantieing| +8518|AAAAAAAAGEBCAAAA|1997-10-27|1999-10-27|Bills provide by a committees. Finan|3.40|1.53|10006002|corpunivamalg #2|6|musical|10|Electronics|309|n stbarpri|N/A|74319cornflower96645|papaya|Case|Unknown|94|eingoughtantieing| +8519|AAAAAAAAGEBCAAAA|1999-10-28|2001-10-26|Sadly alive books could not obtain heavily soviet marginal audiences. Closer |4.77|2.05|3003002|exportiexporti #2|6|toddlers|3|Children|638|n stbarpri|large|9604723644336tomato1|saddle|Dozen|Unknown|6|n stoughtantieing| +8520|AAAAAAAAGEBCAAAA|2001-10-27||Too everyday studies would not make companies. Great long organisations express sharply enough teenage streets. Entirely cri|6.38|2.05|9013001|exportiunivamalg #1|6|self-help|9|Books|265|anticallyable|N/A|puff4332897097424264|chocolate|Pound|Unknown|64|barableantieing| +8521|AAAAAAAAJEBCAAAA|1997-10-27||Fundamental arms could depend. Members shall see other project|2.43|1.72|9015002|scholarunivamalg #2|15|fiction|9|Books|427|ationableese|N/A|floral58458623300085|drab|Cup|Unknown|18|oughtableantieing| +8522|AAAAAAAAKEBCAAAA|1997-10-27|2000-10-26|Fresh procedures should not think only perhaps broken grounds. Main yards|4.90|2.94|10005006|scholarunivamalg #6|5|karoke|10|Electronics|227|ationableable|N/A|14415orchid967403415|sienna|Bundle|Unknown|12|ableableantieing| +8523|AAAAAAAAKEBCAAAA|2000-10-27||Centres could not come too later rural b|0.55|0.28|9007004|brandmaxi #4|7|reference|9|Books|968|ationableable|N/A|14415orchid967403415|rose|Bunch|Unknown|8|priableantieing| +8524|AAAAAAAAMEBCAAAA|1997-10-27|1999-10-27|Weeks might not find original elections. Active hands might enjoy occasional, young proposals. Slight, necessary studies prevent frequently industrial, private reasons. Inherently single effects o|0.62|0.25|9010005|univunivamalg #5|10|travel|9|Books|141|oughteseought|N/A|88657burnished655100|khaki|Tsp|Unknown|34|eseableantieing| +8525|AAAAAAAAMEBCAAAA|1999-10-28|2001-10-26|Weeks might not find original elections. Active hands might enjoy occasional, young proposals. Slight, necessary studies prevent frequently industrial, private reasons. Inherently single effects o|1.90|0.91|2001002|amalgimporto #2|10|accessories|2|Men|141|oughteseought|large|88657burnished655100|tan|Carton|Unknown|8|antiableantieing| +8526|AAAAAAAAMEBCAAAA|2001-10-27||Weeks might not find original elections. Active hands might enjoy occasional, young proposals. Slight, necessary studies prevent frequently industrial, private reasons. Inherently single effects o|4.54|2.04|2001002|edu packedu pack #1|10|athletic|4|Shoes|141|oughteseought|extra large|88657burnished655100|plum|Case|Unknown|57|callyableantieing| +8527|AAAAAAAAPEBCAAAA|1997-10-27||English authorities can take; sometimes mental eyes know quickly; immediate jobs should think below critical villages. Red, international diff|1.36|0.96|8002008|importonameless #8|2|baseball|8|Sports|642|ableesecally|N/A|692spring36100967811|royal|Dozen|Unknown|60|ationableantieing| +8528|AAAAAAAAAFBCAAAA|1997-10-27|2000-10-26|Young plates afford. Hot, clean systems force following wheels. Always english homes multiply te|5.67|3.34|1004001|edu packamalg #1|4|swimwear|1|Women|563|pricallyanti|large|977945296653457sky73|peru|Ton|Unknown|24|eingableantieing| +8529|AAAAAAAAAFBCAAAA|2000-10-27||Frequent bodies shall expect very similar areas; left others must like tonight irish functions. Foreign, modern children used to love necessarily high stones; standards go readily|6.80|3.34|1004001|exportiimporto #2|3|pants|2|Men|563|pricallyanti|medium|977945296653457sky73|turquoise|Unknown|Unknown|41|n stableantieing| +8530|AAAAAAAACFBCAAAA|1997-10-27|1999-10-27|Very free heels enable easily in a services; results used to visit more only, other firms. Hands need much commercial faces. Still likely actions may not ext|29.98|19.18|3003001|exportiexporti #1|3|toddlers|3|Children|246|callyeseable|small|819462beige720289913|slate|Bundle|Unknown|50|barpriantieing| +8531|AAAAAAAACFBCAAAA|1999-10-28|2001-10-26|Very free heels enable easily in a services; results used to visit more only, other firms. Hands need much commercial faces. Still likely actions may not ext|7.54|19.18|3003001|amalgamalg #2|3|dresses|1|Women|210|callyeseable|large|184white922458378927|plum|Unknown|Unknown|9|oughtpriantieing| +8532|AAAAAAAACFBCAAAA|2001-10-27||Both large dishes might provide steadily into a prizes. Projects build groups. Together national plans |0.21|0.13|2004001|edu packimporto #1|3|sports-apparel|2|Men|535|callyeseable|medium|184white922458378927|maroon|Box|Unknown|19|ablepriantieing| +8533|AAAAAAAAFFBCAAAA|1997-10-27||Illustrations|0.54|0.27|8002004|importonameless #4|2|baseball|8|Sports|764|esecallyation|N/A|24034325plum35130371|dark|Dozen|Unknown|8|pripriantieing| +8534|AAAAAAAAGFBCAAAA|1997-10-27|2000-10-26|Details might not heat much against a vehicles. Successful scientists perform. Indeed very users keep high practices; usually low conferences should |4.87|1.85|5002001|importoscholar #1|2|country|5|Music|443|prieseese|N/A|0605698383thistle333|plum|Cup|Unknown|4|esepriantieing| +8535|AAAAAAAAGFBCAAAA|2000-10-27||Civil relationships ensure accordingly in a amounts. Degrees wish subsequent thousands. Forces continue separately; all required trees would not make highly assistant books. Satisfactorily inadequate|9.85|6.69|8012006|importomaxi #6|12|guns|8|Sports|821|prieseese|N/A|0605698383thistle333|sky|Cup|Unknown|18|antipriantieing| +8536|AAAAAAAAIFBCAAAA|1997-10-27|1999-10-27|Colleague|3.23|2.29|1001001|amalgamalg #1|1|dresses|1|Women|6|cally|large|739962khaki799051265|peru|Carton|Unknown|39|callypriantieing| +8537|AAAAAAAAIFBCAAAA|1999-10-28|2001-10-26|Colleague|4.10|3.69|4001002|amalgedu pack #2|1|womens|4|Shoes|531|cally|small|739962khaki799051265|rose|Gross|Unknown|67|ationpriantieing| +8538|AAAAAAAAIFBCAAAA|2001-10-27||Colleague|39.99|3.69|2003001|exportiimporto #1|1|pants|2|Men|531|oughtprianti|petite|739962khaki799051265|seashell|Dozen|Unknown|2|eingpriantieing| +8539|AAAAAAAALFBCAAAA|1997-10-27||Separate boys light only national samples. Other, given lengths include only under natural circumstance|1.71|1.07|7009010|maxibrand #10|9|mattresses|7|Home|592|ablen stanti|N/A|738navajo55796897243|tomato|Dozen|Unknown|26|n stpriantieing| +8540|AAAAAAAAMFBCAAAA|1997-10-27|2000-10-26|So coastal schools add hard from a developments. Ready, large representatives moderate. There simple hundreds restructure greatly in the years. Only other changes would try ago ill inevitable clo|1.36|0.88|8011001|amalgmaxi #1|11|archery|8|Sports|237|ationpriable|N/A|28cornflower90837906|salmon|Gross|Unknown|13|bareseantieing| +8541|AAAAAAAAMFBCAAAA|2000-10-27||So coastal schools add hard from a developments. Ready, large representatives moderate. There simple hundreds restructure greatly in the years. Only other changes would try ago ill inevitable clo|6.13|0.88|8009002|maxinameless #2|9|optics|8|Sports|237|ationpriable|N/A|28cornflower90837906|papaya|Carton|Unknown|27|oughteseantieing| +8542|AAAAAAAAOFBCAAAA|1997-10-27|1999-10-27|Only automatic principles used to accept too hot colleagues. Modern arguments enjoy general, likely respondents; good years should look |3.81|3.27|4002001|importoedu pack #1|2|mens|4|Shoes|75|antiation|medium|2374954991salmon2486|rosy|Bunch|Unknown|37|ableeseantieing| +8543|AAAAAAAAOFBCAAAA|1999-10-28|2001-10-26|Only automatic principles used to accept too hot colleagues. Modern arguments enjoy general, likely respondents; good years should look |7.49|2.24|4002001|amalgmaxi #12|1|arts|9|Books|279|n stationable|N/A|85365080467orchid202|tomato|Oz|Unknown|36|prieseantieing| +8544|AAAAAAAAOFBCAAAA|2001-10-27||Global officials should not break much in a women. Apparent payments lie ashamed courses. Further irish cities would play various studies. Materials should attract. Big, capital glasses behave. Days |9.45|6.70|4002001|amalgscholar #1|1|rock|5|Music|715|antioughtation|N/A|85365080467orchid202|light|Each|Unknown|62|eseeseantieing| +8545|AAAAAAAABGBCAAAA|1997-10-27||Recent, detailed resources derive obvious, white families. Often full hours used |4.77|2.76|5001002|amalgscholar #2|1|rock|5|Music|313|prioughtpri|N/A|8567lime608836990380|magenta|N/A|Unknown|50|antieseantieing| +8546|AAAAAAAACGBCAAAA|1997-10-27|2000-10-26|Basic, british rules can|8.05|3.94|2001001|amalgimporto #1|1|accessories|2|Men|373|priationpri|medium|073972pale5571909538|azure|Each|Unknown|15|callyeseantieing| +8547|AAAAAAAACGBCAAAA|2000-10-27||Basic, british rules can|4.80|1.72|2001001|corpunivamalg #14|6|musical|10|Electronics|373|priationpri|N/A|073972pale5571909538|wheat|Bundle|Unknown|16|ationeseantieing| +8548|AAAAAAAAEGBCAAAA|1997-10-27|1999-10-27|Clothes mig|0.91|0.36|9010005|univunivamalg #5|10|travel|9|Books|379|n stationpri|N/A|50247514386peru90867|orchid|Carton|Unknown|74|eingeseantieing| +8549|AAAAAAAAEGBCAAAA|1999-10-28|2001-10-26|Clothes mig|3.54|2.79|9010005|exportiedu pack #2|10|kids|4|Shoes|379|n stationpri|medium|97920smoke2328106459|peach|Oz|Unknown|41|n steseantieing| +8550|AAAAAAAAEGBCAAAA|2001-10-27||Clothes mig|8.16|7.09|9010005|amalgimporto #1|10|accessories|2|Men|652|ableantically|petite|737550235445321snow8|peach|Unknown|Unknown|69|barantiantieing| +8551|AAAAAAAAHGBCAAAA|1997-10-27||There other applications should not leave about slightly industrial actions. Never appropriate hopes ought to amuse solid, complete developments. Ver|8.08|7.02|5004002|edu packscholar #2|4|classical|5|Music|552|ableantianti|N/A|441012sky29038858755|wheat|Ounce|Unknown|87|oughtantiantieing| +8552|AAAAAAAAIGBCAAAA|1997-10-27|2000-10-26|Liberal, civil customers refuse. For the most part real areas should ask mainly carefully|6.46|4.97|9007009|brandmaxi #9|7|reference|9|Books|24|eseable|N/A|1764625998sandy16586|saddle|Box|Unknown|72|ableantiantieing| +8553|AAAAAAAAIGBCAAAA|2000-10-27||Children|7.75|4.97|2001002|amalgimporto #2|7|accessories|2|Men|51|eseable|extra large|1764625998sandy16586|sky|Ounce|Unknown|34|priantiantieing| +8554|AAAAAAAAKGBCAAAA|1997-10-27|1999-10-27|Concerned, complete characters must not appoint in order secondary parents. More cold lands see other, economic sides. Old volumes get normally political judges. Even psychiatric years commit new co|1.65|1.15|5003001|exportischolar #1|3|pop|5|Music|42|ableese|N/A|06181014snow60500591|smoke|Oz|Unknown|25|eseantiantieing| +8555|AAAAAAAAKGBCAAAA|1999-10-28|2001-10-26|Concerned, complete characters must not appoint in order secondary parents. More cold lands see other, economic sides. Old volumes get normally political judges. Even psychiatric years commit new co|2.87|1.15|3004002|edu packexporti #2|4|school-uniforms|3|Children|46|callyese|small|00693464violet380495|tan|Each|Unknown|17|antiantiantieing| +8556|AAAAAAAAKGBCAAAA|2001-10-27||Concerned, complete characters must not appoint in order secondary parents. More cold lands see other, economic sides. Old volumes get normally political judges. Even psychiatric years commit new co|4.32|1.15|3004002|scholarbrand #7|5|blinds/shades|7|Home|46|callyese|N/A|9533372615violet8124|thistle|Carton|Unknown|3|callyantiantieing| +8557|AAAAAAAANGBCAAAA|1997-10-27||Individual children show unlikely years; capital, southern police might not go so collective, military individuals. Long serious years lie severely local, similar others. Poll|7.78|3.57|4002002|importoedu pack #2|2|mens|4|Shoes|132|ablepriought|large|0001145946rose283864|wheat|Ton|Unknown|63|ationantiantieing| +8558|AAAAAAAAOGBCAAAA|1997-10-27|2000-10-26|Australian, |6.99|5.45|2004001|edu packimporto #1|4|sports-apparel|2|Men|211|oughtoughtable|small|11261748914340snow44|papaya|Tbl|Unknown|14|eingantiantieing| +8559|AAAAAAAAOGBCAAAA|2000-10-27||Nice, professional trends used to invest. Findings will not know clearly individual doors. Now crucial estimates keep so as usual financia|9.33|5.78|2004001|exportiimporto #2|3|pants|2|Men|211|oughtoughtable|medium|219604242183sienna55|spring|Dram|Unknown|3|n stantiantieing| +8560|AAAAAAAAAHBCAAAA|1997-10-27|1999-10-27|Earnings cannot understand for example even top police. Red, sexual eyes dispose nearly in a results. Real, principal members like from a years. |0.73|0.64|2003001|exportiimporto #1|3|pants|2|Men|28|eingable|petite|41561755426spring940|tan|Bundle|Unknown|4|barcallyantieing| +8561|AAAAAAAAAHBCAAAA|1999-10-28|2001-10-26|Women would move both on a activities. Personal forms may not provide later japanese charges. Generally complex difficulties make in a shares. Places weaken possible, clear w|3.47|1.11|7005002|scholarbrand #2|5|blinds/shades|7|Home|68|eingable|N/A|41561755426spring940|rose|Dram|Unknown|39|oughtcallyantieing| +8562|AAAAAAAAAHBCAAAA|2001-10-27||Women would move both on a activities. Personal forms may not provide later japanese charges. Generally complex difficulties make in a shares. Places weaken possible, clear w|2.56|1.58|10004012|edu packunivamalg #12|4|audio|10|Electronics|440|bareseese|N/A|41561755426spring940|ghost|Oz|Unknown|43|ablecallyantieing| +8563|AAAAAAAADHBCAAAA|1997-10-27||No longer aware ideas would not finance at a answers. Satisfactory children study away regular dogs; fast, new students could not look never |7.85|4.00|6013004|exportibrand #4|13|loose stones|6|Jewelry|520|barableanti|N/A|141609deep7776409002|powder|Gram|Unknown|8|pricallyantieing| +8564|AAAAAAAAEHBCAAAA|1997-10-27|2000-10-26|Sooner young nights may not sign substances. Effective months wait too reasons. Definite|0.33|0.09|1003001|exportiamalg #1|3|maternity|1|Women|229|n stableable|petite|1pink977747298417980|rose|Bunch|Unknown|50|esecallyantieing| +8565|AAAAAAAAEHBCAAAA|2000-10-27||Big letters convey yet obvious skills. Extremely poor elections prot|2.55|0.79|1003001|importoscholar #2|3|country|5|Music|566|n stableable|N/A|1pink977747298417980|pale|Ounce|Unknown|12|anticallyantieing| +8566|AAAAAAAAGHBCAAAA|1997-10-27|1999-10-27|International, tall things shall try special, new forces; popular places raise yet powerful examples. Assistant, changing days ask in the hands; advantages ought to know to a sections.|6.12|5.26|6009007|maxicorp #7|9|womens watch|6|Jewelry|199|n stn stought|N/A|17156898sky074800603|tan|Case|Unknown|17|callycallyantieing| +8567|AAAAAAAAGHBCAAAA|1999-10-28|2001-10-26|As well national sports may expect s|2.94|1.55|3001002|amalgexporti #2|9|newborn|3|Children|73|n stn stought|medium|09977snow71763568213|red|Case|Unknown|19|ationcallyantieing| +8568|AAAAAAAAGHBCAAAA|2001-10-27||Small firms should not receive by the officials. Other children might not take so years. Local measures would not see national, private fees. Loyal, black com|2.56|1.55|3001002|edu packedu pack #1|4|athletic|4|Shoes|73|n stn stought|extra large|09977snow71763568213|slate|Box|Unknown|87|eingcallyantieing| +8569|AAAAAAAAJHBCAAAA|1997-10-27||Open, future cattle should develop indeed funds. Healthy years shall let|2.31|0.71|3002002|importoexporti #2|2|infants|3|Children|71|oughtation|medium|5104973770snow263716|ghost|Ton|Unknown|20|n stcallyantieing| +8570|AAAAAAAAKHBCAAAA|1997-10-27|2000-10-26|Cases will not explain al|3.37|2.69|8015001|scholarmaxi #1|15|fishing|8|Sports|615|antioughtcally|N/A|01727073royal9346047|thistle|Bundle|Unknown|25|barationantieing| +8571|AAAAAAAAKHBCAAAA|2000-10-27||Cases will not explain al|8.36|2.69|2004002|edu packimporto #2|4|sports-apparel|2|Men|353|priantipri|medium|263slate940330668748|sky|Unknown|Unknown|18|oughtationantieing| +8572|AAAAAAAAMHBCAAAA|1997-10-27|1999-10-27|Notes take representative, premier assessments. Related, federal homes |74.63|29.10|2003001|exportiimporto #1|3|pants|2|Men|940|baresen st|small|8884yellow5730754100|ivory|Box|Unknown|36|ableationantieing| +8573|AAAAAAAAMHBCAAAA|1999-10-28|2001-10-26|Royal forces should end. Growing divisions ought to meet simply so local limit|1.25|29.10|6006006|corpcorp #6|6|rings|6|Jewelry|254|eseantiable|N/A|3833487238smoke16344|snow|Oz|Unknown|3|priationantieing| +8574|AAAAAAAAMHBCAAAA|2001-10-27||However only rights shift only; royal, black developments protect for the letters. Rises pursue. Internal fires can run especially. Charges lose vast,|7.31|29.10|6006006|importoimporto #1|6|shirts|2|Men|136|eseantiable|extra large|031230840098salmon20|saddle|Gross|Unknown|46|eseationantieing| +8575|AAAAAAAAPHBCAAAA|1997-10-27||Future birds must not boost thus by a objectives; also good centuries used to relax a bit on a women. Private, appropriate lines see maybe regulations; perfect, old examples withdraw|8.68|4.68|1004002|edu packamalg #2|4|swimwear|1|Women|300|barbarpri|petite|3917turquoise7321678|red|Box|Unknown|66|antiationantieing| +8576|AAAAAAAAAIBCAAAA|1997-10-27|2000-10-26|Directors used to fail all but western samples; impressive, special friends see mechanisms. Details leave. Months shall not convert by a policies. Simple statements know abroad massi|2.88|2.10|6006001|corpcorp #1|6|rings|6|Jewelry|280|bareingable|N/A|7smoke12873745874619|tan|Pallet|Unknown|12|callyationantieing| +8577|AAAAAAAAAIBCAAAA|2000-10-27||Directors used to fail all but western samples; impressive, special friends see mechanisms. Details leave. Months shall not convert by a policies. Simple statements know abroad massi|3.84|2.49|5003002|exportischolar #2|6|pop|5|Music|280|bareingable|N/A|037816tomato12275070|pale|Ton|Unknown|31|ationationantieing| +8578|AAAAAAAACIBCAAAA|1997-10-27|1999-10-27|At once good friends limit. Too simple stations|1.88|1.44|9007011|brandmaxi #11|7|reference|9|Books|353|priantipri|N/A|16900838tan229115309|violet|Case|Unknown|60|eingationantieing| +8579|AAAAAAAACIBCAAAA|1999-10-28|2001-10-26|Automatic years should run only lively categories; ever human items|7.16|2.72|9007011|edu packexporti #2|4|school-uniforms|3|Children|641|oughtesecally|medium|16900838tan229115309|tan|Dozen|Unknown|66|n stationantieing| +8580|AAAAAAAACIBCAAAA|2001-10-27||Automatic years should run only lively categories; ever human items|4.56|2.69|9007011|brandbrand #1|4|decor|7|Home|243|prieseable|N/A|973858352743snow4298|khaki|N/A|Unknown|11|bareingantieing| +8581|AAAAAAAAFIBCAAAA|1997-10-27||Children take. About entire respects trade maybe still long trends. Basic, subtle courts avoid statements. Managers would help then visual beliefs. Liberal, severe republics could sit dark, disastr|3.18|1.59|6010006|univbrand #6|10|jewelry boxes|6|Jewelry|372|ableationpri|N/A|641062494996almond25|white|Dozen|Unknown|14|oughteingantieing| +8582|AAAAAAAAGIBCAAAA|1997-10-27|2000-10-26|Victoria|4.68|2.80|5004001|edu packscholar #1|4|classical|5|Music|254|eseantiable|N/A|14614698spring407214|sky|Box|Unknown|18|ableeingantieing| +8583|AAAAAAAAGIBCAAAA|2000-10-27||Reliable papers picture ever more financial hours. More full hours write often effective, different sources. National soldiers may not introduce often to a files. Anyway personal roles could apply ner|7.06|5.01|5004001|corpunivamalg #16|4|musical|10|Electronics|634|eseprically|N/A|14614698spring407214|pink|Carton|Unknown|37|prieingantieing| +8584|AAAAAAAAIIBCAAAA|1997-10-27|1999-10-27|Great |85.17|52.80|6011001|amalgbrand #1|11|semi-precious|6|Jewelry|27|ationable|N/A|84454254305linen7523|khaki|N/A|Unknown|16|eseeingantieing| +8585|AAAAAAAAIIBCAAAA|1999-10-28|2001-10-26|Conditions broaden as shows; welsh years should not arrive |88.80|52.80|6011001|amalgscholar #2|11|rock|5|Music|162|ationable|N/A|84454254305linen7523|turquoise|Gross|Unknown|35|antieingantieing| +8586|AAAAAAAAIIBCAAAA|2001-10-27||And so on heavy products need english, interested men. Inland wealthy companies cannot happen ever domestic, american pupils|9.63|52.80|2001001|amalgimporto #1|1|accessories|2|Men|162|ationable|small|84454254305linen7523|misty|Gram|Unknown|19|callyeingantieing| +8587|AAAAAAAALIBCAAAA|1997-10-27||Prices want near flo|1.92|0.86|7014008|edu packnameless #8|14|glassware|7|Home|951|oughtantin st|N/A|43769sandy9321704350|violet|Tbl|Unknown|15|ationeingantieing| +8588|AAAAAAAAMIBCAAAA|1997-10-27|2000-10-26|Views present rapidly in the relations. Average winners could fall double stations; also corresponding heroes promote direct,|3.17|1.55|9011009|amalgunivamalg #9|11|cooking|9|Books|600|barbarcally|N/A|7063metallic14971802|rose|Gross|Unknown|53|eingeingantieing| +8589|AAAAAAAAMIBCAAAA|2000-10-27||Less poor rights could see years. Potatoes might meet earlier large activities. Owners would concentrate even important objects. Views squeeze early specific posts; possible countries discove|0.19|1.55|3003002|exportiexporti #2|3|toddlers|3|Children|230|barpriable|extra large|7063metallic14971802|sky|Pallet|Unknown|32|n steingantieing| +8590|AAAAAAAAOIBCAAAA|1997-10-27|1999-10-27|Nearly full speakers shall not live lexical, chief figures. Excepti|3.91|2.73|2001001|amalgimporto #1|1|accessories|2|Men|293|prin stable|petite|74333snow87689504525|tan|Each|Unknown|1|barn stantieing| +8591|AAAAAAAAOIBCAAAA|1999-10-28|2001-10-26|Nearly full speakers shall not live lexical, chief figures. Excepti|79.14|54.60|2003002|exportiimporto #2|1|pants|2|Men|293|prin stable|extra large|indian16241962135626|thistle|Bundle|Unknown|73|oughtn stantieing| +8592|AAAAAAAAOIBCAAAA|2001-10-27||Nearly full speakers shall not live lexical, chief figures. Excepti|9.57|54.60|7015003|scholarnameless #3|1|tables|7|Home|293|prin stable|N/A|indian16241962135626|khaki|Carton|Unknown|20|ablen stantieing| +8593|AAAAAAAABJBCAAAA|1997-10-27||Contacts avoid better british, sad men. New, autonomous abilities must appeal only accounts. European, recent companies tell approxim|5.10|3.67|2002002|importoimporto #2|2|shirts|2|Men|798|eingn station|small|2944rosy614880051598|moccasin|Tsp|Unknown|25|prin stantieing| +8594|AAAAAAAACJBCAAAA|1997-10-27|2000-10-26|Negative, late offices hire just sound buyers. Doubtful seconds would die just structures. Higher i|0.94|0.34|2003001|exportiimporto #1|3|pants|2|Men|528|eingableanti|medium|36seashell0072494741|powder|Lb|Unknown|91|esen stantieing| +8595|AAAAAAAACJBCAAAA|2000-10-27||Negative, late offices hire just sound buyers. Doubtful seconds would die just structures. Higher i|0.90|0.34|2003001|maxinameless #6|9|optics|8|Sports|528|eingableanti|N/A|36seashell0072494741|smoke|Unknown|Unknown|36|antin stantieing| +8596|AAAAAAAAEJBCAAAA|1997-10-27|1999-10-27|Subjects may derive later interested, big goals. Jus|7.29|5.75|2002001|importoimporto #1|2|shirts|2|Men|234|esepriable|petite|566501149salmon82700|navy|Case|Unknown|6|callyn stantieing| +8597|AAAAAAAAEJBCAAAA|1999-10-28|2001-10-26|Subjects may derive later interested, big goals. Jus|7.27|5.75|8015008|scholarmaxi #8|2|fishing|8|Sports|442|esepriable|N/A|67673782712sienna403|tomato|Oz|Unknown|37|ationn stantieing| +8598|AAAAAAAAEJBCAAAA|2001-10-27||Subjects may derive later interested, big goals. Jus|6.14|3.74|8015008|amalgbrand #9|1|bathroom|7|Home|442|esepriable|N/A|67673782712sienna403|peach|Unknown|Unknown|15|eingn stantieing| +8599|AAAAAAAAHJBCAAAA|1997-10-27||Everyday, available functions must go too; particular, strict processes would increase so common years. International, absolute cases consider to the questions. Categories|3.11|1.77|4001002|amalgedu pack #2|1|womens|4|Shoes|153|priantiought|medium|9715799477192powder0|white|Carton|Unknown|48|n stn stantieing| +8600|AAAAAAAAIJBCAAAA|1997-10-27|2000-10-26|Terms will see soon. D|5.17|2.22|6010001|univbrand #1|10|jewelry boxes|6|Jewelry|685|antieingcally|N/A|4787wheat29697146457|yellow|Dram|Unknown|32|barbarcallyeing| +8601|AAAAAAAAIJBCAAAA|2000-10-27||Centres affect in a structures. Explanations should remember rather rich, good forms. Companies shall build yet hastily various systems; papers must not begin ways. Useful students used to extract w|3.39|2.06|7015002|scholarnameless #2|15|tables|7|Home|53|prianti|N/A|8154921rose655837632|cyan|Ounce|Unknown|22|oughtbarcallyeing| +8602|AAAAAAAAKJBCAAAA|1997-10-27|1999-10-27|Particular, grey items regain then there satisfactory reactions. Readily only men give neither. Painful orders privatise teache|8.14|7.00|10014001|edu packamalgamalg #1|14|automotive|10|Electronics|76|callyation|N/A|4235703349160violet5|peach|Dozen|Unknown|12|ablebarcallyeing| +8603|AAAAAAAAKJBCAAAA|1999-10-28|2001-10-26|Different, safe bills implement more final, real minutes; common, protective buildings double; false, available branches ought to make more local, r|0.34|0.11|3002002|importoexporti #2|14|infants|3|Children|76|callyation|small|4235703349160violet5|rose|Pound|Unknown|10|pribarcallyeing| +8604|AAAAAAAAKJBCAAAA|2001-10-27||Bills would include well hours; soon other children give also social, high hands. Tough months will not solve parties. Loose, great eyes might learn perhaps marin|5.95|0.11|6010005|univbrand #5|10|jewelry boxes|6|Jewelry|76|callyation|N/A|382959569red98150928|turquoise|Pallet|Unknown|22|esebarcallyeing| +8605|AAAAAAAANJBCAAAA|1997-10-27||Too dangerous arrangeme|3.29|2.53|1004002|edu packamalg #2|4|swimwear|1|Women|211|oughtoughtable|extra large|42523968snow39513240|saddle|Tsp|Unknown|30|antibarcallyeing| +8606|AAAAAAAAOJBCAAAA|1997-10-27|2000-10-26|Criminal employers suggest easily determined goods. Correct, wrong expenses used to come earlier to a varieties. Powerful, confident sons should not move visits. More |47.58|20.93|3002001|importoexporti #1|2|infants|3|Children|889|n steingeing|N/A|49rose96607005536849|royal|Dozen|Unknown|8|callybarcallyeing| +8607|AAAAAAAAOJBCAAAA|2000-10-27||Final, exotic students might not fight practically into the conclusions. Ago other sources cannot include present officers.|8.22|2.79|3002001|scholarunivamalg #4|2|fiction|9|Books|18|eingought|N/A|323586thistle5285206|misty|N/A|Unknown|25|ationbarcallyeing| +8608|AAAAAAAAAKBCAAAA|1997-10-27|1999-10-27|Even southern orders think british, poor committees. Successful, poor books ought to deny real others. There neces|0.70|0.23|2001001|amalgimporto #1|1|accessories|2|Men|72|ableation|petite|18pale03076546747198|yellow|Bundle|Unknown|38|eingbarcallyeing| +8609|AAAAAAAAAKBCAAAA|1999-10-28|2001-10-26|Even southern orders think british, poor committees. Successful, poor books ought to deny real others. There neces|33.31|21.98|10009008|maxiunivamalg #8|1|televisions|10|Electronics|515|antioughtanti|N/A|86134443651honeydew6|gainsboro|Dram|Unknown|14|n stbarcallyeing| +8610|AAAAAAAAAKBCAAAA|2001-10-27||Even southern orders think british, poor committees. Successful, poor books ought to deny real others. There neces|8.87|7.80|10009008|importoexporti #1|2|infants|3|Children|515|antioughtanti|petite|86134443651honeydew6|indian|Bundle|Unknown|1|baroughtcallyeing| +8611|AAAAAAAADKBCAAAA|1997-10-27||Sole windows might send highly for a users. Academic risks may get often no doubt possible months. Lines find in a clothes. Sexual, proper times will decline. Kinds ought to improve dis|6.09|3.59|1003002|exportiamalg #2|3|maternity|1|Women|944|eseesen st|extra large|rose7237966930280681|moccasin|Unknown|Unknown|11|oughtoughtcallyeing| +8612|AAAAAAAAEKBCAAAA|1997-10-27|2000-10-26|Ho|0.49|0.19|1002001|importoamalg #1|2|fragrances|1|Women|111|oughtoughtought|petite|72045burlywood641028|royal|Oz|Unknown|5|ableoughtcallyeing| +8613|AAAAAAAAEKBCAAAA|2000-10-27||Hopefully black conditions see between a notes. Organisational, central offices may look likely, deep years. Responsible schemes might ring|6.01|0.19|6006006|corpcorp #6|6|rings|6|Jewelry|111|oughtoughtought|N/A|72045burlywood641028|dodger|Cup|Unknown|12|prioughtcallyeing| +8614|AAAAAAAAGKBCAAAA|1997-10-27|1999-10-27|Countries analyse at all poor, rare years; inner, hard songs follow so nearby pensioners. Governments will not gain almost physical versions. Correct, different |1.59|0.85|6013007|exportibrand #7|13|loose stones|6|Jewelry|626|callyablecally|N/A|58408835sienna502966|indian|N/A|Unknown|24|eseoughtcallyeing| +8615|AAAAAAAAGKBCAAAA|1999-10-28|2001-10-26|Primary, honest jobs say religious millions; revolutionary, common plans use |1.34|0.85|10012014|importoamalgamalg #14|12|monitors|10|Electronics|457|ationantiese|N/A|58408835sienna502966|white|Bundle|Unknown|61|antioughtcallyeing| +8616|AAAAAAAAGKBCAAAA|2001-10-27||Fast, sensible lengths ought to go for a feelings. Considerable, old years can remove brutally situations; often accurate consequences ought to confine of course indian attacks. Har|0.91|0.76|10012014|amalgimporto #1|1|accessories|2|Men|47|ationantiese|large|874245396169sienna99|purple|Gross|Unknown|2|callyoughtcallyeing| +8617|AAAAAAAAJKBCAAAA|1997-10-27||Troops could declare. Yet likely units escape as. Annual, early others see so just domestic expenses. Possible, unlikely drugs make much financial, young governors. Dramatic programs should not roll |1.31|0.94|10007016|brandunivamalg #16|7|personal|10|Electronics|135|antipriought|N/A|65612hot159296101287|royal|Ounce|Unknown|9|ationoughtcallyeing| +8618|AAAAAAAAKKBCAAAA|1997-10-27|2000-10-26|Childre|4.75|3.46|2003001|exportiimporto #1|3|pants|2|Men|202|ablebarable|extra large|5273800027819royal04|thistle|Each|Unknown|44|eingoughtcallyeing| +8619|AAAAAAAAKKBCAAAA|2000-10-27||Interests fill charg|5.12|3.94|9004004|edu packmaxi #4|4|entertainments|9|Books|202|ablebarable|N/A|5273800027819royal04|smoke|Tsp|Unknown|41|n stoughtcallyeing| +8620|AAAAAAAAMKBCAAAA|1997-10-27|1999-10-27|Quiet, irish questions might crack almost head ways; capital attempts ought to earn with a circumstances. Meetings receive bad roads. Observatio|1.15|1.03|10004002|edu packunivamalg #2|4|audio|10|Electronics|305|antibarpri|N/A|2royal92464679846796|spring|Tsp|Unknown|2|barablecallyeing| +8621|AAAAAAAAMKBCAAAA|1999-10-28|2001-10-26|Quiet, irish questions might crack almost head ways; capital attempts ought to earn with a circumstances. Meetings receive bad roads. Observatio|3.82|1.03|10004002|importoexporti #2|4|infants|3|Children|305|antibarpri|small|2royal92464679846796|saddle|Ton|Unknown|22|oughtablecallyeing| +8622|AAAAAAAAMKBCAAAA|2001-10-27||Flames come under mature attempts; present institutions settle recently at a hopes. Suddenly attractive uses must apply behin|8.86|3.36|10004002|exporticorp #7|4|gold|6|Jewelry|91|oughtn st|N/A|453702midnight847944|seashell|Pound|Unknown|18|ableablecallyeing| +8623|AAAAAAAAPKBCAAAA|1997-10-27||Away full counties seek at the goals. Apparently daily police maintain just. Gold, financial companies should improve rights; public, critical states should not lead as there tradi|9.74|7.49|5003002|exportischolar #2|3|pop|5|Music|636|callyprically|N/A|0626140spring9839467|rose|Case|Unknown|9|priablecallyeing| +8624|AAAAAAAAALBCAAAA|1997-10-27|2000-10-26|Less responsible feelings will use however pregnant measures. Simply honest partners show most less major hands. Federal, international employees should sell more banks. Originally curren|7.42|6.30|6002001|importocorp #1|2|diamonds|6|Jewelry|792|ablen station|N/A|19thistle69180896417|navy|Bundle|Unknown|51|eseablecallyeing| +8625|AAAAAAAAALBCAAAA|2000-10-27||Once only losses must not let other officials. Pleased, gradual materials will relax regularly from a thousands; complex days must dislike indeed indirect days; common pounds ought to think major ac|0.21|6.30|6002001|edu packedu pack #2|2|athletic|4|Shoes|792|ablen station|medium|1764458olive69277907|sienna|Ounce|Unknown|34|antiablecallyeing| +8626|AAAAAAAACLBCAAAA|1997-10-27|1999-10-27|Really geographical bars come from the demands. For example severe benefits would not come by a concentrations.|9.28|6.31|4003001|exportiedu pack #1|3|kids|4|Shoes|39|n stpri|large|386919572red54874035|white|Unknown|Unknown|3|callyablecallyeing| +8627|AAAAAAAACLBCAAAA|1999-10-28|2001-10-26|Really geographical bars come from the demands. For example severe benefits would not come by a concentrations.|20.76|6.31|4003001|exportiedu pack #2|3|kids|4|Shoes|39|n stpri|medium|386919572red54874035|tan|Each|Unknown|77|ationablecallyeing| +8628|AAAAAAAACLBCAAAA|2001-10-27||Really geographical bars come from the demands. For example severe benefits would not come by a concentrations.|3.74|6.31|2004001|edu packimporto #1|3|sports-apparel|2|Men|193|prin stought|economy|386919572red54874035|steel|Box|Unknown|15|eingablecallyeing| +8629|AAAAAAAAFLBCAAAA|1997-10-27||Short men cross available flowers. Clear men see in the steps. Upper, natural railways insure new, pr|5.94|4.81|5003002|exportischolar #2|3|pop|5|Music|694|esen stcally|N/A|4389932539seashell32|ivory|Box|Unknown|23|n stablecallyeing| +8630|AAAAAAAAGLBCAAAA|1997-10-27|2000-10-26|Comfortable, regional rights throw. Then attractive poems lead yet convenient boys. English, necessary opponents see mental proposals; also usual years |7.26|2.25|3001001|amalgexporti #1|1|newborn|3|Children|4|ese|petite|22622068312034plum01|sienna|Dram|Unknown|30|barpricallyeing| +8631|AAAAAAAAGLBCAAAA|2000-10-27||Early, loose devices organize thus expected, spiritual resources. Pockets stick even democratic police. Electric forms know services. Sad, great months reduce direct. Centr|7.88|2.25|3001001|maxiunivamalg #13|1|televisions|10|Electronics|199|ese|N/A|7704669746673puff336|gainsboro|Lb|Unknown|14|oughtpricallyeing| +8632|AAAAAAAAILBCAAAA|1997-10-27|1999-10-27|Tomorrow young schemes would not recognise understandably lips. Men might become just. Much public days make surely large, daily facts. Rela|4.76|3.99|1004001|edu packamalg #1|4|swimwear|1|Women|542|ableeseanti|petite|34425559983036tan854|ghost|Dram|Unknown|3|ablepricallyeing| +8633|AAAAAAAAILBCAAAA|1999-10-28|2001-10-26|Tomorrow young schemes would not recognise understandably lips. Men might become just. Much public days make surely large, daily facts. Rela|2.98|1.10|5003002|exportischolar #2|3|pop|5|Music|542|ableeseanti|N/A|967071209wheat034332|steel|Unknown|Unknown|19|pripricallyeing| +8634|AAAAAAAAILBCAAAA|2001-10-27||Tomorrow young schemes would not recognise understandably lips. Men might become just. Much public days make surely large, daily facts. Rela|44.22|1.10|5003002|edu packedu pack #1|4|athletic|4|Shoes|211|ableeseanti|extra large|967071209wheat034332|turquoise|Pound|Unknown|81|esepricallyeing| +8635|AAAAAAAALLBCAAAA|1997-10-27||Groups allow largely unknown services. Politi|63.84|30.00|6003004|exporticorp #4|3|gold|6|Jewelry|543|prieseanti|N/A|5066689445red2830225|steel|Lb|Unknown|2|antipricallyeing| +8636|AAAAAAAAMLBCAAAA|1997-10-27|2000-10-26|Nervous, alt|9.38|8.06|8016007|corpmaxi #7|16|golf|8|Sports|458|eingantiese|N/A|666101611peru7818909|dim|Dozen|Unknown|10|callypricallyeing| +8637|AAAAAAAAMLBCAAAA|2000-10-27||Minor, new houses ought to decide more awards. Demonstrations shall suffer just from a allies; soon polite y|1.04|0.32|6012006|importobrand #6|16|costume|6|Jewelry|458|eingantiese|N/A|666101611peru7818909|peach|Pallet|Unknown|18|ationpricallyeing| +8638|AAAAAAAAOLBCAAAA|1997-10-27|1999-10-27|Able services know books. Little new coins might not protect; social, young affairs account too into the|7.57|6.13|9008011|namelessmaxi #11|8|romance|9|Books|156|callyantiought|N/A|6599002855529white94|smoke|Box|Unknown|10|eingpricallyeing| +8639|AAAAAAAAOLBCAAAA|1999-10-28|2001-10-26|Able services know books. Little new coins might not protect; social, young affairs account too into the|5.57|1.94|9010012|univunivamalg #12|8|travel|9|Books|903|pribarn st|N/A|6599002855529white94|puff|Bundle|Unknown|3|n stpricallyeing| +8640|AAAAAAAAOLBCAAAA|2001-10-27||Able services know books. Little new coins might not protect; social, young affairs account too into the|5.69|2.95|9010012|univnameless #1|10|flatware|7|Home|903|pribarn st|N/A|355877220tomato00531|pink|Unknown|Unknown|34|baresecallyeing| +8641|AAAAAAAABMBCAAAA|1997-10-27||Musica|88.83|71.95|10015006|scholaramalgamalg #6|15|portable|10|Electronics|189|n steingought|N/A|454804yellow88237771|white|Ton|Unknown|2|oughtesecallyeing| +8642|AAAAAAAACMBCAAAA|1997-10-27|2000-10-26|Very front flowers could not get too to a notes. Other models find national, unlikely beings. Sel|62.56|46.29|2003001|exportiimporto #1|3|pants|2|Men|14|eseought|economy|6073186linen69947010|orchid|Carton|Unknown|9|ableesecallyeing| +8643|AAAAAAAACMBCAAAA|2000-10-27||Very front flowers could not get too to a notes. Other models find national, unlikely beings. Sel|17.37|9.20|2003001|importoimporto #2|3|shirts|2|Men|14|eseought|medium|6073186linen69947010|lime|Pallet|Unknown|81|priesecallyeing| +8644|AAAAAAAAEMBCAAAA|1997-10-27|1999-10-27|Right areas tell off the events. Dangerous, other loans might not investigate small children. Large offices might happen right. Static, new expressions used to de|6.39|4.85|7005005|scholarbrand #5|5|blinds/shades|7|Home|375|antiationpri|N/A|9sandy63721351124878|yellow|Carton|Unknown|78|eseesecallyeing| +8645|AAAAAAAAEMBCAAAA|1999-10-28|2001-10-26|Beds shall go well european books. |2.90|2.26|7005005|exportiexporti #2|5|toddlers|3|Children|333|antiationpri|large|84404lavender2675435|snow|Cup|Unknown|21|antiesecallyeing| +8646|AAAAAAAAEMBCAAAA|2001-10-27||Obviously german results must not understand rather weapons. Presumably apparent possibilities take only |1.70|1.51|7005005|exporticorp #7|5|gold|6|Jewelry|726|callyableation|N/A|307375snow1535160706|burlywood|N/A|Unknown|15|callyesecallyeing| +8647|AAAAAAAAHMBCAAAA|1997-10-27||Directors call annual, dry years. Sudden, long bars contemplate recent, political changes. Smooth relations prescribe ethnic, free forces; social options enable; on|0.31|0.12|5002002|importoscholar #2|2|country|5|Music|181|oughteingought|N/A|turquoise99722755367|spring|Box|Unknown|59|ationesecallyeing| +8648|AAAAAAAAIMBCAAAA|1997-10-27|2000-10-26|Over recent build|6.57|5.19|8008009|namelessnameless #9|8|outdoor|8|Sports|170|barationought|N/A|37044193279rose12845|pink|Unknown|Unknown|22|eingesecallyeing| +8649|AAAAAAAAIMBCAAAA|2000-10-27||Nearly daily mechanisms might convince clear, left |9.44|5.19|7011010|amalgnameless #10|8|accent|7|Home|326|barationought|N/A|37044193279rose12845|royal|Unknown|Unknown|93|n stesecallyeing| +8650|AAAAAAAAKMBCAAAA|1997-10-27|1999-10-27|Permanent, regular records will vote also on a goods. Resolutely true quarters used to se|4.66|3.12|6002003|importocorp #3|2|diamonds|6|Jewelry|36|callypri|N/A|80905478815salmon825|magenta|Case|Unknown|1|baranticallyeing| +8651|AAAAAAAAKMBCAAAA|1999-10-28|2001-10-26|Else available flowers mean; about important contracts get nowadays high, able officials. Differences would get now entirely indivi|81.73|63.74|5003002|exportischolar #2|2|pop|5|Music|36|callypri|N/A|80905478815salmon825|slate|Tbl|Unknown|33|oughtanticallyeing| +8652|AAAAAAAAKMBCAAAA|2001-10-27||Special experiments win harshly after a leaves. Upstairs possible origins check just old |2.78|1.27|5003002|importobrand #5|2|costume|6|Jewelry|172|ableationought|N/A|80905478815salmon825|sienna|Ton|Unknown|45|ableanticallyeing| +8653|AAAAAAAANMBCAAAA|1997-10-27||Specialist, unusual windows put special pieces. Only european dreams get however english, fixed words. British|3.07|2.76|10004001|edu packunivamalg #1|4|audio|10|Electronics|89|n steing|N/A|056761631red82528394|steel|Unknown|Unknown|28|prianticallyeing| +8654|AAAAAAAAOMBCAAAA|1997-10-27|2000-10-26|Available services can ex|3.93|2.16|6004007|edu packcorp #7|4|bracelets|6|Jewelry|703|pribaration|N/A|66hot405088349153558|smoke|Case|Unknown|43|eseanticallyeing| +8655|AAAAAAAAOMBCAAAA|2000-10-27||Foods could retire also to the attitudes. Arrangements earn equally therefore active points. Policies would take hence readers. Months might not replace mentally markets. Socialist types shall not|25.62|10.76|5004002|edu packscholar #2|4|classical|5|Music|703|pribaration|N/A|18925976081163linen1|lace|Tbl|Unknown|25|antianticallyeing| +8656|AAAAAAAAANBCAAAA|1997-10-27|1999-10-27|Other men replace years. Different, major games shall test so various birds. Round, visual wor|0.76|0.68|5001001|amalgscholar #1|1|rock|5|Music|319|n stoughtpri|N/A|894purple45608479618|pale|Gram|Unknown|85|callyanticallyeing| +8657|AAAAAAAAANBCAAAA|1999-10-28|2001-10-26|Other men replace years. Different, major games shall test so various birds. Round, visual wor|4.13|3.55|5001001|univbrand #2|10|jewelry boxes|6|Jewelry|319|n stoughtpri|N/A|894purple45608479618|snow|Box|Unknown|2|ationanticallyeing| +8658|AAAAAAAAANBCAAAA|2001-10-27||Years would not maintain only small songs. Civil, aca|0.58|3.55|5001001|corpmaxi #9|10|golf|8|Sports|319|n stoughtpri|N/A|32162888brown7215845|midnight|Dram|Unknown|12|einganticallyeing| +8659|AAAAAAAADNBCAAAA|1997-10-27||Strong, model changes should see also huge, apparent months. Men would sto|9.85|7.68|1003002|exportiamalg #2|3|maternity|1|Women|204|esebarable|medium|1papaya9265999934577|sienna|Pallet|Unknown|64|n stanticallyeing| +8660|AAAAAAAAENBCAAAA|1997-10-27|2000-10-26|Agencies will pick different authorities. Whole, academic moments will include again perhaps other profits. Months can lay in a effects. Feet must want gentle, central sections. Even visible he|5.71|3.59|9015009|scholarunivamalg #9|15|fiction|9|Books|50|baranti|N/A|916525186steel777514|plum|Cup|Unknown|90|barcallycallyeing| +8661|AAAAAAAAENBCAAAA|2000-10-27||Agencies will pick different authorities. Whole, academic moments will include again perhaps other profits. Months can lay in a effects. Feet must want gentle, central sections. Even visible he|2.70|3.59|9015009|scholarnameless #2|15|tables|7|Home|580|baranti|N/A|498peru9515162946441|thistle|Lb|Unknown|36|oughtcallycallyeing| +8662|AAAAAAAAGNBCAAAA|1997-10-27|1999-10-27|Kids must not know sharp, post-war babies. Democratic alternatives result quite at a activities. Deep, various institutions might not return extremely special,|1.85|1.49|9008011|namelessmaxi #11|8|romance|9|Books|252|ableantiable|N/A|574265417peach056031|smoke|Lb|Unknown|32|ablecallycallyeing| +8663|AAAAAAAAGNBCAAAA|1999-10-28|2001-10-26|Visual costs will play relations. Ideas may claim once years. Conditions starve tight standards. Criteria will agree centrally ways; police set still also diverse developments; for example |3.63|1.48|3002002|importoexporti #2|8|infants|3|Children|443|ableantiable|extra large|574265417peach056031|saddle|Oz|Unknown|62|pricallycallyeing| +8664|AAAAAAAAGNBCAAAA|2001-10-27||Visual costs will play relations. Ideas may claim once years. Conditions starve tight standards. Criteria will agree centrally ways; police set still also diverse developments; for example |7.43|4.01|4004001|edu packedu pack #1|4|athletic|4|Shoes|172|ableantiable|petite|4364338839turquoise9|slate|Bundle|Unknown|14|esecallycallyeing| +8665|AAAAAAAAJNBCAAAA|1997-10-27||Common branches ought to |9.13|6.66|7010006|univnameless #6|10|flatware|7|Home|595|antin stanti|N/A|123187777beige204751|steel|Ounce|Unknown|12|anticallycallyeing| +8666|AAAAAAAAKNBCAAAA|1997-10-27|2000-10-26|Early, powerful towns add mainly english savings. Years assist then new, public colleagues. Things might encounter then right new features|6.89|5.51|9001003|amalgmaxi #3|1|arts|9|Books|465|anticallyese|N/A|9663093goldenrod1367|rose|Dram|Unknown|52|callycallycallyeing| +8667|AAAAAAAAKNBCAAAA|2000-10-27||Pleased hospitals use again inc links; hard, supreme st|5.57|4.56|2004002|edu packimporto #2|4|sports-apparel|2|Men|241|oughteseable|petite|9663093goldenrod1367|blue|Ounce|Unknown|23|ationcallycallyeing| +8668|AAAAAAAAMNBCAAAA|1997-10-27|1999-10-27|Vital numbers speak similar, english appointments. True mothers walk frequently cruel, emotional patterns. Original elections shall find inst|2.62|2.20|4002001|importoedu pack #1|2|mens|4|Shoes|707|ationbaration|small|82271wheat9047271074|purple|Ton|Unknown|12|eingcallycallyeing| +8669|AAAAAAAAMNBCAAAA|1999-10-28|2001-10-26|Vital numbers speak similar, english appointments. True mothers walk frequently cruel, emotional patterns. Original elections shall find inst|7.94|2.20|4002001|importoscholar #2|2|country|5|Music|707|ationbaration|N/A|82271wheat9047271074|orchid|Carton|Unknown|23|n stcallycallyeing| +8670|AAAAAAAAMNBCAAAA|2001-10-27||Plants end eventually french arts; precisely full factors|1.44|2.20|3002001|importoexporti #1|2|infants|3|Children|230|barpriable|medium|090ghost658841329916|peru|Gram|Unknown|24|barationcallyeing| +8671|AAAAAAAAPNBCAAAA|1997-10-27||Managerial seconds m|9.89|6.32|1002002|importoamalg #2|2|fragrances|1|Women|370|barationpri|small|271130920salmon79308|sky|Tbl|Unknown|3|oughtationcallyeing| +8672|AAAAAAAAAOBCAAAA|1997-10-27|2000-10-26|Years should try in line with a conditions. Pp. spend well evenings. Other, afraid sides speculate at a years. Options ought to know leading, app|5.23|1.77|9003009|exportimaxi #9|3|computers|9|Books|750|barantiation|N/A|8623077204052wheat08|snow|Case|Unknown|51|ableationcallyeing| +8673|AAAAAAAAAOBCAAAA|2000-10-27||Mothers could not eliminate at once parents. Big, usual eggs cannot help. Relevant friends lead richly negative weapons. Present clothes map well true potatoes. Especially |2.42|0.91|9003009|exportinameless #4|3|basketball|8|Sports|111|barantiation|N/A|17snow17145576107784|smoke|Tsp|Unknown|29|priationcallyeing| +8674|AAAAAAAACOBCAAAA|1997-10-27|1999-10-27|Proper effects could not seem much royal others. Loyal transactions will replace legal, identical days. At|0.91|0.37|7002005|importobrand #5|2|bedding|7|Home|822|ableableeing|N/A|sienna02446628180626|royal|Gross|Unknown|8|eseationcallyeing| +8675|AAAAAAAACOBCAAAA|1999-10-28|2001-10-26|Elect|9.53|8.57|7002005|edu packamalg #2|4|swimwear|1|Women|822|ableableeing|extra large|sienna02446628180626|rosy|Gram|Unknown|54|antiationcallyeing| +8676|AAAAAAAACOBCAAAA|2001-10-27||Powerful, entire breasts will not find also eyes. Particles may mind properly marvellous, little issues. Bones enable individual, established artists.|1.18|8.57|6016005|corpbrand #5|16|consignment|6|Jewelry|259|ableableeing|N/A|8034spring1109781746|royal|Pallet|Unknown|12|callyationcallyeing| +8677|AAAAAAAAFOBCAAAA|1997-10-27||Just |0.58|0.49|4002002|importoedu pack #2|2|mens|4|Shoes|433|pripriese|small|10694641papaya079626|spring|Pound|Unknown|53|ationationcallyeing| +8678|AAAAAAAAGOBCAAAA|1997-10-27|2000-10-26|Vast colours shall hear. Black senses reduce so into a individuals. Small years handle apparently young mothers. Right, pleased assumptions reduce ill in a fields. Important, sure mi|23.95|15.56|2001001|amalgimporto #1|1|accessories|2|Men|648|eingesecally|medium|46seashell2338105106|seashell|Pallet|Unknown|13|eingationcallyeing| +8679|AAAAAAAAGOBCAAAA|2000-10-27||Subject years must determine. Houses include christian, shallow towns. Publicly st|4.67|15.56|5003002|exportischolar #2|1|pop|5|Music|74|eseation|N/A|46seashell2338105106|green|Gross|Unknown|64|n stationcallyeing| +8680|AAAAAAAAIOBCAAAA|1997-10-27|1999-10-27|Walls would need instead to the times. Somehow early transactions claim. Liable, gay corporations will seem however properly female men. Cars give long in a months.|9.84|5.31|7006001|corpbrand #1|6|rugs|7|Home|176|callyationought|N/A|81907027185813misty4|spring|Box|Unknown|22|bareingcallyeing| +8681|AAAAAAAAIOBCAAAA|1999-10-28|2001-10-26|Walls would need instead to the times. Somehow early transactions claim. Liable, gay corporations will seem however properly female men. Cars give long in a months.|2.86|5.31|5003002|exportischolar #2|6|pop|5|Music|176|callyationought|N/A|81907027185813misty4|plum|Gram|Unknown|96|oughteingcallyeing| +8682|AAAAAAAAIOBCAAAA|2001-10-27||Walls would need instead to the times. Somehow early transactions claim. Liable, gay corporations will seem however properly female men. Cars give long in a months.|3.43|1.71|10006013|corpunivamalg #13|6|musical|10|Electronics|322|ableablepri|N/A|81907027185813misty4|royal|Oz|Unknown|53|ableeingcallyeing| +8683|AAAAAAAALOBCAAAA|1997-10-27||Edges come most high residents. Opponents may not provide perhaps at a details. English, specific minutes obtain from a parts. More able holidays happen deeply. Natural o|2.33|1.28|7003004|exportibrand #4|3|kids|7|Home|445|antieseese|N/A|874731089magenta0516|red|Ton|Unknown|16|prieingcallyeing| +8684|AAAAAAAAMOBCAAAA|1997-10-27|2000-10-26|Occasional areas use along with a numbers. Reasons break increasingly|4.99|1.84|6005005|scholarcorp #5|5|earings|6|Jewelry|160|barcallyought|N/A|5338308yellow8140447|ghost|Box|Unknown|93|eseeingcallyeing| +8685|AAAAAAAAMOBCAAAA|2000-10-27||Considerable problems get special, bad pupils. Words agree then sadly national h|2.47|1.84|6005005|exportiamalg #2|5|maternity|1|Women|617|ationoughtcally|N/A|5338308yellow8140447|violet|Ton|Unknown|24|antieingcallyeing| +8686|AAAAAAAAOOBCAAAA|1997-10-27|1999-10-27|Formerly huge doubts raise alone periods. Soon appropriate winners avoid quite. Concerns arouse even old, christian groups. Less|4.05|1.74|9009011|maximaxi #11|9|science|9|Books|700|barbaration|N/A|9peach58634395280463|thistle|Dozen|Unknown|11|callyeingcallyeing| +8687|AAAAAAAAOOBCAAAA|1999-10-28|2001-10-26|Psychological mountains m|4.93|4.28|9009011|importoscholar #2|9|country|5|Music|700|barbaration|N/A|491391478plum2172705|red|Bunch|Unknown|18|ationeingcallyeing| +8688|AAAAAAAAOOBCAAAA|2001-10-27||Psychological mountains m|5.11|2.86|3004001|edu packexporti #1|4|school-uniforms|3|Children|53|prianti|extra large|1438455pink537165859|turquoise|Each|Unknown|17|eingeingcallyeing| +8689|AAAAAAAABPBCAAAA|1997-10-27||Possible, critical tons shall not keep especially political, used scientists. Just, natural guests may come virtually very concerned members|0.53|0.31|3002002|importoexporti #2|2|infants|3|Children|58|einganti|petite|283522579553royal342|peach|Gram|Unknown|17|n steingcallyeing| +8690|AAAAAAAACPBCAAAA|1997-10-27|2000-10-26|Little full combinations expect over moments; real, likely women worry in a clubs. Old, successful str|2.71|2.27|3004001|edu packexporti #1|4|school-uniforms|3|Children|19|n stought|medium|049813sandy158419049|tomato|Each|Unknown|14|barn stcallyeing| +8691|AAAAAAAACPBCAAAA|2000-10-27||Little full combinations expect over moments; real, likely women worry in a clubs. Old, successful str|33.37|23.35|3002002|importoexporti #2|2|infants|3|Children|469|n stought|large|9857lemon65958883147|papaya|Unknown|Unknown|29|oughtn stcallyeing| +8692|AAAAAAAAEPBCAAAA|1997-10-27|1999-10-27|Often red odds continue over known streets. Poor thanks could not find. Others must not happen a little long eyes. Aware parties make before in a things. Often|3.50|1.96|2001001|amalgimporto #1|1|accessories|2|Men|306|callybarpri|medium|1140537873962steel02|royal|Carton|Unknown|60|ablen stcallyeing| +8693|AAAAAAAAEPBCAAAA|1999-10-28|2001-10-26|Often particular holidays can last however following only plans. Now vast friends turn upwards frenc|9.14|1.96|2001001|importobrand #6|1|costume|6|Jewelry|27|callybarpri|N/A|1140537873962steel02|peru|Cup|Unknown|73|prin stcallyeing| +8694|AAAAAAAAEPBCAAAA|2001-10-27||Often particular holidays can last however following only plans. Now vast friends turn upwards frenc|3.86|1.96|2001001|exportiexporti #1|3|toddlers|3|Children|27|callybarpri|extra large|534921saddle19665110|sky|Dozen|Unknown|46|esen stcallyeing| +8695|AAAAAAAAHPBCAAAA|1997-10-27||Pr|0.18|0.11|1004002|edu packamalg #2|4|swimwear|1|Women|904|esebarn st|small|6459232wheat55265547|orchid|Gram|Unknown|35|antin stcallyeing| +8696|AAAAAAAAIPBCAAAA|1997-10-27|2000-10-26|Rocks would include never rocks. Rich, national workers affect mammals. Alone commercial parties take still through the bills. Much related angles provide ahead hard reserves. Near |4.60|1.51|1002001|importoamalg #1|2|fragrances|1|Women|272|ableationable|medium|371977887636839sky68|spring|Dram|Unknown|39|callyn stcallyeing| +8697|AAAAAAAAIPBCAAAA|2000-10-27||Rocks would include never rocks. Rich, national workers affect mammals. Alone commercial parties take still through the bills. Much related angles provide ahead hard reserves. Near |2.90|0.87|3002002|importoexporti #2|2|infants|3|Children|515|antioughtanti|large|371977887636839sky68|purple|Tsp|Unknown|25|ationn stcallyeing| +8698|AAAAAAAAKPBCAAAA|1997-10-27|1999-10-27|Other stars must credit. Scottish, anxious gardens used to wait hardly. Alternatively spectacular sales change with the aspects; harsh, other activities would m|3.08|2.18|7013009|exportinameless #9|13|wallpaper|7|Home|32|ablepri|N/A|4548100olive17184065|chiffon|Ton|Unknown|3|eingn stcallyeing| +8699|AAAAAAAAKPBCAAAA|1999-10-28|2001-10-26|Other stars must credit. Scottish, anxious gardens used to wait hardly. Alternatively spectacular sales change with the aspects; harsh, other activities would m|1.33|2.18|7013009|importoedu pack #2|2|mens|4|Shoes|308|ablepri|medium|772seashell189308160|mint|Lb|Unknown|18|n stn stcallyeing| +8700|AAAAAAAAKPBCAAAA|2001-10-27||Other stars must credit. Scottish, anxious gardens used to wait hardly. Alternatively spectacular sales change with the aspects; harsh, other activities would m|0.92|0.75|7013009|corpmaxi #1|2|parenting|9|Books|308|eingbarpri|N/A|2596007407snow553914|spring|Pound|Unknown|61|barbarationeing| +8701|AAAAAAAANPBCAAAA|1997-10-27||Disputes may put just men; opinions could appear various books. Big metres fulfil high cases. Courts like now new races. Calmly possible t|3.17|2.12|4002002|importoedu pack #2|2|mens|4|Shoes|118|eingoughtought|small|329997408spring11686|indian|Dram|Unknown|98|oughtbarationeing| +8702|AAAAAAAAOPBCAAAA|1997-10-27|2000-10-26|Personal years confine sometimes in order whole times. New, cheap costs should not agree more on a allegations. Big y|50.42|41.34|1003001|exportiamalg #1|3|maternity|1|Women|84|eseeing|small|02869137deep22146572|moccasin|Oz|Unknown|15|ablebarationeing| +8703|AAAAAAAAOPBCAAAA|2000-10-27||Prices shall not score here times. Poor, other cards sit just; industrial regulations ought to cause more skills. Features quote only possible, french records. More significant design|6.84|41.34|1003001|amalgimporto #2|1|accessories|2|Men|503|eseeing|large|02869137deep22146572|purple|Dram|Unknown|14|pribarationeing| +8704|AAAAAAAAAACCAAAA|1997-10-27|1999-10-27|Strong, other letters cope years. Neighbours s|0.53|0.27|2004001|edu packimporto #1|4|sports-apparel|2|Men|618|eingoughtcally|large|21499pink55925758990|green|Cup|Unknown|54|esebarationeing| +8705|AAAAAAAAAACCAAAA|1999-10-28|2001-10-26|Relations could |2.78|0.27|10012002|importoamalgamalg #2|4|monitors|10|Electronics|193|eingoughtcally|N/A|21499pink55925758990|floral|Ounce|Unknown|18|antibarationeing| +8706|AAAAAAAAAACCAAAA|2001-10-27||Relations could |2.07|0.27|10002003|importounivamalg #3|2|camcorders|10|Electronics|9|n st|N/A|177864646turquoise72|pale|Oz|Unknown|11|callybarationeing| +8707|AAAAAAAADACCAAAA|1997-10-27||Reduced connections will justify at the users. Easy, human girls can stay further dead, various shares. Big, french|16.50|13.36|7002008|importobrand #8|2|bedding|7|Home|173|priationought|N/A|forest75687803075761|pale|Box|Unknown|62|ationbarationeing| +8708|AAAAAAAAEACCAAAA|1997-10-27|2000-10-26|Already common doors get centres. Never typical lives mean with a populations. A little importan|7.72|4.40|10011005|amalgamalgamalg #5|11|disk drives|10|Electronics|233|pripriable|N/A|peru2144432569430520|papaya|Cup|Unknown|5|eingbarationeing| +8709|AAAAAAAAEACCAAAA|2000-10-27||Sudden thanks cope arts. Papers ought to become in a spaces. Common|6.47|5.49|10011005|amalgedu pack #2|1|womens|4|Shoes|233|pripriable|extra large|peru2144432569430520|forest|Dozen|Unknown|3|n stbarationeing| +8710|AAAAAAAAGACCAAAA|1997-10-27|1999-10-27|Mediterranean sheets recognise mostly. Sports might launch as techniques; hardly current times help relatively sure hot times. Extremely liberal drivers will live|9.56|2.96|10010007|univamalgamalg #7|10|memory|10|Electronics|348|eingesepri|N/A|violet35193562458852|smoke|Lb|Unknown|26|baroughtationeing| +8711|AAAAAAAAGACCAAAA|1999-10-28|2001-10-26|Prayers persuade countries. Buildings must review forwards on a investors. As other values could want subsequently assessments; major, annual taxes can settle perhaps originally impo|8.00|2.96|4002002|importoedu pack #2|10|mens|4|Shoes|348|eingesepri|medium|violet35193562458852|linen|N/A|Unknown|42|oughtoughtationeing| +8712|AAAAAAAAGACCAAAA|2001-10-27||Great figures come on a eyes; machines used to change finally to the preferences. Others will stabilise so as estates. Holy, certain considerations pass permanent, upper m|4.52|2.96|5003001|exportischolar #1|10|pop|5|Music|348|eingesepri|N/A|violet35193562458852|indian|Ounce|Unknown|59|ableoughtationeing| +8713|AAAAAAAAJACCAAAA|1997-10-27||Even rural schemes lead bombs. Ready minutes expect quite matters. Old flowers s|4.45|1.46|9012002|importounivamalg #2|12|home repair|9|Books|552|ableantianti|N/A|96ivory9748268478252|smoke|Tsp|Unknown|26|prioughtationeing| +8714|AAAAAAAAKACCAAAA|1997-10-27|2000-10-26|Young teachers may feel indeed pale styles. Common, single families may not use now soviet, well-known appearances. Nuclear, great strangers used to tell in a me|4.28|2.39|8011005|amalgmaxi #5|11|archery|8|Sports|213|prioughtable|N/A|276818thistle8561258|turquoise|Pallet|Unknown|66|eseoughtationeing| +8715|AAAAAAAAKACCAAAA|2000-10-27||Equal holidays buy above common times. Subjects see with a brothers. Badly consisten|1.70|2.39|7009006|maxibrand #6|11|mattresses|7|Home|213|prioughtable|N/A|45787423saddle992482|hot|Lb|Unknown|25|antioughtationeing| +8716|AAAAAAAAMACCAAAA|1997-10-27|1999-10-27|Lovely interests become recently great, unlikely changes. Able, simple |3.84|1.57|9014005|edu packunivamalg #5|14|sports|9|Books|443|prieseese|N/A|53538373978smoke8301|smoke|Pound|Unknown|5|callyoughtationeing| +8717|AAAAAAAAMACCAAAA|1999-10-28|2001-10-26|Lovely interests become recently great, unlikely changes. Able, simple |0.93|1.57|1004002|edu packamalg #2|4|swimwear|1|Women|243|prieseese|petite|66moccasin9533163471|sky|Carton|Unknown|53|ationoughtationeing| +8718|AAAAAAAAMACCAAAA|2001-10-27||Lovely interests become recently great, unlikely changes. Able, simple |8.53|5.71|1001001|amalgamalg #1|4|dresses|1|Women|243|prieseese|medium|66moccasin9533163471|puff|Pallet|Unknown|12|eingoughtationeing| +8719|AAAAAAAAPACCAAAA|1997-10-27||Even attractive aspects bite nearly from a women; quite sure shoulders end later with a services. Now re|3.12|0.99|1001002|amalgamalg #2|1|dresses|1|Women|199|n stn stought|large|62746298456wheat6627|saddle|Pound|Unknown|1|n stoughtationeing| +8720|AAAAAAAAABCCAAAA|1997-10-27|2000-10-26|Useful goods change of course very military readers. Better evident pupils will ride agai|9.38|4.78|10013017|exportiamalgamalg #17|13|stereo|10|Electronics|214|eseoughtable|N/A|80311906540pink59193|spring|Oz|Unknown|31|barableationeing| +8721|AAAAAAAAABCCAAAA|2000-10-27||Useful goods change of course very military readers. Better evident pupils will ride agai|1.85|4.78|10013017|importoedu pack #2|2|mens|4|Shoes|214|eseoughtable|medium|80311906540pink59193|smoke|Ton|Unknown|17|oughtableationeing| +8722|AAAAAAAACBCCAAAA|1997-10-27|1999-10-27|Willingly great seats may observe old, useful interactions; even national efforts bring banks. Again central men go closely only employers. Brilliant|25.10|18.57|8004003|edu packnameless #3|4|camping|8|Sports|260|barcallyable|N/A|95078324843408red281|salmon|Pallet|Unknown|4|ableableationeing| +8723|AAAAAAAACBCCAAAA|1999-10-28|2001-10-26|Willingly great seats may observe old, useful interactions; even national efforts bring banks. Again central men go closely only employers. Brilliant|3.90|18.57|8004003|importoamalg #2|2|fragrances|1|Women|636|barcallyable|medium|95078324843408red281|peach|Lb|Unknown|50|priableationeing| +8724|AAAAAAAACBCCAAAA|2001-10-27||Indeed detailed schools enjoy at a terms. Friends go as namely small paintings. Most|3.20|18.57|9007001|brandmaxi #1|7|reference|9|Books|636|callyprically|N/A|white108518008512052|rose|Pound|Unknown|96|eseableationeing| +8725|AAAAAAAAFBCCAAAA|1997-10-27||Decisions want heads. Documents could involve different sales. Particular tables adopt statistic|4.81|4.08|8008006|namelessnameless #6|8|outdoor|8|Sports|174|eseationought|N/A|8maroon8596997635683|snow|Gram|Unknown|36|antiableationeing| +8726|AAAAAAAAGBCCAAAA|1997-10-27|2000-10-26|Sharp, religious magazines restore national spots. Elements might guess goods. Matches send o|7.22|2.88|10013006|exportiamalgamalg #6|13|stereo|10|Electronics|2|able|N/A|5rose135455587663954|mint|Lb|Unknown|35|callyableationeing| +8727|AAAAAAAAGBCCAAAA|2000-10-27||Temperatures would not support highly in the rules. Questions must not learn nevertheless economic |2.20|0.85|4002002|importoedu pack #2|13|mens|4|Shoes|142|ableeseought|extra large|5rose135455587663954|coral|Pound|Unknown|24|ationableationeing| +8728|AAAAAAAAIBCCAAAA|1997-10-27|1999-10-27|Independent negotiations excuse real weapons. Much overall men ask much adverse, democratic states. Only old cases include desperately indeed high banks. Areas must not|6.29|4.90|5003001|exportischolar #1|3|pop|5|Music|732|ablepriation|N/A|9pale573497648524328|violet|Ton|Unknown|54|eingableationeing| +8729|AAAAAAAAIBCCAAAA|1999-10-28|2001-10-26|Independent negotiations excuse real weapons. Much overall men ask much adverse, democratic states. Only old cases include desperately indeed high banks. Areas must not|0.17|0.06|6010002|univbrand #2|10|jewelry boxes|6|Jewelry|659|ablepriation|N/A|9pale573497648524328|smoke|Dozen|Unknown|24|n stableationeing| +8730|AAAAAAAAIBCCAAAA|2001-10-27||Close, civil studies establish now black, radical quantities. Elderly, sophisticated thoughts live also reactions. Good, complete levels like rights. Explicitly sacred lines might say exc|4.21|3.45|6010002|importoamalg #1|10|fragrances|1|Women|239|ablepriation|extra large|9pale573497648524328|seashell|Gram|Unknown|68|barpriationeing| +8731|AAAAAAAALBCCAAAA|1997-10-27||American, evolutionary circles will sell files. Services increase surely by a functions. Great ways will not deny events. Strong, explicit months see very|3.11|2.73|7012002|importonameless #2|12|paint|7|Home|61|oughtcally|N/A|447011470peach562750|magenta|Pound|Unknown|57|oughtpriationeing| +8732|AAAAAAAAMBCCAAAA|1997-10-27|2000-10-26|Schools ought to consider married sources. Then opening modules matter generally this apparent deals; times shall read units. Steps may stop. About modern others alter |8.40|6.13|9011009|amalgunivamalg #9|11|cooking|9|Books|912|ableoughtn st|N/A|1seashell83823707870|peach|Oz|Unknown|28|ablepriationeing| +8733|AAAAAAAAMBCCAAAA|2000-10-27||At once environmental patients go quickly rural terms. Old trends would arise others; thus female windows ought to confer roughly. Deep year|1.76|6.13|9011009|namelesscorp #6|11|mens watch|6|Jewelry|912|ableoughtn st|N/A|1seashell83823707870|ivory|Lb|Unknown|54|pripriationeing| +8734|AAAAAAAAOBCCAAAA|1997-10-27|1999-10-27|Current definitions reflect already soldiers. Children arrange fat, linear requirements. Open ideas lay poor, important forms. Other bars fall none|1.71|1.40|7013005|exportinameless #5|13|wallpaper|7|Home|377|ationationpri|N/A|5grey904960537127683|pink|Gram|Unknown|58|esepriationeing| +8735|AAAAAAAAOBCCAAAA|1999-10-28|2001-10-26|Current definitions reflect already soldiers. Children arrange fat, linear requirements. Open ideas lay poor, important forms. Other bars fall none|1.97|1.40|4003002|exportiedu pack #2|13|kids|4|Shoes|968|eingcallyn st|large|1navajo6319777144511|tan|Unknown|Unknown|27|antipriationeing| +8736|AAAAAAAAOBCCAAAA|2001-10-27||Teachers go to a processes. Accordingly inner companies stay prayers; games expect per annum systems; educ|9.35|3.17|6012001|importobrand #1|12|costume|6|Jewelry|148|eingcallyn st|N/A|90241375494royal1882|slate|Dram|Unknown|20|callypriationeing| +8737|AAAAAAAABCCCAAAA|1997-10-27||Mostly other areas ought to help just so evil circumstances; investigations may use in general heavily comparable issues; unfortunately new centuries must forget |4.36|3.74|6005002|scholarcorp #2|5|earings|6|Jewelry|566|callycallyanti|N/A|2639seashell39993524|turquoise|Ton|Unknown|12|ationpriationeing| +8738|AAAAAAAACCCCAAAA|1997-10-27|2000-10-26|Wide, southern skills will not provide difficulties. Complete, american years must increase abroad normal, positive divisions. Low buildi|0.96|0.53|2004001|edu packimporto #1|4|sports-apparel|2|Men|5|anti|medium|610152242530plum5621|yellow|Each|Unknown|8|eingpriationeing| +8739|AAAAAAAACCCCAAAA|2000-10-27||Appropriate, huge elections should not submit in front of a children. Shoes should contain at a fal|1.66|0.76|2004001|scholarnameless #10|5|fitness|8|Sports|452|ableantiese|N/A|79827480698salmon084|puff|Carton|Unknown|75|n stpriationeing| +8740|AAAAAAAAECCCAAAA|1997-10-27|1999-10-27|More random books should not take here english sales. Approximately major firms will involve increasingly reports. |0.71|0.63|10014003|edu packamalgamalg #3|14|automotive|10|Electronics|597|ationn stanti|N/A|39plum03858689005067|rose|Dozen|Unknown|5|bareseationeing| +8741|AAAAAAAAECCCAAAA|1999-10-28|2001-10-26|More random books should not take here english sales. Approximately major firms will involve increasingly reports. |7.79|6.15|6013006|exportibrand #6|13|loose stones|6|Jewelry|268|eingcallyable|N/A|275475865goldenrod06|smoke|Carton|Unknown|45|oughteseationeing| +8742|AAAAAAAAECCCAAAA|2001-10-27||More random books should not take here english sales. Approximately major firms will involve increasingly reports. |3.88|6.15|6013006|exportiimporto #1|13|pants|2|Men|502|ablebaranti|extra large|275475865goldenrod06|spring|Case|Unknown|15|ableeseationeing| +8743|AAAAAAAAHCCCAAAA|1997-10-27||Swiss, capable lovers handle at least present details. Over here uncomfortable stations take by a lips. Private organisations can supply in a examinatio|8.15|3.99|4001002|amalgedu pack #2|1|womens|4|Shoes|248|eingeseable|extra large|28271712039884pale29|seashell|Oz|Unknown|94|prieseationeing| +8744|AAAAAAAAICCCAAAA|1997-10-27|2000-10-26|Implications come really forces. Free, insufficient games may not put. Officials force and so on others. Typical, different supplies may try onl|0.90|0.66|4001001|amalgedu pack #1|1|womens|4|Shoes|133|pripriought|extra large|9021486568tan7596692|pale|Ton|Unknown|73|eseeseationeing| +8745|AAAAAAAAICCCAAAA|2000-10-27||Implications come really forces. Free, insufficient games may not put. Officials force and so on others. Typical, different supplies may try onl|2.74|0.66|3003002|exportiexporti #2|3|toddlers|3|Children|133|pripriought|small|9021486568tan7596692|lawn|Bunch|Unknown|74|antieseationeing| +8746|AAAAAAAAKCCCAAAA|1997-10-27|1999-10-27|English, western services may not place less separate, new injuries. Wings might not refine. M|0.73|0.62|7002007|importobrand #7|2|bedding|7|Home|636|callyprically|N/A|388021526694orange77|sky|Bundle|Unknown|100|callyeseationeing| +8747|AAAAAAAAKCCCAAAA|1999-10-28|2001-10-26|More french researchers may find able stations. Black, various terms take. Mild resources exercise enough through |0.29|0.62|9012012|importounivamalg #12|2|home repair|9|Books|112|callyprically|N/A|388021526694orange77|plum|Case|Unknown|59|ationeseationeing| +8748|AAAAAAAAKCCCAAAA|2001-10-27||Policies compete extra serious accounts. Further good clubs would not wonder much constantly wrong blacks. In short poor creditors used to expect ago. Cases wi|1.83|0.62|9012012|namelessunivamalg #11|8|scanners|10|Electronics|112|callyprically|N/A|388021526694orange77|grey|Dozen|Unknown|39|eingeseationeing| +8749|AAAAAAAANCCCAAAA|1997-10-27||Faces would not read ever professional girls. Complete, briti|6.73|2.08|7006010|corpbrand #10|6|rugs|7|Home|37|ationpri|N/A|649377651thistle3976|wheat|Bundle|Unknown|28|n steseationeing| +8750|AAAAAAAAOCCCAAAA|1997-10-27|2000-10-26|Available yards should not guess justifiably international, brilliant eyes. National, short-term traditions write average, civil conditions. Angles pay possibilities. Assistant rooms h|1.80|0.57|3002001|importoexporti #1|2|infants|3|Children|604|esebarcally|large|30518515299slate7096|sandy|Unknown|Unknown|85|barantiationeing| +8751|AAAAAAAAOCCCAAAA|2000-10-27||Available yards should not guess justifiably international, brilliant eyes. National, short-term traditions write average, civil conditions. Angles pay possibilities. Assistant rooms h|3.06|1.19|3002001|amalgexporti #2|1|newborn|3|Children|214|esebarcally|medium|134028891207steel658|hot|N/A|Unknown|31|oughtantiationeing| +8752|AAAAAAAAADCCAAAA|1997-10-27|1999-10-27|Old-fashioned, traditional proportions used to operate very excellent, previous preparations. Now advisory blacks should not translate perfect walls. Directors cannot advise subsequently depar|5.05|2.42|2002001|importoimporto #1|2|shirts|2|Men|18|eingought|petite|midnight181796168952|brown|Box|Unknown|6|ableantiationeing| +8753|AAAAAAAAADCCAAAA|1999-10-28|2001-10-26|Old-fashioned, traditional proportions used to operate very excellent, previous preparations. Now advisory blacks should not translate perfect walls. Directors cannot advise subsequently depar|4.19|2.17|3002002|importoexporti #2|2|infants|3|Children|18|eingought|petite|midnight181796168952|saddle|N/A|Unknown|11|priantiationeing| +8754|AAAAAAAAADCCAAAA|2001-10-27||Much coastal manufacturers would not change less families. Feet must mean military, important days; guests know o|3.89|2.17|3002002|amalgedu pack #1|2|womens|4|Shoes|18|eingought|economy|50dodger430531077336|lemon|Lb|Unknown|30|eseantiationeing| +8755|AAAAAAAADDCCAAAA|1997-10-27||Japanese programs shall come less situations. Too used components enable senses. Feet develop only only whole allies. Right years bring on the asso|5.79|5.09|2002002|importoimporto #2|2|shirts|2|Men|226|callyableable|small|salmon79773572734158|smoke|Case|Unknown|45|antiantiationeing| +8756|AAAAAAAAEDCCAAAA|1997-10-27|2000-10-26|Residents think effectively. Good, local nerves must provide. Poli|4.60|2.43|10005002|scholarunivamalg #2|5|karoke|10|Electronics|788|eingeingation|N/A|847903179rosy7985229|pale|Dram|Unknown|68|callyantiationeing| +8757|AAAAAAAAEDCCAAAA|2000-10-27||Residents think effectively. Good, local nerves must provide. Poli|7.75|4.41|6007006|brandcorp #6|7|pendants|6|Jewelry|788|eingeingation|N/A|847903179rosy7985229|red|Gram|Unknown|100|ationantiationeing| +8758|AAAAAAAAGDCCAAAA|1997-10-27|1999-10-27|Hours consider teenage days; minutes play possibly available|0.49|0.24|1001001|amalgamalg #1|1|dresses|1|Women|9|n st|large|5188470971132powder6|navy|Bundle|Unknown|80|eingantiationeing| +8759|AAAAAAAAGDCCAAAA|1999-10-28|2001-10-26|Foreign, frequent images cover good, international police. Farmers cannot mask clean, relative holes. Here mental numbers become shortly incidents. Objections acquir|6.81|6.12|1001001|amalgunivamalg #12|11|cooking|9|Books|9|n st|N/A|507448106861plum4640|sandy|Ounce|Unknown|10|n stantiationeing| +8760|AAAAAAAAGDCCAAAA|2001-10-27||Methods get always obviously importa|8.02|6.12|10004006|edu packunivamalg #6|4|audio|10|Electronics|9|n st|N/A|6943sky0909990421616|turquoise|Ounce|Unknown|18|barcallyationeing| +8761|AAAAAAAAJDCCAAAA|1997-10-27||Double, other individuals used to stay historically in a groups. A|0.22|0.15|10012007|importoamalgamalg #7|12|monitors|10|Electronics|421|oughtableese|N/A|32436705889peach3710|white|Ounce|Unknown|32|oughtcallyationeing| +8762|AAAAAAAAKDCCAAAA|1997-10-27|2000-10-26|Ho|42.11|21.47|4001001|amalgedu pack #1|1|womens|4|Shoes|25|antiable|large|884314274purple94087|snow|Dram|Unknown|15|ablecallyationeing| +8763|AAAAAAAAKDCCAAAA|2000-10-27||Soviet feet should not concentrate between a police. Other addresses may identify essentially. Together dynamic affairs must kill hot, bad organisations; early, cert|7.42|21.47|1003002|exportiamalg #2|3|maternity|1|Women|210|antiable|large|238khaki986734866864|pink|Oz|Unknown|25|pricallyationeing| +8764|AAAAAAAAMDCCAAAA|1997-10-27|1999-10-27|Very good costs will not come yesterday intensive positive towns. Here new units should show rather likely, new clothes. Upper rules may not|3.24|2.26|4001001|amalgedu pack #1|1|womens|4|Shoes|49|n stese|small|158492988341papaya25|red|Tbl|Unknown|50|esecallyationeing| +8765|AAAAAAAAMDCCAAAA|1999-10-28|2001-10-26|Practices look during a ends. Yesterday easy organisations could serve. Decent, main matters may not contend surely financial fe|0.44|2.26|4001001|importoedu pack #2|2|mens|4|Shoes|49|n stese|economy|69716974steel7759924|plum|Box|Unknown|76|anticallyationeing| +8766|AAAAAAAAMDCCAAAA|2001-10-27||Practices look during a ends. Yesterday easy organisations could serve. Decent, main matters may not contend surely financial fe|0.30|2.26|4001001|edu packexporti #1|4|school-uniforms|3|Children|49|n stese|extra large|69716974steel7759924|sandy|Lb|Unknown|22|callycallyationeing| +8767|AAAAAAAAPDCCAAAA|1997-10-27||Extremely independent women may not constitute incorrectly to a processes. Formid|73.12|57.76|5002002|importoscholar #2|2|country|5|Music|503|pribaranti|N/A|97182698280223royal5|smoke|Dram|Unknown|8|ationcallyationeing| +8768|AAAAAAAAAECCAAAA|1997-10-27|2000-10-26|Hands might retain; public, simple channels should give. Yet ap|1.84|1.58|5004001|edu packscholar #1|4|classical|5|Music|373|priationpri|N/A|820387811violet93103|smoke|Bundle|Unknown|41|eingcallyationeing| +8769|AAAAAAAAAECCAAAA|2000-10-27||Hands might retain; public, simple channels should give. Yet ap|6.14|4.78|1002002|importoamalg #2|2|fragrances|1|Women|530|priationpri|large|569ivory709531290466|yellow|Ton|Unknown|6|n stcallyationeing| +8770|AAAAAAAACECCAAAA|1997-10-27|1999-10-27|Cards used to make. Interested dogs get. Redundant feet set yet true outside years. So-called patients repeat f|2.85|2.30|5003001|exportischolar #1|3|pop|5|Music|139|n stpriought|N/A|16644370356tomato816|sky|Cup|Unknown|67|barationationeing| +8771|AAAAAAAACECCAAAA|1999-10-28|2001-10-26|Cards used to make. Interested dogs get. Redundant feet set yet true outside years. So-called patients repeat f|4.95|2.30|5003001|amalgexporti #2|3|newborn|3|Children|98|n stpriought|medium|02thistle01818353628|violet|Bundle|Unknown|29|oughtationationeing| +8772|AAAAAAAACECCAAAA|2001-10-27||Services realise as a days. Special eyes could live at least fingers. Just green roles would use effectively insufficient, thick men. Attractive, japanese times shall see from a flowers. Living, n|2.49|2.30|8012003|importomaxi #3|12|guns|8|Sports|98|n stpriought|N/A|95888447peru31343309|navajo|Ounce|Unknown|50|ableationationeing| +8773|AAAAAAAAFECCAAAA|1997-10-27||Areas must think always. Longer responsible standards reappear. Other powers cover various players. Areas accept with a resources. As necessary things might not take more than top, |6.09|5.11|8008004|namelessnameless #4|8|outdoor|8|Sports|377|ationationpri|N/A|63574018khaki5294302|slate|Pound|Unknown|73|priationationeing| +8774|AAAAAAAAGECCAAAA|1997-10-27|2000-10-26|Representatives mean abruptly suddenly great cells. New, living rates see simply out of a styles. Terrible students import all public types; remarkably original costs try. |8.89|3.46|7013005|exportinameless #5|13|wallpaper|7|Home|84|eseeing|N/A|797957954413665navy6|plum|Unknown|Unknown|27|eseationationeing| +8775|AAAAAAAAGECCAAAA|2000-10-27||Everywhere hidden shows fight marginally full, significant places. National eyes should|2.11|3.46|1002002|importoamalg #2|13|fragrances|1|Women|84|eseeing|economy|797957954413665navy6|saddle|Tbl|Unknown|17|antiationationeing| +8776|AAAAAAAAIECCAAAA|1997-10-27|1999-10-27|Prime women will not see then together large estates. Peoples eat so national stocks. Activities|2.00|1.58|6009001|maxicorp #1|9|womens watch|6|Jewelry|867|ationcallyeing|N/A|09479348peach0611242|thistle|Gram|Unknown|36|callyationationeing| +8777|AAAAAAAAIECCAAAA|1999-10-28|2001-10-26|Large, red resources go ibid. for |6.81|5.85|6009001|importonameless #8|12|paint|7|Home|431|ationcallyeing|N/A|09479348peach0611242|spring|Gross|Unknown|45|ationationationeing| +8778|AAAAAAAAIECCAAAA|2001-10-27||Times stand evenly mental eyes. Goods may end however countries. High, english months would like true, associated doubts. Arts would not get new, detailed signs. All other months include then su|3.23|5.85|10003007|exportiunivamalg #7|3|dvd/vcr players|10|Electronics|386|ationcallyeing|N/A|8pink539678466872038|sky|N/A|Unknown|29|eingationationeing| +8779|AAAAAAAALECCAAAA|1997-10-27||So overall|4.39|2.37|8006010|corpnameless #10|6|football|8|Sports|525|antiableanti|N/A|516navy8313252383355|steel|Carton|Unknown|9|n stationationeing| +8780|AAAAAAAAMECCAAAA|1997-10-27|2000-10-26|Desirable stars should introduce to|6.99|4.54|8013001|exportimaxi #1|13|sailing|8|Sports|43|priese|N/A|5026180619mint263971|linen|Dram|Unknown|15|bareingationeing| +8781|AAAAAAAAMECCAAAA|2000-10-27||Rates must not maintain over. Quite possible reports allow other types; highest hard versions used to descend greatly p|3.90|4.54|8013001|amalgexporti #2|13|newborn|3|Children|43|priese|large|5026180619mint263971|powder|Bunch|Unknown|5|oughteingationeing| +8782|AAAAAAAAOECCAAAA|1997-10-27|1999-10-27|Obvious, democratic statements become rates. Common journalists could not think just children. Windows will not bring urban ser|97.36|78.86|6006007|corpcorp #7|6|rings|6|Jewelry|716|callyoughtation|N/A|8536447856350grey200|sky|Oz|Unknown|20|ableeingationeing| +8783|AAAAAAAAOECCAAAA|1999-10-28|2001-10-26|Reluctant, electric images finish usually other citizens. Colonial rooms could not ensure. New, modern models should go ideas. Appearances could n|3.06|78.86|2003002|exportiimporto #2|3|pants|2|Men|281|callyoughtation|medium|951rose7914018040726|orchid|Pallet|Unknown|28|prieingationeing| +8784|AAAAAAAAOECCAAAA|2001-10-27||Months might |8.22|78.86|2003002|importoimporto #1|2|shirts|2|Men|281|callyoughtation|medium|35844259002white0869|chartreuse|Carton|Unknown|37|eseeingationeing| +8785|AAAAAAAABFCCAAAA|1997-10-27||Endless, professional others create by a years; large vis|1.24|0.74|8009006|maxinameless #6|9|optics|8|Sports|229|n stableable|N/A|0051837steel70210375|yellow|Tsp|Unknown|40|antieingationeing| +8786|AAAAAAAACFCCAAAA|1997-10-27|2000-10-26|Military streets prove much easy toys; women deal particular, musical men. Black, great minutes used to live just skills. Basic, great tasks earn extremely wonderful chiefs; local, nat|3.01|2.25|7003007|exportibrand #7|3|kids|7|Home|622|ableablecally|N/A|4271567517rosy364713|turquoise|Ounce|Unknown|50|callyeingationeing| +8787|AAAAAAAACFCCAAAA|2000-10-27||Wonderful companies used to tell then at the men. Elaborate, common cuts operate around from a holes. Key, new purposes may find in order suitable, formal developmen|4.43|2.25|7003007|edu packnameless #8|3|camping|8|Sports|622|ableablecally|N/A|045navajo97767947461|purple|Unknown|Unknown|57|ationeingationeing| +8788|AAAAAAAAEFCCAAAA|1997-10-27|1999-10-27|Considerable, economic events find expensive clients. Techniques work eventually cases. Letters make; practices think meanwhile also capital scienti|1.45|0.89|3003001|exportiexporti #1|3|toddlers|3|Children|613|prioughtcally|medium|7850sienna9857618349|sky|N/A|Unknown|10|eingeingationeing| +8789|AAAAAAAAEFCCAAAA|1999-10-28|2001-10-26|Enemies grant st|10.01|0.89|9015006|scholarunivamalg #6|15|fiction|9|Books|91|oughtn st|N/A|7850sienna9857618349|spring|Cup|Unknown|61|n steingationeing| +8790|AAAAAAAAEFCCAAAA|2001-10-27||Small arguments can bring yesterday women. Shadows may produce leaders; successful implications become outside major costs; provisions give even arms. Relations should concentrate of course. |58.48|0.89|9015006|importounivamalg #7|15|home repair|9|Books|440|bareseese|N/A|906985624152royal794|snow|Pound|Unknown|30|barn stationeing| +8791|AAAAAAAAHFCCAAAA|1997-10-27||Others think only so favorite events. Relations rely local results. Obvious, main fu|7.30|4.01|5002002|importoscholar #2|2|country|5|Music|248|eingeseable|N/A|92087028salmon687856|smoke|Ounce|Unknown|19|oughtn stationeing| +8792|AAAAAAAAIFCCAAAA|1997-10-27|2000-10-26|Eyes establish therefor|0.60|0.41|5004001|edu packscholar #1|4|classical|5|Music|263|pricallyable|N/A|38817sandy6723043029|indian|N/A|Unknown|13|ablen stationeing| +8793|AAAAAAAAIFCCAAAA|2000-10-27||Right possible proposals used to halt other, bloody mach|6.47|0.41|5004001|edu packexporti #2|4|school-uniforms|3|Children|291|oughtn stable|extra large|38817sandy6723043029|rose|Each|Unknown|2|prin stationeing| +8794|AAAAAAAAKFCCAAAA|1997-10-27|1999-10-27|Civil firms say; prospective technologies used to take there. Easy, high assets enter so practical, structural buildings. Studies woul|9.09|3.45|10009006|maxiunivamalg #6|9|televisions|10|Electronics|472|ableationese|N/A|8linen61147702103557|lemon|Carton|Unknown|24|esen stationeing| +8795|AAAAAAAAKFCCAAAA|1999-10-28|2001-10-26|Civil firms say; prospective technologies used to take there. Easy, high assets enter so practical, structural buildings. Studies woul|8.09|2.75|10009006|corpcorp #4|6|rings|6|Jewelry|317|ationoughtpri|N/A|8linen61147702103557|lavender|Carton|Unknown|45|antin stationeing| +8796|AAAAAAAAKFCCAAAA|2001-10-27||Services should not argue ever however good patterns. Forms argue hard windows. Direc|6.92|2.35|10009006|edu packbrand #7|4|curtains/drapes|7|Home|317|ationoughtpri|N/A|8linen61147702103557|steel|Pound|Unknown|35|callyn stationeing| +8797|AAAAAAAANFCCAAAA|1997-10-27||Companies would protect greatly firms. Exceptions disagree highly; wrong difficulties put once aga|2.22|1.86|7002008|importobrand #8|2|bedding|7|Home|90|barn st|N/A|2yellow7759651785716|lawn|N/A|Unknown|9|ationn stationeing| +8798|AAAAAAAAOFCCAAAA|1997-10-27|2000-10-26|Responsible, western effects say far. Supplies show in a bodies. Orig|0.59|0.33|5003001|exportischolar #1|3|pop|5|Music|642|ableesecally|N/A|466346pink8896314780|red|Case|Unknown|28|eingn stationeing| +8799|AAAAAAAAOFCCAAAA|2000-10-27||Different, good agents wind very animals. Others invite rather than|3.67|0.33|5003001|importoimporto #2|3|shirts|2|Men|929|n stablen st|medium|8605360556sandy82232|blue|Ounce|Unknown|58|n stn stationeing| +8800|AAAAAAAAAGCCAAAA|1997-10-27|1999-10-27|Enough direct events take only long honours; fast, impossible sources look particular times. Very personal homes should rest so-called kilometres. Good visitors |3.19|2.80|10012012|importoamalgamalg #12|12|monitors|10|Electronics|640|baresecally|N/A|4299234036cyan984482|thistle|Dram|Unknown|9|barbareingeing| +8801|AAAAAAAAAGCCAAAA|1999-10-28|2001-10-26|Enough direct events take only long honours; fast, impossible sources look particular times. Very personal homes should rest so-called kilometres. Good visitors |4.35|2.80|10012012|edu packexporti #2|12|school-uniforms|3|Children|640|baresecally|extra large|3710230peach67545655|peach|Each|Unknown|55|oughtbareingeing| +8802|AAAAAAAAAGCCAAAA|2001-10-27||Enough direct events take only long honours; fast, impossible sources look particular times. Very personal homes should rest so-called kilometres. Good visitors |6.45|2.80|5002001|importoscholar #1|2|country|5|Music|640|baresecally|N/A|070206865317smoke375|violet|N/A|Unknown|45|ablebareingeing| +8803|AAAAAAAADGCCAAAA|1997-10-27||Deep parent|1.30|0.52|9009008|maximaxi #8|9|science|9|Books|82|ableeing|N/A|4588461078240slate32|sky|N/A|Unknown|20|pribareingeing| +8804|AAAAAAAAEGCCAAAA|1997-10-27|2000-10-26|Still corporate departments make pressures. Workers shall not last much out of a walls. Successful ideas snap. Public candidates shall tell. Human, entire prob|4.43|3.32|7001005|amalgbrand #5|1|bathroom|7|Home|298|eingn stable|N/A|2212orange2416213814|plum|Carton|Unknown|32|esebareingeing| +8805|AAAAAAAAEGCCAAAA|2000-10-27||Still corporate departments make pressures. Workers shall not last much out of a walls. Successful ideas snap. Public candidates shall tell. Human, entire prob|5.80|3.13|7001005|amalgimporto #2|1|accessories|2|Men|42|ableese|petite|2212orange2416213814|sky|Case|Unknown|45|antibareingeing| +8806|AAAAAAAAGGCCAAAA|1997-10-27|1999-10-27|English pictures evolve either to a factors. Detailed, ultimate months manage never mild eyes. High commi|5.86|3.80|7004007|edu packbrand #7|4|curtains/drapes|7|Home|554|eseantianti|N/A|snow8077768413058001|gainsboro|Tbl|Unknown|30|callybareingeing| +8807|AAAAAAAAGGCCAAAA|1999-10-28|2001-10-26|English pictures evolve either to a factors. Detailed, ultimate months manage never mild eyes. High commi|3.04|1.42|7001008|amalgbrand #8|1|bathroom|7|Home|583|prieinganti|N/A|51117pale78261204100|pink|Dram|Unknown|32|ationbareingeing| +8808|AAAAAAAAGGCCAAAA|2001-10-27||English pictures evolve either to a factors. Detailed, ultimate months manage never mild eyes. High commi|5.04|1.66|6011001|amalgbrand #1|11|semi-precious|6|Jewelry|583|prieinganti|N/A|9963466540921red1188|tan|Pallet|Unknown|54|eingbareingeing| +8809|AAAAAAAAJGCCAAAA|1997-10-27||Arab eyes love both public, excellent problems. Just black countries see pa|2.50|1.77|2003002|exportiimporto #2|3|pants|2|Men|371|oughtationpri|petite|556471748491salmon78|lavender|Pallet|Unknown|5|n stbareingeing| +8810|AAAAAAAAKGCCAAAA|1997-10-27|2000-10-26|Young, legitimate passengers move legitimately with a animals. As generous rocks could need now about a tons; violent, expected comments |0.49|0.39|10008005|namelessunivamalg #5|8|scanners|10|Electronics|813|prioughteing|N/A|88954527linen5643875|frosted|Lb|Unknown|8|baroughteingeing| +8811|AAAAAAAAKGCCAAAA|2000-10-27||Young, legitimate passengers move legitimately with a animals. As generous rocks could need now about a tons; violent, expected comments |1.24|0.39|10013006|exportiamalgamalg #6|13|stereo|10|Electronics|290|barn stable|N/A|67707yellow724456048|blanched|Unknown|Unknown|14|oughtoughteingeing| +8812|AAAAAAAAMGCCAAAA|1997-10-27|1999-10-27|Broadly alternative sectors smooth against a characteristics; aware contents preserve thus beautiful, certain gods. Living children must recall in a activities; econo|3.43|2.70|6010005|univbrand #5|10|jewelry boxes|6|Jewelry|155|antiantiought|N/A|340546turquoise09758|powder|Dozen|Unknown|30|ableoughteingeing| +8813|AAAAAAAAMGCCAAAA|1999-10-28|2001-10-26|Cultural, dear personnel make areas. Financially representative arms conceal only pains. Expected, previous newspapers pray free, firm |3.36|2.70|5004002|edu packscholar #2|10|classical|5|Music|155|antiantiought|N/A|340546turquoise09758|slate|Pallet|Unknown|22|prioughteingeing| +8814|AAAAAAAAMGCCAAAA|2001-10-27||Enthusiastic, individual guns place far marine phenomena; only differ|1.36|2.70|2003001|exportiimporto #1|10|pants|2|Men|122|antiantiought|small|86557rosy99198494176|red|Dram|Unknown|18|eseoughteingeing| +8815|AAAAAAAAPGCCAAAA|1997-10-27||Social cattle arise pal|0.21|0.07|5001002|amalgscholar #2|1|rock|5|Music|124|eseableought|N/A|2white19095175540547|drab|Unknown|Unknown|8|antioughteingeing| +8816|AAAAAAAAAHCCAAAA|1997-10-27|2000-10-26|Fingers go instead vast, high premises. Modern artists continue shortly specified, real mothers. Probably cri|0.52|0.30|2001001|amalgimporto #1|1|accessories|2|Men|53|prianti|petite|050958sky77488362292|sienna|Each|Unknown|26|callyoughteingeing| +8817|AAAAAAAAAHCCAAAA|2000-10-27||Fingers go instead vast, high premises. Modern artists continue shortly specified, real mothers. Probably cri|6.55|4.51|2001001|edu packedu pack #2|1|athletic|4|Shoes|53|prianti|medium|62238298431dark31404|white|Unknown|Unknown|20|ationoughteingeing| +8818|AAAAAAAACHCCAAAA|1997-10-27|1999-10-27|Other groups will understand too genetically only jobs. Systems would get. Stocks stop. Passive, normal teachers would not conduct soon at a obligations. High lessons work managerial, different|8.20|3.52|5003001|exportischolar #1|3|pop|5|Music|198|eingn stought|N/A|9443051tan5987504961|powder|Case|Unknown|40|eingoughteingeing| +8819|AAAAAAAACHCCAAAA|1999-10-28|2001-10-26|Values let concerned conclusions. Namely national men see alone from a years. Again single rules used t|4.50|3.52|3003002|exportiexporti #2|3|toddlers|3|Children|907|eingn stought|extra large|9443051tan5987504961|saddle|Bundle|Unknown|33|n stoughteingeing| +8820|AAAAAAAACHCCAAAA|2001-10-27||Rights support so immediately little |1.25|0.93|5002001|importoscholar #1|2|country|5|Music|983|eingn stought|N/A|809800752royal712298|navajo|Ounce|Unknown|3|barableeingeing| +8821|AAAAAAAAFHCCAAAA|1997-10-27||Total, british lips take yet at the organisations. As immediate manufacturers make forward coloured resources. Particular, military surfaces might not predict together tired groups.|8.26|6.44|2001002|amalgimporto #2|1|accessories|2|Men|41|oughtese|medium|9salmon1038434880142|misty|Gram|Unknown|1|oughtableeingeing| +8822|AAAAAAAAGHCCAAAA|1997-10-27|2000-10-26|Scottish, particular championships keep as joint wages. Guests could not agree away from a characters; walls could not achieve actors. Toys |7.56|3.02|10009017|maxiunivamalg #17|9|televisions|10|Electronics|996|callyn stn st|N/A|426354730lime7687557|tan|Tbl|Unknown|75|ableableeingeing| +8823|AAAAAAAAGHCCAAAA|2000-10-27||Scottish, particular championships keep as joint wages. Guests could not agree away from a characters; walls could not achieve actors. Toys |6.39|2.87|10009017|namelessbrand #4|8|lighting|7|Home|43|callyn stn st|N/A|426354730lime7687557|spring|Lb|Unknown|65|priableeingeing| +8824|AAAAAAAAIHCCAAAA|1997-10-27|1999-10-27|Decisions cannot take loosely very royal areas; easy, competitive unions rush now practical goods; black officials oc|2.70|1.48|10016002|corpamalgamalg #2|16|wireless|10|Electronics|241|oughteseable|N/A|0325572520786wheat28|papaya|Bundle|Unknown|31|eseableeingeing| +8825|AAAAAAAAIHCCAAAA|1999-10-28|2001-10-26|Decisions cannot take loosely very royal areas; easy, competitive unions rush now practical goods; black officials oc|8.82|1.48|10016002|corpbrand #2|16|consignment|6|Jewelry|202|ablebarable|N/A|812029889783smoke304|pale|Each|Unknown|45|antiableeingeing| +8826|AAAAAAAAIHCCAAAA|2001-10-27||Decisions cannot take loosely very royal areas; easy, competitive unions rush now practical goods; black officials oc|2.39|1.48|6016003|corpbrand #3|16|consignment|6|Jewelry|40|ablebarable|N/A|03997819floral147382|gainsboro|Tbl|Unknown|41|callyableeingeing| +8827|AAAAAAAALHCCAAAA|1997-10-27||Very friendly teeth manage today ambitious, medical roads; more early materials serve into a children. D|32.83|16.08|1003002|exportiamalg #2|3|maternity|1|Women|866|callycallyeing|medium|8439midnight05862076|royal|Cup|Unknown|44|ationableeingeing| +8828|AAAAAAAAMHCCAAAA|1997-10-27|2000-10-26|Reasons grant for once various, huge men. Good, full feelings go. As|8.40|7.56|10008006|namelessunivamalg #6|8|scanners|10|Electronics|371|oughtationpri|N/A|steel499452937668884|wheat|Bundle|Unknown|4|eingableeingeing| +8829|AAAAAAAAMHCCAAAA|2000-10-27||Beautiful, old conditions stem boundaries. Concessions may change there young, available properties. Musical, regional phenomena get only in a pains. Home unlikely |4.29|7.56|5003002|exportischolar #2|3|pop|5|Music|261|oughtcallyable|N/A|steel499452937668884|sky|Pallet|Unknown|73|n stableeingeing| +8830|AAAAAAAAOHCCAAAA|1997-10-27|1999-10-27|Sex|3.66|1.46|8009001|maxinameless #1|9|optics|8|Sports|106|callybarought|N/A|1057honeydew50960133|brown|Pallet|Unknown|14|barprieingeing| +8831|AAAAAAAAOHCCAAAA|1999-10-28|2001-10-26|Sex|7.51|1.46|1002002|importoamalg #2|9|fragrances|1|Women|106|callybarought|extra large|7725magenta883700400|khaki|Tbl|Unknown|93|oughtprieingeing| +8832|AAAAAAAAOHCCAAAA|2001-10-27||Sex|3.91|1.46|6002001|importocorp #1|2|diamonds|6|Jewelry|12|callybarought|N/A|7725magenta883700400|purple|Tsp|Unknown|46|ableprieingeing| +8833|AAAAAAAABICCAAAA|1997-10-27||Women cost average, inner images. Only immediate exercises could like fair; other hours would de|4.08|3.14|10010011|univamalgamalg #11|10|memory|10|Electronics|105|antibarought|N/A|3saddle5178520503876|seashell|Cup|Unknown|9|priprieingeing| +8834|AAAAAAAACICCAAAA|1997-10-27|2000-10-26|Other, other changes used to sort light facts. Issues help fully usual, fair gr|2.25|1.44|8014005|edu packmaxi #5|14|tennis|8|Sports|244|eseeseable|N/A|53675white8559402070|saddle|Each|Unknown|13|eseprieingeing| +8835|AAAAAAAACICCAAAA|2000-10-27||Alternative students ought to dance. Hours take small, large obligations. Occupational, differe|4.50|1.44|8014005|corpmaxi #4|14|parenting|9|Books|244|eseeseable|N/A|2963989754black44906|lavender|Ton|Unknown|86|antiprieingeing| +8836|AAAAAAAAEICCAAAA|1997-10-27|1999-10-27|Members allow well dogs. Programmes look other, french rights. New, cl|7.73|6.87|5002001|importoscholar #1|2|country|5|Music|225|antiableable|N/A|932326484bisque74501|steel|Dozen|Unknown|1|callyprieingeing| +8837|AAAAAAAAEICCAAAA|1999-10-28|2001-10-26|Minor concentrations shall not communicate of course hot documents; legal, white numbers shall form great|8.84|5.48|9003006|exportimaxi #6|3|computers|9|Books|757|ationantiation|N/A|932326484bisque74501|peru|Ounce|Unknown|89|ationprieingeing| +8838|AAAAAAAAEICCAAAA|2001-10-27||Commercial, real lines can think simply in a docu|6.08|5.48|4001001|amalgedu pack #1|3|womens|4|Shoes|330|ationantiation|medium|932326484bisque74501|red|Bunch|Unknown|14|eingprieingeing| +8839|AAAAAAAAHICCAAAA|1997-10-27||Low, difficult services disarm nowhere by the tests. Observations will evolve scientific weeks. Good, easy pu|3.73|1.11|7009010|maxibrand #10|9|mattresses|7|Home|506|callybaranti|N/A|04864763481powder960|steel|Each|Unknown|23|n stprieingeing| +8840|AAAAAAAAIICCAAAA|1997-10-27|2000-10-26|Low protective actors may not bite far items. Hence new eyes |8.30|5.56|8008001|namelessnameless #1|8|outdoor|8|Sports|430|barpriese|N/A|258691navy8252388096|peru|Tsp|Unknown|29|bareseeingeing| +8841|AAAAAAAAIICCAAAA|2000-10-27||Ne|1.78|5.56|8008001|amalgedu pack #2|1|womens|4|Shoes|258|barpriese|extra large|258691navy8252388096|orchid|Tsp|Unknown|30|oughteseeingeing| +8842|AAAAAAAAKICCAAAA|1997-10-27|1999-10-27|Black laws get accordingly eyes. Tightly rural systems trust heavily coming tests; personal, bad boards go. Electric looks may not rec|9.05|6.15|8003003|exportinameless #3|3|basketball|8|Sports|85|antieing|N/A|44126621191566peru63|smoke|Gram|Unknown|8|ableeseeingeing| +8843|AAAAAAAAKICCAAAA|1999-10-28|2001-10-26|Heavy sources wear full, average countries. Just invisible workers secure by the nations. Awful daughters go also clear crowds. Acts curl however important lips. Chapters speak though in a rule|2.80|2.29|8003003|exportischolar #2|3|pop|5|Music|492|ablen stese|N/A|7638ivory59873533577|medium|Each|Unknown|37|prieseeingeing| +8844|AAAAAAAAKICCAAAA|2001-10-27||Also internal weeks may make enough disabled effec|7.27|6.10|8003003|amalgimporto #1|1|accessories|2|Men|985|ablen stese|medium|7638ivory59873533577|white|Unknown|Unknown|64|eseeseeingeing| +8845|AAAAAAAANICCAAAA|1997-10-27||Possible services can think in addition in a institutions. Able, hard grounds will choose mixed kilometres|4.44|2.66|7014006|edu packnameless #6|14|glassware|7|Home|470|barationese|N/A|23586699852737rosy99|red|N/A|Unknown|38|antieseeingeing| +8846|AAAAAAAAOICCAAAA|1997-10-27|2000-10-26|Social, proper allegations criticise particularly beneficial workers. |1.38|1.07|10003007|exportiunivamalg #7|3|dvd/vcr players|10|Electronics|196|callyn stought|N/A|936100710476maroon90|violet|Cup|Unknown|29|callyeseeingeing| +8847|AAAAAAAAOICCAAAA|2000-10-27||Social, proper allegations criticise particularly beneficial workers. |7.88|4.96|10003007|scholarnameless #8|15|tables|7|Home|929|n stablen st|N/A|878sandy199539737430|slate|Each|Unknown|2|ationeseeingeing| +8848|AAAAAAAAAJCCAAAA|1997-10-27|1999-10-27|So surprising co|87.36|45.42|10005009|scholarunivamalg #9|5|karoke|10|Electronics|397|ationn stpri|N/A|053493648blue1250568|peru|Unknown|Unknown|20|eingeseeingeing| +8849|AAAAAAAAAJCCAAAA|1999-10-28|2001-10-26|So surprising co|5.66|4.41|5001002|amalgscholar #2|5|rock|5|Music|457|ationn stpri|N/A|5pale894876891048356|pink|Dozen|Unknown|24|n steseeingeing| +8850|AAAAAAAAAJCCAAAA|2001-10-27||Rather liberal companies want suddenly expenses; surely following shows will decrease events; years launch oft|0.99|4.41|5001002|namelessnameless #1|5|outdoor|8|Sports|457|ationantiese|N/A|5pale894876891048356|mint|Each|Unknown|66|barantieingeing| +8851|AAAAAAAADJCCAAAA|1997-10-27||Financial windows come|0.24|0.09|5001002|amalgscholar #2|1|rock|5|Music|78|eingation|N/A|223557328617pale7079|purple|Tsp|Unknown|44|oughtantieingeing| +8852|AAAAAAAAEJCCAAAA|1997-10-27|2000-10-26|English, good complaints ought to counteract past democr|17.77|7.46|7007003|brandbrand #3|7|decor|7|Home|226|callyableable|N/A|244681642253puff1888|spring|Ton|Unknown|29|ableantieingeing| +8853|AAAAAAAAEJCCAAAA|2000-10-27||English, good complaints ought to counteract past democr|6.56|4.39|1001002|amalgamalg #2|7|dresses|1|Women|226|callyableable|medium|244681642253puff1888|linen|Lb|Unknown|5|priantieingeing| +8854|AAAAAAAAGJCCAAAA|1997-10-27|1999-10-27|Often professional centres work in a hands. Also sufficient years shall become even c|2.79|1.39|6015007|scholarbrand #7|15|custom|6|Jewelry|56|callyanti|N/A|260snow2091324201537|tomato|Dram|Unknown|15|eseantieingeing| +8855|AAAAAAAAGJCCAAAA|1999-10-28|2001-10-26|Often professional centres work in a hands. Also sufficient years shall become even c|3.71|2.70|6015007|scholarunivamalg #16|5|karoke|10|Electronics|329|n stablepri|N/A|40474795954misty7617|sky|Gram|Unknown|36|antiantieingeing| +8856|AAAAAAAAGJCCAAAA|2001-10-27||Pretty, very rates acquire new practices. Particular, human seats ought to handle most top occasions; beautiful, military scientists could belong previously democratic, scottish days.|7.40|2.70|4004001|edu packedu pack #1|5|athletic|4|Shoes|329|n stablepri|small|1110864594706spring9|linen|Lb|Unknown|6|callyantieingeing| +8857|AAAAAAAAJJCCAAAA|1997-10-27||Causal, im|1.81|1.28|3004002|edu packexporti #2|4|school-uniforms|3|Children|624|eseablecally|medium|52085677grey06264680|saddle|Dozen|Unknown|4|ationantieingeing| +8858|AAAAAAAAKJCCAAAA|1997-10-27|2000-10-26|Firms shall not achieve in a rules. Never strong copies represent small clothes. Studies |0.71|0.40|5003001|exportischolar #1|3|pop|5|Music|760|barcallyation|N/A|30942285midnight5519|puff|Dozen|Unknown|48|eingantieingeing| +8859|AAAAAAAAKJCCAAAA|2000-10-27||Royal, improved states could follow now so main crops. Weapons expect. Blind winners might deal tiny researchers. Compulsory jobs would facilitate together little nee|4.61|0.40|5003001|exportiamalg #2|3|maternity|1|Women|397|barcallyation|small|3733988301pale055464|red|Pallet|Unknown|16|n stantieingeing| +8860|AAAAAAAAMJCCAAAA|1997-10-27|1999-10-27|Future teams appreciate really modern, fine libraries; free adults will keep as only important executives. Deaf |0.98|0.73|8005001|scholarnameless #1|5|fitness|8|Sports|875|antiationeing|N/A|241385761saddle85453|red|N/A|Unknown|53|barcallyeingeing| +8861|AAAAAAAAMJCCAAAA|1999-10-28|2001-10-26|Future teams appreciate really modern, fine libraries; free adults will keep as only important executives. Deaf |0.66|0.53|8005001|amalgexporti #2|1|newborn|3|Children|573|priationanti|large|241385761saddle85453|white|Unknown|Unknown|23|oughtcallyeingeing| +8862|AAAAAAAAMJCCAAAA|2001-10-27||Theoretical, national arms would know at last years. Social rocks s|4.63|0.53|8005001|importounivamalg #6|2|camcorders|10|Electronics|597|ationn stanti|N/A|89835526550honeydew7|seashell|Gross|Unknown|9|ablecallyeingeing| +8863|AAAAAAAAPJCCAAAA|1997-10-27||Strong, british horses may not choose less. Results will not carry harsh workers. False claims will want over labour increases. Co|1.05|0.73|9003008|exportimaxi #8|3|computers|9|Books|339|n stpripri|N/A|790365213363rosy6539|slate|Case|Unknown|60|pricallyeingeing| +8864|AAAAAAAAAKCCAAAA|1997-10-27|2000-10-26|Responses find up to faint rates. Hours should not expire at a|9.54|7.53|9009009|maximaxi #9|9|science|9|Books|258|eingantiable|N/A|1499583rose458236914|almond|Dram|Unknown|5|esecallyeingeing| +8865|AAAAAAAAAKCCAAAA|2000-10-27||Responses find up to faint rates. Hours should not expire at a|6.55|7.53|2001002|amalgimporto #2|9|accessories|2|Men|187|eingantiable|economy|789peach756191879388|dim|Gram|Unknown|11|anticallyeingeing| +8866|AAAAAAAACKCCAAAA|1997-10-27|1999-10-27|Jewish standards should see from a images; women identify. Differences could transform clear to a changes; final designers support then functional, ethical groups. Backs find prett|3.56|2.17|1003001|exportiamalg #1|3|maternity|1|Women|364|esecallypri|large|2664royal56670033756|lace|Bunch|Unknown|13|callycallyeingeing| +8867|AAAAAAAACKCCAAAA|1999-10-28|2001-10-26|Etc necessary victims break. Truly frequent privileges rise most about a shares. Short skills will apply estimates; presidential trader|4.07|2.17|1003001|importoscholar #2|3|country|5|Music|364|esecallypri|N/A|2664royal56670033756|purple|Pound|Unknown|17|ationcallyeingeing| +8868|AAAAAAAACKCCAAAA|2001-10-27||Etc necessary victims break. Truly frequent privileges rise most about a shares. Short skills will apply estimates; presidential trader|7.36|2.17|10007012|brandunivamalg #12|7|personal|10|Electronics|571|esecallypri|N/A|violet71472653057485|pink|Case|Unknown|45|eingcallyeingeing| +8869|AAAAAAAAFKCCAAAA|1997-10-27||Main conditions can form further |7.56|5.82|7009010|maxibrand #10|9|mattresses|7|Home|869|n stcallyeing|N/A|27112907065papaya814|sienna|Tsp|Unknown|63|n stcallyeingeing| +8870|AAAAAAAAGKCCAAAA|1997-10-27|2000-10-26|Simple funds must keep. Anyway star members must seem to a stages; questions deal with a reserves; early|5.19|2.33|1001001|amalgamalg #1|1|dresses|1|Women|69|n stcally|medium|7moccasin44929301600|lawn|Ton|Unknown|15|barationeingeing| +8871|AAAAAAAAGKCCAAAA|2000-10-27||European, equal communications beat small reasons. From time to time certain lives rela|8.06|4.91|1001001|amalgamalg #2|1|dresses|1|Women|158|n stcally|economy|7moccasin44929301600|goldenrod|Ounce|Unknown|97|oughtationeingeing| +8872|AAAAAAAAIKCCAAAA|1997-10-27|1999-10-27|Men should take into the legs. Vital, new others must say late for a representatives; now low weeks could not consider as councils. Wrong, educational e|5.56|2.11|2002001|importoimporto #1|2|shirts|2|Men|425|antiableese|extra large|0383313771saddle2620|gainsboro|Ton|Unknown|52|ableationeingeing| +8873|AAAAAAAAIKCCAAAA|1999-10-28|2001-10-26|Men should take into the legs. Vital, new others must say late for a representatives; now low weeks could not consider as councils. Wrong, educational e|6.25|5.18|9011006|amalgunivamalg #6|2|cooking|9|Books|425|antiableese|N/A|0383313771saddle2620|saddle|Bundle|Unknown|7|priationeingeing| +8874|AAAAAAAAIKCCAAAA|2001-10-27||Officially central players reflect then national cons|6.32|2.52|7008005|namelessbrand #5|2|lighting|7|Home|222|ableableable|N/A|0383313771saddle2620|sky|Gram|Unknown|36|eseationeingeing| +8875|AAAAAAAALKCCAAAA|1997-10-27||Weekly cases make enough. Other, like tables go rates. Elderly, mass years think for example small available offen|6.10|3.11|3002002|importoexporti #2|2|infants|3|Children|560|barcallyanti|medium|2puff540588717200132|pink|Ton|Unknown|86|antiationeingeing| +8876|AAAAAAAAMKCCAAAA|1997-10-27|2000-10-26|Ago reasonable developers must represent in particular fair wild resources. Right gates shall not pass months. H|1.95|1.32|6003005|exporticorp #5|3|gold|6|Jewelry|341|oughtesepri|N/A|17wheat9097213460009|rosy|N/A|Unknown|88|callyationeingeing| +8877|AAAAAAAAMKCCAAAA|2000-10-27||Ago reasonable developers must represent in particular fair wild resources. Right gates shall not pass months. H|2.89|1.32|7005008|scholarbrand #8|5|blinds/shades|7|Home|221|oughtableable|N/A|17wheat9097213460009|tan|Dozen|Unknown|37|ationationeingeing| +8878|AAAAAAAAOKCCAAAA|1997-10-27|1999-10-27|Sentences can belong as. Prime, british records might imagine also teachers. Countries can|3.57|1.21|8004009|edu packnameless #9|4|camping|8|Sports|226|callyableable|N/A|yellow77926814875541|puff|Dozen|Unknown|67|eingationeingeing| +8879|AAAAAAAAOKCCAAAA|1999-10-28|2001-10-26|Clearly rare programs ought to try local, environment|8.61|1.21|8012010|importomaxi #10|4|guns|8|Sports|14|callyableable|N/A|yellow77926814875541|sandy|Pound|Unknown|64|n stationeingeing| +8880|AAAAAAAAOKCCAAAA|2001-10-27||Round arts play clearly very new levels. Children split perhaps double jobs. Events must respect often products. Reports|3.62|1.21|10014007|edu packamalgamalg #7|4|automotive|10|Electronics|14|callyableable|N/A|yellow77926814875541|rosy|Tsp|Unknown|20|bareingeingeing| +8881|AAAAAAAABLCCAAAA|1997-10-27||Plans must not provide more col|0.35|0.12|2001002|amalgimporto #2|1|accessories|2|Men|90|barn st|extra large|300yellow63382392919|powder|Bunch|Unknown|9|oughteingeingeing| +8882|AAAAAAAACLCCAAAA|1997-10-27|2000-10-26|Complex sports satisfy as. Backwards whole women could give suddenly at a bod|94.58|41.61|8007003|brandnameless #3|7|hockey|8|Sports|317|ationoughtpri|N/A|55257888202misty3505|chartreuse|Unknown|Unknown|42|ableeingeingeing| +8883|AAAAAAAACLCCAAAA|2000-10-27||Complex sports satisfy as. Backwards whole women could give suddenly at a bod|9.66|3.38|10002010|importounivamalg #10|7|camcorders|10|Electronics|193|ationoughtpri|N/A|9704903986329light51|lace|Box|Unknown|27|prieingeingeing| +8884|AAAAAAAAELCCAAAA|1997-10-27|1999-10-27|Nations create. Apparent, appropriate businesses use right |96.45|45.33|2002001|importoimporto #1|2|shirts|2|Men|265|anticallyable|petite|455456142saddle07744|pink|Gram|Unknown|8|eseeingeingeing| +8885|AAAAAAAAELCCAAAA|1999-10-28|2001-10-26|Nuclear police provide even weekends. Little consequences would go perhaps from a |1.88|45.33|2002001|corpcorp #6|6|rings|6|Jewelry|265|anticallyable|N/A|455456142saddle07744|indian|Tsp|Unknown|32|antieingeingeing| +8886|AAAAAAAAELCCAAAA|2001-10-27||Nuclear police provide even weekends. Little consequences would go perhaps from a |98.69|45.33|7014007|edu packnameless #7|14|glassware|7|Home|265|anticallyable|N/A|455456142saddle07744|steel|Dozen|Unknown|5|callyeingeingeing| +8887|AAAAAAAAHLCCAAAA|1997-10-27||Procedures give principal, social interviews. All star police could prevent highly new poor days; british systems hear also full troop|13.98|8.80|4001002|amalgedu pack #2|1|womens|4|Shoes|319|n stoughtpri|large|9728911391063sandy85|violet|Ounce|Unknown|28|ationeingeingeing| +8888|AAAAAAAAILCCAAAA|1997-10-27|2000-10-26|Able, actual men contribute beautiful, national orders. Days get just subsequently useful differences. Generally useful doctors look nations. Heavy minutes celebrate as good te|9.69|4.16|7004009|edu packbrand #9|4|curtains/drapes|7|Home|54|eseanti|N/A|65peru14196962263868|purple|Tsp|Unknown|51|eingeingeingeing| +8889|AAAAAAAAILCCAAAA|2000-10-27||Too whole abilities will overcome even in a|2.66|4.16|6009002|maxicorp #2|9|womens watch|6|Jewelry|54|eseanti|N/A|623salmon47341728078|burnished|Pallet|Unknown|58|n steingeingeing| +8890|AAAAAAAAKLCCAAAA|1997-10-27|1999-10-27|Then formal options|4.43|2.48|6002003|importocorp #3|2|diamonds|6|Jewelry|159|n stantiought|N/A|97196264papaya017352|thistle|Pound|Unknown|32|barn steingeing| +8891|AAAAAAAAKLCCAAAA|1999-10-28|2001-10-26|Then formal options|4.48|2.48|6002003|importobrand #4|2|costume|6|Jewelry|88|eingeing|N/A|97196264papaya017352|steel|N/A|Unknown|24|oughtn steingeing| +8892|AAAAAAAAKLCCAAAA|2001-10-27||Then formal options|2.35|1.24|3003001|exportiexporti #1|3|toddlers|3|Children|88|eingeing|extra large|43116233854thistle08|sky|Bunch|Unknown|10|ablen steingeing| +8893|AAAAAAAANLCCAAAA|1997-10-27||Ancient, original tests would go carefully on a months. A|89.91|47.65|6012006|importobrand #6|12|costume|6|Jewelry|935|antiprin st|N/A|34olive7163189601481|slate|Gram|Unknown|65|prin steingeing| +8894|AAAAAAAAOLCCAAAA|1997-10-27|2000-10-26|Bodies exercise however. Already political police must|4.31|1.81|5001001|amalgscholar #1|1|rock|5|Music|460|barcallyese|N/A|104659957puff5442970|tan|Case|Unknown|54|esen steingeing| +8895|AAAAAAAAOLCCAAAA|2000-10-27||Bodies exercise however. Already political police must|1.26|0.84|5001001|edu packscholar #2|4|classical|5|Music|460|barcallyese|N/A|86861560plum69023086|burlywood|Bunch|Unknown|32|antin steingeing| +8896|AAAAAAAAAMCCAAAA|1997-10-27|1999-10-27|Services clear tory, insufficient solutions. Tonnes would not provide more great parties. Human matters must use at all old students. Historica|8.58|5.40|5004001|edu packscholar #1|4|classical|5|Music|90|barn st|N/A|7892546medium2749470|navy|Box|Unknown|3|callyn steingeing| +8897|AAAAAAAAAMCCAAAA|1999-10-28|2001-10-26|Services clear tory, insufficient solutions. Tonnes would not provide more great parties. Human matters must use at all old students. Historica|0.26|5.40|5004001|exportiimporto #2|3|pants|2|Men|131|barn st|small|salmon93571200393349|tomato|Ounce|Unknown|13|ationn steingeing| +8898|AAAAAAAAAMCCAAAA|2001-10-27||Special classes attempt usually. Days shall avoid over pupils. Open implications illustrate confid|7.70|5.40|5004001|importoamalg #1|3|fragrances|1|Women|498|barn st|large|380368894red48634570|seashell|Gross|Unknown|25|eingn steingeing| +8899|AAAAAAAADMCCAAAA|1997-10-27||Children dictate however|1.66|0.53|6009004|maxicorp #4|9|womens watch|6|Jewelry|300|barbarpri|N/A|6pale399362494359366|plum|N/A|Unknown|40|n stn steingeing| +8900|AAAAAAAAEMCCAAAA|1997-10-27|2000-10-26|Players leave before too great decisions. Happy occupations used to thrive |2.31|1.33|1004001|edu packamalg #1|4|swimwear|1|Women|254|eseantiable|petite|09769369891641pink72|saddle|Box|Unknown|6|barbarn steing| +8901|AAAAAAAAEMCCAAAA|2000-10-27||Players leave before too great decisions. Happy occupations used to thrive |1.59|1.33|3001002|amalgexporti #2|4|newborn|3|Children|254|eseantiable|medium|09769369891641pink72|papaya|Oz|Unknown|31|oughtbarn steing| +8902|AAAAAAAAGMCCAAAA|1997-10-27|1999-10-27|Arguments let temporarily popular, interesting authorities. Open words used to confide simple, large pupils. Direct careers might see |1.18|0.57|3003001|exportiexporti #1|3|toddlers|3|Children|757|ationantiation|large|12272seashell8249272|metallic|Oz|Unknown|7|ablebarn steing| +8903|AAAAAAAAGMCCAAAA|1999-10-28|2001-10-26|Low, inc attempts get tomorrow long small tests. Possible |3.58|1.21|4001002|amalgedu pack #2|1|womens|4|Shoes|757|ationantiation|economy|421165593786pale5604|smoke|Unknown|Unknown|10|pribarn steing| +8904|AAAAAAAAGMCCAAAA|2001-10-27||Low, inc attempts get tomorrow long small tests. Possible |3.38|1.21|5003001|exportischolar #1|1|pop|5|Music|757|ationantiation|N/A|421165593786pale5604|slate|Cup|Unknown|31|esebarn steing| +8905|AAAAAAAAJMCCAAAA|1997-10-27||Interesting programmes used to appear even. Symbolic prices go beautifu|97.63|65.41|9003002|exportimaxi #2|3|computers|9|Books|253|priantiable|N/A|78346142428honeydew0|violet|Carton|Unknown|25|antibarn steing| +8906|AAAAAAAAKMCCAAAA|1997-10-27|2000-10-26|Burning instruments promote rarely attempts. Dry feet try british, entire changes. Chemicals find well important situations. Planes emerge clubs. Less extra others provide old, indus|3.89|2.95|3003001|exportiexporti #1|3|toddlers|3|Children|165|anticallyought|small|306850green664877688|salmon|Bunch|Unknown|76|callybarn steing| +8907|AAAAAAAAKMCCAAAA|2000-10-27||Different conditions shall understand. Urban, regional arguments might sell in|1.68|0.97|4001002|amalgedu pack #2|1|womens|4|Shoes|165|anticallyought|medium|pink3244668313915258|slate|Lb|Unknown|16|ationbarn steing| +8908|AAAAAAAAMMCCAAAA|1997-10-27|1999-10-27|Junior, victorian hon|3.01|1.02|5003001|exportischolar #1|3|pop|5|Music|45|antiese|N/A|0810668papaya8938637|blush|Box|Unknown|42|eingbarn steing| +8909|AAAAAAAAMMCCAAAA|1999-10-28|2001-10-26|Small days|1.09|1.02|7016010|corpnameless #10|3|furniture|7|Home|63|prically|N/A|0810668papaya8938637|maroon|Each|Unknown|29|n stbarn steing| +8910|AAAAAAAAMMCCAAAA|2001-10-27||Small days|6.25|2.62|2003001|exportiimporto #1|3|pants|2|Men|63|prically|medium|226peru2618876974090|maroon|N/A|Unknown|100|baroughtn steing| +8911|AAAAAAAAPMCCAAAA|1997-10-27||All royal|83.26|39.96|1003002|exportiamalg #2|3|maternity|1|Women|204|esebarable|petite|36471318263plum61922|violet|Case|Unknown|28|oughtoughtn steing| +8912|AAAAAAAAANCCAAAA|1997-10-27|2000-10-26|European results can send questions; essentially high soldiers dry more almost european families; public, economic years should not clos|3.25|2.92|2004001|edu packimporto #1|4|sports-apparel|2|Men|224|eseableable|small|223296royal651166789|thistle|Case|Unknown|25|ableoughtn steing| +8913|AAAAAAAAANCCAAAA|2000-10-27||Observations could make only male words. Different, net plants shall not carry then left rules. Others may |9.26|2.92|5002002|importoscholar #2|2|country|5|Music|51|eseableable|N/A|26741356sandy9429715|saddle|N/A|Unknown|75|prioughtn steing| +8914|AAAAAAAACNCCAAAA|1997-10-27|1999-10-27|French authorities like literally j|6.63|3.97|1003001|exportiamalg #1|3|maternity|1|Women|514|eseoughtanti|medium|wheat491075091395283|green|Bundle|Unknown|31|eseoughtn steing| +8915|AAAAAAAACNCCAAAA|1999-10-28|2001-10-26|French authorities like literally j|4.72|3.97|8012006|importomaxi #6|12|guns|8|Sports|725|antiableation|N/A|51smoke9675795060891|red|Box|Unknown|12|antioughtn steing| +8916|AAAAAAAACNCCAAAA|2001-10-27||Much acceptable intentions end by a files. Obviously regional documents come so well reasonable courts. Different hours use even musical|9.98|3.97|4003001|exportiedu pack #1|3|kids|4|Shoes|725|antiableation|petite|51smoke9675795060891|spring|Tsp|Unknown|31|callyoughtn steing| +8917|AAAAAAAAFNCCAAAA|1997-10-27||At best favorite goods make far different days. Unable, labour hills use live, old ladies. Days could withdraw longer important responses. Groups tell substanti|6.29|2.89|3003002|exportiexporti #2|3|toddlers|3|Children|753|priantiation|petite|6318870tomato4864812|metallic|Ounce|Unknown|48|ationoughtn steing| +8918|AAAAAAAAGNCCAAAA|1997-10-27|2000-10-26|Closed, good condition|7.02|5.40|8015009|scholarmaxi #9|15|fishing|8|Sports|10|barought|N/A|535334565335white287|moccasin|Oz|Unknown|5|eingoughtn steing| +8919|AAAAAAAAGNCCAAAA|2000-10-27||Partially real duties can receive also now linguistic chains. Red films would not put around white consequences; financial employers will beat. Late young kinds mig|6.50|5.40|2001002|amalgimporto #2|1|accessories|2|Men|10|barought|medium|535334565335white287|cream|Bundle|Unknown|42|n stoughtn steing| +8920|AAAAAAAAINCCAAAA|1997-10-27|1999-10-27|Rich, keen requirements face at |8.60|5.33|2004001|edu packimporto #1|4|sports-apparel|2|Men|81|oughteing|petite|35769purple157804644|rosy|Unknown|Unknown|32|barablen steing| +8921|AAAAAAAAINCCAAAA|1999-10-28|2001-10-26|Twice commercial readers would not recognize i|4.56|2.41|10014014|edu packamalgamalg #14|14|automotive|10|Electronics|81|oughteing|N/A|35769purple157804644|snow|Bunch|Unknown|27|oughtablen steing| +8922|AAAAAAAAINCCAAAA|2001-10-27||Also real colours participate too. Units must retire proper sites. Somewhat historic types cut softly beautiful groups. Good, executive |1.70|2.41|10014014|univunivamalg #7|14|travel|9|Books|226|callyableable|N/A|634hot08327723546276|violet|Ounce|Unknown|9|ableablen steing| +8923|AAAAAAAALNCCAAAA|1997-10-27||Popular, heavy companies create over various reforms. Other parts organise legs. Private rounds file clearly. Christians stop weekly effectively social examinations; p|2.04|1.40|7012004|importonameless #4|12|paint|7|Home|103|pribarought|N/A|5144599purple8275417|wheat|Dram|Unknown|29|priablen steing| +8924|AAAAAAAAMNCCAAAA|1997-10-27|2000-10-26|Forwards public lives could not stabilise enough national economic cars. False, public versions look places. Surprising, unique years drag; potential, common tr|0.23|0.08|4002001|importoedu pack #1|2|mens|4|Shoes|267|ationcallyable|N/A|5769seashell04767913|midnight|Tbl|Unknown|60|eseablen steing| +8925|AAAAAAAAMNCCAAAA|2000-10-27||Forwards public lives could not stabilise enough national economic cars. False, public versions look places. Surprising, unique years drag; potential, common tr|8.29|3.56|4002001|importoamalg #2|2|fragrances|1|Women|267|ationcallyable|large|27702508orchid094211|medium|Bunch|Unknown|26|antiablen steing| +8926|AAAAAAAAONCCAAAA|1997-10-27|1999-10-27|More just men used to say more certai|96.36|73.23|2003001|exportiimporto #1|3|pants|2|Men|157|ationantiought|medium|477535269541lace7873|smoke|Ounce|Unknown|21|callyablen steing| +8927|AAAAAAAAONCCAAAA|1999-10-28|2001-10-26|Boys show less. Deputies sleep really more excessive values. Different offenders mean increasi|1.10|73.23|2003001|edu packimporto #2|3|sports-apparel|2|Men|743|prieseation|large|69964powder560272411|sienna|Tbl|Unknown|36|ationablen steing| +8928|AAAAAAAAONCCAAAA|2001-10-27||Boys show less. Deputies sleep really more excessive values. Different offenders mean increasi|81.09|71.35|2003001|namelessunivamalg #4|8|scanners|10|Electronics|324|prieseation|N/A|69964powder560272411|peru|Ounce|Unknown|16|eingablen steing| +8929|AAAAAAAABOCCAAAA|1997-10-27||Specific, medical users could afford to a aspects; male pictures could think certainly english, short years. Profound choices establish in a products. Vital residents might take far t|1.09|0.64|4004002|edu packedu pack #2|4|athletic|4|Shoes|98|eingn st|small|2615195mint272362864|sienna|Tsp|Unknown|32|n stablen steing| +8930|AAAAAAAACOCCAAAA|1997-10-27|2000-10-26|Chosen interests want. Christian companies explore detailed, short-term talks. Una|9.67|3.67|10002006|importounivamalg #6|2|camcorders|10|Electronics|121|oughtableought|N/A|9735759791lavender27|mint|Bunch|Unknown|26|barprin steing| +8931|AAAAAAAACOCCAAAA|2000-10-27||Chosen interests want. Christian companies explore detailed, short-term talks. Una|3.23|1.19|10002006|namelessmaxi #4|8|romance|9|Books|121|oughtableought|N/A|9735759791lavender27|smoke|Cup|Unknown|4|oughtprin steing| +8932|AAAAAAAAEOCCAAAA|1997-10-27|1999-10-27|Deeper effective components would not export somehow familiar black men. Rapidly likely things turn at a teams. Others should risk resources. Players ret|5.78|4.45|6009005|maxicorp #5|9|womens watch|6|Jewelry|24|eseable|N/A|03723307799papaya502|saddle|Dozen|Unknown|31|ableprin steing| +8933|AAAAAAAAEOCCAAAA|1999-10-28|2001-10-26|Able criteria could break normal, normal documents. Then metropolitan taxes may possess more huge early eyes. Sure long |1.64|1.16|2003002|exportiimporto #2|3|pants|2|Men|350|barantipri|petite|62308842peru44293571|indian|Tbl|Unknown|6|priprin steing| +8934|AAAAAAAAEOCCAAAA|2001-10-27||Disputes will take hard roads. Societies will not offer so black members. Chemical laws would recover lesser parties. Up to narrow skills apply comparatively old contracts. Men as|18.03|14.96|2003002|amalgedu pack #1|1|womens|4|Shoes|242|ableeseable|small|62308842peru44293571|yellow|Bunch|Unknown|98|eseprin steing| +8935|AAAAAAAAHOCCAAAA|1997-10-27||Residents used to live royal children; deaf, sensitive observations may include ahead increased, outstandin|0.72|0.45|3004002|edu packexporti #2|4|school-uniforms|3|Children|590|barn stanti|petite|3plum048979712820264|midnight|Lb|Unknown|51|antiprin steing| +8936|AAAAAAAAIOCCAAAA|1997-10-27|2000-10-26|Technological characters want a|4.64|3.24|9011009|amalgunivamalg #9|11|cooking|9|Books|559|n stantianti|N/A|slate627041161462084|grey|Carton|Unknown|54|callyprin steing| +8937|AAAAAAAAIOCCAAAA|2000-10-27||Technological characters want a|56.22|32.60|9011009|amalgscholar #2|11|rock|5|Music|559|n stantianti|N/A|slate627041161462084|salmon|Box|Unknown|9|ationprin steing| +8938|AAAAAAAAKOCCAAAA|1997-10-27|1999-10-27|Chief minerals go from time to time others. Sales can enter remarkable |18.43|14.37|10011014|amalgamalgamalg #14|11|disk drives|10|Electronics|218|eingoughtable|N/A|612486996861tomato91|lawn|Dram|Unknown|29|eingprin steing| +8939|AAAAAAAAKOCCAAAA|1999-10-28|2001-10-26|Ro|0.55|0.26|10011014|amalgimporto #2|11|accessories|2|Men|218|eingoughtable|petite|612486996861tomato91|brown|Dozen|Unknown|100|n stprin steing| +8940|AAAAAAAAKOCCAAAA|2001-10-27||Financial parameters used to take forthwith. Effects break gently high weeks. Articles make. Po|7.16|0.26|9012001|importounivamalg #1|11|home repair|9|Books|143|eingoughtable|N/A|612486996861tomato91|yellow|Oz|Unknown|52|baresen steing| +8941|AAAAAAAANOCCAAAA|1997-10-27||Values could not travel later on the companies. Leading things miss more available years. Together related sources should know voluntary papers. Commercial e|4.17|1.54|6016006|corpbrand #6|16|consignment|6|Jewelry|982|ableeingn st|N/A|438707149272680plum8|beige|Unknown|Unknown|54|oughtesen steing| +8942|AAAAAAAAOOCCAAAA|1997-10-27|2000-10-26|Still unlikely rivers go very. Almost eastern years grow special, empty journalists; good laws comply just possible teachers. Low, economic documents want in a birds. American, integrate|9.51|7.79|1004001|edu packamalg #1|4|swimwear|1|Women|167|ationcallyought|large|12847912828salmon630|seashell|Unknown|Unknown|14|ableesen steing| +8943|AAAAAAAAOOCCAAAA|2000-10-27||Once inner claims|1.81|7.79|1004001|importoscholar #2|2|country|5|Music|167|ationcallyought|N/A|12847912828salmon630|steel|Gram|Unknown|59|priesen steing| +8944|AAAAAAAAAPCCAAAA|1997-10-27|1999-10-27|Important trends might not work. So political institutions keep above minute cases. Again blind participants record. Now minor col|6.06|5.02|5001001|amalgscholar #1|1|rock|5|Music|223|priableable|N/A|2380883magenta969153|purple|Bunch|Unknown|2|eseesen steing| +8945|AAAAAAAAAPCCAAAA|1999-10-28|2001-10-26|Centres read now musical bills. Deaf, norm|57.89|49.78|6015002|scholarbrand #2|15|custom|6|Jewelry|223|priableable|N/A|2380883magenta969153|hot|Box|Unknown|49|antiesen steing| +8946|AAAAAAAAAPCCAAAA||||0.52||8008007|namelessnameless #7|15|outdoor|||223||N/A|2380883magenta969153|metallic|Bundle|Unknown||callyesen steing| +8947|AAAAAAAADPCCAAAA|1997-10-27||Especially true items might supply particularly. Black, automatic words might develop post-war problems. Fresh, visible workers could not appe|4.23|2.24|9001008|amalgmaxi #8|1|arts|9|Books|206|callybarable|N/A|858681669953purple99|red|Tsp|Unknown|32|ationesen steing| +8948|AAAAAAAAEPCCAAAA|1997-10-27|2000-10-26|African, far months will touch in a findings. Important pieces shall not remain quite at a models. Cases worry. Crudely magnificent ministers should not postpone worse members. Current|0.66|0.23|2004001|edu packimporto #1|4|sports-apparel|2|Men|53|prianti|medium|5772539seashell38109|peach|Ton|Unknown|12|eingesen steing| +8949|AAAAAAAAEPCCAAAA|2000-10-27||Good, good developments believe so. Significant, real firms u|3.34|2.87|1001002|amalgamalg #2|1|dresses|1|Women|255|antiantiable|N/A|0668lavender17128083|pale|Dozen|Unknown|18|n stesen steing| +8950|AAAAAAAAGPCCAAAA|1997-10-27|1999-10-27|Even real wheels could crumble new, industrial plants. Almost mass blacks tend really. Mediterranean changes turn false too local police. More than conventional servic|4.68|4.21|8004001|edu packnameless #1|4|camping|8|Sports|85|antieing|N/A|061753410585963lime5|cyan|Lb|Unknown|44|barantin steing| +8951|AAAAAAAAGPCCAAAA|1999-10-28|2001-10-26|Old services may like well wooden, italian eyes. Great, electric reactions used to like. Pockets produce everywhere acids. Dramatic, sorry boys give public lines; other ministers|8.86|3.27|5003002|exportischolar #2|4|pop|5|Music|634|antieing|N/A|2534royal59201163628|peach|Dozen|Unknown|97|oughtantin steing| +8952|AAAAAAAAGPCCAAAA|2001-10-27||Conservatives will write earlier international, necessary years. Arms would prevent international, singl|0.76|3.27|6016001|corpbrand #1|4|consignment|6|Jewelry|634|eseprically|N/A|2534royal59201163628|linen|Oz|Unknown|22|ableantin steing| +8953|AAAAAAAAJPCCAAAA|1997-10-27||Sources cannot fight as on a names. Years ought to contact well in front of a arms. Prisoners try upwards. Nice, nice drivers vary up to as enormo|1.28|0.96|8006004|corpnameless #4|6|football|8|Sports|274|eseationable|N/A|615153277881978lime6|orange|Carton|Unknown|20|priantin steing| +8954|AAAAAAAAKPCCAAAA|1997-10-27|2000-10-26|American, medical ties should not halt too necessary soldiers. Grey hands escape sufficient, other parties. Centuries prove also ordinary managers. Then western problems|5.59|3.13|2004001|edu packimporto #1|4|sports-apparel|2|Men|874|eseationeing|small|835928516166653pale3|wheat|Pallet|Unknown|5|eseantin steing| +8955|AAAAAAAAKPCCAAAA|2000-10-27||American, medical ties should not halt too necessary soldiers. Grey hands escape sufficient, other parties. Centuries prove also ordinary managers. Then western problems|2.16|3.13|8007006|brandnameless #6|7|hockey|8|Sports|973|eseationeing|N/A|0pink220721028457766|bisque|Bunch|Unknown|2|antiantin steing| +8956|AAAAAAAAMPCCAAAA|1997-10-27|1999-10-27|Patterns catch already mothers. Tight swiss activit|11.74|3.87|3002001|importoexporti #1|2|infants|3|Children|146|callyeseought|large|2278ivory53937353079|pink|Cup|Unknown|37|callyantin steing| +8957|AAAAAAAAMPCCAAAA|1999-10-28|2001-10-26|In general estimated reasons say the|2.87|1.40|4002002|importoedu pack #2|2|mens|4|Shoes|414|eseoughtese|large|2278ivory53937353079|pink|Unknown|Unknown|28|ationantin steing| +8958|AAAAAAAAMPCCAAAA|2001-10-27||In general estimated reasons say the|3.97|3.01|4002002|amalgbrand #9|2|bathroom|7|Home|414|eseoughtese|N/A|172008peru3084888885|slate|Ounce|Unknown|60|eingantin steing| +8959|AAAAAAAAPPCCAAAA|1997-10-27||Pleasant kinds would not seek opportunities. Local methods react home excellent, video-taped cars. Most ideal signs suggest very on a areas. Often easy developments visit rates. Relig|5.79|3.76|8004010|edu packnameless #10|4|camping|8|Sports|24|eseable|N/A|46909rose57592524798|royal|Tbl|Unknown|70|n stantin steing| +8960|AAAAAAAAAADCAAAA|1997-10-27|2000-10-26|Whole reports will not acquire; looks get then japanese, basic creditors. New, fortunate professionals encourage firmly rich roles; however secondary projects might |2.72|1.55|8009001|maxinameless #1|9|optics|8|Sports|221|oughtableable|N/A|74613096yellow449748|purple|Cup|Unknown|24|barcallyn steing| +8961|AAAAAAAAAADCAAAA|2000-10-27||Whole reports will not acquire; looks get then japanese, basic creditors. New, fortunate professionals encourage firmly rich roles; however secondary projects might |7.18|4.30|5002002|importoscholar #2|2|country|5|Music|221|oughtableable|N/A|74613096yellow449748|powder|Oz|Unknown|23|oughtcallyn steing| +8962|AAAAAAAACADCAAAA|1997-10-27|1999-10-27|Cells could believe en|9.29|6.50|6007003|brandcorp #3|7|pendants|6|Jewelry|640|baresecally|N/A|6642097violet3784679|steel|Unknown|Unknown|59|ablecallyn steing| +8963|AAAAAAAACADCAAAA|1999-10-28|2001-10-26|Major drivers fall perhaps expert needs. Wives support indicators|9.01|4.59|6011004|amalgbrand #4|7|semi-precious|6|Jewelry|640|baresecally|N/A|6642097violet3784679|seashell|Box|Unknown|15|pricallyn steing| +8964|AAAAAAAACADCAAAA|2001-10-27||Constant, special tests used to dine most also recent fields. Hours agree also readers. Functional, red shares should cut soon within the populations. Teachers go increasingly nervous wom|7.65|4.59|6011004|univbrand #5|10|jewelry boxes|6|Jewelry|39|baresecally|N/A|6642097violet3784679|rose|Pound|Unknown|10|esecallyn steing| +8965|AAAAAAAAFADCAAAA|1997-10-27||Also interesting lawyers will show more; always international eyes can cover. Schools shall excite in the drugs.|8.68|4.51|5002002|importoscholar #2|2|country|5|Music|155|antiantiought|N/A|royal186048873066200|rose|Cup|Unknown|11|anticallyn steing| +8966|AAAAAAAAGADCAAAA|1997-10-27|2000-10-26|New, good opportu|4.99|3.04|8001007|amalgnameless #7|1|athletic shoes|8|Sports|389|n steingpri|N/A|0sky5904235574466406|peru|Lb|Unknown|5|callycallyn steing| +8967|AAAAAAAAGADCAAAA|2000-10-27||New, good opportu|3.92|3.04|3003002|exportiexporti #2|3|toddlers|3|Children|452|n steingpri|N/A|10grey80877884409534|peru|Case|Unknown|68|ationcallyn steing| +8968|AAAAAAAAIADCAAAA|1997-10-27|1999-10-27|Women ignore. Sure, new collections must not look there entire seats. New, regional others sleep too large children. New points issue in a men; times might go local, german males. Inc, short se|3.12|1.21|5002001|importoscholar #1|2|country|5|Music|102|ablebarought|N/A|85426639grey60624903|powder|Box|Unknown|86|eingcallyn steing| +8969|AAAAAAAAIADCAAAA|1999-10-28|2001-10-26|Women ignore. Sure, new collections must not look there entire seats. New, regional others sleep too large children. New points issue in a men; times might go local, german males. Inc, short se|7.95|6.28|5002001|importomaxi #6|2|business|9|Books|371|oughtationpri|N/A|73641863340thistle64|wheat|Dozen|Unknown|26|n stcallyn steing| +8970|AAAAAAAAIADCAAAA|2001-10-27||Now industrial aspects shut on a occupations. Current, true terms cannot add increasingly|0.92|0.71|9002007|importomaxi #7|2|business|9|Books|37|oughtationpri|N/A|73641863340thistle64|metallic|Bunch|Unknown|52|barationn steing| +8971|AAAAAAAALADCAAAA|1997-10-27||Rural, good models throw then from an dollars. Sudden buildings affect always quick children. Legal, quick members ensure then; times pick|3.37|1.21|2003002|exportiimporto #2|3|pants|2|Men|146|callyeseought|medium|41837rose00000341276|rose|Each|Unknown|17|oughtationn steing| +8972|AAAAAAAAMADCAAAA|1997-10-27|2000-10-26|Simply effective patients attack possibly black details. Functional, successful decisions make legislative others; japanese channels will obta|3.14|2.60|1002001|importoamalg #1|2|fragrances|1|Women|126|callyableought|medium|4800957805664wheat88|plum|Case|Unknown|14|ableationn steing| +8973|AAAAAAAAMADCAAAA|2000-10-27||Simply effective patients attack possibly black details. Functional, successful decisions make legislative others; japanese channels will obta|1.55|2.60|1002001|exportinameless #4|3|basketball|8|Sports|861|oughtcallyeing|N/A|4800957805664wheat88|white|Oz|Unknown|20|priationn steing| +8974|AAAAAAAAOADCAAAA|1997-10-27|1999-10-27|Facilities increase. Economic holders see ancient animals. Little e|0.98|0.55|8013005|exportimaxi #5|13|sailing|8|Sports|87|ationeing|N/A|424499515458powder99|violet|Bundle|Unknown|7|eseationn steing| +8975|AAAAAAAAOADCAAAA|1999-10-28|2001-10-26|Exclusively damp terms shall not recover ages. High, industrial boards reach major citizens. Words decide comparatively. Current influences shall not increase at a ambitions; then delicious organis|3.33|1.86|8013005|importoedu pack #2|2|mens|4|Shoes|281|oughteingable|small|424499515458powder99|orange|Box|Unknown|45|antiationn steing| +8976|AAAAAAAAOADCAAAA|2001-10-27||Exclusively damp terms shall not recover ages. High, industrial boards reach major citizens. Words decide comparatively. Current influences shall not increase at a ambitions; then delicious organis|2.68|1.86|4003001|exportiedu pack #1|2|kids|4|Shoes|281|oughteingable|medium|68205852686656red219|medium|Dram|Unknown|14|callyationn steing| +8977|AAAAAAAABBDCAAAA|1997-10-27||Extremely nervous cases sleep per annum owners; men should deal independently rather inc clergy. Architects achieve t|56.72|34.03|2004002|edu packimporto #2|4|sports-apparel|2|Men|506|callybaranti|large|89199snow52640659385|slate|Cup|Unknown|28|ationationn steing| +8978|AAAAAAAACBDCAAAA|1997-10-27|2000-10-26|Well independent scores fight rare changes. Scottish rights would not give; implicit, modern services like yet. Conservative, effective yards should marry about a buildings. Valid, m|0.50|0.40|9011003|amalgunivamalg #3|11|cooking|9|Books|834|eseprieing|N/A|5260472595pink032540|turquoise|Tbl|Unknown|15|eingationn steing| +8979|AAAAAAAACBDCAAAA|2000-10-27||Books would not move far men. Too black problems steal gardens. Ways work now absolute, monetary voices. Subtle rooms sort of course di|9.81|4.12|1003002|exportiamalg #2|11|maternity|1|Women|834|eseprieing|medium|5260472595pink032540|peach|Dozen|Unknown|23|n stationn steing| +8980|AAAAAAAAEBDCAAAA|1997-10-27|1999-10-27|Normally general products might not escape now frequent affairs. Words cannot see. Magazines get hardly subsequent windows. Dramatic, minute requirements m|1.08|0.96|10012005|importoamalgamalg #5|12|monitors|10|Electronics|660|barcallycally|N/A|57691668yellow499255|purple|Ounce|Unknown|31|bareingn steing| +8981|AAAAAAAAEBDCAAAA|1999-10-28|2001-10-26|Normally general products might not escape now frequent affairs. Words cannot see. Magazines get hardly subsequent windows. Dramatic, minute requirements m|3.59|0.96|3003002|exportiexporti #2|3|toddlers|3|Children|96|callyn st|medium|67897440924309linen5|turquoise|Unknown|Unknown|8|oughteingn steing| +8982|AAAAAAAAEBDCAAAA|2001-10-27||Hours would ensure now. Variables should stamp; recent attempts maintain below far rivals. Trying refugees stick deeply quite p|4.81|0.96|3003002|univunivamalg #7|3|travel|9|Books|224|callyn st|N/A|67897440924309linen5|snow|Tsp|Unknown|58|ableeingn steing| +8983|AAAAAAAAHBDCAAAA|1997-10-27||At all pub|5.68|2.55|6006008|corpcorp #8|6|rings|6|Jewelry|118|eingoughtought|N/A|70771plum58045953846|powder|Case|Unknown|65|prieingn steing| +8984|AAAAAAAAIBDCAAAA|1997-10-27|2000-10-26|Contacts mak|4.56|2.23|7016005|corpnameless #5|16|furniture|7|Home|275|antiationable|N/A|20pink32245317020014|white|Pallet|Unknown|60|eseeingn steing| +8985|AAAAAAAAIBDCAAAA|2000-10-27||Contacts mak|1.73|2.23|7016005|exportiamalg #2|3|maternity|1|Women|334|esepripri|small|04drab65085586280326|powder|Lb|Unknown|26|antieingn steing| +8986|AAAAAAAAKBDCAAAA|1997-10-27|1999-10-27|Sons would free. There natural structures used to fight never gay names. Vast stairs put still up to a payments. Long, sovie|0.98|0.72|6009003|maxicorp #3|9|womens watch|6|Jewelry|106|callybarought|N/A|55336995rosy59765693|plum|Tsp|Unknown|19|callyeingn steing| +8987|AAAAAAAAKBDCAAAA|1999-10-28|2001-10-26|Sons would free. There natural structures used to fight never gay names. Vast stairs put still up to a payments. Long, sovie|2.41|0.72|9002012|importomaxi #12|2|business|9|Books|106|callybarought|N/A|55336995rosy59765693|pale|Box|Unknown|37|ationeingn steing| +8988|AAAAAAAAKBDCAAAA|2001-10-27||Demands pay on a awards. Complete figures rai|4.21|2.18|5004001|edu packscholar #1|4|classical|5|Music|106|callybarought|N/A|55336995rosy59765693|violet|Dozen|Unknown|4|eingeingn steing| +8989|AAAAAAAANBDCAAAA|1997-10-27||Hours should look very usually darling men. Single pounds would see else results. Tired courts may not improve wide records; findings ca|3.81|2.20|8004010|edu packnameless #10|4|camping|8|Sports|631|oughtprically|N/A|953ivory125295469015|snow|Oz|Unknown|97|n steingn steing| +8990|AAAAAAAAOBDCAAAA|1997-10-27|2000-10-26|Still given studies reach strong systems. Base countries study modern, independent eyebrows. Alt|3.11|1.80|5003001|exportischolar #1|3|pop|5|Music|272|ableationable|N/A|688569533759sandy587|thistle|Box|Unknown|52|barn stn steing| +8991|AAAAAAAAOBDCAAAA|2000-10-27||Vast banks make i|2.86|2.03|2002002|importoimporto #2|3|shirts|2|Men|272|ableationable|medium|688569533759sandy587|sandy|Case|Unknown|32|oughtn stn steing| +8992|AAAAAAAAACDCAAAA|1997-10-27|1999-10-27|Processes cannot insure ago betwe|82.30|67.48|5004001|edu packscholar #1|4|classical|5|Music|325|antiablepri|N/A|0840196910goldenrod4|purple|Gram|Unknown|60|ablen stn steing| +8993|AAAAAAAAACDCAAAA|1999-10-28|2001-10-26|Genuine clothes can find still meanwhile major sales. Cases use essentially huge firms. Palestinian terms should accept further all free workers. Closely relative cars might lead in particular natio|3.04|1.82|5004001|edu packcorp #2|4|bracelets|6|Jewelry|367|ationcallypri|N/A|0840196910goldenrod4|ivory|Case|Unknown|25|prin stn steing| +8994|AAAAAAAAACDCAAAA|2001-10-27||Interesting months could remain just discussions; actors avail in a trees. Trees will |7.50|1.82|5004001|amalgamalg #1|1|dresses|1|Women|367|ationcallypri|medium|0840196910goldenrod4|peach|Pallet|Unknown|42|esen stn steing| +8995|AAAAAAAADCDCAAAA|1997-10-27||Recent, unable partners may continue good, blac|0.69|0.35|9005008|scholarmaxi #8|5|history|9|Books|936|callyprin st|N/A|0808600275red7442169|navy|Ounce|Unknown|9|antin stn steing| +8996|AAAAAAAAECDCAAAA|1997-10-27|2000-10-26|Main practices will seem with the issues; members could not keep seriously at a resources; full, environmental days might not end late, dutch children. In private small applica|3.98|1.75|7006007|corpbrand #7|6|rugs|7|Home|624|eseablecally|N/A|232876plum9567427764|peach|Each|Unknown|5|callyn stn steing| +8997|AAAAAAAAECDCAAAA|2000-10-27||Considerable children listen again on a circumstances. Miles decrease as. Fine, heavy skills mak|8.67|4.24|8015008|scholarmaxi #8|15|fishing|8|Sports|624|eseablecally|N/A|323622201linen812728|yellow|Carton|Unknown|28|ationn stn steing| +8998|AAAAAAAAGCDCAAAA|1997-10-27|1999-10-27|Relatives carry sheer stu|3.74|2.80|4001001|amalgedu pack #1|1|womens|4|Shoes|267|ationcallyable|large|6448266navy443239490|white|Dozen|Unknown|36|eingn stn steing| +8999|AAAAAAAAGCDCAAAA|1999-10-28|2001-10-26|Fundame|2.67|2.26|4001001|importoscholar #2|1|country|5|Music|206|ationcallyable|N/A|6448266navy443239490|rosy|Dozen|Unknown|50|n stn stn steing| +9000|AAAAAAAAGCDCAAAA|2001-10-27||Best different forms would tell beyond urgent letters. Materials shall not make just worthwhile, high times. Wide other americans see students. Populations suffer clear years. Howe|3.86|2.26|4001001|edu packimporto #1|4|sports-apparel|2|Men|206|callybarable|large|525wheat193050373931|turquoise|Ounce|Unknown|48|barbarbarn st| +9001|AAAAAAAAJCDCAAAA|1997-10-27||Parties may not happen long wages. Bizarre, military trusts could s|1.58|0.82|8001002|amalgnameless #2|1|athletic shoes|8|Sports|612|ableoughtcally|N/A|7750metallic40286665|powder|N/A|Unknown|22|oughtbarbarn st| +9002|AAAAAAAAKCDCAAAA|1997-10-27|2000-10-26|So alternative bones make very blind, foreign things. Policies find main, industrial cases. Funds must buy enough quite quiet years. Much different photographs clear serious|7.09|3.82|9012003|importounivamalg #3|12|home repair|9|Books|207|ationbarable|N/A|7547seashell20821974|rosy|Ounce|Unknown|55|ablebarbarn st| +9003|AAAAAAAAKCDCAAAA|2000-10-27||Very friendly rumours produce maybe open manufacturers. Tall, other conditions emerge that is in addition alone needs. Sexual, back functions might not suit arrangeme|4.43|3.82|8014004|edu packmaxi #4|14|tennis|8|Sports|496|ationbarable|N/A|6thistle376590082051|rose|Gross|Unknown|84|pribarbarn st| +9004|AAAAAAAAMCDCAAAA|1997-10-27|1999-10-27|Other others must seem increasingly despite a exhibitions. Literary types enable quite by no means criminal pictures. Marks obtain around savings; average, quiet years attack also. Well separate pric|5.99|3.41|7016005|corpnameless #5|16|furniture|7|Home|11|oughtought|N/A|297red57463716411413|tan|Dram|Unknown|22|esebarbarn st| +9005|AAAAAAAAMCDCAAAA|1999-10-28|2001-10-26|Thankfully nuclear eyes take. Methods release future boxes; bright, practical bags cope better royal, parliamentary conclusions. Far large men will alert somewhere open, small funds; so regional o|0.35|3.41|7016005|importoexporti #2|16|infants|3|Children|11|oughtought|large|297red57463716411413|royal|Unknown|Unknown|50|antibarbarn st| +9006|AAAAAAAAMCDCAAAA|2001-10-27||Thankfully nuclear eyes take. Methods release future boxes; bright, practical bags cope better royal, parliamentary conclusions. Far large men will alert somewhere open, small funds; so regional o|1.50|3.41|7014007|edu packnameless #7|14|glassware|7|Home|260|barcallyable|N/A|297red57463716411413|purple|Cup|Unknown|49|callybarbarn st| +9007|AAAAAAAAPCDCAAAA|1997-10-27||Front words must not develop societies. Eventual, grey countries make strangely times; ever old indicators send often tomorrow prime computers. Full, high days will come unique companies. Of course s|4.39|3.95|8012008|importomaxi #8|12|guns|8|Sports|52|ableanti|N/A|92245922504slate5959|lawn|N/A|Unknown|70|ationbarbarn st| +9008|AAAAAAAAADDCAAAA|1997-10-27|2000-10-26|Clearly successful values should agree thus homes. Historical hours measure just. Needs cater thus on the comments. Cattle must no|7.03|4.71|5004001|edu packscholar #1|4|classical|5|Music|149|n steseought|N/A|3549693705980white77|cream|Carton|Unknown|26|eingbarbarn st| +9009|AAAAAAAAADDCAAAA|2000-10-27||Either precious photographs might include economic moves. Rural, direct visitors cannot conc|8.87|7.27|4001002|amalgedu pack #2|1|womens|4|Shoes|275|n steseought|medium|3549693705980white77|honeydew|Gross|Unknown|52|n stbarbarn st| +9010|AAAAAAAACDDCAAAA|1997-10-27|1999-10-27|Really standard employees prove near a fields. Large, uncomfortable authorities|5.64|4.73|10007001|brandunivamalg #1|7|personal|10|Electronics|43|priese|N/A|1923385linen13802783|rosy|Pound|Unknown|43|baroughtbarn st| +9011|AAAAAAAACDDCAAAA|1999-10-28|2001-10-26|Really standard employees prove near a fields. Large, uncomfortable authorities|3.10|4.73|10007001|corpmaxi #12|7|parenting|9|Books|40|priese|N/A|70246peach6244667661|orange|Oz|Unknown|30|oughtoughtbarn st| +9012|AAAAAAAACDDCAAAA|2001-10-27||American, high theories could change only often|77.38|66.54|10007001|exportischolar #1|3|pop|5|Music|136|callypriought|N/A|70246peach6244667661|peach|Gross|Unknown|17|ableoughtbarn st| +9013|AAAAAAAAFDDCAAAA|1997-10-27||Always big grounds deter with a boats. So rough benefits should not happen elsewhere others. Actual surveys kno|1.08|0.35|5002002|importoscholar #2|2|country|5|Music|195|antin stought|N/A|3278royal71969655369|cornflower|Case|Unknown|21|prioughtbarn st| +9014|AAAAAAAAGDDCAAAA|1997-10-27|2000-10-26|Here poor tasks learn short curtains. Single children discuss finally during a persons. Top, young years raise occasionally faintly necessary vehicles. Good feet used to e|1.01|0.60|8005005|scholarnameless #5|5|fitness|8|Sports|611|oughtoughtcally|N/A|90394tomato032716819|lawn|Ounce|Unknown|13|eseoughtbarn st| +9015|AAAAAAAAGDDCAAAA|2000-10-27||Here poor tasks learn short curtains. Single children discuss finally during a persons. Top, young years raise occasionally faintly necessary vehicles. Good feet used to e|4.62|0.60|8005005|exportinameless #6|3|basketball|8|Sports|611|oughtoughtcally|N/A|31706480978rosy75080|pale|Box|Unknown|7|antioughtbarn st| +9016|AAAAAAAAIDDCAAAA|1997-10-27|1999-10-27|New, difficult writings should arrange too never social years. Fresh seasons can stand. Full accountants reserve as the words. Good, public facts see. Inadequate, marin|4.77|3.95|8012007|importomaxi #7|12|guns|8|Sports|72|ableation|N/A|burlywood04901404521|honeydew|Tsp|Unknown|59|callyoughtbarn st| +9017|AAAAAAAAIDDCAAAA|1999-10-28|2001-10-26|New, difficult writings should arrange too never social years. Fresh seasons can stand. Full accountants reserve as the words. Good, public facts see. Inadequate, marin|3.67|1.13|8012008|importomaxi #8|12|guns|8|Sports|217|ationoughtable|N/A|3897205196tan3514310|violet|Box|Unknown|5|ationoughtbarn st| +9018|AAAAAAAAIDDCAAAA|2001-10-27||About large glasses find|1.67|1.13|8012008|exportiexporti #1|12|toddlers|3|Children|192|ationoughtable|extra large|3897205196tan3514310|plum|Bunch|Unknown|8|eingoughtbarn st| +9019|AAAAAAAALDDCAAAA|1997-10-27||Common subjects create pages. Angry leaders consider so new relationships. Common signs halt so. Large, pale accounts|4.29|1.37|1001002|amalgamalg #2|1|dresses|1|Women|222|ableableable|petite|6555ghost06465291774|sienna|Case|Unknown|17|n stoughtbarn st| +9020|AAAAAAAAMDDCAAAA|1997-10-27|2000-10-26|Artists solve never good thanks. Completely medieval homes enhance real goods. Other fears cannot make earlier; packages describe despite a companies; other numbers work similar roles.|3.11|2.58|6010005|univbrand #5|10|jewelry boxes|6|Jewelry|433|pripriese|N/A|05441856998335sandy9|linen|Unknown|Unknown|34|barablebarn st| +9021|AAAAAAAAMDDCAAAA|2000-10-27||Artists solve never good thanks. Completely medieval homes enhance real goods. Other fears cannot make earlier; packages describe despite a companies; other numbers work similar roles.|9.15|4.66|6010005|corpmaxi #10|10|parenting|9|Books|168|eingcallyought|N/A|05441856998335sandy9|yellow|Tbl|Unknown|11|oughtablebarn st| +9022|AAAAAAAAODDCAAAA|1997-10-27|1999-10-27|Times must take well possibly ill|6.68|3.80|7006003|corpbrand #3|6|rugs|7|Home|165|anticallyought|N/A|thistle1000394655229|sky|Oz|Unknown|30|ableablebarn st| +9023|AAAAAAAAODDCAAAA|1999-10-28|2001-10-26|Years receive long. Activities might make thus at a|3.35|3.80|3003002|exportiexporti #2|3|toddlers|3|Children|189|anticallyought|extra large|thistle1000394655229|burnished|Each|Unknown|61|priablebarn st| +9024|AAAAAAAAODDCAAAA|2001-10-27||Presumably complete years set|2.68|2.35|3003002|edu packexporti #1|4|school-uniforms|3|Children|320|barablepri|extra large|thistle1000394655229|spring|Pound|Unknown|19|eseablebarn st| +9025|AAAAAAAABEDCAAAA|1997-10-27||Successfully interesting forests en|8.02|5.05|4003002|exportiedu pack #2|3|kids|4|Shoes|112|ableoughtought|small|521542434wheat865916|red|Ounce|Unknown|6|antiablebarn st| +9026|AAAAAAAACEDCAAAA|1997-10-27|2000-10-26|Offences free somehow. Right, statistical eyes hate also now senior themes. Local, new planes prevent carefully |4.08|2.77|2002001|importoimporto #1|2|shirts|2|Men|808|eingbareing|large|5652614650184violet9|gainsboro|Box|Unknown|63|callyablebarn st| +9027|AAAAAAAACEDCAAAA|2000-10-27||Also likely activities ought to rent here hours. Contemporary, labour days may not cut only. New |3.67|2.77|2002001|univmaxi #8|2|pools|8|Sports|65|eingbareing|N/A|5652614650184violet9|bisque|Tbl|Unknown|64|ationablebarn st| +9028|AAAAAAAAEEDCAAAA|1997-10-27|1999-10-27|Soon clear systems show mainly by a supporters. Particular, corporate others may make much politi|8.23|3.37|2004001|edu packimporto #1|4|sports-apparel|2|Men|150|barantiought|medium|4312417yellow3756658|black|Box|Unknown|15|eingablebarn st| +9029|AAAAAAAAEEDCAAAA|1999-10-28|2001-10-26|Actual bodies canno|26.98|11.87|2004001|maxicorp #6|9|womens watch|6|Jewelry|150|barantiought|N/A|4312417yellow3756658|violet|Each|Unknown|44|n stablebarn st| +9030|AAAAAAAAEEDCAAAA|2001-10-27||French, general reactions can vote. Lives hear already able machines. Rare brothers sho|1.06|0.75|5003001|exportischolar #1|9|pop|5|Music|256|barantiought|N/A|4312417yellow3756658|sienna|Bunch|Unknown|28|barpribarn st| +9031|AAAAAAAAHEDCAAAA|1997-10-27||Victims comply routine images. Patient, helpful weeks locate things. Well lovely creatures can run far terms. Regional hands shall not discuss trees; shy s|7.07|6.08|10002005|importounivamalg #5|2|camcorders|10|Electronics|485|antieingese|N/A|150587402510slate582|puff|Bunch|Unknown|7|oughtpribarn st| +9032|AAAAAAAAIEDCAAAA|1997-10-27|2000-10-26|Recent organisa|4.24|2.58|5001001|amalgscholar #1|1|rock|5|Music|479|n stationese|N/A|977937751powder98182|frosted|Bundle|Unknown|21|ablepribarn st| +9033|AAAAAAAAIEDCAAAA|2000-10-27||Recent organisa|9.09|6.72|5001001|amalgimporto #2|1|accessories|2|Men|213|prioughtable|petite|065675706sandy457050|mint|Box|Unknown|70|pripribarn st| +9034|AAAAAAAAKEDCAAAA|1997-10-27|1999-10-27|Very essential years assist significantly other, major considerations. Now recent products take. Good factors could no|6.78|4.94|10003008|exportiunivamalg #8|3|dvd/vcr players|10|Electronics|436|callypriese|N/A|420181463085030plum1|maroon|Dram|Unknown|4|esepribarn st| +9035|AAAAAAAAKEDCAAAA|1999-10-28|2001-10-26|Quite effective contracts would not draw pure, local students. Buses would assess mutual costs. Doubtful subjects may interfere nearly. As french reasons m|1.43|0.80|4003002|exportiedu pack #2|3|kids|4|Shoes|436|callypriese|N/A|420181463085030plum1|spring|N/A|Unknown|37|antipribarn st| +9036|AAAAAAAAKEDCAAAA|2001-10-27||Quite effective contracts would not draw pure, local students. Buses would assess mutual costs. Doubtful subjects may interfere nearly. As french reasons m|2.29|0.80|8004007|edu packnameless #7|4|camping|8|Sports|436|callypriese|N/A|420181463085030plum1|lime|Carton|Unknown|44|callypribarn st| +9037|AAAAAAAANEDCAAAA|1997-10-27||Human, human children used to go base situations. Innocent sources might conform over dark, arb|1.25|0.83|2002002|importoimporto #2|2|shirts|2|Men|479|n stationese|medium|388188tan88212792696|lavender|Oz|Unknown|36|ationpribarn st| +9038|AAAAAAAAOEDCAAAA|1997-10-27|2000-10-26|Successes might correspond just certain reactions. Figures may offer unexpected subjects. Scientists construct entire rules|3.14|2.16|8005009|scholarnameless #9|5|fitness|8|Sports|323|priablepri|N/A|755310386gainsboro74|rosy|Tbl|Unknown|8|eingpribarn st| +9039|AAAAAAAAOEDCAAAA|2000-10-27||Successes might correspond just certain reactions. Figures may offer unexpected subjects. Scientists construct entire rules|4.02|2.16|8005009|importoimporto #2|5|shirts|2|Men|323|priablepri|petite|755310386gainsboro74|wheat|Each|Unknown|13|n stpribarn st| +9040|AAAAAAAAAFDCAAAA|1997-10-27|1999-10-27|Eventual, adverse assumptions should help no longer from a databases. Globa|5.68|3.01|6008001|namelesscorp #1|8|mens watch|6|Jewelry|135|antipriought|N/A|42267sandy3086903166|thistle|Bundle|Unknown|4|baresebarn st| +9041|AAAAAAAAAFDCAAAA|1999-10-28|2001-10-26|Minutes should stir. Respectively other statements could decide in a leaders. Only gree|1.01|3.01|4001002|amalgedu pack #2|1|womens|4|Shoes|597|ationn stanti|N/A|7008885papaya1750056|royal|Pound|Unknown|49|oughtesebarn st| +9042|AAAAAAAAAFDCAAAA|2001-10-27||Minutes should stir. Respectively other statements could decide in a leaders. Only gree|7.23|3.01|4001002|importoscholar #1|1|country|5|Music|5|ationn stanti|N/A|7008885papaya1750056|peru|N/A|Unknown|4|ableesebarn st| +9043|AAAAAAAADFDCAAAA|1997-10-27||Neither national accounts can claim. Legal, capable features earn international, old versions. So old friends used to issue now political colleagues; police would |5.36|3.75|5002002|importoscholar #2|2|country|5|Music|288|eingeingable|N/A|peach257824537461337|light|Cup|Unknown|33|priesebarn st| +9044|AAAAAAAAEFDCAAAA|1997-10-27|2000-10-26|Deliberately good years appear for a mechanisms. Only preliminary controls might not discuss only late products; devices will replace still very possi|18.97|9.29|6005005|scholarcorp #5|5|earings|6|Jewelry|384|eseeingpri|N/A|4734672774233snow643|magenta|Case|Unknown|20|eseesebarn st| +9045|AAAAAAAAEFDCAAAA|2000-10-27||Deliberately good years appear for a mechanisms. Only preliminary controls might not discuss only late products; devices will replace still very possi|7.53|9.29|8014006|edu packmaxi #6|14|tennis|8|Sports|814|eseeingpri|N/A|4734672774233snow643|purple|Box|Unknown|17|antiesebarn st| +9046|AAAAAAAAGFDCAAAA|1997-10-27|1999-10-27|Particularly medieval blocks would not find slightly with a carers. Years respond about at a sec|6.00|3.30|9002011|importomaxi #11|2|business|9|Books|420|barableese|N/A|721005818424hot30968|white|Ton|Unknown|29|callyesebarn st| +9047|AAAAAAAAGFDCAAAA|1999-10-28|2001-10-26|Particularly medieval blocks would not find slightly with a carers. Years respond about at a sec|7.30|3.79|9002011|scholarmaxi #8|15|fishing|8|Sports|281|oughteingable|N/A|79016024moccasin5843|peru|Cup|Unknown|69|ationesebarn st| +9048|AAAAAAAAGFDCAAAA|2001-10-27||Actually normal proposals clear then by a appearances. Equal, royal weeks will expect|3.56|2.42|5001001|amalgscholar #1|15|rock|5|Music|76|callyation|N/A|79016024moccasin5843|moccasin|Gram|Unknown|60|eingesebarn st| +9049|AAAAAAAAJFDCAAAA|1997-10-27||Great authorities can hear thus sheets. R|2.74|1.83|9008002|namelessmaxi #2|8|romance|9|Books|277|ationationable|N/A|0448320plum244210316|grey|Pound|Unknown|15|n stesebarn st| +9050|AAAAAAAAKFDCAAAA|1997-10-27|2000-10-26|Jobs must not train only contents. Much soft experiences might smell fully. Suspicious houses remain prime mem|0.12|0.09|5002001|importoscholar #1|2|country|5|Music|29|n stable|N/A|6blue223667973236658|royal|Bundle|Unknown|15|barantibarn st| +9051|AAAAAAAAKFDCAAAA|2000-10-27||Jobs must not train only contents. Much soft experiences might smell fully. Suspicious houses remain prime mem|2.79|0.09|5002001|exportischolar #2|2|pop|5|Music|1|ought|N/A|6blue223667973236658|thistle|Lb|Unknown|20|oughtantibarn st| +9052|AAAAAAAAMFDCAAAA|1997-10-27|1999-10-27|Central, additional mile|8.30|6.80|10012009|importoamalgamalg #9|12|monitors|10|Electronics|636|callyprically|N/A|89999380sandy3876766|sky|Gross|Unknown|57|ableantibarn st| +9053|AAAAAAAAMFDCAAAA|1999-10-28|2001-10-26|Women should stay in a criteria; still asleep activities talk societies. Sorry, other hours should introduce. Solar, psychological members take now else new services; ec|19.58|12.13|9006006|corpmaxi #6|6|parenting|9|Books|36|callypri|N/A|89999380sandy3876766|snow|Lb|Unknown|19|priantibarn st| +9054|AAAAAAAAMFDCAAAA|2001-10-27||Women should stay in a criteria; still asleep activities talk societies. Sorry, other hours should introduce. Solar, psychological members take now else new services; ec|3.46|2.80|9006006|corpunivamalg #7|6|mystery|9|Books|36|callypri|N/A|violet53567129318531|pink|Carton|Unknown|42|eseantibarn st| +9055|AAAAAAAAPFDCAAAA|1997-10-27||At all contemporary problems leave arrangements. Children tell again local references. Students think around a nights. Arguments shall incr|4.23|3.17|10012012|importoamalgamalg #12|12|monitors|10|Electronics|204|esebarable|N/A|30771641ivory1127199|steel|Ounce|Unknown|95|antiantibarn st| +9056|AAAAAAAAAGDCAAAA|1997-10-27|2000-10-26|As slight relationships give such as a alternatives. Properly british minutes impose modern, dependent subsidies. Only w|9.25|6.84|5003001|exportischolar #1|3|pop|5|Music|179|n stationought|N/A|pale0479370611384730|azure|Pound|Unknown|19|callyantibarn st| +9057|AAAAAAAAAGDCAAAA|2000-10-27||Considerable, long years need tiny studies. Urgent, complete parties used to wei|7.86|5.34|3003002|exportiexporti #2|3|toddlers|3|Children|179|n stationought|medium|008416086875slate419|maroon|Ounce|Unknown|6|ationantibarn st| +9058|AAAAAAAACGDCAAAA|1997-10-27|1999-10-27|Special members ought to invest ago small, real members. Social, like rules used to write under quiet, full drugs. Feet drink. Powerful, cons|4.93|1.82|3002001|importoexporti #1|2|infants|3|Children|130|barpriought|economy|0slate73740781660334|orange|Gross|Unknown|62|eingantibarn st| +9059|AAAAAAAACGDCAAAA|1999-10-28|2001-10-26|Capable studies become now; american, old spirits wait so major representations; horses drive easier long conditions. Prizes ease police; usually short-term experts build else frag|3.02|2.05|3002001|exportiexporti #2|2|toddlers|3|Children|94|esen st|petite|12171104301408puff13|sandy|Box|Unknown|36|n stantibarn st| +9060|AAAAAAAACGDCAAAA|2001-10-27||Capable studies become now; american, old spirits wait so major representations; horses drive easier long conditions. Prizes ease police; usually short-term experts build else frag|44.63|2.05|3002001|exportinameless #1|3|basketball|8|Sports|94|esen st|N/A|12171104301408puff13|mint|Oz|Unknown|57|barcallybarn st| +9061|AAAAAAAAFGDCAAAA|1997-10-27||Variations shall want now wild |0.92|0.80|10004001|edu packunivamalg #1|4|audio|10|Electronics|55|antianti|N/A|0140621128orange7752|khaki|Pound|Unknown|82|oughtcallybarn st| +9062|AAAAAAAAGGDCAAAA|1997-10-27|2000-10-26|Long, fat problems think with the boys. Readers may take elections. Different brothers know especially due, upper players. Early, australian angles used to set then detail|3.93|2.55|8006003|corpnameless #3|6|football|8|Sports|44|eseese|N/A|7839217525289navy369|sandy|Lb|Unknown|31|ablecallybarn st| +9063|AAAAAAAAGGDCAAAA|2000-10-27||Aware, married parents allow so. Result|6.31|2.55|8006004|corpnameless #4|6|football|8|Sports|668|eseese|N/A|3goldenrod9228660164|snow|Gram|Unknown|17|pricallybarn st| +9064|AAAAAAAAIGDCAAAA|1997-10-27|1999-10-27|Christian, traditional problems reply conscious, important men. Spanish, large types open also similar men. Loose, |0.53|0.45|6003001|exporticorp #1|3|gold|6|Jewelry|508|eingbaranti|N/A|82225655pale32022892|blanched|Box|Unknown|18|esecallybarn st| +9065|AAAAAAAAIGDCAAAA|1999-10-28|2001-10-26|Christian, traditional problems reply conscious, important men. Spanish, large types open also similar men. Loose, |3.00|2.52|8001006|amalgnameless #6|3|athletic shoes|8|Sports|508|eingbaranti|N/A|118slate852777569709|red|Ounce|Unknown|18|anticallybarn st| +9066|AAAAAAAAIGDCAAAA|2001-10-27||Christian, traditional problems reply conscious, important men. Spanish, large types open also similar men. Loose, |7.07|5.23|8001006|exportiimporto #1|3|pants|2|Men|508|eingbaranti|medium|76068967376red117711|pale|Cup|Unknown|30|callycallybarn st| +9067|AAAAAAAALGDCAAAA|1997-10-27||Rather english applications get a little. J|7.88|4.64|4003002|exportiedu pack #2|3|kids|4|Shoes|557|ationantianti|medium|royal318971746885346|snow|Box|Unknown|38|ationcallybarn st| +9068|AAAAAAAAMGDCAAAA|1997-10-27|2000-10-26|Bright individuals would use possibly normal tables. Aware beaches depend to a companies; different, mi|5.55|1.72|4004001|edu packedu pack #1|4|athletic|4|Shoes|414|eseoughtese|large|4899577chocolate4506|peru|Bunch|Unknown|64|eingcallybarn st| +9069|AAAAAAAAMGDCAAAA|2000-10-27||Bright individuals would use possibly normal tables. Aware beaches depend to a companies; different, mi|57.09|1.72|4004001|namelessbrand #10|8|lighting|7|Home|414|eseoughtese|N/A|4899577chocolate4506|sienna|Oz|Unknown|60|n stcallybarn st| +9070|AAAAAAAAOGDCAAAA|1997-10-27|1999-10-27|Even psychiatric members could not make previously real important students. Always new banks cannot drag already families. However political years pick too patient wages. More tha|7.84|2.90|4002001|importoedu pack #1|2|mens|4|Shoes|635|antiprically|extra large|501802595steel410443|maroon|Cup|Unknown|38|barationbarn st| +9071|AAAAAAAAOGDCAAAA|1999-10-28|2001-10-26|Even psychiatric members could not make previously real important students. Always new banks cannot drag already families. However political years pick too patient wages. More tha|52.48|2.90|3002002|importoexporti #2|2|infants|3|Children|324|antiprically|small|501802595steel410443|moccasin|Tsp|Unknown|8|oughtationbarn st| +9072|AAAAAAAAOGDCAAAA|2001-10-27||Even psychiatric members could not make previously real important students. Always new banks cannot drag already families. However political years pick too patient wages. More tha|77.52|2.90|3002002|importoexporti #1|2|infants|3|Children|324|eseablepri|small|1104888256steel13270|orchid|Dram|Unknown|52|ableationbarn st| +9073|AAAAAAAABHDCAAAA|1997-10-27||Minor heads close common children; recently strong firms provide. Useful, young men ought to create changes. Popular, common regulations might decide. Points fit. Obvious, glad officials |3.88|2.32|9001002|amalgmaxi #2|1|arts|9|Books|144|eseeseought|N/A|17128580677248puff83|rosy|Bunch|Unknown|10|priationbarn st| +9074|AAAAAAAACHDCAAAA|1997-10-27|2000-10-26|Prim|92.01|52.44|4003001|exportiedu pack #1|3|kids|4|Shoes|258|eingantiable|large|9923270020041ivory77|sandy|Pound|Unknown|27|eseationbarn st| +9075|AAAAAAAACHDCAAAA|2000-10-27||Late patients can continue once into the organisations. Somewhat long trees ought to|4.23|3.51|5002002|importoscholar #2|2|country|5|Music|258|eingantiable|N/A|366863misty432770589|wheat|Gram|Unknown|42|antiationbarn st| +9076|AAAAAAAAEHDCAAAA|1997-10-27|1999-10-27|Socialist, inc departments get on an options. Public, normal surroundings market good presents. Tired, great committees could not discourage therefo|7.05|2.32|1003001|exportiamalg #1|3|maternity|1|Women|250|barantiable|medium|639snow2652851677198|rose|Bunch|Unknown|37|callyationbarn st| +9077|AAAAAAAAEHDCAAAA|1999-10-28|2001-10-26|Events form employees. Powerful, white services inspire british examples. Chemical tiles may not persuade low fixed matters. Feelings could suggest to |4.81|2.32|1003002|exportiamalg #2|3|maternity|1|Women|250|barantiable|extra large|wheat527094148359902|lavender|Case|Unknown|50|ationationbarn st| +9078|AAAAAAAAEHDCAAAA|2001-10-27||Events form employees. Powerful, white services inspire british examples. Chemical tiles may not persuade low fixed matters. Feelings could suggest to |59.46|36.86|1003002|maxiunivamalg #1|3|televisions|10|Electronics|250|barantiable|N/A|087669720snow9056744|turquoise|Ton|Unknown|38|eingationbarn st| +9079|AAAAAAAAHHDCAAAA|1997-10-27||Pupils may not land ago on to a millions; popular plants stop otherwise foreign, official lights; material, warm factors may put |0.39|0.32|10006002|corpunivamalg #2|6|musical|10|Electronics|431|oughtpriese|N/A|1014510767thistle070|salmon|Unknown|Unknown|16|n stationbarn st| +9080|AAAAAAAAIHDCAAAA|1997-10-27|2000-10-26|Adverse, early members build only small numbers. Head feet must sink profitable books. Friends kick thus in a jobs. Little, complicated departments |0.58|0.51|7001001|amalgbrand #1|1|bathroom|7|Home|50|baranti|N/A|4856rose003917392464|wheat|Dram|Unknown|18|bareingbarn st| +9081|AAAAAAAAIHDCAAAA|2000-10-27||Hard, social brothers ought to serve french, serious orders. Instruments learn tight feelings. Then right hospitals will not move across the signals. Well wealthy materials must press bright,|9.84|8.85|2001002|amalgimporto #2|1|accessories|2|Men|50|baranti|medium|4856rose003917392464|salmon|Oz|Unknown|8|oughteingbarn st| +9082|AAAAAAAAKHDCAAAA|1997-10-27|1999-10-27|Residents explain there current politicians. Domestic, o|4.63|1.89|2001001|amalgimporto #1|1|accessories|2|Men|41|oughtese|large|5854light03684247579|cornflower|Oz|Unknown|59|ableeingbarn st| +9083|AAAAAAAAKHDCAAAA|1999-10-28|2001-10-26|Residents explain there current politicians. Domestic, o|2.46|1.54|8016004|corpmaxi #4|16|golf|8|Sports|41|oughtese|N/A|32340tomato330759827|peru|Ton|Unknown|13|prieingbarn st| +9084|AAAAAAAAKHDCAAAA|2001-10-27||Levels|0.94|1.54|9014001|edu packunivamalg #1|16|sports|9|Books|41|oughtese|N/A|7406758violet5555302|smoke|Ounce|Unknown|9|eseeingbarn st| +9085|AAAAAAAANHDCAAAA|1997-10-27||Serious forces ought to feel worse |6.84|4.10|10009008|maxiunivamalg #8|9|televisions|10|Electronics|566|callycallyanti|N/A|28478439powder092057|steel|Bunch|Unknown|15|antieingbarn st| +9086|AAAAAAAAOHDCAAAA|1997-10-27|2000-10-26|Proposals like quickly certain schemes. Matters know now christians. Metropolitan modules advise problems. New, daily periods can wo|0.99|0.51|10016009|corpamalgamalg #9|16|wireless|10|Electronics|636|callyprically|N/A|640lawn8074240511022|linen|Bunch|Unknown|88|callyeingbarn st| +9087|AAAAAAAAOHDCAAAA|2000-10-27||Proposals like quickly certain schemes. Matters know now christians. Metropolitan modules advise problems. New, daily periods can wo|5.28|3.69|4001002|amalgedu pack #2|16|womens|4|Shoes|307|callyprically|small|6116657042snow870134|thistle|Carton|Unknown|58|ationeingbarn st| +9088|AAAAAAAAAIDCAAAA|1997-10-27|1999-10-27|Public, free |0.65|0.57|6002001|importocorp #1|2|diamonds|6|Jewelry|390|barn stpri|N/A|312469slate136904875|chartreuse|Case|Unknown|28|eingeingbarn st| +9089|AAAAAAAAAIDCAAAA|1999-10-28|2001-10-26|Public, free |4.37|3.71|6002001|importoscholar #2|2|country|5|Music|264|esecallyable|N/A|312469slate136904875|peach|Cup|Unknown|38|n steingbarn st| +9090|AAAAAAAAAIDCAAAA|2001-10-27||Only prominent thoughts would not close. Certainly future thoughts see as british, new women. Apparently likely voters represent as in a others. New, effecti|3.53|2.89|9006007|corpmaxi #7|6|parenting|9|Books|892|esecallyable|N/A|5321933163904tomato5|grey|Dram|Unknown|11|barn stbarn st| +9091|AAAAAAAADIDCAAAA|1997-10-27||Days occur legitimate, premier restrictions. Certain eyes might decrease full forests; times oug|1.31|0.52|10007014|brandunivamalg #14|7|personal|10|Electronics|586|callyeinganti|N/A|1775330786wheat03156|pale|Dozen|Unknown|30|oughtn stbarn st| +9092|AAAAAAAAEIDCAAAA|1997-10-27|2000-10-26|Substantial hands refer hours. White factors must take other, enthusiastic men. Political details stick pounds. Now powerful services must not settle sufficient markets; |27.12|18.98|7005003|scholarbrand #3|5|blinds/shades|7|Home|508|eingbaranti|N/A|986931700steel619865|powder|Lb|Unknown|25|ablen stbarn st| +9093|AAAAAAAAEIDCAAAA|2000-10-27||Organic rates follow international, actual ambitions. Subsequent options must bring at least environmental costs. Enthusias|6.76|18.98|10005016|scholarunivamalg #16|5|karoke|10|Electronics|508|eingbaranti|N/A|986931700steel619865|peru|Oz|Unknown|29|prin stbarn st| +9094|AAAAAAAAGIDCAAAA|1997-10-27|1999-10-27|Others manufacture even into the children. Seldom small years get reluctantly. Involved words cannot begin words. Close eyes ought|1.90|0.60|3004001|edu packexporti #1|4|school-uniforms|3|Children|332|ablepripri|N/A|93704252papaya086255|salmon|Tbl|Unknown|5|esen stbarn st| +9095|AAAAAAAAGIDCAAAA|1999-10-28|2001-10-26|Others manufacture even into the children. Seldom small years get reluctantly. Involved words cannot begin words. Close eyes ought|21.23|11.03|4001002|amalgedu pack #2|1|womens|4|Shoes|332|ablepripri|extra large|9613382yellow1695242|hot|Ounce|Unknown|13|antin stbarn st| +9096|AAAAAAAAGIDCAAAA|2001-10-27||Others manufacture even into the children. Seldom small years get reluctantly. Involved words cannot begin words. Close eyes ought|3.30|1.02|4001002|edu packmaxi #7|14|tennis|8|Sports|129|n stableought|N/A|9613382yellow1695242|lawn|Ton|Unknown|19|callyn stbarn st| +9097|AAAAAAAAJIDCAAAA|1997-10-27||Alive, abstract contacts will not know easy, classical paintings. Above styles control frequently directors; powerful powers allow only c|2.84|2.15|4002002|importoedu pack #2|2|mens|4|Shoes|633|priprically|large|190sandy833473950687|purple|Gram|Unknown|6|ationn stbarn st| +9098|AAAAAAAAKIDCAAAA|1997-10-27|2000-10-26|Well different centuries mean also foreign, large years; agents can draw almost in respect of a qualities. Left produc|2.46|1.37|7009009|maxibrand #9|9|mattresses|7|Home|70|baration|N/A|9971058872turquoise8|rose|Dozen|Unknown|15|eingn stbarn st| +9099|AAAAAAAAKIDCAAAA|2000-10-27||Well different centuries mean also foreign, large years; agents can draw almost in respect of a qualities. Left produc|6.15|2.58|7001010|amalgbrand #10|1|bathroom|7|Home|70|baration|N/A|63463294212cream9246|pale|Lb|Unknown|41|n stn stbarn st| +9100|AAAAAAAAMIDCAAAA|1997-10-27|1999-10-27|Times used to occur even again open leaves. Then external dogs tell for example figures. Dark, relative skills should not think for a premises; v|16.84|13.80|3002001|importoexporti #1|2|infants|3|Children|321|oughtablepri|extra large|27350wheat7024930140|bisque|Box|Unknown|93|barbaroughtn st| +9101|AAAAAAAAMIDCAAAA|1999-10-28|2001-10-26|Also little arms result feelings; minutes order. Feet explore. Trying, important contexts can observe however out of a men. Organisational, other products must administer as old, m|1.20|0.38|3002001|importoimporto #2|2|shirts|2|Men|321|oughtablepri|petite|27350wheat7024930140|smoke|Oz|Unknown|63|oughtbaroughtn st| +9102|AAAAAAAAMIDCAAAA|2001-10-27||Largely certain factors submit basically local values. Men can contain here large, gothic groups. However immediate states cannot believe too. Powers may not see on the responses; orders shall not gr|10.17|4.06|3002001|amalgscholar #1|1|rock|5|Music|321|oughtablepri|N/A|27350wheat7024930140|burlywood|Ton|Unknown|23|ablebaroughtn st| +9103|AAAAAAAAPIDCAAAA|1997-10-27||Existing, new concentrations could discriminate. Instead dark d|4.05|2.47|3001002|amalgexporti #2|1|newborn|3|Children|544|eseeseanti|N/A|46002095spring060268|salmon|Ounce|Unknown|52|pribaroughtn st| +9104|AAAAAAAAAJDCAAAA|1997-10-27|2000-10-26|Priorities discuss from the colleagues. Combinations could not weave possibly. Genuine, common police wear yesterday other, limited|0.25|0.11|10015010|scholaramalgamalg #10|15|portable|10|Electronics|884|eseeingeing|N/A|0462168617sandy97054|puff|Cup|Unknown|15|esebaroughtn st| +9105|AAAAAAAAAJDCAAAA|2000-10-27||Stupid, unknown meetings may rise co|18.97|7.01|10014011|edu packamalgamalg #11|14|automotive|10|Electronics|884|eseeingeing|N/A|466717958turquoise02|powder|Ounce|Unknown|1|antibaroughtn st| +9106|AAAAAAAACJDCAAAA|1997-10-27|1999-10-27|Regulatory managers may use at a indians. Poems can begin new, back conditions. Soon proper committees used to prosecute highly there old eyes. Nearly new seats would not address from no days. Importa|1.84|1.01|9014011|edu packunivamalg #11|14|sports|9|Books|54|eseanti|N/A|770222803674white338|aquamarine|Pallet|Unknown|29|callybaroughtn st| +9107|AAAAAAAACJDCAAAA|1999-10-28|2001-10-26|Regulatory managers may use at a indians. Poems can begin new, back conditions. Soon proper committees used to prosecute highly there old eyes. Nearly new seats would not address from no days. Importa|1.40|0.88|9014011|amalgexporti #2|14|newborn|3|Children|54|eseanti|medium|770222803674white338|steel|Bundle|Unknown|69|ationbaroughtn st| +9108|AAAAAAAACJDCAAAA|2001-10-27||Regulatory managers may use at a indians. Poems can begin new, back conditions. Soon proper committees used to prosecute highly there old eyes. Nearly new seats would not address from no days. Importa|3.94|0.88|9014011|importoexporti #1|14|infants|3|Children|47|eseanti|medium|770222803674white338|magenta|Each|Unknown|60|eingbaroughtn st| +9109|AAAAAAAAFJDCAAAA|1997-10-27||Happy, sufficient pupils seem competitive, occupational classes. Other, white others should compare international properties. Difficult, numerous players would not increase clear. Doors may make u|3.80|3.11|10005015|scholarunivamalg #15|5|karoke|10|Electronics|108|eingbarought|N/A|234855502gainsboro19|light|Unknown|Unknown|68|n stbaroughtn st| +9110|AAAAAAAAGJDCAAAA|1997-10-27|2000-10-26|True banks say thus to the n|1.28|0.66|2004001|edu packimporto #1|4|sports-apparel|2|Men|116|callyoughtought|extra large|6832661936714sky8906|drab|Gross|Unknown|26|baroughtoughtn st| +9111|AAAAAAAAGJDCAAAA|2000-10-27||True banks say thus to the n|0.46|0.35|2004001|corpbrand #8|4|consignment|6|Jewelry|66|callyoughtought|N/A|6832661936714sky8906|papaya|Tsp|Unknown|78|oughtoughtoughtn st| +9112|AAAAAAAAIJDCAAAA|1997-10-27|1999-10-27|However legal surroundings ought to reflect probably foreign new materials. Little genuine waves ought to need even product|1.08|0.42|5002001|importoscholar #1|2|country|5|Music|442|ableeseese|N/A|572331730693thistle8|salmon|Bunch|Unknown|8|ableoughtoughtn st| +9113|AAAAAAAAIJDCAAAA|1999-10-28|2001-10-26|However legal surroundings ought to reflect probably foreign new materials. Little genuine waves ought to need even product|3.75|1.31|5002001|exportiimporto #2|2|pants|2|Men|148|ableeseese|petite|572331730693thistle8|red|Pallet|Unknown|34|prioughtoughtn st| +9114|AAAAAAAAIJDCAAAA|2001-10-27||Perhaps easy benefits live still. Colleagues get no doubt central leaves. Reactions shall predict very |50.39|17.13|5002001|exportischolar #1|3|pop|5|Music|148|ableeseese|N/A|572331730693thistle8|turquoise|Bundle|Unknown|17|eseoughtoughtn st| +9115|AAAAAAAALJDCAAAA|1997-10-27||Memories suggest with a achievements; now aware events can let at a streets; ancient, historic examinations would|1.00|0.87|4001002|amalgedu pack #2|1|womens|4|Shoes|669|n stcallycally|large|503639papaya12281834|maroon|Dram|Unknown|7|antioughtoughtn st| +9116|AAAAAAAAMJDCAAAA|1997-10-27|2000-10-26|Average features detect instead in a consequences; single organisations |3.98|3.02|9012009|importounivamalg #9|12|home repair|9|Books|156|callyantiought|N/A|rosy7865393063605633|saddle|Lb|Unknown|34|callyoughtoughtn st| +9117|AAAAAAAAMJDCAAAA|2000-10-27||Increasingly close molecules used to see already good laws; small, special changes used to cross young, inner regions. Long-term language|8.00|5.68|9012009|exportibrand #6|13|loose stones|6|Jewelry|468|eingcallyese|N/A|rosy7865393063605633|white|Each|Unknown|17|ationoughtoughtn st| +9118|AAAAAAAAOJDCAAAA|1997-10-27|1999-10-27|Designs combat ultimate years. Commonly official terms decide about meetings. Ideas w|6.64|2.05|1004001|edu packamalg #1|4|swimwear|1|Women|313|prioughtpri|extra large|492493120448magenta6|hot|Tbl|Unknown|30|eingoughtoughtn st| +9119|AAAAAAAAOJDCAAAA|1999-10-28|2001-10-26|Other, likely parties would form tightly critical colleges. Problems shall enha|0.94|2.05|8015010|scholarmaxi #10|15|fishing|8|Sports|313|prioughtpri|N/A|7896navajo5906571139|grey|Pound|Unknown|3|n stoughtoughtn st| +9120|AAAAAAAAOJDCAAAA|2001-10-27||Other, likely parties would form tightly critical colleges. Problems shall enha|3.77|2.05|8015010|exportibrand #1|3|kids|7|Home|239|n stpriable|N/A|4475856328806royal81|tomato|Dram|Unknown|71|barableoughtn st| +9121|AAAAAAAABKDCAAAA|1997-10-27||Wide, certain v|5.44|2.23|8012002|importomaxi #2|12|guns|8|Sports|296|callyn stable|N/A|944819447560smoke804|smoke|Case|Unknown|45|oughtableoughtn st| +9122|AAAAAAAACKDCAAAA|1997-10-27|2000-10-26|Perfect injuries reconcile on a conditions; negotiations uncover etc directly sure men. Leaders shed. Likely services would like from a women. Forward times wait totally st|3.69|1.10|3002001|importoexporti #1|2|infants|3|Children|440|bareseese|medium|305seashell138359420|steel|Pallet|Unknown|42|ableableoughtn st| +9123|AAAAAAAACKDCAAAA|2000-10-27||Presumably canad|8.94|1.10|10016012|corpamalgamalg #12|2|wireless|10|Electronics|199|n stn stought|N/A|305seashell138359420|maroon|Ounce|Unknown|80|priableoughtn st| +9124|AAAAAAAAEKDCAAAA|1997-10-27|1999-10-27|Traditional, spatial personnel must not return very little, free wings. Actual, black letters lead secretly european materi|0.78|0.31|6012005|importobrand #5|12|costume|6|Jewelry|566|callycallyanti|N/A|7lavender51622391783|green|Bundle|Unknown|88|eseableoughtn st| +9125|AAAAAAAAEKDCAAAA|1999-10-28|2001-10-26|Here permanent rights alert for a trees. Federal, dark women select always good pp.. Teachers will not return more by a quantities. Young, mental delegates should come active, different c|0.85|0.72|9002006|importomaxi #6|2|business|9|Books|340|baresepri|N/A|7lavender51622391783|maroon|Ton|Unknown|25|antiableoughtn st| +9126|AAAAAAAAEKDCAAAA|2001-10-27||Here permanent rights alert for a trees. Federal, dark women select always good pp.. Teachers will not return more by a quantities. Young, mental delegates should come active, different c|1.93|0.82|9002006|amalgimporto #1|1|accessories|2|Men|263|baresepri|N/A|7lavender51622391783|moccasin|Cup|Unknown|78|callyableoughtn st| +9127|AAAAAAAAHKDCAAAA|1997-10-27||Ahead old properties would give poor duties. Good, professional families sign better with a jews. Great|58.18|26.18|6002008|importocorp #8|2|diamonds|6|Jewelry|318|eingoughtpri|N/A|158253022lemon594447|saddle|N/A|Unknown|33|ationableoughtn st| +9128|AAAAAAAAIKDCAAAA|1997-10-27|2000-10-26|Utterly possible decisions require assumptions. Obviously secret skills might disturb special, satisfactory levels. Categories oug|11.10|7.77|4002001|importoedu pack #1|2|mens|4|Shoes|322|ableablepri|large|95metallic3872299491|frosted|Gross|Unknown|47|eingableoughtn st| +9129|AAAAAAAAIKDCAAAA|2000-10-27||Utterly possible decisions require assumptions. Obviously secret skills might disturb special, satisfactory levels. Categories oug|2.50|7.77|1002002|importoamalg #2|2|fragrances|1|Women|322|ableablepri|extra large|55723488161pink41222|navy|Pallet|Unknown|4|n stableoughtn st| +9130|AAAAAAAAKKDCAAAA|1997-10-27|1999-10-27|Likely proposals ought to stop away rare generations. Points should include today disciplinary places. Small, positive policies square underlying poles. However united plans could not|5.08|2.59|4001001|amalgedu pack #1|1|womens|4|Shoes|257|ationantiable|small|42289552586coral6930|bisque|Pallet|Unknown|34|barprioughtn st| +9131|AAAAAAAAKKDCAAAA|1999-10-28|2001-10-26|Enormous, english children remember recent, old payments. Records would go p|6.49|5.19|4001001|amalgamalg #2|1|dresses|1|Women|476|ationantiable|petite|335717802450puff5026|red|Ounce|Unknown|20|oughtprioughtn st| +9132|AAAAAAAAKKDCAAAA|2001-10-27||Enormous, english children remember recent, old payments. Records would go p|3.82|5.19|4001001|exportiamalg #1|1|maternity|1|Women|476|ationantiable|small|67435902627olive9297|salmon|Gram|Unknown|60|ableprioughtn st| +9133|AAAAAAAANKDCAAAA|1997-10-27||Half original organisations allocate quite earnings. Either political parts might prove quite; statutory factors walk both new influence|3.89|2.87|10001005|amalgunivamalg #5|1|cameras|10|Electronics|261|oughtcallyable|N/A|95945steel3531204957|snow|Case|Unknown|61|priprioughtn st| +9134|AAAAAAAAOKDCAAAA|1997-10-27|2000-10-26|Relevant, frequent drugs enable international foods. Complete waves may save always long o|50.24|34.16|5001001|amalgscholar #1|1|rock|5|Music|181|oughteingought|N/A|08532308103rose83853|brown|Box|Unknown|10|eseprioughtn st| +9135|AAAAAAAAOKDCAAAA|2000-10-27||Oth|4.36|34.16|5001001|namelessnameless #6|1|outdoor|8|Sports|181|oughteingought|N/A|12tomato635405471118|powder|Bundle|Unknown|62|antiprioughtn st| +9136|AAAAAAAAALDCAAAA|1997-10-27|1999-10-27|Fees should not fix initiall|2.99|1.70|8011007|amalgmaxi #7|11|archery|8|Sports|291|oughtn stable|N/A|7992185smoke93136443|rose|Tbl|Unknown|7|callyprioughtn st| +9137|AAAAAAAAALDCAAAA|1999-10-28|2001-10-26|Fees should not fix initiall|3.89|1.86|1002002|importoamalg #2|2|fragrances|1|Women|464|esecallyese|medium|416629565slate797857|puff|Cup|Unknown|10|ationprioughtn st| +9138|AAAAAAAAALDCAAAA|2001-10-27||Fees should not fix initiall|7.04|1.86|1002002|exportiamalg #1|3|maternity|1|Women|464|esecallyese|medium|416629565slate797857|ghost|Gross|Unknown|43|eingprioughtn st| +9139|AAAAAAAADLDCAAAA|1997-10-27||Political, lucky standards learn appeals. Eventual, influential services involve numerous, coming scientists. Eyes play less|9.95|6.56|9006008|corpmaxi #8|6|parenting|9|Books|239|n stpriable|N/A|7810ghost69408824996|royal|Bunch|Unknown|59|n stprioughtn st| +9140|AAAAAAAAELDCAAAA|1997-10-27|2000-10-26|Lucky, new buses place aged a packages; new forces|2.33|1.56|7009001|maxibrand #1|9|mattresses|7|Home|242|ableeseable|N/A|429108875ghost483885|red|Ounce|Unknown|69|bareseoughtn st| +9141|AAAAAAAAELDCAAAA|2000-10-27||Lucky, new buses place aged a packages; new forces|0.77|0.50|7009001|importoimporto #2|9|shirts|2|Men|242|ableeseable|economy|28867309sandy3427834|plum|Pound|Unknown|74|oughteseoughtn st| +9142|AAAAAAAAGLDCAAAA|1997-10-27|1999-10-27|National, other ministers should spend more than increased programmes. Now psychological goods could change h|3.09|2.22|7008003|namelessbrand #3|8|lighting|7|Home|610|baroughtcally|N/A|0774614437navajo8849|rose|Tsp|Unknown|53|ableeseoughtn st| +9143|AAAAAAAAGLDCAAAA|1999-10-28|2001-10-26|National, other ministers should spend more than increased programmes. Now psychological goods could change h|8.12|2.22|3004002|edu packexporti #2|8|school-uniforms|3|Children|610|baroughtcally|large|0774614437navajo8849|navajo|N/A|Unknown|34|prieseoughtn st| +9144|AAAAAAAAGLDCAAAA|2001-10-27||Heavily federal demands comply usually here various ways. Private, naked birds may illustrate as gentle, unpleasant villages; high, conventional forms m|1.29|2.22|5004001|edu packscholar #1|4|classical|5|Music|610|baroughtcally|N/A|0774614437navajo8849|turquoise|Carton|Unknown|66|eseeseoughtn st| +9145|AAAAAAAAJLDCAAAA|1997-10-27||Parliamentary courts make cases; new parents might pitch following parts. Romantic children give simply old, genetic pools. Centu|90.55|53.42|8003006|exportinameless #6|3|basketball|8|Sports|266|callycallyable|N/A|071874tan59386416412|blush|Each|Unknown|58|antieseoughtn st| +9146|AAAAAAAAKLDCAAAA|1997-10-27|2000-10-26|Agricultural things may get certainly nationally local eyes. Crude, rigid teachers stop ago beaut|62.20|27.36|3002001|importoexporti #1|2|infants|3|Children|317|ationoughtpri|medium|866757104502puff0956|slate|Dozen|Unknown|14|callyeseoughtn st| +9147|AAAAAAAAKLDCAAAA|2000-10-27||Agricultural things may get certainly nationally local eyes. Crude, rigid teachers stop ago beaut|8.78|27.36|7005008|scholarbrand #8|5|blinds/shades|7|Home|317|ationoughtpri|N/A|866757104502puff0956|purple|Box|Unknown|31|ationeseoughtn st| +9148|AAAAAAAAMLDCAAAA|1997-10-27|1999-10-27|Different, sexual results will finish yet head, main companies. Clearly judicial periods see fat students. Economic regulat|7.91|4.27|4002001|importoedu pack #1|2|mens|4|Shoes|489|n steingese|small|4231424599wheat32882|turquoise|Ton|Unknown|98|eingeseoughtn st| +9149|AAAAAAAAMLDCAAAA|1999-10-28|2001-10-26|Early, similar bits must look with the |3.87|2.16|1003002|exportiamalg #2|2|maternity|1|Women|489|n steingese|large|4231424599wheat32882|purple|Gram|Unknown|66|n steseoughtn st| +9150|AAAAAAAAMLDCAAAA|2001-10-27||Only special plants would see later social, british minutes. Dual, new pilot|26.51|2.16|6007007|brandcorp #7|2|pendants|6|Jewelry|287|ationeingable|N/A|962439papaya57818759|orange|Dram|Unknown|88|barantioughtn st| +9151|AAAAAAAAPLDCAAAA|1997-10-27||Most economic elections bring to the questions. Important years c|1.65|0.70|5003002|exportischolar #2|3|pop|5|Music|151|oughtantiought|N/A|14279sky920500474787|medium|Unknown|Unknown|51|oughtantioughtn st| +9152|AAAAAAAAAMDCAAAA|1997-10-27|2000-10-26|Things keep at a others. Full, central wage|2.94|2.61|8012003|importomaxi #3|12|guns|8|Sports|570|barationanti|N/A|182018575olive876764|lawn|Gram|Unknown|61|ableantioughtn st| +9153|AAAAAAAAAMDCAAAA|2000-10-27||Very good purposes could see apparently forward international buildings. Naturally impo|76.28|2.61|8012003|edu packimporto #2|4|sports-apparel|2|Men|570|barationanti|medium|096medium63311065125|turquoise|N/A|Unknown|34|priantioughtn st| +9154|AAAAAAAACMDCAAAA|1997-10-27|1999-10-27|Black, powerful others go now years. Diverse orders might not mean away medium minutes; tight authorities ought to put however for the things|2.75|2.33|9001011|amalgmaxi #11|1|arts|9|Books|236|callypriable|N/A|2spring0198608390890|cream|Cup|Unknown|5|eseantioughtn st| +9155|AAAAAAAACMDCAAAA|1999-10-28|2001-10-26|Black, powerful others go now years. Diverse orders might not mean away medium minutes; tight authorities ought to put however for the things|8.36|2.33|9001011|exportiamalg #2|3|maternity|1|Women|12|ableought|medium|2spring0198608390890|pale|Gram|Unknown|22|antiantioughtn st| +9156|AAAAAAAACMDCAAAA|2001-10-27||Black, powerful others go now years. Diverse orders might not mean away medium minutes; tight authorities ought to put however for the things|6.91|3.86|1004001|edu packamalg #1|3|swimwear|1|Women|140|bareseought|small|72292538thistle94937|maroon|Dozen|Unknown|7|callyantioughtn st| +9157|AAAAAAAAFMDCAAAA|1997-10-27||Old, wrong departments may not know straigh|2.18|1.96|4003002|exportiedu pack #2|3|kids|4|Shoes|6|cally|medium|887grey2105292877893|sky|Gram|Unknown|27|ationantioughtn st| +9158|AAAAAAAAGMDCAAAA|1997-10-27|2000-10-26|Actors get. Small, early proposals will not care only. Other patients can build all observations; angry, young children conceal. Relative, mad rules organise especially ago stable mountai|3.84|2.91|3004001|edu packexporti #1|4|school-uniforms|3|Children|511|oughtoughtanti|petite|69038orchid961402813|linen|Ton|Unknown|29|eingantioughtn st| +9159|AAAAAAAAGMDCAAAA|2000-10-27||Necessary, limited results run quickly left, common minutes. Other guests find weak eyes. Silly, late months know earlier years; joint tears gain sick|7.80|6.78|6013008|exportibrand #8|13|loose stones|6|Jewelry|511|oughtoughtanti|N/A|69038orchid961402813|rose|Dram|Unknown|17|n stantioughtn st| +9160|AAAAAAAAIMDCAAAA|1997-10-27|1999-10-27|Other, suc|0.91|0.58|6013001|exportibrand #1|13|loose stones|6|Jewelry|922|ableablen st|N/A|5rose523596633633226|pink|Oz|Unknown|82|barcallyoughtn st| +9161|AAAAAAAAIMDCAAAA|1999-10-28|2001-10-26|Central systems shall sustain also human, independent authorities. Human, exact skills could not drive qui|3.98|1.43|6013001|importoamalg #2|13|fragrances|1|Women|922|ableablen st|medium|97468688116midnight1|red|Each|Unknown|25|oughtcallyoughtn st| +9162|AAAAAAAAIMDCAAAA|2001-10-27||Central systems shall sustain also human, independent authorities. Human, exact skills could not drive qui|7.50|4.57|6013001|exportiedu pack #1|3|kids|4|Shoes|742|ableeseation|large|273saddle15039226541|saddle|Ounce|Unknown|39|ablecallyoughtn st| +9163|AAAAAAAALMDCAAAA|1997-10-27||Military, black bodies enjoy. Poor, old supplies ought to exercise longer past cases. About white lives could hear penalties. Main, economic c|3.02|2.05|3002002|importoexporti #2|2|infants|3|Children|629|n stablecally|extra large|3301778seashell01372|seashell|Case|Unknown|22|pricallyoughtn st| +9164|AAAAAAAAMMDCAAAA|1997-10-27|2000-10-26|Wooden, capable talks may mark also|5.28|4.54|3003001|exportiexporti #1|3|toddlers|3|Children|444|eseeseese|medium|281865tomato36725045|pink|Pallet|Unknown|95|esecallyoughtn st| +9165|AAAAAAAAMMDCAAAA|2000-10-27||Wooden, capable talks may mark also|2.94|4.54|10013003|exportiamalgamalg #3|3|stereo|10|Electronics|444|eseeseese|N/A|281865tomato36725045|saddle|Pallet|Unknown|5|anticallyoughtn st| +9166|AAAAAAAAOMDCAAAA|1997-10-27|1999-10-27|New, low companies arrange times. Available, foreign troops can complain usuall|80.57|60.42|8012007|importomaxi #7|12|guns|8|Sports|320|barablepri|N/A|34089471115papaya492|salmon|Case|Unknown|8|callycallyoughtn st| +9167|AAAAAAAAOMDCAAAA|1999-10-28|2001-10-26|New, low companies arrange times. Available, foreign troops can complain usuall|6.43|60.42|6007008|brandcorp #8|7|pendants|6|Jewelry|377|ationationpri|N/A|34089471115papaya492|medium|Cup|Unknown|2|ationcallyoughtn st| +9168|AAAAAAAAOMDCAAAA|2001-10-27||New, low companies arrange times. Available, foreign troops can complain usuall|5.20|1.76|6007008|edu packimporto #1|4|sports-apparel|2|Men|657|ationationpri|medium|34089471115papaya492|light|Each|Unknown|83|eingcallyoughtn st| +9169|AAAAAAAABNDCAAAA|1997-10-27||Inherent, personal miles could go yet special, early wives. Sometime|9.84|6.69|6003002|exporticorp #2|3|gold|6|Jewelry|611|oughtoughtcally|N/A|75582348332365smoke8|navy|Tbl|Unknown|38|n stcallyoughtn st| +9170|AAAAAAAACNDCAAAA|1997-10-27|2000-10-26|Culturally short agreements used to obtain far but for the hundreds; also british owners shall return usually. Little workers use again for a l|0.54|0.38|10002008|importounivamalg #8|2|camcorders|10|Electronics|123|priableought|N/A|10061862linen4186355|smoke|Tbl|Unknown|12|barationoughtn st| +9171|AAAAAAAACNDCAAAA|2000-10-27||Rights land in a prisons. Plants tell for ever still judicial services. Ago poor functions cannot deny over. Anyway industrial variables could contact very patients. There original pp. ought to stu|2.76|0.38|9014004|edu packunivamalg #4|2|sports|9|Books|333|pripripri|N/A|31484208steel3569461|navy|Bundle|Unknown|82|oughtationoughtn st| +9172|AAAAAAAAENDCAAAA|1997-10-27|1999-10-27|Fair, black childr|4.77|4.19|10015010|scholaramalgamalg #10|15|portable|10|Electronics|550|barantianti|N/A|2582408royal81007976|yellow|Unknown|Unknown|8|ableationoughtn st| +9173|AAAAAAAAENDCAAAA|1999-10-28|2001-10-26|Private engineers take especially fellow attempts. Reforms reply. Aga|5.62|3.59|10015010|exportiimporto #2|3|pants|2|Men|550|barantianti|medium|2582408royal81007976|smoke|Gross|Unknown|25|priationoughtn st| +9174|AAAAAAAAENDCAAAA|2001-10-27||Competen|9.40|6.20|10015010|edu packexporti #1|3|school-uniforms|3|Children|550|barantianti|medium|617870491salmon16120|royal|Pound|Unknown|7|eseationoughtn st| +9175|AAAAAAAAHNDCAAAA|1997-10-27||About european ways remain secondly expensive, powerful kinds. Dome|6.36|2.54|4002002|importoedu pack #2|2|mens|4|Shoes|116|callyoughtought|medium|557260130312plum3537|navajo|Dozen|Unknown|11|antiationoughtn st| +9176|AAAAAAAAINDCAAAA|1997-10-27|2000-10-26|Visible, national regulations should tax. Delegates draw; away green residents cannot land less at last intelligent observers|1.48|0.56|2002001|importoimporto #1|2|shirts|2|Men|609|n stbarcally|petite|460pale5279785045836|royal|Lb|Unknown|74|callyationoughtn st| +9177|AAAAAAAAINDCAAAA|2000-10-27||Visible, national regulations should tax. Delegates draw; away green residents cannot land less at last intelligent observers|4.60|2.16|2002001|univamalgamalg #15|10|memory|10|Electronics|821|n stbarcally|N/A|28blush7515994311056|forest|Gross|Unknown|35|ationationoughtn st| +9178|AAAAAAAAKNDCAAAA|1997-10-27|1999-10-27|New, middle-class men would not get new words. Before unlikely sentences should see rivers. Yesterday |8.20|6.56|1001001|amalgamalg #1|1|dresses|1|Women|49|n stese|small|53582566sienna941462|brown|Carton|Unknown|39|eingationoughtn st| +9179|AAAAAAAAKNDCAAAA|1999-10-28|2001-10-26|New, middle-class men would not get new words. Before unlikely sentences should see rivers. Yesterday |2.77|1.21|8011010|amalgmaxi #10|11|archery|8|Sports|827|ationableeing|N/A|53582566sienna941462|turquoise|Ounce|Unknown|6|n stationoughtn st| +9180|AAAAAAAAKNDCAAAA|2001-10-27||Painfully democratic classes would not distinguish so big executives. Mere, early lips would shake also tory profits. Technological students will not shop at present from |55.83|1.21|8011010|importoimporto #1|2|shirts|2|Men|188|eingeingought|small|3678pale915703251058|ivory|Pallet|Unknown|2|bareingoughtn st| +9181|AAAAAAAANNDCAAAA|1997-10-27||Black, public shareholders amuse given books. Temporary authorities cannot order only directly due bottles. Parties would play soon only small deci|7.13|3.42|6012006|importobrand #6|12|costume|6|Jewelry|348|eingesepri|N/A|502811purple64637467|spring|Oz|Unknown|17|oughteingoughtn st| +9182|AAAAAAAAONDCAAAA|1997-10-27|2000-10-26|Libraries will result too cond|0.63|0.40|9007003|brandmaxi #3|7|reference|9|Books|91|oughtn st|N/A|7417058223789rose347|wheat|Pallet|Unknown|9|ableeingoughtn st| +9183|AAAAAAAAONDCAAAA|2000-10-27||Proud, defensive programmes bother just, selective spots; superb fields hear alone on a participants. Publicly vertical hands tackle high, professional instructions; social, present develop|7.75|0.40|9007003|exportiexporti #2|7|toddlers|3|Children|631|oughtn st|medium|7417058223789rose347|snow|Dozen|Unknown|74|prieingoughtn st| +9184|AAAAAAAAAODCAAAA|1997-10-27|1999-10-27|Well major enemies might access only extra good parties. Other, quiet eyes can buy completely western, effective feelings; materi|3.89|1.94|8014005|edu packmaxi #5|14|tennis|8|Sports|752|ableantiation|N/A|76714705sky844652045|tan|Pound|Unknown|21|eseeingoughtn st| +9185|AAAAAAAAAODCAAAA|1999-10-28|2001-10-26|Well major enemies might access only extra good parties. Other, quiet eyes can buy completely western, effective feelings; materi|7.44|5.80|2003002|exportiimporto #2|3|pants|2|Men|752|ableantiation|economy|76714705sky844652045|powder|Ton|Unknown|53|antieingoughtn st| +9186|AAAAAAAAAODCAAAA|2001-10-27||Systems make merely for example rational times. As regular grounds should give more rich shoulders. Significant children expect; wars may result available conditions. Dange|4.57|5.80|2003002|amalgunivamalg #7|1|cameras|10|Electronics|20|ableantiation|N/A|76714705sky844652045|violet|Ton|Unknown|3|callyeingoughtn st| +9187|AAAAAAAADODCAAAA|1997-10-27||Clearly global feet should validate also dates. Recent posts shall apply |15.64|5.00|10004008|edu packunivamalg #8|4|audio|10|Electronics|106|callybarought|N/A|61579sienna290139028|navy|Pound|Unknown|16|ationeingoughtn st| +9188|AAAAAAAAEODCAAAA|1997-10-27|2000-10-26|Terribly necessary systems take other, difficult improvements. Effective, simple places make at all. Minds might |9.60|7.20|7005009|scholarbrand #9|5|blinds/shades|7|Home|179|n stationought|N/A|6295836741seashell23|violet|Pallet|Unknown|6|eingeingoughtn st| +9189|AAAAAAAAEODCAAAA|2000-10-27||Terribly necessary systems take other, difficult improvements. Effective, simple places make at all. Minds might |4.99|4.39|1002002|importoamalg #2|5|fragrances|1|Women|179|n stationought|small|6295836741seashell23|rosy|Dram|Unknown|44|n steingoughtn st| +9190|AAAAAAAAGODCAAAA|1997-10-27|1999-10-27|Forces must not lift less than large, dependent weekends. Real cattle used to show freshly. Main, national courses marry. Coloured partners go then by a rule|77.04|48.53|2004001|edu packimporto #1|4|sports-apparel|2|Men|176|callyationought|large|7225494seashell47890|royal|Dram|Unknown|4|barn stoughtn st| +9191|AAAAAAAAGODCAAAA|1999-10-28|2001-10-26|Forces must not lift less than large, dependent weekends. Real cattle used to show freshly. Main, national courses marry. Coloured partners go then by a rule|3.79|48.53|2004001|namelessbrand #2|8|lighting|7|Home|587|ationeinganti|N/A|378697426yellow86566|wheat|Dozen|Unknown|4|oughtn stoughtn st| +9192|AAAAAAAAGODCAAAA|2001-10-27||Days tick further|9.57|6.41|1002001|importoamalg #1|8|fragrances|1|Women|587|ationeinganti|large|4670248822413spring8|misty|Case|Unknown|26|ablen stoughtn st| +9193|AAAAAAAAJODCAAAA|1997-10-27||More important humans cannot say into a troops. Irritably long standards may pass again costs. Ugly, expected changes confirm|4.78|2.24|4003002|exportiedu pack #2|3|kids|4|Shoes|293|prin stable|medium|363258602sienna21610|smoke|Case|Unknown|3|prin stoughtn st| +9194|AAAAAAAAKODCAAAA|1997-10-27|2000-10-26|Truly alone losses would amoun|1.15|0.88|10009015|maxiunivamalg #15|9|televisions|10|Electronics|192|ablen stought|N/A|2997546153101powder6|orange|Case|Unknown|76|esen stoughtn st| +9195|AAAAAAAAKODCAAAA|2000-10-27||Ships would not portray go|7.33|4.25|4002002|importoedu pack #2|2|mens|4|Shoes|26|callyable|medium|538128816068khaki887|white|Each|Unknown|28|antin stoughtn st| +9196|AAAAAAAAMODCAAAA|1997-10-27|1999-10-27|Present things pay painfully. Sharp, automatic workers see detailed, old authorities. Customers might cite very days. Complete issues used to tel|3.21|1.73|3003001|exportiexporti #1|3|toddlers|3|Children|70|baration|small|95427043snow15379425|honeydew|Bunch|Unknown|40|callyn stoughtn st| +9197|AAAAAAAAMODCAAAA|1999-10-28|2001-10-26|International patterns imply freshly with a patient|0.44|0.35|3003001|amalgamalgamalg #1|11|disk drives|10|Electronics|70|baration|N/A|176757045puff1441446|pale|Pallet|Unknown|18|ationn stoughtn st| +9198|AAAAAAAAMODCAAAA|2001-10-27||Only normal games should provide earlier definitions. Skilled, important relations shall write political, professional proposals. Heroes replace mass times. Relevant premises try further shallo|4.41|0.35|3003001|amalgcorp #7|11|birdal|6|Jewelry|70|baration|N/A|594025212871sienna32|saddle|Unknown|Unknown|14|eingn stoughtn st| +9199|AAAAAAAAPODCAAAA|1997-10-27||Rooms swallow again; materials take all certainly individual pupils. All social children can keep to a friends. Groups|93.21|56.85|2002002|importoimporto #2|2|shirts|2|Men|590|barn stanti|extra large|9656545558257smoke57|snow|Ton|Unknown|9|n stn stoughtn st| +9200|AAAAAAAAAPDCAAAA|1997-10-27|2000-10-26|Able, new legs ought to get whole, minor patterns; evident groups hit for a elections. Interests cannot r|28.41|21.02|6012001|importobrand #1|12|costume|6|Jewelry|542|ableeseanti|N/A|54000mint28465823510|pink|Gram|Unknown|19|barbarablen st| +9201|AAAAAAAAAPDCAAAA|2000-10-27||Little lights will not kill silent assumptions. Artists must go. Great, simple mi|4.79|21.02|9013010|exportiunivamalg #10|12|self-help|9|Books|28|eingable|N/A|54000mint28465823510|violet|Tsp|Unknown|65|oughtbarablen st| +9202|AAAAAAAACPDCAAAA|1997-10-27|1999-10-27|Cells must raise away. Rather prominent ships lead; catholic, full heads must not tell especially double, international police; modest years may fetch readily important, final pubs. Softly|2.53|1.03|2003001|exportiimporto #1|3|pants|2|Men|260|barcallyable|medium|033625deep7172896325|violet|Gram|Unknown|30|ablebarablen st| +9203|AAAAAAAACPDCAAAA|1999-10-28|2001-10-26|Cells must raise away. Rather prominent ships lead; catholic, full heads must not tell especially double, international police; modest years may fetch readily important, final pubs. Softly|3.48|2.40|2003001|importocorp #4|3|diamonds|6|Jewelry|206|callybarable|N/A|120chartreuse0042618|spring|Tsp|Unknown|12|pribarablen st| +9204|AAAAAAAACPDCAAAA|2001-10-27||All true associations assist better. Perfectly large legs capture therefore. Happy orders will conceal at least |8.82|5.82|10003008|exportiunivamalg #8|3|dvd/vcr players|10|Electronics|206|callybarable|N/A|snow4191086336270220|slate|Tbl|Unknown|50|esebarablen st| +9205|AAAAAAAAFPDCAAAA|1997-10-27||Available, simple accounts might get also exotic minutes. Circumstances may balance surprising polls. Stories will think naturally both social months. Wrong, fair persons flee full lines;|2.91|1.45|2002002|importoimporto #2|2|shirts|2|Men|391|oughtn stpri|medium|66616094370341lemon1|pink|Bunch|Unknown|20|antibarablen st| +9206|AAAAAAAAGPDCAAAA|1997-10-27|2000-10-26|Designs would know inner armies. Also whole months belie|0.67|0.47|4004001|edu packedu pack #1|4|athletic|4|Shoes|155|antiantiought|N/A|0247147spring5785937|sky|Tsp|Unknown|29|callybarablen st| +9207|AAAAAAAAGPDCAAAA|2000-10-27||English industries move; soon upper stores may cool then then high arrangements. More afraid newspapers prevent. Expected lines might not know inclu|3.09|0.47|2003002|exportiimporto #2|3|pants|2|Men|288|antiantiought|small|2167puff328389880529|lime|Ton|Unknown|14|ationbarablen st| +9208|AAAAAAAAIPDCAAAA|1997-10-27|1999-10-27|Wrong signals carry at last phenomena; aims provide much as a schools. Global seats press old offic|9.08|6.90|2004001|edu packimporto #1|4|sports-apparel|2|Men|924|eseablen st|large|0909590643971salmon1|lavender|Cup|Unknown|94|eingbarablen st| +9209|AAAAAAAAIPDCAAAA|1999-10-28|2001-10-26|Wrong signals carry at last phenomena; aims provide much as a schools. Global seats press old offic|4.75|3.32|3002002|importoexporti #2|4|infants|3|Children|924|eseablen st|medium|0909590643971salmon1|red|Case|Unknown|25|n stbarablen st| +9210|AAAAAAAAIPDCAAAA|2001-10-27||Wrong signals carry at last phenomena; aims provide much as a schools. Global seats press old offic|3.16|1.01|7008001|namelessbrand #1|8|lighting|7|Home|276|eseablen st|N/A|0909590643971salmon1|lime|Bundle|Unknown|18|baroughtablen st| +9211|AAAAAAAALPDCAAAA|1997-10-27||So small edges will understand currently in a things. New trains point usually systems. Years look growing questions. Different cases could sell just alive, late rules; big, large results will make |4.12|1.23|9002008|importomaxi #8|2|business|9|Books|498|eingn stese|N/A|5seashell68995921499|tan|Tsp|Unknown|15|oughtoughtablen st| +9212|AAAAAAAAMPDCAAAA|1997-10-27|2000-10-26|Settlements should work rather french states. Still advisory words shall not come also democratic characters. Otherwise very managers should not realise students. Cities will not find|4.86|4.17|3003001|exportiexporti #1|3|toddlers|3|Children|63|prically|medium|24073691175honeydew6|magenta|Dram|Unknown|64|ableoughtablen st| +9213|AAAAAAAAMPDCAAAA|2000-10-27||Easy machines can pick in order then natural practices. Indeed rough efforts must compete to a faces. Years run interested developers. Scientists get desirable times. Medieval|2.59|2.30|6008006|namelesscorp #6|3|mens watch|6|Jewelry|524|prically|N/A|24073691175honeydew6|cream|Gram|Unknown|30|prioughtablen st| +9214|AAAAAAAAOPDCAAAA|1997-10-27|1999-10-27|Especially local thousands withdraw as workers. Else direct teams renew long indu|3.03|1.78|7003005|exportibrand #5|3|kids|7|Home|185|antieingought|N/A|1342267smoke08418977|rosy|Lb|Unknown|17|eseoughtablen st| +9215|AAAAAAAAOPDCAAAA|1999-10-28|2001-10-26|Especially local thousands withdraw as workers. Else direct teams renew long indu|8.40|1.78|7003005|exportiedu pack #2|3|kids|4|Shoes|185|antieingought|economy|49691918navy77003244|saddle|Case|Unknown|83|antioughtablen st| +9216|AAAAAAAAOPDCAAAA|2001-10-27||Especially local thousands withdraw as workers. Else direct teams renew long indu|3.70|1.78|7003005|importoamalg #1|2|fragrances|1|Women|185|antieingought|extra large|49691918navy77003244|lemon|N/A|Unknown|24|callyoughtablen st| +9217|AAAAAAAABAECAAAA|1997-10-27||Likely, roman barriers take in a incomes. As central competitors write only employees. Leading, able shares make recently full approaches. True efforts suggest briti|3.83|1.95|5004002|edu packscholar #2|4|classical|5|Music|251|oughtantiable|N/A|orchid85644064787615|sienna|Box|Unknown|39|ationoughtablen st| +9218|AAAAAAAACAECAAAA|1997-10-27|2000-10-26|Courses come then political terms. African women inform about powerful eyes. Years will escape bold benefits. Offices as|0.60|0.22|7013009|exportinameless #9|13|wallpaper|7|Home|504|esebaranti|N/A|03838643779indian090|lace|Oz|Unknown|5|eingoughtablen st| +9219|AAAAAAAACAECAAAA|2000-10-27||Contacts bring little payments. Incredible, healthy men would see at least mines. Brave, vital questions see good events. Large, precise nations could not drive. Courses extend late|3.64|2.29|2001002|amalgimporto #2|1|accessories|2|Men|504|esebaranti|extra large|97492654light9896505|rosy|Dram|Unknown|93|n stoughtablen st| +9220|AAAAAAAAEAECAAAA|1997-10-27|1999-10-27|Deaf germans love especially possible, just relat|3.06|0.97|6014005|edu packbrand #5|14|estate|6|Jewelry|148|eingeseought|N/A|12papaya821800000794|seashell|N/A|Unknown|19|barableablen st| +9221|AAAAAAAAEAECAAAA|1999-10-28|2001-10-26|Deaf germans love especially possible, just relat|1.37|0.97|6002006|importocorp #6|14|diamonds|6|Jewelry|148|eingeseought|N/A|089199peru8183937135|tan|Case|Unknown|21|oughtableablen st| +9222|AAAAAAAAEAECAAAA|2001-10-27||Deaf germans love especially possible, just relat|2.92|0.97|6002006|exportiedu pack #1|14|kids|4|Shoes|148|eingeseought|petite|4876064purple0854806|seashell|Bundle|Unknown|88|ableableablen st| +9223|AAAAAAAAHAECAAAA|1997-10-27||Forces may ruin mainly popular payments. Vessels ought to end again white years. Extensive, advisory troops stand particularly busy, old beings. Areas persuade slightly external pupils; anci|5.28|4.48|6003008|exporticorp #8|3|gold|6|Jewelry|322|ableablepri|N/A|2232438tan4946942511|papaya|Unknown|Unknown|64|priableablen st| +9224|AAAAAAAAIAECAAAA|1997-10-27|2000-10-26|Elsewhere high sentences will not cancel; usually soviet communications prevent chapters; estimated costs shall not r|1.64|1.42|5002001|importoscholar #1|2|country|5|Music|16|callyought|N/A|9646621tan6277516648|indian|Tbl|Unknown|42|eseableablen st| +9225|AAAAAAAAIAECAAAA|2000-10-27||Elsewhere high sentences will not cancel; usually soviet communications prevent chapters; estimated costs shall not r|2.64|1.42|5002001|exporticorp #2|2|gold|6|Jewelry|16|callyought|N/A|steel761666962583238|tan|Tsp|Unknown|1|antiableablen st| +9226|AAAAAAAAKAECAAAA|1997-10-27|1999-10-27|Important, unique shows used to allow once; lines|4.57|1.87|1004001|edu packamalg #1|4|swimwear|1|Women|700|barbaration|small|697458726dark0284669|lime|Tbl|Unknown|63|callyableablen st| +9227|AAAAAAAAKAECAAAA|1999-10-28|2001-10-26|Important, unique shows used to allow once; lines|7.82|2.97|5001002|amalgscholar #2|1|rock|5|Music|666|barbaration|N/A|5pale194981457135798|plum|Tsp|Unknown|100|ationableablen st| +9228|AAAAAAAAKAECAAAA|2001-10-27||Also following offences ought to meet hard, wrong negotiations. Sounds could not answer m|1.24|0.43|9009001|maximaxi #1|9|science|9|Books|666|barbaration|N/A|587531815354papaya72|sandy|Bunch|Unknown|18|eingableablen st| +9229|AAAAAAAANAECAAAA|1997-10-27||Walls used to define then. Also tropical metals might shrink so joint, alternative techniques. Greatl|65.63|21.65|10013016|exportiamalgamalg #16|13|stereo|10|Electronics|314|eseoughtpri|N/A|73steel4858871850489|yellow|Box|Unknown|81|n stableablen st| +9230|AAAAAAAAOAECAAAA|1997-10-27|2000-10-26|Perhaps other hands indulge. Classes identify especially important issues. Chief, full pounds try present problems. Categories summarise then national women. Unable children might no|9.45|6.52|7008001|namelessbrand #1|8|lighting|7|Home|272|ableationable|N/A|6407350685white73819|lime|Ounce|Unknown|12|barpriablen st| +9231|AAAAAAAAOAECAAAA|2000-10-27||Improvements could ship tomorrow children. More marvellous books shall see even initial structures. Important, local terms would not come tota|9.94|8.84|7008001|exportiedu pack #2|8|kids|4|Shoes|272|ableationable|economy|6407350685white73819|spring|Bundle|Unknown|28|oughtpriablen st| +9232|AAAAAAAAABECAAAA|1997-10-27|1999-10-27|Critical, other governments reconcile inside; men calcu|80.11|32.84|2004001|edu packimporto #1|4|sports-apparel|2|Men|238|eingpriable|medium|631474161116140red00|papaya|Ounce|Unknown|31|ablepriablen st| +9233|AAAAAAAAABECAAAA|1999-10-28|2001-10-26|Advanced, lucky investments shall see opposite, old calculations. Real numbers will not watch however by no means in|3.92|32.84|2004001|amalgedu pack #2|4|womens|4|Shoes|238|eingpriable|petite|98346627chartreuse82|dodger|Dram|Unknown|23|pripriablen st| +9234|AAAAAAAAABECAAAA|2001-10-27||Advanced, lucky investments shall see opposite, old calculations. Real numbers will not watch however by no means in|1.02|0.79|2004001|exportiedu pack #1|3|kids|4|Shoes|238|eingpriable|small|98346627chartreuse82|turquoise|Ounce|Unknown|6|esepriablen st| +9235|AAAAAAAADBECAAAA|1997-10-27||Able authorities care ago advisory windows. Results m|3.78|2.23|1001002|amalgamalg #2|1|dresses|1|Women|616|callyoughtcally|extra large|83358sky795155636469|goldenrod|Dozen|Unknown|28|antipriablen st| +9236|AAAAAAAAEBECAAAA|1997-10-27|2000-10-26|Eastern, rural activities mak|1.60|1.04|8015007|scholarmaxi #7|15|fishing|8|Sports|22|ableable|N/A|7402887282tan1783807|goldenrod|Box|Unknown|51|callypriablen st| +9237|AAAAAAAAEBECAAAA|2000-10-27||Often central seconds will block then loans; different events begin neatly tired results. Right hundreds will tempt ever foreign, armed appeals. Other, honest components shall not change social |1.36|1.04|8015007|importoimporto #2|15|shirts|2|Men|22|ableable|economy|37634burnished781572|thistle|Pound|Unknown|22|ationpriablen st| +9238|AAAAAAAAGBECAAAA|1997-10-27|1999-10-27|Lovely years can stretch there possible members. Then black lakes help then reliable troops. Difficult rights would feel; new ar|4.70|2.72|4002001|importoedu pack #1|2|mens|4|Shoes|294|esen stable|petite|44417240884plum08337|seashell|Lb|Unknown|47|eingpriablen st| +9239|AAAAAAAAGBECAAAA|1999-10-28|2001-10-26|Lovely years can stretch there possible members. Then black lakes help then reliable troops. Difficult rights would feel; new ar|1.64|2.72|4002001|edu packscholar #2|4|classical|5|Music|294|esen stable|N/A|82991465plum91008536|pale|Bunch|Unknown|16|n stpriablen st| +9240|AAAAAAAAGBECAAAA|2001-10-27||Lovely years can stretch there possible members. Then black lakes help then reliable troops. Difficult rights would feel; new ar|2.90|2.40|10015010|scholaramalgamalg #10|15|portable|10|Electronics|294|esen stable|N/A|82991465plum91008536|plum|Ounce|Unknown|4|bareseablen st| +9241|AAAAAAAAJBECAAAA|1997-10-27||Workers could reveal more|6.04|3.68|2003002|exportiimporto #2|3|pants|2|Men|594|esen stanti|large|0680232751578rosy119|antique|N/A|Unknown|21|oughteseablen st| +9242|AAAAAAAAKBECAAAA|1997-10-27|2000-10-26|Average, dead pairs search more from the judges; german, international police might come exactly important tasks. Separate, urban eyes may confront. Subjects kill c|3.39|1.01|6010003|univbrand #3|10|jewelry boxes|6|Jewelry|484|eseeingese|N/A|548powder43718207491|sandy|Dozen|Unknown|36|ableeseablen st| +9243|AAAAAAAAKBECAAAA|2000-10-27||Average, dead pairs search more from the judges; german, international police might come exactly important tasks. Separate, urban eyes may confront. Subjects kill c|2.87|1.01|6010003|importoexporti #2|2|infants|3|Children|484|eseeingese|petite|7496553364264olive40|navy|Oz|Unknown|55|prieseablen st| +9244|AAAAAAAAMBECAAAA|1997-10-27|1999-10-27|Blue magazines shall remain anyway crops. Silver practices shall find cautious, top|6.95|4.03|10016014|corpamalgamalg #14|16|wireless|10|Electronics|124|eseableought|N/A|27drab76103678707793|lime|N/A|Unknown|79|eseeseablen st| +9245|AAAAAAAAMBECAAAA|1999-10-28|2001-10-26|Blue magazines shall remain anyway crops. Silver practices shall find cautious, top|5.96|2.86|10016014|importoamalg #2|16|fragrances|1|Women|124|eseableought|petite|27drab76103678707793|saddle|Tbl|Unknown|34|antieseablen st| +9246|AAAAAAAAMBECAAAA|2001-10-27||Blue magazines shall remain anyway crops. Silver practices shall find cautious, top|8.83|4.67|6016007|corpbrand #7|16|consignment|6|Jewelry|124|eseableought|N/A|27drab76103678707793|rosy|Dram|Unknown|23|callyeseablen st| +9247|AAAAAAAAPBECAAAA|1997-10-27||Places look; students sell especially. Right black tests make once again|2.18|0.78|7004008|edu packbrand #8|4|curtains/drapes|7|Home|39|n stpri|N/A|814427salmon20718407|slate|Cup|Unknown|21|ationeseablen st| +9248|AAAAAAAAACECAAAA|1997-10-27|2000-10-26|Quiet, modern institutions ought to mean already thin, sweet|2.00|0.68|2002001|importoimporto #1|2|shirts|2|Men|80|bareing|petite|87162830108snow55811|burnished|Oz|Unknown|42|eingeseablen st| +9249|AAAAAAAAACECAAAA|2000-10-27||Quiet, modern institutions ought to mean already thin, sweet|5.52|4.47|7016010|corpnameless #10|16|furniture|7|Home|80|bareing|N/A|80981298sandy1227744|plum|Cup|Unknown|39|n steseablen st| +9250|AAAAAAAACCECAAAA|1997-10-27|1999-10-27|Objects see often changes. Conceivably current sectors would not deal therefore today good things. Only other votes might judge great items. Certainly accessible drugs must want in a funds; past |6.95|2.91|3002001|importoexporti #1|2|infants|3|Children|28|eingable|petite|5123navajo4715597498|wheat|Dram|Unknown|45|barantiablen st| +9251|AAAAAAAACCECAAAA|1999-10-28|2001-10-26|Objects see often changes. Conceivably current sectors would not deal therefore today good things. Only other votes might judge great items. Certainly accessible drugs must want in a funds; past |6.02|2.91|9005012|scholarmaxi #12|5|history|9|Books|28|eingable|N/A|97474428518slate7110|violet|Tsp|Unknown|86|oughtantiablen st| +9252|AAAAAAAACCECAAAA|2001-10-27||Objects see often changes. Conceivably current sectors would not deal therefore today good things. Only other votes might judge great items. Certainly accessible drugs must want in a funds; past |4.21|3.62|9005012|importonameless #3|12|paint|7|Home|28|eingable|N/A|25470846484salmon971|yellow|Tbl|Unknown|23|ableantiablen st| +9253|AAAAAAAAFCECAAAA|1997-10-27||Overnight relevant systems will not address tensions. Considerable, political conditions might not dance real changes; actual, |5.68|4.65|7012004|importonameless #4|12|paint|7|Home|144|eseeseought|N/A|firebrick94746225943|thistle|Lb|Unknown|72|priantiablen st| +9254|AAAAAAAAGCECAAAA|1997-10-27|2000-10-26|Students help factors. Seats take matters; likely sources make ridiculous children. Police might say then just natural characters. A|1.15|0.62|5003001|exportischolar #1|3|pop|5|Music|456|callyantiese|N/A|549990439652slate943|midnight|Oz|Unknown|48|eseantiablen st| +9255|AAAAAAAAGCECAAAA|2000-10-27||Given leaders come differences; trends raise names. Old, dead books would not zero. Especially ready tickets get uniformly complaints. Deep, able bags use local films. Ye|13.28|6.50|5003001|importoamalg #2|2|fragrances|1|Women|456|callyantiese|petite|6830800550651green19|smoke|N/A|Unknown|32|antiantiablen st| +9256|AAAAAAAAICECAAAA|1997-10-27|1999-10-27|Long proposed artists send just minimal, m|0.77|0.23|1001001|amalgamalg #1|1|dresses|1|Women|179|n stationought|extra large|822667132plum7397682|papaya|Dram|Unknown|7|callyantiablen st| +9257|AAAAAAAAICECAAAA|1999-10-28|2001-10-26|Long proposed artists send just minimal, m|1.91|0.23|10014010|edu packamalgamalg #10|1|automotive|10|Electronics|179|n stationought|N/A|822667132plum7397682|pink|Dram|Unknown|6|ationantiablen st| +9258|AAAAAAAAICECAAAA|2001-10-27||Long proposed artists send just minimal, m|1.34|0.92|10004011|edu packunivamalg #11|1|audio|10|Electronics|179|n stationought|N/A|067502844686469rosy5|navajo|Cup|Unknown|6|eingantiablen st| +9259|AAAAAAAALCECAAAA|1997-10-27||Effectively entire workers achieve earnings. Requirements ought to associate even colours. Broad, conservative hours cannot choose. Invol|2.62|1.44|5003002|exportischolar #2|3|pop|5|Music|67|ationcally|N/A|05802304694123smoke1|white|Box|Unknown|30|n stantiablen st| +9260|AAAAAAAAMCECAAAA|1997-10-27|2000-10-26|Large students may not show simply nuclear countries. Kee|61.63|39.44|7005001|scholarbrand #1|5|blinds/shades|7|Home|949|n stesen st|N/A|1547310943thistle248|gainsboro|Pound|Unknown|99|barcallyablen st| +9261|AAAAAAAAMCECAAAA|2000-10-27||Large students may not show simply nuclear countries. Kee|2.91|39.44|10003014|exportiunivamalg #14|3|dvd/vcr players|10|Electronics|17|ationought|N/A|1547310943thistle248|pink|Tsp|Unknown|14|oughtcallyablen st| +9262|AAAAAAAAOCECAAAA|1997-10-27|1999-10-27|Today deep women serve. Ages die official discussions. Neutral, revolutiona|41.94|17.19|2003001|exportiimporto #1|3|pants|2|Men|451|oughtantiese|N/A|5peach65198547641019|sky|Ton|Unknown|42|ablecallyablen st| +9263|AAAAAAAAOCECAAAA|1999-10-28|2001-10-26|Today deep women serve. Ages die official discussions. Neutral, revolutiona|3.59|2.47|2003001|corpunivamalg #12|3|mystery|9|Books|451|oughtantiese|N/A|5peach65198547641019|floral|N/A|Unknown|22|pricallyablen st| +9264|AAAAAAAAOCECAAAA|2001-10-27||Mostly social police enable therefore in a terms. Simple, important actions will want. Successful visitors persuade very urgent officials. Social, cool pp. protect by no means. Difficu|6.86|2.47|2004001|edu packimporto #1|3|sports-apparel|2|Men|451|oughtantiese|petite|748mint9536377035278|tan|Dram|Unknown|21|esecallyablen st| +9265|AAAAAAAABDECAAAA|1997-10-27||White employees name; figures feed sure. Speeches can achieve. Extremely final seats may exist too all religious studies. Medite|2.46|1.89|3001002|amalgexporti #2|1|newborn|3|Children|59|n stanti|large|83804burnished679223|wheat|Ton|Unknown|39|anticallyablen st| +9266|AAAAAAAACDECAAAA|1997-10-27|2000-10-26|Patterns should say more personal indicators. Unlike, ready posts will go less able, left trades. Minerals cost in a incomes. Small, military details must make further in a servic|1.02|0.62|3004001|edu packexporti #1|4|school-uniforms|3|Children|107|ationbarought|medium|92lace69126332344489|tan|Gross|Unknown|22|callycallyablen st| +9267|AAAAAAAACDECAAAA|2000-10-27||Patterns should say more personal indicators. Unlike, ready posts will go less able, left trades. Minerals cost in a incomes. Small, military details must make further in a servic|7.17|4.87|3004001|amalgimporto #2|1|accessories|2|Men|230|ationbarought|large|92lace69126332344489|salmon|Gross|Unknown|64|ationcallyablen st| +9268|AAAAAAAAEDECAAAA|1997-10-27|1999-10-27|Local comments would appear failures. Sim|0.55|0.42|7008009|namelessbrand #9|8|lighting|7|Home|292|ablen stable|N/A|sandy675991598582277|steel|N/A|Unknown|13|eingcallyablen st| +9269|AAAAAAAAEDECAAAA|1999-10-28|2001-10-26|Local comments would appear failures. Sim|8.52|0.42|1004002|edu packamalg #2|8|swimwear|1|Women|217|ablen stable|small|5250045slate20130085|wheat|Tsp|Unknown|49|n stcallyablen st| +9270|AAAAAAAAEDECAAAA|2001-10-27||Local comments would appear failures. Sim|9.15|7.32|1004001|edu packamalg #1|8|swimwear|1|Women|130|ablen stable|large|5250045slate20130085|turquoise|Ton|Unknown|25|barationablen st| +9271|AAAAAAAAHDECAAAA|1997-10-27||Today fundamental forces consist yet units. Projects understand again roads. Only large waters can take offices. Now welsh reactions continue traditional laws. Women d|3.28|1.83|8003002|exportinameless #2|3|basketball|8|Sports|311|oughtoughtpri|N/A|02421351140167royal3|tomato|Ounce|Unknown|68|oughtationablen st| +9272|AAAAAAAAIDECAAAA|1997-10-27|2000-10-26|Late levels move statutory, level offices. Golden, classic trees treat little including a patients. Ideas grab actual|43.01|34.83|7005003|scholarbrand #3|5|blinds/shades|7|Home|159|n stantiought|N/A|907701036wheat687072|slate|Dram|Unknown|61|ableationablen st| +9273|AAAAAAAAIDECAAAA|2000-10-27||Late levels move statutory, level offices. Golden, classic trees treat little including a patients. Ideas grab actual|74.32|34.83|5003002|exportischolar #2|3|pop|5|Music|159|n stantiought|N/A|907701036wheat687072|violet|Dozen|Unknown|13|priationablen st| +9274|AAAAAAAAKDECAAAA|1997-10-27|1999-10-27|Therefore local communications ought to establish thereafter g|2.87|0.97|10004010|edu packunivamalg #10|4|audio|10|Electronics|86|callyeing|N/A|360539smoke015464860|magenta|Ounce|Unknown|10|eseationablen st| +9275|AAAAAAAAKDECAAAA|1999-10-28|2001-10-26|Therefore local communications ought to establish thereafter g|7.40|0.97|3002002|importoexporti #2|2|infants|3|Children|86|callyeing|petite|64186861rose22103581|rose|Bunch|Unknown|30|antiationablen st| +9276|AAAAAAAAKDECAAAA|2001-10-27||Then terrible skills should not |0.86|0.97|3002002|exportiamalg #1|3|maternity|1|Women|86|callyeing|medium|64186861rose22103581|thistle|N/A|Unknown|30|callyationablen st| +9277|AAAAAAAANDECAAAA|1997-10-27||French detectives might discuss as objective rewards; trees should not allocate. Civil images cause here year|8.44|3.37|7014008|edu packnameless #8|14|glassware|7|Home|86|callyeing|N/A|cyan4915762467330960|blue|Tbl|Unknown|21|ationationablen st| +9278|AAAAAAAAODECAAAA|1997-10-27|2000-10-26|Indian, recent occupations mind too effects; days will discuss today also significant meanings; short foreign services shall direct early, electrical children. Else old years say latterly complete co|4.36|2.00|9008003|namelessmaxi #3|8|romance|9|Books|106|callybarought|N/A|26azure4084634996402|pink|Ounce|Unknown|33|eingationablen st| +9279|AAAAAAAAODECAAAA|2000-10-27||Indian, recent occupations mind too effects; days will discuss today also significant meanings; short foreign services shall direct early, electrical children. Else old years say latterly complete co|4.67|1.72|8010010|univmaxi #10|10|pools|8|Sports|106|callybarought|N/A|4221peru329646166634|sandy|Bundle|Unknown|59|n stationablen st| +9280|AAAAAAAAAEECAAAA|1997-10-27|1999-10-27|Eyes turn even urban problems; current lines must use rather parents. Certainly married shops would justify implicitly british high points. Strategic, star circumstances shall not tell rather s|2.60|1.69|2003001|exportiimporto #1|3|pants|2|Men|583|prieinganti|large|555084magenta6629023|grey|Ton|Unknown|60|bareingablen st| +9281|AAAAAAAAAEECAAAA|1999-10-28|2001-10-26|Public, dry deals |21.37|1.69|7011002|amalgnameless #2|11|accent|7|Home|583|prieinganti|N/A|18284811390thistle63|tomato|N/A|Unknown|27|oughteingablen st| +9282|AAAAAAAAAEECAAAA|2001-10-27||Public, dry deals |4.16|3.28|6007003|brandcorp #3|7|pendants|6|Jewelry|583|prieinganti|N/A|4992tomato6004750085|red|Cup|Unknown|16|ableeingablen st| +9283|AAAAAAAADEECAAAA|1997-10-27||Important, direct players might follow just expenses. Cups cannot drive red goals. Minute areas see to a guests. Champions could adopt even other findings. Systems get. Beautiful, usual other|1.70|0.98|4004002|edu packedu pack #2|4|athletic|4|Shoes|509|n stbaranti|medium|peru4339884131154800|pale|Each|Unknown|55|prieingablen st| +9284|AAAAAAAAEEECAAAA|1997-10-27|2000-10-26|Now level families record today organisational questions. Friendly investors meet even young females. Groups appear only boundaries. Afterwards minimal banks may see alone. Gladly yellow purposes wi|1.08|0.70|5001001|amalgscholar #1|1|rock|5|Music|266|callycallyable|N/A|4772299162tomato1652|royal|Pallet|Unknown|27|eseeingablen st| +9285|AAAAAAAAEEECAAAA|2000-10-27||Teachers make by a banks. Over here quick movements shall not loosen |9.21|4.32|7006006|corpbrand #6|6|rugs|7|Home|266|callycallyable|N/A|41315pink81741872959|orange|Gross|Unknown|20|antieingablen st| +9286|AAAAAAAAGEECAAAA|1997-10-27|1999-10-27|Other, human years used to give simply. Words may carry for the pictures; general month|4.85|2.86|9004011|edu packmaxi #11|4|entertainments|9|Books|875|antiationeing|N/A|32312399175navy33469|peru|Gross|Unknown|29|callyeingablen st| +9287|AAAAAAAAGEECAAAA|1999-10-28|2001-10-26|Requests will not say sometime new, western voices. Countries make now adjacent men. Daily, true months should go |1.77|2.86|6002008|importocorp #8|2|diamonds|6|Jewelry|875|antiationeing|N/A|179608034steel944091|rose|Pallet|Unknown|69|ationeingablen st| +9288|AAAAAAAAGEECAAAA|2001-10-27||Parents used to record briefly computers. Dogs reclaim enough. Previous years shall se|0.66|2.86|6004001|edu packcorp #1|2|bracelets|6|Jewelry|875|antiationeing|N/A|8powder6761893685664|slate|Bunch|Unknown|8|eingeingablen st| +9289|AAAAAAAAJEECAAAA|1997-10-27||Children used to mean contracts. Difficult runs spot here. Aspects ought to take unfortunately prepared women. Groups believe very public patients. Low terms must stop as different, political cou|4.94|3.75|7002010|importobrand #10|2|bedding|7|Home|311|oughtoughtpri|N/A|946033530050156plum9|white|Pound|Unknown|19|n steingablen st| +9290|AAAAAAAAKEECAAAA|1997-10-27|2000-10-26|Original standards live far possible, no|1.57|0.62|6003003|exporticorp #3|3|gold|6|Jewelry|403|pribarese|N/A|524smoke975371596386|smoke|Gross|Unknown|17|barn stablen st| +9291|AAAAAAAAKEECAAAA|2000-10-27||Local friends like with the stones. Forms move with the curtains. Developing, northern findings fill sheets. Models can yield indirectly indi|3.84|0.62|4001002|amalgedu pack #2|1|womens|4|Shoes|403|pribarese|small|524smoke975371596386|orange|Bunch|Unknown|36|oughtn stablen st| +9292|AAAAAAAAMEECAAAA|1997-10-27|1999-10-27|Direct records would not marry in a suggestions. External standards avoid nice services. Large secrets|0.42|0.37|8016003|corpmaxi #3|16|golf|8|Sports|952|ableantin st|N/A|26lace02168211745104|antique|Bundle|Unknown|9|ablen stablen st| +9293|AAAAAAAAMEECAAAA|1999-10-28|2001-10-26|Direct records would not marry in a suggestions. External standards avoid nice services. Large secrets|2.76|1.84|3001002|amalgexporti #2|16|newborn|3|Children|952|ableantin st|medium|26lace02168211745104|smoke|Tsp|Unknown|16|prin stablen st| +9294|AAAAAAAAMEECAAAA|2001-10-27||Direct records would not marry in a suggestions. External standards avoid nice services. Large secrets|9.86|1.84|2004001|edu packimporto #1|16|sports-apparel|2|Men|952|ableantin st|extra large|26lace02168211745104|pale|Box|Unknown|1|esen stablen st| +9295|AAAAAAAAPEECAAAA|1997-10-27||Also right police invest|2.21|1.65|10003014|exportiunivamalg #14|3|dvd/vcr players|10|Electronics|58|einganti|N/A|8630tan9314510351871|puff|Ounce|Unknown|12|antin stablen st| +9296|AAAAAAAAAFECAAAA|1997-10-27|2000-10-26|Secondary, middle arms make social, light aims. So as mysterious police take final, other cards. Used ways can happen nearly different prices. Considerably financial priorities may harm solutions|26.35|22.66|8002007|importonameless #7|2|baseball|8|Sports|91|oughtn st|N/A|puff8754246333863981|seashell|Case|Unknown|35|callyn stablen st| +9297|AAAAAAAAAFECAAAA|2000-10-27||Adult children find. Soon main workers apply. A little immediate cont|2.28|1.55|8002007|corpcorp #2|6|rings|6|Jewelry|91|oughtn st|N/A|15570375wheat0961181|royal|Lb|Unknown|31|ationn stablen st| +9298|AAAAAAAACFECAAAA|1997-10-27|1999-10-27|As administ|8.83|3.97|2001001|amalgimporto #1|1|accessories|2|Men|157|ationantiought|large|1tomato1140006967440|sienna|Case|Unknown|7|eingn stablen st| +9299|AAAAAAAACFECAAAA|1999-10-28|2001-10-26|As administ|5.75|3.97|5002002|importoscholar #2|2|country|5|Music|450|barantiese|N/A|1tomato1140006967440|green|Gram|Unknown|34|n stn stablen st| +9300|AAAAAAAACFECAAAA|2001-10-27||As administ|0.20|0.14|6005005|scholarcorp #5|2|earings|6|Jewelry|450|barantiese|N/A|1tomato1140006967440|peach|Bundle|Unknown|34|barbarprin st| +9301|AAAAAAAAFFECAAAA|1997-10-27||Certain, conscious songs include inevitably for example previous candidates. Parties matter too at the others. Environmental, similar bits |7.38|2.50|5004002|edu packscholar #2|4|classical|5|Music|167|ationcallyought|N/A|055720spring05231930|powder|Dram|Unknown|5|oughtbarprin st| +9302|AAAAAAAAGFECAAAA|1997-10-27|2000-10-26|Ready direct|54.53|39.80|10001004|amalgunivamalg #4|1|cameras|10|Electronics|141|oughteseought|N/A|2791spring0417055804|peach|Unknown|Unknown|18|ablebarprin st| +9303|AAAAAAAAGFECAAAA|2000-10-27||Ready direct|0.86|39.80|10005005|scholarunivamalg #5|5|karoke|10|Electronics|126|oughteseought|N/A|2791spring0417055804|metallic|Gross|Unknown|3|pribarprin st| +9304|AAAAAAAAIFECAAAA|1997-10-27|1999-10-27|Heavily industrial problems intend thick able patients. Labour, wide patients take into the functions. International windows suggest hard t|8.52|6.81|5004001|edu packscholar #1|4|classical|5|Music|443|prieseese|N/A|4701sandy15355614028|steel|N/A|Unknown|2|esebarprin st| +9305|AAAAAAAAIFECAAAA|1999-10-28|2001-10-26|Heavily industrial problems intend thick able patients. Labour, wide patients take into the functions. International windows suggest hard t|1.83|0.62|5004001|importoexporti #2|4|infants|3|Children|387|ationeingpri|large|05738489636sienna680|moccasin|Bundle|Unknown|9|antibarprin st| +9306|AAAAAAAAIFECAAAA|2001-10-27||Only light reserves measure fine, top minds. Locations should think. Across open holidays would pay so women. Social, good practices may expand just fin|1.44|1.18|7003007|exportibrand #7|3|kids|7|Home|18|ationeingpri|N/A|05738489636sienna680|royal|Gross|Unknown|56|callybarprin st| +9307|AAAAAAAALFECAAAA|1997-10-27||Forces shall proceed to a days. More current police prepare. Total, polish expectations work substantial, excell|5.95|2.61|3004002|edu packexporti #2|4|school-uniforms|3|Children|276|callyationable|medium|4161purple9930644108|floral|Each|Unknown|32|ationbarprin st| +9308|AAAAAAAAMFECAAAA|1997-10-27|2000-10-26|Brief cases act. Favorite, personal elements promote therefore scores; now brave earnings think investigatio|1.20|0.51|2002001|importoimporto #1|2|shirts|2|Men|234|esepriable|extra large|917081646929smoke448|smoke|N/A|Unknown|37|eingbarprin st| +9309|AAAAAAAAMFECAAAA|2000-10-27||Eyes achieve ju|1.16|0.80|10001011|amalgunivamalg #11|2|cameras|10|Electronics|234|esepriable|N/A|97673549turquoise382|peach|Lb|Unknown|9|n stbarprin st| +9310|AAAAAAAAOFECAAAA|1997-10-27|1999-10-27|Later high interests |5.61|4.15|9015011|scholarunivamalg #11|15|fiction|9|Books|205|antibarable|N/A|8489779435burlywood7|turquoise|Bundle|Unknown|13|baroughtprin st| +9311|AAAAAAAAOFECAAAA|1999-10-28|2001-10-26|Later high interests |1.36|4.15|9015011|importoscholar #2|2|country|5|Music|205|antibarable|N/A|8489779435burlywood7|burnished|Pound|Unknown|36|oughtoughtprin st| +9312|AAAAAAAAOFECAAAA|2001-10-27||Later high interests |0.47|4.15|9015011|corpnameless #3|2|furniture|7|Home|205|antibarable|N/A|8489779435burlywood7|violet|Unknown|Unknown|45|ableoughtprin st| +9313|AAAAAAAABGECAAAA|1997-10-27||Dominant, small col|4.54|3.90|5001002|amalgscholar #2|1|rock|5|Music|196|callyn stought|N/A|2296thistle587976357|spring|Unknown|Unknown|62|prioughtprin st| +9314|AAAAAAAACGECAAAA|1997-10-27|2000-10-26|Certain countries produce then recent, foreign ends. Details shall claim at a figures. Effective days should see local interests. Outer courses mig|6.66|3.13|4002001|importoedu pack #1|2|mens|4|Shoes|121|oughtableought|large|2violet0728588247595|yellow|N/A|Unknown|77|eseoughtprin st| +9315|AAAAAAAACGECAAAA|2000-10-27||Certain countries produce then recent, foreign ends. Details shall claim at a figures. Effective days should see local interests. Outer courses mig|5.06|3.13|9004004|edu packmaxi #4|2|entertainments|9|Books|965|anticallyn st|N/A|190909738saddle46159|pale|Carton|Unknown|76|antioughtprin st| +9316|AAAAAAAAEGECAAAA|1997-10-27|1999-10-27|Hot products signal together big, working roads. Now funny universities|2.53|1.36|9010005|univunivamalg #5|10|travel|9|Books|42|ableese|N/A|68654blue62749541282|grey|Oz|Unknown|6|callyoughtprin st| +9317|AAAAAAAAEGECAAAA|1999-10-28|2001-10-26|Hot products signal together big, working roads. Now funny universities|5.49|2.03|9011006|amalgunivamalg #6|11|cooking|9|Books|410|baroughtese|N/A|68654blue62749541282|pale|Pallet|Unknown|13|ationoughtprin st| +9318|AAAAAAAAEGECAAAA|2001-10-27||Curious months like subtle, linguistic privileges. High, civil expressions make seemin|0.96|0.66|9011006|amalgimporto #1|11|accessories|2|Men|410|baroughtese|petite|9476575puff017898552|green|Bundle|Unknown|1|eingoughtprin st| +9319|AAAAAAAAHGECAAAA|1997-10-27||Real, positive members could come general leaves. Open affairs change most red universities. Outside sales will not recover accurately other members. Inland, specific members sit more painful f|4.75|3.18|2002002|importoimporto #2|2|shirts|2|Men|228|eingableable|extra large|787149blush488894432|red|Ton|Unknown|4|n stoughtprin st| +9320|AAAAAAAAIGECAAAA|1997-10-27|2000-10-26|Impressive plans sleep previously br|9.57|5.83|6011001|amalgbrand #1|11|semi-precious|6|Jewelry|319|n stoughtpri|N/A|53555royal8008636687|white|Tsp|Unknown|28|barableprin st| +9321|AAAAAAAAIGECAAAA|2000-10-27||Related times distinguish simultaneously operational presents. More victorian d|2.08|1.04|6011001|corpamalgamalg #6|11|wireless|10|Electronics|319|n stoughtpri|N/A|53555royal8008636687|wheat|Tsp|Unknown|12|oughtableprin st| +9322|AAAAAAAAKGECAAAA|1997-10-27|1999-10-27|Only problems unite most by a feet; major problems can get partly once very mice. As other parties look different twins.|54.71|48.69|1001001|amalgamalg #1|1|dresses|1|Women|630|barprically|petite|8lace217870053231428|light|Case|Unknown|21|ableableprin st| +9323|AAAAAAAAKGECAAAA|1999-10-28|2001-10-26|Local years lie very in a books. Full, national banks will not believe. Sure, diplomatic pi|0.66|48.69|1001001|edu packexporti #2|4|school-uniforms|3|Children|630|barprically|economy|4262337381floral0397|gainsboro|N/A|Unknown|19|priableprin st| +9324|AAAAAAAAKGECAAAA|2001-10-27||Local years lie very in a books. Full, national banks will not believe. Sure, diplomatic pi|3.06|48.69|1001001|edu packbrand #5|14|estate|6|Jewelry|630|barprically|N/A|4262337381floral0397|red|Box|Unknown|12|eseableprin st| +9325|AAAAAAAANGECAAAA|1997-10-27||Well large movements shall show in the pressures. Responsible, physical bottles could play economic, relative parts. Desperately permanent relationships might not stay |2.84|1.44|2004002|edu packimporto #2|4|sports-apparel|2|Men|138|eingpriought|petite|081yellow69327299123|peru|Ounce|Unknown|6|antiableprin st| +9326|AAAAAAAAOGECAAAA|1997-10-27|2000-10-26|Much busy situations develop. Together long events add in a members. Legal parts answer. Far painful limits offer fully major floors. All but possible years can get on a modes. Distinctive journalists|2.84|2.47|10013011|exportiamalgamalg #11|13|stereo|10|Electronics|86|callyeing|N/A|4268376009341misty55|burnished|Carton|Unknown|17|callyableprin st| +9327|AAAAAAAAOGECAAAA|2000-10-27||Much busy situations develop. Together long events add in a members. Legal parts answer. Far painful limits offer fully major floors. All but possible years can get on a modes. Distinctive journalists|0.25|0.21|10013011|importoedu pack #2|2|mens|4|Shoes|39|n stpri|medium|908midnight932178063|spring|Bunch|Unknown|37|ationableprin st| +9328|AAAAAAAAAHECAAAA|1997-10-27|1999-10-27|Regional le|5.05|4.49|5004001|edu packscholar #1|4|classical|5|Music|581|oughteinganti|N/A|7purple4027434983275|rose|Gross|Unknown|53|eingableprin st| +9329|AAAAAAAAAHECAAAA|1999-10-28|2001-10-26|Tears involve on a cars. Alone|0.30|0.10|10012014|importoamalgamalg #14|4|monitors|10|Electronics|581|oughteinganti|N/A|honeydew286136740395|linen|Ounce|Unknown|15|n stableprin st| +9330|AAAAAAAAAHECAAAA|2001-10-27||Tears involve on a cars. Alone|6.85|4.79|7015001|scholarnameless #1|15|tables|7|Home|88|eingeing|N/A|85725391003210rose09|purple|Each|Unknown|13|barpriprin st| +9331|AAAAAAAADHECAAAA|1997-10-27||Apparent, beneficial years should pay hard arrangements. Impatiently positive cities take undoubtedly so noble centuries; old ideas must afford early drivers. Li|2.47|1.40|6010004|univbrand #4|10|jewelry boxes|6|Jewelry|661|oughtcallycally|N/A|088218ghost089673395|spring|Oz|Unknown|5|oughtpriprin st| +9332|AAAAAAAAEHECAAAA|1997-10-27|2000-10-26|Environmental, academic disciplines |1.10|0.63|4003001|exportiedu pack #1|3|kids|4|Shoes|477|ationationese|medium|5347320544779peru532|tomato|Ounce|Unknown|54|ablepriprin st| +9333|AAAAAAAAEHECAAAA|2000-10-27||British, physical books could not beat a little labour, attractive parents; remote shareholders give rural, judicial sessions. Performances may question as strictly|6.76|0.63|4003001|edu packimporto #2|4|sports-apparel|2|Men|276|ationationese|medium|5347320544779peru532|royal|Pallet|Unknown|31|pripriprin st| +9334|AAAAAAAAGHECAAAA|1997-10-27|1999-10-27|Changes set even on a subsidies. Exactly severe soldiers must not prevent now then free h|7.85|5.33|9014011|edu packunivamalg #11|14|sports|9|Books|78|eingation|N/A|53704yellow039207111|linen|Carton|Unknown|19|esepriprin st| +9335|AAAAAAAAGHECAAAA|1999-10-28|2001-10-26|Changes set even on a subsidies. Exactly severe soldiers must not prevent now then free h|1.16|5.33|3004002|edu packexporti #2|4|school-uniforms|3|Children|318|eingation|small|547607585turquoise90|lace|Unknown|Unknown|21|antipriprin st| +9336|AAAAAAAAGHECAAAA|2001-10-27||Changes set even on a subsidies. Exactly severe soldiers must not prevent now then free h|8.53|5.03|7016007|corpnameless #7|16|furniture|7|Home|377|eingation|N/A|547607585turquoise90|snow|Gram|Unknown|30|callypriprin st| +9337|AAAAAAAAJHECAAAA|1997-10-27||Clean, critical journalists will go. Then english others would te|1.21|1.06|1001002|amalgamalg #2|1|dresses|1|Women|258|eingantiable|petite|90brown5552150480698|grey|Pallet|Unknown|12|ationpriprin st| +9338|AAAAAAAAKHECAAAA|1997-10-27|2000-10-26|Only, subsequent minerals should exist just f|4.69|2.90|7014009|edu packnameless #9|14|glassware|7|Home|176|callyationought|N/A|9723351438smoke21874|purple|Pallet|Unknown|67|eingpriprin st| +9339|AAAAAAAAKHECAAAA|2000-10-27||Only, subsequent minerals should exist just f|2.69|2.90|7014009|namelesscorp #4|8|mens watch|6|Jewelry|818|eingoughteing|N/A|9723351438smoke21874|red|Each|Unknown|46|n stpriprin st| +9340|AAAAAAAAMHECAAAA|1997-10-27|1999-10-27|Miles operate usually impossible troubles. Well other banks protect just very long-term changes. Content months might convince somewhat compara|0.23|0.08|4002001|importoedu pack #1|2|mens|4|Shoes|642|ableesecally|medium|1300honeydew94036612|rosy|Each|Unknown|19|bareseprin st| +9341|AAAAAAAAMHECAAAA|1999-10-28|2001-10-26|Miles operate usually impossible troubles. Well other banks protect just very long-term changes. Content months might convince somewhat compara|1.86|0.87|4002001|edu packamalg #2|4|swimwear|1|Women|7|ation|large|1300honeydew94036612|powder|Dozen|Unknown|29|oughteseprin st| +9342|AAAAAAAAMHECAAAA|2001-10-27||Miles operate usually impossible troubles. Well other banks protect just very long-term changes. Content months might convince somewhat compara|5.48|0.87|3002001|importoexporti #1|2|infants|3|Children|988|eingeingn st|large|032693rose1896735415|linen|Dozen|Unknown|92|ableeseprin st| +9343|AAAAAAAAPHECAAAA|1997-10-27||Sides must complain never per a clergy; economies should put necessarily; long fixed animals might raise instead wrong structural legs; ru|1.14|0.96|5004002|edu packscholar #2|4|classical|5|Music|297|ationn stable|N/A|9410095peach64095187|ghost|Dram|Unknown|31|prieseprin st| +9344|AAAAAAAAAIECAAAA|1997-10-27|2000-10-26|Lovers should not afford good, technological gains; distinctive cats might not find ambitious, different|1.10|0.46|6016001|corpbrand #1|16|consignment|6|Jewelry|363|pricallypri|N/A|22514987445wheat4381|spring|Ounce|Unknown|51|eseeseprin st| +9345|AAAAAAAAAIECAAAA|2000-10-27||Lovers should not afford good, technological gains; distinctive cats might not find ambitious, different|0.15|0.07|9002010|importomaxi #10|16|business|9|Books|269|n stcallyable|N/A|40413snow24000618120|salmon|Gram|Unknown|25|antieseprin st| +9346|AAAAAAAACIECAAAA|1997-10-27|1999-10-27|Just professional facilit|8.12|5.11|8010007|univmaxi #7|10|pools|8|Sports|127|ationableought|N/A|05036violet372868330|chiffon|Ton|Unknown|23|callyeseprin st| +9347|AAAAAAAACIECAAAA|1999-10-28|2001-10-26|High stages grant for example sounds. Good lakes should believe japanese, distinct buildings; services serve. Rare, european workers help ago short departments. Aga|64.39|39.27|10014015|edu packamalgamalg #15|10|automotive|10|Electronics|127|ationableought|N/A|05036violet372868330|midnight|Case|Unknown|40|ationeseprin st| +9348|AAAAAAAACIECAAAA|2001-10-27||High stages grant for example sounds. Good lakes should believe japanese, distinct buildings; services serve. Rare, european workers help ago short departments. Aga|28.11|39.27|2002001|importoimporto #1|2|shirts|2|Men|679|ationableought|petite|427747202871hot30751|seashell|Unknown|Unknown|22|eingeseprin st| +9349|AAAAAAAAFIECAAAA|1997-10-27||No longer statistical months should find well basic, american bu|7.24|5.28|6011006|amalgbrand #6|11|semi-precious|6|Jewelry|71|oughtation|N/A|272chartreuse0073156|lemon|Ounce|Unknown|9|n steseprin st| +9350|AAAAAAAAGIECAAAA|1997-10-27|2000-10-26|Relations bowl very by a years. Odd, christian days may not describe easy children. Recent neighbours run however police. Full, british eyes |9.83|5.50|10011001|amalgamalgamalg #1|11|disk drives|10|Electronics|698|eingn stcally|N/A|701043salmon30714822|hot|Bundle|Unknown|84|barantiprin st| +9351|AAAAAAAAGIECAAAA|2000-10-27||Relations bowl very by a years. Odd, christian days may not describe easy children. Recent neighbours run however police. Full, british eyes |4.40|5.50|10011001|amalgscholar #2|1|rock|5|Music|698|eingn stcally|N/A|574970violet03134343|tan|Dram|Unknown|27|oughtantiprin st| +9352|AAAAAAAAIIECAAAA|1997-10-27|1999-10-27|Green patients will tell impossible skills. Seconds might write sadly ove|1.51|0.48|7008003|namelessbrand #3|8|lighting|7|Home|300|barbarpri|N/A|63389319saddle558014|seashell|Pound|Unknown|75|ableantiprin st| +9353|AAAAAAAAIIECAAAA|1999-10-28|2001-10-26|New banks condemn. Questions pay home twice independent heads; olympic clients will not protest here like a others; fundamental attitudes claim especia|0.52|0.48|5004002|edu packscholar #2|4|classical|5|Music|300|barbarpri|N/A|965084116smoke857990|indian|Pound|Unknown|3|priantiprin st| +9354|AAAAAAAAIIECAAAA|2001-10-27||Other, only cells remove|4.47|2.72|5004002|exportischolar #1|4|pop|5|Music|1000|barbarbarought|N/A|965084116smoke857990|moccasin|Box|Unknown|60|eseantiprin st| +9355|AAAAAAAALIECAAAA|1997-10-27||Huma|1.55|0.65|2002002|importoimporto #2|2|shirts|2|Men|729|n stableation|petite|312156494989peach629|sky|Box|Unknown|32|antiantiprin st| +9356|AAAAAAAAMIECAAAA|1997-10-27|2000-10-26|Le|9.98|5.98|7010007|univnameless #7|10|flatware|7|Home|317|ationoughtpri|N/A|413460942saddle02772|peach|Each|Unknown|47|callyantiprin st| +9357|AAAAAAAAMIECAAAA|2000-10-27||Key, current levels used to find usually intelligent, |8.47|5.59|5004002|edu packscholar #2|10|classical|5|Music|317|ationoughtpri|N/A|413460942saddle02772|indian|Cup|Unknown|38|ationantiprin st| +9358|AAAAAAAAOIECAAAA|1997-10-27|1999-10-27|Historians move actually religious shops. Physical members ought to go difficult children. Added, successful studies form only. High, different pubs fit before in the |5.87|3.28|8009009|maxinameless #9|9|optics|8|Sports|832|ableprieing|N/A|734265046sandy215905|goldenrod|N/A|Unknown|14|eingantiprin st| +9359|AAAAAAAAOIECAAAA|1999-10-28|2001-10-26|Historians move actually religious shops. Physical members ought to go difficult children. Added, successful studies form only. High, different pubs fit before in the |68.95|3.28|7010010|univnameless #10|9|flatware|7|Home|832|ableprieing|N/A|734265046sandy215905|powder|Case|Unknown|41|n stantiprin st| +9360|AAAAAAAAOIECAAAA|2001-10-27||Nice new|4.31|3.87|7010010|importoamalgamalg #11|12|monitors|10|Electronics|832|ableprieing|N/A|734265046sandy215905|plum|Tsp|Unknown|86|barcallyprin st| +9361|AAAAAAAABJECAAAA|1997-10-27||Men could refer regularly churches. Original parts could return international, general members. Professional others could become. Big versions ought to mean. Mutual collections hea|7.12|3.63|4004002|edu packedu pack #2|4|athletic|4|Shoes|51|oughtanti|large|926907turquoise57389|pink|Case|Unknown|1|oughtcallyprin st| +9362|AAAAAAAACJECAAAA|1997-10-27|2000-10-26|Other conditions m|35.25|12.33|8003003|exportinameless #3|3|basketball|8|Sports|344|eseesepri|N/A|12879peru44608138609|midnight|Tsp|Unknown|5|ablecallyprin st| +9363|AAAAAAAACJECAAAA|2000-10-27||Yet judicial communications can secure especially full, various processes. Now small actions shall not suffer thus patients; pupils should dispense packages; domestic pri|2.44|12.33|8003003|amalgbrand #4|1|bathroom|7|Home|830|eseesepri|N/A|67279515337rosy61928|yellow|Tsp|Unknown|25|pricallyprin st| +9364|AAAAAAAAEJECAAAA|1997-10-27|1999-10-27|Inland, black horses could not feel partly. Years might use however; personal papers should supply particularly experienced issues. Years could go also tired, |54.17|35.75|10016015|corpamalgamalg #15|16|wireless|10|Electronics|266|callycallyable|N/A|90460113570pale23817|moccasin|Dozen|Unknown|13|esecallyprin st| +9365|AAAAAAAAEJECAAAA|1999-10-28|2001-10-26|However national members take very possible, hot prices. |4.24|1.65|9014006|edu packunivamalg #6|14|sports|9|Books|178|callycallyable|N/A|170medium56551896062|royal|Box|Unknown|91|anticallyprin st| +9366|AAAAAAAAEJECAAAA|2001-10-27||Issues detect. So christian times shall move apart from a pictures; yesterday apparent roles should let still foreigners. Years study hastily f|8.18|1.65|9014006|edu packexporti #1|14|school-uniforms|3|Children|178|eingationought|medium|34460tomato337780411|powder|Ounce|Unknown|4|callycallyprin st| +9367|AAAAAAAAHJECAAAA|1997-10-27||Worried, other programmes come. Terms announce above a contributions. Wrong surveys might pay from the requirements. Traditional interests will enter of course surprised, separate connections. Ag|52.94|40.76|6009008|maxicorp #8|9|womens watch|6|Jewelry|352|ableantipri|N/A|550520377832red27806|azure|Tsp|Unknown|19|ationcallyprin st| +9368|AAAAAAAAIJECAAAA|1997-10-27|2000-10-26|Difficult, royal units put particularly significant, other plans. Essential, contemporary journals will need players. Alternatively parental|4.34|3.25|9002009|importomaxi #9|2|business|9|Books|365|anticallypri|N/A|73620violet501543846|pink|Gram|Unknown|4|eingcallyprin st| +9369|AAAAAAAAIJECAAAA|2000-10-27||Suggestions pay. Possible statements must not put early prices; players command also commercial, lovely pr|4.93|3.25|9002009|importoedu pack #2|2|mens|4|Shoes|185|antieingought|N/A|73620violet501543846|cream|Bundle|Unknown|9|n stcallyprin st| +9370|AAAAAAAAKJECAAAA|1997-10-27|1999-10-27|Stud|3.37|1.95|9008011|namelessmaxi #11|8|romance|9|Books|492|ablen stese|N/A|19391996red373486880|gainsboro|Box|Unknown|23|barationprin st| +9371|AAAAAAAAKJECAAAA|1999-10-28|2001-10-26|Stud|0.31|1.95|1002002|importoamalg #2|2|fragrances|1|Women|324|eseablepri|medium|19391996red373486880|spring|Bunch|Unknown|25|oughtationprin st| +9372|AAAAAAAAKJECAAAA|2001-10-27||Students must indicate large societies. Social workers ought to see earlier developments; se|8.43|3.87|8003003|exportinameless #3|2|basketball|8|Sports|249|n steseable|N/A|violet87935735919806|orange|Ounce|Unknown|18|ableationprin st| +9373|AAAAAAAANJECAAAA|1997-10-27||Previous, critical proposals come of course bad parents; currently public thousands must not build at a girls; honest, elderly employ|2.49|1.61|5002002|importoscholar #2|2|country|5|Music|650|barantically|N/A|279795727spring64929|black|Dram|Unknown|33|priationprin st| +9374|AAAAAAAAOJECAAAA|1997-10-27|2000-10-26|British regulations will block at all improvements; visual, managerial assumptions should examine in a fears. Effects become sensitive firms|9.88|3.45|7011005|amalgnameless #5|11|accent|7|Home|814|eseoughteing|N/A|57white8663088993586|orchid|Ounce|Unknown|87|eseationprin st| +9375|AAAAAAAAOJECAAAA|2000-10-27||British regulations will block at all improvements; visual, managerial assumptions should examine in a fears. Effects become sensitive firms|2.17|0.97|9011004|amalgunivamalg #4|11|cooking|9|Books|730|eseoughteing|N/A|57white8663088993586|plum|Each|Unknown|8|antiationprin st| +9376|AAAAAAAAAKECAAAA|1997-10-27|1999-10-27|Importantly differen|7.92|3.08|8014007|edu packmaxi #7|14|tennis|8|Sports|64|esecally|N/A|88254524706007peach7|thistle|Case|Unknown|49|callyationprin st| +9377|AAAAAAAAAKECAAAA|1999-10-28|2001-10-26|Comm|2.16|3.08|8014007|amalgexporti #2|14|newborn|3|Children|64|esecally|extra large|61404570704white6990|rosy|Bunch|Unknown|65|ationationprin st| +9378|AAAAAAAAAKECAAAA|2001-10-27||Somewhat educational ideas mend just to a vessels. Romantic tonnes would keep. Really major committees might not lose much separate events. Re|3.12|1.40|8014007|edu packbrand #9|4|curtains/drapes|7|Home|64|esecally|N/A|rosy4974707119134458|lavender|Tsp|Unknown|57|eingationprin st| +9379|AAAAAAAADKECAAAA|1997-10-27||Industrial students run communities. Home old differences change soon. There new tale|4.05|1.66|7012010|importonameless #10|12|paint|7|Home|618|eingoughtcally|N/A|872208saddle67297479|steel|Pallet|Unknown|35|n stationprin st| +9380|AAAAAAAAEKECAAAA|1997-10-27|2000-10-26|Improvements must not reduce therefore in a friend|1.89|1.51|4001001|amalgedu pack #1|1|womens|4|Shoes|197|ationn stought|medium|9468655992purple7280|mint|Dram|Unknown|19|bareingprin st| +9381|AAAAAAAAEKECAAAA|2000-10-27||Deaf arts help other, whole degrees. Expenses ought to house both. Young, direct orders might let then out of a hands; constant propos|28.50|1.51|5002002|importoscholar #2|1|country|5|Music|411|oughtoughtese|N/A|47goldenrod419260690|thistle|Dozen|Unknown|20|oughteingprin st| +9382|AAAAAAAAGKECAAAA|1997-10-27|1999-10-27|||1.51|||15||||563||||pale||||| +9383|AAAAAAAAGKECAAAA|1999-10-28|2001-10-26|British days might think just british, successful clothes; conditions draw. Apparent rates clear in a rates; only, serious schemes lose no young eyes. Common, extreme lines can test even political, |0.93|1.51|4003002|exportiedu pack #2|3|kids|4|Shoes|145|pricallyanti|extra large|18989587288smoke7583|firebrick|Tsp|Unknown|13|prieingprin st| +9384|AAAAAAAAGKECAAAA|2001-10-27||British days might think just british, successful clothes; conditions draw. Apparent rates clear in a rates; only, serious schemes lose no young eyes. Common, extreme lines can test even political, |5.92|1.51|8014005|edu packmaxi #5|14|tennis|8|Sports|145|antieseought|N/A|9rose597339641425887|saddle|Gram|Unknown|17|eseeingprin st| +9385|AAAAAAAAJKECAAAA|1997-10-27||Too economic materials sell forms. Tired indians continue always with the problems. Great plants avoid overwhelmingly resulting companies. Bones explore even straigh|12.47|10.47|10009002|maxiunivamalg #2|9|televisions|10|Electronics|448|eingeseese|N/A|95538474turquoise746|snow|Bundle|Unknown|14|antieingprin st| +9386|AAAAAAAAKKECAAAA|1997-10-27|2000-10-26|Methods detect significantly vast, federal considerations. Only full hours find facilitie|31.61|25.28|6009003|maxicorp #3|9|womens watch|6|Jewelry|283|prieingable|N/A|papaya52937497174774|pink|Carton|Unknown|3|callyeingprin st| +9387|AAAAAAAAKKECAAAA|2000-10-27||Different, new players accept problem|3.11|25.28|6009003|importoscholar #2|9|country|5|Music|2|able|N/A|papaya52937497174774|thistle|Carton|Unknown|17|ationeingprin st| +9388|AAAAAAAAMKECAAAA|1997-10-27|1999-10-27|Languages want as with a offenders. Common, damp experts will gain cases; at first long years would remind later recently old decades. Simple, regional customers shall fi|0.55|0.25|8010009|univmaxi #9|10|pools|8|Sports|80|bareing|N/A|31turquoise401854548|orange|Gram|Unknown|17|eingeingprin st| +9389|AAAAAAAAMKECAAAA|1999-10-28|2001-10-26|Languages want as with a offenders. Common, damp experts will gain cases; at first long years would remind later recently old decades. Simple, regional customers shall fi|68.59|45.95|8010009|importobrand #10|10|bedding|7|Home|564|bareing|N/A|31turquoise401854548|moccasin|Case|Unknown|33|n steingprin st| +9390|AAAAAAAAMKECAAAA|2001-10-27||Languages want as with a offenders. Common, damp experts will gain cases; at first long years would remind later recently old decades. Simple, regional customers shall fi|3.00|2.40|3004001|edu packexporti #1|4|school-uniforms|3|Children|45|bareing|large|5sky2885812721258643|puff|Bundle|Unknown|1|barn stprin st| +9391|AAAAAAAAPKECAAAA|1997-10-27||Other, light walls live obviously. Too clean friends add mixed, basic powers. Primary years mig|4.20|1.72|10013008|exportiamalgamalg #8|13|stereo|10|Electronics|97|ationn st|N/A|60808719264114pale93|snow|Gross|Unknown|2|oughtn stprin st| +9392|AAAAAAAAALECAAAA|1997-10-27|2000-10-26|Revolutionary investors will not consider often black questions; lines want probably contemp|1.19|0.41|8016003|corpmaxi #3|16|golf|8|Sports|830|barprieing|N/A|49461702yellow119277|rosy|N/A|Unknown|2|ablen stprin st| +9393|AAAAAAAAALECAAAA|2000-10-27||Services ought to send to a teeth. General instructions see too. Both full years co-operate outside a forces. Daily, industrial profits increase today|9.88|0.41|7006004|corpbrand #4|16|rugs|7|Home|830|barprieing|N/A|beige051790784858659|dark|Gross|Unknown|44|prin stprin st| +9394|AAAAAAAACLECAAAA|1997-10-27|1999-10-27|Please white buildings support very wrong, swiss gifts. Young, other shoes print relatively joint, good m|2.52|1.58|3003001|exportiexporti #1|3|toddlers|3|Children|75|antiation|petite|09032773smoke6187545|slate|N/A|Unknown|26|esen stprin st| +9395|AAAAAAAACLECAAAA|1999-10-28|2001-10-26|Please white buildings support very wrong, swiss gifts. Young, other shoes print relatively joint, good m|8.42|6.90|3003001|edu packmaxi #6|14|tennis|8|Sports|231|antiation|N/A|09032773smoke6187545|sienna|Gross|Unknown|33|antin stprin st| +9396|AAAAAAAACLECAAAA|2001-10-27||Please white buildings support very wrong, swiss gifts. Young, other shoes print relatively joint, good m|0.15|0.09|5004001|edu packscholar #1|14|classical|5|Music|28|antiation|N/A|09032773smoke6187545|royal|Gross|Unknown|88|callyn stprin st| +9397|AAAAAAAAFLECAAAA|1997-10-27||Major police ought to evaluate often funds. Large resources know for a copies. Changes adopt hi|1.08|0.45|5002002|importoscholar #2|2|country|5|Music|37|ationpri|N/A|49slate3239756864179|slate|Ton|Unknown|27|ationn stprin st| +9398|AAAAAAAAGLECAAAA|1997-10-27|2000-10-26|Damp towns find as modern, different y|7.18|4.73|8003009|exportinameless #9|3|basketball|8|Sports|53|prianti|N/A|4615740595light02952|yellow|Cup|Unknown|32|eingn stprin st| +9399|AAAAAAAAGLECAAAA|2000-10-27||Damp towns find as modern, different y|7.27|2.90|8005010|scholarnameless #10|5|fitness|8|Sports|274|eseationable|N/A|4615740595light02952|peru|Carton|Unknown|5|n stn stprin st| +9400|AAAAAAAAILECAAAA|1997-10-27|1999-10-27|Minor, keen yards serve usually at a elements. Assump|1.63|1.20|10002017|importounivamalg #17|2|camcorders|10|Electronics|196|callyn stought|N/A|92161800535346peach9|salmon|Gram|Unknown|5|barbaresen st| +9401|AAAAAAAAILECAAAA|1999-10-28|2001-10-26|Rough properties find meanwhile different years. Statutory policies m|22.07|1.20|7002002|importobrand #2|2|bedding|7|Home|196|callyn stought|N/A|43274975107violet244|lavender|Dozen|Unknown|53|oughtbaresen st| +9402|AAAAAAAAILECAAAA|2001-10-27||Thin conditions will give soviet words. Determined, recent cases might marry previously as adult parts. Educational parties shall not receive into a policies. Significant years go over full-|6.20|5.45|1004001|edu packamalg #1|4|swimwear|1|Women|196|callyn stought|extra large|1350593811232sandy36|violet|N/A|Unknown|47|ablebaresen st| +9403|AAAAAAAALLECAAAA|1997-10-27||Unemplo|1.65|0.82|10003003|exportiunivamalg #3|3|dvd/vcr players|10|Electronics|25|antiable|N/A|5819254589powder7510|white|Bunch|Unknown|27|pribaresen st| +9404|AAAAAAAAMLECAAAA|1997-10-27|2000-10-26|Lines identify perhaps. Red teams must kill interviews. Necessary, great schools connect so empty, visible steps. Other crowds must insist passages; |4.06|2.88|4002001|importoedu pack #1|2|mens|4|Shoes|604|esebarcally|petite|72050622695navajo410|navajo|Ton|Unknown|68|esebaresen st| +9405|AAAAAAAAMLECAAAA|2000-10-27||Lines identify perhaps. Red teams must kill interviews. Necessary, great schools connect so empty, visible steps. Other crowds must insist passages; |4.04|1.57|3002002|importoexporti #2|2|infants|3|Children|346|esebarcally|medium|7958397922pale503717|aquamarine|Dram|Unknown|5|antibaresen st| +9406|AAAAAAAAOLECAAAA|1997-10-27|1999-10-27|Forces should like generally military lights. Existing, perfect males could prepare much original photographs. Cases increase below entire polic|0.56|0.49|10001006|amalgunivamalg #6|1|cameras|10|Electronics|77|ationation|N/A|37094874turquoise107|sandy|Tsp|Unknown|78|callybaresen st| +9407|AAAAAAAAOLECAAAA|1999-10-28|2001-10-26|Sectors seek never environmental, necessary seats. Blue, medium forms will not sleep officers. Never abov|0.10|0.07|10001006|edu packmaxi #12|1|entertainments|9|Books|77|ationation|N/A|77saddle681289117837|lime|Case|Unknown|13|ationbaresen st| +9408|AAAAAAAAOLECAAAA|2001-10-27||Sectors seek never environmental, necessary seats. Blue, medium forms will not sleep officers. Never abov|0.53|0.07|6007001|brandcorp #1|7|pendants|6|Jewelry|227|ationableable|N/A|810311hot27818220500|tan|Cup|Unknown|72|eingbaresen st| +9409|AAAAAAAABMECAAAA|1997-10-27||Over sexual activities should not distinguish so. Really large goals provide to a attitudes; already free arms used to accept even for a days. Black, video-taped names may present both to the|9.14|7.86|8016010|corpmaxi #10|16|golf|8|Sports|433|pripriese|N/A|18044168302678rose47|spring|Dram|Unknown|21|n stbaresen st| +9410|AAAAAAAACMECAAAA|1997-10-27|2000-10-26|Basic, major months may slow to the reasons. Popular jobs shall not give traditional sections. Remote, inevitable functions must beco|4.16|3.16|6004003|edu packcorp #3|4|bracelets|6|Jewelry|179|n stationought|N/A|0pink325280654189533|midnight|Lb|Unknown|48|baroughtesen st| +9411|AAAAAAAACMECAAAA|2000-10-27||Basic, major months may slow to the reasons. Popular jobs shall not give traditional sections. Remote, inevitable functions must beco|3.04|3.16|6004003|exportinameless #2|13|wallpaper|7|Home|179|n stationought|N/A|0pink325280654189533|indian|Gross|Unknown|92|oughtoughtesen st| +9412|AAAAAAAAEMECAAAA|1997-10-27|1999-10-27|Similar, other causes used to want distinctive, green members. Dangers can participate expectations. Particular limits project organi|7.03|6.11|6015005|scholarbrand #5|15|custom|6|Jewelry|329|n stablepri|N/A|4843turquoise8573622|smoke|N/A|Unknown|83|ableoughtesen st| +9413|AAAAAAAAEMECAAAA|1999-10-28|2001-10-26|Similar, other causes used to want distinctive, green members. Dangers can participate expectations. Particular limits project organi|4.16|6.11|6015005|scholarunivamalg #6|15|fiction|9|Books|329|n stablepri|N/A|4843turquoise8573622|turquoise|Lb|Unknown|28|prioughtesen st| +9414|AAAAAAAAEMECAAAA|2001-10-27||Late parliamentary months may learn also to the functions. Less deaf eyes must provide exactly kilometres. Participants should not mind |2.66|1.91|2002001|importoimporto #1|15|shirts|2|Men|65|n stablepri|large|9296958868675slate50|wheat|Bunch|Unknown|8|eseoughtesen st| +9415|AAAAAAAAHMECAAAA|1997-10-27||Active, |2.51|1.00|1003002|exportiamalg #2|3|maternity|1|Women|90|barn st|medium|52pink01394559784408|smoke|Tbl|Unknown|87|antioughtesen st| +9416|AAAAAAAAIMECAAAA|1997-10-27|2000-10-26|Social eyes hear. Important, other fields say ago small, desirable inco|0.70|0.57|8006007|corpnameless #7|6|football|8|Sports|19|n stought|N/A|49178800tan889760538|steel|Ounce|Unknown|61|callyoughtesen st| +9417|AAAAAAAAIMECAAAA|2000-10-27||Fresh, genuine scenes shall feed. Stores shall not work just prob|9.86|6.80|7013008|exportinameless #8|13|wallpaper|7|Home|118|eingoughtought|N/A|0225028665854medium5|smoke|Bunch|Unknown|81|ationoughtesen st| +9418|AAAAAAAAKMECAAAA|1997-10-27|1999-10-27|Years ought to change perhaps neighbouring issues. Cultures regard most pot|6.35|2.98|10015001|scholaramalgamalg #1|15|portable|10|Electronics|580|bareinganti|N/A|20088031rose06151159|snow|Bundle|Unknown|32|eingoughtesen st| +9419|AAAAAAAAKMECAAAA|1999-10-28|2001-10-26|Years ought to change perhaps neighbouring issues. Cultures regard most pot|2.00|1.36|7006010|corpbrand #10|6|rugs|7|Home|580|bareinganti|N/A|45midnight4016801586|rose|Dram|Unknown|25|n stoughtesen st| +9420|AAAAAAAAKMECAAAA|2001-10-27||Instead annual con|3.53|1.16|7006010|brandnameless #1|7|hockey|8|Sports|927|bareinganti|N/A|348530356slate434666|powder|Lb|Unknown|5|barableesen st| +9421|AAAAAAAANMECAAAA|1997-10-27||Main arrangements support red, chief othe|1.91|1.08|3003002|exportiexporti #2|3|toddlers|3|Children|807|ationbareing|medium|7690787peru250071057|yellow|Pallet|Unknown|1|oughtableesen st| +9422|AAAAAAAAOMECAAAA|1997-10-27|2000-10-26|Romantic events should not seem still constitutional, nervous savings. United conditions would not react |2.65|1.40|3002001|importoexporti #1|2|infants|3|Children|580|bareinganti|economy|403528peru4458354927|white|Case|Unknown|27|ableableesen st| +9423|AAAAAAAAOMECAAAA|2000-10-27||Romantic events should not seem still constitutional, nervous savings. United conditions would not react |2.57|1.40|5002002|importoscholar #2|2|country|5|Music|580|bareinganti|N/A|steel369145117758364|tan|Gross|Unknown|64|priableesen st| +9424|AAAAAAAAANECAAAA|1997-10-27|1999-10-27|Conventional, responsible products discuss delicately then actual findings. Extremel|3.67|2.82|8008005|namelessnameless #5|8|outdoor|8|Sports|65|antically|N/A|10206white5516384563|pale|Oz|Unknown|2|eseableesen st| +9425|AAAAAAAAANECAAAA|1999-10-28|2001-10-26|Othe|4.99|2.09|2004002|edu packimporto #2|4|sports-apparel|2|Men|65|antically|medium|927thistle8070567649|lemon|Tbl|Unknown|6|antiableesen st| +9426|AAAAAAAAANECAAAA|2001-10-27||American, additional services should not suppose only. A little unknown authorities offer in a forces. Bad buyers cause frequent, empty sugg|52.48|42.50|7005007|scholarbrand #7|4|blinds/shades|7|Home|488|eingeingese|N/A|927thistle8070567649|ivory|Tsp|Unknown|9|callyableesen st| +9427|AAAAAAAADNECAAAA|1997-10-27||National computers learn documents; things ma|0.57|0.20|5003002|exportischolar #2|3|pop|5|Music|634|eseprically|N/A|005575580959olive113|rosy|Gram|Unknown|43|ationableesen st| +9428|AAAAAAAAENECAAAA|1997-10-27|2000-10-26|Machines predict. Severe engineers must resist just; dead sales should see only different characters. Military, colonial grounds say in order close companies. Small governments may assume seri|9.93|7.24|1001001|amalgamalg #1|1|dresses|1|Women|269|n stcallyable|medium|237601powder41310996|spring|Bunch|Unknown|2|eingableesen st| +9429|AAAAAAAAENECAAAA|2000-10-27||Examples move also women. Young costs could abandon ever terms. Positions may not hurt voluntar|4.10|7.24|9003010|exportimaxi #10|3|computers|9|Books|269|n stcallyable|N/A|237601powder41310996|plum|Bundle|Unknown|15|n stableesen st| +9430|AAAAAAAAGNECAAAA|1997-10-27|1999-10-27|Clear, thick times lead completely political occupation|8.62|3.44|1001001|amalgamalg #1|1|dresses|1|Women|339|n stpripri|medium|6steel39247084600119|lace|Oz|Unknown|42|barpriesen st| +9431|AAAAAAAAGNECAAAA|1999-10-28|2001-10-26|Clear, thick times lead completely political occupation|2.55|1.24|1001001|edu packedu pack #2|1|athletic|4|Shoes|339|n stpripri|medium|6steel39247084600119|saddle|N/A|Unknown|17|oughtpriesen st| +9432|AAAAAAAAGNECAAAA|2001-10-27||Clear, thick times lead completely political occupation|13.97|1.24|1003001|exportiamalg #1|3|maternity|1|Women|173|priationought|medium|6steel39247084600119|white|Bundle|Unknown|65|ablepriesen st| +9433|AAAAAAAAJNECAAAA|1997-10-27||Annual, true authorities see just out of a trains. Then annual markets will make in a policies. Children may explain different, various industries. Unduly rare paintings ought to |3.87|1.19|1001002|amalgamalg #2|1|dresses|1|Women|72|ableation|medium|3522969971450rose952|smoke|Pound|Unknown|71|pripriesen st| +9434|AAAAAAAAKNECAAAA|1997-10-27|2000-10-26|Likely, fine manage|9.60|7.39|7016005|corpnameless #5|16|furniture|7|Home|214|eseoughtable|N/A|80048navy30916858096|light|Bunch|Unknown|26|esepriesen st| +9435|AAAAAAAAKNECAAAA|2000-10-27||Just good guests shoul|6.86|2.33|1004002|edu packamalg #2|4|swimwear|1|Women|618|eseoughtable|large|35322rose44689511940|yellow|Oz|Unknown|25|antipriesen st| +9436|AAAAAAAAMNECAAAA|1997-10-27|1999-10-27|Legal, central years could arrive gree|1.97|0.74|1001001|amalgamalg #1|1|dresses|1|Women|236|callypriable|small|3277117975drab429984|gainsboro|Tsp|Unknown|46|callypriesen st| +9437|AAAAAAAAMNECAAAA|1999-10-28|2001-10-26|Legal, central years could arrive gree|5.94|3.50|10009003|maxiunivamalg #3|9|televisions|10|Electronics|236|callypriable|N/A|76932867683pale02285|grey|Carton|Unknown|20|ationpriesen st| +9438|AAAAAAAAMNECAAAA|2001-10-27||Various benefits show. Also modern rates could see at a peasants. For example welsh classes produce here by a enterprises; hands move together central, direct files. Cl|6.95|6.04|1001001|amalgamalg #1|1|dresses|1|Women|98|eingn st|small|76932867683pale02285|lace|Each|Unknown|26|eingpriesen st| +9439|AAAAAAAAPNECAAAA|1997-10-27||Warm methods learn now genuine boards; local, exact transactions step well at the days. New, ful|4.05|1.70|9013008|exportiunivamalg #8|13|self-help|9|Books|486|callyeingese|N/A|86786232honeydew4583|red|Tsp|Unknown|25|n stpriesen st| +9440|AAAAAAAAAOECAAAA|1997-10-27|2000-10-26|Individual, alone thous|3.44|1.30|6002001|importocorp #1|2|diamonds|6|Jewelry|24|eseable|N/A|4010spring9012235134|indian|Pound|Unknown|36|bareseesen st| +9441|AAAAAAAAAOECAAAA|2000-10-27||Here grey officers shal|31.66|1.30|6016002|corpbrand #2|2|consignment|6|Jewelry|122|ableableought|N/A|4010spring9012235134|pink|Box|Unknown|18|oughteseesen st| +9442|AAAAAAAACOECAAAA|1997-10-27|1999-10-27|Owners might not confirm steady level, fresh developments. Possible police ought to provide deeply organic arms. Old, black courts fetch criteria. Basic commentators must not|6.63|5.43|5004001|edu packscholar #1|4|classical|5|Music|266|callycallyable|N/A|566589rosy9309226365|plum|Dozen|Unknown|29|ableeseesen st| +9443|AAAAAAAACOECAAAA|1999-10-28|2001-10-26|Owners might not confirm steady level, fresh developments. Possible police ought to provide deeply organic arms. Old, black courts fetch criteria. Basic commentators must not|7.01|5.43|9015012|scholarunivamalg #12|15|fiction|9|Books|308|eingbarpri|N/A|0tomato6359778239906|slate|Gram|Unknown|53|prieseesen st| +9444|AAAAAAAACOECAAAA|2001-10-27||Functions come more villages. Effects must come to a genes. Complex countries would sound well natural members. Easier poor results plan habit|4.81|5.43|9015012|amalgedu pack #1|1|womens|4|Shoes|26|eingbarpri|small|0tomato6359778239906|powder|Dram|Unknown|23|eseeseesen st| +9445|AAAAAAAAFOECAAAA|1997-10-27||Members should not soar today today royal |1.24|0.60|3003002|exportiexporti #2|3|toddlers|3|Children|398|eingn stpri|medium|43353165smoke0641469|plum|Ton|Unknown|77|antieseesen st| +9446|AAAAAAAAGOECAAAA|1997-10-27|2000-10-26|Individual, medium sides talk to the dreams. Terms send finally subsequent weapons. Suitable rooms might marry somewhat leading things. Inappropriate,|2.65|2.22|1003001|exportiamalg #1|3|maternity|1|Women|117|ationoughtought|large|793551peach629163171|tomato|Each|Unknown|36|callyeseesen st| +9447|AAAAAAAAGOECAAAA|2000-10-27||Individual, private men know later only main provisions; mor|84.11|2.22|5001002|amalgscholar #2|1|rock|5|Music|117|ationoughtought|N/A|958280steel979256919|olive|Each|Unknown|13|ationeseesen st| +9448|AAAAAAAAIOECAAAA|1997-10-27|1999-10-27|Ag|8.22|6.90|7002009|importobrand #9|2|bedding|7|Home|726|callyableation|N/A|8998014indian5305373|violet|Case|Unknown|52|eingeseesen st| +9449|AAAAAAAAIOECAAAA|1999-10-28|2001-10-26|Ag|2.51|0.95|6004002|edu packcorp #2|4|bracelets|6|Jewelry|25|callyableation|N/A|46634steel4599356393|lime|Dram|Unknown|42|n steseesen st| +9450|AAAAAAAAIOECAAAA|2001-10-27||Actions shall not include rather secret costs. Raw objects reduce theoretically pieces. Ne|4.61|3.31|6004002|amalgexporti #1|4|newborn|3|Children|25|antiable|large|46634steel4599356393|dodger|Tsp|Unknown|10|barantiesen st| +9451|AAAAAAAALOECAAAA|1997-10-27||Frequently |5.63|4.10|1001002|amalgamalg #2|1|dresses|1|Women|82|ableeing|medium|94047972yellow131909|tan|Oz|Unknown|21|oughtantiesen st| +9452|AAAAAAAAMOECAAAA|1997-10-27|2000-10-26|Long impressive words raise also. Ears want small eyes. Groups should fit sexual campaigns. Difficult results must not make depe|9.77|8.01|1001001|amalgamalg #1|1|dresses|1|Women|146|callyeseought|economy|5saddle4753720734799|medium|Box|Unknown|58|ableantiesen st| +9453|AAAAAAAAMOECAAAA|2000-10-27||Subject orders asses|7.71|3.00|1001001|corpmaxi #4|1|golf|8|Sports|146|callyeseought|N/A|903169767846slate069|maroon|Gram|Unknown|17|priantiesen st| +9454|AAAAAAAAOOECAAAA|1997-10-27|1999-10-27|Foreign scenes qualify of course objectively|3.63|1.12|9008011|namelessmaxi #11|8|romance|9|Books|21|oughtable|N/A|5linen70938875711658|snow|Box|Unknown|65|eseantiesen st| +9455|AAAAAAAAOOECAAAA|1999-10-28|2001-10-26|Foreign scenes qualify of course objectively|0.95|1.12|9008011|exportinameless #6|13|wallpaper|7|Home|21|oughtable|N/A|5linen70938875711658|rosy|Cup|Unknown|64|antiantiesen st| +9456|AAAAAAAAOOECAAAA|2001-10-27||Foreign scenes qualify of course objectively|0.28|0.19|10002005|importounivamalg #5|2|camcorders|10|Electronics|21|oughtable|N/A|538white587976399207|snow|Pallet|Unknown|28|callyantiesen st| +9457|AAAAAAAABPECAAAA|1997-10-27||So great buildings may not tell dirty, pure keys; already bare days|6.00|3.84|8005008|scholarnameless #8|5|fitness|8|Sports|194|esen stought|N/A|5392peru548245855998|lawn|Ton|Unknown|38|ationantiesen st| +9458|AAAAAAAACPECAAAA|1997-10-27|2000-10-26|Official, able powers like so. Liable corners learn markets. |4.17|1.70|2004001|edu packimporto #1|4|sports-apparel|2|Men|90|barn st|medium|60849927turquoise804|pale|Tbl|Unknown|4|eingantiesen st| +9459|AAAAAAAACPECAAAA|2000-10-27||Also hard years could remain. Long, only times must express new days. Too other parties get. National servants lead pure, usual times; then f|2.23|1.18|8011010|amalgmaxi #10|11|archery|8|Sports|90|barn st|N/A|243314555linen581878|navy|Ounce|Unknown|26|n stantiesen st| +9460|AAAAAAAAEPECAAAA|1997-10-27|1999-10-27|Sentences loose; available, similar yards will not re|7.56|3.32|7011001|amalgnameless #1|11|accent|7|Home|761|oughtcallyation|N/A|goldenrod20595183332|dodger|Oz|Unknown|63|barcallyesen st| +9461|AAAAAAAAEPECAAAA|1999-10-28|2001-10-26|Junior, invisible implications protect international members. Beautiful, apparent committees may not satisfy else. Police watch perhaps visible, labour systems. |7.77|3.65|7011001|exportiimporto #2|3|pants|2|Men|33|oughtcallyation|extra large|ivory811191518691303|steel|Ton|Unknown|94|oughtcallyesen st| +9462|AAAAAAAAEPECAAAA|2001-10-27||As racial patients would think there trusts. Properly small sums must think to a recommendations. Other, local children should not form generally. More than |0.44|3.65|3002001|importoexporti #1|2|infants|3|Children|33|pripri|small|ivory811191518691303|steel|Bunch|Unknown|86|ablecallyesen st| +9463|AAAAAAAAHPECAAAA|1997-10-27||Years can decide by the sides. Total ways ship. Atomic proposals will not kill private surveys. Particularly different months imagine enough in a conditions. Right, cold n|2.27|0.79|5003002|exportischolar #2|3|pop|5|Music|663|pricallycally|N/A|39003912301orange951|tan|Box|Unknown|59|pricallyesen st| +9464|AAAAAAAAIPECAAAA|1997-10-27|2000-10-26|Female units receive. Well available processes should regard thus clear workers. Very medical stages send. Officers stay as at all american eyes. Central, key hands|4.59|3.53|2004001|edu packimporto #1|4|sports-apparel|2|Men|153|priantiought|small|495247papaya81748993|bisque|Tsp|Unknown|38|esecallyesen st| +9465|AAAAAAAAIPECAAAA|2000-10-27||Structures introduce now in a priorities. |2.99|2.63|2004001|univmaxi #6|10|pools|8|Sports|427|priantiought|N/A|295684white051515378|violet|Dozen|Unknown|73|anticallyesen st| +9466|AAAAAAAAKPECAAAA|1997-10-27|1999-10-27|Full, rural artists must not notice deeper historical stages; other years may preserve apparently traditional solicitors. Central, old years will not manage best qu|1.81|1.52|8016007|corpmaxi #7|16|golf|8|Sports|227|ationableable|N/A|867dodger65956201085|goldenrod|Tsp|Unknown|74|callycallyesen st| +9467|AAAAAAAAKPECAAAA|1999-10-28|2001-10-26|Full, rural artists must not notice deeper historical stages; other years may preserve apparently traditional solicitors. Central, old years will not manage best qu|5.18|1.52|6002004|importocorp #4|16|diamonds|6|Jewelry|227|ationableable|N/A|867dodger65956201085|orange|Pallet|Unknown|30|ationcallyesen st| +9468|AAAAAAAAKPECAAAA|2001-10-27||Phenomena shall not need speeches; enough international structures must not follow further years. Deeply origi|72.85|32.05|6002004|univbrand #5|10|jewelry boxes|6|Jewelry|227|ationableable|N/A|36salmon549061241635|pink|Box|Unknown|15|eingcallyesen st| +9469|AAAAAAAANPECAAAA|1997-10-27||Officials help home through a problems. Positive heads might reach also here difficult machines. Countries might lead french, liab|3.60|2.98|7009010|maxibrand #10|9|mattresses|7|Home|315|antioughtpri|N/A|20054913730cornsilk0|navy|Tsp|Unknown|13|n stcallyesen st| +9470|AAAAAAAAOPECAAAA|1997-10-27|2000-10-26|Certain roots send forward only, western parts. Slim cards could feel already with the calls. Valid benefits go hospitals. Very, like incident|2.42|1.11|3001001|amalgexporti #1|1|newborn|3|Children|322|ableablepri|large|388933447967saddle29|tan|Unknown|Unknown|57|barationesen st| +9471|AAAAAAAAOPECAAAA|2000-10-27||Calculations could not give easily single references. Customs can help too sometimes bright flowers. Busy, level years see. Possible, professional factors shall not like more conscious, easy rules. |3.32|2.78|3001001|namelesscorp #8|8|mens watch|6|Jewelry|380|ableablepri|N/A|4209yellow5465625882|seashell|Dozen|Unknown|54|oughtationesen st| +9472|AAAAAAAAAAFCAAAA|1997-10-27|1999-10-27|Necessary eggs advise complete elements. Strong, leading teachers shall not continue so docume|4.45|3.20|3002001|importoexporti #1|2|infants|3|Children|57|ationanti|extra large|42918cornsilk0098869|smoke|Bunch|Unknown|26|ableationesen st| +9473|AAAAAAAAAAFCAAAA|1999-10-28|2001-10-26|Oral, final studies will like after a families. Future, high profits see evident acts. Unnecessary, political clothes expect truly. Fundamental, single lines used to take environmenta|5.10|3.20|3002001|importoimporto #2|2|shirts|2|Men|57|ationanti|extra large|3291650941679pale884|slate|N/A|Unknown|20|priationesen st| +9474|AAAAAAAAAAFCAAAA|2001-10-27||Oral, final studies will like after a families. Future, high profits see evident acts. Unnecessary, political clothes expect truly. Fundamental, single lines used to take environmenta|6.18|5.31|3002001|exportimaxi #5|2|sailing|8|Sports|448|eingeseese|N/A|peru0184127895623494|rose|Dram|Unknown|8|eseationesen st| +9475|AAAAAAAADAFCAAAA|1997-10-27||Now poor police shall take now often considerable powers. Only industr|3.40|2.07|10005007|scholarunivamalg #7|5|karoke|10|Electronics|236|callypriable|N/A|0283197329949linen05|plum|Carton|Unknown|13|antiationesen st| +9476|AAAAAAAAEAFCAAAA|1997-10-27|2000-10-26|Certain, happy miles bet well scientific quantities; companies could find actually important players. Other firms mig|2.18|0.91|5002001|importoscholar #1|2|country|5|Music|824|eseableeing|N/A|53130238207hot549630|tan|Gram|Unknown|28|callyationesen st| +9477|AAAAAAAAEAFCAAAA|2000-10-27||Sad, |7.51|0.91|6015006|scholarbrand #6|2|custom|6|Jewelry|421|oughtableese|N/A|53130238207hot549630|wheat|Ounce|Unknown|22|ationationesen st| +9478|AAAAAAAAGAFCAAAA|1997-10-27|1999-10-27|Meanwhile certai|6.87|4.25|9014011|edu packunivamalg #11|14|sports|9|Books|244|eseeseable|N/A|45607498navajo024259|smoke|Gross|Unknown|1|eingationesen st| +9479|AAAAAAAAGAFCAAAA|1999-10-28|2001-10-26|Meanwhile certai|9.72|4.25|9009012|maximaxi #12|14|science|9|Books|244|eseeseable|N/A|0261595frosted143286|powder|Box|Unknown|4|n stationesen st| +9480|AAAAAAAAGAFCAAAA|2001-10-27||Meanwhile certai|5.56|4.25|9009012|edu packimporto #1|14|sports-apparel|2|Men|244|eseeseable|large|0261595frosted143286|snow|Each|Unknown|39|bareingesen st| +9481|AAAAAAAAJAFCAAAA|1997-10-27||Hardly grim exhibitions draw low times. Video-taped, spectacul|5.45|2.07|1001002|amalgamalg #2|1|dresses|1|Women|55|antianti|small|3868407pink838297899|rose|Box|Unknown|33|oughteingesen st| +9482|AAAAAAAAKAFCAAAA|1997-10-27|2000-10-26|Emotional women can raise excessively normal, monetary years. Private, regular families intensify thus with a lectures. Temporarily personal shoulders call rather apparent, post-war words|2.17|1.10|8004003|edu packnameless #3|4|camping|8|Sports|37|ationpri|N/A|38609981915tan863645|seashell|Pallet|Unknown|26|ableeingesen st| +9483|AAAAAAAAKAFCAAAA|2000-10-27||Emotional women can raise excessively normal, monetary years. Private, regular families intensify thus with a lectures. Temporarily personal shoulders call rather apparent, post-war words|1.86|1.10|3003002|exportiexporti #2|4|toddlers|3|Children|344|eseesepri|large|3787plum865896445062|sandy|Bundle|Unknown|12|prieingesen st| +9484|AAAAAAAAMAFCAAAA|1997-10-27|1999-10-27|National, unemployed kids get already clean, single topics. Also foreign protests must make english issues. Manufac|1.14|0.44|2001001|amalgimporto #1|1|accessories|2|Men|581|oughteinganti|large|568076337687yellow54|mint|Tsp|Unknown|29|eseeingesen st| +9485|AAAAAAAAMAFCAAAA|1999-10-28|2001-10-26|En route successful colours cannot speak. At least industrial stocks recognize of course. Nervously coming systems|3.83|0.44|2001001|exportiexporti #2|3|toddlers|3|Children|581|oughteinganti|large|568076337687yellow54|yellow|Carton|Unknown|10|antieingesen st| +9486|AAAAAAAAMAFCAAAA|2001-10-27||En route successful colours cannot speak. At least industrial stocks recognize of course. Nervously coming systems|1.10|0.44|5001001|amalgscholar #1|1|rock|5|Music|581|oughteinganti|N/A|96068188white3184936|saddle|Tsp|Unknown|32|callyeingesen st| +9487|AAAAAAAAPAFCAAAA|1997-10-27||Rational, grateful laws may allow in a mountains; usually increased requirements might not follow even usual particular years. As yet sweet trends meet v|0.10|0.03|8005008|scholarnameless #8|5|fitness|8|Sports|140|bareseought|N/A|8wheat51201313279597|tomato|Lb|Unknown|50|ationeingesen st| +9488|AAAAAAAAABFCAAAA|1997-10-27|2000-10-26|Once tight cattl|4.93|3.54|6005001|scholarcorp #1|5|earings|6|Jewelry|87|ationeing|N/A|7750389383188royal04|red|N/A|Unknown|3|eingeingesen st| +9489|AAAAAAAAABFCAAAA|2000-10-27||Fairly likely pages ought to facilitate even high situations. Major, other differences draft strongly years. Faint, large exports ought to slow short societies. Free molecules can buy jus|1.92|1.65|7006010|corpbrand #10|5|rugs|7|Home|87|ationeing|N/A|60puff34421990905788|salmon|Pound|Unknown|23|n steingesen st| +9490|AAAAAAAACBFCAAAA|1997-10-27|1999-10-27|Now total solicitors state elaborate messages. Even so sexual years will not express by a months. Again military partners might bear regions. Also minor fingers define since a fo|7.52|4.36|3004001|edu packexporti #1|4|school-uniforms|3|Children|279|n stationable|medium|28556tomato709516543|navajo|Dozen|Unknown|3|barn stesen st| +9491|AAAAAAAACBFCAAAA|1999-10-28|2001-10-26|Now total solicitors state elaborate messages. Even so sexual years will not express by a months. Again military partners might bear regions. Also minor fingers define since a fo|3.83|4.36|7001002|amalgbrand #2|4|bathroom|7|Home|279|n stationable|N/A|929505311071saddle17|violet|Pallet|Unknown|28|oughtn stesen st| +9492|AAAAAAAACBFCAAAA|2001-10-27||Now total solicitors state elaborate messages. Even so sexual years will not express by a months. Again military partners might bear regions. Also minor fingers define since a fo|3.13|4.36|9008001|namelessmaxi #1|8|romance|9|Books|279|n stationable|N/A|929505311071saddle17|cornsilk|Bunch|Unknown|26|ablen stesen st| +9493|AAAAAAAAFBFCAAAA|1997-10-27||Mechanisms make. Most small colleagues remember only. Previous, clear years measure at once. Words find already representatives. Lucky restaurants mark parts. Local, prime grants cannot find so|3.98|3.54|9005002|scholarmaxi #2|5|history|9|Books|745|antieseation|N/A|thistle6362901862637|khaki|Oz|Unknown|34|prin stesen st| +9494|AAAAAAAAGBFCAAAA|1997-10-27|2000-10-26|Democratic members die now together only requirements. Still possible studies used to get however shares. Formidable, conventional years could represent capable workshops. Wonde|4.15|3.27|8006005|corpnameless #5|6|football|8|Sports|424|eseableese|N/A|steel620653506310549|aquamarine|Unknown|Unknown|59|esen stesen st| +9495|AAAAAAAAGBFCAAAA|2000-10-27||Legitimate, corpo|0.12|3.27|8006005|univamalgamalg #10|10|memory|10|Electronics|134|esepriought|N/A|68535yellow082884205|pink|Gross|Unknown|7|antin stesen st| +9496|AAAAAAAAIBFCAAAA|1997-10-27|1999-10-27|Soon sophisticated schools succeed etc late groups. Genes should not keep more industrial places. Cleve|2.49|1.56|7006007|corpbrand #7|6|rugs|7|Home|71|oughtation|N/A|99390950208157light9|tomato|Tsp|Unknown|18|callyn stesen st| +9497|AAAAAAAAIBFCAAAA|1999-10-28|2001-10-26|Soon sophisticated schools succeed etc late groups. Genes should not keep more industrial places. Cleve|1.75|1.19|7006007|edu packexporti #2|4|school-uniforms|3|Children|71|oughtation|petite|416185260624936puff0|red|Tsp|Unknown|30|ationn stesen st| +9498|AAAAAAAAIBFCAAAA|2001-10-27||Absent, large groups shall not list in a attitudes; bones challenge either middle beings. True pensioners could give o|0.36|1.19|7006007|importobrand #9|4|bedding|7|Home|71|oughtation|N/A|416185260624936puff0|snow|Box|Unknown|62|eingn stesen st| +9499|AAAAAAAALBFCAAAA|1997-10-27||Rough skills would introduce black affairs. Intellectual, new facts ensure skills|4.47|2.05|5004002|edu packscholar #2|4|classical|5|Music|318|eingoughtpri|N/A|2898rosy036510191520|ivory|Unknown|Unknown|20|n stn stesen st| +9500|AAAAAAAAMBFCAAAA|1997-10-27|2000-10-26|Private, democratic hands could not compete now anxious levels; pure supporters would not question furt|7.76|6.36|9005009|scholarmaxi #9|5|history|9|Books|239|n stpriable|N/A|2midnight92678610895|yellow|Ounce|Unknown|10|barbarantin st| +9501|AAAAAAAAMBFCAAAA|2000-10-27||Private, democratic hands could not compete now anxious levels; pure supporters would not question furt|1.36|0.92|9008010|namelessmaxi #10|5|romance|9|Books|532|ableprianti|N/A|2midnight92678610895|grey|Each|Unknown|54|oughtbarantin st| +9502|AAAAAAAAOBFCAAAA|1997-10-27|1999-10-27|Main, open votes shall handle scottish strengths. Common, great participants may record today for a fears. Regional, important marks would cost below a sales. Premier re|0.82|0.68|2004001|edu packimporto #1|4|sports-apparel|2|Men|208|eingbarable|medium|87indian871350339483|thistle|Ton|Unknown|10|ablebarantin st| +9503|AAAAAAAAOBFCAAAA|1999-10-28|2001-10-26|Main, open votes shall handle scottish strengths. Common, great participants may record today for a fears. Regional, important marks would cost below a sales. Premier re|7.63|4.12|6015008|scholarbrand #8|4|custom|6|Jewelry|208|eingbarable|N/A|266peach726735257805|olive|Case|Unknown|22|pribarantin st| +9504|AAAAAAAAOBFCAAAA|2001-10-27||Technological ministers kill in a seats. Developments would end above situations. Findings would not work parts. Unfortunately good relations live quite here original ingre|0.80|0.24|3001001|amalgexporti #1|1|newborn|3|Children|179|n stationought|small|266peach726735257805|thistle|Box|Unknown|28|esebarantin st| +9505|AAAAAAAABCFCAAAA|1997-10-27||Poor letters think even internal, personal rates. Technical elements cannot follow jus|0.14|0.12|4003002|exportiedu pack #2|3|kids|4|Shoes|303|pribarpri|economy|5750pale264055257032|maroon|Tbl|Unknown|37|antibarantin st| +9506|AAAAAAAACCFCAAAA|1997-10-27|2000-10-26|Already enormous lives use around by a parents. Tropical, old teams need days. Recently only consequences may leave just issues. Volunteers will move further foreign views. National, ameri|0.53|0.41|2003001|exportiimporto #1|3|pants|2|Men|668|eingcallycally|small|45909033sandy1515373|violet|Lb|Unknown|54|callybarantin st| +9507|AAAAAAAACCFCAAAA|2000-10-27||Already enormous lives use around by a parents. Tropical, old teams need days. Recently only consequences may leave just issues. Volunteers will move further foreign views. National, ameri|8.73|0.41|2001002|amalgimporto #2|1|accessories|2|Men|668|eingcallycally|large|8402746925mint412921|yellow|N/A|Unknown|32|ationbarantin st| +9508|AAAAAAAAECFCAAAA|1997-10-27|1999-10-27|Prepared others convey elsewhere environmental, british tactics. Sorry adults hear. So working texts release wor|1.98|0.97|9003005|exportimaxi #5|3|computers|9|Books|281|oughteingable|N/A|5249red9483717848855|chartreuse|N/A|Unknown|47|eingbarantin st| +9509|AAAAAAAAECFCAAAA|1999-10-28|2001-10-26|Again medical movements supervise. Just yo|4.24|0.97|6009006|maxicorp #6|9|womens watch|6|Jewelry|352|oughteingable|N/A|5249red9483717848855|papaya|N/A|Unknown|26|n stbarantin st| +9510|AAAAAAAAECFCAAAA|2001-10-27||Chief, considerable campaigns could publish else. Well political others might exploit. Papers expand briefly. C|77.20|57.12|6009006|importonameless #1|9|paint|7|Home|53|oughteingable|N/A|light892324514057565|sandy|Ounce|Unknown|22|baroughtantin st| +9511|AAAAAAAAHCFCAAAA|1997-10-27||Purposes document. For instance glad hopes say confident, concrete initiatives. Pat|8.08|2.58|5004002|edu packscholar #2|4|classical|5|Music|733|pripriation|N/A|2647royal87308334984|sandy|Ton|Unknown|38|oughtoughtantin st| +9512|AAAAAAAAICFCAAAA|1997-10-27|2000-10-26|So inland children might not want much forms. Police could expect merely technical |9.10|2.82|3003001|exportiexporti #1|3|toddlers|3|Children|406|callybarese|medium|190434wheat386213063|turquoise|Dram|Unknown|16|ableoughtantin st| +9513|AAAAAAAAICFCAAAA|2000-10-27||Dangerous, simple circles used to reach namely in a bodies. Coherent, romantic cards may get about impossible, british others. Asleep changes mean determined arguments. Changing chemical|9.09|4.81|3003001|exportiexporti #2|3|toddlers|3|Children|548|callybarese|large|190434wheat386213063|antique|Each|Unknown|12|prioughtantin st| +9514|AAAAAAAAKCFCAAAA|1997-10-27|1999-10-27|Countries arrive features; maybe i|8.57|5.82|1004001|edu packamalg #1|4|swimwear|1|Women|343|priesepri|medium|87117394964snow71707|rosy|Each|Unknown|15|eseoughtantin st| +9515|AAAAAAAAKCFCAAAA|1999-10-28|2001-10-26|Points get. Far other institutions achieve financial, continuous details. Enough str|1.69|1.50|1004001|namelessunivamalg #13|8|scanners|10|Electronics|343|priesepri|N/A|29332928296grey92700|magenta|Ton|Unknown|14|antioughtantin st| +9516|AAAAAAAAKCFCAAAA|2001-10-27||Points get. Far other institutions achieve financial, continuous details. Enough str|6.56|1.50|10004014|edu packunivamalg #14|8|audio|10|Electronics|227|priesepri|N/A|peach576289009558046|forest|Unknown|Unknown|24|callyoughtantin st| +9517|AAAAAAAANCFCAAAA|1997-10-27||Scientific courses set different questions. Various, likely surfaces prevent also vague days. Critical, grand clothes save from a duties; powerful|1.45|1.21|9003002|exportimaxi #2|3|computers|9|Books|501|oughtbaranti|N/A|65plum34380199345601|khaki|Gross|Unknown|5|ationoughtantin st| +9518|AAAAAAAAOCFCAAAA|1997-10-27|2000-10-26|External forms survive just quite environmental men; teeth recommend arts. Exactly important areas must not keep hard really total numbers. |1.53|0.45|4002001|importoedu pack #1|2|mens|4|Shoes|944|eseesen st|petite|713332457snow6753454|misty|Pound|Unknown|24|eingoughtantin st| +9519|AAAAAAAAOCFCAAAA|2000-10-27||Sure, social sons shall not want busy years; much imperial |3.68|2.24|4002001|exportimaxi #10|13|sailing|8|Sports|242|eseesen st|N/A|713332457snow6753454|khaki|Cup|Unknown|35|n stoughtantin st| +9520|AAAAAAAAADFCAAAA|1997-10-27|1999-10-27|Silver, spare words c|5.95|3.98|6013001|exportibrand #1|13|loose stones|6|Jewelry|7|ation|N/A|99310wheat9956704087|spring|Bunch|Unknown|26|barableantin st| +9521|AAAAAAAAADFCAAAA|1999-10-28|2001-10-26|Silver, spare words c|4.15|2.61|7002002|importobrand #2|2|bedding|7|Home|7|ation|N/A|7822195spring5875516|gainsboro|N/A|Unknown|64|oughtableantin st| +9522|AAAAAAAAADFCAAAA|2001-10-27||Bars think heavily over a shoulders. Quiet, japanese cuts work so new times. Certainly mean applications should permit. Labour, right sports re|2.12|1.14|7002002|amalgamalg #1|2|dresses|1|Women|512|ableoughtanti|large|44639055white9031279|peach|Bundle|Unknown|47|ableableantin st| +9523|AAAAAAAADDFCAAAA|1997-10-27||Misleading, royal orders ought to attempt away single schools. Fat generations could not get h|5.94|5.22|9005008|scholarmaxi #8|5|history|9|Books|171|oughtationought|N/A|1987591951cornsilk82|grey|Gross|Unknown|42|priableantin st| +9524|AAAAAAAAEDFCAAAA|1997-10-27|2000-10-26|Decent things borrow well times. H|4.95|2.27|7007005|brandbrand #5|7|decor|7|Home|647|ationesecally|N/A|44255saddle278220001|peru|Box|Unknown|27|eseableantin st| +9525|AAAAAAAAEDFCAAAA|2000-10-27||Decent things borrow well times. H|7.43|6.68|7007005|scholarnameless #6|7|fitness|8|Sports|374|eseationpri|N/A|44255saddle278220001|yellow|Pound|Unknown|17|antiableantin st| +9526|AAAAAAAAGDFCAAAA|1997-10-27|1999-10-27|Political, va|4.54|2.99|8002007|importonameless #7|2|baseball|8|Sports|306|callybarpri|N/A|72turquoise951537668|violet|Each|Unknown|27|callyableantin st| +9527|AAAAAAAAGDFCAAAA|1999-10-28|2001-10-26|Political, va|2.87|2.99|8002007|importoexporti #2|2|infants|3|Children|85|callybarpri|economy|33561391110646red531|linen|N/A|Unknown|30|ationableantin st| +9528|AAAAAAAAGDFCAAAA|2001-10-27||Added skills may undo there particular students. Old friends set as a patients. Parts provide still; superb waters could forget really in a |3.11|1.89|8010009|univmaxi #9|2|pools|8|Sports|814|eseoughteing|N/A|26248802spring571178|smoke|Cup|Unknown|19|eingableantin st| +9529|AAAAAAAAJDFCAAAA|1997-10-27||Scottish, old men attract correspondingly different pupils. So willing words can tell just narrow, poor thanks. Streets know early to a savings. Outside major figures place pairs. Cells set in |25.08|17.05|10008010|namelessunivamalg #10|8|scanners|10|Electronics|684|eseeingcally|N/A|6442018red1559930240|ghost|Unknown|Unknown|49|n stableantin st| +9530|AAAAAAAAKDFCAAAA|1997-10-27|2000-10-26|Blue children can get grim, central eyes. New, reasonable meetings me|7.03|2.46|9008003|namelessmaxi #3|8|romance|9|Books|204|esebarable|N/A|3586414889394rosy972|rose|Each|Unknown|35|barpriantin st| +9531|AAAAAAAAKDFCAAAA|2000-10-27||Changes count stor|4.50|2.46|9008003|exportiexporti #2|3|toddlers|3|Children|204|esebarable|large|8135374honeydew45493|white|Each|Unknown|80|oughtpriantin st| +9532|AAAAAAAAMDFCAAAA|1997-10-27|1999-10-27|Characters seek hard also basic banks. Abroad different applications give especially attitudes. Men might bre|5.81|2.44|4002001|importoedu pack #1|2|mens|4|Shoes|269|n stcallyable|medium|283267hot00982414009|lavender|Carton|Unknown|2|ablepriantin st| +9533|AAAAAAAAMDFCAAAA|1999-10-28|2001-10-26|Characters seek hard also basic banks. Abroad different applications give especially attitudes. Men might bre|4.87|2.09|7004004|edu packbrand #4|4|curtains/drapes|7|Home|523|n stcallyable|N/A|4pink306027508155009|seashell|Lb|Unknown|73|pripriantin st| +9534|AAAAAAAAMDFCAAAA|2001-10-27||Procedures lead as deaf regions; original firms organise left, head sheets. Clearly competitive conditions see at a images. Outer servants achieve coming, natural cells. Keen |0.39|2.09|7004004|importoamalg #1|4|fragrances|1|Women|523|n stcallyable|petite|525olive186320083029|powder|Oz|Unknown|60|esepriantin st| +9535|AAAAAAAAPDFCAAAA|1997-10-27||Well used yar|7.72|4.86|3001002|amalgexporti #2|1|newborn|3|Children|529|n stableanti|petite|86sandy5893860326360|powder|Pallet|Unknown|24|antipriantin st| +9536|AAAAAAAAAEFCAAAA|1997-10-27|2000-10-26|Else mental recordings find seriously royal, certain lines. Perhaps big hours put. Necess|5.49|1.70|6014001|edu packbrand #1|14|estate|6|Jewelry|558|eingantianti|N/A|162101833145sandy333|yellow|Box|Unknown|49|callypriantin st| +9537|AAAAAAAAAEFCAAAA|2000-10-27||Else mental recordings find seriously royal, certain lines. Perhaps big hours put. Necess|0.70|1.70|6014001|exportischolar #2|14|pop|5|Music|71|oughtation|N/A|sienna60298014099714|white|Oz|Unknown|12|ationpriantin st| +9538|AAAAAAAACEFCAAAA|1997-10-27|1999-10-27|Indeed alone police wo|8.38|6.36|4001001|amalgedu pack #1|1|womens|4|Shoes|486|callyeingese|petite|749408440004medium73|saddle|Oz|Unknown|66|eingpriantin st| +9539|AAAAAAAACEFCAAAA|1999-10-28|2001-10-26|Indeed alone police wo|5.13|6.36|5001002|amalgscholar #2|1|rock|5|Music|486|callyeingese|N/A|6268970164papaya3967|navy|Pound|Unknown|5|n stpriantin st| +9540|AAAAAAAACEFCAAAA|2001-10-27||Ways will take statistical heads. Largely new stories should not investigate today spatial hard requirements; significant libraries affect patterns. Nicely big structures should forget so|71.90|61.11|5001002|edu packunivamalg #4|1|audio|10|Electronics|486|callyeingese|N/A|316506olive985514635|green|Tsp|Unknown|12|bareseantin st| +9541|AAAAAAAAFEFCAAAA|1997-10-27||Exceptionally detailed fortunes collect sure plans. Unable, urban services defend. Today primary contributions write wi|4.11|1.72|10003005|exportiunivamalg #5|3|dvd/vcr players|10|Electronics|112|ableoughtought|N/A|930744204puff7256483|wheat|Ton|Unknown|41|oughteseantin st| +9542|AAAAAAAAGEFCAAAA|1997-10-27|2000-10-26|Standards drink. Royal weeks settle premier years. Various causes could say. Rough, free terms let more |4.98|4.28|4003001|exportiedu pack #1|3|kids|4|Shoes|389|n steingpri|medium|2chocolate6593149242|tan|Ton|Unknown|18|ableeseantin st| +9543|AAAAAAAAGEFCAAAA|2000-10-27||Standards drink. Royal weeks settle premier years. Various causes could say. Rough, free terms let more |2.49|2.09|8008004|namelessnameless #4|8|outdoor|8|Sports|305|antibarpri|N/A|2chocolate6593149242|papaya|Pound|Unknown|74|prieseantin st| +9544|AAAAAAAAIEFCAAAA|1997-10-27|1999-10-27|Immediate men ought to look various, broad commentators. Original procedures keep all right male counties; almost|6.82|4.97|5002001|importoscholar #1|2|country|5|Music|13|priought|N/A|4426white96541364064|rosy|Dram|Unknown|10|eseeseantin st| +9545|AAAAAAAAIEFCAAAA|1999-10-28|2001-10-26|Normally maximum cases seem traditionally by now wrong shows. Very human leaders shall not play perfectly from the books. Companies augment rough directors; common, cert|1.49|0.52|5002001|exportiunivamalg #6|2|self-help|9|Books|134|priought|N/A|4426white96541364064|misty|Pound|Unknown|24|antieseantin st| +9546|AAAAAAAAIEFCAAAA|2001-10-27||Normally maximum cases seem traditionally by now wrong shows. Very human leaders shall not play perfectly from the books. Companies augment rough directors; common, cert|5.18|0.52|5002001|scholarnameless #7|2|tables|7|Home|182|priought|N/A|59622618408grey33511|sienna|Ounce|Unknown|3|callyeseantin st| +9547|AAAAAAAALEFCAAAA|1997-10-27||Home elderly tears should not recognise bright, new letters. Environmental, sole |3.04|1.12|3004002|edu packexporti #2|4|school-uniforms|3|Children|105|antibarought|small|2lavender56830332896|purple|Dozen|Unknown|15|ationeseantin st| +9548|AAAAAAAAMEFCAAAA|1997-10-27|2000-10-26|Objectives object so remaining, young thousands. Fires need years. Like years shall like either times. Hands demonstrate direct just happy bodies; though arab functions should n|7.24|2.60|8016009|corpmaxi #9|16|golf|8|Sports|162|ablecallyought|N/A|tan15095655178184661|khaki|Unknown|Unknown|57|eingeseantin st| +9549|AAAAAAAAMEFCAAAA|2000-10-27||Only different goals expect again remaining forces. National chips warn all right well comfortable profits. Prominent supplies propose as long-term, long-term comments. Upper, light pp. w|0.80|0.63|8016009|amalgedu pack #2|16|womens|4|Shoes|162|ablecallyought|medium|3seashell55255030064|sandy|Case|Unknown|21|n steseantin st| +9550|AAAAAAAAOEFCAAAA|1997-10-27|1999-10-27|Great cases see recently even financial days. Necessary, relevant trees se|6.58|3.48|5004001|edu packscholar #1|4|classical|5|Music|7|ation|N/A|25dodger634439823751|rosy|Cup|Unknown|50|barantiantin st| +9551|AAAAAAAAOEFCAAAA|1999-10-28|2001-10-26|Just effective tables connect services. Yet valuable students will not see little free days. Ways squ|4.16|3.66|5004001|edu packedu pack #2|4|athletic|4|Shoes|7|ation|large|00533474pale13835860|plum|Case|Unknown|51|oughtantiantin st| +9552|AAAAAAAAOEFCAAAA|2001-10-27||Papers wish coherent times. Areas can pay new, young feelings. There true factors will not work unfortunately. Then huge newspapers allow only fully political answers. Pris|0.84|3.66|5004001|exportimaxi #1|3|computers|9|Books|171|ation|N/A|07210224215white2217|powder|Box|Unknown|9|ableantiantin st| +9553|AAAAAAAABFFCAAAA|1997-10-27||Levels trust new, thin terms; also specific developments shall discover ever. Certain books must not join today boys|0.49|0.32|5004002|edu packscholar #2|4|classical|5|Music|219|n stoughtable|N/A|5939220062011wheat46|sienna|Tbl|Unknown|18|priantiantin st| +9554|AAAAAAAACFFCAAAA|1997-10-27|2000-10-26|Pensioners share; obvious populations turn forever p|4.17|1.87|3003001|exportiexporti #1|3|toddlers|3|Children|221|oughtableable|medium|8red0487859113355208|sky|Dram|Unknown|78|eseantiantin st| +9555|AAAAAAAACFFCAAAA|2000-10-27||Pensioners share; obvious populations turn forever p|9.58|1.87|10014002|edu packamalgamalg #2|14|automotive|10|Electronics|185|oughtableable|N/A|8red0487859113355208|royal|Bunch|Unknown|56|antiantiantin st| +9556|AAAAAAAAEFFCAAAA|1997-10-27|1999-10-27|Animals cannot make most sides; just wealthy babies could fulfil as before a records. Now literary results used to say human, unique genes. Bo|4.85|2.57|8014007|edu packmaxi #7|14|tennis|8|Sports|333|pripripri|N/A|tan31487283901019388|sky|Ton|Unknown|49|callyantiantin st| +9557|AAAAAAAAEFFCAAAA|1999-10-28|2001-10-26|So difficult meanings may match in order sharehol|64.06|46.12|8014007|edu packunivamalg #4|4|audio|10|Electronics|333|pripripri|N/A|44935bisque607417588|salmon|Dram|Unknown|33|ationantiantin st| +9558|AAAAAAAAEFFCAAAA|2001-10-27||Negative, white days will not be|1.86|46.12|10004005|edu packunivamalg #5|4|audio|10|Electronics|333|pripripri|N/A|8651526648tomato5554|sandy|Unknown|Unknown|43|eingantiantin st| +9559|AAAAAAAAHFFCAAAA|1997-10-27||Royal stars take as on the characteristic|9.34|3.17|1004002|edu packamalg #2|4|swimwear|1|Women|610|baroughtcally|medium|97maroon624786363489|peach|Dram|Unknown|3|n stantiantin st| +9560|AAAAAAAAIFFCAAAA|1997-10-27|2000-10-26|Most neat years must pitch with a minutes. Quite symbolic accounts should not engage never either normal girls. Somehow specific s|3.56|2.17|8014001|edu packmaxi #1|14|tennis|8|Sports|171|oughtationought|N/A|74202711354tan991666|honeydew|Box|Unknown|56|barcallyantin st| +9561|AAAAAAAAIFFCAAAA|2000-10-27||Certain, blue women must tell american words. Elsewhere real kinds might not love etc regular systems. Other, full-time reasons destroy also most dear |5.73|3.20|8014002|edu packmaxi #2|14|tennis|8|Sports|171|oughtationought|N/A|74202711354tan991666|violet|Cup|Unknown|8|oughtcallyantin st| +9562|AAAAAAAAKFFCAAAA|1997-10-27|1999-10-27|Recent, alone circumstances amount. Average, private polls get rather products. Financial deputies work no longer|1.32|0.89|2004001|edu packimporto #1|4|sports-apparel|2|Men|156|callyantiought|large|02407403025992slate1|slate|Carton|Unknown|44|ablecallyantin st| +9563|AAAAAAAAKFFCAAAA|1999-10-28|2001-10-26|Recent, alone circumstances amount. Average, private polls get rather products. Financial deputies work no longer|9.57|7.56|8002004|importonameless #4|2|baseball|8|Sports|295|antin stable|N/A|02407403025992slate1|misty|Ton|Unknown|25|pricallyantin st| +9564|AAAAAAAAKFFCAAAA|2001-10-27||Recent, alone circumstances amount. Average, private polls get rather products. Financial deputies work no longer|2.45|1.93|8002004|scholarunivamalg #1|2|fiction|9|Books|295|antin stable|N/A|02407403025992slate1|pink|Dram|Unknown|31|esecallyantin st| +9565|AAAAAAAANFFCAAAA|1997-10-27||New, civ|2.78|2.30|3003002|exportiexporti #2|3|toddlers|3|Children|301|oughtbarpri|extra large|84185deep24032640381|papaya|Cup|Unknown|29|anticallyantin st| +9566|AAAAAAAAOFFCAAAA|1997-10-27|2000-10-26|Comparable, centr|45.44|37.26|1001001|amalgamalg #1|1|dresses|1|Women|410|baroughtese|small|699412273white005414|steel|Lb|Unknown|5|callycallyantin st| +9567|AAAAAAAAOFFCAAAA|2000-10-27||Literary, international resources know private, public agents. Sexual, domestic partners eat nearly|4.21|2.52|10001014|amalgunivamalg #14|1|cameras|10|Electronics|253|priantiable|N/A|699412273white005414|yellow|Gram|Unknown|3|ationcallyantin st| +9568|AAAAAAAAAGFCAAAA|1997-10-27|1999-10-27|Never early trustees cannot reduce nationally in a women; enough burning relatives keep however to a databases. Even |3.92|1.48|2002001|importoimporto #1|2|shirts|2|Men|297|ationn stable|petite|135717powder75340080|medium|Pallet|Unknown|51|eingcallyantin st| +9569|AAAAAAAAAGFCAAAA|1999-10-28|2001-10-26|Men would write names. Full, alive makers know simply mutual men. Children must work; actually great workshops speak now techniques. Ago conventional girls cannot appear now im|4.27|1.48|2002001|importounivamalg #16|2|camcorders|10|Electronics|297|ationn stable|N/A|135717powder75340080|olive|Bundle|Unknown|42|n stcallyantin st| +9570|AAAAAAAAAGFCAAAA|2001-10-27||Men would write names. Full, alive makers know simply mutual men. Children must work; actually great workshops speak now techniques. Ago conventional girls cannot appear now im|0.53|1.48|6010003|univbrand #3|10|jewelry boxes|6|Jewelry|72|ationn stable|N/A|543445179662702sky58|tomato|Cup|Unknown|1|barationantin st| +9571|AAAAAAAADGFCAAAA|1997-10-27||Candidates should say british, various workers. Above new gentlemen shall put on a years. Main, human pieces must develop as a products. H|1.01|0.36|2004002|edu packimporto #2|4|sports-apparel|2|Men|532|ableprianti|large|262696462steel921174|tomato|Tsp|Unknown|65|oughtationantin st| +9572|AAAAAAAAEGFCAAAA|1997-10-27|2000-10-26|International applications|8.29|3.06|8003003|exportinameless #3|3|basketball|8|Sports|322|ableablepri|N/A|566409363792saddle45|papaya|Gross|Unknown|69|ableationantin st| +9573|AAAAAAAAEGFCAAAA|2000-10-27||International applications|0.65|0.34|8003003|exportimaxi #4|13|sailing|8|Sports|738|eingpriation|N/A|566409363792saddle45|orange|Bundle|Unknown|16|priationantin st| +9574|AAAAAAAAGGFCAAAA|1997-10-27|1999-10-27|So very cells would see less individual camps. At last appropriate chemicals maintain better familiar, other c|97.34|84.68|5002001|importoscholar #1|2|country|5|Music|788|eingeingation|N/A|363purple42201775574|smoke|Dozen|Unknown|63|eseationantin st| +9575|AAAAAAAAGGFCAAAA|1999-10-28|2001-10-26|So very cells would see less individual camps. At last appropriate chemicals maintain better familiar, other c|56.48|84.68|7015006|scholarnameless #6|15|tables|7|Home|788|eingeingation|N/A|363purple42201775574|olive|Dram|Unknown|61|antiationantin st| +9576|AAAAAAAAGGFCAAAA|2001-10-27||Privately international th|4.28|3.16|4002001|importoedu pack #1|2|mens|4|Shoes|224|eingeingation|large|007rose3348886743822|rose|Unknown|Unknown|33|callyationantin st| +9577|AAAAAAAAJGFCAAAA|1997-10-27||Also remarkable audiences follow both marked, great activities. Well-known, possible ideas enable particularly just black concentrations. Absolute, individual metres shall not reflec|3.04|2.43|4002002|importoedu pack #2|2|mens|4|Shoes|638|eingprically|small|3992333094plum843045|seashell|Cup|Unknown|66|ationationantin st| +9578|AAAAAAAAKGFCAAAA|1997-10-27|2000-10-26|Much necessary flowers may not admit citizens; international casualties will not marr|4.14|1.73|8014009|edu packmaxi #9|14|tennis|8|Sports|141|oughteseought|N/A|0955772702plum982498|red|Tbl|Unknown|66|eingationantin st| +9579|AAAAAAAAKGFCAAAA|2000-10-27||Legal posts might not ensure economically. Just different pages used to go widely original friends. Others will publicise suddenly best old periods; |3.29|1.08|2001002|amalgimporto #2|14|accessories|2|Men|364|oughteseought|small|pink3196949623327182|violet|Oz|Unknown|43|n stationantin st| +9580|AAAAAAAAMGFCAAAA|1997-10-27|1999-10-27|Ways may gain there total skills. Identical, industrial men ought to say. Fo|0.59|0.36|6006005|corpcorp #5|6|rings|6|Jewelry|341|oughtesepri|N/A|7tan6899527297594895|hot|Each|Unknown|94|bareingantin st| +9581|AAAAAAAAMGFCAAAA|1999-10-28|2001-10-26|Ways may gain there total skills. Identical, industrial men ought to say. Fo|1.31|0.58|6006005|exportiedu pack #2|3|kids|4|Shoes|861|oughtesepri|medium|7tan6899527297594895|rosy|Ounce|Unknown|84|oughteingantin st| +9582|AAAAAAAAMGFCAAAA|2001-10-27||Ways may gain there total skills. Identical, industrial men ought to say. Fo|24.14|0.58|6006005|importonameless #3|3|baseball|8|Sports|530|oughtesepri|N/A|7tan6899527297594895|puff|Pallet|Unknown|94|ableeingantin st| +9583|AAAAAAAAPGFCAAAA|1997-10-27||Walls visit best then fun feelings. Reports might reply new, industrial m|4.48|2.91|10008013|namelessunivamalg #13|8|scanners|10|Electronics|977|ationationn st|N/A|78spring810618971596|purple|Cup|Unknown|54|prieingantin st| +9584|AAAAAAAAAHFCAAAA|1997-10-27|2000-10-26|Societies find never soft enterprises. Techniques hunt still. Weeks shall not stop houses. Figures win rather men. Legal, possible qualifications shall dev|82.46|65.14|2004001|edu packimporto #1|4|sports-apparel|2|Men|284|eseeingable|medium|23388navy87967413990|salmon|Dozen|Unknown|18|eseeingantin st| +9585|AAAAAAAAAHFCAAAA|2000-10-27||Local households speak rather events. Buildings ring; women give below for the states. About english committe|1.83|65.14|10010015|univamalgamalg #15|4|memory|10|Electronics|198|eingn stought|N/A|23388navy87967413990|brown|Pound|Unknown|33|antieingantin st| +9586|AAAAAAAACHFCAAAA|1997-10-27|1999-10-27|Good rewards copy most. Below alternative proceedings make even. Enough new vehicles can discover by a teams. Neighb|3.74|1.19|3001001|amalgexporti #1|1|newborn|3|Children|174|eseationought|extra large|5630776peach67763712|smoke|Cup|Unknown|48|callyeingantin st| +9587|AAAAAAAACHFCAAAA|1999-10-28|2001-10-26|Generations win too new days. Public pupils could not come parental, identical lectures. Important trees work usually. Sets attack cases. Subsequent, sim|5.80|1.19|3001001|edu packimporto #2|4|sports-apparel|2|Men|174|eseationought|large|5630776peach67763712|peach|Pallet|Unknown|64|ationeingantin st| +9588|AAAAAAAACHFCAAAA|2001-10-27||Generations win too new days. Public pupils could not come parental, identical lectures. Important trees work usually. Sets attack cases. Subsequent, sim|2.39|1.19|3001001|amalgnameless #9|4|athletic shoes|8|Sports|143|eseationought|N/A|902lime4151418518171|pale|Ounce|Unknown|3|eingeingantin st| +9589|AAAAAAAAFHFCAAAA|1997-10-27||R|2.46|1.37|7004010|edu packbrand #10|4|curtains/drapes|7|Home|234|esepriable|N/A|261394970837steel268|slate|Tsp|Unknown|61|n steingantin st| +9590|AAAAAAAAGHFCAAAA|1997-10-27|2000-10-26|Then young applications want perhaps most extra weeks. Citizens see goods. Good, confident drugs seek actually various, controversial lawyers. Welsh, high s|6.71|4.29|5004001|edu packscholar #1|4|classical|5|Music|340|baresepri|N/A|766646649yellow00363|lace|Tbl|Unknown|43|barn stantin st| +9591|AAAAAAAAGHFCAAAA|2000-10-27||Quite national problems cover as most common words. Sad plates may fight as from a plans. Long procedures play slightly following, bad others. Common, d|9.52|5.33|5004001|amalgmaxi #2|11|archery|8|Sports|340|baresepri|N/A|317084381sandy619407|turquoise|Tbl|Unknown|9|oughtn stantin st| +9592|AAAAAAAAIHFCAAAA|1997-10-27|1999-10-27|Patient materials ought to run about sure, minute weeks. Highly other meetings compromis|2.33|1.93|2003001|exportiimporto #1|3|pants|2|Men|762|ablecallyation|medium|078040505lavender323|lavender|Carton|Unknown|52|ablen stantin st| +9593|AAAAAAAAIHFCAAAA|1999-10-28|2001-10-26|Patient materials ought to run about sure, minute weeks. Highly other meetings compromis|6.08|3.10|7013004|exportinameless #4|3|wallpaper|7|Home|623|ablecallyation|N/A|078040505lavender323|smoke|Ton|Unknown|26|prin stantin st| +9594|AAAAAAAAIHFCAAAA|2001-10-27||Ruling products must write usually bad, assistant words. Personal, ava|3.44|3.10|10015007|scholaramalgamalg #7|3|portable|10|Electronics|623|priablecally|N/A|389214lace8496494596|steel|Gram|Unknown|96|esen stantin st| +9595|AAAAAAAALHFCAAAA|1997-10-27||T|64.22|21.19|1002002|importoamalg #2|2|fragrances|1|Women|189|n steingought|large|641spring00694149389|rosy|Tsp|Unknown|9|antin stantin st| +9596|AAAAAAAAMHFCAAAA|1997-10-27|2000-10-26|Alone, small conditions get either likely companies. Inner, long-term patients make hot rebels. Procedures see fi|8.30|6.14|2002001|importoimporto #1|2|shirts|2|Men|247|ationeseable|large|115132003orange03383|rose|Dozen|Unknown|53|callyn stantin st| +9597|AAAAAAAAMHFCAAAA|2000-10-27||All right lonely officers ta|94.50|6.14|4002002|importoedu pack #2|2|mens|4|Shoes|243|ationeseable|medium|24514beige2816572369|royal|Tbl|Unknown|82|ationn stantin st| +9598|AAAAAAAAOHFCAAAA|1997-10-27|1999-10-27|Further fine efforts might not put con|3.20|1.47|6012007|importobrand #7|12|costume|6|Jewelry|830|barprieing|N/A|16white4568749975983|brown|Lb|Unknown|3|eingn stantin st| +9599|AAAAAAAAOHFCAAAA|1999-10-28|2001-10-26|Further fine efforts might not put con|6.94|1.47|6012007|edu packimporto #2|4|sports-apparel|2|Men|396|callyn stpri|extra large|3521279487red4539327|pale|Tbl|Unknown|54|n stn stantin st| +9600|AAAAAAAAOHFCAAAA|2001-10-27||Ministers used to enjoy about to a teachers; indeed arbitrary kinds live. Again big mothe|7.95|6.43|6012007|maxicorp #1|9|womens watch|6|Jewelry|396|callyn stpri|N/A|9403502491139spring1|sandy|Lb|Unknown|30|barbarcallyn st| +9601|AAAAAAAABIFCAAAA|1997-10-27||Prospective, moving pupils gain more here interested conditions. More likely questions boost quite short feelings. Rece|3.81|3.27|1001002|amalgamalg #2|1|dresses|1|Women|489|n steingese|petite|776556408393733sky35|orange|Cup|Unknown|32|oughtbarcallyn st| +9602|AAAAAAAACIFCAAAA|1997-10-27|2000-10-26|Direct, special fires ask long reports. Institutions could examine more than similar words. Also sure|9.09|3.09|2004001|edu packimporto #1|4|sports-apparel|2|Men|123|priableought|extra large|9046928rose257620345|spring|Dozen|Unknown|23|ablebarcallyn st| +9603|AAAAAAAACIFCAAAA|2000-10-27||Direct, special fires ask long reports. Institutions could examine more than similar words. Also sure|1.83|1.18|2004001|importobrand #4|12|costume|6|Jewelry|227|ationableable|N/A|25783spring137134982|spring|Cup|Unknown|3|pribarcallyn st| +9604|AAAAAAAAEIFCAAAA|1997-10-27|1999-10-27|Wide sources supplement physical, famous cells; keen words stop lightly white parties. Public, wrong friends may think such as a women. Great others secure effective cases; |0.80|0.66|6010005|univbrand #5|10|jewelry boxes|6|Jewelry|511|oughtoughtanti|N/A|5pale709761222764457|puff|Box|Unknown|59|esebarcallyn st| +9605|AAAAAAAAEIFCAAAA|1999-10-28|2001-10-26|Wide sources supplement physical, famous cells; keen words stop lightly white parties. Public, wrong friends may think such as a women. Great others secure effective cases; |1.81|1.53|6010005|edu packamalg #2|4|swimwear|1|Women|229|n stableable|small|0843901blanched20535|powder|Ounce|Unknown|30|antibarcallyn st| +9606|AAAAAAAAEIFCAAAA|2001-10-27||Parties will take for example bits. Important, autonomous companies could not prevent new, important ideas. However young concerns use. Even tall|1.94|1.22|3004001|edu packexporti #1|4|school-uniforms|3|Children|229|n stableable|medium|pale1489584156167399|red|Pallet|Unknown|82|callybarcallyn st| +9607|AAAAAAAAHIFCAAAA|1997-10-27||Perfect producers should cover traditionally sure groups. Now bitter years outweigh on a difficulties. Judicial companies nod soon; opening computers achieve almost fr|9.96|4.68|2002002|importoimporto #2|2|shirts|2|Men|254|eseantiable|small|798224355red77357163|lemon|Lb|Unknown|24|ationbarcallyn st| +9608|AAAAAAAAIIFCAAAA|1997-10-27|2000-10-26|Properly recent consultants fly more poor writings. Unusual jobs used to suggest as well right black fans. Adequate eyes cannot provide only to |4.70|3.90|8001009|amalgnameless #9|1|athletic shoes|8|Sports|21|oughtable|N/A|962995583198linen874|sky|Case|Unknown|8|eingbarcallyn st| +9609|AAAAAAAAIIFCAAAA|2000-10-27||Properly recent consultants fly more poor writings. Unusual jobs used to suggest as well right black fans. Adequate eyes cannot provide only to |3.96|1.18|8001009|edu packexporti #2|1|school-uniforms|3|Children|607|oughtable|medium|962995583198linen874|purple|Pallet|Unknown|6|n stbarcallyn st| +9610|AAAAAAAAKIFCAAAA|1997-10-27|1999-10-27|Efficiently evolutionary points tell incredibly immediate contracts.|1.23|0.59|6008003|namelesscorp #3|8|mens watch|6|Jewelry|425|antiableese|N/A|925626257332puff5018|metallic|Pallet|Unknown|19|baroughtcallyn st| +9611|AAAAAAAAKIFCAAAA|1999-10-28|2001-10-26|Efficiently evolutionary points tell incredibly immediate contracts.|1.49|0.59|6008003|edu packimporto #2|8|sports-apparel|2|Men|629|n stablecally|large|925626257332puff5018|turquoise|Bunch|Unknown|13|oughtoughtcallyn st| +9612|AAAAAAAAKIFCAAAA|2001-10-27||Efficiently evolutionary points tell incredibly immediate contracts.|4.04|0.59|6008003|edu packimporto #1|4|sports-apparel|2|Men|324|n stablecally|petite|925626257332puff5018|puff|Lb|Unknown|58|ableoughtcallyn st| +9613|AAAAAAAANIFCAAAA|1997-10-27||Terrible, new bills swap hardly |3.53|1.83|8003004|exportinameless #4|3|basketball|8|Sports|505|antibaranti|N/A|46403262880577steel5|almond|Gross|Unknown|80|prioughtcallyn st| +9614|AAAAAAAAOIFCAAAA|1997-10-27|2000-10-26|Only final contributions could take though specialist experiments. There possible arrangements respect emotions. Public groups seem peaceful spirits. Criminal conservatives ought to give as in |7.48|6.20|9012003|importounivamalg #3|12|home repair|9|Books|301|oughtbarpri|N/A|0751740727royal83331|red|Each|Unknown|42|eseoughtcallyn st| +9615|AAAAAAAAOIFCAAAA|2000-10-27||Only final contributions could take though specialist experiments. There possible arrangements respect emotions. Public groups seem peaceful spirits. Criminal conservatives ought to give as in |9.57|6.20|9012003|brandbrand #6|12|decor|7|Home|301|oughtbarpri|N/A|22681rose72568190089|spring|Box|Unknown|62|antioughtcallyn st| +9616|AAAAAAAAAJFCAAAA|1997-10-27|1999-10-27|Real, brown girls used to go across a effects. Legal questions may assess able, false others. Policies put about; capable provisions get at a opportunities; prime, b|7.98|6.14|9015005|scholarunivamalg #5|15|fiction|9|Books|409|n stbarese|N/A|124376763peach532792|spring|Dram|Unknown|16|callyoughtcallyn st| +9617|AAAAAAAAAJFCAAAA|1999-10-28|2001-10-26|Sales go islands. Grey, main reports know just light, industrial incomes. Just tasks must not make sensible child|4.54|6.14|7001008|amalgbrand #8|1|bathroom|7|Home|504|esebaranti|N/A|2623187162smoke08960|purple|Tbl|Unknown|39|ationoughtcallyn st| +9618|AAAAAAAAAJFCAAAA|2001-10-27||Sales go islands. Grey, main reports know just light, industrial incomes. Just tasks must not make sensible child|0.92|6.14|7001008|univnameless #9|1|flatware|7|Home|504|esebaranti|N/A|2623187162smoke08960|sky|Box|Unknown|20|eingoughtcallyn st| +9619|AAAAAAAADJFCAAAA|1997-10-27||Significant, preliminary boys can remain lightly more pale discussion|2.74|2.43|9001008|amalgmaxi #8|1|arts|9|Books|258|eingantiable|N/A|01203037555salmon236|seashell|Dram|Unknown|31|n stoughtcallyn st| +9620|AAAAAAAAEJFCAAAA|1997-10-27|2000-10-26|Contracts use entire years. Somewhere literary children stop awkward employees. Also new views can locate boxes. Soviet, surprising arms must not loo|4.18|1.25|6015005|scholarbrand #5|15|custom|6|Jewelry|228|eingableable|N/A|146pale2495213613555|seashell|Bundle|Unknown|58|barablecallyn st| +9621|AAAAAAAAEJFCAAAA|2000-10-27||Contracts use entire years. Somewhere literary children stop awkward employees. Also new views can locate boxes. Soviet, surprising arms must not loo|5.21|1.25|6015005|scholarnameless #2|5|fitness|8|Sports|228|eingableable|N/A|90421025396526puff71|misty|Bunch|Unknown|4|oughtablecallyn st| +9622|AAAAAAAAGJFCAAAA|1997-10-27|1999-10-27|Fundamental posts simulate importa|7.66|3.52|7015003|scholarnameless #3|15|tables|7|Home|295|antin stable|N/A|seashell704198193869|spring|Gross|Unknown|28|ableablecallyn st| +9623|AAAAAAAAGJFCAAAA|1999-10-28|2001-10-26|Fundamental posts simulate importa|3.99|3.52|4004002|edu packedu pack #2|4|athletic|4|Shoes|525|antin stable|medium|8439thistle022291869|sienna|Lb|Unknown|23|priablecallyn st| +9624|AAAAAAAAGJFCAAAA|2001-10-27||Fundamental posts simulate importa|4.92|1.62|6008001|namelesscorp #1|4|mens watch|6|Jewelry|516|antin stable|N/A|8439thistle022291869|tan|Dozen|Unknown|7|eseablecallyn st| +9625|AAAAAAAAJJFCAAAA|1997-10-27||Appropriate organisms ought to stay relations. Already open obligations cannot play also small, identical parents. Democratic resources might not resist. Later annual|5.83|3.26|8003006|exportinameless #6|3|basketball|8|Sports|113|prioughtought|N/A|7steel54437489213205|thistle|Each|Unknown|49|antiablecallyn st| +9626|AAAAAAAAKJFCAAAA|1997-10-27|2000-10-26|Now french records shall not taste both. Direct, outside notes cut of course necessary things. European, solid |5.37|4.29|3003001|exportiexporti #1|3|toddlers|3|Children|230|barpriable|extra large|524sandy859034661214|purple|Unknown|Unknown|27|callyablecallyn st| +9627|AAAAAAAAKJFCAAAA|2000-10-27||Small, poor examples could put quite everyday, attractive windows. Inside likely roles can ensure especially skills. Regular, likely shoulders wear most classical walls. Alive, s|7.75|4.29|3003001|namelessbrand #8|3|lighting|7|Home|210|baroughtable|N/A|003832970slate097514|papaya|Box|Unknown|55|ationablecallyn st| +9628|AAAAAAAAMJFCAAAA|1997-10-27|1999-10-27|Civil, beautiful|5.48|3.06|1003001|exportiamalg #1|3|maternity|1|Women|151|oughtantiought|large|272269320rosy7259645|turquoise|Gram|Unknown|7|eingablecallyn st| +9629|AAAAAAAAMJFCAAAA|1999-10-28|2001-10-26|Northern, important standards might give general, fierce cases. Producers may not love for a females. Military years respond ago physical, familiar tears. Then inner |3.15|3.06|2001002|amalgimporto #2|1|accessories|2|Men|314|oughtantiought|extra large|0833051866002puff754|spring|Gross|Unknown|29|n stablecallyn st| +9630|AAAAAAAAMJFCAAAA|2001-10-27||Tired, natural comparisons like to a investments. Others might not make now live, powerful pictures.|8.99|3.06|2001002|exportinameless #1|1|wallpaper|7|Home|314|oughtantiought|N/A|00637pale25873164297|salmon|Lb|Unknown|95|barpricallyn st| +9631|AAAAAAAAPJFCAAAA|1997-10-27||Here necessary patients ought to carry best healthy symptoms; anywhere judicial presents investigate here. Significant cases come most daily military years. New miles could appl|2.32|1.90|1004002|edu packamalg #2|4|swimwear|1|Women|570|barationanti|medium|9drab909444217833266|moccasin|Case|Unknown|66|oughtpricallyn st| +9632|AAAAAAAAAKFCAAAA|1997-10-27|2000-10-26|Interpretations pro|3.42|1.77|2002001|importoimporto #1|2|shirts|2|Men|692|ablen stcally|medium|4907rosy457201662795|navajo|Oz|Unknown|7|ablepricallyn st| +9633|AAAAAAAAAKFCAAAA|2000-10-27||Interpretations pro|2.01|1.32|1001002|amalgamalg #2|2|dresses|1|Women|692|ablen stcally|small|76935886155tomato312|tan|Dozen|Unknown|33|pripricallyn st| +9634|AAAAAAAACKFCAAAA|1997-10-27|1999-10-27|Aspects should reply individual teams. Even possible lives shall give years. Negative, difficult images might commit through a patients. Too central children say onl|1.56|1.31|5004001|edu packscholar #1|4|classical|5|Music|288|eingeingable|N/A|30pale23391026977722|white|Bunch|Unknown|43|esepricallyn st| +9635|AAAAAAAACKFCAAAA|1999-10-28|2001-10-26|Aspects should reply individual teams. Even possible lives shall give years. Negative, difficult images might commit through a patients. Too central children say onl|1.02|0.65|2002002|importoimporto #2|4|shirts|2|Men|446|eingeingable|medium|644215mint5606327282|cornsilk|Lb|Unknown|16|antipricallyn st| +9636|AAAAAAAACKFCAAAA|2001-10-27||Aspects should reply individual teams. Even possible lives shall give years. Negative, difficult images might commit through a patients. Too central children say onl|8.93|7.67|2002002|edu packedu pack #1|4|athletic|4|Shoes|446|callyeseese|extra large|2366401334linen03977|peru|Gram|Unknown|49|callypricallyn st| +9637|AAAAAAAAFKFCAAAA|1997-10-27||Meals may make as aware, ancient events; favorable opponents would think most star, lovely groups. Only quick troops will not clear too whole, western policies. Thus famous tables should not b|1.13|0.35|4003002|exportiedu pack #2|3|kids|4|Shoes|127|ationableought|medium|39633thistle72570304|peach|Bundle|Unknown|86|ationpricallyn st| +9638|AAAAAAAAGKFCAAAA|1997-10-27|2000-10-26|Notably hot hundreds could not mention certainly young, safe bodies. Qualities embrace all sacre|62.69|21.94|10004017|edu packunivamalg #17|4|audio|10|Electronics|942|ableesen st|N/A|075889933646snow3349|tomato|Bundle|Unknown|30|eingpricallyn st| +9639|AAAAAAAAGKFCAAAA|2000-10-27||Notably hot hundreds could not mention certainly young, safe bodies. Qualities embrace all sacre|3.15|21.94|10004017|scholarbrand #8|4|custom|6|Jewelry|207|ableesen st|N/A|075889933646snow3349|plum|Carton|Unknown|59|n stpricallyn st| +9640|AAAAAAAAIKFCAAAA|1997-10-27|1999-10-27|By no means overall horses should listen as a measures. Gothic letters ref|8.82|5.55|1002001|importoamalg #1|2|fragrances|1|Women|207|ationbarable|large|476hot53337392075718|peru|Unknown|Unknown|1|baresecallyn st| +9641|AAAAAAAAIKFCAAAA|1999-10-28|2001-10-26|Industrial families shall shoot all right subsequent studies; months help by a buildings. Forward social exercise|2.19|0.72|1002001|importoamalg #2|2|fragrances|1|Women|4|ationbarable|medium|476hot53337392075718|peach|Lb|Unknown|30|oughtesecallyn st| +9642|AAAAAAAAIKFCAAAA|2001-10-27||Industrial families shall shoot all right subsequent studies; months help by a buildings. Forward social exercise|4.29|0.72|6007003|brandcorp #3|2|pendants|6|Jewelry|82|ableeing|N/A|476hot53337392075718|violet|Ounce|Unknown|65|ableesecallyn st| +9643|AAAAAAAALKFCAAAA|1997-10-27||Girls will try as local shops. Probably spanish dimensions attack recently. Nice drugs come there after a events. Leaves close however far other places; geographically wrong worker|3.05|1.46|9013008|exportiunivamalg #8|13|self-help|9|Books|383|prieingpri|N/A|2450927thistle429785|misty|Tbl|Unknown|26|priesecallyn st| +9644|AAAAAAAAMKFCAAAA|1997-10-27|2000-10-26|Ex|6.75|2.36|8015005|scholarmaxi #5|15|fishing|8|Sports|281|oughteingable|N/A|44seashell3045493874|steel|Dram|Unknown|4|eseesecallyn st| +9645|AAAAAAAAMKFCAAAA|2000-10-27||Ex|40.25|14.08|8015005|amalgunivamalg #10|11|cooking|9|Books|898|oughteingable|N/A|44seashell3045493874|moccasin|Tsp|Unknown|13|antiesecallyn st| +9646|AAAAAAAAOKFCAAAA|1997-10-27|1999-10-27|Miles might stand american, other sources; national r|0.38|0.17|5001001|amalgscholar #1|1|rock|5|Music|657|ationantically|N/A|664misty634101782693|white|Ton|Unknown|56|callyesecallyn st| +9647|AAAAAAAAOKFCAAAA|1999-10-28|2001-10-26|Small materials publish apart from a meanings. Instead strong times give|5.50|1.70|1004002|edu packamalg #2|1|swimwear|1|Women|657|ationantically|medium|5royal40681689688141|orange|Oz|Unknown|93|ationesecallyn st| +9648|AAAAAAAAOKFCAAAA|2001-10-27||Small materials publish apart from a meanings. Instead strong times give|5.69|4.66|1004002|scholarcorp #1|1|earings|6|Jewelry|418|eingoughtese|N/A|38505686seashell3437|pale|Cup|Unknown|32|eingesecallyn st| +9649|AAAAAAAABLFCAAAA|1997-10-27||Roots create now like years. Confident employers used to see at a appointments. Concessions might present warm mechanisms. Finally late entries may |5.08|3.70|7015010|scholarnameless #10|15|tables|7|Home|934|eseprin st|N/A|58422862431352red911|tan|Dram|Unknown|37|n stesecallyn st| +9650|AAAAAAAACLFCAAAA|1997-10-27|2000-10-26|Original, tall patients might benefit and so on alone statutory centres. Further red legs must say readily important, maximum years. Customers could call very phys|2.13|1.06|8003001|exportinameless #1|3|basketball|8|Sports|3|pri|N/A|67233seashell1943772|maroon|Gross|Unknown|93|baranticallyn st| +9651|AAAAAAAACLFCAAAA|2000-10-27||Firmly planned teachers say simple stages. Just internal others must keep distinct years. Daily temporary aims help voters; mercifully wealthy qualities tell m|6.11|1.06|8003001|scholarunivamalg #13|3|karoke|10|Electronics|316|callyoughtpri|N/A|59065092017161rose81|puff|Unknown|Unknown|32|oughtanticallyn st| +9652|AAAAAAAAELFCAAAA|1997-10-27|1999-10-27|Objective|1.28|1.03|9010005|univunivamalg #5|10|travel|9|Books|473|priationese|N/A|51115smoke9624957043|navy|Unknown|Unknown|5|ableanticallyn st| +9653|AAAAAAAAELFCAAAA|1999-10-28|2001-10-26|Well-known, empirical things put more. Local, good reforms admire suitable judges. Children must make a little honestly golden j|3.82|1.03|9010005|amalgcorp #6|1|birdal|6|Jewelry|473|priationese|N/A|51115smoke9624957043|salmon|Bunch|Unknown|83|prianticallyn st| +9654|AAAAAAAAELFCAAAA|2001-10-27||Well-known, empirical things put more. Local, good reforms admire suitable judges. Children must make a little honestly golden j|42.53|1.03|9010005|edu packexporti #1|1|school-uniforms|3|Children|109|n stbarought|petite|58391446458mint46158|salmon|Dram|Unknown|3|eseanticallyn st| +9655|AAAAAAAAHLFCAAAA|1997-10-27||Suitable, public theories make |0.73|0.49|6004008|edu packcorp #8|4|bracelets|6|Jewelry|364|esecallypri|N/A|06898425681146sandy0|dim|Case|Unknown|30|antianticallyn st| +9656|AAAAAAAAILFCAAAA|1997-10-27|2000-10-26|Lives protect comparisons; inside unable sons sha|69.40|57.60|3003001|exportiexporti #1|3|toddlers|3|Children|467|ationcallyese|medium|7906441537857red1778|goldenrod|Lb|Unknown|43|callyanticallyn st| +9657|AAAAAAAAILFCAAAA|2000-10-27||Lives protect comparisons; inside unable sons sha|5.08|57.60|3003001|exportinameless #8|3|basketball|8|Sports|247|ationcallyese|N/A|4973447slate25777941|royal|Gross|Unknown|25|ationanticallyn st| +9658|AAAAAAAAKLFCAAAA|1997-10-27|1999-10-27|Intact times reach recordings; diseases meet very primary workers; economic, unknown aspects inhibit notoriously colleagues. Vague, smal|0.74|0.66|8010009|univmaxi #9|10|pools|8|Sports|753|priantiation|N/A|41335grey18757139030|thistle|Lb|Unknown|63|einganticallyn st| +9659|AAAAAAAAKLFCAAAA|1999-10-28|2001-10-26|Then significant foreigners come more able conservatives|85.33|61.43|8010009|amalgedu pack #2|1|womens|4|Shoes|115|priantiation|petite|242209smoke259518393|powder|Pallet|Unknown|14|n stanticallyn st| +9660|AAAAAAAAKLFCAAAA|2001-10-27||Then significant foreigners come more able conservatives|1.02|61.43|9007001|brandmaxi #1|1|reference|9|Books|53|priantiation|N/A|242209smoke259518393|cream|Oz|Unknown|11|barcallycallyn st| +9661|AAAAAAAANLFCAAAA|1997-10-27||Suitable fingers would go then new men. Efficient, noble drawings think probably |4.22|2.53|8006002|corpnameless #2|6|football|8|Sports|264|esecallyable|N/A|87335772rose36764032|dodger|Unknown|Unknown|43|oughtcallycallyn st| +9662|AAAAAAAAOLFCAAAA|1997-10-27|2000-10-26|Therefore ethnic policies used to mean at a grounds; buildings change smoothly in a |3.64|2.07|1004001|edu packamalg #1|4|swimwear|1|Women|43|priese|medium|068spring72410995530|linen|Carton|Unknown|53|ablecallycallyn st| +9663|AAAAAAAAOLFCAAAA|2000-10-27||Therefore ethnic policies used to mean at a grounds; buildings change smoothly in a |0.19|2.07|1004002|edu packamalg #2|4|swimwear|1|Women|431|oughtpriese|extra large|068spring72410995530|linen|Gross|Unknown|90|pricallycallyn st| +9664|AAAAAAAAAMFCAAAA|1997-10-27|1999-10-27|Permanent, horizontal workers tell bad in a concepts. Indeed familiar parents should make under a researchers. Trees ought to get surely now sound soldiers. Negotiations will talk |4.19|2.68|7001005|amalgbrand #5|1|bathroom|7|Home|127|ationableought|N/A|69lace23235429493059|smoke|Unknown|Unknown|51|esecallycallyn st| +9665|AAAAAAAAAMFCAAAA|1999-10-28|2001-10-26|Permanent, horizontal workers tell bad in a concepts. Indeed familiar parents should make under a researchers. Trees ought to get surely now sound soldiers. Negotiations will talk |3.62|1.15|2002002|importoimporto #2|1|shirts|2|Men|610|baroughtcally|medium|69lace23235429493059|thistle|Bundle|Unknown|43|anticallycallyn st| +9666|AAAAAAAAAMFCAAAA|2001-10-27||Permanent, horizontal workers tell bad in a concepts. Indeed familiar parents should make under a researchers. Trees ought to get surely now sound soldiers. Negotiations will talk |2.88|1.15|2002002|exportiamalg #1|1|maternity|1|Women|610|baroughtcally|extra large|69lace23235429493059|saddle|Tsp|Unknown|6|callycallycallyn st| +9667|AAAAAAAADMFCAAAA|1997-10-27||Words promote years. Always military incomes suggest persistently only mutual agreements. Messages sail all. Constitutional others might produce with a hours. Aside like countrie|2.44|1.04|3004002|edu packexporti #2|4|school-uniforms|3|Children|352|ableantipri|medium|66046628418774pink97|tomato|Bundle|Unknown|17|ationcallycallyn st| +9668|AAAAAAAAEMFCAAAA|1997-10-27|2000-10-26|Also middle cards assemble roles; english question|72.67|34.15|3001001|amalgexporti #1|1|newborn|3|Children|310|baroughtpri|petite|57779862036magenta31|turquoise|Unknown|Unknown|44|eingcallycallyn st| +9669|AAAAAAAAEMFCAAAA|2000-10-27||Foreign, fresh systems cannot restore again for the problems. Likely, familiar minutes may not wait other rights. Wonderful houses find then. Resulting, extra letters must gain;|9.88|34.15|3001002|amalgexporti #2|1|newborn|3|Children|310|baroughtpri|medium|57779862036magenta31|rosy|Ton|Unknown|14|n stcallycallyn st| +9670|AAAAAAAAGMFCAAAA|1997-10-27|1999-10-27|Around important birds r|19.06|10.10|4003001|exportiedu pack #1|3|kids|4|Shoes|883|prieingeing|medium|33986wheat5785059703|dodger|Cup|Unknown|17|barationcallyn st| +9671|AAAAAAAAGMFCAAAA|1999-10-28|2001-10-26|Around important birds r|2.16|10.10|3003002|exportiexporti #2|3|toddlers|3|Children|883|prieingeing|medium|1351393964olive05419|royal|Dozen|Unknown|65|oughtationcallyn st| +9672|AAAAAAAAGMFCAAAA|2001-10-27||Around important birds r|1.86|0.68|3003002|corpnameless #3|3|furniture|7|Home|115|antioughtought|N/A|1351393964olive05419|puff|Tbl|Unknown|95|ableationcallyn st| +9673|AAAAAAAAJMFCAAAA|1997-10-27||New shoes support still. Explicit agencies imply mad, english children. Different times should sit very new sources. Special, alleged streets should relate unusual, industrial|6.14|3.56|4002002|importoedu pack #2|2|mens|4|Shoes|294|esen stable|economy|186509040287magenta6|wheat|N/A|Unknown|13|priationcallyn st| +9674|AAAAAAAAKMFCAAAA|1997-10-27|2000-10-26|Usual, little copies j|5.06|1.72|8002005|importonameless #5|2|baseball|8|Sports|49|n stese|N/A|197586093171spring07|powder|Cup|Unknown|23|eseationcallyn st| +9675|AAAAAAAAKMFCAAAA|2000-10-27||Sizes stay professionally shapes. Mines could try outside |67.57|1.72|8002005|amalgexporti #2|1|newborn|3|Children|314|eseoughtpri|medium|197586093171spring07|cornflower|Oz|Unknown|22|antiationcallyn st| +9676|AAAAAAAAMMFCAAAA|1997-10-27|1999-10-27|Important, liable travellers make to a commentators. Particularly likely colleagues might avoid early quickly good towns; so partial decisions grieve about for a corners. Forever social sources se|1.70|0.68|6002005|importocorp #5|2|diamonds|6|Jewelry|762|ablecallyation|N/A|thistle8632804671497|hot|Tbl|Unknown|36|callyationcallyn st| +9677|AAAAAAAAMMFCAAAA|1999-10-28|2001-10-26|Important, liable travellers make to a commentators. Particularly likely colleagues might avoid early quickly good towns; so partial decisions grieve about for a corners. Forever social sources se|7.76|3.95|4001002|amalgedu pack #2|2|womens|4|Shoes|96|ablecallyation|extra large|445salmon55494381001|sienna|Gram|Unknown|22|ationationcallyn st| +9678|AAAAAAAAMMFCAAAA|2001-10-27||Cold patients used to relocate patently. Political teachers will say then modest minutes. Religious windows rear|5.02|3.95|1004001|edu packamalg #1|2|swimwear|1|Women|96|ablecallyation|large|9178439beige55328201|turquoise|Bundle|Unknown|49|eingationcallyn st| +9679|AAAAAAAAPMFCAAAA|1997-10-27||As general sites seem less. Also alone relations might not li|5.76|3.74|5004002|edu packscholar #2|4|classical|5|Music|118|eingoughtought|N/A|606969698738medium82|indian|Unknown|Unknown|19|n stationcallyn st| +9680|AAAAAAAAANFCAAAA|1997-10-27|2000-10-26|Un|8.50|5.10|4004001|edu packedu pack #1|4|athletic|4|Shoes|120|barableought|small|12slate9887577913158|royal|Dram|Unknown|80|bareingcallyn st| +9681|AAAAAAAAANFCAAAA|2000-10-27||Major parents get only past a products. Obvious structures make now; original, british parties end totally. Most working policies would honour years. Arguments know; |9.32|5.10|6001002|amalgcorp #2|4|birdal|6|Jewelry|120|barableought|N/A|12slate9887577913158|khaki|N/A|Unknown|14|oughteingcallyn st| +9682|AAAAAAAACNFCAAAA|1997-10-27|1999-10-27|Top questions resign opposite characteristics; classic, local patients dine very years. Local police wear ever environmental teachers. Necessary, male rights ens|2.30|1.56|6002003|importocorp #3|2|diamonds|6|Jewelry|461|oughtcallyese|N/A|497758yellow67396475|pink|Ounce|Unknown|56|ableeingcallyn st| +9683|AAAAAAAACNFCAAAA|1999-10-28|2001-10-26|Standards could feed still roots. Colonies will talk better participants. Vital stations make above; cautious children might not boil national republics. Free allies would not |3.42|2.70|6011004|amalgbrand #4|2|semi-precious|6|Jewelry|461|oughtcallyese|N/A|497758yellow67396475|light|Gross|Unknown|4|prieingcallyn st| +9684|AAAAAAAACNFCAAAA|2001-10-27||Small, actual cases might introduce now individual goods. Electronic lawyers use fond goa|7.87|2.70|10004012|edu packunivamalg #12|4|audio|10|Electronics|461|oughtcallyese|N/A|290060641thistle2824|white|Tbl|Unknown|28|eseeingcallyn st| +9685|AAAAAAAAFNFCAAAA|1997-10-27||Years make recent leaves. Perhaps far kinds respond just. Glorious forces matter. Grounds shall not give just oth|0.32|0.16|9015002|scholarunivamalg #2|15|fiction|9|Books|224|eseableable|N/A|622037852violet04335|ghost|Tsp|Unknown|2|antieingcallyn st| +9686|AAAAAAAAGNFCAAAA|1997-10-27|2000-10-26|Natural, particular books feed home to a police. Authorities used to play adequately. Children adapt |7.95|2.62|8003007|exportinameless #7|3|basketball|8|Sports|122|ableableought|N/A|03287124514908white5|violet|Gross|Unknown|16|callyeingcallyn st| +9687|AAAAAAAAGNFCAAAA|2000-10-27||Very other features give new members. New, valuable deposits shall go yesterday vast changes. Too past countries might take too true, old powers. Other, recent prices keep systems. Heavy |1.04|2.62|8003007|brandcorp #8|7|pendants|6|Jewelry|122|ableableought|N/A|671lavender899437762|sandy|Bunch|Unknown|14|ationeingcallyn st| +9688|AAAAAAAAINFCAAAA|1997-10-27|1999-10-27|Equally current members take enough negative, various words. Workshops could prevent. So solar personnel must want in the industries. |4.44|2.88|3002001|importoexporti #1|2|infants|3|Children|377|ationationpri|medium|11turquoise900451849|sandy|Case|Unknown|6|eingeingcallyn st| +9689|AAAAAAAAINFCAAAA|1999-10-28|2001-10-26|Equally current members take enough negative, various words. Workshops could prevent. So solar personnel must want in the industries. |3.28|1.47|6001002|amalgcorp #2|2|birdal|6|Jewelry|377|ationationpri|N/A|11turquoise900451849|hot|Tsp|Unknown|63|n steingcallyn st| +9690|AAAAAAAAINFCAAAA|2001-10-27||Years activate areas. Intense|1.18|1.47|6001002|exportiimporto #1|3|pants|2|Men|388|eingeingpri|petite|813516327561smoke247|violet|Tbl|Unknown|43|barn stcallyn st| +9691|AAAAAAAALNFCAAAA|1997-10-27||Swiss, lovely residents turn by the positions. Current, responsible books advise incredible method|4.67|2.75|5004002|edu packscholar #2|4|classical|5|Music|230|barpriable|N/A|5tan2382642594628174|powder|Tbl|Unknown|61|oughtn stcallyn st| +9692|AAAAAAAAMNFCAAAA|1997-10-27|2000-10-26|Deaf, final societies shall not appear expensive relationships. Legal police use|2.35|1.99|1003001|exportiamalg #1|3|maternity|1|Women|33|pripri|large|7459560184743yellow0|slate|Pound|Unknown|4|ablen stcallyn st| +9693|AAAAAAAAMNFCAAAA|2000-10-27||More national owners would not enable newly in a attitudes. Employees persuade cool ita|85.56|1.99|1003001|importoimporto #2|3|shirts|2|Men|33|pripri|petite|905saddle27119622507|goldenrod|Carton|Unknown|18|prin stcallyn st| +9694|AAAAAAAAONFCAAAA|1997-10-27|1999-10-27|Sufficient, current jobs shall appear neither little flights. Times can remain by a |0.87|0.73|6014007|edu packbrand #7|14|estate|6|Jewelry|972|ableationn st|N/A|922lavender855507509|seashell|Case|Unknown|28|esen stcallyn st| +9695|AAAAAAAAONFCAAAA|1999-10-28|2001-10-26|Rights could make ahead. As good foreigners occur too central minutes|2.16|0.73|6014007|amalgedu pack #2|14|womens|4|Shoes|972|ableationn st|large|151452286136rose6967|sky|Case|Unknown|2|antin stcallyn st| +9696|AAAAAAAAONFCAAAA|2001-10-27||Rights could make ahead. As good foreigners occur too central minutes|95.44|79.21|6014007|edu packscholar #1|4|classical|5|Music|972|ableationn st|N/A|5802tan2196935130552|peach|Dozen|Unknown|34|callyn stcallyn st| +9697|AAAAAAAABOFCAAAA|1997-10-27||Highly english employees matter never for a thousands; then rough nights must believe e|0.62|0.21|4002002|importoedu pack #2|2|mens|4|Shoes|522|ableableanti|medium|07578354603284royal2|peru|Each|Unknown|39|ationn stcallyn st| +9698|AAAAAAAACOFCAAAA|1997-10-27|2000-10-26|Civil, patient companies|76.13|55.57|3001001|amalgexporti #1|1|newborn|3|Children|203|pribarable|medium|852404moccasin646834|puff|Tbl|Unknown|23|eingn stcallyn st| +9699|AAAAAAAACOFCAAAA|2000-10-27||Fresh pp. may undermine glad, old costs. No doubt urban resources could work even clothes. Forces deal as new, secret parents; together definite police shall cov|44.30|55.57|3001001|corpunivamalg #10|1|musical|10|Electronics|482|pribarable|N/A|62651174peru58813566|medium|Gross|Unknown|22|n stn stcallyn st| +9700|AAAAAAAAEOFCAAAA|1997-10-27|1999-10-27|N|77.79|27.22|2001001|amalgimporto #1|1|accessories|2|Men|124|eseableought|medium|red06792946905734793|steel|Oz|Unknown|60|barbarationn st| +9701|AAAAAAAAEOFCAAAA|1999-10-28|2001-10-26|N|5.77|3.57|2001001|scholaramalgamalg #12|15|portable|10|Electronics|16|eseableought|N/A|79417676700sandy4285|rosy|Ounce|Unknown|10|oughtbarationn st| +9702|AAAAAAAAEOFCAAAA|2001-10-27||Eyes turn newly right women. Particularly willing types must think then european issues. Grand, pleasant cells enter only; really sexual sentences find seriously in the men. Finally|5.29|3.57|4001001|amalgedu pack #1|15|womens|4|Shoes|16|callyought|medium|7011560003puff376860|saddle|Cup|Unknown|25|ablebarationn st| +9703|AAAAAAAAHOFCAAAA|1997-10-27||Conditions go as however friendly falls. Expressions learn ultimate numbers; yesterday main steps think to a programmes; soft days will know today efficient|4.43|1.59|6015008|scholarbrand #8|15|custom|6|Jewelry|103|pribarought|N/A|75157778906misty6413|hot|Lb|Unknown|37|pribarationn st| +9704|AAAAAAAAIOFCAAAA|1997-10-27|2000-10-26|Gradually general affairs come at a gardens. Members may liv|28.90|11.27|10014015|edu packamalgamalg #15|14|automotive|10|Electronics|386|callyeingpri|N/A|7595942058351rose646|metallic|Lb|Unknown|63|esebarationn st| +9705|AAAAAAAAIOFCAAAA|2000-10-27||Cultural requirements shall identify below labour years; safe, wealthy ch|92.84|11.27|10014015|amalgbrand #2|11|semi-precious|6|Jewelry|145|antieseought|N/A|962pink7035776253645|pink|Ounce|Unknown|61|antibarationn st| +9706|AAAAAAAAKOFCAAAA|1997-10-27|1999-10-27|Unusual years might buy others. Enough mutual facilities could not respond views. Differences s|1.01|0.56|9007011|brandmaxi #11|7|reference|9|Books|90|barn st|N/A|5032186904rose807093|smoke|Pallet|Unknown|30|callybarationn st| +9707|AAAAAAAAKOFCAAAA|1999-10-28|2001-10-26|Small years shall |36.62|0.56|10001001|amalgunivamalg #1|1|cameras|10|Electronics|90|barn st|N/A|0353white54290471514|aquamarine|Dozen|Unknown|57|ationbarationn st| +9708|AAAAAAAAKOFCAAAA|2001-10-27||Small years shall |4.15|0.56|8005009|scholarnameless #9|1|fitness|8|Sports|90|barn st|N/A|993793094909905tan11|moccasin|Ounce|Unknown|47|eingbarationn st| +9709|AAAAAAAANOFCAAAA|1997-10-27||Right, standard scenes annou|5.37|1.87|3001002|amalgexporti #2|1|newborn|3|Children|302|ablebarpri|petite|10gainsboro069381744|rosy|Bundle|Unknown|16|n stbarationn st| +9710|AAAAAAAAOOFCAAAA|1997-10-27|2000-10-26|Severe, whole partners must surprise too to the children. Minutes rush foreign, early lists. Strange,|2.07|1.63|3001001|amalgexporti #1|1|newborn|3|Children|675|antiationcally|petite|3sienna4960013425336|tan|Ounce|Unknown|6|baroughtationn st| +9711|AAAAAAAAOOFCAAAA|2000-10-27||Severe, whole partners must surprise too to the children. Minutes rush foreign, early lists. Strange,|2.49|1.63|3001001|exportiunivamalg #5|1|dvd/vcr players|10|Electronics|675|antiationcally|N/A|2986812786puff883151|sandy|Pound|Unknown|8|oughtoughtationn st| +9712|AAAAAAAAAPFCAAAA|1997-10-27|1999-10-27|Little, particular jobs become most hard symptoms. Regular, everyday systems cannot benefit in the diseases. International, flexible stones return for a elements. Future tables wou|1.59|1.04|9008005|namelessmaxi #5|8|romance|9|Books|171|oughtationought|N/A|28547576485705sky318|orange|N/A|Unknown|99|ableoughtationn st| +9713|AAAAAAAAAPFCAAAA|1999-10-28|2001-10-26|Little, particular jobs become most hard symptoms. Regular, everyday systems cannot benefit in the diseases. International, flexible stones return for a elements. Future tables wou|2.09|1.46|5004002|edu packscholar #2|8|classical|5|Music|51|oughtanti|N/A|28547576485705sky318|steel|Cup|Unknown|45|prioughtationn st| +9714|AAAAAAAAAPFCAAAA|2001-10-27||Facilities deny. Patients provide yesterday on a payments; deliberately ancient grounds work most schools; also logical workers thi|2.05|1.46|9009007|maximaxi #7|9|science|9|Books|98|oughtanti|N/A|28547576485705sky318|olive|Lb|Unknown|29|eseoughtationn st| +9715|AAAAAAAADPFCAAAA|1997-10-27||Special, american camps keep consequently late local walls. New, twin talks get once more mental books. Bad mal|1.90|1.36|5001002|amalgscholar #2|1|rock|5|Music|170|barationought|N/A|67512violet352837223|olive|Tbl|Unknown|31|antioughtationn st| +9716|AAAAAAAAEPFCAAAA|1997-10-27|2000-10-26|Local tears trust thus. Temperatures grip customers. Growing, tall years continue personally. Religious, impressive mechanisms might not foll|8.41|6.47|1001001|amalgamalg #1|1|dresses|1|Women|987|ationeingn st|petite|27141812869658plum83|thistle|Oz|Unknown|17|callyoughtationn st| +9717|AAAAAAAAEPFCAAAA|2000-10-27||Only likely accidents shall launch necessary opponents. Leaders may not help indeed appropriate grounds. Particularly boring children b|7.40|4.58|2003002|exportiimporto #2|3|pants|2|Men|987|ationeingn st|N/A|27141812869658plum83|violet|Lb|Unknown|4|ationoughtationn st| +9718|AAAAAAAAGPFCAAAA|1997-10-27|1999-10-27|Companies ignore as against the eyes. Camps used to defend away leading skills; new kinds should fit net, mediterranean days. Parties must try parties. Children see successfully minute appoint|9.22|6.45|4002001|importoedu pack #1|2|mens|4|Shoes|177|ationationought|large|65316452292slate5047|brown|Box|Unknown|5|eingoughtationn st| +9719|AAAAAAAAGPFCAAAA|1999-10-28|2001-10-26|Companies ignore as against the eyes. Camps used to defend away leading skills; new kinds should fit net, mediterranean days. Parties must try parties. Children see successfully minute appoint|9.83|6.45|3003002|exportiexporti #2|2|toddlers|3|Children|6|ationationought|extra large|65316452292slate5047|plum|Each|Unknown|78|n stoughtationn st| +9720|AAAAAAAAGPFCAAAA|2001-10-27||Companies ignore as against the eyes. Camps used to defend away leading skills; new kinds should fit net, mediterranean days. Parties must try parties. Children see successfully minute appoint|3.97|6.45|3001001|amalgexporti #1|2|newborn|3|Children|431|oughtpriese|petite|65316452292slate5047|purple|N/A|Unknown|42|barableationn st| +9721|AAAAAAAAJPFCAAAA|1997-10-27||Living, concerned qualifications emphasise over true stories; |0.49|0.21|6010002|univbrand #2|10|jewelry boxes|6|Jewelry|746|callyeseation|N/A|603099362orange37518|turquoise|Carton|Unknown|30|oughtableationn st| +9722|AAAAAAAAKPFCAAAA|1997-10-27|2000-10-26|Games come. Rough forces used to worry truly on a boats. Nowhere historical pp. would not minimize too often necessary advantages. Original, competitive students withdraw muscles. Normal, new advanta|27.64|11.60|10010016|univamalgamalg #16|10|memory|10|Electronics|64|esecally|N/A|197royal482092201938|peru|Dozen|Unknown|50|ableableationn st| +9723|AAAAAAAAKPFCAAAA|2000-10-27||New obligations disrupt above quite different changes|1.49|11.60|10010016|importoamalg #2|2|fragrances|1|Women|578|eingationanti|small|1075472734123grey936|peru|Lb|Unknown|63|priableationn st| +9724|AAAAAAAAMPFCAAAA|1997-10-27|1999-10-27|European, so|0.13|0.09|4002001|importoedu pack #1|2|mens|4|Shoes|323|priablepri|large|185016765750rose1241|peru|Cup|Unknown|26|eseableationn st| +9725|AAAAAAAAMPFCAAAA|1999-10-28|2001-10-26|European, so|8.89|0.09|4002001|univbrand #6|10|jewelry boxes|6|Jewelry|482|ableeingese|N/A|185016765750rose1241|powder|Gross|Unknown|21|antiableationn st| +9726|AAAAAAAAMPFCAAAA|2001-10-27||European, so|4.92|0.09|8014007|edu packmaxi #7|10|tennis|8|Sports|208|ableeingese|N/A|856370296151powder84|moccasin|Tbl|Unknown|3|callyableationn st| +9727|AAAAAAAAPPFCAAAA|1997-10-27||Wet, awful affairs argue. New teachers consider moral, english terms. Customers sue almost possible, economic minutes. Domestic, confident parents ma|9.55|2.86|5004002|edu packscholar #2|4|classical|5|Music|105|antibarought|N/A|763296metallic079561|brown|Pallet|Unknown|1|ationableationn st| +9728|AAAAAAAAAAGCAAAA|1997-10-27|2000-10-26|Longer other prices give here old examples. Much silent police might go including a perceptions. Early, new programmes promote too for a laws. Actors should not speak as relationships. Children cou|6.28|2.26|9013009|exportiunivamalg #9|13|self-help|9|Books|337|ationpripri|N/A|57949491royal7555524|cream|Tsp|Unknown|25|eingableationn st| +9729|AAAAAAAAAAGCAAAA|2000-10-27||Longer other prices give here old examples. Much silent police might go including a perceptions. Early, new programmes promote too for a laws. Actors should not speak as relationships. Children cou|4.10|2.26|9013009|edu packmaxi #10|13|tennis|8|Sports|337|ationpripri|N/A|57949491royal7555524|linen|Ton|Unknown|28|n stableationn st| +9730|AAAAAAAACAGCAAAA|1997-10-27|1999-10-27|Excellent, general officers discover complex clients. Old, protective customers wou|2.91|1.42|2002001|importoimporto #1|2|shirts|2|Men|220|barableable|large|antique9419878306084|coral|Ounce|Unknown|49|barpriationn st| +9731|AAAAAAAACAGCAAAA|1999-10-28|2001-10-26|More other ways gain then specialist others. Ago large arguments would get never measures. Players work however too unusual careers. Dogs will|3.55|3.15|6004004|edu packcorp #4|2|bracelets|6|Jewelry|220|barableable|N/A|776568116391floral90|peach|Gross|Unknown|48|oughtpriationn st| +9732|AAAAAAAACAGCAAAA|2001-10-27||More other ways gain then specialist others. Ago large arguments would get never measures. Players work however too unusual careers. Dogs will|9.53|3.15|3004001|edu packexporti #1|2|school-uniforms|3|Children|46|barableable|large|776568116391floral90|white|Bundle|Unknown|56|ablepriationn st| +9733|AAAAAAAAFAGCAAAA|1997-10-27||Personnel can leave old, wide cases. Parents might check obviously serious, other members. Too grim thousands cannot give well. Passengers hold players; modes |1.07|0.39|1003002|exportiamalg #2|3|maternity|1|Women|674|eseationcally|large|73036818837lemon5082|rose|Dram|Unknown|16|pripriationn st| +9734|AAAAAAAAGAGCAAAA|1997-10-27|2000-10-26|Ever unable pools ought to say at least perfect, young cars. Elderly, interesting weeks tell approx|1.04|0.54|2003001|exportiimporto #1|3|pants|2|Men|507|ationbaranti|small|0302245834rose860462|red|Ton|Unknown|3|esepriationn st| +9735|AAAAAAAAGAGCAAAA|2000-10-27||Usual countries ought to consider just additional ta|4.94|0.54|2003001|univbrand #8|10|jewelry boxes|6|Jewelry|507|ationbaranti|N/A|0302245834rose860462|sky|Bundle|Unknown|24|antipriationn st| +9736|AAAAAAAAIAGCAAAA|1997-10-27|1999-10-27|Well local years contain. Probably able proposals need precious, |1.05|0.86|6015001|scholarbrand #1|15|custom|6|Jewelry|275|antiationable|N/A|900073royal247137028|almond|Gross|Unknown|38|callypriationn st| +9737|AAAAAAAAIAGCAAAA|1999-10-28|2001-10-26|Often english levels can pass differently in the advantages. Private, handsome machines expose so left products. Local, statistical candidates maintain hesitantly rat|2.93|1.28|2002002|importoimporto #2|15|shirts|2|Men|557|ationantianti|medium|82144151slate5609405|spring|Oz|Unknown|30|ationpriationn st| +9738|AAAAAAAAIAGCAAAA|2001-10-27||Often english levels can pass differently in the advantages. Private, handsome machines expose so left products. Local, statistical candidates maintain hesitantly rat|5.73|1.94|4001001|amalgedu pack #1|1|womens|4|Shoes|557|ationantianti|economy|754670440spring80349|red|Each|Unknown|23|eingpriationn st| +9739|AAAAAAAALAGCAAAA|1997-10-27||Ready, kind rocks help indeed|6.61|2.44|3004002|edu packexporti #2|4|school-uniforms|3|Children|61|oughtcally|extra large|25087401190130lemon9|yellow|Pallet|Unknown|31|n stpriationn st| +9740|AAAAAAAAMAGCAAAA|1997-10-27|2000-10-26|Before social members could last. Primitive, statistical winds can work issues. Fundamental authorities throw for a men. Far, black subjects visit ru|88.88|38.21|4004001|edu packedu pack #1|4|athletic|4|Shoes|475|antiationese|extra large|519508012214tan78883|saddle|Carton|Unknown|6|bareseationn st| +9741|AAAAAAAAMAGCAAAA|2000-10-27||Before social members could last. Primitive, statistical winds can work issues. Fundamental authorities throw for a men. Far, black subjects visit ru|3.80|1.67|4004001|exportischolar #2|3|pop|5|Music|177|ationationought|N/A|7chiffon117045764806|powder|Lb|Unknown|29|oughteseationn st| +9742|AAAAAAAAOAGCAAAA|1997-10-27|1999-10-27|Extensive things will come precisely boys. Just post-war consequ|3.04|1.67|5003001|exportischolar #1|3|pop|5|Music|28|eingable|N/A|22168805824pink33327|rosy|Tsp|Unknown|44|ableeseationn st| +9743|AAAAAAAAOAGCAAAA|1999-10-28|2001-10-26|Extensive things will come precisely boys. Just post-war consequ|5.50|3.02|5003001|importoedu pack #2|2|mens|4|Shoes|28|eingable|small|22168805824pink33327|orchid|Lb|Unknown|8|prieseationn st| +9744|AAAAAAAAOAGCAAAA|2001-10-27||Extensive things will come precisely boys. Just post-war consequ|1.97|1.59|5003001|exportiedu pack #1|3|kids|4|Shoes|171|oughtationought|extra large|4steel21424572082772|lavender|Pound|Unknown|40|eseeseationn st| +9745|AAAAAAAABBGCAAAA|1997-10-27||About existing results ensure as foreign so|15.86|8.08|7011006|amalgnameless #6|11|accent|7|Home|497|ationn stese|N/A|990cyan9853053756112|snow|N/A|Unknown|26|antieseationn st| +9746|AAAAAAAACBGCAAAA|1997-10-27|2000-10-26|As simple matters find english, royal values. Beliefs shall |6.75|3.24|3003001|exportiexporti #1|3|toddlers|3|Children|970|barationn st|large|3tomato2090005193323|light|Tsp|Unknown|28|callyeseationn st| +9747|AAAAAAAACBGCAAAA|2000-10-27||Subjects can apply presently musical reports. Certainly unlikely shares let absolutely european rates. Western, special sciences want indeed. Well visua|0.57|0.28|7009008|maxibrand #8|3|mattresses|7|Home|970|barationn st|N/A|royal553911004127728|pink|Bundle|Unknown|20|ationeseationn st| +9748|AAAAAAAAEBGCAAAA|1997-10-27|1999-10-27|Streets should create so chemical, close pp.. Principal falls might create as to a offices; much freq|1.08|0.74|1004001|edu packamalg #1|4|swimwear|1|Women|82|ableeing|medium|0064sky6890999816999|peach|Pound|Unknown|16|eingeseationn st| +9749|AAAAAAAAEBGCAAAA|1999-10-28|2001-10-26|Streets should create so chemical, close pp.. Principal falls might create as to a offices; much freq|3.09|0.74|1004001|importoimporto #2|4|shirts|2|Men|82|ableeing|petite|0064sky6890999816999|purple|Cup|Unknown|59|n steseationn st| +9750|AAAAAAAAEBGCAAAA|2001-10-27||Easier appropriate companies consider. Other words would not divorce then national forces. Later silly figures should set. Elderly, criminal schools might defend great, adequate eyes|5.04|0.74|6012007|importobrand #7|4|costume|6|Jewelry|82|ableeing|N/A|0064sky6890999816999|hot|Oz|Unknown|25|barantiationn st| +9751|AAAAAAAAHBGCAAAA|1997-10-27||Major, able children make always small priests. Young hands aim for a locations; compatible, female films finish a|8.90|5.34|2003002|exportiimporto #2|3|pants|2|Men|721|oughtableation|medium|43619pink70284565550|medium|Bundle|Unknown|33|oughtantiationn st| +9752|AAAAAAAAIBGCAAAA|1997-10-27|2000-10-26|Everyday rates debate silently f|3.09|2.03|2003001|exportiimporto #1|3|pants|2|Men|113|prioughtought|extra large|82532342783navajo039|spring|Unknown|Unknown|42|ableantiationn st| +9753|AAAAAAAAIBGCAAAA|2000-10-27||Everyday rates debate silently f|7.84|3.37|2003001|edu packnameless #4|3|glassware|7|Home|113|prioughtought|N/A|thistle6974648687811|pink|Ounce|Unknown|48|priantiationn st| +9754|AAAAAAAAKBGCAAAA|1997-10-27|1999-10-27|Strong miles relate weapons. Isolated, individual duties would not mean later expensive customers. Leaders argue necessary schools|2.06|1.83|10011014|amalgamalgamalg #14|11|disk drives|10|Electronics|423|priableese|N/A|606puff8373350928603|frosted|Unknown|Unknown|27|eseantiationn st| +9755|AAAAAAAAKBGCAAAA|1999-10-28|2001-10-26|Vague, twin years should mean selectively useful years. Buses promote disabled reasons; large customers shall en|49.09|1.83|10011014|amalgexporti #2|1|newborn|3|Children|423|priableese|small|606puff8373350928603|rosy|Bunch|Unknown|13|antiantiationn st| +9756|AAAAAAAAKBGCAAAA|2001-10-27||Then comfortable studies should make meanwhile. Friends start busy, useful problems. Post-war implications could not give following, unusual children. Young, dif|1.26|0.51|10011014|corpnameless #7|1|furniture|7|Home|532|priableese|N/A|6849thistle256065536|cornflower|Pound|Unknown|40|callyantiationn st| +9757|AAAAAAAANBGCAAAA|1997-10-27||Due, helpful items provide hours. Other differences see so unable theories. Visual times used to know patterns. Very re|2.52|1.28|10012017|importoamalgamalg #17|12|monitors|10|Electronics|202|ablebarable|N/A|5spring9967309502363|powder|Cup|Unknown|2|ationantiationn st| +9758|AAAAAAAAOBGCAAAA|1997-10-27|2000-10-26|Differences ring always always bad members. Characteristics prevent sufficiently regional|9.75|5.94|4001001|amalgedu pack #1|1|womens|4|Shoes|806|callybareing|extra large|82522981405263snow65|turquoise|Bunch|Unknown|28|eingantiationn st| +9759|AAAAAAAAOBGCAAAA|2000-10-27||Details shall try with a eyes; arms could not go others. Reasonable, separate sales land expert amounts. Cases ought to recommend perhaps everywhere dear pp.; |2.83|5.94|5002002|importoscholar #2|1|country|5|Music|806|callybareing|N/A|842015301smoke613507|yellow|Carton|Unknown|7|n stantiationn st| +9760|AAAAAAAAACGCAAAA|1997-10-27|1999-10-27|Cards might complete recently against a rules; easy shoulders p|4.61|4.14|8005001|scholarnameless #1|5|fitness|8|Sports|106|callybarought|N/A|762374115255rose4584|cyan|Dram|Unknown|87|barcallyationn st| +9761|AAAAAAAAACGCAAAA|1999-10-28|2001-10-26|Cards might complete recently against a rules; easy shoulders p|1.36|4.14|8005001|edu packexporti #2|5|school-uniforms|3|Children|106|callybarought|medium|steel198808289865938|burnished|Case|Unknown|59|oughtcallyationn st| +9762|AAAAAAAAACGCAAAA|2001-10-27||Cards might complete recently against a rules; easy shoulders p|8.43|4.14|8005001|exportiunivamalg #7|5|self-help|9|Books|229|n stableable|N/A|310682646839912snow9|frosted|N/A|Unknown|25|ablecallyationn st| +9763|AAAAAAAADCGCAAAA|1997-10-27||Games cannot see all available emotions. Days would not b|4.46|3.70|3003002|exportiexporti #2|3|toddlers|3|Children|529|n stableanti|extra large|21808512506264tan199|puff|Each|Unknown|14|pricallyationn st| +9764|AAAAAAAAECGCAAAA|1997-10-27|2000-10-26|Spatial, chief boats discuss deeper theoretical, similar groups. Members might take of |1.14|0.69|5001001|amalgscholar #1|1|rock|5|Music|365|anticallypri|N/A|536777335271thistle4|snow|Case|Unknown|33|esecallyationn st| +9765|AAAAAAAAECGCAAAA|2000-10-27||Spatial, chief boats discuss deeper theoretical, similar groups. Members might take of |1.36|0.69|8016006|corpmaxi #6|1|golf|8|Sports|152|anticallypri|N/A|536777335271thistle4|steel|Tsp|Unknown|1|anticallyationn st| +9766|AAAAAAAAGCGCAAAA|1997-10-27|1999-10-27|More other parties catch difficult, canadian types. New criteria shrink gratefully nuclear units. P|1.99|1.43|10008009|namelessunivamalg #9|8|scanners|10|Electronics|361|oughtcallypri|N/A|490263193yellow90412|powder|Ton|Unknown|48|callycallyationn st| +9767|AAAAAAAAGCGCAAAA|1999-10-28|2001-10-26|White, nat|2.84|1.43|7010008|univnameless #8|8|flatware|7|Home|361|oughtcallypri|N/A|490263193yellow90412|red|Box|Unknown|30|ationcallyationn st| +9768|AAAAAAAAGCGCAAAA|2001-10-27||White, nat|0.17|1.43|3002001|importoexporti #1|8|infants|3|Children|426|oughtcallypri|small|57879violet518686104|violet|Tbl|Unknown|30|eingcallyationn st| +9769|AAAAAAAAJCGCAAAA|1997-10-27||Annual, remote details would know only to a eyes. Laws construct teachers. Little armed prices used to charge economic, associated masters. Home available firms may tell however |3.30|2.11|9004002|edu packmaxi #2|4|entertainments|9|Books|515|antioughtanti|N/A|306277275brown382145|khaki|Pound|Unknown|43|n stcallyationn st| +9770|AAAAAAAAKCGCAAAA|1997-10-27|2000-10-26|Notably other chemicals might carry again there interesting problems. Electronic, new foods recall legs.|2.81|1.65|7010001|univnameless #1|10|flatware|7|Home|244|eseeseable|N/A|4spring1859959749042|royal|Cup|Unknown|31|barationationn st| +9771|AAAAAAAAKCGCAAAA|2000-10-27||Northern bacteria used to know seemingly houses. Other, desperate groups could like so. Small proposals put public, effective forces; small parties should lo|9.58|1.65|7010001|importonameless #2|2|baseball|8|Sports|244|eseeseable|N/A|161427spring22853698|brown|Each|Unknown|92|oughtationationn st| +9772|AAAAAAAAMCGCAAAA|1997-10-27|1999-10-27|Others cannot run most arts. Managerial, mere systems get and so on usual, other places. Addresses must not ceas|5.24|2.82|4002001|importoedu pack #1|2|mens|4|Shoes|53|prianti|extra large|071211038805spring84|sandy|N/A|Unknown|27|ableationationn st| +9773|AAAAAAAAMCGCAAAA|1999-10-28|2001-10-26|Others cannot run most arts. Managerial, mere systems get and so on usual, other places. Addresses must not ceas|9.92|2.82|8007004|brandnameless #4|2|hockey|8|Sports|53|prianti|N/A|64187034391purple440|royal|Case|Unknown|7|priationationn st| +9774|AAAAAAAAMCGCAAAA|2001-10-27||Others cannot run most arts. Managerial, mere systems get and so on usual, other places. Addresses must not ceas|2.31|1.50|8007004|scholarbrand #7|15|custom|6|Jewelry|53|prianti|N/A|64187034391purple440|light|Gram|Unknown|2|eseationationn st| +9775|AAAAAAAAPCGCAAAA|1997-10-27||Representatives will not get all others. Beforehand real papers grow bold books. Straight for|4.19|1.63|10012001|importoamalgamalg #1|12|monitors|10|Electronics|179|n stationought|N/A|9560pink805831158545|chocolate|Tbl|Unknown|49|antiationationn st| +9776|AAAAAAAAADGCAAAA|1997-10-27|2000-10-26|As scientific conservatives shall try of course military, new parents. Angry, huge differences could try more studen|5.00|2.95|10013002|exportiamalgamalg #2|13|stereo|10|Electronics|171|oughtationought|N/A|2715purple0817660133|puff|Bunch|Unknown|26|callyationationn st| +9777|AAAAAAAAADGCAAAA|2000-10-27||As scientific conservatives shall try of course military, new parents. Angry, huge differences could try more studen|8.64|2.95|7014008|edu packnameless #8|13|glassware|7|Home|690|barn stcally|N/A|2715purple0817660133|lawn|N/A|Unknown|56|ationationationn st| +9778|AAAAAAAACDGCAAAA|1997-10-27|1999-10-27|Substantial, old days lose ahead for the messages. Victims involve views. Soviet emotions defy ways. Coming, c|3.46|1.79|4002001|importoedu pack #1|2|mens|4|Shoes|551|oughtantianti|medium|9319419664564slate96|tomato|Gram|Unknown|21|eingationationn st| +9779|AAAAAAAACDGCAAAA|1999-10-28|2001-10-26|Substantial, old days lose ahead for the messages. Victims involve views. Soviet emotions defy ways. Coming, c|93.76|1.79|6011004|amalgbrand #4|11|semi-precious|6|Jewelry|933|oughtantianti|N/A|74139942530014linen2|saddle|Pallet|Unknown|33|n stationationn st| +9780|AAAAAAAACDGCAAAA|2001-10-27||Coloured words could exercise urban, french polici|2.53|1.56|6011004|scholarbrand #5|11|custom|6|Jewelry|635|antiprically|N/A|74139942530014linen2|deep|Lb|Unknown|28|bareingationn st| +9781|AAAAAAAAFDGCAAAA|1997-10-27||Names prefer; stands define independent affairs. Usual, intimate members ought to see indeed plus a boys. Also co||1.13|7010002|||flatware|7||207|ationbarable||||Tbl|Unknown|55|oughteingationn st| +9782|AAAAAAAAGDGCAAAA|1997-10-27|2000-10-26|Traditional waters may afford there|1.27|0.48|9010003|univunivamalg #3|10|travel|9|Books|693|prin stcally|N/A|5526metallic04067304|goldenrod|Ton|Unknown|17|ableeingationn st| +9783|AAAAAAAAGDGCAAAA|2000-10-27||New, front trends can say. Good, pale women may get miles; others condemn. Minutes might increase|5.48|0.48|9010003|importoedu pack #2|10|mens|4|Shoes|152|ableantiought|N/A|5526metallic04067304|tan|Dozen|Unknown|27|prieingationn st| +9784|AAAAAAAAIDGCAAAA|1997-10-27|1999-10-27|Important, different task|74.21|34.13|4001001|amalgedu pack #1|1|womens|4|Shoes|98|eingn st|medium|8665spring7686726473|cream|Dozen|Unknown|65|eseeingationn st| +9785|AAAAAAAAIDGCAAAA|1999-10-28|2001-10-26|Brown, social offenders might expect just full, special comments. Male doors run adequate months. Spat|4.36|34.13|4004002|edu packedu pack #2|1|athletic|4|Shoes|98|eingn st|petite|662007495khaki867291|spring|N/A|Unknown|11|antieingationn st| +9786|AAAAAAAAIDGCAAAA|2001-10-27||Brown, social offenders might expect just full, special comments. Male doors run adequate months. Spat|0.44|0.18|10015012|scholaramalgamalg #12|15|portable|10|Electronics|98|eingn st|N/A|7sienna5666515396252|slate|Bundle|Unknown|10|callyeingationn st| +9787|AAAAAAAALDGCAAAA|1997-10-27||Arms claim earlier into a countries. Important hours stay now for the attitudes. A|6.37|4.77|10004013|edu packunivamalg #13|4|audio|10|Electronics|764|esecallyation|N/A|0736lawn854675813240|purple|Gram|Unknown|8|ationeingationn st| +9788|AAAAAAAAMDGCAAAA|1997-10-27|2000-10-26|Groups live then. Free, victorian members may not aid all players. Genes dive in the farmers. Groups make on a|6.93|2.97|4003001|exportiedu pack #1|3|kids|4|Shoes|439|n stpriese|small|17112468rosy73695749|plum|Case|Unknown|6|eingeingationn st| +9789|AAAAAAAAMDGCAAAA|2000-10-27||Groups live then. Free, victorian members may not aid all players. Genes dive in the farmers. Groups make on a|3.88|2.97|1003002|exportiamalg #2|3|maternity|1|Women|357|ationantipri|petite|17112468rosy73695749|wheat|Bundle|Unknown|51|n steingationn st| +9790|AAAAAAAAODGCAAAA|1997-10-27|1999-10-27|Bizarre, national goods pass in the police. Isolated colours use always prices. Also creative patients say even in the numbers. Proposed brothers find services. Crazy, whole aspects woul|54.41|48.96|7011001|amalgnameless #1|11|accent|7|Home|573|priationanti|N/A|794849snow9162332145|hot|Tbl|Unknown|10|barn stationn st| +9791|AAAAAAAAODGCAAAA|1999-10-28|2001-10-26|Bizarre, national goods pass in the police. Isolated colours use always prices. Also creative patients say even in the numbers. Proposed brothers find services. Crazy, whole aspects woul|4.49|48.96|2003002|exportiimporto #2|3|pants|2|Men|283|priationanti|medium|pale6196208163253894|yellow|Bunch|Unknown|7|oughtn stationn st| +9792|AAAAAAAAODGCAAAA|2001-10-27||Quick, small guns think indeed. Benefits know. Etc essential seats would say fields. Cases join again relations|3.26|2.37|9004001|edu packmaxi #1|4|entertainments|9|Books|105|antibarought|N/A|7816601011pink543543|pale|Ounce|Unknown|10|ablen stationn st| +9793|AAAAAAAABEGCAAAA|1997-10-27||Values plan negotiations. Deaths extend annually. Nice, green processes keep with a|8.01|7.20|6009002|maxicorp #2|9|womens watch|6|Jewelry|526|callyableanti|N/A|9305691498indian3620|smoke|Unknown|Unknown|40|prin stationn st| +9794|AAAAAAAACEGCAAAA|1997-10-27|2000-10-26|Imperial, firm ways used to overwhelm rarely|4.35|1.82|4002001|importoedu pack #1|2|mens|4|Shoes|162|ablecallyought|medium|46209416339068wheat9|blue|Box|Unknown|18|esen stationn st| +9795|AAAAAAAACEGCAAAA|2000-10-27||Important, quiet women want tiny events; secret, central organisations must not decide downstairs personal futures. So far men render good, protective t|0.09|1.82|4002001|exportischolar #2|3|pop|5|Music|887|ablecallyought|N/A|70364peru08070160701|linen|Gross|Unknown|62|antin stationn st| +9796|AAAAAAAAEEGCAAAA|1997-10-27|1999-10-27|Partners used to lessen subject manufacturers. Possible others rehearse also. Turkish, other stories regard then shows. Static, social foreigners ought to anger yet in a items. Very national |5.76|4.37|3001001|amalgexporti #1|1|newborn|3|Children|493|prin stese|extra large|6papaya8549057863183|white|Carton|Unknown|10|callyn stationn st| +9797|AAAAAAAAEEGCAAAA|1999-10-28|2001-10-26|Partners used to lessen subject manufacturers. Possible others rehearse also. Turkish, other stories regard then shows. Static, social foreigners ought to anger yet in a items. Very national |9.48|5.68|3001001|importoscholar #2|2|country|5|Music|32|ablepri|N/A|6papaya8549057863183|chartreuse|Oz|Unknown|39|ationn stationn st| +9798|AAAAAAAAEEGCAAAA|2001-10-27||Partners used to lessen subject manufacturers. Possible others rehearse also. Turkish, other stories regard then shows. Static, social foreigners ought to anger yet in a items. Very national |8.20|5.68|7004009|edu packbrand #9|4|curtains/drapes|7|Home|323|priablepri|N/A|6papaya8549057863183|rose|Oz|Unknown|1|eingn stationn st| +9799|AAAAAAAAHEGCAAAA|1997-10-27||Other groups should not veto more areas. Apparent, other causes could not find for a representatives; willing arguments leave;|1.23|0.63|1002002|importoamalg #2|2|fragrances|1|Women|271|oughtationable|small|695736508445tomato93|salmon|Ounce|Unknown|6|n stn stationn st| +9800|AAAAAAAAIEGCAAAA|1997-10-27|2000-10-26|Passages use also nice, strong schools. Fine w|9.49|8.44|6009001|maxicorp #1|9|womens watch|6|Jewelry|609|n stbarcally|N/A|9534875257thistle582|powder|Bundle|Unknown|8|barbareingn st| +9801|AAAAAAAAIEGCAAAA|2000-10-27||Half soviet steps get from a shoes. Leaders can fill all in a ways. As natural machines look well. Desirable, local tickets present well studies. National companies cannot protect existing profits|1.91|8.44|6009001|amalgamalg #2|9|dresses|1|Women|609|n stbarcally|large|9534875257thistle582|indian|Lb|Unknown|1|oughtbareingn st| +9802|AAAAAAAAKEGCAAAA|1997-10-27|1999-10-27|Fond, exceptional goals can ask necessarily only clear skills. Always full v|4.67|3.54|3002001|importoexporti #1|2|infants|3|Children|215|antioughtable|medium|8904yellow9390909538|burlywood|N/A|Unknown|26|ablebareingn st| +9803|AAAAAAAAKEGCAAAA|1999-10-28|2001-10-26|Full, large-scale children get possibly details. There geographical practices should not increase both ill, formal trees; only modern parents might recreat|6.66|3.54|4001002|amalgedu pack #2|2|womens|4|Shoes|215|antioughtable|medium|374011sienna27865760|slate|Case|Unknown|41|pribareingn st| +9804|AAAAAAAAKEGCAAAA|2001-10-27||Full, large-scale children get possibly details. There geographical practices should not increase both ill, formal trees; only modern parents might recreat|4.96|2.13|4001002|exportischolar #1|3|pop|5|Music|598|antioughtable|N/A|374011sienna27865760|powder|Ton|Unknown|79|esebareingn st| +9805|AAAAAAAANEGCAAAA|1997-10-27||Selectively impossible brothers used to suffer social, english cars. Heads could not become undo|4.15|2.86|4004002|edu packedu pack #2|4|athletic|4|Shoes|546|callyeseanti|extra large|34136237789steel8005|bisque|Dram|Unknown|65|antibareingn st| +9806|AAAAAAAAOEGCAAAA|1997-10-27|2000-10-26|Leaves can arrive never to a u|98.39|36.40|6014007|edu packbrand #7|14|estate|6|Jewelry|470|barationese|N/A|56576517frosted34265|purple|Dram|Unknown|98|callybareingn st| +9807|AAAAAAAAOEGCAAAA|2000-10-27||British reasons come all just stupid properties. Justly other institutions must enable to a securities. Never total pains withstand|0.66|36.40|8006008|corpnameless #8|6|football|8|Sports|470|barationese|N/A|6979yellow9160854878|thistle|Each|Unknown|82|ationbareingn st| +9808|AAAAAAAAAFGCAAAA|1997-10-27|1999-10-27|Hands may not take in a affairs. Early details shall keep often weekly, relevant months. Local, informal companie|2.29|0.93|7003009|exportibrand #9|3|kids|7|Home|151|oughtantiought|N/A|544280267921575red55|white|Pound|Unknown|35|eingbareingn st| +9809|AAAAAAAAAFGCAAAA|1999-10-28|2001-10-26|Regional sets buy mild, young lives. Head terms fail about though available circles. Lexical fri|7.06|4.51|7003009|importoamalg #2|3|fragrances|1|Women|356|oughtantiought|large|064733214snow0952889|magenta|Pound|Unknown|26|n stbareingn st| +9810|AAAAAAAAAFGCAAAA|2001-10-27||Regional sets buy mild, young lives. Head terms fail about though available circles. Lexical fri|1.72|4.51|8005001|scholarnameless #1|3|fitness|8|Sports|233|oughtantiought|N/A|9602921204239coral56|burlywood|N/A|Unknown|45|baroughteingn st| +9811|AAAAAAAADFGCAAAA|1997-10-27||Reasons begin much. Illegal, basic materials gain in a months. So small levels counter heavily whole lives; big rooms win clear, alone months. Necessary, commo|0.35|0.17|4002002|importoedu pack #2|2|mens|4|Shoes|178|eingationought|medium|9midnight45576437871|pink|Pallet|Unknown|41|oughtoughteingn st| +9812|AAAAAAAAEFGCAAAA|1997-10-27|2000-10-26|Also local lights would talk completely into a companies. Most possible genes would give to a concerns. Very other inc|9.36|3.65|3004001|edu packexporti #1|4|school-uniforms|3|Children|394|esen stpri|large|9234281drab900294513|pale|Tbl|Unknown|19|ableoughteingn st| +9813|AAAAAAAAEFGCAAAA|2000-10-27||Here short curtains would not gain finally damp tickets. Problems hear other individuals; users go in the statements. Significantly young girls pl|9.04|6.59|7006004|corpbrand #4|4|rugs|7|Home|368|esen stpri|N/A|9234281drab900294513|purple|Lb|Unknown|9|prioughteingn st| +9814|AAAAAAAAGFGCAAAA|1997-10-27|1999-10-27|Also multiple lives might see just well nice wars. Exact, reasonable regulations used to move simply old|54.16|27.62|5004001|edu packscholar #1|4|classical|5|Music|435|antipriese|N/A|77sky750144013631086|sandy|Gram|Unknown|19|eseoughteingn st| +9815|AAAAAAAAGFGCAAAA|1999-10-28|2001-10-26|Also multiple lives might see just well nice wars. Exact, reasonable regulations used to move simply old|9.59|3.06|4001002|amalgedu pack #2|4|womens|4|Shoes|332|antipriese|economy|9654018805violet4747|medium|Gross|Unknown|21|antioughteingn st| +9816|AAAAAAAAGFGCAAAA|2001-10-27||Also multiple lives might see just well nice wars. Exact, reasonable regulations used to move simply old|55.98|47.02|10014008|edu packamalgamalg #8|4|automotive|10|Electronics|332|ablepripri|N/A|3791988542turquoise6|moccasin|Oz|Unknown|28|callyoughteingn st| +9817|AAAAAAAAJFGCAAAA|1997-10-27||Short-term students occupy never visitors. Remaining, fine owners should treat. Weapons discover finally more traditional centres; ev|0.93|0.43|10003009|exportiunivamalg #9|3|dvd/vcr players|10|Electronics|453|priantiese|N/A|427176993red37987621|smoke|Case|Unknown|40|ationoughteingn st| +9818|AAAAAAAAKFGCAAAA|1997-10-27|2000-10-26|Women forget |9.50|5.60|4001001|amalgedu pack #1|1|womens|4|Shoes|378|eingationpri|large|4plum735888268779226|misty|Tbl|Unknown|45|eingoughteingn st| +9819|AAAAAAAAKFGCAAAA|2000-10-27||Annual, normal days shall not sing monthly, careful lives. Full, light intervals would call tomorrow long methods; usually correct variations address enough dead years. General, sorry player|9.73|3.69|4001001|exportiexporti #2|1|toddlers|3|Children|401|eingationpri|medium|16881sandy5139650412|puff|Ounce|Unknown|84|n stoughteingn st| +9820|AAAAAAAAMFGCAAAA|1997-10-27|1999-10-27|Radical, other girls laugh. Various offices change available, central sections. Deli|1.27|0.54|9011005|amalgunivamalg #5|11|cooking|9|Books|275|antiationable|N/A|296620336tomato05393|lime|Dram|Unknown|35|barableeingn st| +9821|AAAAAAAAMFGCAAAA|1999-10-28|2001-10-26|Radical, other girls laugh. Various offices change available, central sections. Deli|3.88|1.59|9011005|importoscholar #2|2|country|5|Music|239|n stpriable|N/A|296620336tomato05393|pale|Gram|Unknown|20|oughtableeingn st| +9822|AAAAAAAAMFGCAAAA|2001-10-27||Less female affairs should apply even. Important |1.36|0.82|9011005|exportiamalg #1|2|maternity|1|Women|239|n stpriable|medium|76242yellow452521978|pink|Tsp|Unknown|20|ableableeingn st| +9823|AAAAAAAAPFGCAAAA|1997-10-27||Now total stations prefer anywhere more imperial times. Particular, international years carry as to a criteria. Qualifications determine with a others. Villages shall not go directly versio|2.43|0.87|9013008|exportiunivamalg #8|13|self-help|9|Books|253|priantiable|N/A|15565557powder128635|puff|Tbl|Unknown|10|priableeingn st| +9824|AAAAAAAAAGGCAAAA|1997-10-27|2000-10-26|Illegal, limited r|2.17|1.64|6008001|namelesscorp #1|8|mens watch|6|Jewelry|304|esebarpri|N/A|627169207red69533062|pale|Lb|Unknown|37|eseableeingn st| +9825|AAAAAAAAAGGCAAAA|2000-10-27||Illegal, limited r|7.36|4.71|6008001|importonameless #6|8|paint|7|Home|304|esebarpri|N/A|276317198violet71878|chocolate|Bundle|Unknown|4|antiableeingn st| +9826|AAAAAAAACGGCAAAA|1997-10-27|1999-10-27|Hands might not take away in a police. Others could arrange even. New companies c|5.95|5.11|1001001|amalgamalg #1|1|dresses|1|Women|239|n stpriable|medium|5745252160metallic58|lime|Pound|Unknown|24|callyableeingn st| +9827|AAAAAAAACGGCAAAA|1999-10-28|2001-10-26|Hands might not take away in a police. Others could arrange even. New companies c|4.09|5.11|4004002|edu packedu pack #2|4|athletic|4|Shoes|238|n stpriable|large|5745252160metallic58|green|Dozen|Unknown|69|ationableeingn st| +9828|AAAAAAAACGGCAAAA|2001-10-27||Strongly thin girls roll etc only great principles. Other sciences supply short years. Sensible conditions shall come almost systems|4.93|5.11|7001009|amalgbrand #9|1|bathroom|7|Home|598|n stpriable|N/A|5745252160metallic58|smoke|Gram|Unknown|22|eingableeingn st| +9829|AAAAAAAAFGGCAAAA|1997-10-27||As territorial fingers develop then humans. Industries put then extra, anxious pairs. Letters give of course voluntary, central times. Committees join thus. Areas produce so long gold eyes. Taxes c|36.14|23.85|7001010|amalgbrand #10|1|bathroom|7|Home|489|n steingese|N/A|579967084drab1963641|puff|Ounce|Unknown|16|n stableeingn st| +9830|AAAAAAAAGGGCAAAA|1997-10-27|2000-10-26|Compulsory, distinguished communications shall not kno|1.65|1.17|3001001|amalgexporti #1|1|newborn|3|Children|629|n stablecally|medium|54598629611745tan383|steel|Dozen|Unknown|61|barprieingn st| +9831|AAAAAAAAGGGCAAAA|2000-10-27||Compulsory, distinguished communications shall not kno|0.52|0.40|5003002|exportischolar #2|1|pop|5|Music|629|n stablecally|N/A|0523133127226cream01|medium|Box|Unknown|23|oughtprieingn st| +9832|AAAAAAAAIGGCAAAA|1997-10-27|1999-10-27|Civil numbers should minimise. Reasonable|3.48|1.39|7012003|importonameless #3|12|paint|7|Home|125|antiableought|N/A|khaki188227313810438|sandy|Box|Unknown|17|ableprieingn st| +9833|AAAAAAAAIGGCAAAA|1999-10-28|2001-10-26|Civil numbers should minimise. Reasonable|5.59|2.34|7012003|namelessnameless #4|12|outdoor|8|Sports|125|antiableought|N/A|974sienna68775797741|maroon|Lb|Unknown|38|priprieingn st| +9834|AAAAAAAAIGGCAAAA|2001-10-27||Central, huge loans might evacuate important, interesting notions. Practices fall afterwards great years; commercial, |1.54|2.34|7012003|importounivamalg #7|12|home repair|9|Books|772|ableationation|N/A|974sienna68775797741|sandy|Box|Unknown|18|eseprieingn st| +9835|AAAAAAAALGGCAAAA|1997-10-27||Ideas might implement enthusiastically. Normal, optimistic pr|6.67|5.26|10005010|scholarunivamalg #10|5|karoke|10|Electronics|45|antiese|N/A|739649225230saddle78|steel|N/A|Unknown|26|antiprieingn st| +9836|AAAAAAAAMGGCAAAA|1997-10-27|2000-10-26|Too national attitudes fall just particular subjects. Almost important circumstances say too keenly excellent implications. Years achieve by a countries. Only left terms live forward costs|6.45|5.67|2004001|edu packimporto #1|4|sports-apparel|2|Men|800|barbareing|economy|730540rose2827045426|peru|Carton|Unknown|98|callyprieingn st| +9837|AAAAAAAAMGGCAAAA|2000-10-27||Too national attitudes fall just particular subjects. Almost important circumstances say too keenly excellent implications. Years achieve by a countries. Only left terms live forward costs|1.96|1.72|2004001|importoexporti #2|4|infants|3|Children|9|barbareing|petite|9gainsboro3058042439|peach|Carton|Unknown|11|ationprieingn st| +9838|AAAAAAAAOGGCAAAA|1997-10-27|1999-10-27|Healthy, due states would like. Also evolutionary miles recove|6.59|3.75|2004001|edu packimporto #1|4|sports-apparel|2|Men|10|barought|extra large|turquoise63357524098|grey|Cup|Unknown|36|eingprieingn st| +9839|AAAAAAAAOGGCAAAA|1999-10-28|2001-10-26|Healthy, due states would like. Also evolutionary miles recove|1.58|0.48|2004002|edu packimporto #2|4|sports-apparel|2|Men|10|barought|large|turquoise63357524098|peru|Lb|Unknown|3|n stprieingn st| +9840|AAAAAAAAOGGCAAAA|2001-10-27||Labour, alternative plans should agree free, tired classes. Carefu|5.30|1.96|1004001|edu packamalg #1|4|swimwear|1|Women|10|barought|extra large|399837682235purple68|grey|Bundle|Unknown|48|bareseeingn st| +9841|AAAAAAAABHGCAAAA|1997-10-27||Girls become from a intervals. Changes shall crash further very initial families. Total, possible systems advertise|5.34|2.56|9006002|corpmaxi #2|6|parenting|9|Books|129|n stableought|N/A|672puff1814471058505|steel|Oz|Unknown|59|oughteseeingn st| +9842|AAAAAAAACHGCAAAA|1997-10-27|2000-10-26|Other, subject states get soon. Probably complete years shall achieve. Comm|2.73|1.91|4003001|exportiedu pack #1|3|kids|4|Shoes|29|n stable|petite|569681877mint9050322|sky|Gram|Unknown|17|ableeseeingn st| +9843|AAAAAAAACHGCAAAA|2000-10-27||Never right words carry suddenly parts. Sites play mea|7.57|1.91|1004002|edu packamalg #2|3|swimwear|1|Women|125|antiableought|petite|9533seashell56743696|papaya|Box|Unknown|28|prieseeingn st| +9844|AAAAAAAAEHGCAAAA|1997-10-27|1999-10-27|As other models might know so ever private processes. Social, white feet encompass here. Tryi|4.90|3.67|7006005|corpbrand #5|6|rugs|7|Home|459|n stantiese|N/A|3998493dark853553188|slate|Gross|Unknown|29|eseeseeingn st| +9845|AAAAAAAAEHGCAAAA|1999-10-28|2001-10-26|As other models might know so ever private processes. Social, white feet encompass here. Tryi|2.61|0.86|7006005|importoamalgamalg #3|6|monitors|10|Electronics|173|priationought|N/A|3998493dark853553188|purple|Ounce|Unknown|61|antieseeingn st| +9846|AAAAAAAAEHGCAAAA|2001-10-27||As other models might know so ever private processes. Social, white feet encompass here. Tryi|7.09|2.48|7006005|amalgmaxi #7|6|archery|8|Sports|173|priationought|N/A|3998493dark853553188|sienna|Tbl|Unknown|19|callyeseeingn st| +9847|AAAAAAAAHHGCAAAA|1997-10-27||Suddenly small terms realise very from a parts. Apart private gifts think then common services. Legs reduce well in order extensive wounds. Quite adverse|3.84|1.88|4002002|importoedu pack #2|2|mens|4|Shoes|25|antiable|medium|81973856851197drab39|papaya|Bundle|Unknown|82|ationeseeingn st| +9848|AAAAAAAAIHGCAAAA|1997-10-27|2000-10-26|About competitive members could not screen; however free performances could not give here in the studies; soft laws deal plans. Bodies complete all right fem|1.18|0.43|9011009|amalgunivamalg #9|11|cooking|9|Books|221|oughtableable|N/A|147546188wheat006326|olive|Unknown|Unknown|27|eingeseeingn st| +9849|AAAAAAAAIHGCAAAA|2000-10-27||About competitive members could not screen; however free performances could not give here in the studies; soft laws deal plans. Bodies complete all right fem|8.76|5.51|3001002|amalgexporti #2|1|newborn|3|Children|573|priationanti|large|35590778774460snow12|deep|Ton|Unknown|59|n steseeingn st| +9850|AAAAAAAAKHGCAAAA|1997-10-27|1999-10-27|Projects try. Part-time characteristics count. Exactly public women involve overseas spare, dangerous boards. Clear needs could deal as theories. |2.64|1.53|3002001|importoexporti #1|2|infants|3|Children|169|n stcallyought|large|38turquoise044804024|tomato|Ounce|Unknown|71|barantieingn st| +9851|AAAAAAAAKHGCAAAA|1999-10-28|2001-10-26|Projects try. Part-time characteristics count. Exactly public women involve overseas spare, dangerous boards. Clear needs could deal as theories. |4.72|1.53|7014002|edu packnameless #2|2|glassware|7|Home|408|eingbarese|N/A|5359295716red7337758|yellow|Cup|Unknown|20|oughtantieingn st| +9852|AAAAAAAAKHGCAAAA|2001-10-27||Other, past laws vote often. Trees go unable |3.97|1.82|7014002|edu packnameless #3|2|glassware|7|Home|408|eingbarese|N/A|5359295716red7337758|sandy|Dozen|Unknown|41|ableantieingn st| +9853|AAAAAAAANHGCAAAA|1997-10-27||Then main incidents will not account always great, complete gaps. Together necessary representatives would not give from |9.72|4.08|5004002|edu packscholar #2|4|classical|5|Music|6|cally|N/A|90223351rose36906962|turquoise|Bundle|Unknown|1|priantieingn st| +9854|AAAAAAAAOHGCAAAA|1997-10-27|2000-10-26|Human, socia|1.30|0.89|3002001|importoexporti #1|2|infants|3|Children|108|eingbarought|large|2142890147360peach62|royal|Tsp|Unknown|34|eseantieingn st| +9855|AAAAAAAAOHGCAAAA|2000-10-27||Shoes will not go studies; also nice feet say probably national, other figures. Domestic units ought to protect old emotions. Quite little companies understand quite things. Positive thought|5.45|3.54|8012006|importomaxi #6|2|guns|8|Sports|108|eingbarought|N/A|2142890147360peach62|indian|Each|Unknown|48|antiantieingn st| +9856|AAAAAAAAAIGCAAAA|1997-10-27|1999-10-27|Sensitive factors shall keep cert|6.94|6.24|3003001|exportiexporti #1|3|toddlers|3|Children|318|eingoughtpri|extra large|977289952navy4163639|green|Ounce|Unknown|70|callyantieingn st| +9857|AAAAAAAAAIGCAAAA|1999-10-28|2001-10-26|Sensitive factors shall keep cert|79.53|6.24|6011002|amalgbrand #2|3|semi-precious|6|Jewelry|318|eingoughtpri|N/A|977289952navy4163639|magenta|Carton|Unknown|12|ationantieingn st| +9858|AAAAAAAAAIGCAAAA|2001-10-27||Able models might join ultimately eyes. Statements find modern, worthy boa|4.69|6.24|8010009|univmaxi #9|10|pools|8|Sports|318|eingoughtpri|N/A|43716738127grey34983|light|Dram|Unknown|18|eingantieingn st| +9859|AAAAAAAADIGCAAAA|1997-10-27||Permanent payments make rarely like a exhibitions. Studies should reflect too worlds. Head, new components|8.98|2.96|5003002|exportischolar #2|3|pop|5|Music|591|oughtn stanti|N/A|12719383seashell5331|turquoise|Bundle|Unknown|19|n stantieingn st| +9860|AAAAAAAAEIGCAAAA|1997-10-27|2000-10-26|Positive|4.07|1.50|10006001|corpunivamalg #1|6|musical|10|Electronics|323|priablepri|N/A|2993330lemon81634184|salmon|Lb|Unknown|34|barcallyeingn st| +9861|AAAAAAAAEIGCAAAA|2000-10-27||European shoulders can see also independent problems. Future, flat ladies must not save per a bands. Relations could know shareholders. Normal workers shal|4.89|1.50|10006001|univunivamalg #10|6|travel|9|Books|323|priablepri|N/A|2993330lemon81634184|tomato|Pallet|Unknown|31|oughtcallyeingn st| +9862|AAAAAAAAGIGCAAAA|1997-10-27|1999-10-27|Traditional, small police will mi|2.92|1.81|10016003|corpamalgamalg #3|16|wireless|10|Electronics|140|bareseought|N/A|4525wheat58867858226|grey|Bunch|Unknown|9|ablecallyeingn st| +9863|AAAAAAAAGIGCAAAA|1999-10-28|2001-10-26|Either judic|4.72|1.81|3001002|amalgexporti #2|1|newborn|3|Children|618|eingoughtcally|extra large|7yellow8573436208759|hot|Bundle|Unknown|22|pricallyeingn st| +9864|AAAAAAAAGIGCAAAA|2001-10-27||Spaces may cut. Neatly female conditions can wish. Proper agreements know for a residents;|5.19|4.20|6016001|corpbrand #1|1|consignment|6|Jewelry|618|eingoughtcally|N/A|9813081156039steel00|lavender|Oz|Unknown|24|esecallyeingn st| +9865|AAAAAAAAJIGCAAAA|1997-10-27||Really considerable eyes provide. Here certain purposes push |3.16|2.62|3002002|importoexporti #2|2|infants|3|Children|200|barbarable|large|1434202222516slate52|hot|Box|Unknown|10|anticallyeingn st| +9866|AAAAAAAAKIGCAAAA|1997-10-27|2000-10-26|Green accounts feel immediate, grateful employees. Necessary galleries will stop only. Bad, previous scholars|7.23|2.60|5004001|edu packscholar #1|4|classical|5|Music|35|antipri|N/A|9747266steel29326699|pink|Bundle|Unknown|6|callycallyeingn st| +9867|AAAAAAAAKIGCAAAA|2000-10-27||Special regulations may not like appropriate benefits; subsidies get easily etc different for|2.41|0.79|1002002|importoamalg #2|4|fragrances|1|Women|35|antipri|large|672156157navajo51840|wheat|Lb|Unknown|73|ationcallyeingn st| +9868|AAAAAAAAMIGCAAAA|1997-10-27|1999-10-27|Carefully physical hotels must put together; similar details cannot appreciate by a standards. Rates can break m|6.63|4.97|9002005|importomaxi #5|2|business|9|Books|117|ationoughtought|N/A|724453924432985rosy0|red|N/A|Unknown|47|eingcallyeingn st| +9869|AAAAAAAAMIGCAAAA|1999-10-28|2001-10-26|Carefully physical hotels must put together; similar details cannot appreciate by a standards. Rates can break m|93.93|78.90|9002005|exportiunivamalg #10|3|dvd/vcr players|10|Electronics|211|ationoughtought|N/A|724453924432985rosy0|sienna|Gram|Unknown|16|n stcallyeingn st| +9870|AAAAAAAAMIGCAAAA|2001-10-27||Only scientific users accept later of course new blues; cases know buildings. Criteria must not find obviously real men. Of course full phot|9.04|7.95|9002005|scholarbrand #1|5|blinds/shades|7|Home|18|ationoughtought|N/A|724453924432985rosy0|peach|N/A|Unknown|51|barationeingn st| +9871|AAAAAAAAPIGCAAAA|1997-10-27||So important pounds would not score precisely at a cells. Clear campaigns would fall now monthly databases. Processes ought to stand in par|37.00|14.06|7003002|exportibrand #2|3|kids|7|Home|174|eseationought|N/A|9887puff344725662830|tomato|Pound|Unknown|13|oughtationeingn st| +9872|AAAAAAAAAJGCAAAA|1997-10-27|2000-10-26|Free feet get generally lacking countries. There sure things build even natural spirits; then young teachers call much. Minutes say long available perceptions. Minute, excellent restaurants go straigh|3.92|1.52|2004001|edu packimporto #1|4|sports-apparel|2|Men|609|n stbarcally|medium|217royal916113668030|tomato|Lb|Unknown|53|ableationeingn st| +9873|AAAAAAAAAJGCAAAA|2000-10-27||Free feet get generally lacking countries. There sure things build even natural spirits; then young teachers call much. Minutes say long available perceptions. Minute, excellent restaurants go straigh|0.44|0.15|2004001|edu packcorp #2|4|bracelets|6|Jewelry|609|n stbarcally|N/A|217royal916113668030|pale|Bundle|Unknown|51|priationeingn st| +9874|AAAAAAAACJGCAAAA|1997-10-27|1999-10-27|Dead small factors receive full, defensive miles. Small, organisational services atta|3.76|2.40|4003001|exportiedu pack #1|3|kids|4|Shoes|460|barcallyese|N/A|145797108172forest77|salmon|Cup|Unknown|58|eseationeingn st| +9875|AAAAAAAACJGCAAAA|1999-10-28|2001-10-26|Dead small factors receive full, defensive miles. Small, organisational services atta|4.01|2.40|4003001|importoscholar #2|3|country|5|Music|460|barcallyese|N/A|145797108172forest77|navy|Box|Unknown|92|antiationeingn st| +9876|AAAAAAAACJGCAAAA|2001-10-27||Dead small factors receive full, defensive miles. Small, organisational services atta|4.24|2.40|7010007|univnameless #7|3|flatware|7|Home|460|barcallyese|N/A|145797108172forest77|white|Oz|Unknown|57|callyationeingn st| +9877|AAAAAAAAFJGCAAAA|1997-10-27||More difficult forces will not force faithfully recordings. Exports agree simply too teenage patients. Civil, deep police ought to feel currently|9.66|8.01|6005006|scholarcorp #6|5|earings|6|Jewelry|394|esen stpri|N/A|00rose64396312777195|moccasin|Carton|Unknown|53|ationationeingn st| +9878|AAAAAAAAGJGCAAAA|1997-10-27|2000-10-26|Things stand alone but for the situations. Dirty engines look almost communists. Fair files see effectively for a changes. Soft tracks used to give however religious forms. Boards allow on|3.35|2.47|4003001|exportiedu pack #1|3|kids|4|Shoes|644|eseesecally|small|9068861615slate10351|peach|Ounce|Unknown|86|eingationeingn st| +9879|AAAAAAAAGJGCAAAA|2000-10-27||Things stand alone but for the situations. Dirty engines look almost communists. Fair files see effectively for a changes. Soft tracks used to give however religious forms. Boards allow on|7.82|4.14|5002002|importoscholar #2|2|country|5|Music|644|eseesecally|N/A|616powder32334478795|violet|Unknown|Unknown|53|n stationeingn st| +9880|AAAAAAAAIJGCAAAA|1997-10-27|1999-10-27|Officers help all. Personal duties conflict well as a others; affairs elect between a sales; respective mammals begin with a official|0.59|0.25|7009001|maxibrand #1|9|mattresses|7|Home|516|callyoughtanti|N/A|95230turquoise006525|pale|Tsp|Unknown|3|bareingeingn st| +9881|AAAAAAAAIJGCAAAA|1999-10-28|2001-10-26|Officers help all. Personal duties conflict well as a others; affairs elect between a sales; respective mammals begin with a official|3.84|0.25|8005002|scholarnameless #2|5|fitness|8|Sports|554|eseantianti|N/A|95230turquoise006525|sienna|Dozen|Unknown|16|oughteingeingn st| +9882|AAAAAAAAIJGCAAAA|2001-10-27||Officers help all. Personal duties conflict well as a others; affairs elect between a sales; respective mammals begin with a official|7.38|4.64|2002001|importoimporto #1|5|shirts|2|Men|554|eseantianti|large|196453436736903rose6|cornsilk|Each|Unknown|40|ableeingeingn st| +9883|AAAAAAAALJGCAAAA|1997-10-27||Different, significant organisations might diminish there. Public gifts remember thus troops. Inte|2.00|1.80|5003002|exportischolar #2|3|pop|5|Music|77|ationation|N/A|01navy34482647229602|rosy|Lb|Unknown|25|prieingeingn st| +9884|AAAAAAAAMJGCAAAA|1997-10-27|2000-10-26|International, public costs must not face however thin, free difficulties; sure questions beat|1.79|1.28|4001001|amalgedu pack #1|1|womens|4|Shoes|605|antibarcally|small|yellow04885495474107|black|Lb|Unknown|41|eseeingeingn st| +9885|AAAAAAAAMJGCAAAA|2000-10-27||International, public costs must not face however thin, free difficulties; sure questions beat|9.88|5.82|4004002|edu packedu pack #2|1|athletic|4|Shoes|213|antibarcally|medium|yellow04885495474107|cornsilk|Each|Unknown|34|antieingeingn st| +9886|AAAAAAAAOJGCAAAA|1997-10-27|1999-10-27|Cases produce always developments. Genuine facilities would give away weeks. Rows can concentrate maximum hills. Romant|4.31|3.87|7001007|amalgbrand #7|1|bathroom|7|Home|487|ationeingese|N/A|3peru839563061531183|turquoise|Ton|Unknown|4|callyeingeingn st| +9887|AAAAAAAAOJGCAAAA|1999-10-28|2001-10-26|Here entire persons will not take still formal small problems. European prisoners can save by the men. Particular students must keep here. Deposits will embrace home powerful|5.19|3.26|4004002|edu packedu pack #2|4|athletic|4|Shoes|329|n stablepri|extra large|3peru839563061531183|sandy|Pound|Unknown|12|ationeingeingn st| +9888|AAAAAAAAOJGCAAAA|2001-10-27||Here entire persons will not take still formal small problems. European prisoners can save by the men. Particular students must keep here. Deposits will embrace home powerful|9.55|3.26|1004001|edu packamalg #1|4|swimwear|1|Women|329|n stablepri|medium|318268684ghost822425|rosy|Bundle|Unknown|41|eingeingeingn st| +9889|AAAAAAAABKGCAAAA|1997-10-27||Controls get so sometimes soviet women; times will predict fair. Lives use with a stages. S|9.00|8.10|1004002|edu packamalg #2|4|swimwear|1|Women|685|antieingcally|small|40898lawn33984990355|mint|Ton|Unknown|82|n steingeingn st| +9890|AAAAAAAACKGCAAAA|1997-10-27|2000-10-26|Sure, new nations ought to spare long drugs. Previous|1.56|1.37|4001001|amalgedu pack #1|1|womens|4|Shoes|36|callypri|medium|81397761light7549077|white|Unknown|Unknown|65|barn steingn st| +9891|AAAAAAAACKGCAAAA|2000-10-27||Sure, new nations ought to spare long drugs. Previous|0.51|1.37|4001001|edu packedu pack #2|1|athletic|4|Shoes|348|eingesepri|extra large|751001258187518red08|orange|Bunch|Unknown|15|oughtn steingn st| +9892|AAAAAAAAEKGCAAAA|1997-10-27|1999-10-27|Densely willin|0.81|0.59|4001001|amalgedu pack #1|1|womens|4|Shoes|291|oughtn stable|large|sky85571656941429876|rose|Dram|Unknown|85|ablen steingn st| +9893|AAAAAAAAEKGCAAAA|1999-10-28|2001-10-26|Personal, growing participants shall influence semantic, top services. Still unable relationships might |1.53|0.59|4001001|importoamalg #2|2|fragrances|1|Women|291|oughtn stable|extra large|sky85571656941429876|green|Carton|Unknown|4|prin steingn st| +9894|AAAAAAAAEKGCAAAA|2001-10-27||Too white minutes find single, young centres. Users will sell often. Other colleagues must exceed musical effec|3.68|0.59|4001001|amalgscholar #1|2|rock|5|Music|291|oughtn stable|N/A|sky85571656941429876|navy|Bunch|Unknown|100|esen steingn st| +9895|AAAAAAAAHKGCAAAA|1997-10-27||Names become also actually social families. Members shall not experience there quiet links; especially ready marks must not v|14.21|8.66|5001002|amalgscholar #2|1|rock|5|Music|207|ationbarable|N/A|28722914smoke9511294|lace|Pallet|Unknown|21|antin steingn st| +9896|AAAAAAAAIKGCAAAA|1997-10-27|2000-10-26|Welsh thoughts may get various centres; ave|1.20|0.60|3003001|exportiexporti #1|3|toddlers|3|Children|811|oughtoughteing|medium|86peach4585974412325|plum|Box|Unknown|22|callyn steingn st| +9897|AAAAAAAAIKGCAAAA|2000-10-27||Welsh thoughts may get various centres; ave|4.79|2.01|3001002|amalgexporti #2|1|newborn|3|Children|811|oughtoughteing|extra large|86peach4585974412325|salmon|Cup|Unknown|4|ationn steingn st| +9898|AAAAAAAAKKGCAAAA||1999-10-27||1.74|||edu packimporto #1|4|||Men|||extra large|00379royal5111914306|saddle||Unknown||| +9899|AAAAAAAAKKGCAAAA|1999-10-28|2001-10-26|Other pra|4.72|2.69|2004001|amalgedu pack #2|4|womens|4|Shoes|619|n stoughtcally|petite|00379royal5111914306|smoke|Pallet|Unknown|2|n stn steingn st| +9900|AAAAAAAAKKGCAAAA|2001-10-27||Relative sources marry now old jobs. Roles leave; big minutes pour. Again great studies play more statutory objectives. L|3.79|2.69|2004001|amalgexporti #1|4|newborn|3|Children|619|n stoughtcally|large|6921077861puff359810|purple|N/A|Unknown|61|barbarn stn st| +9901|AAAAAAAANKGCAAAA|1997-10-27||Detailed, cognitive friends go less so domestic terms. Again accurate children would break|7.44|3.79|8012002|importomaxi #2|12|guns|8|Sports|356|callyantipri|N/A|13761006smoke5788472|azure|Pallet|Unknown|11|oughtbarn stn st| +9902|AAAAAAAAOKGCAAAA|1997-10-27|2000-10-26|Roots sign briefly aside left months. Fields shall make farmers. For instance interesting responsibilities use too sure outcomes. Corpor|39.85|14.34|6006007|corpcorp #7|6|rings|6|Jewelry|44|eseese|N/A|08peru59373198240719|tomato|Ounce|Unknown|9|ablebarn stn st| +9903|AAAAAAAAOKGCAAAA|2000-10-27||Just red pictures must know sexual communities. Other others would consider physical, financial systems. Only names will not make since diseases. Too heal|3.45|14.34|6006007|scholaramalgamalg #10|6|portable|10|Electronics|44|eseese|N/A|08peru59373198240719|lavender|Tsp|Unknown|9|pribarn stn st| +9904|AAAAAAAAALGCAAAA|1997-10-27|1999-10-27|New, central weaknesses blow distinguished leaders. Grateful, apparent parties become right attempts|2.90|1.65|2001001|amalgimporto #1|1|accessories|2|Men|713|prioughtation|large|983645655278193dim52|lawn|Box|Unknown|77|esebarn stn st| +9905|AAAAAAAAALGCAAAA|1999-10-28|2001-10-26|Unpleasant months go ho|3.76|2.85|2001001|amalgbrand #6|1|bathroom|7|Home|195|prioughtation|N/A|30868210665299steel3|peach|Lb|Unknown|85|antibarn stn st| +9906|AAAAAAAAALGCAAAA|2001-10-27||Regional processes used to feel else new groups. Forces change about other purposes; there statistical days must catch because of a areas. New members used to pro|0.18|0.10|9008007|namelessmaxi #7|1|romance|9|Books|954|prioughtation|N/A|70641086brown3178345|metallic|Bundle|Unknown|10|callybarn stn st| +9907|AAAAAAAADLGCAAAA|1997-10-27||Really scottish appeals |2.19|1.31|3004002|edu packexporti #2|4|school-uniforms|3|Children|441|oughteseese|medium|6748655ghost75580945|sky|Gram|Unknown|26|ationbarn stn st| +9908|AAAAAAAAELGCAAAA|1997-10-27|2000-10-26|Sure design|9.11|2.73|1002001|importoamalg #1|2|fragrances|1|Women|316|callyoughtpri|petite|937197163654papaya36|peach|Pallet|Unknown|3|eingbarn stn st| +9909|AAAAAAAAELGCAAAA|2000-10-27||Sure design|2.01|2.73|1002001|scholarunivamalg #10|2|fiction|9|Books|710|baroughtation|N/A|07963131slate5973240|rose|Ounce|Unknown|89|n stbarn stn st| +9910|AAAAAAAAGLGCAAAA|1997-10-27|1999-10-27|Open plants end. Newly |5.40|3.72|7006001|corpbrand #1|6|rugs|7|Home|178|eingationought|N/A|46914royal1228443433|forest|N/A|Unknown|94|baroughtn stn st| +9911|AAAAAAAAGLGCAAAA|1999-10-28|2001-10-26|Open plants end. Newly |8.79|4.57|2002002|importoimporto #2|6|shirts|2|Men|196|callyn stought|economy|109papaya57694223097|turquoise|Bundle|Unknown|1|oughtoughtn stn st| +9912|AAAAAAAAGLGCAAAA|2001-10-27||Only emp|0.42|0.32|2002002|importoedu pack #1|2|mens|4|Shoes|306|callybarpri|medium|109papaya57694223097|turquoise|Dozen|Unknown|55|ableoughtn stn st| +9913|AAAAAAAAJLGCAAAA|1997-10-27||Measures will not sustain inevitable, successful years. Common, sound students blame probably wooden beans; secret objects can read almo|6.10|4.88|6002002|importocorp #2|2|diamonds|6|Jewelry|32|ablepri|N/A|677927royal376474274|steel|Bundle|Unknown|28|prioughtn stn st| +9914|AAAAAAAAKLGCAAAA|1997-10-27|2000-10-26|Children must want before a men. Items provide then educational flowers. Developments will feel fully double workers. |88.64|44.32|2003001|exportiimporto #1|3|pants|2|Men|149|n steseought|medium|78640aquamarine95822|lace|Lb|Unknown|14|eseoughtn stn st| +9915|AAAAAAAAKLGCAAAA|2000-10-27||Increased, technological customers destroy behind a homes; also bare drawings in|7.01|44.32|8015006|scholarmaxi #6|3|fishing|8|Sports|149|n steseought|N/A|87papaya723871406374|smoke|Tbl|Unknown|39|antioughtn stn st| +9916|AAAAAAAAMLGCAAAA|1997-10-27|1999-10-27|Angrily good children matter again domestic, main memories. Less following approaches could mean great, labour numbers. Basic, unnecessary habits co|3.19|1.30|1004001|edu packamalg #1|4|swimwear|1|Women|118|eingoughtought|small|4881537slate61592317|sky|Tbl|Unknown|15|callyoughtn stn st| +9917|AAAAAAAAMLGCAAAA|1999-10-28|2001-10-26|Angrily good children matter again domestic, main memories. Less following approaches could mean great, labour numbers. Basic, unnecessary habits co|2.01|1.58|2004002|edu packimporto #2|4|sports-apparel|2|Men|118|eingoughtought|medium|542pale6879172751772|midnight|Pallet|Unknown|29|ationoughtn stn st| +9918|AAAAAAAAMLGCAAAA|2001-10-27||Dual, true communists watch over legs. Different, novel creatures talk under a moves. Quite premier words learn by the concern|2.07|1.82|10012008|importoamalgamalg #8|4|monitors|10|Electronics|118|eingoughtought|N/A|275207976590tan55872|purple|Carton|Unknown|13|eingoughtn stn st| +9919|AAAAAAAAPLGCAAAA|1997-10-27||National opportunities would guide both easy windows. Then able items might depend for the sites. Very faces will not control millions; assumptions discuss true reasons; effects could |2.17|1.04|10008009|namelessunivamalg #9|8|scanners|10|Electronics|535|antiprianti|N/A|71purple222855085687|peru|Tsp|Unknown|59|n stoughtn stn st| +9920|AAAAAAAAAMGCAAAA|1997-10-27|2000-10-26|Sorry versions simplify just upper disputes. Inc|48.42|21.78|2003001|exportiimporto #1|3|pants|2|Men|653|priantically|medium|921coral584127052333|sky|Ounce|Unknown|8|barablen stn st| +9921|AAAAAAAAAMGCAAAA|2000-10-27||Sorry versions simplify just upper disputes. Inc|9.78|8.21|9012010|importounivamalg #10|12|home repair|9|Books|268|priantically|N/A|921coral584127052333|gainsboro|Case|Unknown|21|oughtablen stn st| +9922|AAAAAAAACMGCAAAA|1997-10-27|1999-10-27|Ho|3.04|2.49|9014011|edu packunivamalg #11|14|sports|9|Books|226|callyableable|N/A|62276thistle73685404|gainsboro|Bunch|Unknown|45|ableablen stn st| +9923|AAAAAAAACMGCAAAA|1999-10-28|2001-10-26|Comprehensive, nerv|3.36|2.01|9014011|exportiamalgamalg #13|13|stereo|10|Electronics|226|callyableable|N/A|62276thistle73685404|papaya|Box|Unknown|56|priablen stn st| +9924|AAAAAAAACMGCAAAA|2001-10-27||Comprehensive, nerv|80.63|60.47|9014011|edu packimporto #1|4|sports-apparel|2|Men|436|callypriese|large|62276thistle73685404|sky|Ton|Unknown|6|eseablen stn st| +9925|AAAAAAAAFMGCAAAA|1997-10-27||Russians will rival with a pupils. Talks used to encourage uniquely only others. Eyes walk european sides; papers forgo relatively important interests. Nuclear teachers |3.97|2.85|1002002|importoamalg #2|2|fragrances|1|Women|470|barationese|economy|22365saddle274809983|turquoise|N/A|Unknown|23|antiablen stn st| +9926|AAAAAAAAGMGCAAAA|1997-10-27|2000-10-26|Immediately regular forms |0.70|0.33|1004001|edu packamalg #1|4|swimwear|1|Women|142|ableeseought|petite|5turquoise9249923812|ivory|Tbl|Unknown|2|callyablen stn st| +9927|AAAAAAAAGMGCAAAA|2000-10-27||Immediately regular forms |4.36|2.61|1004001|exportiedu pack #2|3|kids|4|Shoes|142|ableeseought|petite|5turquoise9249923812|sienna|Gross|Unknown|27|ationablen stn st| +9928|AAAAAAAAIMGCAAAA|1997-10-27|1999-10-27|However little parties open straightforward months; new judges used t|7.23|5.56|7004009|edu packbrand #9|4|curtains/drapes|7|Home|230|barpriable|N/A|8189seashell72303834|midnight|Gross|Unknown|4|eingablen stn st| +9929|AAAAAAAAIMGCAAAA|1999-10-28|2001-10-26|However little parties open straightforward months; new judges used t|0.37|0.19|4002002|importoedu pack #2|2|mens|4|Shoes|230|barpriable|medium|728olive948039279597|sienna|Bunch|Unknown|62|n stablen stn st| +9930|AAAAAAAAIMGCAAAA|2001-10-27||However little parties open straightforward months; new judges used t|0.53|0.19|4002002|exportiamalg #1|2|maternity|1|Women|230|barpriable|small|728olive948039279597|thistle|Unknown|Unknown|5|barprin stn st| +9931|AAAAAAAALMGCAAAA|1997-10-27||Enough financial clients may figure now old problems. Real funds hear at least also tall schools. Quite new authorities mu|4.28|1.28|9009008|maximaxi #8|9|science|9|Books|3|pri|N/A|01drab54224269418309|magenta|Dram|Unknown|54|oughtprin stn st| +9932|AAAAAAAAMMGCAAAA|1997-10-27|2000-10-26|Sharp months might make then for the conservatives. Never rec|6.64|2.39|5004001|edu packscholar #1|4|classical|5|Music|575|antiationanti|N/A|971red56828254711601|smoke|Bundle|Unknown|9|ableprin stn st| +9933|AAAAAAAAMMGCAAAA|2000-10-27||Sharp months might make then for the conservatives. Never rec|5.97|2.39|5004001|exportiimporto #2|3|pants|2|Men|575|antiationanti|large|475273809938hot97112|firebrick|Box|Unknown|6|priprin stn st| +9934|AAAAAAAAOMGCAAAA|1997-10-27|1999-10-27|Guilty drinks must not understand almost into a advantages. Prices call often rather social benefits; men load more women. There european videos will get. Proper, complete authori|3.25|2.82|5004001|edu packscholar #1|4|classical|5|Music|680|bareingcally|N/A|766754orchid62072431|salmon|Gross|Unknown|27|eseprin stn st| +9935|AAAAAAAAOMGCAAAA|1999-10-28|2001-10-26|Guilty drinks must not understand almost into a advantages. Prices call often rather social benefits; men load more women. There european videos will get. Proper, complete authori|2.21|2.82|5004001|exportischolar #2|4|pop|5|Music|218|eingoughtable|N/A|766754orchid62072431|pink|Unknown|Unknown|64|antiprin stn st| +9936|AAAAAAAAOMGCAAAA|2001-10-27||Local points may not understand away from a students. Sometimes united figures w|46.36|40.79|5004001|importomaxi #1|2|business|9|Books|103|eingoughtable|N/A|766754orchid62072431|seashell|Lb|Unknown|33|callyprin stn st| +9937|AAAAAAAABNGCAAAA|1997-10-27||Used proceedings can serve. Severe schools may possess enough to a eyes. Equal, small figures will assure economic, easy methods. Mostly central weeks can state superb|2.13|1.81|8008008|namelessnameless #8|8|outdoor|8|Sports|246|callyeseable|N/A|4065313pale773856567|rose|Bundle|Unknown|13|ationprin stn st| +9938|AAAAAAAACNGCAAAA|1997-10-27|2000-10-26|Again avail|3.02|2.32|8005009|scholarnameless #9|5|fitness|8|Sports|98|eingn st|N/A|89ivory7175382275140|wheat|Tsp|Unknown|17|eingprin stn st| +9939|AAAAAAAACNGCAAAA|2000-10-27||Again avail|5.16|2.32|7004010|edu packbrand #10|4|curtains/drapes|7|Home|82|eingn st|N/A|89ivory7175382275140|peru|Dram|Unknown|1|n stprin stn st| +9940|AAAAAAAAENGCAAAA|1997-10-27|1999-10-27|Ways begin according to a windows; capable, main actions |5.98|4.00|10005013|scholarunivamalg #13|5|karoke|10|Electronics|257|ationantiable|N/A|010turquoise38270550|chartreuse|Ton|Unknown|33|baresen stn st| +9941|AAAAAAAAENGCAAAA|1999-10-28|2001-10-26|Eyes may not provide well special, different fingers; processes can work as average, willing fears. Lessons could not rush more different, intermediate vessel|37.30|11.56|10005013|edu packscholar #2|5|classical|5|Music|257|ationantiable|N/A|010turquoise38270550|rosy|Lb|Unknown|24|oughtesen stn st| +9942|AAAAAAAAENGCAAAA|2001-10-27||Free tons might offer early functions. Extra, extraordinary fees cannot happen. New, necessary problems ought to go finall|5.64|3.04|1003001|exportiamalg #1|3|maternity|1|Women|257|ationantiable|small|09516750335150rose32|khaki|Lb|Unknown|94|ableesen stn st| +9943|AAAAAAAAHNGCAAAA|1997-10-27||Acute, uncertain consequences might want; certainly unlikely trials used to take also. Years stay no longer. Only right trees will notice generally. Excessive, peaceful vis|7.75|6.35|2002002|importoimporto #2|2|shirts|2|Men|647|ationesecally|small|7215wheat99655183547|sky|Pound|Unknown|40|priesen stn st| +9944|AAAAAAAAINGCAAAA|1997-10-27|2000-10-26|Good, pale letters include often later thinking shares. Details offer again|3.30|1.28|7015005|scholarnameless #5|15|tables|7|Home|188|eingeingought|N/A|641blush724929399407|powder|Gross|Unknown|58|eseesen stn st| +9945|AAAAAAAAINGCAAAA|2000-10-27||Voters include well prices. Just important words call in a legs. Cultural, tory arguments sleep serious, great eyes. Strictly old references could sit local, main months; full arts can push in a |8.05|1.28|1001002|amalgamalg #2|15|dresses|1|Women|188|eingeingought|medium|641blush724929399407|lavender|Tbl|Unknown|79|antiesen stn st| +9946|AAAAAAAAKNGCAAAA|1997-10-27|1999-10-27|Levels could say pointedly original, happy sessions; immense, technological decisions might discourage basic difficulties. Officials find. Simple, |8.70|2.61|9007011|brandmaxi #11|7|reference|9|Books|132|ablepriought|N/A|015902130993salmon63|turquoise|Dozen|Unknown|14|callyesen stn st| +9947|AAAAAAAAKNGCAAAA|1999-10-28|2001-10-26|Aims could not wait therefore in a transactions. Concerned colleagues give behind imaginative guidelines. Areas manage very|6.45|1.99|6016004|corpbrand #4|16|consignment|6|Jewelry|342|ablepriought|N/A|015902130993salmon63|midnight|Ton|Unknown|8|ationesen stn st| +9948|AAAAAAAAKNGCAAAA|2001-10-27||Cold years accept satisfactorily objections. Gaps ought to attend times. Together single requirements may not loo|1.69|1.33|6016004|brandunivamalg #4|16|personal|10|Electronics|603|pribarcally|N/A|015902130993salmon63|lemon|Cup|Unknown|32|eingesen stn st| +9949|AAAAAAAANNGCAAAA|1997-10-27||Costs will write certainly years. Clothes expo|8.25|7.09|3003002|exportiexporti #2|3|toddlers|3|Children|615|antioughtcally|petite|660pale4340810055239|rose|Dram|Unknown|32|n stesen stn st| +9950|AAAAAAAAONGCAAAA|1997-10-27|2000-10-26|Outcomes will become high wide, substantial clients. Sufficient, new resources weaken only over the moments. Of cour|1.32|0.47|9011003|amalgunivamalg #3|11|cooking|9|Books|540|bareseanti|N/A|9067704926097royal66|puff|Cup|Unknown|37|barantin stn st| +9951|AAAAAAAAONGCAAAA|2000-10-27||Jobs suggest multiple, upper eyebrows. Clear, prime allowan|2.30|1.17|9011004|amalgunivamalg #4|11|cooking|9|Books|540|bareseanti|N/A|9067704926097royal66|peru|Dozen|Unknown|12|oughtantin stn st| +9952|AAAAAAAAAOGCAAAA|1997-10-27|1999-10-27|Early colours see both together total sites. Small, labour directors leave. Valid rules let in addition civil, military positions; social negotia|4.50|3.42|2001001|amalgimporto #1|1|accessories|2|Men|114|eseoughtought|petite|1537lemon46015807034|purple|Ton|Unknown|42|ableantin stn st| +9953|AAAAAAAAAOGCAAAA|1999-10-28|2001-10-26|Early colours see both together total sites. Small, labour directors leave. Valid rules let in addition civil, military positions; social negotia|89.01|37.38|2001001|scholarunivamalg #9|1|karoke|10|Electronics|114|eseoughtought|N/A|1537lemon46015807034|midnight|Bunch|Unknown|40|priantin stn st| +9954|AAAAAAAAAOGCAAAA|2001-10-27||Eyes may provide human supporters. Companies will grip now o|1.17|0.83|10007010|brandunivamalg #10|1|personal|10|Electronics|795|antin station|N/A|1537lemon46015807034|salmon|Ounce|Unknown|6|eseantin stn st| +9955|AAAAAAAADOGCAAAA|1997-10-27||Other cars know actually minute moments. Extra old police could choose more. White, sympathetic awards put just commitments. Waste families|9.59|7.00|1001002|amalgamalg #2|1|dresses|1|Women|313|prioughtpri|medium|863928752pale7670400|spring|Ounce|Unknown|5|antiantin stn st| +9956|AAAAAAAAEOGCAAAA|1997-10-27|2000-10-26|Clinical languages see foreign, public members. Subsequent |4.16|2.99|5002001|importoscholar #1|2|country|5|Music|580|bareinganti|N/A|569914184122spring27|linen|Lb|Unknown|25|callyantin stn st| +9957|AAAAAAAAEOGCAAAA|2000-10-27||Clinical languages see foreign, public members. Subsequent |5.20|2.99|7002008|importobrand #8|2|bedding|7|Home|580|bareinganti|N/A|6599669703orchid1314|light|Dozen|Unknown|19|ationantin stn st| +9958|AAAAAAAAGOGCAAAA|1997-10-27|1999-10-27|Immense fields find on a measures. Followers may not want on a details. Occasions look also worthw|2.40|1.70|7010009|univnameless #9|10|flatware|7|Home|260|barcallyable|N/A|9807632161562steel72|steel|Bundle|Unknown|63|eingantin stn st| +9959|AAAAAAAAGOGCAAAA|1999-10-28|2001-10-26|Days keep mixed, good requirements. Below previous doors used to know national, small objects. Hands would stay at the languages; always open systems shall not get e|0.61|0.49|8006010|corpnameless #10|6|football|8|Sports|260|barcallyable|N/A|9807632161562steel72|sienna|Pallet|Unknown|5|n stantin stn st| +9960|AAAAAAAAGOGCAAAA|2001-10-27||Days keep mixed, good requirements. Below previous doors used to know national, small objects. Hands would stay at the languages; always open systems shall not get e|20.73|0.49|4004001|edu packedu pack #1|6|athletic|4|Shoes|260|barcallyable|medium|9807632161562steel72|drab|Cup|Unknown|65|barcallyn stn st| +9961|AAAAAAAAJOGCAAAA|1997-10-27||Pounds would fit very significant weeks. Open, single churches provide. Meetings lose financial members. Things reduce too. Waters place usually determined agents. |4.57|3.10|9009002|maximaxi #2|9|science|9|Books|230|barpriable|N/A|25518241554sienna109|peru|Ounce|Unknown|22|oughtcallyn stn st| +9962|AAAAAAAAKOGCAAAA|1997-10-27|2000-10-26|Mines cannot like together independent women. Away only times could not know common, o|9.17|7.24|2004001|edu packimporto #1|4|sports-apparel|2|Men|584|eseeinganti|medium|7purple9863773436001|tan|Each|Unknown|8|ablecallyn stn st| +9963|AAAAAAAAKOGCAAAA|2000-10-27||Mines cannot like together independent women. Away only times could not know common, o|3.13|2.37|8003004|exportinameless #4|3|basketball|8|Sports|447|ationeseese|N/A|92033693puff18821371|plum|Case|Unknown|40|pricallyn stn st| +9964|AAAAAAAAMOGCAAAA|1997-10-27|1999-10-27|Events could play instead silly, strong musicians. Regions shall not reduce however to a |6.15|4.30|8001005|amalgnameless #5|1|athletic shoes|8|Sports|77|ationation|N/A|7639823661peru574333|chiffon|Dram|Unknown|33|esecallyn stn st| +9965|AAAAAAAAMOGCAAAA|1999-10-28|2001-10-26|Probably united profits used to embody for a practices. Private, healthy lines get following, main studies. Upward sexual lines must not go. Other, huge transactions used to build.|73.87|26.59|8001005|namelessnameless #6|1|outdoor|8|Sports|77|ationation|N/A|7639823661peru574333|sky|Case|Unknown|13|anticallyn stn st| +9966|AAAAAAAAMOGCAAAA|2001-10-27||Probably united profits used to embody for a practices. Private, healthy lines get following, main studies. Upward sexual lines must not go. Other, huge transactions used to build.|3.94|3.15|8001005|amalgimporto #1|1|accessories|2|Men|93|prin st|large|7639823661peru574333|beige|Carton|Unknown|11|callycallyn stn st| +9967|AAAAAAAAPOGCAAAA|1997-10-27||Usually massive passengers ought to treat light, charming accounts. Requirements adopt never; comparative questions fulfil public, old schools. Occasions wai|1.55|0.69|5003002|exportischolar #2|3|pop|5|Music|369|n stcallypri|N/A|437pale7740222486424|plum|Gross|Unknown|4|ationcallyn stn st| +9968|AAAAAAAAAPGCAAAA|1997-10-27|2000-10-26|Seasons go abruptly present girls. Significant, new processes rely of course weapons. Solid, social years ought to secure fully after a words. Western, french goals would not draw pressures. Recen|2.59|1.16|3002001|importoexporti #1|2|infants|3|Children|95|antin st|small|5wheat50475854665141|tan|Gross|Unknown|63|eingcallyn stn st| +9969|AAAAAAAAAPGCAAAA|2000-10-27||Seasons go abruptly present girls. Significant, new processes rely of course weapons. Solid, social years ought to secure fully after a words. Western, french goals would not draw pressures. Recen|2.37|1.16|3002001|amalgunivamalg #8|2|cameras|10|Electronics|95|antin st|N/A|174284wheat109031004|peru|Lb|Unknown|7|n stcallyn stn st| +9970|AAAAAAAACPGCAAAA|1997-10-27|1999-10-27|At first strange minds might not think. Impressive, special systems like beautifully by a instructions; necessarily little hearts will not decide partly economic real habits. Hard angry wages shall|1.78|0.89|2002001|importoimporto #1|2|shirts|2|Men|703|pribaration|small|95797092138849green4|powder|Dram|Unknown|24|barationn stn st| +9971|AAAAAAAACPGCAAAA|1999-10-28|2001-10-26|Public, new minutes should not ma|5.43|0.89|8012002|importomaxi #2|12|guns|8|Sports|300|barbarpri|N/A|deep3902782785607628|sky|Lb|Unknown|17|oughtationn stn st| +9972|AAAAAAAACPGCAAAA|2001-10-27||Public, new minutes should not ma|1.96|0.89|8012002|amalgexporti #1|1|newborn|3|Children|878|eingationeing|small|deep3902782785607628|red|Tbl|Unknown|13|ableationn stn st| +9973|AAAAAAAAFPGCAAAA|1997-10-27||Components pretend please financial, outer penalties. Good, comm|8.18|6.38|5002002|importoscholar #2|2|country|5|Music|499|n stn stese|N/A|1572964295419sienna7|saddle|Dozen|Unknown|24|priationn stn st| +9974|AAAAAAAAGPGCAAAA|1997-10-27|2000-10-26|Numerous chil|4.47|1.78|5001001|amalgscholar #1|1|rock|5|Music|124|eseableought|N/A|069royal742391124387|medium|Carton|Unknown|16|eseationn stn st| +9975|AAAAAAAAGPGCAAAA|2000-10-27||Numerous chil|6.44|4.63|4001002|amalgedu pack #2|1|womens|4|Shoes|124|eseableought|medium|6549016356lemon70999|mint|Gross|Unknown|48|antiationn stn st| +9976|AAAAAAAAIPGCAAAA|1997-10-27|1999-10-27|Now local years express losses. Impossible performances cost to the plans. Very implicit results cannot see by a representatives. Relations ought to give|3.88|3.45|6009001|maxicorp #1|9|womens watch|6|Jewelry|164|esecallyought|N/A|98892670262thistle24|spring|Ton|Unknown|88|callyationn stn st| +9977|AAAAAAAAIPGCAAAA|1999-10-28|2001-10-26|Special, cruci|8.13|2.60|1004002|edu packamalg #2|9|swimwear|1|Women|20|esecallyought|extra large|98892670262thistle24|salmon|Unknown|Unknown|4|ationationn stn st| +9978|AAAAAAAAIPGCAAAA|2001-10-27||Special, cruci|0.69|2.60|1004002|edu packimporto #1|9|sports-apparel|2|Men|20|esecallyought|small|98892670262thistle24|peach|Oz|Unknown|60|eingationn stn st| +9979|AAAAAAAALPGCAAAA|1997-10-27||Free, relevant facilities used to include on a assumpt|0.21|0.08|9012008|importounivamalg #8|12|home repair|9|Books|932|ableprin st|N/A|33398042thistle18879|yellow|Lb|Unknown|65|n stationn stn st| +9980|AAAAAAAAMPGCAAAA|1997-10-27|2000-10-26|Man|6.46|3.48|8010001|univmaxi #1|10|pools|8|Sports|283|prieingable|N/A|4099767wheat04247942|yellow|Each|Unknown|50|bareingn stn st| +9981|AAAAAAAAMPGCAAAA|2000-10-27||Man|1.39|3.48|7015002|scholarnameless #2|15|tables|7|Home|209|n stbarable|N/A|66142yellow093671294|deep|Pallet|Unknown|4|oughteingn stn st| +9982|AAAAAAAAOPGCAAAA|1997-10-27|1999-10-27|Already corporate tests will give trees. Relationships understand peacefully at the plans. Schools could afford away on a languages. Able, pure years may defend over a hands. More valid|18.30|10.06|5004001|edu packscholar #1|4|classical|5|Music|286|callyeingable|N/A|8085yellow9821087518|violet|Lb|Unknown|19|ableeingn stn st| +9983|AAAAAAAAOPGCAAAA|1999-10-28|2001-10-26|Often new doubts shall not take great friends. Industrial weekends put there senior, new cases. Nowadays actual cells explain afterwards. Days may expect visitors. Equal patients must r|62.75|20.70|5001002|amalgscholar #2|1|rock|5|Music|54|eseanti|N/A|midnight600120416031|turquoise|Case|Unknown|17|prieingn stn st| +9984|AAAAAAAAOPGCAAAA|2001-10-27||Political, new towns will not fight to a patients. Almost normal concerns sit eventually substantial |88.10|37.00|8002005|importonameless #5|2|baseball|8|Sports|22|eseanti|N/A|turquoise64574113737|steel|Oz|Unknown|32|eseeingn stn st| +9985|AAAAAAAABAHCAAAA|1997-10-27||Fascinating, royal comments use customers; trees would simplify. Familiar sons trav|9.31|5.77|4003002|exportiedu pack #2|3|kids|4|Shoes|148|eingeseought|large|43saddle217019015334|red|Case|Unknown|25|antieingn stn st| +9986|AAAAAAAACAHCAAAA|1997-10-27|2000-10-26|Open assistant days think in vitro annual talks. French, maximum animals used to pay simply on a husbands. Young images keep clearly deeper complete others. Afraid bits es|6.39|4.85|1003001|exportiamalg #1|3|maternity|1|Women|27|ationable|medium|9484914047523powder6|salmon|Box|Unknown|90|callyeingn stn st| +9987|AAAAAAAACAHCAAAA|2000-10-27||Open assistant days think in vitro annual talks. French, maximum animals used to pay simply on a husbands. Young images keep clearly deeper complete others. Afraid bits es|1.66|1.06|1003001|scholarmaxi #8|3|fishing|8|Sports|27|ationable|N/A|4287195281plum121462|peach|Box|Unknown|10|ationeingn stn st| +9988|AAAAAAAAEAHCAAAA|1997-10-27|1999-10-27|Only, local advisers disrupt formerly royal lips. Social procedures ought to pursue th|8.93|4.82|2003001|exportiimporto #1|3|pants|2|Men|120|barableought|medium|010260830884mint5464|pale|Each|Unknown|20|eingeingn stn st| +9989|AAAAAAAAEAHCAAAA|1999-10-28|2001-10-26|Judges ought to mean so indian findings; flowers love totally royal poets. Obvious|1.89|4.82|5001002|amalgscholar #2|3|rock|5|Music|120|barableought|N/A|rosy5138179750909306|salmon|Case|Unknown|21|n steingn stn st| +9990|AAAAAAAAEAHCAAAA|2001-10-27||A little so-called schemes verify little, gastric engineers. Great spots ought to contemplate good, only forms. Young, small sorts know preferably standards. At present small hours face in full. Othe|4.76|4.82|5001002|scholarnameless #1|15|tables|7|Home|428|eingableese|N/A|72sky003620034807461|sandy|Pound|Unknown|5|barn stn stn st| +9991|AAAAAAAAHAHCAAAA|1997-10-27||Free women might say home with a systems. Here direct ears include outside. Very proposed solutions read minutes. E|7.26|4.06|3001002|amalgexporti #2|1|newborn|3|Children|311|oughtoughtpri|medium|1yellow7810219070192|burlywood|Each|Unknown|25|oughtn stn stn st| +9992|AAAAAAAAIAHCAAAA|1997-10-27|2000-10-26|True services proceed then spanish problems. Real, front elements may support. Activities may s|8.88|5.41|10005014|scholarunivamalg #14|5|karoke|10|Electronics|214|eseoughtable|N/A|777514765540lemon452|moccasin|Oz|Unknown|29|ablen stn stn st| +9993|AAAAAAAAIAHCAAAA|2000-10-27||Very unhappy workers will eliminate h|1.37|5.41|8001004|amalgnameless #4|1|athletic shoes|8|Sports|251|oughtantiable|N/A|777514765540lemon452|magenta|Carton|Unknown|10|prin stn stn st| +9994|AAAAAAAAKAHCAAAA|1997-10-27|1999-10-27|Relatively busy others split etc political agencies. Badly easy schemes spend really. Women might make probably as other p|1.39|0.75|3003001|exportiexporti #1|3|toddlers|3|Children|580|bareinganti|medium|5629790seashell40591|peach|Oz|Unknown|53|esen stn stn st| +9995|AAAAAAAAKAHCAAAA|1999-10-28|2001-10-26|Relatively busy others split etc political agencies. Badly easy schemes spend really. Women might make probably as other p|5.54|0.75|1001002|amalgamalg #2|3|dresses|1|Women|503|pribaranti|petite|88powder223868524486|hot|Lb|Unknown|44|antin stn stn st| +9996|AAAAAAAAKAHCAAAA|2001-10-27||Relatively busy others split etc political agencies. Badly easy schemes spend really. Women might make probably as other p|1.83|0.75|6014005|edu packbrand #5|14|estate|6|Jewelry|503|pribaranti|N/A|25052578slate9382023|puff|Ounce|Unknown|1|callyn stn stn st| +9997|AAAAAAAANAHCAAAA|1997-10-27||Green, absolute conferences reflect well conditions. Benefits get essentially there other questions. Dead, nuclea|3.06|2.38|2001002|amalgimporto #2|1|accessories|2|Men|342|ableesepri|economy|686456misty432408494|lawn|Case|Unknown|88|ationn stn stn st| +9998|AAAAAAAAOAHCAAAA|1997-10-27|2000-10-26|Detailed, original uses may not pick bad, small records. Ways can come practices. Welcome, obvious stages gain strongly so p|4.54|2.08|3001001|amalgexporti #1|1|newborn|3|Children|477|ationationese|large|4pink191245757419056|saddle|Bundle|Unknown|33|eingn stn stn st| +9999|AAAAAAAAOAHCAAAA|2000-10-27||Detailed, original uses may not pick bad, small records. Ways can come practices. Welcome, obvious stages gain strongly so p|5.84|2.08|10013004|exportiamalgamalg #4|1|stereo|10|Electronics|789|ationationese|N/A|7212762614569blue095|ghost|Ton|Unknown|2|n stn stn stn st| +10000|AAAAAAAAABHCAAAA|1997-10-27|1999-10-27|Labour, golden subjects come more. Nuclear, full results must require very young things. Teachers find sure, wild aspirations. Liberal, normal authorities might obtain charges. Authorities will get |6.19|1.85|10015005|scholaramalgamalg #5|15|portable|10|Electronics|272|ableationable|N/A|97324889red281015064|orange|Box|Unknown|49|barbarbarbarought| +10001|AAAAAAAAABHCAAAA|1999-10-28|2001-10-26|Labour, golden subjects come more. Nuclear, full results must require very young things. Teachers find sure, wild aspirations. Liberal, normal authorities might obtain charges. Authorities will get |61.98|20.45|10015005|corpnameless #2|6|football|8|Sports|272|ableationable|N/A|013205rose7997844674|pale|Unknown|Unknown|54|oughtbarbarbarought| +10002|AAAAAAAAABHCAAAA|2001-10-27||Labour, golden subjects come more. Nuclear, full results must require very young things. Teachers find sure, wild aspirations. Liberal, normal authorities might obtain charges. Authorities will get |50.26|20.45|10015005|exportischolar #1|6|pop|5|Music|174|ableationable|N/A|013205rose7997844674|gainsboro|Box|Unknown|17|ablebarbarbarought| +10003|AAAAAAAADBHCAAAA|1997-10-27||Once recent stations used to accept limited, original |2.81|1.01|4002002|importoedu pack #2|2|mens|4|Shoes|196|callyn stought|petite|98907sandy9971912000|olive|Pallet|Unknown|26|pribarbarbarought| +10004|AAAAAAAAEBHCAAAA|1997-10-27|2000-10-26|Partly sure experiences afford even more complete stations; studies used to mention more. German, initial minutes might cost damp alternatives. Free pla|0.53|0.29|1002001|importoamalg #1|2|fragrances|1|Women|774|eseationation|petite|54turquoise404780252|saddle|Ton|Unknown|53|esebarbarbarought| +10005|AAAAAAAAEBHCAAAA|2000-10-27||Rights glimpse usually to a strategies. Electronic elements take fiercely formal, other clubs. At all gold words sh|44.21|38.02|1002001|exportiunivamalg #10|2|dvd/vcr players|10|Electronics|774|eseationation|N/A|54turquoise404780252|seashell|Pallet|Unknown|1|antibarbarbarought| +10006|AAAAAAAAGBHCAAAA|1997-10-27|1999-10-27|Reasonable, complex towns might |0.37|0.28|3001001|amalgexporti #1|1|newborn|3|Children|313|prioughtpri|extra large|03429355171saddle864|smoke|Ton|Unknown|33|callybarbarbarought| +10007|AAAAAAAAGBHCAAAA|1999-10-28|2001-10-26|Products take very average assessments. Grounds might not qualify reluctantly from a rates. Brief aspects die as yards. There powerful requirements devise lo|3.83|1.87|3001001|maxibrand #8|9|mattresses|7|Home|313|prioughtpri|N/A|03429355171saddle864|purple|Ounce|Unknown|1|ationbarbarbarought| +10008|AAAAAAAAGBHCAAAA|2001-10-27||Products take very average assessments. Grounds might not qualify reluctantly from a rates. Brief aspects die as yards. There powerful requirements devise lo|8.94|1.87|3001001|exportinameless #9|9|wallpaper|7|Home|313|prioughtpri|N/A|03429355171saddle864|steel|Lb|Unknown|80|eingbarbarbarought| +10009|AAAAAAAAJBHCAAAA|1997-10-27||Structures might join effective, mass edges. Personal, working americans begin less than mature, familiar leaves. Excessive, young pp. might put little, traditiona|6.83|3.55|1004002|edu packamalg #2|4|swimwear|1|Women|311|oughtoughtpri|medium|800553salmon97448113|turquoise|Gram|Unknown|20|n stbarbarbarought| +10010|AAAAAAAAKBHCAAAA|1997-10-27|2000-10-26|New, british politicians fail particularly in a things. Personal books get; as political nig|1.17|0.76|8012001|importomaxi #1|12|guns|8|Sports|36|callypri|N/A|43148611violet666193|slate|Oz|Unknown|24|baroughtbarbarought| +10011|AAAAAAAAKBHCAAAA|2000-10-27||Fully annual items tell meanwhile hence unable pages. Stations shall indicat|1.00|0.76|2003002|exportiimporto #2|12|pants|2|Men|36|callypri|large|819tomato52498638959|thistle|Ton|Unknown|50|oughtoughtbarbarought| +10012|AAAAAAAAMBHCAAAA|1997-10-27|1999-10-27|Video-taped reasons used to threaten more interests. Then apparent quantities determine now easy activities. Thin offences should not struggle abou|7.68|5.83|5003001|exportischolar #1|3|pop|5|Music|55|antianti|N/A|40323aquamarine89905|thistle|Tsp|Unknown|23|ableoughtbarbarought| +10013|AAAAAAAAMBHCAAAA|1999-10-28|2001-10-26|Altogether small agencies collaborate. Differences might feel nearl|9.99|3.29|5003001|edu packamalg #2|3|swimwear|1|Women|55|antianti|medium|40323aquamarine89905|green|Carton|Unknown|64|prioughtbarbarought| +10014|AAAAAAAAMBHCAAAA|2001-10-27||Terms should not execute even in a years. Extremely local lights care with a practices; p|60.71|37.03|5003001|brandmaxi #7|3|reference|9|Books|609|antianti|N/A|20759803seashell9343|red|N/A|Unknown|6|eseoughtbarbarought| +10015|AAAAAAAAPBHCAAAA|1997-10-27||International changes would follow as marginal individuals. Children change. Important|0.67|0.58|5002002|importoscholar #2|2|country|5|Music|221|oughtableable|N/A|3002513indian6895170|sienna|Ounce|Unknown|80|antioughtbarbarought| +10016|AAAAAAAAACHCAAAA|1997-10-27|2000-10-26|Early words make only. Enough tiny industries make about current, other factors. Other seconds provid|1.34|0.41|10010004|univamalgamalg #4|10|memory|10|Electronics|723|priableation|N/A|3061195magenta653105|snow|Gross|Unknown|57|callyoughtbarbarought| +10017|AAAAAAAAACHCAAAA|2000-10-27||Perhaps massive visitors should help female, new ministers; relative, complex|2.83|2.34|10010004|maxibrand #8|10|mattresses|7|Home|723|priableation|N/A|9575orange0563620060|tan|Bunch|Unknown|89|ationoughtbarbarought| +10018|AAAAAAAACCHCAAAA|1997-10-27|1999-10-27|Quickly able ways |3.10|2.07|7008009|namelessbrand #9|8|lighting|7|Home|164|esecallyought|N/A|4255931smoke97711632|sandy|Ounce|Unknown|44|eingoughtbarbarought| +10019|AAAAAAAACCHCAAAA|1999-10-28|2001-10-26|Quickly able ways |0.27|2.07|3003002|exportiexporti #2|3|toddlers|3|Children|168|eingcallyought|medium|110740pale2697900251|powder|Ton|Unknown|5|n stoughtbarbarought| +10020|AAAAAAAACCHCAAAA|2001-10-27||Quickly able ways |6.24|3.93|3003002|importonameless #1|2|baseball|8|Sports|556|eingcallyought|N/A|110740pale2697900251|goldenrod|Carton|Unknown|27|barablebarbarought| +10021|AAAAAAAAFCHCAAAA|1997-10-27||Patterns spend quickly presidential, left managers. Political, happy questions want strong changes. Only medical workers could not speak dead, conscious |0.11|0.07|1001002|amalgamalg #2|1|dresses|1|Women|17|ationought|small|smoke063608337641922|steel|Tsp|Unknown|41|oughtablebarbarought| +10022|AAAAAAAAGCHCAAAA|1997-10-27|2000-10-26|Certainly national contracts determine to a prices. New, russi|1.20|1.05|5003001|exportischolar #1|3|pop|5|Music|199|n stn stought|N/A|44812568230ivory3599|sky|Gram|Unknown|66|ableablebarbarought| +10023|AAAAAAAAGCHCAAAA|2000-10-27||Social, primar|3.75|1.87|5003001|edu packamalg #2|4|swimwear|1|Women|419|n stoughtese|medium|purple05260078983843|mint|Oz|Unknown|10|priablebarbarought| +10024|AAAAAAAAICHCAAAA|1997-10-27|1999-10-27|Particular, cheap buildings turn unsuccessfully illegal products; a little similar times can help welsh, democratic peasants; distinct waters believe here. Champions may go just f|67.72|55.53|4001001|amalgedu pack #1|1|womens|4|Shoes|74|eseation|medium|567puff2582201985244|floral|Case|Unknown|11|eseablebarbarought| +10025|AAAAAAAAICHCAAAA|1999-10-28|2001-10-26|Together international months hear about pregnant, english personnel. Gay, detailed measures explain votes. Voluntary, unknown teachers cannot create; ton|3.06|0.94|4001001|importoedu pack #2|2|mens|4|Shoes|74|eseation|petite|567puff2582201985244|metallic|N/A|Unknown|5|antiablebarbarought| +10026|AAAAAAAAICHCAAAA|2001-10-27||Together international months hear about pregnant, english personnel. Gay, detailed measures explain votes. Voluntary, unknown teachers cannot create; ton|0.68|0.59|4001001|exportiedu pack #1|3|kids|4|Shoes|74|eseation|petite|567puff2582201985244|peru|Pallet|Unknown|6|callyablebarbarought| +10027|AAAAAAAALCHCAAAA|1997-10-27||Existing executives may work mo|2.55|2.11|5002002|importoscholar #2|2|country|5|Music|544|eseeseanti|N/A|21bisque880034586637|light|Dram|Unknown|7|ationablebarbarought| +10028|AAAAAAAAMCHCAAAA|1997-10-27|2000-10-26|English germans anger systematically for the plans. Lights attract only leading sides. Points conceal. Widely other levels require political t|4.86|3.11|9015009|scholarunivamalg #9|15|fiction|9|Books|227|ationableable|N/A|0314596953plum861921|lace|Bundle|Unknown|9|eingablebarbarought| +10029|AAAAAAAAMCHCAAAA|2000-10-27||English germans anger systematically for the plans. Lights attract only leading sides. Points conceal. Widely other levels require political t|8.75|5.86|1003002|exportiamalg #2|15|maternity|1|Women|227|ationableable|medium|3503605tomato0872603|violet|N/A|Unknown|12|n stablebarbarought| +10030|AAAAAAAAOCHCAAAA|1997-10-27|1999-10-27|Agencies shall not consider false in a others. Obviously interesting authorities come anyway men. Small, |6.57|3.08|9014011|edu packunivamalg #11|14|sports|9|Books|59|n stanti|N/A|4063259237202167tan0|sandy|Tsp|Unknown|12|barpribarbarought| +10031|AAAAAAAAOCHCAAAA|1999-10-28|2001-10-26|Neighbouring facilities cannot retire sufficiently social, original resources. Statements conclude fina|98.63|3.08|9014011|exportiexporti #2|3|toddlers|3|Children|59|n stanti|large|4063259237202167tan0|red|Dozen|Unknown|31|oughtpribarbarought| +10032|AAAAAAAAOCHCAAAA|2001-10-27||Suspicious studies would pay much operators. Emotionally brief examples come more historic, particular|4.70|2.11|2003001|exportiimporto #1|3|pants|2|Men|59|n stanti|medium|485light399000801183|tomato|Tsp|Unknown|30|ablepribarbarought| +10033|AAAAAAAABDHCAAAA|1997-10-27||Banks control|0.94|0.36|10008004|namelessunivamalg #4|8|scanners|10|Electronics|123|priableought|N/A|718royal274845964311|metallic|Tbl|Unknown|3|pripribarbarought| +10034|AAAAAAAACDHCAAAA|1997-10-27|2000-10-26|Additional responses will|8.91|7.03|3002001|importoexporti #1|2|infants|3|Children|336|callypripri|medium|49011chartreuse19794|turquoise|Gram|Unknown|62|esepribarbarought| +10035|AAAAAAAACDHCAAAA|2000-10-27||Additional responses will|0.61|0.36|3002001|amalgimporto #2|2|accessories|2|Men|336|callypripri|extra large|15powder997568131462|peach|Carton|Unknown|16|antipribarbarought| +10036|AAAAAAAAEDHCAAAA|1997-10-27|1999-10-27|Soft shops used to take less; public assets might appreciate professional, big courses. Just democratic doubts want so random panels. Projects point more|4.27|2.81|4002001|importoedu pack #1|2|mens|4|Shoes|610|baroughtcally|medium|83peach0745588136164|sienna|Unknown|Unknown|10|callypribarbarought| +10037|AAAAAAAAEDHCAAAA|1999-10-28|2001-10-26|Variations see |3.57|1.64|8014008|edu packmaxi #8|2|tennis|8|Sports|610|baroughtcally|N/A|6889thistle775633838|powder|Dozen|Unknown|50|ationpribarbarought| +10038|AAAAAAAAEDHCAAAA|2001-10-27||Variations see |90.94|1.64|8014008|exportiunivamalg #9|2|dvd/vcr players|10|Electronics|610|baroughtcally|N/A|6889thistle775633838|seashell|Pound|Unknown|5|eingpribarbarought| +10039|AAAAAAAAHDHCAAAA|1997-10-27||Northern, blue requirements attempt by a cars; types lead political, nuclear rooms. Yet possible needs come merely english pairs. Main, clever men will not buy well equal |40.38|13.72|6014008|edu packbrand #8|14|estate|6|Jewelry|479|n stationese|N/A|5663681pink108087554|violet|N/A|Unknown|26|n stpribarbarought| +10040|AAAAAAAAIDHCAAAA|1997-10-27|2000-10-26|Main, different pockets should not make|7.17|3.65|4002001|importoedu pack #1|2|mens|4|Shoes|207|ationbarable|petite|902871857928thistle1|steel|Bundle|Unknown|18|baresebarbarought| +10041|AAAAAAAAIDHCAAAA|2000-10-27||Main, different pockets should not make|47.19|3.65|5002002|importoscholar #2|2|country|5|Music|207|ationbarable|N/A|55189214sandy9603776|firebrick|Oz|Unknown|90|oughtesebarbarought| +10042|AAAAAAAAKDHCAAAA|1997-10-27|1999-10-27|Memories could change on |2.63|1.57|10015013|scholaramalgamalg #13|15|portable|10|Electronics|460|barcallyese|N/A|34895754turquoise134|pink|Pound|Unknown|36|ableesebarbarought| +10043|AAAAAAAAKDHCAAAA|1999-10-28|2001-10-26|Statements may damage. Dependent courses will look a|4.39|1.57|10015013|amalgamalg #2|1|dresses|1|Women|427|barcallyese|medium|34895754turquoise134|thistle|Bunch|Unknown|11|priesebarbarought| +10044|AAAAAAAAKDHCAAAA|2001-10-27||Statements may damage. Dependent courses will look a|2.87|2.43|10015013|corpmaxi #1|1|parenting|9|Books|427|barcallyese|N/A|833slate333177237309|wheat|Pound|Unknown|27|eseesebarbarought| +10045|AAAAAAAANDHCAAAA|1997-10-27||Matters cannot choose still jewish rates. Private places give ago big programmes. Afterwards british conditions must return in order. Nuclear legs catch now early, open tenants. So furious|7.42|5.63|5002002|importoscholar #2|2|country|5|Music|724|eseableation|N/A|761528ghost237008823|slate|Ton|Unknown|30|antiesebarbarought| +10046|AAAAAAAAODHCAAAA|1997-10-27|2000-10-26|Objects speed once houses; proper, proposed bars may not see specimens; dark, old things cost separate places. Parliamentary, certain supporters will look companies; fine me|56.15|27.51|2003001|exportiimporto #1|3|pants|2|Men|404|esebarese|extra large|030967yellow11985002|purple|Bundle|Unknown|70|callyesebarbarought| +10047|AAAAAAAAODHCAAAA|2000-10-27||Less alone remains go appropriately institutional, common|7.55|4.60|7008008|namelessbrand #8|8|lighting|7|Home|356|esebarese|N/A|7754624300912spring5|thistle|Oz|Unknown|52|ationesebarbarought| +10048|AAAAAAAAAEHCAAAA|1997-10-27|1999-10-27|Quite social police choose. Recent, old lives go in a voices. Inherent, busy competitors ought to win local, basic titles. However ready years need m|1.71|0.82|8014009|edu packmaxi #9|14|tennis|8|Sports|653|priantically|N/A|7993662074221purple8|saddle|Dram|Unknown|35|eingesebarbarought| +10049|AAAAAAAAAEHCAAAA|1999-10-28|2001-10-26|Quite social police choose. Recent, old lives go in a voices. Inherent, busy competitors ought to win local, basic titles. However ready years need m|2.48|0.82|8014009|namelessunivamalg #3|14|scanners|10|Electronics|653|priantically|N/A|7993662074221purple8|papaya|Gross|Unknown|33|n stesebarbarought| +10050|AAAAAAAAAEHCAAAA|2001-10-27||Quite social police choose. Recent, old lives go in a voices. Inherent, busy competitors ought to win local, basic titles. However ready years need m|3.49|1.08|2003001|exportiimporto #1|14|pants|2|Men|653|priantically|extra large|7993662074221purple8|wheat|Bunch|Unknown|35|barantibarbarought| +10051|AAAAAAAADEHCAAAA|1997-10-27||High huge targets would work of course below good figures. Light spaces own able images. Good changes must appre|0.41|0.34|4004002|edu packedu pack #2|4|athletic|4|Shoes|366|callycallypri|extra large|3139630887762orange2|tomato|Bundle|Unknown|66|oughtantibarbarought| +10052|AAAAAAAAEEHCAAAA|1997-10-27|2000-10-26|Factors get only days. Adult toys will notice conventional, low che|4.33|1.81|1002001|importoamalg #1|2|fragrances|1|Women|293|prin stable|extra large|100464342peach549761|papaya|Gross|Unknown|79|ableantibarbarought| +10053|AAAAAAAAEEHCAAAA|2000-10-27||Factors get only days. Adult toys will notice conventional, low che|2.96|1.81|1002001|importoamalg #2|2|fragrances|1|Women|557|ationantianti|petite|7923009711658slate82|tan|Pound|Unknown|34|priantibarbarought| +10054|AAAAAAAAGEHCAAAA|1997-10-27|1999-10-27|Somehow new conditions start more particularly sexual words; most british men may mask very constant, discipli|2.01|0.96|9013011|exportiunivamalg #11|13|self-help|9|Books|87|ationeing|N/A|439sienna87800552373|tan|Gram|Unknown|82|eseantibarbarought| +10055|AAAAAAAAGEHCAAAA|1999-10-28|2001-10-26|Then early journalists may suit. Long, new areas share with the women. Either significant artists could fill today useful groups. Cases should not |7.95|0.96|1002002|importoamalg #2|13|fragrances|1|Women|635|antiprically|medium|439sienna87800552373|saddle|Gross|Unknown|22|antiantibarbarought| +10056|AAAAAAAAGEHCAAAA|2001-10-27||National, philosophical payments used to stay over a arguments. In short political au|1.02|0.96|8016007|corpmaxi #7|13|golf|8|Sports|635|antiprically|N/A|439sienna87800552373|wheat|Bundle|Unknown|57|callyantibarbarought| +10057|AAAAAAAAJEHCAAAA|1997-10-27||Broadly left categories may hate. Centres reach similarly; big examples should not pay particularly in a things. Quickly significant principles appea|4.77|1.90|4004002|edu packedu pack #2|4|athletic|4|Shoes|217|ationoughtable|medium|0899928812528navajo9|yellow|Ounce|Unknown|43|ationantibarbarought| +10058|AAAAAAAAKEHCAAAA|1997-10-27|2000-10-26|Littl|0.70|0.42|6001003|amalgcorp #3|1|birdal|6|Jewelry|540|bareseanti|N/A|0589269rosy037246635|yellow|Pallet|Unknown|62|eingantibarbarought| +10059|AAAAAAAAKEHCAAAA|2000-10-27||Facilities learn makers. Large, aware objections would appeal later. Ill explanations must tr|9.89|0.42|9010004|univunivamalg #4|1|travel|9|Books|366|bareseanti|N/A|0589269rosy037246635|tan|Tbl|Unknown|25|n stantibarbarought| +10060|AAAAAAAAMEHCAAAA|1997-10-27|1999-10-27|Now large mountains ought to prosecute also. Human events can talk so. Large services might choose female posts; homes can follow english words. Similar, different forests help; plans shall come rela|2.23|1.33|1003001|exportiamalg #1|3|maternity|1|Women|325|antiablepri|medium|20683slate8263905645|aquamarine|Ton|Unknown|9|barcallybarbarought| +10061|AAAAAAAAMEHCAAAA|1999-10-28|2001-10-26|Now large mountains ought to prosecute also. Human events can talk so. Large services might choose female posts; homes can follow english words. Similar, different forests help; plans shall come rela|8.26|5.53|9008006|namelessmaxi #6|8|romance|9|Books|325|antiablepri|N/A|68002118981orchid712|honeydew|Oz|Unknown|73|oughtcallybarbarought| +10062|AAAAAAAAMEHCAAAA|2001-10-27||Now large mountains ought to prosecute also. Human events can talk so. Large services might choose female posts; homes can follow english words. Similar, different forests help; plans shall come rela|3.21|1.79|9008006|exportiimporto #1|8|pants|2|Men|453|priantiese|medium|lace6284699408462563|slate|Dozen|Unknown|4|ablecallybarbarought| +10063|AAAAAAAAPEHCAAAA|1997-10-27||Subjects put. Constant children try. Women receive categories. So subsequent forms get on a parts. Concessions mean with a ships. Cases know by a relations. Key|7.12|4.69|1002002|importoamalg #2|2|fragrances|1|Women|885|antieingeing|N/A|2211942smoke06304856|peach|N/A|Unknown|19|pricallybarbarought| +10064|AAAAAAAAAFHCAAAA|1997-10-27|2000-10-26|Soft origins will not exclude suddenly technical centres. Good current boys may not make there hard new proteins. Good, chemical features could not divide as th|3.77|2.18|5004001|edu packscholar #1|4|classical|5|Music|264|esecallyable|N/A|483817175maroon54972|slate|Gross|Unknown|35|esecallybarbarought| +10065|AAAAAAAAAFHCAAAA|2000-10-27||New standards ought to surprise prospects. In particular precise meetings would give |2.96|2.18|5004001|namelessnameless #6|8|outdoor|8|Sports|895|esecallyable|N/A|483817175maroon54972|slate|Dram|Unknown|58|anticallybarbarought| +10066|AAAAAAAACFHCAAAA|1997-10-27|1999-10-27|Important, ready e|2.31|1.61|1002001|importoamalg #1|2|fragrances|1|Women|38|eingpri|medium|65988spring421587613|seashell|Dozen|Unknown|38|callycallybarbarought| +10067|AAAAAAAACFHCAAAA|1999-10-28|2001-10-26|Eastwards different papers ought to see t|3.08|1.61|1002001|edu packbrand #8|4|curtains/drapes|7|Home|38|eingpri|N/A|65988spring421587613|violet|Ounce|Unknown|72|ationcallybarbarought| +10068|AAAAAAAACFHCAAAA|2001-10-27||Eastwards different papers ought to see t|1.69|1.61|10003005|exportiunivamalg #5|3|dvd/vcr players|10|Electronics|38|eingpri|N/A|65988spring421587613|orange|Lb|Unknown|46|eingcallybarbarought| +10069|AAAAAAAAFFHCAAAA|1997-10-27||Eyes would call today healthy, fresh sets. General victims may refer especi|4.16|1.49|3003002|exportiexporti #2|3|toddlers|3|Children|191|oughtn stought|medium|7613846rosy647480423|puff|Gram|Unknown|19|n stcallybarbarought| +10070|AAAAAAAAGFHCAAAA|1997-10-27|2000-10-26|New, hot terms would end probabl|7.81|5.85|9010003|univunivamalg #3|10|travel|9|Books|609|n stbarcally|N/A|7435283664890pale032|peach|Dram|Unknown|87|barationbarbarought| +10071|AAAAAAAAGFHCAAAA|2000-10-27||Funds let experiences. Days would buy indeed; double, local occasions kill for example. Rates must apply. Great affairs will begin. Needs consider. Low, broad traditions w|3.25|5.85|9010003|edu packimporto #2|10|sports-apparel|2|Men|201|n stbarcally|medium|04437seashell5164152|moccasin|Bunch|Unknown|23|oughtationbarbarought| +10072|AAAAAAAAIFHCAAAA|1997-10-27|1999-10-27|Liberal parties create fast weeks. Good eyes could protect most free, recent chang|3.16|1.89|9005005|scholarmaxi #5|5|history|9|Books|343|priesepri|N/A|79256204grey59646862|misty|Oz|Unknown|2|ableationbarbarought| +10073|AAAAAAAAIFHCAAAA|1999-10-28|2001-10-26|Liberal parties create fast weeks. Good eyes could protect most free, recent chang|3.15|1.89|9005005|exportiexporti #2|3|toddlers|3|Children|256|priesepri|petite|34083976737272coral9|papaya|Dozen|Unknown|77|priationbarbarought| +10074|AAAAAAAAIFHCAAAA|2001-10-27||Partners should not produce to a personnel. Either upper rivers must help long years. Square, prou|9.50|7.31|8013005|exportimaxi #5|3|sailing|8|Sports|653|priantically|N/A|238007lace4600889299|plum|N/A|Unknown|4|eseationbarbarought| +10075|AAAAAAAALFHCAAAA|1997-10-27||Environmental, private charges cannot break huge lines. Of course social securities create extremely large others. Ind|5.67|3.51|5002002|importoscholar #2|2|country|5|Music|249|n steseable|N/A|38289305331slate7904|wheat|Tbl|Unknown|36|antiationbarbarought| +10076|AAAAAAAAMFHCAAAA|1997-10-27|2000-10-26|Eventually effective leads see grey brothers. Others show both for no sorts. Authoriti|8.46|7.61|8012007|importomaxi #7|12|guns|8|Sports|114|eseoughtought|N/A|9030332pink617593060|dark|Each|Unknown|83|callyationbarbarought| +10077|AAAAAAAAMFHCAAAA|2000-10-27||Overseas moments shall listen principal firms. Again true sites used to make very. Foreign, painful earnings admit just good aims. Methods save. Subjects prove various colu|5.02|2.51|7007008|brandbrand #8|7|decor|7|Home|114|eseoughtought|N/A|9030332pink617593060|violet|Each|Unknown|66|ationationbarbarought| +10078|AAAAAAAAOFHCAAAA|1997-10-27|1999-10-27|Mainly isolated ends justify from a shots; occupat|2.06|0.65|9014011|edu packunivamalg #11|14|sports|9|Books|350|barantipri|N/A|869rosy3451317253419|sky|Carton|Unknown|32|eingationbarbarought| +10079|AAAAAAAAOFHCAAAA|1999-10-28|2001-10-26|Mainly isolated ends justify from a shots; occupat|0.83|0.28|1004002|edu packamalg #2|4|swimwear|1|Women|350|barantipri|economy|80203584198tan046416|seashell|Ounce|Unknown|63|n stationbarbarought| +10080|AAAAAAAAOFHCAAAA|2001-10-27||Mainly isolated ends justify from a shots; occupat|2.91|0.28|1004002|exportiexporti #1|4|toddlers|3|Children|350|barantipri|petite|80203584198tan046416|linen|Box|Unknown|13|bareingbarbarought| +10081|AAAAAAAABGHCAAAA|1997-10-27||Details would care again easily mini|4.54|2.22|10012001|importoamalgamalg #1|12|monitors|10|Electronics|136|callypriought|N/A|9621purple4627248547|saddle|Gross|Unknown|42|oughteingbarbarought| +10082|AAAAAAAACGHCAAAA|1997-10-27|2000-10-26|Labour others ought to learn best pure, possible programmes. Services may hope then inc c|2.03|0.73|5003001|exportischolar #1|3|pop|5|Music|605|antibarcally|N/A|2755408steel37329965|tomato|Cup|Unknown|26|ableeingbarbarought| +10083|AAAAAAAACGHCAAAA|2000-10-27||Labour others ought to learn best pure, possible programmes. Services may hope then inc c|4.21|0.73|5003001|amalgexporti #2|3|newborn|3|Children|605|antibarcally|medium|33671471magenta86786|navajo|Carton|Unknown|22|prieingbarbarought| +10084|AAAAAAAAEGHCAAAA|1997-10-27|1999-10-27|Public activities pre|3.25|1.82|1004001|edu packamalg #1|4|swimwear|1|Women|304|esebarpri|petite|838732162192orange02|pale|Tbl|Unknown|12|eseeingbarbarought| +10085|AAAAAAAAEGHCAAAA|1999-10-28|2001-10-26|Public activities pre|5.33|4.26|6003006|exporticorp #6|4|gold|6|Jewelry|304|esebarpri|N/A|838732162192orange02|lavender|Gross|Unknown|10|antieingbarbarought| +10086|AAAAAAAAEGHCAAAA|2001-10-27||Large, long players would assist assistant, true motives; main, easy designs used to play just forward great feelings. Moder|7.04|3.37|6003006|importoedu pack #1|4|mens|4|Shoes|304|esebarpri|extra large|838732162192orange02|antique|Tsp|Unknown|52|callyeingbarbarought| +10087|AAAAAAAAHGHCAAAA|1997-10-27||Y|9.58|3.83|2003002|exportiimporto #2|3|pants|2|Men|607|ationbarcally|large|21157894724saddle596|tan|Unknown|Unknown|81|ationeingbarbarought| +10088|AAAAAAAAIGHCAAAA|1997-10-27|2000-10-26|Professionals should become now annual courts. Reliable, eastern grounds could wait all right new, british planes;|1.56|0.76|10003008|exportiunivamalg #8|3|dvd/vcr players|10|Electronics|62|ablecally|N/A|90navy82502431320043|powder|Bunch|Unknown|40|eingeingbarbarought| +10089|AAAAAAAAIGHCAAAA|2000-10-27||Professionals should become now annual courts. Reliable, eastern grounds could wait all right new, british planes;|3.45|2.89|10003008|edu packamalgamalg #9|3|automotive|10|Electronics|473|priationese|N/A|90navy82502431320043|rose|Dram|Unknown|9|n steingbarbarought| +10090|AAAAAAAAKGHCAAAA|1997-10-27|1999-10-27|Even big conditions enforce there nev|9.41|6.96|5003001|exportischolar #1|3|pop|5|Music|85|antieing|N/A|5rosy305607374982253|navajo|Bunch|Unknown|50|barn stbarbarought| +10091|AAAAAAAAKGHCAAAA|1999-10-28|2001-10-26|Even big conditions enforce there nev|7.31|3.80|5003001|exportischolar #2|3|pop|5|Music|85|antieing|N/A|5rosy305607374982253|seashell|Dozen|Unknown|23|oughtn stbarbarought| +10092|AAAAAAAAKGHCAAAA|2001-10-27||Even big conditions enforce there nev|1.20|0.36|4003001|exportiedu pack #1|3|kids|4|Shoes|85|antieing|petite|5rosy305607374982253|sienna|Carton|Unknown|92|ablen stbarbarought| +10093|AAAAAAAANGHCAAAA|1997-10-27||More sorry women will not meet for example; days enter available, financial unions. Dominant, full payments should not make more wide |0.88|0.57|1002002|importoamalg #2|2|fragrances|1|Women|188|eingeingought|petite|21white2604054678502|puff|Tsp|Unknown|29|prin stbarbarought| +10094|AAAAAAAAOGHCAAAA|1997-10-27|2000-10-26|Political, only hours measure eventually. Urban months blow basically for long new terms. Low minu|0.75|0.30|10001014|amalgunivamalg #14|1|cameras|10|Electronics|207|ationbarable|N/A|31turquoise508829021|peru|Carton|Unknown|8|esen stbarbarought| +10095|AAAAAAAAOGHCAAAA|2000-10-27||Political, only hours measure eventually. Urban months blow basically for long new terms. Low minu|3.00|1.41|3004002|edu packexporti #2|1|school-uniforms|3|Children|207|ationbarable|small|46016peach8649510084|lime|Dram|Unknown|59|antin stbarbarought| +10096|AAAAAAAAAHHCAAAA|1997-10-27|1999-10-27|However other lines could afford just for the groups. Tenants must purchase. British arrangements continue domestic, quick tasks. Traditiona|1.65|1.02|9007005|brandmaxi #5|7|reference|9|Books|80|bareing|N/A|71orchid212766024730|pink|Box|Unknown|84|callyn stbarbarought| +10097|AAAAAAAAAHHCAAAA|1999-10-28|2001-10-26|However other lines could afford just for the groups. Tenants must purchase. British arrangements continue domestic, quick tasks. Traditiona|4.27|1.02|9007005|exportiamalg #2|7|maternity|1|Women|80|bareing|medium|71orchid212766024730|peru|Tbl|Unknown|13|ationn stbarbarought| +10098|AAAAAAAAAHHCAAAA|2001-10-27||However other lines could afford just for the groups. Tenants must purchase. British arrangements continue domestic, quick tasks. Traditiona|5.31|3.87|9007005|maxibrand #9|9|mattresses|7|Home|80|bareing|N/A|71orchid212766024730|wheat|Gross|Unknown|2|eingn stbarbarought| +10099|AAAAAAAADHHCAAAA|1997-10-27||Holes ought to offer much severe, suitable ministers. For example independent steps pick approximately huge relations. Alone, available boats might express in a years; level pati|5.70|4.16|7005010|scholarbrand #10|5|blinds/shades|7|Home|180|bareingought|N/A|63259sienna382872782|pink|Carton|Unknown|23|n stn stbarbarought| +10100|AAAAAAAAEHHCAAAA|1997-10-27|2000-10-26|Just new descriptions help. Organizations must lik|3.70|3.25|10006003|corpunivamalg #3|6|musical|10|Electronics|747|ationeseation|N/A|319077metallic394641|sienna|Tsp|Unknown|89|barbaroughtbarought| +10101|AAAAAAAAEHHCAAAA|2000-10-27||Chemical, wrong words meet less light machines. Useful, sensitive items must take in search of a problems; poor companies must not help resources. Natur|2.84|3.25|10010004|univamalgamalg #4|6|memory|10|Electronics|747|ationeseation|N/A|319077metallic394641|purple|Bundle|Unknown|14|oughtbaroughtbarought| +10102|AAAAAAAAGHHCAAAA|1997-10-27|1999-10-27|Most full steps come upwards. Young, valid images smell good. Open children would get specifically previous, good friends. High, natural visitors will make blindly to a channels|5.57|3.50|3002001|importoexporti #1|2|infants|3|Children|189|n steingought|medium|83532583gainsboro317|sky|Tsp|Unknown|26|ablebaroughtbarought| +10103|AAAAAAAAGHHCAAAA|1999-10-28|2001-10-26|Walls used to see. Related, new items may appoint. Strong areas may use forward beds. Most practical authors move available, pale |6.32|4.42|3002001|exportiedu pack #2|3|kids|4|Shoes|572|n steingought|medium|83532583gainsboro317|white|Bunch|Unknown|97|pribaroughtbarought| +10104|AAAAAAAAGHHCAAAA|2001-10-27||Walls used to see. Related, new items may appoint. Strong areas may use forward beds. Most practical authors move available, pale |8.57|4.42|3002001|scholarunivamalg #7|3|karoke|10|Electronics|572|ableationanti|N/A|sky73824464946850504|plum|Tsp|Unknown|24|esebaroughtbarought| +10105|AAAAAAAAJHHCAAAA|1997-10-27||Definitely vital trends will operate hard, reliable forms. Almost significant friends hide enough domestic conclusions; whole clubs make now causal, loose schools. Cases make twice. De|3.77|2.45|10006008|corpunivamalg #8|6|musical|10|Electronics|153|priantiought|N/A|862576seashell489148|peru|Box|Unknown|11|antibaroughtbarought| +10106|AAAAAAAAKHHCAAAA|1997-10-27|2000-10-26|Especially firm buses listen major, individual calls. New daughters help internal, general eggs. Here l|2.78|1.44|1001001|amalgamalg #1|1|dresses|1|Women|515|antioughtanti|small|6353620goldenrod6280|sienna|Tbl|Unknown|63|callybaroughtbarought| +10107|AAAAAAAAKHHCAAAA|2000-10-27||Especially firm buses listen major, individual calls. New daughters help internal, general eggs. Here l|2.81|1.40|9002004|importomaxi #4|2|business|9|Books|578|antioughtanti|N/A|034cyan9296148739677|slate|Gram|Unknown|40|ationbaroughtbarought| +10108|AAAAAAAAMHHCAAAA|1997-10-27|1999-10-27|Later sure estates give long wonderful signs. Wide divisions warm with a observers. Formal, necessary colleg|2.57|1.36|8004009|edu packnameless #9|4|camping|8|Sports|811|oughtoughteing|N/A|9972sienna8783280796|thistle|Ounce|Unknown|33|eingbaroughtbarought| +10109|AAAAAAAAMHHCAAAA|1999-10-28|2001-10-26|Later sure estates give long wonderful signs. Wide divisions warm with a observers. Formal, necessary colleg|9.99|1.36|9005006|scholarmaxi #6|5|history|9|Books|207|ationbarable|N/A|9972sienna8783280796|seashell|Ounce|Unknown|7|n stbaroughtbarought| +10110|AAAAAAAAMHHCAAAA|2001-10-27||Later sure estates give long wonderful signs. Wide divisions warm with a observers. Formal, necessary colleg|9.70|1.36|9005006|exportimaxi #1|13|sailing|8|Sports|207|ationbarable|N/A|pale5267705846996992|tan|Pallet|Unknown|25|baroughtoughtbarought| +10111|AAAAAAAAPHHCAAAA|1997-10-27||Content, short villages develop with the children. Matters come shares. Criminal, other senses report quite early |9.70|5.91|1001002|amalgamalg #2|1|dresses|1|Women|412|ableoughtese|medium|08654929016pale95494|rose|Cup|Unknown|28|oughtoughtoughtbarought| +10112|AAAAAAAAAIHCAAAA|1997-10-27|2000-10-26|Still arguments must undertake su|5.27|3.32|1002001|importoamalg #1|2|fragrances|1|Women|239|n stpriable|small|46058608333wheat3607|tan|Carton|Unknown|100|ableoughtoughtbarought| +10113|AAAAAAAAAIHCAAAA|2000-10-27||Still arguments must undertake su|3.35|1.50|8012004|importomaxi #4|2|guns|8|Sports|188|eingeingought|N/A|2541719steel70882488|seashell|Case|Unknown|41|prioughtoughtbarought| +10114|AAAAAAAACIHCAAAA|1997-10-27|1999-10-27|English words ought to achieve much about a laws. Strong, british areas expect here major modules. Ethnic, liable lengths see equally terms. Large neighbours will hope minutes; o|0.74|0.54|7014005|edu packnameless #5|14|glassware|7|Home|334|esepripri|N/A|8944831red8279241298|violet|Gram|Unknown|29|eseoughtoughtbarought| +10115|AAAAAAAACIHCAAAA|1999-10-28|2001-10-26|English words ought to achieve much about a laws. Strong, british areas expect here major modules. Ethnic, liable lengths see equally terms. Large neighbours will hope minutes; o|7.30|5.54|7014005|exportimaxi #12|3|computers|9|Books|334|esepripri|N/A|7389914733pale351411|plum|Dozen|Unknown|23|antioughtoughtbarought| +10116|AAAAAAAACIHCAAAA|2001-10-27||English words ought to achieve much about a laws. Strong, british areas expect here major modules. Ethnic, liable lengths see equally terms. Large neighbours will hope minutes; o|2.81|5.54|10004002|edu packunivamalg #2|3|audio|10|Electronics|275|antiationable|N/A|55972626navajo265938|powder|N/A|Unknown|65|callyoughtoughtbarought| +10117|AAAAAAAAFIHCAAAA|1997-10-27||Hard specialists could deal now royal beds. Now high vehicles boycott fingers. National, british students operate pop|2.46|1.79|9012002|importounivamalg #2|12|home repair|9|Books|149|n steseought|N/A|42899910chartreuse42|papaya|Ounce|Unknown|5|ationoughtoughtbarought| +10118|AAAAAAAAGIHCAAAA|1997-10-27|2000-10-26|Friendly, italian years return preferably ne|8.16|5.46|8011009|amalgmaxi #9|11|archery|8|Sports|81|oughteing|N/A|8345393186188peach57|pink|Lb|Unknown|12|eingoughtoughtbarought| +10119|AAAAAAAAGIHCAAAA|2000-10-27||Old managers learn happy, black refugees. Patients may convey already in addition major years. Formal, small letters may not say yesterday conditions. Elections shal|55.89|5.46|5001002|amalgscholar #2|11|rock|5|Music|81|oughteing|N/A|8345393186188peach57|papaya|Each|Unknown|8|n stoughtoughtbarought| +10120|AAAAAAAAIIHCAAAA|1997-10-27|1999-10-27|Largely certain others face human neighbours. Active arts will not sell as; clothes would not tell tonight|7.85|2.35|4003001|exportiedu pack #1|3|kids|4|Shoes|505|antibaranti|small|440354wheat034877675|wheat|Carton|Unknown|7|barableoughtbarought| +10121|AAAAAAAAIIHCAAAA|1999-10-28|2001-10-26|Outside possible germans study somehow so inc revenues. Lives mount doubtless more than sharp claims. Only, natural parts may harness schools. Turkish girls must n|0.46|0.32|4003001|maxiunivamalg #7|3|televisions|10|Electronics|505|antibaranti|N/A|440354wheat034877675|pale|Dozen|Unknown|23|oughtableoughtbarought| +10122|AAAAAAAAIIHCAAAA|2001-10-27||Outside possible germans study somehow so inc revenues. Lives mount doubtless more than sharp claims. Only, natural parts may harness schools. Turkish girls must n|9.79|0.32|3003001|exportiexporti #1|3|toddlers|3|Children|266|callycallyable|extra large|88652166goldenrod355|wheat|Dozen|Unknown|29|ableableoughtbarought| +10123|AAAAAAAALIHCAAAA|1997-10-27||Things select increased views. Tools imagine for example; ever likely developments live so scottish benefits. Soft, short-term services should give indeed good illu|86.90|64.30|3004002|edu packexporti #2|4|school-uniforms|3|Children|270|barationable|petite|86rosy98224943879396|spring|Ounce|Unknown|20|priableoughtbarought| +10124|AAAAAAAAMIHCAAAA|1997-10-27|2000-10-26|Clients could attempt that is to say now warm days; national problems would not belong for a stars. Issues write thereafter cases. Successful years add together perhaps easy ye|9.95|7.76|9007009|brandmaxi #9|7|reference|9|Books|113|prioughtought|N/A|1484398858salmon1798|steel|Bundle|Unknown|47|eseableoughtbarought| +10125|AAAAAAAAMIHCAAAA|2000-10-27||Clients could attempt that is to say now warm days; national problems would not belong for a stars. Issues write thereafter cases. Successful years add together perhaps easy ye|3.79|7.76|7010006|univnameless #6|10|flatware|7|Home|113|prioughtought|N/A|1484398858salmon1798|turquoise|N/A|Unknown|8|antiableoughtbarought| +10126|AAAAAAAAOIHCAAAA|1997-10-27|1999-10-27|Economic terms will not establish certain carers; distinguished acids go for example. Tory resources shall put normally perhaps detailed subjects. Wide emotions|82.56|28.89|8009007|maxinameless #7|9|optics|8|Sports|829|n stableeing|N/A|13297752seashell4170|steel|Tbl|Unknown|83|callyableoughtbarought| +10127|AAAAAAAAOIHCAAAA|1999-10-28|2001-10-26|Economic terms will not establish certain carers; distinguished acids go for example. Tory resources shall put normally perhaps detailed subjects. Wide emotions|1.94|1.37|8009007|scholarnameless #8|9|fitness|8|Sports|37|n stableeing|N/A|13297752seashell4170|tan|Pound|Unknown|23|ationableoughtbarought| +10128|AAAAAAAAOIHCAAAA|2001-10-27||Economic terms will not establish certain carers; distinguished acids go for example. Tory resources shall put normally perhaps detailed subjects. Wide emotions|98.56|76.87|1003001|exportiamalg #1|9|maternity|1|Women|35|antipri|petite|13297752seashell4170|papaya|Carton|Unknown|13|eingableoughtbarought| +10129|AAAAAAAABJHCAAAA|1997-10-27||Always financial muscles rest really standard walls. Frames cannot work particularly nearly real conferences. Central, special services result upstairs sharp forests. Eastern,|6.50|4.35|1003002|exportiamalg #2|3|maternity|1|Women|167|ationcallyought|extra large|5221771076seashell58|mint|Unknown|Unknown|73|n stableoughtbarought| +10130|AAAAAAAACJHCAAAA|1997-10-27|2000-10-26|Electoral |2.01|1.28|6003003|exporticorp #3|3|gold|6|Jewelry|123|priableought|N/A|13074056violet767972|peru|Pound|Unknown|21|barprioughtbarought| +10131|AAAAAAAACJHCAAAA|2000-10-27||Probable, central babies maintain difficult, subtle choices. General words can call more. Possible, personal terms |2.66|1.28|6003003|scholarnameless #2|15|tables|7|Home|123|priableought|N/A|67navajo979114511390|powder|Box|Unknown|31|oughtprioughtbarought| +10132|AAAAAAAAEJHCAAAA|1997-10-27|1999-10-27|Gains find hard original p|0.65|0.22|5004001|edu packscholar #1|4|classical|5|Music|517|ationoughtanti|N/A|7289861706341navajo1|yellow|Gross|Unknown|30|ableprioughtbarought| +10133|AAAAAAAAEJHCAAAA|1999-10-28|2001-10-26|Inside central hours obtain enough however blue sales. Tourists should not ban inner, great muscles. Dead students could supply things. Com|5.25|3.09|5004001|edu packscholar #2|4|classical|5|Music|517|ationoughtanti|N/A|7289861706341navajo1|honeydew|Gram|Unknown|6|priprioughtbarought| +10134|AAAAAAAAEJHCAAAA|2001-10-27||Inside central hours obtain enough however blue sales. Tourists should not ban inner, great muscles. Dead students could supply things. Com|8.44|4.72|6015007|scholarbrand #7|4|custom|6|Jewelry|317|ationoughtpri|N/A|287666679royal144260|linen|Tsp|Unknown|46|eseprioughtbarought| +10135|AAAAAAAAHJHCAAAA|1997-10-27||Other accountants may not f|8.49|4.32|1002002|importoamalg #2|2|fragrances|1|Women|524|eseableanti|petite|4043700126turquoise7|purple|Oz|Unknown|61|antiprioughtbarought| +10136|AAAAAAAAIJHCAAAA|1997-10-27|2000-10-26|Suddenly current difficulties should get only usual, fair proposals; quiet parts may not give points. Small cars run so human, good sources. Men pay. Social, organic offenders take passengers|1.06|0.46|3004001|edu packexporti #1|4|school-uniforms|3|Children|185|antieingought|large|192268935yellow88139|purple|N/A|Unknown|27|callyprioughtbarought| +10137|AAAAAAAAIJHCAAAA|2000-10-27||Modest fruits can discharge fine eyes; working, firm actions ought to follow more plans. Even other results will learn for a magazines|0.41|0.28|1004002|edu packamalg #2|4|swimwear|1|Women|185|antieingought|petite|192268935yellow88139|turquoise|Each|Unknown|38|ationprioughtbarought| +10138|AAAAAAAAKJHCAAAA|1997-10-27|1999-10-27|Female writers speak. Then fo|3.98|1.31|10002007|importounivamalg #7|2|camcorders|10|Electronics|71|oughtation|N/A|1153526894440pink464|smoke|Ton|Unknown|63|eingprioughtbarought| +10139|AAAAAAAAKJHCAAAA|1999-10-28|2001-10-26|Female writers speak. Then fo|9.43|1.31|10002007|exportischolar #2|2|pop|5|Music|71|oughtation|N/A|9754083cornsilk50206|lime|Dozen|Unknown|21|n stprioughtbarought| +10140|AAAAAAAAKJHCAAAA|2001-10-27||Female writers speak. Then fo|5.66|3.50|10002007|exportimaxi #1|13|sailing|8|Sports|559|oughtation|N/A|22161orchid114933644|lemon|Dram|Unknown|9|bareseoughtbarought| +10141|AAAAAAAANJHCAAAA|1997-10-27||Fully early countries ought to amount of course children. Over total |9.43|6.50|2001002|amalgimporto #2|1|accessories|2|Men|507|ationbaranti|small|6828midnight33076724|red|Carton|Unknown|96|oughteseoughtbarought| +10142|AAAAAAAAOJHCAAAA|1997-10-27|2000-10-26|Quietly complex samples may not indicate eventually good major parties. Markets suppose shortly little, only centres. Cultural guests can experience institutional damages. Sales must not|2.50|1.72|4004001|edu packedu pack #1|4|athletic|4|Shoes|311|oughtoughtpri|small|7862679536goldenrod9|slate|Unknown|Unknown|100|ableeseoughtbarought| +10143|AAAAAAAAOJHCAAAA|2000-10-27||Quietly complex samples may not indicate eventually good major parties. Markets suppose shortly little, only centres. Cultural guests can experience institutional damages. Sales must not|8.22|2.79|4004001|exportiamalg #2|4|maternity|1|Women|217|oughtoughtpri|medium|7862679536goldenrod9|sienna|Carton|Unknown|1|prieseoughtbarought| +10144|AAAAAAAAAKHCAAAA|1997-10-27|1999-10-27|Yesterday ill charges used to know further slow, beautiful proposal|0.63|0.44|10011013|amalgamalgamalg #13|11|disk drives|10|Electronics|181|oughteingought|N/A|403690608steel098044|slate|Case|Unknown|5|eseeseoughtbarought| +10145|AAAAAAAAAKHCAAAA|1999-10-28|2001-10-26|Yesterday ill charges used to know further slow, beautiful proposal|6.93|0.44|2002002|importoimporto #2|2|shirts|2|Men|269|n stcallyable|small|403690608steel098044|lime|Gross|Unknown|8|antieseoughtbarought| +10146|AAAAAAAAAKHCAAAA|2001-10-27||Projects catch here sound year|61.34|0.44|8002007|importonameless #7|2|baseball|8|Sports|54|eseanti|N/A|403690608steel098044|seashell|Gross|Unknown|31|callyeseoughtbarought| +10147|AAAAAAAADKHCAAAA|1997-10-27||Programmes meet there average colours. Difficult students say etc cultural times. Children mig|3.45|1.34|2001002|amalgimporto #2|1|accessories|2|Men|36|callypri|extra large|chartreuse3427696501|smoke|Pound|Unknown|100|ationeseoughtbarought| +10148|AAAAAAAAEKHCAAAA|1997-10-27|2000-10-26|Volunteers ought to cover similarly useful tracks; mixed, young|4.34|1.69|10005017|scholarunivamalg #17|5|karoke|10|Electronics|451|oughtantiese|N/A|697507seashell972981|red|N/A|Unknown|34|eingeseoughtbarought| +10149|AAAAAAAAEKHCAAAA|2000-10-27||Available yards should form commonly of course strange stages. Explicitly local documents imply expected proposals. Cases can close surely chemical officers. Women find again in the func|1.50|0.88|7013010|exportinameless #10|13|wallpaper|7|Home|309|oughtantiese|N/A|697507seashell972981|cornflower|Ton|Unknown|21|n steseoughtbarought| +10150|AAAAAAAAGKHCAAAA|1997-10-27|1999-10-27|Subjects stay mo|2.48|1.76|6007007|brandcorp #7|7|pendants|6|Jewelry|333|pripripri|N/A|54sandy4194470336705|purple|Carton|Unknown|81|barantioughtbarought| +10151|AAAAAAAAGKHCAAAA|1999-10-28|2001-10-26|Subjects stay mo|2.19|1.76|10015003|scholaramalgamalg #3|7|portable|10|Electronics|41|pripripri|N/A|17601turquoise384113|sky|Dram|Unknown|49|oughtantioughtbarought| +10152|AAAAAAAAGKHCAAAA|2001-10-27||Subjects stay mo|9.82|2.94|7003003|exportibrand #3|7|kids|7|Home|41|oughtese|N/A|6990013195980440tan0|puff|Tbl|Unknown|72|ableantioughtbarought| +10153|AAAAAAAAJKHCAAAA|1997-10-27||Numbers|0.18|0.09|4004002|edu packedu pack #2|4|athletic|4|Shoes|381|oughteingpri|petite|308327lawn5269696966|steel|Each|Unknown|71|priantioughtbarought| +10154|AAAAAAAAKKHCAAAA|1997-10-27|2000-10-26|Never practical foods include too events. Still close sports understand weekly, essential areas|3.96|3.36|5002001|importoscholar #1|2|country|5|Music|67|ationcally|N/A|4582549839162salmon6|ivory|Tsp|Unknown|41|eseantioughtbarought| +10155|AAAAAAAAKKHCAAAA|2000-10-27||Practices would evacuate at a fi|1.04|0.36|5002001|edu packimporto #2|2|sports-apparel|2|Men|456|callyantiese|small|4582549839162salmon6|midnight|Gross|Unknown|44|antiantioughtbarought| +10156|AAAAAAAAMKHCAAAA|1997-10-27|1999-10-27|Much national technologies settle so. Items see between a years. Following bills must not get. Cats say still average, genetic times. New hours could control cha|0.25|0.22|10008008|namelessunivamalg #8|8|scanners|10|Electronics|500|barbaranti|N/A|51730264895466wheat2|plum|Cup|Unknown|53|callyantioughtbarought| +10157|AAAAAAAAMKHCAAAA|1999-10-28|2001-10-26|Much national technologies settle so. Items see between a years. Following bills must not get. Cats say still average, genetic times. New hours could control cha|9.84|0.22|10008008|importoimporto #2|8|shirts|2|Men|500|barbaranti|large|92tan982117831011361|papaya|Ton|Unknown|3|ationantioughtbarought| +10158|AAAAAAAAMKHCAAAA|2001-10-27||Much national technologies settle so. Items see between a years. Following bills must not get. Cats say still average, genetic times. New hours could control cha|6.39|4.47|5003001|exportischolar #1|3|pop|5|Music|245|barbaranti|N/A|92tan982117831011361|drab|Tbl|Unknown|27|eingantioughtbarought| +10159|AAAAAAAAPKHCAAAA|1997-10-27||Ministers hear less christian securities. Democratic rates can conjure as black, right effects. Individual agents cannot h|9.37|7.40|6015008|scholarbrand #8|15|custom|6|Jewelry|287|ationeingable|N/A|86smoke6428453713085|steel|Pound|Unknown|13|n stantioughtbarought| +10160|AAAAAAAAALHCAAAA|1997-10-27|2000-10-26|Other differences must not make as quiet, modern students.|23.35|18.91|5003001|exportischolar #1|3|pop|5|Music|954|eseantin st|N/A|335150756066sandy758|red|Tsp|Unknown|7|barcallyoughtbarought| +10161|AAAAAAAAALHCAAAA|2000-10-27||Urban, possible colonies must not face with the arms. Most limited resources ought to take extremely by a views; relations come there current, unknown authorities. Supe|1.93|18.91|5003001|maxibrand #2|9|mattresses|7|Home|420|barableese|N/A|335150756066sandy758|rosy|Box|Unknown|76|oughtcallyoughtbarought| +10162|AAAAAAAACLHCAAAA|1997-10-27|1999-10-27|So small commentato|8.88|4.52|6005003|scholarcorp #3|5|earings|6|Jewelry|455|antiantiese|N/A|47102449tan516880930|seashell|Gram|Unknown|58|ablecallyoughtbarought| +10163|AAAAAAAACLHCAAAA|1999-10-28|2001-10-26|So small commentato|9.72|4.52|10007015|brandunivamalg #15|7|personal|10|Electronics|455|antiantiese|N/A|90903543burnished023|yellow|Tsp|Unknown|28|pricallyoughtbarought| +10164|AAAAAAAACLHCAAAA|2001-10-27||So small commentato|9.49|4.52|10009016|maxiunivamalg #16|7|televisions|10|Electronics|455|antiantiese|N/A|90903543burnished023|steel|Cup|Unknown|17|esecallyoughtbarought| +10165|AAAAAAAAFLHCAAAA|1997-10-27||Weeks may keep definitely for certain labour positions. Quick, local types scale fiercely general things; home small mod|2.40|0.86|10010017|univamalgamalg #17|10|memory|10|Electronics|386|callyeingpri|N/A|royal360115083651618|sky|Dram|Unknown|17|anticallyoughtbarought| +10166|AAAAAAAAGLHCAAAA|1997-10-27|2000-10-26|Dimensions might eat closely ways; r|5.58|3.01|4003001|exportiedu pack #1|3|kids|4|Shoes|120|barableought|extra large|58608765plum54080247|red|Cup|Unknown|24|callycallyoughtbarought| +10167|AAAAAAAAGLHCAAAA|2000-10-27||Dimensions might eat closely ways; r|7.02|3.86|4003001|exportiimporto #2|3|pants|2|Men|288|eingeingable|medium|2magenta282568861646|wheat|Oz|Unknown|41|ationcallyoughtbarought| +10168|AAAAAAAAILHCAAAA|1997-10-27|1999-10-27|As full attitudes feel usually largely cheap seconds. Books send on a interactions. Personal, large mistakes push on a reports. Players choose into a schools. All organ|3.70|1.55|5001001|amalgscholar #1|1|rock|5|Music|154|eseantiought|N/A|60160833054962peach2|sandy|Dozen|Unknown|38|eingcallyoughtbarought| +10169|AAAAAAAAILHCAAAA|1999-10-28|2001-10-26|White, original ladies would not check yesterday easy moments. Key, able estimates blame red police. Other, only subjects could not take numbers. Improvements used to look to|4.64|1.55|5001001|brandmaxi #6|1|reference|9|Books|154|eseantiought|N/A|11936misty0248373869|purple|Cup|Unknown|12|n stcallyoughtbarought| +10170|AAAAAAAAILHCAAAA|2001-10-27||White, original ladies would not check yesterday easy moments. Key, able estimates blame red police. Other, only subjects could not take numbers. Improvements used to look to|3.99|1.55|7007001|brandbrand #1|1|decor|7|Home|154|eseantiought|N/A|08778271cornsilk6398|navajo|Box|Unknown|62|barationoughtbarought| +10171|AAAAAAAALLHCAAAA|1997-10-27||Secondary, british forces cou|3.20|1.02|8016002|corpmaxi #2|16|golf|8|Sports|873|priationeing|N/A|2983739357salmon8776|royal|Box|Unknown|79|oughtationoughtbarought| +10172|AAAAAAAAMLHCAAAA|1997-10-27|2000-10-26|Still blue others imagine together initial letters. Political thousands beat in a animals|4.13|2.51|10010007|univamalgamalg #7|10|memory|10|Electronics|634|eseprically|N/A|0595664244yellow5211|salmon|N/A|Unknown|53|ableationoughtbarought| +10173|AAAAAAAAMLHCAAAA|2000-10-27||Still blue others imagine together initial letters. Political thousands beat in a animals|0.79|0.70|2001002|amalgimporto #2|1|accessories|2|Men|23|eseprically|petite|64674268firebrick058|mint|Ounce|Unknown|60|priationoughtbarought| +10174|AAAAAAAAOLHCAAAA|1997-10-27|1999-10-27|Cases tell most european attitudes. Still intern|48.22|42.43|2004001|edu packimporto #1|4|sports-apparel|2|Men|181|oughteingought|large|spring56353673495058|tomato|Dram|Unknown|84|eseationoughtbarought| +10175|AAAAAAAAOLHCAAAA|1999-10-28|2001-10-26|Cases tell most european attitudes. Still intern|10.00|7.40|2004001|importoexporti #2|2|infants|3|Children|86|oughteingought|extra large|774952548pale6003324|plum|Lb|Unknown|98|antiationoughtbarought| +10176|AAAAAAAAOLHCAAAA|2001-10-27||Cases tell most european attitudes. Still intern|1.57|0.94|7002007|importobrand #7|2|bedding|7|Home|86|callyeing|N/A|goldenrod17275171097|thistle|Cup|Unknown|18|callyationoughtbarought| +10177|AAAAAAAABMHCAAAA|1997-10-27||Forces might place home. Professional lawyers might not grant for the schools. Competiti|92.40|81.31|7008008|namelessbrand #8|8|lighting|7|Home|240|bareseable|N/A|1565231294slate04783|tan|Box|Unknown|100|ationationoughtbarought| +10178|AAAAAAAACMHCAAAA|1997-10-27|2000-10-26|Partners used to say in a measures; special uses know symptoms. New, high minutes eat. Extremely vital candidates might go parties; we|8.52|4.94|3003001|exportiexporti #1|3|toddlers|3|Children|155|antiantiought|large|906936929103snow1349|snow|Dram|Unknown|89|eingationoughtbarought| +10179|AAAAAAAACMHCAAAA|2000-10-27||Partners used to say in a measures; special uses know symptoms. New, high minutes eat. Extremely vital candidates might go parties; we|3.18|4.94|9003004|exportimaxi #4|3|computers|9|Books|155|antiantiought|N/A|317926khaki967313267|salmon|Each|Unknown|7|n stationoughtbarought| +10180|AAAAAAAAEMHCAAAA|1997-10-27|1999-10-27|As great officials |2.21|1.43|6004005|edu packcorp #5|4|bracelets|6|Jewelry|513|prioughtanti|N/A|24slate4100897090108|slate|Ounce|Unknown|58|bareingoughtbarought| +10181|AAAAAAAAEMHCAAAA|1999-10-28|2001-10-26|As great officials |3.64|3.27|6004005|importoexporti #2|4|infants|3|Children|382|ableeingpri|small|06702311965846white1|metallic|Unknown|Unknown|64|oughteingoughtbarought| +10182|AAAAAAAAEMHCAAAA|2001-10-27||As great officials |2.69|1.64|6004005|corpamalgamalg #17|4|wireless|10|Electronics|382|ableeingpri|N/A|0peach63964566586533|saddle|Unknown|Unknown|66|ableeingoughtbarought| +10183|AAAAAAAAHMHCAAAA|1997-10-27||Commercial views must eat directly memories. National, great arms make regional years. Single professionals shall|3.48|1.46|10011001|amalgamalgamalg #1|11|disk drives|10|Electronics|154|eseantiought|N/A|magenta1478094465282|turquoise|Dram|Unknown|19|prieingoughtbarought| +10184|AAAAAAAAIMHCAAAA|1997-10-27|2000-10-26|Finally faint schools cannot reflect about from the lines. Particularly overseas tests encoura|1.70|0.59|5001001|amalgscholar #1|1|rock|5|Music|115|antioughtought|N/A|452553070light534089|royal|Oz|Unknown|14|eseeingoughtbarought| +10185|AAAAAAAAIMHCAAAA|2000-10-27||Social, social hands will find still pale services. Wonderful, close components s|2.69|1.96|4004002|edu packedu pack #2|4|athletic|4|Shoes|405|antioughtought|petite|661goldenrod57151440|puff|Bundle|Unknown|11|antieingoughtbarought| +10186|AAAAAAAAKMHCAAAA|1997-10-27|1999-10-27|Environmental car|1.91|1.58|9015011|scholarunivamalg #11|15|fiction|9|Books|58|einganti|N/A|94peru29552095161734|royal|Ton|Unknown|11|callyeingoughtbarought| +10187|AAAAAAAAKMHCAAAA|1999-10-28|2001-10-26|Days use more in a sales. Occasions d|7.44|3.05|7014008|edu packnameless #8|15|glassware|7|Home|78|eingation|N/A|94peru29552095161734|pink|Box|Unknown|59|ationeingoughtbarought| +10188|AAAAAAAAKMHCAAAA|2001-10-27||Forces release almost in a thanks. Strong stars shall write apart for a plants. Main prisoners plan over. Roots would not ignore western,|4.79|3.35|5002001|importoscholar #1|2|country|5|Music|78|eingation|N/A|696737420ivory207216|firebrick|Ton|Unknown|20|eingeingoughtbarought| +10189|AAAAAAAANMHCAAAA|1997-10-27||Both inner employees would think also now financial houses. High, good programmes make most personal doctors. Hands see originally single structures. Flexible times refer. Years arrive big, bi|1.28|1.01|5003002|exportischolar #2|3|pop|5|Music|64|esecally|N/A|110750474118640lace6|purple|Ounce|Unknown|24|n steingoughtbarought| +10190|AAAAAAAAOMHCAAAA|1997-10-27|2000-10-26|Full, japanese planes make par|84.35|60.73|8001001|amalgnameless #1|1|athletic shoes|8|Sports|87|ationeing|N/A|61goldenrod952078704|puff|Oz|Unknown|51|barn stoughtbarought| +10191|AAAAAAAAOMHCAAAA|2000-10-27||Small bills coincide. Diplomatic markets stick always alone implications.|8.80|60.73|5002002|importoscholar #2|2|country|5|Music|87|ationeing|N/A|369109109pale9016376|salmon|Unknown|Unknown|8|oughtn stoughtbarought| +10192|AAAAAAAAANHCAAAA|1997-10-27|1999-10-27|High ministers should not remove for a stations. Certain, linear weeks might not ask so from a improvements. Lakes must not implement f|4.80|1.77|9002005|importomaxi #5|2|business|9|Books|42|ableese|N/A|96215698980spring163|midnight|Pallet|Unknown|15|ablen stoughtbarought| +10193|AAAAAAAAANHCAAAA|1999-10-28|2001-10-26|High ministers should not remove for a stations. Certain, linear weeks might not ask so from a improvements. Lakes must not implement f|8.12|4.46|9002005|importomaxi #4|12|guns|8|Sports|42|ableese|N/A|5goldenrod2332893711|spring|Pound|Unknown|75|prin stoughtbarought| +10194|AAAAAAAAANHCAAAA|2001-10-27||Cold, different branches notice all wide things. Ready chapters eat true, |2.58|4.46|2001001|amalgimporto #1|12|accessories|2|Men|645|ableese|extra large|5goldenrod2332893711|yellow|Ounce|Unknown|3|esen stoughtbarought| +10195|AAAAAAAADNHCAAAA|1997-10-27||Particular, grand plans could not invite significant shareholders. Ye|7.39|3.17|6004004|edu packcorp #4|4|bracelets|6|Jewelry|982|ableeingn st|N/A|898802peru1951086765|seashell|Lb|Unknown|26|antin stoughtbarought| +10196|AAAAAAAAENHCAAAA|1997-10-27|2000-10-26|Valuable, public years stare remarkably soft problems. Issues used to ask careful, brief rates. Big, special conditions must climb to the|0.89|0.28|10013014|exportiamalgamalg #14|13|stereo|10|Electronics|99|n stn st|N/A|0431361050metallic83|papaya|Ton|Unknown|73|callyn stoughtbarought| +10197|AAAAAAAAENHCAAAA|2000-10-27||Universal circumstances should return so only following members. Affairs should not learn; too new |4.40|2.64|10013014|exportiimporto #2|3|pants|2|Men|99|n stn st|large|0431361050metallic83|indian|N/A|Unknown|77|ationn stoughtbarought| +10198|AAAAAAAAGNHCAAAA|1997-10-27|1999-10-27|Central scientists shall not keep also in the countries. Other, financial authorities could not experience deep, other banks. Cells may avoid on the animals;|4.28|3.59|9009011|maximaxi #11|9|science|9|Books|505|antibaranti|N/A|8665115999turquoise8|snow|Case|Unknown|90|eingn stoughtbarought| +10199|AAAAAAAAGNHCAAAA|1999-10-28|2001-10-26|Central scientists shall not keep also in the countries. Other, financial authorities could not experience deep, other banks. Cells may avoid on the animals;|2.52|1.91|8002010|importonameless #10|9|baseball|8|Sports|234|antibaranti|N/A|73030447indian763294|slate|Each|Unknown|76|n stn stoughtbarought| +10200|AAAAAAAAGNHCAAAA|2001-10-27||Central scientists shall not keep also in the countries. Other, financial authorities could not experience deep, other banks. Cells may avoid on the animals;|1.67|1.91|8002010|brandunivamalg #1|9|personal|10|Electronics|17|ationought|N/A|446105112188178mint9|peru|Pallet|Unknown|3|barbarablebarought| +10201|AAAAAAAAJNHCAAAA|1997-10-27||E|1.04|0.83|5001002|amalgscholar #2|1|rock|5|Music|573|priationanti|N/A|187047121pale7074892|forest|Each|Unknown|21|oughtbarablebarought| +10202|AAAAAAAAKNHCAAAA|1997-10-27|2000-10-26|French figures help all years. Other, afraid deaths would cope also purposes; female minutes enter then instead incredible positions. Outsid|7.58|5.68|4003001|exportiedu pack #1|3|kids|4|Shoes|462|ablecallyese|medium|12636894042lace94893|rosy|Dozen|Unknown|40|ablebarablebarought| +10203|AAAAAAAAKNHCAAAA|2000-10-27||Other, great terms control local, young elements. Well impressive toys ignore interests. Writers can make |19.35|7.35|7004004|edu packbrand #4|3|curtains/drapes|7|Home|327|ablecallyese|N/A|5751snow917496331228|misty|Dozen|Unknown|43|pribarablebarought| +10204|AAAAAAAAMNHCAAAA|1997-10-27|1999-10-27|Dead sports get longer; degrees must not spend directly military, natural possibilities. English institutions might receive. Below tory workers cling so new, illegal |3.35|2.07|5004001|edu packscholar #1|4|classical|5|Music|570|barationanti|N/A|0567451plum897448506|tomato|Lb|Unknown|72|esebarablebarought| +10205|AAAAAAAAMNHCAAAA|1999-10-28|2001-10-26|Dead sports get longer; degrees must not spend directly military, natural possibilities. English institutions might receive. Below tory workers cling so new, illegal |7.26|2.07|5004001|importonameless #6|4|baseball|8|Sports|152|barationanti|N/A|0567451plum897448506|royal|Dozen|Unknown|91|antibarablebarought| +10206|AAAAAAAAMNHCAAAA|2001-10-27||Dead sports get longer; degrees must not spend directly military, natural possibilities. English institutions might receive. Below tory workers cling so new, illegal |1.86|1.09|1002001|importoamalg #1|4|fragrances|1|Women|152|ableantiought|petite|0567451plum897448506|royal|Box|Unknown|31|callybarablebarought| +10207|AAAAAAAAPNHCAAAA|1997-10-27||Fresh, industrial vegetables could proceed quite i|7.16|2.57|10005008|scholarunivamalg #8|5|karoke|10|Electronics|80|bareing|N/A|52135rose73305213980|seashell|Pallet|Unknown|12|ationbarablebarought| +10208|AAAAAAAAAOHCAAAA|1997-10-27|2000-10-26|Animals conduct primary, sorry tickets. Bare, labour matters used to buy between a patterns. Light authorities explain rules; future eggs can |0.86|0.47|1004001|edu packamalg #1|4|swimwear|1|Women|676|callyationcally|small|83550607838sienna227|midnight|Gross|Unknown|5|eingbarablebarought| +10209|AAAAAAAAAOHCAAAA|2000-10-27||Social, regional years could not conduct hard so|8.99|0.47|9014010|edu packunivamalg #10|14|sports|9|Books|31|oughtpri|N/A|83550607838sienna227|sky|Dozen|Unknown|19|n stbarablebarought| +10210|AAAAAAAACOHCAAAA|1997-10-27|1999-10-27|Various numbers help only. Instru|5.75|2.18|6010003|univbrand #3|10|jewelry boxes|6|Jewelry|168|eingcallyought|N/A|641007violet50206566|peru|Case|Unknown|55|baroughtablebarought| +10211|AAAAAAAACOHCAAAA|1999-10-28|2001-10-26|Various numbers help only. Instru|4.76|3.66|6010003|importounivamalg #12|12|home repair|9|Books|200|eingcallyought|N/A|1857magenta196975858|papaya|Pallet|Unknown|28|oughtoughtablebarought| +10212|AAAAAAAACOHCAAAA|2001-10-27||Various numbers help only. Instru|7.75|3.87|6010003|importoedu pack #1|2|mens|4|Shoes|200|barbarable|medium|34589216011metallic3|mint|Pallet|Unknown|11|ableoughtablebarought| +10213|AAAAAAAAFOHCAAAA|1997-10-27||Existing computers occur es|0.47|0.33|4001002|amalgedu pack #2|1|womens|4|Shoes|308|eingbarpri|large|99powder310947673781|dim|Box|Unknown|84|prioughtablebarought| +10214|AAAAAAAAGOHCAAAA|1997-10-27|2000-10-26|Other women like only in a dangers. Modern, strong acts might pay even so often poor personnel. Companies track still social activities.|9.05|6.42|3001001|amalgexporti #1|1|newborn|3|Children|256|callyantiable|large|177pink2312186968761|ivory|Ounce|Unknown|57|eseoughtablebarought| +10215|AAAAAAAAGOHCAAAA|2000-10-27||Efficient dangers should not levy pounds. Routes could not point yesterday |9.09|3.27|3001001|exportiamalg #2|3|maternity|1|Women|256|callyantiable|petite|177pink2312186968761|papaya|Oz|Unknown|7|antioughtablebarought| +10216|AAAAAAAAIOHCAAAA|1997-10-27|1999-10-27|Industrial ministers fill |5.65|2.99|1002001|importoamalg #1|2|fragrances|1|Women|402|ablebarese|medium|27085981483misty8655|saddle|Unknown|Unknown|34|callyoughtablebarought| +10217|AAAAAAAAIOHCAAAA|1999-10-28|2001-10-26|External, medieval |8.96|2.99|8014008|edu packmaxi #8|2|tennis|8|Sports|402|ablebarese|N/A|3424smoke07045567914|steel|Dozen|Unknown|90|ationoughtablebarought| +10218|AAAAAAAAIOHCAAAA|2001-10-27||Parental, direct areas would make available reasons. Substantial films produce around attempts. Certain, ancient components reduce prior proposals. Long|8.75|2.99|8014008|corpnameless #9|2|football|8|Sports|156|ablebarese|N/A|3424smoke07045567914|seashell|Pallet|Unknown|97|eingoughtablebarought| +10219|AAAAAAAALOHCAAAA|1997-10-27||Other, clinical senses display more. Suddenly video-taped friends take here local, african policies. Muscles think much local letters. Tired, parti|2.50|1.20|8014010|edu packmaxi #10|14|tennis|8|Sports|553|priantianti|N/A|1795584peach61884532|violet|Bundle|Unknown|26|n stoughtablebarought| +10220|AAAAAAAAMOHCAAAA|1997-10-27|2000-10-26|Available, fine elections would not like to a parts; just other poets go. More alternative firms imagine also local years. Too principal suggestions recei|0.91|0.71|6001005|amalgcorp #5|1|birdal|6|Jewelry|311|oughtoughtpri|N/A|73959706226346green2|lace|Gross|Unknown|28|barableablebarought| +10221|AAAAAAAAMOHCAAAA|2000-10-27||Available, fine elections would not like to a parts; just other poets go. More alternative firms imagine also local years. Too principal suggestions recei|8.90|3.56|9009010|maximaxi #10|1|science|9|Books|311|oughtoughtpri|N/A|yellow71395936909612|saddle|Ounce|Unknown|54|oughtableablebarought| +10222|AAAAAAAAOOHCAAAA|1997-10-27|1999-10-27|Current, considerable skills act earlier in a books. Perfect companies see else complaints. Following streets should b|3.33|2.56|8012003|importomaxi #3|12|guns|8|Sports|237|ationpriable|N/A|22738354chocolate255|honeydew|N/A|Unknown|52|ableableablebarought| +10223|AAAAAAAAOOHCAAAA|1999-10-28|2001-10-26|Current, considerable skills act earlier in a books. Perfect companies see else complaints. Following streets should b|2.16|1.92|4001002|amalgedu pack #2|12|womens|4|Shoes|237|ationpriable|extra large|22738354chocolate255|spring|Carton|Unknown|48|priableablebarought| +10224|AAAAAAAAOOHCAAAA|2001-10-27||Current, considerable skills act earlier in a books. Perfect companies see else complaints. Following streets should b|5.52|1.92|4001002|amalgimporto #1|1|accessories|2|Men|237|ationpriable|medium|22738354chocolate255|salmon|Cup|Unknown|21|eseableablebarought| +10225|AAAAAAAABPHCAAAA|1997-10-27||French, neighbouring words speak evenly big divisions. Completely urban employers may return efforts. Almost rich buildings keep. Rows wear. Royal, new meetings must lift still |8.31|3.98|2002002|importoimporto #2|2|shirts|2|Men|163|pricallyought|small|312plum4746940589392|orchid|Pallet|Unknown|100|antiableablebarought| +10226|AAAAAAAACPHCAAAA|1997-10-27|2000-10-26|Other values intervene rather; descriptions escape legal notions. Unlikely, other members must not produce both photog|9.22|2.95|5002001|importoscholar #1|2|country|5|Music|500|barbaranti|N/A|56white5469365865048|plum|Each|Unknown|14|callyableablebarought| +10227|AAAAAAAACPHCAAAA|2000-10-27||Rocks blame in a others; practical, past p|2.38|0.90|5002001|amalgimporto #2|2|accessories|2|Men|500|barbaranti|medium|56white5469365865048|metallic|Box|Unknown|17|ationableablebarought| +10228|AAAAAAAAEPHCAAAA|1997-10-27|1999-10-27|Equally adequate schools obtain for a commentators. Women would keep suddenly systems. Disastrous, old authorities enforc|0.23|0.12|8014009|edu packmaxi #9|14|tennis|8|Sports|642|ableesecally|N/A|15papaya565709209529|white|Bunch|Unknown|2|eingableablebarought| +10229|AAAAAAAAEPHCAAAA|1999-10-28|2001-10-26|Equally adequate schools obtain for a commentators. Women would keep suddenly systems. Disastrous, old authorities enforc|1.47|0.12|8014009|exportiedu pack #2|14|kids|4|Shoes|642|ableesecally|large|15papaya565709209529|white|Bundle|Unknown|76|n stableablebarought| +10230|AAAAAAAAEPHCAAAA|2001-10-27||Psychiatric, difficult activities need still; certain, cold claims may ensure plants. Arrangements calculate as always painful doctors. In part simple reports might not offer nev|0.77|0.36|2004001|edu packimporto #1|14|sports-apparel|2|Men|32|ableesecally|medium|415962seashell013933|pink|Unknown|Unknown|8|barpriablebarought| +10231|AAAAAAAAHPHCAAAA|1997-10-27||Scottish guidelines will concern. White, constant problems write buildings. Aware, physical estimates try here red, original units. Plants cannot go just |9.09|8.18|6003008|exporticorp #8|3|gold|6|Jewelry|164|esecallyought|N/A|6447162979652yellow1|frosted|Ounce|Unknown|22|oughtpriablebarought| +10232|AAAAAAAAIPHCAAAA|1997-10-27|2000-10-26|Suddenly other students shall mean home hopes. Rural years||3.97|||||||309|n stbarpri||||Pallet|||ablepriablebarought| +10233|AAAAAAAAIPHCAAAA|2000-10-27||Perhaps immediate settlements would protect specifically social rooms; again raw eyes lead less than patterns. Single, annual activities raise as impossible shares. Stiff res|7.01|2.80|4001002|amalgedu pack #2|9|womens|4|Shoes|294|n stbarpri|small|62737smoke8369761140|ghost|Ounce|Unknown|31|pripriablebarought| +10234|AAAAAAAAKPHCAAAA|1997-10-27|1999-10-27|Complaints produce easier at a rights. Cases build above elsewhere mad books. Particular, adult winners play over long women. Parties tackle ends. Much local concessions used to obtain as pr|98.79|38.52|5003001|exportischolar #1|3|pop|5|Music|214|eseoughtable|N/A|4650salmon7179395964|tomato|Tbl|Unknown|20|esepriablebarought| +10235|AAAAAAAAKPHCAAAA|1999-10-28|2001-10-26|Complaints produce easier at a rights. Cases build above elsewhere mad books. Particular, adult winners play over long women. Parties tackle ends. Much local concessions used to obtain as pr|0.45|0.16|5003001|edu packscholar #2|4|classical|5|Music|214|eseoughtable|N/A|4650salmon7179395964|steel|Case|Unknown|59|antipriablebarought| +10236|AAAAAAAAKPHCAAAA|2001-10-27||Open, possible agents appear continuo|98.53|0.16|5003001|exportischolar #1|3|pop|5|Music|214|eseoughtable|N/A|10746light8059323780|black|Pound|Unknown|22|callypriablebarought| +10237|AAAAAAAANPHCAAAA|1997-10-27||So other thousands resist. Performances win; important accounts could leave. Sorry years may not make. True dem|3.50|2.24|10013004|exportiamalgamalg #4|13|stereo|10|Electronics|304|esebarpri|N/A|2702snow034417820720|smoke|Tbl|Unknown|27|ationpriablebarought| +10238|AAAAAAAAOPHCAAAA|1997-10-27|2000-10-26|Sales hold soon at the men. Strong persons commemorat|0.52|0.40|2001001|amalgimporto #1|1|accessories|2|Men|508|eingbaranti|large|1433goldenrod0613802|smoke|Dram|Unknown|6|eingpriablebarought| +10239|AAAAAAAAOPHCAAAA|2000-10-27||Grounds may ask suddenly long costs. Economic, right lives say even quite continuous lines. Large, manufacturing demands can answer alone hug|0.60|0.40|4004002|edu packedu pack #2|4|athletic|4|Shoes|201|eingbaranti|extra large|5030267peach76112949|sky|Gross|Unknown|15|n stpriablebarought| +10240|AAAAAAAAAAICAAAA|1997-10-27|1999-10-27|Able, long mammals can want new, serious years. Questions would not cope again mainly unable contributions. Less responsible shelves lose records; leading, similar |1.69|0.65|1003001|exportiamalg #1|3|maternity|1|Women|72|ableation|small|6230935946065snow836|ghost|Gross|Unknown|29|bareseablebarought| +10241|AAAAAAAAAAICAAAA|1999-10-28|2001-10-26|Able, long mammals can want new, serious years. Questions would not cope again mainly unable contributions. Less responsible shelves lose records; leading, similar |2.49|1.49|4001002|amalgedu pack #2|1|womens|4|Shoes|645|antiesecally|extra large|6230935946065snow836|green|Each|Unknown|20|oughteseablebarought| +10242|AAAAAAAAAAICAAAA|2001-10-27||Soon asian banks can cross more nights. Obviously exclusive recordings ought to intervene yet problems. Principally true problems see home single, flat authori|8.10|1.49|1001001|amalgamalg #1|1|dresses|1|Women|999|antiesecally|medium|4ghost60951517074464|white|Each|Unknown|37|ableeseablebarought| +10243|AAAAAAAADAICAAAA|1997-10-27||Benefits could not mention authori|2.65|2.09|6001004|amalgcorp #4|1|birdal|6|Jewelry|52|ableanti|N/A|thistle2292959800552|sienna|Gross|Unknown|9|prieseablebarought| +10244|AAAAAAAAEAICAAAA|1997-10-27|2000-10-26|Still similar weeks should struggle fast practical |1.00|0.48|10006011|corpunivamalg #11|6|musical|10|Electronics|10|barought|N/A|18274slate4072400785|goldenrod|Each|Unknown|14|eseeseablebarought| +10245|AAAAAAAAEAICAAAA|2000-10-27||Still similar weeks should struggle fast practical |4.45|0.48|10006011|brandcorp #6|7|pendants|6|Jewelry|807|ationbareing|N/A|5101sandy54135811651|red|Box|Unknown|37|antieseablebarought| +10246|AAAAAAAAGAICAAAA|1997-10-27|1999-10-27|Comfortable drawings may know more inherent judges. Complex authorities would not tu|2.47|1.63|2003001|exportiimporto #1|3|pants|2|Men|70|baration|small|837salmon87579727858|seashell|Unknown|Unknown|37|callyeseablebarought| +10247|AAAAAAAAGAICAAAA|1999-10-28|2001-10-26|Designs would talk. There open plans mind again concrete, true solutions. Alone considerable figures kill extra, popular orders. Good, fixed republics may hear also. Similar, angry authorities spread |0.25|1.63|2003001|amalgedu pack #2|1|womens|4|Shoes|101|baration|large|9752firebrick6589872|seashell|Carton|Unknown|28|ationeseablebarought| +10248|AAAAAAAAGAICAAAA|2001-10-27||Bitterly british trials may specify together old-fashioned schools. Elected things cut. Too good photographs come all present, other meetings. Proceedings pro|1.70|1.63|2003001|brandnameless #9|1|hockey|8|Sports|101|oughtbarought|N/A|9slate52127013306544|steel|Dozen|Unknown|3|eingeseablebarought| +10249|AAAAAAAAJAICAAAA|1997-10-27||Too good effects telephone too crazy students. Specific, scottish elements might not tell nuclear variables. Following stations receive more responsible|8.80|4.66|9012002|importounivamalg #2|12|home repair|9|Books|53|prianti|N/A|2misty95651407958934|snow|Oz|Unknown|17|n steseablebarought| +10250|AAAAAAAAKAICAAAA|1997-10-27|2000-10-26|Divine, entire cuts must play by a hands. Relative days ca|2.68|2.14|7012001|importonameless #1|12|paint|7|Home|348|eingesepri|N/A|97mint34226576493241|pink|Gross|Unknown|55|barantiablebarought| +10251|AAAAAAAAKAICAAAA|2000-10-27||Europe|3.12|1.09|8008002|namelessnameless #2|12|outdoor|8|Sports|348|eingesepri|N/A|042638729613plum4773|snow|Unknown|Unknown|13|oughtantiablebarought| +10252|AAAAAAAAMAICAAAA|1997-10-27|1999-10-27|Just old operations might reach here views. Widely high others provide almost away raw items. Regions flourish. Old beds collaborate inc|4.43|3.80|2003001|exportiimporto #1|3|pants|2|Men|186|callyeingought|N/A|23565224779saddle254|powder|Cup|Unknown|88|ableantiablebarought| +10253|AAAAAAAAMAICAAAA|1999-10-28|2001-10-26|Civil reports prevent strangely traditional injuries. Powerful, soft preparations go much|4.53|3.80|3001002|amalgexporti #2|3|newborn|3|Children|464|callyeingought|extra large|50177779991447tan289|lace|Oz|Unknown|71|priantiablebarought| +10254|AAAAAAAAMAICAAAA|2001-10-27||Civil reports prevent strangely traditional injuries. Powerful, soft preparations go much|0.46|0.20|3001002|exportiexporti #1|3|toddlers|3|Children|464|callyeingought|N/A|4slate06409948049156|dodger|Lb|Unknown|95|eseantiablebarought| +10255|AAAAAAAAPAICAAAA|1997-10-27||Other ambitions seek aloud to a measurements; other hands could provide children; also particular pp. could push fine, huge mines. Just coun|4.72|4.10|9006008|corpmaxi #8|6|parenting|9|Books|72|ableation|N/A|39steel9715037691616|violet|Ounce|Unknown|98|antiantiablebarought| +10256|AAAAAAAAABICAAAA|1997-10-27|2000-10-26|Small, new tasks used to stop political, other years. Arr|6.36|4.57|5004001|edu packscholar #1|4|classical|5|Music|373|priationpri|N/A|833321puff8452841455|rose|N/A|Unknown|26|callyantiablebarought| +10257|AAAAAAAAABICAAAA|2000-10-27||Here british days find. As aware thousands should go of course men. Sisters would define effectively obviously long reports.|5.46|4.57|9006010|corpmaxi #10|6|parenting|9|Books|373|priationpri|N/A|877218915206457rose6|antique|Box|Unknown|54|ationantiablebarought| +10258|AAAAAAAACBICAAAA|1997-10-27|1999-10-27|Rare standards will not identify that local, necessary ideas. Safe, sma|3.34|2.57|6015003|scholarbrand #3|15|custom|6|Jewelry|166|callycallyought|N/A|red27649303121491525|steel|Cup|Unknown|22|eingantiablebarought| +10259|AAAAAAAACBICAAAA|1999-10-28|2001-10-26|Rare standards will not identify that local, necessary ideas. Safe, sma|7.62|4.03|6015003|amalgimporto #2|1|accessories|2|Men|166|callycallyought|large|274047ghost546000319|saddle|N/A|Unknown|4|n stantiablebarought| +10260|AAAAAAAACBICAAAA|2001-10-27||Rare standards will not identify that local, necessary ideas. Safe, sma|4.53|4.03|6015003|brandunivamalg #10|1|personal|10|Electronics|166|callycallyought|N/A|274047ghost546000319|thistle|Lb|Unknown|16|barcallyablebarought| +10261|AAAAAAAAFBICAAAA|1997-10-27||Civil, poor cases shall realise about suitable, parental situations. Even left pounds continue however in|3.76|2.97|6014006|edu packbrand #6|14|estate|6|Jewelry|638|eingprically|N/A|39891420391powder945|pink|Case|Unknown|61|oughtcallyablebarought| +10262|AAAAAAAAGBICAAAA|1997-10-27|2000-10-26|Possible services tell manufacturing properties. Japanese, internal policies used to draw. Farms might com|4.14|1.53|4003001|exportiedu pack #1|3|kids|4|Shoes|1|ought|small|688988036steel457874|royal|Dozen|Unknown|22|ablecallyablebarought| +10263|AAAAAAAAGBICAAAA|2000-10-27||Eventually small cats find with a goods. Today italian|1.47|1.53|4003001|exportiunivamalg #4|13|self-help|9|Books|1|ought|N/A|688988036steel457874|sandy|Box|Unknown|9|pricallyablebarought| +10264|AAAAAAAAIBICAAAA|1997-10-27|1999-10-27|As basic fingers can survive exact, star years. Dead, far metres should agree even above extended questions. Cultural assumptions used to |78.41|51.75|1003001|exportiamalg #1|3|maternity|1|Women|2|able|economy|13674470sienna157719|salmon|Tbl|Unknown|28|esecallyablebarought| +10265|AAAAAAAAIBICAAAA|1999-10-28|2001-10-26|As basic fingers can survive exact, star years. Dead, far metres should agree even above extended questions. Cultural assumptions used to |8.18|6.95|3002002|importoexporti #2|3|infants|3|Children|588|able|petite|13674470sienna157719|pale|Cup|Unknown|85|anticallyablebarought| +10266|AAAAAAAAIBICAAAA|2001-10-27||Most full factors describe doubts. Simple, financial cases become also from a members. Unemployed children will|1.51|6.95|7011007|amalgnameless #7|11|accent|7|Home|463|able|N/A|3pale843306983698693|rose|Bunch|Unknown|48|callycallyablebarought| +10267|AAAAAAAALBICAAAA|1997-10-27||Enquiries will n|7.33|5.20|10005017|scholarunivamalg #17|5|karoke|10|Electronics|170|barationought|N/A|34076453puff20482041|papaya|Dozen|Unknown|23|ationcallyablebarought| +10268|AAAAAAAAMBICAAAA|1997-10-27|2000-10-26|Important, standard owners require any longer. Earlier royal jobs help still presu|0.50|0.39|3002001|importoexporti #1|2|infants|3|Children|594|esen stanti|medium|879plum2952376014147|saddle|Ounce|Unknown|28|eingcallyablebarought| +10269|AAAAAAAAMBICAAAA|2000-10-27||Alone, common residents would not fire. Urban, obvious police shall walk nowadays. Mediterranean, red scenes detect quickly de|1.90|0.39|4002002|importoedu pack #2|2|mens|4|Shoes|646|esen stanti|medium|879plum2952376014147|spring|Carton|Unknown|44|n stcallyablebarought| +10270|AAAAAAAAOBICAAAA|1997-10-27|1999-10-27|Rewards|4.26|3.74|2004001|edu packimporto #1|4|sports-apparel|2|Men|200|barbarable|medium|0384320951burnished3|plum|Dozen|Unknown|21|barationablebarought| +10271|AAAAAAAAOBICAAAA|1999-10-28|2001-10-26|New, inadequate a|1.43|3.74|1001002|amalgamalg #2|1|dresses|1|Women|200|barbarable|medium|741821peru9141990270|tomato|Lb|Unknown|31|oughtationablebarought| +10272|AAAAAAAAOBICAAAA|2001-10-27||New, inadequate a|4.61|3.73|9010001|univunivamalg #1|10|travel|9|Books|200|barbarable|N/A|741821peru9141990270|saddle|Ton|Unknown|71|ableationablebarought| +10273|AAAAAAAABCICAAAA|1997-10-27||Below specific feelings take close cases. British systems might get again different guests; forces remember socialist, visual minutes; continued characters need alive copies; fresh, broke|4.41|2.69|7011004|amalgnameless #4|11|accent|7|Home|46|callyese|N/A|127337745151365plum2|moccasin|Gram|Unknown|22|priationablebarought| +10274|AAAAAAAACCICAAAA|1997-10-27|2000-10-26|Re|9.41|8.18|10016007|corpamalgamalg #7|16|wireless|10|Electronics|48|eingese|N/A|24plum86449724204290|grey|N/A|Unknown|23|eseationablebarought| +10275|AAAAAAAACCICAAAA|2000-10-27||Just economic lines turn apart familiar others. Clear, t|1.90|1.40|4004002|edu packedu pack #2|4|athletic|4|Shoes|665|eingese|medium|757518powder82482822|smoke|Gram|Unknown|100|antiationablebarought| +10276|AAAAAAAAECICAAAA|1997-10-27|1999-10-27|Old, n|1.37|0.76|8012007|importomaxi #7|12|guns|8|Sports|341|oughtesepri|N/A|37610150376104lemon6|yellow|Tsp|Unknown|64|callyationablebarought| +10277|AAAAAAAAECICAAAA|1999-10-28|2001-10-26|Old, n|3.88|0.76|8012007|scholarnameless #8|12|fitness|8|Sports|373|oughtesepri|N/A|2saddle8448164121134|metallic|Cup|Unknown|5|ationationablebarought| +10278|AAAAAAAAECICAAAA|2001-10-27||Commitments would interact there women. Books extend frequently in a others. Unique, late developments try then new terms. Designers used to go o|8.65|0.76|8012007|amalgmaxi #9|12|archery|8|Sports|1|oughtesepri|N/A|7104turquoise2465540|sienna|Tbl|Unknown|92|eingationablebarought| +10279|AAAAAAAAHCICAAAA|1997-10-27||National numbers may increase aware followe|3.63|2.54|10012012|importoamalgamalg #12|12|monitors|10|Electronics|305|antibarpri|N/A|63419909697spring235|snow|Tbl|Unknown|28|n stationablebarought| +10280|AAAAAAAAICICAAAA|1997-10-27|2000-10-26|British teachers return really possible, dominant hours; seriously vital differences play for a items. Just outer shoes modify at most different customers. In order particular horses|66.97|34.82|2001001|amalgimporto #1|1|accessories|2|Men|563|pricallyanti|medium|3255704019474rosy807|sienna|Bunch|Unknown|41|bareingablebarought| +10281|AAAAAAAAICICAAAA|2000-10-27||Successive parties hold clear fresh increases. Also royal buildings hold about foreign directors. Local exchanges keep economic, legislative mothers. Outside police would bring into a|6.03|1.80|6009002|maxicorp #2|9|womens watch|6|Jewelry|563|pricallyanti|N/A|3255704019474rosy807|honeydew|Lb|Unknown|45|oughteingablebarought| +10282|AAAAAAAAKCICAAAA|1997-10-27|1999-10-27|Final, political terms make hardly. Good years say so. Sources can drift current, nat|2.63|2.26|4002001|importoedu pack #1|2|mens|4|Shoes|301|oughtbarpri|large|36white1920784762933|cream|Gram|Unknown|90|ableeingablebarought| +10283|AAAAAAAAKCICAAAA|1999-10-28|2001-10-26|Final, political terms make hardly. Good years say so. Sources can drift current, nat|7.28|6.33|6006004|corpcorp #4|6|rings|6|Jewelry|301|oughtbarpri|N/A|36white1920784762933|puff|Tsp|Unknown|11|prieingablebarought| +10284|AAAAAAAAKCICAAAA|2001-10-27||Final, political terms make hardly. Good years say so. Sources can drift current, nat|2.22|6.33|6006004|edu packunivamalg #17|4|audio|10|Electronics|360|oughtbarpri|N/A|36white1920784762933|lemon|Tsp|Unknown|60|eseeingablebarought| +10285|AAAAAAAANCICAAAA|1997-10-27||Different years ought to use in a eyes. Taxes can recognise pro|4.38|2.97|6008006|namelesscorp #6|8|mens watch|6|Jewelry|540|bareseanti|N/A|17883340steel0121930|floral|Box|Unknown|31|antieingablebarought| +10286|AAAAAAAAOCICAAAA|1997-10-27|2000-10-26|Otherwise political systems know surely unable|4.94|2.56|8007007|brandnameless #7|7|hockey|8|Sports|206|callybarable|N/A|411575white911137090|red|Tbl|Unknown|40|callyeingablebarought| +10287|AAAAAAAAOCICAAAA|2000-10-27||Poor, established members could reduce at a months. Mere, international teams ensure straight to a standards. Short pupils deny still; e|6.85|2.56|8007007|amalgmaxi #8|11|archery|8|Sports|290|callybarable|N/A|428638orchid97027063|antique|Bunch|Unknown|24|ationeingablebarought| +10288|AAAAAAAAADICAAAA|1997-10-27|1999-10-27|True members take successfully complete months. As new offices impose still urgent, general things. Islamic, new schools might play mo|1.56|1.13|2002001|importoimporto #1|2|shirts|2|Men|160|barcallyought|extra large|0567826almond8164698|smoke|Tsp|Unknown|20|eingeingablebarought| +10289|AAAAAAAAADICAAAA|1999-10-28|2001-10-26|True members take successfully complete months. As new offices impose still urgent, general things. Islamic, new schools might play mo|6.13|1.13|2002001|exportiimporto #2|3|pants|2|Men|160|barcallyought|medium|0567826almond8164698|steel|N/A|Unknown|46|n steingablebarought| +10290|AAAAAAAAADICAAAA|2001-10-27||True members take successfully complete months. As new offices impose still urgent, general things. Islamic, new schools might play mo|3.73|1.13|5004001|edu packscholar #1|3|classical|5|Music|236|callypriable|N/A|89735989797purple554|peach|Bundle|Unknown|10|barn stablebarought| +10291|AAAAAAAADDICAAAA|1997-10-27||International, young seats see really worker|85.28|70.78|5001002|amalgscholar #2|1|rock|5|Music|768|eingcallyation|N/A|3purple3838669047681|steel|Case|Unknown|18|oughtn stablebarought| +10292|AAAAAAAAEDICAAAA|1997-10-27|2000-10-26|Full foreigners contain currently from a months. New, statutory feelings will not tr|3.37|1.95|4002001|importoedu pack #1|2|mens|4|Shoes|32|ablepri|extra large|smoke558200941836846|steel|Case|Unknown|36|ablen stablebarought| +10293|AAAAAAAAEDICAAAA|2000-10-27||Full foreigners contain currently from a months. New, statutory feelings will not tr|4.68|1.95|4002001|namelessmaxi #10|2|romance|9|Books|891|ablepri|N/A|smoke558200941836846|smoke|Unknown|Unknown|28|prin stablebarought| +10294|AAAAAAAAGDICAAAA|1997-10-27|1999-10-27|Major agents used to start tria|2.56|0.94|4001001|amalgedu pack #1|1|womens|4|Shoes|172|ableationought|large|25637107882khaki7156|royal|Dram|Unknown|29|esen stablebarought| +10295|AAAAAAAAGDICAAAA|1999-10-28|2001-10-26|Standard, different features control. Only other lectures would find sound sales. New, various wars may not visit at least correct, prime st|3.91|1.99|4001001|brandunivamalg #11|1|personal|10|Electronics|936|callyprin st|N/A|4214steel53055137215|magenta|Tbl|Unknown|12|antin stablebarought| +10296|AAAAAAAAGDICAAAA|2001-10-27||Standard, different features control. Only other lectures would find sound sales. New, various wars may not visit at least correct, prime st|1.68|0.75|6009001|maxicorp #1|1|womens watch|6|Jewelry|612|callyprin st|N/A|9963530498papaya9598|navajo|Case|Unknown|16|callyn stablebarought| +10297|AAAAAAAAJDICAAAA|1997-10-27||Acres move. Differences use tomorrow northern relations. Other tears may manipulate still. Highly secondary occasions take. Sharp folk|6.58|5.32|2002002|importoimporto #2|2|shirts|2|Men|255|antiantiable|small|612green999055631049|linen|Tsp|Unknown|13|ationn stablebarought| +10298|AAAAAAAAKDICAAAA|1997-10-27|2000-10-26|Extremely responsible individuals sh|9.32|7.08|5001001|amalgscholar #1|1|rock|5|Music|202|ablebarable|N/A|152mint5825153276395|seashell|Pallet|Unknown|19|eingn stablebarought| +10299|AAAAAAAAKDICAAAA|2000-10-27||Extremely responsible individuals sh|4.64|7.08|7013010|exportinameless #10|13|wallpaper|7|Home|202|ablebarable|N/A|350578348thistle7828|white|Ton|Unknown|29|n stn stablebarought| +10300|AAAAAAAAMDICAAAA|1997-10-27|1999-10-27|Clear, professional issues shall not hide tiny, economic units. Almost new patterns seem in the ways. Ideas ask parents. Red, sure nations shall not want days. Planned, good |7.20|3.31|1004001|edu packamalg #1|4|swimwear|1|Women|65|antically|medium|495529989floral35384|honeydew|Dozen|Unknown|59|barbarpribarought| +10301|AAAAAAAAMDICAAAA|1999-10-28|2001-10-26|Traditional problems ought to want only new charges. Additional streets make finally agents. Full, irish factories could say |3.28|1.57|6014006|edu packbrand #6|14|estate|6|Jewelry|322|ableablepri|N/A|47hot764488627686712|metallic|Dozen|Unknown|25|oughtbarpribarought| +10302|AAAAAAAAMDICAAAA|2001-10-27||Changes discuss well. Students implement also british, local things. Continuing feet be|65.66|55.81|6014006|exportiamalg #1|3|maternity|1|Women|322|ableablepri|petite|47hot764488627686712|saddle|Dram|Unknown|57|ablebarpribarought| +10303|AAAAAAAAPDICAAAA|1997-10-27||Industrial prisoners overcome english ends. There numerous rights pack even important components. Bright hands should bargain precisely. Divisions produce distinctly other, ca|4.59|3.12|2004002|edu packimporto #2|4|sports-apparel|2|Men|129|n stableought|medium|7711896528908rose054|puff|Gram|Unknown|9|pribarpribarought| +10304|AAAAAAAAAEICAAAA|1997-10-27|2000-10-26|Sorry options might not draw very employees. Now positive poles see police. St|0.51|0.34|5002001|importoscholar #1|2|country|5|Music|167|ationcallyought|N/A|488sienna64300713204|tomato|Unknown|Unknown|32|esebarpribarought| +10305|AAAAAAAAAEICAAAA|2000-10-27||Sorry options might not draw very employees. Now positive poles see police. St|6.44|2.51|2002002|importoimporto #2|2|shirts|2|Men|92|ationcallyought|small|6236951365314slate93|maroon|Bunch|Unknown|3|antibarpribarought| +10306|AAAAAAAACEICAAAA|1997-10-27|1999-10-27|Experiments may find there political groups. Groups take on a structures. Ministers stop gentl|1.49|0.46|8001007|amalgnameless #7|1|athletic shoes|8|Sports|154|eseantiought|N/A|98421024gainsboro410|tan|Carton|Unknown|7|callybarpribarought| +10307|AAAAAAAACEICAAAA|1999-10-28|2001-10-26|Experiments may find there political groups. Groups take on a structures. Ministers stop gentl|0.25|0.46|4001002|amalgedu pack #2|1|womens|4|Shoes|154|eseantiought|medium|98421024gainsboro410|salmon|Dram|Unknown|60|ationbarpribarought| +10308|AAAAAAAACEICAAAA|2001-10-27||Somewhere minor miles a|3.65|2.48|6012005|importobrand #5|12|costume|6|Jewelry|154|eseantiought|N/A|190668turquoise08949|tan|Oz|Unknown|27|eingbarpribarought| +10309|AAAAAAAAFEICAAAA|1997-10-27||Old, small variations divide on a methods. Alone, american calls appear now. Great wo|4.34|2.73|6016006|corpbrand #6|16|consignment|6|Jewelry|292|ablen stable|N/A|89993256powder409595|drab|Each|Unknown|62|n stbarpribarought| +10310|AAAAAAAAGEICAAAA|1997-10-27|2000-10-26|Proceedings might not put employees; industrial, poor areas supplement subsequent copies. Low sales grow cases. A|8.79|5.36|2001001|amalgimporto #1|1|accessories|2|Men|201|oughtbarable|petite|6201533899688puff598|plum|Oz|Unknown|12|baroughtpribarought| +10311|AAAAAAAAGEICAAAA|2000-10-27||Public, western prices might intervene mainly thick records. Bizarre, necessary policies answer usually. Tiny, ordinary men shall stay better with a eggs. Commonly only intervals may t|4.86|5.36|10003010|exportiunivamalg #10|3|dvd/vcr players|10|Electronics|50|baranti|N/A|95470149558703blush8|honeydew|Tbl|Unknown|41|oughtoughtpribarought| +10312|AAAAAAAAIEICAAAA|1997-10-27|1999-10-27|All numerous reasons explain upper teachers; necessary, inte|59.59|38.13|1003001|exportiamalg #1|3|maternity|1|Women|231|oughtpriable|N/A|8984sandy48793814057|sienna|Cup|Unknown|5|ableoughtpribarought| +10313|AAAAAAAAIEICAAAA|1999-10-28|2001-10-26|All numerous reasons explain upper teachers; necessary, inte|9.37|5.99|9006006|corpmaxi #6|3|parenting|9|Books|137|ationpriought|N/A|8984sandy48793814057|pale|Cup|Unknown|91|prioughtpribarought| +10314|AAAAAAAAIEICAAAA|2001-10-27||All numerous reasons explain upper teachers; necessary, inte|6.67|5.46|10007013|brandunivamalg #13|3|personal|10|Electronics|53|ationpriought|N/A|8984sandy48793814057|dodger|Box|Unknown|51|eseoughtpribarought| +10315|AAAAAAAALEICAAAA|1997-10-27||Internal services used to oppose consistently talks. Green documents would feed as the wives. Administrative songs help still main tiles. Wives warm quite safe |7.14|2.35|8009006|maxinameless #6|9|optics|8|Sports|177|ationationought|N/A|3912577maroon1012666|moccasin|Pound|Unknown|25|antioughtpribarought| +10316|AAAAAAAAMEICAAAA|1997-10-27|2000-10-26|Detailed, prime parts should go recently commercial, grand forces. Clear notes think both old, per|2.85|1.56|3004001|edu packexporti #1|4|school-uniforms|3|Children|280|bareingable|medium|77burlywood953806460|violet|Carton|Unknown|14|callyoughtpribarought| +10317|AAAAAAAAMEICAAAA|2000-10-27||Detailed, prime parts should go recently commercial, grand forces. Clear notes think both old, per|4.55|1.56|4004002|edu packedu pack #2|4|athletic|4|Shoes|39|n stpri|petite|77burlywood953806460|coral|Dram|Unknown|11|ationoughtpribarought| +10318|AAAAAAAAOEICAAAA|1997-10-27|1999-10-27|Very famous children shall miss. Subsequent, quiet questions could not go also. Upper crim|1.31|1.10|10013017|exportiamalgamalg #17|13|stereo|10|Electronics|414|eseoughtese|N/A|79585thistle10662991|yellow|Gram|Unknown|14|eingoughtpribarought| +10319|AAAAAAAAOEICAAAA|1999-10-28|2001-10-26|Even warm opportunities go; early, free eyes can encourage existing effects; in additi|2.32|1.10|10013017|maxiunivamalg #1|13|televisions|10|Electronics|414|eseoughtese|N/A|947432163957705peru5|lace|Bundle|Unknown|13|n stoughtpribarought| +10320|AAAAAAAAOEICAAAA|2001-10-27||Managers get things. Cheap pieces lead. Sincerely social shareholders meet earlier perfectly real conventions. Metr|2.27|1.10|10013017|edu packedu pack #1|4|athletic|4|Shoes|143|prieseought|extra large|947432163957705peru5|powder|Pound|Unknown|14|barablepribarought| +10321|AAAAAAAABFICAAAA|1997-10-27||Increased, severe times make again at a parts. As full operators focus even. Rights correlate improved women. Typical meals scramble relatively therefore other|0.26|0.09|5003002|exportischolar #2|3|pop|5|Music|51|oughtanti|N/A|91antique29263136538|moccasin|Gross|Unknown|28|oughtablepribarought| +10322|AAAAAAAACFICAAAA|1997-10-27|2000-10-26|Regularly british officers associate. Messages know special forces. Children replace |0.96|0.86|2002001|importoimporto #1|2|shirts|2|Men|267|ationcallyable|petite|3ghost18173618621564|saddle|Tbl|Unknown|41|ableablepribarought| +10323|AAAAAAAACFICAAAA|2000-10-27||Books will not interview. Again gay courses should estimate together to a others. White, likely weeks last. Weapons ought to cause around personally social clubs. Slightly northern feet determine|0.14|0.86|2002001|amalgmaxi #4|11|archery|8|Sports|267|ationcallyable|N/A|4024353626419peach71|aquamarine|Tbl|Unknown|10|priablepribarought| +10324|AAAAAAAAEFICAAAA|1997-10-27|1999-10-27|Enough social topics should make now sets. Modern statements yield always|9.19|4.22|5004001|edu packscholar #1|4|classical|5|Music|277|ationationable|N/A|snow8057225941570181|seashell|Oz|Unknown|17|eseablepribarought| +10325|AAAAAAAAEFICAAAA|1999-10-28|2001-10-26|As academic men ought to mind here other previous problems; years used to show brit|9.17|2.93|1001002|amalgamalg #2|1|dresses|1|Women|277|ationationable|medium|snow8057225941570181|orchid|Ounce|Unknown|63|antiablepribarought| +10326|AAAAAAAAEFICAAAA|2001-10-27|||||8016007||16|golf||Sports|277|ationationable||9200791854395lemon30|||||| +10327|AAAAAAAAHFICAAAA|1997-10-27||Expensive, original hours worry important, other words. Really different members get terms. Changes buy particu|3.26|1.46|3002002|importoexporti #2|2|infants|3|Children|580|bareinganti|extra large|2665844406medium9291|powder|Case|Unknown|31|ationablepribarought| +10328|AAAAAAAAIFICAAAA|1997-10-27|2000-10-26|Women will demonstrate at last|8.01|6.80|4003001|exportiedu pack #1|3|kids|4|Shoes|8|eing|small|46salmon774557047012|grey|Ton|Unknown|61|eingablepribarought| +10329|AAAAAAAAIFICAAAA|2000-10-27||Full users know merely inner explanations. Liable developments breathe women; of course administrative colleagues worry both worried husbands. Literally bad bodies|1.88|6.80|4003001|exportiexporti #2|3|toddlers|3|Children|8|eing|medium|46salmon774557047012|lawn|Unknown|Unknown|1|n stablepribarought| +10330|AAAAAAAAKFICAAAA|1997-10-27|1999-10-27|Simple, fine parties give late responsible bags. There interesting standards like doctors. Much other countries need relations. Early, other pieces mini|0.66|0.38|5001001|amalgscholar #1|1|rock|5|Music|126|callyableought|N/A|9859medium1773681839|olive|Each|Unknown|2|barpripribarought| +10331|AAAAAAAAKFICAAAA|1999-10-28|2001-10-26|Simple, fine parties give late responsible bags. There interesting standards like doctors. Much other countries need relations. Early, other pieces mini|7.20|0.38|9005012|scholarmaxi #12|1|history|9|Books|126|callyableought|N/A|9859medium1773681839|purple|Case|Unknown|69|oughtpripribarought| +10332|AAAAAAAAKFICAAAA|2001-10-27||Simple, fine parties give late responsible bags. There interesting standards like doctors. Much other countries need relations. Early, other pieces mini|6.13|0.38|9016001|corpunivamalg #1|1|mystery|9|Books|126|callyableought|N/A|9859medium1773681839|thistle|Lb|Unknown|26|ablepripribarought| +10333|AAAAAAAANFICAAAA|1997-10-27||Yet only months can repeat reader|1.82|0.76|8007004|brandnameless #4|7|hockey|8|Sports|233|pripriable|N/A|5682251almond2427734|wheat|Gram|Unknown|23|pripripribarought| +10334|AAAAAAAAOFICAAAA|1997-10-27|2000-10-26|Other, relevant numbers put now economic customers. New drawings need|5.22|4.01|6016007|corpbrand #7|16|consignment|6|Jewelry|526|callyableanti|N/A|9870steel50125428258|ghost|Gross|Unknown|14|esepripribarought| +10335|AAAAAAAAOFICAAAA|2000-10-27||Other, relevant numbers put now economic customers. New drawings need|0.99|0.88|6016007|amalgnameless #6|16|accent|7|Home|526|callyableanti|N/A|9870steel50125428258|sky|Dram|Unknown|64|antipripribarought| +10336|AAAAAAAAAGICAAAA|1997-10-27|1999-10-27|Basic, rea|0.27|0.18|1004001|edu packamalg #1|4|swimwear|1|Women|118|eingoughtought|medium|5691344383736sky5001|royal|Ton|Unknown|12|callypripribarought| +10337|AAAAAAAAAGICAAAA|1999-10-28|2001-10-26|Basic, rea|1.58|0.97|9009006|maximaxi #6|9|science|9|Books|39|n stpri|N/A|5691344383736sky5001|wheat|Pallet|Unknown|10|ationpripribarought| +10338|AAAAAAAAAGICAAAA|2001-10-27||Rates picture optimistic, personal eyes. Earlier modest arms may interrupt maybe informal good words. European families present now. Modern units let rarel|6.95|0.97|6007003|brandcorp #3|7|pendants|6|Jewelry|39|n stpri|N/A|272081pink6439472847|grey|Box|Unknown|73|eingpripribarought| +10339|AAAAAAAADGICAAAA|1997-10-27||Twin years can co-ordinate ever magistrates; useless, natura|8.27|6.03|1004002|edu packamalg #2|4|swimwear|1|Women|12|ableought|small|5425plum999272078576|thistle|Bundle|Unknown|67|n stpripribarought| +10340|AAAAAAAAEGICAAAA|1997-10-27|2000-10-26|Functions affect in order. Above inc stages risk. Usually suitable members ought to hear personally immediately total weapons. However western letters mi|2.82|1.94|5004001|edu packscholar #1|4|classical|5|Music|84|eseeing|N/A|791654sandy700379265|peach|Unknown|Unknown|66|baresepribarought| +10341|AAAAAAAAEGICAAAA|2000-10-27||Countries fill in a hills. Professional victims keep appointed, fine systems. Local, useful e|9.39|1.94|5004001|amalgamalg #2|4|dresses|1|Women|146|eseeing|large|516290smoke791428955|royal|Bunch|Unknown|100|oughtesepribarought| +10342|AAAAAAAAGGICAAAA|1997-10-27|1999-10-27|Nowhere new points will not go places. Capable, local courses explore both barely distinctive numbers. Seriously recent areas begin rare, reas|2.21|0.86|9009011|maximaxi #11|9|science|9|Books|146|callyeseought|N/A|499powder84975530982|dark|Gross|Unknown|23|ableesepribarought| +10343|AAAAAAAAGGICAAAA|1999-10-28|2001-10-26|Nowhere new points will not go places. Capable, local courses explore both barely distinctive numbers. Seriously recent areas begin rare, reas|4.48|0.86|9009011|edu packexporti #2|4|school-uniforms|3|Children|480|bareingese|medium|499powder84975530982|purple|Case|Unknown|82|priesepribarought| +10344|AAAAAAAAGGICAAAA|2001-10-27||Nowhere new points will not go places. Capable, local courses explore both barely distinctive numbers. Seriously recent areas begin rare, reas|1.95|0.86|9009011|exportiedu pack #1|3|kids|4|Shoes|480|bareingese|petite|499powder84975530982|pale|Tsp|Unknown|56|eseesepribarought| +10345|AAAAAAAAJGICAAAA|1997-10-27||Homes would open with a authors. Then high times should find away that single cultures. Quite pos|4.59|2.98|4003002|exportiedu pack #2|3|kids|4|Shoes|603|pribarcally|medium|02143962638387khaki4|rosy|Gram|Unknown|20|antiesepribarought| +10346|AAAAAAAAKGICAAAA|1997-10-27|2000-10-26|Officers insert reasonable parents. Perhaps unlike situations might control then. Indeed western routes seize new citizens; then electronic years shall|0.81|0.25|4002001|importoedu pack #1|2|mens|4|Shoes|161|oughtcallyought|large|50685721azure3711857|snow|Each|Unknown|75|callyesepribarought| +10347|AAAAAAAAKGICAAAA|2000-10-27||Around aware respects prevent only demands. Compulsory, other degrees get now |3.84|0.25|2002002|importoimporto #2|2|shirts|2|Men|161|oughtcallyought|medium|50685721azure3711857|ivory|Box|Unknown|28|ationesepribarought| +10348|AAAAAAAAMGICAAAA|1997-10-27|1999-10-27|Arms get at most more alone troops. Singl|6.16|5.48|9005005|scholarmaxi #5|5|history|9|Books|109|n stbarought|N/A|0thistle081061898909|yellow|Ton|Unknown|17|eingesepribarought| +10349|AAAAAAAAMGICAAAA|1999-10-28|2001-10-26|Arms get at most more alone troops. Singl|7.89|5.48|8010010|univmaxi #10|10|pools|8|Sports|109|n stbarought|N/A|882448383medium81484|indian|Tbl|Unknown|20|n stesepribarought| +10350|AAAAAAAAMGICAAAA|2001-10-27||Arms get at most more alone troops. Singl|4.75|1.75|3001001|amalgexporti #1|10|newborn|3|Children|396|n stbarought|extra large|047597white581723339|magenta|Gross|Unknown|32|barantipribarought| +10351|AAAAAAAAPGICAAAA|1997-10-27||Capable, apparent lines look eng|2.25|1.05|1002002|importoamalg #2|2|fragrances|1|Women|504|esebaranti|petite|692786seashell727863|honeydew|Bundle|Unknown|8|oughtantipribarought| +10352|AAAAAAAAAHICAAAA|1997-10-27|2000-10-26|Particular, concerned odds should see conditions. Limited, existing|7.71|3.23|9015009|scholarunivamalg #9|15|fiction|9|Books|183|prieingought|N/A|1871007320plum692464|honeydew|Dram|Unknown|18|ableantipribarought| +10353|AAAAAAAAAHICAAAA|2000-10-27||Able, previous boys tell much to a lessons. Financial women would write generally days. Lives aim by a obligations; fair, independent homes would try days. National times take either as a business|6.54|3.23|9015009|importoamalgamalg #1|12|monitors|10|Electronics|183|prieingought|N/A|293spring68960136797|drab|Ounce|Unknown|20|priantipribarought| +10354|AAAAAAAACHICAAAA|1997-10-27|1999-10-27|Tables detain probably labour systems; european, particular cells cannot escape much level persons. Other, m|2.09|1.63|2001001|amalgimporto #1|1|accessories|2|Men|233|pripriable|N/A|16051royal6097949687|turquoise|Tsp|Unknown|7|eseantipribarought| +10355|AAAAAAAACHICAAAA|1999-10-28|2001-10-26|Tables detain probably labour systems; european, particular cells cannot escape much level persons. Other, m|86.10|43.05|2001001|importoimporto #2|2|shirts|2|Men|386|pripriable|medium|258661842powder15218|powder|Gram|Unknown|18|antiantipribarought| +10356|AAAAAAAACHICAAAA|2001-10-27||Tables detain probably labour systems; european, particular cells cannot escape much level persons. Other, m|4.79|43.05|3003001|exportiexporti #1|2|toddlers|3|Children|79|n station|extra large|136066388pale3391527|metallic|Tsp|Unknown|53|callyantipribarought| +10357|AAAAAAAAFHICAAAA|1997-10-27||Assets regard even through the journals. Subjects might mean still. Critics might get big moments. Ages lead new statements. Legal,|7.65|3.21|1003002|exportiamalg #2|3|maternity|1|Women|155|antiantiought|medium|10926671khaki2068715|rose|Each|Unknown|19|ationantipribarought| +10358|AAAAAAAAGHICAAAA|1997-10-27|2000-10-26|Directly general daughters change years. Particularly immediate matters touch so in terms of a women; above real tears get later; yards le|4.38|3.19|5002001|importoscholar #1|2|country|5|Music|299|n stn stable|N/A|41blue63702597524367|sky|Ounce|Unknown|4|eingantipribarought| +10359|AAAAAAAAGHICAAAA|2000-10-27||Fat arguments ought to consider orders. British sources put too historical parents; areas cause other, average laws. Knowingly psychological firms will know|9.62|3.19|4002002|importoedu pack #2|2|mens|4|Shoes|299|n stn stable|small|41blue63702597524367|rosy|Case|Unknown|24|n stantipribarought| +10360|AAAAAAAAIHICAAAA|1997-10-27|1999-10-27|Public, financial events go notably massive, additional estates. New units come twice pretty visual courses. Managers declare highly scots; |2.36|1.43|1001001|amalgamalg #1|1|dresses|1|Women|324|eseablepri|small|100752021532sienna25|puff|Pallet|Unknown|28|barcallypribarought| +10361|AAAAAAAAIHICAAAA|1999-10-28|2001-10-26|Financial voices go; however solid cars will not help only effective, legal troops. Permanent colleag|3.31|1.43|1004002|edu packamalg #2|4|swimwear|1|Women|324|eseablepri|medium|100752021532sienna25|chartreuse|Gross|Unknown|7|oughtcallypribarought| +10362|AAAAAAAAIHICAAAA|2001-10-27||Labour, old things feed easily smooth stag|6.91|2.62|1004002|brandcorp #3|4|pendants|6|Jewelry|42|ableese|N/A|194459339488614sky58|ivory|Lb|Unknown|1|ablecallypribarought| +10363|AAAAAAAALHICAAAA|1997-10-27||Hands will judge in the shots. Extra, other services will clarify; possible chapters defend rapidly too civil s|2.63|1.63|7012004|importonameless #4|12|paint|7|Home|205|antibarable|N/A|675090007salmon83576|purple|Ounce|Unknown|76|pricallypribarought| +10364|AAAAAAAAMHICAAAA|1997-10-27|2000-10-26|To and fro silent consultants used to help on|0.70|0.23|2001001|amalgimporto #1|1|accessories|2|Men|212|ableoughtable|medium|64031190635puff03206|plum|Pound|Unknown|56|esecallypribarought| +10365|AAAAAAAAMHICAAAA|2000-10-27||Reliable students should create countries. Figures make costs. Constant paintings w|6.91|3.38|2001001|amalgbrand #6|1|bathroom|7|Home|633|ableoughtable|N/A|8103purple1233966459|pale|Unknown|Unknown|48|anticallypribarought| +10366|AAAAAAAAOHICAAAA|1997-10-27|1999-10-27|Mere bishops should happen surely gastric, natural standards. |6.47|1.94|5002001|importoscholar #1|2|country|5|Music|304|esebarpri|N/A|slate351905572067861|white|Dozen|Unknown|41|callycallypribarought| +10367|AAAAAAAAOHICAAAA|1999-10-28|2001-10-26|Mere bishops should happen surely gastric, natural standards. |59.98|17.99|5002001|exportiamalg #2|2|maternity|1|Women|304|esebarpri|petite|92linen6749716821006|slate|Pallet|Unknown|18|ationcallypribarought| +10368|AAAAAAAAOHICAAAA|2001-10-27||Men can exceed never even huge activities. Beneficial, legislative conditions could curb too political|3.41|17.99|7005009|scholarbrand #9|2|blinds/shades|7|Home|246|esebarpri|N/A|9220030moccasin81328|misty|Box|Unknown|50|eingcallypribarought| +10369|AAAAAAAABIICAAAA|1997-10-27||Blues may manage distinctly modern fears. Workers need clearly |2.99|1.37|1002002|importoamalg #2|2|fragrances|1|Women|155|antiantiought|medium|3410smoke74670170520|snow|Gram|Unknown|9|n stcallypribarought| +10370|AAAAAAAACIICAAAA|1997-10-27|2000-10-26|Certainly other girls take by the cha|8.69|4.43|8008001|namelessnameless #1|8|outdoor|8|Sports|601|oughtbarcally|N/A|orchid64981557440069|pink|Gram|Unknown|25|barationpribarought| +10371|AAAAAAAACIICAAAA|2000-10-27||Black, institu|1.61|1.35|8008001|amalgnameless #2|1|athletic shoes|8|Sports|276|oughtbarcally|N/A|orchid64981557440069|snow|Tbl|Unknown|1|oughtationpribarought| +10372|AAAAAAAAEIICAAAA|1997-10-27|1999-10-27|Experimental, public proposals could take for a comments. Especially magic drivers may arise. Fixed purposes could face with the prices; then important movements depend so. Almost low ta|0.21|0.13|1003001|exportiamalg #1|3|maternity|1|Women|490|barn stese|large|707733mint9817897813|smoke|Each|Unknown|87|ableationpribarought| +10373|AAAAAAAAEIICAAAA|1999-10-28|2001-10-26|Greek, civil mountains must generalize flat great, existing numbers. Reliable bits wait mature cups. Lives might tell suddenly even soviet polic|5.93|0.13|1003001|exportiamalgamalg #4|13|stereo|10|Electronics|490|barn stese|N/A|707733mint9817897813|honeydew|Dozen|Unknown|65|priationpribarought| +10374|AAAAAAAAEIICAAAA|2001-10-27||Greek, civil mountains must generalize flat great, existing numbers. Reliable bits wait mature cups. Lives might tell suddenly even soviet polic|47.25|22.68|2002001|importoimporto #1|2|shirts|2|Men|329|n stablepri|petite|529718violet40246810|peru|Ounce|Unknown|32|eseationpribarought| +10375|AAAAAAAAHIICAAAA|1997-10-27||Just possible |8.81|3.78|6001008|amalgcorp #8|1|birdal|6|Jewelry|151|oughtantiought|N/A|9700487423035rose067|purple|Case|Unknown|45|antiationpribarought| +10376|AAAAAAAAIIICAAAA|1997-10-27|2000-10-26|Mutual examples could keep almost small prospects. Flowers apply later at least whole weapons. However round statements work more for a c|2.84|1.19|6015001|scholarbrand #1|15|custom|6|Jewelry|543|prieseanti|N/A|114895271forest53827|violet|Tsp|Unknown|8|callyationpribarought| +10377|AAAAAAAAIIICAAAA|2000-10-27||Mutual examples could keep almost small prospects. Flowers apply later at least whole weapons. However round statements work more for a c|3.22|1.19|6015001|scholarmaxi #10|15|history|9|Books|543|prieseanti|N/A|114895271forest53827|rose|Gram|Unknown|14|ationationpribarought| +10378|AAAAAAAAKIICAAAA|1997-10-27|1999-10-27|Harsh bottles should not consider dependent associations. Good seats can come most in a mice. Administrative, relevant damages risk so economic services. Briefly|5.17|3.51|3002001|importoexporti #1|2|infants|3|Children|180|bareingought|extra large|5563770rose140613239|wheat|Box|Unknown|18|eingationpribarought| +10379|AAAAAAAAKIICAAAA|1999-10-28|2001-10-26|Constant, succe|58.07|31.93|3002002|importoexporti #2|2|infants|3|Children|346|bareingought|medium|32powder331164331228|snow|N/A|Unknown|48|n stationpribarought| +10380|AAAAAAAAKIICAAAA|2001-10-27||Constant, succe|28.53|19.97|3002002|edu packexporti #1|4|school-uniforms|3|Children|4|bareingought|extra large|lace9406743993630850|olive|Tsp|Unknown|38|bareingpribarought| +10381|AAAAAAAANIICAAAA|1997-10-27||Much sick girls could coincide catholic, annual possibilities. Firm, great passages will not declare most various, direct systems; pieces belie|9.92|3.07|5002002|importoscholar #2|2|country|5|Music|259|n stantiable|N/A|359593peach811507985|steel|Each|Unknown|30|oughteingpribarought| +10382|AAAAAAAAOIICAAAA|1997-10-27|2000-10-26|Separate participants cannot support. Decisions can find only stairs. Literary implications could tell ironically apparent, proper tests. Historical, natural notes live there chapters. Babies provide|5.31|2.07|2002001|importoimporto #1|2|shirts|2|Men|80|bareing|extra large|90155287551purple256|saddle|Each|Unknown|6|ableeingpribarought| +10383|AAAAAAAAOIICAAAA|2000-10-27||Separate participants cannot support. Decisions can find only stairs. Literary implications could tell ironically apparent, proper tests. Historical, natural notes live there chapters. Babies provide|8.63|5.00|2002001|edu packimporto #2|4|sports-apparel|2|Men|408|bareing|small|90155287551purple256|rosy|Bunch|Unknown|27|prieingpribarought| +10384|AAAAAAAAAJICAAAA|1997-10-27|1999-10-27|For instance alleged others know so new agents. Rounds might feel slow mothers. Owners gain interested, valuable structures. Gradually useful months broaden right partic|2.50|1.20|1002001|importoamalg #1|2|fragrances|1|Women|235|antipriable|medium|02852595081087wheat0|red|Ounce|Unknown|57|eseeingpribarought| +10385|AAAAAAAAAJICAAAA|1999-10-28|2001-10-26|At least large guards shall not control eventually serious years; wide, significant weeks get too rea|9.54|6.20|1002001|edu packedu pack #2|4|athletic|4|Shoes|235|antipriable|N/A|28123moccasin3869047|salmon|N/A|Unknown|91|antieingpribarought| +10386|AAAAAAAAAJICAAAA|2001-10-27||Ill, free children say pale, financial guests. Bad, tiny months would tell; very low shoulders consider recently general, rural forces; firms ought to spend very high, other |2.25|1.82|1002001|edu packimporto #1|4|sports-apparel|2|Men|841|antipriable|large|28123moccasin3869047|pink|Dozen|Unknown|80|callyeingpribarought| +10387|AAAAAAAADJICAAAA|1997-10-27||Techniques stop real changes. Racial shares continue only feelings. Political figures can take as problems. For long full weeks contain yesterday other guidelines. Organisational deaths must decide |6.77|5.14|1003002|exportiamalg #2|3|maternity|1|Women|565|anticallyanti|large|86641forest258325687|grey|Gross|Unknown|96|ationeingpribarought| +10388|AAAAAAAAEJICAAAA|1997-10-27|2000-10-26|Systems provide only particular companies. Also l|1.59|0.60|2002001|importoimporto #1|2|shirts|2|Men|471|oughtationese|small|20234269lace23108730|white|Gram|Unknown|15|eingeingpribarought| +10389|AAAAAAAAEJICAAAA|2000-10-27||Systems provide only particular companies. Also l|7.94|6.27|2002001|exportiamalg #2|3|maternity|1|Women|471|oughtationese|small|067421895908lemon585|navy|Pound|Unknown|61|n steingpribarought| +10390|AAAAAAAAGJICAAAA|1997-10-27|1999-10-27|Written leaders could help by a buildings; symbols could take more far isolated patients. Other, different stages flow new words. Rightly american thousands ought to contact away only|6.88|3.50|9016011|corpunivamalg #11|16|mystery|9|Books|97|ationn st|N/A|192thistle5637848486|sienna|Gram|Unknown|18|barn stpribarought| +10391|AAAAAAAAGJICAAAA|1999-10-28|2001-10-26|Above competitive wee|3.91|3.51|8004002|edu packnameless #2|4|camping|8|Sports|97|ationn st|N/A|192thistle5637848486|light|Case|Unknown|33|oughtn stpribarought| +10392|AAAAAAAAGJICAAAA|2001-10-27||Firms stay. Periods go at large consequences. Major, guilty woods could hold about political, german memories. Eyes support; new, special persons ought to alert thus imp|3.14|2.38|10011006|amalgamalgamalg #6|4|disk drives|10|Electronics|205|ationn st|N/A|192thistle5637848486|thistle|Bunch|Unknown|3|ablen stpribarought| +10393|AAAAAAAAJJICAAAA|1997-10-27||Organic, reasonable needs try briefly ever commercial activities. Newly dramatic offences will get probably english professional children. Glad, early groups set usuall|1.80|1.60|10013007|exportiamalgamalg #7|13|stereo|10|Electronics|151|oughtantiought|N/A|854038052rosy4324233|peru|Tbl|Unknown|14|prin stpribarought| +10394|AAAAAAAAKJICAAAA|1997-10-27|2000-10-26|Nice, old issues need there most slow sports. Married times cannot find then; nece|0.18|0.14|4004001|edu packedu pack #1|4|athletic|4|Shoes|114|eseoughtought|extra large|55firebrick809922390|navajo|N/A|Unknown|39|esen stpribarought| +10395|AAAAAAAAKJICAAAA|2000-10-27||Weapons imagine no longer glad, expensive levels. So christian classes may n|3.73|0.14|4004001|importounivamalg #9|2|camcorders|10|Electronics|114|eseoughtought|N/A|55firebrick809922390|cream|Tsp|Unknown|42|antin stpribarought| +10396|AAAAAAAAMJICAAAA|1997-10-27|1999-10-27|Now confidential facts must not pay however against a feet. Existing, personal windows pa|4.75|3.18|2004001|edu packimporto #1|4|sports-apparel|2|Men|451|oughtantiese|medium|40sandy1801057722607|yellow|Tbl|Unknown|27|callyn stpribarought| +10397|AAAAAAAAMJICAAAA|1999-10-28|2001-10-26|Now confidential facts must not pay however against a feet. Existing, personal windows pa|1.70|3.18|2003002|exportiimporto #2|4|pants|2|Men|171|oughtantiese|large|40sandy1801057722607|ivory|Ounce|Unknown|1|ationn stpribarought| +10398|AAAAAAAAMJICAAAA|2001-10-27||Only impossible thoughts review never signals; visible engineers tell usually likely, mental effe|9.29|3.18|2003002|importoimporto #1|4|shirts|2|Men|657|ationantically|small|3713spring3191365153|lawn|Each|Unknown|69|eingn stpribarought| +10399|AAAAAAAAPJICAAAA|1997-10-27||Political, warm months may not enter other, clear pounds. C|2.72|1.44|5004002|edu packscholar #2|4|classical|5|Music|12|ableought|N/A|9918119917tan7187679|violet|Unknown|Unknown|16|n stn stpribarought| +10400|AAAAAAAAAKICAAAA|1997-10-27|2000-10-26|Financial times continue into a decisions. Currently full services should know everywhere about a contents. So trying roles identify especially in a practices. Objectives|7.73|3.40|3004001|edu packexporti #1|4|school-uniforms|3|Children|620|barablecally|small|850790gainsboro09783|saddle|Ton|Unknown|12|barbaresebarought| +10401|AAAAAAAAAKICAAAA|2000-10-27||Emotional, rural windows may not drive now. Maps |2.49|2.04|1002002|importoamalg #2|4|fragrances|1|Women|620|barablecally|petite|850790gainsboro09783|violet|Pallet|Unknown|24|oughtbaresebarought| +10402|AAAAAAAACKICAAAA|1997-10-27|1999-10-27|Long active contexts make other, dry modules. Police shall stand successful lengths. Away early practices reassure rates. Or|35.85|29.75|5004001|edu packscholar #1|4|classical|5|Music|38|eingpri|N/A|342476767yellow06642|smoke|N/A|Unknown|33|ablebaresebarought| +10403|AAAAAAAACKICAAAA|1999-10-28|2001-10-26|Long active contexts make other, dry modules. Police shall stand successful lengths. Away early practices reassure rates. Or|2.12|0.82|6013004|exportibrand #4|13|loose stones|6|Jewelry|536|callyprianti|N/A|8khaki43715044699986|snow|Lb|Unknown|97|pribaresebarought| +10404|AAAAAAAACKICAAAA|2001-10-27||Ever personal pounds shall stand white members. Following, domestic difficulties might conform merely levels. Other, international positions might race often other minds. Lucky children may afford i|6.33|5.69|6013004|corpamalgamalg #1|13|wireless|10|Electronics|536|callyprianti|N/A|8khaki43715044699986|tomato|Unknown|Unknown|9|esebaresebarought| +10405|AAAAAAAAFKICAAAA|1997-10-27||Later significant pages cannot unite occasionally. Please complete lives get mentally most exotic results. Ever av|5.30|3.12|7012006|importonameless #6|12|paint|7|Home|268|eingcallyable|N/A|57salmon136808219529|red|Case|Unknown|7|antibaresebarought| +10406|AAAAAAAAGKICAAAA|1997-10-27|2000-10-26|Extraordinary|1.74|1.47|8009007|maxinameless #7|9|optics|8|Sports|951|oughtantin st|N/A|681335252212salmon26|burnished|Dozen|Unknown|28|callybaresebarought| +10407|AAAAAAAAGKICAAAA|2000-10-27||Other businesses match on a conservatives. Due thanks might push almost h|2.67|2.21|8009007|univbrand #8|10|jewelry boxes|6|Jewelry|710|baroughtation|N/A|681335252212salmon26|magenta|Tsp|Unknown|31|ationbaresebarought| +10408|AAAAAAAAIKICAAAA|1997-10-27|1999-10-27|Commercial countries appear useful, total values. Special, mass ser|0.62|0.53|6007001|brandcorp #1|7|pendants|6|Jewelry|165|anticallyought|N/A|0393salmon3850617605|seashell|Case|Unknown|20|eingbaresebarought| +10409|AAAAAAAAIKICAAAA|1999-10-28|2001-10-26|Clear, systematic parents appear please so young days; children may not cost howe|6.73|5.51|6009002|maxicorp #2|7|womens watch|6|Jewelry|113|anticallyought|N/A|0393salmon3850617605|violet|Gram|Unknown|66|n stbaresebarought| +10410|AAAAAAAAIKICAAAA|2001-10-27||Social, united workers may arrive vessels; social, western groups ought to discuss finally arra|3.96|5.51|7009001|maxibrand #1|7|mattresses|7|Home|113|prioughtought|N/A|0393salmon3850617605|light|Tbl|Unknown|85|baroughtesebarought| +10411|AAAAAAAALKICAAAA|1997-10-27||Humans might not place for example prominent horses. Simply final defendants would say under specific years. Old, new leaves may relate thus northern,|7.03|4.56|2001002|amalgimporto #2|1|accessories|2|Men|310|baroughtpri|medium|3495813805509sky6697|snow|Gram|Unknown|39|oughtoughtesebarought| +10412|AAAAAAAAMKICAAAA|1997-10-27|2000-10-26|Signals may set directly for a rights. Cases increase. Labour, supposed users shall not benefit then out of a rises. Today clear resources shall assess well numbers. Old, apparent depths|3.46|2.87|6006005|corpcorp #5|6|rings|6|Jewelry|542|ableeseanti|N/A|6651407876slate19637|maroon|Each|Unknown|2|ableoughtesebarought| +10413|AAAAAAAAMKICAAAA|2000-10-27||Still, heavy members write kinds; at all remaining authorities will take|4.84|2.95|6006005|exportiamalg #2|3|maternity|1|Women|542|ableeseanti|medium|3cornsilk86285537536|plum|Box|Unknown|19|prioughtesebarought| +10414|AAAAAAAAOKICAAAA|1997-10-27|1999-10-27|Typical, other offers can address to the others. Natural members should go most. Medical, molecular villages shall not counter reasonable, huge programmes. Implicat|1.19|0.44|7012005|importonameless #5|12|paint|7|Home|77|ationation|N/A|182581363359sandy511|metallic|Bunch|Unknown|6|eseoughtesebarought| +10415|AAAAAAAAOKICAAAA|1999-10-28|2001-10-26|Typical, other offers can address to the others. Natural members should go most. Medical, molecular villages shall not counter reasonable, huge programmes. Implicat|7.47|0.44|7012005|amalgexporti #2|1|newborn|3|Children|77|ationation|extra large|182581363359sandy511|turquoise|Unknown|Unknown|74|antioughtesebarought| +10416|AAAAAAAAOKICAAAA|2001-10-27||Actually correct writers take active weeks. For example underlying acids cannot believe again good, |4.78|0.44|7012005|edu packedu pack #1|4|athletic|4|Shoes|77|ationation|economy|884turquoise32401498|tomato|Ounce|Unknown|1|callyoughtesebarought| +10417|AAAAAAAABLICAAAA|1997-10-27||Disabled results will lead still however right armies. Much social men shift then teenage em|0.12|0.08|4002002|importoedu pack #2|2|mens|4|Shoes|306|callybarpri|economy|576585632379928sky48|pale|Dozen|Unknown|77|ationoughtesebarought| +10418|AAAAAAAACLICAAAA|1997-10-27|2000-10-26|American, other activities lower often rational services; collections exist. Competent reasons ough|2.42|1.50|7001009|amalgbrand #9|1|bathroom|7|Home|429|n stableese|N/A|4smoke19492956514054|navy|Cup|Unknown|85|eingoughtesebarought| +10419|AAAAAAAACLICAAAA|2000-10-27||Fees shall throw. Lives tell howe|3.53|1.50|2003002|exportiimporto #2|1|pants|2|Men|101|n stableese|medium|4smoke19492956514054|plum|Case|Unknown|22|n stoughtesebarought| +10420|AAAAAAAAELICAAAA|1997-10-27|1999-10-27|Public units should laugh there signs. Details see really almost corporate beds. Inc|4.27|1.79|2002001|importoimporto #1|2|shirts|2|Men|50|baranti|small|70wheat3050452754644|sky|Box|Unknown|25|barableesebarought| +10421|AAAAAAAAELICAAAA|1999-10-28|2001-10-26|Public units should laugh there signs. Details see really almost corporate beds. Inc|3.38|2.26|10015001|scholaramalgamalg #1|2|portable|10|Electronics|331|baranti|N/A|635007tan93377942406|sandy|Unknown|Unknown|98|oughtableesebarought| +10422|AAAAAAAAELICAAAA|2001-10-27||Villages need for example about the duties. Technical, structural tables stay often for a signs; rooms would work just moving, late goods. Democr|4.72|2.64|10015001|corpmaxi #7|6|parenting|9|Books|120|barableought|N/A|91536883royal2155237|thistle|Bunch|Unknown|58|ableableesebarought| +10423|AAAAAAAAHLICAAAA|1997-10-27||Thick miles might involve|6.09|2.19|1002002|importoamalg #2|2|fragrances|1|Women|705|antibaration|medium|95453186480lawn64048|medium|N/A|Unknown|54|priableesebarought| +10424|AAAAAAAAILICAAAA|1997-10-27|2000-10-26|Boots see still far elected members. Fully appoint|3.71|2.26|10005004|scholarunivamalg #4|5|karoke|10|Electronics|100|barbarought|N/A|9854purple0872156388|ghost|N/A|Unknown|74|eseableesebarought| +10425|AAAAAAAAILICAAAA|2000-10-27||White, extraordinary crimes could offer |4.99|2.79|3003002|exportiexporti #2|5|toddlers|3|Children|114|barbarought|petite|466saddle33828555191|tomato|Dozen|Unknown|51|antiableesebarought| +10426|AAAAAAAAKLICAAAA|1997-10-27|1999-10-27|Rich, financial possibilities must see contemporary|9.64|8.09|4001001|amalgedu pack #1|1|womens|4|Shoes|608|eingbarcally|large|201900peach366492849|yellow|Bundle|Unknown|13|callyableesebarought| +10427|AAAAAAAAKLICAAAA|1999-10-28|2001-10-26|Rich, financial possibilities must see contemporary|1.49|8.09|1002002|importoamalg #2|2|fragrances|1|Women|608|eingbarcally|medium|navajo26758534011038|linen|Bunch|Unknown|23|ationableesebarought| +10428|AAAAAAAAKLICAAAA|2001-10-27||Rich, financial possibilities must see contemporary|3.03|1.03|1002002|univmaxi #9|2|pools|8|Sports|608|eingbarcally|N/A|saddle56474844709245|metallic|Unknown|Unknown|99|eingableesebarought| +10429|AAAAAAAANLICAAAA|1997-10-27||Significantly professional appointments would not |10.87|7.82|4004002|edu packedu pack #2|4|athletic|4|Shoes|651|oughtantically|large|22341657917sandy4911|sky|Pound|Unknown|30|n stableesebarought| +10430|AAAAAAAAOLICAAAA|1997-10-27|2000-10-26|Good, open studies dream more; industrial, social organisations could understand recently quick tall theories. Lines must answer functions; subtle factor|1.28|0.62|1001001|amalgamalg #1|1|dresses|1|Women|485|antieingese|small|9441303513921wheat57|spring|Pound|Unknown|45|barpriesebarought| +10431|AAAAAAAAOLICAAAA|2000-10-27||Directly difficult friends could seek actions. Then numerous looks must fit too good areas. Common, true sales travel curious, particula|8.22|5.26|5004002|edu packscholar #2|1|classical|5|Music|485|antieingese|N/A|909royal001383449409|peru|Case|Unknown|62|oughtpriesebarought| +10432|AAAAAAAAAMICAAAA|1997-10-27|1999-10-27|Ago difficult processes can show increasingly things. Subsequent opportunities exist largely favorable journalists. Also other creatures work long little po|4.79|3.83|2001001|amalgimporto #1|1|accessories|2|Men|117|ationoughtought|medium|96664926625440green6|salmon|Gram|Unknown|18|ablepriesebarought| +10433|AAAAAAAAAMICAAAA|1999-10-28|2001-10-26|Respectable, certain hills can use today for the plants. Musicians could set then happy problems. Groups can overcome only old|54.94|3.83|2001001|edu packimporto #2|1|sports-apparel|2|Men|117|ationoughtought|large|07653468288moccasin6|thistle|Bunch|Unknown|34|pripriesebarought| +10434|AAAAAAAAAMICAAAA|2001-10-27||Respectable, certain hills can use today for the plants. Musicians could set then happy problems. Groups can overcome only old|8.29|3.83|2001001|edu packnameless #5|1|glassware|7|Home|117|ationoughtought|N/A|127354628860steel956|snow|Cup|Unknown|61|esepriesebarought| +10435|AAAAAAAADMICAAAA|1997-10-27||Teachers shall rebuild later as unique years. Certainly international shares may help. Good causes spare in order from the years. Groups|7.63|3.43|8010006|univmaxi #6|10|pools|8|Sports|143|prieseought|N/A|2565810843bisque8427|tomato|Carton|Unknown|25|antipriesebarought| +10436|AAAAAAAAEMICAAAA|1997-10-27|2000-10-26|Mor|2.42|1.13|10013016|exportiamalgamalg #16|13|stereo|10|Electronics|57|ationanti|N/A|48228703857lawn84679|slate|Oz|Unknown|17|callypriesebarought| +10437|AAAAAAAAEMICAAAA|2000-10-27||Mor|4.49|3.09|10013016|amalgscholar #2|1|rock|5|Music|276|ationanti|N/A|48228703857lawn84679|saddle|Cup|Unknown|62|ationpriesebarought| +10438|AAAAAAAAGMICAAAA|1997-10-27|1999-10-27|Major, useful rules would not achieve other bills. Strong|2.44|2.17|10009001|maxiunivamalg #1|9|televisions|10|Electronics|206|callybarable|N/A|38602papaya287988338|thistle|Ounce|Unknown|21|eingpriesebarought| +10439|AAAAAAAAGMICAAAA|1999-10-28|2001-10-26|Major, useful rules would not achieve other bills. Strong|4.75|4.18|10009001|exportimaxi #12|9|computers|9|Books|504|callybarable|N/A|steel981290922618044|olive|Bundle|Unknown|9|n stpriesebarought| +10440|AAAAAAAAGMICAAAA|2001-10-27||Small deaths think then american,|2.29|4.18|9011001|amalgunivamalg #1|11|cooking|9|Books|504|callybarable|N/A|84350479590sandy6290|grey|Case|Unknown|52|bareseesebarought| +10441|AAAAAAAAJMICAAAA|1997-10-27||Existing clients cry both. Considerable suppliers help; fortunate, double types should get over the humans. Underlying, mere services take also nice sheets. Rapidly able arms help soon. You|2.56|2.22|5003002|exportischolar #2|3|pop|5|Music|179|n stationought|N/A|5721750550violet8193|powder|Dozen|Unknown|99|oughteseesebarought| +10442|AAAAAAAAKMICAAAA|1997-10-27|2000-10-26|Poor measurements might give really senior results. Japanese modules s|6.75|5.80|10005005|scholarunivamalg #5|5|karoke|10|Electronics|174|eseationought|N/A|566549792798lawn5721|peach|Lb|Unknown|10|ableeseesebarought| +10443|AAAAAAAAKMICAAAA|2000-10-27||Theories ought to represent increasingly still white needs; open, whole varia|3.14|5.80|2002002|importoimporto #2|5|shirts|2|Men|104|esebarought|large|6123rose109723997144|turquoise|Ounce|Unknown|25|prieseesebarought| +10444|AAAAAAAAMMICAAAA|1997-10-27|1999-10-27|Books can focus for a activities. Voices should not feel months. Rough nurses ought to rush in a residents. Experiences must describe british considerations. Difficult mem|2.61|1.51|8001005|amalgnameless #5|1|athletic shoes|8|Sports|443|prieseese|N/A|044229921852orange69|papaya|Box|Unknown|52|eseeseesebarought| +10445|AAAAAAAAMMICAAAA|1999-10-28|2001-10-26|Of course influential calls may cease just here unacceptable fingers; yet italian businesses will not look necessarily international improvements. Modern, cultural aspirations sell b|2.39|1.93|4004002|edu packedu pack #2|1|athletic|4|Shoes|443|prieseese|extra large|06992766metallic7902|lace|Cup|Unknown|28|antieseesebarought| +10446|AAAAAAAAMMICAAAA|2001-10-27||Readers believe. Voices experience only animals. Managerial horses may work indeed suitable so|2.52|1.78|4004002|importoscholar #1|2|country|5|Music|38|prieseese|N/A|5480306seashell25992|plum|Tsp|Unknown|80|callyeseesebarought| +10447|AAAAAAAAPMICAAAA|1997-10-27||Contacts remain allegedly associations. Users show soldiers. Con|0.87|0.78|6007008|brandcorp #8|7|pendants|6|Jewelry|268|eingcallyable|N/A|485211595slate740464|ivory|Pallet|Unknown|6|ationeseesebarought| +10448|AAAAAAAAANICAAAA|1997-10-27|2000-10-26|Reasonable, large effects will not play also in the numbers. Purposes u|2.19|1.09|2002001|importoimporto #1|2|shirts|2|Men|996|callyn stn st|petite|737780169red56989621|frosted|Unknown|Unknown|40|eingeseesebarought| +10449|AAAAAAAAANICAAAA|2000-10-27||Reasonable, large effects will not play also in the numbers. Purposes u|2.77|1.09|6007002|brandcorp #2|7|pendants|6|Jewelry|996|callyn stn st|N/A|85569320green0845226|grey|Gram|Unknown|57|n steseesebarought| +10450|AAAAAAAACNICAAAA|1997-10-27|1999-10-27|New, bad eyebrows touch into the fields. As sudden views switch. Full, lucky employees must exhibit alongside natural members; particularly present considerations get o|3.68|1.21|3002001|importoexporti #1|2|infants|3|Children|605|antibarcally|extra large|2433peru370891685503|spring|Case|Unknown|87|barantiesebarought| +10451|AAAAAAAACNICAAAA|1999-10-28|2001-10-26|Special, local needs give most into a theories. Strong, complete practitioners make officials; a|3.01|1.21|5004002|edu packscholar #2|2|classical|5|Music|605|antibarcally|N/A|890370959820royal376|mint|Pallet|Unknown|28|oughtantiesebarought| +10452|AAAAAAAACNICAAAA|2001-10-27||Reports fit so relatively young modes; appropriate, vertical numbers vary tonight. Physical estimates develop rural, square parents. Different, red drawings submit there general, p|1.91|1.26|5004002|brandcorp #5|2|pendants|6|Jewelry|170|barationought|N/A|17170red026643699438|snow|Bundle|Unknown|24|ableantiesebarought| +10453|AAAAAAAAFNICAAAA|1997-10-27||Alrea|9.31|6.05|8015004|scholarmaxi #4|15|fishing|8|Sports|139|n stpriought|N/A|saddle78562622178705|rose|Oz|Unknown|45|priantiesebarought| +10454|AAAAAAAAGNICAAAA|1997-10-27|2000-10-26|Results highlight as patterns; so right years show. Sometimes suitable lips move with the critics. English, old mothers ought to lift now perhaps future managers. Active, single ch|2.88|1.84|9008003|namelessmaxi #3|8|romance|9|Books|614|eseoughtcally|N/A|2390110142366violet5|steel|Cup|Unknown|39|eseantiesebarought| +10455|AAAAAAAAGNICAAAA|2000-10-27||Results highlight as patterns; so right years show. Sometimes suitable lips move with the critics. English, old mothers ought to lift now perhaps future managers. Active, single ch|0.17|1.84|9008003|importoscholar #2|2|country|5|Music|614|eseoughtcally|N/A|92ghost6136667594963|rose|Gram|Unknown|22|antiantiesebarought| +10456|AAAAAAAAINICAAAA|1997-10-27|1999-10-27|Advanced, clear services will appear still on a newspapers. Representative, innocent matters must tell generally. Cold figures might purchase upwards peacefully classical movements. Then wrong states|3.97|1.42|5001001|amalgscholar #1|1|rock|5|Music|43|priese|N/A|164997209olive149652|spring|Bunch|Unknown|24|callyantiesebarought| +10457|AAAAAAAAINICAAAA|1999-10-28|2001-10-26|Bad, lin|3.16|1.42|10011003|amalgamalgamalg #3|1|disk drives|10|Electronics|253|priese|N/A|164997209olive149652|papaya|Dozen|Unknown|78|ationantiesebarought| +10458|AAAAAAAAINICAAAA|2001-10-27||Final, other taxes cannot present|3.46|1.42|8003009|exportinameless #9|3|basketball|8|Sports|253|priantiable|N/A|3852lavender71683114|green|Tsp|Unknown|13|eingantiesebarought| +10459|AAAAAAAALNICAAAA|1997-10-27||Potentially close things relea|5.38|2.09|4002002|importoedu pack #2|2|mens|4|Shoes|156|callyantiought|medium|588210921tan59938493|orchid|Pallet|Unknown|56|n stantiesebarought| +10460|AAAAAAAAMNICAAAA|1997-10-27|2000-10-26|Efficient, good eyes last more friendly, famous ideas. Letters could go. Financial, central eyes can find then ready courses. Common horses work inter|9.08|4.81|9007009|brandmaxi #9|7|reference|9|Books|171|oughtationought|N/A|40016240rose40660724|wheat|Bundle|Unknown|53|barcallyesebarought| +10461|AAAAAAAAMNICAAAA|2000-10-27||Efficient, good eyes last more friendly, famous ideas. Letters could go. Financial, central eyes can find then ready courses. Common horses work inter|4.30|4.81|9007009|importobrand #6|7|costume|6|Jewelry|241|oughtationought|N/A|40016240rose40660724|wheat|Pound|Unknown|49|oughtcallyesebarought| +10462|AAAAAAAAONICAAAA|1997-10-27|1999-10-27|Recent features place markets. Bright, new students say various waters; refugees must not see still well common heads. S|4.43|3.49|6016007|corpbrand #7|16|consignment|6|Jewelry|832|ableprieing|N/A|424tomato70278023591|moccasin|N/A|Unknown|77|ablecallyesebarought| +10463|AAAAAAAAONICAAAA|1999-10-28|2001-10-26|Disciplines might advertise; other problems should bring just violent problems. Almost violent others sup|0.23|0.09|6001008|amalgcorp #8|1|birdal|6|Jewelry|808|ableprieing|N/A|549730293spring01385|red|Carton|Unknown|29|pricallyesebarought| +10464|AAAAAAAAONICAAAA|2001-10-27||Disciplines might advertise; other problems should bring just violent problems. Almost violent others sup|0.55|0.09|6001008|exportischolar #1|3|pop|5|Music|808|ableprieing|N/A|12407mint11450991300|almond|Bundle|Unknown|8|esecallyesebarought| +10465|AAAAAAAABOICAAAA|1997-10-27||Right flexible plants may come old agents. Cheerful, reliable funds get from a shops. Forthcoming instances will not cure even lives; hitherto poor winners used to stop well rapidl|4.34|3.16|6010002|univbrand #2|10|jewelry boxes|6|Jewelry|159|n stantiought|N/A|339123powder80474425|tan|Case|Unknown|18|anticallyesebarought| +10466|AAAAAAAACOICAAAA|1997-10-27|2000-10-26|Future, right objects shall facilitate highly new plans; employees can pull once more mental differences. S|21.82|16.36|6006003|corpcorp #3|6|rings|6|Jewelry|449|n steseese|N/A|2161spring1261613207|lemon|Unknown|Unknown|52|callycallyesebarought| +10467|AAAAAAAACOICAAAA|2000-10-27||Perhaps social parents shall avoid curiously subjects. Members will make more likely researchers; other, wrong years could not wear by a rights. Strategic pr|4.38|16.36|6006003|corpcorp #4|6|rings|6|Jewelry|186|callyeingought|N/A|323sienna84284215258|salmon|Tbl|Unknown|12|ationcallyesebarought| +10468|AAAAAAAAEOICAAAA|1997-10-27|1999-10-27|Royal leaders burst please nice walls; good, parliamentary hours might report. So full cases can take green, social pools. Russians recogniz|0.57|0.49|3003001|exportiexporti #1|3|toddlers|3|Children|46|callyese|economy|salmon64971256604207|red|Ounce|Unknown|23|eingcallyesebarought| +10469|AAAAAAAAEOICAAAA|1999-10-28|2001-10-26|Women pass at least tremendous cells. Impressive operations might communicate extremely human, capable forms. O|7.01|3.22|3003001|exportiedu pack #2|3|kids|4|Shoes|193|callyese|large|salmon64971256604207|sienna|Gross|Unknown|1|n stcallyesebarought| +10470|AAAAAAAAEOICAAAA|2001-10-27||Women pass at least tremendous cells. Impressive operations might communicate extremely human, capable forms. O|0.45|3.22|3003001|amalgedu pack #1|1|womens|4|Shoes|269|callyese|large|salmon64971256604207|hot|N/A|Unknown|28|barationesebarought| +10471|AAAAAAAAHOICAAAA|1997-10-27||Exclusively political countries would pay small payments. More inevitable shops find only items. Natural beds used to choose associated days. Strong men used to sto|4.03|1.85|4004002|edu packedu pack #2|4|athletic|4|Shoes|190|barn stought|extra large|0995097176brown87733|chiffon|Tsp|Unknown|30|oughtationesebarought| +10472|AAAAAAAAIOICAAAA|1997-10-27|2000-10-26|Contributions move obviously now recent losses. Develo|3.67|2.27|9011009|amalgunivamalg #9|11|cooking|9|Books|126|callyableought|N/A|996green786099551545|saddle|Case|Unknown|16|ableationesebarought| +10473|AAAAAAAAIOICAAAA|2000-10-27||Contributions move obviously now recent losses. Develo|1.03|0.90|4004002|edu packedu pack #2|4|athletic|4|Shoes|940|callyableought|medium|996green786099551545|yellow|Each|Unknown|56|priationesebarought| +10474|AAAAAAAAKOICAAAA|1997-10-27|1999-10-27|Violent words raise quietly over the buildings. Customers must agree terms. Always modern questions|2.73|1.20|10014003|edu packamalgamalg #3|14|automotive|10|Electronics|136|callypriought|N/A|472958magenta4532539|sandy|Oz|Unknown|39|eseationesebarought| +10475|AAAAAAAAKOICAAAA|1999-10-28|2001-10-26|Responsible, major women give else great, literary actions. Good, small causes kill women. Alone, serious artists could not send glad parts. Other, little authorities introduce|3.75|1.20|4002002|importoedu pack #2|2|mens|4|Shoes|567|callypriought|large|98386turquoise347472|pale|Case|Unknown|79|antiationesebarought| +10476|AAAAAAAAKOICAAAA|2001-10-27||Responsible, major women give else great, literary actions. Good, small causes kill women. Alone, serious artists could not send glad parts. Other, little authorities introduce|15.36|4.91|4002001|importoedu pack #1|2|mens|4|Shoes|567|ationcallyanti|petite|98386turquoise347472|saddle|Oz|Unknown|58|callyationesebarought| +10477|AAAAAAAANOICAAAA|1997-10-27||Specia|0.60|0.27|2004002|edu packimporto #2|4|sports-apparel|2|Men|726|callyableation|extra large|6413154886819slate64|wheat|N/A|Unknown|32|ationationesebarought| +10478|AAAAAAAAOOICAAAA|1997-10-27|2000-10-26|Less live duties ensure less totally possible forests. Days would drag never potential, international years. Probably influential scales may like relations. Ready aspects used to a|95.38|55.32|5004001|edu packscholar #1|4|classical|5|Music|421|oughtableese|N/A|01metallic0311616258|goldenrod|Gram|Unknown|16|eingationesebarought| +10479|AAAAAAAAOOICAAAA|2000-10-27||Ideas should not want early near a demonstrations. Probably possible |5.79|3.35|5004001|univmaxi #10|10|pools|8|Sports|421|oughtableese|N/A|3aquamarine501790535|spring|Tbl|Unknown|26|n stationesebarought| +10480|AAAAAAAAAPICAAAA|1997-10-27|1999-10-27|Great hands would not publish here. Even grand years speak silver, certain rewards. M|3.78|2.23|1001001|amalgamalg #1|1|dresses|1|Women|90|barn st|extra large|5709517papaya8817145|navajo|Carton|Unknown|26|bareingesebarought| +10481|AAAAAAAAAPICAAAA|1999-10-28|2001-10-26|Perfect policies may not attempt angrily constant, familiar risks. Apparently clean months would reach humans. Ever local decades will work repeatedly normal, |8.25|7.01|1001001|corpmaxi #6|6|parenting|9|Books|90|barn st|N/A|69924481floral745202|khaki|Bundle|Unknown|80|oughteingesebarought| +10482|AAAAAAAAAPICAAAA|2001-10-27||Good, physical men can wait right recent, relevant calls; difficult plants cannot require poor legs. Also single members li|2.93|2.08|1001001|edu packexporti #1|4|school-uniforms|3|Children|90|barn st|medium|1738555173651powder9|slate|Gram|Unknown|1|ableeingesebarought| +10483|AAAAAAAADPICAAAA|1997-10-27||Letters affect yesterday. Huge, silver relations complain fortunately white, serious|7.01|3.22|2002002|importoimporto #2|2|shirts|2|Men|579|n stationanti|extra large|74137rosy03374276853|steel|Each|Unknown|70|prieingesebarought| +10484|AAAAAAAAEPICAAAA|1997-10-27|2000-10-26|Important, mass views can work. Very political men |7.57|4.46|4003001|exportiedu pack #1|3|kids|4|Shoes|13|priought|extra large|4898461wheat04113834|wheat|Gram|Unknown|31|eseeingesebarought| +10485|AAAAAAAAEPICAAAA|2000-10-27||Important, mass views can work. Very political men |0.48|4.46|4003001|edu packamalgamalg #14|3|automotive|10|Electronics|202|ablebarable|N/A|4898461wheat04113834|saddle|Unknown|Unknown|85|antieingesebarought| +10486|AAAAAAAAGPICAAAA|1997-10-27|1999-10-27|Similar players love nearly. Already specific goods occupy just; never beautiful troops should try forward papers. Clear streets want off a ages. Partly recent sections must not receive here po|6.90|4.76|7011007|amalgnameless #7|11|accent|7|Home|24|eseable|N/A|4sky0380561166967021|antique|Case|Unknown|38|callyeingesebarought| +10487|AAAAAAAAGPICAAAA|1999-10-28|2001-10-26|Computers make however political shares. Professional words could see please; loans should not feel; sophisticated stars should not go most home ultimate seats. Men|5.05|4.14|9013012|exportiunivamalg #12|13|self-help|9|Books|51|eseable|N/A|16710422550259peru99|goldenrod|Cup|Unknown|25|ationeingesebarought| +10488|AAAAAAAAGPICAAAA|2001-10-27||Newly practical purposes ought to give still |20.37|4.14|9013012|importobrand #1|12|costume|6|Jewelry|509|n stbaranti|N/A|6563868violet7995696|rosy|Case|Unknown|7|eingeingesebarought| +10489|AAAAAAAAJPICAAAA|1997-10-27||There italian pairs should make benefi|9.71|4.56|1004002|edu packamalg #2|4|swimwear|1|Women|608|eingbarcally|extra large|79117632040light0393|ghost|Cup|Unknown|60|n steingesebarought| +10490|AAAAAAAAKPICAAAA|1997-10-27|2000-10-26|Easily adv|4.25|2.29|7014001|edu packnameless #1|14|glassware|7|Home|435|antipriese|N/A|551405039mint7972382|spring|Ounce|Unknown|1|barn stesebarought| +10491|AAAAAAAAKPICAAAA|2000-10-27||Easily adv|9.15|2.29|3004002|edu packexporti #2|14|school-uniforms|3|Children|435|antipriese|large|551405039mint7972382|frosted|Ton|Unknown|2|oughtn stesebarought| +10492|AAAAAAAAMPICAAAA|1997-10-27|1999-10-27|Particular, rapid women must pretend only silent, strange pounds; armies think above markets. Thin, hard eyes bear cities. Sides lay very suddenly legal barriers. Rocks must not forestall suitable, b|2.74|1.06|3003001|exportiexporti #1|3|toddlers|3|Children|196|callyn stought|petite|50203bisque237784829|light|Case|Unknown|95|ablen stesebarought| +10493|AAAAAAAAMPICAAAA|1999-10-28|2001-10-26|Particular, rapid women must pretend only silent, strange pounds; armies think above markets. Thin, hard eyes bear cities. Sides lay very suddenly legal barriers. Rocks must not forestall suitable, b|2.27|1.06|10007005|brandunivamalg #5|7|personal|10|Electronics|196|callyn stought|N/A|50203bisque237784829|royal|Dram|Unknown|17|prin stesebarought| +10494|AAAAAAAAMPICAAAA|2001-10-27||Particular, rapid women must pretend only silent, strange pounds; armies think above markets. Thin, hard eyes bear cities. Sides lay very suddenly legal barriers. Rocks must not forestall suitable, b|4.23|2.62|10007005|corpunivamalg #7|7|mystery|9|Books|288|callyn stought|N/A|8431149947gainsboro3|saddle|Gram|Unknown|10|esen stesebarought| +10495|AAAAAAAAPPICAAAA|1997-10-27||Asian, large-scale plans shall not talk. Old issues lobby furthermore random features. At least theoretical organisations use evidently guns. Courts form rich criteria; just succ|5.07|3.59|3001002|amalgexporti #2|1|newborn|3|Children|491|oughtn stese|small|308goldenrod33668220|lemon|Unknown|Unknown|59|antin stesebarought| +10496|AAAAAAAAAAJCAAAA|1997-10-27|2000-10-26|Prime workers shall not know a|1.62|0.90|3004001|edu packexporti #1|4|school-uniforms|3|Children|865|anticallyeing|extra large|steel798742722702642|sky|N/A|Unknown|19|callyn stesebarought| +10497|AAAAAAAAAAJCAAAA|2000-10-27||So internal stations can ensure; cases shall find forward |1.56|0.90|3004001|exportiexporti #2|4|toddlers|3|Children|237|ationpriable|medium|207912red93994343252|spring|Cup|Unknown|35|ationn stesebarought| +10498|AAAAAAAACAJCAAAA|1997-10-27|1999-10-27|Championships will not reconstruct white, big others. White others shall generalize strategic, working schools. Places used to de|2.17|1.21|10015010|scholaramalgamalg #10|15|portable|10|Electronics|231|oughtpriable|N/A|06625170sienna293738|peru|Dram|Unknown|32|eingn stesebarought| +10499|AAAAAAAACAJCAAAA|1999-10-28|2001-10-26|Championships will not reconstruct white, big others. White others shall generalize strategic, working schools. Places used to de|1.20|1.21|3002002|importoexporti #2|15|infants|3|Children|326|oughtpriable|extra large|06625170sienna293738|sky|Each|Unknown|81|n stn stesebarought| +10500|AAAAAAAACAJCAAAA|2001-10-27||Championships will not reconstruct white, big others. White others shall generalize strategic, working schools. Places used to de|4.94|1.21|3002002|edu packscholar #1|4|classical|5|Music|247|oughtpriable|N/A|847472763ghost875129|wheat|Unknown|Unknown|48|barbarantibarought| +10501|AAAAAAAAFAJCAAAA|1997-10-27||Times should alleviate again whole positions.|58.29|44.88|8011002|amalgmaxi #2|11|archery|8|Sports|415|antioughtese|N/A|6393570ivory07898239|grey|N/A|Unknown|12|oughtbarantibarought| +10502|AAAAAAAAGAJCAAAA|1997-10-27|2000-10-26|Users should debate. Great, major plants will find. Natural processes marry more blue systems. Surely single families will not express combined, industrial resources. Volun|4.95|2.32|3004001|edu packexporti #1|4|school-uniforms|3|Children|450|barantiese|medium|magenta6714548272980|white|Each|Unknown|28|ablebarantibarought| +10503|AAAAAAAAGAJCAAAA|2000-10-27||Often foreign politicians distribute for the fe|4.98|2.32|4003002|exportiedu pack #2|3|kids|4|Shoes|466|callycallyese|petite|80sienna585872184286|white|Pound|Unknown|14|pribarantibarought| +10504|AAAAAAAAIAJCAAAA|1997-10-27|1999-10-27|Alone days must undertake children; pages will face cases. Days fit especially black standards. Certain, involved numbers may not intend high, |9.53|8.19|2001001|amalgimporto #1|1|accessories|2|Men|109|n stbarought|petite|638439papaya36694175|sandy|Bunch|Unknown|10|esebarantibarought| +10505|AAAAAAAAIAJCAAAA|1999-10-28|2001-10-26|Alone days must undertake children; pages will face cases. Days fit especially black standards. Certain, involved numbers may not intend high, |2.82|8.19|10015017|scholaramalgamalg #17|15|portable|10|Electronics|109|n stbarought|N/A|638439papaya36694175|wheat|Pallet|Unknown|41|antibarantibarought| +10506|AAAAAAAAIAJCAAAA|2001-10-27||Rar|3.14|8.19|6010003|univbrand #3|15|jewelry boxes|6|Jewelry|158|eingantiought|N/A|638439papaya36694175|pale|Bunch|Unknown|7|callybarantibarought| +10507|AAAAAAAALAJCAAAA|1997-10-27||Plans must go well horizontal players; e|9.14|6.76|3002002|importoexporti #2|2|infants|3|Children|917|ationoughtn st|medium|97tan092892808811626|seashell|Unknown|Unknown|1|ationbarantibarought| +10508|AAAAAAAAMAJCAAAA|1997-10-27|2000-10-26|Provinces shall bury in a |5.16|1.54|4003001|exportiedu pack #1|3|kids|4|Shoes|233|pripriable|extra large|27295682568534peru42|tan|Box|Unknown|79|eingbarantibarought| +10509|AAAAAAAAMAJCAAAA|2000-10-27||Provinces shall bury in a |3.59|1.29|7002010|importobrand #10|2|bedding|7|Home|233|pripriable|N/A|27295682568534peru42|tomato|Tsp|Unknown|40|n stbarantibarought| +10510|AAAAAAAAOAJCAAAA|1997-10-27|1999-10-27|Heavy plans ought to sound too just young users; further traditional eyes welcome neither too el|3.45|2.10|9004011|edu packmaxi #11|4|entertainments|9|Books|343|priesepri|N/A|5violet8060578932931|blush|Ton|Unknown|23|baroughtantibarought| +10511|AAAAAAAAOAJCAAAA|1999-10-28|2001-10-26|Heavy plans ought to sound too just young users; further traditional eyes welcome neither too el|8.73|5.76|5002002|importoscholar #2|2|country|5|Music|232|priesepri|N/A|5violet8060578932931|wheat|Tsp|Unknown|83|oughtoughtantibarought| +10512|AAAAAAAAOAJCAAAA|2001-10-27||Heavy plans ought to sound too just young users; further traditional eyes welcome neither too el|1.60|5.76|5002002|edu packimporto #1|2|sports-apparel|2|Men|159|priesepri|small|5violet8060578932931|salmon|N/A|Unknown|49|ableoughtantibarought| +10513|AAAAAAAABBJCAAAA|1997-10-27||Communities used to find away to the ears. Equally fixed trees will fall rapidly sacred points. Shows get mentally high |9.21|4.88|10010008|univamalgamalg #8|10|memory|10|Electronics|159|n stantiought|N/A|8papaya5359093523559|dodger|N/A|Unknown|66|prioughtantibarought| +10514|AAAAAAAACBJCAAAA|1997-10-27|2000-10-26|Once olympic sorts could control governments. Final sections mig|7.76|4.42|5004001|edu packscholar #1|4|classical|5|Music|562|ablecallyanti|N/A|332623violet28691953|papaya|Tsp|Unknown|58|eseoughtantibarought| +10515|AAAAAAAACBJCAAAA|2000-10-27||Once olympic sorts could control governments. Final sections mig|2.58|4.42|1001002|amalgamalg #2|4|dresses|1|Women|684|eseeingcally|medium|332623violet28691953|mint|Unknown|Unknown|8|antioughtantibarought| +10516|AAAAAAAAEBJCAAAA|1997-10-27|1999-10-27|Far similar weapons might remind downstairs all alone fingers. Recommendations must not form now used eyes. Deep items may|7.30|2.33|3001001|amalgexporti #1|1|newborn|3|Children|319|n stoughtpri|medium|043733493smoke128426|sienna|Pound|Unknown|76|callyoughtantibarought| +10517|AAAAAAAAEBJCAAAA|1999-10-28|2001-10-26|Strangely significant factors may take now attractive product|33.22|25.57|3001001|corpmaxi #6|1|parenting|9|Books|232|n stoughtpri|N/A|043733493smoke128426|medium|Cup|Unknown|7|ationoughtantibarought| +10518|AAAAAAAAEBJCAAAA|2001-10-27||Strangely significant factors may take now attractive product|7.56|4.46|4003001|exportiedu pack #1|1|kids|4|Shoes|158|n stoughtpri|economy|5964254violet6079240|tomato|Each|Unknown|63|eingoughtantibarought| +10519|AAAAAAAAHBJCAAAA|1997-10-27||Vast, lively periods will not treat new, average r|6.01|5.34|9005008|scholarmaxi #8|5|history|9|Books|364|esecallypri|N/A|7153890418sienna7641|papaya|Carton|Unknown|53|n stoughtantibarought| +10520|AAAAAAAAIBJCAAAA|1997-10-27|2000-10-26|Eyes contain as walls; psychological, active views will grant. Courses kill before in a ports; very worried leaders might make quite judicial, altern|3.19|2.83|7005001|scholarbrand #1|5|blinds/shades|7|Home|181|oughteingought|N/A|7822papaya8653421227|powder|Pound|Unknown|7|barableantibarought| +10521|AAAAAAAAIBJCAAAA|2000-10-27||Eyes contain as walls; psychological, active views will grant. Courses kill before in a ports; very worried leaders might make quite judicial, altern|1.08|0.87|7005001|edu packamalg #2|4|swimwear|1|Women|206|oughteingought|extra large|1ivory81700356116625|black|Bunch|Unknown|6|oughtableantibarought| +10522|AAAAAAAAKBJCAAAA|1997-10-27|1999-10-27|For example physical events shall find far fires; courts reveal poor experiences. Others control to the activities. Square features answ|2.63|1.63|8016003|corpmaxi #3|16|golf|8|Sports|156|callyantiought|N/A|152red05369697093961|tan|Carton|Unknown|23|ableableantibarought| +10523|AAAAAAAAKBJCAAAA|1999-10-28|2001-10-26|For example physical events shall find far fires; courts reveal poor experiences. Others control to the activities. Square features answ|3.05|1.46|8016003|exportiamalg #2|3|maternity|1|Women|95|callyantiought|medium|253084936ghost464886|sky|Dram|Unknown|24|priableantibarought| +10524|AAAAAAAAKBJCAAAA|2001-10-27||Long lines seize as at the sons. Situations leave too formal, final children. Able hands become. Increasingly bu|3.26|1.46|2002001|importoimporto #1|3|shirts|2|Men|95|callyantiought|extra large|8304363857rose069025|blush|Each|Unknown|35|eseableantibarought| +10525|AAAAAAAANBJCAAAA|1997-10-27||About appropriate divisions win eastern hearts. Usual, clear boots will prosper soon more aware families. Red, satisfactory interests receive|1.49|0.95|1002002|importoamalg #2|2|fragrances|1|Women|681|oughteingcally|large|6512997powder8033839|sky|Tsp|Unknown|94|antiableantibarought| +10526|AAAAAAAAOBJCAAAA|1997-10-27|2000-10-26|Far, recent settlements help no both unique stages; commercial, british months should stay very to a instruments. Female, wonderful rates could see just ever foreign c|9.95|7.96|4004001|edu packedu pack #1|4|athletic|4|Shoes|141|oughteseought|economy|90slate6319327486737|smoke|Tbl|Unknown|69|callyableantibarought| +10527|AAAAAAAAOBJCAAAA|2000-10-27||Far, recent settlements help no both unique stages; commercial, british months should stay very to a instruments. Female, wonderful rates could see just ever foreign c|9.66|4.54|4004001|univunivamalg #4|10|travel|9|Books|105|antibarought|N/A|90slate6319327486737|saddle|Dozen|Unknown|23|ationableantibarought| +10528|AAAAAAAAACJCAAAA|1997-10-27|1999-10-27|Fundamentally joint committees should live all right; others|9.87|5.72|10014006|edu packamalgamalg #6|14|automotive|10|Electronics|220|barableable|N/A|127royal957686819826|thistle|Ton|Unknown|48|eingableantibarought| +10529|AAAAAAAAACJCAAAA|1999-10-28|2001-10-26|Whole years like by a shots. Pictures cry western animals. I|4.25|1.65|7005010|scholarbrand #10|5|blinds/shades|7|Home|220|barableable|N/A|127royal957686819826|metallic|Lb|Unknown|56|n stableantibarought| +10530|AAAAAAAAACJCAAAA|2001-10-27||Metals last notes. Political|3.19|1.65|7005010|edu packbrand #3|5|estate|6|Jewelry|980|barableable|N/A|127royal957686819826|orange|Cup|Unknown|64|barpriantibarought| +10531|AAAAAAAADCJCAAAA|1997-10-27||Authorities ought to survey even local children. Genetic qualities invent now advanced worlds. British vari|4.41|2.24|1004002|edu packamalg #2|4|swimwear|1|Women|585|antieinganti|medium|tomato58809838167402|khaki|Case|Unknown|22|oughtpriantibarought| +10532|AAAAAAAAECJCAAAA|1997-10-27|2000-10-26|Other, mediterranean requirements fall certainly somewhat major a|0.64|0.24|4004001|edu packedu pack #1|4|athletic|4|Shoes|457|ationantiese|large|16976139150yellow009|wheat|Box|Unknown|45|ablepriantibarought| +10533|AAAAAAAAECJCAAAA|2000-10-27||Perfect cars rejoin so following years. Animals begin so about acceptable vehicles. Independent commitments shall not behave scarcely difficult, old elections. Domestic, |0.59|0.44|4004001|exportischolar #2|3|pop|5|Music|52|ableanti|N/A|9926sienna0024486793|white|Ounce|Unknown|89|pripriantibarought| +10534|AAAAAAAAGCJCAAAA|1997-10-27|1999-10-27|Capital injuries identify. Wrong, suitable vessels must not hook quickly in a meals. As different causes might exercise fre|2.44|1.09|2003001|exportiimporto #1|3|pants|2|Men|453|priantiese|medium|275247334sandy625363|red|Pallet|Unknown|66|esepriantibarought| +10535|AAAAAAAAGCJCAAAA|1999-10-28|2001-10-26|Neatly slim systems get hard existing voices. Different, possible abilities manage again shares. Far social officers lift as. Marginal, efficient moves initiate other, huge e|9.86|7.49|9007012|brandmaxi #12|7|reference|9|Books|184|eseeingought|N/A|47752774731452dark88|turquoise|Bundle|Unknown|21|antipriantibarought| +10536|AAAAAAAAGCJCAAAA|2001-10-27||Small areas must |5.71|1.82|9007012|importoexporti #1|7|infants|3|Children|275|eseeingought|petite|635424478015125puff0|orange|Ton|Unknown|39|callypriantibarought| +10537|AAAAAAAAJCJCAAAA|1997-10-27||Probably local years will live tonnes. Step|4.89|1.90|7010008|univnameless #8|10|flatware|7|Home|111|oughtoughtought|N/A|4861869navajo1156058|beige|Bundle|Unknown|13|ationpriantibarought| +10538|AAAAAAAAKCJCAAAA|1997-10-27|2000-10-26|Plants must fulfil home olympic operations. Powers could enter so finally marvellou|3.85|2.11|1003001|exportiamalg #1|3|maternity|1|Women|308|eingbarpri|medium|731smoke605057437722|salmon|Box|Unknown|30|eingpriantibarought| +10539|AAAAAAAAKCJCAAAA|2000-10-27||Plants must fulfil home olympic operations. Powers could enter so finally marvellou|3.80|1.25|1003001|corpbrand #4|3|consignment|6|Jewelry|308|eingbarpri|N/A|731smoke605057437722|bisque|Cup|Unknown|16|n stpriantibarought| +10540|AAAAAAAAMCJCAAAA|1997-10-27|1999-10-27|Exactly left yea|0.54|0.20|8013001|exportimaxi #1|13|sailing|8|Sports|270|barationable|N/A|958837657spring82551|violet|Tsp|Unknown|30|bareseantibarought| +10541|AAAAAAAAMCJCAAAA|1999-10-28|2001-10-26|Exactly left yea|8.32|0.20|7010002|univnameless #2|13|flatware|7|Home|270|barationable|N/A|958837657spring82551|ivory|Tbl|Unknown|59|oughteseantibarought| +10542|AAAAAAAAMCJCAAAA|2001-10-27||Elderly doubts indicate as yet anxious arms; political, financial supporters should greet no longer freely chosen times. Important colours remember phys|3.76|0.20|1004001|edu packamalg #1|4|swimwear|1|Women|318|barationable|medium|47437smoke2197061550|puff|Unknown|Unknown|30|ableeseantibarought| +10543|AAAAAAAAPCJCAAAA|1997-10-27||Nice studies increase at a leads; indirectly foreign techniques should not happen never overseas, civil gods. Criminal, typical theories shall boost ab|1.40|1.17|4002002|importoedu pack #2|2|mens|4|Shoes|181|oughteingought|petite|62912royal7721482397|sienna|Box|Unknown|24|prieseantibarought| +10544|AAAAAAAAADJCAAAA|1997-10-27|2000-10-26|Brothers would count other partners; private patients know. Never joint farmers c|3.21|2.31|9009009|maximaxi #9|9|science|9|Books|177|ationationought|N/A|7240638smoke36335420|olive|Tsp|Unknown|2|eseeseantibarought| +10545|AAAAAAAAADJCAAAA|2000-10-27||Young states respond between a eyes|9.93|2.31|9009009|scholarunivamalg #6|5|karoke|10|Electronics|177|ationationought|N/A|7240638smoke36335420|plum|Box|Unknown|34|antieseantibarought| +10546|AAAAAAAACDJCAAAA|1997-10-27|1999-10-27|Thirdly urb|0.28|0.24|7016007|corpnameless #7|16|furniture|7|Home|62|ablecally|N/A|48943ivory3553656922|plum|Lb|Unknown|41|callyeseantibarought| +10547|AAAAAAAACDJCAAAA|1999-10-28|2001-10-26|Thirdly urb|9.85|6.40|7016007|brandcorp #4|16|pendants|6|Jewelry|62|ablecally|N/A|48943ivory3553656922|thistle|Ton|Unknown|34|ationeseantibarought| +10548|AAAAAAAACDJCAAAA|2001-10-27||Thirdly urb|1.46|6.40|7016007|edu packnameless #9|16|glassware|7|Home|317|ationoughtpri|N/A|48943ivory3553656922|tomato|Ton|Unknown|19|eingeseantibarought| +10549|AAAAAAAAFDJCAAAA|1997-10-27||Soon british records must tolerate often to a children. Forward, running women understand residential, necessary executives. Impossible, new classes should elect so remarkable yea|2.05|1.02|8011010|amalgmaxi #10|11|archery|8|Sports|365|anticallypri|N/A|555068876684569puff4|royal|Gross|Unknown|33|n steseantibarought| +10550|AAAAAAAAGDJCAAAA|1997-10-27|2000-10-26|Clear, social |3.69|1.99|5003001|exportischolar #1|3|pop|5|Music|39|n stpri|N/A|01883088275334rose95|sienna|Each|Unknown|49|barantiantibarought| +10551|AAAAAAAAGDJCAAAA|2000-10-27||Clear, social |2.55|1.35|8007002|brandnameless #2|3|hockey|8|Sports|39|n stpri|N/A|01883088275334rose95|plum|Carton|Unknown|54|oughtantiantibarought| +10552|AAAAAAAAIDJCAAAA|1997-10-27|1999-10-27|Working, deep stars should not make from a tables. Thousands used to see other, essential animals. Sources could address difficult, useless workers. Brief girls look in a wines. Full, simple|1.26|0.46|3002001|importoexporti #1|2|infants|3|Children|282|ableeingable|small|627719709073rose3781|deep|Ton|Unknown|29|ableantiantibarought| +10553|AAAAAAAAIDJCAAAA|1999-10-28|2001-10-26|Close, elected germans begin also |1.54|0.46|1004002|edu packamalg #2|2|swimwear|1|Women|144|eseeseought|small|627719709073rose3781|sienna|Bunch|Unknown|17|priantiantibarought| +10554|AAAAAAAAIDJCAAAA|2001-10-27||Close, elected germans begin also |4.49|0.46|1004002|scholarunivamalg #15|2|karoke|10|Electronics|144|eseeseought|N/A|816715157849spring48|papaya|N/A|Unknown|18|eseantiantibarought| +10555|AAAAAAAALDJCAAAA|1997-10-27||Economic mountains inhibit now. Messages keep useful, special figures; social, fat objections ought to meet available, cool things|3.19|2.80|4002002|importoedu pack #2|2|mens|4|Shoes|278|eingationable|petite|86powder064455211001|linen|Bunch|Unknown|32|antiantiantibarought| +10556|AAAAAAAAMDJCAAAA|1997-10-27|2000-10-26|Methods transmit. Outdoor, young purposes may se|0.14|0.07|6013005|exportibrand #5|13|loose stones|6|Jewelry|15|antiought|N/A|12896373213872smoke5|lace|Ton|Unknown|26|callyantiantibarought| +10557|AAAAAAAAMDJCAAAA|2000-10-27||Parents deny both movements. Police agree careers. Terms cannot obtain namely influential windows; boys sell somehow extrao|7.01|0.07|4004002|edu packedu pack #2|4|athletic|4|Shoes|15|antiought|medium|55514moccasin3855879|tan|Pallet|Unknown|46|ationantiantibarought| +10558|AAAAAAAAODJCAAAA|1997-10-27|1999-10-27|Mechanisms may fight. Definite, complete corners shall appear even social, concerned deputies. Even bad hands shall detect true events. Gr|8.48|6.27|4004001|edu packedu pack #1|4|athletic|4|Shoes|614|eseoughtcally|medium|799948honeydew067297|purple|Carton|Unknown|12|eingantiantibarought| +10559|AAAAAAAAODJCAAAA|1999-10-28|2001-10-26|Animals offer still; very secret possibilities establish loose fingers. Go|8.63|3.88|4004001|exportiimporto #2|4|pants|2|Men|614|eseoughtcally|small|152lavender502949817|puff|Lb|Unknown|14|n stantiantibarought| +10560|AAAAAAAAODJCAAAA|2001-10-27||More final comparisons believe ever hard women. Little, regular groups can arrange long attractive creditors. Possible neighbours acknowledge obviously courses. Existing, only |4.43|3.98|4004001|importoamalgamalg #4|12|monitors|10|Electronics|614|eseoughtcally|N/A|152lavender502949817|grey|N/A|Unknown|32|barcallyantibarought| +10561|AAAAAAAABEJCAAAA|1997-10-27||Controversial, different docto|7.99|3.67|10009005|maxiunivamalg #5|9|televisions|10|Electronics|421|oughtableese|N/A|358786puff5189218955|bisque|Case|Unknown|58|oughtcallyantibarought| +10562|AAAAAAAACEJCAAAA|1997-10-27|2000-10-26|Persons get following str|0.16|0.10|6010003|univbrand #3|10|jewelry boxes|6|Jewelry|147|ationeseought|N/A|0547ghost54902305215|turquoise|Box|Unknown|59|ablecallyantibarought| +10563|AAAAAAAACEJCAAAA|2000-10-27||Persons get following str|8.11|5.83|1003002|exportiamalg #2|10|maternity|1|Women|147|ationeseought|extra large|rosy3096771307572383|sienna|N/A|Unknown|2|pricallyantibarought| +10564|AAAAAAAAEEJCAAAA|1997-10-27|1999-10-27|Soldiers may look generally specific forces. Functions shall provide even negative pensioners. Real, soviet opportunities cry no lon|52.92|44.98|9007005|brandmaxi #5|7|reference|9|Books|368|eingcallypri|N/A|63243snow87886386376|snow|Pound|Unknown|29|esecallyantibarought| +10565|AAAAAAAAEEJCAAAA|1999-10-28|2001-10-26|Members identify usefully steep problems. General clothes could not face merely areas. Councils could contact tonight other, social reasons. Tools show uniformly massiv|0.87|0.40|9007005|namelesscorp #6|8|mens watch|6|Jewelry|368|eingcallypri|N/A|960870pale5106100314|seashell|Ounce|Unknown|53|anticallyantibarought| +10566|AAAAAAAAEEJCAAAA|2001-10-27||Members identify usefully steep problems. General clothes could not face merely areas. Councils could contact tonight other, social reasons. Tools show uniformly massiv|2.05|1.08|10001010|amalgunivamalg #10|8|cameras|10|Electronics|249|eingcallypri|N/A|960870pale5106100314|wheat|Unknown|Unknown|49|callycallyantibarought| +10567|AAAAAAAAHEJCAAAA|1997-10-27||Ill able planes should like. Collections could score over. Already unlikely purposes used to need illustrations. Human, experimental services pursue. Hard alleged|9.41|5.45|4004002|edu packedu pack #2|4|athletic|4|Shoes|36|callypri|medium|8papaya9600139336775|seashell|Ton|Unknown|21|ationcallyantibarought| +10568|AAAAAAAAIEJCAAAA|1997-10-27|2000-10-26|Days make there great, firm voters. Friends listen now lively tenants; also italian views used to know|8.41|4.20|7003009|exportibrand #9|3|kids|7|Home|526|callyableanti|N/A|26154217952174peach4|purple|Tsp|Unknown|50|eingcallyantibarought| +10569|AAAAAAAAIEJCAAAA|2000-10-27||Of course inner members watch yet independent, new banks; likely, rational merchants move islamic, blue cars. Potential, soviet years might not make then doctors. Always distinctive mo|4.54|4.20|10015013|scholaramalgamalg #13|3|portable|10|Electronics|354|callyableanti|N/A|26154217952174peach4|yellow|Ounce|Unknown|4|n stcallyantibarought| +10570|AAAAAAAAKEJCAAAA|1997-10-27|1999-10-27|Companies must not topple sometimes big masters. Distinctive doctors can know as for a streets. Items might not see to a children; other, crucial ships must like conventional bands. Broad |1.25|0.58|6007003|brandcorp #3|7|pendants|6|Jewelry|91|oughtn st|N/A|66854powder129424088|goldenrod|Carton|Unknown|5|barationantibarought| +10571|AAAAAAAAKEJCAAAA|1999-10-28|2001-10-26|Sensitive trees give there values; increasing, practical supporters start about. Hot,|3.88|0.58|1002002|importoamalg #2|7|fragrances|1|Women|812|oughtn st|medium|66854powder129424088|powder|Dozen|Unknown|4|oughtationantibarought| +10572|AAAAAAAAKEJCAAAA|2001-10-27||Able trees ought to resist so. Academic, contemporary groups shall get; red, open patients work entirely up a children. Quite current factors must |4.19|1.25|1002002|corpamalgamalg #16|7|wireless|10|Electronics|812|oughtn st|N/A|66854powder129424088|magenta|Case|Unknown|17|ableationantibarought| +10573|AAAAAAAANEJCAAAA|1997-10-27||At most new pictures keep. American, different clients assume always problems; forward just years used to formulate just actually full indivi|0.72|0.43|8009004|maxinameless #4|9|optics|8|Sports|250|barantiable|N/A|24134orange610728146|midnight|Each|Unknown|4|priationantibarought| +10574|AAAAAAAAOEJCAAAA|1997-10-27|2000-10-26|Favorite, global premises look european, different actions; favourably major fees complain |8.76|3.06|9015003|scholarunivamalg #3|15|fiction|9|Books|278|eingationable|N/A|072824509sandy946326|rosy|Bundle|Unknown|71|eseationantibarought| +10575|AAAAAAAAOEJCAAAA|2000-10-27||Favorite, global premises look european, different actions; favourably major fees complain |0.88|0.29|9015003|amalgunivamalg #4|15|cooking|9|Books|278|eingationable|N/A|421371lime5799251788|blanched|Tsp|Unknown|24|antiationantibarought| +10576|AAAAAAAAAFJCAAAA|1997-10-27|1999-10-27|Orthodox members celebrate deeply either essential exa|39.05|15.62|1003001|exportiamalg #1|3|maternity|1|Women|92|ablen st|extra large|96691441663saddle520|magenta|Gross|Unknown|18|callyationantibarought| +10577|AAAAAAAAAFJCAAAA|1999-10-28|2001-10-26|Orthodox members celebrate deeply either essential exa|5.09|15.62|1003001|importoscholar #2|3|country|5|Music|174|ablen st|N/A|96691441663saddle520|tan|Tbl|Unknown|57|ationationantibarought| +10578|AAAAAAAAAFJCAAAA|2001-10-27||Active effect|6.64|15.62|1003001|amalgimporto #1|3|accessories|2|Men|174|eseationought|medium|15020641powder579566|purple|Tsp|Unknown|8|eingationantibarought| +10579|AAAAAAAADFJCAAAA|1997-10-27||Unlikely, inner years print very ultimate branches.|3.92|3.09|10016006|corpamalgamalg #6|16|wireless|10|Electronics|83|prieing|N/A|10turquoise191920288|khaki|Carton|Unknown|3|n stationantibarought| +10580|AAAAAAAAEFJCAAAA|1997-10-27|2000-10-26|Big, payable areas might put more in a books. European men sit left, conservative types. Patient,|63.25|25.30|4003001|exportiedu pack #1|3|kids|4|Shoes|488|eingeingese|extra large|6123powder5864312702|tomato|Unknown|Unknown|20|bareingantibarought| +10581|AAAAAAAAEFJCAAAA|2000-10-27||Big, payable areas might put more in a books. European men sit left, conservative types. Patient,|9.82|25.30|1001002|amalgamalg #2|3|dresses|1|Women|488|eingeingese|small|63397830882violet234|moccasin|Carton|Unknown|77|oughteingantibarought| +10582|AAAAAAAAGFJCAAAA|1997-10-27|1999-10-27|Even old examples shall take very. Local legs shall last nu|3.47|1.21|8010003|univmaxi #3|10|pools|8|Sports|370|barationpri|N/A|puff4590156454290335|papaya|Lb|Unknown|100|ableeingantibarought| +10583|AAAAAAAAGFJCAAAA|1999-10-28|2001-10-26|However good women know real, national looks. Fun, emotional members may provide on a months. Over rapid differences shall give laws. Primary factories might g|2.33|1.46|8010003|edu packscholar #2|4|classical|5|Music|370|barationpri|N/A|728038puff0589472287|sandy|Ton|Unknown|13|prieingantibarought| +10584|AAAAAAAAGFJCAAAA|2001-10-27||More important neighbours follow up to various types. Public hours may make from a changes. Labour sides reflec|79.93|1.46|8010003|importoedu pack #1|4|mens|4|Shoes|65|antically|economy|728038puff0589472287|peru|Oz|Unknown|12|eseeingantibarought| +10585|AAAAAAAAJFJCAAAA|1997-10-27||Other win|2.46|1.91|2001002|amalgimporto #2|1|accessories|2|Men|70|baration|economy|1934889761299papaya8|ivory|Lb|Unknown|33|antieingantibarought| +10586|AAAAAAAAKFJCAAAA|1997-10-27|2000-10-26|Specific, blue publications will help. In order successful files could import seldom black, scottish bombs. Other things remain days. Mo|6.84|6.08|4002001|importoedu pack #1|2|mens|4|Shoes|536|callyprianti|large|3781540salmon3797833|deep|Dozen|Unknown|52|callyeingantibarought| +10587|AAAAAAAAKFJCAAAA|2000-10-27||Years would not convert still songs. Capable, golden colleagues register so firms. Successful, lesser cases say with a fields. Sch|9.72|6.22|10001014|amalgunivamalg #14|2|cameras|10|Electronics|536|callyprianti|N/A|47smoke0994984672147|lawn|Oz|Unknown|5|ationeingantibarought| +10588|AAAAAAAAMFJCAAAA|1997-10-27|1999-10-27|Good, economic estates make right other, primitive eyes. Moments might save terrible,|87.32|46.27|2001001|amalgimporto #1|1|accessories|2|Men|240|bareseable|petite|4415peru084983579903|peach|Oz|Unknown|17|eingeingantibarought| +10589|AAAAAAAAMFJCAAAA|1999-10-28|2001-10-26|Good, economic estates make right other, primitive eyes. Moments might save terrible,|2.86|1.74|5003002|exportischolar #2|3|pop|5|Music|240|bareseable|N/A|314522255234saddle71|violet|Dozen|Unknown|2|n steingantibarought| +10590|AAAAAAAAMFJCAAAA|2001-10-27||Good, economic estates make right other, primitive eyes. Moments might save terrible,|3.00|1.74|5003002|scholarnameless #1|3|fitness|8|Sports|240|bareseable|N/A|304red09555895476550|sienna|Box|Unknown|95|barn stantibarought| +10591|AAAAAAAAPFJCAAAA|1997-10-27||Military wives would not add then implications. Ashamed, ben|2.71|1.65|6014008|edu packbrand #8|14|estate|6|Jewelry|908|eingbarn st|N/A|1373sky5861412211072|tomato|Oz|Unknown|28|oughtn stantibarought| +10592|AAAAAAAAAGJCAAAA|1997-10-27|2000-10-26|Unknown, new children may move gold, in|69.40|57.60|5001001|amalgscholar #1|1|rock|5|Music|360|barcallypri|N/A|0858015882midnight87|steel|N/A|Unknown|74|ablen stantibarought| +10593|AAAAAAAAAGJCAAAA|2000-10-27||Unknown, new children may move gold, in|4.09|57.60|8004004|edu packnameless #4|4|camping|8|Sports|360|barcallypri|N/A|0858015882midnight87|rose|Tbl|Unknown|4|prin stantibarought| +10594|AAAAAAAACGJCAAAA|1997-10-27|1999-10-27|Social, new members reply stations. Different years can break areas. Never gre|3.22|1.89|7004005|edu packbrand #5|4|curtains/drapes|7|Home|199|n stn stought|N/A|19142175saddle477324|violet|Gross|Unknown|31|esen stantibarought| +10595|AAAAAAAACGJCAAAA|1999-10-28|2001-10-26|Middle, consistent executives could cut primary, brave children. Redundant organisations could walk ready, bi|0.76|0.41|3004002|edu packexporti #2|4|school-uniforms|3|Children|284|eseeingable|extra large|19142175saddle477324|rosy|Oz|Unknown|8|antin stantibarought| +10596|AAAAAAAACGJCAAAA|2001-10-27||Middle, consistent executives could cut primary, brave children. Redundant organisations could walk ready, bi|1.41|0.41|6012005|importobrand #5|12|costume|6|Jewelry|284|eseeingable|N/A|098salmon57663071983|pink|Ton|Unknown|1|callyn stantibarought| +10597|AAAAAAAAFGJCAAAA|1997-10-27||Secondary, red structures may seek eyes. High true titles should make now junior fat thoughts. Partly excellent authorities receive direct, net parties. Parents look most also other issues. Empty, con|8.59|6.01|9002002|importomaxi #2|2|business|9|Books|35|antipri|N/A|436366657443sandy794|royal|Carton|Unknown|11|ationn stantibarought| +10598|AAAAAAAAGGJCAAAA|1997-10-27|2000-10-26|Pr|62.21|50.39|1002001|importoamalg #1|2|fragrances|1|Women|549|n steseanti|large|6royal37545679591674|pink|Each|Unknown|9|eingn stantibarought| +10599|AAAAAAAAGGJCAAAA|2000-10-27||Pr|89.75|43.08|3002002|importoexporti #2|2|infants|3|Children|549|n steseanti|medium|2honeydew86369047918|lawn|Gross|Unknown|32|n stn stantibarought| +10600|AAAAAAAAIGJCAAAA|1997-10-27|1999-10-27|Approx residential memories enforce there in a parts. Neutral children supply already great, gastri|3.14|0.94|6008001|namelesscorp #1|8|mens watch|6|Jewelry|399|n stn stpri|N/A|6269moccasin72881834|magenta|Ton|Unknown|16|barbarcallybarought| +10601|AAAAAAAAIGJCAAAA|1999-10-28|2001-10-26|Remaining customers fi|6.34|0.94|5004002|edu packscholar #2|4|classical|5|Music|15|antiought|N/A|43655orange009708149|tomato|N/A|Unknown|10|oughtbarcallybarought| +10602|AAAAAAAAIGJCAAAA|2001-10-27||Deep, sufficient systems would overcome immediately; alone, chi|5.92|2.90|5004002|exportiexporti #1|3|toddlers|3|Children|15|antiought|petite|43655orange009708149|hot|Tsp|Unknown|31|ablebarcallybarought| +10603|AAAAAAAALGJCAAAA|1997-10-27||Less considerable communities used to die superior officers. Major contributions raise just with a men|1.09|0.80|2004002|edu packimporto #2|4|sports-apparel|2|Men|374|eseationpri|medium|5670pale929872474688|medium|Pound|Unknown|24|pribarcallybarought| +10604|AAAAAAAAMGJCAAAA|1997-10-27|2000-10-26|Even prime dreams support words. Ministers attempt at the difficulties. Quick women would meet still others. Great, specific decisions accept just top, social merchants. Techn|1.78|0.65|6003005|exporticorp #5|3|gold|6|Jewelry|277|ationationable|N/A|4295585566slate94186|wheat|Pound|Unknown|20|esebarcallybarought| +10605|AAAAAAAAMGJCAAAA|2000-10-27||Quickly french words might not come there. Russian forces want already prime, agri|1.31|0.65|6003005|edu packnameless #6|3|camping|8|Sports|277|ationationable|N/A|4295585566slate94186|peach|Bundle|Unknown|20|antibarcallybarought| +10606|AAAAAAAAOGJCAAAA|1997-10-27|1999-10-27|Also visible hopes play active, contrary conditions. Sections race too to a users. Domestic terms cause generally as a arrangements. New groups|3.80|1.59|10016016|corpamalgamalg #16|16|wireless|10|Electronics|152|ableantiought|N/A|snow3002895118973866|pink|Carton|Unknown|29|callybarcallybarought| +10607|AAAAAAAAOGJCAAAA|1999-10-28|2001-10-26|Also visible hopes play active, contrary conditions. Sections race too to a users. Domestic terms cause generally as a arrangements. New groups|9.46|2.83|10016016|amalgscholar #2|16|rock|5|Music|152|ableantiought|N/A|snow3002895118973866|khaki|Tsp|Unknown|9|ationbarcallybarought| +10608|AAAAAAAAOGJCAAAA|2001-10-27||Also visible hopes play active, contrary conditions. Sections race too to a users. Domestic terms cause generally as a arrangements. New groups|1.01|2.83|3003001|exportiexporti #1|16|toddlers|3|Children|133|pripriought|small|93417350888royal1226|thistle|Bundle|Unknown|19|eingbarcallybarought| +10609|AAAAAAAABHJCAAAA|1997-10-27||More weak months believe today unnecessary sources. Years tread difficult emissions. Intermediate, personal farms could sail as without a causes. New offices illust|1.75|0.77|9005002|scholarmaxi #2|5|history|9|Books|498|eingn stese|N/A|7949wheat72594981095|saddle|Dozen|Unknown|16|n stbarcallybarought| +10610|AAAAAAAACHJCAAAA|1997-10-27|2000-10-26|About electrical solutions might not mark normal, brief directors. Minu|18.07|12.82|1003001|exportiamalg #1|3|maternity|1|Women|360|barcallypri|extra large|07924309430smoke0081|aquamarine|Each|Unknown|35|baroughtcallybarought| +10611|AAAAAAAACHJCAAAA|2000-10-27||True, actual models should go factors. Much wond|6.13|12.82|9009004|maximaxi #4|9|science|9|Books|230|barcallypri|N/A|44753783819823wheat4|white|Bundle|Unknown|1|oughtoughtcallybarought| +10612|AAAAAAAAEHJCAAAA|1997-10-27|1999-10-27|Small agents used to approve most finally simple words. Horses check dangerous, typical cuts. Clear polls can come only around central lines. Perhaps heavy officers tell involved sch|5.88|4.52|7008003|namelessbrand #3|8|lighting|7|Home|255|antiantiable|N/A|977274198043forest78|turquoise|Case|Unknown|4|ableoughtcallybarought| +10613|AAAAAAAAEHJCAAAA|1999-10-28|2001-10-26|Small agents used to approve most finally simple words. Horses check dangerous, typical cuts. Clear polls can come only around central lines. Perhaps heavy officers tell involved sch|9.60|3.36|7008003|importonameless #4|8|baseball|8|Sports|96|antiantiable|N/A|01906338yellow734373|papaya|Ton|Unknown|56|prioughtcallybarought| +10614|AAAAAAAAEHJCAAAA|2001-10-27||Obviously good jobs study nearly feet. Small, major materials must enter; accordingly nice drinks buy more so able weeks. Else progressive titles wa|4.23|1.39|7008003|amalgscholar #1|8|rock|5|Music|96|antiantiable|N/A|36thistle63118352749|rosy|Tsp|Unknown|13|eseoughtcallybarought| +10615|AAAAAAAAHHJCAAAA|1997-10-27||Then you|8.13|3.65|4002002|importoedu pack #2|2|mens|4|Shoes|257|ationantiable|small|07midnight8732536183|royal|Tbl|Unknown|6|antioughtcallybarought| +10616|AAAAAAAAIHJCAAAA|1997-10-27|2000-10-26|More dramatic experts remain organizations. Blue, white grounds used to regard even ready, slow services. Both forward artists shall fall more free territories; welcome|29.49|10.02|1001001|amalgamalg #1|1|dresses|1|Women|420|barableese|medium|8343696hot9405862447|plum|Tsp|Unknown|13|callyoughtcallybarought| +10617|AAAAAAAAIHJCAAAA|2000-10-27||More dramatic experts remain organizations. Blue, white grounds used to regard even ready, slow services. Both forward artists shall fall more free territories; welcome|9.50|5.13|1001001|amalgedu pack #2|1|womens|4|Shoes|420|barableese|medium|21738086724128hot404|slate|Bunch|Unknown|26|ationoughtcallybarought| +10618|AAAAAAAAKHJCAAAA|1997-10-27|1999-10-27|Famous tourists will make. Sensible, potential teams lead armed, democratic types. Social, growing recommendations get in |1.26|1.02|7005009|scholarbrand #9|5|blinds/shades|7|Home|451|oughtantiese|N/A|95ivory3193875967738|turquoise|Cup|Unknown|24|eingoughtcallybarought| +10619|AAAAAAAAKHJCAAAA|1999-10-28|2001-10-26|Famous tourists will make. Sensible, potential teams lead armed, democratic types. Social, growing recommendations get in |1.60|1.23|9006012|corpmaxi #12|5|parenting|9|Books|297|oughtantiese|N/A|95ivory3193875967738|red|Pound|Unknown|32|n stoughtcallybarought| +10620|AAAAAAAAKHJCAAAA|2001-10-27||Famous tourists will make. Sensible, potential teams lead armed, democratic types. Social, growing recommendations get in |5.21|1.23|8006001|corpnameless #1|6|football|8|Sports|162|ablecallyought|N/A|752643953turquoise79|royal|Bunch|Unknown|98|barablecallybarought| +10621|AAAAAAAANHJCAAAA|1997-10-27||True visits show somewhat. Boxes bear much true, new gro|9.27|6.76|1003002|exportiamalg #2|3|maternity|1|Women|55|antianti|medium|powder90301570452450|salmon|Ounce|Unknown|24|oughtablecallybarought| +10622|AAAAAAAAOHJCAAAA|1997-10-27|2000-10-26|Actions ought to ask again centres; young, comprehensive expectations escape of course royal fragments; similar tonnes cannot carry aged a q|2.41|1.78|1004001|edu packamalg #1|4|swimwear|1|Women|147|ationeseought|petite|687791676orange00321|plum|Ton|Unknown|6|ableablecallybarought| +10623|AAAAAAAAOHJCAAAA|2000-10-27||Actions ought to ask again centres; young, comprehensive expectations escape of course royal fragments; similar tonnes cannot carry aged a q|9.22|4.88|1004001|importoedu pack #2|2|mens|4|Shoes|926|callyablen st|medium|687791676orange00321|turquoise|Each|Unknown|30|priablecallybarought| +10624|AAAAAAAAAIJCAAAA|1997-10-27|1999-10-27|Simply small grounds use exactly effects. Services could kill especially aware, large observers. Civil, relevant years ensure regulations; clear drawings realize actors. Products employ a|1.76|0.58|9001005|amalgmaxi #5|1|arts|9|Books|233|pripriable|N/A|14749358burlywood036|pale|Gram|Unknown|19|eseablecallybarought| +10625|AAAAAAAAAIJCAAAA|1999-10-28|2001-10-26|Simply small grounds use exactly effects. Services could kill especially aware, large observers. Civil, relevant years ensure regulations; clear drawings realize actors. Products employ a|5.10|0.58|2004002|edu packimporto #2|4|sports-apparel|2|Men|233|pripriable|petite|14749358burlywood036|slate|Pound|Unknown|27|antiablecallybarought| +10626|AAAAAAAAAIJCAAAA|2001-10-27||Financial, dead studies must not view soon groups. Very, strong parents will find clearly real countries. Simply expensive rocks creep at all |1.72|1.18|5004001|edu packscholar #1|4|classical|5|Music|233|pripriable|N/A|96866131805808olive9|tan|Case|Unknown|4|callyablecallybarought| +10627|AAAAAAAADIJCAAAA|1997-10-27||Clubs shall defeat so extra movements. Ago free books give alternatives. Different makers can|4.41|1.94|4003002|exportiedu pack #2|3|kids|4|Shoes|321|oughtablepri|large|86900181papaya604029|cornflower|Each|Unknown|80|ationablecallybarought| +10628|AAAAAAAAEIJCAAAA|1997-10-27|2000-10-26|Acres help now for a pp.. Marginal exhibitions find men. Also |4.13|2.93|10016004|corpamalgamalg #4|16|wireless|10|Electronics|76|callyation|N/A|6127724945peach74380|dodger|Gram|Unknown|93|eingablecallybarought| +10629|AAAAAAAAEIJCAAAA|2000-10-27||Acres help now for a pp.. Marginal exhibitions find men. Also |2.08|0.89|10016004|importoamalg #2|16|fragrances|1|Women|147|callyation|extra large|6127724945peach74380|white|Bunch|Unknown|94|n stablecallybarought| +10630|AAAAAAAAGIJCAAAA|1997-10-27|1999-10-27|Police should not decline employ|2.32|1.78|1004001|edu packamalg #1|4|swimwear|1|Women|618|eingoughtcally|extra large|grey1433102423600449|sandy|Each|Unknown|30|barpricallybarought| +10631|AAAAAAAAGIJCAAAA|1999-10-28|2001-10-26|Cities draw too assessments. Better separate transactions should find both clothes. Proper, local applications could concentrate blue taxes. Real governments can strik|36.94|1.78|1004001|edu packexporti #2|4|school-uniforms|3|Children|618|eingoughtcally|small|grey1433102423600449|steel|Unknown|Unknown|8|oughtpricallybarought| +10632|AAAAAAAAGIJCAAAA|2001-10-27||Cities draw too assessments. Better separate transactions should find both clothes. Proper, local applications could concentrate blue taxes. Real governments can strik|2.02|0.72|1004001|importocorp #1|4|diamonds|6|Jewelry|618|eingoughtcally|N/A|5pink761141048508162|rosy|Gross|Unknown|54|ablepricallybarought| +10633|AAAAAAAAJIJCAAAA|1997-10-27||Skills might swallow together. Also emotional styles should not address on|8.91|3.74|8016004|corpmaxi #4|16|golf|8|Sports|575|antiationanti|N/A|774510457pink7315759|plum|Bunch|Unknown|1|pripricallybarought| +10634|AAAAAAAAKIJCAAAA|1997-10-27|2000-10-26|Frequent, full walls make young, main pressures; funds contend right thinking parts. Schemes should not perform enough middle nations. Thus red modules dispense. Only individu|2.68|1.42|1001001|amalgamalg #1|1|dresses|1|Women|745|antieseation|small|82594turquoise729963|metallic|Cup|Unknown|17|esepricallybarought| +10635|AAAAAAAAKIJCAAAA|2000-10-27||Thus metropolitan hours set only currently democratic weeks. Also recent poles used to determine free, available boys. Old, small pains ought to carry letters; nevertheless radical projects could ach|2.93|2.43|1001001|amalgimporto #2|1|accessories|2|Men|377|ationationpri|economy|82594turquoise729963|forest|Ton|Unknown|40|antipricallybarought| +10636|AAAAAAAAMIJCAAAA|1997-10-27|1999-10-27|Regular, legal relationships get circumstances. Ever busy rules give up to such as a months. Executives happen ranks. Independent fligh|2.50|1.75|6012005|importobrand #5|12|costume|6|Jewelry|395|antin stpri|N/A|5687olive67234101981|linen|Gross|Unknown|19|callypricallybarought| +10637|AAAAAAAAMIJCAAAA|1999-10-28|2001-10-26|Regular, legal relationships get circumstances. Ever busy rules give up to such as a months. Executives happen ranks. Independent fligh|6.04|1.75|6012005|importoscholar #2|2|country|5|Music|395|antin stpri|N/A|4052287452violet0600|puff|Gram|Unknown|17|ationpricallybarought| +10638|AAAAAAAAMIJCAAAA|2001-10-27||Successful strings shall choose interesting, universal supporters. Firms co|2.64|0.89|6012005|edu packbrand #9|4|curtains/drapes|7|Home|395|antin stpri|N/A|4052287452violet0600|orchid|Bunch|Unknown|6|eingpricallybarought| +10639|AAAAAAAAPIJCAAAA|1997-10-27||Matters feel even gastric others. Notes may break also international, head powers. Then contemporary rounds start literally reason|9.85|4.92|4004002|edu packedu pack #2|4|athletic|4|Shoes|293|prin stable|small|62955917698tan328208|purple|Gram|Unknown|16|n stpricallybarought| +10640|AAAAAAAAAJJCAAAA|1997-10-27|2000-10-26|Once old minutes must not contain in a occasions. Economically old shelves could go con|2.15|1.22|4003001|exportiedu pack #1|3|kids|4|Shoes|10|barought|large|4872grey422253754007|puff|Pound|Unknown|13|baresecallybarought| +10641|AAAAAAAAAJJCAAAA|2000-10-27||Fundamental sections interpret more much full feet. Constant babies introduce long single legs. French, remaining lives stop|6.33|1.22|4003001|importoscholar #2|2|country|5|Music|10|barought|N/A|4872grey422253754007|plum|Dozen|Unknown|39|oughtesecallybarought| +10642|AAAAAAAACJJCAAAA|1997-10-27|1999-10-27|Men would not welcome sure very rem|60.55|29.66|7001003|amalgbrand #3|1|bathroom|7|Home|392|ablen stpri|N/A|52steel8722089421935|pink|Ton|Unknown|10|ableesecallybarought| +10643|AAAAAAAACJJCAAAA|1999-10-28|2001-10-26|Men would not welcome sure very rem|4.56|1.77|5002002|importoscholar #2|2|country|5|Music|392|ablen stpri|N/A|52steel8722089421935|red|Pallet|Unknown|39|priesecallybarought| +10644|AAAAAAAACJJCAAAA|2001-10-27||Apparent, sure levels could produce sorry, free ti|3.23|1.77|6012005|importobrand #5|2|costume|6|Jewelry|15|antiought|N/A|0papaya5667454442260|spring|Tsp|Unknown|25|eseesecallybarought| +10645|AAAAAAAAFJJCAAAA|1997-10-27||Key industries print closely elegant households. Accounts clear only to a prisoners. Certain incentives reach. Keen animals deny directly telecommunications; internationa|2.80|1.09|8005006|scholarnameless #6|5|fitness|8|Sports|89|n steing|N/A|969252048tomato28549|medium|Box|Unknown|33|antiesecallybarought| +10646|AAAAAAAAGJJCAAAA|1997-10-27|2000-10-26|Regional, political changes accept yet friends. There high territories may rely of course various p|4.69|2.39|5004001|edu packscholar #1|4|classical|5|Music|449|n steseese|N/A|6orange4209631160036|misty|Carton|Unknown|29|callyesecallybarought| +10647|AAAAAAAAGJJCAAAA|2000-10-27||Books enter effectively financial societies. Causes exist briskly all new sessions. Appropriate|27.58|2.39|3004002|edu packexporti #2|4|school-uniforms|3|Children|620|n steseese|medium|7722317ghost30953475|violet|Tbl|Unknown|6|ationesecallybarought| +10648|AAAAAAAAIJJCAAAA|1997-10-27|1999-10-27|Euro|3.01|2.58|9002005|importomaxi #5|2|business|9|Books|621|oughtablecally|N/A|708836335515powder60|red|Gram|Unknown|29|eingesecallybarought| +10649|AAAAAAAAIJJCAAAA|1999-10-28|2001-10-26|Then small metres |5.65|2.58|1003002|exportiamalg #2|2|maternity|1|Women|621|oughtablecally|medium|708836335515powder60|tan|Ton|Unknown|29|n stesecallybarought| +10650|AAAAAAAAIJJCAAAA|2001-10-27||At all wide soldiers learn alternatively too p|4.23|3.59|1002001|importoamalg #1|2|fragrances|1|Women|621|oughtablecally|large|708836335515powder60|plum|Ounce|Unknown|10|baranticallybarought| +10651|AAAAAAAALJJCAAAA|1997-10-27||Philosophical, financial consultants must see indeed regions. Random notes shall not understand chemical, recent companies. Roads absorb. Grand, environmen|4.75|1.61|3002002|importoexporti #2|2|infants|3|Children|628|eingablecally|medium|6259730505saddle3135|sky|Carton|Unknown|24|oughtanticallybarought| +10652|AAAAAAAAMJJCAAAA|1997-10-27|2000-10-26|Forei|0.55|0.44|4004001|edu packedu pack #1|4|athletic|4|Shoes|118|eingoughtought|medium|29539892salmon610018|yellow|Bundle|Unknown|52|ableanticallybarought| +10653|AAAAAAAAMJJCAAAA|2000-10-27||New, afraid eyes could not define as disciplinary, certain feelings. National thoughts would not head high drawings. Countries eat twice parliamentary, whole films. Hearts will not provide with a |9.87|0.44|4004001|exportinameless #4|13|wallpaper|7|Home|118|eingoughtought|N/A|1091669072pale068230|papaya|Each|Unknown|84|prianticallybarought| +10654|AAAAAAAAOJJCAAAA|1997-10-27|1999-10-27|Troops take only, right dogs. Briefly genuine eyes used to provide mutually coming, just parents. Too social services shall feel only rec|6.40|5.37|9001011|amalgmaxi #11|1|arts|9|Books|218|eingoughtable|N/A|goldenrod17650380908|tomato|Pound|Unknown|65|eseanticallybarought| +10655|AAAAAAAAOJJCAAAA|1999-10-28|2001-10-26|Troops take only, right dogs. Briefly genuine eyes used to provide mutually coming, just parents. Too social services shall feel only rec|4.26|3.40|9001011|importounivamalg #14|1|camcorders|10|Electronics|29|eingoughtable|N/A|goldenrod17650380908|sandy|Unknown|Unknown|15|antianticallybarought| +10656|AAAAAAAAOJJCAAAA|2001-10-27||Separate years ought to contain aga|2.40|1.99|9015001|scholarunivamalg #1|15|fiction|9|Books|196|callyn stought|N/A|goldenrod17650380908|turquoise|Cup|Unknown|21|callyanticallybarought| +10657|AAAAAAAABKJCAAAA|1997-10-27||Particular recordings will refer a|1.76|1.16|2001002|amalgimporto #2|1|accessories|2|Men|519|n stoughtanti|medium|18721smoke0993018650|spring|N/A|Unknown|22|ationanticallybarought| +10658|AAAAAAAACKJCAAAA|1997-10-27|2000-10-26|Possibilities should not fit almost eggs; seriously little members del|3.40|1.08|9007003|brandmaxi #3|7|reference|9|Books|187|ationeingought|N/A|99213962508powder530|rosy|Ton|Unknown|4|einganticallybarought| +10659|AAAAAAAACKJCAAAA|2000-10-27||Professional examples will not watch perhaps s|74.03|1.08|9007003|amalgedu pack #2|7|womens|4|Shoes|485|antieingese|medium|6803040spring0700026|pale|Gross|Unknown|42|n stanticallybarought| +10660|AAAAAAAAEKJCAAAA|1997-10-27|1999-10-27|Twice|6.89|3.58|2003001|exportiimporto #1|3|pants|2|Men|242|ableeseable|extra large|tomato19615816829507|violet|Ton|Unknown|13|barcallycallybarought| +10661|AAAAAAAAEKJCAAAA|1999-10-28|2001-10-26|Facilities can draw days; days can take. More current stages should contribute in the methods. H|0.64|0.55|2003001|corpmaxi #2|3|golf|8|Sports|242|ableeseable|N/A|578purple62383398292|tan|Case|Unknown|3|oughtcallycallybarought| +10662|AAAAAAAAEKJCAAAA|2001-10-27||Experiments can avoid young, healthy questions; public factors shall make junior, environmental reasons. Again electoral influence|1.98|1.22|2003001|scholarunivamalg #4|3|karoke|10|Electronics|72|ableation|N/A|578purple62383398292|rosy|Unknown|Unknown|35|ablecallycallybarought| +10663|AAAAAAAAHKJCAAAA|1997-10-27||However short-term parties create thanks; exotic, normal nerves see. New, healthy machines can satisfy possibly new positions. Completely internal signs|5.52|1.76|8002004|importonameless #4|2|baseball|8|Sports|50|baranti|N/A|48613060169puff00077|burnished|Dram|Unknown|29|pricallycallybarought| +10664|AAAAAAAAIKJCAAAA|1997-10-27|2000-10-26|Vocational, economic years should remain. Systems could keep badly characters. Representative operations provide poor, future devices. Applications feel good average a|3.40|1.83|5002001|importoscholar #1|2|country|5|Music|89|n steing|N/A|puff7163819149168754|pink|Each|Unknown|2|esecallycallybarought| +10665|AAAAAAAAIKJCAAAA|2000-10-27||Vocational, economic years should remain. Systems could keep badly characters. Representative operations provide poor, future devices. Applications feel good average a|3.19|1.83|5002001|scholarmaxi #10|2|history|9|Books|230|barpriable|N/A|49022140medium897304|sienna|Tbl|Unknown|58|anticallycallybarought| +10666|AAAAAAAAKKJCAAAA|1997-10-27|1999-10-27|Wages|5.92|2.36|9007011|brandmaxi #11|7|reference|9|Books|453|priantiese|N/A|81beige7484277139334|slate|Box|Unknown|43|callycallycallybarought| +10667|AAAAAAAAKKJCAAAA|1999-10-28|2001-10-26|Wages|3.89|1.78|9007011|amalgamalg #2|7|dresses|1|Women|453|priantiese|medium|117yellow37826779941|navy|Unknown|Unknown|30|ationcallycallybarought| +10668|AAAAAAAAKKJCAAAA|2001-10-27||Historic, psychiat|4.42|2.34|4001001|amalgedu pack #1|1|womens|4|Shoes|453|priantiese|petite|117yellow37826779941|thistle|N/A|Unknown|66|eingcallycallybarought| +10669|AAAAAAAANKJCAAAA|1997-10-27||Questions see by a representatives. Short questions pass respectively progressive pp.. Sufficiently |27.90|18.97|7007010|brandbrand #10|7|decor|7|Home|519|n stoughtanti|N/A|1842444204826olive58|seashell|Pallet|Unknown|54|n stcallycallybarought| +10670|AAAAAAAAOKJCAAAA|1997-10-27|2000-10-26|So helpful problems could not|2.30|2.02|1004001|edu packamalg #1|4|swimwear|1|Women|560|barcallyanti|medium|6483788259plum892398|sienna|Tbl|Unknown|9|barationcallybarought| +10671|AAAAAAAAOKJCAAAA|2000-10-27||Models visit alternative, poor kids. Grounds would not leave separate, thinking communities. Almost public knee|3.76|3.27|7006002|corpbrand #2|4|rugs|7|Home|560|barcallyanti|N/A|6483788259plum892398|goldenrod|Dram|Unknown|14|oughtationcallybarought| +10672|AAAAAAAAALJCAAAA|1997-10-27|1999-10-27|Roughly green aspects like closer. Associated, content|1.44|0.93|1001001|amalgamalg #1|1|dresses|1|Women|984|eseeingn st|large|12373188523royal5295|plum|Case|Unknown|33|ableationcallybarought| +10673|AAAAAAAAALJCAAAA|1999-10-28|2001-10-26|Roughly green aspects like closer. Associated, content|0.67|0.32|1001001|importoamalg #2|1|fragrances|1|Women|212|ableoughtable|petite|6026olive56231400713|chocolate|Dram|Unknown|64|priationcallybarought| +10674|AAAAAAAAALJCAAAA|2001-10-27||So as real friends ruin similarly costly types. Teachers can use then chapters. More than extreme centuries could not develop major, current organisations; internal customers become simple, new tee|7.42|4.30|7003005|exportibrand #5|3|kids|7|Home|212|ableoughtable|N/A|6026olive56231400713|turquoise|Box|Unknown|29|eseationcallybarought| +10675|AAAAAAAADLJCAAAA|1997-10-27||Major authorities ought to penetrate so banks. Bills will |9.36|2.80|7010006|univnameless #6|10|flatware|7|Home|37|ationpri|N/A|74025414royal7840202|tan|Dozen|Unknown|6|antiationcallybarought| +10676|AAAAAAAAELJCAAAA|1997-10-27|2000-10-26|Classical, pink hills may not cope ever silver couples. Whole things ought to develop chemical rules; then other words o|5.62|2.81|2001001|amalgimporto #1|1|accessories|2|Men|986|callyeingn st|petite|0505897033552misty58|green|Ounce|Unknown|7|callyationcallybarought| +10677|AAAAAAAAELJCAAAA|2000-10-27||Necessary institutions |1.23|0.89|2001001|univamalgamalg #2|10|memory|10|Electronics|986|callyeingn st|N/A|0505897033552misty58|plum|Lb|Unknown|2|ationationcallybarought| +10678|AAAAAAAAGLJCAAAA|1997-10-27|1999-10-27|Complex names let significantly royal, equal photographs. As intense troops abandon then. So prime children could not make to a men.|8.73|4.19|3001001|amalgexporti #1|1|newborn|3|Children|570|barationanti|large|3wheat52486795879414|steel|Oz|Unknown|18|eingationcallybarought| +10679|AAAAAAAAGLJCAAAA|1999-10-28|2001-10-26|Strategic, western points could not supply modern suggestions|6.23|4.19|3001001|univbrand #8|1|jewelry boxes|6|Jewelry|198|barationanti|N/A|3wheat52486795879414|plum|Dozen|Unknown|8|n stationcallybarought| +10680|AAAAAAAAGLJCAAAA|2001-10-27||Strategic, western points could not supply modern suggestions|1.48|4.19|3001001|amalgnameless #1|1|accent|7|Home|198|eingn stought|N/A|9light73580476450280|sienna|Pound|Unknown|36|bareingcallybarought| +10681|AAAAAAAAJLJCAAAA|1997-10-27||Fully complete points play sacred steps. Anxiously french claims want about a years. Green points apply individuals. Exciting, medical parties make previous elections. Poor t|1.23|0.82|1004002|edu packamalg #2|4|swimwear|1|Women|177|ationationought|large|09529370816steel3194|seashell|Gross|Unknown|17|oughteingcallybarought| +10682|AAAAAAAAKLJCAAAA|1997-10-27|2000-10-26|Stairs will stop in a aspects. Thereby monthly elections may twist so in a offices. All the same free designs fly wonderful, worthwhile winners. Members |4.88|2.68|3003001|exportiexporti #1|3|toddlers|3|Children|90|barn st|extra large|5887226077khaki57725|tan|Each|Unknown|3|ableeingcallybarought| +10683|AAAAAAAAKLJCAAAA|2000-10-27||Simple, nice levels succeed|3.57|2.68|9011004|amalgunivamalg #4|3|cooking|9|Books|140|bareseought|N/A|8739643735341slate63|royal|Pound|Unknown|48|prieingcallybarought| +10684|AAAAAAAAMLJCAAAA|1997-10-27|1999-10-27|Rather psychiatric points dare difficult benefits. At first scottish writings may not |2.92|2.30|6005005|scholarcorp #5|5|earings|6|Jewelry|101|oughtbarought|N/A|489629047peach059741|indian|Box|Unknown|47|eseeingcallybarought| +10685|AAAAAAAAMLJCAAAA|1999-10-28|2001-10-26|Western, radical fields ought to emerge too conscious values. So forthcoming equations should get ab|0.18|0.09|6005005|amalgscholar #2|5|rock|5|Music|101|oughtbarought|N/A|489629047peach059741|tan|Each|Unknown|51|antieingcallybarought| +10686|AAAAAAAAMLJCAAAA|2001-10-27||Economic objects can know scarcely constant notes. Methods generate all st|0.65|0.51|6005005|corpunivamalg #7|16|mystery|9|Books|101|oughtbarought|N/A|489629047peach059741|goldenrod|Pallet|Unknown|8|callyeingcallybarought| +10687|AAAAAAAAPLJCAAAA|1997-10-27||Mainly alone trees would join quite military projects. Unexpected, royal developments would agree today then good cups. Very foreign representatives show necessarily similar costs. Rele|3.34|1.70|8010008|univmaxi #8|10|pools|8|Sports|47|ationese|N/A|202376442287lime3858|turquoise|Box|Unknown|47|ationeingcallybarought| +10688|AAAAAAAAAMJCAAAA|1997-10-27|2000-10-26|Perfect, whole christians shall develop reforms. Labour councils would not use careful, other tensions. Wild sist|3.50|2.24|1003001|exportiamalg #1|3|maternity|1|Women|591|oughtn stanti|small|9173wheat86101713543|ghost|Dozen|Unknown|4|eingeingcallybarought| +10689|AAAAAAAAAMJCAAAA|2000-10-27||Chiefs can receive necessary movements. R|9.06|2.24|1003001|edu packamalg #2|3|swimwear|1|Women|591|oughtn stanti|medium|9173wheat86101713543|pale|Box|Unknown|64|n steingcallybarought| +10690|AAAAAAAACMJCAAAA|1997-10-27|1999-10-27|National thousands overthrow all right term|5.57|1.94|5003001|exportischolar #1|3|pop|5|Music|694|esen stcally|N/A|861833274orange88830|turquoise|Ounce|Unknown|43|barn stcallybarought| +10691|AAAAAAAACMJCAAAA|1999-10-28|2001-10-26|Sufficient, possible parents regard effects. Visual requirements intervene more. Other clothes would not find. Risks|0.85|1.94|5003001|amalgimporto #2|3|accessories|2|Men|694|esen stcally|extra large|861833274orange88830|navajo|N/A|Unknown|84|oughtn stcallybarought| +10692|AAAAAAAACMJCAAAA|2001-10-27||Sufficient, possible parents regard effects. Visual requirements intervene more. Other clothes would not find. Risks|4.69|1.94|5003001|amalgamalg #1|3|dresses|1|Women|694|esen stcally|extra large|861833274orange88830|peru|Tsp|Unknown|89|ablen stcallybarought| +10693|AAAAAAAAFMJCAAAA|1997-10-27||Equivalent, perfect remains could not find more mines. Recent schools commemorate situations; |4.26|2.98|6009006|maxicorp #6|9|womens watch|6|Jewelry|284|eseeingable|N/A|metallic019390928795|papaya|Tbl|Unknown|38|prin stcallybarought| +10694|AAAAAAAAGMJCAAAA|1997-10-27|2000-10-26|Almost easy years used to breed in a years. Ideas will miss through a facilities|1.49|0.84|6001007|amalgcorp #7|1|birdal|6|Jewelry|11|oughtought|N/A|coral428622087755794|olive|Gram|Unknown|41|esen stcallybarought| +10695|AAAAAAAAGMJCAAAA|2000-10-27||Almost easy years used to breed in a years. Ideas will miss through a facilities|5.43|4.66|5001002|amalgscholar #2|1|rock|5|Music|255|antiantiable|N/A|coral428622087755794|gainsboro|Cup|Unknown|8|antin stcallybarought| +10696|AAAAAAAAIMJCAAAA|1997-10-27|1999-10-27|Usually different views shall serve personally unknown symbols. Countries prove methods. Necessary men consider also to a communications. Always inner hundreds will not share suddenly from a shops. P|8.94|4.02|9006005|corpmaxi #5|6|parenting|9|Books|138|eingpriought|N/A|94324441blush4993376|pink|Pallet|Unknown|2|callyn stcallybarought| +10697|AAAAAAAAIMJCAAAA|1999-10-28|2001-10-26|Miserable gates show offices. Stages ought to start home physical, easy schools. Types must stare for a films. Of course convenient materials come absolutely. Movin|8.03|4.02|1004002|edu packamalg #2|6|swimwear|1|Women|138|eingpriought|extra large|4520294364rosy286874|turquoise|Ton|Unknown|5|ationn stcallybarought| +10698|AAAAAAAAIMJCAAAA|2001-10-27||Clear, cheap interests repair; males set as military materials. Sorts may not quan|7.48|4.78|6001003|amalgcorp #3|1|birdal|6|Jewelry|837|ationprieing|N/A|4520294364rosy286874|rosy|Unknown|Unknown|43|eingn stcallybarought| +10699|AAAAAAAALMJCAAAA|1997-10-27||Beautiful incomes could not spread apart wooden talks. Hopefully short individuals might say stil|4.48|2.95|7002010|importobrand #10|2|bedding|7|Home|65|antically|N/A|849888657273ivory388|rosy|Dozen|Unknown|55|n stn stcallybarought| +10700|AAAAAAAAMMJCAAAA|1997-10-27|2000-10-26|Sexual strangers should eat around horrible observations. Applications |6.23|5.29|7008001|namelessbrand #1|8|lighting|7|Home|475|antiationese|N/A|34sandy4818196891609|honeydew|Box|Unknown|45|barbarationbarought| +10701|AAAAAAAAMMJCAAAA|2000-10-27||Possible, active number|4.71|1.64|7008001|exportiexporti #2|3|toddlers|3|Children|475|antiationese|petite|34sandy4818196891609|snow|Tbl|Unknown|7|oughtbarationbarought| +10702|AAAAAAAAOMJCAAAA|1997-10-27|1999-10-27|Good ships get young points. Rarely extra countries like. Women rise better. Further permanent representatives ought to say substantial buildings. Less typical pre|4.76|2.33|9007011|brandmaxi #11|7|reference|9|Books|326|callyablepri|N/A|3957186117074saddle8|royal|Carton|Unknown|26|ablebarationbarought| +10703|AAAAAAAAOMJCAAAA|1999-10-28|2001-10-26|Good ships get young points. Rarely extra countries like. Women rise better. Further permanent representatives ought to say substantial buildings. Less typical pre|7.08|3.25|9007011|edu packedu pack #2|7|athletic|4|Shoes|452|ableantiese|medium|3957186117074saddle8|misty|Lb|Unknown|5|pribarationbarought| +10704|AAAAAAAAOMJCAAAA|2001-10-27||Public, average galleries could take. Old, general ships say in the horses. Full standards set actually; high losses discuss home only sure documents. Of course jewish designs shall not know |1.82|3.25|9007011|namelessmaxi #1|8|romance|9|Books|292|ableantiese|N/A|3957186117074saddle8|lace|Tbl|Unknown|29|esebarationbarought| +10705|AAAAAAAABNJCAAAA|1997-10-27||At least assistant bands can address certainly black trees. Terms ought to knock ex|0.49|0.26|9007002|brandmaxi #2|7|reference|9|Books|331|oughtpripri|N/A|94835goldenrod699157|navajo|Lb|Unknown|61|antibarationbarought| +10706|AAAAAAAACNJCAAAA|1997-10-27|2000-10-26|Very federal years relate thus thin, aware transactions. Good, similar ideas like clearly national, direct forces; british|77.22|37.83|3001001|amalgexporti #1|1|newborn|3|Children|597|ationn stanti|large|396003smoke889498272|maroon|N/A|Unknown|25|callybarationbarought| +10707|AAAAAAAACNJCAAAA|2000-10-27||Very federal years relate thus thin, aware transactions. Good, similar ideas like clearly national, direct forces; british|7.07|3.95|10003015|exportiunivamalg #15|3|dvd/vcr players|10|Electronics|597|ationn stanti|N/A|11939337966blanched0|ghost|Case|Unknown|54|ationbarationbarought| +10708|AAAAAAAAENJCAAAA|1997-10-27|1999-10-27|Bad, parliamentary things can burst then back, high prices. New children shall not prefer quietly different leaders. Home assistant refe|3.77|3.16|10004016|edu packunivamalg #16|4|audio|10|Electronics|285|antieingable|N/A|6605105cornflower183|turquoise|Gross|Unknown|75|eingbarationbarought| +10709|AAAAAAAAENJCAAAA|1999-10-28|2001-10-26|Most subsequent plants relieve always substantial steps. Clearly future topics may move seriously on a materials. Guests use average efforts. En|2.56|1.81|3004002|edu packexporti #2|4|school-uniforms|3|Children|323|priablepri|large|6605105cornflower183|moccasin|Each|Unknown|12|n stbarationbarought| +10710|AAAAAAAAENJCAAAA|2001-10-27||Important, intense words get. Fina|52.70|1.81|3003001|exportiexporti #1|4|toddlers|3|Children|502|priablepri|petite|98879107thistle00667|peru|Unknown|Unknown|64|baroughtationbarought| +10711|AAAAAAAAHNJCAAAA|1997-10-27||M|2.64|2.32|8006002|corpnameless #2|6|football|8|Sports|506|callybaranti|N/A|43260835371312green2|purple|Oz|Unknown|47|oughtoughtationbarought| +10712|AAAAAAAAINJCAAAA|1997-10-27|2000-10-26|Considerations would bear n|3.36|2.48|5004001|edu packscholar #1|4|classical|5|Music|197|ationn stought|N/A|13838108wheat8321279|tomato|Dram|Unknown|9|ableoughtationbarought| +10713|AAAAAAAAINJCAAAA|2000-10-27||High circumstances used to l|2.20|1.54|10001004|amalgunivamalg #4|4|cameras|10|Electronics|197|ationn stought|N/A|13838108wheat8321279|snow|Ounce|Unknown|13|prioughtationbarought| +10714|AAAAAAAAKNJCAAAA|1997-10-27|1999-10-27|Economic, scientific schemes talk significant, social hands. Interior cases|3.15|2.14|3003001|exportiexporti #1|3|toddlers|3|Children|49|n stese|petite|metallic992093506613|tomato|Lb|Unknown|55|eseoughtationbarought| +10715|AAAAAAAAKNJCAAAA|1999-10-28|2001-10-26|Good, concerned minds protect off the difficulties. Dogs shall not know records. Civil, typical windows say lists. Indian days see social, sufficient schools. Expectations make more in|0.39|2.14|3003001|exportiunivamalg #6|3|dvd/vcr players|10|Electronics|16|n stese|N/A|metallic992093506613|tomato|Ton|Unknown|17|antioughtationbarought| +10716|AAAAAAAAKNJCAAAA|2001-10-27||Good, concerned minds protect off the difficulties. Dogs shall not know records. Civil, typical windows say lists. Indian days see social, sufficient schools. Expectations make more in|3.40|1.25|9015001|scholarunivamalg #1|15|fiction|9|Books|16|n stese|N/A|87802649803grey60468|spring|Lb|Unknown|36|callyoughtationbarought| +10717|AAAAAAAANNJCAAAA|1997-10-27||Good, public systems should act very top trees. Monetary, determined words could alleviate then hills. |26.29|15.24|8010008|univmaxi #8|10|pools|8|Sports|466|callycallyese|N/A|2tomato1000894840626|papaya|Dram|Unknown|4|ationoughtationbarought| +10718|AAAAAAAAONJCAAAA|1997-10-27|2000-10-26|Especially other parts could make over blank choices; subjects constrain only social, new respects. Brown, particular reports m|6.82|2.52|7001009|amalgbrand #9|1|bathroom|7|Home|177|ationationought|N/A|1660rosy837910595631|light|Bundle|Unknown|15|eingoughtationbarought| +10719|AAAAAAAAONJCAAAA|2000-10-27||Effective, difficult restrictions l|86.86|2.52|5001002|amalgscholar #2|1|rock|5|Music|177|ationationought|N/A|0913027276red7206509|plum|Ounce|Unknown|60|n stoughtationbarought| +10720|AAAAAAAAAOJCAAAA|1997-10-27|1999-10-27|Industrial instances get about; reforms could abuse original projects; servants render here in the sales; however medium rights explain only simple techniques. Exchanges give. All|2.52|2.04|6014001|edu packbrand #1|14|estate|6|Jewelry|130|barpriought|N/A|8434885357yellow0211|misty|Carton|Unknown|50|barableationbarought| +10721|AAAAAAAAAOJCAAAA|1999-10-28|2001-10-26|Industrial instances get about; reforms could abuse original projects; servants render here in the sales; however medium rights explain only simple techniques. Exchanges give. All|1.81|2.04|2003002|exportiimporto #2|14|pants|2|Men|884|eseeingeing|small|4714007turquoise8069|pale|Gram|Unknown|45|oughtableationbarought| +10722|AAAAAAAAAOJCAAAA|2001-10-27||Industrial instances get about; reforms could abuse original projects; servants render here in the sales; however medium rights explain only simple techniques. Exchanges give. All|2.04|2.04|2003002|maxinameless #3|14|optics|8|Sports|884|eseeingeing|N/A|18735768698589rose11|smoke|Carton|Unknown|10|ableableationbarought| +10723|AAAAAAAADOJCAAAA|1997-10-27||Animals shall participate evenly to a findings. Ma|2.04|0.89|2004002|edu packimporto #2|4|sports-apparel|2|Men|176|callyationought|petite|8steel43045506927324|papaya|N/A|Unknown|27|priableationbarought| +10724|AAAAAAAAEOJCAAAA|1997-10-27|2000-10-26|Patients must come generally popular developments. Systematically great languages find single, specific thoughts. About central years might hear fully. Clients |0.42|0.22|3003001|exportiexporti #1|3|toddlers|3|Children|662|ablecallycally|medium|49947wheat0737387553|gainsboro|Gross|Unknown|29|eseableationbarought| +10725|AAAAAAAAEOJCAAAA|2000-10-27||Patients must come generally popular developments. Systematically great languages find single, specific thoughts. About central years might hear fully. Clients |3.30|2.93|3003001|edu packamalg #2|3|swimwear|1|Women|273|ablecallycally|medium|80171778puff61066947|magenta|Each|Unknown|3|antiableationbarought| +10726|AAAAAAAAGOJCAAAA|1997-10-27|1999-10-27|Different, asleep children must see at once to a patients. However important organisers agre|1.35|1.06|6008007|namelesscorp #7|8|mens watch|6|Jewelry|144|eseeseought|N/A|82576470steel0710948|powder|Tsp|Unknown|25|callyableationbarought| +10727|AAAAAAAAGOJCAAAA|1999-10-28|2001-10-26|Imm|2.84|1.06|6008007|importobrand #8|2|bedding|7|Home|229|n stableable|N/A|5413087937601orchid1|mint|Bunch|Unknown|17|ationableationbarought| +10728|AAAAAAAAGOJCAAAA|2001-10-27||Often damp fields achieve both deliberate quantities; well new forces reach hungry, e|6.56|1.06|10012002|importoamalgamalg #2|2|monitors|10|Electronics|27|n stableable|N/A|5413087937601orchid1|magenta|Lb|Unknown|20|eingableationbarought| +10729|AAAAAAAAJOJCAAAA|1997-10-27||Personal authorities may admire again little men. Current students should dwell very fav|3.03|1.96|6005002|scholarcorp #2|5|earings|6|Jewelry|18|eingought|N/A|saddle66293560807979|sienna|Gross|Unknown|30|n stableationbarought| +10730|AAAAAAAAKOJCAAAA|1997-10-27|2000-10-26|Economies increase sometimes consultants; available, special emissions need otherwise. Patients say eve|2.41|1.97|4003001|exportiedu pack #1|3|kids|4|Shoes|208|eingbarable|medium|445purple74344623081|ivory|Bunch|Unknown|72|barpriationbarought| +10731|AAAAAAAAKOJCAAAA|2000-10-27||Foreign details should not leave about a parents. Independent runs should not receive good, private events. Moving leads |0.10|0.07|4003001|importoimporto #2|2|shirts|2|Men|208|eingbarable|N/A|445purple74344623081|turquoise|Lb|Unknown|53|oughtpriationbarought| +10732|AAAAAAAAMOJCAAAA|1997-10-27|1999-10-27|Clear teache|0.77|0.46|6006005|corpcorp #5|6|rings|6|Jewelry|23|priable|N/A|557595384slate363408|snow|Oz|Unknown|51|ablepriationbarought| +10733|AAAAAAAAMOJCAAAA|1999-10-28|2001-10-26|Clear teache|2.06|1.64|9014006|edu packunivamalg #6|6|sports|9|Books|574|priable|N/A|22129659873peru66046|white|Gross|Unknown|64|pripriationbarought| +10734|AAAAAAAAMOJCAAAA|2001-10-27||Clear teache|1.01|1.64|10005008|scholarunivamalg #8|5|karoke|10|Electronics|971|oughtationn st|N/A|67280rosy68316538139|frosted|Tsp|Unknown|48|esepriationbarought| +10735|AAAAAAAAPOJCAAAA|1997-10-27||Advanced, certain fields miss electronically for the books. Open measures match therefore s|3.97|1.70|2003002|exportiimporto #2|3|pants|2|Men|164|esecallyought|extra large|059424powder53170076|puff|Tbl|Unknown|20|antipriationbarought| +10736|AAAAAAAAAPJCAAAA|1997-10-27|2000-10-26|Nuclear leaders might anticipate yesterday different, respectable visitors. Still general households could not admit now or|6.09|2.07|5004001|edu packscholar #1|4|classical|5|Music|296|callyn stable|N/A|3pale366491847946986|lime|Bundle|Unknown|62|callypriationbarought| +10737|AAAAAAAAAPJCAAAA|2000-10-27||Open, regulatory neighbours could not want ill permanent magistrates; multiple, old goods must work pressures. Comfortable, essent|0.91|0.66|10011011|amalgamalgamalg #11|11|disk drives|10|Electronics|28|callyn stable|N/A|3pale366491847946986|lawn|Unknown|Unknown|45|ationpriationbarought| +10738|AAAAAAAACPJCAAAA|1997-10-27|1999-10-27|Schools would not show. Socialist, international houses|9.02|8.11|4003001|exportiedu pack #1|3|kids|4|Shoes|119|n stoughtought|medium|42367purple555391807|saddle|N/A|Unknown|58|eingpriationbarought| +10739|AAAAAAAACPJCAAAA|1999-10-28|2001-10-26|Strong millions cannot get widely pure weeks; other, other talks know as well. Also |2.46|8.11|8010010|univmaxi #10|10|pools|8|Sports|294|n stoughtought|N/A|67steel5512166979276|peach|Tsp|Unknown|41|n stpriationbarought| +10740|AAAAAAAACPJCAAAA|2001-10-27||Metres must not explain as as political s|2.53|0.93|5004001|edu packscholar #1|4|classical|5|Music|265|anticallyable|N/A|67steel5512166979276|medium|Bunch|Unknown|17|bareseationbarought| +10741|AAAAAAAAFPJCAAAA|1997-10-27||Correct, huge options could give quickly around the governments. Real options get deeply social, dea|4.74|1.75|2001002|amalgimporto #2|1|accessories|2|Men|73|priation|medium|97847084444431pale38|rosy|Dram|Unknown|60|oughteseationbarought| +10742|AAAAAAAAGPJCAAAA|1997-10-27|2000-10-26|Very, great fingers shall not receive open experiences. Back years grow extensive, eng|9.36|4.96|7016003|corpnameless #3|16|furniture|7|Home|321|oughtablepri|N/A|82567318almond436495|tan|Dram|Unknown|51|ableeseationbarought| +10743|AAAAAAAAGPJCAAAA|2000-10-27||Very, great fingers shall not receive open experiences. Back years grow extensive, eng|0.61|4.96|7016003|amalgexporti #2|1|newborn|3|Children|209|n stbarable|small|82567318almond436495|seashell|Tbl|Unknown|100|prieseationbarought| +10744|AAAAAAAAIPJCAAAA|1997-10-27|1999-10-27|Available companies identify quickly available, wrong visitors. Pu|86.52|37.20|5003001|exportischolar #1|3|pop|5|Music|471|oughtationese|N/A|052903rose1232204320|burlywood|Dram|Unknown|23|eseeseationbarought| +10745|AAAAAAAAIPJCAAAA|1999-10-28|2001-10-26|Available companies identify quickly available, wrong visitors. Pu|6.99|37.20|8008006|namelessnameless #6|3|outdoor|8|Sports|271|oughtationable|N/A|052903rose1232204320|tan|Gram|Unknown|20|antieseationbarought| +10746|AAAAAAAAIPJCAAAA|2001-10-27||Methods could survive hardly far clear minds. Major, important arms sign only with a cases. Right participants come here at a books. Legal materials might enjoy |1.91|37.20|5003001|exportischolar #1|3|pop|5|Music|192|ablen stought|N/A|546925429108884dim87|sky|Oz|Unknown|72|callyeseationbarought| +10747|AAAAAAAALPJCAAAA|1997-10-27||So new reactions want basic forces. Quite quiet sounds cause casually. Central developments ought to withhold terms. Really|32.73|26.51|10007004|brandunivamalg #4|7|personal|10|Electronics|174|eseationought|N/A|7073139304navy028132|turquoise|Ton|Unknown|38|ationeseationbarought| +10748|AAAAAAAAMPJCAAAA|1997-10-27|2000-10-26|Significant, high reforms sit totally. As equal races can produce ago as young values. Only industrial gardens must not vote stories. Suitable, patient issues might not get entirely military pl|3.14|1.25|3004001|edu packexporti #1|4|school-uniforms|3|Children|649|n stesecally|medium|1643198070296violet4|seashell|Carton|Unknown|44|eingeseationbarought| +10749|AAAAAAAAMPJCAAAA|2000-10-27||Significant, high reforms sit totally. As equal races can produce ago as young values. Only industrial gardens must not vote stories. Suitable, patient issues might not get entirely military pl|3.53|1.25|7001010|amalgbrand #10|1|bathroom|7|Home|649|n stesecally|N/A|732royal371641646834|moccasin|Dram|Unknown|14|n steseationbarought| +10750|AAAAAAAAOPJCAAAA|1997-10-27|1999-10-27|Then consistent lines might not consider the|0.41|0.14|5003001|exportischolar #1|3|pop|5|Music|752|ableantiation|N/A|steel704062034679326|peach|Each|Unknown|19|barantiationbarought| +10751|AAAAAAAAOPJCAAAA|1999-10-28|2001-10-26|Then consistent lines might not consider the|3.60|0.14|2001002|amalgimporto #2|1|accessories|2|Men|641|ableantiation|N/A|456481853624purple98|orchid|Lb|Unknown|29|oughtantiationbarought| +10752|AAAAAAAAOPJCAAAA|2001-10-27||Then consistent lines might not consider the|6.61|2.57|8014003|edu packmaxi #3|14|tennis|8|Sports|98|eingn st|N/A|456481853624purple98|snow|Gram|Unknown|43|ableantiationbarought| +10753|AAAAAAAABAKCAAAA|1997-10-27||Teams waste very back children. Wide, private years might help cells. Heavy,|0.57|0.17|7002004|importobrand #4|2|bedding|7|Home|604|esebarcally|N/A|9339puff577049308931|gainsboro|Each|Unknown|28|priantiationbarought| +10754|AAAAAAAACAKCAAAA|1997-10-27|2000-10-26|Always complex areas would convince less much local lawyers; modern others can sue home reasonable proposals.|4.59|2.66|8012005|importomaxi #5|12|guns|8|Sports|343|priesepri|N/A|7995022980642389sky0|sandy|Lb|Unknown|7|eseantiationbarought| +10755|AAAAAAAACAKCAAAA|||Always complex areas would convince less much local lawyers; modern others can sue home reasonable proposals.||2.66|3003002|||toddlers||Children|||petite||lime|Pound|Unknown||antiantiationbarought| +10756|AAAAAAAAEAKCAAAA|1997-10-27|1999-10-27|Or|3.00|1.86|4001001|amalgedu pack #1|1|womens|4|Shoes|200|barbarable|large|2091734714moccasin81|rosy|Bundle|Unknown|60|callyantiationbarought| +10757|AAAAAAAAEAKCAAAA|1999-10-28|2001-10-26|Merchants can test experts. Various years respect fast. Hours go probably so gener|1.74|0.97|4001001|exportiunivamalg #14|3|dvd/vcr players|10|Electronics|26|barbarable|N/A|2091734714moccasin81|tomato|Dozen|Unknown|27|ationantiationbarought| +10758|AAAAAAAAEAKCAAAA|2001-10-27||Merchants can test experts. Various years respect fast. Hours go probably so gener|0.68|0.53|4001001|corpbrand #7|16|consignment|6|Jewelry|26|barbarable|N/A|2091734714moccasin81|thistle|Case|Unknown|12|eingantiationbarought| +10759|AAAAAAAAHAKCAAAA|1997-10-27||Personal, economic shares could hear wide in a girls. Books might not contemplate words. Details experience. Economic refugees walk only economic, main parts. P|57.39|46.48|7009010|maxibrand #10|9|mattresses|7|Home|308|eingbarpri|N/A|5489183733tomato8483|olive|Tsp|Unknown|42|n stantiationbarought| +10760|AAAAAAAAIAKCAAAA|1997-10-27|2000-10-26|Separate stories get surprisingly. Pp. should make etc different leaves; signs leave almost wind|0.13|0.11|3002001|importoexporti #1|2|infants|3|Children|28|eingable|large|4081sky6548304734181|wheat|Ton|Unknown|17|barcallyationbarought| +10761|AAAAAAAAIAKCAAAA|2000-10-27||Explicit, african needs must|8.57|0.11|3002001|univmaxi #2|2|pools|8|Sports|11|oughtought|N/A|14pale73436206531572|steel|Gross|Unknown|42|oughtcallyationbarought| +10762|AAAAAAAAKAKCAAAA|1997-10-27|1999-10-27|Later biological plans must not come howe|4.47|3.79|10016002|corpamalgamalg #2|16|wireless|10|Electronics|512|ableoughtanti|N/A|6107171343pink297213|purple|Pallet|Unknown|8|ablecallyationbarought| +10763|AAAAAAAAKAKCAAAA|1999-10-28|2001-10-26|Later biological plans must not come howe|9.66|4.05|10016002|edu packamalg #2|4|swimwear|1|Women|512|ableoughtanti|large|0564280sienna8275350|midnight|Pallet|Unknown|97|pricallyationbarought| +10764|AAAAAAAAKAKCAAAA|2001-10-27||Shares know; surprising services assist clearly gingerly open points; only irish researchers reduce familiar cases. Large, modern children used to resolve never educat|1.43|1.11|10016002|amalgunivamalg #1|11|cooking|9|Books|617|ationoughtcally|N/A|91151919spring698297|purple|Tsp|Unknown|21|esecallyationbarought| +10765|AAAAAAAANAKCAAAA|1997-10-27||False concerns shall concentrate either useful animals. Companies requ|5.38|4.14|7014006|edu packnameless #6|14|glassware|7|Home|89|n steing|N/A|874frosted2948787852|khaki|Pallet|Unknown|17|anticallyationbarought| +10766|AAAAAAAAOAKCAAAA|1997-10-27|2000-10-26|English months ensure so. Serious, scientific numb|0.88|0.33|9007003|brandmaxi #3|7|reference|9|Books|139|n stpriought|N/A|3371426443663salmon5|violet|Bunch|Unknown|23|callycallyationbarought| +10767|AAAAAAAAOAKCAAAA|2000-10-27||Royal discussions come; winds shall remain early to the facilities. Shops would make more yet small thoughts; nuclear thanks can go a little hours. Around british |7.03|0.33|9007003|amalgscholar #2|7|rock|5|Music|137|ationpriought|N/A|3371426443663salmon5|pink|Lb|Unknown|42|ationcallyationbarought| +10768|AAAAAAAAABKCAAAA|1997-10-27|1999-10-27|Assessments get barely simple, pro|0.10|0.04|7009009|maxibrand #9|9|mattresses|7|Home|248|eingeseable|N/A|82pale10397091538486|tomato|Tbl|Unknown|39|eingcallyationbarought| +10769|AAAAAAAAABKCAAAA|1999-10-28|2001-10-26|Assessments get barely simple, pro|4.02|0.04|7009009|importoedu pack #2|2|mens|4|Shoes|248|eingeseable|medium|264176402smoke698001|steel|N/A|Unknown|87|n stcallyationbarought| +10770|AAAAAAAAABKCAAAA|2001-10-27||Modern, soviet materials will eliminate early at the incomes. Completely private results alter environmental, fair|6.92|0.04|7009009|amalgscholar #1|2|rock|5|Music|248|eingeseable|N/A|264176402smoke698001|papaya|Unknown|Unknown|38|barationationbarought| +10771|AAAAAAAADBKCAAAA|1997-10-27||Dogs wonder trousers. Friends will enjoy more as bitter lists. Monthly, exciting pictures would not run |3.96|1.50|10007011|brandunivamalg #11|7|personal|10|Electronics|7|ation|N/A|1011rose205398617807|royal|Oz|Unknown|36|oughtationationbarought| +10772|AAAAAAAAEBKCAAAA|1997-10-27|2000-10-26|Grounds will maintain merely white faces; existing figures replace possible, literary firms. Visitors might not look all strict keys. Ever prime children shall consider even real wi|5.47|3.33|7012003|importonameless #3|12|paint|7|Home|825|antiableeing|N/A|70423870white0097260|tomato|Gross|Unknown|13|ableationationbarought| +10773|AAAAAAAAEBKCAAAA|2000-10-27||More numerous forests would aid operations. Efficient facts hold also certain, sure tasks. Even particular adults cannot answer wo|7.08|3.33|7012003|edu packbrand #4|12|curtains/drapes|7|Home|825|antiableeing|N/A|70423870white0097260|white|Bunch|Unknown|9|priationationbarought| +10774|AAAAAAAAGBKCAAAA|1997-10-27|1999-10-27|Responsible strings would get more easy seasons; values grow only companies; forces cannot act slightly. Usual, basic germans earn less extra roses. Local, annual securiti|4.65|2.37|4002001|importoedu pack #1|2|mens|4|Shoes|595|antin stanti|large|21drab31874375080394|royal|Oz|Unknown|25|eseationationbarought| +10775|AAAAAAAAGBKCAAAA|1999-10-28|2001-10-26|Responsible strings would get more easy seasons; values grow only companies; forces cannot act slightly. Usual, basic germans earn less extra roses. Local, annual securiti|4.33|2.37|4002001|scholarbrand #6|2|blinds/shades|7|Home|329|n stablepri|N/A|354091peach715111547|sandy|Gross|Unknown|7|antiationationbarought| +10776|AAAAAAAAGBKCAAAA|2001-10-27||Necessary managers might give. Early weeks close. Recent, middle terms make unfortunately recommendations. Basic, adequate pages collect various, cognitive systems. New scots face second|3.26|1.17|4002001|amalgimporto #1|1|accessories|2|Men|329|n stablepri|medium|354091peach715111547|tomato|Tbl|Unknown|13|callyationationbarought| +10777|AAAAAAAAJBKCAAAA|1997-10-27||Big, far strategies shall not come even average, close developments; great clients might not pay never taxes. Still early children pretend yet sorry societies; again important proceedings continue |2.67|1.68|1004002|edu packamalg #2|4|swimwear|1|Women|33|pripri|small|75899782943rosy38134|dim|Dram|Unknown|11|ationationationbarought| +10778|AAAAAAAAKBKCAAAA|1997-10-27|2000-10-26|Years will not avoid times. Actual, outer texts would live. Little, sufficient attempts used to give finally governmen|2.67|1.62|7008009|namelessbrand #9|8|lighting|7|Home|306|callybarpri|N/A|54392022988puff70106|wheat|Case|Unknown|41|eingationationbarought| +10779|AAAAAAAAKBKCAAAA|2000-10-27||Years will not avoid times. Actual, outer texts would live. Little, sufficient attempts used to give finally governmen|2.95|1.65|9012004|importounivamalg #4|8|home repair|9|Books|306|callybarpri|N/A|80980papaya234288805|pale|Gram|Unknown|94|n stationationbarought| +10780|AAAAAAAAMBKCAAAA|1997-10-27|1999-10-27|Clear, long cats should not accept more beds. Inadequate, imperial attitudes use electrical states. Wines|4.97|4.07|8005001|scholarnameless #1|5|fitness|8|Sports|408|eingbarese|N/A|956941indian31099478|tomato|Each|Unknown|9|bareingationbarought| +10781|AAAAAAAAMBKCAAAA|1999-10-28|2001-10-26|Else good forms can hear hence ye|7.58|4.07|8005001|exportiedu pack #2|5|kids|4|Shoes|838|eingbarese|large|956941indian31099478|thistle|Case|Unknown|20|oughteingationbarought| +10782|AAAAAAAAMBKCAAAA|2001-10-27||So material |2.86|1.05|5004001|edu packscholar #1|5|classical|5|Music|838|eingprieing|N/A|956941indian31099478|powder|Each|Unknown|23|ableeingationbarought| +10783|AAAAAAAAPBKCAAAA|1997-10-27||Heavy, traditional unions could come markedly international, warm n|4.19|3.43|2003002|exportiimporto #2|3|pants|2|Men|262|ablecallyable|large|73sienna959735642894|powder|Lb|Unknown|85|prieingationbarought| +10784|AAAAAAAAACKCAAAA|1997-10-27|2000-10-26|Long states identify dangerou|1.44|0.74|1002001|importoamalg #1|2|fragrances|1|Women|955|antiantin st|extra large|009585546puff2396362|orange|Ounce|Unknown|50|eseeingationbarought| +10785|AAAAAAAAACKCAAAA|2000-10-27||Available gifts promote surprised, smooth act|6.95|0.74|1002001|amalgamalg #2|1|dresses|1|Women|955|antiantin st|small|009585546puff2396362|khaki|Carton|Unknown|9|antieingationbarought| +10786|AAAAAAAACCKCAAAA|1997-10-27|1999-10-27|Finan|4.64|1.67|5004001|edu packscholar #1|4|classical|5|Music|14|eseought|N/A|58navajo971316317182|snow|Unknown|Unknown|17|callyeingationbarought| +10787|AAAAAAAACCKCAAAA|1999-10-28|2001-10-26|Finan|2.11|1.51|3003002|exportiexporti #2|3|toddlers|3|Children|14|eseought|large|58523sandy4807143970|rosy|Cup|Unknown|35|ationeingationbarought| +10788|AAAAAAAACCKCAAAA|2001-10-27||Finan|8.18|1.51|3003002|exportiamalg #1|3|maternity|1|Women|820|barableeing|petite|58523sandy4807143970|peru|Gram|Unknown|8|eingeingationbarought| +10789|AAAAAAAAFCKCAAAA|1997-10-27||Brilliant, acceptable resources might not pick as. Positive, married parties support only strongly impossible needs. Photogra|2.44|1.26|9001002|amalgmaxi #2|1|arts|9|Books|150|barantiought|N/A|539983linen728728900|midnight|N/A|Unknown|36|n steingationbarought| +10790|AAAAAAAAGCKCAAAA|1997-10-27|2000-10-26|Parties com|19.12|8.03|3004001|edu packexporti #1|4|school-uniforms|3|Children|128|eingableought|medium|4plum729037383213458|pink|Each|Unknown|96|barn stationbarought| +10791|AAAAAAAAGCKCAAAA|2000-10-27||Permanent contents consider. Contents should show most now sole lips; exhibitions shall enter usual weeks; here official organisers win well |1.52|8.03|2003002|exportiimporto #2|3|pants|2|Men|162|ablecallyought|petite|4plum729037383213458|red|Tbl|Unknown|50|oughtn stationbarought| +10792|AAAAAAAAICKCAAAA|1997-10-27|1999-10-27|Fears take sudden developments. Central cells might try forward for instance special banks. Feet must not mean also. Flat times shall ask over the days. Regulations may consider; |7.20|5.18|9006005|corpmaxi #5|6|parenting|9|Books|453|priantiese|N/A|almond26941073814942|spring|Unknown|Unknown|62|ablen stationbarought| +10793|AAAAAAAAICKCAAAA|1999-10-28|2001-10-26|Fears take sudden developments. Central cells might try forward for instance special banks. Feet must not mean also. Flat times shall ask over the days. Regulations may consider; |1.18|5.18|9006005|importobrand #4|2|bedding|7|Home|453|priantiese|N/A|almond26941073814942|sandy|N/A|Unknown|15|prin stationbarought| +10794|AAAAAAAAICKCAAAA|2001-10-27||Walls move at present. Well considerable hours examine w|2.43|1.99|4003001|exportiedu pack #1|3|kids|4|Shoes|453|priantiese|small|almond26941073814942|burlywood|Tsp|Unknown|87|esen stationbarought| +10795|AAAAAAAALCKCAAAA|1997-10-27||Bloody, new provinces recognize low underlying figures. Nuclear years say little, significant relatives. Rather developing armies show new, new benefits. Quite initial users crack muc|8.49|7.30|1001002|amalgamalg #2|1|dresses|1|Women|112|ableoughtought|large|79123789smoke6282884|coral|Cup|Unknown|4|antin stationbarought| +10796|AAAAAAAAMCKCAAAA|1997-10-27|2000-10-26|Probably ashamed courses may hear never changing years. Clothes ough|1.39|0.62|3003001|exportiexporti #1|3|toddlers|3|Children|87|ationeing|small|182247peach082863248|rosy|N/A|Unknown|56|callyn stationbarought| +10797|AAAAAAAAMCKCAAAA|2000-10-27||Probably ashamed courses may hear never changing years. Clothes ough|1.36|0.91|3003001|scholarbrand #6|3|custom|6|Jewelry|184|ationeing|N/A|182247peach082863248|lavender|N/A|Unknown|82|ationn stationbarought| +10798|AAAAAAAAOCKCAAAA|1997-10-27|1999-10-27|Provincial statements shall expect other, dead eyes. Perfect differences must lose too musical events. Competitive, goo|1.86|1.07|7010009|univnameless #9|10|flatware|7|Home|331|oughtpripri|N/A|5354904949994puff075|lavender|Carton|Unknown|73|eingn stationbarought| +10799|AAAAAAAAOCKCAAAA|1999-10-28|2001-10-26|Provincial statements shall expect other, dead eyes. Perfect differences must lose too musical events. Competitive, goo|3.59|1.07|7010009|importoimporto #2|2|shirts|2|Men|331|oughtpripri|extra large|5354904949994puff075|misty|Lb|Unknown|48|n stn stationbarought| +10800|AAAAAAAAOCKCAAAA|2001-10-27||Purposes used to arrest so|9.79|7.14|7010009|edu packamalg #1|4|swimwear|1|Women|331|oughtpripri|small|5418589powder5662855|sky|Case|Unknown|25|barbareingbarought| +10801|AAAAAAAABDKCAAAA|1997-10-27||Supporters may arrive cars; conventional cheeks maintain all the same substantial fingers|0.13|0.04|4004002|edu packedu pack #2|4|athletic|4|Shoes|473|priationese|small|129110227655peru0205|peru|Bunch|Unknown|52|oughtbareingbarought| +10802|AAAAAAAACDKCAAAA|1997-10-27|2000-10-26|Economic, big weeks must change pos|1.80|1.51|1001001|amalgamalg #1|1|dresses|1|Women|38|eingpri|medium|21915312798peru36223|metallic|Pound|Unknown|78|ablebareingbarought| +10803|AAAAAAAACDKCAAAA|2000-10-27||Economic, big weeks must change pos|6.17|1.51|1001001|edu packunivamalg #9|1|audio|10|Electronics|487|ationeingese|N/A|0royal65802352724288|peru|Tsp|Unknown|51|pribareingbarought| +10804|AAAAAAAAEDKCAAAA|1997-10-27|1999-10-27|Possible, vulnerable feet ask economic, different changes. Times come so african questions. Months will not feel sorry, linguistic results. New, classical eyes know by a pages. Various, ready women a|23.00|16.56|6009005|maxicorp #5|9|womens watch|6|Jewelry|366|callycallypri|N/A|6964264429saddle8163|moccasin|Ton|Unknown|48|esebareingbarought| +10805|AAAAAAAAEDKCAAAA|1999-10-28|2001-10-26|Political wages fall to a changes.|3.39|16.56|6009005|exportischolar #2|9|pop|5|Music|366|callycallypri|N/A|1823434salmon1939851|purple|Dozen|Unknown|25|antibareingbarought| +10806|AAAAAAAAEDKCAAAA|2001-10-27||Level, open improvements should choose most always good courses. Etc thin hotels speak with the structures. Minute, federal communities swing factors. Stations need more throughout a parties. |3.58|2.93|6009005|amalgimporto #1|1|accessories|2|Men|585|antieinganti|large|1823434salmon1939851|wheat|Gram|Unknown|10|callybareingbarought| +10807|AAAAAAAAHDKCAAAA|1997-10-27||Rare, scientific parts halt then after the prospects. Structures train white, high mountains. Often primary markets adapt well scholars. Considerable scores s|2.19|1.55|6015008|scholarbrand #8|15|custom|6|Jewelry|572|ableationanti|N/A|6627291light70831375|indian|Tsp|Unknown|45|ationbareingbarought| +10808|AAAAAAAAIDKCAAAA|1997-10-27|2000-10-26|Leaders miss inevitably new, central u|3.18|1.43|4002001|importoedu pack #1|2|mens|4|Shoes|212|ableoughtable|medium|8114392beige64249426|snow|Gross|Unknown|14|eingbareingbarought| +10809|AAAAAAAAIDKCAAAA|2000-10-27||Silver, blue samples h|27.06|19.48|7003010|exportibrand #10|2|kids|7|Home|212|ableoughtable|N/A|8114392beige64249426|sky|Carton|Unknown|44|n stbareingbarought| +10810|AAAAAAAAKDKCAAAA|1997-10-27|1999-10-27|Limitations must maintain present, new trusts. Modern preferences shall answer financial, sure individuals. Only due projects know also perhaps con|1.60|0.83|1004001|edu packamalg #1|4|swimwear|1|Women|421|oughtableese|small|068spring91930830680|maroon|Dram|Unknown|43|baroughteingbarought| +10811|AAAAAAAAKDKCAAAA|1999-10-28|2001-10-26|Limitations must maintain present, new trusts. Modern preferences shall answer financial, sure individuals. Only due projects know also perhaps con|8.60|4.12|1004001|scholarmaxi #2|4|fishing|8|Sports|234|esepriable|N/A|068spring91930830680|wheat|Ton|Unknown|6|oughtoughteingbarought| +10812|AAAAAAAAKDKCAAAA|2001-10-27||Limitations must maintain present, new trusts. Modern preferences shall answer financial, sure individuals. Only due projects know also perhaps con|3.58|4.12|1004001|edu packamalg #1|4|swimwear|1|Women|69|esepriable|extra large|0225ghost42598533655|ghost|Bundle|Unknown|20|ableoughteingbarought| +10813|AAAAAAAANDKCAAAA|1997-10-27||True tears continue currently pale, close men. Soon medical numbers might not take other, |1.42|1.05|3004002|edu packexporti #2|4|school-uniforms|3|Children|332|ablepripri|extra large|6494pale607565668777|sienna|Each|Unknown|59|prioughteingbarought| +10814|AAAAAAAAODKCAAAA|1997-10-27|2000-10-26|Almost critical firms ought to encourage previously meetings. Also british reports come even nice beans. Free children change over hostile limitations. De|8.26|4.21|7002005|importobrand #5|2|bedding|7|Home|185|antieingought|N/A|715172805212spring52|steel|Unknown|Unknown|19|eseoughteingbarought| +10815|AAAAAAAAODKCAAAA|2000-10-27||Almost critical firms ought to encourage previously meetings. Also british reports come even nice beans. Free children change over hostile limitations. De|8.40|5.79|7002005|edu packexporti #2|2|school-uniforms|3|Children|571|antieingought|petite|715172805212spring52|green|Bundle|Unknown|57|antioughteingbarought| +10816|AAAAAAAAAEKCAAAA|1997-10-27|1999-10-27|Ordinary metals would transport with a policies; about arbitrary balls must go sites. Clear prices continue of course. I|54.72|29.00|8007007|brandnameless #7|7|hockey|8|Sports|700|barbaration|N/A|072875348476964grey7|sienna|Tsp|Unknown|11|callyoughteingbarought| +10817|AAAAAAAAAEKCAAAA|1999-10-28|2001-10-26|Crucial cells would receive secondly star, available months. Remaining, fair schools love damp missiles. Chief gues|3.91|1.64|8007007|edu packscholar #2|4|classical|5|Music|700|barbaration|N/A|756181631648rosy5506|rosy|Ounce|Unknown|8|ationoughteingbarought| +10818|AAAAAAAAAEKCAAAA|2001-10-27||Dif|7.00|2.66|8007007|exportiamalg #1|3|maternity|1|Women|490|barn stese|extra large|756181631648rosy5506|pink|Bundle|Unknown|15|eingoughteingbarought| +10819|AAAAAAAADEKCAAAA|1997-10-27||Realistic, left exercises encourage productive trees. Tools go however. New, central years assume together english, secondary proposals. |2.26|1.15|10002008|importounivamalg #8|2|camcorders|10|Electronics|271|oughtationable|N/A|743540611868papaya17|ghost|N/A|Unknown|7|n stoughteingbarought| +10820|AAAAAAAAEEKCAAAA|1997-10-27|2000-10-26|Occasionally complete vessels make. Regions sit about|4.38|3.24|3002001|importoexporti #1|2|infants|3|Children|932|ableprin st|economy|3274376pale198359226|royal|Dozen|Unknown|41|barableeingbarought| +10821|AAAAAAAAEEKCAAAA|2000-10-27||Customs will take far colonies. Leaves ought to emphasise slightly away perfect videos. Men work plain i|1.81|1.14|3002001|amalgamalg #2|1|dresses|1|Women|122|ableableought|small|2448976931snow933915|rose|Bundle|Unknown|16|oughtableeingbarought| +10822|AAAAAAAAGEKCAAAA|1997-10-27|1999-10-27|Convenient, scientific police will play moving, very damages. Hard points form as animals. Groups go as complex, late states. However new heads may like|2.42|1.74|1001001|amalgamalg #1|1|dresses|1|Women|180|bareingought|petite|59pink00969698791093|dim|Ton|Unknown|23|ableableeingbarought| +10823|AAAAAAAAGEKCAAAA|1999-10-28|2001-10-26|Parameters expect already prime, free approaches. Bodies shall use at the phenomena. Arts shall extend difficul|4.76|2.52|1001001|scholarmaxi #12|5|history|9|Books|180|bareingought|N/A|3515261maroon8854151|olive|Ounce|Unknown|62|priableeingbarought| +10824|AAAAAAAAGEKCAAAA|2001-10-27||Parameters expect already prime, free approaches. Bodies shall use at the phenomena. Arts shall extend difficul|5.02|4.01|1001001|corpamalgamalg #13|5|wireless|10|Electronics|180|bareingought|N/A|78809069sienna824261|tan|Bundle|Unknown|4|eseableeingbarought| +10825|AAAAAAAAJEKCAAAA|1997-10-27||Years win probably after the teams. More possible teachers shall hand|7.22|3.03|7004006|edu packbrand #6|4|curtains/drapes|7|Home|398|eingn stpri|N/A|309740596sandy759168|lawn|Box|Unknown|33|antiableeingbarought| +10826|AAAAAAAAKEKCAAAA|1997-10-27|2000-10-26|Universal, active restrictions force case|2.39|2.15|6016003|corpbrand #3|16|consignment|6|Jewelry|201|oughtbarable|N/A|9681577365papaya9761|aquamarine|Dozen|Unknown|14|callyableeingbarought| +10827|AAAAAAAAKEKCAAAA|2000-10-27||Orders mean great financial occasions; domestic actions find even informal|0.79|0.52|6016003|importoexporti #2|2|infants|3|Children|308|eingbarpri|large|9681577365papaya9761|hot|Tsp|Unknown|11|ationableeingbarought| +10828|AAAAAAAAMEKCAAAA|1997-10-27|1999-10-27|So international methods hold hence as senior glasses. So direct complaints will not attract far. Even narrow members must happen in a vehicles. Institution|3.31|1.09|9009005|maximaxi #5|9|science|9|Books|188|eingeingought|N/A|54powder810354941693|sienna|Ounce|Unknown|25|eingableeingbarought| +10829|AAAAAAAAMEKCAAAA|1999-10-28|2001-10-26|Peaceful, big women recognize supreme, interested books. Normal, boring jobs should not return still also total countries; risks know home comfortable shows. Concerned, na|8.14|1.09|9009005|edu packamalgamalg #1|9|automotive|10|Electronics|188|eingeingought|N/A|54powder810354941693|tomato|Box|Unknown|34|n stableeingbarought| +10830|AAAAAAAAMEKCAAAA|2001-10-27||Compatible terms in|56.27|35.45|9009005|amalgedu pack #1|1|womens|4|Shoes|992|ablen stn st|extra large|4853518254saddle2659|cream|Case|Unknown|66|barprieingbarought| +10831|AAAAAAAAPEKCAAAA|1997-10-27||Alone countries must use so old, international functions. Only public cases see in a words. Normal methods forget even communist changes; technical numbers convert either natu|4.67|4.15|9002008|importomaxi #8|2|business|9|Books|565|anticallyanti|N/A|snow8478848535533443|smoke|Pound|Unknown|52|oughtprieingbarought| +10832|AAAAAAAAAFKCAAAA|1997-10-27|2000-10-26|Coming, new agencies inherit yet adequate, high members. Students win so bags. New records destroy. Real grounds prefer|61.46|43.02|3004001|edu packexporti #1|4|school-uniforms|3|Children|61|oughtcally|large|6078116mint828581693|sky|Each|Unknown|50|ableprieingbarought| +10833|AAAAAAAAAFKCAAAA|2000-10-27||Coming, new agencies inherit yet adequate, high members. Students win so bags. New records destroy. Real grounds prefer|6.54|43.02|4002002|importoedu pack #2|4|mens|4|Shoes|61|oughtcally|large|986297964712pink8876|spring|Dram|Unknown|73|priprieingbarought| +10834|AAAAAAAACFKCAAAA|1997-10-27|1999-10-27|Items will not see most strong, large scots. Guests restore values; chief, star armies may complete certainly studies. Psychological deals grow again. Just roman years s|3.84|1.34|6014003|edu packbrand #3|14|estate|6|Jewelry|156|callyantiought|N/A|487637004434slate895|green|Pound|Unknown|65|eseprieingbarought| +10835|AAAAAAAACFKCAAAA|1999-10-28|2001-10-26|Items will not see most strong, large scots. Guests restore values; chief, star armies may complete certainly studies. Psychological deals grow again. Just roman years s|9.10|1.34|6014003|scholarcorp #4|5|earings|6|Jewelry|156|callyantiought|N/A|487637004434slate895|puff|Gross|Unknown|32|antiprieingbarought| +10836|AAAAAAAACFKCAAAA|2001-10-27||Today new others ought to turn indirec|6.21|1.34|6014003|exportimaxi #7|5|sailing|8|Sports|156|callyantiought|N/A|1410orchid8922560941|gainsboro|Gram|Unknown|5|callyprieingbarought| +10837|AAAAAAAAFFKCAAAA|1997-10-27||Possible, hot members depend also necessary authorities; modest types please far from reasons; still male faces would become original charges. Basic relations cou|6.29|4.21|1003002|exportiamalg #2|3|maternity|1|Women|400|barbarese|economy|2sandy99886431394724|gainsboro|Tbl|Unknown|8|ationprieingbarought| +10838|AAAAAAAAGFKCAAAA|1997-10-27|2000-10-26|Facilities need both common men. Investments involve however; consequences disappear quite statutory sides; sure, present relationships consider today outside personal parts|8.09|5.82|3003001|exportiexporti #1|3|toddlers|3|Children|245|antieseable|N/A|03652982193khaki3328|seashell|Pound|Unknown|23|eingprieingbarought| +10839|AAAAAAAAGFKCAAAA|2000-10-27||Recent, new games can hope still apparent workers. However deaf|5.24|5.82|6007008|brandcorp #8|7|pendants|6|Jewelry|245|antieseable|N/A|03652982193khaki3328|medium|Tbl|Unknown|17|n stprieingbarought| +10840|AAAAAAAAIFKCAAAA|1997-10-27|1999-10-27|Other, theoretical men mi|5.28|3.27|4002001|importoedu pack #1|2|mens|4|Shoes|50|baranti|medium|62214793684honeydew1|pale|Dozen|Unknown|3|bareseeingbarought| +10841|AAAAAAAAIFKCAAAA|1999-10-28|2001-10-26|Upwards soviet bacteria see followi|38.09|3.27|4002001|amalgedu pack #2|1|womens|4|Shoes|50|baranti|extra large|62214793684honeydew1|papaya|Oz|Unknown|46|oughteseeingbarought| +10842|AAAAAAAAIFKCAAAA|2001-10-27||New years affect complex arts. Constant, full resources might not rise confident, available participants. Clear duties can reveal thus usually silent names. Diseases realize there. Games shall d|7.65|4.20|7012003|importonameless #3|12|paint|7|Home|525|baranti|N/A|pale7911043424327844|royal|Dram|Unknown|44|ableeseeingbarought| +10843|AAAAAAAALFKCAAAA|1997-10-27||Agricultural, selective groups follow much worthwhile panels. Fully sim|1.95|1.69|5002002|importoscholar #2|2|country|5|Music|93|prin st|N/A|00612purple515216179|hot|Cup|Unknown|66|prieseeingbarought| +10844|AAAAAAAAMFKCAAAA|1997-10-27|2000-10-26|Once electronic workers eat. Eastern girls will cause then key goods; things go at a operations. Other, delicious explanations could deny safe, particular courts. Really big fla|8.23|3.86|4003001|exportiedu pack #1|3|kids|4|Shoes|363|pricallypri|small|351142578729peach304|tan|Pallet|Unknown|25|eseeseeingbarought| +10845|AAAAAAAAMFKCAAAA|2000-10-27||Fortunately new months hear her|7.19|3.59|4003001|importoscholar #2|2|country|5|Music|41|pricallypri|N/A|34075820tan581057627|wheat|Pound|Unknown|19|antieseeingbarought| +10846|AAAAAAAAOFKCAAAA|1997-10-27|1999-10-27|Years give maybe bright, domestic variations; public standards may use especially necessary|2.27|2.04|7014007|edu packnameless #7|14|glassware|7|Home|115|antioughtought|N/A|94758844692rosy68608|peru|Dram|Unknown|17|callyeseeingbarought| +10847|AAAAAAAAOFKCAAAA|1999-10-28|2001-10-26|Years give maybe bright, domestic variations; public standards may use especially necessary|0.77|0.58|3004002|edu packexporti #2|4|school-uniforms|3|Children|245|antieseable|extra large|94758844692rosy68608|yellow|Lb|Unknown|37|ationeseeingbarought| +10848|AAAAAAAAOFKCAAAA|2001-10-27||Opportunities might betray out of a organizations. Perhaps separat|8.03|0.58|4004001|edu packedu pack #1|4|athletic|4|Shoes|245|antieseable|large|94758844692rosy68608|saddle|Each|Unknown|1|eingeseeingbarought| +10849|AAAAAAAABGKCAAAA|1997-10-27||More natural jobs deliver front needs. Roman tons go eventually indian gardens. Black judges assert hence. C|9.81|6.18|10015004|scholaramalgamalg #4|15|portable|10|Electronics|213|prioughtable|N/A|5351steel55750496046|rosy|Pound|Unknown|55|n steseeingbarought| +10850|AAAAAAAACGKCAAAA|1997-10-27|2000-10-26|Fast, material regions shall not beat across with the windows. Important cases used to solve too old, widespread principles; rich, cent|8.75|6.65|5004001|edu packscholar #1|4|classical|5|Music|787|ationeingation|N/A|590444seashell451924|rosy|Cup|Unknown|6|barantieingbarought| +10851|AAAAAAAACGKCAAAA|2000-10-27||Strange, fresh companies compete ill heavy measures. Prospective, confidential wo|6.39|3.19|1002002|importoamalg #2|2|fragrances|1|Women|304|esebarpri|medium|590444seashell451924|purple|Case|Unknown|5|oughtantieingbarought| +10852|AAAAAAAAEGKCAAAA|1997-10-27|1999-10-27|Automatically opt|9.44|5.94|7013003|exportinameless #3|13|wallpaper|7|Home|816|callyoughteing|N/A|59449120657526black0|steel|Gross|Unknown|29|ableantieingbarought| +10853|AAAAAAAAEGKCAAAA|1999-10-28|2001-10-26|Troops must get also young sorts. Past, serious minutes arrive|0.81|5.94|7013003|amalgamalg #2|13|dresses|1|Women|21|callyoughteing|petite|8royal37356879731503|rosy|Unknown|Unknown|65|priantieingbarought| +10854|AAAAAAAAEGKCAAAA|2001-10-27||Grey, long exports may not freeze sometimes additional, new efforts. Old servants must make still events. Visible provisions will not overcome ne|3.79|5.94|2003001|exportiimporto #1|13|pants|2|Men|888|eingeingeing|small|8royal37356879731503|sienna|Tbl|Unknown|15|eseantieingbarought| +10855|AAAAAAAAHGKCAAAA|1997-10-27||Here available cl|1.28|0.39|3004002|edu packexporti #2|4|school-uniforms|3|Children|49|n stese|large|2242430656turquoise1|violet|Dozen|Unknown|61|antiantieingbarought| +10856|AAAAAAAAIGKCAAAA|1997-10-27|2000-10-26|Local, total restrictions must claim only apparently different times. Inches cannot thank just empty minutes. Able, bare generation|9.23|4.79|7011007|amalgnameless #7|11|accent|7|Home|417|ationoughtese|N/A|6253pale402690072762|purple|Gross|Unknown|86|callyantieingbarought| +10857|AAAAAAAAIGKCAAAA|2000-10-27||Local, grim tests know consequences. Clear, black lists cannot use positive needs. Small bits |7.67|4.79|2001002|amalgimporto #2|11|accessories|2|Men|417|ationoughtese|extra large|6253pale402690072762|salmon|Pound|Unknown|57|ationantieingbarought| +10858|AAAAAAAAKGKCAAAA|1997-10-27|1999-10-27|Ei|4.30|2.92|8015009|scholarmaxi #9|15|fishing|8|Sports|324|eseablepri|N/A|4572275pale283252521|aquamarine|Case|Unknown|29|eingantieingbarought| +10859|AAAAAAAAKGKCAAAA|1999-10-28|2001-10-26|Indeed working definitions may feel urban, long plans. Female grants used to contain; properties would s|3.01|2.92|9012012|importounivamalg #12|15|home repair|9|Books|324|eseablepri|N/A|39125seashell8925150|turquoise|Gram|Unknown|33|n stantieingbarought| +10860|AAAAAAAAKGKCAAAA|2001-10-27||Holidays hold again central costs. Small, only examples know wide earnings. Necessarily critical initiatives starve complex, whole artists. Friends may wield well junior hundreds. Bir|9.00|2.70|1002001|importoamalg #1|2|fragrances|1|Women|324|eseablepri|petite|2521orchid3201850689|rose|Bundle|Unknown|58|barcallyeingbarought| +10861|AAAAAAAANGKCAAAA|1997-10-27||Faces could experiment presently. New arrangements tell already; causal females constitu|4.02|3.57|10002016|importounivamalg #16|2|camcorders|10|Electronics|54|eseanti|N/A|7243goldenrod5431099|pale|Cup|Unknown|9|oughtcallyeingbarought| +10862|AAAAAAAAOGKCAAAA|1997-10-27|2000-10-26|Popular trees get possibilities. Drivers should allow certainly dynamic directors. Today impressive conclusions meet often close tools; |2.95|2.30|10006017|corpunivamalg #17|6|musical|10|Electronics|159|n stantiought|N/A|81yellow409674323626|purple|Gram|Unknown|55|ablecallyeingbarought| +10863|AAAAAAAAOGKCAAAA|2000-10-27||Popular trees get possibilities. Drivers should allow certainly dynamic directors. Today impressive conclusions meet often close tools; |1.54|1.17|2003002|exportiimporto #2|6|pants|2|Men|568|n stantiought|large|3pink191880775152356|peach|Lb|Unknown|44|pricallyeingbarought| +10864|AAAAAAAAAHKCAAAA|1997-10-27|1999-10-27|Somewhere hot arms touch however before a members. New developers ought to deal polish cells. Days achieve into an interests. Bodie|5.86|1.93|9011005|amalgunivamalg #5|11|cooking|9|Books|572|ableationanti|N/A|13133529sienna618496|steel|Each|Unknown|56|esecallyeingbarought| +10865|AAAAAAAAAHKCAAAA|1999-10-28|2001-10-26|Somewhere hot arms touch however before a members. New developers ought to deal polish cells. Days achieve into an interests. Bodie|0.71|1.93|9011005|importoamalg #2|11|fragrances|1|Women|572|ableationanti|large|13133529sienna618496|papaya|Oz|Unknown|58|anticallyeingbarought| +10866|AAAAAAAAAHKCAAAA|2001-10-27||Somewhere hot arms touch however before a members. New developers ought to deal polish cells. Days achieve into an interests. Bodie|0.89|1.93|9011005|importobrand #3|11|costume|6|Jewelry|572|ableationanti|N/A|09351865sienna749067|seashell|Bunch|Unknown|21|callycallyeingbarought| +10867|AAAAAAAADHKCAAAA|1997-10-27||Deep, interesting units play with a children. More democratic difficulties control however additional attitudes. Nearby, likely connecti|3.48|2.05|3001002|amalgexporti #2|1|newborn|3|Children|107|ationbarought|small|81766440773170hot801|peru|Bundle|Unknown|64|ationcallyeingbarought| +10868|AAAAAAAAEHKCAAAA|1997-10-27|2000-10-26|Only growing clothes teach often to a months. Muscles provide mentally difficult chapters; certain others will advise base figures. Important bombs s|4.02|1.68|6005005|scholarcorp #5|5|earings|6|Jewelry|94|esen st|N/A|8765saddle7960858121|lime|Oz|Unknown|78|eingcallyeingbarought| +10869|AAAAAAAAEHKCAAAA|2000-10-27||Years should look small policies. Other, huge concepts learn global, young sections; also international reforms b|2.64|1.68|5002002|importoscholar #2|2|country|5|Music|166|esen st|N/A|68orchid498188436730|linen|Gross|Unknown|44|n stcallyeingbarought| +10870|AAAAAAAAGHKCAAAA|1997-10-27|1999-10-27|Relations ought to ensure. Ordinary, common policemen realise years. Fans would win less significant masters. Effects expect years. Advisers pre|77.70|55.16|1002001|importoamalg #1|2|fragrances|1|Women|101|oughtbarought|medium|97488290snow74694402|puff|Ton|Unknown|72|barationeingbarought| +10871|AAAAAAAAGHKCAAAA|1999-10-28|2001-10-26|Relations ought to ensure. Ordinary, common policemen realise years. Fans would win less significant masters. Effects expect years. Advisers pre|5.42|55.16|10010009|univamalgamalg #9|10|memory|10|Electronics|197|ationn stought|N/A|97488290snow74694402|sky|Pound|Unknown|12|oughtationeingbarought| +10872|AAAAAAAAGHKCAAAA|2001-10-27||Relations ought to ensure. Ordinary, common policemen realise years. Fans would win less significant masters. Effects expect years. Advisers pre|4.16|55.16|10010009|amalgimporto #1|10|accessories|2|Men|543|ationn stought|medium|13888399wheat8813154|gainsboro|Gross|Unknown|46|ableationeingbarought| +10873|AAAAAAAAJHKCAAAA|1997-10-27||Years will tell more other |1.34|0.60|1001002|amalgamalg #2|1|dresses|1|Women|6|cally|large|679gainsboro01450574|puff|Pound|Unknown|65|priationeingbarought| +10874|AAAAAAAAKHKCAAAA|1997-10-27|2000-10-26|Years close originally rural words. Single, other resources like young, managerial names. Well british weapons get in a areas. New, english women co|3.18|2.09|3002001|importoexporti #1|2|infants|3|Children|331|oughtpripri|small|tomato07284622524348|grey|Dram|Unknown|10|eseationeingbarought| +10875|AAAAAAAAKHKCAAAA|2000-10-27||Top systems used to avoid late in the pounds. Prod|4.39|2.09|3001002|amalgexporti #2|2|newborn|3|Children|901|oughtbarn st|medium|400578gainsboro33073|tan|N/A|Unknown|12|antiationeingbarought| +10876|AAAAAAAAMHKCAAAA|1997-10-27|1999-10-27|At once male habits cannot need normal friends; never great activities could apply home worlds. Physical years shall not make terrible carers. Industries control there. Visits descri|5.06|3.23|2003001|exportiimporto #1|3|pants|2|Men|105|antibarought|N/A|109162tomato99976068|lime|Pallet|Unknown|23|callyationeingbarought| +10877|AAAAAAAAMHKCAAAA|1999-10-28|2001-10-26|At once male habits cannot need normal friends; never great activities could apply home worlds. Physical years shall not make terrible carers. Industries control there. Visits descri|96.04|3.23|5004002|edu packscholar #2|4|classical|5|Music|316|antibarought|N/A|9626553242507white74|wheat|Box|Unknown|25|ationationeingbarought| +10878|AAAAAAAAMHKCAAAA|2001-10-27||Small grounds take suddenly; black taxes fin|7.90|3.23|6016007|corpbrand #7|4|consignment|6|Jewelry|316|callyoughtpri|N/A|1909479554papaya3896|royal|Bundle|Unknown|25|eingationeingbarought| +10879|AAAAAAAAPHKCAAAA|1997-10-27||Women expect. Over net names must not get small, right years. Yesterday silly companies favour understandably particularly complicated cases|59.55|20.24|10013017|exportiamalgamalg #17|13|stereo|10|Electronics|338|eingpripri|N/A|74213red741017633151|pale|Bundle|Unknown|13|n stationeingbarought| +10880|AAAAAAAAAIKCAAAA|1997-10-27|2000-10-26|Troubles must know wise indicators. Kinds enter technical, new doubts. Likely, annual eyes see equivalent payments. Both inadequate feelings decide ever initial |5.04|2.92|9004009|edu packmaxi #9|4|entertainments|9|Books|35|antipri|N/A|2923702686419tan9474|ghost|Gram|Unknown|67|bareingeingbarought| +10881|AAAAAAAAAIKCAAAA|2000-10-27||Sch|3.92|1.41|9004009|exportiunivamalg #2|4|dvd/vcr players|10|Electronics|35|antipri|N/A|833sky68632133770781|slate|Dram|Unknown|3|oughteingeingbarought| +10882|AAAAAAAACIKCAAAA|1997-10-27|1999-10-27|Refugees rise then expert, orange boys. Young |5.17|2.84|9013011|exportiunivamalg #11|13|self-help|9|Books|177|ationationought|N/A|993270306610khaki177|sandy|Bundle|Unknown|27|ableeingeingbarought| +10883|AAAAAAAACIKCAAAA|1999-10-28|2001-10-26|Personal, impossible patterns shall not argue often for a sounds. Students would implement. Letters |5.35|2.24|9013011|maxinameless #4|13|optics|8|Sports|177|ationationought|N/A|993270306610khaki177|steel|Bundle|Unknown|32|prieingeingbarought| +10884|AAAAAAAACIKCAAAA|2001-10-27||Questions used to find even full buildings. So serious eyes may not limit on a customers. Ordinary houses shall not think here only unknown school|3.28|2.24|3001001|amalgexporti #1|13|newborn|3|Children|243|prieseable|large|993270306610khaki177|turquoise|Lb|Unknown|24|eseeingeingbarought| +10885|AAAAAAAAFIKCAAAA|1997-10-27||Masters help in terms of the hours. Still different details used to find always long black savings. Now free shares demonstrate behind. Extended, empty sentences ask ago |9.52|5.14|7002006|importobrand #6|2|bedding|7|Home|63|prically|N/A|misty653070551025608|tomato|Cup|Unknown|36|antieingeingbarought| +10886|AAAAAAAAGIKCAAAA|1997-10-27|2000-10-26|Able prisoners used to say more than similar materials. Long-term, pale women go. Male, likely doors remain simple, young jews. Very single letters sit thereby scottish poi|7.87|4.64|4001001|amalgedu pack #1|1|womens|4|Shoes|370|barationpri|small|671024365024violet00|dark|Tsp|Unknown|29|callyeingeingbarought| +10887|AAAAAAAAGIKCAAAA|2000-10-27||Areas tell. Simple, complete concessions must occur labour, back publications. Financial, related poems shall minimise medieval, clear workers. Indeed fine views accept in a prisons. Savings lower |8.00|4.48|5003002|exportischolar #2|3|pop|5|Music|223|priableable|N/A|671024365024violet00|salmon|Case|Unknown|6|ationeingeingbarought| +10888|AAAAAAAAIIKCAAAA|1997-10-27|1999-10-27|Provisions get consistently biological, private demands. Given holes establish british, double reactions; rather|4.65|1.39|5004001|edu packscholar #1|4|classical|5|Music|121|oughtableought|N/A|1cream46189643682184|pale|Pound|Unknown|55|eingeingeingbarought| +10889|AAAAAAAAIIKCAAAA|1999-10-28|2001-10-26|Quickly great subjects move civil societies. Private relations wil|90.14|1.39|5004001|univbrand #2|10|jewelry boxes|6|Jewelry|584|eseeinganti|N/A|03992866483507steel7|powder|Bunch|Unknown|24|n steingeingbarought| +10890|AAAAAAAAIIKCAAAA|2001-10-27||Quickly great subjects move civil societies. Private relations wil|1.32|0.44|10011011|amalgamalgamalg #11|10|disk drives|10|Electronics|584|eseeinganti|N/A|03992866483507steel7|papaya|Dozen|Unknown|83|barn steingbarought| +10891|AAAAAAAALIKCAAAA|1997-10-27||Practical situations produce even so; military, unlike organizations encourage good, great industries. Regular, efficient others cover modern classes. Usually new lives would remember inside to the b|35.61|25.99|4002002|importoedu pack #2|2|mens|4|Shoes|942|ableesen st|medium|88444046661677plum40|indian|Oz|Unknown|45|oughtn steingbarought| +10892|AAAAAAAAMIKCAAAA|1997-10-27|2000-10-26|Commonly economic visitors promote. Aside other voices may make. Outer animals shall cut. Other, solid patients confirm hospitals. Indeed foreign companies work in order. Joint y|2.44|1.19|9010009|univunivamalg #9|10|travel|9|Books|36|callypri|N/A|8897145975221wheat39|indian|Gram|Unknown|14|ablen steingbarought| +10893|AAAAAAAAMIKCAAAA|2000-10-27||Occasional, mean responsi|7.67|1.19|8012004|importomaxi #4|10|guns|8|Sports|36|callypri|N/A|6hot4870440822255165|sky|Pound|Unknown|63|prin steingbarought| +10894|AAAAAAAAOIKCAAAA|1997-10-27|1999-10-27|Just holy purposes fail notably narrow|1.61|0.57|5001001|amalgscholar #1|1|rock|5|Music|186|callyeingought|N/A|086957048296tomato63|puff|Dram|Unknown|18|esen steingbarought| +10895|AAAAAAAAOIKCAAAA|1999-10-28|2001-10-26|Just holy purposes fail notably narrow|1.10|0.67|10011016|amalgamalgamalg #16|1|disk drives|10|Electronics|44|callyeingought|N/A|4607177381053pale814|pink|Lb|Unknown|36|antin steingbarought| +10896|AAAAAAAAOIKCAAAA|2001-10-27||Just holy purposes fail notably narrow|1.65|0.67|10011016|maxinameless #7|9|optics|8|Sports|44|eseese|N/A|969616728violet44803|olive|Ton|Unknown|16|callyn steingbarought| +10897|AAAAAAAABJKCAAAA|1997-10-27||Necessary, normal youngsters become then basic, old packages. Programs paint sharply in addition naked fe|5.91|4.90|1004002|edu packamalg #2|4|swimwear|1|Women|534|eseprianti|N/A|415572432974maroon12|tomato|Tbl|Unknown|15|ationn steingbarought| +10898|AAAAAAAACJKCAAAA|1997-10-27|2000-10-26|Full, new things climb more. Relatively appropriate dreams might not concentrate slow re|2.93|1.64|6004003|edu packcorp #3|4|bracelets|6|Jewelry|458|eingantiese|N/A|0140923299555white09|smoke|Each|Unknown|55|eingn steingbarought| +10899|AAAAAAAACJKCAAAA|2000-10-27||Parental animals need entire requirements. Loca|7.84|1.64|6004003|importonameless #10|4|baseball|8|Sports|458|eingantiese|N/A|0140923299555white09|goldenrod|Ton|Unknown|4|n stn steingbarought| +10900|AAAAAAAAEJKCAAAA|1997-10-27|1999-10-27|Short, total children come points; direct pupils clear strong boxes. Wide things might not budge simply dead, minor settlements. National forms smell |3.56|2.88|5004001|edu packscholar #1|4|classical|5|Music|33|pripri|N/A|58666625peach0774866|tan|Pallet|Unknown|42|barbarn stbarought| +10901|AAAAAAAAEJKCAAAA|1999-10-28|2001-10-26|Short, total children come points; direct pupils clear strong boxes. Wide things might not budge simply dead, minor settlements. National forms smell |3.93|1.88|5004001|exportiimporto #2|4|pants|2|Men|33|pripri|medium|2101turquoise6232861|lavender|Carton|Unknown|12|oughtbarn stbarought| +10902|AAAAAAAAEJKCAAAA|2001-10-27||Short, total children come points; direct pupils clear strong boxes. Wide things might not budge simply dead, minor settlements. National forms smell |0.89|1.88|5004001|edu packbrand #7|14|estate|6|Jewelry|33|pripri|N/A|5087034809279indian3|smoke|Ton|Unknown|49|ablebarn stbarought| +10903|AAAAAAAAHJKCAAAA|||Categories feel responsible, prime groups. Proce||||amalgexporti #2|1|newborn||Children|||||||Unknown|43|pribarn stbarought| +10904|AAAAAAAAIJKCAAAA|1997-10-27|2000-10-26|Occasionally bright markets could provide individual, strong c|2.99|2.60|3002001|importoexporti #1|2|infants|3|Children|145|antieseought|petite|150392512honeydew871|seashell|Dram|Unknown|66|esebarn stbarought| +10905|AAAAAAAAIJKCAAAA|2000-10-27||Occasionally bright markets could provide individual, strong c|3.51|2.60|3002001|exportiimporto #2|3|pants|2|Men|407|antieseought|extra large|150392512honeydew871|sienna|Carton|Unknown|13|antibarn stbarought| +10906|AAAAAAAAKJKCAAAA|1997-10-27|1999-10-27|Small cases used to take then previous, potential imports. American|7.56|5.67|6008003|namelesscorp #3|8|mens watch|6|Jewelry|398|eingn stpri|N/A|5587517766761salmon6|violet|Case|Unknown|43|callybarn stbarought| +10907|AAAAAAAAKJKCAAAA|1999-10-28|2001-10-26|Then personal officials can answer periods. Different, biological wheels give now demanding, aware shoulders. All other conditions allow so nasty systems. Rational, interested stations|9.45|5.57|2001002|amalgimporto #2|1|accessories|2|Men|398|eingn stpri|medium|5wheat18811416031951|yellow|Dram|Unknown|36|ationbarn stbarought| +10908|AAAAAAAAKJKCAAAA|2001-10-27||Never western kinds protect there oth|7.80|5.57|4003001|exportiedu pack #1|3|kids|4|Shoes|47|ationese|small|5wheat18811416031951|lawn|Box|Unknown|46|eingbarn stbarought| +10909|AAAAAAAANJKCAAAA|1997-10-27||Products bring. Authorities bring. Magic parents lea|7.94|3.73|6003006|exporticorp #6|3|gold|6|Jewelry|75|antiation|N/A|08940wheat3102975755|snow|Each|Unknown|99|n stbarn stbarought| +10910|AAAAAAAAOJKCAAAA|1997-10-27|2000-10-26|Eventually individual terms will collapse about also nea|3.64|1.38|5002001|importoscholar #1|2|country|5|Music|883|prieingeing|N/A|16113977coral8024623|linen|Cup|Unknown|10|baroughtn stbarought| +10911|AAAAAAAAOJKCAAAA|2000-10-27||Ever conscious signs press probably territorial, a|3.23|1.38|6010008|univbrand #8|2|jewelry boxes|6|Jewelry|883|prieingeing|N/A|458906violet75079008|blue|Gross|Unknown|55|oughtoughtn stbarought| +10912|AAAAAAAAAKKCAAAA|1997-10-27|1999-10-27|Sharp|5.70|4.27|10010016|univamalgamalg #16|10|memory|10|Electronics|417|ationoughtese|N/A|59205774271621black4|maroon|Pound|Unknown|60|ableoughtn stbarought| +10913|AAAAAAAAAKKCAAAA|1999-10-28|2001-10-26|Good, sad difference|8.05|6.52|2002002|importoimporto #2|10|shirts|2|Men|417|ationoughtese|large|59205774271621black4|snow|Tsp|Unknown|66|prioughtn stbarought| +10914|AAAAAAAAAKKCAAAA|2001-10-27||Problems save simply main, short weeks. Anyway practical orders may rule tiny, complicated patterns; existing, economic horses take ye|8.67|6.52|2002002|importoexporti #1|10|infants|3|Children|417|ationoughtese|extra large|59205774271621black4|peach|Each|Unknown|53|eseoughtn stbarought| +10915|AAAAAAAADKKCAAAA|1997-10-27||Small, solid prices must go indoors. Less past|7.49|5.31|4001002|amalgedu pack #2|1|womens|4|Shoes|467|ationcallyese|medium|74652394peru69160477|firebrick|Gram|Unknown|63|antioughtn stbarought| +10916|AAAAAAAAEKKCAAAA|1997-10-27|2000-10-26|Universities should care as earlier apparent terms; major obligations settle brig|21.29|16.60|4003001|exportiedu pack #1|3|kids|4|Shoes|58|einganti|medium|52487526dim123217818|seashell|Oz|Unknown|35|callyoughtn stbarought| +10917|AAAAAAAAEKKCAAAA|2000-10-27||Painful definitions prove surely facts. Still able years calm neither; si|4.30|3.39|4001002|amalgedu pack #2|3|womens|4|Shoes|162|ablecallyought|petite|52487526dim123217818|smoke|Bunch|Unknown|20|ationoughtn stbarought| +10918|AAAAAAAAGKKCAAAA|1997-10-27|1999-10-27|Both organic shoulders may not become however supporters. Other followers must not go. Residents give old boys; old-fashioned, current questions esca|0.32|0.11|4004001|edu packedu pack #1|4|athletic|4|Shoes|813|prioughteing|medium|1hot8530216675430253|plum|Bundle|Unknown|25|eingoughtn stbarought| +10919|AAAAAAAAGKKCAAAA|1999-10-28|2001-10-26|Both organic shoulders may not become however supporters. Other followers must not go. Residents give old boys; old-fashioned, current questions esca|9.28|7.60|7010010|univnameless #10|10|flatware|7|Home|813|prioughteing|N/A|6493497purple3245911|powder|Box|Unknown|33|n stoughtn stbarought| +10920|AAAAAAAAGKKCAAAA|2001-10-27||Likely, european terms hear. Developments beg initially open clients. Comparative games pick so industrial, prime |10.25|7.60|6015001|scholarbrand #1|10|custom|6|Jewelry|454|eseantiese|N/A|6493497purple3245911|pale|Gram|Unknown|6|barablen stbarought| +10921|AAAAAAAAJKKCAAAA|1997-10-27||Intimate arrangements may sail prese|2.82|1.21|5004002|edu packscholar #2|4|classical|5|Music|293|prin stable|N/A|2238742353500wheat14|peru|Oz|Unknown|45|oughtablen stbarought| +10922|AAAAAAAAKKKCAAAA|1997-10-27|2000-10-26|Whole professionals might protect already so american workers. Pounds may pay subsequent shoes; teachers affect shops. Times restore in a sections; main bands would not escape act|0.16|0.13|3001001|amalgexporti #1|1|newborn|3|Children|289|n steingable|petite|261160light253591959|white|Bundle|Unknown|4|ableablen stbarought| +10923|AAAAAAAAKKKCAAAA|2000-10-27||Hands might expand therefore |3.40|2.85|7014004|edu packnameless #4|1|glassware|7|Home|289|n steingable|N/A|703776454azure290398|bisque|Tbl|Unknown|34|priablen stbarought| +10924|AAAAAAAAMKKCAAAA|1997-10-27|1999-10-27|Other, united methods will put in a plans. Late, desirable others ask christian, minute departments. Afraid, rec|86.21|58.62|4002001|importoedu pack #1|2|mens|4|Shoes|177|ationationought|medium|58626356ivory3823642|hot|Lb|Unknown|33|eseablen stbarought| +10925|AAAAAAAAMKKCAAAA|1999-10-28|2001-10-26|High, present friends hear just british phenomena. Sons pay boys. Long, civil children tolerate almost less useful firms. Christian measurements mind hard services. True, |75.86|58.62|10014012|edu packamalgamalg #12|2|automotive|10|Electronics|177|ationationought|N/A|58626356ivory3823642|antique|Tsp|Unknown|8|antiablen stbarought| +10926|AAAAAAAAMKKCAAAA|2001-10-27||High, present friends hear just british phenomena. Sons pay boys. Long, civil children tolerate almost less useful firms. Christian measurements mind hard services. True, |9.17|4.30|10014012|scholarmaxi #7|15|fishing|8|Sports|606|callybarcally|N/A|508white777299119874|rosy|Carton|Unknown|32|callyablen stbarought| +10927|AAAAAAAAPKKCAAAA|1997-10-27||So narrow pages would not say. Desirable users used to mount now on a eyebrows. Aback inadequate years carry implicit days. Subjects think ever unknown imports. More new ports walk; girls take. |2.68|2.09|10016014|corpamalgamalg #14|16|wireless|10|Electronics|450|barantiese|N/A|119071cornflower3018|rosy|Box|Unknown|11|ationablen stbarought| +10928|AAAAAAAAALKCAAAA|1997-10-27|2000-10-26|Joint places receive with a daughters. No longer contemporary flower|3.28|1.57|6010001|univbrand #1|10|jewelry boxes|6|Jewelry|42|ableese|N/A|5093081tomato9175078|rose|Carton|Unknown|32|eingablen stbarought| +10929|AAAAAAAAALKCAAAA|2000-10-27||Primary difficulties encourage as from a hours; less certain words may not look somewhat at a days. Different years can |9.14|2.74|6010001|maxiunivamalg #16|9|televisions|10|Electronics|42|ableese|N/A|012862556smoke137351|peach|Cup|Unknown|54|n stablen stbarought| +10930|AAAAAAAACLKCAAAA|1997-10-27|1999-10-27|Distinctive, narrow members will think too rules. Teenage, rigid patients occur steadily public, local databases|1.50|1.14|9006011|corpmaxi #11|6|parenting|9|Books|404|esebarese|N/A|394780pale7718400705|misty|Pound|Unknown|57|barprin stbarought| +10931|AAAAAAAACLKCAAAA|1999-10-28|2001-10-26|Highly level children handle hardly young, pleasant periods; environmental, irrelevant advantages can lead as with a times. Here legal arr|4.81|2.45|8010002|univmaxi #2|6|pools|8|Sports|45|antiese|N/A|80031774852pale91384|peru|Dram|Unknown|52|oughtprin stbarought| +10932|AAAAAAAACLKCAAAA|2001-10-27||Compr|7.93|4.12|8013003|exportimaxi #3|6|sailing|8|Sports|743|prieseation|N/A|80031774852pale91384|pink|Each|Unknown|64|ableprin stbarought| +10933|AAAAAAAAFLKCAAAA|1997-10-27||Other, open fees get later. Human pupils could not preserve slowly. Professional things can employ |9.97|7.97|2004002|edu packimporto #2|4|sports-apparel|2|Men|82|ableeing|petite|594khaki164806953127|snow|Tsp|Unknown|59|priprin stbarought| +10934|AAAAAAAAGLKCAAAA|1997-10-27|2000-10-26|Dry developers could not respond even meanwhile gay doctors; hands happen; police|1.09|0.92|5002001|importoscholar #1|2|country|5|Music|184|eseeingought|N/A|445violet23132151479|purple|Case|Unknown|4|eseprin stbarought| +10935|AAAAAAAAGLKCAAAA|2000-10-27||A l|0.14|0.92|6002008|importocorp #8|2|diamonds|6|Jewelry|96|eseeingought|N/A|445violet23132151479|metallic|Gram|Unknown|7|antiprin stbarought| +10936|AAAAAAAAILKCAAAA|1997-10-27|1999-10-27|Possible, advisory conclusions could not reply. Preliminary rooms should provide initiatives. Still constitutional women should take into a chemicals. Well good effects must a|74.38|33.47|9012005|importounivamalg #5|12|home repair|9|Books|163|pricallyought|N/A|27759seashell7989383|peach|Lb|Unknown|24|callyprin stbarought| +10937|AAAAAAAAILKCAAAA|1999-10-28|2001-10-26|Possible, advisory conclusions could not reply. Preliminary rooms should provide initiatives. Still constitutional women should take into a chemicals. Well good effects must a|35.17|23.21|9012005|edu packunivamalg #6|12|sports|9|Books|163|pricallyought|N/A|69aquamarine24561937|saddle|Tsp|Unknown|54|ationprin stbarought| +10938|AAAAAAAAILKCAAAA|2001-10-27||Political penalties shall come quite organisations. Legislative drugs make now straight other services. Great, civil views protect as ne|6.49|23.21|9012005|corpunivamalg #7|12|mystery|9|Books|531|pricallyought|N/A|snow6279513768240658|thistle|Gross|Unknown|29|eingprin stbarought| +10939|AAAAAAAALLKCAAAA|1997-10-27||Groups might begin still howeve|5.26|4.62|1002002|importoamalg #2|2|fragrances|1|Women|640|baresecally|large|7822447266peach15542|yellow|Ounce|Unknown|16|n stprin stbarought| +10940|AAAAAAAAMLKCAAAA|1997-10-27|2000-10-26|Already voluntary children may not appoint very unlikely, p|5.76|2.59|3003001|exportiexporti #1|3|toddlers|3|Children|154|eseantiought|medium|75698247251salmon926|hot|Tsp|Unknown|1|baresen stbarought| +10941|AAAAAAAAMLKCAAAA|2000-10-27||Often possible pat|5.03|2.59|9012010|importounivamalg #10|12|home repair|9|Books|4|eseantiought|N/A|75698247251salmon926|ghost|Dram|Unknown|9|oughtesen stbarought| +10942|AAAAAAAAOLKCAAAA|1997-10-27|1999-10-27|Banks must not go then. Single, tiny feelings can cut beautiful, literary tories|3.25|2.37|4001001|amalgedu pack #1|1|womens|4|Shoes|916|callyoughtn st|small|559blush432204935163|peach|Each|Unknown|37|ableesen stbarought| +10943|AAAAAAAAOLKCAAAA|1999-10-28|2001-10-26|Chief, new systems try. Main, western threats shall add even. Parts drop that short workers. Problems expect for instance. Traditionally aware girl|0.75|0.38|4001001|edu packscholar #2|4|classical|5|Music|916|callyoughtn st|N/A|6015turquoise6299645|tomato|Case|Unknown|43|priesen stbarought| +10944|AAAAAAAAOLKCAAAA|2001-10-27||Chief, new systems try. Main, western threats shall add even. Parts drop that short workers. Problems expect for instance. Traditionally aware girl|4.15|0.38|4001001|importoscholar #1|4|country|5|Music|916|callyoughtn st|N/A|6534864997navajo0020|tomato|N/A|Unknown|1|eseesen stbarought| +10945|AAAAAAAABMKCAAAA|1997-10-27||Effective times sell machines. Comments could not set. British, fresh aspects shall not ensure here young, human organizations. Only, other centres could join in a sections. Clear purposes may|4.00|1.28|8001006|amalgnameless #6|1|athletic shoes|8|Sports|592|ablen stanti|N/A|7585566steel38814494|honeydew|Bunch|Unknown|29|antiesen stbarought| +10946|AAAAAAAACMKCAAAA|1997-10-27|2000-10-26|Movements might not hide still currently net times; times will take equally to a players. Tight small surfaces wear rather by a schools; re|18.10|10.67|6010003|univbrand #3|10|jewelry boxes|6|Jewelry|165|anticallyought|N/A|215062763653100pale7|saddle|Bunch|Unknown|8|callyesen stbarought| +10947|AAAAAAAACMKCAAAA|2000-10-27||Crimes could like very names. Sudden, sorry others run again new years. Domestic, net pe|5.14|4.36|6010003|amalgbrand #8|1|bathroom|7|Home|23|anticallyought|N/A|215062763653100pale7|steel|Bunch|Unknown|93|ationesen stbarought| +10948|AAAAAAAAEMKCAAAA|1997-10-27|1999-10-27|Hot, foreign buyers shall produce there rules. Sorry examples will not cut recently. Human gaps will not contain yesterday at a drugs; long bodies will not stay d|3.06|1.98|10005001|scholarunivamalg #1|5|karoke|10|Electronics|125|antiableought|N/A|33314chartreuse61171|grey|Gross|Unknown|28|eingesen stbarought| +10949|AAAAAAAAEMKCAAAA|1999-10-28|2001-10-26|Ever nearby industries ought to feed best peac|1.32|0.46|10005001|importoedu pack #2|5|mens|4|Shoes|577|ationationanti|extra large|33314chartreuse61171|powder|Tbl|Unknown|27|n stesen stbarought| +10950|AAAAAAAAEMKCAAAA|2001-10-27||International, current lists might mention soon great aspects. Likely, african children see more labour, social men. Full terms ought to hide already to|3.88|1.35|10005001|exportiedu pack #1|3|kids|4|Shoes|577|ationationanti|small|33314chartreuse61171|orange|Bunch|Unknown|35|barantin stbarought| +10951|AAAAAAAAHMKCAAAA|1997-10-27||Closely new stories d|86.12|64.59|5001002|amalgscholar #2|1|rock|5|Music|174|eseationought|N/A|85208898540594peru95|maroon|Unknown|Unknown|32|oughtantin stbarought| +10952|AAAAAAAAIMKCAAAA|1997-10-27|2000-10-26|Local resources determine biological customers. Small, primary customers will bother to a students. Schemes make also. New, level demands teac|4.22|1.51|2003001|exportiimporto #1|3|pants|2|Men|644|eseesecally|small|85587796642plum08222|indian|Gram|Unknown|30|ableantin stbarought| +10953|AAAAAAAAIMKCAAAA|2000-10-27||Attractive banks must blow high tales. As internal demands can ask voices; so certain courses develop well unaware arrangements. Relationships should not carry worth a difficulties. Very german stars |6.89|1.51|2003001|edu packamalg #2|4|swimwear|1|Women|644|eseesecally|medium|85587796642plum08222|dodger|Each|Unknown|17|priantin stbarought| +10954|AAAAAAAAKMKCAAAA|1997-10-27|1999-10-27|Sometimes physical theories allow ever differences. Crucial, common things can land often high, increased children. Apart european troops pay easily problems. More clear descriptions m|4.09|3.35|9005011|scholarmaxi #11|5|history|9|Books|161|oughtcallyought|N/A|298493803powder38092|rosy|Bundle|Unknown|22|eseantin stbarought| +10955|AAAAAAAAKMKCAAAA|1999-10-28|2001-10-26|Possible copies help religious a|4.79|3.35|9005011|maxinameless #6|5|optics|8|Sports|161|oughtcallyought|N/A|26280962258violet831|olive|Gross|Unknown|20|antiantin stbarought| +10956|AAAAAAAAKMKCAAAA|2001-10-27||Possible copies help religious a|5.08|3.35|9005011|edu packscholar #1|5|classical|5|Music|161|oughtcallyought|N/A|273627light461497697|violet|Carton|Unknown|74|callyantin stbarought| +10957|AAAAAAAANMKCAAAA|1997-10-27||Royal strategies shall not go daughters. Absent, coming courses will cross then badly interesting details. Views clear on board in a relations. Streets make social c|0.22|0.14|1002002|importoamalg #2|2|fragrances|1|Women|633|priprically|large|691311773483white667|rose|Pallet|Unknown|26|ationantin stbarought| +10958|AAAAAAAAOMKCAAAA|1997-10-27|2000-10-26|Seriously civil managers make. Sure, rapid interests call only po|7.77|3.49|3003001|exportiexporti #1|3|toddlers|3|Children|985|antieingn st|petite|78snow91351354743565|thistle|Tbl|Unknown|38|eingantin stbarought| +10959|AAAAAAAAOMKCAAAA|2000-10-27||Mental negotiations must explain sure. At last long kids overcome|2.67|0.93|5002002|importoscholar #2|2|country|5|Music|640|antieingn st|N/A|78snow91351354743565|purple|Ounce|Unknown|59|n stantin stbarought| +10960|AAAAAAAAANKCAAAA|1997-10-27|1999-10-27|There old offences help sensitive hands. New eyes stop nevertheless local arms. Phase|9.33|6.53|5004001|edu packscholar #1|4|classical|5|Music|11|oughtought|N/A|2powder5700864079045|steel|Bundle|Unknown|14|barcallyn stbarought| +10961|AAAAAAAAANKCAAAA|1999-10-28|2001-10-26|There old offences help sensitive hands. New eyes stop nevertheless local arms. Phase|6.84|6.53|10003014|exportiunivamalg #14|4|dvd/vcr players|10|Electronics|162|ablecallyought|N/A|0636657387sandy19501|pink|Cup|Unknown|18|oughtcallyn stbarought| +10962|AAAAAAAAANKCAAAA|2001-10-27||There old offences help sensitive hands. New eyes stop nevertheless local arms. Phase|1.12|6.53|10003014|edu packbrand #3|4|curtains/drapes|7|Home|624|ablecallyought|N/A|463119567smoke239433|powder|Lb|Unknown|32|ablecallyn stbarought| +10963|AAAAAAAADNKCAAAA|1997-10-27||Prime, industr|3.43|1.13|10014016|edu packamalgamalg #16|14|automotive|10|Electronics|288|eingeingable|N/A|43peru10645936959921|hot|Oz|Unknown|11|pricallyn stbarought| +10964|AAAAAAAAENKCAAAA|1997-10-27|2000-10-26|Prime funds reproduce bad|6.43|4.75|3003001|exportiexporti #1|3|toddlers|3|Children|658|eingantically|extra large|398sandy358179288808|peru|Gram|Unknown|88|esecallyn stbarought| +10965|AAAAAAAAENKCAAAA|2000-10-27||Prime funds reproduce bad|3.26|4.75|2001002|amalgimporto #2|3|accessories|2|Men|885|antieingeing|extra large|398sandy358179288808|turquoise|Unknown|Unknown|46|anticallyn stbarought| +10966|AAAAAAAAGNKCAAAA|1997-10-27|1999-10-27|Perhaps necessary performances might visit. Fundamental terms might avoid. More regional opportunities may not decline then successful features. Already current ideas will not think mor|0.89|0.71|6012007|importobrand #7|12|costume|6|Jewelry|236|callypriable|N/A|851934372041papaya52|seashell|Unknown|Unknown|24|callycallyn stbarought| +10967|AAAAAAAAGNKCAAAA|1999-10-28|2001-10-26|Plants used to produce already true effects. Small things used to trace now problems. Reductions might not go ago basic years. Confident, reliable men ask i|4.66|2.70|6012007|corpnameless #8|12|furniture|7|Home|43|priese|N/A|8867742285pale391208|rosy|Case|Unknown|8|ationcallyn stbarought| +10968|AAAAAAAAGNKCAAAA|2001-10-27||Chinese, available camps see even to a pupils. Thousands would make superficially local, english points; genuine, live words may admit environmental e|1.79|2.70|7008009|namelessbrand #9|8|lighting|7|Home|993|prin stn st|N/A|8867742285pale391208|red|Unknown|Unknown|41|eingcallyn stbarought| +10969|AAAAAAAAJNKCAAAA|1997-10-27||Perfectly complete |8.04|2.97|5001002|amalgscholar #2|1|rock|5|Music|334|esepripri|N/A|81sandy7045144154571|papaya|Gross|Unknown|23|n stcallyn stbarought| +10970|AAAAAAAAKNKCAAAA|1997-10-27|2000-10-26|Similar efforts may not escort heavily national, black hands. Uncertain months keep including the personnel. Strange things tell details; pop|1.54|1.27|6010003|univbrand #3|10|jewelry boxes|6|Jewelry|189|n steingought|N/A|195468756557sandy618|powder|Tbl|Unknown|25|barationn stbarought| +10971|AAAAAAAAKNKCAAAA|2000-10-27||Accountants use sensitive, happy skills. Basic travellers clean losses. Nice, particular details shall wonder there local ministers. Hitherto personal exports find primarily therefore other|5.00|1.27|10007007|brandunivamalg #7|7|personal|10|Electronics|189|n steingought|N/A|526143wheat998603213|pink|N/A|Unknown|43|oughtationn stbarought| +10972|AAAAAAAAMNKCAAAA|1997-10-27|1999-10-27|Also likely patients may write. Firm drugs fetch very into a travellers.|4.04|3.23|5003001|exportischolar #1|3|pop|5|Music|150|barantiought|N/A|6043495969sienna4352|ivory|Carton|Unknown|23|ableationn stbarought| +10973|AAAAAAAAMNKCAAAA|1999-10-28|2001-10-26|Also likely patients may write. Firm drugs fetch very into a travellers.|4.28|3.42|5003001|importonameless #4|2|baseball|8|Sports|150|barantiought|N/A|6043495969sienna4352|moccasin|Cup|Unknown|32|priationn stbarought| +10974|AAAAAAAAMNKCAAAA|2001-10-27||Political modes choose main institutions. I|4.41|3.42|5003001|importomaxi #7|2|business|9|Books|150|barantiought|N/A|5401540864spring0752|purple|Each|Unknown|10|eseationn stbarought| +10975|AAAAAAAAPNKCAAAA|1997-10-27||Regardless unable services go vehicles; in order western standards may curtail hardly scientists; cou|2.33|2.00|8005006|scholarnameless #6|5|fitness|8|Sports|226|callyableable|N/A|03677585528salmon374|puff|Case|Unknown|27|antiationn stbarought| +10976|AAAAAAAAAOKCAAAA|1997-10-27|2000-10-26|Sections shall need there even outsid|9.81|7.35|5002001|importoscholar #1|2|country|5|Music|68|eingcally|N/A|438510264tan86773189|violet|N/A|Unknown|57|callyationn stbarought| +10977|AAAAAAAAAOKCAAAA|2000-10-27||New speeches talk good, serious events; hardly new trousers would not meet with a officials; international, |2.27|1.24|7002008|importobrand #8|2|bedding|7|Home|11|eingcally|N/A|438510264tan86773189|sandy|Each|Unknown|52|ationationn stbarought| +10978|AAAAAAAACOKCAAAA|1997-10-27|1999-10-27|Political, alive fragments recognize then friendly ladies. Domestic, top proposals monitor never |1.55|0.55|5003001|exportischolar #1|3|pop|5|Music|244|eseeseable|N/A|34slate2886457160236|peru|Oz|Unknown|30|eingationn stbarought| +10979|AAAAAAAACOKCAAAA|1999-10-28|2001-10-26|Political, alive fragments recognize then friendly ladies. Domestic, top proposals monitor never |4.08|0.55|5001002|amalgscholar #2|1|rock|5|Music|335|antipripri|N/A|8081805wheat73138949|light|Bunch|Unknown|44|n stationn stbarought| +10980|AAAAAAAACOKCAAAA|2001-10-27||Familiar, modern fingers shall catch virtually new posts; different, natural networks might|88.32|74.18|5001002|edu packexporti #1|1|school-uniforms|3|Children|335|antipripri|large|8081805wheat73138949|light|Pallet|Unknown|38|bareingn stbarought| +10981|AAAAAAAAFOKCAAAA|1997-10-27||Women go in a musicians. Different, english months would not help fair primary, long arts; pounds ought to|3.06|1.49|1001002|amalgamalg #2|1|dresses|1|Women|33|pripri|economy|440turquoise24685617|sienna|Ounce|Unknown|13|oughteingn stbarought| +10982|AAAAAAAAGOKCAAAA|1997-10-27|2000-10-26|British loans balance issues. Only new measures help always worldwide serious equations. Western |0.35|0.30|4004001|edu packedu pack #1|4|athletic|4|Shoes|136|callypriought|petite|turquoise60650074855|white|Dozen|Unknown|9|ableeingn stbarought| +10983|AAAAAAAAGOKCAAAA|2000-10-27||British loans balance issues. Only new measures help always worldwide serious equations. Western |7.04|2.25|4004001|amalgamalg #2|4|dresses|1|Women|136|callypriought|medium|turquoise60650074855|misty|Each|Unknown|35|prieingn stbarought| +10984|AAAAAAAAIOKCAAAA|1997-10-27|1999-10-27|Pr|27.24|17.43|10006003|corpunivamalg #3|6|musical|10|Electronics|584|eseeinganti|N/A|powder45682327554299|indian|Tbl|Unknown|69|eseeingn stbarought| +10985|AAAAAAAAIOKCAAAA|1999-10-28|2001-10-26|Pr|5.69|3.58|1003002|exportiamalg #2|3|maternity|1|Women|27|eseeinganti|medium|powder45682327554299|light|Dozen|Unknown|3|antieingn stbarought| +10986|AAAAAAAAIOKCAAAA|2001-10-27||Tears will abolish significant, visible moments. National samples go natural, modern arguments. Apparent, local places relax shortly left times. Regional, rich facts live. Popu|54.15|3.58|1003002|amalgcorp #3|3|birdal|6|Jewelry|27|ationable|N/A|353black402762073974|red|Dram|Unknown|23|callyeingn stbarought| +10987|AAAAAAAALOKCAAAA|1997-10-27||Wounds establis|9.31|4.84|4003002|exportiedu pack #2|3|kids|4|Shoes|252|ableantiable|medium|5398894563brown09204|beige|Bunch|Unknown|65|ationeingn stbarought| +10988|AAAAAAAAMOKCAAAA|1997-10-27|2000-10-26|Fa|7.37|3.46|9014009|edu packunivamalg #9|14|sports|9|Books|558|eingantianti|N/A|9005859995green97032|linen|Bundle|Unknown|29|eingeingn stbarought| +10989|AAAAAAAAMOKCAAAA|2000-10-27||Fa|6.85|3.46|1001002|amalgamalg #2|1|dresses|1|Women|558|eingantianti|petite|9005859995green97032|pale|Unknown|Unknown|28|n steingn stbarought| +10990|AAAAAAAAOOKCAAAA|1997-10-27|1999-10-27|Necessarily li|1.49|0.75|2002001|importoimporto #1|2|shirts|2|Men|80|bareing|extra large|956343338924peach161|red|Oz|Unknown|27|barn stn stbarought| +10991|AAAAAAAAOOKCAAAA|1999-10-28|2001-10-26|Necessarily li|3.47|0.75|2002001|exportiimporto #2|2|pants|2|Men|222|ableableable|large|956343338924peach161|orange|Each|Unknown|11|oughtn stn stbarought| +10992|AAAAAAAAOOKCAAAA|2001-10-27||Necessarily li|0.35|0.12|2002001|importoimporto #1|2|shirts|2|Men|727|ableableable|large|956343338924peach161|linen|Unknown|Unknown|57|ablen stn stbarought| +10993|AAAAAAAABPKCAAAA|1997-10-27||Good, sure structures involve main branches. Again |3.23|1.38|6003002|exporticorp #2|3|gold|6|Jewelry|276|callyationable|N/A|5116914483661papaya9|royal|Ton|Unknown|45|prin stn stbarought| +10994|AAAAAAAACPKCAAAA|1997-10-27|2000-10-26|Leaders encourage; estimated, new changes could not allow old, rough trials|3.47|2.70|2004001|edu packimporto #1|4|sports-apparel|2|Men|469|n stcallyese|large|8732060sienna2882850|green|Pound|Unknown|14|esen stn stbarought| +10995|AAAAAAAACPKCAAAA|2000-10-27||Early, deep firms|92.08|2.70|2003002|exportiimporto #2|3|pants|2|Men|179|n stcallyese|medium|8732060sienna2882850|royal|Ounce|Unknown|9|antin stn stbarought| +10996|AAAAAAAAEPKCAAAA|1997-10-27|1999-10-27|Even pleasant manufacturers win merely tall, good assessments. Foreign, only months used to put thus |4.55|3.91|7015007|scholarnameless #7|15|tables|7|Home|96|callyn st|N/A|409739413371wheat449|spring|Oz|Unknown|58|callyn stn stbarought| +10997|AAAAAAAAEPKCAAAA|1999-10-28|2001-10-26|Residential, little aims should confront all real facilities. Broadly open ages turn backwards proposa|1.49|3.91|7015007|edu packbrand #8|15|curtains/drapes|7|Home|96|callyn st|N/A|98704cornflower98749|purple|Each|Unknown|80|ationn stn stbarought| +10998|AAAAAAAAEPKCAAAA|2001-10-27||No longer following examples ought to relieve model time|5.19|3.91|7015007|maxicorp #7|15|womens watch|6|Jewelry|91|callyn st|N/A|98704cornflower98749|papaya|Pallet|Unknown|5|eingn stn stbarought| +10999|AAAAAAAAHPKCAAAA|1997-10-27||Civil, present paintings formulate once hard healthy issues. Shortly old men improve enough cars. Equa|2.54|1.70|5001002|amalgscholar #2|1|rock|5|Music|405|antibarese|N/A|75111026846ghost9448|medium|Carton|Unknown|15|n stn stn stbarought| +11000|AAAAAAAAIPKCAAAA|1997-10-27|2000-10-26|Main, new experts must not seem; enough poor projects spend now social, tory institutions. Contributions win; early customers appear never human circumstances. Interests could |6.12|4.16|3003001|exportiexporti #1|3|toddlers|3|Children|231|oughtpriable|small|97402papaya035436751|sandy|Carton|Unknown|4|barbarbaroughtought| +11001|AAAAAAAAIPKCAAAA|2000-10-27||Individual clients require maybe. Holidays should not guide |3.36|2.72|3003001|brandnameless #2|3|hockey|8|Sports|686|callyeingcally|N/A|97402papaya035436751|red|Tsp|Unknown|6|oughtbarbaroughtought| +11002|AAAAAAAAKPKCAAAA|1997-10-27|1999-10-27|Pupils change. Frequently nice rates shall not decide future yards. Over upper girls ought to lower in a developments. Formal|2.19|1.16|3004001|edu packexporti #1|4|school-uniforms|3|Children|521|oughtableanti|large|865347758223rosy5462|slate|Bundle|Unknown|31|ablebarbaroughtought| +11003|AAAAAAAAKPKCAAAA|1999-10-28|2001-10-26|Pupils change. Frequently nice rates shall not decide future yards. Over upper girls ought to lower in a developments. Formal|2.04|1.16|3004002|edu packexporti #2|4|school-uniforms|3|Children|521|oughtableanti|large|11403gainsboro051325|pale|Gross|Unknown|44|pribarbaroughtought| +11004|AAAAAAAAKPKCAAAA|2001-10-27||Possib|3.12|1.16|3004002|importoexporti #1|4|infants|3|Children|296|callyn stable|large|0307409422307slate47|rose|Case|Unknown|21|esebarbaroughtought| +11005|AAAAAAAANPKCAAAA|1997-10-27||Over small forms bring only active facilities. Now complete resources go almost bad shops. Correct years should trace enormous grounds. Sweet acres might allow techniques. |90.21|42.39|4004002|edu packedu pack #2|4|athletic|4|Shoes|327|ationablepri|large|360894966460green708|cornflower|Oz|Unknown|11|antibarbaroughtought| +11006|AAAAAAAAOPKCAAAA|1997-10-27|2000-10-26|Less imp|9.12|8.20|9001003|amalgmaxi #3|1|arts|9|Books|38|eingpri|N/A|8463565091477violet3|papaya|Oz|Unknown|40|callybarbaroughtought| +11007|AAAAAAAAOPKCAAAA|2000-10-27||Countries perform henceforth. Democrats see. Public me|3.31|1.68|5001002|amalgscholar #2|1|rock|5|Music|38|eingpri|N/A|8463565091477violet3|pink|Unknown|Unknown|26|ationbarbaroughtought| +11008|AAAAAAAAAALCAAAA|1997-10-27|1999-10-27|More than very services could not feel away issues. Limited |5.63|3.88|4004001|edu packedu pack #1|4|athletic|4|Shoes|350|barantipri|extra large|3933light31381438749|goldenrod|Each|Unknown|23|eingbarbaroughtought| +11009|AAAAAAAAAALCAAAA|1999-10-28|2001-10-26|Companies must say equally too gastric looks. Key men note even so foreign|4.67|3.88|7004010|edu packbrand #10|4|curtains/drapes|7|Home|269|n stcallyable|N/A|594sky06984816551245|rosy|Dram|Unknown|11|n stbarbaroughtought| +11010|AAAAAAAAAALCAAAA|2001-10-27||Final, ready numbers |4.97|3.88|6011003|amalgbrand #3|11|semi-precious|6|Jewelry|269|n stcallyable|N/A|16022saddle200110630|lemon|Gross|Unknown|3|baroughtbaroughtought| +11011|AAAAAAAADALCAAAA|1997-10-27||Shortly desperate nat|8.20|6.88|6008004|namelesscorp #4|8|mens watch|6|Jewelry|278|eingationable|N/A|594447327dodger22278|brown|Dozen|Unknown|32|oughtoughtbaroughtought| +11012|AAAAAAAAEALCAAAA|1997-10-27|2000-10-26|Constant, main examinations will not say benefits. Common, dependen|3.42|3.00|4004001|edu packedu pack #1|4|athletic|4|Shoes|133|pripriought|large|38290080smoke4194199|goldenrod|Case|Unknown|59|ableoughtbaroughtought| +11013|AAAAAAAAEALCAAAA|2000-10-27||Necessary, insufficient rights secure ever forms; effects take with|6.27|3.63|4004001|exportiimporto #2|4|pants|2|Men|519|n stoughtanti|large|6azure20609045614403|papaya|Pound|Unknown|29|prioughtbaroughtought| +11014|AAAAAAAAGALCAAAA|1997-10-27|1999-10-27|Cars might feel. Available, controversial players draw readily new events; official, old patterns could not monitor quite little aspect|96.43|83.89|1003001|exportiamalg #1|3|maternity|1|Women|157|ationantiought|petite|2149942699446bisque4|tomato|Each|Unknown|60|eseoughtbaroughtought| +11015|AAAAAAAAGALCAAAA|1999-10-28|2001-10-26|Biological communities might practise good, clear words; figures might not find easily proper, old heroes. American changes like flat partners. Differ|4.64|2.50|1003001|edu packamalgamalg #17|3|automotive|10|Electronics|456|callyantiese|N/A|turquoise33200193316|forest|Bundle|Unknown|19|antioughtbaroughtought| +11016|AAAAAAAAGALCAAAA|2001-10-27||Biological communities might practise good, clear words; figures might not find easily proper, old heroes. American changes like flat partners. Differ|89.61|2.50|8011007|amalgmaxi #7|3|archery|8|Sports|63|callyantiese|N/A|539687209seashell732|royal|Carton|Unknown|87|callyoughtbaroughtought| +11017|AAAAAAAAJALCAAAA|1997-10-27||Natural, only views follow etc young, parental parents. Great, clinical flats gro|1.01|0.87|1001002|amalgamalg #2|1|dresses|1|Women|266|callycallyable|extra large|291598586675yellow05|red|Ton|Unknown|31|ationoughtbaroughtought| +11018|AAAAAAAAKALCAAAA|1997-10-27|2000-10-26|Small players shall tend often other, conservative demands. Particular, common resources keep for example prime hours; politically little customers look much. New, in|1.86|0.83|4001001|amalgedu pack #1|1|womens|4|Shoes|136|callypriought|petite|5146548688seashell87|violet|Tbl|Unknown|42|eingoughtbaroughtought| +11019|AAAAAAAAKALCAAAA|2000-10-27||Interior, old groups care strong boys. Here complex children shall not show. Things show seriously. Before senior notes us|2.11|0.83|3003002|exportiexporti #2|3|toddlers|3|Children|136|callypriought|medium|357409salmon41681712|yellow|N/A|Unknown|47|n stoughtbaroughtought| +11020|AAAAAAAAMALCAAAA|1997-10-27|1999-10-27|Just difficult criteria used to find t|0.28|0.21|10011005|amalgamalgamalg #5|11|disk drives|10|Electronics|151|oughtantiought|N/A|royal213196812349423|misty|Each|Unknown|2|barablebaroughtought| +11021|AAAAAAAAMALCAAAA|1999-10-28|2001-10-26|Liable, round ends will become democratic, prime pressures. Old, similar days ought to achieve only with a ideas. Wives concentrate|38.63|0.21|7007002|brandbrand #2|11|decor|7|Home|246|oughtantiought|N/A|royal213196812349423|rosy|Cup|Unknown|3|oughtablebaroughtought| +11022|AAAAAAAAMALCAAAA|2001-10-27||Functions get indeed. Hard bad supplies might sa|4.33|0.21|4003001|exportiedu pack #1|11|kids|4|Shoes|246|oughtantiought|extra large|8rose603665568580889|pink|Oz|Unknown|56|ableablebaroughtought| +11023|AAAAAAAAPALCAAAA|1997-10-27||Other, practical sp|4.94|2.91|2002002|importoimporto #2|2|shirts|2|Men|312|ableoughtpri|small|876188478268608peru5|royal|Gram|Unknown|7|priablebaroughtought| +11024|AAAAAAAAABLCAAAA|1997-10-27|2000-10-26|Again extreme years see weekly front days. Regardless future things serve just actually competent consequences. Different groups will keep severe, long years; main connections include |2.60|1.92|3004001|edu packexporti #1|4|school-uniforms|3|Children|119|n stoughtought|medium|373695342007steel064|papaya|Gross|Unknown|3|eseablebaroughtought| +11025|AAAAAAAAABLCAAAA|2000-10-27||Again extreme years see weekly front days. Regardless future things serve just actually competent consequences. Different groups will keep severe, long years; main connections include |2.92|1.92|9005010|scholarmaxi #10|4|history|9|Books|202|ablebarable|N/A|81711531942242rosy85|dodger|Case|Unknown|36|antiablebaroughtought| +11026|AAAAAAAACBLCAAAA|1997-10-27|1999-10-27|Right difficulties feed too directly medieval years. Vocational services see here; abroad sure relationships would sit against the principles; injuries would not assist bare, safe adve|5.98|5.14|9005011|scholarmaxi #11|5|history|9|Books|186|callyeingought|N/A|90422423chartreuse54|plum|N/A|Unknown|61|callyablebaroughtought| +11027|AAAAAAAACBLCAAAA|1999-10-28|2001-10-26|Characters should not appear even products. Black, american officials may sen|5.15|5.14|9007012|brandmaxi #12|5|reference|9|Books|186|callyeingought|N/A|90422423chartreuse54|lavender|Unknown|Unknown|3|ationablebaroughtought| +11028|AAAAAAAACBLCAAAA|2001-10-27||Important brothers ought to|1.80|5.14|9007012|exportiedu pack #1|3|kids|4|Shoes|186|callyeingought|petite|8992467536505rosy740|honeydew|Gross|Unknown|24|eingablebaroughtought| +11029|AAAAAAAAFBLCAAAA|1997-10-27||As conventional agencies make less than complete, bright officers. Of cou|4.28|3.80|10003014|exportiunivamalg #14|3|dvd/vcr players|10|Electronics|816|callyoughteing|N/A|982899341219white602|royal|Pallet|Unknown|55|n stablebaroughtought| +11030|AAAAAAAAGBLCAAAA|1997-10-27|2000-10-26|Arts must reduce probably territorial methods. Green, econ|0.14|0.04|6003007|exporticorp #7|3|gold|6|Jewelry|583|prieinganti|N/A|495962430439tan82284|turquoise|Ounce|Unknown|13|barpribaroughtought| +11031|AAAAAAAAGBLCAAAA|2000-10-27||Arts must reduce probably territorial methods. Green, econ|1.55|1.37|10005016|scholarunivamalg #16|3|karoke|10|Electronics|583|prieinganti|N/A|495962430439tan82284|dark|Each|Unknown|28|oughtpribaroughtought| +11032|AAAAAAAAIBLCAAAA|1997-10-27|1999-10-27|Intensely equal years match philosophical, american colours. International, male terms suggest around. Violently only interests would incorporate with a steps. Publi|31.06|13.35|1003001|exportiamalg #1|3|maternity|1|Women|759|n stantiation|extra large|743390610pale7663943|saddle|Pallet|Unknown|9|ablepribaroughtought| +11033|AAAAAAAAIBLCAAAA|1999-10-28|2001-10-26|Even christian schools continue into the thoughts. Usually easy patients give for the areas. Woods adopt nations. Significant, significant jobs|5.77|13.35|6015002|scholarbrand #2|15|custom|6|Jewelry|759|n stantiation|N/A|743390610pale7663943|sandy|Carton|Unknown|52|pripribaroughtought| +11034|AAAAAAAAIBLCAAAA|2001-10-27||Real duties might see at least weapons; small mice serve faint seats. British, pink ways call in a examples. Restrictions reconcile; gifts must secure. Safe forms will not obtain cool re|96.76|71.60|6015002|exportischolar #1|15|pop|5|Music|422|ableableese|N/A|60663turquoise918674|saddle|Dozen|Unknown|75|esepribaroughtought| +11035|AAAAAAAALBLCAAAA|1997-10-27||Occasions can view so customers. Likely hospitals jo|74.97|53.97|9010008|univunivamalg #8|10|travel|9|Books|132|ablepriought|N/A|596959232155tomato31|powder|Gram|Unknown|5|antipribaroughtought| +11036|AAAAAAAAMBLCAAAA|1997-10-27|2000-10-26|Various frames cannot prevent in a jeans. Different, global processes decide all middle|6.28|4.89|4004001|edu packedu pack #1|4|athletic|4|Shoes|718|eingoughtation|petite|690002456powder58772|peach|Box|Unknown|8|callypribaroughtought| +11037|AAAAAAAAMBLCAAAA|2000-10-27||Various frames cannot prevent in a jeans. Different, global processes decide all middle|0.34|4.89|7004008|edu packbrand #8|4|curtains/drapes|7|Home|718|eingoughtation|N/A|690002456powder58772|black|Cup|Unknown|16|ationpribaroughtought| +11038|AAAAAAAAOBLCAAAA|1997-10-27|1999-10-27|Large, industrial services bring accurately cultural, only men. Products may abandon m|3.05|0.94|6002007|importocorp #7|2|diamonds|6|Jewelry|510|baroughtanti|N/A|sandy287100218856060|khaki|Dram|Unknown|20|eingpribaroughtought| +11039|AAAAAAAAOBLCAAAA|1999-10-28|2001-10-26|Hot scots feel again el|1.17|0.94|3003002|exportiexporti #2|3|toddlers|3|Children|510|baroughtanti|petite|sandy287100218856060|moccasin|Unknown|Unknown|51|n stpribaroughtought| +11040|AAAAAAAAOBLCAAAA|2001-10-27||Hot scots feel again el|3.77|3.05|3003002|univamalgamalg #8|10|memory|10|Electronics|359|n stantipri|N/A|sandy287100218856060|violet|Case|Unknown|8|baresebaroughtought| +11041|AAAAAAAABCLCAAAA|1997-10-27||Popular costs help never so essential years. Commercial children cannot assume below requirements. Normal purposes shall help al|3.01|1.23|7015002|scholarnameless #2|15|tables|7|Home|308|eingbarpri|N/A|58steel0977203560662|orange|Box|Unknown|2|oughtesebaroughtought| +11042|AAAAAAAACCLCAAAA|1997-10-27|2000-10-26|Multiple, personal attitudes change so. Major, international companies can give scales. Strong women may take there expensive scores|45.80|14.65|9001003|amalgmaxi #3|1|arts|9|Books|364|esecallypri|N/A|7263294white31373918|white|Unknown|Unknown|60|ableesebaroughtought| +11043|AAAAAAAACCLCAAAA|2000-10-27||Multiple, personal attitudes change so. Major, international companies can give scales. Strong women may take there expensive scores|1.03|0.55|9001003|amalgamalgamalg #11|11|disk drives|10|Electronics|364|esecallypri|N/A|7263294white31373918|powder|Pound|Unknown|43|priesebaroughtought| +11044|AAAAAAAAECLCAAAA|1997-10-27|1999-10-27|Subsequent ideas rely extra as a eyes. Separate, surprised trees call then old, well-known |8.68|7.46|2003001|exportiimporto #1|3|pants|2|Men|225|antiableable|medium|4301505saddle1992776|purple|Tsp|Unknown|27|eseesebaroughtought| +11045|AAAAAAAAECLCAAAA|1999-10-28|2001-10-26|Subsequent ideas rely extra as a eyes. Separate, surprised trees call then old, well-known |0.39|0.28|5004002|edu packscholar #2|3|classical|5|Music|225|antiableable|N/A|755tomato74754215988|seashell|Gram|Unknown|28|antiesebaroughtought| +11046|AAAAAAAAECLCAAAA|2001-10-27||Ago difficult seats reveal extensive, short requirements; voluntary, formal concerns feel still only attractive schools; there only p|4.60|0.28|7013007|exportinameless #7|3|wallpaper|7|Home|344|antiableable|N/A|755tomato74754215988|brown|Carton|Unknown|17|callyesebaroughtought| +11047|AAAAAAAAHCLCAAAA|1997-10-27||Pools ought to get progressive, yellow workers. Effectively new women can convict fully. Kids would not know minutes. Friendly forests read educational, possible products; very, politic|9.73|8.65|4002002|importoedu pack #2|2|mens|4|Shoes|797|ationn station|extra large|68515939096yellow912|sienna|Pallet|Unknown|21|ationesebaroughtought| +11048|AAAAAAAAICLCAAAA|1997-10-27|2000-10-26|Letters may produce quite natural, available elections. Important, white sides |7.28|2.18|9009009|maximaxi #9|9|science|9|Books|376|callyationpri|N/A|moccasin066800906124|powder|Each|Unknown|30|eingesebaroughtought| +11049|AAAAAAAAICLCAAAA|2000-10-27||Yesterday local countries choose states. Potential workers sleep regular, alternative minutes. Necessarily average times look greatly|0.48|2.18|9009009|importoedu pack #2|9|mens|4|Shoes|376|callyationpri|petite|moccasin066800906124|steel|Gross|Unknown|9|n stesebaroughtought| +11050|AAAAAAAAKCLCAAAA|1997-10-27|1999-10-27|Less neutral environments confirm also international, natural times. Less chronic restrictions see instead charts. All good managers tell. Whole, de|57.92|25.48|10006001|corpunivamalg #1|6|musical|10|Electronics|135|antipriought|N/A|9180powder9589716482|goldenrod|Gross|Unknown|28|barantibaroughtought| +11051|AAAAAAAAKCLCAAAA|1999-10-28|2001-10-26|Less neutral environments confirm also international, natural times. Less chronic restrictions see instead charts. All good managers tell. Whole, de|5.75|2.93|10006001|edu packnameless #2|6|glassware|7|Home|135|antipriought|N/A|9180powder9589716482|salmon|Pound|Unknown|17|oughtantibaroughtought| +11052|AAAAAAAAKCLCAAAA|2001-10-27||Less neutral environments confirm also international, natural times. Less chronic restrictions see instead charts. All good managers tell. Whole, de|9.61|2.93|5001001|amalgscholar #1|6|rock|5|Music|135|antipriought|N/A|9180powder9589716482|midnight|Ton|Unknown|23|ableantibaroughtought| +11053|AAAAAAAANCLCAAAA|1997-10-27||Cells need especially from a things; only urban paths bring indeed with a changes. Also reduced bands dominate accessible structures. Independent persons will know e|7.71|4.31|6011006|amalgbrand #6|11|semi-precious|6|Jewelry|680|bareingcally|N/A|14violet051163829094|powder|Each|Unknown|17|priantibaroughtought| +11054|AAAAAAAAOCLCAAAA|1997-10-27|2000-10-26|Federal|4.73|1.75|1004001|edu packamalg #1|4|swimwear|1|Women|104|esebarought|medium|thistle4843230619320|seashell|Gram|Unknown|16|eseantibaroughtought| +11055|AAAAAAAAOCLCAAAA|2000-10-27||Federal|2.34|1.24|3001002|amalgexporti #2|4|newborn|3|Children|104|esebarought|extra large|thistle4843230619320|indian|Dozen|Unknown|1|antiantibaroughtought| +11056|AAAAAAAAADLCAAAA|1997-10-27|1999-10-27|Affairs must work here with a arms. Rules should equip high problems. Little, confident studies should cover ways. Groups may give late communist proble|3.27|2.64|5002001|importoscholar #1|2|country|5|Music|507|ationbaranti|N/A|2203grey209578335932|sienna|Gross|Unknown|1|callyantibaroughtought| +11057|AAAAAAAAADLCAAAA|1999-10-28|2001-10-26|Structural police greet early to the lives; religious, willing soldiers should not meet also. Red, old soldiers should not thank natural, short-term stud|7.02|2.64|6008002|namelesscorp #2|2|mens watch|6|Jewelry|523|ationbaranti|N/A|78520papaya029256417|turquoise|Tsp|Unknown|4|ationantibaroughtought| +11058|AAAAAAAAADLCAAAA|2001-10-27||Structural police greet early to the lives; religious, willing soldiers should not meet also. Red, old soldiers should not thank natural, short-term stud|1.08|2.64|4002001|importoedu pack #1|2|mens|4|Shoes|339|ationbaranti|medium|78520papaya029256417|mint|Box|Unknown|29|eingantibaroughtought| +11059|AAAAAAAADDLCAAAA|1997-10-27||Metals may need increasing|9.25|6.19|6015004|scholarbrand #4|15|custom|6|Jewelry|36|callypri|N/A|795745009tomato32879|lavender|Gram|Unknown|9|n stantibaroughtought| +11060|AAAAAAAAEDLCAAAA|1997-10-27|2000-10-26|Competitive sales diagnose so arrangements. Earnings used to stimulate days; usual tracks must not pick in order|1.79|1.57|10005011|scholarunivamalg #11|5|karoke|10|Electronics|136|callypriought|N/A|3turquoise2312365852|wheat|Tsp|Unknown|35|barcallybaroughtought| +11061|AAAAAAAAEDLCAAAA|2000-10-27||Competitive sales diagnose so arrangements. Earnings used to stimulate days; usual tracks must not pick in order|78.14|32.81|10005011|edu packedu pack #2|4|athletic|4|Shoes|665|callypriought|medium|718977rose5351914686|spring|Tbl|Unknown|16|oughtcallybaroughtought| +11062|AAAAAAAAGDLCAAAA|1997-10-27|1999-10-27|As extra numbers must not observe public, quiet offences. Considerations may look detailed heads. New eyes could extend desperately to a students. Alleged languages could not ask by a details.|1.13|0.79|5003001|exportischolar #1|3|pop|5|Music|765|anticallyation|N/A|065252679puff1776094|wheat|Tbl|Unknown|38|ablecallybaroughtought| +11063|AAAAAAAAGDLCAAAA|1999-10-28|2001-10-26|Here political minds could show big allies. Teachers might feel inc members. Significant records feel still european letters. For example academic conditions wo|62.41|0.79|10012014|importoamalgamalg #14|3|monitors|10|Electronics|227|anticallyation|N/A|065252679puff1776094|midnight|Ounce|Unknown|26|pricallybaroughtought| +11064|AAAAAAAAGDLCAAAA|2001-10-27||Here political minds could show big allies. Teachers might feel inc members. Significant records feel still european letters. For example academic conditions wo|3.53|0.79|9010001|univunivamalg #1|10|travel|9|Books|425|anticallyation|N/A|3545714373honeydew31|slate|Ton|Unknown|33|esecallybaroughtought| +11065|AAAAAAAAJDLCAAAA|1997-10-27||Scottish, secondary eggs ought to spot principles. Local, national fees must colour little, only yards. Early, additional plans might know more modern deb|65.70|37.44|6008002|namelesscorp #2|8|mens watch|6|Jewelry|297|ationn stable|N/A|540493plum9695650612|ghost|Unknown|Unknown|1|anticallybaroughtought| +11066|AAAAAAAAKDLCAAAA|1997-10-27|2000-10-26|So dry differences take so basic, complete opportunities. Large provisions get hard financial, primary buildings. Police say now wrong rates. Occasional, other piece|8.13|4.06|4004001|edu packedu pack #1|4|athletic|4|Shoes|934|eseprin st|economy|59927902470seashell4|hot|Oz|Unknown|31|callycallybaroughtought| +11067|AAAAAAAAKDLCAAAA|2000-10-27||Short, soft hours move here personal styles. Specific insects can try before police. Years send under the reserves. Problems enter a|8.50|3.91|9011004|amalgunivamalg #4|11|cooking|9|Books|154|eseantiought|N/A|8176421259thistle078|light|Unknown|Unknown|3|ationcallybaroughtought| +11068|AAAAAAAAMDLCAAAA|1997-10-27|1999-10-27|Areas may happen more. Able, other detectives turn here more little rights; wonderful, political incentives shall think currently out a increases. Services despise more politicians. New orga|3.64|1.78|8002009|importonameless #9|2|baseball|8|Sports|167|ationcallyought|N/A|474797drab1598841076|slate|Ounce|Unknown|54|eingcallybaroughtought| +11069|AAAAAAAAMDLCAAAA|1999-10-28|2001-10-26|Areas may happen more. Able, other detectives turn here more little rights; wonderful, political incentives shall think currently out a increases. Services despise more politicians. New orga|83.27|1.78|5004002|edu packscholar #2|4|classical|5|Music|167|ationcallyought|N/A|25980779317lawn22454|peach|Carton|Unknown|1|n stcallybaroughtought| +11070|AAAAAAAAMDLCAAAA|2001-10-27||Areas may happen more. Able, other detectives turn here more little rights; wonderful, political incentives shall think currently out a increases. Services despise more politicians. New orga|5.49|1.78|5004002|edu packimporto #1|4|sports-apparel|2|Men|170|barationought|large|25980779317lawn22454|salmon|Unknown|Unknown|60|barationbaroughtought| +11071|AAAAAAAAPDLCAAAA|1997-10-27||Inc considerations should dare sales. Little, long chapters check better exciting employers. Still english unions could pull wrong shoes. Factors would kee|70.39|47.16|9003008|exportimaxi #8|3|computers|9|Books|197|ationn stought|N/A|3khaki51295279699567|thistle|N/A|Unknown|51|oughtationbaroughtought| +11072|AAAAAAAAAELCAAAA|1997-10-27|2000-10-26|Normal, russian names provide also. Lips favour now vocational, frequent streets. Manufacturing muscles shall mould new, other residents. Afterwards special arms |3.92|1.88|8008003|namelessnameless #3|8|outdoor|8|Sports|91|oughtn st|N/A|012slate946673090435|snow|Gram|Unknown|12|ableationbaroughtought| +11073|AAAAAAAAAELCAAAA|2000-10-27||There narrow patients can labour ahead available rounds. Easy, milit|4.93|2.41|7012004|importonameless #4|12|paint|7|Home|325|oughtn st|N/A|012slate946673090435|slate|Unknown|Unknown|8|priationbaroughtought| +11074|AAAAAAAACELCAAAA|1997-10-27|1999-10-27|Following, labour needs might stand asleep procedures. Right cells may com|4.42|3.00|5002001|importoscholar #1|2|country|5|Music|957|ationantin st|N/A|317028720385246pink4|thistle|Lb|Unknown|30|eseationbaroughtought| +11075|AAAAAAAACELCAAAA|1999-10-28|2001-10-26|More american factors may not feel strategically. As unable merchants change less tall eyes. Other forests sustain in a recommendations. V|4.43|3.00|5002001|univmaxi #6|2|pools|8|Sports|205|antibarable|N/A|54608135612266peach1|pale|Ton|Unknown|7|antiationbaroughtought| +11076|AAAAAAAACELCAAAA|2001-10-27||Sure official spea|3.55|1.17|10013010|exportiamalgamalg #10|2|stereo|10|Electronics|205|antibarable|N/A|3072442pale086025880|pale|Dozen|Unknown|22|callyationbaroughtought| +11077|AAAAAAAAFELCAAAA|1997-10-27||Residential observations tolerate quickly english ministe|2.25|2.02|10004011|edu packunivamalg #11|4|audio|10|Electronics|180|bareingought|N/A|32287550528royal8568|navy|Bunch|Unknown|74|ationationbaroughtought| +11078|AAAAAAAAGELCAAAA|1997-10-27|2000-10-26|Marked, liberal boys develop regular creditors. Regional police cope up to a incidents. Good, aggressive forces go thus. Net, brit|8.27|4.96|7003009|exportibrand #9|3|kids|7|Home|129|n stableought|N/A|6light70128493357974|thistle|Pound|Unknown|23|eingationbaroughtought| +11079|AAAAAAAAGELCAAAA|2000-10-27||Young attitudes keep forces. British benefits must not show best general town|8.72|4.97|7004010|edu packbrand #10|3|curtains/drapes|7|Home|277|n stableought|N/A|56507158401096white7|salmon|Pound|Unknown|82|n stationbaroughtought| +11080|AAAAAAAAIELCAAAA|1997-10-27|1999-10-27|Debts necessita|0.14|0.12|6002001|importocorp #1|2|diamonds|6|Jewelry|85|antieing|N/A|2168mint405718608269|rose|Lb|Unknown|55|bareingbaroughtought| +11081|AAAAAAAAIELCAAAA|1999-10-28|2001-10-26|Debts necessita|4.92|3.93|9003006|exportimaxi #6|3|computers|9|Books|825|antieing|N/A|2168mint405718608269|lace|Gross|Unknown|78|oughteingbaroughtought| +11082|AAAAAAAAIELCAAAA|2001-10-27||Sharp, tender eyes use up to. Cultures will not deceive; likely kinds shall evade as again skilled areas. Different heads might protect technical, jewish taxes. Key, politica|60.75|18.22|9003006|exporticorp #3|3|gold|6|Jewelry|46|callyese|N/A|2168mint405718608269|puff|Ounce|Unknown|26|ableeingbaroughtought| +11083|AAAAAAAALELCAAAA|1997-10-27||Cigarettes say parents. Great statements will consider. Cases consult students. Proposals correct. Years jump in a guards. Future users lock suitable relations. Primary, essential q|5.02|1.65|1003002|exportiamalg #2|3|maternity|1|Women|94|esen st|medium|824598202red39231546|seashell|Oz|Unknown|56|prieingbaroughtought| +11084|AAAAAAAAMELCAAAA|1997-10-27|2000-10-26|Single charges stand eventually then mental wines. Flexible days find through the men; surprising producers improve for a churches; mental officials might not oust particularly m|9.99|4.29|7009005|maxibrand #5|9|mattresses|7|Home|213|prioughtable|N/A|28775799808784tan069|rosy|Each|Unknown|24|eseeingbaroughtought| +11085|AAAAAAAAMELCAAAA|2000-10-27||Religious, local bacteria cannot take sometimes. Players believe very foreign, modest materials; words ought to break well ends. Hardly thin men should move that attractive, social restaurants. |3.84|4.29|7009005|edu packamalg #2|9|swimwear|1|Women|255|prioughtable|small|944772navajo71793150|papaya|Dozen|Unknown|80|antieingbaroughtought| +11086|AAAAAAAAOELCAAAA|1997-10-27|1999-10-27|Buses break maybe. International varieties would die new clients. Real preferences shall date however in a others. Individuals get almost safe counties. Specific, suspicious friends s|61.51|46.13|7010007|univnameless #7|10|flatware|7|Home|873|priationeing|N/A|96400yellow879810295|sienna|Bunch|Unknown|7|callyeingbaroughtought| +11087|AAAAAAAAOELCAAAA|1999-10-28|2001-10-26|Buses break maybe. International varieties would die new clients. Real preferences shall date however in a others. Individuals get almost safe counties. Specific, suspicious friends s|5.62|46.13|2004002|edu packimporto #2|10|sports-apparel|2|Men|873|priationeing|extra large|powder59256909342851|light|Gram|Unknown|12|ationeingbaroughtought| +11088|AAAAAAAAOELCAAAA|2001-10-27||Bodies could repo|33.93|21.37|7011009|amalgnameless #9|11|accent|7|Home|106|priationeing|N/A|powder59256909342851|sandy|Cup|Unknown|10|eingeingbaroughtought| +11089|AAAAAAAABFLCAAAA|1997-10-27||Units shall seem finally. Managers might not point common matters. Both strong |3.03|2.36|3001002|amalgexporti #2|1|newborn|3|Children|183|prieingought|small|483930830014pale3909|violet|Each|Unknown|47|n steingbaroughtought| +11090|AAAAAAAACFLCAAAA|1997-10-27|2000-10-26|Small processes might grab only. Particular, possible respects could make recently. Loans might handle also available, old teachers. Low, regional pat|8.98|5.47|2001001|amalgimporto #1|1|accessories|2|Men|640|baresecally|petite|5indian0908460531398|turquoise|Cup|Unknown|55|barn stbaroughtought| +11091|AAAAAAAACFLCAAAA|2000-10-27||Words come closer. Enormous, urgent libraries keep careful levels. Revenues provide agreements. Current|7.40|2.66|2001001|exportiamalg #2|1|maternity|1|Women|640|baresecally|large|5indian0908460531398|moccasin|Bunch|Unknown|18|oughtn stbaroughtought| +11092|AAAAAAAAEFLCAAAA|1997-10-27|1999-10-27|Eyes build private, lexical fans; months would go parents; great, simple eyes could not accompany now wit|1.25|0.78|3002001|importoexporti #1|2|infants|3|Children|380|bareingpri|large|6235571ivory77280432|tomato|Gram|Unknown|26|ablen stbaroughtought| +11093|AAAAAAAAEFLCAAAA|1999-10-28|2001-10-26|However modest homes expect rights. Children get even cheap, grey groups. Social, vast dogs realise long with a ears. Rights can reach; friends ought to think broad eyes. Other syst|65.69|47.29|10013010|exportiamalgamalg #10|2|stereo|10|Electronics|380|bareingpri|N/A|lawn2398406002942379|chartreuse|Tsp|Unknown|12|prin stbaroughtought| +11094|AAAAAAAAEFLCAAAA|2001-10-27||Large cases exist somewhat often young chapters; new, vague m|1.92|47.29|10013010|edu packimporto #1|2|sports-apparel|2|Men|228|bareingpri|large|lawn2398406002942379|wheat|Ounce|Unknown|28|esen stbaroughtought| +11095|AAAAAAAAHFLCAAAA|1997-10-27||Comparative notes lose; decisions shall consider still in the tasks. Only urban years like here at once s|0.69|0.51|4001002|amalgedu pack #2|1|womens|4|Shoes|158|eingantiought|small|30986110ghost1992917|tan|Dozen|Unknown|30|antin stbaroughtought| +11096|AAAAAAAAIFLCAAAA|1997-10-27|2000-10-26|Subsequent, conservative shares say frequently entire professi|3.04|1.33|3001001|amalgexporti #1|1|newborn|3|Children|177|ationationought|extra large|297063373060powder97|black|Dozen|Unknown|93|callyn stbaroughtought| +11097|AAAAAAAAIFLCAAAA|2000-10-27||Australian, big policies get merely here left boards. Stages boost relevant, recent decades. Able men might not accompany as a fathers. Very specialist shapes will not wish in |9.16|1.33|10011014|amalgamalgamalg #14|11|disk drives|10|Electronics|664|esecallycally|N/A|109535124papaya95276|blush|Pallet|Unknown|17|ationn stbaroughtought| +11098|AAAAAAAAKFLCAAAA|1997-10-27|1999-10-27|Personnel would settle rather. Titles may not win wonderful companies. Nurses could destroy stro|5.55|4.21|1004001|edu packamalg #1|4|swimwear|1|Women|441|oughteseese|medium|8yellow6786993283421|saddle|Ounce|Unknown|22|eingn stbaroughtought| +11099|AAAAAAAAKFLCAAAA|1999-10-28|2001-10-26|Personnel would settle rather. Titles may not win wonderful companies. Nurses could destroy stro|0.66|4.21|1004001|exportiamalg #2|3|maternity|1|Women|441|oughteseese|medium|1371621salmon0415769|spring|Tsp|Unknown|27|n stn stbaroughtought| +11100|AAAAAAAAKFLCAAAA|2001-10-27||Personnel would settle rather. Titles may not win wonderful companies. Nurses could destroy stro|2.03|4.21|1004001|edu packedu pack #1|4|athletic|4|Shoes|276|callyationable|petite|1371621salmon0415769|goldenrod|Dram|Unknown|25|barbaroughtoughtought| +11101|AAAAAAAANFLCAAAA|1997-10-27||Explicit, warm poems eat yesterday with the politicians. Locally |4.37|2.14|5002002|importoscholar #2|2|country|5|Music|1|ought|N/A|2976royal70087777960|salmon|Box|Unknown|25|oughtbaroughtoughtought| +11102|AAAAAAAAOFLCAAAA|1997-10-27|2000-10-26|Large counties would act tight on the seasons. Inside mass views would not combine then are|3.80|2.24|7015003|scholarnameless #3|15|tables|7|Home|647|ationesecally|N/A|895925570734purple17|wheat|Oz|Unknown|27|ablebaroughtoughtought| +11103|AAAAAAAAOFLCAAAA|2000-10-27||Even familiar classes paint english, prime problems. Prime, overseas students shou|60.92|44.47|8011004|amalgmaxi #4|11|archery|8|Sports|647|ationesecally|N/A|895925570734purple17|smoke|Lb|Unknown|19|pribaroughtoughtought| +11104|AAAAAAAAAGLCAAAA|1997-10-27|1999-10-27|Tools will bring to a initiatives. Long-term, european uses know warm, wise examples; pupils disappear at least in a shoes. So blue views might help now e|27.12|17.89|10016004|corpamalgamalg #4|16|wireless|10|Electronics|550|barantianti|N/A|0953lime611979623716|papaya|Case|Unknown|42|esebaroughtoughtought| +11105|AAAAAAAAAGLCAAAA|1999-10-28|2001-10-26|Red, growing matters must not worry downstream high, continued sites. Poor contracts pay now. Co|0.25|17.89|4002002|importoedu pack #2|2|mens|4|Shoes|550|barantianti|petite|73287135lace29735569|royal|Ounce|Unknown|26|antibaroughtoughtought| +11106|AAAAAAAAAGLCAAAA|2001-10-27||Red, growing matters must not worry downstream high, continued sites. Poor contracts pay now. Co|3.08|17.89|4002002|univunivamalg #7|2|travel|9|Books|348|eingesepri|N/A|73287135lace29735569|frosted|Ton|Unknown|100|callybaroughtoughtought| +11107|AAAAAAAADGLCAAAA|1997-10-27||Substanti|3.31|1.45|4003002|exportiedu pack #2|3|kids|4|Shoes|41|oughtese|economy|6saddle3166733519107|honeydew|Pallet|Unknown|19|ationbaroughtoughtought| +11108|AAAAAAAAEGLCAAAA|1997-10-27|2000-10-26|Authorities bow. Available facilities may not nee|0.57|0.37|2003001|exportiimporto #1|3|pants|2|Men|284|eseeingable|small|6960921blanched56177|wheat|Dram|Unknown|74|eingbaroughtoughtought| +11109|AAAAAAAAEGLCAAAA|2000-10-27||Upper methods become deep bad, likely sections; previous, hard levels may travel somewhat small wom|2.03|1.17|2004002|edu packimporto #2|3|sports-apparel|2|Men|284|eseeingable|small|6960921blanched56177|pink|Bundle|Unknown|16|n stbaroughtoughtought| +11110|AAAAAAAAGGLCAAAA|1997-10-27|1999-10-27|More economic holidays would not feel so good, short duties. New, ol|3.21|2.11|10007010|brandunivamalg #10|7|personal|10|Electronics|84|eseeing|N/A|979469425286pale8370|sienna|Ounce|Unknown|15|baroughtoughtoughtought| +11111|AAAAAAAAGGLCAAAA|1999-10-28|2001-10-26|More economic holidays would not feel so good, short duties. New, ol|6.06|2.11|10007010|corpmaxi #12|7|parenting|9|Books|143|eseeing|N/A|979469425286pale8370|drab|Ton|Unknown|18|oughtoughtoughtoughtought| +11112|AAAAAAAAGGLCAAAA|2001-10-27||Lines enable. Most short circumstances used to generate. Again true goods c|2.08|2.11|1003001|exportiamalg #1|3|maternity|1|Women|49|eseeing|medium|2634wheat91926098139|steel|Lb|Unknown|1|ableoughtoughtoughtought| +11113|AAAAAAAAJGLCAAAA|1997-10-27||More british sides provide there. Other, positive rights must not|0.15|0.07|10006013|corpunivamalg #13|6|musical|10|Electronics|317|ationoughtpri|N/A|20406079915yellow684|navajo|Each|Unknown|17|prioughtoughtoughtought| +11114|AAAAAAAAKGLCAAAA|1997-10-27|2000-10-26|Here a|8.84|5.21|5004001|edu packscholar #1|4|classical|5|Music|435|antipriese|N/A|85723slate4169011885|tomato|Each|Unknown|22|eseoughtoughtoughtought| +11115|AAAAAAAAKGLCAAAA|2000-10-27||Mainly other galleries ought to know extremely surprising public risks. Now close countries keep also serious pictures. Orders contact again gen|1.97|0.70|5004001|edu packedu pack #2|4|athletic|4|Shoes|199|antipriese|medium|85723slate4169011885|navajo|Ton|Unknown|23|antioughtoughtoughtought| +11116|AAAAAAAAMGLCAAAA|1997-10-27|1999-10-27|Much general articles think exceptional dreams. Customs recognize little particular buildings. Specia|6.16|2.89|6005005|scholarcorp #5|5|earings|6|Jewelry|81|oughteing|N/A|360587sandy214999960|lace|Pallet|Unknown|11|callyoughtoughtoughtought| +11117|AAAAAAAAMGLCAAAA|1999-10-28|2001-10-26|Much general articles think exceptional dreams. Customs recognize little particular buildings. Specia|4.14|2.89|6005005|edu packnameless #8|5|glassware|7|Home|1000|barbarbarought|N/A|360587sandy214999960|puff|Ounce|Unknown|1|ationoughtoughtoughtought| +11118|AAAAAAAAMGLCAAAA|2001-10-27||Much general articles think exceptional dreams. Customs recognize little particular buildings. Specia|7.17|2.58|4004001|edu packedu pack #1|4|athletic|4|Shoes|1000|barbarbarought|medium|58957775280ivory5802|cream|Box|Unknown|2|eingoughtoughtoughtought| +11119|AAAAAAAAPGLCAAAA|1997-10-27||Points regulate universal potatoes. Cars make feet. Also american beaches should not take lonely, british rewards. Very coming systems might mind quickly regular careers. Individuals get foreign days;|7.97|5.33|1004002|edu packamalg #2|4|swimwear|1|Women|89|n steing|small|white813329305132191|medium|N/A|Unknown|65|n stoughtoughtoughtought| +11120|AAAAAAAAAHLCAAAA|1997-10-27|2000-10-26|Minute cells shall not hunt jewish, french women. Always medical churches shall not enter entirely successive medical men; criminal, e|3.36|1.88|3002001|importoexporti #1|2|infants|3|Children|246|callyeseable|medium|9peru220410436848663|pink|Tsp|Unknown|46|barableoughtoughtought| +11121|AAAAAAAAAHLCAAAA|2000-10-27||Minute cells shall not hunt jewish, french women. Always medical churches shall not enter entirely successive medical men; criminal, e|0.48|1.88|3002001|importounivamalg #10|2|home repair|9|Books|333|pripripri|N/A|9peru220410436848663|honeydew|Each|Unknown|27|oughtableoughtoughtought| +11122|AAAAAAAACHLCAAAA|1997-10-27|1999-10-27|Cultural plants bring in a numbers. Other, possi|6.42|4.81|3001001|amalgexporti #1|1|newborn|3|Children|150|barantiought|small|70497799809antique76|slate|Ton|Unknown|44|ableableoughtoughtought| +11123|AAAAAAAACHLCAAAA|1999-10-28|2001-10-26|Cultural plants bring in a numbers. Other, possi|5.85|2.39|3001001|edu packedu pack #2|1|athletic|4|Shoes|184|eseeingought|extra large|70497799809antique76|metallic|Bunch|Unknown|29|priableoughtoughtought| +11124|AAAAAAAACHLCAAAA|2001-10-27||Cultural plants bring in a numbers. Other, possi|7.35|2.39|3001001|exportiexporti #1|3|toddlers|3|Children|98|eingn st|medium|39411488315sienna293|light|Dozen|Unknown|33|eseableoughtoughtought| +11125|AAAAAAAAFHLCAAAA|1997-10-27||Significant stones suggest there. Practical groups may include however practical scholars. Actions marry. Words mea|3.20|2.36|2003002|exportiimporto #2|3|pants|2|Men|211|oughtoughtable|medium|616331turquoise97991|red|Ton|Unknown|65|antiableoughtoughtought| +11126|AAAAAAAAGHLCAAAA|1997-10-27|2000-10-26|Clients must not feel also ever private cars; names think. Concerned meals used to go still chapters; remarkable, minimal terms get at first. Obvious articles must |0.71|0.41|7001007|amalgbrand #7|1|bathroom|7|Home|991|oughtn stn st|N/A|5plum802542841385158|sky|Ton|Unknown|35|callyableoughtoughtought| +11127|AAAAAAAAGHLCAAAA|2000-10-27||Clients must not feel also ever private cars; names think. Concerned meals used to go still chapters; remarkable, minimal terms get at first. Obvious articles must |9.37|7.77|7001007|brandmaxi #4|1|reference|9|Books|47|ationese|N/A|5plum802542841385158|steel|Each|Unknown|31|ationableoughtoughtought| +11128|AAAAAAAAIHLCAAAA|1997-10-27|1999-10-27|Different words|9.77|7.13|8011009|amalgmaxi #9|11|archery|8|Sports|16|callyought|N/A|24425284yellow080277|sandy|Case|Unknown|50|eingableoughtoughtought| +11129|AAAAAAAAIHLCAAAA|1999-10-28|2001-10-26|Once clear relations produce confiden|9.29|7.13|8011009|edu packedu pack #2|11|athletic|4|Shoes|50|baranti|medium|96883puff85049531433|purple|Oz|Unknown|9|n stableoughtoughtought| +11130|AAAAAAAAIHLCAAAA|2001-10-27||Steps should mean much. Individuals shall acquire numbers; open ultimate rates can assist as national russian things. Accurately extended lips cannot formulate imp|11.65|7.13|8011009|corpnameless #1|16|furniture|7|Home|201|oughtbarable|N/A|27036861808573royal5|salmon|Cup|Unknown|86|barprioughtoughtought| +11131|AAAAAAAALHLCAAAA|1997-10-27||Local ministers destroy now like a implications; heavy, unknown buses know afraid, old proposals. Willing men decide; remaining, excellent children shall help suddenly largely plain eyes. Equ|74.57|53.69|3001002|amalgexporti #2|1|newborn|3|Children|514|eseoughtanti|medium|02714red061476194653|olive|Box|Unknown|11|oughtprioughtoughtought| +11132|AAAAAAAAMHLCAAAA|1997-10-27|2000-10-26|Never existing times argue so without the attitudes. Political, new plans replace still in a computers. Even extra towns would come even false |6.24|3.86|2004001|edu packimporto #1|4|sports-apparel|2|Men|26|callyable|small|0562352136922ivory45|salmon|Tbl|Unknown|61|ableprioughtoughtought| +11133|AAAAAAAAMHLCAAAA|2000-10-27||Never existing times argue so without the attitudes. Political, new plans replace still in a computers. Even extra towns would come even false |3.40|3.86|2004001|edu packscholar #2|4|classical|5|Music|26|callyable|N/A|68501puff51883219100|metallic|Gross|Unknown|26|priprioughtoughtought| +11134|AAAAAAAAOHLCAAAA|1997-10-27|1999-10-27|Ancient ideas prevent estates. Sides see as inner cars. Local advertisements meet little f|0.60|0.26|3004001|edu packexporti #1|4|school-uniforms|3|Children|222|ableableable|large|364732729149053pale6|navajo|Each|Unknown|19|eseprioughtoughtought| +11135|AAAAAAAAOHLCAAAA|1999-10-28|2001-10-26|Ancient ideas prevent estates. Sides see as inner cars. Local advertisements meet little f|6.49|5.58|3004001|scholarnameless #6|15|tables|7|Home|222|ableableable|N/A|364732729149053pale6|rose|Bunch|Unknown|16|antiprioughtoughtought| +11136|AAAAAAAAOHLCAAAA|2001-10-27||Ancient ideas prevent estates. Sides see as inner cars. Local advertisements meet little f|9.87|5.58|2004001|edu packimporto #1|15|sports-apparel|2|Men|534|ableableable|small|364732729149053pale6|white|Bundle|Unknown|4|callyprioughtoughtought| +11137|AAAAAAAABILCAAAA|1997-10-27||Implicit, cautious structures shall not recover new, delighted pri|9.09|6.54|4004002|edu packedu pack #2|4|athletic|4|Shoes|996|callyn stn st|small|545989269160sandy736|saddle|Cup|Unknown|43|ationprioughtoughtought| +11138|AAAAAAAACILCAAAA|1997-10-27|2000-10-26|Existing, hot substances come as intern|3.67|2.05|2002001|importoimporto #1|2|shirts|2|Men|640|baresecally|medium|7235046light19394769|white|Bunch|Unknown|3|eingprioughtoughtought| +11139|AAAAAAAACILCAAAA|2000-10-27||Existing, hot substances come as intern|7.57|2.05|3003002|exportiexporti #2|3|toddlers|3|Children|470|baresecally|extra large|0185272726887salmon9|seashell|Box|Unknown|17|n stprioughtoughtought| +11140|AAAAAAAAEILCAAAA|1997-10-27|1999-10-27|Existing troops keep together secret, inland groups. Adult, sound years cannot pay more commerci|2.39|1.95|4002001|importoedu pack #1|2|mens|4|Shoes|96|callyn st|economy|35212667medium226399|ghost|Carton|Unknown|8|bareseoughtoughtought| +11141|AAAAAAAAEILCAAAA|1999-10-28|2001-10-26|Full, other years should wear european members; magnetic pictures would show; today cultural feet must not watch. Int|4.11|1.95|4002001|importoedu pack #2|2|mens|4|Shoes|96|callyn st|economy|35212667medium226399|rosy|Bundle|Unknown|9|oughteseoughtoughtought| +11142|AAAAAAAAEILCAAAA|2001-10-27||Full, other years should wear european members; magnetic pictures would show; today cultural feet must not watch. Int|2.00|1.22|4002001|edu packamalg #1|4|swimwear|1|Women|96|callyn st|medium|35212667medium226399|royal|N/A|Unknown|20|ableeseoughtoughtought| +11143|AAAAAAAAHILCAAAA|1997-10-27||Correctly accurate experiences cannot show however |8.96|4.65|5002002|importoscholar #2|2|country|5|Music|252|ableantiable|N/A|170367papaya39146464|rose|Bundle|Unknown|37|prieseoughtoughtought| +11144|AAAAAAAAIILCAAAA|1997-10-27|2000-10-26|At least low personnel might a|9.13|2.83|9003009|exportimaxi #9|3|computers|9|Books|179|n stationought|N/A|lemon891155497137307|turquoise|Each|Unknown|5|eseeseoughtoughtought| +11145|AAAAAAAAIILCAAAA|2000-10-27||Companies can serve legal intentions; local hands|6.08|4.86|2001002|amalgimporto #2|3|accessories|2|Men|319|n stoughtpri|large|peach096012544269808|navajo|Gram|Unknown|14|antieseoughtoughtought| +11146|AAAAAAAAKILCAAAA|1997-10-27|1999-10-27|Perfect, popular circumstances curb beautifully profits. Then british winners will not find most compulsory legislative models; circles shall remove later other factors. Others may pose so |49.87|18.45|4004001|edu packedu pack #1|4|athletic|4|Shoes|939|n stprin st|large|73saddle355049856492|thistle|Gram|Unknown|20|callyeseoughtoughtought| +11147|AAAAAAAAKILCAAAA|1999-10-28|2001-10-26|Perfect, popular circumstances curb beautifully profits. Then british winners will not find most compulsory legislative models; circles shall remove later other factors. Others may pose so |6.96|18.45|7001008|amalgbrand #8|1|bathroom|7|Home|939|n stprin st|N/A|73saddle355049856492|tan|Dozen|Unknown|10|ationeseoughtoughtought| +11148|AAAAAAAAKILCAAAA|2001-10-27||Actors could count also political intervals. Costs survive about no areas; problems will give almost private newspapers. Aut|1.17|0.79|7001008|importoimporto #1|2|shirts|2|Men|939|n stprin st|medium|73saddle355049856492|maroon|N/A|Unknown|19|eingeseoughtoughtought| +11149|AAAAAAAANILCAAAA|1997-10-27||New rocks might not assist. Poor fields cope. Even critical patients cannot change. Police rain to the hundreds. Tears want english, large feelings. German, tradition|2.72|1.11|8002010|importonameless #10|2|baseball|8|Sports|7|ation|N/A|757peru2312007587970|royal|Oz|Unknown|27|n steseoughtoughtought| +11150|AAAAAAAAOILCAAAA|1997-10-27|2000-10-26|Large, limited boys indicate. Firmly given files used to read northern, recent friends. Different, spiritual rounds clear clear materials. Ag|2.50|1.22|2001001|amalgimporto #1|1|accessories|2|Men|827|ationableeing|petite|456069694316212pale9|steel|Gross|Unknown|40|barantioughtoughtought| +11151|AAAAAAAAOILCAAAA|2000-10-27||Large, limited boys indicate. Firmly given files used to read northern, recent friends. Different, spiritual rounds clear clear materials. Ag|1.07|1.22|10016017|corpamalgamalg #17|1|wireless|10|Electronics|245|ationableeing|N/A|8391322524violet5486|smoke|Pound|Unknown|89|oughtantioughtoughtought| +11152|AAAAAAAAAJLCAAAA|1997-10-27|1999-10-27|Military, living |0.64|0.35|6005001|scholarcorp #1|5|earings|6|Jewelry|644|eseesecally|N/A|0472191076orange6461|pink|Oz|Unknown|10|ableantioughtoughtought| +11153|AAAAAAAAAJLCAAAA|1999-10-28|2001-10-26|Less new men shall examine. Whole years visit in a images. Either part-time problems qualify a|8.94|0.35|6005001|brandmaxi #6|7|reference|9|Books|644|eseesecally|N/A|0472191076orange6461|rose|Pound|Unknown|32|priantioughtoughtought| +11154|AAAAAAAAAJLCAAAA|2001-10-27||Figures arrive closer good risks. Main, irish changes believe financial, healthy things. Difficult kinds would |4.58|3.20|9008007|namelessmaxi #7|8|romance|9|Books|87|ationeing|N/A|6019spring8090981095|blue|Unknown|Unknown|17|eseantioughtoughtought| +11155|AAAAAAAADJLCAAAA|1997-10-27||Large services shall not add. Affairs prepare about brief issues. Average roads increase silent pairs. Approximately private arguments shall approach rule|2.43|1.48|4002002|importoedu pack #2|2|mens|4|Shoes|194|esen stought|petite|2257705490930purple0|sky|Unknown|Unknown|3|antiantioughtoughtought| +11156|AAAAAAAAEJLCAAAA|1997-10-27|2000-10-26|Identical, recent students know just; low, square proceedings make seriously bars; leaders should not save clearly as religious papers. Ago male interpretations cannot carry separate, s|3.74|1.98|2001001|amalgimporto #1|1|accessories|2|Men|671|oughtationcally|extra large|4018111330goldenrod2|slate|Pallet|Unknown|26|callyantioughtoughtought| +11157|AAAAAAAAEJLCAAAA|2000-10-27||Close structures must join facilities. Then other appeals may not |4.81|1.98|2004002|edu packimporto #2|4|sports-apparel|2|Men|671|oughtationcally|large|4018111330goldenrod2|spring|Cup|Unknown|9|ationantioughtoughtought| +11158|AAAAAAAAGJLCAAAA|1997-10-27|1999-10-27|Different, responsible numbers could increas|89.07|58.78|2003001|exportiimporto #1|3|pants|2|Men|12|ableought|small|88352yellow862697733|purple|Case|Unknown|88|eingantioughtoughtought| +11159|AAAAAAAAGJLCAAAA|1999-10-28|2001-10-26|Different, responsible numbers could increas|6.77|||||mens|4|Shoes|12|||67650sienna281390245|mint||Unknown||n stantioughtoughtought| +11160|AAAAAAAAGJLCAAAA|2001-10-27||Different, responsible numbers could increas|4.49|4.46|9007001|brandmaxi #1|7|reference|9|Books|12|ableought|N/A|67650sienna281390245|light|Unknown|Unknown|11|barcallyoughtoughtought| +11161|AAAAAAAAJJLCAAAA|1997-10-27||Minutes sit successfully parliamentary drawings. Pop|4.79|1.43|7014002|edu packnameless #2|14|glassware|7|Home|97|ationn st|N/A|sky75092912448310135|smoke|Box|Unknown|41|oughtcallyoughtoughtought| +11162|AAAAAAAAKJLCAAAA|1997-10-27|2000-10-26|Afraid, old meals will get chronic, strong applicants. Arms could look with a needs. Hence wor|7.02|3.15|9014003|edu packunivamalg #3|14|sports|9|Books|81|oughteing|N/A|45630honeydew8717556|misty|Bundle|Unknown|6|ablecallyoughtoughtought| +11163|AAAAAAAAKJLCAAAA|2000-10-27||Afraid, old meals will get chronic, strong applicants. Arms could look with a needs. Hence wor|6.41|3.15|9014003|edu packscholar #2|14|classical|5|Music|81|oughteing|N/A|turquoise30347357714|rose|Dram|Unknown|2|pricallyoughtoughtought| +11164|AAAAAAAAMJLCAAAA|1997-10-27|1999-10-27|Male patients say on a plans. Silent orders support. Other, normal levels work strongly in the brothers. Rights cannot walk now french, goo|7.31|4.75|7010005|univnameless #5|10|flatware|7|Home|620|barablecally|N/A|3lavender37575934722|seashell|Tsp|Unknown|38|esecallyoughtoughtought| +11165|AAAAAAAAMJLCAAAA|1999-10-28|2001-10-26|Male patients say on a plans. Silent orders support. Other, normal levels work strongly in the brothers. Rights cannot walk now french, goo|1.64|1.13|7010005|corpunivamalg #6|10|mystery|9|Books|620|barablecally|N/A|0948457643497smoke47|medium|Each|Unknown|1|anticallyoughtoughtought| +11166|AAAAAAAAMJLCAAAA|2001-10-27||Male patients say on a plans. Silent orders support. Other, normal levels work strongly in the brothers. Rights cannot walk now french, goo|64.67|1.13|6012007|importobrand #7|10|costume|6|Jewelry|620|barablecally|N/A|0948457643497smoke47|violet|Ounce|Unknown|16|callycallyoughtoughtought| +11167|AAAAAAAAPJLCAAAA|1997-10-27||Only asleep areas dance sometimes. References could ignore; severe, important seasons may bother however long, level jobs. Readers |2.74|2.21|2002002|importoimporto #2|2|shirts|2|Men|85|antieing|small|626smoke039614597049|wheat|Each|Unknown|14|ationcallyoughtoughtought| +11168|AAAAAAAAAKLCAAAA|1997-10-27|2000-10-26|Regulations go almost. Complex operations may stay at present countries. Widely special modules can rest also in ne|7.23|6.29|7011009|amalgnameless #9|11|accent|7|Home|630|barprically|N/A|brown566082258231233|khaki|Tsp|Unknown|25|eingcallyoughtoughtought| +11169|AAAAAAAAAKLCAAAA|2000-10-27||Regulations go almost. Complex operations may stay at present countries. Widely special modules can rest also in ne|76.12|42.62|9001010|amalgmaxi #10|1|arts|9|Books|961|barprically|N/A|brown566082258231233|red|Box|Unknown|73|n stcallyoughtoughtought| +11170|AAAAAAAACKLCAAAA|1997-10-27|1999-10-27|More other courses shall adopt white loans. Socially diffic|5.34|2.02|1002001|importoamalg #1|2|fragrances|1|Women|217|ationoughtable|large|1116789456414powder0|saddle|Tbl|Unknown|66|barationoughtoughtought| +11171|AAAAAAAACKLCAAAA|1999-10-28|2001-10-26|Likely, silent miles go closely other differences; very huge parts come centrally safel|5.52|4.96|6005004|scholarcorp #4|2|earings|6|Jewelry|1000|ationoughtable|N/A|793188169puff0809418|hot|Gross|Unknown|32|oughtationoughtoughtought| +11172|AAAAAAAACKLCAAAA|2001-10-27||Strong, apparent members subsidise courses; movements stand also musical girls. Rich earnings must wait a bit lightly brown needs. However new girls shall rely enough overall,|1.44|0.50|5003001|exportischolar #1|2|pop|5|Music|102|ablebarought|N/A|793188169puff0809418|aquamarine|Dozen|Unknown|48|ableationoughtoughtought| +11173|AAAAAAAAFKLCAAAA|1997-10-27||Deaf services stop also pink candidates. Particular colours might share only direct problems. Superb, public workers would include also o|1.23|0.52|1001002|amalgamalg #2|1|dresses|1|Women|246|callyeseable|large|933red14626984582725|lime|Bunch|Unknown|4|priationoughtoughtought| +11174|AAAAAAAAGKLCAAAA|1997-10-27|2000-10-26|Extraordinary skills emulate now. Exclusive, excellent elections will zero more other changes. Rights wil|1.04|0.39|9006003|corpmaxi #3|6|parenting|9|Books|587|ationeinganti|N/A|00703843514royal6005|mint|Lb|Unknown|18|eseationoughtoughtought| +11175|AAAAAAAAGKLCAAAA|2000-10-27||Good lives might provide positive letters; national, right courses will afford slightly new rates; local, successful papers risk once. Major firms move sectio|1.97|0.39|9006003|edu packscholar #2|4|classical|5|Music|304|esebarpri|N/A|00703843514royal6005|dim|N/A|Unknown|42|antiationoughtoughtought| +11176|AAAAAAAAIKLCAAAA|1997-10-27|1999-10-27|Inevitably scientific museums lower socialist, present others. Free, young minutes may see bodies. African standards could prove actually. Straight main margins happen|6.32|3.91|5004001|edu packscholar #1|4|classical|5|Music|123|priableought|N/A|4345708orchid3767958|salmon|Bundle|Unknown|6|callyationoughtoughtought| +11177|AAAAAAAAIKLCAAAA|1999-10-28|2001-10-26|Inevitably scientific museums lower socialist, present others. Free, young minutes may see bodies. African standards could prove actually. Straight main margins happen|8.15|5.78|8003008|exportinameless #8|3|basketball|8|Sports|123|priableought|N/A|78steel0746741895209|turquoise|Pallet|Unknown|29|ationationoughtoughtought| +11178|AAAAAAAAIKLCAAAA|2001-10-27||Diverse, atomic targets could stop very. True views shall not imagine about a holders; years affect of course presidential individua|7.60|2.96|8003008|importoamalgamalg #10|3|monitors|10|Electronics|229|n stableable|N/A|7powder8809553735318|linen|Pallet|Unknown|49|eingationoughtoughtought| +11179|AAAAAAAALKLCAAAA|1997-10-27||Matters would control merely short girls. Judicial clouds w|4.38|1.35|5001002|amalgscholar #2|1|rock|5|Music|65|antically|N/A|15077047purple643759|goldenrod|Ounce|Unknown|20|n stationoughtoughtought| +11180|AAAAAAAAMKLCAAAA|1997-10-27|2000-10-26|Years used to talk from time to time to a soldiers. Unable miles continue fast from a suppliers. Dear, everyday individuals must imagine always workers.|5.99|3.65|3002001|importoexporti #1|2|infants|3|Children|182|ableeingought|extra large|39606994pale82827918|rosy|Tbl|Unknown|15|bareingoughtoughtought| +11181|AAAAAAAAMKLCAAAA|2000-10-27||Years used to talk from time to time to a soldiers. Unable miles continue fast from a suppliers. Dear, everyday individuals must imagine always workers.|4.27|2.94|4003002|exportiedu pack #2|3|kids|4|Shoes|410|baroughtese|extra large|8827301537082linen41|light|Pound|Unknown|16|oughteingoughtoughtought| +11182|AAAAAAAAOKLCAAAA|1997-10-27|1999-10-27|Companies must think always french, central police. Great practitioners could not t|1.52|1.20|10004014|edu packunivamalg #14|4|audio|10|Electronics|897|ationn steing|N/A|08947powder120157826|papaya|Ounce|Unknown|30|ableeingoughtoughtought| +11183|AAAAAAAAOKLCAAAA|1999-10-28|2001-10-26|Neither dull animals believe|2.07|1.53|1002002|importoamalg #2|2|fragrances|1|Women|215|ationn steing|extra large|9orchid9005410763174|tan|Each|Unknown|15|prieingoughtoughtought| +11184|AAAAAAAAOKLCAAAA|2001-10-27||Neither dull animals believe|2.69|2.36|1002002|exportinameless #5|2|wallpaper|7|Home|215|antioughtable|N/A|4450salmon1595961332|white|Tbl|Unknown|39|eseeingoughtoughtought| +11185|AAAAAAAABLLCAAAA|1997-10-27||Global days could occur thoroughly ethnic unions. Different costs may demonstrate else; police continue for a men. European, social weeks meet sure social, significant ways. Rare, ordinary structures |6.03|4.28|10004017|edu packunivamalg #17|4|audio|10|Electronics|586|callyeinganti|N/A|606912907turquoise16|light|Ton|Unknown|11|antieingoughtoughtought| +11186|AAAAAAAACLLCAAAA|1997-10-27|2000-10-26|Ends may not grow rarely red winds. Different, local pictures may change around new efforts. Well special appointments|8.74|2.79|2004001|edu packimporto #1|4|sports-apparel|2|Men|41|oughtese|medium|2673sky6671357883260|mint|Bunch|Unknown|12|callyeingoughtoughtought| +11187|AAAAAAAACLLCAAAA|2000-10-27||Ends may not grow rarely red winds. Different, local pictures may change around new efforts. Well special appointments|6.53|4.96|2004001|amalgexporti #2|4|newborn|3|Children|41|oughtese|medium|2673sky6671357883260|puff|Cup|Unknown|38|ationeingoughtoughtought| +11188|AAAAAAAAELLCAAAA|1997-10-27|1999-10-27|Changes ought to sell about. Independent, unable facts can rise in the eggs. Independent doors ask in a papers; changes might think old police. Beautiful, natural memories must realise princi|1.27|0.91|10009003|maxiunivamalg #3|9|televisions|10|Electronics|982|ableeingn st|N/A|78836273antique17964|tomato|N/A|Unknown|2|eingeingoughtoughtought| +11189|AAAAAAAAELLCAAAA|1999-10-28|2001-10-26|British, frequent instructions might|7.65|3.97|8016010|corpmaxi #10|9|golf|8|Sports|982|ableeingn st|N/A|78836273antique17964|violet|Gram|Unknown|91|n steingoughtoughtought| +11190|AAAAAAAAELLCAAAA|2001-10-27||New, able risks will resist however then great tests. Ultimate, nuclear americans may cost. Indian concessions wish about both aware scientists. Basic crews propose through th|9.61|3.97|8016010|corpbrand #1|6|rugs|7|Home|373|ableeingn st|N/A|78836273antique17964|navy|Ounce|Unknown|47|barn stoughtoughtought| +11191|AAAAAAAAHLLCAAAA|1997-10-27||More full messages behave chips. Professionals must know high tenants. Light clothes must answer values.|0.97|0.85|8003002|exportinameless #2|3|basketball|8|Sports|31|oughtpri|N/A|16736402saddle576043|spring|Dram|Unknown|25|oughtn stoughtoughtought| +11192|AAAAAAAAILLCAAAA|1997-10-27|2000-10-26|Sure, small unions score. More national systems ought to verify between a skills. Even huge arrangements may move long current economic countries. Available, good |9.16|4.94|1002001|importoamalg #1|2|fragrances|1|Women|180|bareingought|large|sienna28085223046720|beige|Gram|Unknown|41|ablen stoughtoughtought| +11193|AAAAAAAAILLCAAAA|2000-10-27||Sure, small unions score. More national systems ought to verify between a skills. Even huge arrangements may move long current economic countries. Available, good |5.14|4.94|1002001|importoimporto #2|2|shirts|2|Men|558|eingantianti|small|sienna28085223046720|plum|Each|Unknown|25|prin stoughtoughtought| +11194|AAAAAAAAKLLCAAAA|1997-10-27|1999-10-27|Therefore public economies will continue si|6.58|2.76|4001001|amalgedu pack #1|1|womens|4|Shoes|878|eingationeing|medium|purple92073921142645|pink|Tsp|Unknown|30|esen stoughtoughtought| +11195|AAAAAAAAKLLCAAAA|1999-10-28|2001-10-26|Therefore public economies will continue si|4.38|2.10|5002002|importoscholar #2|1|country|5|Music|473|eingationeing|N/A|purple92073921142645|powder|Dram|Unknown|8|antin stoughtoughtought| +11196|AAAAAAAAKLLCAAAA|2001-10-27||Products move also; short farmers will speak wi|86.95|2.10|10003011|exportiunivamalg #11|1|dvd/vcr players|10|Electronics|473|priationese|N/A|0419992papaya0542394|royal|Dram|Unknown|34|callyn stoughtoughtought| +11197|AAAAAAAANLLCAAAA|1997-10-27||Here existing objects will help marvellous, inner beds. Rich aims act particularly divine studies. Political, empty areas could use terms. No doubt leading doubts used to construct |5.89|5.12|2004002|edu packimporto #2|4|sports-apparel|2|Men|134|esepriought|small|1931orange2685630235|turquoise|Ton|Unknown|38|ationn stoughtoughtought| +11198|AAAAAAAAOLLCAAAA|1997-10-27|2000-10-26|Necessary, old controls may not attend as sharp arguments. Reaction|5.62|3.09|3003001|exportiexporti #1|3|toddlers|3|Children|389|n steingpri|medium|6716yellow9683491284|purple|Box|Unknown|59|eingn stoughtoughtought| +11199|AAAAAAAAOLLCAAAA|2000-10-27||Necessary, old controls may not attend as sharp arguments. Reaction|8.93|7.59|3003001|importomaxi #4|2|business|9|Books|623|priablecally|N/A|6716yellow9683491284|royal|Gross|Unknown|34|n stn stoughtoughtought| +11200|AAAAAAAAAMLCAAAA|1997-10-27|1999-10-27|Things cover cheeks. Other minutes might take only white things. Recent, monetary activities come level, serious companies; e|74.68|61.23|7013001|exportinameless #1|13|wallpaper|7|Home|299|n stn stable|N/A|3974835649tomato8070|spring|Pound|Unknown|59|barbarableoughtought| +11201|AAAAAAAAAMLCAAAA|1999-10-28|2001-10-26|Things cover cheeks. Other minutes might take only white things. Recent, monetary activities come level, serious companies; e|8.65|6.22|4004002|edu packedu pack #2|4|athletic|4|Shoes|299|n stn stable|large|5snow565571026155989|pink|Dram|Unknown|41|oughtbarableoughtought| +11202|AAAAAAAAAMLCAAAA|2001-10-27||Things cover cheeks. Other minutes might take only white things. Recent, monetary activities come level, serious companies; e|7.93|4.75|4004002|brandnameless #3|4|hockey|8|Sports|636|callyprically|N/A|5snow565571026155989|salmon|Gram|Unknown|45|ablebarableoughtought| +11203|AAAAAAAADMLCAAAA|1997-10-27||Less traditional visitors might say services. Decades receive here redundant voters. So academic techniques used to pro|4.22|1.39|3003002|exportiexporti #2|3|toddlers|3|Children|329|n stablepri|medium|44902998969443peach7|powder|Box|Unknown|17|pribarableoughtought| +11204|AAAAAAAAEMLCAAAA|1997-10-27|2000-10-26|Only familiar shareholders could ensure very military electoral needs. Troubles imagine at fi|0.84|0.73|7002005|importobrand #5|2|bedding|7|Home|890|barn steing|N/A|8169454365tomato7556|turquoise|Oz|Unknown|42|esebarableoughtought| +11205|AAAAAAAAEMLCAAAA|2000-10-27||Points would go. So unaware heroes can know thus to a levels. Shareholders watch definite roads; very bloody books get so; short, desperate values c|4.13|0.73|7002005|amalgnameless #6|1|athletic shoes|8|Sports|50|baranti|N/A|359mint9040814419015|orange|Each|Unknown|68|antibarableoughtought| +11206|AAAAAAAAGMLCAAAA|1997-10-27|1999-10-27|Ago possible girls should know holidays. A|55.31|47.01|4003001|exportiedu pack #1|3|kids|4|Shoes|552|ableantianti|large|792208puff2583589795|red|Pound|Unknown|3|callybarableoughtought| +11207|AAAAAAAAGMLCAAAA|1999-10-28|2001-10-26|Ago possible girls should know holidays. A|7.37|47.01|9001012|amalgmaxi #12|1|arts|9|Books|552|ableantianti|N/A|4218551royal48012398|light|Tbl|Unknown|42|ationbarableoughtought| +11208|AAAAAAAAGMLCAAAA|2001-10-27||Ago possible girls should know holidays. A|1.59|0.47|9001012|edu packexporti #1|4|school-uniforms|3|Children|2|ableantianti|medium|4218551royal48012398|papaya|Oz|Unknown|65|eingbarableoughtought| +11209|AAAAAAAAJMLCAAAA|1997-10-27||Cultural, french theories die instead t|0.41|0.29|6013002|exportibrand #2|13|loose stones|6|Jewelry|277|ationationable|N/A|57817073321thistle21|medium|Bundle|Unknown|59|n stbarableoughtought| +11210|AAAAAAAAKMLCAAAA|1997-10-27|2000-10-26|Deaf, involved churches cannot conduct especially in public fundamental shares. Brief difficulties use|9.15|7.32|4001001|amalgedu pack #1|1|womens|4|Shoes|382|ableeingpri|petite|0460907298019snow749|midnight|Ounce|Unknown|63|baroughtableoughtought| +11211|AAAAAAAAKMLCAAAA|2000-10-27||Deaf, involved churches cannot conduct especially in public fundamental shares. Brief difficulties use|8.99|7.32|4001001|exportiimporto #2|1|pants|2|Men|382|ableeingpri|medium|0460907298019snow749|pink|Bunch|Unknown|7|oughtoughtableoughtought| +11212|AAAAAAAAMMLCAAAA|1997-10-27|1999-10-27|Even logical men enjoy bones. Whole troops will save well secondly disabled boundaries. American, poor mind|4.76|1.95|7009003|maxibrand #3|9|mattresses|7|Home|167|ationcallyought|N/A|40024064538steel4345|royal|N/A|Unknown|16|ableoughtableoughtought| +11213|AAAAAAAAMMLCAAAA|1999-10-28|2001-10-26|Right,|0.51|0.33|6010006|univbrand #6|10|jewelry boxes|6|Jewelry|737|ationcallyought|N/A|24879seashell8890033|lime|Carton|Unknown|52|prioughtableoughtought| +11214|AAAAAAAAMMLCAAAA|2001-10-27||Right,|2.76|0.33|6006007|corpcorp #7|6|rings|6|Jewelry|207|ationbarable|N/A|8turquoise1979375178|snow|Lb|Unknown|96|eseoughtableoughtought| +11215|AAAAAAAAPMLCAAAA|1997-10-27||Only guilty games will write always flexible studies. Involved, regular procedures will not feel; dishes try daily resources. Environmental devices might not see sooner purposes. Full, |7.21|3.82|1004002|edu packamalg #2|4|swimwear|1|Women|322|ableablepri|medium|0510341khaki98355693|goldenrod|Tsp|Unknown|49|antioughtableoughtought| +11216|AAAAAAAAANLCAAAA|1997-10-27|2000-10-26|United, personal shops work very needs. Clients focus radically different conditions. Outwards cheerful boys will not surrender most possible fut|7.99|6.23|7015007|scholarnameless #7|15|tables|7|Home|2|able|N/A|83528022spring296815|papaya|Oz|Unknown|1|callyoughtableoughtought| +11217|AAAAAAAAANLCAAAA|||Symptoms pay soon in a traders. German, necessary goals take between a shows; local, civil men will not agree |1.58|1.10|7015007||4||4||2||||smoke|||3|| +11218|AAAAAAAACNLCAAAA|1997-10-27|1999-10-27|Enough new |4.42|2.34|1003001|exportiamalg #1|3|maternity|1|Women|61|oughtcally|economy|2345701grey728414262|seashell|Gram|Unknown|41|eingoughtableoughtought| +11219|AAAAAAAACNLCAAAA|1999-10-28|2001-10-26|Once more similar fingers can exercise well without a parties. |1.70|2.34|1003001|amalgexporti #2|1|newborn|3|Children|57|ationanti|medium|56150408467104peach0|steel|Each|Unknown|21|n stoughtableoughtought| +11220|AAAAAAAACNLCAAAA|2001-10-27||Now other foreigners should not obtain either new, english woods. Never flat shoulders go further difficu|11.72|2.34|4004001|edu packedu pack #1|1|athletic|4|Shoes|511|oughtoughtanti|medium|0251sienna8666840069|purple|Oz|Unknown|83|barableableoughtought| +11221|AAAAAAAAFNLCAAAA|1997-10-27||Recently other animals hesitate real, theoretical examples. Afraid varieties cope more as the differences. Other, political cars operate scientists. New, subject yea|0.64|0.40|4004002|edu packedu pack #2|4|athletic|4|Shoes|68|eingcally|petite|7058522royal10630180|pale|Pallet|Unknown|42|oughtableableoughtought| +11222|AAAAAAAAGNLCAAAA|1997-10-27|2000-10-26|Other, foreign policies pay however in the symptoms. Alone readers will not offer sometimes in a bedrooms. More other years ought to restore more new, angry runs. Unique, |1.23|0.72|10002003|importounivamalg #3|2|camcorders|10|Electronics|613|prioughtcally|N/A|98aquamarine40085777|tan|Box|Unknown|49|ableableableoughtought| +11223|AAAAAAAAGNLCAAAA|2000-10-27||Other, foreign policies pay however in the symptoms. Alone readers will not offer sometimes in a bedrooms. More other years ought to restore more new, angry runs. Unique, |8.75|6.91|10002003|amalgimporto #2|2|accessories|2|Men|613|prioughtcally|petite|98aquamarine40085777|smoke|Dram|Unknown|6|priableableoughtought| +11224|AAAAAAAAINLCAAAA|1997-10-27|1999-10-27|Religious, other agencies pay; too domestic women promote together digital, commercial tec|0.74|0.25|5004001|edu packscholar #1|4|classical|5|Music|257|ationantiable|N/A|6thistle366541548460|floral|Case|Unknown|27|eseableableoughtought| +11225|AAAAAAAAINLCAAAA|1999-10-28|2001-10-26|Religious, other agencies pay; too domestic women promote together digital, commercial tec|3.83|2.52|5004001|importoscholar #2|2|country|5|Music|257|ationantiable|N/A|3294664555tan6397027|plum|Each|Unknown|43|antiableableoughtought| +11226|AAAAAAAAINLCAAAA|2001-10-27||Religious, other agencies pay; too domestic women promote together digital, commercial tec|3.82|2.52|5004001|exportimaxi #7|3|computers|9|Books|257|ationantiable|N/A|3294664555tan6397027|smoke|Carton|Unknown|17|callyableableoughtought| +11227|AAAAAAAALNLCAAAA|1997-10-27||Also impossible trains shall not achieve there by a humans. Ideas say. Other men shall take only from a differences. Police practise incorrectly on a institutions. Important|9.10|7.73|6014004|edu packbrand #4|14|estate|6|Jewelry|577|ationationanti|N/A|228947salmon00468481|wheat|Each|Unknown|33|ationableableoughtought| +11228|AAAAAAAAMNLCAAAA|1997-10-27|2000-10-26|Busy theories shall see also. English files will bring eur|4.63|3.47|6004005|edu packcorp #5|4|bracelets|6|Jewelry|139|n stpriought|N/A|068902ivory460824367|red|Lb|Unknown|7|eingableableoughtought| +11229|AAAAAAAAMNLCAAAA|2000-10-27||Fair members believe now in a pp.. Individuals could not involve new, national pr|8.63|3.47|6004005|scholarbrand #6|4|custom|6|Jewelry|173|n stpriought|N/A|52662442348631green2|snow|Pound|Unknown|4|n stableableoughtought| +11230|AAAAAAAAONLCAAAA|1997-10-27|1999-10-27|Ever small eyes buy though round, strong companies. Essential, proper towns might die enough sensitive, scottish times; countries go for a im|7.12|4.69|6003007|exporticorp #7|3|gold|6|Jewelry|603|pribarcally|N/A|68691782metallic9623|ghost|Cup|Unknown|62|barpriableoughtought| +11231|AAAAAAAAONLCAAAA|1999-10-28|2001-10-26|Generally bitter words will make to the cities. Casual, left files expect as russian, useful tourists; political sources answer german, resulting relations. Issues must |6.67|4.80|7013002|exportinameless #2|13|wallpaper|7|Home|302|pribarcally|N/A|9745822177752yellow4|red|Lb|Unknown|14|oughtpriableoughtought| +11232|AAAAAAAAONLCAAAA|2001-10-27||Generally bitter words will make to the cities. Casual, left files expect as russian, useful tourists; political sources answer german, resulting relations. Issues must |2.29|4.80|6006001|corpcorp #1|6|rings|6|Jewelry|276|pribarcally|N/A|38893sky620356149880|sky|Each|Unknown|2|ablepriableoughtought| +11233|AAAAAAAABOLCAAAA|1997-10-27||Beautiful balls must not know fully new parts. Clear, aware others include also too new hundre|3.22|2.73|10010014|univamalgamalg #14|10|memory|10|Electronics|617|ationoughtcally|N/A|934100midnight292931|powder|Carton|Unknown|24|pripriableoughtought| +11234|AAAAAAAACOLCAAAA|1997-10-27|2000-10-26|Facilities should benefit so absolute, british gates. Old, new hands shall clear better. Weeks could keep women; perceptions fit. So critical holidays point|6.84|2.32|3002001|importoexporti #1|2|infants|3|Children|86|callyeing|petite|06310209seashell3220|snow|Ounce|Unknown|73|esepriableoughtought| +11235|AAAAAAAACOLCAAAA|2000-10-27||Facilities should benefit so absolute, british gates. Old, new hands shall clear better. Weeks could keep women; perceptions fit. So critical holidays point|7.71|4.24|3002001|exportiexporti #2|2|toddlers|3|Children|548|eingeseanti|extra large|06310209seashell3220|rose|Ounce|Unknown|10|antipriableoughtought| +11236|AAAAAAAAEOLCAAAA|1997-10-27|1999-10-27|High regions gain images. Unhappy women shall advise more military years. Occasional, social letters dream again. Lawyers need of course. Foreign, traditional eyes divide still. Private childre|8.80|7.48|5003001|exportischolar #1|3|pop|5|Music|569|n stcallyanti|N/A|57232moccasin4719721|purple|Ounce|Unknown|17|callypriableoughtought| +11237|AAAAAAAAEOLCAAAA|1999-10-28|2001-10-26|High regions gain images. Unhappy women shall advise more military years. Occasional, social letters dream again. Lawyers need of course. Foreign, traditional eyes divide still. Private childre|1.10|7.48|5003001|maxiunivamalg #1|9|televisions|10|Electronics|569|n stcallyanti|N/A|311white656836492125|plum|Tbl|Unknown|10|ationpriableoughtought| +11238|AAAAAAAAEOLCAAAA|2001-10-27||Slim, indirect services s|1.85|0.94|5002001|importoscholar #1|2|country|5|Music|569|n stcallyanti|N/A|311white656836492125|lime|Gram|Unknown|3|eingpriableoughtought| +11239|AAAAAAAAHOLCAAAA|1997-10-27||Also new actions cannot tell friends; perfect forms adopt. Dogs could take either separate systems. Very burning directors draw established, labou|8.33|2.58|3002002|importoexporti #2|2|infants|3|Children|86|callyeing|small|50376030050230snow06|puff|Bundle|Unknown|18|n stpriableoughtought| +11240|AAAAAAAAIOLCAAAA|1997-10-27|2000-10-26|Confident, good men should decide small men. Literary, bad parties ought to balance successfully|1.30|0.53|2002001|importoimporto #1|2|shirts|2|Men|343|priesepri|small|20thistle57650699704|rose|Dozen|Unknown|56|bareseableoughtought| +11241|AAAAAAAAIOLCAAAA|2000-10-27||Confident, good men should decide small men. Literary, bad parties ought to balance successfully|2.38|1.88|8014002|edu packmaxi #2|2|tennis|8|Sports|243|prieseable|N/A|20thistle57650699704|steel|Ton|Unknown|9|oughteseableoughtought| +11242|AAAAAAAAKOLCAAAA|1997-10-27|1999-10-27|Wo|9.89|7.12|2002001|importoimporto #1|2|shirts|2|Men|253|priantiable|large|795369514red65789605|dodger|Dram|Unknown|3|ableeseableoughtought| +11243|AAAAAAAAKOLCAAAA|1999-10-28|2001-10-26|White stories last employees; just fatal meetings know only in a churches. Right periods come away full plants. Particular, nervous lists should mean so. Never academic lines ought to let|3.34|7.12|2002001|scholarmaxi #4|2|fishing|8|Sports|253|priantiable|N/A|795369514red65789605|pink|Tbl|Unknown|19|prieseableoughtought| +11244|AAAAAAAAKOLCAAAA|2001-10-27||Medical, competit|2.91|1.65|2002001|exportiimporto #1|3|pants|2|Men|234|esepriable|petite|436112636cream525756|green|Dram|Unknown|7|eseeseableoughtought| +11245|AAAAAAAANOLCAAAA|1997-10-27||Operations ought to launch already for a hands. However popular changes accrue p|4.72|3.25|1002002|importoamalg #2|2|fragrances|1|Women|155|antiantiought|extra large|5528166130beige31941|peach|Gram|Unknown|40|antieseableoughtought| +11246|AAAAAAAAOOLCAAAA|1997-10-27|2000-10-26|Wages will view. Still popular contracts say otherwise left men. Either local associations might raise successful problems. As main publishers like for the problems. Effectively local cases |4.52|3.25|10003010|exportiunivamalg #10|3|dvd/vcr players|10|Electronics|592|ablen stanti|N/A|4994seashell68601381|seashell|Tsp|Unknown|15|callyeseableoughtought| +11247|AAAAAAAAOOLCAAAA|2000-10-27||Bodies can see different, fine fem|0.61|0.28|10003010|maxinameless #8|9|optics|8|Sports|393|ablen stanti|N/A|4994seashell68601381|sienna|Cup|Unknown|42|ationeseableoughtought| +11248|AAAAAAAAAPLCAAAA|1997-10-27|1999-10-27|Atomic, unique shelves might distinguish very, only artists. New, ill floors must change. Months worry later here unexpected facts. Other, other resources |6.79|4.20|6002001|importocorp #1|2|diamonds|6|Jewelry|71|oughtation|N/A|00597280016metallic3|magenta|Case|Unknown|28|eingeseableoughtought| +11249|AAAAAAAAAPLCAAAA|1999-10-28|2001-10-26|Assistant, original manufacturers exploit roughly little years. Of course sure models may not integrate particularly. Institutions ought to look in particular month|3.71|4.20|10008013|namelessunivamalg #13|8|scanners|10|Electronics|71|oughtation|N/A|5443439248powder7537|lavender|Ounce|Unknown|56|n steseableoughtought| +11250|AAAAAAAAAPLCAAAA|2001-10-27||Assistant, original manufacturers exploit roughly little years. Of course sure models may not integrate particularly. Institutions ought to look in particular month|51.47|46.32|4004001|edu packedu pack #1|8|athletic|4|Shoes|263|oughtation|petite|hot74333609489811931|powder|Oz|Unknown|14|barantiableoughtought| +11251|AAAAAAAADPLCAAAA|1997-10-27||Super, large attacks produce later in a guidelines. Thirdly british meals give |4.35|3.17|2001002|amalgimporto #2|1|accessories|2|Men|402|ablebarese|large|18olive9301487482370|royal|Pound|Unknown|12|oughtantiableoughtought| +11252|AAAAAAAAEPLCAAAA|1997-10-27|2000-10-26|Guilty arms cannot go publicly new others. Respective, new details used to happen approximately subsequent pictures; low countries beat officials. Clear, new fees say. Letters become so sou|0.96|0.62|4004001|edu packedu pack #1|4|athletic|4|Shoes|238|eingpriable|large|3royal53067442083141|pale|Ton|Unknown|49|ableantiableoughtought| +11253|AAAAAAAAEPLCAAAA|2000-10-27||Guilty arms cannot go publicly new others. Respective, new details used to happen approximately subsequent pictures; low countries beat officials. Clear, new fees say. Letters become so sou|2.81|1.20|4004001|amalgamalg #2|4|dresses|1|Women|238|eingpriable|large|3royal53067442083141|rosy|Pallet|Unknown|79|priantiableoughtought| +11254|AAAAAAAAGPLCAAAA|1997-10-27|1999-10-27|Even indian methods leave de|1.92|1.61|1004001|edu packamalg #1|4|swimwear|1|Women|281|oughteingable|medium|02559powder119688356|maroon|Ton|Unknown|14|eseantiableoughtought| +11255|AAAAAAAAGPLCAAAA|1999-10-28|2001-10-26|Even indian methods leave de|0.16|1.61|1004001|corpmaxi #12|4|parenting|9|Books|281|oughteingable|N/A|02559powder119688356|peach|Dram|Unknown|18|antiantiableoughtought| +11256|AAAAAAAAGPLCAAAA|2001-10-27||Now regional factors move much |7.90|4.42|1004001|amalgexporti #1|1|newborn|3|Children|281|oughteingable|large|26297368881purple516|thistle|Ounce|Unknown|19|callyantiableoughtought| +11257|AAAAAAAAJPLCAAAA|1997-10-27||Bands speak even white pp.. Reliable, brown cases gather sure, big books; complex eyes know much other, special numbers. Findings will conven|2.94|0.91|5002002|importoscholar #2|2|country|5|Music|182|ableeingought|N/A|9353685goldenrod3086|tan|Each|Unknown|29|ationantiableoughtought| +11258|AAAAAAAAKPLCAAAA|1997-10-27|2000-10-26|Therefore blue participants must not try abroad cases. Complex,|6.38|4.91|5004001|edu packscholar #1|4|classical|5|Music|50|baranti|N/A|84726646purple827101|medium|Case|Unknown|9|eingantiableoughtought| +11259|AAAAAAAAKPLCAAAA|2000-10-27||Therefore blue participants must not try abroad cases. Complex,|56.71|4.91|7013010|exportinameless #10|13|wallpaper|7|Home|50|baranti|N/A|84726646purple827101|wheat|N/A|Unknown|78|n stantiableoughtought| +11260|AAAAAAAAMPLCAAAA|1997-10-27|1999-10-27|Welsh, red hours shall not agree public, certain components; then exciting minutes should avoid quite white blank organisers. That real systems will put at last measures. Never|0.81|0.71|7014001|edu packnameless #1|14|glassware|7|Home|239|n stpriable|N/A|7273472honeydew15198|spring|Ton|Unknown|63|barcallyableoughtought| +11261|AAAAAAAAMPLCAAAA|1999-10-28|2001-10-26|Welsh, red hours shall not agree public, certain components; then exciting minutes should avoid quite white blank organisers. That real systems will put at last measures. Never|2.22|0.71|2004002|edu packimporto #2|4|sports-apparel|2|Men|430|n stpriable|medium|7273472honeydew15198|cream|Bundle|Unknown|85|oughtcallyableoughtought| +11262|AAAAAAAAMPLCAAAA|2001-10-27||Welsh, red hours shall not agree public, certain components; then exciting minutes should avoid quite white blank organisers. That real systems will put at last measures. Never|1.75|0.56|3003001|exportiexporti #1|4|toddlers|3|Children|430|n stpriable|extra large|7273472honeydew15198|linen|Ounce|Unknown|51|ablecallyableoughtought| +11263|AAAAAAAAPPLCAAAA|1997-10-27||Also little situations recover profits. Due, proper animals make seats. Managers explain histor|40.23|28.96|1002002|importoamalg #2|2|fragrances|1|Women|671|oughtationcally|medium|0873587531purple5588|plum|Cup|Unknown|5|pricallyableoughtought| +11264|AAAAAAAAAAMCAAAA|1997-10-27|2000-10-26||||8011005|amalgmaxi #5||||Sports||eseableable|N/A|||Pound||2|esecallyableoughtought| +11265|AAAAAAAAAAMCAAAA|2000-10-27||Quickly controversial f|17.79|7.82|8011005|importoedu pack #2|2|mens|4|Shoes|168|eseableable|N/A|9211rose420012808674|burnished|Tbl|Unknown|21|anticallyableoughtought| +11266|AAAAAAAACAMCAAAA|1997-10-27|1999-10-27|Whole critics kiss executives. Slow, usual movements circumvent as from a difficulties. Low, european children make also girls. Individuals work yet eligible topics. Social unions t|2.50|1.82|4003001|exportiedu pack #1|3|kids|4|Shoes|610|baroughtcally|large|429684orchid38724979|gainsboro|Unknown|Unknown|53|callycallyableoughtought| +11267|AAAAAAAACAMCAAAA|1999-10-28|2001-10-26|Relevant conditions worry by the stages. Attractive products wish also |3.63|1.82|2002002|importoimporto #2|2|shirts|2|Men|610|baroughtcally|extra large|967026806199slate559|mint|Dram|Unknown|64|ationcallyableoughtought| +11268|AAAAAAAACAMCAAAA|2001-10-27||Relevant conditions worry by the stages. Attractive products wish also |4.19|2.43|2002002|importoscholar #1|2|country|5|Music|33|baroughtcally|N/A|60825563sandy6334706|violet|N/A|Unknown|13|eingcallyableoughtought| +11269|AAAAAAAAFAMCAAAA|1997-10-27||Weekly ships cannot sit police. Simple, social projects perform varieties. Parents see more in general possible policies. Players see medical sales; regular, personal standar|2.05|0.82|2001002|amalgimporto #2|1|accessories|2|Men|665|anticallycally|extra large|549737tan28498124982|pink|Ton|Unknown|29|n stcallyableoughtought| +11270|AAAAAAAAGAMCAAAA|1997-10-27|2000-10-26|Studies search national, recent transactions. Other, moving degrees give young, provincial eyes. Legally new cattle would address none the less as a concentrations. Facts might seem either comparable |6.13|4.29|9012003|importounivamalg #3|12|home repair|9|Books|280|bareingable|N/A|3898863royal31414419|snow|Dram|Unknown|1|barationableoughtought| +11271|AAAAAAAAGAMCAAAA|2000-10-27||Studies search national, recent transactions. Other, moving degrees give young, provincial eyes. Legally new cattle would address none the less as a concentrations. Facts might seem either comparable |7.03|4.29|1002002|importoamalg #2|12|fragrances|1|Women|280|bareingable|medium|66103734377tomato963|pale|Dram|Unknown|11|oughtationableoughtought| +11272|AAAAAAAAIAMCAAAA|1997-10-27|1999-10-27|Very financial ministers eat vigorously. Other questions may research upside down blue matters. Weak, electronic forces relax military keys. Especially enormous police collapse per|7.85|2.90|9007005|brandmaxi #5|7|reference|9|Books|533|priprianti|N/A|1peach48053923149641|brown|Tbl|Unknown|61|ableationableoughtought| +11273|AAAAAAAAIAMCAAAA|1999-10-28|2001-10-26|Very financial ministers eat vigorously. Other questions may research upside down blue matters. Weak, electronic forces relax military keys. Especially enormous police collapse per|2.90|2.90|9007005|amalgbrand #4|7|bathroom|7|Home|18|eingought|N/A|1peach48053923149641|red|Oz|Unknown|45|priationableoughtought| +11274|AAAAAAAAIAMCAAAA|2001-10-27||Very financial ministers eat vigorously. Other questions may research upside down blue matters. Weak, electronic forces relax military keys. Especially enormous police collapse per|5.03|2.90|9007005|importoedu pack #1|7|mens|4|Shoes|18|eingought|large|1peach48053923149641|tomato|Bunch|Unknown|9|eseationableoughtought| +11275|AAAAAAAALAMCAAAA|1997-10-27||Different, social years should resign. International clothes exist |4.54|2.90|4001002|amalgedu pack #2|1|womens|4|Shoes|520|barableanti|large|276369482snow1957005|thistle|Ton|Unknown|29|antiationableoughtought| +11276|AAAAAAAAMAMCAAAA|1997-10-27|2000-10-26|Authorities used to leave exactly other co|2.14|0.72|6003005|exporticorp #5|3|gold|6|Jewelry|325|antiablepri|N/A|44wheat8876395919008|pink|Ton|Unknown|24|callyationableoughtought| +11277|AAAAAAAAMAMCAAAA|2000-10-27||Outwards statutory investors will not refuse still towards a professionals. Entire, rich years want early, worthy chiefs. Years join syste|7.90|0.72|5003002|exportischolar #2|3|pop|5|Music|31|antiablepri|N/A|44wheat8876395919008|sienna|Ton|Unknown|27|ationationableoughtought| +11278|AAAAAAAAOAMCAAAA|1997-10-27|1999-10-27|Specific, possible sentences ought to run pictures. Parents should summarize and so on fine households. Other concepts explore too years. Honest stars must cost psychologi|3.18|2.44|7010009|univnameless #9|10|flatware|7|Home|80|bareing|N/A|505408607indian04293|seashell|N/A|Unknown|56|eingationableoughtought| +11279|AAAAAAAAOAMCAAAA|1999-10-28|2001-10-26|Dead areas come senior, direct books. Likely complaints may discuss high on the po|4.94|1.58|6009008|maxicorp #8|9|womens watch|6|Jewelry|172|ableationought|N/A|505408607indian04293|saddle|Pallet|Unknown|29|n stationableoughtought| +11280|AAAAAAAAOAMCAAAA|2001-10-27||Minutes would get just american, economic animals. Following years ensure from a facilities. Racial boots reach already policies. Yet strong reports hold to a feet. Active origins cause. O|2.25|1.62|1001001|amalgamalg #1|1|dresses|1|Women|172|ableationought|small|505408607indian04293|smoke|Gram|Unknown|4|bareingableoughtought| +11281|AAAAAAAABBMCAAAA|1997-10-27||Colleges shall return interests. Late, outer stairs accept in spite of a standards. Estates pull almost from a issues. Hardly dead elements clear over there chinese hours; europea|4.09|3.02|10012011|importoamalgamalg #11|12|monitors|10|Electronics|550|barantianti|N/A|tomato61282220932512|plum|Each|Unknown|12|oughteingableoughtought| +11282|AAAAAAAACBMCAAAA|1997-10-27|2000-10-26|Modern, primary arms might wait often international, real doctors. So-called, whole things may deci|4.14|2.19|5002001|importoscholar #1|2|country|5|Music|677|ationationcally|N/A|05711powder357636895|wheat|Bunch|Unknown|85|ableeingableoughtought| +11283|AAAAAAAACBMCAAAA|2000-10-27||Bars could not tell reduced, academic soldiers. Here judicial negotiations cou|2.10|2.19|9008004|namelessmaxi #4|8|romance|9|Books|275|ationationcally|N/A|05711powder357636895|olive|Pound|Unknown|31|prieingableoughtought| +11284|AAAAAAAAEBMCAAAA|1997-10-27|1999-10-27|Reactions will|4.49|2.87|8001005|amalgnameless #5|1|athletic shoes|8|Sports|249|n steseable|N/A|42pale64117246656506|sienna|Each|Unknown|10|eseeingableoughtought| +11285|AAAAAAAAEBMCAAAA|1999-10-28|2001-10-26|Reactions will|9.18|8.17|8001005|edu packamalg #2|1|swimwear|1|Women|249|n steseable|large|43906715smoke6087061|plum|Tbl|Unknown|3|antieingableoughtought| +11286|AAAAAAAAEBMCAAAA|2001-10-27||Stations would like feet. Strong difficulties may bring. Very light others go original years. So-called books could contact alternat|4.89|2.83|8001005|edu packscholar #1|4|classical|5|Music|52|n steseable|N/A|43906715smoke6087061|spring|Ounce|Unknown|16|callyeingableoughtought| +11287|AAAAAAAAHBMCAAAA|1997-10-27||More short personnel find so coming items; happy years gather usually only very agents. Bodies disappear but bombs. Quickly anxious premises should give english, honest letters. C|7.99|6.39|1001002|amalgamalg #2|1|dresses|1|Women|292|ablen stable|N/A|4015791120brown69240|royal|Ton|Unknown|31|ationeingableoughtought| +11288|AAAAAAAAIBMCAAAA|1997-10-27|2000-10-26|Literally experienced women le|3.83|2.79|7001009|amalgbrand #9|1|bathroom|7|Home|164|esecallyought|N/A|0092053021purple9527|wheat|Pallet|Unknown|42|eingeingableoughtought| +11289|AAAAAAAAIBMCAAAA|2000-10-27||Today new rates help little relevant rights; regardless romantic directions will vote in a facilities. Very bright feet may come definitely old, le|3.63|1.99|6002002|importocorp #2|2|diamonds|6|Jewelry|564|esecallyought|N/A|purple94648591549916|steel|Oz|Unknown|58|n steingableoughtought| +11290|AAAAAAAAKBMCAAAA|1997-10-27|1999-10-27|Widesp|4.73|1.89|8001001|amalgnameless #1|1|athletic shoes|8|Sports|581|oughteinganti|N/A|4941895605orange3994|yellow|Bundle|Unknown|44|barn stableoughtought| +11291|AAAAAAAAKBMCAAAA|1999-10-28|2001-10-26|Appointed, literary reader|3.48|1.89|8001001|amalgimporto #2|1|accessories|2|Men|581|oughteinganti|large|2chartreuse671197864|maroon|Box|Unknown|79|oughtn stableoughtought| +11292|AAAAAAAAKBMCAAAA|2001-10-27||Able courses stretch only just substantial areas. New supplies can come larg|71.50|1.89|9005001|scholarmaxi #1|1|history|9|Books|581|oughteinganti|N/A|2chartreuse671197864|pink|Carton|Unknown|70|ablen stableoughtought| +11293|AAAAAAAANBMCAAAA|1997-10-27||Never other women used to leave past, only consultants. Much combined ages shall not watch today present local spirits. Women think s|9.67|8.70|5004002|edu packscholar #2|4|classical|5|Music|435|antipriese|N/A|593103olive428672983|mint|Carton|Unknown|32|prin stableoughtought| +11294|AAAAAAAAOBMCAAAA|1997-10-27|2000-10-26|Again secret |6.39|5.23|7015005|scholarnameless #5|15|tables|7|Home|312|ableoughtpri|N/A|91946082252788plum42|red|Lb|Unknown|10|esen stableoughtought| +11295|AAAAAAAAOBMCAAAA|2000-10-27||Safe, english gifts must not succeed strongly parts. Businessmen might creep. Solid, standard eyes |84.94|5.23|10002008|importounivamalg #8|2|camcorders|10|Electronics|16|ableoughtpri|N/A|5peru807610374823587|peach|Cup|Unknown|14|antin stableoughtought| +11296|AAAAAAAAACMCAAAA|1997-10-27|1999-10-27|In particular explicit publications used to like well babies. Participants used to|26.87|22.30|7014007|edu packnameless #7|14|glassware|7|Home|33|pripri|N/A|05706474lemon9068002|peru|Dram|Unknown|16|callyn stableoughtought| +11297|AAAAAAAAACMCAAAA|1999-10-28|2001-10-26|In particular explicit publications used to like well babies. Participants used to|0.52|0.29|7014007|amalgexporti #2|1|newborn|3|Children|232|pripri|petite|05706474lemon9068002|steel|Cup|Unknown|70|ationn stableoughtought| +11298|AAAAAAAAACMCAAAA|2001-10-27||In particular explicit publications used to like well babies. Participants used to|3.01|1.05|7014007|importobrand #9|2|bedding|7|Home|232|ablepriable|N/A|95210054066rose30969|salmon|Ounce|Unknown|14|eingn stableoughtought| +11299|AAAAAAAADCMCAAAA|1997-10-27||For instance demanding illustrations seem too. Problems find much economic qualifications. Communist, little user|4.48|3.53|3002002|importoexporti #2|2|infants|3|Children|313|prioughtpri|large|misty992763493380721|steel|Lb|Unknown|29|n stn stableoughtought| +11300|AAAAAAAAECMCAAAA|1997-10-27|2000-10-26|Rather vast companies pose quiet, actual carers. Close times take only simple possibilities. Current events might say only on a foundation|67.28|36.33|9004009|edu packmaxi #9|4|entertainments|9|Books|270|barationable|N/A|1951plum146663973544|lawn|Carton|Unknown|20|barbarprioughtought| +11301|AAAAAAAAECMCAAAA|2000-10-27||Prime, hard taxes could not return elderly miners. About naked hours pick even men; |7.21|4.61|5003002|exportischolar #2|4|pop|5|Music|270|barationable|N/A|1951plum146663973544|peach|Ounce|Unknown|8|oughtbarprioughtought| +11302|AAAAAAAAGCMCAAAA|1997-10-27|1999-10-27|Options suffer soon other rooms; professionals can get at all by a men. Still good times eat clearly on a subjects. All practical cattle improve quickly alone courts. Top, comm|3.36|2.99|1004001|edu packamalg #1|4|swimwear|1|Women|594|esen stanti|economy|362navy8863445051371|wheat|Gram|Unknown|8|ablebarprioughtought| +11303|AAAAAAAAGCMCAAAA|1999-10-28|2001-10-26|British lines ought to realise then clothes. Unlikely plants exercise absently fully new neighbours. Capitalist agencies start more. Eventual, lucky children must pronounce on a minerals|0.98|0.50|1004001|amalgedu pack #2|4|womens|4|Shoes|594|esen stanti|large|376589538198saddle64|plum|Pound|Unknown|62|pribarprioughtought| +11304|AAAAAAAAGCMCAAAA|2001-10-27||Technical, other assets undertake children. Now golden families succeed for example reluctant values. Practical memories a|9.07|4.71|1004001|edu packexporti #1|4|school-uniforms|3|Children|594|esen stanti|medium|376589538198saddle64|papaya|Tbl|Unknown|12|esebarprioughtought| +11305|AAAAAAAAJCMCAAAA|1997-10-27||Special, cold supporters see operators. Sales must meet. Doubtfully distinct months cannot raise painful, other parts; evolutionary animals might choose also;|24.74|9.40|2003002|exportiimporto #2|3|pants|2|Men|243|prieseable|medium|42680252096pale71999|pale|Bundle|Unknown|26|antibarprioughtought| +11306|AAAAAAAAKCMCAAAA|1997-10-27|2000-10-26|Strange facts used to show much only, possible|2.51|1.73|5003001|exportischolar #1|3|pop|5|Music|111|oughtoughtought|N/A|5600535324345khaki71|khaki|Each|Unknown|51|callybarprioughtought| +11307|AAAAAAAAKCMCAAAA|2000-10-27||Strange facts used to show much only, possible|3.02|2.11|8001008|amalgnameless #8|1|athletic shoes|8|Sports|205|antibarable|N/A|5600535324345khaki71|purple|Tsp|Unknown|48|ationbarprioughtought| +11308|AAAAAAAAMCMCAAAA|1997-10-27|1999-10-27|Reasonable boys shall not cause quite. Fingers explain. Private, young countries bury political, |4.17|2.83|5003001|exportischolar #1|3|pop|5|Music|614|eseoughtcally|N/A|6694486715160tan6839|puff|N/A|Unknown|23|eingbarprioughtought| +11309|AAAAAAAAMCMCAAAA|1999-10-28|2001-10-26|Rates come obvious, nuclear parents. Popular, perfect schools warrant automatically specifi|2.56|2.83|5003001|amalgimporto #2|1|accessories|2|Men|614|eseoughtcally|small|6694486715160tan6839|sky|Unknown|Unknown|43|n stbarprioughtought| +11310|AAAAAAAAMCMCAAAA|2001-10-27||Rates come obvious, nuclear parents. Popular, perfect schools warrant automatically specifi|33.62|2.83|7011001|amalgnameless #1|1|accent|7|Home|614|eseoughtcally|N/A|1326363229462611tan8|wheat|Dram|Unknown|34|baroughtprioughtought| +11311|AAAAAAAAPCMCAAAA|1997-10-27||Local cases might not authorise considerably upper millions; careful defences suggest other tonnes. Reports begin also just conservative years. Practical, fun women could tap demands; studies|2.85|1.73|3004002|edu packexporti #2|4|school-uniforms|3|Children|485|antieingese|medium|9502878indian2927122|thistle|Ton|Unknown|31|oughtoughtprioughtought| +11312|AAAAAAAAADMCAAAA|1997-10-27|2000-10-26|Public dangers provide usually powerful, old pains. Interesting, professional children should not conceal conditions. Therefore imp|8.64|4.32|3002001|importoexporti #1|2|infants|3|Children|220|barableable|extra large|4471554407lime567786|orange|Tsp|Unknown|14|ableoughtprioughtought| +11313|AAAAAAAAADMCAAAA|2000-10-27||Nerves may provide most to a representatives. Sad, available mechanisms give foreign projects. Loans imagine once recent, biological facilities. Economic directors |3.39|2.67|3002001|corpunivamalg #10|2|mystery|9|Books|220|barableable|N/A|4471554407lime567786|gainsboro|Tsp|Unknown|25|prioughtprioughtought| +11314|AAAAAAAACDMCAAAA|1997-10-27|1999-10-27|Typically internal agents run figures. Blue aff|2.22|0.68|5003001|exportischolar #1|3|pop|5|Music|199|n stn stought|N/A|43806670pale93346492|steel|Bundle|Unknown|13|eseoughtprioughtought| +11315|AAAAAAAACDMCAAAA|1999-10-28|2001-10-26|Typically internal agents run figures. Blue aff|60.76|0.68|2001002|amalgimporto #2|3|accessories|2|Men|199|n stn stought|extra large|43806670pale93346492|salmon|Case|Unknown|84|antioughtprioughtought| +11316|AAAAAAAACDMCAAAA|2001-10-27||United, english years defend genetic colours. Countries may not get either long unions. Banks know still alike |1.55|0.68|2001002|edu packscholar #1|3|classical|5|Music|199|n stn stought|N/A|43806670pale93346492|orchid|Unknown|Unknown|44|callyoughtprioughtought| +11317|AAAAAAAAFDMCAAAA|1997-10-27||Long walls may clarify cases. New chairs will attract legal patients. Functions disc|8.06|5.15|9006002|corpmaxi #2|6|parenting|9|Books|523|priableanti|N/A|slate785024685218198|rosy|Pound|Unknown|17|ationoughtprioughtought| +11318|AAAAAAAAGDMCAAAA|1997-10-27|2000-10-26|General, full visits may appeal once heavy days. Artists |4.59|1.37|1004001|edu packamalg #1|4|swimwear|1|Women|355|antiantipri|extra large|306078pink8091669429|sienna|N/A|Unknown|1|eingoughtprioughtought| +11319|AAAAAAAAGDMCAAAA|2000-10-27||General, full visits may appeal once heavy days. Artists |3.57|1.92|3003002|exportiexporti #2|3|toddlers|3|Children|48|antiantipri|large|1salmon0053122618400|papaya|Pound|Unknown|50|n stoughtprioughtought| +11320|AAAAAAAAIDMCAAAA|1997-10-27|1999-10-27|Female, |1.58|0.97|8013001|exportimaxi #1|13|sailing|8|Sports|26|callyable|N/A|59593757356dodger494|spring|Pallet|Unknown|41|barableprioughtought| +11321|AAAAAAAAIDMCAAAA|1999-10-28|2001-10-26|Female, |51.97|19.22|5003002|exportischolar #2|13|pop|5|Music|548|eingeseanti|N/A|0papaya1760503044858|royal|N/A|Unknown|5|oughtableprioughtought| +11322|AAAAAAAAIDMCAAAA|2001-10-27||Recent walls used to take however separate officers. Both fresh doors can meet more possible plans. Common principles keep about major preparations. Very right dimensions know c|4.58|19.22|6016003|corpbrand #3|16|consignment|6|Jewelry|325|eingeseanti|N/A|8729307473328misty09|smoke|Pound|Unknown|2|ableableprioughtought| +11323|AAAAAAAALDMCAAAA|1997-10-27||Other yards should not impede less detailed metres. There tired|57.15|49.72|5003002|exportischolar #2|3|pop|5|Music|27|ationable|N/A|1458smoke68887836362|mint|Gram|Unknown|18|priableprioughtought| +11324|AAAAAAAAMDMCAAAA|1997-10-27|2000-10-26|Total theories fail over p|3.15|0.97|4003001|exportiedu pack #1|3|kids|4|Shoes|53|prianti|large|392smoke263953045246|steel|Gram|Unknown|25|eseableprioughtought| +11325|AAAAAAAAMDMCAAAA|2000-10-27||Religious heads shall place local reasons. Great parents |8.27|0.97|1001002|amalgamalg #2|3|dresses|1|Women|53|prianti|medium|78steel6627524825355|tomato|Tbl|Unknown|55|antiableprioughtought| +11326|AAAAAAAAODMCAAAA|1997-10-27|1999-10-27|Resources leave directly types. Benefits might talk to the techniques. Prisoners will not suggest away british defendants. As very police look at a prisons. Important, |5.68|4.94|7009007|maxibrand #7|9|mattresses|7|Home|854|eseantieing|N/A|4snow852530821540867|ghost|Dozen|Unknown|25|callyableprioughtought| +11327|AAAAAAAAODMCAAAA|1999-10-28|2001-10-26|Resources leave directly types. Benefits might talk to the techniques. Prisoners will not suggest away british defendants. As very police look at a prisons. Important, |3.20|4.94|7009007|edu packexporti #2|9|school-uniforms|3|Children|67|eseantieing|medium|9587133ivory59032003|saddle|Gross|Unknown|43|ationableprioughtought| +11328|AAAAAAAAODMCAAAA|2001-10-27||Resources leave directly types. Benefits might talk to the techniques. Prisoners will not suggest away british defendants. As very police look at a prisons. Important, |3.10|0.96|10010007|univamalgamalg #7|9|memory|10|Electronics|67|eseantieing|N/A|lime7217367960349661|violet|Ounce|Unknown|56|eingableprioughtought| +11329|AAAAAAAABEMCAAAA|1997-10-27||Evident roots think below; specialist beds join marked roads. Well as|1.61|1.30|8015010|scholarmaxi #10|15|fishing|8|Sports|351|oughtantipri|N/A|942566268631wheat811|saddle|Each|Unknown|47|n stableprioughtought| +11330|AAAAAAAACEMCAAAA|1997-10-27|2000-10-26|Early, victorian preparations shall produce expensive, thick respects. Effective periods agree only police. Perfect, potential times keep ideally in a pieces. Patients allocate much linguist|2.10|1.32|10005009|scholarunivamalg #9|5|karoke|10|Electronics|378|eingationpri|N/A|4325320ghost98808755|navy|Pound|Unknown|29|barpriprioughtought| +11331|AAAAAAAACEMCAAAA|2000-10-27||Early, victorian preparations shall produce expensive, thick respects. Effective periods agree only police. Perfect, potential times keep ideally in a pieces. Patients allocate much linguist|1.75|1.32|10005009|edu packexporti #2|5|school-uniforms|3|Children|378|eingationpri|economy|4325320ghost98808755|red|Carton|Unknown|6|oughtpriprioughtought| +11332|AAAAAAAAEEMCAAAA|1997-10-27|1999-10-27|Extra, annual kinds produce other lights. Successful pp. should not tell home in a husbands. Centres ho|87.93|58.91|9005005|scholarmaxi #5|5|history|9|Books|180|bareingought|N/A|59229green2781866473|dodger|Gram|Unknown|20|ablepriprioughtought| +11333|AAAAAAAAEEMCAAAA|1999-10-28|2001-10-26|Extra, annual kinds produce other lights. Successful pp. should not tell home in a husbands. Centres ho|1.01|0.56|9005005|edu packamalg #2|5|swimwear|1|Women|180|bareingought|large|527goldenrod36161318|peru|Unknown|Unknown|18|pripriprioughtought| +11334|AAAAAAAAEEMCAAAA|2001-10-27||Particular rights must leave past, other contracts. Suddenly essential letters can preach finall|4.21|0.56|9005005|amalgbrand #7|5|semi-precious|6|Jewelry|88|bareingought|N/A|527goldenrod36161318|smoke|Box|Unknown|92|esepriprioughtought| +11335|AAAAAAAAHEMCAAAA|1997-10-27||Vaguely other departments would sleep for example. Symbolic, interesting nations shall not build for the schools. Primary, dear figures should not get adult movements. Similar, |0.42|0.26|1003002|exportiamalg #2|3|maternity|1|Women|450|barantiese|large|85grey37463054175433|wheat|Box|Unknown|52|antipriprioughtought| +11336|AAAAAAAAIEMCAAAA|1997-10-27|2000-10-26|Regional types cannot master practically capable models. Sad areas withdraw ago new charges. Simple, certain activities would grab other teams; smal|3.60|2.70|2003001|exportiimporto #1|3|pants|2|Men|105|antibarought|petite|166263lace6984997235|peach|Tsp|Unknown|25|callypriprioughtought| +11337|AAAAAAAAIEMCAAAA|2000-10-27||Regional types cannot master practically capable models. Sad areas withdraw ago new charges. Simple, certain activities would grab other teams; smal|24.12|2.70|2003001|corpcorp #2|6|rings|6|Jewelry|105|antibarought|N/A|166263lace6984997235|seashell|Dram|Unknown|83|ationpriprioughtought| +11338|AAAAAAAAKEMCAAAA|1997-10-27|1999-10-27|Political authorities must not change continually per annum common initiatives. At once central facts change; environmental, classical materials vary als|0.91|0.79|2002001|importoimporto #1|2|shirts|2|Men|656|callyantically|medium|7596lemon72162087561|honeydew|Each|Unknown|7|eingpriprioughtought| +11339|AAAAAAAAKEMCAAAA|1999-10-28|2001-10-26|Political authorities must not change continually per annum common initiatives. At once central facts change; environmental, classical materials vary als|0.32|0.10|3002002|importoexporti #2|2|infants|3|Children|949|n stesen st|medium|7596lemon72162087561|white|Carton|Unknown|41|n stpriprioughtought| +11340|AAAAAAAAKEMCAAAA|2001-10-27||Local, safe women use over birds. Possible tests must develop nearly heavy, judicial thanks. Restrictions can love solemnly for a designers. Electron|3.78|0.10|3002002|amalgbrand #1|1|bathroom|7|Home|949|n stesen st|N/A|755169083242blush403|tomato|Dram|Unknown|5|bareseprioughtought| +11341|AAAAAAAANEMCAAAA|1997-10-27||Br|3.77|1.77|6012006|importobrand #6|12|costume|6|Jewelry|309|n stbarpri|N/A|2394330844spring5013|rose|Bundle|Unknown|25|oughteseprioughtought| +11342|AAAAAAAAOEMCAAAA|1997-10-27|2000-10-26|Conclusions achieve as industr|5.43|2.66|9005003|scholarmaxi #3|5|history|9|Books|456|callyantiese|N/A|200448394581plum3934|thistle|Case|Unknown|65|ableeseprioughtought| +11343|AAAAAAAAOEMCAAAA|2000-10-27||Old, private members maintain pupils. Frequently young readers get ins|9.20|3.77|9005003|edu packexporti #2|4|school-uniforms|3|Children|82|ableeing|medium|9886781046goldenrod8|tomato|Ton|Unknown|1|prieseprioughtought| +11344|AAAAAAAAAFMCAAAA|1997-10-27|1999-10-27|Initial, neat initiatives cannot adapt in a views. Permanent patients control then more familiar kids. Current, rich matters will use. Too able systems define pages|82.29|72.41|9014005|edu packunivamalg #5|14|sports|9|Books|30|barpri|N/A|60tomato922416918782|grey|Dozen|Unknown|21|eseeseprioughtought| +11345|AAAAAAAAAFMCAAAA|1999-10-28|2001-10-26|Initial, neat initiatives cannot adapt in a views. Permanent patients control then more familiar kids. Current, rich matters will use. Too able systems define pages|5.26|72.41|7016006|corpnameless #6|14|furniture|7|Home|353|priantipri|N/A|60tomato922416918782|peru|Bundle|Unknown|13|antieseprioughtought| +11346|AAAAAAAAAFMCAAAA|2001-10-27||Initial, neat initiatives cannot adapt in a views. Permanent patients control then more familiar kids. Current, rich matters will use. Too able systems define pages|0.60|0.18|7016006|amalgnameless #7|1|athletic shoes|8|Sports|757|priantipri|N/A|3325pink347771212722|rose|Pound|Unknown|66|callyeseprioughtought| +11347|AAAAAAAADFMCAAAA|1997-10-27||Individuals go criminal, equivalent humans; female objects stand tools. Big changes tell even problems; religious, separate followers cannot jump; economic, old |31.20|26.20|10013009|exportiamalgamalg #9|13|stereo|10|Electronics|246|callyeseable|N/A|7205rosy808790160512|red|Ounce|Unknown|28|ationeseprioughtought| +11348|AAAAAAAAEFMCAAAA|1997-10-27|2000-10-26|Too sensitive results might breach|5.47|3.22|2002001|importoimporto #1|2|shirts|2|Men|723|priableation|large|88840982765violet864|plum|Bundle|Unknown|7|eingeseprioughtought| +11349|AAAAAAAAEFMCAAAA|2000-10-27||Too sensitive results might breach|7.46|4.40|2002002|importoimporto #2|2|shirts|2|Men|723|priableation|petite|88840982765violet864|tomato|Tsp|Unknown|16|n steseprioughtought| +11350|AAAAAAAAGFMCAAAA|1997-10-27|1999-10-27|Later warm sports might not believe once; miners cannot take apparently never true rules. Talks used to seem even stable ideas. Intimate, coherent payments help. Years see|3.31|2.61|7008001|namelessbrand #1|8|lighting|7|Home|281|oughteingable|N/A|689759333sky50211576|thistle|Dozen|Unknown|30|barantiprioughtought| +11351|AAAAAAAAGFMCAAAA|1999-10-28|2001-10-26|Only, regional shares recognise seriously. Cos|8.34|2.61|7008001|edu packscholar #2|8|classical|5|Music|104|oughteingable|N/A|689759333sky50211576|peach|Lb|Unknown|28|oughtantiprioughtought| +11352|AAAAAAAAGFMCAAAA|2001-10-27||Only, regional shares recognise seriously. Cos|1.51|2.61|7008001|exportiexporti #1|8|toddlers|3|Children|104|oughteingable|small|689759333sky50211576|lace|N/A|Unknown|64|ableantiprioughtought| +11353|AAAAAAAAJFMCAAAA|1997-10-27||Unusual, single months gain nuclear purposes; occasional patients might not hear central branches; modern resources influence faintly military comparisons. White thi|98.66|36.50|3002002|importoexporti #2|2|infants|3|Children|243|prieseable|large|99654948wheat7315641|smoke|Cup|Unknown|1|priantiprioughtought| +11354|AAAAAAAAKFMCAAAA|1997-10-27|2000-10-26|Endless, vocational contracts would not stabilise churches. French, good cities light somehow on a offices. Now serious things raise for a walls; certain, c|0.23|0.09|9011003|amalgunivamalg #3|11|cooking|9|Books|21|oughtable|N/A|099501095977974pale9|seashell|Cup|Unknown|31|eseantiprioughtought| +11355|AAAAAAAAKFMCAAAA|2000-10-27||Duly sad remains call yet workers; final grounds buy sharply. Following proposals might prevent other lakes. Evil, excellent resources ope|7.02|6.03|9011003|scholarnameless #6|11|fitness|8|Sports|250|barantiable|N/A|099501095977974pale9|peru|Each|Unknown|60|antiantiprioughtought| +11356|AAAAAAAAMFMCAAAA|1997-10-27|1999-10-27|Friends suffice also comple|3.30|1.84|5004001|edu packscholar #1|4|classical|5|Music|117|ationoughtought|N/A|31brown9438819504889|plum|Oz|Unknown|52|callyantiprioughtought| +11357|AAAAAAAAMFMCAAAA|1999-10-28|2001-10-26|Friends suffice also comple|55.92|1.84|5004001|exportischolar #2|3|pop|5|Music|117|ationoughtought|N/A|982438365misty768526|saddle|Lb|Unknown|53|ationantiprioughtought| +11358|AAAAAAAAMFMCAAAA|2001-10-27||Later early languages arrive only. Things should not survive particularly experimental, different approaches. Tenants will know aware payments. Huge meals cannot offer findings. Outsi|6.43|1.84|5004001|exportiexporti #1|3|toddlers|3|Children|84|ationoughtought|extra large|982438365misty768526|deep|Oz|Unknown|10|eingantiprioughtought| +11359|AAAAAAAAPFMCAAAA|1997-10-27||Global, ugly flowers can pray just parti|8.53|7.50|8008010|namelessnameless #10|8|outdoor|8|Sports|104|esebarought|N/A|87031782rose39573134|slate|Unknown|Unknown|79|n stantiprioughtought| +11360|AAAAAAAAAGMCAAAA|1997-10-27|2000-10-26|Irish elections may wear even. Offi|2.58|1.57|3002001|importoexporti #1|2|infants|3|Children|139|n stpriought|petite|72indian019323739269|violet|Cup|Unknown|10|barcallyprioughtought| +11361|AAAAAAAAAGMCAAAA|2000-10-27||Never happy occupations can make mainly individuals. Dark women stimulate days. Both growing authorities get different cases. Households see thin, low animals.|71.98|63.34|10007006|brandunivamalg #6|2|personal|10|Electronics|139|n stpriought|N/A|67rose15762787602110|spring|Oz|Unknown|28|oughtcallyprioughtought| +11362|AAAAAAAACGMCAAAA|1997-10-27|1999-10-27|Following feet explore. Actually long addresses should put especially constant, global movements. Hot nations should not fill oddly without a|1.08|0.90|6009003|maxicorp #3|9|womens watch|6|Jewelry|307|ationbarpri|N/A|17548769spring398689|midnight|Dozen|Unknown|10|ablecallyprioughtought| +11363|AAAAAAAACGMCAAAA|1999-10-28|2001-10-26|Following feet explore. Actually long addresses should put especially constant, global movements. Hot nations should not fill oddly without a|1.67|0.51|6015004|scholarbrand #4|9|custom|6|Jewelry|225|antiableable|N/A|17548769spring398689|mint|Box|Unknown|12|pricallyprioughtought| +11364|AAAAAAAACGMCAAAA|2001-10-27||Following feet explore. Actually long addresses should put especially constant, global movements. Hot nations should not fill oddly without a|4.12|0.51|8005005|scholarnameless #5|9|fitness|8|Sports|502|ablebaranti|N/A|17548769spring398689|rosy|Case|Unknown|26|esecallyprioughtought| +11365|AAAAAAAAFGMCAAAA|1997-10-27||Firms give home original efforts. Prices fail blind sessions. Spiritual, white computers should not sa|2.63|1.99|1001002|amalgamalg #2|1|dresses|1|Women|146|callyeseought|small|789sienna86701054317|sandy|N/A|Unknown|39|anticallyprioughtought| +11366|AAAAAAAAGGMCAAAA|1997-10-27|2000-10-26|Lightly mental views might not involve partly carefully real figures. Just continued terms look. Only new artists used to go very orders; even great women listen apparently. Formal, similar |5.35|4.65|8010007|univmaxi #7|10|pools|8|Sports|261|oughtcallyable|N/A|47040sandy5535515048|grey|Carton|Unknown|17|callycallyprioughtought| +11367|AAAAAAAAGGMCAAAA|2000-10-27||Lightly mental views might not involve partly carefully real figures. Just continued terms look. Only new artists used to go very orders; even great women listen apparently. Formal, similar |3.06|1.10|6010008|univbrand #8|10|jewelry boxes|6|Jewelry|261|oughtcallyable|N/A|47040sandy5535515048|saddle|Case|Unknown|27|ationcallyprioughtought| +11368|AAAAAAAAIGMCAAAA|1997-10-27|1999-10-27|Right parents exert by the drivers. Somehow important women indicate suddenly unexpected communities. Very children must remember well. Areas must consider quite now a|2.85|1.71|6014001|edu packbrand #1|14|estate|6|Jewelry|400|barbarese|N/A|10834326aquamarine77|tomato|Box|Unknown|43|eingcallyprioughtought| +11369|AAAAAAAAIGMCAAAA|1999-10-28|2001-10-26|Right parents exert by the drivers. Somehow important women indicate suddenly unexpected communities. Very children must remember well. Areas must consider quite now a|5.44|3.15|5001002|amalgscholar #2|1|rock|5|Music|400|barbarese|N/A|10834326aquamarine77|orange|N/A|Unknown|36|n stcallyprioughtought| +11370|AAAAAAAAIGMCAAAA|2001-10-27||Files might not take more early, national decades. Formal, scientific sides leave very on to a systems. Expectations add for a impl|7.00|3.15|10011015|amalgamalgamalg #15|11|disk drives|10|Electronics|400|barbarese|N/A|10834326aquamarine77|lawn|Case|Unknown|25|barationprioughtought| +11371|AAAAAAAALGMCAAAA|1997-10-27||Flowers cultivate still so-called, available|3.84|1.22|9004008|edu packmaxi #8|4|entertainments|9|Books|74|eseation|N/A|60407325118sandy6176|plum|Bundle|Unknown|71|oughtationprioughtought| +11372|AAAAAAAAMGMCAAAA|1997-10-27|2000-10-26|Complaints should make originally apparent, only experiences. Efficiently welsh prices will not say to a authorities. Other, small changes should know immensely. Dead, perfect examples take forwar|8.25|6.27|2004001|edu packimporto #1|4|sports-apparel|2|Men|174|eseationought|economy|7963207lemon72018966|turquoise|Case|Unknown|30|ableationprioughtought| +11373|AAAAAAAAMGMCAAAA|2000-10-27||Major, traditional sales learn european, sexual women. British pieces might not develo|2.94|1.96|3002002|importoexporti #2|4|infants|3|Children|527|ationableanti|large|7963207lemon72018966|moccasin|Tsp|Unknown|68|priationprioughtought| +11374|AAAAAAAAOGMCAAAA|1997-10-27|1999-10-27|Already necessary buildings will not |0.90|0.34|6013007|exportibrand #7|13|loose stones|6|Jewelry|974|eseationn st|N/A|612418559907white131|honeydew|Carton|Unknown|28|eseationprioughtought| +11375|AAAAAAAAOGMCAAAA|1999-10-28|2001-10-26|Already necessary buildings will not |2.38|2.14|1002002|importoamalg #2|13|fragrances|1|Women|974|eseationn st|large|612418559907white131|white|Pallet|Unknown|37|antiationprioughtought| +11376|AAAAAAAAOGMCAAAA|2001-10-27||Already necessary buildings will not |4.17|2.14|1002002|exportiamalg #1|3|maternity|1|Women|217|eseationn st|medium|612418559907white131|thistle|Dram|Unknown|45|callyationprioughtought| +11377|AAAAAAAABHMCAAAA|1997-10-27||Representatives co-operate open facilities. Late, local terms may not take however square, underlying p|42.98|25.35|6010002|univbrand #2|10|jewelry boxes|6|Jewelry|103|pribarought|N/A|1986702saddle5281764|rosy|Case|Unknown|91|ationationprioughtought| +11378|AAAAAAAACHMCAAAA|1997-10-27|2000-10-26|Available, ancient parties would veto all right genes. So|7.56|2.94|1001001|amalgamalg #1|1|dresses|1|Women|14|eseought|extra large|02225866maroon477234|papaya|Pallet|Unknown|66|eingationprioughtought| +11379|AAAAAAAACHMCAAAA|2000-10-27||Too industrial persons would not meet hardly new places; moral, false forms would not bring; other, basic members should not make in a children. |8.29|2.94|8005010|scholarnameless #10|5|fitness|8|Sports|14|eseought|N/A|02225866maroon477234|pink|Each|Unknown|91|n stationprioughtought| +11380|AAAAAAAAEHMCAAAA|1997-10-27|1999-10-27|Smart arts believe sure keen girls. Full parties explain alread|5.35|4.65|4003001|exportiedu pack #1|3|kids|4|Shoes|98|eingn st|large|yellow04361892491518|pink|Gross|Unknown|4|bareingprioughtought| +11381|AAAAAAAAEHMCAAAA|1999-10-28|2001-10-26|Smart arts believe sure keen girls. Full parties explain alread|79.11|4.65|4003001|edu packamalg #2|4|swimwear|1|Women|98|eingn st|small|877776907red62281889|purple|Ton|Unknown|9|oughteingprioughtought| +11382|AAAAAAAAEHMCAAAA|2001-10-27||Never legal eyebrows put in the condi|0.68|0.57|10002010|importounivamalg #10|2|camcorders|10|Electronics|147|ationeseought|N/A|86975168765seashell4|tomato|Oz|Unknown|94|ableeingprioughtought| +11383|AAAAAAAAHHMCAAAA|1997-10-27||Services might not catch accordingly shoes. More formal reasons break eyes; particular conditions display magnetic, full managers. Entirely historical approache|2.31|1.68|8004004|edu packnameless #4|4|camping|8|Sports|238|eingpriable|N/A|231535dodger22606421|snow|Dram|Unknown|30|prieingprioughtought| +11384|AAAAAAAAIHMCAAAA|1997-10-27|2000-10-26|Social, identical doubts might|4.59|2.15|8016005|corpmaxi #5|16|golf|8|Sports|57|ationanti|N/A|81032860floral995580|sienna|Each|Unknown|34|eseeingprioughtought| +11385|AAAAAAAAIHMCAAAA|2000-10-27||Over large aspects shall go. Issues will see already environmental months. Slightly effective changes live here|9.84|4.72|1002002|importoamalg #2|2|fragrances|1|Women|57|ationanti|large|81032860floral995580|smoke|Bunch|Unknown|17|antieingprioughtought| +11386|AAAAAAAAKHMCAAAA|1997-10-27|1999-10-27|Heavily alone schools might respond in a men. Local workshops support to a sides. Monthly wars wo|0.50|0.18|2002001|importoimporto #1|2|shirts|2|Men|158|eingantiought|extra large|7323royal54678089948|midnight|Case|Unknown|17|callyeingprioughtought| +11387|AAAAAAAAKHMCAAAA|1999-10-28|2001-10-26|Also appropriate courts let long critical conditions; enough current |3.99|0.18|8007008|brandnameless #8|7|hockey|8|Sports|158|eingantiought|N/A|9516medium4717600144|seashell|Box|Unknown|40|ationeingprioughtought| +11388|AAAAAAAAKHMCAAAA|2001-10-27||Much other rats release with the words. Rich, effective auditors might not know now severe diseases. Grateful, fair features take. Compani|61.78|0.18|5002001|importoscholar #1|2|country|5|Music|158|eingantiought|N/A|9516medium4717600144|slate|Pallet|Unknown|12|eingeingprioughtought| +11389|AAAAAAAANHMCAAAA|1997-10-27||About other levels should proceed certainly fine, severe facts.|3.78|2.98|6010006|univbrand #6|10|jewelry boxes|6|Jewelry|165|anticallyought|N/A|320059turquoise15039|salmon|Pallet|Unknown|37|n steingprioughtought| +11390|AAAAAAAAOHMCAAAA|1997-10-27|2000-10-26|Trees could include ahead marked payments; sizes could see golden, left members. Books can raise generally western circumstances. Military, s|6.13|1.90|4003001|exportiedu pack #1|3|kids|4|Shoes|748|eingeseation|medium|84113wheat1982467058|peach|Unknown|Unknown|41|barn stprioughtought| +11391|AAAAAAAAOHMCAAAA|2000-10-27||Trees could include ahead marked payments; sizes could see golden, left members. Books can raise generally western circumstances. Military, s|9.74|6.72|4003001|amalgcorp #8|3|birdal|6|Jewelry|810|eingeseation|N/A|964788saddle88950069|salmon|Case|Unknown|87|oughtn stprioughtought| +11392|AAAAAAAAAIMCAAAA|1997-10-27|1999-10-27|Planned, principal months could play excellent, immediate ideas. Little, hostile services will not react slowly by a features. R|6.76|2.77|9013005|exportiunivamalg #5|13|self-help|9|Books|274|eseationable|N/A|9puff539674173232055|snow|Carton|Unknown|81|ablen stprioughtought| +11393|AAAAAAAAAIMCAAAA|1999-10-28|2001-10-26|Planned, principal months could play excellent, immediate ideas. Little, hostile services will not react slowly by a features. R|1.13|2.77|9013005|exportinameless #4|13|wallpaper|7|Home|386|eseationable|N/A|992018472744lace1253|rosy|Tsp|Unknown|35|prin stprioughtought| +11394|AAAAAAAAAIMCAAAA|2001-10-27||Planned, principal months could play excellent, immediate ideas. Little, hostile services will not react slowly by a features. R|24.34|2.77|9013005|exportiimporto #1|3|pants|2|Men|386|callyeingpri|petite|992018472744lace1253|hot|Carton|Unknown|52|esen stprioughtought| +11395|AAAAAAAADIMCAAAA|1997-10-27||Over wide doctors measure successfully urban terms. Highly small women find with a years. Softly typical talks should not go now. Eventually primary places |56.39|32.70|5003002|exportischolar #2|3|pop|5|Music|71|oughtation|N/A|35247513seashell9646|sienna|Oz|Unknown|29|antin stprioughtought| +11396|AAAAAAAAEIMCAAAA|1997-10-27|2000-10-26|Particularly new cases join. Military, christian eyes lead widely suspicious players; finally special beings shall date at a trees; narrow aspects |9.61|6.53|7001007|amalgbrand #7|1|bathroom|7|Home|39|n stpri|N/A|1096203linen20022595|steel|Case|Unknown|8|callyn stprioughtought| +11397|AAAAAAAAEIMCAAAA|2000-10-27||Statistical versions avoid there like a companies; junior, trying months provide clearly ladies. Specif|1.29|0.86|5002002|importoscholar #2|2|country|5|Music|38|eingpri|N/A|198tan74241492733479|smoke|N/A|Unknown|28|ationn stprioughtought| +11398|AAAAAAAAGIMCAAAA|1997-10-27|1999-10-27|Traditional times buy on a operations. Clear, ne|9.63|8.47|7001009|amalgbrand #9|1|bathroom|7|Home|139|n stpriought|N/A|974699peru0098008944|violet|Box|Unknown|9|eingn stprioughtought| +11399|AAAAAAAAGIMCAAAA|1999-10-28|2001-10-26|Traditional times buy on a operations. Clear, ne|3.77|3.24|7001009|exportiunivamalg #10|3|dvd/vcr players|10|Electronics|139|n stpriought|N/A|974699peru0098008944|thistle|Dozen|Unknown|3|n stn stprioughtought| +11400|AAAAAAAAGIMCAAAA|2001-10-27||Traditional times buy on a operations. Clear, ne|7.14|2.21|8007001|brandnameless #1|7|hockey|8|Sports|294|esen stable|N/A|0065salmon0999320868|indian|Pallet|Unknown|18|barbareseoughtought| +11401|AAAAAAAAJIMCAAAA|1997-10-27||Furthermore royal developments may not unload later huge c|0.84|0.68|9014002|edu packunivamalg #2|14|sports|9|Books|304|esebarpri|N/A|9708292yellow2750942|smoke|Dram|Unknown|37|oughtbareseoughtought| +11402|AAAAAAAAKIMCAAAA|1997-10-27|2000-10-26|Alternatively bi|28.72|11.77|4004001|edu packedu pack #1|4|athletic|4|Shoes|598|eingn stanti|medium|5lawn482981158985750|peru|Bunch|Unknown|30|ablebareseoughtought| +11403|AAAAAAAAKIMCAAAA|2000-10-27||Wrong decisions used to let human suppliers. Possible birds can fill more very windows. Members change|99.63|63.76|8016004|corpmaxi #4|16|golf|8|Sports|598|eingn stanti|N/A|5lawn482981158985750|powder|Bundle|Unknown|5|pribareseoughtought| +11404|AAAAAAAAMIMCAAAA|1997-10-27|1999-10-27|About likely houses like international members. Final, relevant birds answer after the paintings. Hungry, personal days borrow tiny, primary resources. As social relations could choose quite also |0.77|0.36|9002005|importomaxi #5|2|business|9|Books|338|eingpripri|N/A|09725lemon4338930374|smoke|Unknown|Unknown|53|esebareseoughtought| +11405|AAAAAAAAMIMCAAAA|1999-10-28|2001-10-26|About likely houses like international members. Final, relevant birds answer after the paintings. Hungry, personal days borrow tiny, primary resources. As social relations could choose quite also |3.42|0.36|9002005|brandunivamalg #16|7|personal|10|Electronics|338|eingpripri|N/A|2482272367peru261746|saddle|Each|Unknown|6|antibareseoughtought| +11406|AAAAAAAAMIMCAAAA|2001-10-27||About likely houses like international members. Final, relevant birds answer after the paintings. Hungry, personal days borrow tiny, primary resources. As social relations could choose quite also |68.23|32.75|9002005|amalgimporto #1|1|accessories|2|Men|21|oughtable|petite|535pink8021631544438|sandy|Carton|Unknown|3|callybareseoughtought| +11407|AAAAAAAAPIMCAAAA|1997-10-27||Low purposes used to serve gradually. Practices may not come now other, basic children. White, close homes commission competent symptoms; blues ought to take now extremely interest|2.56|1.20|8001008|amalgnameless #8|1|athletic shoes|8|Sports|30|barpri|N/A|5560128081lace510042|coral|Dozen|Unknown|17|ationbareseoughtought| +11408|AAAAAAAAAJMCAAAA|1997-10-27|2000-10-26|African years may give nearly problems. New circumstances tell just among the shows. Repeatedly thick d|4.36|1.56|8005009|scholarnameless #9|5|fitness|8|Sports|524|eseableanti|N/A|804293307273red81224|pink|Box|Unknown|2|eingbareseoughtought| +11409|AAAAAAAAAJMCAAAA|2000-10-27||Existing communications remove still |47.75|1.56|8005009|importoscholar #2|5|country|5|Music|524|eseableanti|N/A|804293307273red81224|tomato|Gross|Unknown|33|n stbareseoughtought| +11410|AAAAAAAACJMCAAAA|1997-10-27|1999-10-27|Enormous, high problems may like nevertheless often possible minutes. Here white benefits|3.03|2.15|7015001|scholarnameless #1|15|tables|7|Home|257|ationantiable|N/A|69olive9132570267907|rosy|Ounce|Unknown|15|baroughteseoughtought| +11411|AAAAAAAACJMCAAAA|1999-10-28|2001-10-26|Other factors ought to write then about a differences. Technical reasons find national questions.|9.52|2.15|7015001|amalgnameless #2|15|accent|7|Home|257|ationantiable|N/A|69olive9132570267907|puff|Gross|Unknown|11|oughtoughteseoughtought| +11412|AAAAAAAACJMCAAAA|2001-10-27||Other factors ought to write then about a differences. Technical reasons find national questions.|1.41|0.76|7015001|amalgbrand #5|11|semi-precious|6|Jewelry|257|ationantiable|N/A|69olive9132570267907|snow|Each|Unknown|39|ableoughteseoughtought| +11413|AAAAAAAAFJMCAAAA|1997-10-27||Long notes protect therefore conservatives. Irish fingers can help dogs. Original institutions will appear really different, current difficulties. Potential governors get more. Only ma|7.91|3.32|3002002|importoexporti #2|2|infants|3|Children|170|barationought|extra large|91285tomato822569668|snow|Pallet|Unknown|84|prioughteseoughtought| +11414|AAAAAAAAGJMCAAAA|1997-10-27|2000-10-26|Long-term game|4.19|3.60|8016005|corpmaxi #5|16|golf|8|Sports|311|oughtoughtpri|N/A|2rosy957304039645268|red|Gross|Unknown|5|eseoughteseoughtought| +11415|AAAAAAAAGJMCAAAA|2000-10-27||Only, common jobs run; great, dependent flowers sh|70.28|40.76|8016005|exportiimporto #2|3|pants|2|Men|297|ationn stable|small|2rosy957304039645268|rosy|Gram|Unknown|53|antioughteseoughtought| +11416|AAAAAAAAIJMCAAAA|1997-10-27|1999-10-27|Positive, permanent makers can f|1.27|0.91|2004001|edu packimporto #1|4|sports-apparel|2|Men|117|ationoughtought|small|9989750446119grey906|slate|Each|Unknown|48|callyoughteseoughtought| +11417|AAAAAAAAIJMCAAAA|1999-10-28|2001-10-26|Positive, permanent makers can f|0.70|0.91|2004001|namelessmaxi #6|8|romance|9|Books|117|ationoughtought|N/A|77507848929puff20791|pale|Ounce|Unknown|61|ationoughteseoughtought| +11418|AAAAAAAAIJMCAAAA|2001-10-27||Girls should soar further. Possible, common farmers go constraints. Characters would get more than widespread police. Present, delightful bases can say right strong, clear solicitors.|7.82|4.14|2004001|edu packedu pack #1|8|athletic|4|Shoes|28|ationoughtought|extra large|77507848929puff20791|slate|Tsp|Unknown|34|eingoughteseoughtought| +11419|AAAAAAAALJMCAAAA|1997-10-27||I|63.40|39.30|10013013|exportiamalgamalg #13|13|stereo|10|Electronics|181|oughteingought|N/A|43rose51680361720553|wheat|Unknown|Unknown|1|n stoughteseoughtought| +11420|AAAAAAAAMJMCAAAA|1997-10-27|2000-10-26|Husbands drive open arms. Then competitive blacks will improve across a couples. Ready sections keep soon able firms. Political rates would continue boys; issues may take presumably|3.82|2.75|4003001|exportiedu pack #1|3|kids|4|Shoes|501|oughtbaranti|petite|plum2014288210824271|honeydew|Carton|Unknown|32|barableeseoughtought| +11421|AAAAAAAAMJMCAAAA|2000-10-27||Bizarre improvements should correct deeply good, sure cards. All right positive vessels fill in a consequences. Recommendations shall use estates. Following jobs fly fully sexual events. Woo|2.83|1.64|6016006|corpbrand #6|16|consignment|6|Jewelry|209|oughtbaranti|N/A|3944rose623009929465|powder|Dram|Unknown|18|oughtableeseoughtought| +11422|AAAAAAAAOJMCAAAA|1997-10-27|1999-10-27|Common games replace often. Etc royal members walk much all difficult trends. Ideas attract too forward low times. Different miles bother in general in the communities. Very visual groups shall se|4.87|4.13|2002001|importoimporto #1|2|shirts|2|Men|263|pricallyable|medium|antique3111961600433|linen|Bundle|Unknown|22|ableableeseoughtought| +11423|AAAAAAAAOJMCAAAA|1999-10-28|2001-10-26|Lucky, private centuries tell still bodies. Instead special readers publicise also alleged drivers. Joint fe|8.37|4.13|2002001|exportiedu pack #2|3|kids|4|Shoes|551|oughtantianti|medium|116368spring59437139|thistle|Bunch|Unknown|93|priableeseoughtought| +11424|AAAAAAAAOJMCAAAA|2001-10-27||Healthy readers stay please parents. Feelings can see that is new, complex sets; almost legal hundreds will not organise newly due, typical options. Only different areas ought to get lives. |2.85|4.13|2004001|edu packimporto #1|4|sports-apparel|2|Men|551|oughtantianti|medium|420272lemon764134857|royal|Pallet|Unknown|40|eseableeseoughtought| +11425|AAAAAAAABKMCAAAA|1997-10-27||Other, beautiful events know with a men. Ethical rights would not support. Further|1.29|0.46|6008002|namelesscorp #2|8|mens watch|6|Jewelry|321|oughtablepri|N/A|32salmon892290439818|peach|Oz|Unknown|65|antiableeseoughtought| +11426|AAAAAAAACKMCAAAA|1997-10-27|2000-10-26|Weak, previous arms use before. Publications must understand problems. New, social schools would not comprehend only large su|2.32|0.95|1003001|exportiamalg #1|3|maternity|1|Women|145|antieseought|large|606173401bisque00330|white|Pound|Unknown|51|callyableeseoughtought| +11427|AAAAAAAACKMCAAAA|2000-10-27||Free moments can wear |7.83|4.14|3001002|amalgexporti #2|3|newborn|3|Children|145|antieseought|medium|606173401bisque00330|khaki|Cup|Unknown|17|ationableeseoughtought| +11428|AAAAAAAAEKMCAAAA|1997-10-27|1999-10-27|Windows account northwards hours. Horses used to slip particular, domestic interests; as spare fields determine nationally major, afraid tensions|3.53|1.30|1004001|edu packamalg #1|4|swimwear|1|Women|130|barpriought|large|9353232312lavender79|blush|Bunch|Unknown|67|eingableeseoughtought| +11429|AAAAAAAAEKMCAAAA|1999-10-28|2001-10-26|Windows account northwards hours. Horses used to slip particular, domestic interests; as spare fields determine nationally major, afraid tensions|0.86|1.30|9007006|brandmaxi #6|4|reference|9|Books|332|ablepripri|N/A|9353232312lavender79|peach|Case|Unknown|26|n stableeseoughtought| +11430|AAAAAAAAEKMCAAAA|2001-10-27||Windows account northwards hours. Horses used to slip particular, domestic interests; as spare fields determine nationally major, afraid tensions|27.91|15.07|9007006|exportinameless #1|4|basketball|8|Sports|475|ablepripri|N/A|9353232312lavender79|salmon|Gram|Unknown|4|barprieseoughtought| +11431|AAAAAAAAHKMCAAAA|1997-10-27||Unexpected examples see both s|5.30|2.70|5001002|amalgscholar #2|1|rock|5|Music|160|barcallyought|N/A|3955680254058medium8|medium|Dozen|Unknown|46|oughtprieseoughtought| +11432|AAAAAAAAIKMCAAAA|1997-10-27|2000-10-26|Concerned man|1.77|1.20|3003001|exportiexporti #1|3|toddlers|3|Children|730|barpriation|petite|89339878692orange845|yellow|Carton|Unknown|47|ableprieseoughtought| +11433|AAAAAAAAIKMCAAAA|2000-10-27||Concerned man|0.33|1.20|1001002|amalgamalg #2|3|dresses|1|Women|128|eingableought|petite|89339878692orange845|wheat|Tsp|Unknown|7|priprieseoughtought| +11434|AAAAAAAAKKMCAAAA|1997-10-27|1999-10-27|There dominant companies used to justify very; more light things lo|8.12|5.84|1003001|exportiamalg #1|3|maternity|1|Women|405|antibarese|large|10136purple613646405|turquoise|Each|Unknown|26|eseprieseoughtought| +11435|AAAAAAAAKKMCAAAA|1999-10-28|2001-10-26|Old schemes would not permit. Methods give precisely. Memories see large recommendations; too elected |5.06|5.84|1003001|exportiunivamalg #12|3|self-help|9|Books|455|antibarese|N/A|10136purple613646405|olive|Cup|Unknown|4|antiprieseoughtought| +11436|AAAAAAAAKKMCAAAA|2001-10-27||Likely writers may not heal yet; light, working views stir beautifully. Obvious police could escape always constant, so-called readers. Very necessary hands get details. Men ch|7.70|4.85|4001001|amalgedu pack #1|1|womens|4|Shoes|86|antibarese|economy|0389metallic61707887|pink|Tbl|Unknown|89|callyprieseoughtought| +11437|AAAAAAAANKMCAAAA|1997-10-27||More african windows would think; special techniques may stick even new members. Just solar authoriti|0.18|0.14|5001002|amalgscholar #2|1|rock|5|Music|638|eingprically|N/A|89741957powder144089|pink|Pallet|Unknown|51|ationprieseoughtought| +11438|AAAAAAAAOKMCAAAA|1997-10-27|2000-10-26|Dark types dismiss surprisingly real, fashionable jews. Short proposals wo|0.72|0.31|6005007|scholarcorp #7|5|earings|6|Jewelry|318|eingoughtpri|N/A|272498074539689peru0|violet|Pallet|Unknown|6|eingprieseoughtought| +11439|AAAAAAAAOKMCAAAA|2000-10-27||Dark types dismiss surprisingly real, fashionable jews. Short proposals wo|20.64|0.31|6005007|corpbrand #10|6|rugs|7|Home|318|eingoughtpri|N/A|4459158643sandy67891|saddle|N/A|Unknown|7|n stprieseoughtought| +11440|AAAAAAAAALMCAAAA|1997-10-27|1999-10-27|Average pubs handle at all in the groups. Voters expect also other, british marks. Countries go always free officers; trousers would preserve. Routine g|4.59|1.97|6013001|exportibrand #1|13|loose stones|6|Jewelry|173|priationought|N/A|52413145peach4648767|tomato|Dozen|Unknown|30|bareseeseoughtought| +11441|AAAAAAAAALMCAAAA|1999-10-28|2001-10-26|Widely high subjects write so men. Forth growing expectations will meet years. Glasses may mind again for the times. Incidentally continuous windows would say functions; most old councillors may ret|0.41|0.22|2002002|importoimporto #2|13|shirts|2|Men|173|priationought|economy|52413145peach4648767|sandy|N/A|Unknown|38|oughteseeseoughtought| +11442|AAAAAAAAALMCAAAA|2001-10-27||Secret, main strings may contain now upward various children. Prisoners construct big, environmental |1.85|0.22|2002002|exportiexporti #1|3|toddlers|3|Children|332|ablepripri|petite|52413145peach4648767|plum|Oz|Unknown|10|ableeseeseoughtought| +11443|AAAAAAAADLMCAAAA|1997-10-27||Neatly single cases oug|1.51|1.11|4004002|edu packedu pack #2|4|athletic|4|Shoes|195|antin stought|economy|3434royal32851545794|papaya|Dozen|Unknown|45|prieseeseoughtought| +11444|AAAAAAAAELMCAAAA|1997-10-27|2000-10-26|Free, military clouds work so for a situations. As naked days must play just good, central services. Then active ages could consider more narrow, big sh|1.98|1.42|5001001|amalgscholar #1|1|rock|5|Music|618|eingoughtcally|N/A|212080373violet75653|sandy|Pallet|Unknown|75|eseeseeseoughtought| +11445|AAAAAAAAELMCAAAA|2000-10-27||Free, military clouds work so for a situations. As naked days must play just good, central services. Then active ages could consider more narrow, big sh|4.79|3.49|9011010|amalgunivamalg #10|1|cooking|9|Books|618|eingoughtcally|N/A|683447176midnight833|peach|Box|Unknown|63|antieseeseoughtought| +11446|AAAAAAAAGLMCAAAA|1997-10-27|1999-10-27|Crossly local relations know surely old excep|37.62|19.56|9002011|importomaxi #11|2|business|9|Books|148|eingeseought|N/A|17002092maroon373331|slate|Unknown|Unknown|26|callyeseeseoughtought| +11447|AAAAAAAAGLMCAAAA|1999-10-28|2001-10-26|Crossly local relations know surely old excep|2.32|19.56|5002002|importoscholar #2|2|country|5|Music|383|eingeseought|N/A|9820625papaya1520923|honeydew|Ounce|Unknown|10|ationeseeseoughtought| +11448|AAAAAAAAGLMCAAAA|2001-10-27||Crossly local relations know surely old excep|9.42|7.15|5002002|scholarunivamalg #1|15|fiction|9|Books|100|barbarought|N/A|9820625papaya1520923|sienna|Unknown|Unknown|22|eingeseeseoughtought| +11449|AAAAAAAAJLMCAAAA|1997-10-27||Bad, costly women m|4.77|1.57|1002002|importoamalg #2|2|fragrances|1|Women|597|ationn stanti|medium|75793165142saddle677|peru|Carton|Unknown|59|n steseeseoughtought| +11450|AAAAAAAAKLMCAAAA|1997-10-27|2000-10-26|Somehow forward prayers like only at a blocks. Never high approaches want indian publishers. Rea|1.11|0.57|5002001|importoscholar #1|2|country|5|Music|161|oughtcallyought|N/A|21875089maroon757640|peru|Ounce|Unknown|3|barantieseoughtought| +11451|AAAAAAAAKLMCAAAA|2000-10-27||Somehow forward prayers like only at a blocks. Never high approaches want indian publishers. Rea|1.94|0.57|3003002|exportiexporti #2|2|toddlers|3|Children|92|ablen st|extra large|21875089maroon757640|saddle|Case|Unknown|64|oughtantieseoughtought| +11452|AAAAAAAAMLMCAAAA|1997-10-27|1999-10-27|Ideal areas give forwards continuous, exciting priorities; parts will not make officers; individuals come future months. Open, big liabilities shall not pull just from a deaths; ju|2.54|1.39|6002005|importocorp #5|2|diamonds|6|Jewelry|169|n stcallyought|N/A|9757321222919floral1|red|Case|Unknown|1|ableantieseoughtought| +11453|AAAAAAAAMLMCAAAA|1999-10-28|2001-10-26|Ideal areas give forwards continuous, exciting priorities; parts will not make officers; individuals come future months. Open, big liabilities shall not pull just from a deaths; ju|19.11|1.39|4003002|exportiedu pack #2|2|kids|4|Shoes|169|n stcallyought|medium|184279837violet02399|sky|Ounce|Unknown|79|priantieseoughtought| +11454|AAAAAAAAMLMCAAAA|2001-10-27||Ideal areas give forwards continuous, exciting priorities; parts will not make officers; individuals come future months. Open, big liabilities shall not pull just from a deaths; ju|3.03|1.39|4003002|edu packunivamalg #7|2|sports|9|Books|169|n stcallyought|N/A|83849459053white1955|peru|Dram|Unknown|7|eseantieseoughtought| +11455|AAAAAAAAPLMCAAAA|1997-10-27||Patients end indeed minor, monetary wishes. Consequences could lend well; excellent, well-known persons cannot make effects; particularly unemployed rights dre|3.81|2.78|1004002|edu packamalg #2|4|swimwear|1|Women|20|barable|petite|237029787rose5523842|antique|Ounce|Unknown|48|antiantieseoughtought| +11456|AAAAAAAAAMMCAAAA|1997-10-27|2000-10-26|Important, green results lie above rare, good stones. Interested, able students shall catch almost proceedings. Probably new questions should not reduce natural, other pairs. Local governments aff|5.68|3.23|5003001|exportischolar #1|3|pop|5|Music|301|oughtbarpri|N/A|0olive11799238960260|yellow|Dram|Unknown|93|callyantieseoughtought| +11457|AAAAAAAAAMMCAAAA|2000-10-27||Schools would say extraordinarily addresses. Heavy members |8.87|6.03|5003001|edu packexporti #2|4|school-uniforms|3|Children|261|oughtcallyable|medium|5288799705368grey820|puff|Oz|Unknown|60|ationantieseoughtought| +11458|AAAAAAAACMMCAAAA|1997-10-27|1999-10-27|Interests fall really head, able theories. B|63.98|23.67|6014003|edu packbrand #3|14|estate|6|Jewelry|567|ationcallyanti|N/A|2481177668violet1154|puff|Case|Unknown|19|eingantieseoughtought| +11459|AAAAAAAACMMCAAAA|1999-10-28|2001-10-26|Interests fall really head, able theories. B|5.26|2.84|6014003|amalgedu pack #2|14|womens|4|Shoes|567|ationcallyanti|petite|87271341416royal1778|tomato|Bunch|Unknown|4|n stantieseoughtought| +11460|AAAAAAAACMMCAAAA|2001-10-27||Interests fall really head, able theories. B|66.09|31.72|5001001|amalgscholar #1|1|rock|5|Music|567|ationcallyanti|N/A|76348758yellow422521|royal|Pallet|Unknown|5|barcallyeseoughtought| +11461|AAAAAAAAFMMCAAAA|1997-10-27||Final, average changes |5.72|3.08|1002002|importoamalg #2|2|fragrances|1|Women|665|anticallycally|large|652260848plum0688462|rose|Tbl|Unknown|6|oughtcallyeseoughtought| +11462|AAAAAAAAGMMCAAAA|1997-10-27|2000-10-26|Chief countries leave actually rural, other fathers. Women discover very otherwise large ministers. Slow, envi|7.35|2.42|9011003|amalgunivamalg #3|11|cooking|9|Books|434|esepriese|N/A|47smoke0771302224583|sienna|N/A|Unknown|90|ablecallyeseoughtought| +11463|AAAAAAAAGMMCAAAA|2000-10-27||Various facts will help with a boys. Existing years evaluate yet new, able children. Feet shall not like else local, g|4.92|2.42|9011003|exportischolar #2|3|pop|5|Music|434|esepriese|N/A|47smoke0771302224583|rose|Oz|Unknown|22|pricallyeseoughtought| +11464|AAAAAAAAIMMCAAAA|1997-10-27|1999-10-27|Rows cannot give then magnetic children. Children join again very labour neighbours. Ways shoot. Horses could prepare little to a heels. Residential, stable issues disappear automaticall|31.00|24.18|8012005|importomaxi #5|12|guns|8|Sports|326|callyablepri|N/A|384087082862170pink3|snow|Box|Unknown|33|esecallyeseoughtought| +11465|AAAAAAAAIMMCAAAA|1999-10-28|2001-10-26|Rows cannot give then magnetic children. Children join again very labour neighbours. Ways shoot. Horses could prepare little to a heels. Residential, stable issues disappear automaticall|35.28|24.18|8012005|maxicorp #2|9|womens watch|6|Jewelry|326|callyablepri|N/A|0904074207920yellow5|navy|Case|Unknown|31|anticallyeseoughtought| +11466|AAAAAAAAIMMCAAAA|2001-10-27||No longer possible knees say at last processes. Naked hands shall keep roughly. Related fees lay rather accord|6.64|5.24|8012005|edu packscholar #1|9|classical|5|Music|326|callyablepri|N/A|37sienna290540780846|royal|Bunch|Unknown|28|callycallyeseoughtought| +11467|AAAAAAAALMMCAAAA|1997-10-27||Only wishes should|3.63|1.92|5003002|exportischolar #2|3|pop|5|Music|846|callyeseeing|N/A|89533snow13007119542|light|Ton|Unknown|30|ationcallyeseoughtought| +11468|AAAAAAAAMMMCAAAA|1997-10-27|2000-10-26|Provisions may use in a years. Likely suppliers inc|37.63|12.04|5001001|amalgscholar #1|1|rock|5|Music|125|antiableought|N/A|09816856767ghost8468|powder|Lb|Unknown|7|eingcallyeseoughtought| +11469|AAAAAAAAMMMCAAAA|2000-10-27||Provisions may use in a years. Likely suppliers inc|5.19|12.04|5001001|corpunivamalg #12|6|musical|10|Electronics|51|antiableought|N/A|09816856767ghost8468|sienna|Cup|Unknown|6|n stcallyeseoughtought| +11470|AAAAAAAAOMMCAAAA|1997-10-27|1999-10-27|Extensive, special opportunities may turn reactions; largely civil brothers |8.37|3.51|2003001|exportiimporto #1|3|pants|2|Men|526|callyableanti|large|5227592462lawn026961|lime|Ton|Unknown|50|barationeseoughtought| +11471|AAAAAAAAOMMCAAAA|1999-10-28|2001-10-26|Extensive, special opportunities may turn reactions; largely civil brothers |5.94|3.51|4002002|importoedu pack #2|2|mens|4|Shoes|397|ationn stpri|medium|5227592462lawn026961|lace|Dozen|Unknown|9|oughtationeseoughtought| +11472|AAAAAAAAOMMCAAAA|2001-10-27||Very intense years reserve only theoretically low users. Suddenly new volunteers shall mar|0.33|0.18|4002002|amalgunivamalg #15|2|cameras|10|Electronics|449|ationn stpri|N/A|5227592462lawn026961|green|Lb|Unknown|84|ableationeseoughtought| +11473|AAAAAAAABNMCAAAA|1997-10-27||Machines taste always top, likely interests. Results must bring only apart from a studies; true issues tell now poor procedures; long rules become almost secret diffi|1.28|0.81|9009002|maximaxi #2|9|science|9|Books|317|ationoughtpri|N/A|7864654525white42189|tan|Unknown|Unknown|28|priationeseoughtought| +11474|AAAAAAAACNMCAAAA|1997-10-27|2000-10-26|Right views may not|7.26|3.26|2004001|edu packimporto #1|4|sports-apparel|2|Men|349|n stesepri|medium|6865236491wheat05380|moccasin|Ton|Unknown|28|eseationeseoughtought| +11475|AAAAAAAACNMCAAAA|2000-10-27||Tired, slim eyes used to react rather; proposed, armed|6.66|2.06|2004001|edu packexporti #2|4|school-uniforms|3|Children|439|n stesepri|medium|984powder58262400685|olive|Tsp|Unknown|1|antiationeseoughtought| +11476|AAAAAAAAENMCAAAA|1997-10-27|1999-10-27|Hot, whole guests could think always on a dreams; only eligible variables seem severely annual, heavy drivers. Unnecessary, sexual themes vary as g|5.47|2.84|2004001|edu packimporto #1|4|sports-apparel|2|Men|316|callyoughtpri|extra large|18345359olive3539731|linen|Oz|Unknown|71|callyationeseoughtought| +11477|AAAAAAAAENMCAAAA|1999-10-28|2001-10-26|Hot, whole guests could think always on a dreams; only eligible variables seem severely annual, heavy drivers. Unnecessary, sexual themes vary as g|2.25|2.84|7001008|amalgbrand #8|4|bathroom|7|Home|316|callyoughtpri|N/A|18345359olive3539731|puff|Pound|Unknown|40|ationationeseoughtought| +11478|AAAAAAAAENMCAAAA|2001-10-27||Genuine, old sports resist all but for the years. Ancient shares will|0.77|0.65|6014007|edu packbrand #7|4|estate|6|Jewelry|654|callyoughtpri|N/A|2326996rosy882861807|smoke|Case|Unknown|9|eingationeseoughtought| +11479|AAAAAAAAHNMCAAAA|1997-10-27||Name|6.71|2.88|2003002|exportiimporto #2|3|pants|2|Men|266|callycallyable|large|58861lemon7019407451|purple|Ounce|Unknown|16|n stationeseoughtought| +11480|AAAAAAAAINMCAAAA|1997-10-27|2000-10-26|Other, alive hours would give. Improved, big weeks announce often original churches. Black phases drink. Very, typical meas|3.31|1.12|5002001|importoscholar #1|2|country|5|Music|209|n stbarable|N/A|57940seashell5316780|wheat|Lb|Unknown|47|bareingeseoughtought| +11481|AAAAAAAAINMCAAAA|2000-10-27||Other, alive hours would give. Improved, big weeks announce often original churches. Black phases drink. Very, typical meas|4.87|1.12|5002001|exportinameless #2|13|wallpaper|7|Home|194|esen stought|N/A|57940seashell5316780|lime|Gram|Unknown|20|oughteingeseoughtought| +11482|AAAAAAAAKNMCAAAA|1997-10-27|1999-10-27|Royal blues sort more systems; much public rules must not build over|5.34|1.65|8005003|scholarnameless #3|5|fitness|8|Sports|463|pricallyese|N/A|1002143776pink077939|indian|Ton|Unknown|34|ableeingeseoughtought| +11483|AAAAAAAAKNMCAAAA|1999-10-28|2001-10-26|Royal blues sort more systems; much public rules must not build over|4.13|2.18|8005003|exportischolar #2|3|pop|5|Music|596|pricallyese|N/A|1002143776pink077939|tan|Tsp|Unknown|29|prieingeseoughtought| +11484|AAAAAAAAKNMCAAAA|2001-10-27||Royal blues sort more systems; much public rules must not build over|6.00|3.54|8005003|edu packamalg #1|4|swimwear|1|Women|44|eseese|petite|671white852171551079|papaya|N/A|Unknown|12|eseeingeseoughtought| +11485|AAAAAAAANNMCAAAA|1997-10-27||Following affairs meet relevant pensioners. Social materials used to take however numbers. Campaigns must not wait devices. Even other sectors get dire|9.93|6.25|1003002|exportiamalg #2|3|maternity|1|Women|540|bareseanti|large|10lavender3911809973|metallic|Lb|Unknown|12|antieingeseoughtought| +11486|AAAAAAAAONMCAAAA|1997-10-27|2000-10-26|Spanish, likely professionals should te|5.56|2.55|9006003|corpmaxi #3|6|parenting|9|Books|276|callyationable|N/A|705plum4140796102535|sandy|Unknown|Unknown|29|callyeingeseoughtought| +11487|AAAAAAAAONMCAAAA|2000-10-27||Spanish, likely professionals should te|3.11|2.55|9006003|edu packimporto #2|4|sports-apparel|2|Men|276|callyationable|economy|705plum4140796102535|rose|Ton|Unknown|43|ationeingeseoughtought| +11488|AAAAAAAAAOMCAAAA|1997-10-27|1999-10-27|Public, simple eyes can say forever against a opportunities. About outside police u|9.04|6.78|7003009|exportibrand #9|3|kids|7|Home|211|oughtoughtable|N/A|855599924658slate459|deep|Tsp|Unknown|83|eingeingeseoughtought| +11489|AAAAAAAAAOMCAAAA|1999-10-28|2001-10-26|Products might regret now capital actions; indian sisters ought to make moreover afterwards vital activities; ministers help both; light, |2.54|6.78|7003009|importounivamalg #15|2|camcorders|10|Electronics|193|prin stought|N/A|855599924658slate459|red|Pallet|Unknown|20|n steingeseoughtought| +11490|AAAAAAAAAOMCAAAA|||||4.11||importonameless #1|||||||N/A|||Carton|||barn steseoughtought| +11491|AAAAAAAADOMCAAAA|1997-10-27||Main pupils could expel followers. Sometimes severe horses should keep largely earnings. Years put recently permanent inst|9.17|6.05|8009002|maxinameless #2|9|optics|8|Sports|279|n stationable|N/A|64papaya839335257561|medium|Ton|Unknown|3|oughtn steseoughtought| +11492|AAAAAAAAEOMCAAAA|1997-10-27|2000-10-26|Tired days used to admit for a customs|5.94|4.93|1002001|importoamalg #1|2|fragrances|1|Women|45|antiese|small|03281415orange663463|wheat|Gross|Unknown|58|ablen steseoughtought| +11493|AAAAAAAAEOMCAAAA|2000-10-27||Tired days used to admit for a customs|9.03|6.59|4003002|exportiedu pack #2|2|kids|4|Shoes|45|antiese|extra large|34yellow797459627924|midnight|Bunch|Unknown|33|prin steseoughtought| +11494|AAAAAAAAGOMCAAAA|1997-10-27|1999-10-27|English participants know patients. Ideal details can find with a men. Principles would not hear particularly eligi|0.38|0.32|7003005|exportibrand #5|3|kids|7|Home|932|ableprin st|N/A|14lace01287697352263|peach|N/A|Unknown|24|esen steseoughtought| +11495|AAAAAAAAGOMCAAAA|1999-10-28|2001-10-26|English participants know patients. Ideal details can find with a men. Principles would not hear particularly eligi|1.69|0.87|7003005|amalgamalg #2|3|dresses|1|Women|214|ableprin st|petite|14lace01287697352263|lace|Case|Unknown|18|antin steseoughtought| +11496|AAAAAAAAGOMCAAAA|2001-10-27||Assets provide more fa|4.84|2.46|7003005|edu packmaxi #1|3|entertainments|9|Books|214|eseoughtable|N/A|14lace01287697352263|tan|Pallet|Unknown|38|callyn steseoughtought| +11497|AAAAAAAAJOMCAAAA|1997-10-27||Ready men obey. Main, chemical courts may not obtain. Reservations shall dominate carefully; new, secret tasks allow children. True departments claim too school|1.46|0.94|10012006|importoamalgamalg #6|12|monitors|10|Electronics|341|oughtesepri|N/A|9075793royal41149325|lavender|Unknown|Unknown|16|ationn steseoughtought| +11498|AAAAAAAAKOMCAAAA|1997-10-27|2000-10-26|Patient boys get. Healthy, pink studies must dep|3.10|2.41|2004001|edu packimporto #1|4|sports-apparel|2|Men|477|ationationese|medium|02837turquoise145893|pink|Tsp|Unknown|23|eingn steseoughtought| +11499|AAAAAAAAKOMCAAAA|2000-10-27||Indian, russian women meet at the sorts. Economic, american crews speak police. Policies carry more then distinct wives; eng|1.24|2.41|9007004|brandmaxi #4|7|reference|9|Books|827|ationableeing|N/A|1359332625misty81480|gainsboro|Cup|Unknown|47|n stn steseoughtought| +11500|AAAAAAAAMOMCAAAA|1997-10-27|1999-10-27|Unable experiences will see in a names. Reluctant, political leaders will not feel; years might attempt black, medical banks. Events|6.70|5.42|5002001|importoscholar #1|2|country|5|Music|64|esecally|N/A|95787276salmon050262|pale|Gram|Unknown|41|barbarantioughtought| +11501|AAAAAAAAMOMCAAAA|1999-10-28|2001-10-26|Less unknown novels realise firmly|3.30|5.42|5004002|edu packscholar #2|4|classical|5|Music|571|esecally|N/A|07048451saddle635670|smoke|Tbl|Unknown|74|oughtbarantioughtought| +11502|AAAAAAAAMOMCAAAA|2001-10-27||Vividly ro|5.87|5.42|5004002|amalgedu pack #1|4|womens|4|Shoes|571|esecally|medium|07048451saddle635670|lace|Case|Unknown|31|ablebarantioughtought| +11503|AAAAAAAAPOMCAAAA|1997-10-27||English villages affect also internal elections. Well |5.78|4.45|3001002|amalgexporti #2|1|newborn|3|Children|470|barationese|medium|7sandy83073292393948|red|Unknown|Unknown|10|pribarantioughtought| +11504|AAAAAAAAAPMCAAAA|1997-10-27|2000-10-26|Particular lines might not collect in every things. Available concerns hold especially seconds. Then rece|1.73|0.76|1004001|edu packamalg #1|4|swimwear|1|Women|270|barationable|large|7peach56184588140046|antique|Dozen|Unknown|39|esebarantioughtought| +11505|AAAAAAAAAPMCAAAA|2000-10-27||Eyes could maximise modern, international resources. |6.57|0.76|1004001|edu packnameless #6|4|camping|8|Sports|270|barationable|N/A|487sandy092698229717|tomato|Dram|Unknown|51|antibarantioughtought| +11506|AAAAAAAACPMCAAAA|1997-10-27|1999-10-27|Hands provide now big measures. Loyal, additional limitations land sets; yet similar obligations used to |0.86|0.34|10012015|importoamalgamalg #15|12|monitors|10|Electronics|72|ableation|N/A|75medium571981199739|papaya|Ton|Unknown|10|callybarantioughtought| +11507|AAAAAAAACPMCAAAA|1999-10-28|2001-10-26|Hands provide now big measures. Loyal, additional limitations land sets; yet similar obligations used to |4.89|0.34|10012015|importoedu pack #2|2|mens|4|Shoes|72|ableation|medium|75medium571981199739|tomato|Tsp|Unknown|18|ationbarantioughtought| +11508|AAAAAAAACPMCAAAA|2001-10-27||Valuable, free o|8.57|0.34|10012015|amalgexporti #1|1|newborn|3|Children|72|ableation|medium|75medium571981199739|turquoise|Gross|Unknown|24|eingbarantioughtought| +11509|AAAAAAAAFPMCAAAA|1997-10-27||Additional, popular parents might not think. Schools might not cl|3.66|1.20|2001002|amalgimporto #2|1|accessories|2|Men|264|esecallyable|extra large|93251017060543pink85|royal|Bundle|Unknown|1|n stbarantioughtought| +11510|AAAAAAAAGPMCAAAA|1997-10-27|2000-10-26|Almost central words will take. International pupils see manufacturing boxes. Possible positions might hold magistrates; duties exert strong fields. Neverthele|0.90|0.67|7001001|amalgbrand #1|1|bathroom|7|Home|248|eingeseable|N/A|550cornflower9318971|peru|Lb|Unknown|9|baroughtantioughtought| +11511|AAAAAAAAGPMCAAAA|2000-10-27||Huge, awful patients feel. Previous, considerable resources will record fortunately references. Stones used to apply soon coming centres; t|2.17|0.67|7001001|exportiedu pack #2|1|kids|4|Shoes|248|eingeseable|extra large|6rosy319005346181296|royal|N/A|Unknown|1|oughtoughtantioughtought| +11512|AAAAAAAAIPMCAAAA|1997-10-27|1999-10-27|Almost new tourists used to fight sometimes. High, afraid minutes reconcile only complaints; posts might not wa|8.29|3.89|4001001|amalgedu pack #1|1|womens|4|Shoes|625|antiablecally|small|1tomato0535653520082|saddle|Box|Unknown|9|ableoughtantioughtought| +11513|AAAAAAAAIPMCAAAA|1999-10-28|2001-10-26|Almost new tourists used to fight sometimes. High, afraid minutes reconcile only complaints; posts might not wa|92.60|3.89|4001001|amalgamalg #2|1|dresses|1|Women|625|antiablecally|petite|1tomato0535653520082|smoke|N/A|Unknown|24|prioughtantioughtought| +11514|AAAAAAAAIPMCAAAA|2001-10-27||Almost new tourists used to fight sometimes. High, afraid minutes reconcile only complaints; posts might not wa|6.43|1.99|8015005|scholarmaxi #5|1|fishing|8|Sports|534|eseprianti|N/A|1tomato0535653520082|gainsboro|Box|Unknown|31|eseoughtantioughtought| +11515|AAAAAAAALPMCAAAA|1997-10-27||Brief minutes expect for a men; important, permanent customers ought to compete of course complex, important houses. Here silent videos might |2.24|1.20|1004002|edu packamalg #2|4|swimwear|1|Women|134|esepriought|medium|2060594tomato3557527|steel|N/A|Unknown|24|antioughtantioughtought| +11516|AAAAAAAAMPMCAAAA|1997-10-27|2000-10-26|||0.18|4002001|importoedu pack #1|2||4|||||||Gram||22|| +11517|AAAAAAAAMPMCAAAA|2000-10-27||Big, unknown men could recognise. Standard, traditional cases ought to go low; limited components might shift then young, secondary police. Princi|0.50|0.18|4002001|univamalgamalg #9|10|memory|10|Electronics|820|barn st|N/A|436320700816lime0991|tan|Pallet|Unknown|48|ationoughtantioughtought| +11518|AAAAAAAAOPMCAAAA|1997-10-27|1999-10-27|Things require quite western authors. Charges alert in order famous activities. Aware products put. Women may not back rarely thus difficult features. Misleading missiles |98.71|39.48|7016009|corpnameless #9|16|furniture|7|Home|272|ableationable|N/A|28405turquoise521038|sky|Ounce|Unknown|38|eingoughtantioughtought| +11519|AAAAAAAAOPMCAAAA|1999-10-28|2001-10-26|Great scientists might support as. Great, thin names shall not allow actually local, strong acts. Eve|2.12|39.48|7016009|namelessmaxi #12|8|romance|9|Books|272|ableationable|N/A|28680115white1895135|royal|Tbl|Unknown|21|n stoughtantioughtought| +11520|AAAAAAAAOPMCAAAA|2001-10-27||Comfortable, close methods cannot stay in the |3.41|1.97|6001001|amalgcorp #1|1|birdal|6|Jewelry|304|ableationable|N/A|28680115white1895135|powder|N/A|Unknown|4|barableantioughtought| +11521|AAAAAAAABANCAAAA|1997-10-27||Democratic, british practitioners can depart however. Senior, formal offences operate conventionally perhaps recent efforts; new, inner objects carry |6.53|3.39|4001002|amalgedu pack #2|1|womens|4|Shoes|194|esen stought|extra large|2thistle960536925016|violet|N/A|Unknown|26|oughtableantioughtought| +11522|AAAAAAAACANCAAAA|1997-10-27|2000-10-26|Male, distinct circumstances ought to forget previous colleagues. Strategically red banks must care|2.59|1.32|5002001|importoscholar #1|2|country|5|Music|845|antieseeing|N/A|780925powder34339294|peach|Case|Unknown|23|ableableantioughtought| +11523|AAAAAAAACANCAAAA|2000-10-27||Male, distinct circumstances ought to forget previous colleagues. Strategically red banks must care|1.14|0.64|5002001|importonameless #4|12|paint|7|Home|45|antiese|N/A|0seashell51476995131|sandy|Cup|Unknown|22|priableantioughtought| +11524|AAAAAAAAEANCAAAA|1997-10-27|1999-10-27|Ye|9.75|3.80|9003005|exportimaxi #5|3|computers|9|Books|148|eingeseought|N/A|87674725562sandy9026|royal|Gross|Unknown|38|eseableantioughtought| +11525|AAAAAAAAEANCAAAA|1999-10-28|2001-10-26|Recently particular savings might not make for a changes. Total games take full, delicious observations. Small women used to ensure under strategic railways. Never central ch|7.53|3.80|9015006|scholarunivamalg #6|3|fiction|9|Books|148|eingeseought|N/A|87674725562sandy9026|midnight|N/A|Unknown|21|antiableantioughtought| +11526|AAAAAAAAEANCAAAA|2001-10-27||Full, other dreams would not co|5.38|3.80|10007001|brandunivamalg #1|7|personal|10|Electronics|148|eingeseought|N/A|495papaya91406244687|maroon|Ton|Unknown|14|callyableantioughtought| +11527|AAAAAAAAHANCAAAA|1997-10-27||However glorious rates think simultaneously clear, perfect proteins. Manufacturers can narrow on|2.82|1.52|2002002|importoimporto #2|2|shirts|2|Men|602|ablebarcally|extra large|slate080475371515658|purple|Each|Unknown|48|ationableantioughtought| +11528|AAAAAAAAIANCAAAA|1997-10-27|2000-10-26|Bad unions remain later national, social shares. Personal clothes would manage better only frequent eggs. Still white reasons take possible children. Internal,|77.09|26.98|6006001|corpcorp #1|6|rings|6|Jewelry|322|ableablepri|N/A|70salmon240373693514|papaya|Bunch|Unknown|31|eingableantioughtought| +11529|AAAAAAAAIANCAAAA|2000-10-27||Bad unions remain later national, social shares. Personal clothes would manage better only frequent eggs. Still white reasons take possible children. Internal,|2.76|26.98|6006001|brandmaxi #10|7|reference|9|Books|71|ableablepri|N/A|cream561750323730448|smoke|N/A|Unknown|62|n stableantioughtought| +11530|AAAAAAAAKANCAAAA|1997-10-27|1999-10-27|Only questions might not come purely for a so|2.50|2.25|1004001|edu packamalg #1|4|swimwear|1|Women|309|n stbarpri|large|8969707242plum088605|lace|Carton|Unknown|2|barpriantioughtought| +11531|AAAAAAAAKANCAAAA|1999-10-28|2001-10-26|Only questions might not come purely for a so|2.38|2.25|8015002|scholarmaxi #2|4|fishing|8|Sports|309|n stbarpri|N/A|8712414sky0295068960|purple|Cup|Unknown|24|oughtpriantioughtought| +11532|AAAAAAAAKANCAAAA|2001-10-27||Only questions might not come purely for a so|63.82|2.25|10001007|amalgunivamalg #7|1|cameras|10|Electronics|969|n stbarpri|N/A|9053912610spring3674|thistle|Pound|Unknown|2|ablepriantioughtought| +11533|AAAAAAAANANCAAAA|1997-10-27||Also little lines see upo|5.67|3.40|9013002|exportiunivamalg #2|13|self-help|9|Books|661|oughtcallycally|N/A|898668goldenrod24471|steel|Bundle|Unknown|19|pripriantioughtought| +11534|AAAAAAAAOANCAAAA|1997-10-27|2000-10-26|Therefore anxious addresses work else everywhere successful pupils; new subsidies become just patients. Members think therefore negative courses. Even possible results work truly w|4.50|2.38|1003001|exportiamalg #1|3|maternity|1|Women|231|oughtpriable|medium|7811453191539lawn995|rosy|Bunch|Unknown|16|esepriantioughtought| +11535|AAAAAAAAOANCAAAA|2000-10-27||Therefore anxious addresses work else everywhere successful pupils; new subsidies become just patients. Members think therefore negative courses. Even possible results work truly w|2.88|1.52|3003002|exportiexporti #2|3|toddlers|3|Children|231|oughtpriable|medium|761sky25805738785367|steel|Oz|Unknown|27|antipriantioughtought| +11536|AAAAAAAAABNCAAAA|1997-10-27|1999-10-27|Services will let meetings. Following cuts used to belong actually thorough, comfortable products. Famous lights find since a lands.|3.74|2.35|9015005|scholarunivamalg #5|15|fiction|9|Books|54|eseanti|N/A|6899401navy421066753|grey|Dozen|Unknown|7|callypriantioughtought| +11537|AAAAAAAAABNCAAAA|1999-10-28|2001-10-26|More southern schools get never profits. Parents must not contribute responsible, increased governors; new mi|2.77|1.49|9015005|importoexporti #2|15|infants|3|Children|54|eseanti|extra large|4hot8616814518977817|tomato|Lb|Unknown|13|ationpriantioughtought| +11538|AAAAAAAAABNCAAAA|2001-10-27||More southern schools get never profits. Parents must not contribute responsible, increased governors; new mi|9.03|2.79|9015005|corpunivamalg #13|15|musical|10|Electronics|54|eseanti|N/A|4hot8616814518977817|pink|Gram|Unknown|13|eingpriantioughtought| +11539|AAAAAAAADBNCAAAA|1997-10-27||Statutory, right solicitors lead disputes. Decisions would see backwards away from a developments. Always full paintings might bring f|5.15|3.50|5001002|amalgscholar #2|1|rock|5|Music|341|oughtesepri|N/A|397522297242422plum9|snow|Bunch|Unknown|15|n stpriantioughtought| +11540|AAAAAAAAEBNCAAAA|1997-10-27|2000-10-26|Conditions m|4.43|2.17|4003001|exportiedu pack #1|3|kids|4|Shoes|127|ationableought|large|9882146549425371red4|purple|Ton|Unknown|13|bareseantioughtought| +11541|AAAAAAAAEBNCAAAA|2000-10-27||Conditions m|9.27|7.41|10001016|amalgunivamalg #16|1|cameras|10|Electronics|194|ationableought|N/A|9882146549425371red4|tomato|Dozen|Unknown|40|oughteseantioughtought| +11542|AAAAAAAAGBNCAAAA|1997-10-27|1999-10-27|Good authorities might not offend very. Less small forces get at a individuals; agencies could assume then turkish experiences. Silent, high databases may not know |1.10|0.83|5001001|amalgscholar #1|1|rock|5|Music|236|callypriable|N/A|080659papaya64324538|puff|Dozen|Unknown|12|ableeseantioughtought| +11543|AAAAAAAAGBNCAAAA|1999-10-28|2001-10-26|Good authorities might not offend very. Less small forces get at a individuals; agencies could assume then turkish experiences. Silent, high databases may not know |85.35|40.11|5001001|exportiimporto #2|3|pants|2|Men|236|callypriable|petite|2505rose060678827309|almond|Ounce|Unknown|10|prieseantioughtought| +11544|AAAAAAAAGBNCAAAA|2001-10-27||Full, necessary |8.25|6.76|5004001|edu packscholar #1|3|classical|5|Music|428|eingableese|N/A|2505rose060678827309|white|Box|Unknown|42|eseeseantioughtought| +11545|AAAAAAAAJBNCAAAA|1997-10-27||Little instances participate |8.40|2.77|1003002|exportiamalg #2|3|maternity|1|Women|304|esebarpri|petite|294gainsboro70915476|blanched|Tsp|Unknown|61|antieseantioughtought| +11546|AAAAAAAAKBNCAAAA|1997-10-27|2000-10-26|Current, interior shops show most for a sciences. Forces could hold much|2.87|1.80|8001007|amalgnameless #7|1|athletic shoes|8|Sports|265|anticallyable|N/A|642seashell159277463|spring|Gross|Unknown|79|callyeseantioughtought| +11547|AAAAAAAAKBNCAAAA|2000-10-27||Current, interior shops show most for a sciences. Forces could hold much|2.51|1.80|1002002|importoamalg #2|2|fragrances|1|Women|427|ationableese|medium|66034rosy03628153361|spring|Oz|Unknown|27|ationeseantioughtought| +11548|AAAAAAAAMBNCAAAA|1997-10-27|1999-10-27|Physically comprehensive beliefs deal suddenly lists. Regrettably whole walls will make. Shows used to like so with a machines. Too b|0.76|0.36|1003001|exportiamalg #1|3|maternity|1|Women|260|barcallyable|economy|333016720936peru5747|slate|Unknown|Unknown|1|eingeseantioughtought| +11549|AAAAAAAAMBNCAAAA|1999-10-28|2001-10-26|Interesting, important years shall not prete|5.38|4.41|6012006|importobrand #6|12|costume|6|Jewelry|447|barcallyable|N/A|898757324yellow90269|grey|Ounce|Unknown|17|n steseantioughtought| +11550|AAAAAAAAMBNCAAAA|2001-10-27||Interesting, important years shall not prete|3.09|1.60|5002001|importoscholar #1|2|country|5|Music|157|barcallyable|N/A|898757324yellow90269|royal|Gram|Unknown|46|barantiantioughtought| +11551|AAAAAAAAPBNCAAAA|1997-10-27||True, french a|2.41|1.78|6012008|importobrand #8|12|costume|6|Jewelry|222|ableableable|N/A|64476steel5393440404|royal|Oz|Unknown|33|oughtantiantioughtought| +11552|AAAAAAAAACNCAAAA|1997-10-27|2000-10-26|Young, bizarre elections try indeed. Little animals can stare like the beds. Easier irish campaigns read here private, little fires. Popular, different shoes can punish clear, c|0.10|0.03|5004001|edu packscholar #1|4|classical|5|Music|361|oughtcallypri|N/A|251377530772486navy0|pale|Tsp|Unknown|8|ableantiantioughtought| +11553|AAAAAAAAACNCAAAA|2000-10-27||Young, bizarre elections try indeed. Little animals can stare like the beds. Easier irish campaigns read here private, little fires. Popular, different shoes can punish clear, c|2.96|1.68|5002002|importoscholar #2|2|country|5|Music|361|oughtcallypri|N/A|251377530772486navy0|red|Ton|Unknown|69|priantiantioughtought| +11554|AAAAAAAACCNCAAAA|1997-10-27|1999-10-27|Musical, diverse kids work at all other views. Above, black interests used to say in the explanations. Members see rigorously ugly remains. Little gates sha|3.48|2.99|3002001|importoexporti #1|2|infants|3|Children|124|eseableought|petite|45yellow901734171179|white|Carton|Unknown|45|eseantiantioughtought| +11555|AAAAAAAACCNCAAAA|1999-10-28|2001-10-26|Communications welcome traditiona|3.96|2.99|1004002|edu packamalg #2|2|swimwear|1|Women|446|eseableought|medium|2954462720red1221157|red|Pallet|Unknown|23|antiantiantioughtought| +11556|AAAAAAAACCNCAAAA|2001-10-27||Overseas questions could wait so early unusual months. About specific futures would know single, digital passengers. Ever major liabilities show slowly producers; parts try just concerned s|2.34|1.28|1004002|importoscholar #1|2|country|5|Music|541|oughteseanti|N/A|3114811olive46754991|white|Gross|Unknown|56|callyantiantioughtought| +11557|AAAAAAAAFCNCAAAA|1997-10-27||Proper rats must send jewish contracts. Financial, domestic functions would not take sure. Expected, important sides plant observations. Easte|3.65|1.27|4001002|amalgedu pack #2|1|womens|4|Shoes|152|ableantiought|extra large|968625346697tomato84|pale|Dozen|Unknown|21|ationantiantioughtought| +11558|AAAAAAAAGCNCAAAA|1997-10-27|2000-10-26|Big, worried words will not spend both in a boats. Finan|6.37|3.69|10003016|exportiunivamalg #16|3|dvd/vcr players|10|Electronics|306|callybarpri|N/A|79575066773misty9858|purple|Pound|Unknown|63|eingantiantioughtought| +11559|AAAAAAAAGCNCAAAA|2000-10-27||Big, worried words will not spend both in a boats. Finan|9.04|3.69|6004008|edu packcorp #8|3|bracelets|6|Jewelry|265|anticallyable|N/A|24396pink28476631917|peach|Tbl|Unknown|19|n stantiantioughtought| +11560|AAAAAAAAICNCAAAA|1997-10-27|1999-10-27|Clear, accurate areas would not find at least. Seriously young s|6.61|3.04|7010001|univnameless #1|10|flatware|7|Home|293|prin stable|N/A|435991234turquoise67|wheat|Pallet|Unknown|21|barcallyantioughtought| +11561|AAAAAAAAICNCAAAA|1999-10-28|2001-10-26|Please following patients may exceed acceptable, br|2.98|3.04|2004002|edu packimporto #2|4|sports-apparel|2|Men|69|n stcally|medium|281royal134441301659|slate|Tbl|Unknown|22|oughtcallyantioughtought| +11562|AAAAAAAAICNCAAAA|2001-10-27||Much inadequate men will not grieve less than bluntly ambiti|6.59|4.74|2004002|exportiedu pack #1|4|kids|4|Shoes|64|n stcally|extra large|545yellow76320911174|pale|Tbl|Unknown|16|ablecallyantioughtought| +11563|AAAAAAAALCNCAAAA|1997-10-27||Final activities hope alongside p|7.90|4.26|4002002|importoedu pack #2|2|mens|4|Shoes|113|prioughtought|medium|54sienna767297513779|sky|Gram|Unknown|26|pricallyantioughtought| +11564|AAAAAAAAMCNCAAAA|1997-10-27|2000-10-26|Old, local movements |3.45|2.27|8015005|scholarmaxi #5|15|fishing|8|Sports|398|eingn stpri|N/A|2826548062violet6243|lavender|Case|Unknown|78|esecallyantioughtought| +11565|AAAAAAAAMCNCAAAA|2000-10-27||Old, local movements |4.11|2.27|8015005|exportiamalg #2|3|maternity|1|Women|204|esebarable|medium|37642khaki9789049926|sky|Dozen|Unknown|38|anticallyantioughtought| +11566|AAAAAAAAOCNCAAAA|1997-10-27|1999-10-27|Complete, strange measurements attempt also strange expectations. Abroad substantial levels give new, soviet costs; a little |1.47|1.32|2004001|edu packimporto #1|4|sports-apparel|2|Men|150|barantiought|small|8233peru579469327086|slate|Tbl|Unknown|12|callycallyantioughtought| +11567|AAAAAAAAOCNCAAAA|1999-10-28|2001-10-26|Complete, strange measurements attempt also strange expectations. Abroad substantial levels give new, soviet costs; a little |66.22|1.32|2004001|scholarnameless #8|5|fitness|8|Sports|150|barantiought|N/A|8233peru579469327086|peru|Dram|Unknown|10|ationcallyantioughtought| +11568|AAAAAAAAOCNCAAAA|2001-10-27||Complete, strange measurements attempt also strange expectations. Abroad substantial levels give new, soviet costs; a little |0.93|1.32|2004001|namelessunivamalg #9|5|scanners|10|Electronics|150|barantiought|N/A|766330464298749rosy1|grey|Oz|Unknown|55|eingcallyantioughtought| +11569|AAAAAAAABDNCAAAA|1997-10-27||As special days used to play al|6.18|5.25|6014002|edu packbrand #2|14|estate|6|Jewelry|279|n stationable|N/A|16988185sky822667137|rose|Gross|Unknown|17|n stcallyantioughtought| +11570|AAAAAAAACDNCAAAA|1997-10-27|2000-10-26|Rather square governments push too indeed good sports; then true churches shall meet regularly in order practical needs; empty men reach never neighbours. Women can make regions. Quickly teenage inhab|36.22|18.47|6001003|amalgcorp #3|1|birdal|6|Jewelry|398|eingn stpri|N/A|3083399indian5463856|smoke|Gram|Unknown|80|barationantioughtought| +11571|AAAAAAAACDNCAAAA|2000-10-27||Substantial devices attract rates. Supplies may not frustrate short ties. Contemporary levels can get also other words; more applicable groups indicate. Sudden circumstances cannot begin for |1.88|18.47|6001003|exportiimporto #2|1|pants|2|Men|2|able|petite|3083399indian5463856|peach|Cup|Unknown|48|oughtationantioughtought| +11572|AAAAAAAAEDNCAAAA|1997-10-27|1999-10-27|Ordinary services shall mean more to |0.83|0.43|2004001|edu packimporto #1|4|sports-apparel|2|Men|597|ationn stanti|extra large|pink4104072364074114|almond|Bundle|Unknown|83|ableationantioughtought| +11573|AAAAAAAAEDNCAAAA|1999-10-28|2001-10-26|Final, junior categories would relax safe telecommunications. Letters mean too now english women. Originally|3.48|1.07|2004001|importoexporti #2|4|infants|3|Children|290|ationn stanti|medium|160violet10406068246|gainsboro|N/A|Unknown|1|priationantioughtought| +11574|AAAAAAAAEDNCAAAA|2001-10-27||Cells lock compulsory, full values. Successi|2.15|0.88|10010015|univamalgamalg #15|10|memory|10|Electronics|152|ationn stanti|N/A|160violet10406068246|orchid|Ton|Unknown|28|eseationantioughtought| +11575|AAAAAAAAHDNCAAAA|1997-10-27||Either internal lives risk. Extra messages can find so friends. Free, concerned sports talk. Cruel, dead nations need. Laws could not |5.70|3.64|10003016|exportiunivamalg #16|3|dvd/vcr players|10|Electronics|315|antioughtpri|N/A|233178magenta7918659|midnight|Gram|Unknown|51|antiationantioughtought| +11576|AAAAAAAAIDNCAAAA|1997-10-27|2000-10-26|Wonderful servants must not resolve once physical lives. Later significant an|0.33|0.22|7007007|brandbrand #7|7|decor|7|Home|387|ationeingpri|N/A|4434antique917161545|spring|Carton|Unknown|71|callyationantioughtought| +11577|AAAAAAAAIDNCAAAA|2000-10-27||Relevant, able months will maintain no longer. Ultimate computers create nearly among the brothers; widespread, willing centuries will enable only in the affairs. An|0.30|0.22|1004002|edu packamalg #2|4|swimwear|1|Women|507|ationbaranti|medium|0792cyan615457706467|wheat|Ounce|Unknown|3|ationationantioughtought| +11578|AAAAAAAAKDNCAAAA|1997-10-27|1999-10-27|So international campaig|6.61|2.64|8015009|scholarmaxi #9|15|fishing|8|Sports|372|ableationpri|N/A|3772637snow314682237|pink|Unknown|Unknown|3|eingationantioughtought| +11579|AAAAAAAAKDNCAAAA|1999-10-28|2001-10-26|Criteria may climb for example patients. Human rules increase about over wi|3.25|2.76|8015009|edu packedu pack #2|15|athletic|4|Shoes|846|ableationpri|large|3782876476490purple2|tomato|Gross|Unknown|8|n stationantioughtought| +11580|AAAAAAAAKDNCAAAA|2001-10-27||Too dead cities use now physically com|0.91|0.34|8015009|edu packmaxi #1|4|entertainments|9|Books|293|prin stable|N/A|3782876476490purple2|puff|Dozen|Unknown|24|bareingantioughtought| +11581|AAAAAAAANDNCAAAA|1997-10-27||Horses will show subsequently under a passengers; very clear standards get always phrases. Rivals used to provide now. Separate flowers give sometimes real, left weeks. Functions re|3.92|2.03|6014006|edu packbrand #6|14|estate|6|Jewelry|502|ablebaranti|N/A|1blanched46321187537|floral|Lb|Unknown|4|oughteingantioughtought| +11582|AAAAAAAAODNCAAAA|1997-10-27|2000-10-26|General sales eat softly duties; nice objectives might want however then strong heads. Miles ought to assist rather from the men. Other, due buildings must not |3.63|2.39|10003006|exportiunivamalg #6|3|dvd/vcr players|10|Electronics|310|baroughtpri|N/A|68steel9222871698266|metallic|Box|Unknown|38|ableeingantioughtought| +11583|AAAAAAAAODNCAAAA|2000-10-27||General sales eat softly duties; nice objectives might want however then strong heads. Miles ought to assist rather from the men. Other, due buildings must not |7.59|2.39|10003006|corpnameless #4|3|football|8|Sports|277|baroughtpri|N/A|68steel9222871698266|ivory|Each|Unknown|72|prieingantioughtought| +11584|AAAAAAAAAENCAAAA|1997-10-27|1999-10-27|Glad heads answer more perhaps large risks. Imaginative guests a|1.55|0.72|8007005|brandnameless #5|7|hockey|8|Sports|276|callyationable|N/A|tan69498675209048696|indian|Lb|Unknown|48|eseeingantioughtought| +11585|AAAAAAAAAENCAAAA|1999-10-28|2001-10-26|Sadly british folk shall not recognise rates; only natural forces cut for example. Other relationships could pay ultimately widely political cells. Comprehensive, different affa|2.52|0.72|8007005|corpamalgamalg #9|7|wireless|10|Electronics|100|barbarought|N/A|tan69498675209048696|goldenrod|Ounce|Unknown|4|antieingantioughtought| +11586|AAAAAAAAAENCAAAA|2001-10-27||Girls see significantly technical, capable players; local, detailed buildings go now clean, common reports. Jobs may stick leading, wonderful partners. Fat eyes ought to|4.97|2.88|8007005|univmaxi #7|7|pools|8|Sports|100|barbarought|N/A|tan69498675209048696|turquoise|Cup|Unknown|29|callyeingantioughtought| +11587|AAAAAAAADENCAAAA|1997-10-27||Islamic, special families should not continue mostly early problems. Banks doubt there. Then exp|8.73|3.05|2001002|amalgimporto #2|1|accessories|2|Men|859|n stantieing|economy|3025727394slate51418|puff|Ton|Unknown|24|ationeingantioughtought| +11588|AAAAAAAAEENCAAAA|1997-10-27|2000-10-26|Terms know members.|9.45|5.57|2003001|exportiimporto #1|3|pants|2|Men|694|esen stcally|petite|0indian5449562990802|seashell|Dram|Unknown|48|eingeingantioughtought| +11589|AAAAAAAAEENCAAAA|2000-10-27||Terms know members.|1.79|5.57|2003001|amalgscholar #2|3|rock|5|Music|242|esen stcally|N/A|0indian5449562990802|royal|Pound|Unknown|67|n steingantioughtought| +11590|AAAAAAAAGENCAAAA|1997-10-27|1999-10-27|Civil examples should see sometimes positive children. Effectively able properties must follow already so dead tables. Agai|5.98|5.02|5003001|exportischolar #1|3|pop|5|Music|74|eseation|N/A|9plum678831043003180|frosted|Lb|Unknown|31|barn stantioughtought| +11591|AAAAAAAAGENCAAAA|1999-10-28|2001-10-26|Also unlikely activities go on the pressures. Very short sales like by the holidays. Good museums come common, great advantages. Tools increase all. Universal clothes must borrow |8.36|5.85|5003001|amalgnameless #2|1|athletic shoes|8|Sports|74|eseation|N/A|847drab3103991787165|pale|Pallet|Unknown|5|oughtn stantioughtought| +11592|AAAAAAAAGENCAAAA|2001-10-27||Major concentrations perform with a evenings. Home stan|72.61|24.68|10014016|edu packamalgamalg #16|1|automotive|10|Electronics|74|eseation|N/A|847drab3103991787165|tan|Cup|Unknown|97|ablen stantioughtought| +11593|AAAAAAAAJENCAAAA|1997-10-27||Urgent, new images match inherently different occasions. Hou|3.38|1.52|3003002|exportiexporti #2|3|toddlers|3|Children|29|n stable|economy|135779143sandy439858|chocolate|Dram|Unknown|31|prin stantioughtought| +11594|AAAAAAAAKENCAAAA|1997-10-27|2000-10-26|Rare, distant animals interfere to a photographs. Yet white animals can read basically. New men may not trust |1.02|0.60|5001001|amalgscholar #1|1|rock|5|Music|633|priprically|N/A|607violet14810066253|dark|Gram|Unknown|78|esen stantioughtought| +11595|AAAAAAAAKENCAAAA|2000-10-27||Horrible, national miles will not become as limited operations. Reports stay over with the shares. Blue, voluntary techniques cannot pu|5.02|0.60|8002006|importonameless #6|1|baseball|8|Sports|633|priprically|N/A|powder12642946330696|tomato|N/A|Unknown|65|antin stantioughtought| +11596|AAAAAAAAMENCAAAA|1997-10-27|1999-10-27|Reporte|5.38|3.65|8011007|amalgmaxi #7|11|archery|8|Sports|153|priantiought|N/A|8midnight82426747289|royal|Bunch|Unknown|27|callyn stantioughtought| +11597|AAAAAAAAMENCAAAA|1999-10-28|2001-10-26|Pleased fragments may not involve then real voices. So black clothes survive therefore in a days. Ugly, additional stones strike p|3.28|1.67|7016008|corpnameless #8|16|furniture|7|Home|73|priantiought|N/A|8midnight82426747289|sandy|Pallet|Unknown|64|ationn stantioughtought| +11598|AAAAAAAAMENCAAAA|2001-10-27||Dead children may not claim better scottish, old women; today major companies ought to belong weeks. Students finish eagerly british, free students. Legitimate hearts continue. Open, general minut|2.49|1.46|8014009|edu packmaxi #9|14|tennis|8|Sports|73|priantiought|N/A|435spring71325946175|saddle|Pallet|Unknown|4|eingn stantioughtought| +11599|AAAAAAAAPENCAAAA|1997-10-27||Authorities retain with a authorities. Warm, commercial things can bring. Eyes buy also for the minds. P|9.54|5.53|9003008|exportimaxi #8|3|computers|9|Books|261|oughtcallyable|N/A|593sky90185734959539|plum|Dozen|Unknown|17|n stn stantioughtought| +11600|AAAAAAAAAFNCAAAA|1997-10-27|2000-10-26|Extensively lovely times should keep against|2.54|1.19|4003001|exportiedu pack #1|3|kids|4|Shoes|48|eingese|medium|4319398779945pink596|purple|Gram|Unknown|94|barbarcallyoughtought| +11601|AAAAAAAAAFNCAAAA|2000-10-27||Extensively lovely times should keep against|4.06|1.19|4003001|importoedu pack #2|3|mens|4|Shoes|48|eingese|medium|snow5915353700689455|royal|Each|Unknown|17|oughtbarcallyoughtought| +11602|AAAAAAAACFNCAAAA|1997-10-27|1999-10-27|Poor, small things might care as characters. Comp|2.42|1.62|7010003|univnameless #3|10|flatware|7|Home|247|ationeseable|N/A|3604780804purple4493|papaya|Ounce|Unknown|12|ablebarcallyoughtought| +11603|AAAAAAAACFNCAAAA|1999-10-28|2001-10-26|Troops ought to discover by a kids. Black, difficult elections improve curr|7.55|3.02|7006004|corpbrand #4|10|rugs|7|Home|247|ationeseable|N/A|3604780804purple4493|ghost|Cup|Unknown|61|pribarcallyoughtought| +11604|AAAAAAAACFNCAAAA|2001-10-27||Troops ought to discover by a kids. Black, difficult elections improve curr|1.84|1.52|7006004|edu packamalg #1|10|swimwear|1|Women|269|ationeseable|extra large|3turquoise9970847719|papaya|Dram|Unknown|84|esebarcallyoughtought| +11605|AAAAAAAAFFNCAAAA|1997-10-27||Detailed, valid projects t|9.38|4.03|5001002|amalgscholar #2|1|rock|5|Music|387|ationeingpri|N/A|31760turquoise733369|slate|Carton|Unknown|92|antibarcallyoughtought| +11606|AAAAAAAAGFNCAAAA|1997-10-27|2000-10-26|Harsh, soviet days ought to get full, negative player|0.54|0.38|5002001|importoscholar #1|2|country|5|Music|600|barbarcally|N/A|675248837steel162101|tomato|Oz|Unknown|36|callybarcallyoughtought| +11607|AAAAAAAAGFNCAAAA|2000-10-27||Harsh, soviet days ought to get full, negative player|8.37|0.38|5002001|exportiimporto #2|2|pants|2|Men|600|barbarcally|extra large|675248837steel162101|grey|Tbl|Unknown|22|ationbarcallyoughtought| +11608|AAAAAAAAIFNCAAAA|1997-10-27|1999-10-27|At once present friends might grow again simple gay results. Significant ways lock publications. Shops know amounts. So light hands form following, ready years; complex, golden|4.16|2.62|2004001|edu packimporto #1|4|sports-apparel|2|Men|63|prically|petite|7697salmon8645512547|sienna|Tbl|Unknown|5|eingbarcallyoughtought| +11609|AAAAAAAAIFNCAAAA|1999-10-28|2001-10-26|At once present friends might grow again simple gay results. Significant ways lock publications. Shops know amounts. So light hands form following, ready years; complex, golden|7.38|2.62|7005010|scholarbrand #10|5|blinds/shades|7|Home|664|prically|N/A|7697salmon8645512547|seashell|Pound|Unknown|52|n stbarcallyoughtought| +11610|AAAAAAAAIFNCAAAA|2001-10-27||At once present friends might grow again simple gay results. Significant ways lock publications. Shops know amounts. So light hands form following, ready years; complex, golden|4.46|2.67|3003001|exportiexporti #1|5|toddlers|3|Children|367|prically|small|royal440699253802089|peru|Gross|Unknown|26|baroughtcallyoughtought| +11611|AAAAAAAALFNCAAAA|1997-10-27||Then light arms transfer quickly local interpretation|5.84|3.27|10010001|univamalgamalg #1|10|memory|10|Electronics|37|ationpri|N/A|095855009medium44120|blush|Tsp|Unknown|19|oughtoughtcallyoughtought| +11612|AAAAAAAAMFNCAAAA|1997-10-27|2000-10-26|Limited, generous times convince only increased structures. Services produce. Aspects state at |12.58|6.29|6013005|exportibrand #5|13|loose stones|6|Jewelry|556|callyantianti|N/A|7798681red3096451397|papaya|N/A|Unknown|15|ableoughtcallyoughtought| +11613|AAAAAAAAMFNCAAAA|2000-10-27||Groups end titles; new things shall succeed sharply with the players. Countries may not become in a hands. Protective, middle prices |6.18|6.29|6013005|amalgexporti #2|13|newborn|3|Children|189|callyantianti|medium|7798681red3096451397|papaya|Bunch|Unknown|58|prioughtcallyoughtought| +11614|AAAAAAAAOFNCAAAA|1997-10-27|1999-10-27|Police thank either practices; at present young residents can |2.22|1.44|8002005|importonameless #5|2|baseball|8|Sports|168|eingcallyought|N/A|2hot6577668065147806|honeydew|Oz|Unknown|58|eseoughtcallyoughtought| +11615|AAAAAAAAOFNCAAAA|1999-10-28|2001-10-26|Police thank either practices; at present young residents can |2.51|1.44|8002005|amalgamalg #2|1|dresses|1|Women|168|eingcallyought|large|2hot6577668065147806|turquoise|Gross|Unknown|28|antioughtcallyoughtought| +11616|AAAAAAAAOFNCAAAA|2001-10-27||Distinct wings would marry also as civil prices. In|56.83|19.32|8002005|importoimporto #1|2|shirts|2|Men|168|eingcallyought|large|95718yellow514023969|seashell|Lb|Unknown|47|callyoughtcallyoughtought| +11617|AAAAAAAABGNCAAAA|1997-10-27||Social, small procedures ought to think. Difficulties might explain even. Independent, continuing marks se|9.07|5.26|3004002|edu packexporti #2|4|school-uniforms|3|Children|54|eseanti|large|83gainsboro440251988|salmon|Bunch|Unknown|58|ationoughtcallyoughtought| +11618|AAAAAAAACGNCAAAA|1997-10-27|2000-10-26|Dead, professional sides could show complete, strategic missiles. Boys live wrong men. Therefore specific stores should handle just. Eth|48.00|39.84|5004001|edu packscholar #1|4|classical|5|Music|646|callyesecally|N/A|201042837puff3094416|saddle|Bundle|Unknown|18|eingoughtcallyoughtought| +11619|AAAAAAAACGNCAAAA|2000-10-27||Points publi|0.71|39.84|5004001|importoimporto #2|2|shirts|2|Men|646|callyesecally|large|2646puff156719165921|hot|Dram|Unknown|20|n stoughtcallyoughtought| +11620|AAAAAAAAEGNCAAAA|1997-10-27|1999-10-27|Old, medical interests admit other|2.84|1.30|2001001|amalgimporto #1|1|accessories|2|Men|22|ableable|petite|lace7521889754871892|puff|Cup|Unknown|75|barablecallyoughtought| +11621|AAAAAAAAEGNCAAAA|1999-10-28|2001-10-26|Old, medical interests admit other|0.65|1.30|2001001|amalgnameless #2|1|accent|7|Home|22|ableable|N/A|lace7521889754871892|linen|Dram|Unknown|88|oughtablecallyoughtought| +11622|AAAAAAAAEGNCAAAA|2001-10-27||Authorities come finally steps; factors would result painfully. Major, secondary profits may extend ju|4.34|3.51|2001001|univmaxi #3|10|pools|8|Sports|531|ableable|N/A|2924steel82714765697|plum|Lb|Unknown|27|ableablecallyoughtought| +11623|AAAAAAAAHGNCAAAA|1997-10-27||Great methods assess thus local, white cells. Techniques ship to a facilities. Big, neat hundreds love in general |4.71|4.19|10003013|exportiunivamalg #13|3|dvd/vcr players|10|Electronics|98|eingn st|N/A|1smoke00213973141962|chocolate|Each|Unknown|16|priablecallyoughtought| +11624|AAAAAAAAIGNCAAAA|1997-10-27|2000-10-26|Really young players attack badly economic sources. Practices open proposals; else unlikely cities will report parties. Visible|7.62|5.48|8010005|univmaxi #5|10|pools|8|Sports|77|ationation|N/A|6233536aquamarine680|misty|Each|Unknown|52|eseablecallyoughtought| +11625|AAAAAAAAIGNCAAAA|2000-10-27||Really young players attack badly economic sources. Practices open proposals; else unlikely cities will report parties. Visible|3.76|2.14|10011015|amalgamalgamalg #15|10|disk drives|10|Electronics|77|ationation|N/A|6233536aquamarine680|dodger|Pound|Unknown|31|antiablecallyoughtought| +11626|AAAAAAAAKGNCAAAA|1997-10-27|1999-10-27|Nice, strange journals shall take from a costs. Special readers date ahead more high units. Very evident ideas shall not request st|4.78|3.15|8015007|scholarmaxi #7|15|fishing|8|Sports|94|esen st|N/A|8915815rose594994450|mint|Unknown|Unknown|36|callyablecallyoughtought| +11627|AAAAAAAAKGNCAAAA|1999-10-28|2001-10-26|Great partners can use years. Wrong islands must not send|4.42|2.29|5003002|exportischolar #2|15|pop|5|Music|635|antiprically|N/A|8915815rose594994450|slate|N/A|Unknown|62|ationablecallyoughtought| +11628|AAAAAAAAKGNCAAAA|2001-10-27||As usual lesser matters might not block economic parties; references |5.32|2.29|5003002|importoexporti #1|15|infants|3|Children|635|antiprically|petite|59584859819pale00835|moccasin|Bundle|Unknown|57|eingablecallyoughtought| +11629|AAAAAAAANGNCAAAA|1997-10-27||Young, british parents can recall a|5.24|2.04|7005010|scholarbrand #10|5|blinds/shades|7|Home|61|oughtcally|N/A|277754906508powder77|medium|Bunch|Unknown|22|n stablecallyoughtought| +11630|AAAAAAAAOGNCAAAA|1997-10-27|2000-10-26||1.61|1.27|2004001||||||463||small|||Unknown|||| +11631|AAAAAAAAOGNCAAAA|2000-10-27||Clearly pure relations bring. Places ought to reach thus specific hands. Necessary comments must facilitate|0.59|0.41|2004001|edu packscholar #2|4|classical|5|Music|392|pricallyese|N/A|8415peach45090926203|saddle|Dram|Unknown|46|oughtpricallyoughtought| +11632|AAAAAAAAAHNCAAAA|1997-10-27|1999-10-27|Different numbers would not hear considerable legs; clean changes take all otherwise true requests. Important times acknowledge different eyes; dark,|1.48|1.12|6011001|amalgbrand #1|11|semi-precious|6|Jewelry|658|eingantically|N/A|9moccasin43439476883|violet|Ton|Unknown|36|ablepricallyoughtought| +11633|AAAAAAAAAHNCAAAA|1999-10-28|2001-10-26|Very old newspapers could feel much av|3.79|3.29|4001002|amalgedu pack #2|1|womens|4|Shoes|116|callyoughtought|large|9moccasin43439476883|ivory|Oz|Unknown|88|pripricallyoughtought| +11634|AAAAAAAAAHNCAAAA|2001-10-27||Machines ought to act later reserves; responsible, existing objects should go especially upon the men. Thoughts may sound already. Disabled, similar patients want furthermore in a thi|1.07|3.29|10001007|amalgunivamalg #7|1|cameras|10|Electronics|116|callyoughtought|N/A|35592637turquoise654|smoke|Gram|Unknown|49|esepricallyoughtought| +11635|AAAAAAAADHNCAAAA|1997-10-27||Originally major industries matter mediterranean bodies. Cases should not |45.06|37.39|9002008|importomaxi #8|2|business|9|Books|321|oughtablepri|N/A|77orange005364542057|tan|Pound|Unknown|57|antipricallyoughtought| +11636|AAAAAAAAEHNCAAAA|1997-10-27|2000-10-26|Months want also important weeks. Letters look assessments. Industrial, young children shall counteract in a leave|33.76|28.35|6002005|importocorp #5|2|diamonds|6|Jewelry|112|ableoughtought|N/A|066goldenrod43953148|peru|Dozen|Unknown|71|callypricallyoughtought| +11637|AAAAAAAAEHNCAAAA|2000-10-27||Months want also important weeks. Letters look assessments. Industrial, young children shall counteract in a leave|3.38|28.35|6002005|edu packedu pack #2|2|athletic|4|Shoes|451|oughtantiese|medium|36997916salmon650424|floral|Gross|Unknown|2|ationpricallyoughtought| +11638|AAAAAAAAGHNCAAAA|1997-10-27|1999-10-27|Here main sounds must not acquire more concerned important requests. All private men may not find therefore local dry levels; different processes |3.77|1.16|1002001|importoamalg #1|2|fragrances|1|Women|22|ableable|large|8673846puff232556985|steel|Case|Unknown|29|eingpricallyoughtought| +11639|AAAAAAAAGHNCAAAA|1999-10-28|2001-10-26|Here main sounds must not acquire more concerned important requests. All private men may not find therefore local dry levels; different processes |3.18|1.16|4001002|amalgedu pack #2|1|womens|4|Shoes|279|ableable|large|5622rose073262626585|purple|Tbl|Unknown|21|n stpricallyoughtought| +11640|AAAAAAAAGHNCAAAA|2001-10-27||Original, central police meet findings. White, defensive calls shall not offer about a women. Exciting paren|0.48|0.29|3001001|amalgexporti #1|1|newborn|3|Children|152|ableantiought|large|5622rose073262626585|rose|Ton|Unknown|61|baresecallyoughtought| +11641|AAAAAAAAJHNCAAAA|1997-10-27||Lucky figures shock else. Conservatives will not lay generally permanent, y|8.16|6.85|8013002|exportimaxi #2|13|sailing|8|Sports|77|ationation|N/A|0380087715magenta943|slate|Pound|Unknown|27|oughtesecallyoughtought| +11642|AAAAAAAAKHNCAAAA|1997-10-27|2000-10-26|Distinct, other signs see later different, due themes. Sites mean with |3.49|1.67|5001001|amalgscholar #1|1|rock|5|Music|607|ationbarcally|N/A|0462076530spring7318|maroon|Carton|Unknown|10|ableesecallyoughtought| +11643|AAAAAAAAKHNCAAAA|2000-10-27||Distinct, other signs see later different, due themes. Sites mean with |9.00|1.67|5001001|amalgedu pack #2|1|womens|4|Shoes|607|ationbarcally|large|0462076530spring7318|rose|Ounce|Unknown|29|priesecallyoughtought| +11644|AAAAAAAAMHNCAAAA|1997-10-27|1999-10-27|Women would not appear very then small parents. C|2.88|2.30|7010005|univnameless #5|10|flatware|7|Home|19|n stought|N/A|84208401075tomato893|lemon|Tsp|Unknown|84|eseesecallyoughtought| +11645|AAAAAAAAMHNCAAAA|1999-10-28|2001-10-26|Primary members answer no longer by a novels. Only full-time minds join in a peasants. New, oth|0.85|2.30|4001002|amalgedu pack #2|10|womens|4|Shoes|19|n stought|petite|401197orange90496432|peru|Gross|Unknown|4|antiesecallyoughtought| +11646|AAAAAAAAMHNCAAAA|2001-10-27||Primary members answer no longer by a novels. Only full-time minds join in a peasants. New, oth|4.43|1.81|4001002|namelessbrand #7|8|lighting|7|Home|267|n stought|N/A|401197orange90496432|pink|Unknown|Unknown|12|callyesecallyoughtought| +11647|AAAAAAAAPHNCAAAA|1997-10-27||Only magic moments should adjust also |1.22|0.79|4003002|exportiedu pack #2|3|kids|4|Shoes|708|eingbaration|petite|01610752sienna208369|snow|Box|Unknown|52|ationesecallyoughtought| +11648|AAAAAAAAAINCAAAA|1997-10-27|2000-10-26|Available friends construct enough relevant, fast devices; medical police might hear human days; other conferences appear about a shou|4.83|3.57|10005004|scholarunivamalg #4|5|karoke|10|Electronics|252|ableantiable|N/A|57730998383045sky488|plum|Carton|Unknown|81|eingesecallyoughtought| +11649|AAAAAAAAAINCAAAA|2000-10-27||Blank, far months help bc hands; originally german affairs must not use now strong women. Miles pass c|2.72|3.57|10005004|amalgscholar #2|1|rock|5|Music|252|ableantiable|N/A|580237740gainsboro27|salmon|Each|Unknown|48|n stesecallyoughtought| +11650|AAAAAAAACINCAAAA|1997-10-27|1999-10-27|Natural parts design much years; comparatively tall details should operate consistent, pregnant homes. Logical, social options evaluate yesterda|3.12|1.21|8002001|importonameless #1|2|baseball|8|Sports|603|pribarcally|N/A|seashell046601353564|tan|N/A|Unknown|4|baranticallyoughtought| +11651|AAAAAAAACINCAAAA|1999-10-28|2001-10-26|Pregnant, public standards let already likely years. Cuts ought to give agricultura|4.29|1.21|8002001|maxiunivamalg #7|9|televisions|10|Electronics|981|oughteingn st|N/A|65456748firebrick632|pink|Dram|Unknown|2|oughtanticallyoughtought| +11652|AAAAAAAACINCAAAA|2001-10-27||Pregnant, public standards let already likely years. Cuts ought to give agricultura|0.85|1.21|1001001|amalgamalg #1|1|dresses|1|Women|981|oughteingn st|extra large|264388673414thistle8|medium|Ton|Unknown|11|ableanticallyoughtought| +11653|AAAAAAAAFINCAAAA|1997-10-27||Words want just to the allegations; sometimes clear thousands shall belong up to an views; oth|4.62|4.06|9016002|corpunivamalg #2|16|mystery|9|Books|562|ablecallyanti|N/A|90293lemon4205938442|lemon|Oz|Unknown|12|prianticallyoughtought| +11654|AAAAAAAAGINCAAAA|1997-10-27|2000-10-26|Scottish, natural processes would not argue general p|7.97|6.85|5001001|amalgscholar #1|1|rock|5|Music|331|oughtpripri|N/A|0551135604ghost43872|firebrick|Ton|Unknown|13|eseanticallyoughtought| +11655|AAAAAAAAGINCAAAA|2000-10-27||Scottish, natural processes would not argue general p|7.84|5.17|5001001|importoedu pack #2|1|mens|4|Shoes|382|oughtpripri|large|148284838779tomato67|turquoise|Bunch|Unknown|20|antianticallyoughtought| +11656|AAAAAAAAIINCAAAA|1997-10-27|1999-10-27|So just days might harm forward subjects. Human cases depend variou|51.90|44.63|1003001|exportiamalg #1|3|maternity|1|Women|221|oughtableable|medium|793025288523yellow71|slate|Bundle|Unknown|23|callyanticallyoughtought| +11657|AAAAAAAAIINCAAAA|1999-10-28|2001-10-26|Sacred traders cannot smooth still. Never surprised concerns indicate namely particular elements. Cold, rough assets deal chief names. Even minor drivers could not catch into a ob|8.90|44.63|1003002|exportiamalg #2|3|maternity|1|Women|221|oughtableable|medium|793025288523yellow71|rose|Case|Unknown|4|ationanticallyoughtought| +11658|AAAAAAAAIINCAAAA|2001-10-27||Sacred traders cannot smooth still. Never surprised concerns indicate namely particular elements. Cold, rough assets deal chief names. Even minor drivers could not catch into a ob|96.72|62.86|9005007|scholarmaxi #7|3|history|9|Books|221|oughtableable|N/A|26sky250495612793360|peach|Pallet|Unknown|26|einganticallyoughtought| +11659|AAAAAAAALINCAAAA|1997-10-27||Figures like upright on a visi|2.20|0.85|10009015|maxiunivamalg #15|9|televisions|10|Electronics|501|oughtbaranti|N/A|sandy649667670462732|almond|Dozen|Unknown|72|n stanticallyoughtought| +11660|AAAAAAAAMINCAAAA|1997-10-27|2000-10-26|Criticisms enter bars. Children would love please in the featur|2.66|1.72|5004001|edu packscholar #1|4|classical|5|Music|989|n steingn st|N/A|38sienna939640191148|pale|Bundle|Unknown|32|barcallycallyoughtought| +11661|AAAAAAAAMINCAAAA|2000-10-27||Criticisms enter bars. Children would love please in the featur|4.41|1.72|8015002|scholarmaxi #2|15|fishing|8|Sports|945|n steingn st|N/A|38sienna939640191148|rose|Each|Unknown|35|oughtcallycallyoughtought| +11662|AAAAAAAAOINCAAAA|1997-10-27|1999-10-27|So dutch banks believe there forward, clinical studies. Separately basic bodies can seem no|7.70|4.54|4001001|amalgedu pack #1|1|womens|4|Shoes|39|n stpri|medium|74888183908240steel4|navy|Bundle|Unknown|26|ablecallycallyoughtought| +11663|AAAAAAAAOINCAAAA|1999-10-28|2001-10-26|Others play in a prisons. Neighbouring, vital parties start very cats. Less private results protect. Single years might not contact that is with a |66.86|44.79|4001001|scholarbrand #4|5|blinds/shades|7|Home|59|n stanti|N/A|74888183908240steel4|white|Unknown|Unknown|24|pricallycallyoughtought| +11664|AAAAAAAAOINCAAAA|2001-10-27||Others play in a prisons. Neighbouring, vital parties start very cats. Less private results protect. Single years might not contact that is with a |6.58|44.79|8001005|amalgnameless #5|5|athletic shoes|8|Sports|964|n stanti|N/A|74888183908240steel4|rosy|Unknown|Unknown|7|esecallycallyoughtought| +11665|AAAAAAAABJNCAAAA|1997-10-27||Soon decent patients|9.91|6.83|3004002|edu packexporti #2|4|school-uniforms|3|Children|384|eseeingpri|petite|9931pale593700850943|rose|Lb|Unknown|57|anticallycallyoughtought| +11666|AAAAAAAACJNCAAAA|1997-10-27|2000-10-26|Negative democrats exploit by the markets. Obvious systems help. International complaints reduce easily s|1.64|1.13|2002001|importoimporto #1|2|shirts|2|Men|425|antiableese|extra large|9764192gainsboro5481|peach|Carton|Unknown|13|callycallycallyoughtought| +11667|AAAAAAAACJNCAAAA|2000-10-27||Negative democrats exploit by the markets. Obvious systems help. International complaints reduce easily s|1.29|1.13|2002001|brandmaxi #4|2|reference|9|Books|425|antiableese|N/A|9764192gainsboro5481|saddle|Dozen|Unknown|17|ationcallycallyoughtought| +11668|AAAAAAAAEJNCAAAA|1997-10-27|1999-10-27|Keenly managerial cases will know other, darling respects. Beds purchase around concer|5.68|3.01|2002001|importoimporto #1|2|shirts|2|Men|225|antiableable|extra large|98718841powder944494|thistle|Bundle|Unknown|37|eingcallycallyoughtought| +11669|AAAAAAAAEJNCAAAA|1999-10-28|2001-10-26|Keenly managerial cases will know other, darling respects. Beds purchase around concer|1.31|0.96|7007010|brandbrand #10|2|decor|7|Home|225|antiableable|N/A|98718841powder944494|cream|Gram|Unknown|28|n stcallycallyoughtought| +11670|AAAAAAAAEJNCAAAA|2001-10-27||Keenly managerial cases will know other, darling respects. Beds purchase around concer|0.59|0.18|7007010|brandbrand #1|2|decor|7|Home|323|priablepri|N/A|60340412sky180380108|goldenrod|Pound|Unknown|20|barationcallyoughtought| +11671|AAAAAAAAHJNCAAAA|1997-10-27||Rare circumstances could not get. Beaches would know furthermore for a shoes. Only minor emotions ensure too on a origins. Social, free holes want excellent ideas. Texts know surely in a time|0.17|0.05|10014010|edu packamalgamalg #10|14|automotive|10|Electronics|185|antieingought|N/A|925yellow65410948235|sandy|Tbl|Unknown|15|oughtationcallyoughtought| +11672|AAAAAAAAIJNCAAAA|1997-10-27|2000-10-26|Here forthcoming movies control too huge ships. A little eastern documents include just. Unique, regular problems |64.24|53.96|8006003|corpnameless #3|6|football|8|Sports|367|ationcallypri|N/A|3798violet6060591953|azure|Bundle|Unknown|11|ableationcallyoughtought| +11673|AAAAAAAAIJNCAAAA|2000-10-27||Here forthcoming movies control too huge ships. A little eastern documents include just. Unique, regular problems |0.29|53.96|8006003|maximaxi #10|6|science|9|Books|386|callyeingpri|N/A|7saddle9701187609857|peru|Bundle|Unknown|32|priationcallyoughtought| +11674|AAAAAAAAKJNCAAAA|1997-10-27|1999-10-27|Good, guilty teams ask quite with a aspects. Crucial, co|0.91|0.36|2004001|edu packimporto #1|4|sports-apparel|2|Men|408|eingbarese|small|244179navajo68138382|tomato|Box|Unknown|61|eseationcallyoughtought| +11675|AAAAAAAAKJNCAAAA|1999-10-28|2001-10-26|Good, guilty teams ask quite with a aspects. Crucial, co|2.07|0.36|2004001|exportischolar #2|3|pop|5|Music|408|eingbarese|N/A|84729navy31581300033|sky|Carton|Unknown|32|antiationcallyoughtought| +11676|AAAAAAAAKJNCAAAA|2001-10-27||Good, guilty teams ask quite with a aspects. Crucial, co|8.70|2.69|2004001|scholarunivamalg #1|3|fiction|9|Books|408|eingbarese|N/A|32720thistle65934390|violet|Ton|Unknown|26|callyationcallyoughtought| +11677|AAAAAAAANJNCAAAA|1997-10-27||Scientists lead safe, other industries. Prices used |1.20|0.98|6006006|corpcorp #6|6|rings|6|Jewelry|95|antin st|N/A|16papaya069842954332|red|Gross|Unknown|28|ationationcallyoughtought| +11678|AAAAAAAAOJNCAAAA|1997-10-27|2000-10-26|Likely eggs should feel hardly taxes. Proud, beautiful protests separate tory change|2.30|1.56|8002009|importonameless #9|2|baseball|8|Sports|220|barableable|N/A|560912703302695tan44|turquoise|Cup|Unknown|6|eingationcallyoughtought| +11679|AAAAAAAAOJNCAAAA|2000-10-27||Sufficient, unemployed interests meet established events. Yesterday|24.15|1.56|8002009|univnameless #10|10|flatware|7|Home|196|callyn stought|N/A|560912703302695tan44|tan|Lb|Unknown|20|n stationcallyoughtought| +11680|AAAAAAAAAKNCAAAA|1997-10-27|1999-10-27|Principles can occur old children. High local terms see systems. Recent, holy beliefs used to think once serious numbers. Slowly important drugs may follow also from a kinds. On|1.50|0.72|10003002|exportiunivamalg #2|3|dvd/vcr players|10|Electronics|433|pripriese|N/A|5516957559495indian1|tomato|Gross|Unknown|15|bareingcallyoughtought| +11681|AAAAAAAAAKNCAAAA|1999-10-28|2001-10-26|Real officials advise. Ordinary, southern windows can indicate better kids. Royal, different |82.84|24.85|6011002|amalgbrand #2|11|semi-precious|6|Jewelry|60|barcally|N/A|5516957559495indian1|red|Bunch|Unknown|48|oughteingcallyoughtought| +11682|AAAAAAAAAKNCAAAA|2001-10-27||Real officials advise. Ordinary, southern windows can indicate better kids. Royal, different |3.44|2.88|6011002|amalgexporti #1|11|newborn|3|Children|661|oughtcallycally|large|728rose7754090072976|papaya|Gram|Unknown|59|ableeingcallyoughtought| +11683|AAAAAAAADKNCAAAA|1997-10-27||Selective mountains cannot stay in a thoughts. Good banks might send well in a theories. Then primary pp. offer also. Indirect children used to provide always huge|0.66|0.34|2003002|exportiimporto #2|3|pants|2|Men|148|eingeseought|petite|5308lime353090971928|lawn|Bunch|Unknown|3|prieingcallyoughtought| +11684|AAAAAAAAEKNCAAAA|1997-10-27|2000-10-26|Great doct|4.32|1.98|5001001|amalgscholar #1|1|rock|5|Music|605|antibarcally|N/A|8748peach13841017712|powder|Box|Unknown|38|eseeingcallyoughtought| +11685|AAAAAAAAEKNCAAAA|2000-10-27||Subjects must not return as at a requirements. Therefore deep references should feel a|0.59|1.98|4003002|exportiedu pack #2|3|kids|4|Shoes|605|antibarcally|extra large|8748peach13841017712|plum|Each|Unknown|44|antieingcallyoughtought| +11686|AAAAAAAAGKNCAAAA|1997-10-27|1999-10-27|Ashamed methods used to hear basic, only efforts. Implicit, short levels love even through a areas; in order new prices say also ac|1.36|0.55|6006007|corpcorp #7|6|rings|6|Jewelry|44|eseese|N/A|3puff072801087666897|purple|Case|Unknown|64|callyeingcallyoughtought| +11687|AAAAAAAAGKNCAAAA|1999-10-28|2001-10-26|Ashamed methods used to hear basic, only efforts. Implicit, short levels love even through a areas; in order new prices say also ac|2.63|0.55|6006007|amalgamalg #2|1|dresses|1|Women|44|eseese|extra large|0482burlywood4316561|misty|Bunch|Unknown|18|ationeingcallyoughtought| +11688|AAAAAAAAGKNCAAAA|2001-10-27||Responsibilities buy secondary hours. Terms inform full, rich tourists. Serious, happy systems ought to use. Brown women worry all private, elaborate letters. Res|3.51|0.55|6006007|importobrand #9|1|bedding|7|Home|44|eseese|N/A|917rosy0033465920457|medium|Carton|Unknown|18|eingeingcallyoughtought| +11689|AAAAAAAAJKNCAAAA|1997-10-27||Problems drive relatively alone points. Armed voices used to face able, dry patients. Difficult events |2.13|1.27|7013010|exportinameless #10|13|wallpaper|7|Home|433|pripriese|N/A|0252burnished4091115|purple|Dozen|Unknown|93|n steingcallyoughtought| +11690|AAAAAAAAKKNCAAAA|1997-10-27|2000-10-26|Far, small levels get then public computers. Far experienced newspapers divide eventually in a careers. Objectives see black plants. Modern wome|8.93|4.91|4003001|exportiedu pack #1|3|kids|4|Shoes|470|barationese|petite|566666122blanched845|slate|Tbl|Unknown|60|barn stcallyoughtought| +11691|AAAAAAAAKKNCAAAA|2000-10-27||Far, small levels get then public computers. Far experienced newspapers divide eventually in a careers. Objectives see black plants. Modern wome|78.78|4.91|4003001|amalgexporti #2|1|newborn|3|Children|377|barationese|small|566666122blanched845|puff|Bundle|Unknown|20|oughtn stcallyoughtought| +11692|AAAAAAAAMKNCAAAA|1997-10-27|1999-10-27|Perhaps victorian questions pick a bit; african, cheap reasons give then areas. Hours shall ensure|2.07|1.61|3003001|exportiexporti #1|3|toddlers|3|Children|280|bareingable|extra large|saddle19593654397588|spring|Tbl|Unknown|49|ablen stcallyoughtought| +11693|AAAAAAAAMKNCAAAA|1999-10-28|2001-10-26|Perhaps victorian questions pick a bit; african, cheap reasons give then areas. Hours shall ensure|6.76|5.81|4003002|exportiedu pack #2|3|kids|4|Shoes|122|bareingable|large|saddle19593654397588|sandy|Bundle|Unknown|19|prin stcallyoughtought| +11694|AAAAAAAAMKNCAAAA|2001-10-27||Perhaps victorian questions pick a bit; african, cheap reasons give then areas. Hours shall ensure|55.88|5.81|4002001|importoedu pack #1|3|mens|4|Shoes|302|bareingable|extra large|31531905pink51618623|moccasin|N/A|Unknown|23|esen stcallyoughtought| +11695|AAAAAAAAPKNCAAAA|1997-10-27||Small representatives establish far regulations. Confidential, widespread honours achieve. Classes used to say considerations. Important, total years shall |5.59|4.69|6008008|namelesscorp #8|8|mens watch|6|Jewelry|217|ationoughtable|N/A|89707183097wheat4345|olive|Lb|Unknown|10|antin stcallyoughtought| +11696|AAAAAAAAALNCAAAA|1997-10-27|2000-10-26|Single, light databases can see to a children. Powerful beliefs establish only great, elaborate sports; democratic, new consequences scratch hardly problems. Comparatively esta|4.61|1.47|6009001|maxicorp #1|9|womens watch|6|Jewelry|504|esebaranti|N/A|86lawn59809529522730|yellow|Ton|Unknown|9|callyn stcallyoughtought| +11697|AAAAAAAAALNCAAAA|2000-10-27||Wrong cl|8.97|1.47|10003002|exportiunivamalg #2|9|dvd/vcr players|10|Electronics|572|esebaranti|N/A|61194sky863980512604|pink|Unknown|Unknown|41|ationn stcallyoughtought| +11698|AAAAAAAACLNCAAAA|1997-10-27|1999-10-27|Efficient, available police may respond as. Conferences would let fine pr|1.89|0.96|2004001|edu packimporto #1|4|sports-apparel|2|Men|77|ationation|extra large|5spring2337700648696|navy|Case|Unknown|43|eingn stcallyoughtought| +11699|AAAAAAAACLNCAAAA|1999-10-28|2001-10-26|Far, confident officers ought to enhance changes. Main, clear branches will stay. Socialist, endless eyes should not want twice rigid, competitive months. Perceptions cannot l|5.41|3.46|9011012|amalgunivamalg #12|4|cooking|9|Books|325|antiablepri|N/A|5spring2337700648696|spring|Dram|Unknown|8|n stn stcallyoughtought| +11700|AAAAAAAACLNCAAAA|2001-10-27||Practitioners expand for a troops; friendly models could not penetrate very political, new efforts. Others like further quite subsequent procedur|2.06|1.13|7008001|namelessbrand #1|4|lighting|7|Home|98|eingn st|N/A|526705puff3461002175|powder|N/A|Unknown|19|barbarationoughtought| +11701|AAAAAAAAFLNCAAAA|1997-10-27||Surprisingly additional dogs go without a glasses; examinations consider schools. Clear workers may not complete ago local nu|4.63|3.70|7009002|maxibrand #2|9|mattresses|7|Home|903|pribarn st|N/A|258463midnight209621|spring|Case|Unknown|15|oughtbarationoughtought| +11702|AAAAAAAAGLNCAAAA|1997-10-27|2000-10-26|Necessary powers could look. Other, public feelings would affect too moving, ultimate duties. Countries make under different problems. Left descriptions would not approve hours|5.31|3.23|5004001|edu packscholar #1|4|classical|5|Music|128|eingableought|N/A|35808210olive8802169|goldenrod|Dozen|Unknown|36|ablebarationoughtought| +11703|AAAAAAAAGLNCAAAA|2000-10-27||Political, waste schools work sometimes authorities; years could not go. Also familiar things play enough even formal poin|1.62|1.29|5004001|edu packunivamalg #8|4|audio|10|Electronics|128|eingableought|N/A|35808210olive8802169|tomato|Case|Unknown|97|pribarationoughtought| +11704|AAAAAAAAILNCAAAA|1997-10-27|1999-10-27|Inevitably good years must understand operations. Originally regular systems help good, skilled sons. Museums could find national parents. Plants find into the needs. Following|7.85|4.63|9008005|namelessmaxi #5|8|romance|9|Books|43|priese|N/A|207141807lemon108604|puff|N/A|Unknown|19|esebarationoughtought| +11705|AAAAAAAAILNCAAAA|1999-10-28|2001-10-26|Splendid, subsequent telecommunications could occupy victorian musicians; partn|7.54|4.63|4004002|edu packedu pack #2|8|athletic|4|Shoes|250|barantiable|medium|7382050604643tomato5|red|Pallet|Unknown|76|antibarationoughtought| +11706|AAAAAAAAILNCAAAA|2001-10-27||Splendid, subsequent telecommunications could occupy victorian musicians; partn|4.47|4.02|6009003|maxicorp #3|8|womens watch|6|Jewelry|795|barantiable|N/A|00141seashell3636610|gainsboro|Gross|Unknown|85|callybarationoughtought| +11707|AAAAAAAALLNCAAAA|1997-10-27||Again integrated circumstances used to remove especially about|1.13|0.61|8012008|importomaxi #8|12|guns|8|Sports|82|ableeing|N/A|709peach177525731766|saddle|Bundle|Unknown|16|ationbarationoughtought| +11708|AAAAAAAAMLNCAAAA|1997-10-27|2000-10-26|Appropriate, special fans may not talk best rather real feet. Generally mass systems define so. Today tragic towns ensure only established, serious players. Good at|6.74|5.32|9012009|importounivamalg #9|12|home repair|9|Books|185|antieingought|N/A|43363494438772pale30|gainsboro|Dram|Unknown|32|eingbarationoughtought| +11709|AAAAAAAAMLNCAAAA|2000-10-27||Appropriate, special fans may not talk best rather real feet. Generally mass systems define so. Today tragic towns ensure only established, serious players. Good at|2.68|2.19|9012009|corpmaxi #10|16|golf|8|Sports|88|eingeing|N/A|8851984423587salmon7|rose|Lb|Unknown|23|n stbarationoughtought| +11710|AAAAAAAAOLNCAAAA|1997-10-27|1999-10-27|Inner, obvious members capture social teams. Literary studies agree developments. Aware, prime injuries must help sites. More corporate problems may explore still days. Unable commun|4.62|4.15|2004001|edu packimporto #1|4|sports-apparel|2|Men|260|barcallyable|extra large|magenta5549822760742|floral|Case|Unknown|78|baroughtationoughtought| +11711|AAAAAAAAOLNCAAAA|1999-10-28|2001-10-26|Inner, obvious members capture social teams. Literary studies agree developments. Aware, prime injuries must help sites. More corporate problems may explore still days. Unable commun|6.02|4.15|9001012|amalgmaxi #12|1|arts|9|Books|260|barcallyable|N/A|5191lime130021556002|puff|Dram|Unknown|3|oughtoughtationoughtought| +11712|AAAAAAAAOLNCAAAA|2001-10-27||Inner, obvious members capture social teams. Literary studies agree developments. Aware, prime injuries must help sites. More corporate problems may explore still days. Unable commun|0.46|4.15|9001012|amalgedu pack #1|1|womens|4|Shoes|260|barcallyable|extra large|257344424yellow50869|royal|Each|Unknown|5|ableoughtationoughtought| +11713|AAAAAAAABMNCAAAA|1997-10-27||Low, fair hours lead other stones. Also clear differences mention eastern contexts; men end essential, ltd. ages. International, cultural months continue earlier. Problems reduce|2.90|1.42|8014004|edu packmaxi #4|14|tennis|8|Sports|62|ablecally|N/A|15675908slate0706777|wheat|Gross|Unknown|16|prioughtationoughtought| +11714|AAAAAAAACMNCAAAA|1997-10-27|2000-10-26|Full, bottom levels would buy. Districts ought to make somehow shareholders. Different counties commit long by the details. English, presidential years afford |3.15|2.55|4001001|amalgedu pack #1|1|womens|4|Shoes|178|eingationought|large|frosted5825788840242|pale|Case|Unknown|16|eseoughtationoughtought| +11715|AAAAAAAACMNCAAAA|2000-10-27||Full, bottom levels would buy. Districts ought to make somehow shareholders. Different counties commit long by the details. English, presidential years afford |3.23|2.55|4001001|amalgimporto #2|1|accessories|2|Men|435|antipriese|petite|frosted5825788840242|sienna|Gram|Unknown|24|antioughtationoughtought| +11716|AAAAAAAAEMNCAAAA|1997-10-27|1999-10-27|Various, old computers ma|7.28|4.14|10004004|edu packunivamalg #4|4|audio|10|Electronics|175|antiationought|N/A|04819966smoke4731604|tan|Gross|Unknown|17|callyoughtationoughtought| +11717|AAAAAAAAEMNCAAAA|1999-10-28|2001-10-26|Abruptly social trains would inject environmental, good visitors. Red streets put primary groups. Very social years ask often old, final schools. Here slight|4.56|4.14|10004004|namelessunivamalg #5|4|scanners|10|Electronics|313|prioughtpri|N/A|slate601279106228230|olive|Unknown|Unknown|16|ationoughtationoughtought| +11718|AAAAAAAAEMNCAAAA|2001-10-27||Abruptly social trains would inject environmental, good visitors. Red streets put primary groups. Very social years ask often old, final schools. Here slight|68.55|51.41|10004004|univmaxi #9|4|pools|8|Sports|84|prioughtpri|N/A|slate601279106228230|medium|Case|Unknown|19|eingoughtationoughtought| +11719|AAAAAAAAHMNCAAAA|1997-10-27||As great eyes ought to talk then. Natural drawings shall not generate to a hands. Artistic seconds |9.23|3.32|7005010|scholarbrand #10|5|blinds/shades|7|Home|49|n stese|N/A|7snow569029587865074|smoke|Oz|Unknown|10|n stoughtationoughtought| +11720|AAAAAAAAIMNCAAAA|1997-10-27|2000-10-26|Applications could make similar observations. Pp. would disappear english units. Mothers start instead in the makers. Empty, public fruits |3.09|1.17|9015009|scholarunivamalg #9|15|fiction|9|Books|266|callycallyable|N/A|7522785903157white43|snow|Oz|Unknown|18|barableationoughtought| +11721|AAAAAAAAIMNCAAAA|2000-10-27||Financial, final categories believe terms; warm, other details walk eagerly just common brothers. Much important toys consider very open files. Polite, national lights ma|3.69|1.17|6002002|importocorp #2|15|diamonds|6|Jewelry|266|callycallyable|N/A|604389914107853tan06|lace|Ounce|Unknown|71|oughtableationoughtought| +11722|AAAAAAAAKMNCAAAA|1997-10-27|1999-10-27|Yet complex diff|6.10|3.41|9016011|corpunivamalg #11|16|mystery|9|Books|99|n stn st|N/A|349851871681red88954|olive|Case|Unknown|58|ableableationoughtought| +11723|AAAAAAAAKMNCAAAA|1999-10-28|2001-10-26|Yet complex diff|5.81|4.06|6015004|scholarbrand #4|16|custom|6|Jewelry|439|n stpriese|N/A|349851871681red88954|rosy|Cup|Unknown|65|priableationoughtought| +11724|AAAAAAAAKMNCAAAA|2001-10-27||Yet complex diff|2.00|4.06|1003001|exportiamalg #1|3|maternity|1|Women|154|n stpriese|large|5333980203lime153188|violet|Cup|Unknown|15|eseableationoughtought| +11725|AAAAAAAANMNCAAAA|1997-10-27||Indirect words should pay now directly|0.30|0.26|1001002|amalgamalg #2|1|dresses|1|Women|266|callycallyable|large|36571735navajo041993|forest|Unknown|Unknown|25|antiableationoughtought| +11726|AAAAAAAAOMNCAAAA|1997-10-27|2000-10-26|Psychiatric, english tonnes afford. Av|2.70|1.26|1002001|importoamalg #1|2|fragrances|1|Women|215|antioughtable|N/A|80659070667741pink10|plum|Case|Unknown|5|callyableationoughtought| +11727|AAAAAAAAOMNCAAAA|2000-10-27||Available, alone rights tackle in a ministers. Acids spend politically terrible stages. Far available things should seem more sole|0.31|1.26|1002001|univmaxi #8|10|pools|8|Sports|215|antioughtable|N/A|9705258366994peru286|rosy|Pound|Unknown|31|ationableationoughtought| +11728|AAAAAAAAANNCAAAA|1997-10-27|1999-10-27|Thorough, disastrous lives can tick different activities. A|0.09|0.05|5001001|amalgscholar #1|1|rock|5|Music|569|n stcallyanti|N/A|9406990320279red8767|rosy|Case|Unknown|31|eingableationoughtought| +11729|AAAAAAAAANNCAAAA|1999-10-28|2001-10-26|Thorough, disastrous lives can tick different activities. A|8.53|6.99|2001002|amalgimporto #2|1|accessories|2|Men|498|eingn stese|medium|9406990320279red8767|rosy|Pallet|Unknown|33|n stableationoughtought| +11730|AAAAAAAAANNCAAAA|2001-10-27||Slow, old periods might not correlate |5.81|6.99|2001002|amalgunivamalg #7|11|cooking|9|Books|498|eingn stese|N/A|4028puff678772353057|seashell|Tbl|Unknown|9|barpriationoughtought| +11731|AAAAAAAADNNCAAAA|1997-10-27||Thick orders would allow a bit negative forms. Increasingly good studies spend with the cases. British, independent devices tackle direct, italian things; tomorrow new members ought t|0.16|0.14|7010002|univnameless #2|10|flatware|7|Home|321|oughtablepri|N/A|484420505146saddle64|lemon|Case|Unknown|40|oughtpriationoughtought| +11732|AAAAAAAAENNCAAAA|1997-10-27|2000-10-26|Necessary, old tonnes preserve only before a features. New boats report brave, small flowers. Terms make occasionally strategic, previous customers. Very, deep activiti|8.87|5.76|1003001|exportiamalg #1|3|maternity|1|Women|110|baroughtought|economy|0441smoke60343939209|salmon|Case|Unknown|19|ablepriationoughtought| +11733|AAAAAAAAENNCAAAA|2000-10-27||British, democratic areas come yet even surprised organisations. Friends could contact adults. Beautiful demands collect then military tales. V|5.66|5.76|1003001|corpmaxi #4|3|golf|8|Sports|110|baroughtought|N/A|93581060691850wheat5|plum|Box|Unknown|20|pripriationoughtought| +11734|AAAAAAAAGNNCAAAA|1997-10-27|1999-10-27|Voters learn both young arms. Victims need less however front cases; shapes can cover|5.46|2.78|7002005|importobrand #5|2|bedding|7|Home|609|n stbarcally|N/A|146927053turquoise12|powder|Tsp|Unknown|16|esepriationoughtought| +11735|AAAAAAAAGNNCAAAA|1999-10-28|2001-10-26|Over automatic years touch too from a ways. Just immediate reports examine now irish inches. New years should|7.29|2.78|7002005|importounivamalg #6|2|camcorders|10|Electronics|623|priablecally|N/A|45tomato352397474639|snow|Gram|Unknown|25|antipriationoughtought| +11736|AAAAAAAAGNNCAAAA|2001-10-27||European reserves ought to help for the measures; civil, essential acids ought to take quickly of|3.59|2.78|1001001|amalgamalg #1|1|dresses|1|Women|623|priablecally|medium|45tomato352397474639|tomato|Dram|Unknown|9|callypriationoughtought| +11737|AAAAAAAAJNNCAAAA|1997-10-27||Men fire old, other affairs. Moral, young shelves could take more after a others; too growing customers must not want reasonably off the talks. Centuries like. Eyes thank much new, special goods; hug|0.20|0.15|8013008|exportimaxi #8|13|sailing|8|Sports|64|esecally|N/A|36864747puff24983669|lemon|Oz|Unknown|12|ationpriationoughtought| +11738|AAAAAAAAKNNCAAAA|1997-10-27|2000-10-26|Overseas comments want particularly intensive, local words. Long churches must cause there by a products; homes get particular, local individ|6.58|3.35|4001001|amalgedu pack #1|1|womens|4|Shoes|997|ationn stn st|small|014401533steel866423|smoke|Dram|Unknown|44|eingpriationoughtought| +11739|AAAAAAAAKNNCAAAA|2000-10-27||Total, english documents used to see in order dramatic possibilities. Reasonable, daily par|8.88|7.19|7005010|scholarbrand #10|5|blinds/shades|7|Home|997|ationn stn st|N/A|014401533steel866423|peru|Tbl|Unknown|23|n stpriationoughtought| +11740|AAAAAAAAMNNCAAAA|1997-10-27|1999-10-27|Old, nuclea|4.40|3.65|6001005|amalgcorp #5|1|birdal|6|Jewelry|97|ationn st|N/A|7777570seashell33561|white|Pound|Unknown|24|bareseationoughtought| +11741|AAAAAAAAMNNCAAAA|1999-10-28|2001-10-26|Old, nuclea|2.70|1.18|6001005|edu packunivamalg #6|14|sports|9|Books|27|ationn st|N/A|080598245goldenrod58|wheat|Each|Unknown|70|oughteseationoughtought| +11742|AAAAAAAAMNNCAAAA|2001-10-27||Old, nuclea|4.49|1.18|6001005|edu packamalgamalg #13|14|automotive|10|Electronics|236|callypriable|N/A|1211154turquoise4432|sky|Each|Unknown|8|ableeseationoughtought| +11743|AAAAAAAAPNNCAAAA|1997-10-27||Often old girls can remain so in the mines. Beautiful times find into a cards. Political, married walls may restrain different, easy demonstrations; nervous prices beli|1.34|1.13|4002002|importoedu pack #2|2|mens|4|Shoes|57|ationanti|large|5931562tan6096653763|tomato|Case|Unknown|58|prieseationoughtought| +11744|AAAAAAAAAONCAAAA|1997-10-27|2000-10-26|Particular boats used to change aside for example small documents. Special, valuable rules cannot assess clear simply large girls. Feet might boycott just southern forces.|5.11|2.40|3003001|exportiexporti #1|3|toddlers|3|Children|462|ablecallyese|medium|4999434626salmon2433|coral|Bunch|Unknown|16|eseeseationoughtought| +11745|AAAAAAAAAONCAAAA|2000-10-27||Domestic, hot governments should improve still. Red reductions can concentrate also too dark studies; variations go able notes. Nations determine. Front christians engage behind|5.96|2.20|5004002|edu packscholar #2|3|classical|5|Music|462|ablecallyese|N/A|32351salmon820867693|spring|Bundle|Unknown|59|antieseationoughtought| +11746|AAAAAAAACONCAAAA|1997-10-27|1999-10-27|Impossible, particular drivers clean aside; institutions identify there defensive years. Fine, individual villages begin however nearly black visito|7.11|3.41|2003001|exportiimporto #1|3|pants|2|Men|264|esecallyable|petite|497papaya37177826837|dark|Cup|Unknown|16|callyeseationoughtought| +11747|AAAAAAAACONCAAAA|1999-10-28|2001-10-26|Services advise. Still geographical beaches continue then animals. Uses can come months. Easy, theoretical suggestions used t|2.30|3.41|2003001|exportiexporti #2|3|toddlers|3|Children|305|esecallyable|medium|29327wheat2325741014|honeydew|Carton|Unknown|22|ationeseationoughtought| +11748|AAAAAAAACONCAAAA|2001-10-27||Services advise. Still geographical beaches continue then animals. Uses can come months. Easy, theoretical suggestions used t|82.41|46.97|2003001|edu packedu pack #1|4|athletic|4|Shoes|305|esecallyable|large|35887369333yellow639|sandy|Pound|Unknown|32|eingeseationoughtought| +11749|AAAAAAAAFONCAAAA|1997-10-27||Annual forests meet. Even small instances occur feet. Different qualities replace. Main, new hands determine however for a ideas. Physical sides worry as|7.41|3.48|1001002|amalgamalg #2|1|dresses|1|Women|235|antipriable|medium|13067468198460lace20|tomato|Carton|Unknown|21|n steseationoughtought| +11750|AAAAAAAAGONCAAAA|1997-10-27|2000-10-26|Minor policies could not think already reliable pockets. Old, essential markets read local, large teams; towns find more cells. Today nic|8.05|3.70|10011004|amalgamalgamalg #4|11|disk drives|10|Electronics|231|oughtpriable|N/A|0dim6354020015534593|midnight|Pound|Unknown|79|barantiationoughtought| +11751|AAAAAAAAGONCAAAA|2000-10-27||Minor policies could not think already reliable pockets. Old, essential markets read local, large teams; towns find more cells. Today nic|31.27|3.70|4002002|importoedu pack #2|11|mens|4|Shoes|231|oughtpriable|large|8801277383orchid0181|rosy|Pallet|Unknown|20|oughtantiationoughtought| +11752|AAAAAAAAIONCAAAA|1997-10-27|1999-10-27|Local women distinguish never materials. Names begin sensibly canadian, additional times. Great meetings join even additional, bright workers. At least remote wa|7.65|6.65|3004001|edu packexporti #1|4|school-uniforms|3|Children|192|ablen stought|medium|7875390219920954tan7|navajo|Dram|Unknown|27|ableantiationoughtought| +11753|AAAAAAAAIONCAAAA|1999-10-28|2001-10-26|Local women distinguish never materials. Names begin sensibly canadian, additional times. Great meetings join even additional, bright workers. At least remote wa|23.16|6.65|3004001|amalgamalgamalg #7|11|disk drives|10|Electronics|192|ablen stought|N/A|48847515656purple253|yellow|Unknown|Unknown|34|priantiationoughtought| +11754|AAAAAAAAIONCAAAA|2001-10-27||Long, careful tourists would lose differences; strong years recognize over with a executives.|7.25|6.65|3004001|edu packimporto #1|11|sports-apparel|2|Men|203|pribarable|large|48847515656purple253|indian|Gram|Unknown|52|eseantiationoughtought| +11755|AAAAAAAALONCAAAA|1997-10-27||Implicit, little students used to think recently into the pictures. Essen|6.27|4.01|8008006|namelessnameless #6|8|outdoor|8|Sports|559|n stantianti|N/A|1359steel97098472737|white|Tbl|Unknown|72|antiantiationoughtought| +11756|AAAAAAAAMONCAAAA|1997-10-27|2000-10-26|Subsequently proper reasons ought to come jewish, other things. Only foreign schemes lead for instance specific students. As key flowers might adjust. Sensitive universities must not think|4.10|2.50|4002001|importoedu pack #1|2|mens|4|Shoes|27|ationable|medium|37877lemon3883965483|spring|Bundle|Unknown|5|callyantiationoughtought| +11757|AAAAAAAAMONCAAAA|2000-10-27||Subsequently proper reasons ought to come jewish, other things. Only foreign schemes lead for instance specific students. As key flowers might adjust. Sensitive universities must not think|5.64|2.50|10001011|amalgunivamalg #11|1|cameras|10|Electronics|390|barn stpri|N/A|939189648070tan69365|rosy|N/A|Unknown|29|ationantiationoughtought| +11758|AAAAAAAAOONCAAAA|1997-10-27|1999-10-27|Away central others argu|3.39|1.55|9012011|importounivamalg #11|12|home repair|9|Books|216|callyoughtable|N/A|923powder98743903732|lavender|Gram|Unknown|40|eingantiationoughtought| +11759|AAAAAAAAOONCAAAA|1999-10-28|2001-10-26|Police |91.24|80.29|9012011|importoedu pack #2|2|mens|4|Shoes|216|callyoughtable|large|923powder98743903732|seashell|Ton|Unknown|60|n stantiationoughtought| +11760|AAAAAAAAOONCAAAA|2001-10-27||Police |8.92|80.29|9012011|corpmaxi #1|16|golf|8|Sports|281|oughteingable|N/A|923powder98743903732|orange|Box|Unknown|26|barcallyationoughtought| +11761|AAAAAAAABPNCAAAA|1997-10-27||Glad, annual resources will not try eyes. Behind new sources would n|76.82|26.88|8008002|namelessnameless #2|8|outdoor|8|Sports|859|n stantieing|N/A|7609402turquoise3846|smoke|Ton|Unknown|87|oughtcallyationoughtought| +11762|AAAAAAAACPNCAAAA|1997-10-27|2000-10-26|Comprehensive terms would not deceive maybe between a things.|1.82|1.25|7010003|univnameless #3|10|flatware|7|Home|78|eingation|N/A|737186446pink7768306|saddle|Carton|Unknown|7|ablecallyationoughtought| +11763|AAAAAAAACPNCAAAA|2000-10-27||Female symbols watch in order possible colleges. New, living teams feature from a parties. Deaths produce tonight in order easy birds. Merely furious firms used to postpone free, mental eyes. P|34.69|31.22|7010003|exportiedu pack #2|10|kids|4|Shoes|576|callyationanti|small|737186446pink7768306|sandy|Case|Unknown|50|pricallyationoughtought| +11764|AAAAAAAAEPNCAAAA|1997-10-27|1999-10-27|Local, good names expect substantial, emotional materials. Recent minutes will not take yet more large services. Completely deep wor|7.09|5.03|7015005|scholarnameless #5|15|tables|7|Home|285|antieingable|N/A|11708620199267lime95|sandy|Dozen|Unknown|25|esecallyationoughtought| +11765|AAAAAAAAEPNCAAAA|1999-10-28|2001-10-26|Local, good names expect substantial, emotional materials. Recent minutes will not take yet more large services. Completely deep wor|2.85|1.85|7015005|univbrand #6|10|jewelry boxes|6|Jewelry|189|n steingought|N/A|11708620199267lime95|pale|Lb|Unknown|84|anticallyationoughtought| +11766|AAAAAAAAEPNCAAAA|2001-10-27||Just local babies must see wild; british, brief sorts will not find innocently lives. Consistent teams solve now common others. Posi|2.91|2.29|1004001|edu packamalg #1|4|swimwear|1|Women|189|n steingought|medium|11708620199267lime95|snow|Dozen|Unknown|37|callycallyationoughtought| +11767|AAAAAAAAHPNCAAAA|1997-10-27||Correct, certain humans cut|37.98|34.18|9001008|amalgmaxi #8|1|arts|9|Books|538|eingprianti|N/A|44rosy21454185573117|peach|Bunch|Unknown|93|ationcallyationoughtought| +11768|AAAAAAAAIPNCAAAA|1997-10-27|2000-10-26|Children know as to a references. Hours can change as yet famous figures. About old states see main, cheap|1.66|1.14|1001001|amalgamalg #1|1|dresses|1|Women|74|eseation|petite|12895358sandy2003500|royal|Bundle|Unknown|18|eingcallyationoughtought| +11769|AAAAAAAAIPNCAAAA|2000-10-27||Much monthly children should understand nationally prac|5.49|3.40|2002002|importoimporto #2|1|shirts|2|Men|930|eseation|medium|12895358sandy2003500|slate|Tsp|Unknown|28|n stcallyationoughtought| +11770|AAAAAAAAKPNCAAAA|1997-10-27|1999-10-27|New, sure systems will not make respectiv|0.84|0.25|9015011|scholarunivamalg #11|15|fiction|9|Books|528|eingableanti|N/A|29657488401450olive7|ghost|Oz|Unknown|17|barationationoughtought| +11771|AAAAAAAAKPNCAAAA|1999-10-28|2001-10-26|Other, political years appeal thereafter local, possible supplies. Genera|4.83|0.25|9005012|scholarmaxi #12|5|history|9|Books|48|eingableanti|N/A|93171026cornflower58|ghost|Dozen|Unknown|18|oughtationationoughtought| +11772|AAAAAAAAKPNCAAAA|2001-10-27||Other, political years appeal thereafter local, possible supplies. Genera|0.62|0.25|6013005|exportibrand #5|5|loose stones|6|Jewelry|105|eingableanti|N/A|pale8027009696689697|olive|Box|Unknown|10|ableationationoughtought| +11773|AAAAAAAANPNCAAAA|1997-10-27||Residential authors would mind perhaps disastrous, effe|2.41|1.47|6015006|scholarbrand #6|15|custom|6|Jewelry|570|barationanti|N/A|4saddle2871395383520|peach|Unknown|Unknown|20|priationationoughtought| +11774|AAAAAAAAOPNCAAAA|1997-10-27|2000-10-26|Considerable, grand lives include now annually opening childr|0.61|0.36|4003001|exportiedu pack #1|3|kids|4|Shoes|250|barantiable|petite|32501978489slate1433|maroon|Gram|Unknown|54|eseationationoughtought| +11775|AAAAAAAAOPNCAAAA|2000-10-27||Considerable, grand lives include now annually opening childr|7.36|0.36|4003001|brandmaxi #4|3|reference|9|Books|250|barantiable|N/A|1271043turquoise2800|violet|Tbl|Unknown|76|antiationationoughtought| +11776|AAAAAAAAAAOCAAAA|1997-10-27|1999-10-27|Wives consider careful, envi|1.28|1.02|1001001|amalgamalg #1|1|dresses|1|Women|569|n stcallyanti|economy|47970wheat3405906042|salmon|Tbl|Unknown|64|callyationationoughtought| +11777|AAAAAAAAAAOCAAAA|1999-10-28|2001-10-26|Legs determine severely privat|3.58|1.02|1001001|importoscholar #2|1|country|5|Music|539|n stprianti|N/A|47970wheat3405906042|violet|Bundle|Unknown|62|ationationationoughtought| +11778|AAAAAAAAAAOCAAAA|2001-10-27||Legs determine severely privat|7.78|5.91|1001001|amalgedu pack #1|1|womens|4|Shoes|539|n stprianti|small|47970wheat3405906042|seashell|Unknown|Unknown|65|eingationationoughtought| +11779|AAAAAAAADAOCAAAA|1997-10-27||Whole, pretty examples may lie. Also previous parents will go duties. |2.06|1.81|1001002|amalgamalg #2|1|dresses|1|Women|272|ableationable|petite|40657012936977linen2|gainsboro|Cup|Unknown|15|n stationationoughtought| +11780|AAAAAAAAEAOCAAAA|1997-10-27|2000-10-26|Yet previous efforts stand certainly ready miles. Young, regular hours say on board. Twice upper children det|2.72|0.87|3001001|amalgexporti #1|1|newborn|3|Children|130|barpriought|large|457574065860thistle9|gainsboro|Gross|Unknown|48|bareingationoughtought| +11781|AAAAAAAAEAOCAAAA|2000-10-27||Clean functions cannot sit almost bad boys. Designs give natural, legislative idea|9.43|0.87|7008002|namelessbrand #2|1|lighting|7|Home|130|barpriought|N/A|457574065860thistle9|powder|Box|Unknown|32|oughteingationoughtought| +11782|AAAAAAAAGAOCAAAA|1997-10-27|1999-10-27|Political, dramatic banks used to st|9.40|4.79|6011007|amalgbrand #7|11|semi-precious|6|Jewelry|174|eseationought|N/A|222154turquoise90672|plum|Dozen|Unknown|18|ableeingationoughtought| +11783|AAAAAAAAGAOCAAAA|1999-10-28|2001-10-26|Political, dramatic banks used to st|8.62|4.79|4002002|importoedu pack #2|11|mens|4|Shoes|174|eseationought|small|222154turquoise90672|powder|Dram|Unknown|35|prieingationoughtought| +11784|AAAAAAAAGAOCAAAA|2001-10-27||Political, dramatic banks used to st|5.36|4.79|4002002|importoscholar #1|2|country|5|Music|871|eseationought|N/A|222154turquoise90672|olive|Unknown|Unknown|8|eseeingationoughtought| +11785|AAAAAAAAJAOCAAAA|1997-10-27||That right mines used to contribute more in order mathematical items. Possible representatives s|8.05|5.87|9008002|namelessmaxi #2|8|romance|9|Books|252|ableantiable|N/A|71328579thistle27046|rosy|Bundle|Unknown|39|antieingationoughtought| +11786|AAAAAAAAKAOCAAAA|1997-10-27|2000-10-26|Directly available agents may make equivalent lengths. Causal|2.90|1.62|1002001|importoamalg #1|2|fragrances|1|Women|260|barcallyable|small|6661pink568066814554|seashell|Pallet|Unknown|26|callyeingationoughtought| +11787|AAAAAAAAKAOCAAAA|2000-10-27||Versions attach relatively. Far continental criteria expect as major stages. Prisoners shall not point behind under a women; different, national claims ask. Goods see even pure clubs. New, deaf r|1.09|0.70|1002001|brandmaxi #4|2|reference|9|Books|260|barcallyable|N/A|3sky0379404222281749|puff|Lb|Unknown|9|ationeingationoughtought| +11788|AAAAAAAAMAOCAAAA|1997-10-27|1999-10-27|Gates might press here solid applicants; novel, probable minutes get basic processes. Happy bonds might admit even for the words. Only, royal languages used to back again yesterday |7.31|3.43|7009009|maxibrand #9|9|mattresses|7|Home|244|eseeseable|N/A|37533935yellow788408|sandy|Tbl|Unknown|58|eingeingationoughtought| +11789|AAAAAAAAMAOCAAAA|1999-10-28|2001-10-26|Gates might press here solid applicants; novel, probable minutes get basic processes. Happy bonds might admit even for the words. Only, royal languages used to back again yesterday |2.05|3.43|7009009|exportiedu pack #2|3|kids|4|Shoes|594|eseeseable|large|37533935yellow788408|white|Lb|Unknown|20|n steingationoughtought| +11790|AAAAAAAAMAOCAAAA|2001-10-27||Gates might press here solid applicants; novel, probable minutes get basic processes. Happy bonds might admit even for the words. Only, royal languages used to back again yesterday |4.66|3.43|7009009|importoscholar #1|3|country|5|Music|594|eseeseable|N/A|50853rose94530504162|snow|Pallet|Unknown|34|barn stationoughtought| +11791|AAAAAAAAPAOCAAAA|1997-10-27||Industrial states choose p|2.71|1.62|8004002|edu packnameless #2|4|camping|8|Sports|425|antiableese|N/A|4629sky3191218698452|powder|Gram|Unknown|54|oughtn stationoughtought| +11792|AAAAAAAAABOCAAAA|1997-10-27|2000-10-26|Other neighbours might become; other, minimum streets may watch elderly, sure events. Records take most to a tanks. Possible troops give potentially in a waves. Also logical practices mean over|3.68|3.05|2001001|amalgimporto #1|1|accessories|2|Men|195|antin stought|medium|5thistle649102963567|lavender|Box|Unknown|91|ablen stationoughtought| +11793|AAAAAAAAABOCAAAA|2000-10-27||Other neighbours might become; other, minimum streets may watch elderly, sure events. Records take most to a tanks. Possible troops give potentially in a waves. Also logical practices mean over|1.93|3.05|8012004|importomaxi #4|12|guns|8|Sports|220|barableable|N/A|5thistle649102963567|goldenrod|Pound|Unknown|60|prin stationoughtought| +11794|AAAAAAAACBOCAAAA|1997-10-27|1999-10-27|Long, separate relations could not approve quietly beside a children. Also only churches happen generally |0.83|0.48|10005014|scholarunivamalg #14|5|karoke|10|Electronics|38|eingpri|N/A|5603118113sky4266033|seashell|Dozen|Unknown|50|esen stationoughtought| +11795|AAAAAAAACBOCAAAA|1999-10-28|2001-10-26|Long, separate relations could not approve quietly beside a children. Also only churches happen generally |9.07|7.70|10005014|scholarmaxi #12|5|history|9|Books|111|oughtoughtought|N/A|74587274971120smoke3|lawn|Lb|Unknown|88|antin stationoughtought| +11796|AAAAAAAACBOCAAAA|2001-10-27||Long, separate relations could not approve quietly beside a children. Also only churches happen generally |2.32|1.09|10005014|edu packbrand #5|5|estate|6|Jewelry|96|callyn st|N/A|74587274971120smoke3|lavender|Each|Unknown|26|callyn stationoughtought| +11797|AAAAAAAAFBOCAAAA|1997-10-27||Old children consider fo|75.57|52.14|7013008|exportinameless #8|13|wallpaper|7|Home|390|barn stpri|N/A|5806696781556purple8|tomato|Tsp|Unknown|12|ationn stationoughtought| +11798|AAAAAAAAGBOCAAAA|1997-10-27|2000-10-26|Likely, elected reasons keep. Parents step mainl|4.40|1.67|8009009|maxinameless #9|9|optics|8|Sports|29|n stable|N/A|6680snow052772795297|lemon|Unknown|Unknown|85|eingn stationoughtought| +11799|AAAAAAAAGBOCAAAA|2000-10-27||Troops help. Good, comple|2.01|1.00|2002002|importoimporto #2|2|shirts|2|Men|147|ationeseought|medium|352289136446thistle6|ivory|Carton|Unknown|51|n stn stationoughtought| +11800|AAAAAAAAIBOCAAAA|1997-10-27|1999-10-27|Greatly ordinary banks should not think here kings; willing, only collections cannot move. Sub|8.86|7.26|3003001|exportiexporti #1|3|toddlers|3|Children|26|callyable|small|42443203651royal9482|slate|Carton|Unknown|32|barbareingoughtought| +11801|AAAAAAAAIBOCAAAA|1999-10-28|2001-10-26|New, industrial women would come only jobs. Even n|6.00|3.36|1001002|amalgamalg #2|1|dresses|1|Women|27|callyable|medium|42443203651royal9482|honeydew|Case|Unknown|40|oughtbareingoughtought| +11802|AAAAAAAAIBOCAAAA|2001-10-27||Police ought to get still royal, delicate trousers. Now early materials watch. Large |6.47|2.52|1001002|amalgunivamalg #5|1|cameras|10|Electronics|67|ationcally|N/A|044518568686purple66|tan|Carton|Unknown|88|ablebareingoughtought| +11803|AAAAAAAALBOCAAAA|1997-10-27||Soviet, other sites used to say closely social, busy minutes. Major, genetic games discuss fair |9.14|3.10|10009006|maxiunivamalg #6|9|televisions|10|Electronics|85|antieing|N/A|47pale50636742934924|rose|Carton|Unknown|31|pribareingoughtought| +11804|AAAAAAAAMBOCAAAA|1997-10-27|2000-10-26|High, essential groups should not weigh more other years; there different papers could announce; large departments c|4.39|2.98|6011005|amalgbrand #5|11|semi-precious|6|Jewelry|373|priationpri|N/A|3565775908moccasin06|smoke|Ounce|Unknown|8|esebareingoughtought| +11805|AAAAAAAAMBOCAAAA|2000-10-27||High, essential groups should not weigh more other years; there different papers could announce; large departments c|3.91|2.98|8007006|brandnameless #6|11|hockey|8|Sports|634|priationpri|N/A|3565775908moccasin06|thistle|Lb|Unknown|53|antibareingoughtought| +11806|AAAAAAAAOBOCAAAA|1997-10-27|1999-10-27|Then fresh bre|12.25|8.45|10005009|scholarunivamalg #9|5|karoke|10|Electronics|265|anticallyable|N/A|637508013wheat788513|wheat|Unknown|Unknown|14|callybareingoughtought| +11807|AAAAAAAAOBOCAAAA|1999-10-28|2001-10-26|Modern hands may not clarify still. Brown, large child|2.04|8.45|8009008|maxinameless #8|9|optics|8|Sports|265|anticallyable|N/A|637508013wheat788513|bisque|Pallet|Unknown|40|ationbareingoughtought| +11808|AAAAAAAAOBOCAAAA|2001-10-27||Abilities may resolve besides perceptions; primary, visual w|0.42|0.16|8009008|importoamalg #1|9|fragrances|1|Women|89|anticallyable|medium|637508013wheat788513|smoke|Bundle|Unknown|63|eingbareingoughtought| +11809|AAAAAAAABCOCAAAA|1997-10-27||Large, wide programmes will keep never faint forces. Dangerous friends get obviously odd women. Large, general boys believe today. Disastrous results may drop then that actual rock|9.03|7.49|10009012|maxiunivamalg #12|9|televisions|10|Electronics|937|ationprin st|N/A|818957941969597hot09|pale|Dram|Unknown|4|n stbareingoughtought| +11810|AAAAAAAACCOCAAAA|1997-10-27|2000-10-26|Late different horses ought to |5.78|4.33|8015001|scholarmaxi #1|15|fishing|8|Sports|193|prin stought|N/A|63thistle34647461893|pink|Tsp|Unknown|25|baroughteingoughtought| +11811|AAAAAAAACCOCAAAA|2000-10-27||Just obvious p|0.84|4.33|8015001|edu packimporto #2|15|sports-apparel|2|Men|193|prin stought|petite|07526127white1855018|purple|Pound|Unknown|6|oughtoughteingoughtought| +11812|AAAAAAAAECOCAAAA|1997-10-27|1999-10-27|Always pregnant victims would not grow financial, italian eyes; slight, particular minutes respond here in a camps. Perhaps serious opportunities make now except|1.94|0.81|2003001|exportiimporto #1|3|pants|2|Men|133|pripriought|extra large|8463620773409smoke40|maroon|Tsp|Unknown|63|ableoughteingoughtought| +11813|AAAAAAAAECOCAAAA|1999-10-28|2001-10-26|Always pregnant victims would not grow financial, italian eyes; slight, particular minutes respond here in a camps. Perhaps serious opportunities make now except|4.94|0.81|2003001|amalgnameless #4|11|accent|7|Home|17|pripriought|N/A|8463620773409smoke40|burlywood|Cup|Unknown|15|prioughteingoughtought| +11814|AAAAAAAAECOCAAAA|2001-10-27||Always pregnant victims would not grow financial, italian eyes; slight, particular minutes respond here in a camps. Perhaps serious opportunities make now except|1.00|0.81|2003001|exportibrand #5|3|kids|7|Home|258|eingantiable|N/A|8463620773409smoke40|orchid|Pallet|Unknown|25|eseoughteingoughtought| +11815|AAAAAAAAHCOCAAAA|1997-10-27||Much darling |5.12|2.66|5003002|exportischolar #2|3|pop|5|Music|344|eseesepri|N/A|3309253371818lawn279|red|Case|Unknown|27|antioughteingoughtought| +11816|AAAAAAAAICOCAAAA|1997-10-27|2000-10-26|Away corresponding eyes can last then young arms. Able servants increase for example over long things. Able years review almost as physical com|2.72|1.55|1001001|amalgamalg #1|1|dresses|1|Women|598|eingn stanti|small|176818635dim50628125|tomato|Gram|Unknown|20|callyoughteingoughtought| +11817|AAAAAAAAICOCAAAA|2000-10-27||Able moments must not believe. Eyes should proceed worldwide with the police. As national members may draw with a men. Nevertheless social compet|9.80|1.55|4004002|edu packedu pack #2|4|athletic|4|Shoes|598|eingn stanti|petite|176818635dim50628125|thistle|Pallet|Unknown|2|ationoughteingoughtought| +11818|AAAAAAAAKCOCAAAA|1997-10-27|1999-10-27|Regional clothes can enjoy feet. Re|8.58|4.71|8011009|amalgmaxi #9|11|archery|8|Sports|9|n st|N/A|05249735slate6736219|goldenrod|Gram|Unknown|1|eingoughteingoughtought| +11819|AAAAAAAAKCOCAAAA|1999-10-28|2001-10-26|Regional clothes can enjoy feet. Re|2.11|4.71|8011009|maxicorp #4|9|womens watch|6|Jewelry|9|n st|N/A|987400650turquoise06|peru|Each|Unknown|31|n stoughteingoughtought| +11820|AAAAAAAAKCOCAAAA|2001-10-27||Regional clothes can enjoy feet. Re|0.29|0.23|10009006|maxiunivamalg #6|9|televisions|10|Electronics|192|n st|N/A|65navy99921720460439|thistle|Case|Unknown|29|barableeingoughtought| +11821|AAAAAAAANCOCAAAA|1997-10-27||Most fine carers o|1.67|1.41|7009002|maxibrand #2|9|mattresses|7|Home|285|antieingable|N/A|635041533448powder79|turquoise|Carton|Unknown|5|oughtableeingoughtought| +11822|AAAAAAAAOCOCAAAA|1997-10-27|2000-10-26|Standards should pay here. Local, subsequent liabilities stay uncertainly|9.66|7.05|1004001|edu packamalg #1|4|swimwear|1|Women|44|eseese|large|725017724purple95595|slate|Bunch|Unknown|70|ableableeingoughtought| +11823|AAAAAAAAOCOCAAAA|2000-10-27||Standards should pay here. Local, subsequent liabilities stay uncertainly|4.31|3.10|1004001|amalgedu pack #2|4|womens|4|Shoes|297|eseese|N/A|984700153royal684437|pink|Dozen|Unknown|62|priableeingoughtought| +11824|AAAAAAAAADOCAAAA|1997-10-27|1999-10-27|Hands know european, absolu|1.88|1.35|9006005|corpmaxi #5|6|parenting|9|Books|142|ableeseought|N/A|2sky6258329060232314|rosy|Gram|Unknown|13|eseableeingoughtought| +11825|AAAAAAAAADOCAAAA|1999-10-28|2001-10-26|Hands know european, absolu|98.85|37.56|9006005|importoexporti #2|6|infants|3|Children|142|ableeseought|large|2sky6258329060232314|lawn|N/A|Unknown|32|antiableeingoughtought| +11826|AAAAAAAAADOCAAAA|2001-10-27||Hands know european, absolu|4.75|37.56|9015007|scholarunivamalg #7|6|fiction|9|Books|199|ableeseought|N/A|626648olive209321116|peach|Bunch|Unknown|58|callyableeingoughtought| +11827|AAAAAAAADDOCAAAA|1997-10-27||Girls lead badly reasonable regions. Also cultural levels suffer best liable, big feet. Open voters make in order expectations. False, regional ports may see years. Quite l|2.74|2.02|9010008|univunivamalg #8|10|travel|9|Books|9|n st|N/A|6768673112blue564363|rose|Lb|Unknown|12|ationableeingoughtought| +11828|AAAAAAAAEDOCAAAA|1997-10-27|2000-10-26|Beautiful, likely methods lead. Enough ameri|1.47|0.45|1001001|amalgamalg #1|1|dresses|1|Women|251|oughtantiable|extra large|4137761675401plum726|orchid|Bundle|Unknown|24|eingableeingoughtought| +11829|AAAAAAAAEDOCAAAA|2000-10-27||Beautiful, likely methods lead. Enough ameri|2.04|1.65|1001001|exportiedu pack #2|1|kids|4|Shoes|251|oughtantiable|extra large|1451909wheat99571567|red|Bundle|Unknown|15|n stableeingoughtought| +11830|AAAAAAAAGDOCAAAA|1997-10-27|1999-10-27|Various changes must shorten together heavy lessons. Doors make later british initiatives. Recently senior courses regret months. Regular, senior children might encounter merely procedures. Then avail|65.54|35.39|9003011|exportimaxi #11|3|computers|9|Books|40|barese|N/A|3518308tomato9754251|purple|Cup|Unknown|57|barprieingoughtought| +11831|AAAAAAAAGDOCAAAA|1999-10-28|2001-10-26|Various changes must shorten together heavy lessons. Doors make later british initiatives. Recently senior courses regret months. Regular, senior children might encounter merely procedures. Then avail|4.64|1.48|5002002|importoscholar #2|2|country|5|Music|40|barese|N/A|3518308tomato9754251|turquoise|Gross|Unknown|10|oughtprieingoughtought| +11832|AAAAAAAAGDOCAAAA|2001-10-27||Always willing possibilities work however beautiful arts. Just present packages take a bit for a policies. Consumers discover domestic, general children; times leave high, critical times; sur|3.50|1.48|3001001|amalgexporti #1|1|newborn|3|Children|40|barese|large|3518308tomato9754251|ghost|Tsp|Unknown|10|ableprieingoughtought| +11833|AAAAAAAAJDOCAAAA|1997-10-27||Forms fetch always new arms. New friends will make successfully journals. Available players take now police. Ulti|8.29|5.88|3003002|exportiexporti #2|3|toddlers|3|Children|594|esen stanti|medium|480133ghost507598806|snow|Dram|Unknown|75|priprieingoughtought| +11834|AAAAAAAAKDOCAAAA|1997-10-27|2000-10-26|Over personal difficulties must win often cruel, female sales. Yet giant costs might not find just never particular events; games shall say to a cars. Very excellent secrets could beat o|23.27|20.94|6015003|scholarbrand #3|15|custom|6|Jewelry|358|eingantipri|N/A|818381690026violet01|burlywood|Tsp|Unknown|15|eseprieingoughtought| +11835|AAAAAAAAKDOCAAAA|2000-10-27||Over personal difficulties must win often cruel, female sales. Yet giant costs might not find just never particular events; games shall say to a cars. Very excellent secrets could beat o|4.51|20.94|8015006|scholarmaxi #6|15|fishing|8|Sports|358|eingantipri|N/A|64818350023red926477|rosy|Pallet|Unknown|59|antiprieingoughtought| +11836|AAAAAAAAMDOCAAAA|1997-10-27|1999-10-27|More past visitors meet also over large outcomes. Centres protect happy, international sheets. Prayers work today years. Able, video-tape|3.33|2.73|10001005|amalgunivamalg #5|1|cameras|10|Electronics|615|antioughtcally|N/A|5479025673seashell87|purple|Box|Unknown|45|callyprieingoughtought| +11837|AAAAAAAAMDOCAAAA|1999-10-28|2001-10-26|More past visitors meet also over large outcomes. Centres protect happy, international sheets. Prayers work today years. Able, video-tape|2.21|2.73|3003002|exportiexporti #2|1|toddlers|3|Children|250|barantiable|medium|5479025673seashell87|snow|Tsp|Unknown|11|ationprieingoughtought| +11838|AAAAAAAAMDOCAAAA|2001-10-27||Words concern apart powerful, old users. Individual, agricultural stages used to want consistently below a reports. Services keep as pleased, english complaints. Able, practical pictur|0.25|2.73|7016009|corpnameless #9|16|furniture|7|Home|250|barantiable|N/A|5479025673seashell87|pale|Pound|Unknown|3|eingprieingoughtought| +11839|AAAAAAAAPDOCAAAA|1997-10-27||Events can worry open select years. Me|1.70|0.71|5004002|edu packscholar #2|4|classical|5|Music|321|oughtablepri|N/A|rosy3824517502973858|purple|Unknown|Unknown|29|n stprieingoughtought| +11840|AAAAAAAAAEOCAAAA|1997-10-27|2000-10-26|Shy, different materials put as in a characteristics. Weekends should not read always too previous inhabitants. Smart kids can divorce to an a|1.05|0.51|4004001|edu packedu pack #1|4|athletic|4|Shoes|942|ableesen st|large|1lace961215183186428|linen|Tsp|Unknown|33|bareseeingoughtought| +11841|AAAAAAAAAEOCAAAA|2000-10-27||Hundreds sense both french contracts. Now available stairs induce again experiences; certain changes ought to force soon funny characters; financial boys raise official, little aut|4.06|2.92|4004001|importoedu pack #2|2|mens|4|Shoes|295|ableesen st|large|1lace961215183186428|spring|Cup|Unknown|60|oughteseeingoughtought| +11842|AAAAAAAACEOCAAAA|1997-10-27|1999-10-27|Legal others would think fully like the things; existing assessments feed nearly upper, very taxes. Pockets eliminate subsidies. Professional, prime interviews might pick implicitly at least hon|1.03|0.52|3003001|exportiexporti #1|3|toddlers|3|Children|110|baroughtought|small|72723954023111rosy06|tan|Dram|Unknown|59|ableeseeingoughtought| +11843|AAAAAAAACEOCAAAA|1999-10-28|2001-10-26|Legal others would think fully like the things; existing assessments feed nearly upper, very taxes. Pockets eliminate subsidies. Professional, prime interviews might pick implicitly at least hon|2.65|0.52|1002002|importoamalg #2|2|fragrances|1|Women|32|baroughtought|petite|72723954023111rosy06|royal|Cup|Unknown|32|prieseeingoughtought| +11844|AAAAAAAACEOCAAAA|2001-10-27||Political men listen now possible projects. Financial issues would get to the managers. Exciting, armed arms can rally under a advertiseme|2.21|0.52|1002002|exporticorp #5|3|gold|6|Jewelry|32|ablepri|N/A|72723954023111rosy06|saddle|Dozen|Unknown|63|eseeseeingoughtought| +11845|AAAAAAAAFEOCAAAA|1997-10-27||Usually american demands play legitimately superb elements. Subsequent, whole sales see more than main students. Tenderly technical powers |4.86|3.59|4001002|amalgedu pack #2|1|womens|4|Shoes|27|ationable|N/A|41105peru71036536944|floral|Lb|Unknown|25|antieseeingoughtought| +11846|AAAAAAAAGEOCAAAA|1997-10-27|2000-10-26|Poor problems satisfy surprisingly right, administrative prices. Sad dishes talk full, negative rivals. Even |0.91|0.68|7010007|univnameless #7|10|flatware|7|Home|311|oughtoughtpri|N/A|3003609083powder7820|smoke|Cup|Unknown|25|callyeseeingoughtought| +11847|AAAAAAAAGEOCAAAA|2000-10-27||Cold accidents should impose alike. Hopefully positive trains will achieve so nice professional respondents. Little origins explain ago different results. |8.81|0.68|7014008|edu packnameless #8|10|glassware|7|Home|311|oughtoughtpri|N/A|3003609083powder7820|salmon|N/A|Unknown|27|ationeseeingoughtought| +11848|AAAAAAAAIEOCAAAA|1997-10-27|1999-10-27|More victorian letters validate |3.98|2.46|1001001|amalgamalg #1|1|dresses|1|Women|336|callypripri|petite|turquoise76865668671|pale|Tsp|Unknown|21|eingeseeingoughtought| +11849|AAAAAAAAIEOCAAAA|1999-10-28|2001-10-26|Temporarily other minutes shall not work most sure, special persons. Major, angry systems |1.30|2.46|1001001|exporticorp #2|3|gold|6|Jewelry|336|callypripri|N/A|turquoise76865668671|spring|Gross|Unknown|20|n steseeingoughtought| +11850|AAAAAAAAIEOCAAAA|2001-10-27||Temporarily other minutes shall not work most sure, special persons. Major, angry systems |0.37|2.46|1001001|namelesscorp #3|3|mens watch|6|Jewelry|336|callypripri|N/A|turquoise76865668671|turquoise|Bunch|Unknown|25|barantieingoughtought| +11851|AAAAAAAALEOCAAAA|1997-10-27||General needs know only. Respons|9.06|6.34|5001002|amalgscholar #2|1|rock|5|Music|562|ablecallyanti|N/A|352539816951hot26795|salmon|Cup|Unknown|15|oughtantieingoughtought| +11852|AAAAAAAAMEOCAAAA|1997-10-27|2000-10-26|Effects advise. Alrea|4.70|3.38|3001001|amalgexporti #1|1|newborn|3|Children|116|callyoughtought|large|973692white434668082|mint|Box|Unknown|24|ableantieingoughtought| +11853|AAAAAAAAMEOCAAAA|2000-10-27||Interested, personal results appear american, political girls. Conditions should get long both peaceful brothers. Wryly international heads must not get as units.|5.98|3.38|3001001|amalgexporti #2|1|newborn|3|Children|116|callyoughtought|medium|9310710619501plum577|pale|Unknown|Unknown|49|priantieingoughtought| +11854|AAAAAAAAOEOCAAAA|1997-10-27|1999-10-27|Completely common pages make other, o|0.53|0.16|1003001|exportiamalg #1|3|maternity|1|Women|43|priese|petite|742482indian86285842|rosy|Carton|Unknown|14|eseantieingoughtought| +11855|AAAAAAAAOEOCAAAA|1999-10-28|2001-10-26|Completely common pages make other, o|8.36|4.26|1003001|edu packimporto #2|4|sports-apparel|2|Men|17|ationought|extra large|742482indian86285842|saddle|Tsp|Unknown|26|antiantieingoughtought| +11856|AAAAAAAAOEOCAAAA|2001-10-27||Completely common pages make other, o|84.65|4.26|3003001|exportiexporti #1|4|toddlers|3|Children|17|ationought|N/A|742482indian86285842|thistle|Cup|Unknown|49|callyantieingoughtought| +11857|AAAAAAAABFOCAAAA|1997-10-27||Separate techniques go great, legal items. Bri|1.23|0.40|6002002|importocorp #2|2|diamonds|6|Jewelry|96|callyn st|N/A|53pink67610988741441|thistle|Dozen|Unknown|25|ationantieingoughtought| +11858|AAAAAAAACFOCAAAA|1997-10-27|2000-10-26|Thanks require now possible, important words; important policies remember bad, different securities. Normal, independent teachers could run particularly normal, social parties. Objects could know |6.71|5.70|6006003|corpcorp #3|6|rings|6|Jewelry|548|eingeseanti|N/A|867484258861orange91|saddle|Tsp|Unknown|7|eingantieingoughtought| +11859|AAAAAAAACFOCAAAA|2000-10-27||Thanks require now possible, important words; important policies remember bad, different securities. Normal, independent teachers could run particularly normal, social parties. Objects could know |8.80|2.72|6006003|scholaramalgamalg #11|15|portable|10|Electronics|553|eingeseanti|N/A|867484258861orange91|snow|Bundle|Unknown|27|n stantieingoughtought| +11860|AAAAAAAAEFOCAAAA|1997-10-27|1999-10-27|Elsewhere favorite groups would swing english, powerful years. Children cannot pref|4.78|3.53|5004001|edu packscholar #1|4|classical|5|Music|295|antin stable|N/A|77834133burnished990|grey|Pallet|Unknown|61|barcallyeingoughtought| +11861|AAAAAAAAEFOCAAAA|1999-10-28|2001-10-26|Elsewhere favorite groups would swing english, powerful years. Children cannot pref|0.32|0.14|8001002|amalgnameless #2|4|athletic shoes|8|Sports|343|antin stable|N/A|77834133burnished990|navajo|Each|Unknown|11|oughtcallyeingoughtought| +11862|AAAAAAAAEFOCAAAA|2001-10-27||Elsewhere favorite groups would swing english, powerful years. Children cannot pref|7.29|0.14|8001002|amalgedu pack #1|4|womens|4|Shoes|122|ableableought|petite|royal573971672608611|yellow|Ton|Unknown|28|ablecallyeingoughtought| +11863|AAAAAAAAHFOCAAAA|1997-10-27||Conditions rise recently nearly aware horses. Difficult areas should anal|0.09|0.07|10008015|namelessunivamalg #15|8|scanners|10|Electronics|281|oughteingable|N/A|867472tomato55200393|violet|Pound|Unknown|29|pricallyeingoughtought| +11864|AAAAAAAAIFOCAAAA|1997-10-27|2000-10-26|Economic, scottish ears could not work so grand market|2.96|2.48|5003001|exportischolar #1|3|pop|5|Music|592|ablen stanti|N/A|755798908violet13677|mint|Ton|Unknown|55|esecallyeingoughtought| +11865|AAAAAAAAIFOCAAAA|2000-10-27||Economic, scottish ears could not work so grand market|9.56|4.30|5003001|exportiexporti #2|3|toddlers|3|Children|592|ablen stanti|extra large|755798908violet13677|powder|Each|Unknown|6|anticallyeingoughtought| +11866|AAAAAAAAKFOCAAAA|1997-10-27|1999-10-27|Of course regular employees go internal faces. Years shall share long mature, particular experiments. O|1.11|0.62|2003001|exportiimporto #1|3|pants|2|Men|653|priantically|large|47slate7635840918686|snow|Ounce|Unknown|19|callycallyeingoughtought| +11867|AAAAAAAAKFOCAAAA|1999-10-28|2001-10-26|Of course regular employees go internal faces. Years shall share long mature, particular experiments. O|3.27|1.53|2003001|importonameless #8|2|baseball|8|Sports|653|priantically|N/A|5steel28898772016860|snow|Each|Unknown|23|ationcallyeingoughtought| +11868|AAAAAAAAKFOCAAAA|2001-10-27||Accurate prizes show grounds. Currently impossible losses match even more practical dogs. Large re|8.75|1.53|5004001|edu packscholar #1|4|classical|5|Music|423|priableese|N/A|5steel28898772016860|peru|Ounce|Unknown|21|eingcallyeingoughtought| +11869|AAAAAAAANFOCAAAA|1997-10-27||Active, red things shall remain from the colleagues; largely high members form barely i|5.94|2.13|8014010|edu packmaxi #10|14|tennis|8|Sports|74|eseation|N/A|218221124241plum6135|linen|Oz|Unknown|2|n stcallyeingoughtought| +11870|AAAAAAAAOFOCAAAA|1997-10-27|2000-10-26|Streets would c|9.53|4.95|3003001|exportiexporti #1|3|toddlers|3|Children|278|eingationable|petite|4004619527purple2644|thistle|Carton|Unknown|6|barationeingoughtought| +11871|AAAAAAAAOFOCAAAA|2000-10-27||Streets would c|1.29|0.39|10006006|corpunivamalg #6|6|musical|10|Electronics|364|esecallypri|N/A|397243929dim15657667|deep|Cup|Unknown|23|oughtationeingoughtought| +11872|AAAAAAAAAGOCAAAA|1997-10-27|1999-10-27|Applicable persons exercise at a marks. About good |3.21|2.85|10007007|brandunivamalg #7|7|personal|10|Electronics|36|callypri|N/A|452774puff5869651305|royal|Bunch|Unknown|27|ableationeingoughtought| +11873|AAAAAAAAAGOCAAAA|1999-10-28|2001-10-26|Increasing, other letters find however; forces might go somehow always high vehicles. Ingredients should d|4.22|3.75|8002004|importonameless #4|7|baseball|8|Sports|36|callypri|N/A|92turquoise754686428|ivory|Bunch|Unknown|71|priationeingoughtought| +11874|AAAAAAAAAGOCAAAA|2001-10-27||Local, free services fill judicial, regular goods. Police achieve negative, cultural pools; errors buy |2.18|3.75|5001001|amalgscholar #1|7|rock|5|Music|130|callypri|N/A|92turquoise754686428|sandy|Gross|Unknown|78|eseationeingoughtought| +11875|AAAAAAAADGOCAAAA|1997-10-27||Substantially olympic leaders leap stars. Average, urban nations find games. Electronic years might not go ago sa|0.09|0.05|8008006|namelessnameless #6|8|outdoor|8|Sports|30|barpri|N/A|1338005wheat18191766|purple|Bunch|Unknown|10|antiationeingoughtought| +11876|AAAAAAAAEGOCAAAA|1997-10-27|2000-10-26|Primary sentences go in a arguments; eventually tiny shows should see. Very present parents say however equal, visible markets. Other,|1.44|0.44|7009007|maxibrand #7|9|mattresses|7|Home|70|baration|N/A|6657463769red9983061|sandy|Lb|Unknown|35|callyationeingoughtought| +11877|AAAAAAAAEGOCAAAA|2000-10-27||Lights start generally internal, long rules. Hundreds might echo national, friendly relations. Witnesses cannot support bette|1.73|1.43|7009007|exportibrand #8|9|kids|7|Home|565|anticallyanti|N/A|9198462790seashell81|saddle|Gross|Unknown|20|ationationeingoughtought| +11878|AAAAAAAAGGOCAAAA|1997-10-27|1999-10-27|Reasons may fill. Domestic offers draw mean parties. Years contact. Resulting, good walls wil|1.04|0.89|4001001|amalgedu pack #1|1|womens|4|Shoes|36|callypri|petite|817870345purple14268|rose|Bunch|Unknown|15|eingationeingoughtought| +11879|AAAAAAAAGGOCAAAA|1999-10-28|2001-10-26|Reasons may fill. Domestic offers draw mean parties. Years contact. Resulting, good walls wil|56.28|0.89|4001001|maxinameless #10|1|optics|8|Sports|36|callypri|N/A|817870345purple14268|magenta|Dozen|Unknown|22|n stationeingoughtought| +11880|AAAAAAAAGGOCAAAA|2001-10-27||Reasons may fill. Domestic offers draw mean parties. Years contact. Resulting, good walls wil|9.10|5.91|4001001|univbrand #1|1|jewelry boxes|6|Jewelry|36|callypri|N/A|817870345purple14268|dark|Gram|Unknown|9|bareingeingoughtought| +11881|AAAAAAAAJGOCAAAA|1997-10-27||Years ought to find from an techniques. All thinking approaches take almost|6.39|5.23|2003002|exportiimporto #2|3|pants|2|Men|18|eingought|extra large|548876094881smoke526|lemon|Ton|Unknown|13|oughteingeingoughtought| +11882|AAAAAAAAKGOCAAAA|1997-10-27|2000-10-26|Studies may admit of cours|7.26|5.95|4003001|exportiedu pack #1|3|kids|4|Shoes|332|ablepripri|small|715728387184rosy6983|midnight|Ton|Unknown|16|ableeingeingoughtought| +11883|AAAAAAAAKGOCAAAA|2000-10-27||Studies may admit of cours|0.32|0.19|1001002|amalgamalg #2|3|dresses|1|Women|332|ablepripri|N/A|126308977salmon36635|pale|Ton|Unknown|11|prieingeingoughtought| +11884|AAAAAAAAMGOCAAAA|1997-10-27|1999-10-27|Children wear with |38.33|28.36|8008005|namelessnameless #5|8|outdoor|8|Sports|360|barcallypri|N/A|180royal656057929563|violet|Bundle|Unknown|14|eseeingeingoughtought| +11885|AAAAAAAAMGOCAAAA|1999-10-28|2001-10-26|Doctors fix completely |3.25|28.36|8008005|amalgedu pack #2|8|womens|4|Shoes|360|barcallypri|medium|00876lawn32413113855|smoke|Unknown|Unknown|56|antieingeingoughtought| +11886|AAAAAAAAMGOCAAAA|2001-10-27||Doctors fix completely |55.45|28.36|8008005|namelesscorp #7|8|mens watch|6|Jewelry|44|barcallypri|N/A|00339180peru23748497|sky|Cup|Unknown|24|callyeingeingoughtought| +11887|AAAAAAAAPGOCAAAA|1997-10-27||Democratic, able eyes must not go actually; numbers deal local points. Then golden shares cannot master too dependent, fed|4.56|2.23|3002002|importoexporti #2|2|infants|3|Children|991|oughtn stn st|small|666lavender449365539|slate|Gram|Unknown|34|ationeingeingoughtought| +11888|AAAAAAAAAHOCAAAA|1997-10-27|2000-10-26|Generally co|2.45|1.34|4004001|edu packedu pack #1|4|athletic|4|Shoes|316|callyoughtpri|medium|68spring433819232011|rose|Box|Unknown|19|eingeingeingoughtought| +11889|AAAAAAAAAHOCAAAA|2000-10-27||More important days add sufficiently banks. Entire sales could put students; together radi|0.72|0.48|4004001|amalgimporto #2|4|accessories|2|Men|316|callyoughtpri|large|12white8098743762630|thistle|Carton|Unknown|15|n steingeingoughtought| +11890|AAAAAAAACHOCAAAA|1997-10-27|1999-10-27|Final governm|6.22|5.28|9003011|exportimaxi #11|3|computers|9|Books|512|ableoughtanti|N/A|29859hot832925878617|turquoise|Ounce|Unknown|31|barn steingoughtought| +11891|AAAAAAAACHOCAAAA|1999-10-28|2001-10-26|Vast, possible activities guarantee deaths. Ju|6.95|5.28|10006009|corpunivamalg #9|6|musical|10|Electronics|512|ableoughtanti|N/A|4623780236657lime853|purple|Ton|Unknown|28|oughtn steingoughtought| +11892|AAAAAAAACHOCAAAA||||2.18|1.19|5002001|importoscholar #1||country|5|Music||ableoughtanti|N/A|4623780236657lime853|||||ablen steingoughtought| +11893|AAAAAAAAFHOCAAAA|1997-10-27||Relations work accidentally american prices. Services ought to gro|2.29|1.09|2003002|exportiimporto #2|3|pants|2|Men|578|eingationanti|extra large|168spring30641694715|orange|Unknown|Unknown|3|prin steingoughtought| +11894|AAAAAAAAGHOCAAAA|1997-10-27|2000-10-26|Annual, intact wages get all. Tr|2.79|1.59|2002001|importoimporto #1|2|shirts|2|Men|244|eseeseable|medium|50126552190red676472|yellow|Tsp|Unknown|73|esen steingoughtought| +11895|AAAAAAAAGHOCAAAA|2000-10-27||Industrial, equal acts operate as so adequate officers. General de|4.98|4.23|2002001|edu packedu pack #2|2|athletic|4|Shoes|139|n stpriought|economy|4761medium1903786052|sandy|Tbl|Unknown|24|antin steingoughtought| +11896|AAAAAAAAIHOCAAAA|1997-10-27|1999-10-27|Growing sites can perform. Rights speak terms; arbitrarily lit|3.59|2.11|10016014|corpamalgamalg #14|16|wireless|10|Electronics|202|ablebarable|N/A|57590871775124red116|navajo|Lb|Unknown|27|callyn steingoughtought| +11897|AAAAAAAAIHOCAAAA|1999-10-28|2001-10-26|Other, southern wings see then both old colleges; writers start usually at once correct moments. Various matters tell directly european forms. Things buy indeed again roman example|8.25|3.96|4001002|amalgedu pack #2|16|womens|4|Shoes|202|ablebarable|small|57590871775124red116|rosy|Oz|Unknown|1|ationn steingoughtought| +11898|AAAAAAAAIHOCAAAA|2001-10-27||Ill leaves may not mind beautiful yards. Girls may take much total meals. Judges finish at least. Constant, financial ideas would avoid even about a arts. Pounds erect useful, other worlds. Bri|2.04|0.73|4001002|edu packedu pack #1|4|athletic|4|Shoes|202|ablebarable|large|57590871775124red116|saddle|Tbl|Unknown|57|eingn steingoughtought| +11899|AAAAAAAALHOCAAAA|1997-10-27||Monetary others will take elsewhere with a sessions. Arms st|7.09|6.23|1004002|edu packamalg #2|4|swimwear|1|Women|517|ationoughtanti|extra large|25485822sandy1078156|salmon|Gross|Unknown|43|n stn steingoughtought| +11900|AAAAAAAAMHOCAAAA|1997-10-27|2000-10-26|Married years used to take helpful, full i|4.68|3.13|1003001|exportiamalg #1|3|maternity|1|Women|295|antin stable|petite|4975054steel18278656|seashell|Gross|Unknown|4|barbarn stoughtought| +11901|AAAAAAAAMHOCAAAA|2000-10-27||Married years used to take helpful, full i|1.66|3.13|10016002|corpamalgamalg #2|3|wireless|10|Electronics|295|antin stable|N/A|158909131240249lace3|slate|N/A|Unknown|23|oughtbarn stoughtought| +11902|AAAAAAAAOHOCAAAA|1997-10-27|1999-10-27|British, domestic girls will not recruit technical officers. Federal policies come cruel industries. Aloft other languages under|2.90|1.21|5002001|importoscholar #1|2|country|5|Music|331|oughtpripri|N/A|7464607767811red5077|purple|Bunch|Unknown|3|ablebarn stoughtought| +11903|AAAAAAAAOHOCAAAA|1999-10-28|2001-10-26|Modern, annual police must operate aside linguistic names. British results might appear. Personal roads can see central children. Well political charges may tend shortly arrangeme|4.50|1.21|5002001|corpmaxi #4|16|golf|8|Sports|15|oughtpripri|N/A|7464607767811red5077|ivory|Gram|Unknown|94|pribarn stoughtought| +11904|AAAAAAAAOHOCAAAA|2001-10-27||Modern, annual police must operate aside linguistic names. British results might appear. Personal roads can see central children. Well political charges may tend shortly arrangeme|4.27|2.90|1001001|amalgamalg #1|1|dresses|1|Women|148|eingeseought|petite|7464607767811red5077|medium|Carton|Unknown|33|esebarn stoughtought| +11905|AAAAAAAABIOCAAAA|1997-10-27||Overall companies will not say senses. So inappropriate circumstances leave yesterday only other mountains. Persons fight else bitter metres. Correctly linguistic patients handle others. Curr|4.63|1.94|8006006|corpnameless #6|6|football|8|Sports|401|oughtbarese|N/A|72243799papaya620333|sky|Cup|Unknown|15|antibarn stoughtought| +11906|AAAAAAAACIOCAAAA|1997-10-27|2000-10-26|Rich powers can look in a reports. Also new towns must read just. Now likely sets help somewhat into a architects. Married, extensive views pay assessments; months lift briti|2.30|1.51|7005007|scholarbrand #7|5|blinds/shades|7|Home|454|eseantiese|N/A|3800wheat52031837159|purple|Gram|Unknown|30|callybarn stoughtought| +11907|AAAAAAAACIOCAAAA|2000-10-27||Competent, commercial choices achieve; as likely years fly red rebels. Russian, particular lovers should contribute forward; systems agree in a hours; just entire types should |4.83|1.51|7005007|edu packimporto #2|5|sports-apparel|2|Men|454|eseantiese|petite|3800wheat52031837159|gainsboro|Ounce|Unknown|9|ationbarn stoughtought| +11908|AAAAAAAAEIOCAAAA|1997-10-27|1999-10-27|Types think gold powers. Costs can watch either clear defences. Enough overall ways see politely interested appeals. Opening nurses withdraw also limited names. Schools give much; possible leve|3.32|2.72|4003001|exportiedu pack #1|3|kids|4|Shoes|123|priableought|small|0939honeydew65359709|smoke|Bundle|Unknown|26|eingbarn stoughtought| +11909|AAAAAAAAEIOCAAAA|1999-10-28|2001-10-26|Functions must not play nearly categories. Professional modules take|0.45|2.72|4003001|namelessnameless #10|8|outdoor|8|Sports|780|bareingation|N/A|0939honeydew65359709|light|Gram|Unknown|74|n stbarn stoughtought| +11910|AAAAAAAAEIOCAAAA|2001-10-27||Forwards wrong activities can bomb most possible existing pounds. More than evident girls can harm also. Upper solicitors must think on a days. Authoriti|5.04|2.72|3004001|edu packexporti #1|8|school-uniforms|3|Children|388|bareingation|N/A|2578782lemon42770577|sky|Carton|Unknown|30|baroughtn stoughtought| +11911|AAAAAAAAHIOCAAAA|1997-10-27||Original, retail poems should ma|0.77|0.49|8011002|amalgmaxi #2|11|archery|8|Sports|577|ationationanti|N/A|90violet546783662948|slate|Pound|Unknown|49|oughtoughtn stoughtought| +11912|AAAAAAAAIIOCAAAA|1997-10-27|2000-10-26|Experimental chips shall fit as beliefs. Eyebrows would exceed just arms. Owners jump reluctant, geographical talks. Likely, other perceptions proceed ag|84.76|47.46|2004001|edu packimporto #1|4|sports-apparel|2|Men|178|eingationought|small|831pale4429936925695|lemon|Lb|Unknown|27|ableoughtn stoughtought| +11913|AAAAAAAAIIOCAAAA|2000-10-27||Experimental chips shall fit as beliefs. Eyebrows would exceed just arms. Owners jump reluctant, geographical talks. Likely, other perceptions proceed ag|3.53|47.46|2004001|scholarmaxi #10|4|history|9|Books|178|eingationought|N/A|1498447874steel72833|dim|Oz|Unknown|24|prioughtn stoughtought| +11914|AAAAAAAAKIOCAAAA|1997-10-27|1999-10-27|White, natural students think eyes; satisfied sources should test capable, new paintings; around necessary ear|30.23|9.67|3004001|edu packexporti #1|4|school-uniforms|3|Children|781|oughteingation|petite|598217469320puff0477|drab|Dozen|Unknown|52|eseoughtn stoughtought| +11915|AAAAAAAAKIOCAAAA|1999-10-28|2001-10-26|White, natural students think eyes; satisfied sources should test capable, new paintings; around necessary ear|0.80|9.67|7010006|univnameless #6|4|flatware|7|Home|781|oughteingation|N/A|642077180pink6124311|powder|Pallet|Unknown|2|antioughtn stoughtought| +11916|AAAAAAAAKIOCAAAA|2001-10-27||White, natural students think eyes; satisfied sources should test capable, new paintings; around necessary ear|3.55|1.49|7010006|amalgunivamalg #17|1|cameras|10|Electronics|781|oughteingation|N/A|642077180pink6124311|papaya|Dozen|Unknown|20|callyoughtn stoughtought| +11917|AAAAAAAANIOCAAAA|1997-10-27||Violent regions must suggest material products. Then nuclear lessons widen already things. Associat|8.58|6.26|2002002|importoimporto #2|2|shirts|2|Men|468|eingcallyese|large|77indian033813791919|snow|Carton|Unknown|32|ationoughtn stoughtought| +11918|AAAAAAAAOIOCAAAA|1997-10-27|2000-10-26|Roughly important ideas form here long tonnes.|4.20|2.10|3004001|edu packexporti #1|4|school-uniforms|3|Children|208|eingbarable|economy|5311835527723peru447|rose|Unknown|Unknown|21|eingoughtn stoughtought| +11919|AAAAAAAAOIOCAAAA|2000-10-27||Roughly important ideas form here long tonnes.|6.59|2.57|3004001|amalgedu pack #2|4|womens|4|Shoes|97|ationn st|medium|5916675549slate03888|turquoise|Bunch|Unknown|15|n stoughtn stoughtought| +11920|AAAAAAAAAJOCAAAA|1997-10-27|1999-10-27|International, alone days get more exceedingly clear homes. There other officers might not make for a problems. Ultimate, wealthy pages cure steadily. Arms could stand old, territorial functions. Ri|5.37|3.22|4001001|amalgedu pack #1|1|womens|4|Shoes|572|ableationanti|large|misty462004420718855|tan|Pound|Unknown|95|barablen stoughtought| +11921|AAAAAAAAAJOCAAAA|1999-10-28|2001-10-26|International, alone days get more exceedingly clear homes. There other officers might not make for a problems. Ultimate, wealthy pages cure steadily. Arms could stand old, territorial functions. Ri|0.85|3.22|4001001|scholarmaxi #6|1|history|9|Books|392|ablen stpri|N/A|37steel7538144408027|peru|Dram|Unknown|13|oughtablen stoughtought| +11922|AAAAAAAAAJOCAAAA|2001-10-27||International, alone days get more exceedingly clear homes. There other officers might not make for a problems. Ultimate, wealthy pages cure steadily. Arms could stand old, territorial functions. Ri|5.95|3.22|4001001|edu packedu pack #1|4|athletic|4|Shoes|392|ablen stpri|small|971531309antique9733|mint|Pallet|Unknown|14|ableablen stoughtought| +11923|AAAAAAAADJOCAAAA|1997-10-27||Rather proper personnel vie|0.67|0.46|7008004|namelessbrand #4|8|lighting|7|Home|77|ationation|N/A|8violet9790413594842|lime|Box|Unknown|9|priablen stoughtought| +11924|AAAAAAAAEJOCAAAA|1997-10-27|2000-10-26|Operations support arab pupils; small, initial surfaces ought to remember now at first outstan|4.39|1.75|3003001|exportiexporti #1|3|toddlers|3|Children|321|oughtablepri|large|3645180smoke27979589|turquoise|Tsp|Unknown|20|eseablen stoughtought| +11925|AAAAAAAAEJOCAAAA|2000-10-27||Then poor families should m|2.08|1.75|3003001|importobrand #6|2|bedding|7|Home|321|oughtablepri|N/A|95255273chartreuse59|thistle|Oz|Unknown|26|antiablen stoughtought| +11926|AAAAAAAAGJOCAAAA|1997-10-27|1999-10-27|Constant seats carry particular, numerous supporters. Large seats receive else rules. British pp. must marry as political eggs; complex, poor relations b|4.75|1.94|1004001|edu packamalg #1|4|swimwear|1|Women|275|antiationable|large|1532613violet7239307|green|Ton|Unknown|5|callyablen stoughtought| +11927|AAAAAAAAGJOCAAAA|1999-10-28|2001-10-26|Other, social frien|3.25|1.62|1004001|corpunivamalg #11|4|musical|10|Electronics|393|antiationable|N/A|1532613violet7239307|smoke|Tbl|Unknown|59|ationablen stoughtought| +11928|AAAAAAAAGJOCAAAA|2001-10-27||Other, social frien|1.18|0.79|8003009|exportinameless #9|4|basketball|8|Sports|255|antiantiable|N/A|1532613violet7239307|deep|Box|Unknown|9|eingablen stoughtought| +11929|AAAAAAAAJJOCAAAA|1997-10-27||Fiercely internal fingers arrange away widespread right|4.98|3.53|5001002|amalgscholar #2|1|rock|5|Music|142|ableeseought|N/A|54220115942floral112|steel|Case|Unknown|46|n stablen stoughtought| +11930|AAAAAAAAKJOCAAAA|1997-10-27|2000-10-26|Champions ought to seem miles; nervous, great police send numbers. Doubt|6.38|4.40|1002001|importoamalg #1|2|fragrances|1|Women|159|n stantiought|small|1549821776858royal43|smoke|Oz|Unknown|3|barprin stoughtought| +11931|AAAAAAAAKJOCAAAA|2000-10-27||British others act at a|9.39|5.63|10007015|brandunivamalg #15|7|personal|10|Electronics|519|n stantiought|N/A|3892turquoise0869738|slate|Each|Unknown|31|oughtprin stoughtought| +11932|AAAAAAAAMJOCAAAA|1997-10-27|1999-10-27|All minute markets provide extremely then different obligations. Necessary fears would not control in the solutions. Young ingredients shall fly in a terms. Accounts will not sum relevant boat|32.34|27.48|4002001|importoedu pack #1|2|mens|4|Shoes|142|ableeseought|large|93408331765803wheat1|smoke|Dozen|Unknown|93|ableprin stoughtought| +11933|AAAAAAAAMJOCAAAA|1999-10-28|2001-10-26|Flowers used to keep di|1.77|0.83|4002001|scholarunivamalg #17|5|karoke|10|Electronics|142|ableeseought|N/A|93408331765803wheat1|powder|Dozen|Unknown|28|priprin stoughtought| +11934|AAAAAAAAMJOCAAAA|2001-10-27||Tomorrow important occasions carry international, good skills. Back, gre|1.54|0.92|8001005|amalgnameless #5|1|athletic shoes|8|Sports|142|ableeseought|N/A|93408331765803wheat1|papaya|Tbl|Unknown|4|eseprin stoughtought| +11935|AAAAAAAAPJOCAAAA|1997-10-27||Dead systems stay even good lines. Ahead late companies might switch emotionally much opposite children. English, important polls can receive well int|3.04|1.61|7006006|corpbrand #6|6|rugs|7|Home|121|oughtableought|N/A|0114318117moccasin47|sky|Bundle|Unknown|20|antiprin stoughtought| +11936|AAAAAAAAAKOCAAAA|1997-10-27|2000-10-26|Households exist great during a leaders. Lucky, sorry profits adopt whole, international eggs; electrical woods occupy again young friends. Papers will not teach too difficult duties. Terrible,|0.51|0.39|5001001|amalgscholar #1|1|rock|5|Music|749|n steseation|N/A|thistle6367212176127|hot|Dram|Unknown|55|callyprin stoughtought| +11937|AAAAAAAAAKOCAAAA|2000-10-27||Procedures must not take |5.36|0.39|8003008|exportinameless #8|3|basketball|8|Sports|749|n steseation|N/A|thistle6367212176127|powder|Bunch|Unknown|44|ationprin stoughtought| +11938|AAAAAAAACKOCAAAA|1997-10-27|1999-10-27|Literary, unable proceedings must recognise full, white estates; national banks would take roughly also capital proceedings.|6.75|4.32|6002003|importocorp #3|2|diamonds|6|Jewelry|235|antipriable|N/A|56126110309897lemon5|spring|Lb|Unknown|63|eingprin stoughtought| +11939|AAAAAAAACKOCAAAA|1999-10-28|2001-10-26|Literary, unable proceedings must recognise full, white estates; national banks would take roughly also capital proceedings.|2.46|4.32|6002003|importoimporto #2|2|shirts|2|Men|107|ationbarought|extra large|9224757694690green07|peach|Dozen|Unknown|8|n stprin stoughtought| +11940|AAAAAAAACKOCAAAA|2001-10-27||Literary, unable proceedings must recognise full, white estates; national banks would take roughly also capital proceedings.|4.43|2.65|6002003|amalgamalg #1|1|dresses|1|Women|107|ationbarought|medium|9224757694690green07|saddle|Gram|Unknown|34|baresen stoughtought| +11941|AAAAAAAAFKOCAAAA|1997-10-27||New, large problems like very easy, capable rates; now accurate rivers may not watch at the boards. Yards shall|4.95|1.73|6004006|edu packcorp #6|4|bracelets|6|Jewelry|562|ablecallyanti|N/A|99129604700sandy9899|maroon|Pound|Unknown|78|oughtesen stoughtought| +11942|AAAAAAAAGKOCAAAA|1997-10-27|2000-10-26|Now good walls deal currently physical proceedings. Important buildings swear around |5.54|4.70|7004003|edu packbrand #3|4|curtains/drapes|7|Home|214|eseoughtable|N/A|snow8084519929129679|yellow|Carton|Unknown|19|ableesen stoughtought| +11943|AAAAAAAAGKOCAAAA|2000-10-27||Now good walls deal currently physical proceedings. Important buildings swear around |0.26|0.13|5004002|edu packscholar #2|4|classical|5|Music|135|eseoughtable|N/A|682red29712232950905|lavender|Pallet|Unknown|38|priesen stoughtought| +11944|AAAAAAAAIKOCAAAA|1997-10-27|1999-10-27|Indeed right affairs would take perfectly new negotiations. Main, recent cases can show also heavy days. Dead, large employees must expect tomorrow to a jobs. Just large relations may come still avai|2.97|1.18|5004001|edu packscholar #1|4|classical|5|Music|215|antioughtable|N/A|34salmon804173036316|violet|Tbl|Unknown|21|eseesen stoughtought| +11945|AAAAAAAAIKOCAAAA|1999-10-28|2001-10-26|Indeed right affairs would take perfectly new negotiations. Main, recent cases can show also heavy days. Dead, large employees must expect tomorrow to a jobs. Just large relations may come still avai|3.86|3.04|10012012|importoamalgamalg #12|12|monitors|10|Electronics|194|antioughtable|N/A|1rosy248177615679645|tomato|Cup|Unknown|18|antiesen stoughtought| +11946|AAAAAAAAIKOCAAAA|2001-10-27||Successful, mental schools count past, physical councils. Previously urban quantities speak. High, golden doctors concede different, free patients. Mas|5.57|3.45|8002007|importonameless #7|2|baseball|8|Sports|194|esen stought|N/A|54999smoke5857799211|snow|Cup|Unknown|86|callyesen stoughtought| +11947|AAAAAAAALKOCAAAA|1997-10-27||Eyes should see on a others. Rare, true problems look local, poor months. Jewish orders will lo|1.23|0.93|4004002|edu packedu pack #2|4|athletic|4|Shoes|12|ableought|extra large|salmon87616471693573|thistle|Bundle|Unknown|50|ationesen stoughtought| +11948|AAAAAAAAMKOCAAAA|1997-10-27|2000-10-26|Quick jobs reflect. Implications ought to believe generally volun|0.39|0.33|10014015|edu packamalgamalg #15|14|automotive|10|Electronics|12|ableought|N/A|12997667650508puff68|plum|Bundle|Unknown|32|eingesen stoughtought| +11949|AAAAAAAAMKOCAAAA|2000-10-27||Whole, other hands used to imagine probably. Running, common boxes shall not hire somewhat others. White authorities used to incorporate straight for a effects; modest, profound hands kill too l|2.63|0.33|10014015|maxiunivamalg #16|9|televisions|10|Electronics|12|ableought|N/A|7359navy501472300975|salmon|Case|Unknown|45|n stesen stoughtought| +11950|AAAAAAAAOKOCAAAA|1997-10-27|1999-10-27|Characters will comment women|9.54|6.67|3001001|amalgexporti #1|1|newborn|3|Children|161|oughtcallyought|medium|935752260211sandy634|snow|Unknown|Unknown|9|barantin stoughtought| +11951|AAAAAAAAOKOCAAAA|1999-10-28|2001-10-26|Women earn also lexical changes. Open factors might not run al|6.51|2.79|3001001|edu packscholar #2|1|classical|5|Music|161|oughtcallyought|N/A|37turquoise027124718|olive|Carton|Unknown|21|oughtantin stoughtought| +11952|AAAAAAAAOKOCAAAA|2001-10-27||Women earn also lexical changes. Open factors might not run al|4.73|2.79|6013001|exportibrand #1|13|loose stones|6|Jewelry|161|oughtcallyought|N/A|87steel8761469365531|yellow|N/A|Unknown|36|ableantin stoughtought| +11953|AAAAAAAABLOCAAAA|1997-10-27||Speakers can consider nuclear, ordinary ex|0.81|0.33|1004002|edu packamalg #2|4|swimwear|1|Women|35|antipri|medium|481482063006419rosy0|smoke|Lb|Unknown|1|priantin stoughtought| +11954|AAAAAAAACLOCAAAA|1997-10-27|2000-10-26|Regional times must seem immediate amounts. Full schools shall record great, respo|0.80|0.28|7008005|namelessbrand #5|8|lighting|7|Home|327|ationablepri|N/A|2651980934purple6826|burnished|Box|Unknown|32|eseantin stoughtought| +11955|AAAAAAAACLOCAAAA|2000-10-27||Regional times must seem immediate amounts. Full schools shall record great, respo|4.31|2.15|10004005|edu packunivamalg #5|4|audio|10|Electronics|456|callyantiese|N/A|2yellow2778620685214|tan|Bundle|Unknown|53|antiantin stoughtought| +11956|AAAAAAAAELOCAAAA|1997-10-27|1999-10-27|Military methods see relatively k|6.38|3.19|10011006|amalgamalgamalg #6|11|disk drives|10|Electronics|175|antiationought|N/A|7035837light23054908|rosy|Dozen|Unknown|30|callyantin stoughtought| +11957|AAAAAAAAELOCAAAA|1999-10-28|2001-10-26|Military methods see relatively k|0.41|0.27|6013006|exportibrand #6|11|loose stones|6|Jewelry|175|antiationought|N/A|400459343018ghost220|white|Ton|Unknown|28|ationantin stoughtought| +11958|AAAAAAAAELOCAAAA|2001-10-27||Structural criteria will not find ava|52.49|20.99|6005007|scholarcorp #7|5|earings|6|Jewelry|289|n steingable|N/A|400459343018ghost220|tan|Bunch|Unknown|9|eingantin stoughtought| +11959|AAAAAAAAHLOCAAAA|1997-10-27||Irish, close variations used to get years. Royal arts see increasingly hundreds. Empty trials prefer again nearly open ways; already human materials seize even|1.39|0.58|4003002|exportiedu pack #2|3|kids|4|Shoes|295|antin stable|extra large|166236434197lemon624|snow|Pallet|Unknown|27|n stantin stoughtought| +11960|AAAAAAAAILOCAAAA|1997-10-27|2000-10-26|Commitments draw authorities. Welsh eyes must not extend later necessary applications. Impossible columns accept. At least everyday theories go nearly ibid christian months. Nowhere early years set a|4.36|1.96|6002001|importocorp #1|2|diamonds|6|Jewelry|48|eingese|N/A|mint7886333539335290|misty|Dram|Unknown|3|barcallyn stoughtought| +11961|AAAAAAAAILOCAAAA|2000-10-27||Mental meals meet traditional employees. Months could not save chief s|80.86|1.96|10003011|exportiunivamalg #11|3|dvd/vcr players|10|Electronics|577|eingese|N/A|mint7886333539335290|maroon|Bundle|Unknown|1|oughtcallyn stoughtought| +11962|AAAAAAAAKLOCAAAA|1997-10-27|1999-10-27|Again available costs could try there. No longer wrong pupils prove inadvertently european, civil times. Words turn as biological fears. Either magnetic benefits produce clearly on|5.93|4.50|6015003|scholarbrand #3|15|custom|6|Jewelry|23|priable|N/A|28peach4753792433524|rosy|Ton|Unknown|3|ablecallyn stoughtought| +11963|AAAAAAAAKLOCAAAA|1999-10-28|2001-10-26|Again available costs could try there. No longer wrong pupils prove inadvertently european, civil times. Words turn as biological fears. Either magnetic benefits produce clearly on|4.91|2.06|1002002|importoamalg #2|15|fragrances|1|Women|23|priable|economy|82mint83471897354777|blue|Dozen|Unknown|37|pricallyn stoughtought| +11964|AAAAAAAAKLOCAAAA|2001-10-27||Real, legal police sha|1.75|1.34|10007014|brandunivamalg #14|15|personal|10|Electronics|23|priable|N/A|82mint83471897354777|peru|Cup|Unknown|54|esecallyn stoughtought| +11965|AAAAAAAANLOCAAAA|1997-10-27||Great parents light enough soviet shoulders. Orange bo|1.87|0.93|1002002|importoamalg #2|2|fragrances|1|Women|424|eseableese|economy|8483medium4957625660|turquoise|Dram|Unknown|14|anticallyn stoughtought| +11966|AAAAAAAAOLOCAAAA|1997-10-27|2000-10-26|Signs perform still then far details. Clear techniques look thirdly out of a colleges. Rather cold students attract fully e|2.33|1.11|4004001|edu packedu pack #1|4|athletic|4|Shoes|286|callyeingable|small|7thistle523173079263|tomato|Tsp|Unknown|8|callycallyn stoughtought| +11967|AAAAAAAAOLOCAAAA|2000-10-27||Signs perform still then far details. Clear techniques look thirdly out of a colleges. Rather cold students attract fully e|4.64|1.11|8003008|exportinameless #8|4|basketball|8|Sports|286|callyeingable|N/A|3589111midnight10584|ghost|Dram|Unknown|71|ationcallyn stoughtought| +11968|AAAAAAAAAMOCAAAA|1997-10-27|1999-10-27|Blank, well-know|5.71|1.88|5003001|exportischolar #1|3|pop|5|Music|196|callyn stought|N/A|757141966burlywood35|snow|Lb|Unknown|13|eingcallyn stoughtought| +11969|AAAAAAAAAMOCAAAA|1999-10-28|2001-10-26|Blank, well-know|1.83|1.57|5003001|namelessmaxi #6|3|romance|9|Books|815|callyn stought|N/A|06166205547984steel9|saddle|Bundle|Unknown|25|n stcallyn stoughtought| +11970|AAAAAAAAAMOCAAAA|2001-10-27||Soon huge cases meet more; costs can develop open languages; fo|6.67|1.57|2002001|importoimporto #1|3|shirts|2|Men|815|callyn stought|N/A|06166205547984steel9|peru|Box|Unknown|3|barationn stoughtought| +11971|AAAAAAAADMOCAAAA|1997-10-27||Linguistic, little authorities perform as well ne|3.74|2.58|1002002|importoamalg #2|2|fragrances|1|Women|253|priantiable|petite|062rosy8381495635487|indian|Box|Unknown|66|oughtationn stoughtought| +11972|AAAAAAAAEMOCAAAA|1997-10-27|2000-10-26|Royal cases deman|57.84|27.76|2003001|exportiimporto #1|3|pants|2|Men|85|antieing|large|24475506972tan493067|white|Unknown|Unknown|60|ableationn stoughtought| +11973|AAAAAAAAEMOCAAAA|2000-10-27||Royal cases deman|6.87|27.76|3002002|importoexporti #2|3|infants|3|Children|85|antieing|medium|802924969509slate696|dim|Gross|Unknown|27|priationn stoughtought| +11974|AAAAAAAAGMOCAAAA|1997-10-27|1999-10-27|Ordinary issues dry only numerous, substantial sheets. Numbers may carry so increased feet; even human peoples drift too; unlikely,|7.54|6.55|7010005|univnameless #5|10|flatware|7|Home|143|prieseought|N/A|50089smoke0979814521|navy|Carton|Unknown|43|eseationn stoughtought| +11975|AAAAAAAAGMOCAAAA|1999-10-28|2001-10-26|American terms survive. Common opportunities would put quite standards. Junior, european miles would not d|0.89|0.44|7010005|exportiexporti #2|10|toddlers|3|Children|143|prieseought|extra large|50089smoke0979814521|pale|Each|Unknown|13|antiationn stoughtought| +11976|AAAAAAAAGMOCAAAA|2001-10-27||American terms survive. Common opportunities would put quite standards. Junior, european miles would not d|1.07|0.44|1002001|importoamalg #1|10|fragrances|1|Women|388|prieseought|medium|19761116green3617075|mint|Cup|Unknown|49|callyationn stoughtought| +11977|AAAAAAAAJMOCAAAA|1997-10-27||Capital, primary comparisons want in a unions. Little schools should establish. Mistakenly mild limits take like prayers. Previous areas rebuild almost so |2.18|0.76|5002002|importoscholar #2|2|country|5|Music|635|antiprically|N/A|0288snow738559792754|smoke|Dram|Unknown|4|ationationn stoughtought| +11978|AAAAAAAAKMOCAAAA|1997-10-27|2000-10-26|Chiefly closed characteristics avoid automatically very men. Certain, new years run poor, continuing hours. Expressions operate acts. Key objections should |81.00|30.78|7014009|edu packnameless #9|14|glassware|7|Home|26|callyable|N/A|seashell914730536887|wheat|Pound|Unknown|2|eingationn stoughtought| +11979|AAAAAAAAKMOCAAAA|2000-10-27||Strong, bad authorities lie well brave, original flights. Local homes may move for a arts; gaps record re|1.59|1.17|7014009|edu packamalg #2|14|swimwear|1|Women|26|callyable|extra large|0084429314salmon7202|pink|Unknown|Unknown|59|n stationn stoughtought| +11980|AAAAAAAAMMOCAAAA|1997-10-27|1999-10-27|Narrow, famous years shall address se|81.29|62.59|10012013|importoamalgamalg #13|12|monitors|10|Electronics|70|baration|N/A|499462021949673sky86|royal|Dozen|Unknown|2|bareingn stoughtought| +11981|AAAAAAAAMMOCAAAA|1999-10-28|2001-10-26|Narrow, famous years shall address se|3.43|2.60|9002006|importomaxi #6|2|business|9|Books|266|callycallyable|N/A|pale8910118457735486|blue|Carton|Unknown|69|oughteingn stoughtought| +11982|AAAAAAAAMMOCAAAA|2001-10-27||Sexual names can find; firms must pay ever so on a privileges. Natural, private sectors move always goods. Years could not continue. Possible units would not use seve|1.82|2.60|8010003|univmaxi #3|2|pools|8|Sports|266|callycallyable|N/A|2837447forest6079603|rose|Cup|Unknown|53|ableeingn stoughtought| +11983|AAAAAAAAPMOCAAAA|1997-10-27||Independent, appropri|0.98|0.72|1004002|edu packamalg #2|4|swimwear|1|Women|315|antioughtpri|large|208red67032637974838|plum|Carton|Unknown|31|prieingn stoughtought| +11984|AAAAAAAAANOCAAAA|1997-10-27|2000-10-26|Outstanding, small friends face here possibly temporary events; joint clothes |9.84|6.49|8013005|exportimaxi #5|13|sailing|8|Sports|239|n stpriable|N/A|21metallic0501366021|lawn|Lb|Unknown|82|eseeingn stoughtought| +11985|AAAAAAAAANOCAAAA|2000-10-27||Outstanding, small friends face here possibly temporary events; joint clothes |2.70|6.49|8013005|importoscholar #2|13|country|5|Music|328|n stpriable|N/A|02669996456272lemon7|floral|Unknown|Unknown|61|antieingn stoughtought| +11986|AAAAAAAACNOCAAAA|1997-10-27|1999-10-27|Especially alone lines hear; chinese careers could ask. Police phone however|6.31|4.60|2002001|importoimporto #1|2|shirts|2|Men|374|eseationpri|petite|251741orange79049553|turquoise|Ton|Unknown|23|callyeingn stoughtought| +11987|AAAAAAAACNOCAAAA|1999-10-28|2001-10-26|Especially alone lines hear; chinese careers could ask. Police phone however|7.92|4.27|2002001|amalgedu pack #2|2|womens|4|Shoes|100|eseationpri|medium|251741orange79049553|royal|Oz|Unknown|34|ationeingn stoughtought| +11988|AAAAAAAACNOCAAAA|2001-10-27||Especially alone lines hear; chinese careers could ask. Police phone however|1.50|4.27|2002001|scholarnameless #9|15|tables|7|Home|136|callypriought|N/A|79432432820violet746|moccasin|Each|Unknown|50|eingeingn stoughtought| +11989|AAAAAAAAFNOCAAAA|1997-10-27||Wonderful systems ask also very parliamentary orders; british companies|87.12|36.59|9002002|importomaxi #2|2|business|9|Books|64|esecally|N/A|linen075448789521743|peach|Tsp|Unknown|17|n steingn stoughtought| +11990|AAAAAAAAGNOCAAAA|1997-10-27|2000-10-26|Basic options used to remove nearly final aims. Subtle, senior islands require too words. Systems cancel long chains. Countries may study most gastric, wide needs. There|0.79|0.63|10011006|amalgamalgamalg #6|11|disk drives|10|Electronics|432|ablepriese|N/A|5748376seashell13091|ghost|Unknown|Unknown|20|barn stn stoughtought| +11991|AAAAAAAAGNOCAAAA|2000-10-27||Wages will not mean always essential, popular trends. True studies ignore again possible h|4.12|2.51|4001002|amalgedu pack #2|1|womens|4|Shoes|999|ablepriese|extra large|5748376seashell13091|seashell|Bunch|Unknown|55|oughtn stn stoughtought| +11992|AAAAAAAAINOCAAAA|1997-10-27|1999-10-27|Royal, available fingers shall fear again modern, running revenues. El|1.60|0.75|6005001|scholarcorp #1|5|earings|6|Jewelry|454|eseantiese|N/A|7175pale321493576392|puff|Gram|Unknown|42|ablen stn stoughtought| +11993|AAAAAAAAINOCAAAA|1999-10-28|2001-10-26|Royal, available fingers shall fear again modern, running revenues. El|4.34|1.99|8007004|brandnameless #4|7|hockey|8|Sports|170|barationought|N/A|7175pale321493576392|red|Case|Unknown|24|prin stn stoughtought| +11994|AAAAAAAAINOCAAAA|2001-10-27||Accessible, appropriate patterns should not step yesterday here great units. Indeed human campaigns believe. Broad services go finally experimental reasons. Obviously whole teache|2.16|1.64|8007004|edu packamalg #1|4|swimwear|1|Women|797|barationought|medium|7175pale321493576392|floral|Bundle|Unknown|60|esen stn stoughtought| +11995|AAAAAAAALNOCAAAA|1997-10-27||Occasions accept in a changes. Problems could tell. Inner friends arrange interviews. New feet ride all right white surfaces; doors seem in order; extensive, familiar years might find today|3.88|2.05|5003002|exportischolar #2|3|pop|5|Music|267|ationcallyable|N/A|31696726tomato477182|white|Dram|Unknown|21|antin stn stoughtought| +11996|AAAAAAAAMNOCAAAA|1997-10-27|2000-10-26|Campaigns expect certainly full funds. Better shared qualities kill in a stars. Special profits note exclusive constraints. Only new workers ought to live. Exhibitions used to assum|6.92|4.29|1004001|edu packamalg #1|4|swimwear|1|Women|84|eseeing|medium|81749852988pink33543|navy|Oz|Unknown|21|callyn stn stoughtought| +11997|AAAAAAAAMNOCAAAA|2000-10-27||Campaigns expect certainly full funds. Better shared qualities kill in a stars. Special profits note exclusive constraints. Only new workers ought to live. Exhibitions used to assum|1.94|1.12|1004001|importobrand #8|4|bedding|7|Home|237|eseeing|N/A|81749852988pink33543|sienna|Box|Unknown|50|ationn stn stoughtought| +11998|AAAAAAAAONOCAAAA|1997-10-27|1999-10-27|Boundaries can suspend by a ways. Good, international powers ought to offer currently rare, full costs. High messages would learn sorry, economic authorities. Major, ava|9.35|5.42|3004001|edu packexporti #1|4|school-uniforms|3|Children|138|eingpriought|petite|honeydew205744710867|goldenrod|Case|Unknown|1|eingn stn stoughtought| +11999|AAAAAAAAONOCAAAA|1999-10-28|2001-10-26|Boundaries can suspend by a ways. Good, international powers ought to offer currently rare, full costs. High messages would learn sorry, economic authorities. Major, ava|53.04|5.42|8008010|namelessnameless #10|8|outdoor|8|Sports|138|eingpriought|N/A|powder79149129463583|turquoise|Bundle|Unknown|17|n stn stn stoughtought| +12000|AAAAAAAAONOCAAAA|2001-10-27||Young, popular affairs will talk well over a children. Hardly |7.39|3.32|8008010|importoexporti #1|2|infants|3|Children|779|eingpriought|extra large|90066282lace11053422|magenta|Pallet|Unknown|61|barbarbarableought| +12001|AAAAAAAABOOCAAAA|1997-10-27||Students get completely terrible, tight negotiations. Failures may meet so into a organisations. Nurses provide impossible days; early silent countries pu|6.63|2.05|4003002|exportiedu pack #2|3|kids|4|Shoes|336|callypripri|extra large|138580820175peach643|sandy|Ounce|Unknown|3|oughtbarbarableought| +12002|AAAAAAAACOOCAAAA|1997-10-27|2000-10-26|Only excellent concentrations shall want more monthly, blind subsid|8.38|3.35|9016003|corpunivamalg #3|16|mystery|9|Books|51|oughtanti|N/A|3715750303217indian9|grey|Each|Unknown|8|ablebarbarableought| +12003|AAAAAAAACOOCAAAA|2000-10-27||Military, recent counties play even particular eyes. Poor, informal areas seem so financial leaders. Good, warm facts make relentlessly worst personal courts. Employees form then environmental, simple|2.70|2.02|9016003|exportiunivamalg #4|13|self-help|9|Books|116|callyoughtought|N/A|3715750303217indian9|tomato|Dozen|Unknown|20|pribarbarableought| +12004|AAAAAAAAEOOCAAAA|1997-10-27|1999-10-27|However low purposes shall not explode. Common, new communities might not use then healthy centuries. Rivers discuss inner, competen|8.38|7.45|4002001|importoedu pack #1|2|mens|4|Shoes|673|priationcally|small|51547683yellow748583|steel|Gross|Unknown|52|esebarbarableought| +12005|AAAAAAAAEOOCAAAA|1999-10-28|2001-10-26|Simple, visual poli|5.08|2.28|5003002|exportischolar #2|2|pop|5|Music|137|priationcally|N/A|34440840black4250515|white|Dram|Unknown|26|antibarbarableought| +12006|AAAAAAAAEOOCAAAA|2001-10-27||Old, given lives take. Else small trees might keep frequently then economic characteristics. Also original lives could command merely ac|2.50|0.87|1003001|exportiamalg #1|2|maternity|1|Women|137|ationpriought|large|34440840black4250515|maroon|Each|Unknown|20|callybarbarableought| +12007|AAAAAAAAHOOCAAAA|1997-10-27||Dramatic, limited|94.57|65.25|6003008|exporticorp #8|3|gold|6|Jewelry|65|antically|N/A|979608227slate998984|saddle|Bundle|Unknown|1|ationbarbarableought| +12008|AAAAAAAAIOOCAAAA|1997-10-27|2000-10-26|Measurements mind false, top funds. Aspects shall reduce already personnel; payable photographs may develop gardens. Processes must feel edges. Certain cases ought to cling from the|7.30|3.06|7006009|corpbrand #9|6|rugs|7|Home|271|oughtationable|N/A|450salmon33890091718|puff|Pallet|Unknown|12|eingbarbarableought| +12009|AAAAAAAAIOOCAAAA|2000-10-27||Measurements mind false, top funds. Aspects shall reduce already personnel; payable photographs may develop gardens. Processes must feel edges. Certain cases ought to cling from the|1.66|3.06|7006009|importocorp #2|6|diamonds|6|Jewelry|271|oughtationable|N/A|450salmon33890091718|moccasin|Oz|Unknown|81|n stbarbarableought| +12010|AAAAAAAAKOOCAAAA|1997-10-27|1999-10-27|Simple, able examples could not lead crucially suitable, correct villages. Immediate lips reply indeed then thick pp.; women may not let skills; internal,|8.65|6.14|3001001|amalgexporti #1|1|newborn|3|Children|358|eingantipri|economy|879284993105pale7733|yellow|N/A|Unknown|8|baroughtbarableought| +12011|AAAAAAAAKOOCAAAA|1999-10-28|2001-10-26|Others will please secondly new, free areas. Democrats say straightforward years; annual, model arms would follow. Sure meals drive previously modern stories. Honours explain in a arms. Wome|1.42|6.14|3001001|importoamalg #2|2|fragrances|1|Women|358|eingantipri|medium|879284993105pale7733|turquoise|N/A|Unknown|55|oughtoughtbarableought| +12012|AAAAAAAAKOOCAAAA|2001-10-27||Others will please secondly new, free areas. Democrats say straightforward years; annual, model arms would follow. Sure meals drive previously modern stories. Honours explain in a arms. Wome|1.95|0.76|3001001|univbrand #5|2|jewelry boxes|6|Jewelry|517|eingantipri|N/A|879284993105pale7733|powder|Tbl|Unknown|19|ableoughtbarableought| +12013|AAAAAAAANOOCAAAA|1997-10-27||Areas must require with a needs; various, personal professionals support married, wester|0.16|0.11|6001006|amalgcorp #6|1|birdal|6|Jewelry|560|barcallyanti|N/A|frosted2233413222997|turquoise|Pound|Unknown|66|prioughtbarableought| +12014|AAAAAAAAOOOCAAAA|1997-10-27|2000-10-26|Paths could gain ever to a futures. Cigarettes stay currently. Weeks shall not d|3.28|0.98|1002001|importoamalg #1|2|fragrances|1|Women|332|ablepripri|small|603papaya49979927391|maroon|Gross|Unknown|91|eseoughtbarableought| +12015|AAAAAAAAOOOCAAAA|2000-10-27||Red, accurate strengths must not place then components. Existing conditions would hear simple hands; police can keep; both good classes used to |1.09|0.92|1002001|exportinameless #6|3|basketball|8|Sports|332|ablepripri|N/A|603papaya49979927391|turquoise|Unknown|Unknown|22|antioughtbarableought| +12016|AAAAAAAAAPOCAAAA|1997-10-27|1999-10-27|Eyes accept all cases; impossible, uncertain instruments used to respect apparently drugs; bab|0.79|0.33|5004001|edu packscholar #1|4|classical|5|Music|111|oughtoughtought|N/A|958999203987wheat705|sandy|Gross|Unknown|7|callyoughtbarableought| +12017|AAAAAAAAAPOCAAAA|1999-10-28|2001-10-26|High, traditional men will base. Primary miles emerge underground in respect of a oper|37.41|0.33|5004001|brandnameless #8|4|hockey|8|Sports|222|ableableable|N/A|green887533619473109|spring|Dram|Unknown|4|ationoughtbarableought| +12018|AAAAAAAAAPOCAAAA|2001-10-27||High, traditional men will base. Primary miles emerge underground in respect of a oper|4.79|0.33|1004001|edu packamalg #1|4|swimwear|1|Women|222|ableableable|small|8887puff735728775800|slate|Dozen|Unknown|3|eingoughtbarableought| +12019|AAAAAAAADPOCAAAA|1997-10-27||Details explain overseas shareholders; in order local windows must ring historic comments. Other rates attempt very forms; well real problems suspend sites. Rates form therefore similar duties. Likel|92.31|36.00|8016010|corpmaxi #10|16|golf|8|Sports|345|antiesepri|N/A|28719775tomato544342|peru|Case|Unknown|7|n stoughtbarableought| +12020|AAAAAAAAEPOCAAAA|1997-10-27|2000-10-26|Financial, british taxes could not go soon european clients. Devices tell detailed visitors. Now|2.60|1.27|6008005|namelesscorp #5|8|mens watch|6|Jewelry|285|antieingable|N/A|8098294732658184sky2|saddle|Gram|Unknown|2|barablebarableought| +12021|AAAAAAAAEPOCAAAA|2000-10-27||Financial, british taxes could not go soon european clients. Devices tell detailed visitors. Now|7.77|5.05|10005003|scholarunivamalg #3|5|karoke|10|Electronics|297|antieingable|N/A|8098294732658184sky2|sky|Dozen|Unknown|15|oughtablebarableought| +12022|AAAAAAAAGPOCAAAA|1997-10-27|1999-10-27|Widespread, comprehen|2.89|2.31|9012011|importounivamalg #11|12|home repair|9|Books|494|esen stese|N/A|1193purple1401776440|plum|Pallet|Unknown|14|ableablebarableought| +12023|AAAAAAAAGPOCAAAA|1999-10-28|2001-10-26|Requests lac|95.08|2.31|9012011|amalgcorp #8|12|birdal|6|Jewelry|393|esen stese|N/A|9724405423red4520872|medium|Case|Unknown|31|priablebarableought| +12024|AAAAAAAAGPOCAAAA|2001-10-27||Requests lac|4.76|1.99|9012011|exportinameless #5|3|basketball|8|Sports|192|ablen stought|N/A|83puff14177852361724|peru|Unknown|Unknown|44|eseablebarableought| +12025|AAAAAAAAJPOCAAAA|1997-10-27||Practices may not make particularly old reasons. Likely circumstances prove elsewhere dead months. European patterns might take likely part|1.74|0.87|5001002|amalgscholar #2|1|rock|5|Music|221|oughtableable|N/A|645927769697puff1478|magenta|Bunch|Unknown|53|antiablebarableought| +12026|AAAAAAAAKPOCAAAA|1997-10-27|2000-10-26|Stones must analyse enough other, good areas. Aware, safe opponents think anyway volumes; essential leaders know. Reliable changes pay n|3.36|2.88|4002001|importoedu pack #1|2|mens|4|Shoes|220|barableable|large|78281violet569664306|salmon|Tsp|Unknown|69|callyablebarableought| +12027|AAAAAAAAKPOCAAAA|2000-10-27||Stones must analyse enough other, good areas. Aware, safe opponents think anyway volumes; essential leaders know. Reliable changes pay n|69.25|34.62|10005009|scholarunivamalg #9|5|karoke|10|Electronics|220|barableable|N/A|78281violet569664306|spring|Each|Unknown|35|ationablebarableought| +12028|AAAAAAAAMPOCAAAA|1997-10-27|1999-10-27|Old, left countries could cover videos. Then international years help still. Once considerable women fight pages. Expensive, full taxes would maintain groups. For example new groups seem finally ope|2.30|0.71|4004001|edu packedu pack #1|4|athletic|4|Shoes|719|n stoughtation|large|10625214sienna624556|honeydew|Cup|Unknown|43|eingablebarableought| +12029|AAAAAAAAMPOCAAAA|1999-10-28|2001-10-26|Then political presents might not develop much orders. All right ultimate li|23.79|0.71|4004001|edu packimporto #2|4|sports-apparel|2|Men|316|callyoughtpri|large|32500948deep20037783|white|Gross|Unknown|4|n stablebarableought| +12030|AAAAAAAAMPOCAAAA|2001-10-27||Full, national|0.10|0.71|8006001|corpnameless #1|6|football|8|Sports|316|callyoughtpri|N/A|13699091201maroon478|smoke|Dozen|Unknown|23|barpribarableought| +12031|AAAAAAAAPPOCAAAA|1997-10-27||Likely systems should get there. Central, young years used to walk. Free wings cannot avoid rarely devices. Never unlike schools may go of cours|84.49|54.91|5001002|amalgscholar #2|1|rock|5|Music|579|n stationanti|N/A|3171875violet3216085|lace|Ounce|Unknown|33|oughtpribarableought| +12032|AAAAAAAAAAPCAAAA|1997-10-27|2000-10-26|Healthy animals find slim, significant workers. Possible, new games stir to a vegetables; copies ca|4.68|2.94|1004001|edu packamalg #1|4|swimwear|1|Women|285|antieingable|economy|1328881brown90298971|sienna|Cup|Unknown|53|ablepribarableought| +12033|AAAAAAAAAAPCAAAA|2000-10-27||Panels used to take accurate employers. Factors say certainly armed, domestic hands.|9.34|6.35|1004001|exportiedu pack #2|4|kids|4|Shoes|587|ationeinganti|large|1328881brown90298971|spring|Box|Unknown|25|pripribarableought| +12034|AAAAAAAACAPCAAAA|1997-10-27|1999-10-27|Skills would not see perhaps. Main horses shall not tell unions. Alongsid|2.98|1.37|3004001|edu packexporti #1|4|school-uniforms|3|Children|108|eingbarought|large|383128329309smoke605|slate|Tsp|Unknown|5|esepribarableought| +12035|AAAAAAAACAPCAAAA|1999-10-28|2001-10-26|Relationships take entirely i|5.95|3.21|3004001|amalgscholar #2|4|rock|5|Music|205|eingbarought|N/A|383128329309smoke605|goldenrod|Oz|Unknown|33|antipribarableought| +12036|AAAAAAAACAPCAAAA|2001-10-27||Financial, big hands find now possible, fine perceptions. Questions investigate in particular less than moral inches. Details date further proper|4.80|3.45|3004001|importoscholar #1|2|country|5|Music|524|eseableanti|N/A|212540rose0819186492|papaya|Tsp|Unknown|54|callypribarableought| +12037|AAAAAAAAFAPCAAAA|1997-10-27||Significant individuals should fill already dirty subj|3.74|2.76|1003002|exportiamalg #2|3|maternity|1|Women|50|baranti|medium|7087523papaya8245991|yellow|Pound|Unknown|12|ationpribarableought| +12038|AAAAAAAAGAPCAAAA|1997-10-27|2000-10-26|New days would consider officials. Only full ears consider also ot|43.14|31.49|1002001|importoamalg #1|2|fragrances|1|Women|200|barbarable|medium|575970879754sandy096|slate|Lb|Unknown|30|eingpribarableought| +12039|AAAAAAAAGAPCAAAA|2000-10-27||New days would consider officials. Only full ears consider also ot|9.03|4.06|1002001|exportischolar #2|2|pop|5|Music|200|barbarable|N/A|575970879754sandy096|white|Dram|Unknown|37|n stpribarableought| +12040|AAAAAAAAIAPCAAAA|1997-10-27|1999-10-27|Cases show there by the foods; periods|4.25|2.38|6010001|univbrand #1|10|jewelry boxes|6|Jewelry|817|ationoughteing|N/A|honeydew930906388172|royal|Carton|Unknown|31|baresebarableought| +12041|AAAAAAAAIAPCAAAA|1999-10-28|2001-10-26|Cases show there by the foods; periods|34.05|27.92|9014006|edu packunivamalg #6|10|sports|9|Books|817|ationoughteing|N/A|honeydew930906388172|royal|Gross|Unknown|95|oughtesebarableought| +12042|AAAAAAAAIAPCAAAA|2001-10-27||Walls should appear about between a weeks. Ministers could not explain times; members trade already all new others; parents could pursue methods; either financial changes would believe in|8.95|27.92|8003003|exportinameless #3|3|basketball|8|Sports|817|ationoughteing|N/A|17956purple894078815|mint|Gross|Unknown|6|ableesebarableought| +12043|AAAAAAAALAPCAAAA|1997-10-27||Police improve here profe|3.37|1.24|7010004|univnameless #4|10|flatware|7|Home|29|n stable|N/A|594234turquoise28350|snow|Gram|Unknown|27|priesebarableought| +12044|AAAAAAAAMAPCAAAA|1997-10-27|2000-10-26|Months would hurt national, new values. Regional, revolutionary advances make decisions. Days die almost; developments like traditional difficulties. Then available |2.75|1.04|6008005|namelesscorp #5|8|mens watch|6|Jewelry|72|ableation|N/A|1593salmon7599086339|pale|Tsp|Unknown|40|eseesebarableought| +12045|AAAAAAAAMAPCAAAA|2000-10-27||Eventually safe requirements might not win male years|0.55|1.04|10001010|amalgunivamalg #10|8|cameras|10|Electronics|72|ableation|N/A|403640908spring61127|powder|Oz|Unknown|2|antiesebarableought| +12046|AAAAAAAAOAPCAAAA|1997-10-27|1999-10-27|Very offers isolate also long runs. Police find now new newspapers. Types ought to base there national |4.89|2.24|7014007|edu packnameless #7|14|glassware|7|Home|592|ablen stanti|N/A|47391saddle886356087|blanched|Case|Unknown|79|callyesebarableought| +12047|AAAAAAAAOAPCAAAA|1999-10-28|2001-10-26|Very offers isolate also long runs. Police find now new newspapers. Types ought to base there national |2.12|1.88|6012008|importobrand #8|14|costume|6|Jewelry|592|ablen stanti|N/A|light500359335919280|powder|Ounce|Unknown|9|ationesebarableought| +12048|AAAAAAAAOAPCAAAA|2001-10-27||Very offers isolate also long runs. Police find now new newspapers. Types ought to base there national |7.23|6.36|6012008|univunivamalg #1|10|travel|9|Books|592|ablen stanti|N/A|1960666085837hot1164|smoke|Bundle|Unknown|18|eingesebarableought| +12049|AAAAAAAABBPCAAAA|1997-10-27||As national firms ought to make shortly virtually true attempts; now unlike views mu|3.32|2.98|5001002|amalgscholar #2|1|rock|5|Music|202|ablebarable|N/A|8800518slate65534364|seashell|Bunch|Unknown|46|n stesebarableought| +12050|AAAAAAAACBPCAAAA|1997-10-27|2000-10-26|High benefits justify still |2.51|1.07|6009003|maxicorp #3|9|womens watch|6|Jewelry|568|eingcallyanti|N/A|papaya06456630652807|puff|Each|Unknown|10|barantibarableought| +12051|AAAAAAAACBPCAAAA|2000-10-27||High benefits justify still |9.36|5.05|6009003|edu packunivamalg #16|9|audio|10|Electronics|568|eingcallyanti|N/A|papaya06456630652807|papaya|Gross|Unknown|60|oughtantibarableought| +12052|AAAAAAAAEBPCAAAA|1997-10-27|1999-10-27|Opportunities might include both cri|3.13|2.09|5003001|exportischolar #1|3|pop|5|Music|152|ableantiought|N/A|175181023puff0734405|antique|Case|Unknown|3|ableantibarableought| +12053|AAAAAAAAEBPCAAAA|1999-10-28|2001-10-26|Opportunities might include both cri|1.05|0.75|9003006|exportimaxi #6|3|computers|9|Books|152|ableantiought|N/A|175181023puff0734405|puff|Dram|Unknown|2|priantibarableought| +12054|AAAAAAAAEBPCAAAA|2001-10-27||Key, main years produce here good years. Houses would not know locally only social levels; old commentators meet sole initiatives. Girls may |5.97|0.75|7004005|edu packbrand #5|4|curtains/drapes|7|Home|152|ableantiought|N/A|8582303purple9704411|honeydew|Unknown|Unknown|53|eseantibarableought| +12055|AAAAAAAAHBPCAAAA|1997-10-27||Now other years adhere early. Pieces would not claim yet. Negative, clear police would not stay more special |5.03|3.52|4004002|edu packedu pack #2|4|athletic|4|Shoes|290|barn stable|large|385green066238083130|powder|Pound|Unknown|10|antiantibarableought| +12056|AAAAAAAAIBPCAAAA|1997-10-27|2000-10-26|||0.12|8002007||2|baseball|8||240||N/A||goldenrod||Unknown|37|| +12057|AAAAAAAAIBPCAAAA|2000-10-27||Economic, profitable applications might complete|1.32|0.62|4004002|edu packedu pack #2|4|athletic|4|Shoes|485|bareseable|large|73316017586sky902159|saddle|Case|Unknown|44|ationantibarableought| +12058|AAAAAAAAKBPCAAAA|1997-10-27|1999-10-27|Round, open details put laboratories. Essential eyes see as again small opponents; ever sophisticated products congratulate also as great changes. Also young agents locate almost by a affairs. E|8.45|4.64|9009011|maximaxi #11|9|science|9|Books|781|oughteingation|N/A|832195660912puff8003|royal|Box|Unknown|74|eingantibarableought| +12059|AAAAAAAAKBPCAAAA|1999-10-28|2001-10-26|Other, public defendants ask so physic|9.39|4.60|2001002|amalgimporto #2|1|accessories|2|Men|781|oughteingation|petite|832195660912puff8003|medium|Gram|Unknown|26|n stantibarableought| +12060|AAAAAAAAKBPCAAAA|2001-10-27||Minutes become thus plain numbers. Officials use genuinely places; local miles should make open, chronic lives. Soph|8.67|2.94|7005001|scholarbrand #1|1|blinds/shades|7|Home|547|oughteingation|N/A|832195660912puff8003|cyan|Carton|Unknown|84|barcallybarableought| +12061|AAAAAAAANBPCAAAA|1997-10-27||Best possible ages tell together new, st|0.87|0.38|1001002|amalgamalg #2|1|dresses|1|Women|870|barationeing|medium|72sky020067680376633|sienna|Bundle|Unknown|88|oughtcallybarableought| +12062|AAAAAAAAOBPCAAAA|1997-10-27|2000-10-26|Either subject shops can provide well just eastern attitudes. Minutes get often in a teachers. Years would achieve pieces; illegal, new |9.07|5.53|2002001|importoimporto #1|2|shirts|2|Men|630|barprically|medium|0thistle891279505484|sky|Dram|Unknown|49|ablecallybarableought| +12063|AAAAAAAAOBPCAAAA|2000-10-27||International, labour links should record largely new seconds. Dogs cannot handle actions. Buildings prove lively conditions; miles hear completely. Underground likely years may relax at a servants. O|3.94|1.65|2002001|edu packscholar #2|2|classical|5|Music|506|barprically|N/A|5saddle6916951880438|royal|Oz|Unknown|5|pricallybarableought| +12064|AAAAAAAAACPCAAAA|1997-10-27|1999-10-27|Extended, local books calm now likely companies. Sometime rich instances improve spanish countries. Crucial flames take further. Rapidly big proposals may not photograph in the opt|0.55|0.19|7008005|namelessbrand #5|8|lighting|7|Home|268|eingcallyable|N/A|2009345822turquoise3|khaki|Dozen|Unknown|64|esecallybarableought| +12065|AAAAAAAAACPCAAAA|1999-10-28|2001-10-26|Industrial parties could find respective, huge leaders. Much serious indicators murder just political, various drugs. Involved, established users m|4.19|0.19|7008005|importoimporto #2|8|shirts|2|Men|268|eingcallyable|medium|2009345822turquoise3|magenta|Bundle|Unknown|15|anticallybarableought| +12066|AAAAAAAAACPCAAAA|2001-10-27||Industrial parties could find respective, huge leaders. Much serious indicators murder just political, various drugs. Involved, established users m|6.91|0.19|7008005|namelesscorp #3|8|mens watch|6|Jewelry|268|eingcallyable|N/A|23882412snow22897763|chocolate|Bunch|Unknown|92|callycallybarableought| +12067|AAAAAAAADCPCAAAA|1997-10-27||Men must|1.07|0.72|7004008|edu packbrand #8|4|curtains/drapes|7|Home|540|bareseanti|N/A|245710087steel613379|papaya|Carton|Unknown|53|ationcallybarableought| +12068|AAAAAAAAECPCAAAA|1997-10-27|2000-10-26|Associated teams ought to experienc|0.30|0.19|6003005|exporticorp #5|3|gold|6|Jewelry|9|n st|N/A|534puff9672308965410|navajo|Carton|Unknown|13|eingcallybarableought| +12069|AAAAAAAAECPCAAAA|2000-10-27||Consequ|6.25|4.56|6003005|amalgmaxi #10|3|archery|8|Sports|9|n st|N/A|7363493994015247tan8|yellow|Pound|Unknown|65|n stcallybarableought| +12070|AAAAAAAAGCPCAAAA|1997-10-27|1999-10-27|Vice versa good rights could bring most bones; even big companies should run probably chronic children. Yet |2.04|1.20|4002001|importoedu pack #1|2|mens|4|Shoes|641|oughtesecally|petite|93honeydew9439240197|slate|Lb|Unknown|26|barationbarableought| +12071|AAAAAAAAGCPCAAAA|1999-10-28|2001-10-26|Machines treat cool forces. Actu|19.57|12.72|10010002|univamalgamalg #2|10|memory|10|Electronics|147|oughtesecally|N/A|93honeydew9439240197|rose|Oz|Unknown|18|oughtationbarableought| +12072|AAAAAAAAGCPCAAAA|2001-10-27||Machines treat cool forces. Actu|8.97|3.67|10010002|amalgcorp #1|1|birdal|6|Jewelry|630|barprically|N/A|93honeydew9439240197|yellow|Ton|Unknown|63|ableationbarableought| +12073|AAAAAAAAJCPCAAAA|1997-10-27||Personal, royal functions should not access american hor|5.84|4.96|6001002|amalgcorp #2|1|birdal|6|Jewelry|687|ationeingcally|N/A|014523786rosy5874345|smoke|Dram|Unknown|10|priationbarableought| +12074|AAAAAAAAKCPCAAAA|1997-10-27|2000-10-26|Far active components must appear now panels; other, public circumstances would not yield. New |0.28|0.10|3003001|exportiexporti #1|3|toddlers|3|Children|547|ationeseanti|medium|0948ivory00875899086|indian|Tbl|Unknown|2|eseationbarableought| +12075|AAAAAAAAKCPCAAAA|2000-10-27||Far active components must appear now panels; other, public circumstances would not yield. New |6.85|0.10|4003002|exportiedu pack #2|3|kids|4|Shoes|547|ationeseanti|medium|0948ivory00875899086|snow|Dozen|Unknown|43|antiationbarableought| +12076|AAAAAAAAMCPCAAAA|1997-10-27|1999-10-27|Isolated times need everywhere uncer|1.65|0.51|7004007|edu packbrand #7|4|curtains/drapes|7|Home|455|antiantiese|N/A|violet15713939027344|powder|Pallet|Unknown|1|callyationbarableought| +12077|AAAAAAAAMCPCAAAA|1999-10-28|2001-10-26|About social experiments open genuinely with the activities. Wonderful cases might|3.20|0.51|7011008|amalgnameless #8|11|accent|7|Home|455|antiantiese|N/A|4478966powder6133666|peach|Tsp|Unknown|43|ationationbarableought| +12078|AAAAAAAAMCPCAAAA|2001-10-27||Socially essential vers|3.29|0.51|3002001|importoexporti #1|11|infants|3|Children|455|antiantiese|small|8magenta586672963383|misty|Bunch|Unknown|31|eingationbarableought| +12079|AAAAAAAAPCPCAAAA|1997-10-27||Systems read too |32.09|15.40|2003002|exportiimporto #2|3|pants|2|Men|459|n stantiese|medium|2381567saddle7819308|light|Gram|Unknown|21|n stationbarableought| +12080|AAAAAAAAADPCAAAA|1997-10-27|2000-10-26|Now old police come usually strong genes. Equally residential elections see from a articles. Members observe i|4.24|2.54|5001001|amalgscholar #1|1|rock|5|Music|91|oughtn st|N/A|36seashell1708719217|rosy|Bundle|Unknown|60|bareingbarableought| +12081|AAAAAAAAADPCAAAA|2000-10-27||Now old police come usually strong genes. Equally residential elections see from a articles. Members observe i|9.01|2.54|5001001|importonameless #2|2|baseball|8|Sports|129|oughtn st|N/A|36seashell1708719217|peach|N/A|Unknown|60|oughteingbarableought| +12082|AAAAAAAACDPCAAAA|1997-10-27|1999-10-27|Basic fingers vote even stupid notes. Black, electrical rates may swim evident things. |1.79|1.03|8011003|amalgmaxi #3|11|archery|8|Sports|683|prieingcally|N/A|86628484papaya688692|magenta|Oz|Unknown|8|ableeingbarableought| +12083|AAAAAAAACDPCAAAA|1999-10-28|2001-10-26|Accessible, real efforts help less matters. Words might not explore limits. Creditors might meet. High, suitable hands derive already applications; other adults get much before |1.62|1.03|4003002|exportiedu pack #2|11|kids|4|Shoes|400|barbarese|petite|778018406sandy078048|tomato|Ounce|Unknown|24|prieingbarableought| +12084|AAAAAAAACDPCAAAA|2001-10-27||Accessible, real efforts help less matters. Words might not explore limits. Creditors might meet. High, suitable hands derive already applications; other adults get much before |9.05|1.03|4003002|edu packunivamalg #15|4|audio|10|Electronics|702|ablebaration|N/A|778018406sandy078048|saddle|Dozen|Unknown|36|eseeingbarableought| +12085|AAAAAAAAFDPCAAAA|1997-10-27||Different, particular ideas call now. Kilometres take rapid, good camps. Final argu|1.91|0.63|6015006|scholarbrand #6|15|custom|6|Jewelry|649|n stesecally|N/A|5346288451sandy07528|indian|Tbl|Unknown|89|antieingbarableought| +12086|AAAAAAAAGDPCAAAA|1997-10-27|2000-10-26|Significant birds mean somet|8.79|2.63|2004001|edu packimporto #1|4|sports-apparel|2|Men|235|antipriable|small|7669013887946salmon5|red|Unknown|Unknown|22|callyeingbarableought| +12087|AAAAAAAAGDPCAAAA|2000-10-27||Cultural, available parents help. Bad, natural months live agencies. Questions would want actually virtually free things. Labour, british s|1.73|1.43|9012004|importounivamalg #4|12|home repair|9|Books|808|antipriable|N/A|001526drab0949931695|royal|Dozen|Unknown|26|ationeingbarableought| +12088|AAAAAAAAIDPCAAAA|1997-10-27|1999-10-27|Social sources work hitherto likely, tight experiences. Low childr|3.29|2.36|1003001|exportiamalg #1|3|maternity|1|Women|170|barationought|small|01255343296338pink98|red|Dozen|Unknown|42|eingeingbarableought| +12089|AAAAAAAAIDPCAAAA|1999-10-28|2001-10-26|Social sources work hitherto likely, tight experiences. Low childr|5.99|4.25|3003002|exportiexporti #2|3|toddlers|3|Children|170|barationought|extra large|01255343296338pink98|pink|Lb|Unknown|27|n steingbarableought| +12090|AAAAAAAAIDPCAAAA|2001-10-27||Trends develop for the children. Controls will sense in a leaders. Fair, other years must appear by a pounds. Old, impressive activities will delete easy,|5.12|3.84|3003002|edu packexporti #1|4|school-uniforms|3|Children|610|barationought|medium|1772saddle2854721548|peru|N/A|Unknown|9|barn stbarableought| +12091|AAAAAAAALDPCAAAA|1997-10-27||Sounds will not think especially. Just clinical patients shall locate as labou|4.79|2.05|10014005|edu packamalgamalg #5|14|automotive|10|Electronics|87|ationeing|N/A|17royal6932642470464|slate|N/A|Unknown|4|oughtn stbarableought| +12092|AAAAAAAAMDPCAAAA|1997-10-27|2000-10-26|Later great problems should not go about on a countries; again natural payments ought to gain more actual artists. Tools get additional angles. Na|9.00|4.05|4001001|amalgedu pack #1|1|womens|4|Shoes|287|ationeingable|large|32olive8566924317144|olive|Gross|Unknown|40|ablen stbarableought| +12093|AAAAAAAAMDPCAAAA|2000-10-27||Later great problems should not go about on a countries; again natural payments ought to gain more actual artists. Tools get additional angles. Na|9.37|8.33|8005004|scholarnameless #4|5|fitness|8|Sports|662|ablecallycally|N/A|32olive8566924317144|steel|Ton|Unknown|29|prin stbarableought| +12094|AAAAAAAAODPCAAAA|1997-10-27|1999-10-27|Useful restrictions recognize thoroughly technical powers; lively years might tell di|9.30|8.18|5004001|edu packscholar #1|4|classical|5|Music|657|ationantically|N/A|74496768chartreuse34|saddle|Each|Unknown|50|esen stbarableought| +12095|AAAAAAAAODPCAAAA|1999-10-28|2001-10-26|Useful restrictions recognize thoroughly technical powers; lively years might tell di|3.08|1.72|5004001|edu packamalg #2|4|swimwear|1|Women|748|eingeseation|medium|74496768chartreuse34|saddle|Tsp|Unknown|38|antin stbarableought| +12096|AAAAAAAAODPCAAAA|2001-10-27||Useful restrictions recognize thoroughly technical powers; lively years might tell di|98.63|1.72|5004001|amalgnameless #7|11|accent|7|Home|738|eingeseation|N/A|74496768chartreuse34|smoke|Ounce|Unknown|95|callyn stbarableought| +12097|AAAAAAAABEPCAAAA|1997-10-27||Good, white children shall know also prime creatures. Big pockets take; often coming stands notice substantially warm parents. Small points sha|8.09|3.80|8013008|exportimaxi #8|13|sailing|8|Sports|437|ationpriese|N/A|410hot72071704826062|pale|Bundle|Unknown|5|ationn stbarableought| +12098|AAAAAAAACEPCAAAA|1997-10-27|2000-10-26|Public, limited pup|9.38|3.93|8011009|amalgmaxi #9|11|archery|8|Sports|906|callybarn st|N/A|72686079222wheat5336|spring|Lb|Unknown|33|eingn stbarableought| +12099|AAAAAAAACEPCAAAA|2000-10-27||Public, limited pup|4.32|3.93|4004002|edu packedu pack #2|11|athletic|4|Shoes|13|callybarn st|petite|72686079222wheat5336|saddle|Unknown|Unknown|10|n stn stbarableought| +12100|AAAAAAAAEEPCAAAA|1997-10-27|1999-10-27|Yellow years wish basic, difficult variations. Main, ultimate games used to find systems. Weeks could not indicate commer|0.30|0.13|10011014|amalgamalgamalg #14|11|disk drives|10|Electronics|132|ablepriought|N/A|4670149peru626330100|spring|Pallet|Unknown|39|barbaroughtableought| +12101|AAAAAAAAEEPCAAAA|1999-10-28|2001-10-26|Sums could not prove; both national years come. Swiftly major doctors move perhaps in a seeds. Sure months used to use; usually fatal interests contr|0.78|0.13|10011014|exportimaxi #2|11|sailing|8|Sports|526|ablepriought|N/A|2158086tan3441949216|tomato|Dozen|Unknown|38|oughtbaroughtableought| +12102|AAAAAAAAEEPCAAAA|2001-10-27||Late women linger speeches. Undoubtedly only words shall give in a risks. Police shall move so very|9.99|0.13|10011014|amalgmaxi #3|11|archery|8|Sports|463|pricallyese|N/A|2158086tan3441949216|turquoise|Dram|Unknown|5|ablebaroughtableought| +12103|AAAAAAAAHEPCAAAA|1997-10-27||Major skills mean below powerful men|0.52|0.30|6001008|amalgcorp #8|1|birdal|6|Jewelry|302|ablebarpri|N/A|09291974828pink15479|turquoise|Carton|Unknown|50|pribaroughtableought| +12104|AAAAAAAAIEPCAAAA|1997-10-27|2000-10-26|New, popular years should think. Shareholders speak also friends; special members could not identify social eyes; indoors full |0.91|0.35|9001009|amalgmaxi #9|1|arts|9|Books|43|priese|N/A|7619364986royal81046|sky|Box|Unknown|55|esebaroughtableought| +12105|AAAAAAAAIEPCAAAA|2000-10-27||Today informal tonnes hope well fixed, small studies. Other, only feet would agree yesterday key, other opportunities. General spirits should state; adult, upper laws may not mee|5.76|2.07|9001009|maxicorp #2|1|womens watch|6|Jewelry|43|priese|N/A|7619364986royal81046|peru|Tbl|Unknown|53|antibaroughtableought| +12106|AAAAAAAAKEPCAAAA|1997-10-27|1999-10-27|Practical stations admit increasingly. Pr|1.53|1.26|8016007|corpmaxi #7|16|golf|8|Sports|657|ationantically|N/A|8538seashell23997652|sky|N/A|Unknown|44|callybaroughtableought| +12107|AAAAAAAAKEPCAAAA|1999-10-28|2001-10-26|Practical stations admit increasingly. Pr|1.44|1.26|3002002|importoexporti #2|2|infants|3|Children|657|ationantically|extra large|8538seashell23997652|forest|Bunch|Unknown|18|ationbaroughtableought| +12108|AAAAAAAAKEPCAAAA|2001-10-27||Low, male regions embrace |2.66|0.93|3002002|edu packmaxi #1|2|entertainments|9|Books|56|callyanti|N/A|964700617545sandy208|orchid|Ton|Unknown|37|eingbaroughtableought| +12109|AAAAAAAANEPCAAAA|1997-10-27||Necessary, small streets shall share. Almost sure places can cover right human, tiny d|31.26|18.75|5001002|amalgscholar #2|1|rock|5|Music|161|oughtcallyought|N/A|67735260201steel3430|sienna|Dozen|Unknown|12|n stbaroughtableought| +12110|AAAAAAAAOEPCAAAA|1997-10-27|2000-10-26|Other, good contents make now villages; databases ask at least current, grateful windows. Remarks tolerate directly by a colours. Around economic leaders find statistical, like|2.07|1.57|10014007|edu packamalgamalg #7|14|automotive|10|Electronics|257|ationantiable|N/A|spring88499212542374|olive|Pound|Unknown|92|baroughtoughtableought| +12111|AAAAAAAAOEPCAAAA|2000-10-27||Other, good contents make now villages; databases ask at least current, grateful windows. Remarks tolerate directly by a colours. Around economic leaders find statistical, like|1.75|1.34|4001002|amalgedu pack #2|1|womens|4|Shoes|364|ationantiable|medium|spring88499212542374|pink|Ounce|Unknown|18|oughtoughtoughtableought| +12112|AAAAAAAAAFPCAAAA|1997-10-27|1999-10-27|Key institutions could not sell happy, general legs; constant, exclusive armies fend. Different, rele|1.28|0.52|4003001|exportiedu pack #1|3|kids|4|Shoes|601|oughtbarcally|extra large|7356984462759smoke31|peru|Case|Unknown|34|ableoughtoughtableought| +12113|AAAAAAAAAFPCAAAA|1999-10-28|2001-10-26|Fine, minute issues would not speculate parts. Effective guidelines reduce white, fine farmers. Occasions tell. More clear methods stop at |2.29|1.25|1003002|exportiamalg #2|3|maternity|1|Women|601|oughtbarcally|petite|01burlywood885759258|hot|Carton|Unknown|5|prioughtoughtableought| +12114|AAAAAAAAAFPCAAAA|2001-10-27||New, foreign comments act. Over total governments suggest then hands. European doct|0.70|1.25|1003002|amalgamalg #1|1|dresses|1|Women|278|oughtbarcally|large|01burlywood885759258|saddle|Pound|Unknown|54|eseoughtoughtableought| +12115|AAAAAAAADFPCAAAA|1997-10-27||Slightly radical futures lose national properties. Never overseas shares can summon files. He|9.42|8.19|10003012|exportiunivamalg #12|3|dvd/vcr players|10|Electronics|124|eseableought|N/A|066981ghost352954787|papaya|Ton|Unknown|16|antioughtoughtableought| +12116|AAAAAAAAEFPCAAAA|1997-10-27|2000-10-26|Open skills might end together fundamental, relevant details. Final options shall thank to a relationships. Free, rural months remain. Tradit|9.59|4.41|5004001|edu packscholar #1|4|classical|5|Music|791|oughtn station|N/A|42924216993666hot847|maroon|Ton|Unknown|20|callyoughtoughtableought| +12117|AAAAAAAAEFPCAAAA|2000-10-27||Religious, personal terms should get quite fully young forests. Finally brilliant activities will participate in a duties. |4.97|1.88|5004001|brandcorp #6|4|pendants|6|Jewelry|791|oughtn station|N/A|42924216993666hot847|white|Carton|Unknown|31|ationoughtoughtableought| +12118|AAAAAAAAGFPCAAAA|1997-10-27|1999-10-27|Figures function already white lines. Of c|9.40|6.67|10005015|scholarunivamalg #15|5|karoke|10|Electronics|19|n stought|N/A|seashell906964248738|pale|Pallet|Unknown|24|eingoughtoughtableought| +12119|AAAAAAAAGFPCAAAA|1999-10-28|2001-10-26|Dead, wide pp. correspond so as national areas. Presidential, mixed times will not happen possible, operational values. Once scientific houses might start both pri|16.26|13.33|10005015|corpmaxi #10|5|golf|8|Sports|508|n stought|N/A|seashell906964248738|white|Tsp|Unknown|13|n stoughtoughtableought| +12120|AAAAAAAAGFPCAAAA|2001-10-27||British actors stay for the features. Awful, small cases may learn openly good studies. Available employees must not complet|4.34|13.33|10005017|scholarunivamalg #17|5|karoke|10|Electronics|121|n stought|N/A|saddle62500864668938|tomato|Gross|Unknown|38|barableoughtableought| +12121|AAAAAAAAJFPCAAAA|1997-10-27||Proposals need political, major months|10.83|6.49|10015001|scholaramalgamalg #1|15|portable|10|Electronics|222|ableableable|N/A|70448661940lime95498|sky|Dram|Unknown|31|oughtableoughtableought| +12122|AAAAAAAAKFPCAAAA|1997-10-27|2000-10-26|Specific, irish features introduce even here obvious ranks. Essential, superb roads will extract; financial newspapers know professional, blu|3.57|2.78|8011003|amalgmaxi #3|11|archery|8|Sports|807|ationbareing|N/A|24624597ghost0098101|salmon|Bundle|Unknown|88|ableableoughtableought| +12123|AAAAAAAAKFPCAAAA|2000-10-27||Specific, irish features introduce even here obvious ranks. Essential, superb roads will extract; financial newspapers know professional, blu|8.98|2.78|4004002|edu packedu pack #2|4|athletic|4|Shoes|807|ationbareing|small|24624597ghost0098101|misty|Unknown|Unknown|63|priableoughtableought| +12124|AAAAAAAAMFPCAAAA|1997-10-27|1999-10-27|Speakers could catch. Other, different branches will cut specifically |0.32|0.21|7011005|amalgnameless #5|11|accent|7|Home|195|antin stought|N/A|8276687806465pale177|sienna|Each|Unknown|37|eseableoughtableought| +12125|AAAAAAAAMFPCAAAA|1999-10-28|2001-10-26|Speakers could catch. Other, different branches will cut specifically |0.61|0.21|8016006|corpmaxi #6|11|golf|8|Sports|249|n steseable|N/A|8276687806465pale177|smoke|Dozen|Unknown|13|antiableoughtableought| +12126|AAAAAAAAMFPCAAAA|2001-10-27||Problems might turn on a christians. Books shall campaign western, huge procedures. Patterns keep fi|9.91|6.44|8016006|exportiexporti #1|11|toddlers|3|Children|249|n steseable|medium|4wheat61170272871585|snow|Bunch|Unknown|17|callyableoughtableought| +12127|AAAAAAAAPFPCAAAA|1997-10-27||Sur|4.10|1.23|10013007|exportiamalgamalg #7|13|stereo|10|Electronics|635|antiprically|N/A|66651421795papaya152|puff|Box|Unknown|43|ationableoughtableought| +12128|AAAAAAAAAGPCAAAA|1997-10-27|2000-10-26|Cool stones shall not occur sometimes by a problems. Clearly opposite criteria could grow probably b|9.04|4.70|8005009|scholarnameless #9|5|fitness|8|Sports|496|callyn stese|N/A|703015421697933rosy2|grey|Cup|Unknown|12|eingableoughtableought| +12129|AAAAAAAAAGPCAAAA|2000-10-27||About wide pp|1.13|4.70|8005009|edu packscholar #2|4|classical|5|Music|246|callyn stese|N/A|703015421697933rosy2|white|Lb|Unknown|13|n stableoughtableought| +12130|AAAAAAAACGPCAAAA|1997-10-27|1999-10-27|Used, ordinary gardens ought to save then about a hundred|67.00|56.95|10009010|maxiunivamalg #10|9|televisions|10|Electronics|996|callyn stn st|N/A|9159264rosy425252075|yellow|Ounce|Unknown|18|barprioughtableought| +12131|AAAAAAAACGPCAAAA|1999-10-28|2001-10-26|Financial, american sums used to|0.73|56.95|10009010|maxiunivamalg #11|9|televisions|10|Electronics|996|callyn stn st|N/A|55510goldenrod806386|red|Lb|Unknown|14|oughtprioughtableought| +12132|AAAAAAAACGPCAAAA|2001-10-27||Financial, american sums used to|1.31|0.45|2001001|amalgimporto #1|9|accessories|2|Men|659|n stantically|large|7148honeydew33532542|thistle|Dozen|Unknown|70|ableprioughtableought| +12133|AAAAAAAAFGPCAAAA|1997-10-27||Heavily free banks used to prevent nearly single values. Suddenly green guests raise parents. Grey, able prices might not finance sta|3.10|1.39|10006013|corpunivamalg #13|6|musical|10|Electronics|533|priprianti|N/A|thistle8136689391775|sandy|Box|Unknown|22|priprioughtableought| +12134|AAAAAAAAGGPCAAAA|1997-10-27|2000-10-26|Capital agencies effect significant parents. Types ask illegal, small events. Deep, great reactions give arrangem|2.99|2.39|8009005|maxinameless #5|9|optics|8|Sports|184|eseeingought|N/A|2498605976ghost53198|slate|Tsp|Unknown|17|eseprioughtableought| +12135|AAAAAAAAGGPCAAAA|2000-10-27||Quickly slow members shall stand away courses; strict, capitalist eyes argue back, labour policies. Little corporate efforts would see hard times. Large, modern councillors will not move f|5.84|2.39|8009005|importoamalg #2|9|fragrances|1|Women|184|eseeingought|small|tomato90141973444639|metallic|Tsp|Unknown|3|antiprioughtableought| +12136|AAAAAAAAIGPCAAAA|1997-10-27|1999-10-27|Currently clear days reduce then stations. Inner, academic steps see at a facts. Old techniques see farmers; simply private men used to begin for the boots. Eas|0.66|0.32|8004007|edu packnameless #7|4|camping|8|Sports|577|ationationanti|N/A|155717ghost251408453|spring|Bunch|Unknown|19|callyprioughtableought| +12137|AAAAAAAAIGPCAAAA|1999-10-28|2001-10-26|Acutely good subsidies take hardly early terms. Merely magnificent areas help res|2.09|0.32|8012008|importomaxi #8|4|guns|8|Sports|577|ationationanti|N/A|155717ghost251408453|white|Oz|Unknown|27|ationprioughtableought| +12138|AAAAAAAAIGPCAAAA|2001-10-27||Acutely good subsidies take hardly early terms. Merely magnificent areas help res|4.38|1.48|8009009|maxinameless #9|4|optics|8|Sports|50|baranti|N/A|0610877white44398819|yellow|Carton|Unknown|64|eingprioughtableought| +12139|AAAAAAAALGPCAAAA|1997-10-27||Possibly small terms make. Busy appearances |6.94|4.71|3004002|edu packexporti #2|4|school-uniforms|3|Children|739|n stpriation|extra large|8620pink902903809700|salmon|N/A|Unknown|43|n stprioughtableought| +12140|AAAAAAAAMGPCAAAA|1997-10-27|2000-10-26|Wonderful, public offices might carry ordinary rivers; girls stay supreme hands; right wide forces afford too internationally impossible lovers. Fresh, social teeth grow. Other, permanent|1.47|1.01|9012009|importounivamalg #9|12|home repair|9|Books|99|n stn st|N/A|34804625red596792355|misty|Ounce|Unknown|22|bareseoughtableought| +12141|AAAAAAAAMGPCAAAA|2000-10-27||Wonderful, public offices might carry ordinary rivers; girls stay supreme hands; right wide forces afford too internationally impossible lovers. Fresh, social teeth grow. Other, permanent|8.47|5.75|4004002|edu packedu pack #2|4|athletic|4|Shoes|267|ationcallyable|medium|34804625red596792355|tomato|Box|Unknown|54|oughteseoughtableought| +12142|AAAAAAAAOGPCAAAA|1997-10-27|1999-10-27|Blind pools stand programmes. Everywhere old objects thi|1.08|0.62|9009011|maximaxi #11|9|science|9|Books|559|n stantianti|N/A|7056489honeydew46952|midnight|Tbl|Unknown|42|ableeseoughtableought| +12143|AAAAAAAAOGPCAAAA|1999-10-28|2001-10-26|Basic arms may include well. So large sections move; widely possible goods take probably females. Accurately able centuries experience rather for a regula|2.65|0.62|7016004|corpnameless #4|16|furniture|7|Home|559|n stantianti|N/A|royal715101731541895|snow|Dozen|Unknown|27|prieseoughtableought| +12144|AAAAAAAAOGPCAAAA|2001-10-27||Local pictures lead inside grim components|7.58|0.62|7016004|exportimaxi #1|3|computers|9|Books|559|n stantianti|N/A|royal715101731541895|khaki|Pallet|Unknown|14|eseeseoughtableought| +12145|AAAAAAAABHPCAAAA|1997-10-27||White, possible changes cost probably new, adult directions. All s|1.19|0.45|1002002|importoamalg #2|2|fragrances|1|Women|258|eingantiable|petite|44114162861rosy68906|peru|Dram|Unknown|18|antieseoughtableought| +12146|AAAAAAAACHPCAAAA|1997-10-27|2000-10-26|Long-term, wide studies shall take twice away conscious cases. Much primary objects would not remember also dead, english developments. Studies might|1.25|0.86|2001001|amalgimporto #1|1|accessories|2|Men|678|eingationcally|economy|8179royal74519079634|peru|Oz|Unknown|47|callyeseoughtableought| +12147|AAAAAAAACHPCAAAA|2000-10-27||Frequently guilty studies push later wide fathers. Prime, giant girls help per|2.73|0.86|1003002|exportiamalg #2|1|maternity|1|Women|811|oughtoughteing|medium|smoke506599708812976|azure|Ton|Unknown|5|ationeseoughtableought| +12148|AAAAAAAAEHPCAAAA|1997-10-27|1999-10-27|Surroundings lead offices. Red, technical employers shall phone english, formidable interests. Already other songs used to not|4.50|2.29|7016009|corpnameless #9|16|furniture|7|Home|326|callyablepri|N/A|3papaya3234935744076|khaki|Pallet|Unknown|48|eingeseoughtableought| +12149|AAAAAAAAEHPCAAAA|1999-10-28|2001-10-26|Beautifully high proceedings imitate in a matters. Political numbers ought to sell considerable, deep workers. Too traditional holes should p|8.47|6.86|7016009|amalgedu pack #2|16|womens|4|Shoes|326|callyablepri|medium|256072604178182navy2|sienna|Carton|Unknown|58|n steseoughtableought| +12150|AAAAAAAAEHPCAAAA|2001-10-27||Beautifully high proceedings imitate in a matters. Political numbers ought to sell considerable, deep workers. Too traditional holes should p|5.87|4.51|1002001|importoamalg #1|16|fragrances|1|Women|326|callyablepri|medium|379806gainsboro50969|snow|Cup|Unknown|90|barantioughtableought| +12151|AAAAAAAAHHPCAAAA|1997-10-27||Other, old students should sleep only only large differences. Grateful, equal cases set just. Trees manage. Other, early years see more simple miles. Mechanisms counter facts. About southern j|2.23|0.69|4001002|amalgedu pack #2|1|womens|4|Shoes|31|oughtpri|medium|4719679360992bisque8|slate|Ounce|Unknown|25|oughtantioughtableought| +12152|AAAAAAAAIHPCAAAA|1997-10-27|2000-10-26|Whole events will pull vice versa; individual, poor techniques cannot affect very; centres sell pounds. Particul|3.98|2.70|5002001|importoscholar #1|2|country|5|Music|113|prioughtought|N/A|0244360sienna4937327|gainsboro|Ounce|Unknown|28|ableantioughtableought| +12153|AAAAAAAAIHPCAAAA|2000-10-27||Whole events will pull vice versa; individual, poor techniques cannot affect very; centres sell pounds. Particul|7.51|2.70|5002002|importoscholar #2|2|country|5|Music|218|eingoughtable|N/A|095396095054sienna27|turquoise|Ton|Unknown|17|priantioughtableought| +12154|AAAAAAAAKHPCAAAA|1997-10-27|1999-10-27|Individual, careful memories produce. Heads go financial difficulties. Old governors search. Circumstances can say living res|6.68|5.67|1001001|amalgamalg #1|1|dresses|1|Women|564|esecallyanti|extra large|0018490732tomato5375|sky|Box|Unknown|38|eseantioughtableought| +12155|AAAAAAAAKHPCAAAA|1999-10-28|2001-10-26|Possible pla|5.35|5.67|1001001|importoscholar #2|1|country|5|Music|564|esecallyanti|N/A|965412hot28579620536|dodger|Carton|Unknown|16|antiantioughtableought| +12156|AAAAAAAAKHPCAAAA|2001-10-27||Home new times peer far new, private arguments; bad angles can|13.32|5.67|2004001|edu packimporto #1|4|sports-apparel|2|Men|564|esecallyanti|economy|965412hot28579620536|turquoise|Oz|Unknown|29|callyantioughtableought| +12157|AAAAAAAANHPCAAAA|1997-10-27||Even fashionable arms stop by a women. Workers used to clear main, crucial seasons. Processes protect for a problems. Slightly social effects shall sell. Globa|1.61|0.67|3001002|amalgexporti #2|1|newborn|3|Children|879|n stationeing|N/A|9lawn291974146231244|sky|Each|Unknown|22|ationantioughtableought| +12158|AAAAAAAAOHPCAAAA|1997-10-27|2000-10-26|Weeks ensure so old participants; altogether white relationships shall complete little more upper arms. Ideas know able chips. Loudly annual |8.39|4.95|10011004|amalgamalgamalg #4|11|disk drives|10|Electronics|589|n steinganti|N/A|75seashell8924962903|slate|Lb|Unknown|43|eingantioughtableought| +12159|AAAAAAAAOHPCAAAA|2000-10-27||Weeks ensure so old participants; altogether white relationships shall complete little more upper arms. Ideas know able chips. Loudly annual |4.86|4.95|4003002|exportiedu pack #2|11|kids|4|Shoes|782|n steinganti|petite|114614733yellow19556|white|Gross|Unknown|22|n stantioughtableought| +12160|AAAAAAAAAIPCAAAA|1997-10-27|1999-10-27|Free, substantial abilities prevent all right cultural trees. Most human scientists give only from a schools. Groups borrow then historical parts. Real, immediate scores could take however offices; |15.39|7.23|2002001|importoimporto #1|2|shirts|2|Men|171|oughtationought|large|1331723873676deep650|moccasin|Lb|Unknown|46|barcallyoughtableought| +12161|AAAAAAAAAIPCAAAA|1999-10-28|2001-10-26|Men ought to regain just. So as interested weaknesses must not allow elements. Early measures will not see hardly statistical times. Ago aware times might disting|25.41|7.23|2002001|edu packexporti #2|2|school-uniforms|3|Children|322|ableablepri|petite|63130607149573ghost4|snow|Unknown|Unknown|15|oughtcallyoughtableought| +12162|AAAAAAAAAIPCAAAA|2001-10-27||Men ought to regain just. So as interested weaknesses must not allow elements. Early measures will not see hardly statistical times. Ago aware times might disting|3.76|2.97|2002001|brandnameless #3|7|hockey|8|Sports|322|ableablepri|N/A|63130607149573ghost4|saddle|Box|Unknown|33|ablecallyoughtableought| +12163|AAAAAAAADIPCAAAA|1997-10-27||Children should incorporate nearly confident activities. Additional benefits will |0.41|0.19|8002004|importonameless #4|2|baseball|8|Sports|347|ationesepri|N/A|831923775midnight000|saddle|Gram|Unknown|62|pricallyoughtableought| +12164|AAAAAAAAEIPCAAAA|1997-10-27|2000-10-26|Boundaries believe on a needs. Respective, sharp agents would not develop western delegates; successful, biological employees wriggle early in a blocks. Policies adopt togethe|1.95|0.83|2002001|importoimporto #1|2|shirts|2|Men|46|callyese|medium|21643165656deep48033|magenta|Gross|Unknown|33|esecallyoughtableought| +12165|AAAAAAAAEIPCAAAA|2000-10-27||Boundaries believe on a needs. Respective, sharp agents would not develop western delegates; successful, biological employees wriggle early in a blocks. Policies adopt togethe|1.02|0.34|2002001|brandnameless #6|2|hockey|8|Sports|46|callyese|N/A|6328751264414misty05|saddle|Pound|Unknown|11|anticallyoughtableought| +12166|AAAAAAAAGIPCAAAA|1997-10-27|1999-10-27|Downstairs nation|1.33|0.91|10001012|amalgunivamalg #12|1|cameras|10|Electronics|133|pripriought|N/A|0115201papaya9498267|peach|Gross|Unknown|80|callycallyoughtableought| +12167|AAAAAAAAGIPCAAAA|1999-10-28|2001-10-26|Downstairs nation|9.86|0.91|1004002|edu packamalg #2|4|swimwear|1|Women|133|pripriought|large|0115201papaya9498267|indian|Oz|Unknown|30|ationcallyoughtableought| +12168|AAAAAAAAGIPCAAAA|2001-10-27||Downstairs nation|63.37|25.34|9010001|univunivamalg #1|10|travel|9|Books|727|ationableation|N/A|75714725741snow25037|violet|Cup|Unknown|32|eingcallyoughtableought| +12169|AAAAAAAAJIPCAAAA|1997-10-27||Running years could not need short british associations. Rich reports happen active, wa|0.83|0.36|2004002|edu packimporto #2|4|sports-apparel|2|Men|50|baranti|extra large|5870721tomato0552870|tan|Gross|Unknown|29|n stcallyoughtableought| +12170|AAAAAAAAKIPCAAAA|1997-10-27|2000-10-26|Classic changes may not feel yet much hungry videos. |0.79|0.69|3002001|importoexporti #1|2|infants|3|Children|153|priantiought|petite|30042pale12328799926|violet|Gross|Unknown|52|barationoughtableought| +12171|AAAAAAAAKIPCAAAA|2000-10-27||Officers give now large terms. Literary, new figures assume more possible savings. At least important sciences would not like important concent|5.43|0.69|8010002|univmaxi #2|2|pools|8|Sports|456|callyantiese|N/A|577456navy6001237482|peru|N/A|Unknown|96|oughtationoughtableought| +12172|AAAAAAAAMIPCAAAA|1997-10-27|1999-10-27|Still working aspects try further. Here empty students roll for the holes. Misleading reservations send. Other, main patients would |2.99|2.33|2001001|amalgimporto #1|1|accessories|2|Men|553|priantianti|extra large|dodger24450773468185|pink|Dozen|Unknown|17|ableationoughtableought| +12173|AAAAAAAAMIPCAAAA|1999-10-28|2001-10-26|Still working aspects try further. Here empty students roll for the holes. Misleading reservations send. Other, main patients would |3.90|1.91|2001001|exportiunivamalg #2|3|dvd/vcr players|10|Electronics|970|priantianti|N/A|69purple540351727094|wheat|Cup|Unknown|5|priationoughtableought| +12174|AAAAAAAAMIPCAAAA|2001-10-27||Still working aspects try further. Here empty students roll for the holes. Misleading reservations send. Other, main patients would |3.72|1.91|2001001|importoexporti #1|2|infants|3|Children|27|priantianti|N/A|69purple540351727094|olive|Ton|Unknown|20|eseationoughtableought| +12175|AAAAAAAAPIPCAAAA|1997-10-27||Private incentives tell. Measures enjoy then grey hands. |9.89|6.42|4003002|exportiedu pack #2|3|kids|4|Shoes|134|esepriought|petite|9steel85552259596638|seashell|Case|Unknown|49|antiationoughtableought| +12176|AAAAAAAAAJPCAAAA|1997-10-27|2000-10-26|Ugly, other authorities used to pay often inc teachers. Forward living accounts find therefore to a orders. Real, beautiful costs stimulate coldly safe spots. J|7.25|6.52|2004001|edu packimporto #1|4|sports-apparel|2|Men|216|callyoughtable|medium|030859449coral762343|papaya|Box|Unknown|16|callyationoughtableought| +12177|AAAAAAAAAJPCAAAA|2000-10-27||Ugly, other authorities used to pay often inc teachers. Forward living accounts find therefore to a orders. Real, beautiful costs stimulate coldly safe spots. J|6.18|3.70|2004001|amalgamalg #2|4|dresses|1|Women|62|ablecally|petite|030859449coral762343|steel|Pallet|Unknown|22|ationationoughtableought| +12178|AAAAAAAACJPCAAAA|1997-10-27|1999-10-27|So prepared eyes find somewhat; possible, add|1.08|0.79|2001001|amalgimporto #1|1|accessories|2|Men|169|n stcallyought|small|64628sandy5028805631|slate|Each|Unknown|38|eingationoughtableought| +12179|AAAAAAAACJPCAAAA|1999-10-28|2001-10-26|Almost disabled women should travel on a terms. Readers will win. Generations extend red arms. Soon large adults take important negotiations. Excellent problems know important, social videos. Acce|0.16|0.79|4002002|importoedu pack #2|1|mens|4|Shoes|169|n stcallyought|large|483forest99153879773|violet|Box|Unknown|29|n stationoughtableought| +12180|AAAAAAAACJPCAAAA|2001-10-27||English, severe messages dry usually hard drugs; final, universal lives must not establish about. External, single weeks know personal, international plac|7.61|4.56|9006001|corpmaxi #1|1|parenting|9|Books|255|n stcallyought|N/A|483forest99153879773|sky|Dozen|Unknown|55|bareingoughtableought| +12181|AAAAAAAAFJPCAAAA|1997-10-27||Less british services change together never considerable reasons. Feelings shall go later later old eyes|6.56|4.13|4002002|importoedu pack #2|2|mens|4|Shoes|366|callycallypri|medium|3medium8072289710594|peach|Gram|Unknown|22|oughteingoughtableought| +12182|AAAAAAAAGJPCAAAA|1997-10-27|2000-10-26|Also able components could not include eventually soldiers. Senior, subject skills go fragments. Ready|7.00|3.08|2002001|importoimporto #1|2|shirts|2|Men|56|callyanti|medium|4426193683809pale437|royal|Carton|Unknown|59|ableeingoughtableought| +12183|AAAAAAAAGJPCAAAA|2000-10-27||Players must allow here high, mass details; very british crops target hopefully responsibilities. However new children return yet. Much small videos see. Establi|4.43|3.08|2002001|edu packbrand #8|14|estate|6|Jewelry|234|callyanti|N/A|8royal55110324032551|orchid|Carton|Unknown|61|prieingoughtableought| +12184|AAAAAAAAIJPCAAAA|1997-10-27|1999-10-27|Thousands increase substantial women. All living representatives will reduce like a principles; much trying teachers replace only now |5.27|2.74|10015013|scholaramalgamalg #13|15|portable|10|Electronics|396|callyn stpri|N/A|867tomato67488528949|puff|Ton|Unknown|64|eseeingoughtableought| +12185|AAAAAAAAIJPCAAAA|1999-10-28|2001-10-26|Mammals win with a modes. Liberal, sensible ladies ought to walk appropriate others. Eyes continue quite annual dangers. Rates throw|0.54|0.45|10015013|importoedu pack #2|2|mens|4|Shoes|396|callyn stpri|small|867tomato67488528949|sky|Each|Unknown|24|antieingoughtableought| +12186|AAAAAAAAIJPCAAAA|2001-10-27||Mammals win with a modes. Liberal, sensible ladies ought to walk appropriate others. Eyes continue quite annual dangers. Rates throw|9.68|4.45|10015013|edu packscholar #1|2|classical|5|Music|396|callyn stpri|N/A|867tomato67488528949|thistle|Lb|Unknown|30|callyeingoughtableought| +12187|AAAAAAAALJPCAAAA|1997-10-27||Just awar|7.38|3.24|2003002|exportiimporto #2|3|pants|2|Men|121|oughtableought|medium|23798560tomato303501|slate|Pound|Unknown|61|ationeingoughtableought| +12188|AAAAAAAAMJPCAAAA|1997-10-27|2000-10-26|Sanctions eat. Deliberately mature camps let common, young banks. Apparently stable days deal very following flights. Players need empty, nuclear talks; far present affair|6.58|3.55|2002001|importoimporto #1|2|shirts|2|Men|15|antiought|petite|3702267746153ghost36|midnight|Box|Unknown|13|eingeingoughtableought| +12189|AAAAAAAAMJPCAAAA|2000-10-27||Sanctions eat. Deliberately mature camps let common, young banks. Apparently stable days deal very following flights. Players need empty, nuclear talks; far present affair|23.89|16.00|2002001|amalgamalg #2|1|dresses|1|Women|15|antiought|extra large|3702267746153ghost36|pale|Pallet|Unknown|99|n steingoughtableought| +12190|AAAAAAAAOJPCAAAA|1997-10-27|1999-10-27|Legal divisions ought to copy. Never dead agencies purchase widely|43.27|37.64|6010007|univbrand #7|10|jewelry boxes|6|Jewelry|651|oughtantically|N/A|snow0247559549105738|yellow|Case|Unknown|2|barn stoughtableought| +12191|AAAAAAAAOJPCAAAA|1999-10-28|2001-10-26|Legal divisions ought to copy. Never dead agencies purchase widely|2.00|0.96|6010007|importomaxi #12|2|business|9|Books|651|oughtantically|N/A|9072593857purple4977|turquoise|Oz|Unknown|89|oughtn stoughtableought| +12192|AAAAAAAAOJPCAAAA|2001-10-27||Available plans could eat a little precise, |3.73|0.96|6010007|maxinameless #3|9|optics|8|Sports|177|ationationought|N/A|9072593857purple4977|seashell|Oz|Unknown|9|ablen stoughtableought| +12193|AAAAAAAABKPCAAAA|1997-10-27||Difficult, normal mothers must know a|2.16|1.46|8011004|amalgmaxi #4|11|archery|8|Sports|184|eseeingought|N/A|64moccasin6839412755|turquoise|Gross|Unknown|80|prin stoughtableought| +12194|AAAAAAAACKPCAAAA|1997-10-27|2000-10-26|That is to say strong women mislead elsewhere social towns; p|4.09|1.22|3001001|amalgexporti #1|1|newborn|3|Children|258|eingantiable|extra large|7677234050green48678|rose|Gross|Unknown|44|esen stoughtableought| +12195|AAAAAAAACKPCAAAA|2000-10-27||Alternative kilometres fit also with the events. Therefore available years seem quite employees. As other users cast before styles. W|3.67|1.22|3001001|exportiimporto #2|3|pants|2|Men|258|eingantiable|small|7677234050green48678|royal|Carton|Unknown|27|antin stoughtableought| +12196|AAAAAAAAEKPCAAAA|1997-10-27|1999-10-27|Bad reasons beat. Cold, serious customers would make|1.83|1.20|3003001|exportiexporti #1|3|toddlers|3|Children|446|callyeseese|petite|94902111790khaki4982|spring|Dram|Unknown|85|callyn stoughtableought| +12197|AAAAAAAAEKPCAAAA|1999-10-28|2001-10-26|Available, total advances coul|38.63|24.72|3003001|importocorp #6|3|diamonds|6|Jewelry|175|callyeseese|N/A|94902111790khaki4982|spring|Carton|Unknown|10|ationn stoughtableought| +12198|AAAAAAAAEKPCAAAA|2001-10-27||Available, total advances coul|3.76|2.14|3003001|amalgunivamalg #10|1|cameras|10|Electronics|175|callyeseese|N/A|8puff548640094336012|seashell|N/A|Unknown|32|eingn stoughtableought| +12199|AAAAAAAAHKPCAAAA|1997-10-27||Talks send. Small questions ought to understand very more quiet weapons. Legislative, powerful matters should celebrate too young, recent parents. Dreams shall finish so misleading months|4.25|2.63|5003002|exportischolar #2|3|pop|5|Music|41|oughtese|N/A|3635173503744peach67|spring|Oz|Unknown|1|n stn stoughtableought| +12200|AAAAAAAAIKPCAAAA|1997-10-27|2000-10-26|Outside periods end never just certain steps; alr|2.91|2.00|8009001|maxinameless #1|9|optics|8|Sports|724|eseableation|N/A|01649ivory7159953036|khaki|N/A|Unknown|98|barbarableableought| +12201|AAAAAAAAIKPCAAAA|2000-10-27||Valid, past others must read traditionally. True effects wait in a jews. Direct, social stories phone only material, modern employees. Practical sciences decide now observations. Relevant|53.35|2.00|8009001|importounivamalg #13|2|camcorders|10|Electronics|697|eseableation|N/A|60858869sienna477479|yellow|Lb|Unknown|27|oughtbarableableought| +12202|AAAAAAAAKKPCAAAA|1997-10-27|1999-10-27|Countrie|72.84|28.40|6011003|amalgbrand #3|11|semi-precious|6|Jewelry|294|esen stable|N/A|62637444811822plum83|navy|Tsp|Unknown|56|ablebarableableought| +12203|AAAAAAAAKKPCAAAA|1999-10-28|2001-10-26|Countrie|8.17|28.40|6011003|importobrand #4|11|bedding|7|Home|294|esen stable|N/A|64861084548saddle155|rose|Each|Unknown|29|pribarableableought| +12204|AAAAAAAAKKPCAAAA|2001-10-27||All dutch seats ask systems; around wrong feet include scales; very potential years dust somehow to a studies; years cannot use. Proposed, good properties ought to bolster even as industrial fo|3.93|2.24|6011003|univbrand #5|11|jewelry boxes|6|Jewelry|91|esen stable|N/A|1salmon1817258077292|khaki|Tbl|Unknown|31|esebarableableought| +12205|AAAAAAAANKPCAAAA||||4.43|||exportiimporto #2|3|pants||||oughtpriese|||||Unknown|60|antibarableableought| +12206|AAAAAAAAOKPCAAAA|1997-10-27|2000-10-26|Weak, manufacturing trusts will sell increasingly sufficient views. Long ne|1.53|1.25|2002001|importoimporto #1|2|shirts|2|Men|153|priantiought|medium|10918203tan194505390|orchid|Lb|Unknown|17|callybarableableought| +12207|AAAAAAAAOKPCAAAA|2000-10-27||Weak, manufacturing trusts will sell increasingly sufficient views. Long ne|6.30|4.97|2002001|amalgedu pack #2|1|womens|4|Shoes|227|priantiought|medium|0484salmon5884366346|red|Case|Unknown|30|ationbarableableought| +12208|AAAAAAAAALPCAAAA|1997-10-27|1999-10-27|Disabled governors may not lift. Since disciplinary buildings serve carefully rapid papers. High, pol|9.76|5.95|10006003|corpunivamalg #3|6|musical|10|Electronics|517|ationoughtanti|N/A|1009476715581red1381|spring|Ton|Unknown|41|eingbarableableought| +12209|AAAAAAAAALPCAAAA|1999-10-28|2001-10-26|Disabled governors may not lift. Since disciplinary buildings serve carefully rapid papers. High, pol|2.39|1.21|10006003|edu packexporti #2|4|school-uniforms|3|Children|517|ationoughtanti|small|1009476715581red1381|gainsboro|Bundle|Unknown|88|n stbarableableought| +12210|AAAAAAAAALPCAAAA|2001-10-27||Disabled governors may not lift. Since disciplinary buildings serve carefully rapid papers. High, pol|8.91|4.09|10006003|edu packunivamalg #7|4|sports|9|Books|517|ationoughtanti|N/A|1009476715581red1381|papaya|Pound|Unknown|33|baroughtableableought| +12211|AAAAAAAADLPCAAAA|1997-10-27||Patient trains will happen even good, central steps. New equations will include by a exercises. Key, psychological deaths apply mainly also foreign bodies. Assistant, inap|9.95|7.76|9005008|scholarmaxi #8|5|history|9|Books|721|oughtableation|N/A|626294011snow0476514|plum|Gram|Unknown|5|oughtoughtableableought| +12212|AAAAAAAAELPCAAAA|1997-10-27|2000-10-26|Direct, american casualties practise alone locations. Sophisticated charges take little in a readers. Individual, social detectives admit similar|0.30|0.18|3003001|exportiexporti #1|3|toddlers|3|Children|281|oughteingable|petite|6586149rosy430829375|royal|Gram|Unknown|3|ableoughtableableought| +12213|AAAAAAAAELPCAAAA|2000-10-27||Inc questions may control also due, satisfied banks. Great terms depend knees; persons take to a girls; colleagues answer less much new problems. Individual, ruling judges post just arguments.|0.29|0.10|3001002|amalgexporti #2|1|newborn|3|Children|281|oughteingable|medium|7437942royal42584563|rosy|Pallet|Unknown|12|prioughtableableought| +12214|AAAAAAAAGLPCAAAA|1997-10-27|1999-10-27|British workers should not go to a ends. New models might divert frequently probable responses. Atomic, technical employees might prescribe for the forms; visual |3.56|2.27|4004001|edu packedu pack #1|4|athletic|4|Shoes|361|oughtcallypri|large|3tomato1843150855135|pink|Tbl|Unknown|26|eseoughtableableought| +12215|AAAAAAAAGLPCAAAA|1999-10-28|2001-10-26|British workers should not go to a ends. New models might divert frequently probable responses. Atomic, technical employees might prescribe for the forms; visual |10.15|8.52|4004001|exportiamalg #2|3|maternity|1|Women|361|oughtcallypri|small|8888sienna2729243165|chartreuse|N/A|Unknown|88|antioughtableableought| +12216|AAAAAAAAGLPCAAAA|2001-10-27||Kind women will land differently; certain, light eyes can make rare members; inches listen obvious, kee|8.66|6.06|4004001|edu packscholar #1|3|classical|5|Music|116|oughtcallypri|N/A|8888sienna2729243165|maroon|Dram|Unknown|11|callyoughtableableought| +12217|AAAAAAAAJLPCAAAA|1997-10-27||Magazines might double legitimately trees. Fields trust purposes. Silver lectures may feel then old sessions. Hence likely boys used to shape. Enormous, |5.85|2.80|5004002|edu packscholar #2|4|classical|5|Music|576|callyationanti|N/A|546373182937838hot78|chiffon|Bunch|Unknown|6|ationoughtableableought| +12218|AAAAAAAAKLPCAAAA|1997-10-27|2000-10-26|Pp. resign away determined, clear guests. Total agents see courts. Usually vocational things help for a contrac|88.40|60.99|10012013|importoamalgamalg #13|12|monitors|10|Electronics|86|callyeing|N/A|27684918sienna331976|turquoise|Lb|Unknown|54|eingoughtableableought| +12219|AAAAAAAAKLPCAAAA|2000-10-27||Pp. resign away determined, clear guests. Total agents see courts. Usually vocational things help for a contrac|3.51|2.59|9009004|maximaxi #4|12|science|9|Books|755|callyeing|N/A|27684918sienna331976|navy|Pallet|Unknown|28|n stoughtableableought| +12220|AAAAAAAAMLPCAAAA|1997-10-27|1999-10-27|Social, old governments impress on|9.68|6.67|10008015|namelessunivamalg #15|8|scanners|10|Electronics|532|ableprianti|N/A|white900635001944280|lavender|Tbl|Unknown|12|barableableableought| +12221|AAAAAAAAMLPCAAAA|1999-10-28|2001-10-26|Social, old governments impress on|4.53|6.67|2003002|exportiimporto #2|8|pants|2|Men|532|ableprianti|petite|4325901ivory67697694|royal|Dram|Unknown|6|oughtableableableought| +12222|AAAAAAAAMLPCAAAA|2001-10-27||Economic rows may not form patiently unique o|16.55|9.43|2003002|edu packimporto #1|8|sports-apparel|2|Men|196|callyn stought|medium|4325901ivory67697694|sandy|Box|Unknown|32|ableableableableought| +12223|AAAAAAAAPLPCAAAA|1997-10-27||Public, national years should use twice in a procedures. Rare members like also other difficulties. Suspicious, possible eyes will make needs. Terms know apart to the marks. Wrong, re|8.60|7.65|2003002|exportiimporto #2|3|pants|2|Men|330|barpripri|medium|2570451828995lemon26|orange|N/A|Unknown|36|priableableableought| +12224|AAAAAAAAAMPCAAAA|1997-10-27|2000-10-26|Hungry, junior hours must make reluctantly physical, local costs. Small criteria must obtain closely. Social events will fail accurate, useful roles. Great, ea|3.00|2.67|2002001|importoimporto #1|2|shirts|2|Men|634|eseprically|extra large|79037purple711467210|saddle|Lb|Unknown|1|eseableableableought| +12225|AAAAAAAAAMPCAAAA|2000-10-27||Seats find just yesterday other injuries. Very, basic roles turn now products; considerable, open mistakes lead frequent, immediate organisations. Natural children enter|75.14|60.86|2002001|brandbrand #6|7|decor|7|Home|163|pricallyought|N/A|79037purple711467210|tomato|Gram|Unknown|40|antiableableableought| +12226|AAAAAAAACMPCAAAA|1997-10-27|1999-10-27|Top chan|7.55|6.71|2004001|edu packimporto #1|4|sports-apparel|2|Men|644|eseesecally|small|897297346royal075210|purple|Tbl|Unknown|47|callyableableableought| +12227|AAAAAAAACMPCAAAA|1999-10-28|2001-10-26|Finally hard things cease enormous, serious examples; northern, conc|5.46|3.33|4004002|edu packedu pack #2|4|athletic|4|Shoes|644|eseesecally|small|67brown1818227295429|sienna|Tbl|Unknown|60|ationableableableought| +12228|AAAAAAAACMPCAAAA|2001-10-27||Finally hard things cease enormous, serious examples; northern, conc|1.89|0.68|4004002|exportimaxi #9|13|sailing|8|Sports|644|eseesecally|N/A|67brown1818227295429|lime|Gram|Unknown|32|eingableableableought| +12229|AAAAAAAAFMPCAAAA|1997-10-27||Considerable, sick years shall pay too adults. Other allegations make proposals. Industries report ne|19.47|6.42|3003002|exportiexporti #2|3|toddlers|3|Children|512|ableoughtanti|medium|wheat565742079818516|pale|Box|Unknown|39|n stableableableought| +12230|AAAAAAAAGMPCAAAA|1997-10-27|2000-10-26|Temporary calls will protect strong, specific police; full government|1.65|1.18|2002001|importoimporto #1|2|shirts|2|Men|441|oughteseese|petite|1papaya3113192527378|pale|Ton|Unknown|1|barpriableableought| +12231|AAAAAAAAGMPCAAAA|2000-10-27||Wrong miners may not turn so sma|91.02|46.42|2002001|edu packimporto #2|4|sports-apparel|2|Men|441|oughteseese|medium|4sky8160510007962381|lemon|Lb|Unknown|12|oughtpriableableought| +12232|AAAAAAAAIMPCAAAA|1997-10-27|1999-10-27|Heavy rates could build. Economic, other group|7.88|3.46|3002001|importoexporti #1|2|infants|3|Children|174|eseationought|N/A|4514905451238salmon3|rose|Dram|Unknown|57|ablepriableableought| +12233|AAAAAAAAIMPCAAAA|1999-10-28|2001-10-26|Heavy rates could build. Economic, other group|1.35|3.46|3002001|exportischolar #2|3|pop|5|Music|515|eseationought|N/A|3044234pink596354572|powder|Tbl|Unknown|18|pripriableableought| +12234|AAAAAAAAIMPCAAAA|2001-10-27||Unusual trends contain. Only, short-term days land. Groups obtain already officially existing minerals. Legal requirements say away black files; different ends would come european,|2.96|3.46|3002001|importobrand #5|2|bedding|7|Home|515|antioughtanti|N/A|tan86010201979377178|peru|Bundle|Unknown|36|esepriableableought| +12235|AAAAAAAALMPCAAAA|1997-10-27||Differences prevent so fiscal, bright years; relatively bare missiles date specifically now square patients. Sin|3.11|1.61|6010004|univbrand #4|10|jewelry boxes|6|Jewelry|223|priableable|N/A|072059846967842tan15|pink|Gram|Unknown|89|antipriableableought| +12236|AAAAAAAAMMPCAAAA|1997-10-27|2000-10-26|Documents add. Early individuals ought to remain now similarly great police. Governments must benefit towards the papers. International inhabitants curb thick in a voices. Local, commercial |7.86|5.10|2004001|edu packimporto #1|4|sports-apparel|2|Men|319|n stoughtpri|petite|7sandy32532111462631|grey|Gram|Unknown|24|callypriableableought| +12237|AAAAAAAAMMPCAAAA|2000-10-27||Presumably social groups detect marked, tough generations. Narrow weeks should tell almost wide values. Favorite waves will not use; so wide patients would deal later fr|2.44|2.02|2004001|brandmaxi #10|7|reference|9|Books|539|n stprianti|N/A|20273543slate8909487|rose|Dozen|Unknown|1|ationpriableableought| +12238|AAAAAAAAOMPCAAAA|1997-10-27|1999-10-27|Now complex carers must use here therefore personal arms. Ideas could gather weapons. Dif|3.56|1.38|9003011|exportimaxi #11|3|computers|9|Books|120|barableought|N/A|1887plum806436146913|powder|Ton|Unknown|43|eingpriableableought| +12239|AAAAAAAAOMPCAAAA|1999-10-28|2001-10-26|Local,|2.22|1.38|10016017|corpamalgamalg #17|3|wireless|10|Electronics|663|pricallycally|N/A|5618348570puff756705|navajo|Each|Unknown|30|n stpriableableought| +12240|AAAAAAAAOMPCAAAA|2001-10-27||Local,|97.41|51.62|10016017|importoexporti #1|3|infants|3|Children|663|pricallycally|medium|5618348570puff756705|pink|Dram|Unknown|4|bareseableableought| +12241|AAAAAAAABNPCAAAA|1997-10-27||Just armed names move for the men. Now private ministers should expand different, long rooms. About eastern members shall rest disproporti|6.33|2.97|1002002|importoamalg #2|2|fragrances|1|Women|982|ableeingn st|large|9234013645slate56627|royal|Lb|Unknown|35|oughteseableableought| +12242|AAAAAAAACNPCAAAA|1997-10-27|2000-10-26|Poor, eventual homes would go all foreign powers. Pupils would find most great laws. Twi|1.07|0.82|8016003|corpmaxi #3|16|golf|8|Sports|615|antioughtcally|N/A|37354sky837236464435|lemon|Tbl|Unknown|100|ableeseableableought| +12243|AAAAAAAACNPCAAAA|2000-10-27||Poor, eventual homes would go all foreign powers. Pupils would find most great laws. Twi|1.09|0.82|3001002|amalgexporti #2|1|newborn|3|Children|135|antioughtcally|petite|37354sky837236464435|linen|Dram|Unknown|22|prieseableableought| +12244|AAAAAAAAENPCAAAA|1997-10-27|1999-10-27|Integrated coins must decide apparently other fans; difficult figures see strong activities. Ideas cultivate activities. Narrow, human eggs would use there economic, great temper|4.25|2.55|7011005|amalgnameless #5|11|accent|7|Home|323|priablepri|N/A|1877119635moccasin83|cornsilk|Oz|Unknown|14|eseeseableableought| +12245|AAAAAAAAENPCAAAA|1999-10-28|2001-10-26|Integrated coins must decide apparently other fans; difficult figures see strong activities. Ideas cultivate activities. Narrow, human eggs would use there economic, great temper|3.86|2.31|10005006|scholarunivamalg #6|5|karoke|10|Electronics|530|priablepri|N/A|1877119635moccasin83|sienna|Oz|Unknown|56|antieseableableought| +12246|AAAAAAAAENPCAAAA|2001-10-27||Bizarre things say already carefull|0.69|0.31|10005006|amalgbrand #7|5|semi-precious|6|Jewelry|530|barprianti|N/A|1877119635moccasin83|wheat|Case|Unknown|26|callyeseableableought| +12247|AAAAAAAAHNPCAAAA|1997-10-27||Definite, retail descriptions would not make in between right, far days. Very english police like at a symptoms. Steep regulations recognise. Rich books explore flowers. Teachers ask to a expe|4.30|2.53|6012008|importobrand #8|12|costume|6|Jewelry|63|prically|N/A|9948658yellow2402358|navy|Pallet|Unknown|39|ationeseableableought| +12248|AAAAAAAAINPCAAAA|1997-10-27|2000-10-26|Significant, traditional soldiers sacrifice shortly. Hands could not get new details; uncomfortable police will block. Total, significant galleries assist |3.35|2.01|9005009|scholarmaxi #9|5|history|9|Books|331|oughtpripri|N/A|4558126978744violet4|yellow|N/A|Unknown|24|eingeseableableought| +12249|AAAAAAAAINPCAAAA|2000-10-27||Significant, traditional soldiers sacrifice shortly. Hands could not get new details; uncomfortable police will block. Total, significant galleries assist |2.99|2.01|9005009|importobrand #10|5|bedding|7|Home|331|oughtpripri|N/A|4558126978744violet4|goldenrod|Case|Unknown|27|n steseableableought| +12250|AAAAAAAAKNPCAAAA|1997-10-27|1999-10-27|Possib|0.40|0.28|4004001|edu packedu pack #1|4|athletic|4|Shoes|32|ablepri|small|tan28982070831721008|maroon|Unknown|Unknown|7|barantiableableought| +12251|AAAAAAAAKNPCAAAA|1999-10-28|2001-10-26|Possib|7.59|0.28|8010002|univmaxi #2|10|pools|8|Sports|115|antioughtought|N/A|5dim1496039216171099|pink|Tbl|Unknown|65|oughtantiableableought| +12252|AAAAAAAAKNPCAAAA|2001-10-27||Possib|9.51|0.28|3004001|edu packexporti #1|4|school-uniforms|3|Children|115|antioughtought|medium|5dim1496039216171099|purple|Each|Unknown|6|ableantiableableought| +12253|AAAAAAAANNPCAAAA|1997-10-27||Early chief models conclude typically central, suitable rates. Long, unlikely cities tell journals. Chapters shall study later natural, intense chiefs. Co|2.12|1.63|7011004|amalgnameless #4|11|accent|7|Home|216|callyoughtable|N/A|383585580pale9532391|slate|Tbl|Unknown|87|priantiableableought| +12254|AAAAAAAAONPCAAAA|1997-10-27|2000-10-26|Economic roles should treat. Tall, soft rocks would assess together. Unique lectures would not |0.13|0.10|9009003|maximaxi #3|9|science|9|Books|336|callypripri|N/A|34111160rosy71768380|wheat|Oz|Unknown|26|eseantiableableought| +12255|AAAAAAAAONPCAAAA|2000-10-27||Parameters should think important women. Over future eyes take clearly right shy women. Worthwhile games used to allow only particularly misleading bu|2.23|0.71|9009003|amalgscholar #2|1|rock|5|Music|336|callypripri|N/A|34111160rosy71768380|lime|Dozen|Unknown|31|antiantiableableought| +12256|AAAAAAAAAOPCAAAA|1997-10-27|1999-10-27|Responsible proceedings would prevent below for a beans. Cases know economic effects. Almost given movem|5.89|4.77|1003001|exportiamalg #1|3|maternity|1|Women|570|barationanti|medium|7661light62916356339|orange|Each|Unknown|7|callyantiableableought| +12257|AAAAAAAAAOPCAAAA|1999-10-28|2001-10-26|Goods should marry again early words. Rapidly generous temperatures should not extend usually on a others; other girls sho|7.15|5.14|1003001|edu packbrand #2|3|estate|6|Jewelry|881|oughteingeing|N/A|7661light62916356339|smoke|Cup|Unknown|33|ationantiableableought| +12258|AAAAAAAAAOPCAAAA|2001-10-27||Important, civil prisoners apply for a stateme|1.30|5.14|1003001|exportiimporto #1|3|pants|2|Men|235|antipriable|petite|7661light62916356339|ivory|Lb|Unknown|9|eingantiableableought| +12259|AAAAAAAADOPCAAAA|1997-10-27||Fields provide substantial, young authorities; hands wait; around different years |7.64|4.20|5003002|exportischolar #2|3|pop|5|Music|433|pripriese|N/A|021aquamarine3845363|turquoise|Unknown|Unknown|62|n stantiableableought| +12260|AAAAAAAAEOPCAAAA|1997-10-27|2000-10-26|Long brilliant plans allow long powers; large, level things must allow vice versa red priorities. Sentences read briefly political programmes. Bad cuts need women. Difficulties publ|8.45|7.43|4003001|exportiedu pack #1|3|kids|4|Shoes|934|eseprin st|N/A|164260813290thistle8|mint|Dozen|Unknown|28|barcallyableableought| +12261|AAAAAAAAEOPCAAAA|2000-10-27||Long brilliant plans allow long powers; large, level things must allow vice versa red priorities. Sentences read briefly political programmes. Bad cuts need women. Difficulties publ|7.45|7.43|5002002|importoscholar #2|2|country|5|Music|636|eseprin st|N/A|164260813290thistle8|goldenrod|Ton|Unknown|18|oughtcallyableableought| +12262|AAAAAAAAGOPCAAAA|1997-10-27|1999-10-27|Concerned agreements may imagine forward large demonstrations. Primary, excellent months would not think clearly by a hopes. Open firms wipe men. Impor|2.27|1.92|9007011|brandmaxi #11|7|reference|9|Books|53|prianti|N/A|6green09135230738235|peach|Tsp|Unknown|69|ablecallyableableought| +12263|AAAAAAAAGOPCAAAA|1999-10-28|2001-10-26|Concerned agreements may imagine forward large demonstrations. Primary, excellent months would not think clearly by a hopes. Open firms wipe men. Impor|3.43|3.01|7014004|edu packnameless #4|7|glassware|7|Home|53|prianti|N/A|82236684thistle97517|salmon|Case|Unknown|6|pricallyableableought| +12264|AAAAAAAAGOPCAAAA|2001-10-27||Concerned agreements may imagine forward large demonstrations. Primary, excellent months would not think clearly by a hopes. Open firms wipe men. Impor|0.72|3.01|7014004|maxicorp #1|7|womens watch|6|Jewelry|144|eseeseought|N/A|644165553puff1269082|sienna|Pallet|Unknown|27|esecallyableableought| +12265|AAAAAAAAJOPCAAAA|1997-10-27||Hence annual forces adapt often simultaneously inner children. Departments shall understand yet requirements. Major, local appoint|1.96|1.74|8011006|amalgmaxi #6|11|archery|8|Sports|357|ationantipri|N/A|97618287838cornsilk2|peach|Lb|Unknown|55|anticallyableableought| +12266|AAAAAAAAKOPCAAAA|1997-10-27|2000-10-26|Wholly capital incidents loose for a numbers. Prisoners shall not fold truly. Other, central investments used to ma|4.22|2.61|6015003|scholarbrand #3|15|custom|6|Jewelry|218|eingoughtable|N/A|0366629636red6426446|spring|Dram|Unknown|42|callycallyableableought| +12267|AAAAAAAAKOPCAAAA|2000-10-27||All glorious wages build at all unique minutes; white studies must offer men. Single, short branches would not award however political lands. Most long improvemen|25.69|18.23|6015003|edu packedu pack #2|4|athletic|4|Shoes|313|prioughtpri|medium|624tan42943238527308|plum|Pallet|Unknown|33|ationcallyableableought| +12268|AAAAAAAAMOPCAAAA|1997-10-27|1999-10-27|Medical, main benefits clear again as other terms. Apparent crowds will host large, high surroundings; accurate hands could not use please wealthy |4.86|2.62|3002001|importoexporti #1|2|infants|3|Children|261|oughtcallyable|extra large|0903534wheat14019373|almond|Carton|Unknown|11|eingcallyableableought| +12269|AAAAAAAAMOPCAAAA|1999-10-28|2001-10-26|Enough strange measurements walk then on the studies; contacts surrender also. Words must not look easy, similar homes. Schools put in a places. English, patient letters know h|0.92|0.66|3001002|amalgexporti #2|2|newborn|3|Children|244|eseeseable|N/A|0276309powder2541300|purple|Dozen|Unknown|1|n stcallyableableought| +12270|AAAAAAAAMOPCAAAA|2001-10-27||Colourful films cause large, happy parties. Complex consumers shall not compare then automatic policies. Nurses may not mat|2.89|1.04|3003001|exportiexporti #1|3|toddlers|3|Children|327|eseeseable|petite|0276309powder2541300|slate|Gross|Unknown|28|barationableableought| +12271|AAAAAAAAPOPCAAAA|1997-10-27||Broad, other tears might back more frequent gardens. Different children impose in the agencies. Better wro|2.44|1.53|6008008|namelesscorp #8|8|mens watch|6|Jewelry|185|antieingought|N/A|814351833sienna82711|navy|Each|Unknown|66|oughtationableableought| +12272|AAAAAAAAAPPCAAAA|1997-10-27|2000-10-26|True, red |9.30|8.09|7003003|exportibrand #3|3|kids|7|Home|173|priationought|N/A|067indian46409546847|saddle|Dozen|Unknown|98|ableationableableought| +12273|AAAAAAAAAPPCAAAA|2000-10-27||True, red |7.84|5.25|3003002|exportiexporti #2|3|toddlers|3|Children|150|priationought|large|067indian46409546847|honeydew|Oz|Unknown|75|priationableableought| +12274|AAAAAAAACPPCAAAA|1997-10-27|1999-10-27|Dry, hard details benefit other copies. Regardless industrial streets cease already sources. Developments can hear military talks. So different m|3.20|2.81|3001001|amalgexporti #1|1|newborn|3|Children|51|oughtanti|small|236199636884700rosy2|green|Case|Unknown|86|eseationableableought| +12275|AAAAAAAACPPCAAAA|1999-10-28|2001-10-26|Armed centres send exactly. More constant women |2.56|2.81|8006006|corpnameless #6|1|football|8|Sports|51|oughtanti|N/A|290tomato63955119553|tan|N/A|Unknown|24|antiationableableought| +12276|AAAAAAAACPPCAAAA|2001-10-27||Normal women shall think on a days; directly fat scots oug|6.82|2.81|8006006|importocorp #5|2|diamonds|6|Jewelry|51|oughtanti|N/A|290tomato63955119553|gainsboro|Each|Unknown|21|callyationableableought| +12277|AAAAAAAAFPPCAAAA|1997-10-27||Reports must not wait on a trousers. Local letters consider just; lesser years put otherwise. Always creative parties would make once unnecessarily tired minutes. Yet nuclear vessels sit reliable|2.78|1.55|5002002|importoscholar #2|2|country|5|Music|479|n stationese|N/A|wheat884975061313943|powder|Gram|Unknown|48|ationationableableought| +12278|AAAAAAAAGPPCAAAA|1997-10-27|2000-10-26|Righ|4.01|1.76|4004001|edu packedu pack #1|4|athletic|4|Shoes|659|n stantically|extra large|8600304001sienna2094|white|Gram|Unknown|45|eingationableableought| +12279|AAAAAAAAGPPCAAAA|2000-10-27||Righ|1.07|1.76|4004001|importoedu pack #2|4|mens|4|Shoes|659|n stantically|small|8600304001sienna2094|peach|Carton|Unknown|3|n stationableableought| +12280|AAAAAAAAIPPCAAAA|1997-10-27|1999-10-27|Other, british divisions may pick against a boots. Full, labour parts extend in a goals; pure allega|6.50|2.86|10007007|brandunivamalg #7|7|personal|10|Electronics|190|barn stought|N/A|6746645069snow171167|rosy|Box|Unknown|8|bareingableableought| +12281|AAAAAAAAIPPCAAAA|1999-10-28|2001-10-26|Direct, active books remain; then heavy managers shall pay typically terrible, soviet men. Soviet thousands look long dogs. Easily musical others|9.15|5.58|2003002|exportiimporto #2|3|pants|2|Men|190|barn stought|medium|95230311183638lawn80|plum|Ton|Unknown|55|oughteingableableought| +12282|AAAAAAAAIPPCAAAA|2001-10-27||Direct, active books remain; then heavy managers shall pay typically terrible, soviet men. Soviet thousands look long dogs. Easily musical others|4.04|3.03|6003003|exporticorp #3|3|gold|6|Jewelry|190|barn stought|N/A|8498887987misty68751|ghost|Tsp|Unknown|82|ableeingableableought| +12283|AAAAAAAALPPCAAAA|1997-10-27||Political ingredients exercise once in order less|4.95|3.86|9003008|exportimaxi #8|3|computers|9|Books|78|eingation|N/A|26498899papaya914393|lavender|Oz|Unknown|27|prieingableableought| +12284|AAAAAAAAMPPCAAAA|1997-10-27|2000-10-26|Categories shall|8.98|7.81|9005009|scholarmaxi #9|5|history|9|Books|389|n steingpri|N/A|peru4819922057959879|seashell|Tbl|Unknown|55|eseeingableableought| +12285|AAAAAAAAMPPCAAAA|2000-10-27||Available, hot days might adopt only public, great memories. Relative princip|2.39|7.81|9003010|exportimaxi #10|3|computers|9|Books|275|antiationable|N/A|77snow94790096959158|magenta|Dram|Unknown|77|antieingableableought| +12286|AAAAAAAAOPPCAAAA|1997-10-27|1999-10-27|Things must wait obvious, other drugs; behind difficult activities shall clarify realistically available, likely partners. Buses go beds. Troops would al|8.50|5.35|7003007|exportibrand #7|3|kids|7|Home|265|anticallyable|N/A|6913939magenta890618|sky|Ounce|Unknown|59|callyeingableableought| +12287|AAAAAAAAOPPCAAAA|1999-10-28|2001-10-26|Things must wait obvious, other drugs; behind difficult activities shall clarify realistically available, likely partners. Buses go beds. Troops would al|6.87|3.50|2004002|edu packimporto #2|3|sports-apparel|2|Men|265|anticallyable|petite|76430164thistle14890|blue|Bundle|Unknown|23|ationeingableableought| +12288|AAAAAAAAOPPCAAAA|2001-10-27||Bad, national debts hear lengthy days. Years cannot reflect very women. Messages earn. Tories shall|8.85|3.50|2004002|exportiamalg #1|3|maternity|1|Women|265|anticallyable|economy|76430164thistle14890|pale|Gram|Unknown|40|eingeingableableought| +12289|AAAAAAAABAADAAAA|1997-10-27||Jeans may not represent relatively young provinces. More other studi|17.10|8.55|7016010|corpnameless #10|16|furniture|7|Home|66|callycally|N/A|959752270812041rosy0|peach|Bundle|Unknown|11|n steingableableought| +12290|AAAAAAAACAADAAAA|1997-10-27|2000-10-26|Ver|1.42|1.19|4003001|exportiedu pack #1|3|kids|4|Shoes|606|callybarcally|extra large|220seashell045888453|sandy|Bunch|Unknown|48|barn stableableought| +12291|AAAAAAAACAADAAAA|2000-10-27||Ver|7.80|1.19|7009002|maxibrand #2|3|mattresses|7|Home|606|callybarcally|N/A|220seashell045888453|wheat|Cup|Unknown|32|oughtn stableableought| +12292|AAAAAAAAEAADAAAA|1997-10-27|1999-10-27|Even vague elements might not make certainly personal rebels. Policies might generate; british, only peasants should withstand necessarily; necessary, a|2.95|1.59|1004001|edu packamalg #1|4|swimwear|1|Women|205|antibarable|small|6323338462268navajo4|pale|Lb|Unknown|28|ablen stableableought| +12293|AAAAAAAAEAADAAAA|1999-10-28|2001-10-26|True, magic adults cannot answer greatly on a times. |4.21|1.59|5002002|importoscholar #2|4|country|5|Music|205|antibarable|N/A|6323338462268navajo4|sienna|Oz|Unknown|18|prin stableableought| +12294|AAAAAAAAEAADAAAA|2001-10-27||True, magic adults cannot answer greatly on a times. |87.64|1.59|5002002|brandunivamalg #4|4|personal|10|Electronics|205|antibarable|N/A|6323338462268navajo4|navy|N/A|Unknown|94|esen stableableought| +12295|AAAAAAAAHAADAAAA|1997-10-27||Men would aid very different hands. Months may not start neither even important refugees. Dealers could matter then very black travellers; southe|5.67|4.19|6001008|amalgcorp #8|1|birdal|6|Jewelry|538|eingprianti|N/A|9277101605867198tan3|saddle|Dram|Unknown|62|antin stableableought| +12296|AAAAAAAAIAADAAAA|1997-10-27|2000-10-26|Long patients would roll. Expensive, different causes would not use so young, full years. Orders will defend. Private, major conferences can test public problems. Hard metres should not |6.01|2.76|4004001|edu packedu pack #1|4|athletic|4|Shoes|158|eingantiought|petite|51844745frosted29508|maroon|Ounce|Unknown|23|callyn stableableought| +12297|AAAAAAAAIAADAAAA|2000-10-27||Long patients would roll. Expensive, different causes would not use so young, full years. Orders will defend. Private, major conferences can test public problems. Hard metres should not |5.11|2.76|4001002|amalgedu pack #2|1|womens|4|Shoes|299|n stn stable|large|51844745frosted29508|royal|Pound|Unknown|57|ationn stableableought| +12298|AAAAAAAAKAADAAAA|1997-10-27|1999-10-27|Long things must adopt further interesting groups; however military lines used to help teams. Long-term, japanese subjects could alter as men. Just other democrats pour necessarily just nece|0.32|0.19|4002001|importoedu pack #1|2|mens|4|Shoes|77|ationation|extra large|61571399navy43496341|ivory|Lb|Unknown|83|eingn stableableought| +12299|AAAAAAAAKAADAAAA|1999-10-28|2001-10-26|Classical, happy models can proceed rather. Judges can expect ago. Instruments get high. Certain notes make still american, academic classes. Major, electric attacks find facts. Crue|7.04|0.19|4002001|importoscholar #2|2|country|5|Music|416|callyoughtese|N/A|61571399navy43496341|spring|Ton|Unknown|93|n stn stableableought| +12300|AAAAAAAAKAADAAAA|2001-10-27||Past large men shall complement there very thin words. Written, dreadful interests evolve as. Successful shops must take overall private, european masters. Still new tears might affect enormous |0.18|0.19|4002001|amalgedu pack #1|1|womens|4|Shoes|96|callyn st|petite|833000473663tomato36|pale|Lb|Unknown|13|barbarpriableought| +12301|AAAAAAAANAADAAAA|1997-10-27||Constant links reveal al|9.08|6.99|7012002|importonameless #2|12|paint|7|Home|466|callycallyese|N/A|00grey59444147743165|purple|Pallet|Unknown|27|oughtbarpriableought| +12302|AAAAAAAAOAADAAAA|1997-10-27|2000-10-26|Trees come. Quickly elaborate problems shall learn more whole points; separate, usual cars can include yet young, cl|2.72|1.65|3003001|exportiexporti #1|3|toddlers|3|Children|23|priable|small|04951486168638olive0|rose|Box|Unknown|5|ablebarpriableought| +12303|AAAAAAAAOAADAAAA|2000-10-27||Trees come. Quickly elaborate problems shall learn more whole points; separate, usual cars can include yet young, cl|7.82|4.14|3003001|amalgcorp #8|3|birdal|6|Jewelry|23|priable|N/A|04951486168638olive0|moccasin|Oz|Unknown|12|pribarpriableought| +12304|AAAAAAAAABADAAAA|1997-10-27|1999-10-27|Dangerous others may not stay either then military products. Social, black members will go members; only upper skills think more than children. Somewhere bad days can fee|7.21|3.17|3001001|amalgexporti #1|1|newborn|3|Children|389|n steingpri|medium|orange92743560524196|green|Tsp|Unknown|99|esebarpriableought| +12305|AAAAAAAAABADAAAA|1999-10-28|2001-10-26|Different, national statements tell now solar, typical strings|0.59|3.17|3001001|importocorp #2|1|diamonds|6|Jewelry|389|n steingpri|N/A|orange92743560524196|pink|Tsp|Unknown|62|antibarpriableought| +12306|AAAAAAAAABADAAAA|2001-10-27||Willing, scottish policemen mean rather yet full needs; s|67.50|3.17|5002001|importoscholar #1|2|country|5|Music|389|n steingpri|N/A|orange92743560524196|sky|Ounce|Unknown|9|callybarpriableought| +12307|AAAAAAAADBADAAAA|1997-10-27||Amazing, efficient areas used to score more more safe consequences. Causes will cause interestingly papers. New, gold women would point usually on the values. Likely, radical eyes wo|42.14|15.59|1003002|exportiamalg #2|3|maternity|1|Women|126|callyableought|petite|86674plum12800043152|white|Dozen|Unknown|51|ationbarpriableought| +12308|AAAAAAAAEBADAAAA|1997-10-27|2000-10-26|Farmers shall want there |0.90|0.54|1002001|importoamalg #1|2|fragrances|1|Women|135|antipriought|medium|591699631navy3107749|salmon|Tbl|Unknown|35|eingbarpriableought| +12309|AAAAAAAAEBADAAAA|2000-10-27||Farmers shall want there |1.25|0.78|1002001|amalgnameless #10|2|accent|7|Home|269|n stcallyable|N/A|591699631navy3107749|peach|Cup|Unknown|66|n stbarpriableought| +12310|AAAAAAAAGBADAAAA|1997-10-27|1999-10-27|Public members may not forgive; boys ought to hurt so early british changes. Faces would |8.17|4.65|2004001|edu packimporto #1|4|sports-apparel|2|Men|629|n stablecally|medium|0135799338107mint508|linen|Dozen|Unknown|81|baroughtpriableought| +12311|AAAAAAAAGBADAAAA|1999-10-28|2001-10-26|Public members may not forgive; boys ought to hurt so early british changes. Faces would |4.43|4.65|2004001|importonameless #2|12|paint|7|Home|629|n stablecally|N/A|0135799338107mint508|tan|Gram|Unknown|15|oughtoughtpriableought| +12312|AAAAAAAAGBADAAAA|2001-10-27||Possible, small boundaries live down the times. More native towns will show direct. Matters keep earlier therefore usual jobs. Traditional audiences would not ensure no doubt m|4.92|3.54|2004001|edu packamalg #1|12|swimwear|1|Women|629|n stablecally|small|6665948saddle9469449|sandy|Oz|Unknown|26|ableoughtpriableought| +12313|AAAAAAAAJBADAAAA|1997-10-27||Mean, good relations wake however strictly white possibilities. About aw|6.42|3.46|9003002|exportimaxi #2|3|computers|9|Books|314|eseoughtpri|N/A|550468817240papaya03|lace|N/A|Unknown|15|prioughtpriableought| +12314|AAAAAAAAKBADAAAA|1997-10-27|2000-10-26|Women will see therefore bad procedures. Therefore specific colleges shall get over public tickets. New, important taxes continue flat asian, strong leaders. Administrative, n|3.25|2.79|5003001|exportischolar #1|3|pop|5|Music|267|ationcallyable|N/A|922577374seashell250|seashell|Bunch|Unknown|56|eseoughtpriableought| +12315|AAAAAAAAKBADAAAA|2000-10-27||Women will see therefore bad procedures. Therefore specific colleges shall get over public tickets. New, important taxes continue flat asian, strong leaders. Administrative, n|5.18|3.41|5003001|exportinameless #6|3|basketball|8|Sports|267|ationcallyable|N/A|922577374seashell250|orange|Each|Unknown|16|antioughtpriableought| +12316|AAAAAAAAMBADAAAA|1997-10-27|1999-10-27|Vividly weak tickets charge more almost very families. Perhaps basic features may not carry shortly local technologies. Light perceptions must get increasingly g|3.17|1.33|3002001|importoexporti #1|2|infants|3|Children|201|oughtbarable|petite|94077296moccasin1588|peru|Pallet|Unknown|20|callyoughtpriableought| +12317|AAAAAAAAMBADAAAA|1999-10-28|2001-10-26|Vividly weak tickets charge more almost very families. Perhaps basic features may not carry shortly local technologies. Light perceptions must get increasingly g|3.79|1.33|9012006|importounivamalg #6|12|home repair|9|Books|201|oughtbarable|N/A|94077296moccasin1588|orange|N/A|Unknown|39|ationoughtpriableought| +12318|AAAAAAAAMBADAAAA|2001-10-27||Vividly weak tickets charge more almost very families. Perhaps basic features may not carry shortly local technologies. Light perceptions must get increasingly g|4.42|1.33|3004001|edu packexporti #1|4|school-uniforms|3|Children|403|pribarese|small|36003468740plum38876|blush|Pound|Unknown|2|eingoughtpriableought| +12319|AAAAAAAAPBADAAAA|1997-10-27||Direct actions receive in a years. Sure, firm cars can examine lines. Others should preach ultimately under th|4.90|4.01|3004002|edu packexporti #2|4|school-uniforms|3|Children|499|n stn stese|medium|878535240thistle6525|linen|Tbl|Unknown|2|n stoughtpriableought| +12320|AAAAAAAAACADAAAA|1997-10-27|2000-10-26|Successes embrace modern, able matters. Schools cannot convert wryly odd surveys. Generally useful techniques would not make enormously public, difficult se|0.78|0.49|6010001|univbrand #1|10|jewelry boxes|6|Jewelry|47|ationese|N/A|5535541010lawn457523|tomato|Box|Unknown|20|barablepriableought| +12321|AAAAAAAAACADAAAA|2000-10-27||Successes embrace modern, able matters. Schools cannot convert wryly odd surveys. Generally useful techniques would not make enormously public, difficult se|9.17|0.49|6010001|maxinameless #2|10|optics|8|Sports|47|ationese|N/A|610034325steel364862|gainsboro|Lb|Unknown|31|oughtablepriableought| +12322|AAAAAAAACCADAAAA|1997-10-27|1999-10-27|Arrangements may try as traditionally new skills. At least general companies see always under a authoriti|2.97|1.06|2004001|edu packimporto #1|4|sports-apparel|2|Men|982|ableeingn st|small|90720752209pale29967|papaya|Dram|Unknown|13|ableablepriableought| +12323|AAAAAAAACCADAAAA|1999-10-28|2001-10-26|Short sources may require otherwise consequences. Similarly empty goods shall get actually. Times would operate in order very social things. Economic places make eventually huge interpretations; manuf|43.99|1.06|9008012|namelessmaxi #12|4|romance|9|Books|286|ableeingn st|N/A|90720752209pale29967|sandy|Oz|Unknown|34|priablepriableought| +12324|AAAAAAAACCADAAAA|2001-10-27||Boots support then errors; costs would come experimental problems.|5.33|3.19|9008012|edu packbrand #5|4|curtains/drapes|7|Home|385|antieingpri|N/A|412980547373saddle04|ghost|Carton|Unknown|87|eseablepriableought| +12325|AAAAAAAAFCADAAAA|1997-10-27||Sure central terms fancy and so on from a notes. Busy communities ought to stop more difficult shops. Wrong, new associations used to understand probably mostly p|3.25|2.66|1002002|importoamalg #2|2|fragrances|1|Women|261|oughtcallyable|medium|314red21558309536459|yellow|N/A|Unknown|23|antiablepriableought| +12326|AAAAAAAAGCADAAAA|1997-10-27|2000-10-26|However fascinating boats imagine so together active owners. Therefore legal activities would receive to a animals. Western, open principles join old women. |1.69|1.47|6003007|exporticorp #7|3|gold|6|Jewelry|932|ableprin st|N/A|364332363490steel516|sienna|Dram|Unknown|59|callyablepriableought| +12327|AAAAAAAAGCADAAAA|2000-10-27||Events conceal abroad blue departments; brilliant fires freeze. Good, great artists could offer. C|4.87|1.47|6003007|amalgamalg #2|1|dresses|1|Women|932|ableprin st|medium|89628yellow872954065|navajo|Dram|Unknown|23|ationablepriableought| +12328|AAAAAAAAICADAAAA|1997-10-27|1999-10-27|Others could get about nearly political houses; ages improve homes. Soft products cry carefully proper numbers; months find more other assessments. Studies bear now patients. Vital, able|3.43|2.16|6005001|scholarcorp #1|5|earings|6|Jewelry|486|callyeingese|N/A|43992978orange068625|burlywood|Cup|Unknown|46|eingablepriableought| +12329|AAAAAAAAICADAAAA|1999-10-28|2001-10-26|Others could get about nearly political houses; ages improve homes. Soft products cry carefully proper numbers; months find more other assessments. Studies bear now patients. Vital, able|1.39|0.56|6005001|corpmaxi #6|5|parenting|9|Books|21|oughtable|N/A|43992978orange068625|turquoise|Carton|Unknown|10|n stablepriableought| +12330|AAAAAAAAICADAAAA|2001-10-27||Local, parliam|3.54|1.16|9004007|edu packmaxi #7|4|entertainments|9|Books|305|antibarpri|N/A|43992978orange068625|misty|Each|Unknown|15|barpripriableought| +12331|AAAAAAAALCADAAAA|1997-10-27||That basic students |2.00|1.14|7015002|scholarnameless #2|15|tables|7|Home|242|ableeseable|N/A|6556396357817khaki21|floral|Dozen|Unknown|11|oughtpripriableought| +12332|AAAAAAAAMCADAAAA|1997-10-27|2000-10-26|Much certain gardens shall not result quick sounds. Of course outer opportunities see very. Recent terms might take a|7.12|5.69|9012009|importounivamalg #9|12|home repair|9|Books|42|ableese|N/A|6849saddle1139972715|wheat|Ton|Unknown|25|ablepripriableought| +12333|AAAAAAAAMCADAAAA|2000-10-27||Only true instructions manufacture for a councils. Look|2.41|1.06|8004004|edu packnameless #4|12|camping|8|Sports|42|ableese|N/A|268ivory963498973084|peach|Unknown|Unknown|78|pripripriableought| +12334|AAAAAAAAOCADAAAA|1997-10-27|1999-10-27|Then new voices might resort never normal average friends. Groups will trust so real, military arts. Minor, able weeks|4.99|3.64|9007011|brandmaxi #11|7|reference|9|Books|657|ationantically|N/A|893752573288saddle01|coral|Ton|Unknown|74|esepripriableought| +12335|AAAAAAAAOCADAAAA|1999-10-28|2001-10-26|Then new voices might resort never normal average friends. Groups will trust so real, military arts. Minor, able weeks|1.12|0.79|1001002|amalgamalg #2|7|dresses|1|Women|456|callyantiese|petite|powder86758325557966|pink|Unknown|Unknown|8|antipripriableought| +12336|AAAAAAAAOCADAAAA|2001-10-27||Large, large courts speak so entire, ordinary quantities. Given differences shall not wake. Stages will produce entirely masters. Bright authorities persuade. Dutch methods shorten thus f|3.70|0.79|1001002|corpcorp #1|6|rings|6|Jewelry|626|callyantiese|N/A|powder86758325557966|puff|Case|Unknown|8|callypripriableought| +12337|AAAAAAAABDADAAAA|1997-10-27||Small, potenti|14.71|11.76|10010013|univamalgamalg #13|10|memory|10|Electronics|468|eingcallyese|N/A|191467145040light900|violet|Case|Unknown|83|ationpripriableought| +12338|AAAAAAAACDADAAAA|1997-10-27|2000-10-26|Environmental, civil examples find of course by a skills. Small weekends concede thick, perfect proposals. Women might not try; underlying charges|4.42|3.18|1002001|importoamalg #1|2|fragrances|1|Women|10|barought|medium|4184981051063493sky9|blanched|N/A|Unknown|24|eingpripriableought| +12339|AAAAAAAACDADAAAA|2000-10-27||Environmental, civil examples find of course by a skills. Small weekends concede thick, perfect proposals. Women might not try; underlying charges|3.42|3.18|5003002|exportischolar #2|2|pop|5|Music|195|barought|N/A|7aquamarine921186824|red|Unknown|Unknown|44|n stpripriableought| +12340|AAAAAAAAEDADAAAA|1997-10-27|1999-10-27|Global actions slip etc windows. Probably true papers know both with a months. Other states let industrial, open lectures. Expressions climb within a doubts. So western details|3.75|2.85|8016001|corpmaxi #1|16|golf|8|Sports|80|bareing|N/A|4074390goldenrod9537|lemon|Cup|Unknown|60|baresepriableought| +12341|AAAAAAAAEDADAAAA|1999-10-28|2001-10-26|Global actions slip etc windows. Probably true papers know both with a months. Other states let industrial, open lectures. Expressions climb within a doubts. So western details|51.87|16.07|4001002|amalgedu pack #2|1|womens|4|Shoes|80|bareing|petite|4074390goldenrod9537|moccasin|Tbl|Unknown|89|oughtesepriableought| +12342|AAAAAAAAEDADAAAA|2001-10-27||Labour components shall spell teeth. Together dutch gates meet extraordinary, original polls. Then positive weapons get simply national forms. Long, steady sounds s|3.50|16.07|4001002|importoedu pack #1|1|mens|4|Shoes|71|bareing|economy|4074390goldenrod9537|slate|Tsp|Unknown|4|ableesepriableought| +12343|AAAAAAAAHDADAAAA|1997-10-27||Classes watch far better scottish cars. Controls trace more voluntary, only enquiries. Special comments assess with the followers. So straight libraries build more with a examp|9.76|3.02|2001002|amalgimporto #2|1|accessories|2|Men|732|ablepriation|economy|8321sky3394112552794|plum|Ton|Unknown|71|priesepriableought| +12344|AAAAAAAAIDADAAAA|1997-10-27|2000-10-26|Just natural companies apply readily damp, economic houses; years tell. Rare scenes meet good friends. Social, keen minutes used t|0.30|0.09|3004001|edu packexporti #1|4|school-uniforms|3|Children|604|esebarcally|extra large|5728red4336162630709|sienna|Bundle|Unknown|3|eseesepriableought| +12345|AAAAAAAAIDADAAAA|2000-10-27||Usually conventional months ease more wonderful limits. Suddenly alone resources should fetch beliefs. Numerous, foreign arms assess now on a men; possibilities e|1.93|1.23|8006006|corpnameless #6|6|football|8|Sports|604|esebarcally|N/A|1710614454sienna5362|turquoise|Dozen|Unknown|14|antiesepriableought| +12346|AAAAAAAAKDADAAAA|1997-10-27|1999-10-27|More than slight sectors examine then. Merely central children may play in a orders. Days use rightly. American, far operators used to know th|19.62|5.88|8002007|importonameless #7|2|baseball|8|Sports|160|barcallyought|N/A|401284215645antique6|moccasin|Cup|Unknown|75|callyesepriableought| +12347|AAAAAAAAKDADAAAA|1999-10-28|2001-10-26|Others take with a |1.59|5.88|8002007|importoexporti #2|2|infants|3|Children|160|barcallyought|large|0119peach45641139682|sienna|Pound|Unknown|18|ationesepriableought| +12348|AAAAAAAAKDADAAAA|2001-10-27||Others take with a |1.37|5.88|1001001|amalgamalg #1|1|dresses|1|Women|160|barcallyought|extra large|0119peach45641139682|sandy|Gram|Unknown|12|eingesepriableought| +12349|AAAAAAAANDADAAAA|1997-10-27||Jewish, central friends suggest thus prime relations. Full, central indicators rise schools. Concerns may not show years. Educational, healthy services used to pr|1.26|0.71|10007008|brandunivamalg #8|7|personal|10|Electronics|939|n stprin st|N/A|342114spring44828768|orchid|Lb|Unknown|90|n stesepriableought| +12350|AAAAAAAAODADAAAA|1997-10-27|2000-10-26|Straight, public figures serve before. Products might give seriously inner, wet colleagues. Needs should read never monetary benefits. |6.79|4.95|1003001|exportiamalg #1|3|maternity|1|Women|88|eingeing|petite|638557228193salmon85|rose|Oz|Unknown|39|barantipriableought| +12351|AAAAAAAAODADAAAA|2000-10-27||New, serious techniques cannot offer. Already left groups shall begin to a sites. Also physical activities may look still young, other numbers. Probably so|2.76|1.40|1003001|amalgimporto #2|3|accessories|2|Men|255|antiantiable|medium|638557228193salmon85|lavender|Dozen|Unknown|8|oughtantipriableought| +12352|AAAAAAAAAEADAAAA|1997-10-27|1999-10-27|O|5.83|4.37|2002001|importoimporto #1|2|shirts|2|Men|558|eingantianti|large|8508164khaki20722816|orange|Each|Unknown|56|ableantipriableought| +12353|AAAAAAAAAEADAAAA|1999-10-28|2001-10-26|Well sure neighbours increase at least in a texts; much silent expenses used to use disabled, difficult tickets. Products incur horses. Modest, social women can matter nearly far, serious rocks;|1.15|0.96|2002001|importobrand #2|12|costume|6|Jewelry|486|callyeingese|N/A|85397989242navajo740|maroon|Case|Unknown|30|priantipriableought| +12354|AAAAAAAAAEADAAAA|2001-10-27||Sheets might offer best commercial leaves; results study much loca|92.37|0.96|2002001|amalgimporto #1|12|accessories|2|Men|486|callyeingese|petite|34red845771335245736|lime|Box|Unknown|15|eseantipriableought| +12355|AAAAAAAADEADAAAA|1997-10-27||Full shareholders alert also such as a men. Dead memories can accompany different estates; national, great years shall develop just on the years. Visitors will not display ser|2.94|2.35|4001002|amalgedu pack #2|1|womens|4|Shoes|489|n steingese|extra large|19sienna542009556636|white|Case|Unknown|29|antiantipriableought| +12356|AAAAAAAAEEADAAAA|1997-10-27|2000-10-26|Long-term cigarettes ensure because of a commentators; days run per a reports; bodies include there in a rocks. Necessary privileges should resist alre|13.77|9.63|8007007|brandnameless #7|7|hockey|8|Sports|175|antiationought|N/A|22733khaki7361451535|peru|N/A|Unknown|42|callyantipriableought| +12357|AAAAAAAAEEADAAAA|2000-10-27||Long-term cigarettes ensure because of a commentators; days run per a reports; bodies include there in a rocks. Necessary privileges should resist alre|2.15|1.84|8007007|brandunivamalg #16|7|personal|10|Electronics|175|antiationought|N/A|22733khaki7361451535|papaya|Gross|Unknown|69|ationantipriableought| +12358|AAAAAAAAGEADAAAA|1997-10-27|1999-10-27|Ends choose sophisticated ministers. Modern feelings face armed individuals; |8.61|4.13|5004001|edu packscholar #1|4|classical|5|Music|415|antioughtese|N/A|4467papaya6492153445|seashell|Bundle|Unknown|13|eingantipriableought| +12359|AAAAAAAAGEADAAAA|1999-10-28|2001-10-26|Ends choose sophisticated ministers. Modern feelings face armed individuals; |0.90|0.47|6009008|maxicorp #8|4|womens watch|6|Jewelry|415|antioughtese|N/A|4962102rose700564210|pink|Ton|Unknown|55|n stantipriableought| +12360|AAAAAAAAGEADAAAA|2001-10-27||Ends choose sophisticated ministers. Modern feelings face armed individuals; |7.29|0.47|6009008|brandcorp #1|4|pendants|6|Jewelry|415|antioughtese|N/A|8040401300635plum135|purple|Oz|Unknown|33|barcallypriableought| +12361|AAAAAAAAJEADAAAA|1997-10-27||Daily packages shall suffer reportedly numbers. Committees begin environmental groups. Different others blame needs; other|35.79|28.98|4004002|edu packedu pack #2|4|athletic|4|Shoes|48|eingese|large|034683953yellow44298|pink|Ton|Unknown|24|oughtcallypriableought| +12362|AAAAAAAAKEADAAAA|1997-10-27|2000-10-26|Christian, good policies seem more results. Parents must lead for example by a pp.. Empty items happen early in a ways; beautiful times fight contracts. Measures might seem so full rates. Diffi|3.96|3.00|2004001|edu packimporto #1|4|sports-apparel|2|Men|525|antiableanti|large|5446365peach66339830|orange|Tsp|Unknown|65|ablecallypriableought| +12363|AAAAAAAAKEADAAAA|2000-10-27||Especially radical events gain ready, likely industries. Alone, white countries give ever with a uses. Immediate, necessary agents go so past bed|21.21|17.60|2004001|corpunivamalg #4|16|mystery|9|Books|443|antiableanti|N/A|5446365peach66339830|purple|Dram|Unknown|26|pricallypriableought| +12364|AAAAAAAAMEADAAAA|1997-10-27|1999-10-27|True, local day|3.55|3.08|6007005|brandcorp #5|7|pendants|6|Jewelry|576|callyationanti|N/A|75846royal1456531313|blanched|Pound|Unknown|35|esecallypriableought| +12365|AAAAAAAAMEADAAAA|1999-10-28|2001-10-26|Unable, german studies cope clearly just points. Conventional services say bloody walls. Feet put therefore goods. Feet mi|1.55|0.86|10005007|scholarunivamalg #7|7|karoke|10|Electronics|598|callyationanti|N/A|75846royal1456531313|saddle|Ounce|Unknown|13|anticallypriableought| +12366|AAAAAAAAMEADAAAA|2001-10-27||Unable, german studies cope clearly just points. Conventional services say bloody walls. Feet put therefore goods. Feet mi|2.76|0.86|10005007|exportiedu pack #1|3|kids|4|Shoes|216|callyoughtable|petite|4610624536spring2812|rose|Tbl|Unknown|34|callycallypriableought| +12367|AAAAAAAAPEADAAAA|1997-10-27||For example decent routes shall give specially ethnic common explanations. Aware animals shoul|1.28|0.61|7003008|exportibrand #8|3|kids|7|Home|865|anticallyeing|N/A|51011114puff52787941|linen|Bunch|Unknown|70|ationcallypriableought| +12368|AAAAAAAAAFADAAAA|1997-10-27|2000-10-26|Other, new contracts want easy vehicles. Smooth industries should ask high students. Facts|1.41|1.12|7016009|corpnameless #9|16|furniture|7|Home|904|esebarn st|N/A|381900blue1428275770|turquoise|Bundle|Unknown|32|eingcallypriableought| +12369|AAAAAAAAAFADAAAA|2000-10-27||Available mechanisms arrange on a accountants. Today single debts shall reduce never. Shareholders compete. Only ill legs shall indicate domestic duties. Professio|95.53|1.12|3003002|exportiexporti #2|16|toddlers|3|Children|904|esebarn st|extra large|381900blue1428275770|ivory|Oz|Unknown|51|n stcallypriableought| +12370|AAAAAAAACFADAAAA|1997-10-27|1999-10-27|Police discuss educational, real others; public, open terms may avoid directly physical offices. Impossible, international times will fill poor keys. At least other miles realise excessive|0.60|0.31|1001001|amalgamalg #1|1|dresses|1|Women|264|esecallyable|large|4006165444825sandy87|thistle|Carton|Unknown|4|barationpriableought| +12371|AAAAAAAACFADAAAA|1999-10-28|2001-10-26|Police discuss educational, real others; public, open terms may avoid directly physical offices. Impossible, international times will fill poor keys. At least other miles realise excessive|2.22|0.31|1001001|exportiimporto #2|3|pants|2|Men|464|esecallyese|large|512262293dark2488376|smoke|Bunch|Unknown|35|oughtationpriableought| +12372|AAAAAAAACFADAAAA|2001-10-27||White resources wonder too families. Economic babies would see briti|9.06|5.34|2002001|importoimporto #1|3|shirts|2|Men|464|esecallyese|extra large|512262293dark2488376|sienna|Unknown|Unknown|30|ableationpriableought| +12373|AAAAAAAAFFADAAAA|1997-10-27||Full others become suddenly from a companies. Artistic, human tests must differ very small functions. Possible police cook quickly dry holes. However wild years break|0.36|0.28|10009015|maxiunivamalg #15|9|televisions|10|Electronics|568|eingcallyanti|N/A|95honeydew7927884843|papaya|Ounce|Unknown|32|priationpriableought| +12374|AAAAAAAAGFADAAAA|1997-10-27|2000-10-26|Organizations would show great, safe schools. Groups ought to complement less hands. Apparent rumours used to |1.33|1.11|8010005|univmaxi #5|10|pools|8|Sports|23|priable|N/A|9white36139787304793|purple|Pallet|Unknown|47|eseationpriableought| +12375|AAAAAAAAGFADAAAA|2000-10-27||Locally steady years prevent students. Recent months would not hear. Police supervise huge, special years. Later legislative solicitors guess enough human glasses. Officers will attain enough ch|3.96|2.41|8010005|namelesscorp #8|8|mens watch|6|Jewelry|736|priable|N/A|9white36139787304793|salmon|Each|Unknown|5|antiationpriableought| +12376|AAAAAAAAIFADAAAA|1997-10-27|1999-10-27|Wide, local mothers should say units. Potential costs affect hard st|1.16|0.37|1003001|exportiamalg #1|3|maternity|1|Women|98|eingn st|petite|3antique765531863818|salmon|Dozen|Unknown|77|callyationpriableought| +12377|AAAAAAAAIFADAAAA|1999-10-28|2001-10-26|Wide, local mothers should say units. Potential costs affect hard st|0.76|0.37|1003001|importoimporto #2|3|shirts|2|Men|98|eingn st|petite|3888480382424pale520|navajo|Pallet|Unknown|24|ationationpriableought| +12378|AAAAAAAAIFADAAAA|2001-10-27||Totally nice matters should make from a intentions. Peculiar, fast goods benefit strongly inc prices. Later organic |98.82|39.52|6002003|importocorp #3|3|diamonds|6|Jewelry|98|eingn st|N/A|59732806powder443426|white|Carton|Unknown|10|eingationpriableought| +12379|AAAAAAAALFADAAAA|1997-10-27||Quiet, foreign stores see however as full things. Individual, electric|7.96|6.28|1003002|exportiamalg #2|3|maternity|1|Women|502|ablebaranti|small|909527878933244rosy8|slate|Oz|Unknown|23|n stationpriableought| +12380|AAAAAAAAMFADAAAA|1997-10-27|2000-10-26|New, poor friends should not remember lines. Generally present techniques will not damage then good problems. Names remove as true questions. Outstanding subjects would reflect tonight|60.22|36.73|7001001|amalgbrand #1|1|bathroom|7|Home|444|eseeseese|N/A|rose7679956292809560|rose|Dozen|Unknown|57|bareingpriableought| +12381|AAAAAAAAMFADAAAA|2000-10-27||New, poor friends should not remember lines. Generally present techniques will not damage then good problems. Names remove as true questions. Outstanding subjects would reflect tonight|2.16|36.73|7001001|amalgamalg #2|1|dresses|1|Women|444|eseeseese|extra large|5sienna0455154965725|pale|Carton|Unknown|51|oughteingpriableought| +12382|AAAAAAAAOFADAAAA|1997-10-27|1999-10-27|Free, likely men mind artificial, good seats.|3.61|3.10|4003001|exportiedu pack #1|3|kids|4|Shoes|173|priationought|extra large|58616905orchid997477|metallic|Dozen|Unknown|87|ableeingpriableought| +12383|AAAAAAAAOFADAAAA|1999-10-28|2001-10-26|Free, likely men mind artificial, good seats.|2.07|1.57|4003001|exportischolar #2|3|pop|5|Music|529|n stableanti|N/A|58616905orchid997477|papaya|Pallet|Unknown|1|prieingpriableought| +12384|AAAAAAAAOFADAAAA|2001-10-27||Free, likely men mind artificial, good seats.|3.93|1.57|4003001|scholarunivamalg #1|3|fiction|9|Books|529|n stableanti|N/A|58616905orchid997477|violet|Lb|Unknown|88|eseeingpriableought| +12385|AAAAAAAABGADAAAA|1997-10-27||Very new sources must sleep foreign horses; products improve very forests. Old, royal families might hurt upon a m|8.64|3.62|7006006|corpbrand #6|6|rugs|7|Home|878|eingationeing|N/A|71tan913631158385968|tomato|Dram|Unknown|7|antieingpriableought| +12386|AAAAAAAACGADAAAA|1997-10-27|2000-10-26|Systems would send sometim|8.01|4.56|6007003|brandcorp #3|7|pendants|6|Jewelry|633|priprically|N/A|3316156976sienna5804|metallic|Dram|Unknown|27|callyeingpriableought| +12387|AAAAAAAACGADAAAA|2000-10-27||Systems would send sometim|9.56|4.56|3002002|importoexporti #2|7|infants|3|Children|633|priprically|small|4624495246lawn194440|peach|Each|Unknown|22|ationeingpriableought| +12388|AAAAAAAAEGADAAAA|1997-10-27|1999-10-27|Individuals act. Merely other phrases notice on a sanctions. Courses can embody. Relatively creative subjects hear very at a letters; financial, useful eyes c|6.23|3.67|7009009|maxibrand #9|9|mattresses|7|Home|326|callyablepri|N/A|168spring75189164704|sky|Carton|Unknown|7|eingeingpriableought| +12389|AAAAAAAAEGADAAAA|1999-10-28|2001-10-26|Both necessary functions make forthcoming flowers. Respons|6.75|3.98|8012010|importomaxi #10|12|guns|8|Sports|326|callyablepri|N/A|168spring75189164704|rose|Carton|Unknown|49|n steingpriableought| +12390|AAAAAAAAEGADAAAA|2001-10-27||Funds used to tighten. Tests could play still other activities. Home single systems include on to a women. At all strong gentlemen wo|1.74|3.98|8012010|amalgedu pack #1|1|womens|4|Shoes|152|ableantiought|large|9296743402992puff864|seashell|Tsp|Unknown|48|barn stpriableought| +12391|AAAAAAAAHGADAAAA|1997-10-27||Even usual teachers ought to sing even different likely males. Universal services expect kindly enou|2.32|1.25|7010002|univnameless #2|10|flatware|7|Home|42|ableese|N/A|3532burlywood5836723|lime|Each|Unknown|26|oughtn stpriableought| +12392|AAAAAAAAIGADAAAA|1997-10-27|2000-10-26|None the less new feelings build well only memories. Symptoms shall work more as ready councils. Political, intellectual courts argue. Always similar films should go here annual arguments. |1.26|0.78|4004001|edu packedu pack #1|4|athletic|4|Shoes|330|barpripri|large|60593105steel1247411|thistle|Cup|Unknown|17|ablen stpriableought| +12393|AAAAAAAAIGADAAAA|2000-10-27||None the less new feelings build well only memories. Symptoms shall work more as ready councils. Political, intellectual courts argue. Always similar films should go here annual arguments. |7.36|0.78|4004001|amalgcorp #2|4|birdal|6|Jewelry|90|barpripri|N/A|76tan503330783495297|steel|Cup|Unknown|36|prin stpriableought| +12394|AAAAAAAAKGADAAAA|1997-10-27|1999-10-27|Various countries teach large statements. |80.17|56.92|2004001|edu packimporto #1|4|sports-apparel|2|Men|293|prin stable|economy|924270113plum9057414|slate|Tbl|Unknown|50|esen stpriableought| +12395|AAAAAAAAKGADAAAA|1999-10-28|2001-10-26|Shared, complicated studies move with a feet; special, other ministers discuss years. Regular activities could guess then standar|7.87|3.06|7002006|importobrand #6|2|bedding|7|Home|512|ableoughtanti|N/A|0207590062772lime766|sandy|Tbl|Unknown|42|antin stpriableought| +12396|AAAAAAAAKGADAAAA|2001-10-27||Firms could not accelerate together finally huge comments. Young, new effects would|5.60|3.36|2002001|importoimporto #1|2|shirts|2|Men|651|oughtantically|large|0207590062772lime766|salmon|Case|Unknown|9|callyn stpriableought| +12397|AAAAAAAANGADAAAA|1997-10-27||Completely small items cry publicly personal investigations. Consequences need at a blues. Natural, |8.73|3.84|6010006|univbrand #6|10|jewelry boxes|6|Jewelry|316|callyoughtpri|N/A|362452832turquoise11|salmon|Dram|Unknown|33|ationn stpriableought| +12398|AAAAAAAAOGADAAAA|1997-10-27|2000-10-26|Only, important needs should think just classical programmes. Sha|0.24|0.21|8009009|maxinameless #9|9|optics|8|Sports|231|oughtpriable|N/A|229812354960368red39|rosy|Pound|Unknown|55|eingn stpriableought| +12399|AAAAAAAAOGADAAAA|2000-10-27||Real, legal sites ought to find so urban students. Teachers establish in general publicly modern orders. That is noble feet build joint, social consequences. Wide, aggress|79.11|0.21|5002002|importoscholar #2|9|country|5|Music|231|oughtpriable|N/A|984powder51415623678|powder|Case|Unknown|22|n stn stpriableought| +12400|AAAAAAAAAHADAAAA|1997-10-27|1999-10-27|Smooth years used to meet during a reasons. As careful questions function just products. Faces must prevent as fundamental, recent problems. More than mere crops may expla|2.66|1.11|2001001|amalgimporto #1|1|accessories|2|Men|499|n stn stese|large|3907puff795885534971|royal|Pallet|Unknown|31|barbareseableought| +12401|AAAAAAAAAHADAAAA|1999-10-28|2001-10-26|Students might agree british, strong profits. Suddenly national policies may carry local, original affairs. Prime, international aspects would use losses. Full, democrati|9.00|1.11|2001001|exportiunivamalg #6|13|self-help|9|Books|40|n stn stese|N/A|3907puff795885534971|steel|N/A|Unknown|30|oughtbareseableought| +12402|AAAAAAAAAHADAAAA|2001-10-27||Societies should sleep under the types. Minutes will dismiss enough. Determined, recent walls should sustain intently. Statutory values must take|2.38|1.33|2001001|importoimporto #1|13|shirts|2|Men|40|n stn stese|medium|3907puff795885534971|drab|Gross|Unknown|32|ablebareseableought| +12403|AAAAAAAADHADAAAA|1997-10-27||Experimental, content records used to eliminate honestly relevant stores. Now willing flats would see human residents. Substantial police can live; customers mean far arts. Else possible measur|9.09|7.90|3002002|importoexporti #2|2|infants|3|Children|409|n stbarese|medium|9133942987thistle840|ghost|Gram|Unknown|13|pribareseableought| +12404|AAAAAAAAEHADAAAA|1997-10-27|2000-10-26|Male hands shall make hungry, asian roots. Then polish lines te|6.32|2.27|10016012|corpamalgamalg #12|16|wireless|10|Electronics|240|bareseable|N/A|2970204780royal16526|thistle|Cup|Unknown|43|esebareseableought| +12405|AAAAAAAAEHADAAAA|2000-10-27||Male hands shall make hungry, asian roots. Then polish lines te|0.51|2.27|10016012|exportiimporto #2|3|pants|2|Men|240|bareseable|medium|2970204780royal16526|sky|Tsp|Unknown|31|antibareseableought| +12406|AAAAAAAAGHADAAAA|1997-10-27|1999-10-27|Premises shall take before from the experiences. Necessary standards include over blue, exciting copies. Great countries get questions. Universal crew|9.96|6.97|10009014|maxiunivamalg #14|9|televisions|10|Electronics|106|callybarought|N/A|42341819618586steel1|wheat|Dram|Unknown|16|callybareseableought| +12407|AAAAAAAAGHADAAAA|1999-10-28|2001-10-26|Premises shall take before from the experiences. Necessary standards include over blue, exciting copies. Great countries get questions. Universal crew|7.31|3.43|10002015|importounivamalg #15|2|camcorders|10|Electronics|407|ationbarese|N/A|42341819618586steel1|turquoise|N/A|Unknown|88|ationbareseableought| +12408|AAAAAAAAGHADAAAA|2001-10-27||Premises shall take before from the experiences. Necessary standards include over blue, exciting copies. Great countries get questions. Universal crew|5.35|3.43|7004009|edu packbrand #9|4|curtains/drapes|7|Home|96|callyn st|N/A|4337seashell76470315|magenta|Ton|Unknown|54|eingbareseableought| +12409|AAAAAAAAJHADAAAA|1997-10-27||Internal situations should happen institutions; arms shall come jus|4.80|2.49|1004002|edu packamalg #2|4|swimwear|1|Women|892|ablen steing|medium|5889609278plum093356|lime|Each|Unknown|13|n stbareseableought| +12410|AAAAAAAAKHADAAAA|1997-10-27|2000-10-26|New students create as in |2.03|1.01|4002001|importoedu pack #1|2|mens|4|Shoes|337|ationpripri|medium|96wheat2571904406562|yellow|Case|Unknown|10|baroughteseableought| +12411|AAAAAAAAKHADAAAA|2000-10-27||New students create as in |6.17|1.01|3001002|amalgexporti #2|2|newborn|3|Children|405|antibarese|large|96wheat2571904406562|medium|Ton|Unknown|24|oughtoughteseableought| +12412|AAAAAAAAMHADAAAA|1997-10-27|1999-10-27|Arti|3.23|2.81|2004001|edu packimporto #1|4|sports-apparel|2|Men|147|ationeseought|extra large|88medium510759040165|cyan|Ounce|Unknown|16|ableoughteseableought| +12413|AAAAAAAAMHADAAAA|1999-10-28|2001-10-26|Most other passages can collect even feet. Common flames maximise more pub|1.34|2.81|2004001|importoimporto #2|2|shirts|2|Men|293|prin stable|small|6201854272852pale122|lemon|Cup|Unknown|84|prioughteseableought| +12414|AAAAAAAAMHADAAAA|2001-10-27||Most other passages can collect even feet. Common flames maximise more pub|4.77|1.81|2004001|edu packimporto #1|2|sports-apparel|2|Men|565|anticallyanti|large|987spring05616267490|thistle|Bunch|Unknown|17|eseoughteseableought| +12415|AAAAAAAAPHADAAAA|1997-10-27||Desirable, important methods make thus observations. Most different tasks may live always traditional, concerned beings. Bad sales would lose. Long, linguistic pairs could not make. Chem|8.20|4.42|9003008|exportimaxi #8|3|computers|9|Books|147|ationeseought|N/A|17097orchid552931229|white|Cup|Unknown|2|antioughteseableought| +12416|AAAAAAAAAIADAAAA|1997-10-27|2000-10-26|Impossible fingers might report similar|2.14|1.17|2002001|importoimporto #1|2|shirts|2|Men|2|able|medium|73955155544white3700|navajo|Ounce|Unknown|78|callyoughteseableought| +12417|AAAAAAAAAIADAAAA|2000-10-27||Impossible fingers might report similar|82.84|24.85|8010008|univmaxi #8|2|pools|8|Sports|723|priableation|N/A|73955155544white3700|salmon|Gross|Unknown|20|ationoughteseableought| +12418|AAAAAAAACIADAAAA|1997-10-27|1999-10-27|Also general goals please deeply dirty, invisible functions. Estimated, expensive clients will recover never like a police. Emissions would |6.61|4.16|8015009|scholarmaxi #9|15|fishing|8|Sports|365|anticallypri|N/A|28757243351thistle63|seashell|Ton|Unknown|3|eingoughteseableought| +12419|AAAAAAAACIADAAAA|1999-10-28|2001-10-26|Also general goals please deeply dirty, invisible functions. Estimated, expensive clients will recover never like a police. Emissions would |8.76|4.16|8015009|corpcorp #4|6|rings|6|Jewelry|800|barbareing|N/A|28113752173159peach4|pale|Oz|Unknown|61|n stoughteseableought| +12420|AAAAAAAACIADAAAA|2001-10-27||Companies shall avoid in a friends; brown rights may not hold of course; nevertheless european changes give so|4.18|2.34|8015009|scholarbrand #5|15|custom|6|Jewelry|295|antin stable|N/A|28113752173159peach4|purple|Oz|Unknown|4|barableeseableought| +12421|AAAAAAAAFIADAAAA|1997-10-27||Pensioners must not release presidential poems. Reasons confirm just events. Silently p|2.23|1.02|3004002|edu packexporti #2|4|school-uniforms|3|Children|234|esepriable|large|5503709869blue273183|tomato|Pound|Unknown|19|oughtableeseableought| +12422|AAAAAAAAGIADAAAA|1997-10-27|2000-10-26|Definite symptoms reveal for a falls. Ages obtain easily for a societies. Great children open always values. More other con|3.47|2.15|10013013|exportiamalgamalg #13|13|stereo|10|Electronics|496|callyn stese|N/A|635075078indian66588|grey|Tsp|Unknown|13|ableableeseableought| +12423|AAAAAAAAGIADAAAA|2000-10-27||Definite symptoms reveal for a falls. Ages obtain easily for a societies. Great children open always values. More other con|36.01|24.48|1003002|exportiamalg #2|3|maternity|1|Women|264|callyn stese|small|635075078indian66588|sky|Pound|Unknown|42|priableeseableought| +12424|AAAAAAAAIIADAAAA|1997-10-27|1999-10-27|Elements mention faintly free railways. Pe|3.00|1.98|8007005|brandnameless #5|7|hockey|8|Sports|96|callyn st|N/A|19815711072mint96638|purple|Unknown|Unknown|11|eseableeseableought| +12425|AAAAAAAAIIADAAAA|1999-10-28|2001-10-26|Elements mention faintly free railways. Pe|7.31|1.98|8007005|namelesscorp #2|8|mens watch|6|Jewelry|96|callyn st|N/A|19815711072mint96638|chocolate|Tbl|Unknown|21|antiableeseableought| +12426|AAAAAAAAIIADAAAA|2001-10-27||Over personal parts say as as empty standards. Laws own by the children; behind powerful arm|0.69|0.48|9007007|brandmaxi #7|7|reference|9|Books|612|callyn st|N/A|910plum1625344036410|slate|Ton|Unknown|71|callyableeseableought| +12427|AAAAAAAALIADAAAA|1997-10-27||Certain courses may celebrate together mental fires. Incomes might listen. Deliberate problems should succeed shoes. Experienced costs find both royal, principal decisions. Cl|1.99|0.63|10016001|corpamalgamalg #1|16|wireless|10|Electronics|603|pribarcally|N/A|453751644slate330599|turquoise|Gross|Unknown|24|ationableeseableought| +12428|AAAAAAAAMIADAAAA|1997-10-27|2000-10-26|Perfec|1.77|1.46|6013005|exportibrand #5|13|loose stones|6|Jewelry|806|callybareing|N/A|973peach818501524947|rose|Cup|Unknown|3|eingableeseableought| +12429|AAAAAAAAMIADAAAA|2000-10-27||Perfec|3.30|1.28|6013005|amalgimporto #2|13|accessories|2|Men|47|ationese|large|29678384451541cyan24|maroon|Gram|Unknown|31|n stableeseableought| +12430|AAAAAAAAOIADAAAA|1997-10-27|1999-10-27|Able, variable others cannot see away other writers. Home poor|0.65|0.24|3003001|exportiexporti #1|3|toddlers|3|Children|533|priprianti|large|62110orchid099212222|black|Gross|Unknown|6|barprieseableought| +12431|AAAAAAAAOIADAAAA|1999-10-28|2001-10-26|Minutes ought to contact there bright, assistant brothers. Suddenly successful experiences divide suddenly over local lovers. Available teams shall provide ther|3.69|1.40|3003001|amalgedu pack #2|3|womens|4|Shoes|331|priprianti|petite|62110orchid099212222|rose|Carton|Unknown|20|oughtprieseableought| +12432|AAAAAAAAOIADAAAA|2001-10-27||Lands face. Warmly very demands must wish there alternative theories; properly constant situations shall keep important, national ears. Profits count as well horizontal|6.01|4.86|10001006|amalgunivamalg #6|1|cameras|10|Electronics|494|priprianti|N/A|62110orchid099212222|saddle|Unknown|Unknown|2|ableprieseableought| +12433|AAAAAAAABJADAAAA|1997-10-27||Needs work even strong proposals. Main windows can pursue dimensions. Necessarily golden centres try qui|3.74|1.94|1004002|edu packamalg #2|4|swimwear|1|Women|984|eseeingn st|large|6979261448orange2274|turquoise|Carton|Unknown|14|priprieseableought| +12434|AAAAAAAACJADAAAA|1997-10-27|2000-10-26|Countries take complicated, basic agencies. Small, final units can identify to a players. Fine stars shall want at all months. Fast social|2.82|1.24|2004001|edu packimporto #1|4|sports-apparel|2|Men|752|ableantiation|medium|5636993103410lace902|powder|Dram|Unknown|10|eseprieseableought| +12435|AAAAAAAACJADAAAA|2000-10-27||Countries take complicated, basic agencies. Small, final units can identify to a players. Fine stars shall want at all months. Fast social|9.45|1.24|2004001|corpnameless #6|4|furniture|7|Home|372|ableationpri|N/A|5636993103410lace902|rose|Bundle|Unknown|79|antiprieseableought| +12436|AAAAAAAAEJADAAAA|1997-10-27|1999-10-27|Here substantial groups must follow very. Relations must prove only easy, l|9.23|4.52|10007010|brandunivamalg #10|7|personal|10|Electronics|452|ableantiese|N/A|2923976694485white99|maroon|Tbl|Unknown|10|callyprieseableought| +12437|AAAAAAAAEJADAAAA|1999-10-28|2001-10-26|Experts seem a|3.48|2.74|10007010|scholarnameless #8|15|tables|7|Home|539|n stprianti|N/A|2923976694485white99|tomato|Gross|Unknown|31|ationprieseableought| +12438|AAAAAAAAEJADAAAA|2001-10-27||Experts seem a|6.35|2.74|8008009|namelessnameless #9|8|outdoor|8|Sports|539|n stprianti|N/A|9731floral6757224950|papaya|Gram|Unknown|9|eingprieseableought| +12439|AAAAAAAAHJADAAAA|1997-10-27||Often contemporary strategies shall not afford terms. Cities sit. Constitutional companies get now natural target|80.52|45.09|7008010|namelessbrand #10|8|lighting|7|Home|222|ableableable|N/A|91503674thistle92890|lace|Bundle|Unknown|21|n stprieseableought| +12440|AAAAAAAAIJADAAAA|1997-10-27|2000-10-26|In|7.89|4.81|6009001|maxicorp #1|9|womens watch|6|Jewelry|597|ationn stanti|N/A|578471floral64451862|pale|Bunch|Unknown|77|bareseeseableought| +12441|AAAAAAAAIJADAAAA|2000-10-27||In|8.64|4.81|2002002|importoimporto #2|9|shirts|2|Men|390|barn stpri|extra large|578471floral64451862|saddle|Case|Unknown|25|oughteseeseableought| +12442|AAAAAAAAKJADAAAA|1997-10-27|1999-10-27|Soviet communications think hence mechanical years. Facts colour also up a efforts. Immediate, brown rules stay also. Implications shall not understand at |2.78|0.83|1004001|edu packamalg #1|4|swimwear|1|Women|730|barpriation|petite|39498859005522sandy3|grey|Pallet|Unknown|34|ableeseeseableought| +12443|AAAAAAAAKJADAAAA|1999-10-28|2001-10-26|Soviet communications think hence mechanical years. Facts colour also up a efforts. Immediate, brown rules stay also. Implications shall not understand at |1.38|0.84|10015017|scholaramalgamalg #17|15|portable|10|Electronics|16|callyought|N/A|39498859005522sandy3|red|Gram|Unknown|65|prieseeseableought| +12444|AAAAAAAAKJADAAAA|2001-10-27||Excellent, rough papers see far. Critical ingredients show then from a years. Guns might represent anyway remote teachers; unlikely, complex wishes cannot say |1.73|1.28|10015017|importoedu pack #1|15|mens|4|Shoes|16|callyought|extra large|1pale138620610511397|lawn|Tsp|Unknown|3|eseeseeseableought| +12445|AAAAAAAANJADAAAA|1997-10-27||More empty units would work only national, giant standards. Modern, difficult examples |4.72|1.51|1004002|edu packamalg #2|4|swimwear|1|Women|778|eingationation|medium|9863059rosy370653698|maroon|Pallet|Unknown|19|antieseeseableought| +12446|AAAAAAAAOJADAAAA|1997-10-27|2000-10-26|Here holy prices sha|7.44|4.61|1003001|exportiamalg #1|3|maternity|1|Women|303|pribarpri|large|394813orchid36884124|metallic|Each|Unknown|16|callyeseeseableought| +12447|AAAAAAAAOJADAAAA|2000-10-27||Here holy prices sha|1.67|4.61|1003001|exportiimporto #2|3|pants|2|Men|303|pribarpri|medium|137908papaya62271128|white|Ton|Unknown|10|ationeseeseableought| +12448|AAAAAAAAAKADAAAA|1997-10-27|1999-10-27|Good homes get also standards; bad men will share then red, good societies. Left, thick scenes fill never capable, extra months. Times go merely parental, personal meetings. Lost, red residents can|8.06|5.80|4004001|edu packedu pack #1|4|athletic|4|Shoes|514|eseoughtanti|medium|1897571161577saddle8|yellow|Tsp|Unknown|24|eingeseeseableought| +12449|AAAAAAAAAKADAAAA|1999-10-28|2001-10-26|Married, new customers ensure vehicles. Maximum sons lift nuclear, structural sources. Crucial, poor r|25.59|5.80|3002002|importoexporti #2|4|infants|3|Children|514|eseoughtanti|medium|73493075059262tan101|yellow|Case|Unknown|26|n steseeseableought| +12450|AAAAAAAAAKADAAAA|2001-10-27||Married, new customers ensure vehicles. Maximum sons lift nuclear, structural sources. Crucial, poor r|6.48|3.11|3002002|edu packcorp #3|4|bracelets|6|Jewelry|35|antipri|N/A|73493075059262tan101|white|N/A|Unknown|23|barantieseableought| +12451|AAAAAAAADKADAAAA|1997-10-27||Square papers choose sometimes able expectations; partially lovely arguments guarantee so women. Co|1.86|1.22|3003002|exportiexporti #2|3|toddlers|3|Children|427|ationableese|medium|0591peach10795735874|grey|Cup|Unknown|85|oughtantieseableought| +12452|AAAAAAAAEKADAAAA|1997-10-27|2000-10-26|Happily good children maintain now classes. Political, old years see houses; of course new standards may find so sorry sounds; also |8.48|5.51|9010009|univunivamalg #9|10|travel|9|Books|118|eingoughtought|N/A|05641926318pale03579|green|Lb|Unknown|39|ableantieseableought| +12453|AAAAAAAAEKADAAAA|2000-10-27||Coming, extraordinary years put only, possible millions. Brown processes should join units. Otherwise large |1.35|0.85|9010009|exportiimporto #2|3|pants|2|Men|241|oughteseable|petite|05641926318pale03579|honeydew|Gross|Unknown|53|priantieseableought| +12454|AAAAAAAAGKADAAAA|1997-10-27|1999-10-27|Normal, sexual reasons laugh. Clouds shall sleep a bit just economic systems. Yellow, slim materials will hold for example for the strings. Traditional thoughts would not begin very again labo|8.76|5.16|1003001|exportiamalg #1|3|maternity|1|Women|94|esen st|petite|27metallic2832645996|rosy|Bundle|Unknown|31|eseantieseableought| +12455|AAAAAAAAGKADAAAA|1999-10-28|2001-10-26|Good journalists shall not get ancient legs. Bare homes upgr|5.97|5.16|1003001|importoamalg #2|3|fragrances|1|Women|427|ationableese|large|27metallic2832645996|plum|Gram|Unknown|28|antiantieseableought| +12456|AAAAAAAAGKADAAAA|2001-10-27||Good journalists shall not get ancient legs. Bare homes upgr|4.73|5.16|1003001|namelessunivamalg #13|8|scanners|10|Electronics|193|prin stought|N/A|630violet06701302161|white|Lb|Unknown|31|callyantieseableought| +12457|AAAAAAAAJKADAAAA|1997-10-27||Empty tests drown both studies. About effective girls may use by a shops; developing words hold quickly forces. General, availabl|7.58|3.94|4004002|edu packedu pack #2|4|athletic|4|Shoes|118|eingoughtought|extra large|6yellow7629962273944|purple|Carton|Unknown|91|ationantieseableought| +12458|AAAAAAAAKKADAAAA|1997-10-27|2000-10-26|Turkish, major words ought to l|3.58|2.43|10004015|edu packunivamalg #15|4|audio|10|Electronics|578|eingationanti|N/A|2931184304949smoke20|white|Bunch|Unknown|4|eingantieseableought| +12459|AAAAAAAAKKADAAAA|2000-10-27||Turkish, major words ought to l|1.73|1.15|10004015|exportiimporto #2|4|pants|2|Men|578|eingationanti|petite|06517067787seashell4|turquoise|N/A|Unknown|31|n stantieseableought| +12460|AAAAAAAAMKADAAAA|1997-10-27|1999-10-27|Brilliant, other pat|3.81|2.93|1001001|amalgamalg #1|1|dresses|1|Women|206|callybarable|medium|9243papaya5559507877|floral|Carton|Unknown|50|barcallyeseableought| +12461|AAAAAAAAMKADAAAA|1999-10-28|2001-10-26|Great profits understand so very domestic societies. Terms want sure. Successful, difficult cases add longer companies. As well crude troops |9.29|2.93|4002002|importoedu pack #2|2|mens|4|Shoes|206|callybarable|petite|9243papaya5559507877|white|Tbl|Unknown|45|oughtcallyeseableought| +12462|AAAAAAAAMKADAAAA|2001-10-27||National vehicles make. Teachers may not mind continually blue ways. As go|0.24|0.09|3001001|amalgexporti #1|2|newborn|3|Children|15|callybarable|extra large|9243papaya5559507877|cornsilk|Ounce|Unknown|60|ablecallyeseableought| +12463|AAAAAAAAPKADAAAA|1997-10-27||Reg|3.33|1.13|10011003|amalgamalgamalg #3|11|disk drives|10|Electronics|170|barationought|N/A|377646379092937pink2|peach|Tbl|Unknown|11|pricallyeseableought| +12464|AAAAAAAAALADAAAA|1997-10-27|2000-10-26|Concerned miles see suitable, certain networks. Powerful,|3.44|2.99|4002001|importoedu pack #1|2|mens|4|Shoes|335|antipripri|medium|09138375midnight5505|floral|Tsp|Unknown|23|esecallyeseableought| +12465|AAAAAAAAALADAAAA|2000-10-27||Great, thinking animals remain conditions. False ordinary provinces go married, brown days. Individuals con|5.10|4.18|4002001|importoscholar #2|2|country|5|Music|698|antipripri|N/A|09138375midnight5505|slate|Each|Unknown|18|anticallyeseableought| +12466|AAAAAAAACLADAAAA|1997-10-27|1999-10-27|Full problems might not split political, serious legs. Also particular minutes transmit thus healthy minute|6.75|2.70|9012011|importounivamalg #11|12|home repair|9|Books|120|barableought|N/A|8628red8311659576223|thistle|Cup|Unknown|26|callycallyeseableought| +12467|AAAAAAAACLADAAAA|1999-10-28|2001-10-26|Improvements would hear sometimes glad, possible customs. High women shall not sound terms. Popular organisers treat eastern factors. Today normal pictures use also. Days can make still clear|61.94|2.70|9012011|scholarunivamalg #7|5|karoke|10|Electronics|120|barableought|N/A|8628red8311659576223|violet|Ton|Unknown|14|ationcallyeseableought| +12468|AAAAAAAACLADAAAA|2001-10-27||Improvements would hear sometimes glad, possible customs. High women shall not sound terms. Popular organisers treat eastern factors. Today normal pictures use also. Days can make still clear|8.98|2.70|4002001|importoedu pack #1|5|mens|4|Shoes|120|barableought|petite|8628red8311659576223|tomato|Dram|Unknown|3|eingcallyeseableought| +12469|AAAAAAAAFLADAAAA|1997-10-27||Psychiatric scientists may not stay hopelessly. Full directors surrender really worldwide long days. Bright, shallow orders enjoy to the activities. Economic roads must not notice at least tall rules|2.48|1.36|7012010|importonameless #10|12|paint|7|Home|411|oughtoughtese|N/A|276495476purple09597|moccasin|Unknown|Unknown|1|n stcallyeseableought| +12470|AAAAAAAAGLADAAAA|1997-10-27|2000-10-26|Losses reassure by the observations. Generally elderly efforts shall begin o|7.71|3.46|5004001|edu packscholar #1|4|classical|5|Music|27|ationable|N/A|56tan165812863898398|snow|Tsp|Unknown|5|barationeseableought| +12471|AAAAAAAAGLADAAAA|2000-10-27||Gay scientists shall not defer well major times. Natural, public comparisons consider accurately adequate rates. Educational, adviso|0.95|0.32|5004001|brandcorp #8|7|pendants|6|Jewelry|27|ationable|N/A|56tan165812863898398|royal|Box|Unknown|55|oughtationeseableought| +12472|AAAAAAAAILADAAAA|1997-10-27|1999-10-27|Yesterday open eyes think to the years. Troops endorse consistently. Uses cope once. Miles contrast. Single children tell addition|0.33|0.17|4001001|amalgedu pack #1|1|womens|4|Shoes|237|ationpriable|medium|38medium618279071389|seashell|Pallet|Unknown|19|ableationeseableought| +12473|AAAAAAAAILADAAAA|1999-10-28|2001-10-26|Massive officers might keep well sounds. Magnificent, inte|8.72|7.58|4001001|importoamalg #2|2|fragrances|1|Women|179|n stationought|small|181350647yellow77716|seashell|Cup|Unknown|15|priationeseableought| +12474|AAAAAAAAILADAAAA|2001-10-27||Empirical changes point measures; solely likely customers enter maybe. Local, old wines must dare also as a years. Possibly constant letters used to make still full rows. Ce|2.74|7.58|7004005|edu packbrand #5|2|curtains/drapes|7|Home|179|n stationought|N/A|181350647yellow77716|red|N/A|Unknown|63|eseationeseableought| +12475|AAAAAAAALLADAAAA|1997-10-27||Financial, independent tears shall give as yet prime leaders. Very roots would|3.88|2.52|7006006|corpbrand #6|6|rugs|7|Home|575|antiationanti|N/A|96572lace84991313205|peru|Carton|Unknown|53|antiationeseableought| +12476|AAAAAAAAMLADAAAA|1997-10-27|2000-10-26|Concerns engage. Fans come heavily political, |0.39|0.16|5004001|edu packscholar #1|4|classical|5|Music|123|priableought|N/A|0722073994454white57|peru|Carton|Unknown|29|callyationeseableought| +12477|AAAAAAAAMLADAAAA|2000-10-27||Fundamental tables shall not catch partly from the duties. Locations want so. Sure incr|9.34|6.72|5004001|scholarbrand #8|5|blinds/shades|7|Home|214|priableought|N/A|0722073994454white57|peru|Lb|Unknown|36|ationationeseableought| +12478|AAAAAAAAOLADAAAA|1997-10-27|1999-10-27|Pati|0.76|0.58|3004001|edu packexporti #1|4|school-uniforms|3|Children|220|barableable|medium|deep8346654800658320|slate|Pound|Unknown|29|eingationeseableought| +12479|AAAAAAAAOLADAAAA|1999-10-28|2001-10-26|Proceedings see then equal quantities. New, complex magistrates overcome good, single institutions|7.33|2.34|3004001|amalgmaxi #12|1|arts|9|Books|564|esecallyanti|N/A|0313thistle522559700|azure|Pallet|Unknown|32|n stationeseableought| +12480|AAAAAAAAOLADAAAA|2001-10-27||Real workers will not expect much perhaps firm sectors. Detailed outcomes must hear final, clear sorts. Perhaps able corne|5.48|4.16|6013001|exportibrand #1|13|loose stones|6|Jewelry|141|esecallyanti|N/A|37896443tan442794037|white|Unknown|Unknown|17|bareingeseableought| +12481|AAAAAAAABMADAAAA|1997-10-27||Other offers demand across on a gates. Also natural employers look sensitive obje|3.83|1.72|7008002|namelessbrand #2|8|lighting|7|Home|83|prieing|N/A|6413tomato5273110504|peru|Pallet|Unknown|32|oughteingeseableought| +12482|AAAAAAAACMADAAAA|1997-10-27|2000-10-26|Most respective women go quite more vit|0.99|0.77|5002001|importoscholar #1|2|country|5|Music|120|barableought|N/A|puff4860164564672815|rose|Gram|Unknown|32|ableeingeseableought| +12483|AAAAAAAACMADAAAA|2000-10-27||Most respective women go quite more vit|4.85|3.39|5002001|brandbrand #4|7|decor|7|Home|120|barableought|N/A|puff4860164564672815|papaya|Lb|Unknown|11|prieingeseableought| +12484|AAAAAAAAEMADAAAA|1997-10-27|1999-10-27|Models shall face ever old children. Naturally similar ministers cannot stop sti|0.50|0.35|2002001|importoimporto #1|2|shirts|2|Men|262|ablecallyable|medium|01674ivory0171640531|red|Oz|Unknown|18|eseeingeseableought| +12485|AAAAAAAAEMADAAAA|1999-10-28|2001-10-26|Free, dependent police used to discuss low, spec|72.21|0.35|4004002|edu packedu pack #2|4|athletic|4|Shoes|262|ablecallyable|medium|456739425686rosy1438|blanched|N/A|Unknown|6|antieingeseableought| +12486|AAAAAAAAEMADAAAA|2001-10-27||P|3.95|0.35|9002007|importomaxi #7|2|business|9|Books|537|ablecallyable|N/A|456739425686rosy1438|khaki|Ton|Unknown|28|callyeingeseableought| +12487|AAAAAAAAHMADAAAA|1997-10-27||Willingly left requests declare changes; old lists ought to apply again in a arms. Students eat german, individual ships. Weak goods |5.83|1.92|9015008|scholarunivamalg #8|15|fiction|9|Books|143|prieseought|N/A|2149377944808powder8|lavender|Ounce|Unknown|15|ationeingeseableought| +12488|AAAAAAAAIMADAAAA|1997-10-27|2000-10-26|Names use hard months. Traditional, irish groups could want markedly operations. Islamic, great facilities choose. Possible s|4.34|2.08|9008009|namelessmaxi #9|8|romance|9|Books|142|ableeseought|N/A|4486steel34652849477|pale|Pound|Unknown|12|eingeingeseableought| +12489|AAAAAAAAIMADAAAA|2000-10-27||Sorry, american|0.77|2.08|9008009|scholarmaxi #10|5|history|9|Books|147|ationeseought|N/A|4486steel34652849477|puff|Ounce|Unknown|98|n steingeseableought| +12490|AAAAAAAAKMADAAAA|1997-10-27|1999-10-27|Areas shall not see. Also new records should see. Young pairs share adequate, gentle vehic|5.36|3.80|6006003|corpcorp #3|6|rings|6|Jewelry|555|antiantianti|N/A|6437386246rose514676|rose|Lb|Unknown|45|barn steseableought| +12491|AAAAAAAAKMADAAAA|1999-10-28|2001-10-26|Areas shall not see. Also new records should see. Young pairs share adequate, gentle vehic|3.36|3.80|6006003|brandunivamalg #14|7|personal|10|Electronics|555|antiantianti|N/A|07963038papaya292181|salmon|N/A|Unknown|31|oughtn steseableought| +12492|AAAAAAAAKMADAAAA|2001-10-27||Areas shall not see. Also new records should see. Young pairs share adequate, gentle vehic|3.17|0.98|2003001|exportiimporto #1|7|pants|2|Men|57|ationanti|small|6green18363686942252|rosy|Ounce|Unknown|37|ablen steseableought| +12493|AAAAAAAANMADAAAA|1997-10-27||Things help usually. Policemen get strong rivals. Powers wait. Public police would file today nuclear users. Public, able indicators must perform however beside a conditions. V|6.93|2.84|7002004|importobrand #4|2|bedding|7|Home|536|callyprianti|N/A|25085257468royal0506|sandy|Tsp|Unknown|31|prin steseableought| +12494|AAAAAAAAOMADAAAA|1997-10-27|2000-10-26|Vital books find always. Economic words share actually difficult, tired stones. Suitable projects may buy also such as a explanations. Black versions may differentiate much only, comparable arm|6.89|3.78|6016007|corpbrand #7|16|consignment|6|Jewelry|182|ableeingought|N/A|153lemon766417913633|turquoise|Lb|Unknown|50|esen steseableought| +12495|AAAAAAAAOMADAAAA|2000-10-27||Social businesses must create at first brilliant, old homes. Po|3.74|2.01|6016007|univbrand #8|16|jewelry boxes|6|Jewelry|182|ableeingought|N/A|153lemon766417913633|ghost|Box|Unknown|62|antin steseableought| +12496|AAAAAAAAANADAAAA|1997-10-27|1999-10-27|Certain photographs seize then pink events; weari|5.88|3.52|1001001|amalgamalg #1|1|dresses|1|Women|406|callybarese|extra large|54265285salmon742316|ghost|Dram|Unknown|50|callyn steseableought| +12497|AAAAAAAAANADAAAA|1999-10-28|2001-10-26|Certain photographs seize then pink events; weari|9.05|3.52|1001001|amalgimporto #2|1|accessories|2|Men|406|callybarese|extra large|54265285salmon742316|maroon|Gross|Unknown|88|ationn steseableought| +12498|AAAAAAAAANADAAAA|2001-10-27||Certain photographs seize then pink events; weari|1.30|1.02|3003001|exportiexporti #1|3|toddlers|3|Children|336|callypripri|small|54265285salmon742316|thistle|Cup|Unknown|53|eingn steseableought| +12499|AAAAAAAADNADAAAA|1997-10-27||Fast black days fight for a publishers; most other thoughts could suit circumstances; elections learn. Hot, black figures invite|91.62|28.40|4003002|exportiedu pack #2|3|kids|4|Shoes|180|bareingought|small|535239saddle74648435|sandy|Bundle|Unknown|35|n stn steseableought| +12500|AAAAAAAAENADAAAA|1997-10-27|2000-10-26|As legal parents can hide below very, functional colours. Questions wish even thick months. Prime, legal policies arrive always methods. More waste borders warrant in general standards. Now outstandin|4.02|3.25|10002006|importounivamalg #6|2|camcorders|10|Electronics|63|prically|N/A|56670429078white5098|blush|Bundle|Unknown|22|barbarantiableought| +12501|AAAAAAAAENADAAAA|2000-10-27||Distinguished books must ascertain from a practices. Inside current boots would list no doubt stu|9.28|4.45|10002006|importoimporto #2|2|shirts|2|Men|254|eseantiable|medium|azure307335361564149|peru|Ton|Unknown|65|oughtbarantiableought| +12502|AAAAAAAAGNADAAAA|1997-10-27|1999-10-27|Difficult, legislative numbers can manage more really naval|4.21|3.49|1001001|amalgamalg #1|1|dresses|1|Women|592|ablen stanti|large|49242aquamarine91008|burlywood|Each|Unknown|57|ablebarantiableought| +12503|AAAAAAAAGNADAAAA|1999-10-28|2001-10-26|Boys can excuse at first responsible, powerful product|18.04|3.49|9009012|maximaxi #12|1|science|9|Books|592|ablen stanti|N/A|49242aquamarine91008|wheat|Pound|Unknown|47|pribarantiableought| +12504|AAAAAAAAGNADAAAA|2001-10-27||Boys can excuse at first responsible, powerful product|3.22|3.49|9009012|importoedu pack #1|2|mens|4|Shoes|592|ablen stanti|petite|45043aquamarine28825|slate|Carton|Unknown|48|esebarantiableought| +12505|AAAAAAAAJNADAAAA|1997-10-27||Late, available estates may |5.15|4.37|4002002|importoedu pack #2|2|mens|4|Shoes|66|callycally|medium|605135moccasin239969|khaki|Tbl|Unknown|41|antibarantiableought| +12506|AAAAAAAAKNADAAAA|1997-10-27|2000-10-26|Technical things would speak often right, long factors. Comfortable, relevant prices may tell here over simple needs. Nations wi|1.31|0.44|1003001|exportiamalg #1|3|maternity|1|Women|128|eingableought|large|298464papaya85160853|pink|Bunch|Unknown|13|callybarantiableought| +12507|AAAAAAAAKNADAAAA|2000-10-27||Substantially other streets will attack to a hundreds. Physical minerals shall |4.53|0.44|1003001|edu packedu pack #2|3|athletic|4|Shoes|300|barbarpri|medium|298464papaya85160853|light|Unknown|Unknown|25|ationbarantiableought| +12508|AAAAAAAAMNADAAAA|1997-10-27|1999-10-27|Provisions shall stand dead individuals. Standards involve including the gardens. Discussions offer simple purposes. Further certain facts cannot learn carefull|9.12|3.00|2002001|importoimporto #1|2|shirts|2|Men|229|n stableable|N/A|17snow54619543381387|sky|Pound|Unknown|27|eingbarantiableought| +12509|AAAAAAAAMNADAAAA|1999-10-28|2001-10-26|Short, competitive eyes suffer however at all concerned earnings. Best following guests would think able, financial hours; nuclear facts will |1.31|0.99|2002001|exportiexporti #2|2|toddlers|3|Children|229|n stableable|large|352pink8144149913287|slate|Pallet|Unknown|6|n stbarantiableought| +12510|AAAAAAAAMNADAAAA|2001-10-27||Short, competitive eyes suffer however at all concerned earnings. Best following guests would think able, financial hours; nuclear facts will |21.94|11.62|2002001|exportischolar #1|3|pop|5|Music|229|n stableable|N/A|5140636771peach55415|peach|Bunch|Unknown|49|baroughtantiableought| +12511|AAAAAAAAPNADAAAA|1997-10-27||Areas might not make used, other years. Social, sensible problems may resign new arms. Black reaso|2.95|1.32|5003002|exportischolar #2|3|pop|5|Music|592|ablen stanti|N/A|036497617057wheat089|chiffon|Box|Unknown|66|oughtoughtantiableought| +12512|AAAAAAAAAOADAAAA|1997-10-27|2000-10-26|Before difficult years ought to override parents. English, true women must provoke more severe times. Only, short cases may not spend usually by a objects|9.10|7.28|4002001|importoedu pack #1|2|mens|4|Shoes|96|callyn st|large|650medium68033100322|sky|Ton|Unknown|21|ableoughtantiableought| +12513|AAAAAAAAAOADAAAA|2000-10-27||Before difficult years ought to override parents. English, true women must provoke more severe times. Only, short cases may not spend usually by a objects|1.76|1.37|3002002|importoexporti #2|2|infants|3|Children|96|callyn st|medium|650medium68033100322|sandy|Cup|Unknown|67|prioughtantiableought| +12514|AAAAAAAACOADAAAA|1997-10-27|1999-10-27|Studies support for example following schools. Rational products must enable. British motives must not kick. Pp. decide in a girls. As good as |75.23|60.93|5002001|importoscholar #1|2|country|5|Music|271|oughtationable|N/A|7388849446thistle603|lawn|Oz|Unknown|43|eseoughtantiableought| +12515|AAAAAAAACOADAAAA|1999-10-28|2001-10-26|Other tories observe under quite sure services; major, new differences crumble lights. As national officers can use following,|6.10|60.93|5002001|importoamalg #2|2|fragrances|1|Women|318|eingoughtpri|medium|776tan55943484973241|yellow|Bundle|Unknown|27|antioughtantiableought| +12516|AAAAAAAACOADAAAA|2001-10-27||Other tories observe under quite sure services; major, new differences crumble lights. As national officers can use following,|9.74|60.93|2002001|importoimporto #1|2|shirts|2|Men|318|eingoughtpri|small|776tan55943484973241|salmon|Tsp|Unknown|11|callyoughtantiableought| +12517|AAAAAAAAFOADAAAA|1997-10-27||Absolutely important customers may get detailed, correct bases. Busily trying clothes shall hear more into a types. Slight products would feel best mi|6.40|3.20|3003002|exportiexporti #2|3|toddlers|3|Children|165|anticallyought|medium|5980570814peru569189|purple|Box|Unknown|59|ationoughtantiableought| +12518|AAAAAAAAGOADAAAA|1997-10-27|2000-10-26|Free, other sons might spread. Also |9.68|7.35|10001007|amalgunivamalg #7|1|cameras|10|Electronics|48|eingese|N/A|8280649197745rosy274|spring|Each|Unknown|64|eingoughtantiableought| +12519|AAAAAAAAGOADAAAA|2000-10-27||Other planes evolve financial seasons. Rare children must see now. Fas|25.54|20.94|3004002|edu packexporti #2|1|school-uniforms|3|Children|381|eingese|petite|8280649197745rosy274|rose|Gram|Unknown|7|n stoughtantiableought| +12520|AAAAAAAAIOADAAAA|1997-10-27|1999-10-27|Major surfaces rush cups. Good letters must meet here losses. Different, short decisions learn theories. Following payments remain perhaps intermediate properties. Surprised, awful p|0.95|0.68|1002001|importoamalg #1|2|fragrances|1|Women|884|eseeingeing|medium|858peach814697851167|sienna|Lb|Unknown|7|barableantiableought| +12521|AAAAAAAAIOADAAAA|1999-10-28|2001-10-26|Major surfaces rush cups. Good letters must meet here losses. Different, short decisions learn theories. Following payments remain perhaps intermediate properties. Surprised, awful p|7.43|5.34|1002001|edu packamalg #2|2|swimwear|1|Women|152|ableantiought|medium|858peach814697851167|papaya|Each|Unknown|53|oughtableantiableought| +12522|AAAAAAAAIOADAAAA|2001-10-27||Major surfaces rush cups. Good letters must meet here losses. Different, short decisions learn theories. Following payments remain perhaps intermediate properties. Surprised, awful p|2.89|5.34|1002001|edu packexporti #1|2|school-uniforms|3|Children|94|ableantiought|large|858peach814697851167|khaki|Ounce|Unknown|12|ableableantiableought| +12523|AAAAAAAALOADAAAA|1997-10-27||Great classes marry. Wooden employees seize communities. Complete days upgrade again under a contributions; contemporary, black neighbours like early on a arrangement|9.56|3.05|4002002|importoedu pack #2|2|mens|4|Shoes|222|ableableable|medium|042moccasin847479515|seashell|Dram|Unknown|59|priableantiableought| +12524|AAAAAAAAMOADAAAA|1997-10-27|2000-10-26|Reforms find long-term, severe pupils. Auto|2.24|0.69|4004001|edu packedu pack #1|4|athletic|4|Shoes|22|ableable|large|7376448881339papaya8|wheat|Unknown|Unknown|65|eseableantiableought| +12525|AAAAAAAAMOADAAAA|2000-10-27||Reforms find long-term, severe pupils. Auto|5.52|1.87|4004001|exportiunivamalg #14|4|dvd/vcr players|10|Electronics|321|ableable|N/A|7376448881339papaya8|navy|Lb|Unknown|23|antiableantiableought| +12526|AAAAAAAAOOADAAAA|1997-10-27|1999-10-27|Net, deep powers could order immediately in a families. Minutes get personal, professional days|4.65|3.58|1003001|exportiamalg #1|3|maternity|1|Women|233|pripriable|small|0313575honeydew10337|turquoise|Unknown|Unknown|25|callyableantiableought| +12527|AAAAAAAAOOADAAAA|1999-10-28|2001-10-26|Net, deep powers could order immediately in a families. Minutes get personal, professional days|2.72|3.58|1003001|amalgamalgamalg #16|11|disk drives|10|Electronics|233|pripriable|N/A|6033499675821wheat55|white|Ounce|Unknown|90|ationableantiableought| +12528|AAAAAAAAOOADAAAA|2001-10-27||Personal teams used to catch equal events. Others shall |4.93|3.58|1003001|brandunivamalg #17|7|personal|10|Electronics|233|pripriable|N/A|6033499675821wheat55|saddle|Dram|Unknown|69|eingableantiableought| +12529|AAAAAAAABPADAAAA|1997-10-27||Certainly severe ch|58.94|41.84|3001002|amalgexporti #2|1|newborn|3|Children|308|eingbarpri|medium|03indian912174051996|navajo|Gram|Unknown|17|n stableantiableought| +12530|AAAAAAAACPADAAAA|1997-10-27|2000-10-26|Large, small jobs might harbour foreign, social plans; totally only rivers spread maybe old lines. Perfect, european friends look long, ordinary states|1.97|0.80|3004001|edu packexporti #1|4|school-uniforms|3|Children|293|prin stable|petite|53696misty1702878194|yellow|Ton|Unknown|93|barpriantiableought| +12531|AAAAAAAACPADAAAA|2000-10-27||Large, small jobs might harbour foreign, social plans; totally only rivers spread maybe old lines. Perfect, european friends look long, ordinary states|84.56|0.80|10010003|univamalgamalg #3|4|memory|10|Electronics|293|prin stable|N/A|53696misty1702878194|lavender|Bunch|Unknown|39|oughtpriantiableought| +12532|AAAAAAAAEPADAAAA|1997-10-27|1999-10-27|Old, informal hours need excellent boards. G|2.07|0.66|5003001|exportischolar #1|3|pop|5|Music|457|ationantiese|N/A|5royal65330558881860|tan|Lb|Unknown|8|ablepriantiableought| +12533|AAAAAAAAEPADAAAA|1999-10-28|2001-10-26|Badly back designers limit right things. Exotic, modern lines signal perhaps. Rationally unexpected flowers set.|3.06|2.47|5003001|exportiedu pack #2|3|kids|4|Shoes|540|ationantiese|large|5royal65330558881860|burnished|Gram|Unknown|29|pripriantiableought| +12534|AAAAAAAAEPADAAAA|2001-10-27||Always asleep fingers shall let commercial, certain signs; obvious men should not safeguar|5.52|3.09|5003001|amalgscholar #1|3|rock|5|Music|175|ationantiese|N/A|sandy962168837564276|navy|Ounce|Unknown|97|esepriantiableought| +12535|AAAAAAAAHPADAAAA|1997-10-27||Used, young sizes take requirements. Electoral, standard stones worry still private scenes. Major, still bedrooms say all once effective years. Long new moments will own after the|9.19|4.13|9003008|exportimaxi #8|3|computers|9|Books|996|callyn stn st|N/A|3salmon6066803919780|red|Carton|Unknown|36|antipriantiableought| +12536|AAAAAAAAIPADAAAA|1997-10-27|2000-10-26|Orders go into the documents. Social, existing specialists will seem twice associated wishes. Finally nation|5.15|4.27|9002009|importomaxi #9|2|business|9|Books|94|esen st|N/A|45slate1327328481415|pink|Ton|Unknown|58|callypriantiableought| +12537|AAAAAAAAIPADAAAA|2000-10-27||Orders go into the documents. Social, existing specialists will seem twice associated wishes. Finally nation|3.22|4.27|9002009|maximaxi #10|2|science|9|Books|298|eingn stable|N/A|09823429sienna281684|wheat|Lb|Unknown|9|ationpriantiableought| +12538|AAAAAAAAKPADAAAA|1997-10-27|1999-10-27|Soldiers assemble current sales; home foreign projects must stop then strict, full categories. Parties give profoundly up a mothers. Others derive general |6.98|2.44|3003001|exportiexporti #1|3|toddlers|3|Children|9|n st|extra large|7112606plum857877188|goldenrod|Gross|Unknown|14|eingpriantiableought| +12539|AAAAAAAAKPADAAAA|1999-10-28|2001-10-26|Great things can find already; exotic possibilities use |0.50|0.21|3003001|corpnameless #10|3|furniture|7|Home|204|n st|N/A|303223hot00539947028|saddle|Cup|Unknown|52|n stpriantiableought| +12540|AAAAAAAAKPADAAAA|2001-10-27||Mainly financial forests|8.03|0.21|1004001|edu packamalg #1|3|swimwear|1|Women|156|callyantiought|medium|96785119171spring156|yellow|Each|Unknown|25|bareseantiableought| +12541|AAAAAAAANPADAAAA|1997-10-27||Secrets will not walk. Plates could listen. Various arms may|4.48|2.91|2002002|importoimporto #2|2|shirts|2|Men|95|antin st|extra large|88629949146985peach0|grey|Gross|Unknown|21|oughteseantiableought| +12542|AAAAAAAAOPADAAAA|1997-10-27|2000-10-26|Right futures announce to a decisions; immense, structural shoulders make italian, gold conditions. Activities roam mo|2.80|2.15|9013003|exportiunivamalg #3|13|self-help|9|Books|971|oughtationn st|N/A|269orchid48580119818|salmon|Each|Unknown|12|ableeseantiableought| +12543|AAAAAAAAOPADAAAA|2000-10-27||Right futures announce to a decisions; immense, structural shoulders make italian, gold conditions. Activities roam mo|84.25|70.77|9013003|edu packbrand #8|14|estate|6|Jewelry|177|ationationought|N/A|401522466025rosy9895|turquoise|Gram|Unknown|55|prieseantiableought| +12544|AAAAAAAAAABDAAAA|1997-10-27|1999-10-27|Good, other flats forget literally physical years. Indeed complete sales shall not|4.98|2.24|7002005|importobrand #5|2|bedding|7|Home|117|ationoughtought|N/A|011963687wheat734866|sandy|Dram|Unknown|19|eseeseantiableought| +12545|AAAAAAAAAABDAAAA|1999-10-28|2001-10-26|Extended, central centuries ought to predict thus early controls; so financial years shall not see|1.36|0.88|2004002|edu packimporto #2|4|sports-apparel|2|Men|458|ationoughtought|petite|sandy367258277291423|yellow|Gram|Unknown|11|antieseantiableought| +12546|AAAAAAAAAABDAAAA|2001-10-27||Far, functional services would not provide warm foreigners; personal songs go most right sites; words assess past, long owners. Current, aware players challenge comp|2.58|2.16|2004002|amalgimporto #1|4|accessories|2|Men|658|eingantically|medium|40983564green2012254|snow|Tsp|Unknown|38|callyeseantiableought| +12547|AAAAAAAADABDAAAA|1997-10-27||Quite american days destroy commonly |8.90|3.64|6010004|univbrand #4|10|jewelry boxes|6|Jewelry|305|antibarpri|N/A|2049papaya8453307244|saddle|Gross|Unknown|24|ationeseantiableought| +12548|AAAAAAAAEABDAAAA|1997-10-27|2000-10-26|Clearly short talks disentangle especially with a systems. Frequently new sides could honour actually wrong personal attempts. Estimated needs ought to think highly|3.04|2.46|8009009|maxinameless #9|9|optics|8|Sports|531|oughtprianti|N/A|19639wheat3888938176|tomato|Bundle|Unknown|30|eingeseantiableought| +12549|AAAAAAAAEABDAAAA|2000-10-27||Clearly short talks disentangle especially with a systems. Frequently new sides could honour actually wrong personal attempts. Estimated needs ought to think highly|0.98|0.76|8009009|edu packscholar #2|4|classical|5|Music|531|oughtprianti|N/A|7604166snow557062751|blush|Case|Unknown|28|n steseantiableought| +12550|AAAAAAAAGABDAAAA|1997-10-27|1999-10-27|Ordinary, surprised territories might incorporate. Elections would explain. Coming looks used to change different, central goods. British, appropriate roots shall not m|8.71|7.83|3001001|amalgexporti #1|1|newborn|3|Children|13|priought|extra large|166192violet66124074|saddle|Cup|Unknown|10|barantiantiableought| +12551|AAAAAAAAGABDAAAA|1999-10-28|2001-10-26|Ordinary, surprised territories might incorporate. Elections would explain. Coming looks used to change different, central goods. British, appropriate roots shall not m|0.55|7.83|5002002|importoscholar #2|1|country|5|Music|13|priought|N/A|92578220691honeydew7|wheat|Ounce|Unknown|45|oughtantiantiableought| +12552|AAAAAAAAGABDAAAA|2001-10-27||Products name more fine arms. Eyes protect positive claims. Large courts risk anyway of course administrative systems. Arts used to inherit intensely effects. Instru|8.19|7.83|5002002|amalgnameless #3|1|accent|7|Home|208|eingbarable|N/A|53blush1809179089988|indian|Tsp|Unknown|13|ableantiantiableought| +12553|AAAAAAAAJABDAAAA|1997-10-27||Well religious claims would occur propo|2.96|1.33|5004002|edu packscholar #2|4|classical|5|Music|210|baroughtable|N/A|5386182thistle516088|peach|N/A|Unknown|48|priantiantiableought| +12554|AAAAAAAAKABDAAAA|1997-10-27|2000-10-26|Hardly good hundreds could give with a windows. National, competitive cuts go great countries. Complete arms sh|2.28|1.41|1003001|exportiamalg #1|3|maternity|1|Women|512|ableoughtanti|petite|923000244141deep9749|rosy|Carton|Unknown|11|eseantiantiableought| +12555|AAAAAAAAKABDAAAA|2000-10-27||Hardly good hundreds could give with a windows. National, competitive cuts go great countries. Complete arms sh|4.91|2.20|1003001|exportiamalg #2|3|maternity|1|Women|512|ableoughtanti|large|923000244141deep9749|red|Bunch|Unknown|28|antiantiantiableought| +12556|AAAAAAAAMABDAAAA|1997-10-27|1999-10-27|F|9.20|6.80|2001001|amalgimporto #1|1|accessories|2|Men|181|oughteingought|economy|5070992peru678490525|peach|N/A|Unknown|21|callyantiantiableought| +12557|AAAAAAAAMABDAAAA|1999-10-28|2001-10-26|Small doors find. Years must come particularly european lists. Usually old occupations take both. Typical beds might n|3.73|6.80|2001002|amalgimporto #2|1|accessories|2|Men|181|oughteingought|large|97334purple327686002|plum|Cup|Unknown|8|ationantiantiableought| +12558|AAAAAAAAMABDAAAA|2001-10-27||Never upper affairs might order young matters. |63.24|47.43|1001001|amalgamalg #1|1|dresses|1|Women|102|ablebarought|small|97334purple327686002|indian|Tsp|Unknown|18|eingantiantiableought| +12559|AAAAAAAAPABDAAAA|1997-10-27||Hardly historical dollars combine quit|3.32|2.72|8016010|corpmaxi #10|16|golf|8|Sports|176|callyationought|N/A|39peach1337531885657|frosted|Oz|Unknown|27|n stantiantiableought| +12560|AAAAAAAAABBDAAAA|1997-10-27|2000-10-26|Unnecessary years appear free members. Texts |1.49|0.64|7008001|namelessbrand #1|8|lighting|7|Home|500|barbaranti|N/A|26072242641white4565|violet|Gross|Unknown|67|barcallyantiableought| +12561|AAAAAAAAABBDAAAA|2000-10-27||Either superb corners ought to force false seats. Systematic, wooden classes cannot make very large,|0.76|0.64|8008002|namelessnameless #2|8|outdoor|8|Sports|500|barbaranti|N/A|27steel3387707130894|chiffon|Ounce|Unknown|66|oughtcallyantiableought| +12562|AAAAAAAACBBDAAAA|1997-10-27|1999-10-27|Current |2.84|1.67|4003001|exportiedu pack #1|3|kids|4|Shoes|207|ationbarable|large|37620998slate4909714|plum|Unknown|Unknown|24|ablecallyantiableought| +12563|AAAAAAAACBBDAAAA|1999-10-28|2001-10-26|Current |0.92|0.76|4003001|exportiamalg #2|3|maternity|1|Women|417|ationoughtese|large|37620998slate4909714|violet|Dozen|Unknown|32|pricallyantiableought| +12564|AAAAAAAACBBDAAAA|2001-10-27||Rare actions earn initially additional, silly years. Cards used to hide almost. Altogether political populations investigate. Trees can secure as cigarettes. Both difficult instruments |4.94|0.76|4003001|univunivamalg #1|10|travel|9|Books|417|ationoughtese|N/A|53216royal1579728228|pink|Lb|Unknown|10|esecallyantiableought| +12565|AAAAAAAAFBBDAAAA|1997-10-27||Advisory, black instances seem for a eggs. Russian, due resources might follow more other systems. Build|9.01|3.15|1004002|edu packamalg #2|4|swimwear|1|Women|441|oughteseese|petite|319972tan85756400456|plum|N/A|Unknown|85|anticallyantiableought| +12566|AAAAAAAAGBBDAAAA|1997-10-27|2000-10-26|Live, important witnesses take. Str|1.28|0.44|5001001|amalgscholar #1|1|rock|5|Music|107|ationbarought|N/A|82salmon983214307112|papaya|Pound|Unknown|21|callycallyantiableought| +12567|AAAAAAAAGBBDAAAA|2000-10-27||Live, important witnesses take. Str|97.43|64.30|10012005|importoamalgamalg #5|1|monitors|10|Electronics|17|ationbarought|N/A|82salmon983214307112|navajo|Pallet|Unknown|24|ationcallyantiableought| +12568|AAAAAAAAIBBDAAAA|1997-10-27|1999-10-27|Examples ignore workers. Indeed similar weekends give inc, heavy police. Certain, new area|7.89|7.02|4002001|importoedu pack #1|2|mens|4|Shoes|570|barationanti|large|8399violet3012057393|red|Bundle|Unknown|58|eingcallyantiableought| +12569|AAAAAAAAIBBDAAAA|1999-10-28|2001-10-26|Examples ignore workers. Indeed similar weekends give inc, heavy police. Certain, new area|2.94|1.76|3001002|amalgexporti #2|2|newborn|3|Children|881|barationanti|petite|7866804390metallic18|sandy|Tsp|Unknown|55|n stcallyantiableought| +12570|AAAAAAAAIBBDAAAA|2001-10-27||Merely only bodies make buildings. Narrow, prime boundaries shall not come temporary practitioners. Personal, successful bars include new issues. Styles shall favour easy in a marks. Real, beaut|8.60|1.76|4001001|amalgedu pack #1|2|womens|4|Shoes|881|barationanti|petite|768cyan3687662764440|grey|Tbl|Unknown|32|barationantiableought| +12571|AAAAAAAALBBDAAAA|1997-10-27||Human offences shall develop environmental, lesser matters. Thereafter critical ro|2.58|1.29|4004002|edu packedu pack #2|4|athletic|4|Shoes|638|eingprically|extra large|07723939gainsboro369|seashell|Carton|Unknown|36|oughtationantiableought| +12572|AAAAAAAAMBBDAAAA|1997-10-27|2000-10-26|Total, hard fingers ought to require in order in the years. Small, private contents might care costs. More fine versions look y|6.38|1.97|1001001|amalgamalg #1|1|dresses|1|Women|594|esen stanti|petite|23775plum06356927988|sandy|Cup|Unknown|24|ableationantiableought| +12573|AAAAAAAAMBBDAAAA|2000-10-27||Total, hard fingers ought to require in order in the years. Small, private contents might care costs. More fine versions look y|1.33|0.85|1001001|amalgunivamalg #10|11|cooking|9|Books|73|esen stanti|N/A|0083slate86500073259|yellow|Tsp|Unknown|77|priationantiableought| +12574|AAAAAAAAOBBDAAAA|1997-10-27|1999-10-27|Facilities move members; altogether capable folk see early social talks. Operations buy thus; also essential goods must implement long important parents. Eyes may not suspect regardless available cond|7.73|5.87|2001001|amalgimporto #1|1|accessories|2|Men|436|callypriese|petite|06pink34273450002430|royal|Oz|Unknown|45|eseationantiableought| +12575|AAAAAAAAOBBDAAAA|1999-10-28|2001-10-26|Direct services keep quickly for a measures. Necessary areas prevent but in a prices. Games include however local books. Well confident years might repeat; concepts shall stress. |6.72|4.83|2001001|importoamalg #2|2|fragrances|1|Women|344|eseesepri|extra large|06pink34273450002430|maroon|Each|Unknown|5|antiationantiableought| +12576|AAAAAAAAOBBDAAAA|2001-10-27||Direct services keep quickly for a measures. Necessary areas prevent but in a prices. Games include however local books. Well confident years might repeat; concepts shall stress. |74.80|4.83|9013001|exportiunivamalg #1|13|self-help|9|Books|165|eseesepri|N/A|822718531945555peru5|steel|Each|Unknown|6|callyationantiableought| +12577|AAAAAAAABCBDAAAA|1997-10-27||Beautiful auditors kill for a minutes. Generally eastern conditions call there around a points. Remarkably new kinds cannot understand accurately complex other y|8.61|5.07|3002002|importoexporti #2|2|infants|3|Children|2|able|petite|360521923727salmon10|papaya|Gram|Unknown|20|ationationantiableought| +12578|AAAAAAAACCBDAAAA|1997-10-27|2000-10-26|Governors will collect systems. Objectives may feel however leading children. Conditions need locall|4.66|2.70|8011009|amalgmaxi #9|11|archery|8|Sports|123|priableought|N/A|261501white890094368|firebrick|Each|Unknown|28|eingationantiableought| +12579|AAAAAAAACCBDAAAA|2000-10-27||Governors will collect systems. Objectives may feel however leading children. Conditions need locall|1.85|1.38|2003002|exportiimporto #2|3|pants|2|Men|123|priableought|extra large|261501white890094368|sky|Gram|Unknown|66|n stationantiableought| +12580|AAAAAAAAECBDAAAA|1997-10-27|1999-10-27|Forces can testify happy, international levels. Performances pay right bands. Items could discourage even in a months; readers simplify ea|0.09|0.02|9015005|scholarunivamalg #5|15|fiction|9|Books|303|pribarpri|N/A|278709567499navy2614|rosy|Each|Unknown|5|bareingantiableought| +12581|AAAAAAAAECBDAAAA|1999-10-28|2001-10-26|Forces can testify happy, international levels. Performances pay right bands. Items could discourage even in a months; readers simplify ea|9.21|0.02|8011002|amalgmaxi #2|15|archery|8|Sports|303|pribarpri|N/A|36875037013floral203|white|Gram|Unknown|29|oughteingantiableought| +12582|AAAAAAAAECBDAAAA|2001-10-27||Even different spirits tr|5.87|2.46|8011002|edu packexporti #1|15|school-uniforms|3|Children|303|pribarpri|small|36875037013floral203|navy|Tsp|Unknown|14|ableeingantiableought| +12583|AAAAAAAAHCBDAAAA|1997-10-27||Then great bombs used to explain more direct social problems. In addition early increases put lately. Gay |0.43|0.15|7012004|importonameless #4|12|paint|7|Home|535|antiprianti|N/A|8burnished5409981045|papaya|N/A|Unknown|22|prieingantiableought| +12584|AAAAAAAAICBDAAAA|1997-10-27|2000-10-26|Plans plan indeed special weeks. Psychiatric boys produce. Around key symptoms attempt as a matter of fact materials. Available, respective benefits will ma|0.78|0.24|7012005|importonameless #5|12|paint|7|Home|265|anticallyable|N/A|48373020orange871379|slate|Unknown|Unknown|44|eseeingantiableought| +12585|AAAAAAAAICBDAAAA|2000-10-27||Plans plan indeed special weeks. Psychiatric boys produce. Around key symptoms attempt as a matter of fact materials. Available, respective benefits will ma|0.27|0.24|7012005|amalgmaxi #6|11|archery|8|Sports|666|callycallycally|N/A|48373020orange871379|plum|Lb|Unknown|57|antieingantiableought| +12586|AAAAAAAAKCBDAAAA|1997-10-27|1999-10-27|Right, daily meals say someti|96.35|29.86|8004007|edu packnameless #7|4|camping|8|Sports|133|pripriought|N/A|3190841tan5595976031|gainsboro|Tsp|Unknown|56|callyeingantiableought| +12587|AAAAAAAAKCBDAAAA|1999-10-28|2001-10-26|Right, daily meals say someti|2.84|1.19|10011008|amalgamalgamalg #8|11|disk drives|10|Electronics|133|pripriought|N/A|3190841tan5595976031|seashell|Cup|Unknown|28|ationeingantiableought| +12588|AAAAAAAAKCBDAAAA|2001-10-27||Right, daily meals say someti|1.92|1.32|10006009|corpunivamalg #9|11|musical|10|Electronics|101|pripriought|N/A|3190841tan5595976031|sky|Each|Unknown|38|eingeingantiableought| +12589|AAAAAAAANCBDAAAA|1997-10-27||White times examine products. Alone, square examples used to get highly. Willing chairs must not conjure immediately recent members; northern societies may seem properly p|3.44|1.89|9012002|importounivamalg #2|12|home repair|9|Books|395|antin stpri|N/A|693094687052rosy0464|lace|Box|Unknown|61|n steingantiableought| +12590|AAAAAAAAOCBDAAAA|1997-10-27|2000-10-26|Progressive surveys succeed still. Systems lack so willing, good names. Years shall not talk really to a buses. Panels fulfil international genes.|7.11|5.68|4003001|exportiedu pack #1|3|kids|4|Shoes|75|antiation|petite|9318660saddle2994081|seashell|Gram|Unknown|13|barn stantiableought| +12591|AAAAAAAAOCBDAAAA|2000-10-27||Progressive surveys succeed still. Systems lack so willing, good names. Years shall not talk really to a buses. Panels fulfil international genes.|0.76|5.68|8009002|maxinameless #2|9|optics|8|Sports|544|antiation|N/A|5849634567298purple6|yellow|Pound|Unknown|66|oughtn stantiableought| +12592|AAAAAAAAADBDAAAA|1997-10-27|1999-10-27|Upper men used to give still different girls. Proposals subsidise famous nerves. C|2.21|1.19|9004005|edu packmaxi #5|4|entertainments|9|Books|701|oughtbaration|N/A|844202steel934199287|sienna|Unknown|Unknown|50|ablen stantiableought| +12593|AAAAAAAAADBDAAAA|1999-10-28|2001-10-26|Only able options may result also with a walls. Average responsibilities end |9.47|8.52|9004005|importoamalg #2|4|fragrances|1|Women|701|oughtbaration|economy|844202steel934199287|slate|Dram|Unknown|19|prin stantiableought| +12594|AAAAAAAAADBDAAAA|2001-10-27||Representatives exist by a versions; actively manufacturing citizens occur now opportunities. Tears may not regard only |4.29|8.52|9004005|exportiamalg #1|3|maternity|1|Women|231|oughtbaration|medium|28rosy90484016974041|olive|Carton|Unknown|70|esen stantiableought| +12595|AAAAAAAADDBDAAAA|1997-10-27||Twice religious persons must harm again principles. Major, extreme a|2.42|2.05|4003002|exportiedu pack #2|3||||252|ableantiable|||grey|||14|| +12596|AAAAAAAAEDBDAAAA|1997-10-27|2000-10-26|Years spare here friends. Italian, uncomfortable thoughts|5.41|1.94|6005005|scholarcorp #5|5|earings|6|Jewelry|200|barbarable|N/A|710866pale1072270334|navy|Tbl|Unknown|13|callyn stantiableought| +12597|AAAAAAAAEDBDAAAA|2000-10-27||Years spare here friends. Italian, uncomfortable thoughts|2.20|1.94|6005005|amalgexporti #2|1|newborn|3|Children|200|barbarable|small|710866pale1072270334|saddle|Pound|Unknown|93|ationn stantiableought| +12598|AAAAAAAAGDBDAAAA|1997-10-27|1999-10-27|Equally main neighbours could make approx agric|9.93|7.54|2004001|edu packimporto #1|4|sports-apparel|2|Men|567|ationcallyanti|medium|791865880blanched214|smoke|Carton|Unknown|50|eingn stantiableought| +12599|AAAAAAAAGDBDAAAA|1999-10-28|2001-10-26|Yet available miles think far maybe permanent boys. Assistant types ought to deny here even anxious weeks; other, local years might destroy behind available, sexual comparisons. Level stand|8.68|3.03|5004002|edu packscholar #2|4|classical|5|Music|10|barought|N/A|791865880blanched214|puff|Pound|Unknown|25|n stn stantiableought| +12600|AAAAAAAAGDBDAAAA|2001-10-27||Long-term pubs might investigate above national affairs. Specialist, s|50.82|20.32|8004001|edu packnameless #1|4|camping|8|Sports|10|barought|N/A|791865880blanched214|snow|Bunch|Unknown|31|barbarcallyableought| +12601|AAAAAAAAJDBDAAAA|1997-10-27||Very single matches look really primary, known machin|3.62|2.96|5001002|amalgscholar #2|1|rock|5|Music|329|n stablepri|N/A|537555394117sandy718|wheat|Gram|Unknown|1|oughtbarcallyableought| +12602|AAAAAAAAKDBDAAAA|1997-10-27|2000-10-26|Firm towns may come only clear, main companies. Enough old groups appoint. Children know in a co|2.74|1.20|9012003|importounivamalg #3|12|home repair|9|Books|310|baroughtpri|N/A|678229navajo46540489|coral|Tbl|Unknown|9|ablebarcallyableought| +12603|AAAAAAAAKDBDAAAA|2000-10-27||Firm towns may come only clear, main companies. Enough old groups appoint. Children know in a co|2.42|1.20|9012003|corpbrand #4|16|consignment|6|Jewelry|271|baroughtpri|N/A|tan64875065701879494|honeydew|Oz|Unknown|61|pribarcallyableought| +12604|AAAAAAAAMDBDAAAA|1997-10-27|1999-10-27|International applications should see then com|9.37|4.12|4001001|amalgedu pack #1|1|womens|4|Shoes|416|callyoughtese|petite|381118277833tomato19|powder|Dozen|Unknown|31|esebarcallyableought| +12605|AAAAAAAAMDBDAAAA|1999-10-28|2001-10-26|International applications should see then com|4.79|3.44|2003002|exportiimporto #2|3|pants|2|Men|259|n stantiable|extra large|lavender352237159724|saddle|Unknown|Unknown|30|antibarcallyableought| +12606|AAAAAAAAMDBDAAAA|2001-10-27||International applications should see then com|7.76|5.66|2003002|importounivamalg #7|3|home repair|9|Books|259|n stantiable|N/A|147379cornsilk064936|steel|N/A|Unknown|66|callybarcallyableought| +12607|AAAAAAAAPDBDAAAA|1997-10-27||Orders satisfy all colleges. Years resist warm, invis|6.29|3.58|7008008|namelessbrand #8|8|lighting|7|Home|225|antiableable|N/A|33176papaya799270583|lime|Pallet|Unknown|59|ationbarcallyableought| +12608|AAAAAAAAAEBDAAAA|1997-10-27|2000-10-26|Proteins might not come together hotly small participants. Exact problems should understand by a producers. Various, open others could not prove against a views. Private, global differences us|0.60|0.24|5004001|edu packscholar #1|4|classical|5|Music|277|ationationable|N/A|4897356676slate14417|papaya|Ton|Unknown|23|eingbarcallyableought| +12609|AAAAAAAAAEBDAAAA|2000-10-27||Concentrations escape yesterday allies. Basic, public recordings used to avoid photographs. Me|1.09|0.24|5004001|amalgamalg #2|4|dresses|1|Women|277|ationationable|medium|4897356676slate14417|steel|Cup|Unknown|56|n stbarcallyableought| +12610|AAAAAAAACEBDAAAA|1997-10-27|1999-10-27|Only major subjects shift however. Cars ought to mea|2.24|1.00|3001001|amalgexporti #1|1|newborn|3|Children|274|eseationable|petite|3722321peach90196038|seashell|Unknown|Unknown|52|baroughtcallyableought| +12611|AAAAAAAACEBDAAAA|1999-10-28|2001-10-26|Only major subjects shift however. Cars ought to mea|4.80|3.40|3003002|exportiexporti #2|3|toddlers|3|Children|274|eseationable|large|3722321peach90196038|saddle|Ton|Unknown|26|oughtoughtcallyableought| +12612|AAAAAAAACEBDAAAA|2001-10-27||Only major subjects shift however. Cars ought to mea|1.31|3.40|3003002|exportiimporto #1|3|pants|2|Men|274|eseationable|medium|2000748sandy36384739|dodger|Tsp|Unknown|48|ableoughtcallyableought| +12613|AAAAAAAAFEBDAAAA|1997-10-27||Future trees express just popular countries. Extended, only lines might not maintain still apparent, obvious fingers. Basic, off|78.37|36.83|3004002|edu packexporti #2|4|school-uniforms|3|Children|463|pricallyese|small|3044947757373violet0|rose|Ounce|Unknown|15|prioughtcallyableought| +12614|AAAAAAAAGEBDAAAA|1997-10-27|2000-10-26|Away new residents could not mean big poli|2.77|1.66|7011005|amalgnameless #5|11|accent|7|Home|132|ablepriought|N/A|005921wheat728008587|white|Bundle|Unknown|70|eseoughtcallyableought| +12615|AAAAAAAAGEBDAAAA|2000-10-27||Main positions should not express that is at once late circumstances. Connections would not improve feet; wives can think soon raw times. So future priorities might go books. Good rivers seize primari|7.58|1.66|7011005|amalgamalg #2|1|dresses|1|Women|89|ablepriought|extra large|068697midnight698143|beige|Oz|Unknown|32|antioughtcallyableought| +12616|AAAAAAAAIEBDAAAA|1997-10-27|1999-10-27|Opposite, original differences wait considerably vehic|6.34|2.85|9008005|namelessmaxi #5|8|romance|9|Books|63|prically|N/A|629350969smoke077022|misty|Bundle|Unknown|4|callyoughtcallyableought| +12617|AAAAAAAAIEBDAAAA|1999-10-28|2001-10-26|As alone cultures may not use also; there sophisticated reasons would pay only, similar races. Different audiences gain never among a issues. Indeed small walls can take a|1.15|2.85|9008005|importoamalgamalg #4|8|monitors|10|Electronics|63|prically|N/A|90118347878143pale30|lime|Ounce|Unknown|33|ationoughtcallyableought| +12618|AAAAAAAAIEBDAAAA|2001-10-27||Secret plants might visit journals. Mixed taxes shoot therefore ancient pupi|1.14|2.85|9008005|importobrand #3|12|costume|6|Jewelry|63|prically|N/A|95389041935276slate4|purple|Bunch|Unknown|63|eingoughtcallyableought| +12619|AAAAAAAALEBDAAAA|1997-10-27||Families must not interact hours. Later new sources used to preserve as from a shares. Then expected chiefs include at all virtually used letters. Flexible, orga|5.65|4.46|5001002|amalgscholar #2|1|rock|5|Music|946|callyesen st|N/A|7589turquoise5450688|rose|Each|Unknown|14|n stoughtcallyableought| +12620|AAAAAAAAMEBDAAAA|1997-10-27|2000-10-26|Recently familiar children shall not fade alternativ|1.96|1.15|4001001|amalgedu pack #1|1|womens|4|Shoes|375|antiationpri|medium|128745magenta2737867|spring|Gross|Unknown|60|barablecallyableought| +12621|AAAAAAAAMEBDAAAA|2000-10-27||Coming, difficult times follow fully full, young rules; instances shall not weave alternative studies. Bottom, new goods show at least studies. Girls must suppose to a children; only able women shoul|3.72|2.97|2003002|exportiimporto #2|1|pants|2|Men|318|eingoughtpri|small|716rose6955277544939|white|Cup|Unknown|80|oughtablecallyableought| +12622|AAAAAAAAOEBDAAAA|1997-10-27|1999-10-27|Men would not fight here in a factories. Little cars would seek much for a students. Hours should go even for example old|0.58|0.27|1002001|importoamalg #1|2|fragrances|1|Women|545|antieseanti|medium|42785849883powder049|red|Ton|Unknown|12|ableablecallyableought| +12623|AAAAAAAAOEBDAAAA|1999-10-28|2001-10-26|Principles ought to feel as; banks must reduce into a kinds. Main lips must know wit|4.43|0.27|10016010|corpamalgamalg #10|2|wireless|10|Electronics|545|antieseanti|N/A|28sienna679228437066|sky|Cup|Unknown|67|priablecallyableought| +12624|AAAAAAAAOEBDAAAA|2001-10-27||Principles ought to feel as; banks must reduce into a kinds. Main lips must know wit|12.72|0.27|5002001|importoscholar #1|2|country|5|Music|545|antieseanti|N/A|28sienna679228437066|blush|Dozen|Unknown|32|eseablecallyableought| +12625|AAAAAAAABFBDAAAA|1997-10-27||Authors can put much free, differen|3.76|2.18|6006002|corpcorp #2|6|rings|6|Jewelry|96|callyn st|N/A|625072621slate032798|powder|Bunch|Unknown|90|antiablecallyableought| +12626|AAAAAAAACFBDAAAA|1997-10-27|2000-10-26|Initial, important ministers used to rely. Young, difficult glasses cannot say european, religious organisations; worried minutes protect action|4.95|3.06|7015007|scholarnameless #7|15|tables|7|Home|6|cally|N/A|8064891450steel37490|seashell|Dram|Unknown|28|callyablecallyableought| +12627|AAAAAAAACFBDAAAA|2000-10-27||Initial, important ministers used to rely. Young, difficult glasses cannot say european, religious organisations; worried minutes protect action|1.88|0.80|5004002|edu packscholar #2|4|classical|5|Music|6|cally|N/A|8064891450steel37490|tomato|Case|Unknown|3|ationablecallyableought| +12628|AAAAAAAAEFBDAAAA|1997-10-27|1999-10-27|Easy, local pilots would knock easily at all criminal patterns. Insufficient plans make slightly particular members; british|6.22|5.22|6008005|namelesscorp #5|8|mens watch|6|Jewelry|253|priantiable|N/A|82306904360281smoke3|smoke|Tbl|Unknown|11|eingablecallyableought| +12629|AAAAAAAAEFBDAAAA|1999-10-28|2001-10-26|Easy, local pilots would knock easily at all criminal patterns. Insufficient plans make slightly particular members; british|57.54|50.05|8003010|exportinameless #10|3|basketball|8|Sports|253|priantiable|N/A|82306904360281smoke3|pale|Dram|Unknown|36|n stablecallyableought| +12630|AAAAAAAAEFBDAAAA|2001-10-27||Easy, local pilots would knock easily at all criminal patterns. Insufficient plans make slightly particular members; british|2.47|50.05|8003010|namelessnameless #1|8|outdoor|8|Sports|253|priantiable|N/A|657419289619purple26|smoke|Dozen|Unknown|14|barpricallyableought| +12631|AAAAAAAAHFBDAAAA|1997-10-27||Structures must go at least. Middle years ought to adopt extensive, old levels. Theories may make much deep, addi|7.62|5.79|10005001|scholarunivamalg #1|5|karoke|10|Electronics|586|callyeinganti|N/A|21214766sienna906301|sandy|Gross|Unknown|10|oughtpricallyableought| +12632|AAAAAAAAIFBDAAAA|1997-10-27|2000-10-26|Unpleasant machines could te|3.52|2.25|5003001|exportischolar #1|3|pop|5|Music|241|oughteseable|N/A|454188111sky88697392|burnished|Oz|Unknown|31|ablepricallyableought| +12633|AAAAAAAAIFBDAAAA|2000-10-27||Activities would show here. Practical|3.00|1.77|5003001|scholarbrand #2|3|custom|6|Jewelry|241|oughteseable|N/A|454188111sky88697392|dodger|Dozen|Unknown|3|pripricallyableought| +12634|AAAAAAAAKFBDAAAA|1997-10-27|1999-10-27|Symbols take to a women. Now separate resources must move services. Years alleviate intermittently it|1.75|0.98|10002004|importounivamalg #4|2|camcorders|10|Electronics|80|bareing|N/A|518spring33393235143|puff|Lb|Unknown|31|esepricallyableought| +12635|AAAAAAAAKFBDAAAA|1999-10-28|2001-10-26|Symbols take to a women. Now separate resources must move services. Years alleviate intermittently it|9.57|0.98|10002004|importoexporti #2|2|infants|3|Children|625|bareing|extra large|0814peach32741245958|seashell|Case|Unknown|19|antipricallyableought| +12636|AAAAAAAAKFBDAAAA|2001-10-27||Symbols take to a women. Now separate resources must move services. Years alleviate intermittently it|96.03|0.98|10002004|amalgedu pack #1|2|womens|4|Shoes|625|antiablecally|medium|0814peach32741245958|honeydew|Bunch|Unknown|23|callypricallyableought| +12637|AAAAAAAANFBDAAAA|1997-10-27||Meanwhile full-time types would understand social, large courses. Prime, economic materials establish police. Vulnerable, odd volunteers pay again networks. Following, pri|0.88|0.42|1004002|edu packamalg #2|4|swimwear|1|Women|542|ableeseanti|medium|223wheat580985882925|salmon|Bunch|Unknown|28|ationpricallyableought| +12638|AAAAAAAAOFBDAAAA|1997-10-27|2000-10-26|Rare evenings could introduce from a counties. Extremely professional facilities seem teachers. Brief camps get then in a circumstances; directly rel|2.31|0.78|3001001|amalgexporti #1|1|newborn|3|Children|799|n stn station|extra large|966107836667steel277|puff|Each|Unknown|42|eingpricallyableought| +12639|AAAAAAAAOFBDAAAA|2000-10-27||Rare evenings could introduce from a counties. Extremely professional facilities seem teachers. Brief camps get then in a circumstances; directly rel|3.34|1.03|2001002|amalgimporto #2|1|accessories|2|Men|57|ationanti|large|966107836667steel277|white|Pound|Unknown|27|n stpricallyableought| +12640|AAAAAAAAAGBDAAAA|1997-10-27|1999-10-27|Late, existing children will help immediate, social seats. More decent wo|2.98|0.89|6001001|amalgcorp #1|1|birdal|6|Jewelry|346|callyesepri|N/A|300961769sandy482583|puff|N/A|Unknown|42|baresecallyableought| +12641|AAAAAAAAAGBDAAAA|1999-10-28|2001-10-26|Pregnant women ought to stay after a rights. Federal sounds listen originally particular, conservative dogs; general, prime states know of course capital taxes. For example ot|7.05|0.89|6001001|exportiamalg #2|1|maternity|1|Women|346|callyesepri|extra large|93firebrick998134088|smoke|Pallet|Unknown|30|oughtesecallyableought| +12642|AAAAAAAAAGBDAAAA|2001-10-27||Pregnant women ought to stay after a rights. Federal sounds listen originally particular, conservative dogs; general, prime states know of course capital taxes. For example ot|5.76|4.03|6001001|importoamalg #1|2|fragrances|1|Women|256|callyesepri|large|93firebrick998134088|pink|Cup|Unknown|21|ableesecallyableought| +12643|AAAAAAAADGBDAAAA|1997-10-27||Political months shall stay in a cells. Only certain states get particularly eastern, crazy days. Again good years will understand from time to time developments. Still othe|0.41|0.31|7011004|amalgnameless #4|11|accent|7|Home|256|callyantiable|N/A|2996907pink451035974|purple|Gram|Unknown|35|priesecallyableought| +12644|AAAAAAAAEGBDAAAA|1997-10-27|2000-10-26|Also subtle flowers create questions. Decisive, thin pairs pay else ashamed lights. Guilty, major proposals make too round trends. Still following associations salvage subsequently. Others get|50.35|43.80|3002001|importoexporti #1|2|infants|3|Children|912|ableoughtn st|large|3saddle5957278893147|mint|Unknown|Unknown|31|eseesecallyableought| +12645|AAAAAAAAEGBDAAAA|2000-10-27||Rich, effective minutes might say moral, careful places. Unions must require otherwise married,|2.09|1.44|9013010|exportiunivamalg #10|2|self-help|9|Books|912|ableoughtn st|N/A|3448585seashell98647|red|Gross|Unknown|3|antiesecallyableought| +12646|AAAAAAAAGGBDAAAA|1997-10-27|1999-10-27|Local, great areas get then national, common teachers. Numbers could not allow minutes. New, unlikely details trust together environmental guidelines; great institutions will|0.65|0.57|1001001|amalgamalg #1|1|dresses|1|Women|406|callybarese|medium|74572521589738blush5|powder|Ounce|Unknown|13|callyesecallyableought| +12647|AAAAAAAAGGBDAAAA|1999-10-28|2001-10-26|Local, great areas get then national, common teachers. Numbers could not allow minutes. New, unlikely details trust together environmental guidelines; great institutions will|5.40|0.57|6009008|maxicorp #8|9|womens watch|6|Jewelry|406|callybarese|N/A|4047828tan6822229904|saddle|Dram|Unknown|24|ationesecallyableought| +12648|AAAAAAAAGGBDAAAA|2001-10-27||Aspects might begin fears. Drugs fit monthly including a years; f|4.83|3.57|6016001|corpbrand #1|16|consignment|6|Jewelry|406|callybarese|N/A|2525756thistle683660|sandy|Tbl|Unknown|7|eingesecallyableought| +12649|AAAAAAAAJGBDAAAA|1997-10-27||Naked, foreign years help here|5.77|3.40|2002002|importoimporto #2|2|shirts|2|Men|124|eseableought|large|14lemon6398710986959|slate|Gram|Unknown|48|n stesecallyableought| +12650|AAAAAAAAKGBDAAAA|1997-10-27|2000-10-26|Able, plain approaches would break yet yesterday soci|81.71|48.20|2002001|importoimporto #1|2|shirts|2|Men|738|eingpriation|medium|981805purple62234674|wheat|Unknown|Unknown|78|baranticallyableought| +12651|AAAAAAAAKGBDAAAA|2000-10-27||Able, plain approaches would break yet yesterday soci|5.96|48.20|2002001|scholarunivamalg #4|5|karoke|10|Electronics|738|eingpriation|N/A|08606739662puff32264|sky|Tbl|Unknown|33|oughtanticallyableought| +12652|AAAAAAAAMGBDAAAA|1997-10-27|1999-10-27|French women must mention so. Employees spell more technological, possible intervals. Eventually able bottles will persist in a ranks. Only, sure efforts ch|95.91|51.79|5003001|exportischolar #1|3|pop|5|Music|58|einganti|N/A|4884aquamarine766518|rose|Dozen|Unknown|18|ableanticallyableought| +12653|AAAAAAAAMGBDAAAA|1999-10-28|2001-10-26|French women must mention so. Employees spell more technological, possible intervals. Eventually able bottles will persist in a ranks. Only, sure efforts ch|4.31|3.66|5003001|corpmaxi #6|3|parenting|9|Books|58|einganti|N/A|4884aquamarine766518|tan|Lb|Unknown|50|prianticallyableought| +12654|AAAAAAAAMGBDAAAA|2001-10-27||Roses would find. Remarkable, increasing options identify ill problems. Controls used to rely so christian margins; workers might remain patterns. Now mass creat|8.99|3.66|5003001|edu packedu pack #1|4|athletic|4|Shoes|684|einganti|extra large|1187lavender39467367|violet|Unknown|Unknown|1|eseanticallyableought| +12655|AAAAAAAAPGBDAAAA|1997-10-27||Men bring difficult stands. Free movements used to stay about. Styles like. Ill sharp years say for instance effective, economic rates. No|8.47|7.11|2003002|exportiimporto #2|3|pants|2|Men|475|antiationese|extra large|420147034082797peru4|goldenrod|Tbl|Unknown|90|antianticallyableought| +12656|AAAAAAAAAHBDAAAA|1997-10-27|2000-10-26|Communist, big others should shake less; rapid, different procedures can enable very badly spiritual men; directly furious live|8.37|2.51|4002001|importoedu pack #1|2|mens|4|Shoes|539|n stprianti|small|8958983345midnight43|metallic|Oz|Unknown|23|callyanticallyableought| +12657|AAAAAAAAAHBDAAAA|2000-10-27||Communist, big others should shake less; rapid, different procedures can enable very badly spiritual men; directly furious live|3.83|1.72|4002001|scholarcorp #2|5|earings|6|Jewelry|539|n stprianti|N/A|steel463274964144124|tomato|Lb|Unknown|60|ationanticallyableought| +12658|AAAAAAAACHBDAAAA|1997-10-27|1999-10-27|Spare, american sports see even posts; views think at the bands; men flow |2.58|1.31|9008011|namelessmaxi #11|8|romance|9|Books|248|eingeseable|N/A|467815353pink5889078|midnight|Gram|Unknown|60|einganticallyableought| +12659|AAAAAAAACHBDAAAA|1999-10-28|2001-10-26|Ways will mean to a interests. Full, diff|4.13|1.31|6002004|importocorp #4|8|diamonds|6|Jewelry|763|pricallyation|N/A|537773319mint0615153|tomato|Pound|Unknown|27|n stanticallyableought| +12660|AAAAAAAACHBDAAAA|2001-10-27||Only |2.84|1.31|8006001|corpnameless #1|6|football|8|Sports|763|pricallyation|N/A|0087azure83560349823|honeydew|Pallet|Unknown|73|barcallycallyableought| +12661|AAAAAAAAFHBDAAAA|1997-10-27||Social conclusions shall not commission lightly. Red clergy may not integrate certainly. Final, inherent governments would complement new feet. Others may make saf|4.88|3.46|6015006|scholarbrand #6|15|custom|6|Jewelry|3|pri|N/A|0324677314086peach40|ivory|Pound|Unknown|22|oughtcallycallyableought| +12662|AAAAAAAAGHBDAAAA|1997-10-27|2000-10-26|Machines mean alre|1.88|1.01|6002007|importocorp #7|2|diamonds|6|Jewelry|875|antiationeing|N/A|066300black392773820|metallic|Tbl|Unknown|41|ablecallycallyableought| +12663|AAAAAAAAGHBDAAAA|2000-10-27||Machines mean alre|51.77|25.88|6002007|exportiedu pack #2|3|kids|4|Shoes|275|antiationeing|large|4red1940273528063838|thistle|Dram|Unknown|69|pricallycallyableought| +12664|AAAAAAAAIHBDAAAA|1997-10-27|1999-10-27|Extra, shared children lay from the practices. Lexical,|4.47|1.51|4002001|importoedu pack #1|2|mens|4|Shoes|272|ableationable|economy|8powder5000146877642|moccasin|Carton|Unknown|22|esecallycallyableought| +12665|AAAAAAAAIHBDAAAA|1999-10-28|2001-10-26|Especially international tools lead early others. Prou|8.62|1.51|4002001|importoamalg #2|2|fragrances|1|Women|117|ableationable|medium|8powder5000146877642|white|Ton|Unknown|58|anticallycallyableought| +12666|AAAAAAAAIHBDAAAA|2001-10-27||Especially international tools lead early others. Prou|1.95|1.51|8009007|maxinameless #7|9|optics|8|Sports|117|ationoughtought|N/A|8powder5000146877642|pale|N/A|Unknown|26|callycallycallyableought| +12667|AAAAAAAALHBDAAAA|1997-10-27||Democratic payments get tomorrow about a numbers; keen pp. prevent financial languages. A little slim proceedings can deal efforts. Strong preparations cannot become however. Women reveal|2.21|1.01|2001002|amalgimporto #2|1|accessories|2|Men|18|eingought|medium|yellow27084180943526|chartreuse|Cup|Unknown|20|ationcallycallyableought| +12668|AAAAAAAAMHBDAAAA|1997-10-27|2000-10-26|Years would not listen.|8.38|5.69|3002001|importoexporti #1|2|infants|3|Children|894|esen steing|large|80923light3515040945|sky|Unknown|Unknown|90|eingcallycallyableought| +12669|AAAAAAAAMHBDAAAA|2000-10-27||So hard members must not get closely simple, leading restrictions. Then various mountains forget. Single bodies outline only new, other beliefs.|5.67|4.25|3002001|exportischolar #2|3|pop|5|Music|269|esen steing|N/A|124tomato12401457662|wheat|Case|Unknown|23|n stcallycallyableought| +12670|AAAAAAAAOHBDAAAA|1997-10-27|1999-10-27|Prime responsibilities believe together considerations; together empirical points disappear small arms. New, american demands may not ensure within|8.98|6.82|1002001|importoamalg #1|2|fragrances|1|Women|521|oughtableanti|petite|1190229926011dim2260|orchid|Pound|Unknown|73|barationcallyableought| +12671|AAAAAAAAOHBDAAAA|1999-10-28|2001-10-26|Prime responsibilities believe together considerations; together empirical points disappear small arms. New, american demands may not ensure within|1.57|6.82|1002001|univbrand #8|10|jewelry boxes|6|Jewelry|521|oughtableanti|N/A|20violet436531407971|powder|N/A|Unknown|5|oughtationcallyableought| +12672|AAAAAAAAOHBDAAAA|2001-10-27||Poor, national negotiations see possibly single blues. Even happy producers play probably. Old, various ends could create also dark, old problems. As old leaders may not bear that is at a gates. A|66.43|6.82|3002001|importoexporti #1|2|infants|3|Children|447|ationeseese|large|2485seashell09599276|rosy|Dram|Unknown|1|ableationcallyableought| +12673|AAAAAAAABIBDAAAA|1997-10-27||Small friends seem jobs. Accounts would not echo on a employees. Recently following facts may compete m|1.66|0.64|3002002|importoexporti #2|2|infants|3|Children|241|oughteseable|petite|816violet31220239851|linen|Cup|Unknown|28|priationcallyableought| +12674|AAAAAAAACIBDAAAA|1997-10-27|2000-10-26|British difficulties decide through a machines. Domestic letters see perfectly; there russian features beat then gastric personnel. Ideological profits must not learn previously experts. Be|7.08|2.40|3002001|importoexporti #1|2|infants|3|Children|243|prieseable|medium|7mint101366837281124|ivory|N/A|Unknown|2|eseationcallyableought| +12675|AAAAAAAACIBDAAAA|2000-10-27||Users ought to ensure aside by a minutes. Persons think specifically other signs. Sad, angry authorities feel other than an readers. Christians can build even |1.70|1.05|1004002|edu packamalg #2|4|swimwear|1|Women|243|prieseable|petite|568sandy438017039839|smoke|Pound|Unknown|26|antiationcallyableought| +12676|AAAAAAAAEIBDAAAA|1997-10-27|1999-10-27|Other days mean inside at a standards. So current details leave so left properties. Regulations ensure heavy children. Sure local horses would turn other, international conditions.|65.30|23.50|8003007|exportinameless #7|3|basketball|8|Sports|330|barpripri|N/A|652829magenta2692465|sienna|Case|Unknown|7|callyationcallyableought| +12677|AAAAAAAAEIBDAAAA|1999-10-28|2001-10-26|Interested, female groups might call small, related technologies. Easily private examples shall break more most good ways. Limitations get for|0.73|0.38|6010006|univbrand #6|10|jewelry boxes|6|Jewelry|544|eseeseanti|N/A|03702480016830plum33|peru|Unknown|Unknown|7|ationationcallyableought| +12678|AAAAAAAAEIBDAAAA|2001-10-27||English, whole things should show old, random paintings. About interested students think very europe|4.51|0.38|6010006|amalgimporto #1|1|accessories|2|Men|102|ablebarought|medium|44157purple705853374|tomato|Box|Unknown|93|eingationcallyableought| +12679|AAAAAAAAHIBDAAAA|1997-10-27||Steps result parts. Brown, nuclear parties shall not advise yet constant letters; major options bear at a instructions; texts widen. Long participants must not visit pla|2.59|2.22|5004002|edu packscholar #2|4|classical|5|Music|262|ablecallyable|N/A|44241301011263white4|thistle|Gram|Unknown|11|n stationcallyableought| +12680|AAAAAAAAIIBDAAAA|1997-10-27|2000-10-26|International neighbours lead. Modern members could benefit rapid, young feet. More basic we|7.97|3.90|4004001|edu packedu pack #1|4|athletic|4|Shoes|435|antipriese|N/A|7511thistle428179160|sky|Ton|Unknown|31|bareingcallyableought| +12681|AAAAAAAAIIBDAAAA|2000-10-27||Only, local requirements ought to shut at least thanks. At once peculiar police remain othe|6.44|3.90|4004001|edu packmaxi #10|4|entertainments|9|Books|435|antipriese|N/A|7511thistle428179160|seashell|N/A|Unknown|22|oughteingcallyableought| +12682|AAAAAAAAKIBDAAAA|1997-10-27|1999-10-27|Original relations might know mechanisms; different others fulfil much always roya|2.72|0.87|1004001|edu packamalg #1|4|swimwear|1|Women|188|eingeingought|petite|83lavender5691774735|turquoise|Bunch|Unknown|31|ableeingcallyableought| +12683|AAAAAAAAKIBDAAAA|1999-10-28|2001-10-26|Original relations might know mechanisms; different others fulfil much always roya|6.49|3.95|1004001|amalgnameless #4|4|accent|7|Home|411|eingeingought|N/A|83lavender5691774735|moccasin|Cup|Unknown|59|prieingcallyableought| +12684|AAAAAAAAKIBDAAAA|2001-10-27||Willing others could offer only so neces|0.88|3.95|1004001|univnameless #5|10|flatware|7|Home|545|eingeingought|N/A|45light5014455859815|royal|Pallet|Unknown|36|eseeingcallyableought| +12685|AAAAAAAANIBDAAAA|1997-10-27||Other, convincing readers shall talk rapidly parents. De|4.31|2.88|9005002|scholarmaxi #2|5|history|9|Books|136|callypriought|N/A|46904649sky008068681|navajo|Gram|Unknown|57|antieingcallyableought| +12686|AAAAAAAAOIBDAAAA|1997-10-27|2000-10-26|Presidential efforts could look. Low workers mean easy|3.78|1.32|9014003|edu packunivamalg #3|14|sports|9|Books|307|ationbarpri|N/A|8518422171260royal72|gainsboro|Pound|Unknown|57|callyeingcallyableought| +12687|AAAAAAAAOIBDAAAA|2000-10-27||Normally new parties can visit again likely, right troubles; strange, royal manufacturers shall try probably shops. New, elegant sh|4.74|2.60|7009008|maxibrand #8|9|mattresses|7|Home|132|ablepriought|N/A|8518422171260royal72|plum|Bundle|Unknown|69|ationeingcallyableought| +12688|AAAAAAAAAJBDAAAA|1997-10-27|1999-10-27|Industrial women would make once. Gastric, wrong rumours used |2.41|1.30|9014005|edu packunivamalg #5|14|sports|9|Books|462|ablecallyese|N/A|1503786063chiffon602|grey|Tsp|Unknown|5|eingeingcallyableought| +12689|AAAAAAAAAJBDAAAA|1999-10-28|2001-10-26|Here good steps must wonder probably considerable bits; fatal miles would like at a aspects. Annual,|5.31|1.30|9007006|brandmaxi #6|14|reference|9|Books|183|ablecallyese|N/A|1503786063chiffon602|plum|Gross|Unknown|5|n steingcallyableought| +12690|AAAAAAAAAJBDAAAA|2001-10-27||Directly head parts will find then now identical issues. Books clear in a players. Local activities guess judicia|1.29|0.70|3002001|importoexporti #1|14|infants|3|Children|183|ablecallyese|large|1503786063chiffon602|plum|Box|Unknown|22|barn stcallyableought| +12691|AAAAAAAADJBDAAAA|1997-10-27||Identical, low roles would minimise over red minutes. White targets must generate now. Scottish deaths can stand to|3.01|1.44|5003002|exportischolar #2|3|pop|5|Music|186|callyeingought|N/A|7758195388075sky7791|red|Gram|Unknown|82|oughtn stcallyableought| +12692|AAAAAAAAEJBDAAAA|1997-10-27|2000-10-26|New, successful sc|6.46|2.97|2001001|amalgimporto #1|1|accessories|2|Men|128|eingableought|extra large|218salmon03892297137|smoke|N/A|Unknown|29|ablen stcallyableought| +12693|AAAAAAAAEJBDAAAA|2000-10-27||New, successful sc|8.73|3.49|2001001|exportischolar #2|3|pop|5|Music|128|eingableought|N/A|195627929480royal373|plum|Unknown|Unknown|31|prin stcallyableought| +12694|AAAAAAAAGJBDAAAA|1997-10-27|1999-10-27|New, separate motives hide never. Regardless local solutions get yet still national years. Colleges dance precisely admittedly great letters; assumptions contain even british details. Local cells may|7.73|6.02|1002001|importoamalg #1|2|fragrances|1|Women|722|ableableation|large|1386767157702sky4739|spring|Ounce|Unknown|54|esen stcallyableought| +12695|AAAAAAAAGJBDAAAA|1999-10-28|2001-10-26|New, separate motives hide never. Regardless local solutions get yet still national years. Colleges dance precisely admittedly great letters; assumptions contain even british details. Local cells may|3.99|3.35|8007006|brandnameless #6|2|hockey|8|Sports|722|ableableation|N/A|1386767157702sky4739|medium|Dozen|Unknown|27|antin stcallyableought| +12696|AAAAAAAAGJBDAAAA|2001-10-27||Literary, similar documents ought to help now deep ships. Small details should worry films. Points shall not receive now in a pieces. |6.56|2.49|8007006|importoexporti #1|2|infants|3|Children|722|ableableation|extra large|1386767157702sky4739|tomato|Pallet|Unknown|17|callyn stcallyableought| +12697|AAAAAAAAJJBDAAAA|1997-10-27||Birds stay foreign, chronic parts. So young cases shall not conclude buildings. About important months may not look; degrees catch just; other societies may not ge|4.67|2.33|5001002|amalgscholar #2|1|rock|5|Music|624|eseablecally|N/A|2470783papaya4849345|green|Bundle|Unknown|27|ationn stcallyableought| +12698|AAAAAAAAKJBDAAAA|1997-10-27|2000-10-26|Workers should argue t|5.14|2.15|2002001|importoimporto #1|2|shirts|2|Men|276|callyationable|medium|peach233808758088590|sienna|Dozen|Unknown|36|eingn stcallyableought| +12699|AAAAAAAAKJBDAAAA|2000-10-27||Strongly full |2.73|2.15|2002001|amalgexporti #2|1|newborn|3|Children|86|callyationable|N/A|741steel489202856349|slate|Pallet|Unknown|40|n stn stcallyableought| +12700|AAAAAAAAMJBDAAAA|1997-10-27|1999-10-27|Different states teach beneath royal houses. British countries could express residents; more educatio|5.66|3.84|7008001|namelessbrand #1|8|lighting|7|Home|378|eingationpri|N/A|936gainsboro74032870|saddle|Bundle|Unknown|57|barbarationableought| +12701|AAAAAAAAMJBDAAAA|1999-10-28|2001-10-26|Even real developers convince at large public occasions; here joint gods close intensely possibly strong gaps. Aware, extreme years must call. Always quick dogs can happen sexual, soviet plans. So|2.29|1.48|5001002|amalgscholar #2|8|rock|5|Music|287|ationeingable|N/A|936gainsboro74032870|seashell|Box|Unknown|16|oughtbarationableought| +12702|AAAAAAAAMJBDAAAA|2001-10-27||New, final jews will invite in a songs. Far, old complaints meet days. High observers might speak small, young months. Strongly likely words migh|4.34|1.48|5001002|exportiedu pack #1|8|kids|4|Shoes|597|ationeingable|small|936gainsboro74032870|slate|Dozen|Unknown|98|ablebarationableought| +12703|AAAAAAAAPJBDAAAA|1997-10-27||No longer good ideas engage in terms of the places. Effects should not make just now global bases. Now cheap lines find all. Motives shall make far windows. Pp. may lose civil yards; short pro|6.29|4.96|6003008|exporticorp #8|3|gold|6|Jewelry|546|callyeseanti|N/A|114859572wheat153034|spring|Cup|Unknown|40|pribarationableought| +12704|AAAAAAAAAKBDAAAA|1997-10-27|2000-10-26|Essential students change even despite a powers. General connections will not maximi|3.10|1.58|9002009|importomaxi #9|2|business|9|Books|589|n steinganti|N/A|4269095saddle7789954|salmon|N/A|Unknown|100|esebarationableought| +12705|AAAAAAAAAKBDAAAA|2000-10-27||English, fatal days used to suggest with a studies. Legs result groups. Old, easy spaces perform rules. Successive facilities shall not like. Things |6.72|2.88|9002009|scholarunivamalg #7|5|karoke|10|Electronics|116|callyoughtought|N/A|4269095saddle7789954|lace|Cup|Unknown|41|antibarationableought| +12706|AAAAAAAACKBDAAAA|1997-10-27|1999-10-27|Good, common problems go like a men. New, right papers move results. Others secure more rough, guilty days. Schools kill much. Natural|0.80|0.46|10007008|brandunivamalg #8|7|personal|10|Electronics|618|eingoughtcally|N/A|23707934534sandy5060|slate|Bundle|Unknown|19|callybarationableought| +12707|AAAAAAAACKBDAAAA|1999-10-28|2001-10-26|Products will deny also local, constitutional charges. Distinctive industries benefit very components. Blue, huge stars should complain only necessary proportions|1.80|0.46|5003002|exportischolar #2|3|pop|5|Music|618|eingoughtcally|N/A|23707934534sandy5060|saddle|Dram|Unknown|18|ationbarationableought| +12708|AAAAAAAACKBDAAAA|2001-10-27||Anywhere soft details might ask home clear|3.37|0.46|5003002|edu packexporti #1|3|school-uniforms|3|Children|184|eingoughtcally|medium|23707934534sandy5060|plum|Ton|Unknown|82|eingbarationableought| +12709|AAAAAAAAFKBDAAAA|1997-10-27||Ministers will no|6.15|3.50|10012011|importoamalgamalg #11|12|monitors|10|Electronics|596|callyn stanti|N/A|7powder8091805210928|gainsboro|Ounce|Unknown|13|n stbarationableought| +12710|AAAAAAAAGKBDAAAA|1997-10-27|2000-10-26|Buyers should not review therefore important homes; super, beneficial statements|2.97|2.67|9014003|edu packunivamalg #3|14|sports|9|Books|617|ationoughtcally|N/A|83370red437551028678|rosy|Ounce|Unknown|45|baroughtationableought| +12711|AAAAAAAAGKBDAAAA|2000-10-27||Western tactics will reveal never so generous circumstances. Too british exports help apparently regulatory, whole homes. Years rise splendid, so-called co|4.72|2.67|8003002|exportinameless #2|3|basketball|8|Sports|208|ationoughtcally|N/A|654833439pink4184161|slate|Bunch|Unknown|17|oughtoughtationableought| +12712|AAAAAAAAIKBDAAAA|1997-10-27|1999-10-27|Assets seem in respect of a organisations; good, correct ta|0.94|0.68|5001001|amalgscholar #1|1|rock|5|Music|996|callyn stn st|N/A|43peach5469935179082|seashell|Cup|Unknown|44|ableoughtationableought| +12713|AAAAAAAAIKBDAAAA|1999-10-28|2001-10-26|Assets seem in respect of a organisations; good, correct ta|56.28|0.68|5001001|exportischolar #2|3|pop|5|Music|308|callyn stn st|N/A|43peach5469935179082|white|N/A|Unknown|28|prioughtationableought| +12714|AAAAAAAAIKBDAAAA|2001-10-27||Assets seem in respect of a organisations; good, correct ta|9.70|6.30|6004003|edu packcorp #3|4|bracelets|6|Jewelry|308|eingbarpri|N/A|4014413093seashell82|sandy|Box|Unknown|32|eseoughtationableought| +12715|AAAAAAAALKBDAAAA|1997-10-27||As separate children might reduce now up to a members; individual cu|0.42|0.27|10007017|brandunivamalg #17|7|personal|10|Electronics|295|antin stable|N/A|2turquoise2860666615|violet|Box|Unknown|50|antioughtationableought| +12716|AAAAAAAAMKBDAAAA|1997-10-27|2000-10-26|As comparable eyes will possess in a bars. Ideas would not get quickly full, beautiful companies. Taxes ought to fear understandably wit|7.76|4.26|4001001|amalgedu pack #1|1|womens|4|Shoes|706|callybaration|extra large|089465powder28291798|orange|Dram|Unknown|36|callyoughtationableought| +12717|AAAAAAAAMKBDAAAA|2000-10-27||As comparable eyes will possess in a bars. Ideas would not get quickly full, beautiful companies. Taxes ought to fear understandably wit|6.81|4.08|10004002|edu packunivamalg #2|4|audio|10|Electronics|119|callybaration|N/A|089465powder28291798|gainsboro|Cup|Unknown|46|ationoughtationableought| +12718|AAAAAAAAOKBDAAAA|1997-10-27|1999-10-27|Please rough groups may see very sisters. Small areas may act. Certainly wide ways ought to exploit. Open millions will inflict mostly excellent difficulties. Movem|9.89|3.16|1003001|exportiamalg #1|3|maternity|1|Women|162|ablecallyought|large|2589759672187rose849|royal|Gram|Unknown|8|eingoughtationableought| +12719|AAAAAAAAOKBDAAAA|1999-10-28|2001-10-26|Please rough groups may see very sisters. Small areas may act. Certainly wide ways ought to exploit. Open millions will inflict mostly excellent difficulties. Movem|3.14|3.16|1003001|edu packedu pack #2|4|athletic|4|Shoes|162|ablecallyought|petite|6019371963943red9777|steel|Dozen|Unknown|24|n stoughtationableought| +12720|AAAAAAAAOKBDAAAA|2001-10-27||Please rough groups may see very sisters. Small areas may act. Certainly wide ways ought to exploit. Open millions will inflict mostly excellent difficulties. Movem|1.32|3.16|1003001|scholarmaxi #1|5|history|9|Books|440|ablecallyought|N/A|6019371963943red9777|royal|Ounce|Unknown|15|barableationableought| +12721|AAAAAAAABLBDAAAA|1997-10-27||Anxious, new patien|0.11|0.05|1001002|amalgamalg #2|1|dresses|1|Women|337|ationpripri|medium|0turquoise9279903522|plum|Tbl|Unknown|6|oughtableationableought| +12722|AAAAAAAACLBDAAAA|1997-10-27|2000-10-26|Common, senior arms might mean indeed huge, new hundreds. Other, lost movements tell then things. Therefore poor authorities insist british, previous mothers. More italian options s|7.41|5.63|3003001|exportiexporti #1|3|toddlers|3|Children|846|callyeseeing|large|53spring645449927998|rosy|Bunch|Unknown|54|ableableationableought| +12723|AAAAAAAACLBDAAAA|2000-10-27||Occasions ask in a doors; new, annual circumstances wil|2.32|1.53|3003001|edu packimporto #2|3|sports-apparel|2|Men|846|callyeseeing|medium|53spring645449927998|dodger|Oz|Unknown|100|priableationableought| +12724|AAAAAAAAELBDAAAA|1997-10-27|1999-10-27|Together young farmers need of course following officers. Early beans gain there continental animals. Local, his|4.94|3.65|7012005|importonameless #5|12|paint|7|Home|488|eingeingese|N/A|91452chartreuse69324|steel|Lb|Unknown|64|eseableationableought| +12725|AAAAAAAAELBDAAAA|1999-10-28|2001-10-26|Cases will not get of course old, objective sales. Groups encourage enough ridiculous ca|2.42|1.35|7012005|namelessnameless #6|12|outdoor|8|Sports|488|eingeingese|N/A|599460837385khaki921|turquoise|Gross|Unknown|3|antiableationableought| +12726|AAAAAAAAELBDAAAA|2001-10-27||National, old hours would|32.92|27.65|7012005|univunivamalg #7|10|travel|9|Books|648|eingeingese|N/A|977759576puff7136456|puff|Carton|Unknown|29|callyableationableought| +12727|AAAAAAAAHLBDAAAA|1997-10-27||Typical, trying institutions ought to intensify top approaches. Different demands shall change previous, defensive pupils. Times ought to know soon weekends. Especially possible kids will |2.93|1.02|4002002|importoedu pack #2|2|mens|4|Shoes|693|prin stcally|medium|931844896sandy962415|rosy|Gram|Unknown|16|ationableationableought| +12728|AAAAAAAAILBDAAAA|1997-10-27|2000-10-26|Financial, italian wages kno|5.30|3.76|8012009|importomaxi #9|12|guns|8|Sports|49|n stese|N/A|510007275274125pale1|sky|Bunch|Unknown|25|eingableationableought| +12729|AAAAAAAAILBDAAAA|2000-10-27||Just key messages should not make. Disabled artists should not tell arts. Strong items get. Current, difficult mice would not mention. Properly h|3.32|2.22|8012009|corpamalgamalg #14|12|wireless|10|Electronics|49|n stese|N/A|399light844464949134|antique|Tbl|Unknown|33|n stableationableought| +12730|AAAAAAAAKLBDAAAA|1997-10-27|1999-10-27|Just various prices will not look furthermore maybe supreme women. Years hear somewhere from a reports. New components cannot charge as. Interests imp|1.75|1.33|5002001|importoscholar #1|2|country|5|Music|195|antin stought|N/A|45thistle06589482753|sienna|N/A|Unknown|97|barpriationableought| +12731|AAAAAAAAKLBDAAAA|1999-10-28|2001-10-26|Just various prices will not look furthermore maybe supreme women. Years hear somewhere from a reports. New components cannot charge as. Interests imp|0.30|0.13|5002001|edu packamalgamalg #16|2|automotive|10|Electronics|195|antin stought|N/A|591728puff4562647517|white|Carton|Unknown|37|oughtpriationableought| +12732|AAAAAAAAKLBDAAAA|2001-10-27||Just ab|3.88|0.13|5002001|exportiexporti #1|3|toddlers|3|Children|195|antin stought|large|591728puff4562647517|thistle|Ton|Unknown|10|ablepriationableought| +12733|AAAAAAAANLBDAAAA|1997-10-27||Local households control public, responsible tools. As loc|0.09|0.06|5004002|edu packscholar #2|4|classical|5|Music|488|eingeingese|N/A|6005454811yellow2352|smoke|Each|Unknown|32|pripriationableought| +12734|AAAAAAAAOLBDAAAA|1997-10-27|2000-10-26|Psychological, main wages would replace as a matt|3.57|1.46|7015005|scholarnameless #5|15|tables|7|Home|209|n stbarable|N/A|76596418931764red381|midnight|Ton|Unknown|30|esepriationableought| +12735|AAAAAAAAOLBDAAAA|2000-10-27||Psychological, main wages would replace as a matt|4.15|1.46|7015005|amalgedu pack #2|1|womens|4|Shoes|209|n stbarable|extra large|76596418931764red381|slate|N/A|Unknown|70|antipriationableought| +12736|AAAAAAAAAMBDAAAA|1997-10-27|1999-10-27|Personal, back colleagues work|18.69|8.03|7009007|maxibrand #7|9|mattresses|7|Home|467|ationcallyese|N/A|salmon18499537476031|thistle|N/A|Unknown|65|callypriationableought| +12737|AAAAAAAAAMBDAAAA|1999-10-28|2001-10-26|Closer warm animals compete low prisons. Hot, successful expectations seem widespread yards. Elec|2.19|8.03|8012008|importomaxi #8|12|guns|8|Sports|251|oughtantiable|N/A|salmon18499537476031|tomato|Ounce|Unknown|25|ationpriationableought| +12738|AAAAAAAAAMBDAAAA|2001-10-27||Closer warm animals compete low prisons. Hot, successful expectations seem widespread yards. Elec|2.53|8.03|8012008|exportiexporti #1|12|toddlers|3|Children|227|ationableable|petite|78916736869thistle16|tan|Box|Unknown|73|eingpriationableought| +12739|AAAAAAAADMBDAAAA|1997-10-27||Social, other resources may know reasonable, distant weeks. New, unexpected rates mean. White, electric generations carry together other t|3.91|2.54|8009010|maxinameless #10|9|optics|8|Sports|155|antiantiought|N/A|00snow32599211288974|steel|Tsp|Unknown|23|n stpriationableought| +12740|AAAAAAAAEMBDAAAA|1997-10-27|2000-10-26|Police think particularly young, political numbers. Centuries shall cover in a roots. Experimental years get y|6.91|4.35|10008008|namelessunivamalg #8|8|scanners|10|Electronics|632|ableprically|N/A|saddle20228956475319|lawn|Box|Unknown|19|bareseationableought| +12741|AAAAAAAAEMBDAAAA|2000-10-27||Police think particularly young, political numbers. Centuries shall cover in a roots. Experimental years get y|5.01|1.70|10013009|exportiamalgamalg #9|8|stereo|10|Electronics|612|ableoughtcally|N/A|248536813puff3600570|wheat|Gram|Unknown|55|oughteseationableought| +12742|AAAAAAAAGMBDAAAA|1997-10-27|1999-10-27|Vulnerable b|0.58|0.44|9003011|exportimaxi #11|3|computers|9|Books|63|prically|N/A|6violet7758075676629|slate|Dozen|Unknown|42|ableeseationableought| +12743|AAAAAAAAGMBDAAAA|1999-10-28|2001-10-26|Vulnerable b|3.95|0.44|2002002|importoimporto #2|3|shirts|2|Men|63|prically|medium|83229tomato165973211|powder|Dram|Unknown|17|prieseationableought| +12744|AAAAAAAAGMBDAAAA|2001-10-27||Vulnerable b|1.30|0.44|7002005|importobrand #5|3|bedding|7|Home|820|barableeing|N/A|peru1128244025868167|plum|Dram|Unknown|30|eseeseationableought| +12745|AAAAAAAAJMBDAAAA|1997-10-27||English patients used to break then issues. At least wild things bear; efforts could complain well for the proceedings. Un|4.72|1.46|2001002|amalgimporto #2|1|accessories|2|Men|620|barablecally|small|05565smoke4011527962|khaki|Ton|Unknown|26|antieseationableought| +12746|AAAAAAAAKMBDAAAA|1997-10-27|2000-10-26|Other hours make small studies. Old lines should run fast different children. Only large tr|9.61|6.43|2003001|exportiimporto #1|3|pants|2|Men|412|ableoughtese|N/A|144164457saddle68128|tan|N/A|Unknown|29|callyeseationableought| +12747|AAAAAAAAKMBDAAAA|2000-10-27||Other hours make small studies. Old lines should run fast different children. Only large tr|1.02|6.43|2003001|importoamalg #2|3|fragrances|1|Women|106|callybarought|medium|22turquoise159961134|metallic|Bunch|Unknown|16|ationeseationableought| +12748|AAAAAAAAMMBDAAAA|1997-10-27|1999-10-27|Samples can betray later usually industrial rates. Senior, final systems serve over with a women. Great, lacking men stand in a teams. Useful, possible forces ought to refer quite directly admi|4.09|2.90|5002001|importoscholar #1|2|country|5|Music|572|ableationanti|N/A|94235577813violet045|thistle|Gross|Unknown|19|eingeseationableought| +12749|AAAAAAAAMMBDAAAA|1999-10-28|2001-10-26|French, late metres will wind then among the politicians. Broad modes would not bind please availabl|4.32|3.02|5001002|amalgscholar #2|2|rock|5|Music|380|bareingpri|N/A|94235577813violet045|green|Lb|Unknown|51|n steseationableought| +12750|AAAAAAAAMMBDAAAA|2001-10-27||Relevant, nuclear wines will follow hence big bones. Workers will not give and so on to a materials. Preliminary, important proposals adopt barely. Best unchanged hom|3.84|3.02|1002001|importoamalg #1|2|fragrances|1|Women|380|bareingpri|medium|94235577813violet045|rose|Gram|Unknown|28|barantiationableought| +12751|AAAAAAAAPMBDAAAA|1997-10-27||Invisible year|2.67|2.18|2003002|exportiimporto #2|3|pants|2|Men|201|oughtbarable|extra large|5549981816188wheat84|midnight|Each|Unknown|5|oughtantiationableought| +12752|AAAAAAAAANBDAAAA|1997-10-27|2000-10-26|Then specific values get there new transactions. All but whole s|3.21|1.38|4004001|edu packedu pack #1|4|athletic|4|Shoes|228|eingableable|large|821328335733blush853|puff|N/A|Unknown|13|ableantiationableought| +12753|AAAAAAAAANBDAAAA|2000-10-27||Then specific values get there new transactions. All but whole s|4.91|2.84|4004001|edu packbrand #4|4|curtains/drapes|7|Home|34|eingableable|N/A|821328335733blush853|violet|Oz|Unknown|22|priantiationableought| +12754|AAAAAAAACNBDAAAA|1997-10-27|1999-10-27|Real cells would take in a women. Then well-known bishops would identify more with a events. Head rates should try player|7.69|2.61|8003005|exportinameless #5|3|basketball|8|Sports|490|barn stese|N/A|81916173594steel6370|seashell|Pallet|Unknown|29|eseantiationableought| +12755|AAAAAAAACNBDAAAA|1999-10-28|2001-10-26|Real cells would take in a women. Then well-known bishops would identify more with a events. Head rates should try player|92.52|36.08|8003005|namelessmaxi #12|3|romance|9|Books|841|barn stese|N/A|1849642560890wheat29|turquoise|Tsp|Unknown|33|antiantiationableought| +12756|AAAAAAAACNBDAAAA|2001-10-27||Real cells would take in a women. Then well-known bishops would identify more with a events. Head rates should try player|23.72|9.48|8003005|amalgamalg #1|3|dresses|1|Women|713|prioughtation|medium|1849642560890wheat29|blanched|Lb|Unknown|33|callyantiationableought| +12757|AAAAAAAAFNBDAAAA|1997-10-27||Other arms pay absolute, royal stages. Hotels take entirely top, ancient cases. Deep courses should recommend in a pairs. Numerous advantages might not deliver also; royal, implicit|4.21|3.53|10014008|edu packamalgamalg #8|14|automotive|10|Electronics|557|ationantianti|N/A|239693186386sky52829|pale|Box|Unknown|58|ationantiationableought| +12758|AAAAAAAAGNBDAAAA|1997-10-27|2000-10-26|Essential others ought to like able, considerable floors. Middle-class daughters ought to call new months. Othe|3.48|3.06|5004001|edu packscholar #1|4|classical|5|Music|665|anticallycally|N/A|86saddle080224548316|sienna|Each|Unknown|83|eingantiationableought| +12759|AAAAAAAAGNBDAAAA|2000-10-27||Essential others ought to like able, considerable floors. Middle-class daughters ought to call new months. Othe|2.44|1.04|5004001|univamalgamalg #10|4|memory|10|Electronics|665|anticallycally|N/A|86saddle080224548316|navajo|N/A|Unknown|19|n stantiationableought| +12760|AAAAAAAAINBDAAAA|1997-10-27|1999-10-27|Areas ought to transfer soon on a m|0.42|0.16|3003001|exportiexporti #1|3|toddlers|3|Children|75|antiation|small|897367261151270plum5|pink|Pound|Unknown|19|barcallyationableought| +12761|AAAAAAAAINBDAAAA|1999-10-28|2001-10-26|Even privat|3.84|1.30|6010002|univbrand #2|3|jewelry boxes|6|Jewelry|15|antiought|N/A|897367261151270plum5|pink|Each|Unknown|39|oughtcallyationableought| +12762|AAAAAAAAINBDAAAA|2001-10-27||Even privat|8.72|1.30|1003001|exportiamalg #1|3|maternity|1|Women|15|antiought|medium|897367261151270plum5|yellow|Pound|Unknown|79|ablecallyationableought| +12763|AAAAAAAALNBDAAAA|1997-10-27||Visitors lead current sorts. Levels reduce both; then real parts shall not continue employers. All radical plants will gain now prof|3.58|1.43|1003002|exportiamalg #2|3|maternity|1|Women|85|antieing|extra large|pink2009067538233583|beige|Ton|Unknown|60|pricallyationableought| +12764|AAAAAAAAMNBDAAAA|1997-10-27|2000-10-26|Public, formidable addresses can prevent further by a circumstances. Ever medical ladies must stay slow institutions; public pregnant groups should fail ver|2.49|1.26|4001001|amalgedu pack #1|1|womens|4|Shoes|570|barationanti|medium|7751753797922salmon8|red|Unknown|Unknown|16|esecallyationableought| +12765|AAAAAAAAMNBDAAAA|2000-10-27||Wet things ought to mean very factors. So comparable years s|2.06|0.63|2002002|importoimporto #2|1|shirts|2|Men|156|barationanti|large|7751753797922salmon8|yellow|Dram|Unknown|15|anticallyationableought| +12766|AAAAAAAAONBDAAAA|1997-10-27|1999-10-27|Old, national lessons seek more spanish worlds. Nights apply here |9.64|6.94|7001007|amalgbrand #7|1|bathroom|7|Home|890|barn steing|N/A|860476372030lime4876|sienna|Oz|Unknown|21|callycallyationableought| +12767|AAAAAAAAONBDAAAA|1999-10-28|2001-10-26|High, explicit particles fail for a materials. Yet v|0.44|6.94|10011001|amalgamalgamalg #1|11|disk drives|10|Electronics|145|antieseought|N/A|860476372030lime4876|tan|Ton|Unknown|34|ationcallyationableought| +12768|AAAAAAAAONBDAAAA|2001-10-27||High, explicit particles fail for a materials. Yet v|3.59|2.08|10011001|amalgimporto #1|11|accessories|2|Men|364|esecallypri|petite|11grey01490777465720|violet|Unknown|Unknown|58|eingcallyationableought| +12769|AAAAAAAABOBDAAAA|1997-10-27||Immediately professional cells may ship properly forward political members. Daily, direct trains can choose clearly. Partners answer everywhere at a chara|0.18|0.07|9007002|brandmaxi #2|7|reference|9|Books|306|callybarpri|N/A|69moccasin4544009775|violet|Gram|Unknown|10|n stcallyationableought| +12770|AAAAAAAACOBDAAAA|1997-10-27|2000-10-26|Nationa|4.20|2.26|2003001|exportiimporto #1|3|pants|2|Men|395|antin stpri|extra large|127094magenta0963297|navy|Bunch|Unknown|77|barationationableought| +12771|AAAAAAAACOBDAAAA|2000-10-27||Nationa|5.93|2.26|2003001|corpmaxi #2|3|golf|8|Sports|948|eingesen st|N/A|127094magenta0963297|navajo|Ounce|Unknown|94|oughtationationableought| +12772|AAAAAAAAEOBDAAAA|1997-10-27|1999-10-27|Children help from the s|7.55|4.53|10001006|amalgunivamalg #6|1|cameras|10|Electronics|269|n stcallyable|N/A|rosy8719315156485276|medium|Ounce|Unknown|9|ableationationableought| +12773|AAAAAAAAEOBDAAAA|1999-10-28|2001-10-26|Nationally crucial leaders will emerge to |4.67|4.53|10001006|corpunivamalg #7|1|musical|10|Electronics|269|n stcallyable|N/A|rosy8719315156485276|slate|Gram|Unknown|69|priationationableought| +12774|AAAAAAAAEOBDAAAA|2001-10-27||Nationally crucial leaders will emerge to |88.69|47.00|10001006|amalgedu pack #1|1|womens|4|Shoes|732|n stcallyable|medium|rosy8719315156485276|sandy|Bunch|Unknown|2|eseationationableought| +12775|AAAAAAAAHOBDAAAA|1997-10-27||Great methods evacuate in order free stations. Seats |1.00|0.62|4002002|importoedu pack #2|2|mens|4|Shoes|664|esecallycally|medium|54tomato463499832545|slate|Lb|Unknown|14|antiationationableought| +12776|AAAAAAAAIOBDAAAA|1997-10-27|2000-10-26|Problems might not get also current minutes. Women wear happily values. Resul|4.65|3.25|7008007|namelessbrand #7|8|lighting|7|Home|17|ationought|N/A|1859goldenrod7104561|pale|Carton|Unknown|49|callyationationableought| +12777|AAAAAAAAIOBDAAAA|2000-10-27||Problems might not get also current minutes. Women wear happily values. Resul|3.74|3.36|7008007|amalgedu pack #2|1|womens|4|Shoes|54|eseanti|medium|1859goldenrod7104561|yellow|Cup|Unknown|17|ationationationableought| +12778|AAAAAAAAKOBDAAAA|1997-10-27|1999-10-27|Low, severe persons keep public, mad employers. Always modern children go by a schemes. In particular national items rise fully widespread, powerful miles. Extremely southern costs design sett|9.08|3.35|8004009|edu packnameless #9|4|camping|8|Sports|328|eingablepri|N/A|18578157496pink99289|tan|Pound|Unknown|31|eingationationableought| +12779|AAAAAAAAKOBDAAAA|1999-10-28|2001-10-26|Low, severe persons keep public, mad employers. Always modern children go by a schemes. In particular national items rise fully widespread, powerful miles. Extremely southern costs design sett|2.61|2.27|9016012|corpunivamalg #12|16|mystery|9|Books|328|eingablepri|N/A|135582red55722912209|rose|Pound|Unknown|50|n stationationableought| +12780|AAAAAAAAKOBDAAAA|2001-10-27||Mechanical years hide possible, small computers. Clubs consider much twice other minutes. Beautiful connections make just round developers. Broad, outstanding birds inspe|1.04|2.27|9016012|amalgnameless #1|16|accent|7|Home|328|eingablepri|N/A|170855748477powder01|steel|Dozen|Unknown|22|bareingationableought| +12781|AAAAAAAANOBDAAAA|1997-10-27||Large businessmen might give successful poles; children believe however. Hard, fine companies must not dismiss likely advantages. Now great nations shall not walk to|3.48|1.07|6007006|brandcorp #6|7|pendants|6|Jewelry|103|pribarought|N/A|09smoke5709582163781|midnight|Pound|Unknown|48|oughteingationableought| +12782|AAAAAAAAOOBDAAAA|1997-10-27|2000-10-26|Green, low houses must not explain in a rules; other miles reduce beautiful, successfu|47.64|14.29|7011003|amalgnameless #3|11|accent|7|Home|161|oughtcallyought|N/A|813883246light855818|snow|Ton|Unknown|2|ableeingationableought| +12783|AAAAAAAAOOBDAAAA|2000-10-27||Green, low houses must not explain in a rules; other miles reduce beautiful, successfu|0.10|0.03|7011003|scholarnameless #4|15|tables|7|Home|161|oughtcallyought|N/A|76723717tan405392554|sienna|N/A|Unknown|4|prieingationableought| +12784|AAAAAAAAAPBDAAAA|1997-10-27|1999-10-27|Studies would seek perhaps blue women. New questions should wrap apart new, increas|2.79|1.28|2001001|amalgimporto #1|1|accessories|2|Men|452|ableantiese|large|691052210101ivory523|aquamarine|Cup|Unknown|69|eseeingationableought| +12785|AAAAAAAAAPBDAAAA|1999-10-28|2001-10-26|Financial, common films must save typically good, bea|4.93|4.09|9012006|importounivamalg #6|12|home repair|9|Books|452|ableantiese|N/A|16311444maroon592338|salmon|Carton|Unknown|48|antieingationableought| +12786|AAAAAAAAAPBDAAAA|2001-10-27||Financial, common films must save typically good, bea|8.17|4.09|7006007|corpbrand #7|6|rugs|7|Home|21|oughtable|N/A|6218683793sandy56923|purple|Tsp|Unknown|5|callyeingationableought| +12787|AAAAAAAADPBDAAAA|1997-10-27||Valuable, territorial dealers help after a s|7.23|4.04|10002004|importounivamalg #4|2|camcorders|10|Electronics|526|callyableanti|N/A|65002sienna533621279|peach|Pallet|Unknown|11|ationeingationableought| +12788|AAAAAAAAEPBDAAAA|1997-10-27|2000-10-26|Private, average clouds yield political, alive runs. Finally interested creatures might rescue. Public years want recently wild figures. Simply economic products should hit as.|8.38|3.35|7005009|scholarbrand #9|5|blinds/shades|7|Home|667|ationcallycally|N/A|276honeydew961030290|violet|Bundle|Unknown|22|eingeingationableought| +12789|AAAAAAAAEPBDAAAA|2000-10-27||Private, average clouds yield political, alive runs. Finally interested creatures might rescue. Public years want recently wild figures. Simply economic products should hit as.|0.13|3.35|4003002|exportiedu pack #2|3|kids|4|Shoes|496|callyn stese|medium|5912934899spring9542|yellow|Each|Unknown|39|n steingationableought| +12790|AAAAAAAAGPBDAAAA|1997-10-27|1999-10-27|Senior problems should indulge. Real, substantial eyes move properly efforts. Ministers can get more. Br|9.93|5.85|8003001|exportinameless #1|3|basketball|8|Sports|161|oughtcallyought|N/A|smoke097921321258488|powder|Bundle|Unknown|46|barn stationableought| +12791|AAAAAAAAGPBDAAAA|1999-10-28|2001-10-26|Western, only authorities use. Literary, possible games cannot|9.29|5.85|6002008|importocorp #8|2|diamonds|6|Jewelry|161|oughtcallyought|N/A|smoke097921321258488|linen|Unknown|Unknown|3|oughtn stationableought| +12792|AAAAAAAAGPBDAAAA|2001-10-27||Western, only authorities use. Literary, possible games cannot|7.67|5.82|6002008|brandmaxi #1|7|reference|9|Books|161|oughtcallyought|N/A|132251violet60698809|ivory|Cup|Unknown|23|ablen stationableought| +12793|AAAAAAAAJPBDAAAA|1997-10-27||Vague, decent years experiment rather rare tensions. Good, commercial parties lead poorly british, helpful others. Ago |4.35|3.52|8010004|univmaxi #4|10|pools|8|Sports|980|bareingn st|N/A|24122salmon940805085|tan|Tsp|Unknown|99|prin stationableought| +12794|AAAAAAAAKPBDAAAA|1997-10-27|2000-10-26|Basic results may cover with a children. About exact rates could agree there subjects. Colours prefer now stories. High, other animals must make notes; either|1.41|0.80|4003001|exportiedu pack #1|3|kids|4|Shoes|239|n stpriable|medium|0442olive37187867518|red|Gram|Unknown|8|esen stationableought| +12795|AAAAAAAAKPBDAAAA|2000-10-27||Basic results may cover with a children. About exact rates could agree there subjects. Colours prefer now stories. High, other animals must make notes; either|4.21|3.15|2002002|importoimporto #2|2|shirts|2|Men|239|n stpriable|petite|0442olive37187867518|tomato|Tbl|Unknown|32|antin stationableought| +12796|AAAAAAAAMPBDAAAA|1997-10-27|1999-10-27|National, great fee|59.75|27.48|3003001|exportiexporti #1|3|toddlers|3|Children|445|antieseese|large|3891909navajo6931584|salmon|Cup|Unknown|45|callyn stationableought| +12797|AAAAAAAAMPBDAAAA|1999-10-28|2001-10-26|National, great fee|7.03|4.49|3003001|amalgnameless #8|1|athletic shoes|8|Sports|445|antieseese|N/A|3891909navajo6931584|slate|Oz|Unknown|19|ationn stationableought| +12798|AAAAAAAAMPBDAAAA|2001-10-27||Properties may not protect eventually new, outstanding words. Thoughts may not work dramatically things. Styles wait members. Possible, competitive weeks could bring to a |50.19|4.49|10013015|exportiamalgamalg #15|13|stereo|10|Electronics|445|antieseese|N/A|3891909navajo6931584|peru|Each|Unknown|58|eingn stationableought| +12799|AAAAAAAAPPBDAAAA|1997-10-27||Useful things mean bonds. Large, healthy rates would make though great pp.; gentlemen may describe national years. Less than wide minutes may not lecture now consciou|13.30|4.78|4004002|edu packedu pack #2|4|athletic|4|Shoes|417|ationoughtese|petite|76114832164rosy84944|thistle|Box|Unknown|6|n stn stationableought| +12800|AAAAAAAAAACDAAAA|1997-10-27|2000-10-26|Tired, coloured items shall manage. Small schools use ever so so|1.32|0.81|4001001|amalgedu pack #1|1|womens|4|Shoes|26|callyable|petite|999256360seashell969|ghost|Dram|Unknown|57|barbareingableought| +12801|AAAAAAAAAACDAAAA|2000-10-27||Tired, coloured items shall manage. Small schools use ever so so|4.95|0.81|4001002|amalgedu pack #2|1|womens|4|Shoes|26|callyable|small|999256360seashell969|papaya|Box|Unknown|37|oughtbareingableought| +12802|AAAAAAAACACDAAAA|1997-10-27|1999-10-27|Large, total failures focus very clear men. Readers could not sho|0.62|0.25|4003001|exportiedu pack #1|3|kids|4|Shoes|257|ationantiable|medium|56aquamarine28290691|steel|Ounce|Unknown|56|ablebareingableought| +12803|AAAAAAAACACDAAAA|1999-10-28|2001-10-26|Large, total failures focus very clear men. Readers could not sho|2.36|0.25|7008004|namelessbrand #4|8|lighting|7|Home|119|ationantiable|N/A|56aquamarine28290691|moccasin|Dozen|Unknown|6|pribareingableought| +12804|AAAAAAAACACDAAAA|2001-10-27||White, marginal trousers find. Officers answer. Black players compet|9.13|6.84|7008004|brandnameless #5|8|hockey|8|Sports|280|bareingable|N/A|67288832tan116825438|royal|Tbl|Unknown|6|esebareingableought| +12805|AAAAAAAAFACDAAAA|1997-10-27||Just young partie|4.58|1.64|8008006|namelessnameless #6|8|outdoor|8|Sports|362|ablecallypri|N/A|12677207azure5770820|linen|Oz|Unknown|6|antibareingableought| +12806|AAAAAAAAGACDAAAA|1997-10-27|2000-10-26|Courses walk less than in a effects. Corners introduce therefore distinct members.|1.89|1.45|8002007|importonameless #7|2|baseball|8|Sports|110|baroughtought|N/A|84235935395orange033|sandy|Gross|Unknown|62|callybareingableought| +12807|AAAAAAAAGACDAAAA|2000-10-27||Courses walk less than in a effects. Corners introduce therefore distinct members.|4.83|1.45|8002007|amalgexporti #2|1|newborn|3|Children|132|ablepriought|small|84235935395orange033|metallic|Ton|Unknown|92|ationbareingableought| +12808|AAAAAAAAIACDAAAA|1997-10-27|1999-10-27|Wrong children might accommodate too. Methods must spend papers. Local possibilities detect in the scots.|1.25|0.60|5002001|importoscholar #1|2|country|5|Music|250|barantiable|N/A|381salmon57096997667|dim|Carton|Unknown|95|eingbareingableought| +12809|AAAAAAAAIACDAAAA|1999-10-28|2001-10-26|Wrong children might accommodate too. Methods must spend papers. Local possibilities detect in the scots.|4.56|1.36|5002001|edu packamalg #2|4|swimwear|1|Women|250|barantiable|medium|058813734904snow2272|turquoise|Cup|Unknown|61|n stbareingableought| +12810|AAAAAAAAIACDAAAA|2001-10-27||Wrong children might accommodate too. Methods must spend papers. Local possibilities detect in the scots.|5.61|1.36|5002001|scholarbrand #1|5|blinds/shades|7|Home|148|barantiable|N/A|058813734904snow2272|metallic|Oz|Unknown|13|baroughteingableought| +12811|AAAAAAAALACDAAAA|1997-10-27||British windows allow |9.97|5.78|4002002|importoedu pack #2|2|mens|4|Shoes|74|eseation|extra large|782red99685824381949|rose|Pallet|Unknown|20|oughtoughteingableought| +12812|AAAAAAAAMACDAAAA|1997-10-27|2000-10-26|Great, high weeks draw external, heavy feet. Available weeks ought to determine yet. Conditions used to make twice soon clear sta|1.33|0.47|7012003|importonameless #3|12|paint|7|Home|516|callyoughtanti|N/A|8765orchid8775042025|white|Dozen|Unknown|6|ableoughteingableought| +12813|AAAAAAAAMACDAAAA|2000-10-27||Great, high weeks draw external, heavy feet. Available weeks ought to determine yet. Conditions used to make twice soon clear sta|3.63|3.15|10016013|corpamalgamalg #13|16|wireless|10|Electronics|516|callyoughtanti|N/A|723096198snow4883677|tan|Box|Unknown|28|prioughteingableought| +12814|AAAAAAAAOACDAAAA|1997-10-27|1999-10-27|Men break for the magistrates. Eager, bad forms must not support very famous things; go|4.67|3.03|8013005|exportimaxi #5|13|sailing|8|Sports|20|barable|N/A|5saddle8898102170481|turquoise|Lb|Unknown|33|eseoughteingableought| +12815|AAAAAAAAOACDAAAA|1999-10-28|2001-10-26|Important, sure words continue forward ways. Old, corporate troops may not speak well public holidays. At all diplomatic years bet on|4.20|3.03|8013005|importoimporto #2|13|shirts|2|Men|137|ationpriought|petite|5saddle8898102170481|slate|Ton|Unknown|16|antioughteingableought| +12816|AAAAAAAAOACDAAAA|2001-10-27||Important, sure words continue forward ways. Old, corporate troops may not speak well public holidays. At all diplomatic years bet on|0.55|3.03|1001001|amalgamalg #1|13|dresses|1|Women|141|oughteseought|extra large|23736000sandy8361035|spring|Bunch|Unknown|36|callyoughteingableought| +12817|AAAAAAAABBCDAAAA|1997-10-27||More civil police put often sure members. Ot|7.74|3.71|4002002|importoedu pack #2|2|mens|4|Shoes|55|antianti|extra large|95140turquoise329503|chiffon|Gram|Unknown|52|ationoughteingableought| +12818|AAAAAAAACBCDAAAA|1997-10-27|2000-10-26|Religious areas grow rapidly on a efforts; statutory networks make for a resources; sides speak easily adequately exact areas; free, occasional comments could not require. Speedily eastern alte|1.09|0.33|2001001|amalgimporto #1|1|accessories|2|Men|128|eingableought|extra large|2310986saddle9757406|slate|Tbl|Unknown|14|eingoughteingableought| +12819|AAAAAAAACBCDAAAA|2000-10-27||Regional expectations seem somehow by a politicians. Plans disappear also chief trees. Following, theoretical rooms provide criminal, fond courses. Demands study|3.57|0.33|2001001|amalgimporto #2|1|accessories|2|Men|261|oughtcallyable|medium|2310986saddle9757406|pink|Bundle|Unknown|19|n stoughteingableought| +12820|AAAAAAAAEBCDAAAA|1997-10-27|1999-10-27|Also particular days sense photographs. There obvious consequences should not meet suddenly also healthy effects. Long important interests c|1.98|1.76|9004005|edu packmaxi #5|4|entertainments|9|Books|122|ableableought|N/A|101046976sienna60435|magenta|Tsp|Unknown|32|barableeingableought| +12821|AAAAAAAAEBCDAAAA|1999-10-28|2001-10-26|Also particular days sense photographs. There obvious consequences should not meet suddenly also healthy effects. Long important interests c|4.21|1.72|8012002|importomaxi #2|12|guns|8|Sports|122|ableableought|N/A|9511yellow8316954651|lime|Pallet|Unknown|17|oughtableeingableought| +12822|AAAAAAAAEBCDAAAA|2001-10-27||New men could achieve later. Heavy students would not match; accidents will not communicate for the doctors. Walls may pull |4.81|2.69|8012002|amalgedu pack #1|12|womens|4|Shoes|816|ableableought|small|9511yellow8316954651|rosy|Box|Unknown|53|ableableeingableought| +12823|AAAAAAAAHBCDAAAA|1997-10-27||Even gross partners send features; away true years |2.89|2.19|2003002|exportiimporto #2|3|pants|2|Men|681|oughteingcally|small|177680pink9891071741|pale|Bunch|Unknown|19|priableeingableought| +12824|AAAAAAAAIBCDAAAA|1997-10-27|2000-10-26|Popular prices will not see constantly policies. Well young exports mark about methods|9.39|4.97|3001001|amalgexporti #1|1|newborn|3|Children|216|callyoughtable|medium|564100585508889sky32|steel|Bunch|Unknown|36|eseableeingableought| +12825|AAAAAAAAIBCDAAAA|2000-10-27||Popular prices will not see constantly policies. Well young exports mark about methods|4.44|3.90|9016010|corpunivamalg #10|16|mystery|9|Books|662|callyoughtable|N/A|564100585508889sky32|chiffon|Box|Unknown|66|antiableeingableought| +12826|AAAAAAAAKBCDAAAA|1997-10-27|1999-10-27|Concerned, formal dogs m|4.03|2.94|3002001|importoexporti #1|2|infants|3|Children|29|n stable|petite|powder18461016892756|rose|Cup|Unknown|4|callyableeingableought| +12827|AAAAAAAAKBCDAAAA|1999-10-28|2001-10-26|Concerned, formal dogs m|8.88|7.28|3002001|importoedu pack #2|2|mens|4|Shoes|29|n stable|small|3269491white63819137|violet|Gram|Unknown|22|ationableeingableought| +12828|AAAAAAAAKBCDAAAA|2001-10-27||Once again hard lives may not appear. Raw customers make sweet, economic difficulties; fine ports might anticipate in a lights|0.88|0.50|5003001|exportischolar #1|2|pop|5|Music|154|n stable|N/A|3269491white63819137|rose|Oz|Unknown|31|eingableeingableought| +12829|AAAAAAAANBCDAAAA|1997-10-27||Various customers notice still fair important points. Common things would work entirely; partly at|0.64|0.53|8007010|brandnameless #10|7|hockey|8|Sports|253|priantiable|N/A|0233113white01436502|tan|Bunch|Unknown|12|n stableeingableought| +12830|AAAAAAAAOBCDAAAA|1997-10-27|2000-10-26|Secrets get ideally annual, physical lines. New, entire measures talk yesterday for a plans. Main parents may reverse moreover also european levels; seconds shall light|5.33|4.31|2004001|edu packimporto #1|4|sports-apparel|2|Men|133|pripriought|medium|223123516761534plum9|steel|N/A|Unknown|48|barprieingableought| +12831|AAAAAAAAOBCDAAAA|2000-10-27||Workers tackle random, western numbers; specialist, normal rights link early find|53.53|39.61|2004001|corpbrand #2|6|rugs|7|Home|133|pripriought|N/A|223123516761534plum9|spring|Lb|Unknown|75|oughtprieingableought| +12832|AAAAAAAAACCDAAAA|1997-10-27|1999-10-27|Figures will not wish late primary, sure members. Recently true techniques could bring a little radically economic processes. Distant lips ought to go only civil words. Days claim aback in the kids; |4.14|1.65|8016003|corpmaxi #3|16|golf|8|Sports|251|oughtantiable|N/A|chartreuse1419077472|grey|Gram|Unknown|4|ableprieingableought| +12833|AAAAAAAAACCDAAAA|1999-10-28|2001-10-26|Figures will not wish late primary, sure members. Recently true techniques could bring a little radically economic processes. Distant lips ought to go only civil words. Days claim aback in the kids; |4.09|1.65|2001002|amalgimporto #2|16|accessories|2|Men|251|oughtantiable|large|chartreuse1419077472|sandy|Lb|Unknown|1|priprieingableought| +12834|AAAAAAAAACCDAAAA|2001-10-27||Most constant actions say negative features. Conditions can see more for instance keen functions; important weapons used to inter|37.12|12.62|2001002|exportischolar #1|3|pop|5|Music|251|oughtantiable|N/A|mint3609354487093628|steel|Pound|Unknown|29|eseprieingableought| +12835|AAAAAAAADCCDAAAA|1997-10-27||Glad users understand very almost original jobs. Towns can understand. Supreme, following days work by a parents; german, crucial weapons work sure; fair pictur|7.18|6.39|9001008|amalgmaxi #8|1|arts|9|Books|79|n station|N/A|54571483thistle83903|sandy|Oz|Unknown|22|antiprieingableought| +12836|AAAAAAAAECCDAAAA|1997-10-27|2000-10-26|Right customs want always widespread subjects. Poor, extra years take obviously leaves. Months may not measure for the companies. Particular, traditional reforms advertise countries. Ends use|2.51|1.88|10001002|amalgunivamalg #2|1|cameras|10|Electronics|609|n stbarcally|N/A|70997859476seashell4|snow|Bundle|Unknown|33|callyprieingableought| +12837|AAAAAAAAECCDAAAA|2000-10-27||Affairs fight against the policies. Intelligent fires think papers. Colourful, digital activities will not stop unknown, general words. Strategies carry a|1.39|0.41|10001002|corpbrand #8|1|rugs|7|Home|587|n stbarcally|N/A|63503spring356473236|seashell|Gross|Unknown|45|ationprieingableought| +12838|AAAAAAAAGCCDAAAA|1997-10-27|1999-10-27|Children know quite only streets. Political, public visitors must not help; men could not pretend then. Weeks close enough. Infrequently complex year|0.15|0.10|10009004|maxiunivamalg #4|9|televisions|10|Electronics|418|eingoughtese|N/A|351593orchid14613735|orchid|Pallet|Unknown|27|eingprieingableought| +12839|AAAAAAAAGCCDAAAA|1999-10-28|2001-10-26|Children know quite only streets. Political, public visitors must not help; men could not pretend then. Weeks close enough. Infrequently complex year|2.45|1.96|5003002|exportischolar #2|3|pop|5|Music|418|eingoughtese|N/A|351593orchid14613735|tomato|Ounce|Unknown|8|n stprieingableought| +12840|AAAAAAAAGCCDAAAA|2001-10-27||Local facilities may eat jointly ridiculous miners. Brown, existing years commence normally to a structures. Changes shall not start offen|4.71|1.96|8003001|exportinameless #1|3|basketball|8|Sports|564|eingoughtese|N/A|351593orchid14613735|lime|Lb|Unknown|33|bareseeingableought| +12841|AAAAAAAAJCCDAAAA|1997-10-27||As tropical products conform also about a proportions. Then final rights would grow major operations. Women leave |4.46|2.45|4001002|amalgedu pack #2|1|womens|4|Shoes|36|callypri|extra large|65459411811150steel9|green|Carton|Unknown|7|oughteseeingableought| +12842|AAAAAAAAKCCDAAAA|1997-10-27|2000-10-26|German, red feet would result perhaps at a reasons. |4.17|3.58|2001001|amalgimporto #1|1|accessories|2|Men|222|ableableable|N/A|turquoise61899999249|papaya|Bundle|Unknown|23|ableeseeingableought| +12843|AAAAAAAAKCCDAAAA|2000-10-27||Key decisions must ensure women. Quite different factors might come hence areas. Gastric appearances override only. Poor|5.04|3.58|7007004|brandbrand #4|7|decor|7|Home|109|ableableable|N/A|65302154482mint56916|slate|Dram|Unknown|29|prieseeingableought| +12844|AAAAAAAAMCCDAAAA|1997-10-27|1999-10-27|Studies will like; results inform too broken, different holes. Western, expensive share|2.64|2.03|1002001|importoamalg #1|2|fragrances|1|Women|238|eingpriable|large|424977white597503272|dark|Box|Unknown|2|eseeseeingableought| +12845|AAAAAAAAMCCDAAAA|1999-10-28|2001-10-26|Studies will like; results inform too broken, different holes. Western, expensive share|4.34|2.17|6005006|scholarcorp #6|2|earings|6|Jewelry|383|eingpriable|N/A|424977white597503272|smoke|Pallet|Unknown|18|antieseeingableought| +12846|AAAAAAAAMCCDAAAA|2001-10-27||Studies will like; results inform too broken, different holes. Western, expensive share|6.60|2.17|1003001|exportiamalg #1|2|maternity|1|Women|383|prieingpri|extra large|424977white597503272|smoke|Ton|Unknown|78|callyeseeingableought| +12847|AAAAAAAAPCCDAAAA|1997-10-27||Structural men solve so very, ready families. Wrong questions might not acknowled|3.80|1.25|6002008|importocorp #8|2|diamonds|6|Jewelry|824|eseableeing|N/A|34235718666thistle78|rosy|N/A|Unknown|60|ationeseeingableought| +12848|AAAAAAAAADCDAAAA|1997-10-27|2000-10-26|Bloody directors reach highly only heavy women. Ministers shall not avoid afte|4.26|2.08|8016009|corpmaxi #9|16|golf|8|Sports|323|priablepri|N/A|551tomato26036266114|pale|Cup|Unknown|16|eingeseeingableought| +12849|AAAAAAAAADCDAAAA|2000-10-27||Bloody directors reach highly only heavy women. Ministers shall not avoid afte|57.90|28.37|8016009|maxinameless #10|16|optics|8|Sports|33|pripri|N/A|551tomato26036266114|yellow|Cup|Unknown|14|n steseeingableought| +12850|AAAAAAAACDCDAAAA|1997-10-27|1999-10-27|Requirements might not set so. Capable, usual resources |4.68|3.36|8015001|scholarmaxi #1|15|fishing|8|Sports|301|oughtbarpri|N/A|7357682160260sky2606|powder|Box|Unknown|37|barantieingableought| +12851|AAAAAAAACDCDAAAA|1999-10-28|2001-10-26|Responsible communists prevent new, physical|8.05|4.66|8015001|amalgimporto #2|1|accessories|2|Men|301|oughtbarpri|N/A|7357682160260sky2606|ivory|Bunch|Unknown|27|oughtantieingableought| +12852|AAAAAAAACDCDAAAA|2001-10-27||Responsible communists prevent new, physical|4.10|3.60|10010001|univamalgamalg #1|1|memory|10|Electronics|86|oughtbarpri|N/A|3781059thistle770686|powder|Gram|Unknown|2|ableantieingableought| +12853|AAAAAAAAFDCDAAAA|1997-10-27||Pink, influential thanks should experience environmentally cases; sad, grey occasions must result later real parents; beautiful calculations would strain operations. Accurate problems should n|4.79|2.82|1004002|edu packamalg #2|4|swimwear|1|Women|130|barpriought|large|violet13643501958214|purple|Case|Unknown|33|priantieingableought| +12854|AAAAAAAAGDCDAAAA|1997-10-27|2000-10-26|Years think. Wealthy, empty figures become again. However white child|7.06|3.53|10004003|edu packunivamalg #3|4|audio|10|Electronics|70|baration|N/A|716lavender473917350|peach|N/A|Unknown|39|eseantieingableought| +12855|AAAAAAAAGDCDAAAA|2000-10-27||Years think. Wealthy, empty figures become again. However white child|0.35|0.13|10004003|exportiedu pack #2|3|kids|4|Shoes|660|barcallycally|large|16443292honeydew3284|sandy|Gross|Unknown|35|antiantieingableought| +12856|AAAAAAAAIDCDAAAA|1997-10-27|1999-10-27|Ultimately sup|2.12|1.86|2002001|importoimporto #1|2|shirts|2|Men|296|callyn stable|small|9snow157793791605834|light|N/A|Unknown|29|callyantieingableought| +12857|AAAAAAAAIDCDAAAA|1999-10-28|2001-10-26|Goods give relatively exact terms. Avail|2.55|1.02|8008008|namelessnameless #8|2|outdoor|8|Sports|296|callyn stable|N/A|9snow157793791605834|tan|Dozen|Unknown|24|ationantieingableought| +12858|AAAAAAAAIDCDAAAA|2001-10-27||Goods give relatively exact terms. Avail|88.86|51.53|8008008|amalgimporto #1|2|accessories|2|Men|233|pripriable|N/A|9snow157793791605834|steel|Unknown|Unknown|1|eingantieingableought| +12859|AAAAAAAALDCDAAAA|1997-10-27||Large, old-fashioned firms change mainly. Aware studies should go jolly steep effects. Either expert plans appear executives. Natural arrangements use n|3.67|2.60|10015008|scholaramalgamalg #8|15|portable|10|Electronics|839|n stprieing|N/A|849salmon03467510013|forest|Ounce|Unknown|8|n stantieingableought| +12860|AAAAAAAAMDCDAAAA|1997-10-27|2000-10-26|Social, high conservatives think over there. Indeed critical affairs cannot come talks. Following, similar quantities|3.99|2.75|3003001|exportiexporti #1|3|toddlers|3|Children|714|eseoughtation|extra large|804slate002979797016|cream|N/A|Unknown|33|barcallyeingableought| +12861|AAAAAAAAMDCDAAAA|2000-10-27||Social, high conservatives think over there. Indeed critical affairs cannot come talks. Following, similar quantities|3.48|2.75|3003001|edu packscholar #2|3|classical|5|Music|252|eseoughtation|N/A|thistle0831016909342|wheat|Cup|Unknown|22|oughtcallyeingableought| +12862|AAAAAAAAODCDAAAA|1997-10-27|1999-10-27|As ce|8.20|5.98|2001001|amalgimporto #1|1|accessories|2|Men|392|ablen stpri|extra large|135626598498114lime0|spring|Oz|Unknown|19|ablecallyeingableought| +12863|AAAAAAAAODCDAAAA|1999-10-28|2001-10-26|U|0.53|0.41|2001001|amalgedu pack #2|1|womens|4|Shoes|392|ablen stpri|large|711659slate588837049|purple|Bunch|Unknown|17|pricallyeingableought| +12864|AAAAAAAAODCDAAAA|2001-10-27||Modern techniques would destroy at a women. Working, large techniques involve quietly cultural, convincing plants. As small rooms cannot merge as; ultimately|8.63|3.96|2001001|edu packcorp #1|4|bracelets|6|Jewelry|227|ationableable|N/A|711659slate588837049|cyan|Pallet|Unknown|18|esecallyeingableought| +12865|AAAAAAAABECDAAAA|1997-10-27||Workers remember more in a programs. Other, real matters will not outline usually on a assets. Regional rules may make therefore both necessary hours. Seconds finance alw|9.42|4.23|7014006|edu packnameless #6|14|glassware|7|Home|188|eingeingought|N/A|4yellow8598614814417|gainsboro|Ounce|Unknown|43|anticallyeingableought| +12866|AAAAAAAACECDAAAA|1997-10-27|2000-10-26|Italian laboratories could want only; unfair, modern relationships could accept further hard, other eyes. Chemical gardens need now fu|4.36|1.87|6013003|exportibrand #3|13|loose stones|6|Jewelry|717|ationoughtation|N/A|32555saddle125575045|violet|Dram|Unknown|23|callycallyeingableought| +12867|AAAAAAAACECDAAAA|2000-10-27||Services learn later in a ori|4.86|1.87|6013003|scholarmaxi #4|13|history|9|Books|717|ationoughtation|N/A|32555saddle125575045|rose|Box|Unknown|15|ationcallyeingableought| +12868|AAAAAAAAEECDAAAA|1997-10-27|1999-10-27|Improved, outdoor years like at a procedures. Particular, british subjects argue great, educational needs. Secondary users cannot catch. Main years might not fill contents. Inner, industrial skills ma|1.48|0.68|4002001|importoedu pack #1|2|mens|4|Shoes|451|oughtantiese|large|65690473691329khaki3|magenta|Unknown|Unknown|5|eingcallyeingableought| +12869|AAAAAAAAEECDAAAA|1999-10-28|2001-10-26|No longer broad players can give either as a records. Especially black years should speak mildly important unlikely films. Attractive, strong falls will ensure victorian, independent values|6.70|4.48|2001002|amalgimporto #2|1|accessories|2|Men|162|ablecallyought|extra large|07878303purple441085|light|Bunch|Unknown|16|n stcallyeingableought| +12870|AAAAAAAAEECDAAAA|2001-10-27||No longer broad players can give either as a records. Especially black years should speak mildly important unlikely films. Attractive, strong falls will ensure victorian, independent values|2.00|4.48|2001002|exportiamalgamalg #2|1|stereo|10|Electronics|162|ablecallyought|N/A|997927063868pale0006|peach|Gram|Unknown|2|barationeingableought| +12871|AAAAAAAAHECDAAAA|1997-10-27||International, new months ought to alter well. Huge blocks wor|25.20|10.83|2001002|amalgimporto #2|1|accessories|2|Men|729|n stableation|economy|52turquoise628660173|pale|Dozen|Unknown|56|oughtationeingableought| +12872|AAAAAAAAIECDAAAA|1997-10-27|2000-10-26|Proportions give suddenly usual, warm guns. Days shall control old animals. In order free courts withdraw for example machines. Never able groups find far communities. Well capital classe|4.94|3.80|6008001|namelesscorp #1|8|mens watch|6|Jewelry|100|barbarought|N/A|555180856pink6446208|smoke|Cup|Unknown|96|ableationeingableought| +12873|AAAAAAAAIECDAAAA|2000-10-27||Proportions give suddenly usual, warm guns. Days shall control old animals. In order free courts withdraw for example machines. Never able groups find far communities. Well capital classe|7.46|3.80|6008001|exportiunivamalg #10|13|self-help|9|Books|323|priablepri|N/A|2820pink129934524259|turquoise|Pound|Unknown|33|priationeingableought| +12874|AAAAAAAAKECDAAAA|1997-10-27|1999-10-27|Etc serious incentives must not call free judges; anyway european voices ought to use other residents. Trees used to like from a changes. Fair, english years take also reluctant doors; complete mo|5.65|3.39|4003001|exportiedu pack #1|3|kids|4|Shoes|926|callyablen st|petite|121743thistle7608729|orange|Case|Unknown|51|eseationeingableought| +12875|AAAAAAAAKECDAAAA|1999-10-28|2001-10-26|Etc serious incentives must not call free judges; anyway european voices ought to use other residents. Trees used to like from a changes. Fair, english years take also reluctant doors; complete mo|4.61|3.27|6006004|corpcorp #4|3|rings|6|Jewelry|926|callyablen st|N/A|36505652563plum05405|sandy|Case|Unknown|7|antiationeingableought| +12876|AAAAAAAAKECDAAAA|2001-10-27||Etc serious incentives must not call free judges; anyway european voices ought to use other residents. Trees used to like from a changes. Fair, english years take also reluctant doors; complete mo|0.53|3.27|2001001|amalgimporto #1|1|accessories|2|Men|5|callyablen st|small|79rosy14091695637301|wheat|Box|Unknown|73|callyationeingableought| +12877|AAAAAAAANECDAAAA|1997-10-27||Researchers develop ever on the elements; resources land as concerned proceedings. Russian, single hours create very real, broad schools. Running police cannot di|3.52|2.88|4002002|importoedu pack #2|2|mens|4|Shoes|509|n stbaranti|medium|1987611920107papaya5|royal|Dozen|Unknown|25|ationationeingableought| +12878|AAAAAAAAOECDAAAA|1997-10-27|2000-10-26|Of course dependent levels might get thin words. Fond, spiritual complaints provide exactly week|0.35|0.20|1002001|importoamalg #1|2|fragrances|1|Women|257|ationantiable|N/A|23lace14655296940105|royal|N/A|Unknown|25|eingationeingableought| +12879|AAAAAAAAOECDAAAA|2000-10-27||Of course dependent levels might get thin words. Fond, spiritual complaints provide exactly week|42.92|0.20|10004011|edu packunivamalg #11|2|audio|10|Electronics|469|n stcallyese|N/A|seashell120097682592|papaya|Pallet|Unknown|40|n stationeingableought| +12880|AAAAAAAAAFCDAAAA|1997-10-27|1999-10-27|Very real critics would not start more. Now bad books continue resident|4.64|2.32|5001001|amalgscholar #1|1|rock|5|Music|291|oughtn stable|N/A|8puff931876611535989|orange|Dozen|Unknown|78|bareingeingableought| +12881|AAAAAAAAAFCDAAAA|1999-10-28|2001-10-26|Circumstances could get for a boys. Members would guarantee; stations|3.27|2.48|3001002|amalgexporti #2|1|newborn|3|Children|291|oughtn stable|petite|8puff931876611535989|seashell|Ton|Unknown|62|oughteingeingableought| +12882|AAAAAAAAAFCDAAAA|2001-10-27||Circumstances could get for a boys. Members would guarantee; stations|5.18|2.48|5003001|exportischolar #1|3|pop|5|Music|230|barpriable|N/A|12148green8813989755|peach|Dram|Unknown|89|ableeingeingableought| +12883|AAAAAAAADFCDAAAA|1997-10-27||Conservative, difficult funds seem more than capital standards. Developments add even miles. Countries imitate modern, other police|0.78|0.57|2003002|exportiimporto #2|3|pants|2|Men|589|n steinganti|extra large|7purple8580673970577|navajo|Lb|Unknown|18|prieingeingableought| +12884|AAAAAAAAEFCDAAAA|1997-10-27|2000-10-26|Months last tight; mere, important months shall return now yesterday lucky subjects. Instantly new chiefs translate totally other months. Major transactions show exactly social, pleasant sh|3.65|2.37|2001001|amalgimporto #1|1|accessories|2|Men|56|callyanti|small|64violet773418010886|antique|Lb|Unknown|46|eseeingeingableought| +12885|AAAAAAAAEFCDAAAA|2000-10-27||Clinical, dutch features pass then yet sex|0.70|2.37|2001001|importobrand #6|1|costume|6|Jewelry|56|callyanti|N/A|64violet773418010886|rose|Dram|Unknown|16|antieingeingableought| +12886|AAAAAAAAGFCDAAAA|1997-10-27|1999-10-27|Rough women must make best important, chief months; equally expensive jobs may keep. Somehow available assumptions laugh more fine games. Rew|0.52|0.35|6010007|univbrand #7|10|jewelry boxes|6|Jewelry|110|baroughtought|N/A|91lime11614121359598|dim|Unknown|Unknown|23|callyeingeingableought| +12887|AAAAAAAAGFCDAAAA|1999-10-28|2001-10-26|Over scottish materials will try s|0.50|0.41|2004002|edu packimporto #2|4|sports-apparel|2|Men|110|baroughtought|large|91lime11614121359598|plum|Cup|Unknown|1|ationeingeingableought| +12888|AAAAAAAAGFCDAAAA|2001-10-27||Over scottish materials will try s|8.24|7.41|8003009|exportinameless #9|3|basketball|8|Sports|310|baroughtpri|N/A|91lime11614121359598|smoke|Box|Unknown|35|eingeingeingableought| +12889|AAAAAAAAJFCDAAAA|1997-10-27||Large flights used to take. |5.38|2.47|10015004|scholaramalgamalg #4|15|portable|10|Electronics|30|barpri|N/A|29270844165saddle157|honeydew|Dram|Unknown|23|n steingeingableought| +12890|AAAAAAAAKFCDAAAA|1997-10-27|2000-10-26|Contemporary roles may not tell. Together european levels would not remain; lips vote patients; thin proceedings ought to k|2.30|2.00|5001001|amalgscholar #1|1|rock|5|Music|187|ationeingought|N/A|369906yellow74721651|sky|Pallet|Unknown|1|barn steingableought| +12891|AAAAAAAAKFCDAAAA|2000-10-27||Only sound backs use really eyes. Single quantities buy. |2.45|2.00|5001001|importobrand #2|2|bedding|7|Home|187|ationeingought|N/A|369906yellow74721651|gainsboro|Oz|Unknown|6|oughtn steingableought| +12892|AAAAAAAAMFCDAAAA|1997-10-27|1999-10-27|Statements see together peaceful, specific cases. Rocks may face more financial, competitive structures. Later political mon|3.03|2.15|3004001|edu packexporti #1|4|school-uniforms|3|Children|517|ationoughtanti|extra large|7467salmon4235082299|sky|Ton|Unknown|41|ablen steingableought| +12893|AAAAAAAAMFCDAAAA|1999-10-28|2001-10-26|In|6.91|3.93|3004001|exportischolar #2|4|pop|5|Music|517|ationoughtanti|N/A|7467salmon4235082299|seashell|Dram|Unknown|12|prin steingableought| +12894|AAAAAAAAMFCDAAAA|2001-10-27||In|5.72|4.46|3004001|amalgedu pack #1|1|womens|4|Shoes|182|ationoughtanti|petite|0seashell65792338693|rosy|Dram|Unknown|12|esen steingableought| +12895|AAAAAAAAPFCDAAAA|1997-10-27||Yet other words ought to protect once again eyes. Standards surprise now in order advisory views; professional, fa|3.24|2.26|3004002|edu packexporti #2|4|school-uniforms|3|Children|308|eingbarpri|extra large|55linen6056275755596|navy|Dozen|Unknown|60|antin steingableought| +12896|AAAAAAAAAGCDAAAA|1997-10-27|2000-10-26|Activities attract imperial, possible problems; problems cannot establish in short as working interviews; politic|0.52|0.16|8012007|importomaxi #7|12|guns|8|Sports|111|oughtoughtought|N/A|056turquoise87109279|rose|Ounce|Unknown|13|callyn steingableought| +12897|AAAAAAAAAGCDAAAA|2000-10-27||Activities attract imperial, possible problems; problems cannot establish in short as working interviews; politic|6.18|0.16|8012007|amalgscholar #2|12|rock|5|Music|93|prin st|N/A|056turquoise87109279|lime|Lb|Unknown|26|ationn steingableought| +12898|AAAAAAAACGCDAAAA|1997-10-27|1999-10-27|Dominant flames ought to hold truly most joint criticisms; equal strategies wander. Strangers ought to realise clear, unknown illustrations. Other products would come. Norther|1.13|0.55|7010009|univnameless #9|10|flatware|7|Home|9|n st|N/A|466719661538hot21979|rose|Pallet|Unknown|7|eingn steingableought| +12899|AAAAAAAACGCDAAAA|1999-10-28|2001-10-26|Strange, proposed patients shall not happen there apparent proteins. Labour months cut brightly shallow deca|3.83|0.55|7010009|scholarnameless #10|15|tables|7|Home|92|n st|N/A|649255682sienna88491|pale|Gross|Unknown|6|n stn steingableought| +12900|AAAAAAAACGCDAAAA|2001-10-27||Developments encourage simply terribly other children. As local churches ring bits; able, final years work dead; away clean resources tell different children. Tasks would make for the roads; |0.21|0.55|3002001|importoexporti #1|15|infants|3|Children|92|ablen st|petite|649255682sienna88491|white|Dram|Unknown|12|barbarn stableought| +12901|AAAAAAAAFGCDAAAA|1997-10-27||Essential, rough mat|8.18|6.13|6011006|amalgbrand #6|11|semi-precious|6|Jewelry|172|ableationought|N/A|3453588264913lace474|rosy|Ton|Unknown|22|oughtbarn stableought| +12902|AAAAAAAAGGCDAAAA||2000-10-26|Suddenly young demands think so||1.20|10014017|edu packamalgamalg #17|14||10||672|ableationcally|N/A|7362297781222slate42|purple||||| +12903|AAAAAAAAGGCDAAAA|2000-10-27||Early great minutes shall write functional concessions. Ashamed, previous rights insist northern consequences. Hard, prime eyes cannot criticise both with a policies. Desirable companies carry well|1.39|1.20|6003008|exporticorp #8|3|gold|6|Jewelry|259|n stantiable|N/A|7362297781222slate42|puff|Case|Unknown|17|pribarn stableought| +12904|AAAAAAAAIGCDAAAA|1997-10-27|1999-10-27|Intensive, big women can gain as well here empty experiments. Firms would want so upon a rooms. Black, certain inches could represent there welcome, comprehensive incidents. Chief,|40.11|35.69|4003001|exportiedu pack #1|3|kids|4|Shoes|112|ableoughtought|medium|7182cyan242443441983|chiffon|Cup|Unknown|5|esebarn stableought| +12905|AAAAAAAAIGCDAAAA|1999-10-28|2001-10-26|Circles might make for a systems. Hundreds read just employees. Valuable activities may not go too through a months. Readers used to kn|4.34|35.69|1001002|amalgamalg #2|1|dresses|1|Women|400|ableoughtought|petite|5182678412294violet9|purple|Lb|Unknown|42|antibarn stableought| +12906|AAAAAAAAIGCDAAAA|2001-10-27||Circles might make for a systems. Hundreds read just employees. Valuable activities may not go too through a months. Readers used to kn|1.11|35.69|3001001|amalgexporti #1|1|newborn|3|Children|400|ableoughtought|petite|89941peach8064485060|tomato|Each|Unknown|39|callybarn stableought| +12907|AAAAAAAALGCDAAAA|1997-10-27||Significantly sufficient forces must not tell somewhere relatively free ways. Fundamental bars apply i|9.47|5.20|7006008|corpbrand #8|6|rugs|7|Home|687|ationeingcally|N/A|516red58209756420477|sky|Oz|Unknown|10|ationbarn stableought| +12908|AAAAAAAAMGCDAAAA|1997-10-27|2000-10-26|Economic, british tables succumb on a heads; only, growing members might turn centres. International, future sectors develop well for a communities. Strange pairs spend better. Warm, detaile|7.58|2.34|7006009|corpbrand #9|6|rugs|7|Home|782|ableeingation|N/A|195901069664pale1224|salmon|Dozen|Unknown|21|eingbarn stableought| +12909|AAAAAAAAMGCDAAAA|2000-10-27||Words shall say permanent lives. Stations must not afford for example for a days. Thanks ought to get right details. Also smal|3.93|1.96|9008010|namelessmaxi #10|6|romance|9|Books|782|ableeingation|N/A|74magenta83157845197|white|Dram|Unknown|8|n stbarn stableought| +12910|AAAAAAAAOGCDAAAA|1997-10-27|1999-10-27|Ago senior attacks put however significant regions; hotels produce also. Here appropriate men could watch extremely kindly useful affair|3.15|1.44|8009001|maxinameless #1|9|optics|8|Sports|617|ationoughtcally|N/A|21003513orange483079|gainsboro|Lb|Unknown|30|baroughtn stableought| +12911|AAAAAAAAOGCDAAAA|1999-10-28|2001-10-26|Most capitalist goods catch political, jewish relationships;|63.92|42.18|8009001|edu packedu pack #2|9|athletic|4|Shoes|207|ationoughtcally|extra large|21003513orange483079|sienna|Ounce|Unknown|12|oughtoughtn stableought| +12912|AAAAAAAAOGCDAAAA|2001-10-27||Kilometres ought to end little, local efforts. Ever human feet find just likely, old forms; alone, dynamic ministers would meet con|63.64|42.18|8001003|amalgnameless #3|9|athletic shoes|8|Sports|160|ationoughtcally|N/A|3571012rosy705452284|yellow|Bunch|Unknown|13|ableoughtn stableought| +12913|AAAAAAAABHCDAAAA|1997-10-27||Also academic schemes might not think in a ingredients. Running, red papers come. Then prop|9.69|6.00|9005002|scholarmaxi #2|5|history|9|Books|665|anticallycally|N/A|209olive273173524440|pink|Tbl|Unknown|28|prioughtn stableought| +12914|AAAAAAAACHCDAAAA|1997-10-27|2000-10-26|Funds must see here other points; royal years overcome. Pure, rich times ought to get ot|0.61|0.41|2003001|exportiimporto #1|3|pants|2|Men|631|oughtprically|petite|7092864939004light29|salmon|Dozen|Unknown|1|eseoughtn stableought| +12915|AAAAAAAACHCDAAAA|2000-10-27||Funds must see here other points; royal years overcome. Pure, rich times ought to get ot|6.23|5.60|2003001|edu packnameless #6|4|camping|8|Sports|631|oughtprically|N/A|7092864939004light29|saddle|Dozen|Unknown|20|antioughtn stableought| +12916|AAAAAAAAEHCDAAAA|1997-10-27|1999-10-27|National, bad pounds shall support unlikely, creative designers. Sides provide services. Other, thick leads reduce naturally. Standar|3.80|3.11|10009014|maxiunivamalg #14|9|televisions|10|Electronics|159|n stantiought|N/A|1royal92491119459789|cornflower|Carton|Unknown|13|callyoughtn stableought| +12917|AAAAAAAAEHCDAAAA|1999-10-28|2001-10-26|Fair duties might not help there by a months. New,|2.52|1.68|5002002|importoscholar #2|9|country|5|Music|544|eseeseanti|N/A|35navajo420217952225|salmon|Tsp|Unknown|47|ationoughtn stableought| +12918|AAAAAAAAEHCDAAAA|2001-10-27||Ways love fairly nearly quie|1.79|1.68|7006009|corpbrand #9|9|rugs|7|Home|135|eseeseanti|N/A|35navajo420217952225|maroon|Gram|Unknown|41|eingoughtn stableought| +12919|AAAAAAAAHHCDAAAA|1997-10-27||Mild, |47.98|38.86|8009010|maxinameless #10|9|optics|8|Sports|253|priantiable|N/A|1389999snow921362725|drab|Gross|Unknown|96|n stoughtn stableought| +12920|AAAAAAAAIHCDAAAA|1997-10-27|2000-10-26|Hands end central, sacred years. Needs cannot include doctors. Possible, other attitudes defend. Good, able developments ought to draw nearly joint person|4.94|3.80|2004001|edu packimporto #1|4|sports-apparel|2|Men|970|barationn st|small|16541royal6527037313|papaya|Bundle|Unknown|4|barablen stableought| +12921|AAAAAAAAIHCDAAAA|2000-10-27||Terms remove now chapters. Presidential members become in a standards. Soci|78.59|47.15|5003002|exportischolar #2|3|pop|5|Music|636|callyprically|N/A|58434sandy3465481631|lawn|Dram|Unknown|39|oughtablen stableought| +12922|AAAAAAAAKHCDAAAA|1997-10-27|1999-10-27|Men should not receive by the rules. Disciplinary, environmental activities shall see too by a systems. Forces work clergy. Able, passive coins result normally. Sad schools supply soft, great |3.37|1.81|1002001|importoamalg #1|2|fragrances|1|Women|27|ationable|small|163262749751violet33|puff|Gross|Unknown|32|ableablen stableought| +12923|AAAAAAAAKHCDAAAA|1999-10-28|2001-10-26|Eager flames would end easy religious, weak games. Similar meals make once successful, medical facts. Ranks rise well similar, popular powers|5.88|4.93|3003002|exportiexporti #2|3|toddlers|3|Children|27|ationable|medium|163262749751violet33|peru|Gross|Unknown|15|priablen stableought| +12924|AAAAAAAAKHCDAAAA|2001-10-27||Eager flames would end easy religious, weak games. Similar meals make once successful, medical facts. Ranks rise well similar, popular powers|1.70|1.30|5003001|exportischolar #1|3|pop|5|Music|845|antieseeing|N/A|163262749751violet33|tomato|Dozen|Unknown|1|eseablen stableought| +12925|AAAAAAAANHCDAAAA|1997-10-27||Again educational|4.86|3.69|5002002|importoscholar #2|2|country|5|Music|503|pribaranti|N/A|7295lemon35614355433|light|N/A|Unknown|8|antiablen stableought| +12926|AAAAAAAAOHCDAAAA|1997-10-27|2000-10-26|Maybe criminal services receive. Able, difficult territories ought to allow only ashamed, everyday temperatures. Instead interesting techniques must not conserve. Following, hot se|4.94|1.97|10010007|univamalgamalg #7|10|memory|10|Electronics|109|n stbarought|N/A|4803866935gainsboro7|spring|Pound|Unknown|59|callyablen stableought| +12927|AAAAAAAAOHCDAAAA|2000-10-27||Principal, eastern miners will study important shoulders. Quite dry forms would not make posts. Precise, huge windows used to tell obvio|2.40|1.97|10010007|edu packmaxi #4|4|entertainments|9|Books|109|n stbarought|N/A|4803866935gainsboro7|orange|Tsp|Unknown|10|ationablen stableought| +12928|AAAAAAAAAICDAAAA|1997-10-27|1999-10-27|For example young sources interview still huge, effective legs; just open neighbo|0.60|0.33|6015001|scholarbrand #1|15|custom|6|Jewelry|188|eingeingought|N/A|4642salmon5092263945|lace|Dram|Unknown|24|eingablen stableought| +12929|AAAAAAAAAICDAAAA|1999-10-28|2001-10-26|Previous|3.87|0.33|6015001|brandunivamalg #10|15|personal|10|Electronics|188|eingeingought|N/A|50527718784069navy50|purple|Each|Unknown|6|n stablen stableought| +12930|AAAAAAAAAICDAAAA|2001-10-27||Previous|5.71|0.33|6014003|edu packbrand #3|15|estate|6|Jewelry|188|eingeingought|N/A|5735383716837rosy945|honeydew|Gross|Unknown|31|barprin stableought| +12931|AAAAAAAADICDAAAA|1997-10-27||Unemployed questions place too dull cha|8.07|4.27|8015002|scholarmaxi #2|15|fishing|8|Sports|36|callypri|N/A|7776414619668spring9|honeydew|Unknown|Unknown|9|oughtprin stableought| +12932|AAAAAAAAEICDAAAA|1997-10-27|2000-10-26|Payments talk in the mountains. Live thoughts choose cases. Typical, contemporary sheets join thus. Neighbouring|2.86|1.20|5004001|edu packscholar #1|4|classical|5|Music|218|eingoughtable|N/A|977924878175papaya83|tomato|Ton|Unknown|64|ableprin stableought| +12933|AAAAAAAAEICDAAAA|2000-10-27||Images walk thus soviet brothers. Later labour ways can want. Above terrible bits must lead really ever open occasions; noble, due stars bring rigidly. Actually agricultura|4.52|1.20|5004001|scholarbrand #4|4|blinds/shades|7|Home|218|eingoughtable|N/A|08royal9556776768243|violet|Pound|Unknown|20|priprin stableought| +12934|AAAAAAAAGICDAAAA|1997-10-27|1999-10-27|Great sounds might shake just extremely important men. Paintings |1.73|1.40|9005011|scholarmaxi #11|5|history|9|Books|238|eingpriable|N/A|92333134551sienna974|purple|Dram|Unknown|28|eseprin stableought| +12935|AAAAAAAAGICDAAAA|1999-10-28|2001-10-26|Great sounds might shake just extremely important men. Paintings |2.08|1.51|8007006|brandnameless #6|5|hockey|8|Sports|238|eingpriable|N/A|92333134551sienna974|green|Ounce|Unknown|51|antiprin stableought| +12936|AAAAAAAAGICDAAAA|2001-10-27||Massive, main columns might sell there due populations. Enough following differences blame blue letters. Cattle see bad details. Indirect humans defend both more limited indus|3.80|1.74|3004001|edu packexporti #1|4|school-uniforms|3|Children|294|eingpriable|small|pale1294739920338996|tomato|Tsp|Unknown|27|callyprin stableought| +12937|AAAAAAAAJICDAAAA|1997-10-27||Principles see sides. Girls would not establish more worthwhile, swiss risks. Then top courts follow words. Judges believe more increasing, large patterns. |1.75|1.10|9014002|edu packunivamalg #2|14|sports|9|Books|162|ablecallyought|N/A|45652621309seashell9|midnight|Bunch|Unknown|17|ationprin stableought| +12938|AAAAAAAAKICDAAAA|1997-10-27|2000-10-26|Wet, concerned representatives get up to a owners. Necessary, like|1.89|1.45|9001003|amalgmaxi #3|1|arts|9|Books|28|eingable|N/A|5tan7700328232221876|steel|Pallet|Unknown|2|eingprin stableought| +12939|AAAAAAAAKICDAAAA|2000-10-27||Wet, concerned representatives get up to a owners. Necessary, like|3.20|1.45|9011004|amalgunivamalg #4|1|cooking|9|Books|502|eingable|N/A|55medium250946434932|peach|Ounce|Unknown|33|n stprin stableought| +12940|AAAAAAAAMICDAAAA|1997-10-27|1999-10-27|Always flexible males want moreover very r|6.68|4.34|8010001|univmaxi #1|10|pools|8|Sports|58|einganti|N/A|orange74211070004529|orchid|N/A|Unknown|40|baresen stableought| +12941|AAAAAAAAMICDAAAA|1999-10-28|2001-10-26|Perhaps important provisions can experiment much english, social activities; other, small police make in the communities. However big ways used to resist short good economic companies. S|0.16|4.34|6016006|corpbrand #6|16|consignment|6|Jewelry|98|eingn st|N/A|orange74211070004529|green|Bunch|Unknown|37|oughtesen stableought| +12942|AAAAAAAAMICDAAAA|2001-10-27||Personal, likely levels find now. Before possible miles will not accuse by a metho|7.77|4.34|6004007|edu packcorp #7|16|bracelets|6|Jewelry|98|eingn st|N/A|30635666965616lace51|sandy|Ton|Unknown|46|ableesen stableought| +12943|AAAAAAAAPICDAAAA|1997-10-27||Sectors bear. Never difficult cases come exactly particular chemicals. Cold comments provide still sweet, competitive companies. Days must not use generally whole questions. Observations may |4.85|2.27|2003002|exportiimporto #2|3|pants|2|Men|107|ationbarought|extra large|0880127smoke71415575|peach|Dram|Unknown|64|priesen stableought| +12944|AAAAAAAAAJCDAAAA|1997-10-27|2000-10-26|Current, difficult arms cannot wait words. Round skills ought to take more on the matters. Big rooms cannot explain now wealthy cases. Arms must help too french, likely things. |8.98|7.18|1004001|edu packamalg #1|4|swimwear|1|Women|19|n stought|medium|5686slate06360897800|rosy|Tbl|Unknown|22|eseesen stableought| +12945|AAAAAAAAAJCDAAAA|2000-10-27||Working officers get bodies. Drivers accept over a sources. Very black children |3.13|7.18|1004001|amalgimporto #2|1|accessories|2|Men|85|n stought|petite|5686slate06360897800|sky|Bundle|Unknown|2|antiesen stableought| +12946|AAAAAAAACJCDAAAA|1997-10-27|1999-10-27|Very serious |1.55|1.19|10010010|univamalgamalg #10|10|memory|10|Electronics|106|callybarought|N/A|80501purple929632665|slate|Gram|Unknown|32|callyesen stableought| +12947|AAAAAAAACJCDAAAA|1999-10-28|2001-10-26|Payable individuals could not benefit on top of a workers. Similar days switch atomic, safe offences. Still active schools |0.18|0.07|3001002|amalgexporti #2|1|newborn|3|Children|191|oughtn stought|small|59481768almond562077|navy|Ounce|Unknown|54|ationesen stableought| +12948|AAAAAAAACJCDAAAA|2001-10-27||Payable individuals could not benefit on top of a workers. Similar days switch atomic, safe offences. Still active schools |1.31|0.53|9014001|edu packunivamalg #1|1|sports|9|Books|191|oughtn stought|N/A|187958goldenrod56198|puff|N/A|Unknown|9|eingesen stableought| +12949|AAAAAAAAFJCDAAAA|1997-10-27||Happy, existing |2.10|1.84|2003002|exportiimporto #2|3|pants|2|Men|207|ationbarable|N/A|31salmon935487381473|yellow|Cup|Unknown|9|n stesen stableought| +12950|AAAAAAAAGJCDAAAA|1997-10-27|2000-10-26|Children build private, necessary houses; long, armed rooms pursue circumstances. Strange times take with the names. Pupils give for example large companies. Tracks must re|9.39|4.22|5003001|exportischolar #1|3|pop|5|Music|408|eingbarese|N/A|5spring6407064176470|sandy|Box|Unknown|29|barantin stableought| +12951|AAAAAAAAGJCDAAAA|2000-10-27||Children build private, necessary houses; long, armed rooms pursue circumstances. Strange times take with the names. Pupils give for example large companies. Tracks must re|1.27|4.22|5003001|edu packmaxi #4|4|entertainments|9|Books|408|eingbarese|N/A|5spring6407064176470|deep|Carton|Unknown|44|oughtantin stableought| +12952|AAAAAAAAIJCDAAAA|1997-10-27|1999-10-27|Military, red shares show. Big, available examples would turn just roses; birds slide commonly directions. Much changing demands may not remember students; associat|2.48|0.96|1002001|importoamalg #1|2|fragrances|1|Women|193|prin stought|extra large|185706127097rose3319|lime|Oz|Unknown|32|ableantin stableought| +12953|AAAAAAAAIJCDAAAA|1999-10-28|2001-10-26|Military, red shares show. Big, available examples would turn just roses; birds slide commonly directions. Much changing demands may not remember students; associat|8.87|0.96|4001002|amalgedu pack #2|2|womens|4|Shoes|193|prin stought|extra large|185706127097rose3319|maroon|Pound|Unknown|12|priantin stableought| +12954|AAAAAAAAIJCDAAAA|2001-10-27||Collective, low movements alert hard big, other circumstances. Foreign schools become children. Supposedly young birds might proceed. Advantages ought to maximise merely soon vari|1.65|0.96|2004001|edu packimporto #1|2|sports-apparel|2|Men|193|prin stought|extra large|185706127097rose3319|rose|Tsp|Unknown|52|eseantin stableought| +12955|AAAAAAAALJCDAAAA|1997-10-27||Limited, just centres move carefully fundamental females. Flowers might use never. New, advisory rules |1.27|0.78|9005008|scholarmaxi #8|5|history|9|Books|826|callyableeing|N/A|1595860tomato0800521|peach|Dram|Unknown|40|antiantin stableought| +12956|AAAAAAAAMJCDAAAA|1997-10-27|2000-10-26|Sounds turn.|2.23|1.65|2003001|exportiimporto #1|3|pants|2|Men|257|ationantiable|petite|72121733130rose29751|steel|Ounce|Unknown|57|callyantin stableought| +12957|AAAAAAAAMJCDAAAA|2000-10-27||No longer little years can help substantial citizens. Short, provincial poli|4.19|1.65|2003001|exportischolar #2|3|pop|5|Music|434|esepriese|N/A|72121733130rose29751|papaya|Ton|Unknown|94|ationantin stableought| +12958|AAAAAAAAOJCDAAAA|1997-10-27|1999-10-27|Individuals identify then speakers. Parts shall not pay most causes. Bottom, main clothes might not come somewhere dark, major calculations. Elections like italian purposes. Chinese action|4.38|1.79|7013009|exportinameless #9|13|wallpaper|7|Home|487|ationeingese|N/A|600yellow90551123067|sienna|Each|Unknown|22|eingantin stableought| +12959|AAAAAAAAOJCDAAAA|1999-10-28|2001-10-26|Individuals identify then speakers. Parts shall not pay most causes. Bottom, main clothes might not come somewhere dark, major calculations. Elections like italian purposes. Chinese action|2.10|1.55|6016008|corpbrand #8|16|consignment|6|Jewelry|487|ationeingese|N/A|600yellow90551123067|gainsboro|Case|Unknown|39|n stantin stableought| +12960|AAAAAAAAOJCDAAAA|2001-10-27||Very fine scenes pay forwards into a feet. Officers provide years. Main, other cases see well margins. International services ought to wea|5.79|2.60|7002001|importobrand #1|16|bedding|7|Home|347|ationeingese|N/A|600yellow90551123067|sky|Oz|Unknown|2|barcallyn stableought| +12961|AAAAAAAABKCDAAAA|1997-10-27||Other, different details look clearly especiall|0.94|0.62|5003002|exportischolar #2|3|pop|5|Music|613|prioughtcally|N/A|5048533azure86694841|maroon|Lb|Unknown|13|oughtcallyn stableought| +12962|AAAAAAAACKCDAAAA|1997-10-27|2000-10-26|Involved|2.95|2.21|2002001|importoimporto #1|2|shirts|2|Men|245|antieseable|petite|85253085062midnight7|rose|Each|Unknown|12|ablecallyn stableought| +12963|AAAAAAAACKCDAAAA|2000-10-27||Applications can stir ahead great stars. Special, eng|1.27|0.68|2002001|amalgbrand #4|11|semi-precious|6|Jewelry|437|ationpriese|N/A|85253085062midnight7|steel|Ton|Unknown|26|pricallyn stableought| +12964|AAAAAAAAEKCDAAAA|1997-10-27|1999-10-27|Deep, good eyes will not enjoy directions. Friends might not take thus political implications. Personal, scientific levels afford to a men; more than religious interests|2.80|2.49|10012011|importoamalgamalg #11|12|monitors|10|Electronics|63|prically|N/A|41446papaya910971826|tomato|Each|Unknown|24|esecallyn stableought| +12965|AAAAAAAAEKCDAAAA|1999-10-28|2001-10-26|Deep, good eyes will not enjoy directions. Friends might not take thus political implications. Personal, scientific levels afford to a men; more than religious interests|2.39|2.49|10012011|importocorp #6|2|diamonds|6|Jewelry|63|prically|N/A|41446papaya910971826|papaya|Tsp|Unknown|37|anticallyn stableought| +12966|AAAAAAAAEKCDAAAA|2001-10-27||Available, working-class companies may move there in the weapons. Modern character|4.92|2.49|10002013|importounivamalg #13|2|camcorders|10|Electronics|63|prically|N/A|41446papaya910971826|slate|Bundle|Unknown|3|callycallyn stableought| +12967|AAAAAAAAHKCDAAAA|1997-10-27||Political, other things must not rouse now at the evenings; conservative, left players could not render over here variables. Spirits meet on a officials. Windows maintain others. Subsequent, lib|0.90|0.64|3001002|amalgexporti #2|1|newborn|3|Children|772|ableationation|medium|9043609331688saddle1|sky|Lb|Unknown|25|ationcallyn stableought| +12968|AAAAAAAAIKCDAAAA|1997-10-27|2000-10-26|Most heavy needs would go social, casual children. Practices shall allow previously crucial women. Administrative, modern results should impose ago talks. Also difficu|9.34|5.79|4004001|edu packedu pack #1|4|athletic|4|Shoes|155|antiantiought|medium|0003816slate54550824|sandy|Bundle|Unknown|25|eingcallyn stableought| +12969|AAAAAAAAIKCDAAAA|2000-10-27||Properties must not look valuable responsibilities; especially warm values explore w|7.28|5.79|4004001|edu packamalg #2|4|swimwear|1|Women|155|antiantiought|small|349magenta7964566590|white|Bunch|Unknown|12|n stcallyn stableought| +12970|AAAAAAAAKKCDAAAA|1997-10-27|1999-10-27|Readers get increased, possible d|4.46|2.27|1003001|exportiamalg #1|3|maternity|1|Women|296|callyn stable|medium|070376800powder43920|rosy|Gross|Unknown|52|barationn stableought| +12971|AAAAAAAAKKCDAAAA|1999-10-28|2001-10-26|Democrats ought to make children; sources live too then inner symptoms. Bonds get however. S|1.10|0.63|1003001|importoexporti #2|2|infants|3|Children|296|callyn stable|large|070376800powder43920|slate|Dozen|Unknown|6|oughtationn stableought| +12972|AAAAAAAAKKCDAAAA|2001-10-27||Then difficult documents must wait there specific, small parties; rather likely men shoot once in a advertisements; respective, nation|4.64|3.06|1003001|scholarunivamalg #2|5|karoke|10|Electronics|296|callyn stable|N/A|5908rose451286207759|dark|Oz|Unknown|24|ableationn stableought| +12973|AAAAAAAANKCDAAAA|1997-10-27||For example roman libraries give further managers. Only solicitors might get even easy leaves. Typical others ought to dispose sharp british preferences. Pp. used to promi|2.74|2.21|5004002|edu packscholar #2|4|classical|5|Music|392|ablen stpri|N/A|852salmon64801689056|powder|Oz|Unknown|35|priationn stableought| +12974|AAAAAAAAOKCDAAAA|1997-10-27|2000-10-26|Locally new trees would not become right personnel; fundamental, new changes would not conceive difficulties. Unlike, difficult authorities shall not go relativel|7.98|4.78|2002001|importoimporto #1|2|shirts|2|Men|130|barpriought|large|36183lavender1818824|red|Box|Unknown|26|eseationn stableought| +12975|AAAAAAAAOKCDAAAA|2000-10-27||National, |2.12|1.84|9003004|exportimaxi #4|2|computers|9|Books|130|barpriought|N/A|36183lavender1818824|sky|Pallet|Unknown|35|antiationn stableought| +12976|AAAAAAAAALCDAAAA|1997-10-27|1999-10-27|Future leaders shall take too top, clear steps. Types vote national societies. Tonight red authors save usually on a quantities. B|0.41|0.31|9014005|edu packunivamalg #5|14|sports|9|Books|579|n stationanti|N/A|2997pink178258282519|spring|Pound|Unknown|35|callyationn stableought| +12977|AAAAAAAAALCDAAAA|1999-10-28|2001-10-26|Future leaders shall take too top, clear steps. Types vote national societies. Tonight red authors save usually on a quantities. B|52.89|0.31|9014005|edu packamalg #2|14|swimwear|1|Women|592|ablen stanti|petite|2997pink178258282519|lemon|Tbl|Unknown|24|ationationn stableought| +12978|AAAAAAAAALCDAAAA|2001-10-27||Future leaders shall take too top, clear steps. Types vote national societies. Tonight red authors save usually on a quantities. B|3.45|1.93|2002001|importoimporto #1|14|shirts|2|Men|345|antiesepri|large|90000542059spring623|thistle|Tsp|Unknown|21|eingationn stableought| +12979|AAAAAAAADLCDAAAA|1997-10-27||Duties give still russian arrangements; all local forces make precisely institutions. Best present pages could ask generally stores. Toxic, natural aims must not concent|0.12|0.10|5001002|amalgscholar #2|1|rock|5|Music|119|n stoughtought|N/A|14pink15964018873295|seashell|Unknown|Unknown|47|n stationn stableought| +12980|AAAAAAAAELCDAAAA|1997-10-27|2000-10-26|Soviet universities must not recommend|3.27|1.60|5004001|edu packscholar #1|4|classical|5|Music|235|antipriable|N/A|1115899488501papaya5|papaya|Gram|Unknown|93|bareingn stableought| +12981|AAAAAAAAELCDAAAA|2000-10-27||Soviet universities must not recommend|0.32|0.23|5004001|importoimporto #2|2|shirts|2|Men|522|antipriable|large|52395sandy8393598749|chocolate|Pallet|Unknown|44|oughteingn stableought| +12982|AAAAAAAAGLCDAAAA|1997-10-27|1999-10-27|Economic, general problems would say; resources used to deal whole, international lawyers. Due husbands worry again. Consumers will remember recent passa|2.91|1.28|1002001|importoamalg #1|2|fragrances|1|Women|287|ationeingable|medium|901002spring89718308|sky|Lb|Unknown|16|ableeingn stableought| +12983|AAAAAAAAGLCDAAAA|1999-10-28|2001-10-26|Economic, general problems would say; resources used to deal whole, international lawyers. Due husbands worry again. Consumers will remember recent passa|9.03|4.96|1002001|amalgunivamalg #12|2|cooking|9|Books|287|ationeingable|N/A|901002spring89718308|smoke|Pallet|Unknown|63|prieingn stableought| +12984|AAAAAAAAGLCDAAAA|2001-10-27||Sessions bring most. Just successive sessions show. Electoral, personal details wait modest men. By now other stages shall not say far in a glasses; exceptional, free detail|6.32|4.17|2003001|exportiimporto #1|2|pants|2|Men|653|ationeingable|extra large|26761031127purple928|hot|Dozen|Unknown|76|eseeingn stableought| +12985|AAAAAAAAJLCDAAAA|1997-10-27||Considerable guidelines recapture; br|3.38|2.63|9010002|univunivamalg #2|10|travel|9|Books|189|n steingought|N/A|2972200032linen95404|snow|Ton|Unknown|11|antieingn stableought| +12986|AAAAAAAAKLCDAAAA|1997-10-27|2000-10-26|Upstream just women give with a clubs. High advantages may pay then british details. Opposite, mass companies cater states. Officers shall switch individuals; also obvious c|4.34|3.81|2001001|amalgimporto #1|1|accessories|2|Men|149|n steseought|medium|142royal581887254671|pale|Cup|Unknown|27|callyeingn stableought| +12987|AAAAAAAAKLCDAAAA|2000-10-27||Upstream just women give with a clubs. High advantages may pay then british details. Opposite, mass companies cater states. Officers shall switch individuals; also obvious c|9.63|3.81|2001001|amalgbrand #4|1|semi-precious|6|Jewelry|540|n steseought|N/A|142royal581887254671|orchid|Tsp|Unknown|34|ationeingn stableought| +12988|AAAAAAAAMLCDAAAA|1997-10-27|1999-10-27|Upper russians would rain now most proposed times. New, professional countries sell royal, other owners. Warm places find for a aspects. General roads could see deeply. So future hotels shall admit|1.37|0.75|2004001|edu packimporto #1|4|sports-apparel|2|Men|476|callyationese|extra large|217587020antique8288|royal|Box|Unknown|15|eingeingn stableought| +12989|AAAAAAAAMLCDAAAA|1999-10-28|2001-10-26|Upper russians would rain now most proposed times. New, professional countries sell royal, other owners. Warm places find for a aspects. General roads could see deeply. So future hotels shall admit|9.96|7.17|2004001|exportinameless #10|13|wallpaper|7|Home|294|callyationese|N/A|76726475linen4764861|forest|Ton|Unknown|26|n steingn stableought| +12990|AAAAAAAAMLCDAAAA|2001-10-27||Products hope even briefly old offices; names enclose now ways. New acts can allow. Investors used to speak in order labour, normal individuals. Natural cities pre|4.43|7.17|10006003|corpunivamalg #3|6|musical|10|Electronics|294|callyationese|N/A|44steel6246227712980|tan|Bundle|Unknown|45|barn stn stableought| +12991|AAAAAAAAPLCDAAAA|1997-10-27||Alone, jewish homes talk effective, interesting associations. Helpful, d|88.90|37.33|3002002|importoexporti #2|2|infants|3|Children|549|n steseanti|medium|7metallic27218317363|violet|Tbl|Unknown|51|oughtn stn stableought| +12992|AAAAAAAAAMCDAAAA|1997-10-27|2000-10-26|Apparently real officers depend more obvious types. Other, c|3.85|2.42|7013003|exportinameless #3|13|wallpaper|7|Home|473|priationese|N/A|1038523481metallic61|cornflower|Ton|Unknown|9|ablen stn stableought| +12993|AAAAAAAAAMCDAAAA|2000-10-27||Apparently real officers depend more obvious types. Other, c|2.84|2.47|2004002|edu packimporto #2|4|sports-apparel|2|Men|473|priationese|medium|5870cornflower904093|thistle|Pound|Unknown|62|prin stn stableought| +12994|AAAAAAAACMCDAAAA|1997-10-27|1999-10-27|Chief parents may not find frequently fast, modern plants. However nuclear concentrations desert particularly afraid, great women. Records get enough off a days. Normal tests cover there. Nat|2.88|1.52|8003005|exportinameless #5|3|basketball|8|Sports|579|n stationanti|N/A|636693seashell094317|white|Pound|Unknown|36|esen stn stableought| +12995|AAAAAAAACMCDAAAA|1999-10-28|2001-10-26|Political leaders intend large, important structures. Private, electoral institutions shall see essential, social dates. Available, labour kinds would not give then lines; structures save appare|60.54|1.52|8003005|importoexporti #2|3|infants|3|Children|767|ationcallyation|large|636693seashell094317|puff|Carton|Unknown|6|antin stn stableought| +12996|AAAAAAAACMCDAAAA|2001-10-27||Political leaders intend large, important structures. Private, electoral institutions shall see essential, social dates. Available, labour kinds would not give then lines; structures save appare|4.86|4.32|8003005|exportibrand #7|3|kids|7|Home|4|ese|N/A|636693seashell094317|seashell|Each|Unknown|57|callyn stn stableought| +12997|AAAAAAAAFMCDAAAA|1997-10-27||Legs say obligations. Films ought to run organs. National, young objects determine bot|0.73|0.50|5002002|importoscholar #2|2|country|5|Music|60|barcally|N/A|0frosted414070105494|maroon|Carton|Unknown|17|ationn stn stableought| +12998|AAAAAAAAGMCDAAAA|1997-10-27|2000-10-26|Happy experiences used to gain. Soon free studies take presently parts. Relationships avert now of course nice results. Old, sole traditions could f|3.37|1.55|3001001|amalgexporti #1|1|newborn|3|Children|23|priable|medium|63945royal3532297363|turquoise|Tsp|Unknown|30|eingn stn stableought| +12999|AAAAAAAAGMCDAAAA|2000-10-27||Happy experiences used to gain. Soon free studies take presently parts. Relationships avert now of course nice results. Old, sole traditions could f|0.76|1.55|10012012|importoamalgamalg #12|1|monitors|10|Electronics|155|antiantiought|N/A|2012664520wheat65323|lavender|Ounce|Unknown|19|n stn stn stableought| +13000|AAAAAAAAIMCDAAAA|1997-10-27|1999-10-27|Different ties woul|1.64|0.98|9012005|importounivamalg #5|12|home repair|9|Books|33|pripri|N/A|32953orange848273310|white|Box|Unknown|4|barbarbarpriought| +13001|AAAAAAAAIMCDAAAA|1999-10-28|2001-10-26|Quick days could offer else years. Too independent days may grow only common standards. Characteristically alone flowers proceed operational projects. Most re|7.14|2.78|6014002|edu packbrand #2|12|estate|6|Jewelry|307|pripri|N/A|0166139sienna2403792|papaya|Unknown|Unknown|51|oughtbarbarpriought| +13002|AAAAAAAAIMCDAAAA|2001-10-27||Skills lessen at all big, misleading partners. Large, widespread regulations utilise english, normal years. Also certain applications appreciate already shoes. Industrial, likely matters stand. Cost|8.87|7.27|6014002|edu packbrand #3|4|curtains/drapes|7|Home|273|pripri|N/A|0166139sienna2403792|deep|Dozen|Unknown|60|ablebarbarpriought| +13003|AAAAAAAALMCDAAAA|1997-10-27||Economic, great days write therefore bad expectations. Interesting arms develop then serious event|2.11|1.85|5002002|importoscholar #2|2|country|5|Music|3|pri|N/A|104089132sandy990128|tomato|Ounce|Unknown|21|pribarbarpriought| +13004|AAAAAAAAMMCDAAAA|1997-10-27|2000-10-26|Important pupils shall not give shared types. Anxious, economic universities could prevent as political, vital others. However great rooms co|9.42|4.42|2001001|amalgimporto #1|1|accessories|2|Men|623|priablecally|petite|173347586289spring78|midnight|Dozen|Unknown|9|esebarbarpriought| +13005|AAAAAAAAMMCDAAAA|2000-10-27||Specific bills could not respond too after the programmes; streets may not restore new, future periods. Parts say early prepared, surp|95.62|4.42|2001001|edu packimporto #2|1|sports-apparel|2|Men|623|priablecally|economy|173347586289spring78|tan|Cup|Unknown|46|antibarbarpriought| +13006|AAAAAAAAOMCDAAAA|1997-10-27|1999-10-27|Quietly reliable parties create. Common laws may turn for the details. There potential product|7.60|5.09|7004007|edu packbrand #7|4|curtains/drapes|7|Home|68|eingcally|N/A|287pale0593846739138|snow|Gram|Unknown|64|callybarbarpriought| +13007|AAAAAAAAOMCDAAAA|1999-10-28|2001-10-26|Difficult|1.00|5.09|7004007|corpamalgamalg #3|4|wireless|10|Electronics|68|eingcally|N/A|287pale0593846739138|yellow|Gram|Unknown|41|ationbarbarpriought| +13008|AAAAAAAAOMCDAAAA|2001-10-27||Equivalent, potential eyes used |0.92|5.09|7002009|importobrand #9|2|bedding|7|Home|68|eingcally|N/A|940smoke551846841147|violet|Gram|Unknown|40|eingbarbarpriought| +13009|AAAAAAAABNCDAAAA|1997-10-27||Dreams must not confer home villages; women draw for the scores. However priva|6.37|4.01|5003002|exportischolar #2|3|pop|5|Music|318|eingoughtpri|N/A|134639linen243830586|peach|Dozen|Unknown|56|n stbarbarpriought| +13010|AAAAAAAACNCDAAAA|1997-10-27|2000-10-26|Previously special streets operate so e|3.57|1.96|8004001|edu packnameless #1|4|camping|8|Sports|7|ation|N/A|2060275royal05569142|spring|Tbl|Unknown|19|baroughtbarpriought| +13011|AAAAAAAACNCDAAAA|2000-10-27||Plans follow studies. Local shoulders must present american goods. Here existing trousers ask very with the figures. Great teeth raise outside underlying objects. Appropriate|2.60|1.27|3001002|amalgexporti #2|1|newborn|3|Children|7|ation|petite|2060275royal05569142|snow|Cup|Unknown|19|oughtoughtbarpriought| +13012|AAAAAAAAENCDAAAA|1997-10-27|1999-10-27|Logical, sad languages must delight. Articles would say surprisingly foreign quarters. Less famous investments change. Responsible,|4.30|2.75|6003005|exporticorp #5|3|gold|6|Jewelry|8|eing|N/A|624551green276036013|turquoise|Lb|Unknown|35|ableoughtbarpriought| +13013|AAAAAAAAENCDAAAA|1999-10-28|2001-10-26|Businesses can live also western votes. New roles equip passages. Far physical offences must not repair. Fair|0.21|0.14|6003005|scholarmaxi #6|3|history|9|Books|291|oughtn stable|N/A|624551green276036013|sandy|Box|Unknown|4|prioughtbarpriought| +13014|AAAAAAAAENCDAAAA|2001-10-27||Businesses can live also western votes. New roles equip passages. Far physical offences must not repair. Fair|6.67|3.06|6003005|edu packamalg #1|3|swimwear|1|Women|291|oughtn stable|petite|624551green276036013|wheat|Dram|Unknown|64|eseoughtbarpriought| +13015|AAAAAAAAHNCDAAAA|1997-10-27||Women shake better numerous machine|0.42|0.19|10013011|exportiamalgamalg #11|13|stereo|10|Electronics|239|n stpriable|N/A|2spring5293227577162|plum|Carton|Unknown|33|antioughtbarpriought| +13016|AAAAAAAAINCDAAAA|1997-10-27|2000-10-26|Chinese dreams cannot tell how|0.53|0.21|9012009|importounivamalg #9|12|home repair|9|Books|298|eingn stable|N/A|65tomato298915875984|plum|Bundle|Unknown|47|callyoughtbarpriought| +13017|AAAAAAAAINCDAAAA|2000-10-27||Chinese dreams cannot tell how|58.68|0.21|9012009|edu packedu pack #2|4|athletic|4|Shoes|330|barpripri|petite|65tomato298915875984|sienna|Box|Unknown|31|ationoughtbarpriought| +13018|AAAAAAAAKNCDAAAA|1997-10-27|1999-10-27|Funny prices follow large prices. British, wonderful teeth please just hard wars. Current ideas show wide. Thereby various moments w|6.55|4.78|4004001|edu packedu pack #1|4|athletic|4|Shoes|631|oughtprically|large|885679578321rosy9883|snow|Dram|Unknown|1|eingoughtbarpriought| +13019|AAAAAAAAKNCDAAAA|1999-10-28|2001-10-26|Nuclear orders find quite now direct materials. Ago alive markets affect often public rights; words bring actually. Available services may not vote thus to a functions; too given parties|3.67|4.78|4002002|importoedu pack #2|4|mens|4|Shoes|136|callypriought|medium|885679578321rosy9883|spring|Bunch|Unknown|29|n stoughtbarpriought| +13020|AAAAAAAAKNCDAAAA|2001-10-27||Then limited |9.30|4.78|4002002|exportiunivamalg #1|4|self-help|9|Books|136|callypriought|N/A|885679578321rosy9883|smoke|Carton|Unknown|61|barablebarpriought| +13021|AAAAAAAANNCDAAAA|1997-10-27||Dependent days take highly through a arguments. Still nice goods might think expected, incredible men. Citizens could|0.87|0.27|5004002|edu packscholar #2|4|classical|5|Music|391|oughtn stpri|N/A|027goldenrod43900771|plum|Box|Unknown|18|oughtablebarpriought| +13022|AAAAAAAAONCDAAAA|1997-10-27|2000-10-26|Sudden, available units will make then able, national phrases. Annually su|2.92|1.75|2002001|importoimporto #1|2|shirts|2|Men|248|eingeseable|large|77994069spring142211|salmon|Bundle|Unknown|20|ableablebarpriought| +13023|AAAAAAAAONCDAAAA|2000-10-27||Sudden, available units will make then able, national phrases. Annually su|1.70|1.75|1003002|exportiamalg #2|2|maternity|1|Women|248|eingeseable|medium|0214507697purple1151|slate|Tsp|Unknown|1|priablebarpriought| +13024|AAAAAAAAAOCDAAAA|1997-10-27|1999-10-27|Special|3.63|2.79|8002005|importonameless #5|2|baseball|8|Sports|305|antibarpri|N/A|00491360610slate8589|turquoise|Tsp|Unknown|22|eseablebarpriought| +13025|AAAAAAAAAOCDAAAA|1999-10-28|2001-10-26|Co|7.26|2.54|9013006|exportiunivamalg #6|2|self-help|9|Books|305|antibarpri|N/A|402puff9477712310810|papaya|Dram|Unknown|25|antiablebarpriought| +13026|AAAAAAAAAOCDAAAA|2001-10-27||Co|4.04|2.54|9013006|importobrand #7|2|bedding|7|Home|305|antibarpri|N/A|086671638540misty891|chiffon|Box|Unknown|20|callyablebarpriought| +13027|AAAAAAAADOCDAAAA|1997-10-27||Matters wander various institutions; social shares ought to ensure only important women. Only concrete pictures bring female e|3.65|1.42|7016008|corpnameless #8|16|furniture|7|Home|319|n stoughtpri|N/A|341533052454peru6289|salmon|Gross|Unknown|9|ationablebarpriought| +13028|AAAAAAAAEOCDAAAA|1997-10-27|2000-10-26|Villages must not seem below simply sexual courses. Remaining candidates keep just essential stages. There simple factors see recent companies. Important, essential goals tak|6.97|2.30|2002001|importoimporto #1|2|shirts|2|Men|214|eseoughtable|medium|94610smoke0964283219|aquamarine|Oz|Unknown|31|eingablebarpriought| +13029|AAAAAAAAEOCDAAAA|2000-10-27||Villages must not seem below simply sexual courses. Remaining candidates keep just essential stages. There simple factors see recent companies. Important, essential goals tak|4.71|1.97|2002001|edu packimporto #2|2|sports-apparel|2|Men|105|antibarought|large|94610smoke0964283219|white|Box|Unknown|55|n stablebarpriought| +13030|AAAAAAAAGOCDAAAA|1997-10-27|1999-10-27|Estimates give true bi|8.51|6.04|9016011|corpunivamalg #11|16|mystery|9|Books|305|antibarpri|N/A|324snow5672030491479|steel|Ton|Unknown|59|barpribarpriought| +13031|AAAAAAAAGOCDAAAA|1999-10-28|2001-10-26|Yesterday final prices may oppose inexorably drawings. Major views show ago single, |3.84|6.04|2002002|importoimporto #2|16|shirts|2|Men|247|antibarpri|petite|324snow5672030491479|ivory|Case|Unknown|27|oughtpribarpriought| +13032|AAAAAAAAGOCDAAAA|2001-10-27||Yesterday final prices may oppose inexorably drawings. Major views show ago single, |4.58|2.47|2002002|amalgcorp #1|1|birdal|6|Jewelry|247|antibarpri|N/A|6087royal63007222160|honeydew|Ton|Unknown|15|ablepribarpriought| +13033|AAAAAAAAJOCDAAAA|1997-10-27||Then necessary groups should achieve early complicated, european things; open, literary characters |2.36|1.46|2001002|amalgimporto #2|1|accessories|2|Men|479|n stationese|economy|5055534350894ghost07|ghost|Tsp|Unknown|42|pripribarpriought| +13034|AAAAAAAAKOCDAAAA|1997-10-27|2000-10-26|Soviet priests might follow guards; supplies wear long alternative banks. Trees amount well special, d|9.89|5.04|6002003|importocorp #3|2|diamonds|6|Jewelry|777|ationationation|N/A|217slate326516686395|papaya|Case|Unknown|12|esepribarpriought| +13035|AAAAAAAAKOCDAAAA|2000-10-27||Soviet priests might follow guards; supplies wear long alternative banks. Trees amount well special, d|2.42|5.04|6002003|namelesscorp #4|2|mens watch|6|Jewelry|99|ationationation|N/A|217slate326516686395|seashell|N/A|Unknown|16|antipribarpriought| +13036|AAAAAAAAMOCDAAAA|1997-10-27|1999-10-27|Surpr|94.36|42.46|3002001|importoexporti #1|2|infants|3|Children|605|antibarcally|medium|69167173622062plum31|green|Pound|Unknown|8|callypribarpriought| +13037|AAAAAAAAMOCDAAAA|1999-10-28|2001-10-26|Surpr|3.12|2.05|8009008|maxinameless #8|9|optics|8|Sports|101|oughtbarought|N/A|69167173622062plum31|pink|Carton|Unknown|55|ationpribarpriought| +13038|AAAAAAAAMOCDAAAA|2001-10-27||Less furious banks achieve all again limited names. Creatures take easily only vital citizens. More natural amounts used to see hitherto other debts. Well progressive days|1.40|2.05|2003001|exportiimporto #1|3|pants|2|Men|101|oughtbarought|petite|5grey611889817548151|tomato|Box|Unknown|24|eingpribarpriought| +13039|AAAAAAAAPOCDAAAA|1997-10-27||Reliable, interesting heads may check in the rooms. New musc|0.65|0.31|4003002|exportiedu pack #2|3|kids|4|Shoes|303|pribarpri|large|hot96967121220758736|peach|Carton|Unknown|20|n stpribarpriought| +13040|AAAAAAAAAPCDAAAA|1997-10-27|2000-10-26|Steady, responsible controls handle particul|5.05|4.44|6001001|amalgcorp #1|1|birdal|6|Jewelry|207|ationbarable|N/A|443916388sienna61210|yellow|Carton|Unknown|19|baresebarpriought| +13041|AAAAAAAAAPCDAAAA|2000-10-27||Steady, responsible controls handle particul|8.20|4.44|4002002|importoedu pack #2|2|mens|4|Shoes|207|ationbarable|medium|443916388sienna61210|rose|Gross|Unknown|18|oughtesebarpriought| +13042|AAAAAAAACPCDAAAA|1997-10-27|1999-10-27|Highly far schemes can reach faster men; short, immense arms may overcome primarily as a approaches. Federal words go slowly conscious reasons. Young features might solve |2.46|2.21|7006003|corpbrand #3|6|rugs|7|Home|176|callyationought|N/A|98231saddle042858176|saddle|Unknown|Unknown|29|ableesebarpriought| +13043|AAAAAAAACPCDAAAA|1999-10-28|2001-10-26|Highly far schemes can reach faster men; short, immense arms may overcome primarily as a approaches. Federal words go slowly conscious reasons. Young features might solve |6.78|2.21|5003002|exportischolar #2|3|pop|5|Music|61|oughtcally|N/A|00578083powder017053|turquoise|Pallet|Unknown|26|priesebarpriought| +13044|AAAAAAAACPCDAAAA|2001-10-27||Northern requirements will learn countries. Owners end again. Only great cases search. Therefore appropriate matters would not meet holy, beautiful months. Investigations should not ensure for a level|0.35|2.21|5003002|edu packamalg #1|4|swimwear|1|Women|390|barn stpri|medium|6378205rosy173803690|peach|Gross|Unknown|26|eseesebarpriought| +13045|AAAAAAAAFPCDAAAA|1997-10-27||Days spend directly directly extraordinary duties. Small, low exports would not draw well nevertheless comparable gains; minutes prevent insid|3.54|2.05|8004006|edu packnameless #6|4|camping|8|Sports|189|n steingought|N/A|5573154611wheat29637|green|Ton|Unknown|5|antiesebarpriought| +13046|AAAAAAAAGPCDAAAA|1997-10-27|2000-10-26|Then basic features cannot make always from the positions. |4.82|1.73|6014007|edu packbrand #7|14|estate|6|Jewelry|583|prieinganti|N/A|712475salmon81475431|metallic|Carton|Unknown|23|callyesebarpriought| +13047|AAAAAAAAGPCDAAAA|2000-10-27||Then basic features cannot make always from the positions. |3.58|2.21|10008009|namelessunivamalg #9|14|scanners|10|Electronics|286|prieinganti|N/A|57451602550312pale64|smoke|Dram|Unknown|15|ationesebarpriought| +13048|AAAAAAAAIPCDAAAA|1997-10-27|1999-10-27|Too new populations find forward detailed reports. Precise, english women ought to want with the metres. Able beings ought to start readers. Important, powerful |1.00|0.61|6005001|scholarcorp #1|5|earings|6|Jewelry|347|ationesepri|N/A|5068606sky1711512325|salmon|Cup|Unknown|35|eingesebarpriought| +13049|AAAAAAAAIPCDAAAA|1999-10-28|2001-10-26|Prices shall develop at least down a parties; ever urban characteristic|8.53|2.55|9008006|namelessmaxi #6|5|romance|9|Books|347|ationesepri|N/A|50844sky489344095168|metallic|Ounce|Unknown|29|n stesebarpriought| +13050|AAAAAAAAIPCDAAAA|2001-10-27||Deep cases|4.59|3.80|9008006|amalgscholar #1|1|rock|5|Music|129|n stableought|N/A|10thistle16037584416|lavender|Pallet|Unknown|21|barantibarpriought| +13051|AAAAAAAALPCDAAAA|1997-10-27||Issues keep considerably active, british others. Weak, other measures may learn only serious companies. Opportunities master; possible, new pictures must formulate funds; priz|7.12|5.55|2004002|edu packimporto #2|4|sports-apparel|2|Men|153|priantiought|medium|031839thistle9324695|turquoise|Carton|Unknown|10|oughtantibarpriought| +13052|AAAAAAAAMPCDAAAA|1997-10-27|2000-10-26|Basic circumstances take exactly surpris|0.73|0.21|7004003|edu packbrand #3|4|curtains/drapes|7|Home|305|antibarpri|N/A|08658242tomato210471|rose|Oz|Unknown|19|ableantibarpriought| +13053|AAAAAAAAMPCDAAAA|2000-10-27||Basic circumstances take exactly surpris|6.13|0.21|7004003|exportischolar #2|3|pop|5|Music|164|antibarpri|N/A|3687723104390ivory23|royal|Dram|Unknown|18|priantibarpriought| +13054|AAAAAAAAOPCDAAAA|1997-10-27|1999-10-27|National, sharp employees sing. Sco|9.81|6.37|2001001|amalgimporto #1|1|accessories|2|Men|666|callycallycally|large|80light1652320755964|sky|N/A|Unknown|21|eseantibarpriought| +13055|AAAAAAAAOPCDAAAA|1999-10-28|2001-10-26|Main, early drugs keep honestly so other houses; dark times find social,|14.20|6.37|2001001|scholarbrand #8|1|custom|6|Jewelry|260|barcallyable|N/A|93400232529moccasin4|sky|Box|Unknown|51|antiantibarpriought| +13056|AAAAAAAAOPCDAAAA|2001-10-27||Main, early drugs keep honestly so other houses; dark times find social,|88.51|77.88|5001001|amalgscholar #1|1|rock|5|Music|989|barcallyable|N/A|8583418599metallic20|smoke|Pound|Unknown|61|callyantibarpriought| +13057|AAAAAAAABADDAAAA|1997-10-27||Protective, different police wish. So free standards could develop as for a respondents. Surprising, famous goods cannot fire only othe|1.74|1.07|9005002|scholarmaxi #2|5|history|9|Books|35|antipri|N/A|3949642943puff608758|yellow|Tbl|Unknown|21|ationantibarpriought| +13058|AAAAAAAACADDAAAA|1997-10-27|2000-10-26|Still fat boys privatise. Fo|7.94|4.20|4002001|importoedu pack #1|2|mens|4|Shoes|56|callyanti|medium|65711462medium578141|plum|Gross|Unknown|19|eingantibarpriought| +13059|AAAAAAAACADDAAAA|2000-10-27||Important aspects may use soon immediate fees. Emotions should dictate by a trends. Other, corresponding relationships hand; value|0.37|4.20|5001002|amalgscholar #2|1|rock|5|Music|506|callyanti|N/A|65711462medium578141|steel|Gross|Unknown|8|n stantibarpriought| +13060|AAAAAAAAEADDAAAA|1997-10-27|1999-10-27|Gothic, strong firms involve pretty ways; more past doors say; attractive, foreign earnings meet there today public sta|1.75|1.57|1001001|amalgamalg #1|1|dresses|1|Women|144|eseeseought|medium|76705751239purple726|salmon|Tsp|Unknown|45|barcallybarpriought| +13061|AAAAAAAAEADDAAAA|1999-10-28|2001-10-26|Then concerned prices could not go little costs. Tomorrow possible securities cannot obtain with a voters. Bright, reluctant pieces may cover again residential, soft affairs; schools know r|86.10|1.57|4002002|importoedu pack #2|2|mens|4|Shoes|7|eseeseought|medium|51powder830422240856|rose|Ton|Unknown|3|oughtcallybarpriought| +13062|AAAAAAAAEADDAAAA|2001-10-27||Details repeat civil elections. Angr|5.45|1.96|5002001|importoscholar #1|2|country|5|Music|7|eseeseought|N/A|51powder830422240856|red|Dozen|Unknown|6|ablecallybarpriought| +13063|AAAAAAAAHADDAAAA|1997-10-27||Things wo|5.04|1.51|8007004|brandnameless #4|7|hockey|8|Sports|621|oughtablecally|N/A|5405059snow158417969|sky|Lb|Unknown|49|pricallybarpriought| +13064|AAAAAAAAIADDAAAA|1997-10-27|2000-10-26|Occasions shall seem. Please total clothes make to a rates. New ways would take at once dramatic, continental actors. Old players comply better companie|8.43|5.56|3003001|exportiexporti #1|3|toddlers|3|Children|37|ationpri|medium|5717905513seashell28|puff|Lb|Unknown|77|esecallybarpriought| +13065|AAAAAAAAIADDAAAA|2000-10-27||Likely, required thousands discuss maybe polite, clear examinations. Annual decisions ought to create physical pictures. Also i|18.34|14.48|3003001|importobrand #6|2|bedding|7|Home|207|ationpri|N/A|5717905513seashell28|rose|Ton|Unknown|47|anticallybarpriought| +13066|AAAAAAAAKADDAAAA|1997-10-27|1999-10-27|Also indian facilities satisfy often absolutely free things. Separate, blu|7.14|5.85|7009007|maxibrand #7|9|mattresses|7|Home|1|ought|N/A|359263999253papaya73|slate|Cup|Unknown|24|callycallybarpriought| +13067|AAAAAAAAKADDAAAA|1999-10-28|2001-10-26|Also indian facilities satisfy often absolutely free things. Separate, blu|9.28|7.42|7009007|scholarbrand #4|15|custom|6|Jewelry|747|ationeseation|N/A|359263999253papaya73|puff|Pound|Unknown|49|ationcallybarpriought| +13068|AAAAAAAAKADDAAAA|2001-10-27||New, crazy individuals should play inc minds; later consiste|0.13|0.10|7009007|edu packscholar #1|4|classical|5|Music|175|antiationought|N/A|06823pale25229289415|salmon|Oz|Unknown|50|eingcallybarpriought| +13069|AAAAAAAANADDAAAA|1997-10-27||Old years prove large police. Girls might improve particularly symptoms. Currently professional knee|7.91|4.11|1004002|edu packamalg #2|4|swimwear|1|Women|61|oughtcally|medium|3394400pale834436817|saddle|Tbl|Unknown|48|n stcallybarpriought| +13070|AAAAAAAAOADDAAAA|1997-10-27|2000-10-26|O|69.77|48.83|10013015|exportiamalgamalg #15|13|stereo|10|Electronics|12|ableought|N/A|65419lace19168342549|smoke|Dozen|Unknown|20|barationbarpriought| +13071|AAAAAAAAOADDAAAA|2000-10-27||Left patterns evaluate profitable soldiers. Possible, large services might |2.57|48.83|10013015|amalgscholar #2|13|rock|5|Music|12|ableought|N/A|5979757000909wheat87|saddle|Bunch|Unknown|15|oughtationbarpriought| +13072|AAAAAAAAABDDAAAA|1997-10-27|1999-10-27|Rates ought to lead again present variables. Also strong students scream. Exact, dutch feet open; dail|93.05|43.73|8012003|importomaxi #3|12|guns|8|Sports|82|ableeing|N/A|peru7415498857785287|green|Pound|Unknown|45|ableationbarpriought| +13073|AAAAAAAAABDDAAAA|1999-10-28|2001-10-26|Competitive beds read electrical, free teachers. National police would rewrite oddly new, subsequent factories. Heavy, nati|2.36|1.15|6002002|importocorp #2|2|diamonds|6|Jewelry|82|ableeing|N/A|545051836929thistle9|papaya|Carton|Unknown|73|priationbarpriought| +13074|AAAAAAAAABDDAAAA|2001-10-27||Competitive beds read electrical, free teachers. National police would rewrite oddly new, subsequent factories. Heavy, nati|4.95|1.78|6002002|amalgamalgamalg #2|11|disk drives|10|Electronics|82|ableeing|N/A|3715240326peru324337|sienna|Box|Unknown|7|eseationbarpriought| +13075|AAAAAAAADBDDAAAA|1997-10-27||Heavily only panels take sometimes exactly other colleagues|4.14|3.35|1004002|edu packamalg #2|4|swimwear|1|Women|7|ation|petite|0826140726rosy124152|rose|Dram|Unknown|78|antiationbarpriought| +13076|AAAAAAAAEBDDAAAA|1997-10-27|2000-10-26|Practical, alone schemes coul|0.25|0.08|6015005|scholarbrand #5|15|custom|6|Jewelry|611|oughtoughtcally|N/A|39309922white6893136|peach|Bunch|Unknown|28|callyationbarpriought| +13077|AAAAAAAAEBDDAAAA|2000-10-27||Practical, alone schemes coul|0.18|0.08|6015005|corpbrand #8|6|rugs|7|Home|418|oughtoughtcally|N/A|434806932807sienna27|tan|Gross|Unknown|44|ationationbarpriought| +13078|AAAAAAAAGBDDAAAA|1997-10-27|1999-10-27|Ways could not scratch ago|7.07|6.36|2001001|amalgimporto #1|1|accessories|2|Men|327|ationablepri|extra large|516230472791rosy3624|peach|Oz|Unknown|45|eingationbarpriought| +13079|AAAAAAAAGBDDAAAA|1999-10-28|2001-10-26|Very new costs might not arise completely strong, chris|3.64|6.36|4003002|exportiedu pack #2|1|kids|4|Shoes|327|ationablepri|large|516230472791rosy3624|pale|N/A|Unknown|61|n stationbarpriought| +13080|AAAAAAAAGBDDAAAA|2001-10-27||Very new costs might not arise completely strong, chris|7.39|6.36|4003002|amalgamalgamalg #8|11|disk drives|10|Electronics|327|ationablepri|N/A|516230472791rosy3624|turquoise|Oz|Unknown|97|bareingbarpriought| +13081|AAAAAAAAJBDDAAAA|1997-10-27||Free, small times begin. Never national conditions fix always well late paintings. More public chi|6.77|5.21|1002002|importoamalg #2|2|fragrances|1|Women|586|callyeinganti|medium|6295423380thistle099|smoke|Tbl|Unknown|17|oughteingbarpriought| +13082|AAAAAAAAKBDDAAAA|1997-10-27|2000-10-26|Enquiries must not miss daughters. Statements consider more by a feet. Examinations pay hotels. Modules |3.51|2.21|1003001|exportiamalg #1|3|maternity|1|Women|252|ableantiable|medium|015062278moccasin151|slate|Carton|Unknown|3|ableeingbarpriought| +13083|AAAAAAAAKBDDAAAA|2000-10-27||Great, english attacks used to proceed new animals. New, reasonable con|4.35|2.21|4003002|exportiedu pack #2|3|kids|4|Shoes|252|ableantiable|large|015062278moccasin151|gainsboro|Dozen|Unknown|98|prieingbarpriought| +13084|AAAAAAAAMBDDAAAA|1997-10-27|1999-10-27|Impossible authorities can look ready individuals. Lips may promote. Successful members challenge about in a children. Basic, normal weeks shall keep really important, permanent proceedings. Ri|0.76|0.50|9005005|scholarmaxi #5|5|history|9|Books|551|oughtantianti|N/A|14indian933837653398|pink|Unknown|Unknown|93|eseeingbarpriought| +13085|AAAAAAAAMBDDAAAA|1999-10-28|2001-10-26|Now creative mice may drop only. Days disclose easy m|1.03|0.70|10015013|scholaramalgamalg #13|5|portable|10|Electronics|551|oughtantianti|N/A|421lavender555672808|pink|N/A|Unknown|10|antieingbarpriought| +13086|AAAAAAAAMBDDAAAA|2001-10-27||Oppon|7.47|6.20|8016007|corpmaxi #7|5|golf|8|Sports|551|oughtantianti|N/A|993371840forest85556|tomato|Bundle|Unknown|5|callyeingbarpriought| +13087|AAAAAAAAPBDDAAAA|1997-10-27||Peaceful adults could attract also|4.69|3.70|8006008|corpnameless #8|6|football|8|Sports|605|antibarcally|N/A|turquoise86312338698|orange|Pallet|Unknown|8|ationeingbarpriought| +13088|AAAAAAAAACDDAAAA|1997-10-27|2000-10-26|Parents may affect perfect conten|0.98|0.39|9006009|corpmaxi #9|6|parenting|9|Books|662|ablecallycally|N/A|79207241108sandy2069|pale|N/A|Unknown|51|eingeingbarpriought| +13089|AAAAAAAAACDDAAAA|2000-10-27||Technical, full par|6.54|3.46|4003002|exportiedu pack #2|6|kids|4|Shoes|662|ablecallycally|large|860780papaya59911693|olive|Unknown|Unknown|15|n steingbarpriought| +13090|AAAAAAAACCDDAAAA|1997-10-27|1999-10-27|Equal grounds shal|8.36|3.34|3003001|exportiexporti #1|3|toddlers|3|Children|562|ablecallyanti|medium|6royal81930837170879|tan|N/A|Unknown|16|barn stbarpriought| +13091|AAAAAAAACCDDAAAA|1999-10-28|2001-10-26|Deaths make later|3.16|1.07|3003001|amalgscholar #2|1|rock|5|Music|785|antieingation|N/A|84chiffon41439025445|turquoise|N/A|Unknown|6|oughtn stbarpriought| +13092|AAAAAAAACCDDAAAA|2001-10-27||Only islands might advance by a genes. Fingers clear more schoo|1.67|1.07|3002001|importoexporti #1|2|infants|3|Children|73|priation|medium|84chiffon41439025445|snow|Each|Unknown|13|ablen stbarpriought| +13093|AAAAAAAAFCDDAAAA|1997-10-27||Extremely safe products make. Obvious lights lock flames. Discussions could n|7.54|3.39|8002004|importonameless #4|2|baseball|8|Sports|51|oughtanti|N/A|242lemon453188397312|royal|Lb|Unknown|21|prin stbarpriought| +13094|AAAAAAAAGCDDAAAA|1997-10-27|2000-10-26|Quiet requests lose correct, friendly men; perhaps subsequent powers would not trap. Major, volunt|3.59|1.57|8006005|corpnameless #5|6|football|8|Sports|447|ationeseese|N/A|940217medium83708125|violet|Bunch|Unknown|63|esen stbarpriought| +13095|AAAAAAAAGCDDAAAA|2000-10-27||Quiet requests lose correct, friendly men; perhaps subsequent powers would not trap. Major, volunt|7.27|3.41|6009008|maxicorp #8|6|womens watch|6|Jewelry|967|ationeseese|N/A|299921988azure442445|violet|Bunch|Unknown|64|antin stbarpriought| +13096|AAAAAAAAICDDAAAA|1997-10-27|1999-10-27|Different members go slightly in a patterns; small events can date obviously public, random cases. Independent feet act fully. Eager, gene|71.41|22.13|1003001|exportiamalg #1|3|maternity|1|Women|51|oughtanti|large|3444936plum835053035|powder|Bundle|Unknown|56|callyn stbarpriought| +13097|AAAAAAAAICDDAAAA|1999-10-28|2001-10-26|Different members go slightly in a patterns; small events can date obviously public, random cases. Independent feet act fully. Eager, gene|5.26|22.13|10001008|amalgunivamalg #8|1|cameras|10|Electronics|51|oughtanti|N/A|3444936plum835053035|purple|Pound|Unknown|6|ationn stbarpriought| +13098|AAAAAAAAICDDAAAA|2001-10-27||Different members go slightly in a patterns; small events can date obviously public, random cases. Independent feet act fully. Eager, gene|40.38|35.53|10001008|edu packexporti #1|1|school-uniforms|3|Children|323|oughtanti|large|3444936plum835053035|rosy|Dram|Unknown|60|eingn stbarpriought| +13099|AAAAAAAALCDDAAAA|1997-10-27||Other, old services violate yet for a schools. Casualties should reappear again by a females. Employees illustrate well never clean fields. Imperial, important appointments consider really orange, |8.46|3.38|7007010|brandbrand #10|7|decor|7|Home|216|callyoughtable|N/A|4432267271yellow4514|ivory|Case|Unknown|33|n stn stbarpriought| +13100|AAAAAAAAMCDDAAAA|1997-10-27|2000-10-26|Additional, comparable races blame never holders. Circumstances should describe important tenants. Else foreign terms might not suggest really speci|2.39|1.02|8001001|amalgnameless #1|1|athletic shoes|8|Sports|329|n stablepri|N/A|678066472889plum1811|sienna|Oz|Unknown|7|barbaroughtpriought| +13101|AAAAAAAAMCDDAAAA|2000-10-27||Additional, comparable races blame never holders. Circumstances should describe important tenants. Else foreign terms might not suggest really speci|96.96|1.02|8001001|scholarunivamalg #10|1|fiction|9|Books|170|n stablepri|N/A|877165464298snow4387|grey|Pound|Unknown|30|oughtbaroughtpriought| +13102|AAAAAAAAOCDDAAAA|1997-10-27|1999-10-27|Apart supreme teams shall see as a angles. Courses would not sell me|0.96|0.82|7008003|namelessbrand #3|8|lighting|7|Home|362|ablecallypri|N/A|7950544797pink864166|sky|Bunch|Unknown|34|ablebaroughtpriought| +13103|AAAAAAAAOCDDAAAA|1999-10-28|2001-10-26|Previous effects would help. Most victorian gains shall emerge rather lines. Forthcoming boxes re|9.53|0.82|7008003|importoedu pack #2|2|mens|4|Shoes|274|eseationable|extra large|8883866thistle542773|midnight|Ounce|Unknown|64|pribaroughtpriought| +13104|AAAAAAAAOCDDAAAA|2001-10-27||Previous effects would help. Most victorian gains shall emerge rather lines. Forthcoming boxes re|7.61|5.93|7008003|importoscholar #1|2|country|5|Music|274|eseationable|N/A|31389702104blanched6|pale|Gross|Unknown|12|esebaroughtpriought| +13105|AAAAAAAABDDDAAAA|1997-10-27||Scientific|1.25|1.05|7015006|scholarnameless #6|15|tables|7|Home|605|antibarcally|N/A|2412620spring9496941|rosy|Each|Unknown|52|antibaroughtpriought| +13106|AAAAAAAACDDDAAAA|1997-10-27|2000-10-26|Arrangem|7.07|6.29|1002001|importoamalg #1|2|fragrances|1|Women|171|oughtationought|extra large|7moccasin58270024378|puff|Each|Unknown|13|callybaroughtpriought| +13107|AAAAAAAACDDDAAAA|2000-10-27||Arrangem|8.75|3.76|1002001|importoedu pack #2|2|mens|4|Shoes|628|eingablecally|extra large|7moccasin58270024378|lavender|Ton|Unknown|29|ationbaroughtpriought| +13108|AAAAAAAAEDDDAAAA|1997-10-27|1999-10-27|Worthy facts cannot show various wounds. Very head roses say chronic days. Special developments might assess members. Char|0.25|0.20|3003001|exportiexporti #1|3|toddlers|3|Children|250|barantiable|petite|7628858488496smoke33|papaya|Case|Unknown|13|eingbaroughtpriought| +13109|AAAAAAAAEDDDAAAA|1999-10-28|2001-10-26|Able languages forgive more as a duties. Royal, primary others cannot demand much new tickets. Customs pr|2.77|0.20|7006010|corpbrand #10|6|rugs|7|Home|250|barantiable|N/A|5pink078645017163007|smoke|N/A|Unknown|2|n stbaroughtpriought| +13110|AAAAAAAAEDDDAAAA|2001-10-27||Conditions penetrate always secret areas. American terms will not increase actual residents. Homes take lo|0.49|0.28|7006010|edu packbrand #1|4|curtains/drapes|7|Home|250|barantiable|N/A|5pink078645017163007|white|Dozen|Unknown|5|baroughtoughtpriought| +13111|AAAAAAAAHDDDAAAA|1997-10-27||Very questions will not come changes. Famous things used to go very personal muscles. Marvellous methods shall ask so large, twin citizens; purposes kill so. Rough tears used to concentrate in |8.39|3.43|9005008|scholarmaxi #8|5|history|9|Books|158|eingantiought|N/A|13318navajo163012527|rosy|N/A|Unknown|17|oughtoughtoughtpriought| +13112|AAAAAAAAIDDDAAAA|1997-10-27|2000-10-26|Wide, late categorie|0.99|0.70|5004001|edu packscholar #1|4|classical|5|Music|449|n steseese|N/A|04magenta74462277806|ghost|Box|Unknown|10|ableoughtoughtpriought| +13113|AAAAAAAAIDDDAAAA|2000-10-27||Quite true collections feel completely in the rules; ideas shall not take within the soldiers; o|9.36|3.46|5004001|univmaxi #4|4|pools|8|Sports|380|n steseese|N/A|4506smoke68391261112|sandy|Ton|Unknown|89|prioughtoughtpriought| +13114|AAAAAAAAKDDDAAAA|1997-10-27|1999-10-27|Adequately unemployed aspects ought to keep on a years. Years get somewhere sometimes late examples; laws must shape determined stones. Recently real decisions may cost now other female thousands. |2.80|2.12|10015008|scholaramalgamalg #8|15|portable|10|Electronics|76|callyation|N/A|violet12221679858281|papaya|Case|Unknown|58|eseoughtoughtpriought| +13115|AAAAAAAAKDDDAAAA|1999-10-28|2001-10-26|Cars follow now broad citizens. New companies may not select now other than a holidays. Safe, human eyes may tackle surprising months; once spiritual mana|0.18|2.12|10015008|amalgimporto #2|15|accessories|2|Men|76|callyation|small|916433sienna27698748|indian|Ton|Unknown|45|antioughtoughtpriought| +13116|AAAAAAAAKDDDAAAA|2001-10-27||Cars follow now broad citizens. New companies may not select now other than a holidays. Safe, human eyes may tackle surprising months; once spiritual mana|1.40|0.42|6016005|corpbrand #5|16|consignment|6|Jewelry|452|callyation|N/A|916433sienna27698748|wheat|Oz|Unknown|48|callyoughtoughtpriought| +13117|AAAAAAAANDDDAAAA|1997-10-27||Private, extreme books will for|0.74|0.28|8016008|corpmaxi #8|16|golf|8|Sports|86|callyeing|N/A|5156017903puff704730|papaya|Unknown|Unknown|2|ationoughtoughtpriought| +13118|AAAAAAAAODDDAAAA|1997-10-27|2000-10-26|Ni|0.83|0.63|7016009|corpnameless #9|16|furniture|7|Home|15|antiought|N/A|47turquoise984228962|tomato|Gram|Unknown|45|eingoughtoughtpriought| +13119|AAAAAAAAODDDAAAA|2000-10-27||Shares may not assume very. General, black things supply for instance economic rules. As religious years control yet on a months. |0.97|0.64|7016009|amalgimporto #2|16|accessories|2|Men|793|prin station|extra large|983005718steel006621|smoke|Ounce|Unknown|8|n stoughtoughtpriought| +13120|AAAAAAAAAEDDAAAA|1997-10-27|1999-10-27|Quietly expert courses attack games. Pale ideas know once up to the systems. Finally old projects pay again russians. Matters used to work |4.12|1.23|3004001|edu packexporti #1|4|school-uniforms|3|Children|83|prieing|petite|3513269616798orchid7|pale|Box|Unknown|35|barableoughtpriought| +13121|AAAAAAAAAEDDAAAA|1999-10-28|2001-10-26|Quietly expert courses attack games. Pale ideas know once up to the systems. Finally old projects pay again russians. Matters used to work |45.35|24.94|6012002|importobrand #2|12|costume|6|Jewelry|168|eingcallyought|N/A|626239942503orange07|powder|Gram|Unknown|27|oughtableoughtpriought| +13122|AAAAAAAAAEDDAAAA|2001-10-27||Quietly expert courses attack games. Pale ideas know once up to the systems. Finally old projects pay again russians. Matters used to work |6.83|24.94|6012002|importobrand #3|12|costume|6|Jewelry|168|eingcallyought|N/A|491490829564sienna40|pink|Gram|Unknown|82|ableableoughtpriought| +13123|AAAAAAAADEDDAAAA|1997-10-27||Original, everyday words may not wish even to a paintings. Domestic movements could explore on a improvements. For example specialist contracts use as more subtle weekends. Annual, good performanc|5.19|3.58|8007004|brandnameless #4|7|hockey|8|Sports|469|n stcallyese|N/A|38395513105339sandy7|plum|Unknown|Unknown|9|priableoughtpriought| +13124|AAAAAAAAEEDDAAAA|1997-10-27|2000-10-26|Particular, basic members used to get angles. Lar|6.55|3.60|10005001|scholarunivamalg #1|5|karoke|10|Electronics|65|antically|N/A|5316green06439148876|pale|Dram|Unknown|23|eseableoughtpriought| +13125|AAAAAAAAEEDDAAAA|2000-10-27||So british boys arrest offices. Ready, new authorities might progress thus now sure lig|6.20|3.60|10005001|edu packunivamalg #10|5|sports|9|Books|65|antically|N/A|5316green06439148876|peach|Gross|Unknown|57|antiableoughtpriought| +13126|AAAAAAAAGEDDAAAA|1997-10-27|1999-10-27|Available workshops might direct directly. Conditions must satisfy also upper reactions. Sufficient words must see young considerations. Terrible, only expres|8.24|6.42|9011011|amalgunivamalg #11|11|cooking|9|Books|751|oughtantiation|N/A|6875ghost44283312260|chiffon|Pallet|Unknown|7|callyableoughtpriought| +13127|AAAAAAAAGEDDAAAA|1999-10-28|2001-10-26|Available workshops might direct directly. Conditions must satisfy also upper reactions. Sufficient words must see young considerations. Terrible, only expres|7.77|6.42|8011008|amalgmaxi #8|11|archery|8|Sports|751|oughtantiation|N/A|6875ghost44283312260|thistle|Unknown|Unknown|14|ationableoughtpriought| +13128|AAAAAAAAGEDDAAAA|2001-10-27||Never independent beliefs conflict impressive, able emotions. Probably american pubs should not go forces; specific, entire cases need again. Terms amount everywhere partly tough offences|4.93|3.45|4001001|amalgedu pack #1|11|womens|4|Shoes|197|ationn stought|medium|6875ghost44283312260|brown|Gross|Unknown|34|eingableoughtpriought| +13129|AAAAAAAAJEDDAAAA|1997-10-27||Male m|7.07|2.33|4001002|amalgedu pack #2|1|womens|4|Shoes|583|prieinganti|large|moccasin864894805511|papaya|Ton|Unknown|36|n stableoughtpriought| +13130|AAAAAAAAKEDDAAAA|1997-10-27|2000-10-26|Else single shoulders let still. Then blue employees invite to a blocks. Old numbers want aside. Still other industries could make free nations|4.31|3.27|1004001|edu packamalg #1|4|swimwear|1|Women|247|ationeseable|medium|587301469blush520365|saddle|Gram|Unknown|74|barprioughtpriought| +13131|AAAAAAAAKEDDAAAA|2000-10-27||Different, national sho|0.99|0.31|1004001|edu packamalg #2|4|swimwear|1|Women|247|ationeseable|medium|587301469blush520365|salmon|Gram|Unknown|13|oughtprioughtpriought| +13132|AAAAAAAAMEDDAAAA|1997-10-27|1999-10-27|Considerable years pick still in a weapons; other, original companies would impose almost. Social windows should not conquer general rules; open, sudden problems become|0.20|0.13|3001001|amalgexporti #1|1|newborn|3|Children|188|eingeingought|economy|2470866510snow884841|lime|Tbl|Unknown|88|ableprioughtpriought| +13133|AAAAAAAAMEDDAAAA|1999-10-28|2001-10-26|Scottish contacts wipe details. American, supreme facilities leave still bad boys. Somehow private shoulders ask |3.40|0.13|3001001|amalgedu pack #2|1|womens|4|Shoes|189|n steingought|petite|8226698671lace785556|drab|Dram|Unknown|29|priprioughtpriought| +13134|AAAAAAAAMEDDAAAA|2001-10-27||Scottish contacts wipe details. American, supreme facilities leave still bad boys. Somehow private shoulders ask |1.65|0.13|3001001|amalgunivamalg #7|1|cooking|9|Books|189|n steingought|N/A|8226698671lace785556|violet|Dozen|Unknown|3|eseprioughtpriought| +13135|AAAAAAAAPEDDAAAA|1997-10-27||Influential, major levels like. Secondary divisions may give factories. There little|1.96|0.64|9012008|importounivamalg #8|12|home repair|9|Books|96|callyn st|N/A|68steel4470201521249|thistle|Dozen|Unknown|1|antiprioughtpriought| +13136|AAAAAAAAAFDDAAAA|1997-10-27|2000-10-26|Other hospitals pursue. Central passengers may install already priests. Good details can see now other, annual orders. Various, clear es|3.87|2.90|4004001|edu packedu pack #1|4|athletic|4|Shoes|709|n stbaration|medium|82376018papaya787176|peru|Oz|Unknown|1|callyprioughtpriought| +13137|AAAAAAAAAFDDAAAA|2000-10-27||Terrible interests will not work far unequivocally perfect parties. Long buildings break closer. Quite intermediate duties must settle also far soft plans. Potent|1.79|1.02|8010008|univmaxi #8|4|pools|8|Sports|709|n stbaration|N/A|82376018papaya787176|puff|Dram|Unknown|23|ationprioughtpriought| +13138|AAAAAAAACFDDAAAA|1997-10-27|1999-10-27|Right, bottom levels might find drastically into a arms; f|5.98|4.36|1002001|importoamalg #1|2|fragrances|1|Women|850|barantieing|large|332490thistle3567231|peach|Gram|Unknown|49|eingprioughtpriought| +13139|AAAAAAAACFDDAAAA|1999-10-28|2001-10-26|Right, bottom levels might find drastically into a arms; f|3.62|4.36|7003010|exportibrand #10|3|kids|7|Home|132|barantieing|N/A|332490thistle3567231|drab|Pallet|Unknown|32|n stprioughtpriought| +13140|AAAAAAAACFDDAAAA|2001-10-27||Right, bottom levels might find drastically into a arms; f|46.64|4.36|7003010|exportinameless #1|13|wallpaper|7|Home|132|barantieing|N/A|332490thistle3567231|sandy|Cup|Unknown|16|bareseoughtpriought| +13141|AAAAAAAAFFDDAAAA|1997-10-27||French, civil hours must report essential values. Reasonable, complete judges vary clearly homes; often pleasant women would watch. Poor,|2.79|1.92|9016002|corpunivamalg #2|16|mystery|9|Books|761|oughtcallyation|N/A|71726567009tan075612|salmon|Oz|Unknown|74|oughteseoughtpriought| +13142|AAAAAAAAGFDDAAAA|1997-10-27|2000-10-26|Average members should put|64.59|40.69|3002001|importoexporti #1|2|infants|3|Children|357|ationantipri|medium|712434yellow58630972|red|Lb|Unknown|33|ableeseoughtpriought| +13143|AAAAAAAAGFDDAAAA|2000-10-27||Super, possible drugs light past villages. M|4.69|3.32|3002001|namelessmaxi #4|8|romance|9|Books|357|ationantipri|N/A|119771176156lawn1083|sandy|N/A|Unknown|43|prieseoughtpriought| +13144|AAAAAAAAIFDDAAAA|1997-10-27|1999-10-27|Key types might not read again only needs. Schools throw rules; well general differences might appear interior, whole illustrations. Major men look always black, superb wounds|0.87|0.60|10010004|univamalgamalg #4|10|memory|10|Electronics|840|bareseeing|N/A|0399forest5689593964|powder|Carton|Unknown|4|eseeseoughtpriought| +13145|AAAAAAAAIFDDAAAA|1999-10-28|2001-10-26|Important, bad policies might not put only qualities. Nearly dangerous systems shall not|2.92|1.92|3001002|amalgexporti #2|1|newborn|3|Children|840|bareseeing|petite|0399forest5689593964|chartreuse|Cup|Unknown|31|antieseoughtpriought| +13146|AAAAAAAAIFDDAAAA|2001-10-27||Local duties ought to leave beforehand; possible, combined bodies obtain again industries. Separate schools allow still ugly flames. Particularly natural rights affect only alway|4.58|1.92|3001002|edu packunivamalg #6|1|audio|10|Electronics|286|callyeingable|N/A|3sky3116277814364695|yellow|Dozen|Unknown|96|callyeseoughtpriought| +13147|AAAAAAAALFDDAAAA|1997-10-27||Colleges hold at the farmers. Days may not win ideally quick readers. Blank walls drop sometimes. True, friendly criticisms would taste very for a friends. Busy bodies |6.99|2.44|5001002|amalgscholar #2|1|rock|5|Music|438|eingpriese|N/A|807854247003papaya01|steel|Tbl|Unknown|44|ationeseoughtpriought| +13148|AAAAAAAAMFDDAAAA|1997-10-27|2000-10-26|Se|3.79|2.69|6010005|univbrand #5|10|jewelry boxes|6|Jewelry|309|n stbarpri|N/A|tan44118922897687236|violet|Bundle|Unknown|63|eingeseoughtpriought| +13149|AAAAAAAAMFDDAAAA|2000-10-27||Se|68.08|2.69|6010005|namelessunivamalg #9|8|scanners|10|Electronics|774|eseationation|N/A|539881turquoise84672|purple|Gross|Unknown|44|n steseoughtpriought| +13150|AAAAAAAAOFDDAAAA|1997-10-27|1999-10-27|Minerals place also nicely other teeth. Trees would not b|3.93|2.39|2001001|amalgimporto #1|1|accessories|2|Men|630|barprically|large|5lemon22440420142836|purple|Case|Unknown|2|barantioughtpriought| +13151|AAAAAAAAOFDDAAAA|1999-10-28|2001-10-26|Patterns swear altogether mathematical magazines. General words ma|8.60|6.10|2001001|exportiimporto #2|1|pants|2|Men|47|ationese|medium|olive966914057477175|turquoise|Gram|Unknown|51|oughtantioughtpriought| +13152|AAAAAAAAOFDDAAAA|2001-10-27||Liberal, other tools prefer only active engines. Then whit|2.35|1.41|5004001|edu packscholar #1|4|classical|5|Music|76|ationese|N/A|olive966914057477175|hot|Oz|Unknown|20|ableantioughtpriought| +13153|AAAAAAAABGDDAAAA|1997-10-27||About commercial men must think near twins. Explanations know expected standards. Social values w|9.30|7.53|10009013|maxiunivamalg #13|9|televisions|10|Electronics|254|eseantiable|N/A|tan30482156741442872|seashell|Gram|Unknown|43|priantioughtpriought| +13154|AAAAAAAACGDDAAAA|1997-10-27|2000-10-26|Always regular rules used to keep finally. Small phenomena shall disturb thereby. Well late schools may afford increasingly e|7.31|5.19|7014005|edu packnameless #5|14|glassware|7|Home|585|antieinganti|N/A|604547043violet73853|navy|Unknown|Unknown|66|eseantioughtpriought| +13155|AAAAAAAACGDDAAAA|2000-10-27||Always regular rules used to keep finally. Small phenomena shall disturb thereby. Well late schools may afford increasingly e|3.18|5.19|10010015|univamalgamalg #15|10|memory|10|Electronics|585|antieinganti|N/A|604547043violet73853|blue|Tbl|Unknown|48|antiantioughtpriought| +13156|AAAAAAAAEGDDAAAA|1997-10-27|1999-10-27|Just personal gardens love other services. Catholic years judge so. Other, other eyes improve seriously |0.74|0.49|7009007|maxibrand #7|9|mattresses|7|Home|44|eseese|N/A|027413678lavender612|wheat|Dram|Unknown|31|callyantioughtpriought| +13157|AAAAAAAAEGDDAAAA|1999-10-28|2001-10-26|Just personal gardens love other services. Catholic years judge so. Other, other eyes improve seriously |9.06|3.89|4001002|amalgedu pack #2|1|womens|4|Shoes|44|eseese|large|72turquoise376741608|slate|Oz|Unknown|2|ationantioughtpriought| +13158|AAAAAAAAEGDDAAAA|2001-10-27||Just personal gardens love other services. Catholic years judge so. Other, other eyes improve seriously |2.19|1.46|10013001|exportiamalgamalg #1|13|stereo|10|Electronics|44|eseese|N/A|889seashell995360488|medium|Pound|Unknown|23|eingantioughtpriought| +13159|AAAAAAAAHGDDAAAA|1997-10-27||Terms used to comprehend to a things. Really busy competitors stop women. Normally certain libraries remain considerably from a centres. Glad countries cannot try together groups. There powerful|4.30|2.79|7002010|importobrand #10|2|bedding|7|Home|107|ationbarought|N/A|01472464plum90938867|peru|Unknown|Unknown|67|n stantioughtpriought| +13160|AAAAAAAAIGDDAAAA|1997-10-27|2000-10-26|Workers make into a members. Ugly goals drive just british, constant instructions. Long different hospitals p|2.56|1.30|3004001|edu packexporti #1|4|school-uniforms|3|Children|45|antiese|large|5517776884purple0144|magenta|Case|Unknown|15|barcallyoughtpriought| +13161|AAAAAAAAIGDDAAAA|2000-10-27||Important things used to obey turkish, high walls; fans ought to work also spiritual, average schools; features realise most new bodies. Ever leading jobs repair f|4.88|3.85|3004001|amalgmaxi #2|11|archery|8|Sports|3|pri|N/A|5517776884purple0144|thistle|Gross|Unknown|13|oughtcallyoughtpriought| +13162|AAAAAAAAKGDDAAAA|1997-10-27|1999-10-27|Good bottles used to train in a cuts. Both prime years play finally games. Circumstances could not act alone other wings. Ways would not help relatively sales. Heels may not co|8.35|4.67|2003001|exportiimporto #1|3|pants|2|Men|62|ablecally|petite|7356indian4998828354|peru|Tsp|Unknown|31|ablecallyoughtpriought| +13163|AAAAAAAAKGDDAAAA|1999-10-28|2001-10-26|Good bottles used to train in a cuts. Both prime years play finally games. Circumstances could not act alone other wings. Ways would not help relatively sales. Heels may not co|4.36|4.67|2003001|edu packimporto #2|4|sports-apparel|2|Men|62|ablecally|medium|7356indian4998828354|steel|Ton|Unknown|76|pricallyoughtpriought| +13164|AAAAAAAAKGDDAAAA|2001-10-27||Good bottles used to train in a cuts. Both prime years play finally games. Circumstances could not act alone other wings. Ways would not help relatively sales. Heels may not co|0.40|4.67|1001001|amalgamalg #1|4|dresses|1|Women|370|ablecally|petite|7356indian4998828354|red|Bunch|Unknown|23|esecallyoughtpriought| +13165|AAAAAAAANGDDAAAA|1997-10-27||Perfect, double funds could not secure as in the dangers. Workers resist. Ex|2.45|0.93|10008008|namelessunivamalg #8|8|scanners|10|Electronics|375|antiationpri|N/A|39485940985rosy77671|khaki|Carton|Unknown|21|anticallyoughtpriought| +13166|AAAAAAAAOGDDAAAA|1997-10-27|2000-10-26|Open blue farmers reach useful, old arrangements. American, short years reach now tender, heavy neighbours. Now top boundaries would not enable emotions. Effectively specific |2.34|2.01|7013007|exportinameless #7|13|wallpaper|7|Home|611|oughtoughtcally|N/A|3381172546323salmon4|tomato|Carton|Unknown|1|callycallyoughtpriought| +13167|AAAAAAAAOGDDAAAA|2000-10-27||Open blue farmers reach useful, old arrangements. American, short years reach now tender, heavy neighbours. Now top boundaries would not enable emotions. Effectively specific |3.79|3.03|7013007|importocorp #8|13|diamonds|6|Jewelry|239|n stpriable|N/A|3381172546323salmon4|wheat|Gram|Unknown|13|ationcallyoughtpriought| +13168|AAAAAAAAAHDDAAAA|1997-10-27|1999-10-27|National, responsible words ask then damp, central items. Popular, environmental women see organs. Int|8.71|6.00|10001011|amalgunivamalg #11|1|cameras|10|Electronics|94|esen st|N/A|73553546bisque986330|antique|Pallet|Unknown|46|eingcallyoughtpriought| +13169|AAAAAAAAAHDDAAAA|1999-10-28|2001-10-26|Human|28.41|20.73|10001011|exportinameless #10|1|wallpaper|7|Home|94|esen st|N/A|73553546bisque986330|red|Unknown|Unknown|63|n stcallyoughtpriought| +13170|AAAAAAAAAHDDAAAA|2001-10-27||Human|4.32|20.73|10001011|exportiedu pack #1|3|kids|4|Shoes|94|esen st|extra large|493074981wheat754880|snow|Each|Unknown|15|barationoughtpriought| +13171|AAAAAAAADHDDAAAA|1997-10-27||Abroad sorry ways must make home satisfied, fat effec|2.87|2.09|4004002|edu packedu pack #2|4|athletic|4|Shoes|76|callyation|large|59501993765thistle60|tomato|Tsp|Unknown|22|oughtationoughtpriought| +13172|AAAAAAAAEHDDAAAA|1997-10-27|2000-10-26|Electronic, possible details would take images. Ends used to confide straight necessary, cold organisms; items buy encouraging, married names; british, new teachers know then re|1.69|0.54|10003015|exportiunivamalg #15|3|dvd/vcr players|10|Electronics|150|barantiought|N/A|01wheat7911376621565|violet|Dozen|Unknown|51|ableationoughtpriought| +13173|AAAAAAAAEHDDAAAA|2000-10-27||Electronic, possible details would take images. Ends used to confide straight necessary, cold organisms; items buy encouraging, married names; british, new teachers know then re|6.46|0.54|2001002|amalgimporto #2|3|accessories|2|Men|244|barantiought|medium|01wheat7911376621565|smoke|Pallet|Unknown|2|priationoughtpriought| +13174|AAAAAAAAGHDDAAAA|1997-10-27|1999-10-27|Endless, interested eyes can unde|5.12|2.86|7009005|maxibrand #5|9|mattresses|7|Home|476|callyationese|N/A|2061puff813459876826|rose|Dram|Unknown|23|eseationoughtpriought| +13175|AAAAAAAAGHDDAAAA|1999-10-28|2001-10-26|Patients should not give still, domestic teachers. Good results would prove; japanese, necessary gr|1.94|2.86|7009005|brandmaxi #12|7|reference|9|Books|163|pricallyought|N/A|2061puff813459876826|steel|Gross|Unknown|12|antiationoughtpriought| +13176|AAAAAAAAGHDDAAAA|2001-10-27||Signs put only|9.88|2.86|7009005|exportimaxi #1|3|computers|9|Books|163|pricallyought|N/A|886837302powder96073|tan|Case|Unknown|38|callyationoughtpriought| +13177|AAAAAAAAJHDDAAAA|1997-10-27||Once alive cases ought to explain minutes; hopes come less than. Similar feet will feel then; large residents appreciate areas. Small, low boys negoti|6.44|3.86|3004002|edu packexporti #2|4|school-uniforms|3|Children|25|antiable|petite|3305slate87143637052|peru|Tsp|Unknown|9|ationationoughtpriought| +13178|AAAAAAAAKHDDAAAA|1997-10-27|2000-10-26|Entire, related interests obtain typically. Narrow patients |8.72|5.05|4001001|amalgedu pack #1|1|womens|4|Shoes|4|ese|small|pale9424583162499500|plum|Pallet|Unknown|6|eingationoughtpriought| +13179|AAAAAAAAKHDDAAAA|||Entire, related interests obtain typically. Narrow patients ||25.65|4001001||||5|Music|295|ese|N/A|pale9424583162499500|peach|||24|n stationoughtpriought| +13180|AAAAAAAAMHDDAAAA|1997-10-27|1999-10-27|Goods go further recent words. Special, specific rights used to challenge then. Tomorrow concerned musicians must not lend from a shelves. Once|9.65|2.99|7014001|edu packnameless #1|14|glassware|7|Home|409|n stbarese|N/A|65474chiffon60519086|orchid|N/A|Unknown|94|bareingoughtpriought| +13181|AAAAAAAAMHDDAAAA|1999-10-28|2001-10-26|Goods go further recent words. Special, specific rights used to challenge then. Tomorrow concerned musicians must not lend from a shelves. Once|3.00|2.99|4001002|amalgedu pack #2|1|womens|4|Shoes|976|callyationn st|large|736210379midnight820|powder|N/A|Unknown|17|oughteingoughtpriought| +13182|AAAAAAAAMHDDAAAA|2001-10-27||Early, wooden words could build again; as so-called t|7.93|2.99|6010007|univbrand #7|10|jewelry boxes|6|Jewelry|976|callyationn st|N/A|736210379midnight820|dark|Unknown|Unknown|12|ableeingoughtpriought| +13183|AAAAAAAAPHDDAAAA|1997-10-27||Individual flowers used to give thanks. Particular doubts refer a bit for a directions. Police could|1.74|1.21|2003002|exportiimporto #2|3|pants|2|Men|193|prin stought|medium|0904832moccasin39528|rosy|Dram|Unknown|15|prieingoughtpriought| +13184|AAAAAAAAAIDDAAAA|1997-10-27|2000-10-26|Environmental, young qualities seek almost policies. Therefore low ears recover political, specific fortunes. Houses ought to promote with a men. Then bad costs answer always with a arms. Users |0.36|0.21|6014001|edu packbrand #1|14|estate|6|Jewelry|834|eseprieing|N/A|3929716442sienna6019|lawn|Lb|Unknown|52|eseeingoughtpriought| +13185|AAAAAAAAAIDDAAAA|2000-10-27||Environmental, young qualities seek almost policies. Therefore low ears recover political, specific fortunes. Houses ought to promote with a men. Then bad costs answer always with a arms. Users |6.28|0.21|10003011|exportiunivamalg #11|14|dvd/vcr players|10|Electronics|167|eseprieing|N/A|3929716442sienna6019|salmon|Lb|Unknown|48|antieingoughtpriought| +13186|AAAAAAAACIDDAAAA|1997-10-27|1999-10-27|Fundamental hopes start civil implications. Educational |2.58|1.75|3002001|importoexporti #1|2|infants|3|Children|998|eingn stn st|medium|0488647853seashell25|spring|Box|Unknown|60|callyeingoughtpriought| +13187|AAAAAAAACIDDAAAA|1999-10-28|2001-10-26|Fundamental hopes start civil implications. Educational |4.14|1.75|1002002|importoamalg #2|2|fragrances|1|Women|998|eingn stn st|small|0488647853seashell25|powder|Ounce|Unknown|95|ationeingoughtpriought| +13188|AAAAAAAACIDDAAAA|2001-10-27||Fundamental hopes start civil implications. Educational |3.97|1.75|1002002|corpmaxi #1|2|parenting|9|Books|998|eingn stn st|N/A|64057593navy19887674|frosted|Gram|Unknown|30|eingeingoughtpriought| +13189|AAAAAAAAFIDDAAAA|1997-10-27||Orange, upper patients buy that is circumstances. Proper, necessary cells must tell complete tenants. Othe|3.30|2.17|5002002|importoscholar #2|2|country|5|Music|268|eingcallyable|N/A|2131887650medium2301|blanched|Carton|Unknown|4|n steingoughtpriought| +13190|AAAAAAAAGIDDAAAA|1997-10-27|2000-10-26|Cases pass cases. Less great struc|9.14|3.65|5002001|importoscholar #1|2|country|5|Music|349|n stesepri|N/A|630367spring86762674|spring|Dozen|Unknown|58|barn stoughtpriought| +13191|AAAAAAAAGIDDAAAA|2000-10-27||Cases pass cases. Less great struc|1.41|0.57|5002001|edu packexporti #2|4|school-uniforms|3|Children|349|n stesepri|economy|630367spring86762674|maroon|Oz|Unknown|13|oughtn stoughtpriought| +13192|AAAAAAAAIIDDAAAA|1997-10-27|1999-10-27|Good, open members come however on a pupils. Manufacturers work sharply remote writings. Negotiations set however ever attractive servan|3.61|1.73|5004001|edu packscholar #1|4|classical|5|Music|78|eingation|N/A|40860papaya738863857|sky|Cup|Unknown|15|ablen stoughtpriought| +13193|AAAAAAAAIIDDAAAA|1999-10-28|2001-10-26|Parents reduce always. Patterns used to investigate basically comparisons. Current, massive lines detect hurriedly active, separate committees. |0.82|1.73|2002002|importoimporto #2|2|shirts|2|Men|78|eingation|large|40860papaya738863857|rosy|Tbl|Unknown|15|prin stoughtpriought| +13194|AAAAAAAAIIDDAAAA|2001-10-27||Parents reduce always. Patterns used to investigate basically comparisons. Current, massive lines detect hurriedly active, separate committees. |5.66|4.75|2002002|edu packedu pack #1|4|athletic|4|Shoes|320|barablepri|medium|20327693rose79893397|papaya|Box|Unknown|46|esen stoughtpriought| +13195|AAAAAAAALIDDAAAA|1997-10-27||Frequent services |2.50|2.25|6006004|corpcorp #4|6|rings|6|Jewelry|256|callyantiable|N/A|42298119lemon9961234|gainsboro|Unknown|Unknown|33|antin stoughtpriought| +13196|AAAAAAAAMIDDAAAA|1997-10-27|2000-10-26|Difficult models used to follow key, different factors. Other levels should find just specific, india|1.48|1.31|3002001|importoexporti #1|2|infants|3|Children|203|pribarable|large|3790857orange1399968|steel|N/A|Unknown|15|callyn stoughtpriought| +13197|AAAAAAAAMIDDAAAA|2000-10-27||Difficult models used to follow key, different factors. Other levels should find just specific, india|8.67|1.31|4003002|exportiedu pack #2|3|kids|4|Shoes|203|pribarable|extra large|306503419291611rose3|goldenrod|Dram|Unknown|24|ationn stoughtpriought| +13198|AAAAAAAAOIDDAAAA|1997-10-27|1999-10-27|Royal, annual companies load today public signs; too considerable offices treat on a times. Strong, reaso|9.27|5.28|10002007|importounivamalg #7|2|camcorders|10|Electronics|702|ablebaration|N/A|2829tan3813861527831|powder|Each|Unknown|18|eingn stoughtpriought| +13199|AAAAAAAAOIDDAAAA|1999-10-28|2001-10-26|Royal, annual companies load today public signs; too considerable offices treat on a times. Strong, reaso|9.81|5.28|10002008|importounivamalg #8|2|camcorders|10|Electronics|378|eingationpri|N/A|97079847851red623447|lavender|Pallet|Unknown|13|n stn stoughtpriought| +13200|AAAAAAAAOIDDAAAA|2001-10-27||Birds seek even odd, close friends. Subsequent lives grow years. Very temporary systems locate. Problems blow accordingly other eyes; silent pupils should not|2.92|2.27|3001001|amalgexporti #1|1|newborn|3|Children|378|eingationpri|medium|5682salmon8451441304|coral|Tsp|Unknown|19|barbarablepriought| +13201|AAAAAAAABJDDAAAA|1997-10-27||Plain simple produc|0.62|0.37|5004002|edu packscholar #2|4|classical|5|Music|320|barablepri|N/A|4156931660peach01725|pale|Bunch|Unknown|35|oughtbarablepriought| +13202|AAAAAAAACJDDAAAA|1997-10-27|2000-10-26|Hu|2.63|1.86|3001001|amalgexporti #1|1|newborn|3|Children|457|ationantiese|petite|lavender790768740884|tomato|Tbl|Unknown|80|ablebarablepriought| +13203|AAAAAAAACJDDAAAA|2000-10-27||Profound countries could break only tough gentlemen. Clients withdraw lights. Merchants survive much near a ideas. Origins call casually terms. Books think |5.20|4.10|5004002|edu packscholar #2|4|classical|5|Music|457|ationantiese|N/A|7582plum079627284872|orchid|Bunch|Unknown|7|pribarablepriought| +13204|AAAAAAAAEJDDAAAA|1997-10-27|1999-10-27|Unknown minutes must not override especially significant flowers. Northern problems mean on the objections. Words mean later econo|2.50|1.20|9012005|importounivamalg #5|12|home repair|9|Books|21|oughtable|N/A|8790815steel11971582|light|Dram|Unknown|20|esebarablepriought| +13205|AAAAAAAAEJDDAAAA|1999-10-28|2001-10-26|Unknown minutes must not override especially significant flowers. Northern problems mean on the objections. Words mean later econo|1.26|0.37|1004002|edu packamalg #2|4|swimwear|1|Women|21|oughtable|petite|5white64604610969302|cyan|Case|Unknown|49|antibarablepriought| +13206|AAAAAAAAEJDDAAAA|2001-10-27||Unknown minutes must not override especially significant flowers. Northern problems mean on the objections. Words mean later econo|3.90|2.22|1004002|univunivamalg #7|10|travel|9|Books|434|esepriese|N/A|7237753seashell30467|thistle|Oz|Unknown|27|callybarablepriought| +13207|AAAAAAAAHJDDAAAA|1997-10-27||New, old cases would not find there high eggs; left countries see. Other stairs look once common, |1.89|0.86|6011008|amalgbrand #8|11|semi-precious|6|Jewelry|281|oughteingable|N/A|53lawn15326498454095|maroon|Cup|Unknown|1|ationbarablepriought| +13208|AAAAAAAAIJDDAAAA|1997-10-27|2000-10-26|Fair resources used to sense there women. Ideas might|6.97|5.15|10007017|brandunivamalg #17|7|personal|10|Electronics|874|eseationeing|N/A|turquoise70955717245|navajo|Gross|Unknown|22|eingbarablepriought| +13209|AAAAAAAAIJDDAAAA|2000-10-27||Fair resources used to sense there women. Ideas might|8.77|6.92|10007017|amalgscholar #2|7|rock|5|Music|761|eseationeing|N/A|1964658571snow866955|rose|N/A|Unknown|16|n stbarablepriought| +13210|AAAAAAAAKJDDAAAA|1997-10-27|1999-10-27|Universities ask low teachers. Particular problems require rather only miles. Forward eyes can remedy for the decades. Cars agree. Less social processes supply inside little, able wishes. Already |3.67|2.82|5001001|amalgscholar #1|1|rock|5|Music|2|able|N/A|38958359turquoise343|sandy|Bundle|Unknown|1|baroughtablepriought| +13211|AAAAAAAAKJDDAAAA|1999-10-28|2001-10-26|Universities ask low teachers. Particular problems require rather only miles. Forward eyes can remedy for the decades. Cars agree. Less social processes supply inside little, able wishes. Already |6.32|2.82|2001002|amalgimporto #2|1|accessories|2|Men|111|able|large|8278purple4037889751|honeydew|N/A|Unknown|37|oughtoughtablepriought| +13212|AAAAAAAAKJDDAAAA|2001-10-27||Universities ask low teachers. Particular problems require rather only miles. Forward eyes can remedy for the decades. Cars agree. Less social processes supply inside little, able wishes. Already |2.78|0.97|5003001|exportischolar #1|3|pop|5|Music|115|antioughtought|N/A|0312snow733888025910|plum|Lb|Unknown|27|ableoughtablepriought| +13213|AAAAAAAANJDDAAAA|1997-10-27||Activities give quite under soft studies. Less main contracts will use. Remarkably small rumours could make men. In|9.97|6.67|10009005|maxiunivamalg #5|9|televisions|10|Electronics|872|ableationeing|N/A|42991567salmon253807|spring|Pallet|Unknown|86|prioughtablepriought| +13214|AAAAAAAAOJDDAAAA|1997-10-27|2000-10-26|Years adopt well musical eyes. Future contents insist in private firm, clinical holders.|3.24|1.29|7012005|importonameless #5|12|paint|7|Home|26|callyable|N/A|8825863260pale501291|olive|Carton|Unknown|65|eseoughtablepriought| +13215|AAAAAAAAOJDDAAAA|2000-10-27||Years adopt well musical eyes. Future contents insist in private firm, clinical holders.|4.20|1.29|7012006|importonameless #6|12|paint|7|Home|322|ableablepri|N/A|5335smoke53547542082|tan|Case|Unknown|32|antioughtablepriought| +13216|AAAAAAAAAKDDAAAA|1997-10-27|1999-10-27|Designers will display slow however valuable conditions. Large, anxious children should not get into a books. Soft, necessary services shall enable european, |3.78|1.96|2002001|importoimporto #1|2|shirts|2|Men|71|oughtation|petite|5702323349979red2240|peach|Dozen|Unknown|29|callyoughtablepriought| +13217|AAAAAAAAAKDDAAAA|1999-10-28|2001-10-26|Designers will display slow however valuable conditions. Large, anxious children should not get into a books. Soft, necessary services shall enable european, |0.96|1.96|2002001|corpnameless #8|2|football|8|Sports|462|oughtation|N/A|31289yellow886251040|puff|Carton|Unknown|25|ationoughtablepriought| +13218|AAAAAAAAAKDDAAAA|2001-10-27||Designers will display slow however valuable conditions. Large, anxious children should not get into a books. Soft, necessary services shall enable european, |4.10|1.96|1004001|edu packamalg #1|2|swimwear|1|Women|462|ablecallyese|extra large|31289yellow886251040|rosy|Oz|Unknown|10|eingoughtablepriought| +13219|AAAAAAAADKDDAAAA|1997-10-27||British services benefi|2.03|1.44|8015010|scholarmaxi #10|15|fishing|8|Sports|303|pribarpri|N/A|60265195487380misty2|spring|Box|Unknown|57|n stoughtablepriought| +13220|AAAAAAAAEKDDAAAA|1997-10-27|2000-10-26|So financial systems answer aside in a intervals. Daughters smell occasionally worthy territories; more clean plans maintain so new, aware policie|2.72|1.68|10012012|importoamalgamalg #12|12|monitors|10|Electronics|379|n stationpri|N/A|7260843958papaya8695|pink|Lb|Unknown|17|barableablepriought| +13221|AAAAAAAAEKDDAAAA|2000-10-27||More electoral metres cha|1.30|1.06|10012012|edu packnameless #2|14|glassware|7|Home|379|n stationpri|N/A|2243322papaya4569679|purple|Box|Unknown|61|oughtableablepriought| +13222|AAAAAAAAGKDDAAAA|1997-10-27|1999-10-27|Previously economic leaders may get etc able positions. No longer |1.11|0.41|1004001|edu packamalg #1|4|swimwear|1|Women|164|esecallyought|petite|755powder22267336323|sienna|Pound|Unknown|50|ableableablepriought| +13223|AAAAAAAAGKDDAAAA|1999-10-28|2001-10-26|Previously economic leaders may get etc able positions. No longer |0.70|0.41|1004001|importoamalg #2|4|fragrances|1|Women|700|barbaration|large|670868237smoke671657|puff|Case|Unknown|7|priableablepriought| +13224|AAAAAAAAGKDDAAAA|2001-10-27||Previously economic leaders may get etc able positions. No longer |0.77|0.41|1004001|amalgexporti #1|1|newborn|3|Children|700|barbaration|medium|670868237smoke671657|wheat|Bundle|Unknown|52|eseableablepriought| +13225|AAAAAAAAJKDDAAAA|1997-10-27||Involunta|6.52|5.41|7007006|brandbrand #6|7|decor|7|Home|331|oughtpripri|N/A|40601papaya105406606|ivory|Gross|Unknown|80|antiableablepriought| +13226|AAAAAAAAKKDDAAAA|1997-10-27|2000-10-26|Here clear difficulties depart. Major matters cannot sell anyway reasons. Previously twin tactics kno|4.28|1.36|10015001|scholaramalgamalg #1|15|portable|10|Electronics|43|priese|N/A|88522497208salmon342|khaki|Tbl|Unknown|19|callyableablepriought| +13227|AAAAAAAAKKDDAAAA|2000-10-27||Only tall games meet both in a periods; bad, misleading homes think to a months. Clear detailed numbers|0.22|1.36|10015001|amalgmaxi #4|1|arts|9|Books|399|priese|N/A|8blue649980933122709|yellow|Ounce|Unknown|6|ationableablepriought| +13228|AAAAAAAAMKDDAAAA|1997-10-27|1999-10-27|Internal circumstances used to use constantly pictures. Great, dutch meetings need somewhere national examples; simple groups might light. Always only figures face aga|6.50|4.22|3002001|importoexporti #1|2|infants|3|Children|419|n stoughtese|medium|indian99020765748368|ivory|Ton|Unknown|54|eingableablepriought| +13229|AAAAAAAAMKDDAAAA|1999-10-28|2001-10-26|Internal circumstances used to use constantly pictures. Great, dutch meetings need somewhere national examples; simple groups might light. Always only figures face aga|6.40|4.22|10001004|amalgunivamalg #4|1|cameras|10|Electronics|661|oughtcallycally|N/A|800navy3093281362146|spring|Lb|Unknown|51|n stableablepriought| +13230|AAAAAAAAMKDDAAAA|2001-10-27||Internal circumstances used to use constantly pictures. Great, dutch meetings need somewhere national examples; simple groups might light. Always only figures face aga|34.43|4.22|8002001|importonameless #1|1|baseball|8|Sports|661|oughtcallycally|N/A|9679654544844sandy72|royal|Gross|Unknown|11|barpriablepriought| +13231|AAAAAAAAPKDDAAAA|1997-10-27||Angry, full observers get. Shou|4.98|1.79|7002002|importobrand #2|2|bedding|7|Home|69|n stcally|N/A|4ivory75557803041192|sandy|Dram|Unknown|29|oughtpriablepriought| +13232|AAAAAAAAALDDAAAA|1997-10-27|2000-10-26|Just social temperatures should like english networks. Together financial collections must |6.24|4.05|7004003|edu packbrand #3|4|curtains/drapes|7|Home|25|antiable|N/A|6571755051bisque7132|honeydew|Unknown|Unknown|51|ablepriablepriought| +13233|AAAAAAAAALDDAAAA|2000-10-27||Just social temperatures should like english networks. Together financial collections must |3.33|4.05|4004002|edu packedu pack #2|4|athletic|4|Shoes|321|antiable|large|7navy280891294927850|ivory|Dram|Unknown|63|pripriablepriought| +13234|AAAAAAAACLDDAAAA|1997-10-27|1999-10-27|Rare items meet anyway. Detailed st|4.35|2.30|2002001|importoimporto #1|2|shirts|2|Men|580|bareinganti|medium|834399white061699746|papaya|Pound|Unknown|93|esepriablepriought| +13235|AAAAAAAACLDDAAAA|1999-10-28|2001-10-26|Rare items meet anyway. Detailed st|4.59|2.30|2002001|edu packamalg #2|4|swimwear|1|Women|580|bareinganti|medium|509585892494701sky87|snow|Box|Unknown|42|antipriablepriought| +13236|AAAAAAAACLDDAAAA|2001-10-27||Rare items meet anyway. Detailed st|9.93|2.30|2002001|amalgscholar #1|4|rock|5|Music|471|bareinganti|N/A|918slate097834911909|rose|Lb|Unknown|10|callypriablepriought| +13237|AAAAAAAAFLDDAAAA|1997-10-27||Guilty, painful families shall separate inadequate, causal feet. Other, dangerous indians boost efficiently like a children. Aggressi|14.96|5.53|8012008|importomaxi #8|12|guns|8|Sports|27|ationable|N/A|2spring0648600606867|tomato|Each|Unknown|12|ationpriablepriought| +13238|AAAAAAAAGLDDAAAA|1997-10-27|2000-10-26|Broad ears might achieve overall co|40.34|17.74|3001001|amalgexporti #1|1|newborn|3|Children|116|callyoughtought|economy|6459hot2173210757413|sandy|Lb|Unknown|40|eingpriablepriought| +13239|AAAAAAAAGLDDAAAA|2000-10-27||Broad ears might achieve overall co|0.83|17.74|3001001|edu packunivamalg #14|1|audio|10|Electronics|323|priablepri|N/A|623snow2550003162851|white|Oz|Unknown|5|n stpriablepriought| +13240|AAAAAAAAILDDAAAA|1997-10-27|1999-10-27|Still big costs might not capture superb, large solic|4.24|2.84|8009001|maxinameless #1|9|optics|8|Sports|325|antiablepri|N/A|7989786682steel59036|steel|Each|Unknown|5|bareseablepriought| +13241|AAAAAAAAILDDAAAA|1999-10-28|2001-10-26|Still big costs might not capture superb, large solic|0.94|0.47|8009002|maxinameless #2|9|optics|8|Sports|325|antiablepri|N/A|7989786682steel59036|tomato|Cup|Unknown|79|oughteseablepriought| +13242|AAAAAAAAILDDAAAA|2001-10-27||Still big costs might not capture superb, large solic|0.95|0.67|8009002|scholarmaxi #7|9|history|9|Books|543|antiablepri|N/A|361480286papaya75235|turquoise|Pound|Unknown|7|ableeseablepriought| +13243|AAAAAAAALLDDAAAA|1997-10-27||Voters shall not ring more as existing tec|0.62|0.19|9008008|namelessmaxi #8|8|romance|9|Books|660|barcallycally|N/A|8189968lavender57481|saddle|Case|Unknown|25|prieseablepriought| +13244|AAAAAAAAMLDDAAAA|1997-10-27|2000-10-26|Sov|3.74|3.36|10016002|corpamalgamalg #2|16|wireless|10|Electronics|570|barationanti|N/A|026611628548white856|peru|Dozen|Unknown|64|eseeseablepriought| +13245|AAAAAAAAMLDDAAAA|2000-10-27||Traders reflect even on a unions. General dogs take still down a d|3.96|3.36|10016002|importoedu pack #2|16|mens|4|Shoes|570|barationanti|small|417287pink6020813078|orange|Bunch|Unknown|66|antieseablepriought| +13246|AAAAAAAAOLDDAAAA|1997-10-27|1999-10-27|Extremely tiny projects used to create good, open eyes; here single miles can take often sovi|2.38|1.47|5002001|importoscholar #1|2|country|5|Music|308|eingbarpri|N/A|81767548yellow249708|sky|Bunch|Unknown|24|callyeseablepriought| +13247|AAAAAAAAOLDDAAAA|1999-10-28|2001-10-26|Extremely tiny projects used to create good, open eyes; here single miles can take often sovi|3.35|1.00|8009008|maxinameless #8|2|optics|8|Sports|308|eingbarpri|N/A|81767548yellow249708|papaya|Pound|Unknown|59|ationeseablepriought| +13248|AAAAAAAAOLDDAAAA|2001-10-27||Nurses smoke thus useful, american eyes; fully new changes might|0.21|1.00|7002009|importobrand #9|2|bedding|7|Home|813|prioughteing|N/A|81767548yellow249708|lime|Box|Unknown|5|eingeseablepriought| +13249|AAAAAAAABMDDAAAA|1997-10-27||Head facts resolve even. Characteristics put. Toxic, genuine officials shall not meet. Difficult chil|3.85|2.46|9002002|importomaxi #2|2|business|9|Books|577|ationationanti|N/A|0538890cornflower790|sandy|Tsp|Unknown|58|n steseablepriought| +13250|AAAAAAAACMDDAAAA|1997-10-27|2000-10-26|Tall, legal features cou|8.07|3.87|4003001|exportiedu pack #1|3|kids|4|Shoes|232|ablepriable|medium|7217363623894puff884|seashell|Ounce|Unknown|22|barantiablepriought| +13251|AAAAAAAACMDDAAAA|2000-10-27||Current, very names withdraw then hours; more western lists m|1.31|1.11|4003001|importoamalgamalg #9|12|monitors|10|Electronics|232|ablepriable|N/A|7217363623894puff884|medium|N/A|Unknown|19|oughtantiablepriought| +13252|AAAAAAAAEMDDAAAA|1997-10-27|1999-10-27|S|8.46|3.80|5004001|edu packscholar #1|4|classical|5|Music|533|priprianti|N/A|143874709650lemon576|peru|Dram|Unknown|61|ableantiablepriought| +13253|AAAAAAAAEMDDAAAA|1999-10-28|2001-10-26|Finally past years see new, final programmes. Othe|10.81|3.80|5004001|scholarbrand #4|4|blinds/shades|7|Home|947|priprianti|N/A|143874709650lemon576|red|Pound|Unknown|20|priantiablepriought| +13254|AAAAAAAAEMDDAAAA|2001-10-27||Finally past years see new, final programmes. Othe|0.28|3.80|3001001|amalgexporti #1|1|newborn|3|Children|692|ablen stcally|small|143874709650lemon576|snow|Case|Unknown|31|eseantiablepriought| +13255|AAAAAAAAHMDDAAAA|1997-10-27||Small, good plants work direct, profitable applications. As fat results smell then; classes must|2.57|2.05|4003002|exportiedu pack #2|3|kids|4|Shoes|357|ationantipri|small|5054298330blanched35|saddle|Gross|Unknown|8|antiantiablepriought| +13256|AAAAAAAAIMDDAAAA|1997-10-27|2000-10-26|Mad, positive circumstances find keen teams. Years account to a efforts. Upper maps would govern additio|3.75|2.70|9013009|exportiunivamalg #9|13|self-help|9|Books|97|ationn st|N/A|398568354forest27870|sandy|Bundle|Unknown|37|callyantiablepriought| +13257|AAAAAAAAIMDDAAAA|2000-10-27||Rates ought to want in a things; friends hire as usual rare foreigners. |7.27|2.70|8016008|corpmaxi #8|16|golf|8|Sports|97|ationn st|N/A|398568354forest27870|wheat|Unknown|Unknown|30|ationantiablepriought| +13258|AAAAAAAAKMDDAAAA|1997-10-27|1999-10-27|Blue skills conform so years. New affairs could get quick, previous questions. Constant, new companies will not supply purely. Tryi|4.12|2.84|6006003|corpcorp #3|6|rings|6|Jewelry|68|eingcally|N/A|85tan436856184549085|white|Unknown|Unknown|30|eingantiablepriought| +13259|AAAAAAAAKMDDAAAA|1999-10-28|2001-10-26|Blue skills conform so years. New affairs could get quick, previous questions. Constant, new companies will not supply purely. Tryi|2.34|2.84|6006003|exporticorp #4|3|gold|6|Jewelry|616|eingcally|N/A|85tan436856184549085|wheat|Bunch|Unknown|12|n stantiablepriought| +13260|AAAAAAAAKMDDAAAA|2001-10-27||Holy, tough problems meet then vast hands. Effectively proposed members will find t|14.26|9.55|6006003|edu packedu pack #1|3|athletic|4|Shoes|156|eingcally|medium|85tan436856184549085|royal|Dram|Unknown|20|barcallyablepriought| +13261|AAAAAAAANMDDAAAA|1997-10-27||Current mothers see both sure, poor months. Interesting, hard cattle shall not learn over on a men; new times handle for the weeks. Much confidential|2.93|1.23|6009006|maxicorp #6|9|womens watch|6|Jewelry|512|ableoughtanti|N/A|13487784470945slate8|sky|Tsp|Unknown|44|oughtcallyablepriought| +13262|AAAAAAAAOMDDAAAA|1997-10-27|2000-10-26|Convincing departments accept grap|1.58|0.77|2003001|exportiimporto #1|3|pants|2|Men|126|callyableought|small|1060219134597salmon1|thistle|Oz|Unknown|78|ablecallyablepriought| +13263|AAAAAAAAOMDDAAAA|2000-10-27||Convincing departments accept grap|7.55|6.79|7001004|amalgbrand #4|1|bathroom|7|Home|126|callyableought|N/A|43papaya396733960182|snow|Unknown|Unknown|31|pricallyablepriought| +13264|AAAAAAAAANDDAAAA|1997-10-27|1999-10-27|Full, american sources would not get little such as a hours. Different yards work less accounts. Important, modern windows choke to a companies. Things must not examine other compa|9.74|8.27|5003001|exportischolar #1|3|pop|5|Music|409|n stbarese|N/A|7steel86858097637081|tomato|N/A|Unknown|17|esecallyablepriought| +13265|AAAAAAAAANDDAAAA|1999-10-28|2001-10-26|Full, american sources would not get little such as a hours. Different yards work less accounts. Important, modern windows choke to a companies. Things must not examine other compa|5.08|8.27|6012002|importobrand #2|12|costume|6|Jewelry|409|n stbarese|N/A|7steel86858097637081|gainsboro|Unknown|Unknown|11|anticallyablepriought| +13266|AAAAAAAAANDDAAAA|2001-10-27||Full, american sources would not get little such as a hours. Different yards work less accounts. Important, modern windows choke to a companies. Things must not examine other compa|9.97|8.27|9006007|corpmaxi #7|12|parenting|9|Books|409|n stbarese|N/A|9804889084851khaki02|lawn|Pallet|Unknown|23|callycallyablepriought| +13267|AAAAAAAADNDDAAAA|1997-10-27||Clear, harsh police used to include large, appropriate plans. Prices could produce more. There white weapons expect directly free conclusions. Responsibl|4.57|3.88|9002008|importomaxi #8|2|business|9|Books|98|eingn st|N/A|ivory261324782275516|sandy|Dram|Unknown|70|ationcallyablepriought| +13268|AAAAAAAAENDDAAAA|1997-10-27|2000-10-26|Jews can alert still scenes. Widely appropriate figures dance numbers. Small sources would suggest now. Green, insufficient fans hold together so normal shares. Players mig|2.37|1.01|3001001|amalgexporti #1|1|newborn|3|Children|383|prieingpri|small|6094910150papaya7328|indian|Pound|Unknown|93|eingcallyablepriought| +13269|AAAAAAAAENDDAAAA|2000-10-27||Groups might clear aspects. Partial presents may not remain only psychological t|9.85|1.01|3004002|edu packexporti #2|4|school-uniforms|3|Children|383|prieingpri|extra large|8582681089709lime383|moccasin|N/A|Unknown|67|n stcallyablepriought| +13270|AAAAAAAAGNDDAAAA|1997-10-27|1999-10-27|Better busy armies become. Forms see later payments; industrial places could accept dead elections. Sons pour more small animal|2.82|2.03|6014007|edu packbrand #7|14|estate|6|Jewelry|84|eseeing|N/A|176mint0602521090843|navajo|Bundle|Unknown|21|barationablepriought| +13271|AAAAAAAAGNDDAAAA|1999-10-28|2001-10-26|Royal, still animals would feel young artists; in partic|3.56|2.03|10005012|scholarunivamalg #12|5|karoke|10|Electronics|420|eseeing|N/A|176mint0602521090843|tan|Each|Unknown|94|oughtationablepriought| +13272|AAAAAAAAGNDDAAAA|2001-10-27||Royal, still animals would feel young artists; in partic|7.39|3.47|10005012|importoscholar #1|5|country|5|Music|420|barableese|N/A|176mint0602521090843|slate|Box|Unknown|33|ableationablepriought| +13273|AAAAAAAAJNDDAAAA|1997-10-27||Others should not contribute very adequate dogs. Substances might involve thus just common assets. Maybe able objects bear sharp from a terms. Probably nasty centres |4.34|3.64|6001002|amalgcorp #2|1|birdal|6|Jewelry|94|esen st|N/A|11violet074950976046|sandy|Case|Unknown|19|priationablepriought| +13274|AAAAAAAAKNDDAAAA|1997-10-27|2000-10-26|Attempts expect so major studies. Audiences will not help later years. Current, |4.92|2.70|10016015|corpamalgamalg #15|16|wireless|10|Electronics|323|priablepri|N/A|7785697172spring2788|powder|Pallet|Unknown|60|eseationablepriought| +13275|AAAAAAAAKNDDAAAA|2000-10-27||Attempts expect so major studies. Audiences will not help later years. Current, |4.13|2.70|7013006|exportinameless #6|13|wallpaper|7|Home|323|priablepri|N/A|7785697172spring2788|puff|Bundle|Unknown|34|antiationablepriought| +13276|AAAAAAAAMNDDAAAA|1997-10-27|1999-10-27|Other schools sell comparative, able reports. Authorities alter perhaps. Happy, double functions help dry practitione|3.45|2.55|5001001|amalgscholar #1|1|rock|5|Music|536|callyprianti|N/A|784031224saddle61279|purple|Carton|Unknown|15|callyationablepriought| +13277|AAAAAAAAMNDDAAAA|1999-10-28|2001-10-26|Huge, very babies ought to deny; fresh gifts ought to grind recently but full forces; s|5.86|2.55|5001001|amalgbrand #6|1|semi-precious|6|Jewelry|566|callyprianti|N/A|082765169769royal658|peru|Ounce|Unknown|7|ationationablepriought| +13278|AAAAAAAAMNDDAAAA|2001-10-27||Huge, very babies ought to deny; fresh gifts ought to grind recently but full forces; s|0.98|0.58|5002001|importoscholar #1|1|country|5|Music|278|callyprianti|N/A|082765169769royal658|peru|Case|Unknown|53|eingationablepriought| +13279|AAAAAAAAPNDDAAAA|1997-10-27||Experiments end. Particular others happen too. Here potential hands might drop warm years. Very coastal years know. Most old states cope again |2.98|1.28|6012008|importobrand #8|12|costume|6|Jewelry|155|antiantiought|N/A|94929sandy8282045052|red|Bundle|Unknown|1|n stationablepriought| +13280|AAAAAAAAAODDAAAA|1997-10-27|2000-10-26|Final techniques could not breathe effective, certain men. Light trees shall lie no doubt. Sure, industrial programmes succeed either. Almost national steps should describe today che|59.56|47.05|5002001|importoscholar #1|2|country|5|Music|352|ableantipri|N/A|929968pink6972600461|white|Gross|Unknown|3|bareingablepriought| +13281|AAAAAAAAAODDAAAA|2000-10-27||English parents let only neighbouring, constitutional branches. More flexible women used to see in order projects. Hot contents should not think under for a neighbours. |4.05|2.55|9010010|univunivamalg #10|10|travel|9|Books|352|ableantipri|N/A|929968pink6972600461|pink|Ounce|Unknown|79|oughteingablepriought| +13282|AAAAAAAACODDAAAA|1997-10-27|1999-10-27|Together common |2.87|0.91|5002001|importoscholar #1|2|country|5|Music|175|antiationought|N/A|84329599199orchid055|sandy|N/A|Unknown|68|ableeingablepriought| +13283|AAAAAAAACODDAAAA|1999-10-28|2001-10-26|Always other suggestions favour police. Supporters ought to obtain most from the men. Parts should die just rules. Evolutionary, average cats attack however from a properties. Crucial,|9.40|7.14|3003002|exportiexporti #2|3|toddlers|3|Children|856|callyantieing|small|27874sandy2387792740|sky|Lb|Unknown|3|prieingablepriought| +13284|AAAAAAAACODDAAAA|2001-10-27||Always other suggestions favour police. Supporters ought to obtain most from the men. Parts should die just rules. Evolutionary, average cats attack however from a properties. Crucial,|2.43|7.14|3003002|importoscholar #1|2|country|5|Music|856|callyantieing|N/A|085546slate674244382|pale|Pound|Unknown|24|eseeingablepriought| +13285|AAAAAAAAFODDAAAA|1997-10-27||International, outstanding eyes cannot come different, international sports. Types realise loudly almost free voters; boats shall look finally final, particula|8.95|7.33|1003002|exportiamalg #2|3|maternity|1|Women|232|ablepriable|large|243purple76799159561|saddle|Dozen|Unknown|39|antieingablepriought| +13286|AAAAAAAAGODDAAAA|1997-10-27|2000-10-26|Traditional, necessary activities would get thick safely aware demands. Annual, military arrangement|4.44|2.22|8012007|importomaxi #7|12|guns|8|Sports|859|n stantieing|N/A|0450912241735papaya6|seashell|N/A|Unknown|33|callyeingablepriought| +13287|AAAAAAAAGODDAAAA|2000-10-27||Other, modern officials must not say. Sorry, major users m|0.89|0.46|5004002|edu packscholar #2|12|classical|5|Music|75|n stantieing|N/A|8955634violet4023434|spring|Case|Unknown|34|ationeingablepriought| +13288|AAAAAAAAIODDAAAA|1997-10-27|1999-10-27|Different days read impossible, old farms. Certain proposals cannot protect long from a pr|5.23|4.60|8006009|corpnameless #9|6|football|8|Sports|311|oughtoughtpri|N/A|tomato52014322101786|red|N/A|Unknown|27|eingeingablepriought| +13289|AAAAAAAAIODDAAAA|1999-10-28|2001-10-26|Different days read impossible, old farms. Certain proposals cannot protect long from a pr|3.20|4.60|1004002|edu packamalg #2|6|swimwear|1|Women|311|oughtoughtpri|medium|767724221887lime7739|pale|Dozen|Unknown|22|n steingablepriought| +13290|AAAAAAAAIODDAAAA|2001-10-27||Different days read impossible, old farms. Certain proposals cannot protect long from a pr|3.45|4.60|3003001|exportiexporti #1|6|toddlers|3|Children|311|oughtoughtpri|small|1855675290823653red6|cream|N/A|Unknown|94|barn stablepriought| +13291|AAAAAAAALODDAAAA|1997-10-27||As generous germans mean almost eastern variables. Long years must not face really good, atomic relations; chemical, corporate bills must honour seasons. Artificial, gold materials determine|4.51|2.07|9006008|corpmaxi #8|6|parenting|9|Books|617|ationoughtcally|N/A|596234lavender431547|bisque|Bundle|Unknown|44|oughtn stablepriought| +13292|AAAAAAAAMODDAAAA|1997-10-27|2000-10-26|Soon dirty prices might walk expensive, basic acres; particular functions die only into a orders. Available years tell affectionately hands. Earlier eag|2.70|2.29|5002001|importoscholar #1|2|country|5|Music|279|n stationable|N/A|244pale6943624687016|dark|Dram|Unknown|65|ablen stablepriought| +13293|AAAAAAAAMODDAAAA|2000-10-27||Old things should sit however public, national universities; conditions withdraw gradually |5.75|1.95|5002001|univbrand #6|2|jewelry boxes|6|Jewelry|421|oughtableese|N/A|rosy2688073658067016|smoke|Pallet|Unknown|30|prin stablepriought| +13294|AAAAAAAAOODDAAAA|1997-10-27|1999-10-27|Widespread restrictions would survive infinitely around adverse years. Goo|0.31|0.17|3001001|amalgexporti #1|1|newborn|3|Children|50|baranti|large|74240958735powder426|slate|Gram|Unknown|20|esen stablepriought| +13295|AAAAAAAAOODDAAAA|1999-10-28|2001-10-26|Times ought to satisfy explicitly practical hours; harsh, polite walls make in a fie|2.80|1.03|3001001|exportiedu pack #2|1|kids|4|Shoes|366|baranti|medium|74240958735powder426|pink|Carton|Unknown|64|antin stablepriought| +13296|AAAAAAAAOODDAAAA|2001-10-27||Practical forms put in a reactions. Conditions |3.14|1.03|2001001|amalgimporto #1|1|accessories|2|Men|77|ationation|large|1394678989papaya5873|wheat|Lb|Unknown|12|callyn stablepriought| +13297|AAAAAAAABPDDAAAA|1997-10-27||Open clothes sit simply. Needs leave soon proud forces. Only capital products will teach upon the hotels. Now bad police speculate direct strange plans. So particular people|0.77|0.30|4003002|exportiedu pack #2|3|kids|4|Shoes|616|callyoughtcally|extra large|3846827795959sandy81|almond|N/A|Unknown|59|ationn stablepriought| +13298|AAAAAAAACPDDAAAA|1997-10-27|2000-10-26|Still head improvements used to head always inevitably small matters. Inner examinations work scottish, conservative times. Total reports used to understand only, young concepts. |4.13|3.67|4003001|exportiedu pack #1|3|kids|4|Shoes|494|esen stese|medium|6183758025497indian1|powder|Pallet|Unknown|22|eingn stablepriought| +13299|AAAAAAAACPDDAAAA|2000-10-27||New reservations can open b|0.68|3.67|1004002|edu packamalg #2|4|swimwear|1|Women|593|esen stese|small|6183758025497indian1|plum|Gram|Unknown|65|n stn stablepriought| +13300|AAAAAAAAEPDDAAAA|1997-10-27|1999-10-27|Services used to work most new provi|2.84|2.27|9001005|amalgmaxi #5|1|arts|9|Books|645|antiesecally|N/A|662451820346puff8577|medium|Tbl|Unknown|45|barbarpripriought| +13301|AAAAAAAAEPDDAAAA|1999-10-28|2001-10-26|Perhaps distinctive years will show refugees; necessary, pure eyes should provide. Apparent years seem as charming, eager figures; essential, other disputes can face sub|1.65|2.27|9001005|exportiimporto #2|1|pants|2|Men|309|antiesecally|extra large|49103787thistle94046|wheat|Case|Unknown|49|oughtbarpripriought| +13302|AAAAAAAAEPDDAAAA|2001-10-27||Meals can go producers. Exhibitions should raise. Resources care only. Uniquely dark thing|1.95|1.24|1003001|exportiamalg #1|3|maternity|1|Women|363|antiesecally|medium|49103787thistle94046|yellow|Carton|Unknown|56|ablebarpripriought| +13303|AAAAAAAAHPDDAAAA|1997-10-27||As local provisions tak|19.20|11.71|4003002|exportiedu pack #2|3|kids|4|Shoes|619|n stoughtcally|extra large|132663813spring15702|smoke|Cup|Unknown|40|pribarpripriought| +13304|AAAAAAAAIPDDAAAA|1997-10-27|2000-10-26|Good, likely associations stop to a members. Still white breasts help sizes; areas will not marry classical leaves. Problems used to bec|98.65|36.50|3003001|exportiexporti #1|3|toddlers|3|Children|182|ableeingought|medium|78729612686455smoke5|smoke|Bundle|Unknown|50|esebarpripriought| +13305|AAAAAAAAIPDDAAAA|2000-10-27||Faintly full accounts blame as. Little, other possib|2.73|36.50|3003001|maximaxi #10|9|science|9|Books|210|baroughtable|N/A|60447rosy29563310974|puff|Tsp|Unknown|8|antibarpripriought| +13306|AAAAAAAAKPDDAAAA|1997-10-27|1999-10-27|Dates may make yet in a causes. Really small hospitals include colours; delighted, remote events used to resolve also safe corporati|4.78|3.96|2004001|edu packimporto #1|4|sports-apparel|2|Men|289|n steingable|petite|487green529410679361|steel|Oz|Unknown|8|callybarpripriought| +13307|AAAAAAAAKPDDAAAA|1999-10-28|2001-10-26|Dates may make yet in a causes. Really small hospitals include colours; delighted, remote events used to resolve also safe corporati|2.05|3.96|2004001|amalgnameless #8|1|athletic shoes|8|Sports|289|n steingable|N/A|670260861light650122|lime|Pallet|Unknown|9|ationbarpripriought| +13308|AAAAAAAAKPDDAAAA|2001-10-27||Dates may make yet in a causes. Really small hospitals include colours; delighted, remote events used to resolve also safe corporati|2.72|2.01|6012005|importobrand #5|12|costume|6|Jewelry|289|n steingable|N/A|670260861light650122|white|Oz|Unknown|58|eingbarpripriought| +13309|AAAAAAAANPDDAAAA|1997-10-27||Particular variables see. At once conservative metres wait even; hence confident pounds might stimulate as for a agents. Girls may not master literally. Once retail th|50.30|25.15|7004010|edu packbrand #10|4|curtains/drapes|7|Home|27|ationable|N/A|26649sky305888267100|sky|Tsp|Unknown|82|n stbarpripriought| +13310|AAAAAAAAOPDDAAAA|1997-10-27|2000-10-26|Thin feet come right indeed large systems. Here lucky feet ought to ask decisions. Main|7.84|3.13|2002001|importoimporto #1|2|shirts|2|Men|323|priablepri|petite|12972331saddle386494|forest|Dram|Unknown|16|baroughtpripriought| +13311|AAAAAAAAOPDDAAAA|2000-10-27||Also fundamental events interfere concerned cases; lips would stop almost important, successful tasks. Light results can take only central poli|5.61|3.13|9012004|importounivamalg #4|12|home repair|9|Books|238|eingpriable|N/A|12972331saddle386494|rose|Tsp|Unknown|5|oughtoughtpripriought| +13312|AAAAAAAAAAEDAAAA|1997-10-27|1999-10-27|Concerned politicians cannot listen there. Sometimes other followers occur urban, physical years. Concerned words might not set. Workers can perform then in a individuals. So strong im|3.30|1.91|8012003|importomaxi #3|12|guns|8|Sports|191|oughtn stought|N/A|848159068264orchid88|salmon|Cup|Unknown|11|ableoughtpripriought| +13313|AAAAAAAAAAEDAAAA|1999-10-28|2001-10-26|Residential, sophisticated children judge critical, big symbols. Days find so weak details; constitutional days remove readers; available, female methods control areas. Technical stud|4.66|3.96|2004002|edu packimporto #2|4|sports-apparel|2|Men|41|oughtn stought|large|12957dim115698590947|powder|Unknown|Unknown|9|prioughtpripriought| +13314|AAAAAAAAAAEDAAAA|2001-10-27||Residential, sophisticated children judge critical, big symbols. Days find so weak details; constitutional days remove readers; available, female methods control areas. Technical stud|1.89|1.05|10013004|exportiamalgamalg #4|13|stereo|10|Electronics|36|callypri|N/A|12957dim115698590947|red|Dozen|Unknown|18|eseoughtpripriought| +13315|AAAAAAAADAEDAAAA|1997-10-27||Steps reproduce hard museums; happy, fundamental schools ought to contain. New, double operations tend identical, guilty remarks. Days enjoy so for a rights. Rationally appointed student|1.69|1.06|3004002|edu packexporti #2|4|school-uniforms|3|Children|481|oughteingese|small|42456226479royal4735|papaya|Dram|Unknown|17|antioughtpripriought| +13316|AAAAAAAAEAEDAAAA|1997-10-27|2000-10-26|Banks shall not make there. Times keep never ultimately real children. Thanks should shoot more then public times. Boxes|3.81|1.67|3002001|importoexporti #1|2|infants|3|Children|215|antioughtable|small|6347178pink166433393|puff|Each|Unknown|7|callyoughtpripriought| +13317|AAAAAAAAEAEDAAAA|2000-10-27||Banks shall not make there. Times keep never ultimately real children. Thanks should shoot more then public times. Boxes|8.54|1.67|3002001|exportiunivamalg #10|2|self-help|9|Books|241|oughteseable|N/A|5459salmon6898231635|rosy|Ton|Unknown|34|ationoughtpripriought| +13318|AAAAAAAAGAEDAAAA|1997-10-27|1999-10-27|Privileges should dare agencies. Contrary, good personnel must move. Then particular programmes k|0.93|0.54|3002001|importoexporti #1|2|infants|3|Children|53|prianti|economy|485sky61754217425267|red|Tbl|Unknown|99|eingoughtpripriought| +13319|AAAAAAAAGAEDAAAA|1999-10-28|2001-10-26|Entirely available girls|1.89|1.60|3002001|edu packmaxi #12|4|entertainments|9|Books|656|prianti|N/A|9powder9652436792962|moccasin|Oz|Unknown|11|n stoughtpripriought| +13320|AAAAAAAAGAEDAAAA|2001-10-27||Entirely available girls|7.46|3.73|3002001|importoexporti #1|2|infants|3|Children|213|prioughtable|medium|462250403091276dim32|red|Unknown|Unknown|32|barablepripriought| +13321|AAAAAAAAJAEDAAAA|1997-10-27||Different months change questions. Serious countries appeal so black, labour women. Enough dangerous crimes can organise here right, local gi|0.79|0.33|1003002|exportiamalg #2|3|maternity|1|Women|413|prioughtese|large|346702287500salmon33|peru|Tsp|Unknown|61|oughtablepripriought| +13322|AAAAAAAAKAEDAAAA|1997-10-27|2000-10-26|Meetings sleep wise needs. Black, other deaths provide on|5.31|3.66|8015003|scholarmaxi #3|15|fishing|8|Sports|678|eingationcally|N/A|4162rose785954674314|violet|Oz|Unknown|46|ableablepripriought| +13323|AAAAAAAAKAEDAAAA|2000-10-27||Christians would not become under to a responses. Words avoid eventually. Possible year|3.46|1.45|8015003|maxiunivamalg #13|15|televisions|10|Electronics|330|barpripri|N/A|lawn5025977477777288|rosy|Pallet|Unknown|65|priablepripriought| +13324|AAAAAAAAMAEDAAAA|1997-10-27|1999-10-27|Still specific conditions go occasions; traditional claims might help great plant|9.08|3.90|10001014|amalgunivamalg #14|1|cameras|10|Electronics|142|ableeseought|N/A|507456934437400dim85|yellow|Pallet|Unknown|13|eseablepripriought| +13325|AAAAAAAAMAEDAAAA|1999-10-28|2001-10-26|Still specific conditions go occasions; traditional claims might help great plant|1.27|0.74|10001014|maxinameless #6|1|optics|8|Sports|835|ableeseought|N/A|507456934437400dim85|spring|Dozen|Unknown|47|antiablepripriought| +13326|AAAAAAAAMAEDAAAA|2001-10-27||Great minutes shall fly dry, main workers; labour remains determine once critical, open games; secret, present appointments might consider in a authorities. Instead you|2.11|0.74|10001014|brandbrand #7|7|decor|7|Home|835|antiprieing|N/A|507456934437400dim85|lavender|Tsp|Unknown|61|callyablepripriought| +13327|AAAAAAAAPAEDAAAA|1997-10-27||Generally delicate police would turn for a discussions. Years might wait |1.94|1.74|3002002|importoexporti #2|2|infants|3|Children|933|priprin st|medium|96lawn48930275403266|powder|Carton|Unknown|24|ationablepripriought| +13328|AAAAAAAAABEDAAAA|1997-10-27|2000-10-26|No longer soft cameras mean select, small policies. Also famous arms could want irish, different connections. Attacks should remove black r|0.50|0.44|3004001|edu packexporti #1|4|school-uniforms|3|Children|89|n steing|medium|royal381261809977723|light|Lb|Unknown|26|eingablepripriought| +13329|AAAAAAAAABEDAAAA|2000-10-27||No longer soft cameras mean select, small policies. Also famous arms could want irish, different connections. Attacks should remove black r|1.52|0.44|3004001|amalgmaxi #10|1|arts|9|Books|89|n steing|N/A|royal381261809977723|yellow|Gross|Unknown|42|n stablepripriought| +13330|AAAAAAAACBEDAAAA|1997-10-27|1999-10-27|National, suitable actions create other firms. Impossible lives cannot produce however such as a men. Details recall tomorrow. Simple, individ|3.58|2.79|2004001|edu packimporto #1|4|sports-apparel|2|Men|265|anticallyable|extra large|7221259413072white94|slate|Carton|Unknown|17|barpripripriought| +13331|AAAAAAAACBEDAAAA|1999-10-28|2001-10-26|National, suitable actions create other firms. Impossible lives cannot produce however such as a men. Details recall tomorrow. Simple, individ|3.91|1.72|2004001|importoimporto #2|4|shirts|2|Men|265|anticallyable|small|7221259413072white94|rose|Gram|Unknown|33|oughtpripripriought| +13332|AAAAAAAACBEDAAAA|2001-10-27||National, suitable actions create other firms. Impossible lives cannot produce however such as a men. Details recall tomorrow. Simple, individ|2.14|1.72|2004001|exportiamalg #1|3|maternity|1|Women|288|eingeingable|economy|7221259413072white94|papaya|Ounce|Unknown|39|ablepripripriought| +13333|AAAAAAAAFBEDAAAA|1997-10-27||For example voluntary structures believe alone calls. Once more european events hope high sets. Units build there. More fundamental doors go notably slowly democratic stages. Acute mer|1.57|1.03|10010006|univamalgamalg #6|10|memory|10|Electronics|101|oughtbarought|N/A|95173506royal8230714|sienna|Gram|Unknown|26|pripripripriought| +13334|AAAAAAAAGBEDAAAA|1997-10-27|2000-10-26|Great, possible arguments should not interfere so studies. Proposals identify a bit. Old courts tell here times. Amounts prove c|92.80|80.73|2002001|importoimporto #1|2|shirts|2|Men|893|prin steing|small|6649323357033tan3344|rosy|Pound|Unknown|26|esepripripriought| +13335|AAAAAAAAGBEDAAAA|2000-10-27||Circumstances must rise fixed implications. Perhaps solid subjects may take. Months support now eyes. Old tests get. Already working environments|42.01|32.76|2002001|corpmaxi #6|16|golf|8|Sports|893|prin steing|N/A|155962498salmon27167|violet|Each|Unknown|41|antipripripriought| +13336|AAAAAAAAIBEDAAAA|1997-10-27|1999-10-27|Alone, classical supporters must come closer on a projects. Current materials exceed very so standard banks. Industrial heads give personnel. Unnecessary, likely females shall put here. Natural, wild |0.62|0.32|4002001|importoedu pack #1|2|mens|4|Shoes|394|esen stpri|small|087turquoise55819271|snow|Tbl|Unknown|88|callypripripriought| +13337|AAAAAAAAIBEDAAAA|1999-10-28|2001-10-26|Alone, classical supporters must come closer on a projects. Current materials exceed very so standard banks. Industrial heads give personnel. Unnecessary, likely females shall put here. Natural, wild |4.92|1.47|4002001|scholarmaxi #8|2|fishing|8|Sports|394|esen stpri|N/A|865magenta1148245126|peru|Gram|Unknown|61|ationpripripriought| +13338|AAAAAAAAIBEDAAAA|2001-10-27||Schools live brilliant years. New books should not match more than bills. Big pictures mentio|2.26|1.47|10014011|edu packamalgamalg #11|2|automotive|10|Electronics|376|esen stpri|N/A|865magenta1148245126|yellow|Pound|Unknown|27|eingpripripriought| +13339|AAAAAAAALBEDAAAA|1997-10-27||Less tiny farmers help efforts. Fast building|3.72|2.82|7004010|edu packbrand #10|4|curtains/drapes|7|Home|203|pribarable|N/A|46628489829489tan423|lavender|Pound|Unknown|6|n stpripripriought| +13340|AAAAAAAAMBEDAAAA|1997-10-27|2000-10-26|Temporarily original roads could carry more with the references. Affairs work never significant years. Weeks shall gain able views. Ling|4.68|2.99|10003013|exportiunivamalg #13|3|dvd/vcr players|10|Electronics|292|ablen stable|N/A|5578286286sandy29982|wheat|Oz|Unknown|34|baresepripriought| +13341|AAAAAAAAMBEDAAAA|2000-10-27||Influential, new heels conceal. Free features will pop then. Very royal interests bring both low sources. So|4.18|2.09|10003013|edu packedu pack #2|3|athletic|4|Shoes|123|priableought|large|5578286286sandy29982|salmon|Each|Unknown|25|oughtesepripriought| +13342|AAAAAAAAOBEDAAAA|1997-10-27|1999-10-27|Local, pretty systems see largely primary decisions. Religious, white arrangements respond however difficulties. Years shall not think small points. Imag|27.39|9.31|5002001|importoscholar #1|2|country|5|Music|252|ableantiable|N/A|99164111047212steel3|sienna|Dozen|Unknown|30|ableesepripriought| +13343|AAAAAAAAOBEDAAAA|1999-10-28|2001-10-26|Local, pretty systems see largely primary decisions. Religious, white arrangements respond however difficulties. Years shall not think small points. Imag|3.93|9.31|1003002|exportiamalg #2|3|maternity|1|Women|252|ableantiable|medium|99164111047212steel3|wheat|Bundle|Unknown|50|priesepripriought| +13344|AAAAAAAAOBEDAAAA|2001-10-27||Local, pretty systems see largely primary decisions. Religious, white arrangements respond however difficulties. Years shall not think small points. Imag|70.38|34.48|4004001|edu packedu pack #1|4|athletic|4|Shoes|207|ationbarable|extra large|2870powder2324239822|lime|Dozen|Unknown|1|eseesepripriought| +13345|AAAAAAAABCEDAAAA|1997-10-27||Famous, criminal meals could not selec|0.81|0.36|3004002|edu packexporti #2|4|school-uniforms|3|Children|20|barable|medium|blush084589849243570|blush|Gram|Unknown|24|antiesepripriought| +13346|AAAAAAAACCEDAAAA|1997-10-27|2000-10-26|Positive colours ban popular, good prices; both inter|3.50|2.87|4004001|edu packedu pack #1|4|athletic|4|Shoes|968|eingcallyn st|small|82914steel2082204989|gainsboro|Carton|Unknown|2|callyesepripriought| +13347|AAAAAAAACCEDAAAA|2000-10-27||Now f|2.92|1.81|4004001|exportiamalg #2|4|maternity|1|Women|968|eingcallyn st|extra large|498173411steel066548|ivory|Cup|Unknown|27|ationesepripriought| +13348|AAAAAAAAECEDAAAA|1997-10-27|1999-10-27|As usual following causes ought to retain perhaps historical native candidates. Recent, internal activities remember only as likely patients. Cars calm ago low rates. Major, complete beings allo|8.29|6.13|6006005|corpcorp #5|6|rings|6|Jewelry|9|n st|N/A|4honeydew27644045538|rose|Each|Unknown|9|eingesepripriought| +13349|AAAAAAAAECEDAAAA|1999-10-28|2001-10-26|Annual exc|3.33|6.13|6006005|corpcorp #6|6|rings|6|Jewelry|9|n st|N/A|1499176670240196red0|sky|Case|Unknown|5|n stesepripriought| +13350|AAAAAAAAECEDAAAA|2001-10-27||Annual exc|9.83|6.13|7009001|maxibrand #1|6|mattresses|7|Home|145|antieseought|N/A|583578magenta4691718|plum|Tbl|Unknown|75|barantipripriought| +13351|AAAAAAAAHCEDAAAA|1997-10-27||Open accounts hear as well possible proteins. Industrial forces could pay favo|1.47|0.45|7012002|importonameless #2|12|paint|7|Home|251|oughtantiable|N/A|20896puff63182232873|peru|Bunch|Unknown|58|oughtantipripriought| +13352|AAAAAAAAICEDAAAA|1997-10-27|2000-10-26|Terrible tears notice well. Great, prime stations must speak more teams. Measures will tell clear|0.79|0.56|2002001|importoimporto #1|2|shirts|2|Men|955|antiantin st|large|657913goldenrod61814|seashell|Oz|Unknown|6|ableantipripriought| +13353|AAAAAAAAICEDAAAA|2000-10-27||Terrible tears notice well. Great, prime stations must speak more teams. Measures will tell clear|6.68|0.56|2002001|amalgbrand #4|1|bathroom|7|Home|89|antiantin st|N/A|657913goldenrod61814|yellow|Dozen|Unknown|42|priantipripriought| +13354|AAAAAAAAKCEDAAAA|1997-10-27|1999-10-27|Prime, broad measures could seek later men. Social children agree heavy, good views; small teeth will not see as. Welsh, original years wan|29.61|9.77|6010003|univbrand #3|10|jewelry boxes|6|Jewelry|302|ablebarpri|N/A|0saddle8280997326365|white|Dram|Unknown|16|eseantipripriought| +13355|AAAAAAAAKCEDAAAA|1999-10-28|2001-10-26|In situ black conditions win by no mea|1.91|9.77|4002002|importoedu pack #2|2|mens|4|Shoes|302|ablebarpri|small|278361879907spring88|rosy|Dram|Unknown|19|antiantipripriought| +13356|AAAAAAAAKCEDAAAA|2001-10-27||Continuous, relevant hours can ask to a weapons. Drugs can find early only chief interests. Shared, diverse groups would make howeve|3.93|9.77|4002002|edu packnameless #7|4|camping|8|Sports|302|ablebarpri|N/A|60641967011117rose05|royal|Cup|Unknown|25|callyantipripriought| +13357|AAAAAAAANCEDAAAA|1997-10-27||Much accessible years can provide never furthermore medical terms. Trusts characterise profits. In particular o|4.17|3.08|1001002|amalgamalg #2|1|dresses|1|Women|147|ationeseought|large|39932tomato505791299|orchid|Gross|Unknown|65|ationantipripriought| +13358|AAAAAAAAOCEDAAAA|1997-10-27|2000-10-26|Months take. Numerous, big companies will make in a papers. Atlantic minutes keep permanently american p|12.18|8.40|5004001|edu packscholar #1|4|classical|5|Music|198|eingn stought|N/A|986345407puff8635513|white|Unknown|Unknown|13|eingantipripriought| +13359|AAAAAAAAOCEDAAAA|2000-10-27||Obvious, just parents kick angril|6.86|3.08|10004015|edu packunivamalg #15|4|audio|10|Electronics|198|eingn stought|N/A|7776518570105sienna7|tomato|Unknown|Unknown|11|n stantipripriought| +13360|AAAAAAAAADEDAAAA|1997-10-27|1999-10-27|Severely wrong districts provi|7.20|4.96|1004001|edu packamalg #1|4|swimwear|1|Women|201|oughtbarable|small|lace0257436456732497|navajo|Pound|Unknown|13|barcallypripriought| +13361|AAAAAAAAADEDAAAA|1999-10-28|2001-10-26|Severely wrong districts provi|20.91|4.96|6006002|corpcorp #2|6|rings|6|Jewelry|201|oughtbarable|N/A|2731888447743peach95|sandy|Cup|Unknown|3|oughtcallypripriought| +13362|AAAAAAAAADEDAAAA|2001-10-27||Severely wrong districts provi|8.02|7.05|6006002|exportiimporto #1|3|pants|2|Men|702|oughtbarable|large|00256012green9537238|royal|Ounce|Unknown|59|ablecallypripriought| +13363|AAAAAAAADDEDAAAA|1997-10-27||Therefore economic wages shall |4.07|3.21|4001002|amalgedu pack #2|1|womens|4|Shoes|207|ationbarable|large|9sienna4526354910023|sandy|Box|Unknown|15|pricallypripriought| +13364|AAAAAAAAEDEDAAAA|1997-10-27|2000-10-26|British papers believe incidents. Long, alive traditions might review earlier years. Working, statis|8.89|6.22|10012003|importoamalgamalg #3|12|monitors|10|Electronics|68|eingcally|N/A|3775889353284smoke29|tomato|Bundle|Unknown|8|esecallypripriought| +13365|AAAAAAAAEDEDAAAA|2000-10-27||At least important moments shout statistically musicians. Girls may prevent only new, clear individuals.|1.80|6.22|4004002|edu packedu pack #2|12|athletic|4|Shoes|68|eingcally|small|0321477157042violet4|grey|Bunch|Unknown|63|anticallypripriought| +13366|AAAAAAAAGDEDAAAA|1997-10-27|1999-10-27|Now german years o|5.55|4.32|1002001|importoamalg #1|2|fragrances|1|Women|184|eseeingought|medium|61079orchid358341526|ghost|Dram|Unknown|8|callycallypripriought| +13367|AAAAAAAAGDEDAAAA|1999-10-28|2001-10-26|Then dirty solutions go also dealers; aware ingredients become then children. Results face instances. Incidents would let; adm|4.21|1.38|1002001|maxiunivamalg #6|2|televisions|10|Electronics|184|eseeingought|N/A|61079orchid358341526|thistle|Tbl|Unknown|36|ationcallypripriought| +13368|AAAAAAAAGDEDAAAA|2001-10-27||Then dirty solutions go also dealers; aware ingredients become then children. Results face instances. Incidents would let; adm|2.13|1.38|1002001|amalgimporto #1|2|accessories|2|Men|579|n stationanti|medium|61079orchid358341526|steel|Carton|Unknown|10|eingcallypripriought| +13369|AAAAAAAAJDEDAAAA|1997-10-27||Ro|3.69|1.69|7016010|corpnameless #10|16|furniture|7|Home|332|ablepripri|N/A|717412364610orchid38|smoke|Oz|Unknown|13|n stcallypripriought| +13370|AAAAAAAAKDEDAAAA|1997-10-27|2000-10-26|Voters shall not go later simultaneously thin floors; good authorities can place quickly in the ways. Sole jobs anticipate; following, bright girls act rooms. New, various units cou|2.65|2.25|2003001|exportiimporto #1|3|pants|2|Men|456|callyantiese|medium|74492557473smoke4350|chiffon|Pallet|Unknown|60|barationpripriought| +13371|AAAAAAAAKDEDAAAA|2000-10-27||Chemicals used to matter quite. Independent democrats might see predominantly as substantial horses. Examples spend just products; ideal, scottish husbands wear n|1.91|2.25|1002002|importoamalg #2|3|fragrances|1|Women|678|callyantiese|medium|9792932metallic47734|purple|Each|Unknown|52|oughtationpripriought| +13372|AAAAAAAAMDEDAAAA|1997-10-27|1999-10-27|Bad, inte|5.47|3.93|1002001|importoamalg #1|2|fragrances|1|Women|336|callypripri|medium|781277632slate889396|white|Gross|Unknown|71|ableationpripriought| +13373|AAAAAAAAMDEDAAAA|1999-10-28|2001-10-26|Americans would not write new sources. Other, critical s|3.72|3.34|1002001|amalgnameless #4|2|athletic shoes|8|Sports|238|callypripri|N/A|781277632slate889396|tomato|Lb|Unknown|78|priationpripriought| +13374|AAAAAAAAMDEDAAAA|2001-10-27||Early, official books ought to look though relations. Common, real centres might allow other, large games. Also southern years ought to deliver; necessary winners cannot want difficult responses. |0.71|3.34|1002001|amalgscholar #1|2|rock|5|Music|237|ationpriable|N/A|781277632slate889396|sky|Carton|Unknown|25|eseationpripriought| +13375|AAAAAAAAPDEDAAAA|1997-10-27||Other workers lay very ancient, proposed examples. Houses could not make high elections. All favorite waters must live fairly political financial parties.|0.13|0.05|9012008|importounivamalg #8|12|home repair|9|Books|370|barationpri|N/A|3026365376snow877127|steel|Unknown|Unknown|70|antiationpripriought| +13376|AAAAAAAAAEEDAAAA|1997-10-27|2000-10-26|Chinese authorities should know painfully |2.68|1.20|1003001|exportiamalg #1|3|maternity|1|Women|528|eingableanti|economy|167turquoise80810095|pink|Bunch|Unknown|82|callyationpripriought| +13377|AAAAAAAAAEEDAAAA|2000-10-27||Close firms grow different activities. Broad men may manage wrong, disabled conclusions. Simply fierce men decide. Australian, smooth lists|5.80|4.35|1003001|importoexporti #2|3|infants|3|Children|270|barationable|petite|5607411472puff862863|peach|N/A|Unknown|55|ationationpripriought| +13378|AAAAAAAACEEDAAAA|1997-10-27|1999-10-27|As good rooms must react military months. Questions may not think after the books. Euro|6.71|2.75|10009017|maxiunivamalg #17|9|televisions|10|Electronics|318|eingoughtpri|N/A|280690031212996grey3|sandy|Oz|Unknown|55|eingationpripriought| +13379|AAAAAAAACEEDAAAA|1999-10-28|2001-10-26|Miles leave so soft, imperial problems. Things confer daily able years; centuries end. New, happy items would raise ago things. Existing, sheer feature|4.86|2.52|10009017|edu packbrand #4|9|estate|6|Jewelry|19|eingoughtpri|N/A|280690031212996grey3|tan|Dozen|Unknown|30|n stationpripriought| +13380|AAAAAAAACEEDAAAA|2001-10-27||Independent expectations will describe either by way of a authorities. Equations shall want especially with |0.40|0.12|8004001|edu packnameless #1|4|camping|8|Sports|19|n stought|N/A|280690031212996grey3|sandy|Pound|Unknown|10|bareingpripriought| +13381|AAAAAAAAFEEDAAAA|1997-10-27||Intentions benefit well now important reasons; massive tracks provide s|2.47|2.07|3004002|edu packexporti #2|4|school-uniforms|3|Children|176|callyationought|extra large|9569851590steel58328|sienna|Tbl|Unknown|24|oughteingpripriought| +13382|AAAAAAAAGEEDAAAA|1997-10-27|2000-10-26|Prospective, indirect years announce in particular from a situations. Days would depend now advisory police. As excellent females will build high more other years. Bad duties cannot stabili|2.05|0.75|8003003|exportinameless #3|3|basketball|8|Sports|54|eseanti|N/A|1841plum681955438989|turquoise|Dozen|Unknown|29|ableeingpripriought| +13383|AAAAAAAAGEEDAAAA|2000-10-27||Sites must use on|0.58|0.19|8003003|univmaxi #4|3|pools|8|Sports|54|eseanti|N/A|92wheat9958893950484|pink|Carton|Unknown|21|prieingpripriought| +13384|AAAAAAAAIEEDAAAA|1997-10-27|1999-10-27|Well interesting symbols receive scenes. Especially equal communities ought to listen directly by a words; following, dramatic c|1.55|0.97|7009005|maxibrand #5|9|mattresses|7|Home|220|barableable|N/A|268537seashell192689|salmon|Dozen|Unknown|31|eseeingpripriought| +13385|AAAAAAAAIEEDAAAA|1999-10-28|2001-10-26|Well interesting symbols receive scenes. Especially equal communities ought to listen directly by a words; following, dramatic c|3.54|2.08|10008007|namelessunivamalg #7|9|scanners|10|Electronics|76|barableable|N/A|983760906921lawn2942|violet|Box|Unknown|15|antieingpripriought| +13386|AAAAAAAAIEEDAAAA|2001-10-27||Well interesting symbols receive scenes. Especially equal communities ought to listen directly by a words; following, dramatic c|4.23|2.15|7009007|maxibrand #7|9|mattresses|7|Home|76|callyation|N/A|8140461890gainsboro2|violet|Lb|Unknown|16|callyeingpripriought| +13387|AAAAAAAALEEDAAAA|1997-10-27||Centres will serve american, accurate variables. Members give near in a measures. Head homes will not come serious, clear areas. More true principles dismiss specifically per a p|7.54|2.78|8003008|exportinameless #8|3|basketball|8|Sports|297|ationn stable|N/A|604seashell608089709|tomato|Each|Unknown|15|ationeingpripriought| +13388|AAAAAAAAMEEDAAAA|1997-10-27|2000-10-26|Popular unions read united, recent initiatives. Thanks know exact recommendations. Hot, low laws |68.46|60.24|2001001|amalgimporto #1|1|accessories|2|Men|333|pripripri|small|705snow6926719265315|aquamarine|Dram|Unknown|66|eingeingpripriought| +13389|AAAAAAAAMEEDAAAA|2000-10-27||Popular unions read united, recent initiatives. Thanks know exact recommendations. Hot, low laws |2.16|1.77|7010010|univnameless #10|1|flatware|7|Home|333|pripripri|N/A|705snow6926719265315|white|Pound|Unknown|67|n steingpripriought| +13390|AAAAAAAAOEEDAAAA|1997-10-27|1999-10-27|Twice ready fears w|7.21|6.12|8010001|univmaxi #1|10|pools|8|Sports|215|antioughtable|N/A|162638navajo58206994|orange|Ounce|Unknown|6|barn stpripriought| +13391|AAAAAAAAOEEDAAAA|1999-10-28|2001-10-26|Quarter|1.53|0.58|3004002|edu packexporti #2|4|school-uniforms|3|Children|215|antioughtable|small|9066slate88328873533|chiffon|Oz|Unknown|85|oughtn stpripriought| +13392|AAAAAAAAOEEDAAAA|2001-10-27||Pages shall check late|5.91|3.48|6004001|edu packcorp #1|4|bracelets|6|Jewelry|557|antioughtable|N/A|9066slate88328873533|blue|Case|Unknown|45|ablen stpripriought| +13393|AAAAAAAABFEDAAAA|1997-10-27||Inevitable plants light in a appointments. Important, bold pictures mea|44.33|36.79|8006004|corpnameless #4|6|football|8|Sports|255|antiantiable|N/A|677879221peach995189|lavender|N/A|Unknown|31|prin stpripriought| +13394|AAAAAAAACFEDAAAA|1997-10-27|2000-10-26|Short, existing activities cannot know too various masses; european, french lists reach by a details. Laws think only. American leaders must not exclude young types. White, dead banks m|9.55|4.01|4004001|edu packedu pack #1|4|athletic|4|Shoes|508|eingbaranti|large|2405272542coral61454|lawn|Dozen|Unknown|60|esen stpripriought| +13395|AAAAAAAACFEDAAAA|2000-10-27||Adequate|2.80|4.01|10006017|corpunivamalg #17|4|musical|10|Electronics|463|eingbaranti|N/A|2405272542coral61454|puff|Pallet|Unknown|34|antin stpripriought| +13396|AAAAAAAAEFEDAAAA|1997-10-27|1999-10-27|New users shall make techniques. Even radical initiatives lead strict babies. Ways will not think more once local stars. T|3.29|1.44|5004001|edu packscholar #1|4|classical|5|Music|929|n stablen st|N/A|826677230389snow9700|smoke|Pound|Unknown|44|callyn stpripriought| +13397|AAAAAAAAEFEDAAAA|1999-10-28|2001-10-26|Efforts believe in order in t|1.39|1.05|5004001|importoamalg #2|2|fragrances|1|Women|707|n stablen st|petite|826677230389snow9700|yellow|Dram|Unknown|12|ationn stpripriought| +13398|AAAAAAAAEFEDAAAA|2001-10-27||Efforts believe in order in t|4.63|3.33|9005007|scholarmaxi #7|5|history|9|Books|707|n stablen st|N/A|826677230389snow9700|tomato|Box|Unknown|19|eingn stpripriought| +13399|AAAAAAAAHFEDAAAA|1997-10-27||At least leading stories ought to use. Modules should not seem please either common rules. Different |2.73|1.69|3004002|edu packexporti #2|4|school-uniforms|3|Children|292|ablen stable|petite|38682sky298122017146|blush|Gram|Unknown|29|n stn stpripriought| +13400|AAAAAAAAIFEDAAAA|1997-10-27|2000-10-26|Scottish men|6.38|3.50|10009005|maxiunivamalg #5|9|televisions|10|Electronics|240|bareseable|N/A|4894016spring9386731|khaki|Cup|Unknown|30|barbaresepriought| +13401|AAAAAAAAIFEDAAAA|2000-10-27||Rules start social strangers. As other roles shall see possible, pleasant parts. International roses hit. Friends bec|9.09|3.50|10009005|importoimporto #2|2|shirts|2|Men|499|n stn stese|petite|4894016spring9386731|smoke|Box|Unknown|45|oughtbaresepriought| +13402|AAAAAAAAKFEDAAAA|1997-10-27|1999-10-27|Now independent birds lend enough; yards go as |0.12|0.05|10002007|importounivamalg #7|2|camcorders|10|Electronics|91|oughtn st|N/A|586010483seashell030|drab|Dozen|Unknown|72|ablebaresepriought| +13403|AAAAAAAAKFEDAAAA|1999-10-28|2001-10-26|Circumstances ought to|2.70|1.02|10002007|maxiunivamalg #8|9|televisions|10|Electronics|510|baroughtanti|N/A|586010483seashell030|yellow|Bunch|Unknown|36|pribaresepriought| +13404|AAAAAAAAKFEDAAAA|2001-10-27||Only fresh members sound meanwhile often dreadful responses; premises shall finish too with a versions; so executive findings give. Children make dif|4.31|1.02|7016005|corpnameless #5|16|furniture|7|Home|510|baroughtanti|N/A|586010483seashell030|lime|Cup|Unknown|9|esebaresepriought| +13405|AAAAAAAANFEDAAAA|1997-10-27||Public prices should|95.68|84.19|4004002|edu packedu pack #2|4|athletic|4|Shoes|600|barbarcally|large|metallic004088997567|plum|Bunch|Unknown|28|antibaresepriought| +13406|AAAAAAAAOFEDAAAA|1997-10-27|2000-10-26|Standards feel over young arts. Various customers suit just alive, original students. Very, good agents could drive once local, other troops. Below automatic oc|34.76|20.85|9009003|maximaxi #3|9|science|9|Books|400|barbarese|N/A|90salmon025134051136|snow|Cup|Unknown|44|callybaresepriought| +13407|AAAAAAAAOFEDAAAA|2000-10-27||Standards feel over young arts. Various customers suit just alive, original students. Very, good agents could drive once local, other troops. Below automatic oc|6.02|2.28|3003002|exportiexporti #2|3|toddlers|3|Children|400|barbarese|medium|5smoke88039304843335|firebrick|Ton|Unknown|17|ationbaresepriought| +13408|AAAAAAAAAGEDAAAA|1997-10-27|1999-10-27|Statements continue here academic members; certain students kill apparently social, available l|1.64|0.55|8011009|amalgmaxi #9|11|archery|8|Sports|396|callyn stpri|N/A|474652indian50673076|rosy|Unknown|Unknown|35|eingbaresepriought| +13409|AAAAAAAAAGEDAAAA|1999-10-28|2001-10-26|Far, neighbouring notes read ver|8.33|0.55|8011009|scholarcorp #2|11|earings|6|Jewelry|396|callyn stpri|N/A|86ivory8045353216840|saddle|Case|Unknown|21|n stbaresepriought| +13410|AAAAAAAAAGEDAAAA|2001-10-27||Far, neighbouring notes read ver|3.48|0.55|8011009|exportinameless #1|13|wallpaper|7|Home|396|callyn stpri|N/A|8741876593cyan711508|yellow|Box|Unknown|16|baroughtesepriought| +13411|AAAAAAAADGEDAAAA|1997-10-27||Small eyes can support significant, electronic scores; boots can stand. Urgent, noble rats wear however. Exchanges might give trades.|3.21|1.47|4002002|importoedu pack #2|2|mens|4|Shoes|110|baroughtought|medium|184398829586papaya50|tomato|Gross|Unknown|19|oughtoughtesepriought| +13412|AAAAAAAAEGEDAAAA|1997-10-27|2000-10-26|Styles used to get inade|52.50|45.15|4004001|edu packedu pack #1|4|athletic|4|Shoes|257|ationantiable|petite|6hot1068813977815805|steel|Pound|Unknown|8|ableoughtesepriought| +13413|AAAAAAAAEGEDAAAA|2000-10-27||Styles used to get inade|95.24|45.15|2003002|exportiimporto #2|4|pants|2|Men|257|ationantiable|medium|6hot1068813977815805|royal|Tsp|Unknown|32|prioughtesepriought| +13414|AAAAAAAAGGEDAAAA|1997-10-27|1999-10-27|Fairly national methods could lead only yards. Crucial, personal sto|0.32|0.23|8007005|brandnameless #5|7|hockey|8|Sports|318|eingoughtpri|N/A|2632649grey180439728|sienna|Gross|Unknown|30|eseoughtesepriought| +13415|AAAAAAAAGGEDAAAA|1999-10-28|2001-10-26|Connections can feel rather old, good proposals. Cases can take in the hills. Public, grey firms know either outer, financial enemies. Hundreds coul|8.93|0.23|4003002|exportiedu pack #2|3|kids|4|Shoes|318|eingoughtpri|medium|268366138turquoise96|grey|Unknown|Unknown|29|antioughtesepriought| +13416|AAAAAAAAGGEDAAAA|2001-10-27||For example european explanations reach. Difficult, like days break reasonably other, separate arms; examples shall stick sometimes particular markets. Jobs curl m|4.72|2.69|3003001|exportiexporti #1|3|toddlers|3|Children|52|ableanti|medium|543azure300485402519|honeydew|Tsp|Unknown|8|callyoughtesepriought| +13417|AAAAAAAAJGEDAAAA|1997-10-27||Therefore free conditions ride actually upper worlds. Tomorrow d|92.26|53.51|2002002|importoimporto #2|2|shirts|2|Men|80|bareing|economy|914905yellow75637196|purple|Cup|Unknown|5|ationoughtesepriought| +13418|AAAAAAAAKGEDAAAA|1997-10-27|2000-10-26|Happy, essential emotions offer at once exotic activities. Other r|8.30|4.23|7014009|edu packnameless #9|14|glassware|7|Home|526|callyableanti|N/A|146742133515papaya89|peach|Ounce|Unknown|11|eingoughtesepriought| +13419|AAAAAAAAKGEDAAAA|2000-10-27||Happy, essential emotions offer at once exotic activities. Other r|6.73|5.72|7014009|amalgexporti #2|14|newborn|3|Children|224|eseableable|small|094420541royal510079|powder|Dozen|Unknown|38|n stoughtesepriought| +13420|AAAAAAAAMGEDAAAA|1997-10-27|1999-10-27|Hot, simple lands sh|0.72|0.37|2001001|amalgimporto #1|1|accessories|2|Men|191|oughtn stought|medium|52521872238784lemon4|turquoise|Bunch|Unknown|46|barableesepriought| +13421|AAAAAAAAMGEDAAAA|1999-10-28|2001-10-26|Hot, simple lands sh|2.51|0.37|2001001|exportiunivamalg #6|13|self-help|9|Books|145|antieseought|N/A|709294252124violet35|rose|Oz|Unknown|26|oughtableesepriought| +13422|AAAAAAAAMGEDAAAA|2001-10-27||Single, clear sales approve roughly brief, real hours. Now dead contributions become further arbitrarily substantial interests. Little, nec|3.05|2.07|8005003|scholarnameless #3|5|fitness|8|Sports|145|antieseought|N/A|009325turquoise90498|smoke|Pound|Unknown|8|ableableesepriought| +13423|AAAAAAAAPGEDAAAA|1997-10-27||Soldiers might appear even to the clothes. Sensible forces please curre|9.12|8.11|9009008|maximaxi #8|9|science|9|Books|237|ationpriable|N/A|4037snow203338691114|burnished|Tsp|Unknown|48|priableesepriought| +13424|AAAAAAAAAHEDAAAA|1997-10-27|2000-10-26|Imports shall not change downwards however other years. So complete others can treat bodies. About alive ter|4.70|3.10|1004001|edu packamalg #1|4|swimwear|1|Women|958|eingantin st|extra large|5133sky0311668709547|lime|Dozen|Unknown|50|eseableesepriought| +13425|AAAAAAAAAHEDAAAA|||Young, psychological cases hang jobs; other varieties must represent t|3.33||10014013||14|automotive|10|Electronics|958||N/A|00slate7481584288220||Oz|Unknown|24|antiableesepriought| +13426|AAAAAAAACHEDAAAA|1997-10-27|1999-10-27|Personal rights used to admit. Feet must offer. Then hot enterprises would not include practices. Essential, limited words will|5.91|3.78|7006007|corpbrand #7|6|rugs|7|Home|189|n steingought|N/A|8824514519736bisque2|spring|Lb|Unknown|7|callyableesepriought| +13427|AAAAAAAACHEDAAAA|1999-10-28|2001-10-26|Past, visual police could happen just goods. Tracks bear days. Only leading legs will not take just desperate, limited children. Hig|6.97|4.87|7006007|amalgamalg #2|6|dresses|1|Women|171|n steingought|small|717746salmon02062497|sandy|Box|Unknown|4|ationableesepriought| +13428|AAAAAAAACHEDAAAA|2001-10-27||Past, visual police could happen just goods. Tracks bear days. Only leading legs will not take just desperate, limited children. Hig|38.40|16.12|7006007|importomaxi #9|6|guns|8|Sports|149|n steseought|N/A|717746salmon02062497|midnight|Bundle|Unknown|35|eingableesepriought| +13429|AAAAAAAAFHEDAAAA|1997-10-27||Burning years would handle additional churches. Able planes r|9.01|5.22|10003017|exportiunivamalg #17|3|dvd/vcr players|10|Electronics|350|barantipri|N/A|786915677royal953800|sandy|Ton|Unknown|30|n stableesepriought| +13430|AAAAAAAAGHEDAAAA|1997-10-27|2000-10-26|Visible women organize other problems. Players know almost psychol|7.94|6.66|1004001|edu packamalg #1|4|swimwear|1|Women|20|barable|small|179orchid56331748191|midnight|Bunch|Unknown|66|barpriesepriought| +13431|AAAAAAAAGHEDAAAA|2000-10-27||Ultimately military proposals might not see local, available cases. Very new books would blo|36.62|6.66|1004001|edu packedu pack #2|4|athletic|4|Shoes|20|barable|medium|78809397lavender6728|slate|Dozen|Unknown|7|oughtpriesepriought| +13432|AAAAAAAAIHEDAAAA|1997-10-27|1999-10-27|Northern, possible courses would admit. National arms conduct times. Attractive, operational comparisons worry studies. At leas|6.98|3.07|7001003|amalgbrand #3|1|bathroom|7|Home|545|antieseanti|N/A|609240008mint5199364|thistle|Oz|Unknown|30|ablepriesepriought| +13433|AAAAAAAAIHEDAAAA|1999-10-28|2001-10-26|Northern, possible courses would admit. National arms conduct times. Attractive, operational comparisons worry studies. At leas|8.31|3.07|7009004|maxibrand #4|9|mattresses|7|Home|545|antieseanti|N/A|214626turquoise20403|spring|Box|Unknown|1|pripriesepriought| +13434|AAAAAAAAIHEDAAAA|2001-10-27||Northern, possible courses would admit. National arms conduct times. Attractive, operational comparisons worry studies. At leas|2.25|1.21|7009004|exportiexporti #1|3|toddlers|3|Children|235|antipriable|small|7cornsilk96473687982|rosy|Lb|Unknown|6|esepriesepriought| +13435|AAAAAAAALHEDAAAA|1997-10-27||Relevant quantities credit respectively single reasons; gentle, white ends devise then frequent centuries. Free, easy authorities say prime, good parts. Military, written children know already eff|2.38|2.07|10002006|importounivamalg #6|2|camcorders|10|Electronics|462|ablecallyese|N/A|49251884286520slate2|khaki|Box|Unknown|26|antipriesepriought| +13436|AAAAAAAAMHEDAAAA|1997-10-27|2000-10-26|Drugs know slowly aw|0.62|0.22|3003001|exportiexporti #1|3|toddlers|3|Children|412|ableoughtese|petite|42396red661230188861|powder|Gram|Unknown|17|callypriesepriought| +13437|AAAAAAAAMHEDAAAA|2000-10-27||Likely, patient numb|5.45|4.85|3003001|edu packbrand #6|3|estate|6|Jewelry|412|ableoughtese|N/A|6770615461lime703352|grey|Tsp|Unknown|10|ationpriesepriought| +13438|AAAAAAAAOHEDAAAA|1997-10-27|1999-10-27|Good, economic solicitors protect ill, new terms. More scientific profits might depict upstairs even progressive visitors; small results might not meet less girls. Only voluntary rates could inte|3.29|1.34|5002001|importoscholar #1|2|country|5|Music|335|antipripri|N/A|447998361powder84764|sienna|Oz|Unknown|34|eingpriesepriought| +13439|AAAAAAAAOHEDAAAA|1999-10-28|2001-10-26|Good, economic solicitors protect ill, new terms. More scientific profits might depict upstairs even progressive visitors; small results might not meet less girls. Only voluntary rates could inte|4.88|1.34|10009010|maxiunivamalg #10|2|televisions|10|Electronics|335|antipripri|N/A|66150sienna671440985|salmon|Each|Unknown|14|n stpriesepriought| +13440|AAAAAAAAOHEDAAAA|2001-10-27||Sure companies might not emphasize other places. Previously irish costs used to learn well frequently public ages; recent friends can reform together standard churches. U|3.94|2.75|10009010|edu packamalg #1|4|swimwear|1|Women|335|antipripri|large|66150sienna671440985|papaya|Case|Unknown|20|bareseesepriought| +13441|AAAAAAAABIEDAAAA|1997-10-27||Free periods discourage then old measures. Activities stick brown, small eyes. Professional, significant interests ought to exist short, evident materials. Real, essential years |5.98|4.78|5001002|amalgscholar #2|1|rock|5|Music|369|n stcallypri|N/A|7674528238powder0379|smoke|Tbl|Unknown|54|oughteseesepriought| +13442|AAAAAAAACIEDAAAA|1997-10-27|2000-10-26|Only daily men used to provide usually remote good periods; references would finish physical, angry positions. Popular, natural problems ought to miss following needs. Dreadful, pure |9.98|8.98|3002001|importoexporti #1|2|infants|3|Children|462|ablecallyese|petite|80830434tan447054210|navajo|Gram|Unknown|8|ableeseesepriought| +13443|AAAAAAAACIEDAAAA|2000-10-27||Only daily men used to provide usually remote good periods; references would finish physical, angry positions. Popular, natural problems ought to miss following needs. Dreadful, pure |9.62|8.36|3002001|brandunivamalg #14|2|personal|10|Electronics|84|eseeing|N/A|12895188grey58046360|sienna|Each|Unknown|5|prieseesepriought| +13444|AAAAAAAAEIEDAAAA|1997-10-27|1999-10-27|Faintl|6.34|5.38|6002005|importocorp #5|2|diamonds|6|Jewelry|215|antioughtable|N/A|tomato87674435668407|royal|Dozen|Unknown|29|eseeseesepriought| +13445|AAAAAAAAEIEDAAAA|1999-10-28|2001-10-26|Faintl|5.32|5.38|10003016|exportiunivamalg #16|3|dvd/vcr players|10|Electronics|951|antioughtable|N/A|5850tomato4690536384|pale|Cup|Unknown|62|antieseesepriought| +13446|AAAAAAAAEIEDAAAA|2001-10-27||Faintl|2.45|1.91|10003016|importoamalg #1|2|fragrances|1|Women|951|antioughtable|large|131saddle44824274443|thistle|Bundle|Unknown|1|callyeseesepriought| +13447|AAAAAAAAHIEDAAAA|1997-10-27||Now french pressures could not die however times. Legal, soviet boys may go. Eyes emphasise notably regional reasons|1.35|0.74|3004002|edu packexporti #2|4|school-uniforms|3|Children|964|esecallyn st|extra large|22787820sky939323782|papaya|Bundle|Unknown|14|ationeseesepriought| +13448|AAAAAAAAIIEDAAAA|1997-10-27|2000-10-26|Less right powers come fast on a writers. Particularly different numbers cannot tackle personal, top studies. Women can want early inherent, british streets. Soon young card|1.45|1.20|7008009|namelessbrand #9|8|lighting|7|Home|431|oughtpriese|N/A|6865579white72598805|red|Gross|Unknown|64|eingeseesepriought| +13449|AAAAAAAAIIEDAAAA|2000-10-27||Less right powers come fast on a writers. Particularly different numbers cannot tackle personal, top studies. Women can want early inherent, british streets. Soon young card|4.23|3.13|7008009|importoamalgamalg #3|8|monitors|10|Electronics|470|barationese|N/A|6865579white72598805|wheat|Dozen|Unknown|41|n steseesepriought| +13450|AAAAAAAAKIEDAAAA|1997-10-27|1999-10-27|Services support forces. Increased me|0.14|0.09|2002001|importoimporto #1|2|shirts|2|Men|410|baroughtese|extra large|honeydew262325241259|magenta|Box|Unknown|59|barantiesepriought| +13451|AAAAAAAAKIEDAAAA|1999-10-28|2001-10-26|Services support forces. Increased me|4.82|0.09|7001002|amalgbrand #2|1|bathroom|7|Home|563|baroughtese|N/A|honeydew262325241259|pink|Lb|Unknown|6|oughtantiesepriought| +13452|AAAAAAAAKIEDAAAA|2001-10-27||Services support forces. Increased me|7.76|2.48|8014003|edu packmaxi #3|14|tennis|8|Sports|563|baroughtese|N/A|honeydew262325241259|sienna|Bundle|Unknown|66|ableantiesepriought| +13453|AAAAAAAANIEDAAAA|1997-10-27||Top studies rely still. Splendid, unacceptable stars |3.32|2.75|5003002|exportischolar #2|3|pop|5|Music|132|ablepriought|N/A|3856202320almond5359|midnight|Tsp|Unknown|4|priantiesepriought| +13454|AAAAAAAAOIEDAAAA|1997-10-27|2000-10-26|Only clear letters begin particularly nearly positive kinds. Private, necessary children might obtain at a servi|6.25|3.87|6011007|amalgbrand #7|11|semi-precious|6|Jewelry|510|baroughtanti|N/A|9grey506742287890302|maroon|Carton|Unknown|22|eseantiesepriought| +13455|AAAAAAAAOIEDAAAA|2000-10-27||Only clear letters begin particularly nearly positive kinds. Private, necessary children might obtain at a servi|6.90|2.62|6011007|importoimporto #2|11|shirts|2|Men|592|ablen stanti|medium|9grey506742287890302|sienna|Carton|Unknown|57|antiantiesepriought| +13456|AAAAAAAAAJEDAAAA|1997-10-27|1999-10-27|Also different others might take great, only problems. Then i|1.32|0.39|7012007|importonameless #7|12|paint|7|Home|442|ableeseese|N/A|007820007725spring42|seashell|Dram|Unknown|30|callyantiesepriought| +13457|AAAAAAAAAJEDAAAA|1999-10-28|2001-10-26|Also different others might take great, only problems. Then i|16.07|4.82|2004002|edu packimporto #2|4|sports-apparel|2|Men|49|n stese|petite|007820007725spring42|steel|Pallet|Unknown|52|ationantiesepriought| +13458|AAAAAAAAAJEDAAAA|2001-10-27||Huge, able children thank applications. Today historical men take for a boys. Well dry words worry still suf|7.99|4.71|6012003|importobrand #3|4|costume|6|Jewelry|377|ationationpri|N/A|4turquoise1907811375|navy|Ounce|Unknown|41|eingantiesepriought| +13459|AAAAAAAADJEDAAAA|1997-10-27||Yet vulnerable values choose especially general engines. Typical directors continue certain findings. Good devel|0.78|0.39|5004002|edu packscholar #2|4|classical|5|Music|319|n stoughtpri|N/A|757721938maroon58312|smoke|Carton|Unknown|29|n stantiesepriought| +13460|AAAAAAAAEJEDAAAA|1997-10-27|2000-10-26|Once financial years fight totally now financial skills. Significant, crazy provisions feel into a railways. So-called jobs land only supplies. Re|8.79|5.97|7005001|scholarbrand #1|5|blinds/shades|7|Home|140|bareseought|N/A|319312995802grey6404|steel|Dozen|Unknown|30|barcallyesepriought| +13461|AAAAAAAAEJEDAAAA|2000-10-27||Once financial years fight totally now financial skills. Significant, crazy provisions feel into a railways. So-called jobs land only supplies. Re|14.47|8.97|4004002|edu packedu pack #2|4|athletic|4|Shoes|197|bareseought|large|87620rosy78985937519|dark|Carton|Unknown|21|oughtcallyesepriought| +13462|AAAAAAAAGJEDAAAA|1997-10-27|1999-10-27|All but huge buildings become then with a resources. Ideas see against the times. |8.06|3.22|10008016|namelessunivamalg #16|8|scanners|10|Electronics|612|ableoughtcally|N/A|656puff1679021471375|tan|Pound|Unknown|60|ablecallyesepriought| +13463|AAAAAAAAGJEDAAAA|1999-10-28|2001-10-26|All but huge buildings become then with a resources. Ideas see against the times. |1.95|1.48|3002002|importoexporti #2|8|infants|3|Children|833|ableoughtcally|petite|656puff1679021471375|peach|Pound|Unknown|11|pricallyesepriought| +13464|AAAAAAAAGJEDAAAA|2001-10-27||All but huge buildings become then with a resources. Ideas see against the times. |0.14|0.11|3002002|importocorp #1|2|diamonds|6|Jewelry|100|barbarought|N/A|656puff1679021471375|yellow|Ounce|Unknown|8|esecallyesepriought| +13465|AAAAAAAAJJEDAAAA|1997-10-27||Houses enter personal, strange lips. Religious, irrelevant hours would bear also. Naked, national miles rely lacking women. Foreign prob|4.02|2.57|2002002|importoimporto #2|2|shirts|2|Men|183|prieingought|extra large|28692plum28101976859|sienna|Dozen|Unknown|2|anticallyesepriought| +13466|AAAAAAAAKJEDAAAA|1997-10-27|2000-10-26|Steady, careful ways take with a persons. Old movements make instinctively unknown others. Risks present all new policies. Separately existing hearts will shoot above black, only stations. Sha|1.11|0.55|5004001|edu packscholar #1|4|classical|5|Music|370|barationpri|N/A|42102338433612navy65|olive|Ounce|Unknown|17|callycallyesepriought| +13467|AAAAAAAAKJEDAAAA|2000-10-27||Steady, careful ways take with a persons. Old movements make instinctively unknown others. Risks present all new policies. Separately existing hearts will shoot above black, only stations. Sha|3.57|0.55|1004002|edu packamalg #2|4|swimwear|1|Women|370|barationpri|petite|682100954682salmon66|red|Unknown|Unknown|27|ationcallyesepriought| +13468|AAAAAAAAMJEDAAAA|1997-10-27|1999-10-27|Simply national events would not survive creative, full plans. Short, able affairs should seem so apart fine directors. Crude fields cut certainly interactions. More than left workers may n|1.36|0.50|8007009|brandnameless #9|7|hockey|8|Sports|316|callyoughtpri|N/A|10turquoise584334866|goldenrod|Dram|Unknown|35|eingcallyesepriought| +13469|AAAAAAAAMJEDAAAA|1999-10-28|2001-10-26|Simply national events would not survive creative, full plans. Short, able affairs should seem so apart fine directors. Crude fields cut certainly interactions. More than left workers may n|4.38|0.50|8007009|edu packscholar #2|4|classical|5|Music|316|callyoughtpri|N/A|4lawn215098652766391|turquoise|Unknown|Unknown|7|n stcallyesepriought| +13470|AAAAAAAAMJEDAAAA|2001-10-27||Simply national events would not survive creative, full plans. Short, able affairs should seem so apart fine directors. Crude fields cut certainly interactions. More than left workers may n|2.72|2.23|8014001|edu packmaxi #1|14|tennis|8|Sports|397|ationn stpri|N/A|4lawn215098652766391|metallic|Carton|Unknown|25|barationesepriought| +13471|AAAAAAAAPJEDAAAA|1997-10-27||Late ways shall thank even days. Real sports used to write virtually amazing roses. Socialist, civi|1.34|1.13|5002002|importoscholar #2|2|country|5|Music|569|n stcallyanti|N/A|9208773263rose362297|smoke|Oz|Unknown|41|oughtationesepriought| +13472|AAAAAAAAAKEDAAAA|1997-10-27|2000-10-26|Original, large kinds suit|9.86|6.60|9015009|scholarunivamalg #9|15|fiction|9|Books|440|bareseese|N/A|4786slate28499681470|tan|Carton|Unknown|17|ableationesepriought| +13473|AAAAAAAAAKEDAAAA|2000-10-27||Original, large kinds suit|3.17|6.60|1002002|importoamalg #2|2|fragrances|1|Women|440|bareseese|small|1760145615purple8228|thistle|N/A|Unknown|82|priationesepriought| +13474|AAAAAAAACKEDAAAA|1997-10-27|1999-10-27|So full strangers may demonstrate still ridiculous low legs; similar, colonial eyes accept so contemporary speakers. Otherwise soviet measures rely depth|23.57|8.01|1004001|edu packamalg #1|4|swimwear|1|Women|279|n stationable|petite|71127red717505755261|hot|Oz|Unknown|6|eseationesepriought| +13475|AAAAAAAACKEDAAAA|1999-10-28|2001-10-26|Hard marginal ears entertain for example. Only actions may produce often changes. Names hear. Formally physical results need with a descriptions|0.37|8.01|1004001|brandnameless #6|4|hockey|8|Sports|264|esecallyable|N/A|pink8728327721920577|white|Bundle|Unknown|31|antiationesepriought| +13476|AAAAAAAACKEDAAAA|2001-10-27||Hard marginal ears entertain for example. Only actions may produce often changes. Names hear. Formally physical results need with a descriptions|5.38|8.01|1004001|amalgscholar #1|4|rock|5|Music|95|esecallyable|N/A|59232369bisque351665|light|Ounce|Unknown|21|callyationesepriought| +13477|AAAAAAAAFKEDAAAA|1997-10-27||Top, long developments shall try relationships; national, ancient inhabitants should stop other contents. Only american homes may sell concerned, thin thanks. Mas|5.24|1.99|1004002|edu packamalg #2|4|swimwear|1|Women|131|oughtpriought|large|089987168navy8627298|saddle|Ton|Unknown|17|ationationesepriought| +13478|AAAAAAAAGKEDAAAA|1997-10-27|2000-10-26|Reforms might create generally french fingers. New, other flowers win then red, perfect thoughts. Most present sessions may go as only, genuine states. Years w|7.98|5.74|8002009|importonameless #9|2|baseball|8|Sports|3|pri|N/A|58588491280seashell3|thistle|Bundle|Unknown|5|eingationesepriought| +13479|AAAAAAAAGKEDAAAA|2000-10-27||Reforms might create generally french fingers. New, other flowers win then red, perfect thoughts. Most present sessions may go as only, genuine states. Years w|4.56|3.10|8002009|edu packunivamalg #16|2|audio|10|Electronics|3|pri|N/A|58588491280seashell3|violet|Unknown|Unknown|14|n stationesepriought| +13480|AAAAAAAAIKEDAAAA|1997-10-27|1999-10-27|Recently brown families could seem especially final, old industries. Various modes may restore helplessly out of the patients. Relevant, |4.15|1.53|4001001|amalgedu pack #1|1|womens|4|Shoes|356|callyantipri|large|0661592036376yellow8|snow|Case|Unknown|27|bareingesepriought| +13481|AAAAAAAAIKEDAAAA|1999-10-28|2001-10-26|Organizations work probably. Material churches used to accept seconds; video-taped, secure subjects may spend good. Designs go reliable, known pupils. Politicians force in a|1.68|1.53|9013006|exportiunivamalg #6|1|self-help|9|Books|356|callyantipri|N/A|0661592036376yellow8|tomato|Cup|Unknown|4|oughteingesepriought| +13482|AAAAAAAAIKEDAAAA|2001-10-27||Organizations work probably. Material churches used to accept seconds; video-taped, secure subjects may spend good. Designs go reliable, known pupils. Politicians force in a|1.76|1.21|7009003|maxibrand #3|9|mattresses|7|Home|356|callyantipri|N/A|0661592036376yellow8|tomato|Dram|Unknown|36|ableeingesepriought| +13483|AAAAAAAALKEDAAAA|1997-10-27||Correct police use fairly final women; other, blue workers discern more applicants. Defensively amazing lines can|6.71|3.95|10008003|namelessunivamalg #3|8|scanners|10|Electronics|453|priantiese|N/A|99red834327274801987|slate|Lb|Unknown|7|prieingesepriought| +13484|AAAAAAAAMKEDAAAA|1997-10-27|2000-10-26|States provide better values. Massive backs will play just underneath relevant procedures. Invariably labour legs insert sti|1.75|0.57|9005009|scholarmaxi #9|5|history|9|Books|140|bareseought|N/A|4732turquoise9218939|tan|N/A|Unknown|47|eseeingesepriought| +13485|AAAAAAAAMKEDAAAA|2000-10-27||States provide better values. Massive backs will play just underneath relevant procedures. Invariably labour legs insert sti|9.30|2.88|9005009|exportiimporto #2|3|pants|2|Men|140|bareseought|medium|4732turquoise9218939|seashell|Tbl|Unknown|14|antieingesepriought| +13486|AAAAAAAAOKEDAAAA|1997-10-27|1999-10-27|New details cannot establish ever essential roots. Theories investigate then round sites. Friendly rounds p|3.02|1.20|5003001|exportischolar #1|3|pop|5|Music|217|ationoughtable|N/A|47680543100spring408|dodger|Bundle|Unknown|2|callyeingesepriought| +13487|AAAAAAAAOKEDAAAA|1999-10-28|2001-10-26|New details cannot establish ever essential roots. Theories investigate then round sites. Friendly rounds p|39.02|1.20|8013008|exportimaxi #8|3|sailing|8|Sports|289|n steingable|N/A|cornsilk578657449779|red|Carton|Unknown|2|ationeingesepriought| +13488|AAAAAAAAOKEDAAAA|2001-10-27||Resources cannot lay definitely at least rough businesses. Doubts must|6.97|6.20|8013008|importoscholar #1|3|country|5|Music|336|callypripri|N/A|cornsilk578657449779|turquoise|Pallet|Unknown|82|eingeingesepriought| +13489|AAAAAAAABLEDAAAA|1997-10-27||Trades like else factors. Approximately real goods shift. Able hands happen willingly deep arrangements. Narrow y|2.06|0.84|3001002|amalgexporti #2|1|newborn|3|Children|419|n stoughtese|small|287701082navy5664696|thistle|Gram|Unknown|16|n steingesepriought| +13490|AAAAAAAACLEDAAAA|1997-10-27|2000-10-26|Interested, physical parts ought to behave furthermore. Primary rates convert just fellow, possib|3.05|1.18|2002001|importoimporto #1|2|shirts|2|Men|525|antiableanti|economy|6574blanched97967702|tomato|Tsp|Unknown|8|barn stesepriought| +13491|AAAAAAAACLEDAAAA|2000-10-27||Interested, physical parts ought to behave furthermore. Primary rates convert just fellow, possib|1.70|0.52|2002001|amalgnameless #2|1|athletic shoes|8|Sports|525|antiableanti|N/A|6574blanched97967702|purple|Oz|Unknown|22|oughtn stesepriought| +13492|AAAAAAAAELEDAAAA|1997-10-27|1999-10-27|Used, total organisers |9.42|3.67|1004001|edu packamalg #1|4|swimwear|1|Women|729|n stableation|N/A|0389627sky8105181192|thistle|Dram|Unknown|13|ablen stesepriought| +13493|AAAAAAAAELEDAAAA|1999-10-28|2001-10-26|Used, total organisers |0.43|3.67|8004004|edu packnameless #4|4|camping|8|Sports|729|n stableation|N/A|215honeydew799907664|grey|Gram|Unknown|70|prin stesepriought| +13494|AAAAAAAAELEDAAAA|2001-10-27||Used, total organisers |0.53|3.67|9010007|univunivamalg #7|4|travel|9|Books|216|n stableation|N/A|215honeydew799907664|saddle|Lb|Unknown|8|esen stesepriought| +13495|AAAAAAAAHLEDAAAA|1997-10-27||Terrible, keen answers think seemingly other, certain guns. Famous, important changes may say alone years. Nearby general figures receive sure nations. Safe groups might see local companies. Busy|1.04|0.36|10005015|scholarunivamalg #15|5|karoke|10|Electronics|518|eingoughtanti|N/A|171475837tan93566451|black|N/A|Unknown|80|antin stesepriought| +13496|AAAAAAAAILEDAAAA|1997-10-27|2000-10-26|Rich layers can talk only by a places. Centres used to greet well. Red, established birds may |7.06|5.57|5001001|amalgscholar #1|1|rock|5|Music|326|callyablepri|N/A|29353851547steel5689|sky|Ounce|Unknown|26|callyn stesepriought| +13497|AAAAAAAAILEDAAAA|2000-10-27||Rules may clear tomorrow sure members. Easily foreign |58.14|5.57|5001001|edu packamalg #2|4|swimwear|1|Women|55|callyablepri|extra large|29353851547steel5689|sienna|Lb|Unknown|56|ationn stesepriought| +13498|AAAAAAAAKLEDAAAA|1997-10-27|1999-10-27|Slowly specific spots hope very alone studies. Local arms say then relations. Other arms find upper, alternative children. Then various fragments may not look. A bit logical users lead useful t|2.11|0.97|2002001|importoimporto #1|2|shirts|2|Men|66|callycally|medium|823155497050lemon773|powder|Ton|Unknown|24|eingn stesepriought| +13499|AAAAAAAAKLEDAAAA|1999-10-28|2001-10-26|Then huge differences can take tonight there coming rates. Prices respond about b|2.95|1.62|2002001|edu packscholar #2|4|classical|5|Music|66|callycally|N/A|825399spring39137494|peru|Tbl|Unknown|24|n stn stesepriought| +13500|AAAAAAAAKLEDAAAA|2001-10-27||Then huge differences can take tonight there coming rates. Prices respond about b|2.49|1.62|2002001|namelessnameless #1|4|outdoor|8|Sports|66|callycally|N/A|184079lemon381542643|sky|Oz|Unknown|75|barbarantipriought| +13501|AAAAAAAANLEDAAAA|1997-10-27||As personal figures hit inevitable, other passengers. Levels might expand most rather other jobs; reduced pages believe and so on futu|3.43|2.16|6012006|importobrand #6|12|costume|6|Jewelry|897|ationn steing|N/A|6046219powder6013719|aquamarine|Oz|Unknown|93|oughtbarantipriought| +13502|AAAAAAAAOLEDAAAA|1997-10-27|2000-10-26|Pregnant, new students go. Lawyers evolve abo|2.25|0.74|6011007|amalgbrand #7|11|semi-precious|6|Jewelry|151|oughtantiought|N/A|218822powder23993692|purple|Lb|Unknown|55|ablebarantipriought| +13503|AAAAAAAAOLEDAAAA|2000-10-27||Partly blue examples might not get for a times. British |11.58|0.74|6011007|amalgnameless #4|11|accent|7|Home|150|oughtantiought|N/A|218822powder23993692|turquoise|Gram|Unknown|42|pribarantipriought| +13504|AAAAAAAAAMEDAAAA|1997-10-27|1999-10-27|Details participate ins|0.78|0.36|10012007|importoamalgamalg #7|12|monitors|10|Electronics|310|baroughtpri|N/A|9420354304pink110860|snow|Pound|Unknown|42|esebarantipriought| +13505|AAAAAAAAAMEDAAAA|1999-10-28|2001-10-26|High, dead clients find. Video-taped women would regain by a chains. As public resources buy subsequently in a |3.94|0.36|10012007|edu packbrand #2|12|estate|6|Jewelry|607|ationbarcally|N/A|9420354304pink110860|lace|Dozen|Unknown|27|antibarantipriought| +13506|AAAAAAAAAMEDAAAA|2001-10-27||Besides social methods must come potentially detectives. Well national gates can revert nearby, possible procedures. Great colleagues say courses. Serious, guilty holes bring sections. White, |0.31|0.24|10012007|corpamalgamalg #9|12|wireless|10|Electronics|136|ationbarcally|N/A|85741283926wheat5628|snow|Box|Unknown|88|callybarantipriought| +13507|AAAAAAAADMEDAAAA|1997-10-27||Eyes might ge|1.79|0.78|8008008|namelessnameless #8|8|outdoor|8|Sports|270|barationable|N/A|039tan11884281095085|white|Box|Unknown|54|ationbarantipriought| +13508|AAAAAAAAEMEDAAAA|1997-10-27|2000-10-26|Distinctive, major eyes pursue as scottish successes. Good, |0.92|0.47|5003001|exportischolar #1|3|pop|5|Music|406|callybarese|N/A|947197981slate498197|yellow|Pallet|Unknown|17|eingbarantipriought| +13509|AAAAAAAAEMEDAAAA|2000-10-27||Distinctive, major eyes pursue as scottish successes. Good, |1.42|0.47|5003001|maxibrand #10|9|mattresses|7|Home|406|callybarese|N/A|07852100spring817827|salmon|Each|Unknown|43|n stbarantipriought| +13510|AAAAAAAAGMEDAAAA|1997-10-27|1999-10-27|Blind, real systems may not intervene even later real standards. Unnecessarily other others might clarify in a doors. Here catholic manager|3.81|2.05|8012001|importomaxi #1|12|guns|8|Sports|300|barbarpri|N/A|465purple95908765095|snow|Ton|Unknown|21|baroughtantipriought| +13511|AAAAAAAAGMEDAAAA|1999-10-28|2001-10-26|Empirical, dry remarks fetch apparently. Human, wrong points meet much bright nurses. Technical, dirty schools can ensure always other, lengthy children. So medical fla|7.19|2.05|6006008|corpcorp #8|12|rings|6|Jewelry|413|prioughtese|N/A|19836628714115rose74|goldenrod|N/A|Unknown|33|oughtoughtantipriought| +13512|AAAAAAAAGMEDAAAA|2001-10-27||Empirical, dry remarks fetch apparently. Human, wrong points meet much bright nurses. Technical, dirty schools can ensure always other, lengthy children. So medical fla|1.33|0.49|6006008|edu packimporto #1|12|sports-apparel|2|Men|413|prioughtese|petite|19836628714115rose74|rosy|Oz|Unknown|19|ableoughtantipriought| +13513|AAAAAAAAJMEDAAAA|1997-10-27||Personal, significant activities agree only by a couples. Elaborate aut|3.06|1.43|9002002|importomaxi #2|2|business|9|Books|65|antically|N/A|4478733873682saddle5|violet|Tsp|Unknown|2|prioughtantipriought| +13514|AAAAAAAAKMEDAAAA|1997-10-27|2000-10-26|Patterns laugh along no customers. Exercises live up|4.15|2.57|10008017|namelessunivamalg #17|8|scanners|10|Electronics|665|anticallycally|N/A|70red204861746998853|smoke|Carton|Unknown|30|eseoughtantipriought| +13515|AAAAAAAAKMEDAAAA|2000-10-27||Total houses ought to go competitive children. Criteria cannot persuade simply recent, dear things. Women take best for the meals. Subsequent, scottish decisions rise. Weste|8.21|2.57|10008017|amalgamalg #2|8|dresses|1|Women|665|anticallycally|medium|4747sandy54164120570|peru|Pallet|Unknown|40|antioughtantipriought| +13516|AAAAAAAAMMEDAAAA|1997-10-27|1999-10-27|Major, german meetings spend very generally political departme|9.39|5.44|2003001|exportiimporto #1|3|pants|2|Men|694|esen stcally|petite|518873726238rosy1339|frosted|Gram|Unknown|14|callyoughtantipriought| +13517|AAAAAAAAMMEDAAAA|1999-10-28|2001-10-26|Strong, developing roads make once and for all to a walls. Contemporary, necessary months observe again on the authorities. Appropriate citizens|9.90|5.44|2004002|edu packimporto #2|4|sports-apparel|2|Men|864|esen stcally|small|518873726238rosy1339|peach|Unknown|Unknown|5|ationoughtantipriought| +13518|AAAAAAAAMMEDAAAA|2001-10-27||Compatible, practical applications must go respectively important students. Detailed things would procure too. Only ord|39.52|5.44|2004002|exportiimporto #1|3|pants|2|Men|412|ableoughtese|medium|83099619pink25213027|linen|Tbl|Unknown|34|eingoughtantipriought| +13519|AAAAAAAAPMEDAAAA|1997-10-27||Divisions should get. Unions repay now centres. Dead, eastern americans|3.26|1.53|5004002|edu packscholar #2|4|classical|5|Music|20|barable|N/A|13953874310indian595|tan|Ounce|Unknown|72|n stoughtantipriought| +13520|AAAAAAAAANEDAAAA|1997-10-27|2000-10-26|Wrong men go a little maximum areas. Views look then n|7.23|5.92|4001001|amalgedu pack #1|1|womens|4|Shoes|217|ationoughtable|medium|25smoke5961555428414|puff|Box|Unknown|37|barableantipriought| +13521|AAAAAAAAANEDAAAA|2000-10-27||Cells might include almost economic, full mines. Years shoul|2.61|5.92|4001001|edu packedu pack #2|1|athletic|4|Shoes|576|ationoughtable|economy|22366661226royal3330|violet|Tsp|Unknown|89|oughtableantipriought| +13522|AAAAAAAACNEDAAAA|1997-10-27|1999-10-27|Particularly labour stores get farmers. Hence true records see rel|6.89|3.99|9001011|amalgmaxi #11|1|arts|9|Books|1|ought|N/A|804534tan33385473764|lace|Oz|Unknown|20|ableableantipriought| +13523|AAAAAAAACNEDAAAA|1999-10-28|2001-10-26|Upper years escape hard unlikely children; far good idea|8.52|5.87|9001011|scholarunivamalg #12|15|fiction|9|Books|938|ought|N/A|6210652antique770559|yellow|Cup|Unknown|90|priableantipriought| +13524|AAAAAAAACNEDAAAA|2001-10-27||Upper years escape hard unlikely children; far good idea|2.28|1.14|9001011|exportiunivamalg #10|15|dvd/vcr players|10|Electronics|220|barableable|N/A|288865013533yellow06|peach|Tsp|Unknown|32|eseableantipriought| +13525|AAAAAAAAFNEDAAAA|1997-10-27||Appropriate, useful abilities include. Modern decisions wait before the miles. Concerned facts extend however trying, old standards. Common, poli|2.55|2.16|2001002|amalgimporto #2|1|accessories|2|Men|248|eingeseable|extra large|40khaki2178502655463|lace|Case|Unknown|87|antiableantipriought| +13526|AAAAAAAAGNEDAAAA|1997-10-27|2000-10-26|Too international arts say other types. Slim, impressive difficulties think only on a systems. British, common companies shall not refuse good, late h|2.11|1.13|3001001|amalgexporti #1|1|newborn|3|Children|573|priationanti|medium|3033038spring6596899|honeydew|Bundle|Unknown|62|callyableantipriought| +13527|AAAAAAAAGNEDAAAA|2000-10-27||Sides know workers. Now useful activities would seek immediately closer major doors. Local, other revenues call alone for a rights. Any longer old friends assume so-called factors; rates must not arg|2.27|1.13|9011004|amalgunivamalg #4|11|cooking|9|Books|573|priationanti|N/A|55880445811peru18352|dodger|Dram|Unknown|44|ationableantipriought| +13528|AAAAAAAAINEDAAAA|1997-10-27|1999-10-27|Political cars screen before a resident|4.54|3.99|1004001|edu packamalg #1|4|swimwear|1|Women|89|n steing|medium|29687peach0161049875|steel|Cup|Unknown|27|eingableantipriought| +13529|AAAAAAAAINEDAAAA|1999-10-28|2001-10-26|Political cars screen before a resident|3.91|3.99|1004001|exportiimporto #2|3|pants|2|Men|41|n steing|large|29687peach0161049875|sky|Cup|Unknown|53|n stableantipriought| +13530|AAAAAAAAINEDAAAA|2001-10-27||Political cars screen before a resident|8.38|7.37|1004001|edu packedu pack #1|4|athletic|4|Shoes|25|antiable|large|58634202844167pale46|royal|Cup|Unknown|33|barpriantipriought| +13531|AAAAAAAALNEDAAAA|1997-10-27||Medical years run. All good dogs used to keep largely medical months. Relevant games may not m|3.44|2.54|2001002|amalgimporto #2|1|accessories|2|Men|119|n stoughtought|large|7461639370817plum388|pale|Pallet|Unknown|35|oughtpriantipriought| +13532|AAAAAAAAMNEDAAAA|1997-10-27|2000-10-26|Leaders fit mild, dry mechanisms. Hours might involve much weeks. Years help too over top pupils. Earlier other years will remain little schools. Topics |9.99|3.79|8001003|amalgnameless #3|1|athletic shoes|8|Sports|349|n stesepri|N/A|93520434351842pale85|purple|Carton|Unknown|28|ablepriantipriought| +13533|AAAAAAAAMNEDAAAA|2000-10-27||Women could explain even extremely native eyes. Chief re|41.69|3.79|8001003|scholarbrand #4|1|blinds/shades|7|Home|349|n stesepri|N/A|93520434351842pale85|puff|Each|Unknown|20|pripriantipriought| +13534|AAAAAAAAONEDAAAA|1997-10-27|1999-10-27|However civil lights surprise most. Suddenly likely cigare|5.81|2.49|10015003|scholaramalgamalg #3|15|portable|10|Electronics|611|oughtoughtcally|N/A|86960611mint52375214|steel|Each|Unknown|100|esepriantipriought| +13535|AAAAAAAAONEDAAAA|1999-10-28|2001-10-26|Often human angles remain completely on a areas. Other consequences might not discuss single, new dangers; quiet, strong papers can bring so businesses. Long-term, excellent documents may not giv|1.66|2.49|3002002|importoexporti #2|15|infants|3|Children|611|oughtoughtcally|large|tan41216174406209728|plum|Ton|Unknown|5|antipriantipriought| +13536|AAAAAAAAONEDAAAA|2001-10-27||Accounts run independent ships. Good, significant individuals might develop by the words. Already heavy functions woul|3.18|2.49|1002001|importoamalg #1|15|fragrances|1|Women|258|oughtoughtcally|medium|016642059medium10522|metallic|Gross|Unknown|57|callypriantipriought| +13537|AAAAAAAABOEDAAAA|1997-10-27||Ever spare ingredients ought to judge never p|3.12|1.87|4002002|importoedu pack #2|2|mens|4|Shoes|324|eseablepri|N/A|smoke618007930612346|seashell|Unknown|Unknown|99|ationpriantipriought| +13538|AAAAAAAACOEDAAAA|1997-10-27|2000-10-26|Married children used to decide more others. Waste, serious interests would get unreasonably symptoms. Full, private liabilities ought to continue ea|6.02|4.33|1002001|importoamalg #1|2|fragrances|1|Women|525|antiableanti|extra large|7778688saddle5746704|lace|Oz|Unknown|6|eingpriantipriought| +13539|AAAAAAAACOEDAAAA|2000-10-27||New requirements arise just upper colleagues; especially minor courses see bloody. Priva|8.63|6.81|1002001|corpnameless #10|6|football|8|Sports|308|antiableanti|N/A|7778688saddle5746704|papaya|Ounce|Unknown|98|n stpriantipriought| +13540|AAAAAAAAEOEDAAAA|1997-10-27|1999-10-27|Simply welcome scholars may mean with a forests. So european fingers run interesting, strategic holes. Great, correct stories should not like best so foreign specialists. Just fin|3.38|1.85|6001005|amalgcorp #5|1|birdal|6|Jewelry|85|antieing|N/A|9103405yellow1563777|papaya|Box|Unknown|10|bareseantipriought| +13541|AAAAAAAAEOEDAAAA|1999-10-28|2001-10-26|Territorial resources enquir|0.84|0.42|6001005|importoedu pack #2|2|mens|4|Shoes|85|antieing|extra large|91115636664misty1860|puff|Tbl|Unknown|43|oughteseantipriought| +13542|AAAAAAAAEOEDAAAA|2001-10-27||Territorial resources enquir|7.77|5.59|6001005|edu packnameless #3|14|glassware|7|Home|450|antieing|N/A|91115636664misty1860|grey|Tbl|Unknown|6|ableeseantipriought| +13543|AAAAAAAAHOEDAAAA|1997-10-27||Open patterns talk equally european, hard women. Thus yellow problems hammer surprisingly actions. Peasants los|1.17|0.85|10012012|importoamalgamalg #12|12|monitors|10|Electronics|961|oughtcallyn st|N/A|3273220134seashell92|chiffon|Ton|Unknown|25|prieseantipriought| +13544|AAAAAAAAIOEDAAAA|1997-10-27|2000-10-26|Politically aware publications must say at a roles. Weapons lear|2.08|1.35|6001001|amalgcorp #1|1|birdal|6|Jewelry|217|ationoughtable|N/A|29029994373616sandy6|steel|Ton|Unknown|15|eseeseantipriought| +13545|AAAAAAAAIOEDAAAA|2000-10-27||Labour, other affairs tell indeed so other paths.|3.09|2.65|6001001|corpbrand #6|6|rugs|7|Home|67|ationoughtable|N/A|29029994373616sandy6|royal|Dozen|Unknown|20|antieseantipriought| +13546|AAAAAAAAKOEDAAAA|1997-10-27|1999-10-27|Aside united experts fight royal, eastern standards. Materials ensure much trees. Images imagine compr|2.23|1.71|6016003|corpbrand #3|16|consignment|6|Jewelry|131|oughtpriought|N/A|13232405330smoke2882|sienna|Tbl|Unknown|1|callyeseantipriought| +13547|AAAAAAAAKOEDAAAA|1999-10-28|2001-10-26|Aside united experts fight royal, eastern standards. Materials ensure much trees. Images imagine compr|9.44|1.71|6016003|amalgedu pack #2|16|womens|4|Shoes|517|ationoughtanti|N/A|13232405330smoke2882|burlywood|Pallet|Unknown|21|ationeseantipriought| +13548|AAAAAAAAKOEDAAAA|2001-10-27||Aside united experts fight royal, eastern standards. Materials ensure much trees. Images imagine compr|2.76|2.48|2004001|edu packimporto #1|16|sports-apparel|2|Men|923|priablen st|medium|32papaya278956308589|tomato|Ton|Unknown|59|eingeseantipriought| +13549|AAAAAAAANOEDAAAA|1997-10-27||Essential, old fans crawl yet. For instance bright arrangements must not say for a patterns. Dead things replace americans; rates could not attack too. Othe|4.70|2.49|10011001|amalgamalgamalg #1|11|disk drives|10|Electronics|18|eingought|N/A|3299pink428058787724|red|Gross|Unknown|5|n steseantipriought| +13550|AAAAAAAAOOEDAAAA|1997-10-27|2000-10-26|Periods eat very different doors; serious problems shed then loose, western inches. Visit|0.83|0.38|3003001|exportiexporti #1|3|toddlers|3|Children|149|n steseought|extra large|329865seashell857123|mint|Oz|Unknown|85|barantiantipriought| +13551|AAAAAAAAOOEDAAAA|2000-10-27||Periods eat very different doors; serious problems shed then loose, western inches. Visit|4.40|0.38|3003001|brandcorp #8|3|pendants|6|Jewelry|149|n steseought|N/A|329865seashell857123|metallic|Ton|Unknown|24|oughtantiantipriought| +13552|AAAAAAAAAPEDAAAA|1997-10-27|1999-10-27|Famous, orthodox sales see tonight for a societies. Practical, necessary women volunteer. Conditions fight off a meetings. Only hands would move. Social sides begin fairly interesting, |2.66|1.14|10002004|importounivamalg #4|2|camcorders|10|Electronics|42|ableese|N/A|09907502166lemon3303|plum|Unknown|Unknown|23|ableantiantipriought| +13553|AAAAAAAAAPEDAAAA|1999-10-28|2001-10-26|New, expected possibilities will not determine with a areas. Great friends fall probably other elections; internal, professional conservatives respond. Early planes establish in |7.03|1.14|9001006|amalgmaxi #6|2|arts|9|Books|915|antioughtn st|N/A|09907502166lemon3303|misty|Gross|Unknown|13|priantiantipriought| +13554|AAAAAAAAAPEDAAAA|2001-10-27||New, expected possibilities will not determine with a areas. Great friends fall probably other elections; internal, professional conservatives respond. Early planes establish in |6.66|1.14|4001001|amalgedu pack #1|2|womens|4|Shoes|915|antioughtn st|small|09907502166lemon3303|red|Bunch|Unknown|20|eseantiantipriought| +13555|AAAAAAAADPEDAAAA|||Lines get often growing markets. Old schemes mu|||1002002|importoamalg #2||fragrances||Women||eingeingese||904thistle5926068040|salmon||Unknown|32|antiantiantipriought| +13556|AAAAAAAAEPEDAAAA|1997-10-27|2000-10-26|Below new weapons note small, good sections. Later new forms may strike years. Isolated, able critics use all but. Forces will not take then little records; windo|2.75|2.25|8009007|maxinameless #7|9|optics|8|Sports|250|barantiable|N/A|39919rose65458823261|slate|Dram|Unknown|64|callyantiantipriought| +13557|AAAAAAAAEPEDAAAA|2000-10-27||High, appropriate skills marry like a subjects. Necessary, bad motives find independent, primary efforts. Satisfactory, differe|2.72|1.22|8009007|amalgunivamalg #10|11|cooking|9|Books|979|n stationn st|N/A|39919rose65458823261|navajo|Unknown|Unknown|22|ationantiantipriought| +13558|AAAAAAAAGPEDAAAA|1997-10-27|1999-10-27|Almost comprehensive cases know unfortunately hard courses; there determined rules shall make even hard, close years. Existing, red sentences name. Experts help slowly players. |78.89|37.86|7014009|edu packnameless #9|14|glassware|7|Home|295|antin stable|N/A|63812376582pale22975|firebrick|Lb|Unknown|10|eingantiantipriought| +13559|AAAAAAAAGPEDAAAA|1999-10-28|2001-10-26|Almost comprehensive cases know unfortunately hard courses; there determined rules shall make even hard, close years. Existing, red sentences name. Experts help slowly players. |49.79|37.86|2001002|amalgimporto #2|1|accessories|2|Men|295|antin stable|extra large|63812376582pale22975|pale|Unknown|Unknown|86|n stantiantipriought| +13560|AAAAAAAAGPEDAAAA|2001-10-27||Generally long firms should want marginal, possible years. Labour visitors know together. Policies ought to make early, loyal services; public, old teacher|1.65|37.86|7013001|exportinameless #1|1|wallpaper|7|Home|159|antin stable|N/A|6897211853157pale443|steel|Lb|Unknown|30|barcallyantipriought| +13561|AAAAAAAAJPEDAAAA|1997-10-27||Social settlements can forget also at a managers; different, flexible countries will know even increasingly fina|7.62|3.50|4001002|amalgedu pack #2|1|womens|4|Shoes|189|n steingought|small|91smoke6278410815568|metallic|Carton|Unknown|5|oughtcallyantipriought| +13562|AAAAAAAAKPEDAAAA|1997-10-27|2000-10-26|Other, top words hurt visitors. Given neighbours cut in particular main, functional changes. Perhaps primary terms will devote later other, natural offi|1.63|1.22|7016003|corpnameless #3|16|furniture|7|Home|546|callyeseanti|N/A|51550bisque611890449|purple|Pound|Unknown|85|ablecallyantipriought| +13563|AAAAAAAAKPEDAAAA|2000-10-27||Other, top words hurt visitors. Given neighbours cut in particular main, functional changes. Perhaps primary terms will devote later other, natural offi|2.17|1.22|7016003|exportibrand #4|13|loose stones|6|Jewelry|239|n stpriable|N/A|51550bisque611890449|peach|Bundle|Unknown|5|pricallyantipriought| +13564|AAAAAAAAMPEDAAAA|1997-10-27|1999-10-27|Interested others may a|5.81|1.85|1001001|amalgamalg #1|1|dresses|1|Women|90|barn st|large|4806641314774peru780|linen|Pound|Unknown|13|esecallyantipriought| +13565|AAAAAAAAMPEDAAAA|1999-10-28|2001-10-26|Journals may note much difficult, literary diffe|7.13|1.85|4001002|amalgedu pack #2|1|womens|4|Shoes|90|barn st|small|46731wheat8972820532|mint|Bundle|Unknown|12|anticallyantipriought| +13566|AAAAAAAAMPEDAAAA|2001-10-27||Journals may note much difficult, literary diffe|6.44|2.18|4001002|exportiedu pack #1|1|kids|4|Shoes|404|barn st|large|14120sienna335904525|tan|Dozen|Unknown|66|callycallyantipriought| +13567|AAAAAAAAPPEDAAAA|1997-10-27||Most upper problems facilitate suddenly quite concerned paintings. Abstract prices get certainly social operations. Christian, other countries perform further democrats. Young teeth take high ey|12.95|11.65|1004002|edu packamalg #2|4|swimwear|1|Women|573|priationanti|medium|391129358706rosy8568|khaki|Dozen|Unknown|13|ationcallyantipriought| +13568|AAAAAAAAAAFDAAAA|1997-10-27|2000-10-26|Blue, old hands cannot |3.14|1.09|2001001|amalgimporto #1|1|accessories|2|Men|662|ablecallycally|large|59848418rose41277000|wheat|Lb|Unknown|64|eingcallyantipriought| +13569|AAAAAAAAAAFDAAAA|2000-10-27||Urban aspects feel; other parties find words. Seconds allow just primary, new programmes. Male powers become sincerely on a needs. Faster new years call fair changes|5.19|1.55|9015010|scholarunivamalg #10|15|fiction|9|Books|662|ablecallycally|N/A|5151saddle7348295674|pink|Tsp|Unknown|62|n stcallyantipriought| +13570|AAAAAAAACAFDAAAA|1997-10-27|1999-10-27|Advisers get patients. Busy regulations|6.34|2.47|3003001|exportiexporti #1|3|toddlers|3|Children|979|n stationn st|medium|712256midnight773573|plum|Bundle|Unknown|12|barationantipriought| +13571|AAAAAAAACAFDAAAA|1999-10-28|2001-10-26|Advisers get patients. Busy regulations|9.46|2.47|3003001|exportinameless #2|3|basketball|8|Sports|372|ableationpri|N/A|78plum97819955278448|saddle|Cup|Unknown|14|oughtationantipriought| +13572|AAAAAAAACAFDAAAA|2001-10-27||In order desirable muscles may not continue surely that is main pressures; slowly new expenses exclude long. Widely running dreams would rectify rightly prices. Parents tell at|0.21|2.47|9005001|scholarmaxi #1|3|history|9|Books|372|ableationpri|N/A|32314196600502royal7|peru|Cup|Unknown|2|ableationantipriought| +13573|AAAAAAAAFAFDAAAA|1997-10-27||As available citizens shall know. Unlikely, social men require. Leaves would make now on a years. Yet industrial interest|9.91|6.14|9012002|importounivamalg #2|12|home repair|9|Books|462|ablecallyese|N/A|39727sienna976146199|brown|Pallet|Unknown|46|priationantipriought| +13574|AAAAAAAAGAFDAAAA|1997-10-27|2000-10-26|Weeks take professional, bright students; equal, traditional months must not consider here most reasonable artists. Twice future manufacturers cannot become men. Military effor|4.88|4.09|10009009|maxiunivamalg #9|9|televisions|10|Electronics|130|barpriought|N/A|2461spring3098470653|peru|Tbl|Unknown|51|eseationantipriought| +13575|AAAAAAAAGAFDAAAA|2000-10-27||Weeks take professional, bright students; equal, traditional months must not consider here most reasonable artists. Twice future manufacturers cannot become men. Military effor|0.69|4.09|4002002|importoedu pack #2|2|mens|4|Shoes|913|prioughtn st|large|2461spring3098470653|salmon|Ton|Unknown|17|antiationantipriought| +13576|AAAAAAAAIAFDAAAA|1997-10-27|1999-10-27|Appropriate, future spirits used to identify only, pale courses; necessary cases see. N|1.52|0.54|1004001|edu packamalg #1|4|swimwear|1|Women|859|n stantieing|economy|2425052982698peru949|navy|Dozen|Unknown|16|callyationantipriought| +13577|AAAAAAAAIAFDAAAA|1999-10-28|2001-10-26|Appropriate, future spirits used to identify only, pale courses; necessary cases see. N|4.99|0.54|1004001|importonameless #8|4|baseball|8|Sports|267|ationcallyable|N/A|826659803puff1116399|yellow|Bunch|Unknown|53|ationationantipriought| +13578|AAAAAAAAIAFDAAAA|2001-10-27||Sections might not jump for a teachers. Still very drivers could use at the women. Millions would seize very tall players; alternatives would come enough. Families should sell too at a women; |9.07|0.54|1003001|exportiamalg #1|4|maternity|1|Women|694|ationcallyable|extra large|826659803puff1116399|peach|Bunch|Unknown|40|eingationantipriought| +13579|AAAAAAAALAFDAAAA|1997-10-27||Me|7.40|3.03|2002002|importoimporto #2|2|shirts|2|Men|342|ableesepri|large|38790violet988005141|burlywood|Lb|Unknown|31|n stationantipriought| +13580|AAAAAAAAMAFDAAAA|1997-10-27|2000-10-26|Usually small clothes will emerge else other areas. Short readers define please; able, special plants could not study now organizations; other, white elements know according |6.32|2.21|6012005|importobrand #5|12|costume|6|Jewelry|506|callybaranti|N/A|4orange6818686985198|pink|Dram|Unknown|24|bareingantipriought| +13581|AAAAAAAAMAFDAAAA|2000-10-27||Usually small clothes will emerge else other areas. Short readers define please; able, special plants could not study now organizations; other, white elements know according |41.79|2.21|6012005|exportinameless #2|3|basketball|8|Sports|506|callybaranti|N/A|4orange6818686985198|rosy|Ton|Unknown|91|oughteingantipriought| +13582|AAAAAAAAOAFDAAAA|1997-10-27|1999-10-27|Homes might not make only hours. Medical, reasonable procedures love too just eyes; existing, only lines p|0.45|0.33|4002001|importoedu pack #1|2|mens|4|Shoes|233|pripriable|medium|19039seashell2240013|medium|Tbl|Unknown|6|ableeingantipriought| +13583|AAAAAAAAOAFDAAAA|1999-10-28|2001-10-26|A lot economic others construct later by a meetings; extraordinary, liable beings might not draw about a legs; early p|0.88|0.55|4002001|scholarnameless #4|15|tables|7|Home|469|n stcallyese|N/A|19039seashell2240013|moccasin|Tsp|Unknown|5|prieingantipriought| +13584|AAAAAAAAOAFDAAAA|2001-10-27||A lot economic others construct later by a meetings; extraordinary, liable beings might not draw about a legs; early p|1.63|0.55|7014005|edu packnameless #5|15|glassware|7|Home|469|n stcallyese|N/A|19039seashell2240013|red|Lb|Unknown|18|eseeingantipriought| +13585|AAAAAAAABBFDAAAA|1997-10-27||Substantially slight tests used to convert national facilities.|2.21|1.03|7003006|exportibrand #6|3|kids|7|Home|64|esecally|N/A|red41775729839242949|tomato|Pound|Unknown|19|antieingantipriought| +13586|AAAAAAAACBFDAAAA|1997-10-27|2000-10-26|Head tasks know on a services. Diplomatic, regional lands find quite huma|4.85|4.17|10010004|univamalgamalg #4|10|memory|10|Electronics|155|antiantiought|N/A|7metallic21345205182|gainsboro|Cup|Unknown|15|callyeingantipriought| +13587|AAAAAAAACBFDAAAA|2000-10-27||Head tasks know on a services. Diplomatic, regional lands find quite huma|37.98|4.17|2003002|exportiimporto #2|3|pants|2|Men|155|antiantiought|extra large|647purple57325239215|navajo|Bundle|Unknown|54|ationeingantipriought| +13588|AAAAAAAAEBFDAAAA|1997-10-27|1999-10-27|Warm banks should research however with the books. Really suspicious points mention. Conservative years can matter very in an interests. Hard common moments|4.02|2.97|5004001|edu packscholar #1|4|classical|5|Music|339|n stpripri|N/A|178987844salmon07463|navajo|Tbl|Unknown|36|eingeingantipriought| +13589|AAAAAAAAEBFDAAAA|1999-10-28|2001-10-26|Substantial, lengthy minutes shall not interfere much to the miners. Strong, good steps can try for the most part on a stands. European, successful bottles cannot answer then new, socia|6.57|4.00|5004001|amalgcorp #6|1|birdal|6|Jewelry|776|n stpripri|N/A|178987844salmon07463|antique|Dram|Unknown|22|n steingantipriought| +13590|AAAAAAAAEBFDAAAA|2001-10-27||Substantial, lengthy minutes shall not interfere much to the miners. Strong, good steps can try for the most part on a stands. European, successful bottles cannot answer then new, socia|1.19|0.49|5004001|amalgbrand #7|11|semi-precious|6|Jewelry|776|callyationation|N/A|178987844salmon07463|slate|Ton|Unknown|21|barn stantipriought| +13591|AAAAAAAAHBFDAAAA|1997-10-27||Eyes attend both populations. Personal, theoretical populatio|0.68|0.44|1002002|importoamalg #2|2|fragrances|1|Women|194|esen stought|medium|2327925447826medium3|midnight|N/A|Unknown|1|oughtn stantipriought| +13592|AAAAAAAAIBFDAAAA|1997-10-27|2000-10-26|Public, other years breed together then american troops. Used problems recruit here german conditions. Cases shall not|38.70|34.44|3004001|edu packexporti #1|4|school-uniforms|3|Children|247|ationeseable|petite|7568061800rosy459767|smoke|N/A|Unknown|56|ablen stantipriought| +13593|AAAAAAAAIBFDAAAA|2000-10-27||Public, other years breed together then american troops. Used problems recruit here german conditions. Cases shall not|6.11|34.44|3004001|amalgscholar #2|4|rock|5|Music|247|ationeseable|N/A|7568061800rosy459767|maroon|Dozen|Unknown|6|prin stantipriought| +13594|AAAAAAAAKBFDAAAA|1997-10-27|1999-10-27|Local, likely funds grow inner studies. Twice close res|9.23|8.03|8008005|namelessnameless #5|8|outdoor|8|Sports|213|prioughtable|N/A|10724saddle743984737|ivory|Carton|Unknown|31|esen stantipriought| +13595|AAAAAAAAKBFDAAAA|1999-10-28|2001-10-26|Different genes used to reveal above around sorry benefits; appropriate, brief leaders bring about significant problems. Really heavy eyes te|4.58|3.57|8008005|amalgscholar #2|8|rock|5|Music|270|prioughtable|N/A|7898sky7879617059572|indian|N/A|Unknown|7|antin stantipriought| +13596|AAAAAAAAKBFDAAAA|2001-10-27||Royal, selective artists can win so. Poor, main views would repeat clear requirements. Social, political sizes will not go as long companies. Windows draw now special men. Pref|1.68|3.57|8008005|amalgscholar #1|8|rock|5|Music|270|barationable|N/A|7898sky7879617059572|royal|Ton|Unknown|14|callyn stantipriought| +13597|AAAAAAAANBFDAAAA|1997-10-27||Schools would know anyway other, classical records. Chosen, poor times c|3.73|3.24|3003002|exportiexporti #2|3|toddlers|3|Children|158|eingantiought|petite|2186447linen82419724|mint|Gross|Unknown|16|ationn stantipriought| +13598|AAAAAAAAOBFDAAAA|1997-10-27|2000-10-26|Old, poor buyers launch so. Temporary years must not move old, unpleasant signs. More christian years could not discriminate; waves would hear ways. Issues |5.61|2.24|6010007|univbrand #7|10|jewelry boxes|6|Jewelry|254|eseantiable|N/A|4snow469843753584343|lawn|Dram|Unknown|74|eingn stantipriought| +13599|AAAAAAAAOBFDAAAA|2000-10-27||Old, poor buyers launch so. Temporary years must not move old, unpleasant signs. More christian years could not discriminate; waves would hear ways. Issues |3.78|3.32|10007017|brandunivamalg #17|7|personal|10|Electronics|302|eseantiable|N/A|4snow469843753584343|goldenrod|N/A|Unknown|45|n stn stantipriought| +13600|AAAAAAAAACFDAAAA|1997-10-27|1999-10-27|British, familiar cups sho|97.01|64.99|7001001|amalgbrand #1|1|bathroom|7|Home|539|n stprianti|N/A|75sandy7513911234346|snow|Pallet|Unknown|46|barbarcallypriought| +13601|AAAAAAAAACFDAAAA|1999-10-28|2001-10-26|British, familiar cups sho|5.45|2.94|10016002|corpamalgamalg #2|1|wireless|10|Electronics|539|n stprianti|N/A|004897178antique6518|snow|Ton|Unknown|23|oughtbarcallypriought| +13602|AAAAAAAAACFDAAAA|2001-10-27||Full nights pass rat|3.31|1.42|10005003|scholarunivamalg #3|5|karoke|10|Electronics|539|n stprianti|N/A|004897178antique6518|thistle|Pallet|Unknown|13|ablebarcallypriought| +13603|AAAAAAAADCFDAAAA|1997-10-27||Long mean ri|8.15|3.34|5001002|amalgscholar #2|1|rock|5|Music|264|esecallyable|N/A|90637870turquoise428|yellow|Lb|Unknown|30|pribarcallypriought| +13604|AAAAAAAAECFDAAAA|1997-10-27|2000-10-26|Old others tell; immediate eggs leave terms. Seats involve sensibly anyway royal individuals. Interesting, american year|3.73|2.23|8016005|corpmaxi #5|16|golf|8|Sports|355|antiantipri|N/A|0167071498turquoise9|yellow|Pallet|Unknown|11|esebarcallypriought| +13605|AAAAAAAAECFDAAAA|2000-10-27||Measures spare still. Beliefs perform. Also financial fruits would get at a hours. True copies would prepare ago too negative critics. Just whole pounds take really existin|5.70|3.13|8008006|namelessnameless #6|8|outdoor|8|Sports|238|antiantipri|N/A|0167071498turquoise9|salmon|Bunch|Unknown|19|antibarcallypriought| +13606|AAAAAAAAGCFDAAAA|1997-10-27|1999-10-27|Young emotions consider too accidents. Illustrations|0.79|0.45|4004001|edu packedu pack #1|4|athletic|4|Shoes|84|eseeing|medium|449738614sandy380935|rosy|Each|Unknown|23|callybarcallypriought| +13607|AAAAAAAAGCFDAAAA|1999-10-28|2001-10-26|Gastric, diplomatic words see somehow assessments. Important courses shall not take most different, responsible program|51.70|0.45|4004001|edu packedu pack #2|4|athletic|4|Shoes|84|eseeing|petite|449738614sandy380935|white|Bundle|Unknown|2|ationbarcallypriought| +13608|AAAAAAAAGCFDAAAA|2001-10-27||Gastric, diplomatic words see somehow assessments. Important courses shall not take most different, responsible program|7.07|0.45|5004001|edu packscholar #1|4|classical|5|Music|88|eseeing|N/A|054powder85188579083|thistle|Dozen|Unknown|22|eingbarcallypriought| +13609|AAAAAAAAJCFDAAAA|1997-10-27||Absolute criteria shall go very pleasant, impossible customers. Political changes cannot strengthen there different authorities. Decisively favorite conditions stud|3.02|1.78|5003002|exportischolar #2|3|pop|5|Music|211|oughtoughtable|N/A|5942809maroon5480110|yellow|Tbl|Unknown|15|n stbarcallypriought| +13610|AAAAAAAAKCFDAAAA|1997-10-27|2000-10-26|Romantic, part-time words question anywhere. Friendly relations try. Technical, small tiles can modify recent, white funds. Supposed miles shall move good arts. Proper, strong prope|2.10|0.79|3003001|exportiexporti #1|3|toddlers|3|Children|550|barantianti|extra large|42775steel9363420066|rosy|Unknown|Unknown|19|baroughtcallypriought| +13611|AAAAAAAAKCFDAAAA|2000-10-27||Major, previous parts might not dry already in a bodies. Ashamed ap|5.18|3.36|4004002|edu packedu pack #2|4|athletic|4|Shoes|529|barantianti|economy|42775steel9363420066|turquoise|Bunch|Unknown|22|oughtoughtcallypriought| +13612|AAAAAAAAMCFDAAAA|1997-10-27|1999-10-27|Poor risks can support as bright, determined tiles; plans comfort. Prin|4.20|2.73|8015003|scholarmaxi #3|15|fishing|8|Sports|116|callyoughtought|N/A|674387gainsboro47938|beige|N/A|Unknown|16|ableoughtcallypriought| +13613|AAAAAAAAMCFDAAAA|1999-10-28|2001-10-26|Games would visit long auditors. Total,|1.72|0.98|7003004|exportibrand #4|15|kids|7|Home|116|callyoughtought|N/A|680812315735sandy790|yellow|Pound|Unknown|96|prioughtcallypriought| +13614|AAAAAAAAMCFDAAAA|2001-10-27||Games would visit long auditors. Total,|1.28|0.98|8010005|univmaxi #5|15|pools|8|Sports|116|callyoughtought|N/A|680812315735sandy790|peru|Gram|Unknown|30|eseoughtcallypriought| +13615|AAAAAAAAPCFDAAAA|1997-10-27||Scientifi|9.92|4.46|5003002|exportischolar #2|3|pop|5|Music|250|barantiable|N/A|76167770693702peach1|rosy|Tsp|Unknown|15|antioughtcallypriought| +13616|AAAAAAAAADFDAAAA|1997-10-27|2000-10-26|Examples can use only considerable cases. Cells will offer individuals. Sure minute weaknesses might write successive prisons. For example black c|3.34|2.57|8013007|exportimaxi #7|13|sailing|8|Sports|650|barantically|N/A|65058815slate1329828|rosy|Each|Unknown|22|callyoughtcallypriought| +13617|AAAAAAAAADFDAAAA|2000-10-27||Fees used to make less between the reports. Common, political relati|2.23|2.57|8013007|exportiexporti #2|3|toddlers|3|Children|650|barantically|extra large|65058815slate1329828|wheat|Ounce|Unknown|79|ationoughtcallypriought| +13618|AAAAAAAACDFDAAAA|1997-10-27|1999-10-27|Whole, thin items |1.75|1.15|9012011|importounivamalg #11|12|home repair|9|Books|962|ablecallyn st|N/A|4106829papaya5054608|purple|Carton|Unknown|36|eingoughtcallypriought| +13619|AAAAAAAACDFDAAAA|1999-10-28|2001-10-26|Contemporary, new arts must charge personal opportunities. Never specialist employees consider democratic, big comparisons. Moral others go alike large-scale chairs. |2.47|1.15|9012011|scholarnameless #10|5|fitness|8|Sports|962|ablecallyn st|N/A|4106829papaya5054608|thistle|Ounce|Unknown|47|n stoughtcallypriought| +13620|AAAAAAAACDFDAAAA|2001-10-27||Contemporary, new arts must charge personal opportunities. Never specialist employees consider democratic, big comparisons. Moral others go alike large-scale chairs. |1.52|0.98|9012011|brandmaxi #1|7|reference|9|Books|962|ablecallyn st|N/A|4106829papaya5054608|smoke|Tsp|Unknown|14|barablecallypriought| +13621|AAAAAAAAFDFDAAAA|1997-10-27||Very silly children laugh single paintings; tests find essenti|4.85|3.92|7015002|scholarnameless #2|15|tables|7|Home|728|eingableation|N/A|4maroon3965341057125|olive|Pallet|Unknown|10|oughtablecallypriought| +13622|AAAAAAAAGDFDAAAA|1997-10-27|2000-10-26|Only formal procedures find hardly |57.45|44.23|5002001|importoscholar #1|2|country|5|Music|842|ableeseeing|N/A|290412medium89593435|sky|Dozen|Unknown|31|ableablecallypriought| +13623|AAAAAAAAGDFDAAAA|2000-10-27||Only formal procedures find hardly |38.74|19.75|7010004|univnameless #4|2|flatware|7|Home|842|ableeseeing|N/A|5641560820ivory60201|rosy|Ton|Unknown|55|priablecallypriought| +13624|AAAAAAAAIDFDAAAA|1997-10-27|1999-10-27|Social, official things buy bare, welcome couples. Much honest problems can change emissions. Political cigarettes marry thin|58.79|41.15|3004001|edu packexporti #1|4|school-uniforms|3|Children|313|prioughtpri|medium|88398256powder362089|lavender|Cup|Unknown|2|eseablecallypriought| +13625|AAAAAAAAIDFDAAAA|1999-10-28|2001-10-26|Social, official things buy bare, welcome couples. Much honest problems can change emissions. Political cigarettes marry thin|0.60|0.39|3004001|exportimaxi #6|3|computers|9|Books|313|prioughtpri|N/A|88398256powder362089|sandy|Box|Unknown|45|antiablecallypriought| +13626|AAAAAAAAIDFDAAAA|2001-10-27||Lo|9.24|0.39|3004001|amalgbrand #3|3|semi-precious|6|Jewelry|134|esepriought|N/A|88398256powder362089|grey|Pound|Unknown|13|callyablecallypriought| +13627|AAAAAAAALDFDAAAA|1997-10-27||Internal years may not pr|3.46|1.24|9014008|edu packunivamalg #8|14|sports|9|Books|98|eingn st|N/A|68sienna037712500760|green|Gram|Unknown|30|ationablecallypriought| +13628|AAAAAAAAMDFDAAAA|1997-10-27|2000-10-26|Things defend even lucky, medical ministers. Individ|7.34|4.03|6007005|brandcorp #5|7|pendants|6|Jewelry|72|ableation|N/A|72788399579226white2|purple|Tbl|Unknown|39|eingablecallypriought| +13629|AAAAAAAAMDFDAAAA|2000-10-27||Specific results explain all in the women. Past tensions ought to find this with a trustees. Separate, joint offices would say more than mutual objects. Ideas legislate infinitely as financ|6.17|4.03|6007005|importoedu pack #2|2|mens|4|Shoes|200|ableation|petite|72788399579226white2|pale|Pallet|Unknown|13|n stablecallypriought| +13630|AAAAAAAAODFDAAAA|1997-10-27|1999-10-27|Marked, large years|0.95|0.74|9009011|maximaxi #11|9|science|9|Books|353|priantipri|N/A|96896256thistle58405|steel|Cup|Unknown|24|barpricallypriought| +13631|AAAAAAAAODFDAAAA|1999-10-28|2001-10-26|Too married humans must come. Clean, old-fashioned systems can remember clear, french teachers. Then equal services should not get with a companies. Faint parts shall speak anywhere years. Al|1.43|1.05|9009011|importoimporto #2|2|shirts|2|Men|353|priantipri|small|96896256thistle58405|lace|Each|Unknown|55|oughtpricallypriought| +13632|AAAAAAAAODFDAAAA|2001-10-27||Social, western programmes exercise suddenl|55.82|1.05|7003003|exportibrand #3|2|kids|7|Home|516|callyoughtanti|N/A|96896256thistle58405|black|Dozen|Unknown|17|ablepricallypriought| +13633|AAAAAAAABEFDAAAA|1997-10-27||Systems may see quickly dead eyes. Probably new|1.08|0.82|1004002|edu packamalg #2|4|swimwear|1|Women|195|antin stought|small|9lemon63880091811568|lace|Bunch|Unknown|51|pripricallypriought| +13634|AAAAAAAACEFDAAAA|1997-10-27|2000-10-26|Ago total goods see similar organizations. Explicitly old regions adapt together. Here p|8.40|3.36|7001005|amalgbrand #5|1|bathroom|7|Home|311|oughtoughtpri|N/A|323514699237violet11|puff|Each|Unknown|1|esepricallypriought| +13635|AAAAAAAACEFDAAAA|2000-10-27||Ago total goods see similar organizations. Explicitly old regions adapt together. Here p|6.75|3.36|4003002|exportiedu pack #2|3|kids|4|Shoes|311|oughtoughtpri|large|323514699237violet11|saddle|Pound|Unknown|26|antipricallypriought| +13636|AAAAAAAAEEFDAAAA|1997-10-27|1999-10-27|Forward certain words get responsible governors. Important, other systems could come now aspects. Even private groups may apply probably in|2.65|0.95|8011007|amalgmaxi #7|11|archery|8|Sports|93|prin st|N/A|56997257436spring565|violet|Cup|Unknown|16|callypricallypriought| +13637|AAAAAAAAEEFDAAAA|1999-10-28|2001-10-26|Forward certain words get responsible governors. Important, other systems could come now aspects. Even private groups may apply probably in|0.71|0.31|9013006|exportiunivamalg #6|13|self-help|9|Books|127|prin st|N/A|56997257436spring565|aquamarine|Box|Unknown|13|ationpricallypriought| +13638|AAAAAAAAEEFDAAAA|2001-10-27||Blacks deteriorate never groups. Responsible businesses suffer|2.92|1.75|9013006|edu packamalg #1|4|swimwear|1|Women|127|prin st|medium|6715202849salmon0085|salmon|Dozen|Unknown|48|eingpricallypriought| +13639|AAAAAAAAHEFDAAAA|1997-10-27||Late right executives shall not appeal in a policies. Following details will look. Only|2.26|1.44|10009006|maxiunivamalg #6|9|televisions|10|Electronics|422|ableableese|N/A|434270puff8472876963|ghost|Bundle|Unknown|28|n stpricallypriought| +13640|AAAAAAAAIEFDAAAA|1997-10-27|2000-10-26|Almost armed animals will maintain always pure, professional days; differe|6.67|2.33|1001001|amalgamalg #1|1|dresses|1|Women|198|eingn stought|extra large|693purple85546391574|slate|Unknown|Unknown|28|baresecallypriought| +13641|AAAAAAAAIEFDAAAA|2000-10-27||New, foreign weapons st|5.69|3.24|1001001|amalgnameless #2|1|athletic shoes|8|Sports|198|eingn stought|N/A|5291333158papaya2050|floral|Pound|Unknown|6|oughtesecallypriought| +13642|AAAAAAAAKEFDAAAA|1997-10-27|1999-10-27|P|88.04|44.02|3001001|amalgexporti #1|1|newborn|3|Children|13|priought|medium|0420moccasin50060700|papaya|Tsp|Unknown|23|ableesecallypriought| +13643|AAAAAAAAKEFDAAAA|1999-10-28|2001-10-26|P|5.92|5.03|3001001|corpnameless #4|1|furniture|7|Home|13|priought|N/A|221tan08477132443135|blanched|Each|Unknown|25|priesecallypriought| +13644|AAAAAAAAKEFDAAAA|2001-10-27||White, likely effects will convince a|8.83|4.94|3004001|edu packexporti #1|4|school-uniforms|3|Children|13|priought|medium|221tan08477132443135|tan|Ton|Unknown|70|eseesecallypriought| +13645|AAAAAAAANEFDAAAA|1997-10-27||Early, other officers should comply ethical, sexual conflicts. Long feet meet fun, known complaints; useful shoulders shall not think now christian, initial husbands. Critica|4.72|1.79|3002002|importoexporti #2|2|infants|3|Children|243|prieseable|economy|59rose63392218399848|peru|Pallet|Unknown|14|antiesecallypriought| +13646|AAAAAAAAOEFDAAAA|1997-10-27|2000-10-26|Now main streets ought to lift streets. Cars see peoples. Black governments enter sudden theories. Different, vulnerable events could not help bills. Designs see wit|6.21|2.67|8002007|importonameless #7|2|baseball|8|Sports|47|ationese|N/A|0392230lawn739815459|thistle|Gross|Unknown|23|callyesecallypriought| +13647|AAAAAAAAOEFDAAAA|2000-10-27||Now main streets ought to lift streets. Cars see peoples. Black governments enter sudden theories. Different, vulnerable events could not help bills. Designs see wit|6.91|2.67|5002002|importoscholar #2|2|country|5|Music|47|ationese|N/A|27pink27872577780733|turquoise|Dozen|Unknown|27|ationesecallypriought| +13648|AAAAAAAAAFFDAAAA|1997-10-27|1999-10-27|Cultural employers bring schools. Eventually domestic children might not fall. Small offices will compet|8.36|2.75|10012015|importoamalgamalg #15|12|monitors|10|Electronics|733|pripriation|N/A|4154199508142plum300|sky|Cup|Unknown|19|eingesecallypriought| +13649|AAAAAAAAAFFDAAAA|1999-10-28|2001-10-26|Cultural employers bring schools. Eventually domestic children might not fall. Small offices will compet|0.28|2.75|10012015|edu packmaxi #6|4|entertainments|9|Books|733|pripriation|N/A|4154199508142plum300|lime|Lb|Unknown|64|n stesecallypriought| +13650|AAAAAAAAAFFDAAAA|2001-10-27||Cultural employers bring schools. Eventually domestic children might not fall. Small offices will compet|4.79|2.75|1001001|amalgamalg #1|1|dresses|1|Women|608|pripriation|petite|4154199508142plum300|saddle|Gram|Unknown|20|baranticallypriought| +13651|AAAAAAAADFFDAAAA|1997-10-27||American units put here despite the others. Local, short years would go somewhere for a eyes. European, simple countries could not negotiate even talks. Again mental areas can |7.42|6.52|8003002|exportinameless #2|3|basketball|8|Sports|470|barationese|N/A|012040864lace3519010|sky|Tbl|Unknown|13|oughtanticallypriought| +13652|AAAAAAAAEFFDAAAA|1997-10-27|2000-10-26|Lacking, new forms let short sides. Women must walk formally. Effective, public teachers see; particular payments urge in a cards. International, lab|0.91|0.32|10004002|edu packunivamalg #2|4|audio|10|Electronics|610|baroughtcally|N/A|5752859561621939tan0|sienna|Dram|Unknown|7|ableanticallypriought| +13653|AAAAAAAAEFFDAAAA|2000-10-27||Lacking, new forms let short sides. Women must walk formally. Effective, public teachers see; particular payments urge in a cards. International, lab|8.40|0.32|4003002|exportiedu pack #2|3|kids|4|Shoes|610|baroughtcally|small|5752859561621939tan0|royal|Cup|Unknown|15|prianticallypriought| +13654|AAAAAAAAGFFDAAAA|1997-10-27|1999-10-27|Horizontal nerves will not study just. Issues shall not imagine workshops. Relevant industries provide british, fresh others. Commercial, new houses give with the |3.34|2.27|7001005|amalgbrand #5|1|bathroom|7|Home|283|prieingable|N/A|83626cyan68320266658|peach|Case|Unknown|32|eseanticallypriought| +13655|AAAAAAAAGFFDAAAA|1999-10-28|2001-10-26|Horizontal nerves will not study just. Issues shall not imagine workshops. Relevant industries provide british, fresh others. Commercial, new houses give with the |7.52|6.31|7001005|namelessunivamalg #5|1|scanners|10|Electronics|545|prieingable|N/A|83626cyan68320266658|snow|Gross|Unknown|6|antianticallypriought| +13656|AAAAAAAAGFFDAAAA|2001-10-27||Programmes get passionately never growing stocks. Benefits might give alternatively. Political, american findings pay new, welsh lectures|7.12|6.31|9007001|brandmaxi #1|7|reference|9|Books|182|ableeingought|N/A|83626cyan68320266658|thistle|Carton|Unknown|5|callyanticallypriought| +13657|AAAAAAAAJFFDAAAA|1997-10-27||Reasons used to pay however schools. British, sure conditions will not see british communists. Sales |2.71|1.08|1004002|edu packamalg #2|4|swimwear|1|Women|754|eseantiation|small|85272596986353slate1|snow|Box|Unknown|19|ationanticallypriought| +13658|AAAAAAAAKFFDAAAA|1997-10-27|2000-10-26|Eventual divisions measure small, uncertain researchers. Obvious operations recognise slowly thin places. Closely obvious magistrates w|6.60|4.02|5003001|exportischolar #1|3|pop|5|Music|547|ationeseanti|N/A|634685smoke386122498|plum|Dram|Unknown|61|einganticallypriought| +13659|AAAAAAAAKFFDAAAA|2000-10-27||Eventual divisions measure small, uncertain researchers. Obvious operations recognise slowly thin places. Closely obvious magistrates w|3.94|4.02|3004002|edu packexporti #2|3|school-uniforms|3|Children|618|ationeseanti|petite|634685smoke386122498|lace|Cup|Unknown|4|n stanticallypriought| +13660|AAAAAAAAMFFDAAAA|1997-10-27|1999-10-27|Only modern experts will matter once again. |6.13|3.00|2001001|amalgimporto #1|1|accessories|2|Men|294|esen stable|N/A|3white32384027600618|spring|Each|Unknown|62|barcallycallypriought| +13661|AAAAAAAAMFFDAAAA|1999-10-28|2001-10-26|Only modern experts will matter once again. |4.68|3.74|2001001|exportiexporti #2|3|toddlers|3|Children|294|esen stable|medium|3white32384027600618|saddle|Ton|Unknown|11|oughtcallycallypriought| +13662|AAAAAAAAMFFDAAAA|2001-10-27||Only modern experts will matter once again. |0.22|3.74|2001001|brandmaxi #7|3|reference|9|Books|294|esen stable|N/A|817334134668papaya24|powder|Bundle|Unknown|20|ablecallycallypriought| +13663|AAAAAAAAPFFDAAAA|1997-10-27||British, modern instances would not put tomorrow with the fields. Problems sit more. Traditional experiences get full definitions. So st|3.68|1.91|3001002|amalgexporti #2|1|newborn|3|Children|543|prieseanti|petite|712thistle5233980626|honeydew|Pallet|Unknown|15|pricallycallypriought| +13664|AAAAAAAAAGFDAAAA|1997-10-27|2000-10-26|Warm, welsh attitudes start over agricultural, eng|4.07|2.03|8009005|maxinameless #5|9|optics|8|Sports|374|eseationpri|N/A|9447859sandy07430019|steel|Ton|Unknown|50|esecallycallypriought| +13665|AAAAAAAAAGFDAAAA|2000-10-27||Warm, welsh attitudes start over agricultural, eng|0.74|0.45|6005002|scholarcorp #2|5|earings|6|Jewelry|183|eseationpri|N/A|90152175071261white8|lawn|Ounce|Unknown|60|anticallycallypriought| +13666|AAAAAAAACGFDAAAA|1997-10-27|1999-10-27|Public clothes ought to open. So principal trials hold again under a feelings; large, economic requirements think for a years; small wages ought to |9.66|7.34|8008007|namelessnameless #7|8|outdoor|8|Sports|704|esebaration|N/A|0229sandy33950902558|cornflower|Case|Unknown|72|callycallycallypriought| +13667|AAAAAAAACGFDAAAA|1999-10-28|2001-10-26|Only, special discussions must not determine miners. New affairs could place. Efficient, other strangers could not form now true, ethnic advertisements. Relevant arts catch visibly si|1.02|7.34|5002002|importoscholar #2|8|country|5|Music|704|esebaration|N/A|212221509smoke943347|thistle|Lb|Unknown|13|ationcallycallypriought| +13668|AAAAAAAACGFDAAAA|2001-10-27||Only, special discussions must not determine miners. New affairs could place. Efficient, other strangers could not form now true, ethnic advertisements. Relevant arts catch visibly si|3.10|7.34|5002002|edu packexporti #1|8|school-uniforms|3|Children|704|esebaration|small|8637287063938smoke80|sienna|Ounce|Unknown|36|eingcallycallypriought| +13669|AAAAAAAAFGFDAAAA|1997-10-27||Ec|4.86|3.54|7001010|amalgbrand #10|1|bathroom|7|Home|841|oughteseeing|N/A|6284848994orchid7035|thistle|Dram|Unknown|32|n stcallycallypriought| +13670|AAAAAAAAGGFDAAAA|1997-10-27|2000-10-26|Bad, various p|8.16|5.38|8014001|edu packmaxi #1|14|tennis|8|Sports|260|barcallyable|N/A|1117196816beige16561|steel|Gram|Unknown|56|barationcallypriought| +13671|AAAAAAAAGGFDAAAA|2000-10-27||Bad, various p|5.09|4.42|8014001|amalgmaxi #2|14|archery|8|Sports|260|barcallyable|N/A|1117196816beige16561|puff|Dram|Unknown|24|oughtationcallypriought| +13672|AAAAAAAAIGFDAAAA|1997-10-27|1999-10-27|Subjects must make virtually careful feelings. Possible years could get fixed ideas. Irish candidates use nuclear, separate years. All able times may not put virtually good employers; however british|3.00|1.26|5003001|exportischolar #1|3|pop|5|Music|987|ationeingn st|N/A|1145mint809825106015|red|Unknown|Unknown|23|ableationcallypriought| +13673|AAAAAAAAIGFDAAAA|1999-10-28|2001-10-26|Subjects must make virtually careful feelings. Possible years could get fixed ideas. Irish candidates use nuclear, separate years. All able times may not put virtually good employers; however british|2.66|1.26|2002002|importoimporto #2|2|shirts|2|Men|584|eseeinganti|petite|1052113110papaya5887|coral|Pound|Unknown|11|priationcallypriought| +13674|AAAAAAAAIGFDAAAA|2001-10-27||Subjects must make virtually careful feelings. Possible years could get fixed ideas. Irish candidates use nuclear, separate years. All able times may not put virtually good employers; however british|2.46|0.81|2002002|exporticorp #3|3|gold|6|Jewelry|584|eseeinganti|N/A|1052113110papaya5887|white|Ton|Unknown|27|eseationcallypriought| +13675|AAAAAAAALGFDAAAA|1997-10-27||Officials resume about. Ever human arts take at least. Decent cases reply now during a|0.38|0.13|7007006|brandbrand #6|7|decor|7|Home|642|ableesecally|N/A|7910068753201white45|white|Cup|Unknown|13|antiationcallypriought| +13676|AAAAAAAAMGFDAAAA|1997-10-27|2000-10-26|Even fair politicians put surely s|9.58|4.40|8011007|amalgmaxi #7|11|archery|8|Sports|270|barationable|N/A|20855992slate3780307|peru|Cup|Unknown|43|callyationcallypriought| +13677|AAAAAAAAMGFDAAAA|2000-10-27||Examples set. Blue, short sides may persuade wives. Rooms admi|70.59|4.40|8011007|exportiedu pack #2|3|kids|4|Shoes|270|barationable|large|20855992slate3780307|yellow|Each|Unknown|19|ationationcallypriought| +13678|AAAAAAAAOGFDAAAA|1997-10-27|1999-10-27|So far wheels can adopt |6.43|4.24|4002001|importoedu pack #1|2|mens|4|Shoes|295|antin stable|small|10613midnight2986285|slate|Cup|Unknown|63|eingationcallypriought| +13679|AAAAAAAAOGFDAAAA|1999-10-28|2001-10-26|So far wheels can adopt |2.17|0.69|4002001|importoimporto #2|2|shirts|2|Men|295|antin stable|petite|811sienna10567188212|papaya|Tsp|Unknown|58|n stationcallypriought| +13680|AAAAAAAAOGFDAAAA|2001-10-27||So far wheels can adopt |5.05|3.63|7016001|corpnameless #1|2|furniture|7|Home|295|antin stable|N/A|811sienna10567188212|snow|Oz|Unknown|61|bareingcallypriought| +13681|AAAAAAAABHFDAAAA|1997-10-27||Whole disciplines might not grow now round tales; references used to go ago in a demands. Polish, universal things would|23.63|16.06|6004002|edu packcorp #2|4|bracelets|6|Jewelry|447|ationeseese|N/A|0269435310021rose581|royal|Dram|Unknown|34|oughteingcallypriought| +13682|AAAAAAAACHFDAAAA|1997-10-27|2000-10-26|Main cities account home new s|7.55|5.13|5003001|exportischolar #1|3|pop|5|Music|640|baresecally|N/A|440654rose0157458940|white|Pound|Unknown|14|ableeingcallypriought| +13683|AAAAAAAACHFDAAAA|2000-10-27||Free feet experience too powerful coins. Mild details |1.16|0.64|5003001|amalgmaxi #4|3|arts|9|Books|92|baresecally|N/A|440654rose0157458940|spring|Dozen|Unknown|41|prieingcallypriought| +13684|AAAAAAAAEHFDAAAA|1997-10-27|1999-10-27|Formal, other eyes may work too eastern departments. Ever potential servic|3.52|1.68|1004001|edu packamalg #1|4|swimwear|1|Women|111|oughtoughtought|medium|531010372452plum2173|smoke|Oz|Unknown|14|eseeingcallypriought| +13685|AAAAAAAAEHFDAAAA|1999-10-28|2001-10-26|French policies hear african regions. New services feel more young parents. Interests reach finally legal years. Future, different issues give then strange nations. Practices wil|1.04|1.68|1004001|maximaxi #6|4|science|9|Books|111|oughtoughtought|N/A|531010372452plum2173|pale|Dozen|Unknown|22|antieingcallypriought| +13686|AAAAAAAAEHFDAAAA|2001-10-27||Words gain more. Old, religious societies should like also also likely babies; so female leaves change no doubt aware, necessary pupils; affairs know actually. Recent, aware proposals |6.48|1.68|5001001|amalgscholar #1|1|rock|5|Music|183|prieingought|N/A|531010372452plum2173|lavender|Each|Unknown|2|callyeingcallypriought| +13687|AAAAAAAAHHFDAAAA|1997-10-27||Small, major workers shall experience for a wings; now pure authorities work then pictures. Correct things will increase about in a cities. Hours will shed. Restrictions|3.28|2.72|2002002|importoimporto #2|2|shirts|2|Men|142|ableeseought|small|960522thistle1328253|smoke|Each|Unknown|67|ationeingcallypriought| +13688|AAAAAAAAIHFDAAAA|1997-10-27|2000-10-26|Concepts deal effectively details. Ideal, public dogs use words. Central eyes m|3.72|1.19|4001001|amalgedu pack #1|1|womens|4|Shoes|227|ationableable|petite|82013594211267rose84|turquoise|Pound|Unknown|26|eingeingcallypriought| +13689|AAAAAAAAIHFDAAAA||||1.89|0.62|||||7|Home|230|barpriable|N/A|||Pound|Unknown|14|n steingcallypriought| +13690|AAAAAAAAKHFDAAAA|1997-10-27|1999-10-27|Usually natural divisions enhan|9.49|3.79|3001001|amalgexporti #1|1|newborn|3|Children|58|einganti|small|1734122tan9628347787|honeydew|Ounce|Unknown|22|barn stcallypriought| +13691|AAAAAAAAKHFDAAAA|1999-10-28|2001-10-26|Blue boys ought to eat long however unknown aspects. Maybe formal police explain recent, national pp.; friendly, special children must need with a talks; exact tickets predict seriousl|7.99|4.47|3001001|maxiunivamalg #7|9|televisions|10|Electronics|721|oughtableation|N/A|435peach305463006373|sienna|Case|Unknown|19|oughtn stcallypriought| +13692|AAAAAAAAKHFDAAAA|2001-10-27||Wealthy dishes should say also|7.71|3.85|5003001|exportischolar #1|3|pop|5|Music|721|oughtableation|N/A|7533powder6491518554|chocolate|Bunch|Unknown|6|ablen stcallypriought| +13693|AAAAAAAANHFDAAAA|1997-10-27||Easier strong operators could not break very; new, permanent animals |1.15|0.60|9003002|exportimaxi #2|3|computers|9|Books|26|callyable|N/A|7seashell60057769877|sienna|Pound|Unknown|52|prin stcallypriought| +13694|AAAAAAAAOHFDAAAA|1997-10-27|2000-10-26|At least large pairs sort for a seconds. External, other months agree associated, different circumstances. More public girls m|73.63|64.79|6015007|scholarbrand #7|15|custom|6|Jewelry|213|prioughtable|N/A|0480267875lemon60239|yellow|N/A|Unknown|18|esen stcallypriought| +13695|AAAAAAAAOHFDAAAA|2000-10-27||Able funds must not investigate distinct, short-term foundations. Sad, soft examples beli|8.11|64.79|10011011|amalgamalgamalg #11|15|disk drives|10|Electronics|138|eingpriought|N/A|powder36556099227216|orange|Gross|Unknown|40|antin stcallypriought| +13696|AAAAAAAAAIFDAAAA|1997-10-27|1999-10-27|Countries enter adequate, full fires. Imperial, available conditions will not continue interested, violent employees; clear, forward quantities ought to issue provisions. Following legs will|2.40|0.91|8012007|importomaxi #7|12|guns|8|Sports|315|antioughtpri|N/A|2130plum098515555998|peach|Ounce|Unknown|38|callyn stcallypriought| +13697|AAAAAAAAAIFDAAAA|1999-10-28|2001-10-26|Countries enter adequate, full fires. Imperial, available conditions will not continue interested, violent employees; clear, forward quantities ought to issue provisions. Following legs will|2.92|0.91|8012007|edu packexporti #2|4|school-uniforms|3|Children|572|antioughtpri|medium|2130plum098515555998|snow|Gram|Unknown|15|ationn stcallypriought| +13698|AAAAAAAAAIFDAAAA|2001-10-27||Countries enter adequate, full fires. Imperial, available conditions will not continue interested, violent employees; clear, forward quantities ought to issue provisions. Following legs will|2.04|0.91|8012007|amalgmaxi #9|11|archery|8|Sports|683|antioughtpri|N/A|58799948firebrick809|powder|Gross|Unknown|13|eingn stcallypriought| +13699|AAAAAAAADIFDAAAA|1997-10-27||Now strategic strategies must continue new, explicit months. Together mate|5.46|3.43|3003002|exportiexporti #2|3|toddlers|3|Children|236|callypriable|medium|344903red52375966195|blue|Lb|Unknown|20|n stn stcallypriought| +13700|AAAAAAAAEIFDAAAA|1997-10-27|2000-10-26|Illegally british days ought to create only. Open notes climb mostly just natural areas. Brief savings get months. Familiar, exclusive women enable critical powers. New, functional ports would|19.85|13.29|7014001|edu packnameless #1|14|glassware|7|Home|302|ablebarpri|N/A|18637418119coral1624|saddle|Oz|Unknown|53|barbarationpriought| +13701|AAAAAAAAEIFDAAAA|2000-10-27||Illegally british days ought to create only. Open notes climb mostly just natural areas. Brief savings get months. Familiar, exclusive women enable critical powers. New, functional ports would|8.18|4.41|4003002|exportiedu pack #2|14|kids|4|Shoes|302|ablebarpri|petite|18637418119coral1624|puff|Each|Unknown|33|oughtbarationpriought| +13702|AAAAAAAAGIFDAAAA|1997-10-27|1999-10-27|Capable, special units used to bring current, particular feet; children f|5.49|3.84|1004001|edu packamalg #1|4|swimwear|1|Women|368|eingcallypri|small|37smoke3146019184253|grey|Tsp|Unknown|29|ablebarationpriought| +13703|AAAAAAAAGIFDAAAA|1999-10-28|2001-10-26|Only previous copies work so to a duties. Old, current workers assess women; genetic, global points can r|3.82|3.84|1004001|importoamalg #2|2|fragrances|1|Women|368|eingcallypri|medium|37smoke3146019184253|lemon|Oz|Unknown|28|pribarationpriought| +13704|AAAAAAAAGIFDAAAA|2001-10-27||Other benefits ought to comprise afraid, able tickets; perh|1.95|1.03|2002001|importoimporto #1|2|shirts|2|Men|368|eingcallypri|extra large|01yellow914255336787|pale|Tsp|Unknown|32|esebarationpriought| +13705|AAAAAAAAJIFDAAAA|1997-10-27||Years wear easier. Other individuals must not take. British players shall not achieve yet sweet activities. Really major purposes may not work more apparent, grand women; guilty,|2.11|1.77|6001002|amalgcorp #2|1|birdal|6|Jewelry|377|ationationpri|N/A|23345311moccasin0268|medium|Pound|Unknown|11|antibarationpriought| +13706|AAAAAAAAKIFDAAAA|1997-10-27|2000-10-26|Advisory, big revenues withdraw stupid instances; satisfactory, personal clients must seek h|7.05|4.30|2001001|amalgimporto #1|1|accessories|2|Men|98|eingn st|petite|589493seashell345061|sky|Bundle|Unknown|24|callybarationpriought| +13707|AAAAAAAAKIFDAAAA|2000-10-27||Boys apply certain troops. Ships assist sad effects. Again single factors shall think even. Reliable, united soldiers would respect past small songs; students see available, open proposals. |7.91|4.66|2001001|amalgedu pack #2|1|womens|4|Shoes|98|eingn st|petite|589493seashell345061|violet|Bunch|Unknown|57|ationbarationpriought| +13708|AAAAAAAAMIFDAAAA|1997-10-27|1999-10-27|Nearly thick cards feel. Participants may get for a damages. Standards drain other experiments. Other, white proposals revert as yet economic steps. Si|8.83|2.82|6002005|importocorp #5|2|diamonds|6|Jewelry|289|n steingable|N/A|3plum230356294296863|sandy|Cup|Unknown|5|eingbarationpriought| +13709|AAAAAAAAMIFDAAAA|1999-10-28|2001-10-26|Young, demanding mice endorse. Di|2.81|2.82|6002005|univmaxi #10|2|pools|8|Sports|74|n steingable|N/A|3plum230356294296863|linen|Box|Unknown|24|n stbarationpriought| +13710|AAAAAAAAMIFDAAAA|2001-10-27||Old, considerable groups must provide broadly vast, ancient territories. Large categories mus|1.27|1.00|1003001|exportiamalg #1|2|maternity|1|Women|60|barcally|extra large|3plum230356294296863|tan|Cup|Unknown|46|baroughtationpriought| +13711|AAAAAAAAPIFDAAAA|1997-10-27||Meanwhile industrial lakes must not check therefore. Then various customers s|2.78|2.50|10011010|amalgamalgamalg #10|11|disk drives|10|Electronics|884|eseeingeing|N/A|6misty27120227167078|sienna|Tsp|Unknown|49|oughtoughtationpriought| +13712|AAAAAAAAAJFDAAAA|1997-10-27|2000-10-26|Then sure casualties shall not know well old, important areas. Well central drugs might sta|8.83|4.41|6016001|corpbrand #1|16|consignment|6|Jewelry|312|ableoughtpri|N/A|15ivory8963530692931|misty|Dozen|Unknown|27|ableoughtationpriought| +13713|AAAAAAAAAJFDAAAA|2000-10-27||Then sure casualties shall not know well old, important areas. Well central drugs might sta|5.82|2.67|6016001|namelessbrand #4|8|lighting|7|Home|504|esebaranti|N/A|07blush7312711326693|pale|Pallet|Unknown|7|prioughtationpriought| +13714|AAAAAAAACJFDAAAA|1997-10-27|1999-10-27|Simple, appropriate films begin romantic stages. Again mental frames occupy at the terms. Later british titles must sound cities. Children shall no|3.60|2.48|5003001|exportischolar #1|3|pop|5|Music|561|oughtcallyanti|N/A|4844495438867navajo2|wheat|Pound|Unknown|10|eseoughtationpriought| +13715|AAAAAAAACJFDAAAA|1999-10-28|2001-10-26|Simple, appropriate films begin romantic stages. Again mental frames occupy at the terms. Later british titles must sound cities. Children shall no|2.76|2.48|5003001|exportiamalg #2|3|maternity|1|Women|606|oughtcallyanti|petite|4844495438867navajo2|sandy|Carton|Unknown|23|antioughtationpriought| +13716|AAAAAAAACJFDAAAA|2001-10-27||Useful pupils cannot move more common clouds. Particular, teenage parties say around dual leads. Human f|8.51|2.48|5003001|edu packexporti #1|3|school-uniforms|3|Children|606|oughtcallyanti|petite|4844495438867navajo2|plum|Cup|Unknown|5|callyoughtationpriought| +13717|AAAAAAAAFJFDAAAA|1997-10-27||Soon royal chairs think relevant, main signals. Armed historians set occasions. Continuous flowers arrange today; th|4.36|2.52|10014016|edu packamalgamalg #16|14|automotive|10|Electronics|182|ableeingought|N/A|8orange2178293567905|lavender|Dozen|Unknown|60|ationoughtationpriought| +13718|AAAAAAAAGJFDAAAA|1997-10-27|2000-10-26|Aspects appoint eligible, black authorities. Levels may not act far old, immediate stations. Left, critical hea|8.11|4.37|9015003|scholarunivamalg #3|15|fiction|9|Books|277|ationationable|N/A|powder28383689313766|plum|Each|Unknown|97|eingoughtationpriought| +13719|AAAAAAAAGJFDAAAA|2000-10-27||Local fees used to ease again. Important strategies used to arise very level, attractive times. Regularly competent arms must not keep readily nuclear fee|0.52|4.37|9015003|exportischolar #2|3|pop|5|Music|277|ationationable|N/A|powder28383689313766|spring|Pallet|Unknown|23|n stoughtationpriought| +13720|AAAAAAAAIJFDAAAA|1997-10-27|1999-10-27|Things ought to catch. Quick men know likely, scottish values. Chief objectives should strengthen never pieces. Suspicious, social genes h|5.13|2.41|3004001|edu packexporti #1|4|school-uniforms|3|Children|861|oughtcallyeing|petite|256920187cornsilk669|puff|Dozen|Unknown|49|barableationpriought| +13721|AAAAAAAAIJFDAAAA|1999-10-28|2001-10-26|Free, comfortable lessons can go almost general forces. Much ancient circumstances should hand as alongside a colours; dead courts would know efforts|3.11|1.86|6001002|amalgcorp #2|4|birdal|6|Jewelry|284|oughtcallyeing|N/A|162798736lemon086579|turquoise|Bunch|Unknown|95|oughtableationpriought| +13722|AAAAAAAAIJFDAAAA|2001-10-27||Free, comfortable lessons can go almost general forces. Much ancient circumstances should hand as alongside a colours; dead courts would know efforts|1.38|1.24|8016003|corpmaxi #3|16|golf|8|Sports|370|barationpri|N/A|162798736lemon086579|green|Box|Unknown|87|ableableationpriought| +13723|AAAAAAAALJFDAAAA|1997-10-27||Detailed, associated police wait already above new opponents; economic individuals will need lights. British, c|2.10|0.75|4003002|exportiedu pack #2|3|kids|4|Shoes|825|antiableeing|small|7648278172629lace991|slate|Ounce|Unknown|7|priableationpriought| +13724|AAAAAAAAMJFDAAAA|1997-10-27|2000-10-26|Workers cannot settle there new camps. Likely leaders could face directly up a cars. Services will work in a needs. Youn|2.17|1.08|2002001|importoimporto #1|2|shirts|2|Men|78|eingation|medium|6rosy939818749493556|maroon|Tbl|Unknown|13|eseableationpriought| +13725|AAAAAAAAMJFDAAAA|2000-10-27||Workers cannot settle there new camps. Likely leaders could face directly up a cars. Services will work in a needs. Youn|42.66|1.08|2002001|namelessunivamalg #7|2|scanners|10|Electronics|78|eingation|N/A|6salmon7093947188460|sky|Cup|Unknown|5|antiableationpriought| +13726|AAAAAAAAOJFDAAAA|1997-10-27|1999-10-27|As recent companies used to assist a |4.23|2.83|5004001|edu packscholar #1|4|classical|5|Music|928|eingablen st|N/A|sandy438415543945595|purple|Unknown|Unknown|24|callyableationpriought| +13727|AAAAAAAAOJFDAAAA|1999-10-28|2001-10-26|Generally |1.11|2.83|5004001|exportiimporto #2|4|pants|2|Men|928|eingablen st|small|1023895023metallic78|smoke|Lb|Unknown|53|ationableationpriought| +13728|AAAAAAAAOJFDAAAA|2001-10-27||Generally |2.92|2.83|5004001|importoexporti #1|4|infants|3|Children|103|eingablen st|medium|1023895023metallic78|steel|Bundle|Unknown|47|eingableationpriought| +13729|AAAAAAAABKFDAAAA|1997-10-27||Females sleep here. Short, current concerns help at least far eager cl|2.48|1.51|3002002|importoexporti #2|2|infants|3|Children|732|ablepriation|large|19256559bisque847699|papaya|Case|Unknown|22|n stableationpriought| +13730|AAAAAAAACKFDAAAA|1997-10-27|2000-10-26|Rather previous argume|2.77|1.41|5004001|edu packscholar #1|4|classical|5|Music|265|anticallyable|N/A|87sandy7461684733127|rose|Pallet|Unknown|75|barpriationpriought| +13731|AAAAAAAACKFDAAAA|2000-10-27||Strongly evident forests fancy as english, white g|3.13|1.25|7002002|importobrand #2|4|bedding|7|Home|393|anticallyable|N/A|086160200hot12650390|turquoise|Ounce|Unknown|15|oughtpriationpriought| +13732|AAAAAAAAEKFDAAAA|1997-10-27|1999-10-27|Young blacks might answer here great factors. Shares will not cond|0.35|0.29|9007005|brandmaxi #5|7|reference|9|Books|68|eingcally|N/A|0magenta052595798459|ghost|Pallet|Unknown|90|ablepriationpriought| +13733|AAAAAAAAEKFDAAAA|1999-10-28|2001-10-26|Galleries must come closely instead objective supporters. Communications act slowly mainly british words. Nuclear, aware options ought to put even important weeks. Green, simple hundreds me|5.65|1.75|9007005|namelessunivamalg #15|8|scanners|10|Electronics|68|eingcally|N/A|0magenta052595798459|puff|Each|Unknown|2|pripriationpriought| +13734|AAAAAAAAEKFDAAAA|2001-10-27||Days stop small, complex merchants. Ancient makers shift. Civil, broad qualifications include forms. Finally nuclear steps u|5.12|2.86|2004001|edu packimporto #1|4|sports-apparel|2|Men|68|eingcally|small|177sky18197180583929|tan|Pallet|Unknown|41|esepriationpriought| +13735|AAAAAAAAHKFDAAAA|1997-10-27||For once ashamed estates collect. On board local eyes would work well still tough visitors. Further following papers shall not mean just somehow late points. Women shall not |4.88|4.19|2003002|exportiimporto #2|3|pants|2|Men|896|callyn steing|medium|916antique7123983595|purple|Case|Unknown|28|antipriationpriought| +13736|AAAAAAAAIKFDAAAA|1997-10-27|2000-10-26|Enough digital women may produce straight o|4.59|1.65|5001001|amalgscholar #1|1|rock|5|Music|479|n stationese|N/A|470496737olive263772|purple|Gross|Unknown|59|callypriationpriought| +13737|AAAAAAAAIKFDAAAA|2000-10-27||Enough digital women may produce straight o|14.03|1.65|2001002|amalgimporto #2|1|accessories|2|Men|479|n stationese|petite|0328peach26464501419|steel|Lb|Unknown|32|ationpriationpriought| +13738|AAAAAAAAKKFDAAAA|1997-10-27|1999-10-27|Heavy goods will not save paintings; long others work; capital books used |4.13|1.69|5002001|importoscholar #1|2|country|5|Music|117|ationoughtought|N/A|8850sienna0045979766|midnight|Unknown|Unknown|26|eingpriationpriought| +13739|AAAAAAAAKKFDAAAA|1999-10-28|2001-10-26|Heavy goods will not save paintings; long others work; capital books used |34.31|1.69|9016012|corpunivamalg #12|2|mystery|9|Books|117|ationoughtought|N/A|866973592399770puff0|seashell|Gram|Unknown|39|n stpriationpriought| +13740|AAAAAAAAKKFDAAAA|2001-10-27||Heavy goods will not save paintings; long others work; capital books used |1.56|0.88|9016012|exportischolar #1|2|pop|5|Music|117|ationoughtought|N/A|6717579765486spring1|indian|Tbl|Unknown|46|bareseationpriought| +13741|AAAAAAAANKFDAAAA|1997-10-27||About other wishes say currently like a parents. Small, italian fields will sound etc commercial foundations. Mines will not receive so full jobs. Elected, long employees should want |5.22|2.40|6014006|edu packbrand #6|14|estate|6|Jewelry|388|eingeingpri|N/A|607078698513sienna67|pale|Oz|Unknown|85|oughteseationpriought| +13742|AAAAAAAAOKFDAAAA|1997-10-27|2000-10-26|Silver, rural children get sometimes. Children cannot limit circumstances. Still similar players should work highest able agricultural techniq|7.04|5.70|9008003|namelessmaxi #3|8|romance|9|Books|559|n stantianti|N/A|5356028487922powder1|khaki|Carton|Unknown|32|ableeseationpriought| +13743|AAAAAAAAOKFDAAAA|2000-10-27||Silver, rural children get sometimes. Children cannot limit circumstances. Still similar players should work highest able agricultural techniq|3.53|1.87|5002002|importoscholar #2|2|country|5|Music|559|n stantianti|N/A|5356028487922powder1|sienna|Oz|Unknown|27|prieseationpriought| +13744|AAAAAAAAALFDAAAA|1997-10-27|1999-10-27|Studies used to issu|0.13|0.10|5004001|edu packscholar #1|4|classical|5|Music|297|ationn stable|N/A|steel827591946400222|blush|Case|Unknown|20|eseeseationpriought| +13745|AAAAAAAAALFDAAAA|1999-10-28|2001-10-26|Close considerable rooms include even |3.99|2.79|5004001|namelessnameless #6|8|outdoor|8|Sports|843|prieseeing|N/A|steel827591946400222|pale|Bunch|Unknown|7|antieseationpriought| +13746|AAAAAAAAALFDAAAA|2001-10-27||Virtually accessible muscles might not stop. Entirely wide forces must help bands; married, independent things direct schools. Only ready votes may not see anyway important, europe|9.83|4.12|3001001|amalgexporti #1|8|newborn|3|Children|843|prieseeing|medium|steel827591946400222|salmon|Dram|Unknown|53|callyeseationpriought| +13747|AAAAAAAADLFDAAAA|1997-10-27||Sweet men begin very; expensive players would provide arrangements. Human, able scales remember also to a creatures. Incidents must enter. Regulations interfere about more great |6.97|4.87|7014008|edu packnameless #8|14|glassware|7|Home|816|callyoughteing|N/A|6732007882tomato5842|saddle|Tsp|Unknown|83|ationeseationpriought| +13748|AAAAAAAAELFDAAAA|1997-10-27|2000-10-26|Children lead independently as well positive banks. Constant accidents make subsequently in the teachers. Solidly serious weeks send very. Likely, political flames will accommodate now standards. Nego|6.44|3.99|7002009|importobrand #9|2|bedding|7|Home|177|ationationought|N/A|396906rose0820842234|light|Pallet|Unknown|7|eingeseationpriought| +13749|AAAAAAAAELFDAAAA|2000-10-27||Quite southern publishers used to lift absolutely. Goods would not speak only pla|1.39|0.45|1002002|importoamalg #2|2|fragrances|1|Women|214|eseoughtable|medium|396906rose0820842234|blush|Bundle|Unknown|24|n steseationpriought| +13750|AAAAAAAAGLFDAAAA|1997-10-27|1999-10-27|Yards can continue different, traditional requirements. However central conditions deliver. Real, other elections take always in the lists. Most bizarre rules should attend minut|4.86|4.17|6006007|corpcorp #7|6|rings|6|Jewelry|656|callyantically|N/A|26metallic7607705714|pale|Gross|Unknown|1|barantiationpriought| +13751|AAAAAAAAGLFDAAAA|1999-10-28|2001-10-26|Items will want also french characteristics; as marked ga|28.40|19.59|3001002|amalgexporti #2|6|newborn|3|Children|656|callyantically|extra large|36636499044pale64575|turquoise|Each|Unknown|1|oughtantiationpriought| +13752|AAAAAAAAGLFDAAAA|2001-10-27||Safe acts could trust little on the colours. Appropriate, large needs ought to begin labour, additional benefits; below ethical movies identify national, n|4.06|19.59|6011001|amalgbrand #1|6|semi-precious|6|Jewelry|179|callyantically|N/A|704828206623627pink2|purple|Ounce|Unknown|64|ableantiationpriought| +13753|AAAAAAAAJLFDAAAA|1997-10-27||Very odd circumstances will not learn after a topics. United workers would initiate now words; really good details ought to |1.15|0.83|4001002|amalgedu pack #2|1|womens|4|Shoes|330|barpripri|medium|7149navy259286787026|magenta|Lb|Unknown|26|priantiationpriought| +13754|AAAAAAAAKLFDAAAA|1997-10-27|2000-10-26|Victorian, artificial studies used to light just to a lives. British consequences shall collect substantially. As it were ruling ma|4.53|1.85|4003001|exportiedu pack #1|3|kids|4|Shoes|643|priesecally|medium|1844violet1885702914|pink|Lb|Unknown|27|eseantiationpriought| +13755|AAAAAAAAKLFDAAAA|2000-10-27||Victorian, artificial studies used to light just to a lives. British consequences shall collect substantially. As it were ruling ma|3.68|1.98|2003002|exportiimporto #2|3|pants|2|Men|643|priesecally|large|5127papaya5617834434|tan|Each|Unknown|10|antiantiationpriought| +13756|AAAAAAAAMLFDAAAA|1997-10-27|1999-10-27|Else irrelevant unions could create there friends. Main, public genes may find about a premises. Eye|1.00|0.63|2002001|importoimporto #1|2|shirts|2|Men|767|ationcallyation|extra large|8745khaki10351645355|seashell|Dram|Unknown|31|callyantiationpriought| +13757|AAAAAAAAMLFDAAAA|1999-10-28|2001-10-26|Else irrelevant unions could create there friends. Main, public genes may find about a premises. Eye|7.31|3.50|2002001|edu packscholar #2|2|classical|5|Music|393|prin stpri|N/A|2608264648934puff430|royal|Dozen|Unknown|55|ationantiationpriought| +13758|AAAAAAAAMLFDAAAA|2001-10-27||Else irrelevant unions could create there friends. Main, public genes may find about a premises. Eye|2.71|1.89|4004001|edu packedu pack #1|4|athletic|4|Shoes|171|oughtationought|petite|2165426ivory43203776|tan|Pound|Unknown|2|eingantiationpriought| +13759|AAAAAAAAPLFDAAAA|1997-10-27||Supporters may not ge|0.62|0.45|9014008|edu packunivamalg #8|14|sports|9|Books|656|callyantically|N/A|673salmon52772785150|red|Gross|Unknown|10|n stantiationpriought| +13760|AAAAAAAAAMFDAAAA|1997-10-27|2000-10-26|Possible, old failures could stand often modern terms. Rooms might write months. Photograp|4.26|1.32|8013001|exportimaxi #1|13|sailing|8|Sports|361|oughtcallypri|N/A|17560241white9988531|smoke|Gross|Unknown|2|barcallyationpriought| +13761|AAAAAAAAAMFDAAAA|2000-10-27||Possible, old failures could stand often modern terms. Rooms might write months. Photograp|5.16|1.32|1002002|importoamalg #2|13|fragrances|1|Women|634|oughtcallypri|medium|13618peach5964035482|gainsboro|Bunch|Unknown|1|oughtcallyationpriought| +13762|AAAAAAAACMFDAAAA|1997-10-27|1999-10-27|Central vehicles matter now at a companies; r|1.11|0.59|9009011|maximaxi #11|9|science|9|Books|155|antiantiought|N/A|27pink70621406833586|turquoise|Gross|Unknown|6|ablecallyationpriought| +13763|AAAAAAAACMFDAAAA|1999-10-28|2001-10-26|Reforms might think again eligible, tough decisions. Initial, popular achievements should apply lines. Dangerous rules shall give again developments; common stars can say less. High, electronic|4.83|2.22|9009011|amalgexporti #2|1|newborn|3|Children|406|callybarese|medium|27pink70621406833586|lemon|Ton|Unknown|9|pricallyationpriought| +13764|AAAAAAAACMFDAAAA|2001-10-27||Reforms might think again eligible, tough decisions. Initial, popular achievements should apply lines. Dangerous rules shall give again developments; common stars can say less. High, electronic|1.61|2.22|2003001|exportiimporto #1|3|pants|2|Men|415|callybarese|small|27pink70621406833586|salmon|Unknown|Unknown|53|esecallyationpriought| +13765|AAAAAAAAFMFDAAAA|1997-10-27||Arbitrary police dem|7.88|5.35|7011006|amalgnameless #6|11|accent|7|Home|631|oughtprically|N/A|8244022salmon8044063|purple|Tsp|Unknown|39|anticallyationpriought| +13766|AAAAAAAAGMFDAAAA|1997-10-27|2000-10-26|Political letters could consider of course new, personal problems. Of cour|6.84|4.51|2001001|amalgimporto #1|1|accessories|2|Men|177|ationationought|economy|95753107827hot574385|chartreuse|Oz|Unknown|6|callycallyationpriought| +13767|AAAAAAAAGMFDAAAA|2000-10-27||Professional, likely elections collect children. New figures would continue however well only rates. Also concerned territories cover at all new, star losses. Much black exports would develop liabili|4.21|4.51|2001001|amalgmaxi #4|1|arts|9|Books|177|ationationought|N/A|95753107827hot574385|wheat|Cup|Unknown|8|ationcallyationpriought| +13768|AAAAAAAAIMFDAAAA|1997-10-27|1999-10-27|Almost major songs afford big characters. International|3.54|1.30|8016009|corpmaxi #9|16|golf|8|Sports|406|callybarese|N/A|purple46385656972944|royal|Lb|Unknown|54|eingcallyationpriought| +13769|AAAAAAAAIMFDAAAA|1999-10-28|2001-10-26|Public, local eyes get much rules. Strong materials could see at least even foreign countries. Others will not rise alone peaceful, regular results; intimate, annual opportunities see companies. |1.43|1.30|3004002|edu packexporti #2|4|school-uniforms|3|Children|406|callybarese|N/A|06559chocolate518684|misty|Oz|Unknown|7|n stcallyationpriought| +13770|AAAAAAAAIMFDAAAA|2001-10-27||Public, local eyes get much rules. Strong materials could see at least even foreign countries. Others will not rise alone peaceful, regular results; intimate, annual opportunities see companies. |3.48|3.06|4002001|importoedu pack #1|2|mens|4|Shoes|406|callybarese|economy|06559chocolate518684|seashell|Cup|Unknown|16|barationationpriought| +13771|AAAAAAAALMFDAAAA|1997-10-27||Professional, local things used to think nervously radical members. Able companies achieve much growing goods; u|9.67|2.99|3001002|amalgexporti #2|1|newborn|3|Children|199|n stn stought|medium|193708055375653peru2|wheat|Box|Unknown|4|oughtationationpriought| +13772|AAAAAAAAMMFDAAAA|1997-10-27|2000-10-26|There regulatory forms may kill sadly sure areas. Fine, permanent plans see in a plants. National, spatial characteristics shall not introduce below t|2.61|1.93|5001001|amalgscholar #1|1|rock|5|Music|46|callyese|N/A|6798peru426491261898|spring|Carton|Unknown|77|ableationationpriought| +13773|AAAAAAAAMMFDAAAA|2000-10-27||More familiar users like; various standards tell more. Only bloody cases can draw; main, other cigarettes discuss into the attacks; mysterious bodies shall purchase h|95.54|59.23|5001001|exportiexporti #2|3|toddlers|3|Children|253|callyese|small|07075thistle67736308|peru|Ounce|Unknown|17|priationationpriought| +13774|AAAAAAAAOMFDAAAA|1997-10-27|1999-10-27|So long times will hear; |1.09|0.75|7013005|exportinameless #5|13|wallpaper|7|Home|16|callyought|N/A|064839532turquoise78|peru|Bunch|Unknown|65|eseationationpriought| +13775|AAAAAAAAOMFDAAAA|1999-10-28|2001-10-26|Children climb twice. Seconds make he|3.67|0.75|4001002|amalgedu pack #2|13|womens|4|Shoes|16|callyought|medium|3678387684smoke44397|rose|Cup|Unknown|31|antiationationpriought| +13776|AAAAAAAAOMFDAAAA|2001-10-27||Children climb twice. Seconds make he|0.65|0.31|3003001|exportiexporti #1|13|toddlers|3|Children|896|callyought|medium|3678387684smoke44397|salmon|Cup|Unknown|16|callyationationpriought| +13777|AAAAAAAABNFDAAAA|1997-10-27||As complete sorts may not facilitate homes. Existi|4.50|2.65|5004002|edu packscholar #2|4|classical|5|Music|225|antiableable|N/A|39614950navy98349764|tomato|Ton|Unknown|9|ationationationpriought| +13778|AAAAAAAACNFDAAAA|1997-10-27|2000-10-26|Critical cities stop often. Earlier tender offenders make. Less than true drawings can provide elsewhere recently vulnerable days. |5.55|2.10|4002001|importoedu pack #1|2|mens|4|Shoes|632|ableprically|large|burnished33277167256|steel|Gross|Unknown|1|eingationationpriought| +13779|AAAAAAAACNFDAAAA|2000-10-27||Again incredible ri|9.70|4.94|4002001|amalgunivamalg #10|1|cameras|10|Electronics|243|ableprically|N/A|burnished33277167256|sky|Bundle|Unknown|54|n stationationpriought| +13780|AAAAAAAAENFDAAAA|1997-10-27|1999-10-27|Formally strategic ideas go exactly arguments. Senses|8.09|5.25|5001001|amalgscholar #1|1|rock|5|Music|316|callyoughtpri|N/A|thistle2879059284671|plum|Case|Unknown|69|bareingationpriought| +13781|AAAAAAAAENFDAAAA|1999-10-28|2001-10-26|Southern, new claims could repeat there stupid, political profits. Changes will reject of course different movies. Other events should silence. Tasks could keep pretty on a discussions. Othe|1.07|0.66|2002002|importoimporto #2|2|shirts|2|Men|976|callyoughtpri|large|thistle2879059284671|plum|Unknown|Unknown|46|oughteingationpriought| +13782|AAAAAAAAENFDAAAA|2001-10-27||Classic, average parts could not expect. Vulnerable, difficult sessions must not move also ambitious jobs. Important, ordinary figures see rather workers. General, recent sides should ins|1.50|0.66|2002002|brandcorp #7|2|pendants|6|Jewelry|976|callyoughtpri|N/A|936098cornflower1303|papaya|Case|Unknown|38|ableeingationpriought| +13783|AAAAAAAAHNFDAAAA|1997-10-27||Surely elderly gains send further friends. Real, uncertain materials use hard|8.64|6.65|9014008|edu packunivamalg #8|14|sports|9|Books|187|ationeingought|N/A|royal781748265658329|seashell|Lb|Unknown|10|prieingationpriought| +13784|AAAAAAAAINFDAAAA|1997-10-27|2000-10-26|New eyes change political, new activities.|9.10|6.18|8012005|importomaxi #5|12|guns|8|Sports|421|oughtableese|N/A|7pink555289234557164|linen|Case|Unknown|28|eseeingationpriought| +13785|AAAAAAAAINFDAAAA|2000-10-27||New eyes change political, new activities.|5.73|6.18|8012005|exportiexporti #2|3|toddlers|3|Children|71|oughtation|petite|7pink555289234557164|yellow|Oz|Unknown|15|antieingationpriought| +13786|AAAAAAAAKNFDAAAA|1997-10-27|1999-10-27|Eyes will exchange fully emotions. Reports discuss social questions; still chief teeth provide there by the hopes. Red words live almost open stands. Words dash just only years. Rural, hungry place|7.33|2.27|4003001|exportiedu pack #1|3|kids|4|Shoes|173|priationought|large|blue7631067782619092|puff|Box|Unknown|24|callyeingationpriought| +13787|AAAAAAAAKNFDAAAA|1999-10-28|2001-10-26|Eyes will exchange fully emotions. Reports discuss social questions; still chief teeth provide there by the hopes. Red words live almost open stands. Words dash just only years. Rural, hungry place|4.70|2.27|4003001|amalgamalg #2|3|dresses|1|Women|173|priationought|medium|3658053380621papaya5|beige|Ounce|Unknown|25|ationeingationpriought| +13788|AAAAAAAAKNFDAAAA|2001-10-27||Alone small genes enjoy. Effective hospitals thank for a beds. Very civil troops must squeeze more by the prices. Repeatedly irish words may keep without the patients; long years share const|5.59|2.27|1002001|importoamalg #1|2|fragrances|1|Women|173|priationought|medium|543509723steel758296|wheat|Pound|Unknown|22|eingeingationpriought| +13789|AAAAAAAANNFDAAAA|1997-10-27||Corporate, general events see outwards old feet. Early windows receive. Skills achieve scottish, wrong |98.36|73.77|8002010|importonameless #10|2|baseball|8|Sports|505|antibaranti|N/A|354901turquoise43895|navajo|Pound|Unknown|3|n steingationpriought| +13790|AAAAAAAAONFDAAAA|1997-10-27|2000-10-26|Unable, low advertisements realize at a jobs. Other, central consumers must give often long, |3.43|1.06|3003001|exportiexporti #1|3|toddlers|3|Children|141|oughteseought|extra large|8511376purple3447559|sky|Unknown|Unknown|33|barn stationpriought| +13791|AAAAAAAAONFDAAAA|2000-10-27||Unable, low advertisements realize at a jobs. Other, central consumers must give often long, |0.43|0.17|2004002|edu packimporto #2|3|sports-apparel|2|Men|255|oughteseought|medium|8511376purple3447559|turquoise|Unknown|Unknown|42|oughtn stationpriought| +13792|AAAAAAAAAOFDAAAA|1997-10-27|1999-10-27|Today heavy owners continue from a interests. Scottish benefits could say everyday standards. Ashamed days begin for sure indian reasons. Good st|96.77|29.03|4001001|amalgedu pack #1|1|womens|4|Shoes|313|prioughtpri|medium|spring32479857720974|burnished|N/A|Unknown|19|ablen stationpriought| +13793|AAAAAAAAAOFDAAAA|1999-10-28|2001-10-26|Today heavy owners continue from a interests. Scottish benefits could say everyday standards. Ashamed days begin for sure indian reasons. Good st|0.37|0.18|3002002|importoexporti #2|1|infants|3|Children|68|prioughtpri|medium|spring32479857720974|steel|Each|Unknown|18|prin stationpriought| +13794|AAAAAAAAAOFDAAAA|2001-10-27||Today heavy owners continue from a interests. Scottish benefits could say everyday standards. Ashamed days begin for sure indian reasons. Good st|75.75|53.78|6012003|importobrand #3|1|costume|6|Jewelry|526|prioughtpri|N/A|spring32479857720974|saddle|Cup|Unknown|38|esen stationpriought| +13795|AAAAAAAADOFDAAAA|1997-10-27||Goals should find however on a samples. Accidents used to blind alone local days. Reasons depend at least against a accident|0.97|0.83|2001002|amalgimporto #2|1|accessories|2|Men|262|ablecallyable|petite|253566wheat180502894|sienna|Oz|Unknown|18|antin stationpriought| +13796|AAAAAAAAEOFDAAAA|1997-10-27|2000-10-26|All early developments may release occasional sides. More dead rules sho|2.51|1.58|3002001|importoexporti #1|2|infants|3|Children|116|callyoughtought|N/A|072832090572blue8811|peru|Dram|Unknown|62|callyn stationpriought| +13797|AAAAAAAAEOFDAAAA|2000-10-27||Alternatively complex husbands decipher n|7.61|4.48|3002001|corpmaxi #8|2|golf|8|Sports|663|pricallycally|N/A|072832090572blue8811|tomato|Dram|Unknown|24|ationn stationpriought| +13798|AAAAAAAAGOFDAAAA|1997-10-27|1999-10-27|About different purposes could convey confident, logical years. Hastily financial branches hope therefore vital fans. Ban|4.15|3.27|4004001|edu packedu pack #1|4|athletic|4|Shoes|335|antipripri|petite|4798132tomato1490847|tan|Box|Unknown|17|eingn stationpriought| +13799|AAAAAAAAGOFDAAAA|1999-10-28|2001-10-26|Vast metals help over european resources. Virtually fresh allies provide thus public steps. Valuable accidents|3.22|2.18|4004001|edu packcorp #8|4|bracelets|6|Jewelry|346|callyesepri|N/A|4798132tomato1490847|purple|Cup|Unknown|20|n stn stationpriought| +13800|AAAAAAAAGOFDAAAA|2001-10-27||Vast metals help over european resources. Virtually fresh allies provide thus public steps. Valuable accidents|4.61|2.58|3002001|importoexporti #1|2|infants|3|Children|344|callyesepri|extra large|1522221953016wheat43|rose|Unknown|Unknown|21|barbareingpriought| +13801|AAAAAAAAJOFDAAAA|1997-10-27||Mental, statutory flowers ought to learn here. So superb thousands control quite isolated results; administrative, good citi|56.60|16.98|5002002|importoscholar #2|2|country|5|Music|403|pribarese|N/A|365smoke349994109830|pink|Unknown|Unknown|2|oughtbareingpriought| +13802|AAAAAAAAKOFDAAAA|1997-10-27|2000-10-26|Plain females sing against a sections. Times live yet social, difficult systems; for instance liberal areas think of course. Governors take darling, old occupations. Long politicians develop wit|2.50|1.65|6016003|corpbrand #3|16|consignment|6|Jewelry|330|barpripri|N/A|84593365110red562919|puff|Oz|Unknown|23|ablebareingpriought| +13803|AAAAAAAAKOFDAAAA|2000-10-27||Plain females sing against a sections. Times live yet social, difficult systems; for instance liberal areas think of course. Governors take darling, old occupations. Long politicians develop wit|3.63|1.65|4002002|importoedu pack #2|2|mens|4|Shoes|300|barpripri|small|84593365110red562919|smoke|Case|Unknown|39|pribareingpriought| +13804|AAAAAAAAMOFDAAAA|1997-10-27|1999-10-27|Wrong, vague margins rise good, efficient powers. New, single particles ought to demonstrate again young, cheerful drugs; probably old years view so. Mental purposes ought to continue appr|9.35|7.85|7009005|maxibrand #5|9|mattresses|7|Home|289|n steingable|N/A|1388sandy41420677012|cream|Pound|Unknown|11|esebareingpriought| +13805|AAAAAAAAMOFDAAAA|1999-10-28|2001-10-26|Wrong, vague margins rise good, efficient powers. New, single particles ought to demonstrate again young, cheerful drugs; probably old years view so. Mental purposes ought to continue appr|2.19|7.85|6001006|amalgcorp #6|1|birdal|6|Jewelry|289|n steingable|N/A|7705white22902921358|sandy|Unknown|Unknown|33|antibareingpriought| +13806|AAAAAAAAMOFDAAAA|2001-10-27||Wrong, vague margins rise good, efficient powers. New, single particles ought to demonstrate again young, cheerful drugs; probably old years view so. Mental purposes ought to continue appr|6.49|2.07|6001006|exportischolar #1|1|pop|5|Music|289|n steingable|N/A|741657854smoke142732|peru|Pound|Unknown|20|callybareingpriought| +13807|AAAAAAAAPOFDAAAA|1997-10-27||English, economic things get as by a parts. Values take wide, american networks. Pa|8.31|4.90|1002002|importoamalg #2|2|fragrances|1|Women|158|eingantiought|medium|66179salmon704210484|chocolate|Ton|Unknown|47|ationbareingpriought| +13808|AAAAAAAAAPFDAAAA|1997-10-27|2000-10-26|Accurate organisations run firmly. English executives concern yet minutes. Political, weak occasions ask. Therefore gay years combine years; loo|6.55|3.99|6009001|maxicorp #1|9|womens watch|6|Jewelry|809|n stbareing|N/A|51670073white4422102|smoke|Tsp|Unknown|9|eingbareingpriought| +13809|AAAAAAAAAPFDAAAA|2000-10-27||Accurate organisations run firmly. English executives concern yet minutes. Political, weak occasions ask. Therefore gay years combine years; loo|1.06|0.78|6009001|brandbrand #10|9|decor|7|Home|809|n stbareing|N/A|51670073white4422102|chartreuse|Oz|Unknown|51|n stbareingpriought| +13810|AAAAAAAACPFDAAAA|1997-10-27|1999-10-27|Decis|8.08|2.50|1001001|amalgamalg #1|1|dresses|1|Women|116|callyoughtought|medium|9light53841950731843|violet|Tsp|Unknown|55|baroughteingpriought| +13811|AAAAAAAACPFDAAAA|1999-10-28|2001-10-26|Decis|87.23|47.10|7008002|namelessbrand #2|8|lighting|7|Home|494|esen stese|N/A|3048096378791sky0248|pale|Gram|Unknown|14|oughtoughteingpriought| +13812|AAAAAAAACPFDAAAA|2001-10-27||Decis|3.21|47.10|8006003|corpnameless #3|8|football|8|Sports|669|n stcallycally|N/A|3048096378791sky0248|pink|Tbl|Unknown|8|ableoughteingpriought| +13813|AAAAAAAAFPFDAAAA|1997-10-27||Scots should not tell before the eyes. About different months constitute valuable characters. Once again false memories can go colleges. Appropriate households|1.99|0.99|2002002|importoimporto #2|2|shirts|2|Men|587|ationeinganti|small|8882431272purple6715|plum|Carton|Unknown|11|prioughteingpriought| +13814|AAAAAAAAGPFDAAAA|1997-10-27|2000-10-26|Strong arrangements include. Possible, united muscles bring therefore in a changes. Personal, other pp. provide twin costs. Reasonable parts would teac|6.82|4.77|3002001|importoexporti #1|2|infants|3|Children|854|eseantieing|large|95seashell7690275148|tomato|N/A|Unknown|32|eseoughteingpriought| +13815|AAAAAAAAGPFDAAAA|2000-10-27||Strategic, young goods could not put even whole books. Delegates pay pri|2.52|4.77|10004012|edu packunivamalg #12|4|audio|10|Electronics|854|eseantieing|N/A|323244042gainsboro20|purple|Each|Unknown|30|antioughteingpriought| +13816|AAAAAAAAIPFDAAAA|1997-10-27|1999-10-27|Right, ultimate equations work offic|6.98|5.65|5004001|edu packscholar #1|4|classical|5|Music|553|priantianti|N/A|snow9832364412890657|yellow|Ounce|Unknown|22|callyoughteingpriought| +13817|AAAAAAAAIPFDAAAA|1999-10-28|2001-10-26|Fond conditions say far. Hence practical concentrations used to play he|8.18|5.65|8016008|corpmaxi #8|4|golf|8|Sports|553|priantianti|N/A|lace7422351521891233|salmon|Case|Unknown|66|ationoughteingpriought| +13818|AAAAAAAAIPFDAAAA|2001-10-27||Fond conditions say far. Hence practical concentrations used to play he|5.82|5.65|3003001|exportiexporti #1|4|toddlers|3|Children|553|priantianti|large|13842625turquoise828|seashell|Pound|Unknown|48|eingoughteingpriought| +13819|AAAAAAAALPFDAAAA|1997-10-27||Usefully great years cannot look. Affairs phone clearly in a weeks. Hands |8.21|7.30|3001002|amalgexporti #2|1|newborn|3|Children|272|ableationable|large|09366moccasin6237936|papaya|Gross|Unknown|11|n stoughteingpriought| +13820|AAAAAAAAMPFDAAAA|1997-10-27|2000-10-26|Close agencies used to see. Certainly continuing eyes could not object etc thin broad organ|7.91|4.66|4001001|amalgedu pack #1|1|womens|4|Shoes|541|oughteseanti|extra large|9074483069355white23|orange|Gram|Unknown|42|barableeingpriought| +13821|AAAAAAAAMPFDAAAA|2000-10-27||Ancient, traditional matters will say surprisingly hard years. Clear courses renew. Perhaps industrial statements ought to digest in a alternatives; obviously interesting women introdu|3.98|4.66|10004001|edu packunivamalg #1|4|audio|10|Electronics|541|oughteseanti|N/A|9074483069355white23|yellow|Lb|Unknown|91|oughtableeingpriought| +13822|AAAAAAAAOPFDAAAA|1997-10-27|1999-10-27|Perfect, successful arms impress here further old charges. Consequently good sales use only. Rules stop still quite large pu|6.66|5.99|5003001|exportischolar #1|3|pop|5|Music|327|ationablepri|N/A|3829331peru872703571|gainsboro|Bundle|Unknown|46|ableableeingpriought| +13823|AAAAAAAAOPFDAAAA|1999-10-28|2001-10-26|Perfect, successful arms impress here further old charges. Consequently good sales use only. Rules stop still quite large pu|9.53|8.38|6004008|edu packcorp #8|3|bracelets|6|Jewelry|519|n stoughtanti|N/A|448papaya82521375102|cornsilk|Gross|Unknown|15|priableeingpriought| +13824|AAAAAAAAOPFDAAAA|2001-10-27||Perfect, successful arms impress here further old charges. Consequently good sales use only. Rules stop still quite large pu|4.71|8.38|8006005|corpnameless #5|3|football|8|Sports|519|n stoughtanti|N/A|3slate87771060786523|turquoise|Dozen|Unknown|1|eseableeingpriought| +13825|AAAAAAAABAGDAAAA|1997-10-27||Here possible nations could think with the ages. Weeks discuss of |2.48|1.53|8016006|corpmaxi #6|16|golf|8|Sports|59|n stanti|N/A|191619sandy344212622|plum|Gross|Unknown|99|antiableeingpriought| +13826|AAAAAAAACAGDAAAA|1997-10-27|2000-10-26|Friendly, efficient stands forget separately. Lega|7.38|5.31|8016007|corpmaxi #7|16|golf|8|Sports|231|oughtpriable|N/A|5373686072651rose034|peru|Each|Unknown|60|callyableeingpriought| +13827|AAAAAAAACAGDAAAA|2000-10-27||Large, advisory children carry properties. Other, current techniques can go occupational authors. Coming regulations depend goals. Central women might not indicate small, disciplinary consider|46.74|40.66|8016007|exportinameless #8|16|basketball|8|Sports|231|oughtpriable|N/A|17tomato793696269595|saddle|Lb|Unknown|65|ationableeingpriought| +13828|AAAAAAAAEAGDAAAA|1997-10-27|1999-10-27|Now holy affairs cannot close only as similar issues. Chur|0.42|0.24|2001001|amalgimporto #1|1|accessories|2|Men|147|ationeseought|petite|1saddle7596302527222|magenta|Cup|Unknown|4|eingableeingpriought| +13829|AAAAAAAAEAGDAAAA|1999-10-28|2001-10-26|Nations may not wait of course adults. Other, national applications|0.56|0.24|10001009|amalgunivamalg #9|1|cameras|10|Electronics|109|ationeseought|N/A|779873409450pale1787|rose|Gram|Unknown|2|n stableeingpriought| +13830|AAAAAAAAEAGDAAAA|2001-10-27||Nations may not wait of course adults. Other, national applications|6.11|0.24|2001001|amalgimporto #1|1|accessories|2|Men|109|ationeseought|small|779873409450pale1787|slate|Bunch|Unknown|37|barprieingpriought| +13831|AAAAAAAAHAGDAAAA|1997-10-27||Areas meet very sections; ministers sound urgently. Public, regulatory winners start much beautiful, trying considerations. Possible,|2.12|0.72|10002011|importounivamalg #11|2|camcorders|10|Electronics|175|antiationought|N/A|999013920lemon567542|yellow|Cup|Unknown|5|oughtprieingpriought| +13832|AAAAAAAAIAGDAAAA|1997-10-27|2000-10-26|B|2.51|1.90|7014003|edu packnameless #3|14|glassware|7|Home|43|priese|N/A|366salmon23624936439|sky|Pound|Unknown|11|ableprieingpriought| +13833|AAAAAAAAIAGDAAAA|2000-10-27||Firmly local premises shall not long recently empty minutes. Bad, individual nations will not choose als|4.06|1.90|7014003|edu packcorp #2|4|bracelets|6|Jewelry|43|priese|N/A|366salmon23624936439|smoke|Dozen|Unknown|1|priprieingpriought| +13834|AAAAAAAAKAGDAAAA|1997-10-27|1999-10-27|Anonymous waves may expand more interested, new projects; undoubtedly advanced or|2.82|1.55|5001001|amalgscholar #1|1|rock|5|Music|657|ationantically|N/A|powder85327053244164|seashell|Dozen|Unknown|1|eseprieingpriought| +13835|AAAAAAAAKAGDAAAA|1999-10-28|2001-10-26|Anonymous waves may expand more interested, new projects; undoubtedly advanced or|1.13|0.33|9010012|univunivamalg #12|10|travel|9|Books|20|barable|N/A|powder85327053244164|snow|Ounce|Unknown|67|antiprieingpriought| +13836|AAAAAAAAKAGDAAAA|2001-10-27||Anonymous waves may expand more interested, new projects; undoubtedly advanced or|3.17|0.33|9010012|amalgamalg #1|10|dresses|1|Women|20|barable|medium|turquoise03604128179|midnight|Gram|Unknown|37|callyprieingpriought| +13837|AAAAAAAANAGDAAAA|1997-10-27||Political times can develop gently members. Acceptable, m|0.29|0.20|3002002|importoexporti #2|2|infants|3|Children|135|antipriought|medium|2276003646862mint050|steel|Carton|Unknown|90|ationprieingpriought| +13838|AAAAAAAAOAGDAAAA|1997-10-27|2000-10-26|Streets stare only much respective twins. National, important branches move today outside upper children. Areas oug|3.81|1.75|7016009|corpnameless #9|16|furniture|7|Home|305|antibarpri|N/A|8995snow801181908277|lawn|Pound|Unknown|44|eingprieingpriought| +13839|AAAAAAAAOAGDAAAA|2000-10-27||Wonderful, possible arms depict home; obvious presents market single cases; please possible ways must not assert very apparent whole children. Inappropriate schools trace then vehemently comfortable r|9.70|3.00|2002002|importoimporto #2|16|shirts|2|Men|305|antibarpri|medium|8995snow801181908277|violet|Ton|Unknown|25|n stprieingpriought| +13840|AAAAAAAAABGDAAAA|1997-10-27|1999-10-27|Imp|4.09|2.69|1001001|amalgamalg #1|1|dresses|1|Women|165|anticallyought|economy|229violet57252636099|white|Oz|Unknown|2|bareseeingpriought| +13841|AAAAAAAAABGDAAAA|1999-10-28|2001-10-26|Ahead social forces may not|70.65|2.69|1001001|maxicorp #2|9|womens watch|6|Jewelry|165|anticallyought|N/A|53104153947lace41966|sienna|Ounce|Unknown|47|oughteseeingpriought| +13842|AAAAAAAAABGDAAAA|2001-10-27||Quite final eyes would not find from a plans. Electronic prices bite y|0.16|2.69|2002001|importoimporto #1|2|shirts|2|Men|224|eseableable|extra large|53104153947lace41966|dim|Unknown|Unknown|58|ableeseeingpriought| +13843|AAAAAAAADBGDAAAA|1997-10-27||Words review. Old, productive pieces can frighten for the most part certainly amazing things; organisations exist satisfactorily |64.05|21.13|2002002|importoimporto #2|2|shirts|2|Men|272|ableationable|medium|3navy878035570882359|misty|Oz|Unknown|92|prieseeingpriought| +13844|AAAAAAAAEBGDAAAA|1997-10-27|2000-10-26|Main, due rooms would come fairly likely, relevant cattle; players avoid otherwise eyes. Fans will not ban potentially. Literally religious peasants can endeavou|1.82|1.12|7002005|importobrand #5|2|bedding|7|Home|80|bareing|N/A|66535saddle197899095|tan|Oz|Unknown|48|eseeseeingpriought| +13845|AAAAAAAAEBGDAAAA|2000-10-27||Main, due rooms would come fairly likely, relevant cattle; players avoid otherwise eyes. Fans will not ban potentially. Literally religious peasants can endeavou|3.31|1.45|9003010|exportimaxi #10|2|computers|9|Books|80|bareing|N/A|66535saddle197899095|sandy|Bundle|Unknown|94|antieseeingpriought| +13846|AAAAAAAAGBGDAAAA|1997-10-27|1999-10-27|Sometimes good areas can ask virtually wonderful social holders. Overall, excellent aspects solve similar points; courses must let probably warm, good da|1.30|0.71|2004001|edu packimporto #1|4|sports-apparel|2|Men|131|oughtpriought|N/A|85998dark23090842929|red|Cup|Unknown|26|callyeseeingpriought| +13847|AAAAAAAAGBGDAAAA|1999-10-28|2001-10-26|Sometimes good areas can ask virtually wonderful social holders. Overall, excellent aspects solve similar points; courses must let probably warm, good da|5.05|0.71|2004001|edu packedu pack #2|4|athletic|4|Shoes|131|oughtpriought|small|3869926308yellow9897|violet|Ton|Unknown|25|ationeseeingpriought| +13848|AAAAAAAAGBGDAAAA|2001-10-27||Sometimes good areas can ask virtually wonderful social holders. Overall, excellent aspects solve similar points; courses must let probably warm, good da|0.64|0.71|9012001|importounivamalg #1|4|home repair|9|Books|131|oughtpriought|N/A|3869926308yellow9897|salmon|Bunch|Unknown|26|eingeseeingpriought| +13849|AAAAAAAAJBGDAAAA|1997-10-27||Systems permit; things give |3.81|2.24|8003010|exportinameless #10|3|basketball|8|Sports|606|callybarcally|N/A|7turquoise9339026452|smoke|Dozen|Unknown|33|n steseeingpriought| +13850|AAAAAAAAKBGDAAAA|1997-10-27|2000-10-26|Perso|6.90|4.14|10005013|scholarunivamalg #13|5|karoke|10|Electronics|286|callyeingable|N/A|63707344681424peru90|olive|Pound|Unknown|10|barantieingpriought| +13851|AAAAAAAAKBGDAAAA|2000-10-27||Critical, net conflicts would prevent neither on a patients. Separately medical prices destroy as possible measures. Machines might not cor|75.99|41.79|5001002|amalgscholar #2|1|rock|5|Music|286|callyeingable|N/A|721975369989puff9551|salmon|Bundle|Unknown|11|oughtantieingpriought| +13852|AAAAAAAAMBGDAAAA|1997-10-27|1999-10-27|Single, different auditors might end continually great boys. Thousands|4.88|1.85|10013015|exportiamalgamalg #15|13|stereo|10|Electronics|127|ationableought|N/A|80815mint40284487763|thistle|Gram|Unknown|1|ableantieingpriought| +13853|AAAAAAAAMBGDAAAA|1999-10-28|2001-10-26|Shares become formerly scottish fu|9.79|5.38|3002002|importoexporti #2|13|infants|3|Children|127|ationableought|extra large|80815mint40284487763|lawn|Dozen|Unknown|84|priantieingpriought| +13854|AAAAAAAAMBGDAAAA|2001-10-27||Poor, sexua|7.00|5.38|3002002|importoimporto #1|2|shirts|2|Men|560|ationableought|large|80815mint40284487763|turquoise|Box|Unknown|1|eseantieingpriought| +13855|AAAAAAAAPBGDAAAA|1997-10-27||Foreign days walk to a consequences. More major students read then. Never new stones will not see sophisticated funds. Words must not say also. Extra, full t|3.44|2.92|6003008|exporticorp #8|3|gold|6|Jewelry|296|callyn stable|N/A|999397491white925577|sandy|Lb|Unknown|15|antiantieingpriought| +13856|AAAAAAAAACGDAAAA|1997-10-27|2000-10-26|Alternative, desperate rules might talk only. Cases get recently. Smooth, public departments mean only c|6.41|3.39|1001001|amalgamalg #1|1|dresses|1|Women|170|barationought|small|2914179709powder8036|rose|Gram|Unknown|18|callyantieingpriought| +13857|AAAAAAAAACGDAAAA|2000-10-27||Alternative, desperate rules might talk only. Cases get recently. Smooth, public departments mean only c|6.60|5.14|7014008|edu packnameless #8|14|glassware|7|Home|170|barationought|N/A|797305snow7319978475|yellow|Lb|Unknown|41|ationantieingpriought| +13858|AAAAAAAACCGDAAAA|1997-10-27|1999-10-27|Countries must suggest evidently waters. Symbols would find. Dark savings get earl|16.01|5.60|5002001|importoscholar #1|2|country|5|Music|21|oughtable|N/A|34burlywood241451640|peach|Tsp|Unknown|32|eingantieingpriought| +13859|AAAAAAAACCGDAAAA|1999-10-28|2001-10-26|Famous friends will set really surprising acts. Laboratories purchase only. Commercial, old months can tell men. Highly inte|9.41|6.02|1004002|edu packamalg #2|2|swimwear|1|Women|21|oughtable|large|34burlywood241451640|burlywood|Carton|Unknown|51|n stantieingpriought| +13860|AAAAAAAACCGDAAAA|2001-10-27||Popular, environmental accounts should pick together well responsible papers. United circumstances shall turn. Actually long foods stay a bit. Afterwa|4.41|3.74|4001001|amalgedu pack #1|2|womens|4|Shoes|709|n stbaration|small|34burlywood241451640|rose|Gram|Unknown|66|barcallyeingpriought| +13861|AAAAAAAAFCGDAAAA|1997-10-27||Rather only relations must not enter hopelessly ideas. Players succeed again little national values. Patient, private schools see far small rights. Now certain changes create too. Local poles accept m|1.38|1.17|10003007|exportiunivamalg #7|3|dvd/vcr players|10|Electronics|83|prieing|N/A|48955261seashell6789|lemon|Gram|Unknown|20|oughtcallyeingpriought| +13862|AAAAAAAAGCGDAAAA|1997-10-27|2000-10-26|Main, bright eyebrows could not discriminate already always proper lines. Open, armed sites use with a firms. Forests may not explain clear, waiting prisoners. Ports s|98.13|45.13|6008007|namelesscorp #7|8|mens watch|6|Jewelry|331|oughtpripri|N/A|916252759195violet74|orchid|Ounce|Unknown|59|ablecallyeingpriought| +13863|AAAAAAAAGCGDAAAA|2000-10-27||Prominently special databases will develop very local, public|1.46|45.13|2001002|amalgimporto #2|8|accessories|2|Men|58|einganti|medium|75085602chartreuse35|slate|Ton|Unknown|9|pricallyeingpriought| +13864|AAAAAAAAICGDAAAA|1997-10-27|1999-10-27|Chief pers|4.92|1.91|8003005|exportinameless #5|3|basketball|8|Sports|518|eingoughtanti|N/A|lemon720729615132189|moccasin|Box|Unknown|25|esecallyeingpriought| +13865|AAAAAAAAICGDAAAA|1999-10-28|2001-10-26|Chief pers|1.39|1.91|6007002|brandcorp #2|3|pendants|6|Jewelry|518|eingoughtanti|N/A|31203349912556grey85|gainsboro|Oz|Unknown|51|anticallyeingpriought| +13866|AAAAAAAAICGDAAAA|2001-10-27||Religious, ordinary guards ought to see otherwise old eyes. Estimated, public phases could work about sti|8.13|5.44|6007002|importoscholar #1|3|country|5|Music|518|eingoughtanti|N/A|31203349912556grey85|peach|Bunch|Unknown|19|callycallyeingpriought| +13867|AAAAAAAALCGDAAAA|1997-10-27||Other changes claim just with the ways. Other ways believe men; national, special daughters head fine, left movements. Well military estates care. More extens|1.38|1.13|7006008|corpbrand #8|6|rugs|7|Home|332|ablepripri|N/A|62651047thistle15981|pale|Pallet|Unknown|43|ationcallyeingpriought| +13868|AAAAAAAAMCGDAAAA|1997-10-27|2000-10-26|Interior governments ought to shore social women. Private losses ought to bother with a problems. Over local resources see|53.46|34.74|3002001|importoexporti #1|2|infants|3|Children|287|ationeingable|economy|moccasin140853459583|rose|Dram|Unknown|63|eingcallyeingpriought| +13869|AAAAAAAAMCGDAAAA|2000-10-27||Interior governments ought to shore social women. Private losses ought to bother with a problems. Over local resources see|0.79|34.74|2003002|exportiimporto #2|2|pants|2|Men|790|barn station|medium|moccasin140853459583|linen|Dram|Unknown|80|n stcallyeingpriought| +13870|AAAAAAAAOCGDAAAA|1997-10-27|1999-10-27|Hands could not see. Long-term types must not remember economic, urban advances. Offers say then similar, fine years. About massive wages cannot get as by a discussions. Part-time neighbouring bonds o|9.51|7.70|2001001|amalgimporto #1|1|accessories|2|Men|542|ableeseanti|extra large|679891648356759pale0|puff|Unknown|Unknown|27|barationeingpriought| +13871|AAAAAAAAOCGDAAAA|1999-10-28|2001-10-26|Hands could not see. Long-term types must not remember economic, urban advances. Offers say then similar, fine years. About massive wages cannot get as by a discussions. Part-time neighbouring bonds o|3.23|7.70|10011017|amalgamalgamalg #17|1|disk drives|10|Electronics|542|ableeseanti|N/A|027453436910spring27|powder|N/A|Unknown|5|oughtationeingpriought| +13872|AAAAAAAAOCGDAAAA|2001-10-27||Hands could not see. Long-term types must not remember economic, urban advances. Offers say then similar, fine years. About massive wages cannot get as by a discussions. Part-time neighbouring bonds o|4.35|3.52|3004001|edu packexporti #1|1|school-uniforms|3|Children|443|ableeseanti|petite|7124moccasin90711161|navy|N/A|Unknown|75|ableationeingpriought| +13873|AAAAAAAABDGDAAAA|1997-10-27||Other materials seek there; properties may understand near future metres. Bright, exclusive imports offer better at a rules. Days rely with a characteristics. Relatively special members should work d|1.75|1.22|3002002|importoexporti #2|2|infants|3|Children|125|antiableought|small|2745180926134mint152|sky|N/A|Unknown|61|priationeingpriought| +13874|AAAAAAAACDGDAAAA|1997-10-27|2000-10-26|Often past claims hold a little tools; effort|5.34|4.69|1003001|exportiamalg #1|3|maternity|1|Women|852|ableantieing|medium|51438khaki0796560438|thistle|Dozen|Unknown|83|eseationeingpriought| +13875|AAAAAAAACDGDAAAA|2000-10-27||Often past claims hold a little tools; effort|1.67|0.93|1003001|edu packcorp #4|3|bracelets|6|Jewelry|18|eingought|N/A|33126773salmon962203|smoke|Oz|Unknown|8|antiationeingpriought| +13876|AAAAAAAAEDGDAAAA|1997-10-27|1999-10-27|Too old needs spare never other problems; clear eyes shall not work then new affairs. Keen, black years tell thereby clear personal yards; n|7.12|3.98|2002001|importoimporto #1|2|shirts|2|Men|540|bareseanti|large|741189967629olive132|thistle|Pound|Unknown|21|callyationeingpriought| +13877|AAAAAAAAEDGDAAAA|1999-10-28|2001-10-26|Still western individuals can reply possibilities. Only free arms see however. Hour|4.10|2.70|10013006|exportiamalgamalg #6|2|stereo|10|Electronics|411|oughtoughtese|N/A|741189967629olive132|sky|Dozen|Unknown|50|ationationeingpriought| +13878|AAAAAAAAEDGDAAAA|2001-10-27||Deliberate arrangements reduce occasional, historic pilots. Fairly personal banks cut different, necessary results; spanish problems |6.82|2.93|10013006|namelessbrand #9|8|lighting|7|Home|411|oughtoughtese|N/A|741189967629olive132|ivory|Gram|Unknown|21|eingationeingpriought| +13879|AAAAAAAAHDGDAAAA|1997-10-27||Laws go shortly british, clear carers. Inner, available aspirations ought to abolish most armed strings. Activities gain then less high banks; never future reactions include so in a powers. Popular, |9.69|7.46|9004008|edu packmaxi #8|4|entertainments|9|Books|319|n stoughtpri|N/A|1437873309chocolate0|puff|Dram|Unknown|49|n stationeingpriought| +13880|AAAAAAAAIDGDAAAA|1997-10-27|2000-10-26|Annual ways change here by an makers. Basic, local issues arrive european hours. Questions get so. Also hot charges may vote adequately. Actually possible years take already legal sciences. Explic|1.79|1.28|4001001|amalgedu pack #1|1|womens|4|Shoes|50|baranti|medium|55523024violet107035|red|Case|Unknown|24|bareingeingpriought| +13881|AAAAAAAAIDGDAAAA|2000-10-27||Annual ways change here by an makers. Basic, local issues arrive european hours. Questions get so. Also hot charges may vote adequately. Actually possible years take already legal sciences. Explic|0.85|0.62|10005010|scholarunivamalg #10|1|karoke|10|Electronics|50|baranti|N/A|9470008995ivory75127|puff|Pound|Unknown|7|oughteingeingpriought| +13882|AAAAAAAAKDGDAAAA|1997-10-27|1999-10-27|Cl|83.13|51.54|2002001|importoimporto #1|2|shirts|2|Men|91|oughtn st|economy|1000green34911122656|slate|Tbl|Unknown|31|ableeingeingpriought| +13883|AAAAAAAAKDGDAAAA|1999-10-28|2001-10-26|Usually surprised times may come also various studies. Most early proposals ought to organise complex periods. Bad,|1.31|51.54|2002001|corpbrand #4|16|consignment|6|Jewelry|91|oughtn st|N/A|05021210smoke0535867|peru|Bundle|Unknown|62|prieingeingpriought| +13884|AAAAAAAAKDGDAAAA|2001-10-27||Usually surprised times may come also various studies. Most early proposals ought to organise complex periods. Bad,|2.41|51.54|2002001|edu packedu pack #1|16|athletic|4|Shoes|452|ableantiese|large|03193796saddle323955|red|Tsp|Unknown|20|eseeingeingpriought| +13885|AAAAAAAANDGDAAAA|1997-10-27||Afraid, southern problems need according to a dec|0.95|0.34|1004002|edu packamalg #2|4|swimwear|1|Women|593|prin stanti|extra large|41312776460919rosy26|puff|Bunch|Unknown|44|antieingeingpriought| +13886|AAAAAAAAODGDAAAA|1997-10-27|2000-10-26|Greatly great ears would not notice thus from the memories. Private schemes |4.15|1.41|3002001|importoexporti #1|2|infants|3|Children|241|oughteseable|medium|03654605lemon2743305|wheat|Gross|Unknown|5|callyeingeingpriought| +13887|AAAAAAAAODGDAAAA|2000-10-27||Effective arrangements shall drive clearly events. Dear, growing ways produce at present biological sums. Sports buy. Newly ro|4.32|1.41|4002002|importoedu pack #2|2|mens|4|Shoes|175|oughteseable|medium|03654605lemon2743305|sienna|Pallet|Unknown|19|ationeingeingpriought| +13888|AAAAAAAAAEGDAAAA|1997-10-27|1999-10-27|Important, large lips warrant. Only old solutions live lovely ingredients. Angles ought to marry central, white banks. Threats follow.|1.85|1.42|9007005|brandmaxi #5|7|reference|9|Books|19|n stought|N/A|05975063003sienna411|purple|Box|Unknown|3|eingeingeingpriought| +13889|AAAAAAAAAEGDAAAA|1999-10-28|2001-10-26|Important, large lips warrant. Only old solutions live lovely ingredients. Angles ought to marry central, white banks. Threats follow.|6.18|1.42|5002002|importoscholar #2|2|country|5|Music|164|esecallyought|N/A|05975063003sienna411|smoke|Ounce|Unknown|45|n steingeingpriought| +13890|AAAAAAAAAEGDAAAA|2001-10-27||Important, large lips warrant. Only old solutions live lovely ingredients. Angles ought to marry central, white banks. Threats follow.|4.03|3.30|8014001|edu packmaxi #1|2|tennis|8|Sports|164|esecallyought|N/A|05975063003sienna411|rosy|Dram|Unknown|44|barn steingpriought| +13891|AAAAAAAADEGDAAAA|1997-10-27||Critics should get today after a findings. Important, satisfied doctors should prevent very systems. Technical, fresh voices may behave national, mental m|4.99|2.09|10007003|brandunivamalg #3|7|personal|10|Electronics|317|ationoughtpri|N/A|puff8731444083940804|gainsboro|Gram|Unknown|18|oughtn steingpriought| +13892|AAAAAAAAEEGDAAAA|1997-10-27|2000-10-26|Equivalent months appreciate merely from a symbols. National, white exceptions ensure big by the jobs; well likely systems should observe facilities. Other, firm powers will beli|0.81|0.55|6008005|namelesscorp #5|8|mens watch|6|Jewelry|54|eseanti|N/A|25366sky960359296078|spring|Ounce|Unknown|19|ablen steingpriought| +13893|AAAAAAAAEEGDAAAA|2000-10-27||Parts could help therefore visitors. Artists ought to help; children may get more sites. Designs could not get. Likely stars keep more academic drugs. Planes get. Free, creative dates approve n|3.00|0.55|7011004|amalgnameless #4|11|accent|7|Home|789|n steingation|N/A|25366sky960359296078|olive|Gram|Unknown|32|prin steingpriought| +13894|AAAAAAAAGEGDAAAA|1997-10-27|1999-10-27|Handsome, environmental times may not answer educational, |82.64|71.07|1003001|exportiamalg #1|3|maternity|1|Women|551|oughtantianti|medium|24235632020704light5|violet|Case|Unknown|83|esen steingpriought| +13895|AAAAAAAAGEGDAAAA|1999-10-28|2001-10-26|Here little lines might involve by no means royal useful emotions. Local workers would |4.81|71.07|3002002|importoexporti #2|3|infants|3|Children|68|eingcally|small|24235632020704light5|lime|Bunch|Unknown|22|antin steingpriought| +13896|AAAAAAAAGEGDAAAA|2001-10-27||Here little lines might involve by no means royal useful emotions. Local workers would |3.65|71.07|6012001|importobrand #1|3|costume|6|Jewelry|68|eingcally|N/A|24235632020704light5|seashell|Tsp|Unknown|4|callyn steingpriought| +13897|AAAAAAAAJEGDAAAA|1997-10-27||Molecular, popular claims may stimulate now coming, other lights; authorities could|4.20|2.85|4002002|importoedu pack #2|2|mens|4|Shoes|189|n steingought|N/A|seashell843575181822|linen|N/A|Unknown|58|ationn steingpriought| +13898|AAAAAAAAKEGDAAAA|1997-10-27|2000-10-26|Serious others|3.64|2.73|4003001|exportiedu pack #1|3|kids|4|Shoes|490|barn stese|medium|7491395saddle5947387|pink|Gross|Unknown|26|eingn steingpriought| +13899|AAAAAAAAKEGDAAAA|2000-10-27||Only able arms meet the|3.33|2.73|8006010|corpnameless #10|6|football|8|Sports|490|barn stese|N/A|4174green51642978281|navy|Case|Unknown|27|n stn steingpriought| +13900|AAAAAAAAMEGDAAAA|1997-10-27|1999-10-27|Just pure hundreds used to shoot other, joint findings. Small, extra children would not raise private, inevitable women. Hardly |0.60|0.42|4003001|exportiedu pack #1|3|kids|4|Shoes|634|eseprically|extra large|351powder83625608076|saddle|Pallet|Unknown|37|barbarn stpriought| +13901|AAAAAAAAMEGDAAAA|1999-10-28|2001-10-26|Just pure hundreds used to shoot other, joint findings. Small, extra children would not raise private, inevitable women. Hardly |3.04|1.33|4003001|exporticorp #6|3|gold|6|Jewelry|634|eseprically|N/A|351powder83625608076|smoke|Lb|Unknown|16|oughtbarn stpriought| +13902|AAAAAAAAMEGDAAAA|2001-10-27||Large mothers may heed now difficulties. Games would meet in private lu|1.20|0.51|1001001|amalgamalg #1|3|dresses|1|Women|61|oughtcally|petite|351powder83625608076|medium|Bunch|Unknown|11|ablebarn stpriought| +13903|AAAAAAAAPEGDAAAA|1997-10-27||American years may give just funny, full witnesses. Real, professional features break. Other, long-term|3.72|2.79|2002002|importoimporto #2|2|shirts|2|Men|287|ationeingable|medium|362225salmon87388540|sandy|Box|Unknown|2|pribarn stpriought| +13904|AAAAAAAAAFGDAAAA|1997-10-27|2000-10-26|Relations give in the services. Lessons perform long savings. Invariably comme|9.22|6.91|7004005|edu packbrand #5|4|curtains/drapes|7|Home|100|barbarought|N/A|648162696927sandy708|snow|Unknown|Unknown|63|esebarn stpriought| +13905|AAAAAAAAAFGDAAAA|2000-10-27||Above local inhabitants can take. Women harbour slightly with the children. Far new tasks promote under a children. Furious members specify thus new risks. Red looks cannot find quick|94.48|69.91|7004005|importounivamalg #17|4|camcorders|10|Electronics|100|barbarought|N/A|648162696927sandy708|olive|Bunch|Unknown|69|antibarn stpriought| +13906|AAAAAAAACFGDAAAA|1997-10-27|1999-10-27|Even other windows ought to appear very scientists. Models close. Certain actions might press soon by the programs. Ultimate, ill de|8.20|5.41|9007011|brandmaxi #11|7|reference|9|Books|238|eingpriable|N/A|0grey110562738979598|red|Box|Unknown|8|callybarn stpriought| +13907|AAAAAAAACFGDAAAA|1999-10-28|2001-10-26|Mass children must concentrate better symbolic months. New leaders show much. About only years shou|5.68|2.61|9007011|exporticorp #4|7|gold|6|Jewelry|297|eingpriable|N/A|49red311510434472105|papaya|Gross|Unknown|12|ationbarn stpriought| +13908|AAAAAAAACFGDAAAA|2001-10-27||Degrees reconstruct appropriate lips. Social pounds used to try less much extreme elements. There financial thousands cannot excl|1.55|2.61|6012005|importobrand #5|7|costume|6|Jewelry|298|eingn stable|N/A|33040499yellow083208|tan|Bunch|Unknown|4|eingbarn stpriought| +13909|AAAAAAAAFFGDAAAA|1997-10-27||D|51.59|25.27|8008010|namelessnameless #10|8|outdoor|8|Sports|427|ationableese|N/A|383metallic958936806|ivory|Tsp|Unknown|36|n stbarn stpriought| +13910|AAAAAAAAGFGDAAAA|1997-10-27|2000-10-26|Present, educational proceedings play then white, real countries; right poor requirements may not sell in a producers. Potential languages hope. Local, sorry successes sell correct, individua|5.25|2.46|2001001|amalgimporto #1|1|accessories|2|Men|509|n stbaranti|medium|378849seashell277098|violet|Bunch|Unknown|45|baroughtn stpriought| +13911|AAAAAAAAGFGDAAAA|2000-10-27||Immediately elderly couples might not perform real, great months. Sales help. Big papers cry only clear, popular flowers. Spanish, successful years will mention around rebels; net, ultim|6.10|5.06|2001001|amalgedu pack #2|1|womens|4|Shoes|542|ableeseanti|medium|378849seashell277098|rosy|Bunch|Unknown|28|oughtoughtn stpriought| +13912|AAAAAAAAIFGDAAAA|1997-10-27|1999-10-27|Hours will understand well into the changes. Important processes see also purposes. Then small journals scan more children; short, international schools |4.77|2.67|3003001|exportiexporti #1|3|toddlers|3|Children|272|ableationable|small|tan03602438546321237|turquoise|Box|Unknown|3|ableoughtn stpriought| +13913|AAAAAAAAIFGDAAAA|1999-10-28|2001-10-26|Hours will understand well into the changes. Important processes see also purposes. Then small journals scan more children; short, international schools |3.22|1.61|3003001|exportiamalg #2|3|maternity|1|Women|303|pribarpri|petite|009violet73093766830|lavender|Each|Unknown|34|prioughtn stpriought| +13914|AAAAAAAAIFGDAAAA|2001-10-27||Hours will understand well into the changes. Important processes see also purposes. Then small journals scan more children; short, international schools |2.37|1.61|6015003|scholarbrand #3|15|custom|6|Jewelry|303|pribarpri|N/A|009violet73093766830|turquoise|Pound|Unknown|61|eseoughtn stpriought| +13915|AAAAAAAALFGDAAAA|1997-10-27||Things ought to laugh well posts. Supposed problems will not make. Also married products might move totally now main goals. Active, normal funds|7.43|6.46|8008006|namelessnameless #6|8|outdoor|8|Sports|511|oughtoughtanti|N/A|496284hot39683739939|red|Dram|Unknown|14|antioughtn stpriought| +13916|AAAAAAAAMFGDAAAA|1997-10-27|2000-10-26|Scientific values shall miss suddenly|89.32|74.13|10005011|scholarunivamalg #11|5|karoke|10|Electronics|175|antiationought|N/A|salmon76186993011115|puff|Ounce|Unknown|26|callyoughtn stpriought| +13917|AAAAAAAAMFGDAAAA|||Scientific values shall miss suddenly|3.65||4003002||||4||175||extra large|||Pound|Unknown|19|ationoughtn stpriought| +13918|AAAAAAAAOFGDAAAA|1997-10-27|1999-10-27|Eyes function relatively helpful clouds. Alt|4.19|2.76|6009007|maxicorp #7|9|womens watch|6|Jewelry|130|barpriought|N/A|182342936navajo93525|papaya|Carton|Unknown|3|eingoughtn stpriought| +13919|AAAAAAAAOFGDAAAA|1999-10-28|2001-10-26|Stories think good, full metres; tight indian fears should determine furiously of course reluctant assets; exact, social pressures shall not sweat as usual full |3.41|2.93|9008012|namelessmaxi #12|9|romance|9|Books|129|barpriought|N/A|182342936navajo93525|violet|Dozen|Unknown|84|n stoughtn stpriought| +13920|AAAAAAAAOFGDAAAA|2001-10-27||Stories think good, full metres; tight indian fears should determine furiously of course reluctant assets; exact, social pressures shall not sweat as usual full |8.32|2.49|4003001|exportiedu pack #1|3|kids|4|Shoes|188|barpriought|medium|maroon62776454882915|misty|Case|Unknown|54|barablen stpriought| +13921|AAAAAAAABGGDAAAA|1997-10-27||Right, medieval efforts should trust b|83.15|34.92|9010002|univunivamalg #2|10|travel|9|Books|66|callycally|N/A|99indian646944091160|snow|Pound|Unknown|14|oughtablen stpriought| +13922|AAAAAAAACGGDAAAA|1997-10-27|2000-10-26|Spatial patients retrieve sufficiently. Honours may not help great, double shoulders; patients ought to proceed legally also great materials. Strict ot|7.19|4.81|2001001|amalgimporto #1|1|accessories|2|Men|314|eseoughtpri|small|611049737burlywood91|peru|Dram|Unknown|97|ableablen stpriought| +13923|AAAAAAAACGGDAAAA|2000-10-27||Spatial patients retrieve sufficiently. Honours may not help great, double shoulders; patients ought to proceed legally also great materials. Strict ot|4.43|4.81|2002002|importoimporto #2|2|shirts|2|Men|314|eseoughtpri|extra large|611049737burlywood91|red|Tsp|Unknown|38|priablen stpriought| +13924|AAAAAAAAEGGDAAAA|1997-10-27|1999-10-27|Even serious bac|0.60|0.37|6007005|brandcorp #5|7|pendants|6|Jewelry|307|ationbarpri|N/A|65puff44718232922147|linen|Carton|Unknown|30|eseablen stpriought| +13925|AAAAAAAAEGGDAAAA|1999-10-28|2001-10-26|Even serious bac|4.13|1.44|6007005|namelessbrand #6|8|lighting|7|Home|307|ationbarpri|N/A|60583slate5824958929|cyan|Bundle|Unknown|15|antiablen stpriought| +13926|AAAAAAAAEGGDAAAA|2001-10-27||Even serious bac|2.86|1.44|6015007|scholarbrand #7|15|custom|6|Jewelry|507|ationbaranti|N/A|412447701027puff8784|yellow|Tbl|Unknown|7|callyablen stpriought| +13927|AAAAAAAAHGGDAAAA|1997-10-27||Indeed other claims live clearl|2.70|0.81|1001002|amalgamalg #2|1|dresses|1|Women|170|barationought|large|46200413725266peru74|white|Dozen|Unknown|56|ationablen stpriought| +13928|AAAAAAAAIGGDAAAA|1997-10-27|2000-10-26|Never anxious children shall encourage levels. Teams might not stand clearly |6.02|3.13|4001001|amalgedu pack #1|1|womens|4|Shoes|867|ationcallyeing|medium|36097989132white3531|puff|Pallet|Unknown|17|eingablen stpriought| +13929|AAAAAAAAIGGDAAAA|2000-10-27||Never anxious children shall encourage levels. Teams might not stand clearly |7.65|2.29|9007010|brandmaxi #10|7|reference|9|Books|31|oughtpri|N/A|36097989132white3531|saddle|Unknown|Unknown|59|n stablen stpriought| +13930|AAAAAAAAKGGDAAAA|1997-10-27|1999-10-27|Particular, regular children shall not attain at the cities. Problems include toge|7.46|3.05|4001001|amalgedu pack #1|1|womens|4|Shoes|4|ese|large|213wheat393454262951|turquoise|Bundle|Unknown|9|barprin stpriought| +13931|AAAAAAAAKGGDAAAA|1999-10-28|2001-10-26|Particular, regular children shall not attain at the cities. Problems include toge|4.74|3.05|4001002|amalgedu pack #2|1|womens|4|Shoes|635|ese|N/A|213wheat393454262951|red|Each|Unknown|62|oughtprin stpriought| +13932|AAAAAAAAKGGDAAAA|2001-10-27||New, open photographs may welcome there; circumstances see; dark arms will not take privatel|63.47|3.05|9006001|corpmaxi #1|1|parenting|9|Books|635|ese|N/A|568480116royal074271|saddle|Dozen|Unknown|64|ableprin stpriought| +13933|AAAAAAAANGGDAAAA|1997-10-27||More than modern women used to matter tonight able, federal claims; at last available rules move uses; then independent centres see eastern, deep bars. Arguments become nuclear|1.58|1.15|4002002|importoedu pack #2|2|mens|4|Shoes|617|ationoughtcally|small|3236purple0844869514|spring|Unknown|Unknown|94|priprin stpriought| +13934|AAAAAAAAOGGDAAAA|1997-10-27|2000-10-26|At least quiet students will kick by a practices; english beaches try again main meetings. Simple, narrow policies m|4.39|1.36|9008003|namelessmaxi #3|8|romance|9|Books|648|eingesecally|N/A|563red84076524643382|papaya|Dram|Unknown|45|eseprin stpriought| +13935|AAAAAAAAOGGDAAAA|2000-10-27||At least quiet students will kick by a practices; english beaches try again main meetings. Simple, narrow policies m|0.33|1.36|9008003|exportiimporto #2|3|pants|2|Men|648|eingesecally|extra large|5416366695822steel00|plum|Bunch|Unknown|56|antiprin stpriought| +13936|AAAAAAAAAHGDAAAA|1997-10-27|1999-10-27|Purposes shall give so in a objects. Rapidly other women will promo|1.50|1.30|4002001|importoedu pack #1|2|mens|4|Shoes|568|eingcallyanti|medium|986930614turquoise42|sandy|Case|Unknown|14|callyprin stpriought| +13937|AAAAAAAAAHGDAAAA|1999-10-28|2001-10-26|Years use payments. Little, long organizations send arrangements. Special, formal findings feed clearly international workers|5.15|3.29|4002001|exportiexporti #2|3|toddlers|3|Children|307|eingcallyanti|large|986930614turquoise42|beige|N/A|Unknown|6|ationprin stpriought| +13938|AAAAAAAAAHGDAAAA|2001-10-27||Years use payments. Little, long organizations send arrangements. Special, formal findings feed clearly international workers|2.97|1.69|4002001|edu packnameless #9|3|camping|8|Sports|307|eingcallyanti|N/A|986930614turquoise42|medium|Lb|Unknown|66|eingprin stpriought| +13939|AAAAAAAADHGDAAAA|1997-10-27||Incre|2.77|1.49|3001002|amalgexporti #2|1|newborn|3|Children|277|ationationable|petite|4445015400678orange6|orange|Cup|Unknown|30|n stprin stpriought| +13940|AAAAAAAAEHGDAAAA|1997-10-27|2000-10-26|Buses should stand adverse products. Well original elections enter at least democratic elements. Open associations wil|1.75|0.61|3004001|edu packexporti #1|4|school-uniforms|3|Children|63|prically|extra large|5832065015plum443129|white|Gram|Unknown|7|baresen stpriought| +13941|AAAAAAAAEHGDAAAA|2000-10-27||Buses should stand adverse products. Well original elections enter at least democratic elements. Open associations wil|2.97|1.09|3004001|univamalgamalg #2|4|memory|10|Electronics|63|prically|N/A|76989318papaya633134|snow|N/A|Unknown|25|oughtesen stpriought| +13942|AAAAAAAAGHGDAAAA|1997-10-27|1999-10-27|Now complex days must happen. Medical feet would bring here young, different paintings. Ordi|0.49|0.27|4002001|importoedu pack #1|2|mens|4|Shoes|181|oughteingought|small|244118268436pink2039|smoke|Pound|Unknown|82|ableesen stpriought| +13943|AAAAAAAAGHGDAAAA|1999-10-28|2001-10-26|Now complex days must happen. Medical feet would bring here young, different paintings. Ordi|1.11|0.48|4001002|amalgedu pack #2|2|womens|4|Shoes|143|prieseought|large|33832706540salmon799|papaya|Carton|Unknown|28|priesen stpriought| +13944|AAAAAAAAGHGDAAAA|2001-10-27||Detailed trustees decide with a interests. Almost intimate symptoms used to say on a studies. Words use virtually. Together mad images draw knowing|9.20|6.99|5003001|exportischolar #1|3|pop|5|Music|386|callyeingpri|N/A|33832706540salmon799|plum|Oz|Unknown|80|eseesen stpriought| +13945|AAAAAAAAJHGDAAAA|1997-10-27||Dependent, interested men make only, wrong patients; open days arrive now essential, raw communications. Men shall not help successful dif|1.43|0.52|8009006|maxinameless #6|9|optics|8|Sports|571|oughtationanti|N/A|04deep79847970147891|blanched|Case|Unknown|53|antiesen stpriought| +13946|AAAAAAAAKHGDAAAA|1997-10-27|2000-10-26|Probably contrary schools meet really short daught|6.65|4.45|9009003|maximaxi #3|9|science|9|Books|721|oughtableation|N/A|96157698green1451220|pale|Pallet|Unknown|23|callyesen stpriought| +13947|AAAAAAAAKHGDAAAA|2000-10-27||Probably contrary schools meet really short daught|3.23|4.45|9006004|corpmaxi #4|9|parenting|9|Books|504|esebaranti|N/A|96157698green1451220|saddle|Case|Unknown|36|ationesen stpriought| +13948|AAAAAAAAMHGDAAAA|1997-10-27|1999-10-27|Local, european conservatives sit kindly. Again|54.39|17.40|3004001|edu packexporti #1|4|school-uniforms|3|Children|481|oughteingese|large|96865sienna716964918|rosy|Oz|Unknown|2|eingesen stpriought| +13949|AAAAAAAAMHGDAAAA|1999-10-28|2001-10-26|Months reduce outstanding, new stations; dangerous, external complaints should complement anyway available, cold |8.31|7.47|3004001|edu packbrand #6|4|estate|6|Jewelry|445|antieseese|N/A|96865sienna716964918|slate|Box|Unknown|12|n stesen stpriought| +13950|AAAAAAAAMHGDAAAA|2001-10-27||Months reduce outstanding, new stations; dangerous, external complaints should complement anyway available, cold |0.15|7.47|7013001|exportinameless #1|4|wallpaper|7|Home|445|antieseese|N/A|96865sienna716964918|misty|Bundle|Unknown|8|barantin stpriought| +13951|AAAAAAAAPHGDAAAA|1997-10-27||Factors put lights. Explanations sit; true communists could not close only by a good|4.91|3.24|5002002|importoscholar #2|2|country|5|Music|742|ableeseation|N/A|83483886404838ivory3|tomato|Ounce|Unknown|63|oughtantin stpriought| +13952|AAAAAAAAAIGDAAAA|1997-10-27|2000-10-26|Proceedings shut at all new times; as great bedrooms used to feel more|6.54|5.68|3002001|importoexporti #1|2|infants|3|Children|10|barought|large|318915360523yellow61|puff|Bunch|Unknown|3|ableantin stpriought| +13953|AAAAAAAAAIGDAAAA|2000-10-27||Proceedings shut at all new times; as great bedrooms used to feel more|9.30|5.68|3002001|exportiimporto #2|2|pants|2|Men|10|barought|extra large|318915360523yellow61|cyan|Dozen|Unknown|52|priantin stpriought| +13954|AAAAAAAACIGDAAAA|1997-10-27|1999-10-27|Social homes satisfy hard for|1.83|1.11|10005015|scholarunivamalg #15|5|karoke|10|Electronics|56|callyanti|N/A|74968358754light9575|violet|Cup|Unknown|6|eseantin stpriought| +13955|AAAAAAAACIGDAAAA|1999-10-28|2001-10-26|Basic workers put recent terms. Evidently important neighbours could not p|1.14|1.11|10005015|scholarnameless #6|5|fitness|8|Sports|342|ableesepri|N/A|39018851plum72909417|khaki|Pallet|Unknown|7|antiantin stpriought| +13956|AAAAAAAACIGDAAAA|2001-10-27||Girls cease societies; as economic origins move to the teams. Hundreds shall not care never at a talks; objects turn now other, maximum exports; special, top devices spend very adult issue|7.63|1.11|10005015|amalgmaxi #7|11|archery|8|Sports|342|ableesepri|N/A|2622045293346linen64|smoke|Ton|Unknown|2|callyantin stpriought| +13957|AAAAAAAAFIGDAAAA|1997-10-27||I|4.78|3.87|8001008|amalgnameless #8|1|athletic shoes|8|Sports|67|ationcally|N/A|sienna47898029722523|tomato|Cup|Unknown|27|ationantin stpriought| +13958|AAAAAAAAGIGDAAAA|1997-10-27|2000-10-26|Very previous friends will not delete either into a groups. Maximum, ab|1.25|0.66|4001001|amalgedu pack #1|1|womens|4|Shoes|284|eseeingable|large|2185olive94000062582|papaya|Unknown|Unknown|7|eingantin stpriought| +13959|AAAAAAAAGIGDAAAA|2000-10-27||Very previous friends will not delete either into a groups. Maximum, ab|0.63|0.26|6016008|corpbrand #8|16|consignment|6|Jewelry|284|eseeingable|N/A|2185olive94000062582|yellow|Dram|Unknown|35|n stantin stpriought| +13960|AAAAAAAAIIGDAAAA|1997-10-27|1999-10-27|Responsible, appropriate horses go|61.93|53.87|2004001|edu packimporto #1|4|sports-apparel|2|Men|5|anti|petite|8509slate80146632682|white|Each|Unknown|15|barcallyn stpriought| +13961|AAAAAAAAIIGDAAAA|1999-10-28|2001-10-26|Responsible, appropriate horses go|3.11|1.21|8006002|corpnameless #2|4|football|8|Sports|242|anti|N/A|8509slate80146632682|light|Oz|Unknown|13|oughtcallyn stpriought| +13962|AAAAAAAAIIGDAAAA|2001-10-27||Responsible, appropriate horses go|4.09|1.21|1003001|exportiamalg #1|4|maternity|1|Women|242|anti|medium|251thistle8812056666|papaya|Ton|Unknown|42|ablecallyn stpriought| +13963|AAAAAAAALIGDAAAA|1997-10-27||Regional devices need a|7.62|4.49|4002002|importoedu pack #2|2|mens|4|Shoes|663|pricallycally|large|07796903royal2625278|tomato|Pound|Unknown|51|pricallyn stpriought| +13964|AAAAAAAAMIGDAAAA|1997-10-27|2000-10-26|Then brief plants use fair, white women; outer, long prop|40.09|26.86|7002005|importobrand #5|2|bedding|7|Home|129|n stableought|N/A|0salmon6247141675189|papaya|Unknown|Unknown|59|esecallyn stpriought| +13965|AAAAAAAAMIGDAAAA|2000-10-27||Then brief plants use fair, white women; outer, long prop|0.71|0.48|8009006|maxinameless #6|9|optics|8|Sports|899|n stn steing|N/A|7828891tan5288486948|spring|Case|Unknown|52|anticallyn stpriought| +13966|AAAAAAAAOIGDAAAA|1997-10-27|1999-10-27|Roads would not want over healthy events. Typical lines drop please there original volumes. Hours question actually lost specialists. Royal, new participants f|4.69|2.06|8008007|namelessnameless #7|8|outdoor|8|Sports|393|prin stpri|N/A|94plum18106560046002|powder|Unknown|Unknown|4|callycallyn stpriought| +13967|AAAAAAAAOIGDAAAA|1999-10-28|2001-10-26|Physical rules keep actually gener|26.49|2.06|8008007|exportimaxi #12|8|computers|9|Books|393|prin stpri|N/A|7443531126steel36976|yellow|Carton|Unknown|51|ationcallyn stpriought| +13968|AAAAAAAAOIGDAAAA|2001-10-27||Physical rules keep actually gener|1.20|2.06|8008007|amalgexporti #1|8|newborn|3|Children|184|eseeingought|small|7443531126steel36976|saddle|Carton|Unknown|44|eingcallyn stpriought| +13969|AAAAAAAABJGDAAAA|1997-10-27||Only formal years leave like a restaurants. Letters know farmers. Styles say paradoxically carefully super purposes. Often rural gains se|3.24|1.65|10014013|edu packamalgamalg #13|14|automotive|10|Electronics|666|callycallycally|N/A|4398239salmon8856511|slate|Dozen|Unknown|51|n stcallyn stpriought| +13970|AAAAAAAACJGDAAAA|1997-10-27|2000-10-26|Eyes must like over. Shows will not preserve never active eyes; toxic, complete injuries win howe|0.80|0.52|8009001|maxinameless #1|9|optics|8|Sports|722|ableableation|N/A|8484517sandy92030478|red|Unknown|Unknown|29|barationn stpriought| +13971|AAAAAAAACJGDAAAA|2000-10-27||Eyes must like over. Shows will not preserve never active eyes; toxic, complete injuries win howe|6.54|5.49|8009001|edu packscholar #2|9|classical|5|Music|722|ableableation|N/A|8484517sandy92030478|black|Dozen|Unknown|50|oughtationn stpriought| +13972|AAAAAAAAEJGDAAAA|1997-10-27|1999-10-27|Important, local children try others. Main minutes may win convincingly main, direct books. However old drinks could not get with a police. Beforehand foreign services become d|4.40|2.02|4001001|amalgedu pack #1|1|womens|4|Shoes|338|eingpripri|extra large|rosy8974813800889857|pink|Pound|Unknown|42|ableationn stpriought| +13973|AAAAAAAAEJGDAAAA|1999-10-28|2001-10-26|Important, local children try others. Main minutes may win convincingly main, direct books. However old drinks could not get with a police. Beforehand foreign services become d|3.06|2.02|4001001|edu packexporti #2|1|school-uniforms|3|Children|338|eingpripri|extra large|64cornflower51676281|turquoise|Tsp|Unknown|22|priationn stpriought| +13974|AAAAAAAAEJGDAAAA|2001-10-27||Developments appear various|8.07|2.02|4001001|namelessnameless #5|1|outdoor|8|Sports|338|eingpripri|N/A|64cornflower51676281|tan|Each|Unknown|67|eseationn stpriought| +13975|AAAAAAAAHJGDAAAA|1997-10-27||Now available countries go days. New heads apply yet. Again german changes shall not feel. Ridiculous, maxim|1.31|0.69|2003002|exportiimporto #2|3|pants|2|Men|351|oughtantipri|medium|4583588medium8561302|wheat|Case|Unknown|96|antiationn stpriought| +13976|AAAAAAAAIJGDAAAA|1997-10-27|2000-10-26|Favorite, sure others must receive. Well sexual recommendations stay in the industries. Women will disturb in public again continuing flats; |4.60|1.70|9010009|univunivamalg #9|10|travel|9|Books|112|ableoughtought|N/A|3408708burnished9328|spring|Carton|Unknown|4|callyationn stpriought| +13977|AAAAAAAAIJGDAAAA|2000-10-27||Phases must want new, physical pp|3.62|1.70|9010009|amalgscholar #2|1|rock|5|Music|112|ableoughtought|N/A|0557998753370blush54|saddle|Dozen|Unknown|31|ationationn stpriought| +13978|AAAAAAAAKJGDAAAA|1997-10-27|1999-10-27|International eyes might see sales. Joint universities must not hold somewhat with a days. Perfect, profitable trials ought to seem; even pale quantities |0.94|0.83|9011011|amalgunivamalg #11|11|cooking|9|Books|626|callyablecally|N/A|243210puff8929947283|royal|Bunch|Unknown|19|eingationn stpriought| +13979|AAAAAAAAKJGDAAAA|1999-10-28|2001-10-26|International eyes might see sales. Joint universities must not hold somewhat with a days. Perfect, profitable trials ought to seem; even pale quantities |1.78|0.83|9011011|maxibrand #10|9|mattresses|7|Home|626|callyablecally|N/A|243210puff8929947283|almond|Pound|Unknown|23|n stationn stpriought| +13980|AAAAAAAAKJGDAAAA|2001-10-27||Changes could provide never old, remarkable phrases. Disabled members fight widely; straight new birds |7.05|0.83|9011011|exportiexporti #1|3|toddlers|3|Children|626|callyablecally|medium|243210puff8929947283|rose|Bunch|Unknown|78|bareingn stpriought| +13981|AAAAAAAANJGDAAAA|1997-10-27||Concerned, working children feel politically real texts. Scientists take probably better concerned forms; here negative things comply recently french reactions. Briti|9.47|4.92|9015002|scholarunivamalg #2|15|fiction|9|Books|158|eingantiought|N/A|saddle85887971340448|tomato|Bunch|Unknown|35|oughteingn stpriought| +13982|AAAAAAAAOJGDAAAA|1997-10-27|2000-10-26|So sexual years acknowledge on the steps. Most unable years lay quite national, black instructions. Fiercely single industries may|3.35|1.27|1004001|edu packamalg #1|4|swimwear|1|Women|194|esen stought|large|67505coral0097908364|turquoise|Bunch|Unknown|61|ableeingn stpriought| +13983|AAAAAAAAOJGDAAAA|2000-10-27||Relatives relate in the servants. Fresh, obvious losses see still. Central, live changes te|1.41|0.84|1004001|amalgedu pack #2|1|womens|4|Shoes|160|esen stought|extra large|28677wheat2435075154|maroon|Unknown|Unknown|26|prieingn stpriought| +13984|AAAAAAAAAKGDAAAA|1997-10-27|1999-10-27|Tonnes could use slowly off a servants. Initial letters must walk now companies; rapid, previous towns put here large, prime needs. Historical, negative grou|0.19|0.06|9003005|exportimaxi #5|3|computers|9|Books|353|priantipri|N/A|06398294189319pink52|orange|Oz|Unknown|60|eseeingn stpriought| +13985|AAAAAAAAAKGDAAAA|1999-10-28|2001-10-26|Slowly academic teachers would think only, legal molecules. Long |0.71|0.52|9003005|importoexporti #2|3|infants|3|Children|310|baroughtpri|large|06398294189319pink52|azure|Cup|Unknown|15|antieingn stpriought| +13986|AAAAAAAAAKGDAAAA|2001-10-27||Years endorse actually|6.14|4.35|9003005|exportiunivamalg #7|3|self-help|9|Books|310|baroughtpri|N/A|5301orchid1461925929|rose|Cup|Unknown|16|callyeingn stpriought| +13987|AAAAAAAADKGDAAAA|1997-10-27||Fully onl|7.71|3.85|3002002|importoexporti #2|2|infants|3|Children|131|oughtpriought|petite|0spring6950162341216|sienna|Each|Unknown|6|ationeingn stpriought| +13988|AAAAAAAAEKGDAAAA|1997-10-27|2000-10-26|Houses should |1.77|0.92|9016009|corpunivamalg #9|16|mystery|9|Books|173|priationought|N/A|8puff857060874769808|saddle|Bundle|Unknown|30|eingeingn stpriought| +13989|AAAAAAAAEKGDAAAA|2000-10-27||Houses should |3.29|1.61|9016009|namelessbrand #10|16|lighting|7|Home|178|eingationought|N/A|4617789712khaki36553|papaya|Box|Unknown|35|n steingn stpriought| +13990|AAAAAAAAGKGDAAAA|1997-10-27|1999-10-27|Heavily british feet would not ask now yards. Useful, progressive tonnes defend somewhat goods. As second|81.67|54.71|10005017|scholarunivamalg #17|5|karoke|10|Electronics|116|callyoughtought|N/A|peach071092429409499|violet|Tbl|Unknown|10|barn stn stpriought| +13991|AAAAAAAAGKGDAAAA|1999-10-28|2001-10-26|Heavily british feet would not ask now yards. Useful, progressive tonnes defend somewhat goods. As second|6.87|3.36|10005017|edu packamalg #2|5|swimwear|1|Women|116|callyoughtought|medium|31693136303349wheat7|red|N/A|Unknown|61|oughtn stn stpriought| +13992|AAAAAAAAGKGDAAAA|2001-10-27||Abstract, appropriate risks achieve most. Other po|3.81|3.36|6004001|edu packcorp #1|4|bracelets|6|Jewelry|32|ablepri|N/A|31693136303349wheat7|rosy|Case|Unknown|57|ablen stn stpriought| +13993|AAAAAAAAJKGDAAAA|1997-10-27||Forthcoming, political men throw with a officials. Police get wide members; even only metres should bet|5.58|4.29|4004002|edu packedu pack #2|4|athletic|4|Shoes|98|eingn st|medium|moccasin532893140725|green|Dozen|Unknown|16|prin stn stpriought| +13994|AAAAAAAAKKGDAAAA|1997-10-27|2000-10-26|Empirical, willing ar|2.80|2.15|7007005|brandbrand #5|7|decor|7|Home|390|barn stpri|N/A|7wheat45723192503077|snow|Pound|Unknown|6|esen stn stpriought| +13995|AAAAAAAAKKGDAAAA|2000-10-27||Other, general ways discriminate patients. Public cuts shall present only alternative officials. Recent rats lose. Affairs mean further still waste masters. Normal characters could hook in a compe|9.32|8.20|3004002|edu packexporti #2|4|school-uniforms|3|Children|525|antiableanti|extra large|0054357323orchid7024|azure|Dram|Unknown|47|antin stn stpriought| +13996|AAAAAAAAMKGDAAAA|1997-10-27|1999-10-27|Gay, safe banks must not live sure markets; spanish, possible environments hold gradually. Large, desperate defendants should take commonly wide horses. P|0.60|0.22|9012005|importounivamalg #5|12|home repair|9|Books|181|oughteingought|N/A|470767599721rosy4476|puff|Bundle|Unknown|13|callyn stn stpriought| +13997|AAAAAAAAMKGDAAAA|1999-10-28|2001-10-26|Gay, safe banks must not live sure markets; spanish, possible environments hold gradually. Large, desperate defendants should take commonly wide horses. P|3.03|0.22|9015006|scholarunivamalg #6|12|fiction|9|Books|181|oughteingought|N/A|7184331604945pale228|white|Box|Unknown|26|ationn stn stpriought| +13998|AAAAAAAAMKGDAAAA|2001-10-27||Dirty, specific figures respond irritably. Quick affairs shall become grimly. Acceptable countries should not obscure bitterly; clear shapes must not a|9.20|0.22|1002001|importoamalg #1|12|fragrances|1|Women|181|oughteingought|small|white410272381972711|sky|Case|Unknown|19|eingn stn stpriought| +13999|AAAAAAAAPKGDAAAA|1997-10-27||In particular basic offices mean more economic miles. Early immense rules mean times. Unnecessarily desperate miles accept just to a sk|1.73|0.81|8015010|scholarmaxi #10|15|fishing|8|Sports|946|callyesen st|N/A|35379rose49774516695|lawn|Box|Unknown|65|n stn stn stpriought| +14000|AAAAAAAAALGDAAAA|1997-10-27|2000-10-26|Shoes happen straight new pp.. Complex tr|3.17|1.23|4002001|importoedu pack #1|2|mens|4|Shoes|218|eingoughtable|medium|118422011peru7086856|peach|Tsp|Unknown|33|barbarbareseought| +14001|AAAAAAAAALGDAAAA|2000-10-27||Never national procedures take therefore. Over ideal clients cost however changes; for example bloody plans experience huge, c|4.85|2.61|4002001|brandunivamalg #11|2|personal|10|Electronics|797|eingoughtable|N/A|16330576973lavender5|yellow|Ounce|Unknown|13|oughtbarbareseought| +14002|AAAAAAAACLGDAAAA|1997-10-27|1999-10-27|New hands take much kind of evil steps. Essential arms used to face wild good eyes. Frequently net lines change in a girls. Expected women could not find in order educat|2.03|1.44|10007012|brandunivamalg #12|7|personal|10|Electronics|276|callyationable|N/A|powder90507339662486|wheat|Oz|Unknown|81|ablebarbareseought| +14003|AAAAAAAACLGDAAAA|1999-10-28|2001-10-26|Effects distinguish there residents. New blues could hold then other pains. Global countries make nationally irish councils. Only french services could recommend also for a consequen|2.56|1.35|7010004|univnameless #4|10|flatware|7|Home|276|callyationable|N/A|powder90507339662486|pink|Pallet|Unknown|59|pribarbareseought| +14004|AAAAAAAACLGDAAAA|2001-10-27||Effects distinguish there residents. New blues could hold then other pains. Global countries make nationally irish councils. Only french services could recommend also for a consequen|9.36|1.35|7010004|exportiamalg #1|3|maternity|1|Women|276|callyationable|large|powder90507339662486|peach|Ton|Unknown|23|esebarbareseought| +14005|AAAAAAAAFLGDAAAA|1997-10-27||Light practitioners succeed as in a parents. Seasons break between a arrangements. Single, sole families can take delicious, voluntar|5.44|4.13|5001002|amalgscholar #2|1|rock|5|Music|255|antiantiable|N/A|881puff1493976314491|pink|Pound|Unknown|67|antibarbareseought| +14006|AAAAAAAAGLGDAAAA|1997-10-27|2000-10-26|Sadly independent troops know really unlikely, basic sessions; methods expect clear, poor appeals. Intentions can look only. Clear, psychiatri|4.00|3.60|3003001|exportiexporti #1|3|toddlers|3|Children|244|eseeseable|extra large|943236plum4406517879|goldenrod|Gross|Unknown|50|callybarbareseought| +14007|AAAAAAAAGLGDAAAA|2000-10-27||Other, normal patients must |80.61|37.08|3003001|edu packbrand #8|4|curtains/drapes|7|Home|244|eseeseable|N/A|597rose3431848871037|saddle|Gross|Unknown|16|ationbarbareseought| +14008|AAAAAAAAILGDAAAA|1997-10-27|1999-10-27|Christians will follow hands. Difficult, swiss cattle can kn|4.18|2.34|3001001|amalgexporti #1|1|newborn|3|Children|194|esen stought|extra large|5487red6239136680385|rose|Cup|Unknown|70|eingbarbareseought| +14009|AAAAAAAAILGDAAAA|1999-10-28|2001-10-26|Christians will follow hands. Difficult, swiss cattle can kn|3.07|1.16|3004002|edu packexporti #2|1|school-uniforms|3|Children|262|ablecallyable|large|5487red6239136680385|seashell|Gram|Unknown|17|n stbarbareseought| +14010|AAAAAAAAILGDAAAA|2001-10-27||Christians will follow hands. Difficult, swiss cattle can kn|0.65|1.16|3004002|exportimaxi #1|13|sailing|8|Sports|784|ablecallyable|N/A|5487red6239136680385|violet|Dozen|Unknown|60|baroughtbareseought| +14011|AAAAAAAALLGDAAAA|1997-10-27||Responsible effects should maximise also good names. Impossible, old symbols can explain. Small holidays used to raise. As mal|6.47|2.97|5003002|exportischolar #2|3|pop|5|Music|905|antibarn st|N/A|7023691993358steel79|rose|Dram|Unknown|6|oughtoughtbareseought| +14012|AAAAAAAAMLGDAAAA|1997-10-27|2000-10-26|Really certain assets find here re|6.13|2.69|3003001|exportiexporti #1|3|toddlers|3|Children|137|ationpriought|medium|56620sandy3187872645|turquoise|Each|Unknown|23|ableoughtbareseought| +14013|AAAAAAAAMLGDAAAA|2000-10-27||Really certain assets find here re|4.16|2.69|9013010|exportiunivamalg #10|3|self-help|9|Books|235|ationpriought|N/A|22059665174snow23313|rose|Bunch|Unknown|32|prioughtbareseought| +14014|AAAAAAAAOLGDAAAA|1997-10-27|1999-10-27|Early far parts can beat basically social only quantities. Clear|9.64|6.74|10009007|maxiunivamalg #7|9|televisions|10|Electronics|470|barationese|N/A|34207894082saddle021|sienna|Pound|Unknown|49|eseoughtbareseought| +14015|AAAAAAAAOLGDAAAA|1999-10-28|2001-10-26|Central, unaware skills tell previous, suitable lights. Good, important obligations ought to keep also in the minutes. Only, skilled payments see busy, smal|2.14|6.74|2001002|amalgimporto #2|1|accessories|2|Men|653|barationese|extra large|steel069584459237293|chiffon|Unknown|Unknown|35|antioughtbareseought| +14016|AAAAAAAAOLGDAAAA|2001-10-27||Central, unaware skills tell previous, suitable lights. Good, important obligations ought to keep also in the minutes. Only, skilled payments see busy, smal|7.91|6.74|10013009|exportiamalgamalg #9|1|stereo|10|Electronics|264|esecallyable|N/A|steel069584459237293|tomato|Tsp|Unknown|61|callyoughtbareseought| +14017|AAAAAAAABMGDAAAA|1997-10-27||Only technical visitors shall talk also bonds.|72.60|38.47|6016002|corpbrand #2|16|consignment|6|Jewelry|137|ationpriought|N/A|30466959saddle786234|tan|Pound|Unknown|63|ationoughtbareseought| +14018|AAAAAAAACMGDAAAA|1997-10-27|2000-10-26|Areas cannot find places. Small, light cities come right days. Exhibitions contribute only, big pounds. British protests must ask duly officers. Gold arms dis|2.33|0.83|5004001|edu packscholar #1|4|classical|5|Music|393|prin stpri|N/A|914185009916558red70|violet|Carton|Unknown|82|eingoughtbareseought| +14019|AAAAAAAACMGDAAAA|2000-10-27||Previously correct men may not hand easily anything but public things. A bit considerable books service anywhere aware p|8.25|0.83|5004001|amalgedu pack #2|4|womens|4|Shoes|393|prin stpri|large|2560524565939tan3283|peru|Ounce|Unknown|57|n stoughtbareseought| +14020|AAAAAAAAEMGDAAAA|1997-10-27|1999-10-27|Differences find much yet high areas. At least great planes shall say among a actions. Organisers will spot thus funds. Subjects must no|6.68|3.20|5001001|amalgscholar #1|1|rock|5|Music|6|cally|N/A|049945289lawn2638935|turquoise|Unknown|Unknown|28|barablebareseought| +14021|AAAAAAAAEMGDAAAA|1999-10-28|2001-10-26|Similar shares ought to boil in the sections; old members take matters. Detailed, experimental schools could not gain a little ready, freq|4.14|2.23|7014002|edu packnameless #2|14|glassware|7|Home|640|cally|N/A|049945289lawn2638935|salmon|Dram|Unknown|5|oughtablebareseought| +14022|AAAAAAAAEMGDAAAA|2001-10-27||Similar shares ought to boil in the sections; old members take matters. Detailed, experimental schools could not gain a little ready, freq|0.27|0.12|6003007|exporticorp #7|3|gold|6|Jewelry|640|cally|N/A|275seashell332670360|rose|Lb|Unknown|42|ableablebareseought| +14023|AAAAAAAAHMGDAAAA|1997-10-27||Fair, modern services assess to a|4.50|1.57|9015008|scholarunivamalg #8|15|fiction|9|Books|309|n stbarpri|N/A|44165medium721128032|seashell|Dram|Unknown|87|priablebareseought| +14024|AAAAAAAAIMGDAAAA|1997-10-27|2000-10-26|Years need much. Good interests use too different, junior services. Young items shall not find. Disastrous hands release fast new, alternative applications. American police make in |7.68|4.45|1003001|exportiamalg #1|3|maternity|1|Women|193|prin stought|medium|39135638129054pink85|royal|Dram|Unknown|5|eseablebareseought| +14025|AAAAAAAAIMGDAAAA|2000-10-27||Teams settle. Prop|4.47|1.74|1003001|edu packmaxi #10|3|entertainments|9|Books|193|prin stought|N/A|39135638129054pink85|navajo|Each|Unknown|28|antiablebareseought| +14026|AAAAAAAAKMGDAAAA|1997-10-27|1999-10-27|Forward toxic areas could stay by a acts. Recent, following patients shall conduct global, social villages. Points develop so british windows. Never amazing changes update ac|0.72|0.50|10008002|namelessunivamalg #2|8|scanners|10|Electronics|758|eingantiation|N/A|2spring6495027136133|sandy|Tsp|Unknown|26|callyablebareseought| +14027|AAAAAAAAKMGDAAAA|1999-10-28|2001-10-26|Systems will not resign for example other applications. At all fi|3.31|0.50|10008002|importoscholar #2|8|country|5|Music|194|eingantiation|N/A|2spring6495027136133|slate|N/A|Unknown|31|ationablebareseought| +14028|AAAAAAAAKMGDAAAA|2001-10-27||Pairs follow above as boring authorities. Prisoners must disarm values. Creatures ought to |5.89|3.23|10008002|edu packnameless #9|8|glassware|7|Home|194|eingantiation|N/A|2spring6495027136133|papaya|Pallet|Unknown|82|eingablebareseought| +14029|AAAAAAAANMGDAAAA|1997-10-27||Available, final colours secure just able babies. Numbers call. Semantic ministers withdraw sure af|1.42|0.85|5004002|edu packscholar #2|4|classical|5|Music|24|eseable|N/A|09282450dodger049970|royal|Gross|Unknown|11|n stablebareseought| +14030|AAAAAAAAOMGDAAAA|1997-10-27|2000-10-26|Clients|1.04|0.60|3002001|importoexporti #1|2|infants|3|Children|298|eingn stable|small|6495220120670pink352|orchid|Oz|Unknown|20|barpribareseought| +14031|AAAAAAAAOMGDAAAA|2000-10-27||Magnetic, palestinian months maximise heavily to a crops. Feelings must not ensure forever able authorities. Strongly political|3.70|0.60|3002001|edu packexporti #2|2|school-uniforms|3|Children|298|eingn stable|small|6495220120670pink352|papaya|Tsp|Unknown|37|oughtpribareseought| +14032|AAAAAAAAANGDAAAA|1997-10-27|1999-10-27|American numbers know. Technically reasonable months shall|3.17|1.36|6001001|amalgcorp #1|1|birdal|6|Jewelry|829|n stableeing|N/A|114red34293208367430|frosted|N/A|Unknown|3|ablepribareseought| +14033|AAAAAAAAANGDAAAA|1999-10-28|2001-10-26|Separate, economic hundreds make. Local police invent. Available, prime lectures assess extensively voluntary sheets. Public,|0.35|0.19|2003002|exportiimporto #2|1|pants|2|Men|829|n stableeing|extra large|82purple054208346969|thistle|Oz|Unknown|28|pripribareseought| +14034|AAAAAAAAANGDAAAA|2001-10-27||Japanese customers match big, good consumers. Further new children ought to see often contexts. Prime, positive premises could not produce premises. Yet steep patterns read systematic |3.04|0.19|3001001|amalgexporti #1|1|newborn|3|Children|829|n stableeing|medium|6dark344502988907778|rose|Lb|Unknown|14|esepribareseought| +14035|AAAAAAAADNGDAAAA|1997-10-27||Active windows shall not find small, relig|5.51|2.53|8004006|edu packnameless #6|4|camping|8|Sports|632|ableprically|N/A|0turquoise7114892684|olive|Tbl|Unknown|19|antipribareseought| +14036|AAAAAAAAENGDAAAA|1997-10-27|2000-10-26|Deputies shall not promote far since a resources; suddenly young plants provide. American, only samples should think then. Routine allegations die physically to a flights. H|2.51|0.80|1003001|exportiamalg #1|3|maternity|1|Women|329|n stablepri|medium|pale5704623483369032|grey|Oz|Unknown|1|callypribareseought| +14037|AAAAAAAAENGDAAAA|2000-10-27||Circumstances will pursue yet inherent, old hours; expensive, military services call. Steady ideas would not take all tal|8.95|0.80|8012008|importomaxi #8|3|guns|8|Sports|329|n stablepri|N/A|pale5704623483369032|sky|Pound|Unknown|78|ationpribareseought| +14038|AAAAAAAAGNGDAAAA|1997-10-27|1999-10-27|Groups shall not teach really members. British, necessar|7.65|5.27|4004001|edu packedu pack #1|4|athletic|4|Shoes|147|ationeseought|medium|453322185dim48003514|grey|Pallet|Unknown|82|eingpribareseought| +14039|AAAAAAAAGNGDAAAA|1999-10-28|2001-10-26|As significant rivers shall take new databases; so as unnecessary publications omit near by a votes. Too specialist years intervene new, interesting papers. Vague colour|3.02|1.93|4004001|namelessnameless #10|4|outdoor|8|Sports|147|ationeseought|N/A|453322185dim48003514|purple|Ton|Unknown|44|n stpribareseought| +14040|AAAAAAAAGNGDAAAA|2001-10-27||As significant rivers shall take new databases; so as unnecessary publications omit near by a votes. Too specialist years intervene new, interesting papers. Vague colour|5.41|2.10|7011001|amalgnameless #1|4|accent|7|Home|158|eingantiought|N/A|453322185dim48003514|turquoise|Bunch|Unknown|11|baresebareseought| +14041|AAAAAAAAJNGDAAAA|1997-10-27||Tomorrow economic others should not free now usually diffe|8.22|5.50|3001002|amalgexporti #2|1|newborn|3|Children|24|eseable|large|251593597114salmon72|peach|Oz|Unknown|66|oughtesebareseought| +14042|AAAAAAAAKNGDAAAA|1997-10-27|2000-10-26|Now narrow studies could achieve continued children. Afrai|3.69|2.87|4004001|edu packedu pack #1|4|athletic|4|Shoes|418|eingoughtese|extra large|98steel0535017883799|powder|Ounce|Unknown|17|ableesebareseought| +14043|AAAAAAAAKNGDAAAA|2000-10-27||Now narrow studies could achieve continued children. Afrai|1.49|2.87|1001002|amalgamalg #2|1|dresses|1|Women|174|eingoughtese|small|6662olive08183673057|thistle|Bunch|Unknown|35|priesebareseought| +14044|AAAAAAAAMNGDAAAA|1997-10-27|1999-10-27|Other advantages would reject with a organizations. Social schools can write. Laws run events; large affairs would not come much. Other players ask throughout a hopes.|2.16|1.68|2004001|edu packimporto #1|4|sports-apparel|2|Men|247|ationeseable|large|721020228958snow7989|rose|Ton|Unknown|65|eseesebareseought| +14045|AAAAAAAAMNGDAAAA|1999-10-28|2001-10-26|Other advantages would reject with a organizations. Social schools can write. Laws run events; large affairs would not come much. Other players ask throughout a hopes.|7.42|1.68|2004001|edu packamalg #2|4|swimwear|1|Women|247|ationeseable|extra large|721020228958snow7989|orange|Lb|Unknown|44|antiesebareseought| +14046|AAAAAAAAMNGDAAAA|2001-10-27||Modern aspects must get most past numbers. New, only d|6.37|3.75|8010007|univmaxi #7|10|pools|8|Sports|247|ationeseable|N/A|4127804350849dark071|ghost|Tbl|Unknown|31|callyesebareseought| +14047|AAAAAAAAPNGDAAAA|1997-10-27||Healthy, old sons would score for example customs; american centres would not develop arrangements. Easy, new companies show of course outside a police|2.19|0.81|10006006|corpunivamalg #6|6|musical|10|Electronics|1|ought|N/A|5tan5558564884603545|rosy|Pound|Unknown|42|ationesebareseought| +14048|AAAAAAAAAOGDAAAA|1997-10-27|2000-10-26|Local types reduce no crucial men. Services may talk most in a parts. Formidable demands|5.82|3.20|2002001|importoimporto #1|2|shirts|2|Men|139|n stpriought|medium|4253170sienna7164997|snow|Bunch|Unknown|6|eingesebareseought| +14049|AAAAAAAAAOGDAAAA|2000-10-27||Small months would obey yet consistent names. Multiple, available minutes know now at a rates. Lost stars raise gradually. Particularly physical records provide. Issues ought to know now d|2.34|1.42|2002001|edu packnameless #10|4|camping|8|Sports|433|n stpriought|N/A|4253170sienna7164997|steel|Dozen|Unknown|9|n stesebareseought| +14050|AAAAAAAACOGDAAAA|1997-10-27|1999-10-27|Social, old letters meet ever human, interior courts. Experimental affairs would want clear, potential years. Earlier internal|68.07|51.05|6015003|scholarbrand #3|15|custom|6|Jewelry|88|eingeing|N/A|106841red58559539197|cornsilk|Cup|Unknown|28|barantibareseought| +14051|AAAAAAAACOGDAAAA|1999-10-28|2001-10-26|Social, old letters meet ever human, interior courts. Experimental affairs would want clear, potential years. Earlier internal|3.05|51.05|6015003|scholarmaxi #12|15|history|9|Books|331|oughtpripri|N/A|106841red58559539197|yellow|Tsp|Unknown|98|oughtantibareseought| +14052|AAAAAAAACOGDAAAA|2001-10-27||Social, old letters meet ever human, interior courts. Experimental affairs would want clear, potential years. Earlier internal|2.59|51.05|6015003|importoscholar #1|15|country|5|Music|331|oughtpripri|N/A|6524893995lime851234|saddle|Cup|Unknown|30|ableantibareseought| +14053|AAAAAAAAFOGDAAAA|1997-10-27||Tonight wooden standards reveal on an figure|4.82|3.13|10016012|corpamalgamalg #12|16|wireless|10|Electronics|207|ationbarable|N/A|801928059068thistle4|violet|Ounce|Unknown|22|priantibareseought| +14054|AAAAAAAAGOGDAAAA|1997-10-27|2000-10-26|Sales turn white, famili|5.39|3.61|10012013|importoamalgamalg #13|12|monitors|10|Electronics|579|n stationanti|N/A|574yellow08777116322|pink|Dram|Unknown|17|eseantibareseought| +14055|AAAAAAAAGOGDAAAA|2000-10-27||Previous activities assume in the institutions. Other profits implement more secondary refugees|6.09|3.10|10012013|amalgedu pack #2|12|womens|4|Shoes|579|n stationanti|extra large|32211714salmon494717|white|Cup|Unknown|49|antiantibareseought| +14056|AAAAAAAAIOGDAAAA|1997-10-27|1999-10-27|Relatively professional pages should face surely because of a studies. Ministers must d|9.48|4.17|4003001|exportiedu pack #1|3|kids|4|Shoes|208|eingbarable|N/A|0154985652peach63239|turquoise|Oz|Unknown|34|callyantibareseought| +14057|AAAAAAAAIOGDAAAA|1999-10-28|2001-10-26|Relatively professional pages should face surely because of a studies. Ministers must d|4.94|3.80|2002002|importoimporto #2|3|shirts|2|Men|405|eingbarable|medium|99powder815019473351|peach|Tbl|Unknown|21|ationantibareseought| +14058|AAAAAAAAIOGDAAAA|2001-10-27||Relatively professional pages should face surely because of a studies. Ministers must d|0.52|0.34|2002002|importoedu pack #1|2|mens|4|Shoes|405|antibarese|extra large|99powder815019473351|tan|Unknown|Unknown|29|eingantibareseought| +14059|AAAAAAAALOGDAAAA|1997-10-27||Stupid, superb powers see. Used, reasonable groups will deny massive lawyers. Generous, fi|8.93|7.67|1004002|edu packamalg #2|4|swimwear|1|Women|208|eingbarable|medium|394051tomato65310971|blue|Unknown|Unknown|30|n stantibareseought| +14060|AAAAAAAAMOGDAAAA|1997-10-27|2000-10-26|Prisons will not illustrate so reports. Historical, old stories might not admit long central prices. Things weaken central parents. Actual, economic grounds go rather national men. Splendid proceedi|53.86|46.31|3003001|exportiexporti #1|3|toddlers|3|Children|475|antiationese|medium|318264557powder57975|powder|Pound|Unknown|62|barcallybareseought| +14061|AAAAAAAAMOGDAAAA|2000-10-27||Prisons will not illustrate so reports. Historical, old stories might not admit long central prices. Things weaken central parents. Actual, economic grounds go rather national men. Splendid proceedi|2.15|0.64|3003001|scholarnameless #2|3|tables|7|Home|475|antiationese|N/A|318264557powder57975|thistle|N/A|Unknown|82|oughtcallybareseought| +14062|AAAAAAAAOOGDAAAA|1997-10-27|1999-10-27|Patient terms attend; groups can embrace widely only rich plans. Points could bring again. Practical, able artists realise sc|78.06|39.03|10013004|exportiamalgamalg #4|13|stereo|10|Electronics|661|oughtcallycally|N/A|1039332039318sienna8|sandy|Box|Unknown|20|ablecallybareseought| +14063|AAAAAAAAOOGDAAAA|1999-10-28|2001-10-26|Patient terms attend; groups can embrace widely only rich plans. Points could bring again. Practical, able artists realise sc|2.14|39.03|10013004|edu packimporto #2|13|sports-apparel|2|Men|661|oughtcallycally|small|1039332039318sienna8|snow|Case|Unknown|16|pricallybareseought| +14064|AAAAAAAAOOGDAAAA|2001-10-27||Patient terms attend; groups can embrace widely only rich plans. Points could bring again. Practical, able artists realise sc|1.87|39.03|10013004|amalgmaxi #5|13|archery|8|Sports|661|oughtcallycally|N/A|1039332039318sienna8|steel|Case|Unknown|24|esecallybareseought| +14065|AAAAAAAABPGDAAAA|1997-10-27||Extensive assets can adapt now fair things. White, other talks trouble sufficient teachers. Helpful days will not vot|4.62|4.01|9010002|univunivamalg #2|10|travel|9|Books|242|ableeseable|N/A|19pink78465110663011|steel|Dozen|Unknown|2|anticallybareseought| +14066|AAAAAAAACPGDAAAA|1997-10-27|2000-10-26|Then mad churches may think flat vast everyday directors.|6.76|5.61|8008007|namelessnameless #7|8|outdoor|8|Sports|45|antiese|N/A|75477713salmon245076|indian|Pound|Unknown|69|callycallybareseought| +14067|AAAAAAAACPGDAAAA|2000-10-27||Then mad churches may think flat vast everyday directors.|4.29|2.87|8008007|edu packmaxi #8|14|tennis|8|Sports|45|antiese|N/A|41850893332peach9119|plum|Tbl|Unknown|26|ationcallybareseought| +14068|AAAAAAAAEPGDAAAA|1997-10-27|1999-10-27|Children tell children. Benefits would photograph high, ancient flights. Only rural friends adhere however until the operators. Years |2.24|0.67|4003001|exportiedu pack #1|3|kids|4|Shoes|5|anti|extra large|701sienna91208925571|slate|Bundle|Unknown|9|eingcallybareseought| +14069|AAAAAAAAEPGDAAAA|1999-10-28|2001-10-26|All different practitioners scream subsequently woods. Later brilliant colleges remember from a organs. Societies cry too left writers. Implications can realize decisions. All m|2.16|1.12|4003001|importoamalg #2|2|fragrances|1|Women|153|anti|small|701sienna91208925571|ghost|Carton|Unknown|95|n stcallybareseought| +14070|AAAAAAAAEPGDAAAA|2001-10-27||Dear, previous firms go likely periods. A little previous children should turn nuclear, rich charges. Pol|1.37|0.79|4003001|exportinameless #1|3|basketball|8|Sports|153|priantiought|N/A|1840turquoise5354441|ivory|Tbl|Unknown|34|barationbareseought| +14071|AAAAAAAAHPGDAAAA|1997-10-27||Different, critical stations might start now so strong women; retail, noble changes remember by a potatoes. Problems can concede managers. Men could reach. Poles clear just |6.22|2.92|1004002|edu packamalg #2|4|swimwear|1|Women|361|oughtcallypri|medium|002938539990wheat432|pale|Dozen|Unknown|53|oughtationbareseought| +14072|AAAAAAAAIPGDAAAA|1997-10-27|2000-10-26|Publishers possess dangerous, intensive tonnes. Else other workers find wrong eyes. Miles should remove very with a rises|4.62|3.78|4003001|exportiedu pack #1|3|kids|4|Shoes|469|n stcallyese|large|62027467459steel5788|plum|Pallet|Unknown|94|ableationbareseought| +14073|AAAAAAAAIPGDAAAA|2000-10-27||Months re|4.11|3.78|4003001|exportiunivamalg #15|3|dvd/vcr players|10|Electronics|469|n stcallyese|N/A|62027467459steel5788|lawn|Cup|Unknown|99|priationbareseought| +14074|AAAAAAAAKPGDAAAA|1997-10-27|1999-10-27|Smooth, specified times must believe men. Dead, bad companies shall not like simply used, overall meetings. Extraordinary, she|2.26|1.76|8005005|scholarnameless #5|5|fitness|8|Sports|49|n stese|N/A|8595ghost87462715513|yellow|Dozen|Unknown|29|eseationbareseought| +14075|AAAAAAAAKPGDAAAA|1999-10-28|2001-10-26|Smooth, specified times must believe men. Dead, bad companies shall not like simply used, overall meetings. Extraordinary, she|0.34|0.11|5003002|exportischolar #2|5|pop|5|Music|853|priantieing|N/A|8595ghost87462715513|green|N/A|Unknown|99|antiationbareseought| +14076|AAAAAAAAKPGDAAAA|2001-10-27||R|3.94|2.12|5003002|namelessunivamalg #1|5|scanners|10|Electronics|853|priantieing|N/A|8595ghost87462715513|maroon|Dozen|Unknown|59|callyationbareseought| +14077|AAAAAAAANPGDAAAA|1997-10-27||Cars will not change too views. Talks put|1.36|0.40|3001002|amalgexporti #2|1|newborn|3|Children|634|eseprically|extra large|430601679888853peru3|rosy|Dozen|Unknown|63|ationationbareseought| +14078|AAAAAAAAOPGDAAAA|1997-10-27|2000-10-26|Households help minutes. C|2.37|1.89|8014009|edu packmaxi #9|14|tennis|8|Sports|204|esebarable|N/A|9246397puff521845782|goldenrod|Gross|Unknown|22|eingationbareseought| +14079|AAAAAAAAOPGDAAAA|2000-10-27||Households help minutes. C|0.91|1.89|4003002|exportiedu pack #2|3|kids|4|Shoes|187|esebarable|petite|188white175054340127|misty|Tbl|Unknown|15|n stationbareseought| +14080|AAAAAAAAAAHDAAAA|1997-10-27|1999-10-27|Now military users resign most national|9.20|4.41|10016005|corpamalgamalg #5|16|wireless|10|Electronics|325|antiablepri|N/A|91012629rose35932119|lime|Cup|Unknown|15|bareingbareseought| +14081|AAAAAAAAAAHDAAAA|1999-10-28|2001-10-26|Now military users resign most national|1.06|4.41|10016005|amalgunivamalg #6|11|cooking|9|Books|129|n stableought|N/A|91012629rose35932119|pink|Each|Unknown|62|oughteingbareseought| +14082|AAAAAAAAAAHDAAAA|2001-10-27||Now military users resign most national|2.14|4.41|4002001|importoedu pack #1|11|mens|4|Shoes|190|barn stought|medium|80339112frosted87334|pink|Dozen|Unknown|11|ableeingbareseought| +14083|AAAAAAAADAHDAAAA|1997-10-27||However irish police could marry naked feet. Agricultural, clinical foundations can ensure friendly readers. Authorit|4.46|2.27|9014008|edu packunivamalg #8|14|sports|9|Books|40|barese|N/A|99973078182624plum33|ghost|N/A|Unknown|13|prieingbareseought| +14084|AAAAAAAAEAHDAAAA|1997-10-27|2000-10-26|Enough japanese walls disguise often. Problems provide however particularly long times; best clear years exceed too old, unpleasant companies. Just left patterns emerge certainly too old|5.71|3.82|2002001|importoimporto #1|2|shirts|2|Men|285|antieingable|N/A|1pink252226065520322|yellow|Bundle|Unknown|26|eseeingbareseought| +14085|AAAAAAAAEAHDAAAA|2000-10-27||Enough japanese walls disguise often. Problems provide however particularly long times; best clear years exceed too old, unpleasant companies. Just left patterns emerge certainly too old|8.26|3.82|10015010|scholaramalgamalg #10|15|portable|10|Electronics|16|antieingable|N/A|832saddle28208698551|wheat|Carton|Unknown|3|antieingbareseought| +14086|AAAAAAAAGAHDAAAA|1997-10-27|1999-10-27|Areas check again. Religious seeds should monitor really nuclear objectives; improvements believe total trouse|2.31|1.36|8013007|exportimaxi #7|13|sailing|8|Sports|282|ableeingable|N/A|1583983688sky0063014|pink|N/A|Unknown|35|callyeingbareseought| +14087|AAAAAAAAGAHDAAAA|1999-10-28|2001-10-26|Areas check again. Religious seeds should monitor really nuclear objectives; improvements believe total trouse|1.65|1.36|8013007|scholarmaxi #8|13|fishing|8|Sports|120|ableeingable|N/A|yellow49496527558013|peru|Tsp|Unknown|61|ationeingbareseought| +14088|AAAAAAAAGAHDAAAA|2001-10-27||Drawings lose earlier cattle. Really commercial arms reflect. Elsewhere cheap weeks can attract. Local features must not spend well more su|6.31|5.48|6012001|importobrand #1|13|costume|6|Jewelry|120|ableeingable|N/A|8572315grey141073530|puff|Case|Unknown|14|eingeingbareseought| +14089|AAAAAAAAJAHDAAAA|1997-10-27||Coastal, |1.93|1.10|10012014|importoamalgamalg #14|12|monitors|10|Electronics|581|oughteinganti|N/A|166423740842718pink2|hot|Pound|Unknown|7|n steingbareseought| +14090|AAAAAAAAKAHDAAAA|1997-10-27|2000-10-26|Prices argue a|0.97|0.50|1003001|exportiamalg #1|3|maternity|1|Women|359|n stantipri|large|62094173146mint75775|floral|Tbl|Unknown|77|barn stbareseought| +14091|AAAAAAAAKAHDAAAA|2000-10-27||Prices argue a|6.79|0.50|1003001|importoedu pack #2|3|mens|4|Shoes|431|n stantipri|medium|0910yellow1395951151|peach|Cup|Unknown|26|oughtn stbareseought| +14092|AAAAAAAAMAHDAAAA|1997-10-27|1999-10-27|Jol|14.38|4.31|9004005|edu packmaxi #5|4|entertainments|9|Books|917|ationoughtn st|N/A|25797182olive0019239|misty|Ounce|Unknown|12|ablen stbareseought| +14093|AAAAAAAAMAHDAAAA|1999-10-28|2001-10-26|Original l|3.64|4.31|9004005|edu packcorp #6|4|bracelets|6|Jewelry|917|ationoughtn st|N/A|9peach24445750591482|yellow|Case|Unknown|35|prin stbareseought| +14094|AAAAAAAAMAHDAAAA|2001-10-27||Original l|70.85|30.46|2002001|importoimporto #1|4|shirts|2|Men|917|ationoughtn st|medium|2198587580tan1357120|peach|Ton|Unknown|9|esen stbareseought| +14095|AAAAAAAAPAHDAAAA|1997-10-27||Sexual, leading arts could mean just. Excessive, full-time claims succeed other, executive buildings. Slightly elderly |56.16|43.80|10016003|corpamalgamalg #3|16|wireless|10|Electronics|19|n stought|N/A|1269indian6029169731|drab|Unknown|Unknown|28|antin stbareseought| +14096|AAAAAAAAABHDAAAA|1997-10-27|2000-10-26|Fathers correspond double years. Really old eyes establish somewhat professional experiences. Visitors confront partly; then ruling thanks would not hurt sure. Front, key institut|5.80|3.30|1002001|importoamalg #1|2|fragrances|1|Women|204|esebarable|medium|1495892deep438076390|plum|Tsp|Unknown|62|callyn stbareseought| +14097|AAAAAAAAABHDAAAA|2000-10-27||New, public patients will pass. Scottish, national drawings get white parents.|4.93|3.40|8005008|scholarnameless #8|5|fitness|8|Sports|864|esebarable|N/A|0450812730663papaya7|rosy|Dram|Unknown|7|ationn stbareseought| +14098|AAAAAAAACBHDAAAA|1997-10-27|1999-10-27|Really royal nations challenge however esse|5.38|2.58|3004001|edu packexporti #1|4|school-uniforms|3|Children|623|priablecally|small|75red413721929070855|seashell|Each|Unknown|54|eingn stbareseought| +14099|AAAAAAAACBHDAAAA|1999-10-28|2001-10-26|Before grey operators find all authorities. |4.98|2.58|10006007|corpunivamalg #7|4|musical|10|Electronics|19|priablecally|N/A|sienna94664445164495|sandy|Gram|Unknown|70|n stn stbareseought| +14100|AAAAAAAACBHDAAAA|2001-10-27||Before grey operators find all authorities. |2.32|2.58|10006007|amalgbrand #5|11|semi-precious|6|Jewelry|19|n stought|N/A|1261037500yellow5913|brown|Lb|Unknown|42|barbaroughteseought| +14101|AAAAAAAAFBHDAAAA|1997-10-27||Flat legs hurt actually. Eastern, fatal sections carry somewhere shoes. Also whole groups shall decide very double spiritual languages; internal, critica|5.53|2.65|4004002|edu packedu pack #2|4|athletic|4|Shoes|102|ablebarought|extra large|3468427038hot6455439|violet|Tsp|Unknown|40|oughtbaroughteseought| +14102|AAAAAAAAGBHDAAAA|1997-10-27|2000-10-26|However wild years get very probably open police. Now clinical women like for example fr|3.72|2.26|2002001|importoimporto #1|2|shirts|2|Men|171|oughtationought|small|8192312764129sienna0|powder|Each|Unknown|41|ablebaroughteseought| +14103|AAAAAAAAGBHDAAAA|2000-10-27||Straight economic girls condemn so international|3.16|2.27|10002011|importounivamalg #11|2|camcorders|10|Electronics|171|oughtationought|N/A|903554317magenta2985|papaya|Unknown|Unknown|93|pribaroughteseought| +14104|AAAAAAAAIBHDAAAA|1997-10-27|1999-10-27|Years might not limit possibly resources. Components might achieve however formidable poles. Wild weeks want both certain factors. Cultural days shall take difficult,|9.42|6.87|10004012|edu packunivamalg #12|4|audio|10|Electronics|349|n stesepri|N/A|3173647958cream52160|puff|Bunch|Unknown|29|esebaroughteseought| +14105|AAAAAAAAIBHDAAAA|1999-10-28|2001-10-26|Monthly, current words might mind properly cultural points. Specialists update other, other moves; so single meetings may not know views. Election|4.05|2.91|10008013|namelessunivamalg #13|4|scanners|10|Electronics|106|callybarought|N/A|7704124332seashell88|maroon|Bundle|Unknown|51|antibaroughteseought| +14106|AAAAAAAAIBHDAAAA|2001-10-27||Botto|6.97|2.91|1002001|importoamalg #1|2|fragrances|1|Women|106|callybarought|extra large|85844midnight9276227|ghost|Ton|Unknown|29|callybaroughteseought| +14107|AAAAAAAALBHDAAAA|1997-10-27||Small, black hours accommodate plans. Great, parental records appl|0.39|0.33|4003002|exportiedu pack #2|3|kids|4|Shoes|243|prieseable|small|20peach9069628868846|plum|Dozen|Unknown|23|ationbaroughteseought| +14108|AAAAAAAAMBHDAAAA|1997-10-27|2000-10-26|Small points examine rightly situations. Curre|1.04|0.35|9014009|edu packunivamalg #9|14|sports|9|Books|37|ationpri|N/A|828446504purple94472|sienna|N/A|Unknown|2|eingbaroughteseought| +14109|AAAAAAAAMBHDAAAA|2000-10-27||Small points examine rightly situations. Curre|3.01|0.35|9014009|importoimporto #2|2|shirts|2|Men|37|ationpri|extra large|721972872metallic303|purple|Tbl|Unknown|22|n stbaroughteseought| +14110|AAAAAAAAOBHDAAAA|1997-10-27|1999-10-27|Large shares die rather. Members produce aside plans; muscles should not say earnings. Mammals know there somewhat major situations. Ever private countries should try gates. Workers impro|3.09|1.91|7001001|amalgbrand #1|1|bathroom|7|Home|461|oughtcallyese|N/A|594931rosy3124882708|turquoise|Gross|Unknown|12|baroughtoughteseought| +14111|AAAAAAAAOBHDAAAA|1999-10-28|2001-10-26|Large shares die rather. Members produce aside plans; muscles should not say earnings. Mammals know there somewhat major situations. Ever private countries should try gates. Workers impro|4.16|1.53|4004002|edu packedu pack #2|1|athletic|4|Shoes|461|oughtcallyese|medium|594931rosy3124882708|lime|Gross|Unknown|30|oughtoughtoughteseought| +14112|AAAAAAAAOBHDAAAA|2001-10-27||Large shares die rather. Members produce aside plans; muscles should not say earnings. Mammals know there somewhat major situations. Ever private countries should try gates. Workers impro|6.10|3.29|4004002|univbrand #1|1|jewelry boxes|6|Jewelry|461|oughtcallyese|N/A|774tan82584054503736|pink|Tbl|Unknown|7|ableoughtoughteseought| +14113|AAAAAAAABCHDAAAA|1997-10-27||Increased, special pound|2.52|1.20|4002002|importoedu pack #2|2|mens|4|Shoes|221|oughtableable|petite|699misty231339407307|cream|Pallet|Unknown|16|prioughtoughteseought| +14114|AAAAAAAACCHDAAAA|1997-10-27|2000-10-26|Full areas say tonight disti|1.04|0.49|3003001|exportiexporti #1|3|toddlers|3|Children|207|ationbarable|extra large|60800013184puff02828|white|Bundle|Unknown|43|eseoughtoughteseought| +14115|AAAAAAAACCHDAAAA|2000-10-27||Never important days shall play good stars. Solid, other opportunities explain also nuclear, difficult persons; tragic fields like famous, british officials. Overall good friends sh|4.09|0.49|3003001|edu packexporti #2|4|school-uniforms|3|Children|207|ationbarable|large|517169897slate592306|violet|Gross|Unknown|6|antioughtoughteseought| +14116|AAAAAAAAECHDAAAA|1997-10-27|1999-10-27|Quiet, minor drawings ought to attract only; minor, high modules might get in the incomes. British miles suggest. Medical copies look sufficiently after a changes. Ancient si|2.30|1.56|2003001|exportiimporto #1|3|pants|2|Men|283|prieingable|large|0526489156rose317085|peru|Cup|Unknown|48|callyoughtoughteseought| +14117|AAAAAAAAECHDAAAA|1999-10-28|2001-10-26|Present improvements approach real, actual investors. Estimated, left grants lick points; very large men will plan very on the needs. Relations constitute scottish followers. Students would n|2.25|1.56|6003006|exporticorp #6|3|gold|6|Jewelry|48|eingese|N/A|116367cornsilk325777|purple|Ton|Unknown|18|ationoughtoughteseought| +14118|AAAAAAAAECHDAAAA|2001-10-27||Present improvements approach real, actual investors. Estimated, left grants lick points; very large men will plan very on the needs. Relations constitute scottish followers. Students would n|4.30|3.52|6003006|scholarnameless #9|3|tables|7|Home|330|barpripri|N/A|116367cornsilk325777|red|Dram|Unknown|28|eingoughtoughteseought| +14119|AAAAAAAAHCHDAAAA|1997-10-27||More american opportunities know only then only schools. Sufficient relations might meet generally far seasons. Always selective clubs will not make sexual events. |7.06|4.23|10008010|namelessunivamalg #10|8|scanners|10|Electronics|3|pri|N/A|182027powder55182715|tomato|N/A|Unknown|97|n stoughtoughteseought| +14120|AAAAAAAAICHDAAAA|1997-10-27|2000-10-26|Grounds must make inevitably ultimately full months. Classes must exist british, considerable changes. Likely, n|1.27|0.55|1001001|amalgamalg #1|1|dresses|1|Women|115|antioughtought|economy|1983905482967linen72|sky|Ounce|Unknown|15|barableoughteseought| +14121|AAAAAAAAICHDAAAA|2000-10-27||Domestic, likely standards see alr|3.64|3.09|3004002|edu packexporti #2|4|school-uniforms|3|Children|74|eseation|medium|5747983223lime315809|wheat|Carton|Unknown|13|oughtableoughteseought| +14122|AAAAAAAAKCHDAAAA|1997-10-27|1999-10-27|Certa|9.40|4.60|1003001|exportiamalg #1|3|maternity|1|Women|115|antioughtought|small|8spring6497554722951|peach|Box|Unknown|25|ableableoughteseought| +14123|AAAAAAAAKCHDAAAA|1999-10-28|2001-10-26|Certa|4.49|4.60|7013004|exportinameless #4|13|wallpaper|7|Home|413|prioughtese|N/A|61733740264dark96049|peru|N/A|Unknown|28|priableoughteseought| +14124|AAAAAAAAKCHDAAAA|2001-10-27||Equally good problems might not understand. Great couples detail social stories. Limited police |4.93|4.60|6006005|corpcorp #5|13|rings|6|Jewelry|269|prioughtese|N/A|61733740264dark96049|peru|Unknown|Unknown|37|eseableoughteseought| +14125|AAAAAAAANCHDAAAA|1997-10-27||Neither alternativ|5.13|3.64|5003002|exportischolar #2|3|pop|5|Music|582|ableeinganti|N/A|415286043tomato73564|slate|Carton|Unknown|16|antiableoughteseought| +14126|AAAAAAAAOCHDAAAA|1997-10-27|2000-10-26|Employees must help unfortunately once |1.14|0.69|2004001|edu packimporto #1|4|sports-apparel|2|Men|158|eingantiought|petite|977724128455253navy3|saddle|Pound|Unknown|43|callyableoughteseought| +14127|AAAAAAAAOCHDAAAA|2000-10-27||Committees resign closer rarely legislative implications. Royal, responsible courses propose pro|0.27|0.69|2004001|exportiexporti #2|3|toddlers|3|Children|54|eingantiought|extra large|977724128455253navy3|tan|Case|Unknown|19|ationableoughteseought| +14128|AAAAAAAAADHDAAAA|1997-10-27|1999-10-27|Here remote conditions could take so. Guilty magazines ought to consider now cl|1.55|1.22|5002001|importoscholar #1|2|country|5|Music|57|ationanti|N/A|484621948918tan84772|wheat|Ton|Unknown|7|eingableoughteseought| +14129|AAAAAAAAADHDAAAA|1999-10-28|2001-10-26|Here remote conditions could take so. Guilty magazines ought to consider now cl|8.55|1.22|5002001|exportischolar #2|2|pop|5|Music|97|ationn st|N/A|484621948918tan84772|plum|Lb|Unknown|36|n stableoughteseought| +14130|AAAAAAAAADHDAAAA|2001-10-27||Old changes can eliminate. Unique discussions help to|4.28|3.42|1003001|exportiamalg #1|3|maternity|1|Women|97|ationn st|large|967980584824frosted7|peach|Ton|Unknown|63|barprioughteseought| +14131|AAAAAAAADDHDAAAA|1997-10-27||Rich words stand still. Now natural tables will die dangers. Forthwith national risks will not lose dat|2.97|1.33|3001002|amalgexporti #2|1|newborn|3|Children|358|eingantipri|extra large|60turquoise239872163|slate|Ounce|Unknown|18|oughtprioughteseought| +14132|AAAAAAAAEDHDAAAA|1997-10-27|2000-10-26|Only, financial elections must not say more essential, new months. Really previous |5.79|3.41|3001001|amalgexporti #1|1|newborn|3|Children|155|antiantiought|large|5361847203slate20052|mint|Tsp|Unknown|28|ableprioughteseought| +14133|AAAAAAAAEDHDAAAA|2000-10-27||Only, financial elections must not say more essential, new months. Really previous |3.74|2.05|3001001|exportiimporto #2|3|pants|2|Men|545|antiantiought|petite|4730648wheat31432067|wheat|Ton|Unknown|7|priprioughteseought| +14134|AAAAAAAAGDHDAAAA|1997-10-27|1999-10-27|Prominently general symptoms will not be|1.29|0.78|2004001|edu packimporto #1|4|sports-apparel|2|Men|392|ablen stpri|large|962391189tomato54900|smoke|Cup|Unknown|44|eseprioughteseought| +14135|AAAAAAAAGDHDAAAA|1999-10-28|2001-10-26|Implicit troubles sell more new, different|70.49|0.78|6009008|maxicorp #8|4|womens watch|6|Jewelry|392|ablen stpri|N/A|962391189tomato54900|seashell|Cup|Unknown|11|antiprioughteseought| +14136|AAAAAAAAGDHDAAAA|2001-10-27||More desirable patients might win too. Existing, bitter marks destroy very for a parts. Also presidential |2.78|1.72|7015007|scholarnameless #7|15|tables|7|Home|392|ablen stpri|N/A|962391189tomato54900|yellow|Dozen|Unknown|17|callyprioughteseought| +14137|AAAAAAAAJDHDAAAA|1997-10-27||Hands tolerate recent, empirical eggs. Goods follow lengths; hospitals can last trees. Too single products should love then previously high rooms. Full, extra injuries would see perhaps|99.96|79.96|10001011|amalgunivamalg #11|1|cameras|10|Electronics|995|antin stn st|N/A|7029654383royal58248|papaya|Ounce|Unknown|15|ationprioughteseought| +14138|AAAAAAAAKDHDAAAA|1997-10-27|2000-10-26|For example gold funds would send deliberately central, distant purposes; quietly full|0.42|0.16|6012003|importobrand #3|12|costume|6|Jewelry|557|ationantianti|N/A|9746salmon8585264508|ivory|Dram|Unknown|30|eingprioughteseought| +14139|AAAAAAAAKDHDAAAA|2000-10-27||For example gold funds would send deliberately central, distant purposes; quietly full|4.01|3.00|2002002|importoimporto #2|2|shirts|2|Men|557|ationantianti|extra large|9746salmon8585264508|olive|Each|Unknown|4|n stprioughteseought| +14140|AAAAAAAAMDHDAAAA|1997-10-27|1999-10-27|Silent, average experiments ought to form away great, basic options. Essential winners can develop so by the years. So f|2.43|0.89|6009005|maxicorp #5|9|womens watch|6|Jewelry|619|n stoughtcally|N/A|54048812snow89522122|tan|Case|Unknown|8|bareseoughteseought| +14141|AAAAAAAAMDHDAAAA|1999-10-28|2001-10-26|Silent, average experiments ought to form away great, basic options. Essential winners can develop so by the years. So f|9.04|0.89|8013002|exportimaxi #2|13|sailing|8|Sports|455|antiantiese|N/A|rose2646027989955449|spring|Carton|Unknown|30|oughteseoughteseought| +14142|AAAAAAAAMDHDAAAA|2001-10-27||Especially other men could like considerable, final mate|7.90|4.97|6016007|corpbrand #7|16|consignment|6|Jewelry|265|anticallyable|N/A|rose2646027989955449|seashell|Bunch|Unknown|36|ableeseoughteseought| +14143|AAAAAAAAPDHDAAAA|1997-10-27||Days shall build crossly communities; p|1.10|0.81|2002002|importoimporto #2|2|shirts|2|Men|262|ablecallyable|medium|peach605168793440080|navy|Gram|Unknown|60|prieseoughteseought| +14144|AAAAAAAAAEHDAAAA|1997-10-27|2000-10-26|Purposes should not see yea|0.56|0.37|2003001|exportiimporto #1|3|pants|2|Men|742|ableeseation|large|58turquoise926801228|red|Dram|Unknown|29|eseeseoughteseought| +14145|AAAAAAAAAEHDAAAA|2000-10-27||Purposes should not see yea|80.81|53.33|2003001|scholarunivamalg #2|5|karoke|10|Electronics|380|bareingpri|N/A|58turquoise926801228|smoke|Carton|Unknown|84|antieseoughteseought| +14146|AAAAAAAACEHDAAAA|1997-10-27|1999-10-27|Scenes help as a prices. Agricultural, vital mountains would highlight other teachers. There unlike months happen even. Terri|2.81|2.27|1002001|importoamalg #1|2|fragrances|1|Women|312|ableoughtpri|large|64563398pink59485225|violet|Tsp|Unknown|28|callyeseoughteseought| +14147|AAAAAAAACEHDAAAA|1999-10-28|2001-10-26|Well new years would feel with a rights. Of course easy places increase always before a schools. Eyes shal|4.66|1.95|6005004|scholarcorp #4|2|earings|6|Jewelry|312|ableoughtpri|N/A|662honeydew996178990|saddle|Cup|Unknown|36|ationeseoughteseought| +14148|AAAAAAAACEHDAAAA|2001-10-27||Well new years would feel with a rights. Of course easy places increase always before a schools. Eyes shal|49.66|1.95|9003001|exportimaxi #1|2|computers|9|Books|312|ableoughtpri|N/A|662honeydew996178990|royal|Case|Unknown|43|eingeseoughteseought| +14149|AAAAAAAAFEHDAAAA|1997-10-27||Clinical limitations keep rather apparent, chinese problems. Real schools exhibit n|4.30|1.72|8005010|scholarnameless #10|5|fitness|8|Sports|664|esecallycally|N/A|3398590salmon9537058|chiffon|Dozen|Unknown|24|n steseoughteseought| +14150|AAAAAAAAGEHDAAAA|1997-10-27|2000-10-26|Emissions will tick social, likely institutions. Specific customs wash still general, financial years. Open nurses could hurt; carefully current troubles must not invest als|4.98|4.13|7007001|brandbrand #1|7|decor|7|Home|481|oughteingese|N/A|287saddle49720526245|sienna|Cup|Unknown|29|barantioughteseought| +14151|AAAAAAAAGEHDAAAA|2000-10-27||Emissions will tick social, likely institutions. Specific customs wash still general, financial years. Open nurses could hurt; carefully current troubles must not invest als|7.57|2.95|6008008|namelesscorp #8|8|mens watch|6|Jewelry|56|callyanti|N/A|287saddle49720526245|peach|Bundle|Unknown|30|oughtantioughteseought| +14152|AAAAAAAAIEHDAAAA|1997-10-27|1999-10-27|Relations must not want. Generally econo|1.21|1.07|7009003|maxibrand #3|9|mattresses|7|Home|262|ablecallyable|N/A|871006389002floral92|white|Gross|Unknown|40|ableantioughteseought| +14153|AAAAAAAAIEHDAAAA|1999-10-28|2001-10-26|Months help students. Small attitudes may provide here british details. Prices would not kill new departme|28.17|1.07|7009003|amalgexporti #2|1|newborn|3|Children|870|barationeing|large|19400pink01874975160|plum|Bunch|Unknown|61|priantioughteseought| +14154|AAAAAAAAIEHDAAAA|2001-10-27||Possible, subsequent hundreds know thanks; powerful, central others start now. Serious, marginal priorities must control then serious rates; es|3.70|2.66|5004001|edu packscholar #1|4|classical|5|Music|202|barationeing|N/A|19400pink01874975160|turquoise|Box|Unknown|5|eseantioughteseought| +14155|AAAAAAAALEHDAAAA|1997-10-27||Essential methods used to identify far different moveme|3.86|3.39|4001002|amalgedu pack #2|1|womens|4|Shoes|351|oughtantipri|medium|072874916snow1511709|ghost|N/A|Unknown|29|antiantioughteseought| +14156|AAAAAAAAMEHDAAAA|1997-10-27|2000-10-26|Successful, current times can counteract perhaps to a women. Almost automatic institutions can want yet popular national unions. Shares think nearly prime gifts; fixed serv|59.16|28.98|10014013|edu packamalgamalg #13|14|automotive|10|Electronics|561|oughtcallyanti|N/A|18924913452253sky489|royal|Unknown|Unknown|48|callyantioughteseought| +14157|AAAAAAAAMEHDAAAA|2000-10-27||Successful, current times can counteract perhaps to a women. Almost automatic institutions can want yet popular national unions. Shares think nearly prime gifts; fixed serv|6.99|2.09|10014013|edu packexporti #2|4|school-uniforms|3|Children|561|oughtcallyanti|medium|3764414dim4405826893|puff|N/A|Unknown|2|ationantioughteseought| +14158|AAAAAAAAOEHDAAAA|1997-10-27|1999-10-27|Eyes would deceive now common towns. Democratic symptoms must not think workers. Parents put eastern attitudes; low child|8.02|3.04|5002001|importoscholar #1|2|country|5|Music|54|eseanti|N/A|95antique33655008008|hot|Bundle|Unknown|32|eingantioughteseought| +14159|AAAAAAAAOEHDAAAA|1999-10-28|2001-10-26|Eyes would deceive now common towns. Democratic symptoms must not think workers. Parents put eastern attitudes; low child|3.18|3.04|5002001|importoscholar #2|2|country|5|Music|54|eseanti|N/A|95antique33655008008|floral|Pallet|Unknown|49|n stantioughteseought| +14160|AAAAAAAAOEHDAAAA|2001-10-27||Certain patients could say of course spiritual savings. Ministers would not leave hence for a nations. Good c|7.16|4.58|5002001|corpnameless #1|16|furniture|7|Home|54|eseanti|N/A|95antique33655008008|ghost|Ounce|Unknown|17|barcallyoughteseought| +14161|AAAAAAAABFHDAAAA|1997-10-27||Little days answer in a emotions; players touch.|2.58|1.10|9001002|amalgmaxi #2|1|arts|9|Books|58|einganti|N/A|310425845violet13327|navajo|Lb|Unknown|18|oughtcallyoughteseought| +14162|AAAAAAAACFHDAAAA|1997-10-27|2000-10-26|Crucial, willing styles used to derive in a women. Catholic, other controls sho|1.49|0.73|8016003|corpmaxi #3|16|golf|8|Sports|552|ableantianti|N/A|70355563487white3322|dodger|Gram|Unknown|65|ablecallyoughteseought| +14163|AAAAAAAACFHDAAAA|2000-10-27||Grey practitioners should not administer women. Mee|1.58|1.21|8016003|scholaramalgamalg #3|15|portable|10|Electronics|410|baroughtese|N/A|84427lavender2454467|wheat|Tsp|Unknown|7|pricallyoughteseought| +14164|AAAAAAAAEFHDAAAA|1997-10-27|1999-10-27|Female birds like still years; |2.27|0.77|7009005|maxibrand #5|9|mattresses|7|Home|320|barablepri|N/A|0916peach88302710003|peru|Case|Unknown|43|esecallyoughteseought| +14165|AAAAAAAAEFHDAAAA|1999-10-28|2001-10-26|Particular shares save far also poor teeth; races pay high long men. Effective, public shops could use; alone paintings become about fede|19.46|15.56|7009005|exportinameless #6|3|basketball|8|Sports|320|barablepri|N/A|0916peach88302710003|papaya|Box|Unknown|9|anticallyoughteseought| +14166|AAAAAAAAEFHDAAAA|2001-10-27||Annual, particular games make awkwardly. Methods should not come; more suitable states hold by a talks. Main, pain|1.54|1.23|3001001|amalgexporti #1|1|newborn|3|Children|64|esecally|small|46939pale36741814935|ivory|Case|Unknown|1|callycallyoughteseought| +14167|AAAAAAAAHFHDAAAA|1997-10-27||Years should not know largely just clear times. Imperial, only qualities used to|1.44|1.29|2001002|amalgimporto #2|1|accessories|2|Men|303|pribarpri|medium|85606turquoise424885|olive|Gross|Unknown|23|ationcallyoughteseought| +14168|AAAAAAAAIFHDAAAA|1997-10-27|2000-10-26|Problems would end days. Grand, electronic directors can open too true papers. Men shall make otherwi|9.93|3.97|10012008|importoamalgamalg #8|12|monitors|10|Electronics|137|ationpriought|N/A|1magenta916465198195|lavender|N/A|Unknown|45|eingcallyoughteseought| +14169|AAAAAAAAIFHDAAAA|2000-10-27||Problems would end days. Grand, electronic directors can open too true papers. Men shall make otherwi|4.81|3.97|10012008|edu packscholar #2|12|classical|5|Music|137|ationpriought|N/A|1magenta916465198195|black|Each|Unknown|2|n stcallyoughteseought| +14170|AAAAAAAAKFHDAAAA|1997-10-27|1999-10-27|Social shops could not marry currently individually continental children; at least nice details offer |2.54|1.70|8010001|univmaxi #1|10|pools|8|Sports|964|esecallyn st|N/A|921086057seashell198|seashell|Lb|Unknown|15|barationoughteseought| +14171|AAAAAAAAKFHDAAAA|1999-10-28|2001-10-26|Social shops could not marry currently individually continental children; at least nice details offer |3.29|2.43|6007004|brandcorp #4|10|pendants|6|Jewelry|256|callyantiable|N/A|921086057seashell198|royal|N/A|Unknown|44|oughtationoughteseought| +14172|AAAAAAAAKFHDAAAA|2001-10-27||Social shops could not marry currently individually continental children; at least nice details offer |2.51|1.10|2003001|exportiimporto #1|10|pants|2|Men|256|callyantiable|petite|921086057seashell198|steel|Dram|Unknown|6|ableationoughteseought| +14173|AAAAAAAANFHDAAAA|1997-10-27||Accessible, sure opportunities used to help; too good films would not see |9.91|6.34|9008002|namelessmaxi #2|8|romance|9|Books|539|n stprianti|N/A|3tan7912625935770585|royal|Gram|Unknown|16|priationoughteseought| +14174|AAAAAAAAOFHDAAAA|1997-10-27|2000-10-26|Most only applicants provide. Feet used to think much new expenses. Automatic, particular products would say relative, |4.92|1.91|10011014|amalgamalgamalg #14|11|disk drives|10|Electronics|198|eingn stought|N/A|580891273176wheat913|hot|Ounce|Unknown|95|eseationoughteseought| +14175|AAAAAAAAOFHDAAAA|2000-10-27||Most only applicants provide. Feet used to think much new expenses. Automatic, particular products would say relative, |2.67|1.91|10011014|importoamalgamalg #15|12|monitors|10|Electronics|16|callyought|N/A|60353527707051sky324|peru|Tbl|Unknown|17|antiationoughteseought| +14176|AAAAAAAAAGHDAAAA|1997-10-27|1999-10-27|Orders support prisoners. Indeed annual participants know over a libraries. Concerned interests write sol|8.08|6.14|5001001|amalgscholar #1|1|rock|5|Music|580|bareinganti|N/A|919061saddle17632073|peach|Unknown|Unknown|2|callyationoughteseought| +14177|AAAAAAAAAGHDAAAA|1999-10-28|2001-10-26|Orders support prisoners. Indeed annual participants know over a libraries. Concerned interests write sol|1.72|6.14|3001002|amalgexporti #2|1|newborn|3|Children|580|bareinganti|petite|9724624547308sky0063|red|Pallet|Unknown|23|ationationoughteseought| +14178|AAAAAAAAAGHDAAAA|2001-10-27||Orders support prisoners. Indeed annual participants know over a libraries. Concerned interests write sol|8.36|6.14|9013007|exportiunivamalg #7|13|self-help|9|Books|580|bareinganti|N/A|578610712linen233488|aquamarine|Dram|Unknown|10|eingationoughteseought| +14179|AAAAAAAADGHDAAAA|1997-10-27||Needs could turn net, delicate flowers. General seconds offer; open fi|2.76|1.68|2004002|edu packimporto #2|4|sports-apparel|2|Men|578|eingationanti|small|08999274saddle771220|navajo|Pallet|Unknown|42|n stationoughteseought| +14180|AAAAAAAAEGHDAAAA|1997-10-27|2000-10-26|Various styles know young women. Eyes will blow perhaps. Whole, un|8.42|3.36|6006005|corpcorp #5|6|rings|6|Jewelry|56|callyanti|N/A|619white795677583374|tan|Gram|Unknown|82|bareingoughteseought| +14181|AAAAAAAAEGHDAAAA|2000-10-27||Details train able citizens. Then white pp. will use though other shelves; children feel no longer proposed, red policies; monthly, redundant pr|5.35|2.62|9013010|exportiunivamalg #10|6|self-help|9|Books|56|callyanti|N/A|0660822894908ivory46|spring|Pallet|Unknown|15|oughteingoughteseought| +14182|AAAAAAAAGGHDAAAA|1997-10-27|1999-10-27|Tracks reappear products. Special days can enjoy of course problems. Attempts cannot ensur|2.75|1.89|7015003|scholarnameless #3|15|tables|7|Home|332|ablepripri|N/A|spring31882623633763|seashell|Case|Unknown|51|ableeingoughteseought| +14183|AAAAAAAAGGHDAAAA|1999-10-28|2001-10-26|Tracks reappear products. Special days can enjoy of course problems. Attempts cannot ensur|9.34|1.89|5004002|edu packscholar #2|15|classical|5|Music|302|ablebarpri|N/A|25721415pale56375563|white|Tsp|Unknown|54|prieingoughteseought| +14184|AAAAAAAAGGHDAAAA|2001-10-27||Tracks reappear products. Special days can enjoy of course problems. Attempts cannot ensur|9.75|1.89|8002005|importonameless #5|2|baseball|8|Sports|13|priought|N/A|25721415pale56375563|turquoise|Ounce|Unknown|9|eseeingoughteseought| +14185|AAAAAAAAJGHDAAAA|1997-10-27||Meetings achieve rational, young wages. W|3.42|1.43|7010006|univnameless #6|10|flatware|7|Home|286|callyeingable|N/A|1155026linen16784308|papaya|Oz|Unknown|43|antieingoughteseought| +14186|AAAAAAAAKGHDAAAA|1997-10-27|2000-10-26|General changes might spend even. Countries will roll strong, attractive ideas. Long services will not trace small, christian attempts. Top users support in the voters; computers ought to announce p|6.31|3.84|2004001|edu packimporto #1|4|sports-apparel|2|Men|77|ationation|extra large|329631447412714tan69|tomato|Gross|Unknown|54|callyeingoughteseought| +14187|AAAAAAAAKGHDAAAA|2000-10-27||General changes might spend even. Countries will roll strong, attractive ideas. Long services will not trace small, christian attempts. Top users support in the voters; computers ought to announce p|4.71|3.84|5003002|exportischolar #2|4|pop|5|Music|77|ationation|N/A|329631447412714tan69|seashell|Carton|Unknown|41|ationeingoughteseought| +14188|AAAAAAAAMGHDAAAA|1997-10-27|1999-10-27|Western areas ought to prepare t|1.13|0.37|5001001|amalgscholar #1|1|rock|5|Music|506|callybaranti|N/A|967817563369thistle7|plum|Dozen|Unknown|64|eingeingoughteseought| +14189|AAAAAAAAMGHDAAAA|1999-10-28|2001-10-26|Usual banks would t|3.36|0.37|3002002|importoexporti #2|2|infants|3|Children|185|antieingought|large|967817563369thistle7|seashell|Case|Unknown|23|n steingoughteseought| +14190|AAAAAAAAMGHDAAAA|2001-10-27||Usual banks would t|0.17|0.13|1002001|importoamalg #1|2|fragrances|1|Women|311|oughtoughtpri|medium|967817563369thistle7|snow|Pallet|Unknown|33|barn stoughteseought| +14191|AAAAAAAAPGHDAAAA|1997-10-27||Single r|9.61|8.45|10008014|namelessunivamalg #14|8|scanners|10|Electronics|194|esen stought|N/A|1puff107752017902381|tan|Dozen|Unknown|54|oughtn stoughteseought| +14192|AAAAAAAAAHHDAAAA|1997-10-27|2000-10-26|Realistic, unusual women close russian seats; new deaths should underestimate men; operational duties used to return most by a jobs. Then emp|6.99|4.75|5004001|edu packscholar #1|4|classical|5|Music|146|callyeseought|N/A|3purple8913194756401|linen|Each|Unknown|13|ablen stoughteseought| +14193|AAAAAAAAAHHDAAAA|2000-10-27||Co|8.30|5.81|5004001|importoamalg #2|2|fragrances|1|Women|146|callyeseought|medium|6472thistle598556443|pale|Carton|Unknown|23|prin stoughteseought| +14194|AAAAAAAACHHDAAAA|1997-10-27|1999-10-27|Total, b|4.46|2.58|6004003|edu packcorp #3|4|bracelets|6|Jewelry|261|oughtcallyable|N/A|1528998light13818967|saddle|Ounce|Unknown|29|esen stoughteseought| +14195|AAAAAAAACHHDAAAA|1999-10-28|2001-10-26|Total, b|1.38|1.18|6004003|importoimporto #2|2|shirts|2|Men|572|oughtcallyable|small|1528998light13818967|cornflower|Each|Unknown|40|antin stoughteseought| +14196|AAAAAAAACHHDAAAA|2001-10-27||Then other schools encourage n|0.32|1.18|3001001|amalgexporti #1|2|newborn|3|Children|572|oughtcallyable|petite|1528998light13818967|spring|Bundle|Unknown|83|callyn stoughteseought| +14197|AAAAAAAAFHHDAAAA|1997-10-27||Foreign children can benefit easy whole, social miles. Dry, other injuries go for ever in the windows. Large, small years c|3.42|2.87|5003002|exportischolar #2|3|pop|5|Music|100|barbarought|N/A|0112gainsboro2630581|spring|Bundle|Unknown|28|ationn stoughteseought| +14198|AAAAAAAAGHHDAAAA|1997-10-27|2000-10-26|Futures should enjoy able galleries. Late blue tickets pass longer urgently dead types. Shoulders will see rigidly institutions. Other con|2.64|2.11|9005003|scholarmaxi #3|5|history|9|Books|110|baroughtought|N/A|2ivory61315500068363|orchid|Gram|Unknown|62|eingn stoughteseought| +14199|AAAAAAAAGHHDAAAA|2000-10-27||Thus european cats work also later psychological problems. Cell|2.71|2.11|9003004|exportimaxi #4|3|computers|9|Books|993|prin stn st|N/A|2ivory61315500068363|grey|Ton|Unknown|27|n stn stoughteseought| +14200|AAAAAAAAIHHDAAAA|1997-10-27|1999-10-27|Rich, continental arrangements must make after a passengers.|5.86|3.80|3004001|edu packexporti #1|4|school-uniforms|3|Children|477|ationationese|extra large|27790779sienna008732|tan|Bunch|Unknown|3|barbarableeseought| +14201|AAAAAAAAIHHDAAAA|1999-10-28|2001-10-26|Always clear operations maintain quite economic teeth. Visitors should make never still clear months. Lexical, general f|3.44|3.80|8016002|corpmaxi #2|16|golf|8|Sports|97|ationn st|N/A|27790779sienna008732|sky|N/A|Unknown|71|oughtbarableeseought| +14202|AAAAAAAAIHHDAAAA|2001-10-27||Long, unlikely schools explain; patient, occupational shares help quite with the records. Institutions shall publish above free players; reasonable birds will make adequate relations|9.21|6.90|3001001|amalgexporti #1|16|newborn|3|Children|36|callypri|petite|27790779sienna008732|powder|Lb|Unknown|17|ablebarableeseought| +14203|AAAAAAAALHHDAAAA|1997-10-27||Obviously careful weeks should not realise only in a services. Leaders hit very actual, necessary pounds. Liberal partners ought to offer as strong directors. Well friendly risks run carers. Never |9.64|3.47|2001002|amalgimporto #2|1|accessories|2|Men|350|barantipri|medium|980338686291yellow18|thistle|Dozen|Unknown|13|pribarableeseought| +14204|AAAAAAAAMHHDAAAA|1997-10-27|2000-10-26|For example relative proceedings would not depend instead by an troops; necessarily op|55.38|34.33|3001001|amalgexporti #1|1|newborn|3|Children|6|cally|medium|0452822303643959red6|thistle|Bundle|Unknown|3|esebarableeseought| +14205|AAAAAAAAMHHDAAAA|2000-10-27||Uncomfortable attacks might organise public stairs. Signs may disappear similarly between a companies. Western relations see abilities. Absolute prisoners|5.23|34.33|3001001|exportibrand #6|13|loose stones|6|Jewelry|502|ablebaranti|N/A|0452822303643959red6|royal|Ton|Unknown|19|antibarableeseought| +14206|AAAAAAAAOHHDAAAA|1997-10-27|1999-10-27|Together sheer samples shall rush so. Hours recognize low real inches. Important, visible chapters belong. Wrong, local arms ought to give actually all |2.44|1.12|1002001|importoamalg #1|2|fragrances|1|Women|77|ationation|medium|03647purple755958111|violet|Bunch|Unknown|54|callybarableeseought| +14207|AAAAAAAAOHHDAAAA|1999-10-28|2001-10-26|Together sheer samples shall rush so. Hours recognize low real inches. Important, visible chapters belong. Wrong, local arms ought to give actually all |3.11|2.70|5003002|exportischolar #2|3|pop|5|Music|88|ationation|N/A|9sienna9422874245718|lawn|Oz|Unknown|50|ationbarableeseought| +14208|AAAAAAAAOHHDAAAA|2001-10-27||Well-known, famous structures work most interested, long years. Slightly similar pp. would bring like, mere cattle; hands should not be|4.87|2.70|5003002|edu packedu pack #1|4|athletic|4|Shoes|929|ationation|large|9sienna9422874245718|tomato|Cup|Unknown|48|eingbarableeseought| +14209|AAAAAAAABIHDAAAA|1997-10-27||Genes say. Changes may not plant to a countries. Other persons may share more blind cameras. New, classic pe|6.43|4.30|5003002|exportischolar #2|3|pop|5|Music|427|ationableese|N/A|147836003938tan52743|metallic|Ounce|Unknown|13|n stbarableeseought| +14210|AAAAAAAACIHDAAAA|1997-10-27|2000-10-26|Other, upper visitors allow still soviet teams; examples could say readily|9.55|7.06|8001001|amalgnameless #1|1|athletic shoes|8|Sports|244|eseeseable|N/A|94991766goldenrod849|powder|Each|Unknown|20|baroughtableeseought| +14211|AAAAAAAACIHDAAAA|2000-10-27||Other, upper visitors allow still soviet teams; examples could say readily|3.09|7.06|4004002|edu packedu pack #2|1|athletic|4|Shoes|244|eseeseable|large|94991766goldenrod849|sandy|Carton|Unknown|22|oughtoughtableeseought| +14212|AAAAAAAAEIHDAAAA|1997-10-27|1999-10-27|Familiar words make in general maps. Most small youngsters spend ever suddenly geographical parts. Entirely full connectio|6.27|3.19|10014001|edu packamalgamalg #1|14|automotive|10|Electronics|374|eseationpri|N/A|plum1594220421775276|pink|Tsp|Unknown|24|ableoughtableeseought| +14213|AAAAAAAAEIHDAAAA|1999-10-28|2001-10-26|Familiar words make in general maps. Most small youngsters spend ever suddenly geographical parts. Entirely full connectio|2.90|3.19|10014001|maxibrand #4|14|mattresses|7|Home|374|eseationpri|N/A|6443tan6532167267723|white|Ounce|Unknown|1|prioughtableeseought| +14214|AAAAAAAAEIHDAAAA|2001-10-27||Familiar words make in general maps. Most small youngsters spend ever suddenly geographical parts. Entirely full connectio|4.85|3.19|2002001|importoimporto #1|2|shirts|2|Men|551|eseationpri|medium|6443tan6532167267723|slate|Ton|Unknown|16|eseoughtableeseought| +14215|AAAAAAAAHIHDAAAA|1997-10-27||So damp tests imagine resources. Innocently prime developments shall work small pl|0.61|0.33|7015006|scholarnameless #6|15|tables|7|Home|246|callyeseable|N/A|3643sky5128192199447|smoke|Tsp|Unknown|39|antioughtableeseought| +14216|AAAAAAAAIIHDAAAA|1997-10-27|2000-10-26|Underlying, new workers would privatise original hospitals; wom|3.97|3.25|4002001|importoedu pack #1|2|mens|4|Shoes|75|antiation|extra large|pink7119497245035502|royal|Pound|Unknown|38|callyoughtableeseought| +14217|AAAAAAAAIIHDAAAA|2000-10-27||Underlying, new workers would privatise original hospitals; wom|4.15|3.25|4002001|amalgamalg #2|1|dresses|1|Women|75|antiation|large|8635961210peru833812|tan|Unknown|Unknown|26|ationoughtableeseought| +14218|AAAAAAAAKIHDAAAA|1997-10-27|1999-10-27|Solutions ask over joint, political scots. General copies meet different police; top meetings make subject solicitors. Possible, local members shall use ve|3.03|2.48|6013003|exportibrand #3|13|loose stones|6|Jewelry|664|esecallycally|N/A|64royal1107335866083|peru|Each|Unknown|16|eingoughtableeseought| +14219|AAAAAAAAKIHDAAAA|1999-10-28|2001-10-26|Solutions ask over joint, political scots. General copies meet different police; top meetings make subject solicitors. Possible, local members shall use ve|0.13|2.48|8003010|exportinameless #10|3|basketball|8|Sports|106|esecallycally|N/A|64royal1107335866083|thistle|Unknown|Unknown|23|n stoughtableeseought| +14220|AAAAAAAAKIHDAAAA|2001-10-27||Important trees ought to take by the feet. Large da|18.71|2.48|3004001|edu packexporti #1|4|school-uniforms|3|Children|144|eseeseought|small|64royal1107335866083|white|Dram|Unknown|28|barableableeseought| +14221|AAAAAAAANIHDAAAA|1997-10-27||Legal, new note|4.33|1.51|10008010|namelessunivamalg #10|8|scanners|10|Electronics|34|esepri|N/A|2154723871plum683641|snow|Pallet|Unknown|39|oughtableableeseought| +14222|AAAAAAAAOIHDAAAA|1997-10-27|2000-10-26|Other decisions will not think with a members. Much actual sections stop still. Together able others come|3.35|2.64|2003001|exportiimporto #1|3|pants|2|Men|121|oughtableought|small|9304750green18642633|olive|Pound|Unknown|26|ableableableeseought| +14223|AAAAAAAAOIHDAAAA|2000-10-27||Other decisions will not think with a members. Much actual sections stop still. Together able others come|23.04|2.64|6004008|edu packcorp #8|4|bracelets|6|Jewelry|758|eingantiation|N/A|9304750green18642633|spring|Dozen|Unknown|29|priableableeseought| +14224|AAAAAAAAAJHDAAAA|1997-10-27|1999-10-27|Specific policies exist for a countries. More dead tears play more police. Examples must not wash still very electrical applications. Separate children become with a police. Probably small relati|2.87|1.92|10005013|scholarunivamalg #13|5|karoke|10|Electronics|223|priableable|N/A|978576483thistle4659|smoke|Bundle|Unknown|97|eseableableeseought| +14225|AAAAAAAAAJHDAAAA|1999-10-28|2001-10-26|New, ful|8.88|3.72|7001006|amalgbrand #6|5|bathroom|7|Home|223|priableable|N/A|3694826387942grey204|white|Bunch|Unknown|20|antiableableeseought| +14226|AAAAAAAAAJHDAAAA|2001-10-27||New, ful|9.12|5.65|8011007|amalgmaxi #7|11|archery|8|Sports|223|priableable|N/A|1382391353196lawn401|smoke|Bundle|Unknown|30|callyableableeseought| +14227|AAAAAAAADJHDAAAA|1997-10-27||Har|0.70|0.57|9013008|exportiunivamalg #8|13|self-help|9|Books|133|pripriought|N/A|045984yellow72531853|peach|N/A|Unknown|12|ationableableeseought| +14228|AAAAAAAAEJHDAAAA|1997-10-27|2000-10-26|Additional pp. come. At first african forces seek maybe; unknown stages may raise off a papers; books lead above hard terms. Inches show intimately ordinary questions. Great, silly anim|4.88|2.73|2002001|importoimporto #1|2|shirts|2|Men|64|esecally|small|548spring97804412683|ghost|Oz|Unknown|47|eingableableeseought| +14229|AAAAAAAAEJHDAAAA|2000-10-27||Additional pp. come. At first african forces seek maybe; unknown stages may raise off a papers; books lead above hard terms. Inches show intimately ordinary questions. Great, silly anim|5.68|4.26|2002001|exportiimporto #2|2|pants|2|Men|64|esecally|large|548spring97804412683|papaya|Bundle|Unknown|26|n stableableeseought| +14230|AAAAAAAAGJHDAAAA|1997-10-27|1999-10-27|Effective needs may not improve old bonds. Courts cannot come only with a sources. Before proud files like just partial authorities. Parliam|0.97|0.63|9007011|brandmaxi #11|7|reference|9|Books|198|eingn stought|N/A|8055175papaya1092423|smoke|Cup|Unknown|50|barpriableeseought| +14231|AAAAAAAAGJHDAAAA|1999-10-28|2001-10-26|Effective needs may not improve old bonds. Courts cannot come only with a sources. Before proud files like just partial authorities. Parliam|6.97|0.63|9007011|edu packimporto #2|4|sports-apparel|2|Men|198|eingn stought|medium|553242696cyan2960513|slate|Tbl|Unknown|23|oughtpriableeseought| +14232|AAAAAAAAGJHDAAAA|2001-10-27||Strong, inner volumes face following arts. Children might not purch|1.27|0.63|9007011|edu packamalg #1|4|swimwear|1|Women|198|eingn stought|large|553242696cyan2960513|chocolate|Gross|Unknown|31|ablepriableeseought| +14233|AAAAAAAAJJHDAAAA|1997-10-27||Files separate all relevant needs. For example prime minutes try satisfactory hands. Religious, old hands may not modify very homes. Well n|70.35|60.50|1002002|importoamalg #2|2|fragrances|1|Women|428|eingableese|large|5823676red8619676189|white|Tsp|Unknown|34|pripriableeseought| +14234|AAAAAAAAKJHDAAAA|1997-10-27|2000-10-26|Resources shall address quick, other differences. New, european records may locate polit|2.03|1.76|1003001|exportiamalg #1|3|maternity|1|Women|330|barpripri|petite|09014936040plum36531|seashell|Bunch|Unknown|7|esepriableeseought| +14235|AAAAAAAAKJHDAAAA|2000-10-27||Resources shall address quick, other differences. New, european records may locate polit|2.98|1.81|4003002|exportiedu pack #2|3|kids|4|Shoes|330|barpripri|medium|09014936040plum36531|papaya|Pallet|Unknown|31|antipriableeseought| +14236|AAAAAAAAMJHDAAAA|1997-10-27|1999-10-27|Words may complete earlier by a members. Secondary, equal forests shall not see equally living, alternative directors. Considerably equivalent procedure|0.63|0.33|6006005|corpcorp #5|6|rings|6|Jewelry|750|barantiation|N/A|7slate71732594769074|puff|Pound|Unknown|21|callypriableeseought| +14237|AAAAAAAAMJHDAAAA|1999-10-28|2001-10-26|Words may complete earlier by a members. Secondary, equal forests shall not see equally living, alternative directors. Considerably equivalent procedure|2.20|0.33|7014008|edu packnameless #8|6|glassware|7|Home|750|barantiation|N/A|7slate71732594769074|magenta|N/A|Unknown|23|ationpriableeseought| +14238|AAAAAAAAMJHDAAAA|2001-10-27||Changes complete now to the conditions. Public, great principles may not seem by a materials. Common arrangements examine together even so other years. Magnetic studies r|2.18|1.70|6006007|corpcorp #7|6|rings|6|Jewelry|750|barantiation|N/A|7slate71732594769074|rose|Gross|Unknown|58|eingpriableeseought| +14239|AAAAAAAAPJHDAAAA|1997-10-27||Nasty drugs beat recently still open facts. Bad sales go here too detailed producers. Priorities account also in a operations.|4.12|2.18|4004002|edu packedu pack #2|4|athletic|4|Shoes|154|eseantiought|small|6870686344558265sky7|wheat|Bundle|Unknown|30|n stpriableeseought| +14240|AAAAAAAAAKHDAAAA|1997-10-27|2000-10-26|Normal products must fulfil merchants. Very decisive difficulties should decre|8.60|3.95|1004001|edu packamalg #1|4|swimwear|1|Women|89|n steing|medium|10040748saddle418426|white|N/A|Unknown|44|bareseableeseought| +14241|AAAAAAAAAKHDAAAA|2000-10-27||All scottish forms think strong, fixed targets. Stocks shall not inform perhaps genuine, narrow operations. Other, blank problems would transform also existing findings. Trousers contempl|6.73|2.82|1004001|amalgexporti #2|4|newborn|3|Children|89|n steing|small|10040748saddle418426|ghost|Pallet|Unknown|9|oughteseableeseought| +14242|AAAAAAAACKHDAAAA|1997-10-27|1999-10-27|Good, old nerves might become recently objectives. However only offices find due. Long, great sentences complain absolutely probably simple hands. |74.30|52.75|10012014|importoamalgamalg #14|12|monitors|10|Electronics|126|callyableought|N/A|sandy040252519564239|sienna|Ounce|Unknown|33|ableeseableeseought| +14243|AAAAAAAACKHDAAAA|1999-10-28|2001-10-26|Doubtfully economic arguments pay inside right, working-class authorities. Years cannot understand in a|3.42|1.16|1004002|edu packamalg #2|4|swimwear|1|Women|139|n stpriought|medium|sandy040252519564239|snow|Cup|Unknown|10|prieseableeseought| +14244|AAAAAAAACKHDAAAA|2001-10-27||Doubtfully economic arguments pay inside right, working-class authorities. Years cannot understand in a|2.43|1.43|1004002|corpbrand #5|4|consignment|6|Jewelry|139|n stpriought|N/A|sandy040252519564239|purple|Bundle|Unknown|51|eseeseableeseought| +14245|AAAAAAAAFKHDAAAA|1997-10-27||Departments make once again police. Very acceptable results call still extended, known ends; relationships shoot strangely. Acids shall discharge in order ethnic, ric|1.38|0.53|4001002|amalgedu pack #2|1|womens|4|Shoes|545|antieseanti|medium|930snow2191785704485|indian|Each|Unknown|6|antieseableeseought| +14246|AAAAAAAAGKHDAAAA|1997-10-27|2000-10-26|W|62.18|42.28|6015007|scholarbrand #7|15|custom|6|Jewelry|110|baroughtought|N/A|97lavender8516700327|khaki|Bunch|Unknown|63|callyeseableeseought| +14247|AAAAAAAAGKHDAAAA|2000-10-27||However medieval beaches should finish good associated existing flames. Average things pronounce about|1.43|42.28|6015007|edu packnameless #8|4|camping|8|Sports|110|baroughtought|N/A|351528485429red79227|ghost|Unknown|Unknown|10|ationeseableeseought| +14248|AAAAAAAAIKHDAAAA|1997-10-27|1999-10-27|No longer local women seem cautious elections. Other elements used to develop usually. Recently urban doors shall pull of course high activities. Now inherent households see. High |2.84|2.41|6009001|maxicorp #1|9|womens watch|6|Jewelry|255|antiantiable|N/A|4spring5621556517142|spring|Oz|Unknown|37|eingeseableeseought| +14249|AAAAAAAAIKHDAAAA|1999-10-28|2001-10-26|No longer local women seem cautious elections. Other elements used to develop usually. Recently urban doors shall pull of course high activities. Now inherent households see. High |3.40|1.83|6009001|amalgexporti #2|9|newborn|3|Children|159|antiantiable|medium|4spring5621556517142|saddle|Gram|Unknown|24|n steseableeseought| +14250|AAAAAAAAIKHDAAAA|2001-10-27||No longer local women seem cautious elections. Other elements used to develop usually. Recently urban doors shall pull of course high activities. Now inherent households see. High |8.46|6.09|1001001|amalgamalg #1|1|dresses|1|Women|159|n stantiought|medium|4spring5621556517142|red|Bunch|Unknown|69|barantiableeseought| +14251|AAAAAAAALKHDAAAA|1997-10-27||Items |3.29|1.08|6002004|importocorp #4|2|diamonds|6|Jewelry|301|oughtbarpri|N/A|7696913137forest0441|grey|Oz|Unknown|5|oughtantiableeseought| +14252|AAAAAAAAMKHDAAAA|1997-10-27|2000-10-26|As well lexical teams identify to a points; large times star|4.08|2.40|8007003|brandnameless #3|7|hockey|8|Sports|2|able|N/A|37723305292206sky795|olive|Gross|Unknown|57|ableantiableeseought| +14253|AAAAAAAAMKHDAAAA|2000-10-27||As well lexical teams identify to a points; large times star|6.85|3.21|8007003|edu packimporto #2|7|sports-apparel|2|Men|62|able|medium|85sky026226177454903|sandy|Oz|Unknown|6|priantiableeseought| +14254|AAAAAAAAOKHDAAAA|1997-10-27|1999-10-27|Easily labour horses work primarily professional factors. Strong clothes believe primarily a little clear resources. Women can make then powerful lad|1.73|0.51|6013007|exportibrand #7|13|loose stones|6|Jewelry|169|n stcallyought|N/A|354717091847sky31527|violet|Gram|Unknown|40|eseantiableeseought| +14255|AAAAAAAAOKHDAAAA|1999-10-28|2001-10-26|Easily labour horses work primarily professional factors. Strong clothes believe primarily a little clear resources. Women can make then powerful lad|1.83|0.51|6013007|exportiexporti #2|13|toddlers|3|Children|233|n stcallyought|small|354717091847sky31527|spring|Lb|Unknown|64|antiantiableeseought| +14256|AAAAAAAAOKHDAAAA|2001-10-27||Easily labour horses work primarily professional factors. Strong clothes believe primarily a little clear resources. Women can make then powerful lad|4.48|2.91|6013007|importoscholar #1|13|country|5|Music|455|n stcallyought|N/A|354717091847sky31527|lemon|Gram|Unknown|33|callyantiableeseought| +14257|AAAAAAAABLHDAAAA|1997-10-27||Long, political terms change there; ugly forms stop apparently. Successful, public things will lose then. Serious banks mov|5.88|1.94|5001002|amalgscholar #2|1|rock|5|Music|63|prically|N/A|2818royal06029944305|peach|Ton|Unknown|28|ationantiableeseought| +14258|AAAAAAAACLHDAAAA|1997-10-27|2000-10-26|Industria|6.09|4.44|5001001|amalgscholar #1|1|rock|5|Music|136|callypriought|N/A|50696421smoke3221317|puff|Dram|Unknown|19|eingantiableeseought| +14259|AAAAAAAACLHDAAAA|2000-10-27||Industria|5.54|4.44|5001001|univunivamalg #4|10|travel|9|Books|136|callypriought|N/A|50696421smoke3221317|tomato|Cup|Unknown|9|n stantiableeseought| +14260|AAAAAAAAELHDAAAA|1997-10-27|1999-10-27|Much leading demonstrations might end once more institutional doubts. Accused authorities should make. Administrative women maintai|3.79|2.57|8006001|corpnameless #1|6|football|8|Sports|846|callyeseeing|N/A|920703682pink5008387|purple|Oz|Unknown|78|barcallyableeseought| +14261|AAAAAAAAELHDAAAA|1999-10-28|2001-10-26|Much leading demonstrations might end once more institutional doubts. Accused authorities should make. Administrative women maintai|72.65|39.95|8006001|corpnameless #2|16|furniture|7|Home|846|callyeseeing|N/A|83761375845yellow520|deep|N/A|Unknown|99|oughtcallyableeseought| +14262|AAAAAAAAELHDAAAA|2001-10-27||Able,|15.06|13.40|2001001|amalgimporto #1|16|accessories|2|Men|846|callyeseeing|large|83761375845yellow520|wheat|Lb|Unknown|24|ablecallyableeseought| +14263|AAAAAAAAHLHDAAAA|1997-10-27||Environmental stations stand there usually regular feet. Women match mainly american, social words; important females used to present more both unable times. Social cells say wel|4.23|1.56|4001002|amalgedu pack #2|1|womens|4|Shoes|158|eingantiought|large|6698777slate65877581|white|Carton|Unknown|1|pricallyableeseought| +14264|AAAAAAAAILHDAAAA|1997-10-27|2000-10-26|Very considerable feet will not live often very round paths. Organic cells think elsewhere medical others; here inc women prepare british, poor rewards. Cul|2.14|1.77|10007002|brandunivamalg #2|7|personal|10|Electronics|756|callyantiation|N/A|lawn5214889915319238|tomato|Case|Unknown|4|esecallyableeseought| +14265|AAAAAAAAILHDAAAA|2000-10-27||Places seem in a details. Certain accounts should not satisfy increases. Years shall hide scots. Ho|2.23|1.85|10007002|importoimporto #2|2|shirts|2|Men|756|callyantiation|petite|008017red28747976902|sienna|Case|Unknown|22|anticallyableeseought| +14266|AAAAAAAAKLHDAAAA|1997-10-27|1999-10-27|Dangerous institutions can come about workers. Sensible years shall not clear during a arts.|5.79|3.24|5001001|amalgscholar #1|1|rock|5|Music|613|prioughtcally|N/A|60puff25953849594845|gainsboro|N/A|Unknown|2|callycallyableeseought| +14267|AAAAAAAAKLHDAAAA|1999-10-28|2001-10-26|Level, national parts cannot install yet bands. Other, good chains signal then on a facilities; right children might get more after a comparisons; there poor parts recruit also; legal, powerful |0.96|3.24|10011005|amalgamalgamalg #5|11|disk drives|10|Electronics|313|prioughtcally|N/A|60puff25953849594845|brown|Tsp|Unknown|20|ationcallyableeseought| +14268|AAAAAAAAKLHDAAAA|2001-10-27||Level, national parts cannot install yet bands. Other, good chains signal then on a facilities; right children might get more after a comparisons; there poor parts recruit also; legal, powerful |8.40|5.71|7004009|edu packbrand #9|4|curtains/drapes|7|Home|427|prioughtcally|N/A|60puff25953849594845|thistle|Tbl|Unknown|14|eingcallyableeseought| +14269|AAAAAAAANLHDAAAA|1997-10-27||Recent communities should not resist political, late relatives. Below essential plans should |0.76|0.47|8006010|corpnameless #10|6|football|8|Sports|88|eingeing|N/A|3302137122steel64489|peach|Bundle|Unknown|50|n stcallyableeseought| +14270|AAAAAAAAOLHDAAAA|1997-10-27|2000-10-26|All dead months consent recently open schemes. Ph|3.96|1.26|8002001|importonameless #1|2|baseball|8|Sports|442|ableeseese|N/A|3562frosted238597200|khaki|Gross|Unknown|22|barationableeseought| +14271|AAAAAAAAOLHDAAAA|2000-10-27||References can place often pale hands; independent, lucky shops look. Great races might believe. Soon different movements counter only permanent, positive insects.|2.06|1.26|8002001|importoexporti #2|2|infants|3|Children|442|ableeseese|petite|78178505447navy16469|thistle|Each|Unknown|55|oughtationableeseought| +14272|AAAAAAAAAMHDAAAA|1997-10-27|1999-10-27|Consistent parents get enough minute boys. Childr|1.69|1.08|5004001|edu packscholar #1|4|classical|5|Music|340|baresepri|N/A|papaya70662494741490|gainsboro|Pound|Unknown|64|ableationableeseought| +14273|AAAAAAAAAMHDAAAA|1999-10-28|2001-10-26|Consistent parents get enough minute boys. Childr|5.76|3.11|5004001|exportiedu pack #2|4|kids|4|Shoes|340|baresepri|medium|papaya70662494741490|thistle|Pallet|Unknown|10|priationableeseought| +14274|AAAAAAAAAMHDAAAA|2001-10-27||Consistent parents get enough minute boys. Childr|1.44|0.60|5004001|exporticorp #3|3|gold|6|Jewelry|340|baresepri|N/A|02rose62403977228327|smoke|Ounce|Unknown|26|eseationableeseought| +14275|AAAAAAAADMHDAAAA|1997-10-27||Already christian regulations tell skills. Available, blank societies should overcome dir|3.41|3.03|3003002|exportiexporti #2|3|toddlers|3|Children|145|antieseought|petite|779644plum5842636205|sky|Pallet|Unknown|44|antiationableeseought| +14276|AAAAAAAAEMHDAAAA|1997-10-27|2000-10-26|Following, new pages should not bring even hungry men. Following, excellent fields used to speak political, black notes; always single hours sit howev|0.40|0.16|10016014|corpamalgamalg #14|16|wireless|10|Electronics|216|callyoughtable|N/A|3985732186slate52052|dim|Case|Unknown|45|callyationableeseought| +14277|AAAAAAAAEMHDAAAA|2000-10-27||Again economic options c|4.14|0.16|4003002|exportiedu pack #2|3|kids|4|Shoes|216|callyoughtable|extra large|2150turquoise4435242|goldenrod|Cup|Unknown|19|ationationableeseought| +14278|AAAAAAAAGMHDAAAA|1997-10-27|1999-10-27|Worse different countries trim over rural, senior mechanisms. Sweet days offer with a games. Quite full vehicles guarantee indeed simply steep values. Exh|4.56|2.37|6008007|namelesscorp #7|8|mens watch|6|Jewelry|334|esepripri|N/A|404907361hot39332223|saddle|Ounce|Unknown|1|eingationableeseought| +14279|AAAAAAAAGMHDAAAA|1999-10-28|2001-10-26|Recent complaints must want economic, open systems. Public, independent players happen for exam|6.84|4.24|8013010|exportimaxi #10|8|sailing|8|Sports|334|esepripri|N/A|404907361hot39332223|spring|Pound|Unknown|26|n stationableeseought| +14280|AAAAAAAAGMHDAAAA|2001-10-27||Extra, good systems start. Especially other stages accept cautious agreements.|0.87|0.75|8013010|brandnameless #1|7|hockey|8|Sports|334|esepripri|N/A|74855200845orange788|thistle|Gross|Unknown|24|bareingableeseought| +14281|AAAAAAAAJMHDAAAA|1997-10-27||Simple, soviet hands get nearly concerns. Outdoor, moral clothes used to rise always|9.00|7.74|3003002|exportiexporti #2|3|toddlers|3|Children|136|callypriought|extra large|758299rosy5366771729|spring|Box|Unknown|32|oughteingableeseought| +14282|AAAAAAAAKMHDAAAA|1997-10-27|2000-10-26|Available, consistent women hold still complex diseases. Married, good children would work there movements|2.01|1.16|4003001|exportiedu pack #1|3|kids|4|Shoes|596|callyn stanti|medium|577368turquoise84893|navy|Dozen|Unknown|18|ableeingableeseought| +14283|AAAAAAAAKMHDAAAA|2000-10-27||Men tell less true countries. Widespread rights would not justify carefully. Vigorously good references |6.61|3.63|2001002|amalgimporto #2|1|accessories|2|Men|449|n steseese|petite|5403sandy28609544816|red|Cup|Unknown|11|prieingableeseought| +14284|AAAAAAAAMMHDAAAA|1997-10-27|1999-10-27|Often old chemicals used to dismiss especially financia|2.06|1.68|5003001|exportischolar #1|3|pop|5|Music|283|prieingable|N/A|478067415plum1591239|rose|Dram|Unknown|91|eseeingableeseought| +14285|AAAAAAAAMMHDAAAA|1999-10-28|2001-10-26|Often old chemicals used to dismiss especially financia|31.56|20.51|2003002|exportiimporto #2|3|pants|2|Men|602|ablebarcally|medium|9197misty04586188309|honeydew|Ton|Unknown|42|antieingableeseought| +14286|AAAAAAAAMMHDAAAA|2001-10-27||Often old chemicals used to dismiss especially financia|4.57|3.97|2003002|importoedu pack #1|2|mens|4|Shoes|24|eseable|small|895331misty954776898|frosted|Box|Unknown|18|callyeingableeseought| +14287|AAAAAAAAPMHDAAAA|1997-10-27||So commercial comparisons might not show together able whole methods. Heavy concepts shall not remain only difficult provisions|3.20|1.37|5003002|exportischolar #2|3|pop|5|Music|128|eingableought|N/A|5566373408puff360782|khaki|N/A|Unknown|56|ationeingableeseought| +14288|AAAAAAAAANHDAAAA|1997-10-27|2000-10-26|Apparent, young features must not save further good knees. Gaps go at all isolated options. Large-scale, english findings send immediately s|87.72|71.93|3001001|amalgexporti #1|1|newborn|3|Children|192|ablen stought|medium|7494565646peach57602|saddle|Tbl|Unknown|31|eingeingableeseought| +14289|AAAAAAAAANHDAAAA|2000-10-27||Apparent, young features must not save further good knees. Gaps go at all isolated options. Large-scale, english findings send immediately s|4.08|1.38|3001001|importoamalg #2|1|fragrances|1|Women|192|ablen stought|large|27409696grey44081916|purple|Box|Unknown|26|n steingableeseought| +14290|AAAAAAAACNHDAAAA|1997-10-27|1999-10-27|Other, effective things will not work even broken reductions. Much empirical hands will e|6.98|5.44|3003001|exportiexporti #1|3|toddlers|3|Children|86|callyeing|large|892753pink1221731893|goldenrod|Dozen|Unknown|77|barn stableeseought| +14291|AAAAAAAACNHDAAAA|1999-10-28|2001-10-26|More upper patterns may exploit of course in a groups. Authorities shall include also workers. Very simple unions give special fri|1.22|0.71|3003001|amalgunivamalg #12|11|cooking|9|Books|508|callyeing|N/A|892753pink1221731893|light|Case|Unknown|63|oughtn stableeseought| +14292|AAAAAAAACNHDAAAA|2001-10-27||Times recover easily. Young changes will find less small members. Authorities find perhaps teeth; prisoners used to dress top, eager feet. Slow years|62.59|0.71|10016013|corpamalgamalg #13|16|wireless|10|Electronics|163|pricallyought|N/A|892753pink1221731893|orchid|Tbl|Unknown|1|ablen stableeseought| +14293|AAAAAAAAFNHDAAAA|1997-10-27||Similar pieces add all truly easy dangers. Opening, main regulations cannot happen saving no versions. Previous lights shall not skip too. As foreign periods can|9.24|5.54|8015004|scholarmaxi #4|15|fishing|8|Sports|283|prieingable|N/A|77049lavender2180844|sienna|Tbl|Unknown|57|prin stableeseought| +14294|AAAAAAAAGNHDAAAA|1997-10-27|2000-10-26|Economic standards shall bring even strong measures. More main improvements want |4.72|3.58|8014005|edu packmaxi #5|14|tennis|8|Sports|976|callyationn st|N/A|77096smoke0741917307|seashell|Case|Unknown|90|esen stableeseought| +14295|AAAAAAAAGNHDAAAA|2000-10-27||Old arrangements should not keep truly familiar workers. Else great ideas should look new questions. Different vehicles will say workers.|96.40|32.77|6009008|maxicorp #8|9|womens watch|6|Jewelry|683|prieingcally|N/A|77096smoke0741917307|violet|Box|Unknown|5|antin stableeseought| +14296|AAAAAAAAINHDAAAA|1997-10-27|1999-10-27|Perhaps dark personnel want with a pupils. Critical friends could not speak. Already new differences might seem thoughts. Rules would not turn immediately sharp results. Easily le|8.91|4.81|10005017|scholarunivamalg #17|5|karoke|10|Electronics|54|eseanti|N/A|45822turquoise792721|royal|Tsp|Unknown|46|callyn stableeseought| +14297|AAAAAAAAINHDAAAA|1999-10-28|2001-10-26|Good, indian holidays bet both never social areas; financially worthy hands weaken so in a claims. Central, major things split troops. Doctors drive isolated girls; too political|6.00|5.28|5002002|importoscholar #2|2|country|5|Music|54|eseanti|N/A|707769sienna28606697|lavender|Pound|Unknown|13|ationn stableeseought| +14298|AAAAAAAAINHDAAAA|2001-10-27||Good, indian holidays bet both never social areas; financially worthy hands weaken so in a claims. Central, major things split troops. Doctors drive isolated girls; too political|4.01|5.28|7011009|amalgnameless #9|2|accent|7|Home|331|eseanti|N/A|707769sienna28606697|plum|Oz|Unknown|36|eingn stableeseought| +14299|AAAAAAAALNHDAAAA|1997-10-27||Friendly judges act between a parties. Asian, bloody hotels isolat|0.39|0.30|9003008|exportimaxi #8|3|computers|9|Books|114|eseoughtought|N/A|8slate73436718997841|slate|Oz|Unknown|65|n stn stableeseought| +14300|AAAAAAAAMNHDAAAA|1997-10-27|2000-10-26|Necessary needs avoid steps. As negative developments anticipate tomorrow; new changes act always necessary reductions; even political variables can wait now public police; |1.83|0.95|3002001|importoexporti #1|2|infants|3|Children|388|eingeingpri|N/A|24172176068rosy48558|ivory|Bundle|Unknown|30|barbarprieseought| +14301|AAAAAAAAMNHDAAAA|2000-10-27||Necessary needs avoid steps. As negative developments anticipate tomorrow; new changes act always necessary reductions; even political variables can wait now public police; |0.11|0.04|3002001|edu packedu pack #2|4|athletic|4|Shoes|699|eingeingpri|petite|24172176068rosy48558|tomato|Tbl|Unknown|35|oughtbarprieseought| +14302|AAAAAAAAONHDAAAA|1997-10-27|1999-10-27|Sessions write however; tests ought to make eithe|6.24|3.30|8008003|namelessnameless #3|8|outdoor|8|Sports|435|antipriese|N/A|23pink31801364553114|sienna|Cup|Unknown|56|ablebarprieseought| +14303|AAAAAAAAONHDAAAA|1999-10-28|2001-10-26|Sessions write however; tests ought to make eithe|1.91|3.30|8008003|importoexporti #2|8|infants|3|Children|435|antipriese|extra large|23pink31801364553114|sky|Case|Unknown|39|pribarprieseought| +14304|AAAAAAAAONHDAAAA|2001-10-27||Sessions write however; tests ought to make eithe|8.43|3.30|8008003|amalgmaxi #1|8|arts|9|Books|135|antipriese|N/A|23pink31801364553114|seashell|Ton|Unknown|24|esebarprieseought| +14305|AAAAAAAABOHDAAAA|1997-10-27||Yet early inches used to inquire very variable, friendly repor|8.38|6.70|7012006|importonameless #6|12|paint|7|Home|23|priable|N/A|09puff78428812932175|tomato|Cup|Unknown|33|antibarprieseought| +14306|AAAAAAAACOHDAAAA|1997-10-27|2000-10-26|Left, important sports shall get on an specialists. Overall, e|3.56|1.74|9003003|exportimaxi #3|3|computers|9|Books|103|pribarought|N/A|22420mint63958554461|smoke|Carton|Unknown|28|callybarprieseought| +14307|AAAAAAAACOHDAAAA|2000-10-27||International, depend|4.23|1.74|9003003|maxiunivamalg #11|3|televisions|10|Electronics|103|pribarought|N/A|goldenrod26158682722|tomato|Carton|Unknown|12|ationbarprieseought| +14308|AAAAAAAAEOHDAAAA|1997-10-27|1999-10-27|Central, clear services ought to know despite a minutes. Acids may wonder british, short interests. Entirely relevant minutes reflect wrong workers. Respective, existing es|49.63|39.70|3003001|exportiexporti #1|3|toddlers|3|Children|120|barableought|medium|163582burlywood42869|rosy|Dram|Unknown|11|eingbarprieseought| +14309|AAAAAAAAEOHDAAAA|1999-10-28|2001-10-26|Directions ought to achieve now remarkable ways. Later formal members go more intense elections. Simple components pay very powerful coins. Just quick months abandon onl|0.80|0.38|3003001|brandcorp #6|7|pendants|6|Jewelry|120|barableought|N/A|163582burlywood42869|yellow|Ounce|Unknown|8|n stbarprieseought| +14310|AAAAAAAAEOHDAAAA|2001-10-27||Vast documents cover with a colours. Still delicious sheets|4.66|2.37|3003001|amalgexporti #1|1|newborn|3|Children|120|barableought|large|163582burlywood42869|navajo|Lb|Unknown|32|baroughtprieseought| +14311|AAAAAAAAHOHDAAAA|1997-10-27||Educational reasons know also through an economies. Countries hope constitutional, rough ministers. Relations would not say also likely gue|6.23|4.73|9007008|brandmaxi #8|7|reference|9|Books|212|ableoughtable|N/A|9277592604706yellow8|red|Tbl|Unknown|31|oughtoughtprieseought| +14312|AAAAAAAAIOHDAAAA|1997-10-27|2000-10-26|Arrangements might not drive now usually scientific teeth. Agricultural, social resources get about. Aspects put flexible plans. Soviet pairs might not get very christian girls. As wel|4.66|1.86|10003016|exportiunivamalg #16|3|dvd/vcr players|10|Electronics|950|barantin st|N/A|574485tomato98915082|papaya|Bundle|Unknown|1|ableoughtprieseought| +14313|AAAAAAAAIOHDAAAA|2000-10-27||Arrangements might not drive now usually scientific teeth. Agricultural, social resources get about. Aspects put flexible plans. Soviet pairs might not get very christian girls. As wel|4.23|1.86|3001002|amalgexporti #2|1|newborn|3|Children|305|barantin st|large|574485tomato98915082|saddle|Pallet|Unknown|8|prioughtprieseought| +14314|AAAAAAAAKOHDAAAA|1997-10-27|1999-10-27|New, free rights improve now words. Walls shall keep from a tr|2.31|2.03|3001001|amalgexporti #1|1|newborn|3|Children|189|n steingought|extra large|violet53556080215406|royal|Ton|Unknown|41|eseoughtprieseought| +14315|AAAAAAAAKOHDAAAA|1999-10-28|2001-10-26|New, free rights improve now words. Walls shall keep from a tr|2.55|1.24|3001001|maximaxi #12|9|science|9|Books|158|eingantiought|N/A|violet53556080215406|smoke|Pallet|Unknown|31|antioughtprieseought| +14316|AAAAAAAAKOHDAAAA|2001-10-27||New, free rights improve now words. Walls shall keep from a tr|7.35|1.24|10006003|corpunivamalg #3|9|musical|10|Electronics|880|bareingeing|N/A|51397046445pale03566|gainsboro|Cup|Unknown|30|callyoughtprieseought| +14317|AAAAAAAANOHDAAAA|1997-10-27||Al|4.59|3.48|8012008|importomaxi #8|12|guns|8|Sports|466|callycallyese|N/A|8521096152thistle423|sandy|Dram|Unknown|32|ationoughtprieseought| +14318|AAAAAAAAOOHDAAAA|1997-10-27|2000-10-26|New years must try cells. Late authorities pa|6.39|4.72|6003007|exporticorp #7|3|gold|6|Jewelry|597|ationn stanti|N/A|8391orchid6585700278|plum|Carton|Unknown|98|eingoughtprieseought| +14319|AAAAAAAAOOHDAAAA|2000-10-27||New years must try cells. Late authorities pa|3.79|1.32|6003007|edu packunivamalg #6|4|audio|10|Electronics|597|ationn stanti|N/A|4sandy21602365482237|grey|Pound|Unknown|4|n stoughtprieseought| +14320|AAAAAAAAAPHDAAAA|1997-10-27|1999-10-27|Principles will rise familiar areas; the|3.60|1.98|10005007|scholarunivamalg #7|5|karoke|10|Electronics|966|callycallyn st|N/A|98burlywood735387183|plum|Dram|Unknown|53|barableprieseought| +14321|AAAAAAAAAPHDAAAA|1999-10-28|2001-10-26|Principles will rise familiar areas; the|2.70|1.98|1003002|exportiamalg #2|5|maternity|1|Women|766|callycallyation|extra large|36325rose10937680937|royal|Bundle|Unknown|9|oughtableprieseought| +14322|AAAAAAAAAPHDAAAA|2001-10-27||Principles will rise familiar areas; the|8.49|4.24|8002003|importonameless #3|5|baseball|8|Sports|766|callycallyation|N/A|51040246rose56226825|salmon|Pallet|Unknown|21|ableableprieseought| +14323|AAAAAAAADPHDAAAA|1997-10-27||Still surprising jobs might not provide |6.65|5.25|3001002|amalgexporti #2|1|newborn|3|Children|209|n stbarable|extra large|557219seashell174042|white|Cup|Unknown|48|priableprieseought| +14324|AAAAAAAAEPHDAAAA|1997-10-27|2000-10-26|More clear charges dry both. More fat days research often strong skills. Now old features admit too good minerals. Abo|1.05|0.60|9006009|corpmaxi #9|6|parenting|9|Books|667|ationcallycally|N/A|948427seashell272850|rose|Box|Unknown|31|eseableprieseought| +14325|AAAAAAAAEPHDAAAA|2000-10-27||Just front seconds ensure to a dreams. Able eyes will not hasten well local trends. Blue techniques agree. International areas will appear very more than cold ties. Genuine, new workers transmi|6.70|3.75|9006009|edu packscholar #2|6|classical|5|Music|667|ationcallycally|N/A|948427seashell272850|wheat|Bunch|Unknown|60|antiableprieseought| +14326|AAAAAAAAGPHDAAAA|1997-10-27|1999-10-27|Brilliant ships see individually also small ministers. Expected, competitive attitudes may send there gross metres; units used |2.00|1.16|8002007|importonameless #7|2|baseball|8|Sports|209|n stbarable|N/A|009841salmon01487170|rosy|Ton|Unknown|21|callyableprieseought| +14327|AAAAAAAAGPHDAAAA|1999-10-28|2001-10-26|Nuclear, long women improve better years; complex characters should not take away middle, clear players. Arms delete. Doors may not remember expensive, old develo|4.61|2.35|8002007|importoimporto #2|2|shirts|2|Men|399|n stn stpri|large|009841salmon01487170|yellow|Gross|Unknown|48|ationableprieseought| +14328|AAAAAAAAGPHDAAAA|2001-10-27||Nuclear, long women improve better years; complex characters should not take away middle, clear players. Arms delete. Doors may not remember expensive, old develo|4.40|2.20|8002007|exportimaxi #1|3|computers|9|Books|218|n stn stpri|N/A|594875843green877382|maroon|Unknown|Unknown|13|eingableprieseought| +14329|AAAAAAAAJPHDAAAA|1997-10-27||Boys obtain still northwards previo|2.52|2.04|10001016|amalgunivamalg #16|1|cameras|10|Electronics|87|ationeing|N/A|654550white349655684|saddle|Ton|Unknown|65|n stableprieseought| +14330|AAAAAAAAKPHDAAAA|1997-10-27|2000-10-26|Mainly future offices help services. Curiously wealthy patients prevent certainly. Consequences appear local gentlemen. Degrees bring t|1.97|1.71|5001001|amalgscholar #1|1|rock|5|Music|472|ableationese|N/A|tomato94025757836252|tan|Tbl|Unknown|3|barpriprieseought| +14331|AAAAAAAAKPHDAAAA|2000-10-27||Mainly future offices help services. Curiously wealthy patients prevent certainly. Consequences appear local gentlemen. Degrees bring t|96.69|80.25|8013002|exportimaxi #2|13|sailing|8|Sports|25|ableationese|N/A|75750520misty6399992|tomato|Bundle|Unknown|25|oughtpriprieseought| +14332|AAAAAAAAMPHDAAAA|1997-10-27|1999-10-27|Directly retail terms ought to afford sooner at a thanks. Islamic, usual examples re-open. Methods would continue; difficult, curious arts claim proposals. Thousands used to bother to the powers; deaf|6.95|4.30|8005003|scholarnameless #3|5|fitness|8|Sports|304|esebarpri|N/A|6056090puff323220078|tan|Gross|Unknown|98|ablepriprieseought| +14333|AAAAAAAAMPHDAAAA|1999-10-28|2001-10-26|Directly retail terms ought to afford sooner at a thanks. Islamic, usual examples re-open. Methods would continue; difficult, curious arts claim proposals. Thousands used to bother to the powers; deaf|8.10|4.30|8005003|amalgexporti #2|5|newborn|3|Children|304|esebarpri|small|27511164orange226963|steel|Dram|Unknown|22|pripriprieseought| +14334|AAAAAAAAMPHDAAAA|2001-10-27||Left faces shall take sufficiently really unaware deaths. Systems will not minimise abroad about the friends. Nevertheless other provinces find neve|1.36|1.16|2003001|exportiimporto #1|5|pants|2|Men|274|eseationable|N/A|232157877447saddle46|peru|Pound|Unknown|33|esepriprieseought| +14335|AAAAAAAAPPHDAAAA|1997-10-27||Correctly various stations might mediate broad|85.89|52.39|3002002|importoexporti #2|2|infants|3|Children|619|n stoughtcally|extra large|8556809snow688880403|royal|Box|Unknown|20|antipriprieseought| +14336|AAAAAAAAAAIDAAAA|1997-10-27|2000-10-26|General, planned allowances ought to confuse recommendations. Direct, foreign details should not to|3.14|1.09|7008007|namelessbrand #7|8|lighting|7|Home|116|callyoughtought|N/A|51tomato083935087108|purple|Ton|Unknown|34|callypriprieseought| +14337|AAAAAAAAAAIDAAAA|2000-10-27||Emotions set highly. Social, important ingredients must not get very substances. Perhaps skilled teachers dismiss dirty windows; different results read. Familiar t|6.88|1.09|4001002|amalgedu pack #2|1|womens|4|Shoes|196|callyoughtought|extra large|810pale5010522143465|pink|Dram|Unknown|3|ationpriprieseought| +14338|AAAAAAAACAIDAAAA|1997-10-27|1999-10-27|Enough fundamental hours count only things; later technological witnesses might pay more in the fears. Common areas may join often as well lexical provisions.|6.91|3.17|6003003|exporticorp #3|3|gold|6|Jewelry|405|antibarese|N/A|5819202379sienna7866|salmon|Tbl|Unknown|36|eingpriprieseought| +14339|AAAAAAAACAIDAAAA|1999-10-28|2001-10-26|Mixed parts might not make old models. New, other years slip just only ready years. Never econo|2.05|1.08|9016012|corpunivamalg #12|16|mystery|9|Books|395|antin stpri|N/A|5819202379sienna7866|royal|Ton|Unknown|15|n stpriprieseought| +14340|AAAAAAAACAIDAAAA|2001-10-27||Good decisions imagine straight women. As inevitable doors find now different consequences. Ther|8.82|1.08|4002001|importoedu pack #1|2|mens|4|Shoes|395|antin stpri|small|5819202379sienna7866|tan|Tsp|Unknown|60|bareseprieseought| +14341|AAAAAAAAFAIDAAAA|1997-10-27||Ready, sound players may not handle together with a|1.75|1.29|9013002|exportiunivamalg #2|13|self-help|9|Books|161|oughtcallyought|N/A|5753840182lavender45|yellow|Bundle|Unknown|23|oughteseprieseought| +14342|AAAAAAAAGAIDAAAA|1997-10-27|2000-10-26|Again sure relations will not amuse criminal, annual thousands. European, sorry minis|2.45|1.69|4001001|amalgedu pack #1|1|womens|4|Shoes|606|callybarcally|small|752989980758882pale5|rose|Gross|Unknown|20|ableeseprieseought| +14343|AAAAAAAAGAIDAAAA|2000-10-27||Again sure relations will not amuse criminal, annual thousands. European, sorry minis|9.69|1.69|1004002|edu packamalg #2|1|swimwear|1|Women|606|callybarcally|economy|sandy809460436282163|pink|Lb|Unknown|15|prieseprieseought| +14344|AAAAAAAAIAIDAAAA|1997-10-27|1999-10-27|Ancient to|3.83|1.72|2003001|exportiimporto #1|3|pants|2|Men|251|oughtantiable|large|5602puff296680403294|powder|Bundle|Unknown|29|eseeseprieseought| +14345|AAAAAAAAIAIDAAAA|1999-10-28|2001-10-26|Ancient to|1.97|1.72|2004002|edu packimporto #2|3|sports-apparel|2|Men|251|oughtantiable|medium|3366895863429sienna2|violet|Dram|Unknown|36|antieseprieseought| +14346|AAAAAAAAIAIDAAAA|2001-10-27||Ancient to|0.41|0.17|3003001|exportiexporti #1|3|toddlers|3|Children|251|oughtantiable|medium|5651255350208powder3|peru|Ton|Unknown|3|callyeseprieseought| +14347|AAAAAAAALAIDAAAA|1997-10-27||Military, easy objectives must not give large, close seats. Redundant, amazing points camp useful players. Social customs could upset. Social, small di|55.19|29.80|5001002|amalgscholar #2|1|rock|5|Music|138|eingpriought|N/A|48salmon241481588220|red|Unknown|Unknown|2|ationeseprieseought| +14348|AAAAAAAAMAIDAAAA|1997-10-27|2000-10-26|Mental, vast persons must not cancel wrong photographs; close difficulties redeem letters. Symbols may ensure demands|2.94|2.61|9009009|maximaxi #9|9|science|9|Books|292|ablen stable|N/A|696155rose4506616679|grey|Pound|Unknown|89|eingeseprieseought| +14349|AAAAAAAAMAIDAAAA|2000-10-27||Over fine variations eat. Constant, political fruits think boys. Thous|0.53|0.18|10006002|corpunivamalg #2|9|musical|10|Electronics|292|ablen stable|N/A|8273422368162852hot1|steel|Carton|Unknown|88|n steseprieseought| +14350|AAAAAAAAOAIDAAAA|1997-10-27|1999-10-27|Paths put often coming walls; enough contemporary toys ought to |2.85|2.16|10014003|edu packamalgamalg #3|14|automotive|10|Electronics|496|callyn stese|N/A|321122241348rose9636|tan|N/A|Unknown|26|barantiprieseought| +14351|AAAAAAAAOAIDAAAA|1999-10-28|2001-10-26|Paths put often coming walls; enough contemporary toys ought to |3.90|2.16|10014003|namelessmaxi #12|8|romance|9|Books|496|callyn stese|N/A|321122241348rose9636|rose|Unknown|Unknown|53|oughtantiprieseought| +14352|AAAAAAAAOAIDAAAA|2001-10-27||Alive glasses could handle sensibly professional years. Specially recent appearances wish lette|2.86|2.20|4002001|importoedu pack #1|2|mens|4|Shoes|969|callyn stese|medium|464803457554violet55|tan|Bunch|Unknown|17|ableantiprieseought| +14353|AAAAAAAABBIDAAAA|1997-10-27||Negotiations could set especially; long expensive units could not change never local new boundaries. Purposes meet cautiously com|1.24|0.78|10009006|maxiunivamalg #6|9|televisions|10|Electronics|286|callyeingable|N/A|6395dark117091579709|pink|Lb|Unknown|5|priantiprieseought| +14354|AAAAAAAACBIDAAAA|1997-10-27|2000-10-26|Miles should not make ideas. Arms used to make also difficult eyes. Waves support also great, possible documents. However labour loans afford by a lands. Windows may get years. Frequently ordina|5.70|3.64|2001001|amalgimporto #1|1|accessories|2|Men|291|oughtn stable|petite|pale4763002010174961|violet|Pallet|Unknown|46|eseantiprieseought| +14355|AAAAAAAACBIDAAAA|2000-10-27||Planned, effective teachers can tr|3.57|3.21|4003002|exportiedu pack #2|3|kids|4|Shoes|291|oughtn stable|medium|406papaya65091864891|spring|Gram|Unknown|59|antiantiprieseought| +14356|AAAAAAAAEBIDAAAA|1997-10-27|1999-10-27|Children should come then together electronic images. Yesterday public walls would go. Movements must not let eyes. Different, interested plans may not produce then modern, scottish fans. Personnel |3.90|2.37|3001001|amalgexporti #1|1|newborn|3|Children|316|callyoughtpri|petite|10097056214smoke7261|red|Tbl|Unknown|5|callyantiprieseought| +14357|AAAAAAAAEBIDAAAA|1999-10-28|2001-10-26|Real, central systems should know low traditional products. Facilities will go else. Early weeks can say once. Modern interests want white women. Like|0.92|0.34|4002002|importoedu pack #2|2|mens|4|Shoes|73|priation|extra large|6254632152430466sky9|royal|Box|Unknown|32|ationantiprieseought| +14358|AAAAAAAAEBIDAAAA|2001-10-27||Happy attempts allow historically men. Human affairs ought to get any longer underlying places; international, thin seconds might tease that is just ready words. Legal, modern quarters go deman|9.95|7.26|4002002|edu packedu pack #1|2|athletic|4|Shoes|73|priation|petite|9684570pale846685563|peach|Tsp|Unknown|29|eingantiprieseought| +14359|AAAAAAAAHBIDAAAA|1997-10-27||Small, blue minutes save also bodies. Presidential, crucial experiments think almost. Past, able years cannot settle only financial cases. Senior, |3.24|2.00|3004002|edu packexporti #2|4|school-uniforms|3|Children|2|able|medium|54522sandy2556280002|sandy|N/A|Unknown|25|n stantiprieseought| +14360|AAAAAAAAIBIDAAAA|1997-10-27|2000-10-26|Attractive, prime standards would colour extra standards. Probably universal sports take within a police. General hands make to a bases; passengers fight others. Bla|5.14|4.36|3003001|exportiexporti #1|3|toddlers|3|Children|498|eingn stese|medium|24236lemon3158516537|wheat|Tsp|Unknown|62|barcallyprieseought| +14361|AAAAAAAAIBIDAAAA|2000-10-27||Local, sure things used to enter results. Groups|0.25|4.36|7015002|scholarnameless #2|15|tables|7|Home|320|eingn stese|N/A|76129indian375896789|white|Gross|Unknown|8|oughtcallyprieseought| +14362|AAAAAAAAKBIDAAAA|1997-10-27|1999-10-27|Available, particular seats should question in response to a police. Discussions may visit stand|2.27|0.97|7009003|maxibrand #3|9|mattresses|7|Home|912|ableoughtn st|N/A|57pink35796988076578|cornflower|Gross|Unknown|14|ablecallyprieseought| +14363|AAAAAAAAKBIDAAAA|1999-10-28|2001-10-26|Available, particular seats should question in response to a police. Discussions may visit stand|90.01|70.20|5002002|importoscholar #2|9|country|5|Music|912|ableoughtn st|N/A|4011183green13716153|deep|N/A|Unknown|19|pricallyprieseought| +14364|AAAAAAAAKBIDAAAA|2001-10-27||Swiss authorities dismiss ideally communities. Wrong, lia|6.54|70.20|2003001|exportiimporto #1|9|pants|2|Men|912|ableoughtn st|economy|4011183green13716153|peach|Unknown|Unknown|33|esecallyprieseought| +14365|AAAAAAAANBIDAAAA|1997-10-27||Dry, dear windows get under a countries. Rights would not opt all civil, economic companies. Apparently interesting crowds break ot|0.69|0.32|10009001|maxiunivamalg #1|9|televisions|10|Electronics|209|n stbarable|N/A|7cornflower836784948|slate|Gross|Unknown|15|anticallyprieseought| +14366|AAAAAAAAOBIDAAAA|1997-10-27|2000-10-26|Great, mixed bits utilise however quickly comprehensive sales. Near ne|1.23|0.39|9003003|exportimaxi #3|3|computers|9|Books|318|eingoughtpri|N/A|12803burlywood535488|salmon|Dozen|Unknown|32|callycallyprieseought| +14367|AAAAAAAAOBIDAAAA|2000-10-27||Procedures will make outside in a changes. Never voluntary processes will roam aside sanctions. Bones come often in a costs. Mad, financial cou|2.53|1.46|10004003|edu packunivamalg #3|3|audio|10|Electronics|162|ablecallyought|N/A|12803burlywood535488|plum|Each|Unknown|34|ationcallyprieseought| +14368|AAAAAAAAACIDAAAA|1997-10-27|1999-10-27|About military signals could consider swiftly also international records. Good, following owners get fairly obvious sales; players might understand with |1.66|1.49|4003001|exportiedu pack #1|3|kids|4|Shoes|206|callybarable|medium|76554steel7296259749|wheat|Dozen|Unknown|26|eingcallyprieseought| +14369|AAAAAAAAACIDAAAA|1999-10-28|2001-10-26|About military signals could consider swiftly also international records. Good, following owners get fairly obvious sales; players might understand with |4.81|1.49|4003001|scholarbrand #2|3|custom|6|Jewelry|206|callybarable|N/A|76554steel7296259749|rose|Dozen|Unknown|6|n stcallyprieseought| +14370|AAAAAAAAACIDAAAA|2001-10-27||About military signals could consider swiftly also international records. Good, following owners get fairly obvious sales; players might understand with |4.31|1.49|4003001|exportimaxi #7|3|computers|9|Books|286|callyeingable|N/A|76554steel7296259749|honeydew|Carton|Unknown|86|barationprieseought| +14371|AAAAAAAADCIDAAAA|1997-10-27||National women find major, able shows. Direct visitors must not want indian clothes. Years must run slowly in the costs. Months mak|8.93|2.85|9015008|scholarunivamalg #8|15|fiction|9|Books|281|oughteingable|N/A|8704097809788khaki11|saddle|Pound|Unknown|94|oughtationprieseought| +14372|AAAAAAAAECIDAAAA|1997-10-27|2000-10-26|Unemployed attacks may not take both later social circumstances. Wide, other owners must not explore teach|3.98|2.30|9005009|scholarmaxi #9|5|history|9|Books|211|oughtoughtable|N/A|2white66821927185094|sky|Unknown|Unknown|21|ableationprieseought| +14373|AAAAAAAAECIDAAAA|2000-10-27||Unemployed attacks may not take both later social circumstances. Wide, other owners must not explore teach|5.01|2.75|1001002|amalgamalg #2|5|dresses|1|Women|352|oughtoughtable|medium|2white66821927185094|violet|Oz|Unknown|1|priationprieseought| +14374|AAAAAAAAGCIDAAAA|1997-10-27|1999-10-27|Significant, smooth computers|1.75|1.31|10004010|edu packunivamalg #10|4|audio|10|Electronics|309|n stbarpri|N/A|ghost769518404248829|khaki|Box|Unknown|17|eseationprieseought| +14375|AAAAAAAAGCIDAAAA|1999-10-28|2001-10-26|Significant, smooth computers|1.25|1.31|7016006|corpnameless #6|4|furniture|7|Home|999|n stbarpri|N/A|46purple816751928347|yellow|Case|Unknown|1|antiationprieseought| +14376|AAAAAAAAGCIDAAAA|2001-10-27||Significant, smooth computers|9.38|1.31|7016006|exportischolar #1|3|pop|5|Music|489|n stbarpri|N/A|46purple816751928347|salmon|Dozen|Unknown|61|callyationprieseought| +14377|AAAAAAAAJCIDAAAA|1997-10-27||Structures may|4.92|3.00|7015008|scholarnameless #8|15|tables|7|Home|658|eingantically|N/A|674805317pink4884656|olive|Bundle|Unknown|29|ationationprieseought| +14378|AAAAAAAAKCIDAAAA|1997-10-27|2000-10-26|Total efforts communicate horribly primary circumstances. Times should meet severely to the resources. Full, economic residents must manipu|2.94|1.35|7016009|corpnameless #9|16|furniture|7|Home|76|callyation|N/A|73357506293wheat4931|saddle|Each|Unknown|54|eingationprieseought| +14379|AAAAAAAAKCIDAAAA|2000-10-27||Total efforts communicate horribly primary circumstances. Times should meet severely to the resources. Full, economic residents must manipu|4.58|1.35|7016009|corpcorp #4|16|rings|6|Jewelry|300|callyation|N/A|375942522197saddle01|sandy|Bunch|Unknown|66|n stationprieseought| +14380|AAAAAAAAMCIDAAAA|1997-10-27|1999-10-27|All growing hours assist quickly interested hands; thanks like. Men used to perform times. Official, constant jeans expose most young, dead thanks. Most|94.29|32.05|10008016|namelessunivamalg #16|8|scanners|10|Electronics|255|antiantiable|N/A|2595884228metallic20|lawn|Each|Unknown|14|bareingprieseought| +14381|AAAAAAAAMCIDAAAA|1999-10-28|2001-10-26|Of course new films must affect on the bodies. Initially essential|4.69|2.48|10008016|amalgamalgamalg #17|11|disk drives|10|Electronics|255|antiantiable|N/A|2595884228metallic20|red|Pound|Unknown|49|oughteingprieseought| +14382|AAAAAAAAMCIDAAAA|2001-10-27||Other police should secure large,|70.17|2.48|1001001|amalgamalg #1|11|dresses|1|Women|255|antiantiable|large|pink1264917712429140|yellow|Pallet|Unknown|50|ableeingprieseought| +14383|AAAAAAAAPCIDAAAA|1997-10-27||Goods mention from a hours; red, sweet procedures say|1.70|1.30|9014008|edu packunivamalg #8|14|sports|9|Books|313|prioughtpri|N/A|6022459royal47754887|rosy|Pound|Unknown|3|prieingprieseought| +14384|AAAAAAAAADIDAAAA|1997-10-27|2000-10-26|Literary movies will include actually at a models. Else other areas would develop then on a consequences; responsibilities must exercise most average, fin|3.29|1.08|9011009|amalgunivamalg #9|11|cooking|9|Books|922|ableablen st|N/A|17900899navy76374285|steel|Carton|Unknown|37|eseeingprieseought| +14385|AAAAAAAAADIDAAAA|2000-10-27||Literary movies will include actually at a models. Else other areas would develop then on a consequences; responsibilities must exercise most average, fin|62.48|33.73|9011009|importobrand #6|2|bedding|7|Home|922|ableablen st|N/A|95royal5586850091892|gainsboro|Cup|Unknown|57|antieingprieseought| +14386|AAAAAAAACDIDAAAA|1997-10-27|1999-10-27|Women could tell still ever mathematical standards|1.26|1.09|8016007|corpmaxi #7|16|golf|8|Sports|704|esebaration|N/A|679318883floral51197|white|Oz|Unknown|61|callyeingprieseought| +14387|AAAAAAAACDIDAAAA|1999-10-28|2001-10-26|Women could tell still ever mathematical standards|8.95|3.93|3002002|importoexporti #2|16|infants|3|Children|704|esebaration|small|679318883floral51197|turquoise|Ton|Unknown|4|ationeingprieseought| +14388|AAAAAAAACDIDAAAA|2001-10-27||Women could tell still ever mathematical standards|9.04|3.93|3002002|amalgamalg #1|16|dresses|1|Women|338|eingpripri|extra large|578487557972sienna60|rose|Case|Unknown|25|eingeingprieseought| +14389|AAAAAAAAFDIDAAAA|1997-10-27||Nearly practical structures close considerable, perfect|5.60|4.03|8001010|amalgnameless #10|1|athletic shoes|8|Sports|518|eingoughtanti|N/A|8179honeydew46887642|powder|N/A|Unknown|74|n steingprieseought| +14390|AAAAAAAAGDIDAAAA|1997-10-27|2000-10-26|Important, other terms say now in the police. Really single assumptions investigate automatically on a lights; r|2.61|0.96|1003001|exportiamalg #1|3|maternity|1|Women|451|oughtantiese|medium|870071white631827390|royal|Oz|Unknown|60|barn stprieseought| +14391|AAAAAAAAGDIDAAAA|2000-10-27||Categories perform aside more big letters. Male roads may not state dangerous |4.30|2.19|7011002|amalgnameless #2|11|accent|7|Home|451|oughtantiese|N/A|870071white631827390|misty|Ounce|Unknown|5|oughtn stprieseought| +14392|AAAAAAAAIDIDAAAA|1997-10-27|1999-10-27|Colourful, crude customers must not give sometimes fine penalties. Firms might learn in a principles. Easily minute things satisfy|15.22|10.95|1001001|amalgamalg #1|1|dresses|1|Women|829|n stableeing|large|460317815peach290050|goldenrod|Dozen|Unknown|53|ablen stprieseought| +14393|AAAAAAAAIDIDAAAA|1999-10-28|2001-10-26|Again weak flames might not find more typical pictures; weekly others must not depend jointly|2.93|2.22|1001001|amalgunivamalg #6|11|cooking|9|Books|829|n stableeing|N/A|756025tomato86359626|sandy|Dram|Unknown|30|prin stprieseought| +14394|AAAAAAAAIDIDAAAA|2001-10-27||Substantial, final drugs would work s|14.86|4.75|1001001|amalgexporti #1|11|newborn|3|Children|300|n stableeing|extra large|756025tomato86359626|purple|Ton|Unknown|18|esen stprieseought| +14395|AAAAAAAALDIDAAAA|1997-10-27||Foreign, international services must answer structural pictures. Big mountains help private, deep partners. Specific|4.73|3.45|4004002|edu packedu pack #2|4|athletic|4|Shoes|182|ableeingought|petite|073002115032yellow05|sienna|N/A|Unknown|24|antin stprieseought| +14396|AAAAAAAAMDIDAAAA|1997-10-27|2000-10-26|Leaves continue |1.98|1.32|3001001|amalgexporti #1|1|newborn|3|Children|282|ableeingable|petite|9299706872slate09316|peach|Box|Unknown|37|callyn stprieseought| +14397|AAAAAAAAMDIDAAAA|2000-10-27||Leaves continue |7.91|4.19|6009006|maxicorp #6|1|womens watch|6|Jewelry|258|ableeingable|N/A|9299706872slate09316|yellow|Case|Unknown|23|ationn stprieseought| +14398|AAAAAAAAODIDAAAA|1997-10-27|1999-10-27|Minutes cease virtually detailed examinations. Ambitious persons look a|2.22|0.66|6009007|maxicorp #7|9|womens watch|6|Jewelry|134|esepriought|N/A|345seashell233165311|saddle|N/A|Unknown|11|eingn stprieseought| +14399|AAAAAAAAODIDAAAA|1999-10-28|2001-10-26|Minutes cease virtually detailed examinations. Ambitious persons look a|36.77|18.75|6009007|exportiamalg #2|9|maternity|1|Women|837|ationprieing|medium|12194268063plum23324|sandy|Tsp|Unknown|23|n stn stprieseought| +14400|AAAAAAAAODIDAAAA|2001-10-27||Minutes cease virtually detailed examinations. Ambitious persons look a|2.75|1.78|3002001|importoexporti #1|2|infants|3|Children|837|ationprieing|large|12194268063plum23324|thistle|Tbl|Unknown|19|barbareseeseought| +14401|AAAAAAAABEIDAAAA|1997-10-27||Just, different women will realise then to a months. Different documents will go far poor areas. |1.57|1.41|7012002|importonameless #2|12|paint|7|Home|595|antin stanti|N/A|68yellow374753307908|seashell|Tbl|Unknown|48|oughtbareseeseought| +14402|AAAAAAAACEIDAAAA|1997-10-27|2000-10-26|Wheels used to complete for example bare seconds. Very natural matters get old lips. Domestic dimensions will not fill below joint, great rules. Only, late crimes support almo|2.15|1.26|4002001|importoedu pack #1|2|mens|4|Shoes|422|ableableese|medium|231298277340197lawn4|wheat|Each|Unknown|53|ablebareseeseought| +14403|AAAAAAAACEIDAAAA|2000-10-27||Wheels used to complete for example bare seconds. Very natural matters get old lips. Domestic dimensions will not fill below joint, great rules. Only, late crimes support almo|1.37|1.26|4002001|exportiedu pack #2|3|kids|4|Shoes|422|ableableese|petite|9309810126spring6221|wheat|Unknown|Unknown|55|pribareseeseought| +14404|AAAAAAAAEEIDAAAA|1997-10-27|1999-10-27|Aims lie no longer very long forms. Perceptions laugh then again intense lines. Working paintings wo|0.91|0.66|4001001|amalgedu pack #1|1|womens|4|Shoes|649|n stesecally|small|9692wheat53310099745|cornflower|Pallet|Unknown|3|esebareseeseought| +14405|AAAAAAAAEEIDAAAA|1999-10-28|2001-10-26|So bad eyes could continue here sides. Major, fair powers used to give in|4.91|0.66|7003006|exportibrand #6|3|kids|7|Home|649|n stesecally|N/A|9692wheat53310099745|royal|Tsp|Unknown|15|antibareseeseought| +14406|AAAAAAAAEEIDAAAA|2001-10-27||So bad eyes could continue here sides. Major, fair powers used to give in|8.76|0.66|7003006|exportiedu pack #1|3|kids|4|Shoes|170|barationought|extra large|9692wheat53310099745|pale|Dozen|Unknown|23|callybareseeseought| +14407|AAAAAAAAHEIDAAAA|1997-10-27||Ideal talks might not think within the strengths; actions can change probably; names provide later in a jews; busy pr|8.79|5.36|7004008|edu packbrand #8|4|curtains/drapes|7|Home|345|antiesepri|N/A|35464405474hot562280|lemon|Pallet|Unknown|72|ationbareseeseought| +14408|AAAAAAAAIEIDAAAA|1997-10-27|2000-10-26|Differe|1.72|0.68|10013010|exportiamalgamalg #10|13|stereo|10|Electronics|325|antiablepri|N/A|spring98006379025877|brown|Each|Unknown|59|eingbareseeseought| +14409|AAAAAAAAIEIDAAAA|2000-10-27||Differe|1.61|0.68|9015010|scholarunivamalg #10|15|fiction|9|Books|83|antiablepri|N/A|03521171958steel1960|misty|Tbl|Unknown|51|n stbareseeseought| +14410|AAAAAAAAKEIDAAAA|1997-10-27|1999-10-27|So much as close reforms would hide at first measures; alone, important contracts lose linguisti|2.37|1.37|8003001|exportinameless #1|3|basketball|8|Sports|971|oughtationn st|N/A|69554790thistle14925|pale|Pallet|Unknown|15|baroughteseeseought| +14411|AAAAAAAAKEIDAAAA|1999-10-28|2001-10-26|So attractive circumstances must ove|80.45|41.83|8003001|edu packscholar #2|4|classical|5|Music|267|oughtationn st|N/A|69554790thistle14925|snow|Bundle|Unknown|24|oughtoughteseeseought| +14412|AAAAAAAAKEIDAAAA|2001-10-27||Gradually bad tories set necessarily national, human guests; colleagues enforce more methods; prime games might play soon on a hours. Somewhere genetic grounds will focus normally in the lev|9.09|41.83|3002001|importoexporti #1|2|infants|3|Children|267|ationcallyable|large|49green2676596336714|white|Gross|Unknown|5|ableoughteseeseought| +14413|AAAAAAAANEIDAAAA|1997-10-27||Relations d|8.44|3.37|7004004|edu packbrand #4|4|curtains/drapes|7|Home|312|ableoughtpri|N/A|7617434243157blue155|medium|Each|Unknown|26|prioughteseeseought| +14414|AAAAAAAAOEIDAAAA|1997-10-27|2000-10-26|Electoral occupations assemble exchanges; als|2.20|0.79|9001003|amalgmaxi #3|1|arts|9|Books|711|oughtoughtation|N/A|305498white509058842|purple|Bunch|Unknown|39|eseoughteseeseought| +14415|AAAAAAAAOEIDAAAA|2000-10-27||However helpful pupils can receive. Recent, successfu|7.31|0.79|9001003|amalgimporto #2|1|accessories|2|Men|663|pricallycally|petite|7papaya7262290423690|slate|Dram|Unknown|25|antioughteseeseought| +14416|AAAAAAAAAFIDAAAA|1997-10-27|1999-10-27|Other, rational years hit suddenly young members. Political goals|89.61|60.93|1001001|amalgamalg #1|1|dresses|1|Women|272|ableationable|N/A|44747seashell5277447|saddle|Cup|Unknown|66|callyoughteseeseought| +14417|AAAAAAAAAFIDAAAA|1999-10-28|2001-10-26|Other, rational years hit suddenly young members. Political goals|0.36|0.24|1001001|importoexporti #2|2|infants|3|Children|272|ableationable|medium|44747seashell5277447|snow|Dram|Unknown|16|ationoughteseeseought| +14418|AAAAAAAAAFIDAAAA|2001-10-27||Other, rational years hit suddenly young members. Political goals|20.37|11.40|1002001|importoamalg #1|2|fragrances|1|Women|280|bareingable|N/A|51slate5875541464427|seashell|Pallet|Unknown|28|eingoughteseeseought| +14419|AAAAAAAADFIDAAAA|1997-10-27||Police develop well. Corporations wish precisely. Th|2.77|1.21|4001002|amalgedu pack #2|1|womens|4|Shoes|149|n steseought|medium|1971165071honeydew09|dim|Unknown|Unknown|8|n stoughteseeseought| +14420|AAAAAAAAEFIDAAAA|1997-10-27|2000-10-26|Minor, welcome doctors project light, human grounds; countries ask long. Simply national systems allocate by the cameras; thus natural neighbours show enou|0.65|0.48|10005005|scholarunivamalg #5|5|karoke|10|Electronics|409|n stbarese|N/A|50637steel9802775058|sky|Dozen|Unknown|21|barableeseeseought| +14421|AAAAAAAAEFIDAAAA|2000-10-27||Minor, welcome doctors project light, human grounds; countries ask long. Simply national systems allocate by the cameras; thus natural neighbours show enou|6.99|2.65|10005005|exportischolar #2|3|pop|5|Music|60|barcally|N/A|50637steel9802775058|seashell|N/A|Unknown|20|oughtableeseeseought| +14422|AAAAAAAAGFIDAAAA|1997-10-27|1999-10-27|Normal times gi|2.88|1.69|8011003|amalgmaxi #3|11|archery|8|Sports|366|callycallypri|N/A|07962048papaya848761|wheat|Oz|Unknown|37|ableableeseeseought| +14423|AAAAAAAAGFIDAAAA|1999-10-28|2001-10-26|Normal times gi|6.08|2.24|8011003|edu packexporti #2|4|school-uniforms|3|Children|32|callycallypri|medium|388450turquoise50055|salmon|Case|Unknown|19|priableeseeseought| +14424|AAAAAAAAGFIDAAAA|2001-10-27||Normal times gi|1.71|0.75|8011003|maximaxi #1|9|science|9|Books|147|ationeseought|N/A|388450turquoise50055|seashell|Gram|Unknown|50|eseableeseeseought| +14425|AAAAAAAAJFIDAAAA|1997-10-27||Things can r|7.52|2.78|8004006|edu packnameless #6|4|camping|8|Sports|184|eseeingought|N/A|139puff7350471132970|white|Gram|Unknown|51|antiableeseeseought| +14426|AAAAAAAAKFIDAAAA|1997-10-27|2000-10-26|Apparently african views speak then; external, necessary tables catch so for a|3.65|2.44|1004001|edu packamalg #1|4|swimwear|1|Women|463|pricallyese|small|9972wheat22097665505|white|Unknown|Unknown|10|callyableeseeseought| +14427|AAAAAAAAKFIDAAAA|2000-10-27||Ties take to a films. Authorities reject sure geographical children. Positions shall not die in a values. Empirical assets know still new, just employees. Minimum, other years eq|1.03|2.44|1004001|amalgscholar #2|4|rock|5|Music|177|pricallyese|N/A|2547099plum533918534|sandy|Each|Unknown|21|ationableeseeseought| +14428|AAAAAAAAMFIDAAAA|1997-10-27|1999-10-27|Correct eyes use often well shared papers. Excellent, living hundreds disagree s|0.51|0.21|4002001|importoedu pack #1|2|mens|4|Shoes|583|prieinganti|petite|1sandy49730045113097|spring|Gram|Unknown|22|eingableeseeseought| +14429|AAAAAAAAMFIDAAAA|1999-10-28|2001-10-26|Correct eyes use often well shared papers. Excellent, living hundreds disagree s|1.84|0.21|5003002|exportischolar #2|3|pop|5|Music|652|ableantically|N/A|sky57651633988921357|orange|Ounce|Unknown|32|n stableeseeseought| +14430|AAAAAAAAMFIDAAAA|2001-10-27||Executive, useful roads will get today. Authorities can create |6.27|3.13|5003002|edu packamalg #1|3|swimwear|1|Women|652|ableantically|petite|sky57651633988921357|salmon|Oz|Unknown|3|barprieseeseought| +14431|AAAAAAAAPFIDAAAA|1997-10-27||Inc, good scores clear only following hours. Involved, pale terms manoeuvre however actu|5.31|3.07|4003002|exportiedu pack #2|3|kids|4|Shoes|149|n steseought|medium|104014711violet70403|royal|Ounce|Unknown|44|oughtprieseeseought| +14432|AAAAAAAAAGIDAAAA|1997-10-27|2000-10-26|Lucky ears push locally figures. New, ethnic pol|1.30|1.09|3004001|edu packexporti #1|4|school-uniforms|3|Children|110|baroughtought|medium|9935539736maroon1364|tan|Ton|Unknown|42|ableprieseeseought| +14433|AAAAAAAAAGIDAAAA|2000-10-27||Early leaders may not call very. Frequently clear savings get for a standards. Organisational, evil rights make up the conclusions. Here english actions must not leave more essential |8.29|6.88|3004001|amalgnameless #4|1|athletic shoes|8|Sports|110|baroughtought|N/A|17purple709452828628|misty|Dozen|Unknown|32|priprieseeseought| +14434|AAAAAAAACGIDAAAA|1997-10-27|1999-10-27|Functional, boring circumstances s|1.67|1.45|4003001|exportiedu pack #1|3|kids|4|Shoes|436|callypriese|extra large|7270888990157powder1|red|N/A|Unknown|20|eseprieseeseought| +14435|AAAAAAAACGIDAAAA|1999-10-28|2001-10-26|Functional, boring circumstances s|3.34|1.45|6012004|importobrand #4|12|costume|6|Jewelry|436|callypriese|N/A|7270888990157powder1|snow|Oz|Unknown|14|antiprieseeseought| +14436|AAAAAAAACGIDAAAA|2001-10-27||Functional, boring circumstances s|1.04|1.45|6008005|namelesscorp #5|12|mens watch|6|Jewelry|436|callypriese|N/A|7270888990157powder1|turquoise|Lb|Unknown|3|callyprieseeseought| +14437|AAAAAAAAFGIDAAAA|1997-10-27||Old relations m|8.97|4.21|6014006|edu packbrand #6|14|estate|6|Jewelry|408|eingbarese|N/A|3638747lime463392015|linen|Dram|Unknown|17|ationprieseeseought| +14438|AAAAAAAAGGIDAAAA|1997-10-27|2000-10-26|Farms may cope long absolute, mysterious serv|1.27|0.53|1001001|amalgamalg #1|1|dresses|1|Women|253|priantiable|petite|44white2546266597029|hot|Tbl|Unknown|96|eingprieseeseought| +14439|AAAAAAAAGGIDAAAA|2000-10-27||Less impossible kids seem then poorly economic systems. Therefore g|4.74|0.53|1001001|scholarunivamalg #7|5|karoke|10|Electronics|253|priantiable|N/A|44white2546266597029|tomato|Unknown|Unknown|63|n stprieseeseought| +14440|AAAAAAAAIGIDAAAA|1997-10-27|1999-10-27|Cheap empty drinks will cause interesting notes; certainly specified areas cannot know central|4.98|1.74|10012008|importoamalgamalg #8|12|monitors|10|Electronics|209|n stbarable|N/A|71092633625081steel7|wheat|Lb|Unknown|61|bareseeseeseought| +14441|AAAAAAAAIGIDAAAA|1999-10-28|2001-10-26|Cheap empty drinks will cause interesting notes; certainly specified areas cannot know central|4.73|1.60|7007002|brandbrand #2|7|decor|7|Home|209|n stbarable|N/A|71092633625081steel7|sandy|Dozen|Unknown|47|oughteseeseeseought| +14442|AAAAAAAAIGIDAAAA|2001-10-27||Courses must train true delegates; local, tory liabilities cannot carry; as local g|9.18|3.76|7006003|corpbrand #3|6|rugs|7|Home|209|n stbarable|N/A|71092633625081steel7|khaki|Dozen|Unknown|38|ableeseeseeseought| +14443|AAAAAAAALGIDAAAA|1997-10-27||Sc|1.08|0.36|8006004|corpnameless #4|6|football|8|Sports|13|priought|N/A|957603yellow87204575|royal|Ounce|Unknown|5|prieseeseeseought| +14444|AAAAAAAAMGIDAAAA|1997-10-27|2000-10-26|New changes impress. Somehow existing plans guarantee at least brothers. Efforts would know here. Unhappy titles must provide various professionals. Outside players |2.97|2.58|1002001|importoamalg #1|2|fragrances|1|Women|55|antianti|extra large|8lime380124369692046|plum|Cup|Unknown|27|eseeseeseeseought| +14445|AAAAAAAAMGIDAAAA|2000-10-27||New changes impress. Somehow existing plans guarantee at least brothers. Efforts would know here. Unhappy titles must provide various professionals. Outside players |0.40|0.26|1002001|importoedu pack #2|2|mens|4|Shoes|55|antianti|petite|8lime380124369692046|wheat|Carton|Unknown|78|antieseeseeseought| +14446|AAAAAAAAOGIDAAAA|1997-10-27|1999-10-27|For example other winners will not keep as different good ideas. Years might not pay right true theories. Again harsh proposals cannot participate later english local standards|1.19|1.03|4004001|edu packedu pack #1|4|athletic|4|Shoes|318|eingoughtpri|large|9392464rose084293338|moccasin|Gross|Unknown|21|callyeseeseeseought| +14447|AAAAAAAAOGIDAAAA|1999-10-28|2001-10-26|For example other winners will not keep as different good ideas. Years might not pay right true theories. Again harsh proposals cannot participate later english local standards|8.14|6.75|4004001|importoamalg #2|2|fragrances|1|Women|317|eingoughtpri|medium|lemon404702490580881|peru|Case|Unknown|37|ationeseeseeseought| +14448|AAAAAAAAOGIDAAAA|2001-10-27||For example other winners will not keep as different good ideas. Years might not pay right true theories. Again harsh proposals cannot participate later english local standards|0.48|0.33|10002016|importounivamalg #16|2|camcorders|10|Electronics|317|eingoughtpri|N/A|53522412050pale85903|sandy|Ton|Unknown|8|eingeseeseeseought| +14449|AAAAAAAABHIDAAAA|1997-10-27||Nearly large-scale score|34.83|12.19|7009010|maxibrand #10|9|mattresses|7|Home|362|ablecallypri|N/A|5955319347light66864|seashell|Cup|Unknown|1|n steseeseeseought| +14450|AAAAAAAACHIDAAAA|1997-10-27|2000-10-26|Annual, other lips distinguish currently for a years. Public members teach now stupid, special years. Th|3.59|1.36|2001001|amalgimporto #1|1|accessories|2|Men|279|n stationable|petite|01894128155718ivory3|green|Oz|Unknown|13|barantieseeseought| +14451|AAAAAAAACHIDAAAA|2000-10-27||Hot views will feel|3.16|1.36|2001001|univmaxi #2|10|pools|8|Sports|279|n stationable|N/A|905powder44548696377|yellow|Ton|Unknown|57|oughtantieseeseought| +14452|AAAAAAAAEHIDAAAA|1997-10-27|1999-10-27|Potential years would lay in order strong jobs. Times cannot allow specif|3.65|1.24|9011005|amalgunivamalg #5|11|cooking|9|Books|579|n stationanti|N/A|268485207247salmon48|slate|Ounce|Unknown|39|ableantieseeseought| +14453|AAAAAAAAEHIDAAAA|1999-10-28|2001-10-26|Seco|0.82|1.24|9011005|amalgimporto #2|11|accessories|2|Men|579|n stationanti|extra large|268485207247salmon48|turquoise|Bundle|Unknown|34|priantieseeseought| +14454|AAAAAAAAEHIDAAAA|2001-10-27||Seco|4.07|1.24|9011005|corpnameless #5|16|furniture|7|Home|58|n stationanti|N/A|97200thistle34176433|rosy|Dram|Unknown|78|eseantieseeseought| +14455|AAAAAAAAHHIDAAAA|1997-10-27||Victorian, far papers could know measures. Only arguments can see bli|3.34|2.73|6005008|scholarcorp #8|5|earings|6|Jewelry|383|prieingpri|N/A|0234458ivory77444196|royal|Tbl|Unknown|61|antiantieseeseought| +14456|AAAAAAAAIHIDAAAA|1997-10-27|2000-10-26|Individu|2.81|0.87|5002001|importoscholar #1|2|country|5|Music|27|ationable|N/A|7761020783rosy463061|metallic|Tbl|Unknown|12|callyantieseeseought| +14457|AAAAAAAAIHIDAAAA|2000-10-27||Afield future benefits will mean much great, full directors. Urgent elections might use that personal services; only conditions stress swiftly by the methods. New, large differences stop. Af|1.84|0.99|5002001|importoimporto #2|2|shirts|2|Men|27|ationable|large|30520thistle80015979|rosy|Pallet|Unknown|13|ationantieseeseought| +14458|AAAAAAAAKHIDAAAA|1997-10-27|1999-10-27|Literary times could make small, professional wives. Elsewhere plain miles consider finally domestic roads. Accounts shall|2.39|0.74|10005009|scholarunivamalg #9|5|karoke|10|Electronics|247|ationeseable|N/A|9730712blush68832905|pale|Dozen|Unknown|12|eingantieseeseought| +14459|AAAAAAAAKHIDAAAA|1999-10-28|2001-10-26|Long, wrong types ought to know. Well-known, full couples see totally. Delighted variables may not pay. Royal, unable allies ought to freeze vast, green branches. Public walls used to allow |0.15|0.74|10005009|importoamalg #2|5|fragrances|1|Women|430|barpriese|petite|9730712blush68832905|grey|Tsp|Unknown|62|n stantieseeseought| +14460|AAAAAAAAKHIDAAAA|2001-10-27||Long, wrong types ought to know. Well-known, full couples see totally. Delighted variables may not pay. Royal, unable allies ought to freeze vast, green branches. Public walls used to allow |4.28|0.74|7007001|brandbrand #1|7|decor|7|Home|430|barpriese|N/A|9730712blush68832905|saddle|Dozen|Unknown|51|barcallyeseeseought| +14461|AAAAAAAANHIDAAAA|1997-10-27||Away clinical communications give angry areas. Difficult commitme|4.01|2.28|2002002|importoimporto #2|2|shirts|2|Men|139|n stpriought|large|56934seashell7064714|sky|Bundle|Unknown|59|oughtcallyeseeseought| +14462|AAAAAAAAOHIDAAAA|1997-10-27|2000-10-26|Very sensitive sets used to continue judicial groups. Large sh|7.79|2.64|1002001|importoamalg #1|2|fragrances|1|Women|561|oughtcallyanti|large|06560metallic2818998|medium|Ton|Unknown|48|ablecallyeseeseought| +14463|AAAAAAAAOHIDAAAA|2000-10-27||Later relevant men stand deep songs. Presumably rare parts would not think tomorrow; red letters compel there by the|13.67|2.64|1002001|importoamalg #2|2|fragrances|1|Women|561|oughtcallyanti|small|06560metallic2818998|navy|Oz|Unknown|5|pricallyeseeseought| +14464|AAAAAAAAAIIDAAAA|1997-10-27|1999-10-27|Again heavy organisms may resu|43.19|32.39|8006005|corpnameless #5|6|football|8|Sports|809|n stbareing|N/A|1559gainsboro8077761|hot|Pound|Unknown|18|esecallyeseeseought| +14465|AAAAAAAAAIIDAAAA|1999-10-28|2001-10-26|Again heavy organisms may resu|6.11|4.94|6014002|edu packbrand #2|6|estate|6|Jewelry|809|n stbareing|N/A|52789703salmon079699|peach|Dram|Unknown|5|anticallyeseeseought| +14466|AAAAAAAAAIIDAAAA|2001-10-27||Very political teeth escape half like a friends; leaves should say nevertheless; real seco|8.83|5.82|6014002|edu packscholar #1|6|classical|5|Music|809|n stbareing|N/A|spring05238150216309|pink|Pallet|Unknown|9|callycallyeseeseought| +14467|AAAAAAAADIIDAAAA|1997-10-27||Elements read separately only other servants. Real relationships might want then well great risks. Usual players used to print members. Strange crimes may refuse. Origins used to prepare as white c|2.76|1.90|5001002|amalgscholar #2|1|rock|5|Music|966|callycallyn st|N/A|49149920885honeydew0|red|N/A|Unknown|41|ationcallyeseeseought| +14468|AAAAAAAAEIIDAAAA|1997-10-27|2000-10-26|Mainly original jobs will care wines. Able, commercial costs will understand of cou|4.73|3.97|1004001|edu packamalg #1|4|swimwear|1|Women|883|prieingeing|small|85946186magenta11365|puff|N/A|Unknown|10|eingcallyeseeseought| +14469|AAAAAAAAEIIDAAAA|2000-10-27||Evident, likely relations look regardless at a workshops. Alternative conditions might r|3.15|2.52|1002002|importoamalg #2|4|fragrances|1|Women|58|einganti|small|6771116401grey130620|tomato|Ton|Unknown|26|n stcallyeseeseought| +14470|AAAAAAAAGIIDAAAA|1997-10-27|1999-10-27|Factors want. Events declare here excellent|2.30|1.95|9005011|scholarmaxi #11|5|history|9|Books|529|n stableanti|N/A|215862yellow63939639|turquoise|Ounce|Unknown|31|barationeseeseought| +14471|AAAAAAAAGIIDAAAA|1999-10-28|2001-10-26|Well full thousands may not worry future, theoretical collect|56.61|34.53|6002008|importocorp #8|5|diamonds|6|Jewelry|592|n stableanti|N/A|215862yellow63939639|sandy|Carton|Unknown|17|oughtationeseeseought| +14472|AAAAAAAAGIIDAAAA|2001-10-27||Well full thousands may not worry future, theoretical collect|39.98|34.53|1004001|edu packamalg #1|5|swimwear|1|Women|592|n stableanti|extra large|215862yellow63939639|plum|Carton|Unknown|5|ableationeseeseought| +14473|AAAAAAAAJIIDAAAA|1997-10-27||Sometimes black devices shall not start now impossible students. Awful, video-taped plans would take then increasingly hos|50.51|41.41|10004007|edu packunivamalg #7|4|audio|10|Electronics|144|eseeseought|N/A|01855015498622dim987|tan|Unknown|Unknown|24|priationeseeseought| +14474|AAAAAAAAKIIDAAAA|1997-10-27|2000-10-26|Global others could not manage much. Yellow projects defraud again projects. Efficient laws shall knit from a measures. Laws should announce in the issues. About coll|81.46|32.58|4002001|importoedu pack #1|2|mens|4|Shoes|592|ablen stanti|large|425589709salmon12291|slate|Tsp|Unknown|3|eseationeseeseought| +14475|AAAAAAAAKIIDAAAA|2000-10-27||Afraid, royal cuts reduce only specimens; conservative, special honours spend secondary, local effects. Proper forces could take either rewards. Labour, comprehensiv|2.89|1.84|4002001|exportibrand #6|3|kids|7|Home|592|ablen stanti|N/A|425589709salmon12291|purple|Pallet|Unknown|12|antiationeseeseought| +14476|AAAAAAAAMIIDAAAA|1997-10-27|1999-10-27|Slim characters will take common, psychological features. Reasons think economically. Good, geographical parties throw committees. Southern costs increa|3.04|1.21|7002007|importobrand #7|2|bedding|7|Home|490|barn stese|N/A|65papaya848504405228|drab|Dram|Unknown|90|callyationeseeseought| +14477|AAAAAAAAMIIDAAAA|1999-10-28|2001-10-26|Slim characters will take common, psychological features. Reasons think economically. Good, geographical parties throw committees. Southern costs increa|9.17|6.23|7002007|namelesscorp #6|2|mens watch|6|Jewelry|572|barn stese|N/A|25sienna782810231731|gainsboro|Cup|Unknown|93|ationationeseeseought| +14478|AAAAAAAAMIIDAAAA|2001-10-27||Slim characters will take common, psychological features. Reasons think economically. Good, geographical parties throw committees. Southern costs increa|1.94|0.91|7002007|scholarnameless #9|15|tables|7|Home|572|ableationanti|N/A|25sienna782810231731|purple|Cup|Unknown|61|eingationeseeseought| +14479|AAAAAAAAPIIDAAAA|1997-10-27||Lips know to a procedures. Modest, warm plants might go now professional, good sentences. Eager, interesting qu|2.38|1.97|5002002|importoscholar #2|2|country|5|Music|427|ationableese|N/A|6989201goldenrod3065|sky|Carton|Unknown|7|n stationeseeseought| +14480|AAAAAAAAAJIDAAAA|1997-10-27|2000-10-26|Possible things must suggest at first; equivalent, poor families fight partly too particular occasions. Professional claims present probably. Trials comply simila|7.92|2.45|2001001|amalgimporto #1|1|accessories|2|Men|199|n stn stought|medium|6puff379138434751423|sienna|Pound|Unknown|29|bareingeseeseought| +14481|AAAAAAAAAJIDAAAA|2000-10-27||Possible things must suggest at first; equivalent, poor families fight partly too particular occasions. Professional claims present probably. Trials comply simila|8.57|2.45|10009015|maxiunivamalg #15|9|televisions|10|Electronics|744|eseeseation|N/A|4564808468419sandy77|violet|Dozen|Unknown|19|oughteingeseeseought| +14482|AAAAAAAACJIDAAAA|1997-10-27|1999-10-27|Southern, physical forms may inherit long forms. Directors find suddenly. Standards should not say under just difficult reasons. Paths join a bit scientific issues. Onl|7.95|5.80|7016003|corpnameless #3|16|furniture|7|Home|727|ationableation|N/A|35641856red442372892|violet|Tbl|Unknown|11|ableeingeseeseought| +14483|AAAAAAAACJIDAAAA|1999-10-28|2001-10-26|Southern, physical forms may inherit long forms. Directors find suddenly. Standards should not say under just difficult reasons. Paths join a bit scientific issues. Onl|6.99|5.80|8008004|namelessnameless #4|8|outdoor|8|Sports|827|ationableation|N/A|76306177227saddle772|puff|Dram|Unknown|58|prieingeseeseought| +14484|AAAAAAAACJIDAAAA|2001-10-27||Clubs see really. Arab sales could not see wrong features. New men used to like in order progressive forms. Never inevitable doctors could tip international heads. Unique versions prove normal,|10.52|5.80|8008004|edu packexporti #1|8|school-uniforms|3|Children|827|ationableation|extra large|2624435823spring1078|honeydew|Case|Unknown|31|eseeingeseeseought| +14485|AAAAAAAAFJIDAAAA|1997-10-27||Military, principal honours reduce hard direct possibilities. High goals undermine too national, economic things. Days bowl private, warm pensio|0.45|0.14|6006006|corpcorp #6|6|rings|6|Jewelry|75|antiation|N/A|8midnight31123772557|sandy|Each|Unknown|5|antieingeseeseought| +14486|AAAAAAAAGJIDAAAA|1997-10-27|2000-10-26|Strong, other eyes address. Expectations ought to need|3.16|2.55|7004007|edu packbrand #7|4|curtains/drapes|7|Home|150|barantiought|N/A|35759turquoise801305|puff|Carton|Unknown|31|callyeingeseeseought| +14487|AAAAAAAAGJIDAAAA|2000-10-27||Circumstances reject most. High, old surfaces admit hence heavy values. Mild, german individuals should vanish established minutes. Considerable, national cases retai|65.93|52.08|6005008|scholarcorp #8|5|earings|6|Jewelry|930|barantiought|N/A|35759turquoise801305|pale|Gram|Unknown|91|ationeingeseeseought| +14488|AAAAAAAAIJIDAAAA|1997-10-27|1999-10-27|Unlikely figures get for example. Just long streets point drily psychiatric weeks. Quite clear factors af|7.95|6.83|5002001|importoscholar #1|2|country|5|Music|263|pricallyable|N/A|sky36539323750027500|rosy|Bundle|Unknown|26|eingeingeseeseought| +14489|AAAAAAAAIJIDAAAA|1999-10-28|2001-10-26|Positively powerful armies may not include more too special stairs. Normally full activities could not help more possible wild comments. Extra effects provide about. Then expensive boys might no|3.16|1.10|4003002|exportiedu pack #2|2|kids|4|Shoes|263|pricallyable|medium|sky36539323750027500|snow|Ton|Unknown|86|n steingeseeseought| +14490|AAAAAAAAIJIDAAAA|2001-10-27||Positively powerful armies may not include more too special stairs. Normally full activities could not help more possible wild comments. Extra effects provide about. Then expensive boys might no|4.72|2.87|4003002|importounivamalg #7|12|home repair|9|Books|26|callyable|N/A|15283black4989166181|ivory|Unknown|Unknown|2|barn steseeseought| +14491|AAAAAAAALJIDAAAA|1997-10-27||Major, important features buy also oral, secondary motives. Physical mechanisms watch firmly possible, awful mea|2.29|1.32|7010002|univnameless #2|10|flatware|7|Home|978|eingationn st|N/A|6592391azure67376959|smoke|Dram|Unknown|11|oughtn steseeseought| +14492|AAAAAAAAMJIDAAAA|1997-10-27|2000-10-26|Sources seek in the ministers. Cells might not keep neatly extra woods. New, little neighbours convince really for a minutes; words give both primary|1.82|0.60|8001003|amalgnameless #3|1|athletic shoes|8|Sports|253|priantiable|N/A|061615438chartreuse0|red|Cup|Unknown|11|ablen steseeseought| +14493|AAAAAAAAMJIDAAAA|2000-10-27||Sources seek in the ministers. Cells might not keep neatly extra woods. New, little neighbours convince really for a minutes; words give both primary|6.86|5.41|2002002|importoimporto #2|2|shirts|2|Men|253|priantiable|petite|2265926papaya0473654|orange|Gram|Unknown|39|prin steseeseought| +14494|AAAAAAAAOJIDAAAA|1997-10-27|1999-10-27|Expert casualties shall go by now for the thoughts; clients used to benefit entirely |8.47|4.82|10007011|brandunivamalg #11|7|personal|10|Electronics|947|ationesen st|N/A|lemon988118441157559|slate|Tsp|Unknown|64|esen steseeseought| +14495|AAAAAAAAOJIDAAAA|1999-10-28|2001-10-26|Expert casualties shall go by now for the thoughts; clients used to benefit entirely |1.70|4.82|10007011|exportischolar #2|7|pop|5|Music|947|ationesen st|N/A|lemon988118441157559|seashell|N/A|Unknown|19|antin steseeseought| +14496|AAAAAAAAOJIDAAAA|2001-10-27||Absolute, coming facts discuss other stars. Urban times will think away dirty styles. Civil, new targets purchase conse|9.13|5.02|10007013|brandunivamalg #13|7|personal|10|Electronics|464|ationesen st|N/A|5white22432767905872|pale|Bunch|Unknown|5|callyn steseeseought| +14497|AAAAAAAABKIDAAAA|1997-10-27||Leaves go most parties. Available, rich masses appear as administrative feet. Times could not explore at a chairs. Assistant, clear prices emerge neve|4.92|3.88|8009008|maxinameless #8|9|optics|8|Sports|77|ationation|N/A|3600powder8972785397|saddle|Box|Unknown|12|ationn steseeseought| +14498|AAAAAAAACKIDAAAA|1997-10-27|2000-10-26|Cells will pay. Possible, interesting pictures secure ou|0.45|0.37|2004001|edu packimporto #1|4|sports-apparel|2|Men|322|ableablepri|small|688171157seashell279|turquoise|Dozen|Unknown|64|eingn steseeseought| +14499|AAAAAAAACKIDAAAA|2000-10-27||Other, red windows would resolve already in the legs. Christian police shall not |4.99|0.37|5001002|amalgscholar #2|1|rock|5|Music|627|ableablepri|N/A|88874686213387rosy33|lawn|N/A|Unknown|4|n stn steseeseought| +14500|AAAAAAAAEKIDAAAA|1997-10-27|1999-10-27|Simple, other concentrations must believe indian, common years. Only statistical standards must sort thus lists. Liberal sign|84.88|43.28|8006001|corpnameless #1|6|football|8|Sports|23|priable|N/A|3703819powder8708929|steel|Gross|Unknown|39|barbarantieseought| +14501|AAAAAAAAEKIDAAAA|1999-10-28|2001-10-26|Simple, other concentrations must believe indian, common years. Only statistical standards must sort thus lists. Liberal sign|2.82|2.50|8006001|edu packscholar #2|6|classical|5|Music|23|priable|N/A|030031693602916lime4|puff|Ton|Unknown|27|oughtbarantieseought| +14502|AAAAAAAAEKIDAAAA|2001-10-27||Simple, other concentrations must believe indian, common years. Only statistical standards must sort thus lists. Liberal sign|0.30|0.24|3004001|edu packexporti #1|4|school-uniforms|3|Children|464|esecallyese|medium|30528970salmon704288|wheat|Pound|Unknown|57|ablebarantieseought| +14503|AAAAAAAAHKIDAAAA|1997-10-27||Perhaps human vehicles command still. Direct, young years will want so exclusive, fine tit|1.96|0.80|2001002|amalgimporto #2|1|accessories|2|Men|393|prin stpri|small|84925dark66203742476|puff|Ounce|Unknown|59|pribarantieseought| +14504|AAAAAAAAIKIDAAAA|1997-10-27|2000-10-26|Handsome trees could not become over lucky, human circumstances. Possible causes shall not make by a proposals. Only effective owners can like at least rates; sure, able|4.36|1.70|9005009|scholarmaxi #9|5|history|9|Books|215|antioughtable|N/A|60265520398saddle095|steel|Each|Unknown|10|esebarantieseought| +14505|AAAAAAAAIKIDAAAA|2000-10-27||Silver doors might need at least; never regular terms used to retain afterwards. Years realise at|9.66|8.01|9005009|importoedu pack #2|5|mens|4|Shoes|215|antioughtable|medium|60265520398saddle095|white|Ounce|Unknown|13|antibarantieseought| +14506|AAAAAAAAKKIDAAAA|1997-10-27|1999-10-27|Even corresponding foods receive techniques. Areas require very broad, special designs. Po|1.88|1.50|2003001|exportiimporto #1|3|pants|2|Men|2|able|extra large|9257royal02353476288|light|Ton|Unknown|9|callybarantieseought| +14507|AAAAAAAAKKIDAAAA|1999-10-28|2001-10-26|Miles can put methods. Too palestinian tools used to mind. Quite royal hearts will determine so l|1.48|0.99|2003001|scholarmaxi #12|5|history|9|Books|2|able|N/A|0554842835764purple3|spring|Dozen|Unknown|61|ationbarantieseought| +14508|AAAAAAAAKKIDAAAA|2001-10-27||Miles can put methods. Too palestinian tools used to mind. Quite royal hearts will determine so l|5.13|0.99|2003001|amalgimporto #1|5|accessories|2|Men|352|able|large|0008983871650maroon9|ghost|Pallet|Unknown|20|eingbarantieseought| +14509|AAAAAAAANKIDAAAA|1997-10-27||Little industries improve most little, expected types. Values eat criteria. Democratic insects cannot decide then comme|19.15|16.08|2004002|edu packimporto #2|4|sports-apparel|2|Men|416|callyoughtese|medium|18066072403peach2071|goldenrod|N/A|Unknown|27|n stbarantieseought| +14510|AAAAAAAAOKIDAAAA|1997-10-27|2000-10-26|Systems might not get especially. Previous terms may not persuade holders. Shares ought to know still special |2.70|1.10|5003001|exportischolar #1|3|pop|5|Music|58|einganti|N/A|24287chiffon65029867|red|Box|Unknown|30|baroughtantieseought| +14511|AAAAAAAAOKIDAAAA|2000-10-27||Systems might not get especially. Previous terms may not persuade holders. Shares ought to know still special |5.12|3.58|8013002|exportimaxi #2|3|sailing|8|Sports|290|barn stable|N/A|7470983puff337514588|mint|Ton|Unknown|57|oughtoughtantieseought| +14512|AAAAAAAAALIDAAAA|1997-10-27|1999-10-27|Other descriptions must not stop other hands; |2.55|2.16|6015001|scholarbrand #1|15|custom|6|Jewelry|200|barbarable|N/A|4387906light75974526|puff|Carton|Unknown|6|ableoughtantieseought| +14513|AAAAAAAAALIDAAAA|1999-10-28|2001-10-26|Other descriptions must not stop other hands; |3.30|1.81|6015001|edu packnameless #4|15|glassware|7|Home|297|ationn stable|N/A|4387906light75974526|thistle|Carton|Unknown|65|prioughtantieseought| +14514|AAAAAAAAALIDAAAA|2001-10-27||Other descriptions must not stop other hands; |2.49|1.81|7015005|scholarnameless #5|15|tables|7|Home|297|ationn stable|N/A|55990blanched8294507|deep|Cup|Unknown|4|eseoughtantieseought| +14515|AAAAAAAADLIDAAAA|1997-10-27||Old, |4.19|1.29|3004002|edu packexporti #2|4|school-uniforms|3|Children|201|oughtbarable|medium|8violet5452373862356|sky|Bunch|Unknown|17|antioughtantieseought| +14516|AAAAAAAAELIDAAAA|1997-10-27|2000-10-26|Jewish others might sort defendants; general events decide physically respective for|9.92|8.72|7010007|univnameless #7|10|flatware|7|Home|636|callyprically|N/A|025634141920rosy6913|peru|Bunch|Unknown|52|callyoughtantieseought| +14517|AAAAAAAAELIDAAAA|2000-10-27||Shots must continue g|6.66|2.46|1002002|importoamalg #2|10|fragrances|1|Women|636|callyprically|large|025634141920rosy6913|midnight|Dram|Unknown|59|ationoughtantieseought| +14518|AAAAAAAAGLIDAAAA|1997-10-27|1999-10-27|R|3.45|2.03|4004001|edu packedu pack #1|4|athletic|4|Shoes|305|antibarpri|medium|15324330289pale72992|yellow|Tbl|Unknown|6|eingoughtantieseought| +14519|AAAAAAAAGLIDAAAA|1999-10-28|2001-10-26|British, other decisions melt writings. Anywhere ordinary facts ought to suggest already laboratories; however compulsory opportunities like perhaps typically senior deputies. Now literary|44.65|14.28|4004001|importoimporto #2|4|shirts|2|Men|299|n stn stable|extra large|00742wheat4213227719|burnished|Bundle|Unknown|47|n stoughtantieseought| +14520|AAAAAAAAGLIDAAAA|2001-10-27||British, other decisions melt writings. Anywhere ordinary facts ought to suggest already laboratories; however compulsory opportunities like perhaps typically senior deputies. Now literary|0.98|14.28|4004001|importoedu pack #1|4|mens|4|Shoes|251|oughtantiable|small|00742wheat4213227719|salmon|Dozen|Unknown|6|barableantieseought| +14521|AAAAAAAAJLIDAAAA|1997-10-27||Chief, new years could press all confident designs. Ethical, possible notions can close still. Events improve in par|1.04|0.59|8015002|scholarmaxi #2|15|fishing|8|Sports|754|eseantiation|N/A|5298003211papaya1064|mint|Each|Unknown|5|oughtableantieseought| +14522|AAAAAAAAKLIDAAAA|1997-10-27|2000-10-26|Open parents can affect. Properly beautiful policies say very separate sides. Ever obvious members can represent in a things. Across objective lives could submit nearly actually st|57.13|47.41|3002001|importoexporti #1|2|infants|3|Children|469|n stcallyese|small|41059purple202219586|turquoise|N/A|Unknown|49|ableableantieseought| +14523|AAAAAAAAKLIDAAAA|2000-10-27||Open parents can affect. Properly beautiful policies say very separate sides. Ever obvious members can represent in a things. Across objective lives could submit nearly actually st|4.53|2.71|7010004|univnameless #4|2|flatware|7|Home|77|n stcallyese|N/A|2midnight50848679475|royal|Dozen|Unknown|36|priableantieseought| +14524|AAAAAAAAMLIDAAAA|1997-10-27|1999-10-27|Fo|0.33|0.23|7009005|maxibrand #5|9|mattresses|7|Home|685|antieingcally|N/A|60484836320honeydew6|tomato|Bunch|Unknown|50|eseableantieseought| +14525|AAAAAAAAMLIDAAAA|1999-10-28|2001-10-26|Legal children should not help halfway tiny sequences. About eager strangers change straight keen g|2.15|0.23|7009005|edu packexporti #2|9|school-uniforms|3|Children|446|antieingcally|medium|60484836320honeydew6|saddle|Gross|Unknown|27|antiableantieseought| +14526|AAAAAAAAMLIDAAAA|2001-10-27||Great eyes must use all present, sure date|8.98|0.23|4004001|edu packedu pack #1|9|athletic|4|Shoes|865|anticallyeing|medium|60484836320honeydew6|sky|Gram|Unknown|32|callyableantieseought| +14527|AAAAAAAAPLIDAAAA|1997-10-27||Items look somewhat new designs. Patients should solve about a officers. Minutes can act still companies. About dangerous records will not run towa|1.43|0.64|9003008|exportimaxi #8|3|computers|9|Books|382|ableeingpri|N/A|74047950salmon429974|spring|Dram|Unknown|6|ationableantieseought| +14528|AAAAAAAAAMIDAAAA|1997-10-27|2000-10-26|Things ought to say now criminal, new circumstances. Colonial systems say projects. Successful, poor pensions will care particularly initially total others.|4.82|2.26|4002001|importoedu pack #1|2|mens|4|Shoes|41|oughtese|small|60850170259white1946|pale|Gross|Unknown|13|eingableantieseought| +14529|AAAAAAAAAMIDAAAA|2000-10-27||Steps believe here instead happy businesses; other, good descriptions|1.32|2.26|6001002|amalgcorp #2|2|birdal|6|Jewelry|199|n stn stought|N/A|442powder30803297260|yellow|Tbl|Unknown|23|n stableantieseought| +14530|AAAAAAAACMIDAAAA|1997-10-27|1999-10-27|Open, violent relations might form normally well british controls. About huge hours tell very words; small police die both functions. More past winners marry finally convention|2.51|1.98|3003001|exportiexporti #1|3|toddlers|3|Children|154|eseantiought|small|0277452861878white30|light|N/A|Unknown|12|barpriantieseought| +14531|AAAAAAAACMIDAAAA|1999-10-28|2001-10-26|Social, heavy differences could not keep in order national, adequate|2.78|1.44|4001002|amalgedu pack #2|1|womens|4|Shoes|154|eseantiought|medium|40230520616072rose84|cream|Gross|Unknown|45|oughtpriantieseought| +14532|AAAAAAAACMIDAAAA|2001-10-27||Very |5.87|1.44|9001001|amalgmaxi #1|1|arts|9|Books|154|eseantiought|N/A|40230520616072rose84|pale|Ounce|Unknown|50|ablepriantieseought| +14533|AAAAAAAAFMIDAAAA|1997-10-27||Senses take really long plants. Small visitors examine however young businesses. Times should |4.03|2.41|1002002|importoamalg #2|2|fragrances|1|Women|209|n stbarable|large|767pale8774485403140|lemon|Pound|Unknown|41|pripriantieseought| +14534|AAAAAAAAGMIDAAAA|1997-10-27|2000-10-26|Anyway initial depths ought to raise over expenses. Little years ought to buy new sides. Phrases see across the folk. Barely considerable workers shall turn ev|2.54|1.85|9008003|namelessmaxi #3|8|romance|9|Books|243|prieseable|N/A|73532812thistle80542|pale|Pound|Unknown|63|esepriantieseought| +14535|AAAAAAAAGMIDAAAA|2000-10-27||Typical,|2.48|1.85|9008003|corpcorp #8|6|rings|6|Jewelry|243|prieseable|N/A|3sandy29550142323905|peru|Tsp|Unknown|25|antipriantieseought| +14536|AAAAAAAAIMIDAAAA|1997-10-27|1999-10-27|Varieties might remain very. Churches may not treat. Today important tears may not go enough delicious societies. Spanish assets may introduce shops. Much n|3.59|3.01|1001001|amalgamalg #1|1|dresses|1|Women|802|ablebareing|petite|898violet71905155106|slate|Gram|Unknown|9|callypriantieseought| +14537|AAAAAAAAIMIDAAAA|1999-10-28|2001-10-26|Varieties might remain very. Churches may not treat. Today important tears may not go enough delicious societies. Spanish assets may introduce shops. Much n|4.65|3.01|1001002|amalgamalg #2|1|dresses|1|Women|802|ablebareing|petite|898violet71905155106|plum|Cup|Unknown|31|ationpriantieseought| +14538|AAAAAAAAIMIDAAAA|2001-10-27||Varieties might remain very. Churches may not treat. Today important tears may not go enough delicious societies. Spanish assets may introduce shops. Much n|1.03|3.01|8013009|exportimaxi #9|1|sailing|8|Sports|802|ablebareing|N/A|898violet71905155106|tan|N/A|Unknown|22|eingpriantieseought| +14539|AAAAAAAALMIDAAAA|1997-10-27||Other voices would jump |43.78|31.95|5001002|amalgscholar #2|1|rock|5|Music|14|eseought|N/A|lace1146479430699849|salmon|Cup|Unknown|9|n stpriantieseought| +14540|AAAAAAAAMMIDAAAA|1997-10-27|2000-10-26|Technical proportions might perform poor jeans. All right subjects see alternative, big hundreds. Likely months guarantee always especially lon|8.87|2.66|9011009|amalgunivamalg #9|11|cooking|9|Books|926|callyablen st|N/A|74819snow39311445673|tomato|Oz|Unknown|57|bareseantieseought| +14541|AAAAAAAAMMIDAAAA|2000-10-27||Technical proportions might perform poor jeans. All right subjects see alternative, big hundreds. Likely months guarantee always especially lon|67.36|37.04|4004002|edu packedu pack #2|11|athletic|4|Shoes|457|callyablen st|N/A|994122813snow0670911|seashell|Pallet|Unknown|90|oughteseantieseought| +14542|AAAAAAAAOMIDAAAA|1997-10-27|1999-10-27|Dramatic, excellent years lose certainly reports. Areas sign actually children. Final, human areas eat gentl|7.11|4.76|1001001|amalgamalg #1|1|dresses|1|Women|494|esen stese|extra large|30649289534white6876|saddle|Ton|Unknown|90|ableeseantieseought| +14543|AAAAAAAAOMIDAAAA|1999-10-28|2001-10-26|Perhaps scientific notions join grateful, fat countri|7.11|4.76|1001001|exportiexporti #2|1|toddlers|3|Children|563|pricallyanti|small|30649289534white6876|pale|Box|Unknown|60|prieseantieseought| +14544|AAAAAAAAOMIDAAAA|2001-10-27||Perhaps scientific notions join grateful, fat countri|5.73|4.76|1001001|amalgscholar #1|1|rock|5|Music|581|pricallyanti|N/A|0801yellow2134895086|bisque|Ton|Unknown|81|eseeseantieseought| +14545|AAAAAAAABNIDAAAA|1997-10-27||Educational tools must get middle plants; american, female things would force now; small eyes would not release smart, unemployed cities. Small years cope for the wo|1.48|0.62|5003002|exportischolar #2|3|pop|5|Music|186|callyeingought|N/A|68145sky731921750542|royal|Ounce|Unknown|28|antieseantieseought| +14546|AAAAAAAACNIDAAAA|1997-10-27|2000-10-26|Major faces cannot support now all official parties. Recent, popular rows might not regret with the prices. More large items argue. Schools purchas|97.49|81.89|8002007|importonameless #7|2|baseball|8|Sports|6|cally|N/A|74732610rose29288795|ivory|Dram|Unknown|25|callyeseantieseought| +14547|AAAAAAAACNIDAAAA|2000-10-27||Movements enjoy effectiv|4.45|1.78|8002007|exportischolar #2|3|pop|5|Music|613|prioughtcally|N/A|6794150navy131725547|smoke|Unknown|Unknown|12|ationeseantieseought| +14548|AAAAAAAAENIDAAAA|1997-10-27|1999-10-27|There important doctors might avoid well both free benefits. Lips come; hours can form yesterday into a parties. Small, international americans rebuild |4.53|1.85|6014005|edu packbrand #5|14|estate|6|Jewelry|227|ationableable|N/A|3041594moccasin22646|chocolate|Dram|Unknown|4|eingeseantieseought| +14549|AAAAAAAAENIDAAAA|1999-10-28|2001-10-26|There important doctors might avoid well both free benefits. Lips come; hours can form yesterday into a parties. Small, international americans rebuild |7.67|1.85|6014005|exportiamalg #2|14|maternity|1|Women|146|callyeseought|large|3041594moccasin22646|khaki|Carton|Unknown|47|n steseantieseought| +14550|AAAAAAAAENIDAAAA|2001-10-27||There important doctors might avoid well both free benefits. Lips come; hours can form yesterday into a parties. Small, international americans rebuild |2.65|1.85|10015016|scholaramalgamalg #16|14|portable|10|Electronics|303|pribarpri|N/A|3041594moccasin22646|snow|Pallet|Unknown|5|barantiantieseought| +14551|AAAAAAAAHNIDAAAA|1997-10-27||Russian, supposed needs secure. Australian eyes shall keep accordingly bad shoes. Other|3.26|2.28|10009017|maxiunivamalg #17|9|televisions|10|Electronics|541|oughteseanti|N/A|613121light623008527|seashell|Ton|Unknown|4|oughtantiantieseought| +14552|AAAAAAAAINIDAAAA|1997-10-27|2000-10-26|Also various outcomes promote students. White moments wish incomes. Entire arms us|5.83|4.13|6011001|amalgbrand #1|11|semi-precious|6|Jewelry|72|ableation|N/A|299ghost470443464173|sandy|Gram|Unknown|11|ableantiantieseought| +14553|AAAAAAAAINIDAAAA|2000-10-27||Also various outcomes promote students. White moments wish incomes. Entire arms us|13.76|7.56|7013004|exportinameless #4|11|wallpaper|7|Home|281|oughteingable|N/A|299ghost470443464173|royal|Oz|Unknown|11|priantiantieseought| +14554|AAAAAAAAKNIDAAAA|1997-10-27|1999-10-27|Big,|8.02|2.72|1004001|edu packamalg #1|4|swimwear|1|Women|835|antiprieing|medium|91147351spring505679|almond|Dozen|Unknown|11|eseantiantieseought| +14555|AAAAAAAAKNIDAAAA|1999-10-28|2001-10-26|Big,|7.62|5.86|1004001|importoimporto #2|2|shirts|2|Men|80|bareing|petite|094531057goldenrod82|moccasin|Ton|Unknown|29|antiantiantieseought| +14556|AAAAAAAAKNIDAAAA|2001-10-27||Big,|3.62|2.20|7001007|amalgbrand #7|2|bathroom|7|Home|495|antin stese|N/A|094531057goldenrod82|pink|Gross|Unknown|52|callyantiantieseought| +14557|AAAAAAAANNIDAAAA|1997-10-27||Months could not take most. Exclusive, musical homes tell to a princip|20.61|12.15|3001002|amalgexporti #2|1|newborn|3|Children|313|prioughtpri|economy|7918897100596mint208|lace|Pound|Unknown|6|ationantiantieseought| +14558|AAAAAAAAONIDAAAA|1997-10-27|2000-10-26|So serious weeks might come weak members. At all young boxes imagine armed girls; fairly political services work technical, local authorities; actu|51.11|31.68|9016003|corpunivamalg #3|16|mystery|9|Books|130|barpriought|N/A|0296982771ivory01196|sky|Cup|Unknown|25|eingantiantieseought| +14559|AAAAAAAAONIDAAAA|2000-10-27||High, broad countri|99.82|31.68|4003002|exportiedu pack #2|3|kids|4|Shoes|985|barpriought|extra large|0296982771ivory01196|sienna|Carton|Unknown|2|n stantiantieseought| +14560|AAAAAAAAAOIDAAAA|1997-10-27|1999-10-27|Long, loose ways will survive just only prime cases. Small, video-taped cameras must not reject new, rural com|1.54|0.60|5004001|edu packscholar #1|4|classical|5|Music|584|eseeinganti|N/A|2105steel90166842924|tan|Gross|Unknown|23|barcallyantieseought| +14561|AAAAAAAAAOIDAAAA|1999-10-28|2001-10-26|Long, loose ways will survive just only prime cases. Small, video-taped cameras must not reject new, rural com|4.20|3.44|5004001|namelessnameless #2|4|outdoor|8|Sports|584|eseeinganti|N/A|2105steel90166842924|smoke|Bunch|Unknown|58|oughtcallyantieseought| +14562|AAAAAAAAAOIDAAAA|2001-10-27||Yellow doubts appear here in a courses. European foreigners|3.01|3.44|2003001|exportiimporto #1|3|pants|2|Men|584|eseeinganti|medium|0525514974778puff702|olive|Case|Unknown|37|ablecallyantieseought| +14563|AAAAAAAADOIDAAAA|1997-10-27||Necessary accounts portray necessary, apparent parties. Valuable schemes would display new communities. Rural birds might|3.80|1.67|4001002|amalgedu pack #2|1|womens|4|Shoes|225|antiableable|large|603white368051497017|slate|Case|Unknown|14|pricallyantieseought| +14564|AAAAAAAAEOIDAAAA|1997-10-27|2000-10-26|Of course old courses run unduly. As primary relations reach perhaps. Players collect full, nearby publications. Strong, rare times recover far physical men. Guilty, valid years must not|4.84|1.54|10001013|amalgunivamalg #13|1|cameras|10|Electronics|117|ationoughtought|N/A|2873410aquamarine492|slate|Gram|Unknown|23|esecallyantieseought| +14565|AAAAAAAAEOIDAAAA|2000-10-27||Of course old courses run unduly. As primary relations reach perhaps. Players collect full, nearby publications. Strong, rare times recover far physical men. Guilty, valid years must not|1.65|0.62|1003002|exportiamalg #2|1|maternity|1|Women|33|pripri|petite|2983298418khaki77206|lavender|Carton|Unknown|69|anticallyantieseought| +14566|AAAAAAAAGOIDAAAA|1997-10-27|1999-10-27|English proposals like early by a symptoms. Operations make then in accordance with the kinds. Part-time, nuclear views feel very never legal schools. Pupils cann|0.77|0.39|6014007|edu packbrand #7|14|estate|6|Jewelry|173|priationought|N/A|157160695sienna75200|papaya|Gross|Unknown|14|callycallyantieseought| +14567|AAAAAAAAGOIDAAAA|1999-10-28|2001-10-26|English proposals like early by a symptoms. Operations make then in accordance with the kinds. Part-time, nuclear views feel very never legal schools. Pupils cann|4.05|0.39|6014007|exportiamalg #2|3|maternity|1|Women|173|priationought|large|157160695sienna75200|snow|Ounce|Unknown|7|ationcallyantieseought| +14568|AAAAAAAAGOIDAAAA|2001-10-27||English proposals like early by a symptoms. Operations make then in accordance with the kinds. Part-time, nuclear views feel very never legal schools. Pupils cann|0.14|0.39|6014007|importoexporti #1|2|infants|3|Children|173|priationought|medium|157160695sienna75200|papaya|Gross|Unknown|34|eingcallyantieseought| +14569|AAAAAAAAJOIDAAAA|1997-10-27||Incredi|0.22|0.15|7005010|scholarbrand #10|5|blinds/shades|7|Home|229|n stableable|N/A|921yellow69640422627|lemon|N/A|Unknown|44|n stcallyantieseought| +14570|AAAAAAAAKOIDAAAA|1997-10-27|2000-10-26|Rights pay|4.07|1.54|7016001|corpnameless #1|16|furniture|7|Home|612|ableoughtcally|N/A|3855454rosy930418503|orange|Pound|Unknown|1|barationantieseought| +14571|AAAAAAAAKOIDAAAA|2000-10-27||Never loyal years get scarcely dark criticisms. Nice criter|68.44|1.54|7016001|exporticorp #4|3|gold|6|Jewelry|612|ableoughtcally|N/A|3855454rosy930418503|smoke|Ton|Unknown|12|oughtationantieseought| +14572|AAAAAAAAMOIDAAAA|1997-10-27|1999-10-27|Chief, causal games wait special qu|9.05|3.34|1003001|exportiamalg #1|3|maternity|1|Women|6|cally|petite|422peach425837932156|smoke|Oz|Unknown|50|ableationantieseought| +14573|AAAAAAAAMOIDAAAA|1999-10-28|2001-10-26|Eyes include very local, active men. No longer united ingredients ought to explain as inc areas. Of course single results will indicate in the houses. Ingredients can take equally sternly sure sta|3.73|3.34|8008004|namelessnameless #4|3|outdoor|8|Sports|6|cally|N/A|829819gainsboro80947|peru|Unknown|Unknown|66|priationantieseought| +14574|AAAAAAAAMOIDAAAA|2001-10-27||Long, ready chi|1.31|3.34|3001001|amalgexporti #1|3|newborn|3|Children|6|cally|medium|829819gainsboro80947|peru|Dram|Unknown|51|eseationantieseought| +14575|AAAAAAAAPOIDAAAA|1997-10-27||Students will land rather so proposed eyes. Views could live newly. Warm, good jeans n|2.68|1.34|2001002|amalgimporto #2|1|accessories|2|Men|147|ationeseought|large|31238midnight7475893|lime|Unknown|Unknown|49|antiationantieseought| +14576|AAAAAAAAAPIDAAAA|1997-10-27|2000-10-26|Above|4.33|3.16|1003001|exportiamalg #1|3|maternity|1|Women|128|eingableought|medium|4893smoke22279200850|hot|Dozen|Unknown|14|callyationantieseought| +14577|AAAAAAAAAPIDAAAA|2000-10-27||Strongly usual thousands argue grimly national workers. International, long patients write there. Too difficult stairs must not look limited, international families. |1.80|1.45|1003001|exportiamalg #2|3|maternity|1|Women|128|eingableought|extra large|39985622340pale36065|smoke|Tsp|Unknown|93|ationationantieseought| +14578|AAAAAAAACPIDAAAA|1997-10-27|1999-10-27|Outcomes bur|3.73|2.12|3003001|exportiexporti #1|3|toddlers|3|Children|54|eseanti|extra large|73287307889slate7949|sandy|Gross|Unknown|63|eingationantieseought| +14579|AAAAAAAACPIDAAAA|1999-10-28|2001-10-26|Leading others must demonstrate common, successful effects; useful, occupational books would not tear no doubt most aware fri|8.90|7.92|3003001|exportinameless #10|3|wallpaper|7|Home|91|oughtn st|N/A|73287307889slate7949|saddle|Bunch|Unknown|56|n stationantieseought| +14580|AAAAAAAACPIDAAAA|2001-10-27||Leading others must demonstrate common, successful effects; useful, occupational books would not tear no doubt most aware fri|0.67|0.40|3003001|exportiexporti #1|3|toddlers|3|Children|167|ationcallyought|small|206grey5983969383913|thistle|Gram|Unknown|51|bareingantieseought| +14581|AAAAAAAAFPIDAAAA|1997-10-27||Efforts put whole sales. Guests used to telephone with a hours; green members marry so total candidates. Acceptable rooms might happen some|9.59|4.41|10003013|exportiunivamalg #13|3|dvd/vcr players|10|Electronics|290|barn stable|N/A|8709144348004indian5|green|Gross|Unknown|28|oughteingantieseought| +14582|AAAAAAAAGPIDAAAA|1997-10-27|2000-10-26|Sexual, foreign attitudes follow previously other, so-called friends. Certain managers match reductions. States used to stop laws. Still similar magi|4.97|1.88|5001001|amalgscholar #1|1|rock|5|Music|48|eingese|N/A|75turquoise942455381|smoke|Pallet|Unknown|97|ableeingantieseought| +14583|AAAAAAAAGPIDAAAA|2000-10-27||Sexual, foreign attitudes follow previously other, so-called friends. Certain managers match reductions. States used to stop laws. Still similar magi|2.53|1.88|7005004|scholarbrand #4|1|blinds/shades|7|Home|48|eingese|N/A|75turquoise942455381|powder|Lb|Unknown|28|prieingantieseought| +14584|AAAAAAAAIPIDAAAA|1997-10-27|1999-10-27|Inches make. Tables |0.44|0.32|7012005|importonameless #5|12|paint|7|Home|297|ationn stable|N/A|1sienna9484963713165|rose|N/A|Unknown|67|eseeingantieseought| +14585|AAAAAAAAIPIDAAAA|1999-10-28|2001-10-26|Inches make. Tables |2.48|0.74|5002002|importoscholar #2|2|country|5|Music|118|eingoughtought|N/A|1sienna9484963713165|goldenrod|Ton|Unknown|53|antieingantieseought| +14586|AAAAAAAAIPIDAAAA|2001-10-27||However correct rights should sing high, historical terms. Large standards might not want about in the lines; real knees receive materials. Potential, traditional researchers can know in a years. Alo|1.62|0.89|5002002|edu packamalg #1|4|swimwear|1|Women|260|eingoughtought|medium|0281248pale421231810|snow|Bundle|Unknown|21|callyeingantieseought| +14587|AAAAAAAALPIDAAAA|1997-10-27||Potential, diplomatic proceedings might not dispose absolutely. Modern, able results see so properties. Short cars may serve together terms. Immense wishes will not stand little games. Quiet, full t|93.27|79.27|4002002|importoedu pack #2|2|mens|4|Shoes|441|oughteseese|extra large|87tan983247323096526|saddle|Carton|Unknown|43|ationeingantieseought| +14588|AAAAAAAAMPIDAAAA|1997-10-27|2000-10-26|Initially neighbouring pages hold then at a words. Direct, socia|9.81|3.53|4004001|edu packedu pack #1|4|athletic|4|Shoes|80|bareing|medium|midnight599711968423|peach|Carton|Unknown|62|eingeingantieseought| +14589|AAAAAAAAMPIDAAAA|2000-10-27||Grey others must not include early links. Large, cold days might follow formally; vulnerable expenses should not marry by a forms; sho|9.36|3.53|2003002|exportiimporto #2|4|pants|2|Men|80|bareing|large|313727red01945037679|navy|Gram|Unknown|24|n steingantieseought| +14590|AAAAAAAAOPIDAAAA|1997-10-27|1999-10-27|Different, equal children make now reasons; other, expensive strategies make traditional, objective politicians; general, blue unions get through the investors. Days thank mu|7.10|3.12|4002001|importoedu pack #1|2|mens|4|Shoes|36|callypri|small|84291391653lemon6946|tomato|Dram|Unknown|46|barn stantieseought| +14591|AAAAAAAAOPIDAAAA|1999-10-28|2001-10-26|Different, equal children make now reasons; other, expensive strategies make traditional, objective politicians; general, blue unions get through the investors. Days thank mu|3.46|3.12|4002001|exportiimporto #2|3|pants|2|Men|36|callypri|medium|84291391653lemon6946|tomato|Tsp|Unknown|7|oughtn stantieseought| +14592|AAAAAAAAOPIDAAAA|2001-10-27||Less large details shall catch really bags. Schemes demand citizens; corporate groups admit later by a affairs. Aware periods shall surprise rules. The|39.65|27.75|4002001|importoamalg #1|3|fragrances|1|Women|245|callypri|extra large|061215067pink2923663|violet|Pound|Unknown|37|ablen stantieseought| +14593|AAAAAAAABAJDAAAA|1997-10-27||Only professional groups remain perhaps in a governments. Private elections expect then |0.87|0.73|10016008|corpamalgamalg #8|16|wireless|10|Electronics|338|eingpripri|N/A|879481008560puff0366|purple|Pound|Unknown|23|prin stantieseought| +14594|AAAAAAAACAJDAAAA|1997-10-27|2000-10-26|Trees recognise r|6.98|2.51|1004001|edu packamalg #1|4|swimwear|1|Women|319|n stoughtpri|large|52474377light3837833|slate|Oz|Unknown|28|esen stantieseought| +14595|AAAAAAAACAJDAAAA|2000-10-27||Trees recognise r|3.23|2.51|1004001|scholarnameless #6|4|tables|7|Home|318|eingoughtpri|N/A|52474377light3837833|mint|Ton|Unknown|16|antin stantieseought| +14596|AAAAAAAAEAJDAAAA|1997-10-27|1999-10-27|Better financial c|25.67|20.02|2002001|importoimporto #1|2|shirts|2|Men|84|eseeing|small|0428sky3197201026740|seashell|Box|Unknown|42|callyn stantieseought| +14597|AAAAAAAAEAJDAAAA|1999-10-28|2001-10-26|Better financial c|4.27|20.02|7009008|maxibrand #8|9|mattresses|7|Home|51|oughtanti|N/A|90456999black1769195|papaya|Tbl|Unknown|18|ationn stantieseought| +14598|AAAAAAAAEAJDAAAA|2001-10-27||International recommendations shall love direct phenomena. Old, small levels hold below in a friends. Deep minerals should not write good, new hands. New, safe boats gain much then eth|4.35|20.02|9011007|amalgunivamalg #7|9|cooking|9|Books|51|oughtanti|N/A|ivory063349152584709|turquoise|Lb|Unknown|40|eingn stantieseought| +14599|AAAAAAAAHAJDAAAA|1997-10-27||Largely orange crews will take as still possible examples. Human, just bishops ought to remove terms; forward, r|5.46|4.47|4004002|edu packedu pack #2|4|athletic|4|Shoes|40|barese|extra large|9steel52601814836041|tan|Bunch|Unknown|39|n stn stantieseought| +14600|AAAAAAAAIAJDAAAA|1997-10-27|2000-10-26|Local, individual cattle keep dimensions. Outstanding, human feet operate further average, old sides. Advantages would not make; l|3.62|2.31|6015001|scholarbrand #1|15|custom|6|Jewelry|31|oughtpri|N/A|sienna71178950414260|yellow|Bunch|Unknown|63|barbarcallyeseought| +14601|AAAAAAAAIAJDAAAA|2000-10-27||Local, individual cattle keep dimensions. Outstanding, human feet operate further average, old sides. Advantages would not make; l|6.18|2.31|6015001|amalgexporti #2|15|newborn|3|Children|31|oughtpri|extra large|sienna71178950414260|pink|Pound|Unknown|2|oughtbarcallyeseought| +14602|AAAAAAAAKAJDAAAA|1997-10-27|1999-10-27|Short neighbours implement innocently tiny titles. Briefly simple years should not tell potentially successful, whole years. Orange workers carry; home hot feet l|4.43|1.59|9002011|importomaxi #11|2|business|9|Books|981|oughteingn st|N/A|9red7091542965463366|royal|Ton|Unknown|4|ablebarcallyeseought| +14603|AAAAAAAAKAJDAAAA|1999-10-28|2001-10-26|Short neighbours implement innocently tiny titles. Briefly simple years should not tell potentially successful, whole years. Orange workers carry; home hot feet l|5.74|2.29|9002011|amalgedu pack #2|1|womens|4|Shoes|245|antieseable|economy|56089460450khaki8003|sienna|Bundle|Unknown|13|pribarcallyeseought| +14604|AAAAAAAAKAJDAAAA|2001-10-27||Short neighbours implement innocently tiny titles. Briefly simple years should not tell potentially successful, whole years. Orange workers carry; home hot feet l|61.19|29.37|3004001|edu packexporti #1|1|school-uniforms|3|Children|706|callybaration|large|41wheat5566040203045|steel|Pallet|Unknown|56|esebarcallyeseought| +14605|AAAAAAAANAJDAAAA|1997-10-27||Religious, delicious ways must a|7.07|3.74|9003002|exportimaxi #2|3|computers|9|Books|182|ableeingought|N/A|42243sandy5055593558|turquoise|Cup|Unknown|20|antibarcallyeseought| +14606|AAAAAAAAOAJDAAAA|1997-10-27|2000-10-26|Initial, g|0.93|0.70|1002001|importoamalg #1|2|fragrances|1|Women|4|ese|small|8890433369678sienna0|goldenrod|Gross|Unknown|27|callybarcallyeseought| +14607|AAAAAAAAOAJDAAAA|2000-10-27||Initial, g|39.24|0.70|3004002|edu packexporti #2|2|school-uniforms|3|Children|4|ese|economy|8890433369678sienna0|lace|Unknown|Unknown|3|ationbarcallyeseought| +14608|AAAAAAAAABJDAAAA|1997-10-27|1999-10-27|Able, sharp offices put similarly also curious connections; so broken decades divert frequently rights. Candidates should want both previous cells. Eyes think european hands. Northern depths|3.54|1.94|10011006|amalgamalgamalg #6|11|disk drives|10|Electronics|202|ablebarable|N/A|049smoke698160426233|lemon|Pallet|Unknown|25|eingbarcallyeseought| +14609|AAAAAAAAABJDAAAA|1999-10-28|2001-10-26|Generous feet get too local, old protests. Positive, ordinary doubts run highly communities. Countries drain to a voices. Years suggest proceedings. Usually basic effects shall say se|2.65|1.53|10011006|exportinameless #10|3|basketball|8|Sports|572|ablebarable|N/A|049smoke698160426233|medium|Oz|Unknown|10|n stbarcallyeseought| +14610|AAAAAAAAABJDAAAA|2001-10-27||Statutory pupils get potential pairs. Good, blue ways might t|0.86|0.76|8002001|importonameless #1|3|baseball|8|Sports|572|ablebarable|N/A|06400sandy1151217400|thistle|Tsp|Unknown|42|baroughtcallyeseought| +14611|AAAAAAAADBJDAAAA|1997-10-27||Quite dry thousands should not save new years. Orthodox sources can form somet|7.73|5.56|2003002|exportiimporto #2|3|pants|2|Men|189|n steingought|small|7057346069776pink991|rosy|Pound|Unknown|16|oughtoughtcallyeseought| +14612|AAAAAAAAEBJDAAAA|1997-10-27|2000-10-26|Unable movements pay as yet abstract lives. Extremely successful plans can ensure totally new years. Long volumes may receive nerves. Domi|7.92|6.57|1003001|exportiamalg #1|3|maternity|1|Women|674|eseationcally|medium|3223671powder7280986|plum|Unknown|Unknown|37|ableoughtcallyeseought| +14613|AAAAAAAAEBJDAAAA|2000-10-27||Unable movements pay as yet abstract lives. Extremely successful plans can ensure totally new years. Long volumes may receive nerves. Domi|7.98|3.99|6014006|edu packbrand #6|3|estate|6|Jewelry|674|eseationcally|N/A|3223671powder7280986|mint|Carton|Unknown|62|prioughtcallyeseought| +14614|AAAAAAAAGBJDAAAA|1997-10-27|1999-10-27|Extremely revolutionary towns used to reduce unique words.|17.95|6.64|2001001|amalgimporto #1|1|accessories|2|Men|663|pricallycally|small|98693411147sienna816|deep|Cup|Unknown|21|eseoughtcallyeseought| +14615|AAAAAAAAGBJDAAAA|1999-10-28|2001-10-26|Arab, free reason|1.07|6.64|2001001|univamalgamalg #13|1|memory|10|Electronics|631|pricallycally|N/A|98693411147sienna816|powder|Carton|Unknown|5|antioughtcallyeseought| +14616|AAAAAAAAGBJDAAAA|2001-10-27||Ve|3.37|6.64|4004001|edu packedu pack #1|1|athletic|4|Shoes|631|oughtprically|extra large|009saddle90488872866|saddle|Oz|Unknown|29|callyoughtcallyeseought| +14617|AAAAAAAAJBJDAAAA|1997-10-27||Ridiculous candidates will show incredible, late authorities. Too underlying things can know sometimes additional, good wee|1.91|0.99|4001002|amalgedu pack #2|1|womens|4|Shoes|102|ablebarought|petite|514780lemon345067926|khaki|Pallet|Unknown|19|ationoughtcallyeseought| +14618|AAAAAAAAKBJDAAAA|1997-10-27|2000-10-26|Recent, local styles need please. Conditions should not wish. Seconds explain below in a elec|0.46|0.14|6004003|edu packcorp #3|4|bracelets|6|Jewelry|332|ablepripri|N/A|89400805687orange754|peach|Dram|Unknown|9|eingoughtcallyeseought| +14619|AAAAAAAAKBJDAAAA|2000-10-27||Recent, local styles need please. Conditions should not wish. Seconds explain below in a elec|3.14|1.85|6004003|importonameless #10|4|baseball|8|Sports|119|n stoughtought|N/A|89400805687orange754|sandy|Each|Unknown|73|n stoughtcallyeseought| +14620|AAAAAAAAMBJDAAAA|1997-10-27|1999-10-27|Inevitably recent examples would not want even patients. Sharp, major places would make together too cen|4.38|1.62|6012005|importobrand #5|12|costume|6|Jewelry|952|ableantin st|N/A|0494265417plum630890|plum|Unknown|Unknown|1|barablecallyeseought| +14621|AAAAAAAAMBJDAAAA|1999-10-28|2001-10-26|Inevitably recent examples would not want even patients. Sharp, major places would make together too cen|3.97|1.62|6007006|brandcorp #6|12|pendants|6|Jewelry|952|ableantin st|N/A|67512542forest247219|tan|Carton|Unknown|8|oughtablecallyeseought| +14622|AAAAAAAAMBJDAAAA|2001-10-27||Other opportunities suspect demands; important, net effects may see generally. International things should not begin then. That is good signals can see again real waves. Soviet |1.48|1.62|6007006|edu packscholar #1|4|classical|5|Music|122|ableableought|N/A|67512542forest247219|ivory|Tsp|Unknown|7|ableablecallyeseought| +14623|AAAAAAAAPBJDAAAA|1997-10-27||Electronic expenses call further financial letters. Also firm sectors would point young banks; others get at a dogs. New, national parents shall go to a forces. French women turn spec|3.05|2.34|5004002|edu packscholar #2|4|classical|5|Music|768|eingcallyation|N/A|609317432758peach770|white|Bundle|Unknown|3|priablecallyeseought| +14624|AAAAAAAAACJDAAAA|1997-10-27|2000-10-26|M|3.93|2.71|7016005|corpnameless #5|16|furniture|7|Home|488|eingeingese|N/A|954ivory621358520917|papaya|Dozen|Unknown|29|eseablecallyeseought| +14625|AAAAAAAAACJDAAAA|2000-10-27||M|4.36|2.13|1001002|amalgamalg #2|1|dresses|1|Women|320|eingeingese|small|954ivory621358520917|saddle|Bundle|Unknown|41|antiablecallyeseought| +14626|AAAAAAAACCJDAAAA|1997-10-27|1999-10-27|Actual, different stories could suggest|0.49|0.24|2003001|exportiimporto #1|3|pants|2|Men|657|ationantically|small|596494727784spring51|slate|Pallet|Unknown|27|callyablecallyeseought| +14627|AAAAAAAACCJDAAAA|1999-10-28|2001-10-26|Hence environmental measures will win cards. Various ru|2.40|1.12|2003001|edu packedu pack #2|4|athletic|4|Shoes|410|baroughtese|medium|596494727784spring51|plum|Unknown|Unknown|13|ationablecallyeseought| +14628|AAAAAAAACCJDAAAA|2001-10-27||Hence environmental measures will win cards. Various ru|5.10|1.12|4002001|importoedu pack #1|2|mens|4|Shoes|410|baroughtese|petite|596494727784spring51|salmon|Unknown|Unknown|42|eingablecallyeseought| +14629|AAAAAAAAFCJDAAAA|1997-10-27||Urban, upper forces may see alone commercial, other terms. Hopes support. St|2.98|0.95|7003010|exportibrand #10|3|kids|7|Home|74|eseation|N/A|889066319snow4325823|violet|Pound|Unknown|9|n stablecallyeseought| +14630|AAAAAAAAGCJDAAAA|1997-10-27|2000-10-26|Lessons |6.67|4.60|8008001|namelessnameless #1|8|outdoor|8|Sports|362|ablecallypri|N/A|8thistle134098322435|steel|Box|Unknown|33|barpricallyeseought| +14631|AAAAAAAAGCJDAAAA|2000-10-27||Sharp, other|0.63|0.32|7002002|importobrand #2|2|bedding|7|Home|65|ablecallypri|N/A|538095810thistle6470|peru|Tbl|Unknown|11|oughtpricallyeseought| +14632|AAAAAAAAICJDAAAA|1997-10-27|1999-10-27|Small, empty windows might echo else certain publishers. New, distinctive publications transfer less friendly |6.37|3.69|2003001|exportiimporto #1|3|pants|2|Men|319|n stoughtpri|large|7930432674snow826333|plum|Unknown|Unknown|1|ablepricallyeseought| +14633|AAAAAAAAICJDAAAA|1999-10-28|2001-10-26|Small, empty windows might echo else certain publishers. New, distinctive publications transfer less friendly |3.46|1.86|1001002|amalgamalg #2|3|dresses|1|Women|319|n stoughtpri|extra large|30434622551powder534|cream|Carton|Unknown|23|pripricallyeseought| +14634|AAAAAAAAICJDAAAA|2001-10-27||Totally current privileges provide. Military districts might boost for the companies|1.10|0.56|10002015|importounivamalg #15|3|camcorders|10|Electronics|563|n stoughtpri|N/A|85tan936001255148965|powder|Each|Unknown|25|esepricallyeseought| +14635|AAAAAAAALCJDAAAA|1997-10-27||High, detailed structures can play well naturally other circ|2.14|1.30|1001002|amalgamalg #2|1|dresses|1|Women|237|ationpriable|medium|9368steel84474651474|smoke|Unknown|Unknown|29|antipricallyeseought| +14636|AAAAAAAAMCJDAAAA|1997-10-27|2000-10-26|Internation|3.23|1.51|6011005|amalgbrand #5|11|semi-precious|6|Jewelry|351|oughtantipri|N/A|17100yellow515187552|royal|Gross|Unknown|72|callypricallyeseought| +14637|AAAAAAAAMCJDAAAA|2000-10-27||Internation|9.23|1.51|2003002|exportiimporto #2|3|pants|2|Men|332|ablepripri|petite|17100yellow515187552|papaya|Bunch|Unknown|11|ationpricallyeseought| +14638|AAAAAAAAOCJDAAAA|1997-10-27|1999-10-27|Industrial, pleased arms choose at all legal, industrial |3.43|1.30|8001009|amalgnameless #9|1|athletic shoes|8|Sports|61|oughtcally|N/A|22mint81145990313569|purple|Dozen|Unknown|42|eingpricallyeseought| +14639|AAAAAAAAOCJDAAAA|1999-10-28|2001-10-26|Poor purposes cover foreign neighbours. Visitors cannot go this different, young matters. As available authorities used to save already separate other policies. Activities harness especiall|2.39|1.30|8001009|edu packscholar #2|1|classical|5|Music|61|oughtcally|N/A|22violet849551560316|white|Bunch|Unknown|13|n stpricallyeseought| +14640|AAAAAAAAOCJDAAAA|2001-10-27||Symbolic, short bishops may t|9.01|5.04|8001009|amalgunivamalg #1|11|cooking|9|Books|61|oughtcally|N/A|9210354531violet0233|dim|Bundle|Unknown|35|baresecallyeseought| +14641|AAAAAAAABDJDAAAA|1997-10-27||Primary, curious reports feel late of course waste weeks; yellow arts imagine still prices; unpleasant, remote forms differ rather than |2.91|1.36|9007002|brandmaxi #2|7|reference|9|Books|460|barcallyese|N/A|9265314566763misty54|khaki|Bunch|Unknown|32|oughtesecallyeseought| +14642|AAAAAAAACDJDAAAA|1997-10-27|2000-10-26|Socially american ways ought to find rich questions. Dynamic, presidential |0.72|0.30|3004001|edu packexporti #1|4|school-uniforms|3|Children|275|antiationable|medium|131252puff9381849922|moccasin|Tbl|Unknown|91|ableesecallyeseought| +14643|AAAAAAAACDJDAAAA|2000-10-27||Clear women will strengthen almost subjects. Sections deal most yet total features; plainly average minutes wish. Societies will leave m|4.54|2.49|3004001|amalgedu pack #2|1|womens|4|Shoes|275|antiationable|small|131252puff9381849922|smoke|Bunch|Unknown|53|priesecallyeseought| +14644|AAAAAAAAEDJDAAAA|1997-10-27|1999-10-27|Plants prevent drivers. Then golden eyebrows could continue now severe other students. Exact|5.04|4.28|4001001|amalgedu pack #1|1|womens|4|Shoes|585|antieinganti|large|901945013pink7180498|royal|Box|Unknown|9|eseesecallyeseought| +14645|AAAAAAAAEDJDAAAA|1999-10-28|2001-10-26|Warm circumstances warrant sudden comments. S|5.04|3.88|4003002|exportiedu pack #2|3|kids|4|Shoes|585|antieinganti|large|901945013pink7180498|antique|Box|Unknown|13|antiesecallyeseought| +14646|AAAAAAAAEDJDAAAA|2001-10-27||Warm circumstances warrant sudden comments. S|1.05|3.88|4003002|importomaxi #7|3|business|9|Books|522|ableableanti|N/A|66422815pale44356481|ivory|Gross|Unknown|3|callyesecallyeseought| +14647|AAAAAAAAHDJDAAAA|1997-10-27||Even irrelevant acres like very through a readers. Already concerned ministers shrink please. Evident findings used to eat about unique|88.04|51.94|9008008|namelessmaxi #8|8|romance|9|Books|1|ought|N/A|668peru4844414967772|tan|Dozen|Unknown|47|ationesecallyeseought| +14648|AAAAAAAAIDJDAAAA|1997-10-27|2000-10-26|Much angry clothes need practically muscles. As appropriate author|7.99|6.79|9014009|edu packunivamalg #9|14|sports|9|Books|221|oughtableable|N/A|4246471frosted350461|pink|Pound|Unknown|29|eingesecallyeseought| +14649|AAAAAAAAIDJDAAAA|2000-10-27||Much angry clothes need practically muscles. As appropriate author|9.93|6.79|9014009|importoimporto #2|2|shirts|2|Men|450|oughtableable|small|4246471frosted350461|violet|Pound|Unknown|65|n stesecallyeseought| +14650|AAAAAAAAKDJDAAAA|1997-10-27|1999-10-27|Separate, possible products hope furthermore poor neighbours. English visitors visit directly daily supplies. Outdoor, traditional clothes use things. Small effects shoot for the shops. |6.00|3.00|3004001|edu packexporti #1|4|school-uniforms|3|Children|583|prieinganti|medium|189147powder55446013|turquoise|Each|Unknown|14|baranticallyeseought| +14651|AAAAAAAAKDJDAAAA|1999-10-28|2001-10-26|For example particular relations cannot sneak fun, opposite schools. Social, irish rules compa|3.47|1.45|3004001|amalgimporto #2|1|accessories|2|Men|583|prieinganti|large|189147powder55446013|yellow|Pallet|Unknown|36|oughtanticallyeseought| +14652|AAAAAAAAKDJDAAAA|2001-10-27||For example particular relations cannot sneak fun, opposite schools. Social, irish rules compa|9.30|7.25|3001001|amalgexporti #1|1|newborn|3|Children|524|eseableanti|small|189147powder55446013|sky|Carton|Unknown|20|ableanticallyeseought| +14653|AAAAAAAANDJDAAAA|1997-10-27||Kind plants may keep so. Representatives evolve at a men. British steps see from an costs. Military, large years s|7.67|5.75|5004002|edu packscholar #2|4|classical|5|Music|982|ableeingn st|N/A|105423firebrick19574|papaya|Bundle|Unknown|12|prianticallyeseought| +14654|AAAAAAAAODJDAAAA|1997-10-27|2000-10-26|Also public waves will not bomb yesterday methods. Eyes |1.43|1.22|4001001|amalgedu pack #1|1|womens|4|Shoes|599|n stn stanti|petite|94030307392lace14211|beige|Bunch|Unknown|8|eseanticallyeseought| +14655|AAAAAAAAODJDAAAA|2000-10-27||Also public waves will not bomb yesterday methods. Eyes |6.15|1.22|2002002|importoimporto #2|2|shirts|2|Men|599|n stn stanti|large|336832575837papaya20|peach|Ton|Unknown|90|antianticallyeseought| +14656|AAAAAAAAAEJDAAAA|1997-10-27|1999-10-27|Parameters try very into a copies. Good levels take victorian developments. Important, financial views may not recall ill simple numb|7.24|5.06|4002001|importoedu pack #1|2|mens|4|Shoes|428|eingableese|medium|05074tan671486642579|red|Dozen|Unknown|16|callyanticallyeseought| +14657|AAAAAAAAAEJDAAAA|1999-10-28|2001-10-26|Parameters try very into a copies. Good levels take victorian developments. Important, financial views may not recall ill simple numb|8.56|4.36|4002001|amalgimporto #2|2|accessories|2|Men|928|eingablen st|large|05074tan671486642579|ghost|Carton|Unknown|86|ationanticallyeseought| +14658|AAAAAAAAAEJDAAAA|2001-10-27||Parameters try very into a copies. Good levels take victorian developments. Important, financial views may not recall ill simple numb|2.44|4.36|1003001|exportiamalg #1|2|maternity|1|Women|290|barn stable|petite|05074tan671486642579|white|Ton|Unknown|94|einganticallyeseought| +14659|AAAAAAAADEJDAAAA|1997-10-27||Unexpected days could change ideally. Years learn widespread, integrated variables. Richly general kinds should assure please silver new taxes. Bad universities attract no|1.05|0.78|5001002|amalgscholar #2|1|rock|5|Music|552|ableantianti|N/A|8155445337salmon6318|wheat|Ounce|Unknown|25|n stanticallyeseought| +14660|AAAAAAAAEEJDAAAA|1997-10-27|2000-10-26|Annual, french authorities safeguard more german, random moments. Quick references feel; colleges|4.22|1.30|8011001|amalgmaxi #1|11|archery|8|Sports|247|ationeseable|N/A|00979334smoke4652844|royal|Bundle|Unknown|32|barcallycallyeseought| +14661|AAAAAAAAEEJDAAAA|2000-10-27||Annual, french authorities safeguard more german, random moments. Quick references feel; colleges|2.33|1.21|8011001|corpmaxi #2|11|golf|8|Sports|247|ationeseable|N/A|00979334smoke4652844|papaya|Box|Unknown|21|oughtcallycallyeseought| +14662|AAAAAAAAGEJDAAAA|1997-10-27|1999-10-27|Top, careful legs work at a women. Only possible errors pay very other companies. Unique kids write alre|1.02|0.77|2004001|edu packimporto #1|4|sports-apparel|2|Men|243|prieseable|petite|06442809910puff71211|spring|Carton|Unknown|28|ablecallycallyeseought| +14663|AAAAAAAAGEJDAAAA|1999-10-28|2001-10-26|Patiently short decisions shall emphasize too labour, friendly words; lad|1.28|0.77|1001002|amalgamalg #2|1|dresses|1|Women|243|prieseable|petite|06442809910puff71211|purple|Tsp|Unknown|67|pricallycallyeseought| +14664|AAAAAAAAGEJDAAAA|2001-10-27||Local states stay industries. Daughters help in part substantial, difficult tanks. Too different metres read then from a areas. Economic, considerable institutions |1.52|0.77|1001002|importoamalg #1|2|fragrances|1|Women|243|prieseable|extra large|399slate490025944479|goldenrod|Pound|Unknown|37|esecallycallyeseought| +14665|AAAAAAAAJEJDAAAA|1997-10-27||Mixed levels want in order changes; victorian, upper bones accept legal, national winners. Often wrong points appreciate freely international companies. Systems say easier perfect eyes; mi|3.93|1.53|4002002|importoedu pack #2|2|mens|4|Shoes|32|ablepri|large|seashell373646594172|rosy|N/A|Unknown|31|anticallycallyeseought| +14666|AAAAAAAAKEJDAAAA|1997-10-27|2000-10-26|In particular small principles reach with the rights; rows should look effective, available words. Northern, thin lists may see more liberal elections. Too necessary figu|5.99|5.03|7008007|namelessbrand #7|8|lighting|7|Home|630|barprically|N/A|07slate4017027146913|sky|Carton|Unknown|4|callycallycallyeseought| +14667|AAAAAAAAKEJDAAAA|2000-10-27||Terms look available years. Natural, small sales know more perhaps original delegates. Applications should see sometimes more part-time ways; industrial factors work as for a days. Hence med|3.54|5.03|7008007|namelessmaxi #4|8|romance|9|Books|630|barprically|N/A|74700696811951lawn93|puff|Tsp|Unknown|25|ationcallycallyeseought| +14668|AAAAAAAAMEJDAAAA|1997-10-27|1999-10-27|Popular, commercial benefits edit both results. Exactly suffi|7.19|3.59|4004001|edu packedu pack #1|4|athletic|4|Shoes|16|callyought|economy|032710416168tomato57|goldenrod|Gross|Unknown|33|eingcallycallyeseought| +14669|AAAAAAAAMEJDAAAA|1999-10-28|2001-10-26|For example front years must see however too britis|3.48|2.33|4004001|edu packedu pack #2|4|athletic|4|Shoes|16|callyought|medium|032710416168tomato57|peru|Dram|Unknown|40|n stcallycallyeseought| +14670|AAAAAAAAMEJDAAAA|2001-10-27||For example front years must see however too britis|8.21|2.33|6002007|importocorp #7|4|diamonds|6|Jewelry|16|callyought|N/A|032710416168tomato57|sky|Ton|Unknown|11|barationcallyeseought| +14671|AAAAAAAAPEJDAAAA|1997-10-27||Patients encourage children. New aspects must win enough legal ways. Impressive, progressive aims shall not live even with the units. Moments lose anyway with a schools. Very foundations say free, re|5.97|4.23|5002002|importoscholar #2|2|country|5|Music|398|eingn stpri|N/A|27574153tan915937665|white|N/A|Unknown|28|oughtationcallyeseought| +14672|AAAAAAAAAFJDAAAA|1997-10-27|2000-10-26|Prob|3.33|1.13|8005003|scholarnameless #3|5|fitness|8|Sports|251|oughtantiable|N/A|4971595701493lemon76|lace|Tsp|Unknown|30|ableationcallyeseought| +14673|AAAAAAAAAFJDAAAA|2000-10-27||Prob|3.25|1.13|3004002|edu packexporti #2|4|school-uniforms|3|Children|251|oughtantiable|extra large|4971595701493lemon76|wheat|Tsp|Unknown|43|priationcallyeseought| +14674|AAAAAAAACFJDAAAA|1997-10-27|1999-10-27|Severely main republics refer areas. Friends mind only tasks. Important, subsequent differences face offences. Easy only quarters sup|3.64|2.32|10013004|exportiamalgamalg #4|13|stereo|10|Electronics|429|n stableese|N/A|9690peach46560430805|maroon|Tbl|Unknown|17|eseationcallyeseought| +14675|AAAAAAAACFJDAAAA|1999-10-28|2001-10-26|Severely main republics refer areas. Friends mind only tasks. Important, subsequent differences face offences. Easy only quarters sup|0.37|2.32|5002002|importoscholar #2|13|country|5|Music|429|n stableese|N/A|42276orchid198184440|yellow|Oz|Unknown|3|antiationcallyeseought| +14676|AAAAAAAACFJDAAAA|2001-10-27||Severely main republics refer areas. Friends mind only tasks. Important, subsequent differences face offences. Easy only quarters sup|2.96|2.07|7002007|importobrand #7|13|bedding|7|Home|19|n stought|N/A|42276orchid198184440|papaya|Tbl|Unknown|69|callyationcallyeseought| +14677|AAAAAAAAFFJDAAAA|1997-10-27||Other games will not deal necessarily turkish chemicals. Homes provide with a tru|4.27|2.43|5004002|edu packscholar #2|4|classical|5|Music|427|ationableese|N/A|96160tomato396096203|midnight|N/A|Unknown|8|ationationcallyeseought| +14678|AAAAAAAAGFJDAAAA|1997-10-27|2000-10-26|Of course potential patients should lose for the universities. Thousands must allow there for a phrases; different students used to deal. Too cautious rates manage various divisions. Facilities regard|2.85|1.22|2004001|edu packimporto #1|4|sports-apparel|2|Men|866|callycallyeing|small|87218028sky376413642|smoke|Ounce|Unknown|50|eingationcallyeseought| +14679|AAAAAAAAGFJDAAAA|2000-10-27||Effective cars should not get recently dear, intere|4.14|2.02|4003002|exportiedu pack #2|3|kids|4|Shoes|277|ationationable|petite|87218028sky376413642|peach|Case|Unknown|63|n stationcallyeseought| +14680|AAAAAAAAIFJDAAAA|1997-10-27|1999-10-27|Then unaware years ought to fetch|2.42|1.52|10012010|importoamalgamalg #10|12|monitors|10|Electronics|64|esecally|N/A|15powder972777026054|olive|Cup|Unknown|22|bareingcallyeseought| +14681|AAAAAAAAIFJDAAAA|1999-10-28|2001-10-26|Then unaware years ought to fetch|26.07|13.29|3004002|edu packexporti #2|4|school-uniforms|3|Children|443|esecally|medium|15powder972777026054|purple|Ton|Unknown|1|oughteingcallyeseought| +14682|AAAAAAAAIFJDAAAA|2001-10-27||Already dark songs might encourage too more limited women; real, able wings know suddenly. Recent writers used to change lives. Inevitably difficult claims ought to ensure warmly differe|1.20|13.29|6005003|scholarcorp #3|5|earings|6|Jewelry|558|esecally|N/A|65206908522100snow48|powder|Case|Unknown|3|ableeingcallyeseought| +14683|AAAAAAAALFJDAAAA|1997-10-27||Friendly offices feel. Delightful servants give almost previously natural earnings. Written, important books press subject, american parents. New, reduced days shall n|0.40|0.14|8014004|edu packmaxi #4|14|tennis|8|Sports|72|ableation|N/A|59377midnight1646335|white|Pallet|Unknown|75|prieingcallyeseought| +14684|AAAAAAAAMFJDAAAA|1997-10-27|2000-10-26|Local, great tories wear rather than the losses. Natural, necessary visitors would seem joint, urban|8.44|6.75|10016014|corpamalgamalg #14|16|wireless|10|Electronics|598|eingn stanti|N/A|200530984934papaya23|wheat|Tbl|Unknown|4|eseeingcallyeseought| +14685|AAAAAAAAMFJDAAAA|2000-10-27||Local, great tories wear rather than the losses. Natural, necessary visitors would seem joint, urban|38.92|6.75|10016014|scholaramalgamalg #15|16|portable|10|Electronics|598|eingn stanti|N/A|89135405138038pink79|magenta|Carton|Unknown|68|antieingcallyeseought| +14686|AAAAAAAAOFJDAAAA|1997-10-27|1999-10-27|Very national windows suggest for the|3.82|2.02|2002001|importoimporto #1|2|shirts|2|Men|595|antin stanti|extra large|851562735indian58195|firebrick|Lb|Unknown|50|callyeingcallyeseought| +14687|AAAAAAAAOFJDAAAA|1999-10-28|2001-10-26|Very national windows suggest for the|9.49|2.02|10013017|exportiamalgamalg #17|13|stereo|10|Electronics|696|callyn stcally|N/A|851562735indian58195|snow|Each|Unknown|19|ationeingcallyeseought| +14688|AAAAAAAAOFJDAAAA|2001-10-27||Comparisons should enter trous|1.17|2.02|3003001|exportiexporti #1|3|toddlers|3|Children|439|callyn stcally|medium|438733588773sandy261|brown|Tbl|Unknown|9|eingeingcallyeseought| +14689|AAAAAAAABGJDAAAA|1997-10-27||Important, frequent councils explore general, local ideas. Representatives last more. Foreign, sensible pupils pay. Social, american reservations used to get so much|0.59|0.25|6011002|amalgbrand #2|11|semi-precious|6|Jewelry|12|ableought|N/A|8515gainsboro5534435|floral|Tsp|Unknown|62|n steingcallyeseought| +14690|AAAAAAAACGJDAAAA|1997-10-27|2000-10-26|Students see now options; so lucky incentives should not refer there large, great pupils; adequate police move |1.75|1.40|5002001|importoscholar #1|2|country|5|Music|364|esecallypri|N/A|57958saddle351796564|peach|Oz|Unknown|35|barn stcallyeseought| +14691|AAAAAAAACGJDAAAA|2000-10-27||Likely systems might provide involved tourists. Members cannot say rules. Other documents enhance; really unusual women would not drive thus public new neighbours. External benefits get rather thus a|7.44|1.40|6015004|scholarbrand #4|2|custom|6|Jewelry|364|esecallypri|N/A|57958saddle351796564|purple|Oz|Unknown|66|oughtn stcallyeseought| +14692|AAAAAAAAEGJDAAAA|1997-10-27|1999-10-27|German, simple feet r|3.62|1.23|10013005|exportiamalgamalg #5|13|stereo|10|Electronics|18|eingought|N/A|62802045082thistle22|sky|Bundle|Unknown|97|ablen stcallyeseought| +14693|AAAAAAAAEGJDAAAA|1999-10-28|2001-10-26|German, simple feet r|0.35|1.23|10013005|exportiexporti #2|3|toddlers|3|Children|18|eingought|medium|62802045082thistle22|rosy|Gram|Unknown|55|prin stcallyeseought| +14694|AAAAAAAAEGJDAAAA|2001-10-27||Animals might ratify longer long stores. Ligh|5.64|1.23|3001001|amalgexporti #1|1|newborn|3|Children|18|eingought|medium|13380239240381rose36|rose|Lb|Unknown|7|esen stcallyeseought| +14695|AAAAAAAAHGJDAAAA|1997-10-27||Really classical sources try fully farmers. Even high managers leave again labour, |0.45|0.27|10013008|exportiamalgamalg #8|13|stereo|10|Electronics|884|eseeingeing|N/A|7066814slate82444084|goldenrod|Bunch|Unknown|33|antin stcallyeseought| +14696|AAAAAAAAIGJDAAAA|1997-10-27|2000-10-26|Correspondingly vast ideas approve. Final, f|1.92|1.57|1004001|edu packamalg #1|4|swimwear|1|Women|108|eingbarought|medium|2793559azure25162223|peru|Pallet|Unknown|7|callyn stcallyeseought| +14697|AAAAAAAAIGJDAAAA|2000-10-27||Correspondingly vast ideas approve. Final, f|3.46|2.14|5004002|edu packscholar #2|4|classical|5|Music|206|callybarable|N/A|3pink995403185260367|puff|Each|Unknown|25|ationn stcallyeseought| +14698|AAAAAAAAKGJDAAAA|1997-10-27|1999-10-27|Contracts explain so possible, basic rooms; problems can think then|4.07|1.50|7002009|importobrand #9|2|bedding|7|Home|703|pribaration|N/A|2peach94745700248607|steel|Dram|Unknown|38|eingn stcallyeseought| +14699|AAAAAAAAKGJDAAAA|1999-10-28|2001-10-26|Young, small generations manage now national, new sectors. Regular, simple prepa|5.25|1.50|8008010|namelessnameless #10|8|outdoor|8|Sports|150|barantiought|N/A|9797798547white54555|magenta|Gram|Unknown|66|n stn stcallyeseought| +14700|AAAAAAAAKGJDAAAA|2001-10-27||Imaginative, large characteristics go able instructions. Anywhere p|2.65|1.50|8008010|exportiamalgamalg #13|13|stereo|10|Electronics|1|barantiought|N/A|39rose79535983962010|almond|Ounce|Unknown|17|barbarationeseought| +14701|AAAAAAAANGJDAAAA|1997-10-27||Big, cl|4.96|1.83|6007006|brandcorp #6|7|pendants|6|Jewelry|250|barantiable|N/A|6941indian8869420080|puff|Cup|Unknown|25|oughtbarationeseought| +14702|AAAAAAAAOGJDAAAA|1997-10-27|2000-10-26|Statements may obscure too available clothes. More jewish premises think alongside that close courts; complete soldiers chec|3.08|2.34|2001001|amalgimporto #1|1|accessories|2|Men|29|n stable|large|26683202763red643333|violet|Box|Unknown|5|ablebarationeseought| +14703|AAAAAAAAOGJDAAAA|2000-10-27||Other tensions used to impress upon a children. Simply canadian countr|7.43|2.34|2001001|edu packbrand #4|4|curtains/drapes|7|Home|719|n stoughtation|N/A|26683202763red643333|salmon|Pallet|Unknown|38|pribarationeseought| +14704|AAAAAAAAAHJDAAAA|1997-10-27|1999-10-27|Old sales should ensure always. Simple arrangements learn|2.15|1.48|5002001|importoscholar #1|2|country|5|Music|609|n stbarcally|N/A|4misty88952751836988|rose|Pallet|Unknown|6|esebarationeseought| +14705|AAAAAAAAAHJDAAAA|1999-10-28|2001-10-26|Old sales should ensure always. Simple arrangements learn|6.98|3.35|3003002|exportiexporti #2|2|toddlers|3|Children|609|n stbarcally|medium|red04418843914827310|salmon|N/A|Unknown|17|antibarationeseought| +14706|AAAAAAAAAHJDAAAA|2001-10-27||Old sales should ensure always. Simple arrangements learn|4.17|3.35|5003001|exportischolar #1|2|pop|5|Music|609|n stbarcally|N/A|red04418843914827310|indian|Pound|Unknown|1|callybarationeseought| +14707|AAAAAAAADHJDAAAA|1997-10-27||Public hands might not|2.74|2.38|9004008|edu packmaxi #8|4|entertainments|9|Books|34|esepri|N/A|49989879558thistle09|lemon|N/A|Unknown|28|ationbarationeseought| +14708|AAAAAAAAEHJDAAAA|1997-10-27|2000-10-26|Everyday kinds could not use briefly electrical mate|7.55|2.79|6013005|exportibrand #5|13|loose stones|6|Jewelry|291|oughtn stable|N/A|649rosy3029949409164|salmon|Ton|Unknown|35|eingbarationeseought| +14709|AAAAAAAAEHJDAAAA|2000-10-27||Everyday kinds could not use briefly electrical mate|8.30|3.40|7008010|namelessbrand #10|8|lighting|7|Home|291|oughtn stable|N/A|649rosy3029949409164|turquoise|Gross|Unknown|44|n stbarationeseought| +14710|AAAAAAAAGHJDAAAA|1997-10-27|1999-10-27|Really blue talks will manage past. Excellent, social systems shall not speak via a groups. Human, intern|3.12|1.62|1002001|importoamalg #1|2|fragrances|1|Women|263|pricallyable|medium|811rosy7096551829955|white|Ounce|Unknown|23|baroughtationeseought| +14711|AAAAAAAAGHJDAAAA|1999-10-28|2001-10-26|Really blue talks will manage past. Excellent, social systems shall not speak via a groups. Human, intern|1.63|1.62|1002001|amalgedu pack #2|1|womens|4|Shoes|530|barprianti|medium|288266powder65821644|salmon|Gross|Unknown|46|oughtoughtationeseought| +14712|AAAAAAAAGHJDAAAA|2001-10-27||Urban, responsible bon|0.61|1.62|1002001|amalgimporto #1|1|accessories|2|Men|189|barprianti|medium|288266powder65821644|rose|Unknown|Unknown|12|ableoughtationeseought| +14713|AAAAAAAAJHJDAAAA|1997-10-27||At all silent aspects find properly apart expected trusts. Offices ought to meet also sweet lights. Yesterday environmental factors could doubt very significant f|4.42|3.09|9008002|namelessmaxi #2|8|romance|9|Books|196|callyn stought|N/A|2thistle039634733515|purple|Dozen|Unknown|20|prioughtationeseought| +14714|AAAAAAAAKHJDAAAA|1997-10-27|2000-10-26|Black demonstrations say usually alone days. Chosen, fine skills ought to take other words. Able, other girls might not understand too usual elections. Close artists prevent no lon|8.59|4.46|3001001|amalgexporti #1|1|newborn|3|Children|724|eseableation|medium|0844sky7951368427651|ivory|Each|Unknown|6|eseoughtationeseought| +14715|AAAAAAAAKHJDAAAA|2000-10-27||Main, exciting hours depend due minutes. Schemes could make never. Foods used to make just months. Pretty, historical dogs fit requirements. Certain machines pack carefully. Ye|8.62|4.46|3001001|exportiexporti #2|3|toddlers|3|Children|724|eseableation|large|0844sky7951368427651|midnight|Bunch|Unknown|21|antioughtationeseought| +14716|AAAAAAAAMHJDAAAA|1997-10-27|1999-10-27|Little, gold books used to laugh only from the forests. More external dates would say districts. There stupid contacts may rec|9.70|4.65|3001001|amalgexporti #1|1|newborn|3|Children|134|esepriought|petite|363686683751steel455|puff|Unknown|Unknown|14|callyoughtationeseought| +14717|AAAAAAAAMHJDAAAA|1999-10-28|2001-10-26|Little, gold books used to laugh only from the forests. More external dates would say districts. There stupid contacts may rec|34.72|29.85|9006006|corpmaxi #6|1|parenting|9|Books|134|esepriought|N/A|8449honeydew03283626|navy|Dram|Unknown|35|ationoughtationeseought| +14718|AAAAAAAAMHJDAAAA|2001-10-27||Little, gold books used to laugh only from the forests. More external dates would say districts. There stupid contacts may rec|0.66|0.53|9006006|amalgmaxi #7|1|arts|9|Books|134|esepriought|N/A|8449honeydew03283626|spring|Gross|Unknown|100|eingoughtationeseought| +14719|AAAAAAAAPHJDAAAA|1997-10-27||Local, italian leads argue attractive centuries. Actually gay advertisements shall not facilitate somewhere courses. Strong members would not hear before necessary sites; standards wi|6.26|4.00|2004002|edu packimporto #2|4|sports-apparel|2|Men|451|oughtantiese|small|60817218wheat9154616|powder|Box|Unknown|63|n stoughtationeseought| +14720|AAAAAAAAAIJDAAAA|1997-10-27|2000-10-26|Central, entire generations like poor, indian loans. Gentle, powerful buildings adopt again activities. Married sounds will write in the organizations. Bodies appear to the days. Already bro|4.06|1.46|9016009|corpunivamalg #9|16|mystery|9|Books|301|oughtbarpri|N/A|633097rose6991084994|yellow|Lb|Unknown|28|barableationeseought| +14721|AAAAAAAAAIJDAAAA|2000-10-27||Hotels know. Present mothers take virtually beautiful, difficult procedures. Old cells |1.97|1.46|9016009|univnameless #2|10|flatware|7|Home|487|oughtbarpri|N/A|61552257sandy8835241|sienna|Cup|Unknown|29|oughtableationeseought| +14722|AAAAAAAACIJDAAAA|1997-10-27|1999-10-27|Just powerful cameras may not demonstrate courses. Theories think. Large, political officers should put at all japanese things. Stud|33.90|23.73|10004001|edu packunivamalg #1|4|audio|10|Electronics|240|bareseable|N/A|07peach3621735995022|slate|Carton|Unknown|1|ableableationeseought| +14723|AAAAAAAACIJDAAAA|1999-10-28|2001-10-26|Just gr|9.21|7.09|10004001|amalgexporti #2|1|newborn|3|Children|240|bareseable|medium|07peach3621735995022|lace|Bundle|Unknown|52|priableationeseought| +14724|AAAAAAAACIJDAAAA|2001-10-27||Just gr|5.53|7.09|10004001|edu packmaxi #5|1|tennis|8|Sports|107|ationbarought|N/A|66180rose14602375604|salmon|Tsp|Unknown|1|eseableationeseought| +14725|AAAAAAAAFIJDAAAA|1997-10-27||Away patient threats will not abandon too. Now coming posts look gently essential events. New products get very with the forces. Part-ti|4.75|2.85|6011006|amalgbrand #6|11|semi-precious|6|Jewelry|326|callyablepri|N/A|3249tomato1004775361|rosy|Bundle|Unknown|46|antiableationeseought| +14726|AAAAAAAAGIJDAAAA|1997-10-27|2000-10-26|Small, delicious eyes can conform also. Corresponding levels catch here in a matters. Traditions freeze finally by an beds. Groups spend certa|1.27|0.63|10007005|brandunivamalg #5|7|personal|10|Electronics|266|callycallyable|N/A|3slate52718945235925|papaya|N/A|Unknown|18|callyableationeseought| +14727|AAAAAAAAGIJDAAAA|2000-10-27||Much different products get he|2.05|0.63|4002002|importoedu pack #2|7|mens|4|Shoes|266|callycallyable|extra large|3slate52718945235925|sandy|Oz|Unknown|44|ationableationeseought| +14728|AAAAAAAAIIJDAAAA|1997-10-27|1999-10-27|Regrettably great children would please as. Obvious penalties will not come yet russian leads. Components shall accompany complete, main results. Hard, difficult levels shall |34.62|15.57|5004001|edu packscholar #1|4|classical|5|Music|310|baroughtpri|N/A|7233586472236orange5|red|Dram|Unknown|18|eingableationeseought| +14729|AAAAAAAAIIJDAAAA|1999-10-28|2001-10-26|Medical, voluntary cars would prot|1.26|1.05|5004001|exportiimporto #2|4|pants|2|Men|94|baroughtpri|small|14880powder389662639|tomato|Tsp|Unknown|56|n stableationeseought| +14730|AAAAAAAAIIJDAAAA|2001-10-27||Medical, voluntary cars would prot|4.09|1.55|5004001|amalgimporto #1|4|accessories|2|Men|840|bareseeing|medium|4778rose607235862093|lace|Gram|Unknown|15|barpriationeseought| +14731|AAAAAAAALIJDAAAA|1997-10-27||New |2.35|1.76|9016008|corpunivamalg #8|16|mystery|9|Books|930|barprin st|N/A|90rosy63812543802143|linen|Unknown|Unknown|56|oughtpriationeseought| +14732|AAAAAAAAMIJDAAAA|1997-10-27|2000-10-26|Symptoms will increase others. Legal days ask upper, current developments. Specific, complete purposes might light again |9.27|3.15|4001001|amalgedu pack #1|1|womens|4|Shoes|353|priantipri|medium|526494metallic900255|smoke|Ton|Unknown|5|ablepriationeseought| +14733|AAAAAAAAMIJDAAAA|2000-10-27||Symptoms will increase others. Legal days ask upper, current developments. Specific, complete purposes might light again |3.37|2.66|5004002|edu packscholar #2|4|classical|5|Music|353|priantipri|N/A|8powder6205877215918|sienna|Pallet|Unknown|92|pripriationeseought| +14734|AAAAAAAAOIJDAAAA|1997-10-27|1999-10-27|Women enjoy too single, proper rivers. Often african activities could not escape as. Only|1.11|0.73|5004001|edu packscholar #1|4|classical|5|Music|451|oughtantiese|N/A|royal002175876442517|rose|Ounce|Unknown|28|esepriationeseought| +14735|AAAAAAAAOIJDAAAA|1999-10-28|2001-10-26|Priorities should not know rapidly visual, new services. Strong |0.88|0.74|5004002|edu packscholar #2|4|classical|5|Music|417|oughtantiese|N/A|royal002175876442517|yellow|Ton|Unknown|63|antipriationeseought| +14736|AAAAAAAAOIJDAAAA|2001-10-27||Other, old days cannot come well on the children. Only capitalist views provide awfully foreign processes. Ultimately special reserves must not solve for example for a |32.72|0.74|9004001|edu packmaxi #1|4|entertainments|9|Books|417|oughtantiese|N/A|8081868pale765809764|dodger|Tbl|Unknown|61|callypriationeseought| +14737|AAAAAAAABJJDAAAA|1997-10-27||Final office|86.90|68.65|7014008|edu packnameless #8|14|glassware|7|Home|36|callypri|N/A|6633265684357sky3671|grey|Oz|Unknown|12|ationpriationeseought| +14738|AAAAAAAACJJDAAAA|1997-10-27|2000-10-26|Great, delighted arrangements conceive as; users cook only mostly small chemicals. Social days compare suitably other lines; immediate, quiet letters could not get in a guests. Children participat|4.67|2.10|7006009|corpbrand #9|6|rugs|7|Home|231|oughtpriable|N/A|3753875663337rosy089|salmon|Oz|Unknown|55|eingpriationeseought| +14739|AAAAAAAACJJDAAAA|2000-10-27||Legal police warrant quite wrong, obvious rules. Forms cannot move even references. Late distant spots see truly. Clear weeks get. French|3.38|2.10|7006009|edu packmaxi #4|4|entertainments|9|Books|13|priought|N/A|3753875663337rosy089|tomato|Oz|Unknown|45|n stpriationeseought| +14740|AAAAAAAAEJJDAAAA|1997-10-27|1999-10-27|New, average legs find long effects. Junior principles could cause for ever historical, equal movements; domest|2.31|1.59|7003001|exportibrand #1|3|kids|7|Home|690|barn stcally|N/A|02951732purple773212|peach|Pallet|Unknown|26|bareseationeseought| +14741|AAAAAAAAEJJDAAAA|1999-10-28|2001-10-26|There acceptable applications dominate fair wrong, linear skills. Sufficient babies may not get high trees|1.22|0.48|5003002|exportischolar #2|3|pop|5|Music|690|barn stcally|N/A|02951732purple773212|violet|Dram|Unknown|73|oughteseationeseought| +14742|AAAAAAAAEJJDAAAA|2001-10-27||There acceptable applications dominate fair wrong, linear skills. Sufficient babies may not get high trees|4.04|0.48|5003002|amalgexporti #1|3|newborn|3|Children|115|barn stcally|medium|seashell361539193841|medium|Dozen|Unknown|31|ableeseationeseought| +14743|AAAAAAAAHJJDAAAA|1997-10-27||Increasingly sudden cases would not choose se|4.23|1.60|6007008|brandcorp #8|7|pendants|6|Jewelry|624|eseablecally|N/A|860953699557navy7632|tan|Ton|Unknown|19|prieseationeseought| +14744|AAAAAAAAIJJDAAAA|1997-10-27|2000-10-26|Liberal, |1.39|1.16|8001005|amalgnameless #5|1|athletic shoes|8|Sports|794|esen station|N/A|6yellow9594112950105|seashell|Box|Unknown|30|eseeseationeseought| +14745|AAAAAAAAIJJDAAAA|2000-10-27||Liberal, |2.83|1.16|8011006|amalgmaxi #6|1|archery|8|Sports|796|esen station|N/A|8686salmon2527255117|royal|Gross|Unknown|11|antieseationeseought| +14746|AAAAAAAAKJJDAAAA|1997-10-27|1999-10-27|Modes be|82.36|65.06|4001001|amalgedu pack #1|1|womens|4|Shoes|19|n stought|petite|8552745smoke67893323|plum|Unknown|Unknown|23|callyeseationeseought| +14747|AAAAAAAAKJJDAAAA|1999-10-28|2001-10-26|Full years like mainly voices. Clear, national implications lift; hot, political members conclude reliable activities. French blocks shall not give only top wings. Countries may n|3.79|65.06|6002004|importocorp #4|2|diamonds|6|Jewelry|19|n stought|N/A|8552745smoke67893323|sienna|Lb|Unknown|1|ationeseationeseought| +14748|AAAAAAAAKJJDAAAA|2001-10-27||Full years like mainly voices. Clear, national implications lift; hot, political members conclude reliable activities. French blocks shall not give only top wings. Countries may n|1.02|0.48|6002004|edu packscholar #1|4|classical|5|Music|705|n stought|N/A|8552745smoke67893323|sienna|Dram|Unknown|9|eingeseationeseought| +14749|AAAAAAAANJJDAAAA|1997-10-27||Roman futures define disproportionately aware issues. Fundamental documents will discover cultural ships. Strong prisoners tell therefore magic aspects. National, specific sites|1.79|1.44|5003002|exportischolar #2|3|pop|5|Music|54|eseanti|N/A|775591069856steel382|tomato|Oz|Unknown|44|n steseationeseought| +14750|AAAAAAAAOJJDAAAA|1997-10-27|2000-10-26|Protective appearances call then new, long-ter|1.26|0.63|8008001|namelessnameless #1|8|outdoor|8|Sports|33|pripri|N/A|2steel29840988544771|seashell|Pound|Unknown|12|barantiationeseought| +14751|AAAAAAAAOJJDAAAA|2000-10-27||Western charges used|4.75|0.63|5003002|exportischolar #2|3|pop|5|Music|91|pripri|N/A|2steel29840988544771|snow|Dozen|Unknown|57|oughtantiationeseought| +14752|AAAAAAAAAKJDAAAA|1997-10-27|1999-10-27|Strong, mass owners would upset followers. All vital colleagues shall remember whole police. Alive, horrible explanations should not earn. Then social |0.98|0.66|8007003|brandnameless #3|7|hockey|8|Sports|348|eingesepri|N/A|716436099wheat591698|peru|Gross|Unknown|87|ableantiationeseought| +14753|AAAAAAAAAKJDAAAA|1999-10-28|2001-10-26|Complicated, empty relations could not stay new points. Major, strong recordings might not secure arms. Sensitive metres must not follow as old miles. However aware schools earn fres|0.84|0.37|7004004|edu packbrand #4|4|curtains/drapes|7|Home|348|eingesepri|N/A|716436099wheat591698|tan|Gram|Unknown|66|priantiationeseought| +14754|AAAAAAAAAKJDAAAA|2001-10-27||Complicated, empty relations could not stay new points. Major, strong recordings might not secure arms. Sensitive metres must not follow as old miles. However aware schools earn fres|9.73|8.17|2001001|amalgimporto #1|1|accessories|2|Men|50|eingesepri|petite|716436099wheat591698|smoke|Pallet|Unknown|5|eseantiationeseought| +14755|AAAAAAAADKJDAAAA|1997-10-27||Neither nice aspects will express contrary, old sets. For example financial problems will attract roughly; subsequently early relationships ought to wait o|7.85|4.94|7014006|edu packnameless #6|14|glassware|7|Home|151|oughtantiought|N/A|157338violet06490454|steel|Case|Unknown|46|antiantiationeseought| +14756|AAAAAAAAEKJDAAAA|1997-10-27|2000-10-26|Positive, collective alternatives forget hastily. Forces establish so in a systems. Just public mon|3.62|2.02|2004001|edu packimporto #1|4|sports-apparel|2|Men|46|callyese|medium|578991997108spring15|smoke|Each|Unknown|87|callyantiationeseought| +14757|AAAAAAAAEKJDAAAA|2000-10-27||Trying, great words cope eventually per annum vital tons. Pupils take now prospects; enough |0.76|0.47|2004001|amalgimporto #2|1|accessories|2|Men|121|oughtableought|extra large|24841midnight1860195|sienna|Tbl|Unknown|30|ationantiationeseought| +14758|AAAAAAAAGKJDAAAA|1997-10-27|1999-10-27|Loose presidential days would appreciate only ways. Stations might g|16.89|13.17|8004009|edu packnameless #9|4|camping|8|Sports|285|antieingable|N/A|0khaki05697104924413|wheat|Gram|Unknown|1|eingantiationeseought| +14759|AAAAAAAAGKJDAAAA|1999-10-28|2001-10-26|Loose presidential days would appreciate only ways. Stations might g|0.22|13.17|10016004|corpamalgamalg #4|4|wireless|10|Electronics|44|antieingable|N/A|6477tan3386466480341|slate|Pallet|Unknown|30|n stantiationeseought| +14760|AAAAAAAAGKJDAAAA|2001-10-27||Loose presidential days would appreciate only ways. Stations might g|2.74|2.35|1004001|edu packamalg #1|4|swimwear|1|Women|44|antieingable|small|6477tan3386466480341|yellow|Cup|Unknown|6|barcallyationeseought| +14761|AAAAAAAAJKJDAAAA|1997-10-27||Systematic, dreadful frames need just substances. Movements will become afterwards. Special, blue papers |1.73|1.53|3001002|amalgexporti #2|1|newborn|3|Children|307|ationbarpri|extra large|6633828pale753948739|brown|Tsp|Unknown|32|oughtcallyationeseought| +14762|AAAAAAAAKKJDAAAA|1997-10-27|2000-10-26|At least other consequences get true islands; writers should go in a rocks. Modest, |0.56|0.29|4001001|amalgedu pack #1|1|womens|4|Shoes|32|ablepri|medium|25371088359pale12341|rose|Cup|Unknown|76|ablecallyationeseought| +14763|AAAAAAAAKKJDAAAA|2000-10-27||Clear degrees eat good, possible areas. Years might settle furthermore efforts. Multiple developers can find characte|7.25|0.29|5002002|importoscholar #2|1|country|5|Music|782|ablepri|N/A|25371088359pale12341|sandy|Unknown|Unknown|52|pricallyationeseought| +14764|AAAAAAAAMKJDAAAA|1997-10-27|1999-10-27|Words must analyse women. Sorry, british eyes prevent now top types. Secret allegations sort personal, inland investors. Primarily international forces |1.65|1.25|4003001|exportiedu pack #1|3|kids|4|Shoes|33|pripri|extra large|5478220722537royal11|lime|Box|Unknown|27|esecallyationeseought| +14765|AAAAAAAAMKJDAAAA|1999-10-28|2001-10-26|Well likely year|1.20|0.37|1001002|amalgamalg #2|1|dresses|1|Women|175|antiationought|large|157567284649magenta2|peru|Tsp|Unknown|9|anticallyationeseought| +14766|AAAAAAAAMKJDAAAA|2001-10-27||Well likely year|3.97|3.17|1001001|amalgamalg #1|1|dresses|1|Women|175|antiationought|petite|757thistle9486271046|pink|Carton|Unknown|19|callycallyationeseought| +14767|AAAAAAAAPKJDAAAA|1997-10-27||English communications meet crucial, likely appeara|9.72|6.02|3004002|edu packexporti #2|4|school-uniforms|3|Children|642|ableesecally|N/A|7869848264790pink298|brown|Each|Unknown|56|ationcallyationeseought| +14768|AAAAAAAAALJDAAAA|1997-10-27|2000-10-26|Ingredients might make international, effective s|3.12|1.84|5004001|edu packscholar #1|4|classical|5|Music|527|ationableanti|N/A|532230303540rose1948|smoke|Bundle|Unknown|24|eingcallyationeseought| +14769|AAAAAAAAALJDAAAA|2000-10-27||Late, ce|7.33|1.84|3001002|amalgexporti #2|1|newborn|3|Children|152|ableantiought|extra large|532230303540rose1948|white|Ton|Unknown|45|n stcallyationeseought| +14770|AAAAAAAACLJDAAAA|1997-10-27|1999-10-27|Techniques sense; times blame by the hands. Much scottish executives would need powerful years. Growing hotels shall take meanwhi|3.09|2.41|7014001|edu packnameless #1|14|glassware|7|Home|804|esebareing|N/A|69299912583saddle285|sienna|Cup|Unknown|11|barationationeseought| +14771|AAAAAAAACLJDAAAA|1999-10-28|2001-10-26|Full corners avoid both other masses. However busy prices read about primary, possible others. Digital workers put up a|9.77|2.41|1001002|amalgamalg #2|1|dresses|1|Women|860|barcallyeing|small|1748554890889peru856|pink|Lb|Unknown|25|oughtationationeseought| +14772|AAAAAAAACLJDAAAA|2001-10-27||Dependent years could not attract actually far huge matters. Years might spot again across whole heads. Economic, personal visitors reflect only normally strong cars. Just new writers use|5.41|2.41|3004001|edu packexporti #1|1|school-uniforms|3|Children|860|barcallyeing|small|1748554890889peru856|brown|Carton|Unknown|26|ableationationeseought| +14773|AAAAAAAAFLJDAAAA|1997-10-27||Countries coul|4.07|3.05|6011006|amalgbrand #6|11|semi-precious|6|Jewelry|229|n stableable|N/A|3396steel41473509949|pale|Pound|Unknown|68|priationationeseought| +14774|AAAAAAAAGLJDAAAA|1997-10-27|2000-10-26|Extra, wooden offices must see please. Masters go shots; powerful degrees should ensure above other, unite|4.69|3.56|10003002|exportiunivamalg #2|3|dvd/vcr players|10|Electronics|83|prieing|N/A|188aquamarine7316646|smoke|Bunch|Unknown|11|eseationationeseought| +14775|AAAAAAAAGLJDAAAA|2000-10-27||Technical vehicles s|3.16|2.68|10003002|edu packscholar #2|3|classical|5|Music|620|barablecally|N/A|8471433274418slate44|slate|Unknown|Unknown|3|antiationationeseought| +14776|AAAAAAAAILJDAAAA|1997-10-27|1999-10-27|Black relations used to build cruelly american rules. Various women test in a countries. Minor families obtain quite with a changes. Far levels should |4.46|1.78|1003001|exportiamalg #1|3|maternity|1|Women|649|n stesecally|large|800506776110160snow4|ghost|Ton|Unknown|20|callyationationeseought| +14777|AAAAAAAAILJDAAAA|1999-10-28|2001-10-26|Local areas may not fall well. Students could avoid over much unable studies. Independent cases would occur less for the cameras. Bad, central|7.89|3.55|5004002|edu packscholar #2|4|classical|5|Music|649|n stesecally|N/A|800506776110160snow4|steel|N/A|Unknown|11|ationationationeseought| +14778|AAAAAAAAILJDAAAA|2001-10-27||Notions can cut as. Lips protect t|5.97|4.59|5004002|importounivamalg #7|4|home repair|9|Books|649|n stesecally|N/A|800506776110160snow4|sandy|Bundle|Unknown|26|eingationationeseought| +14779|AAAAAAAALLJDAAAA|1997-10-27||Parts say expected, political buses. Actions might mean. Long, gene|8.38|6.03|3003002|exportiexporti #2|3|toddlers|3|Children|861|oughtcallyeing|medium|9612112688linen27012|papaya|N/A|Unknown|19|n stationationeseought| +14780|AAAAAAAAMLJDAAAA|1997-10-27|2000-10-26|Fresh beaches can see days. Good, romantic rules might appear later men. Then like sites may find so here great authorities. Cold days will find su|9.88|3.85|2002001|importoimporto #1|2|shirts|2|Men|277|ationationable|extra large|3837424678mint477819|drab|Gram|Unknown|57|bareingationeseought| +14781|AAAAAAAAMLJDAAAA|2000-10-27||Fresh beaches can see days. Good, romantic rules might appear later men. Then like sites may find so here great authorities. Cold days will find su|3.48|2.43|2002001|edu packbrand #2|4|curtains/drapes|7|Home|277|ationationable|N/A|01253sky781805037665|white|Ounce|Unknown|1|oughteingationeseought| +14782|AAAAAAAAOLJDAAAA|1997-10-27|1999-10-27|British, inc groups play a little certainly ver|5.33|2.66|1001001|amalgamalg #1|1|dresses|1|Women|25|antiable|petite|440730551214red32583|salmon|Unknown|Unknown|9|ableeingationeseought| +14783|AAAAAAAAOLJDAAAA|1999-10-28|2001-10-26|Male, regular parties get photographs. Grand spaces mind defensive, goo|3.78|2.66|1001001|exportimaxi #12|1|computers|9|Books|25|antiable|N/A|440730551214red32583|slate|Dram|Unknown|58|prieingationeseought| +14784|AAAAAAAAOLJDAAAA|2001-10-27||Male, regular parties get photographs. Grand spaces mind defensive, goo|0.78|0.66|9010001|univunivamalg #1|10|travel|9|Books|146|callyeseought|N/A|4945024699salmon7286|brown|Carton|Unknown|26|eseeingationeseought| +14785|AAAAAAAABMJDAAAA|1997-10-27||Lines shall talk usually blue, vague cards. Popular years increa|59.09|34.86|9005002|scholarmaxi #2|5|history|9|Books|379|n stationpri|N/A|71265turquoise770420|sky|Bundle|Unknown|5|antieingationeseought| +14786|AAAAAAAACMJDAAAA|1997-10-27|2000-10-26|Circumstances match very international proposals. He|8.65|7.09|6014003|edu packbrand #3|14|estate|6|Jewelry|241|oughteseable|N/A|1708thistle154678460|tomato|Ounce|Unknown|22|callyeingationeseought| +14787|AAAAAAAACMJDAAAA|2000-10-27||Often elderly laws share fully bodies. Institutions investi|4.74|7.09|5004002|edu packscholar #2|14|classical|5|Music|241|oughteseable|N/A|1708thistle154678460|slate|Cup|Unknown|21|ationeingationeseought| +14788|AAAAAAAAEMJDAAAA|1997-10-27|1999-10-27|Christian considerations shall not find actual, personal hands. Tonnes used to make even old, available errors; jobs shall take. Other approaches inspect double, full communities. Finan|6.08|2.61|5001001|amalgscholar #1|1|rock|5|Music|395|antin stpri|N/A|595linen015320017446|saddle|Dozen|Unknown|93|eingeingationeseought| +14789|AAAAAAAAEMJDAAAA|1999-10-28|2001-10-26|Christian considerations shall not find actual, personal hands. Tonnes used to make even old, available errors; jobs shall take. Other approaches inspect double, full communities. Finan|1.54|0.56|6005006|scholarcorp #6|5|earings|6|Jewelry|55|antianti|N/A|82230686242420drab14|tan|Bunch|Unknown|55|n steingationeseought| +14790|AAAAAAAAEMJDAAAA|2001-10-27||Christian considerations shall not find actual, personal hands. Tonnes used to make even old, available errors; jobs shall take. Other approaches inspect double, full communities. Finan|3.80|2.77|7006001|corpbrand #1|6|rugs|7|Home|180|bareingought|N/A|82230686242420drab14|blush|Each|Unknown|24|barn stationeseought| +14791|AAAAAAAAHMJDAAAA|1997-10-27||Rules expect key,|4.21|2.77|3001002|amalgexporti #2|1|newborn|3|Children|627|ationablecally|petite|218432206765khaki504|plum|Pallet|Unknown|27|oughtn stationeseought| +14792|AAAAAAAAIMJDAAAA|1997-10-27|2000-10-26|Other, available police should make previously from a rises; great, confident designs run still, occasional years. Facts used to get exactly.|7.50|4.12|4003001|exportiedu pack #1|3|kids|4|Shoes|271|oughtationable|economy|9732246rosy842151649|red|Case|Unknown|1|ablen stationeseought| +14793|AAAAAAAAIMJDAAAA|2000-10-27||Visitors might recognise in a aims. Walls used t|5.86|2.28|7007004|brandbrand #4|3|decor|7|Home|271|oughtationable|N/A|9732246rosy842151649|peach|Ounce|Unknown|44|prin stationeseought| +14794|AAAAAAAAKMJDAAAA|1997-10-27|1999-10-27|Options save seriously fast services. Square limits used to implement more subjects. Weeks reside following, female rivers. Dire|0.58|0.26|2004001|edu packimporto #1|4|sports-apparel|2|Men|3|pri|extra large|84626peach3781262821|beige|Carton|Unknown|16|esen stationeseought| +14795|AAAAAAAAKMJDAAAA|1999-10-28|2001-10-26|Homes let routine, keen miles. Commitments must not distinguish frequently pleasant women; other materials criticize clear, physical eyes. Grand residents must receive more|1.16|1.04|2004001|corpmaxi #12|6|parenting|9|Books|222|ableableable|N/A|84626peach3781262821|lawn|Dram|Unknown|27|antin stationeseought| +14796|AAAAAAAAKMJDAAAA|2001-10-27||Good, wrong attacks can make a bit doctors. Regional years used to get meanwhile. Best direct pounds|4.69|1.04|2004001|exportiedu pack #1|3|kids|4|Shoes|29|n stable|large|84626peach3781262821|lawn|Unknown|Unknown|23|callyn stationeseought| +14797|AAAAAAAANMJDAAAA|1997-10-27||Annual, eligible pupils should tell yet politi|8.74|4.37|6001006|amalgcorp #6|1|birdal|6|Jewelry|291|oughtn stable|N/A|7938732096rosy703744|midnight|Carton|Unknown|79|ationn stationeseought| +14798|AAAAAAAAOMJDAAAA|1997-10-27|2000-10-26|Level men get most working years. Corporate duties remember o|0.25|0.14|10009009|maxiunivamalg #9|9|televisions|10|Electronics|185|antieingought|N/A|9381075peach02823512|sandy|Bunch|Unknown|18|eingn stationeseought| +14799|AAAAAAAAOMJDAAAA|2000-10-27||Relevant, japanese regulations learn. Proposed problems should not examine fair, continuous ta|2.21|0.14|1003002|exportiamalg #2|3|maternity|1|Women|185|antieingought|petite|1344mint698538989649|steel|Gross|Unknown|2|n stn stationeseought| +14800|AAAAAAAAANJDAAAA|1997-10-27|1999-10-27|Annual women must claim in no sizes. Minimum|6.64|3.65|3004001|edu packexporti #1|4|school-uniforms|3|Children|245|antieseable|medium|9747914violet8010705|sienna|Case|Unknown|13|barbareingeseought| +14801|AAAAAAAAANJDAAAA|1999-10-28|2001-10-26|Annual women must claim in no sizes. Minimum|5.80|4.64|1002002|importoamalg #2|2|fragrances|1|Women|496|antieseable|medium|9747914violet8010705|white|Ounce|Unknown|17|oughtbareingeseought| +14802|AAAAAAAAANJDAAAA|2001-10-27||Annual women must claim in no sizes. Minimum|84.63|4.64|3004001|edu packexporti #1|4|school-uniforms|3|Children|496|antieseable|large|9747914violet8010705|goldenrod|Tbl|Unknown|19|ablebareingeseought| +14803|AAAAAAAADNJDAAAA|1997-10-27||Special flights go. Good, apparent words will protest more needs. High positions should live far honours. Specially small teeth |4.45|3.07|6013004|exportibrand #4|13|loose stones|6|Jewelry|540|bareseanti|N/A|24088peach0218548986|pale|Dozen|Unknown|31|pribareingeseought| +14804|AAAAAAAAENJDAAAA|1997-10-27|2000-10-26|British lips may speak about senses. Ready comments start better british relations. Good, neutral days say names. Considerable, good thi|0.13|0.05|7012005|importonameless #5|12|paint|7|Home|493|prin stese|N/A|993cornflower3689447|cornflower|Bunch|Unknown|29|esebareingeseought| +14805|AAAAAAAAENJDAAAA|2000-10-27||Areas fall male sisters; due hills strike from a techniques. Enough successful corporations shall go of course cities. Per se japanese schools find most u|1.06|0.05|7012005|exportimaxi #6|13|sailing|8|Sports|493|prin stese|N/A|94638649white5241653|sandy|Tsp|Unknown|7|antibareingeseought| +14806|AAAAAAAAGNJDAAAA|1997-10-27|1999-10-27|Hard small poles can put together more corresponding terms. New, detailed women tell plants. Immediate documents will halt a|1.16|0.61|4002001|importoedu pack #1|2|mens|4|Shoes|847|ationeseeing|medium|84727pink18976759034|khaki|Carton|Unknown|14|callybareingeseought| +14807|AAAAAAAAGNJDAAAA|1999-10-28|2001-10-26|Questions believe also in a issues; international servants spend very considerable resources. Simple patients might begin even broad, democratic inches|5.77|4.15|7012008|importonameless #8|12|paint|7|Home|275|antiationable|N/A|695825pink3257439679|steel|Pallet|Unknown|32|ationbareingeseought| +14808|AAAAAAAAGNJDAAAA|2001-10-27||Questions believe also in a issues; international servants spend very considerable resources. Simple patients might begin even broad, democratic inches|2.91|4.15|7012008|importoamalg #1|12|fragrances|1|Women|230|antiationable|medium|08091383115salmon946|navy|Unknown|Unknown|45|eingbareingeseought| +14809|AAAAAAAAJNJDAAAA|1997-10-27||More real losses say so only passengers. Sorts recognise softly yet following goods. Just internal patients get. Hardly interested gates might |5.86|4.86|4001002|amalgedu pack #2|1|womens|4|Shoes|219|n stoughtable|medium|ghost127205634316584|pale|Bundle|Unknown|39|n stbareingeseought| +14810|AAAAAAAAKNJDAAAA|1997-10-27|2000-10-26|Immediately main principles shall wonder months. Problems come but great patients. British, afraid facilities can want profits. Total settlements may |9.46|3.02|1004001|edu packamalg #1|4|swimwear|1|Women|455|antiantiese|economy|7433089798peru646553|magenta|Case|Unknown|100|baroughteingeseought| +14811|AAAAAAAAKNJDAAAA|2000-10-27||Immediately main principles shall wonder months. Problems come but great patients. British, afraid facilities can want profits. Total settlements may |73.17|3.02|1004001|scholarnameless #2|15|tables|7|Home|455|antiantiese|N/A|7433089798peru646553|sienna|Ounce|Unknown|2|oughtoughteingeseought| +14812|AAAAAAAAMNJDAAAA|1997-10-27|1999-10-27|Just, white papers perceive all great things. Below clean elements c|9.99||4002001|importoedu pack #1|2|mens||Shoes|||small||||Unknown|17|ableoughteingeseought| +14813|AAAAAAAAMNJDAAAA|1999-10-28|2001-10-26|Wide projects should invest routinely with a parents. Strikingly soviet minutes must not exploit in a trousers. Wrong owners shall place philosophical, lexical cases. Suit|3.06|7.39|4002001|edu packbrand #6|14|estate|6|Jewelry|279|n stationable|N/A|25736royal3877396932|pale|Carton|Unknown|84|prioughteingeseought| +14814|AAAAAAAAMNJDAAAA|2001-10-27||Wide projects should invest routinely with a parents. Strikingly soviet minutes must not exploit in a trousers. Wrong owners shall place philosophical, lexical cases. Suit|6.61|3.30|6011007|amalgbrand #7|14|semi-precious|6|Jewelry|279|n stationable|N/A|sienna19146199034386|beige|Oz|Unknown|8|eseoughteingeseought| +14815|AAAAAAAAPNJDAAAA|1997-10-27||Issu|8.30|6.05|4004002|edu packedu pack #2|4|athletic|4|Shoes|184|eseeingought|petite|34094yellow952744503|rosy|Dozen|Unknown|35|antioughteingeseought| +14816|AAAAAAAAAOJDAAAA|1997-10-27|2000-10-26|Bits should celebrate almost social expectations. |6.54|4.90|4001001|amalgedu pack #1|1|womens|4|Shoes|114|eseoughtought|medium|976759976189pale0163|rosy|Pallet|Unknown|25|callyoughteingeseought| +14817|AAAAAAAAAOJDAAAA|2000-10-27||Cheap, public rates market quite records. Appropriate, overseas areas obtain in a families. Drivers would not convert statistical|30.26|18.15|4001001|edu packamalg #2|4|swimwear|1|Women|114|eseoughtought|large|13seashell5811209797|smoke|Tbl|Unknown|24|ationoughteingeseought| +14818|AAAAAAAACOJDAAAA|1997-10-27|1999-10-27|Able lines might happen premises; now real russians might not get aroun|4.95|1.53|2001001|amalgimporto #1|1|accessories|2|Men|275|antiationable|large|2047080740royal83456|khaki|Pound|Unknown|22|eingoughteingeseought| +14819|AAAAAAAACOJDAAAA|1999-10-28|2001-10-26|Lat|0.98|0.43|1004002|edu packamalg #2|4|swimwear|1|Women|275|antiationable|medium|59821087steel0352639|peach|Carton|Unknown|69|n stoughteingeseought| +14820|AAAAAAAACOJDAAAA|2001-10-27||Lat|7.66|0.43|10003014|exportiunivamalg #14|3|dvd/vcr players|10|Electronics|275|antiationable|N/A|10782631white7380318|red|Unknown|Unknown|94|barableeingeseought| +14821|AAAAAAAAFOJDAAAA|1997-10-27||Consta|4.74|3.74|4003002|exportiedu pack #2|3|kids|4|Shoes|313|prioughtpri|large|58225659543797grey58|puff|Cup|Unknown|64|oughtableeingeseought| +14822|AAAAAAAAGOJDAAAA|1997-10-27|2000-10-26|Foreign firms disagree anonymous, social s|4.50|2.70|2004001|edu packimporto #1|4|sports-apparel|2|Men|455|antiantiese|small|0192061450532smoke20|wheat|Tbl|Unknown|32|ableableeingeseought| +14823|AAAAAAAAGOJDAAAA|2000-10-27||Legal, due drawings ought to exist too steps. Fresh, long documents persuade more broken seconds. Full, glad theories dominate. Alternative, large machi|1.49|0.61|2004001|brandmaxi #4|4|reference|9|Books|455|antiantiese|N/A|069290103goldenrod49|navajo|Pallet|Unknown|16|priableeingeseought| +14824|AAAAAAAAIOJDAAAA|1997-10-27|1999-10-27|Little, old years used to get hotels; common chips might not improve still into a members. Curious para|0.16|0.06|1001001|amalgamalg #1|1|dresses|1|Women|169|n stcallyought|small|708midnight365477471|khaki|Each|Unknown|6|eseableeingeseought| +14825|AAAAAAAAIOJDAAAA|1999-10-28|2001-10-26|Little, old years used to get hotels; common chips might not improve still into a members. Curious para|32.98|0.06|1001001|importoimporto #2|2|shirts|2|Men|169|n stcallyought|small|1286110045640719red9|pink|Box|Unknown|51|antiableeingeseought| +14826|AAAAAAAAIOJDAAAA|2001-10-27||Real, great buildings import with a ways. Months would not catch for a sequences. Proposed levels reach pressures. Decidedly extra parties hit beforehand. Automatic, social |1.23|0.06|1001001|amalgexporti #1|2|newborn|3|Children|169|n stcallyought|large|1286110045640719red9|thistle|Dozen|Unknown|78|callyableeingeseought| +14827|AAAAAAAALOJDAAAA|1997-10-27||Loyal budgets cannot appreciate largely both low spirits. Periods like provisions. Actually |3.52|1.58|1002002|importoamalg #2|2|fragrances|1|Women|98|eingn st|petite|lavender267439465901|purple|Dram|Unknown|45|ationableeingeseought| +14828|AAAAAAAAMOJDAAAA|1997-10-27|2000-10-26|Years put by a tables. Other arguments ought to work very earnings; calculations may not use from a corners. Times might understand pilots. Emotional, difficult restaura|1.29|1.12|6012005|importobrand #5|12|costume|6|Jewelry|298|eingn stable|N/A|1ivory85600707074108|puff|Each|Unknown|15|eingableeingeseought| +14829|AAAAAAAAMOJDAAAA|2000-10-27||Years put by a tables. Other arguments ought to work very earnings; calculations may not use from a corners. Times might understand pilots. Emotional, difficult restaura|60.69|1.12|6012005|importounivamalg #10|12|home repair|9|Books|302|ablebarpri|N/A|61violet192611834445|puff|Tsp|Unknown|29|n stableeingeseought| +14830|AAAAAAAAOOJDAAAA|1997-10-27|1999-10-27|Successful, specific bedrooms cannot flow more economic, minor facilities; large, constant plans refuse over. Dogs woul|0.93|0.73|4002001|importoedu pack #1|2|mens|4|Shoes|182|ableeingought|medium|3turquoise3188123248|powder|Dram|Unknown|38|barprieingeseought| +14831|AAAAAAAAOOJDAAAA|1999-10-28|2001-10-26|Eastern, black servants take at a goods. Dead computers pack again small, actual publications. Far direct institutions complete at all willing, vertical plans. Joint, pr|7.77|0.73|4002001|namelessbrand #2|8|lighting|7|Home|32|ableeingought|N/A|3turquoise3188123248|rose|Pound|Unknown|20|oughtprieingeseought| +14832|AAAAAAAAOOJDAAAA|2001-10-27||Studies shall know new reasons; hence long weeks get still to a stocks. Full resources ask for a beliefs. Extraordinary, fo|1.38|1.21|4002001|importounivamalg #1|12|home repair|9|Books|185|ableeingought|N/A|3turquoise3188123248|snow|Oz|Unknown|11|ableprieingeseought| +14833|AAAAAAAABPJDAAAA|1997-10-27||More local years take to the parts. Normal drawings exist huge parts. Great, american vehicles should not fix. Ot|0.77|0.26|1003002|exportiamalg #2|3|maternity|1|Women|143|prieseought|small|5190926763peach76666|pink|Lb|Unknown|20|priprieingeseought| +14834|AAAAAAAACPJDAAAA|1997-10-27|2000-10-26|No longer positive problems prove. Fair british men has|6.38|2.99|8001005|amalgnameless #5|1|athletic shoes|8|Sports|263|pricallyable|N/A|93snow63855888609841|pink|Bunch|Unknown|26|eseprieingeseought| +14835|AAAAAAAACPJDAAAA|2000-10-27||No longer positive problems prove. Fair british men has|8.37|2.99|8001005|edu packedu pack #2|1|athletic|4|Shoes|414|pricallyable|medium|25911green3665920592|sienna|Tbl|Unknown|23|antiprieingeseought| +14836|AAAAAAAAEPJDAAAA|1997-10-27|1999-10-27|Over demanding subjects may not look of course after a pos|6.49|1.94|9011005|amalgunivamalg #5|11|cooking|9|Books|161|oughtcallyought|N/A|13292346orchid381727|sky|Lb|Unknown|40|callyprieingeseought| +14837|AAAAAAAAEPJDAAAA|1999-10-28|2001-10-26|Special, new issues could return there. Different dates live together recent profits. New, whole teachers know full, likely officials. Human, s|3.82|1.52|10010014|univamalgamalg #14|10|memory|10|Electronics|161|oughtcallyought|N/A|13292346orchid381727|seashell|Pallet|Unknown|21|ationprieingeseought| +14838|AAAAAAAAEPJDAAAA|2001-10-27||Special, new issues could return there. Different dates live together recent profits. New, whole teachers know full, likely officials. Human, s|4.97|4.27|1002001|importoamalg #1|2|fragrances|1|Women|236|oughtcallyought|N/A|13292346orchid381727|bisque|Bunch|Unknown|34|eingprieingeseought| +14839|AAAAAAAAHPJDAAAA|1997-10-27||Services prepare always conventional conditions. British children ought to see seconds. Regional rivers preserve much royal, eligible millions; anxious, past customers shall not accompany. Names c|1.77|1.59|9012008|importounivamalg #8|12|home repair|9|Books|377|ationationpri|N/A|35181slate5015093945|royal|Gram|Unknown|3|n stprieingeseought| +14840|AAAAAAAAIPJDAAAA|1997-10-27|2000-10-26|Forms must hear for the children. |4.83|2.60|3004001|edu packexporti #1|4|school-uniforms|3|Children|308|eingbarpri|extra large|peach833400054447113|lime|Oz|Unknown|1|bareseeingeseought| +14841|AAAAAAAAIPJDAAAA|2000-10-27||Forms must hear for the children. |7.60|2.60|4002002|importoedu pack #2|4|mens|4|Shoes|149|n steseought|large|peach833400054447113|white|Bunch|Unknown|13|oughteseeingeseought| +14842|AAAAAAAAKPJDAAAA|1997-10-27|1999-10-27|Sensible, academic partners can campaign so full, public polls; roses ought to visit difficult legs. Faster capable blacks may not introduce too in a proceedings. Marks should ex|7.39|2.80|6015003|scholarbrand #3|15|custom|6|Jewelry|31|oughtpri|N/A|549595192966wheat503|violet|Lb|Unknown|7|ableeseeingeseought| +14843|AAAAAAAAKPJDAAAA|1999-10-28|2001-10-26|Sensible, academic partners can campaign so full, public polls; roses ought to visit difficult legs. Faster capable blacks may not introduce too in a proceedings. Marks should ex|1.63|1.27|6015003|importoimporto #2|2|shirts|2|Men|31|oughtpri|extra large|549595192966wheat503|light|Pallet|Unknown|27|prieseeingeseought| +14844|AAAAAAAAKPJDAAAA|2001-10-27||Sensible, academic partners can campaign so full, public polls; roses ought to visit difficult legs. Faster capable blacks may not introduce too in a proceedings. Marks should ex|3.07|1.27|8016005|corpmaxi #5|2|golf|8|Sports|31|oughtpri|N/A|526435016plum0052955|red|Box|Unknown|31|eseeseeingeseought| +14845|AAAAAAAANPJDAAAA|1997-10-27||Little capital wheels involve never police. Federal attacks would understand there distinctive, crucial recommendations. Merely length|6.13|2.81|10003005|exportiunivamalg #5|3|dvd/vcr players|10|Electronics|270|barationable|N/A|0452070649272royal46|peru|Each|Unknown|12|antieseeingeseought| +14846|AAAAAAAAOPJDAAAA|1997-10-27|2000-10-26|Even western cases work in a years. Systems used to derive; only basic walls emphasise seriously other habits. Past social officials may think even aware governments. At first deep courses help defini|0.66|0.37|2004001|edu packimporto #1|4|sports-apparel|2|Men|173|priationought|petite|puff2180864656539103|sandy|Ton|Unknown|91|callyeseeingeseought| +14847|AAAAAAAAOPJDAAAA|2000-10-27||Beautiful, only|0.94|0.37|2004001|importomaxi #8|4|guns|8|Sports|487|ationeingese|N/A|puff2180864656539103|puff|Dozen|Unknown|45|ationeseeingeseought| +14848|AAAAAAAAAAKDAAAA|1997-10-27|1999-10-27|New losses shall move quite usually aware schools. Powerful, long peasants would open economic, massive characteristics. Nevertheless main games may not want activ|2.29|1.21|1004001|edu packamalg #1|4|swimwear|1|Women|12|ableought|petite|3962990692429purple5|peach|Lb|Unknown|9|eingeseeingeseought| +14849|AAAAAAAAAAKDAAAA|1999-10-28|2001-10-26|New losses shall move quite usually aware schools. Powerful, long peasants would open economic, massive characteristics. Nevertheless main games may not want activ|3.50|1.21|8015010|scholarmaxi #10|4|fishing|8|Sports|12|ableought|N/A|3962990692429purple5|sky|Gram|Unknown|68|n steseeingeseought| +14850|AAAAAAAAAAKDAAAA|2001-10-27||New losses shall move quite usually aware schools. Powerful, long peasants would open economic, massive characteristics. Nevertheless main games may not want activ|5.35|1.21|2002001|importoimporto #1|2|shirts|2|Men|257|ableought|petite|3962990692429purple5|peach|Oz|Unknown|17|barantieingeseought| +14851|AAAAAAAADAKDAAAA|1997-10-27||Sexual buildings wake immediately commercial, little witnesses. Just local collections sit all right common con|4.75|4.18|4002002|importoedu pack #2|2|mens|4|Shoes|150|barantiought|small|130lemon775512703535|sky|Unknown|Unknown|79|oughtantieingeseought| +14852|AAAAAAAAEAKDAAAA|1997-10-27|2000-10-26|Too independent|8.17|6.29|5003001|exportischolar #1|3|pop|5|Music|320|barablepri|N/A|87011908608papaya927|linen|Pallet|Unknown|58|ableantieingeseought| +14853|AAAAAAAAEAKDAAAA|2000-10-27||Large, nearby structures confirm types. Higher small thousan|2.41|6.29|5003001|edu packexporti #2|3|school-uniforms|3|Children|320|barablepri|petite|635691453pale5222348|azure|Ounce|Unknown|11|priantieingeseought| +14854|AAAAAAAAGAKDAAAA|1997-10-27|1999-10-27|Left, recent restrictions result only other forms. Agents ensure likely points. Signs could not seem more than free hours. Forces make black, christian arrangem|4.50|3.78|10010014|univamalgamalg #14|10|memory|10|Electronics|662|ablecallycally|N/A|violet49042108845440|plum|Oz|Unknown|60|eseantieingeseought| +14855|AAAAAAAAGAKDAAAA|1999-10-28|2001-10-26|Left, recent restrictions result only other forms. Agents ensure likely points. Signs could not seem more than free hours. Forces make black, christian arrangem|||5004002|edu packscholar #2|4|||Music||ableationation|N/A|||Tsp||6|antiantieingeseought| +14856|AAAAAAAAGAKDAAAA|2001-10-27||Old, radical children should root there wrong principles. Als|9.67|3.78|5004002|edu packcorp #1|4|bracelets|6|Jewelry|772|ableationation|N/A|violet49042108845440|wheat|Pallet|Unknown|13|callyantieingeseought| +14857|AAAAAAAAJAKDAAAA|1997-10-27||Maj|0.15|0.07|5002002|importoscholar #2|2|country|5|Music|536|callyprianti|N/A|6594180navajo6981681|purple|Cup|Unknown|88|ationantieingeseought| +14858|AAAAAAAAKAKDAAAA|1997-10-27|2000-10-26|Forward, severe parties know public plants; s|2.71|1.35|1003001|exportiamalg #1|3|maternity|1|Women|664|esecallycally|medium|5039purple5281931440|saddle|Cup|Unknown|30|eingantieingeseought| +14859|AAAAAAAAKAKDAAAA|2000-10-27||Words notice evidently difficult years. Defensively other women thi|0.76|0.48|1003001|amalgamalg #2|3|dresses|1|Women|763|esecallycally|petite|42310324658445wheat3|plum|Box|Unknown|54|n stantieingeseought| +14860|AAAAAAAAMAKDAAAA|1997-10-27|1999-10-27|Then available arms should generate by a mac|5.54|4.48|9013005|exportiunivamalg #5|13|self-help|9|Books|409|n stbarese|N/A|505sky53687943359330|salmon|Dram|Unknown|29|barcallyeingeseought| +14861|AAAAAAAAMAKDAAAA|1999-10-28|2001-10-26|Then available arms should generate by a mac|2.43|1.62|3001002|amalgexporti #2|1|newborn|3|Children|190|n stbarese|large|2040560640peru013710|lemon|Unknown|Unknown|6|oughtcallyeingeseought| +14862|AAAAAAAAMAKDAAAA|2001-10-27||Then available arms should generate by a mac|7.17|1.62|7014003|edu packnameless #3|14|glassware|7|Home|190|n stbarese|N/A|598712maroon34780818|pink|Dram|Unknown|47|ablecallyeingeseought| +14863|AAAAAAAAPAKDAAAA|1997-10-27||Germans consider high, representative doors; somewhat right countries would not strive absolutely rough old needs. Whole, possible communities will go also past, other departments. Ele|3.46|1.69|4003002|exportiedu pack #2|3|kids|4|Shoes|498|eingn stese|petite|4809lawn076531300272|royal|Cup|Unknown|48|pricallyeingeseought| +14864|AAAAAAAAABKDAAAA|1997-10-27|2000-10-26|Impressive, northern curtains sustain as old, technical copies. Pop|3.96|2.53|4004001|edu packedu pack #1|4|athletic|4|Shoes|61|oughtcally|petite|5873357334green56316|purple|Gross|Unknown|78|esecallyeingeseought| +14865|AAAAAAAAABKDAAAA|2000-10-27||Impressive, northern curtains sustain as old, technical copies. Pop|1.81|0.95|4001002|amalgedu pack #2|4|womens|4|Shoes|352|ableantipri|medium|67365666404slate3013|turquoise|N/A|Unknown|63|anticallyeingeseought| +14866|AAAAAAAACBKDAAAA|1997-10-27|1999-10-27|Major, global suggestions will cal|5.11|1.99|10012009|importoamalgamalg #9|12|monitors|10|Electronics|21|oughtable|N/A|819295176854432plum6|purple|Bundle|Unknown|9|callycallyeingeseought| +14867|AAAAAAAACBKDAAAA|1999-10-28|2001-10-26|Global wines would not want names. Familiar firms change. Black, rich papers consider examples. Economic, moving feelings cannot take gently cou|1.76|1.99|8002008|importonameless #8|12|baseball|8|Sports|21|oughtable|N/A|819295176854432plum6|puff|Tbl|Unknown|49|ationcallyeingeseought| +14868|AAAAAAAACBKDAAAA|2001-10-27||Global wines would not want names. Familiar firms change. Black, rich papers consider examples. Economic, moving feelings cannot take gently cou|5.87|1.99|10012011|importoamalgamalg #11|12|monitors|10|Electronics|21|oughtable|N/A|8080309476hot8189731|red|Tsp|Unknown|46|eingcallyeingeseought| +14869|AAAAAAAAFBKDAAAA|1997-10-27||Pieces can judge as. More other daughters discover in a companies. Never new laws undertak|2.21|1.21|8003010|exportinameless #10|3|basketball|8|Sports|239|n stpriable|N/A|peach978641515437627|pale|Unknown|Unknown|1|n stcallyeingeseought| +14870|AAAAAAAAGBKDAAAA|1997-10-27|2000-10-26|Other men can keep in a customers. Surprised premises might not allow. Technical, british cler|2.42|1.21|1003001|exportiamalg #1|3|maternity|1|Women|468|eingcallyese|extra large|0587thistle563639413|pink|Gram|Unknown|58|barationeingeseought| +14871|AAAAAAAAGBKDAAAA|2000-10-27||Other men can keep in a customers. Surprised premises might not allow. Technical, british cler|8.65|6.22|4004002|edu packedu pack #2|4|athletic|4|Shoes|856|eingcallyese|extra large|0587thistle563639413|rosy|Each|Unknown|23|oughtationeingeseought| +14872|AAAAAAAAIBKDAAAA|1997-10-27|1999-10-27|Hundreds drop nearly unacceptable accidents. Then strong methods tell large unions. Short companies should help so. Moves shall not set later chief problems. R|0.78|0.45|9002005|importomaxi #5|2|business|9|Books|617|ationoughtcally|N/A|2932829saddle1417342|ghost|Dram|Unknown|17|ableationeingeseought| +14873|AAAAAAAAIBKDAAAA|1999-10-28|2001-10-26|Peaceful customers try now still past races; devices should not buy so as a clergy; in order remaining trees agree sure charts. Evenings might not mean otherwise please fine audiences. Physical,|3.95|0.45|9002005|exportiimporto #2|3|pants|2|Men|617|ationoughtcally|petite|1545427517754sandy31|mint|Pound|Unknown|29|priationeingeseought| +14874|AAAAAAAAIBKDAAAA|2001-10-27||Communists see again existing affairs. About national chips get ancient, toxic cards. Seriously certain resources mean even deep figures. Th|4.97|3.82|8015005|scholarmaxi #5|15|fishing|8|Sports|617|ationoughtcally|N/A|185679781958wheat597|azure|Cup|Unknown|33|eseationeingeseought| +14875|AAAAAAAALBKDAAAA|1997-10-27||Little working prisoners correct more at least hard awards. Kinds applaud again|4.46|3.47|5001002|amalgscholar #2|1|rock|5|Music|882|ableeingeing|N/A|78196693191violet001|royal|Dozen|Unknown|22|antiationeingeseought| +14876|AAAAAAAAMBKDAAAA|1997-10-27|2000-10-26|Redundant languages know very english aspects. Bones own just huge, physical affairs. La|6.08|4.92|6016005|corpbrand #5|16|consignment|6|Jewelry|232|ablepriable|N/A|0798628rose134727834|lime|N/A|Unknown|57|callyationeingeseought| +14877|AAAAAAAAMBKDAAAA|2000-10-27||Makers recognize effectively alone operations; properties forget very|3.18|4.92|3004002|edu packexporti #2|16|school-uniforms|3|Children|297|ationn stable|extra large|13389631050coral9056|snow|Box|Unknown|25|ationationeingeseought| +14878|AAAAAAAAOBKDAAAA|1997-10-27|1999-10-27|Patterns want rapidly average, african sales. Surprising approaches die so round, revolutionary pages. Long, advisory areas eat. Now high times can carry other youngsters. Never fr|6.49|2.40|2004001|edu packimporto #1|4|sports-apparel|2|Men|113|prioughtought|small|4803545808tomato6176|magenta|Case|Unknown|66|eingationeingeseought| +14879|AAAAAAAAOBKDAAAA|1999-10-28|2001-10-26|Patterns want rapidly average, african sales. Surprising approaches die so round, revolutionary pages. Long, advisory areas eat. Now high times can carry other youngsters. Never fr|4.55|1.77|2004001|amalgscholar #2|1|rock|5|Music|269|prioughtought|N/A|4803545808tomato6176|hot|Dram|Unknown|51|n stationeingeseought| +14880|AAAAAAAAOBKDAAAA|2001-10-27||Patterns want rapidly average, african sales. Surprising approaches die so round, revolutionary pages. Long, advisory areas eat. Now high times can carry other youngsters. Never fr|47.28|29.78|2004001|importoscholar #1|2|country|5|Music|269|n stcallyable|N/A|9steel74767375899332|rosy|Unknown|Unknown|64|bareingeingeseought| +14881|AAAAAAAABCKDAAAA|1997-10-27||Away legal pressures might protect wide kilometr|3.13|1.84|6006002|corpcorp #2|6|rings|6|Jewelry|460|barcallyese|N/A|508beige757596614551|salmon|Cup|Unknown|64|oughteingeingeseought| +14882|AAAAAAAACCKDAAAA|1997-10-27|2000-10-26|Direct pieces exclude surveys. Actually successful waters could pa|7.56|2.57|1002001|importoamalg #1|2|fragrances|1|Women|182|ableeingought|medium|magenta3532358247655|grey|Carton|Unknown|3|ableeingeingeseought| +14883|AAAAAAAACCKDAAAA|2000-10-27||Inside good interests shall attend only ltd. liberal couples. Hot margins start; over social markets shall influence better |4.66|3.68|6002004|importocorp #4|2|diamonds|6|Jewelry|182|ableeingought|N/A|magenta3532358247655|lime|Unknown|Unknown|61|prieingeingeseought| +14884|AAAAAAAAECKDAAAA|1997-10-27|1999-10-27|Political, bare enterprises can grow|2.44|0.75|4002001|importoedu pack #1|2|mens|4|Shoes|208|eingbarable|N/A|salmon40521116735762|steel|Dozen|Unknown|54|eseeingeingeseought| +14885|AAAAAAAAECKDAAAA|1999-10-28|2001-10-26|Political, bare enterprises can grow|8.72|6.54|4002001|scholarmaxi #6|2|fishing|8|Sports|208|eingbarable|N/A|salmon40521116735762|peru|Gram|Unknown|1|antieingeingeseought| +14886|AAAAAAAAECKDAAAA|2001-10-27||Bands progress until a days. European, sure miles boycott designs. Quiet, new owners present women; terrible effects demonstrate. Industrial families can achieve difficu|1.41|6.54|4002001|amalgexporti #1|2|newborn|3|Children|208|eingbarable|petite|899341415salmon54982|hot|N/A|Unknown|2|callyeingeingeseought| +14887|AAAAAAAAHCKDAAAA|1997-10-27||Labour things tell light, comprehensive areas; movements can take about short, joint children. Constitutional, respo|7.69|6.69|3002002|importoexporti #2|2|infants|3|Children|185|antieingought|large|262turquoise43429787|goldenrod|Lb|Unknown|16|ationeingeingeseought| +14888|AAAAAAAAICKDAAAA|1997-10-27|2000-10-26|Better social i|2.23|1.78|4004001|edu packedu pack #1|4|athletic|4|Shoes|293|prin stable|petite|3851373saddle4365230|tomato|Gross|Unknown|54|eingeingeingeseought| +14889|AAAAAAAAICKDAAAA|2000-10-27||Better social i|4.09|1.78|3001002|amalgexporti #2|4|newborn|3|Children|293|prin stable|small|3851373saddle4365230|chartreuse|Lb|Unknown|15|n steingeingeseought| +14890|AAAAAAAAKCKDAAAA|1997-10-27|1999-10-27|Areas ought to make some|3.06|2.26|5003001|exportischolar #1|3|pop|5|Music|128|eingableought|N/A|369wheat434692590960|tan|Unknown|Unknown|29|barn steingeseought| +14891|AAAAAAAAKCKDAAAA|1999-10-28|2001-10-26|Areas ought to make some|4.56|2.26|10008017|namelessunivamalg #17|3|scanners|10|Electronics|128|eingableought|N/A|91805028856powder728|cream|Each|Unknown|41|oughtn steingeseought| +14892|AAAAAAAAKCKDAAAA|2001-10-27||Lexical, sophisticated directions must lose quite just|5.70|2.26|10008017|amalgedu pack #1|3|womens|4|Shoes|128|eingableought|medium|91805028856powder728|snow|Bunch|Unknown|26|ablen steingeseought| +14893|AAAAAAAANCKDAAAA|1997-10-27||Southern, final minerals cannot link later early efficient others. Perhaps serious men used to require not military local dogs. Days |3.80|1.86|3003002|exportiexporti #2|3|toddlers|3|Children|71|oughtation|large|1358056481192khaki89|tomato|Bunch|Unknown|53|prin steingeseought| +14894|AAAAAAAAOCKDAAAA|1997-10-27|2000-10-26|Active, different governments used to keep unable, chief things. Subtle, releva|3.70|2.84|7010005|univnameless #5|10|flatware|7|Home|293|prin stable|N/A|315749447465saddle17|ghost|Case|Unknown|24|esen steingeseought| +14895|AAAAAAAAOCKDAAAA|2000-10-27||Active, different governments used to keep unable, chief things. Subtle, releva|6.27|2.84|7010005|amalgamalg #2|10|dresses|1|Women|293|prin stable|medium|315749447465saddle17|red|Bunch|Unknown|54|antin steingeseought| +14896|AAAAAAAAADKDAAAA|1997-10-27|1999-10-27|Currently major appointments could become in a occupations. Tests record today|1.67|1.41|8005007|scholarnameless #7|5|fitness|8|Sports|206|callybarable|N/A|82626227953powder875|wheat|N/A|Unknown|7|callyn steingeseought| +14897|AAAAAAAAADKDAAAA|1999-10-28|2001-10-26|Tracks judge on a women. Happy,|6.02|3.55|8005007|edu packamalg #2|5|swimwear|1|Women|206|callybarable|petite|82626227953powder875|steel|Case|Unknown|80|ationn steingeseought| +14898|AAAAAAAAADKDAAAA|2001-10-27||Ready, huge places realise similarly narrow arms; tory negotiations may come badly for a objectives. Civil days wor|2.90|3.55|2002001|importoimporto #1|2|shirts|2|Men|288|callybarable|medium|82626227953powder875|royal|Cup|Unknown|28|eingn steingeseought| +14899|AAAAAAAADDKDAAAA|1997-10-27||Popular writers must change courses. Consequences might destroy as parties. Visual, efficient eyes should info|8.30|4.89|1002002|importoamalg #2|2|fragrances|1|Women|295|antin stable|medium|28451031yellow758877|white|Bundle|Unknown|93|n stn steingeseought| +14900|AAAAAAAAEDKDAAAA|1997-10-27|2000-10-26|Times live now to a sales. British years bring all financ|4.24|1.44|9001009|amalgmaxi #9|1|arts|9|Books|408|eingbarese|N/A|376tan25271592712537|navajo|Box|Unknown|8|barbarn steseought| +14901|AAAAAAAAEDKDAAAA|2000-10-27||Times live now to a sales. British years bring all financ|7.44|1.44|9001009|edu packexporti #2|4|school-uniforms|3|Children|408|eingbarese|extra large|4997magenta153101921|lime|Tsp|Unknown|32|oughtbarn steseought| +14902|AAAAAAAAGDKDAAAA|1997-10-27|1999-10-27|About working feelings could produce only types. Electoral, new visitors will not make more afraid, large tr|6.92|2.69|4003001|exportiedu pack #1|3|kids|4|Shoes|51|oughtanti|medium|315681043033sky41619|steel|Gram|Unknown|28|ablebarn steseought| +14903|AAAAAAAAGDKDAAAA|1999-10-28|2001-10-26|Only favorite minutes will not drive ever empty decades; feet illustrate then|1.38|1.10|2001002|amalgimporto #2|1|accessories|2|Men|51|oughtanti|large|80895543sky887073109|drab|Dozen|Unknown|19|pribarn steseought| +14904|AAAAAAAAGDKDAAAA|2001-10-27||Only favorite minutes will not drive ever empty decades; feet illustrate then|0.71|1.10|5003001|exportischolar #1|3|pop|5|Music|302|ablebarpri|N/A|8926134pink804838955|salmon|Dozen|Unknown|35|esebarn steseought| +14905|AAAAAAAAJDKDAAAA|1997-10-27||Revolutionary, neat years ought to pay short, similar children. Total, difficult debts would take enough religious events. So social objectives ge|0.58|0.17|2001002|amalgimporto #2|1|accessories|2|Men|969|n stcallyn st|small|75536purple602465129|spring|N/A|Unknown|1|antibarn steseought| +14906|AAAAAAAAKDKDAAAA|1997-10-27|2000-10-26|Reports might find ago aware questions|8.31|3.57|2002001|importoimporto #1|2|shirts|2|Men|541|oughteseanti|N/A|16284230772peach8397|pale|Pallet|Unknown|26|callybarn steseought| +14907|AAAAAAAAKDKDAAAA|2000-10-27||Reports might find ago aware questions|40.94|35.20|2002001|exportinameless #8|3|basketball|8|Sports|541|oughteseanti|N/A|16284230772peach8397|salmon|Ton|Unknown|78|ationbarn steseought| +14908|AAAAAAAAMDKDAAAA|1997-10-27|1999-10-27|Long only eyes used to accept light, american|8.72|3.48|8016009|corpmaxi #9|16|golf|8|Sports|123|priableought|N/A|5939605561midnight37|rose|Oz|Unknown|49|eingbarn steseought| +14909|AAAAAAAAMDKDAAAA|1999-10-28|2001-10-26|Long only eyes used to accept light, american|6.00|4.68|5004002|edu packscholar #2|4|classical|5|Music|123|priableought|N/A|5939605561midnight37|yellow|Case|Unknown|15|n stbarn steseought| +14910|AAAAAAAAMDKDAAAA|2001-10-27||Well consistent years take experiences. Literally fu|6.01|4.68|9007007|brandmaxi #7|4|reference|9|Books|123|priableought|N/A|521salmon39483188978|royal|Pound|Unknown|18|baroughtn steseought| +14911|AAAAAAAAPDKDAAAA|1997-10-27||Senior legs would understand groups. Positions mean inc requirements. Distinct, full persons expect altogether great, improved requirements. Businesses hope|7.77|4.81|5003002|exportischolar #2|3|pop|5|Music|57|ationanti|N/A|86lace64214309483022|rose|Cup|Unknown|32|oughtoughtn steseought| +14912|AAAAAAAAAEKDAAAA|1997-10-27|2000-10-26|Continental others understand proceedings. Up to empty sequences mention for example military, central consequences. Ill southern associations |3.55|1.13|4004001|edu packedu pack #1|4|athletic|4|Shoes|516|callyoughtanti|petite|55997589763164white4|steel|Case|Unknown|61|ableoughtn steseought| +14913|AAAAAAAAAEKDAAAA|2000-10-27||Continental others understand proceedings. Up to empty sequences mention for example military, central consequences. Ill southern associations |2.07|1.13|3004002|edu packexporti #2|4|school-uniforms|3|Children|865|anticallyeing|small|74095296183grey57976|red|Tbl|Unknown|64|prioughtn steseought| +14914|AAAAAAAACEKDAAAA|1997-10-27|1999-10-27|Always reliable records say both by the problems; researchers shall not sail somewhat good, environmental legs. Else welcome germans must afford centuries. European, exceptional women would suppos|23.91|21.27|8002005|importonameless #5|2|baseball|8|Sports|122|ableableought|N/A|6magenta605978477086|sandy|Gram|Unknown|52|eseoughtn steseought| +14915|AAAAAAAACEKDAAAA|1999-10-28|2001-10-26|Always reliable records say both by the problems; researchers shall not sail somewhat good, environmental legs. Else welcome germans must afford centuries. European, exceptional women would suppos|6.34|4.81|4002002|importoedu pack #2|2|mens|4|Shoes|122|ableableought|large|34799pale72149899681|white|Ounce|Unknown|48|antioughtn steseought| +14916|AAAAAAAACEKDAAAA|2001-10-27||Always reliable records say both by the problems; researchers shall not sail somewhat good, environmental legs. Else welcome germans must afford centuries. European, exceptional women would suppos|3.66|4.81|4002002|importounivamalg #1|2|home repair|9|Books|122|ableableought|N/A|32847077106232khaki0|steel|Box|Unknown|35|callyoughtn steseought| +14917|AAAAAAAAFEKDAAAA|1997-10-27||Statements |7.82|4.30|6002006|importocorp #6|2|diamonds|6|Jewelry|892|ablen steing|N/A|0380puff943307044380|yellow|Case|Unknown|39|ationoughtn steseought| +14918|AAAAAAAAGEKDAAAA|1997-10-27|2000-10-26|Level, likely goods share never riv|3.37|1.48|5003001|exportischolar #1|3|pop|5|Music|497|ationn stese|N/A|08forest075561152382|puff|Each|Unknown|10|eingoughtn steseought| +14919|AAAAAAAAGEKDAAAA|2000-10-27||In general national practices should not continue on a lists. African descriptions illustrate seldom thinking services. Days continue bri|3.48|1.48|5003001|edu packbrand #10|4|curtains/drapes|7|Home|494|esen stese|N/A|08forest075561152382|spring|Ton|Unknown|26|n stoughtn steseought| +14920|AAAAAAAAIEKDAAAA|1997-10-27|1999-10-27|Main hours spe|9.76|5.85|9014005|edu packunivamalg #5|14|sports|9|Books|289|n steingable|N/A|25506powder714294593|sky|Lb|Unknown|17|barablen steseought| +14921|AAAAAAAAIEKDAAAA|1999-10-28|2001-10-26|Other, local christians cannot like likely others. Other conditions could devote q|5.08|5.85|9014005|scholarnameless #2|14|tables|7|Home|109|n stbarought|N/A|25506powder714294593|saddle|N/A|Unknown|34|oughtablen steseought| +14922|AAAAAAAAIEKDAAAA|2001-10-27||Other, local christians cannot like likely others. Other conditions could devote q|2.01|1.70|8006003|corpnameless #3|6|football|8|Sports|109|n stbarought|N/A|736208002bisque07894|gainsboro|Ton|Unknown|33|ableablen steseought| +14923|AAAAAAAALEKDAAAA|1997-10-27||Too particular sites look regularly catholic spots; subjects drive in a children. Cheeks exist now specific lights. Average forces will max|3.75|2.73|7007004|brandbrand #4|7|decor|7|Home|15|antiought|N/A|073429811579461mint1|saddle|Pallet|Unknown|19|priablen steseought| +14924|AAAAAAAAMEKDAAAA|1997-10-27|2000-10-26|Large levels should add futures; fresh, good limitations may carry later units|0.50|0.36|4003001|exportiedu pack #1|3|kids|4|Shoes|518|eingoughtanti|small|680984misty552384332|white|Unknown|Unknown|27|eseablen steseought| +14925|AAAAAAAAMEKDAAAA|2000-10-27||Large levels should add futures; fresh, good limitations may carry later units|3.03|0.36|10001017|amalgunivamalg #17|3|cameras|10|Electronics|518|eingoughtanti|N/A|680984misty552384332|rose|Box|Unknown|38|antiablen steseought| +14926|AAAAAAAAOEKDAAAA|1997-10-27|1999-10-27|Years could escort great circumstances; extra, alleged shows pick even still main accountants. Hands may not explain pretty rights. Even dull examples hear else. Low commu|0.97|0.56|2001001|amalgimporto #1|1|accessories|2|Men|59|n stanti|small|581437234369650grey5|brown|Box|Unknown|14|callyablen steseought| +14927|AAAAAAAAOEKDAAAA|1999-10-28|2001-10-26|Years could escort great circumstances; extra, alleged shows pick even still main accountants. Hands may not explain pretty rights. Even dull examples hear else. Low commu|0.56|0.56|2001001|amalgscholar #2|1|rock|5|Music|59|n stanti|N/A|581437234369650grey5|firebrick|Case|Unknown|5|ationablen steseought| +14928|AAAAAAAAOEKDAAAA|2001-10-27||Opinions put in |7.83|5.01|7014009|edu packnameless #9|1|glassware|7|Home|59|n stanti|N/A|8668029769673papaya2|violet|Cup|Unknown|5|eingablen steseought| +14929|AAAAAAAABFKDAAAA|1997-10-27||Enough old responsibilities avoid of course as the man|4.47|1.43|6003002|exporticorp #2|3|gold|6|Jewelry|214|eseoughtable|N/A|208rose7182198843239|hot|Dozen|Unknown|64|n stablen steseought| +14930|AAAAAAAACFKDAAAA|1997-10-27|2000-10-26|Legal, likely eyes would deny in a changes. Agencies could say on a telecommunications; low, above schools would not tell. Again necessary children ought to go|8.41|3.11|3003001|exportiexporti #1|3|toddlers|3|Children|747|ationeseation|small|07837760949162rosy81|lace|N/A|Unknown|98|barprin steseought| +14931|AAAAAAAACFKDAAAA|2000-10-27||Legal, likely eyes would deny in a changes. Agencies could say on a telecommunications; low, above schools would not tell. Again necessary children ought to go|84.72|3.11|3003001|scholarunivamalg #6|3|karoke|10|Electronics|747|ationeseation|N/A|63892tan137608046707|ghost|Ounce|Unknown|62|oughtprin steseought| +14932|AAAAAAAAEFKDAAAA|1997-10-27|1999-10-27|Academic experts may like very other hours. Today occasional men prevent then annual, small relationships; as great titles |4.81|2.78|6002005|importocorp #5|2|diamonds|6|Jewelry|3|pri|N/A|2984575pale764351541|snow|Dram|Unknown|33|ableprin steseought| +14933|AAAAAAAAEFKDAAAA|1999-10-28|2001-10-26|However large examples should worsen locally reasonable, concerned newspapers. Very, recent men can n|2.55|2.09|6002005|amalgamalgamalg #8|11|disk drives|10|Electronics|352|pri|N/A|611431890sky19598653|peach|Case|Unknown|50|priprin steseought| +14934|AAAAAAAAEFKDAAAA|2001-10-27||However large examples should worsen locally reasonable, concerned newspapers. Very, recent men can n|4.75|2.09|6002005|univunivamalg #7|10|travel|9|Books|168|eingcallyought|N/A|4goldenrod9953274645|spring|Pallet|Unknown|20|eseprin steseought| +14935|AAAAAAAAHFKDAAAA|1997-10-27||Physical servants cannot want promptly large holidays. Financially social flowers maintain then difficult sha|3.75|1.31|5004002|edu packscholar #2|4|classical|5|Music|572|ableationanti|N/A|6peru356022886752483|sky|Bundle|Unknown|21|antiprin steseought| +14936|AAAAAAAAIFKDAAAA|1997-10-27|2000-10-26|Very judicial purposes join as rather difficult users. Questions would know too rich, constant areas; |0.93|0.80|2001001|amalgimporto #1|1|accessories|2|Men|221|oughtableable|extra large|760492293809peach077|honeydew|Lb|Unknown|23|callyprin steseought| +14937|AAAAAAAAIFKDAAAA|2000-10-27||Statutory, complete products should find especially beneficial votes; things may not give |93.60|0.80|2001001|amalgexporti #2|1|newborn|3|Children|237|ationpriable|economy|760492293809peach077|midnight|Dram|Unknown|35|ationprin steseought| +14938|AAAAAAAAKFKDAAAA|1997-10-27|1999-10-27|Free, expensive rivers can mind. Jobs change otherwise charming issues. Children cannot look generally careers; reforms take into a blacks. Aware, attractive grounds will add as yet econom|30.34|13.95|7001009|amalgbrand #9|1|bathroom|7|Home|428|eingableese|N/A|632842738102frosted0|sky|Tbl|Unknown|45|eingprin steseought| +14939|AAAAAAAAKFKDAAAA|1999-10-28|2001-10-26|Free, expensive rivers can mind. Jobs change otherwise charming issues. Children cannot look generally careers; reforms take into a blacks. Aware, attractive grounds will add as yet econom|5.88|5.29|6004004|edu packcorp #4|4|bracelets|6|Jewelry|131|oughtpriought|N/A|632842738102frosted0|plum|Pound|Unknown|10|n stprin steseought| +14940|AAAAAAAAKFKDAAAA|2001-10-27||Good restrictions meet from a children; schemes should keep companies. Free, productive errors used to meet closely great, independent titles. Far white years collect below then sure jour|7.83|2.50|6009005|maxicorp #5|9|womens watch|6|Jewelry|666|callycallycally|N/A|turquoise61239007388|papaya|Oz|Unknown|1|baresen steseought| +14941|AAAAAAAANFKDAAAA|1997-10-27||Levels get conditions. Mutual acts can|3.28|2.75|6005006|scholarcorp #6|5|earings|6|Jewelry|102|ablebarought|N/A|50606773035350white6|snow|Case|Unknown|35|oughtesen steseought| +14942|AAAAAAAAOFKDAAAA|1997-10-27|2000-10-26|Cheerful, profitable visitors may develop actua|1.28|0.78|5002001|importoscholar #1|2|country|5|Music|727|ationableation|N/A|555270294steel351128|forest|Unknown|Unknown|16|ableesen steseought| +14943|AAAAAAAAOFKDAAAA|2000-10-27||Patient, secure individuals ought t|29.02|9.28|5002001|edu packimporto #2|4|sports-apparel|2|Men|707|ationableation|medium|555270294steel351128|olive|Box|Unknown|22|priesen steseought| +14944|AAAAAAAAAGKDAAAA|1997-10-27|1999-10-27|Free eyes talk biolog|6.75|2.90|9008005|namelessmaxi #5|8|romance|9|Books|156|callyantiought|N/A|64191108pink76997556|hot|Unknown|Unknown|33|eseesen steseought| +14945|AAAAAAAAAGKDAAAA|1999-10-28|2001-10-26|Free eyes talk biolog|4.93|3.64|9008005|amalgimporto #2|1|accessories|2|Men|67|ationcally|petite|64191108pink76997556|plum|Dram|Unknown|21|antiesen steseought| +14946|AAAAAAAAAGKDAAAA|2001-10-27||Children would not grow just hot words. Institutions pay young teams. Foreign, available walls co|0.59|0.18|9008005|amalgscholar #1|1|rock|5|Music|67|ationcally|N/A|11911444852mint85719|peru|Pound|Unknown|40|callyesen steseought| +14947|AAAAAAAADGKDAAAA|1997-10-27||Senior days shift. Annua|8.94|4.91|7009008|maxibrand #8|9|mattresses|7|Home|128|eingableought|N/A|869978922mint0289177|slate|Lb|Unknown|19|ationesen steseought| +14948|AAAAAAAAEGKDAAAA|1997-10-27|2000-10-26|Golden, united flowers diminish. As good languages will compl|5.89|3.41|6008005|namelesscorp #5|8|mens watch|6|Jewelry|269|n stcallyable|N/A|2012857339rose662654|lawn|Pound|Unknown|37|eingesen steseought| +14949|AAAAAAAAEGKDAAAA|2000-10-27||Usual years support far. English, high views shall give during a windows. National names say mainly able, bitter letters; ever soft miners take over. Full, good friends arouse even left|4.70|3.41|2003002|exportiimporto #2|8|pants|2|Men|94|n stcallyable|medium|764578chartreuse8655|yellow|Each|Unknown|92|n stesen steseought| +14950|AAAAAAAAGGKDAAAA|1997-10-27|1999-10-27|Techniques recognize even to a affairs. Ways speak more than residential, advisory times. Central effects reconsider also actually sure years. Readily sensible deman|3.87|2.82|2003001|exportiimporto #1|3|pants|2|Men|193|prin stought|medium|plum2404996365908692|blanched|Tsp|Unknown|27|barantin steseought| +14951|AAAAAAAAGGKDAAAA|1999-10-28|2001-10-26|Techniques recognize even to a affairs. Ways speak more than residential, advisory times. Central effects reconsider also actually sure years. Readily sensible deman|7.01|2.10|2003001|amalgimporto #2|1|accessories|2|Men|193|prin stought|medium|plum2404996365908692|slate|Case|Unknown|33|oughtantin steseought| +14952|AAAAAAAAGGKDAAAA|2001-10-27||Techniques recognize even to a affairs. Ways speak more than residential, advisory times. Central effects reconsider also actually sure years. Readily sensible deman|1.73|0.84|4001001|amalgedu pack #1|1|womens|4|Shoes|193|prin stought|medium|1137305282salmon6836|peru|Carton|Unknown|65|ableantin steseought| +14953|AAAAAAAAJGKDAAAA|1997-10-27||Main weeks surrender more beyond a views. Popular, payable agencies cannot c|6.05|2.78|7008004|namelessbrand #4|8|lighting|7|Home|460|barcallyese|N/A|8308gainsboro9211271|saddle|Cup|Unknown|15|priantin steseought| +14954|AAAAAAAAKGKDAAAA|1997-10-27|2000-10-26|Patients could help fascinating forces. Gross, neat parents see big years. Methods could correspond children. Companies ought to know so. Areas involve ahead |4.02|3.09|10005012|scholarunivamalg #12|5|karoke|10|Electronics|824|eseableeing|N/A|791743wheat967154215|tomato|Tsp|Unknown|31|eseantin steseought| +14955|AAAAAAAAKGKDAAAA|2000-10-27||Tiny, burning services contest hopefully sad, relevant soldiers. S|3.67|2.86|6010004|univbrand #4|5|jewelry boxes|6|Jewelry|246|eseableeing|N/A|snow7555140453307799|plum|Ounce|Unknown|18|antiantin steseought| +14956|AAAAAAAAMGKDAAAA|1997-10-27|1999-10-27|Views cannot come databases; geographical securities speak tomorrow electoral, thin c|3.64|2.36|3003001|exportiexporti #1|3|toddlers|3|Children|223|priableable|petite|736026504730037red72|tan|Pound|Unknown|22|callyantin steseought| +14957|AAAAAAAAMGKDAAAA|1999-10-28|2001-10-26|Views cannot come databases; geographical securities speak tomorrow electoral, thin c|6.38|2.36|3003001|amalgamalg #2|1|dresses|1|Women|223|priableable|medium|55132white9670319491|plum|Box|Unknown|12|ationantin steseought| +14958|AAAAAAAAMGKDAAAA|2001-10-27||Views cannot come databases; geographical securities speak tomorrow electoral, thin c|46.60|2.36|7015009|scholarnameless #9|15|tables|7|Home|32|ablepri|N/A|5495757sandy45533368|peach|Each|Unknown|61|eingantin steseought| +14959|AAAAAAAAPGKDAAAA|1997-10-27||Also great policies should insert to the minutes. Old, spectacular examples get once again green days. Sales use around with the|4.90|1.76|3002002|importoexporti #2|2|infants|3|Children|298|eingn stable|large|507260702071yellow45|pale|Lb|Unknown|52|n stantin steseought| +14960|AAAAAAAAAHKDAAAA|1997-10-27|2000-10-26|Decisions play actually exclusive activities. Well assistant e|8.77|3.50|9003009|exportimaxi #9|3|computers|9|Books|20|barable|N/A|490591370627puff2340|sandy|Gross|Unknown|7|barcallyn steseought| +14961|AAAAAAAAAHKDAAAA|2000-10-27||Measures find late for a messages. Light terms shall no|1.19|0.76|9003009|namelessbrand #2|3|lighting|7|Home|111|barable|N/A|490591370627puff2340|powder|Tsp|Unknown|17|oughtcallyn steseought| +14962|AAAAAAAACHKDAAAA|1997-10-27|1999-10-27|Unlikely costs should risk low whole, new officials. Other eyes carry in the students. Main, lovely feelings must not allow|4.66|3.86|7002003|importobrand #3|2|bedding|7|Home|15|antiought|N/A|306049188seashell820|misty|Unknown|Unknown|85|ablecallyn steseought| +14963|AAAAAAAACHKDAAAA|1999-10-28|2001-10-26|Unlikely costs should risk low whole, new officials. Other eyes carry in the students. Main, lovely feelings must not allow|6.70|3.86|2003002|exportiimporto #2|2|pants|2|Men|602|ablebarcally|large|306049188seashell820|powder|Unknown|Unknown|31|pricallyn steseought| +14964|AAAAAAAACHKDAAAA|2001-10-27||Unlikely costs should risk low whole, new officials. Other eyes carry in the students. Main, lovely feelings must not allow|8.27|3.86|7016005|corpnameless #5|16|furniture|7|Home|291|ablebarcally|N/A|306049188seashell820|maroon|Bundle|Unknown|24|esecallyn steseought| +14965|AAAAAAAAFHKDAAAA|1997-10-27||Broad policies maintain id|8.32|6.15|1003002|exportiamalg #2|3|maternity|1|Women|112|ableoughtought|large|8midnight17661353096|tomato|Box|Unknown|88|anticallyn steseought| +14966|AAAAAAAAGHKDAAAA|1997-10-27|2000-10-26|Organisations shall guide tory organizations. Social, modest systems gro|7.74|5.26|9004003|edu packmaxi #3|4|entertainments|9|Books|422|ableableese|N/A|22702sky323663036795|midnight|Cup|Unknown|57|callycallyn steseought| +14967|AAAAAAAAGHKDAAAA|2000-10-27||Abroad soft others will become right. Growing, safe clothes keep ab|7.07|3.67|9004003|exportischolar #2|3|pop|5|Music|497|ableableese|N/A|22702sky323663036795|plum|Ounce|Unknown|53|ationcallyn steseought| +14968|AAAAAAAAIHKDAAAA|1997-10-27|1999-10-27|Invisible, excitin|9.35|4.76|2002001|importoimporto #1|2|shirts|2|Men|972|ableationn st|medium|57849912395619hot664|saddle|Pallet|Unknown|29|eingcallyn steseought| +14969|AAAAAAAAIHKDAAAA|1999-10-28|2001-10-26|Industrial parts justify manufacturing, certain successes. Western observations argue out of the devices; large, different fees must resign particular o|2.70|2.16|1004002|edu packamalg #2|4|swimwear|1|Women|516|ableationn st|small|354760272453plum4174|honeydew|Bunch|Unknown|23|n stcallyn steseought| +14970|AAAAAAAAIHKDAAAA|2001-10-27||Industrial parts justify manufacturing, certain successes. Western observations argue out of the devices; large, different fees must resign particular o|6.56|2.16|4002001|importoedu pack #1|4|mens|4|Shoes|516|ableationn st|medium|6turquoise0473827074|green|Pallet|Unknown|31|barationn steseought| +14971|AAAAAAAALHKDAAAA|1997-10-27||Costs send more schools. Causes start later. Both human |5.13|4.00|9014008|edu packunivamalg #8|14|sports|9|Books|639|n stprically|N/A|13366989353568slate7|salmon|Cup|Unknown|7|oughtationn steseought| +14972|AAAAAAAAMHKDAAAA|1997-10-27|2000-10-26|Desperate communications select indeed on a children. Reactions might debate. Scottish years learn so in a accidents. Major, broad shoulders say however late,|69.60|42.45|5003001|exportischolar #1|3|pop|5|Music|116|callyoughtought|N/A|75309rosy41696283387|papaya|Gram|Unknown|40|ableationn steseought| +14973|AAAAAAAAMHKDAAAA|2000-10-27||Integrated hands believe in a teeth. Foreign, basic parents might not help nume|9.84|42.45|5003001|scholaramalgamalg #14|3|portable|10|Electronics|887|ationeingeing|N/A|359913maroon90951051|sienna|Bundle|Unknown|13|priationn steseought| +14974|AAAAAAAAOHKDAAAA|1997-10-27|1999-10-27|Significant, internal words will respond very eyes. Too legal plates seek so |3.72|1.86|3002001|importoexporti #1|2|infants|3|Children|96|callyn st|extra large|481575966mint9036015|powder|Tsp|Unknown|20|eseationn steseought| +14975|AAAAAAAAOHKDAAAA|1999-10-28|2001-10-26|Quite other towns give together by a weeks. Concepts kn|8.54|1.86|5004002|edu packscholar #2|4|classical|5|Music|96|callyn st|N/A|firebrick33051477033|wheat|Dozen|Unknown|8|antiationn steseought| +14976|AAAAAAAAOHKDAAAA|2001-10-27||Wrong, unable fires maintain to the properties. Rich, environmental methods establish essentia|1.92|1.86|5003001|exportischolar #1|3|pop|5|Music|765|anticallyation|N/A|9559604peach01035817|slate|Carton|Unknown|16|callyationn steseought| +14977|AAAAAAAABIKDAAAA|1997-10-27||Open ro|2.18|1.85|10005001|scholarunivamalg #1|5|karoke|10|Electronics|317|ationoughtpri|N/A|66164465923532wheat4|orchid|Each|Unknown|3|ationationn steseought| +14978|AAAAAAAACIKDAAAA|1997-10-27|2000-10-26|Farmers see now by the minut|5.98|4.60|1004001|edu packamalg #1|4|swimwear|1|Women|8|eing|small|67437648961steel9518|olive|Tbl|Unknown|81|eingationn steseought| +14979|AAAAAAAACIKDAAAA|2000-10-27||Important lines contribute act|2.71|1.95|1004001|corpmaxi #4|4|parenting|9|Books|207|ationbarable|N/A|67437648961steel9518|violet|N/A|Unknown|62|n stationn steseought| +14980|AAAAAAAAEIKDAAAA|1997-10-27|1999-10-27|Beliefs used to sell too numerous issues. Major, whole companies murder nearly to a levels. Also electronic hills discover. Eyes try yet with a months. Following, round wars used t|0.50|0.27|10012004|importoamalgamalg #4|12|monitors|10|Electronics|520|barableanti|N/A|09253wheat2995963298|violet|Each|Unknown|14|bareingn steseought| +14981|AAAAAAAAEIKDAAAA|1999-10-28|2001-10-26|Beliefs used to sell too numerous issues. Major, whole companies murder nearly to a levels. Also electronic hills discover. Eyes try yet with a months. Following, round wars used t|8.90|0.27|10014005|edu packamalgamalg #5|14|automotive|10|Electronics|273|priationable|N/A|09253wheat2995963298|blue|Case|Unknown|32|oughteingn steseought| +14982|AAAAAAAAEIKDAAAA|2001-10-27||Indeed various items cross probably. Christian lines visit just large, big lakes. As short limits talk there most annual groups. Forms use for example players. Police discuss police. |0.13|0.27|10001006|amalgunivamalg #6|14|cameras|10|Electronics|929|priationable|N/A|949481050565lawn4356|dark|Tsp|Unknown|9|ableeingn steseought| +14983|AAAAAAAAHIKDAAAA|1997-10-27||Failures keep somewhat on the events. Also broad rates affo|9.53|4.66|3003002|exportiexporti #2|3|toddlers|3|Children|808|eingbareing|medium|740099169433blue1794|tan|N/A|Unknown|48|prieingn steseought| +14984|AAAAAAAAIIKDAAAA|1997-10-27|2000-10-26|Happy arts believe downwards out of a years. More true events say quite trying managers. Stages keep fast all internal facts. Ne|59.72|38.81|6008001|namelesscorp #1|8|mens watch|6|Jewelry|170|barationought|N/A|2434386638powder7226|wheat|N/A|Unknown|28|eseeingn steseought| +14985|AAAAAAAAIIKDAAAA|2000-10-27||Happy arts believe downwards out of a years. More true events say quite trying managers. Stages keep fast all internal facts. Ne|2.03|1.64|10014009|edu packamalgamalg #9|14|automotive|10|Electronics|170|barationought|N/A|821185963937743pale5|sandy|Pound|Unknown|16|antieingn steseought| +14986|AAAAAAAAKIKDAAAA|1997-10-27|1999-10-27|Sure eyes would sleep following, other mothers; organisations get indeed true walls. Legal interests receive safe terms. Increases seem jobs. Sub|1.14|0.64|4003001|exportiedu pack #1|3|kids|4|Shoes|131|oughtpriought|small|5956peru000428171963|white|Tsp|Unknown|20|callyeingn steseought| +14987|AAAAAAAAKIKDAAAA|1999-10-28|2001-10-26|Sure eyes would sleep following, other mothers; organisations get indeed true walls. Legal interests receive safe terms. Increases seem jobs. Sub|0.18|0.12|4003001|edu packexporti #2|3|school-uniforms|3|Children|131|oughtpriought|medium|263mint1545267800022|medium|Lb|Unknown|9|ationeingn steseought| +14988|AAAAAAAAKIKDAAAA|2001-10-27||Sure eyes would sleep following, other mothers; organisations get indeed true walls. Legal interests receive safe terms. Increases seem jobs. Sub|3.60|0.12|1004001|edu packamalg #1|4|swimwear|1|Women|297|oughtpriought|small|263mint1545267800022|plum|Lb|Unknown|64|eingeingn steseought| +14989|AAAAAAAANIKDAAAA|1997-10-27||Chemical effects relate formally behind|14.32|8.73|6005006|scholarcorp #6|5|earings|6|Jewelry|642|ableesecally|N/A|midnight496156762621|papaya|Bundle|Unknown|11|n steingn steseought| +14990|AAAAAAAAOIKDAAAA|1997-10-27|2000-10-26|Economic residents should not start though wrong actual proposals. Appointed, sufficient indians hear nuclear standards. Anyway cold things could not |1.67|0.80|3001001|amalgexporti #1|1|newborn|3|Children|552|ableantianti|medium|7451197727242puff308|saddle|Each|Unknown|15|barn stn steseought| +14991|AAAAAAAAOIKDAAAA|2000-10-27||Economic residents should not start though wrong actual proposals. Appointed, sufficient indians hear nuclear standards. Anyway cold things could not |55.34|18.81|3001001|importoimporto #2|2|shirts|2|Men|552|ableantianti|medium|7451197727242puff308|gainsboro|Carton|Unknown|42|oughtn stn steseought| +14992|AAAAAAAAAJKDAAAA|1997-10-27|1999-10-27|Today literary controls could start white glasses. Below individual police ought to succeed old projects. New races must belong in addition british, little types. Others lend cautious|5.12|2.04|1002001|importoamalg #1|2|fragrances|1|Women|195|antin stought|extra large|8429519654slate84474|linen|Tsp|Unknown|28|ablen stn steseought| +14993|AAAAAAAAAJKDAAAA|1999-10-28|2001-10-26|Inside favorite women deal over fields. Teeth ought to know severely. Public pictures tell more federal sciences; as|2.72|2.04|1002001|importoamalg #2|2|fragrances|1|Women|195|antin stought|extra large|642olive837895859348|linen|Ounce|Unknown|32|prin stn steseought| +14994|AAAAAAAAAJKDAAAA|2001-10-27||Inside favorite women deal over fields. Teeth ought to know severely. Public pictures tell more federal sciences; as|6.43|2.04|1002001|exportiunivamalg #7|2|self-help|9|Books|374|antin stought|N/A|53wheat2694156531067|seashell|Pallet|Unknown|29|esen stn steseought| +14995|AAAAAAAADJKDAAAA|1997-10-27||Social, only relationships speak even societies. Most basic areas cover basic things; main resourc|93.54|42.09|2002002|importoimporto #2|2|shirts|2|Men|233|pripriable|large|16513seashell7613742|sky|N/A|Unknown|88|antin stn steseought| +14996|AAAAAAAAEJKDAAAA|1997-10-27|2000-10-26|Little, past years need actually architects. Material, quiet clouds pretend a little under a areas. Exhibitions ought to sit both to a years. Now rural tonnes survive simultaneously visual, rea|45.95|23.89|3003001|exportiexporti #1|3|toddlers|3|Children|79|n station|N/A|0507981311947powder4|saddle|Ton|Unknown|18|callyn stn steseought| +14997|AAAAAAAAEJKDAAAA|2000-10-27||Little, past years need actually architects. Material, quiet clouds pretend a little under a areas. Exhibitions ought to sit both to a years. Now rural tonnes survive simultaneously visual, rea|8.25|7.42|3003001|importomaxi #8|3|guns|8|Sports|30|barpri|N/A|0507981311947powder4|rose|Ton|Unknown|72|ationn stn steseought| +14998|AAAAAAAAGJKDAAAA|1997-10-27|1999-10-27|Experienced theories shall square clergy. Difficult, religious sales observe. Yet past movies travel properly defences. Financial resources think more different eyes. Objec|4.65|2.37|6008007|namelesscorp #7|8|mens watch|6|Jewelry|458|eingantiese|N/A|917149floral69532688|grey|Pound|Unknown|20|eingn stn steseought| +14999|AAAAAAAAGJKDAAAA|1999-10-28|2001-10-26|There domestic letters call. Topics sell low factors. General, tall specialists might determine by the regulations|6.36|2.16|8007010|brandnameless #10|8|hockey|8|Sports|458|eingantiese|N/A|917149floral69532688|turquoise|Gram|Unknown|21|n stn stn steseought| +15000|AAAAAAAAGJKDAAAA|2001-10-27||There domestic letters call. Topics sell low factors. General, tall specialists might determine by the regulations|4.43|2.16|6008001|namelesscorp #1|8|mens watch|6|Jewelry|458|eingantiese|N/A|917149floral69532688|purple|Gram|Unknown|80|barbarbarantiought| +15001|AAAAAAAAJJKDAAAA|1997-10-27||Disputes should not ensure. Spaces might see neither. Of course other students love. Proud bodies must wait too. New ways shall not ensure interests; professiona|21.30|12.78|4004002|edu packedu pack #2|4|athletic|4|Shoes|890|barn steing|N/A|58575708099664snow48|steel|Pallet|Unknown|93|oughtbarbarantiought| +15002|AAAAAAAAKJKDAAAA|1997-10-27|2000-10-26|Political, following police fit perceptions. Also vague recommendations retaliate less other scientists. Economic interests see jus|1.56|0.46|4001001|amalgedu pack #1|1|womens|4|Shoes|253|priantiable|medium|7481377830439olive63|orange|Cup|Unknown|46|ablebarbarantiought| +15003|AAAAAAAAKJKDAAAA|2000-10-27||Political, following police fit perceptions. Also vague recommendations retaliate less other scientists. Economic interests see jus|74.61|38.05|2001002|amalgimporto #2|1|accessories|2|Men|253|priantiable|large|7481377830439olive63|ghost|Pallet|Unknown|24|pribarbarantiought| +15004|AAAAAAAAMJKDAAAA|1997-10-27|1999-10-27|Outstanding goods may not build almost|1.32|0.96|5001001|amalgscholar #1|1|rock|5|Music|219|n stoughtable|N/A|54074777thistle45906|sandy|Pound|Unknown|29|esebarbarantiought| +15005|AAAAAAAAMJKDAAAA|1999-10-28|2001-10-26|British shares respond foreign, economic things. Great, normal teachers restructure almost |5.40|3.88|5001001|exportischolar #2|3|pop|5|Music|219|n stoughtable|N/A|37677951230mint09151|powder|Carton|Unknown|52|antibarbarantiought| +15006|AAAAAAAAMJKDAAAA|2001-10-27||Legal, distinct services should not mean really extra agreements. Inc, pleasant males widen. Merely human effects can benefit so voluntary advertisements. C|2.40|3.88|8014007|edu packmaxi #7|3|tennis|8|Sports|156|callyantiought|N/A|3turquoise9351456830|tan|Lb|Unknown|77|callybarbarantiought| +15007|AAAAAAAAPJKDAAAA|1997-10-27||Difficulties might move.|35.78|14.66|5003002|exportischolar #2|3|pop|5|Music|314|eseoughtpri|N/A|7711plum716915038182|olive|Cup|Unknown|16|ationbarbarantiought| +15008|AAAAAAAAAKKDAAAA|1997-10-27|2000-10-26|All right deliberate difficulties wait still between a seats; final, actual jobs may mee|8.41|4.20|5001001|amalgscholar #1|1|rock|5|Music|937|ationprin st|N/A|5610090papaya9643394|salmon|Oz|Unknown|67|eingbarbarantiought| +15009|AAAAAAAAAKKDAAAA|2000-10-27||Division|2.23|4.20|5001001|corpmaxi #10|6|parenting|9|Books|201|oughtbarable|N/A|5610090papaya9643394|rosy|Ton|Unknown|10|n stbarbarantiought| +15010|AAAAAAAACKKDAAAA|1997-10-27|1999-10-27|Russians think wryly all red markets; other proposals must risk without the rates. O|49.67|26.32|8003001|exportinameless #1|3|basketball|8|Sports|127|ationableought|N/A|267plum4068345969493|tan|Ton|Unknown|12|baroughtbarantiought| +15011|AAAAAAAACKKDAAAA|1999-10-28|2001-10-26|Bold pieces could not review days; foreign systems must feel products. Amounts would open about systems. Moments join exciting, political telecommunications. Long-term places c|33.12|26.82|9010012|univunivamalg #12|10|travel|9|Books|127|ationableought|N/A|0994315909snow023311|indian|Ounce|Unknown|9|oughtoughtbarantiought| +15012|AAAAAAAACKKDAAAA|2001-10-27||Bold pieces could not review days; foreign systems must feel products. Amounts would open about systems. Moments join exciting, political telecommunications. Long-term places c|35.42|14.52|1001001|amalgamalg #1|10|dresses|1|Women|199|ationableought|medium|066slate107460219019|sky|Cup|Unknown|14|ableoughtbarantiought| +15013|AAAAAAAAFKKDAAAA|1997-10-27||Governments could see also. Policies used to rely only new dealers. Boats used to participate then for a forests. Front banks breathe behind a wings; i|7.46|2.90|7014004|edu packnameless #4|14|glassware|7|Home|109|n stbarought|N/A|84549463powder712674|grey|Cup|Unknown|16|prioughtbarantiought| +15014|AAAAAAAAGKKDAAAA|1997-10-27|2000-10-26|Indeed sensitive words used to come significantly alone european matters. Sufficient, arbitrary attitudes should come again local, foreign comparisons. Public, patient initiatives migh|1.97|0.66|2003001|exportiimporto #1|3|pants|2|Men|58|einganti|large|2tan3707765468580212|sky|Unknown|Unknown|25|eseoughtbarantiought| +15015|AAAAAAAAGKKDAAAA|2000-10-27||Indeed sensitive words used to come significantly alone european matters. Sufficient, arbitrary attitudes should come again local, foreign comparisons. Public, patient initiatives migh|4.98|0.66|7016006|corpnameless #6|16|furniture|7|Home|58|einganti|N/A|3354409121274plum781|pale|Ounce|Unknown|56|antioughtbarantiought| +15016|AAAAAAAAIKKDAAAA|1997-10-27|1999-10-27|Able|87.47|65.60|3003001|exportiexporti #1|3|toddlers|3|Children|243|prieseable|extra large|sandy385273424293924|papaya|Cup|Unknown|16|callyoughtbarantiought| +15017|AAAAAAAAIKKDAAAA|1999-10-28|2001-10-26|Extremely civil technologies will not pay even from a rivals. Other schools might not meet friends; units must not buy too relative, expensive jobs. Good questions appear programmes. Large|4.03|65.60|10013007|exportiamalgamalg #7|13|stereo|10|Electronics|375|antiationpri|N/A|845925259340indian09|red|Tsp|Unknown|63|ationoughtbarantiought| +15018|AAAAAAAAIKKDAAAA|2001-10-27||Extremely civil technologies will not pay even from a rivals. Other schools might not meet friends; units must not buy too relative, expensive jobs. Good questions appear programmes. Large|29.74|17.84|5004001|edu packscholar #1|4|classical|5|Music|375|antiationpri|N/A|98metallic5152184564|chartreuse|Ounce|Unknown|36|eingoughtbarantiought| +15019|AAAAAAAALKKDAAAA|1997-10-27||Consistent things survive good, blue views. Later urban daughters know subsequently as a sites. New events |4.82|1.44|3001002|amalgexporti #2|1|newborn|3|Children|12|ableought|petite|696seashell933541217|saddle|Box|Unknown|66|n stoughtbarantiought| +15020|AAAAAAAAMKKDAAAA|1997-10-27|2000-10-26|Literally special schemes steady nuclear houses. Certain, working rules shift. Productive, determined mech|6.38|2.87|4003001|exportiedu pack #1|3|kids|4|Shoes|228|eingableable|N/A|56625319364053slate7|white|Tsp|Unknown|43|barablebarantiought| +15021|AAAAAAAAMKKDAAAA|2000-10-27||Prayers will not pick also local areas. Strongly other systems belong surely in a fortunes; anywhere happy stars communicate healthy, atlantic blocks. References may not move yet really availabl|4.56|3.96|4003001|exportiamalgamalg #11|3|stereo|10|Electronics|343|eingableable|N/A|56625319364053slate7|dim|Gram|Unknown|57|oughtablebarantiought| +15022|AAAAAAAAOKKDAAAA|1997-10-27|1999-10-27|Contexts supply suddenly; new, smooth phrases alter only apparently new specialists. Loca|1.03|0.92|10016012|corpamalgamalg #12|16|wireless|10|Electronics|527|ationableanti|N/A|5117809midnight46598|spring|Dozen|Unknown|50|ableablebarantiought| +15023|AAAAAAAAOKKDAAAA|1999-10-28|2001-10-26|Contexts supply suddenly; new, smooth phrases alter only apparently new specialists. Loca|10.77|4.20|10016012|amalgedu pack #2|1|womens|4|Shoes|527|ationableanti|extra large|63778883731snow90411|tomato|Carton|Unknown|10|priablebarantiought| +15024|AAAAAAAAOKKDAAAA|2001-10-27||Contexts supply suddenly; new, smooth phrases alter only apparently new specialists. Loca|4.67|1.58|10005014|scholarunivamalg #14|1|karoke|10|Electronics|527|ationableanti|N/A|719370026631696rosy0|wheat|Pallet|Unknown|65|eseablebarantiought| +15025|AAAAAAAABLKDAAAA|1997-10-27||Firstly other eyes can compete obviously. Immediate, public studies keep in a factors; religious, political scientists can say early linguistic counties; left, result|4.25|2.33|2001002|amalgimporto #2|1|accessories|2|Men|300|barbarpri|medium|4895499449turquoise0|lace|Oz|Unknown|5|antiablebarantiought| +15026|AAAAAAAACLKDAAAA|1997-10-27|2000-10-26|Experts would focus fingers. Difficult stairs b|9.96|7.96|5002001|importoscholar #1|2|country|5|Music|278|eingationable|N/A|811002848993thistle6|tomato|Tbl|Unknown|5|callyablebarantiought| +15027|AAAAAAAACLKDAAAA|2000-10-27||Categories get interestingly in a players. Theories promo|4.12|7.96|5002001|corpunivamalg #4|2|mystery|9|Books|278|eingationable|N/A|811002848993thistle6|thistle|Lb|Unknown|37|ationablebarantiought| +15028|AAAAAAAAELKDAAAA|1997-10-27|1999-10-27|Most previous beliefs must make ago addresses. Bills used to use strangely. Individual, gr|8.98|7.27|2001001|amalgimporto #1|1|accessories|2|Men|788|eingeingation|large|41goldenrod616324950|thistle|Bunch|Unknown|19|eingablebarantiought| +15029|AAAAAAAAELKDAAAA|1999-10-28|2001-10-26|Never clear reports know married relations. Particular teeth should lift generally different women; simple, major dogs must not create always. Good examples back neve|7.84|6.74|6015006|scholarbrand #6|1|custom|6|Jewelry|156|eingeingation|N/A|469225523618sandy178|gainsboro|Ton|Unknown|25|n stablebarantiought| +15030|AAAAAAAAELKDAAAA|2001-10-27||About |21.04|14.51|1002001|importoamalg #1|2|fragrances|1|Women|659|n stantically|petite|89898908102saddle995|purple|Dozen|Unknown|4|barpribarantiought| +15031|AAAAAAAAHLKDAAAA|1997-10-27||Lips add for a effects. Concerned, very visits shall finish single budgets. Now interesting hands shall hear. Respectively english vegetables deal hotly. Combined, fascinating farme|1.39|0.69|4003002|exportiedu pack #2|3|kids|4|Shoes|166|callycallyought|medium|42328304598911sandy5|white|N/A|Unknown|63|oughtpribarantiought| +15032|AAAAAAAAILKDAAAA|1997-10-27|2000-10-26|High, real differences continue. Relatively electronic yards find for a months. Anyw|6.11|3.54|7005003|scholarbrand #3|5|blinds/shades|7|Home|58|einganti|N/A|365royal631022131173|smoke|Ton|Unknown|82|ablepribarantiought| +15033|AAAAAAAAILKDAAAA|2000-10-27||High, real differences continue. Relatively electronic yards find for a months. Anyw|0.80|3.54|2002002|importoimporto #2|2|shirts|2|Men|77|ationation|extra large|39snow65119493050130|saddle|Carton|Unknown|34|pripribarantiought| +15034|AAAAAAAAKLKDAAAA|1997-10-27|1999-10-27|Faces tell significantly artists. Much central steps should project. Enthusiastic, male schemes might pla|2.63|1.18|1003001|exportiamalg #1|3|maternity|1|Women|598|eingn stanti|economy|139420159907peach590|turquoise|Dozen|Unknown|49|esepribarantiought| +15035|AAAAAAAAKLKDAAAA|1999-10-28|2001-10-26|Delightful, primary services will use professional, economic materials. Electric years shall not complain regions. Then maximum pockets complete in a faces. Improvements go from time to ti|3.47|1.18|1003001|importomaxi #6|12|guns|8|Sports|598|eingn stanti|N/A|139420159907peach590|sienna|Pound|Unknown|59|antipribarantiought| +15036|AAAAAAAAKLKDAAAA|2001-10-27||Delightful, primary services will use professional, economic materials. Electric years shall not complain regions. Then maximum pockets complete in a faces. Improvements go from time to ti|2.43|1.18|5003001|exportischolar #1|12|pop|5|Music|20|eingn stanti|N/A|1203615snow022035988|olive|Oz|Unknown|13|callypribarantiought| +15037|AAAAAAAANLKDAAAA|1997-10-27||Intentions should understand however controversial colleagues; just clear letters ought to last inadequate pictures. Changes control upon a terms. Very imposs|1.43|0.68|4003002|exportiedu pack #2|3|kids|4|Shoes|51|oughtanti|extra large|5red5537664512063546|turquoise|Ounce|Unknown|36|ationpribarantiought| +15038|AAAAAAAAOLKDAAAA|1997-10-27|2000-10-26|Ag|39.14|19.96|1001001|amalgamalg #1|1|dresses|1|Women|175|antiationought|economy|4686572615946rose374|hot|Carton|Unknown|18|eingpribarantiought| +15039|AAAAAAAAOLKDAAAA|2000-10-27||Ag|4.20|3.06|10006012|corpunivamalg #12|1|musical|10|Electronics|175|antiationought|N/A|5715387yellow9819909|saddle|Pound|Unknown|24|n stpribarantiought| +15040|AAAAAAAAAMKDAAAA|1997-10-27|1999-10-27|Theoretical, expert m|0.14|0.10|1002001|importoamalg #1|2|fragrances|1|Women|633|priprically|large|88730018948linen8236|tan|Box|Unknown|45|baresebarantiought| +15041|AAAAAAAAAMKDAAAA|1999-10-28|2001-10-26|Theoretical, expert m|2.22|0.10|4001002|amalgedu pack #2|2|womens|4|Shoes|633|priprically|medium|88730018948linen8236|powder|Each|Unknown|17|oughtesebarantiought| +15042|AAAAAAAAAMKDAAAA|2001-10-27||Theoretical, expert m|9.83|8.06|4001002|amalgedu pack #1|1|womens|4|Shoes|633|priprically|extra large|88730018948linen8236|steel|N/A|Unknown|31|ableesebarantiought| +15043|AAAAAAAADMKDAAAA|1997-10-27||Local rights may not grow that is. Frequent purposes control invariably social men. Far brow|6.16|4.00|6013004|exportibrand #4|13|loose stones|6|Jewelry|317|ationoughtpri|N/A|32281bisque110260244|medium|Gram|Unknown|42|priesebarantiought| +15044|AAAAAAAAEMKDAAAA|1997-10-27|2000-10-26|Famous, working bodies finish at all other, good f|2.08|1.18|6009005|maxicorp #5|9|womens watch|6|Jewelry|336|callypripri|N/A|8395puff936545966640|bisque|Ounce|Unknown|87|eseesebarantiought| +15045|AAAAAAAAEMKDAAAA|2000-10-27||Famous, working bodies finish at all other, good f|9.57|1.18|2003002|exportiimporto #2|9|pants|2|Men|129|n stableought|large|8395puff936545966640|sky|Cup|Unknown|30|antiesebarantiought| +15046|AAAAAAAAGMKDAAAA|1997-10-27|1999-10-27|Best odd changes used to pass underlying minutes; good others could |4.29|2.91|8016007|corpmaxi #7|16|golf|8|Sports|631|oughtprically|N/A|21royal5190946618091|rose|Each|Unknown|51|callyesebarantiought| +15047|AAAAAAAAGMKDAAAA|1999-10-28|2001-10-26|Best odd changes used to pass underlying minutes; good others could |4.79|2.87|8016007|importoedu pack #2|2|mens|4|Shoes|249|oughtprically|medium|21royal5190946618091|gainsboro|N/A|Unknown|16|ationesebarantiought| +15048|AAAAAAAAGMKDAAAA|2001-10-27||Cars may send. Companies consider true, left p|8.12|3.49|1003001|exportiamalg #1|2|maternity|1|Women|333|pripripri|N/A|21royal5190946618091|plum|Box|Unknown|8|eingesebarantiought| +15049|AAAAAAAAJMKDAAAA|1997-10-27||Even main changes might not break great, new arms. Imaginative children accept then difficult, sure leaders. Questions market commercial girls. Libraries should win as other classes. Stars serve after|7.77|3.34|7013010|exportinameless #10|13|wallpaper|7|Home|896|callyn steing|N/A|8176059343311white60|turquoise|Pallet|Unknown|68|n stesebarantiought| +15050|AAAAAAAAKMKDAAAA|1997-10-27|2000-10-26|Completely foreign lips could not try as. Investors concern more linear children. Sacred habits walk dramatically from a year|3.17|0.95|1002001|importoamalg #1|2|fragrances|1|Women|24|eseable|medium|92563589ghost5513398|salmon|N/A|Unknown|58|barantibarantiought| +15051|AAAAAAAAKMKDAAAA|2000-10-27||Then soft operations show then |1.44|0.74|1002001|amalgimporto #2|1|accessories|2|Men|24|eseable|small|92563589ghost5513398|purple|Gram|Unknown|31|oughtantibarantiought| +15052|AAAAAAAAMMKDAAAA|1997-10-27|1999-10-27|Lines must set away features. Organization|43.77|21.44|3001001|amalgexporti #1|1|newborn|3|Children|295|antin stable|medium|4725386911antique810|papaya|Carton|Unknown|1|ableantibarantiought| +15053|AAAAAAAAMMKDAAAA|1999-10-28|2001-10-26|Only terrible comments know surely black districts. Outer, spectacular materials delegate so above a reactions. Positive costs may force thus parent|9.44|21.44|3001001|edu packedu pack #2|4|athletic|4|Shoes|59|n stanti|medium|4725386911antique810|steel|Tbl|Unknown|24|priantibarantiought| +15054|AAAAAAAAMMKDAAAA|2001-10-27||Large senten|4.41|3.17|10001010|amalgunivamalg #10|1|cameras|10|Electronics|59|n stanti|N/A|2470207461472misty05|papaya|Ton|Unknown|20|eseantibarantiought| +15055|AAAAAAAAPMKDAAAA|1997-10-27||Fair successful advertisements will s|4.01|3.04|6001008|amalgcorp #8|1|birdal|6|Jewelry|252|ableantiable|N/A|5144916308sienna3385|rose|Each|Unknown|13|antiantibarantiought| +15056|AAAAAAAAANKDAAAA|1997-10-27|2000-10-26|Perfectly other documents respect almost; wide capital prices put quiet months. Please professi|4.01|2.60|7003007|exportibrand #7|3|kids|7|Home|281|oughteingable|N/A|2753340rosy372781248|yellow|Pallet|Unknown|44|callyantibarantiought| +15057|AAAAAAAAANKDAAAA|2000-10-27||Recent statements must spare payments. Drivers allow to the services. Damp |3.08|1.07|3002002|importoexporti #2|2|infants|3|Children|538|eingprianti|medium|2753340rosy372781248|yellow|Box|Unknown|17|ationantibarantiought| +15058|AAAAAAAACNKDAAAA|1997-10-27|1999-10-27|Preferably good events shall sit often cold national pu|2.44|1.41|7015009|scholarnameless #9|15|tables|7|Home|61|oughtcally|N/A|256695022purple69963|violet|Tsp|Unknown|60|eingantibarantiought| +15059|AAAAAAAACNKDAAAA|1999-10-28|2001-10-26|Preferably good events shall sit often cold national pu|3.12|1.41|8014010|edu packmaxi #10|15|tennis|8|Sports|207|oughtcally|N/A|256695022purple69963|violet|Ounce|Unknown|49|n stantibarantiought| +15060|AAAAAAAACNKDAAAA|2001-10-27||Preferably good events shall sit often cold national pu|78.29|64.19|5002001|importoscholar #1|2|country|5|Music|207|oughtcally|N/A|256695022purple69963|chartreuse|Box|Unknown|74|barcallybarantiought| +15061|AAAAAAAAFNKDAAAA|1997-10-27||Pale, normal schools used to separate long-term, significant drug|1.48|0.59|8013002|exportimaxi #2|13|sailing|8|Sports|521|oughtableanti|N/A|57469685681snow02193|slate|Pallet|Unknown|36|oughtcallybarantiought| +15062|AAAAAAAAGNKDAAAA|1997-10-27|2000-10-26|At least literary months might arise incomes. Just industrial fingers use only precise agreements. Also spanish hands could perform through the communications. So as beautiful |1.39||7016003|corpnameless #3||furniture|7|Home|985|||||Tsp|||ablecallybarantiought| +15063|AAAAAAAAGNKDAAAA|2000-10-27||Certainly different indicators lie corporate, other applications. Central, major parties allow thin, economic tonnes. Offenders could|3.67|1.17|10007002|brandunivamalg #2|7|personal|10|Electronics|985|antieingn st|N/A|77725011red940995163|wheat|N/A|Unknown|39|pricallybarantiought| +15064|AAAAAAAAINKDAAAA|1997-10-27|1999-10-27|Available, friendly articles call liable weeks; quarters lose hopefully hard theories; low schools say still overseas questions; interior hours used to seek rat|4.65|3.30|2003001|exportiimporto #1|3|pants|2|Men|321|oughtablepri|small|5light85212949392222|powder|Each|Unknown|43|esecallybarantiought| +15065|AAAAAAAAINKDAAAA|1999-10-28|2001-10-26|Available, friendly articles call liable weeks; quarters lose hopefully hard theories; low schools say still overseas questions; interior hours used to seek rat|2.21|3.30|2003001|amalgunivamalg #6|11|cooking|9|Books|321|oughtablepri|N/A|6263530steel53195512|smoke|Carton|Unknown|56|anticallybarantiought| +15066|AAAAAAAAINKDAAAA|2001-10-27||Available, friendly articles call liable weeks; quarters lose hopefully hard theories; low schools say still overseas questions; interior hours used to seek rat|2.54|3.30|2003001|edu packimporto #1|11|sports-apparel|2|Men|321|oughtablepri|medium|331843thistle0736555|wheat|Lb|Unknown|9|callycallybarantiought| +15067|AAAAAAAALNKDAAAA|1997-10-27||Inevitably busy voters re|1.07|0.56|6008004|namelesscorp #4|8|mens watch|6|Jewelry|663|pricallycally|N/A|7wheat51935973781755|purple|Pound|Unknown|12|ationcallybarantiought| +15068|AAAAAAAAMNKDAAAA|1997-10-27|2000-10-26|Payments used to understand about mothers. |3.19|1.91|7010009|univnameless #9|10|flatware|7|Home|79|n station|N/A|55pink56807515669034|olive|Carton|Unknown|1|eingcallybarantiought| +15069|AAAAAAAAMNKDAAAA|2000-10-27||Payments used to understand about mothers. |7.64|1.91|7010009|exportinameless #10|10|wallpaper|7|Home|280|bareingable|N/A|005maroon10744227390|tan|Box|Unknown|96|n stcallybarantiought| +15070|AAAAAAAAONKDAAAA|1997-10-27|1999-10-27|Welcome men should reveal recently together with a advantages. Years hide at all. Skills could not find big, lucky homes. Difficult markets could not hear|4.88|2.34|5003001|exportischolar #1|3|pop|5|Music|38|eingpri|N/A|833875linen507861885|saddle|Ton|Unknown|64|barationbarantiought| +15071|AAAAAAAAONKDAAAA|1999-10-28|2001-10-26|Welcome men should reveal recently together with a advantages. Years hide at all. Skills could not find big, lucky homes. Difficult markets could not hear|7.39|2.34|7006002|corpbrand #2|3|rugs|7|Home|38|eingpri|N/A|833875linen507861885|tan|Carton|Unknown|68|oughtationbarantiought| +15072|AAAAAAAAONKDAAAA|2001-10-27||Welcome men should reveal recently together with a advantages. Years hide at all. Skills could not find big, lucky homes. Difficult markets could not hear|27.57|14.61|7006002|exportischolar #1|3|pop|5|Music|74|eingpri|N/A|70184midnight2454594|lemon|Gram|Unknown|66|ableationbarantiought| +15073|AAAAAAAABOKDAAAA|1997-10-27||Immediate women sell. Slowly sure measures might ensure in order from a duties. Privately aware damages may compensate so for example previous letters|2.97|1.78|1002002|importoamalg #2|2|fragrances|1|Women|524|eseableanti|large|095920maroon41278945|navajo|Bunch|Unknown|42|priationbarantiought| +15074|AAAAAAAACOKDAAAA|1997-10-27|2000-10-26|Years attend earlier connections. Enormous, smart kinds ought to close formerly. Just excell|7.30|3.72|2003001|exportiimporto #1|3|pants|2|Men|288|eingeingable|medium|7752900magenta510149|purple|Dram|Unknown|8|eseationbarantiought| +15075|AAAAAAAACOKDAAAA|2000-10-27||Attitudes unlock small increases; statistical, important fam|56.67|3.72|2003001|amalgimporto #2|1|accessories|2|Men|750|barantiation|petite|7752900magenta510149|goldenrod|Ton|Unknown|98|antiationbarantiought| +15076|AAAAAAAAEOKDAAAA|1997-10-27|1999-10-27|Almost subject men could add more huge, current customers. Major colours |0.22|0.10|7002007|importobrand #7|2|bedding|7|Home|371|oughtationpri|N/A|2802386585tan4100444|powder|Ounce|Unknown|79|callyationbarantiought| +15077|AAAAAAAAEOKDAAAA|1999-10-28|2001-10-26|Almost subject men could add more huge, current customers. Major colours |4.03|2.82|7002007|exportiamalg #2|3|maternity|1|Women|371|oughtationpri|extra large|2802386585tan4100444|turquoise|Bunch|Unknown|18|ationationbarantiought| +15078|AAAAAAAAEOKDAAAA|2001-10-27||High, open expenses put also. Generous members shall keep lazily for a offices. Surely extra ports weave else. Able|2.36|0.70|3003001|exportiexporti #1|3|toddlers|3|Children|17|ationought|medium|2802386585tan4100444|rosy|Ton|Unknown|72|eingationbarantiought| +15079|AAAAAAAAHOKDAAAA|1997-10-27||Still good patients subsidise damp couples; never high systems save low available, european years. Teams appear also examples.|4.57|1.69|2004002|edu packimporto #2|4|sports-apparel|2|Men|145|antieseought|N/A|5671594149plum953615|orchid|Gross|Unknown|62|n stationbarantiought| +15080|AAAAAAAAIOKDAAAA|1997-10-27|2000-10-26|Never equal islands go. Public groups could set just to a |4.24|2.50|2003001|exportiimporto #1|3|pants|2|Men|538|eingprianti|small|396666800474285grey8|red|Dozen|Unknown|65|bareingbarantiought| +15081|AAAAAAAAIOKDAAAA|2000-10-27||Never equal islands go. Public groups could set just to a |4.01|3.32|4004002|edu packedu pack #2|4|athletic|4|Shoes|205|antibarable|small|396666800474285grey8|peach|Unknown|Unknown|23|oughteingbarantiought| +15082|AAAAAAAAKOKDAAAA|1997-10-27|1999-10-27|Only cold differences operate probably isolated figures. Old, specific elements wou|3.95|3.39|2001001|amalgimporto #1|1|accessories|2|Men|662|ablecallycally|small|30356rose43827417043|seashell|Gross|Unknown|84|ableeingbarantiought| +15083|AAAAAAAAKOKDAAAA|1999-10-28|2001-10-26|There local products stop individual terms. Open attempts shall agree before. Economic subsidies defuse to a practitioners. Very interesting repo|2.84|2.07|1004002|edu packamalg #2|4|swimwear|1|Women|308|eingbarpri|large|42166738800misty3963|green|Cup|Unknown|31|prieingbarantiought| +15084|AAAAAAAAKOKDAAAA|2001-10-27||There local products stop individual terms. Open attempts shall agree before. Economic subsidies defuse to a practitioners. Very interesting repo|6.20|2.07|1004002|univunivamalg #1|10|travel|9|Books|646|callyesecally|N/A|42166738800misty3963|plum|Dram|Unknown|18|eseeingbarantiought| +15085|AAAAAAAANOKDAAAA|1997-10-27||Too contemporary ideas measure now as a teeth. Only modern problems concentrate local animals. Whole regulations shall put as texts; also magnetic homes could not explain also types. Car|6.02|2.04|7011006|amalgnameless #6|11|accent|7|Home|263|pricallyable|N/A|spring99179119126932|seashell|Bunch|Unknown|11|antieingbarantiought| +15086|AAAAAAAAOOKDAAAA|1997-10-27|2000-10-26|Real, human elections find auditors. Black employees would comply. Bad eyes sell recent lines. Obvious issues describe|7.40|6.36|7011007|amalgnameless #7|11|accent|7|Home|517|ationoughtanti|N/A|660962116tan15116602|papaya|Carton|Unknown|21|callyeingbarantiought| +15087|AAAAAAAAOOKDAAAA|2000-10-27||Real, human elections find auditors. Black employees would comply. Bad eyes sell recent lines. Obvious issues describe|7.94|6.90|7011007|exportimaxi #8|11|sailing|8|Sports|517|ationoughtanti|N/A|44379yellow777706985|grey|Tsp|Unknown|10|ationeingbarantiought| +15088|AAAAAAAAAPKDAAAA|1997-10-27|1999-10-27|Following friends exceed bodies; small stages look on a lines. Comfortable books send in a numb|59.78|50.21|7011009|amalgnameless #9|11|accent|7|Home|146|callyeseought|N/A|11239625914peach5718|white|Cup|Unknown|9|eingeingbarantiought| +15089|AAAAAAAAAPKDAAAA|1999-10-28|2001-10-26|Following friends exceed bodies; small stages look on a lines. Comfortable books send in a numb|7.89|50.21|5004002|edu packscholar #2|4|classical|5|Music|146|callyeseought|N/A|11239625914peach5718|saddle|N/A|Unknown|59|n steingbarantiought| +15090|AAAAAAAAAPKDAAAA|2001-10-27||Following friends exceed bodies; small stages look on a lines. Comfortable books send in a numb|9.50|50.21|7011001|amalgnameless #1|4|accent|7|Home|146|callyeseought|N/A|2563465grey481594372|pale|Tbl|Unknown|20|barn stbarantiought| +15091|AAAAAAAADPKDAAAA|1997-10-27||Really special ministers supplement men. Relevant, new goods tell about at least ot|4.89|3.86|4002002|importoedu pack #2|2|mens|4|Shoes|41|oughtese|extra large|720pale1534432366477|khaki|Tbl|Unknown|27|oughtn stbarantiought| +15092|AAAAAAAAEPKDAAAA|1997-10-27|2000-10-26|Simple, pure services serve to a students. Sometimes yellow tons should give together alone |1.62|0.55|4001001|amalgedu pack #1|1|womens|4|Shoes|298|eingn stable|medium|458835052840slate957|lemon|N/A|Unknown|5|ablen stbarantiought| +15093|AAAAAAAAEPKDAAAA|2000-10-27||Simple, pure services serve to a students. Sometimes yellow tons should give together alone |6.09|0.55|4001001|scholaramalgamalg #15|1|portable|10|Electronics|128|eingn stable|N/A|04tan590149035997105|yellow|Tsp|Unknown|90|prin stbarantiought| +15094|AAAAAAAAGPKDAAAA|1997-10-27|1999-10-27|Medical, uni|1.00|0.61|2003001|exportiimporto #1|3|pants|2|Men|392|ablen stpri|N/A|38808slate9531460256|honeydew|Case|Unknown|61|esen stbarantiought| +15095|AAAAAAAAGPKDAAAA|1999-10-28|2001-10-26|Medical, uni|9.36|6.08|1004002|edu packamalg #2|3|swimwear|1|Women|392|ablen stpri|extra large|500olive372215214347|rose|Tsp|Unknown|2|antin stbarantiought| +15096|AAAAAAAAGPKDAAAA|2001-10-27||Services will choose fully external, local workers. Suitable members may help interestingly agencies. Certain, likely citizens would thi|8.53|7.50|1003001|exportiamalg #1|3|maternity|1|Women|311|ablen stpri|small|708652149slate375302|pink|Box|Unknown|11|callyn stbarantiought| +15097|AAAAAAAAJPKDAAAA|1997-10-27||Recent |0.35|0.25|7002008|importobrand #8|2|bedding|7|Home|160|barcallyought|N/A|289312904006035snow7|royal|Oz|Unknown|13|ationn stbarantiought| +15098|AAAAAAAAKPKDAAAA|1997-10-27|2000-10-26|Late |59.90|35.34|1003001|exportiamalg #1|3|maternity|1|Women|628|eingablecally|large|8919snow928064640376|red|Dram|Unknown|47|eingn stbarantiought| +15099|AAAAAAAAKPKDAAAA|2000-10-27||Late |5.12|3.22|3002002|importoexporti #2|2|infants|3|Children|628|eingablecally|large|8919snow928064640376|light|Oz|Unknown|55|n stn stbarantiought| +15100|AAAAAAAAMPKDAAAA|1997-10-27|1999-10-27|Domestic doors s|23.24|16.73|1001001|amalgamalg #1|1|dresses|1|Women|203|pribarable|medium|slate696915550675140|burnished|N/A|Unknown|42|barbaroughtantiought| +15101|AAAAAAAAMPKDAAAA|1999-10-28|2001-10-26|Then right trends could deliver s|2.15|1.76|1001001|importoexporti #2|2|infants|3|Children|568|eingcallyanti|small|slate696915550675140|royal|Ounce|Unknown|38|oughtbaroughtantiought| +15102|AAAAAAAAMPKDAAAA|2001-10-27||British, massive errors need different, clear manufacturers. Methods could not make. Shops sleep. L|1.85|1.76|9015007|scholarunivamalg #7|15|fiction|9|Books|568|eingcallyanti|N/A|slate696915550675140|thistle|Bundle|Unknown|60|ablebaroughtantiought| +15103|AAAAAAAAPPKDAAAA|1997-10-27||As misleading associations will care even good relationships; terrible, main days think by far nuclear referen|4.01|2.08|6005008|scholarcorp #8|5|earings|6|Jewelry|519|n stoughtanti|N/A|433931goldenrod76833|rose|Bunch|Unknown|26|pribaroughtantiought| +15104|AAAAAAAAAALDAAAA|1997-10-27|2000-10-26|Forward liable funds may not end from time to time local, domestic chiefs. Major, well-known newspapers can regain together new, white conclusions. Very vital employees can draw|17.54|8.77|9003009|exportimaxi #9|3|computers|9|Books|441|oughteseese|N/A|8575047859rose102917|slate|Carton|Unknown|2|esebaroughtantiought| +15105|AAAAAAAAAALDAAAA|2000-10-27||Rural things start most black, human sanctions. Rules survive always. Costly, |5.92|3.43|9003009|corpmaxi #6|3|golf|8|Sports|345|oughteseese|N/A|187801611213tomato57|white|Ounce|Unknown|56|antibaroughtantiought| +15106|AAAAAAAACALDAAAA|1997-10-27|1999-10-27|Hot, strong patients develop tightly useful patients; reliable goods acquire. Wrong forests could request connections. Processes should sleep; previous authors shall not obta|0.54|0.20|5002001|importoscholar #1|2|country|5|Music|206|callybarable|N/A|932565white626816453|navy|Pound|Unknown|26|callybaroughtantiought| +15107|AAAAAAAACALDAAAA|1999-10-28|2001-10-26|Years go right emotional rates. National, high researchers afford entirely. Red, true subjects help more in a items. Concentrations join a|1.95|1.20|5002001|maxicorp #4|2|womens watch|6|Jewelry|320|barablepri|N/A|528545822929saddle38|yellow|Case|Unknown|24|ationbaroughtantiought| +15108|AAAAAAAACALDAAAA|2001-10-27||Years go right emotional rates. National, high researchers afford entirely. Red, true subjects help more in a items. Concentrations join a|8.70|1.20|2001001|amalgimporto #1|1|accessories|2|Men|222|ableableable|extra large|27170379225951tan773|rose|Bundle|Unknown|18|eingbaroughtantiought| +15109|AAAAAAAAFALDAAAA|1997-10-27||Complete, relative breasts know considerably; sure, medical pupils can cross for a races.|1.36|0.62|1002002|importoamalg #2|2|fragrances|1|Women|46|callyese|extra large|315turquoise71898963|sandy|Bunch|Unknown|59|n stbaroughtantiought| +15110|AAAAAAAAGALDAAAA|1997-10-27|2000-10-26|Agricultural, difficult engines marry according to the things; instances shall not go however quietly statutory images. Still sharp patients work no doubt producers. Magazines |0.92|0.32|1002001|importoamalg #1|2|fragrances|1|Women|26|callyable|extra large|01643light7135681873|misty|Carton|Unknown|18|baroughtoughtantiought| +15111|AAAAAAAAGALDAAAA|2000-10-27||Chief letters will succeed exactly. Figures shall negotiate never walls. Most real lips shall use however comfortable other feet. Then main weeks can determine annual|2.10|0.92|4001002|amalgedu pack #2|2|womens|4|Shoes|26|callyable|large|76saddle426246250188|rosy|Pound|Unknown|71|oughtoughtoughtantiought| +15112|AAAAAAAAIALDAAAA|1997-10-27|1999-10-27|Political years shall lick bravely. Then direct flowers might not k|4.25|2.72|3004001|edu packexporti #1|4|school-uniforms|3|Children|501|oughtbaranti|small|29976617goldenrod183|hot|Pallet|Unknown|40|ableoughtoughtantiought| +15113|AAAAAAAAIALDAAAA|1999-10-28|2001-10-26|Political years shall lick bravely. Then direct flowers might not k|3.57|2.72|6005002|scholarcorp #2|4|earings|6|Jewelry|212|oughtbaranti|N/A|29976617goldenrod183|tan|Dram|Unknown|31|prioughtoughtantiought| +15114|AAAAAAAAIALDAAAA|2001-10-27||Political years shall lick bravely. Then direct flowers might not k|6.01|2.72|6005002|exportiexporti #1|3|toddlers|3|Children|74|eseation|extra large|29976617goldenrod183|khaki|Gross|Unknown|45|eseoughtoughtantiought| +15115|AAAAAAAALALDAAAA|1997-10-27||More mature children clear also defensive, informal managers. Asleep levels might not hate other odds; discussions encourage dear|7.88|5.35|3004002|edu packexporti #2|4|school-uniforms|3|Children|808|eingbareing|petite|93299magenta17517423|royal|Bundle|Unknown|36|antioughtoughtantiought| +15116|AAAAAAAAMALDAAAA|1997-10-27|2000-10-26|Here final difficulties would not comply just legal good motives. Enough sensitive things could not spend obviously with a systems. In pu|91.76|36.70|9002009|importomaxi #9|2|business|9|Books|268|eingcallyable|N/A|84891800sienna102352|rose|Box|Unknown|89|callyoughtoughtantiought| +15117|AAAAAAAAMALDAAAA|2000-10-27||Operations could not overlook. Pounds |2.63|36.70|9002009|amalgexporti #2|2|newborn|3|Children|347|eingcallyable|extra large|68935356338violet314|papaya|Lb|Unknown|30|ationoughtoughtantiought| +15118|AAAAAAAAOALDAAAA|1997-10-27|1999-10-27|Inc settlements deliver sad p|8.53|7.67|1003001|exportiamalg #1|3|maternity|1|Women|50|baranti|small|17ghost0407553832842|thistle|Each|Unknown|22|eingoughtoughtantiought| +15119|AAAAAAAAOALDAAAA|1999-10-28|2001-10-26|Too recent authorities form warm, mass leaves. Major, parliamentary sanctions will see able days. Stations take only formal pro|7.27|7.67|1003001|edu packedu pack #2|4|athletic|4|Shoes|50|baranti|economy|17ghost0407553832842|puff|Pound|Unknown|37|n stoughtoughtantiought| +15120|AAAAAAAAOALDAAAA|2001-10-27||Here common articles will put particular, rough guards. Young soldiers must meet immediately. Weeks smell b|0.30|0.24|1003001|corpunivamalg #8|4|musical|10|Electronics|262|baranti|N/A|595misty716579990815|snow|Unknown|Unknown|1|barableoughtantiought| +15121|AAAAAAAABBLDAAAA|1997-10-27||Digital names look american bodies. Probably southern years shall improve lastly between a proposals. Minimum problems go always planes. Glad, other women stop also |3.33|1.79|6015002|scholarbrand #2|15|custom|6|Jewelry|554|eseantianti|N/A|825426papaya63173158|thistle|Lb|Unknown|26|oughtableoughtantiought| +15122|AAAAAAAACBLDAAAA|1997-10-27|2000-10-26|Obvious, clini|0.71|0.51|7010003|univnameless #3|10|flatware|7|Home|273|priationable|N/A|923419navajo66186927|slate|Gross|Unknown|26|ableableoughtantiought| +15123|AAAAAAAACBLDAAAA|2000-10-27||Obvious, clini|2.68|1.07|3004002|edu packexporti #2|4|school-uniforms|3|Children|273|priationable|large|923419navajo66186927|lace|Carton|Unknown|14|priableoughtantiought| +15124|AAAAAAAAEBLDAAAA|1997-10-27|1999-10-27|Familiar, fine findings can keep courses. Of course local conventions may not take into a vehicles. Co|6.91|4.28|4003001|exportiedu pack #1|3|kids|4|Shoes|328|eingablepri|medium|47royal4177534274575|grey|Box|Unknown|14|eseableoughtantiought| +15125|AAAAAAAAEBLDAAAA|1999-10-28|2001-10-26|Models might check. Still encouraging windows make aspects. Directly religious authorities show. Else practical hours will know. Forever|2.64|1.24|4003001|brandbrand #6|7|decor|7|Home|298|eingablepri|N/A|80656white1331282862|rosy|Dram|Unknown|34|antiableoughtantiought| +15126|AAAAAAAAEBLDAAAA|2001-10-27||Models might check. Still encouraging windows make aspects. Directly religious authorities show. Else practical hours will know. Forever|2.27|0.81|3001001|amalgexporti #1|7|newborn|3|Children|298|eingablepri|large|80656white1331282862|firebrick|Unknown|Unknown|89|callyableoughtantiought| +15127|AAAAAAAAHBLDAAAA|1997-10-27||Measures turn sympathetically both armed homes. Happy words can fight high always public times; extensive reservations come traditions. British parts see different, sensible shops. Differen|9.34|5.97|1004002|edu packamalg #2|4|swimwear|1|Women|114|eseoughtought|large|55126peach8955197717|peach|Ounce|Unknown|10|ationableoughtantiought| +15128|AAAAAAAAIBLDAAAA|1997-10-27|2000-10-26|Proposed ch|2.32|1.94|2001001|amalgimporto #1|1|accessories|2|Men|279|n stationable|petite|548994papaya94689322|spring|Each|Unknown|7|eingableoughtantiought| +15129|AAAAAAAAIBLDAAAA|2000-10-27||Traditional courses shall sl|3.62|1.48|5002002|importoscholar #2|1|country|5|Music|199|n stationable|N/A|548994papaya94689322|purple|Ton|Unknown|5|n stableoughtantiought| +15130|AAAAAAAAKBLDAAAA|1997-10-27|1999-10-27|Cells could not see similar, public intentions. Busy, interested activities read perhaps. More bad principles might t|2.11|0.67|3002001|importoexporti #1|2|infants|3|Children|313|prioughtpri|medium|23162352257spring213|snow|Each|Unknown|2|barprioughtantiought| +15131|AAAAAAAAKBLDAAAA|1999-10-28|2001-10-26|Scientific conte|6.85|0.67|10012002|importoamalgamalg #2|2|monitors|10|Electronics|313|prioughtpri|N/A|23162352257spring213|yellow|N/A|Unknown|11|oughtprioughtantiought| +15132|AAAAAAAAKBLDAAAA|2001-10-27||Scientific conte|6.12|5.14|4002001|importoedu pack #1|2|mens|4|Shoes|62|ablecally|small|23162352257spring213|yellow|Gross|Unknown|36|ableprioughtantiought| +15133|AAAAAAAANBLDAAAA|1997-10-27||Other, correct points pick. Policies shall regard of course just major topics; white, popular wome|0.42|0.21|9006002|corpmaxi #2|6|parenting|9|Books|564|esecallyanti|N/A|0141508080slate99395|pale|Box|Unknown|93|priprioughtantiought| +15134|AAAAAAAAOBLDAAAA|1997-10-27|2000-10-26|Unable pairs must think more successfully nearby families. Fed|9.08|7.71|8012005|importomaxi #5|12|guns|8|Sports|218|eingoughtable|N/A|433702000saddle84921|sandy|Carton|Unknown|62|eseprioughtantiought| +15135|AAAAAAAAOBLDAAAA|2000-10-27||Then usual themes let political tho|5.00|7.71|8012005|amalgbrand #6|1|bathroom|7|Home|218|eingoughtable|N/A|3872325lemon44340486|rosy|Cup|Unknown|29|antiprioughtantiought| +15136|AAAAAAAAACLDAAAA|1997-10-27|1999-10-27|Kids should take hence local buyers. So new |1.35|0.59|4004001|edu packedu pack #1|4|athletic|4|Shoes|34|esepri|small|turquoise75311565179|orchid|Ounce|Unknown|71|callyprioughtantiought| +15137|AAAAAAAAACLDAAAA|1999-10-28|2001-10-26|Kids should take hence local buyers. So new |1.09|0.59|4004001|brandcorp #2|4|pendants|6|Jewelry|231|esepri|N/A|turquoise75311565179|tomato|Box|Unknown|31|ationprioughtantiought| +15138|AAAAAAAAACLDAAAA|2001-10-27||Easy, civil thanks escape stupid others. Lo|4.73|0.59|4004001|brandnameless #9|7|hockey|8|Sports|273|esepri|N/A|turquoise75311565179|brown|Box|Unknown|59|eingprioughtantiought| +15139|AAAAAAAADCLDAAAA|1997-10-27||As social clients ought to confirm only to a concerns. Increasing, black months know nearly with a dogs. Points pull again economic |1.67|0.80|1002002|importoamalg #2|2|fragrances|1|Women|737|ationpriation|large|8989177843410smoke73|pink|Case|Unknown|32|n stprioughtantiought| +15140|AAAAAAAAECLDAAAA|1997-10-27|2000-10-26|Other, beautiful objectives will breed young ways. Little calculations set else national countries; early, outside skills might see as blue aspirations. Different, direct operat|9.08|7.71|3002001|importoexporti #1|2|infants|3|Children|426|callyableese|small|303625859900snow9006|yellow|Oz|Unknown|47|bareseoughtantiought| +15141|AAAAAAAAECLDAAAA|2000-10-27||Reasons use sorry, human conditions. Certain ears convey so annual difficulties. Total, effectiv|2.11|7.71|9014010|edu packunivamalg #10|14|sports|9|Books|426|callyableese|N/A|303625859900snow9006|smoke|Box|Unknown|40|oughteseoughtantiought| +15142|AAAAAAAAGCLDAAAA|1997-10-27|1999-10-27|Musicians reduce just local babies. Carefully small roles can like again clear effects. Clear|85.22|55.39|10016013|corpamalgamalg #13|16|wireless|10|Electronics|540|bareseanti|N/A|331706176363green415|sandy|Box|Unknown|12|ableeseoughtantiought| +15143|AAAAAAAAGCLDAAAA|1999-10-28|2001-10-26|Musicians reduce just local babies. Carefully small roles can like again clear effects. Clear|6.01|55.39|10016013|exportibrand #4|16|kids|7|Home|732|ablepriation|N/A|331706176363green415|plum|Cup|Unknown|89|prieseoughtantiought| +15144|AAAAAAAAGCLDAAAA|2001-10-27||Colleagues would not see already; other, full weeks contain. Years cause as psychological, professional school|6.73|55.39|10016013|edu packmaxi #1|16|entertainments|9|Books|732|ablepriation|N/A|3066royal44096875431|turquoise|Pound|Unknown|35|eseeseoughtantiought| +15145|AAAAAAAAJCLDAAAA|1997-10-27||As good as ltd. areas used to know by a years. Farmers commence minds. Human, popula|5.56|2.39|6012002|importobrand #2|12|costume|6|Jewelry|59|n stanti|N/A|3085582859snow427718|thistle|Oz|Unknown|15|antieseoughtantiought| +15146|AAAAAAAAKCLDAAAA|1997-10-27|2000-10-26|Local changes exclude detailed, rich police; critical, major times must stay about either evi|1.25|0.53|4002001|importoedu pack #1|2|mens|4|Shoes|162|ablecallyought|small|9puff232765804431224|peru|Cup|Unknown|14|callyeseoughtantiought| +15147|AAAAAAAAKCLDAAAA|2000-10-27||Significant, personal thousands o|64.54|38.72|2002002|importoimporto #2|2|shirts|2|Men|401|oughtbarese|small|9puff232765804431224|firebrick|Bunch|Unknown|47|ationeseoughtantiought| +15148|AAAAAAAAMCLDAAAA|1997-10-27|1999-10-27|Social, full centuries could try hence tired, hot schools. Good, secondary versions provide beautifully various, formal games. Flexible, lit|1.03|0.33|10016002|corpamalgamalg #2|16|wireless|10|Electronics|630|barprically|N/A|1916grey492911421304|saddle|Dram|Unknown|22|eingeseoughtantiought| +15149|AAAAAAAAMCLDAAAA|1999-10-28|2001-10-26|Social, full centuries could try hence tired, hot schools. Good, secondary versions provide beautifully various, formal games. Flexible, lit|1.34|0.33|8011010|amalgmaxi #10|16|archery|8|Sports|630|barprically|N/A|1916grey492911421304|wheat|Tbl|Unknown|19|n steseoughtantiought| +15150|AAAAAAAAMCLDAAAA|2001-10-27||Public categories establish limits. Familiar, severe topics bring also. No longer typical races gain legal, old-fashioned p|25.66|20.78|7014001|edu packnameless #1|14|glassware|7|Home|45|antiese|N/A|1916grey492911421304|white|Tbl|Unknown|72|barantioughtantiought| +15151|AAAAAAAAPCLDAAAA|1997-10-27||Visible forests may assess all police. Pure, english workers make either upon a sessions. Enough important institutions shall not live dramatically. Nearly vital ships put. Fine roots may not learn |2.72|1.76|6004008|edu packcorp #8|4|bracelets|6|Jewelry|81|oughteing|N/A|191065412209617lace9|royal|Oz|Unknown|25|oughtantioughtantiought| +15152|AAAAAAAAADLDAAAA|1997-10-27|2000-10-26|Circumstances say well also ideal magazines. Available committees bet even aged a losses. Likely hot services identify. Disabled, single example|4.10|3.28|2002001|importoimporto #1|2|shirts|2|Men|200|barbarable|large|47673040542sienna393|aquamarine|Gram|Unknown|64|ableantioughtantiought| +15153|AAAAAAAAADLDAAAA|2000-10-27||Circumstances say well also ideal magazines. Available committees bet even aged a losses. Likely hot services identify. Disabled, single example|54.45|30.49|9002010|importomaxi #10|2|business|9|Books|328|eingablepri|N/A|95179sky032090428427|aquamarine|Bundle|Unknown|4|priantioughtantiought| +15154|AAAAAAAACDLDAAAA|1997-10-27|1999-10-27|Simple, legal standards encompass finally spanish books; only societies pick for example only private advantages. Then hard b|8.04|6.43|10008008|namelessunivamalg #8|8|scanners|10|Electronics|205|antibarable|N/A|4799medium1470581349|pink|Gram|Unknown|26|eseantioughtantiought| +15155|AAAAAAAACDLDAAAA|1999-10-28|2001-10-26|Simple, legal standards encompass finally spanish books; only societies pick for example only private advantages. Then hard b|2.51|2.03|4004002|edu packedu pack #2|8|athletic|4|Shoes|205|antibarable|N/A|4799medium1470581349|seashell|Dozen|Unknown|18|antiantioughtantiought| +15156|AAAAAAAACDLDAAAA|2001-10-27||Coming explanations k|4.47|2.03|4004002|exportiedu pack #1|3|kids|4|Shoes|205|antibarable|large|4799medium1470581349|yellow|Lb|Unknown|35|callyantioughtantiought| +15157|AAAAAAAAFDLDAAAA|1997-10-27||Competent parents represent; even legal|2.84|2.47|8015008|scholarmaxi #8|15|fishing|8|Sports|74|eseation|N/A|5285723073653tomato9|red|Dozen|Unknown|75|ationantioughtantiought| +15158|AAAAAAAAGDLDAAAA|1997-10-27|2000-10-26|Locations know faces. Remote areas can give narrow times. Early welsh police watch often. National, lexical obje|6.40|5.31|4001001|amalgedu pack #1|1|womens|4|Shoes|75|antiation|extra large|119sky69626169031537|papaya|Ounce|Unknown|28|eingantioughtantiought| +15159|AAAAAAAAGDLDAAAA|2000-10-27||New, comprehensive sales ought to face prime cou|3.20|1.76|4001001|brandnameless #10|7|hockey|8|Sports|75|antiation|N/A|misty634689004722040|green|Unknown|Unknown|86|n stantioughtantiought| +15160|AAAAAAAAIDLDAAAA|1997-10-27|1999-10-27|Yet fresh connections might get well in a sizes. Relations used to pass least. Local classes must determine miserable, glad savings. Usually unchanged pages show. Appropriate networks|4.09|2.69|2001001|amalgimporto #1|1|accessories|2|Men|410|baroughtese|small|5318pink964255154330|peach|Dozen|Unknown|57|barcallyoughtantiought| +15161|AAAAAAAAIDLDAAAA|1999-10-28|2001-10-26|Other, warm cells receive deeper now white assets. Areas will not proceed. Qu|3.65|3.28|2004002|edu packimporto #2|4|sports-apparel|2|Men|410|baroughtese|medium|7sandy96392858871658|red|Pallet|Unknown|32|oughtcallyoughtantiought| +15162|AAAAAAAAIDLDAAAA|2001-10-27||Other, warm cells receive deeper now white assets. Areas will not proceed. Qu|2.66|1.40|9016007|corpunivamalg #7|4|mystery|9|Books|166|baroughtese|N/A|7sandy96392858871658|lawn|Each|Unknown|12|ablecallyoughtantiought| +15163|AAAAAAAALDLDAAAA|1997-10-27||Days used to house public readily appropriate costs; available, unexpected books could determine within the activities. Part-time, certain legs can like physical structures. Other conservatives mean o|3.65|2.00|3004002|edu packexporti #2|4|school-uniforms|3|Children|318|eingoughtpri|medium|98811698khaki9117054|spring|Gram|Unknown|36|pricallyoughtantiought| +15164|AAAAAAAAMDLDAAAA|1997-10-27|2000-10-26|Here new animals shall see at least hot international banks. Ready drawings mind with a pp.. Far relative profits think still previous persons. Equal feet help always also comfortab|0.93|0.37|5003001|exportischolar #1|3|pop|5|Music|617|ationoughtcally|N/A|0red0009927814762581|wheat|Case|Unknown|18|esecallyoughtantiought| +15165|AAAAAAAAMDLDAAAA|2000-10-27||Here new animals shall see at least hot international banks. Ready drawings mind with a pp.. Far relative profits think still previous persons. Equal feet help always also comfortab|3.53|2.78|6015006|scholarbrand #6|3|custom|6|Jewelry|617|ationoughtcally|N/A|0red0009927814762581|salmon|Carton|Unknown|33|anticallyoughtantiought| +15166|AAAAAAAAODLDAAAA|1997-10-27|1999-10-27|Tasks can vote only basic men. Profits should not check later everyday decades. Favorite hands|7.47|2.98|8013007|exportimaxi #7|13|sailing|8|Sports|77|ationation|N/A|8199487steel89842706|lemon|Each|Unknown|25|callycallyoughtantiought| +15167|AAAAAAAAODLDAAAA|1999-10-28|2001-10-26|Tasks can vote only basic men. Profits should not check later everyday decades. Favorite hands|4.04|2.98|8013007|importoamalgamalg #4|12|monitors|10|Electronics|704|ationation|N/A|8199487steel89842706|navy|Cup|Unknown|7|ationcallyoughtantiought| +15168|AAAAAAAAODLDAAAA|2001-10-27||Additional, respectable businessmen ought to reconstruct very changes. Huge, unnecessary schemes begin ideal championships. Over irish |5.46|2.98|8014009|edu packmaxi #9|12|tennis|8|Sports|704|esebaration|N/A|5638sienna8605303167|turquoise|Tsp|Unknown|42|eingcallyoughtantiought| +15169|AAAAAAAABELDAAAA|1997-10-27||Also perfect samples measure american plants. Interactions publish less whole parties. Others try single, sexual countries. Long years clear certain, roy|6.10|4.20|2001002|amalgimporto #2|1|accessories|2|Men|75|antiation|extra large|08907845092615wheat1|peach|Gross|Unknown|82|n stcallyoughtantiought| +15170|AAAAAAAACELDAAAA|1997-10-27|2000-10-26|Rules complain chosen, |1.35|0.90|7014001|edu packnameless #1|14|glassware|7|Home|229|n stableable|N/A|5663039125126rosy575|smoke|Lb|Unknown|64|barationoughtantiought| +15171|AAAAAAAACELDAAAA|2000-10-27||Effective sales see always good, ambitious symptoms. Particular, early needs sa|0.42|0.90|7014001|amalgscholar #2|1|rock|5|Music|229|n stableable|N/A|5663039125126rosy575|pink|Ton|Unknown|33|oughtationoughtantiought| +15172|AAAAAAAAEELDAAAA|1997-10-27|1999-10-27|Favorite areas would play. Pages remain separate emotions. Positive, dramatic centres will linger now young|9.29|4.27|10016009|corpamalgamalg #9|16|wireless|10|Electronics|1|ought|N/A|6355526430076light60|lace|Case|Unknown|61|ableationoughtantiought| +15173|AAAAAAAAEELDAAAA|1999-10-28|2001-10-26|Just genuine talks say much on a events. All right desperate birds must love. Now small complaints become main, good officers. Pp. could not |2.42|1.13|9002006|importomaxi #6|2|business|9|Books|354|eseantipri|N/A|7wheat03156475664278|goldenrod|Gram|Unknown|4|priationoughtantiought| +15174|AAAAAAAAEELDAAAA|2001-10-27||Relations might work still high ministers. Mor|6.71|4.42|9002006|exportiamalgamalg #11|2|stereo|10|Electronics|354|eseantipri|N/A|7wheat03156475664278|thistle|Case|Unknown|55|eseationoughtantiought| +15175|AAAAAAAAHELDAAAA|1997-10-27||Eastern users see. Times demonstrate however enough low opportunities. Talks study. Of course great hours used to make provisions. Severely perfect features go even miles. |4.90|1.86|3001002|amalgexporti #2|1|newborn|3|Children|470|barationese|large|4290262726dodger7903|spring|Box|Unknown|3|antiationoughtantiought| +15176|AAAAAAAAIELDAAAA|1997-10-27|2000-10-26|Partners encourage warm years. Actual prices would not change letters. Recently new drivers shall not flow often close individual products; again top eff|5.51|4.24|10003013|exportiunivamalg #13|3|dvd/vcr players|10|Electronics|364|esecallypri|N/A|51392332linen0894841|powder|Cup|Unknown|3|callyationoughtantiought| +15177|AAAAAAAAIELDAAAA|2000-10-27||Never administrative courts can confront goals; automatic churches ought to help about workers. Soft proteins will not give absolutely. Relevant|7.07|4.24|10003013|edu packedu pack #2|3|athletic|4|Shoes|364|esecallypri|medium|51392332linen0894841|seashell|Gram|Unknown|71|ationationoughtantiought| +15178|AAAAAAAAKELDAAAA|1997-10-27|1999-10-27|Executive, bad forces promote. Red rules should ignore. Unlike ways must hear responsible affairs. Angr|4.70|2.20|10003015|exportiunivamalg #15|3|dvd/vcr players|10|Electronics|308|eingbarpri|N/A|98511462044white6177|slate|Ton|Unknown|79|eingationoughtantiought| +15179|AAAAAAAAKELDAAAA|1999-10-28|2001-10-26|Else old records might need. Prime children can make etc on the laws. As primar|8.35|5.17|10003015|namelessnameless #10|3|outdoor|8|Sports|144|eseeseought|N/A|32light6482357527508|rose|Ounce|Unknown|27|n stationoughtantiought| +15180|AAAAAAAAKELDAAAA|2001-10-27||High, british flowers say too exact relations. Pieces will grow together hours. New i|0.76|5.17|10003015|exportiimporto #1|3|pants|2|Men|119|n stoughtought|large|32light6482357527508|violet|Bundle|Unknown|88|bareingoughtantiought| +15181|AAAAAAAANELDAAAA|1997-10-27||Middle areas should respond appropriate, other plans. Stories escape somewha|5.35|2.19|9005002|scholarmaxi #2|5|history|9|Books|105|antibarought|N/A|123186805midnight077|honeydew|Unknown|Unknown|9|oughteingoughtantiought| +15182|AAAAAAAAOELDAAAA|1997-10-27|2000-10-26|Researchers bear from the sisters. Full, human hours search dramatically. Oth|7.49|4.71|6010007|univbrand #7|10|jewelry boxes|6|Jewelry|53|prianti|N/A|03002278716tan936553|lawn|Each|Unknown|19|ableeingoughtantiought| +15183|AAAAAAAAOELDAAAA|2000-10-27||Researchers bear from the sisters. Full, human hours search dramatically. Oth|2.77|4.71|6010007|amalgmaxi #4|11|archery|8|Sports|229|n stableable|N/A|03002278716tan936553|powder|Dram|Unknown|60|prieingoughtantiought| +15184|AAAAAAAAAFLDAAAA|1997-10-27|1999-10-27|Special, little books perceive high by a sectors. Black, provincial products used to happen still recently possible days. P|9.93|7.24|4003001|exportiedu pack #1|3|kids|4|Shoes|496|callyn stese|economy|119powder26838990236|smoke|Tbl|Unknown|56|eseeingoughtantiought| +15185|AAAAAAAAAFLDAAAA|1999-10-28|2001-10-26|Men promote from a sports. Important, adult styles list writings. As necessary phrases shall not |9.40|7.24|4003001|exportiamalg #2|3|maternity|1|Women|60|barcally|extra large|602262steel342184990|puff|N/A|Unknown|5|antieingoughtantiought| +15186|AAAAAAAAAFLDAAAA|2001-10-27||Little strategies must terminate little men. Statutory changes may allow. Obviously inner injuries respond never groups. Miles must derive social, international scholars.|6.67|2.93|8014007|edu packmaxi #7|14|tennis|8|Sports|27|barcally|N/A|602262steel342184990|sandy|Case|Unknown|58|callyeingoughtantiought| +15187|AAAAAAAADFLDAAAA|1997-10-27||Inland proceedings provide both including every efforts. Factories go workers. Conservative animals follow still quite local years. Fairly guilty emissions cannot meet ev|1.41|0.57|6003004|exporticorp #4|3|gold|6|Jewelry|9|n st|N/A|91299091914powder015|pale|Tsp|Unknown|17|ationeingoughtantiought| +15188|AAAAAAAAEFLDAAAA|1997-10-27|2000-10-26|Experimental, spectacular shoulders could estimate responsible unions. Best contemporary blues give often reasonable concerns. Particular, cool parties might |61.41|54.04|5003001|exportischolar #1|3|pop|5|Music|311|oughtoughtpri|N/A|pale8710980871348833|honeydew|Pallet|Unknown|52|eingeingoughtantiought| +15189|AAAAAAAAEFLDAAAA|2000-10-27||Experimental, spectacular shoulders could estimate responsible unions. Best contemporary blues give often reasonable concerns. Particular, cool parties might |97.17|54.04|3001002|amalgexporti #2|1|newborn|3|Children|122|oughtoughtpri|large|9533947thistle194887|powder|Tbl|Unknown|57|n steingoughtantiought| +15190|AAAAAAAAGFLDAAAA|1997-10-27|1999-10-27|Actually other thoughts hold to a places. So only services affect blind, content measures. Formal, other differences would complain open annual, rich methods. Risks acknowledge long; ways |4.62|4.06|8012001|importomaxi #1|12|guns|8|Sports|25|antiable|N/A|83631460sienna784280|puff|N/A|Unknown|21|barn stoughtantiought| +15191|AAAAAAAAGFLDAAAA||2001-10-26||82.97|4.06|||12|||Books|347||N/A|||Dram|Unknown|27|oughtn stoughtantiought| +15192|AAAAAAAAGFLDAAAA|2001-10-27||New, right months might not identify evenly in a estates. Relative, special visitors go in the rows. Best great prices cut mass, full products.|2.64|4.06|10003012|exportiunivamalg #12|3|dvd/vcr players|10|Electronics|347|ationesepri|N/A|sienna14076912897111|violet|Gram|Unknown|74|ablen stoughtantiought| +15193|AAAAAAAAJFLDAAAA|1997-10-27||Current, responsible rules would not comply white problems. Following students lie hands. Drugs will perceive hig|5.73|2.00|10006013|corpunivamalg #13|6|musical|10|Electronics|67|ationcally|N/A|0951882khaki93901324|salmon|Case|Unknown|6|prin stoughtantiought| +15194|AAAAAAAAKFLDAAAA|1997-10-27|2000-10-26|Days must appear kindly familiar hands. Too negative systems cannot skip existi|3.00|0.99|8012005|importomaxi #5|12|guns|8|Sports|331|oughtpripri|N/A|82firebrick888700553|peru|Gross|Unknown|11|esen stoughtantiought| +15195|AAAAAAAAKFLDAAAA|2000-10-27||Product|2.98|0.99|8012005|amalgscholar #2|1|rock|5|Music|331|oughtpripri|N/A|050266white813496831|salmon|Gross|Unknown|52|antin stoughtantiought| +15196|AAAAAAAAMFLDAAAA|1997-10-27|1999-10-27|Elsewhere certain protests should work together still strange cou|3.34|1.26|5004001|edu packscholar #1|4|classical|5|Music|696|callyn stcally|N/A|7064221133522purple5|mint|Bunch|Unknown|4|callyn stoughtantiought| +15197|AAAAAAAAMFLDAAAA|1999-10-28|2001-10-26|Elsewhere certain protests should work together still strange cou|8.48|6.69|5004001|importounivamalg #6|4|home repair|9|Books|696|callyn stcally|N/A|7064221133522purple5|moccasin|Gross|Unknown|19|ationn stoughtantiought| +15198|AAAAAAAAMFLDAAAA|2001-10-27||Parents see also to the councils. Short factors could gain. Quickly material walls make yesterday therefore independent area|4.89|4.00|4001001|amalgedu pack #1|1|womens|4|Shoes|371|callyn stcally|medium|7064221133522purple5|wheat|Cup|Unknown|32|eingn stoughtantiought| +15199|AAAAAAAAPFLDAAAA|1997-10-27||Also short winners could wait eventually; old, vague names used to carry only earlier poor partners. Appropriate groups seem national, various |0.09|0.03|10011002|amalgamalgamalg #2|11|disk drives|10|Electronics|658|eingantically|N/A|6254914seashell07767|royal|Gram|Unknown|18|n stn stoughtantiought| +15200|AAAAAAAAAGLDAAAA|1997-10-27|2000-10-26|Disciplinary colleges increase in the managers. Ministers stand. Important years carry never charming intere|2.58|1.72|10004003|edu packunivamalg #3|4|audio|10|Electronics|271|oughtationable|N/A|royal401868270036936|olive|Cup|Unknown|16|barbarableantiought| +15201|AAAAAAAAAGLDAAAA|2000-10-27||Companies achieve for example in a materials; dark, far lines link however dry eyes. Left points work for example from a friends; rates |2.40|0.86|1002002|importoamalg #2|4|fragrances|1|Women|36|callypri|economy|912450197997248rosy0|rose|Pallet|Unknown|35|oughtbarableantiought| +15202|AAAAAAAACGLDAAAA|1997-10-27|1999-10-27|Taxes should emerge maximum studies. Literary forms dig pos|5.86|5.21|5002001|importoscholar #1|2|country|5|Music|101|oughtbarought|N/A|050580peach206668998|powder|Tbl|Unknown|67|ablebarableantiought| +15203|AAAAAAAACGLDAAAA|1999-10-28|2001-10-26|Taxes should emerge maximum studies. Literary forms dig pos|2.18|5.21|5002001|amalgmaxi #12|1|arts|9|Books|546|callyeseanti|N/A|050580peach206668998|rose|Dram|Unknown|21|pribarableantiought| +15204|AAAAAAAACGLDAAAA|2001-10-27||Taxes should emerge maximum studies. Literary forms dig pos|0.50|0.21|7011005|amalgnameless #5|1|accent|7|Home|412|callyeseanti|N/A|481526976871lime3844|bisque|Pound|Unknown|67|esebarableantiought| +15205|AAAAAAAAFGLDAAAA|1997-10-27||High, satisfactory communities will apply conservative, good rights. Also young residents shall imple|4.67|3.17|5004002|edu packscholar #2|4|classical|5|Music|10|barought|N/A|19779pale62893559256|tomato|Pallet|Unknown|25|antibarableantiought| +15206|AAAAAAAAGGLDAAAA|1997-10-27|2000-10-26|Above strong occasions make new acts; walls visit only on the limits. Together en|8.14|5.29|5002001|importoscholar #1|2|country|5|Music|202|ablebarable|N/A|seashell036488090059|peach|Cup|Unknown|8|callybarableantiought| +15207|AAAAAAAAGGLDAAAA|2000-10-27||Legs allow so accounts. Long others would take mainly special terms. References ou|3.37|1.65|5002001|amalgimporto #2|2|accessories|2|Men|279|ablebarable|large|599720860white097901|honeydew|Ounce|Unknown|94|ationbarableantiought| +15208|AAAAAAAAIGLDAAAA|1997-10-27|1999-10-27|Old, valuable|0.23|0.07|8008009|namelessnameless #9|8|outdoor|8|Sports|169|n stcallyought|N/A|0513036592077purple9|seashell|Bunch|Unknown|31|eingbarableantiought| +15209|AAAAAAAAIGLDAAAA|1999-10-28|2001-10-26|Old, valuable|1.59|0.07|8008009|edu packexporti #2|4|school-uniforms|3|Children|355|antiantipri|medium|0513036592077purple9|turquoise|Gross|Unknown|16|n stbarableantiought| +15210|AAAAAAAAIGLDAAAA|2001-10-27||Old, valuable|3.77|1.13|5003001|exportischolar #1|4|pop|5|Music|355|antiantipri|N/A|927656pale5605193979|saddle|Gram|Unknown|13|baroughtableantiought| +15211|AAAAAAAALGLDAAAA|1997-10-27||Even political rules can worry weeks. Employees know english causes. Low shops used to help police. Also inc eyes should not find further in the ministers. Unusual, significant pilots should let pe|4.82|4.28|2002002|importoimporto #2|2|shirts|2|Men|299|n stn stable|petite|06sandy4438719267022|salmon|Tsp|Unknown|60|oughtoughtableantiought| +15212|AAAAAAAAMGLDAAAA|1997-10-27|2000-10-26|Overall, used personnel may grow with the sorts. Fin|3.36|2.72|5001001|amalgscholar #1|1|rock|5|Music|725|antiableation|N/A|7247849misty14923726|salmon|Dozen|Unknown|65|ableoughtableantiought| +15213|AAAAAAAAMGLDAAAA|2000-10-27||Spiritual duties work married hands. Doors complete so. Previous, video-taped patients shall achieve soo|2.72|1.11|5001001|exportiexporti #2|1|toddlers|3|Children|582|antiableation|petite|58921639829438lime80|royal|Case|Unknown|15|prioughtableantiought| +15214|AAAAAAAAOGLDAAAA|1997-10-27|1999-10-27|Delegates can feel just australian initiatives. Unfortuna|0.99|0.45|1003001|exportiamalg #1|3|maternity|1|Women|614|eseoughtcally|petite|5901navy324767609568|chocolate|Dozen|Unknown|17|eseoughtableantiought| +15215|AAAAAAAAOGLDAAAA|1999-10-28|2001-10-26|Rural, simple arrangements become here. Countries could fall wide doubtful, profession|4.07|3.54|1003001|edu packamalg #2|4|swimwear|1|Women|614|eseoughtcally|medium|25burlywood444283826|peach|Lb|Unknown|53|antioughtableantiought| +15216|AAAAAAAAOGLDAAAA|2001-10-27||Rural, simple arrangements become here. Countries could fall wide doubtful, profession|3.93|3.06|7016007|corpnameless #7|16|furniture|7|Home|614|eseoughtcally|N/A|94588466smoke1747901|wheat|Oz|Unknown|7|callyoughtableantiought| +15217|AAAAAAAABHLDAAAA|1997-10-27||Rules can come largely deep wings; soviet, yellow kilometres could eat never bright, entire proposals. More pleased museums may n|9.78|5.96|9008002|namelessmaxi #2|8|romance|9|Books|311|oughtoughtpri|N/A|85823117528saddle798|lawn|Gross|Unknown|1|ationoughtableantiought| +15218|AAAAAAAACHLDAAAA|1997-10-27|2000-10-26|Early ways work russian banks. Fair services can locate excellent persons. At all public cars stand before a scholars. British, new years should not draw in order in a forms. Ce|3.63|2.64|1004001|edu packamalg #1|4|swimwear|1|Women|131|oughtpriought|economy|7930481545mint488810|rosy|Gram|Unknown|22|eingoughtableantiought| +15219|AAAAAAAACHLDAAAA|2000-10-27||Early ways work russian banks. Fair services can locate excellent persons. At all public cars stand before a scholars. British, new years should not draw in order in a forms. Ce|1.47|2.64|6008004|namelesscorp #4|4|mens watch|6|Jewelry|751|oughtantiation|N/A|34681996965rose99035|orchid|Oz|Unknown|82|n stoughtableantiought| +15220|AAAAAAAAEHLDAAAA|1997-10-27|1999-10-27|Obvious, afraid poli|4.05|1.94|7002001|importobrand #1|2|bedding|7|Home|300|barbarpri|N/A|060540138845spring85|maroon|Tbl|Unknown|2|barableableantiought| +15221|AAAAAAAAEHLDAAAA|1999-10-28|2001-10-26|Following directors take big properties; specific, black sons used to send merely real, tiny refugees. Sufficiently open acids must prove also current competitors. |7.86|1.94|4003002|exportiedu pack #2|2|kids|4|Shoes|340|baresepri|petite|060540138845spring85|tomato|N/A|Unknown|26|oughtableableantiought| +15222|AAAAAAAAEHLDAAAA|2001-10-27||Important, political decisions continue to a personnel. Possible lips put earlier subjective countries. Republics ought to go teachers. Arms permit th|2.74|0.90|7011003|amalgnameless #3|11|accent|7|Home|386|baresepri|N/A|71400656020mint05062|maroon|Bunch|Unknown|14|ableableableantiought| +15223|AAAAAAAAHHLDAAAA|1997-10-27||Provincial, high matters can perform mutual duties; old, adequate months advise much good times. There reasonable authorities disturb similar things. Now gol|0.93|0.50|4003002|exportiedu pack #2|3|kids|4|Shoes|52|ableanti|medium|5781435peach81770598|tan|Unknown|Unknown|20|priableableantiought| +15224|AAAAAAAAIHLDAAAA|1997-10-27|2000-10-26|Fully young sentences accept usually in a employers. Social tonnes must remember twice quiet prices. Already french shoulders improve meanwhile symptoms. Very politi|6.32|4.17|6009001|maxicorp #1|9|womens watch|6|Jewelry|197|ationn stought|N/A|393pink4766488568018|beige|Lb|Unknown|25|eseableableantiought| +15225|AAAAAAAAIHLDAAAA|2000-10-27||Fully young sentences accept usually in a employers. Social tonnes must remember twice quiet prices. Already french shoulders improve meanwhile symptoms. Very politi|5.45|2.72|4001002|amalgedu pack #2|9|womens|4|Shoes|887|ationn stought|petite|393pink4766488568018|peru|Case|Unknown|20|antiableableantiought| +15226|AAAAAAAAKHLDAAAA|1997-10-27|1999-10-27|Trials shall continue already readily valid plants. Inte|5.13|3.28|5002001|importoscholar #1|2|country|5|Music|341|oughtesepri|N/A|4505089396770salmon6|rosy|Case|Unknown|1|callyableableantiought| +15227|AAAAAAAAKHLDAAAA|1999-10-28|2001-10-26|Trials shall continue already readily valid plants. Inte|2.90|3.28|2001002|amalgimporto #2|2|accessories|2|Men|242|ableeseable|large|50sandy3478744314751|lime|Ounce|Unknown|9|ationableableantiought| +15228|AAAAAAAAKHLDAAAA|2001-10-27||Trials shall continue already readily valid plants. Inte|1.92|0.99|2001002|univamalgamalg #14|10|memory|10|Electronics|242|ableeseable|N/A|50sandy3478744314751|steel|Cup|Unknown|9|eingableableantiought| +15229|AAAAAAAANHLDAAAA|1997-10-27||For instance bitter expenses talk too more national servants; tales should say natural concerns; international questions may compare yesterd|70.70|52.31|10016015|corpamalgamalg #15|16|wireless|10|Electronics|571|oughtationanti|N/A|violet55645127085865|purple|Oz|Unknown|80|n stableableantiought| +15230|AAAAAAAAOHLDAAAA|1997-10-27|2000-10-26|Good, fast facilities admir|2.69|1.10|1001001|amalgamalg #1|1|dresses|1|Women|318|eingoughtpri|medium|9plum624642738382500|powder|Carton|Unknown|3|barpriableantiought| +15231|AAAAAAAAOHLDAAAA|2000-10-27||Full, southern ideas may w|2.63|1.10|3002002|importoexporti #2|2|infants|3|Children|204|esebarable|small|2154585873294sky6622|orchid|Lb|Unknown|35|oughtpriableantiought| +15232|AAAAAAAAAILDAAAA|1997-10-27|1999-10-27|Thirdly christian fragments shave very well large structures. Young, coming attitudes may i|9.17|5.31|7004003|edu packbrand #3|4|curtains/drapes|7|Home|206|callybarable|N/A|3958894592steel18250|smoke|Dozen|Unknown|37|ablepriableantiought| +15233|AAAAAAAAAILDAAAA|1999-10-28|2001-10-26|Thirdly christian fragments shave very well large structures. Young, coming attitudes may i|11.67|5.31|9014006|edu packunivamalg #6|14|sports|9|Books|169|callybarable|N/A|1149629royal27243956|saddle|Gross|Unknown|58|pripriableantiought| +15234|AAAAAAAAAILDAAAA|2001-10-27||Lines cut substantial, coming visitors. Low, open cuts return never old thousands. Accidents should not lead like a circumstances. Pale thin|1.62|0.95|4003001|exportiedu pack #1|3|kids|4|Shoes|201|callybarable|large|3379167272693peru900|papaya|Carton|Unknown|27|esepriableantiought| +15235|AAAAAAAADILDAAAA|1997-10-27||Countries must lose home. Difficult structures could believe conservatives. Things may not stand african pounds. Readers|65.80|38.82|2001002|amalgimporto #2|1|accessories|2|Men|240|bareseable|N/A|67737155spring496319|salmon|Ounce|Unknown|17|antipriableantiought| +15236|AAAAAAAAEILDAAAA|1997-10-27|2000-10-26|Bad, able systems shall fall else. Nuclear, economic ways put in an paths. Serious, labour women must not muster however. Wide new readers ought to help |1.36|0.78|9002009|importomaxi #9|2|business|9|Books|96|callyn st|N/A|759853278plum7255268|sandy|N/A|Unknown|92|callypriableantiought| +15237|AAAAAAAAEILDAAAA|2000-10-27||Bad, able systems shall fall else. Nuclear, economic ways put in an paths. Serious, labour women must not muster however. Wide new readers ought to help |1.52|0.78|4001002|amalgedu pack #2|2|womens|4|Shoes|96|callyn st|medium|759853278plum7255268|sienna|Case|Unknown|54|ationpriableantiought| +15238|AAAAAAAAGILDAAAA|1997-10-27|1999-10-27|Movements shall not indicate relatively on the terms; easily loose reasons prove there either prime boundaries. Disciplinary, nervous years cut better scottish, english contracts; regul|2.94|2.14|10005007|scholarunivamalg #7|5|karoke|10|Electronics|304|esebarpri|N/A|768731695906709lime7|royal|Oz|Unknown|51|eingpriableantiought| +15239|AAAAAAAAGILDAAAA|1999-10-28|2001-10-26|Shares change for a enemies. Today other flowers withdraw before a pare|3.42|3.00|3002002|importoexporti #2|2|infants|3|Children|475|antiationese|extra large|768731695906709lime7|frosted|Bunch|Unknown|40|n stpriableantiought| +15240|AAAAAAAAGILDAAAA|2001-10-27||So new systems shall meet into a studies. Sweet tears can close so nasty, close applications. Middle-class bodies construct right net birds. Angry courts ou|4.59|3.48|7012001|importonameless #1|12|paint|7|Home|208|antiationese|N/A|6477667874seashell65|salmon|Pallet|Unknown|18|bareseableantiought| +15241|AAAAAAAAJILDAAAA|1997-10-27||Birds feel no longer much general cattle. Right, various cameras get closer. Resources could not offer just times. Only schemes should see so cards. Extreme, open girl|6.02|4.15|7003002|exportibrand #2|3|kids|7|Home|483|prieingese|N/A|515067663032puff2442|saddle|Bunch|Unknown|19|oughteseableantiought| +15242|AAAAAAAAKILDAAAA|1997-10-27|2000-10-26|Also parental feet must suggest now relationships|1.19|0.98|9006003|corpmaxi #3|6|parenting|9|Books|450|barantiese|N/A|52448752097955peru34|spring|Ounce|Unknown|22|ableeseableantiought| +15243|AAAAAAAAKILDAAAA|2000-10-27||Also parental feet must suggest now relationships|0.52|0.19|6011004|amalgbrand #4|6|semi-precious|6|Jewelry|450|barantiese|N/A|7120835red9422839594|ivory|Lb|Unknown|8|prieseableantiought| +15244|AAAAAAAAMILDAAAA|1997-10-27|1999-10-27|So thick services might leave very only retail c|2.84|2.13|7009005|maxibrand #5|9|mattresses|7|Home|528|eingableanti|N/A|73royal9984066745794|pale|Pallet|Unknown|12|eseeseableantiought| +15245|AAAAAAAAMILDAAAA|1999-10-28|2001-10-26|Only capitalist situations shall ask more locally royal unions. Legs will suit then basic proceedings. Names see on a occupati|9.82|2.13|3004002|edu packexporti #2|9|school-uniforms|3|Children|528|eingableanti|medium|73royal9984066745794|wheat|Gram|Unknown|19|antieseableantiought| +15246|AAAAAAAAMILDAAAA|2001-10-27||Services encourage. Centrally fun figures go even. Social, unique firms draw yet problems. Political plants mean apart social, responsible |0.68|0.31|3004002|edu packnameless #7|14|glassware|7|Home|719|eingableanti|N/A|825514826pink7517526|rosy|Carton|Unknown|5|callyeseableantiought| +15247|AAAAAAAAPILDAAAA|1997-10-27||Special matters may not forget a little other drugs. Also possible standards might retain sales. Difficult, small prices forget frequently for a hours. Explicit, true things may exchange modern cases|0.66|0.20|7004008|edu packbrand #8|4|curtains/drapes|7|Home|299|n stn stable|N/A|16470948salmon567895|seashell|Ton|Unknown|29|ationeseableantiought| +15248|AAAAAAAAAJLDAAAA|1997-10-27|2000-10-26|Quick possible jobs m|0.57|0.27|4001001|amalgedu pack #1|1|womens|4|Shoes|615|antioughtcally|medium|1813926120midnight34|grey|Bunch|Unknown|70|eingeseableantiought| +15249|AAAAAAAAAJLDAAAA|2000-10-27||Voluntary, other changes might not explain even appropriate metres. Quickly pleased numbers shall carry just about long clear proposals. Fine christian goals ask both thus running carers;|0.25|0.11|9010010|univunivamalg #10|1|travel|9|Books|615|antioughtcally|N/A|421126111tan46558142|lace|Pallet|Unknown|10|n steseableantiought| +15250|AAAAAAAACJLDAAAA|1997-10-27|1999-10-27|Free, american organisms will inform at all. Today elaborate laws kiss sympathetic, old police. Comforta|4.51|2.39|10013002|exportiamalgamalg #2|13|stereo|10|Electronics|529|n stableanti|N/A|1768maroon3992968227|smoke|Bundle|Unknown|11|barantiableantiought| +15251|AAAAAAAACJLDAAAA|1999-10-28|2001-10-26|Free, american organisms will inform at all. Today elaborate laws kiss sympathetic, old police. Comforta|4.29|3.73|5003002|exportischolar #2|13|pop|5|Music|529|n stableanti|N/A|1768maroon3992968227|lawn|Dozen|Unknown|30|oughtantiableantiought| +15252|AAAAAAAACJLDAAAA|2001-10-27||Winds will build as intense, medical processes. Just star directions would come also in|3.68|3.73|5003002|importoamalg #1|13|fragrances|1|Women|529|n stableanti|large|1768maroon3992968227|peach|Ton|Unknown|13|ableantiableantiought| +15253|AAAAAAAAFJLDAAAA|1997-10-27||Courses urge towns. Simply small words must not read more with a stocks. Centres lean available objections; external officials will edit around stories; op|2.14|1.66|3002002|importoexporti #2|2|infants|3|Children|972|ableationn st|small|928298233powder10349|sandy|Pallet|Unknown|44|priantiableantiought| +15254|AAAAAAAAGJLDAAAA|1997-10-27|2000-10-26|Single, professional tenants co|||9009003||9|science|9|Books||||3695671608672powder1|||Unknown|36|| +15255|AAAAAAAAGJLDAAAA|2000-10-27||Single, professional tenants co|3.49|3.07|9009003|amalgamalg #2|1|dresses|1|Women|936|callyprin st|small|3695671608672powder1|plum|Ounce|Unknown|3|antiantiableantiought| +15256|AAAAAAAAIJLDAAAA|1997-10-27|1999-10-27|Problems will leave also known, british cars. More than alone heads determine afterwards serious observations; years locate on the problems. Logical forces|5.19|2.07|4001001|amalgedu pack #1|1|womens|4|Shoes|850|barantieing|N/A|3398057258seashell76|sienna|N/A|Unknown|59|callyantiableantiought| +15257|AAAAAAAAIJLDAAAA|1999-10-28|2001-10-26|Problems will leave also known, british cars. More than alone heads determine afterwards serious observations; years locate on the problems. Logical forces|5.11|2.07|4001001|exportibrand #2|13|loose stones|6|Jewelry|850|barantieing|N/A|525201005222sienna92|royal|Case|Unknown|21|ationantiableantiought| +15258|AAAAAAAAIJLDAAAA|2001-10-27||Problems will leave also known, british cars. More than alone heads determine afterwards serious observations; years locate on the problems. Logical forces|3.07|1.87|4001001|amalgamalg #1|1|dresses|1|Women|967|ationcallyn st|large|80598118188lime40901|salmon|Tsp|Unknown|66|eingantiableantiought| +15259|AAAAAAAALJLDAAAA|1997-10-27||Black, trying systems help ever businessmen. Children illus|3.09|2.59|7013010|exportinameless #10|13|wallpaper|7|Home|369|n stcallypri|N/A|85011purple645784578|light|Pallet|Unknown|58|n stantiableantiought| +15260|AAAAAAAAMJLDAAAA|1997-10-27|2000-10-26|Lovely, main police take very at a towns. Current, basic customers imply either. Good, pink sites die also good plans; miserable supporters should not receive. Roman, relevant funds shall|0.52|0.46|10015012|scholaramalgamalg #12|15|portable|10|Electronics|220|barableable|N/A|5navajo6629529366590|turquoise|N/A|Unknown|66|barcallyableantiought| +15261|AAAAAAAAMJLDAAAA|2000-10-27||Lovely, main police take very at a towns. Current, basic customers imply either. Good, pink sites die also good plans; miserable supporters should not receive. Roman, relevant funds shall|3.38|0.46|6015006|scholarbrand #6|15|custom|6|Jewelry|220|barableable|N/A|1955572134smoke14685|pink|Pound|Unknown|46|oughtcallyableantiought| +15262|AAAAAAAAOJLDAAAA|1997-10-27|1999-10-27|Forms take very electoral witnesses. Then effective examples will not win other, continuous workers. Very small books may retain certai|8.27|6.78|9014011|edu packunivamalg #11|14|sports|9|Books|15|antiought|N/A|ivory168616006038467|plum|Gross|Unknown|7|ablecallyableantiought| +15263|AAAAAAAAOJLDAAAA|1999-10-28|2001-10-26|Forms take very electoral witnesses. Then effective examples will not win other, continuous workers. Very small books may retain certai|1.92|6.78|9014011|edu packscholar #2|4|classical|5|Music|88|antiought|N/A|ivory168616006038467|purple|Tsp|Unknown|2|pricallyableantiought| +15264|AAAAAAAAOJLDAAAA|2001-10-27||Forms take very electoral witnesses. Then effective examples will not win other, continuous workers. Very small books may retain certai|3.60|6.78|9014011|exportiimporto #1|3|pants|2|Men|4|antiought|petite|yellow84066692212763|salmon|Unknown|Unknown|29|esecallyableantiought| +15265|AAAAAAAABKLDAAAA|1997-10-27||Tenants must not review now certain, right buses; daily strings used to overcome votes. Formal bombs make less tiny, statutory countries. W|5.46|4.31|2004002|edu packimporto #2|4|sports-apparel|2|Men|261|oughtcallyable|extra large|9814706531pink302509|powder|Bunch|Unknown|2|anticallyableantiought| +15266|AAAAAAAACKLDAAAA|1997-10-27|2000-10-26|British, active changes must take with a babies. Leading, electoral muscles leave states. Parents take thus othe|85.50|64.12|4004001|edu packedu pack #1|4|athletic|4|Shoes|437|ationpriese|petite|25534599743564mint60|white|Oz|Unknown|34|callycallyableantiought| +15267|AAAAAAAACKLDAAAA|2000-10-27||Facilities sense for instance under a meetings; lo|78.03|64.12|1003002|exportiamalg #2|3|maternity|1|Women|540|bareseanti|petite|25534599743564mint60|purple|Tbl|Unknown|27|ationcallyableantiought| +15268|AAAAAAAAEKLDAAAA|1997-10-27|1999-10-27|Also other farmers spare with a c|3.01|2.37|1002001|importoamalg #1|2|fragrances|1|Women|248|eingeseable|extra large|972768522orange52688|navy|Pound|Unknown|69|eingcallyableantiought| +15269|AAAAAAAAEKLDAAAA|1999-10-28|2001-10-26|Always interesting fields can use also funn|2.42|1.28|4003002|exportiedu pack #2|3|kids|4|Shoes|319|n stoughtpri|large|523035308indian28000|turquoise|Case|Unknown|85|n stcallyableantiought| +15270|AAAAAAAAEKLDAAAA|2001-10-27||Longer private prisoners shall not make fellow boys; impressive, major objects can go slow. Relative|1.64|1.28|4003001|exportiedu pack #1|3|kids|4|Shoes|319|n stoughtpri|petite|5689768763wheat30422|rose|Case|Unknown|32|barationableantiought| +15271|AAAAAAAAHKLDAAAA|1997-10-27||Days should not tak|4.80|1.68|2004002|edu packimporto #2|4|sports-apparel|2|Men|320|barablepri|medium|107turquoise11836242|red|N/A|Unknown|57|oughtationableantiought| +15272|AAAAAAAAIKLDAAAA|1997-10-27|2000-10-26|Detailed calls kill perhaps so-called activities. Particular, back proposals shall not observe just normally administrative commitments. Girls begin single trousers. Consequently yo|4.94|4.34|2002001|importoimporto #1|2|shirts|2|Men|528|eingableanti|medium|022722450861thistle7|steel|Unknown|Unknown|27|ableationableantiought| +15273|AAAAAAAAIKLDAAAA|2000-10-27||Positive needs respond within a matters; places listen hot mountains. Operations will go from a |4.58|3.43|2002001|importoimporto #2|2|shirts|2|Men|528|eingableanti|medium|022722450861thistle7|sienna|N/A|Unknown|63|priationableantiought| +15274|AAAAAAAAKKLDAAAA|1997-10-27|1999-10-27|Levels may use essentially within the effects. Quickly local pictures should call enough officials. Here opening hours would pray ot|9.51|7.13|8015005|scholarmaxi #5|15|fishing|8|Sports|173|priationought|N/A|18tomato762471039354|goldenrod|Tsp|Unknown|5|eseationableantiought| +15275|AAAAAAAAKKLDAAAA|1999-10-28|2001-10-26|Levels may use essentially within the effects. Quickly local pictures should call enough officials. Here opening hours would pray ot|3.65|2.66|7013006|exportinameless #6|15|wallpaper|7|Home|173|priationought|N/A|3227694171lime350065|sky|Dram|Unknown|38|antiationableantiought| +15276|AAAAAAAAKKLDAAAA|2001-10-27||At all important systems may operate there. A bit harsh cases would occur probably large rates. |3.48|1.87|7013006|importoexporti #1|15|infants|3|Children|644|priationought|large|121338973936powder96|mint|Pallet|Unknown|30|callyationableantiought| +15277|AAAAAAAANKLDAAAA|1997-10-27||Of course able men cannot see usually old, annual friends. Aware, different things protect small, other activities; exactly quiet experiences must h|5.34|3.89|1003002|exportiamalg #2|3|maternity|1|Women|102|ablebarought|large|8peach83494640287405|sandy|Pound|Unknown|14|ationationableantiought| +15278|AAAAAAAAOKLDAAAA|1997-10-27|2000-10-26|Cold, correct messages used to repeat political experiences. Real, far horses leave about friends. Relationships may|0.35|0.25|5004001|edu packscholar #1|4|classical|5|Music|328|eingablepri|N/A|514230127navajo55335|almond|Unknown|Unknown|66|eingationableantiought| +15279|AAAAAAAAOKLDAAAA|2000-10-27||Products recover shares; particular communities should not act for a forests. Again small years believe. Operations ought to al|7.40|0.25|4001002|amalgedu pack #2|1|womens|4|Shoes|600|barbarcally|medium|514230127navajo55335|cornsilk|N/A|Unknown|43|n stationableantiought| +15280|AAAAAAAAALLDAAAA|1997-10-27|1999-10-27|Poor, evolutionary cases might understand much white stars. High stages should not move terms. Lines ought to find firmly universal members. Gastric ages help doors; cheerful, old fees fall; nation|9.74|5.94|7008001|namelessbrand #1|8|lighting|7|Home|1000|barbarbarought|N/A|57lemon8009218765114|plum|Unknown|Unknown|40|bareingableantiought| +15281|AAAAAAAAALLDAAAA|1999-10-28|2001-10-26|Poor, evolutionary cases might understand much white stars. High stages should not move terms. Lines ought to find firmly universal members. Gastric ages help doors; cheerful, old fees fall; nation|9.22|5.16|7008001|amalgcorp #2|1|birdal|6|Jewelry|1000|barbarbarought|N/A|8137aquamarine214779|sky|Dram|Unknown|21|oughteingableantiought| +15282|AAAAAAAAALLDAAAA|2001-10-27||Poor, evolutionary cases might understand much white stars. High stages should not move terms. Lines ought to find firmly universal members. Gastric ages help doors; cheerful, old fees fall; nation|0.46|0.30|9010007|univunivamalg #7|1|travel|9|Books|465|anticallyese|N/A|8137aquamarine214779|spring|Case|Unknown|5|ableeingableantiought| +15283|AAAAAAAADLLDAAAA|1997-10-27||Remaining options should deal straight extra, ruling|5.26|3.73|5003002|exportischolar #2|3|pop|5|Music|467|ationcallyese|N/A|909039740violet10016|ivory|Cup|Unknown|60|prieingableantiought| +15284|AAAAAAAAELLDAAAA|1997-10-27|2000-10-26|Fundamental firms should encourage relations. Folk move maybe statutory circumstances. Only european rules restric|3.38|1.96|2004001|edu packimporto #1|4|sports-apparel|2|Men|653|priantically|large|61015dim530181896351|violet|Tbl|Unknown|28|eseeingableantiought| +15285|AAAAAAAAELLDAAAA|2000-10-27||Considerable, new songs accept only, environmental hotels. Social weekends worry quite experiments. Necessary, social forms go at least up to a |3.60|3.13|2004001|exportinameless #6|4|basketball|8|Sports|412|ableoughtese|N/A|61015dim530181896351|navajo|Gross|Unknown|19|antieingableantiought| +15286|AAAAAAAAGLLDAAAA|1997-10-27|1999-10-27|Prov|2.61|1.12|9008011|namelessmaxi #11|8|romance|9|Books|113|prioughtought|N/A|834373176774pale0608|powder|Dozen|Unknown|26|callyeingableantiought| +15287|AAAAAAAAGLLDAAAA|1999-10-28|2001-10-26|Spec|5.11|3.16|1002002|importoamalg #2|8|fragrances|1|Women|809|n stbareing|petite|834373176774pale0608|salmon|Oz|Unknown|75|ationeingableantiought| +15288|AAAAAAAAGLLDAAAA|2001-10-27||Interested results should furnish welsh fingers. Contracts take. Only trees get rarely in an aspects. Even new children might not imitate locally long, other authorities; there vital vie|9.78|3.16|1002002|scholarbrand #1|15|custom|6|Jewelry|809|n stbareing|N/A|8094713299282azure46|lavender|Bundle|Unknown|28|eingeingableantiought| +15289|AAAAAAAAJLLDAAAA|1997-10-27||Expectations arise together feelings. Difficult allies make musical, indian conditions. Fairly healthy effects might make equally stones.|7.13|6.13|4004002|edu packedu pack #2|4|athletic|4|Shoes|95|antin st|small|551tan10101926935571|steel|Unknown|Unknown|33|n steingableantiought| +15290|AAAAAAAAKLLDAAAA|1997-10-27|2000-10-26|Partners could contact from a efforts. Mysterious, royal reports could suffer excellent, other divisions. Strong elements may enable for example small things. Judges used to make. Suffi|58.19|29.09|9005003|scholarmaxi #3|5|history|9|Books|70|baration|N/A|594439442559sky82851|smoke|Unknown|Unknown|14|barn stableantiought| +15291|AAAAAAAAKLLDAAAA|2000-10-27||Partners could contact from a efforts. Mysterious, royal reports could suffer excellent, other divisions. Strong elements may enable for example small things. Judges used to make. Suffi|0.29|0.13|10002009|importounivamalg #9|2|camcorders|10|Electronics|70|baration|N/A|594439442559sky82851|khaki|Tbl|Unknown|13|oughtn stableantiought| +15292|AAAAAAAAMLLDAAAA|1997-10-27|1999-10-27|Sales work urgently years. Large, suitable police try about a letters|9.98|2.99|1001001|amalgamalg #1|1|dresses|1|Women|502|ablebaranti|large|173tomato90930247646|tomato|Carton|Unknown|46|ablen stableantiought| +15293|AAAAAAAAMLLDAAAA|1999-10-28|2001-10-26|Young, difficult cats should not suit well then financial windows; social, polish metres discover about. Female, young increases use sufficiently fresh, similar feelings; new, social foods seem|6.10|2.99|8015004|scholarmaxi #4|1|fishing|8|Sports|502|ablebaranti|N/A|970887336plum9730898|medium|Lb|Unknown|18|prin stableantiought| +15294|AAAAAAAAMLLDAAAA|2001-10-27||Sufficiently urban years must encourage straight important, distinctive periods. Scottish, european agents affect; states know more after a partners. Other volumes push months. Fixed fields used to|3.80|2.77|8015004|edu packexporti #1|4|school-uniforms|3|Children|502|ablebaranti|medium|970887336plum9730898|puff|Bundle|Unknown|18|esen stableantiought| +15295|AAAAAAAAPLLDAAAA|1997-10-27||Old eyes could not give later issues. Claims might|9.00|5.13|9001008|amalgmaxi #8|1|arts|9|Books|489|n steingese|N/A|343sky42265204574789|salmon|Gram|Unknown|29|antin stableantiought| +15296|AAAAAAAAAMLDAAAA|1997-10-27|2000-10-26|Yellow, important supplies will not take; more safe months would go here almost disabled hands. Blocks would com|6.59|2.43|7002007|importobrand #7|2|bedding|7|Home|503|pribaranti|N/A|55578851618181ghost5|sky|Each|Unknown|11|callyn stableantiought| +15297|AAAAAAAAAMLDAAAA|2000-10-27||||||importoimporto #2||||Men|||||thistle||Unknown|65|| +15298|AAAAAAAACMLDAAAA|1997-10-27|1999-10-27|Purposes appear well eyes. Of course possible ways used|3.54|3.00|7004009|edu packbrand #9|4|curtains/drapes|7|Home|194|esen stought|N/A|13372464dodger962871|slate|Pound|Unknown|12|eingn stableantiought| +15299|AAAAAAAACMLDAAAA|1999-10-28|2001-10-26|Purposes appear well eyes. Of course possible ways used|5.29|2.16|7004009|amalgscholar #2|1|rock|5|Music|378|eingationpri|N/A|13372464dodger962871|beige|Ounce|Unknown|46|n stn stableantiought| +15300|AAAAAAAACMLDAAAA|2001-10-27||Purposes appear well eyes. Of course possible ways used|1.12|2.16|1004001|edu packamalg #1|1|swimwear|1|Women|378|eingationpri|small|7365557679198tan1597|sandy|N/A|Unknown|79|barbarpriantiought| +15301|AAAAAAAAFMLDAAAA|1997-10-27||Green, right pounds take very barely little systems. Police ought to appear then old movies. Strictly national products can decide below. Just emotional sides could not r|8.06|3.70|4001002|amalgedu pack #2|1|womens|4|Shoes|437|ationpriese|medium|6349841040sienna5192|violet|Oz|Unknown|31|oughtbarpriantiought| +15302|AAAAAAAAGMLDAAAA|1997-10-27|2000-10-26|Complaints assist most necessary, multiple|30.52|22.89|10003003|exportiunivamalg #3|3|dvd/vcr players|10|Electronics|243|prieseable|N/A|5240424tan1269978592|goldenrod|Pound|Unknown|32|ablebarpriantiought| +15303|AAAAAAAAGMLDAAAA|2000-10-27||Complaints assist most necessary, multiple|7.68|22.89|10003003|corpcorp #8|3|rings|6|Jewelry|28|eingable|N/A|5240424tan1269978592|smoke|Ton|Unknown|3|pribarpriantiought| +15304|AAAAAAAAIMLDAAAA|1997-10-27|1999-10-27|Industrial, inc p|3.07|1.81|5004001|edu packscholar #1|4|classical|5|Music|566|callycallyanti|N/A|1688056salmon8991820|white|Bunch|Unknown|47|esebarpriantiought| +15305|AAAAAAAAIMLDAAAA|1999-10-28|2001-10-26|So available roads investigate less; yesterday other hands enable. Changes learn offici|2.36|0.92|6003002|exporticorp #2|3|gold|6|Jewelry|133|pripriought|N/A|1688056salmon8991820|metallic|Carton|Unknown|65|antibarpriantiought| +15306|AAAAAAAAIMLDAAAA|2001-10-27||So available roads investigate less; yesterday other hands enable. Changes learn offici|0.81|0.49|4001001|amalgedu pack #1|3|womens|4|Shoes|133|pripriought|petite|1688056salmon8991820|powder|Ounce|Unknown|46|callybarpriantiought| +15307|AAAAAAAALMLDAAAA|1997-10-27||Then detailed games will remember here in the problems. Justifiably important c|4.07|2.88|1003002|exportiamalg #2|3|maternity|1|Women|19|n stought|large|003564134811pale0676|plum|Unknown|Unknown|7|ationbarpriantiought| +15308|AAAAAAAAMMLDAAAA|1997-10-27|2000-10-26|Wet, new increases bring. Questions used to answer even. Systems must respond specifically classical atti|3.36|1.20|4002001|importoedu pack #1|2|mens|4|Shoes|155|antiantiought|petite|916329181antique6915|ivory|Tbl|Unknown|58|eingbarpriantiought| +15309|AAAAAAAAMMLDAAAA|2000-10-27||Wet, new increases bring. Questions used to answer even. Systems must respond specifically classical atti|2.09|1.20|1002002|importoamalg #2|2|fragrances|1|Women|155|antiantiought|petite|93363441758magenta16|plum|Bundle|Unknown|8|n stbarpriantiought| +15310|AAAAAAAAOMLDAAAA|1997-10-27|1999-10-27|Measures should create over tal|1.56|1.13|4001001|amalgedu pack #1|1|womens|4|Shoes|151|oughtantiought|medium|7564pink961535825599|sienna|Bunch|Unknown|98|baroughtpriantiought| +15311|AAAAAAAAOMLDAAAA|1999-10-28|2001-10-26|Raw, white w|3.45|1.13|4001002|amalgedu pack #2|1|womens|4|Shoes|298|oughtantiought|large|35deep20601626169981|slate|N/A|Unknown|100|oughtoughtpriantiought| +15312|AAAAAAAAOMLDAAAA|2001-10-27||Amazing, new hands shall give on a advantages. Councils make old, available rights. Sure, strong eyes may not run; irrelevant sets sit rather |3.69|1.13|4001002|importoscholar #1|1|country|5|Music|55|oughtantiought|N/A|35deep20601626169981|plum|N/A|Unknown|66|ableoughtpriantiought| +15313|AAAAAAAABNLDAAAA|1997-10-27||Natural sounds can take unexpectedly collective books. Finally wonderful months ought to yield financial, jewish stories; small, awful rules w|1.09|0.40|6006002|corpcorp #2|6|rings|6|Jewelry|403|pribarese|N/A|863351purple63756930|lemon|Oz|Unknown|53|prioughtpriantiought| +15314|AAAAAAAACNLDAAAA|1997-10-27|2000-10-26|Profoundly useless women might go desperate, international remarks. Different, subject lines can arrange. Personal conditions should fin|9.50|6.55|9015003|scholarunivamalg #3|15|fiction|9|Books|905|antibarn st|N/A|white098315930615133|maroon|Pallet|Unknown|9|eseoughtpriantiought| +15315|AAAAAAAACNLDAAAA|2000-10-27||Profoundly useless women might go desperate, international remarks. Different, subject lines can arrange. Personal conditions should fin|7.77|2.48|9015003|edu packnameless #6|4|camping|8|Sports|905|antibarn st|N/A|5896971pink206524424|cream|Bunch|Unknown|18|antioughtpriantiought| +15316|AAAAAAAAENLDAAAA|1997-10-27|1999-10-27|Chinese features may want later unique, similar readers. Rebels could knock well fixed films|8.09|4.69|2001001|amalgimporto #1|1|accessories|2|Men|377|ationationpri|large|8violet6498884564440|grey|Dozen|Unknown|21|callyoughtpriantiought| +15317|AAAAAAAAENLDAAAA|1999-10-28|2001-10-26|Chinese features may want later unique, similar readers. Rebels could knock well fixed films|8.32|5.74|6013006|exportibrand #6|13|loose stones|6|Jewelry|58|einganti|N/A|8violet6498884564440|salmon|Ounce|Unknown|47|ationoughtpriantiought| +15318|AAAAAAAAENLDAAAA|2001-10-27||Chinese features may want later unique, similar readers. Rebels could knock well fixed films|0.75|5.74|6013006|importoedu pack #1|2|mens|4|Shoes|329|n stablepri|medium|8violet6498884564440|snow|Bundle|Unknown|32|eingoughtpriantiought| +15319|AAAAAAAAHNLDAAAA|1997-10-27||Terms teach much difficult workers. Important, major arms shall not allay in order living restaurants. Able skills used to affect private, decisive ideas. Eyes ou|4.99|4.44|4004002|edu packedu pack #2|4|athletic|4|Shoes|293|prin stable|extra large|red99232429559473165|peru|Bunch|Unknown|8|n stoughtpriantiought| +15320|AAAAAAAAINLDAAAA|1997-10-27|2000-10-26|Extremely normal houses should meet broad, available partners. Concerned, nuclear firms look origins. British beliefs try. Lacki|7.18|5.45|10010004|univamalgamalg #4|10|memory|10|Electronics|287|ationeingable|N/A|62742224284854ghost0|olive|Tsp|Unknown|12|barablepriantiought| +15321|AAAAAAAAINLDAAAA|2000-10-27||Extremely normal houses should meet broad, available partners. Concerned, nuclear firms look origins. British beliefs try. Lacki|6.94|5.82|10010004|edu packunivamalg #10|10|sports|9|Books|287|ationeingable|N/A|5moccasin44893939300|powder|Pound|Unknown|86|oughtablepriantiought| +15322|AAAAAAAAKNLDAAAA|1997-10-27|1999-10-27|Secretly soviet categori|4.99|2.94|10001006|amalgunivamalg #6|1|cameras|10|Electronics|315|antioughtpri|N/A|1831762slate74997138|turquoise|Gram|Unknown|40|ableablepriantiought| +15323|AAAAAAAAKNLDAAAA|1999-10-28|2001-10-26|Secretly soviet categori|9.63|2.88|10001006|importoscholar #2|1|country|5|Music|109|n stbarought|N/A|64388307523400rose01|lace|Bunch|Unknown|59|priablepriantiought| +15324|AAAAAAAAKNLDAAAA|2001-10-27||Secretly soviet categori|1.61|2.88|10001006|scholarnameless #5|1|fitness|8|Sports|583|prieinganti|N/A|64388307523400rose01|khaki|Bunch|Unknown|57|eseablepriantiought| +15325|AAAAAAAANNLDAAAA|1997-10-27||Important offices clear now; official compon|7.38|4.42|4001002|amalgedu pack #2|1|womens|4|Shoes|624|eseablecally|N/A|lemon228390003872065|indian|Carton|Unknown|17|antiablepriantiought| +15326|AAAAAAAAONLDAAAA|1997-10-27|2000-10-26|Backs stop very prices. Very available years withstand often eyebrows. Syste|6.43|5.14|4002001|importoedu pack #1|2|mens|4|Shoes|110|baroughtought|petite|8295392seashell24570|yellow|Tsp|Unknown|19|callyablepriantiought| +15327|AAAAAAAAONLDAAAA|2000-10-27||Backs stop very prices. Very available years withstand often eyebrows. Syste|2.83|2.17|4002001|amalgamalg #2|1|dresses|1|Women|229|n stableable|petite|8295392seashell24570|turquoise|Bunch|Unknown|22|ationablepriantiought| +15328|AAAAAAAAAOLDAAAA|1997-10-27|1999-10-27|Certain individuals shall race safely cruelly necessary terms; young, high guns take to a hands. Vali|2.84|2.13|7001009|amalgbrand #9|1|bathroom|7|Home|169|n stcallyought|N/A|2sienna9503437362179|lawn|Case|Unknown|5|eingablepriantiought| +15329|AAAAAAAAAOLDAAAA|1999-10-28|2001-10-26|Smooth, private scenes might say widely useful|5.97|2.13|7001009|edu packedu pack #2|4|athletic|4|Shoes|539|n stprianti|large|501midnight361670537|salmon|Gram|Unknown|49|n stablepriantiought| +15330|AAAAAAAAAOLDAAAA|2001-10-27||Obviously small ideas could not keep honours. Months should not prepare also. Po|4.09|3.06|7001009|corpbrand #1|6|rugs|7|Home|160|barcallyought|N/A|501midnight361670537|medium|Tsp|Unknown|97|barpripriantiought| +15331|AAAAAAAADOLDAAAA|1997-10-27||Young bottles dance with a activities. Most strategic risks stop in the children. Rather ready years pay at a proposals. Human features filter level, dependent co|0.90|0.50|10015015|scholaramalgamalg #15|15|portable|10|Electronics|506|callybaranti|N/A|177violet07266439952|green|Box|Unknown|46|oughtpripriantiought| +15332|AAAAAAAAEOLDAAAA|1997-10-27|2000-10-26|Further big results shield. So new channels will think as here chinese sheets. Minutes show lives. Various, high services could take forms; nasty photographs interpret policemen; nake|72.90|26.97|5002001|importoscholar #1|2|country|5|Music|125|antiableought|N/A|542333metallic886556|rose|Gram|Unknown|20|ablepripriantiought| +15333|AAAAAAAAEOLDAAAA|2000-10-27||Further big results shield. So new channels will think as here chinese sheets. Minutes show lives. Various, high services could take forms; nasty photographs interpret policemen; nake|0.84|26.97|5002001|exporticorp #6|2|gold|6|Jewelry|625|antiablecally|N/A|542333metallic886556|turquoise|Oz|Unknown|22|pripripriantiought| +15334|AAAAAAAAGOLDAAAA|1997-10-27|1999-10-27|Generally ideal lips must reach beautiful, top patterns. Disabled methods find commercial things. Less happy co|6.19|2.90|9004011|edu packmaxi #11|4|entertainments|9|Books|241|oughteseable|N/A|8303lime976936618185|olive|Lb|Unknown|8|esepripriantiought| +15335|AAAAAAAAGOLDAAAA|1999-10-28|2001-10-26|Generally ideal lips must reach beautiful, top patterns. Disabled methods find commercial things. Less happy co|3.57|2.03|9004011|edu packimporto #2|4|sports-apparel|2|Men|241|oughteseable|extra large|9purple4950342005534|seashell|Ounce|Unknown|3|antipripriantiought| +15336|AAAAAAAAGOLDAAAA|2001-10-27||True, wrong channels find as limited women; very special recommendations can contr|2.02|2.03|10016003|corpamalgamalg #3|4|wireless|10|Electronics|241|oughteseable|N/A|4715213492864135sky0|seashell|Tsp|Unknown|24|callypripriantiought| +15337|AAAAAAAAJOLDAAAA|1997-10-27||Now famous results would try too no longer sacred tons. Expectations could communicate |3.72|2.30|5002002|importoscholar #2|2|country|5|Music|245|antieseable|N/A|922744595ghost912183|thistle|N/A|Unknown|8|ationpripriantiought| +15338|AAAAAAAAKOLDAAAA|1997-10-27|2000-10-26|Peasants would blow almost relations; old clothes will not see ingredients. Effective questions cause|5.12|1.94|2003001|exportiimporto #1|3|pants|2|Men|365|anticallypri|petite|spring95624947660072|midnight|Tbl|Unknown|57|eingpripriantiought| +15339|AAAAAAAAKOLDAAAA|2000-10-27||Peasants would blow almost relations; old clothes will not see ingredients. Effective questions cause|4.06|1.94|9011004|amalgunivamalg #4|3|cooking|9|Books|115|anticallypri|N/A|spring95624947660072|red|Ounce|Unknown|19|n stpripriantiought| +15340|AAAAAAAAMOLDAAAA|1997-10-27|1999-10-27|As fu|5.16|2.16|6016005|corpbrand #5|16|consignment|6|Jewelry|184|eseeingought|N/A|cyan9094507599938372|royal|Gross|Unknown|28|baresepriantiought| +15341|AAAAAAAAMOLDAAAA|1999-10-28|2001-10-26|Asleep results must not make now. Allowances ask benefits. Dead years must not revert too animals. Later comprehensive causes may not campaign in part international units; fo|4.39|2.54|6016005|edu packimporto #2|4|sports-apparel|2|Men|184|eseeingought|medium|67287gainsboro021798|pale|Gross|Unknown|34|oughtesepriantiought| +15342|AAAAAAAAMOLDAAAA|2001-10-27||Asleep results must not make now. Allowances ask benefits. Dead years must not revert too animals. Later comprehensive causes may not campaign in part international units; fo|4.43|2.08|6016005|brandnameless #3|7|hockey|8|Sports|192|ablen stought|N/A|448898230645896pink8|powder|Pallet|Unknown|54|ableesepriantiought| +15343|AAAAAAAAPOLDAAAA|1997-10-27||Ago interesting clothes set only with the pressures. Deep|0.73|0.63|3004002|edu packexporti #2|4|school-uniforms|3|Children|28|eingable|extra large|127408burnished77180|saddle|Case|Unknown|63|priesepriantiought| +15344|AAAAAAAAAPLDAAAA|1997-10-27|2000-10-26|Times would miss low, national methods. Versions stick real partners; sports characterize spatial, upper grounds. Values might reveal togeth|1.46|0.56|7005005|scholarbrand #5|5|blinds/shades|7|Home|138|eingpriought|N/A|7040057tan0436291086|dodger|Ounce|Unknown|16|eseesepriantiought| +15345|AAAAAAAAAPLDAAAA|2000-10-27||Times would miss low, national methods. Versions stick real partners; sports characterize spatial, upper grounds. Values might reveal togeth|2.44|1.41|7005005|edu packmaxi #6|14|tennis|8|Sports|531|eingpriought|N/A|42456117708sienna049|steel|N/A|Unknown|22|antiesepriantiought| +15346|AAAAAAAACPLDAAAA|1997-10-27|1999-10-27|Grounds will find afield. Often actual benefits qualify effective, angry factors. Very rural police shall not tell together able lonely months. Efficient figures meet in a costs.|9.61|4.61|3003001|exportiexporti #1|3|toddlers|3|Children|621|oughtablecally|medium|397948664755papaya99|steel|Dram|Unknown|28|callyesepriantiought| +15347|AAAAAAAACPLDAAAA|1999-10-28|2001-10-26|Customers allow entirely imaginative gentlemen. Ago healthy terms move by the features; simple, small bodies live therefore at a services. Just|8.67|3.46|3003001|exportibrand #4|13|loose stones|6|Jewelry|621|oughtablecally|N/A|10953olive0475373586|rosy|Ounce|Unknown|51|ationesepriantiought| +15348|AAAAAAAACPLDAAAA|2001-10-27||Customers allow entirely imaginative gentlemen. Ago healthy terms move by the features; simple, small bodies live therefore at a services. Just|2.25|1.64|3003001|exportiunivamalg #15|3|dvd/vcr players|10|Electronics|811|oughtablecally|N/A|9142spring6046609766|burlywood|Cup|Unknown|21|eingesepriantiought| +15349|AAAAAAAAFPLDAAAA|1997-10-27||Unusual, victorian readers may open however tons. Worldwide special russians should get however items. Most divine flats |7.57|5.07|8004010|edu packnameless #10|4|camping|8|Sports|173|priationought|N/A|232729609171thistle4|ghost|Pallet|Unknown|29|n stesepriantiought| +15350|AAAAAAAAGPLDAAAA|1997-10-27|2000-10-26|Civil horses share. Authorities could see extremely only sexual hours. Far liberal ways would maintain allegations. Now tired classes could inve|4.39|2.37|1001001|amalgamalg #1|1|dresses|1|Women|397|ationn stpri|N/A|604688orange87416115|spring|Gross|Unknown|16|barantipriantiought| +15351|AAAAAAAAGPLDAAAA|2000-10-27||Civil horses share. Authorities could see extremely only sexual hours. Far liberal ways would maintain allegations. Now tired classes could inve|4.87|2.19|8006002|corpnameless #2|1|football|8|Sports|854|ationn stpri|N/A|604688orange87416115|slate|Bunch|Unknown|23|oughtantipriantiought| +15352|AAAAAAAAIPLDAAAA|1997-10-27|1999-10-27|New years wish also confident, unaware contents. Sound doubts will check right. Economic, potential eyes can say this welco|1.80|0.82|7015003|scholarnameless #3|15|tables|7|Home|330|barpripri|N/A|085440123spring64132|thistle|Box|Unknown|79|ableantipriantiought| +15353|AAAAAAAAIPLDAAAA|1999-10-28|2001-10-26|Desperately cultural products dine sole, long children. Possibl|4.86|2.72|7015003|scholarmaxi #6|15|history|9|Books|277|ationationable|N/A|42559286salmon381991|powder|Dozen|Unknown|45|priantipriantiought| +15354|AAAAAAAAIPLDAAAA|2001-10-27||Always surprising penalties haul funny, multiple sources. Comfortable, hot houses might take more else |2.60|1.06|7015003|edu packnameless #5|14|glassware|7|Home|277|ationationable|N/A|42559286salmon381991|snow|Ton|Unknown|1|eseantipriantiought| +15355|AAAAAAAALPLDAAAA|1997-10-27||Quite annual missiles refute later years; as dead materials include smoothly examples. Major, independent standards could not mean extra, young points. Different coloni|3.06|2.57|7002006|importobrand #6|2|bedding|7|Home|986|callyeingn st|N/A|peach191835968895349|honeydew|Pallet|Unknown|41|antiantipriantiought| +15356|AAAAAAAAMPLDAAAA|1997-10-27|2000-10-26|Potential, final reasons win times; nearly attractive reports should meet sizes. Sufficient, main voluntee|2.55|2.09|1004001|edu packamalg #1|4|swimwear|1|Women|675|antiationcally|petite|114ivory673504739877|snow|Ton|Unknown|8|callyantipriantiought| +15357|AAAAAAAAMPLDAAAA|2000-10-27||Tragic, scientific trustees will not like more in a positions. Events contribute home men. Fast, general minutes may return individual, |9.12|2.09|3001002|amalgexporti #2|1|newborn|3|Children|587|antiationcally|large|114ivory673504739877|ivory|Carton|Unknown|94|ationantipriantiought| +15358|AAAAAAAAOPLDAAAA|1997-10-27|1999-10-27|Fast, international obligations used to answer and so on metres. Great, other songs mi|9.83|3.83|8009009|maxinameless #9|9|optics|8|Sports|97|ationn st|N/A|3126373564019salmon9|smoke|Lb|Unknown|12|eingantipriantiought| +15359|AAAAAAAAOPLDAAAA|1999-10-28|2001-10-26|Historical, sweet tasks will move so sons. Eyes must not hide superior, anonymous texts. Huge, royal terms start much massive powers. Local countries can focus. M|0.33|3.83|8009009|importoscholar #2|9|country|5|Music|97|ationn st|N/A|3126373564019salmon9|yellow|Pallet|Unknown|65|n stantipriantiought| +15360|AAAAAAAAOPLDAAAA|2001-10-27||Little girls feel now. Common elements come again. Only minutes light police; realistic effects start central, social meetings. Other, possible newspapers reach at the peasants. Gastric, fun w|0.85|3.83|8009009|importoamalg #1|2|fragrances|1|Women|837|ationn st|small|7146503366390tomato3|tomato|Bundle|Unknown|46|barcallypriantiought| +15361|AAAAAAAABAMDAAAA|1997-10-27||Extra respondents would improve encoura|0.63|0.23|4001002|amalgedu pack #2|1|womens|4|Shoes|507|ationbaranti|petite|15126377170583slate8|peru|Each|Unknown|45|oughtcallypriantiought| +15362|AAAAAAAACAMDAAAA|1997-10-27|2000-10-26|Good bones vary then extremely rough standards. Appointments go more to a nurses. Good falls provide soon. Available, profound teachers shall no|4.54|3.85|5002001|importoscholar #1|2|country|5|Music|222|ableableable|N/A|5030thistle028558949|purple|Dram|Unknown|3|ablecallypriantiought| +15363|AAAAAAAACAMDAAAA|2000-10-27||Good bones vary then extremely rough standards. Appointments go more to a nurses. Good falls provide soon. Available, profound teachers shall no|0.57|0.36|5002001|amalgbrand #4|1|bathroom|7|Home|222|ableableable|N/A|206169933yellow96648|slate|Dram|Unknown|18|pricallypriantiought| +15364|AAAAAAAAEAMDAAAA|1997-10-27|1999-10-27|Products wish only stones. Miles must hear cases. Again additional followers must not get else |66.19|33.09|4002001|importoedu pack #1|2|mens|4|Shoes|92|ablen st|medium|20sienna220250963015|pale|Lb|Unknown|50|esecallypriantiought| +15365|AAAAAAAAEAMDAAAA|1999-10-28|2001-10-26|Products wish only stones. Miles must hear cases. Again additional followers must not get else |0.61|33.09|4002001|maximaxi #6|9|science|9|Books|428|ablen st|N/A|42794slate3772912932|peru|Bundle|Unknown|2|anticallypriantiought| +15366|AAAAAAAAEAMDAAAA|2001-10-27||Products wish only stones. Miles must hear cases. Again additional followers must not get else |9.94|33.09|4002001|amalgamalgamalg #16|9|disk drives|10|Electronics|516|callyoughtanti|N/A|42794slate3772912932|violet|Dozen|Unknown|64|callycallypriantiought| +15367|AAAAAAAAHAMDAAAA|1997-10-27||Easy able ends can wear secondly old pp.. Level products come positive arms. Very opposite parties ought to celebrate purely so as correct regulations. Local times sit then new|0.19|0.06|1003002|exportiamalg #2|3|maternity|1|Women|293|prin stable|medium|138575184822yellow72|ghost|Box|Unknown|2|ationcallypriantiought| +15368|AAAAAAAAIAMDAAAA|1997-10-27|2000-10-26|In order ope|4.75|2.47|10011001|amalgamalgamalg #1|11|disk drives|10|Electronics|142|ableeseought|N/A|516989844peach372383|pale|Pallet|Unknown|12|eingcallypriantiought| +15369|AAAAAAAAIAMDAAAA|2000-10-27||Strategic activities used to restore ever. Other, welsh relationships could shut consistent charts. Even ill industries support of course happy techniques. So new boy|1.10|2.47|10011001|amalgexporti #2|1|newborn|3|Children|458|ableeseought|small|65978spring806134264|sky|Ton|Unknown|44|n stcallypriantiought| +15370|AAAAAAAAKAMDAAAA|1997-10-27|1999-10-27|Free, inc tonnes shall need still possible proportions. Artists go still global dir|2.44|0.95|6013003|exportibrand #3|13|loose stones|6|Jewelry|235|antipriable|N/A|1019seashell56264579|moccasin|Tsp|Unknown|64|barationpriantiought| +15371|AAAAAAAAKAMDAAAA|1999-10-28|2001-10-26|Overall, key opinions mu|3.97|0.95|6013003|exportiedu pack #2|13|kids|4|Shoes|235|antipriable|extra large|1019seashell56264579|sienna|Ton|Unknown|26|oughtationpriantiought| +15372|AAAAAAAAKAMDAAAA|2001-10-27||Overall, key opinions mu|36.18|0.95|4004001|edu packedu pack #1|13|athletic|4|Shoes|117|antipriable|petite|6657065072powder7945|powder|Ton|Unknown|1|ableationpriantiought| +15373|AAAAAAAANAMDAAAA|1997-10-27||Average services could try unfortunately plants; extensive procedures must|4.94|3.26|9015002|scholarunivamalg #2|15|fiction|9|Books|531|oughtprianti|N/A|4447091928snow067713|pink|Bundle|Unknown|98|priationpriantiought| +15374|AAAAAAAAOAMDAAAA|1997-10-27|2000-10-26|Somehow current beds relish cheeks; social, anxious girls can balance events. Really tropical decisions write then available eyes. Finally small refugees drink|8.32|4.57|2004001|edu packimporto #1|4|sports-apparel|2|Men|478|eingationese|small|485811tan41421614793|papaya|Cup|Unknown|54|eseationpriantiought| +15375|AAAAAAAAOAMDAAAA|2000-10-27||Patients should achieve courses. Common changes get later main, serious factors; possible, fine homes like f|37.38|16.07|3001002|amalgexporti #2|1|newborn|3|Children|8|eingationese|large|485811tan41421614793|lace|Cup|Unknown|89|antiationpriantiought| +15376|AAAAAAAAABMDAAAA|1997-10-27|1999-10-27|Years might not arrive available years; prime studies might show only, different laws. Weeks should review particularly men. Available, afraid operations obtain later free, cr|1.51|0.52|8003007|exportinameless #7|3|basketball|8|Sports|115|antioughtought|N/A|5midnight08046585098|thistle|Box|Unknown|82|callyationpriantiought| +15377|AAAAAAAAABMDAAAA|1999-10-28|2001-10-26|Deeply special ho|1.54|0.70|8003007|edu packnameless #8|3|camping|8|Sports|412|ableoughtese|N/A|06978382894misty9798|wheat|Bundle|Unknown|1|ationationpriantiought| +15378|AAAAAAAAABMDAAAA|2001-10-27||Deeply special ho|2.69|0.70|8003007|brandnameless #9|3|hockey|8|Sports|35|ableoughtese|N/A|86447058302rose55182|plum|Case|Unknown|42|eingationpriantiought| +15379|AAAAAAAADBMDAAAA|1997-10-27||Abilities approach. Affairs study now remarkable funds. Policies exercise considerable, recent |4.57|2.33|5003002|exportischolar #2|3|pop|5|Music|143|prieseought|N/A|smoke628596531001917|puff|Carton|Unknown|35|n stationpriantiought| +15380|AAAAAAAAEBMDAAAA|1997-10-27|2000-10-26|Practical passages might not form nevertheless simply unable incentives. Social, significant audiences last still away apparent countries. Centres catch often around a suggestions. Profess|0.43|0.36|4001001|amalgedu pack #1|1|womens|4|Shoes|113|prioughtought|extra large|pink2478713288077027|slate|N/A|Unknown|27|bareingpriantiought| +15381|AAAAAAAAEBMDAAAA|2000-10-27||Practical passages might not form nevertheless simply unable incentives. Social, significant audiences last still away apparent countries. Centres catch often around a suggestions. Profess|1.78|1.37|4001001|edu packscholar #2|1|classical|5|Music|201|prioughtought|N/A|85385347982633lemon0|wheat|Each|Unknown|94|oughteingpriantiought| +15382|AAAAAAAAGBMDAAAA|1997-10-27|1999-10-27|Fine feelings may recall; there damp memories know even incomes. Judicial, new others mitigate below too alone emotions. Early, liberal children cannot let in the writers. Available|9.50|5.51|4001001|amalgedu pack #1|1|womens|4|Shoes|125|antiableought|economy|powder43296827510294|rosy|Dram|Unknown|69|ableeingpriantiought| +15383|AAAAAAAAGBMDAAAA|1999-10-28|2001-10-26|Fine feelings may recall; there damp memories know even incomes. Judicial, new others mitigate below too alone emotions. Early, liberal children cannot let in the writers. Available|1.45|0.98|4002002|importoedu pack #2|1|mens|4|Shoes|125|antiableought|medium|purple99389634004701|royal|N/A|Unknown|3|prieingpriantiought| +15384|AAAAAAAAGBMDAAAA|2001-10-27||Foreign links would not reduce easier. Earlier wide expenses would elicit more than b|3.90|0.98|4002002|scholarmaxi #5|1|fishing|8|Sports|125|antiableought|N/A|purple99389634004701|purple|Tsp|Unknown|59|eseeingpriantiought| +15385|AAAAAAAAJBMDAAAA|1997-10-27||Presumably social provisions see photographs; other prices might not decide unduly european, unusual levels. Illegal, military men shall choose here high birds. Key drawi|3.35|1.13|9009002|maximaxi #2|9|science|9|Books|102|ablebarought|N/A|6633982714710light26|royal|Tsp|Unknown|39|antieingpriantiought| +15386|AAAAAAAAKBMDAAAA|1997-10-27|2000-10-26|Much necessary buildings will |4.15|1.86|4001001|amalgedu pack #1|1|womens|4|Shoes|283|prieingable|medium|219611yellow42602752|pink|Gram|Unknown|10|callyeingpriantiought| +15387|AAAAAAAAKBMDAAAA|2000-10-27||Much necessary buildings will |7.30|5.69|7002008|importobrand #8|1|bedding|7|Home|127|ationableought|N/A|219611yellow42602752|almond|Tsp|Unknown|81|ationeingpriantiought| +15388|AAAAAAAAMBMDAAAA|1997-10-27|1999-10-27|Diverse, content sorts could not supplement practices. Unlikely criteria keep there labour, n|1.12|0.89|10010004|univamalgamalg #4|10|memory|10|Electronics|156|callyantiought|N/A|9smoke21306263058924|plum|Pound|Unknown|13|eingeingpriantiought| +15389|AAAAAAAAMBMDAAAA|1999-10-28|2001-10-26|All practical players would characterise just games. Successive, able bodies reflect best children. Hopes think now especially popular aspec|6.12|0.89|3003002|exportiexporti #2|3|toddlers|3|Children|483|prieingese|medium|9smoke21306263058924|smoke|Tbl|Unknown|30|n steingpriantiought| +15390|AAAAAAAAMBMDAAAA|2001-10-27||All practical players would characterise just games. Successive, able bodies reflect best children. Hopes think now especially popular aspec|6.33|2.02|4003001|exportiedu pack #1|3|kids|4|Shoes|483|prieingese|economy|9smoke21306263058924|blue|N/A|Unknown|45|barn stpriantiought| +15391|AAAAAAAAPBMDAAAA|1997-10-27||Superior contributions speed. Areas should en|95.22|35.23|8014002|edu packmaxi #2|14|tennis|8|Sports|257|ationantiable|N/A|0811389turquoise5717|red|Oz|Unknown|7|oughtn stpriantiought| +15392|AAAAAAAAACMDAAAA|1997-10-27|2000-10-26|Classical goods involve long, blank interpretations. Committees cause immensely trials. Passive ways can demonstrate mainly very clean years. Vast, other agreements prove good expensive, hard |1.61|1.33|4002001|importoedu pack #1|2|mens|4|Shoes|426|callyableese|petite|9033479blush86664586|white|Pound|Unknown|86|ablen stpriantiought| +15393|AAAAAAAAACMDAAAA|2000-10-27||Classical goods involve long, blank interpretations. Committees cause immensely trials. Passive ways can demonstrate mainly very clean years. Vast, other agreements prove good expensive, hard |4.44|1.33|1004002|edu packamalg #2|4|swimwear|1|Women|426|callyableese|petite|9033479blush86664586|orange|Case|Unknown|29|prin stpriantiought| +15394|AAAAAAAACCMDAAAA|1997-10-27|1999-10-27|Realistic communities know times. Soft days might not stop rights. General g|2.83|1.01|7008005|namelessbrand #5|8|lighting|7|Home|866|callycallyeing|N/A|80657911359white1748|green|Pallet|Unknown|24|esen stpriantiought| +15395|AAAAAAAACCMDAAAA|1999-10-28|2001-10-26|Realistic communities know times. Soft days might not stop rights. General g|1.18|1.06|7008005|exportiamalg #2|3|maternity|1|Women|866|callycallyeing|large|981navajo09723609426|steel|Ounce|Unknown|18|antin stpriantiought| +15396|AAAAAAAACCMDAAAA|2001-10-27||Substantial actions read extensive terms. Plants shall star|3.21|2.18|2003001|exportiimporto #1|3|pants|2|Men|866|callycallyeing|medium|981navajo09723609426|indian|Bunch|Unknown|3|callyn stpriantiought| +15397|AAAAAAAAFCMDAAAA|1997-10-27||Ladies share available, sorry hours. National eyes evolve eggs. Simply great thoughts could form scientists. Interesting names |2.65|1.69|2003002|exportiimporto #2|3|pants|2|Men|303|pribarpri|large|830black626021598915|puff|Unknown|Unknown|65|ationn stpriantiought| +15398|AAAAAAAAGCMDAAAA|1997-10-27|2000-10-26|Legal agencies oppose overwhelmingly full targets. Unlikely, open levels might expect young, responsible charges. Substantial, successful circumstances drown somewhat. Local m|3.69|1.21|8005009|scholarnameless #9|5|fitness|8|Sports|232|ablepriable|N/A|1magenta289465296835|misty|Case|Unknown|3|eingn stpriantiought| +15399|AAAAAAAAGCMDAAAA|2000-10-27||Average, pink stores would not |3.58|2.57|8005009|edu packscholar #2|4|classical|5|Music|232|ablepriable|N/A|64yellow469024192589|rosy|Carton|Unknown|19|n stn stpriantiought| +15400|AAAAAAAAICMDAAAA|1997-10-27|1999-10-27|Clear, sensitive customers must last from a cells. Slow potential children attend doctors. Ltd. eyebrows like white cases.|3.24|2.17|1004001|edu packamalg #1|4|swimwear|1|Women|158|eingantiought|petite|0181246070163snow859|seashell|Lb|Unknown|13|barbareseantiought| +15401|AAAAAAAAICMDAAAA|1999-10-28|2001-10-26|Also political forms find high periodically heavy waters; issues would remain in the germans. Reduced others help. Years might change single, royal years. Tin|1.91|0.95|1004001|edu packedu pack #2|4|athletic|4|Shoes|158|eingantiought|medium|6255677smoke83281138|turquoise|Unknown|Unknown|80|oughtbareseantiought| +15402|AAAAAAAAICMDAAAA|2001-10-27||Currently double rules ought to struggle insects. Valuable, urgent documents feel polite enemies. Large, urban group|7.80|0.95|1004001|importoscholar #1|4|country|5|Music|858|eingantiought|N/A|6255677smoke83281138|gainsboro|Lb|Unknown|43|ablebareseantiought| +15403|AAAAAAAALCMDAAAA|1997-10-27||Special, private situations shall ask l|4.31|1.93|6007004|brandcorp #4|7|pendants|6|Jewelry|545|antieseanti|N/A|8165984756009violet0|turquoise|Bunch|Unknown|59|pribareseantiought| +15404|AAAAAAAAMCMDAAAA|1997-10-27|2000-10-26|Liberal, coastal girls punch. Members worry entirely annually regional awards. Schools act also relationships. |8.46|5.83|10011003|amalgamalgamalg #3|11|disk drives|10|Electronics|640|baresecally|N/A|1290274thistle637021|smoke|Tsp|Unknown|24|esebareseantiought| +15405|AAAAAAAAMCMDAAAA|2000-10-27||New, racial communities must not make still past the parts. Legislative eyes must end creative, married nations. More constant patients may not look usually go|0.49|5.83|3003002|exportiexporti #2|3|toddlers|3|Children|640|baresecally|large|1290274thistle637021|peach|Carton|Unknown|33|antibareseantiought| +15406|AAAAAAAAOCMDAAAA|1997-10-27|1999-10-27|Issues will give. Even executive differences discover somewhere high, recent days. Doors may not save then members. |3.45|2.07|7011007|amalgnameless #7|11|accent|7|Home|62|ablecally|N/A|041seashell834536489|pale|Carton|Unknown|40|callybareseantiought| +15407|AAAAAAAAOCMDAAAA|1999-10-28|2001-10-26|Foreign, traditional countries will adapt linguistic, fixed prices. Little, imp|4.35|1.47|3002002|importoexporti #2|11|infants|3|Children|163|ablecally|petite|041seashell834536489|red|Box|Unknown|25|ationbareseantiought| +15408|AAAAAAAAOCMDAAAA|2001-10-27||Electoral, effective words could offer here legal, new cities. Public, odd cases see; large, popular attempts used to take details. Long lives would not form h|5.49|3.40|3002002|edu packmaxi #9|14|tennis|8|Sports|619|ablecally|N/A|041seashell834536489|navajo|N/A|Unknown|15|eingbareseantiought| +15409|AAAAAAAABDMDAAAA|1997-10-27||Else english events used to understand f|2.61|1.09|3001002|amalgexporti #2|1|newborn|3|Children|158|eingantiought|large|75875331slate5599099|wheat|Ounce|Unknown|61|n stbareseantiought| +15410|AAAAAAAACDMDAAAA|1997-10-27|2000-10-26|Useful, top needs will invite to a societies. However |1.82|0.78|7012001|importonameless #1|12|paint|7|Home|279|n stationable|N/A|298734508064352snow0|pink|N/A|Unknown|33|baroughteseantiought| +15411|AAAAAAAACDMDAAAA|2000-10-27||Useful, top needs will invite to a societies. However |2.95|0.94|7012001|scholarnameless #2|12|tables|7|Home|279|n stationable|N/A|702584tomato23693228|puff|Case|Unknown|49|oughtoughteseantiought| +15412|AAAAAAAAEDMDAAAA|1997-10-27|1999-10-27|Then happy bars will know largely to a personnel. Just good reasons would hear bills; internation|3.55|1.52|8003003|exportinameless #3|3|basketball|8|Sports|282|ableeingable|N/A|5400puff098716381221|tomato|Cup|Unknown|93|ableoughteseantiought| +15413|AAAAAAAAEDMDAAAA|1999-10-28|2001-10-26|Then happy bars will know largely to a personnel. Just good reasons would hear bills; internation|8.04|5.46|3002002|importoexporti #2|2|infants|3|Children|459|ableeingable|petite|5400puff098716381221|gainsboro|Case|Unknown|1|prioughteseantiought| +15414|AAAAAAAAEDMDAAAA|2001-10-27||Then happy bars will know largely to a personnel. Just good reasons would hear bills; internation|62.30|18.69|3002002|brandnameless #5|2|hockey|8|Sports|459|n stantiese|N/A|5400puff098716381221|turquoise|Pallet|Unknown|18|eseoughteseantiought| +15415|AAAAAAAAHDMDAAAA|1997-10-27||Cultural, monetary assets might not feel faintly babies. Proposed weeks know. Southern procedures become there men. Types make most. Absolute years illustrate priorities|2.68|1.98|3002002|importoexporti #2|2|infants|3|Children|244|eseeseable|large|499787732red04890629|steel|Case|Unknown|32|antioughteseantiought| +15416|AAAAAAAAIDMDAAAA|1997-10-27|2000-10-26|Enterprises ought to spend further during the men. General, numerous loans produce just then foreign|8.48|3.81|10001015|amalgunivamalg #15|1|cameras|10|Electronics|325|antiablepri|N/A|242rose3991710442077|pink|Tsp|Unknown|20|callyoughteseantiought| +15417|AAAAAAAAIDMDAAAA|2000-10-27||Up to back mice fancy lines; early, legitimate conditions get then now legal members. Right true pages need fatal duties. Other, primary artists used to tak|2.19|3.81|10001015|amalgscholar #2|1|rock|5|Music|325|antiablepri|N/A|64530095075orange747|seashell|Gross|Unknown|26|ationoughteseantiought| +15418|AAAAAAAAKDMDAAAA|1997-10-27|1999-10-27|Small birds draw possibly clubs. Catholic directions may |6.92|4.22|1004001|edu packamalg #1|4|swimwear|1|Women|11|oughtought|medium|031steel539594473424|yellow|Carton|Unknown|20|eingoughteseantiought| +15419|AAAAAAAAKDMDAAAA|1999-10-28|2001-10-26|Finally public wages know naturally. Long advances try out of the groups; nice contents would carry both nuclear quantities; names will know children. Remarkable classes come|0.84|4.22|3003002|exportiexporti #2|3|toddlers|3|Children|208|oughtought|extra large|48795709powder032631|royal|Dozen|Unknown|20|n stoughteseantiought| +15420|AAAAAAAAKDMDAAAA|2001-10-27||Simply personal choices attend particularly elderly offices. Otherwise single years understand. Modern, right parents pr|9.79|4.22|5004001|edu packscholar #1|4|classical|5|Music|208|oughtought|N/A|8yellow3139778331250|seashell|Bundle|Unknown|41|barableeseantiought| +15421|AAAAAAAANDMDAAAA|1997-10-27||Well existing buyers used to tell numbers. British, wide police leave at first final, impossible subjects. Different issues might not feel so to a arms. Afterwards broad specimens could keep f|4.91|1.47|3004002|edu packexporti #2|4|school-uniforms|3|Children|695|antin stcally|medium|99649124455peru84451|tan|Bundle|Unknown|11|oughtableeseantiought| +15422|AAAAAAAAODMDAAAA|1997-10-27|2000-10-26|Evidently critical streets could interpret in a houses. Local decades read to a meals. Common, desperate officials divert before economic buildings. Surprised, exciting mon|3.63|2.03|3001001|amalgexporti #1|1|newborn|3|Children|654|eseantically|medium|5256puff364524260923|snow|Gross|Unknown|14|ableableeseantiought| +15423|AAAAAAAAODMDAAAA|2000-10-27||Evidently critical streets could interpret in a houses. Local decades read to a meals. Common, desperate officials divert before economic buildings. Surprised, exciting mon|15.90|6.20|4004002|edu packedu pack #2|4|athletic|4|Shoes|162|ablecallyought|small|592885plum3849274694|peach|Ounce|Unknown|50|priableeseantiought| +15424|AAAAAAAAAEMDAAAA|1997-10-27|1999-10-27|Half significant teachers can come surely|0.48|0.15|4001001|amalgedu pack #1|1|womens|4|Shoes|228|eingableable|small|0562695953peach73846|red|Dozen|Unknown|27|eseableeseantiought| +15425|AAAAAAAAAEMDAAAA|1999-10-28|2001-10-26|National movements would avail dull, british bodi|2.80|1.68|4001001|amalgamalg #2|1|dresses|1|Women|228|eingableable|N/A|219954yellow57678611|sky|Box|Unknown|63|antiableeseantiought| +15426|AAAAAAAAAEMDAAAA|2001-10-27||National movements would avail dull, british bodi|1.57|1.00|4001001|amalgunivamalg #8|1|cameras|10|Electronics|228|eingableable|N/A|219954yellow57678611|navajo|Gram|Unknown|26|callyableeseantiought| +15427|AAAAAAAADEMDAAAA|1997-10-27||Brown, i|17.06|10.23|3003002|exportiexporti #2|3|toddlers|3|Children|680|bareingcally|extra large|5008244299691slate55|red|Cup|Unknown|1|ationableeseantiought| +15428|AAAAAAAAEEMDAAAA|1997-10-27|2000-10-26|Operators may take running days. Minimal things used to take without the sectors. Funny, high clients wait with a employees. Important, quick workers should save as. Chief sources woul|3.19|1.14|4002001|importoedu pack #1|2|mens|4|Shoes|604|esebarcally|large|789726peru9105317830|pink|Pound|Unknown|17|eingableeseantiought| +15429|AAAAAAAAEEMDAAAA|2000-10-27||Important cars talk within a levels. Years change tight. Thick, nice numbers decide here at all other animals. Light women cou|4.67|1.49|4002001|edu packamalgamalg #11|14|automotive|10|Electronics|604|esebarcally|N/A|789726peru9105317830|tomato|N/A|Unknown|64|n stableeseantiought| +15430|AAAAAAAAGEMDAAAA|1997-10-27|1999-10-27|Forward important pro|1.91|1.48|5002001|importoscholar #1|2|country|5|Music|33|pripri|N/A|87060699rose20516193|snow|Lb|Unknown|89|barprieseantiought| +15431|AAAAAAAAGEMDAAAA|1999-10-28|2001-10-26|Social sheets may explore marvellous, o|6.59|1.48|4004002|edu packedu pack #2|4|athletic|4|Shoes|68|pripri|large|447338325025275pale3|thistle|Bundle|Unknown|99|oughtprieseantiought| +15432|AAAAAAAAGEMDAAAA|2001-10-27||Single characteristics would not say much independently commercial women. Improved markets may not describe more social elements. Mere, new variations may lay. Figures could |9.77|1.48|4004002|corpunivamalg #1|4|mystery|9|Books|68|eingcally|N/A|034lime3776486802862|olive|Box|Unknown|50|ableprieseantiought| +15433|AAAAAAAAJEMDAAAA|1997-10-27||Of course just windows apply for a explanations; much new names used to show proteins. Young, neighbouring years drive quite in the dogs. Forms might not think most in a windows. Legs may|4.93|2.61|5004002|edu packscholar #2|4|classical|5|Music|306|callybarpri|N/A|0077019smoke57680607|purple|Each|Unknown|7|priprieseantiought| +15434|AAAAAAAAKEMDAAAA|1997-10-27|2000-10-26|Parliamentary expectations would not result too deliberately serious problems. Issues deliver more|7.10|5.18|4001001|amalgedu pack #1|1|womens|4|Shoes|590|barn stanti|small|0992papaya8768738771|khaki|Tsp|Unknown|7|eseprieseantiought| +15435|AAAAAAAAKEMDAAAA|2000-10-27||Industrial, golden horses must know then deep main animals. Political copies take so in an prices. Eventually decisive pupils know as lucky mistakes. Other degrees visit more actions; as bl|5.15|5.18|4001001|maxinameless #6|1|optics|8|Sports|395|barn stanti|N/A|0992papaya8768738771|violet|Dram|Unknown|60|antiprieseantiought| +15436|AAAAAAAAMEMDAAAA|1997-10-27|1999-10-27|Free, closed forms shall not accept strictly members. Private, pre|7.20|3.67|4003001|exportiedu pack #1|3|kids|4|Shoes|264|esecallyable|economy|60764986salmon010047|brown|Each|Unknown|57|callyprieseantiought| +15437|AAAAAAAAMEMDAAAA|1999-10-28|2001-10-26|Free, closed forms shall not accept strictly members. Private, pre|5.51|4.07|9013006|exportiunivamalg #6|3|self-help|9|Books|264|esecallyable|N/A|60764986salmon010047|tan|N/A|Unknown|12|ationprieseantiought| +15438|AAAAAAAAMEMDAAAA|2001-10-27||Increases communicate urban, great teeth. Social chiefs will come generally different fields. Separate numbers must see certainly civil objectives. Full games can feel to|5.84|4.84|9013006|edu packexporti #1|4|school-uniforms|3|Children|264|esecallyable|small|60764986salmon010047|sienna|Dram|Unknown|24|eingprieseantiought| +15439|AAAAAAAAPEMDAAAA|1997-10-27||Also black patterns may call other others. Pressures must come so; there young relations can want towards a galleries; new, left services at|8.37|7.28|7004010|edu packbrand #10|4|curtains/drapes|7|Home|968|eingcallyn st|N/A|30rosy35449773111527|violet|Oz|Unknown|57|n stprieseantiought| +15440|AAAAAAAAAFMDAAAA|1997-10-27|2000-10-26|Empirical, light parties conceive stores. Silent, good points care. Other, basic historians should arouse entirely so main events. Hands ought to attract just |9.43|8.48|5003001|exportischolar #1|3|pop|5|Music|112|ableoughtought|N/A|422575452883white952|maroon|Tsp|Unknown|54|bareseeseantiought| +15441|AAAAAAAAAFMDAAAA|2000-10-27||Real,|2.39|1.86|3003002|exportiexporti #2|3|toddlers|3|Children|213|ableoughtought|medium|422575452883white952|violet|Box|Unknown|9|oughteseeseantiought| +15442|AAAAAAAACFMDAAAA|1997-10-27|1999-10-27|Benefits cannot happen so|6.21|4.53|4002001|importoedu pack #1|2|mens|4|Shoes|571|oughtationanti|medium|snow3728366151920222|honeydew|N/A|Unknown|19|ableeseeseantiought| +15443|AAAAAAAACFMDAAAA|1999-10-28|2001-10-26|Benefits cannot happen so|66.85|4.53|4002001|exportiimporto #2|3|pants|2|Men|543|prieseanti|extra large|snow3728366151920222|dodger|Cup|Unknown|2|prieseeseantiought| +15444|AAAAAAAACFMDAAAA|2001-10-27||Then new hills would create far inches. Permanent studies supply onwards. Feet will clarify now also numerous institutions. Difficult, cultural seeds might not take abroad isolate|53.72|16.65|4002001|corpbrand #5|3|rugs|7|Home|543|prieseanti|N/A|648331380699powder44|rosy|Gram|Unknown|26|eseeseeseantiought| +15445|AAAAAAAAFFMDAAAA|1997-10-27||Working, gold proteins lie wide possi|17.12|10.78|7014006|edu packnameless #6|14|glassware|7|Home|502|ablebaranti|N/A|24367314291moccasin1|maroon|Case|Unknown|48|antieseeseantiought| +15446|AAAAAAAAGFMDAAAA|1997-10-27|2000-10-26|Other years ought to draw earlier documents; very chief forms would leave. Implications consult daily teeth. Other concentrations appreciate voters. Merely recent authoriti|2.31|0.92|10003011|exportiunivamalg #11|3|dvd/vcr players|10|Electronics|610|baroughtcally|N/A|860974105midnight893|pink|Gram|Unknown|45|callyeseeseantiought| +15447|AAAAAAAAGFMDAAAA|2000-10-27||Normal, eastern tears join broad opportunities. Home fellow rates should need then in the women. Competitive, great |2.39|1.84|10003011|scholarnameless #8|15|tables|7|Home|206|callybarable|N/A|900268219smoke763470|papaya|Each|Unknown|55|ationeseeseantiought| +15448|AAAAAAAAIFMDAAAA|1997-10-27|1999-10-27|Considerable, long-term cases co|2.16|1.57|8008009|namelessnameless #9|8|outdoor|8|Sports|595|antin stanti|N/A|peru4406833032187937|spring|Oz|Unknown|34|eingeseeseantiought| +15449|AAAAAAAAIFMDAAAA|1999-10-28|2001-10-26|Considerable, long-term cases co|3.21|1.57|3002002|importoexporti #2|8|infants|3|Children|595|antin stanti|small|peru4406833032187937|red|Oz|Unknown|27|n steseeseantiought| +15450|AAAAAAAAIFMDAAAA|2001-10-27||Forever mediterranean features will|2.46|1.57|7008001|namelessbrand #1|8|lighting|7|Home|595|antin stanti|N/A|4075855105735rosy571|light|Pound|Unknown|2|barantieseantiought| +15451|AAAAAAAALFMDAAAA|1997-10-27||Conditions want well enormous, proper cells; claims ought to clear now to the times. As well divine surfaces know persistent, ha|74.70|47.06|8010002|univmaxi #2|10|pools|8|Sports|231|oughtpriable|N/A|4088804925837snow045|hot|Pound|Unknown|58|oughtantieseantiought| +15452|AAAAAAAAMFMDAAAA|1997-10-27|2000-10-26|Other periods emerge neither associations. Top, dark users could know very other companies; other flats should say more more british movies. Young, great books can make less certain women. Furth|9.52|4.37|1001001|amalgamalg #1|1|dresses|1|Women|559|n stantianti|small|yellow62824335281326|slate|Lb|Unknown|50|ableantieseantiought| +15453|AAAAAAAAMFMDAAAA|2000-10-27||Other periods emerge neither associations. Top, dark users could know very other companies; other flats should say more more british movies. Young, great books can make less certain women. Furth|3.52|4.37|1001001|exportibrand #4|3|kids|7|Home|559|n stantianti|N/A|yellow62824335281326|slate|Pound|Unknown|26|priantieseantiought| +15454|AAAAAAAAOFMDAAAA|1997-10-27|1999-10-27|Units might cover red jobs;|2.16|1.03|2002001|importoimporto #1|2|shirts|2|Men|283|prieingable|economy|54117934524661royal4|tomato|N/A|Unknown|16|eseantieseantiought| +15455|AAAAAAAAOFMDAAAA|1999-10-28|2001-10-26|Positive, senior teachers determine men; redundant, previous fortunes sustain twice still, black months. Main, sufficient expenses will cho|2.50|1.03|2002001|maxiunivamalg #3|2|televisions|10|Electronics|283|prieingable|N/A|218607480346green412|saddle|Each|Unknown|12|antiantieseantiought| +15456|AAAAAAAAOFMDAAAA|2001-10-27||Positive, senior teachers determine men; redundant, previous fortunes sustain twice still, black months. Main, sufficient expenses will cho|3.22|1.03|2002001|exportibrand #1|13|loose stones|6|Jewelry|422|prieingable|N/A|218607480346green412|sandy|Cup|Unknown|50|callyantieseantiought| +15457|AAAAAAAABGMDAAAA|||||||||entertainments|9|Books|216|callyoughtable|N/A||midnight|Dozen|Unknown||ationantieseantiought| +15458|AAAAAAAACGMDAAAA|1997-10-27|2000-10-26|Aware sentences used to find very by the months; difficulties bring finally. Years turn maybe shots. Apparent, bad lives try more. Physical, voluntary activ|6.55|3.86|9004003|edu packmaxi #3|4|entertainments|9|Books|791|oughtn station|N/A|39pink28499291139425|green|Gross|Unknown|9|eingantieseantiought| +15459|AAAAAAAACGMDAAAA|2000-10-27||Standard, other police would tell ahead; away lost stars think darkly to a stranger|0.54|0.23|9004003|amalgnameless #10|4|athletic shoes|8|Sports|237|oughtn station|N/A|337dodger71626350451|sandy|Ounce|Unknown|57|n stantieseantiought| +15460|AAAAAAAAEGMDAAAA|1997-10-27|1999-10-27|Instances will see during the resources. Available, simple tools occur full-time, recent standards; part|6.95|3.47|6014005|edu packbrand #5|14|estate|6|Jewelry|256|callyantiable|N/A|papaya86664441669882|powder|Case|Unknown|30|barcallyeseantiought| +15461|AAAAAAAAEGMDAAAA|1999-10-28|2001-10-26|Local, other achievements must stop. Hard, competitive owners would enable again teachers. Creative feet leave male |1.91|3.47|7003002|exportibrand #2|14|kids|7|Home|471|oughtationese|N/A|782189steel577986529|tomato|Pound|Unknown|4|oughtcallyeseantiought| +15462|AAAAAAAAEGMDAAAA|2001-10-27||Common, apparent reservations prove technically curtains. Later little features o|3.74|3.47|4004001|edu packedu pack #1|4|athletic|4|Shoes|48|oughtationese|medium|70509214brown3532273|thistle|Cup|Unknown|16|ablecallyeseantiought| +15463|AAAAAAAAHGMDAAAA|1997-10-27||New consultants must not provide in order ordinary boys; mothers throw only at a wishes. Procedures stay. Total, clear years used to mount both actually|3.50|1.96|4001002|amalgedu pack #2|1|womens|4|Shoes|866|callycallyeing|large|637884peru0540225365|royal|Gross|Unknown|14|pricallyeseantiought| +15464|AAAAAAAAIGMDAAAA|1997-10-27|2000-10-26|Soft, great arrangements simplify long male, black errors. Wonderful, personal men should get really to a homes. In order full groups p|0.92|0.28|10015012|scholaramalgamalg #12|15|portable|10|Electronics|307|ationbarpri|N/A|puff9759428284871241|powder|Carton|Unknown|33|esecallyeseantiought| +15465|AAAAAAAAIGMDAAAA|2000-10-27||Programmes could not understand still. Animals will facilitate today hard, we|4.75|3.61|10015012|exportiamalg #2|15|maternity|1|Women|307|ationbarpri|small|2438358sandy63880458|yellow|Box|Unknown|41|anticallyeseantiought| +15466|AAAAAAAAKGMDAAAA|1997-10-27|1999-10-27|Ages must enjoy soon new tools. Hands would identify closely capable|5.19|2.95|10007014|brandunivamalg #14|7|personal|10|Electronics|270|barationable|N/A|00571plum01484225249|sandy|Dozen|Unknown|9|callycallyeseantiought| +15467|AAAAAAAAKGMDAAAA|1999-10-28|2001-10-26|Ages must enjoy soon new tools. Hands would identify closely capable|3.08|2.95|4003002|exportiedu pack #2|7|kids|4|Shoes|270|barationable|small|2139939027263wheat28|orchid|Case|Unknown|65|ationcallyeseantiought| +15468|AAAAAAAAKGMDAAAA|2001-10-27||Communic|2.92|2.95|4003002|amalgamalgamalg #16|7|disk drives|10|Electronics|270|barationable|N/A|2139939027263wheat28|smoke|Dram|Unknown|27|eingcallyeseantiought| +15469|AAAAAAAANGMDAAAA|1997-10-27||Sound, similar components make wages. Rules might discover over|89.96|28.78|5004002|edu packscholar #2|4|classical|5|Music|33|pripri|N/A|5266549488593papaya7|violet|Pound|Unknown|47|n stcallyeseantiought| +15470|AAAAAAAAOGMDAAAA|1997-10-27|2000-10-26|Chemical, |8.98|4.31|4003001|exportiedu pack #1|3|kids|4|Shoes|181|oughteingought|small|pink9221226945639552|misty|Dozen|Unknown|46|barationeseantiought| +15471|AAAAAAAAOGMDAAAA|2000-10-27||Soviet, recent drugs shall see clear, natural students; necessary, moral weeks carry large, responsible days. General, decent number|4.28|4.31|2004002|edu packimporto #2|3|sports-apparel|2|Men|181|oughteingought|medium|822150lace4813510003|sky|Dram|Unknown|32|oughtationeseantiought| +15472|AAAAAAAAAHMDAAAA|1997-10-27|1999-10-27|Single, considerable reactions mature low; cases can reduce in a streets. Gre|0.13|0.10|4002001|importoedu pack #1|2|mens|4|Shoes|31|oughtpri|small|4433707smoke45458186|red|Lb|Unknown|64|ableationeseantiought| +15473|AAAAAAAAAHMDAAAA|1999-10-28|2001-10-26|Dogs worry still new, labour societies. Around low things would not cont|1.81|0.79|4002001|amalgamalg #2|1|dresses|1|Women|392|oughtpri|large|0129207894magenta051|khaki|Carton|Unknown|52|priationeseantiought| +15474|AAAAAAAAAHMDAAAA|2001-10-27||Vital games used to occur other councils. Logically new purposes ask more traditional categor|5.27|0.79|4002001|amalgamalg #1|1|dresses|1|Women|59|oughtpri|medium|0129207894magenta051|seashell|Pallet|Unknown|7|eseationeseantiought| +15475|AAAAAAAADHMDAAAA|1997-10-27||Issues control also with a patients. International, fundamental needs shall accept only i|4.99|3.49|6016004|corpbrand #4|16|consignment|6|Jewelry|68|eingcally|N/A|71peach2219164971011|smoke|Each|Unknown|62|antiationeseantiought| +15476|AAAAAAAAEHMDAAAA|1997-10-27|2000-10-26|Natural events would take. Modern, blue times lead on the requirements. Others supply near a places. Others arrive natural, new weaknesses. Later only chemicals|2.34|0.70|3001001|amalgexporti #1|1|newborn|3|Children|270|barationable|petite|2mint496755751928657|sienna|Case|Unknown|6|callyationeseantiought| +15477|AAAAAAAAEHMDAAAA|2000-10-27||Moral, full races would not come inevitably skills; much recent employees might come never clear, critical dealers. Divine, little things see. Acceptable bands address through|23.54|20.00|3001001|amalgedu pack #2|1|womens|4|Shoes|135|barationable|medium|2mint496755751928657|sky|Dozen|Unknown|26|ationationeseantiought| +15478|AAAAAAAAGHMDAAAA|1997-10-27|1999-10-27|Inner mothers think pupils. Vast things may |0.70|0.22|6005007|scholarcorp #7|5|earings|6|Jewelry|551|oughtantianti|N/A|2857566672165sky3377|rosy|Oz|Unknown|85|eingationeseantiought| +15479|AAAAAAAAGHMDAAAA|1999-10-28|2001-10-26|Extra, eager needs should leave positive, new knees. Senior, acute things appoint too very personal channels. Benefit|5.91|0.22|6005007|exportiedu pack #2|3|kids|4|Shoes|2|able|small|2857566672165sky3377|royal|Lb|Unknown|3|n stationeseantiought| +15480|AAAAAAAAGHMDAAAA|2001-10-27||Extra, eager needs should leave positive, new knees. Senior, acute things appoint too very personal channels. Benefit|4.97|2.43|9004001|edu packmaxi #1|3|entertainments|9|Books|853|able|N/A|2857566672165sky3377|royal|Ounce|Unknown|61|bareingeseantiought| +15481|AAAAAAAAJHMDAAAA|1997-10-27||Physical polls melt as eyes. Clear, special sources might invent at once. As immediate things will not|3.77|1.80|1004002|edu packamalg #2|4|swimwear|1|Women|445|antieseese|petite|791peru7885204860842|wheat|Box|Unknown|23|oughteingeseantiought| +15482|AAAAAAAAKHMDAAAA|1997-10-27|2000-10-26|Yet physical refugees could not ask. Also right days could depend annual responses. Enormous objects cannot|2.97|1.21|4004001|edu packedu pack #1|4|athletic|4|Shoes|571|oughtationanti|large|0khaki72009434137214|seashell|Pallet|Unknown|26|ableeingeseantiought| +15483|AAAAAAAAKHMDAAAA|2000-10-27||Complex, white measures must|9.30|1.21|4004001|amalgunivamalg #4|4|cooking|9|Books|138|eingpriought|N/A|7283073medium8835417|saddle|Bunch|Unknown|39|prieingeseantiought| +15484|AAAAAAAAMHMDAAAA|1997-10-27|1999-10-27|Risks must wonder bad, necessary shares. Availabl|4.20|3.78|6010005|univbrand #5|10|jewelry boxes|6|Jewelry|970|barationn st|N/A|287105linen190241243|orchid|Each|Unknown|6|eseeingeseantiought| +15485|AAAAAAAAMHMDAAAA|1999-10-28|2001-10-26|Risks must wonder bad, necessary shares. Availabl|5.83|3.78|6010005|amalgscholar #2|10|rock|5|Music|489|barationn st|N/A|77rose71315791513722|white|Gram|Unknown|25|antieingeseantiought| +15486|AAAAAAAAMHMDAAAA|2001-10-27||Other signals lead imaginative, adverse parts. Safe implications lie. General,|0.52|0.44|6006007|corpcorp #7|10|rings|6|Jewelry|63|prically|N/A|830562720peach207076|sky|Pound|Unknown|14|callyeingeseantiought| +15487|AAAAAAAAPHMDAAAA|1997-10-27||Small, true years will d|0.51|0.35|10003001|exportiunivamalg #1|3|dvd/vcr players|10|Electronics|645|antiesecally|N/A|9561844462papaya0748|sky|Case|Unknown|22|ationeingeseantiought| +15488|AAAAAAAAAIMDAAAA|1997-10-27|2000-10-26|Other gentlemen can concentrate. Public units intend. Schools get banks. Social, e|5.41|4.11|3001001|amalgexporti #1|1|newborn|3|Children|76|callyation|medium|58sky294449230500976|saddle|Gross|Unknown|17|eingeingeseantiought| +15489|AAAAAAAAAIMDAAAA|2000-10-27||Other gentlemen can concentrate. Public units intend. Schools get banks. Social, e|8.18|6.05|7014010|edu packnameless #10|14|glassware|7|Home|76|callyation|N/A|58sky294449230500976|powder|Unknown|Unknown|35|n steingeseantiought| +15490|AAAAAAAACIMDAAAA|1997-10-27|1999-10-27|Pink parts |9.36|7.58|8005001|scholarnameless #1|5|fitness|8|Sports|301|oughtbarpri|N/A|48842727149steel7287|sky|Gram|Unknown|16|barn steseantiought| +15491|AAAAAAAACIMDAAAA|1999-10-28|2001-10-26|Pink parts |4.32|7.58|8005001|univmaxi #2|10|pools|8|Sports|301|oughtbarpri|N/A|19996plum61408775838|yellow|Oz|Unknown|7|oughtn steseantiought| +15492|AAAAAAAACIMDAAAA|2001-10-27||Sorry, close |1.67|7.58|6015005|scholarbrand #5|10|custom|6|Jewelry|467|ationcallyese|N/A|046603736563328plum2|tomato|Box|Unknown|50|ablen steseantiought| +15493|AAAAAAAAFIMDAAAA|1997-10-27||Almost scientific parts construct inwards for the changes. Notions feel even complete sounds|3.25|2.89|3003002|exportiexporti #2|3|toddlers|3|Children|1000|barbarbarought|small|8001831634red5326043|coral|Pallet|Unknown|57|prin steseantiought| +15494|AAAAAAAAGIMDAAAA|1997-10-27|2000-10-26|Never precise needs meet never mothers. Po|1.34|0.46|8007005|brandnameless #5|7|hockey|8|Sports|406|callybarese|N/A|65686888853magenta18|lawn|Oz|Unknown|8|esen steseantiought| +15495|AAAAAAAAGIMDAAAA|2000-10-27||National, determined groups see at least men. Twins shall not think offices. Hence important things will not produce. Then following artists run rather|0.45|0.46|3002002|importoexporti #2|2|infants|3|Children|406|callybarese|small|pale4639802373394994|gainsboro|Gross|Unknown|10|antin steseantiought| +15496|AAAAAAAAIIMDAAAA|1997-10-27|1999-10-27|Contacts will want further local institutions. Democratic feet learn so poor officers. Thorough, regional materials should not like strongl|7.84|2.82|3001001|amalgexporti #1|1|newborn|3|Children|5|anti|petite|5203240maroon2313655|salmon|Case|Unknown|74|callyn steseantiought| +15497|AAAAAAAAIIMDAAAA|1999-10-28|2001-10-26|Black successes cure with a results. Typical, hot lakes ought to resist blue systems. Other students ought to address. Appropriate, polite connections allow then through the needs; effor|1.32|2.82|3001001|importoamalgamalg #11|12|monitors|10|Electronics|26|anti|N/A|5203240maroon2313655|spring|Dozen|Unknown|58|ationn steseantiought| +15498|AAAAAAAAIIMDAAAA|2001-10-27||Black successes cure with a results. Typical, hot lakes ought to resist blue systems. Other students ought to address. Appropriate, polite connections allow then through the needs; effor|5.00|4.30|3001001|exportiamalg #1|3|maternity|1|Women|26|callyable|large|34922246sky981753386|smoke|Ounce|Unknown|84|eingn steseantiought| +15499|AAAAAAAALIMDAAAA|1997-10-27||Affairs come average, commercial wives. Real minds must resign traditionally. Still combined foods give |4.50|2.16|6007004|brandcorp #4|7|pendants|6|Jewelry|890|barn steing|N/A|90981peach1376903509|salmon|N/A|Unknown|51|n stn steseantiought| +15500|AAAAAAAAMIMDAAAA|1997-10-27|2000-10-26|Appropriate, great hands refuse clearly double pools. Passages could not observe for example objects; immediately important days may dry immediate, military wounds. Her|7.00|5.18|4002001|importoedu pack #1|2|mens|4|Shoes|311|oughtoughtpri|large|2178527violet6831990|steel|Lb|Unknown|40|barbarantiantiought| +15501|AAAAAAAAMIMDAAAA|2000-10-27||Once written centuries can say processes. Concerned, other systems could like perfectly fully interesting families; minds should become |2.90|1.30|3001002|amalgexporti #2|2|newborn|3|Children|311|oughtoughtpri|medium|2178527violet6831990|orange|Gram|Unknown|18|oughtbarantiantiought| +15502|AAAAAAAAOIMDAAAA|1997-10-27|1999-10-27|Capable profits work. Large, powerful instruments shall insist officially too royal cases. Traditional days check sharp, small ministers. Old child|7.18|2.44|10014016|edu packamalgamalg #16|14|automotive|10|Electronics|164|esecallyought|N/A|53648780718pale80769|goldenrod|Ounce|Unknown|34|ablebarantiantiought| +15503|AAAAAAAAOIMDAAAA|1999-10-28|2001-10-26|Capable profits work. Large, powerful instruments shall insist officially too royal cases. Traditional days check sharp, small ministers. Old child|8.31|2.44|2003002|exportiimporto #2|3|pants|2|Men|17|esecallyought|large|53648780718pale80769|sienna|Dozen|Unknown|64|pribarantiantiought| +15504|AAAAAAAAOIMDAAAA|2001-10-27||For example new cases could matter statements; other, long-t|2.03|1.80|2004001|edu packimporto #1|3|sports-apparel|2|Men|18|eingought|extra large|53648780718pale80769|sienna|Tbl|Unknown|12|esebarantiantiought| +15505|AAAAAAAABJMDAAAA|1997-10-27||Productive, prime hours eat particularly royal, original courts. Assumptions cannot clarify; resources can get parts; friendly, bad no|47.52|25.66|10012002|importoamalgamalg #2|12|monitors|10|Electronics|179|n stationought|N/A|110steel280453163909|pink|Bundle|Unknown|24|antibarantiantiought| +15506|AAAAAAAACJMDAAAA|1997-10-27|2000-10-26|Obligations res|5.28|2.53|5003001|exportischolar #1|3|pop|5|Music|222|ableableable|N/A|9white49361624528407|red|Unknown|Unknown|34|callybarantiantiought| +15507|AAAAAAAACJMDAAAA|2000-10-27||Ways take as important pp.|7.86|3.30|9002004|importomaxi #4|3|business|9|Books|130|barpriought|N/A|148474rosy8223558206|beige|Bundle|Unknown|29|ationbarantiantiought| +15508|AAAAAAAAEJMDAAAA|1997-10-27|1999-10-27|Most special arrangements say sure just slow knees. International rules take separately empty problems. There existing hour|2.25|0.69|5002001|importoscholar #1|2|country|5|Music|461|oughtcallyese|N/A|84230298453yellow240|wheat|Case|Unknown|31|eingbarantiantiought| +15509|AAAAAAAAEJMDAAAA|1999-10-28|2001-10-26|Conditions may not come in a police. Individual paintings add enough wrong voters; f|7.62|0.69|5002001|edu packimporto #2|4|sports-apparel|2|Men|822|ableableeing|medium|84230298453yellow240|deep|Pound|Unknown|64|n stbarantiantiought| +15510|AAAAAAAAEJMDAAAA|2001-10-27||Conditions may not come in a police. Individual paintings add enough wrong voters; f|0.25|0.13|5002001|exportiamalg #1|4|maternity|1|Women|95|ableableeing|extra large|758650wheat827026803|violet|Tsp|Unknown|4|baroughtantiantiought| +15511|AAAAAAAAHJMDAAAA|1997-10-27||Words must long both near a|5.95|2.91|8007002|brandnameless #2|7|hockey|8|Sports|195|antin stought|N/A|2tomato7940462484140|forest|Ton|Unknown|22|oughtoughtantiantiought| +15512|AAAAAAAAIJMDAAAA|1997-10-27|2000-10-26|Growing, organic rights go now cases. Main windows shake there committees. Citizens love occasionally hot, common habits. Important, average miles help however flo|1.80|0.64|4003001|exportiedu pack #1|3|kids|4|Shoes|621|oughtablecally|extra large|989metallic592002445|royal|Case|Unknown|83|ableoughtantiantiought| +15513|AAAAAAAAIJMDAAAA|2000-10-27||Growing, organic rights go now cases. Main windows shake there committees. Citizens love occasionally hot, common habits. Important, average miles help however flo|0.90|0.64|1001002|amalgamalg #2|1|dresses|1|Women|127|oughtablecally|petite|8puff378386112608557|purple|Bundle|Unknown|55|prioughtantiantiought| +15514|AAAAAAAAKJMDAAAA|1997-10-27|1999-10-27|Misleading fe|5.39|3.12|2001001|amalgimporto #1|1|accessories|2|Men|535|antiprianti|petite|86077583rosy90933754|khaki|Dozen|Unknown|30|eseoughtantiantiought| +15515|AAAAAAAAKJMDAAAA|1999-10-28|2001-10-26|Hard, eastern wings could not feel with the officers|0.98|3.12|8009006|maxinameless #6|9|optics|8|Sports|253|antiprianti|N/A|0154265771tomato4489|plum|Oz|Unknown|42|antioughtantiantiought| +15516|AAAAAAAAKJMDAAAA|2001-10-27||Hard, eastern wings could not feel with the officers|3.88|1.47|8009006|corpamalgamalg #13|16|wireless|10|Electronics|253|priantiable|N/A|0154265771tomato4489|salmon|Pound|Unknown|25|callyoughtantiantiought| +15517|AAAAAAAANJMDAAAA|1997-10-27||Factories cause please public children. Dry cases will last. Religious, double names lo|43.53|34.38|5002002|importoscholar #2|2|country|5|Music|636|callyprically|N/A|284590523linen431847|wheat|Carton|Unknown|22|ationoughtantiantiought| +15518|AAAAAAAAOJMDAAAA|1997-10-27|2000-10-26|Too other events might not quote police. Social, technical|5.82|1.97|1004001|edu packamalg #1|4|swimwear|1|Women|207|ationbarable|large|4741navy888172010418|thistle|Unknown|Unknown|64|eingoughtantiantiought| +15519|AAAAAAAAOJMDAAAA|2000-10-27||Too other events might not quote police. Social, technical|8.18|1.97|1004001|edu packexporti #2|4|school-uniforms|3|Children|234|esepriable|large|4741navy888172010418|papaya|Ounce|Unknown|4|n stoughtantiantiought| +15520|AAAAAAAAAKMDAAAA|1997-10-27|1999-10-27|Sexual markets might not miss central plants. Physical relationships can leave probably p|2.87|1.72|7010001|univnameless #1|10|flatware|7|Home|121|oughtableought|N/A|6382098red3667250469|honeydew|Box|Unknown|34|barableantiantiought| +15521|AAAAAAAAAKMDAAAA|1999-10-28|2001-10-26|Huge, forthcoming aims block much with the issues. Forthcoming, a|2.74|2.38|7005002|scholarbrand #2|5|blinds/shades|7|Home|121|oughtableought|N/A|4red7145447416503555|violet|Gram|Unknown|30|oughtableantiantiought| +15522|AAAAAAAAAKMDAAAA|2001-10-27||Huge, forthcoming aims block much with the issues. Forthcoming, a|1.30|0.91|7005002|maxinameless #3|5|optics|8|Sports|100|oughtableought|N/A|4red7145447416503555|pale|Lb|Unknown|12|ableableantiantiought| +15523|AAAAAAAADKMDAAAA|1997-10-27||Waiting arts would|56.62|26.61|3001002|amalgexporti #2|1|newborn|3|Children|87|ationeing|large|chiffon7188410957646|lace|Gram|Unknown|30|priableantiantiought| +15524|AAAAAAAAEKMDAAAA|1997-10-27|2000-10-26|British, new poems gain however social deals. Performances take years. Years might not see less. Personnel used to recognize about with an payments; shoes may not let ski|0.30|0.15|1003001|exportiamalg #1|3|maternity|1|Women|472|ableationese|medium|15044493violet503951|saddle|Dozen|Unknown|15|eseableantiantiought| +15525|AAAAAAAAEKMDAAAA|2000-10-27||British, new poems gain however social deals. Performances take years. Years might not see less. Personnel used to recognize about with an payments; shoes may not let ski|1.78|0.15|3002002|importoexporti #2|3|infants|3|Children|472|ableationese|petite|misty459510612583998|violet|Ounce|Unknown|88|antiableantiantiought| +15526|AAAAAAAAGKMDAAAA|1997-10-27|1999-10-27|Now contemporary results exist for example by the pounds. Environmental terms would not arrange. Physical, likely voices will realis|2.24|0.85|10016006|corpamalgamalg #6|16|wireless|10|Electronics|157|ationantiought|N/A|7white47719650339395|rosy|N/A|Unknown|58|callyableantiantiought| +15527|AAAAAAAAGKMDAAAA|1999-10-28|2001-10-26|Now contemporary results exist for example by the pounds. Environmental terms would not arrange. Physical, likely voices will realis|9.37|5.62|8007008|brandnameless #8|16|hockey|8|Sports|157|ationantiought|N/A|7white47719650339395|plum|Box|Unknown|20|ationableantiantiought| +15528|AAAAAAAAGKMDAAAA|2001-10-27||Deep, outer skills would not save sides. Officials list now continuous, whole|41.48|20.74|7002009|importobrand #9|2|bedding|7|Home|841|ationantiought|N/A|smoke779335976789523|peru|Oz|Unknown|2|eingableantiantiought| +15529|AAAAAAAAJKMDAAAA|1997-10-27||Designs shall not deal. Ideal, alternative aims say further changes. Often contemporary techniques used t|1.92|0.74|9010002|univunivamalg #2|10|travel|9|Books|432|ablepriese|N/A|rose5558353121057667|wheat|Lb|Unknown|63|n stableantiantiought| +15530|AAAAAAAAKKMDAAAA|1997-10-27|2000-10-26|Co|2.71|1.02|7011001|amalgnameless #1|11|accent|7|Home|375|antiationpri|N/A|813821100yellow86704|pink|Tsp|Unknown|17|barpriantiantiought| +15531|AAAAAAAAKKMDAAAA|2000-10-27||Co|7.32|5.56|7011001|amalgnameless #2|11|accent|7|Home|375|antiationpri|N/A|6952672hot5595888332|smoke|Pound|Unknown|2|oughtpriantiantiought| +15532|AAAAAAAAMKMDAAAA|1997-10-27|1999-10-27|Children aid ever pictures. Abstract, ra|0.28|0.20|9010005|univunivamalg #5|10|travel|9|Books|558|eingantianti|N/A|446098273salmon87734|blanched|Dozen|Unknown|66|ablepriantiantiought| +15533|AAAAAAAAMKMDAAAA|1999-10-28|2001-10-26|Children aid ever pictures. Abstract, ra|0.09|0.20|10015013|scholaramalgamalg #13|15|portable|10|Electronics|215|antioughtable|N/A|857037271snow9006958|ivory|Bunch|Unknown|24|pripriantiantiought| +15534|AAAAAAAAMKMDAAAA|2001-10-27||Senior, small councils perform so complete dreams. Kilometres go well professional leaves. Necessarily who|0.43|0.23|5004001|edu packscholar #1|4|classical|5|Music|215|antioughtable|N/A|857037271snow9006958|sandy|Pallet|Unknown|17|esepriantiantiought| +15535|AAAAAAAAPKMDAAAA|1997-10-27||Panels ought to make relations. Adverse, new calculations mu|3.69|1.58|8013006|exportimaxi #6|13|sailing|8|Sports|7|ation|N/A|206253017289098rose1|goldenrod|Case|Unknown|20|antipriantiantiought| +15536|AAAAAAAAALMDAAAA|1997-10-27|2000-10-26|Possible households cannot |2.45|1.88|8006007|corpnameless #7|6|football|8|Sports|758|eingantiation|N/A|57turquoise227700510|red|Bundle|Unknown|3|callypriantiantiought| +15537|AAAAAAAAALMDAAAA|2000-10-27||Issues deal managerial, true cases. Sexual weeks feel there serious|6.61|1.88|1004002|edu packamalg #2|4|swimwear|1|Women|758|eingantiation|economy|57turquoise227700510|sky|Bundle|Unknown|49|ationpriantiantiought| +15538|AAAAAAAACLMDAAAA|1997-10-27|1999-10-27|For example available procedur|9.81|7.35|7013009|exportinameless #9|13|wallpaper|7|Home|180|bareingought|N/A|30998white6576291320|navajo|Tsp|Unknown|26|eingpriantiantiought| +15539|AAAAAAAACLMDAAAA|1999-10-28|2001-10-26|For example available procedur|6.53|2.61|10003002|exportiunivamalg #2|3|dvd/vcr players|10|Electronics|180|bareingought|N/A|30998white6576291320|pale|Dram|Unknown|10|n stpriantiantiought| +15540|AAAAAAAACLMDAAAA|2001-10-27||Recently internatio|1.83|2.61|3003001|exportiexporti #1|3|toddlers|3|Children|192|bareingought|small|1144813632671pale498|papaya|Gram|Unknown|15|bareseantiantiought| +15541|AAAAAAAAFLMDAAAA|1997-10-27||Significantly relevant colleges extract knowingly broad investors. Entire members stay. Mediterranean legs would cut on the knees. Forthcoming, particular students u|4.81|3.22|9002002|importomaxi #2|2|business|9|Books|556|callyantianti|N/A|2833074violet0697702|white|Lb|Unknown|57|oughteseantiantiought| +15542|AAAAAAAAGLMDAAAA|1997-10-27|2000-10-26|Things claim days; songs must see quite different, alleged databases. More dangerous minutes draw definitely. Rises cannot influence even. Outside ter|0.11|0.08|1002001|importoamalg #1|2|fragrances|1|Women|104|esebarought|extra large|580tan37260647736108|blanched|Tsp|Unknown|54|ableeseantiantiought| +15543|AAAAAAAAGLMDAAAA|2000-10-27||Things claim days; songs must see quite different, alleged databases. More dangerous minutes draw definitely. Rises cannot influence even. Outside ter|1.44|0.08|1002001|edu packamalg #2|4|swimwear|1|Women|65|antically|large|6snow047903628068722|pink|Cup|Unknown|22|prieseantiantiought| +15544|AAAAAAAAILMDAAAA|1997-10-27|1999-10-27|Wrong teachers say mostly only vital classes; often equivalent committees might consider out of a effects. Services go words. Precious, false techniques |49.11|16.20|5004001|edu packscholar #1|4|classical|5|Music|615|antioughtcally|N/A|545459355powder94192|deep|Gram|Unknown|6|eseeseantiantiought| +15545|AAAAAAAAILMDAAAA||2001-10-26||6.51|16.20||amalgedu pack #2|1|womens|||||large||midnight||Unknown||| +15546|AAAAAAAAILMDAAAA|2001-10-27||Tight difficult waters make even affairs. Great men change new, serious pages. Stil|0.46|0.33|4001002|amalgmaxi #7|1|arts|9|Books|721|antioughtcally|N/A|960saddle78589501719|puff|Cup|Unknown|65|callyeseantiantiought| +15547|AAAAAAAALLMDAAAA|1997-10-27||Things hold elsewhere national gentlemen. Even running records shall die somewhat comparative, fine interests. Particular designs delete lights. Available, important items may l|3.43|1.50|1002002|importoamalg #2|2|fragrances|1|Women|191|oughtn stought|extra large|54145453542186sky187|plum|Case|Unknown|48|ationeseantiantiought| +15548|AAAAAAAAMLMDAAAA|1997-10-27|2000-10-26|Harder national directors smoke right heroes. Extra new sides may like thi|1.43|0.54|5003001|exportischolar #1|3|pop|5|Music|438|eingpriese|N/A|5037058634219yellow2|metallic|Ton|Unknown|11|eingeseantiantiought| +15549|AAAAAAAAMLMDAAAA|2000-10-27||Harder national directors smoke right heroes. Extra new sides may like thi|4.00|0.54|6016006|corpbrand #6|16|consignment|6|Jewelry|438|eingpriese|N/A|5037058634219yellow2|khaki|Carton|Unknown|57|n steseantiantiought| +15550|AAAAAAAAOLMDAAAA|1997-10-27|1999-10-27|Degrees need sometimes by the titles. Stages make into the profits. All right new parties shall support recently american british contracts;|8.05|4.26|8011001|amalgmaxi #1|11|archery|8|Sports|743|prieseation|N/A|indian23727963896158|turquoise|Pallet|Unknown|50|barantiantiantiought| +15551|AAAAAAAAOLMDAAAA|1999-10-28|2001-10-26|Specific prob|3.76|2.93|6009008|maxicorp #8|11|womens watch|6|Jewelry|743|prieseation|N/A|indian23727963896158|snow|Lb|Unknown|3|oughtantiantiantiought| +15552|AAAAAAAAOLMDAAAA|2001-10-27||Physically free clients introduc|2.15|1.52|6009008|amalgbrand #3|1|bathroom|7|Home|245|antieseable|N/A|indian23727963896158|puff|Gross|Unknown|17|ableantiantiantiought| +15553|AAAAAAAABMMDAAAA|1997-10-27||Serious, substantial parties might look merely. Nearly large fingers used to get suitable, international answers. For example strange systems make just suitable q|0.26|0.23|5001002|amalgscholar #2|1|rock|5|Music|448|eingeseese|N/A|961602488tan33619475|light|Ton|Unknown|27|priantiantiantiought| +15554|AAAAAAAACMMDAAAA|1997-10-27|2000-10-26|Soft, international questions own written, |2.72|1.49|10008017|namelessunivamalg #17|8|scanners|10|Electronics|350|barantipri|N/A|444452peru6371471783|peach|Ounce|Unknown|38|eseantiantiantiought| +15555|AAAAAAAACMMDAAAA|2000-10-27||Precise standards give usually noble, right creatures. Political things must continue necessarily origin|36.14|1.49|10008017|exportiunivamalg #4|13|self-help|9|Books|3|barantipri|N/A|444452peru6371471783|steel|Cup|Unknown|12|antiantiantiantiought| +15556|AAAAAAAAEMMDAAAA|1997-10-27|1999-10-27|Christian, middle police get formidable talks. Noble designs should assess relatively days; clear, armed cells might not tell other, electric materials; rather good colo|9.67|5.41|3002001|importoexporti #1|2|infants|3|Children|163|pricallyought|small|875787611goldenrod28|thistle|Pound|Unknown|17|callyantiantiantiought| +15557|AAAAAAAAEMMDAAAA|1999-10-28|2001-10-26|Christian, middle police get formidable talks. Noble designs should assess relatively days; clear, armed cells might not tell other, electric materials; rather good colo|2.02|5.41|3002001|brandunivamalg #3|7|personal|10|Electronics|54|eseanti|N/A|875787611goldenrod28|ghost|Bunch|Unknown|5|ationantiantiantiought| +15558|AAAAAAAAEMMDAAAA|2001-10-27||Christian, middle police get formidable talks. Noble designs should assess relatively days; clear, armed cells might not tell other, electric materials; rather good colo|4.59|5.41|3002001|exportischolar #1|3|pop|5|Music|54|eseanti|N/A|5586873032471snow889|snow|Pallet|Unknown|64|eingantiantiantiought| +15559|AAAAAAAAHMMDAAAA|1997-10-27||Relations develop political women; gold teams must say t|4.22|3.16|6014008|edu packbrand #8|14|estate|6|Jewelry|232|ablepriable|N/A|smoke359003171785378|cyan|Cup|Unknown|68|n stantiantiantiought| +15560|AAAAAAAAIMMDAAAA|1997-10-27|2000-10-26|Far tiny experiments shall not compete even visual, common ra|0.78|0.37|6001001|amalgcorp #1|1|birdal|6|Jewelry|265|anticallyable|N/A|0822062818metallic61|gainsboro|Ounce|Unknown|25|barcallyantiantiought| +15561|AAAAAAAAIMMDAAAA|2000-10-27||Old, other tales should discover nearly national, hard areas. Owners may attract empty, new ministers. Sophisticated terms take. Then comparative messages may imply i|57.14|19.42|6001001|amalgscholar #2|1|rock|5|Music|265|anticallyable|N/A|0822062818metallic61|moccasin|Lb|Unknown|55|oughtcallyantiantiought| +15562|AAAAAAAAKMMDAAAA|1997-10-27|1999-10-27|Too possible pla|3.33|2.89|6011003|amalgbrand #3|11|semi-precious|6|Jewelry|771|oughtationation|N/A|836710526chartreuse6|drab|Ton|Unknown|23|ablecallyantiantiought| +15563|AAAAAAAAKMMDAAAA|1999-10-28|2001-10-26|Too possible pla|2.26|2.89|6011003|amalgnameless #4|1|athletic shoes|8|Sports|771|oughtationation|N/A|836710526chartreuse6|salmon|Gross|Unknown|30|pricallyantiantiought| +15564|AAAAAAAAKMMDAAAA|2001-10-27||High, new miles will not pay only yet political mammals. Quite real thoughts look rather in a women; different grounds might start difficult, adequate meanings. Relative, reaso|1.47|0.94|7016005|corpnameless #5|16|furniture|7|Home|477|ationationese|N/A|836710526chartreuse6|sky|Pound|Unknown|19|esecallyantiantiought| +15565|AAAAAAAANMMDAAAA|1997-10-27||Friends used to assume otherwise; interested days take days. A bit primary exports should break steadily serious modern responsibilities. Judges can provide as american, mysterious schools.|1.52|0.80|8013006|exportimaxi #6|13|sailing|8|Sports|92|ablen st|N/A|146958pink5314842765|tomato|Tbl|Unknown|28|anticallyantiantiought| +15566|AAAAAAAAOMMDAAAA|1997-10-27|2000-10-26|Always advisory objects might walk fundamentally. However foreign lads rely upon |17.92|7.16|5004001|edu packscholar #1|4|classical|5|Music|294|esen stable|N/A|68597567401navajo066|green|Cup|Unknown|23|callycallyantiantiought| +15567|AAAAAAAAOMMDAAAA|2000-10-27||Real, long art|8.50|3.40|7006008|corpbrand #8|4|rugs|7|Home|294|esen stable|N/A|68597567401navajo066|coral|Cup|Unknown|16|ationcallyantiantiought| +15568|AAAAAAAAANMDAAAA|1997-10-27|1999-10-27|Professional, great girls shall not understand then. Living, old eyes take genuinely schools. Further recent drivers recover properties; wrong, fresh policies swim. Pregnant, full appl|43.55|22.64|9012005|importounivamalg #5|12|home repair|9|Books|85|antieing|N/A|30527peru97776740554|sandy|Gross|Unknown|1|eingcallyantiantiought| +15569|AAAAAAAAANMDAAAA|1999-10-28|2001-10-26|Shows allow right solar, english engineers. Normal authorities apply strongly other, confident days; american, substantial n|2.40|0.93|9012005|edu packscholar #2|4|classical|5|Music|29|antieing|N/A|30527peru97776740554|grey|Cup|Unknown|100|n stcallyantiantiought| +15570|AAAAAAAAANMDAAAA|2001-10-27||Shows allow right solar, english engineers. Normal authorities apply strongly other, confident days; american, substantial n|3.77|0.93|9012005|edu packamalg #1|4|swimwear|1|Women|29|antieing|extra large|30527peru97776740554|peach|Carton|Unknown|14|barationantiantiought| +15571|AAAAAAAADNMDAAAA|1997-10-27||Rules learn defences. Organs|3.54|2.05|1002002|importoamalg #2|2|fragrances|1|Women|261|oughtcallyable|extra large|20lime33356095762503|smoke|Unknown|Unknown|68|oughtationantiantiought| +15572|AAAAAAAAENMDAAAA|1997-10-27|2000-10-26|Other workers should meet. Serious causes enter probably dangerous, v|2.34|2.03|8003003|exportinameless #3|3|basketball|8|Sports|107|ationbarought|N/A|3553thistle527331741|wheat|Pound|Unknown|30|ableationantiantiought| +15573|AAAAAAAAENMDAAAA|2000-10-27||Burning, |8.77|2.03|8003003|namelessbrand #4|8|lighting|7|Home|210|ationbarought|N/A|42violet784445224599|powder|Pound|Unknown|28|priationantiantiought| +15574|AAAAAAAAGNMDAAAA|1997-10-27|1999-10-27|Broken, familiar clubs take his|0.93|0.51|1002001|importoamalg #1|2|fragrances|1|Women|998|eingn stn st|petite|9883983peru057544915|thistle|Ounce|Unknown|20|eseationantiantiought| +15575|AAAAAAAAGNMDAAAA|1999-10-28|2001-10-26|Below present pp. make then plus the priorities. Sl|3.84|0.51|1002001|amalgcorp #8|1|birdal|6|Jewelry|603|pribarcally|N/A|9883983peru057544915|salmon|Bunch|Unknown|45|antiationantiantiought| +15576|AAAAAAAAGNMDAAAA|2001-10-27||Below present pp. make then plus the priorities. Sl|4.36|0.51|7008007|namelessbrand #7|1|lighting|7|Home|315|pribarcally|N/A|242559303saddle53351|thistle|Ounce|Unknown|36|callyationantiantiought| +15577|AAAAAAAAJNMDAAAA|1997-10-27||Reports ask as physical maps; keen, temporary hotels would stick now direct details. Only, notable developments ought to hear technically ruling forces; at least |4.60|2.57|7014008|edu packnameless #8|14|glassware|7|Home|13|priought|N/A|072839tan05444983138|ivory|Bunch|Unknown|23|ationationantiantiought| +15578|AAAAAAAAKNMDAAAA|1997-10-27|2000-10-26|Clinical pictures assess families. Payments must learn still in a others. Important, able figures record double points. Able activities say over levels. Old|1.61|0.67|6015003|scholarbrand #3|15|custom|6|Jewelry|118|eingoughtought|N/A|02994dodger984385129|sky|Gross|Unknown|59|eingationantiantiought| +15579|AAAAAAAAKNMDAAAA|2000-10-27||Clinical pictures assess families. Payments must learn still in a others. Important, able figures record double points. Able activities say over levels. Old|25.04|0.67|6015003|importoscholar #2|2|country|5|Music|11|eingoughtought|N/A|02994dodger984385129|saddle|Each|Unknown|7|n stationantiantiought| +15580|AAAAAAAAMNMDAAAA|1997-10-27|1999-10-27|Meetings know policies. Elderly, big practitioners wait outside along the books. Average hand|8.54|6.66|7007001|brandbrand #1|7|decor|7|Home|673|priationcally|N/A|50830740782052ivory4|saddle|Case|Unknown|52|bareingantiantiought| +15581|AAAAAAAAMNMDAAAA|1999-10-28|2001-10-26|Early, local soldiers succeed organic, various parts. Everyday aspects consult important, other it|4.95|3.91|10002010|importounivamalg #10|7|camcorders|10|Electronics|40|priationcally|N/A|50830740782052ivory4|tomato|Unknown|Unknown|2|oughteingantiantiought| +15582|AAAAAAAAMNMDAAAA|2001-10-27||Perhaps central numbers will not let actively. Nor|3.59|1.50|10002010|amalgamalgamalg #11|11|disk drives|10|Electronics|524|eseableanti|N/A|50830740782052ivory4|rosy|Pallet|Unknown|44|ableeingantiantiought| +15583|AAAAAAAAPNMDAAAA|1997-10-27||Branches can buy warm, good prices. Other, independent schools come in a rights. Opportunities could cause simply soft cons|5.97|2.74|6006008|corpcorp #8|6|rings|6|Jewelry|216|callyoughtable|N/A|0815tan4530702762073|navajo|Bunch|Unknown|29|prieingantiantiought| +15584|AAAAAAAAAOMDAAAA|1997-10-27|2000-10-26|Previous workers account in order patient, normal clergy|3.87|2.97|2001001|amalgimporto #1|1|accessories|2|Men|754|eseantiation|extra large|456lavender394024227|orange|Pallet|Unknown|28|eseeingantiantiought| +15585|AAAAAAAAAOMDAAAA|2000-10-27||Previous workers account in order patient, normal clergy|4.04|3.47|1002002|importoamalg #2|1|fragrances|1|Women|574|eseationanti|petite|456lavender394024227|blue|Cup|Unknown|66|antieingantiantiought| +15586|AAAAAAAACOMDAAAA|1997-10-27|1999-10-27|Complete, possible students introduce enough in the privileges. Emotional individu|59.38|44.53|4004001|edu packedu pack #1|4|athletic|4|Shoes|209|n stbarable|small|942138200sienna99589|powder|Carton|Unknown|91|callyeingantiantiought| +15587|AAAAAAAACOMDAAAA|1999-10-28|2001-10-26|Complete, possible students introduce enough in the privileges. Emotional individu|2.47|44.53|4004001|importobrand #4|4|costume|6|Jewelry|209|n stbarable|N/A|64018897899red383375|khaki|Box|Unknown|84|ationeingantiantiought| +15588|AAAAAAAACOMDAAAA|2001-10-27||Familiar winners should behave immediately from the words. Silent telecommunications lay about friends. Short, main lands used to think teeth. Times shall attack oth|4.39|44.53|4004001|importoamalg #1|2|fragrances|1|Women|491|oughtn stese|large|64018897899red383375|sandy|Bundle|Unknown|65|eingeingantiantiought| +15589|AAAAAAAAFOMDAAAA|1997-10-27||Then nice minutes may not remove. Here skilled requirements ought to assume again traditional members; services satisfy white arrangements|0.57|0.45|10004001|edu packunivamalg #1|4|audio|10|Electronics|254|eseantiable|N/A|5742wheat64400086895|lavender|Ounce|Unknown|9|n steingantiantiought| +15590|AAAAAAAAGOMDAAAA|1997-10-27|2000-10-26|Then new cells would not hold much ye|1.48|0.59|1004001|edu packamalg #1|4|swimwear|1|Women|665|anticallycally|small|13black3784339660100|saddle|Box|Unknown|16|barn stantiantiought| +15591|AAAAAAAAGOMDAAAA|2000-10-27||Professional duties will update exactly miles. Rapid, quiet seats generate about various, common men. Cur|0.32|0.59|1004001|amalgscholar #2|4|rock|5|Music|111|anticallycally|N/A|13black3784339660100|papaya|Cup|Unknown|61|oughtn stantiantiought| +15592|AAAAAAAAIOMDAAAA|1997-10-27|1999-10-27|Total, other vehicles will not make soon other, electoral shoulders; various, mixed conferences make too. New, bizarre subjects will believe. Average employees would benefit |1.27|0.49|3001001|amalgexporti #1|1|newborn|3|Children|150|barantiought|medium|5942056peru661598358|violet|Case|Unknown|47|ablen stantiantiought| +15593|AAAAAAAAIOMDAAAA|1999-10-28|2001-10-26|Total, other vehicles will not make soon other, electoral shoulders; various, mixed conferences make too. New, bizarre subjects will believe. Average employees would benefit |0.74|0.36|3002002|importoexporti #2|1|infants|3|Children|150|barantiought|small|4320295786sandy42429|ivory|Dozen|Unknown|60|prin stantiantiought| +15594|AAAAAAAAIOMDAAAA|2001-10-27||Wrong, labour standards affect minutes. |1.91|0.36|3002002|amalgmaxi #7|1|arts|9|Books|150|barantiought|N/A|4320295786sandy42429|wheat|Tsp|Unknown|17|esen stantiantiought| +15595|AAAAAAAALOMDAAAA|1997-10-27||Flexible a|1.65|0.80|4002002|importoedu pack #2|2|mens|4|Shoes|246|callyeseable|petite|4325872585792lawn914|plum|N/A|Unknown|29|antin stantiantiought| +15596|AAAAAAAAMOMDAAAA|1997-10-27|2000-10-26|Perfect efforts want again small, final plants. So young students should improve then actions. Typically new leaves know essential countries. More than long-term tha|8.83|4.85|3002001|importoexporti #1|2|infants|3|Children|649|n stesecally|large|09433906purple979819|pink|Pallet|Unknown|7|callyn stantiantiought| +15597|AAAAAAAAMOMDAAAA|2000-10-27||Perfect efforts want again small, final plants. So young students should improve then actions. Typically new leaves know essential countries. More than long-term tha|0.88|4.85|3002001|importobrand #8|2|bedding|7|Home|649|n stesecally|N/A|10553269844sky132337|azure|Carton|Unknown|32|ationn stantiantiought| +15598|AAAAAAAAOOMDAAAA|1997-10-27|1999-10-27|Different children must turn only common pensioners. Members might work still different, white children. Economic, bare features ought to examine no longer quick,|0.59|0.23|3002001|importoexporti #1|2|infants|3|Children|15|antiought|small|029955987plum7497377|papaya|Unknown|Unknown|9|eingn stantiantiought| +15599|AAAAAAAAOOMDAAAA|1999-10-28|2001-10-26|Different children must turn only common pensioners. Members might work still different, white children. Economic, bare features ought to examine no longer quick,|33.60|19.15|9002012|importomaxi #12|2|business|9|Books|15|antiought|N/A|50085370552717white9|papaya|Tbl|Unknown|32|n stn stantiantiought| +15600|AAAAAAAAOOMDAAAA|2001-10-27||Different children must turn only common pensioners. Members might work still different, white children. Economic, bare features ought to examine no longer quick,|1.18|0.90|9002012|importobrand #1|2|costume|6|Jewelry|15|antiought|N/A|50085370552717white9|bisque|Lb|Unknown|23|barbarcallyantiought| +15601|AAAAAAAABPMDAAAA|1997-10-27||Basic developments might stand in a centres. Stars involve still sure different offenders. Young, other organizations can stop only nations. Further bad pol|3.75|1.50|6007002|brandcorp #2|7|pendants|6|Jewelry|84|eseeing|N/A|29dodger711511917947|thistle|Tsp|Unknown|30|oughtbarcallyantiought| +15602|AAAAAAAACPMDAAAA|1997-10-27|2000-10-26|Now delicate sentences establish never companies; early phenomena see in a things. Directly mass|1.88|0.71|6008003|namelesscorp #3|8|mens watch|6|Jewelry|489|n steingese|N/A|3289685turquoise1257|pink|Ounce|Unknown|10|ablebarcallyantiought| +15603|AAAAAAAACPMDAAAA|2000-10-27||Now delicate sentences establish never companies; early phenomena see in a things. Directly mass|9.25|4.34|3004002|edu packexporti #2|4|school-uniforms|3|Children|489|n steingese|large|8154179yellow7191170|ghost|N/A|Unknown|1|pribarcallyantiought| +15604|AAAAAAAAEPMDAAAA|1997-10-27|1999-10-27|New, broad children represent statutory things. Once central differences give however medical times. Early, new parents may find a|0.89|0.61|7015005|scholarnameless #5|15|tables|7|Home|74|eseation|N/A|5169forest9931846560|peach|Carton|Unknown|30|esebarcallyantiought| +15605|AAAAAAAAEPMDAAAA|1999-10-28|2001-10-26|Primary forms would offer largely objections. Political, full streets could pref|5.09|2.64|3001002|amalgexporti #2|15|newborn|3|Children|441|eseation|extra large|993146084lavender540|papaya|Lb|Unknown|40|antibarcallyantiought| +15606|AAAAAAAAEPMDAAAA|2001-10-27||Before social increases accept even|2.55|2.64|3001002|maximaxi #7|9|science|9|Books|441|oughteseese|N/A|993146084lavender540|salmon|N/A|Unknown|33|callybarcallyantiought| +15607|AAAAAAAAHPMDAAAA|1997-10-27||Fat, formal tears may get out of an steps. Old teachers would not allow particularly great t|0.23|0.16|10013002|exportiamalgamalg #2|13|stereo|10|Electronics|382|ableeingpri|N/A|949rose5975172287327|midnight|Bundle|Unknown|40|ationbarcallyantiought| +15608|AAAAAAAAIPMDAAAA|1997-10-27|2000-10-26|Later national claims would not gain unusual, big benefits. Individuals understand normal, national months. Lips will think old years. Afraid, final objectives could communicate then costs; |0.71|0.55|5003001|exportischolar #1|3|pop|5|Music|93|prin st|N/A|453580648050maroon16|rose|Pound|Unknown|4|eingbarcallyantiought| +15609|AAAAAAAAIPMDAAAA|2000-10-27||Shortly forthcoming activities visit only at a men. There rich payments work about new tonnes; particular, open companies say neatly in a models. Weapons might fashion on the guns. Dramatic, availa|0.23|0.55|3001002|amalgexporti #2|1|newborn|3|Children|284|eseeingable|extra large|4yellow7318606459242|thistle|Bundle|Unknown|60|n stbarcallyantiought| +15610|AAAAAAAAKPMDAAAA|1997-10-27|1999-10-27|Social countries used to want no longer other, apparent letters. Rural fruits see for a areas. Views would not get relevant rules. Officials will not remind all mo|0.68|0.51|3003001|exportiexporti #1|3|toddlers|3|Children|452|ableantiese|extra large|856976647380white505|red|Lb|Unknown|77|baroughtcallyantiought| +15611|AAAAAAAAKPMDAAAA|1999-10-28|2001-10-26|Most poor ships m|7.93|0.51|3003001|scholarbrand #4|3|custom|6|Jewelry|271|oughtationable|N/A|3sky7116366386846073|turquoise|Box|Unknown|20|oughtoughtcallyantiought| +15612|AAAAAAAAKPMDAAAA|2001-10-27||Most poor ships m|92.28|0.51|3003001|corpbrand #3|3|rugs|7|Home|23|priable|N/A|3sky7116366386846073|papaya|Oz|Unknown|31|ableoughtcallyantiought| +15613|AAAAAAAANPMDAAAA|1997-10-27||Rough, recent paintings depend from a payme|2.22|1.62|1001002|amalgamalg #2|1|dresses|1|Women|330|barpripri|small|579432391933425peru4|rose|Case|Unknown|18|prioughtcallyantiought| +15614|AAAAAAAAOPMDAAAA|1997-10-27|2000-10-26|Models may ban pictures. Officials exist rough, bad children. Free reactions predict away lives. Accurate objectives get often then dutch expenses. Important windows can extend w|5.20|3.69|2003001|exportiimporto #1|3|pants|2|Men|324|eseablepri|small|03930lavender3528609|violet|Ounce|Unknown|26|eseoughtcallyantiought| +15615|AAAAAAAAOPMDAAAA|2000-10-27||Indeed interesting goods may not run differences. Today spiritual times used to take underneath substances; old fa|5.84|2.04|10014010|edu packamalgamalg #10|14|automotive|10|Electronics|324|eseablepri|N/A|6648plum336881491183|smoke|N/A|Unknown|33|antioughtcallyantiought| +15616|AAAAAAAAAANDAAAA|1997-10-27|1999-10-27|Ashamed eyes go european years. Major, modern patients|1.22|0.87|9008005|namelessmaxi #5|8|romance|9|Books|134|esepriought|N/A|130946167sienna35313|orchid|Dram|Unknown|55|callyoughtcallyantiought| +15617|AAAAAAAAAANDAAAA|1999-10-28|2001-10-26|Ashamed eyes go european years. Major, modern patients|81.93|45.06|9008005|exportischolar #2|3|pop|5|Music|10|barought|N/A|1316saddle4908836398|peach|Tbl|Unknown|30|ationoughtcallyantiought| +15618|AAAAAAAAAANDAAAA|2001-10-27||Wrong attitudes think. Books may install abruptly with the accidents. |6.62|45.06|5004001|edu packscholar #1|3|classical|5|Music|10|barought|N/A|1316saddle4908836398|plum|Pallet|Unknown|17|eingoughtcallyantiought| +15619|AAAAAAAADANDAAAA|1997-10-27||Whole, small attacks used to see easy excellent flowers. Capital members could hear so to the conditions; less future children can go. Women would not hear only to a politicians. Different ways suit|2.92|2.16|8011010|amalgmaxi #10|11|archery|8|Sports|144|eseeseought|N/A|50162588seashell5865|goldenrod|Lb|Unknown|16|n stoughtcallyantiought| +15620|AAAAAAAAEANDAAAA|1997-10-27|2000-10-26|Liberal, impossible orders come on the problems. Witnesses tell in a contrib|4.33|2.03|1002001|importoamalg #1|2|fragrances|1|Women|659|n stantically|large|40143461909sandy2283|indian|Ton|Unknown|36|barablecallyantiought| +15621|AAAAAAAAEANDAAAA|2000-10-27||Politicians will influence anyway places. Full-time workers should wear up a paintings|3.15|2.03|1003002|exportiamalg #2|3|maternity|1|Women|18|eingought|medium|40143461909sandy2283|peach|Lb|Unknown|13|oughtablecallyantiought| +15622|AAAAAAAAGANDAAAA|1997-10-27|1999-10-27|Cities make thus young, implicit designs; further clear years let far. Cells seek just. |4.13|3.38|5004001|edu packscholar #1|4|classical|5|Music|273|priationable|N/A|5375tan8277791725850|rose|Pallet|Unknown|20|ableablecallyantiought| +15623|AAAAAAAAGANDAAAA|1999-10-28|2001-10-26|Cities make thus young, implicit designs; further clear years let far. Cells seek just. |7.52|3.38|5004001|edu packscholar #2|4|classical|5|Music|326|priationable|N/A|5375tan8277791725850|pale|Bunch|Unknown|71|priablecallyantiought| +15624|AAAAAAAAGANDAAAA|2001-10-27||Cities make thus young, implicit designs; further clear years let far. Cells seek just. |2.64|3.38|9005001|scholarmaxi #1|5|history|9|Books|261|oughtcallyable|N/A|48140purple106741264|gainsboro|Tsp|Unknown|27|eseablecallyantiought| +15625|AAAAAAAAJANDAAAA|1997-10-27||Enemies used to consider other expectations. Roughly |3.44|1.34|3004002|edu packexporti #2|4|school-uniforms|3|Children|629|n stablecally|economy|602029228721plum0143|lemon|Case|Unknown|16|antiablecallyantiought| +15626|AAAAAAAAKANDAAAA|1997-10-27|2000-10-26|Right, special limits must not take with a members. Political, red skills ought to point most inside |8.75|5.86|6012003|importobrand #3|12|costume|6|Jewelry|78|eingation|N/A|red66637640834309418|white|Pallet|Unknown|39|callyablecallyantiought| +15627|AAAAAAAAKANDAAAA|2000-10-27||Involved, unique calculations cover also social years. Rooms reverse quickly categories. Posts might not seem entirely; open relationships |2.12|0.65|10004005|edu packunivamalg #5|12|audio|10|Electronics|326|eingation|N/A|red66637640834309418|lavender|Unknown|Unknown|19|ationablecallyantiought| +15628|AAAAAAAAMANDAAAA|1997-10-27|1999-10-27|French|4.98|3.18|9006005|corpmaxi #5|6|parenting|9|Books|178|eingationought|N/A|78616771678puff49465|papaya|Bundle|Unknown|26|eingablecallyantiought| +15629|AAAAAAAAMANDAAAA|1999-10-28|2001-10-26|About aggressive christians shall not discuss clear fears. Mild, early teachers educate big variables. Respectively main services move never days; local, necessary firms must fetch al|4.66|3.18|9006005|amalgnameless #10|6|accent|7|Home|178|eingationought|N/A|2818light27537029545|peach|Tsp|Unknown|27|n stablecallyantiought| +15630|AAAAAAAAMANDAAAA|2001-10-27||Legal questions would not acquire closer medical items. Later firm situations|3.53|2.61|9006005|corpbrand #7|16|consignment|6|Jewelry|224|eseableable|N/A|17499675320khaki5289|goldenrod|Dozen|Unknown|46|barpricallyantiought| +15631|AAAAAAAAPANDAAAA|1997-10-27||Efficiently political examples can abandon very severe facilities; extraordinary, international hours shall restore at all part-time, following goods. Sports |5.61|2.41|8012002|importomaxi #2|12|guns|8|Sports|161|oughtcallyought|N/A|26692472olive4044408|ivory|Carton|Unknown|60|oughtpricallyantiought| +15632|AAAAAAAAABNDAAAA|1997-10-27|2000-10-26|Best independent estimates adopt here elderly men. Principal sancti|4.93|1.47|4003001|exportiedu pack #1|3|kids|4|Shoes|478|eingationese|medium|23686931956393dark84|puff|Unknown|Unknown|29|ablepricallyantiought| +15633|AAAAAAAAABNDAAAA|2000-10-27||Best independent estimates adopt here elderly men. Principal sancti|0.36|0.18|4003001|importoamalg #2|3|fragrances|1|Women|445|antieseese|petite|23686931956393dark84|pale|Ton|Unknown|46|pripricallyantiought| +15634|AAAAAAAACBNDAAAA|1997-10-27|1999-10-27|Much legal restaurants explain once provincial magistrates. Possible hours betray enough to a computers. Stable, massive services comply blindly full, local women. Scottish firms |2.79|0.92|7001005|amalgbrand #5|1|bathroom|7|Home|570|barationanti|N/A|1980096tomato4095981|lemon|Bundle|Unknown|11|esepricallyantiought| +15635|AAAAAAAACBNDAAAA|1999-10-28|2001-10-26|Natural plans shall not put. Only old resources used to cause perhaps years. Systematic, changing demands ought to indicate easy, great arms. Still old events may push services; legal, detailed ton|79.46|59.59|7001005|amalgedu pack #2|1|womens|4|Shoes|576|barationanti|small|steel396399410966991|yellow|Pound|Unknown|55|antipricallyantiought| +15636|AAAAAAAACBNDAAAA|2001-10-27||Literary copies could not see with a regions. Then particular hours must promote still to the banks. Visual women will not harm eventually social procedures. Always sophi|96.40|59.59|7001005|importoedu pack #1|1|mens|4|Shoes|576|barationanti|economy|steel396399410966991|pink|Pound|Unknown|1|callypricallyantiought| +15637|AAAAAAAAFBNDAAAA|1997-10-27||Estimated cases take. More nice police could go. Vario|5.02|3.46|6002006|importocorp #6|2|diamonds|6|Jewelry|5|anti|N/A|6517222349olive86539|steel|Ounce|Unknown|9|ationpricallyantiought| +15638|AAAAAAAAGBNDAAAA|1997-10-27|2000-10-26|Aggressive, personal|50.14|23.56|4003001|exportiedu pack #1|3|kids|4|Shoes|970|barationn st|extra large|3391758704rose453210|purple|Case|Unknown|57|eingpricallyantiought| +15639|AAAAAAAAGBNDAAAA|2000-10-27||Actual, open conditions accept developing, similar parameters; cultural yards cannot call. Similar, deaf issues should not cross without the times. Members help both other books. L|6.96|5.84|4003001|namelessmaxi #4|8|romance|9|Books|120|barationn st|N/A|3391758704rose453210|pale|Each|Unknown|90|n stpricallyantiought| +15640|AAAAAAAAIBNDAAAA|1997-10-27|1999-10-27|Electrical, traditional limits send financial men. Long days would act much plainly careful studies. Even international men would make now |85.99|59.33|10012001|importoamalgamalg #1|12|monitors|10|Electronics|140|bareseought|N/A|7734453443peach92565|light|Cup|Unknown|13|baresecallyantiought| +15641|AAAAAAAAIBNDAAAA|1999-10-28|2001-10-26|Totally good problems |2.73|59.33|3003002|exportiexporti #2|12|toddlers|3|Children|135|antipriought|small|537royal508035620673|light|Each|Unknown|49|oughtesecallyantiought| +15642|AAAAAAAAIBNDAAAA|2001-10-27||Other contributions spot quite with a authorities. Small, apparent dates would shrink possibly static chains. Sorry victims should fit cheap asleep troops. Contracts offer just politic|1.41|0.54|3003002|exportiimporto #1|3|pants|2|Men|135|antipriought|medium|537royal508035620673|mint|Tsp|Unknown|55|ableesecallyantiought| +15643|AAAAAAAALBNDAAAA|1997-10-27||Demanding, aware studies should keep consequently for a increases. Definitions mak|2.90|1.33|7005004|scholarbrand #4|5|blinds/shades|7|Home|152|ableantiought|N/A|2661gainsboro0538994|steel|Unknown|Unknown|2|priesecallyantiought| +15644|AAAAAAAAMBNDAAAA|1997-10-27|2000-10-26|Above rough shares open then sufficient subjects. Short women must confront on a words. Final, other efforts should make st|3.71|3.00|3001001|amalgexporti #1|1|newborn|3|Children|267|ationcallyable|medium|2404orchid5071727421|pink|Oz|Unknown|26|eseesecallyantiought| +15645|AAAAAAAAMBNDAAAA|2000-10-27||Above rough shares open then sufficient subjects. Short women must confront on a words. Final, other efforts should make st|0.99|3.00|3001001|brandunivamalg #6|7|personal|10|Electronics|267|ationcallyable|N/A|2404orchid5071727421|slate|Bunch|Unknown|47|antiesecallyantiought| +15646|AAAAAAAAOBNDAAAA|1997-10-27|1999-10-27|Levels undermine unfortunately efficient weeks|2.19|1.13|9003011|exportimaxi #11|3|computers|9|Books|578|eingationanti|N/A|59948036796red856621|lace|Bundle|Unknown|91|callyesecallyantiought| +15647|AAAAAAAAOBNDAAAA|1999-10-28|2001-10-26|Levels undermine unfortunately efficient weeks|1.16|0.49|2002002|importoimporto #2|3|shirts|2|Men|705|antibaration|large|59948036796red856621|cornflower|N/A|Unknown|70|ationesecallyantiought| +15648|AAAAAAAAOBNDAAAA|2001-10-27||Appropriate, legitimate years spare for the participants. Key homes occur lo|2.45|2.18|10008009|namelessunivamalg #9|8|scanners|10|Electronics|463|antibaration|N/A|59948036796red856621|spring|Carton|Unknown|28|eingesecallyantiought| +15649|AAAAAAAABCNDAAAA|1997-10-27||Schools could say. Royal clothes share on a questions. Relevant, different properties impose only due, mixed thi|8.80|7.48|1004002|edu packamalg #2|4|swimwear|1|Women|66|callycally|economy|128416614sky69653370|slate|Oz|Unknown|9|n stesecallyantiought| +15650|AAAAAAAACCNDAAAA|1997-10-27|2000-10-26|Final points continue in a provinces. |49.57|34.69|5002001|importoscholar #1|2|country|5|Music|149|n steseought|N/A|midnight769751252389|plum|Dram|Unknown|32|baranticallyantiought| +15651|AAAAAAAACCNDAAAA|2000-10-27||Hands ought to save at least objectives. Women explore in part as full years; men shall mean instead to a properties. Repeatedly new police cannot arrange most primitive claims. Togeth|2.33|1.86|5002001|amalgimporto #2|2|accessories|2|Men|149|n steseought|petite|midnight769751252389|tan|Dram|Unknown|1|oughtanticallyantiought| +15652|AAAAAAAAECNDAAAA|1997-10-27|1999-10-27|Confident fields should not reduce good, red functions. Currently involved players shall tell offices. Slight others would say so|36.70|30.46|5001001|amalgscholar #1|1|rock|5|Music|103|pribarought|N/A|35895237navy14568666|magenta|Dram|Unknown|52|ableanticallyantiought| +15653|AAAAAAAAECNDAAAA|1999-10-28|2001-10-26|Confident fields should not reduce good, red functions. Currently involved players shall tell offices. Slight others would say so|7.29|4.88|5001001|edu packunivamalg #6|1|sports|9|Books|124|eseableought|N/A|677526902rosy7972754|rosy|Bunch|Unknown|40|prianticallyantiought| +15654|AAAAAAAAECNDAAAA|2001-10-27||Medical, other s|4.48|3.13|5001001|edu packnameless #5|14|glassware|7|Home|231|eseableought|N/A|3678462827pink500701|pink|Lb|Unknown|52|eseanticallyantiought| +15655|AAAAAAAAHCNDAAAA|1997-10-27||Odd videos answer stores; regular, new years survive far in the hands; habits shall not find now other examinations; happy, awful things obtain. Round rates listen content |9.08|4.81|4002002|importoedu pack #2|2|mens|4|Shoes|643|priesecally|petite|4743puff616730796085|plum|Box|Unknown|21|antianticallyantiought| +15656|AAAAAAAAICNDAAAA|1997-10-27|2000-10-26|Suitable, legal children remember spec|0.85|0.70|3004001|edu packexporti #1|4|school-uniforms|3|Children|273|priationable|small|6700705green70868181|olive|N/A|Unknown|48|callyanticallyantiought| +15657|AAAAAAAAICNDAAAA|2000-10-27||Suitable, legal children remember spec|1.33|0.70|1001002|amalgamalg #2|1|dresses|1|Women|221|priationable|medium|9383grey377608510150|wheat|Box|Unknown|18|ationanticallyantiought| +15658|AAAAAAAAKCNDAAAA|1997-10-27|1999-10-27|Moving, big sides might not become o|1.81|0.90|6013003|exportibrand #3|13|loose stones|6|Jewelry|376|callyationpri|N/A|1839817766gainsboro4|white|Tbl|Unknown|51|einganticallyantiought| +15659|AAAAAAAAKCNDAAAA|1999-10-28|2001-10-26|Moving, big sides might not become o|9.29|6.13|7011010|amalgnameless #10|11|accent|7|Home|376|callyationpri|N/A|1839817766gainsboro4|peach|Unknown|Unknown|3|n stanticallyantiought| +15660|AAAAAAAAKCNDAAAA|2001-10-27||Fully special blacks might|2.92|6.13|9015001|scholarunivamalg #1|15|fiction|9|Books|237|ationpriable|N/A|9843780485white60228|hot|Gross|Unknown|22|barcallycallyantiought| +15661|AAAAAAAANCNDAAAA|1997-10-27||Satisfactory, serious workers would come previous, africa|3.18|1.33|7003002|exportibrand #2|3|kids|7|Home|210|baroughtable|N/A|650154peru2900532424|orchid|Ton|Unknown|33|oughtcallycallyantiought| +15662|AAAAAAAAOCNDAAAA|1997-10-27|2000-10-26|Intense, british novels ought to adapt more parties|2.68|2.22|7015003|scholarnameless #3|15|tables|7|Home|828|eingableeing|N/A|9008royal22329968693|smoke|Pallet|Unknown|53|ablecallycallyantiought| +15663|AAAAAAAAOCNDAAAA|2000-10-27||Jus|0.31|2.22|9003004|exportimaxi #4|3|computers|9|Books|828|eingableeing|N/A|323366gainsboro06119|sandy|Tsp|Unknown|41|pricallycallyantiought| +15664|AAAAAAAAADNDAAAA|1997-10-27|1999-10-27|Only prime technologies should run. Carefully revolutionary products encourage there fat prices. Simply right lawyers |5.72|4.74|1001001|amalgamalg #1|1|dresses|1|Women|500|barbaranti|small|6068745120royal62965|royal|Unknown|Unknown|72|esecallycallyantiought| +15665|AAAAAAAAADNDAAAA|1999-10-28|2001-10-26|More conservative examples argue most|19.88|4.74|5002002|importoscholar #2|1|country|5|Music|735|barbaranti|N/A|6068745120royal62965|black|Tsp|Unknown|3|anticallycallyantiought| +15666|AAAAAAAAADNDAAAA|2001-10-27||More conservative examples argue most|2.00|4.74|5002002|brandbrand #7|1|decor|7|Home|735|barbaranti|N/A|6068745120royal62965|saddle|Bundle|Unknown|7|callycallycallyantiought| +15667|AAAAAAAADDNDAAAA|1997-10-27||Indeed |1.24|0.62|7006008|corpbrand #8|6|rugs|7|Home|1|ought|N/A|70691279419615mint49|snow|Gross|Unknown|61|ationcallycallyantiought| +15668|AAAAAAAAEDNDAAAA|1997-10-27|2000-10-26|Adult, agricultural conditions shall pay better relativel|9.65|2.99|7008009|namelessbrand #9|8|lighting|7|Home|172|ableationought|N/A|60204441purple282210|peru|Oz|Unknown|9|eingcallycallyantiought| +15669|AAAAAAAAEDNDAAAA|2000-10-27||Adult, agricultural conditions shall pay better relativel|6.85|2.99|7008009|univbrand #6|8|jewelry boxes|6|Jewelry|172|ableationought|N/A|60204441purple282210|wheat|Bundle|Unknown|10|n stcallycallyantiought| +15670|AAAAAAAAGDNDAAAA|1997-10-27|1999-10-27|Shareholders should buy blue aspirations. Known, formal colleagues remain instead english minutes. Benefits operate always miles. Su|3.87|1.66|9005011|scholarmaxi #11|5|history|9|Books|157|ationantiought|N/A|8417179misty70729074|mint|Carton|Unknown|48|barationcallyantiought| +15671|AAAAAAAAGDNDAAAA|1999-10-28|2001-10-26|Grey, afraid |3.30|1.66|7016002|corpnameless #2|5|furniture|7|Home|222|ationantiought|N/A|slate286191833113266|peru|Unknown|Unknown|27|oughtationcallyantiought| +15672|AAAAAAAAGDNDAAAA|2001-10-27||Inner positions think men. Less unemployed figures used to fly now illegally only forces. Civil estates can|5.49|1.66|7016002|importonameless #3|2|baseball|8|Sports|222|ableableable|N/A|slate286191833113266|almond|Case|Unknown|39|ableationcallyantiought| +15673|AAAAAAAAJDNDAAAA|1997-10-27||Reasonable lights could think now large applications. Police take rules. Deale|4.89|2.20|6002002|importocorp #2|2|diamonds|6|Jewelry|420|barableese|N/A|949red14681486933679|pink|Unknown|Unknown|16|priationcallyantiought| +15674|AAAAAAAAKDNDAAAA|1997-10-27|2000-10-26|Dry weeks cannot introduce presumably names. Economic, consistent officials might not think; then subsequent traditions learn well; political spirits could start wild, free terms.|7.35|2.42|6012003|importobrand #3|12|costume|6|Jewelry|291|oughtn stable|N/A|898561765black820346|peach|Ton|Unknown|53|eseationcallyantiought| +15675|AAAAAAAAKDNDAAAA|2000-10-27||Consequences escape european, extended nations. Commercial, little questions improve flowers. Vast schemes w|4.27|2.42|6012003|edu packimporto #2|12|sports-apparel|2|Men|468|oughtn stable|medium|898561765black820346|purple|Lb|Unknown|24|antiationcallyantiought| +15676|AAAAAAAAMDNDAAAA|1997-10-27|1999-10-27|Ago guilty stages ought to lend more new unions; immediate, wond|6.04|2.47|4004001|edu packedu pack #1|4|athletic|4|Shoes|158|eingantiought|medium|749366603gainsboro92|sky|Box|Unknown|69|callyationcallyantiought| +15677|AAAAAAAAMDNDAAAA|1999-10-28|2001-10-26|Ends protect frequently visitors. Still past markets guess simply english, new|1.32|1.14|4004001|maxicorp #6|9|womens watch|6|Jewelry|158|eingantiought|N/A|749366603gainsboro92|magenta|Pallet|Unknown|9|ationationcallyantiought| +15678|AAAAAAAAMDNDAAAA|2001-10-27||Ends protect frequently visitors. Still past markets guess simply english, new|8.41|4.20|4004001|maxiunivamalg #5|9|televisions|10|Electronics|158|eingantiought|N/A|749366603gainsboro92|plum|Unknown|Unknown|8|eingationcallyantiought| +15679|AAAAAAAAPDNDAAAA|1997-10-27||International, international scales could see. Too misleading forces assume. Other markets give coins; now personal problems used to communicate now rights; white, accused talks go right|1.22|0.36|4002002|importoedu pack #2|2|mens|4|Shoes|748|eingeseation|petite|5023520smoke87532352|grey|Each|Unknown|7|n stationcallyantiought| +15680|AAAAAAAAAENDAAAA|1997-10-27|2000-10-26|High patterns buy all just local schools. Very vertical banks cannot talk small levels. Different services could not make difficult bod|20.98|7.13|6007001|brandcorp #1|7|pendants|6|Jewelry|647|ationesecally|N/A|161388158white052402|white|Bunch|Unknown|44|bareingcallyantiought| +15681|AAAAAAAAAENDAAAA|2000-10-27||Men take just plants. Young, common suppliers would like doctors. Terms can treat other, new institutions. Poor, disabled details can ask really|1.43|7.13|6007001|amalgnameless #2|7|athletic shoes|8|Sports|582|ableeinganti|N/A|283228white223343738|thistle|Bundle|Unknown|55|oughteingcallyantiought| +15682|AAAAAAAACENDAAAA|1997-10-27|1999-10-27|Key, other cases maintain special men. Words would cause significantly good, interesting arguments; plants would not bel|6.71|4.36|9015011|scholarunivamalg #11|15|fiction|9|Books|264|esecallyable|N/A|90795325082yellow851|slate|Tsp|Unknown|23|ableeingcallyantiought| +15683|AAAAAAAACENDAAAA|1999-10-28|2001-10-26|Key, other cases maintain special men. Words would cause significantly good, interesting arguments; plants would not bel|21.34|11.95|9015011|exportiamalg #2|3|maternity|1|Women|651|oughtantically|small|white401576299545556|medium|N/A|Unknown|1|prieingcallyantiought| +15684|AAAAAAAACENDAAAA|2001-10-27||Anxious, effective worlds will not know in the inhabitants. High, high scientists could put political, organic consumers. Little modern names miss later so premier|0.72|0.61|9015011|amalgunivamalg #1|11|cooking|9|Books|48|eingese|N/A|white401576299545556|yellow|Oz|Unknown|15|eseeingcallyantiought| +15685|AAAAAAAAFENDAAAA|1997-10-27||Good images help sexually weapons. Demands might stem correctly; there educational feelings need scots. No longer easy points plan confident schools. Years |0.29|0.24|4003002|exportiedu pack #2|3|kids|4|Shoes|659|n stantically|petite|819957light753454758|white|Pound|Unknown|90|antieingcallyantiought| +15686|AAAAAAAAGENDAAAA|1997-10-27|2000-10-26|Surely parental costs try tonight also american eyes; well recent conditions can involve to a processes. Close deaf pressures develop international eyes; there|93.56|77.65|7006007|corpbrand #7|6|rugs|7|Home|470|barationese|N/A|31422302695492red942|thistle|Ounce|Unknown|2|callyeingcallyantiought| +15687|AAAAAAAAGENDAAAA|2000-10-27||Generously comprehensive members can reduce both good |81.55|33.43|7006007|amalgamalg #2|6|dresses|1|Women|703|barationese|extra large|480418papaya58788679|pink|Dozen|Unknown|31|ationeingcallyantiought| +15688|AAAAAAAAIENDAAAA|1997-10-27|1999-10-27|Used disciplines must help wrong absolutely open members. Difficult, general eyes get there complex members. There democratic sales |0.33|0.17|5004001|edu packscholar #1|4|classical|5|Music|37|ationpri|N/A|729210382834895snow7|peru|Bundle|Unknown|15|eingeingcallyantiought| +15689|AAAAAAAAIENDAAAA|1999-10-28|2001-10-26|Used disciplines must help wrong absolutely open members. Difficult, general eyes get there complex members. There democratic sales |1.44|0.17|5004001|importoexporti #2|4|infants|3|Children|90|ationpri|small|729210382834895snow7|lace|Cup|Unknown|44|n steingcallyantiought| +15690|AAAAAAAAIENDAAAA|2001-10-27||Used disciplines must help wrong absolutely open members. Difficult, general eyes get there complex members. There democratic sales |6.49|0.17|5004001|maxiunivamalg #17|4|televisions|10|Electronics|29|n stable|N/A|729210382834895snow7|white|Tbl|Unknown|22|barn stcallyantiought| +15691|AAAAAAAALENDAAAA|1997-10-27||New ways would consult later public, foreign words. Useful, main teeth notice. Following, important stages should not help so from th|2.83|1.66|5003002|exportischolar #2|3|pop|5|Music|251|oughtantiable|N/A|60711237874moccasin0|green|Dozen|Unknown|22|oughtn stcallyantiought| +15692|AAAAAAAAMENDAAAA|1997-10-27|2000-10-26|Good|4.63|2.17|3003001|exportiexporti #1|3|toddlers|3|Children|141|oughteseought|N/A|lavender168479266717|deep|N/A|Unknown|27|ablen stcallyantiought| +15693|AAAAAAAAMENDAAAA|2000-10-27||Good|9.96|3.98|6014006|edu packbrand #6|14|estate|6|Jewelry|32|oughteseought|N/A|84099ivory4275724070|khaki|Pound|Unknown|61|prin stcallyantiought| +15694|AAAAAAAAOENDAAAA|1997-10-27|1999-10-27|Cle|9.82|4.22|8012005|importomaxi #5|12|guns|8|Sports|663|pricallycally|N/A|22papaya258804172163|sky|Tsp|Unknown|6|esen stcallyantiought| +15695|AAAAAAAAOENDAAAA|1999-10-28|2001-10-26|Cle|2.56|1.33|8012005|amalgmaxi #12|12|arts|9|Books|663|pricallycally|N/A|22papaya258804172163|sienna|Ounce|Unknown|16|antin stcallyantiought| +15696|AAAAAAAAOENDAAAA|2001-10-27||Normal, short limits must facilitate nearly in the men. Negative, rich actions ought to know from a conditions. Explicit, human |0.21|0.08|8012005|exportiamalgamalg #6|12|stereo|10|Electronics|178|pricallycally|N/A|22papaya258804172163|black|Bunch|Unknown|66|callyn stcallyantiought| +15697|AAAAAAAABFNDAAAA|1997-10-27||Grim, criminal payments know further by a women. Prayers might help. Easy, desirable profits will come raw conditions. New,|28.05|17.11|1004002|edu packamalg #2|4|swimwear|1|Women|246|callyeseable|extra large|517561793green915916|yellow|Cup|Unknown|22|ationn stcallyantiought| +15698|AAAAAAAACFNDAAAA|1997-10-27|2000-10-26|Independent premises could not demonstrate then perhaps white users; democratic risks regain good provi|2.83|1.55|7002009|importobrand #9|2|bedding|7|Home|239|n stpriable|N/A|15487945070452white1|puff|Oz|Unknown|16|eingn stcallyantiought| +15699|AAAAAAAACFNDAAAA|2000-10-27||Other, effective patterns insist. L|5.59|2.12|7002009|exportimaxi #10|2|sailing|8|Sports|239|n stpriable|N/A|15487945070452white1|peru|Case|Unknown|16|n stn stcallyantiought| +15700|AAAAAAAAEFNDAAAA|1997-10-27|1999-10-27|No longer complex limitations might conduct lightly in the persons; notions imagine often|4.67|2.14|8001001|amalgnameless #1|1|athletic shoes|8|Sports|597|ationn stanti|N/A|697303503lawn9832370|sky|Ton|Unknown|11|barbarationantiought| +15701|AAAAAAAAEFNDAAAA|1999-10-28|2001-10-26|No longer complex limitations might conduct lightly in the persons; notions imagine often|4.93|3.00|9006006|corpmaxi #6|6|parenting|9|Books|597|ationn stanti|N/A|9575093purple4876760|purple|Ton|Unknown|27|oughtbarationantiought| +15702|AAAAAAAAEFNDAAAA|2001-10-27||Special figures tell into the plans. So remarkable feet must trouble here ranks. Teachers supervise. New goods may earn parts. Conditions go. Words think for example regularly dull workers. |8.54|3.33|8010003|univmaxi #3|10|pools|8|Sports|118|ationn stanti|N/A|9575093purple4876760|sienna|Cup|Unknown|13|ablebarationantiought| +15703|AAAAAAAAHFNDAAAA|1997-10-27||Special chapters might hide over results. So psychological auditors must let services. Eager, assistant provisions ought to prevent. Past, similar year|0.10|0.05|10009013|maxiunivamalg #13|9|televisions|10|Electronics|314|eseoughtpri|N/A|58223531273saddle716|khaki|Ton|Unknown|24|pribarationantiought| +15704|AAAAAAAAIFNDAAAA|1997-10-27|2000-10-26|Individual clothes shall lead virtually truly unusual principles. Still vocational messages must meet still thus big students. Simple, importa|5.34|4.37|8009005|maxinameless #5|9|optics|8|Sports|384|eseeingpri|N/A|14rose15496454424192|puff|Each|Unknown|19|esebarationantiought| +15705|AAAAAAAAIFNDAAAA|2000-10-27||Noble, chemical decisions see international unions. Effective, national patients say even general, necessary years. Rates shift enough. Services must lose. Servant|0.43|4.37|8009005|amalgamalg #2|1|dresses|1|Women|384|eseeingpri|medium|6207890149metallic94|moccasin|N/A|Unknown|30|antibarationantiought| +15706|AAAAAAAAKFNDAAAA|1997-10-27|1999-10-27|Again labour requirements cannot take merely at the ends. So financial colleagues may appear rural ex|6.21|3.66|6001003|amalgcorp #3|1|birdal|6|Jewelry|262|ablecallyable|N/A|48979020tan288168231|slate|Gross|Unknown|23|callybarationantiought| +15707|AAAAAAAAKFNDAAAA|1999-10-28|2001-10-26|Again labour requirements cannot take merely at the ends. So financial colleagues may appear rural ex|89.77|53.86|1003002|exportiamalg #2|1|maternity|1|Women|161|oughtcallyought|medium|75papaya218891347664|steel|Tsp|Unknown|10|ationbarationantiought| +15708|AAAAAAAAKFNDAAAA|2001-10-27||Top, french grounds mean remarkable, afraid services; fresh teachers explain; close troops hesitate below humans; h|60.11|40.87|1003001|exportiamalg #1|1|maternity|1|Women|266|oughtcallyought|economy|linen897290038326511|thistle|Each|Unknown|28|eingbarationantiought| +15709|AAAAAAAANFNDAAAA|1997-10-27||Sometimes famous difficulties find possibly old times. Orders make too colleagues. Nuclear c|9.61|7.88|10014002|edu packamalgamalg #2|14|automotive|10|Electronics|107|ationbarought|N/A|5linen57204213526448|blanched|Gross|Unknown|5|n stbarationantiought| +15710|AAAAAAAAOFNDAAAA|1997-10-27|2000-10-26|Anywhere proper men will not run remarkable, revolutionary libraries. Poor rates used to hear also. Huge years see structural churches.|7.36|6.18|9006003|corpmaxi #3|6|parenting|9|Books|54|eseanti|N/A|317167727seashell017|slate|Oz|Unknown|10|baroughtationantiought| +15711|AAAAAAAAOFNDAAAA|2000-10-27||Anywhere proper men will not run remarkable, revolutionary libraries. Poor rates used to hear also. Huge years see structural churches.|8.73|4.62|8009002|maxinameless #2|9|optics|8|Sports|54|eseanti|N/A|190516645855685pink1|peru|Ounce|Unknown|50|oughtoughtationantiought| +15712|AAAAAAAAAGNDAAAA|1997-10-27|1999-10-27|Big, odd institutions c|7.41|3.18|4003001|exportiedu pack #1|3|kids|4|Shoes|11|oughtought|small|turquoise24644240639|steel|Cup|Unknown|36|ableoughtationantiought| +15713|AAAAAAAAAGNDAAAA|1999-10-28|2001-10-26|Black streets can face still passive, gre|2.26|0.72|9010006|univunivamalg #6|10|travel|9|Books|11|oughtought|N/A|311445peru8022852920|puff|Pound|Unknown|29|prioughtationantiought| +15714|AAAAAAAAAGNDAAAA|2001-10-27||Ago academic feet tell eggs. Also other children agree to th|3.14|0.72|3001001|amalgexporti #1|10|newborn|3|Children|177|ationationought|medium|046515sienna70937776|pink|N/A|Unknown|35|eseoughtationantiought| +15715|AAAAAAAADGNDAAAA|1997-10-27||Present governments cannot come also objective, particular sorts. Blue children support real roses. Ther|5.22|2.66|6009004|maxicorp #4|9|womens watch|6|Jewelry|112|ableoughtought|N/A|676511737333230lime7|green|Bunch|Unknown|35|antioughtationantiought| +15716|AAAAAAAAEGNDAAAA|1997-10-27|2000-10-26|Highly inadequate circumstances come any longer indeed effectiv|4.13|1.40|1002001|importoamalg #1|2|fragrances|1|Women|308|eingbarpri|medium|68445659421red641865|peru|Pound|Unknown|57|callyoughtationantiought| +15717|AAAAAAAAEGNDAAAA|2000-10-27||Highly inadequate circumstances come any longer indeed effectiv|0.48|1.40|10003010|exportiunivamalg #10|3|dvd/vcr players|10|Electronics|308|eingbarpri|N/A|605008antique4190655|sienna|Bundle|Unknown|83|ationoughtationantiought| +15718|AAAAAAAAGGNDAAAA|1997-10-27|1999-10-27|Churches pose rather specific, specialist degrees. Common, physical children must matter on board massive, true competitors. Perceptions confine more than legal refugees. Limited, other specialists sh|36.82|11.04|5002001|importoscholar #1|2|country|5|Music|69|n stcally|N/A|2135664015319smoke50|yellow|Pound|Unknown|7|eingoughtationantiought| +15719|AAAAAAAAGGNDAAAA|1999-10-28|2001-10-26|Churches pose rather specific, specialist degrees. Common, physical children must matter on board massive, true competitors. Perceptions confine more than legal refugees. Limited, other specialists sh|46.83|11.04|7004010|edu packbrand #10|4|curtains/drapes|7|Home|69|n stcally|N/A|2135664015319smoke50|seashell|Each|Unknown|9|n stoughtationantiought| +15720|AAAAAAAAGGNDAAAA|2001-10-27||Specific, reasonable artists would meet never large levels. Years join. Either|0.69|0.30|7004010|corpcorp #1|6|rings|6|Jewelry|108|eingbarought|N/A|2135664015319smoke50|medium|Ounce|Unknown|33|barableationantiought| +15721|AAAAAAAAJGNDAAAA|1997-10-27||Other shoulders ought to seek at a cou|30.96|17.64|7009002|maxibrand #2|9|mattresses|7|Home|233|pripriable|N/A|252698174white934408|seashell|Gross|Unknown|7|oughtableationantiought| +15722|AAAAAAAAKGNDAAAA|1997-10-27|2000-10-26|There different aspects stay often middle, special police. Molecular, scientific efforts define long without the years. Appropriate companies abide doubtless |6.99|5.87|9015003|scholarunivamalg #3|15|fiction|9|Books|554|eseantianti|N/A|44564rose17041413963|moccasin|N/A|Unknown|49|ableableationantiought| +15723|AAAAAAAAKGNDAAAA|2000-10-27||Broken, serious organizations shall not work upper games. Lines can resist please. True human|5.34|2.72|9015003|edu packimporto #2|4|sports-apparel|2|Men|554|eseantianti|medium|782powder92267282649|pale|N/A|Unknown|23|priableationantiought| +15724|AAAAAAAAMGNDAAAA|1997-10-27|1999-10-27|Comparable, domestic papers must not help vividly. Less international components learn very courts. Dull, sound units|4.09|2.45|1001001|amalgamalg #1|1|dresses|1|Women|68|eingcally|economy|13188274486782wheat6|smoke|Tsp|Unknown|35|eseableationantiought| +15725|AAAAAAAAMGNDAAAA|1999-10-28|2001-10-26|Comparable, domestic papers must not help vividly. Less international components learn very courts. Dull, sound units|9.61|2.45|1001001|scholarunivamalg #1|5|karoke|10|Electronics|267|ationcallyable|N/A|35358711white4789508|spring|Oz|Unknown|37|antiableationantiought| +15726|AAAAAAAAMGNDAAAA|2001-10-27||Comparable, domestic papers must not help vividly. Less international components learn very courts. Dull, sound units|1.10|0.40|1001001|amalgamalg #1|1|dresses|1|Women|267|ationcallyable|extra large|35358711white4789508|sienna|Box|Unknown|29|callyableationantiought| +15727|AAAAAAAAPGNDAAAA|1997-10-27||Specific months could say then. Extensive, old inches use equall|0.74|0.27|6010008|univbrand #8|10|jewelry boxes|6|Jewelry|468|eingcallyese|N/A|947851snow9934773871|wheat|Tbl|Unknown|17|ationableationantiought| +15728|AAAAAAAAAHNDAAAA|1997-10-27|2000-10-26|More diplomatic universities escape. Wealthy, constant ways turn rathe|3.24|1.00|5004001|edu packscholar #1|4|classical|5|Music|550|barantianti|N/A|6040265metallic33989|pale|Carton|Unknown|17|eingableationantiought| +15729|AAAAAAAAAHNDAAAA|2000-10-27||More diplomatic universities escape. Wealthy, constant ways turn rathe|24.58|1.00|2002002|importoimporto #2|4|shirts|2|Men|550|barantianti|medium|6040265metallic33989|indian|Bunch|Unknown|23|n stableationantiought| +15730|AAAAAAAACHNDAAAA|1997-10-27|1999-10-27|Western activiti|6.61|2.64|8009001|maxinameless #1|9|optics|8|Sports|315|antioughtpri|N/A|1526737tan7368172664|rose|Box|Unknown|53|barpriationantiought| +15731|AAAAAAAACHNDAAAA|1999-10-28|2001-10-26|Functional, criminal services discern victims. Frequently different firms ought |70.16|2.64|8009001|importoedu pack #2|2|mens|4|Shoes|315|antioughtpri|large|3772250294799rosy078|pink|Pallet|Unknown|3|oughtpriationantiought| +15732|AAAAAAAACHNDAAAA|2001-10-27||Main elements might hope even otherwise burning girls|9.24|7.85|8009001|maxiunivamalg #8|9|televisions|10|Electronics|246|antioughtpri|N/A|3772250294799rosy078|blush|Tsp|Unknown|5|ablepriationantiought| +15733|AAAAAAAAFHNDAAAA|1997-10-27||Machines cannot fit too successive levels. Inner, european eyes could call now misleading,|4.86|2.13|8014004|edu packmaxi #4|14|tennis|8|Sports|481|oughteingese|N/A|927369103466350grey3|red|Ounce|Unknown|20|pripriationantiought| +15734|AAAAAAAAGHNDAAAA|1997-10-27|2000-10-26|Rights achieve alone rights. Well digital states touch enough eyes. Attitudes can move. Ago roman schemes send for example special regulations. United companies ought to oper|2.78|2.30|1003001|exportiamalg #1|3|maternity|1|Women|38|eingpri|extra large|25rosy60753737202861|plum|Carton|Unknown|9|esepriationantiought| +15735|AAAAAAAAGHNDAAAA|2000-10-27||Rights achieve alone rights. Well digital states touch enough eyes. Attitudes can move. Ago roman schemes send for example special regulations. United companies ought to oper|2.90|2.30|1003001|exportischolar #2|3|pop|5|Music|34|esepri|N/A|25rosy60753737202861|smoke|Bunch|Unknown|16|antipriationantiought| +15736|AAAAAAAAIHNDAAAA|1997-10-27|1999-10-27|Awful components expect. Sophisticated regulations hide here nice actions. Teenage areas keep constitutional, social databases. Assistant police ought to measure again different sales. Rather|3.83|1.68|2001001|amalgimporto #1|1|accessories|2|Men|456|callyantiese|medium|7peru876008326803260|navy|Case|Unknown|17|callypriationantiought| +15737|AAAAAAAAIHNDAAAA|1999-10-28|2001-10-26|Awful components expect. Sophisticated regulations hide here nice actions. Teenage areas keep constitutional, social databases. Assistant police ought to measure again different sales. Rather|6.41|5.70|2001001|exportiexporti #2|1|toddlers|3|Children|456|callyantiese|medium|7peru876008326803260|green|Each|Unknown|24|ationpriationantiought| +15738|AAAAAAAAIHNDAAAA|2001-10-27||New courts hear wide, local figures. Authorities must appoint ahead available, fiscal members. Respectively neu|13.64|5.31|9007007|brandmaxi #7|1|reference|9|Books|666|callycallycally|N/A|tomato75135158956153|frosted|Gram|Unknown|16|eingpriationantiought| +15739|AAAAAAAALHNDAAAA|1997-10-27||Sheer, significant problems make half brown facts. Successful, good doubts secure among a sites. Good beliefs address now socially major organisations; friends must s|43.21|28.08|5003002|exportischolar #2|3|pop|5|Music|42|ableese|N/A|02545758732161peach4|ivory|Cup|Unknown|52|n stpriationantiought| +15740|AAAAAAAAMHNDAAAA|1997-10-27|2000-10-26|Additional stands cannot hear always|4.85|3.44|5004001|edu packscholar #1|4|classical|5|Music|158|eingantiought|N/A|625480red06787831223|maroon|N/A|Unknown|7|bareseationantiought| +15741|AAAAAAAAMHNDAAAA|2000-10-27||Additional stands cannot hear always|0.29|0.17|5004001|brandbrand #2|4|decor|7|Home|158|eingantiought|N/A|00557moccasin6188912|red|Box|Unknown|5|oughteseationantiought| +15742|AAAAAAAAOHNDAAAA|1997-10-27|1999-10-27|Following, usual needs would allow industrial communists. Conclusions should not manufacture afterwards right emot|9.33|5.31|3003001|exportiexporti #1|3|toddlers|3|Children|416|callyoughtese|petite|6785505851gainsboro9|hot|Lb|Unknown|36|ableeseationantiought| +15743|AAAAAAAAOHNDAAAA|1999-10-28|2001-10-26|Dominant days would work probably soon medical newspapers. Pr|2.13|1.91|3003001|univamalgamalg #2|10|memory|10|Electronics|416|callyoughtese|N/A|734156green476347556|royal|Case|Unknown|63|prieseationantiought| +15744|AAAAAAAAOHNDAAAA|2001-10-27||Dominant days would work probably soon medical newspapers. Pr|0.14|1.91|8016005|corpmaxi #5|10|golf|8|Sports|416|callyoughtese|N/A|734156green476347556|moccasin|Tsp|Unknown|16|eseeseationantiought| +15745|AAAAAAAABINDAAAA|1997-10-27||Degrees prohibit. Correct feelings welcome nowadays weekends; easily amazing divisions cannot produce much today dry minds.|2.16|1.23|3002002|importoexporti #2|2|infants|3|Children|157|ationantiought|extra large|27735navajo664232711|seashell|Pound|Unknown|36|antieseationantiought| +15746|AAAAAAAACINDAAAA|1997-10-27|2000-10-26|Only famous officers would not happen subsequently active, top shares. Dangerous problems adopt please most national recordings. Advances look then clear cats. Redundant, round sou|9.89|4.74|6001003|amalgcorp #3|1|birdal|6|Jewelry|916|callyoughtn st|N/A|4132papaya2858449447|moccasin|Unknown|Unknown|37|callyeseationantiought| +15747|AAAAAAAACINDAAAA|2000-10-27||Different budgets could think just great, national places. Easy, weak minutes become broad products; certainly early minutes can ask as well specif|66.29|4.74|6001003|namelessmaxi #4|1|romance|9|Books|539|n stprianti|N/A|282salmon71520214507|pale|Case|Unknown|61|ationeseationantiought| +15748|AAAAAAAAEINDAAAA|1997-10-27|1999-10-27|German, thin experiences will not contribute. Issues must not explain later again democr|0.70|0.23|7004009|edu packbrand #9|4|curtains/drapes|7|Home|463|pricallyese|N/A|79292084659tomato852|puff|Cup|Unknown|17|eingeseationantiought| +15749|AAAAAAAAEINDAAAA|1999-10-28|2001-10-26|German, thin experiences will not contribute. Issues must not explain later again democr|17.78|12.97|7004009|importoimporto #2|2|shirts|2|Men|629|pricallyese|small|12310968952hot322662|powder|Box|Unknown|1|n steseationantiought| +15750|AAAAAAAAEINDAAAA|2001-10-27||German, thin experiences will not contribute. Issues must not explain later again democr|4.50|3.15|6011007|amalgbrand #7|2|semi-precious|6|Jewelry|23|priable|N/A|12310968952hot322662|linen|Oz|Unknown|77|barantiationantiought| +15751|AAAAAAAAHINDAAAA|1997-10-27||New, specific students track sentences. Items mean onl|3.59|2.40|7012002|importonameless #2|12|paint|7|Home|9|n st|N/A|792969105380709plum5|rosy|Cup|Unknown|33|oughtantiationantiought| +15752|AAAAAAAAIINDAAAA|1997-10-27|2000-10-26|Frames can park highly parents. White ma|6.97|5.36|9002009|importomaxi #9|2|business|9|Books|142|ableeseought|N/A|95502royal9216476536|saddle|Tsp|Unknown|46|ableantiationantiought| +15753|AAAAAAAAIINDAAAA|2000-10-27||Frames can park highly parents. White ma|2.16|1.83|6013002|exportibrand #2|13|loose stones|6|Jewelry|142|ableeseought|N/A|6spring2110749838173|peach|Unknown|Unknown|15|priantiationantiought| +15754|AAAAAAAAKINDAAAA|1997-10-27|1999-10-27|Personal shoulders must not tell widely impressive students. So english courts grow somewhere social classes. Conditions come earlier from a |9.33|7.55|8008005|namelessnameless #5|8|outdoor|8|Sports|140|bareseought|N/A|99222slate6221454285|spring|Tbl|Unknown|100|eseantiationantiought| +15755|AAAAAAAAKINDAAAA|1999-10-28|2001-10-26|Personal shoulders must not tell widely impressive students. So english courts grow somewhere social classes. Conditions come earlier from a |4.71|3.86|9006012|corpmaxi #12|8|parenting|9|Books|140|bareseought|N/A|631violet55483803450|saddle|Unknown|Unknown|46|antiantiationantiought| +15756|AAAAAAAAKINDAAAA|2001-10-27||Personal shoulders must not tell widely impressive students. So english courts grow somewhere social classes. Conditions come earlier from a |3.00|3.86|9007001|brandmaxi #1|7|reference|9|Books|140|bareseought|N/A|9631111144317pink733|spring|Cup|Unknown|49|callyantiationantiought| +15757|AAAAAAAANINDAAAA|1997-10-27||Matters join. Securities make perfectly as a products; above important children ask as in a classes. Limitations cannot indicate already t|1.50|0.81|8009008|maxinameless #8|9|optics|8|Sports|959|n stantin st|N/A|290007563706wheat960|orchid|N/A|Unknown|14|ationantiationantiought| +15758|AAAAAAAAOINDAAAA|1997-10-27|2000-10-26|Large systems would read wildly; bad schools need workers; peculiar, written women take frequently increases. Misleading consumers shall install never ite|2.59|2.22|6006007|corpcorp #7|6|rings|6|Jewelry|639|n stprically|N/A|151pink9459152800773|salmon|Lb|Unknown|72|eingantiationantiought| +15759|AAAAAAAAOINDAAAA|2000-10-27||Only, long patients shall not examine. Limited, powerful moments should examine in the horses; common, ideal profits can support |7.46|2.22|6006007|amalgimporto #2|1|accessories|2|Men|756|callyantiation|medium|88440violet905649811|wheat|Cup|Unknown|9|n stantiationantiought| +15760|AAAAAAAAAJNDAAAA|1997-10-27|1999-10-27|Physical, political decis|6.76|4.59|9011005|amalgunivamalg #5|11|cooking|9|Books|87|ationeing|N/A|600719sandy879998603|red|Bundle|Unknown|2|barcallyationantiought| +15761|AAAAAAAAAJNDAAAA|1999-10-28|2001-10-26|Physical, political decis|6.59|4.59|9011005|amalgexporti #2|1|newborn|3|Children|471|oughtationese|extra large|snow9172442098566931|plum|Ton|Unknown|24|oughtcallyationantiought| +15762|AAAAAAAAAJNDAAAA|2001-10-27||Procedures make earlier important, environme|9.94|4.59|9011005|amalgbrand #3|1|semi-precious|6|Jewelry|471|oughtationese|N/A|snow9172442098566931|plum|Dozen|Unknown|64|ablecallyationantiought| +15763|AAAAAAAADJNDAAAA|1997-10-27||Certainly alone police see well bad, german parts. Alone police pay even. |34.44|28.92|4003002|exportiedu pack #2|3|kids|4|Shoes|782|ableeingation|small|7999324165089plum422|spring|Bundle|Unknown|30|pricallyationantiought| +15764|AAAAAAAAEJNDAAAA|1997-10-27|2000-10-26|Also public times make flat, personal instances. Almost old remains used to reverse yesterday wryly lucky |1.94|0.91|9005009|scholarmaxi #9|5|history|9|Books|375|antiationpri|N/A|5686663violet3230438|smoke|Case|Unknown|79|esecallyationantiought| +15765|AAAAAAAAEJNDAAAA|2000-10-27||Also public times make flat, personal instances. Almost old remains used to reverse yesterday wryly lucky |5.26|2.84|9005009|corpmaxi #6|16|golf|8|Sports|375|antiationpri|N/A|6131199922white20071|white|Bundle|Unknown|65|anticallyationantiought| +15766|AAAAAAAAGJNDAAAA|1997-10-27|1999-10-27|Intellectual sons ask thanks. Simple, rural players provide about interests. Irish, working balls contribute original, close groups. Too new proportions will engage however diffi|8.72|2.61|6016007|corpbrand #7|16|consignment|6|Jewelry|204|esebarable|N/A|8718526aquamarine407|wheat|Pound|Unknown|56|callycallyationantiought| +15767|AAAAAAAAGJNDAAAA|1999-10-28|2001-10-26|Anonymous hands get then. Reasonably public beliefs would go exactly. Inner, different photographs should like massive, pure yards. Strong, interna|4.01|2.84|3004002|edu packexporti #2|16|school-uniforms|3|Children|204|esebarable|medium|8718526aquamarine407|violet|Gram|Unknown|10|ationcallyationantiought| +15768|AAAAAAAAGJNDAAAA|2001-10-27||Anonymous hands get then. Reasonably public beliefs would go exactly. Inner, different photographs should like massive, pure yards. Strong, interna|3.88|2.87|1003001|exportiamalg #1|16|maternity|1|Women|301|oughtbarpri|extra large|8718526aquamarine407|wheat|Lb|Unknown|70|eingcallyationantiought| +15769|AAAAAAAAJJNDAAAA|1997-10-27||Even contemporary pieces worry accounts. Co|1.54|0.83|3004002|edu packexporti #2|4|school-uniforms|3|Children|188|eingeingought|extra large|rose2590600630750086|spring|Carton|Unknown|62|n stcallyationantiought| +15770|AAAAAAAAKJNDAAAA|1997-10-27|2000-10-26|About dangerous women used to|1.32|0.88|3001001|amalgexporti #1|1|newborn|3|Children|486|callyeingese|medium|816thistle9642680185|hot|Unknown|Unknown|22|barationationantiought| +15771|AAAAAAAAKJNDAAAA|2000-10-27||About dangerous women used to|5.05|3.53|9011004|amalgunivamalg #4|1|cooking|9|Books|486|callyeingese|N/A|816thistle9642680185|white|Cup|Unknown|81|oughtationationantiought| +15772|AAAAAAAAMJNDAAAA|1997-10-27|1999-10-27|Experienced, auto|3.95|1.30|6013005|exportibrand #5|13|loose stones|6|Jewelry|761|oughtcallyation|N/A|218757thistle9321758|light|N/A|Unknown|21|ableationationantiought| +15773|AAAAAAAAMJNDAAAA|1999-10-28|2001-10-26|Experienced, auto|70.17|1.30|6013005|edu packamalg #2|13|swimwear|1|Women|761|oughtcallyation|medium|218757thistle9321758|powder|Gram|Unknown|6|priationationantiought| +15774|AAAAAAAAMJNDAAAA|2001-10-27||Parts ought to yield neither|4.73|1.30|6013005|edu packamalgamalg #16|14|automotive|10|Electronics|322|oughtcallyation|N/A|218757thistle9321758|red|Pound|Unknown|35|eseationationantiought| +15775|AAAAAAAAPJNDAAAA|1997-10-27||Terms used to settle with the considerations; final contents would address more old agreements; areas would not get either hard, deaf heads. Successfully standard hours will reconstruct. Events|1.27|1.13|8016006|corpmaxi #6|16|golf|8|Sports|597|ationn stanti|N/A|613745puff0125080787|royal|Lb|Unknown|71|antiationationantiought| +15776|AAAAAAAAAKNDAAAA|1997-10-27|2000-10-26|Help|3.36|1.51|2004001|edu packimporto #1|4|sports-apparel|2|Men|650|barantically|petite|2647135136rosy530778|sky|Pallet|Unknown|20|callyationationantiought| +15777|AAAAAAAAAKNDAAAA|2000-10-27||Reasonable, good pictures may connect strong girls. Common movements will not see basi|1.36|1.51|2004001|exportiexporti #2|3|toddlers|3|Children|650|barantically|small|6342960533566maroon2|red|Cup|Unknown|7|ationationationantiought| +15778|AAAAAAAACKNDAAAA|1997-10-27|1999-10-27|Nice instructions will not laugh really scientific users. More temporary leaders u|1.60|0.99|9009011|maximaxi #11|9|science|9|Books|112|ableoughtought|N/A|metallic922010774370|goldenrod|Ton|Unknown|26|eingationationantiought| +15779|AAAAAAAACKNDAAAA|1999-10-28|2001-10-26|As scientific things call again occupations. Barriers see large possibilities. At random local p|3.74|0.99|6009004|maxicorp #4|9|womens watch|6|Jewelry|112|ableoughtought|N/A|metallic922010774370|powder|Oz|Unknown|22|n stationationantiought| +15780|AAAAAAAACKNDAAAA|2001-10-27||As scientific things call again occupations. Barriers see large possibilities. At random local p|4.10|0.99|6009004|edu packedu pack #1|4|athletic|4|Shoes|97|ableoughtought|extra large|95smoke1047917865183|peach|Ton|Unknown|63|bareingationantiought| +15781|AAAAAAAAFKNDAAAA|1997-10-27||Affairs take new years. Nations ought to pa|6.98|4.18|1002002|importoamalg #2|2|fragrances|1|Women|541|oughteseanti|large|54royal8627921473822|thistle|Cup|Unknown|63|oughteingationantiought| +15782|AAAAAAAAGKNDAAAA|1997-10-27|2000-10-26|Certainly just spots might stay. Ever possible person|5.60|4.08|2001001|amalgimporto #1|1|accessories|2|Men|72|ableation|extra large|4517864236595violet4|seashell|Bunch|Unknown|66|ableeingationantiought| +15783|AAAAAAAAGKNDAAAA|2000-10-27||Just careful variations use on board native advantages. Enough economic cases may settle thinking crowds. Less digital professionals will not agree only concern|89.14|61.50|1002002|importoamalg #2|1|fragrances|1|Women|911|oughtoughtn st|economy|1555816maroon5850390|pale|Ounce|Unknown|9|prieingationantiought| +15784|AAAAAAAAIKNDAAAA|1997-10-27|1999-10-27|Only, disabled bills see well small, russian tasks; annual cups place easily in a boys. Rooms might not see concerned, very pla|2.34|2.03|2001001|amalgimporto #1|1|accessories|2|Men|500|barbaranti|large|7rosy741448204920497|cyan|Carton|Unknown|16|eseeingationantiought| +15785|AAAAAAAAIKNDAAAA|1999-10-28|2001-10-26|Operations mobilize presumably ago sole sources; usual, large letters cannot mean corporate rocks. Forward possible class|7.52|3.76|2001001|importoamalg #2|2|fragrances|1|Women|500|barbaranti|large|7rosy741448204920497|red|Bundle|Unknown|40|antieingationantiought| +15786|AAAAAAAAIKNDAAAA|2001-10-27||Operations mobilize presumably ago sole sources; usual, large letters cannot mean corporate rocks. Forward possible class|2.99|1.73|2001001|edu packnameless #7|14|glassware|7|Home|500|barbaranti|N/A|479papaya58172005939|slate|Bundle|Unknown|6|callyeingationantiought| +15787|AAAAAAAALKNDAAAA|1997-10-27||Super, or|7.74|5.18|2003002|exportiimporto #2|3|pants|2|Men|665|anticallycally|medium|2784522snow486715973|sandy|Dram|Unknown|94|ationeingationantiought| +15788|AAAAAAAAMKNDAAAA|1997-10-27|2000-10-26|Special, clear elements would buy at a games. Things should spot today strange, only devices. Armies should like at a patients. Hands could perform simply narrow values. N|1.28|1.04|9002009|importomaxi #9|2|business|9|Books|278|eingationable|N/A|450purple68644486707|wheat|Ton|Unknown|98|eingeingationantiought| +15789|AAAAAAAAMKNDAAAA|2000-10-27||Special, clear elements would buy at a games. Things should spot today strange, only devices. Armies should like at a patients. Hands could perform simply narrow values. N|7.49|1.04|1003002|exportiamalg #2|3|maternity|1|Women|278|eingationable|large|634steel876688803234|maroon|Dozen|Unknown|11|n steingationantiought| +15790|AAAAAAAAOKNDAAAA|1997-10-27|1999-10-27|Numerous, retail options may pay often critical, special weapons. Important, single times attract then in the workers. Chi|1.23|0.36|6005007|scholarcorp #7|5|earings|6|Jewelry|722|ableableation|N/A|4papaya6682226855311|royal|Dozen|Unknown|6|barn stationantiought| +15791|AAAAAAAAOKNDAAAA|1999-10-28|2001-10-26|Waste, tiny studies|4.45|0.36|6005007|brandmaxi #12|7|reference|9|Books|306|callybarpri|N/A|4papaya6682226855311|violet|Tbl|Unknown|59|oughtn stationantiought| +15792|AAAAAAAAOKNDAAAA|2001-10-27||Marginal, nuclear streets may not concentrate words. White streets add occasionally i|2.41|0.36|4004001|edu packedu pack #1|7|athletic|4|Shoes|279|n stationable|economy|4papaya6682226855311|thistle|Case|Unknown|31|ablen stationantiought| +15793|AAAAAAAABLNDAAAA|1997-10-27||Other stones vanish enough chief police; small, poor quantities shall promote clearly services. British, domestic publishers shall place o|2.63|1.36|4003002|exportiedu pack #2|3|kids|4|Shoes|131|oughtpriought|petite|9436432503midnight37|powder|Bunch|Unknown|45|prin stationantiought| +15794|AAAAAAAACLNDAAAA|1997-10-27|2000-10-26|Responses treat best interesting ministers. Contemporary shows hear never only thanks. Traditi|4.57|2.05|3004001|edu packexporti #1|4|school-uniforms|3|Children|526|callyableanti|medium|1214purple0981552392|ivory|Oz|Unknown|22|esen stationantiought| +15795|AAAAAAAACLNDAAAA|2000-10-27||Responses treat best interesting ministers. Contemporary shows hear never only thanks. Traditi|5.57|2.05|3004001|exportiimporto #2|4|pants|2|Men|526|callyableanti|small|1214purple0981552392|pale|Gram|Unknown|18|antin stationantiought| +15796|AAAAAAAAELNDAAAA|1997-10-27|1999-10-27|Scores could make even commercial days; final, good studies shall look really low, fine districts. Months like even agricultural systems. Others look industrial things; bas|15.38|8.61|7014007|edu packnameless #7|14|glassware|7|Home|369|n stcallypri|N/A|9005spring4460428695|powder|Case|Unknown|14|callyn stationantiought| +15797|AAAAAAAAELNDAAAA|1999-10-28|2001-10-26|Similar instructions betray letters. Critics find; flowers indicate days. Places express c|44.51|37.38|7014007|exportiunivamalg #5|14|dvd/vcr players|10|Electronics|369|n stcallypri|N/A|7wheat40543439285176|slate|Dram|Unknown|66|ationn stationantiought| +15798|AAAAAAAAELNDAAAA|2001-10-27||Present, expected costs move flowers. New, new dealers must strike instead so additional feet. Double regulations en|0.48|37.38|3003001|exportiexporti #1|3|toddlers|3|Children|369|n stcallypri|medium|7wheat40543439285176|purple|Each|Unknown|42|eingn stationantiought| +15799|AAAAAAAAHLNDAAAA|1997-10-27||Markets must say for ever then green weeks. Better fresh forces find also similar restaurants; proposals materialise for a procedures. Here other results |2.44|1.26|9007008|brandmaxi #8|7|reference|9|Books|596|callyn stanti|N/A|23167tomato602933977|spring|Gross|Unknown|79|n stn stationantiought| +15800|AAAAAAAAILNDAAAA|1997-10-27|2000-10-26|Companies would not cut then. Hardly right steps examine more great deposits. National, practical groups would like representative, great te|6.18|4.57|4003001|exportiedu pack #1|3|kids|4|Shoes|614|eseoughtcally|economy|9523745wheat62919611|peach|Ton|Unknown|46|barbareingantiought| +15801|AAAAAAAAILNDAAAA|2000-10-27||Keen programmes emerge again different, social issues. New boys should cancel. Certain, concrete systems can embark publicly things. Local, electric areas make then true count|0.94|4.57|7013002|exportinameless #2|13|wallpaper|7|Home|614|eseoughtcally|N/A|9523745wheat62919611|powder|Tsp|Unknown|22|oughtbareingantiought| +15802|AAAAAAAAKLNDAAAA|1997-10-27|1999-10-27|Much|21.70|16.92|1003001|exportiamalg #1|3|maternity|1|Women|57|ationanti|extra large|4003854759papaya3022|yellow|Each|Unknown|9|ablebareingantiought| +15803|AAAAAAAAKLNDAAAA|1999-10-28|2001-10-26|Used regulations shop nowadays remaining, recent questions. Goods wi|1.01|16.92|1003001|amalgimporto #2|3|accessories|2|Men|158|ationanti|large|8676494spring1999462|firebrick|Carton|Unknown|3|pribareingantiought| +15804|AAAAAAAAKLNDAAAA|2001-10-27||Again civil characters mention. Already dead ties might see. Other, commercial emotions note once other members. Happy grounds listen early in a losses; f|31.93|18.51|4002001|importoedu pack #1|3|mens|4|Shoes|174|ationanti|medium|5394538779peach94769|deep|N/A|Unknown|46|esebareingantiought| +15805|AAAAAAAANLNDAAAA|1997-10-27||Perhaps guilty users go more private systems. Simply national months will eat in a colours. Clinical patients keep only in a feeling|9.21|5.71|2002002|importoimporto #2|2|shirts|2|Men|967|ationcallyn st|petite|0552663medium6827103|ghost|Each|Unknown|62|antibareingantiought| +15806|AAAAAAAAOLNDAAAA|1997-10-27|2000-10-26|Senior judges save. Possib|3.12|2.46|8016007|corpmaxi #7|16|golf|8|Sports|296|callyn stable|N/A|blush121077448942136|spring|Tsp|Unknown|62|callybareingantiought| +15807|AAAAAAAAOLNDAAAA|2000-10-27||Senior judges save. Possib|9.46|2.46|8016007|edu packmaxi #8|16|tennis|8|Sports|318|callyn stable|N/A|blush121077448942136|wheat|Oz|Unknown|32|ationbareingantiought| +15808|AAAAAAAAAMNDAAAA|1997-10-27|1999-10-27|Days might provide; distant times might ride thus despite the grounds; days deal definitely things. Only, real films sound easily able case|5.28|2.64|5003001|exportischolar #1|3|pop|5|Music|273|priationable|N/A|67royal6407986686118|rosy|Bundle|Unknown|1|eingbareingantiought| +15809|AAAAAAAAAMNDAAAA|1999-10-28|2001-10-26|Appr|8.60|2.64|9008006|namelessmaxi #6|8|romance|9|Books|537|priationable|N/A|5025480thistle018594|violet|Oz|Unknown|69|n stbareingantiought| +15810|AAAAAAAAAMNDAAAA|2001-10-27||Appr|8.61|2.66|2004001|edu packimporto #1|4|sports-apparel|2|Men|537|priationable|small|5025480thistle018594|powder|Tsp|Unknown|22|baroughteingantiought| +15811|AAAAAAAADMNDAAAA|1997-10-27||Social, good roles assist thus also appropriate troops. Trees may not pay together concerned tasks. Waiting supporters should practice only so great|8.81|5.37|10007002|brandunivamalg #2|7|personal|10|Electronics|787|ationeingation|N/A|042618970996smoke503|light|Ounce|Unknown|18|oughtoughteingantiought| +15812|AAAAAAAAEMNDAAAA|1997-10-27|2000-10-26|Standards might teach too together possible groups. Children shall exc|0.10|0.06|1002001|importoamalg #1|2|fragrances|1|Women|685|antieingcally|medium|495041390moccasin121|maroon|Bunch|Unknown|39|ableoughteingantiought| +15813|AAAAAAAAEMNDAAAA|2000-10-27||Specifically shy millions imagine behind a remains; properly likely arrangements share about foreign businesses. Less|9.18|0.06|1002001|edu packamalg #2|4|swimwear|1|Women|685|antieingcally|economy|6094744rosy130822280|snow|Dram|Unknown|39|prioughteingantiought| +15814|AAAAAAAAGMNDAAAA|1997-10-27|1999-10-27|Better elderly records fight so. Different, able activities cannot depend sure provisions. Careful, political women reinforce only with a times. Able imp|0.56|0.31|3001001|amalgexporti #1|1|newborn|3|Children|575|antiationanti|small|621352151291frosted1|sandy|Pound|Unknown|26|eseoughteingantiought| +15815|AAAAAAAAGMNDAAAA|1999-10-28|2001-10-26|Metals will not give. Movements prevent else images. Together physical rates offer etc or|1.11|0.31|3001001|amalgamalg #2|1|dresses|1|Women|575|antiationanti|small|621352151291frosted1|tan|Pound|Unknown|27|antioughteingantiought| +15816|AAAAAAAAGMNDAAAA|2001-10-27||Cultural, central parts used to accept arrangements. Successful, traditional players should includ|45.18|0.31|4002001|importoedu pack #1|2|mens|4|Shoes|98|eingn st|large|621352151291frosted1|snow|Carton|Unknown|26|callyoughteingantiought| +15817|AAAAAAAAJMNDAAAA|1997-10-27||Domestic, american days w|0.45|0.36|6009002|maxicorp #2|9|womens watch|6|Jewelry|197|ationn stought|N/A|93625947164floral155|powder|Each|Unknown|26|ationoughteingantiought| +15818|AAAAAAAAKMNDAAAA|1997-10-27|2000-10-26|Lists please particularly other characteristics. Alive reductions think afterwards. Authorities would notice on the eyes. Savings give. Authori|37.13|25.99|8007009|brandnameless #9|7|hockey|8|Sports|140|bareseought|N/A|2117792slate00915549|snow|Bunch|Unknown|51|eingoughteingantiought| +15819|AAAAAAAAKMNDAAAA|2000-10-27||Lists please particularly other characteristics. Alive reductions think afterwards. Authorities would notice on the eyes. Savings give. Authori|7.19|2.66|8007009|amalgexporti #2|1|newborn|3|Children|34|bareseought|petite|2117792slate00915549|violet|N/A|Unknown|84|n stoughteingantiought| +15820|AAAAAAAAMMNDAAAA|1997-10-27|1999-10-27|Only brown things can see difficult, soviet weekends. Ever large uses bring more for a years. Difficulties pick literally clearly other occasions. |11.64|3.72|7006001|corpbrand #1|6|rugs|7|Home|160|barcallyought|N/A|5176898mint371630017|red|Bundle|Unknown|4|barableeingantiought| +15821|AAAAAAAAMMNDAAAA|1999-10-28|2001-10-26|Narrow approaches worry other, appare|6.43|3.27|7006001|edu packbrand #2|6|curtains/drapes|7|Home|45|antiese|N/A|0958874558550wheat80|turquoise|Dozen|Unknown|6|oughtableeingantiought| +15822|AAAAAAAAMMNDAAAA|2001-10-27||Turkish, dy|1.48|3.27|7006001|amalgnameless #3|6|athletic shoes|8|Sports|826|callyableeing|N/A|821089210498cyan0206|red|Ounce|Unknown|6|ableableeingantiought| +15823|AAAAAAAAPMNDAAAA|1997-10-27||Left, minor sets used to take for a contributions. Level feet can touch there great owners. Technical campaigns mu|1.67|1.36|4003002|exportiedu pack #2|3|kids|4|Shoes|53|prianti|petite|81380244papaya982457|snow|Gross|Unknown|42|priableeingantiought| +15824|AAAAAAAAANNDAAAA|1997-10-27|2000-10-26|Likely breasts should claim straight properly limited lines. Simple, other priorities include with a coll|6.63|2.05|1002001|importoamalg #1|2|fragrances|1|Women|621|oughtablecally|medium|477843707navy8254187|frosted|Each|Unknown|52|eseableeingantiought| +15825|AAAAAAAAANNDAAAA|2000-10-27||National, moral backs exceed. Impossible names shall not enjoy. Bare, electoral differences say political dreams; stand|7.12|2.05|7013006|exportinameless #6|13|wallpaper|7|Home|20|barable|N/A|477843707navy8254187|thistle|Case|Unknown|32|antiableeingantiought| +15826|AAAAAAAACNNDAAAA|1997-10-27|1999-10-27|Automatic, southern customs launch. Kings might a|4.21|2.02|1002001|importoamalg #1|2|fragrances|1|Women|38|eingpri|economy|3787867967484slate60|smoke|Bunch|Unknown|3|callyableeingantiought| +15827|AAAAAAAACNNDAAAA|1999-10-28|2001-10-26|Automatic, southern customs launch. Kings might a|0.79|2.02|1002001|corpbrand #8|2|rugs|7|Home|38|eingpri|N/A|50447223navy08870472|puff|Pound|Unknown|15|ationableeingantiought| +15828|AAAAAAAACNNDAAAA|2001-10-27||Automatic, southern customs launch. Kings might a|0.66|0.56|6005005|scholarcorp #5|2|earings|6|Jewelry|1|eingpri|N/A|4265463457396rosy427|smoke|Carton|Unknown|18|eingableeingantiought| +15829|AAAAAAAAFNNDAAAA|1997-10-27||Impressive studies fight above individuals. Exciting, economic families look now scientists. Major, early organizations must not rise more. Global, confident |5.99|2.87|10002003|importounivamalg #3|2|camcorders|10|Electronics|62|ablecally|N/A|36529407673green4302|yellow|Bunch|Unknown|77|n stableeingantiought| +15830|AAAAAAAAGNNDAAAA|1997-10-27|2000-10-26|Documents ought to find similarly armies. Difficult women might not prevent too great e|3.89|3.11|6016007|corpbrand #7|16|consignment|6|Jewelry|850|barantieing|N/A|8933557955rosy347088|salmon|Bundle|Unknown|40|barprieingantiought| +15831|AAAAAAAAGNNDAAAA|2000-10-27||Tasks might apply very. Apparent tasks describe also extra, organic com|88.75|3.11|5003002|exportischolar #2|16|pop|5|Music|620|barantieing|N/A|11508salmon014513575|pink|Oz|Unknown|6|oughtprieingantiought| +15832|AAAAAAAAINNDAAAA|1997-10-27|1999-10-27|Strange, wrong drivers shall not provide as previous changes. Western, public units alter goals. Wide bodies become in order a little financial |5.39|4.79|4003001|exportiedu pack #1|3|kids|4|Shoes|241|oughteseable|large|636631771356885rosy4|pale|Cup|Unknown|20|ableprieingantiought| +15833|AAAAAAAAINNDAAAA|1999-10-28|2001-10-26|Strange, wrong drivers shall not provide as previous changes. Western, public units alter goals. Wide bodies become in order a little financial |3.34|1.33|5004002|edu packscholar #2|3|classical|5|Music|241|oughteseable|N/A|49567166007chiffon01|firebrick|Box|Unknown|26|priprieingantiought| +15834|AAAAAAAAINNDAAAA|2001-10-27||Informal, only services could not cause for a awards.|1.76|1.33|5004002|exportiedu pack #1|3|kids|4|Shoes|70|baration|medium|49567166007chiffon01|spring|Each|Unknown|33|eseprieingantiought| +15835|AAAAAAAALNNDAAAA|1997-10-27||Assets will not think. Civil, commercial investigations see traditional others; then parliame|4.39|3.46|6006004|corpcorp #4|6|rings|6|Jewelry|27|ationable|N/A|886482652spring92842|pale|Each|Unknown|8|antiprieingantiought| +15836|AAAAAAAAMNNDAAAA|1997-10-27|2000-10-26|Theoretical degrees sho|3.90|2.65|9006009|corpmaxi #9|6|parenting|9|Books|253|priantiable|N/A|52burlywood573717086|slate|Oz|Unknown|37|callyprieingantiought| +15837|AAAAAAAAMNNDAAAA|2000-10-27||Only temporary studies used to encourage scottish levels. Years walk other minutes. Experienced|1.33|0.70|7014008|edu packnameless #8|6|glassware|7|Home|253|priantiable|N/A|625744tomato44961712|white|Box|Unknown|34|ationprieingantiought| +15838|AAAAAAAAONNDAAAA|1997-10-27|1999-10-27|Left german goods ought to cause material, voluntary words. Always famous women turn somehow personal goals. Things will reply small minutes. Casualties pay no longer never substantial friends. Howe|1.72|1.35|3002001|importoexporti #1|2|infants|3|Children|51|oughtanti|N/A|713974white190963197|sienna|Dram|Unknown|24|eingprieingantiought| +15839|AAAAAAAAONNDAAAA|1999-10-28|2001-10-26|Medical symptoms drag. Good full winners might pursue able errors. Still obvious managers |1.84|1.56|9011012|amalgunivamalg #12|11|cooking|9|Books|267|ationcallyable|N/A|289578357872spring95|spring|Box|Unknown|18|n stprieingantiought| +15840|AAAAAAAAONNDAAAA|2001-10-27||Medical symptoms drag. Good full winners might pursue able errors. Still obvious managers |4.65|3.53|8008001|namelessnameless #1|8|outdoor|8|Sports|130|ationcallyable|N/A|289578357872spring95|lemon|Case|Unknown|46|bareseeingantiought| +15841|AAAAAAAABONDAAAA|1997-10-27||Joint, foreign relationships ring other, physical representations. Illustrations will not understand more flat pupils. Soft, grateful constraints train little, short par|0.09|0.03|9016002|corpunivamalg #2|16|mystery|9|Books|866|callycallyeing|N/A|79763866324431grey62|puff|Bundle|Unknown|42|oughteseeingantiought| +15842|AAAAAAAACONDAAAA|1997-10-27|2000-10-26|Real, necessary names display fully. Words must not get hard today polite governments. About old feelings cri|0.22|0.11|4002001|importoedu pack #1|2|mens|4|Shoes|289|n steingable|medium|7sandy66254521226401|turquoise|Ounce|Unknown|29|ableeseeingantiought| +15843|AAAAAAAACONDAAAA|2000-10-27||Real, necessary names display fully. Words must not get hard today polite governments. About old feelings cri|38.30|22.59|4002001|exportiedu pack #2|3|kids|4|Shoes|289|n steingable|economy|3139steel29496095679|moccasin|Ton|Unknown|31|prieseeingantiought| +15844|AAAAAAAAEONDAAAA|1997-10-27|1999-10-27|Possible, little men see of course to the plans. Labour, young leaders will think particularly products. Currently important problems would enable scientif|1.91|1.16|4002001|importoedu pack #1|2|mens|4|Shoes|439|n stpriese|economy|7900rosy182063609307|floral|Gross|Unknown|28|eseeseeingantiought| +15845|AAAAAAAAEONDAAAA|1999-10-28|2001-10-26|Possible, little men see of course to the plans. Labour, young leaders will think particularly products. Currently important problems would enable scientif|6.01|1.16|4002001|edu packscholar #2|2|classical|5|Music|186|n stpriese|N/A|7900rosy182063609307|tan|Tsp|Unknown|26|antieseeingantiought| +15846|AAAAAAAAEONDAAAA|2001-10-27||Possible, little men see of course to the plans. Labour, young leaders will think particularly products. Currently important problems would enable scientif|3.13|1.16|2004001|edu packimporto #1|2|sports-apparel|2|Men|491|oughtn stese|large|704833494208ivory178|sandy|Oz|Unknown|39|callyeseeingantiought| +15847|AAAAAAAAHONDAAAA|1997-10-27||Available, northern lines ensure in a plants. Situations may return powerful, reasonable projects. New committees use all. Old books run remarks. Great, ancient purpose|5.34|4.32|5003002|exportischolar #2|3|pop|5|Music|423|priableese|N/A|5171rosy512972607664|smoke|Pound|Unknown|37|ationeseeingantiought| +15848|AAAAAAAAIONDAAAA|1997-10-27|2000-10-26|Central nights shall note acutely patients. National years take about an sets. Only critical cattle press very as a effects. Most occasional devices ought to work ab|7.83|3.52|9005009|scholarmaxi #9|5|history|9|Books|391|oughtn stpri|N/A|92883orchid612708272|red|Cup|Unknown|29|eingeseeingantiought| +15849|AAAAAAAAIONDAAAA|2000-10-27||Common |8.68|5.20|2003002|exportiimporto #2|5|pants|2|Men|400|oughtn stpri|economy|92883orchid612708272|sienna|Tbl|Unknown|4|n steseeingantiought| +15850|AAAAAAAAKONDAAAA|1997-10-27|1999-10-27|Outer, centra|8.50|6.37|4002001|importoedu pack #1|2|mens|4|Shoes|340|baresepri|large|785511490peru5694433|salmon|Oz|Unknown|30|barantieingantiought| +15851|AAAAAAAAKONDAAAA|1999-10-28|2001-10-26|Other sons may not hold millions. Ends will come massive colours. Even unfair countries feel thus. Minds used to k|52.73|6.37|4002001|importoedu pack #2|2|mens|4|Shoes|340|baresepri|medium|785511490peru5694433|chocolate|Cup|Unknown|25|oughtantieingantiought| +15852|AAAAAAAAKONDAAAA|2001-10-27||Other sons may not hold millions. Ends will come massive colours. Even unfair countries feel thus. Minds used to k|6.87|4.05|8007003|brandnameless #3|7|hockey|8|Sports|340|baresepri|N/A|785511490peru5694433|magenta|Dram|Unknown|37|ableantieingantiought| +15853|AAAAAAAANONDAAAA|1997-10-27||Mass, active orders apply formally only unusual teams. Other, potential services might not ask only for|0.65|0.50|3002002|importoexporti #2|2|infants|3|Children|141|oughteseought|large|74996lime85632116332|thistle|Unknown|Unknown|24|priantieingantiought| +15854|AAAAAAAAOONDAAAA|1997-10-27|2000-10-26|So global premises fly for good. Men join territorial, dear shows. New, ltd. cases may not decide also sometimes scottish earni|5.89|3.88|8010005|univmaxi #5|10|pools|8|Sports|89|n steing|N/A|452843698113deep7053|snow|Box|Unknown|29|eseantieingantiought| +15855|AAAAAAAAOONDAAAA|2000-10-27||Critics ask video-taped, precise relations; ordinary heads would form frequently only social members; difficulties must start wrong buildings. Too imp|91.10|3.88|9004004|edu packmaxi #4|4|entertainments|9|Books|312|n steing|N/A|49022204papaya886673|pale|Each|Unknown|39|antiantieingantiought| +15856|AAAAAAAAAPNDAAAA|1997-10-27|1999-10-27|Proceedings used to take therefore public, hot grounds. Holy boys stand respectively other months. Great wou|0.82|0.71|10010013|univamalgamalg #13|10|memory|10|Electronics|247|ationeseable|N/A|34783802dodger679749|magenta|Tsp|Unknown|12|callyantieingantiought| +15857|AAAAAAAAAPNDAAAA|1999-10-28|2001-10-26|Eyes develop most great projects. Very small children could oppose also for the farmers; never blind members fetch members. English policemen can cause in a facilities. Available, huge sugge|0.37|0.71|10010013|edu packscholar #2|4|classical|5|Music|247|ationeseable|N/A|84523177710red926884|wheat|Dozen|Unknown|85|ationantieingantiought| +15858|AAAAAAAAAPNDAAAA|2001-10-27||Constantly slow notes should not take in an consumers. Civil, public animals used to h|6.44|3.47|10010013|edu packimporto #1|4|sports-apparel|2|Men|247|ationeseable|large|8301464305seashell19|plum|Gross|Unknown|41|eingantieingantiought| +15859|AAAAAAAADPNDAAAA|1997-10-27||Controversial funds dictate forward, national girls. Future, sharp years discuss special, envi|4.92|4.42|7016010|corpnameless #10|16|furniture|7|Home|342|ableesepri|N/A|9989097965thistle979|gainsboro|Ton|Unknown|24|n stantieingantiought| +15860|AAAAAAAAEPNDAAAA|1997-10-27|2000-10-26|Actual incidents improve never terrible, gentle factors. Impatie|2.63|0.92|9013009|exportiunivamalg #9|13|self-help|9|Books|360|barcallypri|N/A|777medium94384974569|almond|Gross|Unknown|28|barcallyeingantiought| +15861|AAAAAAAAEPNDAAAA|2000-10-27||Really particu|5.57|2.17|5001002|amalgscholar #2|13|rock|5|Music|360|barcallypri|N/A|777medium94384974569|peru|Dozen|Unknown|9|oughtcallyeingantiought| +15862|AAAAAAAAGPNDAAAA|1997-10-27|1999-10-27|Grateful villages used to substantiate then good, obvious letters. Extra, deaf leaders understand late prime, model allies; en|2.83|2.00|4003001|exportiedu pack #1|3|kids|4|Shoes|14|eseought|medium|22918lemon2465494508|linen|Carton|Unknown|10|ablecallyeingantiought| +15863|AAAAAAAAGPNDAAAA|1999-10-28|2001-10-26|Fingers produce always instead international years. Other |0.61|0.26|4003001|exportischolar #2|3|pop|5|Music|14|eseought|N/A|22918lemon2465494508|purple|Tsp|Unknown|51|pricallyeingantiought| +15864|AAAAAAAAGPNDAAAA|2001-10-27||Different, w|15.84|4.91|4003001|scholarnameless #5|5|fitness|8|Sports|407|ationbarese|N/A|9930152turquoise8529|spring|Tbl|Unknown|32|esecallyeingantiought| +15865|AAAAAAAAJPNDAAAA|1997-10-27||Sound sectors take far public benefits. Great prices must use in a advantages. Consumers could not cause most therefore simple thin|2.10|1.17|1003002|exportiamalg #2|3|maternity|1|Women|724|eseableation|small|66936120sky392667766|thistle|Bundle|Unknown|43|anticallyeingantiought| +15866|AAAAAAAAKPNDAAAA|1997-10-27|2000-10-26|Slowly small communicat|6.35|4.82|9012003|importounivamalg #3|12|home repair|9|Books|818|eingoughteing|N/A|050621383slate813229|royal|Lb|Unknown|8|callycallyeingantiought| +15867|AAAAAAAAKPNDAAAA|2000-10-27||Slowly small communicat|3.99|4.82|9012003|scholarnameless #8|5|fitness|8|Sports|529|n stableanti|N/A|050621383slate813229|lace|Box|Unknown|30|ationcallyeingantiought| +15868|AAAAAAAAMPNDAAAA|1997-10-27|1999-10-27|Over important allowances recommend present charges; at least philosophical equations cannot attract please steps. More early sides look permanent years. Low, civil events try also at a theori|7.59|5.46|8012009|importomaxi #9|12|guns|8|Sports|243|prieseable|N/A|83853orchid186326893|ivory|Lb|Unknown|13|eingcallyeingantiought| +15869|AAAAAAAAMPNDAAAA|1999-10-28|2001-10-26|Major, small cases can feel justly related priorities. Common, royal numbers may want also. Visitors may prevent very tightly typical houses. Other parents f|4.89|1.66|2002002|importoimporto #2|12|shirts|2|Men|243|prieseable|extra large|metallic392272900931|peru|Pallet|Unknown|24|n stcallyeingantiought| +15870|AAAAAAAAMPNDAAAA|2001-10-27||Then young regions can rot specifically to a emotions; huge relationships can know nonetheless much assistant types. Duties must get essen|3.78|2.68|2002002|exporticorp #7|3|gold|6|Jewelry|243|prieseable|N/A|metallic392272900931|pink|Gross|Unknown|43|barationeingantiought| +15871|AAAAAAAAPPNDAAAA|1997-10-27||Readers reflect; special churches love easily wild, applicable patients. Anxious, red arms ought to explore also brown, long specialist|5.30|2.06|3001002|amalgexporti #2|1|newborn|3|Children|128|eingableought|large|99168664001violet852|indian|Bundle|Unknown|15|oughtationeingantiought| +15872|AAAAAAAAAAODAAAA|1997-10-27|2000-10-26|Courses would facilitate professionally; intentionally extensive degrees ought to take in a cards. Now national reasons could get functions. Public strategies will not lift. Problems know a|9.41|8.46|4004001|edu packedu pack #1|4|athletic|4|Shoes|232|ablepriable|small|73920812679violet785|sky|Dozen|Unknown|86|ableationeingantiought| +15873|AAAAAAAAAAODAAAA|2000-10-27||Words cannot result under solely national indicators. Quick plants decide yet after a wom|6.36|3.62|4002002|importoedu pack #2|4|mens|4|Shoes|232|ablepriable|large|288red23772176323887|seashell|Unknown|Unknown|29|priationeingantiought| +15874|AAAAAAAACAODAAAA|1997-10-27|1999-10-27|Different questions talk against the prices. Even personal mice digest today options. Shor|2.73|1.93|10008014|namelessunivamalg #14|8|scanners|10|Electronics|3|pri|N/A|indian56233670662321|tan|Each|Unknown|18|eseationeingantiought| +15875|AAAAAAAACAODAAAA|1999-10-28|2001-10-26|Annual, big birds might clear computers. Important clients address later to a women. Litt|3.01|0.96|10008014|importoscholar #2|2|country|5|Music|3|pri|N/A|indian56233670662321|papaya|N/A|Unknown|9|antiationeingantiought| +15876|AAAAAAAACAODAAAA|2001-10-27||Annual, big birds might clear computers. Important clients address later to a women. Litt|1.16|0.96|10008014|importoamalg #1|2|fragrances|1|Women|212|ableoughtable|petite|indian56233670662321|wheat|Ounce|Unknown|55|callyationeingantiought| +15877|AAAAAAAAFAODAAAA|1997-10-27||Things will not compete over ever lo|2.67|1.14|2004002|edu packimporto #2|4|sports-apparel|2|Men|162|ablecallyought|economy|1antique765534348100|pale|Each|Unknown|2|ationationeingantiought| +15878|AAAAAAAAGAODAAAA|1997-10-27|2000-10-26|Wrong, good values could not work in a minds. Normal structures correct here later massive boo|4.17|1.45|4002001|importoedu pack #1|2|mens|4|Shoes|506|callybaranti|N/A|4139281081seashell91|navy|Gross|Unknown|16|eingationeingantiought| +15879|AAAAAAAAGAODAAAA|2000-10-27||Forward|7.88|6.22|3003002|exportiexporti #2|3|toddlers|3|Children|506|callybaranti|petite|4139281081seashell91|honeydew|Tsp|Unknown|34|n stationeingantiought| +15880|AAAAAAAAIAODAAAA|1997-10-27|1999-10-27|European, solid councils might oppose usually dull, busy indians; public, adequate drugs |40.11|13.23|8013001|exportimaxi #1|13|sailing|8|Sports|198|eingn stought|N/A|7522926541104tan3622|peru|Oz|Unknown|52|bareingeingantiought| +15881|AAAAAAAAIAODAAAA|1999-10-28|2001-10-26|European, solid councils might oppose usually dull, busy indians; public, adequate drugs |3.52|1.44|8013001|importomaxi #6|13|business|9|Books|198|eingn stought|N/A|26peru45479583534225|white|Box|Unknown|5|oughteingeingantiought| +15882|AAAAAAAAIAODAAAA|2001-10-27||Cold, good items borrow still massive, old opinions. Probably necessary needs recognise very then traditional movements. Days must not remove yet at a hours. Official reactions|9.90|6.03|6015003|scholarbrand #3|15|custom|6|Jewelry|198|eingn stought|N/A|26peru45479583534225|tomato|Ounce|Unknown|15|ableeingeingantiought| +15883|AAAAAAAALAODAAAA|1997-10-27||Directors can shrink by the members. Railways could view firms; recently large appointments prefer|4.56|4.05|3002002|importoexporti #2|2|infants|3|Children|708|eingbaration|extra large|8913purple6351383958|turquoise|Tsp|Unknown|30|prieingeingantiought| +15884|AAAAAAAAMAODAAAA|1997-10-27|2000-10-26|Peculiar reports stay late however apparent hands; centuries validate now small months. Only deep e|7.77|2.79|3002001|importoexporti #1|2|infants|3|Children|82|ableeing|medium|6225tan5422341068287|lace|Gross|Unknown|9|eseeingeingantiought| +15885|AAAAAAAAMAODAAAA|2000-10-27||Colleagues notice angrily witnesses. Private organs may not reduce visitors; also old pain|2.26|1.15|3002001|corpnameless #6|2|football|8|Sports|82|ableeing|N/A|05583874395seashell9|grey|Case|Unknown|43|antieingeingantiought| +15886|AAAAAAAAOAODAAAA|1997-10-27|1999-10-27|Quite economic movements shall train t|22.07|15.00|1003001|exportiamalg #1|3|maternity|1|Women|195|antin stought|small|3802933papaya6401020|sky|Each|Unknown|54|callyeingeingantiought| +15887|AAAAAAAAOAODAAAA|1999-10-28|2001-10-26|Quite economic movements shall train t|0.36|0.13|1003001|exportinameless #8|3|basketball|8|Sports|195|antin stought|N/A|3802933papaya6401020|tomato|Pound|Unknown|32|ationeingeingantiought| +15888|AAAAAAAAOAODAAAA|2001-10-27||Basic, detailed millions deal red|1.05|0.55|6010001|univbrand #1|10|jewelry boxes|6|Jewelry|195|antin stought|N/A|71674956671powder799|pink|Case|Unknown|47|eingeingeingantiought| +15889|AAAAAAAABBODAAAA|1997-10-27||Words may not deal always. Cold |3.78|2.49|5003002|exportischolar #2|3|pop|5|Music|771|oughtationation|N/A|768941431364slate951|ivory|Tsp|Unknown|20|n steingeingantiought| +15890|AAAAAAAACBODAAAA|1997-10-27|2000-10-26|Hospitals lose. Able children smoke still in the earnings. Central cases|0.86|0.57|7005001|scholarbrand #1|5|blinds/shades|7|Home|206|callybarable|N/A|3088103papaya1216036|sky|Case|Unknown|22|barn steingantiought| +15891|AAAAAAAACBODAAAA|2000-10-27||Others can initiate able units. External operations would not see later. I|1.51|0.57|7005001|maxinameless #2|5|optics|8|Sports|206|callybarable|N/A|3088103papaya1216036|steel|N/A|Unknown|59|oughtn steingantiought| +15892|AAAAAAAAEBODAAAA|1997-10-27|1999-10-27|Recent babies ex|2.10|1.82|10013015|exportiamalgamalg #15|13|stereo|10|Electronics|619|n stoughtcally|N/A|503402yellow72059537|spring|Bundle|Unknown|22|ablen steingantiought| +15893|AAAAAAAAEBODAAAA|1999-10-28|2001-10-26|Popular, video-taped letters would drag so local, international levels; twi|98.80|1.82|4002002|importoedu pack #2|2|mens|4|Shoes|619|n stoughtcally|petite|269950385royal953874|plum|Gram|Unknown|61|prin steingantiought| +15894|AAAAAAAAEBODAAAA|2001-10-27||Popular, video-taped letters would drag so local, international levels; twi|29.66|1.82|4002002|corpmaxi #7|2|parenting|9|Books|619|n stoughtcally|N/A|269950385royal953874|turquoise|Ounce|Unknown|3|esen steingantiought| +15895|AAAAAAAAHBODAAAA|1997-10-27||Beautiful bombs save safely by a authorities; even british changes feel pools. Large losses will find efforts; very gross clubs reconcile ce|53.54|33.73|10013001|exportiamalgamalg #1|13|stereo|10|Electronics|380|bareingpri|N/A|835023puff3699683286|snow|Bunch|Unknown|9|antin steingantiought| +15896|AAAAAAAAIBODAAAA|1997-10-27|2000-10-26|Words write here. Aware, secure words use in the countries; restrictions count then human, new women. Little, beautiful relationships pass angry, particular days; |4.65|3.06|6006001|corpcorp #1|6|rings|6|Jewelry|7|ation|N/A|733royal353108377806|thistle|Gross|Unknown|66|callyn steingantiought| +15897|AAAAAAAAIBODAAAA|2000-10-27||Words write here. Aware, secure words use in the countries; restrictions count then human, new women. Little, beautiful relationships pass angry, particular days; |4.65|3.20|6004002|edu packcorp #2|4|bracelets|6|Jewelry|515|antioughtanti|N/A|733royal353108377806|thistle|N/A|Unknown|57|ationn steingantiought| +15898|AAAAAAAAKBODAAAA|1997-10-27|1999-10-27|Scientific, thick studies could look precisely following, simple stages; long difficult lives could|1.21|0.64|3001001|amalgexporti #1|1|newborn|3|Children|11|oughtought|medium|12peach8502274592130|wheat|Box|Unknown|44|eingn steingantiought| +15899|AAAAAAAAKBODAAAA|1999-10-28|2001-10-26|Scientific, thick studies could look precisely following, simple stages; long difficult lives could|3.29|0.64|3001001|corpunivamalg #5|6|musical|10|Electronics|218|eingoughtable|N/A|57moccasin9108337913|magenta|Carton|Unknown|53|n stn steingantiought| +15900|AAAAAAAAKBODAAAA|2001-10-27||Scientific, thick studies could look precisely following, simple stages; long difficult lives could|8.09|0.64|3001001|importonameless #1|6|paint|7|Home|230|eingoughtable|N/A|0salmon6064175012728|orchid|Box|Unknown|2|barbarn stantiought| +15901|AAAAAAAANBODAAAA|1997-10-27||Only obvious profits produce now. Swiftly necessary times used to formulate here circles. Primary drugs inform doubtless low cases; too previous concessions pay. V|3.96|1.74|9009002|maximaxi #2|9|science|9|Books|135|antipriought|N/A|547263557731powder03|moccasin|Tbl|Unknown|27|oughtbarn stantiought| +15902|AAAAAAAAOBODAAAA|1997-10-27|2000-10-26|Abstract, red patterns return definitely police; different studies wear mainly pr|8.18|6.62|4004001|edu packedu pack #1|4|athletic|4|Shoes|359|n stantipri|small|1930137061seashell94|ivory|Bunch|Unknown|75|ablebarn stantiought| +15903|AAAAAAAAOBODAAAA|2000-10-27||Abstract, red patterns return definitely police; different studies wear mainly pr|3.45|2.44|4003002|exportiedu pack #2|4|kids|4|Shoes|359|n stantipri|large|1930137061seashell94|purple|N/A|Unknown|6|pribarn stantiought| +15904|AAAAAAAAACODAAAA|1997-10-27|1999-10-27|Satisfactory, careful ways would move however common, clear windows. Yesterday existing hours thin|6.21|4.71|7012005|importonameless #5|12|paint|7|Home|561|oughtcallyanti|N/A|7329honeydew83230454|medium|Gram|Unknown|47|esebarn stantiought| +15905|AAAAAAAAACODAAAA|1999-10-28|2001-10-26|Satisfactory, careful ways would move however common, clear windows. Yesterday existing hours thin|6.16|3.75|7012005|importocorp #2|12|diamonds|6|Jewelry|561|oughtcallyanti|N/A|0995107pink807319370|yellow|Lb|Unknown|24|antibarn stantiought| +15906|AAAAAAAAACODAAAA|2001-10-27||Satisfactory, careful ways would move however common, clear windows. Yesterday existing hours thin|0.62|0.26|5003001|exportischolar #1|3|pop|5|Music|561|oughtcallyanti|N/A|0995107pink807319370|plum|Tsp|Unknown|20|callybarn stantiought| +15907|AAAAAAAADCODAAAA|1997-10-27||Foreign, existing implications ought to ride meanwhile. Clear feet give economic authorities. At first attractive changes may make high regulations|6.76|4.73|1001002|amalgamalg #2|1|dresses|1|Women|295|antin stable|medium|tan22047256980187727|snow|Case|Unknown|34|ationbarn stantiought| +15908|AAAAAAAAECODAAAA|1997-10-27|2000-10-26|Long minutes may lead only mostly private buildings. O|0.72|0.35|7010009|univnameless #9|10|flatware|7|Home|108|eingbarought|N/A|51556143997lavender1|forest|Pallet|Unknown|19|eingbarn stantiought| +15909|AAAAAAAAECODAAAA|2000-10-27||Long minutes may lead only mostly private buildings. O|2.55|2.11|6009006|maxicorp #6|9|womens watch|6|Jewelry|108|eingbarought|N/A|51556143997lavender1|sandy|Ounce|Unknown|46|n stbarn stantiought| +15910|AAAAAAAAGCODAAAA|1997-10-27|1999-10-27|Tiles see a little. Standards say post-war|29.80|22.35|3002001|importoexporti #1|2|infants|3|Children|547|ationeseanti|medium|67937967495ivory5288|saddle|Unknown|Unknown|47|baroughtn stantiought| +15911|AAAAAAAAGCODAAAA|1999-10-28|2001-10-26|Tiles see a little. Standards say post-war|81.22|66.60|3002001|importoimporto #2|2|shirts|2|Men|547|ationeseanti|economy|7738836906174linen88|sienna|Each|Unknown|15|oughtoughtn stantiought| +15912|AAAAAAAAGCODAAAA|2001-10-27||Tiles see a little. Standards say post-war|4.52|3.43|7009003|maxibrand #3|2|mattresses|7|Home|308|ationeseanti|N/A|7738836906174linen88|white|Bundle|Unknown|44|ableoughtn stantiought| +15913|AAAAAAAAJCODAAAA|1997-10-27||Professionally alive documents examine thin, industrial pages; european, dark effects use rivers. Difficult, simple rules must build lawyers. Video-taped departments test also upp|6.86|5.41|7009004|maxibrand #4|9|mattresses|7|Home|24|eseable|N/A|458182436bisque53965|navajo|Ton|Unknown|62|prioughtn stantiought| +15914|AAAAAAAAKCODAAAA|1997-10-27|2000-10-26|Weeks used to obtain true; certain rules make with a movements. Criteria know traditional, mutual |2.38|1.64|4002001|importoedu pack #1|2|mens|4|Shoes|747|ationeseation|large|7wheat26202817510206|violet|Dozen|Unknown|62|eseoughtn stantiought| +15915|AAAAAAAAKCODAAAA|2000-10-27||Weeks used to obtain true; certain rules make with a movements. Criteria know traditional, mutual |4.92|1.64|5001002|amalgscholar #2|1|rock|5|Music|747|ationeseation|N/A|7wheat26202817510206|snow|Tbl|Unknown|97|antioughtn stantiought| +15916|AAAAAAAAMCODAAAA|1997-10-27|1999-10-27|Foreign customers believe following records. Times used to tell here strange, direct patterns. So private exp|23.48|8.45|6004005|edu packcorp #5|4|bracelets|6|Jewelry|890|barn steing|N/A|431298405943235peru2|turquoise|Bunch|Unknown|20|callyoughtn stantiought| +15917|AAAAAAAAMCODAAAA|1999-10-28|2001-10-26|Foreign customers believe following records. Times used to tell here strange, direct patterns. So private exp|3.43|8.45|6004005|importoscholar #2|4|country|5|Music|375|barn steing|N/A|431298405943235peru2|violet|N/A|Unknown|64|ationoughtn stantiought| +15918|AAAAAAAAMCODAAAA|2001-10-27||Foreign customers believe following records. Times used to tell here strange, direct patterns. So private exp|1.10|8.45|10014007|edu packamalgamalg #7|4|automotive|10|Electronics|375|barn steing|N/A|810thistle7257464120|tan|Carton|Unknown|39|eingoughtn stantiought| +15919|AAAAAAAAPCODAAAA|1997-10-27||Superior, real applications bring tonight; computers shall supply variations. Scottish, tall fingers construct also executive hundreds. Annual, pract|0.46|0.22|8005010|scholarnameless #10|5|fitness|8|Sports|727|ationableation|N/A|595maroon50819437789|sandy|Case|Unknown|28|n stoughtn stantiought| +15920|AAAAAAAAADODAAAA|1997-10-27|2000-10-26|Awful directors play. Difficult, massive feet should not help never successful metres; below difficult divisions can fin|3.58|1.64|10016009|corpamalgamalg #9|16|wireless|10|Electronics|289|n steingable|N/A|5151441white18916944|rose|Box|Unknown|3|barablen stantiought| +15921|AAAAAAAAADODAAAA|2000-10-27||Again economic calls can realize late by a men. Dark problems will attribute too conservative, economic rocks. Old, british fields pay; tragic, leve|0.16|1.64|10016009|edu packimporto #2|4|sports-apparel|2|Men|795|n steingable|medium|5151441white18916944|black|Cup|Unknown|12|oughtablen stantiought| +15922|AAAAAAAACDODAAAA|1997-10-27|1999-10-27|Middle shapes fore|7.66|2.98|3003001|exportiexporti #1|3|toddlers|3|Children|635|antiprically|petite|924peach699173476556|magenta|Pound|Unknown|11|ableablen stantiought| +15923|AAAAAAAACDODAAAA|1999-10-28|2001-10-26|Middle shapes fore|89.55|79.69|3003001|amalgexporti #2|3|newborn|3|Children|635|antiprically|medium|924peach699173476556|linen|Case|Unknown|15|priablen stantiought| +15924|AAAAAAAACDODAAAA|2001-10-27||Actual members stop mediterranean men. Moral, international strings assist constantly foods. Clear concerned movements could renew sometimes services. Right advantages ensure long main children|4.22|3.41|3003001|exportiexporti #1|3|toddlers|3|Children|842|antiprically|extra large|0brown58286820016060|spring|Pallet|Unknown|17|eseablen stantiought| +15925|AAAAAAAAFDODAAAA|1997-10-27||Military men replace well again great claims. Relationships mean problems. Goods could gather records. Masses try real possibilities. Already weak names possess as other, |2.26|1.89|10016014|corpamalgamalg #14|16|wireless|10|Electronics|266|callycallyable|N/A|0629plum929573943241|frosted|Dram|Unknown|26|antiablen stantiought| +15926|AAAAAAAAGDODAAAA|1997-10-27|2000-10-26|Other ingredients cannot look less than small shares. Authorities hang more in a m|2.55|1.70|5003001|exportischolar #1|3|pop|5|Music|180|bareingought|N/A|2931902turquoise9809|sienna|Box|Unknown|46|callyablen stantiought| +15927|AAAAAAAAGDODAAAA|2000-10-27||Really late artists weep sadly |7.17|1.70|5003001|exportiamalg #2|3|maternity|1|Women|180|bareingought|extra large|933352669271spring25|peach|N/A|Unknown|58|ationablen stantiought| +15928|AAAAAAAAIDODAAAA|1997-10-27|1999-10-27|Positive, deep pounds might trust just national, financial sheets; answers will take nice, early degrees. Very other votes ought to meet soon international various towns. Changes understand. Delib|7.72|3.47|9004005|edu packmaxi #5|4|entertainments|9|Books|856|callyantieing|N/A|9602072moccasin33161|slate|Gram|Unknown|25|eingablen stantiought| +15929|AAAAAAAAIDODAAAA|1999-10-28|2001-10-26|Positive, deep pounds might trust just national, financial sheets; answers will take nice, early degrees. Very other votes ought to meet soon international various towns. Changes understand. Delib|3.39|3.47|9004005|amalgscholar #2|1|rock|5|Music|412|ableoughtese|N/A|2547508984139pale052|yellow|Gram|Unknown|60|n stablen stantiought| +15930|AAAAAAAAIDODAAAA|2001-10-27||Proportions might turn here heads. Interested, long-term things might know even good changes. Personnel|68.10|3.47|9004005|importoscholar #1|1|country|5|Music|19|n stought|N/A|2547508984139pale052|slate|Tbl|Unknown|31|barprin stantiought| +15931|AAAAAAAALDODAAAA|1997-10-27||Then long times hope wide sole, new legs. Students might not dig more swiss, isolated children. Real words may negotiate so. Left circumstances repeat; stil|0.81|0.42|7007002|brandbrand #2|7|decor|7|Home|412|ableoughtese|N/A|2693105068blanched02|goldenrod|Dozen|Unknown|16|oughtprin stantiought| +15932|AAAAAAAAMDODAAAA|1997-10-27|2000-10-26|Major, private revenues should see there other minds; always fine articles can attack medical, new changes; most western parts could appear separate contexts; available jobs might like oth|3.93|1.21|2001001|amalgimporto #1|1|accessories|2|Men|619|n stoughtcally|medium|8360712002pink034446|papaya|Pallet|Unknown|39|ableprin stantiought| +15933|AAAAAAAAMDODAAAA|2000-10-27||Here hard hundreds go quickly major, military weeks. Entire opportunities see for instance. Voters would spend naturally profitable demands. Em|3.84|1.21|5002002|importoscholar #2|2|country|5|Music|619|n stoughtcally|N/A|purple22704513064307|salmon|Pound|Unknown|22|priprin stantiought| +15934|AAAAAAAAODODAAAA|1997-10-27|1999-10-27|Labour, vast questions used to release very black, joint movements. Eyes ought to offe|2.89|0.95|8007005|brandnameless #5|7|hockey|8|Sports|100|barbarought|N/A|moccasin631034129191|red|Box|Unknown|31|eseprin stantiought| +15935|AAAAAAAAODODAAAA|1999-10-28|2001-10-26|Shoulders used to settle additional, valid wings. Largely difficult churches must speak far extra spanish police. Characteristics laugh thus. Only large regulations might not happen too pr|5.61|0.95|8007005|univnameless #6|7|flatware|7|Home|639|barbarought|N/A|moccasin631034129191|puff|N/A|Unknown|2|antiprin stantiought| +15936|AAAAAAAAODODAAAA|2001-10-27||Shoulders used to settle additional, valid wings. Largely difficult churches must speak far extra spanish police. Characteristics laugh thus. Only large regulations might not happen too pr|5.72|0.95|8007005|edu packscholar #1|7|classical|5|Music|346|callyesepri|N/A|moccasin631034129191|yellow|Tbl|Unknown|28|callyprin stantiought| +15937|AAAAAAAABEODAAAA|1997-10-27||Absolutely |1.01|0.79|9005002|scholarmaxi #2|5|history|9|Books|833|priprieing|N/A|332632805380967sky04|tomato|Dram|Unknown|2|ationprin stantiought| +15938|AAAAAAAACEODAAAA|1997-10-27|2000-10-26|Commercial regulations shall tell free, necessary children. Effective, convincing issues aid from the students. Goods o|4.63|1.66|8004009|edu packnameless #9|4|camping|8|Sports|837|ationprieing|N/A|08371yellow677014830|puff|N/A|Unknown|21|eingprin stantiought| +15939|AAAAAAAACEODAAAA|2000-10-27||Rational interests solve at once wrong documents. Only private customs say financial, social thousands. Times die sharply. Only principal decisions wo|4.58|1.66|8004009|importoscholar #2|2|country|5|Music|15|ationprieing|N/A|847496sienna36593280|peru|Bundle|Unknown|31|n stprin stantiought| +15940|AAAAAAAAEEODAAAA|1997-10-27|1999-10-27|National, following organs try then new, unaware actions; adult institutions mean. Most necessary directors could define over. Thoroug|8.01|5.44|9014005|edu packunivamalg #5|14|sports|9|Books|146|callyeseought|N/A|sienna70305194598531|rosy|Dozen|Unknown|52|baresen stantiought| +15941|AAAAAAAAEEODAAAA|1999-10-28|2001-10-26|Available difficulties respect behind for a troubles. More upp|4.85|3.44|9014005|amalgbrand #2|1|bathroom|7|Home|241|oughteseable|N/A|031413turquoise55355|lace|Case|Unknown|8|oughtesen stantiought| +15942|AAAAAAAAEEODAAAA|2001-10-27||Scientists study almost wrong operators. Available, political contracts leave too presidential,|3.80|1.36|9014005|amalgscholar #1|1|rock|5|Music|241|oughteseable|N/A|031413turquoise55355|rosy|Bundle|Unknown|51|ableesen stantiought| +15943|AAAAAAAAHEODAAAA|1997-10-27||Confident words may enable old, human files; frequent, active patients could not give good immediately positive c|20.68|13.02|4003002|exportiedu pack #2|3|kids|4|Shoes|359|n stantipri|large|1295456244spring9740|plum|Pallet|Unknown|29|priesen stantiought| +15944|AAAAAAAAIEODAAAA|1997-10-27|2000-10-26|Likely findings can maintain suddenly into the aspects; ideas would n|8.74|3.93|9005009|scholarmaxi #9|5|history|9|Books|315|antioughtpri|N/A|832884314pink1730099|plum|Case|Unknown|6|eseesen stantiought| +15945|AAAAAAAAIEODAAAA|2000-10-27||Cheerful functions should not see local stones. Excellent, extr|7.21|4.75|10005017|scholarunivamalg #17|5|karoke|10|Electronics|315|antioughtpri|N/A|832884314pink1730099|papaya|Pallet|Unknown|8|antiesen stantiought| +15946|AAAAAAAAKEODAAAA|1997-10-27|1999-10-27|Rare rocks encourage equal, local figures. Completely open criteria could not start personal arts; backs must dea|9.65|5.30|5004001|edu packscholar #1|4|classical|5|Music|945|antiesen st|N/A|0steel09942873419420|metallic|Bunch|Unknown|23|callyesen stantiought| +15947|AAAAAAAAKEODAAAA|1999-10-28|2001-10-26|Rare rocks encourage equal, local figures. Completely open criteria could not start personal arts; backs must dea|67.89|52.95|5004001|exportiexporti #2|4|toddlers|3|Children|945|antiesen st|economy|3736463078khaki92317|pink|Bundle|Unknown|58|ationesen stantiought| +15948|AAAAAAAAKEODAAAA|2001-10-27||Rare rocks encourage equal, local figures. Completely open criteria could not start personal arts; backs must dea|8.86|7.35|1002001|importoamalg #1|2|fragrances|1|Women|292|ablen stable|extra large|7090974rose952224701|white|Pallet|Unknown|43|eingesen stantiought| +15949|AAAAAAAANEODAAAA|1997-10-27||Single, successive birds involve really in a poets. Various, public colours build over. Level, grey troops relax average, sensible clergy. Proud authorities read prayers. Stores may shoo|6.65|5.85|7001010|amalgbrand #10|1|bathroom|7|Home|119|n stoughtought|N/A|turquoise83522531038|snow|Pound|Unknown|61|n stesen stantiought| +15950|AAAAAAAAOEODAAAA|1997-10-27|2000-10-26|Acts|3.03|2.69|3003001|exportiexporti #1|3|toddlers|3|Children|642|ableesecally|large|232133764028violet99|royal|Carton|Unknown|71|barantin stantiought| +15951|AAAAAAAAOEODAAAA|2000-10-27||Acts|8.02|2.69|1002002|importoamalg #2|3|fragrances|1|Women|642|ableesecally|small|357524527271592peru1|spring|Gram|Unknown|48|oughtantin stantiought| +15952|AAAAAAAAAFODAAAA|1997-10-27|1999-10-27|Women ride still careful engines. More brave crimes must close with the knees. Ways shall consider main schools. Clearly opposite me|8.92|3.12|2003001|exportiimporto #1|3|pants|2|Men|139|n stpriought|large|396salmon71802594500|snow|Carton|Unknown|59|ableantin stantiought| +15953|AAAAAAAAAFODAAAA|1999-10-28|2001-10-26|Women ride still careful engines. More brave crimes must close with the knees. Ways shall consider main schools. Clearly opposite me|9.80|4.60|2003001|exportiexporti #2|3|toddlers|3|Children|339|n stpriought|small|5violet6947206249573|midnight|Bunch|Unknown|16|priantin stantiought| +15954|AAAAAAAAAFODAAAA|2001-10-27||Women ride still careful engines. More brave crimes must close with the knees. Ways shall consider main schools. Clearly opposite me|3.58|4.60|7006005|corpbrand #5|6|rugs|7|Home|841|oughteseeing|N/A|722838265magenta2061|snow|Each|Unknown|51|eseantin stantiought| +15955|AAAAAAAADFODAAAA|1997-10-27||Foreigners used to depart as. Left, extensive times see very again direct reasons. Open, good topics provide new, favorite response|3.90|2.57|4003002|exportiedu pack #2|3|kids|4|Shoes|836|callyprieing|petite|9thistle626164678827|thistle|Unknown|Unknown|9|antiantin stantiought| +15956|AAAAAAAAEFODAAAA|1997-10-27|2000-10-26|Principles take open knees; substantial studies can clean long doctors. Supreme women formulate then full, important plans. Nevertheless great weeks look just alone points. Months believe also f|96.59|54.09|1001001|amalgamalg #1|1|dresses|1|Women|237|ationpriable|petite|0066sienna5371461587|tan|Lb|Unknown|31|callyantin stantiought| +15957|AAAAAAAAEFODAAAA|2000-10-27||Principles take open knees; substantial studies can clean long doctors. Supreme women formulate then full, important plans. Nevertheless great weeks look just alone points. Months believe also f|3.65|1.93|1001001|exportiamalg #2|1|maternity|1|Women|237|ationpriable|extra large|03860956757011steel3|papaya|Dram|Unknown|58|ationantin stantiought| +15958|AAAAAAAAGFODAAAA|1997-10-27|1999-10-27|Current, great women can bear. Intellectual, irish others would catch as usual areas; pp. will not keep in a lives. Hours|3.86|2.97|6016007|corpbrand #7|16|consignment|6|Jewelry|446|callyeseese|N/A|orchid42152148668614|steel|Box|Unknown|62|eingantin stantiought| +15959|AAAAAAAAGFODAAAA|1999-10-28|2001-10-26|Awful homes ma|9.64|7.23|5001002|amalgscholar #2|1|rock|5|Music|446|callyeseese|N/A|845546047wheat979683|mint|Oz|Unknown|8|n stantin stantiought| +15960|AAAAAAAAGFODAAAA|2001-10-27||Moments sort more on a shares; more fi|6.91|2.21|5001002|exportischolar #1|3|pop|5|Music|446|callyeseese|N/A|21001steel4384424502|seashell|Each|Unknown|20|barcallyn stantiought| +15961|AAAAAAAAJFODAAAA|1997-10-27||For long local li|41.61|14.97|6006002|corpcorp #2|6|rings|6|Jewelry|20|barable|N/A|2526107812misty30124|tan|Cup|Unknown|34|oughtcallyn stantiought| +15962|AAAAAAAAKFODAAAA|1997-10-27|2000-10-26|Pretty biological patients catch relatively just american circumstances. Others could extend loudly offi|5.19|4.30|8015003|scholarmaxi #3|15|fishing|8|Sports|270|barationable|N/A|1266094373lace038944|powder|Ounce|Unknown|34|ablecallyn stantiought| +15963|AAAAAAAAKFODAAAA|2000-10-27||Pretty biological patients catch relatively just american circumstances. Others could extend loudly offi|2.32|1.87|1001002|amalgamalg #2|1|dresses|1|Women|493|prin stese|extra large|1266094373lace038944|peach|Cup|Unknown|7|pricallyn stantiought| +15964|AAAAAAAAMFODAAAA|1997-10-27|1999-10-27|Enough possible policemen call as racial stairs. Leve|7.89|5.99|8006005|corpnameless #5|6|football|8|Sports|88|eingeing|N/A|00319316653seashell9|snow|Tbl|Unknown|39|esecallyn stantiought| +15965|AAAAAAAAMFODAAAA|1999-10-28|2001-10-26|Enough possible policemen call as racial stairs. Leve|0.39|0.33|3001002|amalgexporti #2|1|newborn|3|Children|72|ableation|large|00319316653seashell9|white|Lb|Unknown|48|anticallyn stantiought| +15966|AAAAAAAAMFODAAAA|2001-10-27||Enough possible policemen call as racial stairs. Leve|1.93|1.67|3001002|importoscholar #1|1|country|5|Music|86|ableation|N/A|476medium53886154256|seashell|Gram|Unknown|8|callycallyn stantiought| +15967|AAAAAAAAPFODAAAA|1997-10-27||Likely actors would shed. Public, curious individuals might count never applications. New|3.77|3.20|4004002|edu packedu pack #2|4|athletic|4|Shoes|509|n stbaranti|small|thistle0451408083845|pink|Ton|Unknown|55|ationcallyn stantiought| +15968|AAAAAAAAAGODAAAA|1997-10-27|2000-10-26|Following rows might not ring real differences. Afraid children must ensure. Generous, large sheets tell there before |0.54|0.27|7001009|amalgbrand #9|1|bathroom|7|Home|73|priation|N/A|4372955351881tan5816|olive|Bundle|Unknown|2|eingcallyn stantiought| +15969|AAAAAAAAAGODAAAA|2000-10-27||Following rows might not ring real differences. Afraid children must ensure. Generous, large sheets tell there before |4.13|3.67|7001009|corpcorp #2|6|rings|6|Jewelry|34|priation|N/A|4372955351881tan5816|plum|Each|Unknown|42|n stcallyn stantiought| +15970|AAAAAAAACGODAAAA|1997-10-27|1999-10-27|Ever excellent towns used to try hard current private services. International, new minutes follow powerful recordings. Schools must not h|9.52|3.71|7010001|univnameless #1|10|flatware|7|Home|82|ableeing|N/A|556131peru6132726337|peru|Ton|Unknown|31|barationn stantiought| +15971|AAAAAAAACGODAAAA|1999-10-28|2001-10-26|Ever excellent towns used to try hard current private services. International, new minutes follow powerful recordings. Schools must not h|3.89|3.71|7010001|edu packedu pack #2|4|athletic|4|Shoes|81|oughteing|medium|069462297184magenta2|azure|Tbl|Unknown|77|oughtationn stantiought| +15972|AAAAAAAACGODAAAA|2001-10-27||Simply main attitudes must bring international, foreign pr|9.65|3.71|7010001|edu packscholar #1|4|classical|5|Music|81|oughteing|N/A|573pale6847221619637|papaya|Dram|Unknown|10|ableationn stantiought| +15973|AAAAAAAAFGODAAAA|1997-10-27||Social months comprise ago true causes; sorry flames build less. Active cells d|1.35|1.21|4003002|exportiedu pack #2|3|kids|4|Shoes|28|eingable|economy|13802192937995peach6|lemon|Ounce|Unknown|8|priationn stantiought| +15974|AAAAAAAAGGODAAAA|1997-10-27|2000-10-26|Systems must rear very conditions. Gove|2.26|1.67|2004001|edu packimporto #1|4|sports-apparel|2|Men|491|oughtn stese|extra large|7359401450smoke09929|sienna|Lb|Unknown|33|eseationn stantiought| +15975|AAAAAAAAGGODAAAA|2000-10-27||Systems must rear very conditions. Gove|12.92|1.67|10003013|exportiunivamalg #13|3|dvd/vcr players|10|Electronics|790|oughtn stese|N/A|699442485rosy9448446|dodger|Case|Unknown|20|antiationn stantiought| +15976|AAAAAAAAIGODAAAA|1997-10-27|1999-10-27|Rumours would kill so strange, dark meetings. Women might co-operate systems. Strong |50.78|38.59|4003001|exportiedu pack #1|3|kids|4|Shoes|297|ationn stable|economy|39957337056wheat4457|goldenrod|Oz|Unknown|30|callyationn stantiought| +15977|AAAAAAAAIGODAAAA|1999-10-28|2001-10-26|Rumours would kill so strange, dark meetings. Women might co-operate systems. Strong |1.97|1.12|4003001|exportiedu pack #2|3|kids|4|Shoes|297|ationn stable|small|66074770maroon102886|saddle|Dozen|Unknown|38|ationationn stantiought| +15978|AAAAAAAAIGODAAAA|2001-10-27||Sources would not say certain others. Steady, light lights operate; social, foreign cases shall not know result|0.34|0.25|4003001|exportiimporto #1|3|pants|2|Men|28|ationn stable|extra large|66074770maroon102886|violet|Box|Unknown|12|eingationn stantiought| +15979|AAAAAAAALGODAAAA|1997-10-27||Urgent plants walk also critical regions. Tremendous, practical implications will obtain open, necessary adults. Fair machines prove; direct, terrible poli|6.67|2.20|2001002|amalgimporto #2|1|accessories|2|Men|412|ableoughtese|medium|244344338puff0442592|steel|Carton|Unknown|16|n stationn stantiought| +15980|AAAAAAAAMGODAAAA|1997-10-27|2000-10-26|Local women will recognize depending on a leads. Fees might result dry, am|3.11|1.92|9008009|namelessmaxi #9|8|romance|9|Books|335|antipripri|N/A|salmon43733990689270|powder|Oz|Unknown|23|bareingn stantiought| +15981|AAAAAAAAMGODAAAA|2000-10-27||Parts cannot reduce new girls; factors would not plan since. As rural activities shall meet quite legal problems; costs could care perhaps important organisations; civil, distant attitudes shall |9.71|5.43|5004002|edu packscholar #2|4|classical|5|Music|39|n stpri|N/A|344909776misty112504|turquoise|Bunch|Unknown|29|oughteingn stantiought| +15982|AAAAAAAAOGODAAAA|1997-10-27|1999-10-27|Bizarre months furnish other, central words. Wide orders might end. Other,|2.25|1.05|9016011|corpunivamalg #11|16|mystery|9|Books|214|eseoughtable|N/A|1272996521877red4442|violet|Ton|Unknown|24|ableeingn stantiought| +15983|AAAAAAAAOGODAAAA|1999-10-28|2001-10-26|Bizarre months furnish other, central words. Wide orders might end. Other,|2.24|1.05|9016011|importoscholar #2|2|country|5|Music|214|eseoughtable|N/A|215peru7313011580980|wheat|Dram|Unknown|21|prieingn stantiought| +15984|AAAAAAAAOGODAAAA|2001-10-27||Bizarre months furnish other, central words. Wide orders might end. Other,|2.82|1.05|2004001|edu packimporto #1|4|sports-apparel|2|Men|52|eseoughtable|large|7294322steel56252289|rose|Each|Unknown|8|eseeingn stantiought| +15985|AAAAAAAABHODAAAA|1997-10-27||Successive clai|73.68|51.57|3003002|exportiexporti #2|3|toddlers|3|Children|614|eseoughtcally|medium|701661steel800626126|peach|Pound|Unknown|30|antieingn stantiought| +15986|AAAAAAAACHODAAAA|1997-10-27|2000-10-26|Successful jobs |9.97|7.77|9013003|exportiunivamalg #3|13|self-help|9|Books|307|ationbarpri|N/A|240301226wheat026886|maroon|Tbl|Unknown|4|callyeingn stantiought| +15987|AAAAAAAACHODAAAA|2000-10-27||Areas contest early wages. Agricultural, human skills could lower also musical minutes. Documents fetch instead departments. Details should give real|5.24|7.77|9013003|exportiedu pack #2|13|kids|4|Shoes|307|ationbarpri|medium|240301226wheat026886|moccasin|Unknown|Unknown|25|ationeingn stantiought| +15988|AAAAAAAAEHODAAAA|1997-10-27|1999-10-27|Interp|9.17|7.15|6013005|exportibrand #5|13|loose stones|6|Jewelry|318|eingoughtpri|N/A|6879319pale048546890|yellow|Gross|Unknown|28|eingeingn stantiought| +15989|AAAAAAAAEHODAAAA|1999-10-28|2001-10-26|Interp|9.16|8.15|8012010|importomaxi #10|13|guns|8|Sports|318|eingoughtpri|N/A|496453magenta6078774|violet|N/A|Unknown|12|n steingn stantiought| +15990|AAAAAAAAEHODAAAA|2001-10-27||Interp|0.62|0.37|10009011|maxiunivamalg #11|9|televisions|10|Electronics|318|eingoughtpri|N/A|496453magenta6078774|violet|Box|Unknown|10|barn stn stantiought| +15991|AAAAAAAAHHODAAAA|1997-10-27||Mere, special penalties provide under only general systems. Problems can grow v|1.23|0.92|1001002|amalgamalg #2|1|dresses|1|Women|128|eingableought|petite|39587447384lime83573|dodger|Case|Unknown|65|oughtn stn stantiought| +15992|AAAAAAAAIHODAAAA|1997-10-27|2000-10-26|Li|3.26|1.40|4003001|exportiedu pack #1|3|kids|4|Shoes|544|eseeseanti|medium|0884035082052grey717|red|Pallet|Unknown|16|ablen stn stantiought| +15993|AAAAAAAAIHODAAAA|2000-10-27||Li|3.29|1.74|4003001|exportiedu pack #2|3|kids|4|Shoes|544|eseeseanti|economy|0884035082052grey717|smoke|Gross|Unknown|40|prin stn stantiought| +15994|AAAAAAAAKHODAAAA|1997-10-27|1999-10-27|Only, front buildings may not achieve entire, important men. Only close trees may beat legal, similar police; elderly,|29.50|9.14|4001001|amalgedu pack #1|1|womens|4|Shoes|547|ationeseanti|medium|9491130411542purple2|ghost|Tsp|Unknown|77|esen stn stantiought| +15995|AAAAAAAAKHODAAAA|1999-10-28|2001-10-26|Other couples see as well ill serious areas. Always content eyes cut. Separate speakers mean far financial teeth. Then powerful years sh|9.81|9.14|8015006|scholarmaxi #6|1|fishing|8|Sports|713|prioughtation|N/A|9491130411542purple2|wheat|Tsp|Unknown|33|antin stn stantiought| +15996|AAAAAAAAKHODAAAA|2001-10-27||Various houses will not serve since apparent standards. Letters can run others. Interesting, aware pounds hear best later statistical years;|5.00|4.35|3001001|amalgexporti #1|1|newborn|3|Children|713|prioughtation|medium|9491130411542purple2|dark|Bunch|Unknown|16|callyn stn stantiought| +15997|AAAAAAAANHODAAAA|1997-10-27||Suppliers produce to a hours. Special, main factors will come. Old, individual recommendations see |30.34|10.92|8012008|importomaxi #8|12|guns|8|Sports|781|oughteingation|N/A|0444936gainsboro8138|dodger|Gross|Unknown|49|ationn stn stantiought| +15998|AAAAAAAAOHODAAAA|1997-10-27|2000-10-26|Massive weapons would fin|4.85|3.73|2004001|edu packimporto #1|4|sports-apparel|2|Men|36|callypri|medium|456875991699pale7399|turquoise|Bundle|Unknown|9|eingn stn stantiought| +15999|AAAAAAAAOHODAAAA|2000-10-27||Merely existing men may develop enough assistant great years. Major, difficult words might bear national, strong schools. Social, old days correspond perhaps visitors. Precisely given stu|3.92|1.56|3003002|exportiexporti #2|4|toddlers|3|Children|152|ableantiought|extra large|08801olive3599991927|yellow|Each|Unknown|57|n stn stn stantiought| +16000|AAAAAAAAAIODAAAA|1997-10-27|1999-10-27|Yards shall not think generally in a years. So late industries accord old, previous ministers. Other types could lead tonight thus right years; other problems can answer. Pat|9.47|2.84|3004001|edu packexporti #1|4|school-uniforms|3|Children|455|antiantiese|small|8wheat62557898234646|gainsboro|Ounce|Unknown|16|barbarbarcallyought| +16001|AAAAAAAAAIODAAAA|1999-10-28|2001-10-26|Yards shall not think generally in a years. So late industries accord old, previous ministers. Other types could lead tonight thus right years; other problems can answer. Pat|1.53|2.84|3004001|exportiexporti #2|4|toddlers|3|Children|68|antiantiese|petite|8wheat62557898234646|peru|Pound|Unknown|44|oughtbarbarcallyought| +16002|AAAAAAAAAIODAAAA|2001-10-27||Never visual things should see now ke|4.66|2.84|3004001|exportischolar #1|4|pop|5|Music|154|eseantiought|N/A|9089265papaya7611518|papaya|Unknown|Unknown|21|ablebarbarcallyought| +16003|AAAAAAAADIODAAAA|1997-10-27||Aware, emotional lips judge only much criminal findings. More than distinctive things must distinguish lines. Primarily established observations lead for the walls; personal, clean parents will not|4.88|4.19|5001002|amalgscholar #2|1|rock|5|Music|600|barbarcally|N/A|7637901491burnished1|magenta|Box|Unknown|45|pribarbarcallyought| +16004|AAAAAAAAEIODAAAA|1997-10-27|2000-10-26|Patient, white wounds should not take years. Artists allow also just brilliant levels. Proposals go then by a towns. Capable schools relax now bla|5.06|3.79|7010005|univnameless #5|10|flatware|7|Home|837|ationprieing|N/A|06549251steel8385512|yellow|Carton|Unknown|19|esebarbarcallyought| +16005|AAAAAAAAEIODAAAA|2000-10-27||Patient, white wounds should not take years. Artists allow also just brilliant levels. Proposals go then by a towns. Capable schools relax now bla|7.43|3.79|4004002|edu packedu pack #2|4|athletic|4|Shoes|268|ationprieing|petite|06549251steel8385512|steel|N/A|Unknown|58|antibarbarcallyought| +16006|AAAAAAAAGIODAAAA|1997-10-27|1999-10-27|Sons bother by a men; above concepts shall not bother yet wooden patterns. Earlier european others can note more into th|2.52|1.91|5001001|amalgscholar #1|1|rock|5|Music|197|ationn stought|N/A|667358465954snow5375|yellow|Cup|Unknown|17|callybarbarcallyought| +16007|AAAAAAAAGIODAAAA|1999-10-28|2001-10-26|Sons bother by a men; above concepts shall not bother yet wooden patterns. Earlier european others can note more into th|0.14|0.11|5001001|amalgamalg #2|1|dresses|1|Women|197|ationn stought|large|0773943321799red1216|saddle|Bundle|Unknown|33|ationbarbarcallyought| +16008|AAAAAAAAGIODAAAA|2001-10-27||Worthy, other days would indicate p|6.59|4.15|5001001|exportischolar #1|1|pop|5|Music|221|ationn stought|N/A|0773943321799red1216|plum|Each|Unknown|59|eingbarbarcallyought| +16009|AAAAAAAAJIODAAAA|1997-10-27||Lists cannot continue rapidly goods. Improved, original applications say on|5.47|3.50|10007013|brandunivamalg #13|7|personal|10|Electronics|690|barn stcally|N/A|17thistle05701964500|turquoise|Cup|Unknown|6|n stbarbarcallyought| +16010|AAAAAAAAKIODAAAA|1997-10-27|2000-10-26|Years lack so. Traditional interests keep separate meetings. Materi|1.60|0.94|2001001|amalgimporto #1|1|accessories|2|Men|137|ationpriought|medium|2130green93196317792|papaya|Box|Unknown|20|baroughtbarcallyought| +16011|AAAAAAAAKIODAAAA|2000-10-27||Years lack so. Traditional interests keep separate meetings. Materi|0.12|0.94|2001001|amalgimporto #2|1|accessories|2|Men|137|ationpriought|large|2130green93196317792|midnight|Case|Unknown|61|oughtoughtbarcallyought| +16012|AAAAAAAAMIODAAAA|1997-10-27|1999-10-27|Presidentia|4.24|3.13|6003005|exporticorp #5|3|gold|6|Jewelry|14|eseought|N/A|0347linen98778632930|royal|Pallet|Unknown|56|ableoughtbarcallyought| +16013|AAAAAAAAMIODAAAA|1999-10-28|2001-10-26|Presidentia|6.84|3.13|2002002|importoimporto #2|3|shirts|2|Men|14|eseought|medium|0347linen98778632930|puff|Gross|Unknown|13|prioughtbarcallyought| +16014|AAAAAAAAMIODAAAA|2001-10-27||Organisations think types; attempts announce nevertheless men. Complete, difficult results seem for a kids. Then massive sales give |2.31|3.13|9002007|importomaxi #7|2|business|9|Books|14|eseought|N/A|155510391880871navy8|thistle|Ounce|Unknown|74|eseoughtbarcallyought| +16015|AAAAAAAAPIODAAAA|1997-10-27||Perceptions throw in a times. Other fu|6.37|1.97|10011002|amalgamalgamalg #2|11|disk drives|10|Electronics|116|callyoughtought|N/A|131203085110848plum3|royal|Oz|Unknown|22|antioughtbarcallyought| +16016|AAAAAAAAAJODAAAA|1997-10-27|2000-10-26|Alternatives ought to look |5.60|1.84|5001001|amalgscholar #1|1|rock|5|Music|208|eingbarable|N/A|890217honeydew114479|spring|Dram|Unknown|13|callyoughtbarcallyought| +16017|AAAAAAAAAJODAAAA|2000-10-27||Political, welcome feelings bring today islamic things. Colourful democrats may take parameters. European skills must supply. Lectures borrow for example. Interesting, british even|1.82|1.84|1003002|exportiamalg #2|3|maternity|1|Women|646|callyesecally|medium|692488498ivory346796|sky|Each|Unknown|55|ationoughtbarcallyought| +16018|AAAAAAAACJODAAAA|1997-10-27|1999-10-27|More only homes can exceed only expensive, white aspects; afraid, old programmes get originally films. Problems must not get so. Supreme winners buy. Diverse|22.74|18.19|6014003|edu packbrand #3|14|estate|6|Jewelry|587|ationeinganti|N/A|739815769purple43115|slate|N/A|Unknown|2|eingoughtbarcallyought| +16019|AAAAAAAACJODAAAA|1999-10-28|2001-10-26|More only homes can exceed only expensive, white aspects; afraid, old programmes get originally films. Problems must not get so. Supreme winners buy. Diverse|2.59|18.19|6014003|amalgscholar #2|1|rock|5|Music|587|ationeinganti|N/A|16120snow45533051107|slate|Ton|Unknown|17|n stoughtbarcallyought| +16020|AAAAAAAACJODAAAA|2001-10-27||More only homes can exceed only expensive, white aspects; afraid, old programmes get originally films. Problems must not get so. Supreme winners buy. Diverse|4.05|18.19|9014001|edu packunivamalg #1|1|sports|9|Books|516|callyoughtanti|N/A|473039997108pale6030|tan|Unknown|Unknown|32|barablebarcallyought| +16021|AAAAAAAAFJODAAAA|1997-10-27||Even effective schools may make ways. Years raise hence main, public countries. Usual, national arguments must tend old, poor masses. Open big |3.60|2.19|7014002|edu packnameless #2|14|glassware|7|Home|145|antieseought|N/A|6554seashell74745190|purple|Pound|Unknown|62|oughtablebarcallyought| +16022|AAAAAAAAGJODAAAA|1997-10-27|2000-10-26|Feet answer high years. Somewhere white points pr|3.64|1.60|5002001|importoscholar #1|2|country|5|Music|628|eingablecally|N/A|944064704turquoise58|smoke|Dram|Unknown|16|ableablebarcallyought| +16023|AAAAAAAAGJODAAAA|2000-10-27||Periods might give details. Professional representations use yet old notes. Fat plans try just neith|7.27|1.60|9012004|importounivamalg #4|2|home repair|9|Books|628|eingablecally|N/A|420smoke912556056972|sandy|Dozen|Unknown|46|priablebarcallyought| +16024|AAAAAAAAIJODAAAA|1997-10-27|1999-10-27|Often appropriat|0.95|0.28|1001001|amalgamalg #1|1|dresses|1|Women|21|oughtable|extra large|0866877584460papaya2|blush|Oz|Unknown|38|eseablebarcallyought| +16025|AAAAAAAAIJODAAAA|1999-10-28|2001-10-26|Colleagues wear buildings. Everyday sources must sound close, future units. About white aspects could mak|0.69|0.28|1001001|univnameless #6|1|flatware|7|Home|21|oughtable|N/A|0866877584460papaya2|snow|Oz|Unknown|54|antiablebarcallyought| +16026|AAAAAAAAIJODAAAA|2001-10-27||Also special doors st|1.37|0.28|4004001|edu packedu pack #1|4|athletic|4|Shoes|394|esen stpri|petite|0866877584460papaya2|puff|Bundle|Unknown|28|callyablebarcallyought| +16027|AAAAAAAALJODAAAA|1997-10-27||As difficult calculations handle home in the acts. Active righ|13.87|4.99|4001002|amalgedu pack #2|1|womens|4|Shoes|59|n stanti|extra large|6758094beige27224454|midnight|Cup|Unknown|9|ationablebarcallyought| +16028|AAAAAAAAMJODAAAA|1997-10-27|2000-10-26|Certain, light forms work years. Political, british movements|56.28|40.52|2003001|exportiimporto #1|3|pants|2|Men|314|eseoughtpri|medium|spring02637655563742|sky|Bundle|Unknown|14|eingablebarcallyought| +16029|AAAAAAAAMJODAAAA|2000-10-27||Certain, light forms work years. Political, british movements|4.16|40.52|6014006|edu packbrand #6|3|estate|6|Jewelry|36|eseoughtpri|N/A|2spring9392955343377|saddle|Unknown|Unknown|30|n stablebarcallyought| +16030|AAAAAAAAOJODAAAA|1997-10-27|1999-10-27|Appropriate, live acts will not administer earlier ready criticisms. Old taxes ought to go resulting, empty institutions; perhaps simpl|68.36|25.97|1002001|importoamalg #1|2|fragrances|1|Women|127|ationableought|medium|213362023585767sky60|peru|Cup|Unknown|15|barpribarcallyought| +16031|AAAAAAAAOJODAAAA|1999-10-28|2001-10-26|Appropriate, live acts will not administer earlier ready criticisms. Old taxes ought to go resulting, empty institutions; perhaps simpl|1.74|25.97|5003002|exportischolar #2|3|pop|5|Music|127|ationableought|N/A|213362023585767sky60|indian|Case|Unknown|32|oughtpribarcallyought| +16032|AAAAAAAAOJODAAAA|2001-10-27||Bodies should complete more various wages; inevitable, educational songs see high|7.63|25.97|5003002|scholaramalgamalg #2|3|portable|10|Electronics|127|ationableought|N/A|213362023585767sky60|moccasin|Bunch|Unknown|6|ablepribarcallyought| +16033|AAAAAAAABKODAAAA|1997-10-27||Full, new courts should not terminate home to a notions. Yellow shoulders could examine too at last little members. Sad courses offer highest addition|4.75|3.13|5004002|edu packscholar #2|4|classical|5|Music|314|eseoughtpri|N/A|1585985745818orange9|turquoise|Ton|Unknown|46|pripribarcallyought| +16034|AAAAAAAACKODAAAA|1997-10-27|2000-10-26|Parents can see beautifully tired teachers. Elegant, uncomfortable matches save regions. Now prime kinds rep|4.71|1.60|10015004|scholaramalgamalg #4|15|portable|10|Electronics|120|barableought|N/A|68134550176792slate0|pale|Unknown|Unknown|13|esepribarcallyought| +16035|AAAAAAAACKODAAAA|2000-10-27||Parents can see beautifully tired teachers. Elegant, uncomfortable matches save regions. Now prime kinds rep|4.65|2.32|7001006|amalgbrand #6|1|bathroom|7|Home|99|barableought|N/A|68134550176792slate0|spring|Lb|Unknown|25|antipribarcallyought| +16036|AAAAAAAAEKODAAAA|1997-10-27|1999-10-27|Directly modest processes could think full|4.05|2.47|7011007|amalgnameless #7|11|accent|7|Home|116|callyoughtought|N/A|9675yellow5808051480|brown|Oz|Unknown|27|callypribarcallyought| +16037|AAAAAAAAEKODAAAA|1999-10-28|2001-10-26|Courses used to rush now yet quiet colours. Drinks would realise hours. Separate shares consider arrangements. Residential consequences see on a colours. British prices le|4.71|2.47|7011007|amalgexporti #2|1|newborn|3|Children|617|callyoughtought|economy|9675yellow5808051480|papaya|Pallet|Unknown|81|ationpribarcallyought| +16038|AAAAAAAAEKODAAAA|2001-10-27||Entire employers tell ago almost famous months. Attractive, other rooms represent reportedly. Like, wet points establish always no|2.83|1.04|2002001|importoimporto #1|2|shirts|2|Men|291|callyoughtought|large|945306pale4166298520|thistle|Lb|Unknown|6|eingpribarcallyought| +16039|AAAAAAAAHKODAAAA|1997-10-27||Small prisoners merge good, social spaces. Seats want only certain children. Current, early mechanisms wo|1.38|0.52|2002002|importoimporto #2|2|shirts|2|Men|187|ationeingought|N/A|19808641140magenta05|turquoise|Tbl|Unknown|75|n stpribarcallyought| +16040|AAAAAAAAIKODAAAA|1997-10-27|2000-10-26|Final, final wives show between an rocks. Final, local participants might sue at all blue hours. Kinds move always generally benefic|4.05|2.67|9012009|importounivamalg #9|12|home repair|9|Books|41|oughtese|N/A|seashell374015852861|purple|Each|Unknown|25|baresebarcallyought| +16041|AAAAAAAAIKODAAAA|2000-10-27||Final, final wives show between an rocks. Final, local participants might sue at all blue hours. Kinds move always generally benefic|4.34|2.67|6001002|amalgcorp #2|12|birdal|6|Jewelry|250|oughtese|N/A|seashell374015852861|royal|Pallet|Unknown|60|oughtesebarcallyought| +16042|AAAAAAAAKKODAAAA|1997-10-27|1999-10-27|American, identical directors go for a males. Units hold just. Months would go longer relationships; certain things maintain out o|4.25|3.65|3001001|amalgexporti #1|1|newborn|3|Children|513|prioughtanti|large|rose1105028242702340|spring|Carton|Unknown|19|ableesebarcallyought| +16043|AAAAAAAAKKODAAAA|1999-10-28|2001-10-26|American, identical directors go for a males. Units hold just. Months would go longer relationships; certain things maintain out o|2.21|1.17|3001002|amalgexporti #2|1|newborn|3|Children|153|priantiought|medium|836501062038thistle7|slate|Oz|Unknown|64|priesebarcallyought| +16044|AAAAAAAAKKODAAAA|2001-10-27||Beans might get static men. Please considerable horses might not celebrate cases; just clear kinds might spend here senior short feet. Roman, different reasons|1.04|0.65|8003005|exportinameless #5|3|basketball|8|Sports|153|priantiought|N/A|836501062038thistle7|saddle|Tsp|Unknown|41|eseesebarcallyought| +16045|AAAAAAAANKODAAAA|1997-10-27||Only certain creatures cater about independent issues. Over present lines might work by the personnel. Visitors scrap. Old, e|4.58|2.79|7011006|amalgnameless #6|11|accent|7|Home|109|n stbarought|N/A|1smoke43884561072685|steel|Ounce|Unknown|63|antiesebarcallyought| +16046|AAAAAAAAOKODAAAA|1997-10-27|2000-10-26|Employers work and so on|0.28|0.09|4003001|exportiedu pack #1|3|kids|4|Shoes|237|ationpriable|extra large|21steel7458398452349|purple|Oz|Unknown|68|callyesebarcallyought| +16047|AAAAAAAAOKODAAAA|2000-10-27||Hours must see telecommunications. Options would not come. Always particular arrangements welcome very medieval arts. Streets make especially male, illegal processes. Only,|2.63|0.09|6003008|exporticorp #8|3|gold|6|Jewelry|237|ationpriable|N/A|34923024262green0957|lawn|Dozen|Unknown|60|ationesebarcallyought| +16048|AAAAAAAAALODAAAA|1997-10-27|1999-10-27|Available, ready divisions free significantly responsible, bad origins.|0.54|0.30|2004001|edu packimporto #1|4|sports-apparel|2|Men|222|ableableable|medium|9196powder2897263501|drab|Oz|Unknown|7|eingesebarcallyought| +16049|AAAAAAAAALODAAAA|1999-10-28|2001-10-26|Available, ready divisions free significantly responsible, bad origins.|8.58|0.30|7014010|edu packnameless #10|14|glassware|7|Home|301|oughtbarpri|N/A|7goldenrod1937715274|red|Each|Unknown|47|n stesebarcallyought| +16050|AAAAAAAAALODAAAA|2001-10-27||Physical protests would develop competitive problems. Angry millions ought to spend about swiftly above seasons. Local months sustain however pounds. Students shal|3.49|2.75|10004003|edu packunivamalg #3|4|audio|10|Electronics|301|oughtbarpri|N/A|7goldenrod1937715274|saddle|Ton|Unknown|35|barantibarcallyought| +16051|AAAAAAAADLODAAAA|1997-10-27||Words would hear successfully unhappily external restaurants. Things must get also ready instruments. Heavy, liberal women learn just general matches. Loudly subjective schools will disturb as|7.94|5.47|8007002|brandnameless #2|7|hockey|8|Sports|259|n stantiable|N/A|7khaki84354232143799|white|Case|Unknown|46|oughtantibarcallyought| +16052|AAAAAAAAELODAAAA|1997-10-27|2000-10-26|Sure russian critics require usually groups. Strong, difficult balls get thus base men. So cold shares sati|9.75|6.33|7005003|scholarbrand #3|5|blinds/shades|7|Home|36|callypri|N/A|7854663gainsboro0879|powder|Tsp|Unknown|63|ableantibarcallyought| +16053|AAAAAAAAELODAAAA|2000-10-27||Changes would say women. Jobs would marry different, positive individuals. Films could not like also very |1.30|6.33|7005003|edu packedu pack #2|4|athletic|4|Shoes|578|eingationanti|medium|7854663gainsboro0879|rose|Bunch|Unknown|81|priantibarcallyought| +16054|AAAAAAAAGLODAAAA|1997-10-27|1999-10-27|Silently foreign years |6.21|3.72|4001001|amalgedu pack #1|1|womens|4|Shoes|958|eingantin st|economy|85996177lawn81965543|steel|Gross|Unknown|33|eseantibarcallyought| +16055|AAAAAAAAGLODAAAA|1999-10-28|2001-10-26|Silently foreign years |53.14|3.72|9015012|scholarunivamalg #12|1|fiction|9|Books|386|eingantin st|N/A|40748843485snow46670|magenta|Ton|Unknown|10|antiantibarcallyought| +16056|AAAAAAAAGLODAAAA|2001-10-27||Here human organisations may push other, necessary bodies. Also inherent sessions get over administrative, old obligations. Yet sexual questions may not choose already into a|3.50|3.15|9015012|amalgnameless #7|1|athletic shoes|8|Sports|198|eingn stought|N/A|40748843485snow46670|smoke|Box|Unknown|60|callyantibarcallyought| +16057|AAAAAAAAJLODAAAA|1997-10-27||Processes shall mean never eyes. Always scottish quarters will guarantee. More significant writers ride already successful, attractive styles. Simpl|6.90|3.10|6005002|scholarcorp #2|5|earings|6|Jewelry|134|esepriought|N/A|34wheat7656354030152|thistle|Dozen|Unknown|87|ationantibarcallyought| +16058|AAAAAAAAKLODAAAA|1997-10-27|2000-10-26|Inner, unable students would continue sexual, deep things. Managers cannot make generally; recent, big pupils need among the children. Possible, steep movem|4.42|3.44|9008003|namelessmaxi #3|8|romance|9|Books|642|ableesecally|N/A|468wheat681778037000|ghost|Cup|Unknown|88|eingantibarcallyought| +16059|AAAAAAAAKLODAAAA|2000-10-27||French, difficult facts should direct more legal, famous artists. Important details draw from a years;|0.87|0.26|9008003|edu packedu pack #2|8|athletic|4|Shoes|340|baresepri|small|24963820709738hot209|white|Ounce|Unknown|25|n stantibarcallyought| +16060|AAAAAAAAMLODAAAA|1997-10-27|1999-10-27|Ready, prospective seconds afford perhaps by a relations. Solar, senior subjects go neatly available words. Delicate, present wages go types. Visitors dress alr|93.59|43.05|5004001|edu packscholar #1|4|classical|5|Music|329|n stablepri|N/A|93860powder987408483|peru|Unknown|Unknown|53|barcallybarcallyought| +16061|AAAAAAAAMLODAAAA|1999-10-28|2001-10-26|Ready, prospective seconds afford perhaps by a relations. Solar, senior subjects go neatly available words. Delicate, present wages go types. Visitors dress alr|76.80|26.88|2003002|exportiimporto #2|4|pants|2|Men|329|n stablepri|extra large|93860powder987408483|sandy|Oz|Unknown|15|oughtcallybarcallyought| +16062|AAAAAAAAMLODAAAA|2001-10-27||Sales must not serve later global fees. Of course unusual activities pursue indeed others. Circumstances|45.45|29.54|6007007|brandcorp #7|4|pendants|6|Jewelry|426|n stablepri|N/A|93860powder987408483|chocolate|Dram|Unknown|69|ablecallybarcallyought| +16063|AAAAAAAAPLODAAAA|1997-10-27||Far classical needs advise |3.40|2.14|3003002|exportiexporti #2|3|toddlers|3|Children|52|ableanti|medium|021yellow47887486826|lawn|Bundle|Unknown|52|pricallybarcallyought| +16064|AAAAAAAAAMODAAAA|1997-10-27|2000-10-26|Civil, occupational views bring. Roles thank best for a prices. Ret|2.14|1.21|6016001|corpbrand #1|16|consignment|6|Jewelry|79|n station|N/A|35063seashell9993961|lime|Tbl|Unknown|15|esecallybarcallyought| +16065|AAAAAAAAAMODAAAA|2000-10-27||Civil, occupational views bring. Roles thank best for a prices. Ret|7.50|3.45|6016001|maxibrand #6|9|mattresses|7|Home|79|n station|N/A|6639034rose683469387|steel|Cup|Unknown|29|anticallybarcallyought| +16066|AAAAAAAACMODAAAA|1997-10-27|1999-10-27|Here other bits canc|2.06|0.96|10011002|amalgamalgamalg #2|11|disk drives|10|Electronics|859|n stantieing|N/A|93292958046536linen6|puff|Ton|Unknown|7|callycallybarcallyought| +16067|AAAAAAAACMODAAAA|1999-10-28|2001-10-26|Normal men make again feelings. Other things shall not judge thus. Benefits sort in par|2.04|0.81|10011002|edu packbrand #4|11|estate|6|Jewelry|327|ationablepri|N/A|89884003orange543603|khaki|Pallet|Unknown|49|ationcallybarcallyought| +16068|AAAAAAAACMODAAAA|2001-10-27||Occupational, increasing budgets cannot relate slo|9.89|6.92|8008009|namelessnameless #9|11|outdoor|8|Sports|385|ationablepri|N/A|0279808500883royal41|peach|Carton|Unknown|64|eingcallybarcallyought| +16069|AAAAAAAAFMODAAAA|1997-10-27||Else italian organisations c|13.13|6.30|10016005|corpamalgamalg #5|16|wireless|10|Electronics|842|ableeseeing|N/A|302533189smoke743248|dodger|Pallet|Unknown|77|n stcallybarcallyought| +16070|AAAAAAAAGMODAAAA|1997-10-27|2000-10-26|Moral matters owe generally excellent steps. Also new players sign too men. Other, technical oth|3.34|2.47|1002001|importoamalg #1|2|fragrances|1|Women|76|callyation|medium|4ivory45985396839875|maroon|Lb|Unknown|10|barationbarcallyought| +16071|AAAAAAAAGMODAAAA|2000-10-27||Clear, important networks want western, difficul|3.88|2.47|8011002|amalgmaxi #2|2|archery|8|Sports|846|callyation|N/A|4ivory45985396839875|lace|Case|Unknown|4|oughtationbarcallyought| +16072|AAAAAAAAIMODAAAA|1997-10-27|1999-10-27|Adults throw close recent women. Orange, guilty libraries let earnings. Initiatives ought to walk. Simple, successful states might work eventually full orders. Formerly very |95.98|46.07|3004001|edu packexporti #1|4|school-uniforms|3|Children|172|ableationought|N/A|6438369tan3022264813|yellow|Bundle|Unknown|13|ableationbarcallyought| +16073|AAAAAAAAIMODAAAA|1999-10-28|2001-10-26|Adults throw close recent women. Orange, guilty libraries let earnings. Initiatives ought to walk. Simple, successful states might work eventually full orders. Formerly very |3.85|46.07|3004001|edu packimporto #2|4|sports-apparel|2|Men|151|oughtantiought|small|6438369tan3022264813|wheat|Bundle|Unknown|9|priationbarcallyought| +16074|AAAAAAAAIMODAAAA|2001-10-27||Reasons hope all. Early services win almost. Funds apply sanctions; central, charming polls shall enjoy various leaders. Old affairs would not dance never electric friends. |1.30|0.91|3004001|maxicorp #3|4|womens watch|6|Jewelry|308|eingbarpri|N/A|3270steel43453186044|spring|Cup|Unknown|9|eseationbarcallyought| +16075|AAAAAAAALMODAAAA|1997-10-27||Aside smooth secrets would come both. Suddenly big officials can pay too problems; programmes seem. Unable times play. Very indian failures use s|3.03|2.69|7002006|importobrand #6|2|bedding|7|Home|281|oughteingable|N/A|649wheat202110391843|puff|Box|Unknown|26|antiationbarcallyought| +16076|AAAAAAAAMMODAAAA|1997-10-27|2000-10-26|Various, inner objects become. New feelings devote very. Contents would take there parents. International police |5.54|4.04|5003001|exportischolar #1|3|pop|5|Music|36|callypri|N/A|220771358535rosy6572|peach|Ton|Unknown|4|callyationbarcallyought| +16077|AAAAAAAAMMODAAAA|2000-10-27||Implications get only extreme instruments. Also solid|0.64|0.56|5003001|exportischolar #2|3|pop|5|Music|36|callypri|N/A|9439482394778royal56|turquoise|Carton|Unknown|17|ationationbarcallyought| +16078|AAAAAAAAOMODAAAA|1997-10-27|1999-10-27|General, wooden examinations offer other, private boys. Back|1.27|1.14|4003001|exportiedu pack #1|3|kids|4|Shoes|528|eingableanti|extra large|smoke106358474093455|red|Ton|Unknown|95|eingationbarcallyought| +16079|AAAAAAAAOMODAAAA|1999-10-28|2001-10-26|General, wooden examinations offer other, private boys. Back|0.82|1.14|3002002|importoexporti #2|3|infants|3|Children|528|eingableanti|extra large|smoke106358474093455|peach|Oz|Unknown|2|n stationbarcallyought| +16080|AAAAAAAAOMODAAAA|2001-10-27||General, wooden examinations offer other, private boys. Back|9.14|8.13|3002002|importoedu pack #1|2|mens|4|Shoes|238|eingpriable|medium|smoke106358474093455|grey|Pound|Unknown|29|bareingbarcallyought| +16081|AAAAAAAABNODAAAA|1997-10-27||Able, notable guidelines see. Away visual structures may not lead previous, hi|6.00|4.02|6002002|importocorp #2|2|diamonds|6|Jewelry|487|ationeingese|N/A|991navy6216659126474|honeydew|Gram|Unknown|5|oughteingbarcallyought| +16082|AAAAAAAACNODAAAA|1997-10-27|2000-10-26|Home current books extend then bloody alleged schools. Likely, like wives provide down the years. Public, eager children shall speak about potential sums. Equally special titles sleep large, |4.73|2.60|1004001|edu packamalg #1|4|swimwear|1|Women|586|callyeinganti|petite|salmon72553268920758|violet|Lb|Unknown|22|ableeingbarcallyought| +16083|AAAAAAAACNODAAAA|2000-10-27||Local, good causes could treat. |0.74|0.48|1004001|edu packamalg #2|4|swimwear|1|Women|586|callyeinganti|large|48283822purple064689|tomato|Ton|Unknown|59|prieingbarcallyought| +16084|AAAAAAAAENODAAAA|1997-10-27|1999-10-27|Accordingly different fai|4.01|2.44|1001001|amalgamalg #1|1|dresses|1|Women|84|eseeing|medium|pale5232439481030292|violet|Ton|Unknown|22|eseeingbarcallyought| +16085|AAAAAAAAENODAAAA|1999-10-28|2001-10-26|Accordingly different fai|1.12|2.44|1004002|edu packamalg #2|4|swimwear|1|Women|545|antieseanti|N/A|0684069snow778617357|rosy|Bunch|Unknown|18|antieingbarcallyought| +16086|AAAAAAAAENODAAAA|2001-10-27||Very full papers ought to picture dark mi|1.24|2.44|1004002|importoamalg #1|4|fragrances|1|Women|58|antieseanti|medium|89pale09706667449074|drab|Carton|Unknown|13|callyeingbarcallyought| +16087|AAAAAAAAHNODAAAA|1997-10-27||Still glorious lines might not appropriate free limits. Important, international parties explain general, brilliant agents. Disabled decades struggl|6.43|2.76|5004002|edu packscholar #2|4|classical|5|Music|622|ableablecally|N/A|48830354841lavender9|snow|Pound|Unknown|58|ationeingbarcallyought| +16088|AAAAAAAAINODAAAA|1997-10-27|2000-10-26|Silly, electronic institutions shout perhaps only, free networks. Underlying r|1.42|0.58|10003007|exportiunivamalg #7|3|dvd/vcr players|10|Electronics|320|barablepri|N/A|peach237176413763284|powder|Lb|Unknown|26|eingeingbarcallyought| +16089|AAAAAAAAINODAAAA|2000-10-27||Silly, electronic institutions shout perhaps only, free networks. Underlying r|1.90|0.72|5002002|importoscholar #2|3|country|5|Music|320|barablepri|N/A|73275756067tomato497|pale|Ton|Unknown|20|n steingbarcallyought| +16090|AAAAAAAAKNODAAAA|1997-10-27|1999-10-27|Even professional f|5.88|3.76|5004001|edu packscholar #1|4|classical|5|Music|116|callyoughtought|N/A|42sky836122595294507|puff|Bunch|Unknown|28|barn stbarcallyought| +16091|AAAAAAAAKNODAAAA|1999-10-28|2001-10-26|Even professional f|5.96|3.76|5004001|brandunivamalg #10|7|personal|10|Electronics|116|callyoughtought|N/A|297white240372331148|rose|Lb|Unknown|63|oughtn stbarcallyought| +16092|AAAAAAAAKNODAAAA|2001-10-27||Even professional f|4.46|2.40|2003001|exportiimporto #1|7|pants|2|Men|266|callyoughtought|petite|297white240372331148|white|Dram|Unknown|31|ablen stbarcallyought| +16093|AAAAAAAANNODAAAA|1997-10-27||Businesses profit probably monetary neighbours. Too important members would produce. Careful tales used to believe far, primary plans. Workers accept again |4.52|1.71|7014004|edu packnameless #4|14|glassware|7|Home|523|priableanti|N/A|5799347029orchid9873|peru|Pound|Unknown|11|prin stbarcallyought| +16094|AAAAAAAAONODAAAA|1997-10-27|2000-10-26|Applications reverse as requirements. Alternative, old motives might get ago odds. Areas comply firms. Unable plates introduce with a hearts. Light member|8.29|7.37|3003001|exportiexporti #1|3|toddlers|3|Children|612|ableoughtcally|small|29saddle893540052888|saddle|Dozen|Unknown|35|esen stbarcallyought| +16095|AAAAAAAAONODAAAA|2000-10-27||Applications reverse as requirements. Alternative, old motives might get ago odds. Areas comply firms. Unable plates introduce with a hearts. Light member|2.94|7.37|4002002|importoedu pack #2|3|mens|4|Shoes|612|ableoughtcally|extra large|444papaya99564639017|coral|Pound|Unknown|20|antin stbarcallyought| +16096|AAAAAAAAAOODAAAA|1997-10-27|1999-10-27|Digita|98.92|73.20|7010007|univnameless #7|10|flatware|7|Home|205|antibarable|N/A|289069554navajo12147|cyan|Ton|Unknown|16|callyn stbarcallyought| +16097|AAAAAAAAAOODAAAA|1999-10-28|2001-10-26|Digita|8.76|6.92|3002002|importoexporti #2|2|infants|3|Children|205|antibarable|small|289069554navajo12147|indian|Tbl|Unknown|6|ationn stbarcallyought| +16098|AAAAAAAAAOODAAAA|2001-10-27||Digita|3.86|1.50|7013009|exportinameless #9|2|wallpaper|7|Home|205|antibarable|N/A|65575451salmon953126|spring|Cup|Unknown|14|eingn stbarcallyought| +16099|AAAAAAAADOODAAAA|1997-10-27||Major feet must prevent other, able problems. Provisions attract. Daughters accept in pri|2.06|0.80|9006008|corpmaxi #8|6|parenting|9|Books|20|barable|N/A|596334797plum2626097|red|Case|Unknown|12|n stn stbarcallyought| +16100|AAAAAAAAEOODAAAA|1997-10-27|2000-10-26|Numbers put otherwise increased enemies. Pub|9.25|3.51|2004001|edu packimporto #1|4|sports-apparel|2|Men|290|barn stable|large|6violet2753647158718|linen|N/A|Unknown|5|barbaroughtcallyought| +16101|AAAAAAAAEOODAAAA|2000-10-27||Numbers put otherwise increased enemies. Pub|0.96|0.83|2004001|edu packbrand #2|4|curtains/drapes|7|Home|290|barn stable|N/A|5turquoise7499308373|violet|Case|Unknown|50|oughtbaroughtcallyought| +16102|AAAAAAAAGOODAAAA|1997-10-27|1999-10-27|Emotional, overseas couples should respond either political losses. Quiet materials explore at the men. Commonly single efforts give by a miles. In order statutory homes cannot lead. Ago|9.88|7.50|4004001|edu packedu pack #1|4|athletic|4|Shoes|33|pripri|extra large|433orange76959521033|medium|Box|Unknown|32|ablebaroughtcallyought| +16103|AAAAAAAAGOODAAAA|1999-10-28|2001-10-26|Authorities leave therefore little, influential authorities. Children enable minutes. Questions cover too extended difficulties. Below national minutes may not examine uni|10.74|7.50|4004001|exportimaxi #12|3|computers|9|Books|140|pripri|N/A|433orange76959521033|rose|Carton|Unknown|15|pribaroughtcallyought| +16104|AAAAAAAAGOODAAAA|2001-10-27||Authorities leave therefore little, influential authorities. Children enable minutes. Questions cover too extended difficulties. Below national minutes may not examine uni|9.05|7.50|4004001|edu packamalg #1|4|swimwear|1|Women|585|pripri|petite|12467348304seashell8|royal|Case|Unknown|22|esebaroughtcallyought| +16105|AAAAAAAAJOODAAAA|1997-10-27||Likely personnel will not produce in an guidelines; freely tory sanctions give most pp.. Cases may let never players. Appropriate,|3.77|1.88|8012006|importomaxi #6|12|guns|8|Sports|839|n stprieing|N/A|355989214154purple41|mint|Lb|Unknown|51|antibaroughtcallyought| +16106|AAAAAAAAKOODAAAA|1997-10-27|2000-10-26|New, black vehicles promote at a travellers. Now local doctors will force already even increasing actions. British cases must get later far, tremendous effects. Ove|2.30|2.04|1003001|exportiamalg #1|3|maternity|1|Women|242|ableeseable|N/A|641violet25965116082|dodger|Oz|Unknown|98|callybaroughtcallyought| +16107|AAAAAAAAKOODAAAA|2000-10-27||Formal eyes ought to exceed yet as detailed individuals; rights implement maximum, whole citizens; resources last fat, political things; calls fit ideally together or|5.28|2.04|1003001|namelessmaxi #4|3|romance|9|Books|242|ableeseable|N/A|641violet25965116082|gainsboro|Tsp|Unknown|64|ationbaroughtcallyought| +16108|AAAAAAAAMOODAAAA|1997-10-27|1999-10-27|Full, following books merge alive, urban farms. Boys take certainly eventually future trees|4.69|3.09|7011009|amalgnameless #9|11|accent|7|Home|132|ablepriought|N/A|139535341lemon994696|seashell|Bundle|Unknown|34|eingbaroughtcallyought| +16109|AAAAAAAAMOODAAAA|1999-10-28|2001-10-26|Full, following books merge alive, urban farms. Boys take certainly eventually future trees|0.22|0.11|4001002|amalgedu pack #2|11|womens|4|Shoes|300|barbarpri|extra large|62280263turquoise260|purple|Gross|Unknown|33|n stbaroughtcallyought| +16110|AAAAAAAAMOODAAAA|2001-10-27||Factors ought to complicate both names; distinctive areas renew then grounds. Estimates must not sit unfo|4.36|1.43|4001002|maximaxi #7|9|science|9|Books|300|barbarpri|N/A|14chartreuse58306160|wheat|Dozen|Unknown|44|baroughtoughtcallyought| +16111|AAAAAAAAPOODAAAA|1997-10-27||Presidential, open books shall not recognize merely fair styles. Signs check most happy, similar rules. Fat demands must see blac|6.91|2.14|7002002|importobrand #2|2|bedding|7|Home|195|antin stought|N/A|2940634563linen09771|pink|Pound|Unknown|27|oughtoughtoughtcallyought| +16112|AAAAAAAAAPODAAAA|1997-10-27|2000-10-26|Badly outside representatives should tax original decisions. Signs trust well front, literary laws. Social sizes can involve for example to a effects. Lists might appear small purposes. Conv|4.64|2.04|10002014|importounivamalg #14|2|camcorders|10|Electronics|409|n stbarese|N/A|96108813773452white5|lace|Case|Unknown|29|ableoughtoughtcallyought| +16113|AAAAAAAAAPODAAAA|2000-10-27||Badly outside representatives should tax original decisions. Signs trust well front, literary laws. Social sizes can involve for example to a effects. Lists might appear small purposes. Conv|1.48|2.04|10002014|amalgmaxi #10|1|arts|9|Books|12|n stbarese|N/A|597124puff7400817017|sandy|N/A|Unknown|30|prioughtoughtcallyought| +16114|AAAAAAAACPODAAAA|1997-10-27|1999-10-27|Cells stay economic, thin members. Soon special conservatives solve to the figu|2.93|2.28|9002011|importomaxi #11|2|business|9|Books|313|prioughtpri|N/A|439184222451669puff9|ghost|Dram|Unknown|61|eseoughtoughtcallyought| +16115|AAAAAAAACPODAAAA|1999-10-28|2001-10-26|Cells stay economic, thin members. Soon special conservatives solve to the figu|2.89|1.35|3003002|exportiexporti #2|2|toddlers|3|Children|670|prioughtpri|large|790436944red46123690|sandy|Dram|Unknown|45|antioughtoughtcallyought| +16116|AAAAAAAACPODAAAA|2001-10-27||Cells stay economic, thin members. Soon special conservatives solve to the figu|0.34|1.35|3004001|edu packexporti #1|4|school-uniforms|3|Children|988|prioughtpri|medium|0pale623584907920502|wheat|Box|Unknown|57|callyoughtoughtcallyought| +16117|AAAAAAAAFPODAAAA|1997-10-27||More italian ways must experience even here eligible enemies. American enquiries work sometimes. Then mere features must not adopt easy crimes. Able, royal lives notice there working-class minutes.|9.06|6.79|5004002|edu packscholar #2|4|classical|5|Music|267|ationcallyable|N/A|81506693875rosy38409|sienna|N/A|Unknown|39|ationoughtoughtcallyought| +16118|AAAAAAAAGPODAAAA|1997-10-27|2000-10-26|Also long subjects know sentences. Real developments should not want just with the weeks; massive, important others|7.39|5.54|2002001|importoimporto #1|2|shirts|2|Men|295|antin stable|large|49579310050sky291862|wheat|Pallet|Unknown|85|eingoughtoughtcallyought| +16119|AAAAAAAAGPODAAAA|2000-10-27||Human eyes should not raise; ports may not look most for a students; common, int|1.03|5.54|2003002|exportiimporto #2|3|pants|2|Men|295|antin stable|extra large|49579310050sky291862|plum|N/A|Unknown|45|n stoughtoughtcallyought| +16120|AAAAAAAAIPODAAAA|1997-10-27|1999-10-27|Houses tell too strong things. Instances could not provide only. Lights like well pr|4.57|2.74|1002001|importoamalg #1|2|fragrances|1|Women|89|n steing|large|60752012purple669064|puff|Oz|Unknown|17|barableoughtcallyought| +16121|AAAAAAAAIPODAAAA|1999-10-28|2001-10-26|Gradually local individuals might not wish; all similar hours might interfere only dead human friends; build|3.24|2.59|1002001|importoexporti #2|2|infants|3|Children|89|n steing|small|7553928puff754380528|white|Ton|Unknown|13|oughtableoughtcallyought| +16122|AAAAAAAAIPODAAAA|2001-10-27||Gradually local individuals might not wish; all similar hours might interfere only dead human friends; build|3.09|1.51|6011003|amalgbrand #3|11|semi-precious|6|Jewelry|259|n stantiable|N/A|7553928puff754380528|slate|Bunch|Unknown|28|ableableoughtcallyought| +16123|AAAAAAAALPODAAAA|1997-10-27||Radical, common friends make even new inches; insufficient, objective photographs shall stop |0.39|0.26|10012008|importoamalgamalg #8|12|monitors|10|Electronics|323|priablepri|N/A|2058549grey943196975|lawn|Pound|Unknown|33|priableoughtcallyought| +16124|AAAAAAAAMPODAAAA|1997-10-27|2000-10-26|British, different birds shall cultivate too; results discover ideas. British colonies apply entirely|0.48|0.43|2003001|exportiimporto #1|3|pants|2|Men|177|ationationought|medium|71264491972598peru37|honeydew|Ounce|Unknown|16|eseableoughtcallyought| +16125|AAAAAAAAMPODAAAA|2000-10-27||Children might encourage other duties; major balls could seem additional, other methods; local teams know social, vocational students. Apparently manufacturing references kn|42.18|22.77|2003001|importocorp #6|2|diamonds|6|Jewelry|500|barbaranti|N/A|35090rosy88989296497|seashell|Tbl|Unknown|63|antiableoughtcallyought| +16126|AAAAAAAAOPODAAAA|1997-10-27|1999-10-27|Traditional, growing restrictions keep at all ethnic, fixed sectors. Police could seek always |9.91|5.84|4002001|importoedu pack #1|2|mens|4|Shoes|879|n stationeing|petite|1233seashell62512045|royal|Lb|Unknown|14|callyableoughtcallyought| +16127|AAAAAAAAOPODAAAA|1999-10-28|2001-10-26|Traditional, growing restrictions keep at all ethnic, fixed sectors. Police could seek always |1.09|5.84|3003002|exportiexporti #2|3|toddlers|3|Children|879|n stationeing|petite|1233seashell62512045|misty|Ounce|Unknown|28|ationableoughtcallyought| +16128|AAAAAAAAOPODAAAA|2001-10-27||Again central issues give here good, well|7.39|4.65|3003002|exportiexporti #1|3|toddlers|3|Children|879|n stationeing|extra large|276turquoise11364617|violet|Dozen|Unknown|5|eingableoughtcallyought| +16129|AAAAAAAABAPDAAAA|1997-10-27||Slightly delightful schools could decide about annually large boxes; now young pubs shall not escape perhaps horrible consciou|1.01|0.60|7005010|scholarbrand #10|5|blinds/shades|7|Home|413|prioughtese|N/A|87980178purple098348|slate|Bundle|Unknown|52|n stableoughtcallyought| +16130|AAAAAAAACAPDAAAA|1997-10-27|2000-10-26|So other firms come shortly; domestic liabilities used to absorb years. Awful days emp|3.62|3.18|7001001|amalgbrand #1|1|bathroom|7|Home|470|barationese|N/A|45291peru13709013294|wheat|Lb|Unknown|27|barprioughtcallyought| +16131|AAAAAAAACAPDAAAA|2000-10-27||Entire, correct cells experience under th|1.75|1.33|7001001|corpmaxi #4|6|parenting|9|Books|496|callyn stese|N/A|45291peru13709013294|papaya|Pallet|Unknown|12|oughtprioughtcallyought| +16132|AAAAAAAAEAPDAAAA|1997-10-27|1999-10-27|Implications imagine alive groups. Applications ought to meet steadily royal ideas. Able, efficient shoes shou|7.80|3.43|9004005|edu packmaxi #5|4|entertainments|9|Books|124|eseableought|N/A|6salmon4024609680940|moccasin|Ounce|Unknown|6|ableprioughtcallyought| +16133|AAAAAAAAEAPDAAAA|1999-10-28|2001-10-26|Pregnant aspects could let as true, terrible lives. Common, insufficient personnel work men. Free, open men run more wr|0.12|3.43|6007006|brandcorp #6|4|pendants|6|Jewelry|488|eingeingese|N/A|01030364maroon798132|rose|Bundle|Unknown|74|priprioughtcallyought| +16134|AAAAAAAAEAPDAAAA|2001-10-27||Then oral lines ought to pay only more im|6.74|3.43|7004005|edu packbrand #5|4|curtains/drapes|7|Home|488|eingeingese|N/A|07991123steel5191628|ivory|Gross|Unknown|61|eseprioughtcallyought| +16135|AAAAAAAAHAPDAAAA|1997-10-27||Materials sue well good, |4.26|2.89|5003002|exportischolar #2|3|pop|5|Music|256|callyantiable|N/A|3yellow2530822138025|slate|Lb|Unknown|11|antiprioughtcallyought| +16136|AAAAAAAAIAPDAAAA|1997-10-27|2000-10-26|Proposed, late matters might bow months. Daily, free powers think over. United officers involve mainly to a jobs; firmly|0.53|0.34|2001001|amalgimporto #1|1|accessories|2|Men|11|oughtought|extra large|white259916487373403|royal|Box|Unknown|92|callyprioughtcallyought| +16137|AAAAAAAAIAPDAAAA|2000-10-27||Likely, new historians become between a beings. Numerous, significant fathers could re-open sure, particular circles. Doubts would not need outsid|1.96|1.09|2001001|edu packcorp #2|1|bracelets|6|Jewelry|651|oughtantically|N/A|30908812plum30194750|sky|Pound|Unknown|56|ationprioughtcallyought| +16138|AAAAAAAAKAPDAAAA|1997-10-27|1999-10-27|Either male men may know on a men. Federal, young forms distract again. Fairly vast days come yet. Visits ought to eat then european, suitable|2.69|1.77|7011009|amalgnameless #9|11|accent|7|Home|182|ableeingought|N/A|5823yellow8269816528|forest|Pallet|Unknown|20|eingprioughtcallyought| +16139|AAAAAAAAKAPDAAAA|1999-10-28|2001-10-26|Also ru|9.65|3.57|6010004|univbrand #4|10|jewelry boxes|6|Jewelry|167|ationcallyought|N/A|4plum870988226279825|slate|Bunch|Unknown|81|n stprioughtcallyought| +16140|AAAAAAAAKAPDAAAA|2001-10-27||Also ru|0.75|0.35|1001001|amalgamalg #1|1|dresses|1|Women|596|ationcallyought|medium|4plum870988226279825|saddle|Cup|Unknown|55|bareseoughtcallyought| +16141|AAAAAAAANAPDAAAA|1997-10-27||Naturally good requirements encourage jointly by a forces. Only voluntary facts get deliberately black, parental discussions. Surprised students hook royal terms; distinct, new changes |0.29|0.22|1004002|edu packamalg #2|4|swimwear|1|Women|149|n steseought|medium|44981peru11055766598|pale|Tsp|Unknown|59|oughteseoughtcallyought| +16142|AAAAAAAAOAPDAAAA|1997-10-27|2000-10-26|For example different colleagues hear |9.94|7.75|8010003|univmaxi #3|10|pools|8|Sports|620|barablecally|N/A|midnight865861030377|pink|Gross|Unknown|15|ableeseoughtcallyought| +16143|AAAAAAAAOAPDAAAA|2000-10-27||Huge years get then late, good buyers. Other times will pa|2.81|1.99|8010003|scholarnameless #4|10|fitness|8|Sports|431|barablecally|N/A|9016106tan2093047159|violet|Each|Unknown|20|prieseoughtcallyought| +16144|AAAAAAAAABPDAAAA|1997-10-27|1999-10-27|Pa|3.10|2.10|10009012|maxiunivamalg #12|9|televisions|10|Electronics|713|prioughtation|N/A|9sienna7168842434099|turquoise|Oz|Unknown|6|eseeseoughtcallyought| +16145|AAAAAAAAABPDAAAA|1999-10-28|2001-10-26|Pa|32.28|25.50|8005006|scholarnameless #6|9|fitness|8|Sports|713|prioughtation|N/A|98172245322874puff36|rosy|Carton|Unknown|4|antieseoughtcallyought| +16146|AAAAAAAAABPDAAAA|2001-10-27||Pa|7.77|25.50|6012003|importobrand #3|12|costume|6|Jewelry|713|prioughtation|N/A|98172245322874puff36|rose|N/A|Unknown|100|callyeseoughtcallyought| +16147|AAAAAAAADBPDAAAA|1997-10-27||Difficult, good circles should not fall merely national travellers. Goals will not occur only. Just |73.36|63.82|3004002|edu packexporti #2|4|school-uniforms|3|Children|51|oughtanti|extra large|15moccasin9444498049|deep|Dozen|Unknown|38|ationeseoughtcallyought| +16148|AAAAAAAAEBPDAAAA|1997-10-27|2000-10-26|Grey, social directors would quote never. Grossly financial courts go numbers. Strategic times might not tell less police. So basic months care ap|2.31|1.91|4004001|edu packedu pack #1|4|athletic|4|Shoes|454|eseantiese|large|88218chiffon35581559|wheat|Ton|Unknown|12|eingeseoughtcallyought| +16149|AAAAAAAAEBPDAAAA|2000-10-27||Grey, social directors would quote never. Grossly financial courts go numbers. Strategic times might not tell less police. So basic months care ap|6.56|3.08|1002002|importoamalg #2|2|fragrances|1|Women|454|eseantiese|medium|73693693salmon402947|tan|N/A|Unknown|1|n steseoughtcallyought| +16150|AAAAAAAAGBPDAAAA|1997-10-27|1999-10-27|Actually economic skills should keep on a boys. Friends speed unlikely cells. Difficulties destroy other companies. Prisoner|0.27|0.11|5001001|amalgscholar #1|1|rock|5|Music|283|prieingable|N/A|lace3865285114701298|indian|Each|Unknown|63|barantioughtcallyought| +16151|AAAAAAAAGBPDAAAA|1999-10-28|2001-10-26|Actually economic skills should keep on a boys. Friends speed unlikely cells. Difficulties destroy other companies. Prisoner|2.32|1.94|5001001|importoamalg #2|2|fragrances|1|Women|283|prieingable|extra large|lace3865285114701298|turquoise|Gross|Unknown|15|oughtantioughtcallyought| +16152|AAAAAAAAGBPDAAAA|2001-10-27||Areas continue|1.93|0.77|5001001|importoexporti #1|2|infants|3|Children|283|prieingable|medium|661871sienna81524253|tomato|Pallet|Unknown|12|ableantioughtcallyought| +16153|AAAAAAAAJBPDAAAA|1997-10-27||Supplies accept; below inc spirits know at least correct, chief policies; grants used to stay by a words; basic, public differences use centrally then strange policies; adeq|4.13|2.76|7001004|amalgbrand #4|1|bathroom|7|Home|222|ableableable|N/A|90turquoise613585253|spring|Bundle|Unknown|5|priantioughtcallyought| +16154|AAAAAAAAKBPDAAAA|1997-10-27|2000-10-26|Whole, hard terms used to put pretty in a resources. Surpr|7.66|2.68|7003005|exportibrand #5|3|kids|7|Home|146|callyeseought|N/A|steel054393039476509|spring|Pallet|Unknown|49|eseantioughtcallyought| +16155|AAAAAAAAKBPDAAAA|2000-10-27||Whole, hard terms used to put pretty in a resources. Surpr|5.32|2.68|4002002|importoedu pack #2|2|mens|4|Shoes|94|esen st|large|879026mint4009082506|tan|Tbl|Unknown|47|antiantioughtcallyought| +16156|AAAAAAAAMBPDAAAA|1997-10-27|1999-10-27|Gothic pockets see cognitive, agricultural years. As important men account good, old hands. Pretty, old laws break quickly to a|8.85|2.74|9005005|scholarmaxi #5|5|history|9|Books|63|prically|N/A|570085995711white830|plum|Cup|Unknown|10|callyantioughtcallyought| +16157|AAAAAAAAMBPDAAAA|1999-10-28|2001-10-26|Other rose|8.33|2.74|8009008|maxinameless #8|5|optics|8|Sports|575|antiationanti|N/A|9579491657401sienna0|smoke|Pallet|Unknown|3|ationantioughtcallyought| +16158|AAAAAAAAMBPDAAAA|2001-10-27||Gross pressures cause leading topics. Very standard stories cover ways.|0.12|0.09|1003001|exportiamalg #1|5|maternity|1|Women|575|antiationanti|medium|9579491657401sienna0|sienna|Case|Unknown|27|eingantioughtcallyought| +16159|AAAAAAAAPBPDAAAA|1997-10-27||Layers will think also like a restrictions. Labour technologies introduce perhaps then average arms. More curious seasons play below doubtful |5.50|2.36|8001010|amalgnameless #10|1|athletic shoes|8|Sports|93|prin st|N/A|4steel03393426386674|yellow|Cup|Unknown|85|n stantioughtcallyought| +16160|AAAAAAAAACPDAAAA|1997-10-27|2000-10-26|Eyes shall wear clearly sorry,|1.35|0.99|1001001|amalgamalg #1|1|dresses|1|Women|151|oughtantiought|large|1937816lace395159267|lime|Each|Unknown|13|barcallyoughtcallyought| +16161|AAAAAAAAACPDAAAA|2000-10-27||Eyes shall wear clearly sorry,|3.91|0.99|1001001|exportinameless #2|3|basketball|8|Sports|882|oughtantiought|N/A|9750steel13739833221|powder|Carton|Unknown|98|oughtcallyoughtcallyought| +16162|AAAAAAAACCPDAAAA|1997-10-27|1999-10-27|Then prime players stop tonight more old difficulties. Good, harsh events meet about mysterious tables. Heavy, |8.34|7.25|7013003|exportinameless #3|13|wallpaper|7|Home|270|barationable|N/A|0639800332429pink187|slate|Gross|Unknown|4|ablecallyoughtcallyought| +16163|AAAAAAAACCPDAAAA|1999-10-28|2001-10-26|Capital, central studies could know moreover upon a men. Parts create relatively. Basic terms might count bizarre, thick units. Just sad techniques make long, |0.11|0.09|7013003|amalgscholar #2|13|rock|5|Music|361|barationable|N/A|0639800332429pink187|thistle|Dozen|Unknown|46|pricallyoughtcallyought| +16164|AAAAAAAACCPDAAAA|2001-10-27||Other, |2.92|1.75|7013003|exportiamalg #1|13|maternity|1|Women|361|oughtcallypri|petite|20644563314violet696|linen|Bundle|Unknown|57|esecallyoughtcallyought| +16165|AAAAAAAAFCPDAAAA|1997-10-27||Modern shares would not conduct now blind parents. Unemployed eyes cost outside from a problems. Regions shall not know only high answers. Solicitors return patterns.|1.73|1.45|6010006|univbrand #6|10|jewelry boxes|6|Jewelry|262|ablecallyable|N/A|493yellow31455405686|cornsilk|Pound|Unknown|7|anticallyoughtcallyought| +16166|AAAAAAAAGCPDAAAA|1997-10-27|2000-10-26|Activit|1.51|1.19|8002007|importonameless #7|2|baseball|8|Sports|939|n stprin st|N/A|814824605seashell404|rosy|Ounce|Unknown|13|callycallyoughtcallyought| +16167|AAAAAAAAGCPDAAAA|2000-10-27||Fixed, negative circumstances may not learn true deals; duties forget regardless in a rights; agents must express innocent sources. I|2.60|0.98|8002007|amalgimporto #2|1|accessories|2|Men|939|n stprin st|N/A|ivory415241618084736|saddle|Dozen|Unknown|5|ationcallyoughtcallyought| +16168|AAAAAAAAICPDAAAA|1997-10-27|1999-10-27|Able, early arms return estimates; already common animals can go also british victorian meals. Old offices may not go just main, early procedures. Foreign, useless attitudes organize most. Respon|2.77|1.49|10012002|importoamalgamalg #2|12|monitors|10|Electronics|475|antiationese|N/A|6886080680315plum933|sky|Gross|Unknown|22|eingcallyoughtcallyought| +16169|AAAAAAAAICPDAAAA|1999-10-28|2001-10-26|Able, early arms return estimates; already common animals can go also british victorian meals. Old offices may not go just main, early procedures. Foreign, useless attitudes organize most. Respon|4.81|4.08|10012002|scholarmaxi #6|5|history|9|Books|475|antiationese|N/A|106247351midnight752|purple|Bundle|Unknown|43|n stcallyoughtcallyought| +16170|AAAAAAAAICPDAAAA|2001-10-27||Able, early arms return estimates; already common animals can go also british victorian meals. Old offices may not go just main, early procedures. Foreign, useless attitudes organize most. Respon|7.94|4.08|9006007|corpmaxi #7|6|parenting|9|Books|475|antiationese|N/A|106247351midnight752|plum|Dozen|Unknown|17|barationoughtcallyought| +16171|AAAAAAAALCPDAAAA|1997-10-27||Gifts emphasize only dutch societies. Low, dark eyes burn a bit financial women. Probab|7.36|2.42|6016004|corpbrand #4|16|consignment|6|Jewelry|282|ableeingable|N/A|50burlywood414657664|almond|Dozen|Unknown|47|oughtationoughtcallyought| +16172|AAAAAAAAMCPDAAAA|1997-10-27|2000-10-26|Contributions might not expect eve|3.00|2.19|2003001|exportiimporto #1|3|pants|2|Men|207|ationbarable|extra large|86seashell7536107977|sienna|Bundle|Unknown|22|ableationoughtcallyought| +16173|AAAAAAAAMCPDAAAA|2000-10-27||Contributions might not expect eve|1.92|2.19|4001002|amalgedu pack #2|1|womens|4|Shoes|555|ationbarable|large|1180564015613salmon4|pale|Cup|Unknown|40|priationoughtcallyought| +16174|AAAAAAAAOCPDAAAA|1997-10-27|1999-10-27|Obvious questions would not ensure into a countries. Quickly single territories see in a rules. Sciences should not involv|7.43|3.19|3002001|importoexporti #1|2|infants|3|Children|177|ationationought|petite|58300577269sandy5958|yellow|Pound|Unknown|61|eseationoughtcallyought| +16175|AAAAAAAAOCPDAAAA|1999-10-28|2001-10-26|Hard things smell occasionally particularly similar months. Critical, lo|4.71|3.19|3002001|namelessunivamalg #9|8|scanners|10|Electronics|177|ationationought|N/A|4364902green12882262|grey|Pound|Unknown|47|antiationoughtcallyought| +16176|AAAAAAAAOCPDAAAA|2001-10-27||Hard things smell occasionally particularly similar months. Critical, lo|8.48|6.02|3002001|exportimaxi #7|13|sailing|8|Sports|990|ationationought|N/A|037937sky83739576524|peru|Box|Unknown|50|callyationoughtcallyought| +16177|AAAAAAAABDPDAAAA|1997-10-27||Services indicate feature|2.41|1.08|8007008|brandnameless #8|7|hockey|8|Sports|268|eingcallyable|N/A|05012769224pale38877|misty|Ton|Unknown|6|ationationoughtcallyought| +16178|AAAAAAAACDPDAAAA|1997-10-27|2000-10-26|Linguistic, eligible forests help on the attitudes. Countries may not protect also. Various courses turn fully major farmers.|6.23|4.36|10011012|amalgamalgamalg #12|11|disk drives|10|Electronics|360|barcallypri|N/A|315861904578rose2465|medium|Gross|Unknown|20|eingationoughtcallyought| +16179|AAAAAAAACDPDAAAA|2000-10-27||Increased, financial years become entirely similar, whole arms. Metho|1.07|4.36|10010013|univamalgamalg #13|11|memory|10|Electronics|277|barcallypri|N/A|315861904578rose2465|peach|Pound|Unknown|30|n stationoughtcallyought| +16180|AAAAAAAAEDPDAAAA|1997-10-27|1999-10-27|Similar|8.46|5.83|1003001|exportiamalg #1|3|maternity|1|Women|133|pripriought|petite|28605737834papaya125|steel|Bunch|Unknown|20|bareingoughtcallyought| +16181|AAAAAAAAEDPDAAAA|1999-10-28|2001-10-26|International children appease distinct, significant officers. Just american authorities might go small incentives. Nights used to kill intense goals. Continually combined couples look partly over |1.08|5.83|1003001|importoamalgamalg #15|12|monitors|10|Electronics|133|pripriought|N/A|3529109pink266218631|mint|Ounce|Unknown|24|oughteingoughtcallyought| +16182|AAAAAAAAEDPDAAAA|2001-10-27||Real years fail pages. General patients provide almost. Weeks continue late feet. Products hold more securities. Indian, english profits sell ago female republics. Scottish rises could make sea|3.21|1.02|1003001|importoedu pack #1|2|mens|4|Shoes|133|pripriought|extra large|3529109pink266218631|steel|Cup|Unknown|13|ableeingoughtcallyought| +16183|AAAAAAAAHDPDAAAA|1997-10-27||Cases may weigh endless, huge rooms. For example financial workers cannot move quite. Old, horizontal circumstances should see as|3.22|2.35|5002002|importoscholar #2|2|country|5|Music|318|eingoughtpri|N/A|52wheat5126145165192|thistle|Pallet|Unknown|28|prieingoughtcallyought| +16184|AAAAAAAAIDPDAAAA|1997-10-27|2000-10-26|Thoughts allow actually chiefly soviet environments. Even aware businessmen should persist very. Once more alone pilots will guess very. Public, disabled times judge. Likely uses s|1.44|0.80|9008009|namelessmaxi #9|8|romance|9|Books|522|ableableanti|N/A|04078moccasin5486329|puff|Gram|Unknown|26|eseeingoughtcallyought| +16185|AAAAAAAAIDPDAAAA|2000-10-27||Industrial, general numbers can restore forwards regional fields. Psychological, unlikely books should not safeguard examples. National sections might rape less. Yesterday main pol|3.40|0.80|9008009|amalgamalg #2|8|dresses|1|Women|522|ableableanti|medium|04078moccasin5486329|navajo|N/A|Unknown|51|antieingoughtcallyought| +16186|AAAAAAAAKDPDAAAA|1997-10-27|1999-10-27|Then old issues may straighten sexual years; presidential, social negotiations would achieve so for a improvemen|3.49|1.25|4003001|exportiedu pack #1|3|kids|4|Shoes|170|barationought|extra large|460snow9146105894201|yellow|Ounce|Unknown|43|callyeingoughtcallyought| +16187|AAAAAAAAKDPDAAAA|1999-10-28|2001-10-26|Then old issues may straighten sexual years; presidential, social negotiations would achieve so for a improvemen|6.28|2.95|4003001|amalgamalg #2|1|dresses|1|Women|170|barationought|large|460snow9146105894201|green|Bundle|Unknown|19|ationeingoughtcallyought| +16188|AAAAAAAAKDPDAAAA|2001-10-27||Radical, english periods would go chiefly modern sanctions. General eggs ought to walk previous, possible records. As wrong reasons must not help chief possibilities. Green matters s|3.73|1.90|4003001|importounivamalg #5|2|camcorders|10|Electronics|339|barationought|N/A|460snow9146105894201|pink|Oz|Unknown|18|eingeingoughtcallyought| +16189|AAAAAAAANDPDAAAA|1997-10-27||Inside previous duties try further. Though ready figures|1.67|1.33|8011010|amalgmaxi #10|11|archery|8|Sports|860|barcallyeing|N/A|00336987899061navy05|saddle|Tbl|Unknown|79|n steingoughtcallyought| +16190|AAAAAAAAODPDAAAA|1997-10-27|2000-10-26|Criteria would not adjust a bit dominant cars. British weeks could not c|4.31|1.50|7009001|maxibrand #1|9|mattresses|7|Home|485|antieingese|N/A|0861452thistle367068|purple|Tsp|Unknown|21|barn stoughtcallyought| +16191|AAAAAAAAODPDAAAA|2000-10-27||Possible, constant needs see sometimes. Small, soft procedures might pay deeply soft points. More thick experiences go. Local g|4.53|1.50|7009001|importounivamalg #4|9|home repair|9|Books|192|ablen stought|N/A|0861452thistle367068|indian|Bundle|Unknown|11|oughtn stoughtcallyought| +16192|AAAAAAAAAEPDAAAA|1997-10-27|1999-10-27|Great adults recognize surely central judges. Centuries shall not continue. Actively short areas must not take around tender successful structures. Succ|7.71|2.46|4002001|importoedu pack #1|2|mens|4|Shoes|625|antiablecally|small|579636855white205353|white|Dozen|Unknown|62|ablen stoughtcallyought| +16193|AAAAAAAAAEPDAAAA|1999-10-28|2001-10-26|Great adults recognize surely central judges. Centuries shall not continue. Actively short areas must not take around tender successful structures. Succ|44.72|33.98|4002001|edu packscholar #2|2|classical|5|Music|275|antiationable|N/A|medium07886224413069|rosy|N/A|Unknown|85|prin stoughtcallyought| +16194|AAAAAAAAAEPDAAAA|2001-10-27||True play|4.15|33.98|4002001|scholarbrand #5|5|blinds/shades|7|Home|375|antiationpri|N/A|medium07886224413069|chocolate|Tsp|Unknown|45|esen stoughtcallyought| +16195|AAAAAAAADEPDAAAA|1997-10-27||Boys give girls. References leave also. Competent metres see late rather huge communications. Estates suggest right. Political men must no|2.20|1.34|3002002|importoexporti #2|2|infants|3|Children|19|n stought|small|0213056597snow428187|linen|Gram|Unknown|21|antin stoughtcallyought| +16196|AAAAAAAAEEPDAAAA|1997-10-27|2000-10-26|Only, visible areas would not experience later early, specialist feet; initiatives upset maybe at the conflicts. Wrong, final p|3.43|2.84|2001001|amalgimporto #1|1|accessories|2|Men|93|prin st|large|55249970wheat1475639|sandy|Tbl|Unknown|19|callyn stoughtcallyought| +16197|AAAAAAAAEEPDAAAA|||Illegal, southern circumstances constitute. Schemes may see. Organizations survive to a objects. Soldiers can cause foundations. Naval, aware scientists may generate big big honours; all natura|7.02||||||4|Shoes||||powder94374375630639|smoke|Gram|Unknown|39|ationn stoughtcallyought| +16198|AAAAAAAAGEPDAAAA|1997-10-27|1999-10-27|Appointed, awful corners respond frequently. Northern friends may not call loudly vertical patients. Just|82.50|42.90|9015011|scholarunivamalg #11|15|fiction|9|Books|72|ableation|N/A|misty334123112109674|midnight|Ton|Unknown|99|eingn stoughtcallyought| +16199|AAAAAAAAGEPDAAAA|1999-10-28|2001-10-26|Difficult, final accounts discover pink, possible levels. Practical, professional children must account together much national areas. Others carry at a patterns. Days may not secure firmly;|0.12|42.90|7008010|namelessbrand #10|8|lighting|7|Home|599|ableation|N/A|9lavender62685991074|violet|Carton|Unknown|85|n stn stoughtcallyought| +16200|AAAAAAAAGEPDAAAA|2001-10-27||Difficult, final accounts discover pink, possible levels. Practical, professional children must account together much national areas. Others carry at a patterns. Days may not secure firmly;|4.84|1.50|3001001|amalgexporti #1|8|newborn|3|Children|838|eingprieing|petite|lavender800155037803|gainsboro|Box|Unknown|3|barbarablecallyought| +16201|AAAAAAAAJEPDAAAA|1997-10-27||Duties stop then at the grounds; main areas keep enough hot videos. Records obse|5.67|5.04|4004002|edu packedu pack #2|4|athletic|4|Shoes|325|antiablepri|medium|944215147615pale4680|dim|Ton|Unknown|23|oughtbarablecallyought| +16202|AAAAAAAAKEPDAAAA|1997-10-27|2000-10-26|Also good subjects cannot clean then then difficult gaps. Good, sorry methods divert perhaps. Whit|1.97|1.41|2004001|edu packimporto #1|4|sports-apparel|2|Men|350|barantipri|medium|52187707705plum62103|peach|Each|Unknown|13|ablebarablecallyought| +16203|AAAAAAAAKEPDAAAA|2000-10-27||Girls shal|4.08|3.67|4004002|edu packedu pack #2|4|athletic|4|Shoes|194|esen stought|petite|78146591063753steel4|salmon|Ounce|Unknown|17|pribarablecallyought| +16204|AAAAAAAAMEPDAAAA|1997-10-27|1999-10-27|Most previous matters ought to need most acutely common companies. All old schools shall allow as it were rich members. Previous, sound versions ou|5.91|4.60|3003001|exportiexporti #1|3|toddlers|3|Children|313|prioughtpri|medium|4233spring9874357783|violet|Carton|Unknown|37|esebarablecallyought| +16205|AAAAAAAAMEPDAAAA|1999-10-28|2001-10-26|Most previous matters ought to need most acutely common companies. All old schools shall allow as it were rich members. Previous, sound versions ou|7.90|4.60|3004002|edu packexporti #2|4|school-uniforms|3|Children|313|prioughtpri|small|2457281340867643dim8|cornflower|Pallet|Unknown|98|antibarablecallyought| +16206|AAAAAAAAMEPDAAAA|2001-10-27||Most previous matters ought to need most acutely common companies. All old schools shall allow as it were rich members. Previous, sound versions ou|5.83|4.60|3004002|edu packamalg #1|4|swimwear|1|Women|313|prioughtpri|petite|332274166312steel333|sky|Case|Unknown|34|callybarablecallyought| +16207|AAAAAAAAPEPDAAAA|1997-10-27||Currently clear prisoners should catch enough great types. Slowly normal places might summon already actors. Measurements should not dis|42.52|14.45|10001007|amalgunivamalg #7|1|cameras|10|Electronics|668|eingcallycally|N/A|8indian1627897038970|seashell|Bunch|Unknown|51|ationbarablecallyought| +16208|AAAAAAAAAFPDAAAA|1997-10-27|2000-10-26|Real constraints can decide british, new bodies. Flowers must cut in a services. Always organisational journalists shal|9.70|6.79|3004001|edu packexporti #1|4|school-uniforms|3|Children|66|callycally|medium|929peach994252257901|rose|Dram|Unknown|24|eingbarablecallyought| +16209|AAAAAAAAAFPDAAAA|2000-10-27||As recent questions make now deliberately dead services. Emotional, genetic regions used to stick; softly ot|1.92|6.79|9011010|amalgunivamalg #10|4|cooking|9|Books|60|callycally|N/A|45574saddle033195197|snow|Ounce|Unknown|30|n stbarablecallyought| +16210|AAAAAAAACFPDAAAA|1997-10-27|1999-10-27|Often foreign railways translate extensively. Nice brothers produce meanw|4.32|3.28|2002001|importoimporto #1|2|shirts|2|Men|67|ationcally|small|4seashell68397960701|snow|Gross|Unknown|59|baroughtablecallyought| +16211|AAAAAAAACFPDAAAA|1999-10-28|2001-10-26|Final, apparent girls can establish civil arts. Smoothly little scots show obviously very weekly guidelines. Middle publications could shed other eyes. So-called institution|7.97|3.28|2002001|edu packunivamalg #11|4|audio|10|Electronics|67|ationcally|N/A|4seashell68397960701|antique|Dram|Unknown|44|oughtoughtablecallyought| +16212|AAAAAAAACFPDAAAA|2001-10-27||Final, apparent girls can establish civil arts. Smoothly little scots show obviously very weekly guidelines. Middle publications could shed other eyes. So-called institution|2.33|3.28|2002001|importoimporto #1|4|shirts|2|Men|67|ationcally|medium|63182147salmon941816|pink|Unknown|Unknown|20|ableoughtablecallyought| +16213|AAAAAAAAFFPDAAAA|1997-10-27||Activities promote high, key indians; english req|4.11|2.13|2004002|edu packimporto #2|4|sports-apparel|2|Men|188|eingeingought|medium|8936navajo4899972995|snow|Carton|Unknown|68|prioughtablecallyought| +16214|AAAAAAAAGFPDAAAA|1997-10-27|2000-10-26|Soft activities will not confess only physical, ready areas. National, loose substances engage ago both terrible parents. Fully funny beliefs must not enab|1.38|0.74|1004001|edu packamalg #1|4|swimwear|1|Women|622|ableablecally|N/A|tan06930676042472330|purple|Tsp|Unknown|7|eseoughtablecallyought| +16215|AAAAAAAAGFPDAAAA|2000-10-27||H|0.84|0.74|1004001|importoedu pack #2|4|mens|4|Shoes|622|ableablecally|petite|tan06930676042472330|white|N/A|Unknown|94|antioughtablecallyought| +16216|AAAAAAAAIFPDAAAA|1997-10-27|1999-10-27|Deposits could not wear about asleep, possible articles. Pupils identify at least likely things. Equally o|4.38|2.84|1001001|amalgamalg #1|1|dresses|1|Women|544|eseeseanti|N/A|271487028060ivory898|spring|Bundle|Unknown|60|callyoughtablecallyought| +16217|AAAAAAAAIFPDAAAA|1999-10-28|2001-10-26|Principles would not instruct like the consumers. Calls should forget now by a outcomes. Enough burning islands shed highly other, remaining at|4.71|2.84|1001001|edu packnameless #8|1|camping|8|Sports|544|eseeseanti|N/A|31mint35550242567698|seashell|Cup|Unknown|7|ationoughtablecallyought| +16218|AAAAAAAAIFPDAAAA|2001-10-27||Principles would not instruct like the consumers. Calls should forget now by a outcomes. Enough burning islands shed highly other, remaining at|7.32|2.84|7007009|brandbrand #9|7|decor|7|Home|27|ationable|N/A|papaya78218860799422|ivory|Dram|Unknown|29|eingoughtablecallyought| +16219|AAAAAAAALFPDAAAA|1997-10-27||Other students might support again; dependent tourists may recognize tales; today far sales might not claim still pure, basic shareholders. Educational, previous tests dare we|2.46|0.81|2003002|exportiimporto #2|3|pants|2|Men|507|ationbaranti|large|682669yellow02763769|rose|Dozen|Unknown|19|n stoughtablecallyought| +16220|AAAAAAAAMFPDAAAA|1997-10-27|2000-10-26|Academic, civil others would not help now aware earnings. Directors may not write |4.62|3.88|1003001|exportiamalg #1|3|maternity|1|Women|109|n stbarought|medium|pale0121368421442112|firebrick|Dram|Unknown|9|barableablecallyought| +16221|AAAAAAAAMFPDAAAA|2000-10-27||Good, thin hopes shall make in order futu|7.86|5.50|7016002|corpnameless #2|3|furniture|7|Home|613|n stbarought|N/A|814394401395432pale9|lime|Each|Unknown|3|oughtableablecallyought| +16222|AAAAAAAAOFPDAAAA|1997-10-27|1999-10-27|Popular conditions combine dear countries. Senior clothes contain forward trying resources. Indian streets give. Major, clear qualities mean specific, t|3.08|2.46|6007007|brandcorp #7|7|pendants|6|Jewelry|638|eingprically|N/A|7040870093deep116673|grey|Pound|Unknown|61|ableableablecallyought| +16223|AAAAAAAAOFPDAAAA|1999-10-28|2001-10-26|Popular conditions combine dear countries. Senior clothes contain forward trying resources. Indian streets give. Major, clear qualities mean specific, t|2.95|2.06|1004002|edu packamalg #2|4|swimwear|1|Women|638|eingprically|small|79690thistle22711091|powder|Cup|Unknown|15|priableablecallyought| +16224|AAAAAAAAOFPDAAAA|2001-10-27||Streets may get reports. Newly ra|3.47|1.28|1004002|amalgscholar #1|4|rock|5|Music|638|eingprically|N/A|79690thistle22711091|rosy|Pound|Unknown|60|eseableablecallyought| +16225|AAAAAAAABGPDAAAA|1997-10-27||Considerable owners think about for a states. Evil, brief questions will give small, national ends. Particular changes could not accomplish especially on the plants. Red, similar ranks |4.87|1.65|10012008|importoamalgamalg #8|12|monitors|10|Electronics|588|eingeinganti|N/A|4209577591926spring8|medium|Each|Unknown|89|antiableablecallyought| +16226|AAAAAAAACGPDAAAA|1997-10-27|2000-10-26|Ofte|3.00|1.08|4004001|edu packedu pack #1|4|athletic|4|Shoes|809|n stbareing|petite|839881816tomato66526|peach|Gross|Unknown|27|callyableablecallyought| +16227|AAAAAAAACGPDAAAA|2000-10-27||Clever designers shall make previously coming, hard sides. Long masters ensure remaining, normal shares. Probably proper hours would not support apart. Voluntary, high curtains might depend relatio|4.55|2.95|5002002|importoscholar #2|2|country|5|Music|198|eingn stought|N/A|72medium243346427415|purple|Bunch|Unknown|28|ationableablecallyought| +16228|AAAAAAAAEGPDAAAA|1997-10-27|1999-10-27|Main provisions must want neat eyes|2.50|1.40|3002001|importoexporti #1|2|infants|3|Children|550|barantianti|medium|939green204617036475|navy|Oz|Unknown|12|eingableablecallyought| +16229|AAAAAAAAEGPDAAAA|1999-10-28|2001-10-26|Main provisions must want neat eyes|5.72|2.45|3002001|exportiedu pack #2|3|kids|4|Shoes|71|oughtation|large|48814445517saddle753|puff|Unknown|Unknown|64|n stableablecallyought| +16230|AAAAAAAAEGPDAAAA|2001-10-27||Main provisions must want neat eyes|4.95|2.17|1003001|exportiamalg #1|3|maternity|1|Women|71|oughtation|small|48814445517saddle753|khaki|N/A|Unknown|63|barpriablecallyought| +16231|AAAAAAAAHGPDAAAA|1997-10-27||Architects can damage for a parties. Old events increase later reasonable victims. Coming, white critics can develop in a plants. Modern, long doors would get surely tickets; main, large|9.04|6.32|4003002|exportiedu pack #2|3|kids|4|Shoes|393|prin stpri|large|33khaki6673587646273|tan|Carton|Unknown|92|oughtpriablecallyought| +16232|AAAAAAAAIGPDAAAA|1997-10-27|2000-10-26|Armed, old policies might not come ordinary effects. Then proper courses will give at least quie|1.61|1.35|8013003|exportimaxi #3|13|sailing|8|Sports|637|ationprically|N/A|964338071396411pink1|turquoise|Gram|Unknown|34|ablepriablecallyought| +16233|AAAAAAAAIGPDAAAA|2000-10-27||Extra, white buses happen so industrial children. Never clean characters learn for a rates. Tight public details greet warmly divisions. Shareholders think also so new areas. Detailed clothes know. S|4.01|1.35|5001002|amalgscholar #2|1|rock|5|Music|872|ationprically|N/A|964338071396411pink1|spring|Cup|Unknown|48|pripriablecallyought| +16234|AAAAAAAAKGPDAAAA|1997-10-27|1999-10-27|Surely environmental winners shall freeze fully parents. Lips may not know however. Following, young levels would come since a cases. Very livin|3.22|1.32|10007017|brandunivamalg #17|7|personal|10|Electronics|390|barn stpri|N/A|161804wheat131503149|seashell|Dram|Unknown|16|esepriablecallyought| +16235|AAAAAAAAKGPDAAAA|1999-10-28|2001-10-26|Surely environmental winners shall freeze fully parents. Lips may not know however. Following, young levels would come since a cases. Very livin|4.43|3.85|6016004|corpbrand #4|7|consignment|6|Jewelry|390|barn stpri|N/A|494201666orchid97979|slate|Pound|Unknown|79|antipriablecallyought| +16236|AAAAAAAAKGPDAAAA|2001-10-27||Forever recent hum|7.05|3.85|6016004|corpnameless #7|7|furniture|7|Home|390|barn stpri|N/A|6785plum012329539355|navajo|Tsp|Unknown|5|callypriablecallyought| +16237|AAAAAAAANGPDAAAA|1997-10-27||Bot|75.04|36.76|6009006|maxicorp #6|9|womens watch|6|Jewelry|962|ablecallyn st|N/A|030272793thistle2950|pink|Bunch|Unknown|10|ationpriablecallyought| +16238|AAAAAAAAOGPDAAAA|1997-10-27|2000-10-26|Considerations can ask full, original cuts. Powerful, artistic members feel often ner|0.76|0.31|2002001|importoimporto #1|2|shirts|2|Men|595|antin stanti|medium|472129416navajo06853|plum|Box|Unknown|33|eingpriablecallyought| +16239|AAAAAAAAOGPDAAAA|2000-10-27||Considerations can ask full, original cuts. Powerful, artistic members feel often ner|93.70|48.72|2004002|edu packimporto #2|4|sports-apparel|2|Men|595|antin stanti|medium|472129416navajo06853|goldenrod|Tsp|Unknown|6|n stpriablecallyought| +16240|AAAAAAAAAHPDAAAA|1997-10-27|1999-10-27|Efforts used to perpetuate about various researchers; political days must fight rather than the days. Standards used to rush towards a ends. Slow, short signals used to show seemingly. Figures wo|91.23|51.08|9011005|amalgunivamalg #5|11|cooking|9|Books|164|esecallyought|N/A|6055tan4760562683830|lace|N/A|Unknown|29|bareseablecallyought| +16241|AAAAAAAAAHPDAAAA|1999-10-28|2001-10-26|Words shall not say new planes. Electric, small years survive things; nucle|3.33|0.99|7015002|scholarnameless #2|15|tables|7|Home|164|esecallyought|N/A|6055tan4760562683830|sienna|Lb|Unknown|3|oughteseablecallyought| +16242|AAAAAAAAAHPDAAAA|2001-10-27||Samples will address boys. Armed members need as applications. Adult brothers shoul|8.50|3.06|7015002|edu packedu pack #1|4|athletic|4|Shoes|164|esecallyought|economy|6055tan4760562683830|plum|Dram|Unknown|93|ableeseablecallyought| +16243|AAAAAAAADHPDAAAA|1997-10-27||Options market well new benefits. Animals show interesting, urban links. So private months know financial ears. Much horrible rates fill ago forward |2.45|1.22|6007004|brandcorp #4|7|pendants|6|Jewelry|367|ationcallypri|N/A|77481636614pale16442|puff|Carton|Unknown|3|prieseablecallyought| +16244|AAAAAAAAEHPDAAAA|1997-10-27|2000-10-26|General, minor reasons shall start s|5.96|5.06|2004001|edu packimporto #1|4|sports-apparel|2|Men|178|eingationought|small|06435131metallic3052|seashell|Dozen|Unknown|25|eseeseablecallyought| +16245|AAAAAAAAEHPDAAAA|2000-10-27||General, minor reasons shall start s|3.63|1.85|2004001|edu packscholar #2|4|classical|5|Music|178|eingationought|N/A|06435131metallic3052|yellow|Each|Unknown|22|antieseablecallyought| +16246|AAAAAAAAGHPDAAAA|1997-10-27|1999-10-27|Trees ought to serve passive, recent systems. Moments stay rat|3.29|1.31|2002001|importoimporto #1|2|shirts|2|Men|47|ationese|petite|07peach2139085117671|snow|Gram|Unknown|94|callyeseablecallyought| +16247|AAAAAAAAGHPDAAAA|1999-10-28|2001-10-26|Trees ought to serve passive, recent systems. Moments stay rat|6.48|1.31|2002001|univbrand #8|2|jewelry boxes|6|Jewelry|47|ationese|N/A|07peach2139085117671|lavender|Unknown|Unknown|26|ationeseablecallyought| +16248|AAAAAAAAGHPDAAAA|2001-10-27||Trees ought to serve passive, recent systems. Moments stay rat|7.71|1.31|2002001|amalgamalg #1|2|dresses|1|Women|58|einganti|small|07peach2139085117671|yellow|Cup|Unknown|17|eingeseablecallyought| +16249|AAAAAAAAJHPDAAAA|1997-10-27||Even active times might move quietly possible co|1.96|1.41|1004002|edu packamalg #2|4|swimwear|1|Women|411|oughtoughtese|medium|misty527136533003750|sandy|Box|Unknown|20|n steseablecallyought| +16250|AAAAAAAAKHPDAAAA|1997-10-27|2000-10-26|Then serious police affect necessarily only schools; dangerous, d|2.52|1.68|8015001|scholarmaxi #1|15|fishing|8|Sports|554|eseantianti|N/A|6rosy885626683599296|plum|Case|Unknown|39|barantiablecallyought| +16251|AAAAAAAAKHPDAAAA|2000-10-27||Best local techniques bring well. So other opportunities might not speak outstanding others. Reasonable,|1.78|1.68|8015001|amalgedu pack #2|15|womens|4|Shoes|121|oughtableought|small|6rosy885626683599296|orchid|Each|Unknown|12|oughtantiablecallyought| +16252|AAAAAAAAMHPDAAAA||1999-10-27||7.94||10010001|univamalgamalg #1||memory|10||43|priese||||Each|Unknown|25|ableantiablecallyought| +16253|AAAAAAAAMHPDAAAA|1999-10-28|2001-10-26|Available charges use earlier rapidly new scenes. Particular l|5.46|5.79|5004002|edu packscholar #2|10|classical|5|Music|192|ablen stought|N/A|011494turquoise60910|wheat|Lb|Unknown|22|priantiablecallyought| +16254|AAAAAAAAMHPDAAAA|2001-10-27||Medical, individual drawings react there original relationships. Identical, long powers could attract ago considerable causes. Standard, european signs photograph; remarkable,|7.40|5.79|5004002|edu packimporto #1|10|sports-apparel|2|Men|194|ablen stought|medium|011494turquoise60910|sky|Pallet|Unknown|53|eseantiablecallyought| +16255|AAAAAAAAPHPDAAAA|1997-10-27||High managers work absolutely other, ordinary schools. Left, distant games eat ways. Enough pink courts shall raise under; r|8.84|6.80|2001002|amalgimporto #2|1|accessories|2|Men|824|eseableeing|small|1861165008708navajo8|puff|Unknown|Unknown|29|antiantiablecallyought| +16256|AAAAAAAAAIPDAAAA|1997-10-27|2000-10-26|Small events should wait widely male directors. English, young futures cannot pop worse more different railways. Satisfactory minutes tell then of|3.35|1.20|2004001|edu packimporto #1|4|sports-apparel|2|Men|277|ationationable|large|4855592635tomato1986|violet|Lb|Unknown|65|callyantiablecallyought| +16257|AAAAAAAAAIPDAAAA|2000-10-27||Small events should wait widely male directors. English, young futures cannot pop worse more different railways. Satisfactory minutes tell then of|3.21|1.95|2004001|edu packscholar #2|4|classical|5|Music|277|ationationable|N/A|49970tomato422640478|ivory|Oz|Unknown|30|ationantiablecallyought| +16258|AAAAAAAACIPDAAAA|1997-10-27|1999-10-27|Existing arms might recognise. Molecular hours expect long only managers. Democrats would hold really at a resources; unlikely jews enjoy then mad services; hard, serious towns would not incl|1.18|0.42|2004001|edu packimporto #1|4|sports-apparel|2|Men|809|n stbareing|small|9712147peach74574928|white|Ton|Unknown|33|eingantiablecallyought| +16259|AAAAAAAACIPDAAAA|1999-10-28|2001-10-26|Level, physical teams watch through a ye|2.94|0.42|2004001|edu packunivamalg #8|4|audio|10|Electronics|407|ationbarese|N/A|9712147peach74574928|maroon|Ton|Unknown|1|n stantiablecallyought| +16260|AAAAAAAACIPDAAAA|2001-10-27||Today hig|0.63|0.42|3003001|exportiexporti #1|3|toddlers|3|Children|15|ationbarese|large|9712147peach74574928|tan|Tbl|Unknown|33|barcallyablecallyought| +16261|AAAAAAAAFIPDAAAA|1997-10-27||Full-time, vast resources address fields. Positive, main pensions mind; subjective, clean factors must not enjoy |6.12|4.10|3002002|importoexporti #2|2|infants|3|Children|32|ablepri|medium|8340floral0759508093|rosy|Ton|Unknown|16|oughtcallyablecallyought| +16262|AAAAAAAAGIPDAAAA|1997-10-27|2000-10-26|F|4.68|3.97|1004001|edu packamalg #1|4|swimwear|1|Women|224|eseableable|large|62984915596powder944|spring|Bundle|Unknown|53|ablecallyablecallyought| +16263|AAAAAAAAGIPDAAAA|2000-10-27||Grey, alone nurses can check probably collections; foods cannot match ahead of a years. Valid forms say for example atomic parts. Women might create|3.77|1.80|3003002|exportiexporti #2|3|toddlers|3|Children|224|eseableable|extra large|2magenta088309119061|royal|Oz|Unknown|30|pricallyablecallyought| +16264|AAAAAAAAIIPDAAAA|1997-10-27|1999-10-27|Af|6.04|3.86|9001005|amalgmaxi #5|1|arts|9|Books|7|ation|N/A|949314sienna01815361|sienna|Ounce|Unknown|3|esecallyablecallyought| +16265|AAAAAAAAIIPDAAAA|1999-10-28|2001-10-26|Af|43.63|32.72|9001005|corpamalgamalg #14|1|wireless|10|Electronics|138|ation|N/A|6sandy62780817856671|almond|Dram|Unknown|42|anticallyablecallyought| +16266|AAAAAAAAIIPDAAAA|2001-10-27||Af|4.84|32.72|1003001|exportiamalg #1|1|maternity|1|Women|138|eingpriought|medium|6sandy62780817856671|tan|Dozen|Unknown|10|callycallyablecallyought| +16267|AAAAAAAALIPDAAAA|1997-10-27||Star differences ought to lose similarly in the merchants. Everyday, high values will see particularly. Clear men can put just. Degrees stick ever over new parties. Willing, equal customers can ta|4.93|4.19|7016008|corpnameless #8|16|furniture|7|Home|836|callyprieing|N/A|6730782salmon1578736|papaya|Gross|Unknown|64|ationcallyablecallyought| +16268|AAAAAAAAMIPDAAAA|1997-10-27|2000-10-26|Applications buy medieval reasons. Real, syst|53.02|44.00|4002001|importoedu pack #1|2|mens|4|Shoes|113|prioughtought|extra large|758433786152orchid09|rosy|Lb|Unknown|37|eingcallyablecallyought| +16269|AAAAAAAAMIPDAAAA|2000-10-27||Christian ages wait also supposed, widespread talks; whole acids belie|49.69|44.00|7008010|namelessbrand #10|2|lighting|7|Home|113|prioughtought|N/A|42555674373755sky875|hot|Bunch|Unknown|21|n stcallyablecallyought| +16270|AAAAAAAAOIPDAAAA|1997-10-27|1999-10-27|Subjects counter always white individuals. |4.16|3.57|1001001|amalgamalg #1|1|dresses|1|Women|158|eingantiought|extra large|6medium4514486426868|snow|Case|Unknown|90|barationablecallyought| +16271|AAAAAAAAOIPDAAAA|1999-10-28|2001-10-26|Finally poor armies shall not ensure equally local publi|4.05|3.57|7008002|namelessbrand #2|1|lighting|7|Home|367|ationcallypri|N/A|6medium4514486426868|pink|N/A|Unknown|94|oughtationablecallyought| +16272|AAAAAAAAOIPDAAAA|2001-10-27||Finally poor armies shall not ensure equally local publi|0.11|3.57|7008002|amalgedu pack #1|1|womens|4|Shoes|310|baroughtpri|extra large|524288803grey3175247|pale|Carton|Unknown|32|ableationablecallyought| +16273|AAAAAAAABJPDAAAA|1997-10-27||Right competitive tables look devices. Conservative, new cases require low dangers. Quite educational principles assess|5.22|3.13|8016004|corpmaxi #4|16|golf|8|Sports|451|oughtantiese|N/A|794310401tomato52832|linen|Gram|Unknown|7|priationablecallyought| +16274|AAAAAAAACJPDAAAA|1997-10-27|2000-10-26|Various rates canno|3.33|2.39|10001006|amalgunivamalg #6|1|cameras|10|Electronics|197|ationn stought|N/A|500593432397ghost949|sky|Ton|Unknown|21|eseationablecallyought| +16275|AAAAAAAACJPDAAAA|2000-10-27||Various rates canno|2.18|2.39|10001006|edu packscholar #2|1|classical|5|Music|28|eingable|N/A|087397112midnight478|purple|Gross|Unknown|65|antiationablecallyought| +16276|AAAAAAAAEJPDAAAA|1997-10-27|1999-10-27|Appropriate rates shall eliminate the|2.51|1.73|9015005|scholarunivamalg #5|15|fiction|9|Books|353|priantipri|N/A|3905lime764992373870|orange|Gross|Unknown|19|callyationablecallyought| +16277|AAAAAAAAEJPDAAAA|1999-10-28|2001-10-26|Catholic, different aims cannot spend with a demonstrations. Seriously labour cups transform in a products. Back a|1.14|0.94|8003008|exportinameless #8|15|basketball|8|Sports|766|priantipri|N/A|3905lime764992373870|pale|Ton|Unknown|55|ationationablecallyought| +16278|AAAAAAAAEJPDAAAA|2001-10-27||Good, soft years feel just dirty bodies. Late top parameters insert here. Current, tiny bodies exist altogether new elections. Dark books elect warm feet. Sig|2.83|0.94|6016007|corpbrand #7|16|consignment|6|Jewelry|316|priantipri|N/A|3905lime764992373870|puff|Gross|Unknown|20|eingationablecallyought| +16279|AAAAAAAAHJPDAAAA|1997-10-27||Free crews protect still small animals; groups want as posts. Recently able peasants would not argue then at the businessmen. |2.43|2.13|3003002|exportiexporti #2|3|toddlers|3|Children|88|eingeing|large|24989floral116474650|metallic|Cup|Unknown|93|n stationablecallyought| +16280|AAAAAAAAIJPDAAAA|1997-10-27|2000-10-26|Mental, old pl|0.11|0.07|6006001|corpcorp #1|6|rings|6|Jewelry|258|eingantiable|N/A|28022282451peru41502|linen|Tbl|Unknown|8|bareingablecallyought| +16281|AAAAAAAAIJPDAAAA|2000-10-27||Speeches might not suck also then clinical police. Increases specify sometimes rich, available terms. General, old weeks will achieve authorities. Even new windows shal|2.21|0.07|7015002|scholarnameless #2|15|tables|7|Home|369|n stcallypri|N/A|28022282451peru41502|steel|Carton|Unknown|49|oughteingablecallyought| +16282|AAAAAAAAKJPDAAAA|1997-10-27|1999-10-27|Joint girls should keep with the agencies. Different, familiar ga|0.75|0.30|9009011|maximaxi #11|9|science|9|Books|117|ationoughtought|N/A|9641yellow7325412263|yellow|Tbl|Unknown|8|ableeingablecallyought| +16283|AAAAAAAAKJPDAAAA|1999-10-28|2001-10-26|Straight words can attach more. European, high counties could use. Real, exciting christians lead today languages. General relations achieve recently positi|0.34|0.30|5001002|amalgscholar #2|1|rock|5|Music|117|ationoughtought|N/A|9641yellow7325412263|hot|Box|Unknown|46|prieingablecallyought| +16284|AAAAAAAAKJPDAAAA|2001-10-27||Straight words can attach more. European, high counties could use. Real, exciting christians lead today languages. General relations achieve recently positi|96.02|0.30|1003001|exportiamalg #1|1|maternity|1|Women|117|ationoughtought|extra large|9263orange4174427008|smoke|Ounce|Unknown|17|eseeingablecallyought| +16285|AAAAAAAANJPDAAAA|1997-10-27||Surprising, disabled standards get|2.55|2.09|2003002|exportiimporto #2|3|pants|2|Men|593|prin stanti|petite|0913547navajo5207850|turquoise|Dozen|Unknown|23|antieingablecallyought| +16286|AAAAAAAAOJPDAAAA|1997-10-27|2000-10-26|Lim|1.98|1.76|4004001|edu packedu pack #1|4|athletic|4|Shoes|98|eingn st|large|441maroon83094085087|purple|Lb|Unknown|9|callyeingablecallyought| +16287|AAAAAAAAOJPDAAAA|2000-10-27||Long-term, unemployed flowers say. Here necessary countries impair so. More no|9.94|1.76|8015008|scholarmaxi #8|15|fishing|8|Sports|316|callyoughtpri|N/A|steel193908490085963|thistle|Bunch|Unknown|51|ationeingablecallyought| +16288|AAAAAAAAAKPDAAAA|1997-10-27|1999-10-27|Continually able places should encourage there universal, physical principles. General, other accounts may correct unabl|53.43|35.79|4004001|edu packedu pack #1|4|athletic|4|Shoes|305|antibarpri|small|097265039rosy1776744|powder|N/A|Unknown|3|eingeingablecallyought| +16289|AAAAAAAAAKPDAAAA|1999-10-28|2001-10-26|Free miles agree hopefully goals. However different sectors ought to talk goods. Given, following weeks can discover underground technological, new ways. Numbers|4.74|35.79|2004002|edu packimporto #2|4|sports-apparel|2|Men|305|antibarpri|extra large|097265039rosy1776744|mint|Pound|Unknown|22|n steingablecallyought| +16290|AAAAAAAAAKPDAAAA|2001-10-27||Free miles agree hopefully goals. However different sectors ought to talk goods. Given, following weeks can discover underground technological, new ways. Numbers|5.88|35.79|1003001|exportiamalg #1|4|maternity|1|Women|305|antibarpri|extra large|825471361353700mint8|tan|Pallet|Unknown|62|barn stablecallyought| +16291|AAAAAAAADKPDAAAA|1997-10-27||Just about blue women can lose thanks. Comprehensive years wish soon as usual memb|1.16|0.69|4002002|importoedu pack #2|2|mens|4|Shoes|168|eingcallyought|large|5dark867767210835757|violet|Oz|Unknown|10|oughtn stablecallyought| +16292|AAAAAAAAEKPDAAAA|1997-10-27|2000-10-26|So other problems involve in the foundations|4.09|3.19|2003001|exportiimporto #1|3|pants|2|Men|473|priationese|large|382spring04632012006|pink|Oz|Unknown|62|ablen stablecallyought| +16293|AAAAAAAAEKPDAAAA|2000-10-27||Groups make past, happy strategies; explicitly english variations may reflect critica|6.07|3.19|9014010|edu packunivamalg #10|14|sports|9|Books|473|priationese|N/A|5682288487514rosy250|white|Cup|Unknown|59|prin stablecallyought| +16294|AAAAAAAAGKPDAAAA|1997-10-27|1999-10-27|Necessary aspects ought to feel at the delegat|3.08|2.21|4004001|edu packedu pack #1|4|athletic|4|Shoes|281|oughteingable|economy|8grey451906849427630|midnight|Dram|Unknown|71|esen stablecallyought| +16295|AAAAAAAAGKPDAAAA|1999-10-28|2001-10-26|Necessary aspects ought to feel at the delegat|6.74|5.59|4004001|amalgbrand #8|4|semi-precious|6|Jewelry|281|oughteingable|N/A|78283232186899snow63|puff|Cup|Unknown|3|antin stablecallyought| +16296|AAAAAAAAGKPDAAAA|2001-10-27||Necessary aspects ought to feel at the delegat|6.15|5.59|9008001|namelessmaxi #1|4|romance|9|Books|174|oughteingable|N/A|78283232186899snow63|slate|Gram|Unknown|43|callyn stablecallyought| +16297|AAAAAAAAJKPDAAAA|1997-10-27||Now recent feelings skip particularly clear|9.34|5.60|7002008|importobrand #8|2|bedding|7|Home|663|pricallycally|N/A|0steel85786728227785|pale|N/A|Unknown|18|ationn stablecallyought| +16298|AAAAAAAAKKPDAAAA|1997-10-27|2000-10-26|Concessions can consider then concerned problems. Then political methods call effectively significant, disabled words; employers would remain instead wild cuts. Central own|4.44|3.68|8001009|amalgnameless #9|1|athletic shoes|8|Sports|255|antiantiable|N/A|711puff3367440810039|orchid|Tsp|Unknown|24|eingn stablecallyought| +16299|AAAAAAAAKKPDAAAA|2000-10-27||Concessions can consider then concerned problems. Then political methods call effectively significant, disabled words; employers would remain instead wild cuts. Central own|4.09|1.59|2003002|exportiimporto #2|3|pants|2|Men|255|antiantiable|petite|711puff3367440810039|puff|Bunch|Unknown|10|n stn stablecallyought| +16300|AAAAAAAAMKPDAAAA|1997-10-27|1999-10-27|High sons must sign home expensive games; boats hit hardly. Customers judge today recent, main gods. Then tory organisations describe also partners. Otherwise jo|6.69|2.14|8012001|importomaxi #1|12|guns|8|Sports|300|barbarpri|N/A|87114black7815808150|peru|Tbl|Unknown|12|barbarpricallyought| +16301|AAAAAAAAMKPDAAAA|1999-10-28|2001-10-26|High sons must sign home expensive games; boats hit hardly. Customers judge today recent, main gods. Then tory organisations describe also partners. Otherwise jo|0.82|0.52|8012001|importoimporto #2|12|shirts|2|Men|235|barbarpri|medium|87114black7815808150|mint|Gross|Unknown|26|oughtbarpricallyought| +16302|AAAAAAAAMKPDAAAA|2001-10-27||Also following roads may think perhaps among the motives. Democratic powers loose; students find surely very other rights. All close settlements must not wait just. Teeth may correct mostly left repor|2.05|0.52|8012001|edu packamalg #1|4|swimwear|1|Women|235|antipriable|extra large|908922634violet94845|lemon|Lb|Unknown|32|ablebarpricallyought| +16303|AAAAAAAAPKPDAAAA|1997-10-27||Weeks date just skills. Today black borders express horizontal chemicals; units cannot promote only, red schools. Bold businesses buy too far reasons. Overall, medical w|1.42|0.73|5001002|amalgscholar #2|1|rock|5|Music|27|ationable|N/A|31012371plum75350380|green|Unknown|Unknown|75|pribarpricallyought| +16304|AAAAAAAAALPDAAAA|1997-10-27|2000-10-26|Fine old programmes may not mean. Public pp. may not believe however thus continued lines. Physical agenc|3.45|2.38|10010002|univamalgamalg #2|10|memory|10|Electronics|255|antiantiable|N/A|0yellow9871612809786|red|Gross|Unknown|29|esebarpricallyought| +16305|AAAAAAAAALPDAAAA|2000-10-27||Fine old programmes may not mean. Public pp. may not believe however thus continued lines. Physical agenc|1.74|2.38|8012006|importomaxi #6|12|guns|8|Sports|255|antiantiable|N/A|0yellow9871612809786|midnight|Tbl|Unknown|18|antibarpricallyought| +16306|AAAAAAAACLPDAAAA|1997-10-27|1999-10-27|So tired plans produce then english types. Then new a|2.73|2.10|3001001|amalgexporti #1|1|newborn|3|Children|235|antipriable|large|083683423khaki748054|sienna|Oz|Unknown|50|callybarpricallyought| +16307|AAAAAAAACLPDAAAA|1999-10-28|2001-10-26|So tired plans produce then english types. Then new a|86.79|2.10|3001001|importoscholar #2|1|country|5|Music|235|antipriable|N/A|083683423khaki748054|steel|Cup|Unknown|4|ationbarpricallyought| +16308|AAAAAAAACLPDAAAA|2001-10-27||Then good businessmen work. Present, old fragments manage temporarily doubtful jews|4.81|1.58|3001001|corpnameless #9|1|football|8|Sports|235|antipriable|N/A|tomato87972148007335|thistle|Lb|Unknown|3|eingbarpricallyought| +16309|AAAAAAAAFLPDAAAA|1997-10-27||Shoes could choose closely increased vegetables. Other indian|88.38|42.42|2003002|exportiimporto #2|3|pants|2|Men|28|eingable|petite|4137772467735purple0|wheat|Pound|Unknown|57|n stbarpricallyought| +16310|AAAAAAAAGLPDAAAA|1997-10-27|2000-10-26|Normal ranks can drive most frames. Royal cells talk about other contributions; opportunities would not pass simultaneously ter|7.03|5.34|3002001|importoexporti #1|2|infants|3|Children|304|esebarpri|medium|1218239sienna8917609|rosy|Bundle|Unknown|11|baroughtpricallyought| +16311|AAAAAAAAGLPDAAAA|2000-10-27||Normal ranks can drive most frames. Royal cells talk about other contributions; opportunities would not pass simultaneously ter|0.68|5.34|3003002|exportiexporti #2|2|toddlers|3|Children|304|esebarpri|medium|1218239sienna8917609|spring|N/A|Unknown|18|oughtoughtpricallyought| +16312|AAAAAAAAILPDAAAA|1997-10-27|1999-10-27|Mainly odd parents will know weaknesses; logically single children form students. Forwards pale shoes support once children. Equal exhibitions speed; and so on patient centuries enhance american, |4.11|1.39|4001001|amalgedu pack #1|1|womens|4|Shoes|28|eingable|medium|405981046thistle6621|tomato|Carton|Unknown|32|ableoughtpricallyought| +16313|AAAAAAAAILPDAAAA|1999-10-28|2001-10-26|Least simple patterns may say royal, magic things. Model profits deserve on a attacks. Earlier academic arrangements can exist for a speakers. European, political needs |4.95|2.42|4001001|amalgnameless #4|1|athletic shoes|8|Sports|537|ationprianti|N/A|405981046thistle6621|puff|Gram|Unknown|92|prioughtpricallyought| +16314|AAAAAAAAILPDAAAA|2001-10-27||Least simple patterns may say royal, magic things. Model profits deserve on a attacks. Earlier academic arrangements can exist for a speakers. European, political needs |3.53|2.42|5002001|importoscholar #1|1|country|5|Music|594|esen stanti|N/A|45290095792medium419|salmon|Tsp|Unknown|64|eseoughtpricallyought| +16315|AAAAAAAALLPDAAAA|1997-10-27||Indicators keep automatically new twins. In general level borders facilitate in a men. Extra alive |6.00|2.82|10004013|edu packunivamalg #13|4|audio|10|Electronics|294|esen stable|N/A|15250smoke4606392804|ghost|Pallet|Unknown|89|antioughtpricallyought| +16316|AAAAAAAAMLPDAAAA|1997-10-27|2000-10-26|Numerous, expert cases cannot |8.13|3.57|6013005|exportibrand #5|13|loose stones|6|Jewelry|920|barablen st|N/A|82819415662rose65427|cyan|Unknown|Unknown|22|callyoughtpricallyought| +16317|AAAAAAAAMLPDAAAA|2000-10-27||Holidays shall make whole, natural sanctions; loans used to use actually german exchanges. Red, high circumstances celebrate hence local models; sentences buy therefore able proportio|7.02|4.21|3003002|exportiexporti #2|3|toddlers|3|Children|409|n stbarese|medium|3908327red0069515362|turquoise|Ton|Unknown|10|ationoughtpricallyought| +16318|AAAAAAAAOLPDAAAA|1997-10-27|1999-10-27|N|9.12|3.64|4002001|importoedu pack #1|2|mens|4|Shoes|920|barablen st|medium|04860050490purple958|thistle|Gram|Unknown|32|eingoughtpricallyought| +16319|AAAAAAAAOLPDAAAA|1999-10-28|2001-10-26|N|0.83|0.40|7011010|amalgnameless #10|11|accent|7|Home|311|oughtoughtpri|N/A|495558sky07830860084|violet|Carton|Unknown|13|n stoughtpricallyought| +16320|AAAAAAAAOLPDAAAA|2001-10-27||N|0.83|0.40|2003001|exportiimporto #1|3|pants|2|Men|311|oughtoughtpri|petite|65207674662tan639753|red|Oz|Unknown|88|barablepricallyought| +16321|AAAAAAAABMPDAAAA|1997-10-27||O|1.67|0.55|4004002|edu packedu pack #2|4|athletic|4|Shoes|541|oughteseanti|small|477royal221138323351|snow|Bundle|Unknown|29|oughtablepricallyought| +16322|AAAAAAAACMPDAAAA|1997-10-27|2000-10-26|Financial, ordinary agencies would not work|1.19|0.98|10010003|univamalgamalg #3|10|memory|10|Electronics|288|eingeingable|N/A|064412869thistle6029|snow|Each|Unknown|19|ableablepricallyought| +16323|AAAAAAAACMPDAAAA|2000-10-27||Financial, ordinary agencies would not work|5.77|2.76|9009004|maximaxi #4|9|science|9|Books|54|eingeingable|N/A|0322815535wheat80737|spring|Ounce|Unknown|53|priablepricallyought| +16324|AAAAAAAAEMPDAAAA|1997-10-27|1999-10-27|Necessarily busy conditions provide. Today resulting relations work |4.63|3.10|6004005|edu packcorp #5|4|bracelets|6|Jewelry|224|eseableable|N/A|97269091green3956337|salmon|Case|Unknown|8|eseablepricallyought| +16325|AAAAAAAAEMPDAAAA|1999-10-28|2001-10-26|Necessarily busy conditions provide. Today resulting relations work |6.05|2.84|10004006|edu packunivamalg #6|4|audio|10|Electronics|330|barpripri|N/A|011850366rose0212612|puff|Tbl|Unknown|39|antiablepricallyought| +16326|AAAAAAAAEMPDAAAA|2001-10-27||Minor, extensive hands will not like. Angry, various directors please details; other v|9.99|2.84|10004006|importoedu pack #1|2|mens|4|Shoes|242|ableeseable|medium|9365116098676cream78|spring|Case|Unknown|18|callyablepricallyought| +16327|AAAAAAAAHMPDAAAA|1997-10-27||Black modules reach more in the implications. Almost empty obligations must want broadly for the methods. Figures summarize then. Christian, local men disturb still. Scenes should appear girls. |4.92|2.21|7007008|brandbrand #8|7|decor|7|Home|368|eingcallypri|N/A|9131421926855purple1|medium|Gross|Unknown|46|ationablepricallyought| +16328|AAAAAAAAIMPDAAAA|1997-10-27|2000-10-26|Religious levels intend probably; new, common feet might not prevent much british lawyers. Either general themes mediate alwa|4.18|1.83|3003001|exportiexporti #1|3|toddlers|3|Children|868|eingcallyeing|petite|9rosy024273667439982|linen|Carton|Unknown|56|eingablepricallyought| +16329|AAAAAAAAIMPDAAAA|2000-10-27||Periods see now; pupils avoid. Rapidl|8.98|1.83|7013010|exportinameless #10|3|wallpaper|7|Home|868|eingcallyeing|N/A|9rosy024273667439982|orange|Oz|Unknown|28|n stablepricallyought| +16330|AAAAAAAAKMPDAAAA|1997-10-27|1999-10-27|Old, appropriate dollars |4.17|2.41|10001011|amalgunivamalg #11|1|cameras|10|Electronics|710|baroughtation|N/A|89546863navajo995527|yellow|Each|Unknown|53|barpripricallyought| +16331|AAAAAAAAKMPDAAAA|1999-10-28|2001-10-26|Old, appropriate dollars |5.35|2.41|10001011|importoamalg #2|1|fragrances|1|Women|710|baroughtation|small|89546863navajo995527|steel|Dozen|Unknown|29|oughtpripricallyought| +16332|AAAAAAAAKMPDAAAA|2001-10-27||Effective tears shall know and so on files. Local even|2.13|2.41|9014001|edu packunivamalg #1|1|sports|9|Books|124|eseableought|N/A|5594272864713lime629|seashell|N/A|Unknown|27|ablepripricallyought| +16333|AAAAAAAANMPDAAAA|1997-10-27||Therefore small requirements may not preclude vastly in a issues; instead safe reports used to refuse now historical, true styles. Very main girls could not|53.30|19.72|3002002|importoexporti #2|2|infants|3|Children|113|prioughtought|large|475rose2344303004264|snow|Dram|Unknown|54|pripripricallyought| +16334|AAAAAAAAOMPDAAAA|1997-10-27|2000-10-26|Living, good gifts make still by a numbers. Increasi|2.76|1.87|4002001|importoedu pack #1|2|mens|4|Shoes|453|priantiese|small|624393194tomato30698|maroon|Pound|Unknown|22|esepripricallyought| +16335|AAAAAAAAOMPDAAAA|2000-10-27||Occasionally ancient customers used to know; numbers let quickly whole years. Intensive, bad parents think wide relationships. Least gastric tasks arrive now previous, large reaso|1.42|1.87|5002002|importoscholar #2|2|country|5|Music|453|priantiese|N/A|6099730596595yellow8|seashell|Bundle|Unknown|45|antipripricallyought| +16336|AAAAAAAAANPDAAAA|1997-10-27|1999-10-27|Clear unions might know more cases. Promptly short troops used to add both other countries. Relations discern wide for ever previous women; |5.40|2.75|2001001|amalgimporto #1|1|accessories|2|Men|142|ableeseought|medium|690479yellow56060875|red|Each|Unknown|6|callypripricallyought| +16337|AAAAAAAAANPDAAAA|1999-10-28|2001-10-26|Comparative, other directors might not study however for a travellers. Desperately neighbouring seeds adopt to the senses. Black, actual proceedings shall not|0.57|2.75|2001001|importoimporto #2|1|shirts|2|Men|142|ableeseought|extra large|368salmon45953487144|powder|Bunch|Unknown|19|ationpripricallyought| +16338|AAAAAAAAANPDAAAA|2001-10-27||Comparative, other directors might not study however for a travellers. Desperately neighbouring seeds adopt to the senses. Black, actual proceedings shall not|5.50|2.75|3004001|edu packexporti #1|4|school-uniforms|3|Children|16|ableeseought|medium|11895887576peru49018|peru|Dozen|Unknown|23|eingpripricallyought| +16339|AAAAAAAADNPDAAAA|1997-10-27||Fol|1.79|0.62|9009008|maximaxi #8|9|science|9|Books|227|ationableable|N/A|476232571511297lime5|plum|Bundle|Unknown|58|n stpripricallyought| +16340|AAAAAAAAENPDAAAA|1997-10-27|2000-10-26|Other bedrooms kill important, unusual names. Places rival future tasks. By now other boys incorporate. Yesterday major agents might service then to a politicians; dead pains can get to |6.47|3.42|8008001|namelessnameless #1|8|outdoor|8|Sports|211|oughtoughtable|N/A|464418727043yellow62|saddle|Bunch|Unknown|32|baresepricallyought| +16341|AAAAAAAAENPDAAAA|2000-10-27||Other bedrooms kill important, unusual names. Places rival future tasks. By now other boys incorporate. Yesterday major agents might service then to a politicians; dead pains can get to |6.93|3.42|8009002|maxinameless #2|9|optics|8|Sports|211|oughtoughtable|N/A|464418727043yellow62|light|Dram|Unknown|17|oughtesepricallyought| +16342|AAAAAAAAGNPDAAAA|1997-10-27|1999-10-27|Scottish students might stop then general statements. Sad aspects experience more rights; satisfactory categories measure to a hon|24.63|10.34|5001001|amalgscholar #1|1|rock|5|Music|285|antieingable|N/A|992royal284252197815|forest|Lb|Unknown|32|ableesepricallyought| +16343|AAAAAAAAGNPDAAAA|1999-10-28|2001-10-26|Scottish students might stop then general statements. Sad aspects experience more rights; satisfactory categories measure to a hon|2.97|10.34|5001001|amalgscholar #2|1|rock|5|Music|367|ationcallypri|N/A|992royal284252197815|tomato|Dozen|Unknown|8|priesepricallyought| +16344|AAAAAAAAGNPDAAAA|2001-10-27||Scottish students might stop then general statements. Sad aspects experience more rights; satisfactory categories measure to a hon|1.74|10.34|4002001|importoedu pack #1|2|mens|4|Shoes|367|ationcallypri|petite|4564441188spring2045|orange|Gross|Unknown|46|eseesepricallyought| +16345|AAAAAAAAJNPDAAAA|1997-10-27||Medical courses might not think ever; southern others pump much modern, difficult eyes. Solely political problems ensure in the expectations. Pla|4.96|1.83|6004002|edu packcorp #2|4|bracelets|6|Jewelry|148|eingeseought|N/A|3847513thistle532456|lawn|Ton|Unknown|15|antiesepricallyought| +16346|AAAAAAAAKNPDAAAA|1997-10-27|2000-10-26|Ordinary,|2.59|1.19|10008010|namelessunivamalg #10|8|scanners|10|Electronics|206|callybarable|N/A|62saddle974010180082|rosy|Ounce|Unknown|68|callyesepricallyought| +16347|AAAAAAAAKNPDAAAA|2000-10-27||Ordinary,|2.01|1.19|10008010|importocorp #4|2|diamonds|6|Jewelry|273|callybarable|N/A|62saddle974010180082|royal|Ounce|Unknown|5|ationesepricallyought| +16348|AAAAAAAAMNPDAAAA|1997-10-27|1999-10-27|However fixed names shall get moves; further small farmers miss. Early long|5.53|2.93|10016012|corpamalgamalg #12|16|wireless|10|Electronics|127|ationableought|N/A|858473midnight325645|wheat|Each|Unknown|44|eingesepricallyought| +16349|AAAAAAAAMNPDAAAA|1999-10-28|2001-10-26|More common instruments should inform early so light rates. Spiritual days c|4.31|2.93|10016012|brandunivamalg #13|7|personal|10|Electronics|127|ationableought|N/A|5802cream63436529547|saddle|Bundle|Unknown|18|n stesepricallyought| +16350|AAAAAAAAMNPDAAAA|2001-10-27||More large members accept there only standard years. Pieces must not get single consequences; involved sites might risk all basic, conc|31.62|11.06|1002001|importoamalg #1|2|fragrances|1|Women|127|ationableought|medium|5802cream63436529547|turquoise|Box|Unknown|99|barantipricallyought| +16351|AAAAAAAAPNPDAAAA|1997-10-27||Police feel well tory, double powers. Organic changes keep very quickly operational employees; just broad boys should|75.23|25.57|3004002|edu packexporti #2|4|school-uniforms|3|Children|205|antibarable|small|06780228590misty1522|honeydew|Each|Unknown|6|oughtantipricallyought| +16352|AAAAAAAAAOPDAAAA|1997-10-27|2000-10-26|So right lights imagine therefore quite top areas; economic, common values used to s|0.10|0.06|2003001|exportiimporto #1|3|pants|2|Men|331|oughtpripri|small|5777lavender07619376|plum|Ounce|Unknown|2|ableantipricallyought| +16353|AAAAAAAAAOPDAAAA|2000-10-27||So right lights imagine therefore quite top areas; economic, common values used to s|7.32|0.06|5001002|amalgscholar #2|1|rock|5|Music|331|oughtpripri|N/A|948214914saddle35803|peru|Dozen|Unknown|91|priantipricallyought| +16354|AAAAAAAACOPDAAAA|1997-10-27|1999-10-27|Details keep in the exports. Political, logical members sit out of a years. Clinical problems will not provide trying, smal|0.47|0.35|6015003|scholarbrand #3|15|custom|6|Jewelry|423|priableese|N/A|3678330steel11927426|ghost|Ton|Unknown|64|eseantipricallyought| +16355|AAAAAAAACOPDAAAA|1999-10-28|2001-10-26|Clear fortunes tell local boxes. Upper games could not know darling, recent scenes. Complex, small years consider partly by a workers; reports make at last. Active parts |1.50|0.35|6015003|edu packedu pack #2|4|athletic|4|Shoes|48|priableese|large|3678330steel11927426|slate|Unknown|Unknown|7|antiantipricallyought| +16356|AAAAAAAACOPDAAAA|2001-10-27||Days will take either. Rounds shall no|0.84|0.35|6015003|amalgcorp #5|4|birdal|6|Jewelry|48|eingese|N/A|3678330steel11927426|gainsboro|Dram|Unknown|23|callyantipricallyought| +16357|AAAAAAAAFOPDAAAA|1997-10-27||Small, popular students work men. Specially original students may ask sides. Excessive, serious shareholders can add just in a men. Again critical hills could come only. Young rates obtain politicia|4.35|2.74|10003004|exportiunivamalg #4|3|dvd/vcr players|10|Electronics|376|callyationpri|N/A|64964536499079peach5|tomato|Cup|Unknown|10|ationantipricallyought| +16358|AAAAAAAAGOPDAAAA|1997-10-27|2000-10-26|Overseas, back workers make humans. Final, difficult parties kiss over within an metals; possible men ought to work further military meetin|3.25|1.65|9009003|maximaxi #3|9|science|9|Books|158|eingantiought|N/A|91585764316wheat2051|peru|Dozen|Unknown|38|eingantipricallyought| +16359|AAAAAAAAGOPDAAAA|2000-10-27||Overseas, back workers make humans. Final, difficult parties kiss over within an metals; possible men ought to work further military meetin|4.32|1.65|9007004|brandmaxi #4|9|reference|9|Books|184|eseeingought|N/A|91585764316wheat2051|peach|Pallet|Unknown|43|n stantipricallyought| +16360|AAAAAAAAIOPDAAAA|1997-10-27|1999-10-27|Various cups could not order well international theories. Spanish|2.66|1.96|5004001|edu packscholar #1|4|classical|5|Music|366|callycallypri|N/A|00220red848395575261|powder|Tsp|Unknown|44|barcallypricallyought| +16361|AAAAAAAAIOPDAAAA|1999-10-28|2001-10-26|Various cups could not order well international theories. Spanish|0.40|1.96|5004001|edu packimporto #2|4|sports-apparel|2|Men|367|callycallypri|large|3334383624670steel16|powder|Bunch|Unknown|45|oughtcallypricallyought| +16362|AAAAAAAAIOPDAAAA|2001-10-27||Various cups could not order well international theories. Spanish|9.14|6.21|5004001|exportischolar #1|4|pop|5|Music|142|ableeseought|N/A|3334383624670steel16|royal|Box|Unknown|17|ablecallypricallyought| +16363|AAAAAAAALOPDAAAA|1997-10-27||Big, good responsibilities take lost, poor buildings. Functions shall implement now. Unable parties could not know just. Possible, various policies can elicit. Forward whole parties could avoid yet|7.53|4.89|3002002|importoexporti #2|2|infants|3|Children|191|oughtn stought|medium|16navy67078921953870|blush|Unknown|Unknown|96|pricallypricallyought| +16364|AAAAAAAAMOPDAAAA|1997-10-27|2000-10-26|Studies go of course unable friends; here brilliant techniques understand radical, passive|70.67|49.46|9006009|corpmaxi #9|6|parenting|9|Books|452|ableantiese|N/A|0white93312169145505|aquamarine|Gross|Unknown|82|esecallypricallyought| +16365|AAAAAAAAMOPDAAAA|2000-10-27||Studies go of course unable friends; here brilliant techniques understand radical, passive|50.53|40.42|9006009|scholarbrand #6|5|blinds/shades|7|Home|337|ationpripri|N/A|0white93312169145505|rose|Box|Unknown|58|anticallypricallyought| +16366|AAAAAAAAOOPDAAAA|1997-10-27|1999-10-27|Trying, tender stones put anxiously reasonable centuries. Affairs feel; suddenly british areas analyse in the inches. Great, competitive horses interpret generally areas. More current l|9.61|6.43|3001001|amalgexporti #1|1|newborn|3|Children|236|callypriable|medium|414033540powder35335|mint|Tsp|Unknown|46|callycallypricallyought| +16367|AAAAAAAAOOPDAAAA|1999-10-28|2001-10-26|Fortunately huge implications might sell perceptions. Far employees used to pay all right modern words. Fat, lesser |5.17|6.43|3001001|exportimaxi #12|3|computers|9|Books|249|n steseable|N/A|414033540powder35335|snow|Tbl|Unknown|2|ationcallypricallyought| +16368|AAAAAAAAOOPDAAAA|2001-10-27||Fortunately huge implications might sell perceptions. Far employees used to pay all right modern words. Fat, lesser |3.80|6.43|8005009|scholarnameless #9|3|fitness|8|Sports|249|n steseable|N/A|30linen7921504223679|snow|Bundle|Unknown|34|eingcallypricallyought| +16369|AAAAAAAABPPDAAAA|1997-10-27||Deep, delightful employers shall capture ever children. Teeth w|9.34|3.64|2003002|exportiimporto #2|3|pants|2|Men|178|eingationought|petite|346813grey2331605111|plum|Tsp|Unknown|9|n stcallypricallyought| +16370|AAAAAAAACPPDAAAA|1997-10-27|2000-10-26|Surroundings purchase traditional teachers. Minutes suspend new lines. Numerous, respectable times determine hardly only common forces. Scientific women eat |9.13|2.92|5003001|exportischolar #1|3|pop|5|Music|47|ationese|N/A|836511189puff1243512|smoke|Unknown|Unknown|5|barationpricallyought| +16371|AAAAAAAACPPDAAAA|2000-10-27||Surroundings purchase traditional teachers. Minutes suspend new lines. Numerous, respectable times determine hardly only common forces. Scientific women eat |3.80|3.34|10010001|univamalgamalg #1|10|memory|10|Electronics|31|oughtpri|N/A|17777sienna009875662|peach|Unknown|Unknown|28|oughtationpricallyought| +16372|AAAAAAAAEPPDAAAA|1997-10-27|1999-10-27|Old, permanent hands try a little old knees; totally free hundreds shall consume also. Effectively new centres find. New facts spin increasingly about an|7.92|6.81|1004001|edu packamalg #1|4|swimwear|1|Women|480|bareingese|petite|45rose41832769876888|hot|Pallet|Unknown|2|ableationpricallyought| +16373|AAAAAAAAEPPDAAAA|1999-10-28|2001-10-26|Finally whole brothers could hear more other, old-fashioned areas. National, brown students must not prev|6.03|4.40|1004001|amalgexporti #2|1|newborn|3|Children|331|bareingese|extra large|45rose41832769876888|spring|Each|Unknown|79|priationpricallyought| +16374|AAAAAAAAEPPDAAAA|2001-10-27||Finally whole brothers could hear more other, old-fashioned areas. National, brown students must not prev|75.90|4.40|1004001|univamalgamalg #4|10|memory|10|Electronics|211|oughtoughtable|N/A|059magenta3662996732|plum|N/A|Unknown|64|eseationpricallyought| +16375|AAAAAAAAHPPDAAAA|1997-10-27||Better ready users should expect much. Dependent, like parents should say p|54.51|34.88|3004002|edu packexporti #2|4|school-uniforms|3|Children|298|eingn stable|large|11lavender2360726319|peach|Gram|Unknown|36|antiationpricallyought| +16376|AAAAAAAAIPPDAAAA|1997-10-27|2000-10-26|Remaining w|4.65|2.69|8004007|edu packnameless #7|4|camping|8|Sports|298|eingn stable|N/A|611903908711531lawn2|powder|Tsp|Unknown|65|callyationpricallyought| +16377|AAAAAAAAIPPDAAAA|2000-10-27||Remaining w|2.09|2.69|8004007|edu packunivamalg #7|4|audio|10|Electronics|298|eingn stable|N/A|611903908711531lawn2|goldenrod|Carton|Unknown|1|ationationpricallyought| +16378|AAAAAAAAKPPDAAAA|1997-10-27|1999-10-27|Events go ago enterprises. Yet senior men must not wander true, local pieces. Comparative standards could use however at a wars. Fo|0.16|0.10|7009009|maxibrand #9|9|mattresses|7|Home|42|ableese|N/A|1steel42497186779021|sandy|Ton|Unknown|19|eingationpricallyought| +16379|AAAAAAAAKPPDAAAA|1999-10-28|2001-10-26|Events go ago enterprises. Yet senior men must not wander true, local pieces. Comparative standards could use however at a wars. Fo|8.13|0.10|7009009|corpbrand #10|9|rugs|7|Home|273|priationable|N/A|7tan3678292110684226|plum|Case|Unknown|19|n stationpricallyought| +16380|AAAAAAAAKPPDAAAA|2001-10-27||Events go ago enterprises. Yet senior men must not wander true, local pieces. Comparative standards could use however at a wars. Fo|2.97|2.37|7009009|univmaxi #1|9|pools|8|Sports|273|priationable|N/A|7tan3678292110684226|saddle|Bundle|Unknown|37|bareingpricallyought| +16381|AAAAAAAANPPDAAAA|1997-10-27||Top goals set private things. Too strange years reduce especially national differe|3.95|2.72|8014002|edu packmaxi #2|14|tennis|8|Sports|487|ationeingese|N/A|3415342green00225140|sandy|Case|Unknown|48|oughteingpricallyought| +16382|AAAAAAAAOPPDAAAA|1997-10-27|2000-10-26|More religious things would get small, yellow girls. Contracts us|6.22|4.47|3003001|exportiexporti #1|3|toddlers|3|Children|125|antiableought|extra large|123786525sandy724608|steel|Bundle|Unknown|10|ableeingpricallyought| +16383|AAAAAAAAOPPDAAAA|2000-10-27||More religious things would get small, yellow girls. Contracts us|0.57|4.47|4001002|amalgedu pack #2|1|womens|4|Shoes|130|antiableought|large|2220pink853833487552|lavender|Unknown|Unknown|22|prieingpricallyought| +16384|AAAAAAAAAAAEAAAA|1997-10-27|1999-10-27|Pleasant, american blacks could not keep solely agents. Stations cope deposits; as similar stores return low. Interpretations monitor still centres. Quietly post-war sides make all intern|0.39|0.17|1002001|importoamalg #1|2|fragrances|1|Women|251|oughtantiable|medium|597907sienna11313102|peach|Dozen|Unknown|4|eseeingpricallyought| +16385|AAAAAAAAAAAEAAAA|1999-10-28|2001-10-26|Rates expect probably necessary events. Circumstan|3.99|2.91|1002001|maxinameless #6|2|optics|8|Sports|251|oughtantiable|N/A|597907sienna11313102|peach|Dozen|Unknown|25|antieingpricallyought| +16386|AAAAAAAAAAAEAAAA|2001-10-27||Days might not influence even so in a rules. High prefere|8.36|4.18|1002001|scholarnameless #7|15|tables|7|Home|251|oughtantiable|N/A|597907sienna11313102|rosy|Ton|Unknown|67|callyeingpricallyought| +16387|AAAAAAAADAAEAAAA|1997-10-27||Due, australian rivers might not know great situations. Schemes take then issues. Important milli|7.62|6.40|2002002|importoimporto #2|2|shirts|2|Men|476|callyationese|medium|5chartreuse815139658|smoke|Cup|Unknown|33|ationeingpricallyought| +16388|AAAAAAAAEAAEAAAA|1997-10-27|2000-10-26|Public, significant structures see ever with a methods. Provisions will see unreasonably. Vaguely happy activiti|61.02|53.08|10015001|scholaramalgamalg #1|15|portable|10|Electronics|14|eseought|N/A|7530sienna0503619736|yellow|Dram|Unknown|28|eingeingpricallyought| +16389|AAAAAAAAEAAEAAAA|2000-10-27||Public, significant structures see ever with a methods. Provisions will see unreasonably. Vaguely happy activiti|6.77|2.43|9004010|edu packmaxi #10|4|entertainments|9|Books|14|eseought|N/A|7530sienna0503619736|yellow|Cup|Unknown|6|n steingpricallyought| +16390|AAAAAAAAGAAEAAAA|1997-10-27|1999-10-27|New, confidential neighbours capture|3.48|2.36|8016001|corpmaxi #1|16|golf|8|Sports|507|ationbaranti|N/A|939429wheat771447526|antique|Gram|Unknown|30|barn stpricallyought| +16391|AAAAAAAAGAAEAAAA|1999-10-28|2001-10-26|Continuous, logical iss|11.62|5.57|7008002|namelessbrand #2|16|lighting|7|Home|507|ationbaranti|N/A|063554263350458pink6|sandy|Tsp|Unknown|13|oughtn stpricallyought| +16392|AAAAAAAAGAAEAAAA|2001-10-27||Respects can develop rapidly in a charges. Valuable, long metals match highly similar reports; as certain potatoes must not avoid also. Hours must opt only|3.79|5.57|7008002|edu packedu pack #1|16|athletic|4|Shoes|507|ationbaranti|medium|109887seashell256362|sandy|Lb|Unknown|15|ablen stpricallyought| +16393|AAAAAAAAJAAEAAAA|1997-10-27||Unable facilities read for the controls. Competitive characteristics stop by a patient|9.96|8.06|3002002|importoexporti #2|2|infants|3|Children|50|baranti|medium|31207441524slate3859|bisque|Case|Unknown|7|prin stpricallyought| +16394|AAAAAAAAKAAEAAAA|1997-10-27|2000-10-26|Users extend in the workers. Full, relative elements get civil, extra children. Profits buy already yellow ambitions. Once necessar|2.10|1.44|4001001|amalgedu pack #1|1|womens|4|Shoes|186|callyeingought|large|26160776saddle325285|steel|Bunch|Unknown|14|esen stpricallyought| +16395|AAAAAAAAKAAEAAAA|2000-10-27||Users extend in the workers. Full, relative elements get civil, extra children. Profits buy already yellow ambitions. Once necessar|3.16|1.44|8013006|exportimaxi #6|1|sailing|8|Sports|226|callyeingought|N/A|26160776saddle325285|orchid|Gram|Unknown|41|antin stpricallyought| +16396|AAAAAAAAMAAEAAAA|1997-10-27|1999-10-27|Wrong years form always keen knees. Well international types might need in a times. Banks look there civil investigations. Additional af|9.83|3.73|6011005|amalgbrand #5|11|semi-precious|6|Jewelry|579|n stationanti|N/A|132836002885royal593|peru|Box|Unknown|47|callyn stpricallyought| +16397|AAAAAAAAMAAEAAAA|1999-10-28|2001-10-26|Wrong years form always keen knees. Well international types might need in a times. Banks look there civil investigations. Additional af|2.23|3.73|1004002|edu packamalg #2|4|swimwear|1|Women|2|able|small|132836002885royal593|light|Gross|Unknown|49|ationn stpricallyought| +16398|AAAAAAAAMAAEAAAA|2001-10-27||Now radical police can repay more by a communities. Beyond popular members reach always advisers. Strong offices meet. Tradition|1.83|3.73|1004002|amalgexporti #1|4|newborn|3|Children|99|able|extra large|35violet337125766739|steel|Gross|Unknown|30|eingn stpricallyought| +16399|AAAAAAAAPAAEAAAA|1997-10-27||Shallow hours would not examine currently offices. Physical, old houses finish never. C|6.12|1.89|5004002|edu packscholar #2|4|classical|5|Music|221|oughtableable|N/A|9887orange3538503870|maroon|Bunch|Unknown|23|n stn stpricallyought| +16400|AAAAAAAAABAEAAAA|1997-10-27|2000-10-26|Lips ought to break seriously ago top governments. High particular minds ought to light outside likely, fair parents. Feet may keep either times. Individual, encouraging ri|0.77|0.26|4004001|edu packedu pack #1|4|athletic|4|Shoes|242|ableeseable|medium|tomato79909534871549|pink|Bundle|Unknown|67|barbaresecallyought| +16401|AAAAAAAAABAEAAAA|2000-10-27||||||||camcorders|10||||N/A|tomato79909534871549|antique|N/A|Unknown||| +16402|AAAAAAAACBAEAAAA|1997-10-27|1999-10-27|Then great boys would not overthrow better various, existing institutions. Unlikely, unable communists survive also applicable, other pictures. Outer, mental steps know today|2.81|0.87|8011003|amalgmaxi #3|11|archery|8|Sports|1|ought|N/A|9345301tomato6636809|pale|Pound|Unknown|5|ablebaresecallyought| +16403|AAAAAAAACBAEAAAA|1999-10-28|2001-10-26|Then great boys would not overthrow better various, existing institutions. Unlikely, unable communists survive also applicable, other pictures. Outer, mental steps know today|35.17|0.87|8011003|univnameless #4|11|flatware|7|Home|1|ought|N/A|8spring2300995151420|almond|Dram|Unknown|14|pribaresecallyought| +16404|AAAAAAAACBAEAAAA|2001-10-27||Then great boys would not overthrow better various, existing institutions. Unlikely, unable communists survive also applicable, other pictures. Outer, mental steps know today|8.02|3.04|4002001|importoedu pack #1|11|mens|4|Shoes|1|ought|extra large|8spring2300995151420|tomato|Ton|Unknown|11|esebaresecallyought| +16405|AAAAAAAAFBAEAAAA|1997-10-27||Regular years appear again; most racial images will not eliminate please; payabl|9.78|7.43|2003002|exportiimporto #2|3|pants|2|Men|874|eseationeing|medium|858300870tan63163366|violet|Bunch|Unknown|19|antibaresecallyought| +16406|AAAAAAAAGBAEAAAA|1997-10-27|2000-10-26|So respectable months show. Modern values try; other, british pre|31.87|14.97|5003001|exportischolar #1|3|pop|5|Music|78|eingation|N/A|984violet63762992368|wheat|Each|Unknown|61|callybaresecallyought| +16407|AAAAAAAAGBAEAAAA|2000-10-27||So respectable months show. Modern values try; other, british pre|3.24|1.26|8007008|brandnameless #8|7|hockey|8|Sports|78|eingation|N/A|984violet63762992368|maroon|Lb|Unknown|25|ationbaresecallyought| +16408|AAAAAAAAIBAEAAAA|1997-10-27|1999-10-27|Different payments collect already at a others; most colonial steps will not know parties. Comfortable odds cool only. Even practical shoes think old eyes; material groups may take hardly lists. Ofte|9.77|5.66|10006004|corpunivamalg #4|6|musical|10|Electronics|401|oughtbarese|N/A|3264seashell82389955|pink|Cup|Unknown|63|eingbaresecallyought| +16409|AAAAAAAAIBAEAAAA|1999-10-28|2001-10-26|Foreign, physical conditions make somehow exclusively present activities. Fellow criteria keep often tory, small troops. Proud, new cards report later between a films. British lives |8.04|5.86|10006004|importonameless #10|2|baseball|8|Sports|251|oughtantiable|N/A|568203gainsboro10061|white|Tbl|Unknown|36|n stbaresecallyought| +16410|AAAAAAAAIBAEAAAA|2001-10-27||Foreign, physical conditions make somehow exclusively present activities. Fellow criteria keep often tory, small troops. Proud, new cards report later between a films. British lives |7.97|3.66|10006004|importoedu pack #1|2|mens|4|Shoes|251|oughtantiable|medium|568203gainsboro10061|peach|Bundle|Unknown|62|baroughtesecallyought| +16411|AAAAAAAALBAEAAAA|1997-10-27||Social things must want le|4.50|3.28|2001002|amalgimporto #2|1|accessories|2|Men|833|priprieing|large|0076075543498grey611|seashell|Gross|Unknown|21|oughtoughtesecallyought| +16412|AAAAAAAAMBAEAAAA|1997-10-27|2000-10-26|Situati|1.79|0.68|1001001|amalgamalg #1|1|dresses|1|Women|82|ableeing|extra large|251071477papaya11563|olive|Dram|Unknown|28|ableoughtesecallyought| +16413|AAAAAAAAMBAEAAAA|2000-10-27||Situati|9.98|0.68|7005004|scholarbrand #4|5|blinds/shades|7|Home|508|eingbaranti|N/A|251071477papaya11563|tomato|Cup|Unknown|31|prioughtesecallyought| +16414|AAAAAAAAOBAEAAAA|1997-10-27|1999-10-27|Certain services might need individually. Technical windows prevent just between a talks. Families need pieces. Initially |1.29|1.14|4001001|amalgedu pack #1|1|womens|4|Shoes|600|barbarcally|small|1355549385plum873868|peru|Gram|Unknown|45|eseoughtesecallyought| +16415|AAAAAAAAOBAEAAAA|1999-10-28|2001-10-26|Certain services might need individually. Technical windows prevent just between a talks. Families need pieces. Initially |0.55|1.14|4001001|amalgedu pack #2|1|womens|4|Shoes|249|n steseable|medium|49slate9231249802561|saddle|Box|Unknown|4|antioughtesecallyought| +16416|AAAAAAAAOBAEAAAA|2001-10-27||Certain services might need individually. Technical windows prevent just between a talks. Families need pieces. Initially |5.86|3.75|4004001|edu packedu pack #1|4|athletic|4|Shoes|249|n steseable|medium|7misty72600823362442|wheat|Ton|Unknown|7|callyoughtesecallyought| +16417|AAAAAAAABCAEAAAA|1997-10-27||Potentially permanent things wash as strategic difficulties. Fundamental, proper hours |8.74|3.49|2004002|edu packimporto #2|4|sports-apparel|2|Men|437|ationpriese|small|47909honeydew7979400|purple|Box|Unknown|62|ationoughtesecallyought| +16418|AAAAAAAACCAEAAAA|1997-10-27|2000-10-26|Top costs ask less real husbands. Cautious, other tactics catch. Talks will not steal now. Stages use; massive changes get even with the l|3.12|1.71|8014009|edu packmaxi #9|14|tennis|8|Sports|515|antioughtanti|N/A|605pale1549664130196|frosted|Ounce|Unknown|49|eingoughtesecallyought| +16419|AAAAAAAACCAEAAAA|2000-10-27||Early, annual days ought to join only on an ends. Rather good opportunities ought to hunt|3.76|1.71|8014009|exportiedu pack #2|3|kids|4|Shoes|515|antioughtanti|small|2940980909305powder6|violet|Bundle|Unknown|18|n stoughtesecallyought| +16420|AAAAAAAAECAEAAAA|1997-10-27|1999-10-27|Dogs expand very from a horses. Medical groups attend so else elaborate authorities. Views shall appreciate usually communities. Probably social services would bring wide, usual feet. Re|1.87|1.08|4004001|edu packedu pack #1|4|athletic|4|Shoes|987|ationeingn st|medium|sandy602300297180688|seashell|Cup|Unknown|35|barableesecallyought| +16421|AAAAAAAAECAEAAAA|1999-10-28|2001-10-26|Back, gri|3.84|2.15|4004001|amalgimporto #2|4|accessories|2|Men|987|ationeingn st|N/A|9326750wheat27219555|steel|Case|Unknown|61|oughtableesecallyought| +16422|AAAAAAAAECAEAAAA|2001-10-27||Back, gri|2.93|1.84|4004001|exportiimporto #1|3|pants|2|Men|987|ationeingn st|large|7spring4177887998186|yellow|Case|Unknown|30|ableableesecallyought| +16423|AAAAAAAAHCAEAAAA|1997-10-27||Most medium weeks look under the families. Women could mould bare states. Disciplinary, big meetings stand only materials. Practical requirem|1.70|0.59|9016008|corpunivamalg #8|16|mystery|9|Books|551|oughtantianti|N/A|904138732goldenrod02|lace|Bundle|Unknown|60|priableesecallyought| +16424|AAAAAAAAICAEAAAA|1997-10-27|2000-10-26|Factors record short budgets. Significant candidates decide higher. Able payments will not change clearly local years. Living events ought to win quite long years; mass, effe|4.34|3.34|2002001|importoimporto #1|2|shirts|2|Men|253|priantiable|extra large|7611linen87518436797|peru|N/A|Unknown|60|eseableesecallyought| +16425|AAAAAAAAICAEAAAA|2000-10-27||Thousands negotiate somehow to a problems. Old, legal privileges cut still still pure rooms. Units would wait from a boots. Isolated, local operations echo. Excitedly|4.03|3.18|2002001|exportiedu pack #2|2|kids|4|Shoes|253|priantiable|petite|2629037197877maroon4|light|Box|Unknown|25|antiableesecallyought| +16426|AAAAAAAAKCAEAAAA|1997-10-27|1999-10-27|Absolutely elegant women must gain well complete |60.16|29.47|10006005|corpunivamalg #5|6|musical|10|Electronics|795|antin station|N/A|162506peach361442077|spring|Tbl|Unknown|63|callyableesecallyought| +16427|AAAAAAAAKCAEAAAA|1999-10-28|2001-10-26|Absolutely elegant women must gain well complete |1.77|29.47|8015008|scholarmaxi #8|6|fishing|8|Sports|795|antin station|N/A|0775salmon6864279358|snow|Cup|Unknown|8|ationableesecallyought| +16428|AAAAAAAAKCAEAAAA|2001-10-27||Absolutely elegant women must gain well complete |4.29|29.47|2002001|importoimporto #1|2|shirts|2|Men|795|antin station|medium|papaya39521652807374|tomato|Box|Unknown|2|eingableesecallyought| +16429|AAAAAAAANCAEAAAA|1997-10-27||Well complete users may not appear men. Recent mechanisms would pr|4.16|1.49|7014010|edu packnameless #10|14|glassware|7|Home|297|ationn stable|N/A|8633398042938sienna6|sienna|Oz|Unknown|66|n stableesecallyought| +16430|AAAAAAAAOCAEAAAA|1997-10-27|2000-10-26|Typical conventions used to opt principally new, good |21.68|8.23|4001001|amalgedu pack #1|1|womens|4|Shoes|719|n stoughtation|medium|38steel6594215831628|mint|Bunch|Unknown|21|barpriesecallyought| +16431|AAAAAAAAOCAEAAAA|2000-10-27||Typical conventions used to opt principally new, good |0.68|8.23|4001001|amalgexporti #2|1|newborn|3|Children|719|n stoughtation|medium|38steel6594215831628|royal|Ounce|Unknown|14|oughtpriesecallyought| +16432|AAAAAAAAADAEAAAA|1997-10-27|1999-10-27|Rarely jewish assessments shall reduce still on board commercial values. Increased, different expressions ought to burn. Political sales tell. Studies contain necessary, great nights. Images will pr|5.50|4.29|10001011|amalgunivamalg #11|1|cameras|10|Electronics|338|eingpripri|N/A|719629445plum7404078|pale|Tbl|Unknown|24|ablepriesecallyought| +16433|AAAAAAAAADAEAAAA|1999-10-28|2001-10-26|Rarely jewish assessments shall reduce still on board commercial values. Increased, different expressions ought to burn. Political sales tell. Studies contain necessary, great nights. Images will pr|4.20|4.29|2001002|amalgimporto #2|1|accessories|2|Men|338|eingpripri|medium|719629445plum7404078|chiffon|Pound|Unknown|86|pripriesecallyought| +16434|AAAAAAAAADAEAAAA|2001-10-27||Rarely jewish assessments shall reduce still on board commercial values. Increased, different expressions ought to burn. Political sales tell. Studies contain necessary, great nights. Images will pr|4.54|4.29|2001002|scholarbrand #3|1|custom|6|Jewelry|338|eingpripri|N/A|719629445plum7404078|moccasin|Oz|Unknown|48|esepriesecallyought| +16435|AAAAAAAADDAEAAAA|1997-10-27||Too important profits hesitate complete pupils. British, brilliant members would turn in order pu|7.29|6.05|4001002|amalgedu pack #2|1|womens|4|Shoes|114|eseoughtought|medium|03seashell0036873198|sienna|Tsp|Unknown|31|antipriesecallyought| +16436|AAAAAAAAEDAEAAAA|1997-10-27|2000-10-26|For example red forms may sing most particularly f|6.18|2.96|8015007|scholarmaxi #7|15|fishing|8|Sports|122|ableableought|N/A|41navajo723570009667|pink|Gram|Unknown|38|callypriesecallyought| +16437|AAAAAAAAEDAEAAAA|2000-10-27||Signs chart advantages; members shake subse|0.61|2.96|8015007|exportiamalg #2|15|maternity|1|Women|323|priablepri|economy|0513294431261tomato3|honeydew|Pound|Unknown|36|ationpriesecallyought| +16438|AAAAAAAAGDAEAAAA|1997-10-27|1999-10-27|Handsome parties could get extra; strongly national letters evaluate british, other lawyers. Officially able units might work countries. Teams know really such as a calculat|1.62|1.29|6006007|corpcorp #7|6|rings|6|Jewelry|74|eseation|N/A|53783turquoise560322|chartreuse|Box|Unknown|31|eingpriesecallyought| +16439|AAAAAAAAGDAEAAAA|1999-10-28|2001-10-26|Just satisfactory acts may not know long normal double pictures. |4.66|3.86|6006007|importounivamalg #12|6|home repair|9|Books|158|eseation|N/A|49587475684378rosy28|maroon|Oz|Unknown|18|n stpriesecallyought| +16440|AAAAAAAAGDAEAAAA|2001-10-27||Very independent bodies used to feature historical, |2.38|3.86|6006007|edu packimporto #1|6|sports-apparel|2|Men|158|eseation|medium|49587475684378rosy28|midnight|Dozen|Unknown|26|bareseesecallyought| +16441|AAAAAAAAJDAEAAAA|1997-10-27||Distinct enemies work only primarily general years. Points employ too. Changes say lo|4.33|3.37|3001002|amalgexporti #2|1|newborn|3|Children|353|priantipri|large|562orange53353762344|turquoise|Case|Unknown|41|oughteseesecallyought| +16442|AAAAAAAAKDAEAAAA|1997-10-27|2000-10-26|Always chinese patterns reserve relatively. Reli|53.60|26.26|6006003|corpcorp #3|6|rings|6|Jewelry|125|antiableought|N/A|9902601613papaya2576|puff|Bundle|Unknown|21|ableeseesecallyought| +16443|AAAAAAAAKDAEAAAA|2000-10-27||Always chinese patterns reserve relatively. Reli|1.44|1.18|6006003|corpamalgamalg #5|16|wireless|10|Electronics|125|antiableought|N/A|032957snow7109397990|seashell|Carton|Unknown|30|prieseesecallyought| +16444|AAAAAAAAMDAEAAAA|1997-10-27|1999-10-27|Strategic foods could not replace now activities; labour, new difficulties improve hopefully necessary standards. Contrary years might stay possible, new hotels; now obvious circumstanc|2.05|1.53|2001001|amalgimporto #1|1|accessories|2|Men|97|ationn st|small|01608447353684puff54|steel|Box|Unknown|32|eseeseesecallyought| +16445|AAAAAAAAMDAEAAAA|1999-10-28|2001-10-26|Social, old states could not look ev|1.24|1.53|1002002|importoamalg #2|1|fragrances|1|Women|149|ationn st|medium|01608447353684puff54|seashell|Dozen|Unknown|50|antieseesecallyought| +16446|AAAAAAAAMDAEAAAA|2001-10-27||General individuals would enable. Increased, different companies would not come also poor coins. Expensive, natural events block worse probably old r|4.32|1.53|5002001|importoscholar #1|2|country|5|Music|231|ationn st|N/A|01608447353684puff54|red|Each|Unknown|9|callyeseesecallyought| +16447|AAAAAAAAPDAEAAAA|1997-10-27||Tech|51.07|35.23|3004002|edu packexporti #2|4|school-uniforms|3|Children|190|barn stought|medium|564drab8539345464041|spring|Unknown|Unknown|30|ationeseesecallyought| +16448|AAAAAAAAAEAEAAAA|1997-10-27|2000-10-26|Men shall not play so financial shares; just black deposits might say probably. Level exhibitions receive safely empty, international investors. Industri|27.60|22.35|8014009|edu packmaxi #9|14|tennis|8|Sports|11|oughtought|N/A|1809481451smoke57717|lawn|Case|Unknown|13|eingeseesecallyought| +16449|AAAAAAAAAEAEAAAA|2000-10-27||Men shall not play so financial shares; just black deposits might say probably. Level exhibitions receive safely empty, international investors. Industri|4.74|22.35|2001002|amalgimporto #2|14|accessories|2|Men|267|oughtought|large|09667tomato577795206|violet|Pound|Unknown|10|n steseesecallyought| +16450|AAAAAAAACEAEAAAA|1997-10-27|1999-10-27|Quite empty countries cannot block briefly in a pupils. Crude sheets take tory solutions. Years occur accurately inc attitudes. Elections present e|3.99|2.19|6012003|importobrand #3|12|costume|6|Jewelry|246|callyeseable|N/A|saddle37629824286919|peach|Case|Unknown|45|barantiesecallyought| +16451|AAAAAAAACEAEAAAA|1999-10-28|2001-10-26|Quite empty countries cannot block briefly in a pupils. Crude sheets take tory solutions. Years occur accurately inc attitudes. Elections present e|3.68|2.19|6012003|maxinameless #2|9|optics|8|Sports|56|callyanti|N/A|94seashell2291874291|snow|Carton|Unknown|6|oughtantiesecallyought| +16452|AAAAAAAACEAEAAAA|2001-10-27||Magnetic, various flowers might not recognize games; easy, total populations spend in a situations.|3.18|2.19|10010014|univamalgamalg #14|10|memory|10|Electronics|56|callyanti|N/A|94seashell2291874291|turquoise|Pound|Unknown|27|ableantiesecallyought| +16453|AAAAAAAAFEAEAAAA|1997-10-27||Still eastern trials used to accelerate. Parents return so already great germans. Rates may leave traditi|0.52|0.16|2004002|edu packimporto #2|4|sports-apparel|2|Men|108|eingbarought|medium|50299930679white8488|purple|Each|Unknown|34|priantiesecallyought| +16454|AAAAAAAAGEAEAAAA|1997-10-27|2000-10-26|Full, wrong intervals attend simple teachers; more early |0.77|0.52|7014005|edu packnameless #5|14|glassware|7|Home|123|priableought|N/A|7401sienna3741645651|wheat|Each|Unknown|32|eseantiesecallyought| +16455|AAAAAAAAGEAEAAAA|2000-10-27||Full, wrong intervals attend simple teachers; more early |2.40|1.60|7014005|exportiexporti #2|14|toddlers|3|Children|123|priableought|medium|7401sienna3741645651|seashell|Gross|Unknown|12|antiantiesecallyought| +16456|AAAAAAAAIEAEAAAA|1997-10-27|1999-10-27|Difficult, low needs ought to notice into a mammals. Towns will support also efficient glasses; common workshops would ch|9.94|3.87|7009007|maxibrand #7|9|mattresses|7|Home|2|able|N/A|95honeydew9079094606|sienna|Bunch|Unknown|13|callyantiesecallyought| +16457|AAAAAAAAIEAEAAAA|1999-10-28|2001-10-26|Professional values become then benefits. British perceptions|3.82|3.87|7009007|importoamalg #2|2|fragrances|1|Women|2|able|medium|366169965plum9456550|white|Each|Unknown|25|ationantiesecallyought| +16458|AAAAAAAAIEAEAAAA|2001-10-27||Quite successful pa|2.42|3.87|7009007|maxinameless #9|2|optics|8|Sports|654|eseantically|N/A|366169965plum9456550|thistle|Each|Unknown|7|eingantiesecallyought| +16459|AAAAAAAALEAEAAAA|1997-10-27||Attitudes seek goods. Very odd feet might uphold sales. Low certain occasions cannot inflict thoughtfully. Huge doctors can reassess in the skills. Tired e|1.14|1.02|5002002|importoscholar #2|2|country|5|Music|841|oughteseeing|N/A|634927421464purple83|aquamarine|Cup|Unknown|46|n stantiesecallyought| +16460|AAAAAAAAMEAEAAAA|1997-10-27|2000-10-26|Difficult matters provide just mental, moving attitudes. In particular active relatives sustain here political participants.|3.01|1.20|4003001|exportiedu pack #1|3|kids|4|Shoes|327|ationablepri|medium|red01602511310776542|purple|Ton|Unknown|16|barcallyesecallyought| +16461|AAAAAAAAMEAEAAAA|2000-10-27||Taxes will search rather overseas points. Women include probably too sensible factors. Dual, ruling experiments may shift papers. Joint eyes used to last views; experimen|0.19|0.08|4003001|amalgimporto #2|3|accessories|2|Men|279|n stationable|small|red01602511310776542|goldenrod|Ounce|Unknown|23|oughtcallyesecallyought| +16462|AAAAAAAAOEAEAAAA|1997-10-27|1999-10-27|Permanent, personal offers shall happen a|8.27|6.11|4002001|importoedu pack #1|2|mens|4|Shoes|278|eingationable|medium|880990602tan63707351|sienna|Tbl|Unknown|18|ablecallyesecallyought| +16463|AAAAAAAAOEAEAAAA|1999-10-28|2001-10-26|General levels may not tel|4.01|2.80|4002001|edu packunivamalg #12|2|sports|9|Books|559|n stantianti|N/A|880990602tan63707351|steel|Carton|Unknown|29|pricallyesecallyought| +16464|AAAAAAAAOEAEAAAA|2001-10-27||General levels may not tel|2.06|2.80|4002001|edu packexporti #1|2|school-uniforms|3|Children|118|eingoughtought|small|880990602tan63707351|violet|Cup|Unknown|15|esecallyesecallyought| +16465|AAAAAAAABFAEAAAA|1997-10-27||Certainly social con|5.13|3.18|3001002|amalgexporti #2|1|newborn|3|Children|435|antipriese|large|53464burlywood719732|floral|Each|Unknown|27|anticallyesecallyought| +16466|AAAAAAAACFAEAAAA|1997-10-27|2000-10-26|Only new effects cannot bring very well ethnic publications. Primary, great miles count like a children. Years try quite schools. Economically sad regulations identify to a experiences. Boxes win mos|7.38|3.61|6011003|amalgbrand #3|11|semi-precious|6|Jewelry|110|baroughtought|N/A|028sienna40892570062|thistle|Cup|Unknown|30|callycallyesecallyought| +16467|AAAAAAAACFAEAAAA|2000-10-27||Only new effects cannot bring very well ethnic publications. Primary, great miles count like a children. Years try quite schools. Economically sad regulations identify to a experiences. Boxes win mos|0.89|3.61|6011003|importonameless #8|12|paint|7|Home|110|baroughtought|N/A|0305433tomato5863481|rosy|Bundle|Unknown|11|ationcallyesecallyought| +16468|AAAAAAAAEFAEAAAA|1997-10-27|1999-10-27|Central blues await new consumers. Different fans shall not make now services. New girls shall maintain. Interesting firms should use just little electronic affai|2.07|1.28|2003001|exportiimporto #1|3|pants|2|Men|38|eingpri|small|24260puff97867109113|thistle|Pound|Unknown|100|eingcallyesecallyought| +16469|AAAAAAAAEFAEAAAA|1999-10-28|2001-10-26|General, satisfactory doubts may invade also public things. Conditions help more years. Central, wrong companies find so always necess|8.45|2.53|4003002|exportiedu pack #2|3|kids|4|Shoes|373|eingpri|N/A|24260puff97867109113|cornflower|Pallet|Unknown|59|n stcallyesecallyought| +16470|AAAAAAAAEFAEAAAA|2001-10-27||Dark, vague measures would |4.81|2.53|9014007|edu packunivamalg #7|14|sports|9|Books|373|eingpri|N/A|24260puff97867109113|ivory|Gram|Unknown|80|barationesecallyought| +16471|AAAAAAAAHFAEAAAA|1997-10-27||Vital, s|6.42|5.13|8001002|amalgnameless #2|1|athletic shoes|8|Sports|609|n stbarcally|N/A|403507730salmon05532|red|Unknown|Unknown|27|oughtationesecallyought| +16472|AAAAAAAAIFAEAAAA|1997-10-27|2000-10-26|Claims cross sooner unlikely classes. Objects will smile. Miners ought to add again european months. Hard quarters would not think more increasing gods; industrial, social depths |9.22|4.42|3003001|exportiexporti #1|3|toddlers|3|Children|204|esebarable|small|596847371993wheat797|wheat|Ounce|Unknown|33|ableationesecallyought| +16473|AAAAAAAAIFAEAAAA|2000-10-27||Public cattle take rural systems. Long selective conditions s|2.92|2.16|3003001|importoimporto #2|2|shirts|2|Men|204|esebarable|small|596847371993wheat797|tan|Each|Unknown|23|priationesecallyought| +16474|AAAAAAAAKFAEAAAA|1997-10-27|1999-10-27|Closed miles may not succeed about at once little cases. Processes discourage living men. Useless, experimental books|1.74|1.46|7006005|corpbrand #5|6|rugs|7|Home|79|n station|N/A|tomato43988036853076|smoke|Each|Unknown|32|eseationesecallyought| +16475|AAAAAAAAKFAEAAAA|1999-10-28|2001-10-26|Paintings marry only respect|1.99|0.69|7006005|edu packscholar #2|4|classical|5|Music|79|n station|N/A|003369215gainsboro06|royal|Bunch|Unknown|10|antiationesecallyought| +16476|AAAAAAAAKFAEAAAA|2001-10-27||Paintings marry only respect|5.11|0.69|6009005|maxicorp #5|4|womens watch|6|Jewelry|79|n station|N/A|lace0792095303149020|papaya|Dram|Unknown|1|callyationesecallyought| +16477|AAAAAAAANFAEAAAA|1997-10-27||Similar, supposed visitors must not satisfy important words. Recent, roman problems take twice outside the notes. Men move behind most available schemes|1.39|1.00|1004002|edu packamalg #2|4|swimwear|1|Women|116|callyoughtought|small|31300444657peach9071|peru|N/A|Unknown|41|ationationesecallyought| +16478|AAAAAAAAOFAEAAAA|1997-10-27|2000-10-26|P|51.57|33.00|4003001|exportiedu pack #1|3|kids|4|Shoes|514|eseoughtanti|medium|0822017557violet5402|red|Cup|Unknown|36|eingationesecallyought| +16479|AAAAAAAAOFAEAAAA|2000-10-27||Conveniently thin p|23.99|16.07|4003001|importoamalg #2|3|fragrances|1|Women|514|eseoughtanti|small|23833tomato913708217|sienna|Each|Unknown|9|n stationesecallyought| +16480|AAAAAAAAAGAEAAAA|1997-10-27|1999-10-27|Other, pink characteristics ought to use never national places. Big miles talk with a unions. Thus particu|3.67|2.78|9014005|edu packunivamalg #5|14|sports|9|Books|292|ablen stable|N/A|880997orchid76504289|royal|Ounce|Unknown|87|bareingesecallyought| +16481|AAAAAAAAAGAEAAAA|1999-10-28|2001-10-26|Other, pink characteristics ought to use never national places. Big miles talk with a unions. Thus particu|2.11|2.78|8002002|importonameless #2|2|baseball|8|Sports|292|ablen stable|N/A|880997orchid76504289|seashell|Tsp|Unknown|23|oughteingesecallyought| +16482|AAAAAAAAAGAEAAAA|2001-10-27||Civil months could practise already concerned colours. Good, psychiatric messages shall keep opposite customers. Afraid, urban germans should not keep right differe|1.99|0.95|1001001|amalgamalg #1|1|dresses|1|Women|463|ablen stable|extra large|880997orchid76504289|tomato|Gram|Unknown|36|ableeingesecallyought| +16483|AAAAAAAADGAEAAAA|1997-10-27||Costs cannot marry totally on a members. At once good acts would carry different, old hands. True, new terms understand clearly on the workers. Privately detailed services |0.63|0.28|4004002|edu packedu pack #2|4|athletic|4|Shoes|37|ationpri|medium|081432946sandy652548|rose|Dozen|Unknown|9|prieingesecallyought| +16484|AAAAAAAAEGAEAAAA|1997-10-27|2000-10-26|Far injuries pay so various arms. Courses could go anywhere universal possibilities; talks stand since mean, colonial scho|9.57|3.92|9001009|amalgmaxi #9|1|arts|9|Books|86|callyeing|N/A|4803861746moccasin33|yellow|Box|Unknown|10|eseeingesecallyought| +16485|AAAAAAAAEGAEAAAA|2000-10-27||Entire areas go radical, christian cultures. Equal, early players examine members; ahead delighted values subsidise again chief, related pr|3.22|3.92|5004002|edu packscholar #2|1|classical|5|Music|86|callyeing|N/A|thistle8926718371123|pink|Unknown|Unknown|20|antieingesecallyought| +16486|AAAAAAAAGGAEAAAA|1997-10-27|1999-10-27|Copies light unfortunately by a periods. Properly desirable leads must go between a windows. New years must take. New contents like much symbolic users. So short-term wheel|4.07|3.37|9008011|namelessmaxi #11|8|romance|9|Books|567|ationcallyanti|N/A|5501orchid9156926202|rosy|Tbl|Unknown|13|callyeingesecallyought| +16487|AAAAAAAAGGAEAAAA|1999-10-28|2001-10-26|Countries must not lend as. Eyes could see warm, wrong days. Tory principles leave both very, sound eyes. Red, exciting implications might see walls. Plants shall study generally|4.17|1.58|9008011|amalgimporto #2|1|accessories|2|Men|567|ationcallyanti|medium|4blush64428213679432|pink|Tsp|Unknown|56|ationeingesecallyought| +16488|AAAAAAAAGGAEAAAA|2001-10-27||Countries must not lend as. Eyes could see warm, wrong days. Tory principles leave both very, sound eyes. Red, exciting implications might see walls. Plants shall study generally|3.60|2.91|1001001|amalgamalg #1|1|dresses|1|Women|567|ationcallyanti|extra large|2581991102tomato3269|sienna|Unknown|Unknown|16|eingeingesecallyought| +16489|AAAAAAAAJGAEAAAA|1997-10-27||Black, internal parties will carry about also aware cases. Splendid trousers can stay always dead times. Contexts say just rates. Harsh, final rules carry very young years; most u|2.49|1.69|3003002|exportiexporti #2|3|toddlers|3|Children|158|eingantiought|medium|6139596062201powder4|deep|Gross|Unknown|37|n steingesecallyought| +16490|AAAAAAAAKGAEAAAA|1997-10-27|2000-10-26|There mixed solicitors tell easy sides. Green, new eyes may give afraid, industrial rooms. At least certain days would list dogs. Children proceed maybe in a |3.14|1.03|3004001|edu packexporti #1|4|school-uniforms|3|Children|605|antibarcally|medium|321529indian38326904|spring|Gram|Unknown|2|barn stesecallyought| +16491|AAAAAAAAKGAEAAAA|2000-10-27||There mixed solicitors tell easy sides. Green, new eyes may give afraid, industrial rooms. At least certain days would list dogs. Children proceed maybe in a |6.16|1.03|1003002|exportiamalg #2|3|maternity|1|Women|605|antibarcally|large|spring46556938543728|orchid|Tbl|Unknown|28|oughtn stesecallyought| +16492|AAAAAAAAMGAEAAAA|1997-10-27|1999-10-27|Necessary countrie|10.28|5.03|7015003|scholarnameless #3|15|tables|7|Home|3|pri|N/A|18277814slate9574310|drab|Each|Unknown|25|ablen stesecallyought| +16493|AAAAAAAAMGAEAAAA|1999-10-28|2001-10-26|Necessary countrie|5.83|2.33|7015003|namelessmaxi #6|8|romance|9|Books|3|pri|N/A|pink0746826147164915|thistle|Each|Unknown|90|prin stesecallyought| +16494|AAAAAAAAMGAEAAAA|2001-10-27||Bc early cases might damage thus. Accused, alternati|0.50|0.40|6007007|brandcorp #7|8|pendants|6|Jewelry|597|ationn stanti|N/A|158853829lace2175998|drab|Each|Unknown|8|esen stesecallyought| +16495|AAAAAAAAPGAEAAAA|1997-10-27||Then sure temperatures can speak still central services. Powerful findings follow. Also successive speakers may not see by now in the nations. Main, good patterns can ben|6.65|2.39|4004002|edu packedu pack #2|4|athletic|4|Shoes|384|eseeingpri|N/A|76538316591blanched9|grey|Oz|Unknown|7|antin stesecallyought| +16496|AAAAAAAAAHAEAAAA|1997-10-27|2000-10-26|Around back institutio|39.85|31.48|7009007|maxibrand #7|9|mattresses|7|Home|193|prin stought|N/A|2sandy72911637310408|purple|Oz|Unknown|73|callyn stesecallyought| +16497|AAAAAAAAAHAEAAAA|2000-10-27||Around back institutio|7.35|31.48|7009007|brandunivamalg #8|9|personal|10|Electronics|193|prin stought|N/A|2sandy72911637310408|peach|Pallet|Unknown|68|ationn stesecallyought| +16498|AAAAAAAACHAEAAAA|1997-10-27|1999-10-27|Especially alone payments would mention free, equal eyes. Facilities ought to benefit there occasions. Big meals might prove skills. Chan|60.91|21.92|8016009|corpmaxi #9|16|golf|8|Sports|542|ableeseanti|N/A|95metallic3295224679|peach|N/A|Unknown|40|eingn stesecallyought| +16499|AAAAAAAACHAEAAAA|1999-10-28|2001-10-26|Typical, difficult states tak|4.76|21.92|8016009|importoamalg #2|16|fragrances|1|Women|542|ableeseanti|medium|4olive54770329424566|hot|Bundle|Unknown|2|n stn stesecallyought| +16500|AAAAAAAACHAEAAAA|2001-10-27||Twin securities might formulate on a restrictions. Premises risk popular women. Boards speak far contrary, far copies. Past, total characters might see then british |2.49|21.92|8016009|amalgnameless #1|16|accent|7|Home|281|oughteingable|N/A|4216midnight73660784|navy|Tbl|Unknown|17|barbaranticallyought| +16501|AAAAAAAAFHAEAAAA|1997-10-27||Thoughts ought to keep only. Joint terms go possibly hours. Other, honest facts may not mean ever dual|3.10|1.51|4003002|exportiedu pack #2|3|kids|4|Shoes|57|ationanti|economy|metallic980474789201|seashell|Oz|Unknown|54|oughtbaranticallyought| +16502|AAAAAAAAGHAEAAAA|1997-10-27|2000-10-26|Paintings may market mistakenly dependent occasions; nearly good children might not put now rights. Current services see for a relationships; faces could keep too nearby, diverse p|7.67|6.74|8016003|corpmaxi #3|16|golf|8|Sports|447|ationeseese|N/A|894834brown907288243|mint|Carton|Unknown|51|ablebaranticallyought| +16503|AAAAAAAAGHAEAAAA|2000-10-27||Open types would look once with the stands; stages pick; new countries can happe|35.34|6.74|8016003|importoamalg #2|16|fragrances|1|Women|447|ationeseese|medium|894834brown907288243|magenta|Bunch|Unknown|23|pribaranticallyought| +16504|AAAAAAAAIHAEAAAA|1997-10-27|1999-10-27|Foreign days trade aware conditions. Schemes ought to ignore later by a rules. Trees care overwhelmingly out of the views. Concerned, broken kin|7.87|5.82|5003001|exportischolar #1|3|pop|5|Music|9|n st|N/A|5257pink051410651076|mint|Pound|Unknown|97|esebaranticallyought| +16505|AAAAAAAAIHAEAAAA|1999-10-28|2001-10-26|Foreign days trade aware conditions. Schemes ought to ignore later by a rules. Trees care overwhelmingly out of the views. Concerned, broken kin|4.36|5.82|5003001|corpmaxi #6|6|parenting|9|Books|293|prin stable|N/A|33468064118peach8617|pink|Tbl|Unknown|51|antibaranticallyought| +16506|AAAAAAAAIHAEAAAA|2001-10-27||Foreign days trade aware conditions. Schemes ought to ignore later by a rules. Trees care overwhelmingly out of the views. Concerned, broken kin|3.85|1.92|5003001|exportinameless #7|13|wallpaper|7|Home|169|prin stable|N/A|33468064118peach8617|wheat|Pallet|Unknown|53|callybaranticallyought| +16507|AAAAAAAALHAEAAAA|1997-10-27||Impressive, easter|0.10|0.06|10010001|univamalgamalg #1|10|memory|10|Electronics|132|ablepriought|N/A|733055600orchid99453|yellow|Bunch|Unknown|15|ationbaranticallyought| +16508|AAAAAAAAMHAEAAAA|1997-10-27|2000-10-26|Reasons might bear unaware soldiers. Eyes used to see for example cruel, close workers. Matches can care. Stupid departments shall get quite to a sources; really positive police must |3.80|1.44|3001001|amalgexporti #1|1|newborn|3|Children|406|callybarese|large|salmon26573629378523|gainsboro|Unknown|Unknown|45|eingbaranticallyought| +16509|AAAAAAAAMHAEAAAA|2000-10-27||Reasons might bear unaware soldiers. Eyes used to see for example cruel, close workers. Matches can care. Stupid departments shall get quite to a sources; really positive police must |9.82|8.24|3001001|scholarunivamalg #10|15|fiction|9|Books|251|callybarese|N/A|salmon26573629378523|beige|Dozen|Unknown|7|n stbaranticallyought| +16510|AAAAAAAAOHAEAAAA|1997-10-27|1999-10-27|Cool others satisfy however simply international minutes. Patients must not cut centuries. Combined terms see. Just complete lines must not want too essential games. Hands will not provide u|2.57|1.20|5003001|exportischolar #1|3|pop|5|Music|195|antin stought|N/A|98seashell0579944519|lime|Tbl|Unknown|58|baroughtanticallyought| +16511|AAAAAAAAOHAEAAAA|1999-10-28|2001-10-26|Careful, certain days believe busy, empty systems; ab|7.58|1.20|6010008|univbrand #8|3|jewelry boxes|6|Jewelry|84|eseeing|N/A|98seashell0579944519|wheat|Gross|Unknown|26|oughtoughtanticallyought| +16512|AAAAAAAAOHAEAAAA|2001-10-27||Democratic associations shall not accept extremely on a methods. Workers discover as possible, given children. Coins might show all more vast choices. Personal studies acquire to|97.25|33.06|3004001|edu packexporti #1|3|school-uniforms|3|Children|84|eseeing|large|398424tomato27657880|sienna|Carton|Unknown|22|ableoughtanticallyought| +16513|AAAAAAAABIAEAAAA|1997-10-27||Coastal, real options matter still on a eyes. Inner children may not extend plants. Literally a|4.19|2.59|3002002|importoexporti #2|2|infants|3|Children|591|oughtn stanti|large|660911958tomato02737|snow|Tsp|Unknown|79|prioughtanticallyought| +16514|AAAAAAAACIAEAAAA|1997-10-27|2000-10-26|United, economic strategies find then. Then grey factors can join occasionally in the manufacturers. Only, large conditions creep most into a consu|10.01|4.40|10011008|amalgamalgamalg #8|11|disk drives|10|Electronics|841|oughteseeing|N/A|720707violet01801399|peru|Tbl|Unknown|17|eseoughtanticallyought| +16515|AAAAAAAACIAEAAAA|2000-10-27||United, economic strategies find then. Then grey factors can join occasionally in the manufacturers. Only, large conditions creep most into a consu|3.27|4.40|10011008|amalgnameless #6|11|athletic shoes|8|Sports|841|oughteseeing|N/A|50776924plum32675337|turquoise|Dram|Unknown|50|antioughtanticallyought| +16516|AAAAAAAAEIAEAAAA|1997-10-27|1999-10-27|At last front mechanisms can |9.64|8.00|8010007|univmaxi #7|10|pools|8|Sports|91|oughtn st|N/A|740182navy8343521499|peru|Bunch|Unknown|16|callyoughtanticallyought| +16517|AAAAAAAAEIAEAAAA|1999-10-28|2001-10-26|Italian, white girls should see special, extra boots. Less very circumstances go important, right windows. More|5.08|8.00|8010007|brandcorp #6|10|pendants|6|Jewelry|317|ationoughtpri|N/A|7539tomato7175974026|royal|N/A|Unknown|47|ationoughtanticallyought| +16518|AAAAAAAAEIAEAAAA|2001-10-27||Italian, white girls should see special, extra boots. Less very circumstances go important, right windows. More|8.68|6.16|8010007|exporticorp #7|3|gold|6|Jewelry|588|eingeinganti|N/A|papaya25549022954828|rosy|Gross|Unknown|36|eingoughtanticallyought| +16519|AAAAAAAAHIAEAAAA|1997-10-27||Authorities must see paradoxically for a americans. Sales would not buy nowhere american bombs. Areas would learn much. Essential times ought to|3.53|1.97|5002002|importoscholar #2|2|country|5|Music|105|antibarought|N/A|69462818174069tan860|lace|Oz|Unknown|46|n stoughtanticallyought| +16520|AAAAAAAAIIAEAAAA|1997-10-27|2000-10-26|Great areas call also quite proper hopes. Well military interests may not see ago animals. Classic studies ought to disappoint more by a towns. Near original pairs|4.93|4.43|3003001|exportiexporti #1|3|toddlers|3|Children|562|ablecallyanti|economy|284208lime3952149719|tomato|N/A|Unknown|58|barableanticallyought| +16521|AAAAAAAAIIAEAAAA|2000-10-27||Great areas call also quite proper hopes. Well military interests may not see ago animals. Classic studies ought to disappoint more by a towns. Near original pairs|7.75|2.32|3003001|importoexporti #2|3|infants|3|Children|562|ablecallyanti|small|284208lime3952149719|wheat|Tbl|Unknown|11|oughtableanticallyought| +16522|AAAAAAAAKIAEAAAA|1997-10-27|1999-10-27|Bad examples must like quickly old, suitable sales. Basic things should|70.46|29.59|8010003|univmaxi #3|10|pools|8|Sports|446|callyeseese|N/A|7dim5924046879915650|salmon|Unknown|Unknown|75|ableableanticallyought| +16523|AAAAAAAAKIAEAAAA|1999-10-28|2001-10-26|Bad examples must like quickly old, suitable sales. Basic things should|1.69|29.59|8010003|amalgnameless #4|11|accent|7|Home|446|callyeseese|N/A|455433thistle7018874|olive|Pallet|Unknown|32|priableanticallyought| +16524|AAAAAAAAKIAEAAAA|2001-10-27||Bad examples must like quickly old, suitable sales. Basic things should|37.39|20.93|8010003|edu packamalg #1|11|swimwear|1|Women|446|callyeseese|large|455433thistle7018874|peach|Oz|Unknown|5|eseableanticallyought| +16525|AAAAAAAANIAEAAAA|1997-10-27||Rich considerations communic|7.29|5.32|6016006|corpbrand #6|16|consignment|6|Jewelry|543|prieseanti|N/A|24peru19650049602186|rosy|Ton|Unknown|27|antiableanticallyought| +16526|AAAAAAAAOIAEAAAA|1997-10-27|2000-10-26|Other, eager christians live very others. Young, financ|8.22|5.01|5004001|edu packscholar #1|4|classical|5|Music|465|anticallyese|N/A|lace7178564827607872|medium|Ounce|Unknown|12|callyableanticallyought| +16527|AAAAAAAAOIAEAAAA|2000-10-27||Sides wear fast bodies. Large babies should wish however inc players. Safe ide|4.52|5.01|5004001|importoscholar #2|2|country|5|Music|81|oughteing|N/A|lace7178564827607872|papaya|Case|Unknown|15|ationableanticallyought| +16528|AAAAAAAAAJAEAAAA|1997-10-27|1999-10-27|Actively different proceedings light yet so similar houses. Good circumstances shall not take only levels. Then moral pounds will clean very only national organisations. |4.35|2.13|2001001|amalgimporto #1|1|accessories|2|Men|156|callyantiought|small|558821873white170744|medium|Dozen|Unknown|24|eingableanticallyought| +16529|AAAAAAAAAJAEAAAA|1999-10-28|2001-10-26|Various shops cope late situations. Personnel might not phone nearby other pages; respects should say indeed certain neighbours. Days used to create pretty br|3.96|2.13|2001001|importoscholar #2|1|country|5|Music|514|callyantiought|N/A|0124507148531navajo6|white|Carton|Unknown|97|n stableanticallyought| +16530|AAAAAAAAAJAEAAAA|2001-10-27||Various shops cope late situations. Personnel might not phone nearby other pages; respects should say indeed certain neighbours. Days used to create pretty br|3.51|1.29|2001001|amalgedu pack #1|1|womens|4|Shoes|157|callyantiought|extra large|khaki622799164389343|snow|Pallet|Unknown|4|barprianticallyought| +16531|AAAAAAAADJAEAAAA|1997-10-27||Patients leave. Perhaps previous readers can give around a refugees. Books take today certain relations. Only letters go existing prizes. Yet early communities behave. Dread|2.37|1.65|7015002|scholarnameless #2|15|tables|7|Home|570|barationanti|N/A|839389520684peru2176|thistle|Gram|Unknown|46|oughtprianticallyought| +16532|AAAAAAAAEJAEAAAA|1997-10-27|2000-10-26|Labour powers might not explain slightly basic students. Dealers become too for the opponents. Likely, civil stations cannot improve now able, glorious problems. Other phases should make greatly in a|1.45|1.27|7005003|scholarbrand #3|5|blinds/shades|7|Home|350|barantipri|N/A|5636527sienna1299324|papaya|Pallet|Unknown|14|ableprianticallyought| +16533|AAAAAAAAEJAEAAAA|2000-10-27||Main, mad projects reach closer;|9.64|1.27|7005003|edu packexporti #2|4|school-uniforms|3|Children|350|barantipri|small|sienna03334130539423|yellow|Tsp|Unknown|14|priprianticallyought| +16534|AAAAAAAAGJAEAAAA|1997-10-27|1999-10-27|Gentl|1.32|1.17|7003005|exportibrand #5|3|kids|7|Home|167|ationcallyought|N/A|551732moccasin261706|pink|Tsp|Unknown|28|eseprianticallyought| +16535|AAAAAAAAGJAEAAAA|1999-10-28|2001-10-26|Gentl|7.81|6.16|7003005|exportiedu pack #2|3|kids|4|Shoes|167|ationcallyought|extra large|91orange102618010697|red|Carton|Unknown|88|antiprianticallyought| +16536|AAAAAAAAGJAEAAAA|2001-10-27||Gentl|1.15|0.64|7003005|edu packmaxi #1|3|entertainments|9|Books|167|ationcallyought|N/A|5681608528731rosy760|puff|Cup|Unknown|14|callyprianticallyought| +16537|AAAAAAAAJJAEAAAA|1997-10-27||Paths pop somewhere then special characters. International, angry signs dare however young respondents. Fundamental strategies identify decisions. Global, blac|4.52|2.62|10002014|importounivamalg #14|2|camcorders|10|Electronics|153|priantiought|N/A|033plum3843562652882|violet|Bundle|Unknown|6|ationprianticallyought| +16538|AAAAAAAAKJAEAAAA|1997-10-27|2000-10-26|Cautiously fair arms find a little plans. Years ought to react common arms. Wrong structures reflect effectively countries. Human ways may get just capital, regional animals; similar, senior pl|2.75|1.76|9016003|corpunivamalg #3|16|mystery|9|Books|124|eseableought|N/A|8955541767violet0393|navajo|Oz|Unknown|9|eingprianticallyought| +16539|AAAAAAAAKJAEAAAA|2000-10-27||Cautiously fair arms find a little plans. Years ought to react common arms. Wrong structures reflect effectively countries. Human ways may get just capital, regional animals; similar, senior pl|3.88|1.76|9016003|edu packbrand #10|4|curtains/drapes|7|Home|27|eseableought|N/A|84snow45632766456972|orange|Pallet|Unknown|30|n stprianticallyought| +16540|AAAAAAAAMJAEAAAA|1997-10-27|1999-10-27|Busy, fun dogs cannot suffer. Valid, dry centres would recover military, partic|3.74|1.23|8005001|scholarnameless #1|5|fitness|8|Sports|869|n stcallyeing|N/A|3seashell73438297447|purple|Bundle|Unknown|63|bareseanticallyought| +16541|AAAAAAAAMJAEAAAA|1999-10-28|2001-10-26|Traditional, old numbers may not help european centuries. Strong, small questions can state most concerns|4.67|1.23|10015001|scholaramalgamalg #1|15|portable|10|Electronics|711|oughtoughtation|N/A|2285007179313medium4|sky|Dram|Unknown|22|oughteseanticallyought| +16542|AAAAAAAAMJAEAAAA|2001-10-27||Suddenly old meals select somewhere additional, green others. Now available conditions draw governors. Poor, young concepts continue less employees. Golden, p|75.20|1.23|10015001|importonameless #3|15|baseball|8|Sports|711|oughtoughtation|N/A|481frosted6487539281|powder|N/A|Unknown|78|ableeseanticallyought| +16543|AAAAAAAAPJAEAAAA|1997-10-27||Things take double properties. Years shall advance sci|0.56|0.32|3003002|exportiexporti #2|3|toddlers|3|Children|397|ationn stpri|small|13840371gainsboro808|lawn|Gross|Unknown|31|prieseanticallyought| +16544|AAAAAAAAAKAEAAAA|1997-10-27|2000-10-26|Angry, familiar charges should start just rapid friends. Together contemporary countries mean truly delightful heads. Feet could remain. Soci|2.04|1.38|6006001|corpcorp #1|6|rings|6|Jewelry|98|eingn st|N/A|8359674pale323424989|plum|Each|Unknown|36|eseeseanticallyought| +16545|AAAAAAAAAKAEAAAA|2000-10-27||British, brief members prove however inhabitants. French, similar sales speak already red councils. High, hard areas should not supervise political charts. Needs beat respons|2.75|1.23|3003002|exportiexporti #2|3|toddlers|3|Children|98|eingn st|extra large|8359674pale323424989|dim|Tsp|Unknown|32|antieseanticallyought| +16546|AAAAAAAACKAEAAAA|1997-10-27|1999-10-27|Totally sudden doubts ought to remember never federal easy faces. English adults can seem in a plants. Errors stop old other|1.43|1.20|8003007|exportinameless #7|3|basketball|8|Sports|485|antieingese|N/A|2930279564sky2104606|burnished|Gram|Unknown|21|callyeseanticallyought| +16547|AAAAAAAACKAEAAAA|1999-10-28|2001-10-26|Totally sudden doubts ought to remember never federal easy faces. English adults can seem in a plants. Errors stop old other|9.82|5.69|8003007|exportiamalg #2|3|maternity|1|Women|485|antieingese|petite|2930279564sky2104606|sandy|Lb|Unknown|94|ationeseanticallyought| +16548|AAAAAAAACKAEAAAA|2001-10-27||Totally sudden doubts ought to remember never federal easy faces. English adults can seem in a plants. Errors stop old other|3.55|2.20|1002001|importoamalg #1|3|fragrances|1|Women|485|antieingese|petite|2930279564sky2104606|papaya|N/A|Unknown|9|eingeseanticallyought| +16549|AAAAAAAAFKAEAAAA|1997-10-27||Tiny, gentle things jeopardise so automatically empty teams. Symbolic representations can win possibly po|2.21|1.59|1002002|importoamalg #2|2|fragrances|1|Women|37|ationpri|N/A|0744smoke48531355248|lawn|Tbl|Unknown|60|n steseanticallyought| +16550|AAAAAAAAGKAEAAAA|1997-10-27|2000-10-26|Popular letters can talk highly by a abilities. Quarters tell fai|8.08|5.17|1004001|edu packamalg #1|4|swimwear|1|Women|507|ationbaranti|medium|9343246snow436828105|salmon|Bundle|Unknown|8|barantianticallyought| +16551|AAAAAAAAGKAEAAAA|2000-10-27||Families would sue also domestic plans. Green, specific lengths ought to feel for a sources; yesterday thorough effects deliver great, rough issues. Necessary sides cannot call here w|3.19|5.17|3001002|amalgexporti #2|4|newborn|3|Children|507|ationbaranti|medium|9343246snow436828105|medium|Pallet|Unknown|39|oughtantianticallyought| +16552|AAAAAAAAIKAEAAAA|1997-10-27|1999-10-27|Small, likely others would remove most existing, eastern churches. National residents draw definitely|8.93|6.16|6011001|amalgbrand #1|11|semi-precious|6|Jewelry|112|ableoughtought|N/A|821017thistle5701311|steel|Gram|Unknown|1|ableantianticallyought| +16553|AAAAAAAAIKAEAAAA|1999-10-28|2001-10-26|Small, likely others would remove most existing, eastern churches. National residents draw definitely|5.49|4.22|8009004|maxinameless #4|9|optics|8|Sports|37|ationpri|N/A|65035938pink11160587|orange|Case|Unknown|63|priantianticallyought| +16554|AAAAAAAAIKAEAAAA|2001-10-27||Small, likely others would remove most existing, eastern churches. National residents draw definitely|1.80|1.27|8009004|exportiexporti #1|9|toddlers|3|Children|64|esecally|large|910287777816peru6669|tan|Cup|Unknown|30|eseantianticallyought| +16555|AAAAAAAALKAEAAAA|1997-10-27||Much labour students build only female studies. Never progressive dates sh|40.27|23.35|2003002|exportiimporto #2|3|pants|2|Men|261|oughtcallyable|extra large|1664seashell75162647|peru|Carton|Unknown|11|antiantianticallyought| +16556|AAAAAAAAMKAEAAAA|1997-10-27|2000-10-26|Major, major vegetables play recently true cells. Numerous, previous schools cannot assess about only ultimate skills. As alon|5.27|3.05|7011007|amalgnameless #7|11|accent|7|Home|258|eingantiable|N/A|2714010029370482sky0|slate|Dram|Unknown|11|callyantianticallyought| +16557|AAAAAAAAMKAEAAAA|2000-10-27||Major, major vegetables play recently true cells. Numerous, previous schools cannot assess about only ultimate skills. As alon|0.42|3.05|10013017|exportiamalgamalg #17|11|stereo|10|Electronics|258|eingantiable|N/A|2714010029370482sky0|snow|Dram|Unknown|33|ationantianticallyought| +16558|AAAAAAAAOKAEAAAA|1997-10-27|1999-10-27|Soldiers can find scenes. Yet willin|4.42|3.66|6007007|brandcorp #7|7|pendants|6|Jewelry|205|antibarable|N/A|923659peru4748767160|medium|Unknown|Unknown|91|eingantianticallyought| +16559|AAAAAAAAOKAEAAAA|1999-10-28|2001-10-26|Soldiers can find scenes. Yet willin|8.50|3.66|6008008|namelesscorp #8|8|mens watch|6|Jewelry|374|antibarable|N/A|62766062777midnight9|metallic|Dram|Unknown|65|n stantianticallyought| +16560|AAAAAAAAOKAEAAAA|2001-10-27||Serious, silent incidents fold satisfactorily meetings. Trying things should predict relatively again blind numbers. Ways highlight; also willing statements cope importan|8.31|2.82|6008008|importoexporti #1|8|infants|3|Children|374|eseationpri|small|62766062777midnight9|indian|N/A|Unknown|17|barcallyanticallyought| +16561|AAAAAAAABLAEAAAA|1997-10-27||Disciplinary signs could prepare particular, other councils. Live criteria ought to stand extra, basic bones. Local leaders tell now tasks. Most old recordings use ago in the governors; par|8.96|4.39|5001002|amalgscholar #2|1|rock|5|Music|267|ationcallyable|N/A|36686mint01133048047|seashell|Gross|Unknown|50|oughtcallyanticallyought| +16562|AAAAAAAACLAEAAAA|1997-10-27|2000-10-26|As direct shoes cannot guarantee there regular given specialists. Teachers say even eyes. True re|1.33|0.41|9004003|edu packmaxi #3|4|entertainments|9|Books|809|n stbareing|N/A|62577129759489snow04|red|Oz|Unknown|29|ablecallyanticallyought| +16563|AAAAAAAACLAEAAAA|2000-10-27||As direct shoes cannot guarantee there regular given specialists. Teachers say even eyes. True re|4.31|0.41|4002002|importoedu pack #2|4|mens|4|Shoes|75|n stbareing|extra large|0104753maroon9580177|ivory|Tbl|Unknown|4|pricallyanticallyought| +16564|AAAAAAAAELAEAAAA|1997-10-27|1999-10-27|More human decades shall understand outside individuals; important factors think however; final schemes give now early, honest tourists; goods make actually female, possible emotions; clients woul|14.75|7.52|5004001|edu packscholar #1|4|classical|5|Music|330|barpripri|N/A|8metallic08400712647|light|Each|Unknown|40|esecallyanticallyought| +16565|AAAAAAAAELAEAAAA|1999-10-28|2001-10-26|Citizens provide. Traditional, private germans matter therefore new figures. Brief, great emotions ge|9.72|5.15|5004001|maxiunivamalg #8|9|televisions|10|Electronics|565|anticallyanti|N/A|38684127815073rosy15|indian|Each|Unknown|4|anticallyanticallyought| +16566|AAAAAAAAELAEAAAA|2001-10-27||Citizens provide. Traditional, private germans matter therefore new figures. Brief, great emotions ge|1.52|5.15|5004001|amalgamalg #1|9|dresses|1|Women|5|anticallyanti|medium|38684127815073rosy15|bisque|Each|Unknown|45|callycallyanticallyought| +16567|AAAAAAAAHLAEAAAA|1997-10-27||Professional risks would not win now. Prime words must not establish only in the eyes. Inwards previous candidates |3.00|2.34|10012010|importoamalgamalg #10|12|monitors|10|Electronics|173|priationought|N/A|146peach327763662662|yellow|Gross|Unknown|63|ationcallyanticallyought| +16568|AAAAAAAAILAEAAAA|1997-10-27|2000-10-26|Alm|6.14|4.66|9008009|namelessmaxi #9|8|romance|9|Books|101|oughtbarought|N/A|09418734656peach6404|rosy|Pound|Unknown|29|eingcallyanticallyought| +16569|AAAAAAAAILAEAAAA|2000-10-27||Alm|27.05|4.66|9008009|importobrand #2|8|costume|6|Jewelry|101|oughtbarought|N/A|09418734656peach6404|rose|Gross|Unknown|37|n stcallyanticallyought| +16570|AAAAAAAAKLAEAAAA|1997-10-27|1999-10-27|Factors locate actually through a programmes. Conservative days know however used words. Healthy classes look generously please satisfa|49.04|30.40|5002001|importoscholar #1|2|country|5|Music|75|antiation|N/A|316268133901yellow78|pale|Gram|Unknown|60|barationanticallyought| +16571|AAAAAAAAKLAEAAAA|1999-10-28|2001-10-26|Christian, good weeks advise as well different terms. Especially real tensions try others. Measures acknowledge pleasant, nor|2.83|30.40|5002001|scholarcorp #4|2|earings|6|Jewelry|76|antiation|N/A|seashell777354648407|red|Carton|Unknown|18|oughtationanticallyought| +16572|AAAAAAAAKLAEAAAA|2001-10-27||Christian, good weeks advise as well different terms. Especially real tensions try others. Measures acknowledge pleasant, nor|0.44|0.22|5002001|corpbrand #3|2|rugs|7|Home|76|callyation|N/A|seashell777354648407|rose|Cup|Unknown|38|ableationanticallyought| +16573|AAAAAAAANLAEAAAA|1997-10-27||Longer aware rooms say neighbouring, estimated offers. Teams might not make per a years; free, central shares will not know both foreign officers. Both lik|2.62|1.62|3002002|importoexporti #2|2|infants|3|Children|625|antiablecally|medium|9157294720snow564581|sandy|Unknown|Unknown|9|priationanticallyought| +16574|AAAAAAAAOLAEAAAA|1997-10-27|2000-10-26|Undoubtedly available minutes fancy low eyes; also great decisions play even just wise arrangements. New studies would keep away pages. White children like als|2.59|0.88|2004001|edu packimporto #1|4|sports-apparel|2|Men|214|eseoughtable|medium|15tomato234719937809|wheat|Bundle|Unknown|56|eseationanticallyought| +16575|AAAAAAAAOLAEAAAA|2000-10-27||New |2.93|2.02|2004001|edu packedu pack #2|4|athletic|4|Shoes|214|eseoughtable|medium|28medium171715593483|light|Cup|Unknown|19|antiationanticallyought| +16576|AAAAAAAAAMAEAAAA|1997-10-27|1999-10-27|Public projects ought to give. Customers deduce forcefully fast individuals. Forwards little clothes used to break. Natural, careful pounds used to c|7.27|2.18|10013002|exportiamalgamalg #2|13|stereo|10|Electronics|60|barcally|N/A|620peach072860014008|powder|Lb|Unknown|7|callyationanticallyought| +16577|AAAAAAAAAMAEAAAA|1999-10-28|2001-10-26|Public projects ought to give. Customers deduce forcefully fast individuals. Forwards little clothes used to break. Natural, careful pounds used to c|3.22|1.93|10013002|scholarnameless #8|13|tables|7|Home|226|callyableable|N/A|620peach072860014008|steel|Cup|Unknown|29|ationationanticallyought| +16578|AAAAAAAAAMAEAAAA|2001-10-27||Buildings shall answer more to a months|0.13|0.06|8016009|corpmaxi #9|16|golf|8|Sports|226|callyableable|N/A|620peach072860014008|plum|Cup|Unknown|53|eingationanticallyought| +16579|AAAAAAAADMAEAAAA|1997-10-27||Maximum, extensive workers used to go others. As young days will not miss in a years. Various details used to d|0.17|0.11|3002002|importoexporti #2|2|infants|3|Children|571|oughtationanti|medium|purple89363590022203|saddle|Cup|Unknown|64|n stationanticallyought| +16580|AAAAAAAAEMAEAAAA|1997-10-27|2000-10-26|Empirical improvements use at once eventually distinct rat|0.33|0.14|6001005|amalgcorp #5|1|birdal|6|Jewelry|316|callyoughtpri|N/A|59928658snow88041333|purple|Lb|Unknown|92|bareinganticallyought| +16581|AAAAAAAAEMAEAAAA|2000-10-27||Usually alone elements used t|82.58|0.14|6001005|edu packamalgamalg #7|1|automotive|10|Electronics|316|callyoughtpri|N/A|59928658snow88041333|powder|Oz|Unknown|3|oughteinganticallyought| +16582|AAAAAAAAGMAEAAAA|1997-10-27|1999-10-27|Partly interesting prices may use primary, big days. Also very forces communicate budgets. As difficult books explain forever never other employers. Still true minutes become lectures. Poor cases mat|3.01|0.99|1002001|importoamalg #1|2|fragrances|1|Women|396|callyn stpri|petite|5884830green26935303|violet|Pallet|Unknown|16|ableeinganticallyought| +16583|AAAAAAAAGMAEAAAA|1999-10-28|2001-10-26|Partly interesting prices may use primary, big days. Also very forces communicate budgets. As difficult books explain forever never other employers. Still true minutes become lectures. Poor cases mat|0.96|0.99|1002001|scholarnameless #4|15|tables|7|Home|396|callyn stpri|N/A|1503707867273pale664|seashell|Ounce|Unknown|14|prieinganticallyought| +16584|AAAAAAAAGMAEAAAA|2001-10-27||Very problems co|4.54|0.99|1002001|edu packbrand #1|14|estate|6|Jewelry|394|esen stpri|N/A|59415479152239coral2|turquoise|Box|Unknown|9|eseeinganticallyought| +16585|AAAAAAAAJMAEAAAA|1997-10-27||Simply likely things will tell then economic young critics; serious, weak animals migh|2.08|1.39|1001002|amalgamalg #2|1|dresses|1|Women|373|priationpri|petite|059salmon64846080525|yellow|Box|Unknown|51|antieinganticallyought| +16586|AAAAAAAAKMAEAAAA|1997-10-27|2000-10-26|True calls stand again now strong musicians; political, lovely directions know more financial charts. Probably overall eyes risk even meetings. Servic|3.81|2.40|8008007|namelessnameless #7|8|outdoor|8|Sports|150|barantiought|N/A|sky58247167129916957|sky|Tbl|Unknown|92|callyeinganticallyought| +16587|AAAAAAAAKMAEAAAA|2000-10-27||Both deaf organisations see just; good, large goods would improve never new officers. Very irish bi|3.43|2.40|7014008|edu packnameless #8|14|glassware|7|Home|150|barantiought|N/A|sky58247167129916957|red|Each|Unknown|9|ationeinganticallyought| +16588|AAAAAAAAMMAEAAAA|1997-10-27|1999-10-27|Religious leaders shall shift quite. Statutory equations|5.90|4.24|3001001|amalgexporti #1|1|newborn|3|Children|130|barpriought|economy|78829459585851slate4|white|Pallet|Unknown|45|eingeinganticallyought| +16589|AAAAAAAAMMAEAAAA|1999-10-28|2001-10-26|Either available effects know lines. Rel|1.03|4.24|3001001|edu packedu pack #2|1|athletic|4|Shoes|18|barpriought|medium|78829459585851slate4|yellow|Pound|Unknown|26|n steinganticallyought| +16590|AAAAAAAAMMAEAAAA|2001-10-27||Effective, short lives come forever; angry units could know british, horrible walls. Political issues should not atta|8.81|4.24|9016007|corpunivamalg #7|1|mystery|9|Books|18|eingought|N/A|78829459585851slate4|pink|Tbl|Unknown|26|barn stanticallyought| +16591|AAAAAAAAPMAEAAAA|1997-10-27||Great, political methods adapt in a characters. Slowly different cases fight|0.81|0.27|7002002|importobrand #2|2|bedding|7|Home|312|ableoughtpri|N/A|410puff2192816717928|pink|Carton|Unknown|93|oughtn stanticallyought| +16592|AAAAAAAAANAEAAAA|1997-10-27|2000-10-26|Known, narrow tr|99.43|49.71|3003001|exportiexporti #1|3|toddlers|3|Children|179|n stationought|large|2832984plum441756700|rosy|Bunch|Unknown|59|ablen stanticallyought| +16593|AAAAAAAAANAEAAAA|2000-10-27||National children regard very plus a schools. Social, separate subjects must add then with a amounts. Huge habits mean|1.47|1.17|3003001|exportiedu pack #2|3|kids|4|Shoes|179|n stationought|medium|54221ghost4383560275|purple|Tsp|Unknown|52|prin stanticallyought| +16594|AAAAAAAACNAEAAAA|1997-10-27|1999-10-27|Doctors obtain only etc financial friends. Big, close routes would copy then homes. Environmental bodies will becom|3.35|2.54|3004001|edu packexporti #1|4|school-uniforms|3|Children|225|antiableable|N/A|752smoke268620642488|tomato|Oz|Unknown|31|esen stanticallyought| +16595|AAAAAAAACNAEAAAA|1999-10-28|2001-10-26|Patients s|23.37|13.08|6015004|scholarbrand #4|15|custom|6|Jewelry|130|antiableable|N/A|1165830spring8654115|sky|Tsp|Unknown|31|antin stanticallyought| +16596|AAAAAAAACNAEAAAA|2001-10-27||Patients s|3.81|13.08|9008001|namelessmaxi #1|8|romance|9|Books|130|antiableable|N/A|1165830spring8654115|indian|Cup|Unknown|23|callyn stanticallyought| +16597|AAAAAAAAFNAEAAAA|1997-10-27||Days used to ease as central days. Different elements ought to give openly up to the computers; national, immediate parents run always hot states. New, fierce ye|6.80|3.19|6004006|edu packcorp #6|4|bracelets|6|Jewelry|703|pribaration|N/A|66236632smoke4508370|sandy|Tbl|Unknown|12|ationn stanticallyought| +16598|AAAAAAAAGNAEAAAA|1997-10-27|2000-10-26|Human, other performances take before with a arrangements. Northern colours should say all. Heavily nuclear reports observe also old magistrate|4.10|3.03|4004001|edu packedu pack #1|4|athletic|4|Shoes|43|priese|small|38seashell8068632155|powder|Each|Unknown|37|eingn stanticallyought| +16599|AAAAAAAAGNAEAAAA|2000-10-27||Necessary eyes shall work operations. Little, deaf discussions improve unequivocally members. Popular activities stop also subjects. Base parents make thus. Ministers bring. Lo|7.09|3.03|6007008|brandcorp #8|4|pendants|6|Jewelry|43|priese|N/A|38seashell8068632155|sandy|Ton|Unknown|22|n stn stanticallyought| +16600|AAAAAAAAINAEAAAA|1997-10-27|1999-10-27|Public, social laws share new, previous scientists. Other regulations include to a facilities. Tox|1.99|1.79|4001001|amalgedu pack #1|1|womens|4|Shoes|129|n stableought|petite|0468354514054violet6|purple|Bundle|Unknown|78|barbarcallycallyought| +16601|AAAAAAAAINAEAAAA|1999-10-28|2001-10-26|Combined techni|4.75|1.79|4001001|edu packscholar #2|1|classical|5|Music|225|antiableable|N/A|51orange057606860381|cornsilk|Carton|Unknown|22|oughtbarcallycallyought| +16602|AAAAAAAAINAEAAAA|2001-10-27||All right open guests may not need considerably. Rules might not make just smooth p|9.44|1.79|4001001|namelessunivamalg #11|8|scanners|10|Electronics|225|antiableable|N/A|51orange057606860381|sky|Carton|Unknown|24|ablebarcallycallyought| +16603|AAAAAAAALNAEAAAA|1997-10-27||Vegetables sell of course carefully peaceful proceedings. Necessary revenues should criticise much; public regulations must see mild pr|2.81|1.29|7004004|edu packbrand #4|4|curtains/drapes|7|Home|238|eingpriable|N/A|648007plum9398235069|lace|Gram|Unknown|46|pribarcallycallyought| +16604|AAAAAAAAMNAEAAAA|1997-10-27|2000-10-26|Once wrong terms can get presumably nuclear, excellent claims. Years possess details. Invisible feet might produce some|1.24|0.81|10011013|amalgamalgamalg #13|11|disk drives|10|Electronics|395|antin stpri|N/A|37thistle57917398193|snow|Ton|Unknown|29|esebarcallycallyought| +16605|AAAAAAAAMNAEAAAA|2000-10-27||Types say into a eyes. Substantial, main classes feel so; significant, particular estimates could win real investors. Apparently usual s|7.88|0.81|10011013|exportimaxi #10|11|computers|9|Books|97|ationn st|N/A|4860210641sandy15197|wheat|Unknown|Unknown|56|antibarcallycallyought| +16606|AAAAAAAAONAEAAAA|1997-10-27|1999-10-27|Quiet others shall not take m|7.14|5.21|6013007|exportibrand #7|13|loose stones|6|Jewelry|225|antiableable|N/A|7585182988sienna5940|pink|Unknown|Unknown|9|callybarcallycallyought| +16607|AAAAAAAAONAEAAAA|1999-10-28|2001-10-26|Colleagues must see still able members; today musical experiments like fruits. Cultural, absolute tec|0.09|5.21|3004002|edu packexporti #2|13|school-uniforms|3|Children|516|antiableable|large|016173228535568peru9|white|N/A|Unknown|82|ationbarcallycallyought| +16608|AAAAAAAAONAEAAAA|2001-10-27||Colleagues must see still able members; today musical experiments like fruits. Cultural, absolute tec|2.68|1.04|3004002|amalgimporto #1|13|accessories|2|Men|516|antiableable|economy|016173228535568peru9|royal|Bundle|Unknown|4|eingbarcallycallyought| +16609|AAAAAAAABOAEAAAA|1997-10-27||Changes take aspects; elements cannot make yesterday about a studies. Ba|7.49|3.59|4002002|importoedu pack #2|2|mens|4|Shoes|526|callyableanti|large|2054purple3480548245|powder|Cup|Unknown|18|n stbarcallycallyought| +16610|AAAAAAAACOAEAAAA|1997-10-27|2000-10-26|Military, economic words shall know |2.54|0.93|8010001|univmaxi #1|10|pools|8|Sports|258|eingantiable|N/A|6rosy924342575952654|gainsboro|Dram|Unknown|11|baroughtcallycallyought| +16611|AAAAAAAACOAEAAAA|2000-10-27||Obvious shares will seem also new, certain heads. Necessary, ro|1.04|0.93|8010001|exportibrand #4|13|loose stones|6|Jewelry|258|eingantiable|N/A|1707797251270lemon52|medium|Tsp|Unknown|12|oughtoughtcallycallyought| +16612|AAAAAAAAEOAEAAAA|1997-10-27|1999-10-27|Old, final citizens lose long distinguished conditions. National, little authorities get already; correctly dramatic communities repeat better local, intense months. Even thin years |0.33|0.20|7007003|brandbrand #3|7|decor|7|Home|628|eingablecally|N/A|977580thistle3623691|salmon|Case|Unknown|25|ableoughtcallycallyought| +16613|AAAAAAAAEOAEAAAA|1999-10-28|2001-10-26|Old, final citizens lose long distinguished conditions. National, little authorities get already; correctly dramatic communities repeat better local, intense months. Even thin years |3.30|0.20|2003002|exportiimporto #2|3|pants|2|Men|586|callyeinganti|extra large|977580thistle3623691|peach|Dram|Unknown|6|prioughtcallycallyought| +16614|AAAAAAAAEOAEAAAA|2001-10-27||Old, final citizens lose long distinguished conditions. National, little authorities get already; correctly dramatic communities repeat better local, intense months. Even thin years |7.48|5.68|2003002|edu packexporti #1|4|school-uniforms|3|Children|718|callyeinganti|economy|977580thistle3623691|sienna|Dram|Unknown|44|eseoughtcallycallyought| +16615|AAAAAAAAHOAEAAAA|1997-10-27||Questions could fig|0.09|0.04|5003002|exportischolar #2|3|pop|5|Music|285|antieingable|N/A|83075665plum22897115|slate|Dozen|Unknown|56|antioughtcallycallyought| +16616|AAAAAAAAIOAEAAAA|1997-10-27|2000-10-26|Frequently popular powers cannot submit rationally only, additional years. Hands split still never weak demonstrations. Easy sheets increase consequentl|0.84|0.59|10009008|maxiunivamalg #8|9|televisions|10|Electronics|547|ationeseanti|N/A|7632wheat21174994176|puff|Pound|Unknown|11|callyoughtcallycallyought| +16617|AAAAAAAAIOAEAAAA|2000-10-27||Frequently popular powers cannot submit rationally only, additional years. Hands split still never weak demonstrations. Easy sheets increase consequentl|8.22|0.59|9010010|univunivamalg #10|9|travel|9|Books|248|eingeseable|N/A|79427yellow027055250|seashell|Pound|Unknown|49|ationoughtcallycallyought| +16618|AAAAAAAAKOAEAAAA|1997-10-27|1999-10-27|Small, dead particles set recently other boxes. Bright, personal locations house novel jobs. Twice residential judges underpin directions. Others want. Other songs star too p|0.78|0.62|8013009|exportimaxi #9|13|sailing|8|Sports|70|baration|N/A|96037586905misty6093|royal|Gram|Unknown|1|eingoughtcallycallyought| +16619|AAAAAAAAKOAEAAAA|1999-10-28|2001-10-26|Small, dead particles set recently other boxes. Bright, personal locations house novel jobs. Twice residential judges underpin directions. Others want. Other songs star too p|8.35|0.62|5003002|exportischolar #2|3|pop|5|Music|70|baration|N/A|18280426spring410977|sienna|Ton|Unknown|52|n stoughtcallycallyought| +16620|AAAAAAAAKOAEAAAA|2001-10-27||Small, dead particles set recently other boxes. Bright, personal locations house novel jobs. Twice residential judges underpin directions. Others want. Other songs star too p|4.75|2.66|5003002|exportiamalg #1|3|maternity|1|Women|467|ationcallyese|petite|449260347honeydew475|lemon|Cup|Unknown|17|barablecallycallyought| +16621|AAAAAAAANOAEAAAA|1997-10-27||Labour, h|35.82|28.29|9012002|importounivamalg #2|12|home repair|9|Books|215|antioughtable|N/A|8slate03413219678684|mint|Dram|Unknown|37|oughtablecallycallyought| +16622|AAAAAAAAOOAEAAAA|1997-10-27|2000-10-26|Rath|4.34|1.34|4002001|importoedu pack #1|2|mens|4|Shoes|389|n steingpri|extra large|3snow158407572478729|cream|Pound|Unknown|61|ableablecallycallyought| +16623|AAAAAAAAOOAEAAAA|2000-10-27||Closely national others used to move now; of course possible days re|4.41|1.34|1002002|importoamalg #2|2|fragrances|1|Women|630|barprically|medium|3snow158407572478729|pale|Unknown|Unknown|7|priablecallycallyought| +16624|AAAAAAAAAPAEAAAA|1997-10-27|1999-10-27|Politically delicate proposals must incorporate however with a arts. Neighbouring, effective premises could not return quite much fi|8.76|3.24|5002001|importoscholar #1|2|country|5|Music|594|esen stanti|N/A|34753salmon978270456|salmon|N/A|Unknown|23|eseablecallycallyought| +16625|AAAAAAAAAPAEAAAA|1999-10-28|2001-10-26|Senior, wooden winds learn however difficult, dangerous qualifications; shallow, formal effects ought to slip to a definitions. Good, inc forces will produce only other colours. Cases result perhaps|5.10|4.08|3004002|edu packexporti #2|2|school-uniforms|3|Children|496|esen stanti|petite|78211432spring256228|honeydew|Lb|Unknown|2|antiablecallycallyought| +16626|AAAAAAAAAPAEAAAA|2001-10-27||Nice gains provide general studies. Interesting things can come out of a agreements; strongly numerous glasses might last only wrong metres. Reasonably steep members learn once |2.81|1.20|9007007|brandmaxi #7|2|reference|9|Books|496|esen stanti|N/A|78211432spring256228|pink|Tsp|Unknown|54|callyablecallycallyought| +16627|AAAAAAAADPAEAAAA|1997-10-27||Weeks used to like here around the legs. Private friends watch very alone essential pairs. Politicians ca|1.73|0.60|5002002|importoscholar #2|2|country|5|Music|759|n stantiation|N/A|76violet544596722506|peru|Oz|Unknown|23|ationablecallycallyought| +16628|AAAAAAAAEPAEAAAA|1997-10-27|2000-10-26|Instructions must change successful clothes. Here appropriate procedures will say hori|3.94|2.91|4002001|importoedu pack #1|2|mens|4|Shoes|370|barationpri|N/A|2181pink455556967986|lawn|Oz|Unknown|52|eingablecallycallyought| +16629|AAAAAAAAEPAEAAAA|2000-10-27||Instructions must change successful clothes. Here appropriate procedures will say hori|7.51|2.91|2002002|importoimporto #2|2|shirts|2|Men|370|barationpri|extra large|87302224068magenta85|spring|Dram|Unknown|36|n stablecallycallyought| +16630|AAAAAAAAGPAEAAAA|1997-10-27|1999-10-27|Now careful pieces could drop small degrees. Genetic witn|1.12|0.43|5004001|edu packscholar #1|4|classical|5|Music|297|ationn stable|N/A|2517rose559222462379|peru|Cup|Unknown|20|barpricallycallyought| +16631|AAAAAAAAGPAEAAAA|1999-10-28|2001-10-26|Forward, welsh comments look social, minor conditions. Related soldiers should not go difficult|2.57|0.43|4003002|exportiedu pack #2|4|kids|4|Shoes|297|ationn stable|petite|2517rose559222462379|pink|Ton|Unknown|23|oughtpricallycallyought| +16632|AAAAAAAAGPAEAAAA|2001-10-27||Main games ought to report too too late insects. Healthy scores cannot comply away citizens. Visual, free costs might shape very necessary, high discussions; liberal, small investigations listen mate|1.18|0.43|4003002|corpunivamalg #1|4|mystery|9|Books|593|prin stanti|N/A|2517rose559222462379|royal|Case|Unknown|82|ablepricallycallyought| +16633|AAAAAAAAJPAEAAAA|1997-10-27||Natural times shall not anticipate black, possible hands|4.16|3.07|9014002|edu packunivamalg #2|14|sports|9|Books|326|callyablepri|N/A|1blue871978362193381|sky|Pallet|Unknown|65|pripricallycallyought| +16634|AAAAAAAAKPAEAAAA|1997-10-27|2000-10-26|Centres would advise here most joint types. Equal forms hear months.|4.82|1.87|8011005|amalgmaxi #5|11|archery|8|Sports|165|anticallyought|N/A|smoke561766884513333|tan|Lb|Unknown|99|esepricallycallyought| +16635|AAAAAAAAKPAEAAAA|2000-10-27||Centres would advise here most joint types. Equal forms hear months.|3.32|2.62|5002002|importoscholar #2|2|country|5|Music|195|antin stought|N/A|76905ghost7984137635|honeydew|Gram|Unknown|9|antipricallycallyought| +16636|AAAAAAAAMPAEAAAA|1997-10-27|1999-10-27|Single bits will allow. Worthy, active letters see idly conscious women. Traditional, popular programmes should not say. G|5.17|2.06|10001011|amalgunivamalg #11|1|cameras|10|Electronics|448|eingeseese|N/A|7752896550426smoke98|steel|Pallet|Unknown|66|callypricallycallyought| +16637|AAAAAAAAMPAEAAAA|1999-10-28|2001-10-26|Elderly departments may take always human members. Neighbouring, other children get moreover rapid others. Low schools would choose times. Factors pump also angry, special feelings; cul|3.13|2.06|10011012|amalgamalgamalg #12|11|disk drives|10|Electronics|448|eingeseese|N/A|9025red7385684589644|wheat|Carton|Unknown|24|ationpricallycallyought| +16638|AAAAAAAAMPAEAAAA|2001-10-27||Elderly departments may take always human members. Neighbouring, other children get moreover rapid others. Low schools would choose times. Factors pump also angry, special feelings; cul|1.33|2.06|10011012|exportiamalgamalg #13|11|stereo|10|Electronics|640|eingeseese|N/A|2655674485983sandy09|puff|Case|Unknown|51|eingpricallycallyought| +16639|AAAAAAAAPPAEAAAA|1997-10-27||Bad, new |2.23|1.11|7009010|maxibrand #10|9|mattresses|7|Home|187|ationeingought|N/A|tan32666224089816154|peru|Oz|Unknown|20|n stpricallycallyought| +16640|AAAAAAAAAABEAAAA|1997-10-27|2000-10-26|La|1.99|1.55|4002001|importoedu pack #1|2|mens|4|Shoes|489|n steingese|extra large|235970382699306plum8|red|Bundle|Unknown|41|baresecallycallyought| +16641|AAAAAAAAAABEAAAA|2000-10-27||New meetings discuss also occasional, corporate approaches. Specialist, mixed groups ought t|8.23|7.40|4002001|edu packscholar #2|2|classical|5|Music|489|n steingese|N/A|235970382699306plum8|lime|Ounce|Unknown|66|oughtesecallycallyought| +16642|AAAAAAAACABEAAAA|1997-10-27|1999-10-27|Married applicants get just between a changes. Only recent arguments exert up a members. Entire, capable result|1.37|0.64|10006017|corpunivamalg #17|6|musical|10|Electronics|108|eingbarought|N/A|505086tan46642343416|white|Gross|Unknown|31|ableesecallycallyought| +16643|AAAAAAAACABEAAAA|1999-10-28|2001-10-26|Married applicants get just between a changes. Only recent arguments exert up a members. Entire, capable result|8.71|0.64|2001002|amalgimporto #2|1|accessories|2|Men|108|eingbarought|small|70white4140141524270|sandy|Case|Unknown|1|priesecallycallyought| +16644|AAAAAAAACABEAAAA|2001-10-27||New things ought to gai|1.10|0.92|3003001|exportiexporti #1|3|toddlers|3|Children|887|eingbarought|medium|335744492788peach177|midnight|Lb|Unknown|33|eseesecallycallyought| +16645|AAAAAAAAFABEAAAA|1997-10-27||Houses design|4.80|1.68|7001006|amalgbrand #6|1|bathroom|7|Home|226|callyableable|N/A|86001lace64338117918|medium|Gram|Unknown|23|antiesecallycallyought| +16646|AAAAAAAAGABEAAAA|1997-10-27|2000-10-26|Quite hu|9.76|4.58|2003001|exportiimporto #1|3|pants|2|Men|232|ablepriable|medium|92018477243olive9762|sky|Dram|Unknown|17|callyesecallycallyought| +16647|AAAAAAAAGABEAAAA|2000-10-27|||1.32|||importobrand #8||||Home||ablepriable||92018477243olive9762||||45|ationesecallycallyought| +16648|AAAAAAAAIABEAAAA|1997-10-27|1999-10-27|However innocent proceedings should want now established photographs. Courts could not want also. Great, aware|1.34|0.95|5004001|edu packscholar #1|4|classical|5|Music|905|antibarn st|N/A|7265purple5099325591|turquoise|Ounce|Unknown|13|eingesecallycallyought| +16649|AAAAAAAAIABEAAAA|1999-10-28|2001-10-26|Levels die women. Strong, broken materials should appreciate really rather than a equations. Simply comparative police should think often finally dual councils. Officials admit sure parliamentary|2.32|1.27|5004001|exportiamalg #2|4|maternity|1|Women|442|antibarn st|small|7265purple5099325591|thistle|Gram|Unknown|73|n stesecallycallyought| +16650|AAAAAAAAIABEAAAA|2001-10-27||More english women used to apply new projects. Constant effor|7.57|1.27|2001001|amalgimporto #1|4|accessories|2|Men|442|ableeseese|large|7265purple5099325591|peru|Tsp|Unknown|66|baranticallycallyought| +16651|AAAAAAAALABEAAAA|1997-10-27||Sudden patients could come royal needs; directly specific contracts fight terrible, raw amounts; simple,|3.21|1.15|5003002|exportischolar #2|3|pop|5|Music|85|antieing|N/A|562886blanched331141|light|Pallet|Unknown|43|oughtanticallycallyought| +16652|AAAAAAAAMABEAAAA|1997-10-27|2000-10-26|Other, blue legs could come heavy, full circumstances. Negotiations would grant further social programmes. Ready methods used to establish regular, extra plants. Comments need much thro|0.73|0.21|10010010|univamalgamalg #10|10|memory|10|Electronics|133|pripriought|N/A|15984587734lavender0|steel|N/A|Unknown|26|ableanticallycallyought| +16653|AAAAAAAAMABEAAAA|2000-10-27||Other, blue legs could come heavy, full circumstances. Negotiations would grant further social programmes. Ready methods used to establish regular, extra plants. Comments need much thro|3.14|0.21|7010004|univnameless #4|10|flatware|7|Home|2|pripriought|N/A|15984587734lavender0|smoke|Dozen|Unknown|29|prianticallycallyought| +16654|AAAAAAAAOABEAAAA|1997-10-27|1999-10-27|Similar engines must not take th|0.22|0.18|10011012|amalgamalgamalg #12|11|disk drives|10|Electronics|57|ationanti|N/A|orange55530082316449|grey|Carton|Unknown|44|eseanticallycallyought| +16655|AAAAAAAAOABEAAAA|1999-10-28|2001-10-26|Seats may think previously teams; narrow, due positions care perhaps properly comparable minds. Future phenomena might suit on the wages. At all sexual animals could not g|10.73|0.18|10011012|namelessunivamalg #13|11|scanners|10|Electronics|140|ationanti|N/A|orange55530082316449|moccasin|Ton|Unknown|52|antianticallycallyought| +16656|AAAAAAAAOABEAAAA|2001-10-27||Speeches ought to override thus at a legs; open streets use. Words will not murder debts. Odd,|1.38|0.18|5001001|amalgscholar #1|11|rock|5|Music|333|pripripri|N/A|rosy2516548371946192|deep|Lb|Unknown|27|callyanticallycallyought| +16657|AAAAAAAABBBEAAAA|1997-10-27||Present, great services can articul|4.20|1.34|6006002|corpcorp #2|6|rings|6|Jewelry|154|eseantiought|N/A|6336802413rosy432196|metallic|Ounce|Unknown|32|ationanticallycallyought| +16658|AAAAAAAACBBEAAAA|1997-10-27|2000-10-26|So white criteria remove only normal words. Traditions would find painfully immediately american proceedings. Soft, local subje|5.23|4.49|3003001|exportiexporti #1|3|toddlers|3|Children|994|esen stn st|large|31675921coral2835914|turquoise|Tsp|Unknown|92|einganticallycallyought| +16659|AAAAAAAACBBEAAAA|2000-10-27||So white criteria remove only normal words. Traditions would find painfully immediately american proceedings. Soft, local subje|7.77|4.49|3003001|amalgexporti #2|3|newborn|3|Children|994|esen stn st|medium|1peach89018855181115|red|Box|Unknown|51|n stanticallycallyought| +16660|AAAAAAAAEBBEAAAA|1997-10-27|1999-10-27|Systems may say strong properties. Open, clear rocks used to occupy together revolutionary, large fears. Females enjoy able, continuing bits. Known, funny t|3.02|1.41|8015001|scholarmaxi #1|15|fishing|8|Sports|277|ationationable|N/A|saddle16032637073089|snow|Dram|Unknown|41|barcallycallycallyought| +16661|AAAAAAAAEBBEAAAA|1999-10-28|2001-10-26|Systems may say strong properties. Open, clear rocks used to occupy together revolutionary, large fears. Females enjoy able, continuing bits. Known, funny t|2.42|1.41|7014002|edu packnameless #2|15|glassware|7|Home|277|ationationable|N/A|46honeydew5333792219|spring|Bunch|Unknown|63|oughtcallycallycallyought| +16662|AAAAAAAAEBBEAAAA|2001-10-27||Central lives sum rarely through the years; members carry carefully pictures. Little trees ought to settle now at the ministers. Accidents shall not organ|6.32|1.41|7014002|edu packscholar #1|15|classical|5|Music|277|ationationable|N/A|46honeydew5333792219|yellow|Cup|Unknown|74|ablecallycallycallyought| +16663|AAAAAAAAHBBEAAAA|1997-10-27||As usual new functions ought to go locally. Eligible, relative russians say privately vulnerable, apparent mechanisms. Accurate americans must make far from networks. A|7.72|3.78|2002002|importoimporto #2|2|shirts|2|Men|33|pripri|economy|tan27998460344019949|pale|Pallet|Unknown|70|pricallycallycallyought| +16664|AAAAAAAAIBBEAAAA|1997-10-27|2000-10-26|Basic observers invite by a nations. |9.20|4.04|5001001|amalgscholar #1|1|rock|5|Music|523|priableanti|N/A|60319891330284snow32|seashell|Dram|Unknown|50|esecallycallycallyought| +16665|AAAAAAAAIBBEAAAA|2000-10-27||Basic observers invite by a nations. |5.44|4.04|5001001|importoamalg #2|2|fragrances|1|Women|523|priableanti|medium|38569magenta86423189|wheat|Gross|Unknown|70|anticallycallycallyought| +16666|AAAAAAAAKBBEAAAA|1997-10-27|1999-10-27|Beaches could make. Clearly implicit experiments move clearly double forests; financial, fond suggestions impart here areas. Flowers might go actually liable cas|8.38|7.45|6013003|exportibrand #3|13|loose stones|6|Jewelry|390|barn stpri|N/A|8773398lace489700037|midnight|Each|Unknown|51|callycallycallycallyought| +16667|AAAAAAAAKBBEAAAA|1999-10-28|2001-10-26|Statutor|5.30|3.44|6013003|importounivamalg #8|13|camcorders|10|Electronics|390|barn stpri|N/A|8773398lace489700037|red|Box|Unknown|40|ationcallycallycallyought| +16668|AAAAAAAAKBBEAAAA|2001-10-27||Elegant, monthly relatives cannot lose even impossible authorities. National, agricultural causes would take types. Foreign, severe values should sell freely technica|37.87|16.28|8013009|exportimaxi #9|13|sailing|8|Sports|390|barn stpri|N/A|8773398lace489700037|orange|Gross|Unknown|75|eingcallycallycallyought| +16669|AAAAAAAANBBEAAAA|1997-10-27||Holy, disastrous tests may leave properly mysterious demands. Bad, insufficient shoulders could step then walls. Below new feet should belong very from|1.79|0.93|3001002|amalgexporti #2|1|newborn|3|Children|429|n stableese|small|5522902slate78225610|tomato|Ton|Unknown|58|n stcallycallycallyought| +16670|AAAAAAAAOBBEAAAA|1997-10-27|2000-10-26|Things could not look also new, likely w|7.16|6.37|6009007|maxicorp #7|9|womens watch|6|Jewelry|520|barableanti|N/A|46614050gainsboro879|seashell|Box|Unknown|29|barationcallycallyought| +16671|AAAAAAAAOBBEAAAA|2000-10-27||Things could not look also new, likely w|0.71|0.31|6009007|amalgscholar #2|9|rock|5|Music|520|barableanti|N/A|46614050gainsboro879|salmon|Each|Unknown|29|oughtationcallycallyought| +16672|AAAAAAAAACBEAAAA|1997-10-27|1999-10-27|Chief, sudden shows may see normally almost young points. Jewish addresses sound british towns. Varieties sit by the clergy. Heavily true measures might not worry also developments. Esse|2.65|0.98|9011005|amalgunivamalg #5|11|cooking|9|Books|967|ationcallyn st|N/A|059002snow7855540462|indian|Dram|Unknown|27|ableationcallycallyought| +16673|AAAAAAAAACBEAAAA|1999-10-28|2001-10-26|Variations used to direct now. Controversial hours ought to admire then main clients; general, unable boys contain so for a prisoners. Little great profits cann|3.33|0.98|9011005|amalgimporto #2|11|accessories|2|Men|570|ationcallyn st|large|059002snow7855540462|sky|Pound|Unknown|82|priationcallycallyought| +16674|AAAAAAAAACBEAAAA|2001-10-27||Variations used to direct now. Controversial hours ought to admire then main clients; general, unable boys contain so for a prisoners. Little great profits cann|9.38|6.56|5003001|exportischolar #1|3|pop|5|Music|570|barationanti|N/A|832992037677forest72|ghost|Tsp|Unknown|88|eseationcallycallyought| +16675|AAAAAAAADCBEAAAA|1997-10-27||New years care away here colonial yards. |1.33|0.43|3004002|edu packexporti #2|4|school-uniforms|3|Children|351|oughtantipri|large|3yellow2911632580272|pink|N/A|Unknown|84|antiationcallycallyought| +16676|AAAAAAAAECBEAAAA|1997-10-27|2000-10-26|In|4.71|1.41|6009005|maxicorp #5|9|womens watch|6|Jewelry|565|anticallyanti|N/A|859maroon94251491360|lavender|Carton|Unknown|26|callyationcallycallyought| +16677|AAAAAAAAECBEAAAA|2000-10-27||Conditions would doubt supposedly recent, rigid doctors. Old, informal tactics control alone in a points; male aspects require more both blac|8.65|1.41|6009005|amalgscholar #2|9|rock|5|Music|565|anticallyanti|N/A|859maroon94251491360|indian|Pound|Unknown|7|ationationcallycallyought| +16678|AAAAAAAAGCBEAAAA|1997-10-27|1999-10-27|Ages must not remain too various kinds; particular, responsible guns use in the rights. Small techniques contain really main, english wages. Firmly industr|17.58|7.38|4003001|exportiedu pack #1|3|kids|4|Shoes|640|baresecally|small|04161061green1641836|dim|Gram|Unknown|55|eingationcallycallyought| +16679|AAAAAAAAGCBEAAAA|1999-10-28|2001-10-26|Minimum, full politicians shall add else. Voices know hardly attractive, spatial levels. Weapons forget now senior, interior services. Centre|4.86|7.38|1001002|amalgamalg #2|3|dresses|1|Women|640|baresecally|extra large|04161061green1641836|sienna|Gram|Unknown|11|n stationcallycallyought| +16680|AAAAAAAAGCBEAAAA|2001-10-27||Comfortable, useful cells make however hopes. Likely, anxious rivers reinforce effective details. Continued questions would not discount over other, medical|6.96|7.38|1001002|edu packmaxi #1|4|entertainments|9|Books|640|baresecally|N/A|66300229wheat9681290|tomato|Lb|Unknown|61|bareingcallycallyought| +16681|AAAAAAAAJCBEAAAA|1997-10-27||Twice aware reports rise very i|2.90|1.76|6003002|exporticorp #2|3|gold|6|Jewelry|10|barought|N/A|849171092tomato48485|seashell|N/A|Unknown|18|oughteingcallycallyought| +16682|AAAAAAAAKCBEAAAA|1997-10-27|2000-10-26|Problems shall leave rapidly real sales. Just fo|1.46|0.67|8004003|edu packnameless #3|4|camping|8|Sports|981|oughteingn st|N/A|14192508814147grey89|cream|Pound|Unknown|1|ableeingcallycallyought| +16683|AAAAAAAAKCBEAAAA|2000-10-27||Problems shall leave rapidly real sales. Just fo|4.30|2.96|4001002|amalgedu pack #2|4|womens|4|Shoes|356|callyantipri|medium|14192508814147grey89|medium|Pound|Unknown|51|prieingcallycallyought| +16684|AAAAAAAAMCBEAAAA|1997-10-27|1999-10-27|Important others last great, left personnel. Almost back levels should conflict by far colleagues. Yards must look personally term|9.94|5.66|2004001|edu packimporto #1|4|sports-apparel|2|Men|208|eingbarable|petite|8253026182926violet1|spring|N/A|Unknown|14|eseeingcallycallyought| +16685|AAAAAAAAMCBEAAAA|1999-10-28|2001-10-26|Helpful, natural change|3.18|1.17|1003002|exportiamalg #2|3|maternity|1|Women|208|eingbarable|petite|8253026182926violet1|tomato|Lb|Unknown|94|antieingcallycallyought| +16686|AAAAAAAAMCBEAAAA|2001-10-27||Helpful, natural change|0.99|0.70|4004001|edu packedu pack #1|3|athletic|4|Shoes|208|eingbarable|medium|19809002642192royal3|pink|Gross|Unknown|22|callyeingcallycallyought| +16687|AAAAAAAAPCBEAAAA|1997-10-27||Actually military individuals resist obviously natural han|73.80|28.04|3001002|amalgexporti #2|1|newborn|3|Children|2|able|N/A|red94356123634693827|pink|Cup|Unknown|33|ationeingcallycallyought| +16688|AAAAAAAAADBEAAAA|1997-10-27|2000-10-26|Also previous organisers learn bands. Mental|9.44|3.96|6010001|univbrand #1|10|jewelry boxes|6|Jewelry|321|oughtablepri|N/A|4992frosted310627688|maroon|Case|Unknown|42|eingeingcallycallyought| +16689|AAAAAAAAADBEAAAA|2000-10-27||Also previous organisers learn bands. Mental|1.44|3.96|4004002|edu packedu pack #2|10|athletic|4|Shoes|321|oughtablepri|medium|4992frosted310627688|sienna|Dozen|Unknown|3|n steingcallycallyought| +16690|AAAAAAAACDBEAAAA|1997-10-27|1999-10-27|Premier, free documents shall not take christian, prime women. Wrong, special rules would obtain into a items. Young, important curtains shall say about police; other, international times|9.56|5.92|2002001|importoimporto #1|2|shirts|2|Men|95|antin st|small|46pink39333028201296|spring|Tsp|Unknown|28|barn stcallycallyought| +16691|AAAAAAAACDBEAAAA|1999-10-28|2001-10-26|Premier, free documents shall not take christian, prime women. Wrong, special rules would obtain into a items. Young, important curtains shall say about police; other, international times|14.19|6.10|2002001|brandcorp #4|2|pendants|6|Jewelry|528|antin st|N/A|285peach945307878626|red|Bundle|Unknown|40|oughtn stcallycallyought| +16692|AAAAAAAACDBEAAAA|2001-10-27||Obvious references will buy extremely good days. Single, old differences shall set new, private r|4.03|6.10|2002001|exportiimporto #1|2|pants|2|Men|99|antin st|extra large|285peach945307878626|red|Pallet|Unknown|22|ablen stcallycallyought| +16693|AAAAAAAAFDBEAAAA|1997-10-27||Especially strong members shall take good areas. |1.74|1.54|1001002|amalgamalg #2|1|dresses|1|Women|119|n stoughtought|large|76043450692pink80633|salmon|Pallet|Unknown|43|prin stcallycallyought| +16694|AAAAAAAAGDBEAAAA|1997-10-27|2000-10-26|Apparent ci|1.68|0.50|3001001|amalgexporti #1|1|newborn|3|Children|267|ationcallyable|medium|213425370peru0475334|saddle|Each|Unknown|25|esen stcallycallyought| +16695|AAAAAAAAGDBEAAAA|2000-10-27||Cultures may make merely fresh, big pieces. Departments get conveniently badly social doors. Serious, huge weeks review victims; still po|2.03|1.70|3001001|edu packedu pack #2|1|athletic|4|Shoes|190|ationcallyable|large|752peru5085990212069|purple|Oz|Unknown|42|antin stcallycallyought| +16696|AAAAAAAAIDBEAAAA|1997-10-27|1999-10-27|Just global faces ought to ensure in a sports. Results say individuals. Things ought to grant |5.79|4.22|6012001|importobrand #1|12|costume|6|Jewelry|382|ableeingpri|N/A|01343911793saddle793|smoke|Case|Unknown|41|callyn stcallycallyought| +16697|AAAAAAAAIDBEAAAA|1999-10-28|2001-10-26|Pupils bear early judges. Other, british things argue in vain unlikely authorities. Somewhere english sons aim good, local projects. New, nece|7.96|6.76|6012001|importoedu pack #2|2|mens|4|Shoes|382|ableeingpri|medium|01343911793saddle793|magenta|Gram|Unknown|13|ationn stcallycallyought| +16698|AAAAAAAAIDBEAAAA|2001-10-27||Pupils bear early judges. Other, british things argue in vain unlikely authorities. Somewhere english sons aim good, local projects. New, nece|1.47|6.76|6012001|amalgimporto #1|1|accessories|2|Men|684|eseeingcally|petite|375882pale5364075550|wheat|Ton|Unknown|37|eingn stcallycallyought| +16699|AAAAAAAALDBEAAAA|1997-10-27||Most new lovers drive real|0.94|0.61|2003002|exportiimporto #2|3|pants|2|Men|143|prieseought|medium|72006053194dim052255|sky|Gross|Unknown|4|n stn stcallycallyought| +16700|AAAAAAAAMDBEAAAA|1997-10-27|2000-10-26|Children must not carry concerned, only costs. Important powers would store bright meals; as bloody men talk also terms. Rare forms may mind with a assessments. Yesterday |4.92|2.60|7003001|exportibrand #1|3|kids|7|Home|113|prioughtought|N/A|182657348blue2405744|midnight|Gross|Unknown|21|barbarationcallyought| +16701|AAAAAAAAMDBEAAAA|2000-10-27||Pools make high others. Immediately impossible cars listen on a dreams. Other schemes can help largely years; grey, informal programmes should not display only, diff|4.08|3.34|7003001|exportinameless #2|3|basketball|8|Sports|309|n stbarpri|N/A|5papaya0263474136721|brown|Box|Unknown|29|oughtbarationcallyought| +16702|AAAAAAAAODBEAAAA|1997-10-27|1999-10-27|||0.25||amalgscholar #1|1|rock|5|||n stoughtpri||||Ounce|Unknown||| +16703|AAAAAAAAODBEAAAA|1999-10-28|2001-10-26|Possibilities must not want open, personal things; things shall not take only resulting trees. English, dynamic year|9.55|7.92|9012012|importounivamalg #12|12|home repair|9|Books|567|n stoughtpri|N/A|10266954592plum34588|pale|Oz|Unknown|85|pribarationcallyought| +16704|AAAAAAAAODBEAAAA|2001-10-27||Possibilities must not want open, personal things; things shall not take only resulting trees. English, dynamic year|2.10|1.57|3001001|amalgexporti #1|12|newborn|3|Children|567|n stoughtpri|extra large|10266954592plum34588|blue|Pallet|Unknown|7|esebarationcallyought| +16705|AAAAAAAABEBEAAAA|1997-10-27||Within crucial links grant fo|4.99|4.29|3003002|exportiexporti #2|3|toddlers|3|Children|182|ableeingought|medium|0520450686red3785907|lemon|Pound|Unknown|66|antibarationcallyought| +16706|AAAAAAAACEBEAAAA|1997-10-27|2000-10-26|Also alone m|7.66|5.05|8008007|namelessnameless #7|8|outdoor|8|Sports|264|esecallyable|N/A|991377orchid90204796|yellow|Each|Unknown|53|callybarationcallyought| +16707|AAAAAAAACEBEAAAA|2000-10-27||Also alone m|3.82|5.05|8008007|importoedu pack #2|2|mens|4|Shoes|264|esecallyable|medium|9wheat79765833200913|seashell|Dram|Unknown|18|ationbarationcallyought| +16708|AAAAAAAAEEBEAAAA|1997-10-27|1999-10-27|Only brief employees may not travel middle-class, strong computers. White days put particularly versions. As main remarks become both in a girls. Irish, flex|1.17|0.67|5003001|exportischolar #1|3|pop|5|Music|627|ationablecally|N/A|06601459166752rosy45|grey|Tsp|Unknown|2|eingbarationcallyought| +16709|AAAAAAAAEEBEAAAA|1999-10-28|2001-10-26|Years may not succeed with a cases. Historians may not compare parts. Objective jobs must buy as too other goals. Statements will not smooth broken p|7.49|0.67|5002002|importoscholar #2|2|country|5|Music|413|prioughtese|N/A|4397530926white49848|slate|Pallet|Unknown|47|n stbarationcallyought| +16710|AAAAAAAAEEBEAAAA|2001-10-27||Circ|1.04|0.71|5002002|exportiexporti #1|2|toddlers|3|Children|210|baroughtable|extra large|4397530926white49848|brown|Unknown|Unknown|41|baroughtationcallyought| +16711|AAAAAAAAHEBEAAAA|1997-10-27||So military words ought to get also able, economic practices. Sometimes open messages might sue. Final, pretty arts sleep more. Wealthy attacks must not want now in |12.25|4.28|1004002|edu packamalg #2|4|swimwear|1|Women|599|n stn stanti|medium|365583009yellow29357|salmon|Box|Unknown|6|oughtoughtationcallyought| +16712|AAAAAAAAIEBEAAAA|1997-10-27|2000-10-26|Historic, basic services compete almost services. Customers must happen tight regarding a companies. Pupils see well. Now|2.97|1.78|8015003|scholarmaxi #3|15|fishing|8|Sports|468|eingcallyese|N/A|olive734083774866376|light|Pound|Unknown|10|ableoughtationcallyought| +16713|AAAAAAAAIEBEAAAA|2000-10-27||Historic, basic services compete almost services. Customers must happen tight regarding a companies. Pupils see well. Now|0.91|1.78|8015003|amalgimporto #2|15|accessories|2|Men|468|eingcallyese|small|548205snow4451130812|lace|Bunch|Unknown|97|prioughtationcallyought| +16714|AAAAAAAAKEBEAAAA|1997-10-27|1999-10-27|Occupational examples come even on the participants. Difficult differences must t|1.81|1.26|6006003|corpcorp #3|6|rings|6|Jewelry|228|eingableable|N/A|6404435turquoise2855|light|Gross|Unknown|32|eseoughtationcallyought| +16715|AAAAAAAAKEBEAAAA|1999-10-28|2001-10-26|Never extra rights must drift at a appointments. Moral, blank expectations|0.23|1.26|8009006|maxinameless #6|6|optics|8|Sports|576|callyationanti|N/A|639465slate916270725|tan|Unknown|Unknown|3|antioughtationcallyought| +16716|AAAAAAAAKEBEAAAA|2001-10-27||Normally clear designers hear then agencies. Thus different jobs look neighbours. Only average years shape affairs. Obviously capital clothes would stand therefore addresses. Diff|4.47|1.26|8009006|scholarmaxi #7|15|fishing|8|Sports|576|callyationanti|N/A|639465slate916270725|tomato|N/A|Unknown|72|callyoughtationcallyought| +16717|AAAAAAAANEBEAAAA|1997-10-27||Huge circumstances result successfully for the fees. Human, green times cannot continue inc colours. British, used stars see too conventional contr|8.10|4.53|3004002|edu packexporti #2|4|school-uniforms|3|Children|1|ought|small|93339859ivory6269055|pink|Cup|Unknown|43|ationoughtationcallyought| +16718|AAAAAAAAOEBEAAAA|1997-10-27|2000-10-26|Always immediate pilots say just young, alive women. Japanese, female words discover early patients. Wonderfu|1.01|0.44|3002001|importoexporti #1|2|infants|3|Children|650|barantically|medium|78605300848tan101556|thistle|Case|Unknown|24|eingoughtationcallyought| +16719|AAAAAAAAOEBEAAAA|2000-10-27||Cautious, teenage women would alter either irish elements. Useful opportunities put just only, other |4.43|3.32|3002001|corpbrand #8|2|consignment|6|Jewelry|650|barantically|N/A|78605300848tan101556|snow|Tsp|Unknown|28|n stoughtationcallyought| +16720|AAAAAAAAAFBEAAAA|1997-10-27|1999-10-27|Only particular errors might hold teams. More part-time owners shall attract; ancient names ought to agree further inappropriate chairs; sensitive times look. Small colours|2.46|0.91|5003001|exportischolar #1|3|pop|5|Music|252|ableantiable|N/A|439357949749dim98585|mint|Unknown|Unknown|5|barableationcallyought| +16721|AAAAAAAAAFBEAAAA|1999-10-28|2001-10-26|General trustees carry for example exclusive, true advantages. Paintings cannot remember gently members. Huge, competitive houses bring. Mothers would continue|3.04|2.34|9007006|brandmaxi #6|3|reference|9|Books|810|baroughteing|N/A|619peach535587447718|sky|Oz|Unknown|27|oughtableationcallyought| +16722|AAAAAAAAAFBEAAAA|2001-10-27||General trustees carry for example exclusive, true advantages. Paintings cannot remember gently members. Huge, competitive houses bring. Mothers would continue|85.17|68.98|5004001|edu packscholar #1|4|classical|5|Music|810|baroughteing|N/A|255134704629382lime8|saddle|Each|Unknown|39|ableableationcallyought| +16723|AAAAAAAADFBEAAAA|1997-10-27||Sharp pools strike e|3.96|1.94|9010008|univunivamalg #8|10|travel|9|Books|852|ableantieing|N/A|035980584257534sky29|seashell|Bunch|Unknown|18|priableationcallyought| +16724|AAAAAAAAEFBEAAAA|1997-10-27|2000-10-26|Police may not recognise never still personal wheels. Joint, sad statements ac|2.70|1.18|2004001|edu packimporto #1|4|sports-apparel|2|Men|125|antiableought|petite|8575227purple3664272|turquoise|Each|Unknown|5|eseableationcallyought| +16725|AAAAAAAAEFBEAAAA|2000-10-27||Young items go here later japanese years. Forms must look model, experiment|3.90|1.18|6011006|amalgbrand #6|4|semi-precious|6|Jewelry|125|antiableought|N/A|8575227purple3664272|peach|Carton|Unknown|65|antiableationcallyought| +16726|AAAAAAAAGFBEAAAA|1997-10-27|1999-10-27|Specifically honest pp. would ensure wide for a miles. Different families put then western, certain children. Only exciting commitments say f|0.51|0.26|9014011|edu packunivamalg #11|14|sports|9|Books|746|callyeseation|N/A|73673steel9132034318|steel|Oz|Unknown|17|callyableationcallyought| +16727|AAAAAAAAGFBEAAAA|1999-10-28|2001-10-26|Specifically honest pp. would ensure wide for a miles. Different families put then western, certain children. Only exciting commitments say f|90.99|40.03|8006008|corpnameless #8|14|football|8|Sports|621|oughtablecally|N/A|64552653462papaya118|thistle|N/A|Unknown|33|ationableationcallyought| +16728|AAAAAAAAGFBEAAAA|2001-10-27||Specifically honest pp. would ensure wide for a miles. Different families put then western, certain children. Only exciting commitments say f|3.32|1.26|8011009|amalgmaxi #9|14|archery|8|Sports|947|ationesen st|N/A|64552653462papaya118|tan|Gram|Unknown|75|eingableationcallyought| +16729|AAAAAAAAJFBEAAAA|1997-10-27||Old, digital systems must go willing systems. Slightly small relationships produce of course;|34.53|10.35|3003002|exportiexporti #2|3|toddlers|3|Children|364|esecallypri|small|4009425723cyan113510|yellow|Ton|Unknown|84|n stableationcallyought| +16730|AAAAAAAAKFBEAAAA|1997-10-27|2000-10-26|Usually present societies should not hear regularly on a characteristics. Qualifications can |2.47|1.87|9015003|scholarunivamalg #3|15|fiction|9|Books|778|eingationation|N/A|olive292226388580383|medium|Bunch|Unknown|31|barpriationcallyought| +16731|AAAAAAAAKFBEAAAA|2000-10-27||Fees could not justify absolutely in the feet. Effective, australian waters ought to rebuild for the workers. Able symptoms will underestimate new wo|0.72|0.41|7016002|corpnameless #2|15|furniture|7|Home|778|eingationation|N/A|olive292226388580383|seashell|Ounce|Unknown|60|oughtpriationcallyought| +16732|AAAAAAAAMFBEAAAA|1997-10-27|1999-10-27|Hopes should not remember more consistent colours. Really new techniques could not consider then forms|5.58|2.56|9006005|corpmaxi #5|6|parenting|9|Books|235|antipriable|N/A|57papaya400214021756|seashell|Bunch|Unknown|19|ablepriationcallyought| +16733|AAAAAAAAMFBEAAAA|1999-10-28|2001-10-26|Hopes should not remember more consistent colours. Really new techniques could not consider then forms|8.61|2.56|9006005|corpunivamalg #6|6|mystery|9|Books|235|antipriable|N/A|57papaya400214021756|peru|Pound|Unknown|55|pripriationcallyought| +16734|AAAAAAAAMFBEAAAA|2001-10-27||Besides likely fields identify also; children shall want also. Hands used to work again full-time lines; significantly random men rely likely, financial contracts. S|2.13|2.56|5001001|amalgscholar #1|1|rock|5|Music|515|antioughtanti|N/A|773072876wheat123894|khaki|Each|Unknown|79|esepriationcallyought| +16735|AAAAAAAAPFBEAAAA|1997-10-27||Armies could go systems; rivers look so careful customs.|8.54|3.33|5004002|edu packscholar #2|4|classical|5|Music|911|oughtoughtn st|N/A|600626steel953423316|salmon|Unknown|Unknown|32|antipriationcallyought| +16736|AAAAAAAAAGBEAAAA|1997-10-27|2000-10-26|Sad, following examples shall not mak|3.42|1.74|2001001|amalgimporto #1|1|accessories|2|Men|279|n stationable|medium|1038339873wheat74650|frosted|N/A|Unknown|18|callypriationcallyought| +16737|AAAAAAAAAGBEAAAA|2000-10-27||Sad, following examples shall not mak|3.33|1.74|8002008|importonameless #8|2|baseball|8|Sports|279|n stationable|N/A|955178993orchid96650|lime|Cup|Unknown|86|ationpriationcallyought| +16738|AAAAAAAACGBEAAAA|1997-10-27|1999-10-27|Right weeks might rain further satisfactorily valuable hospitals. Yellow years could create so large, right changes. Rows must spend only.|0.97|0.34|8014009|edu packmaxi #9|14|tennis|8|Sports|175|antiationought|N/A|201brown533433008859|steel|Carton|Unknown|4|eingpriationcallyought| +16739|AAAAAAAACGBEAAAA|1999-10-28|2001-10-26|Right weeks might rain further satisfactorily valuable hospitals. Yellow years could create so large, right changes. Rows must spend only.|6.50|4.35|8014009|importoamalg #2|14|fragrances|1|Women|175|antiationought|extra large|4magenta883733045036|red|Tsp|Unknown|64|n stpriationcallyought| +16740|AAAAAAAACGBEAAAA|2001-10-27||Final engineers define in order western groups. Lucky shots c|8.58|7.63|8014009|importoamalg #1|2|fragrances|1|Women|175|antiationought|small|8seashell33106463634|turquoise|Dozen|Unknown|30|bareseationcallyought| +16741|AAAAAAAAFGBEAAAA|1997-10-27||Centres may spend in the problems. Important journals may give suddenly. Years must not wash now success|2.03|1.19|2002002|importoimporto #2|2|shirts|2|Men|321|oughtablepri|small|78yellow371518409745|ivory|Bundle|Unknown|100|oughteseationcallyought| +16742|AAAAAAAAGGBEAAAA|1997-10-27|2000-10-26|Only extra rates will know highly available railways. More tha|7.92|6.01|8002003|importonameless #3|2|baseball|8|Sports|131|oughtpriought|N/A|0061581762snow166920|sky|N/A|Unknown|16|ableeseationcallyought| +16743|AAAAAAAAGGBEAAAA|2000-10-27||Only extra rates will know highly available railways. More tha|4.47|6.01|8002003|amalgbrand #4|2|bathroom|7|Home|296|callyn stable|N/A|0061581762snow166920|orange|Dozen|Unknown|37|prieseationcallyought| +16744|AAAAAAAAIGBEAAAA|1997-10-27|1999-10-27|Bones pra|8.32|3.91|5001001|amalgscholar #1|1|rock|5|Music|36|callypri|N/A|536977904grey8582448|slate|Box|Unknown|58|eseeseationcallyought| +16745|AAAAAAAAIGBEAAAA|1999-10-28|2001-10-26|Costly rooms feel so closely new authorities. Houses l|8.46|3.91|5001001|brandunivamalg #1|1|personal|10|Electronics|36|callypri|N/A|480snow8915683930917|deep|Oz|Unknown|15|antieseationcallyought| +16746|AAAAAAAAIGBEAAAA|2001-10-27||Heavy|9.05|3.91|5001001|importoamalg #1|1|fragrances|1|Women|626|callyablecally|large|886345rosy1265891297|peru|Pallet|Unknown|3|callyeseationcallyought| +16747|AAAAAAAALGBEAAAA|1997-10-27||Inland memories c|9.31|5.21|7002008|importobrand #8|2|bedding|7|Home|538|eingprianti|N/A|46medium900771458477|lime|Bundle|Unknown|83|ationeseationcallyought| +16748|AAAAAAAAMGBEAAAA|1997-10-27|2000-10-26|Months can understand social responses. Other, local questions stand now for the girls. Ordinary, financial years must not face democratic records. Sad,|8.87|5.58|5001001|amalgscholar #1|1|rock|5|Music|516|callyoughtanti|N/A|071rosy6251665670892|salmon|Case|Unknown|5|eingeseationcallyought| +16749|AAAAAAAAMGBEAAAA|2000-10-27||Then english values will erect important, long amounts. More minimal areas will know even always clear schemes.|8.31|5.58|5001001|exportiamalg #2|1|maternity|1|Women|516|callyoughtanti|medium|071rosy6251665670892|chartreuse|N/A|Unknown|10|n steseationcallyought| +16750|AAAAAAAAOGBEAAAA|1997-10-27|1999-10-27|Natural, long troops satisfy only top, political houses; more happy relations will hang really. Used purposes should not sit successfully fixed, possible sciences. Large scenes|1.85|0.98|4001001|amalgedu pack #1|1|womens|4|Shoes|198|eingn stought|small|01784salmon210748627|puff|Lb|Unknown|59|barantiationcallyought| +16751|AAAAAAAAOGBEAAAA|1999-10-28|2001-10-26|Around criminal schools want late also important effects; new, technical youngsters put only now great firms; scenes must not |99.10|70.36|10016007|corpamalgamalg #7|16|wireless|10|Electronics|372|eingn stought|N/A|01784salmon210748627|tan|Gross|Unknown|59|oughtantiationcallyought| +16752|AAAAAAAAOGBEAAAA|2001-10-27||Around criminal schools want late also important effects; new, technical youngsters put only now great firms; scenes must not |2.67|70.36|3001001|amalgexporti #1|16|newborn|3|Children|372|eingn stought|small|0401695958lavender43|puff|Each|Unknown|92|ableantiationcallyought| +16753|AAAAAAAABHBEAAAA|1997-10-27||British pubs should not get well heavy, good studies. Environmental examples cause as intensive men. Best long programmes must occupy now functional moving years. High, dear women gain very |5.01|4.40|8004004|edu packnameless #4|4|camping|8|Sports|498|eingn stese|N/A|76816840563peach9273|yellow|Unknown|Unknown|42|priantiationcallyought| +16754|AAAAAAAACHBEAAAA|1997-10-27|2000-10-26|Remote, unusual developments used to infer here right sheer sites. Dogs used to benefit therefore only new un|2.91|1.33|10003010|exportiunivamalg #10|3|dvd/vcr players|10|Electronics|19|n stought|N/A|24seashell7754707661|yellow|Each|Unknown|21|eseantiationcallyought| +16755|AAAAAAAACHBEAAAA|2000-10-27||Injuries may get even super schools. Full, great organi|9.30|1.33|10003010|importoedu pack #2|3|mens|4|Shoes|19|n stought|large|24seashell7754707661|sky|Box|Unknown|73|antiantiationcallyought| +16756|AAAAAAAAEHBEAAAA|1997-10-27|1999-10-27|Competent, continuous boots acquire from a cuts. Australian, acute pages can offer consumers. Physical gods discover also. Other |5.09|1.78|2002001|importoimporto #1|2|shirts|2|Men|614|eseoughtcally|large|4609drab963055926705|olive|Unknown|Unknown|14|callyantiationcallyought| +16757|AAAAAAAAEHBEAAAA|1999-10-28|2001-10-26|Competent, continuous boots acquire from a cuts. Australian, acute pages can offer consumers. Physical gods discover also. Other |94.29|56.57|10005013|scholarunivamalg #13|5|karoke|10|Electronics|614|eseoughtcally|N/A|4609drab963055926705|navy|Gram|Unknown|54|ationantiationcallyought| +16758|AAAAAAAAEHBEAAAA|2001-10-27||Competent, continuous boots acquire from a cuts. Australian, acute pages can offer consumers. Physical gods discover also. Other |0.17|56.57|6010007|univbrand #7|5|jewelry boxes|6|Jewelry|614|eseoughtcally|N/A|3smoke38924693362719|saddle|Box|Unknown|33|eingantiationcallyought| +16759|AAAAAAAAHHBEAAAA|1997-10-27||Young symptoms shall not work now there odd eyes. Slow jews stand again full bodies. Different inst|0.69|0.57|3002002|importoexporti #2|2|infants|3|Children|644|eseesecally|petite|57232965018red629870|pale|Gross|Unknown|12|n stantiationcallyought| +16760|AAAAAAAAIHBEAAAA|1997-10-27|2000-10-26|Appropriate limits close elsewhere national manufacturers. Forms could not identify more over other effects. White processes must ease less; irish methods calm at once. V|50.68|38.51|3001001|amalgexporti #1|1|newborn|3|Children|255|antiantiable|extra large|0376180340779puff903|puff|Unknown|Unknown|37|barcallyationcallyought| +16761|AAAAAAAAIHBEAAAA|2000-10-27||Appropriate limits close elsewhere national manufacturers. Forms could not identify more over other effects. White processes must ease less; irish methods calm at once. V|3.25|2.76|7014002|edu packnameless #2|14|glassware|7|Home|255|antiantiable|N/A|4146185plum519979103|puff|Unknown|Unknown|4|oughtcallyationcallyought| +16762|AAAAAAAAKHBEAAAA|1997-10-27|1999-10-27|Relatively original tories might not sing now. Lights could not eliminate only almost visual origins; additional, strong leaders keep presumably numerous, fresh factors. New walls used to k|1.32|0.85|3004001|edu packexporti #1|4|school-uniforms|3|Children|246|callyeseable|medium|271911sienna14353474|violet|Lb|Unknown|27|ablecallyationcallyought| +16763|AAAAAAAAKHBEAAAA|1999-10-28|2001-10-26|Voluntarily big months should not achieve matters. New lovers could cut ultimately needs. Expensive, private prices catch wo|1.02|0.81|1003002|exportiamalg #2|4|maternity|1|Women|246|callyeseable|small|83599orchid193858637|peru|Box|Unknown|1|pricallyationcallyought| +16764|AAAAAAAAKHBEAAAA|2001-10-27||Voluntarily big months should not achieve matters. New lovers could cut ultimately needs. Expensive, private prices catch wo|4.39|2.19|1003002|brandmaxi #1|4|reference|9|Books|246|callyeseable|N/A|83599orchid193858637|rosy|Ounce|Unknown|74|esecallyationcallyought| +16765|AAAAAAAANHBEAAAA|1997-10-27||Clean pictures would become through a clients. Legs sell up to a effects. Powerful, german areas may come in general at least little changes. Too medical years may suck probably soon pub|6.36|2.16|9016002|corpunivamalg #2|16|mystery|9|Books|282|ableeingable|N/A|205397749snow4544907|pink|Ton|Unknown|21|anticallyationcallyought| +16766|AAAAAAAAOHBEAAAA|1997-10-27|2000-10-26|Children write true, old seasons. Stupid, nationa|5.97|1.79|7007007|brandbrand #7|7|decor|7|Home|366|callycallypri|N/A|1sky0669579072518587|slate|Pound|Unknown|11|callycallyationcallyought| +16767|AAAAAAAAOHBEAAAA|2000-10-27||Either contemporary concessions ought to get keen interests. Teams ought to come still real contents; e|2.40|0.96|7009008|maxibrand #8|7|mattresses|7|Home|366|callycallypri|N/A|1sky0669579072518587|almond|Ton|Unknown|48|ationcallyationcallyought| +16768|AAAAAAAAAIBEAAAA|1997-10-27|1999-10-27|Even young stores extend largely. Wro|3.51|1.82|5001001|amalgscholar #1|1|rock|5|Music|47|ationese|N/A|0723930smoke56197144|saddle|Oz|Unknown|81|eingcallyationcallyought| +16769|AAAAAAAAAIBEAAAA|1999-10-28|2001-10-26|Even young stores extend largely. Wro|0.67|0.50|5001001|importoimporto #2|2|shirts|2|Men|79|n station|medium|468634370197steel186|midnight|Bunch|Unknown|28|n stcallyationcallyought| +16770|AAAAAAAAAIBEAAAA|2001-10-27||Afraid patients should know in a objects. Thoughts get carefully general minds; streets rent fast, social matters. Offici|6.69|0.50|1002001|importoamalg #1|2|fragrances|1|Women|79|n station|economy|468634370197steel186|papaya|Lb|Unknown|82|barationationcallyought| +16771|AAAAAAAADIBEAAAA|1997-10-27||Alone relevant nights pretend so complete years. Currently new unions make horizontally bills. Most political troops could give most possible, australian elements; flowers shall recall most pop|3.55|1.24|3002002|importoexporti #2|2|infants|3|Children|17|ationought|petite|purple24808786966784|lime|Case|Unknown|14|oughtationationcallyought| +16772|AAAAAAAAEIBEAAAA|1997-10-27|2000-10-26|Rights shall meet drugs. Natural places could see here towards a months. Fiercely social farmers sh|29.28|23.42|10008011|namelessunivamalg #11|8|scanners|10|Electronics|40|barese|N/A|568046851342saddle61|grey|Lb|Unknown|59|ableationationcallyought| +16773|AAAAAAAAEIBEAAAA|2000-10-27||Rights shall meet drugs. Natural places could see here towards a months. Fiercely social farmers sh|3.31|1.22|10008012|namelessunivamalg #12|8|scanners|10|Electronics|513|barese|N/A|tan43559422725755016|puff|Carton|Unknown|29|priationationcallyought| +16774|AAAAAAAAGIBEAAAA|1997-10-27|1999-10-27|Grateful, ru|8.49|6.36|8011005|amalgmaxi #5|11|archery|8|Sports|611|oughtoughtcally|N/A|83002467354561hot775|pale|Lb|Unknown|56|eseationationcallyought| +16775|AAAAAAAAGIBEAAAA|1999-10-28|2001-10-26|Different women shall force only critical, russian years; only bright years may not pin. Required schemes call a bit |6.74|5.25|8011005|importonameless #6|11|baseball|8|Sports|611|oughtoughtcally|N/A|83002467354561hot775|purple|Pallet|Unknown|25|antiationationcallyought| +16776|AAAAAAAAGIBEAAAA|2001-10-27||Marginal men could link in the chemicals; rather ready claims might keep under way at the unions. Commonly useful claims must not marr|1.24|0.71|8011005|amalgnameless #7|11|accent|7|Home|611|oughtoughtcally|N/A|83002467354561hot775|sienna|Gross|Unknown|49|callyationationcallyought| +16777|AAAAAAAAJIBEAAAA|1997-10-27||Constitutional, marvellous schools will go by the women; large workers must return just. Much personal times make at last. Stories know mixed cards. Just environmental varia|4.93|3.25|5002002|importoscholar #2|2|country|5|Music|102|ablebarought|N/A|linen131211519028072|violet|Oz|Unknown|52|ationationationcallyought| +16778|AAAAAAAAKIBEAAAA|1997-10-27|2000-10-26|Parts see little notes; almost dead spots|1.38|0.73|7004009|edu packbrand #9|4|curtains/drapes|7|Home|10|barought|N/A|0117809203seashell22|snow|Dram|Unknown|5|eingationationcallyought| +16779|AAAAAAAAKIBEAAAA|2000-10-27||Engineers would not take at all in order consistent groups; recen|0.66|0.34|7004009|importoamalg #2|2|fragrances|1|Women|277|barought|large|0117809203seashell22|spring|N/A|Unknown|83|n stationationcallyought| +16780|AAAAAAAAMIBEAAAA|1997-10-27|1999-10-27|Meanwhile big researchers solve american,|42.72|32.46|4002001|importoedu pack #1|2|mens|4|Shoes|20|barable|petite|116439587light906905|red|Ton|Unknown|5|bareingationcallyought| +16781|AAAAAAAAMIBEAAAA|1999-10-28|2001-10-26|Worthy presents ask just slim vessels; inadequate youngste|2.88|1.61|4002001|amalgunivamalg #6|11|cooking|9|Books|20|barable|N/A|236turquoise81428837|peach|Cup|Unknown|14|oughteingationcallyought| +16782|AAAAAAAAMIBEAAAA|2001-10-27||Worthy presents ask just slim vessels; inadequate youngste|0.71|1.61|9001007|amalgmaxi #7|11|arts|9|Books|10|barable|N/A|608159seashell712438|red|Bundle|Unknown|2|ableeingationcallyought| +16783|AAAAAAAAPIBEAAAA|1997-10-27||New buildings should visit forcefully certainly fine aspects. Shows must not take totally lights. Full teachers say still. Today local units shall know exactly by a services. Patient|8.39|7.04|7007004|brandbrand #4|7|decor|7|Home|873|priationeing|N/A|99873410hot937878697|lace|Tbl|Unknown|1|prieingationcallyought| +16784|AAAAAAAAAJBEAAAA|1997-10-27|2000-10-26|Relevant, distinctive years speak. Fac|0.42|0.21|8006005|corpnameless #5|6|football|8|Sports|382|ableeingpri|N/A|95691155green5071411|spring|Dram|Unknown|24|eseeingationcallyought| +16785|AAAAAAAAAJBEAAAA|2000-10-27||Apparently central children should drive boats. Fed|8.27|0.21|8006005|scholarnameless #6|6|tables|7|Home|382|ableeingpri|N/A|73763835midnight6496|gainsboro|Bundle|Unknown|60|antieingationcallyought| +16786|AAAAAAAACJBEAAAA|1997-10-27|1999-10-27|Scientists stay small patients; easy, thin authorities kill; cases must settle other stocks; employees ought to acquire together men. For instance obvious|4.46|3.96|7009007|maxibrand #7|9|mattresses|7|Home|91|oughtn st|N/A|46maroon894989049177|seashell|Cup|Unknown|4|callyeingationcallyought| +16787|AAAAAAAACJBEAAAA|1999-10-28|2001-10-26|Scientists stay small patients; easy, thin authorities kill; cases must settle other stocks; employees ought to acquire together men. For instance obvious|2.27|2.02|7009008|maxibrand #8|9|mattresses|7|Home|13|priought|N/A|193394926298801lace9|slate|Gram|Unknown|4|ationeingationcallyought| +16788|AAAAAAAACJBEAAAA|2001-10-27||Scientists stay small patients; easy, thin authorities kill; cases must settle other stocks; employees ought to acquire together men. For instance obvious|9.35|3.74|7009008|importoexporti #1|2|infants|3|Children|13|priought|petite|16896482red463910085|pink|Box|Unknown|2|eingeingationcallyought| +16789|AAAAAAAAFJBEAAAA|1997-10-27||Independent, philosophical examples attend nights; primary times would recapture low. Principles used to stop wonderful inches; small scots should appeas|83.94|60.43|10005011|scholarunivamalg #11|5|karoke|10|Electronics|134|esepriought|N/A|5407557472sandy93709|purple|Tbl|Unknown|49|n steingationcallyought| +16790|AAAAAAAAGJBEAAAA|1997-10-27|2000-10-26|Elaborate periods bother also considerable republics. Streets cannot serve freshly |2.34|1.17|7016001|corpnameless #1|16|furniture|7|Home|83|prieing|N/A|839912520678wheat257|salmon|Pallet|Unknown|78|barn stationcallyought| +16791|AAAAAAAAGJBEAAAA|2000-10-27||Full, new holders follow keen re|0.50|0.22|7016001|amalgunivamalg #13|1|cameras|10|Electronics|83|prieing|N/A|21saddle186711298087|smoke|Ton|Unknown|27|oughtn stationcallyought| +16792|AAAAAAAAIJBEAAAA|1997-10-27|1999-10-27|National, early|9.07|7.98|2004001|edu packimporto #1|4|sports-apparel|2|Men|321|oughtablepri|petite|slate110824771932110|pink|Tsp|Unknown|5|ablen stationcallyought| +16793|AAAAAAAAIJBEAAAA|1999-10-28|2001-10-26|Common, heavy forms agree maybe bare weeks. Dimensions shall accelerate much official, local actions. Servi|2.18|0.74|2004001|univnameless #4|4|flatware|7|Home|321|oughtablepri|N/A|slate110824771932110|chiffon|Ounce|Unknown|47|prin stationcallyought| +16794|AAAAAAAAIJBEAAAA|2001-10-27||True peaceful arrangements used to drink forces. Here primitive bacteria must prop there at the words. Simple objectives slow again above vast things;|52.74|23.20|2004001|amalgunivamalg #7|11|cooking|9|Books|115|antioughtought|N/A|slate110824771932110|almond|Pallet|Unknown|72|esen stationcallyought| +16795|AAAAAAAALJBEAAAA|1997-10-27||Efforts might come ever. Even agricultural operations live plus a members. Arts may ring too carefully numerous windows. In particular new ch|1.49|0.95|1004002|edu packamalg #2|4|swimwear|1|Women|438|eingpriese|medium|1295424059098red7770|thistle|Ton|Unknown|20|antin stationcallyought| +16796|AAAAAAAAMJBEAAAA|1997-10-27|2000-10-26|Now wide men might speculate now probably scientific failures. Important, new fe|3.98|2.46|1001001|amalgamalg #1|1|dresses|1|Women|773|priationation|petite|146534674tomato02337|white|Unknown|Unknown|25|callyn stationcallyought| +16797|AAAAAAAAMJBEAAAA|2000-10-27||Now wide men might speculate now probably scientific failures. Important, new fe|9.67|6.96|10010002|univamalgamalg #2|1|memory|10|Electronics|773|priationation|N/A|146534674tomato02337|peru|N/A|Unknown|88|ationn stationcallyought| +16798|AAAAAAAAOJBEAAAA|1997-10-27|1999-10-27|Social, joint relationships follow entirely out of a periods. Rights will make also even narrow tasks. Obvious, still quarters give generously. Co|2.12|1.16|6011007|amalgbrand #7|11|semi-precious|6|Jewelry|66|callycally|N/A|4smoke92309366808791|thistle|Dram|Unknown|24|eingn stationcallyought| +16799|AAAAAAAAOJBEAAAA|1999-10-28|2001-10-26|Reasons credit best. Parts become free, conservative institutions. Different, prime goods could come that is; large, determined eyes conduct together cars. Even clever trusts simplify actually wo|12.87|8.62|3004002|edu packexporti #2|11|school-uniforms|3|Children|251|oughtantiable|small|69138118699pale05408|slate|Dozen|Unknown|2|n stn stationcallyought| +16800|AAAAAAAAOJBEAAAA|2001-10-27||Reasons credit best. Parts become free, conservative institutions. Different, prime goods could come that is; large, determined eyes conduct together cars. Even clever trusts simplify actually wo|5.12|8.62|3004002|importoexporti #1|11|infants|3|Children|370|barationpri|extra large|69138118699pale05408|white|Pallet|Unknown|8|barbareingcallyought| +16801|AAAAAAAABKBEAAAA|1997-10-27||Shared, big controls used to seem monetary, environmental sides. Lively, military wor|3.71|2.00|10008006|namelessunivamalg #6|8|scanners|10|Electronics|154|eseantiought|N/A|839803517papaya63189|thistle|Carton|Unknown|32|oughtbareingcallyought| +16802|AAAAAAAACKBEAAAA|1997-10-27|2000-10-26|Goals remove; other, historical years will sign firm relationships. Consequences persuade total, pale layers. Precise, significant tables would not avoid there values; gai|5.54|1.93|5003001|exportischolar #1|3|pop|5|Music|113|prioughtought|N/A|78524880699puff06261|thistle|Gross|Unknown|45|ablebareingcallyought| +16803|AAAAAAAACKBEAAAA|2000-10-27||Alone, low places issue only involved, coherent trees. Clients would receive; ce|51.00|38.25|3004002|edu packexporti #2|4|school-uniforms|3|Children|157|ationantiought|N/A|1427508186khaki20167|magenta|Each|Unknown|34|pribareingcallyought| +16804|AAAAAAAAEKBEAAAA|1997-10-27|1999-10-27|Much old books shall make strong years; events shall not appear. Hard rights detect interests. Ways organise clearly wide, symbolic resources; as well outside sales speak never chemical, environ|0.79|0.28|1002001|importoamalg #1|2|fragrances|1|Women|58|einganti|N/A|3130spring1964898680|pale|Unknown|Unknown|29|esebareingcallyought| +16805|AAAAAAAAEKBEAAAA|1999-10-28|2001-10-26|Gradually distinctive requirements ought to make new, religious resources. So other parties bring about a doors; significantly young balls perform often|4.17|0.28|3001002|amalgexporti #2|2|newborn|3|Children|128|eingableought|petite|74501818990595smoke1|puff|N/A|Unknown|45|antibareingcallyought| +16806|AAAAAAAAEKBEAAAA|2001-10-27||General improvements go late journalists. Estimated, equivalent rates may not take new, strong years. Overall countries|95.59|68.82|4002001|importoedu pack #1|2|mens|4|Shoes|637|eingableought|petite|74501818990595smoke1|rose|Case|Unknown|9|callybareingcallyought| +16807|AAAAAAAAHKBEAAAA|1997-10-27||Significant, fa|4.86|3.25|7001008|amalgbrand #8|1|bathroom|7|Home|17|ationought|N/A|7059682thistle947559|turquoise|Pound|Unknown|5|ationbareingcallyought| +16808|AAAAAAAAIKBEAAAA|1997-10-27|2000-10-26|Thick|8.85|4.60|7009009|maxibrand #9|9|mattresses|7|Home|593|prin stanti|N/A|987217727turquoise58|white|Unknown|Unknown|74|eingbareingcallyought| +16809|AAAAAAAAIKBEAAAA|2000-10-27||Thick|6.71|2.01|1001002|amalgamalg #2|1|dresses|1|Women|593|prin stanti|medium|987217727turquoise58|sky|Cup|Unknown|7|n stbareingcallyought| +16810|AAAAAAAAKKBEAAAA|1997-10-27|1999-10-27|Professional, essential parents ratify financial, able officers. Most small responsibilities might see. Civil, good hours can pronounce then shares. Exciting faces must not allow|0.96|0.65|6003003|exporticorp #3|3|gold|6|Jewelry|12|ableought|N/A|122713rosy7872384001|mint|Bunch|Unknown|26|baroughteingcallyought| +16811|AAAAAAAAKKBEAAAA|1999-10-28|2001-10-26|Professional, essential parents ratify financial, able officers. Most small responsibilities might see. Civil, good hours can pronounce then shares. Exciting faces must not allow|2.72|0.65|4004002|edu packedu pack #2|3|athletic|4|Shoes|12|ableought|petite|7112rose699072982601|spring|Bundle|Unknown|11|oughtoughteingcallyought| +16812|AAAAAAAAKKBEAAAA|2001-10-27||Professional, essential parents ratify financial, able officers. Most small responsibilities might see. Civil, good hours can pronounce then shares. Exciting faces must not allow|4.45|0.65|10013017|exportiamalgamalg #17|3|stereo|10|Electronics|12|ableought|N/A|717150linen855630367|lace|Dram|Unknown|17|ableoughteingcallyought| +16813|AAAAAAAANKBEAAAA|1997-10-27||Dry, friendly situations ask thus grey floors. Letters must discuss steep chapters. Members act ago on a feet. Standards exploit sounds. Arguments shall come|4.77|3.52|7006004|corpbrand #4|6|rugs|7|Home|70|baration|N/A|602252lawn8599431475|blanched|Gross|Unknown|9|prioughteingcallyought| +16814|AAAAAAAAOKBEAAAA|1997-10-27|2000-10-26|Over important habits may safeguard therefore true av|3.28|2.52|6002007|importocorp #7|2|diamonds|6|Jewelry|136|callypriought|N/A|8168382257slate40584|plum|Tsp|Unknown|39|eseoughteingcallyought| +16815|AAAAAAAAOKBEAAAA|2000-10-27||Tiny years bring there. Proper, e|9.86|5.42|7001006|amalgbrand #6|1|bathroom|7|Home|255|callypriought|N/A|6powder9163510860365|green|Bunch|Unknown|28|antioughteingcallyought| +16816|AAAAAAAAALBEAAAA|1997-10-27|1999-10-27|Others would desire a little various passengers; able results control most etc willi|3.62|3.04|2004001|edu packimporto #1|4|sports-apparel|2|Men|84|eseeing|medium|912turquoise87958252|rose|Gram|Unknown|94|callyoughteingcallyought| +16817|AAAAAAAAALBEAAAA|1999-10-28|2001-10-26|Sections work always. Gradually public models can take however brief useful clients; frequent, major hands apply scottish, wide |1.12|0.40|3002002|importoexporti #2|2|infants|3|Children|84|eseeing|large|912turquoise87958252|tan|Oz|Unknown|24|ationoughteingcallyought| +16818|AAAAAAAAALBEAAAA|2001-10-27||Sections work always. Gradually public models can take however brief useful clients; frequent, major hands apply scottish, wide |9.61|0.40|1004001|edu packamalg #1|4|swimwear|1|Women|84|eseeing|large|9618rosy480648518127|salmon|Each|Unknown|36|eingoughteingcallyought| +16819|AAAAAAAADLBEAAAA|1997-10-27||Traditional cars shall not think often states. Necessary issues apply increased contracts. About royal lines cannot take to the papers. Nevertheless crucial|54.92|32.95|5001002|amalgscholar #2|1|rock|5|Music|300|barbarpri|N/A|78822sienna125201504|salmon|Ton|Unknown|30|n stoughteingcallyought| +16820|AAAAAAAAELBEAAAA|1997-10-27|2000-10-26|Materials may know enough catholic, legitimate christians. Large, current things|7.30|6.57|4004001|edu packedu pack #1|4|athletic|4|Shoes|443|prieseese|large|saddle10261499937084|saddle|Bundle|Unknown|4|barableeingcallyought| +16821|AAAAAAAAELBEAAAA|2000-10-27||Materials may know enough catholic, legitimate christians. Large, current things|2.69|2.39|1002002|importoamalg #2|2|fragrances|1|Women|443|prieseese|small|turquoise57800498136|orchid|Tbl|Unknown|7|oughtableeingcallyought| +16822|AAAAAAAAGLBEAAAA|1997-10-27|1999-10-27|Differences make etc about a pp.. Inevitable, lively bodies borrow more than old gir|7.00|2.31|4002001|importoedu pack #1|2|mens|4|Shoes|603|pribarcally|medium|179542874542royal779|rose|Ounce|Unknown|58|ableableeingcallyought| +16823|AAAAAAAAGLBEAAAA|1999-10-28|2001-10-26|Differences make etc about a pp.. Inevitable, lively bodies borrow more than old gir|1.63|1.46|3001002|amalgexporti #2|2|newborn|3|Children|58|einganti|economy|42201papaya109861947|goldenrod|Gram|Unknown|80|priableeingcallyought| +16824|AAAAAAAAGLBEAAAA|2001-10-27||Also irish facts see raw pieces; changes describe almost unlikely, mature friends. Guests fly at the regions. Both pure reports must not wear surprisingly |40.44|1.46|3001002|edu packnameless #5|2|glassware|7|Home|58|einganti|N/A|42201papaya109861947|puff|Pallet|Unknown|2|eseableeingcallyought| +16825|AAAAAAAAJLBEAAAA|1997-10-27||Important, old communities declare more successful, private members. In|1.37|1.06|8001006|amalgnameless #6|1|athletic shoes|8|Sports|451|oughtantiese|N/A|771266256powder55691|black|Case|Unknown|63|antiableeingcallyought| +16826|AAAAAAAAKLBEAAAA|1997-10-27|2000-10-26|Steps steal vast, financial amounts. Others should feature more from the rates. Far industrial ages used to contain. Asian colleges |4.03|2.90|3004001|edu packexporti #1|4|school-uniforms|3|Children|190|barn stought|N/A|218009595384snow5878|sandy|Ton|Unknown|17|callyableeingcallyought| +16827|AAAAAAAAKLBEAAAA|2000-10-27||Again urban dimensions can extract at last good dangerous police. Sufficient, natural vessels may not read so actual months. Long-term feet would not specify necessari|4.15|1.61|10011015|amalgamalgamalg #15|11|disk drives|10|Electronics|190|barn stought|N/A|626670890691rose1720|peach|Case|Unknown|32|ationableeingcallyought| +16828|AAAAAAAAMLBEAAAA|1997-10-27|1999-10-27|Further dirty police cannot think universally committees. Genuine soldiers might not cancel urgently additional, vast participants; only hot years take usually sums; materials cannot shake|2.32|0.74|7014009|edu packnameless #9|14|glassware|7|Home|100|barbarought|N/A|98205smoke5095541496|snow|Each|Unknown|21|eingableeingcallyought| +16829|AAAAAAAAMLBEAAAA|1999-10-28|2001-10-26|Further dirty police cannot think universally committees. Genuine soldiers might not cancel urgently additional, vast participants; only hot years take usually sums; materials cannot shake|3.87|1.50|1003002|exportiamalg #2|14|maternity|1|Women|100|barbarought|extra large|8648976070829linen13|wheat|Box|Unknown|70|n stableeingcallyought| +16830|AAAAAAAAMLBEAAAA|2001-10-27||Open, numerous criteria may not get barely valid commitments. Financial advances name changes; alone different organisations should pinpoint indeed inside a texts. Sometime les|3.70|1.50|5002001|importoscholar #1|2|country|5|Music|100|barbarought|N/A|2503324897512steel75|thistle|Oz|Unknown|45|barprieingcallyought| +16831|AAAAAAAAPLBEAAAA|1997-10-27||Procedures should see much sides. Brothers get cases. Re|1.09|0.68|3004002|edu packexporti #2|4|school-uniforms|3|Children|620|barablecally|petite|01414328deep56037132|red|Gross|Unknown|20|oughtprieingcallyought| +16832|AAAAAAAAAMBEAAAA|1997-10-27|2000-10-26|Long seats should not come whole, available students. Possible, blue p|1.48|1.28|8011003|amalgmaxi #3|11|archery|8|Sports|721|oughtableation|N/A|9slate83160940920173|tan|Box|Unknown|54|ableprieingcallyought| +16833|AAAAAAAAAMBEAAAA|2000-10-27||Long seats should not come whole, available students. Possible, blue p|1.83|1.61|8011003|univunivamalg #10|11|travel|9|Books|721|oughtableation|N/A|82834218581salmon564|snow|Ton|Unknown|27|priprieingcallyought| +16834|AAAAAAAACMBEAAAA|1997-10-27|1999-10-27|Only hard years would take just. Only proud men matter again less interested days; video-taped, unlikely shares bear now into the rivers|1.95|0.68|7009005|maxibrand #5|9|mattresses|7|Home|654|eseantically|N/A|5292367655tan2908167|gainsboro|Oz|Unknown|20|eseprieingcallyought| +16835|AAAAAAAACMBEAAAA|1999-10-28|2001-10-26|Only hard years would take just. Only proud men matter again less interested days; video-taped, unlikely shares bear now into the rivers|3.98|2.82|8010006|univmaxi #6|9|pools|8|Sports|654|eseantically|N/A|5292367655tan2908167|powder|Carton|Unknown|43|antiprieingcallyought| +16836|AAAAAAAACMBEAAAA|2001-10-27||Only hard years would take just. Only proud men matter again less interested days; video-taped, unlikely shares bear now into the rivers|2.73|2.82|4001001|amalgedu pack #1|1|womens|4|Shoes|654|eseantically|medium|3282658954thistle900|saddle|Dozen|Unknown|39|callyprieingcallyought| +16837|AAAAAAAAFMBEAAAA|1997-10-27||Local, final users must not make below; thus significant deputies find widely by the affairs. Anonymous, british instruments enter almost written, expensive shareholders.|7.88|4.80|8007008|brandnameless #8|7|hockey|8|Sports|193|prin stought|N/A|1394846891forest6924|salmon|Cup|Unknown|39|ationprieingcallyought| +16838|AAAAAAAAGMBEAAAA|1997-10-27|2000-10-26|Then royal observations make even. British, quick employees must save typically british politicians. Left, reasonable words get even types. Never united members show corres|4.01|3.04|2003001|exportiimporto #1|3|pants|2|Men|880|bareingeing|medium|8spring8011189195314|maroon|Oz|Unknown|25|eingprieingcallyought| +16839|AAAAAAAAGMBEAAAA|2000-10-27||Then royal observations make even. British, quick employees must save typically british politicians. Left, reasonable words get even types. Never united members show corres|0.89|3.04|9014004|edu packunivamalg #4|14|sports|9|Books|678|eingationcally|N/A|8spring8011189195314|plum|Pound|Unknown|57|n stprieingcallyought| +16840|AAAAAAAAIMBEAAAA|1997-10-27|1999-10-27|Papers review easily forces; students make regional years. Flames focus american, foreign employees; more far premises see so|6.10|3.35|3003001|exportiexporti #1|3|toddlers|3|Children|428|eingableese|N/A|363709030puff0897604|peru|Bunch|Unknown|16|bareseeingcallyought| +16841|AAAAAAAAIMBEAAAA|1999-10-28|2001-10-26|Papers review easily forces; students make regional years. Flames focus american, foreign employees; more far premises see so|3.69|2.06|3003001|maxibrand #2|3|mattresses|7|Home|428|eingableese|N/A|7navy301991542237537|lavender|Tbl|Unknown|46|oughteseeingcallyought| +16842|AAAAAAAAIMBEAAAA|2001-10-27||Valuable, productive banks become even with the churches. Central, americ|9.40|2.06|9010007|univunivamalg #7|3|travel|9|Books|428|eingableese|N/A|7navy301991542237537|peach|Pallet|Unknown|86|ableeseeingcallyought| +16843|AAAAAAAALMBEAAAA|1997-10-27||Implicit, likely nati|9.93|6.85|5004002|edu packscholar #2|4|classical|5|Music|424|eseableese|N/A|6415517white12448155|wheat|Gross|Unknown|40|prieseeingcallyought| +16844|AAAAAAAAMMBEAAAA|1997-10-27|2000-10-26|Poor waves might encompass slowly about a members. Famous concerns could not provoke always neighbouring, electoral schemes. Events may not investigate d|7.07|6.15|7011005|amalgnameless #5|11|accent|7|Home|340|baresepri|N/A|1357869smoke97729411|steel|Cup|Unknown|91|eseeseeingcallyought| +16845|AAAAAAAAMMBEAAAA|2000-10-27||Temporary, economic components like big words. Never remote patients should murder. Also appropriate approaches work sometimes at a children. Common, firm sources should change even |4.08|3.22|6016006|corpbrand #6|16|consignment|6|Jewelry|340|baresepri|N/A|1357869smoke97729411|rose|Box|Unknown|12|antieseeingcallyought| +16846|AAAAAAAAOMBEAAAA|1997-10-27|1999-10-27|Known, american talks can direct. Outer, apparent tools play still great, ma|1.30|0.42|7009007|maxibrand #7|9|mattresses|7|Home|252|ableantiable|N/A|13259621119549lawn99|pink|Pound|Unknown|54|callyeseeingcallyought| +16847|AAAAAAAAOMBEAAAA|1999-10-28|2001-10-26|Inland books should target at lea|8.77|0.42|2004002|edu packimporto #2|4|sports-apparel|2|Men|229|n stableable|economy|13259621119549lawn99|slate|Unknown|Unknown|70|ationeseeingcallyought| +16848|AAAAAAAAOMBEAAAA|2001-10-27||Words show on a teachers. Equally real officers could want clean from a ideas. European, right costs will not keep eyes. Aut|4.75|1.85|2004002|amalgmaxi #1|4|arts|9|Books|543|n stableable|N/A|13259621119549lawn99|slate|Gross|Unknown|14|eingeseeingcallyought| +16849|AAAAAAAABNBEAAAA|1997-10-27||Years could not give even ago huge levels. Serious ministers can make here from a things; annual, sorry activities can come with a types. Also previous |1.74|0.71|10010003|univamalgamalg #3|10|memory|10|Electronics|226|callyableable|N/A|93903889421saddle041|khaki|Unknown|Unknown|11|n steseeingcallyought| +16850|AAAAAAAACNBEAAAA|1997-10-27|2000-10-26|Users must help particularly new doors. Small parties expect like additional, economic days. Pink, dry days know animals. Friends used to proceed reportedly however imperial investments; bold|0.09|0.03|10011004|amalgamalgamalg #4|11|disk drives|10|Electronics|456|callyantiese|N/A|88magenta57711287877|wheat|Pallet|Unknown|11|barantieingcallyought| +16851|AAAAAAAACNBEAAAA|2000-10-27||Already good days shou|4.79|4.21|2001002|amalgimporto #2|11|accessories|2|Men|137|ationpriought|extra large|445430yellow23172553|purple|Tbl|Unknown|100|oughtantieingcallyought| +16852|AAAAAAAAENBEAAAA|1997-10-27|1999-10-27|Even good friends may give lesser, old-fashioned bu|91.00|33.67|1001001|amalgamalg #1|1|dresses|1|Women|190|barn stought|petite|6631seashell19588218|plum|Gross|Unknown|20|ableantieingcallyought| +16853|AAAAAAAAENBEAAAA|1999-10-28|2001-10-26|Well extensive politicians will say clean doubts. Easy years shall provide then others. Particular, developing skills see far|6.90|33.67|8006004|corpnameless #4|6|football|8|Sports|309|n stbarpri|N/A|4223orange5425559938|ivory|Oz|Unknown|10|priantieingcallyought| +16854|AAAAAAAAENBEAAAA|2001-10-27||Keen pressures leave right available interests. New, local committees pull statistic|4.04|2.62|8006004|exportiedu pack #1|3|kids|4|Shoes|309|n stbarpri|large|8100704mint452426950|green|Pound|Unknown|9|eseantieingcallyought| +16855|AAAAAAAAHNBEAAAA|1997-10-27||Tomorrow able reasons might take grey, major activities. Sensitive, so-called factors must sho|4.12|2.18|7016006|corpnameless #6|16|furniture|7|Home|229|n stableable|N/A|7226939931yellow9911|white|Dram|Unknown|17|antiantieingcallyought| +16856|AAAAAAAAINBEAAAA|1997-10-27|2000-10-26|Small, close companies ought to pick later. Suggestions claim for example there quiet officials. Aspects take slowly today opposite years. Western, f|2.50|1.00|1002001|importoamalg #1|2|fragrances|1|Women|271|oughtationable|extra large|plum9483957432732325|gainsboro|Tsp|Unknown|15|callyantieingcallyought| +16857|AAAAAAAAINBEAAAA|2000-10-27||Small, close companies ought to pick later. Suggestions claim for example there quiet officials. Aspects take slowly today opposite years. Western, f|6.08|1.00|5002002|importoscholar #2|2|country|5|Music|242|oughtationable|N/A|plum9483957432732325|seashell|Cup|Unknown|55|ationantieingcallyought| +16858|AAAAAAAAKNBEAAAA|1997-10-27|1999-10-27|Tall animals swim extra commercial, special politicians; requirements punish; services relate always |45.77|29.75|9005011|scholarmaxi #11|5|history|9|Books|197|ationn stought|N/A|8318202812869961red1|mint|Carton|Unknown|76|eingantieingcallyought| +16859|AAAAAAAAKNBEAAAA|1999-10-28|2001-10-26|Private regulations find with a members; fortunately young activities secure real, possible miles. Organisations may not take only open opportunities. Informal, equal kinds ought to render before|7.75|29.75|9005011|importoscholar #2|5|country|5|Music|197|ationn stought|N/A|8318202812869961red1|rose|Bunch|Unknown|36|n stantieingcallyought| +16860|AAAAAAAAKNBEAAAA|2001-10-27||Private regulations find with a members; fortunately young activities secure real, possible miles. Organisations may not take only open opportunities. Informal, equal kinds ought to render before|3.03|1.72|8009001|maxinameless #1|5|optics|8|Sports|197|ationn stought|N/A|2grey310865128128366|peru|Pallet|Unknown|9|barcallyeingcallyought| +16861|AAAAAAAANNBEAAAA|1997-10-27||Possible, friendly goods slow certainly prepared, obviou|0.69|0.57|8014002|edu packmaxi #2|14|tennis|8|Sports|89|n steing|N/A|2rosy716710719867899|honeydew|Tsp|Unknown|48|oughtcallyeingcallyought| +16862|AAAAAAAAONBEAAAA|1997-10-27|2000-10-26|Settlements cover closely different groups. Thin results appreciate present, unacceptable circumstances. Edges show pure, reliable meetings. At present independent journalists can a|2.85|1.65|4002001|importoedu pack #1|2|mens|4|Shoes|170|barationought|small|9195sienna9545733709|snow|Ton|Unknown|6|ablecallyeingcallyought| +16863|AAAAAAAAONBEAAAA|2000-10-27||Individual, new compl|4.77|1.65|4002001|scholarcorp #8|5|earings|6|Jewelry|170|barationought|N/A|05573220rose91529884|sky|Tsp|Unknown|17|pricallyeingcallyought| +16864|AAAAAAAAAOBEAAAA|1997-10-27|1999-10-27|Large, western bodies match already sensitive, overall others. General, willing duties reach assistant parents. Emotional representations would not assure. Alternative, crucial sales may make runnin|4.69|2.01|7012005|importonameless #5|12|paint|7|Home|179|n stationought|N/A|4746041saddle0953841|sky|Cup|Unknown|23|esecallyeingcallyought| +16865|AAAAAAAAAOBEAAAA|1999-10-28|2001-10-26|Able others must not try there able, fine feet. Historical, political earnings accept further at once post-war tears; new patients stop together possible, other officers. Even economi|0.36|0.31|10006002|corpunivamalg #2|6|musical|10|Electronics|464|esecallyese|N/A|238turquoise76638200|snow|Gross|Unknown|35|anticallyeingcallyought| +16866|AAAAAAAAAOBEAAAA|2001-10-27||Able others must not try there able, fine feet. Historical, political earnings accept further at once post-war tears; new patients stop together possible, other officers. Even economi|36.38|0.31|10006002|maxinameless #7|9|optics|8|Sports|264|esecallyese|N/A|1933075seashell68801|thistle|Pound|Unknown|55|callycallyeingcallyought| +16867|AAAAAAAADOBEAAAA|1997-10-27||Expected decisions could establish often rural standards; involved, main men|2.49|2.16|1002002|importoamalg #2|2|fragrances|1|Women|72|ableation|extra large|194267720773thistle8|khaki|Pallet|Unknown|11|ationcallyeingcallyought| +16868|AAAAAAAAEOBEAAAA|1997-10-27|2000-10-26|Then african stars used to blow as clergy. Recently acute sessions could understand in a diseases. Im|6.84|5.40|8005009|scholarnameless #9|5|fitness|8|Sports|260|barcallyable|N/A|355480336029sienna01|grey|Unknown|Unknown|45|eingcallyeingcallyought| +16869|AAAAAAAAEOBEAAAA|2000-10-27||Diplomatic things would discover best great reforms. Signa|82.63|5.40|8005009|amalgexporti #2|1|newborn|3|Children|260|barcallyable|large|355480336029sienna01|puff|Pound|Unknown|18|n stcallyeingcallyought| +16870|AAAAAAAAGOBEAAAA|1997-10-27|1999-10-27|Yet existing prisons may not meet straight often adult players. |4.50|1.48|9016011|corpunivamalg #11|16|mystery|9|Books|481|oughteingese|N/A|62239peach9742821889|white|Tsp|Unknown|52|barationeingcallyought| +16871|AAAAAAAAGOBEAAAA|1999-10-28|2001-10-26|Annual, brief privileges mean there personnel. Spatial, good elections may give. Necessary, true questions ignore more just other decis|3.90|1.87|3004002|edu packexporti #2|4|school-uniforms|3|Children|481|oughteingese|extra large|0824puff707228105361|beige|Cup|Unknown|49|oughtationeingcallyought| +16872|AAAAAAAAGOBEAAAA|2001-10-27||Annual, brief privileges mean there personnel. Spatial, good elections may give. Necessary, true questions ignore more just other decis|3.79|2.57|3002001|importoexporti #1|2|infants|3|Children|481|oughteingese|medium|2587898338898papaya1|tan|Gross|Unknown|61|ableationeingcallyought| +16873|AAAAAAAAJOBEAAAA|1997-10-27||Thoughts ought to happen national relationships. Urban, sorry students |9.48|5.78|2001002|amalgimporto #2|1|accessories|2|Men|393|prin stpri|medium|292950453270medium78|thistle|Carton|Unknown|68|priationeingcallyought| +16874|AAAAAAAAKOBEAAAA|1997-10-27|2000-10-26|Main, british women will visualise for a countries. Numerous, white legs worry. Later entire c|5.35|1.60|6011003|amalgbrand #3|11|semi-precious|6|Jewelry|152|ableantiought|N/A|725sandy080886046402|red|Gram|Unknown|52|eseationeingcallyought| +16875|AAAAAAAAKOBEAAAA|2000-10-27||Main, british women will visualise for a countries. Numerous, white legs worry. Later entire c|2.26|1.17|6011003|edu packnameless #6|14|glassware|7|Home|105|antibarought|N/A|24864peach5663234379|lime|Pound|Unknown|60|antiationeingcallyought| +16876|AAAAAAAAMOBEAAAA|1997-10-27|1999-10-27|Now young patients will produce now to a comments; newly racial ears let financial, active studies. Only parameters may move hardly spontaneously unexpected beliefs. Small|4.47|3.30|5002001|importoscholar #1|2|country|5|Music|971|oughtationn st|N/A|slate018957388208942|wheat|Cup|Unknown|60|callyationeingcallyought| +16877|AAAAAAAAMOBEAAAA|1999-10-28|2001-10-26|Now young patients will produce now to a comments; newly racial ears let financial, active studies. Only parameters may move hardly spontaneously unexpected beliefs. Small|0.68|0.58|3001002|amalgexporti #2|1|newborn|3|Children|971|oughtationn st|extra large|slate018957388208942|rosy|Dram|Unknown|22|ationationeingcallyought| +16878|AAAAAAAAMOBEAAAA|2001-10-27||More possible meetings shall reach single, hot communists. Dependent, free changes should not want as constant doors; tight, other forces regard before centres. Legal|4.62|0.58|3001002|edu packamalg #1|1|swimwear|1|Women|120|oughtationn st|large|631474936263126red68|pale|Gross|Unknown|60|eingationeingcallyought| +16879|AAAAAAAAPOBEAAAA|1997-10-27||Dynamic holidays should involve already stations. Potential, following contracts used to need private, christian machines. Clear sports feel less |0.72|0.36|3004002|edu packexporti #2|4|school-uniforms|3|Children|593|prin stanti|medium|97960074753247peach8|thistle|Ounce|Unknown|15|n stationeingcallyought| +16880|AAAAAAAAAPBEAAAA|1997-10-27|2000-10-26|Gentle, main differences need to a be|0.83|0.58|8002001|importonameless #1|2|baseball|8|Sports|494|esen stese|N/A|1212medium6765878155|powder|Bundle|Unknown|33|bareingeingcallyought| +16881|AAAAAAAAAPBEAAAA|2000-10-27||Gentle, main differences need to a be|4.31|3.40|8002001|importoamalg #2|2|fragrances|1|Women|494|esen stese|small|31022285khaki6692062|rose|Lb|Unknown|12|oughteingeingcallyought| +16882|AAAAAAAACPBEAAAA|1997-10-27|1999-10-27|Labour modules rely together with the problems. Later available schola|8.93|3.30|3004001|edu packexporti #1|4|school-uniforms|3|Children|128|eingableought|large|955675920rose0147142|steel|Gram|Unknown|24|ableeingeingcallyought| +16883|AAAAAAAACPBEAAAA|1999-10-28|2001-10-26|Forms equip al|6.76|3.30|3004001|exportischolar #2|4|pop|5|Music|290|barn stable|N/A|955675920rose0147142|wheat|Unknown|Unknown|43|prieingeingcallyought| +16884|AAAAAAAACPBEAAAA|2001-10-27||Forms equip al|3.82|2.59|5001001|amalgscholar #1|1|rock|5|Music|290|barn stable|N/A|955675920rose0147142|plum|Ounce|Unknown|8|eseeingeingcallyought| +16885|AAAAAAAAFPBEAAAA|1997-10-27||Common, white details can spend from a factors. |1.29|1.00|2001002|amalgimporto #2|1|accessories|2|Men|619|n stoughtcally|small|638royal412345604461|mint|Ounce|Unknown|60|antieingeingcallyought| +16886|AAAAAAAAGPBEAAAA|1997-10-27|2000-10-26|Above upper shares should recall from a emotions. Books could not help british, |1.23|0.86|9008003|namelessmaxi #3|8|romance|9|Books|149|n steseought|N/A|26652tomato218478726|peach|Box|Unknown|6|callyeingeingcallyought| +16887|AAAAAAAAGPBEAAAA|2000-10-27||Above upper shares should recall from a emotions. Books could not help british, |80.84|0.86|7002008|importobrand #8|8|bedding|7|Home|149|n steseought|N/A|604616203salmon30155|wheat|Oz|Unknown|24|ationeingeingcallyought| +16888|AAAAAAAAIPBEAAAA|1997-10-27|1999-10-27|Types can bump away philosophical camps. French things would face excellent, grey goods; tonight relative loans restructure earlier; alone, |3.33|2.49|3003001|exportiexporti #1|3|toddlers|3|Children|185|antieingought|petite|4662814seashell23841|grey|Pound|Unknown|12|eingeingeingcallyought| +16889|AAAAAAAAIPBEAAAA|1999-10-28|2001-10-26|Types can bump away philosophical camps. French things would face excellent, grey goods; tonight relative loans restructure earlier; alone, |54.65|35.52|8013010|exportimaxi #10|3|sailing|8|Sports|185|antieingought|N/A|4662814seashell23841|violet|Carton|Unknown|37|n steingeingcallyought| +16890|AAAAAAAAIPBEAAAA|2001-10-27||Councils appear often political, comprehensive plans. Enthusiastically disabled numbers shall watch far to the computers. Musical, positive products prevent perhaps years. Indeed terrible progra|28.80|25.34|8013010|edu packexporti #1|3|school-uniforms|3|Children|185|antieingought|medium|87dark46983318491147|pink|Carton|Unknown|28|barn steingcallyought| +16891|AAAAAAAALPBEAAAA|1997-10-27||Corporate politicians must break local doors. Free, great experiences would not take best too following claims. Elderly points used to know aware, good cent|2.92|1.37|4003002|exportiedu pack #2|3|kids|4|Shoes|223|priableable|large|4363605turquoise7215|violet|Box|Unknown|46|oughtn steingcallyought| +16892|AAAAAAAAMPBEAAAA|1997-10-27|2000-10-26|Useful contents shall decide then british children. Wounds might not add also co|98.45|47.25|5004001|edu packscholar #1|4|classical|5|Music|251|oughtantiable|N/A|140salmon24203733698|forest|Pallet|Unknown|21|ablen steingcallyought| +16893|AAAAAAAAMPBEAAAA|2000-10-27||Blank, live plants look however |2.66|47.25|5004001|edu packamalg #2|4|swimwear|1|Women|251|oughtantiable|medium|140salmon24203733698|khaki|Gross|Unknown|11|prin steingcallyought| +16894|AAAAAAAAOPBEAAAA|1997-10-27|1999-10-27|Ever positive children must mean abroad quiet physical observers. Almost good rates fall perhaps britis|2.52|1.48|10002014|importounivamalg #14|2|camcorders|10|Electronics|373|priationpri|N/A|635295954sandy768078|cyan|Pallet|Unknown|52|esen steingcallyought| +16895|AAAAAAAAOPBEAAAA|1999-10-28|2001-10-26|High, combined visitors go particularly. Police share loans. Romantic doors will not take abo|7.12|1.48|10002014|importoedu pack #2|2|mens|4|Shoes|373|priationpri|large|635295954sandy768078|midnight|Case|Unknown|32|antin steingcallyought| +16896|AAAAAAAAOPBEAAAA|2001-10-27||High, combined visitors go particularly. Police share loans. Romantic doors will not take abo|8.08|1.48|10002014|importoedu pack #1|2|mens|4|Shoes|373|priationpri|medium|635295954sandy768078|frosted|Bunch|Unknown|28|callyn steingcallyought| +16897|AAAAAAAABACEAAAA|1997-10-27||Over small premises may bring also. Objectives used to ensure adequate others. Italian |6.21|3.97|8005008|scholarnameless #8|5|fitness|8|Sports|681|oughteingcally|N/A|533218241puff0282298|tan|Lb|Unknown|97|ationn steingcallyought| +16898|AAAAAAAACACEAAAA|1997-10-27|2000-10-26|Approximately similar examples must not incur. Communities look explicit, additional responsibilities; new symptoms get so best big others. Jobs sell even. Small |0.62|0.28|8010009|univmaxi #9|10|pools|8|Sports|121|oughtableought|N/A|090689349papaya68553|thistle|Box|Unknown|9|eingn steingcallyought| +16899|AAAAAAAACACEAAAA|2000-10-27||Approximately similar examples must not incur. Communities look explicit, additional responsibilities; new symptoms get so best big others. Jobs sell even. Small |8.20|0.28|8005010|scholarnameless #10|5|fitness|8|Sports|121|oughtableought|N/A|090689349papaya68553|salmon|Unknown|Unknown|32|n stn steingcallyought| +16900|AAAAAAAAEACEAAAA|1997-10-27|1999-10-27|Public, successful resources intend to a figures; available hands could not agree more visitors. Possible, other months k|3.18|1.01|2004001|edu packimporto #1|4|sports-apparel|2|Men|197|ationn stought|medium|pink7567474504743620|honeydew|Gross|Unknown|31|barbarn stcallyought| +16901|AAAAAAAAEACEAAAA|1999-10-28|2001-10-26|Public, successful resources intend to a figures; available hands could not agree more visitors. Possible, other months k|3.47|1.01|2004001|brandunivamalg #4|4|personal|10|Electronics|197|ationn stought|N/A|25073696932midnight6|white|Pound|Unknown|65|oughtbarn stcallyought| +16902|AAAAAAAAEACEAAAA|2001-10-27||Races think too. Guilty odds put all right. Dishes will find as. Individuals buy as widespread, different plants. Weeks honour from|1.50|1.01|2004001|amalgnameless #3|11|accent|7|Home|205|antibarable|N/A|547978773yellow38661|peach|Gross|Unknown|9|ablebarn stcallyought| +16903|AAAAAAAAHACEAAAA|1997-10-27||Real values wish wide. Brothers would not serve currently forward related edges. Groups should not waste subsequently below real|6.43|5.46|1003002|exportiamalg #2|3|maternity|1|Women|11|oughtought|extra large|3408823718metallic61|misty|Each|Unknown|19|pribarn stcallyought| +16904|AAAAAAAAIACEAAAA|1997-10-27|2000-10-26|Transactions must p|3.42|2.15|2004001|edu packimporto #1|4|sports-apparel|2|Men|305|antibarpri|petite|45violet122543798710|coral|Gross|Unknown|52|esebarn stcallyought| +16905|AAAAAAAAIACEAAAA|2000-10-27||Departments assess also almost comprehensive students. Well tough funds can act effectively months. Good words mu|7.20|3.38|10001008|amalgunivamalg #8|1|cameras|10|Electronics|194|esen stought|N/A|43358spring660961132|seashell|Lb|Unknown|42|antibarn stcallyought| +16906|AAAAAAAAKACEAAAA|1997-10-27|1999-10-27|R|2.32|1.20|10006009|corpunivamalg #9|6|musical|10|Electronics|647|ationesecally|N/A|0716350546613sky0906|lace|Each|Unknown|30|callybarn stcallyought| +16907|AAAAAAAAKACEAAAA|1999-10-28|2001-10-26|Only aware flowers used to return again vast teams. Drugs|6.58|1.20|10001010|amalgunivamalg #10|6|cameras|10|Electronics|647|ationesecally|N/A|0716350546613sky0906|tomato|N/A|Unknown|19|ationbarn stcallyought| +16908|AAAAAAAAKACEAAAA|2001-10-27||For instance physical instances engage views. British|85.72|29.14|10001010|amalgimporto #1|6|accessories|2|Men|647|ationesecally|petite|2linen02429898180232|beige|Each|Unknown|94|eingbarn stcallyought| +16909|AAAAAAAANACEAAAA|1997-10-27||Likely leaders must feel very absolutely military lips. Complete|0.84|0.35|3003002|exportiexporti #2|3|toddlers|3|Children|54|eseanti|N/A|1691985986deep334499|peach|Each|Unknown|95|n stbarn stcallyought| +16910|AAAAAAAAOACEAAAA|1997-10-27|2000-10-26|Grand years must not provide c|5.39|1.83|7014001|edu packnameless #1|14|glassware|7|Home|516|callyoughtanti|N/A|2004827498smoke17389|white|Unknown|Unknown|52|baroughtn stcallyought| +16911|AAAAAAAAOACEAAAA|2000-10-27||Grand years must not provide c|68.01|1.83|3002002|importoexporti #2|2|infants|3|Children|148|eingeseought|medium|140puff1329345214908|spring|Pound|Unknown|18|oughtoughtn stcallyought| +16912|AAAAAAAAABCEAAAA|1997-10-27|1999-10-27|Other, recent representations resolve both normal talks. Old, unlikely specialists apply just complete cl|5.17|3.05|8007003|brandnameless #3|7|hockey|8|Sports|58|einganti|N/A|80274919908lemon3182|yellow|Box|Unknown|18|ableoughtn stcallyought| +16913|AAAAAAAAABCEAAAA|1999-10-28|2001-10-26|Japanese conditions get as then traditional nights. Blue, long patients live home courts. Esp|1.83|3.05|5004002|edu packscholar #2|4|classical|5|Music|58|einganti|N/A|80274919908lemon3182|purple|Tsp|Unknown|13|prioughtn stcallyought| +16914|AAAAAAAAABCEAAAA|2001-10-27||Alone, small kinds seem extremely aware, potential points. Classes must increase good places. St|0.68|3.05|2001001|amalgimporto #1|1|accessories|2|Men|58|einganti|extra large|27250red087236465787|olive|Pallet|Unknown|25|eseoughtn stcallyought| +16915|AAAAAAAADBCEAAAA|1997-10-27||Free, difficult thoughts sit more tall companies; important churches transform nuclear, extra counties. Socially tight questions must provide highly similar symbolic weeks; aware,|3.72|1.37|10011001|amalgamalgamalg #1|11|disk drives|10|Electronics|535|antiprianti|N/A|0rose140964590015709|seashell|Bunch|Unknown|34|antioughtn stcallyought| +16916|AAAAAAAAEBCEAAAA|1997-10-27|2000-10-26|Different eyes know markets. Quickly atlantic objects might not call clearly in a points. Losses end then in a provisions. Studies pay about with the|1.15|0.73|1003001|exportiamalg #1|3|maternity|1|Women|276|callyationable|small|2steel69467126532896|yellow|Dram|Unknown|61|callyoughtn stcallyought| +16917|AAAAAAAAEBCEAAAA|2000-10-27||Different eyes know markets. Quickly atlantic objects might not call clearly in a points. Losses end then in a provisions. Studies pay about with the|51.61|0.73|1003001|amalgbrand #6|3|semi-precious|6|Jewelry|766|callyationable|N/A|2steel69467126532896|chocolate|Pallet|Unknown|23|ationoughtn stcallyought| +16918|AAAAAAAAGBCEAAAA|1997-10-27|1999-10-27|Creative, urban cells provide for once historical ideas. Delegates could fire directly lines. Huge, electrical teachers contribute only by a wives. Aggressive|4.15|1.61|8004009|edu packnameless #9|4|camping|8|Sports|558|eingantianti|N/A|947308525midnight229|saddle|Ton|Unknown|9|eingoughtn stcallyought| +16919|AAAAAAAAGBCEAAAA|1999-10-28|2001-10-26|Very popular beans recognise only names. Present, direct citizens remain then in the hours. Even straight cells sho|2.90|0.89|8004009|corpbrand #8|4|consignment|6|Jewelry|558|eingantianti|N/A|947308525midnight229|violet|Gram|Unknown|57|n stoughtn stcallyought| +16920|AAAAAAAAGBCEAAAA|2001-10-27||Very popular beans recognise only names. Present, direct citizens remain then in the hours. Even straight cells sho|2.67|0.89|5004001|edu packscholar #1|4|classical|5|Music|282|ableeingable|N/A|yellow28429498447372|coral|N/A|Unknown|66|barablen stcallyought| +16921|AAAAAAAAJBCEAAAA|1997-10-27||Relationships fit. Lines should mind for example as the things. Also chemical goods die both short proud systems. Chemical|1.37|0.95|5003002|exportischolar #2|3|pop|5|Music|147|ationeseought|N/A|5266141929lime698878|burnished|Lb|Unknown|49|oughtablen stcallyought| +16922|AAAAAAAAKBCEAAAA|1997-10-27|2000-10-26|Cultural, great readers used to specify aloud only horses. Things shall find further russian, practical r|42.61|27.69|6006003|corpcorp #3|6|rings|6|Jewelry|158|eingantiought|N/A|peach626441971833768|royal|Tsp|Unknown|26|ableablen stcallyought| +16923|AAAAAAAAKBCEAAAA|2000-10-27||Cultural, great readers used to specify aloud only horses. Things shall find further russian, practical r|0.57|27.69|2004002|edu packimporto #2|6|sports-apparel|2|Men|41|oughtese|small|909lavender031678359|maroon|Gross|Unknown|51|priablen stcallyought| +16924|AAAAAAAAMBCEAAAA|1997-10-27|1999-10-27|Quite new problems can apply often right other exam|6.48|5.63|1004001|edu packamalg #1|4|swimwear|1|Women|92|ablen st|medium|3316296218pale192497|navy|Oz|Unknown|24|eseablen stcallyought| +16925|AAAAAAAAMBCEAAAA|1999-10-28|2001-10-26|Disciplines examine perhaps long necessary witnesses. Highly new letters bite wisely indeed public interests. Deep, |1.24|0.48|1004001|edu packunivamalg #11|4|audio|10|Electronics|643|priesecally|N/A|0846178880257pale492|magenta|Dozen|Unknown|95|antiablen stcallyought| +16926|AAAAAAAAMBCEAAAA|2001-10-27||Disciplines examine perhaps long necessary witnesses. Highly new letters bite wisely indeed public interests. Deep, |3.32|0.48|1004001|edu packbrand #7|4|curtains/drapes|7|Home|643|priesecally|N/A|0846178880257pale492|blush|Unknown|Unknown|33|callyablen stcallyought| +16927|AAAAAAAAPBCEAAAA|1997-10-27||Still different businesses shall secure better material certain ways. Foreign, f|2.83|2.51|6015008|scholarbrand #8|15|custom|6|Jewelry|46|callyese|N/A|01671563254615wheat4|plum|Pound|Unknown|64|ationablen stcallyought| +16928|AAAAAAAAACCEAAAA|1997-10-27|2000-10-26|Homes make. Hospitals might gain for a lives. As|3.27|2.87|4001001|amalgedu pack #1|1|womens|4|Shoes|545|antieseanti|economy|887683ivory409080704|red|Dozen|Unknown|16|eingablen stcallyought| +16929|AAAAAAAAACCEAAAA|2000-10-27||Homes make. Hospitals might gain for a lives. As|7.91|2.87|7011010|amalgnameless #10|1|accent|7|Home|545|antieseanti|N/A|887683ivory409080704|maroon|Gram|Unknown|32|n stablen stcallyought| +16930|AAAAAAAACCCEAAAA|1997-10-27|1999-10-27|New, medical men ought to relinquish quicker away primary cases. Male, only products make entirely between a types. Blue chiefs ought to become hot, new differences|0.83|0.55|6002003|importocorp #3|2|diamonds|6|Jewelry|311|oughtoughtpri|N/A|79413red830099257166|navajo|Carton|Unknown|63|barprin stcallyought| +16931|AAAAAAAACCCEAAAA|1999-10-28|2001-10-26|New, medical men ought to relinquish quicker away primary cases. Male, only products make entirely between a types. Blue chiefs ought to become hot, new differences|2.17|0.55|6002003|edu packedu pack #2|4|athletic|4|Shoes|152|oughtoughtpri|economy|772goldenrod63078108|snow|Bunch|Unknown|88|oughtprin stcallyought| +16932|AAAAAAAACCCEAAAA|2001-10-27||Men ought to dry easy, alone reactions. Underlying, male economies recognise simultaneously by a things. Of course clear leaders apply thus methods. Public, respectable things wo|1.21|0.59|6002003|amalgscholar #1|1|rock|5|Music|276|callyationable|N/A|772goldenrod63078108|smoke|N/A|Unknown|44|ableprin stcallyought| +16933|AAAAAAAAFCCEAAAA|1997-10-27||Whole companies teach more subsequent, similar priests. From time to time united tests should talk men. Fine standards come to|7.77|5.74|9005002|scholarmaxi #2|5|history|9|Books|320|barablepri|N/A|4100000589slate32537|honeydew|Cup|Unknown|29|priprin stcallyought| +16934|AAAAAAAAGCCEAAAA|1997-10-27|2000-10-26|Then narrow problems show now just social competitors. Lives may not become individual, bloody resources; roots |1.10|0.63|8016005|corpmaxi #5|16|golf|8|Sports|7|ation|N/A|2767yellow4851665128|puff|Case|Unknown|33|eseprin stcallyought| +16935|AAAAAAAAGCCEAAAA|2000-10-27||Then narrow problems show now just social competitors. Lives may not become individual, bloody resources; roots |1.34|1.04|8016005|scholarmaxi #6|15|fishing|8|Sports|704|ation|N/A|2767yellow4851665128|thistle|Dram|Unknown|20|antiprin stcallyought| +16936|AAAAAAAAICCEAAAA|1997-10-27|1999-10-27|Most present groups will matter already about a players; happy, e|4.26|3.70|9014005|edu packunivamalg #5|14|sports|9|Books|307|ationbarpri|N/A|75slate4007647402006|lavender|Dozen|Unknown|79|callyprin stcallyought| +16937|AAAAAAAAICCEAAAA|1999-10-28|2001-10-26|Most present groups will matter already about a players; happy, e|55.16|18.20|7001008|amalgbrand #8|14|bathroom|7|Home|285|antieingable|N/A|092870hot65875284215|steel|Bundle|Unknown|40|ationprin stcallyought| +16938|AAAAAAAAICCEAAAA|2001-10-27||Most present groups will matter already about a players; happy, e|0.29|0.20|7001008|edu packexporti #1|14|school-uniforms|3|Children|773|antieingable|medium|092870hot65875284215|plum|Box|Unknown|47|eingprin stcallyought| +16939|AAAAAAAALCCEAAAA|1997-10-27||Daily lines must say as. Ready conditions avoid police. Girls ought to reveal however managerial affairs;|19.65|17.48|9012008|importounivamalg #8|12|home repair|9|Books|1|ought|N/A|1889slate51639443591|turquoise|Box|Unknown|61|n stprin stcallyought| +16940|AAAAAAAAMCCEAAAA|1997-10-27|2000-10-26|Fingers would call. Federal, fierce twins fit. Ordinary approaches mean peculiar, new standards. Sometimes intense contributions concern as political differences.|3.86|3.12|3004001|edu packexporti #1|4|school-uniforms|3|Children|38|eingpri|extra large|813903goldenrod77788|purple|Box|Unknown|55|baresen stcallyought| +16941|AAAAAAAAMCCEAAAA|2000-10-27||Tories shall not stay. As a whole high ways can buy agricultural, present papers. Cases should deal again here good strategies. National|3.02|2.05|3004001|exportiimporto #2|3|pants|2|Men|38|eingpri|large|813903goldenrod77788|rose|Unknown|Unknown|59|oughtesen stcallyought| +16942|AAAAAAAAOCCEAAAA|1997-10-27|1999-10-27|Themes will work all penalties. Old, radical germans confess of course econom|4.03|2.49|5004001|edu packscholar #1|4|classical|5|Music|525|antiableanti|N/A|750788thistle7965213|sky|Gross|Unknown|21|ableesen stcallyought| +16943|AAAAAAAAOCCEAAAA|1999-10-28|2001-10-26|Themes will work all penalties. Old, radical germans confess of course econom|9.66|2.49|4003002|exportiedu pack #2|3|kids|4|Shoes|525|antiableanti|large|750788thistle7965213|sandy|Ounce|Unknown|13|priesen stcallyought| +16944|AAAAAAAAOCCEAAAA|2001-10-27||Traditional, individual ministers disappear perfectly never everyday standards. Earnestly availab|2.89|2.34|8002005|importonameless #5|2|baseball|8|Sports|525|antiableanti|N/A|750788thistle7965213|royal|Tbl|Unknown|24|eseesen stcallyought| +16945|AAAAAAAABDCEAAAA|1997-10-27||Else married minutes must not believe|1.22|1.01|8010006|univmaxi #6|10|pools|8|Sports|630|barprically|N/A|37powder885905691060|tomato|Unknown|Unknown|13|antiesen stcallyought| +16946|AAAAAAAACDCEAAAA|1997-10-27|2000-10-26|Girls exceed so. Evenings shall not come so american, british shares. Interesting interests mark retail, historic studies; h|88.60|44.30|7004007|edu packbrand #7|4|curtains/drapes|7|Home|310|baroughtpri|N/A|579545turquoise25027|tomato|Carton|Unknown|6|callyesen stcallyought| +16947|AAAAAAAACDCEAAAA|2000-10-27||Enough automatic effects look slowly. Races should stay maps; nervous, high hands might make me|4.14|44.30|8008008|namelessnameless #8|8|outdoor|8|Sports|611|oughtoughtcally|N/A|579545turquoise25027|orchid|Oz|Unknown|30|ationesen stcallyought| +16948|AAAAAAAAEDCEAAAA|1997-10-27|1999-10-27|Much grey crowds dominate rigorously elderly groups. Counties might not grow on a meetings. Fair, essential drugs enjoy. Dir|4.28|3.46|6003005|exporticorp #5|3|gold|6|Jewelry|622|ableablecally|N/A|7249092366plum150340|white|Ton|Unknown|43|eingesen stcallyought| +16949|AAAAAAAAEDCEAAAA|1999-10-28|2001-10-26|Sales make new resources. Sepa|7.77|3.46|5003002|exportischolar #2|3|pop|5|Music|958|ableablecally|N/A|saddle12214331976187|steel|N/A|Unknown|20|n stesen stcallyought| +16950|AAAAAAAAEDCEAAAA|2001-10-27||National depths must look major, tory systems. Services will operate for the engineers. Specific, estimated cats consume har|2.41|3.46|2001001|amalgimporto #1|3|accessories|2|Men|115|ableablecally|medium|saddle12214331976187|thistle|Tsp|Unknown|28|barantin stcallyought| +16951|AAAAAAAAHDCEAAAA|1997-10-27||Poor sons hold to a cattle; folk make real words. Close new mistakes would say investigations. Top bo|0.29|0.11|4001002|amalgedu pack #2|1|womens|4|Shoes|188|eingeingought|extra large|93734puff71066768104|violet|Case|Unknown|21|oughtantin stcallyought| +16952|AAAAAAAAIDCEAAAA|1997-10-27|2000-10-26|Here equivalent expectations should stop since to th|7.88|5.04|10005004|scholarunivamalg #4|5|karoke|10|Electronics|250|barantiable|N/A|093cornsilk453333921|steel|Gram|Unknown|49|ableantin stcallyought| +16953|AAAAAAAAIDCEAAAA|2000-10-27||Here equivalent expectations should stop since to th|1.98|5.04|4003002|exportiedu pack #2|5|kids|4|Shoes|168|barantiable|large|5599576886metallic47|tan|Ounce|Unknown|94|priantin stcallyought| +16954|AAAAAAAAKDCEAAAA|1997-10-27|1999-10-27|Big, huge goals add usually here commercial things; keen, pregnant years might imagine somewhere rules. Highly respo|2.11|0.92|8014005|edu packmaxi #5|14|tennis|8|Sports|459|n stantiese|N/A|4775295thistle046302|puff|Tbl|Unknown|33|eseantin stcallyought| +16955|AAAAAAAAKDCEAAAA|1999-10-28|2001-10-26|Big, huge goals add usually here commercial things; keen, pregnant years might imagine somewhere rules. Highly respo|0.71|0.32|8014005|importoimporto #2|14|shirts|2|Men|4|n stantiese|small|royal736922496629557|tomato|Each|Unknown|6|antiantin stcallyought| +16956|AAAAAAAAKDCEAAAA|2001-10-27||Practical, central changes could light in a clothes. European decisions learn among a managers. Advisory, natural fields protect more still main pubs. Needs protest little adverse legs. Roug|3.40|2.04|3003001|exportiexporti #1|3|toddlers|3|Children|4|n stantiese|petite|royal736922496629557|powder|Tbl|Unknown|21|callyantin stcallyought| +16957|AAAAAAAANDCEAAAA|1997-10-27||Only benefits will|1.81|1.52|4002002|importoedu pack #2|2|mens|4|Shoes|615|antioughtcally|petite|2762987red1310951674|turquoise|Each|Unknown|8|ationantin stcallyought| +16958|AAAAAAAAODCEAAAA|1997-10-27|2000-10-26|Similar, only groups meet long. Poems shall like|9.98|4.89|9010003|univunivamalg #3|10|travel|9|Books|381|oughteingpri|N/A|78almond841747866552|wheat|Dram|Unknown|82|eingantin stcallyought| +16959|AAAAAAAAODCEAAAA|2000-10-27||Similar, only groups meet long. Poems shall like|2.59|4.89|6009008|maxicorp #8|9|womens watch|6|Jewelry|89|n steing|N/A|78almond841747866552|powder|Tsp|Unknown|30|n stantin stcallyought| +16960|AAAAAAAAAECEAAAA|1997-10-27|1999-10-27|Substantial colleagues would take away sexual needs; surprising, commercial features feed abroad from |4.71|2.16|2004001|edu packimporto #1|4|sports-apparel|2|Men|321|oughtablepri|petite|17933592goldenrod972|powder|Ounce|Unknown|38|barcallyn stcallyought| +16961|AAAAAAAAAECEAAAA|1999-10-28|2001-10-26|Substantial colleagues would take away sexual needs; surprising, commercial features feed abroad from |9.21|7.18|4001002|amalgedu pack #2|4|womens|4|Shoes|321|oughtablepri|medium|135white637678494180|khaki|Gross|Unknown|15|oughtcallyn stcallyought| +16962|AAAAAAAAAECEAAAA|2001-10-27||Substantial colleagues would take away sexual needs; surprising, commercial features feed abroad from |12.61|10.21|4001002|exportischolar #1|3|pop|5|Music|212|oughtablepri|N/A|2356894583043peach61|pink|Pallet|Unknown|99|ablecallyn stcallyought| +16963|AAAAAAAADECEAAAA|1997-10-27||Open, other words include a little sharply anxious soldiers. Conditions mean now rules. Patients shall vary around a problems. Difficult edges take stil|7.66|3.67|9012008|importounivamalg #8|12|home repair|9|Books|322|ableablepri|N/A|3519845rosy727318181|thistle|Dram|Unknown|52|pricallyn stcallyought| +16964|AAAAAAAAEECEAAAA|1997-10-27|2000-10-26|Political insects will not recognise. Certain, above areas could fit ahead aware,|8.41|7.14|3003001|exportiexporti #1|3|toddlers|3|Children|43|priese|large|23935304958puff09788|yellow|Bunch|Unknown|20|esecallyn stcallyought| +16965|AAAAAAAAEECEAAAA|2000-10-27||Political insects will not recognise. Certain, above areas could fit ahead aware,|4.62|2.81|3003001|importoamalgamalg #17|3|monitors|10|Electronics|43|priese|N/A|530230814violet23782|pale|Tbl|Unknown|11|anticallyn stcallyought| +16966|AAAAAAAAGECEAAAA|1997-10-27|1999-10-27|Continental, different departments must interpret also hard real transactions; methods win cuts; parental, similar polls|6.39|3.13|10011001|amalgamalgamalg #1|11|disk drives|10|Electronics|995|antin stn st|N/A|smoke805479624604595|powder|Each|Unknown|65|callycallyn stcallyought| +16967|AAAAAAAAGECEAAAA|1999-10-28|2001-10-26|Industrial samples stay as good steps; entire, necessary guns shall deal more private, long-term women. More interesting machines must |5.78|4.97|3004002|edu packexporti #2|11|school-uniforms|3|Children|183|prieingought|large|0503smoke24902190486|slate|Ounce|Unknown|46|ationcallyn stcallyought| +16968|AAAAAAAAGECEAAAA|2001-10-27||Northern forces get. Especially level humans participate huge, surprisin|5.72|4.97|3004002|exportibrand #9|3|kids|7|Home|309|prieingought|N/A|4042thistle570399582|orange|Ounce|Unknown|45|eingcallyn stcallyought| +16969|AAAAAAAAJECEAAAA|1997-10-27||Occasional, physical schools allow away also adequate processes. Countries get only on a cultures. Structures shall drift high british groups. Seriously colle|34.42|15.14|10005004|scholarunivamalg #4|5|karoke|10|Electronics|332|ablepripri|N/A|9359845turquoise7308|medium|Ton|Unknown|43|n stcallyn stcallyought| +16970|AAAAAAAAKECEAAAA|1997-10-27|2000-10-26|Distinct, general numb|47.84|25.83|10011005|amalgamalgamalg #5|11|disk drives|10|Electronics|286|callyeingable|N/A|87557862sienna630913|indian|Unknown|Unknown|19|barationn stcallyought| +16971|AAAAAAAAKECEAAAA|2000-10-27||Distinct, general numb|3.22|25.83|10011005|edu packscholar #2|4|classical|5|Music|286|callyeingable|N/A|8676577910metallic78|rosy|Ton|Unknown|41|oughtationn stcallyought| +16972|AAAAAAAAMECEAAAA|1997-10-27|1999-10-27|Then military letters give british, rural lips. Things begin wistfully stages. Magnificent women use medical rates. Visible, absolute relationships emerge basically lengthy |3.27|1.63|8001003|amalgnameless #3|1|athletic shoes|8|Sports|18|eingought|N/A|96300843970royal9064|salmon|Bunch|Unknown|4|ableationn stcallyought| +16973|AAAAAAAAMECEAAAA|1999-10-28|2001-10-26|Then military letters give british, rural lips. Things begin wistfully stages. Magnificent women use medical rates. Visible, absolute relationships emerge basically lengthy |8.45|3.46|8001003|edu packnameless #4|1|camping|8|Sports|18|eingought|N/A|8saddle3362386265800|rose|Gross|Unknown|85|priationn stcallyought| +16974|AAAAAAAAMECEAAAA|2001-10-27||Then military letters give british, rural lips. Things begin wistfully stages. Magnificent women use medical rates. Visible, absolute relationships emerge basically lengthy |3.61|3.46|6004007|edu packcorp #7|1|bracelets|6|Jewelry|668|eingcallycally|N/A|8saddle3362386265800|peru|Gross|Unknown|18|eseationn stcallyought| +16975|AAAAAAAAPECEAAAA|1997-10-27||Very internal students collect well large years. Right cases may solve in a instructions. Legal, n|4.93|1.67|2001002|amalgimporto #2|1|accessories|2|Men|576|callyationanti|medium|062navy8123387578000|ghost|Ounce|Unknown|57|antiationn stcallyought| +16976|AAAAAAAAAFCEAAAA|1997-10-27|2000-10-26|Areas could avoid. Initial, evident members shall not think planes; meanings would come even sound grants. Primary ma|4.94|2.17|8003007|exportinameless #7|3|basketball|8|Sports|861|oughtcallyeing|N/A|3271690656magenta832|turquoise|Gram|Unknown|80|callyationn stcallyought| +16977|AAAAAAAAAFCEAAAA|2000-10-27||Areas could avoid. Initial, evident members shall not think planes; meanings would come even sound grants. Primary ma|9.54|2.17|8004008|edu packnameless #8|3|camping|8|Sports|861|oughtcallyeing|N/A|7peru517358406703254|snow|Ounce|Unknown|66|ationationn stcallyought| +16978|AAAAAAAACFCEAAAA|1997-10-27|1999-10-27|Additional, terrible characters shall examine. Ago lexical conditions get into a weeks. Barely trying results perform still hot men. Great kinds end also committees. Police should live only on the |4.46|1.33|9006011|corpmaxi #11|6|parenting|9|Books|229|n stableable|N/A|77155108978881smoke1|smoke|Case|Unknown|36|eingationn stcallyought| +16979|AAAAAAAACFCEAAAA|1999-10-28|2001-10-26|Items may not justify. Regional relations strip effectively with the cells. Are|4.61|3.36|9006011|importonameless #10|6|baseball|8|Sports|450|barantiese|N/A|29919snow21902791518|puff|Bundle|Unknown|12|n stationn stcallyought| +16980|AAAAAAAACFCEAAAA|2001-10-27||Items may not justify. Regional relations strip effectively with the cells. Are|2.84|1.76|7003001|exportibrand #1|3|kids|7|Home|510|baroughtanti|N/A|spring97561719905704|honeydew|Case|Unknown|4|bareingn stcallyought| +16981|AAAAAAAAFFCEAAAA|1997-10-27||Imports must communicate on a women. Level difficulties c|3.93|1.76|7002002|importobrand #2|2|bedding|7|Home|138|eingpriought|N/A|64267pale91676610160|navajo|Gram|Unknown|85|oughteingn stcallyought| +16982|AAAAAAAAGFCEAAAA|1997-10-27|2000-10-26|Effects start comfortably now payable opportunities; separately great times like mainly dutch seats. Decisions obtain jus|7.13|6.41|5004001|edu packscholar #1|4|classical|5|Music|276|callyationable|N/A|99831lavender2398109|rose|Tbl|Unknown|30|ableeingn stcallyought| +16983|AAAAAAAAGFCEAAAA|2000-10-27||Knees should keep presumably pos|4.63|6.41|5004001|exportiamalg #2|4|maternity|1|Women|276|callyationable|medium|1thistle992136936677|khaki|Case|Unknown|77|prieingn stcallyought| +16984|AAAAAAAAIFCEAAAA|1997-10-27|1999-10-27|Possible companies will admire less things. Systems can pay. Small quantities see then as a boys; different designers make well for a personn|4.20|1.55|8010005|univmaxi #5|10|pools|8|Sports|467|ationcallyese|N/A|8527571327brown39966|peru|Ounce|Unknown|40|eseeingn stcallyought| +16985|AAAAAAAAIFCEAAAA|1999-10-28|2001-10-26|Total courts might see later problems. Anonymous plants could give with a businessmen. Other|3.97|1.55|2003002|exportiimporto #2|10|pants|2|Men|467|ationcallyese|small|7592spring0666257021|saddle|Each|Unknown|22|antieingn stcallyought| +16986|AAAAAAAAIFCEAAAA|2001-10-27||Aims run perhaps. Books take growing tears. Nevertheless british years recover at last similar times|1.29|1.55|2003002|edu packcorp #3|4|bracelets|6|Jewelry|467|ationcallyese|N/A|651olive348553135417|tomato|Carton|Unknown|5|callyeingn stcallyought| +16987|AAAAAAAALFCEAAAA|1997-10-27||Modern, natural prisoners should establish as modern weaknesses. Long, economic modules wish almost matters. Momen|4.47|2.45|9008008|namelessmaxi #8|8|romance|9|Books|322|ableablepri|N/A|3401197rose497846991|seashell|Pallet|Unknown|2|ationeingn stcallyought| +16988|AAAAAAAAMFCEAAAA|1997-10-27|2000-10-26|Yesterday complex achievements will care firmly wrong loc|8.67|3.20|1004001|edu packamalg #1|4|swimwear|1|Women|80|bareing|extra large|287138065violet18671|khaki|Carton|Unknown|61|eingeingn stcallyought| +16989|AAAAAAAAMFCEAAAA|2000-10-27||Yesterday complex achievements will care firmly wrong loc|9.98|3.20|5002002|importoscholar #2|4|country|5|Music|80|bareing|N/A|287138065violet18671|steel|Gross|Unknown|8|n steingn stcallyought| +16990|AAAAAAAAOFCEAAAA|1997-10-27|1999-10-27|Patients might apply so wholly large wives. S|4.08|2.32|5004001|edu packscholar #1|4|classical|5|Music|115|antioughtought|N/A|475thistle5364016566|misty|Tsp|Unknown|57|barn stn stcallyought| +16991|AAAAAAAAOFCEAAAA|1999-10-28|2001-10-26|Vast proposals may vary like a principles. Short methods project quite like a effects; sometimes only shoulders classify. Administrative divisions continue well. Charming, foll|2.06|2.32|5002002|importoscholar #2|2|country|5|Music|115|antioughtought|N/A|thistle8971602439009|pale|Tbl|Unknown|100|oughtn stn stcallyought| +16992|AAAAAAAAOFCEAAAA|2001-10-27||Vast proposals may vary like a principles. Short methods project quite like a effects; sometimes only shoulders classify. Administrative divisions continue well. Charming, foll|1.61|0.82|5002002|exportiexporti #1|3|toddlers|3|Children|115|antioughtought|medium|dark3855877613599121|olive|Each|Unknown|9|ablen stn stcallyought| +16993|AAAAAAAABGCEAAAA|1997-10-27||Absolute proteins will happen huge, important unions. Varieties might not climb old, dead memories. Social, efficient governments form especially. Deputies may encourage for ever years. |0.79|0.38|9004002|edu packmaxi #2|4|entertainments|9|Books|268|eingcallyable|N/A|808slate215070379203|plum|Cup|Unknown|15|prin stn stcallyought| +16994|AAAAAAAACGCEAAAA|1997-10-27|2000-10-26|Here aggressive elements must take. Years think early, effective components. Other, easy reasons used to buy always ther|9.14|3.10|6009003|maxicorp #3|9|womens watch|6|Jewelry|176|callyationought|N/A|476221pink5544471002|thistle|Ton|Unknown|28|esen stn stcallyought| +16995|AAAAAAAACGCEAAAA|2000-10-27||Happy, similar times go at all basic, high institutions. Troubles can point maybe right left employees. Technical,|3.51|2.52|10011013|amalgamalgamalg #13|11|disk drives|10|Electronics|176|callyationought|N/A|476221pink5544471002|gainsboro|Tbl|Unknown|5|antin stn stcallyought| +16996|AAAAAAAAEGCEAAAA|1997-10-27|1999-10-27|Thirdly cruel governors ought to mimic with a centuries. Only available courses used to retrieve less|4.51|3.78|10002014|importounivamalg #14|2|camcorders|10|Electronics|539|n stprianti|N/A|42124959037powder897|saddle|Cup|Unknown|18|callyn stn stcallyought| +16997|AAAAAAAAEGCEAAAA|1999-10-28|2001-10-26|Thirdly cruel governors ought to mimic with a centuries. Only available courses used to retrieve less|0.81|3.78|10002014|edu packcorp #6|2|bracelets|6|Jewelry|586|callyeinganti|N/A|42124959037powder897|thistle|Each|Unknown|22|ationn stn stcallyought| +16998|AAAAAAAAEGCEAAAA|2001-10-27||Straight technical drawings could become successful managers. Police see in a players. Yet military effects can seem now major wages. Cultural, remaining environments used to carry then o|1.44|3.78|7010009|univnameless #9|10|flatware|7|Home|906|callyeinganti|N/A|42124959037powder897|white|Carton|Unknown|22|eingn stn stcallyought| +16999|AAAAAAAAHGCEAAAA|1997-10-27||Digital patients gain to a colours. Years make tem|16.58|9.45|9008008|namelessmaxi #8|8|romance|9|Books|283|prieingable|N/A|152green886785158154|pale|Unknown|Unknown|56|n stn stn stcallyought| +17000|AAAAAAAAIGCEAAAA|1997-10-27|2000-10-26|Concerned numbers can attempt now particular, white friends; un|3.38|2.23|9003009|exportimaxi #9|3|computers|9|Books|276|callyationable|N/A|34aquamarine02975527|rose|Dram|Unknown|51|barbarbarationought| +17001|AAAAAAAAIGCEAAAA|2000-10-27||Concerned numbers can attempt now particular, white friends; un|52.86|21.14|9003009|amalgnameless #2|3|accent|7|Home|276|callyationable|N/A|34aquamarine02975527|green|N/A|Unknown|19|oughtbarbarationought| +17002|AAAAAAAAKGCEAAAA|1997-10-27|1999-10-27|Just popular others |4.83|3.76|2002001|importoimporto #1|2|shirts|2|Men|10|barought|medium|5233503lace766094501|grey|Dram|Unknown|28|ablebarbarationought| +17003|AAAAAAAAKGCEAAAA|1999-10-28|2001-10-26|Just popular others |8.54|3.76|5004002|edu packscholar #2|2|classical|5|Music|10|barought|N/A|8898180violet0103868|seashell|Cup|Unknown|44|pribarbarationought| +17004|AAAAAAAAKGCEAAAA|2001-10-27||Odds might bring then between an types. Boo|0.30|3.76|5004002|univmaxi #5|2|pools|8|Sports|10|barought|N/A|8898180violet0103868|wheat|Gross|Unknown|42|esebarbarationought| +17005|AAAAAAAANGCEAAAA|1997-10-27||Poor, practical ties lie just available skills; little humans look so. There minute troops will use just then|3.95|1.73|1002002|importoamalg #2|2|fragrances|1|Women|239|n stpriable|extra large|906665linen403449077|steel|Ton|Unknown|5|antibarbarationought| +17006|AAAAAAAAOGCEAAAA|1997-10-27|2000-10-26|Quickly sure benefits talk overnight ships. Briefly foreign bedrooms suggest however key soldiers. Individuals might become later royal circum|0.91|0.79|10013007|exportiamalgamalg #7|13|stereo|10|Electronics|136|callypriought|N/A|4411934sienna2971242|red|Gram|Unknown|52|callybarbarationought| +17007|AAAAAAAAOGCEAAAA|2000-10-27||Lively, regular decisions ought to think separately clear networks; english, little levels ought to keep years. Students make then until the organisers. Significant, st|2.42|0.79|10013007|importomaxi #4|13|business|9|Books|136|callypriought|N/A|67420652tan828858358|navajo|Carton|Unknown|15|ationbarbarationought| +17008|AAAAAAAAAHCEAAAA|1997-10-27|1999-10-27|Machines reject reasons. Dead cells go predominantly civil signs. Across small woods could lo|2.31|1.13|3001001|amalgexporti #1|1|newborn|3|Children|292|ablen stable|petite|74moccasin9664350134|seashell|Bunch|Unknown|55|eingbarbarationought| +17009|AAAAAAAAAHCEAAAA|1999-10-28|2001-10-26|Political prisoners take feet. Payments say immediately obviously white voices. Ways protect again quite standard shops. Benefits can say electoral voters. Rates get countries. Tests fade bad,|7.56|3.40|3001001|corpbrand #10|1|rugs|7|Home|292|ablen stable|N/A|74moccasin9664350134|spring|Tsp|Unknown|45|n stbarbarationought| +17010|AAAAAAAAAHCEAAAA|2001-10-27||Possible scots settle again leaders. Urban others relax. Ends remember as regional sentences; however national schools find very advantages.|3.51|3.40|2003001|exportiimporto #1|3|pants|2|Men|254|ablen stable|medium|1violet6844077226621|pink|Carton|Unknown|20|baroughtbarationought| +17011|AAAAAAAADHCEAAAA|1997-10-27||Alone, international clients can retire at least other services; even major properties come in a grounds.|68.55|47.98|8014002|edu packmaxi #2|14|tennis|8|Sports|722|ableableation|N/A|3239olive34329693505|tomato|Dozen|Unknown|22|oughtoughtbarationought| +17012|AAAAAAAAEHCEAAAA|1997-10-27|2000-10-26|Authorities shall not m|9.99|8.89|5003001|exportischolar #1|3|pop|5|Music|104|esebarought|N/A|647midnight253127466|drab|Carton|Unknown|59|ableoughtbarationought| +17013|AAAAAAAAEHCEAAAA|2000-10-27||Authorities shall not m|9.24|8.89|5003001|exportischolar #2|3|pop|5|Music|104|esebarought|N/A|steel747562244719825|floral|Dram|Unknown|75|prioughtbarationought| +17014|AAAAAAAAGHCEAAAA|1997-10-27|1999-10-27|Important hours complain. Now necessary applications mean so yet private observers. Academic, new tactics would |7.67|2.91|5002001|importoscholar #1|2|country|5|Music|786|callyeingation|N/A|12red377907465312602|hot|Dozen|Unknown|57|eseoughtbarationought| +17015|AAAAAAAAGHCEAAAA|1999-10-28|2001-10-26|Rather boring homes recognise as islands; written pictures go in a women. Empty events shall fight briefly financial goods; here social windows drive on the guidelines. Changes ma|0.25|0.15|8013006|exportimaxi #6|2|sailing|8|Sports|141|callyeingation|N/A|1828lace931453356656|salmon|Tbl|Unknown|23|antioughtbarationought| +17016|AAAAAAAAGHCEAAAA|2001-10-27||So minor paintings convince friends. Labour sons test feet. Communications help indeed. Essentially prime |8.62|7.15|2001001|amalgimporto #1|2|accessories|2|Men|141|callyeingation|small|1828lace931453356656|papaya|Oz|Unknown|45|callyoughtbarationought| +17017|AAAAAAAAJHCEAAAA|1997-10-27||Boats must encounter. Minor leaders think then to a folk. Weapons leave. Similarly british questions can take eventually explicit artificial chemicals. Also black lists sh|4.45|1.42|3001002|amalgexporti #2|1|newborn|3|Children|101|oughtbarought|small|50914linen9664121251|indian|Case|Unknown|76|ationoughtbarationought| +17018|AAAAAAAAKHCEAAAA|1997-10-27|2000-10-26|Nearly clear countries will learn in addition over the ages; also interesting eyes exercise also available years. More b|3.98|1.35|7013009|exportinameless #9|13|wallpaper|7|Home|375|antiationpri|N/A|9910424958703white86|sky|Oz|Unknown|69|eingoughtbarationought| +17019|AAAAAAAAKHCEAAAA|2000-10-27||Nearly clear countries will learn in addition over the ages; also interesting eyes exercise also available years. More b|9.72|5.05|7013009|exportiimporto #2|13|pants|2|Men|375|antiationpri|extra large|85544912light4600419|salmon|Ton|Unknown|34|n stoughtbarationought| +17020|AAAAAAAAMHCEAAAA|1997-10-27|1999-10-27|Only, main artists could work also more male needs. Forthcoming coins cannot take single, good resources. Women find beyond a farmers. Likely sums would release again easy years; official, national p|4.87|1.65|4002001|importoedu pack #1|2|mens|4|Shoes|218|eingoughtable|medium|65434367755064pale37|dim|Pallet|Unknown|30|barablebarationought| +17021|AAAAAAAAMHCEAAAA|1999-10-28|2001-10-26|British weeks remark. Small, guilty eyes m|0.47|1.65|4002001|importoscholar #2|2|country|5|Music|548|eingeseanti|N/A|4seashell76481573800|sandy|Tsp|Unknown|10|oughtablebarationought| +17022|AAAAAAAAMHCEAAAA|2001-10-27||Typical policies will explore between a rights. Right clothes shall leave long too dead procedures. Countries explain too therefore impossible concentrations; international machi|0.39|1.65|1001001|amalgamalg #1|2|dresses|1|Women|227|eingeseanti|petite|4seashell76481573800|powder|Case|Unknown|65|ableablebarationought| +17023|AAAAAAAAPHCEAAAA|1997-10-27||Again alone men keep mental pp..|2.84|1.33|6011008|amalgbrand #8|11|semi-precious|6|Jewelry|84|eseeing|N/A|salmon12044765140965|seashell|Each|Unknown|7|priablebarationought| +17024|AAAAAAAAAICEAAAA|1997-10-27|2000-10-26|Low women abolish. Involved, main windows l|8.78|4.91|3002001|importoexporti #1|2|infants|3|Children|354|eseantipri|petite|91171784724839grey39|sienna|Tsp|Unknown|23|eseablebarationought| +17025|AAAAAAAAAICEAAAA|2000-10-27||Low women abolish. Involved, main windows l|0.53|4.91|3002001|brandunivamalg #9|7|personal|10|Electronics|354|eseantipri|N/A|91171784724839grey39|lemon|Bunch|Unknown|70|antiablebarationought| +17026|AAAAAAAACICEAAAA|1997-10-27|1999-10-27|Only other services must evoke muc|6.97|2.30|6009003|maxicorp #3|9|womens watch|6|Jewelry|48|eingese|N/A|152375791pale6071510|orange|Unknown|Unknown|25|callyablebarationought| +17027|AAAAAAAACICEAAAA|1999-10-28|2001-10-26|Odd, easy citizens provide often french, basic things. |3.76|2.30|8010008|univmaxi #8|10|pools|8|Sports|48|eingese|N/A|0617228147613tomato8|light|Tsp|Unknown|2|ationablebarationought| +17028|AAAAAAAACICEAAAA|2001-10-27||Political elections grow now ill, complete partners. Ways must consider ill more than severe conditions. Situations can grow tomorrow actually high messages. Magazines will not stop english books; |8.00|2.30|8010008|namelessunivamalg #12|10|scanners|10|Electronics|250|eingese|N/A|0617228147613tomato8|orange|Case|Unknown|17|eingablebarationought| +17029|AAAAAAAAFICEAAAA|1997-10-27||All brown crowds create be|1.83|0.56|2003002|exportiimporto #2|3|pants|2|Men|707|ationbaration|petite|79591olive0492009590|yellow|Dozen|Unknown|69|n stablebarationought| +17030|AAAAAAAAGICEAAAA|1997-10-27|2000-10-26|Police may effect short, foreign pubs. Jobs must not show often foreign, constitutional times. Just new studies appeal great, big days; determined, certain pp. may suit ahead claims|7.52|5.03|8006001|corpnameless #1|6|football|8|Sports|650|barantically|N/A|6432397662seashell31|red|Gross|Unknown|13|barpribarationought| +17031|AAAAAAAAGICEAAAA|2000-10-27||Police may effect short, foreign pubs. Jobs must not show often foreign, constitutional times. Just new studies appeal great, big days; determined, certain pp. may suit ahead claims|9.09|5.03|7008002|namelessbrand #2|6|lighting|7|Home|650|barantically|N/A|6432397662seashell31|tan|Bunch|Unknown|52|oughtpribarationought| +17032|AAAAAAAAIICEAAAA|1997-10-27|1999-10-27|Due, only artists assume always from a friends. Already huge controls might replace theoretically yesterday spiritual terms. Little issues reduce much past a letters. Numbers deal to|1.99|1.15|4003001|exportiedu pack #1|3|kids|4|Shoes|286|callyeingable|extra large|moccasin858832594010|turquoise|Tsp|Unknown|80|ablepribarationought| +17033|AAAAAAAAIICEAAAA|1999-10-28|2001-10-26|Due, only artists assume always from a friends. Already huge controls might replace theoretically yesterday spiritual terms. Little issues reduce much past a letters. Numbers deal to|6.76|5.74|4003001|edu packedu pack #2|3|athletic|4|Shoes|286|callyeingable|petite|medium57237557611929|tomato|Lb|Unknown|17|pripribarationought| +17034|AAAAAAAAIICEAAAA|2001-10-27||Due, only artists assume always from a friends. Already huge controls might replace theoretically yesterday spiritual terms. Little issues reduce much past a letters. Numbers deal to|2.70|5.74|4004001|edu packedu pack #1|4|athletic|4|Shoes|937|ationprin st|petite|5230829760indian4357|thistle|Oz|Unknown|41|esepribarationought| +17035|AAAAAAAALICEAAAA|1997-10-27||Fears shall lodge islands. Confident, full opportunities would not live under. Police comfort for a books. Possible, new areas used to buy totally letters; ships ens|4.69|2.20|2001002|amalgimporto #2|1|accessories|2|Men|501|oughtbaranti|large|6137305white50978372|tomato|Case|Unknown|89|antipribarationought| +17036|AAAAAAAAMICEAAAA|1997-10-27|2000-10-26|No doubt full things must appear often poor, terrible names; numbers introduce at large questions. Around roman damages should not get also. Forces must not act performances. Scie|24.73|7.41|2002001|importoimporto #1|2|shirts|2|Men|416|callyoughtese|large|75700sienna975959538|slate|Dozen|Unknown|5|callypribarationought| +17037|AAAAAAAAMICEAAAA|2000-10-27||No doubt full things must appear often poor, terrible names; numbers introduce at large questions. Around roman damages should not get also. Forces must not act performances. Scie|3.46|2.42|8001008|amalgnameless #8|2|athletic shoes|8|Sports|233|pripriable|N/A|75700sienna975959538|dim|Dozen|Unknown|16|ationpribarationought| +17038|AAAAAAAAOICEAAAA|1997-10-27|1999-10-27|Children enjoy still into a spots. Circumstances used to obtain there to a years. Foreign schools cross real goods. Soviet, new payments will |7.03|5.55|2002001|importoimporto #1|2|shirts|2|Men|320|barablepri|medium|54031wheat8032990526|lime|Case|Unknown|24|eingpribarationought| +17039|AAAAAAAAOICEAAAA|1999-10-28|2001-10-26|Children enjoy still into a spots. Circumstances used to obtain there to a years. Foreign schools cross real goods. Soviet, new payments will |90.84|35.42|2002001|importoexporti #2|2|infants|3|Children|320|barablepri|petite|95991659cyan38617393|slate|Each|Unknown|49|n stpribarationought| +17040|AAAAAAAAOICEAAAA|2001-10-27||There pretty indians take c|4.77|2.71|2002001|amalgimporto #1|1|accessories|2|Men|320|barablepri|small|95991659cyan38617393|salmon|Ounce|Unknown|49|baresebarationought| +17041|AAAAAAAABJCEAAAA|1997-10-27||Other, visible eyes shall not raise with a methods. Evenings like for a months; problems must interfere on a others; cards may pe|3.64|3.20|1001002|amalgamalg #2|1|dresses|1|Women|950|barantin st|N/A|312863smoke860106640|papaya|Tbl|Unknown|63|oughtesebarationought| +17042|AAAAAAAACJCEAAAA|1997-10-27|2000-10-26|Available, dramatic areas see never living, curious forces. |4.84|4.01|10004009|edu packunivamalg #9|4|audio|10|Electronics|212|ableoughtable|N/A|3602849478burnished8|plum|Cup|Unknown|71|ableesebarationought| +17043|AAAAAAAACJCEAAAA|2000-10-27||Available, dramatic areas see never living, curious forces. |94.92|4.01|10004009|corpmaxi #4|4|parenting|9|Books|212|ableoughtable|N/A|7thistle885188130240|smoke|Tbl|Unknown|29|priesebarationought| +17044|AAAAAAAAEJCEAAAA|1997-10-27|1999-10-27|New, involved patients would play usually nuclear cond|2.36|1.03|4002001|importoedu pack #1|2|mens|4|Shoes|408|eingbarese|petite|0white31804509729984|royal|Bundle|Unknown|54|eseesebarationought| +17045|AAAAAAAAEJCEAAAA|1999-10-28|2001-10-26|New, involved patients would play usually nuclear cond|5.64|4.11|4002001|importoscholar #2|2|country|5|Music|408|eingbarese|N/A|052422plum4209353735|thistle|Unknown|Unknown|7|antiesebarationought| +17046|AAAAAAAAEJCEAAAA|2001-10-27||New, involved patients would play usually nuclear cond|1.47|0.57|4004001|edu packedu pack #1|2|athletic|4|Shoes|266|callycallyable|N/A|31290honeydew9570386|violet|Pound|Unknown|4|callyesebarationought| +17047|AAAAAAAAHJCEAAAA|1997-10-27||Different needs protect hundreds. Classes may happen quite all english categories. Closed parents last on a failures. As right cars apply even ingredients. Real, financial losses should n|7.16|6.01|8013008|exportimaxi #8|13|sailing|8|Sports|103|pribarought|N/A|14786721khaki7879864|saddle|Tsp|Unknown|16|ationesebarationought| +17048|AAAAAAAAIJCEAAAA|1997-10-27|2000-10-26|Patients stand still respective possibilities|2.66|1.14|7002009|importobrand #9|2|bedding|7|Home|645|antiesecally|N/A|803449royal712002784|rose|Case|Unknown|4|eingesebarationought| +17049|AAAAAAAAIJCEAAAA|2000-10-27||Traditional, total eyes used to pick ever clinical, original worlds. Social, left poin|3.14|1.72|7001010|amalgbrand #10|2|bathroom|7|Home|645|antiesecally|N/A|thistle0026234230309|white|Bundle|Unknown|28|n stesebarationought| +17050|AAAAAAAAKJCEAAAA|1997-10-27|1999-10-27|Successfully new patterns m|8.48|6.27|2001001|amalgimporto #1|1|accessories|2|Men|177|ationationought|large|0950372167711767red5|tomato|Ounce|Unknown|29|barantibarationought| +17051|AAAAAAAAKJCEAAAA|1999-10-28|2001-10-26|Successfully new patterns m|5.34|3.89|2001001|scholarnameless #2|1|tables|7|Home|368|ationationought|N/A|0950372167711767red5|tan|Cup|Unknown|21|oughtantibarationought| +17052|AAAAAAAAKJCEAAAA|2001-10-27||Successfully new patterns m|7.67|5.44|9014001|edu packunivamalg #1|1|sports|9|Books|368|eingcallypri|N/A|0950372167711767red5|lace|Carton|Unknown|60|ableantibarationought| +17053|AAAAAAAANJCEAAAA|1997-10-27||Ago fundamental shares should enforce almost psychological, german schools. Arbitrary calls compete. Turkish branches should operate rapidly great, current features. Occasi|0.88|0.57|6013006|exportibrand #6|13|loose stones|6|Jewelry|287|ationeingable|N/A|orange12455796194099|rose|Dozen|Unknown|64|priantibarationought| +17054|AAAAAAAAOJCEAAAA|1997-10-27|2000-10-26|Approaches used to worsen forwards yellow, effective days. Personal, musical dreams appreciate in a claims; future, natural doors make thus. Empirical,|3.81|3.20|9015003|scholarunivamalg #3|15|fiction|9|Books|632|ableprically|N/A|1917sky1917079469743|grey|Tbl|Unknown|15|eseantibarationought| +17055|AAAAAAAAOJCEAAAA|2000-10-27||Approaches used to worsen forwards yellow, effective days. Personal, musical dreams appreciate in a claims; future, natural doors make thus. Empirical,|2.28|3.20|9015003|amalgscholar #2|15|rock|5|Music|926|ableprically|N/A|1917sky1917079469743|plum|Case|Unknown|29|antiantibarationought| +17056|AAAAAAAAAKCEAAAA|1997-10-27|1999-10-27|Regrettably deep rivers make absolutely then major demands. Cold dangers open of course less essential stories. Legal, statistical studies amount more well sovi|4.23|1.94|7016007|corpnameless #7|16|furniture|7|Home|288|eingeingable|N/A|23993030809rosy79996|plum|Ounce|Unknown|7|callyantibarationought| +17057|AAAAAAAAAKCEAAAA|1999-10-28|2001-10-26|Secondary, contemporary children offer still really clear customers. Perfect, obvious documents must |5.57|1.94|8010008|univmaxi #8|16|pools|8|Sports|473|eingeingable|N/A|23993030809rosy79996|peach|Bunch|Unknown|6|ationantibarationought| +17058|AAAAAAAAAKCEAAAA|2001-10-27||Loosely german factors talk visibly at a members. Just equal claims |4.86|1.94|8010008|maxiunivamalg #8|16|televisions|10|Electronics|473|priationese|N/A|5886steel80226183758|seashell|Carton|Unknown|4|eingantibarationought| +17059|AAAAAAAADKCEAAAA|1997-10-27||Economic characteristics see earlier jobs. Specific lives carry once. Essentia|8.05|6.68|5002002|importoscholar #2|2|country|5|Music|137|ationpriought|N/A|08140066red352575236|violet|Gross|Unknown|24|n stantibarationought| +17060|AAAAAAAAEKCEAAAA|1997-10-27|2000-10-26|To and fro english officers used to judge there coun|0.12|0.08|3001001|amalgexporti #1|1|newborn|3|Children|846|callyeseeing|medium|17chiffon09737623622|spring|Dozen|Unknown|77|barcallybarationought| +17061|AAAAAAAAEKCEAAAA|2000-10-27||Labour, other years cut deliberately at the eyes. Anxious preferences could name there into a parents. Standard criteria support at a chapters. Penalties will see; large skills can suspend as on|9.97|4.48|2003002|exportiimporto #2|3|pants|2|Men|846|callyeseeing|extra large|17chiffon09737623622|tomato|N/A|Unknown|9|oughtcallybarationought| +17062|AAAAAAAAGKCEAAAA|1997-10-27|1999-10-27|Rarely great police lose even about wrong ways. Large shoes ought to learn as tiny days. Channels could not intend with the institutions. Etc free personnel lose heavy, little purpos|3.24|1.29|1003001|exportiamalg #1|3|maternity|1|Women|113|prioughtought|small|8534101thistle805057|saddle|Tsp|Unknown|79|ablecallybarationought| +17063|AAAAAAAAGKCEAAAA|1999-10-28|2001-10-26|Other, likely incentives appeal obviously according to the patients. Detailed interests will not work perhaps. Able |1.83|0.73|1003001|edu packnameless #4|4|camping|8|Sports|344|prioughtought|N/A|754889turquoise23734|linen|Lb|Unknown|18|pricallybarationought| +17064|AAAAAAAAGKCEAAAA|2001-10-27||Now fair reports should take most cars. Open personnel find probably happy practices; possible, different eyes raise therefore particular, little parties. Soo|2.46|1.47|8012005|importomaxi #5|12|guns|8|Sports|616|prioughtought|N/A|754889turquoise23734|snow|Cup|Unknown|57|esecallybarationought| +17065|AAAAAAAAJKCEAAAA|1997-10-27||Empty, comfortable trousers may meet straight in a officers. Parents challenge then. Top, unlikely ways investigate potential details. Complex sites follow|2.96|2.51|5004002|edu packscholar #2|4|classical|5|Music|89|n steing|N/A|27878powder161860062|grey|Dozen|Unknown|23|anticallybarationought| +17066|AAAAAAAAKKCEAAAA||2000-10-26|||0.34|8014007||14|tennis||Sports|315|antioughtpri||457593sandy274087400|lawn|N/A|||callycallybarationought| +17067|AAAAAAAAKKCEAAAA|2000-10-27||Rates should go at least only claims. Right musical books refer clear women. Again mysterious |3.65|3.17|2001002|amalgimporto #2|14|accessories|2|Men|315|antioughtpri|medium|457593sandy274087400|metallic|Unknown|Unknown|52|ationcallybarationought| +17068|AAAAAAAAMKCEAAAA|1997-10-27|1999-10-27|So small heads ought to help parents. Second|9.32|6.52|7003009|exportibrand #9|3|kids|7|Home|958|eingantin st|N/A|614675546844peru5486|slate|Oz|Unknown|2|eingcallybarationought| +17069|AAAAAAAAMKCEAAAA|1999-10-28|2001-10-26|So small heads ought to help parents. Second|3.35|6.52|4003002|exportiedu pack #2|3|kids|4|Shoes|958|eingantin st|economy|110steel508632251358|linen|Case|Unknown|46|n stcallybarationought| +17070|AAAAAAAAMKCEAAAA|2001-10-27||So small heads ought to help parents. Second|60.11|46.88|8012001|importomaxi #1|3|guns|8|Sports|564|esecallyanti|N/A|110steel508632251358|honeydew|N/A|Unknown|17|barationbarationought| +17071|AAAAAAAAPKCEAAAA|1997-10-27||Real areas might tackle brief, exceptional points. Conditions u|6.16|2.95|10012004|importoamalgamalg #4|12|monitors|10|Electronics|26|callyable|N/A|0026153375tomato2303|tomato|Bunch|Unknown|1|oughtationbarationought| +17072|AAAAAAAAALCEAAAA|1997-10-27|2000-10-26|Conditions ought to need specially with a eyes. Large, african pounds appreciate chinese, only items; old, special uses examine|1.08|0.44|5001001|amalgscholar #1|1|rock|5|Music|276|callyationable|N/A|black041243430624193|smoke|Gross|Unknown|56|ableationbarationought| +17073|AAAAAAAAALCEAAAA|2000-10-27||Different, soft farms meet on the implications. Even dark stations pay all rarely nice facts; daughters may |4.50|0.44|5001001|importoexporti #2|2|infants|3|Children|477|ationationese|medium|black041243430624193|spring|Bunch|Unknown|33|priationbarationought| +17074|AAAAAAAACLCEAAAA|1997-10-27|1999-10-27|Cars take less geographical objects. Kids give obvious eyes|8.81|3.17|3004001|edu packexporti #1|4|school-uniforms|3|Children|57|ationanti|large|83603494salmon682809|maroon|Box|Unknown|17|eseationbarationought| +17075|AAAAAAAACLCEAAAA|1999-10-28|2001-10-26|Cars take less geographical objects. Kids give obvious eyes|6.17|3.17|3004001|edu packamalg #2|4|swimwear|1|Women|57|ationanti|medium|2238021moccasin86760|purple|Case|Unknown|98|antiationbarationought| +17076|AAAAAAAACLCEAAAA|2001-10-27||Permanent users make other, new eyes. More new numbers cannot |2.43|0.72|7011007|amalgnameless #7|11|accent|7|Home|142|ableeseought|N/A|5122ivory01575871676|lime|Pound|Unknown|8|callyationbarationought| +17077|AAAAAAAAFLCEAAAA|1997-10-27||Soviet ships will perform partly. Responses like already historical years. So respo|6.42|2.43|7015008|scholarnameless #8|15|tables|7|Home|5|anti|N/A|949587593tan92635734|white|Cup|Unknown|31|ationationbarationought| +17078|AAAAAAAAGLCEAAAA|1997-10-27|2000-10-26|Times may serve a.d. significant companies. Immediate families |8.98|4.84|10004011|edu packunivamalg #11|4|audio|10|Electronics|189|n steingought|N/A|72554mint71661534507|mint|Gross|Unknown|94|eingationbarationought| +17079|AAAAAAAAGLCEAAAA|2000-10-27||Times may serve a.d. significant companies. Immediate families |8.14|7.08|10012012|importoamalgamalg #12|4|monitors|10|Electronics|189|n steingought|N/A|11mint76592868118926|linen|Oz|Unknown|5|n stationbarationought| +17080|AAAAAAAAILCEAAAA|1997-10-27|1999-10-27|Multiple, left attempts build yet views. Criminal skills review ea|0.88|0.44|1001001|amalgamalg #1|1|dresses|1|Women|180|bareingought|extra large|010536597756red09763|peach|Dozen|Unknown|7|bareingbarationought| +17081|AAAAAAAAILCEAAAA|1999-10-28|2001-10-26|Long immediate costs reduce above parental soldiers. Lives sustain catholic, british wat|64.16|0.44|2001002|amalgimporto #2|1|accessories|2|Men|481|oughteingese|large|96479222peru21537030|slate|Tbl|Unknown|53|oughteingbarationought| +17082|AAAAAAAAILCEAAAA|2001-10-27||Long immediate costs reduce above parental soldiers. Lives sustain catholic, british wat|2.00|1.32|5004001|edu packscholar #1|4|classical|5|Music|481|oughteingese|N/A|100214706940228puff4|steel|Bunch|Unknown|2|ableeingbarationought| +17083|AAAAAAAALLCEAAAA|1997-10-27||Peaceful, realistic lips withstand public, clear problems. Conferences cannot educate more hardly different fingers. |4.35|1.52|1002002|importoamalg #2|2|fragrances|1|Women|397|ationn stpri|medium|0663spring7265291049|magenta|Unknown|Unknown|4|prieingbarationought| +17084|AAAAAAAAMLCEAAAA|1997-10-27|2000-10-26|Then strategic things help stiff main participants. Values would speak really with the camps; roman, old interests reflect all horses. Important, square yards may explain independent programmes|83.23|65.75|8011005|amalgmaxi #5|11|archery|8|Sports|640|baresecally|N/A|270273aquamarine0447|sky|Cup|Unknown|13|eseeingbarationought| +17085|AAAAAAAAMLCEAAAA|2000-10-27||Beautiful, va|3.25|2.69|3001002|amalgexporti #2|11|newborn|3|Children|640|baresecally|medium|270273aquamarine0447|ivory|Cup|Unknown|89|antieingbarationought| +17086|AAAAAAAAOLCEAAAA|1997-10-27|1999-10-27|Final years ask perfectly reasons. Maybe careful winners allow simply hard days. Businesses excite always premier pieces; concepts|4.35|1.69|3004001|edu packexporti #1|4|school-uniforms|3|Children|75|antiation|small|6713795789764violet3|powder|Ounce|Unknown|26|callyeingbarationought| +17087|AAAAAAAAOLCEAAAA|1999-10-28|2001-10-26|Grounds se|3.79|2.91|3004001|edu packscholar #2|4|classical|5|Music|18|antiation|N/A|17267783salmon335372|misty|Cup|Unknown|16|ationeingbarationought| +17088|AAAAAAAAOLCEAAAA|2001-10-27||Grounds se|5.53|2.91|5003001|exportischolar #1|4|pop|5|Music|859|antiation|N/A|17267783salmon335372|royal|Carton|Unknown|25|eingeingbarationought| +17089|AAAAAAAABMCEAAAA|1997-10-27||Dangerous,|2.32|1.18|1003002|exportiamalg #2|3|maternity|1|Women|357|ationantipri|large|98031912maroon460401|purple|Gross|Unknown|20|n steingbarationought| +17090|AAAAAAAACMCEAAAA|1997-10-27|2000-10-26|Available, sure|0.21|0.11|2001001|amalgimporto #1|1|accessories|2|Men|248|eingeseable|petite|0red1183187006976410|honeydew|Box|Unknown|58|barn stbarationought| +17091|AAAAAAAACMCEAAAA|2000-10-27||Rather whole others may sit anyway services. Hands might bring ever areas. Of course marked tories could not chat however new traditional years. Similarly confidential groups put |19.61|0.11|2001001|importoedu pack #2|2|mens|4|Shoes|248|eingeseable|petite|0red1183187006976410|steel|Tbl|Unknown|17|oughtn stbarationought| +17092|AAAAAAAAEMCEAAAA|1997-10-27|1999-10-27|Patterns prevent out of the members. Shows should guarantee therefore stairs. Paths see on to a firms; as base rules might not kick working-cla|9.97|7.37|1003001|exportiamalg #1|3|maternity|1|Women|449|n steseese|economy|521164658772puff8971|lavender|Pallet|Unknown|34|ablen stbarationought| +17093|AAAAAAAAEMCEAAAA|1999-10-28|2001-10-26|Patterns prevent out of the members. Shows should guarantee therefore stairs. Paths see on to a firms; as base rules might not kick working-cla|4.56|2.37|1003001|edu packmaxi #6|3|entertainments|9|Books|449|n steseese|N/A|92692306074267brown9|yellow|Case|Unknown|14|prin stbarationought| +17094|AAAAAAAAEMCEAAAA|2001-10-27||Patterns prevent out of the members. Shows should guarantee therefore stairs. Paths see on to a firms; as base rules might not kick working-cla|6.02|2.37|1003001|edu packexporti #1|3|school-uniforms|3|Children|173|n steseese|petite|558279770635spring59|steel|Each|Unknown|38|esen stbarationought| +17095|AAAAAAAAHMCEAAAA|1997-10-27||Ordinary orders can inspect. New int|9.58|3.92|4003002|exportiedu pack #2|3|kids|4|Shoes|608|eingbarcally|economy|21043026pale31276318|firebrick|Lb|Unknown|23|antin stbarationought| +17096|AAAAAAAAIMCEAAAA|1997-10-27|2000-10-26|Relationships can endeavour also other ways. Vast, good sites should arrange. Soon commercial eyes must give as well other available directors. Som|7.37|5.74|4003001|exportiedu pack #1|3|kids|4|Shoes|151|oughtantiought|extra large|20608766861pale47433|gainsboro|Oz|Unknown|40|callyn stbarationought| +17097|AAAAAAAAIMCEAAAA|2000-10-27||Local, labour police would not get. So significant employees give perhaps walls; arrangements calculate sure. Single, intermediate workers could not exercise real|5.57|5.74|4003001|maxiunivamalg #13|3|televisions|10|Electronics|151|oughtantiought|N/A|20608766861pale47433|plum|Cup|Unknown|30|ationn stbarationought| +17098|AAAAAAAAKMCEAAAA|1997-10-27|1999-10-27|Alternatives operate pe|9.87|3.05|6003003|exporticorp #3|3|gold|6|Jewelry|670|barationcally|N/A|69510498792091white5|white|Cup|Unknown|64|eingn stbarationought| +17099|AAAAAAAAKMCEAAAA|1999-10-28|2001-10-26|Friends map well from a ears. Everyday, basic advertisements shall see attra|6.84|3.05|8012010|importomaxi #10|3|guns|8|Sports|1|barationcally|N/A|69510498792091white5|peach|Cup|Unknown|99|n stn stbarationought| +17100|AAAAAAAAKMCEAAAA|2001-10-27||Patients satisfy promptly crucial statements. Hardly particular legs pick countries. Especially preliminary speakers would speak about a men. Political, local samples shall ma|3.45|3.05|1004001|edu packamalg #1|4|swimwear|1|Women|254|barationcally|large|69510498792091white5|pink|Lb|Unknown|51|barbaroughtationought| +17101|AAAAAAAANMCEAAAA|1997-10-27||Forces shall not operate to the groups. Fresh methods could achieve enough on a feet. Right, inc rights survive then economic legs. Blind birds say quiet ca|4.62|3.23|4002002|importoedu pack #2|2|mens|4|Shoes|417|ationoughtese|large|337thistle2933211140|pale|Pallet|Unknown|19|oughtbaroughtationought| +17102|AAAAAAAAOMCEAAAA|1997-10-27|2000-10-26|Subject sisters recognise happily reports; proper, known years investigate really doctors. Ever bi|6.59|3.03|6001007|amalgcorp #7|1|birdal|6|Jewelry|239|n stpriable|N/A|1342068turquoise6070|violet|Oz|Unknown|31|ablebaroughtationought| +17103|AAAAAAAAOMCEAAAA|2000-10-27||Single, othe|97.38|46.74|6001007|namelessmaxi #4|8|romance|9|Books|278|eingationable|N/A|065sandy130152590595|steel|Tbl|Unknown|19|pribaroughtationought| +17104|AAAAAAAAANCEAAAA|1997-10-27|1999-10-27|Merely organic terms shall outline high alone companies; small problems take by a lives; |4.10|3.15|3003001|exportiexporti #1|3|toddlers|3|Children|321|oughtablepri|large|746moccasin447918393|wheat|Ounce|Unknown|5|esebaroughtationought| +17105|AAAAAAAAANCEAAAA|1999-10-28|2001-10-26|Safe pages get extraordinarily mentally able others. Social, differ|28.75|3.15|4003002|exportiedu pack #2|3|kids|4|Shoes|509|oughtablepri|medium|435115violet08449329|royal|Tbl|Unknown|52|antibaroughtationought| +17106|AAAAAAAAANCEAAAA|2001-10-27||Central others should put limited women. Similar hours go often essential, japanese re|1.10|0.69|4003002|scholarunivamalg #7|3|fiction|9|Books|509|oughtablepri|N/A|17921201steel1441634|magenta|Unknown|Unknown|26|callybaroughtationought| +17107|AAAAAAAADNCEAAAA|1997-10-27||Courses take very jobs. Different, gold losses note fu|3.34|2.97|3001002|amalgexporti #2|1|newborn|3|Children|96|callyn st|medium|53952732puff68662202|salmon|Pound|Unknown|81|ationbaroughtationought| +17108|AAAAAAAAENCEAAAA|1997-10-27|2000-10-26|Services go certain beans. Away american words lose quickly powerful skills. Certainly physical films would turn rather later central miles; great governments re|0.71|0.50|8016009|corpmaxi #9|16|golf|8|Sports|96|callyn st|N/A|474metallic549841651|midnight|Ounce|Unknown|65|eingbaroughtationought| +17109|AAAAAAAAENCEAAAA|2000-10-27||Services go certain beans. Away american words lose quickly powerful skills. Certainly physical films would turn rather later central miles; great governments re|1.36|0.50|8016009|namelesscorp #6|8|mens watch|6|Jewelry|469|callyn st|N/A|59849firebrick833623|brown|Each|Unknown|93|n stbaroughtationought| +17110|AAAAAAAAGNCEAAAA|1997-10-27|1999-10-27|Child|4.93|3.00|2003001|exportiimporto #1|3|pants|2|Men|70|baration|small|8866968257metallic69|saddle|Pallet|Unknown|30|baroughtoughtationought| +17111|AAAAAAAAGNCEAAAA|1999-10-28|2001-10-26|Child|9.70|6.88|2003001|amalgedu pack #2|3|womens|4|Shoes|46|callyese|small|8866968257metallic69|peach|Each|Unknown|4|oughtoughtoughtationought| +17112|AAAAAAAAGNCEAAAA|2001-10-27||Child|7.24|6.88|2003001|exportiedu pack #1|3|kids|4|Shoes|59|n stanti|large|0seashell74619113703|thistle|Cup|Unknown|30|ableoughtoughtationought| +17113|AAAAAAAAJNCEAAAA|1997-10-27||Police accompany powerful things. Already other chains get indeed more proper shows. Other, very lines d|0.21|0.10|4004002|edu packedu pack #2|4|athletic|4|Shoes|758|eingantiation|economy|116722561461thistle8|peach|Dozen|Unknown|21|prioughtoughtationought| +17114|AAAAAAAAKNCEAAAA|1997-10-27|2000-10-26|Different, adult bonds affect countries. Recordi|5.57|3.06|2001001|amalgimporto #1|1|accessories|2|Men|207|ationbarable|medium|58papaya726018820786|slate|Pound|Unknown|26|eseoughtoughtationought| +17115|AAAAAAAAKNCEAAAA|2000-10-27||Different, adult bonds affect countries. Recordi|24.87|3.06|2001001|exportiimporto #2|3|pants|2|Men|207|ationbarable|large|59rosy08639285353133|powder|Pallet|Unknown|60|antioughtoughtationought| +17116|AAAAAAAAMNCEAAAA|1997-10-27|1999-10-27|Pleasant, small estates should head so criminal rights. Professional, particular organi|2.35|0.84|5004001|edu packscholar #1|4|classical|5|Music|928|eingablen st|N/A|7cornflower254398670|wheat|Pallet|Unknown|28|callyoughtoughtationought| +17117|AAAAAAAAMNCEAAAA|1999-10-28|2001-10-26|Pleasant, small estates should head so criminal rights. Professional, particular organi|1.01|0.50|1003002|exportiamalg #2|3|maternity|1|Women|449|eingablen st|large|7cornflower254398670|sky|Each|Unknown|42|ationoughtoughtationought| +17118|AAAAAAAAMNCEAAAA|2001-10-27||Streets might not hit. Immediate, social readers see after a pockets. Old days produce chips. Other factors d|3.67|0.50|1003002|exportimaxi #7|3|computers|9|Books|568|eingablen st|N/A|7cornflower254398670|violet|Ton|Unknown|60|eingoughtoughtationought| +17119|AAAAAAAAPNCEAAAA|1997-10-27||Steady pieces like; circumstances shall think then on a instruments. Highly other students conclude simultaneously in a trees. Figures contend in the schools. Projects continue laws. Suitable questio|8.48|5.68|1003002|exportiamalg #2|3|maternity|1|Women|262|ablecallyable|medium|4988637511slate38863|pale|Dozen|Unknown|12|n stoughtoughtationought| +17120|AAAAAAAAAOCEAAAA|1997-10-27|2000-10-26|Cold, previous women catch m|6.94|3.81|2004001|edu packimporto #1|4|sports-apparel|2|Men|650|barantically|medium|60621638chocolate246|seashell|N/A|Unknown|15|barableoughtationought| +17121|AAAAAAAAAOCEAAAA|2000-10-27||Religious, new reductions ought to affect for a commitments. Current, responsible measures help hardly. R|9.56|3.81|7007002|brandbrand #2|4|decor|7|Home|650|barantically|N/A|4947steel43317298263|pink|Unknown|Unknown|10|oughtableoughtationought| +17122|AAAAAAAACOCEAAAA|1997-10-27|1999-10-27|Old, statutory prob|4.48|3.44|3004001|edu packexporti #1|4|school-uniforms|3|Children|13|priought|large|5lace877506008978559|khaki|Box|Unknown|35|ableableoughtationought| +17123|AAAAAAAACOCEAAAA|1999-10-28|2001-10-26|Old, statutory prob|6.49|2.59|10008005|namelessunivamalg #5|8|scanners|10|Electronics|13|priought|N/A|5lace877506008978559|tan|Bunch|Unknown|32|priableoughtationought| +17124|AAAAAAAACOCEAAAA|2001-10-27||Old, statutory prob|0.95|0.46|10008005|exportiedu pack #1|8|kids|4|Shoes|593|priought|petite|040221572326lemon594|wheat|Case|Unknown|12|eseableoughtationought| +17125|AAAAAAAAFOCEAAAA|1997-10-27||Very only cases help. Mere, dangerous figures could not note quickly political wea|1.92|0.82|9015002|scholarunivamalg #2|15|fiction|9|Books|39|n stpri|N/A|49196steel1556280043|slate|Case|Unknown|27|antiableoughtationought| +17126|AAAAAAAAGOCEAAAA|1997-10-27|2000-10-26|Everyday volumes ma|3.67|2.64|1003001|exportiamalg #1|3|maternity|1|Women|181|oughteingought|medium|9036khaki98808015471|forest|Bunch|Unknown|11|callyableoughtationought| +17127|AAAAAAAAGOCEAAAA|2000-10-27||Strong words used to live women; rules reduce thus at a conditions; able positions must contain so from a competitors. Happy premises might help once again emotions. Less religious threat|52.28|21.43|1003001|exportiedu pack #2|3|kids|4|Shoes|181|oughteingought|petite|0peach85600305012602|purple|Case|Unknown|21|ationableoughtationought| +17128|AAAAAAAAIOCEAAAA|1997-10-27|1999-10-27|Additional hours will put. Beneficial communities would not talk probably controls. At least bare links serve very details. Years would get commonly particular, sure aims. Current,|2.03|1.66|4003001|exportiedu pack #1|3|kids|4|Shoes|237|ationpriable|medium|01967714peach6286123|midnight|Ounce|Unknown|15|eingableoughtationought| +17129|AAAAAAAAIOCEAAAA|1999-10-28|2001-10-26|Additional hours will put. Beneficial communities would not talk probably controls. At least bare links serve very details. Years would get commonly particular, sure aims. Current,|5.79|1.66|10001011|amalgunivamalg #11|1|cameras|10|Electronics|237|ationpriable|N/A|666038peach939455174|wheat|Ounce|Unknown|51|n stableoughtationought| +17130|AAAAAAAAIOCEAAAA|2001-10-27||Quickly pale cases must not sing at all beautifully good men; please ruling adults used to follow more than suddenly powerful issues. Recent characteristics perce|1.93|0.81|5003001|exportischolar #1|3|pop|5|Music|237|ationpriable|N/A|96211322mint33135099|slate|Case|Unknown|69|barprioughtationought| +17131|AAAAAAAALOCEAAAA|1997-10-27||Public, crucial institutions get. Years could materialise. Nice plans involve; details must not see about a sounds. Very medical activities may remain offices. Yet high lovers carry only future p|29.87|14.03|9012008|importounivamalg #8|12|home repair|9|Books|630|barprically|N/A|46pink71478069657420|burnished|Cup|Unknown|91|oughtprioughtationought| +17132|AAAAAAAAMOCEAAAA|1997-10-27|2000-10-26|Never subject chapters may not arrive years. Pretty, free men must not come highly current eyes. Vastly medical tonnes could not cove|1.39|1.01|1004001|edu packamalg #1|4|swimwear|1|Women|420|barableese|extra large|3713424244053saddle6|navy|Gross|Unknown|23|ableprioughtationought| +17133|AAAAAAAAMOCEAAAA|2000-10-27||Properly old papers shall get head streets. Hard low|1.86|1.01|1004001|univnameless #4|4|flatware|7|Home|1|ought|N/A|920514151papaya76240|rosy|Pound|Unknown|25|priprioughtationought| +17134|AAAAAAAAOOCEAAAA|1997-10-27|1999-10-27|Sadly similar dogs should feed recently. Other achievements con|93.10|35.37|4001001|amalgedu pack #1|1|womens|4|Shoes|48|eingese|extra large|white548116621505118|brown|Unknown|Unknown|92|eseprioughtationought| +17135|AAAAAAAAOOCEAAAA|1999-10-28|2001-10-26|Early components require; de|9.31|7.82|4001001|brandmaxi #12|1|reference|9|Books|48|eingese|N/A|white548116621505118|black|Case|Unknown|14|antiprioughtationought| +17136|AAAAAAAAOOCEAAAA|2001-10-27||Early components require; de|33.90|7.82|2003001|exportiimporto #1|1|pants|2|Men|40|eingese|medium|white548116621505118|plum|Unknown|Unknown|31|callyprioughtationought| +17137|AAAAAAAABPCEAAAA|1997-10-27||Exhibitions must not get calmly for a reservations. Nuclear, horizontal feet mean only later royal reforms; firmly likely eyes enter always. Young, violent visitors might let quickly cases. Practical|3.76|2.40|10008002|namelessunivamalg #2|8|scanners|10|Electronics|538|eingprianti|N/A|magenta5029532226310|ghost|Pallet|Unknown|64|ationprioughtationought| +17138|AAAAAAAACPCEAAAA|1997-10-27|2000-10-26|Large groups should pay big, particular results. Tiny police will watch white, right clothes. Res|29.61|16.58|6009003|maxicorp #3|9|womens watch|6|Jewelry|364|esecallypri|N/A|825625891569papaya80|steel|Lb|Unknown|9|eingprioughtationought| +17139|AAAAAAAACPCEAAAA|2000-10-27||Large groups should pay big, particular results. Tiny police will watch white, right clothes. Res|0.33|16.58|6009003|importounivamalg #4|2|camcorders|10|Electronics|13|esecallypri|N/A|765504snow6650077362|white|Gram|Unknown|44|n stprioughtationought| +17140|AAAAAAAAEPCEAAAA|1997-10-27|1999-10-27|Results decide hence eventually economic races. American, underlying tourists shall secure too adult sig|64.31|36.65|8016001|corpmaxi #1|16|golf|8|Sports|149|n steseought|N/A|27009462066252grey98|peru|Lb|Unknown|29|bareseoughtationought| +17141|AAAAAAAAEPCEAAAA|1999-10-28|2001-10-26|Results decide hence eventually economic races. American, underlying tourists shall secure too adult sig|4.65|36.65|8016001|univnameless #2|10|flatware|7|Home|253|n steseought|N/A|27009462066252grey98|beige|Dram|Unknown|2|oughteseoughtationought| +17142|AAAAAAAAEPCEAAAA|2001-10-27||Ancient products achieve very large mothers. Obviously brown colours used to use thus. Legal policies work no doubt. Married reforms shall distin|5.85|36.65|8016001|edu packscholar #1|4|classical|5|Music|82|ableeing|N/A|27009462066252grey98|spring|Dozen|Unknown|7|ableeseoughtationought| +17143|AAAAAAAAHPCEAAAA|1997-10-27||Deep, large sides boost policies; so fixed charts c|5.77|4.55|5004002|edu packscholar #2|4|classical|5|Music|547|ationeseanti|N/A|52117488278honeydew6|yellow|Oz|Unknown|7|prieseoughtationought| +17144|AAAAAAAAIPCEAAAA|1997-10-27|2000-10-26|Short working places might w|1.12|0.49|9016009|corpunivamalg #9|16|mystery|9|Books|46|callyese|N/A|9snow316959574122090|royal|Bunch|Unknown|89|eseeseoughtationought| +17145|AAAAAAAAIPCEAAAA|2000-10-27||Systems used to tell new, very weapons; vulnerable effor|9.10|0.49|9016009|exportimaxi #6|16|sailing|8|Sports|46|callyese|N/A|9snow316959574122090|yellow|Ton|Unknown|30|antieseoughtationought| +17146|AAAAAAAAKPCEAAAA|1997-10-27|1999-10-27|New, causal hands give so then ideal articles. Complex companies would not look so vast characteristics. Always modern changes shift in a men. Simply average eyes should not make scots. |1.98|1.62|6001003|amalgcorp #3|1|birdal|6|Jewelry|7|ation|N/A|166gainsboro06321498|rose|Dozen|Unknown|3|callyeseoughtationought| +17147|AAAAAAAAKPCEAAAA|1999-10-28|2001-10-26|New, causal hands give so then ideal articles. Complex companies would not look so vast characteristics. Always modern changes shift in a men. Simply average eyes should not make scots. |1.25|0.73|6001003|amalgscholar #2|1|rock|5|Music|426|callyableese|N/A|32302turquoise829485|lace|Tsp|Unknown|5|ationeseoughtationought| +17148|AAAAAAAAKPCEAAAA|2001-10-27||Local men consider dif|1.97|0.73|6001003|edu packamalg #1|1|swimwear|1|Women|172|callyableese|large|48111plum40639428195|powder|Gram|Unknown|22|eingeseoughtationought| +17149|AAAAAAAANPCEAAAA|1997-10-27||Full-time clothes discharge glad, concerned details. Customs must survive|8.52|6.73|10003014|exportiunivamalg #14|3|dvd/vcr players|10|Electronics|25|antiable|N/A|45197711lavender6618|snow|Dram|Unknown|32|n steseoughtationought| +17150|AAAAAAAAOPCEAAAA|1997-10-27|2000-10-26|Valid friends place nuclear, extreme organisations. Automatic|79.10|30.84|5002001|importoscholar #1|2|country|5|Music|385|antieingpri|N/A|6rosy548796072254560|red|Dram|Unknown|20|barantioughtationought| +17151|AAAAAAAAOPCEAAAA|2000-10-27||Valid friends place nuclear, extreme organisations. Automatic|1.83|30.84|5002001|exportiexporti #2|3|toddlers|3|Children|626|callyablecally|small|64azure1533814598445|misty|Tbl|Unknown|79|oughtantioughtationought| +17152|AAAAAAAAAADEAAAA|1997-10-27|1999-10-27|Groups may wear never behind a centres. Major forces may not destroy activities. Single pieces would expect m|65.88|43.48|6008001|namelesscorp #1|8|mens watch|6|Jewelry|59|n stanti|N/A|879667306peach487677|bisque|Dozen|Unknown|55|ableantioughtationought| +17153|AAAAAAAAAADEAAAA|1999-10-28|2001-10-26|Groups may wear never behind a centres. Major forces may not destroy activities. Single pieces would expect m|84.44|43.06|6008001|edu packscholar #2|4|classical|5|Music|610|baroughtcally|N/A|879667306peach487677|salmon|Tsp|Unknown|61|priantioughtationought| +17154|AAAAAAAAAADEAAAA|2001-10-27||Groups may wear never behind a centres. Major forces may not destroy activities. Single pieces would expect m|3.29|43.06|6008001|brandmaxi #7|4|reference|9|Books|610|baroughtcally|N/A|18seashell5237450733|white|Dozen|Unknown|29|eseantioughtationought| +17155|AAAAAAAADADEAAAA|1997-10-27||Obviously international officers may view moreover individual copies. Invisible, distant forces may not throw from time to time civil days. Endless months minimis|4.47|1.78|5001002|amalgscholar #2|1|rock|5|Music|337|ationpripri|N/A|204indian95251741627|khaki|Box|Unknown|60|antiantioughtationought| +17156|AAAAAAAAEADEAAAA|1997-10-27|2000-10-26|Ships should help private, possible societies. C|8.10|6.48|9012009|importounivamalg #9|12|home repair|9|Books|197|ationn stought|N/A|9980830497salmon2856|papaya|Case|Unknown|31|callyantioughtationought| +17157|AAAAAAAAEADEAAAA|2000-10-27||Ships should help private, possible societies. C|5.89|6.48|9012009|exportiamalg #2|3|maternity|1|Women|197|ationn stought|petite|63lavender1240575299|grey|Pallet|Unknown|7|ationantioughtationought| +17158|AAAAAAAAGADEAAAA|1997-10-27|1999-10-27|Girls may use chri|4.37|3.23|9011011|amalgunivamalg #11|11|cooking|9|Books|150|barantiought|N/A|285021334thistle8271|sienna|Cup|Unknown|40|eingantioughtationought| +17159|AAAAAAAAGADEAAAA|1999-10-28|2001-10-26|Appropria|98.47|80.74|2003002|exportiimporto #2|11|pants|2|Men|333|pripripri|small|357202045aquamarine7|steel|Pound|Unknown|78|n stantioughtationought| +17160|AAAAAAAAGADEAAAA|2001-10-27||Appropria|2.08|80.74|5001001|amalgscholar #1|1|rock|5|Music|773|pripripri|N/A|357202045aquamarine7|khaki|Pallet|Unknown|1|barcallyoughtationought| +17161|AAAAAAAAJADEAAAA|1997-10-27||Also special children shall unite instructions. Too pure dealers might not find very convenient conventions. Necessary needs must work even fun oth|3.26|2.64|4004002|edu packedu pack #2|4|athletic|4|Shoes|272|ableationable|N/A|39484084486353rosy43|peru|Case|Unknown|59|oughtcallyoughtationought| +17162|AAAAAAAAKADEAAAA|1997-10-27|2000-10-26|Deliberately significant customers |1.90|1.04|4002001|importoedu pack #1|2|mens|4|Shoes|178|eingationought|extra large|51341205767seashell0|papaya|Bunch|Unknown|12|ablecallyoughtationought| +17163|AAAAAAAAKADEAAAA|2000-10-27||Critics might press about initial strangers. Forms pick probably on the users. High causal c|0.18|1.04|4002001|amalgscholar #2|2|rock|5|Music|178|eingationought|N/A|557050485metallic961|plum|Each|Unknown|53|pricallyoughtationought| +17164|AAAAAAAAMADEAAAA|1997-10-27|1999-10-27|Arguments could not include just arab, general relationships; particular matters |0.84|0.47|1001001|amalgamalg #1|1|dresses|1|Women|330|barpripri|extra large|528868thistle7222484|white|Unknown|Unknown|4|esecallyoughtationought| +17165|AAAAAAAAMADEAAAA|1999-10-28|2001-10-26|Arguments could not include just arab, general relationships; particular matters |7.39|2.58|1001001|scholaramalgamalg #13|1|portable|10|Electronics|161|barpripri|N/A|528868thistle7222484|rosy|Bunch|Unknown|14|anticallyoughtationought| +17166|AAAAAAAAMADEAAAA|2001-10-27||Const|8.21|2.58|1001001|exportimaxi #7|3|computers|9|Books|213|prioughtable|N/A|528868thistle7222484|turquoise|Cup|Unknown|35|callycallyoughtationought| +17167|AAAAAAAAPADEAAAA|1997-10-27||Goals should not make in|4.09|1.79|9003008|exportimaxi #8|3|computers|9|Books|167|ationcallyought|N/A|9959241100sienna6576|lime|Each|Unknown|92|ationcallyoughtationought| +17168|AAAAAAAAABDEAAAA|1997-10-27|2000-10-26|Indeed similar feel|3.48|1.49|2002001|importoimporto #1|2|shirts|2|Men|196|callyn stought|medium|71430900241smoke7760|turquoise|Dram|Unknown|39|eingcallyoughtationought| +17169|AAAAAAAAABDEAAAA|2000-10-27||Other, black places may not justify other firms. Great, black deposits care much parties. More available developments ought to exist inst|0.31|0.12|2002001|importoscholar #2|2|country|5|Music|196|callyn stought|N/A|71430900241smoke7760|peach|Gross|Unknown|16|n stcallyoughtationought| +17170|AAAAAAAACBDEAAAA|1997-10-27|1999-10-27|Technical, desirable benefits hear much from the blocks. Gastric vehicles used to give strong, other rights. Always big amo|1.39|0.87|5003001|exportischolar #1|3|pop|5|Music|79|n station|N/A|3143031drab471389133|lace|Case|Unknown|63|barationoughtationought| +17171|AAAAAAAACBDEAAAA|1999-10-28|2001-10-26|Technical, desirable benefits hear much from the blocks. Gastric vehicles used to give strong, other rights. Always big amo|31.46|13.52|6014004|edu packbrand #4|3|estate|6|Jewelry|275|n station|N/A|3143031drab471389133|sienna|Bundle|Unknown|45|oughtationoughtationought| +17172|AAAAAAAACBDEAAAA|2001-10-27||Differences may matter native, certain contracts. British weeks may not go women. Correct, comparative details would marry|4.82|3.18|6014004|amalgimporto #1|3|accessories|2|Men|74|n station|small|3143031drab471389133|sky|Ounce|Unknown|27|ableationoughtationought| +17173|AAAAAAAAFBDEAAAA|1997-10-27||Present|2.50|2.00|6012006|importobrand #6|12|costume|6|Jewelry|218|eingoughtable|N/A|720plum8757705569046|medium|Bunch|Unknown|79|priationoughtationought| +17174|AAAAAAAAGBDEAAAA|1997-10-27|2000-10-26|Times shall take previously likely, ultimate goods. Substantial, profound times will not produce sensitive, clear representati|2.57|2.26|2001001|amalgimporto #1|1|accessories|2|Men|439|n stpriese|medium|069white904390544799|snow|Oz|Unknown|28|eseationoughtationought| +17175|AAAAAAAAGBDEAAAA|2000-10-27||Available, entire groups cease still such as a questions. Limited prospects must bring usually just other sides. Often great trades make weeks. Costs require with a |2.77|1.52|2001001|importoexporti #2|1|infants|3|Children|439|n stpriese|large|069white904390544799|pink|Pound|Unknown|19|antiationoughtationought| +17176|AAAAAAAAIBDEAAAA|1997-10-27|1999-10-27|Continental standards will meet often human, entire men. Reliable areas come|2.73|2.40|5003001|exportischolar #1|3|pop|5|Music|193|prin stought|N/A|651773425199grey1622|powder|Ton|Unknown|45|callyationoughtationought| +17177|AAAAAAAAIBDEAAAA|1999-10-28|2001-10-26|Continental standards will meet often human, entire men. Reliable areas come|1.44|2.40|5003001|corpmaxi #6|3|parenting|9|Books|130|prin stought|N/A|651773425199grey1622|green|Pound|Unknown|32|ationationoughtationought| +17178|AAAAAAAAIBDEAAAA|2001-10-27||Good, female forms cannot lose outer, social publications. Necessary funds used t|4.32|1.46|5003001|importoexporti #1|2|infants|3|Children|212|ableoughtable|large|651773425199grey1622|navy|Cup|Unknown|27|eingationoughtationought| +17179|AAAAAAAALBDEAAAA|1997-10-27||Compatible, western governments establish already in a project|7.74|4.02|1001002|amalgamalg #2|1|dresses|1|Women|945|antiesen st|large|8plum422810716317061|royal|Dozen|Unknown|2|n stationoughtationought| +17180|AAAAAAAAMBDEAAAA|1997-10-27|2000-10-26|Direct perfect funds come long easily national problems; traditional men live forward volumes. Alo|2.92|2.07|4001001|amalgedu pack #1|1|womens|4|Shoes|626|callyablecally|large|9669539678red6666983|violet|Pound|Unknown|52|bareingoughtationought| +17181|AAAAAAAAMBDEAAAA|2000-10-27||Direct perfect funds come long easily national problems; traditional men live forward volumes. Alo|5.75|2.07|4001001|importoimporto #2|2|shirts|2|Men|626|callyablecally|medium|9669539678red6666983|smoke|Cup|Unknown|66|oughteingoughtationought| +17182|AAAAAAAAOBDEAAAA|1997-10-27|1999-10-27|Periods must allow late for the centres. Now easy years learn above patients. Purposes counter occasionally busy kids; impossible, true keys must offer really super, available missiles. E|4.49|2.82|6006007|corpcorp #7|6|rings|6|Jewelry|35|antipri|N/A|5512523568511white34|maroon|Cup|Unknown|18|ableeingoughtationought| +17183|AAAAAAAAOBDEAAAA|1999-10-28|2001-10-26|Appeals listen also great, various words. Important, local practices will|5.08|2.28|6006007|univamalgamalg #14|6|memory|10|Electronics|35|antipri|N/A|5512523568511white34|steel|Carton|Unknown|22|prieingoughtationought| +17184|AAAAAAAAOBDEAAAA|2001-10-27||All various schools would check simply social papers. New expectations drop in a plants. Days can render outside personal, central collections. Complete areas want. New, good habits sug|2.50|1.92|6006007|importoedu pack #1|6|mens|4|Shoes|218|eingoughtable|extra large|5512523568511white34|spring|Carton|Unknown|76|eseeingoughtationought| +17185|AAAAAAAABCDEAAAA|1997-10-27||Members must enjoy especially simply left letters. Even nec|76.80|47.61|6007002|brandcorp #2|7|pendants|6|Jewelry|511|oughtoughtanti|N/A|73218246dodger842736|rose|Pound|Unknown|8|antieingoughtationought| +17186|AAAAAAAACCDEAAAA|1997-10-27|2000-10-26|Crucial, familiar positions ought to occupy trees;|8.11|5.83|7014007|edu packnameless #7|14|glassware|7|Home|585|antieinganti|N/A|541359198027263pink3|lavender|Ton|Unknown|13|callyeingoughtationought| +17187|AAAAAAAACCDEAAAA|2000-10-27||Never national months impose far level|8.18|5.48|5001002|amalgscholar #2|1|rock|5|Music|585|antieinganti|N/A|541359198027263pink3|bisque|Dram|Unknown|35|ationeingoughtationought| +17188|AAAAAAAAECDEAAAA|1997-10-27|1999-10-27|Bones join over groups; only military boards see much; better special others will accept. Kilometres check in addition unions. Serious, previous days find once. Delightf|1.08|0.64|8006009|corpnameless #9|6|football|8|Sports|313|prioughtpri|N/A|orchid07370081946167|medium|Case|Unknown|13|eingeingoughtationought| +17189|AAAAAAAAECDEAAAA|1999-10-28|2001-10-26|Bones join over groups; only military boards see much; better special others will accept. Kilometres check in addition unions. Serious, previous days find once. Delightf|4.40|1.80|8006009|edu packamalg #2|4|swimwear|1|Women|313|prioughtpri|small|6653355118wheat89730|purple|Dozen|Unknown|49|n steingoughtationought| +17190|AAAAAAAAECDEAAAA|2001-10-27||Bones join over groups; only military boards see much; better special others will accept. Kilometres check in addition unions. Serious, previous days find once. Delightf|7.05|1.80|8006009|amalgscholar #1|1|rock|5|Music|319|prioughtpri|N/A|6653355118wheat89730|tan|Dram|Unknown|64|barn stoughtationought| +17191|AAAAAAAAHCDEAAAA|1997-10-27||Famous, free cars develop |1.43|0.61|8011002|amalgmaxi #2|11|archery|8|Sports|53|prianti|N/A|92652287253midnight2|turquoise|N/A|Unknown|84|oughtn stoughtationought| +17192|AAAAAAAAICDEAAAA|1997-10-27|2000-10-26|Large, mass clubs give christian, valid applications. Tired signals may analyse on a o|9.94|8.94|3003001|exportiexporti #1|3|toddlers|3|Children|550|barantianti|large|steel478722219638867|turquoise|Tsp|Unknown|7|ablen stoughtationought| +17193|AAAAAAAAICDEAAAA|2000-10-27||Natural, particular words speak dangerous suppliers. Necess|2.82|8.94|3003001|amalgmaxi #10|1|arts|9|Books|175|barantianti|N/A|1075red0521467382195|chocolate|Ton|Unknown|2|prin stoughtationought| +17194|AAAAAAAAKCDEAAAA|1997-10-27|1999-10-27|Light practices shall not get really as the services. So significant plans know so for a programs. Long |7.50|6.52|8005005|scholarnameless #5|5|fitness|8|Sports|110|baroughtought|N/A|8460770indian0731566|thistle|Bunch|Unknown|23|esen stoughtationought| +17195|AAAAAAAAKCDEAAAA|1999-10-28|2001-10-26|Light practices shall not get really as the services. So significant plans know so for a programs. Long |4.11|6.52|8005005|amalgedu pack #2|5|womens|4|Shoes|84|eseeing|petite|584051476lavender555|white|Tbl|Unknown|15|antin stoughtationought| +17196|AAAAAAAAKCDEAAAA|2001-10-27||Furious, interested conditions intervene harder policies. |9.84|3.14|5002001|importoscholar #1|5|country|5|Music|84|eseeing|N/A|71851pink94692456690|white|Carton|Unknown|19|callyn stoughtationought| +17197|AAAAAAAANCDEAAAA|1997-10-27||Modern areas include indeed political children. White, widespread services attend also. Pink boundaries explain early because of a letters. Often assistant men make never pale windows. Then inte|6.20|4.21|4003002|exportiedu pack #2|3|kids|4|Shoes|309|n stbarpri|medium|406916snow7262226473|white|Ounce|Unknown|43|ationn stoughtationought| +17198|AAAAAAAAOCDEAAAA|1997-10-27|2000-10-26|Electrical, warm buildings die; more poor hopes must monitor never evident patients. Heavy issues would identify real, british armies; big, enormous claims lie yet home |5.78|4.73|8013009|exportimaxi #9|13|sailing|8|Sports|25|antiable|N/A|plum8130867505014547|powder|Unknown|Unknown|23|eingn stoughtationought| +17199|AAAAAAAAOCDEAAAA|2000-10-27||Mental items achi|1.34|0.87|8013009|edu packcorp #8|13|bracelets|6|Jewelry|25|antiable|N/A|plum8130867505014547|purple|Lb|Unknown|13|n stn stoughtationought| +17200|AAAAAAAAADDEAAAA|1997-10-27|1999-10-27|Long-term, powerful members help. Thus great standards cannot promote rather essential, average months. Heel|4.39|3.51|5004001|edu packscholar #1|4|classical|5|Music|888|eingeingeing|N/A|034royal404337951447|peach|Unknown|Unknown|83|barbarableationought| +17201|AAAAAAAAADDEAAAA|1999-10-28|2001-10-26|Long-term, powerful members help. Thus great standards cannot promote rather essential, average months. Heel|73.85|3.51|5004001|importoscholar #2|4|country|5|Music|888|eingeingeing|N/A|034royal404337951447|tan|Ounce|Unknown|11|oughtbarableationought| +17202|AAAAAAAAADDEAAAA|2001-10-27||Long-term, powerful members help. Thus great standards cannot promote rather essential, average months. Heel|7.06|6.00|6004003|edu packcorp #3|4|bracelets|6|Jewelry|888|eingeingeing|N/A|034royal404337951447|rosy|Bunch|Unknown|45|ablebarableationought| +17203|AAAAAAAADDDEAAAA|1997-10-27||Male terms may provide laws; friends add truly rare points. Separate, whole hours may change over. Prime interests could not pretend indeed by a goods. Just past countries get how|2.27|1.20|9015008|scholarunivamalg #8|15|fiction|9|Books|789|n steingation|N/A|787267606065rosy4168|black|Bunch|Unknown|50|pribarableationought| +17204|AAAAAAAAEDDEAAAA|1997-10-27|2000-10-26|Areas impose on a places. Agricultural masses might persuade since. Professional, heavy months might not give essential classes. There old homes rise tomorrow exampl|0.48|0.22|4002001|importoedu pack #1|2|mens|4|Shoes|317|ationoughtpri|small|royal911616332106765|spring|Pound|Unknown|54|esebarableationought| +17205|AAAAAAAAEDDEAAAA|2000-10-27||Concerned, full customs will apply yet carefully good addresses; authorities ought to face names. New police know|29.53|23.03|6010006|univbrand #6|10|jewelry boxes|6|Jewelry|915|ationoughtpri|N/A|201medium15550612191|chiffon|Lb|Unknown|18|antibarableationought| +17206|AAAAAAAAGDDEAAAA|1997-10-27|1999-10-27|Medical relations may invade long united breasts. Impossible, relevant newspapers deepen neither.|1.58|0.93|2003001|exportiimporto #1|3|pants|2|Men|424|eseableese|medium|654frosted0705228309|maroon|Oz|Unknown|60|callybarableationought| +17207|AAAAAAAAGDDEAAAA|1999-10-28|2001-10-26|Both objective skills try deep, exceptional members. Engines know so hard implications. Vital, excessive sites may leave |8.10|0.93|3004002|edu packexporti #2|4|school-uniforms|3|Children|424|eseableese|petite|654frosted0705228309|papaya|N/A|Unknown|33|ationbarableationought| +17208|AAAAAAAAGDDEAAAA|2001-10-27||Both objective skills try deep, exceptional members. Engines know so hard implications. Vital, excessive sites may leave |2.96|1.98|7002009|importobrand #9|4|bedding|7|Home|219|eseableese|N/A|7652058salmon6142639|white|Tsp|Unknown|22|eingbarableationought| +17209|AAAAAAAAJDDEAAAA|1997-10-27||Courts happen just important lights. Running, deep results try then. Young eggs keep sexual l|6.68|2.93|10014006|edu packamalgamalg #6|14|automotive|10|Electronics|246|callyeseable|N/A|994629seashell505675|magenta|Dram|Unknown|17|n stbarableationought| +17210|AAAAAAAAKDDEAAAA|1997-10-27|2000-10-26|Forward profound islands cannot create there little girls; demands play as forces; interests shall not seem very certainly wide fans. Formal workers should choose for a cases. Tight,|3.85|2.96|2001001|amalgimporto #1|1|accessories|2|Men|110|baroughtought|medium|48058552244white5546|red|Gram|Unknown|32|baroughtableationought| +17211|AAAAAAAAKDDEAAAA|2000-10-27||Forward profound islands cannot create there little girls; demands play as forces; interests shall not seem very certainly wide fans. Formal workers should choose for a cases. Tight,|0.80|2.96|4002002|importoedu pack #2|1|mens|4|Shoes|110|baroughtought|petite|48058552244white5546|tomato|Gross|Unknown|15|oughtoughtableationought| +17212|AAAAAAAAMDDEAAAA|1997-10-27|1999-10-27|Minutes provide on a letters; paintings might bring now namely long locations; contemporary, commercial words prevent far; issues must come hands. Previous|4.44|3.24|4003001|exportiedu pack #1|3|kids|4|Shoes|37|ationpri|petite|6honeydew42092153663|misty|Dozen|Unknown|20|ableoughtableationought| +17213|AAAAAAAAMDDEAAAA|1999-10-28|2001-10-26|Minutes provide on a letters; paintings might bring now namely long locations; contemporary, commercial words prevent far; issues must come hands. Previous|8.95|3.24|5002002|importoscholar #2|3|country|5|Music|821|oughtableeing|N/A|0796157189frosted016|papaya|Ounce|Unknown|1|prioughtableationought| +17214|AAAAAAAAMDDEAAAA|2001-10-27||Methods should not worry domestic sales. Urban muscles ge|0.10|3.24|5002002|namelessnameless #5|8|outdoor|8|Sports|802|ablebareing|N/A|0796157189frosted016|salmon|Dram|Unknown|25|eseoughtableationought| +17215|AAAAAAAAPDDEAAAA|1997-10-27||Schools must evaluate secondly; quite democratic recommendations will assess however lines. Always effective strings can step just; sides could work. However normal operatio|2.31|2.05|8009006|maxinameless #6|9|optics|8|Sports|78|eingation|N/A|5885linen80802871853|snow|Ounce|Unknown|21|antioughtableationought| +17216|AAAAAAAAAEDEAAAA|1997-10-27|2000-10-26|Industrial pools used to use. Islamic, low months would not get only areas. Additional, strong computers withdraw just true, minor women. |0.55|0.37|4001001|amalgedu pack #1|1|womens|4|Shoes|247|ationeseable|medium|5500chocolate1750791|linen|Dram|Unknown|63|callyoughtableationought| +17217|AAAAAAAAAEDEAAAA|2000-10-27||Industrial pools used to use. Islamic, low months would not get only areas. Additional, strong computers withdraw just true, minor women. |8.76|7.62|6016002|corpbrand #2|1|consignment|6|Jewelry|238|ationeseable|N/A|5500chocolate1750791|seashell|Oz|Unknown|66|ationoughtableationought| +17218|AAAAAAAACEDEAAAA|1997-10-27|1999-10-27|Unfortunately reasonable houses |3.09|1.73|10015015|scholaramalgamalg #15|15|portable|10|Electronics|481|oughteingese|N/A|99565721plum77357413|navajo|Box|Unknown|55|eingoughtableationought| +17219|AAAAAAAACEDEAAAA|1999-10-28|2001-10-26|Ordinary groups will resist reportedly years. Careful, dry pubs shall get again considerable nights. Incidents like inc, poor examinations. New, skilled resources must not smell le|78.72|50.38|10015015|exportiedu pack #2|15|kids|4|Shoes|481|oughteingese|small|732149lavender598850|azure|N/A|Unknown|32|n stoughtableationought| +17220|AAAAAAAACEDEAAAA|2001-10-27||Early happy methods ought to wait at the days. Investigations develop different,|0.98|50.38|10015015|amalgbrand #1|15|bathroom|7|Home|158|eingantiought|N/A|6814048604283purple0|medium|Gross|Unknown|31|barableableationought| +17221|AAAAAAAAFEDEAAAA|1997-10-27||Governments remember very competitive adults. Reasons lose. Happy hearts could |3.10|1.14|1003002|exportiamalg #2|3|maternity|1|Women|361|oughtcallypri|small|1003650483652pink832|steel|Bunch|Unknown|9|oughtableableationought| +17222|AAAAAAAAGEDEAAAA|1997-10-27|2000-10-26|Still formal lights retur|5.36|3.80|2004001|edu packimporto #1|4|sports-apparel|2|Men|106|callybarought|small|085322gainsboro00748|chartreuse|Gross|Unknown|56|ableableableationought| +17223|AAAAAAAAGEDEAAAA|2000-10-27||Still formal lights retur|9.83|3.14|2004001|importobrand #4|4|bedding|7|Home|106|callybarought|N/A|022068307038335plum5|tomato|Box|Unknown|46|priableableationought| +17224|AAAAAAAAIEDEAAAA|1997-10-27|1999-10-27|Different years complain merely comprehensive, effective weeks. Images will discuss honours; similar centres get now needs. Foreign errors last sepa|0.85|0.36|8016005|corpmaxi #5|16|golf|8|Sports|547|ationeseanti|N/A|91orchid554471620670|pale|Tbl|Unknown|38|eseableableationought| +17225|AAAAAAAAIEDEAAAA|1999-10-28|2001-10-26|Arms think certainly on the eyes. Already pure things shore much. Parliamentary workers get. Exactly main pages meet around small, relevant horses; cle|0.31|0.16|7006006|corpbrand #6|16|rugs|7|Home|547|ationeseanti|N/A|53025maroon033153490|seashell|Bundle|Unknown|59|antiableableationought| +17226|AAAAAAAAIEDEAAAA|2001-10-27||Proud, strict interests tell only new areas; keys help public contacts. Tall bonds reach suddenly inadequate frie|3.94|0.16|10005006|scholarunivamalg #6|16|karoke|10|Electronics|364|ationeseanti|N/A|87270619549071ghost0|saddle|Gross|Unknown|77|callyableableationought| +17227|AAAAAAAALEDEAAAA|1997-10-27||Villages shall vary in order formal, able moments. Old figures will happen significantly in a incidents. Working-class pow|6.75|3.57|7010008|univnameless #8|10|flatware|7|Home|258|eingantiable|N/A|3301929white46763112|chiffon|Lb|Unknown|19|ationableableationought| +17228|AAAAAAAAMEDEAAAA|1997-10-27|2000-10-26|Most natural teeth date needs. Minutes would improve ruefully final shares. Forms shall remember before. Wild, other governments |6.15|3.01|2001001|amalgimporto #1|1|accessories|2|Men|200|barbarable|extra large|58611spring390039207|steel|Tsp|Unknown|21|eingableableationought| +17229|AAAAAAAAMEDEAAAA|2000-10-27||Most natural teeth date needs. Minutes would improve ruefully final shares. Forms shall remember before. Wild, other governments |4.99|3.99|4001002|amalgedu pack #2|1|womens|4|Shoes|827|barbarable|small|58611spring390039207|red|N/A|Unknown|53|n stableableationought| +17230|AAAAAAAAOEDEAAAA|1997-10-27|1999-10-27|National, new hotels mean for a variables. Countries may not spend on the quarters. Else common differences used to call much on a months. New events perform too. Immense, perfect things reform|0.27|0.14|7016001|corpnameless #1|16|furniture|7|Home|115|antioughtought|N/A|998857351slate717181|lime|Pallet|Unknown|6|barpriableationought| +17231|AAAAAAAAOEDEAAAA|1999-10-28|2001-10-26|National, new hotels mean for a variables. Countries may not spend on the quarters. Else common differences used to call much on a months. New events perform too. Immense, perfect things reform|9.13|8.12|2003002|exportiimporto #2|16|pants|2|Men|45|antioughtought|small|45449tan407978283232|plum|Box|Unknown|23|oughtpriableationought| +17232|AAAAAAAAOEDEAAAA|2001-10-27||Coming points used to solve. Central techniques catch valuable others; also american things might not persuade old entr|5.35|8.12|5002001|importoscholar #1|2|country|5|Music|315|antioughtought|N/A|7256912629plum047798|puff|Tbl|Unknown|57|ablepriableationought| +17233|AAAAAAAABFDEAAAA|1997-10-27||Efficient, bad commitments ought to form grounds. Alone vast competitors might|19.40|12.80|9012002|importounivamalg #2|12|home repair|9|Books|191|oughtn stought|N/A|40299sienna007782732|purple|Box|Unknown|59|pripriableationought| +17234|AAAAAAAACFDEAAAA|1997-10-27|2000-10-26|Positions could play no longer main needs. Smart police find also mental quantities. Recently casual keys equate indeed well large cities. Patients shall not call females. Somehow |72.47|26.08|10005014|scholarunivamalg #14|5|karoke|10|Electronics|188|eingeingought|N/A|442855orchid98650239|plum|Pound|Unknown|95|esepriableationought| +17235|AAAAAAAACFDEAAAA|2000-10-27||Positions could play no longer main needs. Smart police find also mental quantities. Recently casual keys equate indeed well large cities. Patients shall not call females. Somehow |1.69|1.18|4001002|amalgedu pack #2|1|womens|4|Shoes|188|eingeingought|large|442855orchid98650239|tomato|Unknown|Unknown|62|antipriableationought| +17236|AAAAAAAAEFDEAAAA|1997-10-27|1999-10-27|Internal fans could say right elsewhere intense problems. Wild happy implications could maintain more cle|3.25|2.60|3002001|importoexporti #1|2|infants|3|Children|111|oughtoughtought|small|58thistle34204007488|purple|Dozen|Unknown|38|callypriableationought| +17237|AAAAAAAAEFDEAAAA|1999-10-28|2001-10-26|Internal fans could say right elsewhere intense problems. Wild happy implications could maintain more cle|5.88|2.64|5003002|exportischolar #2|2|pop|5|Music|111|oughtoughtought|N/A|58thistle34204007488|maroon|Each|Unknown|94|ationpriableationought| +17238|AAAAAAAAEFDEAAAA|2001-10-27||Internal fans could say right elsewhere intense problems. Wild happy implications could maintain more cle|41.32|28.51|5003002|exportiexporti #1|2|toddlers|3|Children|111|oughtoughtought|large|911465ivory925962521|plum|Ounce|Unknown|17|eingpriableationought| +17239|AAAAAAAAHFDEAAAA|1997-10-27||Substantial, afraid effects must close. Areas could make only|6.37|5.66|9011008|amalgunivamalg #8|11|cooking|9|Books|93|prin st|N/A|156407485451indian31|indian|Case|Unknown|8|n stpriableationought| +17240|AAAAAAAAIFDEAAAA|1997-10-27|2000-10-26|Free crowds will determine also tonight married plans. Warm states arrange full-time class|42.40|24.59|1002001|importoamalg #1|2|fragrances|1|Women|123|priableought|small|85pale21292355493367|magenta|Carton|Unknown|58|bareseableationought| +17241|AAAAAAAAIFDEAAAA|2000-10-27||Free crowds will determine also tonight married plans. Warm states arrange full-time class|44.97|13.49|1002001|exportinameless #2|2|wallpaper|7|Home|154|priableought|N/A|85pale21292355493367|snow|Gram|Unknown|5|oughteseableationought| +17242|AAAAAAAAKFDEAAAA|1997-10-27|1999-10-27|Operators complete thus thick owners. Wrong, friendly strengths lift significant, strong regions. Kinds afford to a bones. Thick men happ|1.72|0.82|5001001|amalgscholar #1|1|rock|5|Music|559|n stantianti|N/A|30265456puff56620522|rosy|Box|Unknown|29|ableeseableationought| +17243|AAAAAAAAKFDEAAAA|1999-10-28|2001-10-26|Men know. Schools should discuss national provisions. Institu|7.89|0.82|1004002|edu packamalg #2|1|swimwear|1|Women|559|n stantianti|extra large|30265456puff56620522|turquoise|Ton|Unknown|20|prieseableationought| +17244|AAAAAAAAKFDEAAAA|2001-10-27||Men know. Schools should discuss national provisions. Institu|2.85|2.36|9001001|amalgmaxi #1|1|arts|9|Books|559|n stantianti|N/A|4662hot1195268416520|wheat|Bundle|Unknown|22|eseeseableationought| +17245|AAAAAAAANFDEAAAA|1997-10-27||Other, young policies must cut well advanced roles. There medical publishers can get to a figures. Cool, sm|0.59|0.27|3002002|importoexporti #2|2|infants|3|Children|722|ableableation|small|97spring097764390766|pale|Gram|Unknown|53|antieseableationought| +17246|AAAAAAAAOFDEAAAA|1997-10-27|2000-10-26|Expected, only experiences distinguish clearly ideal artists; relatively future regions guide now about a authorities. So|9.64|3.66|7010007|univnameless #7|10|flatware|7|Home|402|ablebarese|N/A|7987125157380235sky5|floral|Case|Unknown|7|callyeseableationought| +17247|AAAAAAAAOFDEAAAA|2000-10-27||Full, black areas will know generally to the contracts. Professionals travel. Current, red paintings speed about to a policies; common layers mus|5.45|3.66|7010007|scholarnameless #8|10|tables|7|Home|402|ablebarese|N/A|7987125157380235sky5|lime|Tbl|Unknown|25|ationeseableationought| +17248|AAAAAAAAAGDEAAAA|1997-10-27|1999-10-27|Governments implement really important windows. Modern, important ideas impose again hours;|4.32|2.37|6005001|scholarcorp #1|5|earings|6|Jewelry|350|barantipri|N/A|2734336103tomato7014|moccasin|Ounce|Unknown|74|eingeseableationought| +17249|AAAAAAAAAGDEAAAA|1999-10-28|2001-10-26|Governments implement really important windows. Modern, important ideas impose again hours;|2.55|2.37|5001002|amalgscholar #2|5|rock|5|Music|188|barantipri|N/A|2734336103tomato7014|salmon|Gram|Unknown|55|n steseableationought| +17250|AAAAAAAAAGDEAAAA|2001-10-27||Governments implement really important windows. Modern, important ideas impose again hours;|0.27|0.15|3001001|amalgexporti #1|1|newborn|3|Children|188|barantipri|petite|04225773003327lime78|purple|Gram|Unknown|32|barantiableationought| +17251|AAAAAAAADGDEAAAA|1997-10-27||Late partners get now from a weeks. Thus signifi|4.55|2.36|7010002|univnameless #2|10|flatware|7|Home|275|antiationable|N/A|7689443339438tomato0|slate|Pallet|Unknown|26|oughtantiableationought| +17252|AAAAAAAAEGDEAAAA|1997-10-27|2000-10-26|Capable interests should not make sorry, free courses. Offences should discuss |2.82|1.35|9008009|namelessmaxi #9|8|romance|9|Books|201|oughtbarable|N/A|3saddle0656478432861|rosy|Carton|Unknown|51|ableantiableationought| +17253|AAAAAAAAEGDEAAAA|2000-10-27||Capable interests should not make sorry, free courses. Offences should discuss |8.06|1.35|9008009|amalgscholar #2|8|rock|5|Music|201|oughtbarable|N/A|1292109967861sienna7|brown|Pallet|Unknown|11|priantiableationought| +17254|AAAAAAAAGGDEAAAA|1997-10-27|1999-10-27|Main hours deny ostensibly in the views. English artists will imagine certain kilometres. As yellow r|0.58|0.34|2004001|edu packimporto #1|4|sports-apparel|2|Men|401|oughtbarese|large|081396451402papaya83|green|Dram|Unknown|19|eseantiableationought| +17255|AAAAAAAAGGDEAAAA|1999-10-28|2001-10-26|Main hours deny ostensibly in the views. English artists will imagine certain kilometres. As yellow r|5.28|0.34|9013012|exportiunivamalg #12|4|self-help|9|Books|401|oughtbarese|N/A|081396451402papaya83|peru|N/A|Unknown|15|antiantiableationought| +17256|AAAAAAAAGGDEAAAA|2001-10-27||Good, national hours see even indirectly attractive rates. Top holidays hand lastly prayers. Pupils examine around very black ve|3.80|0.34|8016007|corpmaxi #7|4|golf|8|Sports|401|oughtbarese|N/A|403122puff3284507513|rose|Ton|Unknown|65|callyantiableationought| +17257|AAAAAAAAJGDEAAAA|1997-10-27||Other, big workers shall not tell all from a councillors. Primar|1.73|1.36|5001002|amalgscholar #2|1|rock|5|Music|606|callybarcally|N/A|43362274821468royal2|lemon|Tbl|Unknown|28|ationantiableationought| +17258|AAAAAAAAKGDEAAAA|1997-10-27|2000-10-26|Possible days must play today of course easy figures. Bitter options listen necessarily disabled, foreign places. Orders shall rush to a images. Weak, top accidents make in the boats; rarel|4.09|2.08|5003001|exportischolar #1|3|pop|5|Music|379|n stationpri|N/A|1629184302712sienna3|sienna|Case|Unknown|5|eingantiableationought| +17259|AAAAAAAAKGDEAAAA|2000-10-27||Both available expec|2.30|1.08|4002002|importoedu pack #2|2|mens|4|Shoes|631|oughtprically|N/A|sky35040508348728791|sienna|N/A|Unknown|9|n stantiableationought| +17260|AAAAAAAAMGDEAAAA|1997-10-27|1999-10-27|Masters provide. Forces shall keep quickly social, foreign amer|4.53|1.90|5001001|amalgscholar #1|1|rock|5|Music|116|callyoughtought|N/A|1135328575771linen55|cornflower|Cup|Unknown|13|barcallyableationought| +17261|AAAAAAAAMGDEAAAA|1999-10-28|2001-10-26|Masters provide. Forces shall keep quickly social, foreign amer|5.11|2.60|2003002|exportiimporto #2|3|pants|2|Men|116|callyoughtought|large|1135328575771linen55|puff|Tsp|Unknown|23|oughtcallyableationought| +17262|AAAAAAAAMGDEAAAA|2001-10-27||Masters provide. Forces shall keep quickly social, foreign amer|3.94|2.87|2003002|univmaxi #3|3|pools|8|Sports|914|eseoughtn st|N/A|1135328575771linen55|rose|Oz|Unknown|62|ablecallyableationought| +17263|AAAAAAAAPGDEAAAA|1997-10-27||Appointed others must trace yesterday with the members. Disabled animals talk also isolated, entire soldiers. Signs join at all lega|0.97|0.59|9015008|scholarunivamalg #8|15|fiction|9|Books|241|oughteseable|N/A|25rosy68032080740655|saddle|Bundle|Unknown|35|pricallyableationought| +17264|AAAAAAAAAHDEAAAA|1997-10-27|2000-10-26|Consistent, regular men find well deaf, open links. However big kinds shall not avoid acceptable, other sources. Ra|1.05|0.69|3003001|exportiexporti #1|3|toddlers|3|Children|272|ableationable|extra large|075saddle93288051014|pink|Ton|Unknown|15|esecallyableationought| +17265|AAAAAAAAAHDEAAAA|2000-10-27||Consistent, regular men find well deaf, open links. However big kinds shall not avoid acceptable, other sources. Ra|9.08|6.26|3003001|edu packexporti #2|4|school-uniforms|3|Children|327|ationablepri|large|075saddle93288051014|wheat|Tsp|Unknown|16|anticallyableationought| +17266|AAAAAAAACHDEAAAA|1997-10-27|1999-10-27|Acids grab below previous standards. Ever large metals will come on a articles. Underlying stories protect at last. Reasonable directions believe rather due to a|2.58|1.62|2004001|edu packimporto #1|4|sports-apparel|2|Men|938|eingprin st|medium|41321632salmon343751|spring|Cup|Unknown|25|callycallyableationought| +17267|AAAAAAAACHDEAAAA|1999-10-28|2001-10-26|Acids grab below previous standards. Ever large metals will come on a articles. Underlying stories protect at last. Reasonable directions believe rather due to a|5.08|1.62|10014013|edu packamalgamalg #13|14|automotive|10|Electronics|938|eingprin st|N/A|41321632salmon343751|violet|Gram|Unknown|29|ationcallyableationought| +17268|AAAAAAAACHDEAAAA|2001-10-27||Acids grab below previous standards. Ever large metals will come on a articles. Underlying stories protect at last. Reasonable directions believe rather due to a|1.43|1.18|4004001|edu packedu pack #1|14|athletic|4|Shoes|86|callyeing|medium|311827224puff6643625|tan|Gram|Unknown|62|eingcallyableationought| +17269|AAAAAAAAFHDEAAAA|1997-10-27||Keys should meet parties. Ministers leave members. Small, new students may take always individual letters. Video-taped levels think russian ingredients. Evident pieces secure merely biological, safe c|1.63|1.23|7008010|namelessbrand #10|8|lighting|7|Home|66|callycally|N/A|khaki659879522323732|seashell|Gross|Unknown|44|n stcallyableationought| +17270|AAAAAAAAGHDEAAAA|1997-10-27|2000-10-26|Natural, estimated names could not give however strongly true proposals; very characters see enough coming, domestic exchanges. Germans must see quick words. More additional sources pro|86.68|33.80|10008016|namelessunivamalg #16|8|scanners|10|Electronics|112|ableoughtought|N/A|674gainsboro51475888|mint|Ounce|Unknown|37|barationableationought| +17271|AAAAAAAAGHDEAAAA|2000-10-27||Almost |9.76|8.58|10008016|exportinameless #2|8|wallpaper|7|Home|112|ableoughtought|N/A|674gainsboro51475888|orange|Pallet|Unknown|28|oughtationableationought| +17272|AAAAAAAAIHDEAAAA|1997-10-27|1999-10-27|Parliamentary shareholders must not want very in a parts. Rich, national conditions might provide finally economic, difficu|5.16|1.54|7008003|namelessbrand #3|8|lighting|7|Home|39|n stpri|N/A|37thistle41684772094|white|Box|Unknown|19|ableationableationought| +17273|AAAAAAAAIHDEAAAA|1999-10-28|2001-10-26|Parliamentary shareholders must not want very in a parts. Rich, national conditions might provide finally economic, difficu|3.53|2.50|7008003|edu packexporti #2|4|school-uniforms|3|Children|232|n stpri|medium|320honeydew925465041|black|Case|Unknown|7|priationableationought| +17274|AAAAAAAAIHDEAAAA|2001-10-27||Parliamentary shareholders must not want very in a parts. Rich, national conditions might provide finally economic, difficu|3.74|2.50|7016005|corpnameless #5|16|furniture|7|Home|232|n stpri|N/A|320honeydew925465041|salmon|Case|Unknown|31|eseationableationought| +17275|AAAAAAAALHDEAAAA|1997-10-27||Extensive laboratories|3.98|1.59|4002002|importoedu pack #2|2|mens|4|Shoes|139|n stpriought|petite|047885plum3513428444|smoke|Ounce|Unknown|35|antiationableationought| +17276|AAAAAAAAMHDEAAAA|1997-10-27|2000-10-26|Existing members make sometimes. Private ways ought to ki|0.75|0.38|10005005|scholarunivamalg #5|5|karoke|10|Electronics|570|barationanti|N/A|7steel16102340183242|slate|N/A|Unknown|22|callyationableationought| +17277|AAAAAAAAMHDEAAAA|2000-10-27||Existing members make sometimes. Private ways ought to ki|87.94|0.38|4002002|importoedu pack #2|2|mens|4|Shoes|570|barationanti|petite|90snow17087129134744|orange|Oz|Unknown|31|ationationableationought| +17278|AAAAAAAAOHDEAAAA|1997-10-27|1999-10-27|Coming, gastric officials apply financial conventions. Eager, economic mothers promise clear, likely effects; star offences would not ask possible eyes. Vaguely british farmers play e|5.35|2.14|1001001|amalgamalg #1|1|dresses|1|Women|371|oughtationpri|small|5377papaya5486168043|rosy|Case|Unknown|40|eingationableationought| +17279|AAAAAAAAOHDEAAAA|1999-10-28|2001-10-26|Coming, gastric officials apply financial conventions. Eager, economic mothers promise clear, likely effects; star offences would not ask possible eyes. Vaguely british farmers play e|5.36|3.53|10007008|brandunivamalg #8|7|personal|10|Electronics|371|oughtationpri|N/A|1336352348plum129146|sandy|Cup|Unknown|18|n stationableationought| +17280|AAAAAAAAOHDEAAAA|2001-10-27||Forces bring especially possible questions. Exciting wee|2.04|3.53|10007008|brandcorp #1|7|pendants|6|Jewelry|87|oughtationpri|N/A|1336352348plum129146|mint|Ounce|Unknown|42|bareingableationought| +17281|AAAAAAAABIDEAAAA|1997-10-27||Practical, important lands discriminate much outstanding relations. Fine, overseas months stop fully fashionable attempts; great, important posts |1.99|1.15|8011002|amalgmaxi #2|11|archery|8|Sports|293|prin stable|N/A|194smoke880943405718|slate|Tsp|Unknown|8|oughteingableationought| +17282|AAAAAAAACIDEAAAA|1997-10-27|2000-10-26|Appropriate stations investigate just to a|3.48|1.87|8008003|namelessnameless #3|8|outdoor|8|Sports|30|barpri|N/A|4407214404483coral46|papaya|Dram|Unknown|94|ableeingableationought| +17283|AAAAAAAACIDEAAAA|2000-10-27||Appropriate stations investigate just to a|3.51|1.87|3002002|importoexporti #2|2|infants|3|Children|30|barpri|medium|4407214404483coral46|tan|Carton|Unknown|72|prieingableationought| +17284|AAAAAAAAEIDEAAAA|1997-10-27|1999-10-27|Main policies remember both artistic, sure passengers. Skills afford further proteins. More wise letters protect in a applications. Straight |3.23|2.87|10016013|corpamalgamalg #13|16|wireless|10|Electronics|241|oughteseable|N/A|0193indian6419066606|tan|Bundle|Unknown|80|eseeingableationought| +17285|AAAAAAAAEIDEAAAA|1999-10-28|2001-10-26|Head shares might not play unfair, soviet eggs. Rough, unlikely pensions refute here social, cert|1.24|0.45|10016013|importoscholar #2|2|country|5|Music|241|oughteseable|N/A|2589860934navy248708|steel|Tsp|Unknown|56|antieingableationought| +17286|AAAAAAAAEIDEAAAA|2001-10-27||Regions must give; also private boys know often on a prices; scottish, great trees must comply|5.76|0.45|10016013|univunivamalg #7|10|travel|9|Books|241|oughteseable|N/A|03sandy2904164367751|honeydew|Gram|Unknown|33|callyeingableationought| +17287|AAAAAAAAHIDEAAAA|1997-10-27||Political paths should go inc years. New materials shall represent results. Very, actual trees will make that is new, la|6.93|3.60|7014008|edu packnameless #8|14|glassware|7|Home|477|ationationese|N/A|04298989gainsboro608|pale|Dram|Unknown|52|ationeingableationought| +17288|AAAAAAAAIIDEAAAA|1997-10-27|2000-10-26|Scottish disciplines could not remember now national books. Military, national lakes go heavy facts. Free, capitalist processes may |8.03|2.73|5003001|exportischolar #1|3|pop|5|Music|138|eingpriought|N/A|8sienna2798026590848|magenta|Cup|Unknown|16|eingeingableationought| +17289|AAAAAAAAIIDEAAAA|2000-10-27||Excellent sharehol|0.18|0.14|5003001|exportiexporti #2|3|toddlers|3|Children|138|eingpriought|medium|2484123800831puff813|thistle|Case|Unknown|28|n steingableationought| +17290|AAAAAAAAKIDEAAAA|1997-10-27|1999-10-27|Free, young women should view with a insects. Constant lives open of course animals. Controls see well. Particular cattle shall treat also then other stud|0.62|0.55|9010011|univunivamalg #11|10|travel|9|Books|78|eingation|N/A|47692842664318pale54|violet|Ton|Unknown|25|barn stableationought| +17291|AAAAAAAAKIDEAAAA|1999-10-28|2001-10-26|Free, young women should view with a insects. Constant lives open of course animals. Controls see well. Particular cattle shall treat also then other stud|1.83|1.02|9010011|corpunivamalg #12|16|mystery|9|Books|254|eseantiable|N/A|47692842664318pale54|indian|Cup|Unknown|64|oughtn stableationought| +17292|AAAAAAAAKIDEAAAA|2001-10-27||Free, young women should view with a insects. Constant lives open of course animals. Controls see well. Particular cattle shall treat also then other stud|56.24|1.02|5003001|exportischolar #1|3|pop|5|Music|77|ationation|N/A|47692842664318pale54|light|Unknown|Unknown|48|ablen stableationought| +17293|AAAAAAAANIDEAAAA|1997-10-27||Defendants act in the sections. Financial rates shall try. Earlier normal plans finance employers. Parties dispose as in the units. Accord|0.41|0.23|3002002|importoexporti #2|2|infants|3|Children|84|eseeing|medium|9548tomato7896887460|wheat|Pallet|Unknown|55|prin stableationought| +17294|AAAAAAAAOIDEAAAA|1997-10-27|2000-10-26|Statutory others make too problems; later supposed patients would build yet also italian rights. Across front proper|0.58|0.25|6001007|amalgcorp #7|1|birdal|6|Jewelry|218|eingoughtable|N/A|807plum2390628603247|tomato|Dozen|Unknown|22|esen stableationought| +17295|AAAAAAAAOIDEAAAA|2000-10-27||Statutory others make too problems; later supposed patients would build yet also italian rights. Across front proper|88.78|0.25|8003006|exportinameless #6|3|basketball|8|Sports|218|eingoughtable|N/A|807plum2390628603247|peach|Gross|Unknown|9|antin stableationought| +17296|AAAAAAAAAJDEAAAA|1997-10-27|1999-10-27|Brown changes would go|6.20|1.86|6004001|edu packcorp #1|4|bracelets|6|Jewelry|432|ablepriese|N/A|17saddle504425875683|ivory|Cup|Unknown|92|callyn stableationought| +17297|AAAAAAAAAJDEAAAA|1999-10-28|2001-10-26|Brown changes would go|3.44|2.88|8007008|brandnameless #8|7|hockey|8|Sports|432|ablepriese|N/A|red62499928677910746|orchid|Lb|Unknown|48|ationn stableationought| +17298|AAAAAAAAAJDEAAAA|2001-10-27||Provisions shall draw inner, odd arguments. Basic, other examinations cannot boost seriously efficiently coming individuals. Unlikely, human animals need aside rather|5.37|2.63|7016009|corpnameless #9|16|furniture|7|Home|227|ationableable|N/A|248402381152884snow8|cornflower|Gross|Unknown|19|eingn stableationought| +17299|AAAAAAAADJDEAAAA|1997-10-27||Growing jobs might offer with the|76.11|50.99|10015011|scholaramalgamalg #11|15|portable|10|Electronics|57|ationanti|N/A|4139954048472peru182|linen|Gross|Unknown|2|n stn stableationought| +17300|AAAAAAAAEJDEAAAA|1997-10-27|2000-10-26|Natural communities create original youngsters; as beautiful children smooth legal, big agreements. Special, other heads make regularly la|6.41|5.51|9007009|brandmaxi #9|7|reference|9|Books|248|eingeseable|N/A|28283175688315sandy5|navy|Ton|Unknown|9|barbarpriationought| +17301|AAAAAAAAEJDEAAAA|2000-10-27||Full hours can strengthen and so on prime views. Efficient, very women shall not provide foreign duties; quite big profits send in a shops. Authorities s|1.42|0.58|10009013|maxiunivamalg #13|9|televisions|10|Electronics|248|eingeseable|N/A|2987821moccasin31669|misty|Bundle|Unknown|10|oughtbarpriationought| +17302|AAAAAAAAGJDEAAAA|1997-10-27|1999-10-27|Heavily positive heroes seem far exciting values; letters might ask still about a r|1.66|1.01|8009003|maxinameless #3|9|optics|8|Sports|30|barpri|N/A|26003862642621slate9|royal|Bundle|Unknown|2|ablebarpriationought| +17303|AAAAAAAAGJDEAAAA|1999-10-28|2001-10-26|Heavily positive heroes seem far exciting values; letters might ask still about a r|5.50|1.01|8010004|univmaxi #4|9|pools|8|Sports|488|eingeingese|N/A|80593970830573snow66|sienna|Oz|Unknown|1|pribarpriationought| +17304|AAAAAAAAGJDEAAAA|2001-10-27||Great numbers express finally at a characteristics. |1.54|1.13|8010004|exportiunivamalg #1|9|self-help|9|Books|488|eingeingese|N/A|76560113813salmon807|turquoise|Box|Unknown|26|esebarpriationought| +17305|AAAAAAAAJJDEAAAA|1997-10-27||Rises wou|0.52|0.44|2003002|exportiimporto #2|3|pants|2|Men|315|antioughtpri|petite|65823842metallic0881|puff|Bunch|Unknown|42|antibarpriationought| +17306|AAAAAAAAKJDEAAAA|1997-10-27|2000-10-26|Constitutional, high books see of course extra rivers. Fields undergo for the students. Teachers contend characteristics. Only messages must not defend only; unusual birds may not stay sectio|0.29|0.13|8004007|edu packnameless #7|4|camping|8|Sports|108|eingbarought|N/A|69lace52017125188953|snow|Oz|Unknown|48|callybarpriationought| +17307|AAAAAAAAKJDEAAAA|2000-10-27||Lonely, british offenders exercise here equally waiting folk. Significant members return almost fun losses. Naval aspirations wipe most lights. Respects think fingers. Islands p|6.31|0.13|5002002|importoscholar #2|2|country|5|Music|108|eingbarought|N/A|7369863pale828470562|purple|Dozen|Unknown|65|ationbarpriationought| +17308|AAAAAAAAMJDEAAAA|1997-10-27|1999-10-27|Etc beaut|38.56|19.28|8015009|scholarmaxi #9|15|fishing|8|Sports|532|ableprianti|N/A|10772peach9397404577|tan|Tsp|Unknown|73|eingbarpriationought| +17309|AAAAAAAAMJDEAAAA|1999-10-28|2001-10-26|Etc beaut|1.35|0.98|8015009|exportibrand #6|15|loose stones|6|Jewelry|352|ableprianti|N/A|98violet030152133534|sky|Ounce|Unknown|49|n stbarpriationought| +17310|AAAAAAAAMJDEAAAA|2001-10-27||Etc beaut|3.22|0.98|5001001|amalgscholar #1|15|rock|5|Music|780|ableprianti|N/A|35sandy7227769383331|tomato|Box|Unknown|24|baroughtpriationought| +17311|AAAAAAAAPJDEAAAA|1997-10-27||Great, large needs take unable, economic flowers. Etc new companies should stay over subjective new approaches; desirable years keep now gold differences. Well full objects will tell also; d|3.23|2.03|2003002|exportiimporto #2|3|pants|2|Men|325|antiablepri|small|purple96062455129015|peru|Bunch|Unknown|46|oughtoughtpriationought| +17312|AAAAAAAAAKDEAAAA|1997-10-27|2000-10-26|Vast, invisible |6.73|5.98|4001001|amalgedu pack #1|1|womens|4|Shoes|17|ationought|medium|341396286cornflower9|puff|Lb|Unknown|85|ableoughtpriationought| +17313|AAAAAAAAAKDEAAAA|2000-10-27||Vast, invisible |5.34|5.98|4001002|amalgedu pack #2|1|womens|4|Shoes|17|ationought|medium|674sandy121192862029|tomato|Unknown|Unknown|65|prioughtpriationought| +17314|AAAAAAAACKDEAAAA|1997-10-27|1999-10-27|Terms stand universal waters. Willing, british studies should not recur as in a committees; objectives must turn so single planes. Later old |3.75|2.51|4001001|amalgedu pack #1|1|womens|4|Shoes|254|eseantiable|small|9515646063drab508261|snow|Dozen|Unknown|1|eseoughtpriationought| +17315|AAAAAAAACKDEAAAA|1999-10-28|2001-10-26|New children used to follow per se only leaders. Formal clergy could ensure material friends. Further british eyes could walk. Worldwide national disciplines should occur also lucky|6.63|4.90|4001001|edu packscholar #2|1|classical|5|Music|254|eseantiable|N/A|9515646063drab508261|papaya|Tbl|Unknown|25|antioughtpriationought| +17316|AAAAAAAACKDEAAAA|2001-10-27||Regularly necessary units close slowly papers. Dirty, available years react now present examples. So willing days should leave at a areas. European times i|4.52|4.90|4001001|amalgbrand #7|1|bathroom|7|Home|550|eseantiable|N/A|9515646063drab508261|almond|Ounce|Unknown|13|callyoughtpriationought| +17317|AAAAAAAAFKDEAAAA|1997-10-27||Western, great eyes return unknown tensions. European years might not signal asleep, reduced countries. S|7.29|4.95|9012002|importounivamalg #2|12|home repair|9|Books|521|oughtableanti|N/A|6341426387296red9239|cornsilk|Lb|Unknown|5|ationoughtpriationought| +17318|AAAAAAAAGKDEAAAA|1997-10-27|2000-10-26|Tests will maintain only. Beautifully local banks make still; particular votes protect during a eyes. Contracts must understand primarily. Difficult countries cast in a |4.13|2.68|2004001|edu packimporto #1|4|sports-apparel|2|Men|725|antiableation|medium|4brown30502018077813|plum|Each|Unknown|41|eingoughtpriationought| +17319|AAAAAAAAGKDEAAAA|2000-10-27||Tests will maintain only. Beautifully local banks make still; particular votes protect during a eyes. Contracts must understand primarily. Difficult countries cast in a |3.46|1.38|3002002|importoexporti #2|2|infants|3|Children|940|baresen st|medium|247174625527pale8931|salmon|Carton|Unknown|31|n stoughtpriationought| +17320|AAAAAAAAIKDEAAAA|1997-10-27|1999-10-27|Small, new spaces shall use foreign pieces; american men defy critical systems; intact, british industries will not remain compl|0.45|0.34|5004001|edu packscholar #1|4|classical|5|Music|538|eingprianti|N/A|3239137orchid9010761|olive|Dozen|Unknown|13|barablepriationought| +17321|AAAAAAAAIKDEAAAA|1999-10-28|2001-10-26|Small, new spaces shall use foreign pieces; american men defy critical systems; intact, british industries will not remain compl|6.96|0.34|3001002|amalgexporti #2|1|newborn|3|Children|945|eingprianti|extra large|3239137orchid9010761|misty|Case|Unknown|60|oughtablepriationought| +17322|AAAAAAAAIKDEAAAA|2001-10-27||Small, new spaces shall use foreign pieces; american men defy critical systems; intact, british industries will not remain compl|9.93|8.14|3001002|amalgimporto #1|1|accessories|2|Men|530|barprianti|medium|27tomato291114052963|grey|Dram|Unknown|30|ableablepriationought| +17323|AAAAAAAALKDEAAAA|1997-10-27||Never hot accounts appreciate possibly available leaders. S|80.88|63.89|4003002|exportiedu pack #2|3|kids|4|Shoes|422|ableableese|medium|693563261papaya28266|magenta|Dram|Unknown|5|priablepriationought| +17324|AAAAAAAAMKDEAAAA|1997-10-27|2000-10-26|Old, available pp. wind actu|9.69|3.77|8005005|scholarnameless #5|5|fitness|8|Sports|523|priableanti|N/A|8391009992537smoke43|turquoise|Gross|Unknown|13|eseablepriationought| +17325|AAAAAAAAMKDEAAAA|2000-10-27||Old, available pp. wind actu|0.96|3.77|2003002|exportiimporto #2|5|pants|2|Men|523|priableanti|medium|9748red1921550088177|indian|Gram|Unknown|34|antiablepriationought| +17326|AAAAAAAAOKDEAAAA|1997-10-27|1999-10-27|Societies go great spaces; significant, practical teachers o|8.63|6.12|2004001|edu packimporto #1|4|sports-apparel|2|Men|493|prin stese|large|gainsboro04980521799|violet|Gram|Unknown|94|callyablepriationought| +17327|AAAAAAAAOKDEAAAA|1999-10-28|2001-10-26|Societies go great spaces; significant, practical teachers o|4.52|6.12|2004001|exportischolar #2|4|pop|5|Music|84|eseeing|N/A|1lawn480359050034677|white|Tbl|Unknown|41|ationablepriationought| +17328|AAAAAAAAOKDEAAAA|2001-10-27||Labour years may date always palestinian, good courts; still fiscal matte|4.24|1.78|5004001|edu packscholar #1|4|classical|5|Music|204|esebarable|N/A|6455547861415plum765|purple|Tbl|Unknown|6|eingablepriationought| +17329|AAAAAAAABLDEAAAA|1997-10-27||Great, able applicants could not |1.13|0.85|5004002|edu packscholar #2|4|classical|5|Music|565|anticallyanti|N/A|64pink74012916204024|tomato|Oz|Unknown|37|n stablepriationought| +17330|AAAAAAAACLDEAAAA|1997-10-27|2000-10-26|Certain messages eliminate. Local, early centuries shall not give significant, entire prices. Terrible, christian|8.89|6.31|10005008|scholarunivamalg #8|5|karoke|10|Electronics|93|prin st|N/A|471311875lavender145|pink|Dozen|Unknown|42|barpripriationought| +17331|AAAAAAAACLDEAAAA|2000-10-27||Certain messages eliminate. Local, early centuries shall not give significant, entire prices. Terrible, christian|5.97|6.31|10005008|importoamalg #2|5|fragrances|1|Women|93|prin st|N/A|5sandy50770751645162|spring|Lb|Unknown|65|oughtpripriationought| +17332|AAAAAAAAELDEAAAA|1997-10-27|1999-10-27|Shortly current classes enter automatically national ministers. Warm, wrong seats would operate only. Readily major days shall develop. Anyway neat specimens may keep then adults. Functions might not|7.84|2.50|7011003|amalgnameless #3|11|accent|7|Home|191|oughtn stought|N/A|87348415salmon170221|sandy|Ton|Unknown|76|ablepripriationought| +17333|AAAAAAAAELDEAAAA|1999-10-28|2001-10-26|Shortly current classes enter automatically national ministers. Warm, wrong seats would operate only. Readily major days shall develop. Anyway neat specimens may keep then adults. Functions might not|6.44|2.44|7011003|exportiamalgamalg #11|11|stereo|10|Electronics|125|antiableought|N/A|87348415salmon170221|goldenrod|Dozen|Unknown|5|pripripriationought| +17334|AAAAAAAAELDEAAAA|2001-10-27||Even public friends apply nevertheless. Boxes shall imagine ago special inches. Levels turn twice at a lives. Possible decades transcend. Pregnant signs shall stick more at a reactions. Critica|5.56|4.78|9008007|namelessmaxi #7|8|romance|9|Books|253|antiableought|N/A|80246613yellow634604|steel|N/A|Unknown|3|esepripriationought| +17335|AAAAAAAAHLDEAAAA|1997-10-27||Loyal, appointed|3.51|1.86|10003013|exportiunivamalg #13|3|dvd/vcr players|10|Electronics|92|ablen st|N/A|purple19423717010557|violet|Dram|Unknown|37|antipripriationought| +17336|AAAAAAAAILDEAAAA|1997-10-27|2000-10-26|Here western modes find total, past principles. Dogs cut joint, f|8.32|6.82|1001001|amalgamalg #1|1|dresses|1|Women|587|ationeinganti|small|2532797rose441489320|rose|Lb|Unknown|31|callypripriationought| +17337|AAAAAAAAILDEAAAA|2000-10-27||Facts used to appear unknown troubles. Close, costly reports must not materialise able, private months; national, financial plans set good children; lovely, available activities mu|71.01|46.86|8002008|importonameless #8|2|baseball|8|Sports|587|ationeinganti|N/A|2532797rose441489320|rosy|Cup|Unknown|27|ationpripriationought| +17338|AAAAAAAAKLDEAAAA|1997-10-27|1999-10-27|Difficult, adult details can know exactly western, other problems. Closed activities might serve easy, open cases. Numbers end even even busy jobs. Social, wrong eggs play of course with a figure|1.10|0.96|8014009|edu packmaxi #9|14|tennis|8|Sports|873|priationeing|N/A|3pale078120021371100|sandy|Lb|Unknown|3|eingpripriationought| +17339|AAAAAAAAKLDEAAAA|1999-10-28|2001-10-26|Difficult, adult details can know exactly western, other problems. Closed activities might serve easy, open cases. Numbers end even even busy jobs. Social, wrong eggs play of course with a figure|0.32|0.96|3002002|importoexporti #2|2|infants|3|Children|177|ationationought|extra large|40642523violet272640|smoke|Tbl|Unknown|54|n stpripriationought| +17340|AAAAAAAAKLDEAAAA|2001-10-27||Lips must watch children. Easy words turn valuable directors. Cold thanks benefit only atomic specimens|8.67|0.96|8004001|edu packnameless #1|4|camping|8|Sports|432|ablepriese|N/A|40642523violet272640|indian|Dram|Unknown|15|baresepriationought| +17341|AAAAAAAANLDEAAAA|1997-10-27||New hours say home. Stories shall hear positive rumours. Initial, british children would not concentrate also gradual, industrial women. Mainly special countries feel as|6.13|4.35|6005006|scholarcorp #6|5|earings|6|Jewelry|220|barableable|N/A|05606131330014green3|indian|Case|Unknown|19|oughtesepriationought| +17342|AAAAAAAAOLDEAAAA|1997-10-27|2000-10-26|Useful, alternative eyes might exclude |3.72|1.37|7006003|corpbrand #3|6|rugs|7|Home|482|ableeingese|N/A|7850652lavender30003|spring|Dram|Unknown|31|ableesepriationought| +17343|AAAAAAAAOLDEAAAA|2000-10-27||Educational troubles will not keep. Social, brief cells shall happen leading, full concentrations. Obvious qualities used to seem black years. Sorry, a|8.64|1.37|7006003|amalgedu pack #2|6|womens|4|Shoes|99|ableeingese|petite|violet70614065362854|sienna|Gross|Unknown|3|priesepriationought| +17344|AAAAAAAAAMDEAAAA|1997-10-27|1999-10-27|Areas prevent real|1.65|1.33|7013005|exportinameless #5|13|wallpaper|7|Home|59|n stanti|N/A|98371yellow474611129|powder|Each|Unknown|11|eseesepriationought| +17345|AAAAAAAAAMDEAAAA|1999-10-28|2001-10-26|Meetings think then in a telecommunications. Independent, ec|0.26|1.33|7013005|edu packbrand #6|13|curtains/drapes|7|Home|59|n stanti|N/A|98371yellow474611129|sienna|Ounce|Unknown|60|antiesepriationought| +17346|AAAAAAAAAMDEAAAA|2001-10-27||Meetings think then in a telecommunications. Independent, ec|3.15|2.67|7009007|maxibrand #7|9|mattresses|7|Home|76|callyation|N/A|98371yellow474611129|thistle|Lb|Unknown|26|callyesepriationought| +17347|AAAAAAAADMDEAAAA|1997-10-27||Long-term services increase sure existing, poor progr|1.92|0.67|6010004|univbrand #4|10|jewelry boxes|6|Jewelry|331|oughtpripri|N/A|2927639206papaya0452|violet|Ton|Unknown|5|ationesepriationought| +17348|AAAAAAAAEMDEAAAA|1997-10-27|2000-10-26|Deep, similar relati|6.02|4.21|8015009|scholarmaxi #9|15|fishing|8|Sports|396|callyn stpri|N/A|90803126190946khaki4|tan|Cup|Unknown|68|eingesepriationought| +17349|AAAAAAAAEMDEAAAA|2000-10-27||As pra|4.56|3.14|7015010|scholarnameless #10|15|tables|7|Home|396|callyn stpri|N/A|90803126190946khaki4|sky|Case|Unknown|29|n stesepriationought| +17350|AAAAAAAAGMDEAAAA|1997-10-27|1999-10-27|El|1.66|0.97|6001007|amalgcorp #7|1|birdal|6|Jewelry|153|priantiought|N/A|594924royal234786776|rosy|Box|Unknown|16|barantipriationought| +17351|AAAAAAAAGMDEAAAA|1999-10-28|2001-10-26|Otherwise other words can complete. Large rivals will not prevent also kinds; so narrow women may start however probably personal players; current, young|2.56|0.97|3004002|edu packexporti #2|1|school-uniforms|3|Children|543|prieseanti|large|594924royal234786776|spring|N/A|Unknown|100|oughtantipriationought| +17352|AAAAAAAAGMDEAAAA|2001-10-27||Otherwise other words can complete. Large rivals will not prevent also kinds; so narrow women may start however probably personal players; current, young|0.09|0.97|3004002|corpamalgamalg #13|16|wireless|10|Electronics|18|prieseanti|N/A|594924royal234786776|steel|Each|Unknown|14|ableantipriationought| +17353|AAAAAAAAJMDEAAAA|1997-10-27||Home warm authorities might recognise overseas. Easy, adequate processes could address about well local styles. Ministers will take. Obviou|8.75|2.71|7001004|amalgbrand #4|1|bathroom|7|Home|7|ation|N/A|539703743387saddle91|slate|Oz|Unknown|68|priantipriationought| +17354|AAAAAAAAKMDEAAAA|1997-10-27|2000-10-26|New workers mean there at a miles; visitors investigate hardly necessary, heavy fact|3.79|2.42|2002001|importoimporto #1|2|shirts|2|Men|932|ableprin st|petite|1132474090salmon1546|powder|Each|Unknown|3|eseantipriationought| +17355|AAAAAAAAKMDEAAAA|2000-10-27||New workers mean there at a miles; visitors investigate hardly necessary, heavy fact|0.28|0.15|6001004|amalgcorp #4|2|birdal|6|Jewelry|932|ableprin st|N/A|1132474090salmon1546|red|Cup|Unknown|42|antiantipriationought| +17356|AAAAAAAAMMDEAAAA|1997-10-27|1999-10-27|Women support almost|4.68|2.66|9003005|exportimaxi #5|3|computers|9|Books|387|ationeingpri|N/A|732031763goldenrod80|thistle|Dozen|Unknown|37|callyantipriationought| +17357|AAAAAAAAMMDEAAAA|1999-10-28|2001-10-26|Women support almost|9.69|6.87|2002002|importoimporto #2|2|shirts|2|Men|387|ationeingpri|medium|13287931438pink44572|white|Dozen|Unknown|12|ationantipriationought| +17358|AAAAAAAAMMDEAAAA|2001-10-27||Women support almost|9.67|6.87|2002002|edu packexporti #1|2|school-uniforms|3|Children|387|ationeingpri|petite|9papaya5040437256329|midnight|Carton|Unknown|36|eingantipriationought| +17359|AAAAAAAAPMDEAAAA|1997-10-27||Earlier educational solicitors shall not want long societies. Skills must not d|8.66|7.01|7014010|edu packnameless #10|14|glassware|7|Home|125|antiableought|N/A|brown560252291871301|rose|Unknown|Unknown|36|n stantipriationought| +17360|AAAAAAAAANDEAAAA|1997-10-27|2000-10-26|Huge, military children fall electronic, ancient days. Tory, remaining instructions sleep accurately dangerou|2.56|2.27|1002001|importoamalg #1|2|fragrances|1|Women|546|callyeseanti|medium|4648855pale910192587|snow|Box|Unknown|10|barcallypriationought| +17361|AAAAAAAAANDEAAAA|2000-10-27||Huge, military children fall electronic, ancient days. Tory, remaining instructions sleep accurately dangerou|1.02|2.27|1002001|maxicorp #2|9|womens watch|6|Jewelry|153|priantiought|N/A|aquamarine0971221323|spring|Pound|Unknown|15|oughtcallypriationought| +17362|AAAAAAAACNDEAAAA|1997-10-27|1999-10-27|Steps should |1.37|0.41|3003001|exportiexporti #1|3|toddlers|3|Children|392|ablen stpri|petite|623saddle89174772139|peach|Oz|Unknown|11|ablecallypriationought| +17363|AAAAAAAACNDEAAAA|1999-10-28|2001-10-26|Steps should |0.33|0.41|9013012|exportiunivamalg #12|3|self-help|9|Books|392|ablen stpri|N/A|26434310976rosy43556|peach|Oz|Unknown|53|pricallypriationought| +17364|AAAAAAAACNDEAAAA|2001-10-27||Difficult, economic agents read here local generations; federal, likely miles learn marginal regions. Associations will admit too reports. Arms used to get in a members. Term|53.44|0.41|9013012|scholaramalgamalg #8|3|portable|10|Electronics|392|ablen stpri|N/A|749011456369mint3590|puff|N/A|Unknown|16|esecallypriationought| +17365|AAAAAAAAFNDEAAAA|1997-10-27||Still young materials explain useful, gentle banks. Slight questions become always. Better relevant men watch ways. Left, individual men agree busy policies. Local, f|4.76|1.66|6006006|corpcorp #6|6|rings|6|Jewelry|228|eingableable|N/A|00883ghost7732796069|hot|Box|Unknown|90|anticallypriationought| +17366|AAAAAAAAGNDEAAAA|1997-10-27|2000-10-26|Old, new instructions obtain. Main, white policies reverse less unfortunately willing concessions. Social notes see muc|8.11|5.51|2004001|edu packimporto #1|4|sports-apparel|2|Men|706|callybaration|small|849smoke712901500268|hot|Cup|Unknown|83|callycallypriationought| +17367|AAAAAAAAGNDEAAAA|2000-10-27||New, capable foods could help women; tired weapons b|3.83|1.18|2004001|exportischolar #2|3|pop|5|Music|175|antiationought|N/A|849smoke712901500268|papaya|Ton|Unknown|25|ationcallypriationought| +17368|AAAAAAAAINDEAAAA|1997-10-27|1999-10-27|Sexual, green processes enjoy so single, vast advisers. Recently c|2.61|1.82|8014009|edu packmaxi #9|14|tennis|8|Sports|607|ationbarcally|N/A|21654sky015849132647|navajo|Bundle|Unknown|93|eingcallypriationought| +17369|AAAAAAAAINDEAAAA|1999-10-28|2001-10-26|Ships see all|2.98|1.82|7008010|namelessbrand #10|14|lighting|7|Home|607|ationbarcally|N/A|21654sky015849132647|white|Box|Unknown|53|n stcallypriationought| +17370|AAAAAAAAINDEAAAA|2001-10-27||Ships see all|9.56|1.82|4003001|exportiedu pack #1|3|kids|4|Shoes|231|ationbarcally|large|21654sky015849132647|turquoise|N/A|Unknown|16|barationpriationought| +17371|AAAAAAAALNDEAAAA|1997-10-27||Public aspects fail far important, passive years. Very cold numbers appear then; women used to take always prime profits. Conventional matters guide too. Detailed, particular women pass. Just |8.19|7.37|7012002|importonameless #2|12|paint|7|Home|307|ationbarpri|N/A|31101780214green5045|puff|Cup|Unknown|58|oughtationpriationought| +17372|AAAAAAAAMNDEAAAA|1997-10-27|2000-10-26|Now available schools ought to form total, heavy quantities. Impossible, actual employees cut|0.94|0.35|1001001|amalgamalg #1|1|dresses|1|Women|67|ationcally|large|04411408302violet680|yellow|Bunch|Unknown|72|ableationpriationought| +17373|AAAAAAAAMNDEAAAA|2000-10-27||Nevertheless physical allies|6.90|5.86|1001001|namelessunivamalg #17|8|scanners|10|Electronics|67|ationcally|N/A|04411408302violet680|turquoise|Box|Unknown|37|priationpriationought| +17374|AAAAAAAAONDEAAAA|1997-10-27|1999-10-27|So identical details can anticipate however new guards. Impossible, ideal facts achieve all |9.49|7.02|4001001|amalgedu pack #1|1|womens|4|Shoes|123|priableought|medium|4489229404sienna6739|plum|Gram|Unknown|64|eseationpriationought| +17375|AAAAAAAAONDEAAAA|1999-10-28|2001-10-26|Managerial, english others might not|2.79|7.02|4004002|edu packedu pack #2|4|athletic|4|Shoes|572|ableationanti|medium|542saddle42939896175|rosy|Bunch|Unknown|13|antiationpriationought| +17376|AAAAAAAAONDEAAAA|2001-10-27||Acutely small lists should not ma|68.17|7.02|1002001|importoamalg #1|2|fragrances|1|Women|572|ableationanti|medium|542saddle42939896175|royal|Case|Unknown|63|callyationpriationought| +17377|AAAAAAAABODEAAAA|1997-10-27||Cruel presents shall not stay brothers. Indian, minor wages carry always significantly sorry employees. Right new looks wil|3.76|3.27|9005002|scholarmaxi #2|5|history|9|Books|588|eingeinganti|N/A|88576071396wheat4445|snow|N/A|Unknown|5|ationationpriationought| +17378|AAAAAAAACODEAAAA|1997-10-27|2000-10-26|Problems should not breed barely wise purposes; chinese, fun decisions may read here healthy, political notes. Eyes know comfortably towards a limitations. |1.55|0.57|3002001|importoexporti #1|2|infants|3|Children|16|callyought|large|56895510red591503536|violet|N/A|Unknown|47|eingationpriationought| +17379|AAAAAAAACODEAAAA|2000-10-27||Problems should not breed barely wise purposes; chinese, fun decisions may read here healthy, political notes. Eyes know comfortably towards a limitations. |24.97|12.98|7011010|amalgnameless #10|2|accent|7|Home|16|callyought|N/A|56895510red591503536|steel|Box|Unknown|15|n stationpriationought| +17380|AAAAAAAAEODEAAAA|1997-10-27|1999-10-27|Blue roses change also autonomous horses. Foreign, green patients mean visitors; hardly global others ought to laugh only foreign only proposals. Methods keep further ros|23.68|20.83|8008001|namelessnameless #1|8|outdoor|8|Sports|527|ationableanti|N/A|7988papaya2937410151|mint|N/A|Unknown|29|bareingpriationought| +17381|AAAAAAAAEODEAAAA|1999-10-28|2001-10-26|Joint, permanent years ought to arouse more clear figures. Also local governments shall focus surely. Main, concerned teachers make ne|61.92|28.48|2004002|edu packimporto #2|8|sports-apparel|2|Men|33|pripri|medium|18088dim508692650680|peru|Carton|Unknown|10|oughteingpriationought| +17382|AAAAAAAAEODEAAAA|2001-10-27||Stories cry already always local attacks. Large members look at least international females. Already public lim|9.32|28.48|2004002|scholarnameless #3|8|fitness|8|Sports|33|pripri|N/A|18088dim508692650680|turquoise|Ton|Unknown|21|ableeingpriationought| +17383|AAAAAAAAHODEAAAA|1997-10-27||Longer exclusive stands used to build. New, working friends come on |0.98|0.56|3001002|amalgexporti #2|1|newborn|3|Children|184|eseeingought|petite|50264668rose72199010|gainsboro|Unknown|Unknown|11|prieingpriationought| +17384|AAAAAAAAIODEAAAA|1997-10-27|2000-10-26|Inevitabl|9.62|5.19|10008011|namelessunivamalg #11|8|scanners|10|Electronics|13|priought|N/A|2278154223sandy38799|wheat|Ton|Unknown|14|eseeingpriationought| +17385|AAAAAAAAIODEAAAA|2000-10-27||Horizontal, true laws used to delay front, possible rates; other eyes say. Positive, liberal children will not replace for example pale trees. Cruel, british times provide references. G|3.84|2.84|10008011|exportiimporto #2|8|pants|2|Men|13|priought|medium|5010587seashell85100|chiffon|Dram|Unknown|37|antieingpriationought| +17386|AAAAAAAAKODEAAAA|1997-10-27|1999-10-27|Hotels think others; possible guests used to take acutely subject to the cases. Familiar bombs run just new levels. Films lose possible, exotic claims; great, critical poun|7.68|3.53|5001001|amalgscholar #1|1|rock|5|Music|873|priationeing|N/A|940172469rosy0588345|ghost|Gram|Unknown|6|callyeingpriationought| +17387|AAAAAAAAKODEAAAA|1999-10-28|2001-10-26|Most increased losses shall not buy below loans. Merely strategic reports come on a members. Considerable, mad ingredients should conver|45.91|3.53|9001012|amalgmaxi #12|1|arts|9|Books|609|n stbarcally|N/A|323sienna35288495235|midnight|Tsp|Unknown|28|ationeingpriationought| +17388|AAAAAAAAKODEAAAA|2001-10-27||Central powers proceed in a levels. Jointly hard roads will|7.67|3.53|9001001|amalgmaxi #1|1|arts|9|Books|609|n stbarcally|N/A|323sienna35288495235|aquamarine|Oz|Unknown|13|eingeingpriationought| +17389|AAAAAAAANODEAAAA|1997-10-27||More mental banks must create particularly unlikely industries. Coherent files solve. Students should|9.31|6.70|10005016|scholarunivamalg #16|5|karoke|10|Electronics|440|bareseese|N/A|4972027moccasin45531|seashell|Tbl|Unknown|41|n steingpriationought| +17390|AAAAAAAAOODEAAAA|1997-10-27|2000-10-26|Special, hard creditors stay. Uncomfortably specific relationships would not threaten rapidly. Recently new points make too. Often other services ought to use. Jobs cause groups. Americans era|7.06|5.01|4004001|edu packedu pack #1|4|athletic|4|Shoes|230|barpriable|medium|4765645grey997129975|navajo|Box|Unknown|29|barn stpriationought| +17391|AAAAAAAAOODEAAAA|2000-10-27||Special, hard creditors stay. Uncomfortably specific relationships would not threaten rapidly. Recently new points make too. Often other services ought to use. Jobs cause groups. Americans era|7.94|3.01|4004001|exportiedu pack #2|4|kids|4|Shoes|306|barpriable|small|821spring56913820570|sandy|Unknown|Unknown|96|oughtn stpriationought| +17392|AAAAAAAAAPDEAAAA||1999-10-27||||8003003|exportinameless #3|3|basketball||||oughtantipri|||peach||Unknown||ablen stpriationought| +17393|AAAAAAAAAPDEAAAA|1999-10-28|2001-10-26|Classes take genuine proposals. Today private points can use helplessly future scores. Left inches should understand less significant forms. Pleasant, personal trees |5.37|3.97|7015004|scholarnameless #4|15|tables|7|Home|351|oughtantipri|N/A|305456274061papaya28|almond|Lb|Unknown|27|prin stpriationought| +17394|AAAAAAAAAPDEAAAA|2001-10-27||Classes take genuine proposals. Today private points can use helplessly future scores. Left inches should understand less significant forms. Pleasant, personal trees |4.26|3.97|7010005|univnameless #5|15|flatware|7|Home|591|oughtn stanti|N/A|305456274061papaya28|yellow|Ounce|Unknown|32|esen stpriationought| +17395|AAAAAAAADPDEAAAA|1997-10-27||Others play. Totally new appearances should not pray strong campaigns; normal, new doors bear still. Skills prove then golden types. Additional, high things make; probably s|1.99|1.21|4002002|importoedu pack #2|2|mens|4|Shoes|107|ationbarought|petite|070596275slate362666|purple|Unknown|Unknown|5|antin stpriationought| +17396|AAAAAAAAEPDEAAAA|1997-10-27|2000-10-26|Weeks sh|2.87|2.46|10008006|namelessunivamalg #6|8|scanners|10|Electronics|995|antin stn st|N/A|0795629papaya0323248|blanched|Ton|Unknown|15|callyn stpriationought| +17397|AAAAAAAAEPDEAAAA|2000-10-27||Right maps used to supplement men; both mad cars shall not state personally. Difficult points may not discard parties; resources think national, rural stages. Only|4.79|2.46|10008006|amalgamalg #2|1|dresses|1|Women|995|antin stn st|medium|167718953powder77727|tomato|Dozen|Unknown|63|ationn stpriationought| +17398|AAAAAAAAGPDEAAAA|1997-10-27|1999-10-27|Even old knees describe for a politicians. So real stories used to come somewhat electrical typical illustrations. Years take likely,|9.80|6.76|6016007|corpbrand #7|16|consignment|6|Jewelry|219|n stoughtable|N/A|1456233966tan9410783|violet|Lb|Unknown|59|eingn stpriationought| +17399|AAAAAAAAGPDEAAAA|1999-10-28|2001-10-26|Unwittingly western conditions should go little marks. Short departments control maybe for the records; necessarily able o|9.09|5.90|6016007|importonameless #10|12|paint|7|Home|846|n stoughtable|N/A|96peach6210605111782|goldenrod|Box|Unknown|26|n stn stpriationought| +17400|AAAAAAAAGPDEAAAA|2001-10-27||Broadly basic members think again about a reforms. Supporters should terminate there able problems. Really wrong pupils cope of course. Regional times get patie|4.10|3.23|6016007|importobrand #1|12|costume|6|Jewelry|846|n stoughtable|N/A|96peach6210605111782|papaya|Each|Unknown|75|barbareseationought| +17401|AAAAAAAAJPDEAAAA|1997-10-27||Substantial, political authorities restrict supreme, global forces. Single feelings may deny unique forces. Colleges take eu|9.83|7.96|2004002|edu packimporto #2|4|sports-apparel|2|Men|18|eingought|medium|38pink64517633405229|grey|Unknown|Unknown|5|oughtbareseationought| +17402|AAAAAAAAKPDEAAAA|1997-10-27|2000-10-26|Simple parties guard capital museums. Abilities distort preliminary minutes. Extended markets must not obtain too various strategies. Arguments might not help by the families. |4.19|2.13|5003001|exportischolar #1|3|pop|5|Music|165|anticallyought|N/A|4720436801antique828|ghost|Bunch|Unknown|94|ablebareseationought| +17403|AAAAAAAAKPDEAAAA|2000-10-27||Simple parties guard capital museums. Abilities distort preliminary minutes. Extended markets must not obtain too various strategies. Arguments might not help by the families. |3.48|2.13|1004002|edu packamalg #2|4|swimwear|1|Women|355|anticallyought|extra large|9167558metallic08942|sky|Bunch|Unknown|5|pribareseationought| +17404|AAAAAAAAMPDEAAAA|1997-10-27|1999-10-27|Psychiatric reasons make in the classes. Sides ought to think schools. Sites cannot give more. Interesting, personal years should drink with a items. F|8.97|6.63|10006014|corpunivamalg #14|6|musical|10|Electronics|131|oughtpriought|N/A|55134094seashell7005|peach|Pallet|Unknown|26|esebareseationought| +17405|AAAAAAAAMPDEAAAA|1999-10-28|2001-10-26|Always sure effects shall tell new sales. Alternative circumstances will bear other, bad arms. Years could see eno|91.90|41.35|2004002|edu packimporto #2|6|sports-apparel|2|Men|267|ationcallyable|petite|6thistle863898373322|medium|Ounce|Unknown|43|antibareseationought| +17406|AAAAAAAAMPDEAAAA|2001-10-27||Always sure effects shall tell new sales. Alternative circumstances will bear other, bad arms. Years could see eno|2.41|1.49|7003007|exportibrand #7|3|kids|7|Home|267|ationcallyable|N/A|79288362sandy2890003|thistle|Bundle|Unknown|33|callybareseationought| +17407|AAAAAAAAPPDEAAAA|1997-10-27||Authorities design through a individuals. Temporary, int|95.84|42.16|8004008|edu packnameless #8|4|camping|8|Sports|311|oughtoughtpri|N/A|3284brown21333077991|yellow|Lb|Unknown|48|ationbareseationought| +17408|AAAAAAAAAAEEAAAA|1997-10-27|2000-10-26|Earnings reduce contrary men. Tomorrow good cheeks say genes; children may not involve only. Examp|9.32|5.03|10009001|maxiunivamalg #1|9|televisions|10|Electronics|860|barcallyeing|N/A|755627971lemon129974|rose|Each|Unknown|36|eingbareseationought| +17409|AAAAAAAAAAEEAAAA|2000-10-27||Pupils shall n|7.93|6.50|10009001|edu packmaxi #10|14|tennis|8|Sports|213|prioughtable|N/A|white291529621534448|white|Tbl|Unknown|12|n stbareseationought| +17410|AAAAAAAACAEEAAAA|1997-10-27|1999-10-27|Parties get still by the farmers; other tickets can occur sometimes concerned nurses. Enough old systems should carry almost|6.07|4.97|3001001|amalgexporti #1|1|newborn|3|Children|553|priantianti|small|461peru7080632970870|plum|Lb|Unknown|47|baroughteseationought| +17411|AAAAAAAACAEEAAAA|1999-10-28|2001-10-26|Now available changes|2.24|1.20|9016012|corpunivamalg #12|1|mystery|9|Books|553|priantianti|N/A|461peru7080632970870|white|Pallet|Unknown|35|oughtoughteseationought| +17412|AAAAAAAACAEEAAAA|2001-10-27||Now available changes|1.43|1.20|7002003|importobrand #3|1|bedding|7|Home|229|n stableable|N/A|461peru7080632970870|orange|Unknown|Unknown|36|ableoughteseationought| +17413|AAAAAAAAFAEEAAAA|1997-10-27||New markets come faintly different readers; possible, right claims should apply most future, elderly functions. Free others result deliberately efforts.|6.47|4.27|2003002|exportiimporto #2|3|pants|2|Men|207|ationbarable|extra large|7259azure59365384600|steel|Tbl|Unknown|29|prioughteseationought| +17414|AAAAAAAAGAEEAAAA|1997-10-27|2000-10-26|Considerably major difficulties shall decide. Other hands open warm subjects. Views shall make national, poor modules. Elements cannot hope once earnings. Most bad conditions might realise never recen|4.67|3.78|5003001|exportischolar #1|3|pop|5|Music|410|baroughtese|N/A|23824780666midnight9|rose|Tbl|Unknown|78|eseoughteseationought| +17415|AAAAAAAAGAEEAAAA|2000-10-27||Considerably major difficulties shall decide. Other hands open warm subjects. Views shall make national, poor modules. Elements cannot hope once earnings. Most bad conditions might realise never recen|0.35|0.11|10003008|exportiunivamalg #8|3|dvd/vcr players|10|Electronics|410|baroughtese|N/A|23824780666midnight9|honeydew|Oz|Unknown|19|antioughteseationought| +17416|AAAAAAAAIAEEAAAA|1997-10-27|1999-10-27|Eyes o|2.28|1.09|3002001|importoexporti #1|2|infants|3|Children|363|pricallypri|petite|8659640942452medium3|seashell|Tsp|Unknown|39|callyoughteseationought| +17417|AAAAAAAAIAEEAAAA|1999-10-28|2001-10-26|Australian, electoral conditions should not provide tough, old holes. Bad workers might not form. Foreign, extra details so|47.91|1.09|1002002|importoamalg #2|2|fragrances|1|Women|106|pricallypri|medium|4214908539957pale728|sky|Case|Unknown|7|ationoughteseationought| +17418|AAAAAAAAIAEEAAAA|2001-10-27||Australian, electoral conditions should not provide tough, old holes. Bad workers might not form. Foreign, extra details so|4.31|1.09|7005009|scholarbrand #9|2|blinds/shades|7|Home|106|callybarought|N/A|1821984791indian6815|red|Tsp|Unknown|64|eingoughteseationought| +17419|AAAAAAAALAEEAAAA|1997-10-27||Great, supreme years can know for example downwards different funds. Just white men shall drain indeed maximum|5.29|4.49|1002002|importoamalg #2|2|fragrances|1|Women|22|ableable|N/A|6587374cyan508591458|slate|Oz|Unknown|4|n stoughteseationought| +17420|AAAAAAAAMAEEAAAA|1997-10-27|2000-10-26|Now mad clouds could not ask closely. Acute, new hundreds should recycle here; angry, simple affairs could dis|7.47|4.78|7011001|amalgnameless #1|11|accent|7|Home|546|callyeseanti|N/A|6131168red3922667311|dodger|Each|Unknown|100|barableeseationought| +17421|AAAAAAAAMAEEAAAA|2000-10-27||Rare students shall indicate apart also normal films. Relations should change only just about real figures. Legitimate a|9.23|4.78|7011001|exportiexporti #2|11|toddlers|3|Children|549|n steseanti|large|8551851green75028709|saddle|Lb|Unknown|21|oughtableeseationought| +17422|AAAAAAAAOAEEAAAA|1997-10-27|1999-10-27|Types may follow in a attitudes. Tory, lovely types could put so on a patients. |1.43|0.91|6013007|exportibrand #7|13|loose stones|6|Jewelry|226|callyableable|N/A|964ghost650120282296|turquoise|Carton|Unknown|19|ableableeseationought| +17423|AAAAAAAAOAEEAAAA|1999-10-28|2001-10-26|Moments must extend in the letters; other ears go correctly general decisions; spanish, difficult settings prove in a recommendatio|2.20|1.84|7004004|edu packbrand #4|13|curtains/drapes|7|Home|226|callyableable|N/A|262237004medium95932|rose|Each|Unknown|96|priableeseationought| +17424|AAAAAAAAOAEEAAAA|2001-10-27||Women head to the rates. Fast australian workers need due papers. Regional limits should work too completely appropriate traders. Basic, soviet things may not examine very in a programmes. Important|1.49|1.84|7004004|amalgscholar #1|13|rock|5|Music|226|callyableable|N/A|262237004medium95932|peach|Bunch|Unknown|23|eseableeseationought| +17425|AAAAAAAABBEEAAAA|1997-10-27||Attractive hands will not mean soon. Other, big hospitals may not get degrees. Probably military feelings meet yesterday major standards; too possible famili|4.44|2.93|4004002|edu packedu pack #2|4|athletic|4|Shoes|313|prioughtpri|medium|54puff16586765652421|white|Pound|Unknown|16|antiableeseationought| +17426|AAAAAAAACBEEAAAA|1997-10-27|2000-10-26|Friends keep important, new plans. Relations receive different, lacking days. Different patterns might believe such as the limits; different, only firms find previous, political candidates.|4.48|2.01|3002001|importoexporti #1|2|infants|3|Children|496|callyn stese|extra large|579747796773438plum6|sky|Pound|Unknown|19|callyableeseationought| +17427|AAAAAAAACBEEAAAA|2000-10-27||Friends keep important, new plans. Relations receive different, lacking days. Different patterns might believe such as the limits; different, only firms find previous, political candidates.|5.26|2.01|3002001|importoamalg #2|2|fragrances|1|Women|496|callyn stese|large|579747796773438plum6|cyan|Tbl|Unknown|10|ationableeseationought| +17428|AAAAAAAAEBEEAAAA|1997-10-27|1999-10-27|Top reactions could evacuate ne|5.74|3.15|1004001|edu packamalg #1|4|swimwear|1|Women|53|prianti|large|34678715753wheat7924|lavender|Gross|Unknown|29|eingableeseationought| +17429|AAAAAAAAEBEEAAAA|1999-10-28|2001-10-26|Top reactions could evacuate ne|0.84|0.75|1004001|edu packamalg #2|4|swimwear|1|Women|36|callypri|economy|34678715753wheat7924|snow|Case|Unknown|32|n stableeseationought| +17430|AAAAAAAAEBEEAAAA|2001-10-27||Topics shape|3.13|2.69|1004001|amalgimporto #1|4|accessories|2|Men|36|callypri|large|27617766turquoise187|gainsboro|Cup|Unknown|28|barprieseationought| +17431|AAAAAAAAHBEEAAAA|1997-10-27||Different aspects get now available hands. Political, physical sums live as later local territories. Legislative, wrong edges must know as well orange,|5.25|1.78|10009007|maxiunivamalg #7|9|televisions|10|Electronics|183|prieingought|N/A|violet60545072135609|purple|Each|Unknown|41|oughtprieseationought| +17432|AAAAAAAAIBEEAAAA|1997-10-27|2000-10-26|Fields should not see then alleged police. Temporary, poor things may not escape crucial, mai|89.65|53.79|6005001|scholarcorp #1|5|earings|6|Jewelry|317|ationoughtpri|N/A|221925186354snow9604|rosy|Lb|Unknown|1|ableprieseationought| +17433|AAAAAAAAIBEEAAAA|2000-10-27||Fields should not see then alleged police. Temporary, poor things may not escape crucial, mai|9.70|53.79|6005001|exportischolar #2|3|pop|5|Music|317|ationoughtpri|N/A|7279violet9086077623|linen|Bunch|Unknown|85|priprieseationought| +17434|AAAAAAAAKBEEAAAA|1997-10-27|1999-10-27|Accurate children will help only european claims. Delighted assets wou|7.67|3.45|7003005|exportibrand #5|3|kids|7|Home|247|ationeseable|N/A|301452light351070413|purple|Bunch|Unknown|19|eseprieseationought| +17435|AAAAAAAAKBEEAAAA|1999-10-28|2001-10-26|Statistical activities love however as well other police; products avoid years. Original, overall months will not include now single p|6.42|3.45|7003005|exportiexporti #2|3|toddlers|3|Children|247|ationeseable|petite|9660482indian6464872|drab|Gram|Unknown|53|antiprieseationought| +17436|AAAAAAAAKBEEAAAA|2001-10-27||Statistical activities love however as well other police; products avoid years. Original, overall months will not include now single p|9.61|3.45|4002001|importoedu pack #1|2|mens|4|Shoes|138|ationeseable|medium|75368425640876sandy3|sienna|Gross|Unknown|26|callyprieseationought| +17437|AAAAAAAANBEEAAAA|1997-10-27||Basically far records take gently immediate, insufficient committees; most unacceptable cla|3.06|2.72|3003002|exportiexporti #2|3|toddlers|3|Children|496|callyn stese|large|5730830488chiffon388|indian|Carton|Unknown|91|ationprieseationought| +17438|AAAAAAAAOBEEAAAA|1997-10-27|2000-10-26|Reaso|3.46|2.83|6003007|exporticorp #7|3|gold|6|Jewelry|54|eseanti|N/A|yellow39637095444524|maroon|Bunch|Unknown|31|eingprieseationought| +17439|AAAAAAAAOBEEAAAA|2000-10-27||Likely, specific exhibitions may believe abstract games. Remote, bad hills help as around available missiles. More reliable films would take well at a schools. Normally working sheets write. Only c|1.61|0.70|1004002|edu packamalg #2|4|swimwear|1|Women|492|eseanti|petite|293420ghost378701202|plum|Gross|Unknown|34|n stprieseationought| +17440|AAAAAAAAACEEAAAA|1997-10-27|1999-10-27|Thus great foreigners would supervise therefore also likely developments. Crucial years could break this large|1.81|1.46|7011001|amalgnameless #1|11|accent|7|Home|110|baroughtought|N/A|6223yellow9062401124|moccasin|Box|Unknown|13|bareseeseationought| +17441|AAAAAAAAACEEAAAA|1999-10-28|2001-10-26|Charges want. P|2.02|1.55|4001002|amalgedu pack #2|11|womens|4|Shoes|110|baroughtought|small|6223yellow9062401124|tomato|Dram|Unknown|7|oughteseeseationought| +17442|AAAAAAAAACEEAAAA|2001-10-27||Stairs used to find surely |84.76|1.55|4001002|scholaramalgamalg #1|11|portable|10|Electronics|110|baroughtought|N/A|6223yellow9062401124|steel|Unknown|Unknown|12|ableeseeseationought| +17443|AAAAAAAADCEEAAAA|1997-10-27||Act|2.70|1.91|9014008|edu packunivamalg #8|14|sports|9|Books|266|callycallyable|N/A|4652002827329violet8|midnight|Dozen|Unknown|63|prieseeseationought| +17444|AAAAAAAAECEEAAAA|1997-10-27|2000-10-26|Too interesting emissions return just large, excessive plants. Real democrats will g|4.95|4.40|6008005|namelesscorp #5|8|mens watch|6|Jewelry|191|oughtn stought|N/A|513steel476753344275|misty|Lb|Unknown|27|eseeseeseationought| +17445|AAAAAAAAECEEAAAA|2000-10-27||Then brief tonnes must make personal, popular benefits; light, obvious partners would dry enough for the institutions; enough g|9.51|3.13|9004010|edu packmaxi #10|4|entertainments|9|Books|260|barcallyable|N/A|3646373867943sandy38|grey|Pallet|Unknown|30|antieseeseationought| +17446|AAAAAAAAGCEEAAAA|1997-10-27|1999-10-27|Tonight soviet windows should make well in a children; foreign, federal amounts hold cars. Just old characteristics might pick slight, able |6.08|4.13|5003001|exportischolar #1|3|pop|5|Music|439|n stpriese|N/A|3474deep196899556971|spring|Gram|Unknown|31|callyeseeseationought| +17447|AAAAAAAAGCEEAAAA|1999-10-28|2001-10-26|Tonight soviet windows should make well in a children; foreign, federal amounts hold cars. Just old characteristics might pick slight, able |4.64|4.13|10009006|maxiunivamalg #6|9|televisions|10|Electronics|568|n stpriese|N/A|742742245pale8387155|powder|Tsp|Unknown|25|ationeseeseationought| +17448|AAAAAAAAGCEEAAAA|2001-10-27||Civil names may achieve lightly solely wrong peasants. Chinese, managerial years may give more disastrous arrangements. Public, solid connections mean extremely typical chiefs. Flowers |2.01|1.30|10009006|importobrand #9|2|bedding|7|Home|52|ableanti|N/A|742742245pale8387155|rose|Box|Unknown|34|eingeseeseationought| +17449|AAAAAAAAJCEEAAAA|1997-10-27||Careful, mature forces could not accept as by a rates. Complete, liberal doors use. Yesterday actual personnel settle with a features; less |4.54|4.08|5001002|amalgscholar #2|1|rock|5|Music|289|n steingable|N/A|3814369646225slate73|magenta|Oz|Unknown|63|n steseeseationought| +17450|AAAAAAAAKCEEAAAA|1997-10-27|2000-10-26|Kings could grow just however safe achievements. Always local resources shall freeze so other victims. Trying, material office|3.89|1.55|9008003|namelessmaxi #3|8|romance|9|Books|401|oughtbarese|N/A|259423462997lawn0207|papaya|Case|Unknown|4|barantieseationought| +17451|AAAAAAAAKCEEAAAA|2000-10-27||Kings could grow just however safe achievements. Always local resources shall freeze so other victims. Trying, material office|1.38|1.02|10010010|univamalgamalg #10|8|memory|10|Electronics|392|oughtbarese|N/A|259423462997lawn0207|mint|Unknown|Unknown|13|oughtantieseationought| +17452|AAAAAAAAMCEEAAAA|1997-10-27|1999-10-27|Great, true teachers might get supposed seats. Very small others appe|2.23|0.82|3001001|amalgexporti #1|1|newborn|3|Children|309|n stbarpri|large|0899137275010maroon7|peru|Each|Unknown|94|ableantieseationought| +17453|AAAAAAAAMCEEAAAA|1999-10-28|2001-10-26|Great, true teachers might get supposed seats. Very small others appe|4.79|3.06|3001001|edu packbrand #4|4|curtains/drapes|7|Home|35|n stbarpri|N/A|35758511331722pink82|thistle|N/A|Unknown|44|priantieseationought| +17454|AAAAAAAAMCEEAAAA|2001-10-27||Great, true teachers might get supposed seats. Very small others appe|8.30|5.64|3001001|amalgnameless #5|4|athletic shoes|8|Sports|35|n stbarpri|N/A|35758511331722pink82|violet|Tbl|Unknown|46|eseantieseationought| +17455|AAAAAAAAPCEEAAAA|1997-10-27||Incidentally productive windows must not fill tightl|1.89|0.60|1004002|edu packamalg #2|4|swimwear|1|Women|574|eseationanti|medium|675207873185peru5026|royal|Oz|Unknown|33|antiantieseationought| +17456|AAAAAAAAADEEAAAA|1997-10-27|2000-10-26|Apparent, solid banks see never new pages; equivalent, growing children must change much in a sides.|6.21|2.23|5004001|edu packscholar #1|4|classical|5|Music|503|pribaranti|N/A|2142163228115plum069|rosy|Dram|Unknown|20|callyantieseationought| +17457|AAAAAAAAADEEAAAA|2000-10-27||Welcome, royal|3.53|2.85|4003002|exportiedu pack #2|4|kids|4|Shoes|507|pribaranti|petite|2142163228115plum069|wheat|Dram|Unknown|1|ationantieseationought| +17458|AAAAAAAACDEEAAAA|1997-10-27|1999-10-27|Correct feet shall not ob|3.53|3.17|5004001|edu packscholar #1|4|classical|5|Music|294|esen stable|N/A|20340879763560tan891|pale|Pallet|Unknown|28|eingantieseationought| +17459|AAAAAAAACDEEAAAA|1999-10-28|2001-10-26|A|0.17|3.17|2002002|importoimporto #2|2|shirts|2|Men|294|esen stable|medium|308tan52111761312910|tan|Box|Unknown|34|n stantieseationought| +17460|AAAAAAAACDEEAAAA|2001-10-27||A|4.97|3.17|9001001|amalgmaxi #1|2|arts|9|Books|244|esen stable|N/A|308tan52111761312910|royal|Oz|Unknown|10|barcallyeseationought| +17461|AAAAAAAAFDEEAAAA|1997-10-27||Feelings sleep at a details. Also competitive devices shall object early in every sales. Almost other ways offer once free tools. Significant, german sheets keep hardl|7.15|2.78|7001002|amalgbrand #2|1|bathroom|7|Home|184|eseeingought|N/A|453534839203saddle07|orchid|Case|Unknown|3|oughtcallyeseationought| +17462|AAAAAAAAGDEEAAAA|1997-10-27|2000-10-26|Carefully european characters drop foreigners. Foreign funds wear; silver, empty councils use personally positive, english matters. Servic|6.37|4.84|8016003|corpmaxi #3|16|golf|8|Sports|173|priationought|N/A|99947turquoise095644|red|Gram|Unknown|52|ablecallyeseationought| +17463|AAAAAAAAGDEEAAAA|2000-10-27||Costs teach major, real races. Joint players might not kill particular men. Recent, signific|4.49|2.06|8016003|edu packedu pack #2|4|athletic|4|Shoes|173|priationought|large|99947turquoise095644|peach|Ounce|Unknown|61|pricallyeseationought| +17464|AAAAAAAAIDEEAAAA|1997-10-27|1999-10-27|Later fatal rules save only books. Principal examples could go no longer more unacceptable readers. Needs can blame quite. More careful parties enjoy so constant, cheap systems. Degrees know. Ne|48.23|22.18|5003001|exportischolar #1|3|pop|5|Music|392|ablen stpri|N/A|923411spring86131679|metallic|Ounce|Unknown|10|esecallyeseationought| +17465|AAAAAAAAIDEEAAAA|1999-10-28|2001-10-26|Hard professional computers should not leave gmt groups. Never experimental troop|0.24|0.08|5003001|edu packamalgamalg #7|14|automotive|10|Electronics|392|ablen stpri|N/A|923411spring86131679|lace|Box|Unknown|14|anticallyeseationought| +17466|AAAAAAAAIDEEAAAA|2001-10-27||Hard professional computers should not leave gmt groups. Never experimental troop|8.55|6.75|5003001|edu packexporti #1|14|school-uniforms|3|Children|225|ablen stpri|medium|923411spring86131679|smoke|Tsp|Unknown|31|callycallyeseationought| +17467|AAAAAAAALDEEAAAA|1997-10-27||Other trains shall not help deliberately feet; conferences would feel yet southern weeks; bad studies would specialise please just human items; global sections|9.69|5.23|10002009|importounivamalg #9|2|camcorders|10|Electronics|70|baration|N/A|668indian55778492825|yellow|Pallet|Unknown|49|ationcallyeseationought| +17468|AAAAAAAAMDEEAAAA|1997-10-27|2000-10-26|Total children used to find men. Carers build. Important, statutory heads write at the points; mar|6.59|2.24|7003009|exportibrand #9|3|kids|7|Home|280|bareingable|N/A|801871637806488pink5|violet|Unknown|Unknown|22|eingcallyeseationought| +17469|AAAAAAAAMDEEAAAA|2000-10-27||Total children used to find men. Carers build. Important, statutory heads write at the points; mar|8.43|2.24|4002002|importoedu pack #2|3|mens|4|Shoes|280|bareingable|small|1689694440316679tan0|violet|N/A|Unknown|4|n stcallyeseationought| +17470|AAAAAAAAODEEAAAA|1997-10-27|1999-10-27|Low sorts understand. Vegetables must not carry. There legal rates shake so democratic styles. Convenient, single committees might forget|7.16|5.79|4001001|amalgedu pack #1|1|womens|4|Shoes|729|n stableation|medium|7salmon7687979127744|salmon|N/A|Unknown|92|barationeseationought| +17471|AAAAAAAAODEEAAAA|1999-10-28|2001-10-26|General managers may not give. Fine costs c|3.24|2.13|4001001|importounivamalg #12|1|home repair|9|Books|729|n stableation|N/A|8813078833274peru273|powder|Case|Unknown|4|oughtationeseationought| +17472|AAAAAAAAODEEAAAA|2001-10-27||General managers may not give. Fine costs c|8.61|2.92|4001001|amalgimporto #1|1|accessories|2|Men|24|n stableation|medium|8813078833274peru273|lemon|Unknown|Unknown|13|ableationeseationought| +17473|AAAAAAAABEEEAAAA|1997-10-27||Waves should leave home black customers. Differences should see forwards words. Unexpected, unexp|3.50|1.61|1001002|amalgamalg #2|1|dresses|1|Women|551|oughtantianti|large|9692457825papaya4521|chocolate|Tsp|Unknown|98|priationeseationought| +17474|AAAAAAAACEEEAAAA|1997-10-27|2000-10-26|Now different families see |2.18|1.43|10014016|edu packamalgamalg #16|14|automotive|10|Electronics|946|callyesen st|N/A|47995258316steel6700|medium|Carton|Unknown|13|eseationeseationought| +17475|AAAAAAAACEEEAAAA|2000-10-27||Now different families see |4.80|1.43|10001017|amalgunivamalg #17|1|cameras|10|Electronics|77|callyesen st|N/A|47995258316steel6700|sandy|Lb|Unknown|23|antiationeseationought| +17476|AAAAAAAAEEEEAAAA|1997-10-27|1999-10-27|Aspects may get also able, good days; new, wise windows retain partly overall hours. Imaginative, extensive characteristics could sign too manufacturing, external ani|16.99|12.74|4004001|edu packedu pack #1|4|athletic|4|Shoes|117|ationoughtought|medium|01668sandy7939369157|powder|Oz|Unknown|11|callyationeseationought| +17477|AAAAAAAAEEEEAAAA|1999-10-28|2001-10-26|Broad times can secure. Studies provide largely individuals. Fals|2.19|1.29|3004002|edu packexporti #2|4|school-uniforms|3|Children|117|ationoughtought|large|01668sandy7939369157|rose|Each|Unknown|33|ationationeseationought| +17478|AAAAAAAAEEEEAAAA|2001-10-27||About unable rats ought to lose so friends. Still annual rooms want enough such as a accountants. Connections must cope again for an problems. Much western men make trees. Shares |6.77|1.29|3004002|edu packimporto #1|4|sports-apparel|2|Men|117|ationoughtought|large|tomato97932596042109|cyan|Ounce|Unknown|6|eingationeseationought| +17479|AAAAAAAAHEEEAAAA|1997-10-27||Plans secure sometimes physical, clinical costs. Representative, front symbols achieve possibly supposed wages. Nevertheless essential |2.04|1.48|7013010|exportinameless #10|13|wallpaper|7|Home|199|n stn stought|N/A|23269050093red147924|slate|Tbl|Unknown|11|n stationeseationought| +17480|AAAAAAAAIEEEAAAA|1997-10-27|2000-10-26|Firms lead by the followers. Estimated, rigid probl|16.16|12.76|7009001|maxibrand #1|9|mattresses|7|Home|252|ableantiable|N/A|9dim2711859836826677|white|Dozen|Unknown|32|bareingeseationought| +17481|AAAAAAAAIEEEAAAA|2000-10-27||Firms lead by the followers. Estimated, rigid probl|2.42|12.76|7015002|scholarnameless #2|9|tables|7|Home|252|ableantiable|N/A|31366powder945075158|seashell|Each|Unknown|15|oughteingeseationought| +17482|AAAAAAAAKEEEAAAA|1997-10-27|1999-10-27|Now other jobs can expect also immediate materials. Homes look arrangements; minutes will change. En|4.32|2.28|2002001|importoimporto #1|2|shirts|2|Men|84|eseeing|petite|97orchid852445559858|wheat|Case|Unknown|24|ableeingeseationought| +17483|AAAAAAAAKEEEAAAA|1999-10-28|2001-10-26|Now other jobs can expect also immediate materials. Homes look arrangements; minutes will change. En|4.35|2.28|2002001|amalgamalg #2|2|dresses|1|Women|84|eseeing|small|97orchid852445559858|steel|Lb|Unknown|55|prieingeseationought| +17484|AAAAAAAAKEEEAAAA|2001-10-27||Now other jobs can expect also immediate materials. Homes look arrangements; minutes will change. En|8.53|4.26|2002001|amalgbrand #5|2|semi-precious|6|Jewelry|41|oughtese|N/A|97orchid852445559858|tan|Pallet|Unknown|5|eseeingeseationought| +17485|AAAAAAAANEEEAAAA|1997-10-27||Even s|1.45|0.71|8016006|corpmaxi #6|16|golf|8|Sports|186|callyeingought|N/A|31409738white7522799|spring|Dram|Unknown|22|antieingeseationought| +17486|AAAAAAAAOEEEAAAA|1997-10-27|2000-10-26|British relationships used to keep to a orders; silent, flexible countries would make unknown |4.52|3.52|6010007|univbrand #7|10|jewelry boxes|6|Jewelry|578|eingationanti|N/A|0172royal53497657024|midnight|Bundle|Unknown|23|callyeingeseationought| +17487|AAAAAAAAOEEEAAAA|2000-10-27||British relationships used to keep to a orders; silent, flexible countries would make unknown |4.91|3.52|6010007|exportiunivamalg #4|10|self-help|9|Books|578|eingationanti|N/A|0172royal53497657024|chocolate|N/A|Unknown|16|ationeingeseationought| +17488|AAAAAAAAAFEEAAAA|1997-10-27|1999-10-27|Cold, able years must give. Southern actions |6.26|4.88|10011013|amalgamalgamalg #13|11|disk drives|10|Electronics|443|prieseese|N/A|1268168632139rose383|rosy|Unknown|Unknown|66|eingeingeseationought| +17489|AAAAAAAAAFEEAAAA|1999-10-28|2001-10-26|Steady resources foster relatively more than other circumstances; old areas destroy low. Free developments take however opportunit|0.49|4.88|9014006|edu packunivamalg #6|14|sports|9|Books|238|eingpriable|N/A|1268168632139rose383|powder|Gross|Unknown|50|n steingeseationought| +17490|AAAAAAAAAFEEAAAA|2001-10-27||Experiences cannot serve more than secret levels. Benefits open months. Acute, available goals might |8.13|4.88|10015015|scholaramalgamalg #15|15|portable|10|Electronics|238|eingpriable|N/A|1268168632139rose383|sandy|Carton|Unknown|27|barn steseationought| +17491|AAAAAAAADFEEAAAA|1997-10-27||Doctors cannot answer more willing boxes. Now pleasant youngsters pick utterly changes; only changing writers help also joint |8.48|7.29|10006016|corpunivamalg #16|6|musical|10|Electronics|217|ationoughtable|N/A|337855612385plum4172|yellow|Box|Unknown|50|oughtn steseationought| +17492|AAAAAAAAEFEEAAAA|1997-10-27|2000-10-26|Strategic, tiny minutes own recently bef|4.98|3.68|3004001|edu packexporti #1|4|school-uniforms|3|Children|47|ationese|medium|72light7880310778410|red|Gram|Unknown|18|ablen steseationought| +17493|AAAAAAAAEFEEAAAA|2000-10-27||Strategic, tiny minutes own recently bef|0.53|0.19|8004004|edu packnameless #4|4|camping|8|Sports|262|ablecallyable|N/A|72light7880310778410|khaki|Dozen|Unknown|57|prin steseationought| +17494|AAAAAAAAGFEEAAAA|1997-10-27|1999-10-27|Loc|1.51|0.93|4004001|edu packedu pack #1|4|athletic|4|Shoes|275|antiationable|medium|843546090674808peru5|salmon|Oz|Unknown|31|esen steseationought| +17495|AAAAAAAAGFEEAAAA|1999-10-28|2001-10-26|Loc|3.23|1.84|4004001|importoedu pack #2|2|mens|4|Shoes|461|oughtcallyese|small|54724peach9599687163|dodger|Cup|Unknown|3|antin steseationought| +17496|AAAAAAAAGFEEAAAA|2001-10-27||Still inadequ|22.35|11.39|4004001|brandunivamalg #4|2|personal|10|Electronics|461|oughtcallyese|N/A|2chocolate5641222558|purple|Bunch|Unknown|59|callyn steseationought| +17497|AAAAAAAAJFEEAAAA|1997-10-27||Quite welsh costs agree specially results. Goth|1.83|0.82|9016002|corpunivamalg #2|16|mystery|9|Books|750|barantiation|N/A|5492903562lemon99832|seashell|Case|Unknown|33|ationn steseationought| +17498|AAAAAAAAKFEEAAAA|1997-10-27|2000-10-26|Detailed problems clear likely difficulties. Central projects should play quiet, certain |57.28|21.19|5003001|exportischolar #1|3|pop|5|Music|135|antipriought|N/A|876506998802356rose9|orchid|Gram|Unknown|18|eingn steseationought| +17499|AAAAAAAAKFEEAAAA|2000-10-27||Royal, elderly problems may not lift all but. Effective cuts spend forward. Large, wonderfu|97.29|21.19|5003001|namelessunivamalg #7|3|scanners|10|Electronics|127|antipriought|N/A|876506998802356rose9|sky|Ounce|Unknown|65|n stn steseationought| +17500|AAAAAAAAMFEEAAAA|1997-10-27|1999-10-27|Reforms must not warn very central patterns. Basic problems might regulate more certain professionals. Friends work. Already coming fund|1.79|0.91|6016005|corpbrand #5|16|consignment|6|Jewelry|226|callyableable|N/A|272885172white364392|pale|Bunch|Unknown|32|barbarantiationought| +17501|AAAAAAAAMFEEAAAA|1999-10-28|2001-10-26|Reforms must not warn very central patterns. Basic problems might regulate more certain professionals. Friends work. Already coming fund|6.06|0.91|6016005|exportiamalg #2|3|maternity|1|Women|276|callyableable|extra large|200yellow85640757570|spring|Tbl|Unknown|31|oughtbarantiationought| +17502|AAAAAAAAMFEEAAAA|2001-10-27||Local, other results demand. Normal, good heels raise unusually in a teeth. Safe employees must not hear most. Unemployed, social |2.23|1.13|2001001|amalgimporto #1|3|accessories|2|Men|276|callyationable|small|05318067369wheat2806|snow|Unknown|Unknown|12|ablebarantiationought| +17503|AAAAAAAAPFEEAAAA|1997-10-27||High, financial doctors provide active instances. Conditions may result to a things. Back, extended relati|1.18|0.94|2004002|edu packimporto #2|4|sports-apparel|2|Men|137|ationpriought|economy|5steel77297560598201|orange|Ton|Unknown|12|pribarantiationought| +17504|AAAAAAAAAGEEAAAA|1997-10-27|2000-10-26|Acts disclose even beautiful developments. Thankfully correct games stand good signal|0.18|0.07|1003001|exportiamalg #1|3|maternity|1|Women|143|prieseought|large|671441567snow3749180|honeydew|Case|Unknown|65|esebarantiationought| +17505|AAAAAAAAAGEEAAAA|2000-10-27||Well italian origins |4.22|3.24|1003001|univamalgamalg #13|10|memory|10|Electronics|112|ableoughtought|N/A|75364380steel3065968|rosy|Box|Unknown|98|antibarantiationought| +17506|AAAAAAAACGEEAAAA|1997-10-27|1999-10-27|National facilities should not fail local, conti|0.64|0.21|5003001|exportischolar #1|3|pop|5|Music|113|prioughtought|N/A|13312188tomato477401|puff|Tbl|Unknown|28|callybarantiationought| +17507|AAAAAAAACGEEAAAA|1999-10-28|2001-10-26|African problems prove in every payments. Owners see men. Strangers show to a increases. Possibilities enter very at a firms. Bars use. International pla|1.91|1.52|6012004|importobrand #4|12|costume|6|Jewelry|273|prioughtought|N/A|13312188tomato477401|yellow|N/A|Unknown|52|ationbarantiationought| +17508|AAAAAAAACGEEAAAA|2001-10-27||Rights mean; black dreams could say however alone things; girls wear prayers; years like in a arrangements; national, irish branches include generally male members; other, important workers shall |99.04|1.52|5003001|exportischolar #1|12|pop|5|Music|273|prioughtought|N/A|13312188tomato477401|white|Ounce|Unknown|21|eingbarantiationought| +17509|AAAAAAAAFGEEAAAA|1997-10-27||Real, good ideas will not take very other, autonomous metres. Men tell social, inappropriate items. Political, solid subj|7.48|3.21|6016006|corpbrand #6|16|consignment|6|Jewelry|965|anticallyn st|N/A|41291874258olive2590|thistle|Pallet|Unknown|7|n stbarantiationought| +17510|AAAAAAAAGGEEAAAA|1997-10-27|2000-10-26|Future, central drinks can provide alternatives; men share surely back yards. Men ought to understand al|1.86|1.41|5003001|exportischolar #1|3|pop|5|Music|427|ationableese|N/A|053540698peach366381|tan|Pound|Unknown|36|baroughtantiationought| +17511|AAAAAAAAGGEEAAAA|2000-10-27||Future, central drinks can provide alternatives; men share surely back yards. Men ought to understand al|3.13|1.41|5003001|brandunivamalg #2|7|personal|10|Electronics|427|ationableese|N/A|0667042burnished7507|grey|Ounce|Unknown|50|oughtoughtantiationought| +17512|AAAAAAAAIGEEAAAA|1997-10-27|1999-10-27|Early long options allow much models. Boys might not cope according to a friends. American ports sa|4.02|3.57|4001001|amalgedu pack #1|1|womens|4|Shoes|273|priationable|extra large|tomato06053097596170|violet|Ounce|Unknown|56|ableoughtantiationought| +17513|AAAAAAAAIGEEAAAA|1999-10-28|2001-10-26|Also special leaves see thus swiss strategies. Tropical, simple courses will buy just again environment|8.66|3.57|4001001|amalgamalg #2|1|dresses|1|Women|273|priationable|medium|014325804seashell807|powder|Carton|Unknown|25|prioughtantiationought| +17514|AAAAAAAAIGEEAAAA|2001-10-27||Simply new activities may burn now about future questions. Circumstances buy by a words; roman papers see now more valid friends. Other areas i|7.30|3.57|4001001|edu packexporti #1|1|school-uniforms|3|Children|273|priationable|medium|962785773423lace2122|tomato|Ton|Unknown|73|eseoughtantiationought| +17515|AAAAAAAALGEEAAAA|1997-10-27||More bad titles get. Earlier economic minu|3.64|2.36|7004006|edu packbrand #6|4|curtains/drapes|7|Home|552|ableantianti|N/A|500red79807339476037|rose|Bunch|Unknown|26|antioughtantiationought| +17516|AAAAAAAAMGEEAAAA|1997-10-27|2000-10-26|Reservations would meet longer easy, daily lights. Exactly critical ref|9.27|2.87|9003009|exportimaxi #9|3|computers|9|Books|180|bareingought|N/A|6021coral09613614097|rose|Gross|Unknown|82|callyoughtantiationought| +17517|AAAAAAAAMGEEAAAA|2000-10-27||Daughters would possess other, experimental notes. Young, extended things make among a conferences. Full ac|2.27|2.87|9003009|brandmaxi #10|7|reference|9|Books|819|bareingought|N/A|6021coral09613614097|white|Case|Unknown|3|ationoughtantiationought| +17518|AAAAAAAAOGEEAAAA|1997-10-27|1999-10-27|Free processes can wake now still important institutions. Traditional, open plans serve better live years. Women should not pack by the experts. Competitors can miss hence op|7.63|2.44|9015011|scholarunivamalg #11|15|fiction|9|Books|693|prin stcally|N/A|8indian6583367891454|yellow|Gram|Unknown|22|eingoughtantiationought| +17519|AAAAAAAAOGEEAAAA|1999-10-28|2001-10-26|Free processes can wake now still important institutions. Traditional, open plans serve better live years. Women should not pack by the experts. Competitors can miss hence op|1.71|1.21|10008010|namelessunivamalg #10|8|scanners|10|Electronics|642|ableesecally|N/A|8indian6583367891454|powder|Pallet|Unknown|55|n stoughtantiationought| +17520|AAAAAAAAOGEEAAAA|2001-10-27||Free processes can wake now still important institutions. Traditional, open plans serve better live years. Women should not pack by the experts. Competitors can miss hence op|44.04|1.21|8005001|scholarnameless #1|5|fitness|8|Sports|642|ableesecally|N/A|8indian6583367891454|blanched|Box|Unknown|45|barableantiationought| +17521|AAAAAAAABHEEAAAA|1997-10-27||Subjects may remain officials. Forward, straight objects used to see wh|6.97|5.71|9003002|exportimaxi #2|3|computers|9|Books|130|barpriought|N/A|4681902wheat22752122|salmon|Lb|Unknown|13|oughtableantiationought| +17522|AAAAAAAACHEEAAAA|1997-10-27|2000-10-26|New, special songs accommodate black objects. Neighbours ought to see both video-taped, continuous issues. Golden, financial calls may not ask usefully dogs; |3.97|3.41|10002013|importounivamalg #13|2|camcorders|10|Electronics|167|ationcallyought|N/A|7510035698078yellow1|seashell|Carton|Unknown|22|ableableantiationought| +17523|AAAAAAAACHEEAAAA|2000-10-27||New, special songs accommodate black objects. Neighbours ought to see both video-taped, continuous issues. Golden, financial calls may not ask usefully dogs; |6.67|5.46|3002002|importoexporti #2|2|infants|3|Children|296|callyn stable|medium|851528steel918422454|wheat|Dozen|Unknown|25|priableantiationought| +17524|AAAAAAAAEHEEAAAA|1997-10-27|1999-10-27|Visitors cease here partners. Spectacular, royal techniques choose neighbours. Right sciences see so much as. More internal funds make however times. Monetary, recent things could obtain; |0.21|0.15|4001001|amalgedu pack #1|1|womens|4|Shoes|130|barpriought|extra large|102369020469rosy7824|lavender|Ton|Unknown|21|eseableantiationought| +17525|AAAAAAAAEHEEAAAA|1999-10-28|2001-10-26|Visitors cease here partners. Spectacular, royal techniques choose neighbours. Right sciences see so much as. More internal funds make however times. Monetary, recent things could obtain; |2.45|1.22|4001001|amalgbrand #6|1|bathroom|7|Home|130|barpriought|N/A|214purple18851319196|saddle|Gross|Unknown|35|antiableantiationought| +17526|AAAAAAAAEHEEAAAA|2001-10-27||Trends would remember as following abilities. Old leaders cannot achieve entirely special, ethical families. Different p|2.85|1.22|10003017|exportiunivamalg #17|3|dvd/vcr players|10|Electronics|271|barpriought|N/A|74915892322powder264|tomato|Case|Unknown|13|callyableantiationought| +17527|AAAAAAAAHHEEAAAA|1997-10-27||Dead years can join again isolated decisions; more good principles will not expect absolutely in a plans. Yet old feet cannot limit here |4.82|2.26|10014001|edu packamalgamalg #1|14|automotive|10|Electronics|51|oughtanti|N/A|857700285steel284288|slate|Bunch|Unknown|44|ationableantiationought| +17528|AAAAAAAAIHEEAAAA|1997-10-27|2000-10-26|Personal, liable years shall not start dramatic, dema|4.92|3.14|9003009|exportimaxi #9|3|computers|9|Books|260|barcallyable|N/A|7008028868misty59310|mint|Oz|Unknown|39|eingableantiationought| +17529|AAAAAAAAIHEEAAAA|2000-10-27||Mainly black events might warm similar friends. Important contents get assets. Local courts shall reach there often white volumes. Social roses seek much |1.18|1.05|9003009|exportiedu pack #2|3|kids|4|Shoes|260|barcallyable|large|19402027419moccasin8|grey|Carton|Unknown|38|n stableantiationought| +17530|AAAAAAAAKHEEAAAA|1997-10-27|1999-10-27|Active values may not capture. Casually political minutes would recognis|2.20|1.82|8014001|edu packmaxi #1|14|tennis|8|Sports|56|callyanti|N/A|43tan339071009006652|turquoise|Cup|Unknown|84|barpriantiationought| +17531|AAAAAAAAKHEEAAAA|1999-10-28|2001-10-26|Angry animals create essentially schemes. Much chief premises help changes. Prime, capitalist issues support financial, pleasant areas; broad situations can let aga|1.79|1.82|8014001|amalgimporto #2|1|accessories|2|Men|958|eingantin st|small|43tan339071009006652|lemon|Bundle|Unknown|22|oughtpriantiationought| +17532|AAAAAAAAKHEEAAAA|2001-10-27||Angry animals create essentially schemes. Much chief premises help changes. Prime, capitalist issues support financial, pleasant areas; broad situations can let aga|2.82|2.53|2003001|exportiimporto #1|3|pants|2|Men|6|eingantin st|medium|43tan339071009006652|smoke|Tsp|Unknown|5|ablepriantiationought| +17533|AAAAAAAANHEEAAAA|1997-10-27||Physical, tender troubles could stop righ|5.92|4.14|6005006|scholarcorp #6|5|earings|6|Jewelry|290|barn stable|N/A|papaya07247448851410|pale|Bundle|Unknown|9|pripriantiationought| +17534|AAAAAAAAOHEEAAAA|1997-10-27|2000-10-26|Personal, cold benefits disappear. Urgent actions |4.67|3.45|6015007|scholarbrand #7|15|custom|6|Jewelry|240|bareseable|N/A|4840thistle502823879|ghost|Tbl|Unknown|30|esepriantiationought| +17535|AAAAAAAAOHEEAAAA|2000-10-27||Wild limited eyes think never high rooms. Japanese, optimistic governments would raise above places. Late months w|9.15|3.45|7012006|importonameless #6|12|paint|7|Home|240|bareseable|N/A|4840thistle502823879|orchid|Each|Unknown|97|antipriantiationought| +17536|AAAAAAAAAIEEAAAA|1997-10-27|1999-10-27|New results used to lead soon african, true penalties. Popular trains follow environmentally classical gates. Final crews will indica|0.41|0.12|9006005|corpmaxi #5|6|parenting|9|Books|656|callyantically|N/A|515thistle4179462145|lawn|Pallet|Unknown|57|callypriantiationought| +17537|AAAAAAAAAIEEAAAA|1999-10-28|2001-10-26|Heavily ready computers write real rights. Secondary sites melt. British, european creditors mean most. British, new fingers go simply exciting polic|1.34|0.12|10004011|edu packunivamalg #11|6|audio|10|Electronics|406|callybarese|N/A|515thistle4179462145|sienna|Tbl|Unknown|13|ationpriantiationought| +17538|AAAAAAAAAIEEAAAA|2001-10-27||Heavily ready computers write real rights. Secondary sites melt. British, european creditors mean most. British, new fingers go simply exciting polic|2.25|0.74|10004011|importoexporti #1|2|infants|3|Children|292|callybarese|large|515thistle4179462145|dark|Unknown|Unknown|17|eingpriantiationought| +17539|AAAAAAAADIEEAAAA|1997-10-27||Previous, other details will talk ahead. Children hear here; true services require children; partly lucky members must make at first uncertain|1.85|1.40|9006008|corpmaxi #8|6|parenting|9|Books|156|callyantiought|N/A|0617829light86131716|rose|Lb|Unknown|74|n stpriantiationought| +17540|AAAAAAAAEIEEAAAA|1997-10-27|2000-10-26|Other, suitable instances will not shield also good, working territories. Small, difficult reforms may cut concessions. Cheap arms find before the institutions. Already little|7.45|6.33|7006001|corpbrand #1|6|rugs|7|Home|190|barn stought|N/A|327slate024817645530|violet|Bunch|Unknown|10|bareseantiationought| +17541|AAAAAAAAEIEEAAAA|2000-10-27||Other, suitable instances will not shield also good, working territories. Small, difficult reforms may cut concessions. Cheap arms find before the institutions. Already little|4.35|1.52|7006001|amalgedu pack #2|1|womens|4|Shoes|714|eseoughtation|medium|770059521moccasin555|peru|Tbl|Unknown|9|oughteseantiationought| +17542|AAAAAAAAGIEEAAAA|1997-10-27|1999-10-27|Years may not keep today as a winners. Left, islamic women move c|1.01|0.70|3003001|exportiexporti #1|3|toddlers|3|Children|694|esen stcally|economy|1402273582ghost96426|lawn|Bundle|Unknown|17|ableeseantiationought| +17543|AAAAAAAAGIEEAAAA|1999-10-28|2001-10-26|Streets ought to i|3.36|0.70|1002002|importoamalg #2|2|fragrances|1|Women|193|prin stought|extra large|1402273582ghost96426|red|Lb|Unknown|45|prieseantiationought| +17544|AAAAAAAAGIEEAAAA|2001-10-27||Overseas, stupid forces supply movements. Violent, european eggs know. Hot orders locate estimated profits. Only serious hours cannot leave often corporate, working-class legs. Fin|4.13|2.35|1002002|importoscholar #1|2|country|5|Music|193|prin stought|N/A|3786929994red2980612|snow|Dozen|Unknown|46|eseeseantiationought| +17545|AAAAAAAAJIEEAAAA|1997-10-27||Labour relationships revive camps. Numbers could see still within a policies; more inc bacte|9.94|6.36|5004002|edu packscholar #2|4|classical|5|Music|578|eingationanti|N/A|613711416chartreuse1|yellow|Pound|Unknown|48|antieseantiationought| +17546|AAAAAAAAKIEEAAAA|1997-10-27|2000-10-26|Essential interests can discover luckily from a activities. Righ|21.45|18.66|9007003|brandmaxi #3|7|reference|9|Books|73|priation|N/A|1395tan6086811447003|sky|Ounce|Unknown|17|callyeseantiationought| +17547|AAAAAAAAKIEEAAAA|2000-10-27||Essential interests can discover luckily from a activities. Righ|8.17|2.77|7008008|namelessbrand #8|7|lighting|7|Home|73|priation|N/A|6seashell24842443881|slate|Dram|Unknown|66|ationeseantiationought| +17548|AAAAAAAAMIEEAAAA|1997-10-27|1999-10-27|Real, local police make economic police; serious values shall convey always under a rights. Men will not stay firms. Thus mental cells cannot look little; primary, lucky unio|2.10|1.47|6004005|edu packcorp #5|4|bracelets|6|Jewelry|95|antin st|N/A|8408907022774tan7139|ivory|Lb|Unknown|12|eingeseantiationought| +17549|AAAAAAAAMIEEAAAA|1999-10-28|2001-10-26|Here stro|1.51|1.11|4003002|exportiedu pack #2|4|kids|4|Shoes|111|oughtoughtought|small|8408907022774tan7139|powder|Dozen|Unknown|27|n steseantiationought| +17550|AAAAAAAAMIEEAAAA|2001-10-27||Only adequate relationships kill at a days. Parties grow just. Houses must retire more royal proposals. Then northern plants may not educate most even basic attacks. Cases ought to |5.78|1.11|4003002|edu packamalg #1|4|swimwear|1|Women|111|oughtoughtought|large|2716736653381puff066|sienna|Gram|Unknown|54|barantiantiationought| +17551|AAAAAAAAPIEEAAAA|1997-10-27||Details must say yet in particular british theories. British trusts return sternly other, daily times. Meetings would account merely windows; formal consequences can determine international, n|0.27|0.15|6003008|exporticorp #8|3|gold|6|Jewelry|232|ablepriable|N/A|sky35685936798194253|turquoise|Gram|Unknown|60|oughtantiantiationought| +17552|AAAAAAAAAJEEAAAA|1997-10-27|2000-10-26|Problems protect at least parents. Therefore early friends used to become positively; consequences shall afford obviously daught|9.22|5.07|3004001|edu packexporti #1|4|school-uniforms|3|Children|888|eingeingeing|large|66202646firebrick435|powder|Ounce|Unknown|48|ableantiantiationought| +17553|AAAAAAAAAJEEAAAA|2000-10-27||Channels consider yesterday. European, unique numbers get to the leaders. Opportunities overtake.|4.55|3.23|4003002|exportiedu pack #2|4|kids|4|Shoes|232|ablepriable|small|66202646firebrick435|white|Gram|Unknown|13|priantiantiationought| +17554|AAAAAAAACJEEAAAA|1997-10-27|1999-10-27|Western schemes matter on a transactions. French experiences tell here for a affairs. Wide main assets penetrate always images. Ev|32.61|10.76|8002005|importonameless #5|2|baseball|8|Sports|549|n steseanti|N/A|652534salmon45109939|navy|Lb|Unknown|45|eseantiantiationought| +17555|AAAAAAAACJEEAAAA|1999-10-28|2001-10-26|Western schemes matter on a transactions. French experiences tell here for a affairs. Wide main assets penetrate always images. Ev|5.98|5.38|7016006|corpnameless #6|16|furniture|7|Home|668|eingcallycally|N/A|1957436pale358873435|midnight|N/A|Unknown|28|antiantiantiationought| +17556|AAAAAAAACJEEAAAA|2001-10-27||Western schemes matter on a transactions. French experiences tell here for a affairs. Wide main assets penetrate always images. Ev|9.48|8.15|5001001|amalgscholar #1|1|rock|5|Music|668|eingcallycally|N/A|1957436pale358873435|royal|Cup|Unknown|10|callyantiantiationought| +17557|AAAAAAAAFJEEAAAA|1997-10-27||Appropriate, prime hours tell. Terms could take. Much new workers settle important, british players. Comprehensive tonnes will eat nearby. Due dec|2.04|0.79|7012008|importonameless #8|12|paint|7|Home|201|oughtbarable|N/A|055724861769light751|steel|Ounce|Unknown|2|ationantiantiationought| +17558|AAAAAAAAGJEEAAAA|1997-10-27|2000-10-26|Models let. Important, great sports eat over; national pp. cannot appeal elsewhere related years; eggs could help then brilliant areas; fine, formal animals warm all;|1.77|0.81|10014015|edu packamalgamalg #15|14|automotive|10|Electronics|520|barableanti|N/A|3181492pale339860308|medium|Pound|Unknown|2|eingantiantiationought| +17559|AAAAAAAAGJEEAAAA|2000-10-27||Models let. Important, great sports eat over; national pp. cannot appeal elsewhere related years; eggs could help then brilliant areas; fine, formal animals warm all;|1.34|0.41|4004002|edu packedu pack #2|4|athletic|4|Shoes|60|barcally|large|6286red5076146039230|tomato|Box|Unknown|27|n stantiantiationought| +17560|AAAAAAAAIJEEAAAA|1997-10-27|1999-10-27|Other things cannot frame always permanent places; just economic services drive as well other, main participants. New women trace in a indians. Social, sufficient powers|4.90|3.87|10003017|exportiunivamalg #17|3|dvd/vcr players|10|Electronics|285|antieingable|N/A|0921039161wheat83779|smoke|Bundle|Unknown|13|barcallyantiationought| +17561|AAAAAAAAIJEEAAAA|1999-10-28|2001-10-26|In|83.99|31.91|10003017|univnameless #2|3|flatware|7|Home|285|antieingable|N/A|71319sienna146931339|white|Each|Unknown|67|oughtcallyantiationought| +17562|AAAAAAAAIJEEAAAA|2001-10-27||Able, ready cars disarm never nuclear nur|6.60|5.67|10003017|edu packimporto #1|4|sports-apparel|2|Men|258|antieingable|large|71319sienna146931339|violet|Ounce|Unknown|82|ablecallyantiationought| +17563|AAAAAAAALJEEAAAA|1997-10-27||Annually cheerful interest|1.43|0.47|4003002|exportiedu pack #2|3|kids|4|Shoes|313|prioughtpri|economy|5014lavender76548113|ivory|Tbl|Unknown|3|pricallyantiationought| +17564|AAAAAAAAMJEEAAAA|1997-10-27|2000-10-26|Often in|8.79|6.15|5003001|exportischolar #1|3|pop|5|Music|940|baresen st|N/A|02112dark30717924547|pink|Gram|Unknown|45|esecallyantiationought| +17565|AAAAAAAAMJEEAAAA|2000-10-27||Often in|5.93|2.66|3004002|edu packexporti #2|3|school-uniforms|3|Children|940|baresen st|small|60turquoise894853732|smoke|Box|Unknown|41|anticallyantiationought| +17566|AAAAAAAAOJEEAAAA|1997-10-27|1999-10-27|Passionately important values should test even with a functions. M|4.01|1.36|6002007|importocorp #7|2|diamonds|6|Jewelry|17|ationought|N/A|81056889495635mint00|rosy|Cup|Unknown|9|callycallyantiationought| +17567|AAAAAAAAOJEEAAAA|1999-10-28|2001-10-26|Passionately important values should test even with a functions. M|3.19|2.16|7016008|corpnameless #8|2|furniture|7|Home|27|ationought|N/A|6104522spring0903095|seashell|Gram|Unknown|45|ationcallyantiationought| +17568|AAAAAAAAOJEEAAAA|2001-10-27||Passionately important values should test even with a functions. M|5.71|3.08|7016008|scholarnameless #9|15|tables|7|Home|567|ationcallyanti|N/A|6104522spring0903095|tomato|Case|Unknown|65|eingcallyantiationought| +17569|AAAAAAAABKEEAAAA|1997-10-27||Churches might put most human, other nations. Different, current employees could damage. Approaches make. Young, essential bedrooms could involve never labour, fixe|0.58|0.19|3004002|edu packexporti #2|4|school-uniforms|3|Children|469|n stcallyese|petite|78793steel3119595423|lemon|N/A|Unknown|19|n stcallyantiationought| +17570|AAAAAAAACKEEAAAA|1997-10-27|2000-10-26|Popular, false eyes would not indulge in particular. Links compare con|8.29|7.21|2002001|importoimporto #1|2|shirts|2|Men|257|ationantiable|petite|5pink587679316666805|pale|Oz|Unknown|24|barationantiationought| +17571|AAAAAAAACKEEAAAA|2000-10-27||Popular, false eyes would not indulge in particular. Links compare con|63.95|7.21|3001002|amalgexporti #2|2|newborn|3|Children|185|ationantiable|medium|7395594aquamarine662|beige|Dram|Unknown|12|oughtationantiationought| +17572|AAAAAAAAEKEEAAAA|1997-10-27|1999-10-27|Long, hard ways use then only, sure times; multiple women appoint poor, wide needs. Industrial departments face cells. So j|0.86|0.28|5001001|amalgscholar #1|1|rock|5|Music|11|oughtought|N/A|08sandy8920281588363|white|Tsp|Unknown|48|ableationantiationought| +17573|AAAAAAAAEKEEAAAA|1999-10-28|2001-10-26|Young holes must not hear afterwards british policies. New, soviet lengths kill just outer, powerful ch|1.75|0.28|5001001|scholarmaxi #4|15|fishing|8|Sports|11|oughtought|N/A|rose2567616094483887|lime|Ounce|Unknown|19|priationantiationought| +17574|AAAAAAAAEKEEAAAA|2001-10-27||Light governors look more usually bad assets. For instance related services |4.01|0.28|8006005|corpnameless #5|6|football|8|Sports|11|oughtought|N/A|rose2567616094483887|peach|Unknown|Unknown|70|eseationantiationought| +17575|AAAAAAAAHKEEAAAA|1997-10-27||Particular writers might not get partly in a creditors. Pains might not manage often now full patients. Strong, important societies get|3.12|1.09|8013006|exportimaxi #6|13|sailing|8|Sports|407|ationbarese|N/A|88996531185lace56946|royal|Ton|Unknown|59|antiationantiationought| +17576|AAAAAAAAIKEEAAAA|1997-10-27|2000-10-26|For example available women enter greatly mental principles. In general crucial hospitals s|0.52|0.38|7004007|edu packbrand #7|4|curtains/drapes|7|Home|229|n stableable|N/A|3401830810667peach90|sandy|Carton|Unknown|49|callyationantiationought| +17577|AAAAAAAAIKEEAAAA|2000-10-27||Afraid messages would record therefore years. Raw children deal worldwide. Old fee|3.27|0.38|7004007|corpcorp #2|6|rings|6|Jewelry|229|n stableable|N/A|317228742754wheat441|royal|Pound|Unknown|52|ationationantiationought| +17578|AAAAAAAAKKEEAAAA|1997-10-27|1999-10-27|Expensive rat|2.48|1.41|5003001|exportischolar #1|3|pop|5|Music|193|prin stought|N/A|22maroon098061915456|medium|Gram|Unknown|73|eingationantiationought| +17579|AAAAAAAAKKEEAAAA|1999-10-28|2001-10-26|Public names shall not use high. Willing families come now ashamed measures. Cigarettes let unquestionably. Else good others make often new thousands. Different boards cannot wish follow|4.42|1.41|4002002|importoedu pack #2|3|mens|4|Shoes|6|cally|economy|22maroon098061915456|sky|Each|Unknown|15|n stationantiationought| +17580|AAAAAAAAKKEEAAAA|2001-10-27||Demonstrations used to travel nevertheless games. Italian, possible hotels should override still fine, di|9.97|1.41|4002002|exportinameless #1|3|basketball|8|Sports|382|ableeingpri|N/A|7838868ivory47655563|royal|Case|Unknown|62|bareingantiationought| +17581|AAAAAAAANKEEAAAA|1997-10-27||Usually new horses present totally by a witnesses. Common, satisfactory tourists start facil|74.78|36.64|5004002|edu packscholar #2|4|classical|5|Music|638|eingprically|N/A|93104sandy5150588749|steel|Carton|Unknown|96|oughteingantiationought| +17582|AAAAAAAAOKEEAAAA|1997-10-27|2000-10-26|Late clothes must receive frequently then large things. Volunteers make parents. Available forces shall tell with a hospitals; however prime days could take long local sweet years. Normal, hard depar|2.08|1.06|4002001|importoedu pack #1|2|mens|4|Shoes|703|pribaration|medium|168208papaya32280350|forest|Carton|Unknown|43|ableeingantiationought| +17583|AAAAAAAAOKEEAAAA|2000-10-27||Suddenly autonomous powers may not get very to the women. Nevertheless solid associations play neither. Communications understand already tools. Patien|5.45|1.06|1004002|edu packamalg #2|2|swimwear|1|Women|295|antin stable|small|76886beige4782947856|rose|Bunch|Unknown|63|prieingantiationought| +17584|AAAAAAAAALEEAAAA|1997-10-27|1999-10-27|Free figures would come momentarily main eyes. Issues shall not face big little plates. Shops follow|4.26|3.32|3001001|amalgexporti #1|1|newborn|3|Children|668|eingcallycally|medium|77694pale67882703991|pale|Ton|Unknown|2|eseeingantiationought| +17585|AAAAAAAAALEEAAAA|1999-10-28|2001-10-26|Free figures would come momentarily main eyes. Issues shall not face big little plates. Shops follow|4.23|3.32|3001001|amalgamalg #2|1|dresses|1|Women|668|eingcallycally|large|77694pale67882703991|powder|Case|Unknown|28|antieingantiationought| +17586|AAAAAAAAALEEAAAA|2001-10-27||Arms would resolve. Invariably married gardens should not benefit correctly to a goals. Even wrong disciplines may produce now concerns; christians move. Here eastern times|1.60|3.32|7007007|brandbrand #7|1|decor|7|Home|668|eingcallycally|N/A|70187307997699smoke5|yellow|Cup|Unknown|57|callyeingantiationought| +17587|AAAAAAAADLEEAAAA|1997-10-27||Professional, delicate settings must raise partially generally common heads. Either|1.78|0.78|9012008|importounivamalg #8|12|home repair|9|Books|789|n steingation|N/A|tan41063086742148121|sky|Pound|Unknown|35|ationeingantiationought| +17588|AAAAAAAAELEEAAAA|1997-10-27|2000-10-26|Big states could not discipli|8.10|3.88|3002001|importoexporti #1|2|infants|3|Children|961|oughtcallyn st|small|286666437puff1733113|midnight|Lb|Unknown|68|eingeingantiationought| +17589|AAAAAAAAELEEAAAA|2000-10-27||However thin applications miss on a clothes. Top, primary conservatives shall not get times. Inv|9.00|6.84|1001002|amalgamalg #2|2|dresses|1|Women|161|oughtcallyought|extra large|389074honeydew269319|seashell|Carton|Unknown|35|n steingantiationought| +17590|AAAAAAAAGLEEAAAA|1997-10-27|1999-10-27|Aware categories relish at all severe, large services. Relations would hate away yesterday previous pounds. Dark, able makers m|0.44|0.15|3001001|amalgexporti #1|1|newborn|3|Children|452|ableantiese|medium|92867387013white1905|pale|Lb|Unknown|35|barn stantiationought| +17591|AAAAAAAAGLEEAAAA|1999-10-28|2001-10-26|Back, scientific years stay at a bodies; jewish inches might worry slowly towards a cuts. Products would not inherit just yesterday large jobs. There full m|3.27|1.56|3001001|importonameless #2|1|baseball|8|Sports|452|ableantiese|N/A|92867387013white1905|olive|Bundle|Unknown|5|oughtn stantiationought| +17592|AAAAAAAAGLEEAAAA|2001-10-27||Pale friends may suffer also home compl|8.07|1.56|9011001|amalgunivamalg #1|11|cooking|9|Books|452|ableantiese|N/A|92867387013white1905|slate|N/A|Unknown|14|ablen stantiationought| +17593|AAAAAAAAJLEEAAAA|1997-10-27||Wrong, high terms make relatively holidays. Major, relevant theories consider difficult, new markets. Sure, real subjec|3.29|1.51|5001002|amalgscholar #2|1|rock|5|Music|401|oughtbarese|N/A|0slate95814557453916|tomato|Each|Unknown|63|prin stantiationought| +17594|AAAAAAAAKLEEAAAA|1997-10-27|2000-10-26|Guilty, mathematical contents used to join as. Ashamed, traditional months go as within a principles. Forward free cases could seek very colleagu|9.61|3.36|9004003|edu packmaxi #3|4|entertainments|9|Books|191|oughtn stought|N/A|4047377077white70926|khaki|Each|Unknown|8|esen stantiationought| +17595|AAAAAAAAKLEEAAAA|2000-10-27||Private, existing applications may not remind most from the agreements. Clear, possible weeks shall see enough through a resources. Hours show in a |8.48|3.36|9004003|edu packnameless #6|4|camping|8|Sports|191|oughtn stought|N/A|4047377077white70926|peru|Each|Unknown|13|antin stantiationought| +17596|AAAAAAAAMLEEAAAA|1997-10-27|1999-10-27|Now fine words give soft samples. Gold, new co|7.17|3.01|7012007|importonameless #7|12|paint|7|Home|209|n stbarable|N/A|4thistle738226734089|peru|Each|Unknown|56|callyn stantiationought| +17597|AAAAAAAAMLEEAAAA|1999-10-28|2001-10-26|Agencies used to strike attacks. Internal individuals can think outer, sexual seats. There basic terms can understand at the grants. Sensibly only designs used |5.14|4.36|10015003|scholaramalgamalg #3|12|portable|10|Electronics|209|n stbarable|N/A|6683495013purple4999|midnight|Tsp|Unknown|71|ationn stantiationought| +17598|AAAAAAAAMLEEAAAA|2001-10-27||Distant, single reports would start heavy, white ideas. Long amounts love knowingly usefully main pictures. Demanding |4.88|4.36|5002001|importoscholar #1|2|country|5|Music|345|n stbarable|N/A|54moccasin8955842041|mint|Case|Unknown|55|eingn stantiationought| +17599|AAAAAAAAPLEEAAAA|1997-10-27||References carry enough; little duties will not restore full, new boards. Advanced manufacturers remain in a wo|2.00|1.16|7010010|univnameless #10|10|flatware|7|Home|327|ationablepri|N/A|370788snow2742927414|plum|Dram|Unknown|48|n stn stantiationought| +17600|AAAAAAAAAMEEAAAA|1997-10-27|2000-10-26|Local, english problems should hide sides. Eyes will tell yesterday. Questions take regularly circumstances. Goods play personally by a|2.84|1.56|1003001|exportiamalg #1|3|maternity|1|Women|377|ationationpri|medium|75953455321rosy43944|purple|N/A|Unknown|8|barbarcallyationought| +17601|AAAAAAAAAMEEAAAA|2000-10-27||Local, english problems should hide sides. Eyes will tell yesterday. Questions take regularly circumstances. Goods play personally by a|3.77|1.56|5001002|amalgscholar #2|3|rock|5|Music|377|ationationpri|N/A|75953455321rosy43944|saddle|Unknown|Unknown|56|oughtbarcallyationought| +17602|AAAAAAAACMEEAAAA|1997-10-27|1999-10-27|Full, entire lines might v|8.85|4.77|2003001|exportiimporto #1|3|pants|2|Men|231|oughtpriable|small|8tomato4217790274462|powder|Carton|Unknown|25|ablebarcallyationought| +17603|AAAAAAAACMEEAAAA|1999-10-28|2001-10-26|High fee|6.41|4.77|3003002|exportiexporti #2|3|toddlers|3|Children|536|callyprianti|N/A|8tomato4217790274462|powder|Ounce|Unknown|34|pribarcallyationought| +17604|AAAAAAAACMEEAAAA|2001-10-27||Dead days must not help. Deliberately legislative stations provide as ranks. Rather married teams ensure pretty small, |2.56|1.30|5001001|amalgscholar #1|1|rock|5|Music|845|antieseeing|N/A|408637thistle4913170|peach|Dozen|Unknown|60|esebarcallyationought| +17605|AAAAAAAAFMEEAAAA|1997-10-27||Creatures must not reinforce otherwise windows. Factors would not dine high teachers. Important|9.49|3.32|3004002|edu packexporti #2|4|school-uniforms|3|Children|88|eingeing|large|524602seashell618584|spring|Gross|Unknown|20|antibarcallyationought| +17606|AAAAAAAAGMEEAAAA|1997-10-27|2000-10-26|Australian, open things shall not cover often small rates. Goals provide ever however grim records. Enough ready objec|4.03|1.77|4002001|importoedu pack #1|2|mens|4|Shoes|661|oughtcallycally|large|231960526tomato16571|puff|Bundle|Unknown|21|callybarcallyationought| +17607|AAAAAAAAGMEEAAAA|2000-10-27||Australian, open things shall not cover often small rates. Goals provide ever however grim records. Enough ready objec|8.25|3.79|4002001|importoscholar #2|2|country|5|Music|661|oughtcallycally|N/A|517268194443tomato55|cream|Ounce|Unknown|25|ationbarcallyationought| +17608|AAAAAAAAIMEEAAAA|1997-10-27|1999-10-27|Hence acute ages react rapidly initial, fixed companies. Northern, true faces may jeopardise quite small hills. Heroes might stick yet certainly appropriate buildings. Public, natural privileges wou|2.14|1.11|3002001|importoexporti #1|2|infants|3|Children|375|antiationpri|large|2318116494462lime895|powder|Dozen|Unknown|22|eingbarcallyationought| +17609|AAAAAAAAIMEEAAAA|1999-10-28|2001-10-26|Indian times must pretend. Surprising, electrical attempts |5.52|2.81|5001002|amalgscholar #2|2|rock|5|Music|375|antiationpri|N/A|2318116494462lime895|snow|Oz|Unknown|17|n stbarcallyationought| +17610|AAAAAAAAIMEEAAAA|2001-10-27||Indian times must pretend. Surprising, electrical attempts |3.98|1.55|10013016|exportiamalgamalg #16|2|stereo|10|Electronics|375|antiationpri|N/A|2318116494462lime895|medium|Dozen|Unknown|15|baroughtcallyationought| +17611|AAAAAAAALMEEAAAA|1997-10-27||Other, new reasons should not sing without the costs. Too clear houses shall reflect all pensions. Parents loose natural cases. Good things shall not suggest aside in a cuts. Speci|6.06|2.84|6006004|corpcorp #4|6|rings|6|Jewelry|143|prieseought|N/A|887021951061violet36|puff|Dram|Unknown|62|oughtoughtcallyationought| +17612|AAAAAAAAMMEEAAAA|1997-10-27|2000-10-26|Stubbornly irrelevant letters implement never locations. Adults shall turn sure. Final writers can get parliamentary patients. Tactics drive swiftly. More certain movies could meet |16.48|10.05|6009005|maxicorp #5|9|womens watch|6|Jewelry|146|callyeseought|N/A|510158282metallic502|thistle|Box|Unknown|46|ableoughtcallyationought| +17613|AAAAAAAAMMEEAAAA|2000-10-27||Stubbornly irrelevant letters implement never locations. Adults shall turn sure. Final writers can get parliamentary patients. Tactics drive swiftly. More certain movies could meet |6.02|2.94|10009002|maxiunivamalg #2|9|televisions|10|Electronics|146|callyeseought|N/A|510158282metallic502|almond|Tsp|Unknown|8|prioughtcallyationought| +17614|AAAAAAAAOMEEAAAA|1997-10-27|1999-10-27|Physical problems could know in a courts. Too major houses sit ultimately often vast reserves. Shareholders force circumstances. Likely notions require with the years; excellent, e|50.76|42.13|1002001|importoamalg #1|2|fragrances|1|Women|333|pripripri|petite|728medium58925818013|navy|Pallet|Unknown|55|eseoughtcallyationought| +17615|AAAAAAAAOMEEAAAA|1999-10-28|2001-10-26|Physical problems could know in a courts. Too major houses sit ultimately often vast reserves. Shareholders force circumstances. Likely notions require with the years; excellent, e|4.05|3.40|1002001|corpbrand #8|16|consignment|6|Jewelry|333|pripripri|N/A|4wheat48556780220792|violet|Dram|Unknown|66|antioughtcallyationought| +17616|AAAAAAAAOMEEAAAA|2001-10-27||Alive, old evenings would not undertake even to a plans. Electoral, middle charts explore citizens; left properties allow by a achievements; different, central nurses can include au|8.08|6.46|4001001|amalgedu pack #1|1|womens|4|Shoes|228|eingableable|petite|145612465rose9564066|maroon|Pound|Unknown|11|callyoughtcallyationought| +17617|AAAAAAAABNEEAAAA|1997-10-27||Pensioners ask economic, beautiful phrases. Special, so-called pieces support half great police. O|3.29|2.82|10002006|importounivamalg #6|2|camcorders|10|Electronics|626|callyablecally|N/A|774173902sky50513457|sandy|Ton|Unknown|31|ationoughtcallyationought| +17618|AAAAAAAACNEEAAAA|1997-10-27|2000-10-26|Important islands could not say appointed wings. Little other emotions would not advise inside eligible times. Vast, basic stairs try here other, colourful ev|7.64|2.59|1001001|amalgamalg #1|1|dresses|1|Women|514|eseoughtanti|small|9lime761915188609799|seashell|Bunch|Unknown|23|eingoughtcallyationought| +17619|AAAAAAAACNEEAAAA|2000-10-27||Important islands could not say appointed wings. Little other emotions would not advise inside eligible times. Vast, basic stairs try here other, colourful ev|0.48|0.18|1001001|edu packexporti #2|1|school-uniforms|3|Children|514|eseoughtanti|small|018136steel299329260|rosy|Cup|Unknown|19|n stoughtcallyationought| +17620|AAAAAAAAENEEAAAA|1997-10-27|1999-10-27|Consistently royal characteristics make both perhaps good songs. Still polish eyes apply in a obligations. Instantly environmental argum|8.93|3.66|4004001|edu packedu pack #1|4|athletic|4|Shoes|58|einganti|extra large|7tan3670555307439255|spring|Carton|Unknown|23|barablecallyationought| +17621|AAAAAAAAENEEAAAA|1999-10-28|2001-10-26|Heavy, exceptional animals notice. Revolutionary un|2.89|3.66|9014006|edu packunivamalg #6|4|sports|9|Books|58|einganti|N/A|7tan3670555307439255|blanched|Unknown|Unknown|1|oughtablecallyationought| +17622|AAAAAAAAENEEAAAA|2001-10-27||Really growing causes shall not get now legal ways. Top, new officials should set for the films. Dark policies used to develop recently with a characters; political, corresponding circumstances ought |0.46|0.28|9014006|amalgexporti #1|1|newborn|3|Children|58|einganti|medium|1783powder8114831460|powder|Case|Unknown|11|ableablecallyationought| +17623|AAAAAAAAHNEEAAAA|1997-10-27||Smooth forms can tell in public. Acts imply both very ready winners; other attitudes must not capture never too prime methods. Short, high candidates die most possible lads.|89.50|58.17|1004002|edu packamalg #2|4|swimwear|1|Women|561|oughtcallyanti|medium|683white613594707893|saddle|Unknown|Unknown|74|priablecallyationought| +17624|AAAAAAAAINEEAAAA|1997-10-27|2000-10-26|Likely, critical keys become similar, nervous experts. Now chief months may wish also deep, true calls. Further alone concerns keep prices; households lie findings. Foreign words try. Once|7.99|2.95|1002001|importoamalg #1|2|fragrances|1|Women|310|baroughtpri|medium|2khaki71348253787692|sienna|Dozen|Unknown|31|eseablecallyationought| +17625|AAAAAAAAINEEAAAA|2000-10-27||Likely, critical keys become similar, nervous experts. Now chief months may wish also deep, true calls. Further alone concerns keep prices; households lie findings. Foreign words try. Once|0.59|2.95|6015002|scholarbrand #2|2|custom|6|Jewelry|310|baroughtpri|N/A|771983maroon15561478|peach|Tbl|Unknown|19|antiablecallyationought| +17626|AAAAAAAAKNEEAAAA|1997-10-27|1999-10-27|Hundreds know |1.70|1.08|10011015|amalgamalgamalg #15|11|disk drives|10|Electronics|269|n stcallyable|N/A|2slate51170827555841|light|Bundle|Unknown|27|callyablecallyationought| +17627|AAAAAAAAKNEEAAAA|1999-10-28|2001-10-26|Labour, new markets would not hang long systems. Good p|9.97|7.97|10011015|brandbrand #8|7|decor|7|Home|269|n stcallyable|N/A|2slate51170827555841|tan|Gram|Unknown|34|ationablecallyationought| +17628|AAAAAAAAKNEEAAAA|2001-10-27||Labour, new markets would not hang long systems. Good p|7.17|3.44|5001001|amalgscholar #1|7|rock|5|Music|269|n stcallyable|N/A|2slate51170827555841|goldenrod|Tsp|Unknown|20|eingablecallyationought| +17629|AAAAAAAANNEEAAAA|1997-10-27||Quite linguistic cells ask already permanent, valuable players. Colours place hastily happy, short bacteria; int|1.59|1.39|9015002|scholarunivamalg #2|15|fiction|9|Books|662|ablecallycally|N/A|824539626983cream352|puff|Tbl|Unknown|79|n stablecallyationought| +17630|AAAAAAAAONEEAAAA|1997-10-27|2000-10-26|Cases try as. Holidays keep just early budgets. Precisely good eyes would come users. Hot units must reduce okay around a limitations. Minutes wish surprising agreements. Terms will lear|3.74|1.30|3003001|exportiexporti #1|3|toddlers|3|Children|283|prieingable|small|93754968868150royal5|thistle|Unknown|Unknown|31|barpricallyationought| +17631|AAAAAAAAONEEAAAA|2000-10-27||Good words go in a variations. Light, equal women prove only thanks. Rapid elements could introduce metropolitan courts. Open, practical fortunes tolerate directly others. High, medical proposals sh|1.34|1.30|1002002|importoamalg #2|3|fragrances|1|Women|211|oughtoughtable|petite|419350326661plum4671|seashell|Unknown|Unknown|1|oughtpricallyationought| +17632|AAAAAAAAAOEEAAAA|1997-10-27|1999-10-27|Advantag|9.64|3.08|4002001|importoedu pack #1|2|mens|4|Shoes|171|oughtationought|extra large|7568289navy282295862|burnished|Carton|Unknown|22|ablepricallyationought| +17633|AAAAAAAAAOEEAAAA|1999-10-28|2001-10-26|Advantag|1.59|3.08|4002001|amalgscholar #2|2|rock|5|Music|226|callyableable|N/A|7568289navy282295862|dodger|Pallet|Unknown|60|pripricallyationought| +17634|AAAAAAAAAOEEAAAA|2001-10-27||Different heads must give only both labour buildings. Honest, detailed heads can derive major, mechanical dreams|93.16|29.81|5001001|amalgscholar #1|1|rock|5|Music|216|callyableable|N/A|36572915sky880456092|indian|Each|Unknown|28|esepricallyationought| +17635|AAAAAAAADOEEAAAA|1997-10-27||Be|8.38|6.11|5002002|importoscholar #2|2|country|5|Music|88|eingeing|N/A|6powder3424155166481|azure|Each|Unknown|13|antipricallyationought| +17636|AAAAAAAAEOEEAAAA|1997-10-27|2000-10-26|Now political pages will refer active frie|7.81|4.92|7016007|corpnameless #7|16|furniture|7|Home|540|bareseanti|N/A|0977656591406papaya4|saddle|Oz|Unknown|1|callypricallyationought| +17637|AAAAAAAAEOEEAAAA|2000-10-27||Clearly s|5.11|3.42|2001002|amalgimporto #2|16|accessories|2|Men|540|bareseanti|large|0977656591406papaya4|medium|Tsp|Unknown|46|ationpricallyationought| +17638|AAAAAAAAGOEEAAAA|1997-10-27|1999-10-27|Literally dark visitors take private, adequate women; lips make always around new elements. However left nurs|1.76|1.32|5002001|importoscholar #1|2|country|5|Music|75|antiation|N/A|16843622349steel5015|yellow|Gram|Unknown|65|eingpricallyationought| +17639|AAAAAAAAGOEEAAAA|1999-10-28|2001-10-26|Literally dark visitors take private, adequate women; lips make always around new elements. However left nurs|4.47|3.93|5002001|corpmaxi #10|16|golf|8|Sports|322|antiation|N/A|896730531seashell791|papaya|Gram|Unknown|11|n stpricallyationought| +17640|AAAAAAAAGOEEAAAA|2001-10-27||Different collections could not telephone old benefits. Only, high spe|3.30|3.93|5002001|exportiexporti #1|3|toddlers|3|Children|322|antiation|medium|212113870117violet28|turquoise|Pallet|Unknown|15|baresecallyationought| +17641|AAAAAAAAJOEEAAAA|1997-10-27||Acute seasons thank alternative, early pages. Full variations can enter problems. Central stories shall give complete servants. Common ston|7.38|5.97|7002002|importobrand #2|2|bedding|7|Home|224|eseableable|N/A|5178orange5186001431|indian|N/A|Unknown|75|oughtesecallyationought| +17642|AAAAAAAAKOEEAAAA|1997-10-27|2000-10-26|Common years like straight. Sides shall make for instance very fine events. White, full incentives head ini|2.86|1.20|3004001|edu packexporti #1|4|school-uniforms|3|Children|117|ationoughtought|petite|134657sienna76899799|royal|Ounce|Unknown|65|ableesecallyationought| +17643|AAAAAAAAKOEEAAAA|2000-10-27||Common years like straight. Sides shall make for instance very fine events. White, full incentives head ini|0.37|0.32|1001002|amalgamalg #2|4|dresses|1|Women|117|ationoughtought|petite|134657sienna76899799|royal|Gram|Unknown|48|priesecallyationought| +17644|AAAAAAAAMOEEAAAA|1997-10-27|1999-10-27|Great details cannot promote forthcoming, elderly restrictions. Lengths start ready, tall games; gentle, able injuries collapse relativel|2.04|0.71|3002001|importoexporti #1|2|infants|3|Children|601|oughtbarcally|large|917478107803367plum1|rose|Unknown|Unknown|30|eseesecallyationought| +17645|AAAAAAAAMOEEAAAA|1999-10-28|2001-10-26|Other, temporary suggestions leave however years. Teachers could not pass anyway short parties. Personal, physical cars phone too dirty children; more young strategies |2.28|0.71|4002002|importoedu pack #2|2|mens|4|Shoes|601|oughtbarcally|small|5lime947048770858465|white|Carton|Unknown|50|antiesecallyationought| +17646|AAAAAAAAMOEEAAAA|2001-10-27||Other, temporary suggestions leave however years. Teachers could not pass anyway short parties. Personal, physical cars phone too dirty children; more young strategies |7.16|0.71|4002002|scholarbrand #7|15|custom|6|Jewelry|601|oughtbarcally|N/A|66moccasin5041279722|metallic|Bundle|Unknown|23|callyesecallyationought| +17647|AAAAAAAAPOEEAAAA|1997-10-27||Terrible, able roads would not replace tracks; other goods shall set under psychological, sole others; small, regional countries should not come employers. Originall|2.27|1.74|10004002|edu packunivamalg #2|4|audio|10|Electronics|330|barpripri|N/A|12574074491111sandy4|lemon|Tbl|Unknown|25|ationesecallyationought| +17648|AAAAAAAAAPEEAAAA|1997-10-27|2000-10-26|Financial bodies shall shape wholly only additional organisations. Perhaps slight relations|9.49|6.07|5001001|amalgscholar #1|1|rock|5|Music|889|n steingeing|N/A|483pink6816464685643|red|N/A|Unknown|76|eingesecallyationought| +17649|AAAAAAAAAPEEAAAA|2000-10-27||Essentially available tears would phone already human, sound presents. Also indus|4.25|1.53|10013004|exportiamalgamalg #4|13|stereo|10|Electronics|329|n steingeing|N/A|23917632144rosy63999|rose|Pound|Unknown|43|n stesecallyationought| +17650|AAAAAAAACPEEAAAA|1997-10-27|1999-10-27|Aware, labour parts remain disputes. Possible, abl|88.26|79.43|2003001|exportiimporto #1|3|pants|2|Men|596|callyn stanti|medium|689plum7326792270450|indian|Ton|Unknown|55|baranticallyationought| +17651|AAAAAAAACPEEAAAA|1999-10-28|2001-10-26|Aware, labour parts remain disputes. Possible, abl|2.95|79.43|9004012|edu packmaxi #12|3|entertainments|9|Books|596|callyn stanti|N/A|181278595501dodger03|plum|Carton|Unknown|60|oughtanticallyationought| +17652|AAAAAAAACPEEAAAA|2001-10-27||Aware, labour parts remain disputes. Possible, abl|1.00|0.69|9004012|exportischolar #1|3|pop|5|Music|153|callyn stanti|N/A|181278595501dodger03|deep|Gram|Unknown|41|ableanticallyationought| +17653|AAAAAAAAFPEEAAAA|1997-10-27||New, enthusiastic shares embrace. Averag|2.00|1.10|9009002|maximaxi #2|9|science|9|Books|633|priprically|N/A|midnight271610298197|smoke|Box|Unknown|2|prianticallyationought| +17654|AAAAAAAAGPEEAAAA|1997-10-27|2000-10-26|Full, narrow outcomes shall not seem so perfect, |74.21|25.97|10014009|edu packamalgamalg #9|14|automotive|10|Electronics|67|ationcally|N/A|711152906yellow67629|purple|Carton|Unknown|11|eseanticallyationought| +17655|AAAAAAAAGPEEAAAA|2000-10-27||Full, narrow outcomes shall not seem so perfect, |17.11|9.41|7010006|univnameless #6|10|flatware|7|Home|67|ationcally|N/A|711152906yellow67629|papaya|Cup|Unknown|9|antianticallyationought| +17656|AAAAAAAAIPEEAAAA|1997-10-27|1999-10-27|Councils know in a goods. Eyes|4.38|3.02|3003001|exportiexporti #1|3|toddlers|3|Children|210|baroughtable|petite|935308132sandy932871|thistle|N/A|Unknown|11|callyanticallyationought| +17657|AAAAAAAAIPEEAAAA|1999-10-28|2001-10-26|Still free arrangements back usually so good museums. Currently social stations could not identify. Industrial, british women shall lose instead;|3.84|3.02|3003001|exportiexporti #2|3|toddlers|3|Children|377|ationationpri|medium|935308132sandy932871|maroon|Ton|Unknown|3|ationanticallyationought| +17658|AAAAAAAAIPEEAAAA|2001-10-27||Still free arrangements back usually so good museums. Currently social stations could not identify. Industrial, british women shall lose instead;|4.98|2.04|3003001|exportischolar #1|3|pop|5|Music|216|ationationpri|N/A|935308132sandy932871|pale|Pound|Unknown|90|einganticallyationought| +17659|AAAAAAAALPEEAAAA|1997-10-27||Forms find more |6.56|2.62|8001010|amalgnameless #10|1|athletic shoes|8|Sports|20|barable|N/A|16864honeydew3794760|sky|Each|Unknown|6|n stanticallyationought| +17660|AAAAAAAAMPEEAAAA|1997-10-27|2000-10-26|Old, peaceful conditions get as less other strings; partly controversial others label similarly. Home actual workers used to look growing, public plans. Just different conditions survive very labou|1.94|0.98|2004001|edu packimporto #1|4|sports-apparel|2|Men|87|ationeing|large|8465942seashell22193|metallic|Gram|Unknown|10|barcallycallyationought| +17661|AAAAAAAAMPEEAAAA|2000-10-27||Old, peaceful conditions get as less other strings; partly controversial others label similarly. Home actual workers used to look growing, public plans. Just different conditions survive very labou|4.33|0.98|7009002|maxibrand #2|9|mattresses|7|Home|87|ationeing|N/A|8465942seashell22193|maroon|Carton|Unknown|25|oughtcallycallyationought| +17662|AAAAAAAAOPEEAAAA|1997-10-27|1999-10-27|Terrible years see also yesterday |44.30|22.59|7005003|scholarbrand #3|5|blinds/shades|7|Home|438|eingpriese|N/A|6155226papaya0255716|white|Lb|Unknown|25|ablecallycallyationought| +17663|AAAAAAAAOPEEAAAA|1999-10-28|2001-10-26|Openly clear fields shall play. Vast, big entries read only so-called shares. Final aims use for a instruments. At last |57.10|22.59|7007004|brandbrand #4|7|decor|7|Home|621|eingpriese|N/A|6155226papaya0255716|wheat|Pallet|Unknown|62|pricallycallyationought| +17664|AAAAAAAAOPEEAAAA|2001-10-27||Yesterday intellectual resources used to shift cattle. Extra good schools should not lead by a others. Northern patients would not think then substantial, brilliant fe|2.89|22.59|10016002|corpamalgamalg #2|7|wireless|10|Electronics|349|eingpriese|N/A|946362089911linen217|sandy|Ton|Unknown|24|esecallycallyationought| +17665|AAAAAAAABAFEAAAA|1997-10-27||Both relevant firms will not talk considerably various possible bodies. Liberal pages could not understand empty, medical|9.86|7.69|2003002|exportiimporto #2|3|pants|2|Men|440|bareseese|medium|3692029197477red9039|misty|Case|Unknown|74|anticallycallyationought| +17666|AAAAAAAACAFEAAAA|1997-10-27|2000-10-26|Fairly new women shall not reach often open activities. Academic fans must not confirm chemically other, bri|8.75|6.65|9008003|namelessmaxi #3|8|romance|9|Books|120|barableought|N/A|67pale08031728279002|red|Each|Unknown|16|callycallycallyationought| +17667|AAAAAAAACAFEAAAA|2000-10-27||Types tempt enti|3.07|6.65|9013004|exportiunivamalg #4|8|self-help|9|Books|120|barableought|N/A|72218919446271rose68|sky|Bundle|Unknown|69|ationcallycallyationought| +17668|AAAAAAAAEAFEAAAA|1997-10-27|1999-10-27|Camps pay wo|0.92|0.42|8008009|namelessnameless #9|8|outdoor|8|Sports|356|callyantipri|N/A|72304662gainsboro075|hot|Ounce|Unknown|47|eingcallycallyationought| +17669|AAAAAAAAEAFEAAAA|1999-10-28|2001-10-26|Camps pay wo|6.78|2.37|9009006|maximaxi #6|8|science|9|Books|8|eing|N/A|2800007566rose079888|navy|Case|Unknown|67|n stcallycallyationought| +17670|AAAAAAAAEAFEAAAA|2001-10-27||Camps pay wo|7.86|2.67|6012007|importobrand #7|8|costume|6|Jewelry|8|eing|N/A|75433154132390green2|yellow|Ounce|Unknown|29|barationcallyationought| +17671|AAAAAAAAHAFEAAAA|1997-10-27||Strong programmes must appear then central patients. Both large men will hang really. Effective na|3.31|1.05|8009002|maxinameless #2|9|optics|8|Sports|558|eingantianti|N/A|008831860royal380238|slate|Ounce|Unknown|8|oughtationcallyationought| +17672|AAAAAAAAIAFEAAAA|1997-10-27|2000-10-26|Free|0.20|0.13|2004001|edu packimporto #1|4|sports-apparel|2|Men|94|esen st|medium|566419520351papaya41|red|Tbl|Unknown|40|ableationcallyationought| +17673|AAAAAAAAIAFEAAAA|2000-10-27||Free|1.45|0.43|2004001|exportibrand #4|4|kids|7|Home|632|ableprically|N/A|566419520351papaya41|lemon|Carton|Unknown|9|priationcallyationought| +17674|AAAAAAAAKAFEAAAA|1997-10-27|1999-10-27|Fingers ought to find lines. High projects us|1.78|1.51|3004001|edu packexporti #1|4|school-uniforms|3|Children|279|n stationable|large|285883633seashell633|royal|Bunch|Unknown|26|eseationcallyationought| +17675|AAAAAAAAKAFEAAAA|1999-10-28|2001-10-26|Too other thanks ought to play illustrations. Main, high terms can tell organizations; together available publi|7.11|1.51|3004001|exportiedu pack #2|3|kids|4|Shoes|328|eingablepri|medium|285883633seashell633|sienna|Carton|Unknown|56|antiationcallyationought| +17676|AAAAAAAAKAFEAAAA|2001-10-27||Too other thanks ought to play illustrations. Main, high terms can tell organizations; together available publi|3.75|1.51|3004001|exportischolar #1|3|pop|5|Music|328|eingablepri|N/A|87419111151white6081|pink|Pallet|Unknown|52|callyationcallyationought| +17677|AAAAAAAANAFEAAAA|1997-10-27||Legal|0.37|0.22|3001002|amalgexporti #2|1|newborn|3|Children|56|callyanti|medium|207443turquoise19775|turquoise|Unknown|Unknown|25|ationationcallyationought| +17678|AAAAAAAAOAFEAAAA|1997-10-27|2000-10-26|Conditions used to take mentally. Local, good chapters must work total, white attitudes. Great, necessary steps used to fail financial, useful|6.64|2.45|5003001|exportischolar #1|3|pop|5|Music|52|ableanti|N/A|2522snow080729414005|thistle|Tsp|Unknown|10|eingationcallyationought| +17679|AAAAAAAAOAFEAAAA|2000-10-27||Guests prohibit appropriate, dependent lips. |0.49|2.45|5003001|exportiamalg #2|3|maternity|1|Women|183|ableanti|medium|2522snow080729414005|smoke|Tsp|Unknown|9|n stationcallyationought| +17680|AAAAAAAAABFEAAAA|1997-10-27|1999-10-27|Existing, new aim|96.46|34.72|3003001|exportiexporti #1|3|toddlers|3|Children|162|ablecallyought|economy|3534slate46654985855|pale|Tbl|Unknown|4|bareingcallyationought| +17681|AAAAAAAAABFEAAAA|1999-10-28|2001-10-26|Existing, new aim|53.32|33.59|3003001|importoscholar #2|3|country|5|Music|283|ablecallyought|N/A|3534slate46654985855|slate|Dram|Unknown|43|oughteingcallyationought| +17682|AAAAAAAAABFEAAAA|2001-10-27||Existing, new aim|3.25|1.88|3003001|amalgunivamalg #7|3|cooking|9|Books|283|prieingable|N/A|3534slate46654985855|pale|Dozen|Unknown|47|ableeingcallyationought| +17683|AAAAAAAADBFEAAAA|1997-10-27||Options enjoy of|2.04|1.63|6005004|scholarcorp #4|5|earings|6|Jewelry|33|pripri|N/A|8sky2089917198137871|violet|Pallet|Unknown|49|prieingcallyationought| +17684|AAAAAAAAEBFEAAAA|1997-10-27|2000-10-26|Chinese computers imply di|1.78|1.29|3004001|edu packexporti #1|4|school-uniforms|3|Children|119|n stoughtought|large|3176932sandy56006557|rosy|Dozen|Unknown|33|eseeingcallyationought| +17685|AAAAAAAAEBFEAAAA|2000-10-27||Missiles should document a little teenage galleries. Effective, tired designers would not feel large, old ways. Ever local shops turn videos. Certain measures will show just like a|0.15|0.11|2001002|amalgimporto #2|4|accessories|2|Men|320|n stoughtought|medium|3176932sandy56006557|peach|Gross|Unknown|6|antieingcallyationought| +17686|AAAAAAAAGBFEAAAA|1997-10-27|1999-10-27|Heavy events climb also guilty patients. Users ought to draft then consistently local years. Police shall fail at last. New units would not love often in the standards. Industrial|2.22|0.71|10010007|univamalgamalg #7|10|memory|10|Electronics|331|oughtpripri|N/A|80983684steel2617027|slate|Tsp|Unknown|10|callyeingcallyationought| +17687|AAAAAAAAGBFEAAAA|1999-10-28|2001-10-26|Never specific shares condemn; papers imagine following, academic levels. Significant, important characteristics should study firmly sweet, supreme meals. Likely directors may get; m|6.74|2.02|10010007|exportiimporto #2|3|pants|2|Men|233|oughtpripri|large|80983684steel2617027|medium|Dram|Unknown|23|ationeingcallyationought| +17688|AAAAAAAAGBFEAAAA|2001-10-27||Never specific shares condemn; papers imagine following, academic levels. Significant, important characteristics should study firmly sweet, supreme meals. Likely directors may get; m|0.67|2.02|10010007|edu packexporti #1|4|school-uniforms|3|Children|233|pripriable|extra large|7lawn227527315391152|spring|Lb|Unknown|10|eingeingcallyationought| +17689|AAAAAAAAJBFEAAAA|1997-10-27||Foreign, greek heads shall form sufficiently except a tonnes. Satisfactory students o|3.40|1.70|1002002|importoamalg #2|2|fragrances|1|Women|401|oughtbarese|medium|55654935181magenta39|sandy|Dozen|Unknown|51|n steingcallyationought| +17690|AAAAAAAAKBFEAAAA|1997-10-27|2000-10-26|British|7.00|4.83|4004001|edu packedu pack #1|4|athletic|4|Shoes|324|eseablepri|small|608469905steel560735|royal|Bundle|Unknown|47|barn stcallyationought| +17691|AAAAAAAAKBFEAAAA|2000-10-27||Well criminal fort|3.79|2.84|4004001|importoscholar #2|2|country|5|Music|482|eseablepri|N/A|89125956856sienna167|smoke|Tbl|Unknown|64|oughtn stcallyationought| +17692|AAAAAAAAMBFEAAAA|1997-10-27|1999-10-27|Else incredible women must tackle smoothly neverthe|2.99|1.37|9007005|brandmaxi #5|7|reference|9|Books|117|ationoughtought|N/A|17plum30649866881855|thistle|Box|Unknown|83|ablen stcallyationought| +17693|AAAAAAAAMBFEAAAA|1999-10-28|2001-10-26|Excellent, great consequences shall respond yet scarcely grateful a|6.13|3.00|9007005|importoscholar #2|2|country|5|Music|504|ationoughtought|N/A|208183104943389grey8|indian|Gross|Unknown|23|prin stcallyationought| +17694|AAAAAAAAMBFEAAAA|2001-10-27||Currently single friends ought to block; main times throw so. Hence ordinary areas should not leave just fond decent forms. Sometimes daily elections pursue in a businesses|3.28|2.39|6008007|namelesscorp #7|8|mens watch|6|Jewelry|504|esebaranti|N/A|823turquoise35739335|sandy|Tsp|Unknown|52|esen stcallyationought| +17695|AAAAAAAAPBFEAAAA|1997-10-27||Pictures u|9.59|8.53|2001002|amalgimporto #2|1|accessories|2|Men|355|antiantipri|petite|6361purple1765516662|saddle|Ton|Unknown|47|antin stcallyationought| +17696|AAAAAAAAACFEAAAA|1997-10-27|2000-10-26|Parts migh|2.90|1.36|6008001|namelesscorp #1|8|mens watch|6|Jewelry|152|ableantiought|N/A|74958thistle55751447|lemon|Ton|Unknown|25|callyn stcallyationought| +17697|AAAAAAAAACFEAAAA|2000-10-27||Parts migh|7.52|1.36|6008001|exportiimporto #2|3|pants|2|Men|51|oughtanti|small|622458992990579navy3|plum|Cup|Unknown|9|ationn stcallyationought| +17698|AAAAAAAACCFEAAAA|1997-10-27|1999-10-27|Lightly right|7.86|3.61|9013011|exportiunivamalg #11|13|self-help|9|Books|237|ationpriable|N/A|0peach78130754685457|thistle|Bunch|Unknown|29|eingn stcallyationought| +17699|AAAAAAAACCFEAAAA|1999-10-28|2001-10-26|Courses will show to the hours. Simple feelings|9.12|3.61|9013011|importoimporto #2|13|shirts|2|Men|101|ationpriable|economy|0peach78130754685457|royal|Ton|Unknown|5|n stn stcallyationought| +17700|AAAAAAAACCFEAAAA|2001-10-27||Courses will show to the hours. Simple feelings|6.37|3.61|9013011|edu packamalg #1|4|swimwear|1|Women|101|ationpriable|N/A|0peach78130754685457|tomato|Carton|Unknown|49|barbarationationought| +17701|AAAAAAAAFCFEAAAA|1997-10-27||Ago other colours will drive to a band|54.73|36.66|4002002|importoedu pack #2|2|mens|4|Shoes|607|ationbarcally|medium|31798711868wheat2784|puff|Box|Unknown|4|oughtbarationationought| +17702|AAAAAAAAGCFEAAAA|1997-10-27|2000-10-26|Quickly bare factors wear early as a meetings. Physical conventions could not survive. However european bands get due, national paintings. Significant, net facilities initi|33.10|26.81|9003003|exportimaxi #3|3|computers|9|Books|501|oughtbaranti|N/A|4bisque9733982886595|seashell|Tbl|Unknown|52|ablebarationationought| +17703|AAAAAAAAGCFEAAAA|2000-10-27||Early, rural poems live daily hotels. Familiar, national centres see to the views. Key institutions might restrain foreign, len|4.62|3.23|9003003|exportinameless #4|13|wallpaper|7|Home|103|oughtbaranti|N/A|4bisque9733982886595|red|Lb|Unknown|10|pribarationationought| +17704|AAAAAAAAICFEAAAA|1997-10-27|1999-10-27|Directly american y|3.70|1.36|4001001|amalgedu pack #1|1|womens|4|Shoes|106|callybarought|medium|2473218781spring6468|turquoise|Gross|Unknown|8|esebarationationought| +17705|AAAAAAAAICFEAAAA|1999-10-28|2001-10-26|Days will become initial others. Real, round reasons all|8.63|5.09|4001001|exportiimporto #2|3|pants|2|Men|58|callybarought|extra large|2473218781spring6468|ghost|Cup|Unknown|32|antibarationationought| +17706|AAAAAAAAICFEAAAA|2001-10-27||Days will become initial others. Real, round reasons all|4.25|5.09|10015010|scholaramalgamalg #10|3|portable|10|Electronics|10|callybarought|N/A|2473218781spring6468|pale|Gross|Unknown|53|callybarationationought| +17707|AAAAAAAALCFEAAAA|1997-10-27||Concerned inhabitants study additionally modern miles. Sanctions miss interesting, other records; possible, great police lead on a eyes. Years kill howev|0.70|0.55|9013008|exportiunivamalg #8|13|self-help|9|Books|677|ationationcally|N/A|17391peach2891138635|sky|Lb|Unknown|7|ationbarationationought| +17708|AAAAAAAAMCFEAAAA|1997-10-27|2000-10-26|Yet public men wo|6.27|5.32|7003009|exportibrand #9|3|kids|7|Home|289|n steingable|N/A|007044565steel665153|steel|Tsp|Unknown|13|eingbarationationought| +17709|AAAAAAAAMCFEAAAA|2000-10-27||Interesting representatives ought to tell then possible tonnes. Local, high others like respectively essentia|2.68|5.32|7003009|amalgamalgamalg #13|3|disk drives|10|Electronics|289|n steingable|N/A|007044565steel665153|steel|Bundle|Unknown|12|n stbarationationought| +17710|AAAAAAAAOCFEAAAA|1997-10-27|1999-10-27|Level, extended emissions would not take clearly with a years. Interesting services should not write. Co|3.81|2.28|4004001|edu packedu pack #1|4|athletic|4|Shoes|252|ableantiable|large|7495049403651tan2068|bisque|Tbl|Unknown|24|baroughtationationought| +17711|AAAAAAAAOCFEAAAA|1999-10-28|2001-10-26|Level, extended emissions would not take clearly with a years. Interesting services should not write. Co|3.38|2.28|4004001|edu packedu pack #2|4|athletic|4|Shoes|252|ableantiable|medium|800198638sienna29483|peach|Box|Unknown|96|oughtoughtationationought| +17712|AAAAAAAAOCFEAAAA|2001-10-27||Level, extended emissions would not take clearly with a years. Interesting services should not write. Co|98.88|2.28|4004001|univmaxi #3|10|pools|8|Sports|475|ableantiable|N/A|800198638sienna29483|white|Oz|Unknown|26|ableoughtationationought| +17713|AAAAAAAABDFEAAAA|1997-10-27||Magnetic students respond small figures. Tasks may not know less european, scottish months. Characters shall concentrate yesterday still usual systems. Projects|4.91|2.25|9007002|brandmaxi #2|7|reference|9|Books|510|baroughtanti|N/A|659339238520steel187|sandy|Pound|Unknown|17|prioughtationationought| +17714|AAAAAAAACDFEAAAA|1997-10-27|2000-10-26|Important years could prove all electoral families. Unhappy requirements would not prevent; nice, big inst|4.05|1.25|3003001|exportiexporti #1|3|toddlers|3|Children|511|oughtoughtanti|medium|4973337994forest9876|steel|Bunch|Unknown|92|eseoughtationationought| +17715|AAAAAAAACDFEAAAA|2000-10-27||Important years could prove all electoral families. Unhappy requirements would not prevent; nice, big inst|9.98|1.25|3003001|scholarnameless #6|5|fitness|8|Sports|511|oughtoughtanti|N/A|4973337994forest9876|white|Lb|Unknown|36|antioughtationationought| +17716|AAAAAAAAEDFEAAAA|1997-10-27|1999-10-27|Just|0.50|0.41|4004001|edu packedu pack #1|4|athletic|4|Shoes|220|barableable|medium|03681012289236snow40|tan|Oz|Unknown|16|callyoughtationationought| +17717|AAAAAAAAEDFEAAAA|1999-10-28|2001-10-26|Just|6.41|5.12|4004001|importoedu pack #2|2|mens|4|Shoes|565|anticallyanti|extra large|03681012289236snow40|steel|Unknown|Unknown|29|ationoughtationationought| +17718|AAAAAAAAEDFEAAAA|2001-10-27||Just|9.93|4.96|9009007|maximaxi #7|2|science|9|Books|565|anticallyanti|N/A|03681012289236snow40|papaya|Each|Unknown|47|eingoughtationationought| +17719|AAAAAAAAHDFEAAAA|1997-10-27||More charming ideas avoid all central keys. Only countries try at least guests; eggs must not signal bones. Electric russians revive here for a offices. Numerous, real weeks obta|5.73|3.15|5001002|amalgscholar #2|1|rock|5|Music|777|ationationation|N/A|334729pale6939255990|powder|Gross|Unknown|66|n stoughtationationought| +17720|AAAAAAAAIDFEAAAA|1997-10-27|2000-10-26|Fr|6.10|2.37|2001001|amalgimporto #1|1|accessories|2|Men|9|n st|medium|6727903549gainsboro6|gainsboro|Ounce|Unknown|9|barableationationought| +17721|AAAAAAAAIDFEAAAA|2000-10-27||Fr|8.83|2.37|1002002|importoamalg #2|2|fragrances|1|Women|9|n st|economy|9319693257rosy869164|blue|Each|Unknown|7|oughtableationationought| +17722|AAAAAAAAKDFEAAAA|1997-10-27|1999-10-27|High courts subscribe small officers; black, vi|1.14|0.35|10009009|maxiunivamalg #9|9|televisions|10|Electronics|171|oughtationought|N/A|52449spring623154405|snow|Tbl|Unknown|47|ableableationationought| +17723|AAAAAAAAKDFEAAAA|1999-10-28|2001-10-26|High courts subscribe small officers; black, vi|1.20|0.87|4003002|exportiedu pack #2|9|kids|4|Shoes|171|oughtationought|large|52449spring623154405|tan|Gram|Unknown|43|priableationationought| +17724|AAAAAAAAKDFEAAAA|2001-10-27||High courts subscribe small officers; black, vi|5.80|1.91|3001001|amalgexporti #1|9|newborn|3|Children|185|oughtationought|extra large|indian84097228337379|khaki|Ounce|Unknown|88|eseableationationought| +17725|AAAAAAAANDFEAAAA|1997-10-27||Degrees help details. Years get hours. Awkward barriers put now; payments cannot bring deaf, pleased matters. Obvious packages re|4.16|2.99|1001002|amalgamalg #2|1|dresses|1|Women|295|antin stable|petite|26khaki0181911550686|brown|Oz|Unknown|94|antiableationationought| +17726|AAAAAAAAODFEAAAA|1997-10-27|2000-10-26|Detailed, creative kings will act also papers. Due shows find soon new, cool publishers. Days can print how|8.31|2.74|2003001|exportiimporto #1|3|pants|2|Men|812|ableoughteing|economy|71488yellow158712919|powder|Gross|Unknown|61|callyableationationought| +17727|AAAAAAAAODFEAAAA|2000-10-27||Rumours overwhelm for instance. Northern plans like european, quiet notes. Again genetic indians provide nearly dead birds. Cl|5.49|3.18|2003001|importoamalg #2|2|fragrances|1|Women|812|ableoughteing|petite|2665476718971peach04|sienna|N/A|Unknown|21|ationableationationought| +17728|AAAAAAAAAEFEAAAA|1997-10-27|1999-10-27|Large, different benefits might not get stands. Unpleasant, finan|7.56|3.70|8001009|amalgnameless #9|1|athletic shoes|8|Sports|48|eingese|N/A|436violet76854462328|red|Box|Unknown|50|eingableationationought| +17729|AAAAAAAAAEFEAAAA|1999-10-28|2001-10-26|Large, different benefits might not get stands. Unpleasant, finan|9.06|7.97|9005006|scholarmaxi #6|5|history|9|Books|113|eingese|N/A|436violet76854462328|pink|Unknown|Unknown|32|n stableationationought| +17730|AAAAAAAAAEFEAAAA|2001-10-27||Large, different benefits might not get stands. Unpleasant, finan|3.70|7.97|9005006|amalgnameless #1|5|accent|7|Home|608|eingbarcally|N/A|436violet76854462328|dodger|Lb|Unknown|27|barpriationationought| +17731|AAAAAAAADEFEAAAA|1997-10-27||Words remember somewhere types. Just economic points prevent grand effects. |6.66|2.73|1002002|importoamalg #2|2|fragrances|1|Women|94|esen st|N/A|233990plum9535012708|sky|Each|Unknown|8|oughtpriationationought| +17732|AAAAAAAAEEFEAAAA|1997-10-27|2000-10-26|Sufficient, only samples indicate still. Streets take clouds. Services know probably royal times. Old, international seconds must not mean clearly now rich managers. Legs est|6.90|3.24|9015009|scholarunivamalg #9|15|fiction|9|Books|318|eingoughtpri|N/A|02seashell9519461031|plum|Case|Unknown|28|ablepriationationought| +17733|AAAAAAAAEEFEAAAA|2000-10-27||Obviously correct friends may move simply clear producers. Just prime students diagnose. Characteristics counter basic associations. Supreme, romantic circumstances complement never public wom|9.61|7.11|9015009|exportiamalg #2|3|maternity|1|Women|318|eingoughtpri|large|02seashell9519461031|burlywood|Box|Unknown|67|pripriationationought| +17734|AAAAAAAAGEFEAAAA|1997-10-27|1999-10-27|Years get later; only european photographs learn small services. Important, main hours |1.49|0.68|5004001|edu packscholar #1|4|classical|5|Music|849|n steseeing|N/A|6100631red4447270462|tan|Unknown|Unknown|36|esepriationationought| +17735|AAAAAAAAGEFEAAAA|1999-10-28|2001-10-26|Years get later; only european photographs learn small services. Important, main hours |7.27|5.88|6004008|edu packcorp #8|4|bracelets|6|Jewelry|849|n steseeing|N/A|navajo20623306147631|misty|Dozen|Unknown|12|antipriationationought| +17736|AAAAAAAAGEFEAAAA|2001-10-27||Years get later; only european photographs learn small services. Important, main hours |4.07|5.88|6004008|exportiamalgamalg #6|4|stereo|10|Electronics|849|n steseeing|N/A|navajo20623306147631|rose|Ounce|Unknown|87|callypriationationought| +17737|AAAAAAAAJEFEAAAA|1997-10-27||Fine, general photographs find thick black, large negotiations. Different, local operations get e|5.64|4.23|2004002|edu packimporto #2|4|sports-apparel|2|Men|554|eseantianti|medium|723874767285lime6514|turquoise|Pound|Unknown|21|ationpriationationought| +17738|AAAAAAAAKEFEAAAA|1997-10-27|2000-10-26|Now large hospitals show wholly of|1.52|1.06|5002001|importoscholar #1|2|country|5|Music|107|ationbarought|N/A|5755831192335lemon56|pink|Tsp|Unknown|41|eingpriationationought| +17739|AAAAAAAAKEFEAAAA|2000-10-27||Simply personal things promote little ends; so light books help only long, forei|3.21|1.06|8016010|corpmaxi #10|2|golf|8|Sports|295|antin stable|N/A|5755831192335lemon56|gainsboro|Box|Unknown|24|n stpriationationought| +17740|AAAAAAAAMEFEAAAA|1997-10-27|1999-10-27|Victorian magistrates shall explain groups. Evident, german horses pay previous, useful posts. Values regain t|7.62|6.62|10013010|exportiamalgamalg #10|13|stereo|10|Electronics|8|eing|N/A|4047steel68111968499|navy|Gross|Unknown|21|bareseationationought| +17741|AAAAAAAAMEFEAAAA|1999-10-28|2001-10-26|Prices go often; southern, classical prisoners can reduce i|4.18|1.58|10013010|scholarcorp #6|5|earings|6|Jewelry|154|eseantiought|N/A|16372785988wheat7561|linen|Box|Unknown|22|oughteseationationought| +17742|AAAAAAAAMEFEAAAA|2001-10-27||Opposite, international events shall improve available patients; automatically economic wings make russian objectives. Functions kn|3.41|1.32|10013010|amalgexporti #1|5|newborn|3|Children|251|eseantiought|medium|1713614558seashell93|azure|Dram|Unknown|4|ableeseationationought| +17743|AAAAAAAAPEFEAAAA|1997-10-27||Sure, important children see almost net, silve|4.08|3.38|8005004|scholarnameless #4|5|fitness|8|Sports|608|eingbarcally|N/A|1268chiffon717154828|purple|Pound|Unknown|74|prieseationationought| +17744|AAAAAAAAAFFEAAAA|1997-10-27|2000-10-26|Ways watch silly laws. Courts agree difficult, following bodies. Humans suggest. Adverse, different reductio|8.29|3.97|10003014|exportiunivamalg #14|3|dvd/vcr players|10|Electronics|228|eingableable|N/A|wheat525722903964620|royal|Tbl|Unknown|22|eseeseationationought| +17745|AAAAAAAAAFFEAAAA|2000-10-27||Ways watch silly laws. Courts agree difficult, following bodies. Humans suggest. Adverse, different reductio|1.50|3.97|10003014|edu packexporti #2|3|school-uniforms|3|Children|288|eingableable|petite|wheat525722903964620|peach|Dram|Unknown|12|antieseationationought| +17746|AAAAAAAACFFEAAAA|1997-10-27|1999-10-27|Running weeks support then. Important eyes cope on the companies. Young wages seem rights. Unreasonably|8.04|6.27|10004016|edu packunivamalg #16|4|audio|10|Electronics|708|eingbaration|N/A|violet40895363564410|lemon|Bunch|Unknown|60|callyeseationationought| +17747|AAAAAAAACFFEAAAA|1999-10-28|2001-10-26|Running weeks support then. Important eyes cope on the companies. Young wages seem rights. Unreasonably|6.25|3.68|10004016|exportiamalgamalg #17|13|stereo|10|Electronics|101|oughtbarought|N/A|violet40895363564410|lavender|Dram|Unknown|37|ationeseationationought| +17748|AAAAAAAACFFEAAAA|2001-10-27||Often essential ladies should not confine from a|4.49|3.68|1002001|importoamalg #1|13|fragrances|1|Women|530|oughtbarought|extra large|violet40895363564410|seashell|Pallet|Unknown|4|eingeseationationought| +17749|AAAAAAAAFFFEAAAA|1997-10-27||Wings hesitate well great gaps. Firm texts know very on a men; territo|23.04|16.58|7014010|edu packnameless #10|14|glassware|7|Home|130|barpriought|N/A|27870572steel8281104|snow|Carton|Unknown|21|n steseationationought| +17750|AAAAAAAAGFFEAAAA|1997-10-27|2000-10-26|Liabilities ensure together closed players. Relevant, only seats contest central, happy animals; total, likely neighbours address to a characters. International tables make however public temperature|9.11|3.09|2003001|exportiimporto #1|3|pants|2|Men|71|oughtation|medium|1518navajo2983922688|thistle|Gram|Unknown|34|barantiationationought| +17751|AAAAAAAAGFFEAAAA|2000-10-27||Real, national |8.97|3.09|8008002|namelessnameless #2|8|outdoor|8|Sports|431|oughtation|N/A|1518navajo2983922688|pale|Cup|Unknown|42|oughtantiationationought| +17752|AAAAAAAAIFFEAAAA|1997-10-27|1999-10-27|Else ashamed temperatures sue negative things. Groups will|41.35|22.74|9006005|corpmaxi #5|6|parenting|9|Books|630|barprically|N/A|631343metallic768820|medium|N/A|Unknown|15|ableantiationationought| +17753|AAAAAAAAIFFEAAAA|1999-10-28|2001-10-26|Else ashamed temperatures sue negative things. Groups will|6.98|22.74|9006005|amalgedu pack #2|1|womens|4|Shoes|248|eingeseable|medium|58142412676014hot404|white|Lb|Unknown|42|priantiationationought| +17754|AAAAAAAAIFFEAAAA|2001-10-27||Else ashamed temperatures sue negative things. Groups will|91.32|22.74|9006005|importoedu pack #1|1|mens|4|Shoes|248|eingeseable|extra large|2spring5958671662839|pale|Ounce|Unknown|64|eseantiationationought| +17755|AAAAAAAALFFEAAAA|1997-10-27||Crews provide by a children. Flowers challenge scarcely things. Different lakes res|7.52|2.48|2004002|edu packimporto #2|4|sports-apparel|2|Men|636|callyprically|extra large|323805552peach455686|powder|Lb|Unknown|28|antiantiationationought| +17756|AAAAAAAAMFFEAAAA|1997-10-27|2000-10-26|Main eyes pay enterprises. D|0.94|0.73|8001007|amalgnameless #7|1|athletic shoes|8|Sports|327|ationablepri|N/A|1865693858sandy94740|sandy|N/A|Unknown|48|callyantiationationought| +17757|AAAAAAAAMFFEAAAA|2000-10-27||Main eyes pay enterprises. D|0.73|0.50|8001007|exportimaxi #10|1|computers|9|Books|327|ationablepri|N/A|117474676lawn8570647|sandy|Dram|Unknown|2|ationantiationationought| +17758|AAAAAAAAOFFEAAAA|1997-10-27|1999-10-27|Clear benefits should not wake here efficient sons. Much other awards shall not give in a reasons. Lesser plants differ rec|3.15|1.07|3001001|amalgexporti #1|1|newborn|3|Children|665|anticallycally|small|308pink9946850824180|medium|Box|Unknown|37|eingantiationationought| +17759|AAAAAAAAOFFEAAAA|1999-10-28|2001-10-26|Clear benefits should not wake here efficient sons. Much other awards shall not give in a reasons. Lesser plants differ rec|4.06|1.74|5001002|amalgscholar #2|1|rock|5|Music|221|oughtableable|N/A|6050003570turquoise6|spring|Gross|Unknown|25|n stantiationationought| +17760|AAAAAAAAOFFEAAAA|2001-10-27||Appropriate, similar businesses must |23.12|12.48|7002001|importobrand #1|1|bedding|7|Home|115|oughtableable|N/A|6007974khaki23088992|lime|Gram|Unknown|4|barcallyationationought| +17761|AAAAAAAABGFEAAAA|1997-10-27||Cultural, excellent years shall not ame|0.68|0.60|7011002|amalgnameless #2|11|accent|7|Home|522|ableableanti|N/A|66485papaya916443214|turquoise|Ounce|Unknown|17|oughtcallyationationought| +17762|AAAAAAAACGFEAAAA|1997-10-27|2000-10-26|For example voluntary restrictions shall dare recently popular streets. Well-known things recover very fin|6.75|2.97|5003001|exportischolar #1|3|pop|5|Music|261|oughtcallyable|N/A|30437301347802wheat2|spring|Unknown|Unknown|10|ablecallyationationought| +17763|AAAAAAAACGFEAAAA|2000-10-27||Religious, mere years used to witness very points. Cool happy issues suggest. More large effects might change a little relat|5.71|2.96|3002002|importoexporti #2|2|infants|3|Children|261|oughtcallyable|petite|salmon94156239009925|sienna|Ounce|Unknown|63|pricallyationationought| +17764|AAAAAAAAEGFEAAAA|1997-10-27|1999-10-27|Substantial, french conditions would not board original tracks. Away small points remain then other, full friends. Just dead countries ring even. Annual patterns would fi|9.44|5.66|6010005|univbrand #5|10|jewelry boxes|6|Jewelry|531|oughtprianti|N/A|391869141861sienna33|cyan|Ounce|Unknown|3|esecallyationationought| +17765|AAAAAAAAEGFEAAAA|1999-10-28|2001-10-26|Tomorrow illegal areas start then certain months. Paths should produce |1.56|0.73|9015006|scholarunivamalg #6|10|fiction|9|Books|248|oughtprianti|N/A|391869141861sienna33|yellow|Lb|Unknown|35|anticallyationationought| +17766|AAAAAAAAEGFEAAAA|2001-10-27||Certainly funny weapons must draw sometimes new, private weapons. Obvious arrangements will not explain right necessary ladies. Unfortunately dual shops |2.13|1.89|9015006|brandnameless #7|7|hockey|8|Sports|229|n stableable|N/A|8714276maroon2666377|slate|Tbl|Unknown|8|callycallyationationought| +17767|AAAAAAAAHGFEAAAA|1997-10-27||Important, bad stars must pass again statutory lessons. Local speakers can make there strong, real patients. In|2.79|1.33|2004002|edu packimporto #2|4|sports-apparel|2|Men|49|n stese|small|874693359088red38147|royal|Unknown|Unknown|62|ationcallyationationought| +17768|AAAAAAAAIGFEAAAA|1997-10-27|2000-10-26|Grounds discuss on a regulations. Somewhere great years might provide identical|6.48|3.17|5001001|amalgscholar #1|1|rock|5|Music|98|eingn st|N/A|8444173652600red6257|grey|Oz|Unknown|29|eingcallyationationought| +17769|AAAAAAAAIGFEAAAA|2000-10-27||Grounds discuss on a regulations. Somewhere great years might provide identical|3.14|3.17|5001001|edu packexporti #2|1|school-uniforms|3|Children|98|eingn st|small|8444173652600red6257|steel|Each|Unknown|43|n stcallyationationought| +17770|AAAAAAAAKGFEAAAA|1997-10-27|1999-10-27|Early restaurants look at a services. Public, mean connections show. Specific terms could occur obviou|3.42|3.00|1002001|importoamalg #1|2|fragrances|1|Women|134|esepriought|large|06872sienna767175386|honeydew|Pallet|Unknown|23|barationationationought| +17771|AAAAAAAAKGFEAAAA|1999-10-28|2001-10-26|Girls under|2.33|3.00|1002001|edu packamalg #2|4|swimwear|1|Women|134|esepriought|medium|06872sienna767175386|cornsilk|Tbl|Unknown|47|oughtationationationought| +17772|AAAAAAAAKGFEAAAA|2001-10-27||Girls under|1.25|0.58|1002001|amalgexporti #1|4|newborn|3|Children|802|ablebareing|large|6369971732snow495192|rosy|Gram|Unknown|84|ableationationationought| +17773|AAAAAAAANGFEAAAA|1997-10-27||Financial terms show again; more full pictures shall meet there. Regional, |3.80|2.92|9010002|univunivamalg #2|10|travel|9|Books|174|eseationought|N/A|5707625475424white24|slate|Lb|Unknown|36|priationationationought| +17774|AAAAAAAAOGFEAAAA|1997-10-27|2000-10-26|As small boundaries might move however consumers. Just brothers allow relatively later tired|3.98|1.99|7014005|edu packnameless #5|14|glassware|7|Home|391|oughtn stpri|N/A|0038334894973869sky2|seashell|Each|Unknown|46|eseationationationought| +17775|AAAAAAAAOGFEAAAA|2000-10-27||Please only shareholders could not displace. Black, obvious theories mix wel|4.13|2.10|7016006|corpnameless #6|16|furniture|7|Home|391|oughtn stpri|N/A|0038334894973869sky2|sky|Dozen|Unknown|58|antiationationationought| +17776|AAAAAAAAAHFEAAAA|1997-10-27|1999-10-27|Special servants might let so mai|4.35|1.39|1002001|importoamalg #1|2|fragrances|1|Women|469|n stcallyese|extra large|6818896goldenrod8162|maroon|Ton|Unknown|14|callyationationationought| +17777|AAAAAAAAAHFEAAAA|1999-10-28|2001-10-26|Animal|4.62|1.39|10007013|brandunivamalg #13|7|personal|10|Electronics|147|n stcallyese|N/A|4ghost97162483042294|powder|Bundle|Unknown|5|ationationationationought| +17778|AAAAAAAAAHFEAAAA|2001-10-27||Used, impressive purposes should take more words. Other circles can prevent social courts. Huge vehicles might lend processes; indirect examples must not accommoda|8.38|5.19|10007013|amalgnameless #9|1|athletic shoes|8|Sports|147|n stcallyese|N/A|520849341960sienna49|violet|Bunch|Unknown|54|eingationationationought| +17779|AAAAAAAADHFEAAAA|1997-10-27||Organisations shall not resolve about in a figures. Personal efforts should hear much. Other, unable units indica|0.39|0.28|1001002|amalgamalg #2|1|dresses|1|Women|676|callyationcally|large|4707961sandy20556031|slate|Tsp|Unknown|17|n stationationationought| +17780|AAAAAAAAEHFEAAAA|1997-10-27|2000-10-26|Patterns may explore from a parts. Services may leave physical issues; again other cases may confuse as on the representatives. Thus old participants might get clear wo|1.98|1.10|1001001|amalgamalg #1|1|dresses|1|Women|656|callyantically|petite|559honeydew266506986|peach|Tsp|Unknown|3|bareingationationought| +17781|AAAAAAAAEHFEAAAA|2000-10-27||Patterns may explore from a parts. Services may leave physical issues; again other cases may confuse as on the representatives. Thus old participants might get clear wo|9.58|4.69|1001001|exportiimporto #2|1|pants|2|Men|656|callyantically|large|29875honeydew3325559|papaya|Case|Unknown|31|oughteingationationought| +17782|AAAAAAAAGHFEAAAA|1997-10-27|1999-10-27|Certain, historical champions can infl|8.82|4.32|5001001|amalgscholar #1|1|rock|5|Music|136|callypriought|N/A|3473pink174173853307|snow|N/A|Unknown|32|ableeingationationought| +17783|AAAAAAAAGHFEAAAA|1999-10-28|2001-10-26|Certain, historical champions can infl|84.93|69.64|4001002|amalgedu pack #2|1|womens|4|Shoes|220|barableable|petite|3473pink174173853307|papaya|Tbl|Unknown|81|prieingationationought| +17784|AAAAAAAAGHFEAAAA|2001-10-27||Certain, historical champions can infl|82.85|62.96|4001002|amalgexporti #1|1|newborn|3|Children|735|barableable|economy|3473pink174173853307|violet|Gram|Unknown|29|eseeingationationought| +17785|AAAAAAAAJHFEAAAA|1997-10-27||Bombs prove polls. Obviously full tears remain never. Political, unique countries hang through a r|99.28|87.36|2004002|edu packimporto #2|4|sports-apparel|2|Men|131|oughtpriought|petite|97501293snow33096216|magenta|Gross|Unknown|6|antieingationationought| +17786|AAAAAAAAKHFEAAAA|1997-10-27|2000-10-26|Adequate parties may not post less strange services. Universities obtain well identical options. Pleased, chief women might force mad seats. Separately angry languages may not live from a visit|3.83|1.22|8004007|edu packnameless #7|4|camping|8|Sports|65|antically|N/A|669146816maroon13743|floral|Gram|Unknown|7|callyeingationationought| +17787|AAAAAAAAKHFEAAAA|2000-10-27||Due cars cannot presume very hungry, present families. Sides s|6.61|1.22|10002006|importounivamalg #6|2|camcorders|10|Electronics|65|antically|N/A|669146816maroon13743|wheat|N/A|Unknown|56|ationeingationationought| +17788|AAAAAAAAMHFEAAAA|1997-10-27|1999-10-27|Again afraid friends expose pairs; women tend additional churches. Only good criticisms think welcome, appropriate points. More private packages choose less relati|3.36|2.62|7009009|maxibrand #9|9|mattresses|7|Home|147|ationeseought|N/A|1378809thistle126488|cyan|Dram|Unknown|15|eingeingationationought| +17789|AAAAAAAAMHFEAAAA|1999-10-28|2001-10-26|Again afraid friends expose pairs; women tend additional churches. Only good criticisms think welcome, appropriate points. More private packages choose less relati|7.10|4.89|7009009|corpunivamalg #6|9|mystery|9|Books|147|ationeseought|N/A|1378809thistle126488|sienna|Unknown|Unknown|27|n steingationationought| +17790|AAAAAAAAMHFEAAAA|2001-10-27||Details breed children. At best ce|47.52|37.54|7009009|edu packbrand #1|4|curtains/drapes|7|Home|836|ationeseought|N/A|94412989221slate4103|peach|Bundle|Unknown|29|barn stationationought| +17791|AAAAAAAAPHFEAAAA|1997-10-27||Answers |6.79|2.17|1004002|edu packamalg #2|4|swimwear|1|Women|225|antiableable|medium|51067695sky460245561|frosted|Unknown|Unknown|3|oughtn stationationought| +17792|AAAAAAAAAIFEAAAA|1997-10-27|2000-10-26|Pieces ought to give away workers. Comfortable, brief libraries will not remain also surely inc points. International year|16.79|13.26|6006001|corpcorp #1|6|rings|6|Jewelry|604|esebarcally|N/A|691296thistle9976312|tomato|Tsp|Unknown|71|ablen stationationought| +17793|AAAAAAAAAIFEAAAA|2000-10-27||Pieces ought to give away workers. Comfortable, brief libraries will not remain also surely inc points. International year|3.64|2.47|4002002|importoedu pack #2|6|mens|4|Shoes|604|esebarcally|small|691296thistle9976312|wheat|Dram|Unknown|33|prin stationationought| +17794|AAAAAAAACIFEAAAA|1997-10-27|1999-10-27|Waves ought to stay once again initial, safe meetings. Independent, easy islands treat unchanged enterprises. Small, african cases ad|5.52|1.76|7012005|importonameless #5|12|paint|7|Home|159|n stantiought|N/A|305396peru1757467727|orchid|Cup|Unknown|10|esen stationationought| +17795|AAAAAAAACIFEAAAA|1999-10-28|2001-10-26|Waves ought to stay once again initial, safe meetings. Independent, easy islands treat unchanged enterprises. Small, african cases ad|7.57|1.76|6015004|scholarbrand #4|12|custom|6|Jewelry|436|n stantiought|N/A|305396peru1757467727|wheat|Pallet|Unknown|8|antin stationationought| +17796|AAAAAAAACIFEAAAA|2001-10-27||British women must help difficult minutes. Other years should not disguise at all general systems. Full, intimate careers may get poles; private men eat from the points. |5.18|1.76|6015004|importoscholar #1|2|country|5|Music|436|callypriese|N/A|42356lawn69824880475|sandy|Pallet|Unknown|53|callyn stationationought| +17797|AAAAAAAAFIFEAAAA|1997-10-27||Tests can press very for a waves. Initial, wooden steps care predominantly boxes. Common, old drivers would not deliver over vast, female interests. Substantial, tory operations mention|6.31|4.98|10012016|importoamalgamalg #16|12|monitors|10|Electronics|159|n stantiought|N/A|50576orchid943460299|salmon|Gross|Unknown|24|ationn stationationought| +17798|AAAAAAAAGIFEAAAA|1997-10-27|2000-10-26|Northern, sure arts |5.33|3.19|8007009|brandnameless #9|7|hockey|8|Sports|609|n stbarcally|N/A|427tan35109981253082|snow|Gram|Unknown|40|eingn stationationought| +17799|AAAAAAAAGIFEAAAA|2000-10-27||Big, financial meetings might defend now environmental quantities. Members must visit little hours. Goods must encourage there. Remaining, local tourist|3.06|3.19|8007009|exportiimporto #2|7|pants|2|Men|793|n stbarcally|medium|427tan35109981253082|sandy|Ton|Unknown|66|n stn stationationought| +17800|AAAAAAAAIIFEAAAA|1997-10-27|1999-10-27|Great, happy museums comply social, national strategies. Fast difficult pressures would remember really good, economic proceedings. International, expensiv|1.03|0.78|4003001|exportiedu pack #1|3|kids|4|Shoes|312|ableoughtpri|extra large|1739947cornsilk11820|peru|Each|Unknown|54|barbareingationought| +17801|AAAAAAAAIIFEAAAA|1999-10-28|2001-10-26|Great, happy museums comply social, national strategies. Fast difficult pressures would remember really good, economic proceedings. International, expensiv|3.30|1.18|4003001|exportischolar #2|3|pop|5|Music|820|ableoughtpri|N/A|24609557393grey00739|misty|Cup|Unknown|11|oughtbareingationought| +17802|AAAAAAAAIIFEAAAA|2001-10-27||Great, happy museums comply social, national strategies. Fast difficult pressures would remember really good, economic proceedings. International, expensiv|1.33|0.61|4003001|brandbrand #3|7|decor|7|Home|820|ableoughtpri|N/A|1334237tomato6364326|steel|Dozen|Unknown|25|ablebareingationought| +17803|AAAAAAAALIFEAAAA|1997-10-27||Even single waters make for instance particular hours. Mental rights may cross as just contemporary m|0.97|0.82|10002005|importounivamalg #5|2|camcorders|10|Electronics|65|antically|N/A|368505164turquoise21|powder|Dram|Unknown|27|pribareingationought| +17804|AAAAAAAAMIFEAAAA|1997-10-27|2000-10-26|New, deaf sites find little small, active towns. Advanced modules come still to a ears. Soviet businessmen may speculate further complicated things; then difficult beliefs like. For instance h|61.72|46.90|5002001|importoscholar #1|2|country|5|Music|350|barantipri|N/A|21667789212plum22836|maroon|Box|Unknown|31|esebareingationought| +17805|AAAAAAAAMIFEAAAA|2000-10-27||New, deaf sites find little small, active towns. Advanced modules come still to a ears. Soviet businessmen may speculate further complicated things; then difficult beliefs like. For instance h|3.56|46.90|5002001|amalgexporti #2|2|newborn|3|Children|66|barantipri|large|6tan0553307297572665|navajo|Case|Unknown|43|antibareingationought| +17806|AAAAAAAAOIFEAAAA|1997-10-27|1999-10-27|Concrete, american reductions could not produce; social, cultural gates use; high |7.30|4.96|6007007|brandcorp #7|7|pendants|6|Jewelry|47|ationese|N/A|73281428728pale12026|thistle|Box|Unknown|8|callybareingationought| +17807|AAAAAAAAOIFEAAAA|1999-10-28|2001-10-26|Concrete, american reductions could not produce; social, cultural gates use; high |6.82|4.96|3003002|exportiexporti #2|7|toddlers|3|Children|47|ationese|medium|papaya55177027537426|wheat|Dram|Unknown|35|ationbareingationought| +17808|AAAAAAAAOIFEAAAA|2001-10-27||Concrete, american reductions could not produce; social, cultural gates use; high |9.92|4.96|3003002|edu packamalg #1|4|swimwear|1|Women|399|n stn stpri|medium|papaya55177027537426|aquamarine|Tbl|Unknown|54|eingbareingationought| +17809|AAAAAAAABJFEAAAA|1997-10-27||Less re|0.99|0.86|5003002|exportischolar #2|3|pop|5|Music|54|eseanti|N/A|999281salmon24573606|smoke|Tbl|Unknown|33|n stbareingationought| +17810|AAAAAAAACJFEAAAA|1997-10-27|2000-10-26|Molecular eyes turn different terms. Details will attack large, implicit members. Acceptable, only drugs br|2.95|1.77|8014001|edu packmaxi #1|14|tennis|8|Sports|423|priableese|N/A|624rosy3318091833618|snow|Box|Unknown|99|baroughteingationought| +17811|AAAAAAAACJFEAAAA|2000-10-27||Molecular eyes turn different terms. Details will attack large, implicit members. Acceptable, only drugs br|1.55|1.77|1003002|exportiamalg #2|14|maternity|1|Women|452|priableese|large|yellow28383644598748|medium|Box|Unknown|41|oughtoughteingationought| +17812|AAAAAAAAEJFEAAAA|1997-10-27|1999-10-27|Soviet girls draw continuous, full faces. High main attempts stay publicly for a affairs. Useful, large activities rely other, weak responses. New words cannot remain. Fellow differences share for lo|8.56|6.24|6015005|scholarbrand #5|15|custom|6|Jewelry|133|pripriought|N/A|22370454699lavender4|orange|Bundle|Unknown|16|ableoughteingationought| +17813|AAAAAAAAEJFEAAAA|1999-10-28|2001-10-26|New, new trees reflect yellow, redundant organizations. Regular women consider central, present windows. For example british arrangem|2.14|6.24|6015005|importoimporto #2|2|shirts|2|Men|528|eingableanti|extra large|22370454699lavender4|linen|Box|Unknown|11|prioughteingationought| +17814|AAAAAAAAEJFEAAAA|2001-10-27||Situations will change on the positions. Practical, acceptable shows indicate unive|1.02|6.24|6015005|exportiamalg #1|2|maternity|1|Women|374|eingableanti|small|06188090922992pale52|saddle|Carton|Unknown|49|eseoughteingationought| +17815|AAAAAAAAHJFEAAAA|1997-10-27||Months boost more. Standards enter certainly full, soft words. Catholic grounds might not reveal. Alike limited years mus|3.06|1.43|8005006|scholarnameless #6|5|fitness|8|Sports|489|n steingese|N/A|5826642125snow514926|wheat|Oz|Unknown|37|antioughteingationought| +17816|AAAAAAAAIJFEAAAA|1997-10-27|2000-10-26|Human thousands appreciate.|26.15|22.48|4001001|amalgedu pack #1|1|womens|4|Shoes|619|n stoughtcally|large|3001337lavender96243|lace|Ounce|Unknown|17|callyoughteingationought| +17817|AAAAAAAAIJFEAAAA|2000-10-27||Human thousands appreciate.|8.81|7.75|5004002|edu packscholar #2|4|classical|5|Music|619|n stoughtcally|N/A|23turquoise032563795|peach|Gram|Unknown|37|ationoughteingationought| +17818|AAAAAAAAKJFEAAAA|1997-10-27|1999-10-27|Facts should not win internal, cool hours. Spare wheels should not choose very other facts. Effective, local teachers keep small lines. Difficult, presiden|1.06|0.71|6003003|exporticorp #3|3|gold|6|Jewelry|79|n station|N/A|65thistle05204647648|purple|Dram|Unknown|9|eingoughteingationought| +17819|AAAAAAAAKJFEAAAA|1999-10-28|2001-10-26|Facts should not win internal, cool hours. Spare wheels should not choose very other facts. Effective, local teachers keep small lines. Difficult, presiden|1.22|0.71|6003003|exportischolar #2|3|pop|5|Music|79|n station|N/A|62378678299red698759|tan|Tsp|Unknown|82|n stoughteingationought| +17820|AAAAAAAAKJFEAAAA|2001-10-27||Conservatives should ask. Likely conditions see abilities; young, armed decisions talk sudden cars. International, ancient problems commit. Stars can ensure different parts. Full, worried problems|6.90|0.71|6003003|amalgimporto #1|1|accessories|2|Men|59|n stanti|large|4763780429papaya6599|tomato|Bunch|Unknown|14|barableeingationought| +17821|AAAAAAAANJFEAAAA|1997-10-27||Very, new trends should not des|3.14|1.38|9004002|edu packmaxi #2|4|entertainments|9|Books|621|oughtablecally|N/A|92peach7987402734572|snow|Tsp|Unknown|16|oughtableeingationought| +17822|AAAAAAAAOJFEAAAA|1997-10-27|2000-10-26|Thereby |31.17|11.84|7007003|brandbrand #3|7|decor|7|Home|181|oughteingought|N/A|16262419136orange401|pale|Ton|Unknown|19|ableableeingationought| +17823|AAAAAAAAOJFEAAAA|2000-10-27||Useful unions may not announce always different girls. Certain, able armies cannot c|9.69|8.43|7007003|importoimporto #2|2|shirts|2|Men|165|anticallyought|medium|16262419136orange401|spring|Ton|Unknown|64|priableeingationought| +17824|AAAAAAAAAKFEAAAA|1997-10-27|1999-10-27|Happy experiments could see now odd fingers. Previous matters might |0.79|0.26|1002001|importoamalg #1|2|fragrances|1|Women|429|n stableese|petite|00356lace94006261653|dodger|Pallet|Unknown|18|eseableeingationought| +17825|AAAAAAAAAKFEAAAA|1999-10-28|2001-10-26|Happy experiments could see now odd fingers. Previous matters might |2.37|0.26|8016006|corpmaxi #6|2|golf|8|Sports|429|n stableese|N/A|31green5309087901591|drab|Box|Unknown|43|antiableeingationought| +17826|AAAAAAAAAKFEAAAA|2001-10-27||Churches used to ask always right, due men. Dark, various children encourage statistically; merely slow possibilities should not get for example half-way prepa|0.39|0.24|8016006|importoedu pack #1|2|mens|4|Shoes|429|n stableese|medium|3red6776168509879555|powder|Gram|Unknown|21|callyableeingationought| +17827|AAAAAAAADKFEAAAA|1997-10-27||Remaining months provide; sessions think all on a years. Absent programmes could not adapt indian, arbitrary groups. True, huge programmes emer|8.03|7.06|6011004|amalgbrand #4|11|semi-precious|6|Jewelry|558|eingantianti|N/A|553882852834thistle8|antique|Carton|Unknown|20|ationableeingationought| +17828|AAAAAAAAEKFEAAAA|1997-10-27|2000-10-26|Afraid, grey officers mean costly institutions. Societi|9.13|3.92|9005009|scholarmaxi #9|5|history|9|Books|231|oughtpriable|N/A|49913850tomato635811|sky|Each|Unknown|31|eingableeingationought| +17829|AAAAAAAAEKFEAAAA|2000-10-27||Often obvious planes would not speak yesterday precious organic companies. Again military ways treat occupational benefits. Ultima|1.97|1.18|9005009|scholarmaxi #10|5|history|9|Books|231|oughtpriable|N/A|49913850tomato635811|peach|Dozen|Unknown|16|n stableeingationought| +17830|AAAAAAAAGKFEAAAA|1997-10-27|1999-10-27|Aware things shall not need then really true values. Other universities lift generally significant persons. There limited patterns can put individually eventual businesses; forwa|3.23|1.71|6013007|exportibrand #7|13|loose stones|6|Jewelry|539|n stprianti|N/A|2585793262978636tan9|misty|Pallet|Unknown|41|barprieingationought| +17831|AAAAAAAAGKFEAAAA|1999-10-28|2001-10-26|Aware things shall not need then really true values. Other universities lift generally significant persons. There limited patterns can put individually eventual businesses; forwa|1.82|1.71|2001002|amalgimporto #2|1|accessories|2|Men|261|n stprianti|medium|59966397363834tan671|honeydew|Ounce|Unknown|8|oughtprieingationought| +17832|AAAAAAAAGKFEAAAA|2001-10-27||Aware things shall not need then really true values. Other universities lift generally significant persons. There limited patterns can put individually eventual businesses; forwa|55.99|32.47|2001002|importoedu pack #1|1|mens|4|Shoes|253|n stprianti|petite|59966397363834tan671|white|Box|Unknown|97|ableprieingationought| +17833|AAAAAAAAJKFEAAAA|1997-10-27||Boards may specify by the debts. Nice arms establish good, external positions; there religious genes used to accept only on the voters. Ever friendly walls used to find officiall|4.27|2.17|3003002|exportiexporti #2|3|toddlers|3|Children|458|eingantiese|petite|05plum61565478915778|papaya|Each|Unknown|75|priprieingationought| +17834|AAAAAAAAKKFEAAAA|1997-10-27|2000-10-26|Reforms may not reduce slowly on a meetings. Opposite, italian boys publish much high traditions. Occasionally traditional ministers|3.13|1.25|8009005|maxinameless #5|9|optics|8|Sports|407|ationbarese|N/A|94cornflower77404373|sky|Cup|Unknown|50|eseprieingationought| +17835|AAAAAAAAKKFEAAAA|2000-10-27||Reforms may not reduce slowly on a meetings. Opposite, italian boys publish much high traditions. Occasionally traditional ministers|7.20|1.25|8009005|edu packunivamalg #3|4|audio|10|Electronics|407|ationbarese|N/A|55448542675151green7|violet|Unknown|Unknown|42|antiprieingationought| +17836|AAAAAAAAMKFEAAAA|1997-10-27|1999-10-27|Strong, extreme agents must not care open reasons. So only sentences might not learn; earlier right hands go immediatel|4.86|4.03|3002001|importoexporti #1|2|infants|3|Children|297|ationn stable|medium|84429040saddle702762|seashell|Oz|Unknown|10|callyprieingationought| +17837|AAAAAAAAMKFEAAAA|1999-10-28|2001-10-26|Complete devices turn considerably obvious, large islands; other, worthy horses will not reveal probably other, involved claims; territorial, negative systems shall appreciate all real ca|1.24|4.03|3002001|exportiexporti #2|2|toddlers|3|Children|297|ationn stable|large|84429040saddle702762|puff|Gram|Unknown|55|ationprieingationought| +17838|AAAAAAAAMKFEAAAA|2001-10-27||Honest tanks let other relatives. Good personal arguments should find in the lives. J|8.59|5.75|1004001|edu packamalg #1|2|swimwear|1|Women|297|ationn stable|medium|243357884162600pale3|yellow|Ounce|Unknown|54|eingprieingationought| +17839|AAAAAAAAPKFEAAAA|1997-10-27||Conditions admit essential, considerable drugs. Pregnant, new parents can help thereby reasons. Different fires used to |3.18|1.59|6008008|namelesscorp #8|8|mens watch|6|Jewelry|527|ationableanti|N/A|46slate1897537342538|peach|Gross|Unknown|28|n stprieingationought| +17840|AAAAAAAAALFEAAAA|1997-10-27|2000-10-26|Distant objectives sign things; fat authorities will get uneasily american things; circumstances must dismantle comparisons. Countries wou|9.28|4.17|10006008|corpunivamalg #8|6|musical|10|Electronics|275|antiationable|N/A|0021142625salmon3238|forest|Dozen|Unknown|10|bareseeingationought| +17841|AAAAAAAAALFEAAAA|2000-10-27||Distant objectives sign things; fat authorities will get uneasily american things; circumstances must dismantle comparisons. Countries wou|36.98|4.17|10006008|corpunivamalg #9|6|musical|10|Electronics|274|antiationable|N/A|0021142625salmon3238|papaya|Dram|Unknown|1|oughteseeingationought| +17842|AAAAAAAACLFEAAAA|1997-10-27|1999-10-27|Young, nation|0.49|0.24|8004003|edu packnameless #3|4|camping|8|Sports|62|ablecally|N/A|794622736591695pale2|sandy|Oz|Unknown|28|ableeseeingationought| +17843|AAAAAAAACLFEAAAA|1999-10-28|2001-10-26|Over late woods apply a|3.57|0.24|8004003|amalgscholar #2|4|rock|5|Music|44|ablecally|N/A|580red92973870226361|yellow|Cup|Unknown|31|prieseeingationought| +17844|AAAAAAAACLFEAAAA|2001-10-27||Over late woods apply a|7.20|0.24|4004001|edu packedu pack #1|4|athletic|4|Shoes|594|ablecally|petite|4624966306126green60|peach|Each|Unknown|55|eseeseeingationought| +17845|AAAAAAAAFLFEAAAA|1997-10-27||Social weeks may hope. However parental objects shall get just potential logical stations. Agreements attend on a arms; circa real reforms may interpret dogs. T|2.06|1.11|9003002|exportimaxi #2|3|computers|9|Books|331|oughtpripri|N/A|1925597purple8858440|salmon|Cup|Unknown|10|antieseeingationought| +17846|AAAAAAAAGLFEAAAA|1997-10-27|2000-10-26|High firms tie open, local unions. Pregnant, accessible affairs shall agree over. Widespread, new spirits offset then good, |1.48|0.54|10014014|edu packamalgamalg #14|14|automotive|10|Electronics|17|ationought|N/A|738962rose2960898378|light|Ounce|Unknown|66|callyeseeingationought| +17847|AAAAAAAAGLFEAAAA|2000-10-27||Pages must protest all right equal types. Vast times read on a standards. Spirits should not comment. New, small eyes include in a issues. About|3.40|1.25|9010004|univunivamalg #4|14|travel|9|Books|239|n stpriable|N/A|738962rose2960898378|sandy|Pallet|Unknown|4|ationeseeingationought| +17848|AAAAAAAAILFEAAAA|1997-10-27|1999-10-27|Present, nervous schools look transactions.|4.02|2.61|7007009|brandbrand #9|7|decor|7|Home|263|pricallyable|N/A|68256613085rosy08756|sandy|Pallet|Unknown|27|eingeseeingationought| +17849|AAAAAAAAILFEAAAA|1999-10-28|2001-10-26|Present, nervous schools look transactions.|8.74|2.61|7007009|edu packnameless #10|7|camping|8|Sports|263|pricallyable|N/A|68256613085rosy08756|royal|Tbl|Unknown|50|n steseeingationought| +17850|AAAAAAAAILFEAAAA|2001-10-27||Harshly dou|3.08|2.40|2001001|amalgimporto #1|7|accessories|2|Men|232|pricallyable|small|68256613085rosy08756|papaya|Tbl|Unknown|93|barantieingationought| +17851|AAAAAAAALLFEAAAA|1997-10-27||More than long times ought to judge. Straight critical symptoms keep interests. Civil chairs pump fully dead politicians; live, vague instructions shal|4.10|2.54|6012004|importobrand #4|12|costume|6|Jewelry|640|baresecally|N/A|0319219620midnight08|purple|Dozen|Unknown|22|oughtantieingationought| +17852|AAAAAAAAMLFEAAAA|1997-10-27|2000-10-26|Times ought to integrate more arms; more real windows used to travel national shares. Weeks last modest shoes. Straightforward mont|3.93|3.30|2004001|edu packimporto #1|4|sports-apparel|2|Men|917|ationoughtn st|medium|30wheat2338262128756|brown|Tsp|Unknown|21|ableantieingationought| +17853|AAAAAAAAMLFEAAAA|2000-10-27||Times ought to integrate more arms; more real windows used to travel national shares. Weeks last modest shoes. Straightforward mont|6.60|3.30|2004001|importoedu pack #2|4|mens|4|Shoes|917|ationoughtn st|N/A|934sienna57779826509|mint|Carton|Unknown|87|priantieingationought| +17854|AAAAAAAAOLFEAAAA|1997-10-27|1999-10-27|Victorian families achieve so at least welsh teachers. Often german changes may not participate hard f|8.75|6.73|1001001|amalgamalg #1|1|dresses|1|Women|586|callyeinganti|medium|5248975957266rose584|rosy|Bunch|Unknown|2|eseantieingationought| +17855|AAAAAAAAOLFEAAAA|1999-10-28|2001-10-26|Victorian families achieve so at least welsh teachers. Often german changes may not participate hard f|7.16|6.08|4001002|amalgedu pack #2|1|womens|4|Shoes|586|callyeinganti|small|5248975957266rose584|dodger|Ton|Unknown|11|antiantieingationought| +17856|AAAAAAAAOLFEAAAA|2001-10-27||Decisions acknowledge low, western services. Industrial aspects enjoy. English hands dismiss in a products. Quite central fathers must set|9.39|6.29|3003001|exportiexporti #1|3|toddlers|3|Children|586|callyeinganti|large|5248975957266rose584|yellow|Box|Unknown|11|callyantieingationought| +17857|AAAAAAAABMFEAAAA|1997-10-27||Active, available books should react more processes. Only democratic fac|0.97|0.69|4002002|importoedu pack #2|2|mens|4|Shoes|508|eingbaranti|large|703midnight103095018|white|Tbl|Unknown|39|ationantieingationought| +17858|AAAAAAAACMFEAAAA|1997-10-27|2000-10-26|Concerned, vulnerable keys should see systems. Monthly, old days develop rules. Obvious, alive items say then accounts. Railways sell then darling workers. Free, natural police shall|4.56|2.23|7012009|importonameless #9|12|paint|7|Home|2|able|N/A|2527pale054305260987|almond|Case|Unknown|51|eingantieingationought| +17859|AAAAAAAACMFEAAAA|2000-10-27||Obviously major earnings may talk in a sides. Comprehensive services will know presidential, legal jobs. Appointed bodies change industries. Most inc house|9.95|8.25|2003002|exportiimporto #2|3|pants|2|Men|8|able|large|827570734946rose3281|sky|Pound|Unknown|77|n stantieingationought| +17860|AAAAAAAAEMFEAAAA|1997-10-27|1999-10-27|Classes clean best public, fresh subjects. Eyes define both in the moves. Twice physical substances lunch earlier; advanced, simple cases depend else individual, single e|4.56|3.05|8007001|brandnameless #1|7|hockey|8|Sports|448|eingeseese|N/A|856064752501728red16|thistle|Each|Unknown|35|barcallyeingationought| +17861|AAAAAAAAEMFEAAAA|1999-10-28|2001-10-26|Strange men should not mean really angry americans. |0.67|0.49|8007001|exportimaxi #6|3|computers|9|Books|448|eingeseese|N/A|856064752501728red16|plum|Pound|Unknown|98|oughtcallyeingationought| +17862|AAAAAAAAEMFEAAAA|2001-10-27||Windows get thereof different, particular scientists. Single problems may not conceal in the habits. Adults could relate yesterd|5.48|0.49|8007001|namelessmaxi #7|3|romance|9|Books|448|eingeseese|N/A|5lavender45235062117|pink|Oz|Unknown|5|ablecallyeingationought| +17863|AAAAAAAAHMFEAAAA|1997-10-27||Together other angles try quite so urban stations. Right, valuable shops keep early sure, full|3.22|1.77|5004002|edu packscholar #2|4|classical|5|Music|327|ationablepri|N/A|57210307thistle09372|orchid|N/A|Unknown|63|pricallyeingationought| +17864|AAAAAAAAIMFEAAAA|1997-10-27|2000-10-26|Feeli|3.27|1.86|5001001|amalgscholar #1|1|rock|5|Music|150|barantiought|N/A|6wheat25446591670087|pink|Dozen|Unknown|95|esecallyeingationought| +17865|AAAAAAAAIMFEAAAA|2000-10-27||Serious, foreign papers|4.66|1.95|5001001|edu packamalg #2|4|swimwear|1|Women|4|barantiought|medium|6wheat25446591670087|aquamarine|Lb|Unknown|51|anticallyeingationought| +17866|AAAAAAAAKMFEAAAA|1997-10-27|1999-10-27|Gardens earn alone between a women. Good settings|0.68|0.32|3004001|edu packexporti #1|4|school-uniforms|3|Children|56|callyanti|medium|840midnight786690656|lawn|Oz|Unknown|22|callycallyeingationought| +17867|AAAAAAAAKMFEAAAA|1999-10-28|2001-10-26|Businesses can study half successful lea|0.50|0.27|9006012|corpmaxi #12|4|parenting|9|Books|56|callyanti|N/A|840midnight786690656|hot|Cup|Unknown|39|ationcallyeingationought| +17868|AAAAAAAAKMFEAAAA|2001-10-27||Businesses can study half successful lea|3.83|2.41|9006012|importoedu pack #1|2|mens|4|Shoes|56|callyanti|medium|840midnight786690656|royal|Unknown|Unknown|4|eingcallyeingationought| +17869|AAAAAAAANMFEAAAA|1997-10-27||Low meals c|6.53|3.13|9015002|scholarunivamalg #2|15|fiction|9|Books|979|n stationn st|N/A|10035945rosy82078174|peach|N/A|Unknown|22|n stcallyeingationought| +17870|AAAAAAAAOMFEAAAA|1997-10-27|2000-10-26|Delicious a|4.27|2.81|3002001|importoexporti #1|2|infants|3|Children|123|priableought|medium|04606748152767peach3|puff|Pallet|Unknown|39|barationeingationought| +17871|AAAAAAAAOMFEAAAA|2000-10-27||Delicious a|45.01|20.25|6007008|brandcorp #8|2|pendants|6|Jewelry|785|antieingation|N/A|578897peru7317060185|gainsboro|Gross|Unknown|10|oughtationeingationought| +17872|AAAAAAAAANFEAAAA|1997-10-27|1999-10-27|Gradual politicians may not go good, increased eyes. Only little signs tell in a hours. Red, important systems prevent badly characters; men provide so round companies. N|0.11|0.05|5004001|edu packscholar #1|4|classical|5|Music|222|ableableable|N/A|9933004632slate58050|peru|Bunch|Unknown|44|ableationeingationought| +17873|AAAAAAAAANFEAAAA|1999-10-28|2001-10-26|Early costs coul|1.23|1.07|8012004|importomaxi #4|12|guns|8|Sports|222|ableableable|N/A|9933004632slate58050|tomato|Lb|Unknown|22|priationeingationought| +17874|AAAAAAAAANFEAAAA|2001-10-27||Soft children might take recent nations. Thus sufficient years cope urgently causes. Popular colleagues evaluate cheap overseas lakes. Large places should not meet co|2.07|1.07|9016007|corpunivamalg #7|12|mystery|9|Books|104|ableableable|N/A|13thistle31223836158|ghost|Tbl|Unknown|14|eseationeingationought| +17875|AAAAAAAADNFEAAAA|1997-10-27||Now practical surveys will vary experts. Personal, common bodies pass quickly. Happy, social representatives may expec|8.77|3.77|3003002|exportiexporti #2|3|toddlers|3|Children|552|ableantianti|medium|508279purple65451938|peach|Box|Unknown|9|antiationeingationought| +17876|AAAAAAAAENFEAAAA|1997-10-27|2000-10-26|British, possible pupils attend nev|0.22|0.13|4001001|amalgedu pack #1|1|womens|4|Shoes|469|n stcallyese|petite|063ivory247286759014|khaki|Each|Unknown|23|callyationeingationought| +17877|AAAAAAAAENFEAAAA|2000-10-27||British, possible pupils attend nev|61.59|52.96|4001001|edu packexporti #2|4|school-uniforms|3|Children|354|eseantipri|economy|40ghost1940893846941|peru|Pound|Unknown|31|ationationeingationought| +17878|AAAAAAAAGNFEAAAA|1997-10-27|1999-10-27|Jeans can take to the times. Free surfaces used|0.36|0.29|3002001|importoexporti #1|2|infants|3|Children|245|antieseable|small|7583474plum892082872|wheat|Case|Unknown|25|eingationeingationought| +17879|AAAAAAAAGNFEAAAA|1999-10-28|2001-10-26|Jeans can take to the times. Free surfaces used|4.74|3.08|1004002|edu packamalg #2|2|swimwear|1|Women|245|antieseable|medium|393312578ivory815967|green|Box|Unknown|43|n stationeingationought| +17880|AAAAAAAAGNFEAAAA|2001-10-27||Extended police improve whole accounts. Very complex elements see very charges; strictly poor countries can ju|2.78|1.97|3001001|amalgexporti #1|1|newborn|3|Children|24|antieseable|medium|393312578ivory815967|navajo|Tbl|Unknown|7|bareingeingationought| +17881|AAAAAAAAJNFEAAAA|1997-10-27||Men lift fit letters. Recent shares can give main, new substances. Chains help at the rights. Straightforward things show just european, useful shelves. Healthy combinati|0.77|0.61|9004002|edu packmaxi #2|4|entertainments|9|Books|86|callyeing|N/A|8902365424seashell53|magenta|Oz|Unknown|21|oughteingeingationought| +17882|AAAAAAAAKNFEAAAA|1997-10-27|2000-10-26|Main eyes learn exhibitio|4.26|1.83|3003001|exportiexporti #1|3|toddlers|3|Children|284|eseeingable|small|97violet321767130033|olive|Ounce|Unknown|22|ableeingeingationought| +17883|AAAAAAAAKNFEAAAA|2000-10-27||Both pretty characteristics freeze. Women spend upon a developments. Books will not intend properly working papers. Rare figures|4.73|1.83|3003001|importoexporti #2|2|infants|3|Children|284|eseeingable|small|70spring962709944588|dark|Ton|Unknown|60|prieingeingationought| +17884|AAAAAAAAMNFEAAAA|1997-10-27|1999-10-27|Only public results become by a days; concerned, dead sales lose confidently from a ar|87.43|49.83|7002005|importobrand #5|2|bedding|7|Home|375|antiationpri|N/A|1pink018362153675350|magenta|Bunch|Unknown|38|eseeingeingationought| +17885|AAAAAAAAMNFEAAAA|1999-10-28|2001-10-26|Bad contributions may pull others; new rights shall imagine particular, nice members. Concerned communities might not bring serious, political details. Sure obligations o|9.59|5.27|7002005|edu packamalg #2|2|swimwear|1|Women|752|antiationpri|extra large|1pink018362153675350|saddle|Oz|Unknown|5|antieingeingationought| +17886|AAAAAAAAMNFEAAAA|2001-10-27||Bad contributions may pull others; new rights shall imagine particular, nice members. Concerned communities might not bring serious, political details. Sure obligations o|6.88|5.27|7002005|importoscholar #1|2|country|5|Music|545|antieseanti|N/A|374913254909pink4503|ghost|Each|Unknown|25|callyeingeingationought| +17887|AAAAAAAAPNFEAAAA|1997-10-27||Wooden, civil fingers keep great, possible scales. Police begin ago in common responsible times. Further open fathers can believe aga|0.33|0.29|9011008|amalgunivamalg #8|11|cooking|9|Books|94|esen st|N/A|yellow58212992535573|turquoise|Case|Unknown|44|ationeingeingationought| +17888|AAAAAAAAAOFEAAAA|1997-10-27|2000-10-26|Harder huge obligations shall provide yesterday sudden wrong years. Dead, actual applications close tools. As male clu|2.78|2.33|2003001|exportiimporto #1|3|pants|2|Men|258|eingantiable|medium|moccasin550177535253|royal|Unknown|Unknown|36|eingeingeingationought| +17889|AAAAAAAAAOFEAAAA|2000-10-27||Empty, private men might not match predictably internal steps. Rate|4.16|2.33|7007010|brandbrand #10|3|decor|7|Home|417|ationoughtese|N/A|90461941279184pale84|steel|Dozen|Unknown|29|n steingeingationought| +17890|AAAAAAAACOFEAAAA|1997-10-27|1999-10-27|Finally corporate terms may throw different, normal authorities. Women might act a litt|0.68|0.25|6009003|maxicorp #3|9|womens watch|6|Jewelry|743|prieseation|N/A|39249sandy9223643376|purple|Tbl|Unknown|27|barn steingationought| +17891|AAAAAAAACOFEAAAA|1999-10-28|2001-10-26|Black resources light ahead mutual, only ways. Good, japanese rights may interr|2.10|0.84|6014004|edu packbrand #4|14|estate|6|Jewelry|445|prieseation|N/A|39249sandy9223643376|puff|Carton|Unknown|100|oughtn steingationought| +17892|AAAAAAAACOFEAAAA|2001-10-27||Great, new minutes must say short main|1.95|0.84|2003001|exportiimporto #1|3|pants|2|Men|445|antieseese|extra large|2599steel31274634674|pale|Dram|Unknown|47|ablen steingationought| +17893|AAAAAAAAFOFEAAAA|1997-10-27||Symbolic cells would generate branches. Relations might find potentially; central, loyal |7.39|3.69|7002004|importobrand #4|2|bedding|7|Home|636|callyprically|N/A|704881thistle1373942|sandy|N/A|Unknown|46|prin steingationought| +17894|AAAAAAAAGOFEAAAA|1997-10-27|2000-10-26|Significant, capable students build. Senses could not use really fortunate lights. Together able centuries must not play so. Questions would find other, large numbers. A|1.19|0.66|5004001|edu packscholar #1|4|classical|5|Music|254|eseantiable|N/A|74292983pink44302058|wheat|Oz|Unknown|66|esen steingationought| +17895|AAAAAAAAGOFEAAAA|2000-10-27||Significant, capable students build. Senses could not use really fortunate lights. Together able centuries must not play so. Questions would find other, large numbers. A|0.37|0.18|5004001|amalgnameless #6|4|accent|7|Home|254|eseantiable|N/A|74292983pink44302058|rosy|Dram|Unknown|10|antin steingationought| +17896|AAAAAAAAIOFEAAAA|1997-10-27|1999-10-27|Clear, italian shelves light even certain, civil teams. Most major areas ought to identify apart main bright adults; high large clients will like often catholic agencies. Darling, unusual b|3.24|1.87|1003001|exportiamalg #1|3|maternity|1|Women|50|baranti|small|41821883200523steel2|powder|Cup|Unknown|79|callyn steingationought| +17897|AAAAAAAAIOFEAAAA|1999-10-28|2001-10-26|Prime, social parents join in full women. Demands stand slowly important laws. Also old members say outside a words. Towns help with a policies. Best ordinary seeds seem |1.81|1.52|4002002|importoedu pack #2|2|mens|4|Shoes|50|baranti|small|41821883200523steel2|beige|Unknown|Unknown|19|ationn steingationought| +17898|AAAAAAAAIOFEAAAA|2001-10-27||Prime, social parents join in full women. Demands stand slowly important laws. Also old members say outside a words. Towns help with a policies. Best ordinary seeds seem |0.98|0.61|4002002|importoscholar #1|2|country|5|Music|50|baranti|N/A|41821883200523steel2|chiffon|Pallet|Unknown|41|eingn steingationought| +17899|AAAAAAAALOFEAAAA|1997-10-27||Clear circumstances know then further white companies. Typical budgets take both required children. Appeals must not make civil, financial representatives. Emotional areas shall wear only. Ef|4.31|3.74|5003002|exportischolar #2|3|pop|5|Music|781|oughteingation|N/A|22853074487313lime36|sky|Case|Unknown|20|n stn steingationought| +17900|AAAAAAAAMOFEAAAA|1997-10-27|2000-10-26|Small, available others pay savings. Never green lights like today. New, warm details can support in a patients. Keen seasons make large personnel. Big, scientific signs greet rocks. Main, available|0.10|0.08|7002001|importobrand #1|2|bedding|7|Home|34|esepri|N/A|567134854turquoise80|plum|Tsp|Unknown|50|barbarn stationought| +17901|AAAAAAAAMOFEAAAA|2000-10-27||Small, available others pay savings. Never green lights like today. New, warm details can support in a patients. Keen seasons make large personnel. Big, scientific signs greet rocks. Main, available|9.45|4.91|9008010|namelessmaxi #10|8|romance|9|Books|593|prin stanti|N/A|428929plum4821590343|mint|Box|Unknown|50|oughtbarn stationought| +17902|AAAAAAAAOOFEAAAA|1997-10-27|1999-10-27|Things question genuine, responsible talks. Strong days retire later busy, famous rights; then easy ties must pour again still curious women. Final others secure a|1.18|0.61|8005003|scholarnameless #3|5|fitness|8|Sports|129|n stableought|N/A|1433788029spring3252|powder|Tsp|Unknown|38|ablebarn stationought| +17903|AAAAAAAAOOFEAAAA|1999-10-28|2001-10-26|Major issue|29.17|0.61|8005003|univmaxi #4|5|pools|8|Sports|28|n stableought|N/A|1433788029spring3252|forest|Tbl|Unknown|51|pribarn stationought| +17904|AAAAAAAAOOFEAAAA|2001-10-27||Main tools mus|76.21|41.15|8005003|amalgscholar #1|1|rock|5|Music|714|eseoughtation|N/A|00732moccasin6634911|turquoise|N/A|Unknown|1|esebarn stationought| +17905|AAAAAAAABPFEAAAA|1997-10-27||Serious, big changes might find populations; leaders make helplessly on a policies; great, likely departments try somehow changes; very right bags pretend new, central villages. No longer |2.64|1.84|9005002|scholarmaxi #2|5|history|9|Books|289|n steingable|N/A|622358puff7726471182|red|Carton|Unknown|40|antibarn stationought| +17906|AAAAAAAACPFEAAAA|1997-10-27|2000-10-26|Alleged, significant powers correct especially fair long journalists. Right, so-called |3.40|2.34|5003001|exportischolar #1|3|pop|5|Music|176|callyationought|N/A|50462675tan115999382|chocolate|Pound|Unknown|27|callybarn stationought| +17907|AAAAAAAACPFEAAAA|2000-10-27||Lawyers wait internal patients. Total sciences should deserve for example in the votes. P|9.35|3.45|5003001|importoscholar #2|3|country|5|Music|388|callyationought|N/A|50462675tan115999382|lemon|Pallet|Unknown|21|ationbarn stationought| +17908|AAAAAAAAEPFEAAAA|1997-10-27|1999-10-27|Sufficient things must not know apparent, wonderful ministers. Personally traditional figures move straight; essential, extensive disputes wish o|0.71|0.34|4003001|exportiedu pack #1|3|kids|4|Shoes|227|ationableable|extra large|3776652882orange8980|tomato|Gram|Unknown|9|eingbarn stationought| +17909|AAAAAAAAEPFEAAAA|1999-10-28|2001-10-26|Commercial crews will not say potentially large, political heroes; average women allow grey, clean feet. Dirty feelings bring however. Wrong, similar teachers |7.37|0.34|4003001|amalgedu pack #2|1|womens|4|Shoes|7|ation|medium|3776652882orange8980|slate|Ounce|Unknown|4|n stbarn stationought| +17910|AAAAAAAAEPFEAAAA|2001-10-27||Then final minutes start just more new animals. Parts|5.32|3.13|4003001|importobrand #1|1|bedding|7|Home|7|ation|N/A|480peach689597144028|dark|Pallet|Unknown|66|baroughtn stationought| +17911|AAAAAAAAHPFEAAAA|1997-10-27||Dark times play between a variations. Years would explain very positive reasons. |16.82|8.41|7010002|univnameless #2|10|flatware|7|Home|391|oughtn stpri|N/A|310880seashell740321|slate|Cup|Unknown|42|oughtoughtn stationought| +17912|AAAAAAAAIPFEAAAA|1997-10-27|2000-10-26|For example usual calculations lend very movies. Talks cannot avoid parallel by a months. Closely terrible years will not go marginal, other parents. Authors obtain american, wrong delegates. P|1.46|0.91|6016001|corpbrand #1|16|consignment|6|Jewelry|218|eingoughtable|N/A|19032sienna018150939|sky|Ton|Unknown|21|ableoughtn stationought| +17913|AAAAAAAAIPFEAAAA|2000-10-27||For example usual calculations lend very movies. Talks cannot avoid parallel by a months. Closely terrible years will not go marginal, other parents. Authors obtain american, wrong delegates. P|5.06|0.91|6016001|importoscholar #2|16|country|5|Music|1|eingoughtable|N/A|0378254150sky0223831|snow|Ton|Unknown|17|prioughtn stationought| +17914|AAAAAAAAKPFEAAAA|1997-10-27|1999-10-27|Recently right|4.52|1.94|2004001|edu packimporto #1|4|sports-apparel|2|Men|434|esepriese|petite|454141733095saddle42|burnished|Carton|Unknown|50|eseoughtn stationought| +17915|AAAAAAAAKPFEAAAA|1999-10-28|2001-10-26|Recently right|2.22|1.94|1001002|amalgamalg #2|4|dresses|1|Women|291|esepriese|large|454141733095saddle42|pale|Unknown|Unknown|37|antioughtn stationought| +17916|AAAAAAAAKPFEAAAA|2001-10-27||Wron|2.93|1.94|4004001|edu packedu pack #1|4|athletic|4|Shoes|291|esepriese|extra large|15104655sky928701219|wheat|Bundle|Unknown|8|callyoughtn stationought| +17917|AAAAAAAANPFEAAAA|1997-10-27||Arrangements should prohibit preferably at a men. Just public flowers should ca|2.16|1.36|6015006|scholarbrand #6|15|custom|6|Jewelry|228|eingableable|N/A|676272337plum4429993|mint|Pound|Unknown|5|ationoughtn stationought| +17918|AAAAAAAAOPFEAAAA|1997-10-27|2000-10-26|Heavy, recent decades think finally. Outstanding, average det|3.45|2.10|7001009|amalgbrand #9|1|bathroom|7|Home|75|antiation|N/A|gainsboro61341337403|green|Box|Unknown|30|eingoughtn stationought| +17919|AAAAAAAAOPFEAAAA|2000-10-27||Heavy, recent decades think finally. Outstanding, average det|4.92|2.10|7001009|exportiedu pack #2|1|kids|4|Shoes|75|antiation|large|4907794256papaya5163|magenta|N/A|Unknown|30|n stoughtn stationought| +17920|AAAAAAAAAAGEAAAA|1997-10-27|1999-10-27|Away high rules might not present better circumstances; black, magic details believe |58.70|29.93|3003001|exportiexporti #1|3|toddlers|3|Children|353|priantipri|medium|58870900689hot299368|slate|Oz|Unknown|29|barablen stationought| +17921|AAAAAAAAAAGEAAAA|1999-10-28|2001-10-26|Away high rules might not present better circumstances; black, magic details believe |0.40|0.17|3003001|importobrand #2|2|bedding|7|Home|353|priantipri|N/A|58870900689hot299368|peru|Pound|Unknown|9|oughtablen stationought| +17922|AAAAAAAAAAGEAAAA|2001-10-27||Commercial errors wish perfectly; available, equal names explain awkwar|4.51|2.88|3003001|importoexporti #1|2|infants|3|Children|190|barn stought|large|011honeydew874135161|sienna|Bundle|Unknown|41|ableablen stationought| +17923|AAAAAAAADAGEAAAA|1997-10-27||Environmental businesses behave settlements. Students might make of course almost organisational goals. Eyes brush on|7.79|4.12|9006008|corpmaxi #8|6|parenting|9|Books|185|antieingought|N/A|0tan4235726675760330|smoke|Lb|Unknown|6|priablen stationought| +17924|AAAAAAAAEAGEAAAA|1997-10-27|2000-10-26|Elaborate rules would want about thoughts; fresh, cheap parents should appe|1.65|0.74|1002001|importoamalg #1|2|fragrances|1|Women|264|esecallyable|small|94dodger479052263425|papaya|Bundle|Unknown|57|eseablen stationought| +17925|AAAAAAAAEAGEAAAA|2000-10-27||Opportunities open quic|6.33|0.74|1002001|amalgimporto #2|2|accessories|2|Men|264|esecallyable|extra large|94dodger479052263425|turquoise|Gross|Unknown|6|antiablen stationought| +17926|AAAAAAAAGAGEAAAA|1997-10-27|1999-10-27|Elements contribute long funny proportions. Late, upper issues go as well; medium, increased problems stay very public mountains; com|1.58|1.23|5001001|amalgscholar #1|1|rock|5|Music|949|n stesen st|N/A|27499265561indian109|peach|Ton|Unknown|12|callyablen stationought| +17927|AAAAAAAAGAGEAAAA|1999-10-28|2001-10-26|Elements contribute long funny proportions. Late, upper issues go as well; medium, increased problems stay very public mountains; com|90.43|48.83|9014012|edu packunivamalg #12|14|sports|9|Books|949|n stesen st|N/A|354slate460849474568|lime|Gram|Unknown|54|ationablen stationought| +17928|AAAAAAAAGAGEAAAA|2001-10-27||Others must take away. Nice visitors charge particularly. Different, fun sharehol|4.80|48.83|4002001|importoedu pack #1|14|mens|4|Shoes|741|n stesen st|petite|354slate460849474568|orange|Case|Unknown|43|eingablen stationought| +17929|AAAAAAAAJAGEAAAA|1997-10-27||Responsible days used to lose afterwards british, considerable standard|5.10|1.93|10004012|edu packunivamalg #12|4|audio|10|Electronics|598|eingn stanti|N/A|5395961571seashell72|steel|Gram|Unknown|54|n stablen stationought| +17930|AAAAAAAAKAGEAAAA|1997-10-27|2000-10-26|Alr|4.14|3.43|10001013|amalgunivamalg #13|1|cameras|10|Electronics|402|ablebarese|N/A|8456107330steel61251|turquoise|Dram|Unknown|75|barprin stationought| +17931|AAAAAAAAKAGEAAAA|2000-10-27||Black controls ride. Civil, precious lips may not supervise t|1.16|3.43|1004002|edu packamalg #2|1|swimwear|1|Women|332|ablepripri|large|8456107330steel61251|yellow|Pound|Unknown|25|oughtprin stationought| +17932|AAAAAAAAMAGEAAAA|1997-10-27|1999-10-27|Rapidly gold techniques tell nevertheless mutual friends. Racial, sophisticated objectives play. Seldom physical images can interpret experien|2.23|1.58|4001001|amalgedu pack #1|1|womens|4|Shoes|213|prioughtable|medium|34863698547navy13631|navy|Bunch|Unknown|54|ableprin stationought| +17933|AAAAAAAAMAGEAAAA|1999-10-28|2001-10-26|Rapidly gold techniques tell nevertheless mutual friends. Racial, sophisticated objectives play. Seldom physical images can interpret experien|2.45|1.58|3003002|exportiexporti #2|1|toddlers|3|Children|659|n stantically|petite|34863698547navy13631|pink|Pound|Unknown|62|priprin stationought| +17934|AAAAAAAAMAGEAAAA|2001-10-27||Rapidly gold techniques tell nevertheless mutual friends. Racial, sophisticated objectives play. Seldom physical images can interpret experien|3.02|1.58|10012017|importoamalgamalg #17|12|monitors|10|Electronics|659|n stantically|N/A|1585479209powder6755|thistle|Tbl|Unknown|3|eseprin stationought| +17935|AAAAAAAAPAGEAAAA|1997-10-27||Anyway hard actors ought to transport often accurate significant limits. Others should try. Only italian words will not make fresh officers; quickly correct operations could recognise just |1.61|0.80|7014006|edu packnameless #6|14|glassware|7|Home|995|antin stn st|N/A|44223944smoke6970094|rosy|Pallet|Unknown|32|antiprin stationought| +17936|AAAAAAAAABGEAAAA|1997-10-27|2000-10-26|Confident areas would happen without a arguments. Soft mountains allow moderately contempora|3.23|2.42|8012007|importomaxi #7|12|guns|8|Sports|23|priable|N/A|77351470548sienna600|purple|Dram|Unknown|27|callyprin stationought| +17937|AAAAAAAAABGEAAAA|2000-10-27||Only trees create easier earlier annual players. Physical, available countries come very hard evident applications; both african issues may rely m|5.52|2.42|8012007|edu packimporto #2|12|sports-apparel|2|Men|692|ablen stcally|medium|77351470548sienna600|linen|Bunch|Unknown|19|ationprin stationought| +17938|AAAAAAAACBGEAAAA|1997-10-27|1999-10-27|Royal borders get always french statements. As young lights shall see little, key efforts. Social, interested words shall not go difficult, other compan|1.09|0.46|3002001|importoexporti #1|2|infants|3|Children|176|callyationought|extra large|2338779medium6429099|smoke|Ton|Unknown|8|eingprin stationought| +17939|AAAAAAAACBGEAAAA|1999-10-28|2001-10-26|Professional ideas claim most for a hands. Only, military days permit much targets. G|0.62|0.46|5003002|exportischolar #2|3|pop|5|Music|176|callyationought|N/A|2338779medium6429099|linen|Carton|Unknown|62|n stprin stationought| +17940|AAAAAAAACBGEAAAA|2001-10-27||New, other m|88.81|63.05|4002001|importoedu pack #1|3|mens|4|Shoes|176|callyationought|petite|619675442880green929|saddle|Ton|Unknown|28|baresen stationought| +17941|AAAAAAAAFBGEAAAA|1997-10-27||Thoughts replace again systems. Small methods tackle over a purposes. English solicitors must run explic|6.21|3.85|4003002|exportiedu pack #2|3|kids|4|Shoes|117|ationoughtought|medium|5misty66757806173413|purple|N/A|Unknown|8|oughtesen stationought| +17942|AAAAAAAAGBGEAAAA|1997-10-27|2000-10-26|There minimum others l|58.24|36.69|10012008|importoamalgamalg #8|12|monitors|10|Electronics|68|eingcally|N/A|49253red508005810790|antique|Bunch|Unknown|41|ableesen stationought| +17943|AAAAAAAAGBGEAAAA|2000-10-27||There minimum others l|2.45|0.75|7003004|exportibrand #4|12|kids|7|Home|60|barcally|N/A|49253red508005810790|saddle|Bundle|Unknown|23|priesen stationought| +17944|AAAAAAAAIBGEAAAA|1997-10-27|1999-10-27|Frequently en|4.10|2.33|6003001|exporticorp #1|3|gold|6|Jewelry|607|ationbarcally|N/A|8honeydew00810991407|medium|Lb|Unknown|57|eseesen stationought| +17945|AAAAAAAAIBGEAAAA|1999-10-28|2001-10-26|Frequently en|1.13|0.54|6003001|importoamalg #2|2|fragrances|1|Women|317|ationoughtpri|small|9893papaya8312584001|plum|Each|Unknown|23|antiesen stationought| +17946|AAAAAAAAIBGEAAAA|2001-10-27||Frequently en|1.27|1.06|5001001|amalgscholar #1|1|rock|5|Music|721|ationoughtpri|N/A|472158600007sandy285|chiffon|Tbl|Unknown|7|callyesen stationought| +17947|AAAAAAAALBGEAAAA|1997-10-27||Services put usual, unemployed persons. Desperate, normal functions think at all bl|39.93|13.17|9014008|edu packunivamalg #8|14|sports|9|Books|239|n stpriable|N/A|20tomato167876322323|goldenrod|Gram|Unknown|13|ationesen stationought| +17948|AAAAAAAAMBGEAAAA|1997-10-27|2000-10-26|Now fixed arms could avert ago minutes. Lads rely also enthusiastic expenses. At least obvious birds go once again. Times produ|54.79|35.06|8005009|scholarnameless #9|5|fitness|8|Sports|83|prieing|N/A|thistle0579841255355|wheat|Gram|Unknown|29|eingesen stationought| +17949|AAAAAAAAMBGEAAAA|2000-10-27||Now fixed arms could avert ago minutes. Lads rely also enthusiastic expenses. At least obvious birds go once again. Times produ|6.45|35.06|4002002|importoedu pack #2|5|mens|4|Shoes|83|prieing|medium|thistle0579841255355|azure|Dram|Unknown|85|n stesen stationought| +17950|AAAAAAAAOBGEAAAA|1997-10-27|1999-10-27|National|3.95|1.50|2003001|exportiimporto #1|3|pants|2|Men|613|prioughtcally|medium|4659797025spring5549|yellow|Dozen|Unknown|95|barantin stationought| +17951|AAAAAAAAOBGEAAAA|1999-10-28|2001-10-26|National|5.01|1.50|2003001|importoexporti #2|2|infants|3|Children|965|prioughtcally|medium|39181snow33349240374|linen|Pound|Unknown|27|oughtantin stationought| +17952|AAAAAAAAOBGEAAAA|2001-10-27||Secure, prospective computers turn national men. Again sorry studies will find on a pupils. Unlikely amounts used to mean always drugs. British articles|1.73|1.50|10013001|exportiamalgamalg #1|13|stereo|10|Electronics|204|esebarable|N/A|ivory727627906581087|red|Pallet|Unknown|8|ableantin stationought| +17953|AAAAAAAABCGEAAAA|1997-10-27||Necessary|0.15|0.04|6016002|corpbrand #2|16|consignment|6|Jewelry|541|oughteseanti|N/A|7592410733seashell83|salmon|Each|Unknown|33|priantin stationought| +17954|AAAAAAAACCGEAAAA|1997-10-27|2000-10-26||8.06||9016003|corpunivamalg #3|16||||237|ationpriable|N/A|||Carton|||eseantin stationought| +17955|AAAAAAAACCGEAAAA|2000-10-27||Other frames will pop now but|7.16|2.57|6005004|scholarcorp #4|16|earings|6|Jewelry|237|ationpriable|N/A|960563383136tan38902|thistle|Tsp|Unknown|16|antiantin stationought| +17956|AAAAAAAAECGEAAAA|1997-10-27|1999-10-27|Otherwise beautiful courts might say so more wide flames. Particular doors might find even legitimate times; more white times discourage approx|4.24|2.24|9014005|edu packunivamalg #5|14|sports|9|Books|225|antiableable|N/A|4115969286violet1626|midnight|N/A|Unknown|47|callyantin stationought| +17957|AAAAAAAAECGEAAAA|1999-10-28|2001-10-26|Able newspapers must not reject a little completely little equations. Needs may|0.09|2.24|1003002|exportiamalg #2|3|maternity|1|Women|3|pri|large|7186papaya1878726470|white|Bundle|Unknown|9|ationantin stationought| +17958|AAAAAAAAECGEAAAA|2001-10-27||Able newspapers must not reject a little completely little equations. Needs may|8.74|6.38|1003002|exportiimporto #1|3|pants|2|Men|894|esen steing|extra large|3red8471938173621610|white|Ounce|Unknown|48|eingantin stationought| +17959|AAAAAAAAHCGEAAAA|1997-10-27||Figures impose easy, distinct terms. Apparent calls kiss currently from a sites. Then recent roots can feel content players. |3.12|2.02|4002002|importoedu pack #2|2|mens|4|Shoes|27|ationable|large|9slate18685384638419|salmon|N/A|Unknown|20|n stantin stationought| +17960|AAAAAAAAICGEAAAA|1997-10-27|2000-10-26|Key women may not complete great relations. Italian, important friends would not go forward giant cases. Strange pl|9.05|3.62|10001009|amalgunivamalg #9|1|cameras|10|Electronics|414|eseoughtese|N/A|288348029aquamarine4|lace|Gross|Unknown|8|barcallyn stationought| +17961|AAAAAAAAICGEAAAA|2000-10-27||Key women may not complete great relations. Italian, important friends would not go forward giant cases. Strange pl|58.55|32.78|2002002|importoimporto #2|2|shirts|2|Men|633|eseoughtese|large|288348029aquamarine4|olive|Pallet|Unknown|14|oughtcallyn stationought| +17962|AAAAAAAAKCGEAAAA|1997-10-27|1999-10-27|Full rules may persuade pregnant cars. Earnings publish worried symptoms. Ready|5.88|4.99|2001001|amalgimporto #1|1|accessories|2|Men|905|antibarn st|medium|78106333289247puff43|olive|Dozen|Unknown|26|ablecallyn stationought| +17963|AAAAAAAAKCGEAAAA|1999-10-28|2001-10-26|Full rules may persuade pregnant cars. Earnings publish worried symptoms. Ready|87.47|4.99|5001002|amalgscholar #2|1|rock|5|Music|905|antibarn st|N/A|5923436136salmon2604|wheat|Cup|Unknown|28|pricallyn stationought| +17964|AAAAAAAAKCGEAAAA|2001-10-27||Yet ric|5.22|1.72|5001002|amalgamalg #1|1|dresses|1|Women|232|ablepriable|medium|5923436136salmon2604|magenta|Gross|Unknown|44|esecallyn stationought| +17965|AAAAAAAANCGEAAAA|1997-10-27||Certain, distinct obligations wish. Buyers can start just circumstances. Events should thank for the places. Difficult agreements would need with the systems. Wome|0.42|0.36|8010006|univmaxi #6|10|pools|8|Sports|383|prieingpri|N/A|9wheat43640628004308|moccasin|Cup|Unknown|28|anticallyn stationought| +17966|AAAAAAAAOCGEAAAA|1997-10-27|2000-10-26|Ashamed, legal phenomena possess officers. Newly inappropriate players lead. Authorities quote children. Instrument|3.37|1.44|8003007|exportinameless #7|3|basketball|8|Sports|287|ationeingable|N/A|18137385336720linen4|salmon|Box|Unknown|21|callycallyn stationought| +17967|AAAAAAAAOCGEAAAA|2000-10-27||Ashamed, legal phenomena possess officers. Newly inappropriate players lead. Authorities quote children. Instrument|3.32|1.44|7015008|scholarnameless #8|3|tables|7|Home|287|ationeingable|N/A|02483pink88377639123|thistle|Tbl|Unknown|29|ationcallyn stationought| +17968|AAAAAAAAADGEAAAA|1997-10-27|1999-10-27|Little holy others need forward long days. Points should inform only british, silent appearances. Administrative services might not appear in full years. Babies gri|3.84|2.38|8011009|amalgmaxi #9|11|archery|8|Sports|231|oughtpriable|N/A|4420990751805tomato2|turquoise|Pound|Unknown|39|eingcallyn stationought| +17969|AAAAAAAAADGEAAAA|1999-10-28|2001-10-26|Wild|7.29|2.38|8011009|exportischolar #2|11|pop|5|Music|564|esecallyanti|N/A|44155567465violet936|puff|Unknown|Unknown|30|n stcallyn stationought| +17970|AAAAAAAAADGEAAAA|2001-10-27||Private patterns accept most trees. Full students prolong universal, internal risks. Internal, royal steps |4.36|2.38|8011009|importonameless #1|11|baseball|8|Sports|358|eingantipri|N/A|77156393214spring632|spring|Carton|Unknown|41|barationn stationought| +17971|AAAAAAAADDGEAAAA|1997-10-27||New photographs will review too once mysterious details. New wings may not go nearly specific child|0.66|0.31|8015002|scholarmaxi #2|15|fishing|8|Sports|160|barcallyought|N/A|660759steel646232569|papaya|Bunch|Unknown|77|oughtationn stationought| +17972|AAAAAAAAEDGEAAAA|1997-10-27|2000-10-26|Never japanese miners put afraid rates; requirements must not arise seriously there double comments. Free years will not identify in order prime winners; services used to displace today o|1.72|1.16|7005003|scholarbrand #3|5|blinds/shades|7|Home|743|prieseation|N/A|9685851167493purple1|plum|Case|Unknown|12|ableationn stationought| +17973|AAAAAAAAEDGEAAAA|2000-10-27||Dependent, organisational hands smell|1.31|0.77|4002002|importoedu pack #2|2|mens|4|Shoes|743|prieseation|medium|4spring6707851465747|snow|Dozen|Unknown|55|priationn stationought| +17974|AAAAAAAAGDGEAAAA|1997-10-27|1999-10-27|Top libraries make well for the problems. Vague papers install immensely from a talks. Often aware children should allow more in a problems. |9.89|4.74|7011005|amalgnameless #5|11|accent|7|Home|229|n stableable|N/A|6211571wheat57313313|frosted|Unknown|Unknown|11|eseationn stationought| +17975|AAAAAAAAGDGEAAAA|1999-10-28|2001-10-26|Top libraries make well for the problems. Vague papers install immensely from a talks. Often aware children should allow more in a problems. |85.93|44.68|7011005|importoexporti #2|11|infants|3|Children|538|eingprianti|large|1honeydew45442608280|sky|Gram|Unknown|24|antiationn stationought| +17976|AAAAAAAAGDGEAAAA|2001-10-27||Top libraries make well for the problems. Vague papers install immensely from a talks. Often aware children should allow more in a problems. |4.41|44.68|7011005|brandnameless #7|7|hockey|8|Sports|429|eingprianti|N/A|4078628672navy996574|honeydew|Carton|Unknown|65|callyationn stationought| +17977|AAAAAAAAJDGEAAAA|1997-10-27||Almost senior numbers must pay sure, human taxes. Ideas use; flexi|3.29|2.36|6016002|corpbrand #2|16|consignment|6|Jewelry|522|ableableanti|N/A|54632831546metallic6|dim|Case|Unknown|27|ationationn stationought| +17978|AAAAAAAAKDGEAAAA|1997-10-27|2000-10-26|Basic foundations would not shoot rarely common cells; generations shall not arise right users; different men must return so industrial contacts. Feel|0.57|0.33|1003001|exportiamalg #1|3|maternity|1|Women|287|ationeingable|medium|3506khaki36515968801|seashell|Case|Unknown|29|eingationn stationought| +17979|AAAAAAAAKDGEAAAA|2000-10-27||Basic foundations would not shoot rarely common cells; generations shall not arise right users; different men must return so industrial contacts. Feel|1.52|0.33|6004004|edu packcorp #4|4|bracelets|6|Jewelry|91|ationeingable|N/A|657metallic457501756|pale|Ounce|Unknown|2|n stationn stationought| +17980|AAAAAAAAMDGEAAAA|1997-10-27|1999-10-27|Easily |1.26|0.71|5004001|edu packscholar #1|4|classical|5|Music|100|barbarought|N/A|174751papaya77633764|rose|Dozen|Unknown|6|bareingn stationought| +17981|AAAAAAAAMDGEAAAA|1999-10-28|2001-10-26|Relations might not help also. Obligations used to list|2.25|1.41|5004001|amalgscholar #2|4|rock|5|Music|498|barbarought|N/A|55797tan159876531612|grey|Dram|Unknown|30|oughteingn stationought| +17982|AAAAAAAAMDGEAAAA|2001-10-27||New, only regions should replace minor events; technical quarters would not|9.29|1.41|10009014|maxiunivamalg #14|4|televisions|10|Electronics|397|barbarought|N/A|588480cyan5107461870|steel|Pound|Unknown|15|ableeingn stationought| +17983|AAAAAAAAPDGEAAAA|1997-10-27||African lives must n|0.88|0.37|9011008|amalgunivamalg #8|11|cooking|9|Books|166|callycallyought|N/A|795819336769plum4701|seashell|Tsp|Unknown|31|prieingn stationought| +17984|AAAAAAAAAEGEAAAA|1997-10-27|2000-10-26|There deliberate christians may avoid ve|3.40|1.97|8005005|scholarnameless #5|5|fitness|8|Sports|531|oughtprianti|N/A|93014727200wheat7984|orange|Carton|Unknown|4|eseeingn stationought| +17985|AAAAAAAAAEGEAAAA|2000-10-27||Happy, like|4.91|1.97|8005005|univbrand #2|10|jewelry boxes|6|Jewelry|531|oughtprianti|N/A|93014727200wheat7984|seashell|N/A|Unknown|40|antieingn stationought| +17986|AAAAAAAACEGEAAAA|1997-10-27|1999-10-27|Years visit. Free, real concepts take arrangements. Also vital bodies cannot open precisely future good schools. Legal businesses|4.32|2.63|6016003|corpbrand #3|16|consignment|6|Jewelry|72|ableation|N/A|49887576849white3568|rosy|Dozen|Unknown|96|callyeingn stationought| +17987|AAAAAAAACEGEAAAA|1999-10-28|2001-10-26|Years visit. Free, real concepts take arrangements. Also vital bodies cannot open precisely future good schools. Legal businesses|6.16|3.51|7008008|namelessbrand #8|16|lighting|7|Home|755|ableation|N/A|51807823322peach8938|pale|Pallet|Unknown|6|ationeingn stationought| +17988|AAAAAAAACEGEAAAA|2001-10-27||Years visit. Free, real concepts take arrangements. Also vital bodies cannot open precisely future good schools. Legal businesses|0.25|0.16|3003001|exportiexporti #1|3|toddlers|3|Children|755|antiantiation|medium|51807823322peach8938|spring|Gross|Unknown|12|eingeingn stationought| +17989|AAAAAAAAFEGEAAAA|1997-10-27||Specific, sad clubs shall achieve too widespread you|2.10|0.79|2002002|importoimporto #2|2|shirts|2|Men|121|oughtableought|N/A|4161spring2800701054|magenta|Dozen|Unknown|48|n steingn stationought| +17990|AAAAAAAAGEGEAAAA|1997-10-27|2000-10-26|Late loans should ensure unable, beneficial films; just distinctive authoritie|6.18|2.03|6014007|edu packbrand #7|14|estate|6|Jewelry|199|n stn stought|N/A|09528thistle77389537|navajo|Oz|Unknown|26|barn stn stationought| +17991|AAAAAAAAGEGEAAAA|2000-10-27||Late loans should ensure unable, beneficial films; just distinctive authoritie|6.73|2.82|6014007|importoexporti #2|14|infants|3|Children|649|n stn stought|petite|09528thistle77389537|sienna|Bundle|Unknown|65|oughtn stn stationought| +17992|AAAAAAAAIEGEAAAA|1997-10-27|1999-10-27|Also black forms accompany bett|4.53|2.21|4002001|importoedu pack #1|2|mens|4|Shoes|356|callyantipri|medium|45432steel7733171743|puff|Ounce|Unknown|5|ablen stn stationought| +17993|AAAAAAAAIEGEAAAA|1999-10-28|2001-10-26|Also black forms accompany bett|0.45|0.14|4002001|scholarnameless #4|5|fitness|8|Sports|515|callyantipri|N/A|45432steel7733171743|lime|Cup|Unknown|35|prin stn stationought| +17994|AAAAAAAAIEGEAAAA|2001-10-27||Weeks assess then. Neither mental differences establish elsewhere. Cards should thank however nice, electoral authorities. |31.95|0.14|4002001|importoedu pack #1|5|mens|4|Shoes|515|antioughtanti|extra large|921811133puff7191308|white|Box|Unknown|22|esen stn stationought| +17995|AAAAAAAALEGEAAAA|1997-10-27||Full, alone |67.63|22.99|6003004|exporticorp #4|3|gold|6|Jewelry|460|barcallyese|N/A|472520turquoise69053|yellow|Dozen|Unknown|24|antin stn stationought| +17996|AAAAAAAAMEGEAAAA|1997-10-27|2000-10-26|Ag|2.85|1.31|8004007|edu packnameless #7|4|camping|8|Sports|225|antiableable|N/A|8642352cornflower835|firebrick|Dozen|Unknown|18|callyn stn stationought| +17997|AAAAAAAAMEGEAAAA|2000-10-27||Present, international minutes supply well generations. Cases cure never between the players. Harsh, international mothers create actually especially sexual scenes. So local terms may|1.03|1.31|8004007|importoamalg #2|4|fragrances|1|Women|629|n stablecally|petite|8642352cornflower835|puff|Bunch|Unknown|61|ationn stn stationought| +17998|AAAAAAAAOEGEAAAA|1997-10-27|1999-10-27|Modern, good friends blow. Southern processes conceal maybe by the foreigners. Hands see already unions. Men cannot feel prices. White, average passages should not provide maybe services. Ot|1.37|0.79|4004001|edu packedu pack #1|4|athletic|4|Shoes|172|ableationought|large|2164951946papaya0330|sienna|Ounce|Unknown|20|eingn stn stationought| +17999|AAAAAAAAOEGEAAAA|1999-10-28|2001-10-26|Modern, good friends blow. Southern processes conceal maybe by the foreigners. Hands see already unions. Men cannot feel prices. White, average passages should not provide maybe services. Ot|0.55|0.26|4004001|amalgmaxi #10|11|archery|8|Sports|191|oughtn stought|N/A|2164951946papaya0330|red|Ton|Unknown|32|n stn stn stationought| +18000|AAAAAAAAOEGEAAAA|2001-10-27||Both possible lines date however in a words. Over cognitive heads follow under. Principles must illustrate so strong benefits. Dual children will land also strong, medical years. Labour, whi|4.06|2.19|8001001|amalgnameless #1|11|athletic shoes|8|Sports|191|oughtn stought|N/A|2164951946papaya0330|moccasin|Ton|Unknown|1|barbarbareingought| diff --git a/spark-connector/spark-common/src/test/resources/data/tpcds/promotion/promotion.dat b/spark-connector/spark-common/src/test/resources/data/tpcds/promotion/promotion.dat new file mode 100644 index 00000000000..9183d332e68 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/data/tpcds/promotion/promotion.dat @@ -0,0 +1,300 @@ +1|AAAAAAAABAAAAAAA|2450164|2450185|10022|1000.00|1|ought|Y|N|N|N|N|N|N|N|Men will not say merely. Old, available |Unknown|N| +2|AAAAAAAACAAAAAAA|2450118|2450150|2410|1000.00|1|able|Y|N|N|N|N|N|N|N|So willing buildings coul|Unknown|N| +3|AAAAAAAADAAAAAAA|2450675|2450712|10843|1000.00|1|pri|Y|N|N|N|N|N|N|N|Companies shall not pr|Unknown|N| +4|AAAAAAAAEAAAAAAA|2450633|2450646|9794|1000.00|1|ese|N|N|N|N|N|N|N|N|High, good shoulders can tell on a proble|Unknown|N| +5|AAAAAAAAFAAAAAAA|2450347|2450357|6655|1000.00|1|anti|N|N|N|N|N|N|N|N|Huge, competent structures can remember different pat|Unknown|N| +6|AAAAAAAAGAAAAAAA|2450516|2450561|9464|1000.00|1|cally|N|N|N|N|N|N|N|N|Boards might not reverse up to a hopes. Now high respon|Unknown|N| +7|AAAAAAAAHAAAAAAA|2450123|2450132|14527|1000.00|1|ation|N|N|N|N|N|N|N|N|Effects used to prefer however new terms. Usually ava|Unknown|N| +8|AAAAAAAAIAAAAAAA|2450350|2450409|304|1000.00|1|eing|Y|N|N|N|N|N|N|N|Offences feel only on a fees.|Unknown|N| +9|AAAAAAAAJAAAAAAA|2450192|2450248|3439|1000.00|1|n st|Y|N|N|N|N|N|N|N|External forces shall comprehend ideal, disciplinary stud|Unknown|N| +10|AAAAAAAAKAAAAAAA|2450324|2450365|3314|1000.00|1|bar|N|N|N|N|N|N|N|N|Only local achievements used to make t|Unknown|N| +11|AAAAAAAALAAAAAAA||||1000.00||ought|||||N||N|N|Teachers shall not make up to fair countries. Roya|Unknown|N| +12|AAAAAAAAMAAAAAAA|2450153|2450169|6688|1000.00|1|able|N|N|N|N|N|N|N|N|Also only times would come right to the obse|Unknown|N| +13|AAAAAAAANAAAAAAA|2450316|2450365|11354|1000.00|1|pri|N|N|N|N|N|N|N|N|Years shall not go later other, given citizens. Excellent, |Unknown|N| +14|AAAAAAAAOAAAAAAA|2450236|2450282|14480|1000.00|1|ese|N|N|N|N|N|N|N|N|Adults would not delay rarely manufacturing, certain|Unknown|N| +15|AAAAAAAAPAAAAAAA||||1000.00||anti|Y||N|N||N|N|N|Old elements would shake pre|Unknown|N| +16|AAAAAAAAABAAAAAA|2450342|2450351|11899|1000.00|1|cally|Y|N|N|N|N|N|N|N|Sudden, wooden theories will not choose|Unknown|N| +17|AAAAAAAABBAAAAAA|2450360|2450406|15529|1000.00|1|ation|Y|N|N|N|N|N|N|N|Almost old churches ought to occur narrow|Unknown|N| +18|AAAAAAAACBAAAAAA|2450581|2450592|8599|1000.00|1|eing|Y|N|N|N|N|N|N|N|National communities use rarely oth|Unknown|N| +19|AAAAAAAADBAAAAAA|2450623|2450640|5185|1000.00|1|n st|N|N|N|N|N|N|N|N|Young families act. Most full prizes avert eastern glasses|Unknown|N| +20|AAAAAAAAEBAAAAAA|2450531|2450545|17860|1000.00|1|bar|Y|N|N|N|N|N|N|N|Usually common courses fit also leading|Unknown|N| +21|AAAAAAAAFBAAAAAA|2450895|2450904|3632|1000.00|1|ought|N|N|N|N|N|N|N|N|British tensions should not make virtually financia|Unknown|N| +22|AAAAAAAAGBAAAAAA|2450596|2450629|8044|1000.00|1|able|Y|N|N|N|N|N|N|N|Powers ought to allow to a c|Unknown|N| +23|AAAAAAAAHBAAAAAA|2450478|2450493|4321|1000.00|1|pri|N|N|N|N|N|N|N|N|As main privileges cannot seize however double games|Unknown|N| +24|AAAAAAAAIBAAAAAA|2450719|2450773|12004|1000.00|1|ese|Y|N|N|N|N|N|N|N|Words appear technical, casual f|Unknown|N| +25|AAAAAAAAJBAAAAAA|2450757|2450780|16328|1000.00|1|anti|Y|N|N|N|N|N|N|N|Scenes might make high |Unknown|N| +26|AAAAAAAAKBAAAAAA|2450528|2450545|8600|1000.00|1|cally|N|N|N|N|N|N|N|N|Legal discussions must not take a|Unknown|N| +27|AAAAAAAALBAAAAAA|2450849|2450879|15421|1000.00|1|ation|Y|N|N|N|N|N|N|N|Mental men go british friends. Forests show lo|Unknown|N| +28|AAAAAAAAMBAAAAAA|2450816|2450856|1885||1||N|N|N|N|N||N|N|Successful opportunities|Unknown|| +29|AAAAAAAANBAAAAAA|2450696|2450732|14660|1000.00|1|n st|Y|N|N|N|N|N|N|N|Residential, necessary |Unknown|N| +30|AAAAAAAAOBAAAAAA|2450718|2450762|10987|1000.00|1|bar|N|N|N|N|N|N|N|N|Flowers used to stop there separate, hi|Unknown|N| +31|AAAAAAAAPBAAAAAA|2450848|2450880|11990|1000.00|1|ought|Y|N|N|N|N|N|N|N|Broad times might satisfy bars. Then short men anticipat|Unknown|N| +32|AAAAAAAAACAAAAAA|2450132|2450178|4184|1000.00|1|able|Y|N|N|N|N|N|N|N|Bad matters offer und|Unknown|N| +33|AAAAAAAABCAAAAAA|2450483|2450485|9607|1000.00|1|pri|N|N|N|N|N|N|N|N|Post-war subjects add possible, new units; nearly presen|Unknown|N| +34|AAAAAAAACCAAAAAA|2450584|2450610|1652|1000.00|1|ese|N|N|N|N|N|N|N|N|Strong cells could dis|Unknown|N| +35|AAAAAAAADCAAAAAA|2450890|2450912|15751|1000.00|1|anti|Y|N|N|N|N|N|N|N|Ideas register hence used provinces; |Unknown|N| +36|AAAAAAAAECAAAAAA|2450577|2450605|1418|1000.00|1|cally|N|N|N|N|N|N|N|N|Companies might stay|Unknown|N| +37|AAAAAAAAFCAAAAAA|2450609|2450669|4202|1000.00|1|ation|Y|N|N|N|N|N|N|N|Studies make little just mad rates. Detailed, flexib|Unknown|N| +38|AAAAAAAAGCAAAAAA|2450542|2450593|4222|1000.00|1|eing|Y|N|N|N|N|N|N|N|Exercises would not contain there; interests|Unknown|N| +39|AAAAAAAAHCAAAAAA|2450575|2450580|14990|1000.00|1|n st|N|N|N|N|N|N|N|N|Further final words must|Unknown|N| +40|AAAAAAAAICAAAAAA|2450125|2450158|17209|1000.00|1|bar|N|N|N|N|N|N|N|N|Deaf, tall members create as well confe|Unknown|N| +41|AAAAAAAAJCAAAAAA|2450694|2450719|17384|1000.00|1|ought|Y|N|N|N|N|N|N|N|There main cards try results. Lonely|Unknown|N| +42|AAAAAAAAKCAAAAAA|2450320|2450334|4166|1000.00|1|able|Y|N|N|N|N|N|N|N|Good, very problems might not look to a door|Unknown|N| +43|AAAAAAAALCAAAAAA|2450704|2450738|10000|1000.00|1|pri|N|N|N|N|N|N|N|N|Probably typical eyes mig|Unknown|N| +44|AAAAAAAAMCAAAAAA|2450469|2450481|14440|1000.00|1|ese|N|N|N|N|N|N|N|N|Roads would not grasp vital, guilty weaknesses. Other, |Unknown|N| +45|AAAAAAAANCAAAAAA|2450327|2450360|5660|1000.00|1|anti|N|N|N|N|N|N|N|N|Wrong, popular issues ought|Unknown|N| +46|AAAAAAAAOCAAAAAA|2450776|2450828|3271|1000.00|1|cally|Y|N|N|N|N|N|N|N|Warm decisions could not make over|Unknown|N| +47|AAAAAAAAPCAAAAAA|2450196|2450200|5689|1000.00|1|ation|N|N|N|N|N|N|N|N|Large, general points descri|Unknown|N| +48|AAAAAAAAADAAAAAA|2450783|2450807|3955|1000.00|1|eing|N|N|N|N|N|N|N|N|Losses must suspect fu|Unknown|N| +49|AAAAAAAABDAAAAAA|2450254|2450289|16123|1000.00|1|n st|N|N|N|N|N|N|N|N|Then particular farms attain to a types. Year|Unknown|N| +50|AAAAAAAACDAAAAAA|2450799|2450806|8722|1000.00|1|bar|Y|N|N|N|N|N|N|N|Favorite, united sales can budge to|Unknown|N| +51|AAAAAAAADDAAAAAA|2450401|2450438|11288|1000.00|1|ought|Y|N|N|N|N|N|N|N|Now liberal characteristics think relationshi|Unknown|N| +52|AAAAAAAAEDAAAAAA|2450272|2450320|3280|1000.00|1|able|Y|N|N|N|N|N|N|N|Authorities must pursue from a principles|Unknown|N| +53|AAAAAAAAFDAAAAAA|2450794|2450798|13370|1000.00|1|pri|N|N|N|N|N|N|N|N|Free hospitals cannot giv|Unknown|N| +54|AAAAAAAAGDAAAAAA|2450687|2450702|17707|1000.00|1|ese|N|N|N|N|N|N|N|N|Claims would not avoid. Approximately smal|Unknown|N| +55|AAAAAAAAHDAAAAAA|2450622|2450677|12104|1000.00|1|anti|N|N|N|N|N|N|N|N|Recent, southern lists go. Strang|Unknown|N| +56|AAAAAAAAIDAAAAAA|2450838|2450888|17563|1000.00|1|cally|Y|N|N|N|N|N|N|N|Procedures can call healthy women. Pla|Unknown|N| +57|AAAAAAAAJDAAAAAA|2450834|2450842|12247|1000.00|1|ation|Y|N|N|N|N|N|N|N|Hours discriminate tracks. Statements migh|Unknown|N| +58|AAAAAAAAKDAAAAAA||2450797|12404|1000.00||eing|Y|N|N|N||N|||||| +59|AAAAAAAALDAAAAAA|2450273|2450322|5102|1000.00|1|n st|N|N|N|N|N|N|N|N|Strong samples participate. Influential legs fa|Unknown|N| +60|AAAAAAAAMDAAAAAA|2450646|2450676|12076|1000.00|1|bar|Y|N|N|N|N|N|N|N|Yards write too only shapes. Real cigarettes boost less mod|Unknown|N| +61|AAAAAAAANDAAAAAA|2450619|2450667|14864|1000.00|1|ought|Y|N|N|N|N|N|N|N|Necessary men shall make much |Unknown|N| +62|AAAAAAAAODAAAAAA|2450296|2450342|356|1000.00|1|able|Y|N|N|N|N|N|N|N|Only prepared minutes see far now outer resource|Unknown|N| +63|AAAAAAAAPDAAAAAA|2450248|2450295|3715|1000.00|1|pri|Y|N|N|N|N|N|N|N|Likely, permanent rights shall not make more members.|Unknown|N| +64|AAAAAAAAAEAAAAAA|2450498|2450519|1867|1000.00|1|ese|N|N|N|N|N|N|N|N|Costs would not operate; wrong, local honours meet just |Unknown|N| +65|AAAAAAAABEAAAAAA|2450775|2450816|3740|1000.00|1|anti|Y|N|N|N|N|N|N|N|Usually local days beg|Unknown|N| +66|AAAAAAAACEAAAAAA|2450486|2450504|4886|1000.00|1|cally|Y|N|N|N|N|N|N|N|Yet unaware methods used to try again only, accepta|Unknown|N| +67|AAAAAAAADEAAAAAA|2450262|2450314|6724|1000.00|1|ation|Y|N|N|N|N|N|N|N|Statutory states will not see directly necessary activ|Unknown|N| +68|AAAAAAAAEEAAAAAA|2450851|2450893|1210|1000.00|1|eing|Y|N|N|N|N|N|N|N|Reasonably additional shows can identify scientists. T|Unknown|N| +69|AAAAAAAAFEAAAAAA|2450202|2450219|7471|1000.00|1|n st|N|N|N|N|N|N|N|N|All right able intere|Unknown|N| +70|AAAAAAAAGEAAAAAA|2450679|2450735|3571|1000.00|1|bar|Y|N|N|N|N|N|N|N|Commercial, cool cells stay presu|Unknown|N| +71|AAAAAAAAHEAAAAAA|2450347||||||Y||N||N|N|N|N|Impossible programmes say acr||| +72|AAAAAAAAIEAAAAAA|2450347|2450373|6662|1000.00|1|able|N|N|N|N|N|N|N|N|Beautiful, british trees lea|Unknown|N| +73|AAAAAAAAJEAAAAAA|2450578|2450637|9032|1000.00|1|pri|Y|N|N|N|N|N|N|N|Tremendous, social towns used to move seemingl|Unknown|N| +74|AAAAAAAAKEAAAAAA|2450425|2450456|13904|1000.00|1|ese|N|N|N|N|N|N|N|N|Decent bodies grant overseas pretty lights. Long, cau|Unknown|N| +75|AAAAAAAALEAAAAAA|2450241|2450257|7630|1000.00|1|anti|Y|N|N|N|N|N|N|N|Sudden weeks dwell for instance for a pp.|Unknown|N| +76|AAAAAAAAMEAAAAAA|2450363|2450381|295|1000.00|1|cally|Y|N|N|N|N|N|N|N|Other, secondary interests know. Disabled, pop|Unknown|N| +77|AAAAAAAANEAAAAAA|2450711|2450736|3343|1000.00|1|ation|Y|N|N|N|N|N|N|N|Characters translate hitherto. Exce|Unknown|N| +78|AAAAAAAAOEAAAAAA|2450454|2450513|5356|1000.00|1|eing|Y|N|N|N|N|N|N|N|Standard studies arise as. Seriously current commen|Unknown|N| +79|AAAAAAAAPEAAAAAA|2450474|2450497|3736|1000.00|1|n st|N|N|N|N|N|N|N|N|Devices might not produce clearly silly items. Resource|Unknown|N| +80|AAAAAAAAAFAAAAAA|2450891|2450910|4148|1000.00|1|bar|N|N|N|N|N|N|N|N|Roles might not seem everyday contra|Unknown|N| +81|AAAAAAAABFAAAAAA|2450235|2450283|4010|1000.00|1|ought|N|N|N|N|N|N|N|N|Accused, dual women must assemble. No|Unknown|N| +82|AAAAAAAACFAAAAAA|2450685|2450718|12373|1000.00|1|able|Y|N|N|N|N|N|N|N|Economic, subject matters go tories. Tonnes say neve|Unknown|N| +83|AAAAAAAADFAAAAAA|2450175|2450235|10738|1000.00|1|pri|N|N|N|N|N|N|N|N|Original, huge projects|Unknown|N| +84|AAAAAAAAEFAAAAAA|2450900|2450954|16621|1000.00|1|ese|Y|N|N|N|N|N|N|N|Houses watch machine|Unknown|N| +85|AAAAAAAAFFAAAAAA|2450696|2450706|16894|1000.00|1|anti|Y|N|N|N|N|N|N|N|Only red systems ring right b|Unknown|N| +86|AAAAAAAAGFAAAAAA|2450745|2450787|11761|1000.00|1|cally|Y|N|N|N|N|N|N|N|Incentives could not look then excellent relationships. Wari|Unknown|N| +87|AAAAAAAAHFAAAAAA|2450712|2450730|16600|1000.00|1|ation|N|N|N|N|N|N|N|N|Numbers drive however. Yesterday ne|Unknown|N| +88|AAAAAAAAIFAAAAAA|2450102|2450151|12728|1000.00|1|eing|N|N|N|N|N|N|N|N|Chosen players try foreign parents. Safe, happy|Unknown|N| +89|AAAAAAAAJFAAAAAA|2450649|2450658|8566|1000.00|1|n st|Y|N|N|N|N|N|N|N|Cultural, obvious constra|Unknown|N| +90|AAAAAAAAKFAAAAAA|2450587|2450615|8527|1000.00|1|bar|Y|N|N|N|N|N|N|N|Labour children replace together.|Unknown|N| +91|AAAAAAAALFAAAAAA|2450723|2450742|2326|1000.00|1|ought|Y|N|N|N|N|N|N|N|Other, standard pupils go honest images. Di|Unknown|N| +92|AAAAAAAAMFAAAAAA|2450737|2450748|13580|1000.00|1|able|N|N|N|N|N|N|N|N|Jobs may not train equally by a leaders. New, red bui|Unknown|N| +93|AAAAAAAANFAAAAAA|2450125|2450154|4790|1000.00|1|pri|Y|N|N|N|N|N|N|N|Monthly ambitions convey less things. Economic occasions|Unknown|N| +94|AAAAAAAAOFAAAAAA|2450567|2450598|28|1000.00|1|ese|N|N|N|N|N|N|N|N|Small ladies shall wear sometimes urban|Unknown|N| +95|AAAAAAAAPFAAAAAA|2450370|2450397|199|1000.00|1|anti|N|N|N|N|N|N|N|N|Long travellers ought to excuse existing ti|Unknown|N| +96|AAAAAAAAAGAAAAAA|2450710|2450752|16226|1000.00|1|cally|Y|N|N|N|N|N|N|N|Little new boys wear only basic,|Unknown|N| +97|AAAAAAAABGAAAAAA|2450660|2450685|16952|1000.00|1|ation|Y|N|N|N|N|N|N|N|New, severe employee|Unknown|N| +98|AAAAAAAACGAAAAAA|2450384|2450387|10414|1000.00|1|eing|Y|N|N|N|N|N|N|N|Periods shall not need as exa|Unknown|N| +99|AAAAAAAADGAAAAAA|2450742|2450799|7544|1000.00|1|n st|Y|N|N|N|N|N|N|N|White, easy shoulders might tell. Black,|Unknown|N| +100|AAAAAAAAEGAAAAAA|2450508|2450562|10645|1000.00|1|bar|N|N|N|N|N|N|N|N|Villages go more. Ministers surprise now blue, usu|Unknown|N| +101|AAAAAAAAFGAAAAAA||2450811||||||N|N||||N|N|Reports would make at a women. Big, big offices may not meet|Unknown|N| +102|AAAAAAAAGGAAAAAA|2450608|2450614|9614|1000.00|1|able|Y|N|N|N|N|N|N|N|Medieval fires may read; much young leaders give quickly|Unknown|N| +103|AAAAAAAAHGAAAAAA|2450499|2450525|12688|1000.00|1|pri|N|N|N|N|N|N|N|N|Still ready eyes will create still worried|Unknown|N| +104|AAAAAAAAIGAAAAAA|2450276|2450293|2461|1000.00|1|ese|N|N|N|N|N|N|N|N|Years will sum always small, clear b|Unknown|N| +105|AAAAAAAAJGAAAAAA|2450873|2450912|14182|1000.00|1|anti|Y|N|N|N|N|N|N|N|Other definitions prevent good, universal stud|Unknown|N| +106|AAAAAAAAKGAAAAAA|2450858|2450872|3754|1000.00|1|cally|N|N|N|N|N|N|N|N|Long large problems should |Unknown|N| +107|AAAAAAAALGAAAAAA|2450749|2450758|10574|1000.00|1|ation|Y|N|N|N|N|N|N|N|Good, foreign problems will n|Unknown|N| +108|AAAAAAAAMGAAAAAA|2450224|2450257|274|1000.00|1|eing|Y|N|N|N|N|N|N|N|High, tiny sides may pay totally|Unknown|N| +109|AAAAAAAANGAAAAAA|2450455|2450467|8302|1000.00|1|n st|Y|N|N|N|N|N|N|N|Reasonable posts become for example available rates. Kind|Unknown|N| +110|AAAAAAAAOGAAAAAA|2450852|2450905|15568|1000.00|1|bar|N|N|N|N|N|N|N|N|Huge, european users reveal|Unknown|N| +111|AAAAAAAAPGAAAAAA|2450758|2450816|2462|1000.00|1|ought|Y|N|N|N|N|N|N|N|Huge, early keys come greatly gay schools; great instan|Unknown|N| +112|AAAAAAAAAHAAAAAA|2450870|2450916|7640|1000.00|1|able|N|N|N|N|N|N|N|N|Greek, young minutes will divide then roman payments|Unknown|N| +113|AAAAAAAABHAAAAAA|2450226|2450276|13916|1000.00|1|pri|Y|N|N|N|N|N|N|N|Warm hands know as much strong terms. Tightly extra|Unknown|N| +114|AAAAAAAACHAAAAAA|2450687|2450720|13507|1000.00|1|ese|N|N|N|N|N|N|N|N|New, necessary kilometres lock so. Deci|Unknown|N| +115|AAAAAAAADHAAAAAA|2450727|2450751|15002|1000.00|1|anti|N|N|N|N|N|N|N|N|Personal wounds see sec|Unknown|N| +116|AAAAAAAAEHAAAAAA|2450352|2450361|9187|1000.00|1|cally|N|N|N|N|N|N|N|N|Bad, close tories could not appreciate also |Unknown|N| +117|AAAAAAAAFHAAAAAA|2450285|2450308|8854|1000.00|1|ation|Y|N|N|N|N|N|N|N|Extreme centres will learn ever blue men; proportions ident|Unknown|N| +118|AAAAAAAAGHAAAAAA|2450460|2450471|7183|1000.00|1|eing|N|N|N|N|N|N|N|N|Very proposed pressures shall not take major, average f|Unknown|N| +119|AAAAAAAAHHAAAAAA|2450224|2450236|13903|1000.00|1|n st|Y|N|N|N|N|N|N|N|Similar ideas go origin|Unknown|N| +120|AAAAAAAAIHAAAAAA|2450302|2450341|5578|1000.00|1|bar|N|N|N|N|N|N|N|N|Children shall come |Unknown|N| +121|AAAAAAAAJHAAAAAA|2450752|2450769|1648|1000.00|1|ought|N|N|N|N|N|N|N|N|Essentially nervous cells should no|Unknown|N| +122|AAAAAAAAKHAAAAAA|2450422|2450452|6778|1000.00|1|able|Y|N|N|N|N|N|N|N|In order certain estates used to answer ho|Unknown|N| +123|AAAAAAAALHAAAAAA|2450628|2450673|2132|1000.00|1|pri|N|N|N|N|N|N|N|N|English, other followers help here claims.|Unknown|N| +124|AAAAAAAAMHAAAAAA|2450517|2450565|7051|1000.00|1|ese|Y|N|N|N|N|N|N|N|Institutional, new men used to win |Unknown|N| +125|AAAAAAAANHAAAAAA|2450704|2450719|6439|1000.00|1|anti|N|N|N|N|N|N|N|N|Societies would justify today at|Unknown|N| +126|AAAAAAAAOHAAAAAA|2450840|2450852|7039|1000.00|1|cally|Y|N|N|N|N|N|N|N|Drivers could overcome more ages; loud short-term children|Unknown|N| +127|AAAAAAAAPHAAAAAA|2450802|2450836|13510|1000.00|1|ation|N|N|N|N|N|N|N|N|Now obvious players make ever|Unknown|N| +128|AAAAAAAAAIAAAAAA|2450373|2450398|9080|1000.00|1|eing|N|N|N|N|N|N|N|N|Too possible curtains stop d|Unknown|N| +129|AAAAAAAABIAAAAAA|2450357|2450373|614|1000.00|1|n st|Y|N|N|N|N|N|N|N|Comments can release still. Really final others might consid|Unknown|N| +130|AAAAAAAACIAAAAAA|2450219|2450236|13598|1000.00|1|bar|Y|N|N|N|N|N|N|N|Years arrange just to the men. Dif|Unknown|N| +131|AAAAAAAADIAAAAAA|2450305|2450313|3793|1000.00|1|ought|Y|N|N|N|N|N|N|N|Unable children cannot answer for an |Unknown|N| +132|AAAAAAAAEIAAAAAA|2450231|2450240|4219|1000.00|1|able|N|N|N|N|N|N|N|N|Warm, blue officers s|Unknown|N| +133|AAAAAAAAFIAAAAAA|2450587|2450593|854|1000.00|1|pri|Y|N|N|N|N|N|N|N|Charming, negative houses buy more in the implicat|Unknown|N| +134|AAAAAAAAGIAAAAAA|2450479|2450501|10922|1000.00|1|ese|N|N|N|N|N|N|N|N|Skills can say very days. Mini|Unknown|N| +135|AAAAAAAAHIAAAAAA|2450607|2450624|9748|1000.00|1|anti|N|N|N|N|N|N|N|N|Officers will allow sig|Unknown|N| +136|AAAAAAAAIIAAAAAA|2450701|2450710|13849|1000.00|1|cally|N|N|N|N|N|N|N|N|Mountains would want just eyebrows. Prime figu|Unknown|N| +137|AAAAAAAAJIAAAAAA|2450123|2450177|15872|1000.00|1|ation|N|N|N|N|N|N|N|N|Finally capable rates will not catch more on a wi|Unknown|N| +138|AAAAAAAAKIAAAAAA|2450758|2450760|12814|1000.00|1|eing|Y|N|N|N|N|N|N|N|So reasonable wages know parents.|Unknown|N| +139|AAAAAAAALIAAAAAA|2450365|2450383|5882|1000.00|1|n st|N|N|N|N|N|N|N|N|Then relevant changes take aga|Unknown|N| +140|AAAAAAAAMIAAAAAA|2450655|2450708|12968|1000.00|1|bar|N|N|N|N|N|N|N|N|Tiny days must feel kinds. Able designs will not rem|Unknown|N| +141|AAAAAAAANIAAAAAA|2450301|2450353|13100|1000.00|1|ought|Y|N|N|N|N|N|N|N|As political structures|Unknown|N| +142|AAAAAAAAOIAAAAAA|2450358|2450385|12956|1000.00|1|able|Y|N|N|N|N|N|N|N|Well female movements wander. Faint origins shall des|Unknown|N| +143|AAAAAAAAPIAAAAAA|2450828|2450854|13006|1000.00|1|pri|N|N|N|N|N|N|N|N|Hands sink good social newspapers. Thin terms used|Unknown|N| +144|AAAAAAAAAJAAAAAA|2450780|2450830|15688|1000.00|1|ese|N|N|N|N|N|N|N|N|Reasonably social imports w|Unknown|N| +145|AAAAAAAABJAAAAAA|2450715|2450756|10406|1000.00|1|anti|N|N|N|N|N|N|N|N|Sorry, unlikely practitioners demand either|Unknown|N| +146|AAAAAAAACJAAAAAA|2450200|2450236|8119|1000.00|1|cally|Y|N|N|N|N|N|N|N|Direct, added regions could |Unknown|N| +147|AAAAAAAADJAAAAAA|2450767|2450827|15925|1000.00|1|ation|N|N|N|N|N|N|N|N|Years project groups; social, regular organizations |Unknown|N| +148|AAAAAAAAEJAAAAAA|2450291|2450345|11275|1000.00|1|eing|Y|N|N|N|N|N|N|N|Just wrong samples strengthen rig|Unknown|N| +149|AAAAAAAAFJAAAAAA|2450739|2450797|17719|1000.00|1|n st|Y|N|N|N|N|N|N|N|Public fields may clear. Priest|Unknown|N| +150|AAAAAAAAGJAAAAAA|2450860|2450915|9578|1000.00|1|bar|Y|N|N|N|N|N|N|N|Nearby figures would run rem|Unknown|N| +151|AAAAAAAAHJAAAAAA|2450707|2450751|4408|1000.00|1|ought|N|N|N|N|N|N|N|N|Much financial centuries shall believe still common, re|Unknown|N| +152|AAAAAAAAIJAAAAAA|2450564|2450573|2314|1000.00|1|able|N|N|N|N|N|N|N|N|Over dependent topics must use apparent, fre|Unknown|N| +153|AAAAAAAAJJAAAAAA|2450712|2450725|16514|1000.00|1|pri|N|N|N|N|N|N|N|N|Organisations make altogether as new nations. Soc|Unknown|N| +154|AAAAAAAAKJAAAAAA|2450869|2450898|2674|1000.00|1|ese|Y|N|N|N|N|N|N|N|Following, current matters|Unknown|N| +155|AAAAAAAALJAAAAAA|2450409|2450419|202|1000.00|1|anti|N|N|N|N|N|N|N|N|Expectations stop exactly. Other, rea|Unknown|N| +156|AAAAAAAAMJAAAAAA|2450681|2450732|7975|1000.00|1|cally|Y|N|N|N|N|N|N|N|Executive, poor jobs look also foreign bodies|Unknown|N| +157|AAAAAAAANJAAAAAA|2450389|2450445|6079|1000.00|1|ation|Y|N|N|N|N|N|N|N|Certainly pleasant features come; as inevitable women bid wi|Unknown|N| +158|AAAAAAAAOJAAAAAA|2450350|2450390|17863|1000.00|1|eing|N|N|N|N|N|N|N|N|Levels must not trans|Unknown|N| +159|AAAAAAAAPJAAAAAA|2450682|2450700|58|1000.00|1|n st|Y|N|N|N|N|N|N|N|Regional schemes would devise even loc|Unknown|N| +160|AAAAAAAAAKAAAAAA|2450583|2450642|13144|1000.00|1|bar|N|N|N|N|N|N|N|N|Blocks generate still on a eyes; just other arrangements get|Unknown|N| +161|AAAAAAAABKAAAAAA|2450900|2450955|680|1000.00|1|ought|Y|N|N|N|N|N|N|N|Concepts might live of cours|Unknown|N| +162|AAAAAAAACKAAAAAA|2450532|2450557|11194|1000.00|1|able|N|N|N|N|N|N|N|N|Once short boots please; small, spe|Unknown|N| +163|AAAAAAAADKAAAAAA|2450102|2450142|5038|1000.00|1|pri|Y|N|N|N|N|N|N|N|Glasses tell even as tight ca|Unknown|N| +164|AAAAAAAAEKAAAAAA|2450423|2450453|17593|1000.00|1|ese|Y|N|N|N|N|N|N|N|Available, blue hours explo|Unknown|N| +165|AAAAAAAAFKAAAAAA|2450854|2450894|3166|1000.00|1|anti|Y|N|N|N|N|N|N|N|Over responsible demonstrations |Unknown|N| +166|AAAAAAAAGKAAAAAA|2450365|2450383|6652|1000.00|1|cally|N|N|N|N|N|N|N|N|Others matter serious catego|Unknown|N| +167|AAAAAAAAHKAAAAAA|2450648|2450651|3922|1000.00|1|ation|Y|N|N|N|N|N|N|N|Urban abilities would|Unknown|N| +168|AAAAAAAAIKAAAAAA|2450271|2450331|17422|1000.00|1|eing|N|N|N|N|N|N|N|N|Slightly technical dut|Unknown|N| +169|AAAAAAAAJKAAAAAA|2450716|2450774|1726|1000.00|1|n st|Y|N|N|N|N|N|N|N|Great chapters would move patently white |Unknown|N| +170|AAAAAAAAKKAAAAAA|2450702|2450724|17498|1000.00|1|bar|Y|N|N|N|N|N|N|N|Alternatives induce very in the peasants. Necessary cont|Unknown|N| +171|AAAAAAAALKAAAAAA|2450635|2450642|5383|1000.00|1|ought|N|N|N|N|N|N|N|N|Needs count from a se|Unknown|N| +172|AAAAAAAAMKAAAAAA|2450735|2450785|10030|1000.00|1|able|Y|N|N|N|N|N|N|N|Complete decisions would not find curio|Unknown|N| +173|AAAAAAAANKAAAAAA|2450172|2450199|12698|1000.00|1|pri|N|N|N|N|N|N|N|N|Constitutional materials |Unknown|N| +174|AAAAAAAAOKAAAAAA|2450463|2450474|17428|1000.00|1|ese|Y|N|N|N|N|N|N|N|Young, funny lines shall tell colours. Single examples|Unknown|N| +175|AAAAAAAAPKAAAAAA|2450675|2450704|6982|1000.00|1|anti|Y|N|N|N|N|N|N|N|Signs hear moreover nations. There perfect|Unknown|N| +176|AAAAAAAAALAAAAAA|2450487|2450498|16496|1000.00|1|cally|N|N|N|N|N|N|N|N|Only british areas understand. Also public i|Unknown|N| +177|AAAAAAAABLAAAAAA|2450729|2450733|4483|1000.00|1|ation|N|N|N|N|N|N|N|N|Models would love high t|Unknown|N| +178|AAAAAAAACLAAAAAA|2450345|2450392|15134|1000.00|1|eing|Y|N|N|N|N|N|N|N|Right obvious pounds could not prove. Political, c|Unknown|N| +179|AAAAAAAADLAAAAAA|2450230|2450281|12496|1000.00|1|n st|Y|N|N|N|N|N|N|N|Left, encouraging bi|Unknown|N| +180|AAAAAAAAELAAAAAA|2450471|2450484|12517|1000.00|1|bar|Y|N|N|N|N|N|N|N|Significant children se|Unknown|N| +181|AAAAAAAAFLAAAAAA|2450649|2450673|11653|1000.00|1|ought|Y|N|N|N|N|N|N|N|Away due moments will not feed specifically. Ple|Unknown|N| +182|AAAAAAAAGLAAAAAA|2450703|2450731|2734|1000.00|1|able|Y|N|N|N|N|N|N|N|Other, full members fin|Unknown|N| +183|AAAAAAAAHLAAAAAA|2450777|2450824|2420|1000.00|1|pri|N|N|N|N|N|N|N|N|Increases could not buy socia|Unknown|N| +184|AAAAAAAAILAAAAAA|2450241|2450300|12238|1000.00|1|ese|N|N|N|N|N|N|N|N|Favourably visible children oug|Unknown|N| +185|AAAAAAAAJLAAAAAA|2450688|2450702|15091|1000.00|1|anti|N|N|N|N|N|N|N|N|Fingers help monthly, onl|Unknown|N| +186|AAAAAAAAKLAAAAAA|2450627|2450643|5512|1000.00|1|cally|N|N|N|N|N|N|N|N|Effectively appropriate eyes achieve m|Unknown|N| +187|AAAAAAAALLAAAAAA|2450484|2450530|4798|1000.00|1|ation|Y|N|N|N|N|N|N|N|Enough detailed months try so chronic, local discussions. R|Unknown|N| +188|AAAAAAAAMLAAAAAA|2450385|2450413|14246|1000.00|1|eing|Y|N|N|N|N|N|N|N|Machines open only unique minut|Unknown|N| +189|AAAAAAAANLAAAAAA|2450420|2450456|12550|1000.00|1|n st|N|N|N|N|N|N|N|N|Junior activities discover de|Unknown|N| +190|AAAAAAAAOLAAAAAA|2450512|2450563|15151|1000.00|1|bar|Y|N|N|N|N|N|N|N|Customers live now. Systematic, free others se|Unknown|N| +191|AAAAAAAAPLAAAAAA|2450126|2450160|7993|1000.00|1|ought|Y|N|N|N|N|N|N|N|Experiences must not win instead surely certain accou|Unknown|N| +192|AAAAAAAAAMAAAAAA|2450556|2450573|3026|1000.00|1|able|N|N|N|N|N|N|N|N|Days come for a quarters. Periods shall not read also as |Unknown|N| +193|AAAAAAAABMAAAAAA|2450259|2450309|8780|1000.00|1|pri|Y|N|N|N|N|N|N|N|Terms should not know. Communists will not describe e|Unknown|N| +194|AAAAAAAACMAAAAAA|2450664|2450723|289|1000.00|1|ese|Y|N|N|N|N|N|N|N|Cultural deaths could develop yet able conditions. Nation|Unknown|N| +195|AAAAAAAADMAAAAAA|2450543|2450601|5167|1000.00|1|anti|N|N|N|N|N|N|N|N|Conventions make immediately glad branche|Unknown|N| +196|AAAAAAAAEMAAAAAA|2450315||||1||Y|||N|N|||N|Able patients will lend as. Southern components must com||| +197|AAAAAAAAFMAAAAAA|2450299||10550||1|||N|N|N|N|||||Unknown|| +198|AAAAAAAAGMAAAAAA|2450617|2450670|15439|1000.00|1|eing|Y|N|N|N|N|N|N|N|German, crude expectations must advanc|Unknown|N| +199|AAAAAAAAHMAAAAAA|2450772|2450825|8587|1000.00|1|n st|N|N|N|N|N|N|N|N|Powerful, considerable programs could operate bloody nor|Unknown|N| +200|AAAAAAAAIMAAAAAA|2450215|2450237|12784|1000.00|1|bar|N|N|N|N|N|N|N|N|Arts attend. Further irish miles decide in the advances|Unknown|N| +201|AAAAAAAAJMAAAAAA|2450464|2450499|3445|1000.00|1|ought|N|N|N|N|N|N|N|N|Square, unusual years erect so outdoor, co|Unknown|N| +202|AAAAAAAAKMAAAAAA|2450913|2450922|3883|1000.00|1|able|Y|N|N|N|N|N|N|N|Strong, traditional paren|Unknown|N| +203|AAAAAAAALMAAAAAA|2450337|2450365|17569|1000.00|1|pri|Y|N|N|N|N|N|N|N|Completely brave charges may sit then observers. So total|Unknown|N| +204|AAAAAAAAMMAAAAAA|2450692|2450736|1564|1000.00|1|ese|Y|N|N|N|N|N|N|N|Sideways numerous departments make just. St|Unknown|N| +205|AAAAAAAANMAAAAAA|2450133|2450160|9889|1000.00|1|anti|Y|N|N|N|N|N|N|N|Little hot conditions look nice, possible results; gen|Unknown|N| +206|AAAAAAAAOMAAAAAA|2450715|2450733|9829|1000.00|1|cally|Y|N|N|N|N|N|N|N|Final workers resign. Now nuclear accounts get r|Unknown|N| +207|AAAAAAAAPMAAAAAA|2450699|2450704|4273|1000.00|1|ation|Y|N|N|N|N|N|N|N|Economic, professional models cannot hit a|Unknown|N| +208|AAAAAAAAANAAAAAA|2450420|2450464|5348|1000.00|1|eing|Y|N|N|N|N|N|N|N|Heavy, main circumstances used to appear |Unknown|N| +209|AAAAAAAABNAAAAAA|2450563|2450580|3430|1000.00|1|n st|N|N|N|N|N|N|N|N|Hours open similarly international companies. Regardless|Unknown|N| +210|AAAAAAAACNAAAAAA|2450847|2450892|7411|1000.00|1|bar|Y|N|N|N|N|N|N|N|So new sports will not discuss of course big diff|Unknown|N| +211|AAAAAAAADNAAAAAA|2450646|2450649|7508|1000.00|1|ought|Y|N|N|N|N|N|N|N|Public, running ladies l|Unknown|N| +212|AAAAAAAAENAAAAAA|2450781|2450834|10768|1000.00|1|able|N|N|N|N|N|N|N|N|Good decades take. Proper, black unions must not |Unknown|N| +213|AAAAAAAAFNAAAAAA|2450403|2450415|6350|1000.00|1|pri|Y|N|N|N|N|N|N|N|However powerful critics split just eyes.|Unknown|N| +214|AAAAAAAAGNAAAAAA||2450779||1000.00|||N||||||N|N|Pupils will meet military travellers. Other sequenc|Unknown|N| +215|AAAAAAAAHNAAAAAA|2450235|2450252|5840|1000.00|1|anti|Y|N|N|N|N|N|N|N|Apparent, experienced bombs |Unknown|N| +216|AAAAAAAAINAAAAAA|2450885|2450898|13504|1000.00|1|cally|N|N|N|N|N|N|N|N|Large, necessary cars support since as a car|Unknown|N| +217|AAAAAAAAJNAAAAAA|2450265|2450316|466|1000.00|1|ation|N|N|N|N|N|N|N|N|Also splendid terms enter never nice, foreign periods; v|Unknown|N| +218|AAAAAAAAKNAAAAAA|2450443|2450490|17338|1000.00|1|eing|N|N|N|N|N|N|N|N|Great texts will blow; mostly |Unknown|N| +219|AAAAAAAALNAAAAAA|2450821|2450859|12662|1000.00|1|n st|N|N|N|N|N|N|N|N|Briefly old records must not find precious instr|Unknown|N| +220|AAAAAAAAMNAAAAAA|2450857|2450907|17108|1000.00|1|bar|N|N|N|N|N|N|N|N|Also long refugees interpret thus; regular sections sha|Unknown|N| +221|AAAAAAAANNAAAAAA|2450134|2450182|2863|1000.00|1|ought|N|N|N|N|N|N|N|N|Probable, available categories will not |Unknown|N| +222|AAAAAAAAONAAAAAA|2450336|2450378|122|1000.00|1|able|Y|N|N|N|N|N|N|N|Ways help. Everywhere ma|Unknown|N| +223|AAAAAAAAPNAAAAAA|2450671|2450721|11452|1000.00|1|pri|N|N|N|N|N|N|N|N|Earlier traditional groups|Unknown|N| +224|AAAAAAAAAOAAAAAA|2450135|2450140|13534|1000.00|1|ese|Y|N|N|N|N|N|N|N|Then general preferences cou|Unknown|N| +225|AAAAAAAABOAAAAAA||2450434|6514|1000.00||anti||N||||N|||Actual, very words used to divide secondly ||| +226|AAAAAAAACOAAAAAA|2450436|2450468|2848|1000.00|1|cally|N|N|N|N|N|N|N|N|Signs must drink certainly buildings. |Unknown|N| +227|AAAAAAAADOAAAAAA|2450751|2450794|3697|1000.00|1|ation|N|N|N|N|N|N|N|N|Arms can find social, great groups. Female, sweet forms|Unknown|N| +228|AAAAAAAAEOAAAAAA|2450337|2450354|11678|1000.00|1|eing|Y|N|N|N|N|N|N|N|Most basic walls used to attack in a hospitals. |Unknown|N| +229|AAAAAAAAFOAAAAAA|2450146|2450188|10486|1000.00|1|n st|N|N|N|N|N|N|N|N|Ships make never netwo|Unknown|N| +230|AAAAAAAAGOAAAAAA|2450741|2450758|17480|1000.00|1|bar|Y|N|N|N|N|N|N|N|References remember ago ove|Unknown|N| +231|AAAAAAAAHOAAAAAA|2450177|2450186|13843|1000.00|1|ought|Y|N|N|N|N|N|N|N|Christians used to arr|Unknown|N| +232|AAAAAAAAIOAAAAAA|2450769|2450822|15055|1000.00|1|able|N|N|N|N|N|N|N|N|Travellers soak very contribut|Unknown|N| +233|AAAAAAAAJOAAAAAA|2450318|2450369|1094|1000.00|1|pri|Y|N|N|N|N|N|N|N|Young, valuable companies watch walls. Payments can flour|Unknown|N| +234|AAAAAAAAKOAAAAAA|2450299|2450354|15476|1000.00|1|ese|Y|N|N|N|N|N|N|N|Inadequate christians can try possible, pal|Unknown|N| +235|AAAAAAAALOAAAAAA|2450877|2450918|4910|1000.00|1|anti|Y|N|N|N|N|N|N|N|Bloody, useful universities wait then now abl|Unknown|N| +236|AAAAAAAAMOAAAAAA|2450438|2450468|8095|1000.00|1|cally|N|N|N|N|N|N|N|N|Months may not enjoy things. Proper cattle go still recomme|Unknown|N| +237|AAAAAAAANOAAAAAA|2450731|2450791|6722|1000.00|1|ation|Y|N|N|N|N|N|N|N|True, major criteria must |Unknown|N| +238|AAAAAAAAOOAAAAAA|2450498|2450510|10280|1000.00|1|eing|N|N|N|N|N|N|N|N|Others deal both companies. Separ|Unknown|N| +239|AAAAAAAAPOAAAAAA|2450627|2450680|17362|1000.00|1|n st|Y|N|N|N|N|N|N|N|Almost following strategie|Unknown|N| +240|AAAAAAAAAPAAAAAA|2450764|2450815|4874|1000.00|1|bar|Y|N|N|N|N|N|N|N|Young, present friends back for the dangers.|Unknown|N| +241|AAAAAAAABPAAAAAA|2450580|2450596|3916|1000.00|1|ought|N|N|N|N|N|N|N|N|Other years support then political, s|Unknown|N| +242|AAAAAAAACPAAAAAA|2450393|2450396|5467|1000.00|1|able|Y|N|N|N|N|N|N|N|Just victorian flowers could n|Unknown|N| +243|AAAAAAAADPAAAAAA|2450809|2450865|1576|1000.00|1|pri|Y|N|N|N|N|N|N|N|Boys shall not rest meanwhile from the police. Co|Unknown|N| +244|AAAAAAAAEPAAAAAA|2450536|2450583|11725|1000.00|1|ese|Y|N|N|N|N|N|N|N|Unusual, key areas might come there all industrial stan|Unknown|N| +245|AAAAAAAAFPAAAAAA|2450578|2450621|16075|1000.00|1|anti|N|N|N|N|N|N|N|N|Alive substances can decid|Unknown|N| +246|AAAAAAAAGPAAAAAA|2450452|2450463|13315|1000.00|1|cally|Y|N|N|N|N|N|N|N|However warm countries could not come now ostensi|Unknown|N| +247|AAAAAAAAHPAAAAAA|2450100|2450145|12976|1000.00|1|ation|Y|N|N|N|N|N|N|N|Therefore distinguished studies enhance as old, ready|Unknown|N| +248|AAAAAAAAIPAAAAAA|2450824|2450853|12451|1000.00|1|eing|N|N|N|N|N|N|N|N|Other facts ensure large children. Easily incre|Unknown|N| +249|AAAAAAAAJPAAAAAA|2450467|2450523|4195|1000.00|1|n st|Y|N|N|N|N|N|N|N|Wrong materials should not |Unknown|N| +250|AAAAAAAAKPAAAAAA|2450606||||1|||N|N||||N|N|Private, political materials will not|Unknown|| +251|AAAAAAAALPAAAAAA|2450150|2450181|9187|1000.00|1|ought|N|N|N|N|N|N|N|N|Other quarters used to keep thus available root|Unknown|N| +252|AAAAAAAAMPAAAAAA|2450290|2450319|1292|1000.00|1|able|N|N|N|N|N|N|N|N|Small characters could not tick fairly true tickets; r|Unknown|N| +253|AAAAAAAANPAAAAAA|2450337|2450364|13351|1000.00|1|pri|Y|N|N|N|N|N|N|N|Similar, true friends ought to allow forc|Unknown|N| +254|AAAAAAAAOPAAAAAA|2450673|2450717|11371|1000.00|1|ese|Y|N|N|N|N|N|N|N|Main designers will qualify obvious, new cuts. Democra|Unknown|N| +255|AAAAAAAAPPAAAAAA|2450738|2450795|10129|1000.00|1|anti|N|N|N|N|N|N|N|N|Welsh, distinct penalties should lose sorry pairs|Unknown|N| +256|AAAAAAAAAABAAAAA|2450265|2450303|3058|1000.00|1|cally|N|N|N|N|N|N|N|N|Police shall put seats. Banks would not become less |Unknown|N| +257|AAAAAAAABABAAAAA|2450789|2450821|8486|1000.00|1|ation|N|N|N|N|N|N|N|N|Books expect international, big interests|Unknown|N| +258|AAAAAAAACABAAAAA|2450687|2450732|16892|1000.00|1|eing|N|N|N|N|N|N|N|N|Uncomfortable, total rights ought to take factors. Pec|Unknown|N| +259|AAAAAAAADABAAAAA|2450293|2450335|11440|1000.00|1|n st|Y|N|N|N|N|N|N|N|United rules recognise active, funny arrangements. Fu|Unknown|N| +260|AAAAAAAAEABAAAAA|2450430|2450485|13129|1000.00|1|bar|Y|N|N|N|N|N|N|N|Phrases must wish just movements; serious,|Unknown|N| +261|AAAAAAAAFABAAAAA|2450243|2450268|14677|1000.00|1|ought|N|N|N|N|N|N|N|N|Respectively similar children go firm la|Unknown|N| +262|AAAAAAAAGABAAAAA|2450849|2450868|2876|1000.00|1|able|N|N|N|N|N|N|N|N|Thus early wheels keep more|Unknown|N| +263|AAAAAAAAHABAAAAA|2450383|2450396|2116|1000.00|1|pri|Y|N|N|N|N|N|N|N|Public, old systems can provide f|Unknown|N| +264|AAAAAAAAIABAAAAA|2450311|2450352|16609|1000.00|1|ese|Y|N|N|N|N|N|N|N|Other parts appear regul|Unknown|N| +265|AAAAAAAAJABAAAAA|2450225|2450236|8414|1000.00|1|anti|Y|N|N|N|N|N|N|N|Ideal camps cannot stop from|Unknown|N| +266|AAAAAAAAKABAAAAA|2450408|2450412|6524|1000.00|1|cally|N|N|N|N|N|N|N|N|Common changes assure golden employees. Large, eastern ins|Unknown|N| +267|AAAAAAAALABAAAAA|2450750|2450776|17926|1000.00|1|ation|Y|N|N|N|N|N|N|N|Premises sit in a terms. Mostly huge grounds will bowl. On|Unknown|N| +268|AAAAAAAAMABAAAAA|2450424|2450445|7258|1000.00|1|eing|N|N|N|N|N|N|N|N|Rather average details must not get by a fingers. Yet e|Unknown|N| +269|AAAAAAAANABAAAAA|2450256|2450283|10297|1000.00|1|n st|Y|N|N|N|N|N|N|N|Wide original seeds might distinguish students; further co|Unknown|N| +270|AAAAAAAAOABAAAAA|2450527|2450562|9244|1000.00|1|bar|N|N|N|N|N|N|N|N|Adequate, able eyes should not look as wholly rural paintin|Unknown|N| +271|AAAAAAAAPABAAAAA|2450356|2450378|12736|1000.00|1|ought|Y|N|N|N|N|N|N|N|Never big houses should not come then including|Unknown|N| +272|AAAAAAAAABBAAAAA|2450377|2450380|14524|1000.00|1|able|N|N|N|N|N|N|N|N|British leaders could not make here con|Unknown|N| +273|AAAAAAAABBBAAAAA|2450803|2450818|5138|1000.00|1|pri|Y|N|N|N|N|N|N|N|Pupils take only years. Capitalist, d|Unknown|N| +274|AAAAAAAACBBAAAAA|2450649|2450705|3856|1000.00|1|ese|Y|N|N|N|N|N|N|N|Little, other visits might make probably broken sympto|Unknown|N| +275|AAAAAAAADBBAAAAA|2450853|2450861|2798|1000.00|1|anti|N|N|N|N|N|N|N|N|Accountants reach especiall|Unknown|N| +276|AAAAAAAAEBBAAAAA|2450196|2450210|5113|1000.00|1|cally|N|N|N|N|N|N|N|N|Positive individuals may not impose. Ab|Unknown|N| +277|AAAAAAAAFBBAAAAA|2450223|2450253|2581|1000.00|1|ation|Y|N|N|N|N|N|N|N|Large, dead miles appear far for a occasio|Unknown|N| +278|AAAAAAAAGBBAAAAA|2450730|2450738|2842|1000.00|1|eing|Y|N|N|N|N|N|N|N|Again bad tickets may fix windows. Years fit eyes. Enough lo|Unknown|N| +279|AAAAAAAAHBBAAAAA|2450609|2450642|10910|1000.00|1|n st|N|N|N|N|N|N|N|N|Effectively related observations may ch|Unknown|N| +280|AAAAAAAAIBBAAAAA|2450475|2450513|16424|1000.00|1|bar|Y|N|N|N|N|N|N|N|Mutual tourists let so long words. |Unknown|N| +281|AAAAAAAAJBBAAAAA|2450740|2450746|4213|1000.00|1|ought|N|N|N|N|N|N|N|N|Crucial cars ought to happe|Unknown|N| +282|AAAAAAAAKBBAAAAA|2450659|2450702|11851|1000.00|1|able|N|N|N|N|N|N|N|N|Inc, superior teeth avoid. Islands will not enjoy|Unknown|N| +283|AAAAAAAALBBAAAAA|2450757|2450786|2870|1000.00|1|pri|N|N|N|N|N|N|N|N|Recent, able boys might take etc oth|Unknown|N| +284|AAAAAAAAMBBAAAAA|2450239|2450267|7066|1000.00|1|ese|N|N|N|N|N|N|N|N|Over stupid guidelines know live projects. Still common num|Unknown|N| +285|AAAAAAAANBBAAAAA|2450370|2450402|10982|1000.00|1|anti|Y|N|N|N|N|N|N|N|Perhaps real customers shall intervene w|Unknown|N| +286|AAAAAAAAOBBAAAAA|2450728|2450747|17779|1000.00|1|cally|N|N|N|N|N|N|N|N|Finally glad reasons mobili|Unknown|N| +287|AAAAAAAAPBBAAAAA|2450574|2450591|4568|1000.00|1|ation|Y|N|N|N|N|N|N|N|Rules say well-known members. Russian ends cou|Unknown|N| +288|AAAAAAAAACBAAAAA|2450539|2450578|1018|1000.00|1|eing|N|N|N|N|N|N|N|N|Lively colleagues avoid once good |Unknown|N| +289|AAAAAAAABCBAAAAA|2450415|2450423|9037|1000.00|1|n st|N|N|N|N|N|N|N|N|Ways prevent at all used things. Sectors look veh|Unknown|N| +290|AAAAAAAACCBAAAAA|2450375|2450415|16634|1000.00|1|bar|N|N|N|N|N|N|N|N|In order industrial examples shall bring separ|Unknown|N| +291|AAAAAAAADCBAAAAA|2450646|2450653|10424|1000.00|1|ought|N|N|N|N|N|N|N|N|Fresh, slow benefits house reali|Unknown|N| +292|AAAAAAAAECBAAAAA|2450308|2450329|6548|1000.00|1|able|Y|N|N|N|N|N|N|N|Sources discover words. Tired, static prices g|Unknown|N| +293|AAAAAAAAFCBAAAAA|2450569|2450616|13009|1000.00|1|pri|N|N|N|N|N|N|N|N|Miles might pay. Average, activ|Unknown|N| +294|AAAAAAAAGCBAAAAA|2450759|2450762|11704|1000.00|1|ese|Y|N|N|N|N|N|N|N|Requests reappear as informally soci|Unknown|N| +295|AAAAAAAAHCBAAAAA|2450889|2450944|8272|1000.00|1|anti|Y|N|N|N|N|N|N|N|Safe women change old, |Unknown|N| +296|AAAAAAAAICBAAAAA|2450631|2450637|15518|1000.00|1|cally|N|N|N|N|N|N|N|N|Loose, extensive bodies can keep a little. Hundreds le|Unknown|N| +297|AAAAAAAAJCBAAAAA|2450217|2450250|5887|1000.00|1|ation|Y|N|N|N|N|N|N|N|Eventually general women present |Unknown|N| +298|AAAAAAAAKCBAAAAA|2450809|2450858|17287|1000.00|1|eing|Y|N|N|N|N|N|N|N|Teeth could go standards. Customs feel long important |Unknown|N| +299|AAAAAAAALCBAAAAA|2450783|2450836|1292|1000.00|1|n st|Y|N|N|N|N|N|N|N|Women contrast never dear, blue minutes. Domestic pl|Unknown|N| +300|AAAAAAAAMCBAAAAA|2450488|2450522|16879|1000.00|1|bar|Y|N|N|N|N|N|N|N|Fine, british schools please in a rules. |Unknown|N| diff --git a/spark-connector/spark-common/src/test/resources/data/tpcds/reason/reason.dat b/spark-connector/spark-common/src/test/resources/data/tpcds/reason/reason.dat new file mode 100644 index 00000000000..f6718968dff --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/data/tpcds/reason/reason.dat @@ -0,0 +1,35 @@ +1|AAAAAAAABAAAAAAA|Package was damaged| +2|AAAAAAAACAAAAAAA|Stopped working| +3|AAAAAAAADAAAAAAA|Did not get it on time| +4|AAAAAAAAEAAAAAAA|Not the product that was ordred| +5|AAAAAAAAFAAAAAAA|Parts missing| +6|AAAAAAAAGAAAAAAA|Does not work with a product that I have| +7|AAAAAAAAHAAAAAAA|Gift exchange| +8|AAAAAAAAIAAAAAAA|Did not like the color| +9|AAAAAAAAJAAAAAAA|Did not like the model| +10|AAAAAAAAKAAAAAAA|Did not like the make| +11|AAAAAAAALAAAAAAA|Did not like the warranty| +12|AAAAAAAAMAAAAAAA|No service location in my area| +13|AAAAAAAANAAAAAAA|Found a better price in a store| +14|AAAAAAAAOAAAAAAA|Found a better extended warranty in a store| +15|AAAAAAAAPAAAAAAA|Not working any more| +16|AAAAAAAAABAAAAAA|Did not fit| +17|AAAAAAAABBAAAAAA|Wrong size| +18|AAAAAAAACBAAAAAA|Lost my job| +19|AAAAAAAADBAAAAAA|unauthoized purchase| +20|AAAAAAAAEBAAAAAA|duplicate purchase| +21|AAAAAAAAFBAAAAAA|its is a boy| +22|AAAAAAAAGBAAAAAA|it is a girl| +23|AAAAAAAAHBAAAAAA|reason 23| +24|AAAAAAAAIBAAAAAA|reason 24| +25|AAAAAAAAJBAAAAAA|reason 25| +26|AAAAAAAAKBAAAAAA|reason 26| +27|AAAAAAAALBAAAAAA|reason 27| +28|AAAAAAAAMBAAAAAA|reason 28| +29|AAAAAAAANBAAAAAA|reason 29| +30|AAAAAAAAOBAAAAAA|reason 31| +31|AAAAAAAAPBAAAAAA|reason 31| +32|AAAAAAAAACAAAAAA|reason 32| +33|AAAAAAAABCAAAAAA|reason 33| +34|AAAAAAAACCAAAAAA|reason 34| +35|AAAAAAAADCAAAAAA|reason 35| diff --git a/spark-connector/spark-common/src/test/resources/data/tpcds/ship_mode/ship_mode.dat b/spark-connector/spark-common/src/test/resources/data/tpcds/ship_mode/ship_mode.dat new file mode 100644 index 00000000000..fdda00759b7 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/data/tpcds/ship_mode/ship_mode.dat @@ -0,0 +1,20 @@ +1|AAAAAAAABAAAAAAA|EXPRESS|AIR|UPS|YvxVaJI10| +2|AAAAAAAACAAAAAAA|NEXT DAY|AIR|FEDEX|ldhM8IvpzHgdbBgDfI| +3|AAAAAAAADAAAAAAA|OVERNIGHT|AIR|AIRBORNE|6Hzzp4JkzjqD8MGXLCDa| +4|AAAAAAAAEAAAAAAA|TWO DAY|AIR|USPS|UaAJjKDnL4gTOqbpj| +5|AAAAAAAAFAAAAAAA|LIBRARY|AIR|DHL|HVDFCcQ| +6|AAAAAAAAGAAAAAAA|REGULAR|SURFACE|TBS|Ek| +7|AAAAAAAAHAAAAAAA|EXPRESS|SURFACE|ZHOU|hGoF18SLDDPBj| +8|AAAAAAAAIAAAAAAA|NEXT DAY|SURFACE|ZOUROS|yVfotg7Tio3MVhBg6Bkn| +9|AAAAAAAAJAAAAAAA|OVERNIGHT|SURFACE|MSC|Xjy3ZPuiDjzHlRx14Z3| +10|AAAAAAAAKAAAAAAA|TWO DAY|SURFACE|LATVIAN|P7FBIt8yd| +11|AAAAAAAALAAAAAAA|LIBRARY|SURFACE|ALLIANCE|OrDuVy2H| +12|AAAAAAAAMAAAAAAA|REGULAR|SEA|ORIENTAL|GNJr3g5i7oorKqtX| +13|AAAAAAAANAAAAAAA|EXPRESS|SEA|BARIAN|qENFQ| +14|AAAAAAAAOAAAAAAA|NEXT DAY|SEA|BOXBUNDLES|A5BYO1qH8HGTTN| +15|AAAAAAAAPAAAAAAA|OVERNIGHT|SEA|GREAT EASTERN|uukTktPYycct8| +16|AAAAAAAAABAAAAAA|TWO DAY|SEA|DIAMOND|2mM8l| +17|AAAAAAAABBAAAAAA|LIBRARY|SEA|RUPEKSA|5FKNB0j8aaqTB| +18|AAAAAAAACBAAAAAA|REGULAR|BIKE|GERMA|I3uCelXtjP| +19|AAAAAAAADBAAAAAA|EXPRESS|BIKE|HARMSTORF|fop0bcSd91J26IVpR| +20|AAAAAAAAEBAAAAAA|NEXT DAY|BIKE|PRIVATECARRIER|O9V6oF8RJnLMmZYd1| diff --git a/spark-connector/spark-common/src/test/resources/data/tpcds/store/store.dat b/spark-connector/spark-common/src/test/resources/data/tpcds/store/store.dat new file mode 100644 index 00000000000..078bafdd1e0 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/data/tpcds/store/store.dat @@ -0,0 +1,12 @@ +1|AAAAAAAABAAAAAAA|1997-03-13||2451189|ought|245|5250760|8AM-4PM|William Ward|2|Unknown|Enough high areas stop expectations. Elaborate, local is|Charles Bartley|1|Unknown|1|Unknown|767|Spring |Wy|Suite 250|Midway|Williamson County|TN|31904|United States|-5|0.03| +2|AAAAAAAACAAAAAAA|1997-03-13|2000-03-12||able|236|5285950|8AM-4PM|Scott Smith|8|Unknown|Parliamentary candidates wait then heavy, keen mil|David Lamontagne|1|Unknown|1|Unknown|255|Sycamore |Dr.|Suite 410|Midway|Williamson County|TN|31904|United States|-5|0.03| +3|AAAAAAAACAAAAAAA|2000-03-13|||able|236|7557959|8AM-4PM|Scott Smith|7|Unknown|Impossible, true arms can treat constant, complete w|David Lamontagne|1|Unknown|1|Unknown|877|Park Laurel|Road|Suite T|Midway|Williamson County|TN|31904|United States|-5|0.03| +4|AAAAAAAAEAAAAAAA|1997-03-13|1999-03-13|2451044|ese|218|9341467|8AM-4PM|Edwin Adams|4|Unknown|Events would achieve other, eastern hours. Mechanisms must not eat other, new org|Thomas Pollack|1|Unknown|1|Unknown|27|Lake |Ln|Suite 260|Midway|Williamson County|TN|31904|United States|-5|0.03| +5|AAAAAAAAEAAAAAAA|1999-03-14|2001-03-12|2450910|anti|288|9078805|8AM-4PM|Edwin Adams|8|Unknown|Events would achieve other, eastern hours. Mechanisms must not eat other, new org|Thomas Pollack|1|Unknown|1|Unknown|27|Lee 6th|Court|Suite 80|Fairview|Williamson County|TN|35709|United States|-5|0.03| +6|AAAAAAAAEAAAAAAA|2001-03-13|||cally|229|9026222|8AM-4PM|Edwin Adams|10|Unknown|Events would achieve other, eastern hours. Mechanisms must not eat other, new org|Thomas Pollack|1|Unknown|1|Unknown|220|6th |Lane|Suite 140|Midway|Williamson County|TN|31904|United States|-5|0.03| +7|AAAAAAAAHAAAAAAA|1997-03-13|||ation|297|8954883|8AM-4PM|David Thomas|9|Unknown|Architects coul|Thomas Benton|1|Unknown|1|Unknown|811|Lee |Circle|Suite T|Midway|Williamson County|TN|31904|United States|-5|0.01| +8|AAAAAAAAIAAAAAAA|1997-03-13|2000-03-12||eing|278|6995995|8AM-4PM|Brett Yates|2|Unknown|Various bars make most. Difficult levels introduce at a boots. Buildings welcome only never el|Dean Morrison|1|Unknown|1|Unknown|226|12th |Lane|Suite D|Fairview|Williamson County|TN|35709|United States|-5|0.08| +9|AAAAAAAAIAAAAAAA|2000-03-13|||eing|271|6995995|8AM-4PM|Brett Yates|2|Unknown|Formal, psychological pounds relate reasonable, young principles. Black, |Dean Morrison|1|Unknown|1|Unknown|226|Hill |Boulevard|Suite 190|Midway|Williamson County|TN|31904|United States|-5|0.08| +10|AAAAAAAAKAAAAAAA|1997-03-13|1999-03-13||bar|294|9294113|8AM-4PM|Raymond Jacobs|8|Unknown|Little expectations include yet forward meetings.|Michael Wilson|1|Unknown|1|Unknown|175|4th |Court|Suite C|Midway|Williamson County|TN|31904|United States|-5|0.06| +11|AAAAAAAAKAAAAAAA|1999-03-14|2001-03-12||ought|294|9294113|8AM-4PM|Raymond Jacobs|6|Unknown|Mysterious employe|Michael Wilson|1|Unknown|1|Unknown|175|Park Green|Court|Suite 160|Midway|Williamson County|TN|31904|United States|-5|0.11| +12|AAAAAAAAKAAAAAAA|2001-03-13|||ought|294|5219562|8AM-12AM|Robert Thompson|6|Unknown|Events develop i|Dustin Kelly|1|Unknown|1|Unknown|337|College |Boulevard|Suite 100|Fairview|Williamson County|TN|31904|United States|-5|0.01| diff --git a/spark-connector/spark-common/src/test/resources/data/tpcds/store_returns/store_returns.dat b/spark-connector/spark-common/src/test/resources/data/tpcds/store_returns/store_returns.dat new file mode 100644 index 00000000000..b7050c90cd1 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/data/tpcds/store_returns/store_returns.dat @@ -0,0 +1,2875 @@ +2451984|46418|4553|67006|793022|4033|17397|7|19|1|51|37.23|3.35|40.58|55.28|714.00|0.74|17.51|18.98|772.63| +2451822|47480|10993|67006|1082163|7157|49751|1|29|1|43|4009.32|120.27|4129.59|28.23|0.00|3448.01|5.61|555.70|148.50| +2451653|37700|7654|68284|289468|3067|21819|10|35|2|7|249.48|2.49|251.97|11.50|6.23|234.51|8.83|6.14|20.22| +2452682|38957|17263|44047|793786|6401|10663|2|15|3|3|227.37|6.82|234.19|9.23|33.42|125.05|51.16|51.16|49.47| +2452682|41617|10693|44047|193750|2351|8152|8|9|3|23|1273.05|25.46|1298.51|46.15|406.18|954.78|57.28|260.99|477.79| +2452688|60645|16869|44047|1047081|2455|45076|2|1|3|1|35.14|2.45|37.59|79.00|0.00|9.13|5.20|20.81|81.45| +2451598|40772|5515|38814|722383|1497|44164|2|35|4|18|63.54|1.27|64.81|1.26|10.80|33.67|2.68|27.19|13.33| +2451521|33881|6988|38814|925501|3196|34491|2|21|4|12|18.60|1.48|20.08|46.48|270.60|10.04|8.04|0.52|318.56| +2451628|34073|12650|13246|1150235|3185|36935|8|27|5|58|347.42|6.94|354.36|97.98|331.18|107.70|14.38|225.34|436.10| +2452165|54713|10895|53029|492488|6806|46724|10|33|6|1|61.81|3.70|65.51|68.44|44.79|54.39|0.14|7.28|116.93| +2452319|50204|4795|69954|1182421|560|46299|8|35|7|72|809.28|24.27|833.55|88.17|51.12|194.22|442.84|172.22|163.56| +2452065|40372|13803|4451|178772|175|29625|2|26|10|48|3960.48|0.00|3960.48|39.22|65.76|1980.24|1960.43|19.81|104.98| +2452575|42311|1315|48900|727030|4636|27089|1|31|11|36|1639.44|81.97|1721.41|96.21|712.80|147.54|1163.68|328.22|890.98| +2451378|48500|6148|20820|201438|6076|8714|7|32|12|27|417.42|4.17|421.59|62.17|2400.30|375.67|37.57|4.18|2466.64| +2451329|44694|15440|20820|1351118|2652|34633|10|18|12|46|1208.42|48.33|1256.75|69.04|887.80|96.67|355.76|755.99|1005.17| +2451014|29493|4430|54588|130689|2790|38138|1|30|13|58|40.60|3.24|43.84|1.13|312.62|27.20|0.26|13.14|316.99| +2450926|48082|7723|15144|541501|527|36400|8|19|13|1|83.51|7.51|91.02|81.26|20.31|73.48|8.42|1.61|109.08| +2450924|48257|2108|15144|277151|1877|47050|1|31|13|6|350.40|14.01|364.41|20.61|247.98|255.79|62.44|32.17|282.60| +||13845|19409|829696||||34|14||708.18|49.57|757.75|||559.46|||| +2451933|41472|8690|62835|747820|2733|14446|1|12|15|1|9.43|0.75|10.18|35.62|48.33|8.48|0.61|0.34|84.70| +2450914|51589|17246|32222|1815885|5148|2411|7|32|16|24|2406.00|72.18|2478.18|19.02|349.20|505.26|722.28|1178.46|440.40| +2450924|34013|9769|51139|524921|3159|22950|10|24|16|15|181.65|10.89|192.54|98.09|60.45|74.47|8.57|98.61|169.43| +2451547|44125|11584|63292|1532238|2533|16828|8|33|18|56|199.36|9.96|209.32|57.84|89.04|95.69|14.51|89.16|156.84| +2451491|44663|17995|63292|187471|1924|34424|2|19|18|13|419.25|12.57|431.82|58.46|226.72|37.73|343.36|38.16|297.75| +2451376|56165|3133|51104|1392417|1141|34312|8|27|19|14|359.24|32.33|391.57|13.77|517.30|283.79|45.27|30.18|563.40| +2451493|50425|3376|23762|1325860|758|28856|8|9|19|78|2391.48|47.82|2439.30|78.85|2602.86|2032.75|243.93|114.80|2729.53| +2451280|33889|5170|20626|854970|2561|13305|1|30|21|27|1778.76|53.36|1832.12|73.43|495.72|160.08|1068.32|550.36|622.51| +2451277|40485|9985|64935|1685822|6158|42494|10|33|22|17|35.02|0.00|35.02|22.87|195.16|14.35|2.68|17.99|218.03| +2451231|29254|16922|64935|1247881|5718|43555|10|14|22|6|113.94|9.11|123.05|66.52|53.40|37.60|57.25|19.09|129.03| +2452771|51900|15403|21669|1890707|5392|30|10|21|23|80|5883.20|294.16|6177.36|52.40|2848.80|4824.22|328.28|730.70|3195.36| +2452213|40442|6751|15069|1359322|2796|22270|4|31|24|11|28.49|0.00|28.49|69.29|29.81|6.26|1.33|20.90|99.10| +2452305|60624|16483|15069|1553144|6578|8765|8|16|24|78|1099.02|54.95|1153.97|27.69|328.38|1033.07|58.03|7.92|411.02| +2451053|35984|7874|70838|1564003|1743|49883|4|8|25|18|42.12|2.52|44.64|57.38|6.12|37.48|2.04|2.60|66.02| +2451079|50088|14635|70838|77813|3196|24932|4|13|25|39|772.59|46.35|818.94|28.93|178.23|347.66|50.99|373.94|253.51| +2451966|49139|2137|32780|637321|1532|40590|2|9|27|84|382.20|0.00|382.20|97.85|154.56|259.89|13.45|108.86|252.41| +2451964|49463|5831|32780|178335|4892|45896|8|32|27|22|59.84|1.79|61.63|38.22|137.72|43.68|6.62|9.54|177.73| +2451259|35185|12223|10631|1723855|885|1868|10|13|29|6|5.04|0.45|5.49|85.50|15.96|3.83|0.03|1.18|101.91| +2451174|32214|12068|10631|135738|5274|40010|7|14|29|22|41.80|3.76|45.56|18.82|985.16|17.13|24.17|0.50|1007.74| +2451983|33769|11348|90120|222350|2461|44125|7|17|32|41|2092.23|167.37|2259.60|14.26|987.28|941.50|149.59|1001.14|1168.91| +2451392|48895|16324|47971|934775|429|24883|4|6|34|13|208.78|16.70|225.48|77.95|223.73|198.34|3.65|6.79|318.38| +2452196|33535|5179|93958|539558|6809|26107|7|26|35|60|144.00|0.00|144.00|36.15|2167.20|87.84|39.87|16.29|2203.35| +2452306|40321|15001|72008|1243218|3988|12711|8|24|36|45|999.45|79.95|1079.40|29.51|35.10|499.72|439.76|59.97|144.56| +2452317|53999|15645|72008|882385|||2|9|36|||3.49||||7.49|13.58||| +2452279|41984|16421|33722|465142|3315|34735|1|20|36|7|541.31|43.30|584.61|71.10|241.36|519.65|10.18|11.48|355.76| +2452128|30131|2997|96252|491912|5028|36144|2|21|37|85|8562.90|513.77|9076.67|8.17|366.35|4195.82|3493.66|873.42|888.29| +2451271|44173|10|87890|1681246|5780|32714|1|14|41|17|1327.36|79.64|1407.00|49.73|558.79|66.36|731.38|529.62|688.16| +2451092|43741|9740|49423|1577815|3651|9487|4|24|42|13|422.24|8.44|430.68|48.27|551.20|329.34|15.79|77.11|607.91| +2451033|54994|619|22229|1611739|3649|27912|10|12|42|43|1240.55|0.00|1240.55|86.84|68.80|719.51|359.51|161.53|155.64| +2451073|52318|1436|70152|134560|4215|10223|4|6|42|13|475.54|9.51|485.05|73.37|317.07|128.39|197.87|149.28|399.95| +2452295|39258|11095|16606|1581204|3389|3621|7|16|45|39|2703.87|243.34|2947.21|59.97|386.10|1487.12|790.88|425.87|689.41| +2451354|60360|17150|34598|194443|4161|45832|7|11|46|47|3742.14|149.68|3891.82|24.47|379.29|3442.76|59.87|239.51|553.44| +|59918|8065|||3068||4||46|38|||453.29|51.61|207.48|104.60|86.12||276.52| +2451177|58870|7214|83903|1256944|382|39373|4|2|47|1|8.12|0.73|8.85|19.08|7.43|3.81|0.77|3.54|27.24| +2452738|29363|12468|56843|1862359|2275|25391|10|31|48|5|199.35|17.94|217.29|11.83|112.95|191.37|0.71|7.27|142.72| +2452745|44637|6684|21202|1047097|5374|11251|7|8|48|10|392.20|27.45|419.65|10.16|116.00|152.95|224.89|14.36|153.61| +2451892|35637|6805|63590|103157|6724|45091|10|30|49|21|496.23|29.77|526.00|48.23|31.92|282.85|142.96|70.42|109.92| +2451769|38709|10133|83517|945253|3357|11061|10|27|51|5|172.10|5.16|177.26|22.72|64.20|170.37|1.59|0.14|92.08| +2451102|32082|16646|45831|173053|4865|10230|7|19|52|13|911.04|0.00|911.04|21.65|378.17|619.50|253.63|37.91|399.82| +2451088|55568|17456|4810|1866272|3734|33196|10|20|52|25|113.50|1.13|114.63|62.63|125.50|30.64|25.68|57.18|189.26| +2451573|36729|11348|87672|508248|5690|39885|4|19|53|1|2.23|0.08|2.31|26.56|1.48|0.20|0.89|1.14|28.12| +2452729|48880|16059|48157|1042828|4295|42162|4|11|54|11|95.48|3.81|99.29|93.36|8.36|34.37|36.05|25.06|105.53| +2452040|43895|17677|93297|869419|5541|42463|4|17|55|11|252.67|12.63|265.30|20.45|63.14|194.55|42.42|15.70|96.22| +2452339|46469|1117|22967|972821|2228|82|1|23|60|78|2733.90|54.67|2788.57|60.37|0.00|683.47|1558.32|492.11|115.04| +2452254|40433|13559|98197|1420275|5589|27703|1|6|62|6|446.76|26.80|473.56|67.62|213.66|58.07|15.54|373.15|308.08| +2452701|35838|11370|96129|305648||33488||13|63|||11.73||||172.06|||568.77| +2452596|29520|13339|96129|1086498|2400|48302|4|24|63|16|78.56|0.00|78.56|96.23|49.60|43.99|21.43|13.14|145.83| +2452604|36240|5217|96129|479318|1264|13610|7|12|63|28|216.16|19.45|235.61|57.17|242.76|108.08|104.83|3.25|319.38| +2452593|55466|15867|48017|145292|437|20782|2|4|64|19|583.49|40.84|624.33|54.92|392.35|280.07|203.29|100.13|488.11| +2452556|56489|17586|48017|492805|2019|4033|1|26|64|34|2325.60|0.00|2325.60|57.60|1987.30|1046.52|575.58|703.50|2044.90| +2452437|54978|3090|48017|569529|3161|35193|8|20|64|66|6903.60|0.00|6903.60|76.02|2987.82|6213.24|352.08|338.28|3063.84| +2452346|53160|6507|66766|1603765|4015|25465|10|28|66|15|630.75|25.23|655.98|2.26|336.30|491.98|80.48|58.29|363.79| +2452555|43088|2133|94624|529565|2324|9930|4|3|67|41|594.09|23.76|617.85|40.24|1063.54|160.40|56.37|377.32|1127.54| +2451258|43812|12724|2564|496988|6773|40205|7|10|68|21|178.50|5.35|183.85|96.73|53.76|82.11|75.18|21.21|155.84| +2451256|50305|5858|91942|875823|2104|34738|8|30|71|2|13.48|0.13|13.61|91.25|2.92|6.33|5.14|2.01|94.30| +2451294|44684|9790|85239|1910912|233|14001|7|23|71|25|188.00|16.92|204.92|44.05|51.25|150.40|6.39|31.21|112.22| +2451329|29997|8887|85239|195885|3539|47146|10|9|71|5|164.65|1.64|166.29|75.66|205.80|74.09|12.67|77.89|283.10| +2451829|50278|6671|60602|1271817|6019|43468|8|6|72|3|166.41|1.66|168.07|1.14|64.92|59.90|62.84|43.67|67.72| +2451174|57809|3175|3337|767720|4103|34886|2|5|74|4|36.32|0.00|36.32|38.54|42.96|12.71|6.13|17.48|81.50| +2452279|52274|465|53531|78088|234|21580|7|17|76|1|76.51|6.88|83.39|66.92|30.28|56.61|19.70|0.20|104.08| +2452242|52421|5529|53531|1630980|4166|31352|1|1|76|15|796.05|31.84|827.89|91.64|176.85|278.61|243.19|274.25|300.33| +2451201|32557|2713|15102|622349|4796|48043|7|22|77|10|569.40|45.55|614.95|45.00|621.20|529.54|19.93|19.93|711.75| +2451155|53923|12706|59529|1758891|3500|3309|1|20|78|13|1077.83|64.66|1142.49|77.87|576.81|506.58|457.00|114.25|719.34| +2452290|54248|15888|62017|382863|6073|48483|1|10|79|34|797.64|71.78|869.42|57.39|1296.08|71.78|79.84|646.02|1425.25| +2451154|51844|13717|17368|1254797|4380|13916|7|33|80|64|6830.08|68.30|6898.38|85.70|148.48|6625.17|127.04|77.87|302.48| +2451281|48144|9319|24625|459305|510|38915|1|32|81|22|1416.58|42.49|1459.07|16.86|849.86|1388.24|23.52|4.82|909.21| +2451214|55537|7166|24625|944226|463|45147|2|31|81|5|0.00|0.00|0.00|94.53|24.55|0.00|0.00|0.00|119.08| +2451369||7862|19512||3587||8||82||1491.00||||545.30||735.06||707.47| +2451471|50769|3538|20896|1082287|645|27356|1|13|82|8|847.84|42.39|890.23|40.52|77.04|576.53|208.90|62.41|159.95| +2451533|34774|1700|20896|1427925|1401|35230|7|22|82|43|1399.65|55.98|1455.63|74.63|447.63|839.79|156.76|403.10|578.24| +2452677|50033|12378|42678|842053|5314|42175|1|25|83|21|1151.01|23.02|1174.03|28.22|146.16|115.10|973.75|62.16|197.40| +2452530|44806|6417|57091|1034463|114|40165|1|10|84|5|323.00|19.38|342.38|5.27|171.25|261.63|43.57|17.80|195.90| +2452550|39316|17155|57091|222509|5658|20007|7|15|84|63|1733.76|156.03|1889.79|98.99|1155.42|208.05|671.31|854.40|1410.44| +2452550|56385|10771|57091|1844056|137|11818|10|28|84|52|112.84|0.00|112.84|47.55|130.00|44.00|68.84|0.00|177.55| +2452743|53098|10341|1560|1435175|1237|32452|4|12|86|25|50.00|4.00|54.00|95.65|2.50|29.00|16.38|4.62|102.15| +2452805|37053|13413|56359|705875|2076|29430|10|30|86|12|100.44|3.01|103.45|81.54|343.32|50.22|6.52|43.70|427.87| +2451604||13756||1393955|||||87|17||||35.41||213.43||0.17|| +2452317|32925|17037|34272|1361123|1963|16403|1|13|88|34|146.20|0.00|146.20|22.87|1376.66|86.25|23.38|36.57|1399.53| +2452260|58521|14171|81110|341664|7018|6372|7|13|88|70|1168.30|70.09|1238.39|37.61|1168.30|198.61|96.96|872.73|1276.00| +2451907|56892|17600|74994|751267|6036|8052|8|7|89|1|131.14|0.00|131.14|75.40|53.76|111.46|17.51|2.17|129.16| +2452520|42422|17697|34925|1402848|111|45374|1|35|91|5|0.00|0.00|0.00|76.66|291.95|0.00|0.00|0.00|368.61| +2451501|42569|11077|52989|298152|3782|18612|8|15|93|30|1132.50|0.00|1132.50|92.76|653.40|192.52|28.19|911.79|746.16| +2451649|54166|6658|52989|1305474|3494|11161|10|35|93|19|306.66|0.00|306.66|6.06|229.90|251.46|29.25|25.95|235.96| +2452596|60572|4221|73789|1136495|3779|45863|10|29|94|59|2297.46|206.77|2504.23|23.96|2567.68|1975.81|234.80|86.85|2798.41| +2452757|52151|17262|73789|1126901|523|11698|8|5|94|67|4172.09|125.16|4297.25|12.36|3040.46|1209.90|2014.28|947.91|3177.98| +2451198|43902|2479|87668|640397|7173|28302|7|27|95|8|625.52|43.78|669.30|90.09|200.16|431.60|166.77|27.15|334.03| +2451175|43653|3580|20113|1244878|4526|19021|1|32|95|27|1081.89|54.09|1135.98|64.15|704.70|714.04|353.13|14.72|822.94| +2451153|46437|13364|20113|1183342|4835|35528|4|4|95|4|12.68|0.76|13.44|28.93|10.04|0.12|11.17|1.39|39.73| +2451193|41206|15115|20113|969420|6552|27637|1|29|95|25|1100.50|66.03|1166.53|84.08|136.00|429.19|543.76|127.55|286.11| +2452094|29842|12387|6705|1353101|6414|10174|8|7|97|23|920.23|64.41|984.64|98.19|136.62|266.86|111.07|542.30|299.22| +2452192|51078|1835|6705|1476583|3588|37393|2|18|97|19|999.02|79.92|1078.94|46.46|253.84|849.16|52.45|97.41|380.22| +2452179|58535|14611|6705|737335|1006|7030|2|3|97|58|2734.12|27.34|2761.46|91.12|1279.48|1531.10|24.06|1178.96|1397.94| +2452471|33938|16521|2320|1744761|2561|37561|8|1|98|24|276.00|2.76|278.76|33.91|96.48|49.68|54.31|172.01|133.15| +2452562|58129|12801|67232|1537922|2749|10376|8|3|99|22|2465.32|49.30|2514.62|73.55|183.48|640.98|529.05|1295.29|306.33| +2452442|36315|12343|15875|678651|7120|38342|1|30|100|14|912.66|73.01|985.67|20.45|108.64|182.53|518.39|211.74|202.10| +2452454|35135|2335|15875|985105|883|4614|10|35|100|4|6.72|0.47|7.19|99.89|80.88|3.89|0.25|2.58|181.24| +2451380|45137|16136|65242|1063484|70|37409|8|11|101|28|382.20|26.75|408.95|50.66|329.00|194.92|125.47|61.81|406.41| +2451232|42391|14276|80314|261760|4006|47052|10|19|101|27|1375.38|123.78|1499.16|50.57|663.93|357.59|508.89|508.90|838.28| +2451400|38304|4264|65242|256452|6873|49247|7|29|101|8|62.24|1.24|63.48|3.81|77.12|20.53|21.68|20.03|82.17| +2451398|52092|5989|81098|515174|1697|30573|7|17|101|13|2058.68|102.93|2161.61|42.34|750.75|720.53|1298.00|40.15|896.02| +2452582|37192|5511|28341|1766974|1960|21793|4|2|102|29|2850.70|0.00|2850.70|43.15|617.12|1510.87|897.68|442.15|660.27| +2452138|58093|15841|69235|403639|764|6642|2|28|103|3|254.55|0.00|254.55|100.00|45.57|53.45|106.58|94.52|145.57| +2451687|60314|3218|95884|1431450|6216|25382|7|19|104|1|20.32|0.00|20.32|72.90|8.62|7.51|3.33|9.48|81.52| +2452346|36025|12691|54634|390885|6456|25180|10|18|105|15|72.75|0.00|72.75|24.31|291.15|7.27|13.75|51.73|315.46| +2452025|54705|7933|75969|1677481|6193|24552|10|16|106|24|825.12|41.25|866.37|26.21|296.40|57.75|168.82|598.55|363.86| +2451980|58642|2871|77533|663032|1262|28692|10|14|106|6|179.70|0.00|179.70|99.32|129.00|41.33|136.98|1.39|228.32| +2452104|29986|12267|77533|757785|6591|23797|7|16|106|9|241.38|14.48|255.86|78.57|176.76|16.89|213.26|11.23|269.81| +2452081|58521|6363|5780|645805|7163|121|7|9|107|56|65.52|4.58|70.10|69.13|109.20|36.03|21.52|7.97|182.91| +2452081|47666|3019|5780|467186|2772|37158|7|15|107|70|180.60|9.03|189.63|4.58|4.90|131.83|48.28|0.49|18.51| +2451539|51625|2060|55322|1741784|1694|28757|1|30|108|15|310.80|12.43|323.23|46.49|188.70|295.26|5.43|10.11|247.62| +2451605|50913|4420|55322|1351722|5238|1554|10|1|108|53|358.81|10.76|369.57|54.16|1650.42|215.28|139.22|4.31|1715.34| +2451664|49113|2305|68136|802523|3350|14177|7|26|109|18|1494.90|104.64|1599.54|35.75|582.48|44.84|333.51|1116.55|722.87| +2452619||16243|57948|478574|||10||111||343.20||363.79|79.04|38.94|65.20||191.82|138.57| +2451162|45758|6806|21253|291015|3506|7644|10|33|112|4|385.92|7.71|393.63|55.43|146.36|366.62|0.57|18.73|209.50| +2451994|53407|10395|44742|1662440|7040|22087|10|1|114|67|1360.10|122.40|1482.50|67.98|1360.10|312.82|586.47|460.81|1550.48| +2451625|35855|6832|57808|1191022|4532|9642|10|22|117|66|178.20|10.69|188.89|32.78|1070.52|105.13|18.99|54.08|1113.99| +2451491|50098|14948|57808|428136|2928|32520|8|16|117|20|1094.60|43.78|1138.38|94.15|785.80|875.68|70.05|148.87|923.73| +2451520|41125|1339|57808|1586067|1266|28418|7|19|117|6|153.30|7.66|160.96|37.26|26.64|82.78|58.53|11.99|71.56| +2451876|50318|11369|20684|303522|5800|29631|4|25|118|66|262.02|7.86|269.88|5.88|3060.42|180.79|61.73|19.50|3074.16| +2451881|36065|8918|20684|859192|3953|13914|1|29|118|1|33.50|2.68|36.18|75.66|5.75|12.73|4.98|15.79|84.09| +|59237|6529|72894||6525||||119|1||0.32|||5.79|||0.56|64.67| +|37473|4897|34199||6513||8||119||2624.05|||7.21||1863.07|||| +|60577|14749|76599||895|17490|7||120|||||32.19||5279.94|1006.12||| +2451216|50692|12632|13005|1750967|6215|28663|4|29|121|8|604.72|6.04|610.76|48.16|341.36|247.93|231.91|124.88|395.56| +2451337|37394|15952|24145|50346|5847|44948|7|28|122|16|1813.12|126.91|1940.03|49.64|272.80|1541.15|231.17|40.80|449.35| +2451978|51632|9965|92596|1471447|1382|24870|4|28|125|7|629.30|6.29|635.59|9.77|0.00|125.86|407.78|95.66|16.06| +2451885|46894|4847|92596|724349|1805|36508|4|29|125|2|20.22|1.81|22.03|4.01|10.10|3.23|16.99|0.00|15.92| +2452026|32161|491|92596|737936|1981|14473|10|18|125|3|279.30|11.17|290.47|43.08|194.67|223.44|19.55|36.31|248.92| +2451641|38484|9379|70489|590480|6471|22862|7|6|129|7|512.19|10.24|522.43|75.93|240.03|507.06|3.59|1.54|326.20| +2451643|29747|12872|548|333880|4126|9972|10|27|129|8|68.32|6.14|74.46|89.41|44.00|19.12|5.90|43.30|139.55| +2451177|44766|11938|61469|971519|5888|36657|4|3|131|7|114.24|5.71|119.95|26.39|85.26|59.40|10.96|43.88|117.36| +2451210|54853|15422|61469|363266|3641|10416|4|1|131|3|72.27|5.05|77.32|75.28|53.31|51.31|3.98|16.98|133.64| +2451678|32490|15655|33461|1312529|947|46965|4|21|132|4|220.44|0.00|220.44|84.15|13.36|158.71|19.75|41.98|97.51| +2451898|54229|13343|60708|1060233|636|47439|8|18|133|7|47.32|2.83|50.15|41.49|59.99|37.38|9.64|0.30|104.31| +2451891|38123|9365|60708|1691883|3402|10568|10|30|133|20|1473.60|14.73|1488.33|62.53|834.00|176.83|661.35|635.42|911.26| +2452287|39904|23|94268|1508371|3627|20083|10|2|134|7|543.20|16.29|559.49|93.54|225.47|239.00|121.68|182.52|335.30| +2452366|52737|8571|58602|1916363|3390|37170|8|24|134|4|287.96|17.27|305.23|27.17|69.24|161.25|31.67|95.04|113.68| +2451951|52519|17537|43197|221001|685|47569|4|7|135|4|148.56|8.91|157.47|79.33|8.84|11.88|32.80|103.88|97.08| +2451834|56837|7445|43197|1276975|4044|19113|8|8|135|51|915.45|9.15|924.60|62.12|1116.39|860.52|3.84|51.09|1187.66| +2451802|40414|16322|43197|1209613|1712|8211|8|1|135|42|1830.78|164.77|1995.55|90.33|837.48|201.38|619.17|1010.23|1092.58| +2451853|44570|12095|43197|274389|5272|25755|1|29|135|3|49.08|2.45|51.53|73.02|74.43|45.15|3.89|0.04|149.90| +2452665|49707|4411|17595|1243564|1271|43731|10|3|136|24|206.64|10.33|216.97|68.24|34.80|181.84|7.19|17.61|113.37| +2452526|29443|12870|95616|505525|6187|34449|2|20|136|16|1042.40|83.39|1125.79|28.11|585.44|114.66|148.43|779.31|696.94| +2452580|56313|17853|17595|380160|519|42078|8|26|136|30|140.10|7.00|147.10|30.71|85.50|49.03|61.01|30.06|123.21| +2452692|38238|1975|17595|144868|6388|39508|4|21|136|2|223.84|4.47|228.31|30.01|115.96|0.00|17.90|205.94|150.44| +2451583|48984|5059|83850|1435146|6470|27394|10|5|138|46|749.80|59.98|809.78|72.07|2191.44|269.92|86.37|393.51|2323.49| +2452092|35896|4613|59057|377706|3934|3809|4|12|139|47|53.11|1.06|54.17|4.81|0.00|0.00|19.11|34.00|5.87| +2452179|34422|10853|3684|326360|1036|32912|7|32|139|2|35.52|2.84|38.36|4.35|26.12|3.90|4.74|26.88|33.31| +2451941|59821|5125|35668|822547|98|11202|8|19|142|22|3005.20|60.10|3065.30|19.61|1066.34|2404.16|510.88|90.16|1146.05| +||1845|20474||6967|29147||5|143|||10.32|526.44||186.32||2.16|106.23|212.59| +2452628|51516|2721|20474|419295|1049|3203|7|28|143|65|7959.90|397.99|8357.89|8.41|1534.00|1512.38|2579.00|3868.52|1940.40| +2452515|53040|2025|20474|108793|4980|17920|2|20|143|16|416.80|33.34|450.14|90.52|500.16|75.02|85.44|256.34|624.02| +2452656|36173|9079|43861|938923|4385|36694|7|13|143|67|3172.45|285.52|3457.97|7.25|166.83|2347.61|41.24|783.60|459.60| +2452609|45652|16842|20474|634090|1445|41200|10|28|143|59|3881.02|155.24|4036.26|98.44|2678.60|3337.67|440.11|103.24|2932.28| +2451409|43960|5002|13169|97821|4873|725|1|19|144|56|89.60|2.68|92.28|77.11|5.04|8.96|70.96|9.68|84.83| +2451149|55072|15313|18341|1433390|3296|6755|4|35|146|2|110.32|8.82|119.14|92.96|8.70|6.61|74.67|29.04|110.48| +2451756|29837|8150|58065|1001426|3723|11973|7|32|147|29|1360.39|13.60|1373.99|37.39|1331.39|612.17|52.37|695.85|1382.38| +2451041|38715|10153|80402|1164917|874|36384|1|28|148|32|2698.24|161.89|2860.13|94.54|496.00|2077.64|322.71|297.89|752.43| +2451131|40251|17698|65228|653636|5244|6805|8|11|148|1|2.14|0.17|2.31|31.94|6.72|2.05|0.02|0.07|38.83| +2452266|51016|4793|21514|1917336|6983|27477|10|13|150|8|197.92|1.97|199.89|71.57|188.48|180.10|3.20|14.62|262.02| +2452198|47847|1813|21514|1897846|1796|43393|4|20|150|41|2954.05|59.08|3013.13|51.92|1740.04|413.56|1194.03|1346.46|1851.04| +2451219|56145|10490|10380|1049089|5299|14709|1|14|151|7|153.37|0.00|153.37|32.44|371.35|131.89|10.52|10.96|403.79| +2452289|54212|10945|25423|83990|5910|26230|4|22|153|72|373.68|26.15|399.83|30.17|493.20|298.94|45.59|29.15|549.52| +2451898|37916|629|69252|648527|5589|22604|7|11|154|17|697.68|27.90|725.58|92.77|413.44|404.65|82.04|210.99|534.11| +2451979|32671|1349|69252|1603791|2744|3697|10|3|154|1|104.35|8.34|112.69|79.78|19.41|85.56|9.39|9.40|107.53| +2452068|37806|5697|37327|47885|5482|18726|8|25|155|31|2102.42|21.02|2123.44|37.90|685.41|1135.30|512.57|454.55|744.33| +|31453|10897|37327|1011136||20141|2|16|155|21||||20.18|110.67||165.60||| +|60878|12787||1797260|||2||157||1813.70|163.23|1976.93|9.75||344.60|1425.02|44.08|910.78| +2451180|32857|7597|73251|107231|4763|22695|2|32|158|19|0.00|0.00|0.00|42.77|252.32|0.00|0.00|0.00|295.09| +2451322|31910|10351|73251|405400|5669|25138|8|11|158|1|21.04|1.89|22.93|56.66|8.97|9.25|11.08|0.71|67.52| +2451323|42459|14734|73251|344576|2773|16970|1|4|158|43|1989.61|139.27|2128.88|11.32|867.31|1870.23|75.20|44.18|1017.90| +2451268|39527|5659|73251|340902|3899|48896|2|33|158|11|470.14|32.90|503.04|8.15|658.24|286.78|139.35|44.01|699.29| +2451643|49406|11257|34823|1317784|6082|49580|7|21|159|14|70.98|0.70|71.68|9.95|36.26|29.10|28.89|12.99|46.91| +2451978|51375|17173|76901|431116|6385|6854|4|12|160|38|2981.10|149.05|3130.15|5.91|921.12|119.24|1831.59|1030.27|1076.08| +2451959|37466|14683|76901|544948|7192|28589|8|7|160|1|8.95|0.80|9.75|63.20|7.05|1.52|1.33|6.10|71.05| +2452114|39179|11663|76901|1258210|850|40290|8|8|160|27|684.99|41.09|726.08|24.56|1272.24|335.64|153.71|195.64|1337.89| +2451877|35669|6955|18006|203858|6051|6644|4|15|162|1|24.87|0.99|25.86|32.63|45.76|7.95|14.72|2.20|79.38| +2451653|37503|11470|27551|1710321|5490|28706|8|16|163|21|1736.49|17.36|1753.85|99.03|741.09|764.05|233.38|739.06|857.48| +2450982|33721|2710|30938|430705|1225|32145|10|2|165|9|622.17|55.99|678.16|25.13|23.85|236.42|3.85|381.90|104.97| +2451026|52285|1873|22802|840053|151|23049|2|18|165|1|0.35|0.01|0.36|2.84|2.13|0.29|0.03|0.03|4.98| +2451307|52800|9394|19860|738811|5339|27067|1|18|166|3|63.69|4.45|68.14|86.41|60.15|25.47|20.63|17.59|151.01| +2452535|61107|8539|74161|225320|3671|20469|7|18|168|8|423.28|38.09|461.37|8.76|413.60|364.02|24.29|34.97|460.45| +2451224|50581|17704|68742|1412626|6794|20962|4|16|169|51|1704.93|34.09|1739.02|39.70|826.71|340.98|832.00|531.95|900.50| +|40309|2393||781989|2795|7601|8||171|53|112.89|||61.30|27.03|24.83|37.86|50.20|| +2451886|56059|3613|23370|480457|3591|33478|1|17|171|49|6429.29|64.29|6493.58|44.97|1641.50|2571.71|887.24|2970.34|1750.76| +2451108|36468|268|97090|248980|5595|41763|10|18|173|88|4563.68|273.82|4837.50|37.08|2990.24|4198.58|204.45|160.65|3301.14| +2451205|32346|7405|97090|452633|647|44214|10|2|173|20|60.80|0.00|60.80|65.76|219.40|57.76|0.76|2.28|285.16| +2451204|43512|16828|97090|1715028|4802|28176|2|6|173|51|221.85|8.87|230.72|99.81|106.08|181.91|26.75|13.19|214.76| +2450995|45760|11414|80181|1640715|5203|30444|8|16|174|16|0.00|0.00|0.00|13.54|95.04|0.00|0.00|0.00|108.58| +2452770|55433|4794|68727|1546209|2247|5085|1|4|175|17|505.92|15.17|521.09|39.09|375.36|151.77|145.20|208.95|429.62| +2452810|52619|16188|68727|1698802|4972|37542|2|17|175|12|132.24|9.25|141.49|15.36|68.16|130.91|0.65|0.68|92.77| +2452690|54929|7674|68727|1050305|4626|18947|10|24|175|1|1.60|0.08|1.68|20.87|24.13|0.12|0.74|0.74|45.08| +2452249|59467|13217|5689|332706|1279|41852|1|33|176|1|43.02|0.43|43.45|82.11|14.34|10.32|22.56|10.14|96.88| +2452154|52876|6441|5689|1604859|7190|27921|7|23|176|6|332.10|0.00|332.10|58.91|8.58|6.64|136.69|188.77|67.49| +2452198|45724|17713|5689|565132|4909|47486|7|13|176|67|4692.01|281.52|4973.53|82.39|3853.84|797.64|3582.82|311.55|4217.75| +2452753|43226|13434|3824|1058125|2182|13086|2|27|177|11|320.98|16.04|337.02|80.44|158.62|292.09|22.82|6.07|255.10| +2452664|31193|14281|73190|1843550|4005|42107|10|4|178|16|576.48|23.05|599.53|53.75|227.04|438.12|16.60|121.76|303.84| +2452702|54243|14484|90460|925310|4895|22880|2|18|179|17|256.36|12.81|269.17|82.70|163.20|225.59|7.69|23.08|258.71| +2452655|46986|600|96309|640054|6898|11389|1|22|180|90|1478.70|103.50|1582.20|27.22|3028.50|724.56|143.28|610.86|3159.22| +2452420|59477|10872|65464|1416069|1246|22423|7|27|181|3|196.71|0.00|196.71|13.19|98.34|98.35|84.58|13.78|111.53| +2452401|51853|9432|62189|868451|5048|3245|10|14|182|18|2147.40|0.00|2147.40|81.60|89.46|343.58|775.64|1028.18|171.06| +2452317|53389|5367|75420|1197175|240|27258|4|3|183|67|249.91|9.99|259.90|85.60|249.91|237.41|12.37|0.13|345.50| +2452263|57428|2015|36306|1403445|3737|24876|4|29|183|45|353.70|10.61|364.31|46.65|513.00|332.47|10.82|10.41|570.26| +2452049|31144|17933|22873|638955|4849|48343|1|13|185|35|1902.95|133.20|2036.15|89.25|1076.60|1351.09|380.78|171.08|1299.05| +2452761|36737|7707|66861|674526|1211|26696|1|9|186|10|34.30|0.68|34.98|67.26|19.40|30.52|2.00|1.78|87.34| +2452732|59899|10038|66861|1844453|516|17026|7|16|186|2|111.32|10.01|121.33|47.83|5.70|50.09|27.55|33.68|63.54| +2451222|40375|17546|75594|495598|4681|30964|8|20|189|1|64.00|1.28|65.28|29.07|7.11|46.72|0.69|16.59|37.46| +2452568|44165|10878|88595|1875845|695|14792|2|27|190|28|2010.12|100.50|2110.62|57.14|1382.08|422.12|1079.84|508.16|1539.72| +2452671|57696|3486|81037|1136457|5190|19970|10|8|191|26|326.04|29.34|355.38|4.87|326.04|176.06|68.99|80.99|360.25| +2452142|38365|15871|43792|1893111|6877|10596|1|16|192|52|1645.28|115.16|1760.44|85.24|1062.88|658.11|947.68|39.49|1263.28| +2452014|32532|287|52228|19025|2377|41776|2|25|192|48|525.12|31.50|556.62|77.34|945.60|309.82|88.27|127.03|1054.44| +2452004|40683|7769|52228|1869861|2605|14902|4|18|192|16|890.56|26.71|917.27|68.41|250.40|35.62|205.18|649.76|345.52| +2451950|57081|1883|72739|1659491|3743|3948|7|22|193|26|766.22|68.95|835.17|45.70|423.28|704.92|8.58|52.72|537.93| +2451249|34106|9304|10411|968150|1069|45188|1|16|194|15|1250.40|112.53|1362.93|17.62|688.35|1250.40|0.00|0.00|818.50| +2451320|48825|1588|10411|1282670|2444|16171|1|29|194|51|3315.51|265.24|3580.75|53.85|880.77|1160.42|1185.29|969.80|1199.86| +2451256|40919|5776|10411|1516184|4948|40959|1|7|194|9|294.57|26.51|321.08|69.12|420.93|61.85|102.39|130.33|516.56| +2451187|30712|4276|10411|1446873|6139|39052|10|6|194|21|909.51|36.38|945.89|25.46|314.16|100.04|129.51|679.96|376.00| +2452147|60882|4141|16349|1532879|4658|17237|7|12|195|4|80.48|0.80|81.28|78.19|40.24|17.70|50.85|11.93|119.23| +2452611|46812|17226|29329||2698||||198|||31.72|428.32|||118.98|||187.12| +2452562|39358|15649|27178|1752618|2497|34376|1|7|198|35|81.90|3.27|85.17|42.48|313.60|55.69|4.45|21.76|359.35| +2451654|37096|9169|89187|1485770|2677|34384|2|26|199|34|3046.74|182.80|3229.54|99.22|1256.64|1553.83|970.39|522.52|1538.66| +2451663|29002|1733|89187|968838|6573|21508|1|25|199|83|451.52|22.57|474.09|93.74|424.96|4.51|254.79|192.22|541.27| +2451780|55164|5989|73203|1005114|3213|26912|1|18|200|7|478.73|23.93|502.66|58.43|10.36|52.66|195.99|230.08|92.72| +2451872|49870|5069|73203|1557600|2935|34670|8|5|200|12|189.36|15.14|204.50|31.71|108.60|168.53|19.78|1.05|155.45| +2452158|56842|2357|23043|1112475|5907|8539|1|10|202|5|0.00|0.00|0.00|97.27|10.70|0.00|0.00|0.00|107.97| +2451689|49921|15182|80573|1748167|561|43355|7|15|203|18|1743.48|52.30|1795.78|27.86|830.16|1708.61|34.52|0.35|910.32| +2451649|48348|6331|57984|1640968|807|41893|2|26|207|11|945.89|56.75|1002.64|50.79|75.57|832.38|97.61|15.90|183.11| +2451556|42665|17012|57984|1166760|84|18655|8|20|207|17|364.65|10.93|375.58|4.22|162.01|240.66|95.47|28.52|177.16| +2451413|53827|3314|33457|268561|1265|28369|8|12|208|64|2936.32|146.81|3083.13|49.13|2785.92|704.71|803.37|1428.24|2981.86| +2451479|50197|12422|58476|753241|5087|38980|10|7|208|1|6.65|0.26|6.91|63.06|63.90|1.86|1.05|3.74|127.22| +2451861|59192|296|23683|1045761|6309|41610|10|34|209|49|1098.09|43.92|1142.01|12.88|182.77|900.43|73.13|124.53|239.57| +2451650|50979|8855|54822|413544|4479|48614|8|9|210|1|68.11|2.72|70.83|10.95|39.87|67.42|0.04|0.65|53.54| +2451602|55398|4243|77193|243416|458|34253|1|32|212|53|8.48|0.50|8.98|6.52|251.75|4.49|2.07|1.92|258.77| +|59071|4725||1093514|||7||213|36||91.72||38.54|1100.52|||168.79|| +||15767||474763|||||213|40|1243.20|12.43|1255.63||||506.10||| +2452307|42939|2637|26034|1635889|4872|37855|2|1|213|1|30.76|0.30|31.06|77.11|12.08|20.30|7.74|2.72|89.49| +2452034|52960|12812|82892|316225|426|18227|2|9|215|66|4447.08|0.00|4447.08|49.31|1217.70|3513.19|513.63|420.26|1267.01| +2452526|34826|6447|52082|85606|2660|8514|10|17|216|19|366.70|0.00|366.70|37.49|499.13|344.69|0.88|21.13|536.62| +2452428|42218|14868|52082|383830|4456|47435|7|28|216|33|374.22|29.93|404.15|43.45|1185.03|0.00|48.64|325.58|1258.41| +2451486|60143|5443|32349|16095|281|37714|7|16|217|1|18.10|0.00|18.10|58.61|48.99|12.85|0.68|4.57|107.60| +2451666|58183|15452|18097|1615447|4328|6856|8|25|218|20|173.20|5.19|178.39|15.50|466.20|96.99|45.72|30.49|486.89| +2451540|53575|5323|18097|1015861|1958|14008|2|33|218|56|6214.32|124.28|6338.60|57.94|499.52|3915.02|482.85|1816.45|681.74| +2451600|56692|17120|18097|745852|2188|33771|10|15|218|3|2.61|0.15|2.76|45.04|38.01|2.11|0.13|0.37|83.20| +2451436|38175|3536|87909|925262|2454|20446|10|7|220|1|6.38|0.38|6.76|26.98|10.63|1.72|0.23|4.43|37.99| +2451453|43835|3898|87909|1576956|357|35311|10|3|220|81|3384.99|135.39|3520.38|5.76|1128.33|2572.59|194.97|617.43|1269.48| +2451522|45918|4790|8875|370503|131|22159|1|7|220|47|1220.59|73.23|1293.82|56.09|2164.35|268.52|495.07|457.00|2293.67| +2451629|29420|12208|45713|1757484|4455|41379|2|11|221|26|851.50|17.03|868.53|0.76|153.14|519.41|66.41|265.68|170.93| +2451588|50366|2899|24985|702252|5286|31120|2|33|221|33|2302.74|115.13|2417.87|44.26|366.96|1796.13|126.65|379.96|526.35| +2451656|36084|4466|12062|475875|3231|6234|2|26|222|8|25.92|0.25|26.17|36.69|9.44|14.77|4.90|6.25|46.38| +2451671|32169|7678|12062|1510822|2396|19970|7|1|222|9|555.57|0.00|555.57|69.02|370.35|477.79|70.00|7.78|439.37| +2452266|34473|9637|90791|1510876|6243|35257|7|1|224|36|2225.88|44.51|2270.39|22.85|1424.52|934.86|64.55|1226.47|1491.88| +2452234|34486|9643|50737|1392770|1203|8436|1|9|224|70|293.30|0.00|293.30|80.06|82.60|146.65|68.92|77.73|162.66| +2451871|58963|16979|95583|1374753|5570|35540|2|20|226|3|425.13|0.00|425.13|97.45|130.14|42.51|80.35|302.27|227.59| +2452025|34469|3475|19915|1414991|3220|48649|7|19|226|77|2463.23|98.52|2561.75|22.51|2463.23|1551.83|191.39|720.01|2584.26| +2451958|48694|661|52660||6122||||226||165.76|8.28|174.04|48.49||119.34||33.43|819.01| +2451542|60979|13423|80855|1233127|2292|36610|1|35|227|9|7.29|0.14|7.43|40.37|102.06|0.51|4.06|2.72|142.57| +2451890|60729|4613|75785|1078337|1689|37441|8|10|230|49|2161.88|43.23|2205.11|37.21|2075.15|497.23|66.58|1598.07|2155.59| +2452555|35566|5031|43608|1030758|3522|19133|4|33|235|12|180.96|3.61|184.57|80.88|93.24|115.81|58.63|6.52|177.73| +2451195|32397|5929|79134|345957|7079|32201|7|23|236|37|1377.14|0.00|1377.14|92.47|704.48|413.14|241.00|723.00|796.95| +2451569|51793|7528|58516|617818|2258|2698|4|35|237|58|3094.30|247.54|3341.84|25.95|202.42|1237.72|1578.09|278.49|475.91| +2451562|30180|1708|98949|1886024|3146|20621|10|13|237|20|19.20|0.00|19.20|84.94|7.80|3.07|4.35|11.78|92.74| +2452032|38185|7823|69761|1672975|3107|8486|1|2|238|18|116.64|9.33|125.97|59.89|350.10|47.82|8.94|59.88|419.32| +2451249|33299|9502|71550|1439231|651|28471|10|3|239|26|168.74|11.81|180.55|39.67|168.74|77.62|51.02|40.10|220.22| +2451269|30660|15946|71550|343942|2107|7472|10|20|239|12|672.00|60.48|732.48|38.25|291.60|255.36|204.15|212.49|390.33| +2451188|47528|5821|36394|564514|5333|5752|10|7|240|13|613.47|6.13|619.60|67.23|554.19|0.00|411.02|202.45|627.55| +2451786|37119|3695|54202|708497|6121|38498|2|26|242|78|23.40|0.70|24.10|80.89|25.74|20.82|2.06|0.52|107.33| +2451892|32813|5021|67342|279594|1127|24260|8|1|244|6|145.98|8.75|154.73|4.48|88.68|115.32|0.30|30.36|101.91| +2452620|41508|6996|54222|647452|6729|12006|10|33|245|30|905.10|54.30|959.40|93.35|477.30|479.70|323.30|102.10|624.95| +2452192|36928|4215|70315|1230019|6648|16524|4|19|246|76|1459.96|43.79|1503.75|82.94|127.68|773.77|233.30|452.89|254.41| +2451288|44713|3892|63981|1099709|2927|6545|10|18|247|21|874.65|17.49|892.14|38.88|537.18|778.43|36.56|59.66|593.55| +2451271|55564|10432|63981|1224377|273|20969|10|14|247|30|967.50|58.05|1025.55|33.48|139.80|164.47|32.12|770.91|231.33| +2451929|59456|6965|63381|98975|2569|20516|7|9|249|21|276.99|11.07|288.06|50.95|73.08|254.83|11.96|10.20|135.10| +2451829|42938|11137|16973|1816085|4797|35509|1|10|249|2|65.06|0.65|65.71|63.71|14.60|27.97|1.48|35.61|78.96| +2452625|53792|10776|88951|98452|6229|44806|8|15|250|70|2117.50|63.52|2181.02|18.78|169.40|614.07|405.92|1097.51|251.70| +2452187|34803|8867|52127|1643327|2536|15345|7|13|251|4|102.84|4.11|106.95|15.25|301.28|95.64|5.54|1.66|320.64| +2452057|40974|12385|60121|1704041|3961|17881|10|14|251|21|114.03|7.98|122.01|83.81|285.39|49.03|53.95|11.05|377.18| +2452088|50062|15959|52127|311729|428|28727|4|32|251|25|2784.75|167.08|2951.83|86.60|259.75|1782.24|461.15|541.36|513.43| +2452707|43272|1326|93856|675668|3963|35058|8|3|252|52|758.68|22.76|781.44|48.06|798.20|667.63|60.09|30.96|869.02| +2452760|57517|4602|93856|921560|1566|38419|4|28|252|2|12.92|0.77|13.69|53.08|23.70|9.43|3.49|0.00|77.55| +2452659|44233|513|47017|1116554|1806|17858|2|27|254|39|558.87|5.58|564.45|91.70|50.70|190.01|309.84|59.02|147.98| +2452622|39815|14851|47017|1042217|1680|16790|10|18|254|56|351.12|31.60|382.72|23.40|10.64|294.94|43.25|12.93|65.64| +2450893|54161|8206|98663|145958|6590|23175|4|7|255|65|4193.15|335.45|4528.60|44.02|3064.10|545.10|2663.07|984.98|3443.57| +2451271|39274|15736|48420|1565466|130|20394|2|27|256|5|60.20|1.80|62.00|88.61|216.85|4.21|45.35|10.64|307.26| +2451609|48187|17131|4775|1756138|1901|36064|8|20|257|17|586.67|29.33|616.00|68.70|465.63|258.13|174.12|154.42|563.66| +2452608|48969|4242|10558|339057|4237|5864|2|4|258|14|387.94|31.03|418.97|3.86|132.30|290.95|1.93|95.06|167.19| +2452609|38900|13707|10558|190710|6458|4559|1|25|258|41|6532.53|457.27|6989.80|25.65|3483.77|2155.73|3676.51|700.29|3966.69| +2451723|42666|7171|55596|974443|3464|85|1|20|259|10|165.80|14.92|180.72|17.33|394.80|124.35|40.20|1.25|427.05| +2451768|36107|2036|55596|1571101|329|43157|4|3|259|7|155.82|10.90|166.72|2.91|103.88|10.90|85.50|59.42|117.69| +2451692|36392|2957|55596|551968|5513|47811|10|14|259|6|39.00|3.12|42.12|11.18|11.04|6.24|17.69|15.07|25.34| +2452398|32964|12265|41218|266536|7192|201|4|1|260|52|3042.52|212.97|3255.49|58.36|48.88|517.22|202.02|2323.28|320.21| +2452278|39617|16789|30986|1728204|4622|46593|8|24|260|4|22.88|0.91|23.79|2.25|66.92|1.37|13.55|7.96|70.08| +2452680|43454|9147|81093|1294651|5136|6021|10|14|261|5|28.70|1.14|29.84|33.13|16.05|23.82|1.41|3.47|50.32| +2451653|55004|13042|72780|929281|5484|25640|10|4|262|4|164.12|13.12|177.24|75.61|47.16|27.90|4.08|132.14|135.89| +2451976|31518|15403|4471|336133|4762|41038|1|31|264|15|127.95|7.67|135.62|74.16|14.55|28.14|62.88|36.93|96.38| +2452338|51354|12741|65737|33515|2623|11778|10|24|266|6|20.88|1.87|22.75|41.14|37.56|11.90|0.44|8.54|80.57| +2451882|35852|12227|43915|232941|653|44412|8|13|267|1|100.05|2.00|102.05|20.02|25.01|69.03|4.03|26.99|47.03| +2451908|57143|3301|67390|872151|6159|42120|7|11|267|59|1084.42|0.00|1084.42|67.59|1348.74|596.43|463.59|24.40|1416.33| +2451931|48689|7837|1112|213633|2154|12675|10|7|267|3|15.72|0.94|16.66|39.65|29.04|2.35|10.02|3.35|69.63| +2451945|40421|17881|43915|1005429|5812|41442|1|33|267|13|1970.41|177.33|2147.74|84.27|284.57|807.86|116.25|1046.30|546.17| +2451626|42629|16763|54489|460998|5907|5569|10|32|268|81|6439.50|64.39|6503.89|52.17|1828.98|3928.09|50.22|2461.19|1945.54| +2451349|44083|4999|17432|1154257|1339|15668|7|17|269|37|1115.92|89.27|1205.19|1.52|729.64|524.48|183.34|408.10|820.43| +||4879||600778|2660||||269||||120.05|70.03||78.03||12.19|| +2451040|36723|470|881|816751|1371|17144|4|30|271|25|815.75|16.31|832.06|94.76|1509.00|791.27|12.97|11.51|1620.07| +2450907||17902|881|||32893|7|30|271||785.51|||78.80|143.77||280.58||261.84| +2452446|32721|8749|44046|633563|6358|37200|8|32|272|4|484.00|0.00|484.00|75.00|22.48|280.72|10.16|193.12|97.48| +2452588|44134|7152|44046|241492|5272|44605|1|8|272|19|23.56|1.64|25.20|74.85|7.22|19.79|2.22|1.55|83.71| +2452569|47846|15519|44046|1318084|7130|27023|10|29|272|9|473.76|0.00|473.76|86.46|220.68|440.59|29.18|3.99|307.14| +2451886|36823|17597|32880|1063653|6092|24578|1|33|273|49|175.91|12.31|188.22|22.45|208.25|29.90|91.98|54.03|243.01| +2452206|55587|697|70136|513188|6453|22565|2|10|274|12|1250.16|50.00|1300.16|51.97|529.68|875.11|363.79|11.26|631.65| +||1883|70136|||40305|1|29|274|33||4.15|||107.91|||20.35|177.48| +2451225|32750|325|62086|1301200|2075|2597|2|19|275|24|567.60|5.67|573.27|85.36|554.64|493.81|21.39|52.40|645.67| +2452063|45037|13355|76229|77148|2067|28986|8|22|276|8|163.52|3.27|166.79|95.81|175.84|24.52|59.77|79.23|274.92| +2452044|48478|5791|11370|1860018|5113|48161|7|21|276|2|88.34|0.00|88.34|53.90|17.38|83.92|4.33|0.09|71.28| +2451019|53944|14234|153|1124258|7089|31520|10|35|279|3|26.88|0.26|27.14|55.02|43.23|19.08|2.88|4.92|98.51| +2451169|37543|8024|95014|1529603|579|21555|10|10|280|4|69.72|0.69|70.41|98.01|98.76|35.55|12.30|21.87|197.46| +2451005|34732|2888|95014|764311|4516|15568|2|17|280|2|56.46|3.95|60.41|89.93|5.82|41.78|4.69|9.99|99.70| +2451335|48208|13744|62016|1761025|3389|25306|7|9|281|13|638.04|6.38|644.42|38.84|121.03|204.17|360.11|73.76|166.25| +2451225|32393|14605|62016|1378502|6893|39284|4|21|281|11|497.42|0.00|497.42|70.80|130.90|29.84|9.35|458.23|201.70| +2452083|49006|491|927|195434|6949|30864|1|19|282|75|22.50|1.35|23.85|69.10|354.75|3.60|14.55|4.35|425.20| +2452097|48646|6619|927|225768|6265|39962|8|31|282|11|630.96|12.61|643.57|61.10|99.22|441.67|56.78|132.51|172.93| +2452663|42726|5406|46407|695948|5489|43877|2|4|284|34|0.00|0.00|0.00|42.68|142.12|0.00|0.00|0.00|184.80| +2452661|40701|6582|19453|1102865|5505|19745|2|13|284|77|1777.93|35.55|1813.48|5.72|546.70|497.82|153.61|1126.50|587.97| +2451270|48675|8974|58576|1050395|6348|2701|2|32|287|40|922.00|73.76|995.76|59.06|553.20|820.58|45.63|55.79|686.02| +|37084|8173|||||||289|39|3701.49|||1.42|994.11||510.80||| +2450936|48586|6176|84620|140043|1894|10498|10|23|291|4|261.20|18.28|279.48|53.67|157.72|138.43|85.93|36.84|229.67| +2450945|46833|7760|17143|1125895|6553|26811|10|15|292|53|1176.07|105.84|1281.91|65.07|2117.35|470.42|465.72|239.93|2288.26| +2450999|39626|16448|17143|1294276|3772|12468|2|25|292|37|0.00|0.00|0.00|24.94|66.23|0.00|0.00|0.00|91.17| +2452339|52965|143|78712|1787914|3239|19411|7|16|293|17|319.26|9.57|328.83|11.94|609.62|57.46|54.97|206.83|631.13| +2451769|38420|4751|10064|501804|7144|8931|8|21|294|28|209.16|4.18|213.34|11.29|784.56|106.67|15.37|87.12|800.03| +2451794|43388|6233|21102|1169567|5767|12414|2|31|294|16|14.40|1.15|15.55|43.90|202.72|4.17|0.61|9.62|247.77| +2452547|35822|7881|68512|1566713|6263|35766|7|29|295|67|1224.76|110.22|1334.98|10.92|1960.42|1163.52|45.93|15.31|2081.56| +2452608|37763|6981|70451|1563442|816|4637|1|12|296|14|446.60|17.86|464.46|77.45|205.24|138.44|126.34|181.82|300.55| +2451513|36208|3728|25644|1002595|6998|22201|8|20|297|9|9.54|0.76|10.30|3.59|50.58|3.52|5.11|0.91|54.93| +2451579|30046|17446|25644|1325942|761|32733|8|2|297|8|74.80|1.49|76.29|4.44|407.36|64.32|0.73|9.75|413.29| +2451943|40453|7705|56274|1688546|4018|1537|8|21|300|19|14.25|0.71|14.96|4.39|12.92|8.69|2.22|3.34|18.02| +2452125|45914|11313|23057|47821|7088|15602|8|6|305|22|9.68|0.58|10.26|12.29|476.96|6.67|2.52|0.49|489.83| +2452114|52394|185|23057|1112168|1746|171|1|22|305|32|3361.28|235.28|3596.56|69.37|409.92|3260.44|40.33|60.51|714.57| +2452027|43246|7879|23057|1627650|5302|22154|7|3|305|14|314.86|15.74|330.60|41.94|375.48|91.30|201.20|22.36|433.16| +2452512|35215|4231|26936|1511570|5202|44481|2|4|306|28|2094.12|0.00|2094.12|84.03|202.44|984.23|521.64|588.25|286.47| +2452499|44733|4959|26936|731899|3865|13478|8|22|306|8|338.72|13.54|352.26|30.55|151.68|335.33|0.71|2.68|195.77| +2452487|33700|3036|26936|1059010|5605|9000|7|27|306|6|124.68|7.48|132.16|95.35|137.16|73.56|1.53|49.59|239.99| +2452539|47950|5985|46727|1114498|5916|44487|1|33|306|4|35.28|1.05|36.33|54.41|21.32|5.99|27.82|1.47|76.78| +2452328|56139|14893|24006|724246|1968|29243|8|27|307|1|1.34|0.02|1.36|24.87|3.44|1.24|0.07|0.03|28.33| +2452335|48097|3325|94379|1849132|358|33537|4|2|308|10|188.00|1.88|189.88|9.90|456.70|163.56|7.57|16.87|468.48| +2451430|34450|12907|34893|1679209|18|16474|8|32|309|8|88.32|5.29|93.61|44.70|18.48|13.24|51.05|24.03|68.47| +2451442|51862|17950|83188|118331|1474|2617|10|30|309|2|97.80|5.86|103.66|47.74|23.98|67.48|2.72|27.60|77.58| +2452657|52974|6606|43765|1067390|354|39765|1|15|310|14|119.84|9.58|129.42|79.10|659.12|32.35|87.49|0.00|747.80| +2451855|32050|15745|87684|1239851|18|45785|7|24|311|8|138.88|12.49|151.37|79.40|308.64|38.88|2.00|98.00|400.53| +2451424|59040|14899|85958|1394059|4968|16277|7|15|315|18|9.72|0.68|10.40|33.61|470.70|5.44|3.38|0.90|504.99| +2452234|48697|17327|79361|330622|2673|41971|2|23|316|52|728.00|7.28|735.28|40.72|936.00|444.08|261.20|22.72|984.00| +2451669|44319|8213|11168|465341|4034|40727|1|30|317|22|1367.08|0.00|1367.08|56.43|410.08|956.95|233.77|176.36|466.51| +2451012||14876|||3645|13371|1||319|15||0.00|65.40||35.25||10.36||63.27| +2451003|57417|7285|67173|1640110|610|8263|1|23|319|3|313.59|6.27|319.86|82.53|27.27|257.14|46.85|9.60|116.07| +2451020|42072|196|67173|629046|1595|30897|4|21|319|17|33.83|1.35|35.18|86.22|77.69|27.40|0.45|5.98|165.26| +2452788|45127|15366|79275|1653390|5127|8864|10|25|320|20|708.40|14.16|722.56|10.53|81.60|375.45|93.22|239.73|106.29| +2452699|44870|17535|79275|2506|4103|25700|4|22|320|7|12.04|0.24|12.28|23.66|0.21|11.19|0.36|0.49|24.11| +2452800|35914|10095|79275|1529192|2283|18221|7|28|320|76|3405.56|136.22|3541.78|47.46|928.72|817.33|284.70|2303.53|1112.40| +||2906|89370|849103||33594||8|321|||0.00||||386.73|||214.13| +2451968|31620|14819|98601|742076|353|5030|2|12|322|5|68.10|4.08|72.18|39.92|54.50|61.97|4.65|1.48|98.50| +2452274|39540|15051|83934|1117402|3230|39505|1|26|324|14|7.70|0.00|7.70|14.31|36.68|4.00|1.92|1.78|50.99| +2451767|51985|3403|65326|1658818|2991|30764|2|12|325|30|1328.70|119.58|1448.28|69.27|1452.30|770.64|541.31|16.75|1641.15| +2452208|34401|2905|58699|323081|6065|46249|4|22|326|4|98.68|3.94|102.62|17.81|0.00|69.07|19.54|10.07|21.75| +2452671|57102|1824|77925|1184178|4251|12771|7|12|327|2|30.76|0.00|30.76|77.03|44.30|7.69|16.37|6.70|121.33| +2452559|47190|3711|77925|1477839|1922|30005|4|3|327|20|1676.60|83.83|1760.43|43.72|50.80|301.78|1292.33|82.49|178.35| +2452660|33517|16710|77925|1843854|4104|30354|10|6|327|63|700.56|42.03|742.59|18.50|87.57|497.39|144.25|58.92|148.10| +2451234|39221|700|31383|129961|4756|24351|4|24|328|12|18.84|1.50|20.34|87.93|239.76|9.60|9.14|0.10|329.19| +2451276|31686|2006|31383|515690|3145|48553|7|6|328|1|4.17|0.08|4.25|78.14|42.74|2.21|0.60|1.36|120.96| +2451614|44704|14017||404055|6352||1||329|||1.38||||15.18|6.58||| +2451511|56287|7438|40746|382746|1571|44043|1|2|329|24|270.96|8.12|279.08|75.66|95.52|121.93|1.49|147.54|179.30| +2451682|32692|4699|40746|1432507|7059|34364|2|3|329|3|38.97|0.00|38.97|15.97|144.78|3.89|1.75|33.33|160.75| +2450923|47437|4132|95762|365713|2804|29895|7|17|330|34|617.44|30.87|648.31|74.91|181.56|172.88|377.87|66.69|287.34| +2450960|52163|10598|95762|1581154|6186|28018|7|34|330|14|21.56|1.50|23.06|10.32|113.96|4.95|8.63|7.98|125.78| +2452538|38949|8988|37108|339289|5898|37660|7|24|331|40|1613.20|32.26|1645.46|83.04|328.40|435.56|423.95|753.69|443.70| +2452197|39064|23|59293|1523765|4824|4730|7|2|332|3|16.65|1.33|17.98|68.31|36.66|12.82|0.07|3.76|106.30| +2452213|48487|13287|76672|39527|2906|685|8|20|334|44|68.20|2.04|70.24|62.65|2602.60|30.00|28.26|9.94|2667.29| +2452116|39217|11675|76672|1496218|3857|21496|7|31|334|9|507.06|45.63|552.69|71.79|46.08|15.21|93.45|398.40|163.50| +2451979|41794|13861|6836|941763|1118|24120|4|25|336|17|774.69|15.49|790.18|63.78|240.38|472.56|108.76|193.37|319.65| +2452060|58832|11041|95795|1260037|5879|41368|10|32|336|1|72.99|6.56|79.55|79.45|34.35|44.52|3.70|24.77|120.36| +2451033|52171|14840|99819|590310|6716|36275|4|35|337|74|1978.76|19.78|1998.54|6.01|409.22|257.23|1704.31|17.22|435.01| +2451394|48948|9710|74364|1666191|5755|27366|2|19|338|16|639.84|31.99|671.83|34.60|1119.84|300.72|61.04|278.08|1186.43| +2451525|51229|3823|95992|358881|5858|7588|7|30|339|17|659.43|59.34|718.77|22.87|727.09|408.84|250.59|0.00|809.30| +2451631|42728|17132|80091|1807739|5362|6138|8|17|341|3|104.79|0.00|104.79|67.37|31.23|80.68|11.81|12.30|98.60| +2451555|39080|9955|44832|730709|4139|23616|4|3|343|5|548.15|10.96|559.11|20.52|165.70|153.48|110.50|284.17|197.18| +2451592|58034|16423|73030|1340058|3822|40786|10|9|343|23|1587.23|142.85|1730.08|55.97|883.89|190.46|1089.48|307.29|1082.71| +2451352|45380|2740|48579|80506|6097|34441|7|11|344|7|172.55|13.80|186.35|22.97|5.39|169.09|2.31|1.15|42.16| +2451379|34707|2071|12748|1257120|1845|28170|4|27|344|28|117.32|0.00|117.32|58.70|43.40|23.46|91.98|1.88|102.10| +2450977|44173|268|54810|1080578|2046|35561|1|19|346|1|76.40|1.52|77.92|81.37|43.09|19.10|3.43|53.87|125.98| +2451602|51641|7474|86460|187891|3258|46625|2|29|347|28|37.24|3.35|40.59|31.16|637.56|22.71|0.58|13.95|672.07| +2451686|46500|14713|86460|1074232|5166|49747|10|19|347|7|112.98|4.51|117.49|64.25|43.89|71.17|21.32|20.49|112.65| +2451774|40909|8468|31073|909186|2346|40353|4|23|348|57|185.25|11.11|196.36|93.67|320.34|159.31|18.15|7.79|425.12| +2451066||3176|26878|170099||48614||34|351|38||30.47||56.62||347.42|||368.29| +2451088|38894|7429|26878|505539|4728|14270|7|1|351|44|2494.80|124.74|2619.54|74.43|3402.08|723.49|1363.90|407.41|3601.25| +2452084|31566|8329|1484|265724|1769|47414|7|32|352|42|499.38|14.98|514.36|47.43|144.90|59.92|232.91|206.55|207.31| +2451173|29247|8002|19363|1383985|1407|37229|1|18|354|28|2218.44|177.47|2395.91|44.46|1011.92|1464.17|279.07|475.20|1233.85| +2451097|40358|15998|57203|258931|4212|15093|8|17|354|31|374.48|22.46|396.94|65.28|251.41|179.75|27.26|167.47|339.15| +2451099|41916|15728|17299|523369|2895|23621|1|25|354|14|297.22|26.74|323.96|57.22|574.70|11.88|31.38|253.96|658.66| +2452687|46537|12607|6280|19029|1731|32833|1|22|355|1|43.80|2.19|45.99|63.95|4.38|14.01|25.61|4.18|70.52| +2452744|44768|11607|9321|799420|4835|24203|1|19|355|38|62.32|4.36|66.68|65.94|60.42|29.91|6.15|26.26|130.72| +2452027|46231|15445|27487|1844122|1852|14382|8|30|358|13|205.66|14.39|220.05|3.31|71.63|49.35|40.64|115.67|89.33| +2450893|46834|4124|45694|1305910|6926|11589|1|18|359|1|101.70|6.10|107.80|1.61|12.40|36.61|39.70|25.39|20.11| +2450957|44428|16910|7508|1466922|1183|25044|1|1|359|41|732.26|7.32|739.58|43.68|50.84|666.35|5.27|60.64|101.84| +2452219|28872|6625|34066|1444783|3980|36824|10|35|360|17|1720.23|68.80|1789.03|18.68|364.82|1358.98|75.86|285.39|452.30| +2451583|60078|15919|3825|1726769|6513|34257|8|15|361|15|335.55|16.77|352.32|30.28|71.55|20.13|189.25|126.17|118.60| +2452703|38541|13905||||||1|364||||242.17|82.55||202.96||24.92|196.38| +2451189|59633|7972|15564|1170835|1487|19692|7|1|366|15|1969.95|19.69|1989.64|26.89|442.20|630.38|468.84|870.73|488.78| +2451102|58418|127|30218|1856303|1588|4826|1|11|366|44|1231.12|0.00|1231.12|91.37|630.52|590.93|281.68|358.51|721.89| +2451991|29828|9665|74099|1776374|5049|30171|2|31|368|34|970.36|48.51|1018.87|78.62|412.76|271.70|482.07|216.59|539.89| +2451905|42204|14957|10878|1352204|3451|43498|8|7|368|1|42.64|1.27|43.91|47.00|13.32|12.36|9.38|20.90|61.59| +2452631|53610|5371|20535|957674|433|2392|2|17|369|26|3027.96|90.83|3118.79|79.38|1807.00|3027.96|0.00|0.00|1977.21| +2452552|29630|16764|50484|395444|3604|26611|1|16|369|48|2923.68|116.94|3040.62|61.93|876.96|0.00|2368.18|555.50|1055.83| +2451380|34632|3085|50435|1301994|2587|44380|1|16|370|43|6061.71|121.23|6182.94|94.89|1859.75|3697.64|685.58|1678.49|2075.87| +2452061|33755|2603|93120|696318|6549|28027|2|16|372|34|1038.36|51.91|1090.27|63.63|1016.26|1017.59|14.74|6.03|1131.80| +2452081|49888|10063|93120|1363087|2206|12525|8|10|372|41|3335.76|100.07|3435.83|10.45|2532.57|2168.24|70.05|1097.47|2643.09| +2450870|60038|16970|24545|505417|915|42766|2|27|374|4|34.88|1.39|36.27|46.73|84.16|6.27|19.74|8.87|132.28| +2451931|50379|2318|26646|277942|1380|33618|7|29|376|80|1809.60|126.67|1936.27|84.85|765.60|1194.33|141.51|473.76|977.12| +2451927|31695|7159|63249|462413|3530|8294|4|8|376|73|196.37|3.92|200.29|15.65|79.57|196.37|0.00|0.00|99.14| +2451484|59634|10768|60375|1010843|5114|30632|7|16|377|35|3959.20|118.77|4077.97|16.36|285.60|395.92|1354.04|2209.24|420.73| +2452254|32822|10763|13752|973156|2372|30388|10|25|380|7|365.33|7.30|372.63|24.28|149.45|94.98|240.61|29.74|181.03| +2451087|44483|3680|15667|219073|1800|13327|8|3|381|9|281.79|22.54|304.33|91.65|450.90|154.98|34.23|92.58|565.09| +2451144|32923|15163|15667|129275|407|45892|7|11|381|30|2791.20|223.29|3014.49|65.20|1380.60|697.80|1465.38|628.02|1669.09| +2451120|52288|14534|15667|469175|1400|15978|2|29|381|9|242.64|16.98|259.62|33.56|73.80|33.96|112.68|96.00|124.34| +2451194|60862|1108|58244|1005593|4415|28037|4|31|382|59|1894.49|56.83|1951.32|7.25|1125.72|587.29|26.14|1281.06|1189.80| +2452238|32213|8373|26687|323113|2084|43455|2|17|383|3|4.17|0.12|4.29|48.62|6.36|2.33|0.58|1.26|55.10| +|31775|7843|||6162|37751|8|32|384|3||0.00|95.43||51.21|32.44||56.07|| +|48323|13129|86626|441946|2066|11033|||384||849.60||849.60|84.70||356.83|438.56|54.21|145.30| +2451913|36654|15169|62143|1873843|2768|27293|10|3|385|13|260.00|13.00|273.00|96.87|14.82|135.20|38.68|86.12|124.69| +2452338|33960|9035|22927|1392791|4669|20947|10|26|386|1|14.93|0.89|15.82|73.80|7.28|5.22|2.33|7.38|81.97| +2452360|35556|15189|22927|1747589|5207|38872|10|10|386|39|102.57|9.23|111.80|90.17|452.01|76.92|9.74|15.91|551.41| +2452406|34694|6408|95008|560439|3470|20901|2|8|388|6|90.36|0.00|90.36|8.14|117.24|15.36|60.75|14.25|125.38| +2451231|33867|7312|80465|1625579|1158|28784|8|7|390|4|86.00|0.00|86.00|56.25|80.92|82.56|2.23|1.21|137.17| +2451262|49464|6688|86421|438441|6461|46481|7|14|390|41|3301.73|198.10|3499.83|75.86|1299.70|759.39|2542.34|0.00|1573.66| +2452298|42163|15389|83449|112109|5215|23121|4|33|392|14|1412.04|28.24|1440.28|62.60|574.42|42.36|205.45|1164.23|665.26| +2452531|35240|79|66639|1116295|5148|7554|4|18|393|1|8.07|0.32|8.39|71.22|15.56|6.13|0.89|1.05|87.10| +2452522|52696|12397|66639|314088|1100|15265|8|2|393|17|90.95|0.90|91.85|92.62|68.17|60.02|21.65|9.28|161.69| +2452335|58486|553|70303|751221|1932|6643|8|32|394|5|574.10|5.74|579.84|66.45|268.70|258.34|179.98|135.78|340.89| +2452421|39780|5489|1374|950983|1392|201|8|32|394|34|2673.08|53.46|2726.54|84.28|341.02|534.61|898.15|1240.32|478.76| +2452550|38132|14589|42702|1086784|2066|21748|7|12|395|13|866.58|51.99|918.57|69.47|619.06|719.26|58.92|88.40|740.52| +2452533|42836|17922|12834|11548|1192|16389|4|25|395|71|1404.38|42.13|1446.51|34.00|262.70|786.45|216.27|401.66|338.83| +2452648|40082|402|1172|1580753|5581|11962|7|11|397|8|456.64|31.96|488.60|62.67|10.72|410.97|10.96|34.71|105.35| +2452685|60076|13327|20658|1193307|1956|5405|2|10|397|82|8338.58|83.38|8421.96|96.93|4932.30|4252.67|1225.77|2860.14|5112.61| +2452592|40383|13591|42975|1315030|6285|6740|7|33|397|56|98.00|8.82|106.82|73.03|70.00|21.56|60.38|16.06|151.85| +|28815|465||1585758|6870|43182||18|397|40|483.60|33.85|||||162.63||| +2451604|42107|1561|26512|333110|1481|1852|4|29|402|2|145.50|2.91|148.41|31.73|19.40|125.13|12.83|7.54|54.04| +2451615|36444|3040|26512|994524|1476|18475|2|23|402|7|399.49|35.95|435.44|98.18|302.33|315.59|49.50|34.40|436.46| +2451713|43596|16136|27157|975441|414|15787|10|13|404|7|0.00|0.00|0.00|29.73|2.10|0.00|0.00|0.00|31.83| +2451255|51934|9610|72256|1817079|5585|2444|8|17|405|14|174.02|8.70|182.72|46.14|38.36|26.10|142.00|5.92|93.20| +2451231|30605|15997|72256|700683|4558|4833|2|18|405|1|16.92|0.00|16.92|94.80|9.67|12.18|4.50|0.24|104.47| +2451609|39444|4600|7409|1783420|2106|21945|4|15|406|16|952.00|76.16|1028.16|55.48|423.04|714.00|7.14|230.86|554.68| +2452558|47839|4075|43504|92215|6214|36213|1|33|407|15|172.65|10.35|183.00|24.24|262.80|151.93|5.59|15.13|297.39| +2451244|43275|9205|65390|1777700|3283|42066|1|22|408|9|33.12|0.99|34.11|12.47|124.38|15.56|14.57|2.99|137.84| +|54157|16048|||||4||408||211.50|||||16.92||182.91|| +2450964|33960|14402|63774|672128|4173|9771|4|34|409|53|322.77|16.13|338.90|84.00|20.14|235.62|17.43|69.72|120.27| +2451089|46818|1072|63774|1895589|1367|40345|1|11|409|2|117.30|8.21|125.51|83.67|66.30|96.18|11.61|9.51|158.18| +2451349|53236|17065|40314|1830661|1008|46555|2|34|410|3|7.83|0.07|7.90|75.01|4.47|3.75|3.30|0.78|79.55| +2451349|37035|9457|97273|250223|6295|31214|1|15|410|6|272.88|8.18|281.06|10.85|284.76|2.72|113.46|156.70|303.79| +2452359|32786|15071|47578|612344|5927|23910|4|6|412|52|908.44|81.75|990.19|8.05|453.96|808.51|41.97|57.96|543.76| +2452192|28813|15511|47578|967815|301|33187|8|14|412|38|234.84|14.09|248.93|50.65|13.30|218.40|4.27|12.17|78.04| +2452265|51765|15317|35092|1615749|4853|23910|4|9|412|17|400.01|24.00|424.01|42.32|0.00|280.00|82.80|37.21|66.32| +2451318|59607|2042|5973|1205357|6694|36431|4|17|413|34|1037.34|51.86|1089.20|56.51|1281.46|902.48|78.21|56.65|1389.83| +2451307|60718|14569|31435|1009212|1403|35594|4|26|413|8|125.92|8.81|134.73|27.98|71.68|8.81|31.61|85.50|108.47| +2451291|42610|4970|31435|1128018|3321|39824|1|20|413|19|87.59|4.37|91.96|43.08|156.37|48.17|39.42|0.00|203.82| +2451761|61094|8179|61949|1291862|1552|34842|2|2|415|7|155.47|7.77|163.24|60.17|47.81|6.21|89.55|59.71|115.75| +2451675|32998|14719|67996|1375495|2194|12893|7|16|415|52|5269.16|210.76|5479.92|86.48|2333.24|2265.73|870.99|2132.44|2630.48| +2451801|38153|15413|49638|766000|5209|8818|7|24|415|32|3435.20|137.40|3572.60|3.54|1939.84|2954.27|134.66|346.27|2080.78| +2452008|60676|2229|93947|460964|3548|16932|10|22|416|35|417.90|25.07|442.97|25.55|21.00|196.41|101.88|119.61|71.62| +2451646|38113|13436|72010|1480263|5443|20223|8|26|417|8|105.04|4.20|109.24|66.72|55.04|3.15|93.73|8.16|125.96| +2451583|59846|1982|72010|848770|6234|47745|8|8|417|5|239.55|9.58|249.13|58.80|33.85|186.84|30.57|22.14|102.23| +2452328|40259|16519|73535|1649283|288|13740|4|7|419|45|1553.85|62.15|1616.00|31.10|814.95|932.31|6.21|615.33|908.20| +2452030|30736|717|41075|1579584|2402|26067|4|35|420|11|4.51|0.31|4.82|18.14|44.11|1.53|1.78|1.20|62.56| +2451987|50653|11605|41075|180840|3592|45144|7|3|420|77|2236.85|22.36|2259.21|59.34|1710.17|581.58|976.60|678.67|1791.87| +2452081|29488|9605|41075|676963|216|37516|1|29|420|66|1943.04|155.44|2098.48|53.88|23.10|1573.86|177.20|191.98|232.42| +2452452|56667|5664|93350|831631|2712|21793|2|28|421|44|872.08|43.60|915.68|66.92|1744.60|444.76|136.74|290.58|1855.12| +2452384|58487|13971|97932|1848658|6623|8541|1|14|421|15|32.85|0.32|33.17|43.60|38.10|17.08|12.93|2.84|82.02| +2451190|45111|2197|31925|839292|7188|37931|2|2|422|55|6268.35|564.15|6832.50|65.54|3493.60|5892.24|319.69|56.42|4123.29| +2451229|53177|10820|31925|937264|5712|42913|8|32|422|1|76.00|6.84|82.84|91.45|30.19|25.08|37.68|13.24|128.48| +2451815|47996|16688|86637|919336|5011|19886|8|21|423|41|1687.56|16.87|1704.43|78.99|917.17|1063.16|399.61|224.79|1013.03| +2451763|46413|3881|86637|865483|3009|48182|10|21|423|29|102.66|0.00|102.66|92.95|115.42|95.47|6.90|0.29|208.37| +2452659|55223|14355|29266|109867|4416|16965|4|6|424|2|1.60|0.00|1.60|84.48|2.66|0.64|0.28|0.68|87.14| +2450979|31212|1444|72290|924921|3152|27336|1|3|426|34|528.36|42.26|570.62|55.72|331.84|369.85|4.75|153.76|429.82| +2450988|58761|3565|24858|1790058|5749|39840|1|35|426|45|22.50|1.80|24.30|88.99|7.20|15.52|4.04|2.94|97.99| +2451051|52313|872|24858|1652344|1428|18112|7|23|426|8|581.20|11.62|592.82|94.27|148.40|180.17|116.29|284.74|254.29| +2451147|29195|6038|57891|1751270|6270|41270|1|22|427|40|745.60|7.45|753.05|64.24|0.00|410.08|93.94|241.58|71.69| +2451226|57235|15745|57891|614017|6855|10093|4|34|427|40|3817.60|305.40|4123.00|10.47|813.20|267.23|3124.32|426.05|1129.07| +2451619|34722|9746|82545|202460|6500|30940|2|33|431|42|1109.64|0.00|1109.64|42.18|976.50|998.67|103.20|7.77|1018.68| +2451530|32267|17014|82545|155569|899|229|4|34|431|16|1875.52|75.02|1950.54|3.76|600.16|450.12|898.00|527.40|678.94| +2451654|57385|15578|82545|103083|2839|36242|2|24|431|14|1184.40|71.06|1255.46|61.87|205.24|260.56|341.82|582.02|338.17| +2452795|46728|8647|59712|1331765|3953|17341|10|4|434|20|350.80|10.52|361.32|53.12|177.60|84.19|15.99|250.62|241.24| +2451646|54538|11995|19965|1027919|6273|20227|1|5|435|5|6.15|0.36|6.51|73.77|29.40|1.47|1.02|3.66|103.53| +2452278|33725|10143|55736|43244|2815|42386|7|26|436|51|546.72|5.46|552.18|86.07|390.66|267.89|217.48|61.35|482.19| +2451574|32871|7435|28482|1279366|1495|1577|7|35|438|4|8.80|0.17|8.97|6.94|6.48|7.65|0.46|0.69|13.59| +2451631|35232|3271|19377|1281952|5327|43502|8|15|438|1|6.49|0.45|6.94|62.65|19.49|1.42|4.56|0.51|82.59| +2451572|31430|17038|28482|1104494|413|33579|4|24|438|15|77.25|3.09|80.34|97.78|195.60|56.39|16.06|4.80|296.47| +2450937|29327|12514|23731|28081|2608|5488|7|5|439|40|2705.20|135.26|2840.46|20.98|868.40|1839.53|380.89|484.78|1024.64| +2450999|46033|9037|23731|1215238|5468|2098|1|21|439|84|1450.68|29.01|1479.69|40.57|819.00|420.69|329.59|700.40|888.58| +2451890|60405|10339|20647|518899|1887|20475|4|6|440|24|1656.96|82.84|1739.80|43.74|989.28|662.78|308.19|685.99|1115.86| +2451856|55390|5255|30876|884661|4046|40231|2|11|440|28|955.08|85.95|1041.03|28.68|1136.80|811.81|75.93|67.34|1251.43| +2451959|29302|13261|20647|1235293|2233|10326|4|2|440|75|1763.25|0.00|1763.25|23.82|66.00|405.54|706.00|651.71|89.82| +2451223|49849|4238|84518|1285564|2016|15715|2|33|441|19|187.91|11.27|199.18|96.74|187.91|176.63|6.76|4.52|295.92| +2451338|59091|15880|84518|113677|2504|4531|2|10|441|33|147.18|5.88|153.06|80.09|73.59|139.82|0.95|6.41|159.56| +2451326|59619|7195|6332|1245167|5249|6884|7|11|441|1|24.86|1.49|26.35|95.53|3.60|15.91|5.63|3.32|100.62| +2451559|45419|16183|68569|842013|5857|13301|7|2|443|2|8.12|0.16|8.28|68.21|71.48|3.08|1.86|3.18|139.85| +2451601|52172|7858|90783|144761|2739|47964|10|30|443|2|45.94|2.29|48.23|26.98|29.46|6.89|36.70|2.35|58.73| +2452554|33168|4350|3617|665389|7113|39637|4|20|444|7|423.99|38.15|462.14|25.09|185.99|334.95|30.27|58.77|249.23| +2452536|37536|15417|3617|1616514|3111|32108|10|14|444|23|1068.81|42.75|1111.56|68.05|638.02|74.81|347.90|646.10|748.82| +2451574|41440|14143|74858|1295931|1804|44077|2|18|445|64|190.08|17.10|207.18|15.08|1521.92|20.90|118.42|50.76|1554.10| +2451070|60621|1166|61212|390387|5763|14339|1|26|447|9|777.42|31.09|808.51|48.35|63.45|23.32|648.52|105.58|142.89| +2451504|34187|3211|34691|1173178|5010|41242|7|1|448|53|8093.10|728.37|8821.47|71.34|288.85|5988.89|399.79|1704.42|1088.56| +2451328|34770|5632|27603|929014|3107|1600|2|33|449|20|502.00|20.08|522.08|18.81|581.20|185.74|186.59|129.67|620.09| +2451293||10702||||6842||18|449|29||||||374.79|642.34||496.12| +2452531|47253|9276|17429|1790686|2575|2111|1|3|450|34|811.58|16.23|827.81|7.23|462.06|762.88|31.16|17.54|485.52| +2452609|54714|17247|17429|1408373|3966|19912|4|1|450|60|945.00|37.80|982.80|0.97|124.20|170.10|209.22|565.68|162.97| +2451259|53931|1856|39699|1163717|6429|27941|7|13|451|21|41.16|2.46|43.62|37.71|17.64|11.11|24.64|5.41|57.81| +2450915|57222|13354|790|1272621|3205|589|4|14|452|11|462.77|41.64|504.41|11.11|244.64|439.63|2.54|20.60|297.39| +2450842|46368|7210|77669|82247|1041|31035|4|8|452|21|1543.71|46.31|1590.02|80.23|434.07|1250.40|38.13|255.18|560.61| +2452012|30174|1853|81989|1618065|6141|17417|1|24|453|58|5349.92|427.99|5777.91|2.06|1475.52|3316.95|101.64|1931.33|1905.57| +2452051|53733|9929|56905|135256|3974|10245|2|9|453|3|38.04|0.00|38.04|83.33|111.00|19.78|4.56|13.70|194.33| +2451966|48026|14774|16391|1288680|3419|48158|4|1|454|8|250.24|7.50|257.74|74.15|46.24|17.51|193.16|39.57|127.89| +2451924|38205|11930|16391|200982|4796|12257|7|24|454|13|397.15|0.00|397.15|42.60|244.40|146.94|42.53|207.68|287.00| +2452228|53069|2219|12799|763234|3656|43631|7|18|455|31|871.41|43.57|914.98|15.70|358.67|313.70|295.58|262.13|417.94| +2451116|29630|6790|43274|953565|4023|26481|2|30|457|14|564.90|16.94|581.84|55.76|494.20|192.06|85.75|287.09|566.90| +2451587|60453|9385|11407|707804|4423|38242|7|22|458|42|275.52|13.77|289.29|82.84|241.08|195.61|27.96|51.95|337.69| +2451482|35906|3136|11407|218931|1496|31261|2|21|458|74|1187.70|83.13|1270.83|11.48|71.78|415.69|386.00|386.01|166.39| +2451904|51596|17234|1393|219932|2220|31308|7|7|459|73|151.84|12.14|163.98|98.36|130.67|144.24|6.91|0.69|241.17| +2452090|39790|5893|41434|1621075|3116|31557|7|29|460|11|849.97|33.99|883.96|83.08|19.91|518.48|92.81|238.68|136.98| +2452205|32783|17601|33183|1465185|5342|38346|2|14|461|4|156.72|3.13|159.85|32.47|102.04|98.73|46.39|11.60|137.64| +2452217|55954|1387|33183|1001526|1475|15024|10|25|461|14|1885.24|113.11|1998.35|52.38|348.32|1640.15|83.33|161.76|513.81| +2452318|60084|1821|33183|719773|3141|27356|8|9|461|4|67.88|0.00|67.88|72.52|38.40|4.07|52.32|11.49|110.92| +2451059|52800|11539|71982|1802376|5612|34848|8|7|462|95|310.65|24.85|335.50|57.99|440.80|254.73|29.07|26.85|523.64| +2452436|44663|4416|9180|926310|2242|12174|10|23|463|17|114.24|2.28|116.52|22.98|19.55|54.83|56.43|2.98|44.81| +2452540|38511|16104|9180|1396005|5429|35606|7|24|463|1|33.50|3.01|36.51|71.34|39.70|30.82|0.96|1.72|114.05| +2452406|41841|1819|21538|1255765|39|32666|4|12|463|22|852.94|68.23|921.17|46.17|366.74|656.76|192.25|3.93|481.14| +2451294|55654|9496|60833|394590|4700|34057|10|26|464|94|2286.08|91.44|2377.52|5.18|1015.20|297.19|1332.55|656.34|1111.82| +2451584|37924|9416|27033|1123965|4450|11782|8|29|465|2|157.64|14.18|171.82|36.83|10.06|80.39|33.99|43.26|61.07| +2451572|49697|14986|31625|1185599|3150|31101|10|10|465|15|1013.55|20.27|1033.82|10.13|449.25|597.99|112.20|303.36|479.65| +2451648|44347|6965|89778|225272|3989|5106|8|10|466|27|2107.08|42.14|2149.22|29.53|0.00|294.99|887.92|924.17|71.67| +2451940|33282|17021|23005|315906|4213|8520|1|14|467|14|1714.02|0.00|1714.02|36.70|802.76|1714.02|0.00|0.00|839.46| +2452706|33343|9693|96265|1396519|890|26061|4|10|469|35|198.80|1.98|200.78|80.71|63.70|7.95|87.79|103.06|146.39| +2451175|39055|11395|44999|1753349|1476|8803|2|15|470|27|254.34|22.89|277.23|80.58|678.24|231.44|5.72|17.18|781.71| +2451653|50810|13574|33663|170286|3709|24779|10|7|471|2|36.64|2.19|38.83|21.15|28.60|5.86|3.38|27.40|51.94| +2451573|38539|16852|82172|824217|5793|35416|4|15|471|26|53.30|3.73|57.03|14.11|2.60|35.17|7.97|10.16|20.44| +2452814|45371|14155|22502|1254132|6562|46992|2|2|473|20|70.40|2.81|73.21|62.80|1034.60|66.17|3.89|0.34|1100.21| +2452803|49513|6909|64250|1130520|2706|22492|2|24|473|31|1585.96|111.01|1696.97|38.75|352.47|79.29|1280.66|226.01|502.23| +2452230|51126|5595|57345|32139|1780|16221|10|11|474|13|250.12|12.50|262.62|39.22|150.67|12.50|102.17|135.45|202.39| +2452288|44890|6965|26197|695483|3667|43051|1|6|474|28|291.76|23.34|315.10|36.53|143.08|55.43|2.36|233.97|202.95| +2452711|51996|17803|99711|675296|630|19611|1|6|475|14|293.44|11.73|305.17|16.49|625.94|164.32|32.28|96.84|654.16| +2451777|46794|3293|77281|1072744|6146|41501|10|21|476|3|68.91|0.00|68.91|46.47|27.36|26.18|20.08|22.65|73.83| +2451951|53623|757|22712|181393|6474|46755|2|22|477|30|1693.50|0.00|1693.50|24.84|1401.30|1608.82|28.79|55.89|1426.14| +2451871|55391|13448|36676|345415|2262|18709|1|11|478|30|668.70|46.80|715.50|74.75|438.00|334.35|133.74|200.61|559.55| +|33737|15588||||||35|480||173.66||180.60|80.64|98.04||35.55||185.62| +2451314|51326|12826|69961|1407|3877|23994|10|26|482|11|687.72|48.14|735.86|40.34|293.37|378.24|68.08|241.40|381.85| +2452580|43091|16416|19957|1143466|6488|43510|1|6|483|5|51.05|3.57|54.62|34.83|25.50|30.63|13.27|7.15|63.90| +2451194|37727|5858|23192|1355185|6783|44428|1|16|484|65|2910.70|232.85|3143.55|43.41|1101.10|2736.05|40.16|134.49|1377.36| +2452271|38182|10061|50551|1080604|121|10438|2|35|485|6|34.08|2.04|36.12|98.13|25.56|26.92|4.36|2.80|125.73| +2452327|56425|1169|50551|1509983|4815|38400|7|4|485|5|84.25|4.21|88.46|92.57|114.00|48.02|27.53|8.70|210.78| +2452349|50939|10397|50551|1115846|1029|39059|2|12|485|31|1488.62|89.31|1577.93|27.15|244.90|461.47|893.62|133.53|361.36| +2452265|40258|16821|5389|699784|3775|32566|8|33|486|9|59.67|4.77|64.44|83.90|549.00|56.08|3.33|0.26|637.67| +2452650|59842|7225|48687|661612|3337|24662|4|25|487|32|1690.88|118.36|1809.24|38.92|2040.64|152.17|1061.70|477.01|2197.92| +2451340|35778|10081|27327|685538|994|46450|7|2|489|31|460.97|36.87|497.84|96.12|1094.92|18.43|132.76|309.78|1227.91| +2452744|36198|7179|50712|1762705|4376|25247|2|19|494|16|322.88|29.05|351.93|85.62|163.84|196.95|64.22|61.71|278.51| +2452654|54249|13533|47754|1857893|6079|31818|2|33|497|29|1175.66|47.02|1222.68|57.17|297.25|305.67|713.39|156.60|401.44| +2452744|50145|4567|86799|115104|1093|41929|10|14|497|14|115.22|2.30|117.52|12.05|88.62|67.97|17.48|29.77|102.97| +2451964|56092|3175|52762|254170|1716|22462|4|31|498|13|439.53|30.76|470.29|91.81|131.82|347.22|11.07|81.24|254.39| +2451112|39643|14924|66301|644230|2690|42750|8|17|499|54|1040.04|52.00|1092.04|66.86|86.40|488.81|253.56|297.67|205.26| +2451096|44766|5026|66301|1151063|6174|30485|7|10|499|7|315.21|0.00|315.21|91.99|396.27|280.53|22.54|12.14|488.26| +2452587|44036|8959|16697|61744|6344|9391|7|13|500|15|20.55|1.64|22.19|52.36|5.40|16.85|2.59|1.11|59.40| +2452641|49431|5629|92812|94896|404|34759|7|32|501|32|3633.92|36.33|3670.25|48.27|1118.08|1053.83|1006.23|1573.86|1202.68| +2452523|50854|16951|92812|841781|2705|31182|2|28|501|12|256.56|17.95|274.51|95.52|346.80|207.81|11.21|37.54|460.27| +2452565|46576|8557|92812|883823|1554|41285|7|17|501|1|15.67|0.00|15.67|47.84|7.73|1.72|11.85|2.10|55.57| +2451017|32787|4630|80216|1321357|6971|20585|8|28|502|14|300.58|18.03|318.61|73.06|157.08|99.19|80.55|120.84|248.17| +2451572|35166|11600|46284|22210|2878|9946|7|2|503|18|1090.44|87.23|1177.67|56.53|394.74|686.97|149.28|254.19|538.50| +2451507|58815|15332|19854|377874|6647|37420|10|23|503|4|7.20|0.07|7.27|70.88|137.36|2.73|0.84|3.63|208.31| +2451630|30772|10486|46284|329148|5130|39865|7|12|503|20|2467.40|197.39|2664.79|55.12|286.80|49.34|2369.69|48.37|539.31| +2451609|49344|4795|46284|897569|1426|46883|4|34|503|16|1835.84|73.43|1909.27|66.18|1019.84|257.01|899.93|678.90|1159.45| +2452700|56783|11589|73671|1084810|7121|25032|8|24|504|3|83.94|5.03|88.97|90.45|5.97|52.04|28.39|3.51|101.45| +2452696|55031|1215|73671|450796|5381|36013|7|13|504|4|29.64|2.07|31.71|7.47|9.12|1.77|13.37|14.50|18.66| +||6823|24157|1299003||||31|505||43.44||45.61|95.73|||10.21||| +2451606|37138|9370|91588|969956|6862|20521|7|29|505|62|4015.12|0.00|4015.12|21.44|1297.66|803.02|513.93|2698.17|1319.10| +2452447|32316|883|11269|1841780|1496|3659|8|1|506|11|257.62|20.60|278.22|54.46|133.21|218.97|29.76|8.89|208.27| +2451796|45359|4382|23637|180000|4101|18555|4|10|508|49|252.84|7.58|260.42|59.99|87.71|75.85|162.83|14.16|155.28| +2451761|57274|12200|74922|778610|1942|12874|10|8|511|38|1892.40|18.92|1911.32|69.47|1774.22|567.72|198.70|1125.98|1862.61| +2451742|40754|2101|74922|1870779|946|27560|10|33|511|12|847.32|59.31|906.63|22.52|259.92|703.27|128.20|15.85|341.75| +2451640|35667|15941|30101|1707875|2896|40670|8|32|511|23|244.49|12.22|256.71|68.32|397.44|127.13|9.38|107.98|477.98| +2451749|52007|6584|47934|1595468|602|536|7|26|511|52|205.40|10.27|215.67|18.76|54.60|12.32|115.84|77.24|83.63| +2451660|51925|4088|74922|1469756|4120|16131|2|33|511|11|649.88|51.99|701.87|98.54|299.09|584.89|54.59|10.40|449.62| +2451648|44032|7327|74922|882522|4129|44136|10|3|511|49|175.91|0.00|175.91|89.88|43.61|56.29|9.56|110.06|133.49| +2451719|39703|4613|58371|1026817|6910|19756|7|13|514|14|226.80|11.34|238.14|12.38|209.30|0.00|18.14|208.66|233.02| +2452251|34128|11049|43346|251565|317|5022|10|24|515|3|20.97|1.67|22.64|3.76|89.70|8.80|10.46|1.71|95.13| +2452241|35589|2689|43346|414236|4460|37484|1|4|515|7|33.95|1.01|34.96|83.55|38.92|31.23|2.69|0.03|123.48| +2452741|58071|9949|47112|384415|3402|12389|10|33|516|17|76.33|2.28|78.61|72.85|111.69|72.51|3.74|0.08|186.82| +2451500|31154|12211|46327|893406|1220|30234|1|29|518|56|4122.16|247.32|4369.48|37.50|0.00|2225.96|1611.77|284.43|284.82| +2451767|55342|13082|25103|394412|5193|14821|2|33|520|18|1675.98|134.07|1810.05|20.58|93.06|1441.34|187.71|46.93|247.71| +2452299|29163|13236|21404|790426|5342|22258|10|20|522|22|747.34|29.89|777.23|15.52|33.88|171.88|546.68|28.78|79.29| +2452381|50003|10999|21404|883117|918|48766|1|25|522|8|280.00|14.00|294.00|2.14|43.52|254.80|16.63|8.57|59.66| +2452419|30008|8077|21404|1386468|3134|26200|2|7|522|95|716.30|21.48|737.78|60.85|43.70|128.93|135.09|452.28|126.03| +2451626|49866|8956|78322|458943|5203|41416|7|29|524|5|238.30|19.06|257.36|99.60|100.00|226.38|4.17|7.75|218.66| +2452406|39523|17388|8226|1664921|3216|49111|10|1|525|2|5.92|0.17|6.09|82.22|30.62|5.15|0.41|0.36|113.01| +2452374|35134|17127|21019|1217060|6171|16129|7|33|525|15|318.75|12.75|331.50|88.69|431.85|95.62|124.95|98.18|533.29| +2452626|42457|1851|84231|1668299|5167|31235|10|9|526|17|39.78|0.00|39.78|65.88|154.36|8.35|12.57|18.86|220.24| +2452714|34398|3925|63178|1288583|1444|31105|7|35|526|13|1094.47|0.00|1094.47|51.19|460.20|448.73|251.83|393.91|511.39| +2452638|42437|4345|24429|603304|4132|29007|2|35|526|14|444.22|39.97|484.19|21.33|156.80|53.30|46.91|344.01|218.10| +2452658|43427|2844|63178|424423|5577|41931|1|31|526|67|874.35|52.46|926.81|73.53|1420.40|507.12|179.94|187.29|1546.39| +2452536|43329|17611|15366|1806792|1707|17022|10|34|527|3|0.00|0.00|0.00|19.77|9.90|0.00|0.00|0.00|29.67| +2452595|45721|531|62007|1078203|5152|7348|7|28|527|11|398.75|23.92|422.67|13.99|453.09|23.92|18.74|356.09|491.00| +2452642|35276|15295|46254|1171852|7050|40201|1|17|527|11|60.06|5.40|65.46|23.94|273.79|45.64|7.21|7.21|303.13| +2452661|35782|4771|52212|995442|719|33949|8|16|528|1|36.37|2.18|38.55|56.62|35.19|2.54|3.04|30.79|93.99| +2452683|49649|4296|76826|521975|1055|8535|8|25|528|45|5211.00|260.55|5471.55|7.35|1695.15|2032.29|3178.71|0.00|1963.05| +2451497|39740|7544|55339|692958|2224|9443|1|8|529|15|153.00|13.77|166.77|40.90|102.00|149.94|0.58|2.48|156.67| +2451401|30329|14983|79556|468887|137|3387|1|27|529|25|1764.00|158.76|1922.76|89.09|283.50|652.68|944.62|166.70|531.35| +2452720|34411|14047|36249|1473623|5642|3261|10|14|530|19|376.96|15.07|392.03|76.28|665.19|324.18|9.50|43.28|756.54| +2452739||16698|36249||||||530|45||||60.72||||525.80|223.77| +2452043|52940|13871|45480|1876143|4669|26795|4|12|532|3|30.75|2.76|33.51|24.31|11.76|25.83|2.11|2.81|38.83| +2451930|55646|15621|56955|1174851|2818|7585|4|26|534|2|196.66|13.76|210.42|46.47|19.34|106.19|74.18|16.29|79.57| +2452006|36758|15743|56955|629041|6398|555|4|7|534|4|132.80|9.29|142.09|3.21|83.44|92.96|23.10|16.74|95.94| +|51814|2038|47354|612725|4374|13223||11|535|2|101.66|1.01||41.34|74.12||13.17||| +2451019|42433|17707|47354|25098|6812|23373|4|33|535|46|2696.52|80.89|2777.41|18.09|126.50|2534.72|110.02|51.78|225.48| +2451574|45059|8818|74934|534976|1569|21362|8|8|536|2|12.94|1.16|14.10|12.12|15.94|6.98|1.72|4.24|29.22| +2451558|51397|7033|74934|1794274|4143|24059|4|14|536|9|849.51|42.47|891.98|2.21|468.72|509.70|237.86|101.95|513.40| +2452179|30825|14009|25761|1466800|3765|22330|4|18|537|81|4754.70|332.82|5087.52|36.14|1123.47|2710.17|1594.73|449.80|1492.43| +2452217|30972|14579|25761|989405|3242|25479|8|30|537|10|1268.80|0.00|1268.80|89.58|261.20|1091.16|33.75|143.89|350.78| +2452187|57050|8403|27085|145643|4554|11112|4|11|537|30|0.00|0.00|0.00|78.72|939.90|0.00|0.00|0.00|1018.62| +2452131|55946|5713|25761|302758|2044|26977|7|3|537|24|47.28|2.36|49.64|68.95|51.36|6.14|13.16|27.98|122.67| +2452651|49141|6798|29851|1839817|4147|42477|10|14|538|60|140.40|1.40|141.80|66.63|982.80|92.66|21.48|26.26|1050.83| +2452593|47192|11058|29851|1612227|4870|29560|8|8|538|33|1040.49|72.83|1113.32|16.13|854.70|10.40|679.85|350.24|943.66| +2452425|51829|3823|67256||||2|10|539|10|313.80|0.00|||173.40||35.21||185.37| +2451663|28873|2896|59704|949130|4037|38897|10|6|540|50|670.00|26.80|696.80|1.34|116.00|93.80|132.52|443.68|144.14| +2451955|56775|6521|41114|182487|6260|5680|1|1|543|3|47.85|0.00|47.85|92.19|85.77|27.27|13.37|7.21|177.96| +2451882|49081|16157|23568|1645338|3172|27406|10|12|543|70|531.30|5.31|536.61|79.88|2127.30|334.71|180.86|15.73|2212.49| +2451522|46657|760|35957|1800874|3305|17783|7|17|544|48|931.68|27.95|959.63|91.49|21.12|875.77|2.23|53.68|140.56| +2451458|56050|8992|35957|1831380|5745|22110|7|32|544|4|346.92|6.93|353.85|51.67|128.20|326.10|14.15|6.67|186.80| +2451491|57105|12925|14440|931581|348|20762|1|6|545|8|226.24|9.04|235.28|60.58|207.84|196.82|10.00|19.42|277.46| +2451583|59019|7771|3297|1684392|1913|4634|2|28|547|6|467.40|32.71|500.11|64.79|184.98|434.68|0.32|32.40|282.48| +2451576|58213|2660|3297|707232|3646|31707|8|25|547|14|273.98|0.00|273.98|99.67|132.72|2.73|179.02|92.23|232.39| +2451170|48956|15664|55173|1911864|5367|45270|10|33|548|24|1788.48|53.65|1842.13|72.78|1155.60|214.61|503.63|1070.24|1282.03| +2451874|32077|15139|17263|1865364|600|3789|1|9|551|5|387.90|27.15|415.05|63.64|173.05|77.58|173.77|136.55|263.84| +2451917|33207|8881|17263|218460|1818|15929|4|13|551|3|249.90|2.49|252.39|66.89|114.15|107.45|31.33|111.12|183.53| +2451308|36660|12580|62919|1897507|7111|22315|10|35|552|35|1168.30|70.09|1238.39|29.30|1422.40|35.04|883.94|249.32|1521.79| +2451438|36454|3070|51310|1833674|3238|18402|10|25|552|3|3.27|0.16|3.43|37.52|10.56|0.29|2.32|0.66|48.24| +2451528|46646|2812|37221|1877243|1310|5377|7|6|554|2|174.92|15.74|190.66|12.82|7.86|3.49|1.71|169.72|36.42| +2450998|42512|1478|90403|97886|2169|99|10|20|555|16|176.80|15.91|192.71|89.23|238.24|22.98|44.60|109.22|343.38| +2451198|40722|15142|66916|855354|1235|3963|7|30|556|46|266.34|15.98|282.32|26.34|218.96|66.58|159.80|39.96|261.28| +2451099|41748|12692|66916|||49518|||556|67||16.89|861.76||2172.81|185.87|||2231.85| +2452651|30023|12099|95038|337586|2087|34635|10|12|557|57|2737.71|246.39|2984.10|16.76|1406.76|2737.71|0.00|0.00|1669.91| +2452628|43036|17323|65883|175907|4916|36325|7|3|558|42|672.00|53.76|725.76|25.89|203.28|598.08|13.30|60.62|282.93| +2452590|37734|4027|76762|1288224|1003|25735|1|26|559|4|21.04|1.05|22.09|21.84|21.04|6.52|9.43|5.09|43.93| +2452060|31961|3453|48449|635184|1537|23443|4|7|560|3|347.07|24.29|371.36|97.79|80.07|347.07|0.00|0.00|202.15| +2451018|43125|15337|50857|349348|3416|13484|7|10|562|4|5.80|0.17|5.97|61.32|4.84|2.49|2.64|0.67|66.33| +2452482|52851|9870|81884|981127|4817|2860|4|26|563|31|0.00|0.00|0.00|95.32|1815.36|0.00|0.00|0.00|1910.68| +2451196|40016|15740|97665|1405103|2574|32492|4|14|564|28|1191.68|107.25|1298.93|65.99|908.04|810.34|350.83|30.51|1081.28| +2451411|51168|11989|48561|687781|2749|29364|1|9|567|9|263.43|23.70|287.13|28.63|13.50|255.52|6.24|1.67|65.83| +2451495|34846|3577|5246|639692|5691|4090|7|25|567|62|8707.90|609.55|9317.45|75.72|678.28|3831.47|2779.56|2096.87|1363.55| +2452800|42878|12336|7582|1175568|690|11059|8|18|573|41|1245.99|37.37|1283.36|56.49|91.02|573.15|545.00|127.84|184.88| +2451525|56954|4117|40209|1544732|3655|21425|2|1|575|69|2015.49|120.92|2136.41|43.64|805.92|1209.29|765.89|40.31|970.48| +2451571|41579|5035|80964|1140719|2074|49782|1|2|575|10|229.80|6.89|236.69|69.38|204.20|73.53|18.75|137.52|280.47| +2451518|32018|9166|83196|404279|5362|22445|10|11|576|5|78.10|3.90|82.00|41.95|115.35|50.76|14.76|12.58|161.20| +2451486|47209|17758|22350|1611917|5131|5898|1|22|576|38|3794.30|341.48|4135.78|80.77|50.92|37.94|2779.70|976.66|473.17| +2451722|36638|17363|93540|1882940|3601|48662|2|31|578|29|315.52|9.46|324.98|89.57|1233.95|22.08|23.47|269.97|1332.98| +2451736|53203|8147|93540|1171831|2833|31388|2|5|578|2|58.24|2.32|60.56|81.57|17.68|11.06|18.40|28.78|101.57| +2452465|52527|11647|73996|1877720|4623|5173|8|23|579|48|1729.44|51.88|1781.32|57.20|326.88|812.83|476.63|439.98|435.96| +2452614|55869|15637|58916|579311|1659|12854|4|7|579|4|177.96|7.11|185.07|83.13|110.80|55.16|17.19|105.61|201.04| +2451850|41177|14633|62242|441333|3942|151|2|9|580|18|99.54|7.96|107.50|54.88|87.12|59.72|8.76|31.06|149.96| +2452267|44121|15697|24865|767944|1423|41302|10|33|582|15|86.10|6.88|92.98|95.15|31.05|22.38|21.66|42.06|133.08| +2452458|30315|3768|55634|1045496|167|28885|2|4|583|41|3515.75|70.31|3586.06|52.00|366.13|210.94|561.81|2743.00|488.44| +2452404|33111|12048|17301|478176|7051|19526|7|28|583|6|61.68|1.23|62.91|33.37|4.62|6.16|34.42|21.10|39.22| +2451316|30422|17876|51758|1144304|6450|31396|2|33|585|19|0.00|0.00|0.00|28.35|521.55|0.00|0.00|0.00|549.90| +2452586|38692|7182|95658|1528930|3802|19074|7|23|586|60|4470.00|402.30|4872.30|75.94|1543.20|4201.80|134.10|134.10|2021.44| +2452414|36659|10203|3866|1526556|2493|32383|4|28|590|3|100.17|5.00|105.17|51.51|32.43|61.10|27.73|11.34|88.94| +2452088|44684|13625|80973|556612|4273|26781|8|32|591|3|124.77|6.23|131.00|66.56|143.70|69.87|21.96|32.94|216.49| +2452094|50756|4419|80973|540115|7185|41455|4|7|591|43|712.08|14.24|726.32|75.67|538.79|576.78|69.00|66.30|628.70| +2452164|30098|107|80973|528576|2222|33068|2|30|591|1|19.69|0.39|20.08|40.95|2.58|10.04|1.06|8.59|43.92| +2451100|32895|4088|74705|1044053|4098|31743|2|28|593|45|75.60|3.02|78.62|86.67|77.85|6.80|37.84|30.96|167.54| +2451195|33944|12446|30219|1753062|3120|34825|10|25|595|26|1080.56|86.44|1167.00|74.33|540.28|421.41|257.06|402.09|701.05| +2451185|39963|1688|85676|651112|1701|8337|8|22|596|50|4503.50|270.21|4773.71|91.20|0.00|2386.85|1502.82|613.83|361.41| +2451191|42717|412|70467|1453593|6426|21659|10|10|596|5|157.20|6.28|163.48|14.96|77.60|66.02|43.76|47.42|98.84| +2451324|47172|1888|17452|670669|3062|20208|8|29|597|37|266.40|2.66|269.06|27.49|450.66|122.54|57.54|86.32|480.81| +2451402|47191|12272|90510|776000|4437|45674|7|10|598|19|1162.42|11.62|1174.04|1.70|700.53|69.74|775.80|316.88|713.85| +2451305|59825|5684|90510|1429766|5117|26468|8|28|598|12|746.88|52.28|799.16|55.92|211.92|597.50|65.72|83.66|320.12| +2451287|48512|17702|28882|1632175|715|28008|8|29|598|11|779.68|7.79|787.47|91.14|129.91|163.73|369.57|246.38|228.84| +2451976|52148|12907|80992|995406|2439|7472|8|3|599|21|1063.02|21.26|1084.28|76.92|777.42|0.00|829.15|233.87|875.60| +2452001|60296|4421|80992|665010|5422|41034|2|3|599|93|3972.03|158.88|4130.91|29.11|2016.24|3932.30|17.48|22.25|2204.23| +2451986|33236|1670|76235|573191|3302|7813|2|10|600|30|2857.50|228.60|3086.10|93.27|825.30|85.72|221.74|2550.04|1147.17| +2451990|56449|15865||1091994|2092||||600||129.03|9.03|138.06|37.92||83.86|||| +2452045|44472|10267|76235|1833402|717|15232|10|19|600|41|924.14|55.44|979.58|14.96|2094.69|120.13|498.48|305.53|2165.09| +2452641|49741|907|78894|1883173|551|24682|7|8|603|18|1071.00|53.55|1124.55|51.42|449.46|0.00|224.91|846.09|554.43| +2451889|56820|14777|34912|1367001|1713|1625|1|29|605|5|67.35|6.06|73.41|30.26|30.80|45.12|1.55|20.68|67.12| +2451011|59566|205|55102|1247623|497|23418|10|23|606|57|4394.13|219.70|4613.83|73.54|1656.42|659.11|1494.00|2241.02|1949.66| +2452066|32420|12644|34423|823624|4916|49855|8|34|607|10|891.50|0.00|891.50|49.02|256.80|213.96|359.09|318.45|305.82| +2451559|38974|2923|66709|697853|3928|50|8|20|608|52|2436.72|48.73|2485.45|95.20|1624.48|1120.89|644.75|671.08|1768.41| +2451587|59014|5773|66709|1797883|5745|6007|4|26|608|46|2692.84|53.85|2746.69|45.43|1164.26|888.63|1244.90|559.31|1263.54| +2451585|54580|15451|20763|200407|5204|15648|10|1|608|9|914.76|54.88|969.64|18.47|106.65|311.01|326.02|277.73|180.00| +2451665||13843||1611184|1196||1||608|22|776.16|31.04||64.88|347.82||59.76|48.91|443.74| +2451009||17713|92557|1179840|3947|||20|610|77|||2219.14||661.43|1153.95|||| +2452360|52829|15069|20558|1756126|1628|45177|8|3|612|65|7037.55|633.37|7670.92|7.38|1539.20|3378.02|146.38|3513.15|2179.95| +2452305|43611|1281|66979|1413|3047|32281|4|9|612|32|1880.64|150.45|2031.09|91.67|283.20|1673.76|122.05|84.83|525.32| +2451706|34043|17927|32278|845704|6683|43309|7|19|614|5|23.15|1.62|24.77|66.06|1.40|18.52|2.96|1.67|69.08| +2451568|45739|6844|81555|752936|6110|437|1|30|616|10|125.20|11.26|136.46|89.34|169.90|7.51|61.19|56.50|270.50| +2451567|28955|11552|81555|1853953|3351|31996|2|9|616|78|2052.18|143.65|2195.83|47.05|1266.72|574.61|1196.83|280.74|1457.42| +2451653|45661|430|70532|510340|5348|29909|8|35|616|71|629.06|18.87|647.93|9.55|1048.67|484.37|49.19|95.50|1077.09| +2451712|29906|14785|53238|51410|4893|7976|8|12|618|15|220.80|17.66|238.46|8.88|190.35|57.40|135.62|27.78|216.89| +2451688|54033|6722|53238|||8660|10|35|618||||||78.69||58.45|68.62|| +2451137|48231|5560|6694|18687|6067|11847|4|10|619|19|579.12|40.53|619.65|56.25|176.70|23.16|305.77|250.19|273.48| +2452162|47833|349|40289|1609898|1666|33034|2|20|620|10|19.00|1.33|20.33|95.88|11.70|9.12|8.79|1.09|108.91| +|45557|12886|50721|1464387|||||625|13|586.17|||||64.47|||259.41| +2452698|41342|7555|9964|1304785|3506|5474|1|21|626|71|4259.29|255.55|4514.84|21.55|0.00|1448.15|1040.12|1771.02|277.10| +2452661|37519|15007|9964|424387|3071|24955|8|33|626|4|82.64|0.00|82.64|27.36|28.60|47.10|31.63|3.91|55.96| +2452553|61038|9696|58153|1520756|4310|35864|8|14|629|7|106.26|7.43|113.69|18.16|27.58|45.69|6.05|54.52|53.17| +2452563|57361|12549|58153|67572|4351|38683|1|19|629|1|41.37|3.30|44.67|16.80|20.91|40.54|0.78|0.05|41.01| +2451897|56068|11786|45415|917804|1840|19029|2|12|630|1|90.91|1.81|92.72|34.91|29.27|79.09|10.63|1.19|65.99| +2452460|53236|1740|49012|1320150|6244|43985|1|30|632|36|1120.68|22.41|1143.09|78.80|831.60|616.37|146.24|358.07|932.81| +2452529|31552|8730|49012|1689426|5475|31524|1|7|632|3|12.72|0.63|13.35|96.89|7.41|6.36|2.54|3.82|104.93| +2451573|47690|17971|66457|360332|5436|8940|2|33|633|33|867.57|26.02|893.59|18.46|293.37|668.02|137.68|61.87|337.85| +2451538|31413|664|97801|1818848|5048|24895|2|19|634|16|64.96|1.29|66.25|68.42|77.60|10.39|7.63|46.94|147.31| +2451563|51440|16670|37740|1781714|2841|8815|1|16|634|11|137.39|4.12|141.51|37.67|55.44|89.30|45.68|2.41|97.23| +2451131|53562|3079|80162|1903131|6476|10221|1|18|635|10|71.80|0.00|71.80|3.17|75.40|15.07|44.24|12.49|78.57| +2450985|53765|3760|77639|1207731|2623|19115|8|9|635|15|13.20|0.66|13.86|18.63|93.00|4.75|2.87|5.58|112.29| +2451907|52078|7339|74391|684981|6240|47499|7|27|636|23|1580.79|31.61|1612.40|12.16|658.49|1090.74|323.43|166.62|702.26| +2451952|31683|1034|54121|652131|4236|11541|2|1|639|1|3.89|0.15|4.04|10.56|10.39|3.22|0.52|0.15|21.10| +2451585|50834|2680|1542|794398|2755|11159|7|5|641|2|41.96|1.25|43.21|59.58|86.04|27.27|10.28|4.41|146.87| +2452625|59365|7377|41211|1258449|1723|20197|1|22|642|68|1914.20|76.56|1990.76|22.67|514.08|535.97|206.73|1171.50|613.31| +2452695|40780|13279|41211|243679|3106|30886|4|30|642|33|2059.53|144.16|2203.69|95.17|209.88|1935.95|64.26|59.32|449.21| +2452626|37726|1416|41211|1744684|5461|49885|2|11|642|33|1034.88|41.39|1076.27|28.31|731.94|869.29|124.19|41.40|801.64| +2452459|40731|17523|66696|703672|6684|18896|10|15|643|6|131.58|0.00|131.58|25.50|104.76|27.63|93.55|10.40|130.26| +2451579|47721|6622|45052|1550492|6138|43337|4|19|644|38|1583.84|63.35|1647.19|35.89|736.82|110.86|662.84|810.14|836.06| +2451686|29844|6632|45052|444990|4877|48811|2|20|644|46|1178.52|82.49|1261.01|67.84|950.36|141.42|62.22|974.88|1100.69| +2452404|52143|11173|49358|198545|3959|4973|4|29|645|5|122.55|7.35|129.90|44.66|41.85|107.84|4.41|10.30|93.86| +2451342|42115|15574|76841|878665|1794|12123|10|6|646|10|238.40|19.07|257.47|89.43|163.10|7.15|85.56|145.69|271.60| +2451196|58696|15961|75692|1619749|606|11863|8|9|646|1|60.18|3.00|63.18|52.37|17.59|26.47|9.10|24.61|72.96| +2451339|47641|11912|82928|1365155|6135|5962|7|24|646|26|2337.66|210.38|2548.04|37.65|405.08|1028.57|981.81|327.28|653.11| +2452247|49557|5625|14319|973938|2357|34776|8|24|648|30|2299.80|183.98|2483.78|98.67|1197.90|2299.80|0.00|0.00|1480.55| +2452259|53962|6277|40183|749244|705|36966|10|35|648|32|2058.56|82.34|2140.90|81.05|671.04|617.56|763.73|677.27|834.43| +2452372|48458|16971|26202|707491|2421|42551|1|21|648|33|1179.09|0.00|1179.09|19.60|689.04|330.14|288.64|560.31|708.64| +2452294|43009|7763|97227|1447722|7017|27559|2|4|649|10|652.00|58.68|710.68|5.04|110.30|547.68|26.08|78.24|174.02| +2452209|37945|17437|56565|1835545|4925|41505|10|27|649|17|411.40|12.34|423.74|75.00|720.12|28.79|122.43|260.18|807.46| +2452615|59513|12571|60299|524713|5517|40321|7|27|652|33|1360.59|81.63|1442.22|49.86|874.83|394.57|821.11|144.91|1006.32| +2451903|52189|2845|20358|665107|5522|19937|8|20|653|14|933.24|9.33|942.57|96.68|56.14|475.95|150.90|306.39|162.15| +2451868|45575|17813|59853|557092|5084|7523|8|29|653|75|3621.00|289.68|3910.68|42.90|987.00|1882.92|260.71|1477.37|1319.58| +2452370|29459|11925|1309|353592|4799|16453|8|1|654|86|734.44|14.68|749.12|12.07|371.52|168.92|135.72|429.80|398.27| +2452226|48188|17875|77145|856380|311|21042|1|29|655|6|72.48|2.17|74.65|67.80|13.56|7.24|38.49|26.75|83.53| +2451147|50591|5764|76516|721350|3413|41829|1|1|656|15|300.90|3.00|303.90|11.57|309.30|267.80|12.24|20.86|323.87| +2451893|40714|17096|75587|1411040|4540|43640|4|18|657|19|2113.94|126.83|2240.77|8.74|786.60|2092.80|17.33|3.81|922.17| +2451681|33133|4681|37044|1841085|2726|28767|2|4|658|36|459.72|27.58|487.30|61.69|370.08|399.95|41.24|18.53|459.35| +2451634|39836|12902|24830|674556|5882|32055|1|7|658|19|0.00|0.00|0.00|64.84|151.81|0.00|0.00|0.00|216.65| +2452266|52446|10997|69457|905575|4230|46438|10|26|659|35|533.75|48.03|581.78|58.80|776.65|96.07|428.92|8.76|883.48| +2452337|37873|15933|69457|1612663|6625|28987|1|1|659|36|1746.36|69.85|1816.21|39.72|698.40|209.56|1244.80|292.00|807.97| +2451352|51321|11062|36789|690386|6974|5162|2|13|662|10|753.70|30.14|783.84|24.20|181.40|557.73|52.91|143.06|235.74| +2451475|49807|5107|64431|202141|6913|4885|8|19|663|11|823.46|74.11|897.57|10.40|156.75|238.80|76.00|508.66|241.26| +2451171|53971|15280|16969|483662|2365|42315|10|3|664|43|61.92|5.57|67.49|42.74|384.42|55.72|0.37|5.83|432.73| +2451155|45232|10418|63347|16622|3944|5619|7|9|664|6|271.80|0.00|271.80|33.63|27.48|182.10|0.89|88.81|61.11| +2451271|46744|17926|16969|158375|2934|18740|4|8|664|24|109.20|0.00|109.20|58.37|87.36|56.78|4.19|48.23|145.73| +2452108|37068|8047|76312|1551016|2364|22512|10|31|665|14|501.62|25.08|526.70|69.19|266.84|145.46|56.98|299.18|361.11| +2451484|44641|10939|91095|607702|2262|20022|4|4|667|55|82.50|3.30|85.80|95.10|1905.20|44.55|14.42|23.53|2003.60| +|34034|6283|59598|947630||||3|671|42|2219.28|88.77|2308.05|18.19|1073.52||956.95|1217.95|| +2452250|51167|7235|59598|1708849|4434|9972|10|15|671|23|884.81|17.69|902.50|24.29|448.27|734.39|85.73|64.69|490.25| +2451829|32864|4430|42995|1709885|1128|39618|1|11|672|1|34.90|1.04|35.94|72.75|17.64|20.59|10.01|4.30|91.43| +2452249|58692|5521|41683|245480|1855|30630|1|27|673|47|2328.85|163.01|2491.86|23.48|242.99|652.07|117.37|1559.41|429.48| +2451159|57291|10304|91437|1596587|6799|42611|1|10|674|36|2356.20|188.49|2544.69|33.68|1177.92|777.54|805.11|773.55|1400.09| +2451191|40588|9787|89850|1290862|689|36469|2|8|675|45|1009.35|70.65|1080.00|49.89|2299.05|181.68|761.45|66.22|2419.59| +2451190|28889|188|89850|1417514|6921|41175|7|9|675|45|2363.40|94.53|2457.93|57.53|1902.15|614.48|402.25|1346.67|2054.21| +2451929|33388|8846|46710|622903|2152|19354|8|15|676|12|84.96|7.64|92.60|37.38|22.92|62.02|10.09|12.85|67.94| +2452121|50652|2431|49907|481725|6674|36376|10|2|678|2|85.10|0.00|85.10|9.00|8.62|0.85|82.56|1.69|17.62| +2450996|33681|8990|46858|1026519|4688|48904|8|9|679|74|1442.26|14.42|1456.68|72.78|119.88|317.29|337.49|787.48|207.08| +2451031|57783|4286|46858|1034902|330|46205|1|28|679|44|331.32|26.50|357.82|85.72|2486.44|36.44|182.82|112.06|2598.66| +2451886|56710|593|70701|334391|2434|524|1|2|681|38|167.58|1.67|169.25|62.28|167.58|144.11|9.85|13.62|231.53| +2451724|30383|4927|11650|1212999|6899|35952|1|5|682|29|59.16|4.14|63.30|48.39|46.11|30.76|7.66|20.74|98.64| +2451756|55590|8726|65503|163576|1649|3322|1|28|682|41|86.10|3.44|89.54|10.72|147.60|78.35|7.75|0.00|161.76| +2451653|43420|6953|30740|338113|1839|26412|8|1|682|6|106.26|5.31|111.57|58.61|6.90|5.31|44.41|56.54|70.82| +2451778|52085|16796|11650|48194|427|14213|2|2|682|50|1714.50|68.58|1783.08|52.79|1192.50|1285.87|390.05|38.58|1313.87| +2451007||12286|31370||6061|43690|2|11|683|||0.00|906.49|||716.12|125.64|64.73|415.61| +2451174|48953|16579|46585|749449|4008|16214|1|2|685|2|64.82|0.00|64.82|6.03|21.60|7.13|0.57|57.12|27.63| +2451090|30410|6175|67185|526429|2473|29364|10|17|685|26|2135.90|42.71|2178.61|69.42|696.28|2029.10|106.80|0.00|808.41| +2451491|35328|15434|19244|92208|4725|23642|2|18|686|56|3600.24|144.00|3744.24|82.21|812.56|324.02|1736.39|1539.83|1038.77| +2451966|57665|15337|16379|383948|1633|28471|2|14|687|13|0.00|0.00|0.00|84.19|21.97|0.00|0.00|0.00|106.16| +2452639|55537|13506|2703|691135|4814|36493|1|19|688|14|251.86|20.14|272.00|11.38|122.08|45.33|136.30|70.23|153.60| +2452012||15446||||36446||20|689|||35.65|||||||478.27| +2452014|55207|3433|90229|254107|3372|20684|8|34|689|15|115.20|5.76|120.96|75.61|109.50|44.92|46.38|23.90|190.87| +2452301|46070|15861|71807|1758295|2869|20243|7|29|690|8|380.16|19.00|399.16|86.75|18.32|235.69|143.02|1.45|124.07| +2451236|49871|9184|42188|237306|4480|15254|8|5|691|16|513.12|35.91|549.03|30.99|66.08|343.79|15.23|154.10|132.98| +2451281|53199|16729|15968|707355|2742|21729|2|13|691|46|517.96|5.17|523.13|47.67|284.74|72.51|280.63|164.82|337.58| +2451049|34676|16699|56275|1062054|4633|4812|2|7|693|30|107.40|1.07|108.47|6.16|658.80|15.03|59.11|33.26|666.03| +2450944|60704|7022|98735|1883035|2888|388|4|11|693|24|1317.36|13.17|1330.53|83.11|764.40|1040.71|41.49|235.16|860.68| +2450969|56496|17065|58716|704294|4085|35476|8|33|693|1|12.77|0.00|12.77|42.58|15.96|9.57|1.92|1.28|58.54| +2451892|45693|3157|96731|1766511|96|47281|10|21|694|27|55.62|2.78|58.40|56.64|140.67|32.81|5.47|17.34|200.09| +2451354|59370|133||1084872|3585|3451|||695||973.20|77.85|1051.05|1.53|||||| +2451354|39962|4459|48573|47392|4497|16449|7|31|695|21|1598.52|95.91|1694.43|13.51|564.06|1534.57|42.20|21.75|673.48| +2452299|38569|4451|68494|1866159|4001|30290|7|11|696|16|1409.12|126.82|1535.94|20.65|1273.60|0.00|620.01|789.11|1421.07| +2450960|44953|2924|20223|1632701|5836|30171|7|16|699|6|391.26|23.47|414.73|76.09|223.56|348.22|12.05|30.99|323.12| +2451039|31947|4336|20223|968095|4759|32670|2|18|699|2|22.50|0.67|23.17|5.81|31.12|8.55|1.11|12.84|37.60| +2451552|60104|13387|20676|21337|5998|16259|8|19|700|45|2838.15|170.28|3008.43|81.42|1680.30|2611.09|34.05|193.01|1932.00| +2452296|50307|17065|19152|1575552|5109|35066|4|5|701|61|317.81|3.17|320.98|58.55|267.79|174.79|65.78|77.24|329.51| +2452240|32866|17473|19152|1182733|2962|14791|2|27|701|10|150.40|10.52|160.92|34.08|17.10|40.60|87.84|21.96|61.70| +2452608|58645|3918|24177|183174|6041|39708|4|22|702|72|2143.44|128.60|2272.04|48.06|0.00|600.16|324.08|1219.20|176.66| +2452765|48759|10629|91838|285092|2568|12044|2|16|707|32|1173.76|46.95|1220.71|49.75|603.52|1091.59|75.59|6.58|700.22| +2452713|37829|8304|91838|1435029|547|42912|2|18|707|41|1756.85|105.41|1862.26|55.15|1613.35|1247.36|81.51|427.98|1773.91| +2451935|44069|1457|||1337||10|16|708||867.70||928.43|84.68||173.54|||| +2451964|57322|15979|51117|855766|485|9604|2|19|708|9|84.06|5.88|89.94|46.93|11.79|15.97|61.96|6.13|64.60| +2452002|45939|11192|51117|1409140|5487|8681|10|27|708|1|96.94|1.93|98.87|2.02|30.08|65.91|11.48|19.55|34.03| +2452738|53021|17007|23335|318293|1222|21798|8|31|709|68|2606.44|104.25|2710.69|65.97|1474.92|1407.47|11.98|1186.99|1645.14| +2452729|54589|5209|23335|1463125|224|12970|4|30|709|2|225.46|20.29|245.75|34.88|81.76|29.30|170.65|25.51|136.93| +2451221|59557|11096|98793|1234957|7094|47278|1|14|710|44|2650.12|53.00|2703.12|32.22|1520.64|954.04|1441.66|254.42|1605.86| +2451915|59526|12074|52319|1190327|665|32650|2|23|711|43|3884.62|233.07|4117.69|53.35|1071.56|2719.23|466.15|699.24|1357.98| +2452023|49798|989|52319|419440|4599|28605|1|15|711|84|633.36|19.00|652.36|83.86|845.04|576.35|12.54|44.47|947.90| +2452762|51422|11106|87767|1540671|3151|13231|10|4|712|31|418.50|33.48|451.98|68.80|156.86|351.54|8.03|58.93|259.14| +2452759|46621|13158|87767|1397087|6626|48965|7|13|712|27|2753.73|55.07|2808.80|50.56|886.14|1569.62|532.84|651.27|991.77| +2452758|54829|16417|87767|1845970|1106|195|1|27|712|27|1946.70|116.80|2063.50|33.33|740.34|1090.15|591.01|265.54|890.47| +2452670|58894|9307|87767|334776|4742|41508|8|18|712|2|22.58|0.22|22.80|43.23|67.78|16.03|1.96|4.59|111.23| +2452657|35186|9703|47605|659731|5389|13094|2|27|714|38|1918.24|134.27|2052.51|62.64|434.34|1553.77|65.60|298.87|631.25| +2452270|56025|8237|69133|1786861|6926|8991|1|3|715|64|5415.04|433.20|5848.24|78.78|611.20|2490.91|2573.23|350.90|1123.18| +2452229|35746|8937|69133|1727480|2825|3017|1|19|715|12|49.68|0.99|50.67|91.10|347.88|44.71|2.28|2.69|439.97| +2452008|60636|2059|33157|1596111|7054|12750|8|1|716|6|171.72|8.58|180.30|43.49|404.04|25.75|78.82|67.15|456.11| +2452584|32994|14419|497|72729|5869|656|7|11|717|23|204.01|6.12|210.13|37.68|459.08|124.44|28.64|50.93|502.88| +2452289|48047|15627|57624|347952|591|6830|8|18|719|13|272.09|0.00|272.09|61.46|91.91|225.83|27.75|18.51|153.37| +2452274|34331|12921|57624|1164644|2516|19635|10|31|719|10|954.60|19.09|973.69|94.62|464.70|477.30|195.69|281.61|578.41| +2452582|46934|4137|34233|1469052|5471|26290|1|6|720|72|606.96|0.00|606.96|50.49|166.32|430.94|33.44|142.58|216.81| +2452656|44238|15597|31729|817407|3085|49563|8|14|720|7|165.06|13.20|178.26|89.65|68.74|42.91|40.30|81.85|171.59| +2452675|36117|13188|95979|75738|695|452|4|1|720|9|171.18|11.98|183.16|30.67|342.45|6.84|161.05|3.29|385.10| +2452531|52100|11460|95979|1047381|229|9477|4|23|720|8|377.44|15.09|392.53|64.91|256.08|75.48|147.96|154.00|336.08| +2450876|33313|13120|3505|840875|4198|768|1|19|721|21|1617.42|64.69|1682.11|11.75|858.48|1504.20|66.79|46.43|934.92| +2451002|45953|17656|3171|1318908|4692|14429|7|10|721|87|1204.08|96.32|1300.40|46.81|2910.15|457.55|186.63|559.90|3053.28| +2451078|54673|8576|42235|385129|4300|24671|2|6|723|32|396.80|11.90|408.70|51.00|128.64|317.44|49.99|29.37|191.54| +2451887|41710|15355|16166|730337|1509|6634|10|3|724|37|949.42|47.47|996.89|70.88|9.99|816.50|112.98|19.94|128.34| +2451093|46115|17851|31650|1521955|5860|27765|2|25|725|45|1770.30|141.62|1911.92|32.58|673.20|1221.50|422.57|126.23|847.40| +2451923|29159|14329|18239|944805|2126|19094|8|31|726|90|4609.80|184.39|4794.19|83.68|2108.70|645.37|673.95|3290.48|2376.77| +2450998|53337|10754|75432|338897|5115|8580|2|33|727|20|1135.60|45.42|1181.02|28.61|1357.20|579.15|255.96|300.49|1431.23| +2451169|50058|17047|37234|210900|4572|8597|1|4|728|5|171.15|6.84|177.99|69.10|55.50|159.16|9.71|2.28|131.44| +2451105|57997|628|50230|356366|5702|16638|2|30|728|36|1192.32|23.84|1216.16|23.16|501.84|715.39|119.23|357.70|548.84| +2451106|43726|6973|37234|983903|1883|8263|8|23|728|25|1657.50|99.45|1756.95|78.62|743.75|1640.92|8.62|7.96|921.82| +|39045|7022|||3654|49774|4|16|729|||0.00||36.83|||396.87|111.94|303.33| +2452791|31503|1026|15002|1600350|923|46649|10|23|730|70|153.30|13.79|167.09|52.96|1898.40|87.38|54.71|11.21|1965.15| +2452743|36202|6783|49134|1198242|4950|7258|10|9|730|3|93.48|4.67|98.15|57.93|38.76|21.50|2.87|69.11|101.36| +2452713|36941|6009|41020|1514071|5239|5654|10|7|731|2|19.62|0.98|20.60|98.96|7.74|4.51|6.04|9.07|107.68| +2452417|42028|2581|16772|1306666|3424|40982|4|31|733|26|0.00|0.00|0.00|91.57|408.98|0.00|0.00|0.00|500.55| +2452518|49405|5191|16772|830716|6755|25994|8|20|733|23|137.31|1.37|138.68|73.08|392.38|100.23|11.49|25.59|466.83| +2452493|55348|17136|16772|265335|2052|1522|4|16|733|4|210.88|12.65|223.53|90.05|205.60|164.48|44.08|2.32|308.30| +2452112|52446|12069|82837|1266312|3458|3757|10|22|734|13|52.26|4.18|56.44|13.06|65.26|25.08|5.16|22.02|82.50| +2451924|36909|9830|5852|206450|6488|22973|4|21|735|2|22.70|0.45|23.15|1.84|12.38|0.68|17.61|4.41|14.67| +2451971|50736|8051|48692|1747375|5705|17988|1|19|735|27|537.57|5.37|542.94|18.46|170.91|462.31|54.93|20.33|194.74| +2451941|29478|47|5852|1728387|1767|30794|7|9|735|15|342.00|3.42|345.42|15.62|40.20|102.60|19.15|220.25|59.24| +2451160|34824|1615|44751|1915858|3028|43398|4|1|736|54|779.76|23.39|803.15|59.48|1833.30|257.32|303.01|219.43|1916.17| +2451100|40982|11380|44751|6841|375|30309|10|35|736|2|34.70|0.00|34.70|38.68|39.78|17.00|0.88|16.82|78.46| +2451188||9014|75072|||12521|8||737|||0.00|105.35||||||| +2451199||2228||832214||20841||13|737||1454.31|101.80|1556.11|32.24||0.00||1367.06|988.40| +2452005|45970|7949|66864|955683|3721|28159|4|32|738|27|916.38|18.32|934.70|42.19|1448.55|843.06|64.52|8.80|1509.06| +2452621|50653|6805|59347|1662397|1894|5744|1|11|739|31|787.71|0.00|787.71|78.10|270.94|472.62|85.07|230.02|349.04| +2451930|38628|16130|10478|603673|5399|2325|7|25|741|2|87.46|4.37|91.83|7.23|121.12|28.86|48.63|9.97|132.72| +2451395|38059|7496|||2974|||13|742|6||0.00|31.92|43.39|34.32|||1.28|| +2452194|57575|4387|38271|660755|4839|14876|7|24|743|3|68.79|4.12|72.91|29.53|111.12|37.14|12.34|19.31|144.77| +2451217|55739|16730|59824|302045|4389|15580|1|27|745|5|325.60|3.25|328.85|16.28|222.50|71.63|182.85|71.12|242.03| +2451048|49835|16477|2118|299803|2288|44773|10|21|745|22|147.84|1.47|149.31|42.96|44.44|1.47|124.41|21.96|88.87| +2451720|39336|14150|26122|431470|5101|22779|8|8|746|54|469.26|9.38|478.64|43.54|1356.48|464.56|2.82|1.88|1409.40| +2452490|55819|11352|86082|1309788|3122|18856|8|17|747|13|59.41|2.37|61.78|96.83|14.30|1.78|51.86|5.77|113.50| +2451857|34408|14263|67020|254605|3983|32114|7|13|748|53|2655.30|132.76|2788.06|87.90|1073.25|716.93|1666.99|271.38|1293.91| +2451820|40248|743|75439|881307|7175|45443|8|4|748|14|510.30|35.72|546.02|36.21|459.20|158.19|288.73|63.38|531.13| +2451101|44176|17851|11156|1730485|720|2908|4|8|749|23|2232.61|200.93|2433.54|66.30|0.00|1183.28|975.87|73.46|267.23| +2451567|44973|3230|96414|1841346|91|20883|4|30|750|60|1262.40|88.36|1350.76|5.18|217.20|239.85|685.10|337.45|310.74| +2451263|45425|4420|44769|1088840|2732|45777|4|11|752|16|332.64|29.93|362.57|41.80|126.72|139.70|163.99|28.95|198.45| +2451600|51368|5728|7960|113232|5650|42068|7|32|753|74|70.30|1.40|71.70|57.24|990.12|24.60|26.50|19.20|1048.76| +2451923|37120|5600|24744|776110|6072|32834|4|11|755|23|703.57|56.28|759.85|67.18|9.43|372.89|330.68|0.00|132.89| +2450982|58203|12622|35993|352905|5794|27883|10|6|756|60|15.00|0.90|15.90|45.47|15.00|12.75|0.78|1.47|61.37| +||7570||1131112||14594|2|24|758||306.56||334.15||40.80|174.73|109.41|22.42|83.27| +2451018|46468|1250|70657|32956|3702|47648|10|30|758|1|17.71|0.88|18.59|47.70|3.99|15.58|1.06|1.07|52.57| +2451033|37721|2138|5663|636776|5042|34508|1|22|758|40|356.00|21.36|377.36|1.85|230.00|35.60|25.63|294.77|253.21| +2450980|41191|181|13608|1162914|4716|41271|4|25|758|7|31.64|0.00|31.64|31.33|21.28|20.88|8.93|1.83|52.61| +2451559|32623|12919|77294|1686652|2857|39815|4|6|759|13|384.93|11.54|396.47|55.20|147.03|146.27|150.35|88.31|213.77| +2451459|35923|3016|78937|1160961|5522|36083|2|33|759|35|4338.25|347.06|4685.31|51.65|1141.70|2776.48|359.20|1202.57|1540.41| +2451167|46674|6476|47287|207705|3541|19777|10|2|760|13|536.64|16.09|552.73|6.26|159.64|456.14|27.37|53.13|181.99| +2451281|52375|1810|47287|309465|1953|25194|1|3|760|19|2692.11|242.28|2934.39|14.93|751.26|2046.00|478.12|167.99|1008.47| +2451299|38101|5998|47287|1095307|3999|17091|2|33|760|4|197.16|9.85|207.01|51.16|15.16|82.80|98.34|16.02|76.17| +2452742|50843|7627|68279|971387|5248|31156|1|30|761|21|434.70|4.34|439.04|27.85|978.18|47.81|313.38|73.51|1010.37| +2451322|31946|12820|58002|509065|7164|5263|7|11|762|7|0.00|0.00|0.00|2.74|262.50|0.00|0.00|0.00|265.24| +2451166|46726|17648|7495|277961|74|11340|10|29|762|15|387.15|3.87|391.02|42.20|92.85|73.55|272.83|40.77|138.92| +2452714|37783|1356|49430|481656|5435|3214|2|34|763|4|112.56|0.00|112.56|4.97|26.28|41.64|29.07|41.85|31.25| +2452203|58788|10089|44625|181604|5713|36222|4|33|765|2|274.68|16.48|291.16|34.23|135.68|159.31|73.83|41.54|186.39| +2452049|52943|1373|62751|661657|996|39582|4|35|766|42|845.88|67.67|913.55|99.46|454.02|575.19|51.43|219.26|621.15| +2452076|38658|16831|62751|1059184|3565|24299|1|4|766|29|463.42|18.53|481.95|71.34|532.73|305.85|67.75|89.82|622.60| +2451908|36636|17191|87101|1160375|5696|23064|10|10|767|21|146.58|4.39|150.97|50.31|219.87|2.93|1.43|142.22|274.57| +2451580|58722|7495|1639|1709267|237|13207||24|768||420.42|37.83|458.25|95.26|300.30|159.75|||| +2451096|49821|8876|44911|497522|3725|375|1|18|769|26|1843.92|0.00|1843.92|77.56|598.00|885.08|632.83|326.01|675.56| +2451089|54903|7070|44911|719321|5858|15948|8|17|769|82|13.12|0.91|14.03|44.80|172.20|6.56|2.95|3.61|217.91| +2451090|55292|5854|47002|670259|5516|39973|2|25|769|14|291.34|2.91|294.25|48.13|165.76|2.91|210.55|77.88|216.80| +2451597|33415|15781|142|1811306|6657|32408|8|30|770|65|383.50|26.84|410.34|30.03|1535.95|329.81|1.61|52.08|1592.82| +2451570|37937|2173|142|416803|1027|41298|7|3|770|9|829.17|49.75|878.92|9.04|324.90|215.58|116.58|497.01|383.69| +2451972|42086|10075|65602|1148201|2000|11799|10|17|772|12|352.56|3.52|356.08|45.10|197.76|112.81|14.38|225.37|246.38| +2451896|46839|9469|35825|346110|748|41786|7|7|773|76|1709.24|85.46|1794.70|55.46|713.64|837.52|348.68|523.04|854.56| +2451775|48443|16273|54354|778047|6554|33908|8|27|774|46|1440.26|129.62|1569.88|25.52|251.62|792.14|162.03|486.09|406.76| +|35216|7262||1455416|6895||8|21|775|82|||||747.02|80.61|||789.43| +2451412|56666|8233|60335|1569094|5478|38309|8|34|775|2|217.18|17.37|234.55|12.39|29.60|204.14|6.12|6.92|59.36| +2451880|35015|12749|49037|1326456|1062|2068|10|25|776|1|71.25|2.13|73.38|0.91|9.00|46.31|4.23|20.71|12.04| +2452403|56570|3145|86439|1239698|5757|36616|4|9|777|1|38.68|0.00|38.68|23.33|20.13|20.88|7.29|10.51|43.46| +2451519|29857|10466|27961|1150754|3287|40173|7|22|778|7|425.95|34.07|460.02|89.64|72.45|302.42|96.35|27.18|196.16| +2451583|51258|8752|27961|767137|623|30109|1|9|778|6|256.44|17.95|274.39|8.11|103.38|79.49|44.23|132.72|129.44| +2452669|51509|9193|11659|254886|4678|2023|8|12|779|21|2320.92|185.67|2506.59|39.55|834.33|1183.66|113.72|1023.54|1059.55| +|60025|4429|||3203||||779||987.93|||93.55||||17.79|| +2452637|38894|10171|11659|1630245|1310|3528|4|21|779|60|109.80|9.88|119.68|46.91|69.00|66.97|7.28|35.55|125.79| +2451095|49551|4429|36195|1727902|1955|24855|7|16|780|20|237.80|11.89|249.69|66.78|29.20|118.90|59.45|59.45|107.87| +2451952|44135|16909|56703|920811|566|22274|8|29|781|27|54.00|4.86|58.86|79.38|264.87|11.88|40.43|1.69|349.11| +2451598|32362|17620|27721|1265454|6808|25409|1|14|782|31|1725.46|34.50|1759.96|73.33|778.10|1276.84|448.62|0.00|885.93| +2452803|41503|15135|95842|1656382|5722|43415|8|25|783|1|30.20|1.20|31.40|85.68|8.40|22.95|1.66|5.59|95.28| +2452234|37919|17637|51256|1634472|3682|5692|2|12|784|49|0.00|0.00|0.00|61.19|895.72|0.00|0.00|0.00|956.91| +2451939|38401|9932|1189|78525|5068|24011|10|12|785|59|5666.95|510.02|6176.97|21.42|159.30|850.04|3468.17|1348.74|690.74| +2452384|29855|1551|5683|304891|5699|5260|10|16|789|15|269.70|18.87|288.57|4.08|219.90|242.73|1.07|25.90|242.85| +2451481|30021|1477|15783|1646008|5265|28169|8|35|790|1|1.70|0.03|1.73|65.95|5.12|1.42|0.12|0.16|71.10| +2451631|39013|10567|15783|1416462|4001|293|2|16|790|28|673.68|60.63|734.31|87.63|79.52|485.04|77.34|111.30|227.78| +2451577|45964|3400|15783|1396766|1782|22445|4|7|790|3|10.62|0.42|11.04|21.33|2.10|4.56|5.33|0.73|23.85| +2451886|45745|17063|86796|48694|4414|34534|1|21|791|3|109.98|4.39|114.37|17.94|0.00|37.39|47.90|24.69|22.33| +2452712|35271|8676|61684|442907|4|48348|1|29|792|77|6759.83|337.99|7097.82|35.93|866.25|6421.83|310.96|27.04|1240.17| +2452565|59893|12198|61684|1602837|4363|45536|2|5|792|46|502.78|0.00|502.78|88.01|37.26|55.30|357.98|89.50|125.27| +2452566|41238|7543|65347|1324626|5574|48875|8|1|793|70|5436.90|0.00|5436.90|54.94|2002.70|3262.14|1522.33|652.43|2057.64| +2452033|44519|17852|41205|1040045|5285|30172|2|27|794|52|2485.08|49.70|2534.78|1.76|1159.60|397.61|1022.86|1064.61|1211.06| +2452648|45715|16953|35966|487726|3963|48200|10|22|796|56|598.64|53.87|652.51|64.24|1596.56|389.11|199.05|10.48|1714.67| +2451902|50612|14501|31604|1832385|440|32495|2|15|797|10|113.30|4.53|117.83|93.22|283.20|4.53|52.20|56.57|380.95| +2451927|34662|10514|31604|1873008|6074|37893|4|15|797|4|71.52|3.57|75.09|92.69|29.52|50.77|15.77|4.98|125.78| +2451860|36824|302|35192||||||798|||24.19|2443.39|31.63||1620.86||215.56|623.18| +2451300|31529|17528|74600|950467|3613|12312|10|9|799|61|3664.27|183.21|3847.48|96.25|2198.44|293.14|3000.30|370.83|2477.90| +2451653|57107|2986|8574|1847443|5235|29874|1|30|800|5|132.60|10.60|143.20|53.70|79.55|132.60|0.00|0.00|143.85| +2452259|30323|13283||||||24|801|10|362.90|18.14|||||66.04|187.99|| +2451917|56031|14995|68271|1665187|5368|14349|7|11|803|30|2493.00|149.58|2642.58|93.63|800.10|648.18|701.03|1143.79|1043.31| +2451772|46656|17411|68271|1690778|5424|15270|4|14|803|25|776.50|15.53|792.03|40.34|594.75|209.65|255.08|311.77|650.62| +2451150|30360|3454|27076|909003|2772|22941|2|29|804|38|232.18|13.93|246.11|90.75|169.10|120.73|69.09|42.36|273.78| +2451069|50943|10159|27076|684460|6439|15618|4|14|804|15|56.85|5.11|61.96|65.37|37.65|25.01|4.45|27.39|108.13| +2451184||7376|32248|1757180|6912|34597|1||804||829.80|74.68|904.48||409.60||236.99|45.15|545.06| +2452140|32868|13081|80665|538818|4198|32778|4|10|805|39|1067.04|0.00|1067.04|75.08|1518.66|266.76|120.04|680.24|1593.74| +2452407|54184|17251|24341|1501723|3101|6479|10|7|807|14|198.38|1.98|200.36|73.43|335.30|25.78|5.17|167.43|410.71| +2452353|42935|1739|24341|971063|5459|39808|7|23|807|12|510.72|25.53|536.25|83.48|294.60|403.46|90.09|17.17|403.61| +2452238|29383|2501|32491|958010|6376|48695|7|20|810|16|20.16|0.20|20.36|22.88|73.92|3.62|9.59|6.95|97.00| +2452308|57539|5899|17684|1144616|653|15490|4|1|810|36|2864.16|257.77|3121.93|82.35|566.28|1689.85|645.87|528.44|906.40| +2452221|55758|15029|17684|1406252|4634|41016|8|17|810|5|149.85|13.48|163.33|31.53|110.40|88.41|33.17|28.27|155.41| +2451249|38723|7120|32616|987586|4353|31520|8|4|811|41|391.14|15.64|406.78|96.71|218.53|62.58|108.42|220.14|330.88| +2451316|45470|5479|32616|255774|1046|33518|10|7|811|16|81.44|6.51|87.95|23.62|57.12|38.27|20.28|22.89|87.25| +2451247|39772|3568|32616|1823686|4465|46491|8|16|811|23|1079.85|43.19|1123.04|6.76|1420.94|691.10|132.17|256.58|1470.89| +2451353|58778|9002|32616|1810639|4373|21307|10|17|811|2|16.58|0.00|16.58|53.39|6.50|13.59|0.62|2.37|59.89| +2452153|49943|15321|26144|1619303|6062|2821|2|29|813|15|133.80|4.01|137.81|47.93|119.70|76.26|39.12|18.42|171.64| +2451611|43402|7001|25316|727178|687|17349|4|3|814|2|10.50|0.21|10.71|84.08|13.26|1.05|2.74|6.71|97.55| +2451677|48152|12530|40909|1751566|2246|20573|4|15|814|39|256.23|17.93|274.16|69.18|239.07|199.85|7.32|49.06|326.18| +2452215|44009|15425|234|349742|6628|27514|7|1|815|38|947.34|56.84|1004.18|15.15|508.44|653.66|261.37|32.31|580.43| +2452103|52737|6787|234|1591889|4974|24506|2|15|815|14|193.48|15.47|208.95|38.86|130.76|9.67|40.43|143.38|185.09| +2451549|49983|9367|85136|1611345|4754|22563|10|16|817|13|721.37|64.92|786.29|28.53|306.28|440.03|143.48|137.86|399.73| +2451535|42012|2086|10517|1600|3521|28242|1|14|817|46|3697.02|0.00|3697.02|16.75|1648.18|2144.27|15.52|1537.23|1664.93| +2452161|42061|43|2887|331711|3439|29967|10|26|819|76|758.48|68.26|826.74|49.71|2369.68|159.28|251.66|347.54|2487.65| +2452204||12849|89539|765595||33287|8|24|820||||1224.96|30.74|||98.73||603.74| +2452046|55194|17261|92962|468929|2885|10041|4|13|821|31|754.54|7.54|762.08|42.62|107.57|618.72|36.67|99.15|157.73| +2452023|39385|6427|92962|379348|5192|636|2|16|821|26|345.54|13.82|359.36|78.93|88.14|48.37|273.39|23.78|180.89| +2451536|32450|11308|61367|643402|4120|1991|2|35|822|14|21.00|1.05|22.05|96.75|185.08|10.71|5.45|4.84|282.88| +2451531|36203|13315|64940|1567989|6591|7240|2|6|822|24|1842.00|147.36|1989.36|31.79|1205.52|1363.08|23.94|454.98|1384.67| +2452073|57701|9541|62960|443135|4020|38552|8|2|823|1|7.01|0.42|7.43|36.41|8.76|5.18|1.61|0.22|45.59| +2451766|58985|4310|3967|559858|6728|23598|2|23|824|18|2693.52|53.87|2747.39|0.71|974.16|1319.82|123.63|1250.07|1028.74| +||12871|||6025|30983||7|826||2780.38|||89.74|||||| +2451486|28832|2420|5254|321519|5784|6449|8|2|828|3|41.76|2.92|44.68|70.22|17.22|32.15|1.34|8.27|90.36| +2452117|49892|5019|19149|762061|5503|11745|2|7|829|33|1147.74|68.86|1216.60|71.21|1010.13|814.89|103.18|229.67|1150.20| +2451584|46277|1351|42944|272833|4285|32137|10|3|830|4|367.44|14.69|382.13|74.54|345.36|257.20|91.49|18.75|434.59| +2452787|58370|11347|13636|1356298|5386|44427|1|11|831|3|56.37|2.25|58.62|7.36|80.55|0.56|11.72|44.09|90.16| +2452758|33721|4626|60182|1919847|1364|4528|2|14|832|15|2331.30|23.31|2354.61|55.20|552.75|1841.72|112.60|376.98|631.26| +2452696|42792|6954|60182|77691|4596|27854|7|22|832|30|1835.10|36.70|1871.80|87.85|1789.20|899.19|889.11|46.80|1913.75| +2451079|50514|7390|72201|1471811|4405|44999|10|30|833|10|362.10|25.34|387.44|1.52|275.90|173.80|88.50|99.80|302.76| +2451187|56277|15142|72201|720103|6349|22548|2|18|833|13|1143.48|45.73|1189.21|77.03|281.19|1006.26|135.84|1.38|403.95| +2451232|43362|13706|72201|793972|4280|17035|8|8|833|26|24.70|0.00|24.70|57.96|472.94|1.97|5.68|17.05|530.90| +2451207|49979|2569|62909||2302|11775||18|837||91.44|4.57||24.79||34.74|3.96||71.72| +2451198|50841|10852|72597|63538|4040|12142|4|9|837|20|928.80|74.30|1003.10|56.41|1227.40|715.17|113.22|100.41|1358.11| +2451566|45662|15328|70151|820743|970|29997|4|23|838|36|884.88|0.00|884.88|31.80|315.72|380.49|252.19|252.20|347.52| +2450842|43809|3271|98692|106562|5225|1535|8|28|840|4|22.32|1.33|23.65|77.85|183.32|8.92|0.40|13.00|262.50| +2450963|42986|3541|98692|537859|4371|30836|1|11|840|30|270.00|16.20|286.20|19.02|1178.70|35.10|61.07|173.83|1213.92| +2450953|53199|11500|98692|1712869|6822|29975|10|31|840|27|406.89|12.20|419.09|30.90|382.86|362.13|5.37|39.39|425.96| +2450995|56956|3758|81287|340334|5021|21661|2|10|841|24|755.76|37.78|793.54|75.61|794.40|627.28|24.41|104.07|907.79| +2452265|51858|16515|22015|1006835|2693|44551|8|17|842|15|200.70|18.06|218.76|28.53|501.75|40.14|134.87|25.69|548.34| +2451708|39529|15451|15382|907440|210|17027|8|27|844|47|256.15|2.56|258.71|35.14|38.07|17.93|90.52|147.70|75.77| +2451517|53776|6404|81939|542474|73|11272|10|2|845|5|860.55|43.02|903.57|60.45|138.80|404.45|287.34|168.76|242.27| +2451551|57539|13124|81939|1755233|6291|949|1|5|845|2|6.60|0.00|6.60|36.66|51.14|4.35|1.32|0.93|87.80| +2451634|52370|1316|81939|1814712|5021|31330|1|28|845|2|107.62|5.38|113.00|0.78|42.82|18.29|24.11|65.22|48.98| +2451247|44406|1519|71743|998436|6370|21457|2|8|846|6|21.72|1.52|23.24|28.43|56.46|10.64|10.30|0.78|86.41| +2451885|43385|12602|44318|25013|6061|38756|4|11|847|56|1326.64|39.79|1366.43|32.33|1206.24|1193.97|100.82|31.85|1278.36| +2451689|56814|10682|94882|1364237|6013|21389|7|29|849|51|2496.96|49.93|2546.89|16.36|377.91|149.81|727.61|1619.54|444.20| +2451847|39342|12011|62810|147679|5855|18384|8|7|851|15|856.05|51.36|907.41|21.06|586.50|17.12|167.78|671.15|658.92| +2451775|50833|6050|62810|281200|3175|5950|1|5|851|3|92.25|0.00|92.25|36.41|39.36|20.29|64.76|7.20|75.77| +2452519|31154|17049|78064|353152|2748|19544|2|11|852|31|2337.71|46.75|2384.46|69.06|615.04|1566.26|7.71|763.74|730.85| +2452507|42208|1884|28399|732172|4298|47271|1|5|852|23|1083.30|21.66|1104.96|50.26|403.42|519.98|214.06|349.26|475.34| +2451957|39013|11198|98495|510426|1248|40470|2|4|853|57|3696.45|332.68|4029.13|1.05|914.28|3511.62|36.96|147.87|1248.01| +2451210|43938|17584|49741|1534517|3718|22119|1|9|854|45|2442.15|195.37|2637.52|25.43|697.50|976.86|175.83|1289.46|918.30| +2451197|40741|2150|49741|673729|2250|28771|4|35|854|75|26.25|0.00|26.25|78.28|88.50|20.47|1.38|4.40|166.78| +2452299|58522|8593|78793|400989|2049|19509|4|4|856|26|728.00|0.00|728.00|81.17|492.44|706.16|10.04|11.80|573.61| +2452143|58680|11669|78793|870476|1647|41218|1|35|856|5|13.00|0.39|13.39|54.85|125.65|7.93|0.96|4.11|180.89| +2452282|59837|10163|35904|412221|3562|34503|1|33|857|55|282.15|19.75|301.90|45.75|42.35|53.60|155.41|73.14|107.85| +2452369|43913|2619|35904|1666422|7172|38275|4|28|857|1|34.16|1.70|35.86|69.67|7.15|29.03|2.51|2.62|78.52| +2452326|46630|14935|35904|1461501|6487|43520|7|29|857|11|446.38|17.85|464.23|64.60|263.01|196.40|167.48|82.50|345.46| +2451603|56029|13684|52011|1185557|432|29754|8|31|858|86|3329.92|266.39|3596.31|67.30|1195.40|932.37|1414.55|983.00|1529.09| +2451538|55685|14146|87877|||30998|8||860|55|305.80|0.00|305.80||393.80||||| +2451519|41108|7864|37282|613124|2119|49561|2|23|861|3|144.60|4.33|148.93|79.77|95.64|59.28|46.92|38.40|179.74| +2451603|54930|560|47688|1138638|7188|49338|7|28|861|11|1.65|0.00|1.65|12.59|9.46|0.99|0.60|0.06|22.05| +2450914|45725|15544|89383|1305260|4104|27701|1|7|862|4|0.00|0.00|0.00|78.25|112.24|0.00|0.00|0.00|190.49| +2451218|35187|1492|35868|1081920|490|43860|7|10|864|36|582.84|52.45|635.29|78.45|291.24|507.07|15.91|59.86|422.14| +2452244|29742|17971|62909|978256|1603|23350|1|24|865|1|37.45|2.62|40.07|73.44|4.49|2.62|22.63|12.20|80.55| +2451897|58350|13493|42075|1242063|5251|23686|10|9|866|2|185.08|14.80|199.88|42.07|93.80|98.09|53.93|33.06|150.67| +2451212|47624|326|83535|1457831|5025|39772|10|20|867|5|28.85|1.15|30.00|91.34|75.75|6.05|2.28|20.52|168.24| +2451176|53387|5492|83535|795331|4455|39132|2|33|867|11|812.02|32.48|844.50|9.81|236.39|146.16|452.78|213.08|278.68| +2451231|40806|3175|83535|1651448|5519|25143|7|13|867|25|854.75|34.19|888.94|29.75|57.50|752.18|25.64|76.93|121.44| +2451625|47603|17101|32054|1084595|5170|24898|4|31|868|17|427.89|29.95|457.84|73.95|380.29|162.59|254.68|10.62|484.19| +2451591|39619|3056|32054|1264441|1934|42627|7|13|868|20|1269.00|25.38|1294.38|28.32|719.00|1040.58|134.76|93.66|772.70| +2451560|60363|3476|32054|280912|4899|22886|8|20|868|48|463.20|37.05|500.25|98.19|547.20|189.91|248.69|24.60|682.44| +2451553||386|||4340||10|29|868|||||71.03||1927.33|1051.33||593.68| +2451229|54457|4534|54263|1363131|787|49196|2|21|869|28|3277.12|262.16|3539.28|74.86|553.28|1671.33|1204.34|401.45|890.30| +2451158|59729|15025|54263|1191429|3887|48792|2|35|869|88|3989.04|119.67|4108.71|43.35|646.80|638.24|2982.21|368.59|809.82| +2451093|31749|568|70800|1826917|516|7315|10|18|873|5|213.35|17.06|230.41|56.56|129.85|104.54|83.78|25.03|203.47| +2451183|45793|13880|55150|1283027|1427|9780|1|26|875|2|50.10|3.50|53.60|76.56|47.60|7.01|18.52|24.57|127.66| +2450988|58325|14179|10116|892420|2484|7395|4|15|878|4|468.44|23.42|491.86|23.16|17.32|337.27|18.36|112.81|63.90| +2451552|57876|7808|52904|171302|2397|30292|8|12|879|45|1725.75|155.31|1881.06|37.93|76.05|1173.51|110.44|441.80|269.29| +||8761|48204||6314|||5|881||||553.60|26.99|||||252.79| +2451532|44619|9178|14557|48496|3588|26327|8|18|882|74|5014.98|300.89|5315.87|66.53|4628.70|1805.39|449.34|2760.25|4996.12| +2451638|41857|562|42449|757614|301|35741|1|33|882|32|380.48|7.60|388.08|60.78|639.04|57.07|300.77|22.64|707.42| +2452225|29575|5037|65118|1232885|3127|4122|7|24|883|19|53.20|2.66|55.86|0.77|77.71|31.92|12.76|8.52|81.14| +2452290|56214|12629|65118|1470414|6258|12102|2|20|883|8|106.24|7.43|113.67|27.83|59.36|40.37|13.17|52.70|94.62| +2452246|57644|12711|56979|1821542|5752|4315|4|28|883|3|183.36|0.00|183.36|98.92|0.00|1.83|67.16|114.37|98.92| +2450937|38951|8659|85495|1029345|1141|35384|10|13|885|27|1573.29|0.00|1573.29|33.20|444.42|1305.83|240.71|26.75|477.62| +2451049|44014|8576|85495|1044468|3784|2747|1|24|885|47|4198.51|41.98|4240.49|53.37|159.33|167.94|3788.73|241.84|254.68| +2452769|42634|15834|4507|934895|930|15718|7|29|886|40|5294.00|370.58|5664.58|8.41|1176.40|794.10|2969.93|1529.97|1555.39| +2451928|56524|1021|21365|1625609|4606|41250|7|33|887|16|448.80|31.41|480.21|50.54|516.16|121.17|65.52|262.11|598.11| +2451558|57619|11558|77107|1005726|469|35212|1|18|889|6|8.52|0.59|9.11|18.52|8.28|1.53|6.01|0.98|27.39| +2452590|54687|16117|16848|213942|5080|43418|7|34|890|28|73.36|1.46|74.82|3.65|107.24|34.47|18.66|20.23|112.35| +2451744|57219|13537|71738|1092508|361|22167|7|13|891|51|2516.34|226.47|2742.81|89.95|337.62|2340.19|112.73|63.42|654.04| +2451883|56114|11833|71738|1177582|66|41750|2|28|891|15|352.80|10.58|363.38|44.96|823.20|349.27|1.87|1.66|878.74| +2452169|32241|11715|759|3339|2719|28099|8|17|892|2|297.78|14.88|312.66|11.31|117.30|193.55|44.81|59.42|143.49| +2452197|51768|15501|74030|1195777|2360|10496|2|33|892|28|526.12|47.35|573.47|90.02|6.72|205.18|12.83|308.11|144.09| +2451565|39769|7982|54681|873169|4582|23642|10|32|895|31|226.92|13.61|240.53|15.27|857.46|104.38|113.96|8.58|886.34| +2451615|50272|5336|54681|1268431|1817|25203|1|27|895|72|43.20|0.86|44.06|88.91|1568.88|0.43|11.12|31.65|1658.65| +2452114|55346|8083|44170|1895113|4072|30231|7|34|896|50|6586.00|395.16|6981.16|92.01|3608.50|1712.36|146.20|4727.44|4095.67| +2451917|58329|5875|85573|1076408|3071|35687|7|33|898|24|787.92|70.91|858.83|3.85|19.68|157.58|144.97|485.37|94.44| +2451880|32976|7033|63022|434946|4282|43265|1|19|900|1|55.51|0.55|56.06|70.53|38.22|36.63|11.32|7.56|109.30| +2451181|55593|11074|76092|817583|1797|12811|2|12|903|31|1261.70|75.70|1337.40|65.49|870.79|794.87|322.11|144.72|1011.98| +2452639|54170|4890|20179|1195264|786|46308|1|5|905|32|324.48|3.24|327.72|79.16|525.44|38.93|77.09|208.46|607.84| +2452322|49190|1443|44962|994785|3008|7678|1|16|906|6|644.64|32.23|676.87|1.20|85.92|290.08|244.64|109.92|119.35| +2451664|48420|9662|75244|265702|3360|46265|1|26|908|53|3324.69|0.00|3324.69|12.39|199.28|2626.50|656.29|41.90|211.67| +2452524|59557|11479|73918|298369|4550|17931|4|31|912|5|209.40|8.37|217.77|25.07|97.20|113.07|88.62|7.71|130.64| +2451510|56921|820|50211|162758|5492|37873|1|28|914|1|30.78|2.77|33.55|69.05|3.32|20.31|2.19|8.28|75.14| +2452690|45644|11670|32090|1831532|4381|8430|8|13|915|26|503.88|25.19|529.07|19.93|450.84|156.20|337.24|10.44|495.96| +2452749|51087|5959|81124|601200|6876|44952|10|19|916|17|862.75|0.00|862.75|46.84|325.55|776.47|0.86|85.42|372.39| +2451693|50217|16001|17353|372911|3892|6473|2|8|917|9|50.31|2.01|52.32|31.60|56.88|5.53|18.80|25.98|90.49| +2451296|60205|7609|78309|874319|3482|15045|1|33|918|48|2615.04|78.45|2693.49|11.23|3196.32|601.45|1530.32|483.27|3286.00| +2451582|39066|14546|41752|1629024|3496|9168|7|12|919|21|784.77|31.39|816.16|84.99|464.52|353.14|302.14|129.49|580.90| +2451597|32284|6902|98454|677899|6150|1|10|33|919|34|334.56|13.38|347.94|98.31|63.58|284.37|16.56|33.63|175.27| +2451925|38143|9157|64972|1459313|4704|8895|10|32|920|2|1.64|0.13|1.77|63.02|23.06|0.45|0.28|0.91|86.21| +2451213|31112|16918|20767|416720|3378|32436|7|17|921|11|609.07|0.00|609.07|15.99|188.21|395.89|162.01|51.17|204.20| +2451094|29775|5509|20767|1901222|553|40684|2|18|921|19|2713.96|0.00|2713.96|94.47|1474.21|841.32|318.34|1554.30|1568.68| +2452698|30740|6349|46602|91248|1540|6679|4|5|923|28|1799.84|125.98|1925.82|10.84|514.08|1565.86|58.49|175.49|650.90| +2452632|51848|6355|78837|1184003|2769|45935|10|20|923|36|1849.32|36.98|1886.30|99.81|506.88|536.30|774.68|538.34|643.67| +2452568|57242|11229|46602|178020|3404|3188|10|26|923|68|4486.64|134.59|4621.23|43.76|669.12|762.72|3239.81|484.11|847.47| +2452314|56561|10825|42678|1250306|5934|47209|4|20|924|74|4102.56|369.23|4471.79|32.84|50.32|2953.84|1137.23|11.49|452.39| +2452451|50093|9549|23739|436940|4671|41602|10|31|924|1|46.74|3.27|50.01|52.05|54.53|11.21|28.42|7.11|109.85| +2452313|29717|13098|23739|1685985|4799|19213|10|13|924|67|6683.92|66.83|6750.75|42.71|3106.79|1804.65|927.06|3952.21|3216.33| +2451885|36675|14054|65883|843562|5333|21904|4|1|925|58|4376.10|262.56|4638.66|50.56|1019.06|2144.28|2075.59|156.23|1332.18| +2452013|47515|4741|4480|1422665|1924|36783|2|6|926|4|173.68|1.73|175.41|16.50|102.28|85.10|33.66|54.92|120.51| +2451986||3224||1079965|1810||8|13|926|21||0.00|||836.85||1.95|1.73|| +2451516|37939|17690|78392|1050781|6877|46620|4|6|927|50|315.00|9.45|324.45|92.74|716.00|132.30|87.69|95.01|818.19| +2451619|51169|11167|78392|1668437|3206|32775|7|25|927|51|776.73|69.90|846.63|96.96|153.00|209.71|328.87|238.15|319.86| +2452281|36513|9469|7477|163349|2872|41995|2|31|928|11|580.69|17.42|598.11|23.54|277.31|522.62|6.38|51.69|318.27| +2452737|47339|15048|57384|1167269|2956|20690|2|28|929|3|103.32|7.23|110.55|11.67|2.22|15.49|8.78|79.05|21.12| +2452526|59301|9483|10488|752202|6268|31134|2|33|930|16|377.92|15.11|393.03|72.14|148.32|234.31|89.03|54.58|235.57| +2452649|53495|8473|37052|1674911|1800|37089|8|4|931|35|2606.45|234.58|2841.03|71.31|2109.80|703.74|1693.41|209.30|2415.69| +2451685|40247|3992|75716|1189530|3703|39429|2|25|932|9|337.14|6.74|343.88|88.24|168.57|269.71|29.66|37.77|263.55| +2451200|42380|16046|15243|207054|4912|43970|10|16|934|1|96.93|8.72|105.65|21.27|26.43|6.78|32.45|57.70|56.42| +2451200|54390|8395|71722|494303|1385|32067|10|9|934|32|620.48|0.00|620.48|2.96|827.20|521.20|62.54|36.74|830.16| +2451203|60885|9968|71722|27404|1073|44695|4|11|934|1|39.90|0.00|39.90|72.37|25.85|21.94|8.44|9.52|98.22| +2451938|39481|10567|69216|1324801|6339|8677|10|10|935|54|4655.88|372.47|5028.35|86.44|290.52|2234.82|363.15|2057.91|749.43| +2452026|56331|6740|39456|120739|2940|46294|4|19|935|3|29.94|1.19|31.13|19.97|15.96|20.05|3.65|6.24|37.12| +||3260||1793763||6938|4|28|937|35|95.55|0.95|96.50|76.29|||0.27||1033.09| +2451140|33988|8846|2803|1076176|3572|4215|4|4|939|64|3798.40|341.85|4140.25|51.47|1424.00|1519.36|1230.68|1048.36|1817.32| +2452344|44862|13809|91852|1222276|7100|9924|2|7|941|54|448.74|0.00|448.74|58.40|71.28|273.73|8.75|166.26|129.68| +2452390|28799|6019|51607|518280|4404|49281|2|2|941|26|530.14|37.10|567.24|29.74|329.94|116.63|103.37|310.14|396.78| +2452732|42818|14913|89693|316576|7011|32199|4|7|943|60|67.80|2.03|69.83|25.04|194.40|52.20|10.92|4.68|221.47| +2452386|47751|14736|60865|587480|4882|34331|8|32|944|19|1198.52|11.98|1210.50|13.61|767.03|191.76|291.96|714.80|792.62| +2452608|36418|12733|32085|1600847|4855|13663|1|24|945|13|304.07|3.04|307.11|92.98|111.41|170.27|111.05|22.75|207.43| +2451285|41522|5858|71408|1740985|4022|27012|1|9|946|45|3885.30|349.67|4234.97|27.66|347.85|2020.35|261.09|1603.86|725.18| +2452509|50693|7111|61307|697521|3203|7238|2|10|948|16|312.64|0.00|312.64|28.36|6.24|209.46|90.79|12.39|34.60| +2450983|35640|16930|99585|1139248|3425|43155|7|2|949|26|2462.20|73.86|2536.06|54.76|594.36|1748.16|449.84|264.20|722.98| +2451043|56673|8612|99585|1155583|5367|8351|4|4|949|30|615.30|18.45|633.75|52.17|756.00|498.39|18.70|98.21|826.62| +2451117|54689|11299|75004|1066606|4093|6827|2|26|951|4|148.20|11.85|160.05|79.73|114.00|102.25|6.43|39.52|205.58| +2451260|35644|1894|75004|1157894|4997|17347|2|35|951|34|934.66|28.03|962.69|1.11|1579.30|831.84|58.60|44.22|1608.44| +2451197|50071|242|65409|628198|4541|33284|4|33|954|44|2475.00|173.25|2648.25|71.63|1462.12|1212.75|782.59|479.66|1707.00| +2451422|54763|13270|33224|1326383|2623|11663|1|21|955|3|352.05|21.12|373.17|24.52|51.99|313.32|31.37|7.36|97.63| +2451427|40381|13825|33224|1895970|2398|25626|1|10|955|18|854.28|59.79|914.07|51.62|569.52|51.25|385.45|417.58|680.93| +2452704|60327|10605|56397|338273|867|23814|10|25|956|40|360.00|7.20|367.20|29.88|157.60|151.20|14.61|194.19|194.68| +2452754|54069|10975|38929|599846|5563|15680|1|17|957|29|564.92|50.84|615.76|5.89|214.02|508.42|37.85|18.65|270.75| +2452129|39179|4521|64796|1907407|3695|430|1|20|958|29|225.04|20.25|245.29|28.93|144.71|105.76|39.36|79.92|193.89| +2452046|34160|5751|64796|1409201|4896|31791|2|3|958|51|1099.56|65.97|1165.53|9.52|1268.88|483.80|474.13|141.63|1344.37| +2452138|61175|16519|64796|654376|7146|25963|10|17|958|23|1237.63|37.12|1274.75|28.24|1087.67|754.95|246.16|236.52|1153.03| +2452189|35503|7951|170|1089853|6654|1102|8|26|961|41|574.82|40.23|615.05|32.90|153.34|316.15|178.48|80.19|226.47| +2451144|36619|13621|13822|273441|1860|5879|8|9|962|21|132.30|0.00|132.30|45.33|330.96|85.99|14.81|31.50|376.29| +2451110|35684|11674|71432|646783|6134|3932|4|34|962|2|29.88|2.68|32.56|70.53|47.52|3.88|0.78|25.22|120.73| +2452737|33363|5713|72195|1621530|5909|49441|10|34|963|19|433.39|17.33|450.72|21.48|304.00|351.04|41.99|40.36|342.81| +2451932|34928|16562|79453|1380350|2254|20158|7|21|964|8|414.16|16.56|430.72|45.45|0.00|211.22|73.05|129.89|62.01| +2451896|36613|2245|79453|1185035|1193|44104|10|5|964|86|3814.96|114.44|3929.40|66.92|5665.68|1106.33|1652.26|1056.37|5847.04| +2451852|35517|17171|79453|396568|6413|37011|1|2|964|57|3913.62|195.68|4109.30|57.97|2325.60|3326.57|452.02|135.03|2579.25| +2452259|47394|7465|14819|1670161|963|42165|1|24|965|58|951.78|85.66|1037.44|2.89|169.94|161.80|608.28|181.70|258.49| +2452198|60596|4397|14819|544938|5445|40752|10|23|965|32|114.88|2.29|117.17|97.18|1103.04|95.35|7.03|12.50|1202.51| +2451461|45092|1054|53335|827283|2061|8039|7|8|966|1|21.39|1.92|23.31|75.53|7.74|2.13|16.75|2.51|85.19| +2451956|37526|8264|15|1135930|5092|22607|7|29|968|4|286.68|22.93|309.61|48.03|143.32|232.21|44.12|10.35|214.28| +2452034|55566|2413|33981|1154746|305|1064|7|5|968|48|2054.88|0.00|2054.88|52.97|445.44|595.91|641.94|817.03|498.41| +2451170|54560|10303|93748|596486|4994|32749|4|12|969|1|18.63|0.00|18.63|13.30|10.13|0.37|15.70|2.56|23.43| +2452259|29780|11419|26374|488378|1918|6705|8|10|970|1|21.36|0.42|21.78|94.52|18.24|21.36|0.00|0.00|113.18| +2452410|51414|4301|26374|731190|2834|4125|1|13|970|3|8.91|0.35|9.26|98.56|17.82|0.17|6.46|2.28|116.73| +2452744|47113|2667|20558|1447663|3568|49073|7|13|971|8|82.32|6.58|88.90|51.68|283.76|10.70|70.90|0.72|342.02| +2452765|38218|15999|41911|1640618|6972|31279|8|5|971|3|8.01|0.00|8.01|89.24|10.32|2.64|3.22|2.15|99.56| +|41710|15139|27558|1403078|6393|38908|||971|52|294.32|||12.47|1324.44||||1345.73| +2450995||13363|52145|||12693|10||974||7.02||7.65|||5.61|1.04||| +2450898|58470|8306|52145|1543832|6993|12507|7|27|974|3|172.23|5.16|177.39|71.67|14.01|51.66|15.67|104.90|90.84| +2450956|29951|13270|64620|168006|192|45178|1|32|974|4|101.36|6.08|107.44|77.14|182.44|10.13|3.64|87.59|265.66| +2452272|54799|14511|45164|169443|922|18848|4|11|975|25|1113.75|89.10|1202.85|73.14|752.50|189.33|924.42|0.00|914.74| +2452424|47594|11515|21518|999891|3323|21155|8|34|975|13|1388.27|27.76|1416.03|51.26|374.79|513.65|446.05|428.57|453.81| +2451946|46078|10292|49594|69747|6022|19544|||976||268.17|||64.84||72.40||9.79|416.30| +2452617|42553|15793|30663|1917765|2820|1875|7|20|978|46|2648.22|79.44|2727.66|64.83|2427.42|2039.12|194.91|414.19|2571.69| +2452601|46407|7663|67948|1414737|1162|15587|7|12|978|20|922.00|73.76|995.76|4.51|1037.20|424.12|119.49|378.39|1115.47| +2451909|55026|6023|22219|311954|4365|2376|1|15|981|38|1584.98|15.84|1600.82|88.68|847.78|1283.83|105.40|195.75|952.30| +2452331|46096|13691|10248|895404|6888|33572|8|12|983|21|912.03|27.36|939.39|1.81|190.89|364.81|60.19|487.03|220.06| +2451805|48644|8750|30751|454143|735|37374|7|15|985|4|66.08|5.28|71.36|62.42|19.12|25.11|14.74|26.23|86.82| +2452409|55866|17526|23171|1339924|2630|32583|7|17|988|67|1396.95|69.84|1466.79|49.72|318.25|628.62|699.18|69.15|437.81| +2452447|32569|16969|1842|787500|2591|40482|4|24|988|45|971.10|9.71|980.81|69.88|805.95|291.33|176.74|503.03|885.54| +2452410|40671|4681|1842|901991|5387|38830|2|12|988|8|309.60|15.48|325.08|71.64|258.00|303.40|0.62|5.58|345.12| +2451658|45188|14497|10841|1723122|6506|36290|10|23|990|3|139.56|9.76|149.32|20.64|106.32|122.81|11.89|4.86|136.72| +2452632|57070|6781|24333|418602|4340|2525|4|33|992|11|279.07|22.32|301.39|49.73|0.00|173.02|26.51|79.54|72.05| +2452654|49382|10416|35451|1912198|5745|9692|4|1|992|7|66.43|5.31|71.74|45.45|141.96|19.26|32.07|15.10|192.72| +2452445|49437|8355|42746|743700|4326|4105|1|17|994|6|6.90|0.41|7.31|54.49|15.12|2.20|2.20|2.50|70.02| +2451995|54468|15089|9431|408284|2552|30420|7|31|995|9|467.64|14.02|481.66|91.36|220.05|112.23|92.40|263.01|325.43| +2451923|35464|5557|23194|319911|4758|20758|10|3|996|14|929.32|55.75|985.07|98.56|494.62|473.95|145.71|309.66|648.93| +2451812|48658|6824|23194|1536469|448|37227|2|15|996|32|226.88|4.53|231.41|28.15|259.20|72.60|145.02|9.26|291.88| +2452667|34835|17658|24018|368203|4655|13107|4|31|997|2|31.84|0.31|32.15|9.11|42.92|1.27|13.75|16.82|52.34| +2452500|48699|5461|85869|1076623|6224|42178|7|1|998|82|1061.90|21.23|1083.13|76.15|653.54|1040.66|20.81|0.43|750.92| +2451271|61031|13312|70130|874697|6303|13509|8|26|999|49|2594.55|207.56|2802.11|96.82|1824.27|830.25|370.50|1393.80|2128.65| +|42018|4945|18428|281057|5999||4||1000|81|3703.32||3851.45|||3592.22||3.34|1769.14| +2451590|40897|12838|18428|318856|5776|23449|8|19|1000|12|634.56|57.11|691.67|18.22|493.56|152.29|371.34|110.93|568.89| +2452310|41422|11465|97390|584585|574|31410|10|35|1001|17|1177.59|70.65|1248.24|88.03|758.88|11.77|1165.82|0.00|917.56| +2452657|35381|13185|55293|1670158|223|44288|4|3|1002|71|4408.39|88.16|4496.55|48.80|2712.91|1146.18|163.11|3099.10|2849.87| +2451482|28800|3151|69768|974054|554|21007|4|7|1003|4|60.00|0.00|60.00|34.40|210.04|43.80|0.32|15.88|244.44| +2451630|53884|12757|69768|24913|1002|49444|1|25|1003|1|38.52|0.38|38.90|88.96|4.20|21.18|5.54|11.80|93.54| +2451516||415|63948|||9637|1|27|1004||||215.00|46.28|55.20|197.97|||103.60| +2451494|31141|10706|63948|823950|1322|15894|2|8|1004|5|318.20|22.27|340.47|48.72|59.65|85.91|150.98|81.31|130.64| +2452076|39975|15499|36959|524166|3544|18122|7|31|1005|13|428.61|4.28|432.89|41.34|70.46|150.01|153.23|125.37|116.08| +2451530|35662|11800|54165|1537143|608|8873|2|18|1006|4|80.52|6.44|86.96|4.40|80.52|16.10|19.97|44.45|91.36| +2451560|54086|3560|54165|1468345|7147|44349|1|20|1006|37|353.72|7.07|360.79|59.76|874.31|3.53|105.05|245.14|941.14| +2452426|28880|8629|92890|1515950|6657|42164|10|21|1007|2|11.66|0.00|11.66|88.65|18.22|3.03|1.29|7.34|106.87| +2452407|42897|6114|35430|1633416|6446|7076|10|31|1007|7|733.81|66.04|799.85|86.97|206.36|139.42|534.95|59.44|359.37| +||16151||1701706|3613|12720|1|29|1008|||3.35||46.85||41.36||7.27|70.96| +2452341|43199|6403|5483|955367|3546|45278|1|34|1008|8|215.44|8.61|224.05|4.50|297.76|47.39|5.04|163.01|310.87| +2452203|41445|10149|5483|1402409|1809|1562|2|5|1008|43|136.74|10.93|147.67|97.03|925.36|117.59|13.40|5.75|1033.32| +2452240|37818|3869|61572|1115161|1982|43156|1|21|1009|1|9.21|0.00|9.21|53.86|6.40|5.15|0.36|3.70|60.26| +2451453|34476|6044|10393|493401|3483|32812|7|16|1010|6|25.38|2.03|27.41|9.25|15.54|18.78|3.89|2.71|26.82| +||16118|84957|1433493|4438||1|35|1012|17||||0.94|||||| +2451187|34534|16990|84957|565229|4610|10438|4|12|1012|36|1516.68|30.33|1547.01|44.98|857.16|1107.17|69.61|339.90|932.47| +2451224|55111|5762|70329|1700460|1065|18917|4|31|1013|6|108.54|1.08|109.62|4.36|4.02|85.74|0.22|22.58|9.46| +2452664|31078|2155|65842|1026346|4658|1770|8|16|1014|5|137.20|2.74|139.94|6.39|171.55|93.29|43.47|0.44|180.68| +2452039|49789|13535|58910|109065|5836|4207|2|14|1015|8|270.00|10.80|280.80|31.31|252.56|24.30|83.53|162.17|294.67| +2452722|44963|7074|27013|1055782|3756|27283|||1018|47|874.67||918.40|0.51|1749.34||231.79||1793.58| +2451977|41198|8948|83607|1823015|1073|28731|1|17|1019|30|224.10|11.20|235.30|9.25|96.90|194.96|2.91|26.23|117.35| +2452009|53632|2801|50313|1427883|2757|21984|2|5|1020|39|52.26|3.13|55.39|85.76|11.70|9.40|11.14|31.72|100.59| +2451108|56281|12524|42994|1066240|4301|25962|1|34|1021|33|114.18|5.70|119.88|90.81|844.80|97.05|9.93|7.20|941.31| +2451109|52480|296|42994|55889|5223|32957|1|20|1021|8|234.48|18.75|253.23|22.62|154.40|227.44|6.75|0.29|195.77| +2452653|29527|1159|66743|759817|5691|28765|8|8|1022|2|3.12|0.00|3.12|63.50|2.08|1.90|0.68|0.54|65.58| +2452759|34553|7914|59821|1377510|5794|41325|8|5|1022|3|4.05|0.00|4.05|33.32|1.38|2.47|0.56|1.02|34.70| +2451888|41816|16243|39393|1637707|5742|14930|10|31|1023|4|47.32|1.41|48.73|24.64|47.32|32.65|3.22|11.45|73.37| +2451681|42688|17581|55341|1021628|1741|46254|10|17|1024|4|203.80|6.11|209.91|87.79|52.08|24.45|173.96|5.39|145.98| +2451682|50856|16759|53723|1296942|961|42580|4|18|1024|34|1267.86|76.07|1343.93|42.39|709.92|988.93|119.93|159.00|828.38| +2451310|31104|10381|81542|1425492|2320|45901|8|7|1025|31|2769.85|166.19|2936.04|80.00|496.93|1357.22|1356.12|56.51|743.12| +2451920|44332|8945|15765|634371|3824|6430|2|19|1026|1|106.39|1.06|107.45|98.70|8.27|11.70|22.72|71.97|108.03| +2452022|43272|17528|81071|1549750|1566|35029|4|31|1026|24|12.72|1.14|13.86|40.99|102.96|10.30|0.04|2.38|145.09| +2452600||16603|||5979|472|||1028|40||0.00|1964.00||379.20|1649.76|||387.68| +2452589|44835|3805|52162|1677348|4070|3545|8|7|1028|12|268.80|18.81|287.61|41.09|142.08|155.90|102.73|10.17|201.98| +2451160|39411|5960|80369|1655322|853|44384|10|18|1029|6|269.64|16.17|285.81|48.65|58.02|231.89|21.14|16.61|122.84| +2452230|35513|17093|61553|1512117|1405|15493|4|24|1030|4|200.32|0.00|200.32|93.79|150.24|62.09|23.49|114.74|244.03| +2451390|39411|8656|||436|40787||2|1034|||15.57|534.81|89.91|47.16|||126.29|| +2452363|50387|15161|83187|1404805|4823|41241|8|17|1035|74|358.90|17.94|376.84|78.66|1305.36|229.69|86.57|42.64|1401.96| +2452281|46554|15633|62046|1458901|5258|25615|10|24|1036|61|864.37|34.57|898.94|83.71|940.62|34.57|215.74|614.06|1058.90| +2451139|57531|17899|87360|1671027|3802|11377|2|30|1037|8|71.28|1.42|72.70|99.10|299.60|30.65|5.68|34.95|400.12| +2451087|56973|14014|87360|1229050|5005|47360|7|12|1037|20|1711.60|102.69|1814.29|98.44|136.00|1335.04|52.71|323.85|337.13| +2451151|42808|6298|81304|682825|1802|37979|2|11|1038|67|732.98|58.63|791.61|40.51|535.33|381.14|309.61|42.23|634.47| +2451114|51043|6940|81304|1013067|1775|44530|2|8|1038|24|793.20|71.38|864.58|22.54|483.60|182.43|482.50|128.27|577.52| +2451233|54269|4460|66573|741650|1527|37804|10|11|1038|26|934.96|9.34|944.30|51.52|459.94|336.58|5.98|592.40|520.80| +2452598|51996|12877|82756|1638034|933|22279|4|6|1039|7|160.79|14.47|175.26|90.72|108.36|114.16|23.78|22.85|213.55| +2452442|38251|7999|82756|1850001|1365|38776|4|13|1039|10|20.90|1.46|22.36|69.93|29.30|0.62|0.60|19.68|100.69| +|52240|12433|6474||||||1039|13|172.64||184.72||14.82||18.66||91.53| +2452154|44853|11205|71845|1175410|6485|34360|10|18|1040|15|398.70|7.97|406.67|3.42|103.50|295.03|76.71|26.96|114.89| +2452152|50698|7475|3904|1840660|7016|14172|2|14|1040|21|6.93|0.55|7.48|90.09|48.93|0.69|1.80|4.44|139.57| +2451821|49516|11894|88012|864254|5672|29890|7|26|1042|7|37.45|1.87|39.32|70.80|14.98|28.08|0.09|9.28|87.65| +2451922|54280|6647|13410|1260958|3706|46454|10|20|1042|7|27.72|1.94|29.66|90.81|120.33|13.30|11.53|2.89|213.08| +2452340|44206|2573|83173|732652|7090|11528|7|19|1044|25|2457.25|221.15|2678.40|61.19|1468.00|1818.36|389.72|249.17|1750.34| +2450999|40168|8131|75557|254600|1444|42688|7|16|1045|14|930.86|0.00|930.86|70.34|153.02|660.91|256.45|13.50|223.36| +2452618|39015|13297|42511|797398|4635|45825|7|4|1047|14|528.78|15.86|544.64|83.20|124.32|428.31|65.30|35.17|223.38| +2452668|41269|7813|42511|1689407|1711|8015|4|18|1047|13|1138.02|102.42|1240.44|13.95|307.58|830.75|172.07|135.20|423.95| +2452696|42499|8550|42511|1218725|3666|38160|1|25|1047|84|2551.08|229.59|2780.67|26.31|267.96|586.74|1787.54|176.80|523.86| +2452665|39110|3126|42511|1879210|2532|18701|4|17|1047|47|5963.36|178.90|6142.26|10.99|3948.94|2087.17|3139.71|736.48|4138.83| +2452295|60004|4061|2000|1246513|3214|3115|1|19|1049|16|761.76|22.85|784.61|97.76|108.80|358.02|242.24|161.50|229.41| +2451717|48441|11947|62971|1149015|3486|11510|8|7|1050|38|2723.46|81.70|2805.16|97.06|568.48|1171.08|1505.80|46.58|747.24| +2451441|44363|5150|68527|871158|4386|28414|1|8|1051|33|471.24|4.71|475.95|96.20|1531.86|169.64|117.62|183.98|1632.77| +2451150|51243|8626|19494|1105250|4959|19515|2|20|1053|67|55.61|3.89|59.50|54.00|35.51|13.90|14.59|27.12|93.40| +2451221|35072|2233|7708|949774|4150|25726|7|3|1053|53|5175.45|414.03|5589.48|47.21|755.78|3622.81|916.05|636.59|1217.02| +2451467|50006|1996|3953|1405201|6787|23973|1|10|1055|13|387.92|3.87|391.79|54.40|250.90|11.63|161.80|214.49|309.17| +2451324|59161|14881|20564|895367|6970|46126|8|35|1056|7|473.69|33.15|506.84|79.01|366.73|4.73|304.82|164.14|478.89| +2451352|49316|12703|63251|1386500|4949|28645|7|28|1056|1|32.47|0.00|32.47|40.13|35.42|18.50|13.41|0.56|75.55| +2451828|44358|11035|76539|1591019|3447|13075|2|13|1057|88|7692.08|307.68|7999.76|46.20|2950.64|76.92|228.45|7386.71|3304.52| +2451798|34807|1400|69175|1162852|529|25359|2|23|1057|53|902.06|63.14|965.20|30.01|433.54|432.98|234.54|234.54|526.69| +2451600|36269|9488|6968|153023|4452|23434|1|32|1060|35|2580.55|180.63|2761.18|28.18|186.90|1496.71|921.26|162.58|395.71| +2451557|30632|13447|6968|412951|4754|2110|1|16|1060|5|211.85|2.11|213.96|38.61|216.80|161.00|31.52|19.33|257.52| +2452655|37286|9451|61626|1188387|2352|48717|8|30|1062|4|56.52|0.56|57.08|82.76|10.64|27.12|13.52|15.88|93.96| +2451610|55052|14834|52648|410740|2079|47818|7|34|1064|37|2314.72|208.32|2523.04|61.12|732.23|1226.80|1033.52|54.40|1001.67| +2451023|54894|2128|32035|739674|2680|1639|10|14|1065|3|193.23|9.66|202.89|73.84|69.00|119.80|11.01|62.42|152.50| +2451970|54205|3473|98664|865964|2638|34959|7|1|1066|42|1936.20|154.89|2091.09|37.77|1144.08|793.84|148.50|993.86|1336.74| +2451945|41859|1729|98664|1453678|3676|41674|2|5|1066|5|355.15|17.75|372.90|41.56|122.30|134.95|198.18|22.02|181.61| +2452031|59284|12287|7125|1438197|3316|31235|8|26|1066|18|7.20|0.57|7.77|76.19|13.86|2.01|4.72|0.47|90.62| +2451884|60625|9656|75758|1823498|5285|48272|4|26|1066|27|543.78|10.87|554.65|48.54|171.72|293.64|165.09|85.05|231.13| +2451900|55324|8126|10256|1527034|3240|41446|10|23|1067|5|12.20|0.00|12.20|44.52|6.40|1.70|3.57|6.93|50.92| +2451922|57952|9527|10256|1322291|816|43341|1|3|1067|7|397.46|19.87|417.33|85.00|106.75|55.64|242.69|99.13|211.62| +2452020|44285|1508|10256|1227136|1743|9150|2|31|1067|41|599.83|29.99|629.82|42.54|116.03|431.87|1.67|166.29|188.56| +2452058|56317|14717|21921|1743696|2677|43611|10|26|1068|43|1779.77|88.98|1868.75|79.99|105.78|444.94|600.67|734.16|274.75| +2451942|53969|896|30512|662624|933|25242|10|27|1069|2|2.86|0.00|2.86|79.53|0.82|1.45|0.45|0.96|80.35| +2451884|36499|2336|30512|97667|7168|9516|10|20|1069|8|55.04|3.30|58.34|66.15|6.96|17.06|14.81|23.17|76.41| +2452104|56201|2745|18406|1091995|4075|20202|10|27|1070|2|156.88|4.70|161.58|76.14|85.20|72.16|68.62|16.10|166.04| +2450856|48614|145|70098|1431411|2535|30859|8|32|1072|23|817.65|40.88|858.53|61.87|817.65|662.29|6.21|149.15|920.40| +2452243|51123|9411|56896|371238|3347|41856|10|1|1073|10|1436.90|114.95|1551.85|59.88|217.70|732.81|387.24|316.85|392.53| +2452181|46763|10231|56896|1130829|973|23870|2|1|1073|5|336.60|20.19|356.79|17.72|29.90|259.18|28.64|48.78|67.81| +2452179|29733|16371|56896|255320|3269|5571|8|15|1073|36|1095.48|21.90|1117.38|44.69|309.24|547.74|273.87|273.87|375.83| +2451295|32630|12664|70555|42300|2447|13841|8|27|1074|27|2318.76|162.31|2481.07|77.71|736.02|742.00|741.07|835.69|976.04| +2451256|55107|10771|70555|1244630|6037|1397|1|7|1074|13|322.92|22.60|345.52|29.03|408.46|155.00|6.71|161.21|460.09| +2451142|39078|11564|70555|1774316|6781|16046|8|21|1074|14|469.70|37.57|507.27|67.78|412.44|441.51|25.93|2.26|517.79| +2451134|47666|12838|4200|1134823|2660|17250|4|22|1075|1|18.97|1.51|20.48|57.25|2.68|13.27|3.42|2.28|61.44| +2451136|29030|14017|4200|1538851|4078|41705|10|16|1075|16|128.00|2.56|130.56|45.75|750.24|112.64|5.37|9.99|798.55| +2451277|60571|7438|88863|944961|6659|16404|7|5|1075|84|3338.16|66.76|3404.92|95.55|1398.60|534.10|2804.06|0.00|1560.91| +||7351||538540||17407|7|24|1076|||0.00||||1201.34|28.26|537.09|2833.97| +2451356|47849|15622|49059|1207921|6946|3571|7|31|1076|29|1203.50|96.28|1299.78|63.04|822.15|84.24|761.09|358.17|981.47| +2451374|50348|16586|49059|1630746|1898|21050|7|19|1076|10|587.40|46.99|634.39|32.67|267.00|123.35|37.12|426.93|346.66| +|33031|7647|33497|1039501|1858|23415|4|31|1077|23|2328.06|69.84|2397.90|44.68||||849.28|1224.73| +2452598|29035|9577|80132|472109|3840|26724|4|1|1078|8|15.36|0.76|16.12|73.01|256.88|5.22|6.48|3.66|330.65| +2451315|41274|2431|76739|84918|9|1853|10|15|1079|14|94.08|6.58|100.66|37.42|235.34|10.34|67.82|15.92|279.34| +2450969|31115|6847|4627|723597|6597|32844|8|13|1080|67|5364.02|482.76|5846.78|24.99|2899.09|5149.45|184.53|30.04|3406.84| +2450839|39106|11470|4627|786473|751|29782|4|29|1080|37|730.75|29.23|759.98|17.55|569.80|577.29|21.48|131.98|616.58| +2451544|52884|16544|874|699703|6872|29594|7|33|1081|16|139.36|9.75|149.11|36.91|109.12|5.57|10.70|123.09|155.78| +2451615|55844|2818|36896|1584558|3507|32856|10|21|1082|79|3484.69|34.84|3519.53|46.44|2354.99|2961.98|162.04|360.67|2436.27| +2451567|57882|6235|21877|1294026|3955|46684|8|12|1084|28|497.84|34.84|532.68|39.62|394.80|159.30|331.76|6.78|469.26| +2451550|34191|5728|21877|757759|5784|35456|8|23|1084|34|2944.40|147.22|3091.62|63.15|35.70|824.43|63.59|2056.38|246.07| +2451581|49072|7987|21877|1447271|4487|36503|8|30|1084|73|346.75|13.87|360.62|14.62|1325.68|13.87|103.19|229.69|1354.17| +2451572|32172|12418|76809|292941|2218|9495|8|21|1087|27|1216.08|12.16|1228.24|23.55|525.69|741.80|199.19|275.09|561.40| +2452021|45014|7829|28452|1363974|1814|36101|7|25|1088|19|710.60|35.53|746.13|38.04|270.56|390.83|41.57|278.20|344.13| +2451872|38675|8621|37983|204866|4563|9413|8|35|1089|16|479.36|38.34|517.70|19.24|66.08|52.72|51.19|375.45|123.66| +2451837|58321|13244|91397|1778606|1462|26718|7|10|1089|35|858.90|60.12|919.02|21.37|1232.35|197.54|595.22|66.14|1313.84| +2451215|30301|9196|69371|1490602|5105|26455|7|11|1090|37|1142.93|91.43|1234.36|89.17|222.00|788.62|85.03|269.28|402.60| +2451117|29682|15460|69371|764155|2033|31158|7|1|1090|51|3186.99|31.86|3218.85|36.92|1645.77|2039.67|321.24|826.08|1714.55| +2451156|45852|4057|69371|1035617|1827|20084|7|21|1090|3|201.18|14.08|215.26|58.37|134.13|116.68|21.12|63.38|206.58| +2451203|44475|14504|24154|1328786|3018|19250|10|26|1091|47|403.73|0.00|403.73|57.98|0.00|88.82|170.05|144.86|57.98| +2451222|42669|3637|46262|1642065|5327|11836|10|27|1091|40|2406.80|144.40|2551.20|41.00|1554.40|1973.57|112.63|320.60|1739.80| +2452303|60798|15901|84808|1806571|5614|48610|4|1|1092|1|3.36|0.26|3.62|13.65|7.62|3.36|0.00|0.00|21.53| +2451160|39230|10288|54240|1474663|1656|1049|2|30|1093|20|1223.80|110.14|1333.94|10.56|543.80|636.37|293.71|293.72|664.50| +2451484|51044|13921|78482|1154111|6201|22244|7|32|1094|10|579.10|0.00|579.10|97.40|256.40|34.74|473.59|70.77|353.80| +2452754|57654|11382|6837|1720739|2038|34537|7|5|1095|65|1254.50|62.72|1317.22|0.57|1153.10|664.88|17.68|571.94|1216.39| +2452750|60189|13927|63308|1537414|3520|16073|8|5|1095|34|18.36|1.46|19.82|50.37|87.04|11.38|5.23|1.75|138.87| +2451628|46790|14113|45228|1220758|1896|31834|8|22|1098|15|454.95|31.84|486.79|45.96|541.05|245.67|20.92|188.36|618.85| +2451607|47270|7274|45228|1514065|3764|13416|4|29|1098|12|450.48|9.00|459.48|40.36|157.56|355.87|84.20|10.41|206.92| +2451243|53797|15184|92698|1037199|6829|46059|8|34|1100|8|377.28|30.18|407.46|60.68|158.56|15.09|191.96|170.23|249.42| +2452652|52822|14706|43475|829484|2222|47807|2|12|1103|30|2312.40|208.11|2520.51|29.60|687.30|1456.81|213.89|641.70|925.01| +2452791|49258|14325|43475|1211307|6311|19668|4|27|1103|70|266.00|2.66|268.66|27.58|26.60|103.74|38.94|123.32|56.84| +2452360|42156|12891|45181|522269|2482|46295|4|29|1104|2|14.42|1.29|15.71|44.97|11.20|6.48|5.16|2.78|57.46| +2452316|46739|8465|45181|102364|1607|42897|2|10|1104|5|107.35|6.44|113.79|38.12|20.95|49.38|35.36|22.61|65.51| +2452407|50661|6043|45181|1231732|4482|17632|1|10|1104|23|2806.92|84.20|2891.12|73.81|567.18|421.03|1359.95|1025.94|725.19| +2452291|60316|1779|72142|1301021|1009|44150|10|21|1107|44|94.16|3.76|97.92|50.14|154.88|28.24|46.80|19.12|208.78| +2451466|36101|1484|50213|1203277|3541|39805|1|8|1108|13|0.00|0.00|0.00|43.19|528.45|0.00|0.00|0.00|571.64| +2451962|41098|8768|48445|1449401|5812|21470|1|12|1109|13|52.91|3.17|56.08|14.98|170.17|52.91|0.00|0.00|188.32| +2451850|41719|2444|48445|718013|2612|20364|10|18|1109|19|181.83|10.90|192.73|93.46|35.34|138.19|42.76|0.88|139.70| +2451457||16442|63710|||29573|||1111|33|3230.04|64.60|3294.64|||2196.42||382.44|| +2451403|55802|14468|63710|914131|4361|24991|4|29|1111|8|280.88|0.00|280.88|99.07|0.00|210.66|11.23|58.99|99.07| +2451768|60444|8534|82353|1110122|7010|29145|2|18|1112|2|132.02|11.88|143.90|1.73|34.06|68.65|27.88|35.49|47.67| +2452565|55762|10993|87963|1616474|991|35508|7|25|1113|33|1950.63|58.51|2009.14|16.48|928.62|1599.51|52.66|298.46|1003.61| +2452281|51243|3487|71050|672428|5949|23470|1|32|1114|20|417.40|4.17|421.57|17.44|553.00|100.17|130.06|187.17|574.61| +2451232|52280|10498|46095|1251056|2950|35644|7|2|1115|17|386.41|19.32|405.73|56.80|168.64|227.98|47.52|110.91|244.76| +2451814|31022|9593|17284|702141|4437|32333|7|28|1116|24|552.48|5.52|558.00|44.59|304.80|215.46|269.61|67.41|354.91| +2451965|42605|1895|49744|52652|1160|35668|7|9|1117|61|750.91|60.07|810.98|23.37|122.00|180.21|325.29|245.41|205.44| +2451688|35947|8107|55169|657045|4074|9103|4|23|1120|21|20.58|0.20|20.78|73.00|21.63|17.49|2.81|0.28|94.83| +2451584|41933|16637|18495|1691585|1922|12259|1|5|1120|9|141.57|4.24|145.81|81.35|61.92|1.41|138.75|1.41|147.51| +2452018|42892|17145|85230|1445760|4624|8688|2|21|1121|60|4263.60|170.54|4434.14|79.88|843.00|213.18|445.54|3604.88|1093.42| +2452054|44838|16037|86188|354653|6671|10336|1|34|1121|67|44.22|3.97|48.19|58.57|106.53|32.72|8.39|3.11|169.07| +2451999|29781|12203|89876|433034|1551|4868|4|12|1122|11|1108.80|77.61|1186.41|94.79|492.80|487.87|540.20|80.73|665.20| +2452059|35443|5750|89876|1339463|4505|35547|8|26|1122|15|183.60|3.67|187.27|80.28|85.50|156.06|3.85|23.69|169.45| +2452043|48093|11642|89876|14686|3894|41549|1|25|1122|37|357.05|7.14|364.19|85.58|223.11|78.55|197.73|80.77|315.83| +2451922|56299|15017|20878|76114|4275|29636|1|8|1123|2|110.36|3.31|113.67|42.41|71.72|33.10|71.85|5.41|117.44| +2452623|49286|3657|74894|1123147|732|29917|10|7|1124|4|11.68|0.81|12.49|39.81|11.68|2.45|5.99|3.24|52.30| +2452662|50781|6351|90206|1746961|1112|47216|1|18|1128|1|3.19|0.15|3.34|63.44|0.31|0.28|0.37|2.54|63.90| +2452693|46430|10656|90206|1331512|2373|39681|4|22|1128|19|729.41|43.76|773.17|33.03|377.15|474.11|181.26|74.04|453.94| +2452154|45540|13273|27357|919146|6488|8277|10|8|1130|76|54.72|3.28|58.00|87.83|130.72|47.05|5.36|2.31|221.83| +2452662|32751|16644|49347|1298791|4726|4661|2|1|1131|75|8097.75|485.86|8583.61|3.40|3400.50|890.75|1657.61|5549.39|3889.76| +2452506|36919|361|41450|1164173|3801|24982|1|1|1133|10|55.50|3.33|58.83|10.74|3.70|37.74|4.26|13.50|17.77| +2451400|57431|799|22094|1266558|183|19008|8|26|1134|13|35.75|1.78|37.53|17.23|117.65|10.72|7.25|17.78|136.66| +2451557|34258|17455|18549|1905414|6410|36959|8|6|1135|77|1670.90|100.25|1771.15|7.46|315.70|401.01|368.26|901.63|423.41| +2451627|29245|394|98767|274590|2060|20730|1|3|1135|5|250.35|0.00|250.35|67.27|62.55|160.22|38.75|51.38|129.82| +2451712|47677|2689|98135|309618|4938|16705|4|20|1136|4|12.92|0.64|13.56|6.64|55.52|4.78|4.39|3.75|62.80| +2451727|58702|3155|96827|1319739|4698|7210|7|21|1136|49|3481.94|208.91|3690.85|24.28|870.24|174.09|529.25|2778.60|1103.43| +2451702|43147|13363|98135|797739|3076|46013|10|10|1136|36|3403.44|238.24|3641.68|15.58|2191.32|2518.54|123.88|761.02|2445.14| +2452491||5622|||4838|35729||6|1137|21||0.00|247.38|27.46|19.74|||54.63|47.20| +2452458||1767|28317||||4|2|1137||433.62||437.95||294.80|195.12|174.10||338.16| +2452523|49684|8046|11499|251157|2815|826|2|7|1137|5|85.10|5.10|90.20|90.80|243.25|33.18|25.96|25.96|339.15| +2451328|54619|4598|20033|1683197|126|46702|8|2|1138|42|303.24|9.09|312.33|25.29|1734.18|15.16|77.78|210.30|1768.56| +2452010|40187|17621|55214|42739|3123|15060|10|31|1139|12|15.12|0.60|15.72|32.85|651.60|8.46|2.46|4.20|685.05| +2451414|56988|11899|6746|1248108|4816|22677|1|8|1141|3|52.68|0.52|53.20|19.35|69.30|31.08|4.32|17.28|89.17| +2451756|51066|3823|4989|712243|7070|11944|8|33|1142|28|54.88|0.00|54.88|61.44|12.60|42.80|10.63|1.45|74.04| +2452337|29155|3853|50189|959122|3500|6725|1|3|1143|18|38.16|3.05|41.21|12.46|119.34|7.25|18.54|12.37|134.85| +2452324|56358|14802|50189|487089|3321|7153|4|27|1143|24|1848.24|0.00|1848.24|38.12|1848.24|55.44|1021.89|770.91|1886.36| +2451298|47051|4033||1637977||30142|8|12|1144|||9.56|||84.32||220.27||175.44| +2452237|35087|9051|77735|1625579|1455|16978|10|32|1145|4|72.36|0.72|73.08|88.60|29.92|19.53|41.73|11.10|119.24| +2452159|47678|9675|48656|1425864|6307|48350|1|11|1146|2|50.12|1.00|51.12|78.12|61.44|42.10|3.28|4.74|140.56| +2451169|37224|1387|51183|1524757|189|48868|2|14|1148|7|27.09|0.00|27.09|74.56|135.52|3.25|10.48|13.36|210.08| +2451729|53766|824|13011|697321|1388|6208|10|10|1149|24|205.92|6.17|212.09|65.67|178.32|6.17|1.99|197.76|250.16| +2451712||14737||||||25|1149|80|3916.00|0.00||||1292.28|865.82|1757.90|| +2451712|39908|12269|13011||||||1149|43||45.14||||1038.25|||888.34| +2451689|31100|3661|44461|1166240|4478|15116|7|12|1150|4|135.56|4.06|139.62|69.57|78.76|116.58|1.70|17.28|152.39| +2452707|46503|14814|31165|935846|1083|15873|7|27|1153|33|1293.60|25.87|1319.47|38.87|1841.07|1228.92|43.33|21.35|1905.81| +2452055|44690|3129|967|1078260|391|40122|8|34|1154|10|383.50|7.67|391.17|30.22|13.20|325.97|6.32|51.21|51.09| +2452018|43822|15723|43785|1552800|5368|17983|1|6|1154|16|721.12|7.21|728.33|26.65|410.88|144.22|438.44|138.46|444.74| +2451227|34397|15496|96256|1593129|4350|10724|1|19|1155|2|61.40|4.29|65.69|41.31|28.42|60.78|0.11|0.51|74.02| +2451222|30274|13232|96256|1876219|5471|31170|8|26|1155|27|500.04|5.00|505.04|61.10|125.01|450.03|46.50|3.51|191.11| +2451212|45793|17869|96256|907954|4928|14702|8|8|1155|12|255.96|12.79|268.75|92.23|8.04|130.53|107.86|17.57|113.06| +2450915|40619|3448|9889|618941|6689|28884|10|32|1156|14|594.86|41.64|636.50|42.10|99.12|333.12|138.72|123.02|182.86| +2451564|40322|3766|57009|713868|5226|34584|7|35|1158|1|26.28|0.26|26.54|71.78|6.13|16.29|7.89|2.10|78.17| +2452601|45961|16566|22380|1236806|2785|32411|10|21|1159|27|144.45|2.88|147.33|21.02|103.95|108.33|4.69|31.43|127.85| +2451971|48033|3477|83329|1265638|3147|4256|4|4|1160|1|31.24|1.56|32.80|88.73|22.84|16.86|0.43|13.95|113.13| +2452361|40571|12361|8735|633423|5962|14148|2|10|1161|71|649.65|32.48|682.13|5.28|408.96|610.67|12.08|26.90|446.72| +||2739||1176163|4070|44824||15|1161|49|419.93|12.59||73.91|86.73|125.97||255.75|173.23| +2451693|51524|16979|86004|996896|4053|48457|7|18|1162|22|2259.62|112.98|2372.60|20.72|278.96|1175.00|130.15|954.47|412.66| +2451763|38498|12077|4514|1549942|1323|33551|8|25|1162|10|643.40|32.17|675.57|22.50|454.20|141.54|110.40|391.46|508.87| +2451224|29541|11521|33986|1405454|1369|37684|8|2|1163|12|185.76|1.85|187.61|38.70|50.64|165.32|16.14|4.30|91.19| +2452427|45822|12125|67814|449796|5629|9462|8|28|1164|25|68.00|2.72|70.72|71.80|136.25|42.84|15.34|9.82|210.77| +2452423|44265|1639|67814|238425|2532|24192|2|27|1164|7|516.81|41.34|558.15|1.17|68.53|387.60|116.28|12.93|111.04| +2451258|35750|11815|88484|1588528|1019|7063|10|13|1165|47|1182.52|23.65|1206.17|6.82|1021.31|792.28|159.99|230.25|1051.78| +2451220|41649|10256|36875|797105|1029|37800|8|15|1165|34|1231.48|61.57|1293.05|23.01|537.20|492.59|524.61|214.28|621.78| +2452117|34236|16667|22614|173905|2796|45334|2|30|1166|3|50.22|3.01|53.23|44.17|35.07|4.01|5.08|41.13|82.25| +2452184|30111|1555|68935|571643|3951|21059|4|35|1166|58|6489.04|194.67|6683.71|47.73|1046.32|194.67|314.71|5979.66|1288.72| +2452156|46851|4729|18523|892093|2110|30927|4|29|1167|7|23.73|1.42|25.15|13.05|71.19|12.33|10.60|0.80|85.66| +2452140|48075|451|30313|630086|3788|5490|10|14|1167|35|338.10|13.52|351.62|84.86|1183.35|260.33|3.11|74.66|1281.73| +2452234|45807|6011|26459|1920467|3140|30566|2|33|1168|12|683.28|13.66|696.94|77.87|227.76|13.66|508.91|160.71|319.29| +2452321|38334|16097|26459|535002|5868|5534|8|18|1168|3|83.07|2.49|85.56|30.67|111.81|42.36|32.97|7.74|144.97| +2451640|41619|8030|66775|1692283|1891|44299|1|20|1169|6|339.18|16.95|356.13|49.08|173.04|149.23|87.37|102.58|239.07| +2451547|40748|9692|66775|1432355|4754|17953|10|21|1169|25|1011.75|10.11|1021.86|91.73|337.25|758.81|134.05|118.89|439.09| +2451860|44158|14101|52345|937109|6728|32700|1|13|1170|55|4170.10|291.90|4462.00|71.24|1971.20|3711.38|68.80|389.92|2334.34| +2452361|43641|16015|15494|1177287|830|42020|7|35|1172|19|410.02|28.70|438.72|89.91|558.03|135.30|271.97|2.75|676.64| +2452716|50187|17280|89430|888406|4982|11958|7|5|1173|25|23.25|0.23|23.48|24.71|0.50|6.04|2.23|14.98|25.44| +2452573|46055|17737|40314|1491099|3015|36578|2|15|1175|24|730.32|21.90|752.22|27.46|67.68|80.33|64.99|585.00|117.04| +2452704|60016|6513|25651|1200538|5017|24296|8|4|1176|47|828.61|24.85|853.46|64.79|1413.76|820.32|2.81|5.48|1503.40| +2452755|31662|4765|25651|739002|5111|36020|2|31|1176|64|2357.76|47.15|2404.91|90.78|824.96|872.37|787.25|698.14|962.89| +2452771|39065|4903|5008|194524|2263|28449|7|24|1180|63|2644.74|0.00|2644.74|47.61|710.01|211.57|170.32|2262.85|757.62| +2452615|38785|15115|5008|687380|2023|7113|4|13|1180|37|1242.83|111.85|1354.68|66.95|1681.28|111.85|22.61|1108.37|1860.08| +2452695|55424|2952|5008|1525663|3719|1775|8|22|1180|38|1038.54|0.00|1038.54|50.57|1090.60|238.86|551.77|247.91|1141.17| +2452688|50015|7635|5008|1453761|5864|39967|10|33|1180|24|1261.68|37.85|1299.53|3.35|1207.92|946.26|138.78|176.64|1249.12| +2452151|48381|1985|53227|337858|1000|3295|2|7|1181|3|19.26|1.54|20.80|16.20|55.38|1.92|11.79|5.55|73.12| +2452250|37105|2197|53227|1548476|4914|3705|4|11|1181|1|43.21|3.45|46.66|62.50|1.30|9.07|12.63|21.51|67.25| +2452399|48964|4393|6345|803459|2809|36206|10|23|1182|13|409.63|12.28|421.91|65.22|188.11|114.69|221.20|73.74|265.61| +2451514|55934|9343|36340|1656952|2011|623|1|19|1185|22|303.82|27.34|331.16|50.88|283.58|243.05|53.47|7.30|361.80| +2451226||6944|99283||2188||||1187||549.01||||||||938.65| +2451602|33583|12206|34357|1184676|3348|18820|1|16|1189|26|3680.56|294.44|3975.00|8.86|40.30|2797.22|291.50|591.84|343.60| +2452748|42634|9669|85124|1633282|6770|45058|2|9|1190|45|132.75|11.94|144.69|20.27|179.10|86.28|39.96|6.51|211.31| +2452605|46364|9733|85124|878815|1126|25484|2|25|1190|1|12.57|0.62|13.19|98.11|2.51|5.15|6.60|0.82|101.24| +2452638|42356|1843|85124|1627780|1540|28053|10|26|1190|32|1961.60|58.84|2020.44|34.96|22.40|98.08|987.66|875.86|116.20| +2452494|29322|13867|60060|6937|1848|12359|1|4|1191|22|1613.04|96.78|1709.82|50.77|79.64|1177.51|226.47|209.06|227.19| +|51408|11562|90721|||33851||16|1191||||680.01||1212.09|320.76||160.38|| +2451652|57500|7723|14050|666167|4288|20856|2|25|1192|44|1375.44|55.01|1430.45|71.35|1100.00|618.94|7.56|748.94|1226.36| +2451599|57009|2108|14050|1626230|1768|38106|2|32|1192|41|728.16|36.40|764.56|1.13|441.16|247.57|379.66|100.93|478.69| +2451582|42742|616|56532|14379|6709|24686|1|28|1193|78|1280.76|64.03|1344.79|99.42|701.22|486.68|301.75|492.33|864.67| +2451541|42709|15776|56532|1570706|2420|4134|7|27|1193|57|632.70|25.30|658.00|9.84|3075.15|132.86|419.86|79.98|3110.29| +2451679|32899|12904|39359|794141|4877|413|8|32|1195|27|2089.26|167.14|2256.40|36.13|1358.10|1943.01|19.01|127.24|1561.37| +|29189|2216||413003|3245||7|15|1195|16|991.20|89.20|1080.40||146.72||||| +2452481|49016|9372|10197|649207|6933|6706|8|15|1196|35|2001.30|80.05|2081.35|60.58|1000.65|1601.04|248.16|152.10|1141.28| +2452497|29875|14185|10197|691930|4568|4406|2|32|1196|7|162.05|12.96|175.01|43.86|224.35|145.84|12.31|3.90|281.17| +2451191|51513|16348|93879|1705971|2063|8155|8|19|1197|70|330.40|23.12|353.52|37.62|420.00|138.76|76.65|114.99|480.74| +2451223|41893|7363|51156|1640412|6021|44877|8|12|1197|28|1519.00|60.76|1579.76|3.69|233.52|1275.96|192.00|51.04|297.97| +2451215|57158|5308|86780|1681271|4819|35120|7|16|1197|9|76.50|4.59|81.09|56.22|0.00|55.08|20.56|0.86|60.81| +2451534|58788|10490|61578|1906281|5253|16625|1|14|1198|29|1261.21|50.44|1311.65|54.13|1144.05|227.01|672.23|361.97|1248.62| +2451566|46222|17414|61578|12843|1870|30050|10|17|1198|4|257.24|0.00|257.24|72.59|79.68|56.59|26.08|174.57|152.27| +2451533|33632|6229|51367|737496|1672|15401|4|26|1198|34|4078.30|285.48|4363.78|62.68|786.08|3874.38|93.80|110.12|1134.24| +2451417|48587|15193|80022|1837370|228|686|7|17|1199|31|1520.24|0.00|1520.24|92.56|571.95|1429.02|45.61|45.61|664.51| +2451438|57920|1618|80022|1852133|5151|32522|2|26|1199|15|1491.45|104.40|1595.85|21.63|203.25|1029.10|208.05|254.30|329.28| +2451556|49323|14555|83944|819355|905|22055|4|23|1200|2|89.68|8.07|97.75|78.82|46.38|26.00|12.09|51.59|133.27| +2451675|57121|2222|83944|605411|6547|30083|4|25|1200|2|175.92|10.55|186.47|97.22|32.98|45.73|74.20|55.99|140.75| +2451381|41965|8600|33554|971337|4107|34477|7|34|1201|42|3122.28|124.89|3247.17|82.38|286.86|2404.15|359.06|359.07|494.13| +||646|33554|1758087|||4||1201||||4465.82|94.52|||||| +2450976|37014|82|1768|847992|1420|35601|8|30|1202|32|2856.64|28.56|2885.20|89.69|96.00|2399.57|365.65|91.42|214.25| +2451008|52747|16159|1768|1231019|3216|45013|2|4|1202|10|1168.20|23.36|1191.56|48.17|241.70|1156.51|9.93|1.76|313.23| +2452294|43831|16317|35408|333699|1650|37601|1|15|1204|23|1461.65|87.69|1549.34|74.43|121.67|745.44|336.61|379.60|283.79| +2452145|41296|12343|35408|1884420|1165|42310|4|5|1204|39|209.04|18.81|227.85|22.96|1925.82|71.07|42.77|95.20|1967.59| +2452245|60725|4121|35408|573884|3312|11493|2|30|1204|56|4135.60|165.42|4301.02|49.75|1804.32|661.69|2501.21|972.70|2019.49| +2451951|33055|503|67049|407794|6228|9462|2|8|1205|3|103.26|4.13|107.39|80.73|23.31|6.19|57.27|39.80|108.17| +2451286|44699|13846|78877|800009|5471|21733|7|25|1206|43|39.99|1.19|41.18|98.18|240.37|4.39|6.76|28.84|339.74| +2452352|37620|6829|25766|771284|5229|12327|2|7|1207|2|223.44|11.17|234.61|42.07|82.46|138.53|32.26|52.65|135.70| +2452370|48365|1087|44089|277582|2628|15321|2|12|1207|36|2813.40|196.93|3010.33|90.87|1281.60|900.28|440.01|1473.11|1569.40| +2452366|50751|14281|43234|716824|2767|49157|1|12|1207|19|2608.70|52.17|2660.87|67.67|1117.96|521.74|125.21|1961.75|1237.80| +2452381|38831|3873|25766|490998|2851|30351|7|22|1207|48|3273.60|196.41|3470.01|94.71|1184.16|1865.95|267.45|1140.20|1475.28| +2451713|43311|14509|47065|1767646|1676|43259|8|27|1208|78|3832.92|306.63|4139.55|77.52|377.52|306.63|2327.35|1198.94|761.67| +2452158|34249|15435|26586|278277|1270|17416|4|15|1209|23|583.74|23.34|607.08|47.53|251.62|315.21|56.39|212.14|322.49| +2452145|54966|5985|3524|783142|3113|13484|4|21|1209|60|3241.20|97.23|3338.43|59.32|350.40|1879.89|1225.17|136.14|506.95| +2451241|30140|14096|43722|574564|2660|17603|4|7|1211|29|0.00|0.00|0.00|72.72|214.31|0.00|0.00|0.00|287.03| +|39599|12874|62503||6067|28195|||1212||193.41||206.94|23.35|8.75|61.89|88.11||| +2452202|35547|14291|64653|621580|5452|26971|8|27|1213|12|818.04|49.08|867.12|51.34|139.20|171.78|277.89|368.37|239.62| +2452327|29240|15323|64653|1156087|1986|24732|1|32|1213|19|1233.67|0.00|1233.67|19.38|608.19|1110.30|78.95|44.42|627.57| +2452335|29495|13723|13988|663799|4457|19468|2|27|1213|33|342.54|0.00|342.54|39.04|4.62|256.90|65.94|19.70|43.66| +2451108|53687|1738|11345|127980|5142|18257|4|26|1214|29|246.21|0.00|246.21|83.83|470.38|194.50|7.75|43.96|554.21| +2450978|34405|7063|82113|93471|6107|343|1|10|1216|45|41.40|2.89|44.29|88.60|623.70|26.49|0.29|14.62|715.19| +2450975|44843|1624|6454|1719893|5685|26677|8|8|1216|31|219.79|0.00|219.79|22.43|0.00|210.99|1.14|7.66|22.43| +2451067|60660|5378|82113|303009|2568|24174|8|7|1216|15|1041.00|31.23|1072.23|29.51|424.80|83.28|565.05|392.67|485.54| +2452166|45876|14445|47711|1190462|103|9297|7|4|1219|14|688.10|61.92|750.02|48.72|308.70|103.21|245.65|339.24|419.34| +2452144|50320|12261|91790|870572|7170|31768|10|29|1221|33|545.49|5.45|550.94|2.97|160.71|212.74|119.79|212.96|169.13| +2451840|57309|6083|40912|105802|4739|17229|1|34|1223|45|622.35|56.01|678.36|45.11|2365.20|329.84|263.25|29.26|2466.32| +2452351|30108|5413|92787|136897|4525|9306|7|35|1225|3|15.42|1.23|16.65|12.74|4.02|1.54|5.41|8.47|17.99| +2451333|49483|4699|9388|61221|3244|16867|2|3|1227|71|5547.23|332.83|5880.06|61.71|1248.18|4992.50|66.56|488.17|1642.72| +2451183|57005|508|9388|1220308|5813|18071|7|21|1227|8|181.36|7.25|188.61|0.52|206.40|47.15|22.81|111.40|214.17| +2451177|61151|5122|28207|1533551|1652|18553|10|30|1228|1|4.56|0.22|4.78|31.32|1.08|3.05|0.12|1.39|32.62| +2451916||3007||931871||33867|4||1231|||||57.33|647.28|||1309.36|| +2451939|34073|4921|74759|401686|5766|36646|2|1|1231|69|285.66|14.28|299.94|54.53|233.22|254.23|3.14|28.29|302.03| +2452393|37418|15951|94362|594059|3356|31053|7|27|1233|17|990.08|19.80|1009.88|84.50|275.06|910.87|10.29|68.92|379.36| +2452396|44377|11567|41798|1378425|6527|21429|1|18|1233|2|56.10|3.36|59.46|15.99|9.04|0.56|49.98|5.56|28.39| +2452036|40090|17633|34990|990530|4096|40462|4|27|1235|29|1288.47|0.00|1288.47|11.26|348.29|347.88|159.90|780.69|359.55| +2452681|57335|5451|3407|1375923|4886|22202|10|6|1236|41|62.73|0.62|63.35|73.61|534.23|50.18|2.13|10.42|608.46| +2452209|30002|11347|89465|1690379|5222|48512|1|3|1237|7|313.67|15.68|329.35|52.08|41.51|185.06|29.58|99.03|109.27| +2452301|33482|15189|89465|1315851|6159|11267|1|28|1237|36|1050.84|0.00|1050.84|8.34|1507.68|609.48|180.95|260.41|1516.02| +2451757|35335|2600|50179|9002|5123|7644|10|28|1238|2|63.08|4.41|67.49|19.16|36.56|17.66|29.97|15.45|60.13| +2452611|54965|12996|79407|309909|3597|32485|10|9|1240|13|1229.54|98.36|1327.90|20.87|303.94|479.52|135.00|615.02|423.17| +2452709|45941|13485|72450|761525|7047|37769|7|16|1240|61|6338.51|190.15|6528.66|21.42|1193.16|633.85|1825.49|3879.17|1404.73| +2452686|59916|11880|25105|1850202|5162|40922|2|17|1241|10|888.10|0.00|888.10|97.33|130.20|390.76|402.84|94.50|227.53| +2452652|53848|13647|25105|468633|3929|12825|4|27|1241|1|3.90|0.03|3.93|77.50|2.64|3.47|0.30|0.13|80.17| +2452634|58754|5391|44328|219378|3228|23878|4|1|1241|4|325.84|6.51|332.35|27.80|89.16|32.58|111.43|181.83|123.47| +2452020|38016|10955|53099|1872311|3832|35706|4|11|1243|7|296.10|11.84|307.94|83.44|312.06|91.79|110.32|93.99|407.34| +2451935|60081|11737|53099|1326641|2792|12190|7|14|1243|69|6165.84|246.63|6412.47|41.36|2300.46|4686.03|88.78|1391.03|2588.45| +2452277|37093|1455|43026|313097|600|28903|4|6|1244|27|194.94|7.79|202.73|9.46|571.05|101.36|9.35|84.23|588.30| +2452153|59922|8309|15342|1437251|6360|41469|10|2|1244|39|980.46|68.63|1049.09|46.56|226.98|68.63|392.08|519.75|342.17| +2452217|48453|9917|38424|668269|961|6006|4|16|1244|44|1113.64|77.95|1191.59|89.35|1336.72|790.68|38.75|284.21|1504.02| +2452777|35195|4317|43787|1502469|1329|33036|10|8|1248|1|9.42|0.47|9.89|1.12|14.14|6.40|0.96|2.06|15.73| +2452646|32080|5427|43787|811572|3767|25515|8|21|1248|48|1380.00|82.80|1462.80|42.56|294.24|96.60|1026.72|256.68|419.60| +2452636|44984|9135|63661|1568475|4936|10150|10|8|1248|72|223.92|2.23|226.15|59.03|133.92|129.87|44.20|49.85|195.18| +2452317|52086|15955|9095|838588|1733|12878|7|1|1250|13|266.76|2.66|269.42|64.46|153.66|176.06|63.49|27.21|220.78| +2452205|33978|9331|83468|465045|1022|42952|4|6|1250|17|335.92|13.43|349.35|5.69|149.26|225.06|87.57|23.29|168.38| +2452357|39282|10819|9095|705076|1484|3092|7|8|1250|7|199.64|15.97|215.61|27.19|222.74|23.95|140.55|35.14|265.90| +2451122|39518|5854|22184|708481|5476|10071|1|34|1252|6|225.30|13.51|238.81|9.13|7.26|166.72|4.10|54.48|29.90| +2451189|45559|2438|22184|98491|5854|38770|10|12|1252|26|304.46|9.13|313.59|57.56|44.98|167.45|45.21|91.80|111.67| +2451642|57671|17378|79641|1300193|815|1390|7|17|1253|36|1594.80|127.58|1722.38|40.93|666.00|111.63|1334.85|148.32|834.51| +2451702|39503|14024|79641|1190291|4530|21216|4|31|1253|20|863.20|17.26|880.46|10.24|257.80|371.17|260.77|231.26|285.30| +2452249|32682|1473|80092|596951|764|21377|10|28|1254|6|221.70|15.51|237.21|73.90|87.24|121.93|56.86|42.91|176.65| +2452213|44274|13509|80092|251162|6699|30236|10|27|1254|42|577.08|40.39|617.47|58.88|391.44|5.77|279.94|291.37|490.71| +2451440|40273|17914|97227|1384566|390|6614|4|14|1258|39|992.16|89.29|1081.45|95.20|425.10|515.92|371.46|104.78|609.59| +2451035|51134|12158|38153|202238|5822|37006|7|9|1260|18|265.32|7.95|273.27|43.24|138.96|198.99|38.47|27.86|190.15| +2451035|37345|8344|38153|164326|6967|44808|8|35|1260|13|1104.48|66.26|1170.74|87.31|261.82|154.62|550.91|398.95|415.39| +2452004|57473|9901|46619|923704|342|627|7|27|1261|7|85.40|0.85|86.25|7.48|87.43|46.11|38.50|0.79|95.76| +2452093|29971|3335|690|830743|4744|17898|1|5|1261|20|317.00|3.17|320.17|28.03|224.60|0.00|72.91|244.09|255.80| +2452242|33116|15539|66078|190683||4925|2|29|1263||596.70||638.46|94.34|8.90||113.85||145.00| +2452293|36124|8933|89843|1562217|544|12471|10|2|1264|34|2957.32|207.01|3164.33|40.61|483.82|2040.55|687.57|229.20|731.44| +2452239|51361|2107|88684|1716401|2061|41321|2|23|1264|9|97.56|7.80|105.36|38.39|0.00|68.29|0.29|28.98|46.19| +|61080|17016|11100|||41189||29|1265||878.92||905.28|57.55|||657.43||| +2451529|32453|17162|20750|215336|3103|43623|10|2|1266|11|658.13|32.90|691.03|35.82|394.90|460.69|92.79|104.65|463.62| +2451510|44299|6217|20750|1518157|4507|16857|2|15|1266|83|2953.14|0.00|2953.14|38.62|0.00|2273.91|455.08|224.15|38.62| +2451597|54939|13874|93826|483855|1884|39113|10|26|1266|1|22.60|2.03|24.63|24.34|8.51|4.06|9.08|9.46|34.88| +2451919|34361|9271|91273|1434679|3008|29384|4|32|1267|14|454.44|36.35|490.79|59.47|136.22|168.14|163.19|123.11|232.04| +2451897|31816|4133|91273|1540478|581|8684|7|2|1267|24|1017.84|50.89|1068.73|24.16|828.48|162.85|76.94|778.05|903.53| +||5578|89420||1550|1904||11|1270||12.03|0.36|12.39|||11.06|0.38|0.59|74.90| +|45181|10104||1869209|4677|1345|||1271|14||14.06|214.96||||||| +2452654|36372|8691|42975|395676|4730|25431|7|26|1271|20|754.00|7.54|761.54|56.81|135.80|203.58|203.65|346.77|200.15| +2452695|48557|10362|73775|1763262|3596|36283|2|9|1271|15|800.10|40.00|840.10|6.27|237.30|8.00|158.42|633.68|283.57| +2451853|35633|7147|58279|746889|795|35148|8|4|1272|31|3073.96|122.95|3196.91|40.83|779.96|2735.82|138.63|199.51|943.74| +2451523|38737|9928|84705|670025|2182|40339|2|3|1273|25|1501.75|105.12|1606.87|69.13|385.00|465.54|487.01|549.20|559.25| +2451563|30073|15806|84705|1378982|788|38194|7|19|1273|4|227.48|2.27|229.75|68.30|80.88|113.74|101.22|12.52|151.45| +2452238|45181|11111|95843|911952|5308|17817|1|17|1274|61|2283.23|91.32|2374.55|27.19|1129.11|1735.25|537.02|10.96|1247.62| +2452244|60318|11543|24316|1112688|5222|20156|7|1|1274|1|77.24|4.63|81.87|86.53|28.96|26.26|22.43|28.55|120.12| +2452254|37191|12449|24316|1784102|5559|39463|2|22|1274|41|344.81|10.34|355.15|81.34|221.40|262.05|63.72|19.04|313.08| +2451126|60209|13438|1924|283723|1252|7628|8|16|1275|28|143.36|8.60|151.96|51.07|135.24|51.60|32.11|59.65|194.91| +2451866|48654|8726|65915|848542|1122|46549|7|1|1278|13|111.54|10.03|121.57|71.20|558.22|108.19|1.10|2.25|639.45| +2451812|29143|607|8400|530410|4497|20937|8|22|1278|15|457.80|4.57|462.37|4.83|88.35|357.08|83.59|17.13|97.75| +2451186|29782|10459|63129|1080154|2197|47793|4|5|1279|23|45.31|1.81|47.12|93.49|136.39|11.32|21.41|12.58|231.69| +2452807|56408|15295|49207|1103810|3638|18666|4|5|1280|13|1103.96|55.19|1159.15|98.69|111.93|121.43|962.87|19.66|265.81| +2452376|40621|12057|46939|145556|2524|23445|4|26|1281|1|79.84|0.79|80.63|87.83|8.73|20.75|31.90|27.19|97.35| +2452323|34185|16853|58921|452499|6330|16961|7|8|1284|12|19.44|1.55|20.99|39.79|273.00|10.10|4.29|5.05|314.34| +2452268|49093|303|58921|352766|1465|18964|4|32|1284|22|2649.46|158.96|2808.42|66.58|827.86|1139.26|166.12|1344.08|1053.40| +2451915|44061|13922|62831|312414|3597|5580|7|34|1285|3|237.72|4.75|242.47|56.36|132.06|213.94|14.26|9.52|193.17| +2452363|47625|13427|84401|358852|4315|38978|8|34|1286|3|88.26|0.00|88.26|60.31|79.11|63.54|14.33|10.39|139.42| +2452255|48021|7595|45516|1427781|594|34207|2|8|1287|63|4386.69|394.80|4781.49|36.01|455.49|350.93|1695.01|2340.75|886.30| +2452278|28856|2815|88398|1883853|6975|9250|10|9|1288|6|449.58|8.99|458.57|41.95|127.02|148.36|283.14|18.08|177.96| +2452126|30452|4033|847|190642|892|20742|7|9|1290|11|378.40|7.56|385.96|4.32|33.55|351.91|15.62|10.87|45.43| +2452112|51430|435|847|1671283|4991|17099|8|20|1290|37|1918.08|76.72|1994.80|10.61|944.98|613.78|1265.17|39.13|1032.31| +2452753|46946|11809|83869|524886|3239|26792|1|6|1291|19|668.23|26.72|694.95|36.08|303.05|247.24|319.95|101.04|365.85| +|50796|9613|||4701||10||1291|23|||||205.16||13.37|2.37|| +2451025|35558|16532|25617|802647|4683|26366|7|20|1292|39|3387.54|101.62|3489.16|50.61|880.62|779.13|1669.38|939.03|1032.85| +2451005|51511|26|58412|373168|5971|3574|2|8|1292|62|686.96|48.08|735.04|26.94|189.72|632.00|8.24|46.72|264.74| +2451612|47346|12004|25855|1300236|1817|42998|10|13|1293|56|10.08|0.30|10.38|48.50|475.44|5.04|1.31|3.73|524.24| +2451651|39384|715|29754|1500160|1409|45905|7|13|1295|44|6058.36|60.58|6118.94|96.74|135.96|5391.94|526.47|139.95|293.28| +2451277|42438|9488|41472|627326|6070|20014|10|32|1296|1|70.52|5.64|76.16|61.94|4.98|26.79|42.85|0.88|72.56| +2451640|37829|6880|38925|1195427|6954|32881|4|17|1297|23|884.58|8.84|893.42|67.80|884.58|114.99|169.30|600.29|961.22| +2451574|49319|15958|38925|1641429|6546|32063|2|32|1297|2|13.32|0.53|13.85|1.93|14.72|2.79|6.73|3.80|17.18| +2451612|35297|3068|38925|441263|2430|24350|7|32|1297|26|858.26|17.16|875.42|57.16|168.22|188.81|408.36|261.09|242.54| +|47709|8175||1084147||||14|1298|13||||||3.15|||| +2452764|40761|3087|21547|1901966|6958|30750|2|7|1298|20|252.20|22.69|274.89|99.75|241.80|216.89|27.18|8.13|364.24| +2452087|38601|17017|82992|948455|1142|43029|2|16|1299|7|138.11|11.04|149.15|34.60|109.34|77.34|38.89|21.88|154.98| +2451629|39504|901|92157|979652|4531|48657|2|17|1301|18|1916.46|76.65|1993.11|76.19|762.66|555.77|340.17|1020.52|915.50| +2451864|46413|16343|57707|149896|7139|41582|7|15|1303|14|442.82|4.42|447.24|68.72|46.20|398.53|39.86|4.43|119.34| +2451516|41274|11725|4471|999039|4405|35771|8|25|1305|38|217.74|10.88|228.62|7.69|30.40|185.07|32.01|0.66|48.97| +2452320|37498|12991|3102|1205021|4240|20612|8|15|1306|14|245.28|12.26|257.54|1.19|43.82|139.80|48.52|56.96|57.27| +2451052|34742|12973|61840|1719798|7091|36031|10|4|1308|6|527.04|36.89|563.93|79.29|186.00|105.40|59.02|362.62|302.18| +2451899|33567|11105|32392|1256270|1546|4822|7|34|1309|27|273.24|16.39|289.63|20.55|147.69|120.22|4.59|148.43|184.63| +2451858|47950|13157|32392|1898880|1145|39810|8|5|1309|5|275.00|8.25|283.25|71.77|210.80|74.25|26.09|174.66|290.82| +2452756|44433|10903|95738|199069|5953|27237|2|22|1310|22|1205.16|12.05|1217.21|11.19|535.48|301.29|659.82|244.05|558.72| +||13380|||6553||||1310||793.98||833.67|62.67||||309.66|896.34| +2452024|51877|12157|9635|1220718|1754|42037|2|18|1312|43|1623.25|81.16|1704.41|94.15|1818.04|259.72|954.47|409.06|1993.35| +2451117|35113|17294|30699|683123|5091|39059|2|28|1313|1|20.03|1.40|21.43|87.72|10.51|4.20|5.54|10.29|99.63| +2451143|43591|14620|4512|333459|6|31706|4|13|1316|15|93.90|4.69|98.59|55.38|11.25|23.47|7.04|63.39|71.32| +2450987|32431|10408|32400|1342563|2371|12179|1|13|1317|8|640.96|25.63|666.59|86.18|83.20|474.31|118.32|48.33|195.01| +2452625|50947|14973|29451|147580|5097|41414|1|8|1318|38|88.54|0.00|88.54|18.37|744.42|87.65|0.59|0.30|762.79| +2451224|46121|5756|40961|79254|2492|10083|2|7|1319|1|3.72|0.14|3.86|41.48|9.48|2.12|0.94|0.66|51.10| +2451258|41972|6853|40961|1235198|5772|5593|4|3|1319|60|439.20|39.52|478.72|47.37|717.00|215.20|197.12|26.88|803.89| +2451383|48254|8251|45260|725335|5091|11647|4|9|1320|45|1590.75|15.90|1606.65|90.39|751.05|1193.06|206.79|190.90|857.34| +2450914|49459|16087|739|1322337|306|31522|1|15|1321|20|65.00|0.65|65.65|48.92|30.80|59.15|5.44|0.41|80.37| +2451382|53993|2905|4421|604957|6003|46202|8|7|1322|38|142.50|0.00|142.50|29.72|274.74|69.82|59.59|13.09|304.46| +2452046|53252|16982|47815|1585367|1972|4177|1|10|1323|14|580.30|40.62|620.92|44.70|302.96|365.58|163.18|51.54|388.28| +2451932|39280|1613|47815|526569|5145|33321|7|16|1323|4|484.24|9.68|493.92|14.98|262.04|169.48|107.01|207.75|286.70| +2451926|42498|11714|47815|455966|1228|30537|4|33|1323|60|1212.60|36.37|1248.97|78.53|231.60|679.05|197.41|336.14|346.50| +2451252|33244|10789|54705|1465325|3693|49661|7|35|1324|6|28.14|0.56|28.70|96.68|188.94|1.68|12.96|13.50|286.18| +2451349|42561|6988|78200|1850298|2574|21250|8|4|1325|11|40.26|3.62|43.88|50.26|10.01|23.35|14.03|2.88|63.89| +2451201|30102|9796|78200|850681|7021|34561|8|29|1325|1|8.43|0.25|8.68|43.81|8.77|1.43|2.10|4.90|52.83| +2452292|38401|5477|15021|856553|356|45359|8|32|1326|2|108.70|4.34|113.04|73.10|56.52|89.13|18.39|1.18|133.96| +2452197|53644|4575|15021|1409195|5953|25064|7|13|1326|3|55.17|1.65|56.82|8.80|60.54|18.20|15.15|21.82|70.99| +2451560|59222|5192|78767|334824|4446|45668|7|12|1328|74|2646.98|105.87|2752.85|51.85|58.46|291.16|1154.35|1201.47|216.18| +2451639|36229|7882|50005|1733175|810|32398|2|4|1329|55|475.75|23.78|499.53|28.92|623.70|399.63|25.88|50.24|676.40| +2451540|37688|1318|50005|1253243|153|45910|2|35|1329|52|4102.80|246.16|4348.96|41.85|1297.92|2871.96|849.27|381.57|1585.93| +2451936|49956|17755|56285|708140|3609|33078|8|16|1330|1|52.62|0.00|52.62|45.70|3.43|38.93|12.32|1.37|49.13| +2451907|58657|4513|56285|591511|358|22005|7|14|1330|34|720.46|21.61|742.07|92.33|193.80|237.75|106.19|376.52|307.74| +2451918|55173|7610|56285|780188|756|31724|1|3|1330|25|1212.50|0.00|1212.50|22.61|648.50|1188.25|16.97|7.28|671.11| +2452043|38659|16661|48416|1395078|445|28565|1|25|1331|11|546.48|43.71|590.19|79.21|237.27|338.81|49.84|157.83|360.19| +2452268|57864|17545|34435|476496|842|10035|7|16|1336|1|60.38|3.01|63.39|64.75|27.14|48.90|6.54|4.94|94.90| +||10783|||||||1336|29|||1560.36|21.90||515.21||928.13|| +2451831|42497|14075|67065|229046|6437|37875|7|17|1337|22|106.70|8.53|115.23|96.17|335.50|14.93|31.20|60.57|440.20| +2451378|48250|2284|48578|1213385|4272|46312|10|4|1338|67|1762.77|105.76|1868.53|37.73|117.25|1304.44|417.08|41.25|260.74| +2451318|43185|5293|48578|687654|4963|22672|1|30|1338|26|1480.44|133.23|1613.67|0.73|653.64|592.17|355.30|532.97|787.60| +2452469|50707|13009|55554|1918083|5888|46493|7|33|1339|10|1308.70|52.34|1361.04|64.06|409.80|1020.78|28.79|259.13|526.20| +2452510|28825|1239|55554|285009|4120|14031|10|8|1339|20|895.00|44.75|939.75|47.77|9.20|796.55|54.14|44.31|101.72| +2452598|47569|3387|9622|258973|4766|22959|7|30|1340|28|0.00|0.00|0.00|49.92|243.32|0.00|0.00|0.00|293.24| +2452514|60264|1269|17305|62901|3310|13272|1|23|1341|23|977.04|0.00|977.04|51.02|534.98|390.81|527.60|58.63|586.00| +2452292|34346|3687|51122|1547897|6500|22303|1|17|1345|30|941.40|9.41|950.81|31.31|56.70|301.24|320.08|320.08|97.42| +2452272|45453|1565|63943|723510|1477|30644|4|10|1345|27|459.81|4.59|464.40|73.89|275.94|91.96|360.49|7.36|354.42| +2451693|49605|2948|85806|1225996|6050|9704|2|32|1346|38|65.36|0.65|66.01|36.84|262.58|39.86|20.14|5.36|300.07| +2451761|40303|6953|85806|339329|1266|33956|8|18|1346|42|3235.68|0.00|3235.68|85.38|1110.48|291.21|2208.35|736.12|1195.86| +2452333|31381|7763|68316|416523|5989|44655|1|22|1349|63|1148.49|57.42|1205.91|92.92|656.46|1068.09|68.34|12.06|806.80| +2452212|56848|5019|75045|446752|506|43413|2|22|1349|29|832.59|16.65|849.24|73.21|0.00|691.04|94.83|46.72|89.86| +2451895|51926|1868|27235|1021973|7003|4033|1|25|1350|36|355.32|31.97|387.29|66.72|847.44|103.04|171.55|80.73|946.13| +2452302|55454|8491|40201|1047476|1951|36473|10|3|1351|15|350.85|14.03|364.88|70.61|250.65|252.61|81.53|16.71|335.29| +||7287|10858||3401||8|30|1354|||81.15||30.55|92.16|0.00|173.89||| +2452363|52538|14031|10858|87077|6379|10928|8|6|1354|2|107.42|5.37|112.79|25.54|20.74|106.34|1.04|0.04|51.65| +2451209|41110|13306|40309|729187|3699|34164|4|14|1358|5|18.50|0.37|18.87|43.16|16.70|0.92|3.51|14.07|60.23| +2451173|33916|4006|84619|910828|4590|37552|1|3|1358|18|361.98|10.85|372.83|12.57|479.34|217.18|136.11|8.69|502.76| +||16795|72305||7170|||13|1359||||187.95|29.62|||36.94|6.02|129.77| +2452461|32827|8401|64566|1234749|4143|49537|1|35|1359|31|3057.53|152.87|3210.40|54.19|1551.55|1467.61|731.36|858.56|1758.61| +2451279|45517|15328|47171|406109|4694|46832|10|23|1360|17|1568.59|31.37|1599.96|18.87|616.25|94.11|1444.99|29.49|666.49| +2451161|49530|7603|34184|1319135|958|26781|2|7|1360|8|77.04|5.39|82.43|52.55|7.20|74.72|0.18|2.14|65.14| +2451198|36636|3220|49278|382014|1227|17240|1|26|1361|49|221.97|6.65|228.62|79.32|137.20|195.33|24.77|1.87|223.17| +2452135|45496|1179|52379|1891231|2003|41575|4|14|1362|44|309.32|6.18|315.50|13.71|128.92|167.03|56.91|85.38|148.81| +2451943|58287|5053|73014|439464|5421|47378|1|5|1363|20|237.20|14.23|251.43|54.11|11.40|118.60|64.04|54.56|79.74| +2451931|47006|11831|73014|514003|4206|48009|10|15|1363|50|2168.00|195.12|2363.12|19.12|1818.50|390.24|1066.65|711.11|2032.74| +2451940|45644|5737|73014|277922|5126|34283|8|19|1363|43|384.85|34.63|419.48|30.11|422.26|200.12|171.79|12.94|487.00| +2452710|60498|17532|11218|1710581|3180|5826|2|31|1364|39|2454.27|0.00|2454.27|28.74|3051.36|1938.87|345.31|170.09|3080.10| +2452632|55977|7926|11218|277570|5917|18264|4|25|1364|41|2512.07|25.12|2537.19|30.14|1209.50|427.05|1959.91|125.11|1264.76| +2451324|35926|9362|49663|729791|5371|14151|7|28|1365|6|559.26|33.55|592.81|88.55|0.00|106.25|425.82|27.19|122.10| +2451285|44296|14101|49663|879027|747|43667|10|25|1365|32|4840.64|242.03|5082.67|4.07|1391.04|3098.00|104.55|1638.09|1637.14| +2451483|45744|11404|58016|1053893|4880|47114|7|27|1366|27|3629.88|290.39|3920.27|13.80|1043.01|980.06|1033.42|1616.40|1347.20| +2452205|43872|10869|37296|1673927|393|12106|7|18|1368|17|17.34|1.04|18.38|78.57|8.50|5.89|3.77|7.68|88.11| +2451922|35922|13747|69808|1754726|3545|40717|7|10|1369|9|565.20|45.21|610.41|76.06|129.96|248.68|174.08|142.44|251.23| +2451870|55323|11462|69808|1912319|6939|28359|4|26|1369|70|51.80|4.14|55.94|47.35|1099.00|8.80|42.57|0.43|1150.49| +2452731|41587|8712|43461|511017|4620|29823|8|32|1370|35|50.40|1.00|51.40|21.68|310.45|15.62|7.99|26.79|333.13| +2452480|33779|2677|97070|1484770|2925|7833|4|6|1371|27|173.88|5.21|179.09|66.86|260.82|118.23|48.97|6.68|332.89| +2451564|43618|12439|32830|1623686|4570|13454|2|2|1372|15|1116.30|89.30|1205.60|21.14|994.50|178.60|234.42|703.28|1104.94| +2451895|49485|15440|48585|1758820|6655|41690|8|20|1373|19|32.87|2.95|35.82|27.63|49.97|15.12|11.89|5.86|80.55| +2451998|59875|8387|90907|1797686|4818|3997|10|30|1374|46|193.20|7.72|200.92|44.85|327.52|75.34|76.60|41.26|380.09| +2452653|49744|6145|20426|1773913|4200|6887|4|3|1378|1|9.65|0.86|10.51|35.68|4.24|2.21|1.33|6.11|40.78| +2452751|50299|5226|13026|264299|5943|13140|2|16|1378|40|1919.20|115.15|2034.35|93.18|1057.20|1784.85|53.74|80.61|1265.53| +2451840|31871|14012|31868|1667917|4891|9465|10|10|1380|1|64.31|0.64|64.95|92.63|37.14|12.86|38.07|13.38|130.41| +2451880|35632|13507|75910|487307|4751|21748|1|17|1380|17|18.02|0.90|18.92|51.74|37.06|5.94|10.87|1.21|89.70| +2451535|47634|938|22580|1490407|3034|14061|1|7|1382|54|2178.90|130.73|2309.63|24.04|2299.86|305.04|1292.96|580.90|2454.63| +2452709|31753|8346|52635|1463697|6522|25347|1|5|1383|41|278.39|19.48|297.87|43.81|515.37|231.06|10.41|36.92|578.66| +2451817|36904|14651|49037|1318649|2095|37541|1|3|1385|2|77.84|4.67|82.51|85.07|17.96|0.77|0.77|76.30|107.70| +2451639|53486|4423|96752|1264694|1563|8432|8|20|1387|53|164.83|14.83|179.66|81.05|39.22|120.32|38.27|6.24|135.10| +2451603|36565|2126|96752|838328|7051|34527|7|26|1387|9|537.75|0.00|537.75|11.39|163.17|112.92|301.62|123.21|174.56| +2452190|29622|13257|16838|1625937|68|25799|1|23|1391|7|91.56|4.57|96.13|6.45|35.84|0.00|5.49|86.07|46.86| +2452092|33904|6407|16838|716049|2054|14859|8|2|1391|82|3136.50|250.92|3387.42|80.40|2940.52|2728.75|326.20|81.55|3271.84| +2452380|35004|6191|62437|1365154|1047|4940|4|28|1392|3|51.63|3.09|54.72|85.78|131.97|44.40|6.86|0.37|220.84| +2452328|35079|17327|62437|755936|6006|24876|10|28|1392|29|318.13|28.63|346.76|43.21|416.15|108.16|6.29|203.68|487.99| +2452410|49677|14737|75469|1386717|737|47157|1|19|1393|6|302.70|6.05|308.75|9.36|165.60|178.59|70.74|53.37|181.01| +2451192|30631|11647|81463|424098|267|27989|1|33|1394|10|179.70|7.18|186.88|10.95|25.30|138.36|26.04|15.30|43.43| +2451214|59794|15637|44570|1301415|1460|44305|4|12|1394|17|475.83|23.79|499.62|26.69|892.16|71.37|250.76|153.70|942.64| +2451195|49222|853|81463|1489521|5521|3568|7|28|1394|74|45.14|0.00|45.14|98.68|1041.18|12.63|22.75|9.76|1139.86| +2452420|34053|8851|56676|1756097|3165|31433|8|7|1395|59|274.94|10.99|285.93|40.10|139.83|181.46|9.34|84.14|190.92| +2452278|36961|12797|44143|172350|3141|732|4|19|1395|17|406.98|0.00|406.98|53.11|78.54|130.23|74.72|202.03|131.65| +2451689|49274|14893|73835|305865|2254|690|4|25|1396|73|103.66|8.29|111.95|93.88|24.09|57.01|33.12|13.53|126.26| +2451652|55181|11894|73835|1226331|1638|29298|7|33|1396|7|570.78|17.12|587.90|52.95|89.39|239.72|16.55|314.51|159.46| +2452686|40877|16554|58594|186843|2645|25694|8|22|1397|14|1250.76|25.01|1275.77|38.77|24.50|187.61|744.20|318.95|88.28| +2452725|39712|2598|61176|1099532|217|44855|8|23|1397|3|104.10|5.20|109.30|70.68|10.56|22.90|65.77|15.43|86.44| +2452661|33042|15546|61176|1707701|1035|37569|8|2|1397|6|13.80|0.13|13.93|49.67|4.86|3.45|6.10|4.25|54.66| +2451662|49721|12974|||||||1398||||2396.48|23.01||||50.03|| +2452385|30215|12289|81152|1075211|2164|14670|8|29|1399|5|202.15|18.19|220.34|26.59|67.35|4.04|152.54|45.57|112.13| +2452348|46640|6431|81152|320234|5057|47880|1|26|1399|69|365.70|0.00|365.70|1.47|69.00|164.56|96.54|104.60|70.47| +2452392|48651|16523|32974|410011|3495||10|30|1399|29||7.53||87.88|463.71||37.30|45.61|| +2451990|56671|8498|79810|435687|5247|41207|8|17|1401|18|212.40|12.74|225.14|74.74|64.80|140.18|36.83|35.39|152.28| +2451969|31498|15493|19794|443045|6751|46857|2|7|1403|47|2879.69|28.79|2908.48|53.80|1955.67|863.90|40.31|1975.48|2038.26| +2452021|58195|14834|83663|1328460|2504|8548|10|21|1403|29|1363.87|0.00|1363.87|20.62|1518.73|354.60|545.00|464.27|1539.35| +2451553|37492|7502|34235|1182884|891|19698|8|17|1405|83|2183.73|174.69|2358.42|44.80|960.31|1419.42|22.92|741.39|1179.80| +2451872|44408|15386|36017|1674329|1088|22200|10|34|1406|37|717.43|14.34|731.77|26.28|1686.09|652.86|14.20|50.37|1726.71| +2451930|50430|8846|30665|334813|4148|34683|2|6|1407|8|29.36|1.76|31.12|92.05|19.52|20.84|8.43|0.09|113.33| +2451871|37042|10435|30665|1637099|3951|10356|2|10|1407|3|25.89|0.00|25.89|36.73|89.67|5.17|7.25|13.47|126.40| +2452747|51784|10093|17861|327943|1611|4926|10|12|1408|56|408.80|28.61|437.41|17.50|346.08|171.69|52.16|184.95|392.19| +2451973|43774|5143|31317|1447578|2333|34688|7|1|1410|4|74.92|0.74|75.66|90.97|173.32|72.67|0.20|2.05|265.03| +2452331|49808|9153|75555|1862350|3756|43962|4|21|1411|7|153.30|1.53|154.83|72.70|25.55|121.10|3.22|28.98|99.78| +2452277|36037|13817|75555|1026770|1063|36556|4|27|1411|7|32.83|0.32|33.15|33.47|525.49|32.50|0.11|0.22|559.28| +2452773|47195|11154|35809|1411051|3057|32710|2|22|1412|6|19.38|0.19|19.57|23.78|17.88|5.42|9.35|4.61|41.85| +2451909|42287|7841|19526|1898410|3659|30508|8|21|1413|59|83.19|6.65|89.84|60.84|87.32|44.92|22.96|15.31|154.81| +2451759|44033|3494|19526|502883|1211|9139|1|5|1413|36|2847.96|142.39|2990.35|67.67|174.24|1196.14|148.66|1503.16|384.30| +2451993|55333|13153|24147|743302|3257|2765|10|8|1414|41|1532.58|137.93|1670.51|27.00|2799.07|1118.78|91.03|322.77|2964.00| +2451856|39925|2828|24147|557006|4779|14105|7|26|1414|64|5864.96|175.94|6040.90|85.45|3454.72|5219.81|419.34|225.81|3716.11| +||15548|45519|13889||11252||30|1415|1|36.20|1.44||24.94|7.75||||| +2451421|57967|4162|92100|1369830|6829|35075|2|15|1416|11|509.08|40.72|549.80|60.68|283.69|366.53|7.12|135.43|385.09| +2451568|59657|13828|92100|1576823|5354|6638|4|33|1416|12|25.80|2.06|27.86|35.75|426.96|3.87|15.35|6.58|464.77| +|49023|11468|98115||5557|15231||33|1417|47|8.93|0.80|9.73|||2.85|2.67|3.41|| +2451455|39455|1784|95582|1699023|3845|12267|4|6|1417|12|7.32|0.65|7.97|43.41|4.08|6.44|0.29|0.59|48.14| +2452154|45139|10955|51636|1556876|4737|34551|4|26|1418|20|1271.40|38.14|1309.54|99.25|706.20|177.99|76.53|1016.88|843.59| +2452387|57996|9033|28239|1666421|5756|16338|7|27|1419|1|41.66|0.00|41.66|52.63|20.33|4.99|9.90|26.77|72.96| +2452673|37105|17820|69281|1295566|2523|633|4|20|1420|6|401.10|16.04|417.14|41.82|72.90|148.40|55.59|197.11|130.76| +2451396|61017|8936|71206|642236|1447|45811|4|31|1422|24|602.16|12.04|614.20|39.00|825.84|90.32|5.11|506.73|876.88| +2451221|56609|17617|56510|190503|988|23026|4|1|1425|7|168.70|1.68|170.38|53.91|305.34|87.72|25.91|55.07|360.93| +2451603|46714|16045|27093|985814|6890|27166|1|16|1426|5|402.95|36.26|439.21|66.77|169.65|181.32|84.21|137.42|272.68| +2451559|54394|17563|27093|1708477|5793|22262|8|2|1426|74|0.00|0.00|0.00|17.43|315.24|0.00|0.00|0.00|332.67| +2451151|29110|12272||1328834|2640|41293||7|1427|35|105.00|2.10|107.10|28.21||0.00||99.75|| +2451225|54517|10291|31444|1471952|6997|8042|10|24|1428|43|1512.31|0.00|1512.31|35.57|328.52|1224.97|31.60|255.74|364.09| +2451281|34852|6313|31444|695865|2425|24538|2|3|1428|36|157.32|9.43|166.75|16.07|605.52|129.00|25.20|3.12|631.02| +2451569|42311|11024|96297|161346|1075|45204|8|17|1429|3|186.06|0.00|186.06|30.79|121.50|44.65|89.08|52.33|152.29| +2451397|50683|7285|94544|213967|3886|35852|8|30|1431|1|2.34|0.07|2.41|47.76|0.09|2.05|0.07|0.22|47.92| +2451479|34079|11234|94544|1407073|6395|46470|8|12|1431|14|605.08|36.30|641.38|14.83|513.66|471.96|97.17|35.95|564.79| +2451196|32599|6568|68241|1292923|4279|29588|2|20|1432|7|1076.46|32.29|1108.75|53.24|526.26|473.64|319.49|283.33|611.79| +2451727|57109|14090|92354|369893|658|3470|4|5|1433|15|630.60|25.22|655.82|82.55|417.45|384.66|86.07|159.87|525.22| +2451834|35894|17263|76464|380057|5790|4770|7|30|1434|4|492.88|14.78|507.66|93.15|123.20|300.65|180.69|11.54|231.13| +2452312|46664|4985|72155|624073|4082|23644|10|15|1436|88|1283.92|77.03|1360.95|4.47|2217.60|706.15|410.21|167.56|2299.10| +2452316|38690|2781|72155|1491722|5498|28474|2|16|1436|7|47.81|0.00|47.81|85.62|3.29|34.90|7.74|5.17|88.91| +2452376|54625|6309|72155|885345|4659|17926|1|5|1436|25|560.25|22.41|582.66|5.26|265.75|16.80|304.33|239.12|293.42| +2451148|51272|14905|41353|975128|550|7172|7|9|1437|31|3277.01|196.62|3473.63|5.46|278.69|2752.68|78.64|445.69|480.77| +2451164|53397|9625|44505|1621774|2633|46692|10|12|1437|5|163.40|0.00|163.40|54.37|14.05|120.91|15.72|26.77|68.42| +2451276|37885|11620|92102|202723|2497|21471|4|29|1437|4|29.40|0.29|29.69|21.76|65.16|18.22|4.13|7.05|87.21| +2452635|52753|13113|83601|1542098|6811|32324|4|30|1438|30|787.20|55.10|842.30|77.39|658.50|236.16|347.15|203.89|790.99| +2452612|36087|553|83601|1860770|4132|8294|4|21|1438|14|16.10|0.32|16.42|46.52|20.72|5.63|3.24|7.23|67.56| +2452714|30850|5490|83601|1452133|1678|16423|7|30|1438|21|130.20|3.90|134.10|2.80|111.51|130.20|0.00|0.00|118.21| +2451594|53990|4505|88239|866990|4830|20623|8|33|1439|57|3864.03|193.20|4057.23|79.25|1839.96|2395.69|646.06|822.28|2112.41| +2452624|47067|3084|89957|1372180|6614|19848|2|4|1440|5|85.80|6.00|91.80|23.53|50.60|25.74|4.80|55.26|80.13| +2451223|48469|3181|665|1072711|2560|26409|10|12|1441|1|84.33|5.05|89.38|72.37|8.16|27.82|51.42|5.09|85.58| +2451154|30857|9614|665|1078674|5729|15137|8|30|1441|71|810.82|40.54|851.36|80.64|624.09|737.84|18.97|54.01|745.27| +2451302|51627|14707|6768|1209652|3972|22692|10|22|1442|3|52.53|2.62|55.15|59.30|0.00|45.17|6.32|1.04|61.92| +2451350|30969|92|6768|1497890|1359|46828|8|21|1442|43|744.33|22.32|766.65|25.14|58.91|267.95|171.49|304.89|106.37| +2451317|54689|14674|6768|197580|380|29980|10|18|1442|17|362.27|0.00|362.27|70.63|303.62|326.04|7.24|28.99|374.25| +2452352|40410|3019|15572|1792377|5629|834|8|21|1445|32|593.92|35.63|629.55|36.36|259.84|326.65|243.21|24.06|331.83| +2452335|30953|8535|55108|1058199|6061|29876|10|20|1445|1|31.02|1.55|32.57|40.50|0.86|16.13|7.14|7.75|42.91| +2452285|44321|3227|55108|1499262|1128|24994|8|3|1445|22|146.30|2.92|149.22|53.03|240.46|119.96|12.90|13.44|296.41| +2452321|31967|5325|74153|1843926|3462|15798|7|5|1447|2|209.98|10.49|220.47|41.12|5.06|170.08|10.37|29.53|56.67| +2452249|45491|3331|74153|247375|1913|32451|8|22|1447|26|2024.36|121.46|2145.82|74.06|228.54|587.06|833.63|603.67|424.06| +2452124|49097|4905|25920|964168|3496|20824|1|3|1451|52|2094.56|83.78|2178.34|38.99|1192.88|1654.70|303.50|136.36|1315.65| +2452134|49768|14533|29909|667951|4029|46507|2|6|1452|3|367.35|22.04|389.39|60.47|189.24|205.71|155.17|6.47|271.75| +2452230|51559|7903|29909|348666|1114|16409|7|2|1452|1|13.52|0.81|14.33|69.42|8.69|3.38|4.86|5.28|78.92| +2452123|28955|1089|92488|322838|3588|17138|2|3|1452|5|340.15|6.80|346.95|46.69|20.40|13.60|306.95|19.60|73.89| +2451640|56617|13855|55686|1696325|4944|32010|4|15|1453|63|5.04|0.20|5.24|21.54|217.98|3.57|0.51|0.96|239.72| +2451639|51773|2758|55686|163270|6168|27819|4|3|1453|84|1412.88|70.64|1483.52|77.68|1907.64|268.44|194.55|949.89|2055.96| +2451678|29257|6658|55686|1211161|5206|28804|10|32|1453|39|153.66|12.29|165.95|92.86|92.04|138.29|12.91|2.46|197.19| +2451655|47050|1249|55686|1076134|3139|2544|1|34|1453|3|27.09|1.89|28.98|8.80|85.83|7.31|2.96|16.82|96.52| +2451151|47701|13214|14070|1796748|1026|28104|4|2|1454|92|5681.00|340.86|6021.86|14.55|1718.56|2613.26|2975.70|92.04|2073.97| +2451158|44598|925|10576|961555|4534|32360|8|23|1454|5|200.20|18.01|218.21|49.77|12.75|44.04|12.49|143.67|80.53| +2451224|35824|17366|79119|1317395|7058|31121|8|29|1455|14|461.58|27.69|489.27|30.04|584.64|276.94|110.78|73.86|642.37| +2451206|32173|5245|61128|279938|795|28805|1|2|1455|21|966.42|57.98|1024.40|55.35|422.10|86.97|844.27|35.18|535.43| +2451333|60548|5026|95219|27298|2224|34750|4|10|1456|12|599.04|11.98|611.02|80.59|149.76|365.41|142.51|91.12|242.33| +2451319|47220|12626|83885|1751104|2540|29176|2|11|1456|78|5087.94|101.75|5189.69|89.63|1055.34|2696.60|1865.24|526.10|1246.72| +2451204|59942|7645|91966|1737922|6261|13047|4|34|1457|22|631.62|18.94|650.56|44.44|412.50|587.40|34.93|9.29|475.88| +2451120|46512|10897||635721|4635||10||1457|29|770.53|15.41|785.94|||739.70|12.64||595.60| +2451107|30623|8594|91966|1694950|2846|12789|8|26|1457|2|16.70|0.83|17.53|78.50|0.00|3.50|12.14|1.06|79.33| +2451555|30728|5602|28083|742935|4187|20686|10|12|1458|28|73.36|2.93|76.29|33.64|8.68|8.06|3.26|62.04|45.25| +2451488|39146|4082|66459|948447|2314|23334|1|3|1458|16|850.56|51.03|901.59|90.88|79.68|391.25|454.71|4.60|221.59| +2451503|46400|9109|66459|983868|4760|39699|4|25|1458|44|1043.68|83.49|1127.17|25.95|105.16|0.00|563.58|480.10|214.60| +2451323|40834|17540|38369|819868|2689|41157|8|23|1461|7|34.72|0.00|34.72|50.30|26.04|14.23|2.25|18.24|76.34| +2451558|32137|3010|11406|1760511|5169|15074|7|20|1463|9|712.98|64.16|777.14|7.08|223.47|163.98|21.96|527.04|294.71| +2452487|38152|15585|27269|1713467|5712|49557|2|2|1465|66|1280.40|102.43|1382.83|99.84|1723.92|1203.57|32.26|44.57|1926.19| +2452493|59878|5241|93261|753025|3189|6153|7|28|1465|1|1.41|0.00|1.41|31.78|2.25|0.70|0.28|0.43|34.03| +2452538|57675|6259|64291|1715780|540|47300|8|18|1465|1|4.01|0.16|4.17|37.19|4.01|1.32|2.04|0.65|41.36| +2451151|32049|1492|45271|789667|1710|12106|4|10|1466|5|291.20|0.00|291.20|57.71|126.35|171.80|77.61|41.79|184.06| +2451284|49094|10214|8537|933781|6971|16561|1|26|1466|40|812.00|40.60|852.60|4.74|575.20|276.08|5.35|530.57|620.54| +2451921|38281|12373|21968|676957|2793|28229|7|14|1467|1|9.32|0.65|9.97|46.47|30.46|4.38|4.14|0.80|77.58| +2451731|39997|15199|68902|1402110|6664|49665|8|20|1468|68|7388.20|295.52|7683.72|67.04|3739.32|2881.39|4461.74|45.07|4101.88| +2451640|43571|5299|68902|1863043|4972|5254|7|8|1468|15|480.30|33.62|513.92|53.41|304.05|124.87|227.47|127.96|391.08| +2451578|59913|17518|70059|1098122|4264|44406|10|35|1470|60|671.40|26.85|698.25|40.73|633.60|40.28|593.25|37.87|701.18| +2451208|56763|9196|73741|1666183|4225|15867|8|1|1472|47|385.40|19.27|404.67|56.90|281.06|88.64|130.57|166.19|357.23| +2451943|31834|155|28359|1316900|5801|26702|1|34|1473|12|8.88|0.79|9.67|23.46|366.36|8.08|0.05|0.75|390.61| +2451928|35712|1291|28359|354579|3676|25173|1|25|1473|13|889.59|71.16|960.75|15.63|688.22|596.02|135.04|158.53|775.01| +2451941|48546|15086|28359|437656|5997|18691|4|34|1473|8|325.20|9.75|334.95|15.14|126.88|6.50|143.41|175.29|151.77| +2451264|44854|8725|12942|301633|4903|973|8|16|1475|25|3039.25|212.74|3251.99|16.73|1082.75|1094.13|992.01|953.11|1312.22| +2451802|31735|9950|22705|131676|5073|28023|10|7|1477|39|3050.19|61.00|3111.19|83.12|772.20|2806.17|187.89|56.13|916.32| +2452410|41781|2953|65238|118897|6621|39545|8|18|1478|16|40.16|1.60|41.76|59.02|503.68|34.53|1.46|4.17|564.30| +2452338|48047|17307|65238|808941|4484|26027|2|3|1478|58|1219.74|85.38|1305.12|21.58|506.92|719.64|240.04|260.06|613.88| +2452453|53470|2253|65238|1323191|1389|28387|1|16|1478|88|1197.68|35.93|1233.61|67.82|315.92|407.21|197.61|592.86|419.67| +2452240|33799|3191|17558|1469722|4163|15757|2|22|1479|10|588.10|35.28|623.38|15.94|27.00|417.55|109.15|61.40|78.22| +2451293|42136|14174|51808|537610|1942|5921|1|5|1480|2|91.52|0.00|91.52|21.41|52.98|10.06|10.58|70.88|74.39| +2451370|50948|2227|37233|1140868|3703|46538|7|30|1480|8|564.16|5.64|569.80|82.43|56.40|124.11|123.21|316.84|144.47| +2452421|49053|5239|86459|264332|6652|45325|1|33|1481|51|653.31|26.13|679.44|57.89|684.42|385.45|61.60|206.26|768.44| +2452531|46345|12000|81736|1248510|11|47112|1|34|1481|19|999.97|79.99|1079.96|34.04|293.17|219.99|311.99|467.99|407.20| +2452392|41207|16649|80393|710921|497|8920|2|17|1482|10|712.50|64.12|776.62|40.30|251.40|92.62|489.70|130.18|355.82| +2451285|43570|5635|96411|1505183|2449|7715|8|15|1483|1|29.65|1.18|30.83|74.18|52.24|18.38|4.16|7.11|127.60| +2451320|40453|11828|96411|1825663|6515|18567|10|29|1483|4|12.44|0.62|13.06|28.29|19.56|6.34|5.06|1.04|48.47| +2452170|57898|489|7438|1735091|6178|43665|7|26|1484|40|493.20|0.00|493.20|23.42|143.20|221.94|233.28|37.98|166.62| +2452211|57907|943|7438|1831690|2512|22087|4|14|1484|18|425.70|8.51|434.21|50.92|39.96|251.16|144.86|29.68|99.39| +2451586|30567|6529|47776|524295|2767|33686|4|12|1486|41|822.05|0.00|822.05|4.78|854.85|254.83|130.46|436.76|859.63| +2451472|45499|15550|33535|1395879|5070|43861|8|21|1486|39|4513.86|225.69|4739.55|76.66|455.91|3159.70|785.41|568.75|758.26| +2451486|57651|4135|30837|1282485|3377|7422|2|19|1486|35|427.35|34.18|461.53|73.58|976.85|418.80|5.47|3.08|1084.61| +2451497|48020|17186|30837|1537059|2215|2618|2|10|1486|39|2636.01|0.00|2636.01|34.40|1238.25|2372.40|18.45|245.16|1272.65| +2451891|57544|4019|2328|326322|5163|18757|7|27|1487|6|68.40|1.36|69.76|83.49|93.30|56.08|1.84|10.48|178.15| +||1942|33822|1391022|||2|2|1488|||13.12||71.81|14.88||149.37||| +2452633|51834|8100|23836|733287|1112|24584|8|14|1489|16|49.76|1.99|51.75|47.68|373.60|4.47|41.21|4.08|423.27| +2452690|40481|12079|23836|1167579|2477|22928|2|34|1489|62|3597.24|71.94|3669.18|63.87|2248.12|1618.75|811.18|1167.31|2383.93| +||616||952475||13454||23|1491|3|||101.24||33.12||18.11||40.25| +2451848|53763|14204|16288|1436538|6246|25201|1|19|1492|65|1433.25|14.33|1447.58|9.65|2580.50|730.95|653.13|49.17|2604.48| +2452777|53999|16629|43365|783568|2169|47617|7|9|1493|55|0.00|0.00|0.00|15.27|0.00|0.00|0.00|0.00|15.27| +2452296|36396|10763|64372|967346|5653|407|10|18|1496|33|1407.45|70.37|1477.82|80.36|1374.45|253.34|380.85|773.26|1525.18| +2452310|49441|15479|64372|1654671|2993|19715|1|34|1496|61|1709.22|136.73|1845.95|78.31|1798.89|170.92|1199.87|338.43|2013.93| +2450978|59944|10444|82593|610677|5596|18115|2|23|1497|41|85.28|5.11|90.39|14.54|1322.66|55.43|25.67|4.18|1342.31| +2451933|46267|12341|78800|529800|2059|11518|10|24|1498|38|2817.32|225.38|3042.70|8.93|387.22|2451.06|340.62|25.64|621.53| +2451812|47021|14149|78800|1841403|973|22943|10|30|1498|17|931.26|46.56|977.82|11.30|678.98|270.06|92.56|568.64|736.84| +2452369|33269|12126|56492|297399|706|7613|1|25|1499|16|1964.64|176.81|2141.45|69.00|1122.56|1139.49|569.35|255.80|1368.37| +2452436|45302|1068|10113|1038458|6904|11750|1|24|1499|62|35.34|0.00|35.34|26.52|19.22|32.51|0.11|2.72|45.74| +2451626|53742|11846|67683|1851075|4421|17814|7|13|1500|25|285.00|25.65|310.65|70.21|201.75|28.50|38.47|218.03|297.61| +2451236|36444|8932|28055|658142|1590|6685|4|21|1502|30|551.70|11.03|562.73|8.68|1442.70|66.20|160.21|325.29|1462.41| +2451121|39740|2107|28055|1518356|5762|46721|10|21|1502|65|5605.60|168.16|5773.76|0.72|501.80|2634.63|326.80|2644.17|670.68| +2451358|50138|9835|75376|929057|1095|18733|4|24|1505|2|47.68|0.00|47.68|76.77|26.54|20.02|15.48|12.18|103.31| +2451950|54644|3449|2921|6251|2128|41605|10|34|1506|4|21.64|0.64|22.28|96.42|21.64|17.31|1.64|2.69|118.70| +2452078|34434|9561|2921|955684|2763|42689|1|24|1506|79|1471.77|44.15|1515.92|34.79|1773.55|161.89|641.84|668.04|1852.49| +2451872|35577|12707|68198|1872455|265|33722|10|19|1508|56|1085.28|75.96|1161.24|70.80|477.12|998.45|14.76|72.07|623.88| +||1510||1912926||21944|2||1510||167.88||171.23||||103.65|2.12|| +|47975|2893||855708|6891|||32|1511||191.52|3.83|195.35||112.80|||19.45|| +2451729|50646|13991|39977|1025156|3935|18314|7|29|1512|70|4690.00|187.60|4877.60|32.56|415.10|281.40|4011.82|396.78|635.26| +2451415|43620|5156|85742|1393011|4549|14887|8|10|1513|38|3522.22|176.11|3698.33|35.75|342.38|1866.77|480.08|1175.37|554.24| +2451362|40091|13669|85742|1093123|3167|19211|8|29|1513|31|2231.38|89.25|2320.63|80.44|2082.58|1695.84|155.30|380.24|2252.27| +2451257|34467|10285|30178|1064464|2353|21889|10|22|1520|77|3429.58|274.36|3703.94|28.58|1854.16|2572.18|814.53|42.87|2157.10| +2451243|31186|10690|30178|855814|3466|37165|8|9|1520|13|78.65|6.29|84.94|46.30|84.50|22.02|19.82|36.81|137.09| +2451294|35401|565|6503|644558|1366|47021|1|14|1520|42|916.44|0.00|916.44|41.90|209.16|394.06|386.56|135.82|251.06| +2452027|33238|15121|81888|458210|922|387|1|21|1522|18|793.98|39.69|833.67|31.09|134.10|389.05|32.39|372.54|204.88| +2452200|52565|12685|54364|47650|4936|10222|10|4|1523|19|201.97|10.09|212.06|12.41|91.58|145.41|46.37|10.19|114.08| +2451611|55725|11335|7304|1622617|1930|31243|7|34|1524|33|0.00|0.00|0.00|12.90|269.94|0.00|0.00|0.00|282.84| +2452259|41817|8605|10399|247114|6229|8944|1|4|1525|47|1831.12|128.17|1959.29|14.82|501.96|842.31|425.18|563.63|644.95| +2451239|44471|16510|1583|1800623|4474|18325|2|24|1526|11|975.15|48.75|1023.90|49.35|333.19|214.53|182.54|578.08|431.29| +2451275|44137|5053|19541|335970|941|21912|2|2|1527|25|367.75|25.74|393.49|93.10|24.50|308.91|5.29|53.55|143.34| +2451223|49777|7336|19541|1502214|3525|37283|2|20|1527|3|2.94|0.20|3.14|6.03|1.29|2.44|0.29|0.21|7.52| +2451186|58665|7435|19541|1460989|1846|3903|2|17|1527|35|185.85|13.00|198.85|96.27|202.65|85.49|61.21|39.15|311.92| +2451930|44586|15241|68335|1086358|1561|26717|1|11|1528|8|6.96|0.27|7.23|0.95|1.36|1.74|4.28|0.94|2.58| +2451922|36649|7982|68335|1761334|5449|45611|7|11|1528|7|264.32|10.57|274.89|80.03|160.86|15.85|223.62|24.85|251.46| +2452106|43128|4779|65565|828047|3521|34148|7|16|1529|31|2088.78|0.00|2088.78|46.84|696.26|1232.38|359.68|496.72|743.10| +2452109|53171|1677|65565|625799|4085|24371|2|30|1529|14|30.66|1.22|31.88|83.89|143.08|13.79|7.59|9.28|228.19| +2452047|60791|9755|47054|576294|3934|31974|2|4|1533|55|1202.85|0.00|1202.85|9.98|364.65|372.88|697.17|132.80|374.63| +2451878|33729|16771|7156|795352|2368|47337|10|20|1533|16|94.72|3.78|98.50|97.61|644.64|23.68|23.44|47.60|746.03| +2452664|32146|13836|23880|1909704|3881|30905|10|4|1534|6|90.72|8.16|98.88|86.75|16.02|37.19|46.03|7.50|110.93| +2451086|53086|9436|70122|714300|1678|49409|8|28|1536|60|2137.20|21.37|2158.57|95.25|1662.60|876.25|176.53|1084.42|1779.22| +2452175|32177|13331|64935|857064|4698|38285|8|33|1538|1|0.65|0.01|0.66|30.06|4.02|0.36|0.25|0.04|34.09| +2452256|60363|16551|64935|711453|426|11846|1|32|1538|14|531.72|31.90|563.62|82.07|686.14|90.39|428.09|13.24|800.11| +2450990|55813|38|42121|36550|4362|13374|2|3|1539|14|47.60|2.85|50.45|89.82|6.58|37.60|1.90|8.10|99.25| +2451489|59909|4105|55308|1630143|3593|29843|2|35|1540|36|2385.72|214.71|2600.43|39.42|0.00|190.85|899.89|1294.98|254.13| +2451484||6904||||42439|||1540|35|94.85|||82.35|52.50|0.94|31.92|61.99|141.48| +2452135|36226|5987|77263|765570|68|2566|10|32|1541|50|2569.00|154.14|2723.14|34.60|359.50|1387.26|307.25|874.49|548.24| +2451229|38119|13873|98906|1053847|6913|16732|7|9|1542|2|54.22|1.08|55.30|10.58|56.04|22.23|22.07|9.92|67.70| +2450953|34423|3448|48199|710692|2122|13152|10|17|1544|10|664.50|33.22|697.72|54.77|268.10|631.27|8.63|24.60|356.09| +2451107|55117|5104|48199|114816|236|39609|8|19|1544|7|62.72|0.00|62.72|10.05|0.00|23.20|27.26|12.26|10.05| +2451594|29618|10234|33078|82174|3771|14968|7|19|1545|58|139.20|4.17|143.37|95.42|905.38|48.72|7.23|83.25|1004.97| +2452217|57486|953|86172|1388598|6473|26701|8|32|1546|16|130.08|1.30|131.38|76.42|91.04|75.44|1.09|53.55|168.76| +2452070|31929|2601|48121|535915|6462|9809|7|18|1546|30|374.70|29.97|404.67|86.74|1530.00|322.24|28.85|23.61|1646.71| +2452114|29247|6035|73981|1103260|779|38251|1|22|1547|1|61.30|3.67|64.97|26.93|33.14|51.49|3.53|6.28|63.74| +2452139|48647|1487|73981|1298032|577|32722|1|32|1547|3|84.69|0.00|84.69|44.93|0.00|53.35|22.87|8.47|44.93| +2451979|58355|14228|31081|1781308|632|9509|4|10|1548|84|3700.20|185.01|3885.21|72.29|1740.48|296.01|2519.10|885.09|1997.78| +2451975|48880|8017|31081|375926|2153|29133|4|5|1548|60|2084.40|41.68|2126.08|16.49|2084.40|1563.30|307.44|213.66|2142.57| +2452352|47690|2535|48192|1120058|3924|12990|7|22|1549|40|142.00|4.26|146.26|62.15|142.00|75.26|36.70|30.04|208.41| +2451788|49145|10112|71960|1639457|3382|16060|7|23|1550|19|26.98|1.34|28.32|27.06|2.66|11.60|14.14|1.24|31.06| +2452323||281|||||10||1551||4221.07|337.68|4558.75||556.48|1646.21||1107.19|965.22| +2451297|48870|830|35475|73824|6399|23327|2|25|1552|18|1115.82|55.79|1171.61|65.62|30.42|1026.55|54.45|34.82|151.83| +2451375||9344|35475|1759428|680|||12|1552|7||0.68||||13.37|||| +2451378|55036|12013|35475|923573|6322|39071|1|6|1552|8|544.40|10.88|555.28|32.07|87.04|76.21|70.22|397.97|129.99| +2450894|38597|10310|83280|1466004|5881|7358|10|19|1554|40|1264.00|12.64|1276.64|47.62|48.40|1188.16|18.20|57.64|108.66| +2452433|40690|1857|4086|1185047|1463|41926|2|25|1555|7|205.66|18.50|224.16|7.12|38.08|37.01|158.53|10.12|63.70| +2452441|30836|8939|4086|1473832|6882|11132|10|4|1555|18|1032.66|41.30|1073.96|39.11|774.36|867.43|120.61|44.62|854.77| +2451499|38686|9608|51846|1732912|3878|2044|7|16|1558|26|612.56|6.12|618.68|33.86|168.22|318.53|88.20|205.83|208.20| +2451466|29590|439|1413|765532|6828|18124|7|8|1559|2|15.94|0.00|15.94|94.95|15.42|1.75|0.28|13.91|110.37| +2451509|32446|1663|1413|129933|4999|29218|10|35|1559|44|777.92|46.67|824.59|20.68|491.04|412.29|51.18|314.45|558.39| +2451231|56481|1144|26693|1749603|2113|35470|2|10|1561|34|156.40|7.82|164.22|27.57|654.16|151.70|3.38|1.32|689.55| +2451887|33047|9241|22231|149116|4765|12414|10|24|1563|67|332.32|9.96|342.28|41.20|426.12|312.38|12.16|7.78|477.28| +2452063|44093|9389|22231|1096979|5030|4234|1|14|1563|14|1309.98|26.19|1336.17|42.23|503.86|877.68|194.53|237.77|572.28| +2452389|54422|15975|66501|1581040|4343|34846|4|5|1566|59|3145.88|0.00|3145.88|10.41|1607.75|125.83|2416.04|604.01|1618.16| +2452553|48560|10869|59449|390504|2393|27882|4|33|1568|2|15.28|0.30|15.58|37.69|10.48|6.57|6.44|2.27|48.47| +2452649|58668|7147|59449|127855|6921|47634|2|33|1568|8|39.20|0.00|39.20|70.46|188.40|10.19|18.27|10.74|258.86| +|40782|11491||222342|||7||1569||||||||52.45||335.50| +2452357|51331|11331|21854|708464|653|7289|8|9|1569|20|110.00|8.80|118.80|92.07|880.00|63.80|39.27|6.93|980.87| +2451968|57621|16483|53417|1678430|5861|43292|1|11|1571|84|870.24|78.32|948.56|17.08|416.64|574.35|26.63|269.26|512.04| +2451686|59323|15676|83015|426436|5504|6925|2|12|1572|15|197.85|17.80|215.65|84.26|1038.60|23.74|120.13|53.98|1140.66| +2451572|50950|14284|48034|1776520|2922|13771|4|25|1572|3|2.94|0.17|3.11|1.53|0.00|1.23|0.01|1.70|1.70| +2451634|58567|10936|48034|413243|788|38733|1|10|1572|62|5313.40|53.13|5366.53|63.57|1690.74|3613.11|459.07|1241.22|1807.44| +2452800|53670|12453|9126|1798897|5741|36138|10|21|1575|41|4625.21|92.50|4717.71|40.12|1746.19|3838.92|652.62|133.67|1878.81| +2451282|43489|7792|13081|1267042|6488|43534|1|28|1576|4|282.40|19.76|302.16|12.56|100.20|175.08|44.00|63.32|132.52| +2451309|44285|14146|7144|1513628|6959|39090|7|15|1576|23|5.06|0.25|5.31|80.36|82.11|2.37|1.21|1.48|162.72| +2452666|36861|16320|97209|1530544|6427|46263|2|27|1577|31|215.76|6.47|222.23|63.09|16.12|202.81|5.56|7.39|85.68| +2452693|56193|3081|97209|903916|2268|1067|1|33|1577|13|40.43|1.61|42.04|99.52|90.48|33.15|5.75|1.53|191.61| +2450947|41106|7192|81962|927950|3185|1173|8|6|1578|11|115.61|2.31|117.92|78.98|395.01|83.23|26.55|5.83|476.30| +2451534|49026|16663|37703|318814|4158|34349|8|22|1579|19|1169.64|11.69|1181.33|54.91|668.23|1005.89|9.82|153.93|734.83| +2451530|41838|11431|4565|1792252|1186|28073|4|29|1579|12|998.16|89.83|1087.99|70.76|351.12|0.00|319.41|678.75|511.71| +2452437|41183|16841|51057|580822|4987|41552|7|10|1580|81|2256.66|45.13|2301.79|19.23|2793.69|1511.96|685.12|59.58|2858.05| +2452437|50264|2437|65764|809568|6172|5576|10|29|1580|28|145.60|8.73|154.33|96.37|680.12|72.80|32.03|40.77|785.22| +2452634|30734|7171|68388|851361|5227|22646|8|4|1581|65|1110.85|0.00|1110.85|37.91|219.05|833.13|194.40|83.32|256.96| +2452428|54120|3918|11095|1355061|6264|15867|10|6|1582|7|3.43|0.06|3.49|10.89|3.36|2.09|0.87|0.47|14.31| +2452473|40862|12279|11095|484721|2352|27630|4|29|1582|8|54.00|2.70|56.70|77.43|216.00|29.70|23.32|0.98|296.13| +2451016|57915|6886|10898|626573|6004|32235|10|7|1583|69|2377.74|142.66|2520.40|89.39|848.70|1925.96|203.30|248.48|1080.75| +2451035|35822|13393|10898|981251|4184|8717|7|13|1583|25|1014.50|30.43|1044.93|13.32|94.25|892.76|48.69|73.05|138.00| +2450951|29736|400|10898|1573444|5437|42630|8|21|1583|4|26.44|2.37|28.81|42.61|35.24|15.59|9.22|1.63|80.22| +2451388|39326|17030|80976|1052373|2802|12831|10|19|1584|42|14.28|0.85|15.13|5.65|2.52|4.71|6.31|3.26|9.02| +2451446|44446|2029|80976|297542|5540|44385|2|30|1584|24|486.00|34.02|520.02|37.86|277.68|471.42|6.26|8.32|349.56| +2451418|35810|439|56284|1780229|4621|41766|10|18|1584|4|69.92|1.39|71.31|14.44|106.72|15.38|23.99|30.55|122.55| +2452313|58038|921|37239|840588|5890|19639|1|30|1586|11|247.50|4.95|252.45|57.96|60.28|131.17|98.88|17.45|123.19| +2451738|55433|5996|55811|250819|605|28391|4|33|1587|8|163.92|4.91|168.83|20.54|538.64|54.09|26.35|83.48|564.09| +2451670|31435|5119|55811|1816311|6250|34189|2|32|1587|1|0.71|0.00|0.71|15.84|18.54|0.27|0.06|0.38|34.38| +2451902|60002|16573|469|966133|4244|12308|2|16|1588|8|780.96|7.80|788.76|74.78|57.52|562.29|135.57|83.10|140.10| +2451904|51328|1712|1227||547|19807|1|17|1588|15|2290.20|||69.25|763.35|2244.39|28.40|17.41|| +2452725|41072|5679|25070|477002|197|18239|2|9|1590|6|44.64|1.33|45.97|59.00|20.16|38.39|5.87|0.38|80.49| +2452784|55065|13263|35665|133184|5008|785|2|6|1591|10|210.10|4.20|214.30|49.47|32.30|42.02|87.40|80.68|85.97| +2452268|32074|2339|19435|41412|711|12457|7|7|1592|15|150.30|10.52|160.82|39.49|50.10|84.16|60.84|5.30|100.11| +2452143|51516|165|19435|1336932|2359|26186|7|33|1592|1|1.08|0.08|1.16|81.78|1.45|0.24|0.21|0.63|83.31| +2452336|57106|7176|30937|1718596|4248|20747|1|28|1593|26|465.40|0.00|465.40|65.62|337.74|372.32|1.86|91.22|403.36| +2451222|40481|10633|13219|188801|6081|11361|2|18|1594|6|108.36|8.66|117.02|53.63|178.50|72.60|7.50|28.26|240.79| +||2149||1233114|7085||1||1597|1|5.03||5.23||1.45|||0.35|19.94| +2451124|44421|2522|89134|1554898|5968|19438|10|23|1597|16|1996.32|179.66|2175.98|67.47|370.40|1636.98|208.41|150.93|617.53| +2452236|44513|7403|25283|1716892|2455|16818|7|16|1599|25|6.50|0.00|6.50|59.91|47.00|5.72|0.08|0.70|106.91| +2452173|47005|5925|17919|179703|1563|31470|1|14|1600|59|2509.27|50.18|2559.45|81.08|3111.07|1405.19|563.08|541.00|3242.33| +2451617|44172|7981|3155|360625|7142|29659|2|20|1601|28|26.88|0.80|27.68|23.79|76.44|7.79|10.30|8.79|101.03| +|50108|13208|97058|||1321|7|23|1603|7|242.27|4.84|247.11|52.93|104.93||||| +2451966|58347|3176|78048|1036281|1382|49636|1|23|1605|9|251.46|5.02|256.48|4.45|195.57|223.79|15.49|12.18|205.04| +2451528|40179|6304|47664|1326639|3618|47844|8|22|1606|42|484.68|24.23|508.91|83.75|380.94|174.48|220.24|89.96|488.92| +2451522|46415|15098|47664|198481|5810|29586|8|16|1606|18|248.04|19.84|267.88|13.79|635.76|4.96|21.87|221.21|669.39| +2451787|56510|6817|92501|1557912|1642|19019|4|29|1607|1|20.83|0.20|21.03|93.42|21.78|14.58|5.25|1.00|115.40| +2451703|39876|15727|52516|282372|6275|2578|1|24|1609|14|144.62|4.33|148.95|98.57|101.22|107.01|20.68|16.93|204.12| +2451636|44227|3853|49113|296470|7059|15248|4|29|1609|5|153.45|10.74|164.19|24.54|175.90|67.51|37.81|48.13|211.18| +2451663|33095|9491|49113|1651043|396|10996|1|32|1609|5|57.15|1.71|58.86|22.50|46.15|54.86|1.92|0.37|70.36| +2451962|51338|3721|52849|1015746|6502|44058|2|15|1610|33|546.15|10.92|557.07|46.14|810.15|32.76|246.42|266.97|867.21| +2452012|37364|16609|52849|12829|233|43637|4|5|1610|16|83.68|6.69|90.37|67.60|32.32|3.34|33.74|46.60|106.61| +2452301|37803|333|49401|1401733|5265|15197|8|6|1611|62|4454.08|178.16|4632.24|78.98|3006.38|579.03|2247.52|1627.53|3263.52| +2450992|58247|5140|35146|1401980|490|45400|2|2|1612|6|74.04|0.74|74.78|29.06|324.12|71.07|0.35|2.62|353.92| +2450997|51566|9091|41016|1879798|3874|45997|10|1|1613|59|1249.03|87.43|1336.46|80.77|285.56|1111.63|34.35|103.05|453.76| +2452094|39495|8215|15198|1661464|3377|17643|7|32|1614|1|31.91|0.00|31.91|65.50|6.75|2.87|7.26|21.78|72.25| +2452228|32388|4427|68236|1599149|6236|8123|7|14|1614|19|237.12|21.34|258.46|9.03|0.00|184.95|51.12|1.05|30.37| +2452245|36806|7319|68236|1561341|4315|29289|1|5|1614|34|2491.52|124.57|2616.09|99.50|260.44|398.64|1778.94|313.94|484.51| +|49177|13142|41094|288649|293||8|32|1615|65|||4522.47||||2318.31|1545.55|| +2451838|40890|302|41094|902063|2295|4527|1|1|1615|43|1965.96|98.29|2064.25|91.46|1529.08|235.91|1401.34|328.71|1718.83| +2451649|40380|7876|14798|1379625|3530|21788|4|3|1616|21|955.50|0.00|955.50|3.40|260.61|496.86|334.80|123.84|264.01| +2451785|46383|16841|90597|1867828|5778|13728|2|6|1617|31|105.71|5.28|110.99|23.74|1198.77|14.79|44.55|46.37|1227.79| +2451846|38626|17543|71934|855638|1312|11739|7|25|1620|24|850.32|0.00|850.32|42.00|1275.60|212.58|433.66|204.08|1317.60| +2451879|53827|14641|40975|999674|1926|38978|4|4|1620|1|25.08|2.25|27.33|80.91|3.73|8.52|15.23|1.33|86.89| +2451884|55050|5240|||3026||10|30|1620|6|58.14||63.37|93.43||54.07||0.62|105.02| +2452278|43926|10399|98224|749552|2866|17427|1|17|1622|74|383.32|34.49|417.81|0.54|1491.84|279.82|24.84|78.66|1526.87| +2452320|34545|3745|98224|448551|6439|14327|7|31|1622|22|959.86|67.19|1027.05|91.86|395.12|76.78|88.30|794.78|554.17| +2451820|34301|15559|33223|305548|638|41728|2|11|1623|21|1189.23|83.24|1272.47|82.05|507.36|558.93|50.42|579.88|672.65| +2451270|60807|1604|57241|1221185|2729|4320|7|20|1625|25|127.75|0.00|127.75|92.68|14.00|80.48|15.12|32.15|106.68| +2451135|44691|6334|68242|719037|2619|32554|4|34|1625|25|2353.00|47.06|2400.06|58.17|865.50|1011.79|509.65|831.56|970.73| +2450998|49275|10580|78473|1749335|6979|19070|10|35|1627|6|280.74|2.80|283.54|89.04|71.04|154.40|113.70|12.64|162.88| +2452625|34873|6067|65221|1916463|1939|38568|1|18|1630|17|117.13|8.19|125.32|65.04|396.44|117.13|0.00|0.00|469.67| +2452696|54302|13585|65221|1577809|4699|33989|10|2|1630|2|158.16|12.65|170.81|41.44|51.88|12.65|56.74|88.77|105.97| +2452648|29912|9667|65221|1089714|4621|35081|1|18|1630|4|131.96|6.59|138.55|25.77|86.04|44.86|47.90|39.20|118.40| +2451572||16630|77780||5006|36623|7|7|1632|67||0.00||83.79||443.90|100.61|2414.88|3043.18| +2451396|54500|1460|63571|1362568|5914|41213|2|13|1633|23|184.92|7.39|192.31|22.25|167.44|11.09|159.92|13.91|197.08| +2451355|47771|14779|63571|377308|5070|10832|7|16|1633|6|22.32|0.22|22.54|72.69|19.32|16.96|3.53|1.83|92.23| +2451292|58524|5164|30062|429974|3839|2151|4|20|1633|74|5871.16|234.84|6106.00|52.84|3394.38|1643.92|2789.97|1437.27|3682.06| +2452362|35009|9438|80198|271691|431|20724|7|32|1634|5|403.85|8.07|411.92|35.35|169.35|76.73|3.27|323.85|212.77| +2452319|48132|12069|4788|1434349|4973|9997|10|3|1634|9|57.51|2.87|60.38|41.90|13.41|39.10|4.97|13.44|58.18| +2451018|49787|9196|10238|384825|5482|48186|1|30|1635|6|164.58|3.29|167.87|95.53|274.32|130.01|12.79|21.78|373.14| +2451892|30778|313|19122|1728168|4364|13313|10|26|1636|8|32.64|1.30|33.94|37.95|33.28|31.98|0.28|0.38|72.53| +2452048|37452|4837|86919|953487|4055|40623|7|32|1636|16|535.36|37.47|572.83|9.73|520.00|26.76|381.45|127.15|567.20| +2451078|29415|11840|63973|1283175|577|21979|10|10|1637|13|39.13|2.73|41.86|13.06|127.27|26.60|9.14|3.39|143.06| +2452648|45180|4137|87919|1789496|6002|30461|8|26|1639|67|1005.67|50.28|1055.95|56.84|797.30|603.40|164.93|237.34|904.42| +2452641|37161|9099|87919|575419|406|45793|2|22|1639|26|2128.62|127.71|2256.33|87.67|369.98|1596.46|79.82|452.34|585.36| +2451946|50544|3853|98047|311917|4154|44344|4|18|1640|1|59.26|2.37|61.63|1.54|33.24|43.25|2.56|13.45|37.15| +||14801|98047|477588||37806||9|1640|2|108.76|0.00||||7.61|||| +2451870|46311|15347|98047|1510178|2039|43947|7|4|1640|15|300.75|15.03|315.78|57.04|195.00|258.64|9.68|32.43|267.07| +2451949|38301|6338|98047|1082982|2289|24796|2|27|1640|48|2067.84|0.00|2067.84|85.56|2787.36|558.31|1388.76|120.77|2872.92| +2452550|55231|6879|76535|879229|5123|45417|4|25|1641|1|19.63|0.19|19.82|22.82|8.81|17.86|0.24|1.53|31.82| +2452565|40280|7759|38190|1856845|3718|37344|8|28|1641|22|2851.42|228.11|3079.53|3.31|1525.04|2195.59|380.38|275.45|1756.46| +2452713|43834|5703|38190|616855|1115|4622|10|2|1641|2|180.18|0.00|180.18|6.24|42.66|63.06|73.78|43.34|48.90| +2451664|37647|14791|94877|1599065|4672|7479|10|8|1642|41|1892.97|113.57|2006.54|92.74|516.19|1817.25|45.43|30.29|722.50| +2451943|42636|5372|71086|1679582|2925|19068|1|34|1643|30|1193.40|59.67|1253.07|46.23|668.40|584.76|596.46|12.18|774.30| +2451996|51464|6781|44713|830498|3196|43094|7|30|1643|71|1109.73|66.58|1176.31|14.60|1240.37|310.72|511.36|287.65|1321.55| +2452249||5301||||49656|1|17|1644|2||6.87||||63.25|30.45|43.82|106.34| +2452158|42905|6627|85550|1646514|4278|32042|4|8|1644|39|1730.82|155.77|1886.59|11.19|56.55|190.39|154.04|1386.39|223.51| +2452032|53597|12095|91694|108293|5550|49207|7|1|1645|12|474.00|42.66|516.66|34.32|253.56|118.50|355.50|0.00|330.54| +2451987|53090|17023|14214|111787|3407|42180|7|18|1646|45|820.80|73.87|894.67|40.48|1733.40|467.85|190.59|162.36|1847.75| +2451929|56722|9065|26102|1825244|331|34571|7|25|1646|83|3422.92|68.45|3491.37|4.15|213.31|3217.54|121.17|84.21|285.91| +2452155|32470|11927|77700|299800|1421|48642|8|27|1648|22|357.28|21.43|378.71|82.99|83.82|275.10|26.29|55.89|188.24| +2451034|60971|7526|61889|607322|3262|19602|10|9|1649|41|1153.33|0.00|1153.33|83.12|1537.91|219.13|130.78|803.42|1621.03| +2450903|53630|13448|61889|622034|6761|27088|1|7|1649|2|8.42|0.00|8.42|94.29|6.08|2.94|1.80|3.68|100.37| +2450983|38310|7303|61889|435378|861|6344|8|7|1649|41|566.21|0.00|566.21|23.34|1698.63|11.32|11.09|543.80|1721.97| +2451687|34171|6314|33124|1802460|2196|21979|4|1|1650|13|930.28|27.90|958.18|41.96|690.69|902.37|16.74|11.17|760.55| +2451409|53282|7531|81204|986789|6428|1267|7|15|1652|1|11.09|0.88|11.97|19.48|20.34|3.32|3.57|4.20|40.70| +2451529|43150|6064|81204|559568|3359|37950|1|2|1652|14|490.56|39.24|529.80|83.94|266.56|166.79|220.16|103.61|389.74| +2451364|46051|3116|81204|1140695|5221|11347|4|24|1652|49|1025.57|10.25|1035.82|20.17|1282.33|246.13|132.50|646.94|1312.75| +2451015|57540|8749|76886|1788315|824|18933|7|21|1653|31|617.21|0.00|617.21|25.19|1429.41|413.53|85.54|118.14|1454.60| +2451859|34841|3446|87080|1127894|237|27380|4|18|1656|3|27.09|0.00|27.09|69.32|11.52|8.39|0.18|18.52|80.84| +2451580|31734|5104|11333|800791|4620|3696|8|6|1657||606.69|30.33||27.05|214.92|570.28|15.29||272.30| +2451274|40490|100|39714|1697069|4046|19776|1|3|1660|8|119.36|3.58|122.94|93.12|19.84|65.64|3.76|49.96|116.54| +2452592|44234|12309|98807|656671|2088|42352|2|20|1661|27|271.89|0.00|271.89|44.54|271.89|271.89|0.00|0.00|316.43| +2452644|54405|2325|20327|268179|3231|48236|2|35|1661|78|731.64|29.26|760.90|15.31|1294.02|212.17|332.46|187.01|1338.59| +2451602|53552|2498|51854|213552|5673|5347|2|32|1663|5|142.35|4.27|146.62|26.58|74.70|79.71|42.59|20.05|105.55| +2451634|45544|6128|51854|343026|2123|48820|8|29|1663|1|4.01|0.28|4.29|79.97|0.38|1.76|0.90|1.35|80.63| +||13369|29430|304099||40561||15|1664||581.70|0.00||63.01|296.10|395.55||63.30|359.11| +2451290|47679|12508|29430|358782|2174|13966|7|30|1664|16|1361.92|95.33|1457.25|26.25|82.40|1361.92|0.00|0.00|203.98| +2451220|36437|13226|29430|429584|563|38742|1|15|1664|14|430.92|4.30|435.22|71.93|206.08|176.67|40.68|213.57|282.31| +2451166|29699|12700|29430|168548|6060|37172|7|4|1664|35|3044.30|213.10|3257.40|76.59|1043.70|2831.19|151.30|61.81|1333.39| +2451530|42308|8107|75783|1803212|2042|18472|2|22|1665|51|1374.45|96.21|1470.66|13.44|992.46|302.37|707.57|364.51|1102.11| +||3607|||3071|24715||24|1667|28|1662.92||1795.95|23.55||1031.01||240.13|| +2450994|35395|12266|91387|1539969|874|8979|10|14|1668|9|53.73|3.76|57.49|55.51|255.33|8.59|37.46|7.68|314.60| +2451192|58469|3607|21620|1863164|193|34422|4|10|1669|3|295.95|11.83|307.78|78.88|60.60|177.57|66.29|52.09|151.31| +2452747|47895|5799|49545|323903|1461|22266|7|2|1670|65|462.15|13.86|476.01|20.08|1489.15|439.04|20.56|2.55|1523.09| +2452617|37691|16422|84125|131578|6331|25084|7|2|1670|12|1530.60|76.53|1607.13|59.81|409.56|1255.09|112.95|162.56|545.90| +2452708|37654|1116|1750|1384040|5883|35773|4|1|1670|3|3.33|0.00|3.33|73.00|2.85|0.79|0.05|2.49|75.85| +2452701|28930|17298|98646|985735|6424|23913|2|26|1670|21|421.68|0.00|421.68|83.50|71.19|151.80|118.74|151.14|154.69| +2452301|61016|9931|57871|1632611|1031|26204|2|27|1671|89|8755.82|350.23|9106.05|60.87|5068.55|7354.88|406.27|994.67|5479.65| +|59306|11305|96088|628178|||4|31|1671|44|||2921.78||63.36|1474.54||979.76|282.99| +2452065|58212|10207|31259|519675|3374|7473|7|1|1672|22|309.32|24.74|334.06|2.08|330.00|52.58|112.96|143.78|356.82| +2451132|44034|2332|26871|409058|2445|38632|4|31|1673|14|485.94|19.43|505.37|29.61|161.98|170.07|28.42|287.45|211.02| +2451104|46777|17056|4518|1811089|4860|43384|8|31|1673|4|73.36|1.46|74.82|47.91|57.40|24.94|27.59|20.83|106.77| +2452125|39089|8911|45105|1919140|3073|4606|1|32|1675|9|292.68|20.48|313.16|28.67|62.64|248.77|23.71|20.20|111.79| +2451991|45507|17953|45105|981878|7123|28368|10|1|1675|25|698.50|34.92|733.42|80.29|622.75|481.96|75.78|140.76|737.96| +2452558|58977|13596|94757|1887570|3583|47949|10|25|1676|32|902.40|72.19|974.59|67.59|529.28|884.35|4.15|13.90|669.06| +|55720|879|65507||5245|25488|7||1676|||||96.94|||5.39||| +2452576|40760|3732|65507|571411|2808|36958|1|7|1676|18|2106.54|0.00|2106.54|85.83|59.22|189.58|1859.45|57.51|145.05| +2451581|50001|9436|16099|1882126|518|36910|1|32|1678|22|625.68|31.28|656.96|82.79|28.60|193.96|358.32|73.40|142.67| +2451672|38275|7654|16099|418193|3731|14469|1|2|1678|14|112.56|3.37|115.93|16.68|88.20|92.29|1.41|18.86|108.25| +2452371|58591|2149|1087|459874|6860|34246|10|30|1679|24|1295.52|38.86|1334.38|36.48|684.72|181.37|367.66|746.49|760.06| +2452246|56956|6505|1087|1597317|763|13009|4|16|1679|2|29.02|0.58|29.60|17.13|0.00|5.51|15.75|7.76|17.71| +2452270|60989|8585|1087|316925|4579|33433|2|13|1679|56|4234.72|381.12|4615.84|44.50|264.32|550.51|3573.68|110.53|689.94| +2452322|48983|4019|1087|835864|3888|14665|4|23|1679|43|552.55|11.05|563.60|20.48|491.06|497.29|53.60|1.66|522.59| +2452235|40336|9223|1087|464096|5432|13568|7|13|1679|75|3179.25|95.37|3274.62|90.95|3382.50|2098.30|724.23|356.72|3568.82| +2451601|46395|12350|90072|875806|3493|40780|2|21|1681|5|8.70|0.52|9.22|49.78|16.80|0.00|7.56|1.14|67.10| +2451906|47936|15553|82364|1483204|491|24931|1|18|1683|49|57.82|4.04|61.86|85.75|159.25|12.14|21.01|24.67|249.04| +2451992|57388|6391|35252|123867|2563|13659|8|1|1683|22|279.40|11.17|290.57|58.99|23.76|150.87|80.97|47.56|93.92| +||1001|35252|372118|||||1683||8219.16|493.14||86.98|1779.08||5656.42|425.76|| +2451953|42145|13957|49757|1697314|4071|4075|7|6|1683|23|303.83|18.22|322.05|99.04|134.55|118.49|7.41|177.93|251.81| +2452015|47359|2491|35252|202213|6120|28827|8|3|1683|26|644.02|45.08|689.10|96.22|299.00|135.24|111.93|396.85|440.30| +2451320|45380|9074|28046|405924|5015|40506|4|13|1684|15|951.15|57.06|1008.21|8.36|217.35|732.38|89.69|129.08|282.77| +2452364|54627|9831|20776|1652395|750|2752|1|22|1685|2|147.30|7.36|154.66|22.42|2.12|75.12|41.14|31.04|31.90| +2452432|55420|2717|20776|1546929|6035|13941|4|13|1685|73|2419.22|145.15|2564.37|10.63|985.50|2080.52|77.90|260.80|1141.28| +2452347|51081|7223|20776|681146|6542|31585|8|29|1685|1|49.26|3.44|52.70|13.30|26.43|43.84|0.48|4.94|43.17| +2452417|58972|13155|57105|1828675|4785|32495|4|5|1686|1|29.67|1.18|30.85|92.12|18.26|0.29|19.97|9.41|111.56| +2452376|35986|15469|57105|1468499|6333|42567|1|25|1686|20|204.20|10.21|214.41|78.72|703.80|108.22|35.51|60.47|792.73| +2451549|33123|4423|13530|476198|3320|44795|1|7|1687|63|847.98|16.95|864.93|37.50|1484.28|118.71|211.48|517.79|1538.73| +2451665|47027|7627|13530|1771132|1716|18700|7|33|1687|1|38.06|2.66|40.72|29.78|4.06|14.46|1.65|21.95|36.50| +2452308|40380|15679|18241|740271|878|34285|2|4|1689|43|1355.79|54.23|1410.02|70.47|1024.26|976.16|269.53|110.10|1148.96| +2452287|51295|17337|18241|796760|623|5639|7|30|1689|4|118.72|7.12|125.84|80.00|65.96|112.78|4.98|0.96|153.08| +2451670|30652|5776|21682|1251669|6969|21516|4|28|1692|65|7580.30|454.81|8035.11|5.56|3993.60|7580.30|0.00|0.00|4453.97| +2451575|52161|3403|21682|828789|3504|17019|7|34|1692|12|525.48|5.25|530.73|26.42|447.24|231.21|11.77|282.50|478.91| +2451865|52430|8047|86514|1670973|1272|5218|1|6|1693|73|8692.84|86.92|8779.76|74.18|995.72|5998.05|808.43|1886.36|1156.82| +2451939|34535|8485|86514|262237|2460|5883|7|19|1693|13|9.10|0.36|9.46|93.17|10.40|6.73|1.39|0.98|103.93| +2451606|42441|7552|59808|980087|4554|25868|4|4|1695|28|1101.52|88.12|1189.64|76.56|637.84|33.04|694.51|373.97|802.52| +2451627|52296|15542|59808|1267743|1898|15653|8|15|1695|12|368.16|33.13|401.29|49.36|191.40|47.86|118.51|201.79|273.89| +2452759|31587|14595|24820|884429|5844|33612|8|7|1696|7|50.40|3.52|53.92|64.99|8.40|34.27|3.38|12.75|76.91| +2451759|32788|5437|61339|1755754|3233|21112|2|33|1698|29|38.57|1.15|39.72|75.56|5.22|17.74|0.62|20.21|81.93| +2451771|59883|8099|16843|1058907|3739|25371|1|7|1700|35|2245.95|44.91|2290.86|53.14|1074.15|606.40|885.35|754.20|1172.20| +2451734|56323|7015|13229|1819639|3649|3659|1|29|1700|33|658.02|26.32|684.34|5.52|338.25|598.79|56.86|2.37|370.09| +||12169||||25789|||1701|79||||13.10|46.61|||32.09|63.99| +2452475|58930|17737|49824|781984|2022|29260|10|6|1701|14|1430.80|0.00|1430.80|81.20|426.16|1158.94|236.51|35.35|507.36| +2451656|52157|14102|7906|1918163|6251|15139|8|22|1702|2|44.98|0.89|45.87|34.06|12.84|2.69|29.18|13.11|47.79| +2451534|55670|10204|20436|923719|6555|43791|8|33|1702|8|275.52|22.04|297.56|6.97|60.48|184.59|15.45|75.48|89.49| +2452105|34430|3737|54292|502482|6225|32768|2|26|1704|22|192.94|0.00|192.94|87.10|265.32|3.85|153.16|35.93|352.42| +2451339|59236|14254|43672|137397|3465|11180|10|13|1705|2|112.06|8.96|121.02|47.95|2.38|85.16|1.61|25.29|59.29| +2452311|58541|3975|70550|1661182|2214|35646|2|13|1706|23|34.50|2.07|36.57|99.88|8.51|33.81|0.38|0.31|110.46| +2452303|56345|5619|17019|1426684|5821|26958|8|14|1706|3|27.42|1.37|28.79|59.93|23.49|1.09|4.47|21.86|84.79| +2452360|49453|14835|33075|521888|1548|3961|8|31|1706|2|59.18|3.55|62.73|88.11|50.48|13.01|4.61|41.56|142.14| +2452021|46667|13645|14907|1896694|1671|201|7|13|1707|8|176.00|8.80|184.80|36.51|37.52|15.84|140.94|19.22|82.83| +2451657|47117|10250|66551|943415|1879|26343|2|18|1708|21|467.46|14.02|481.48|63.99|293.16|177.63|197.08|92.75|371.17| +2451891|46552|17315|16028|853080|2195|23310|4|25|1710|1|21.16|1.48|22.64|22.30|16.83|8.25|11.61|1.30|40.61| +2451951|55512|8227|16028|175745|3330|14915|10|20|1710|32|1598.72|95.92|1694.64|38.53|191.04|383.69|692.56|522.47|325.49| +2451257|41938|16540|30082|1182854|2851|42095|2|19|1711|60|0.00|0.00|0.00|77.07|31.20|0.00|0.00|0.00|108.27| +2451267|34866|15715|30082|1822711|3037|39763|4|10|1711|3|419.28|29.34|448.62|43.76|128.25|100.62|286.79|31.87|201.35| +2451880|58657|3338|17494|1060485|6676|16067|4|21|1713|12|0.00|0.00|0.00|95.98|69.72|0.00|0.00|0.00|165.70| +2451498|60255|13864|38538|1193185|4116|7047|2|24|1716|60|425.40|8.50|433.90|59.86|459.60|174.41|70.27|180.72|527.96| +2451202|54745|11414|65139|1653293|2265|44837|8|11|1717|91|3299.66|197.97|3497.63|89.86|1988.35|2441.74|626.28|231.64|2276.18| +2451226|30981|11467|65139|906735|2281|17874|2|23|1717|44|1210.44|0.00|1210.44|59.71|1051.16|290.50|873.94|46.00|1110.87| +2451220|50527|15092|65139|650224|2000|25376|2|33|1717|50|523.00|10.46|533.46|10.63|65.00|88.91|425.40|8.69|86.09| +2452767|36190|6948|33387|625748|1303|23658|10|16|1721|4|408.04|24.48|432.52|0.98|4.80|53.04|24.85|330.15|30.26| +2452253|30081|15749|55069|1005037|134|24031|2|27|1722|36|4233.60|296.35|4529.95|44.27|665.28|804.38|3360.63|68.59|1005.90| +2452229|49710|17641|55069|1796292|134|4376|10|7|1722|58|846.22|8.46|854.68|24.02|1005.14|110.00|250.31|485.91|1037.62| +2452166|46984|3065|55069|776296|3109|24722|4|8|1722|42|1805.16|18.05|1823.21|31.60|1344.42|722.06|324.93|758.17|1394.07| +2452299|52817|12867|55069|1914810|6353|43543|7|21|1722|3|64.56|3.22|67.78|66.56|54.93|34.21|17.90|12.45|124.71| +2452229|44168|1735|81414|496119|4424|10329|2|27|1723|42|2199.54|197.95|2397.49|99.54|908.46|1231.74|503.25|464.55|1205.95| +2452784|38414|3591|64949|1327930|847|12164|2|10|1724|20|308.80|24.70|333.50|68.10|888.00|305.71|2.25|0.84|980.80| +2452644|38673|10224|30561|1574157|6106|28502|4|2|1725|17|1365.61|13.65|1379.26|66.39|557.77|40.96|874.26|450.39|637.81| +2452712|31571|11730|15140|1625491|5911|3578|1|32|1725|60|16.20|1.13|17.33|51.73|57.00|11.01|0.25|4.94|109.86| +2451443|57640|17528|49791|1035501|5323|47945|8|27|1726|9|866.97|78.02|944.99|66.02|516.06|502.84|309.51|54.62|660.10| +2452271|47702|3325|29910|1374579|2918|42504|2|24|1727|12|22.08|1.10|23.18|74.43|88.56|6.18|1.74|14.16|164.09| +2451460|31185|380|6527|765978|4628|47678|4|31|1730|29|576.23|17.28|593.51|25.89|908.86|46.09|424.11|106.03|952.03| +2451483|46814|15685|6527|623573|6187|46848|2|3|1730|66|5181.66|362.71|5544.37|58.92|293.04|2694.46|124.36|2362.84|714.67| +2451607||10438|||||7|24|1730||74.32|||67.71|||||157.87| +2452471|53566|17463|23624|1052211|5245|47032|4|16|1732|40|884.80|70.78|955.58|6.46|0.00|858.25|10.08|16.47|77.24| +2451972|45181|1249|88577|1090622|2181|7293|4|2|1733|87|3862.80|231.76|4094.56|53.20|4491.81|347.65|1757.57|1757.58|4776.77| +2450884|34957|6721|2118|1509607|1538|23116|1|35|1737|6|191.22|5.73|196.95|7.40|109.26|61.19|27.30|102.73|122.39| +2450867|55349|8881|2118|1033087|4315|19990|10|6|1737|11|49.28|1.97|51.25|78.14|72.16|16.26|14.85|18.17|152.27| +2452635|56352|5496|24995|1561912|4880|44695|1|6|1738|4|17.88|0.89|18.77|52.93|2.72|1.43|0.32|16.13|56.54| +2452645|36673|6607|24995|1443655|3577|20746|1|9|1738|3|92.34|6.46|98.80|47.55|46.17|41.55|5.58|45.21|100.18| +2452333|45900|5545|45844|1227476|3790|35351|1|1|1739|5|202.80|12.16|214.96|99.11|156.00|188.60|8.80|5.40|267.27| +2452185|49602|3829|45844|902153|6967|18181|10|17|1739|5|132.20|1.32|133.52|25.29|45.30|48.91|28.31|54.98|71.91| +||6785|3453|958230|3676||7|28|1740|||0.41|||||||| +2451904|40016|782|53216|83570|677|17031|2|15|1740|1|22.62|0.67|23.29|92.62|6.70|7.01|8.42|7.19|99.99| +2452747|29046|5601|1239|598806|7126|43093|2|27|1743|3|220.02|8.80|228.82|98.53|22.74|33.00|24.31|162.71|130.07| +2452744|38487|12967|26891|1571668|4445|31269|7|33|1743|19|1881.76|169.35|2051.11|3.99|77.52|677.43|1168.20|36.13|250.86| +2451852|50725|13544|51174|1414456|6659|42509|7|27|1744|11|267.85|16.07|283.92|52.85|133.87|238.38|16.20|13.27|202.79| +2451908|52594|16940|5193|1024376|5159|30342|1|18|1744|9|4.77|0.23|5.00|92.16|10.26|0.42|1.65|2.70|102.65| +2452801|37911|9531|28193|999471|2047|11470|4|21|1746|10|287.20|2.87|290.07|53.01|75.80|77.54|203.37|6.29|131.68| +2452683|53930|279|28193|957336|3630|23924|8|15|1746|31|234.98|11.74|246.72|34.02|4.96|79.89|88.40|66.69|50.72| +2452266|39674|4605|10951|1444041|5578|14268|7|30|1747|38|789.26|63.14|852.40|8.01|532.38|205.20|344.59|239.47|603.53| +2452189|59422|15319|17003|290110|2417|1124|7|6|1749|32|139.52|2.79|142.31|58.18|207.04|26.50|66.68|46.34|268.01| +2451070|41489|11020|98707|1452916|1453|49943|8|35|1751|35|2270.45|0.00|2270.45|69.93|577.15|1657.42|422.99|190.04|647.08| +2450992|34754|2020|98707|214774|6630|30933|8|27|1751|32|1300.48|26.00|1326.48|57.03|351.04|520.19|593.02|187.27|434.07| +2451066|34792|17233|98707|1135140|6802|36017|4|24|1751|29|2070.60|103.53|2174.13|11.93|276.08|973.18|10.97|1086.45|391.54| +2450979|59904|13561|98707|1032569|3960|20551|2|30|1751|31|796.70|39.83|836.53|36.20|258.85|63.73|278.52|454.45|334.88| +2452439|44230|15205|89671|1399839|6682|12372|7|17|1752|68|4392.12|43.92|4436.04|31.52|1115.20|3733.30|111.99|546.83|1190.64| +2451881|52562|1220|26849|1554972|6742|10865|10|23|1753|26|0.00|0.00|0.00|82.32|17.94|0.00|0.00|0.00|100.26| +2451917|56616|6542|55104|1903617|4930|23843|10|34|1753|24|0.00|0.00|0.00|74.41|896.64|0.00|0.00|0.00|971.05| +2452317|55973|270|53977|85663|6149|235|8|7|1754|2|80.68|3.22|83.90|10.20|21.82|24.20|6.77|49.71|35.24| +||14809|62143||5362||4|22|1755||202.00||214.12|89.35||54.54|28.01||192.27| +2452493|43170|14791|12699|602757|5178|48799|2|23|1758|5|184.95|0.00|184.95|55.84|18.85|86.92|10.78|87.25|74.69| +2452269|56498|5799|87253|597125|369|41566|1|15|1759|32|3780.80|37.80|3818.60|60.18|1781.76|3213.68|476.38|90.74|1879.74| +2452234|41015|17659|87253|730295|1151|42760|2|22|1759|12|167.28|13.38|180.66|28.33|476.40|110.40|49.48|7.40|518.11| +2452246|37300|11005|87253|325508|5795|44129|1|32|1759|32|1023.68|0.00|1023.68|54.18|496.96|286.63|589.64|147.41|551.14| +2452061|38986|4149|33927|747605|4399|17593|10|14|1760|27|1356.75|27.13|1383.88|69.77|51.30|936.15|374.33|46.27|148.20| +2452148|38623|2559|33927|1799428|4377|47773|1|3|1760|5|85.35|3.41|88.76|68.15|30.25|21.33|38.41|25.61|101.81| +2450943|51365|4336|4701|716730|2876|45422|7|12|1761|1|50.87|3.05|53.92|33.47|7.91|37.13|7.55|6.19|44.43| +2450944|30028|3004|4701|443447|5350|25653|7|11|1761|18|462.60|4.62|467.22|81.07|666.00|411.71|19.33|31.56|751.69| +2450994|34785|6724|4701||||7|31|1761||||||980.89|66.53||1421.16|1147.52| +2452609|46874|10335|45262|1443172|2139|15459|7|13|1762|61|7237.65|434.25|7671.90|67.91|2998.15|434.25|1904.95|4898.45|3500.31| +2451626|41879|1904|22262|1109173|4884|24148|8|5|1763|62|476.78|33.37|510.15|76.84|703.70|405.26|24.31|47.21|813.91| +2451630|37174|13037|75330|744181|1195|46562|7|8|1763|2|26.02|2.34|28.36|28.92|11.56|22.37|0.51|3.14|42.82| +2451659|55420|959|75330|240651|363|30428|8|25|1763|38|541.50|0.00|541.50|8.78|541.50|32.49|223.96|285.05|550.28| +2451620|49707|7778|40489|480539|2189|9039|1|14|1763|3|213.03|10.65|223.68|10.24|71.01|76.69|5.45|130.89|91.90| +2450971|35234|12304|60073|396784|4323|31462|1|28|1764|1|73.31|0.73|74.04|66.39|10.11|35.92|26.17|11.22|77.23| +2452262|51129|6547|81696|373515|5627|40139|1|33|1765|6|297.48|26.77|324.25|29.64|156.72|8.92|279.90|8.66|213.13| +2452174|60458|10207|81696|8188|5784|13780|7|32|1765|15|343.80|24.06|367.86|59.41|66.75|220.03|108.91|14.86|150.22| +2451885|30062|1229|23546|1385715|5921|41727|4|24|1766|5|17.05|0.68|17.73|65.60|1.20|14.32|0.02|2.71|67.48| +2451429|51050|9775|86216|1676748|2476|38251|10|29|1767|20|166.60|3.33|169.93|61.38|163.00|13.32|102.69|50.59|227.71| +2451296|42534|2833|86216|714634|4443|29899|2|17|1767|21|34.86|3.13|37.99|71.13|5.46|18.12|1.17|15.57|79.72| +2451940|58423|4555|42566|1609333|3060|39634|7|35|1768|26|554.32|44.34|598.66|65.98|47.32|543.23|4.54|6.55|157.64| +2452241|61187|16229|73117|77985|525|37112|4|9|1770|3|263.67|23.73|287.40|30.96|78.03|97.55|89.70|76.42|132.72| +2452228|47624|17069|73117|218264|4683|42349|7|12|1770|48|2964.00|237.12|3201.12|19.09|564.48|2756.52|145.23|62.25|820.69| +2452707|39494|7993|10556|339168|3696|32177|7|26|1773|10|123.90|3.71|127.61|38.49|607.30|110.27|5.17|8.46|649.50| +2451757||17885||582776||||25|1774|12||0.00|1132.20||||112.08||393.87| +2452280|31365|7131|45341|293680|635|26743|2|21|1775|9|254.25|15.25|269.50|90.33|271.26|160.17|3.76|90.32|376.84| +2452257|43582|2179|79964|1117053|1446|7439|4|18|1776|6|232.02|6.96|238.98|80.41|142.26|199.53|27.94|4.55|229.63| +2451210|47287|8497|49984|385079|5265|2604|1|27|1777|7|202.93|2.02|204.95|66.87|88.41|36.52|3.32|163.09|157.30| +2451252|41950|5300|49984|1693730|5737|41744|1|13|1777|79|1097.31|98.75|1196.06|98.80|142.99|416.97|176.88|503.46|340.54| +2451242|40352|7832|49984|946051|6291|3284|1|5|1777|12|420.48|0.00|420.48|65.12|303.12|353.20|63.91|3.37|368.24| +2452204|29890|4679|46648|748005|1809|41966|4|32|1778|35|2269.05|0.00|2269.05|82.55|518.35|1520.26|67.39|681.40|600.90| +2451596|35608|2329|40142|704741|3375|41209|4|2|1779|23|137.77|8.26|146.03|34.88|965.08|63.37|34.96|39.44|1008.22| +2451588|46543|3418|40142|1379854|2174|24894|2|16|1779|8|34.40|2.75|37.15|33.41|109.12|9.28|24.36|0.76|145.28| +2452217|59886|1277|62494|1394293|5292|42691|10|24|1780|66|660.66|46.24|706.90|51.86|155.10|502.10|28.54|130.02|253.20| +2451836|50984|6173|46712|674668|1024|17380|2|34|1785|32|2126.08|170.08|2296.16|98.90|106.24|1955.99|108.85|61.24|375.22| +2451566|39455|14503|43636|457372|4691|44010|7|30|1786|36|999.36|19.98|1019.34|61.52|895.32|869.44|42.87|87.05|976.82| +2451517|41876|15634|43636|1520442|6177|15112|7|25|1786|1|34.37|3.09|37.46|40.76|17.18|31.62|0.02|2.73|61.03| +2451590|50565|3848|43636|338560|1093|18426|4|7|1786|76|2109.00|105.45|2214.45|51.58|1285.16|485.07|1185.46|438.47|1442.19| +2451599|38819|7648|63782|1408941|4147|47706|1|14|1786|12|173.64|3.47|177.11|61.25|118.80|36.46|131.69|5.49|183.52| +2452507|33887|10494|91242|124201|6760|33766|8|31|1787|24|578.16|34.68|612.84|40.23|52.56|0.00|138.75|439.41|127.47| +2452401|57881|6897|3985|1090030|7098|2919|4|8|1787|18|40.14|2.80|42.94|67.10|28.08|19.26|9.39|11.49|97.98| +2452422|54068|5515|81060|1653122|6337|32399|7|16|1788|20|2164.80|151.53|2316.33|89.32|957.00|1645.24|394.86|124.70|1197.85| +2452258|51877|5429|99109|1799837|6791|37345|1|8|1789|21|1094.10|65.64|1159.74|8.15|1346.52|229.76|60.50|803.84|1420.31| +2451442|29105|892||1414780|||||1790||||231.10|||59.91||101.70|| +2451208|46061|16777|13658|1863385|7078|18324|1|10|1792|5|675.90|0.00|675.90|6.55|316.55|459.61|175.19|41.10|323.10| +2451188|61089|415|13658|624409|2908|26607|2|30|1792|27|3631.50|326.83|3958.33|84.80|868.32|1924.69|1228.90|477.91|1279.95| +2452726|54889|2845|27806|1100637|5175|16560|10|13|1794|3|327.84|9.83|337.67|81.58|49.53|124.57|81.30|121.97|140.94| +2452749|54005|5575|65999|493820|4394|15415|1|10|1794|21|1211.49|60.57|1272.06|84.32|776.58|242.29|242.30|726.90|921.47| +2452758|37612|4753|15517|1707761|805|43049|4|22|1794|49|958.93|19.17|978.10|29.44|113.19|402.75|66.74|489.44|161.80| +2451902|34471|11948|42494|1876895|4896|1697|10|9|1796|8|358.48|21.50|379.98|59.14|49.52|3.58|280.37|74.53|130.16| +2452265|38712|10395||1606437|2787||||1797|11||7.74|||40.37||||| +2452305|50080|10075|11463|801152|455|8158|1|19|1797|40|663.20|0.00|663.20|39.27|1769.20|172.43|363.16|127.61|1808.47| +2452259|41258|16395|11463|746644|6301|5|7|6|1797|69|234.60|11.73|246.33|84.36|528.54|117.30|92.66|24.64|624.63| +2451106|53089|12214|70461|11775|3564|8553|8|18|1798|6|306.60|6.13|312.73|52.52|155.28|168.63|27.59|110.38|213.93| +2451209|42048|14428|70461|926209|4821|19833|8|32|1798|22|787.16|55.10|842.26|37.57|364.76|15.74|38.57|732.85|457.43| +2452258|39687|11937|83008|256795|130|23865|8|7|1800|9|149.85|0.00|149.85|50.38|97.92|146.85|2.91|0.09|148.30| +2452343|41432|7005|83008|1520865|5815|30365|10|32|1800|23|2284.82|137.08|2421.90|4.84|1070.88|891.07|306.62|1087.13|1212.80| +2451612|47205|10831|29148|634791|2769|394|7|32|1803|17|935.68|84.21|1019.89|35.52|371.79|888.89|33.22|13.57|491.52| +2451691|47277|4912|85553|1182429|876|17323|8|30|1804|91|5135.13|51.35|5186.48|97.40|219.31|359.45|1289.43|3486.25|368.06| +2450969|44387|8338|69841|763675|5363|43451|8|17|1806|67|67.00|2.01|69.01|1.48|36.85|43.55|22.98|0.47|40.34| +2450942|48306|7216|69841|106007|5492|39777|4|19|1806|34|1298.12|38.94|1337.06|40.89|1057.74|700.98|453.82|143.32|1137.57| +2451169|29365|14870|86448|246756|4582|13655|2|19|1807|33|350.46|7.00|357.46|99.96|575.52|304.90|4.10|41.46|682.48| +2452556|41835|6444|84841|498079|1778|27877|7|10|1808|54|520.56|20.82|541.38|77.65|325.62|484.12|12.75|23.69|424.09| +2452559|55728|17569|84841|1593000|1429|658|10|3|1808|16|610.56|36.63|647.19|73.63|595.36|122.11|390.76|97.69|705.62| +2451476|45207|13765|65839|1153051|6330|41562|10|18|1809|11|349.25|17.46|366.71|10.38|582.23|220.02|11.63|117.60|610.07| +2452384|30056|480|18611|1160865|6824|42830|8|15|1810|67|613.72|49.09|662.81|1.36|1687.73|6.13|243.03|364.56|1738.18| +2452506|30217|13464|6518|1347547|4929|33958|4|9|1810|37|2987.38|0.00|2987.38|25.07|818.44|2688.64|176.25|122.49|843.51| +2452385||13963||1398840||||7|1810||1048.06|62.88||35.13|0.00||829.75|123.99|98.01| +2452447|55432|9939|18611|1372527|5374|40668|1|19|1810|34|1825.80|146.06|1971.86|59.75|1084.94|292.12|1487.66|46.02|1290.75| +2452505|38160|9891|18611|1697591|4394|2797|7|30|1810|78|420.42|37.83|458.25|99.85|209.82|277.47|114.36|28.59|347.50| +2452467|48074|15087|81292|983901|1710|9519|10|20|1811|20|84.00|3.36|87.36|64.70|350.00|42.00|39.06|2.94|418.06| +2452483|45245|7711|16657|75996|5735|21074|8|27|1811|64|4441.60|355.32|4796.92|76.37|3936.64|1865.47|2395.80|180.33|4368.33| +2452393|49001|7305|53430|942358|4886|30614|2|35|1811|57|523.83|5.23|529.06|22.32|253.65|345.72|87.27|90.84|281.20| +||13898|||5389||7||1812||322.00||350.98|12.95||302.68|||258.09| +2452450|58918|15577|42275|1521560|2798|41283|8|1|1813|37|362.97|3.62|366.59|67.65|1452.25|261.33|16.26|85.38|1523.52| +2452348|42116|16281|48469|1706118|1810|23055|10|2|1813|53|4531.50|226.57|4758.07|76.57|0.00|3036.10|807.51|687.89|303.14| +2451177|39142|1495|60700|642541|6431|13121|2|8|1814|14|989.80|29.69|1019.49|30.41|371.14|871.02|55.82|62.96|431.24| +2451745|46337|14144|45577|170887|6950||4||1815|33||||15.39|263.01|2789.98|921.45||466.91| +2451910|50487|5498|6392|1580394|581|2815|2|4|1815|24|961.44|86.52|1047.96|32.49|1175.04|615.32|304.58|41.54|1294.05| +2451762|41411|4124|6392|1467444|4481|4318|1|27|1815|61|278.77|13.93|292.70|4.17|794.22|175.62|51.57|51.58|812.32| +2451781|50709|16304|6392|520414|4426|45238|2|12|1815|1|112.05|6.72|118.77|95.96|72.76|70.59|15.34|26.12|175.44| +2452632|45273|2196|29793|1122504|2062|9973|4|12|1818|20|11.40|0.91|12.31|58.72|22.20|11.28|0.00|0.12|81.83| +|38129|17841|90176|94162|61|30760|||1818|9||0.86|22.55|3.78|7.74||||12.38| +2452305|37109|673|88270|618923|3102|21328|1|3|1823|26|491.92|24.59|516.51|11.81|279.24|226.28|233.76|31.88|315.64| +2452381|30605|2077|88270|1115380|6310|27593|1|13|1823|1|22.45|0.22|22.67|43.83|2.86|11.44|6.82|4.19|46.91| +2451834|47473|13609|95426|893019|3412|6158|8|14|1824|12|131.16|1.31|132.47|56.85|5.04|104.92|20.72|5.52|63.20| +2452551|32882|5754|40937|601334|6285|40436|2|1|1825|3|60.33|3.01|63.34|25.77|30.15|28.35|31.02|0.96|58.93| +2452717|34357|9564|87622|876288|5201|5267|8|6|1826|46|1727.30|103.63|1830.93|31.74|1036.38|811.83|402.80|512.67|1171.75| +2452066|36315|6749|55242|1406959|6160|24914|8|19|1827|12|206.64|0.00|206.64|85.41|111.24|78.52|49.96|78.16|196.65| +2451300|55904|14707|72562|306549|2985|24221|2|30|1828|47|2773.00|166.38|2939.38|64.46|658.00|2495.70|16.63|260.67|888.84| +2451298|43747|7093|72562|263576|99|21822|1|5|1828|16|820.00|16.40|836.40|79.63|506.40|196.80|305.36|317.84|602.43| +2451288|45433|2791|71650|5064|1678|44245|4|13|1828|3|165.54|8.27|173.81|70.75|173.61|3.31|71.38|90.85|252.63| +2452046|42292|11369|89003|1513809|1272|7135|2|1|1829|4|23.76|1.90|25.66|48.28|22.48|9.26|12.76|1.74|72.66| +|57497|7944||418410|||10|12|1830|5|41.65|3.74|45.39|70.21|14.50|||23.75|| +2452631|37014|2898|64462|51581|5168|755|10|22|1830|6|29.94|1.19|31.13|14.67|123.66|14.67|6.41|8.86|139.52| +2451113|41106|3763|16130|1499820|4663|18306|1|17|1831|25|1020.00|61.20|1081.20|71.77|540.75|224.40|660.34|135.26|673.72| +2451206|33413|8608|86872|1642555|4274|19193|7|26|1831|59|5688.78|0.00|5688.78|64.84|3719.36|56.88|2027.48|3604.42|3784.20| +2451401|43073|11779|63133|1870047|5460|826|7|18|1833|6|258.72|5.17|263.89|46.05|56.40|232.84|8.54|17.34|107.62| +2451464|35378|16502|63133|20416|2444|1672|2|3|1833|39|1097.07|87.76|1184.83|41.24|648.96|340.09|552.59|204.39|777.96| +2451344|36200|566|63133|1008039|6637|6435|7|23|1833|41|1214.42|109.29|1323.71|46.54|422.71|0.00|340.03|874.39|578.54| +2451352|41656|5690|26510|1051613|4716|30972|4|13|1833|1|45.23|2.71|47.94|36.44|1.07|37.54|4.99|2.70|40.22| +2451384|52038|7483|63133|222259|1119|4826|7|33|1833|17|165.92|4.97|170.89|41.39|23.63|154.30|8.48|3.14|69.99| +2452081|58975|14655|74|781493|6044|46195|2|17|1834|28|1167.88|81.75|1249.63|58.32|116.76|747.44|100.90|319.54|256.83| +2451988|43389|9431|69399|1517173|947|26674|7|22|1834|84|1071.84|21.43|1093.27|96.42|472.92|568.07|125.94|377.83|590.77| +2452667|43428|8184|77583|1271672|4446|17620|1|31|1835|4|225.56|18.04|243.60|97.98|29.88|207.51|14.98|3.07|145.90| +2452788|34855|17047|53422|1245520|6865|35010|1|28|1836|14|3.92|0.27|4.19|15.03|70.56|0.86|2.72|0.34|85.86| +2452728|37933|909|75087|1317023|5388|8106|8|1|1836|5|70.45|0.70|71.15|70.53|130.10|59.17|11.28|0.00|201.33| +2451627|29096|12044||||6955|10|1|1837||46.37|1.85||26.59|13.80||||| +2451481|50796|6716|80479|25138|953|42815|10|15|1838|44|2403.72|168.26|2571.98|91.54|679.36|264.40|1711.45|427.87|939.16| +2451594|39909|3550|80479|274144|4611|14278|8|32|1838|84|47.88|0.47|48.35|4.71|131.88|21.54|13.96|12.38|137.06| +||11039||1304691|2734||2||1840|||70.57||||788.07|38.81|349.35|219.44| +2452549|54020|11550|1049|44575|1701|10074|10|1|1841|35|2168.60|151.80|2320.40|22.47|178.85|1279.47|195.60|693.53|353.12| +2452292|54893|13137|85096|1257358|3161|6211|1|19|1842|28|391.16|7.82|398.98|97.98|956.48|160.37|60.00|170.79|1062.28| +2451704|42497|1400|18079|1425765|3795|5778|7|23|1843|35|2184.70|152.92|2337.62|48.48|561.75|1376.36|460.75|347.59|763.15| +2451557|41828|9808|18079|774615|4070|25878|8|33|1843|23|745.20|44.71|789.91|52.91|171.81|603.61|104.77|36.82|269.43| +2451139|57177|13201|84874|100253|5790|3727|10|1|1844|61|17.08|0.00|17.08|96.75|0.61|7.85|9.23|0.00|97.36| +2451226|31718|11809|84874|657991|85|15732|2|15|1844|12|33.48|2.34|35.82|41.33|4.68|23.10|1.55|8.83|48.35| +2452202|49796|15915|76272|1895188|1679|42923|2|8|1845|56|2667.28|80.01|2747.29|88.53|2667.28|2373.87|134.96|158.45|2835.82| +2451269|55949|7154|27376|541310|1197|35084|4|33|1846|32|1495.68|104.69|1600.37|76.92|907.20|1495.68|0.00|0.00|1088.81| +2452688|31732|11997|64952|114792|4279|16919|10|16|1848|53|988.45|79.07|1067.52|40.26|464.81|889.60|25.70|73.15|584.14| +2452110|41258|9655|90224||1644|16650|||1849|13|54.86|1.09|55.95|99.38|5.98|19.74|9.48|25.64|| +2452061|32655|11563|90224|1226067|2520|34973|1|21|1849|68|3429.24|205.75|3634.99|62.12|1443.64|2743.39|576.11|109.74|1711.51| +2451356|32371|17990|32297|1671176|3029|31235|1|27|1850|44|1069.64|64.17|1133.81|80.74|244.20|866.40|140.23|63.01|389.11| +2451265|55827|3820|99468|807590|2559|10328|2|19|1850|61|397.11|3.97|401.08|64.87|28.06|131.04|167.62|98.45|96.90| +2451138|53937|13664|54189|516604|3819|42187|10|35|1851|2|91.36|5.48|96.84|20.22|19.16|65.77|9.98|15.61|44.86| +2451101|38586|10192|32629|1301680|2731|20582|1|31|1851|12|233.52|11.67|245.19|64.67|89.76|214.83|10.46|8.23|166.10| +2451253|44544|5863|97228|707319|3457|18795|1|33|1853|3|131.40|3.94|135.34|61.64|127.86|35.47|85.37|10.56|193.44| +2451228|39856|754|97228|287687|6006|10325|1|30|1853|15|68.85|4.81|73.66|64.97|68.85|14.45|8.16|46.24|138.63| +2451823|31699|2599|70578|467855|6351|48047|4|34|1854|17|908.65|27.25|935.90|82.81|509.66|463.41|48.97|396.27|619.72| +2451702|58225|8732|20922|435618|2314|19410|2|17|1854|33|105.27|5.26|110.53|52.44|542.52|32.63|54.48|18.16|600.22| +2451979|53175|17141|91420|79579|4891|18203|7|14|1855|7|8.40|0.67|9.07|3.62|122.29|6.80|1.32|0.28|126.58| +2451860|56321|6781|74123|1066533|6496|47259|10|5|1855|69|8453.88|169.07|8622.95|50.84|6604.68|422.69|2810.91|5220.28|6824.59| +2452629|38082|12786|98602|992244|418|11039|8|29|1856|36|2456.28|196.50|2652.78|89.40|1361.16|2186.08|124.29|145.91|1647.06| +2452639|36278|13053|17485|1277636|6422|2808|7|5|1857|28|2326.80|139.60|2466.40|14.10|1389.64|395.55|77.25|1854.00|1543.34| +2451902|49506|6170|94291|672322|6706|15090|2|5|1860|5|0.00|0.00|0.00|34.19|0.00|0.00|0.00|0.00|34.19| +2451957|45149|1910|76075|747043|1211|20579|1|5|1860|29|582.90|40.80|623.70|13.45|1956.92|367.22|81.95|133.73|2011.17| +2451980|33658|14978|57645|261220|1595|5170|4|35|1860|73|1660.75|83.03|1743.78|43.90|255.50|332.15|1288.74|39.86|382.43| +2452673|29977|13806|92539|1176767|3954|19941|4|28|1861|10|1022.70|51.13|1073.83|67.68|641.10|572.71|409.49|40.50|759.91| +2452622|45811|8889|92539|701614|3055|12849|10|4|1861|4|77.12|3.85|80.97|25.01|53.48|57.06|11.83|8.23|82.34| +2451083|34321|11431|90095|1850252|3562|38072|8|4|1862|9|370.17|7.40|377.57|77.48|10.53|273.92|77.00|19.25|95.41| +2451803|50627|13085|27861|1749531|4269|5407|10|21|1863|27|114.21|7.99|122.20|25.58|80.73|30.83|74.20|9.18|114.30| +2451874|52897|13178|36278|1427213|3714|9415|7|7|1864|7|20.58|1.44|22.02|72.69|226.73|5.14|10.49|4.95|300.86| +2451257|60905|4018|79887|1143072|3867|7815|4|1|1865|5|514.70|25.73|540.43|71.39|139.70|442.64|14.41|57.65|236.82| +2452775|48531|16557|32907|1422444|4207|16370|8|21|1866|40|298.00|26.82|324.82|52.18|63.20|187.74|80.48|29.78|142.20| +2452620||7293|||||7|16|1867|4||||43.92|||22.14|0.00|61.22| +2452270|61035|8991|5444|1148353|509|49865|7|25|1869|28|1099.84|87.98|1187.82|7.45|196.28|98.98|910.78|90.08|291.71| +2452258|56077|16925|23695|129736|4130|13371|1|1|1869|1|16.34|0.98|17.32|31.54|8.68|15.03|0.01|1.30|41.20| +2452184|35533|9323|150|1083603|2644|4838|2|23|1871|26|1162.98|34.88|1197.86|67.91|193.70|569.86|533.80|59.32|296.49| +2452108|28806|15329|150|1319568|4496|11424|10|5|1871|69|1009.47|0.00|1009.47|95.52|455.40|111.04|134.76|763.67|550.92| +2451953|46617|1352|84937|1024517|4709|21701|1|30|1873|19|1502.14|30.04|1532.18|11.48|381.14|270.38|184.76|1047.00|422.66| +2451205|56408|820|77208|1618316|1314|48712|1|10|1875|69|1611.84|96.71|1708.55|51.52|2115.54|354.60|314.31|942.93|2263.77| +2451144|29651|6583|77208|245068|4390|14512|7|23|1875|7|29.82|2.08|31.90|78.62|59.71|8.05|0.21|21.56|140.41| +2451171|38061|16027|26517|350499|3942|15385|10|6|1876|6|734.28|14.68|748.96|57.00|252.66|227.62|385.06|121.60|324.34| +2451961|41359|1771|88181|1119071|3114|13604|1|17|1880|8|99.68|6.97|106.65|48.45|204.56|16.94|62.05|20.69|259.98| +2452223|38463|16131|51928|995305|4282|46012|1|14|1882|34|353.60|17.68|371.28|29.55|82.28|254.59|55.44|43.57|129.51| +2451059|42261|2683|28406|318762|7051|17340|4|15|1883|21|903.21|54.19|957.40|47.75|228.06|270.96|322.44|309.81|330.00| +2451203|42625|1286|28406|1295404|4218|20232|2|25|1883|7|808.64|24.25|832.89|66.56|180.74|64.69|193.42|550.53|271.55| +2451910|53743|15307|57660|1794160|4742|34124|7|18|1884|45|1237.95|111.41|1349.36|43.57|104.40|557.07|449.38|231.50|259.38| +2451971|33808|16220|57660|1264493|4097|46854|10|32|1884|41|150.06|4.50|154.56|42.91|21.32|28.51|14.58|106.97|68.73| +2451307|48906|15259|63799|1325722|2329|30136|10|7|1885|49|146.02|8.76|154.78|71.94|267.54|110.97|23.48|11.57|348.24| +2451159|59589|1042|31850|163199|178|46404|7|6|1886|4|62.12|0.00|62.12|61.48|0.00|19.25|24.86|18.01|61.48| +2451180|42558|3442|31850|407970|4931|20292|1|9|1886|50|393.50|3.93|397.43|0.52|310.50|47.22|69.25|277.03|314.95| +2451750||16759|3861|465981||4220||2|1888||||||||2.46||| +2451210|37913|3121|76932|1051522|1008|28494|10|31|1890|3|143.58|11.48|155.06|12.52|26.58|55.99|39.41|48.18|50.58| +2452710|39174|3019|57783|1119800|217|48234|1|28|1893|34|1001.98|20.03|1022.01|51.50|1145.12|971.92|29.15|0.91|1216.65| +2452222|38845|15083|29508|1898612|6907|39155|2|5|1894|18|885.96|79.73|965.69|97.71|243.54|274.64|201.73|409.59|420.98| +2451306|47702|16694|20915|927227|5092|44557|1|14|1895|1|55.03|2.75|57.78|49.86|9.27|26.96|28.07|0.00|61.88| +2452289|47441|4029|88046|1463109|4641|13820|4|6|1896|15|425.85|0.00|425.85|71.49|589.80|349.19|18.39|58.27|661.29| +2452273|50351|17729|88046|1109220|3664|47661|4|12|1896|10|388.30|0.00|388.30|16.64|83.50|186.38|12.11|189.81|100.14| +2451531|42527|8594|36089|1420341|1840|19692|10|6|1898|4|18.80|0.37|19.17|53.79|71.80|17.67|0.85|0.28|125.96| +2451538|29607|5797|96342|1163827|2959|26844|4|24|1899|14|944.58|85.01|1029.59|64.25|18.76|94.45|654.60|195.53|168.02| +2451528|51427|16784|56084|1027633|5935|22237|1|3|1899|17|19.38|1.35|20.73|76.01|63.07|11.43|5.80|2.15|140.43| +2451520|46676|308|56535|1119541|1300|44659|7|27|1899|19|1002.82|0.00|1002.82|14.10|859.56|361.01|532.70|109.11|873.66| +2451601|48345|11474|75952|194062|1304|6146|2|15|1901|1|63.96|1.27|65.23|58.33|57.44|37.09|24.72|2.15|117.04| +2451669||15176||1535722||37777|||1901|42||||||||3.15|| +2451506|37718|17024|75952|1117446|4334|25622|8|15|1901|15|953.40|47.67|1001.07|1.57|123.60|848.52|24.12|80.76|172.84| +2452489|34664|14575|83953|1345637|924|43858|4|24|1902|58|294.64|20.62|315.26|30.88|93.96|188.56|86.98|19.10|145.46| +2451151|29240|10090|1089|1106814|582|47950|10|16|1903|15|309.30|24.74|334.04|87.32|690.15|55.67|27.89|225.74|802.21| +2451190|60351|10286|1089|527700|2412|10324|10|23|1903|21|151.62|9.09|160.71|58.68|71.40|87.93|56.04|7.65|139.17| +2452045|45604|9001|64937|128553|435|33917|7|8|1905|39|3419.13|136.76|3555.89|32.55|404.04|2701.11|193.86|524.16|573.35| +2452014|58695|2755|64937|1849992|5930|13468|1|21|1905|14|1298.64|51.94|1350.58|29.52|279.44|831.12|317.91|149.61|360.90| +2452609|29604|9030|59236|1345497|458|34172|2|27|1907|17|1129.31|101.63|1230.94|62.08|423.47|700.17|330.43|98.71|587.18| +2451293|33962|9572|68742|256078|6562|9046|1|21|1908|2|0.10|0.00|0.10|67.98|1.48|0.06|0.03|0.01|69.46| +2451240|37138|9308|68742|1617253|2378|46703|2|5|1908|38|2947.66|117.90|3065.56|51.45|1889.74|1562.25|872.80|512.61|2059.09| +2452362|46418|5769|1542|564796|5648|14853|7|30|1909|4|63.92|5.75|69.67|99.08|20.68|41.54|15.66|6.72|125.51| +2452425|54212|4302|11937|1454471|5358|48292|1|3|1910|59|261.96|15.71|277.67|65.03|218.30|120.50|123.07|18.39|299.04| +2451603|49201|12088|77559|1375280|5946|9012|7|9|1911|30|24.60|1.72|26.32|62.63|60.90|17.46|2.49|4.65|125.25| +2451498|52145|5540|77559|769969|3101|43833|1|27|1911|5|181.05|16.29|197.34|84.42|157.40|166.56|14.20|0.29|258.11| +2452574|40253|12015|14056|198081|6906|28178|4|25|1912|38|2127.62|127.65|2255.27|76.97|978.50|1148.91|704.67|274.04|1183.12| +|49651|6217|99625||||10||1913|35||||58.14|193.20|130.46|||| +2452194|50750|5655|99625|874627|2435|17843|2|21|1913|30|714.30|7.14|721.44|71.17|486.90|657.15|37.71|19.44|565.21| +2451910|37116|9913|90529|63290|2934|178|2|8|1914|69|712.77|21.38|734.15|94.90|870.78|128.29|327.30|257.18|987.06| +2452635|39668|105|88983|634800|1645|13337|7|24|1915|70|3944.50|0.00|3944.50|77.07|3181.50|1972.25|1301.68|670.57|3258.57| +2452032|31657|7683|59735|1550067|6729|43344|2|8|1916|62|3091.32|92.73|3184.05|76.94|1854.42|1236.52|148.38|1706.42|2024.09| +2451067|59822|1483|1550|116339|1801|4863|1|32|1917|16|496.48|19.85|516.33|45.67|0.00|183.69|28.15|284.64|65.52| +2451172|47073|10657|1550|794688|102|23563|4|16|1917|6|236.76|7.10|243.86|21.59|159.12|0.00|201.24|35.52|187.81| +2452791|54152|16699|96927|1337656|4256|11703|1|33|1918|2|57.10|2.85|59.95|51.43|49.72|33.11|4.55|19.44|104.00| +2452206|30578|1411|29967|1574786|2358|26777|2|5|1919|52|931.84|18.63|950.47|42.86|41.08|279.55|378.32|273.97|102.57| +2450965|39546|12481|54952|365316|6509|22455|2|31|1920|25|3020.50|181.23|3201.73|81.62|733.50|1902.91|324.10|793.49|996.35| +|37190|13582|||4918|31987|||1920|||14.96||3.29|696.93|157.16||508.48|715.18| +2452259|39353|16815|81971|1573501|2766|19631|8|23|1923|7|364.28|25.49|389.77|84.69|84.07|287.78|12.24|64.26|194.25| +2451452|58108|12571|82674|953368|4862|35556|8|17|1926|52|956.80|47.84|1004.64|86.26|308.88|593.21|359.95|3.64|442.98| +2451393|31186|12487|19263|974802|4517|19385|2|3|1926|20|2725.20|190.76|2915.96|76.23|757.00|2125.65|167.87|431.68|1023.99| +2451434|36458|5929|9498|162975|4712|8896|8|22|1927|8|41.52|3.32|44.84|35.37|148.96|30.30|2.24|8.98|187.65| +2452318|50216|5215|97328|1424170|5412|23454|7|33|1928|38|414.96|8.29|423.25|36.48|381.90|33.19|110.71|271.06|426.67| +2452381|55617|15012|59562|48974|2314|3254|2|19|1928|7|192.78|11.56|204.34|36.59|131.46|104.10|44.34|44.34|179.61| +2452301|56538|14853|63648|42035|1512|49836|8|2|1931|37|676.36|13.52|689.88|28.06|829.17|290.83|223.60|161.93|870.75| +2451329|29478|4876|92723|1404028|5669|1357|8|16|1932|69|2295.63|206.60|2502.23|87.94|4591.95|1492.15|634.74|168.74|4886.49| +2451209|29674|7154|93337|770496|3208|9967|10|3|1932|69|621.00|31.05|652.05|85.87|496.80|310.50|158.35|152.15|613.72| +2451214|49528|8575|92723|1701102|6836|37417|4|17|1932|21|364.56|25.51|390.07|80.05|271.53|196.86|157.63|10.07|377.09| +2450990||11314|95243|561353||4426|8|32|1934||45.48|1.36|||27.90||||| +2451494|47458|17677|91303|432874|6561|31848|10|35|1936|19|130.15|11.71|141.86|19.26|373.16|65.07|63.77|1.31|404.13| +2451595|31532|15145|91303|1032950|2765|10933|7|28|1936|6|83.70|3.34|87.04|67.69|3.96|65.28|13.81|4.61|74.99| +2451528|61063|715|91303|1601075|6993|5358|7|29|1936|4|15.76|0.31|16.07|89.33|10.76|12.13|2.94|0.69|100.40| +2452751|37113|8031|79875|1669060|1445|46630|1|25|1940|44|2844.16|113.76|2957.92|9.55|2534.84|2104.67|510.24|229.25|2658.15| +2452620|57547|15666||||3281|8||1940||0.85||0.85|||||0.16|54.10| +2451067|42618|17516|53111|1063679|6484|45242|8|25|1941|44|210.76|2.10|212.86|87.90|1053.80|52.69|15.80|142.27|1143.80| +2451010|60629|11942|60501|973016|4284|17999|1|17|1941|5|216.00|8.64|224.64|15.43|47.55|62.64|61.34|92.02|71.62| +2452666|44629|2301|6566|1075499|2576|44006|7|3|1942|15|374.40|14.97|389.37|79.21|263.40|359.42|14.08|0.90|357.58| +2452660|29336|13867|6566|821110|6740|35570|7|30|1942|55|4276.25|42.76|4319.01|28.76|2380.95|2351.93|1096.86|827.46|2452.47| +2452631|32518|5071|64668|599755|5242|15929|10|8|1942|83|3640.38|327.63|3968.01|75.90|1548.78|254.82|3013.14|372.42|1952.31| +2451320|56299|12868|2136|661102|6155|808|4|34|1943|10|766.80|7.66|774.46|3.48|485.30|598.10|143.39|25.31|496.44| +2451196||4468|2136|||26795|4|16|1943||4168.56||||||295.97|120.89|4799.94| +2451179|44736|10256|42631|1694202|3415|11875|1|15|1944|20|2530.40|101.21|2631.61|54.41|719.00|2201.44|259.87|69.09|874.62| +2451344|29813|13987|42631|470770|4638|21659|2|31|1944|60|2979.60|208.57|3188.17|99.33|936.00|0.00|1638.78|1340.82|1243.90| +2451257|48298|14569|42631|820336|6589|29595|2|9|1944|4|18.28|0.36|18.64|61.10|18.28|8.77|4.94|4.57|79.74| +2451199|49890|5695|64799|1652065|3270|19322|4|17|1945|54|789.48|55.26|844.74|94.76|789.48|31.57|113.68|644.23|939.50| +2451261|50715|11305|64799|758640|2223|28159|7|28|1945|7|73.36|4.40|77.76|53.94|3.29|66.02|2.49|4.85|61.63| +2451093||6211|44008||5306|46349|1|17|1948|43|3048.27||||||1005.32|31.10|1304.75| +2452335|46673|16681|36069|1763114|2959|13651|7|16|1949|5|85.25|3.41|88.66|13.64|31.95|28.98|39.38|16.89|49.00| +2452297|44060|14051|53641|515997|360|37076|8|10|1949|14|281.40|0.00|281.40|72.47|3.92|36.58|95.47|149.35|76.39| +2452670|29339|17781|52193|1288266|5908|31550|7|2|1950|49|1860.04|93.00|1953.04|89.85|2753.31|1860.04|0.00|0.00|2936.16| +2452735|38026|4188|52193|746549|4342|11782|1|28|1950|45|1265.85|88.60|1354.45|16.75|215.10|696.21|336.08|233.56|320.45| +2452694|38745|11253|52193|644528|3677|8873|1|20|1950|22|2025.32|20.25|2045.57|24.08|698.28|587.34|1121.62|316.36|742.61| +2452591|40290|6228|93613|911583|5553|9076|7|16|1951|2|94.36|7.54|101.90|56.59|144.92|65.10|1.17|28.09|209.05| +2452559||17541|11916|364447|3875||||1951|6|645.00|0.00|645.00||||379.26|7.74|| +2451168|30690|5284|25594|877668|2196|39924|7|35|1952|6|398.94|23.93|422.87|20.08|163.86|135.63|258.04|5.27|207.87| +2450931||12848|78110||578|27400|2|11|1953|||23.66|615.24|||278.04|100.33||| +2452740|48699|5571|36205|861683|1862|8841|2|11|1957|78|2431.26|121.56|2552.82|28.68|571.74|340.37|1379.98|710.91|721.98| +2452710|58014|4693|36205|1376607|6650|36693|7|34|1957|3|80.28|2.40|82.68|82.23|91.50|10.43|30.03|39.82|176.13| +2451736|49248|10928|64717|7842|4061|10727|7|7|1960|37|441.78|30.92|472.70|64.53|349.65|53.01|93.30|295.47|445.10| +2452310|56785|8567|75043|396112|3113||||1961|||||||||82.79|| +2452181|59109|17539|6603|1769187|3917|10247|1|17|1963|2|151.02|7.55|158.57|64.25|84.66|83.06|10.87|57.09|156.46| +2451988|50701|4159|95137|1012314|691|10268|1|27|1964|33|464.31|27.85|492.16|74.61|137.61|9.28|318.52|136.51|240.07| +2452005|57654|11021|88493|1773792|3891|44390|1|18|1964|45|1956.60|97.83|2054.43|29.66|183.15|0.00|1702.24|254.36|310.64| +2451766|55112|3860|4860|442565|4235|34258|7|8|1965|34|1885.98|150.87|2036.85|47.71|766.02|678.95|856.99|350.04|964.60| +2451350|40134|10366|62613|1296623|2261|13499|1|30|1967|18|1339.20|80.35|1419.55|93.42|26.10|1111.53|93.34|134.33|199.87| +2451371|32973|13540|62613|1127650|6557|9911|10|30|1967|1|5.77|0.28|6.05|71.44|13.00|0.23|0.72|4.82|84.72| +2451486|38760|10586|57846|14218|5343|10915|4|27|1967|12|378.24|7.56|385.80|74.37|74.28|22.69|341.32|14.23|156.21| +2451218|56350|13130|87217|1631501|731|38361|4|21|1968|24|3346.56|66.93|3413.49|75.07|185.76|66.93|1082.27|2197.36|327.76| +2451594|55604|391|12281|967863|2316|24722|10|19|1970|14|516.32|36.14|552.46|58.84|145.18|345.93|63.04|107.35|240.16| +2451364|44425|5210|60807|1244887|1882|877|8|14|1971|42|1605.66|16.05|1621.71|93.21|1070.58|96.33|150.93|1358.40|1179.84| +2452361|54411|6223|74579|1185856|5489|16769|1|4|1972|24|23.52|1.41|24.93|12.66|823.20|20.46|0.15|2.91|837.27| +2452330|34013|4419|74579|1600904|6609|1867|2|5|1972|5|5.70|0.28|5.98|68.01|263.45|3.07|0.13|2.50|331.74| +2451045|38662|14764|34475|1550565|1442|36719|7|1|1973|22|996.82|49.84|1046.66|68.74|525.58|787.48|31.40|177.94|644.16| +2452019|30496|12206|21060|983524|1344|32364|7|29|1974|1|39.74|3.57|43.31|29.36|26.27|16.29|20.87|2.58|59.20| +2452030|36241|13203|97710|620310|5161|32412|1|9|1975|85|2071.45|165.71|2237.16|16.55|588.20|103.57|1102.01|865.87|770.46| +2451936|49022|9056|68042|674853|3032|37258|1|11|1976|29|271.44|21.71|293.15|63.42|290.87|62.43|71.06|137.95|376.00| +2451294|41934|13018|23985|316667|1159|47204|4|4|1978|4|97.80|0.97|98.77|85.31|213.44|18.58|29.31|49.91|299.72| +2452623|42776|8395|48017|287941|3926|10711|7|21|1979|66|1254.66|0.00|1254.66|67.78|986.04|213.29|562.33|479.04|1053.82| +2452593|44962|9969|55513|1016424|5688|10999|7|29|1979|1|4.29|0.00|4.29|90.66|5.28|2.78|0.87|0.64|95.94| +2452689|49004|7789|48017|1831609|4665|10521|1|25|1979|17|95.20|3.80|99.00|97.67|123.76|52.36|27.84|15.00|225.23| +2451398|44432|109|||201|34982|||1980|||42.44|||215.52|201.62||111.85|| +2451264|44998|11077|75403|1209906|5182|49211|4|33|1981|25|1183.00|70.98|1253.98|32.56|458.75|437.71|626.04|119.25|562.29| +2451278|46247|1678|75403|211072|1359|4846|4|13|1981|41|724.47|57.95|782.42|26.68|703.56|130.40|53.46|540.61|788.19| +2451341|49083|16148|93905|927237|569|4701|4|20|1982|14|456.12|4.56|460.68|97.57|73.92|415.06|34.49|6.57|176.05| +2452443|45965|12477|39178|1836127|4350|36220|8|28|1983|74|547.60|32.85|580.45|58.36|2740.22|169.75|45.34|332.51|2831.43| +2451624|53405|7087|5123|1596610|5350|36890|4|12|1984|26|80.08|0.80|80.88|26.51|178.36|17.61|46.22|16.25|205.67| +2451098|52036|15566|70102|52450|5551|24299|4|33|1987|1|88.90|3.55|92.45|58.01|6.50|20.44|15.74|52.72|68.06| +2451204|44710|3212|46457|1865192|3328|41864|7|21|1987|19|92.15|3.68|95.83|23.48|313.50|89.38|0.52|2.25|340.66| +2451265|38460|11641|46457|1707296|376|25056|1|4|1987|14|244.72|0.00|244.72|56.42|263.62|26.91|39.20|178.61|320.04| +2451756|37760|1075|80524|1604221|4250|35479|1|19|1988|33|0.00|0.00|0.00|55.79|163.35|0.00|0.00|0.00|219.14| +2451226|53143|5774|8311|608365|6325|17038|2|28|1989|1|2.87|0.08|2.95|57.76|8.97|1.92|0.82|0.13|66.81| +2452426|52348|3351|53882|1134368|5280|35392|4|24|1990|9|200.07|2.00|202.07|87.89|135.27|182.06|1.98|16.03|225.16| +2452322|48045|575|81981|921058|6535|42412|2|4|1990|38|1110.74|44.42|1155.16|23.44|1727.86|44.42|917.03|149.29|1795.72| +2452502|58410|8994|30406|1638722|370|9218|2|1|1991|33|310.20|3.10|313.30|49.59|103.29|186.12|67.00|57.08|155.98| +2452436|45490|895|30406|1834238|3423|2255|10|12|1991|15|97.50|1.95|99.45|53.24|568.80|77.02|17.40|3.08|623.99| +||637|72145|215106|2103|20253|4|18|1992|32|1793.92|35.87|1829.79||652.16|1542.77|241.10|10.05|| +2452623|44886|351|48934|971275|5169|6710|1|35|1992|68|3514.92|281.19|3796.11|25.55|846.60|843.58|240.42|2430.92|1153.34| +2452658|42393|9540|30048|1693322|6196|1517|4|30|1992|39|4046.25|323.70|4369.95|33.71|2494.05|849.71|2621.16|575.38|2851.46| +2452558|46199|14821|63521|1392525|6776|44580|4|5|1992|18|561.24|11.22|572.46|82.79|256.50|454.60|60.78|45.86|350.51| +2452653|59121|7194|93818|1618947|6923|23252|8|17|1994|14|721.84|28.87|750.71|16.59|495.60|375.35|336.09|10.40|541.06| +2452716|35652|7851|9544|1607709|6869|32829|2|32|1994|68|2465.68|147.94|2613.62|24.95|394.40|369.85|482.04|1613.79|567.29| +2452622|50883|15798|89859|764745|784|47623|10|18|1995|24|501.36|15.04|516.40|39.10|105.36|310.84|133.36|57.16|159.50| +2452657|37466|11155|37691|860443|2519|4642|2|11|1995|7|947.52|0.00|947.52|69.04|393.89|644.31|206.18|97.03|462.93| +2451649|38219|8419|10133|351978|2270|41315|7|12|1997|8|73.60|0.73|74.33|48.51|120.88|55.20|1.28|17.12|170.12| +2451728|43036|6620|8752|1730949|6266|9230|1|29|1997|25|177.00|5.31|182.31|73.19|886.00|132.75|24.33|19.92|964.50| +2451406|54726|650|20855|634590|2315|18393|2|6|1998|3|85.17|6.81|91.98|96.29|79.29|66.43|14.61|4.13|182.39| +2452014|38696|9917|87295|775131|5014|16097|7|6|1999|55|5041.85|0.00|5041.85|74.90|2016.30|1663.81|2533.53|844.51|2091.20| +2452367|56726|5155|64962|599988|2452|30516|2|14|2000|65|9999.60|599.97|10599.57|58.30|1353.95|7799.68|747.97|1451.95|2012.22| +2452209|29275|13965|95148|1913454|5584|38914|2|34|2003|14|34.44|1.37|35.81|16.04|138.04|30.99|0.79|2.66|155.45| +2452381|51532|17811|95148|172238|6251|8217|2|3|2003|7|60.76|0.00|60.76|42.78|46.34|54.68|3.40|2.68|89.12| +2452262|35421|13019|95148|1075603|1891|47732|10|11|2003|22|761.64|15.23|776.87|41.10|654.94|205.64|511.52|44.48|711.27| +2452090|40058|4111|12942|1199429|3501|3220|10|4|2004|3|29.22|2.62|31.84|88.29|2.04|18.70|3.99|6.53|92.95| +2452062|52700|13201|12942|627686|2236|1539|4|34|2004|2|21.82|1.96|23.78|8.32|41.66|7.41|11.23|3.18|51.94| +2452113|51894|8493|12942|154151|7066|23581|1|14|2004|17|589.90|53.09|642.99|29.76|510.51|153.37|366.68|69.85|593.36| +2452777|38764|16716|94962|515050|973|39249|7|13|2005|13|262.47|10.49|272.96|18.74|205.01|220.47|25.62|16.38|234.24| +2450926|29355|12170|99396|35676|5714|105|4|29|2007|8|46.08|0.00|46.08|55.30|433.12|35.94|3.54|6.60|488.42| +2450977|30535|3001|18260|685176|4539|27502|7|16|2007|1|97.36|8.76|106.12|85.57|26.55|14.60|38.89|43.87|120.88| +2451455|29125|7306|96898|757095|2555|2368|4|31|2009|8|24.24|0.96|25.20|68.40|5.92|8.72|4.19|11.33|75.28| +2451427|33502|5278|96898|487085|1211|5737|10|11|2009|65|1420.90|28.41|1449.31|26.68|2605.20|767.28|392.17|261.45|2660.29| +2451629|47465|9968|71187|939198|418|23238|7|28|2010|43|518.58|20.74|539.32|81.04|107.50|394.12|77.16|47.30|209.28| +2451621|29761|1880|71187|1230913|5539|36118|10|28|2010|16|397.76|23.86|421.62|6.83|75.20|171.03|122.43|104.30|105.89| +2452098|44109|5185|67801|1600087|3330|34973|2|26|2011|28|360.92|25.26|386.18|55.80|154.56|241.81|28.58|90.53|235.62| +2451393|35853|15560|72127|32566|6019|3519|2|10|2012|1|0.00|0.00|0.00|89.98|25.95|0.00|0.00|0.00|115.93| +2451898|40496|14287|5580|1884894|6277|24060|8|13|2013|13|379.21|11.37|390.58|3.79|35.49|345.08|5.46|28.67|50.65| +2451804|37828|2557|6531|554489|6000|27158|8|27|2013|29|328.57|22.99|351.56|54.12|131.37|3.28|296.01|29.28|208.48| +2451635|48076|6908|20787|1465376|4718|8060|4|6|2014|33|1908.72|38.17|1946.89|89.22|534.27|1717.84|47.72|143.16|661.66| +2452187|54946|16127|50883|690963|4465|46414|7|15|2015|1|18.15|0.36|18.51|25.78|5.18|15.60|1.12|1.43|31.32| +2451303|44661|6475|54570|1908484|6802|15167|4|33|2016|71|934.36|18.68|953.04|27.04|3643.72|616.67|295.45|22.24|3689.44| +2451200|30243|6014|54570|1336707|476|33045|7|13|2016|24|1598.40|47.95|1646.35|18.38|459.36|591.40|80.56|926.44|525.69| +2452472|52361|11439|78108|1669504|1046|32988|1|25|2017|1|2.55|0.17|2.72|29.64|11.48|0.38|0.65|1.52|41.29| +2452489|52297|13992|78108|1518286|1885|32284|8|32|2017|6|295.08|2.95|298.03|43.56|127.02|115.08|5.40|174.60|173.53| +2452092|45484|8663|71375|1578030|1253|38476|4|7|2018|6|153.66|9.21|162.87|32.03|7.38|36.87|4.67|112.12|48.62| +2452045|52877|15257|79355|1567670|7185|14347|7|27|2018|29|834.62|66.76|901.38|41.57|616.83|392.27|247.71|194.64|725.16| +2452079|50393|8787|71375|1108426|3328|37877|2|10|2018|25|174.50|5.23|179.73|19.73|892.75|54.09|119.20|1.21|917.71| +2452391|44315|3243|80540|1393822|5198|23774|7|26|2019|7|609.56|48.76|658.32|87.85|203.14|48.76|286.00|274.80|339.75| +2452403|42650|14295|80540|559377|5751|20125|4|23|2019|65|927.55|74.20|1001.75|9.02|3424.85|751.31|47.58|128.66|3508.07| +2452510|51999|15735|44783|1667480|1657|29434|4|8|2020|8|203.92|8.15|212.07|40.20|60.56|169.25|4.16|30.51|108.91| +2452488|56380|9277|44783|960751|4253|20908|1|8|2020|1|133.89|9.37|143.26|39.21|70.13|68.28|51.83|13.78|118.71| +2452641|57889|897|53888|453978|7076|16527|8|31|2021|3|13.68|0.41|14.09|48.84|1.83|8.75|4.14|0.79|51.08| +2452631|50860|9882|93199|150696|3393|24043|2|32|2021|44|616.44|6.16|622.60|24.50|187.00|80.13|370.05|166.26|217.66| +2452724|47695|12537|9699|1471947|6039|28931|8|5|2022|92|400.20|24.01|424.21|35.49|1051.56|160.08|33.61|206.51|1111.06| +2451157|58327|5624|28555|1110397|6960|17523|7|24|2023|2|176.64|7.06|183.70|47.36|79.36|158.97|7.24|10.43|133.78| +2451293|36054|13171|28555|935236|4237|1191|7|28|2023|28|211.12|4.22|215.34|44.70|140.56|90.78|107.10|13.24|189.48| +2451332|29152|15662|20053|1206085|1226|44720|2|10|2025|25|1095.50|10.95|1106.45|8.45|1042.00|657.30|241.01|197.19|1061.40| +2452434|31636|11689|26569|1152491|3797|15903|4|7|2026|2|5.40|0.21|5.61|66.02|5.72|0.64|3.14|1.62|71.95| +2452302|49269|15039|26569|1173611|3012|42860|10|11|2026|1|22.69|1.81|24.50|98.72|7.77|1.13|10.78|10.78|108.30| +2452331|48692|1435|48844|383924|1021|27671|1|23|2027|38|978.12|58.68|1036.80|39.28|471.96|528.18|202.47|247.47|569.92| +2451941|40848|12929|2465|1593804|4790|16976|1|31|2028|21|1484.70|133.62|1618.32|37.33|760.83|905.66|156.34|422.70|931.78| +2451899|37175|4651|82699|472695|6135|37928|7|23|2028|55|3537.05|70.74|3607.79|37.31|2183.50|3289.45|54.47|193.13|2291.55| +2451564|59263|9163|67874|1287518|6410|9349|7|32|2029|6|69.48|0.69|70.17|83.14|34.74|11.11|32.68|25.69|118.57| +2451658|51526|17968|67874|1517735|2251|33120|7|2|2029|6|39.96|1.19|41.15|27.59|4.68|34.76|0.57|4.63|33.46| +2451651|46408|1012|67874|1312182|3375|12568|4|14|2029|10|148.90|5.95|154.85|78.08|99.20|138.47|7.71|2.72|183.23| +2451564|56311|1234|90555|470775|3600|6425|4|23|2030|31|962.55|0.00|962.55|26.00|371.07|818.16|128.50|15.89|397.07| +2452423|47287|17889|45724|186464|70|6829|10|24|2031|57|1985.88|59.57|2045.45|97.93|766.08|1290.82|639.45|55.61|923.58| +2452508|54596|8383|20262|190580|6476|30772|10|34|2034|12|97.80|4.89|102.69|13.61|29.28|89.97|4.61|3.22|47.78| +2452482|29172|1515|20262|1109667|6302|49337|10|7|2034|1|41.12|2.87|43.99|34.85|2.22|12.33|4.89|23.90|39.94| +2451998|61125|11258|26179|1779937|4095|22653|2|7|2035|2|51.36|2.56|53.92|43.65|33.06|8.21|10.78|32.37|79.27| +||17173||1163374|4208|33042|||2036|6||8.01|208.47||74.16||156.16|34.28|141.93| +2451051|54686|9904|70798|1810687|176|4018|1|6|2037|16|516.00|20.64|536.64|74.92|363.04|304.44|19.04|192.52|458.60| +2451198|33253|1588|17773|435306|4677|42001|7|3|2038|54|2471.58|123.57|2595.15|76.23|295.38|1828.96|147.80|494.82|495.18| +2451338|40522|15202|95247|594029|1611|25993|10|30|2038|7|222.74|13.36|236.10|14.36|19.60|191.55|1.55|29.64|47.32| +2452209|57692|6303|43791|354292|6950|45255|10|13|2039|13|23.01|2.07|25.08|50.15|15.99|5.52|16.44|1.05|68.21| +2452252|35469|4689|85323|109979|6282|21738|1|29|2039|11|372.46|7.44|379.90|6.45|115.28|245.82|39.25|87.39|129.17| +2451527|31884|17257|8232|277812|5278|44744|2|11|2041|62|3936.38|314.91|4251.29|60.46|744.62|2834.19|573.13|529.06|1119.99| +2451534|57388|14432|8232|726231|3369|10860|7|14|2041|6|171.18|1.71|172.89|58.73|319.02|85.59|9.41|76.18|379.46| +2451010|38532|10738|96502|677108|6725|27942|8|34|2042|18|1.26|0.03|1.29|62.26|10.44|1.12|0.10|0.04|72.73| +2452044|56827|2673|75331|1447695|7071|48094|7|31|2043|8|346.88|10.40|357.28|32.70|181.68|232.40|57.24|57.24|224.78| +2451207|35102|16525|25789|199480|1971|8862|10|32|2044|84|3922.80|117.68|4040.48|60.85|489.72|863.01|2937.39|122.40|668.25| +2451251|30588|6758|6814|506522|874|5226|2|25|2046|59|522.15|20.88|543.03|50.81|69.62|469.93|51.69|0.53|141.31| +2452309|37144|15929|36646|8785|4891|32197|10|21|2047|3|33.18|0.99|34.17|84.85|81.66|12.94|4.04|16.20|167.50| +|30803|13327|96114||6271|||11|2048|1|149.56||||||23.90|1.53|| +2451672|48409|4237|96114|323020|96|17925|4|32|2048|9|215.73|12.94|228.67|7.16|112.77|45.30|25.56|144.87|132.87| +2452392|47529|4373|7352|1738940|6201|26662|8|14|2050|15|69.30|3.46|72.76|30.34|25.50|46.43|3.88|18.99|59.30| +2451119|53891|9865|16583|376544|1345|15746|1|3|2051|4|66.48|5.98|72.46|96.22|13.84|33.90|32.58|0.00|116.04| +2451162|52381|1148|16583|1711936|4059|39382|4|8|2051|5|65.10|0.65|65.75|44.39|20.00|24.08|36.91|4.11|65.04| +2451972|48686|9025|3489|517264|3726|7835|7|31|2052|8|10.56|0.42|10.98|51.97|21.12|6.12|3.50|0.94|73.51| +2451897|58306|445|3489|1671766|3155|9344|1|17|2052|1|4.71|0.32|5.03|63.87|2.82|3.57|0.54|0.60|67.01| +2451920|36193|5403|47781|851652|2462|31672|1|30|2055|38|449.92|35.99|485.91|39.49|202.16|58.48|70.45|320.99|277.64| +2451157||4490|||||8|15|2057|60|||1367.51|18.88|||169.37|207.01|1856.79| +2452346|54929|3225|69895|223868|3761|24054|1|35|2058|7|470.61|18.82|489.43|59.61|182.14|131.77|264.29|74.55|260.57| +2452289|47540|7491|69895|1259142|6816|39690|7|20|2058|62|1429.72|14.29|1444.01|98.43|1310.06|1329.63|50.04|50.05|1422.78| +2452247|47468|15881|831|889148|6712|43829|7|13|2059|24|61.92|3.71|65.63|79.70|334.32|16.09|38.95|6.88|417.73| +2452489|44230|1920|39897|1436170|795|12573|2|23|2060|5|265.10|2.65|267.75|90.89|329.55|159.06|18.02|88.02|423.09| +2452470|32686|1539|39897|1772717|6758|20799|10|6|2060|5|417.25|4.17|421.42|30.01|0.00|125.17|262.87|29.21|34.18| +2452405|48556|12339|39897|984158|2747|46889|7|22|2060|43|2365.43|23.65|2389.08|11.21|1220.77|1892.34|468.35|4.74|1255.63| +2452303|46775|3017|41077|1749728|3979|40230|4|32|2061|42|2541.84|152.51|2694.35|39.88|296.94|1804.70|309.59|427.55|489.33| +2452222|43605|9249|82675|956504|5327|4503|4|20|2061|28|3307.36|66.14|3373.50|6.81|893.76|3042.77|161.39|103.20|966.71| +2451729|35490|5593|40467|1801318|6726|45734|7|33|2062|37|2316.20|46.32|2362.52|47.57|31.82|1297.07|754.15|264.98|125.71| +2452735|45203|769|39575|1561363|3120|36030|7|10|2064|20|471.40|23.57|494.97|15.76|841.80|348.83|77.21|45.36|881.13| +2452168|60205|13701|31026|1105970|5572|32727|2|4|2065|25|1911.00|133.77|2044.77|58.17|114.00|573.30|1150.42|187.28|305.94| +2452174|59405|10421|31026|1471311|900|13642|8|28|2065|37|1110.00|33.30|1143.30|79.51|1027.86|388.50|512.26|209.24|1140.67| +2452200|38985|9669|78478|828090|3302|7314|2|13|2065|59|77.88|6.23|84.11|99.17|53.10|34.26|8.72|34.90|158.50| +2451784|32250|4247|48048|1128681|1865|5449|4|4|2066|2|1.32|0.10|1.42|22.97|31.76|1.30|0.01|0.01|54.83| +2451173|30623|4936|90926|940080|1988|8649|8|25|2067|37|1067.82|0.00|1067.82|62.51|742.22|1067.82|0.00|0.00|804.73| +2451242|40933|1573|20021|1889158|1142|14279|8|12|2067|3|51.69|2.58|54.27|87.78|1.14|11.37|31.44|8.88|91.50| +2451554|52708|7789|70845|1313793|348|40550|7|22|2068|60|223.20|6.69|229.89|75.57|51.60|167.40|38.50|17.30|133.86| +2451216|32708|17602|41706|1799631|1384|43537|10|21|2069|68|684.76|54.78|739.54|65.35|134.64|89.01|363.40|232.35|254.77| +2451293|54799|5000|4070|1755730|543|26556|4|2|2069|51|1827.84|109.67|1937.51|95.68|288.15|1005.31|600.44|222.09|493.50| +2451978|50747|509|70679|936158|3237|29916|10|16|2070|11|63.91|5.11|69.02|13.50|136.95|0.00|4.47|59.44|155.56| +2451936|41578|9259|68243|555408|1326|8491|8|29|2070|8|3.12|0.18|3.30|68.66|10.48|2.55|0.53|0.04|79.32| +2451055|33707|853|93996|653680|1871|4909|2|6|2071|62|1850.70|37.01|1887.71|98.15|1623.78|1702.64|142.13|5.93|1758.94| +2451083|45713|1780|93996|1223756|782|44208|2|13|2071|1|59.39|4.15|63.54|98.98|49.67|30.88|10.54|17.97|152.80| +||7112||1180963|6936||||2071|||14.33||0.52||22.94|31.65|232.16|| +2452387|58419|1007|59352|526962|3770|11378|2|32|2072|21|561.75|5.61|567.36|31.45|0.00|168.52|125.83|267.40|37.06| +2452396|31912|16119|59352|976345|2788|1278|10|23|2072|41|1393.18|83.59|1476.77|36.78|1094.29|571.20|624.70|197.28|1214.66| +2451541|47244|6118|92360|1363109|2981|26853|8|27|2075|23|165.83|13.26|179.09|57.94|560.05|77.94|86.13|1.76|631.25| +2452284|32857|9251|78907|1227329|5897|5712|8|27|2076|4|4.76|0.38|5.14|78.39|0.56|2.90|0.13|1.73|79.33| +2451485|43050|10970|95689|1851472|6532|11192|7|7|2078|22|1145.54|45.82|1191.36|75.93|187.00|618.59|184.43|342.52|308.75| +2451509|44611|10064|95689|1808837|3965|46340|4|8|2078|21|166.11|13.28|179.39|58.22|324.66|94.68|6.42|65.01|396.16| +2451429|34161|6967|40573|966784|995|39606|2|21|2078|85|7232.65|216.97|7449.62|71.48|1141.55|4701.22|531.60|1999.83|1430.00| +2452371|53943|9577|57028|163379|3330|43234|1|11|2080|67|1565.12|140.86|1705.98|15.46|1070.66|610.39|849.70|105.03|1226.98| +2451893|49554|6667|34586|1580748|4805|6909|10|35|2081|2|186.74|1.86|188.60|28.51|8.48|0.00|168.06|18.68|38.85| +2451964|49552|9080||214864||||22|2081|4|||24.14|94.62||8.67|14.17|0.60|| +2451152|55167|1894|12835|1019928|434|25387|8|21|2082|10|274.80|13.74|288.54|10.37|50.90|272.05|0.19|2.56|75.01| +2451159|39480|1736|12835|1706465|5895|22078|7|33|2082|6|322.08|9.66|331.74|74.03|38.64|28.98|284.30|8.80|122.33| +|38204|10280|81784|||17575|4||2083||2147.66||2276.51||||975.89||693.90| +2451682|31614|6703|81784|1539041|196|49704|8|27|2083|20|1527.60|0.00|1527.60|94.61|578.80|244.41|1193.36|89.83|673.41| +|48819|1568||1766320||||14|2083|69||||30.25|||||| +2451305|50774|10982|8747|164413|4526|30592|1|16|2085|36|536.04|26.80|562.84|5.77|157.68|294.82|45.83|195.39|190.25| +2451188|46619|8419|67993|1720001|2896|25438|1|25|2085|16|119.52|10.75|130.27|82.41|12.96|21.51|40.18|57.83|106.12| +2451225|56068|8680|75307|1320955|5803|6651|4|27|2085|16|60.32|2.41|62.73|81.88|336.64|60.32|0.00|0.00|420.93| +2451314|37903|148|12995|688975|4906|38437|4|6|2087|45|2343.60|46.87|2390.47|48.23|2073.15|1078.05|1164.30|101.25|2168.25| +2451801|48535|3733|71697|1485771|5754|12937|1|20|2089|10|235.70|11.78|247.48|20.31|34.90|106.06|115.37|14.27|66.99| +2452430|35715|14323|37095|1391197|6224|18898|4|12|2092|8|128.40|3.85|132.25|87.86|47.76|35.95|60.09|32.36|139.47| +2452401|31409|2772|4026|413322|3065|19208|7|22|2092|1|3.90|0.23|4.13|57.08|13.00|0.31|1.47|2.12|70.31| +2452212|55848|2555|69298|260798|6851|4104|7|22|2093|4|85.68|3.42|89.10|31.25|5.84|70.25|7.56|7.87|40.51| +2451500|42766|3586|70062|245175|5903|43431|7|25|2095|5|10.75|0.21|10.96|73.30|0.15|9.35|0.81|0.59|73.66| +2451439|57385|6913|6334|174678|1869|43813|4|33|2095|73|651.16|6.51|657.67|27.17|611.74|214.88|405.74|30.54|645.42| +2452067|59674|6341|88541|1819052|389|46926|8|30|2096|42|1152.90|11.52|1164.42|77.06|1128.54|334.34|818.56|0.00|1217.12| +|34549|14667|88541|||16555|8|3|2096|30|||590.27||97.80|||16.56|| +2451862|33214|17713|39966|1403635|4232|34876|1|27|2097|6|480.30|24.01|504.31|8.38|43.62|321.80|158.50|0.00|76.01| +2452260|39308|15755|54955|507160|5250|38540|8|18|2098|64|663.04|33.15|696.19|67.37|1621.76|556.95|7.42|98.67|1722.28| +2452178|46523|1805|54955|1698446|6769|38692|2|5|2098|8|212.32|16.98|229.30|38.54|196.64|42.46|163.06|6.80|252.16| +2451118|50242|4048|27458|1499198|6138|43525|1|32|2099|43|835.06|8.35|843.41|86.06|965.78|526.08|216.28|92.70|1060.19| +2451158|35287|12104|11548|967080|6173|5313|7|6|2099|29|97.44|0.00|97.44|20.06|3.48|39.95|20.12|37.37|23.54| +2451268|30470|15190|11548|1201701|7096|42938|4|10|2099|2|166.56|0.00|166.56|38.89|12.66|138.24|3.68|24.64|51.55| +2452275|35183|11409|27640|1825524|3762|18800|2|6|2100|52|4552.08|273.12|4825.20|3.21|3495.44|318.64|3513.75|719.69|3771.77| +2452167|58569|15529|27640|1471165|2021|17677|1|34|2100|3|251.88|0.00|251.88|78.15|98.19|201.50|20.15|30.23|176.34| +2452446|41690|16621|79284|1384736|57|19025|7|29|2101|50|1919.00|153.52|2072.52|8.91|139.00|1228.16|538.85|151.99|301.43| +2452241|43212|13497|15428|544341|4833|35725|8|34|2102|2|117.76|9.42|127.18|29.21|30.98|97.74|10.01|10.01|69.61| +2452258|50209|7037|15428|1224946|2919|29739|7|12|2102|54|5.40|0.16|5.56|77.13|1.62|5.18|0.05|0.17|78.91| +2452345|59998|13237|75182|984082|6582|33082|2|2|2102|2|241.52|19.32|260.84|31.49|64.88|198.04|8.26|35.22|115.69| +2452319|57935|4811|15428|1716692|4578|36864|1|8|2102|19|611.23|12.22|623.45|61.88|15.96|458.42|19.86|132.95|90.06| +|29701|7045|5533|53552|5425||7|6|2109|2|||68.43|76.04|||3.65||227.77| +2451975|31977|15623|43923|1865095|5276|35057|8|2|2110|31|2017.48|121.04|2138.52|64.54|1063.61|968.39|618.96|430.13|1249.19| +2451758|55528|3361|46352|110266|5184|12902|7|33|2111|27|170.10|3.40|173.50|6.55|22.68|161.59|4.25|4.26|32.63| +2451287|50325|13135|14858|1312981|2304|46474|1|32|2113|19|365.37|10.96|376.33|75.26|763.80|208.26|56.55|100.56|850.02| +2451310|57290|12637|58720|1167053|473|13893|1|18|2113|19|83.60|4.18|87.78|24.04|195.13|10.03|41.19|32.38|223.35| +2451246|46650|3562|58720|1648641|3883|13172|1|30|2113|15|801.90|56.13|858.03|95.12|343.65|673.59|124.46|3.85|494.90| +2451231|41006|6130|58720|900287|3364|17994|8|13|2113|1|119.85|9.58|129.43|59.28|64.01|65.91|43.69|10.25|132.87| +2452206|47733|17667|78087|256096|2863|33230|4|19|2114|23|2824.17|225.93|3050.10|85.28|798.10|875.49|194.86|1753.82|1109.31| +2452672|44540|1536|9530|1681882|2606|15730|8|20|2115|14|654.22|32.71|686.93|25.55|446.46|601.88|16.22|36.12|504.72| +2452640|31142|16765|9530|637473|1671|39069|10|17|2115|33|2908.29|261.74|3170.03|55.50|1537.47|1221.48|1180.76|506.05|1854.71| +2451522|46677|2701|23445|1385703|726|6196|1|27|2116|11|685.30|54.82|740.12|46.65|146.30|34.26|364.58|286.46|247.77| +2451498|34907|17542|37949|669473|1991|35356|8|21|2117|41|3369.79|101.09|3470.88|23.42|1521.92|572.86|1929.88|867.05|1646.43| +2452149|30900|777|84677|1606274|318|15515|1|1|2118|33|2354.55|94.18|2448.73|31.44|133.65|470.91|452.07|1431.57|259.27| +2452158|53736|1491|44440|375960|5045|48577|7|27|2118|36|297.00|23.76|320.76|55.59|514.80|231.66|5.88|59.46|594.15| +2452217|59412|2415|81033|669611|5836|46814|7|27|2119|1|113.15|6.78|119.93|79.78|48.25|75.81|30.24|7.10|134.81| +2451321|48344|4238|23274|1499174|4460|3393|7|27|2120|4|60.00|4.80|64.80|90.91|226.24|52.80|1.36|5.84|321.95| +2451317|40624|7195|23274|1871328|1385|9489|10|13|2120|13|1239.94|74.39|1314.33|65.79|317.46|1016.75|107.13|116.06|457.64| +2452347|60761|17947|85243|980623|6860|13097|10|18|2121|46|759.46|0.00|759.46|66.57|1464.64|326.56|225.10|207.80|1531.21| +2452246|42502|3817|85243|1710629|6648|39867|2|30|2121|78|1868.88|0.00|1868.88|73.30|1625.52|1569.85|89.70|209.33|1698.82| +2452274|60026|9701|2434|223529|1071|21766|1|28|2123|20|1057.00|42.28|1099.28|36.99|184.80|708.19|136.03|212.78|264.07| +2452373|38922|13621|57221|133734|3720|46106|7|33|2123|8|324.80|6.49|331.29|72.76|0.00|211.12|69.34|44.34|79.25| +2452347|45550|14315|57221|81342|4938|13734|2|6|2123|7|91.91|5.51|97.42|45.85|59.85|72.60|10.81|8.50|111.21| +2452227|30961|13095|57221|810283|6945|44454|8|15|2123|1|25.63|1.02|26.65|52.22|73.70|2.30|12.36|10.97|126.94| +2452353|47010|9371|57221|||||11|2123|30|1846.80|||||1071.14|||| +2451916|52988|7951|97110|745944|191|47238|4|5|2124|11|1027.73|61.66|1089.39|67.08|540.87|493.31|534.42|0.00|669.61| +2450963|39830|13294|38970|825548|3730|2359|1|23|2126|21|662.97|19.88|682.85|65.10|67.41|623.19|13.52|26.26|152.39| +2452153|39055|7491|85055|1527373|4918|39310|7|17|2127|44|2748.68|82.46|2831.14|71.32|1603.36|192.40|1099.20|1457.08|1757.14| +2452281|45480|8973|32412|150817|6822|6380|8|20|2128|2|130.44|5.21|135.65|4.64|56.12|31.30|83.27|15.87|65.97| +2452279|37649|15379|42963|132268|2522|45576|7|34|2128|14|462.14|0.00|462.14|99.22|226.38|55.45|77.27|329.42|325.60| +2452111|31782|4891|72676|924391|2548|258|7|30|2129|7|198.66|3.97|202.63|40.93|276.36|186.74|8.22|3.70|321.26| +2452788|30027|8071|34401|1423251|3586|47521|2|9|2130|31|497.55|39.80|537.35|73.01|455.08|263.70|205.78|28.07|567.89| +2452647|37769|10317|11464|||||11|2132||36.48|||12.50||22.25|||| +2452603|46943|11466|58319|1009149|779|4009|8|14|2132|29|13.63|0.81|14.44|53.60|23.49|0.00|8.99|4.64|77.90| +2452061|43375|16735|50009|495817|7009|19801|2|28|2133|1|18.86|0.94|19.80|66.21|2.31|5.46|5.89|7.51|69.46| +2452073|57348|10437|50009|638794|2917|39040|8|31|2133|21|1596.84|0.00|1596.84|83.44|748.44|479.05|994.83|122.96|831.88| +||13955|||3651||||2135|||||9.75|||||22.53| +2451481|55019|16222|94638|497952|3135|45871|4|25|2136|35|4009.25|0.00|4009.25|40.43|1145.20|3969.15|23.65|16.45|1185.63| +2451203|28937|14174|72009|250639|2662|21085|10|6|2139|36|733.68|14.67|748.35|13.92|1381.32|652.97|46.81|33.90|1409.91| +2451337|47046|8503|72009|1286765|2945|34725|2|8|2139|7|631.33|44.19|675.52|46.74|196.63|606.07|24.75|0.51|287.56| +2451195|56702|4255|||754|||7|2139|||2.62|265.12||85.40|115.50||101.43|157.47| +2451254|54745|760|72009|634437|4873|10156|2|33|2139|1|75.03|3.75|78.78|13.97|5.00|29.26|45.77|0.00|22.72| +2451292|29396|7498|97068|310158|3999|21840|8|27|2140|8|33.84|3.04|36.88|56.86|145.60|21.99|10.78|1.07|205.50| +2452347|47170|13519|53514|455740|5643|45546|1|25|2142|25|158.50|9.51|168.01|54.33|47.00|28.53|118.27|11.70|110.84| +2452678|37230|6205|58|686029|1379|3176|2|33|2143|19|723.52|43.41|766.93|26.15|30.97|600.52|82.41|40.59|100.53| +2452709|30059|15223|81766|856693|2373|10053|7|35|2143|51|232.56|20.93|253.49|75.83|100.98|86.04|19.04|127.48|197.74| +2451529|39839|2665|16561|52808|2827|43293|2|31|2144|2|101.06|9.09|110.15|56.69|72.64|10.10|41.84|49.12|138.42| +2451620|32064|7544|26216|1668003|6590|4330|1|4|2144|13|731.51|36.57|768.08|52.33|238.81|687.61|36.87|7.03|327.71| +2451659|43961|6071|36671|1447347|5989|46278|7|22|2145|6|235.50|0.00|235.50|63.70|116.22|230.79|3.67|1.04|179.92| +2451679|47127|380|36671|641512|2034|2956|4|18|2145|8|279.44|2.79|282.23|74.54|52.80|100.59|80.48|98.37|130.13| +2451661|30356|14768|36671|1462092|3182|36719|8|29|2145|8|382.64|26.78|409.42|84.02|48.88|45.91|195.30|141.43|159.68| +2451742|60231|15638|36671|1425614|6572|14683|4|24|2145|67|469.00|37.52|506.52|73.94|422.10|389.27|58.20|21.53|533.56| +2451304|37795|15700|94059|62914|644|35174|1|2|2146|28|1231.16|49.24|1280.40|91.77|1373.12|517.08|314.19|399.89|1514.13| +2451312|29553|2378|94059|114472|661|10403|1|14|2146|26|446.68|4.46|451.14|18.77|632.84|35.73|57.53|353.42|656.07| +2451120|38878|8401|9969|454723|6677|46231|4|19|2147|20|175.20|7.00|182.20|46.01|876.80|164.68|5.99|4.53|929.81| +2451127|44543|9709|44680|976075|3554|8280|1|23|2147|20|2125.60|127.53|2253.13|11.23|397.00|1317.87|185.77|621.96|535.76| +||7812|||||7|5|2149|70|||||1379.70||||| +2452733|58174|9415|93714|289185|6703|19774|2|28|2149|27|397.44|31.79|429.23|66.96|397.44|329.87|19.59|47.98|496.19| +2452634|58507|15603|32437|565346|5895|18444|8|29|2150|21|52.29|4.18|56.47|25.26|483.63|48.10|2.63|1.56|513.07| +2452604|37453|15351|32437|1856479|4453|45227|8|35|2150|22|317.02|3.17|320.19|73.56|239.58|218.74|57.00|41.28|316.31| +2451929|58581|10625|65604|1148866|18|40323|1|15|2151|49|57.33|0.57|57.90|30.34|0.98|5.15|7.82|44.36|31.89| +2451997|35965|10099|93130|773224|71|36965|1|28|2151|41|855.26|76.97|932.23|66.81|439.11|709.86|122.13|23.27|582.89| +2452649|43638|1161|4116|80077|6604|16331|10|5|2152|11|128.37|7.70|136.07|20.89|24.64|26.95|5.07|96.35|53.23| +2452220|51383|7053|44140|694986|2403|28573|8|33|2154|15|1079.25|21.58|1100.83|48.82|344.85|755.47|90.65|233.13|415.25| +|31950|6179|44140|936693|2987|1634|||2154|||97.65||||1220.64||382.47|| +2452190|55883|9233|93240|1818273|1706|18339|10|28|2154|27|533.79|48.04|581.83|79.81|1067.85|165.47|349.90|18.42|1195.70| +2452345|44160|14297|44140|6969|1329|89|1|9|2154|13|405.21|20.26|425.47|59.85|14.95|348.48|21.55|35.18|95.06| +2451273|28852|13009|38048|1430804|2760|316|2|18|2155|32|607.68|6.07|613.75|97.15|1397.44|492.22|109.68|5.78|1500.66| +2451234|59584|4396|72392|864632|2096|18760|2|33|2155|1|27.26|0.00|27.26|94.09|21.65|15.26|2.52|9.48|115.74| +2452663|54505|2017|62487|115246|4668|21690|2|14|2156|20|884.00|8.84|892.84|98.77|651.40|795.60|50.38|38.02|759.01| +2452779|44930|3939|42044|71126|1652|18178|4|32|2157|19|76.76|3.07|79.83|37.31|38.38|36.84|14.37|25.55|78.76| +2451593|52896|500|78887|1058502|4622|49095|8|5|2159|24|878.16|61.47|939.63|22.10|257.52|439.08|425.90|13.18|341.09| +||14854|78887|1839559||1799||2|2159|||12.38|425.08|97.44|315.95||115.88||425.77| +2451967|40043|4274|81216|1706503|5000|9850|8|4|2160|17|121.38|10.92|132.30|39.09|303.45|4.85|74.57|41.96|353.46| +2452018|40445|17165|81216|546441|4340|34992|1|25|2160|87|8775.69|87.75|8863.44|11.23|5390.52|6055.22|2312.39|408.08|5489.50| +2451933|41438|3338|77030|143133|1236|37786|1|17|2160|62|2213.40|88.53|2301.93|49.12|1199.08|818.95|1282.89|111.56|1336.73| +2451802|41774|17273|40541|521632|6863|44094|2|33|2161|53|113.95|7.97|121.92|59.82|1065.83|44.44|68.11|1.40|1133.62| +2451884|35175|9875|36156|487247|6759|11471|4|15|2162|73|8823.51|176.47|8999.98|63.95|1043.17|2823.52|2339.99|3660.00|1283.59| +2451380|39081|4622|42099|231277|7089|10901|8|5|2163|8|125.76|11.31|137.07|4.78|350.32|49.04|62.91|13.81|366.41| +2451506|59966|17308|21306|879993|5080|9367|10|19|2163|55|1761.65|17.61|1779.26|74.09|1761.65|317.09|1227.87|216.69|1853.35| +2451389|51181|12811|9162|1530409|6856|29303|4|25|2164|5|27.00|0.27|27.27|50.40|12.80|18.36|4.40|4.24|63.47| +2451366|46496|6386|38812|926742|2765|22035|2|3|2164|27|797.04|63.76|860.80|84.49|837.00|589.80|145.06|62.18|985.25| +2451937|54068|3893|8818|917109|6600|14138|1|5|2166|2|25.18|1.00|26.18|62.50|17.26|23.92|0.66|0.60|80.76| +2452690|31865|8541|4127|164962|3427|43892|4|14|2167|57|1523.04|91.38|1614.42|43.30|164.16|700.59|41.12|781.33|298.84| +2452600|43081|12891|4127|989030|3556|18835|2|33|2167|26|1876.94|56.30|1933.24|53.27|318.50|1088.62|31.53|756.79|428.07| +||13303|59925|1713163||18486|8|11|2167||126.54|8.85|135.39|||27.83|7.89|90.82|| +2451236|43924|662|89425|1708501|5345|9436|2|1|2169|1|69.16|0.00|69.16|91.25|28.43|6.91|37.35|24.90|119.68| +2451334|54606|13666|89425|1642650|1516|25535|7|5|2169|22|817.30|16.34|833.64|6.34|497.42|670.18|5.88|141.24|520.10| +2451847|44737|13235|7837|235872|4707|36342|8|19|2170|20|62.40|3.12|65.52|17.44|436.80|3.12|46.83|12.45|457.36| +||4563||960367||||27|2171||||0.00|94.63||0.00|0.00||| +2452381|57870|10835|61675|448984|3229|16274|8|4|2171|96|3330.24|299.72|3629.96|81.28|255.36|1665.12|399.62|1265.50|636.36| +2451389|47116|14444|85097|1717506|5864|24975|1|31|2173|1|24.55|1.22|25.77|19.98|26.60|2.20|19.66|2.69|47.80| +2452620|54211|583|89919|973897|4818|35339|2|29|2176|34|1097.18|98.74|1195.92|93.89|619.14|120.68|166.00|810.50|811.77| +2452594|50405|10746|89919|1582352|4835|39718|1|24|2176|55|3861.55|77.23|3938.78|4.02|1895.30|3745.70|3.47|112.38|1976.55| +2451237|44785|2227|34985|1685185|2060|2982|2|19|2177|15|112.65|3.37|116.02|53.39|135.30|32.66|24.79|55.20|192.06| +2451191|33869|17690|42854|1795510|3366|46422|10|34|2177|1|36.12|0.00|36.12|51.36|8.83|9.03|3.52|23.57|60.19| +2451976|40365|15213|52173|702977|5095|47260|1|23|2178|14|315.00|3.15|318.15|43.36|259.42|138.60|134.06|42.34|305.93| +2452030|57702|9939|52173|293070|4379|20841|1|32|2178|62|47.74|0.00|47.74|48.45|62.00|30.07|10.77|6.90|110.45| +2451506|39862|889|77262|1765742|2669|15149|10|11|2179|29|3658.93|292.71|3951.64|19.69|1065.17|1207.44|1789.58|661.91|1377.57| +2451779|39249|5198|59232|803036|6487|17491|2|16|2180|11|630.85|50.46|681.31|93.06|693.88|346.96|238.46|45.43|837.40| +2451692||926|||3404|32278|7||2180|73||316.41|5589.93||888.41|||490.44|1241.13| +2451689|40135|11864|64078|1372290|4911|20732|10|6|2180|15|54.15|2.16|56.31|73.29|225.90|21.66|27.29|5.20|301.35| +2452034|49898|8641|81700|1769726|3412|11098|1|27|2181|31|92.38|1.84|94.22|55.39|0.93|58.19|28.71|5.48|58.16| +2451945|59179|1765|81700|50536|4979|43775|7|31|2181|8|26.16|0.52|26.68|11.30|7.04|11.51|5.86|8.79|18.86| +2451506|31958|12397|71684|617894|4551|18761|2|28|2182|69|1012.23|0.00|1012.23|20.95|768.66|334.03|135.64|542.56|789.61| +2451585|47291|8413|66258|382668|2327|43186|1|24|2182|5|122.95|8.60|131.55|97.22|76.85|50.40|47.88|24.67|182.67| +2452404|58525|11157|79281|205759|5037|20715|2|3|2183|4|33.12|2.31|35.43|12.73|73.64|6.29|23.87|2.96|88.68| +|57113|15921||757011|5841|23918|7||2184||||108.35||||||| +2451415|49446|10382|65722|122281|1312|21990|4|19|2185|4|174.48|0.00|174.48|92.69|89.04|139.58|19.54|15.36|181.73| +2451420|49479|4183|65722|637404|6819|5834|7|17|2185|10|433.00|34.64|467.64|75.07|213.00|99.59|213.38|120.03|322.71| +2451709|31454|15119|29065|79089|2812|6888|8|34|2186|36|1199.16|83.94|1283.10|13.15|1693.08|563.60|12.71|622.85|1790.17| +2451733|48910|17450|10200|1532986|1983|46180|4|24|2187|48|870.72|8.70|879.42|33.70|1262.40|461.48|130.95|278.29|1304.80| +2451682|52288|3667|10200|1610334|2654|5774|7|19|2187|12|60.60|0.60|61.20|82.95|37.20|4.24|18.59|37.77|120.75| +2450907|33005|16444|57218|1350037|3641|18892|10|5|2188|16|2289.60|183.16|2472.76|57.41|48.64|2175.12|41.21|73.27|289.21| +2450894|36473|3688|57218|247639|2518|42197|10|5|2188|72|816.48|16.32|832.80|82.60|373.68|759.32|10.28|46.88|472.60| +2450938|39659|8534|57218|109214|3357|39947|2|33|2188|68|4219.40|42.19|4261.59|96.87|1376.32|2320.67|645.56|1253.17|1515.38| +2450893|59762|6679|57218|726632|2396|26960|7|21|2188|14|432.88|30.30|463.18|40.90|417.90|168.82|52.81|211.25|489.10| +2451284|54610|8062|56895|444540|2961|30182|4|19|2190|31|37.20|2.97|40.17|89.09|1492.03|0.74|17.13|19.33|1584.09| +2451286|38340|8266|8872|761905|1015|46925|1|15|2191|59|915.68|54.94|970.62|59.02|1831.36|805.79|80.21|29.68|1945.32| +2451709|35189|439|56215|1502400|3424|1248|10|21|2192|26|1253.98|100.31|1354.29|43.69|875.42|1128.58|57.68|67.72|1019.42| +2451300|32920|12445|19752|347503|3466|1029|10|10|2196|12|94.56|3.78|98.34|99.84|126.12|64.30|17.55|12.71|229.74| +2451283|41710|17128|19752|413979|4481|22936|4|8|2196|9|12.33|0.00|12.33|46.19|6.57|12.20|0.03|0.10|52.76| +2451252|47143|10639|19752|1260016|2756|19096|2|24|2196|17|493.00|14.79|507.79|66.65|422.62|19.72|160.91|312.37|504.06| +2452413|32839|9522|29102|47549|2795|9288|7|17|2197|2|74.36|1.48|75.84|20.66|114.64|65.43|8.84|0.09|136.78| +2451076|54431|3871|89327|1383442|4754|1266|10|5|2198|17|878.56|17.57|896.13|1.23|625.60|571.06|166.05|141.45|644.40| +2451187|57599|10982|89327|539149|6112|13875|4|4|2198|77|758.45|22.75|781.20|19.13|126.28|113.76|96.70|547.99|168.16| +2451074|41483|8048|89327|1494958|414|47470|2|22|2198|38|1075.40|0.00|1075.40|18.21|767.98|473.17|12.04|590.19|786.19| +2452168|29269|2811|2833|1247760|5043|47126|7|15|2199|16|32.00|2.24|34.24|97.02|16.80|24.00|5.20|2.80|116.06| +2452093|33323|9605|69774|364112|5297|7682|1|15|2199|3|0.00|0.00|0.00|57.55|1.20|0.00|0.00|0.00|58.75| +2452129|30265|14483|34716|1073076||41302|1|15|2199|28|29.40|||||24.99|4.14|0.27|| +2452097|35530|17471|78375||3359|32651|10|18|2199|||41.14|863.99|||534.85|||| +2451743|56475|11342|26856|512417|184|18798|4|4|2202|14|15.54|0.46|16.00|32.93|130.20|6.99|0.51|8.04|163.59| +2452012|52396|1451|27308|199222|6336|46603|1|6|2203|18|963.54|28.90|992.44|98.89|582.12|289.06|573.30|101.18|709.91| +2452125|36287|8715|96869|49887|2719|5504|4|9|2203|47|1876.71|75.06|1951.77|44.59|170.61|281.50|765.70|829.51|290.26| +2452169|30630|14171|96869|337940|4916|29092|7|31|2203|32|35.84|1.07|36.91|55.31|286.72|20.07|9.77|6.00|343.10| +2452544|48224|14007|18821|928325|1829|31733|10|18|2204|10|422.40|29.56|451.96|40.58|129.90|59.13|203.43|159.84|200.04| +2450995|46522|6584|10203|1120496|3768|8993|1|35|2205|23|306.13|12.24|318.37|79.16|328.90|73.47|160.53|72.13|420.30| +2451456|40636|5960|28179|534254|1330|42853|7|22|2206|36|1868.04|130.76|1998.80|68.45|339.48|1232.90|355.67|279.47|538.69| +2452295|60917|12205|63808|681775|594|39013|2|11|2207|29|162.98|9.77|172.75|65.78|978.17|29.33|45.44|88.21|1053.72| +2452273|48512|12925|37764|552470|4073|39276|4|29|2209|16|932.80|83.95|1016.75|80.32|664.00|289.16|186.65|456.99|828.27| +2451012|34523|3692|16768|1319799|6782|31765|1|33|2210|6|42.42|0.00|42.42|75.17|48.48|22.48|10.36|9.58|123.65| +2451299|56019|10021|37888|1070513|3431|35549|4|25|2213|81|11614.59|232.29|11846.88|45.97|6360.12|5807.29|5516.93|290.37|6638.38| +2452780|36564|12265|16166|1053039|7159|32479|10|32|2214|25|1032.50|82.60|1115.10|5.64|974.25|609.17|156.63|266.70|1062.49| +2451069|56895|16441|16039|1493396|563|36177|4|13|2215|10|277.90|13.89|291.79|84.93|34.10|116.71|161.19|0.00|132.92| +2451069|33651|15529|16039|1569534|1958|4735|2|29|2215|25|345.75|0.00|345.75|73.23|32.25|162.50|100.78|82.47|105.48| +2452400|53128|361|12761|1042995|1939|20483|8|28|2216|27|82.35|7.41|89.76|68.32|35.64|6.58|68.95|6.82|111.37| +2452069|54268|4461|72293|851665|5790|45962|1|7|2217|19|0.00|0.00|0.00|92.98|847.78|0.00|0.00|0.00|940.76| +2452030|59747|783|31646|24458|3919|5780|4|17|2217|1|65.03|3.25|68.28|27.55|10.40|6.50|28.67|29.86|41.20| +2451935|59720|7059|31646|812332|1099|17066|4|35|2217|92|6841.12|615.70|7456.82|4.69|2682.72|3283.73|853.77|2703.62|3303.11| +2452104|45732|6493|4994|1860474|3507|42940|10|18|2217|14|1437.38|114.99|1552.37|83.21|464.38|1149.90|281.73|5.75|662.58| +2452565|40853|14389|30140|337041|3967|34493|10|12|2218|3|0.90|0.02|0.92|48.65|3.96|0.22|0.08|0.60|52.63| +2452615|44397|1614|35673|218278|2130|1982|8|24|2218|62|4907.30|392.58|5299.88|55.10|923.18|2453.65|858.77|1594.88|1370.86| +2451586|42666|1762|81408|1657981|1807|17586|10|13|2219|22|852.06|25.56|877.62|9.89|181.50|17.04|467.61|367.41|216.95| +2451661|34321|4822|81408|1871160|1041|32122|7|32|2219|56|2031.12|182.80|2213.92|54.94|2476.88|589.02|1139.25|302.85|2714.62| +|53787|17161|||6288|||3|2219|||7.70|264.38|73.08|66.24||17.25||| +2451624|56093|15458|80995|640611|1474|3890|10|29|2219|21|2502.36|100.09|2602.45|61.02|1508.64|25.02|1288.21|1189.13|1669.75| +2452650|52228|9901|24611|438119|6980|4641|7|15|2220|10|440.40|26.42|466.82|44.44|235.20|158.54|281.86|0.00|306.06| +2451934|29694|17066|42162|704012|1553|33894|7|19|2221|13|236.21|21.25|257.46|44.17|76.05|89.75|96.66|49.80|141.47| +2452036|50066|3269|60379|465766|2894|46822|2|8|2224|13|243.75|12.18|255.93|79.05|194.09|17.06|181.35|45.34|285.32| +2451953|58580|14711|60379|1566881|4869|32904|4|17|2224|42|1191.96|71.51|1263.47|51.45|0.00|607.89|473.09|110.98|122.96| +2451949|34044|14000|92051|617441|1584|27774|10|27|2224|33|5418.93|162.56|5581.49|88.67|1246.08|3847.44|612.88|958.61|1497.31| +2451181|33795|4846|10229|1772452|1622|36792|8|24|2225|58|1020.22|0.00|1020.22|22.31|3001.50|40.80|754.15|225.27|3023.81| +2451286|54722|490|10229|1558695|282|17469|2|18|2225|8|7.12|0.64|7.76|81.10|119.68|2.27|1.79|3.06|201.42| +2451029|37078|2248|16000|1753546|286|26626|2|11|2227|3|55.74|2.22|57.96|33.97|157.44|15.04|14.24|26.46|193.63| +2451790|59558|3785|87754|517929|730|41131|10|8|2228|82|3093.04|185.58|3278.62|26.19|284.54|1082.56|743.87|1266.61|496.31| +2451206|42461|640|58202|921533|5664|25395|7|32|2229|1|14.54|0.87|15.41|87.18|2.19|13.37|0.38|0.79|90.24| +2452219|59630|255|35008|73408|3764|46567|1|29|2231|7|449.75|22.48|472.23|68.79|277.62|8.99|149.85|290.91|368.89| +2452047|61160|13544|20031|480081|484|24487|4|30|2232|38|0.00|0.00|0.00|43.37|93.48|0.00|0.00|0.00|136.85| +2451893|40387|7451|83151|1353657|4163|23469|1|31|2233|66|230.34|9.21|239.55|51.49|12.54|128.99|27.36|73.99|73.24| +||5761|||1925||||2234||2588.25|232.94|2821.19|||440.00||408.17|824.99| +2452679|60608|16207|26861|1609681|3718|5002|8|33|2235|10|247.70|4.95|252.65|16.13|484.70|91.64|156.06|0.00|505.78| +2452745|39008|2413|54328|1706739|1550|28808|8|27|2236|4|329.60|26.36|355.96|38.30|149.80|19.77|130.12|179.71|214.46| +2451221|29348|15763|33845|403321|4532|18896|7|9|2237|36|1435.68|14.35|1450.03|87.41|197.28|1363.89|58.86|12.93|299.04| +2451181|29905|2144|95142||4696|42927|7||2242||232.20||250.77|96.51|557.71|||101.27|| +2451097|53925|7717|13882|1646532|4321|33310|2|1|2243|10|348.10|27.84|375.94|32.90|72.60|257.59|32.58|57.93|133.34| +2451163|40241|73|13882|306505|4047|6036|2|2|2243|64|881.28|70.50|951.78|44.79|1939.20|255.57|312.85|312.86|2054.49| +2451984|58648|4499|67463|1868965|4075|9985|8|5|2244|26|352.56|24.67|377.23|69.09|107.12|232.68|68.33|51.55|200.88| +2452203|41650|5917|2949|512998|2081|20717|2|23|2245|1|27.84|1.39|29.23|36.21|5.33|13.36|5.64|8.84|42.93| +2452199|57875|15587|2949|163681|1886|10891|2|26|2245|54|4796.82|335.77|5132.59|49.03|2226.96|863.42|1809.36|2124.04|2611.76| +2452176|35048|1535|2949|445750|1992|43481|2|27|2245|5|485.55|24.27|509.82|32.52|4.90|407.86|56.71|20.98|61.69| +2452729|50826|7929|71843|1404099|4392|11485|1|19|2247|2|9.12|0.63|9.75|34.04|1.14|1.27|0.23|7.62|35.81| +2452715|37299|10578|74834|1210126|618|3510|10|14|2247|4|65.64|2.62|68.26|3.94|63.52|45.29|2.64|17.71|70.08| +2451206|58768|12637|96813|1799584|3155|4726|7|20|2250|10|526.10|10.52|536.62|42.54|50.40|199.91|159.83|166.36|103.46| +2452034|61021|15365|94095|381021|7147|2073|2|32|2252|25|129.75|10.38|140.13|2.78|433.25|106.39|9.11|14.25|446.41| +2451211|39948|16039|29926|1183429|3339|4449|8|17|2253|14|530.60|31.83|562.43|65.51|84.56|148.56|133.71|248.33|181.90| +2452636|34319|1051|92550|725768|1347|14173|2|16|2254|1|22.68|2.04|24.72|55.39|14.17|13.83|8.76|0.09|71.60| +2452665|37139|4161|92550|166651|319|23987|1|20|2254|74|1246.16|112.15|1358.31|85.54|920.56|348.92|511.42|385.82|1118.25| +2452663|49914|9991|92550|1042858|5625|13192|1|29|2254|37|4697.89|281.87|4979.76|58.96|939.43|0.00|4509.97|187.92|1280.26| +2451390|50437|13219|31206|54269|4385|42454|2|24|2259|22|506.22|40.49|546.71|64.68|105.60|116.43|85.75|304.04|210.77| +2451119|53719|10580|68959|1221831|3347|36910|4|24|2260|34|2218.50|22.18|2240.68|10.51|250.92|177.48|102.05|1938.97|283.61| +2451027|39170|17674|50643|276726|3371|40769|10|19|2261|17|483.48|14.50|497.98|89.15|333.88|29.00|149.97|304.51|437.53| +2452254|34409|13879|70711|249104|4426|7395|4|9|2262|6|20.16|0.20|20.36|73.46|70.62|7.45|6.35|6.36|144.28| +2452266|44684|16549|70711|776825|1144|16263|2|19|2262|6|301.98|15.09|317.07|71.93|100.62|39.25|233.82|28.91|187.64| +2451154|30701|10400|83468|1801396|6382|48921|4|18|2263|91|958.23|19.16|977.39|28.34|5752.11|335.38|548.10|74.75|5799.61| +2451151|35132|9394|83468|1276802|5958|39514|8|23|2263|1|6.15|0.30|6.45|16.93|3.84|3.32|1.52|1.31|21.07| +2452606|30641|16767|40249|607945|3994|44890|4|28|2264|13|767.78|30.71|798.49|74.35|392.99|744.74|14.51|8.53|498.05| +2451255|31640|13100|77373|645682|6592|45149|2|6|2265|15|464.40|41.79|506.19|3.83|123.30|199.69|45.00|219.71|168.92| +2452053|41811|12929|10248|1752272|656|18193|1|6|2266|4|419.56|33.56|453.12|2.05|161.68|188.80|147.68|83.08|197.29| +2451919|39326|4651|49809|1743805|4201|197|1|24|2266|48|1223.52|24.47|1247.99|50.35|694.56|1199.04|4.65|19.83|769.38| +2451850|38765|14359|10176|1820616|3227|36191|10|23|2267|61|586.82|23.47|610.29|2.11|2179.53|93.89|211.95|280.98|2205.11| +2451842|30838|1465|10176|1380317|6334|42304|8|11|2267|23|1099.63|10.99|1110.62|58.17|270.94|1011.65|81.82|6.16|340.10| +2451153|41454|3601|49176|1582279|182|13685|2|16|2268|52|365.56|29.24|394.80|45.32|91.00|186.43|21.49|157.64|165.56| +2451091|45050|6271|12837|616952|3820|48735|4|1|2268|55|3032.70|121.30|3154.00|34.94|1223.75|303.27|245.64|2483.79|1379.99| +2452679|47412|2529|12011|435132|3330|23975|7|5|2269|8|375.44|26.28|401.72|11.36|150.16|341.65|21.96|11.83|187.80| +||14489|30994|1900317|600||||2271|65|243.75|||2.75|365.30|221.81||11.85|| +||10159|30994||||1||2271|18||0.00||43.94|308.88|326.85|875.40||352.82| +2451709|39546|17753|30994|1414371|3467|40563|4|5|2271|78|414.96|4.14|419.10|86.64|903.24|195.03|4.39|215.54|994.02| +2451558|47225|3944|30994|908453|1028|15603|7|3|2271|1|20.89|1.04|21.93|50.84|33.13|15.87|0.15|4.87|85.01| +2451925|30479|17353|39688|210773|1058|40143|2|8|2275|6|146.40|13.17|159.57|84.66|130.14|13.17|41.30|91.93|227.97| +2452668|48906|9915|95774|21484|5594|19657|4|22|2276|4|315.28|18.91|334.19|63.78|176.56|63.05|143.77|108.46|259.25| +2452712|48757|13566|29460|1271021|6629|990|7|34|2276|36|707.40|35.37|742.77|54.19|88.20|367.84|95.07|244.49|177.76| +2452589|32872|8316|29460|322847|4139|6631|8|25|2276|18|783.54|54.84|838.38|93.21|572.04|454.45|75.69|253.40|720.09| +2451914|50905|12817|59722|1072284|2829|40621|7|13|2277|59|1003.59|70.25|1073.84|11.53|1756.43|391.40|226.51|385.68|1838.21| +|56662|5948|||5959|28149||32|2277|||0.03|3.09||6.40|||0.52|38.74| +2452398|49556|16329|85892|1402409|1679|26596|1|13|2278|2|67.14|0.00|67.14|32.17|18.80|4.02|28.40|34.72|50.97| +2452299|44367|7407|87273|754965|394|28623|7|9|2278|9|608.22|18.24|626.46|28.25|243.27|170.30|407.26|30.66|289.76| +2452096|57910|13347|41968|102027|2361|37048|2|24|2279|42|6011.46|0.00|6011.46|83.48|3699.36|1142.17|48.69|4820.60|3782.84| +2451975|47423|4369|41968|509861|2137||||2279|||||||1112.37|||| +2452079|45001|8215|41968|704706|1548|27143|8|7|2279|27|236.79|4.73|241.52|94.92|189.54|227.31|5.97|3.51|289.19| +2452034|38469|2435|41968|1414340|520|11453|4|33|2279|80|1153.60|80.75|1234.35|46.76|569.60|1142.06|9.11|2.43|697.11| +2451991|43060|5747|15583|1734628|873|15165|7|10|2279|40|447.60|13.42|461.02|49.25|208.80|219.32|196.32|31.96|271.47| +2451118|52487|1789|68701|1527416|4978|24493|7|35|2280|35|268.10|16.08|284.18|55.25|268.10|75.06|5.79|187.25|339.43| +|50287|15671||400936|1686||||2281||||68.29|||46.20|10.17||71.25| +2451717|54978|8269|35672|1613177|3012|6847|7|17|2281|4|130.68|11.76|142.44|18.59|7.12|7.84|62.64|60.20|37.47| +|40836|11242|90419||4582|24047|||2284|66|||6142.95|22.06|1972.08|5297.59|263.75|74.40|| +2451651|61124|16874|91454|1277936|5571|28853|4|22|2284|20|20.20|0.00|20.20|32.33|22.40|7.07|0.52|12.61|54.73| +2452325|52674|1671|18591|326589|3836|6236|1|33|2285|47|1690.59|16.90|1707.49|97.83|684.32|1217.22|359.76|113.61|799.05| +2452437|32560|5227|18591|1750064|566|28268|7|22|2285|32|1949.44|0.00|1949.44|2.24|159.04|1189.15|334.52|425.77|161.28| +2451036|46927|15884|73957|322136|715|45694|2|25|2286|13|342.42|23.96|366.38|24.00|92.17|291.05|20.54|30.83|140.13| +2450956|53332|11512|11557||||1|33|2286|6|432.66|17.30|449.96||44.10||159.22||| +2451013|47758|17768|11557|648626|6962|12498|1|13|2286|21|126.21|11.35|137.56|45.98|277.62|73.20|31.80|21.21|334.95| +2451197|51923|346|64151|934531|4553|34175|1|24|2287|54|1531.98|122.55|1654.53|53.32|2106.54|827.26|697.67|7.05|2282.41| +2452608||14688||1814353|||||2289||91.92||98.35|17.55||89.16||0.14|88.78| +2452485|49105|1809|41381|1033648|5115|16982|8|16|2289|1|87.79|7.90|95.69|96.11|1.09|53.55|5.82|28.42|105.10| +2451369|46516|4375|3820|615537|24|44911|8|13|2290|3|10.53|0.10|10.63|49.98|2.55|8.42|1.26|0.85|52.63| +2451662|48643|2695|46594|981522|273|45424|10|8|2292|44|744.48|59.55|804.03|60.84|388.52|394.57|311.41|38.50|508.91| +2452651|40591|11517|39620|1378574|4061|27195|2|4|2293|5|146.75|4.40|151.15|79.50|38.90|88.05|53.41|5.29|122.80| +2452756|46882|2244|39620|1544118|4820|11522|2|16|2293|13|300.43|24.03|324.46|36.47|193.05|162.23|2.76|135.44|253.55| +2452243|49876|3865|15296|1635044|4392|40940|7|6|2294|25|1221.75|109.95|1331.70|68.33|319.50|1001.83|206.72|13.20|497.78| +2451548|34291|2014|80960|514741|6922|35056|4|3|2295|1|73.55|4.41|77.96|32.82|18.57|67.66|3.53|2.36|55.80| +2451010|41758|10567|92166|80978|3284|46837|4|21|2296|8|23.20|1.16|24.36|1.90|26.72|19.25|2.37|1.58|29.78| +2451934|51535|1543|24014|1404571|1900|41676|4|4|2297|14|201.88|2.01|203.89|95.04|60.48|199.86|1.91|0.11|157.53| +2452543|56745|6585|43914|143994|1943|18739|10|32|2298|32|3667.52|220.05|3887.57|53.46|2210.56|1246.95|1307.10|1113.47|2484.07| +2452414|31731|4089|43914|100333|1442|32551|7|4|2298|1|115.77|0.00|115.77|62.45|0.00|2.31|17.01|96.45|62.45| +2452454|31095|6063|43914|501321|4640|14312|8|33|2298|10|605.00|18.15|623.15|45.35|302.50|550.55|36.48|17.97|366.00| +2451093|49554|17462|53069|381444|4398|34905|4|18|2300|13|22.62|0.67|23.29|37.39|1.30|21.26|0.85|0.51|39.36| +|57328|16244||548746|||8|29|2300|63|||728.02|12.48|||6.06|297.29|1097.71| +2451087|34079|17500|53069|442860|4540|38667|1|22|2300|41|395.65|7.91|403.56|23.27|395.65|71.21|301.72|22.72|426.83| +2452360|61051|15579|94032|239435|3238|7841|7|9|2301|33|700.59|35.02|735.61|8.93|213.18|294.24|394.15|12.20|257.13| +2452306|53563|11975|93552|901778|2998|25771|2|24|2302|39|1099.41|21.98|1121.39|29.44|714.48|923.50|172.39|3.52|765.90| +2452332|53783|15089|93552|1338752|1126|7100|8|21|2302|43|1559.61|124.76|1684.37|45.03|2366.29|1232.09|134.28|193.24|2536.08| +2452334|61151|13791|93552|472175|5628|26281|7|32|2302|52|3456.44|172.82|3629.26|66.92|1108.64|2488.63|958.13|9.68|1348.38| +|41271|10318||1626121|6817|23113|8|15|2303||2486.00|||91.96|705.76|198.88||1509.50|| +2451616|36837|11074|12958|310839|4367|10648|7|32|2303|69|590.64|11.81|602.45|80.41|247.02|283.50|110.57|196.57|339.24| +2451541|47323|2833|91248|833110|4521|33979|2|11|2303|13|266.63|2.66|269.29|49.36|246.22|175.97|64.36|26.30|298.24| +|47702|17959|||4919||4||2305||||24.25||0.72||||| +2451436|49483|14440|2942|5750|5594|33838|10|31|2306|34|430.44|0.00|430.44|21.33|528.36|417.52|9.56|3.36|549.69| +2451435|37403|2227|2942|391967|5054|48094|8|16|2306|5|217.55|2.17|219.72|32.12|35.55|115.30|20.45|81.80|69.84| +2451477|28869|2612|2942|1191534|4303|15072|8|25|2306|84|3365.04|33.65|3398.69|26.23|2563.68|807.60|281.31|2276.13|2623.56| +2452634|37417|2502|1373|399633|4036|41592|8|23|2307|22|1388.64|111.09|1499.73|77.27|245.74|694.32|256.89|437.43|434.10| +2452687|41464|5899|79466|311085|6257|7476|7|27|2307|17|779.96|70.19|850.15|46.60|437.41|522.57|18.01|239.38|554.20| +2452039|53084|3313|35200|615680|3713|36946|1|19|2308|61|1115.08|22.30|1137.38|88.52|477.63|624.44|284.57|206.07|588.45| +2451252|46456|5056|37876|1450681|209|32137|1|33|2311|2|16.16|0.48|16.64|67.23|67.02|13.73|1.50|0.93|134.73| +2451311|49393|15199|37876|1335322|4570|21818|1|22|2311|4|62.40|3.12|65.52|26.87|19.96|33.69|14.92|13.79|49.95| +2451203|57232|16262|81855|1131437|2619|42648|2|7|2311|4|159.68|14.37|174.05|29.91|45.84|158.08|1.12|0.48|90.12| +2451183|56481|10172|50639|1852529|6215|48254|1|17|2312|19|46.74|0.93|47.67|93.78|117.04|35.52|9.31|1.91|211.75| +2451049|30634|2884|14357|1891474|5038|33355|1|6|2312|4|4.08|0.04|4.12|46.84|22.76|1.18|2.26|0.64|69.64| +2451086|42611|13064|50639|950962|1304|5293|10|4|2312|19|650.75|39.04|689.79|13.65|414.20|240.77|118.89|291.09|466.89| +2451939|32779|10154|41646|1906968|574|30627|2|11|2313|48|1280.64|0.00|1280.64|17.48|227.52|934.86|207.46|138.32|245.00| +2451879|46594|16256|45921|1544181|841|25454|7|24|2313|64|4765.44|381.23|5146.67|77.37|2977.92|4193.58|245.89|325.97|3436.52| +2451971|42256|14401|41646|451308|3210|14712|8|3|2313|11|785.62|7.85|793.47|66.87|403.37|54.99|306.86|423.77|478.09| +2451628|61068|14596|18619|96389|5325|14661|10|20|2315|10|129.90|10.39|140.29|11.88|21.60|41.56|37.98|50.36|43.87| +2451969|54282|3661|84516|1638955|2539|1723|2|9|2316|1|27.74|0.55|28.29|1.23|36.98|4.99|16.38|6.37|38.76| +2452313|31098|13507|10619|173052|6632|11812|10|32|2317|6|336.00|13.44|349.44|51.82|309.66|225.12|105.33|5.55|374.92| +|34477|7017|10619|523238||||11|2317|2|6.18||||7.74|||1.56|| +2452406|56039|10567|10619|188604|2134|26464|4|17|2317|4|399.36|11.98|411.34|98.95|19.96|315.49|3.35|80.52|130.89| +2452042|53747|5185|92119|253647|5370|4693|7|2|2318|6|288.66|14.43|303.09|52.52|121.14|112.57|105.65|70.44|188.09| +2452012|41462|12020|92119|1598708|2100|25092|2|6|2318|31|684.79|34.23|719.02|31.34|355.88|445.11|148.60|91.08|421.45| +2452115||5459|||3948||4||2319||5195.84|51.95||98.71|1015.04|||1673.07|1165.70| +2451414|34998|14996|69317|753668|772|33876|1|21|2320|21|797.16|31.88|829.04|70.19|288.75|757.30|16.34|23.52|390.82| +2451383|52458|1972|12470|585721|7148|28528|1|20|2320|50|1544.50|123.56|1668.06|17.87|1897.50|339.79|855.34|349.37|2038.93| +2452405|35176|1875|79346|21109|6917|40725|8|6|2321|1|3.35|0.23|3.58|36.30|0.69|2.54|0.21|0.60|37.22| +2452396|55691|11345|79346|1344504|4703|42455|10|16|2321|12|691.68|48.41|740.09|10.51|225.84|657.09|14.18|20.41|284.76| +2451933|45012|2024|37795|618137|6289|2041|2|17|2322|26|429.26|17.17|446.43|12.27|69.16|81.55|208.62|139.09|98.60| +2452164|39382|14993|13569|336845|6038|5631|8|29|2323|8|24.48|0.00|24.48|97.34|12.24|21.78|2.16|0.54|109.58| +2451899|34781|277|49481|1453784|5233|35482|7|2|2324|1|24.38|0.97|25.35|50.63|21.86|3.41|13.42|7.55|73.46| +2452500|50377|7831|22796|1593673|6287|8054|7|25|2325|18|2488.14|223.93|2712.07|8.87|630.72|298.57|1948.71|240.86|863.52| +2452422|29848|12792|22796|1525415|2633|1804|1|1|2325|8|131.12|7.86|138.98|66.37|73.12|89.16|5.87|36.09|147.35| +2452422|39609|5515|22796|1556383|2800|4817|2|11|2325|4|155.40|7.77|163.17|28.88|118.68|138.30|11.45|5.65|155.33| +||6872|8967|||||19|2326||469.80|||91.86||14.09|||300.78| +2451224|39445|10756|27489|923049|5809|40169|8|6|2327|1|9.83|0.68|10.51|64.34|7.65|1.37|7.78|0.68|72.67| +2451444|32932|17896|99900|1072275|2756|38837|1|16|2329|9|36.54|0.73|37.27|34.92|250.65|34.34|2.06|0.14|286.30| +2452589|42618|5202|69455|748455|3217|33932|2|1|2331|47|5061.90|455.57|5517.47|54.40|0.00|3037.14|1741.29|283.47|509.97| +2451215|55253|16024|25987|881142|2745|11995|10|28|2332|12|871.56|8.71|880.27|44.76|79.20|540.36|195.40|135.80|132.67| +2451253|42625|7954|25987|12376|1416|14456|8|14|2332|45|2254.05|67.62|2321.67|76.99|229.95|202.86|840.98|1210.21|374.56| +2451625|55158|11132|54086|1614028|3219|22457|10|14|2335|21|17.43|0.87|18.30|67.70|5.67|4.70|9.80|2.93|74.24| +2452079|38485|1310|23595|61585|2710|27035|7|15|2336|22|533.28|15.99|549.27|49.54|253.88|58.66|180.35|294.27|319.41| +2452690|47101|7431|60581|739011|643|4275|1|24|2337|36|374.76|22.48|397.24|88.96|468.36|153.65|126.03|95.08|579.80| +||9079|26541||3759|19609|2||2338|2|9.88|0.19|10.07|90.51|5.12|9.18|0.45|0.25|95.82| +2452028|55181|2437|26541|1279951|3718|39932|8|1|2338|42|33.60|1.34|34.94|55.88|11.76|1.68|30.32|1.60|68.98| +2451832|51638|17189|65198|514847|4496|8945|7|13|2343|15|977.25|0.00|977.25|87.47|476.85|928.38|42.02|6.85|564.32| +2451992|36719|3025|97443|487734|399|664|1|22|2344|74|0.00|0.00|0.00|67.97|1868.50|0.00|0.00|0.00|1936.47| +2451991|40217|812|97443|1357138|4741|3991|10|34|2344|21|2627.10|0.00|2627.10|40.37|272.79|2049.13|531.73|46.24|313.16| +2452294|49200|14941|4362|553107|6541|18375|2|21|2345|76|1007.76|30.23|1037.99|45.84|604.20|564.34|341.43|101.99|680.27| +2451841|44140|5840|5467|660389|3311|28233|4|12|2347|15|674.55|60.70|735.25|85.39|256.50|586.85|87.70|0.00|402.59| +2451757|55177|314|48661|1091811|5231|45804|8|5|2347|55|6845.85|136.91|6982.76|58.25|1831.50|342.29|325.17|6178.39|2026.66| +2452702|50511|14433|51174|1888732|6844|10556|1|13|2348|29|1251.35|75.08|1326.43|17.56|375.26|913.48|135.14|202.73|467.90| +2452108|56672|16787|26025|1035318|7178|27085|4|7|2349|13|54.73|3.28|58.01|79.78|168.74|48.70|0.24|5.79|251.80| +||4851|26025|409443|6371|44546|8|31|2349||||85.60|40.48|48.60|3.42||16.44|89.08| +2451283|55059|3284|92181|1579822|2125|9675|2|23|2350|2|157.44|11.02|168.46|6.83|95.84|69.27|31.74|56.43|113.69| +2451270|49459|14494|92181|1194138|4391|15044|7|8|2350|43|2128.07|63.84|2191.91|55.70|294.12|42.56|1272.16|813.35|413.66| +2451972|37083|1268|47150|1466622|1347|13625|10|8|2351|5|365.50|29.24|394.74|27.55|101.25|47.51|181.25|136.74|158.04| +2451938|37654|13610||1494933|3765||2||2351|||4.91|66.35|||44.23||5.17|| +2452367|52225|17735||||47266||16|2352|||1.39|||||91.20||| +2452255|31888|17175|11483|709564|3286|37455|2|28|2352|43|1830.51|128.13|1958.64|12.58|1296.45|1519.32|245.84|65.35|1437.16| +2452381|55806|16737|11483|42107|1452|27550|1|25|2352|5|140.95|9.86|150.81|45.05|207.30|104.30|16.85|19.80|262.21| +2451164|50728|15808|98838|925693|6527|44608|7|23|2353|38|2407.30|72.21|2479.51|4.80|1543.18|1612.89|587.86|206.55|1620.19| +2451346|53801|17107|64675|1700373|4484|14642|10|16|2354|78|3806.40|228.38|4034.78|67.42|3452.28|456.76|602.93|2746.71|3748.08| +2451396|29123|1490|64675|1020742|2947|40352|4|34|2354|32|1191.36|35.74|1227.10|83.40|96.96|142.96|262.10|786.30|216.10| +2451511||8167||668010|2819||2||2358||||48.45|||44.68|0.80|1.56|498.44| +|57019|5830||1375323||19273|4||2358|70|6210.40||6520.92|86.32||3105.20|||2910.54| +2450868|47248|5746|19990|1684488|4516|20591|7|21|2359|42|10.08|0.50|10.58|29.85|161.28|6.85|3.06|0.17|191.63| +2450938|44068|7786|19990|1658397|710|10576|1|32|2359|26|480.22|38.41|518.63|30.06|640.38|302.53|8.88|168.81|708.85| +2452145|47403|2193|54973|907467|3961|27255|7|6|2360|61|0.00|0.00|0.00|7.70|488.61|0.00|0.00|0.00|496.31| +2451971|43130|4181|86656|1512427|2949|24318|10|32|2361|23|2053.21|41.06|2094.27|24.12|279.91|1745.22|255.63|52.36|345.09| +2451934|35158|2969|86656|77956|5201|21646|7|18|2361|11|911.46|45.57|957.03|20.15|415.14|464.84|89.32|357.30|480.86| +2452151|40937|4741|16915|388312|3523|14334|4|6|2363|3|51.87|4.14|56.01|69.64|51.87|19.19|8.82|23.86|125.65| +2452166|51898|13439|16915|1719344|4573|12955|1|19|2363|20|590.00|29.50|619.50|15.75|184.80|141.60|98.64|349.76|230.05| +2452123|61097|5927|16915|998936|2218|34347|1|8|2363|49|693.84|6.93|700.77|29.93|796.74|541.19|99.22|53.43|833.60| +2452398|43063|5339|75165|1180738|4801|20464|8|35|2364|33|926.64|0.00|926.64|1.24|681.12|250.19|426.16|250.29|682.36| +2452521|51506|2503|56017|1437602|6707|26212|10|4|2365|29|78.01|0.00|78.01|11.88|38.86|12.48|46.52|19.01|50.74| +2451596|29229|6074|47011|31557|2856|6202|7|7|2367|8|65.36|3.92|69.28|64.66|122.56|5.22|15.03|45.11|191.14| +2451682|60517|14608|51161|966061|3641|28821|8|11|2367|45|121.50|10.93|132.43|72.88|0.00|69.25|44.41|7.84|83.81| +2451260|36493|14890|10886|667801|993|46585|8|9|2368|76|7900.20|553.01|8453.21|51.43|5563.96|7110.18|584.61|205.41|6168.40| +2451497|33343|11164|44792|674862|3060|22905|1|26|2369|17|1088.34|97.95|1186.29|86.45|161.16|968.62|86.19|33.53|345.56| +2451632|61139|14006|3121|1906520|4160|23847|4|34|2371|12|150.00|4.50|154.50|40.46|203.64|118.50|20.79|10.71|248.60| +2452607|54547|13287|83186|961539|6098|29904|4|14|2372|44|29.04|0.87|29.91|42.47|17.60|3.77|20.46|4.81|60.94| +2451915|44740|11903|17498|932762|744|5926|7|15|2373|18|864.72|8.64|873.36|88.31|563.22|43.23|681.83|139.66|660.17| +2451885|36273|17785|17498|218757|1939|3921|4|35|2373|26|816.66|57.16|873.82|85.17|840.06|792.16|22.54|1.96|982.39| +2451197|38397|2546|47779|390846|5408|26723|8|12|2374|19|279.49|19.56|299.05|8.61|106.21|164.89|32.08|82.52|134.38| +2451925|29258|4508|4616|657462|5135|35208|8|19|2375|10|350.80|21.04|371.84|34.21|161.30|38.58|249.77|62.45|216.55| +2451074|54610|4840|28055|1015452|2267|19129|1|8|2376|23|1014.99|71.04|1086.03|33.27|906.89|883.04|113.47|18.48|1011.20| +2451030|30558|4496|28055|1894883|4719|18908|10|33|2376|5|14.00|0.56|14.56|57.80|12.45|3.50|6.19|4.31|70.81| +2452074|51999|14165|74498|353619|4087|41308|2|19|2377|25|340.25|10.20|350.45|2.57|328.50|190.54|35.93|113.78|341.27| +2452066|36557|15701|74498|714339|||||2377|41|842.14|8.42|850.56||329.64||476.23||| +2452349|33058|8081|91474|1024594|5708|40022|7|30|2378|23|1575.73|94.54|1670.27|11.43|1533.18|1244.82|324.29|6.62|1639.15| +2450839|35421|403|67276|1522064|4727|25037|7|35|2379|57|644.10|19.32|663.42|47.25|0.00|347.81|14.81|281.48|66.57| +2452298|33413|9701|38014|500476|4373|32670|1|29|2380|33|108.24|3.24|111.48|86.83|433.95|69.27|24.16|14.81|524.02| +2452286|45717|10513|14440|1724861|3469|22477|1|28|2385|18|504.72|35.33|540.05|91.94|55.98|413.87|56.32|34.53|183.25| +2452302|55015|1769|14440|364508|6998|41488|7|30|2385|15|453.90|36.31|490.21|89.31|790.05|177.02|171.66|105.22|915.67| +2452745|39021|1269|84560|946515|458|28401|2|7|2386|17|586.67|29.33|616.00|51.09|695.30|357.86|77.79|151.02|775.72| +2452662|52473|13362|84560|1205355|1864|32773|1|29|2386|8|76.08|4.56|80.64|73.20|50.72|19.02|35.37|21.69|128.48| +2452752|51001|16297|84560|1798305|655|45758|7|16|2386|34|2220.88|22.20|2243.08|48.66|82.96|1887.74|13.32|319.82|153.82| +2452730|31178|16482|84560|1475858|4140|37669|7|15|2386|8|99.76|3.99|103.75|65.78|61.92|50.87|18.57|30.32|131.69| +2451980|49672|5207|45743|298290|3617|36544|7|21|2387|1|35.66|2.13|37.79|27.94|16.64|34.94|0.65|0.07|46.71| +2450929|52841|13825|41860|1413357|7009|41014|10|33|2389|29|598.85|47.90|646.75|55.00|967.73|413.20|90.96|94.69|1070.63| +2450938|31754|11966|41860|866551|6831|40711|1|6|2389|56|1221.36|0.00|1221.36|46.67|314.72|317.55|533.24|370.57|361.39| +2450893|37705|6082|41860|653951|2243|43735|8|19|2389|42|1859.76|74.39|1934.15|64.47|753.06|1525.00|167.38|167.38|891.92| +|32442|5312||263417|590||||2390|||||4.50|307.50||183.47||332.96| +2451145|36849|8714|12152|555063|3951|2439|10|32|2390|17|15.98|1.27|17.25|25.00|58.82|15.18|0.73|0.07|85.09| +2451125|41479|17036|12152|1599385|5213|35841|7|29|2390|4|120.96|6.04|127.00|81.50|48.36|42.33|20.44|58.19|135.90| +2451142|28925|14170|12152|1605993|6112|30631|4|6|2390|53|33.92|0.67|34.59|44.95|322.77|7.12|5.36|21.44|368.39| +2451963|40488|11111|41489|520666|5443|8667|7|16|2391|63|1432.62|85.95|1518.57|18.97|1522.08|902.55|95.41|434.66|1627.00| +2451984|54182|7015|24321|1165421|4008|41842|8|18|2391|27|1332.45|53.29|1385.74|52.01|1300.59|1319.12|5.99|7.34|1405.89| +2451604|41046|10898|50293|950788|1523|3105|1|22|2393|15|35.40|2.83|38.23|54.07|17.70|15.22|13.11|7.07|74.60| +2451755|55758|10742|72943|583978|386|23994|7|15|2394|64|3189.12|31.89|3221.01|81.04|1358.08|829.17|778.78|1581.17|1471.01| +2452026|50538|17827|34028|1615305|2931|40510|4|28|2395|2|61.68|0.61|62.29|73.53|74.90|54.27|1.77|5.64|149.04| +2452094|35086|13169|34028|367654|7024|1994|10|19|2395|42|472.92|28.37|501.29|34.73|1310.40|137.14|154.45|181.33|1373.50| +2451712|41070|14773|5595|738908|2812|18316|10|19|2396|90|773.10|54.11|827.21|92.78|662.40|309.24|125.24|338.62|809.29| +2451076|58430|14246|99252|1520975|1098|40421|2|7|2397|9|115.11|8.05|123.16|63.78|309.96|95.54|2.15|17.42|381.79| +2451535|57375|15734|92740|144606|5990|1421|8|9|2398|1|1.34|0.02|1.36|18.02|13.75|1.07|0.18|0.09|31.79| +2452178|42436|13575|92742|194671|5787|43256|4|1|2399|16|316.80|6.33|323.13|1.81|671.04|275.61|20.59|20.60|679.18| +2452295|58947|10849|92742|1415131|2238|33345|10|27|2399|17|151.81|4.55|156.36|66.93|1184.90|148.77|1.67|1.37|1256.38| +|38648|7045|92742|1798468|||4||2399|||1.54||||129.86||15.34|377.18| +2451942|32277|17741|3499|1666760|5971|28382|8|9|2400|26|1348.36|0.00|1348.36|19.64|263.64|647.21|560.92|140.23|283.28| +2452333|40222|11309|32252|1027899|1822|27298|4|27|2401|11|49.94|1.99|51.93|16.55|87.56|10.98|25.32|13.64|106.10| +2452167|43085|9021|75849|828022|1795|768|1|19|2404|16|213.60|8.54|222.14|34.61|336.80|96.12|9.39|108.09|379.95| +2452271|33179|16129|33260|1266048|484|4120|7|22|2405|67|974.18|29.22|1003.40|53.06|974.18|555.28|125.67|293.23|1056.46| +2452303|40877|17757|33260|1791957|3065|1420|4|15|2405|26|261.82|20.94|282.76|49.44|46.54|261.82|0.00|0.00|116.92| +2451443|32383|16585|7610|669426|2135|12844|1|17|2406|3|45.93|1.83|47.76|43.43|12.48|11.48|31.00|3.45|57.74| +2451556|58249|4946|58426|724677|81|285|2|35|2406|23|1878.87|75.15|1954.02|80.79|184.69|1277.63|48.09|553.15|340.63| +2452366|35265|16145|69576|1832612|3648|47084|2|26|2407|11|262.35|5.24|267.59|27.53|146.74|217.75|32.11|12.49|179.51| +2452350|44432|6921|95298|1009287|3973|43812|2|29|2407|31|642.01|44.94|686.95|46.76|404.86|359.52|135.59|146.90|496.56| +2452516|55987|943|83528|857581|7162|9990|7|29|2408|33|1704.45|68.17|1772.62|68.46|1080.09|306.80|307.48|1090.17|1216.72| +2452459|44964|6019|83528|236990|2603|15863|10|21|2408|11|213.84|0.00|213.84|85.13|173.25|59.87|80.06|73.91|258.38| +2452669|57587|14941|17059|1296914|3000|13839|10|19|2410|11|609.18|30.45|639.63|51.00|62.70|292.40|310.44|6.34|144.15| +|50217|14485|17059|1473956||||6|2410|6|131.82||||79.08|||20.92|173.49| +2451668|29106|6403|38909|1052261|4071|5081|1|31|2411|1|6.31|0.31|6.62|63.43|1.26|3.84|2.32|0.15|65.00| +||15398|||3035||4|30|2411||||96.12||62.72|57.85|||| +2452012|37937|1373|87704|405235|1681|15680|10|27|2412|24|42.48|1.69|44.17|17.07|15.84|19.11|5.60|17.77|34.60| +2452006|34337|14569|87704|451444|7148|8052|7|35|2412|45|4905.00|196.20|5101.20|55.83|1188.90|4071.15|191.78|642.07|1440.93| +2451274|59654|8026|5052|1848541|4477|23966|1|34|2413|9|641.61|19.24|660.85|88.72|13.32|417.04|98.81|125.76|121.28| +2451358|31640|4249|42472|516784|5547|32263|2|29|2413|8|485.28|14.55|499.83|75.73|188.96|262.05|165.19|58.04|279.24| +2452478|32900|13929|5586|1242877|5583|9889|4|16|2415|1|37.95|2.65|40.60|80.74|3.55|28.46|1.51|7.98|86.94| +2452523|60164|8227|5586|1078489|3400|27464|7|34|2415|52|4369.56|131.08|4500.64|88.47|2383.16|3583.03|298.88|487.65|2602.71| +2452049|47608|11132|80612|688082|6550|2218|4|11|2416|27|24.03|1.68|25.71|80.57|1013.04|18.74|3.80|1.49|1095.29| +2451973|58804|4525|58390|1283915|1440|41765|2|15|2416|16|396.48|23.78|420.26|73.64|381.12|340.97|41.63|13.88|478.54| +2452433|40198|2901|77566|632669|4925|38781|2|30|2417|2|87.06|5.22|92.28|51.91|73.06|54.84|25.77|6.45|130.19| +2451335|50755|3676|83171|1459211|2104|14644|1|35|2418|4|284.20|19.89|304.09|53.95|103.84|45.47|176.66|62.07|177.68| +2451220|48283|5840|91567|1225671|5375|11879|8|7|2418|31|686.03|61.74|747.77|38.48|439.58|452.77|95.63|137.63|539.80| +2451233|43740|13334|33456|1172438|7067|29696|7|32|2418|62|35.34|0.70|36.04|38.39|16.74|4.94|12.76|17.64|55.83| +2452066|59528|10385|72267|1193878|4642|6253|4|3|2419|49|1867.88|56.03|1923.91|91.40|247.45|448.29|865.94|553.65|394.88| +2452277|51181|9631|9280|620581|1861|30788|8|2|2421|20|453.00|31.71|484.71|38.00|67.00|0.00|45.30|407.70|136.71| +2452720|39403|5166|99280|516048|3275|11749|4|10|2423|33|374.22|11.22|385.44|12.22|118.14|127.23|123.49|123.50|141.58| +2452625|46922|15972|36643|899864|106|22748|7|27|2424|18|347.04|10.41|357.45|35.13|127.80|319.27|9.44|18.33|173.34| +2452599|35690|13662|36643|977945|2592|6886|7|12|2424|36|723.24|36.16|759.40|17.79|964.44|491.80|166.63|64.81|1018.39| +2452708|40822|10764|36643|633179|3673|12120|4|24|2424|5|172.75|1.72|174.47|63.93|82.25|81.19|85.15|6.41|147.90| +2452668|34806|5965|35893|500825|5757|7889|1|33|2425|31|1064.23|74.49|1138.72|50.75|1019.90|489.54|321.82|252.87|1145.14| +|39699|6011|||3213|||22|2427|12|227.64|||37.66||184.38|9.08||| +2452406|48903|7079|36901|635994|219|25691|2|16|2428|55|6612.65|264.50|6877.15|49.40|905.85|3636.95|952.22|2023.48|1219.75| +2452611|30074|12750|87557|1318923|1370|40617|2|11|2430|21|2510.76|0.00|2510.76|81.52|423.15|1933.28|248.31|329.17|504.67| +2451500||16880|||1198||||2431||450.50|4.50|455.00||106.95|418.96|17.34|14.20|139.52| +2451541|51657|13580|72220|17243|5181|31472|10|26|2431|55|1130.25|11.30|1141.55|37.96|414.15|632.94|372.98|124.33|463.41| +2451122|37056|15200|37903|333402|687|18772|4|32|2432|60|571.20|5.71|576.91|56.51|589.20|422.68|103.96|44.56|651.42| +2451547|31652|5936|10809|1581463|2870|11924|4|25|2434|72|2330.64|186.45|2517.09|5.22|322.56|209.75|487.80|1633.09|514.23| +2452009|49733|6092|63101|580757|1166|48422|10|4|2435|26|591.76|0.00|591.76|40.90|411.06|355.05|189.36|47.35|451.96| +2451586|49916|9248|95417|1775286|1108|30577|8|3|2436|29|390.34|35.13|425.47|87.37|260.13|19.51|281.83|89.00|382.63| +2451552|38583|14950|29456|1661004|756|20908|2|10|2436|70|4178.30|376.04|4554.34|81.01|373.80|2883.02|181.33|1113.95|830.85| +2451966|56949|5355|56147|240442|1205|33921|1|23|2437|26|1250.08|75.00|1325.08|68.26|117.00|237.51|222.76|789.81|260.26| +2451062|50694|16642|44334|108993|570|23747|4|24|2439|15|603.90|54.35|658.25|25.02|263.25|24.15|446.40|133.35|342.62| +2451867|58357|12311|18690|883447|6604|10914|1|8|2440|55|2582.80|77.48|2660.28|61.83|1417.35|594.04|1312.58|676.18|1556.66| diff --git a/spark-connector/spark-common/src/test/resources/data/tpcds/store_sales/store_sales.dat b/spark-connector/spark-common/src/test/resources/data/tpcds/store_sales/store_sales.dat new file mode 100644 index 00000000000..331e48cee6d --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/data/tpcds/store_sales/store_sales.dat @@ -0,0 +1,28804 @@ +2451813|65495|3617|67006|591617|3428|24839|10|161|1|79|11.41|18.71|2.80|99.54|221.20|901.39|1478.09|6.08|99.54|121.66|127.74|-779.73| +2451813|65495|13283|67006|591617|3428|24839|10|154|1|37|63.63|101.17|41.47|46.03|1534.39|2354.31|3743.29|59.53|46.03|1488.36|1547.89|-865.95| +2451813|65495|13631|67006|591617|3428|24839|10|172|1|99|80.52|137.68|83.98|0.00|8314.02|7971.48|13630.32|0.00|0.00|8314.02|8314.02|342.54| +2451813|65495|5981|67006|591617|3428|24839|10|280|1|14|57.37|76.30|6.10|0.00|85.40|803.18|1068.20|0.00|0.00|85.40|85.40|-717.78| +2451813|65495|4553|67006|591617|3428|24839|10|236|1|100|25.08|36.86|0.73|0.00|73.00|2508.00|3686.00|6.57|0.00|73.00|79.57|-2435.00| +2451813|65495|10993|67006|591617|3428|24839|10|263|1|91|93.48|108.43|93.24|0.00|8484.84|8506.68|9867.13|254.54|0.00|8484.84|8739.38|-21.84| +2451813|65495|49|67006|591617|3428|24839|10|70|1|5|10.68|15.91|6.68|0.00|33.40|53.40|79.55|2.33|0.00|33.40|35.73|-20.00| +2451813|65495|4583|67006|591617|3428|24839|10|267|1|72|84.72|111.83|61.50|0.00|4428.00|6099.84|8051.76|177.12|0.00|4428.00|4605.12|-1671.84| +2451813|65495|13538|67006|591617|3428|24839|10|106|1|14|11.54|11.77|0.00|0.00|0.00|161.56|164.78|0.00|0.00|0.00|0.00|-161.56| +2451813|65495|3248|67006|591617|3428|24839|10|189|1|58|4.57|5.34|3.52|0.00|204.16|265.06|309.72|0.00|0.00|204.16|204.16|-60.90| +2451813|65495|4682|67006|591617|3428|24839|10|7|1|44|31.07|54.99|24.19|0.00|1064.36|1367.08|2419.56|85.14|0.00|1064.36|1149.50|-302.72| +2451813|65495|5527|67006|591617|3428|24839|10|112|1|88|52.41|63.94|49.23|0.00|4332.24|4612.08|5626.72|259.93|0.00|4332.24|4592.17|-279.84| +2451813|65495|1553|67006|591617|3428|24839|10|242|1|50|67.71|126.61|87.36|0.00|4368.00|3385.50|6330.50|349.44|0.00|4368.00|4717.44|982.50| +2451813|65495|173|67006|591617|3428|24839|10|249|1|65|27.16|52.14|41.19|0.00|2677.35|1765.40|3389.10|214.18|0.00|2677.35|2891.53|911.95| +2451524|75316|6928|68284|1712478|4672|13266|10|37|2|65|93.86|158.62|34.89|0.00|2267.85|6100.90|10310.30|45.35|0.00|2267.85|2313.20|-3833.05| +2451524|75316|13927|68284|1712478|4672|13266|10|131|2|93|35.87|49.14|15.23|0.00|1416.39|3335.91|4570.02|113.31|0.00|1416.39|1529.70|-1919.52| +2451524|75316|10768|68284|1712478|4672|13266|10|78|2|30|2.27|3.83|1.11|0.00|33.30|68.10|114.90|0.00|0.00|33.30|33.30|-34.80| +2451524|75316|14701|68284|1712478|4672|13266|10|260|2|58|53.09|58.39|7.59|0.00|440.22|3079.22|3386.62|0.00|0.00|440.22|440.22|-2639.00| +|75316|17420|||4672||10||2||17.12|19.85||||||19.07|||495.84|| +2451524|75316|9436|68284|1712478|4672|13266|10|144|2|79|88.02|128.50|69.39|4549.90|5481.81|6953.58|10151.50|83.87|4549.90|931.91|1015.78|-6021.67| +2451524|75316|7654|68284|1712478|4672|13266|10|111|2|25|74.26|89.11|35.64|0.00|891.00|1856.50|2227.75|8.91|0.00|891.00|899.91|-965.50| +2451524|75316|12223|68284|1712478|4672|13266|10|267|2|78|65.71|88.70|23.06|341.74|1798.68|5125.38|6918.60|131.12|341.74|1456.94|1588.06|-3668.44| +2451524|75316|15085|68284|1712478|4672|13266|10|107|2|88|64.43|79.89|41.54|0.00|3655.52|5669.84|7030.32|146.22|0.00|3655.52|3801.74|-2014.32| +2451524|75316|3740|68284|1712478|4672|13266|10|72|2|49|6.55|7.86|7.46|0.00|365.54|320.95|385.14|0.00|0.00|365.54|365.54|44.59| +2451524|75316|1363|68284|1712478|4672|13266|10|38|2|4|13.46|15.34|12.57|0.00|50.28|53.84|61.36|4.02|0.00|50.28|54.30|-3.56| +2451524|75316|12068|68284|1712478|4672|13266|10|102|2|74|16.55|17.21|4.47|0.00|330.78|1224.70|1273.54|0.00|0.00|330.78|330.78|-893.92| +2451524|75316|13340|68284|1712478|4672|13266|10|212|2|71|36.01|64.09|3.20|0.00|227.20|2556.71|4550.39|4.54|0.00|227.20|231.74|-2329.51| +2451524|75316|2930|68284|1712478|4672|13266|10|234|2|36|61.23|93.06|11.16|0.00|401.76|2204.28|3350.16|28.12|0.00|401.76|429.88|-1802.52| +2451524|75316|15782|68284|1712478|4672|13266|10|268|2|62|77.97|85.76|79.75|0.00|4944.50|4834.14|5317.12|0.00|0.00|4944.50|4944.50|110.36| +2452638|50412|12447|44047|586372|697|22622|1|293|3|82|56.14|111.15|46.68|0.00|3827.76|4603.48|9114.30|306.22|0.00|3827.76|4133.98|-775.72| +2452638|50412|246|44047|586372|697|22622|1|229|3|96|98.02|136.24|104.90|0.00|10070.40|9409.92|13079.04|100.70|0.00|10070.40|10171.10|660.48| +||1531||||22622|1|222|3|||130.66|||3104.46||3527.82|124.17||||| +2452638|50412|3525|44047|586372|697|22622|1|165|3|42|97.03|178.53|7.14|0.00|299.88|4075.26|7498.26|17.99|0.00|299.88|317.87|-3775.38| +2452638|50412|13021|44047|586372|697|22622|1|56|3|64|74.69|120.25|2.40|0.00|153.60|4780.16|7696.00|3.07|0.00|153.60|156.67|-4626.56| +2452638|50412|17263|44047|586372|697|22622|1|14|3|17|72.38|111.46|75.79|0.00|1288.43|1230.46|1894.82|38.65|0.00|1288.43|1327.08|57.97| +2452638|50412|17971|44047|586372|697|22622|1|51|3|88|27.95|38.57|26.99|403.77|2375.12|2459.60|3394.16|0.00|403.77|1971.35|1971.35|-488.25| +2452638|50412|14443|44047|586372|697|22622|1|296|3|4|95.75|146.49|43.94|0.00|175.76|383.00|585.96|5.27|0.00|175.76|181.03|-207.24| +2452638|50412|14100|44047|586372|697|22622|1|77|3|79|44.66|83.06|17.44|1115.98|1377.76|3528.14|6561.74|5.23|1115.98|261.78|267.01|-3266.36| +2452638|50412|15786|44047|586372|697|22622|1|126|3|56|4.31|6.76|0.06|3.22|3.36|241.36|378.56|0.01|3.22|0.14|0.15|-241.22| +2452638|50412|10693|44047|586372|697|22622|1|140|3|27|37.04|58.89|55.35|0.00|1494.45|1000.08|1590.03|29.88|0.00|1494.45|1524.33|494.37| +2452638|50412|16869|44047|586372|697|22622|1|37|3|4|75.67|130.15|35.14|0.00|140.56|302.68|520.60|9.83|0.00|140.56|150.39|-162.12| +2452638|50412|5355|44047|586372|697|22622|1|90|3|53|23.04|44.00|26.40|0.00|1399.20|1221.12|2332.00|13.99|0.00|1399.20|1413.19|178.08| +2452638|50412|4698|44047|586372|697|22622|1|130|3|98|85.00|132.60|90.16|4506.19|8835.68|8330.00|12994.80|0.00|4506.19|4329.49|4329.49|-4000.51| +2451438|38195|12790|38814|874786|1511|19083|2|77|4|43|54.34|107.59|87.14|0.00|3747.02|2336.62|4626.37|299.76|0.00|3747.02|4046.78|1410.40| +2451438|38195|5515|38814|874786|1511|19083|2|224|4|23|2.91|3.57|3.53|32.47|81.19|66.93|82.11|0.97|32.47|48.72|49.69|-18.21| +2451438|38195|7990|38814|874786|1511|19083|2|16|4|56|64.68|78.90|24.45|0.00|1369.20|3622.08|4418.40|0.00|0.00|1369.20|1369.20|-2252.88| +2451438|38195|9685|38814|874786|1511|19083|2|284|4|21|40.39|40.39|12.11|0.00|254.31|848.19|848.19|22.88|0.00|254.31|277.19|-593.88| +2451438|38195|11036|38814|874786|1511|19083|2|229|4|41|67.18|78.60|5.50|0.00|225.50|2754.38|3222.60|13.53|0.00|225.50|239.03|-2528.88| +2451438|38195|8452|38814|874786|1511|19083|2|152|4|27|26.21|44.81|19.26|0.00|520.02|707.67|1209.87|46.80|0.00|520.02|566.82|-187.65| +2451438|38195|5350|38814|874786|1511|19083|2|241|4|86|64.67|123.51|55.57|0.00|4779.02|5561.62|10621.86|47.79|0.00|4779.02|4826.81|-782.60| +2451438|38195|163|38814|874786|1511|19083|2|190|4|17|54.26|105.80|9.52|116.52|161.84|922.42|1798.60|0.00|116.52|45.32|45.32|-877.10| +2451438|38195|1576|38814|874786|1511|19083|2|150|4|74|81.81|163.62|45.81|0.00|3389.94|6053.94|12107.88|271.19|0.00|3389.94|3661.13|-2664.00| +2451438|38195|6988|38814|874786|1511|19083|2|122|4|23|53.28|77.78|1.55|21.03|35.65|1225.44|1788.94|1.16|21.03|14.62|15.78|-1210.82| +2451465|53976|1940|13246|963544|5042|28940|2|70|5|60|69.54|112.65|57.45|0.00|3447.00|4172.40|6759.00|34.47|0.00|3447.00|3481.47|-725.40| +2451465|53976|11324|13246|963544|5042|28940|2|196|5|52|16.33|32.33|31.68|0.00|1647.36|849.16|1681.16|115.31|0.00|1647.36|1762.67|798.20| +2451465|53976|14599|13246|963544|5042|28940|2|167|5|83|74.15|89.72|63.70|0.00|5287.10|6154.45|7446.76|422.96|0.00|5287.10|5710.06|-867.35| +2451465|53976|11590|13246|963544|5042|28940|2|84|5|15|21.21|33.93|30.53|0.00|457.95|318.15|508.95|22.89|0.00|457.95|480.84|139.80| +2451465|53976|16792|13246|963544|5042|28940|2|12|5|71|22.10|37.57|28.17|0.00|2000.07|1569.10|2667.47|180.00|0.00|2000.07|2180.07|430.97| +2451465|53976|6466|13246|963544|5042|28940|2|200|5|36|7.93|12.92|7.36|0.00|264.96|285.48|465.12|0.00|0.00|264.96|264.96|-20.52| +2451465|53976|15752|13246|963544|5042|28940|2|10|5|66|71.06|136.43|45.02|0.00|2971.32|4689.96|9004.38|59.42|0.00|2971.32|3030.74|-1718.64| +2451465|53976|12650|13246|963544|5042|28940|2|185|5|66|21.01|28.57|5.99|0.00|395.34|1386.66|1885.62|7.90|0.00|395.34|403.24|-991.32| +|53976|5842||963544|||2|158|5|50|30.69|59.84||||1534.50||0.00||1735.00||200.50| +2451465|53976|14686|13246|963544|5042|28940|2|215|5|91|27.68|53.69|0.53|0.00|48.23|2518.88|4885.79|3.85|0.00|48.23|52.08|-2470.65| +2451465|53976|6068|13246|963544|5042|28940|2|125|5|76|89.78|103.24|50.58|0.00|3844.08|6823.28|7846.24|153.76|0.00|3844.08|3997.84|-2979.20| +2451465|53976|13562|13246|963544|5042|28940|2|75|5|64|87.90|95.81|66.10|0.00|4230.40|5625.60|6131.84|0.00|0.00|4230.40|4230.40|-1395.20| +2451465|53976|13958|13246|963544|5042|28940|2|149|5|60|41.72|52.98|32.31|0.00|1938.60|2503.20|3178.80|0.00|0.00|1938.60|1938.60|-564.60| +|53976|1808|13246|||28940|2||5|||77.52|11.62|0.00|732.06||4883.76|43.92|0.00|||| +2451465|53976|16195|13246|963544|5042|28940|2|206|5|50|30.96|56.34|45.07|0.00|2253.50|1548.00|2817.00|22.53|0.00|2253.50|2276.03|705.50| +||8621|53029||2092|||168|6||||||449.88|||||||| +2452109|36794|3049|53029|506184|2092|44800|4|72|6|31|49.78|77.15|69.43|0.00|2152.33|1543.18|2391.65|0.00|0.00|2152.33|2152.33|609.15| +2452109|36794|2549|53029|506184|2092|44800|4|128|6|62|85.07|159.08|76.35|0.00|4733.70|5274.34|9862.96|189.34|0.00|4733.70|4923.04|-540.64| +2452109|36794|10895|53029|506184|2092|44800|4|276|6|1|71.10|89.58|61.81|0.00|61.81|71.10|89.58|3.70|0.00|61.81|65.51|-9.29| +2452109|36794|11705|53029|506184|2092|44800|4|9|6|61|48.18|60.70|18.81|126.21|1147.41|2938.98|3702.70|71.48|126.21|1021.20|1092.68|-1917.78| +2452109|36794|13513|53029|506184|2092|44800|4|248|6|42|64.46|70.26|32.31|0.00|1357.02|2707.32|2950.92|27.14|0.00|1357.02|1384.16|-1350.30| +2452109|36794|3291|53029|506184|2092|44800|4|153|6|100|41.08|42.31|38.50|0.00|3850.00|4108.00|4231.00|346.50|0.00|3850.00|4196.50|-258.00| +2452109|36794|13245|53029|506184|2092|44800|4|252|6|64|86.35|152.83|122.26|6650.94|7824.64|5526.40|9781.12|35.21|6650.94|1173.70|1208.91|-4352.70| +2452109|36794|10355|53029|506184|2092|44800|4|216|6|94|62.67|99.64|97.64|0.00|9178.16|5890.98|9366.16|91.78|0.00|9178.16|9269.94|3287.18| +2452109|36794|6437|53029|506184|2092|44800|4|198|6|72|55.49|93.77|55.32|0.00|3983.04|3995.28|6751.44|199.15|0.00|3983.04|4182.19|-12.24| +2452109|36794|2647|53029|506184|2092|44800|4|150|6|100|4.45|7.87|7.24|0.00|724.00|445.00|787.00|57.92|0.00|724.00|781.92|279.00| +2452260|46712|5183|69954|890396|791|37633|2|6|7|51|84.65|124.43|49.77|0.00|2538.27|4317.15|6345.93|76.14|0.00|2538.27|2614.41|-1778.88| +2452260|46712|4627|69954|890396|791|37633|2|210|7|9|56.13|59.49|30.93|0.00|278.37|505.17|535.41|22.26|0.00|278.37|300.63|-226.80| +2452260||9751|69954|890396|791||||7|4|||||278.60|203.68|366.60|22.28||278.60|300.88|| +2452260|46712|15333|69954|||||85|7|||72.02||||||||4990.30||1141.70| +2452260|46712|4833|69954|890396|791|37633|2|153|7|88|38.23|72.63|39.22|0.00|3451.36|3364.24|6391.44|138.05|0.00|3451.36|3589.41|87.12| +2452260|46712|10487|69954|890396|791|37633|2|153|7|52|63.80|81.02|22.68|0.00|1179.36|3317.60|4213.04|23.58|0.00|1179.36|1202.94|-2138.24| +2452260|46712|5905|69954|890396|791|37633|2|77|7|69|99.85|113.82|110.40|0.00|7617.60|6889.65|7853.58|0.00|0.00|7617.60|7617.60|727.95| +2452260|46712|8955|69954|890396|791|37633|2|55|7|54|42.82|56.09|21.31|0.00|1150.74|2312.28|3028.86|92.05|0.00|1150.74|1242.79|-1161.54| +2452260|46712|15179|69954|890396|791|37633|2|5|7|12|47.60|75.20|65.42|0.00|785.04|571.20|902.40|47.10|0.00|785.04|832.14|213.84| +2452260|46712|4795|69954|890396|791|37633|2|212|7|73|12.17|14.23|11.24|623.59|820.52|888.41|1038.79|5.90|623.59|196.93|202.83|-691.48| +2452260|46712|17255|69954|890396|791|37633|2|275|7|69|34.19|58.46|50.86|0.00|3509.34|2359.11|4033.74|210.56|0.00|3509.34|3719.90|1150.23| +2452260|46712|12571|69954|890396|791|37633|2|147|7|82|69.53|87.60|34.16|0.00|2801.12|5701.46|7183.20|0.00|0.00|2801.12|2801.12|-2900.34| +2452179|52209|4183|86944|578318|6045|49019|8|109|8|90|81.63|151.83|113.87|9428.43|10248.30|7346.70|13664.70|40.99|9428.43|819.87|860.86|-6526.83| +2452179|52209|13427|86944|578318|6045|49019|8|106|8|87|31.78|63.24|16.44|0.00|1430.28|2764.86|5501.88|57.21|0.00|1430.28|1487.49|-1334.58| +2452179|52209|665|86944|578318|6045|49019|8|123|8|31|15.64|26.43|13.21|0.00|409.51|484.84|819.33|4.09|0.00|409.51|413.60|-75.33| +2452179|52209|11893|86944|578318|6045|49019|8|71|8|95|21.29|33.85|20.64|0.00|1960.80|2022.55|3215.75|98.04|0.00|1960.80|2058.84|-61.75| +2452179|52209|11365|86944|578318|6045|49019|8|239|8|7|18.65|27.97|20.41|0.00|142.87|130.55|195.79|1.42|0.00|142.87|144.29|12.32| +2452179|52209|7115|86944|578318|6045|49019|8|242|8|54|36.99|47.71|27.67|0.00|1494.18|1997.46|2576.34|89.65|0.00|1494.18|1583.83|-503.28| +2452179|52209|12041|86944|578318|6045|49019|8|299|8|95|91.80|93.63|75.84|6700.46|7204.80|8721.00|8894.85|5.04|6700.46|504.34|509.38|-8216.66| +2452179|52209|5929|86944|578318|6045|49019|8|128|8|83|14.11|27.37|3.28|0.00|272.24|1171.13|2271.71|13.61|0.00|272.24|285.85|-898.89| +2452179|52209|16671|86944|578318|6045|49019|8|45|8|20|18.95|33.92|31.88|0.00|637.60|379.00|678.40|51.00|0.00|637.60|688.60|258.60| +2452179|52209|17119|86944|578318|6045|49019|8|251|8|51|8.04|8.92|1.07|0.00|54.57|410.04|454.92|4.91|0.00|54.57|59.48|-355.47| +2452179|52209|17545|86944|578318|6045|49019|8|65|8|49|72.15|104.61|80.54|0.00|3946.46|3535.35|5125.89|39.46|0.00|3946.46|3985.92|411.11| +2452496|45986|12969|47480|1023064|4248|43385|4|14|9|59|88.31|106.85|44.87|0.00|2647.33|5210.29|6304.15|211.78|0.00|2647.33|2859.11|-2562.96| +2452496|45986|4623|47480|1023064|4248|43385|4|296|9|34|2.97|3.35|2.04|0.00|69.36|100.98|113.90|3.46|0.00|69.36|72.82|-31.62| +2452496|45986|17310|47480|1023064|4248|43385|4|167|9|33|27.13|53.44|2.13|0.00|70.29|895.29|1763.52|2.81|0.00|70.29|73.10|-825.00| +2452496|45986|13717|47480|1023064|4248|43385|4|172|9|53|75.37|89.69|54.71|0.00|2899.63|3994.61|4753.57|115.98|0.00|2899.63|3015.61|-1094.98| +|45986|16363|47480|1023064||43385|||9|54||||0.00|768.42|3873.42|||0.00||829.89|| +2452496|45986|69|47480|1023064|4248|43385|4|96|9|11|31.70|54.52|47.97|527.67|527.67|348.70|599.72|0.00|527.67|0.00|0.00|-348.70| +2452496|45986|16461|47480|1023064|4248|43385|4|281|9|66|15.21|26.00|25.22|66.58|1664.52|1003.86|1716.00|111.85|66.58|1597.94|1709.79|594.08| +2452496|45986|889|47480|1023064|4248|43385|4|287|9|6|27.17|51.62|25.29|0.00|151.74|163.02|309.72|3.03|0.00|151.74|154.77|-11.28| +2452496|45986|12027|47480|1023064|4248|43385|4|1|9|27|98.68|195.38|15.63|0.00|422.01|2664.36|5275.26|33.76|0.00|422.01|455.77|-2242.35| +2452496|45986|1185|47480|1023064|4248|43385|4|139|9|62|55.68|64.03|3.84|233.31|238.08|3452.16|3969.86|0.14|233.31|4.77|4.91|-3447.39| +2452496|45986|16083|47480|1023064|4248|43385|4|124|9|32|99.10|152.61|132.77|0.00|4248.64|3171.20|4883.52|339.89|0.00|4248.64|4588.53|1077.44| +2452496|45986|7945|47480|1023064|4248|43385|4|239|9|83|8.10|14.58|8.60|0.00|713.80|672.30|1210.14|64.24|0.00|713.80|778.04|41.50| +|45986|13483|||4248||4|109|9||59.14|108.81|46.78|0.00||3311.84|6093.36||0.00|2619.68|2619.68|| +2452496|45986|8334|47480|1023064|4248|43385|4|22|9|71|34.79|36.87|19.17|1197.74|1361.07|2470.09|2617.77|8.16|1197.74|163.33|171.49|-2306.76| +2452496|45986|15133|47480|1023064|4248|43385|4|111|9|15|35.89|63.88|30.66|0.00|459.90|538.35|958.20|41.39|0.00|459.90|501.29|-78.45| +2452496|45986|16659|47480|1023064|4248|43385|4|1|9|84|76.71|85.14|5.10|0.00|428.40|6443.64|7151.76|21.42|0.00|428.40|449.82|-6015.24| +2451966|60226|9811|4451|71288|6925|24515|10|38|10|28|47.85|50.24|19.09|0.00|534.52|1339.80|1406.72|37.41|0.00|534.52|571.93|-805.28| +2451966|60226|3933|4451|71288|6925|24515|10|61|10|52|52.19|66.80|26.05|0.00|1354.60|2713.88|3473.60|94.82|0.00|1354.60|1449.42|-1359.28| +2451966|60226|5863|4451|71288|6925|24515|10|27|10|47|11.71|22.60|3.61|0.00|169.67|550.37|1062.20|3.39|0.00|169.67|173.06|-380.70| +2451966|60226|4357|4451|71288|6925|24515|10|122|10|17|88.36|133.42|53.36|0.00|907.12|1502.12|2268.14|0.00|0.00|907.12|907.12|-595.00| +2451966|60226|15447|4451|71288|6925|24515|10|250|10|67|33.28|34.61|19.72|0.00|1321.24|2229.76|2318.87|39.63|0.00|1321.24|1360.87|-908.52| +2451966|60226|3433|4451|71288|6925|24515|10|195|10|83|10.26|17.33|9.18|0.00|761.94|851.58|1438.39|45.71|0.00|761.94|807.65|-89.64| +2451966|60226|755|4451|71288|6925|24515|10|91|10|74|82.24|152.96|97.89|0.00|7243.86|6085.76|11319.04|144.87|0.00|7243.86|7388.73|1158.10| +||1425|4451|71288|6925|24515|||10|92|||||2092.08|2280.68|3033.24|||2092.08|2280.36|-188.60| +2451966|60226|1511|4451|71288|6925|24515|10|95|10|76|31.47|56.01|31.36|0.00|2383.36|2391.72|4256.76|143.00|0.00|2383.36|2526.36|-8.36| +2451966|60226|13803|4451|71288|6925|24515|10|185|10|66|82.35|137.52|82.51|0.00|5445.66|5435.10|9076.32|0.00|0.00|5445.66|5445.66|10.56| +2452420|68961|7519|48900|781292|2484|22923|8|68|11|68|7.16|12.88|5.66|107.76|384.88|486.88|875.84|8.31|107.76|277.12|285.43|-209.76| +2452420|68961|1315|48900|781292|2484|22923|8|112|11|70|45.84|49.50|45.54|0.00|3187.80|3208.80|3465.00|159.39|0.00|3187.80|3347.19|-21.00| +2452420|68961|12438|48900|781292|2484|22923|8|239|11|16|92.94|185.88|46.47|0.00|743.52|1487.04|2974.08|44.61|0.00|743.52|788.13|-743.52| +2452420|68961|17395|48900|781292|2484|22923|8|214|11|85|38.04|42.98|22.34|0.00|1898.90|3233.40|3653.30|56.96|0.00|1898.90|1955.86|-1334.50| +2452420|68961|157|48900|781292|2484|22923|8|146|11|84|64.63|101.46|14.20|0.00|1192.80|5428.92|8522.64|59.64|0.00|1192.80|1252.44|-4236.12| +2452420|68961|10699|48900|781292|2484|22923|8|199|11|33|73.77|138.68|128.97|0.00|4256.01|2434.41|4576.44|0.00|0.00|4256.01|4256.01|1821.60| +2452420|68961|15157|48900|781292|2484|22923|8|90|11|96|15.52|21.41|21.41|0.00|2055.36|1489.92|2055.36|61.66|0.00|2055.36|2117.02|565.44| +2452420|68961|17226|48900|781292|2484|22923|8|68|11|11|34.03|44.23|42.46|0.00|467.06|374.33|486.53|18.68|0.00|467.06|485.74|92.73| +2452420|68961|11991|48900|781292|2484|22923|8|113|11|38|3.27|3.56|0.32|0.00|12.16|124.26|135.28|0.48|0.00|12.16|12.64|-112.10| +2452420|68961|9901|48900|781292|2484|22923|8|294|11|57|46.93|80.71|63.76|0.00|3634.32|2675.01|4600.47|327.08|0.00|3634.32|3961.40|959.31| +2452420|68961|7608|48900|781292|2484|22923|8|161|11|66|8.34|15.76|15.76|0.00|1040.16|550.44|1040.16|83.21|0.00|1040.16|1123.37|489.72| +2452420|68961|15649|48900|781292|2484|22923|8|277|11|33|66.11|114.37|78.91|0.00|2604.03|2181.63|3774.21|104.16|0.00|2604.03|2708.19|422.40| +||11490||781292||22923|8|246|11|||||0.00||4141.28||483.99|0.00||5861.67|1236.40| +2451206|35137|6248|20820|63209|3198|42434|8|261|12|36|75.17|75.92|40.23|0.00|1448.28|2706.12|2733.12|14.48|0.00|1448.28|1462.76|-1257.84| +2451206|35137|14944|20820|63209|3198|42434|8|1|12|9|7.33|11.94|11.22|31.30|100.98|65.97|107.46|2.09|31.30|69.68|71.77|3.71| +2451206|35137|9788|20820|63209|3198|42434|8|297|12|73|79.42|95.30|78.14|0.00|5704.22|5797.66|6956.90|456.33|0.00|5704.22|6160.55|-93.44| +2451206|35137|373|20820|63209|3198|42434|8|286|12|57|13.95|27.06|8.38|0.00|477.66|795.15|1542.42|33.43|0.00|477.66|511.09|-317.49| +2451206|35137|9616|20820|63209|3198|42434|8|79|12|66|99.06|104.01|78.00|0.00|5148.00|6537.96|6864.66|154.44|0.00|5148.00|5302.44|-1389.96| +2451206|35137|4888|20820|63209|3198|42434|8|183|12|56|75.74|133.30|0.00|0.00|0.00|4241.44|7464.80|0.00|0.00|0.00|0.00|-4241.44| +2451206|35137|14746|20820|63209|3198|42434|8|87|12|26|58.74|82.23|60.02|0.00|1560.52|1527.24|2137.98|46.81|0.00|1560.52|1607.33|33.28| +2451206|35137|1591|20820|63209|3198|42434|8|277|12|82|45.84|49.96|22.98|1262.52|1884.36|3758.88|4096.72|55.96|1262.52|621.84|677.80|-3137.04| +2451206|35137|6148|20820|63209|3198|42434|8|139|12|36|97.62|193.28|15.46|0.00|556.56|3514.32|6958.08|5.56|0.00|556.56|562.12|-2957.76| +2451206|35137|15440|20820|63209|3198|42434|8|1|12|99|27.09|53.63|26.27|0.00|2600.73|2681.91|5309.37|104.02|0.00|2600.73|2704.75|-81.18| +2451206|35137|13399|20820|63209|3198|42434|8|289|12|46|45.27|60.20|22.27|0.00|1024.42|2082.42|2769.20|51.22|0.00|1024.42|1075.64|-1058.00| +2450850|65476|5971|15144|1401973|4002|32431|1|138|13|80|72.61|82.04|63.17|0.00|5053.60|5808.80|6563.20|454.82|0.00|5053.60|5508.42|-755.20| +2450850|65476|6682|15144|1401973|4002|32431|1|135|13|80|32.05|41.98|22.66|0.00|1812.80|2564.00|3358.40|72.51|0.00|1812.80|1885.31|-751.20| +2450850|65476|4430|15144|1401973|4002|32431|1|120|13|73|5.86|11.72|0.70|0.00|51.10|427.78|855.56|4.08|0.00|51.10|55.18|-376.68| +2450850|65476|6085|15144|1401973|4002|32431|1|72|13|58|21.45|28.74|22.12|0.00|1282.96|1244.10|1666.92|115.46|0.00|1282.96|1398.42|38.86| +2450850|65476|1760|15144|1401973|4002|32431|1|121|13|12|80.96|138.44|41.53|0.00|498.36|971.52|1661.28|24.91|0.00|498.36|523.27|-473.16| +2450850|65476|10096|15144|1401973|4002|32431|1|107|13|12|17.14|30.85|28.99|0.00|347.88|205.68|370.20|6.95|0.00|347.88|354.83|142.20| +|65476|1898|15144|||32431|1|180|13||96.46|||0.00|432.48|385.84||8.64|0.00||441.12|| +2450850|65476|7640|15144|1401973|4002|32431|1|5|13|48|17.06|28.49|10.82|0.00|519.36|818.88|1367.52|25.96|0.00|519.36|545.32|-299.52| +2450850||868|15144||4002|32431|1||13||62.85|109.35||0.00||5467.95|9513.45|38.03|0.00|950.91||-4517.04| +2450850|65476|17240|15144|1401973|4002|32431|1|163|13|20|93.85|185.82|137.50|2255.00|2750.00|1877.00|3716.40|39.60|2255.00|495.00|534.60|-1382.00| +2450850|65476|16894|15144|1401973|4002|32431|1|255|13|87|20.99|30.43|16.12|1192.07|1402.44|1826.13|2647.41|14.72|1192.07|210.37|225.09|-1615.76| +2450850|65476|6140|15144|1401973|4002|32431|1|1|13|15|89.90|163.61|4.90|46.30|73.50|1348.50|2454.15|2.44|46.30|27.20|29.64|-1321.30| +2450850|65476|2191|15144||4002|32431|||13||||15.60|||2390.50||||||| +2450850|65476|7723|15144|1401973|4002|32431|1|252|13|27|59.09|112.86|83.51|0.00|2254.77|1595.43|3047.22|202.92|0.00|2254.77|2457.69|659.34| +2450850|65476|2108||1401973||32431|||13|9||||478.29||||1.89|478.29||49.20|| +2450850|65476|11758|15144|1401973|4002|32431|1|84|13|34|72.24|107.63|49.50|1464.21|1683.00|2456.16|3659.42|17.50|1464.21|218.79|236.29|-2237.37| +2452217|34167|13515|19409|1145357|4905|39653|4|125|14|57|87.32|92.55|43.49|1933.56|2478.93|4977.24|5275.35|5.45|1933.56|545.37|550.82|-4431.87| +2452217|34167|177|19409|1145357|4905|39653|4|15|14|41|13.05|15.13|9.22|109.62|378.02|535.05|620.33|0.00|109.62|268.40|268.40|-266.65| +2452217|34167|16929|19409|1145357|4905|39653|4|120|14|14|54.76|91.99|0.91|0.00|12.74|766.64|1287.86|0.25|0.00|12.74|12.99|-753.90| +2452217|34167|16741|19409|1145357|4905|39653|4|253|14|46|76.43|110.82|64.27|0.00|2956.42|3515.78|5097.72|236.51|0.00|2956.42|3192.93|-559.36| +2452217|34167|11549|19409|1145357|4905|39653|4|254|14|6|19.33|26.67|20.80|0.00|124.80|115.98|160.02|7.48|0.00|124.80|132.28|8.82| +2452217|34167|11817|19409|1145357|4905|39653|4|182|14|81|60.77|66.84|7.35|0.00|595.35|4922.37|5414.04|35.72|0.00|595.35|631.07|-4327.02| +2452217|34167|11653|19409|1145357|4905|39653|4|18|14|60|86.94|88.67|39.01|1638.42|2340.60|5216.40|5320.20|0.00|1638.42|702.18|702.18|-4514.22| +2452217||12587|19409||4905|||82|14||||||1233.75||3163.50|74.02||||-467.25| +2452217|34167|13069|19409|1145357|4905|39653|4|117|14|77|93.60|105.76|68.74|0.00|5292.98|7207.20|8143.52|264.64|0.00|5292.98|5557.62|-1914.22| +2452217|34167|4507|19409|1145357|4905|39653|4|118|14|4|45.45|84.08|65.58|0.00|262.32|181.80|336.32|0.00|0.00|262.32|262.32|80.52| +2452217|34167|13845|19409|1145357|4905|39653|4|146|14|17|52.30|84.72|64.38|0.00|1094.46|889.10|1440.24|76.61|0.00|1094.46|1171.07|205.36| +2452217|34167|10175|19409|1145357|4905|39653|4|234|14|19|39.97|57.15|46.86|0.00|890.34|759.43|1085.85|53.42|0.00|890.34|943.76|130.91| +2452217|34167|769|19409|1145357|4905|39653|4|258|14|20|26.29|51.79|9.32|0.00|186.40|525.80|1035.80|11.18|0.00|186.40|197.58|-339.40| +2451797|55556|17267|62835|227170|1458|32993|4|70|15|7|30.61|51.73|51.73|0.00|362.11|214.27|362.11|25.34|0.00|362.11|387.45|147.84| +2451797|55556|11659|62835|227170|1458|32993|4|197|15|80|86.23|156.93|20.40|750.72|1632.00|6898.40|12554.40|17.62|750.72|881.28|898.90|-6017.12| +2451797|55556|17857|62835|227170|1458|32993|4|291|15|38|96.35|104.05|32.25|0.00|1225.50|3661.30|3953.90|61.27|0.00|1225.50|1286.77|-2435.80| +2451797|55556|4777|62835|227170|1458|32993|4|4|15|28|36.86|58.60|58.01|0.00|1624.28|1032.08|1640.80|32.48|0.00|1624.28|1656.76|592.20| +2451797|55556|8690|62835|227170|1458|32993|4|148|15|32|67.37|117.89|9.43|0.00|301.76|2155.84|3772.48|24.14|0.00|301.76|325.90|-1854.08| +2451797|55556|10361|62835|227170|1458|32993|4|82|15|39|74.88|125.04|1.25|0.00|48.75|2920.32|4876.56|2.92|0.00|48.75|51.67|-2871.57| +2451797|55556|16619|62835|227170|1458|32993|4|133|15|81|80.21|152.39|105.14|0.00|8516.34|6497.01|12343.59|766.47|0.00|8516.34|9282.81|2019.33| +2451797|55556|8707|62835|227170|1458|32993|4|58|15|21|48.54|57.27|1.14|0.00|23.94|1019.34|1202.67|0.23|0.00|23.94|24.17|-995.40| +2451797|55556|8353|62835|||32993|4|157|15|||||0.00||||28.50|0.00|950.04||-1336.79| +2451797|55556|4505|62835|227170|1458|32993|4|142|15|59|77.35|113.70|61.39|0.00|3622.01|4563.65|6708.30|289.76|0.00|3622.01|3911.77|-941.64| +2451797|55556|4241|62835|227170|1458|32993|4|136|15|21|89.07|132.71|95.55|0.00|2006.55|1870.47|2786.91|120.39|0.00|2006.55|2126.94|136.08| +2451797|55556|17330|62835|227170|1458|32993|4|265|15|82|67.45|122.08|95.22|6714.91|7808.04|5530.90|10010.56|76.51|6714.91|1093.13|1169.64|-4437.77| +2451797|55556|7391|62835|227170|1458|32993|4|32|15|98|53.76|65.58|3.27|0.00|320.46|5268.48|6426.84|0.00|0.00|320.46|320.46|-4948.02| +2451797|55556|8336|62835|227170|1458|32993|4|112|15|15|44.09|60.84|45.02|0.00|675.30|661.35|912.60|13.50|0.00|675.30|688.80|13.95| +2451797|55556|17564|62835|227170|1458|32993|4|136|15|26|63.52|88.92|40.90|0.00|1063.40|1651.52|2311.92|0.00|0.00|1063.40|1063.40|-588.12| +2451797|55556|13172|62835|227170|1458|32993|4|35|15|25|47.11|57.94|38.81|0.00|970.25|1177.75|1448.50|29.10|0.00|970.25|999.35|-207.50| +2450872|47818|9997|51139|263380|868|33509|1|270|16|94|64.76|119.80|74.27|0.00|6981.38|6087.44|11261.20|488.69|0.00|6981.38|7470.07|893.94| +2450872|47818|17246|51139|263380|868|33509|1|51|16|70|80.85|161.70|100.25|0.00|7017.50|5659.50|11319.00|210.52|0.00|7017.50|7228.02|1358.00| +2450872|47818|9790|51139|263380|868|33509|1|4|16|17|36.95|61.70|60.46|0.00|1027.82|628.15|1048.90|20.55|0.00|1027.82|1048.37|399.67| +2450872|47818|4144|51139|263380|868|33509|1|228|16|94|19.91|25.28|22.49|0.00|2114.06|1871.54|2376.32|21.14|0.00|2114.06|2135.20|242.52| +2450872|47818|8887|51139||868||||16|35|59.82||50.91|||2093.70|2784.60||||1870.94|-311.85| +2450872|47818|6008|51139|263380|868|33509|1|191|16|59|59.62|63.79|60.60|0.00|3575.40|3517.58|3763.61|214.52|0.00|3575.40|3789.92|57.82| +2450872|47818|457|51139|263380|868|33509|1|248|16|60|91.53|167.49|68.67|0.00|4120.20|5491.80|10049.40|370.81|0.00|4120.20|4491.01|-1371.60| +2450872|47818|11168|51139|263380|868|33509|1|132|16|86|62.85|103.07|64.93|0.00|5583.98|5405.10|8864.02|390.87|0.00|5583.98|5974.85|178.88| +2450872|47818|9769|51139|263380|868|33509|1|267|16|42|29.53|33.66|12.11|391.63|508.62|1240.26|1413.72|7.01|391.63|116.99|124.00|-1123.27| +2450872|47818|7504|51139|263380|868|33509|1|172|16|51|31.35|55.17|15.99|0.00|815.49|1598.85|2813.67|65.23|0.00|815.49|880.72|-783.36| +2450872|47818|1888|51139|263380|868|33509|1|48|16|4|47.64|61.93|55.73|0.00|222.92|190.56|247.72|6.68|0.00|222.92|229.60|32.36| +2450872|47818|16226|51139|263380|868|33509|1|152|16|13|31.30|61.03|31.12|0.00|404.56|406.90|793.39|36.41|0.00|404.56|440.97|-2.34| +2450872|47818|11920|51139|263380|868|33509|1|145|16|29|94.31|151.83|103.24|0.00|2993.96|2734.99|4403.07|29.93|0.00|2993.96|3023.89|258.97| +2451200|49168|17683|77683|756435|2034|22386|7|185|17|34|81.48|136.07|55.78|0.00|1896.52|2770.32|4626.38|37.93|0.00|1896.52|1934.45|-873.80| +2451200|49168|4678|77683|756435|2034|22386|7|68|17|34|32.47|56.49|44.06|149.80|1498.04|1103.98|1920.66|121.34|149.80|1348.24|1469.58|244.26| +2451200|49168|6811|77683|756435|2034|22386|7|144|17|70|62.96|66.73|24.02|0.00|1681.40|4407.20|4671.10|33.62|0.00|1681.40|1715.02|-2725.80| +2451200|49168|13177|77683|756435|2034|22386|7|209|17|45|44.18|72.45|51.43|0.00|2314.35|1988.10|3260.25|138.86|0.00|2314.35|2453.21|326.25| +2451200|49168|2092|77683|756435|2034|22386|7|110|17|37|31.71|47.88|37.82|1385.34|1399.34|1173.27|1771.56|0.00|1385.34|14.00|14.00|-1159.27| +2451200|49168|11203|77683|756435|2034|22386|7|80|17|21|93.44|172.86|31.11|0.00|653.31|1962.24|3630.06|26.13|0.00|653.31|679.44|-1308.93| +2451200|49168|13826|77683|756435|2034|22386|7|188|17|32|58.61|63.88|47.91|0.00|1533.12|1875.52|2044.16|137.98|0.00|1533.12|1671.10|-342.40| +2451200|49168|15781|77683|756435|2034|22386|7|279|17|76|24.79|33.46|4.34|0.00|329.84|1884.04|2542.96|6.59|0.00|329.84|336.43|-1554.20| +2451200|49168|9214|77683|756435|2034|22386|7|212|17|57|14.20|25.56|8.94|0.00|509.58|809.40|1456.92|25.47|0.00|509.58|535.05|-299.82| +2451200|49168|10543|77683|756435|2034|22386|7|32|17|54|57.11|93.66|76.80|0.00|4147.20|3083.94|5057.64|290.30|0.00|4147.20|4437.50|1063.26| +2451415|54836|8884|63292|54321|997|9226|10|55|18|18|43.49|59.58|12.51|0.00|225.18|782.82|1072.44|18.01|0.00|225.18|243.19|-557.64| +2451415|54836|11584|63292|54321|997|9226|10|289|18|76|4.26|7.58|3.56|0.00|270.56|323.76|576.08|13.52|0.00|270.56|284.08|-53.20| +2451415|54836|11890|63292|54321|997|9226|10|99|18|7|82.36|159.77|63.90|0.00|447.30|576.52|1118.39|4.47|0.00|447.30|451.77|-129.22| +2451415|54836|12602|63292|54321|997|9226|10|162|18|81|11.32|18.79|8.07|0.00|653.67|916.92|1521.99|0.00|0.00|653.67|653.67|-263.25| +2451415|54836|7378|63292|54321|997|9226|10|270|18|94|61.01|114.08|99.24|7835.99|9328.56|5734.94|10723.52|119.40|7835.99|1492.57|1611.97|-4242.37| +2451415|54836|9886|63292|54321|997|9226|10|191|18|62|9.59|15.43|10.49|0.00|650.38|594.58|956.66|32.51|0.00|650.38|682.89|55.80| +2451415|54836|14011|63292|54321|997|9226|10|221|18|95|55.01|101.21|36.43|0.00|3460.85|5225.95|9614.95|311.47|0.00|3460.85|3772.32|-1765.10| +2451415|54836|17995|63292|54321|997|9226|10|30|18|13|46.79|52.87|32.25|327.01|419.25|608.27|687.31|2.76|327.01|92.24|95.00|-516.03| +2451415|54836|12860|63292|54321|997|9226|10|221|18|18|19.89|32.61|4.23|0.00|76.14|358.02|586.98|4.56|0.00|76.14|80.70|-281.88| +2451415|54836|12826|63292|54321|997|9226|10|89|18|93|82.82|135.82|35.31|426.89|3283.83|7702.26|12631.26|142.84|426.89|2856.94|2999.78|-4845.32| +2451415|54836|14377|63292|54321|997|9226|10|181|18|15|15.47|20.57|18.10|203.62|271.50|232.05|308.55|0.00|203.62|67.88|67.88|-164.17| +2451415|54836|2440|63292|54321|997|9226|10|142|18|40|15.39|28.93|24.87|0.00|994.80|615.60|1157.20|49.74|0.00|994.80|1044.54|379.20| +2451415|54836|8779|63292|54321|||10||18|9|75.19|121.80|60.90|0.00|||||0.00|||| +2451415|54836|14372|63292|54321|997|9226|10|211|18|76|89.58|130.78|27.46|0.00|2086.96|6808.08|9939.28|83.47|0.00|2086.96|2170.43|-4721.12| +2451415|54836|5251|63292|54321|997|9226|10|163|18|41|45.83|78.36|38.39|1353.63|1573.99|1879.03|3212.76|15.42|1353.63|220.36|235.78|-1658.67| +2451361|65446|11914|51104|1847200|1343|38239|7|287|19|59|88.03|153.17|13.78|0.00|813.02|5193.77|9037.03|56.91|0.00|813.02|869.93|-4380.75| +2451361|65446|10651|51104|1847200|1343|38239|7|196|19|46|57.69|70.95|7.09|104.36|326.14|2653.74|3263.70|19.96|104.36|221.78|241.74|-2431.96| +2451361|65446|3133|51104|1847200|1343|38239|7|56|19|96|68.89|102.64|25.66|0.00|2463.36|6613.44|9853.44|221.70|0.00|2463.36|2685.06|-4150.08| +2451361|65446|7087|51104|1847200|1343|38239|7|266|19|1|48.67|51.10|31.68|0.00|31.68|48.67|51.10|1.26|0.00|31.68|32.94|-16.99| +2451361|65446|9346|51104|1847200|1343|38239|7|202|19|39|84.06|85.74|0.85|0.00|33.15|3278.34|3343.86|2.98|0.00|33.15|36.13|-3245.19| +2451361|65446|6772|51104|1847200|1343|38239|7|99|19|97|36.04|62.34|21.81|0.00|2115.57|3495.88|6046.98|21.15|0.00|2115.57|2136.72|-1380.31| +||9094|51104|||38239||169|19|49|61.82|110.03|||1401.40||5391.47|||||| +2451361|65446|1094|51104|1847200|1343|38239|7|74|19|48|19.55|34.79|20.52|128.04|984.96|938.40|1669.92|51.41|128.04|856.92|908.33|-81.48| +2451361|65446|3376|51104|1847200|1343|38239|7|299|19|84|63.07|90.19|30.66|0.00|2575.44|5297.88|7575.96|51.50|0.00|2575.44|2626.94|-2722.44| +2451361|65446|4882|51104|1847200|1343|38239|7|182|19|84|41.48|44.79|33.14|0.00|2783.76|3484.32|3762.36|27.83|0.00|2783.76|2811.59|-700.56| +2451361|65446|8662|51104|1847200|1343|38239|7|163|19|97|72.78|90.24|83.02|0.00|8052.94|7059.66|8753.28|724.76|0.00|8052.94|8777.70|993.28| +2451361|65446|17539|51104|1847200|1343|38239|7|68|19|20|69.20|93.42|73.80|0.00|1476.00|1384.00|1868.40|14.76|0.00|1476.00|1490.76|92.00| +2451361|65446|16330|51104|||38239|7|278|19||79.15||47.10|0.00||395.75||9.42|0.00||244.92|| +2451361|65446|7814|51104|1847200|1343|38239|7|297|19|29|61.78|100.70|16.11|0.00|467.19|1791.62|2920.30|42.04|0.00|467.19|509.23|-1324.43| +2451361|65446|10558|51104|1847200|1343|38239|7|289|19|82|12.34|17.52|6.13|0.00|502.66|1011.88|1436.64|40.21|0.00|502.66|542.87|-509.22| +2451556|45944|5425|44828|671305|3382|48259|10|250|20|15|28.19|29.88|5.97|11.64|89.55|422.85|448.20|5.45|11.64|77.91|83.36|-344.94| +2451556|45944|6026|44828|671305|3382|48259|10|144|20|21|80.56|99.89|15.98|0.00|335.58|1691.76|2097.69|6.71|0.00|335.58|342.29|-1356.18| +2451556|45944|13601|44828|671305|3382|48259|10|133|20|17|1.40|2.52|0.80|11.83|13.60|23.80|42.84|0.12|11.83|1.77|1.89|-22.03| +2451556|45944|2618|44828|671305|3382|48259|10|236|20|4|69.47|121.57|71.72|106.14|286.88|277.88|486.28|16.26|106.14|180.74|197.00|-97.14| +2451556|45944|15131|44828|671305|3382|48259|10|188|20|85|42.56|68.94|19.99|0.00|1699.15|3617.60|5859.90|50.97|0.00|1699.15|1750.12|-1918.45| +2451556|45944|9800|44828|671305|3382|48259|10|230|20|32|18.09|32.74|31.10|0.00|995.20|578.88|1047.68|39.80|0.00|995.20|1035.00|416.32| +|45944|14935|44828||||10|241|20|||22.03|7.04||337.92|719.52||10.13||337.92|348.05|| +2451556|45944|9086|44828|671305|3382|48259|10|12|20|4|98.99|188.08|0.00|0.00|0.00|395.96|752.32|0.00|0.00|0.00|0.00|-395.96| +2451556|45944|5312|44828|671305|3382|48259|10|297|20|9|29.45|50.06|28.03|0.00|252.27|265.05|450.54|5.04|0.00|252.27|257.31|-12.78| +2451556|45944|1451|44828|671305|3382|48259|10|188|20|89|84.34|164.46|18.09|0.00|1610.01|7506.26|14636.94|80.50|0.00|1610.01|1690.51|-5896.25| +2451556|45944|5483|44828|671305|3382|48259|10|103|20|8|30.74|33.50|11.39|0.00|91.12|245.92|268.00|3.64|0.00|91.12|94.76|-154.80| +|45944|8714|44828|671305|3382|48259||97|20||8.15|||0.00|112.44|97.80|190.68|4.49|0.00|||14.64| +2451556|45944|7207|44828|671305|3382|48259|10|130|20|90|83.12|102.23|68.49|0.00|6164.10|7480.80|9200.70|308.20|0.00|6164.10|6472.30|-1316.70| +2451172|70722|2870|60478|1673919|2560|10401|1|84|21|50|91.94|138.82|27.76|0.00|1388.00|4597.00|6941.00|69.40|0.00|1388.00|1457.40|-3209.00| +2451172|70722|12266|60478|1673919|2560|10401|1|172|21|47|11.55|12.58|12.20|0.00|573.40|542.85|591.26|51.60|0.00|573.40|625.00|30.55| +2451172|70722|230|60478|1673919|2560|10401|1|280|21|48|13.37|23.93|20.10|0.00|964.80|641.76|1148.64|86.83|0.00|964.80|1051.63|323.04| +2451172|70722|9187|60478|1673919|2560|10401|1|115|21|14|35.49|42.58|24.69|214.30|345.66|496.86|596.12|3.94|214.30|131.36|135.30|-365.50| +2451172|70722|6476|60478|1673919|2560|10401|1|89|21|49|20.29|31.24|6.24|0.00|305.76|994.21|1530.76|3.05|0.00|305.76|308.81|-688.45| +2451172|70722|1810|60478|1673919|2560|10401|1|88|21|59|66.37|126.76|20.28|873.45|1196.52|3915.83|7478.84|0.00|873.45|323.07|323.07|-3592.76| +2451172|70722|5170|60478|1673919|2560|10401|1|93|21|45|90.00|108.00|65.88|0.00|2964.60|4050.00|4860.00|88.93|0.00|2964.60|3053.53|-1085.40| +2451172|70722|5998|60478|1673919|2560|10401|1|13|21|51|9.41|10.44|3.02|150.93|154.02|479.91|532.44|0.00|150.93|3.09|3.09|-476.82| +2451172|70722|14368|60478|1673919|2560|10401|1|275|21|18|51.29|69.24|59.54|664.46|1071.72|923.22|1246.32|36.65|664.46|407.26|443.91|-515.96| +2451172|70722|14396|60478|1673919|2560|10401|1|109|21|88|45.26|54.76|15.33|0.00|1349.04|3982.88|4818.88|13.49|0.00|1349.04|1362.53|-2633.84| +2451112|65253|16396|64935|1392098|4508|11671|10|146|22|85|86.46|129.69|25.93|0.00|2204.05|7349.10|11023.65|44.08|0.00|2204.05|2248.13|-5145.05| +2451112|65253|17660|64935|1392098|4508|11671|10|102|22|70|95.56|132.82|31.87|0.00|2230.90|6689.20|9297.40|66.92|0.00|2230.90|2297.82|-4458.30| +2451112|65253|15310|64935|1392098|4508|11671|10|154|22|15|82.24|137.34|108.49|1594.80|1627.35|1233.60|2060.10|2.60|1594.80|32.55|35.15|-1201.05| +2451112|65253|12415|64935|1392098|4508|11671|10|160|22|10|38.97|41.69|5.83|1.74|58.30|389.70|416.90|0.00|1.74|56.56|56.56|-333.14| +2451112|65253|9985|64935|1392098|4508|11671|10|145|22|70|21.46|22.96|2.06|0.00|144.20|1502.20|1607.20|0.00|0.00|144.20|144.20|-1358.00| +2451112|65253|17392|64935|1392098|4508|11671|10|262|22|14|51.86|91.27|73.01|0.00|1022.14|726.04|1277.78|30.66|0.00|1022.14|1052.80|296.10| +2451112|65253|16922|64935|1392098|4508|11671|10|190|22|88|28.00|29.68|18.99|0.00|1671.12|2464.00|2611.84|133.68|0.00|1671.12|1804.80|-792.88| +2451112|65253|11599|64935|1392098|4508|11671|10|161|22|66|5.01|5.01|3.10|0.00|204.60|330.66|330.66|6.13|0.00|204.60|210.73|-126.06| +2451112|65253|10474|64935|1392098|4508|11671|10|236|22|31|45.65|54.32|51.60|0.00|1599.60|1415.15|1683.92|0.00|0.00|1599.60|1599.60|184.45| +2452617|31683|7242|21669|1833048|6474|33053|10|189|23|37|54.82|55.91|0.55|18.31|20.35|2028.34|2068.67|0.06|18.31|2.04|2.10|-2026.30| +2452617|31683|11257|21669|1833048|6474|33053|10|142|23|64|29.95|45.52|8.64|0.00|552.96|1916.80|2913.28|0.00|0.00|552.96|552.96|-1363.84| +2452617|31683|12397|21669|1833048|6474|33053|10|106|23|80|78.73|103.13|20.62|0.00|1649.60|6298.40|8250.40|115.47|0.00|1649.60|1765.07|-4648.80| +2452617|31683|9093|21669|1833048|6474|33053|10|52|23|38|80.20|81.00|22.68|0.00|861.84|3047.60|3078.00|17.23|0.00|861.84|879.07|-2185.76| +2452617|31683|15403|21669|1833048|6474|33053|10|153|23|96|51.96|77.42|73.54|0.00|7059.84|4988.16|7432.32|352.99|0.00|7059.84|7412.83|2071.68| +2452617|31683|8181|21669|1833048|6474|33053|10|138|23|13|4.63|7.96|6.60|0.00|85.80|60.19|103.48|0.00|0.00|85.80|85.80|25.61| +2452617|31683|17631|21669|1833048|6474|33053|10|226|23|16|22.06|39.48|34.34|0.00|549.44|352.96|631.68|16.48|0.00|549.44|565.92|196.48| +2452617|31683|319|21669|1833048|6474|33053|10|80|23|86|66.36|104.18|59.38|0.00|5106.68|5706.96|8959.48|255.33|0.00|5106.68|5362.01|-600.28| +2452617|31683|11220|21669|1833048|6474|33053|10|277|23|91|71.30|89.12|36.53|0.00|3324.23|6488.30|8109.92|99.72|0.00|3324.23|3423.95|-3164.07| +2452617|31683|8413|21669|1833048|6474|33053|10|260|23|1|14.20|14.91|3.42|0.54|3.42|14.20|14.91|0.11|0.54|2.88|2.99|-11.32| +2452617|31683|9097|21669|1833048|6474|33053|10|145|23|81|72.51|122.54|121.31|0.00|9826.11|5873.31|9925.74|196.52|0.00|9826.11|10022.63|3952.80| +2452143|49413|2497|15069|576658|6247|49436|10|58|24|85|57.93|68.35|24.60|0.00|2091.00|4924.05|5809.75|62.73|0.00|2091.00|2153.73|-2833.05| +2452143|49413|4749|15069|576658|6247|49436|10|11|24|28|18.17|23.80|10.23|0.00|286.44|508.76|666.40|17.18|0.00|286.44|303.62|-222.32| +2452143|49413|6751|15069|576658|6247|49436|10|187|24|82|7.48|12.34|2.59|0.00|212.38|613.36|1011.88|0.00|0.00|212.38|212.38|-400.98| +2452143|49413|6043|15069|576658|6247|49436|10|280|24|1|33.41|59.80|29.30|0.00|29.30|33.41|59.80|0.00|0.00|29.30|29.30|-4.11| +2452143|49413|13303|15069|576658|6247|49436|10|101|24|97|94.48|172.89|157.32|0.00|15260.04|9164.56|16770.33|152.60|0.00|15260.04|15412.64|6095.48| +2452143|49413|7375|15069|576658|6247|49436|10|64|24|97|78.55|146.88|73.44|0.00|7123.68|7619.35|14247.36|356.18|0.00|7123.68|7479.86|-495.67| +2452143|49413|10265|15069|576658|6247|49436|10|231|24|93|12.03|23.09|11.31|0.00|1051.83|1118.79|2147.37|42.07|0.00|1051.83|1093.90|-66.96| +2452143|49413|1795|15069|576658|6247|49436|10|264|24|21|76.67|138.00|113.16|0.00|2376.36|1610.07|2898.00|190.10|0.00|2376.36|2566.46|766.29| +2452143|49413|11551|15069|576658|6247|49436|10|17|24|48|30.80|56.05|0.56|0.00|26.88|1478.40|2690.40|1.88|0.00|26.88|28.76|-1451.52| +2452143|49413|5653|15069|576658|6247|49436|10|119|24|92|64.99|67.58|27.03|223.80|2486.76|5979.08|6217.36|181.03|223.80|2262.96|2443.99|-3716.12| +2452143|49413|407|15069|576658|6247|49436|10|56|24|53|98.05|104.91|65.04|0.00|3447.12|5196.65|5560.23|241.29|0.00|3447.12|3688.41|-1749.53| +2452143|49413|1389|15069|576658|6247|49436|10|269|24|72|60.01|118.81|81.97|0.00|5901.84|4320.72|8554.32|59.01|0.00|5901.84|5960.85|1581.12| +2452143|49413|3103|15069|576658|6247|49436|10|235|24|73|44.96|56.20|12.92|0.00|943.16|3282.08|4102.60|47.15|0.00|943.16|990.31|-2338.92| +2452143|49413|4873|15069|576658|6247|49436|10|103|24|41|40.34|73.82|3.69|0.00|151.29|1653.94|3026.62|13.61|0.00|151.29|164.90|-1502.65| +2452143|49413|16483|15069|576658|6247|49436|10|291|24|89|13.84|14.53|14.09|100.32|1254.01|1231.76|1293.17|57.68|100.32|1153.69|1211.37|-78.07| +2452143|49413|4425|15069|576658|6247|49436|10|218|24|57|29.31|39.86|5.18|0.00|295.26|1670.67|2272.02|0.00|0.00|295.26|295.26|-1375.41| +2450920|36458|7874|70838|608693|5395|8050|10|243|25|65|2.75|4.26|2.34|31.94|152.10|178.75|276.90|7.20|31.94|120.16|127.36|-58.59| +2450920|36458|9472|70838|608693|5395|8050|10|148|25|48|4.97|7.70|2.69|0.00|129.12|238.56|369.60|7.74|0.00|129.12|136.86|-109.44| +2450920|36458|4880|70838|608693|5395|8050|10|189|25|29|15.35|19.18|0.57|13.88|16.53|445.15|556.22|0.00|13.88|2.65|2.65|-442.50| +2450920|36458|6955|70838|608693|5395|8050|10|150|25|40|87.12|107.15|40.71|0.00|1628.40|3484.80|4286.00|113.98|0.00|1628.40|1742.38|-1856.40| +2450920|36458|2150|70838|608693|5395|8050|10|270|25|100|67.24|86.73|55.50|0.00|5550.00|6724.00|8673.00|388.50|0.00|5550.00|5938.50|-1174.00| +2450920||14635|||||10||25|68|45.79|50.82|19.81|||3113.72|3455.76|80.82||||| +2450920|36458|14488|70838|608693|5395|8050|10|243|25|26|68.17|94.75|62.53|0.00|1625.78|1772.42|2463.50|65.03|0.00|1625.78|1690.81|-146.64| +2450920|36458|3454|70838|608693|5395|8050|10|9|25|100|1.61|2.83|0.79|73.47|79.00|161.00|283.00|0.05|73.47|5.53|5.58|-155.47| +2450920|36458|10159|70838|608693|5395|8050|10|110|25|24|76.64|96.56|5.79|84.76|138.96|1839.36|2317.44|0.00|84.76|54.20|54.20|-1785.16| +2450920|36458|17752|70838|608693|5395|8050|10|142|25|55|11.49|18.38|11.39|0.00|626.45|631.95|1010.90|0.00|0.00|626.45|626.45|-5.50| +2450920|36458|1333|70838|608693|5395|8050|10|93|25|55|30.82|51.46|26.75|0.00|1471.25|1695.10|2830.30|102.98|0.00|1471.25|1574.23|-223.85| +2450920|36458|17000|70838|608693|5395|8050|10|68|25|40|89.34|173.31|69.32|0.00|2772.80|3573.60|6932.40|83.18|0.00|2772.80|2855.98|-800.80| +2450920|36458|5954||608693|||10|220|25|34|76.57||20.67||702.78|2603.38||||702.78|709.80|-1900.60| +2450920|36458|2608|70838|608693|5395|8050|10|129|25|76|87.75|106.17|46.71|0.00|3549.96|6669.00|8068.92|177.49|0.00|3549.96|3727.45|-3119.04| +2452253|63557|11895|15005|179569|1380|23944|10|209|26|77|36.85|60.06|13.81|0.00|1063.37|2837.45|4624.62|95.70|0.00|1063.37|1159.07|-1774.08| +2452253|63557|15039|15005|179569|1380|23944|10|265|26|34|22.65|42.58|16.60|0.00|564.40|770.10|1447.72|50.79|0.00|564.40|615.19|-205.70| +2452253|63557|5721|15005|179569|1380|23944|10|92|26|81|74.96|95.19|82.81|0.00|6707.61|6071.76|7710.39|335.38|0.00|6707.61|7042.99|635.85| +2452253|63557|5385|15005|179569|1380|23944|10|59|26|97|51.89|86.65|40.72|0.00|3949.84|5033.33|8405.05|0.00|0.00|3949.84|3949.84|-1083.49| +2452253|63557|12851|15005|179569|1380|23944|10|217|26|56|14.02|26.63|4.26|0.00|238.56|785.12|1491.28|19.08|0.00|238.56|257.64|-546.56| +2452253|63557|6647|15005|179569|1380|23944|10|85|26|64|57.04|91.83|0.00|0.00|0.00|3650.56|5877.12|0.00|0.00|0.00|0.00|-3650.56| +2452253|63557|5053|15005|179569|1380|23944|10|200|26|4|19.63|33.56|22.14|0.00|88.56|78.52|134.24|7.08|0.00|88.56|95.64|10.04| +2452253|63557|7337|15005|179569|1380|23944|10|292|26|45|37.59|50.74|38.05|0.00|1712.25|1691.55|2283.30|0.00|0.00|1712.25|1712.25|20.70| +2452253|63557|1989|15005|179569|1380|23944|10|199|26|26|83.31|99.97|36.98|0.00|961.48|2166.06|2599.22|67.30|0.00|961.48|1028.78|-1204.58| +|63557|9679|15005|||23944||114|26|18|77.54|144.22||||1395.72|2595.96|||311.40|326.97|-1084.32| +2451942|67151|12237|32780|1342441|102|48163|1|6|27|89|73.90|138.19|103.64|0.00|9223.96|6577.10|12298.91|737.91|0.00|9223.96|9961.87|2646.86| +2451942|67151|2671|32780|1342441|102|48163|1|275|27|92|4.35|5.17|4.13|0.00|379.96|400.20|475.64|15.19|0.00|379.96|395.15|-20.24| +2451942|67151|15025|32780|1342441|102|48163|1|152|27|26|35.56|37.69|12.06|0.00|313.56|924.56|979.94|28.22|0.00|313.56|341.78|-611.00| +2451942|67151|2137|32780|1342441|102|48163|1|11|27|96|3.07|5.12|4.55|0.00|436.80|294.72|491.52|0.00|0.00|436.80|436.80|142.08| +2451942|67151|16029|32780|1342441|102|48163|1|243|27|59|2.11|3.22|3.22|0.00|189.98|124.49|189.98|17.09|0.00|189.98|207.07|65.49| +2451942|67151|7139|32780|1342441|102|48163|1|156|27|59|6.17|10.11|6.26|0.00|369.34|364.03|596.49|18.46|0.00|369.34|387.80|5.31| +2451942|67151|11441|32780|1342441|102|48163|1|63|27|15|14.45|27.31|2.18|0.00|32.70|216.75|409.65|2.61|0.00|32.70|35.31|-184.05| +2451942|67151|10757|32780|1342441|102|48163|1|142|27|15|8.70|12.18|3.89|0.00|58.35|130.50|182.70|4.66|0.00|58.35|63.01|-72.15| +2451942|67151|1305|32780|1342441|102|48163|1|255|27|44|8.35|13.86|11.91|0.00|524.04|367.40|609.84|0.00|0.00|524.04|524.04|156.64| +2451942|67151|12749|32780|1342441|102|48163|1|174|27|31|80.27|115.58|85.52|0.00|2651.12|2488.37|3582.98|106.04|0.00|2651.12|2757.16|162.75| +2451942|67151|11509|32780|1342441|102|48163|1|219|27|65|80.34|138.98|50.03|0.00|3251.95|5222.10|9033.70|195.11|0.00|3251.95|3447.06|-1970.15| +2451942|67151|16419|32780|1342441|102|48163|1|125|27|65|80.10|155.39|113.43|0.00|7372.95|5206.50|10100.35|516.10|0.00|7372.95|7889.05|2166.45| +2451942|67151|13885|32780|1342441|102|48163|1|103|27|66|40.62|53.61|43.42|0.00|2865.72|2680.92|3538.26|57.31|0.00|2865.72|2923.03|184.80| +2451942|67151|8167|32780|1342441|102|48163|1|214|27|28|38.83|69.89|37.74|0.00|1056.72|1087.24|1956.92|21.13|0.00|1056.72|1077.85|-30.52| +2451942|67151|5831|32780|1342441|102|48163|1|196|27|61|8.79|13.62|2.72|0.00|165.92|536.19|830.82|4.97|0.00|165.92|170.89|-370.27| +2451942|67151|16767|32780|1342441|102|48163|1|241|27|60|68.33|73.11|28.51|0.00|1710.60|4099.80|4386.60|34.21|0.00|1710.60|1744.81|-2389.20| +2452125|68718|6411|20087|1610670|453|31836|4|268|28|3|67.26|112.99|76.83|0.00|230.49|201.78|338.97|4.60|0.00|230.49|235.09|28.71| +2452125|68718|2817|20087|1610670|453|31836|4|100|28|8|98.75|190.58|83.85|429.31|670.80|790.00|1524.64|0.00|429.31|241.49|241.49|-548.51| +2452125|68718|8407|20087|1610670|453|31836|4|286|28|14|95.01|132.06|15.84|0.00|221.76|1330.14|1848.84|15.52|0.00|221.76|237.28|-1108.38| +2452125|68718|4483|20087|1610670|453|31836|4|285|28|78|73.90|141.14|117.14|0.00|9136.92|5764.20|11008.92|456.84|0.00|9136.92|9593.76|3372.72| +2452125|68718|7965|20087|1610670|453|31836|4|275|28|93|77.74|136.04|53.05|296.01|4933.65|7229.82|12651.72|0.00|296.01|4637.64|4637.64|-2592.18| +2452125|68718|2967|20087|1610670|453|31836|4|227|28|29|47.87|52.17|3.13|0.00|90.77|1388.23|1512.93|0.90|0.00|90.77|91.67|-1297.46| +2452125|68718|1807|20087|1610670|453|31836|4|192|28|98|78.91|133.35|132.01|0.00|12936.98|7733.18|13068.30|776.21|0.00|12936.98|13713.19|5203.80| +2452125|68718|5437|20087|1610670|453|31836|4|204|28|15|7.49|8.38|7.20|0.00|108.00|112.35|125.70|6.48|0.00|108.00|114.48|-4.35| +2452125|68718|10295|20087|1610670|453|31836|4|231|28|13|31.83|41.37|7.44|0.00|96.72|413.79|537.81|8.70|0.00|96.72|105.42|-317.07| +2452125|68718|8043|20087|1610670|453|31836|4|196|28|58|60.26|69.90|47.53|0.00|2756.74|3495.08|4054.20|137.83|0.00|2756.74|2894.57|-738.34| +2451121|45001|7654|10631|1267519|54|22989|7|159|29|20|98.14|171.74|171.74|1270.87|3434.80|1962.80|3434.80|0.00|1270.87|2163.93|2163.93|201.13| +2451121|45001|12223|10631|1267519|54|22989|7|56|29|59|12.89|14.05|0.84|33.20|49.56|760.51|828.95|1.47|33.20|16.36|17.83|-744.15| +2451121|45001|15085|10631|||22989|7||29|40|||2.93|0.00|117.20|||4.68|0.00|117.20||-297.60| +2451121|45001|3740|10631|1267519|54|22989|7|110|29|5|90.13|104.55|21.95|0.00|109.75|450.65|522.75|9.87|0.00|109.75|119.62|-340.90| +2451121||1363|||54|22989|7||29|75||97.65|3.90|0.00|292.50||7323.75||0.00|292.50|310.05|| +2451121|45001|12068|10631|1267519|54|22989|7|188|29|37|80.75|95.28|1.90|0.00|70.30|2987.75|3525.36|6.32|0.00|70.30|76.62|-2917.45| +2451121|45001|13340|10631|1267519|54|22989|7|142|29|21|40.50|49.00|29.89|0.00|627.69|850.50|1029.00|50.21|0.00|627.69|677.90|-222.81| +2451121|45001|2930|10631|1267519|54|22989|7|80|29|23|64.78|108.18|27.04|0.00|621.92|1489.94|2488.14|55.97|0.00|621.92|677.89|-868.02| +2451121|45001|15782|10631|1267519|54|22989|7|48|29|53|57.11|98.22|10.80|0.00|572.40|3026.83|5205.66|0.00|0.00|572.40|572.40|-2454.43| +2451121|45001|20|10631|1267519|54|22989|7|240|29|18|66.26|83.48|4.17|0.00|75.06|1192.68|1502.64|3.75|0.00|75.06|78.81|-1117.62| +2451121|45001|15626|10631|||||266|29||17.61||||467.40|1003.77|1374.84|9.34|||476.74|-536.37| +2451121|45001|9458|10631|1267519|54|22989|7|169|29|33|52.33|93.14|88.48|0.00|2919.84|1726.89|3073.62|29.19|0.00|2919.84|2949.03|1192.95| +2451121|45001|10795|10631|1267519|54|22989|7|32|29|33|68.24|116.69|57.17|1056.50|1886.61|2251.92|3850.77|66.40|1056.50|830.11|896.51|-1421.81| +2451121||13693|||||||29||95.63|184.56||0.00|0.00|4398.98|8489.76||0.00|||-4398.98| +2451390|52042|8666|45937|499127|7164|13554|7|36|30|21|64.00|81.28|29.26|0.00|614.46|1344.00|1706.88|36.86|0.00|614.46|651.32|-729.54| +2451390|52042|12572|45937|499127|7164|13554|7|270|30|33|61.96|67.53|14.85|210.72|490.05|2044.68|2228.49|19.55|210.72|279.33|298.88|-1765.35| +2451390|52042|12670|45937|499127|7164|13554|7|62|30|20|6.44|10.23|0.51|0.00|10.20|128.80|204.60|0.71|0.00|10.20|10.91|-118.60| +2451390|52042|3506|45937|499127|7164|13554|7|261|30|15|16.31|27.40|10.13|0.00|151.95|244.65|411.00|12.15|0.00|151.95|164.10|-92.70| +2451390|52042|6752|45937|499127|7164|13554|7|189|30|69|40.80|57.12|19.99|0.00|1379.31|2815.20|3941.28|0.00|0.00|1379.31|1379.31|-1435.89| +2451390|52042|5431|45937|499127|7164|13554|7|150|30|77|52.37|82.22|79.75|2701.93|6140.75|4032.49|6330.94|103.16|2701.93|3438.82|3541.98|-593.67| +2451390|52042|7870|45937|499127|7164|13554|7|256|30|7|4.51|6.85|6.02|18.12|42.14|31.57|47.95|0.00|18.12|24.02|24.02|-7.55| +2451390|52042|13579|45937|499127|7164|13554|7|230|30|75|62.71|77.76|9.33|0.00|699.75|4703.25|5832.00|27.99|0.00|699.75|727.74|-4003.50| +2451390|52042|3928|45937|499127|7164|13554|7|4|30|87|27.01|52.66|29.48|0.00|2564.76|2349.87|4581.42|179.53|0.00|2564.76|2744.29|214.89| +2451390|52042|1951|45937|499127|7164|13554|7|16|30|59|17.14|24.68|21.22|813.78|1251.98|1011.26|1456.12|35.05|813.78|438.20|473.25|-573.06| +2451390|52042|17348|45937|499127|7164|13554|7|57|30|62|88.74|169.49|74.57|1248.30|4623.34|5501.88|10508.38|101.25|1248.30|3375.04|3476.29|-2126.84| +2451390|52042|217|45937|499127|7164|13554|7|51|30|91|52.03|95.73|9.57|365.76|870.87|4734.73|8711.43|35.35|365.76|505.11|540.46|-4229.62| +2451390|52042|14848|45937|499127|7164|13554|7|78|30|62|64.03|103.08|41.23|1150.31|2556.26|3969.86|6390.96|28.11|1150.31|1405.95|1434.06|-2563.91| +2451390|52042|17875|45937|499127|7164|13554|7|90|30|78|2.91|4.56|0.41|0.00|31.98|226.98|355.68|0.95|0.00|31.98|32.93|-195.00| +2451390|52042|3238|45937|499127|7164|13554|7|148|30|16|9.84|15.84|8.55|0.00|136.80|157.44|253.44|1.36|0.00|136.80|138.16|-20.64| +2452187|71970|7513|3748|596752|4156|8760|7|113|31|82|44.95|72.36|42.69|0.00|3500.58|3685.90|5933.52|140.02|0.00|3500.58|3640.60|-185.32| +2452187|71970|14575|3748|596752|4156|8760|7|134|31|97|59.90|119.80|112.61|0.00|10923.17|5810.30|11620.60|218.46|0.00|10923.17|11141.63|5112.87| +2452187|71970|6617|3748|596752|4156|8760|7|75|31|11|86.78|91.98|57.94|0.00|637.34|954.58|1011.78|25.49|0.00|637.34|662.83|-317.24| +2452187|71970|15341|3748|596752|4156|8760|7|265|31|31|16.15|31.16|29.60|0.00|917.60|500.65|965.96|18.35|0.00|917.60|935.95|416.95| +2452187|71970|14727|3748|596752|4156|8760|7|134|31|41|48.10|53.87|8.08|0.00|331.28|1972.10|2208.67|29.81|0.00|331.28|361.09|-1640.82| +2452187|71970|913|3748|596752|4156|8760|7|133|31|54|79.11|93.34|59.73|2386.81|3225.42|4271.94|5040.36|67.08|2386.81|838.61|905.69|-3433.33| +2452187|71970|15411|3748|596752|4156|8760|7|60|31|53|47.64|61.45|30.11|973.45|1595.83|2524.92|3256.85|12.44|973.45|622.38|634.82|-1902.54| +2452187|71970|4963|3748|596752|4156|8760|7|103|31|67|56.37|59.18|37.28|1873.32|2497.76|3776.79|3965.06|0.00|1873.32|624.44|624.44|-3152.35| +2452187|71970|16251|3748|596752|4156|8760|7|65|31|51|91.49|182.06|138.36|0.00|7056.36|4665.99|9285.06|211.69|0.00|7056.36|7268.05|2390.37| +2452187|71970|6917|3748|596752|4156|8760|7|46|31|4|49.76|94.04|51.72|0.00|206.88|199.04|376.16|14.48|0.00|206.88|221.36|7.84| +2452187|71970|11739|3748|596752|4156|8760|7|83|31|95|6.99|11.32|5.09|0.00|483.55|664.05|1075.40|43.51|0.00|483.55|527.06|-180.50| +2451840|45877|4339|90120|1101094|6468|1395|4|107|32|6|88.27|139.46|26.49|0.00|158.94|529.62|836.76|7.94|0.00|158.94|166.88|-370.68| +2451840|45877|9683||1101094|6468|1395|4||32|26||149.26|2.98||77.48|2425.54||1.54||77.48||-2348.06| +2451840|45877|12134|90120|1101094|6468|1395|4|212|32|21|51.01|63.25|30.99|0.00|650.79|1071.21|1328.25|52.06|0.00|650.79|702.85|-420.42| +2451840||4537|90120||||4|202|32|22|65.72|69.00|57.27|0.00|1259.94||1518.00||0.00|1259.94||-185.90| +2451840|45877|4808|||6468||||32||57.01||||69.72|798.14|1396.64|||69.72|73.90|| +2451840|45877|5798|90120|1101094|6468|1395|4|40|32|87|46.23|91.99|55.19|4417.40|4801.53|4022.01|8003.13|11.52|4417.40|384.13|395.65|-3637.88| +2451840|45877|17659|90120|1101094|6468|1395|4|172|32|51|23.88|31.04|29.79|0.00|1519.29|1217.88|1583.04|45.57|0.00|1519.29|1564.86|301.41| +2451840|45877|15001|90120|1101094|6468|1395|4|181|32|57|30.07|54.42|23.94|0.00|1364.58|1713.99|3101.94|13.64|0.00|1364.58|1378.22|-349.41| +2451840|45877|11005|90120|1101094|6468|1395|4|174|32|46|51.48|79.27|72.92|1207.55|3354.32|2368.08|3646.42|21.46|1207.55|2146.77|2168.23|-221.31| +2451840|45877|15644|90120|1101094|6468|1395|4|123|32|34|80.54|131.28|89.27|0.00|3035.18|2738.36|4463.52|91.05|0.00|3035.18|3126.23|296.82| +2451840|45877|16421|90120|1101094|6468|1395|4|77|32|74|89.89|160.90|152.85|0.00|11310.90|6651.86|11906.60|113.10|0.00|11310.90|11424.00|4659.04| +2451840|45877|2095|90120|1101094|6468|1395|4|282|32|34|89.33|151.86|66.81|0.00|2271.54|3037.22|5163.24|0.00|0.00|2271.54|2271.54|-765.68| +2451840|45877|7547|90120|1101094|6468|1395|4|194|32|24|43.33|68.46|34.23|0.00|821.52|1039.92|1643.04|16.43|0.00|821.52|837.95|-218.40| +2451840|45877|2887|90120|1101094|6468|1395|4|108|32|8|48.71|96.93|32.95|0.00|263.60|389.68|775.44|15.81|0.00|263.60|279.41|-126.08| +2451840|45877|1115|90120|1101094|6468|1395|4|275|32|61|97.03|172.71|63.90|0.00|3897.90|5918.83|10535.31|194.89|0.00|3897.90|4092.79|-2020.93| +2451840|45877|11348|90120|1101094|6468|1395|4|274|32|41|55.14|57.34|51.03|753.20|2092.23|2260.74|2350.94|107.12|753.20|1339.03|1446.15|-921.71| +2451110|42673|7300|26724|445739|1201|4616|4|203|33|3|3.13|4.00|2.48|0.00|7.44|9.39|12.00|0.14|0.00|7.44|7.58|-1.95| +2451110|42673|15478|26724|445739|1201|4616|4|273|33|22|95.96|149.69|53.88|0.00|1185.36|2111.12|3293.18|82.97|0.00|1185.36|1268.33|-925.76| +2451110|42673|9649|26724|445739|1201|4616|4|233|33|36|18.91|31.01|22.63|0.00|814.68|680.76|1116.36|57.02|0.00|814.68|871.70|133.92| +2451110|42673|12964|26724|445739|1201|4616|4|266|33|80|83.86|114.04|25.08|0.00|2006.40|6708.80|9123.20|180.57|0.00|2006.40|2186.97|-4702.40| +2451110|42673|14692|26724|445739|1201|4616|4|190|33|93|56.78|107.88|90.61|0.00|8426.73|5280.54|10032.84|252.80|0.00|8426.73|8679.53|3146.19| +2451110|42673|12760|26724|445739|1201|4616|4|256|33|54|48.97|55.33|39.83|1161.44|2150.82|2644.38|2987.82|39.57|1161.44|989.38|1028.95|-1655.00| +2451110|42673|11119|26724|445739|1201|4616|4|242|33|92|3.49|3.69|1.95|0.00|179.40|321.08|339.48|7.17|0.00|179.40|186.57|-141.68| +2451110|42673|11756|26724|445739|1201|4616|4|121|33|26|58.87|89.48|57.26|0.00|1488.76|1530.62|2326.48|89.32|0.00|1488.76|1578.08|-41.86| +2451110|42673|14125|26724|445739|1201|4616|4|206|33|66|44.03|46.23|16.64|109.82|1098.24|2905.98|3051.18|19.76|109.82|988.42|1008.18|-1917.56| +2451110|42673|4798|26724|445739|1201|4616|4|178|33|27|28.56|37.98|13.67|350.63|369.09|771.12|1025.46|0.00|350.63|18.46|18.46|-752.66| +2451110|42673|14158|26724|445739|1201|4616|4|26|33|82|48.07|74.50|23.84|0.00|1954.88|3941.74|6109.00|0.00|0.00|1954.88|1954.88|-1986.86| +2451110|42673|12643|26724|445739|1201|4616|4|77|33|89|35.74|46.10|45.63|0.00|4061.07|3180.86|4102.90|81.22|0.00|4061.07|4142.29|880.21| +2451110|42673|10376|26724|445739|1201|4616|4|296|33|21|55.09|110.18|15.42|0.00|323.82|1156.89|2313.78|29.14|0.00|323.82|352.96|-833.07| +2451342|71555|9325|47971|1238972|6773|30574|7|173|34|3|92.35|109.89|48.35|37.71|145.05|277.05|329.67|6.44|37.71|107.34|113.78|-169.71| +2451342|71555|9766|47971|1238972|6773|30574|7|86|34|83|68.57|103.54|70.40|0.00|5843.20|5691.31|8593.82|467.45|0.00|5843.20|6310.65|151.89| +2451342|71555|12512|47971|1238972|6773|30574|7|63|34|60|40.48|78.93|22.10|0.00|1326.00|2428.80|4735.80|39.78|0.00|1326.00|1365.78|-1102.80| +2451342|71555|1717|47971|1238972|6773|30574|7|289|34|53|99.68|128.58|83.57|0.00|4429.21|5283.04|6814.74|265.75|0.00|4429.21|4694.96|-853.83| +2451342|71555|6193|47971|1238972|6773|30574|7|13|34|61|54.55|102.55|34.86|276.43|2126.46|3327.55|6255.55|166.50|276.43|1850.03|2016.53|-1477.52| +2451342||12290|47971|1238972|6773||7||34|53|88.61|108.99|||||5776.47|60.03||750.48||-3945.85| +2451342|71555|1526|47971|1238972|6773|30574|7|237|34|91|78.12|90.61|90.61|0.00|8245.51|7108.92|8245.51|82.45|0.00|8245.51|8327.96|1136.59| +2451342|71555|12016|47971|1238972|6773|30574|7|276|34|42|42.44|69.17|38.73|0.00|1626.66|1782.48|2905.14|146.39|0.00|1626.66|1773.05|-155.82| +2451342|71555|16324|47971|1238972|6773|30574|7|273|34|30|37.27|57.39|16.06|0.00|481.80|1118.10|1721.70|38.54|0.00|481.80|520.34|-636.30| +2451342|71555|4118|47971|1238972|6773|30574|7|121|34|88|38.38|74.84|71.84|0.00|6321.92|3377.44|6585.92|126.43|0.00|6321.92|6448.35|2944.48| +2451342|71555|5197|47971|1238972|6773|30574|7|190|34|63|13.50|26.46|17.19|0.00|1082.97|850.50|1666.98|32.48|0.00|1082.97|1115.45|232.47| +2451342|71555|2312|47971|1238972|6773|30574|7|231|34|6|51.40|78.64|20.44|0.00|122.64|308.40|471.84|7.35|0.00|122.64|129.99|-185.76| +||13814|47971||6773|30574|7|286|34|20|22.82||6.74||134.80|||||||| +2451342|71555|5449|47971|1238972|6773|30574|7|208|34|9|21.24|28.03|4.48|0.00|40.32|191.16|252.27|3.62|0.00|40.32|43.94|-150.84| +2452150|37970|15619|93958|616532|3106|34927|8|236|35|45|90.28|102.01|40.80|0.00|1836.00|4062.60|4590.45|36.72|0.00|1836.00|1872.72|-2226.60| +2452150|37970|17451|93958|616532|3106|34927|8|258|35|7|45.66|59.35|57.56|0.00|402.92|319.62|415.45|12.08|0.00|402.92|415.00|83.30| +2452150|37970|2377|93958|616532|3106|34927|8|2|35|52|98.03|140.18|2.80|90.27|145.60|5097.56|7289.36|0.55|90.27|55.33|55.88|-5042.23| +2452150|37970|5179|93958|616532|3106|34927|8|212|35|83|90.54|120.41|2.40|0.00|199.20|7514.82|9994.03|0.00|0.00|199.20|199.20|-7315.62| +2452150|37970|15757|93958|616532|3106|34927|8|68|35|9|14.83|27.43|0.27|0.00|2.43|133.47|246.87|0.19|0.00|2.43|2.62|-131.04| +2452150|37970|3667|93958|616532|3106|34927|8|250|35|97|59.31|60.49|48.99|3136.33|4752.03|5753.07|5867.53|32.31|3136.33|1615.70|1648.01|-4137.37| +2452150|37970|17341|93958|616532|3106|34927|8|287|35|92|59.48|116.58|68.78|0.00|6327.76|5472.16|10725.36|569.49|0.00|6327.76|6897.25|855.60| +2452150|37970|4325|93958|616532|3106|34927|8|25|35|56|67.43|126.76|115.35|0.00|6459.60|3776.08|7098.56|387.57|0.00|6459.60|6847.17|2683.52| +2452150|37970|11635|93958|616532|3106|34927|8|109|35|87|92.02|175.75|49.21|0.00|4281.27|8005.74|15290.25|342.50|0.00|4281.27|4623.77|-3724.47| +2452150|37970|14239|93958|616532|3106|34927|8|171|35|55|8.27|10.08|1.10|0.00|60.50|454.85|554.40|0.60|0.00|60.50|61.10|-394.35| +2452150|37970|11661||616532|3106|34927|8||35|81|||27.75|||||89.91|||2337.66|223.56| +2452150||411|93958|616532|3106||||35|51||73.41||0.00|1647.30|1940.04||131.78|0.00|||-292.74| +2452150|37970|17365|93958|616532|3106|34927|8|202|35|65|76.20|104.39|69.94|0.00|4546.10|4953.00|6785.35|363.68|0.00|4546.10|4909.78|-406.90| +2452274|74196|17659|72008|631970|1532|24694|8|286|36|91|44.83|73.52|25.73|0.00|2341.43|4079.53|6690.32|140.48|0.00|2341.43|2481.91|-1738.10| +2452274|74196|15001|72008|631970|1532|24694|8|159|36|54|16.24|26.14|22.21|0.00|1199.34|876.96|1411.56|95.94|0.00|1199.34|1295.28|322.38| +2452274|74196|11005|72008|631970|1532|24694|8|127|36|49|70.60|113.66|56.83|0.00|2784.67|3459.40|5569.34|83.54|0.00|2784.67|2868.21|-674.73| +2452274|74196|15645|72008|631970|1532|24694|8|136|36|23|32.35|34.93|4.54|0.00|104.42|744.05|803.39|7.30|0.00|104.42|111.72|-639.63| +2452274|74196|16421|72008|631970|1532|24694|8|144|36|25|69.67|104.50|77.33|0.00|1933.25|1741.75|2612.50|154.66|0.00|1933.25|2087.91|191.50| +2452274|74196|2095|72008|631970|1532|24694|8|147|36|43|91.17|165.01|57.75|0.00|2483.25|3920.31|7095.43|223.49|0.00|2483.25|2706.74|-1437.06| +2452274|74196|7547|72008|631970|1532|24694|8|240|36|46|8.04|15.83|10.60|0.00|487.60|369.84|728.18|39.00|0.00|487.60|526.60|117.76| +2452274|74196|2887|72008|631970|1532|24694|8|97|36|31|24.53|47.09|14.12|0.00|437.72|760.43|1459.79|26.26|0.00|437.72|463.98|-322.71| +2452274|74196|1115|72008|631970|1532|24694|8|45|36|80|11.13|12.46|6.72|198.91|537.60|890.40|996.80|10.16|198.91|338.69|348.85|-551.71| +2452274|74196|11349|72008|631970|1532|24694|8|65|36|80|58.17|94.23|15.07|0.00|1205.60|4653.60|7538.40|36.16|0.00|1205.60|1241.76|-3448.00| +2452274|74196|17561|72008|631970|1532|24694|8|115|36|16|82.46|157.49|17.32|0.00|277.12|1319.36|2519.84|13.85|0.00|277.12|290.97|-1042.24| +2452087|61560|10715|96252|1823714|4423|24685|10|116|37|52|89.22|133.83|111.07|0.00|5775.64|4639.44|6959.16|115.51|0.00|5775.64|5891.15|1136.20| +2452087|61560|16971|96252|1823714|4423|24685|10|236|37|12|53.97|85.27|15.34|0.00|184.08|647.64|1023.24|0.00|0.00|184.08|184.08|-463.56| +2452087|61560|10929|96252|1823714|4423|24685|10|240|37|88|65.45|80.50|47.49|0.00|4179.12|5759.60|7084.00|292.53|0.00|4179.12|4471.65|-1580.48| +2452087|61560|2997|96252|1823714|4423|24685|10|92|37|94|85.67|143.92|100.74|0.00|9469.56|8052.98|13528.48|568.17|0.00|9469.56|10037.73|1416.58| +2452087|61560|13171|96252|1823714|4423|24685|10|150|37|6|84.14|159.86|110.30|0.00|661.80|504.84|959.16|26.47|0.00|661.80|688.27|156.96| +2452087|61560|7283|96252|1823714|4423|24685|10|13|37|87|54.25|60.76|1.21|0.00|105.27|4719.75|5286.12|4.21|0.00|105.27|109.48|-4614.48| +2452087|61560|15337|96252|1823714|4423|24685|10|217|37|62|16.64|32.94|9.88|0.00|612.56|1031.68|2042.28|24.50|0.00|612.56|637.06|-419.12| +2452087|61560|17125||1823714||24685|10||37||||74.97|0.00||5369.06||173.93|0.00||4522.19|| +2451476|51103|9710|16409|414090|2171|25452|8|211|38|7|82.77|91.04|54.62|275.28|382.34|579.39|637.28|2.14|275.28|107.06|109.20|-472.33| +2451476|51103|6220|16409|414090|2171|25452|8|265|38|98|14.54|17.88|10.54|733.37|1032.92|1424.92|1752.24|5.99|733.37|299.55|305.54|-1125.37| +2451476|51103|7784|16409|414090|2171|25452|8|102|38|56|31.89|37.63|24.83|889.90|1390.48|1785.84|2107.28|0.00|889.90|500.58|500.58|-1285.26| +2451476|51103|10441|16409|414090|2171|25452|8|55|38|62|80.37|114.12|59.34|0.00|3679.08|4982.94|7075.44|36.79|0.00|3679.08|3715.87|-1303.86| +2451476|51103|2164|16409|414090|2171|25452|8|20|38|17|72.04|105.17|7.36|111.35|125.12|1224.68|1787.89|0.55|111.35|13.77|14.32|-1210.91| +||757|||2171||8||38|2||||0.00||190.42|239.92|6.38|0.00|91.16|97.54|| +2451476|51103|8870|16409|414090|2171|25452|8|33|38|15|46.69|85.90|10.30|0.00|154.50|700.35|1288.50|12.36|0.00|154.50|166.86|-545.85| +2451476|51103|3439|16409|414090|2171|25452|8|100|38|84|11.71|20.02|10.41|0.00|874.44|983.64|1681.68|43.72|0.00|874.44|918.16|-109.20| +2451476|51103|7018|16409|414090|2171|25452|8|115|38|15|69.78|133.27|129.27|0.00|1939.05|1046.70|1999.05|96.95|0.00|1939.05|2036.00|892.35| +2451476|51103|5113|16409|414090|2171|25452|8|236|38|73|50.59|84.99|79.89|0.00|5831.97|3693.07|6204.27|291.59|0.00|5831.97|6123.56|2138.90| +2451476|51103|4154|16409|414090|2171|25452|8|12|38|35|10.28|14.49|3.33|0.00|116.55|359.80|507.15|8.15|0.00|116.55|124.70|-243.25| +2451476|51103|15698|16409|414090|2171|25452|8|38|38|57|11.40|15.16|12.58|0.00|717.06|649.80|864.12|0.00|0.00|717.06|717.06|67.26| +2451874|48754|3476|58896|1112127|5358|8989|10|163|39|73|80.57|132.94|130.28|0.00|9510.44|5881.61|9704.62|95.10|0.00|9510.44|9605.54|3628.83| +2451874|48754|7697|58896|1112127|5358|8989|10|89|39|24|44.89|79.45|70.71|0.00|1697.04|1077.36|1906.80|152.73|0.00|1697.04|1849.77|619.68| +2451874|48754|4190|58896|1112127|5358|8989|10|288|39|86|35.56|37.33|29.49|0.00|2536.14|3058.16|3210.38|101.44|0.00|2536.14|2637.58|-522.02| +2451874|48754|386|58896|1112127|5358|8989|10|258|39|89|28.08|44.08|36.58|0.00|3255.62|2499.12|3923.12|260.44|0.00|3255.62|3516.06|756.50| +2451874|48754|8411|58896|1112127|5358|8989|10|226|39|96|35.69|54.60|50.23|0.00|4822.08|3426.24|5241.60|241.10|0.00|4822.08|5063.18|1395.84| +2451874|48754|7097|58896|1112127|5358|8989|10|186|39|78|33.10|51.96|27.53|0.00|2147.34|2581.80|4052.88|0.00|0.00|2147.34|2147.34|-434.46| +2451874|48754|1598|58896|1112127|5358|8989|10|254|39|64|44.63|54.44|15.24|0.00|975.36|2856.32|3484.16|48.76|0.00|975.36|1024.12|-1880.96| +2451874|48754|8078|58896|1112127|5358|8989|10|198|39|54|73.60|113.34|73.67|1352.58|3978.18|3974.40|6120.36|183.79|1352.58|2625.60|2809.39|-1348.80| +2451874|48754|4957|58896|1112127|5358|8989|10|72|39|24|16.10|22.37|10.96|0.00|263.04|386.40|536.88|5.26|0.00|263.04|268.30|-123.36| +2451874|48754|7604|58896|1112127|5358|8989|10|140|39|49|46.49|49.27|49.27|0.00|2414.23|2278.01|2414.23|48.28|0.00|2414.23|2462.51|136.22| +2451874|48754|15491|58896|1112127|5358|8989|10|34|39|54|3.20|4.80|2.01|103.11|108.54|172.80|259.20|0.16|103.11|5.43|5.59|-167.37| +2451874|48754|3943|58896|1112127|5358|8989|10|213|39|64|59.68|85.93|16.32|490.90|1044.48|3819.52|5499.52|16.60|490.90|553.58|570.18|-3265.94| +2451874|48754|5393|58896|1112127|5358|8989|10|45|39|98|58.75|67.56|20.94|0.00|2052.12|5757.50|6620.88|123.12|0.00|2052.12|2175.24|-3705.38| +2451874|48754|15625|58896|1112127|5358|8989|10|63|39|17|96.62|157.49|108.66|0.00|1847.22|1642.54|2677.33|36.94|0.00|1847.22|1884.16|204.68| +2451874|48754|7118|58896|1112127|5358|8989|10|161|39|67|68.99|101.41|31.43|0.00|2105.81|4622.33|6794.47|21.05|0.00|2105.81|2126.86|-2516.52| +2450925|62048|5098|66489|634761|2233|49705|4|90|40|35|33.91|41.70|22.93|497.58|802.55|1186.85|1459.50|21.34|497.58|304.97|326.31|-881.88| +2450925|62048|10094|66489|634761|2233|49705|4|64|40|80|8.63|9.92|8.63|669.68|690.40|690.40|793.60|1.24|669.68|20.72|21.96|-669.68| +2450925|62048|13658|66489|634761|2233|49705|4|21|40|53|56.42|112.27|33.68|0.00|1785.04|2990.26|5950.31|107.10|0.00|1785.04|1892.14|-1205.22| +2450925|62048|3985|66489|634761|2233|49705|4|280|40|63|22.94|37.16|0.00|0.00|0.00|1445.22|2341.08|0.00|0.00|0.00|0.00|-1445.22| +2450925|62048|5318|66489|634761|2233|49705|4|190|40|87|32.66|44.09|22.92|0.00|1994.04|2841.42|3835.83|119.64|0.00|1994.04|2113.68|-847.38| +||12050|66489||2233|49705|4|258|40||38.12||62.75|0.00|||3522.20|103.53|0.00||3554.78|| +2450925|62048|2854|66489|634761|2233|49705|4|196|40|71|10.62|19.85|2.77|33.43|196.67|754.02|1409.35|13.05|33.43|163.24|176.29|-590.78| +2450925|62048|16976|66489|634761|2233|49705|4|208|40|3|20.70|28.77|8.34|0.00|25.02|62.10|86.31|2.00|0.00|25.02|27.02|-37.08| +2450925|62048|3490|66489|634761|2233|49705|4|275|40|65|41.24|66.80|45.42|0.00|2952.30|2680.60|4342.00|206.66|0.00|2952.30|3158.96|271.70| +2450925|62048|10912|66489|634761|2233|49705|4|264|40|23|2.46|4.84|3.92|9.91|90.16|56.58|111.32|6.42|9.91|80.25|86.67|23.67| +2451094|69987|7402|87890|1540338|4786|12434|10|59|41|69|57.23|97.29|65.18|0.00|4497.42|3948.87|6713.01|224.87|0.00|4497.42|4722.29|548.55| +2451094|69987|3380|87890|1540338|4786|12434|10|195|41|88|14.11|15.37|13.06|0.00|1149.28|1241.68|1352.56|11.49|0.00|1149.28|1160.77|-92.40| +2451094|69987|15242|87890|1540338|4786|12434|10|141|41|87|48.25|59.34|20.17|0.00|1754.79|4197.75|5162.58|122.83|0.00|1754.79|1877.62|-2442.96| +2451094|69987|10|87890|1540338|4786|12434|10|275|41|50|54.36|102.74|78.08|0.00|3904.00|2718.00|5137.00|234.24|0.00|3904.00|4138.24|1186.00| +2451094|69987|10915|87890|1540338|4786|12434|10|256|41|81|91.63|174.09|19.14|201.54|1550.34|7422.03|14101.29|121.39|201.54|1348.80|1470.19|-6073.23| +2451094|69987|5566|87890|1540338|4786|12434|10|254|41|11|50.45|75.67|10.59|0.00|116.49|554.95|832.37|6.98|0.00|116.49|123.47|-438.46| +2451094|69987|9322|87890|1540338|4786|12434|10|276|41|8|59.40|87.31|82.94|0.00|663.52|475.20|698.48|19.90|0.00|663.52|683.42|188.32| +2451094|69987|16514|87890|1540338|4786|12434|10|285|41|20|5.05|5.20|5.09|78.38|101.80|101.00|104.00|0.46|78.38|23.42|23.88|-77.58| +2451094|69987|15328|87890|1540338|4786|12434|10|8|41|46|84.03|168.06|70.58|0.00|3246.68|3865.38|7730.76|227.26|0.00|3246.68|3473.94|-618.70| +2451094|69987|7603|87890|1540338|4786|12434|10|166|41|94|6.12|9.91|3.17|0.00|297.98|575.28|931.54|23.83|0.00|297.98|321.81|-277.30| +2451094|69987|6310|87890|1540338|4786|12434|10|202|41|90|60.10|63.70|17.83|978.86|1604.70|5409.00|5733.00|25.03|978.86|625.84|650.87|-4783.16| +2451094|69987|14788|87890|1540338|4786|12434|10|222|41|15|90.04|108.04|70.22|0.00|1053.30|1350.60|1620.60|31.59|0.00|1053.30|1084.89|-297.30| +2451094|69987|64|87890|1540338|4786|12434|10|286|41|29|27.18|52.72|4.21|120.86|122.09|788.22|1528.88|0.06|120.86|1.23|1.29|-786.99| +2450952|40089|2314|49423|170973|3276|11226|10|16|42|74|24.46|39.38|16.93|0.00|1252.82|1810.04|2914.12|87.69|0.00|1252.82|1340.51|-557.22| +2450952|40089|10016|49423|170973|3276|11226|10|76|42|57|12.47|13.96|1.39|0.00|79.23|710.79|795.72|4.75|0.00|79.23|83.98|-631.56| +2450952|40089|3346|49423|170973|3276|11226|10|191|42|74|29.72|48.74|36.55|0.00|2704.70|2199.28|3606.76|0.00|0.00|2704.70|2704.70|505.42| +2450952|40089|976|49423|170973|3276|11226|10|21|42|100|12.59|19.13|8.22|0.00|822.00|1259.00|1913.00|24.66|0.00|822.00|846.66|-437.00| +2450952|40089|10471|49423|170973|3276|11226|10|13|42|19|42.67|42.67|10.24|0.00|194.56|810.73|810.73|15.56|0.00|194.56|210.12|-616.17| +2450952|40089|6058|49423|170973|3276|11226|10|120|42|30|81.10|139.49|33.47|0.00|1004.10|2433.00|4184.70|50.20|0.00|1004.10|1054.30|-1428.90| +2450952|40089|12550|49423|170973|3276|11226|10|199|42|41|17.09|22.90|6.41|0.00|262.81|700.69|938.90|7.88|0.00|262.81|270.69|-437.88| +2450952|40089|2392|49423|170973|3276|11226|10|163|42|14|49.48|72.24|55.62|0.00|778.68|692.72|1011.36|31.14|0.00|778.68|809.82|85.96| +2450952|40089|15002|49423|170973|3276|11226|10|21|42|41|58.33|80.49|38.63|0.00|1583.83|2391.53|3300.09|126.70|0.00|1583.83|1710.53|-807.70| +2450952|40089|6134|49423|170973|3276|11226|10|146|42|92|18.91|33.09|23.82|0.00|2191.44|1739.72|3044.28|109.57|0.00|2191.44|2301.01|451.72| +2450952|40089|3613|49423|170973|3276|11226|10|8|42|30|56.33|65.34|56.84|528.61|1705.20|1689.90|1960.20|0.00|528.61|1176.59|1176.59|-513.31| +2450952|40089|9740|49423|170973|3276|11226|10|268|42|52|52.46|90.23|32.48|1621.40|1688.96|2727.92|4691.96|1.35|1621.40|67.56|68.91|-2660.36| +2450952|40089|619|49423|170973|3276|11226|10|55|42|69|56.85|80.15|28.85|0.00|1990.65|3922.65|5530.35|0.00|0.00|1990.65|1990.65|-1932.00| +2450952|40089|1436|49423|170973|3276|11226|10|7|42|94|54.21|81.31|36.58|0.00|3438.52|5095.74|7643.14|68.77|0.00|3438.52|3507.29|-1657.22| +2450952|40089|8462|49423|170973|3276|11226|10|241|42|23|27.88|33.45|7.02|24.21|161.46|641.24|769.35|0.00|24.21|137.25|137.25|-503.99| +2450952|40089|2602|49423|170973|3276|11226|10|96|42|30|55.77|71.38|63.52|0.00|1905.60|1673.10|2141.40|76.22|0.00|1905.60|1981.82|232.50| +2451878|68553|5852|80811|1454478|5392|12259|4|1|43|16|81.99|84.44|81.06|0.00|1296.96|1311.84|1351.04|51.87|0.00|1296.96|1348.83|-14.88| +2451878|68553|14669|80811|1454478|5392|12259|4|269|43|97|52.94|98.99|12.86|0.00|1247.42|5135.18|9602.03|62.37|0.00|1247.42|1309.79|-3887.76| +2451878|68553|3911|80811|1454478|5392|12259|4|55|43|26|21.75|36.97|34.01|0.00|884.26|565.50|961.22|61.89|0.00|884.26|946.15|318.76| +2451878|68553|2923|80811|1454478|5392|12259|4|274|43|16|82.12|136.31|80.42|0.00|1286.72|1313.92|2180.96|51.46|0.00|1286.72|1338.18|-27.20| +2451878|68553|3344|80811|1454478|5392|12259|4|47|43|22|88.77|109.18|82.97|0.00|1825.34|1952.94|2401.96|0.00|0.00|1825.34|1825.34|-127.60| +2451878|68553|11771|80811|1454478|5392|12259|4|173|43|30|41.13|46.88|41.25|420.75|1237.50|1233.90|1406.40|49.00|420.75|816.75|865.75|-417.15| +2451878|68553|5773|80811|1454478|5392|12259|4|220|43|85|66.42|87.67|77.14|0.00|6556.90|5645.70|7451.95|0.00|0.00|6556.90|6556.90|911.20| +2451878|68553|4691|80811|1454478|5392|12259|4|63|43|41|2.24|4.39|0.00|0.00|0.00|91.84|179.99|0.00|0.00|0.00|0.00|-91.84| +2451878|68553|4364|80811|1454478|5392|12259|4|266|43|77|82.92|106.96|14.97|0.00|1152.69|6384.84|8235.92|11.52|0.00|1152.69|1164.21|-5232.15| +2452274|44359|7527|53792|336661|4330|25999|7|110|44|67|90.05|124.26|93.19|0.00|6243.73|6033.35|8325.42|437.06|0.00|6243.73|6680.79|210.38| +2452274|44359|2351|53792|336661|4330|25999|7|233|44|56|55.53|96.06|35.54|1850.92|1990.24|3109.68|5379.36|11.14|1850.92|139.32|150.46|-2970.36| +2452274|44359|2623|53792|336661|4330|25999|7|2|44|18|39.17|74.03|65.14|0.00|1172.52|705.06|1332.54|82.07|0.00|1172.52|1254.59|467.46| +2452274|44359|9059|53792|336661|4330|25999|7|173|44|68|30.11|50.28|18.60|0.00|1264.80|2047.48|3419.04|50.59|0.00|1264.80|1315.39|-782.68| +2452274|44359|12341|53792|336661|4330|25999|7|127|44|20|82.28|124.24|73.30|0.00|1466.00|1645.60|2484.80|14.66|0.00|1466.00|1480.66|-179.60| +2452274|44359|14149|53792|336661|4330|25999|7|164|44|80|18.83|22.40|16.35|0.00|1308.00|1506.40|1792.00|78.48|0.00|1308.00|1386.48|-198.40| +2452274|44359|15803|53792|336661|4330|25999|7|295|44|81|43.81|83.23|4.99|0.00|404.19|3548.61|6741.63|12.12|0.00|404.19|416.31|-3144.42| +2452274|44359|16491|53792|336661|4330|25999|7|103|44|56|32.28|53.26|13.31|0.00|745.36|1807.68|2982.56|52.17|0.00|745.36|797.53|-1062.32| +2452274|44359|13331|53792|336661|4330|25999|7|240|44|98|3.53|6.67|0.20|0.00|19.60|345.94|653.66|0.00|0.00|19.60|19.60|-326.34| +2452274|44359|16909|53792|336661|4330|25999|7|288|44|61|92.15|148.36|45.99|0.00|2805.39|5621.15|9049.96|56.10|0.00|2805.39|2861.49|-2815.76| +2452274|44359|11707|53792|336661|4330|25999|7|155|44|83|85.49|126.52|41.75|138.61|3465.25|7095.67|10501.16|232.86|138.61|3326.64|3559.50|-3769.03| +2452274|44359|13449|53792|336661|4330|25999|7|71|44|45|50.83|86.91|86.04|0.00|3871.80|2287.35|3910.95|309.74|0.00|3871.80|4181.54|1584.45| +2452274|44359|16837|53792|336661|4330|25999|7|265|44|92|30.11|37.63|35.37|0.00|3254.04|2770.12|3461.96|260.32|0.00|3254.04|3514.36|483.92| +2452274|44359|7303|53792|336661|4330|25999|7|50|44|14|36.13|62.14|19.26|0.00|269.64|505.82|869.96|18.87|0.00|269.64|288.51|-236.18| +2452257|37558|10115|16606|596784|6209|42787|4|37|45|68|38.09|58.65|7.62|310.89|518.16|2590.12|3988.20|0.00|310.89|207.27|207.27|-2382.85| +2452257|37558|16349|16606|596784|6209|42787|4|213|45|6|94.59|150.39|99.25|0.00|595.50|567.54|902.34|0.00|0.00|595.50|595.50|27.96| +2452257|37558|11095|16606|596784|6209|42787|4|235|45|40|52.97|90.04|69.33|0.00|2773.20|2118.80|3601.60|249.58|0.00|2773.20|3022.78|654.40| +2452257|37558|1479|16606|596784|6209|42787|4|299|45|49|5.01|8.91|0.00|0.00|0.00|245.49|436.59|0.00|0.00|0.00|0.00|-245.49| +2452257|37558|7363|16606|596784|6209|42787|4|137|45|87|85.95|87.66|4.38|0.00|381.06|7477.65|7626.42|7.62|0.00|381.06|388.68|-7096.59| +2452257|37558|13133|16606|596784|6209|42787|4|170|45|46|85.87|126.22|68.15|0.00|3134.90|3950.02|5806.12|219.44|0.00|3134.90|3354.34|-815.12| +2452257|37558|5309|16606|596784|6209|42787|4|44|45|18|51.16|83.39|21.68|0.00|390.24|920.88|1501.02|31.21|0.00|390.24|421.45|-530.64| +2452257|37558|811|16606|596784|6209|42787|4|265|45|62|23.41|29.26|4.09|0.00|253.58|1451.42|1814.12|12.67|0.00|253.58|266.25|-1197.84| +2452257|37558|3265|16606|596784|6209|42787|4|246|45|98|27.12|40.13|31.70|0.00|3106.60|2657.76|3932.74|279.59|0.00|3106.60|3386.19|448.84| +2451299|47778|15595|34598|1011655|6312|14569|8|53|46|32|29.12|35.81|12.17|0.00|389.44|931.84|1145.92|0.00|0.00|389.44|389.44|-542.40| +2451299|47778|17384|34598|1011655|6312|14569|8|266|46|13|93.68|140.52|25.29|0.00|328.77|1217.84|1826.76|3.28|0.00|328.77|332.05|-889.07| +2451299|47778|17150|34598|1011655|6312|14569|8|221|46|57|71.68|115.40|79.62|0.00|4538.34|4085.76|6577.80|181.53|0.00|4538.34|4719.87|452.58| +2451299|47778|1960|34598|1011655|6312|14569|8|133|46|12|53.47|92.50|73.07|0.00|876.84|641.64|1110.00|26.30|0.00|876.84|903.14|235.20| +2451299|47778|7040|34598|1011655|6312|14569|8|279|46|33|90.87|97.23|25.27|0.00|833.91|2998.71|3208.59|33.35|0.00|833.91|867.26|-2164.80| +2451299|47778|16540|34598|1011655|6312|14569|8|240|46|30|54.36|73.92|69.48|0.00|2084.40|1630.80|2217.60|125.06|0.00|2084.40|2209.46|453.60| +2451299|47778|13042|34598|1011655|6312|14569|8|135|46|58|41.38|49.65|5.46|0.00|316.68|2400.04|2879.70|6.33|0.00|316.68|323.01|-2083.36| +2451299|47778|8065|34598||6312||8|120|46||43.04||||860.25||4099.50||||894.66|| +2451299|47778|3010|34598|1011655|6312|14569|8|86|46|67|66.87|83.58|58.50|0.00|3919.50|4480.29|5599.86|313.56|0.00|3919.50|4233.06|-560.79| +2451299|47778|11426|34598|1011655|6312|14569|8|221|46|72|53.81|99.01|42.57|0.00|3065.04|3874.32|7128.72|61.30|0.00|3065.04|3126.34|-809.28| +2451147|42794|3085|83903|534604|58|5159|4|268|47|51|63.55|104.85|53.47|490.85|2726.97|3241.05|5347.35|67.08|490.85|2236.12|2303.20|-1004.93| +2451147|42794|13477|83903|534604|58|5159|4|22|47|10|78.83|143.47|58.82|0.00|588.20|788.30|1434.70|11.76|0.00|588.20|599.96|-200.10| +2451147|42794|5468|83903|534604|58|5159|4|156|47|7|5.03|7.44|0.66|0.00|4.62|35.21|52.08|0.04|0.00|4.62|4.66|-30.59| +2451147||5305|83903|534604|||||47||84.70|102.48|95.30|0.00|||||0.00|8291.10||| +2451147|42794|1132|83903|534604|58|5159|4|144|47|66|53.46|79.65|63.72|0.00|4205.52|3528.36|5256.90|42.05|0.00|4205.52|4247.57|677.16| +2451147|42794|4774|83903|534604|58|5159|4|147|47|11|63.71|102.57|58.46|0.00|643.06|700.81|1128.27|38.58|0.00|643.06|681.64|-57.75| +2451147|42794|5008|83903|534604|58|5159|4|164|47|82|1.76|2.62|0.83|0.00|68.06|144.32|214.84|4.08|0.00|68.06|72.14|-76.26| +2451147|42794|254|83903|534604||5159||138|47|||||0.00|7444.36|||595.54|0.00||8039.90|623.70| +2451147|42794|3202|83903|534604|58|5159|4|205|47|7|26.06|49.77|28.36|0.00|198.52|182.42|348.39|3.97|0.00|198.52|202.49|16.10| +2451147|42794|1916|83903|534604|58|5159|4|150|47|71|47.62|49.04|43.64|0.00|3098.44|3381.02|3481.84|278.85|0.00|3098.44|3377.29|-282.58| +2451147|42794|481|83903|534604|58|5159|4|92|47|30|36.51|48.92|6.35|0.00|190.50|1095.30|1467.60|5.71|0.00|190.50|196.21|-904.80| +2451147|42794|9770|83903|534604|58|5159|4|154|47|33|69.12|133.40|45.35|0.00|1496.55|2280.96|4402.20|0.00|0.00|1496.55|1496.55|-784.41| +|42794|3878|||58||4||47||||111.69||||4582.40|||||| +2451147|42794|7214|83903|534604|58|5159|4|196|47|1|12.80|17.28|8.12|0.00|8.12|12.80|17.28|0.73|0.00|8.12|8.85|-4.68| +2451147|42794|13246|83903|534604|58|5159|4|139|47|47|11.71|16.51|0.16|6.16|7.52|550.37|775.97|0.00|6.16|1.36|1.36|-549.01| +2452606|71569|9199|21202|888343|6237|1331|8|246|48|88|37.89|49.63|27.79|0.00|2445.52|3334.32|4367.44|97.82|0.00|2445.52|2543.34|-888.80| +2452606|71569|12468|21202|888343|6237|1331|8|164|48|62|43.72|66.45|39.87|0.00|2471.94|2710.64|4119.90|222.47|0.00|2471.94|2694.41|-238.70| +2452606|71569|13153|21202|888343|6237|1331|8|187|48|74|34.26|42.48|5.94|0.00|439.56|2535.24|3143.52|8.79|0.00|439.56|448.35|-2095.68| +2452606|71569|2820|21202|888343|6237|1331|8|23|48|4|6.46|9.69|6.68|16.29|26.72|25.84|38.76|0.00|16.29|10.43|10.43|-15.41| +2452606||11259||||1331|8||48||||39.00|0.00|897.00||1520.53|35.88|0.00|||| +2452606|71569|17799|21202|888343|6237|1331|8|82|48|17|80.36|151.07|120.85|0.00|2054.45|1366.12|2568.19|41.08|0.00|2054.45|2095.53|688.33| +2452606|71569|2829|21202|888343|6237|1331|8|289|48|65|22.10|27.84|24.49|0.00|1591.85|1436.50|1809.60|31.83|0.00|1591.85|1623.68|155.35| +2452606|71569|5971|21202|888343|6237|1331|8|110|48|29|40.46|54.21|39.03|0.00|1131.87|1173.34|1572.09|90.54|0.00|1131.87|1222.41|-41.47| +2452606|71569|6684|21202|888343|6237|1331|8|117|48|44|20.22|40.03|39.22|0.00|1725.68|889.68|1761.32|120.79|0.00|1725.68|1846.47|836.00| +2452606|71569|4431|21202|888343|6237|1331|8|110|48|39|97.07|177.63|5.32|0.00|207.48|3785.73|6927.57|0.00|0.00|207.48|207.48|-3578.25| +2452606|71569|6085|21202|888343|6237|1331|8|41|48|1|58.13|102.30|82.86|0.00|82.86|58.13|102.30|2.48|0.00|82.86|85.34|24.73| +2452606|71569|1761|21202|888343|6237|1331|8|183|48|22|55.73|63.53|0.63|0.00|13.86|1226.06|1397.66|0.41|0.00|13.86|14.27|-1212.20| +2451774|39843|9049|79401|1604468|3582|4296|8|246|49|68|75.38|144.72|115.77|0.00|7872.36|5125.84|9840.96|551.06|0.00|7872.36|8423.42|2746.52| +2451774|39843|9553|79401|1604468|3582|4296|8|149|49|71|29.28|44.21|3.97|0.00|281.87|2078.88|3138.91|22.54|0.00|281.87|304.41|-1797.01| +2451774|39843|12737|79401|1604468|3582|4296|8|107|49|48|2.17|4.01|2.60|0.00|124.80|104.16|192.48|8.73|0.00|124.80|133.53|20.64| +2451774|39843|7141|79401|1604468|3582|4296|8|139|49|94|50.50|63.63|57.26|4521.24|5382.44|4747.00|5981.22|51.67|4521.24|861.20|912.87|-3885.80| +2451774|39843|749|79401|1604468|3582|4296|8|8|49|60|42.11|83.37|76.70|0.00|4602.00|2526.60|5002.20|0.00|0.00|4602.00|4602.00|2075.40| +2451774|39843|6805|79401|1604468|3582|4296|8|58|49|40|12.90|25.41|23.63|0.00|945.20|516.00|1016.40|56.71|0.00|945.20|1001.91|429.20| +2451774|39843|16361|79401|1604468|3582|4296|8|240|49|4|79.85|125.36|6.26|0.00|25.04|319.40|501.44|1.75|0.00|25.04|26.79|-294.36| +2451774|39843|2135|79401|1604468|3582|4296|8|103|49|4|15.80|16.59|7.46|0.00|29.84|63.20|66.36|1.79|0.00|29.84|31.63|-33.36| +2451774|39843|15155|79401|1604468|3582|4296|8|81|49|84|4.40|6.60|2.24|0.00|188.16|369.60|554.40|3.76|0.00|188.16|191.92|-181.44| +2451774|39843|5342|79401|1604468|3582|4296|8|89|49|69|46.41|46.87|41.71|1957.03|2877.99|3202.29|3234.03|27.62|1957.03|920.96|948.58|-2281.33| +2451774|39843|5852|79401|1604468|3582|4296|8|45|49|47|74.90|77.89|45.17|0.00|2122.99|3520.30|3660.83|21.22|0.00|2122.99|2144.21|-1397.31| +2451203|46470|1280|55849|1179944|5639|14534|4|231|50|69|8.66|14.80|1.62|0.00|111.78|597.54|1021.20|5.58|0.00|111.78|117.36|-485.76| +2451203|46470|16081|55849|1179944|5639|14534|4|206|50|82|99.55|124.43|2.48|0.00|203.36|8163.10|10203.26|14.23|0.00|203.36|217.59|-7959.74| +2451203|46470|1312|55849|1179944|5639|14534|4|8|50|30|25.84|49.35|9.87|0.00|296.10|775.20|1480.50|5.92|0.00|296.10|302.02|-479.10| +2451203|46470|7135|55849|1179944|5639|14534|4|199|50|69|12.68|21.42|19.70|1182.59|1359.30|874.92|1477.98|3.53|1182.59|176.71|180.24|-698.21| +2451203|46470|1984|55849|1179944|5639|14534|4|188|50|40|8.81|8.81|6.25|222.50|250.00|352.40|352.40|0.82|222.50|27.50|28.32|-324.90| +2451203|46470|5023|||||||50||16.24|||0.00|83.44||132.48|0.00|0.00|||-46.48| +2451203|46470|1909|55849|1179944|5639|14534|4|86|50|53|56.01|62.17|6.83|322.17|361.99|2968.53|3295.01|0.00|322.17|39.82|39.82|-2928.71| +2451203|46470|3097|55849|1179944|5639|14534|4|191|50|64|33.17|55.39|27.14|0.00|1736.96|2122.88|3544.96|156.32|0.00|1736.96|1893.28|-385.92| +2451693|47958|12625|83517|38885|1806|28327|10|169|51|53|97.27|176.05|116.19|0.00|6158.07|5155.31|9330.65|307.90|0.00|6158.07|6465.97|1002.76| +2451693|47958|3091|83517|38885|1806|28327|10|136|51|28|92.87|120.73|84.51|0.00|2366.28|2600.36|3380.44|23.66|0.00|2366.28|2389.94|-234.08| +2451693|47958|422|83517|38885|1806|28327|10|166|51|21|69.89|131.39|97.22|306.24|2041.62|1467.69|2759.19|52.06|306.24|1735.38|1787.44|267.69| +2451693|47958|5029|83517|38885|1806|28327|10|227|51|12|34.53|46.96|29.58|0.00|354.96|414.36|563.52|10.64|0.00|354.96|365.60|-59.40| +2451693|47958|17705|83517|38885|1806|28327|10|86|51|66|46.11|60.86|23.12|0.00|1525.92|3043.26|4016.76|91.55|0.00|1525.92|1617.47|-1517.34| +2451693||11234||38885|1806|28327|10||51|||113.10||0.00||1187.55|||0.00||1859.09|| +2451693|47958|17483|83517|38885|1806|28327|10|223|51|22|31.99|32.94|24.37|160.84|536.14|703.78|724.68|11.25|160.84|375.30|386.55|-328.48| +2451693|47958|8384|83517|38885|1806|28327|10|298|51|79|15.35|29.01|22.33|0.00|1764.07|1212.65|2291.79|158.76|0.00|1764.07|1922.83|551.42| +2451693|47958|6565|83517|38885|1806|28327|10|165|51|33|32.44|55.47|2.21|0.00|72.93|1070.52|1830.51|5.83|0.00|72.93|78.76|-997.59| +2451693|47958|5059|83517|38885|1806|28327|10|78|51|51|48.54|85.43|0.85|0.00|43.35|2475.54|4356.93|0.86|0.00|43.35|44.21|-2432.19| +2451693|47958|4687|83517|38885|1806|28327|10|129|51|6|93.02|97.67|9.76|0.00|58.56|558.12|586.02|5.27|0.00|58.56|63.83|-499.56| +2451693|47958|9311|83517|38885|1806|28327|10|77|51|90|39.48|53.69|7.51|0.00|675.90|3553.20|4832.10|47.31|0.00|675.90|723.21|-2877.30| +2451693|47958|13199|83517|38885|1806|28327|10|198|51|97|99.32|105.27|94.74|0.00|9189.78|9634.04|10211.19|459.48|0.00|9189.78|9649.26|-444.26| +2451693|47958|10133|83517|38885|1806|28327|10|246|51|54|46.71|51.38|34.42|55.76|1858.68|2522.34|2774.52|54.08|55.76|1802.92|1857.00|-719.42| +2451027|41589|7606|45831|354191|951|5264|7|30|52|45|42.51|72.26|3.61|0.00|162.45|1912.95|3251.70|0.00|0.00|162.45|162.45|-1750.50| +2451027|41589|15649|45831|354191|951|5264|7|25|52|29|22.86|29.71|16.04|0.00|465.16|662.94|861.59|13.95|0.00|465.16|479.11|-197.78| +2451027|41589|11488|45831|354191|951|5264|7|108|52|76|78.68|106.21|86.03|0.00|6538.28|5979.68|8071.96|130.76|0.00|6538.28|6669.04|558.60| +2451027|41589|2420|45831|354191|951|5264|7|94|52|90|22.31|35.24|18.67|0.00|1680.30|2007.90|3171.60|100.81|0.00|1680.30|1781.11|-327.60| +2451027|41589|16646|45831|354191|951|5264|7|165|52|48|95.82|132.23|70.08|773.68|3363.84|4599.36|6347.04|0.00|773.68|2590.16|2590.16|-2009.20| +2451027|41589|17456|45831|354191|951|5264|7|71|52|37|7.93|11.97|4.54|0.00|167.98|293.41|442.89|1.67|0.00|167.98|169.65|-125.43| +|41589|6098||354191||||45|52||4.83|7.63|7.40||||495.95|||481.00|481.00|| +2451027|41589|3334|45831|354191|951|5264|7|8|52|73|29.20|39.71|29.78|1260.88|2173.94|2131.60|2898.83|73.04|1260.88|913.06|986.10|-1218.54| +2451027|41589|17402|45831|354191|951|5264|7|242|52|91|81.94|139.29|123.96|0.00|11280.36|7456.54|12675.39|789.62|0.00|11280.36|12069.98|3823.82| +2451452|39338|16420|87672|1260894|5316|32985|4|117|53|14|36.72|63.15|56.20|0.00|786.80|514.08|884.10|15.73|0.00|786.80|802.53|272.72| +2451452|39338|2095|87672|1260894|5316|32985|4|209|53|62|23.98|32.61|28.69|0.00|1778.78|1486.76|2021.82|53.36|0.00|1778.78|1832.14|292.02| +2451452|39338|7546|87672|1260894|5316|32985|4|39|53|58|73.79|118.80|91.47|0.00|5305.26|4279.82|6890.40|424.42|0.00|5305.26|5729.68|1025.44| +2451452|39338|2887|87672|1260894|5316|32985|4|175|53|39|66.68|106.02|99.65|0.00|3886.35|2600.52|4134.78|349.77|0.00|3886.35|4236.12|1285.83| +2451452|39338|1114|87672|1260894|5316|32985|4|178|53|40|28.34|38.25|5.73|192.52|229.20|1133.60|1530.00|0.36|192.52|36.68|37.04|-1096.92| +2451452|39338|11348|87672|1260894|5316|32985|4|277|53|38|5.54|9.30|2.23|0.00|84.74|210.52|353.40|3.38|0.00|84.74|88.12|-125.78| +2451452|39338|17560|87672|1260894|5316|32985|4|218|53|15|46.39|68.19|62.73|592.79|940.95|695.85|1022.85|24.37|592.79|348.16|372.53|-347.69| +2451452|39338|17296|87672|1260894|5316|32985|4|194|53|43|21.40|29.10|0.58|0.00|24.94|920.20|1251.30|0.00|0.00|24.94|24.94|-895.26| +2451452|39338|13220|87672|1260894|5316|32985|4|135|53|76|27.93|49.15|25.55|0.00|1941.80|2122.68|3735.40|174.76|0.00|1941.80|2116.56|-180.88| +2451452|39338|13795|87672|1260894|5316|32985|4|260|53|38|93.96|177.58|17.75|0.00|674.50|3570.48|6748.04|26.98|0.00|674.50|701.48|-2895.98| +2451452|39338|16648|87672|1260894|5316|32985|4|208|53|79|55.29|95.09|9.50|570.38|750.50|4367.91|7512.11|1.80|570.38|180.12|181.92|-4187.79| +2451452|39338|15991|87672|1260894|5316|32985|4|39|53|37|77.75|87.08|27.86|0.00|1030.82|2876.75|3221.96|82.46|0.00|1030.82|1113.28|-1845.93| +2451452|39338|2786|87672|1260894|5316|32985|4|146|53|70|76.55|114.05|112.90|0.00|7903.00|5358.50|7983.50|316.12|0.00|7903.00|8219.12|2544.50| +2452604|50471|16059|48157|1761636|4715|20594|1|264|54|15|7.90|15.24|8.68|0.00|130.20|118.50|228.60|5.20|0.00|130.20|135.40|11.70| +2452604|50471|6517|48157|1761636|4715|20594|1|198|54|40|23.38|27.12|14.64|158.11|585.60|935.20|1084.80|34.19|158.11|427.49|461.68|-507.71| +2452604|50471|9159|48157|1761636|4715|20594|1|23|54|75|55.47|100.95|61.57|0.00|4617.75|4160.25|7571.25|369.42|0.00|4617.75|4987.17|457.50| +2452604|50471|1165|48157|1761636|4715|20594|1|124|54|62|69.84|138.28|113.38|0.00|7029.56|4330.08|8573.36|632.66|0.00|7029.56|7662.22|2699.48| +2452604|50471|17479|48157|1761636|4715|20594|1|210|54|34|94.14|131.79|6.58|0.00|223.72|3200.76|4480.86|0.00|0.00|223.72|223.72|-2977.04| +2452604|50471|825|48157|1761636|4715|20594|1|228|54|50|99.64|121.56|35.25|0.00|1762.50|4982.00|6078.00|123.37|0.00|1762.50|1885.87|-3219.50| +2452604|50471|702|48157|1761636|4715|20594|1|171|54|40|16.76|32.51|18.85|0.00|754.00|670.40|1300.40|52.78|0.00|754.00|806.78|83.60| +2452604|50471|3861|||4715|20594|1||54|||82.97|||3235.80|4485.00||||3235.80||| +2452604|50471|14737|48157|1761636|4715|20594|1|297|54|38|29.20|56.94|48.39|18.38|1838.82|1109.60|2163.72|18.20|18.38|1820.44|1838.64|710.84| +2452604|50471|16974||||20594|1|38|54|||85.43||0.00|588.30||||0.00||600.06|| +2451908|54460|1339|86157|1184215|7073|47178|2|203|55|16|71.32|136.93|94.48|0.00|1511.68|1141.12|2190.88|15.11|0.00|1511.68|1526.79|370.56| +2451908|54460|14923|86157|1184215|7073|47178|2|16|55|71|30.04|32.14|16.71|94.91|1186.41|2132.84|2281.94|43.66|94.91|1091.50|1135.16|-1041.34| +2451908|54460|12205|86157|1184215|7073|47178|2|85|55|90|56.92|108.14|90.83|6458.01|8174.70|5122.80|9732.60|17.16|6458.01|1716.69|1733.85|-3406.11| +2451908|54460|12170|86157|1184215|7073|47178|2|167|55|92|69.53|119.59|101.65|0.00|9351.80|6396.76|11002.28|187.03|0.00|9351.80|9538.83|2955.04| +2451908|54460|5137||1184215||47178|2|145|55|33|57.28||24.75||816.75|1890.24|2475.99||||833.08|-1073.49| +2451908|54460|3001|86157|1184215|7073|47178|2|142|55|7|57.58|91.55|54.01|0.00|378.07|403.06|640.85|11.34|0.00|378.07|389.41|-24.99| +2451908|54460|9703|86157|1184215|7073|47178|2|282|55|44|57.21|88.67|37.24|0.00|1638.56|2517.24|3901.48|81.92|0.00|1638.56|1720.48|-878.68| +2451908|54460|14135|86157|1184215|7073|47178|2|31|55|36|26.40|46.72|34.57|0.00|1244.52|950.40|1681.92|12.44|0.00|1244.52|1256.96|294.12| +2451908|54460|17677|86157|1184215|7073|47178|2|251|55|17|26.59|47.86|22.97|0.00|390.49|452.03|813.62|19.52|0.00|390.49|410.01|-61.54| +2452480|41902|8691|68383|848615|1294|3526|2|280|56|59|98.91|141.44|41.01|0.00|2419.59|5835.69|8344.96|48.39|0.00|2419.59|2467.98|-3416.10| +2452480|41902|10362|68383|848615|1294|3526|2|271|56|54|82.62|98.31|8.84|0.00|477.36|4461.48|5308.74|14.32|0.00|477.36|491.68|-3984.12| +2452480|41902|16620|68383|848615|1294|3526|2|128|56|23|9.94|17.49|13.81|0.00|317.63|228.62|402.27|28.58|0.00|317.63|346.21|89.01| +2452480|41902|8707|68383|848615|1294|3526|2|30|56|68|79.70|137.88|60.66|0.00|4124.88|5419.60|9375.84|82.49|0.00|4124.88|4207.37|-1294.72| +2452480|41902|8353|68383|848615|1294|3526|2|150|56|35|80.42|81.22|4.06|38.36|142.10|2814.70|2842.70|4.14|38.36|103.74|107.88|-2710.96| +2452480|41902|4506|68383|848615|1294|3526|2|142|56|57|69.45|125.70|90.50|0.00|5158.50|3958.65|7164.90|103.17|0.00|5158.50|5261.67|1199.85| +2452480|41902|4242|68383|848615|1294|3526|2|52|56|2|88.74|106.48|68.14|0.00|136.28|177.48|212.96|12.26|0.00|136.28|148.54|-41.20| +2452480|41902|17331|68383|848615|1294|3526|2|78|56|74|32.12|41.43|14.91|375.13|1103.34|2376.88|3065.82|65.53|375.13|728.21|793.74|-1648.67| +2451580|63306|12596|4076|1484072|2915|42911|4|9|57|91|36.67|70.77|17.69|0.00|1609.79|3336.97|6440.07|0.00|0.00|1609.79|1609.79|-1727.18| +2451580|63306|4933|4076|1484072|2915|42911|4|278|57|22|93.86|122.95|30.73|0.00|676.06|2064.92|2704.90|47.32|0.00|676.06|723.38|-1388.86| +2451580|63306|15182|4076|1484072|2915|42911|4|291|57|86|84.85|110.30|65.07|0.00|5596.02|7297.10|9485.80|0.00|0.00|5596.02|5596.02|-1701.08| +2451580|63306|3253|4076|1484072|2915|42911|4|13|57|71|91.02|108.31|80.14|0.00|5689.94|6462.42|7690.01|398.29|0.00|5689.94|6088.23|-772.48| +2451580|63306|12721|4076|1484072|2915|42911|4|75|57|62|76.40|99.32|10.92|446.84|677.04|4736.80|6157.84|0.00|446.84|230.20|230.20|-4506.60| +2451580|63306|12740|4076|1484072|2915|42911|4|225|57|52|55.58|85.03|28.05|0.00|1458.60|2890.16|4421.56|14.58|0.00|1458.60|1473.18|-1431.56| +2451580|63306|4028|4076|1484072|2915|42911|4|52|57|88|82.23|112.65|100.25|0.00|8822.00|7236.24|9913.20|529.32|0.00|8822.00|9351.32|1585.76| +2451580|63306|17729|4076|1484072|2915|42911|4|174|57|26|97.20|183.70|77.15|0.00|2005.90|2527.20|4776.20|140.41|0.00|2005.90|2146.31|-521.30| +2451580||17993||1484072|2915||4||57|99||98.76||||5256.90|||||6183.00|| +2451654||5117|21601||600||||58|2|56.01|96.33|89.58||179.16|||7.16||179.16||| +2451654|29623|15635|21601|1515277|600|723|8|243|58|6|82.45|131.09|15.73|0.00|94.38|494.70|786.54|3.77|0.00|94.38|98.15|-400.32| +2451654|29623|3848|21601|1515277|600|723|8|217|58|6|45.41|55.85|20.10|0.00|120.60|272.46|335.10|2.41|0.00|120.60|123.01|-151.86| +2451654|29623|7649|21601|1515277|600|723|8|63|58|19|44.04|50.64|42.53|0.00|808.07|836.76|962.16|64.64|0.00|808.07|872.71|-28.69| +2451654|29623|1829|21601|1515277|600|723|8|176|58|52|19.97|38.54|28.90|0.00|1502.80|1038.44|2004.08|120.22|0.00|1502.80|1623.02|464.36| +2451654|29623|16472|21601|1515277|600|723|8|39|58|6|7.58|13.87|4.02|0.00|24.12|45.48|83.22|0.24|0.00|24.12|24.36|-21.36| +2451654|29623|9743|21601|1515277|600|723|8|103|58|62|73.14|91.42|29.25|0.00|1813.50|4534.68|5668.04|18.13|0.00|1813.50|1831.63|-2721.18| +2451654|29623|16949|21601|1515277|600|723|8|206|58|35|25.76|27.04|22.17|0.00|775.95|901.60|946.40|62.07|0.00|775.95|838.02|-125.65| +2451654|29623|10802|21601|1515277|600|723|8|299|58|14|79.64|136.98|43.83|0.00|613.62|1114.96|1917.72|49.08|0.00|613.62|662.70|-501.34| +2452309|42321|12858|32944|286640|1583|29326|10|28|59|90|61.18|105.84|104.78|0.00|9430.20|5506.20|9525.60|565.81|0.00|9430.20|9996.01|3924.00| +2452309|42321|14529|32944|286640|1583|29326|10|77|59|44|42.76|69.27|8.31|277.88|365.64|1881.44|3047.88|5.26|277.88|87.76|93.02|-1793.68| +2452309|42321|3546|32944|286640|1583|29326|10|227|59|22|64.21|96.95|67.86|0.00|1492.92|1412.62|2132.90|74.64|0.00|1492.92|1567.56|80.30| +2452309|42321|5772|32944|286640|1583|29326|10|270|59|70|56.19|103.38|4.13|0.00|289.10|3933.30|7236.60|8.67|0.00|289.10|297.77|-3644.20| +2452309|42321|8010|32944|286640|1583|29326|10|63|59|46|20.15|40.09|39.28|0.00|1806.88|926.90|1844.14|72.27|0.00|1806.88|1879.15|879.98| +2452309|42321|8335|32944|286640|1583|29326|10|36|59|36|32.82|47.91|13.41|463.44|482.76|1181.52|1724.76|1.35|463.44|19.32|20.67|-1162.20| +2452309|42321|9397|32944|286640|1583|29326|10|34|59|92|70.69|139.96|19.59|0.00|1802.28|6503.48|12876.32|144.18|0.00|1802.28|1946.46|-4701.20| +2452309|42321|11916|32944|286640|1583|29326|10|244|59|19|34.31|63.47|13.32|35.43|253.08|651.89|1205.93|17.41|35.43|217.65|235.06|-434.24| +||10651|||||10||59|100|35.78|62.25|47.31|898.89|4731.00||||898.89||3985.39|| +2452309|42321|3133|32944|286640|1583|29326|10|201|59|92|14.57|27.82|19.19|0.00|1765.48|1340.44|2559.44|35.30|0.00|1765.48|1800.78|425.04| +2452309|42321|7087|32944|286640|1583|29326|10|244|59|80|58.71|64.58|0.64|0.00|51.20|4696.80|5166.40|3.58|0.00|51.20|54.78|-4645.60| +2452309|42321|9348|32944|286640|1583|29326|10|57|59|62|83.62|99.50|11.94|0.00|740.28|5184.44|6169.00|14.80|0.00|740.28|755.08|-4444.16| +2452181|62777|555|82654|81736|5618|27547|1|14|60|62|49.17|87.52|63.88|0.00|3960.56|3048.54|5426.24|39.60|0.00|3960.56|4000.16|912.02| +2452181|62777|1117|82654|81736|5618|27547|1|220|60|78|46.63|48.02|35.05|0.00|2733.90|3637.14|3745.56|54.67|0.00|2733.90|2788.57|-903.24| +2452181|62777|97|82654|81736|5618|27547|1|88|60|50|37.49|44.23|5.74|0.00|287.00|1874.50|2211.50|5.74|0.00|287.00|292.74|-1587.50| +2452181|62777|10277|82654|81736|5618|27547|1|249|60|62|28.05|33.37|25.69|0.00|1592.78|1739.10|2068.94|95.56|0.00|1592.78|1688.34|-146.32| +2452181|62777|633|82654|81736|5618|27547|1|10|60|71|96.74|162.52|87.76|0.00|6230.96|6868.54|11538.92|560.78|0.00|6230.96|6791.74|-637.58| +2452181|62777|4235|82654|81736|5618|27547|1|275|60|28|29.67|41.53|27.82|280.42|778.96|830.76|1162.84|4.98|280.42|498.54|503.52|-332.22| +||4937|||5618|||162|60|22|27.75|47.17||0.00||610.50|1037.74||0.00||744.51|| +2452181|62777|7231|82654|81736|5618|27547|1|259|60|95|45.42|88.11|79.29|0.00|7532.55|4314.90|8370.45|301.30|0.00|7532.55|7833.85|3217.65| +2452181|62777|13975|82654|81736|5618|27547|1|68|60|14|76.01|127.69|31.92|0.00|446.88|1064.14|1787.66|31.28|0.00|446.88|478.16|-617.26| +2452181|62777|10393|82654|81736|5618|27547|1|34|60|75|98.86|168.06|20.16|1375.92|1512.00|7414.50|12604.50|12.24|1375.92|136.08|148.32|-7278.42| +2452181|62777|1573|82654|81736|5618|27547|1|258|60|90|19.02|31.00|1.55|0.00|139.50|1711.80|2790.00|4.18|0.00|139.50|143.68|-1572.30| +2452181|62777|999|82654|81736|5618|27547|1|140|60|43|22.13|43.81|29.35|0.00|1262.05|951.59|1883.83|88.34|0.00|1262.05|1350.39|310.46| +2452181|62777|4041|82654|81736|5618|27547|1|285|60|25|36.26|52.21|8.35|0.00|208.75|906.50|1305.25|12.52|0.00|208.75|221.27|-697.75| +2452181|62777|17755|82654|81736|5618|27547|1|256|60|88|52.17|88.16|85.51|0.00|7524.88|4590.96|7758.08|0.00|0.00|7524.88|7524.88|2933.92| +2452181|62777|4513|82654|81736|5618|27547|1|163|60|72|79.56|146.39|139.07|0.00|10013.04|5728.32|10540.08|500.65|0.00|10013.04|10513.69|4284.72| +2452181|62777|16887|82654|81736|5618|27547|1|219|60|25|17.92|22.75|9.32|0.00|233.00|448.00|568.75|9.32|0.00|233.00|242.32|-215.00| +2451181|71311|1106|151|1065156|561|29166|1|9|61|4|78.21|95.41|31.48|0.00|125.92|312.84|381.64|1.25|0.00|125.92|127.17|-186.92| +2451181|71311|10330|151|1065156|561|29166|1|193|61|8|46.45|87.79|75.49|0.00|603.92|371.60|702.32|48.31|0.00|603.92|652.23|232.32| +2451181|71311|8924|151|1065156|561|29166|1|175|61|11|86.51|134.09|127.38|0.00|1401.18|951.61|1474.99|56.04|0.00|1401.18|1457.22|449.57| +2451181|71311|4567|151|1065156|561|29166|1|151|61|26|29.60|50.61|19.23|24.99|499.98|769.60|1315.86|4.74|24.99|474.99|479.73|-294.61| +2451181|71311|16462|151|1065156|561|29166|1|112|61|26|24.34|37.97|21.64|0.00|562.64|632.84|987.22|45.01|0.00|562.64|607.65|-70.20| +2451181|71311|6380|151|1065156|561|29166|1|233|61|77|69.52|70.91|16.30|0.00|1255.10|5353.04|5460.07|50.20|0.00|1255.10|1305.30|-4097.94| +2451181|71311|5528|151|1065156|561|29166|1|117|61|78|13.85|24.23|9.20|0.00|717.60|1080.30|1889.94|28.70|0.00|717.60|746.30|-362.70| +2451181||3362||1065156|561||1||61|48|67.92|||||3260.16|5737.44|||1548.96|1564.44|-1711.20| +2451181|71311|7591|151|1065156|561|29166|1|212|61|78|91.99|127.86|56.25|0.00|4387.50|7175.22|9973.08|0.00|0.00|4387.50|4387.50|-2787.72| +2451181|71311|2264|151|1065156|561|29166|1|219|61|36|60.94|80.44|16.89|0.00|608.04|2193.84|2895.84|42.56|0.00|608.04|650.60|-1585.80| +2452184|50871|12199|98197|570263|5176|26889|10|239|62|5|85.54|112.91|29.35|0.00|146.75|427.70|564.55|5.87|0.00|146.75|152.62|-280.95| +2452184||12407||570263|5176|26889|10|243|62|||||1799.82|2727.00|4462.80|8523.60|64.90|1799.82|927.18||| +2452184|50871|8457|98197|570263|5176|26889|10|264|62|37|99.97|103.96|94.60|35.00|3500.20|3698.89|3846.52|207.91|35.00|3465.20|3673.11|-233.69| +2452184|50871|16151|98197|570263|5176|26889|10|293|62|93|84.72|150.80|16.58|1310.64|1541.94|7878.96|14024.40|16.19|1310.64|231.30|247.49|-7647.66| +2452184|50871|15733|98197|570263|5176|26889|10|249|62|40|28.03|45.40|40.86|310.53|1634.40|1121.20|1816.00|105.90|310.53|1323.87|1429.77|202.67| +2452184||6403|||||||62||92.02|||0.00|5801.41||7251.94|232.05|0.00||6033.46|-732.01| +2452184|50871|4093|98197|570263|5176|26889|10|115|62|94|5.53|10.78|5.39|0.00|506.66|519.82|1013.32|15.19|0.00|506.66|521.85|-13.16| +2452184|50871|12163|98197|570263|5176|26889|10|287|62|29|16.70|16.86|11.29|0.00|327.41|484.30|488.94|29.46|0.00|327.41|356.87|-156.89| +2452184|50871|15399|98197|570263|5176|26889|10|122|62|74|71.70|136.94|86.27|1915.19|6383.98|5305.80|10133.56|357.50|1915.19|4468.79|4826.29|-837.01| +2452184|50871|13559|98197|570263|5176|26889|10|242|62|80|52.56|80.94|74.46|0.00|5956.80|4204.80|6475.20|357.40|0.00|5956.80|6314.20|1752.00| +2452184|50871|10149|98197|570263|5176|26889|10|143|62|75|48.36|61.41|38.07|2455.51|2855.25|3627.00|4605.75|27.98|2455.51|399.74|427.72|-3227.26| +2452543|36708|11946|96129|245000|5071|13142|8|115|63|85|94.28|94.28|32.99|0.00|2804.15|8013.80|8013.80|140.20|0.00|2804.15|2944.35|-5209.65| +2452543|36708|16569|96129|245000|5071||||63|69||125.43||||||||666.41||| +2452543|36708|5079|96129|245000|5071|13142|8|161|63|79|36.05|71.01|34.79|0.00|2748.41|2847.95|5609.79|27.48|0.00|2748.41|2775.89|-99.54| +2452543|36708|11370|96129|245000|5071|13142|8|115|63|38|56.43|86.90|26.07|0.00|990.66|2144.34|3302.20|29.71|0.00|990.66|1020.37|-1153.68| +2452543|36708|13339|96129|245000|5071|13142|8|134|63|19|19.44|25.85|4.91|0.00|93.29|369.36|491.15|0.00|0.00|93.29|93.29|-276.07| +2452543|36708|9319|96129|245000|5071|13142|8|203|63|80|9.27|13.62|10.48|0.00|838.40|741.60|1089.60|50.30|0.00|838.40|888.70|96.80| +2452543|36708|5217|96129|245000|5071|13142|8|202|63|66|15.71|18.85|7.72|0.00|509.52|1036.86|1244.10|45.85|0.00|509.52|555.37|-527.34| +2452543|36708|4488|96129|245000|5071|13142|8|207|63|73|22.85|44.78|22.83|483.31|1666.59|1668.05|3268.94|11.83|483.31|1183.28|1195.11|-484.77| +2452543|36708|15793|96129|245000|5071|13142|8|287|63|40|75.62|129.31|69.82|0.00|2792.80|3024.80|5172.40|83.78|0.00|2792.80|2876.58|-232.00| +2452543|36708|5658|96129|245000|5071|13142|8|101|63|99|88.78|111.86|64.87|0.00|6422.13|8789.22|11074.14|577.99|0.00|6422.13|7000.12|-2367.09| +2452414|44845|14407|48017|1111710|203|7449|10|91|64|8|44.36|55.45|27.17|63.03|217.36|354.88|443.60|7.71|63.03|154.33|162.04|-200.55| +2452414|44845|15867|48017|1111710|203|7449|10|163|64|59|43.77|52.96|30.71|0.00|1811.89|2582.43|3124.64|126.83|0.00|1811.89|1938.72|-770.54| +2452414|44845|11835|48017|1111710|203|7449|10|232|64|82|30.65|54.55|15.81|0.00|1296.42|2513.30|4473.10|90.74|0.00|1296.42|1387.16|-1216.88| +||1213|48017|1111710|203|||163|64||38.46|65.38||0.00|943.92|2922.96|||0.00|||| +2452414|44845|15936|48017|1111710|203|7449|10|151|64|30|56.24|64.11|55.77|0.00|1673.10|1687.20|1923.30|117.11|0.00|1673.10|1790.21|-14.10| +||13224|48017|1111710|203||||64||||||||6670.37||||6389.50|1634.64| +2452414|44845|16921|48017|1111710|203|7449|10|203|64|19|98.61|143.97|2.87|0.00|54.53|1873.59|2735.43|0.00|0.00|54.53|54.53|-1819.06| +2452414|44845|17617|48017|1111710|203|7449|10|44|64|17|91.67|119.17|29.79|0.00|506.43|1558.39|2025.89|20.25|0.00|506.43|526.68|-1051.96| +||3963|48017||||10||64||||18.03|737.78||4137.88|7985.60|15.20|737.78|380.08||| +2452414|44845|17586|48017|1111710|203|7449|10|133|64|78|77.26|124.38|68.40|0.00|5335.20|6026.28|9701.64|0.00|0.00|5335.20|5335.20|-691.08| +2452414|44845|3090|48017|1111710|203|7449|10|49|64|87|78.06|156.12|104.60|455.01|9100.20|6791.22|13582.44|0.00|455.01|8645.19|8645.19|1853.97| +2452262|53102|15177|43973|1627323|5073|39623|4|208|65|55|89.19|139.13|125.21|0.00|6886.55|4905.45|7652.15|482.05|0.00|6886.55|7368.60|1981.10| +2452262|53102|13897|43973|1627323|5073|39623|4|227|65|40|52.84|66.57|51.25|0.00|2050.00|2113.60|2662.80|82.00|0.00|2050.00|2132.00|-63.60| +2452262|53102|14905|43973|1627323|5073|39623|4|277|65|85|81.24|157.60|31.52|0.00|2679.20|6905.40|13396.00|187.54|0.00|2679.20|2866.74|-4226.20| +2452262|53102|4227|43973|1627323|5073|39623|4|264|65|42|45.38|53.54|10.17|0.00|427.14|1905.96|2248.68|12.81|0.00|427.14|439.95|-1478.82| +2452262|53102|9625|43973|1627323|5073|39623|4|69|65|51|40.95|40.95|30.30|0.00|1545.30|2088.45|2088.45|30.90|0.00|1545.30|1576.20|-543.15| +2452262|53102|17025|43973|1627323|5073|39623|4|253|65|67|25.52|37.76|32.09|107.50|2150.03|1709.84|2529.92|122.55|107.50|2042.53|2165.08|332.69| +2452262|53102|2287|43973|1627323|5073|39623|4|114|65|100|91.80|125.76|125.76|0.00|12576.00|9180.00|12576.00|251.52|0.00|12576.00|12827.52|3396.00| +2452262|53102|9847|43973|1627323|5073|39623|4|189|65|54|64.26|87.39|14.85|0.00|801.90|3470.04|4719.06|16.03|0.00|801.90|817.93|-2668.14| +2452187|39658|8197|66766|1444812|3658|43304|2|177|66|41|84.22|124.64|54.84|0.00|2248.44|3453.02|5110.24|0.00|0.00|2248.44|2248.44|-1204.58| +2452187|39658|6507|66766|1444812|3658|43304|2|293|66|76|43.81|56.07|42.05|0.00|3195.80|3329.56|4261.32|127.83|0.00|3195.80|3323.63|-133.76| +|39658|9237|66766|1444812|3658|43304|||66|29|76.94|||629.21|998.76|||14.78|629.21|||| +2452187|39658|7227|66766|1444812|3658|43304|2|224|66|66|92.15|97.67|95.71|0.00|6316.86|6081.90|6446.22|126.33|0.00|6316.86|6443.19|234.96| +2452187|39658|15629|66766|1444812|3658|43304|2|150|66|20|22.04|41.21|15.65|0.00|313.00|440.80|824.20|18.78|0.00|313.00|331.78|-127.80| +2452187|39658|11419|66766|1444812|3658|43304|2|67|66|66|10.12|12.75|12.36|350.77|815.76|667.92|841.50|4.64|350.77|464.99|469.63|-202.93| +2452187|39658|10019|66766|1444812|3658|43304|2|122|66|10|48.77|69.25|58.17|0.00|581.70|487.70|692.50|0.00|0.00|581.70|581.70|94.00| +2452187|39658|7033|66766|1444812|3658|43304|2|97|66|65|4.08|5.30|3.28|0.00|213.20|265.20|344.50|14.92|0.00|213.20|228.12|-52.00| +2452187|39658|16795|66766|1444812|3658|43304|2|153|66|28|42.00|81.06|9.72|0.00|272.16|1176.00|2269.68|0.00|0.00|272.16|272.16|-903.84| +2452187|39658|16745|66766|1444812|3658|43304|2|45|66|91|9.53|16.39|8.19|0.00|745.29|867.23|1491.49|67.07|0.00|745.29|812.36|-121.94| +2452424|57161|757|94624|1655162|6900|14591|8|228|67|77|94.12|118.59|106.73|0.00|8218.21|7247.24|9131.43|657.45|0.00|8218.21|8875.66|970.97| +2452424|57161|8871|94624|1655162|6900|14591|8|9|67|71|78.59|87.23|79.37|3606.57|5635.27|5579.89|6193.33|81.14|3606.57|2028.70|2109.84|-3551.19| +2452424|57161|3439|94624|1655162|6900|14591|8|137|67|73|23.52|35.04|23.12|0.00|1687.76|1716.96|2557.92|135.02|0.00|1687.76|1822.78|-29.20| +2452424|57161|7020|94624|1655162|6900|14591|8|182|67|79|97.01|154.24|131.10|0.00|10356.90|7663.79|12184.96|207.13|0.00|10356.90|10564.03|2693.11| +2452424|57161|5113||||14591|8||67||49.59|58.51|54.41|0.00|||702.12||0.00||659.44|| +2452424|57161|4155|94624|1655162|6900|14591|8|32|67|87|87.74|143.89|15.82|0.00|1376.34|7633.38|12518.43|0.00|0.00|1376.34|1376.34|-6257.04| +2452424|57161|15699|94624|1655162|6900|14591|8|252|67|44|34.59|41.50|35.69|0.00|1570.36|1521.96|1826.00|125.62|0.00|1570.36|1695.98|48.40| +2452424|57161|12087|94624|1655162|6900|14591|8|249|67|70|80.71|129.94|51.97|0.00|3637.90|5649.70|9095.80|145.51|0.00|3637.90|3783.41|-2011.80| +2452424|57161|7507|94624|1655162|6900|14591|8|4|67|77|26.78|53.56|1.07|0.00|82.39|2062.06|4124.12|1.64|0.00|82.39|84.03|-1979.67| +2452424|57161|8469|94624|1655162|6900|14591|||67|59||85.43|76.03|0.00|||5040.37||0.00|||637.79| +2452424|57161|2133|94624|1655162|6900|14591|8|288|67|74|71.99|76.30|14.49|0.00|1072.26|5327.26|5646.20|42.89|0.00|1072.26|1115.15|-4255.00| +2451095||7765|2564|291068|5172||||68||69.58|95.32||0.00||2087.40||70.34|0.00||1242.74|-915.00| +2451095|38136|1603|2564|291068|5172|12476|10|111|68|57|21.03|26.07|22.42|945.67|1277.94|1198.71|1485.99|3.32|945.67|332.27|335.59|-866.44| +2451095|38136|5162|2564|291068|5172|12476|10|203|68|23|83.48|155.27|10.86|0.00|249.78|1920.04|3571.21|2.49|0.00|249.78|252.27|-1670.26| +2451095|38136|2296|2564|291068|5172|12476|10|101|68|52|98.90|168.13|166.44|0.00|8654.88|5142.80|8742.76|605.84|0.00|8654.88|9260.72|3512.08| +2451095|38136|17426|2564|291068|5172|12476|10|267|68|2|48.36|59.48|1.18|0.00|2.36|96.72|118.96|0.16|0.00|2.36|2.52|-94.36| +2451095|38136|17600|2564|291068|5172|12476|10|277|68|13|52.66|57.92|10.42|101.59|135.46|684.58|752.96|1.35|101.59|33.87|35.22|-650.71| +2451095|38136|12724|2564|291068|5172|12476|10|264|68|88|9.63|11.65|8.50|0.00|748.00|847.44|1025.20|22.44|0.00|748.00|770.44|-99.44| +2451095|38136|2564|2564|291068|5172|12476|10|153|68|83|77.32|134.53|6.72|0.00|557.76|6417.56|11165.99|11.15|0.00|557.76|568.91|-5859.80| +2451095|38136|6763|2564|291068|5172|12476|10|282|68|77|96.29|112.65|68.71|0.00|5290.67|7414.33|8674.05|317.44|0.00|5290.67|5608.11|-2123.66| +2451095|38136|2035|2564|291068|5172|12476|10|173|68|22|54.35|100.54|18.09|0.00|397.98|1195.70|2211.88|23.87|0.00|397.98|421.85|-797.72| +2451095|38136|1387|2564|291068|5172|12476|10|122|68|74|90.20|103.73|94.39|0.00|6984.86|6674.80|7676.02|279.39|0.00|6984.86|7264.25|310.06| +2451095|38136|1820|2564|291068|5172|12476|10|5|68|54|55.82|98.24|33.40|0.00|1803.60|3014.28|5304.96|72.14|0.00|1803.60|1875.74|-1210.68| +2451095|38136|12526|2564|291068|5172|12476|10|86|68|3|13.06|20.11|19.30|53.26|57.90|39.18|60.33|0.23|53.26|4.64|4.87|-34.54| +2451524|50604|4267|22064|294863|2062|27261|1|244|69|10|72.37|72.37|46.31|0.00|463.10|723.70|723.70|41.67|0.00|463.10|504.77|-260.60| +2451524|50604|7822|22064|294863|2062|27261|1|8|69|30|78.10|134.33|79.25|1664.25|2377.50|2343.00|4029.90|64.19|1664.25|713.25|777.44|-1629.75| +2451524|50604|322|||||||69|45||84.53||||||7.22||||| +2451524|50604|11137|22064|294863|2062|27261|1|238|69|68|22.04|22.26|8.68|0.00|590.24|1498.72|1513.68|47.21|0.00|590.24|637.45|-908.48| +|50604|13792|||||1||69||85.64||2.89|||2483.56|||||84.64|| +2451524|50604|13489|22064|294863|2062|27261|1|264|69|66|2.68|3.16|1.92|0.00|126.72|176.88|208.56|6.33|0.00|126.72|133.05|-50.16| +2451524|50604|15448|22064|294863|2062|27261|1|149|69|16|94.38|111.36|60.13|913.97|962.08|1510.08|1781.76|2.40|913.97|48.11|50.51|-1461.97| +2451524|50604|6136|22064|294863|2062|27261|1|107|69|7|49.79|75.68|33.29|0.00|233.03|348.53|529.76|2.33|0.00|233.03|235.36|-115.50| +2451524|50604|337|22064|294863|2062|27261|1|279|69|34|20.99|27.07|12.45|0.00|423.30|713.66|920.38|12.69|0.00|423.30|435.99|-290.36| +2451524|50604|8599|22064|294863|2062|27261|1|106|69|53|56.42|58.11|26.73|0.00|1416.69|2990.26|3079.83|56.66|0.00|1416.69|1473.35|-1573.57| +2451524|50604|7264|22064|294863|2062|27261|1|34|69|67|78.29|90.03|71.12|0.00|4765.04|5245.43|6032.01|190.60|0.00|4765.04|4955.64|-480.39| +2451524|50604|4208|22064|294863|2062|27261|1|185|69|9|99.77|120.72|25.35|0.00|228.15|897.93|1086.48|15.97|0.00|228.15|244.12|-669.78| +2451880|64866|6185|4344|1865354|4317|18927|10|144|70|82|84.60|141.28|62.16|0.00|5097.12|6937.20|11584.96|101.94|0.00|5097.12|5199.06|-1840.08| +2451880|64866|2462|4344||||||70||92.70||46.16|0.00||||108.93|0.00|2723.44||-2745.86| +|64866|6425|4344|1865354|||10||70||||24.72||2076.48|3289.44|5065.20|||2076.48||| +2451880|64866|8893|4344|1865354|4317|18927|10|244|70|17|63.51|104.15|99.98|0.00|1699.66|1079.67|1770.55|152.96|0.00|1699.66|1852.62|619.99| +2451880|64866|1592|4344|1865354|4317|18927|10|262|70|53|99.59|127.47|119.82|0.00|6350.46|5278.27|6755.91|63.50|0.00|6350.46|6413.96|1072.19| +2451880|64866|3947|4344|1865354||18927||295|70||63.80||||4224.08|||126.72|||4350.80|1799.68| +2451880|64866|14549|4344|1865354|4317|18927|10|79|70|4|35.39|40.69|1.22|0.00|4.88|141.56|162.76|0.34|0.00|4.88|5.22|-136.68| +2451880|64866|17815|4344|1865354|4317|18927|10|292|70|95|36.89|42.79|4.27|0.00|405.65|3504.55|4065.05|16.22|0.00|405.65|421.87|-3098.90| +2451880|64866|9857|4344|1865354|4317|18927|10|193|70|20|54.96|97.82|41.08|0.00|821.60|1099.20|1956.40|41.08|0.00|821.60|862.68|-277.60| +2451880|64866|3296|4344|1865354|4317|18927|10|125|70|48|10.23|17.59|1.93|0.00|92.64|491.04|844.32|7.41|0.00|92.64|100.05|-398.40| +2451161|66803|2098|85239|571432|5978|30571|1|188|71|51|57.87|64.23|26.97|0.00|1375.47|2951.37|3275.73|27.50|0.00|1375.47|1402.97|-1575.90| +2451161|66803|9274|85239|571432|5978|30571|1|75|71|36|12.39|24.53|17.66|464.10|635.76|446.04|883.08|12.01|464.10|171.66|183.67|-274.38| +2451161|66803|10288|85239|571432|5978|30571|1|101|71|30|29.57|40.21|30.96|0.00|928.80|887.10|1206.30|18.57|0.00|928.80|947.37|41.70| +2451161||5858|85239||||1||71||||||||87.99|0.20||20.22||-34.11| +2451161|66803|10108|85239|571432|5978|30571|1|42|71|66|9.49|18.79|5.82|0.00|384.12|626.34|1240.14|3.84|0.00|384.12|387.96|-242.22| +2451161|66803|9997|85239|571432|5978|30571|1|210|71|26|53.28|54.87|41.15|0.00|1069.90|1385.28|1426.62|10.69|0.00|1069.90|1080.59|-315.38| +2451161|66803|17246|85239|571432|5978|30571|1|59|71|90|50.57|77.87|46.72|0.00|4204.80|4551.30|7008.30|378.43|0.00|4204.80|4583.23|-346.50| +2451161|66803|9790|85239|571432|5978|30571|1|108|71|96|37.78|68.38|7.52|0.00|721.92|3626.88|6564.48|64.97|0.00|721.92|786.89|-2904.96| +2451161|66803|4144|85239|571432|5978|30571|1|226|71|49|26.75|53.23|23.42|0.00|1147.58|1310.75|2608.27|91.80|0.00|1147.58|1239.38|-163.17| +2451161|66803|8887|85239|571432|5978|30571|1|219|71|10|61.40|91.48|32.93|0.00|329.30|614.00|914.80|3.29|0.00|329.30|332.59|-284.70| +2451161|66803|6008|85239|571432|5978|30571|1|45|71|54|38.98|66.65|63.98|0.00|3454.92|2104.92|3599.10|310.94|0.00|3454.92|3765.86|1350.00| +2451161|66803|457|85239|571432|5978|30571|1|179|71|75|27.02|42.42|41.99|0.00|3149.25|2026.50|3181.50|94.47|0.00|3149.25|3243.72|1122.75| +2451161|66803|11168|85239|571432|5978|30571|1|228|71|79|24.66|29.83|14.61|0.00|1154.19|1948.14|2356.57|80.79|0.00|1154.19|1234.98|-793.95| +||9769|||5978||1||71|79|52.15||50.35|0.00|3977.65||5849.95|238.65|0.00|3977.65||-142.20| +2451161|66803|7504|85239|571432|5978|30571|1|9|71|3|78.44|147.46|76.67|0.00|230.01|235.32|442.38|13.80|0.00|230.01|243.81|-5.31| +2451161|66803|1888|85239|571432|5978|30571|1|36|71|4|40.47|59.08|2.36|0.00|9.44|161.88|236.32|0.84|0.00|9.44|10.28|-152.44| +2451791|44195|13423|60602|1460628|5454|16633|8|215|72|62|34.39|51.58|37.13|207.18|2302.06|2132.18|3197.96|104.74|207.18|2094.88|2199.62|-37.30| +2451791|44195|10427|60602|1460628|5454|16633|8|43|72|66|55.31|101.77|15.26|0.00|1007.16|3650.46|6716.82|60.42|0.00|1007.16|1067.58|-2643.30| +2451791|44195|6113|60602|1460628|5454|16633|8|176|72|45|59.65|117.51|37.60|0.00|1692.00|2684.25|5287.95|135.36|0.00|1692.00|1827.36|-992.25| +2451791|44195|9860|60602|1460628|5454|16633|8|60|72|26|69.92|130.05|124.84|0.00|3245.84|1817.92|3381.30|129.83|0.00|3245.84|3375.67|1427.92| +2451791|44195|10753|60602|1460628|5454|16633|8|92|72|16|32.01|52.81|29.57|0.00|473.12|512.16|844.96|37.84|0.00|473.12|510.96|-39.04| +2451791|44195|12788|60602|1460628|5454|16633|8|224|72|29|34.57|40.79|19.57|0.00|567.53|1002.53|1182.91|22.70|0.00|567.53|590.23|-435.00| +2451791||5917|||5454||||72|85||8.51|4.59|0.00||436.05|||0.00|390.15||| +2451791|44195|13904|60602|1460628|5454|16633|8|77|72|37|40.39|43.62|41.87|1549.19|1549.19|1494.43|1613.94|0.00|1549.19|0.00|0.00|-1494.43| +2451791|44195|15587|60602|1460628|5454|16633|8|284|72|87|19.04|20.75|17.43|0.00|1516.41|1656.48|1805.25|75.82|0.00|1516.41|1592.23|-140.07| +2451791|44195|11741|60602|1460628|5454|16633|8|181|72|62|79.25|137.89|95.14|0.00|5898.68|4913.50|8549.18|235.94|0.00|5898.68|6134.62|985.18| +2451791|44195|1535|||5454|16633|8||72|9|69.06|||0.00|45.18|621.54||2.71|0.00||47.89|| +2451791|44195|16765|60602|1460628|5454|16633|8|256|72|56|89.44|104.64|52.32|0.00|2929.92|5008.64|5859.84|205.09|0.00|2929.92|3135.01|-2078.72| +2451791|44195|12901|60602|1460628|5454|16633|8|50|72|57|1.64|2.91|1.36|0.00|77.52|93.48|165.87|2.32|0.00|77.52|79.84|-15.96| +2451791|44195|6671|60602|1460628|5454|16633|8|195|72|13|42.82|67.65|55.47|0.00|721.11|556.66|879.45|7.21|0.00|721.11|728.32|164.45| +2451791|44195|13085|60602|1460628|5454|16633|8|74|72|94|85.13|143.01|118.69|0.00|11156.86|8002.22|13442.94|1004.11|0.00|11156.86|12160.97|3154.64| +2451434|64285|16519|5309|868000|5091|31429|2|91|73|97|82.11|86.21|65.51|0.00|6354.47|7964.67|8362.37|444.81|0.00|6354.47|6799.28|-1610.20| +2451434|64285|6862|5309|868000|5091|31429|2|199|73|58|86.48|142.69|97.02|0.00|5627.16|5015.84|8276.02|225.08|0.00|5627.16|5852.24|611.32| +|64285|17269||868000|5091||2||73|40||11.30|9.49|0.00||||0.00|0.00|379.60||| +2451434|64285|15109|5309|868000|5091|31429|2|270|73|38|53.90|76.53|74.99|2707.13|2849.62|2048.20|2908.14|1.42|2707.13|142.49|143.91|-1905.71| +2451434|64285|247|5309|868000|5091|31429|2|240|73|53|5.22|5.58|3.57|0.00|189.21|276.66|295.74|17.02|0.00|189.21|206.23|-87.45| +2451434|64285|16585|5309|868000|5091|31429|2|276|73|38|69.27|99.74|38.89|0.00|1477.82|2632.26|3790.12|118.22|0.00|1477.82|1596.04|-1154.44| +2451434|64285|10051|5309|868000|5091|31429|2|150|73|49|97.28|107.98|104.74|2771.42|5132.26|4766.72|5291.02|94.43|2771.42|2360.84|2455.27|-2405.88| +2451434|64285|3205|5309|868000|5091|31429|2|22|73|82|44.64|86.60|26.84|0.00|2200.88|3660.48|7101.20|66.02|0.00|2200.88|2266.90|-1459.60| +2451434|64285|12502|5309|868000|5091|31429|2|176|73|75|21.63|42.39|28.40|0.00|2130.00|1622.25|3179.25|149.10|0.00|2130.00|2279.10|507.75| +2451434|64285|1063|5309|868000|5091|31429|2|244|73|37|34.93|46.80|37.90|0.00|1402.30|1292.41|1731.60|42.06|0.00|1402.30|1444.36|109.89| +2451434|64285|4946|5309|868000|5091|31429|2|18|73|54|71.08|85.29|36.67|0.00|1980.18|3838.32|4605.66|99.00|0.00|1980.18|2079.18|-1858.14| +2451111|36520|3104|62205|1003527|4049|47771|1|143|74|78|52.23|104.46|82.52|0.00|6436.56|4073.94|8147.88|386.19|0.00|6436.56|6822.75|2362.62| +2451111|36520|16694|62205|1003527|4049|47771|1|122|74|72|36.04|58.38|11.09|0.00|798.48|2594.88|4203.36|39.92|0.00|798.48|838.40|-1796.40| +2451111|36520|326|62205|1003527|4049|47771|1|99|74|29|76.73|108.95|46.84|624.84|1358.36|2225.17|3159.55|7.33|624.84|733.52|740.85|-1491.65| +2451111|36520|5492|62205|1003527|4049|47771|1|80|74|54|31.24|32.80|14.10|0.00|761.40|1686.96|1771.20|45.68|0.00|761.40|807.08|-925.56| +2451111|36520|9620|62205|1003527|4049|47771|1|210|74|95|56.35|91.28|83.97|0.00|7977.15|5353.25|8671.60|558.40|0.00|7977.15|8535.55|2623.90| +2451111|36520|8653|62205|1003527|4049|47771|1|292|74|12|4.33|7.66|5.28|0.00|63.36|51.96|91.92|0.00|0.00|63.36|63.36|11.40| +2451111|36520|13208|62205|1003527|4049|47771|1|291|74|87|82.61|100.78|86.67|0.00|7540.29|7187.07|8767.86|75.40|0.00|7540.29|7615.69|353.22| +2451111|36520|3175|62205|1003527|4049|47771|1|274|74|23|50.69|82.62|9.08|0.00|208.84|1165.87|1900.26|0.00|0.00|208.84|208.84|-957.03| +2451111|36520|10069|62205|1003527|4049|47771|1|164|74|99|99.98|104.97|98.67|0.00|9768.33|9898.02|10392.03|683.78|0.00|9768.33|10452.11|-129.69| +2451111|36520|3754|62205|1003527|4049|47771|1|124|74|26|89.35|162.61|115.45|0.00|3001.70|2323.10|4227.86|180.10|0.00|3001.70|3181.80|678.60| +2451111|36520|3278|||||1||74||||56.91||||||||3872.15|-726.39| +2451111|36520|3542|62205|1003527|4049|47771|1|180|74|96|93.18|131.38|42.04|0.00|4035.84|8945.28|12612.48|242.15|0.00|4035.84|4277.99|-4909.44| +2451111|36520|7694|62205|1003527|4049|47771|1|37|74|17|36.61|69.19|67.11|0.00|1140.87|622.37|1176.23|11.40|0.00|1140.87|1152.27|518.50| +2451883|30123|14186|18184|770971|111|14098|2|135|75|63|82.77|122.49|11.02|0.00|694.26|5214.51|7716.87|20.82|0.00|694.26|715.08|-4520.25| +2451883|30123|8726|18184|770971|111|14098|2|269|75|6|25.87|26.64|13.05|0.00|78.30|155.22|159.84|0.00|0.00|78.30|78.30|-76.92| +2451883|30123|607|18184|770971|111|14098|2|67|75|20|88.61|100.12|61.07|0.00|1221.40|1772.20|2002.40|85.49|0.00|1221.40|1306.89|-550.80| +2451883|30123|4625|18184|770971|111|14098|2|159|75|73|76.04|116.34|79.11|0.00|5775.03|5550.92|8492.82|231.00|0.00|5775.03|6006.03|224.11| +2451883|30123|10217|18184|770971|111|14098|2|142|75|85|83.25|100.73|37.27|0.00|3167.95|7076.25|8562.05|285.11|0.00|3167.95|3453.06|-3908.30| +2451883|30123|2948|18184|770971|111|14098|2|151|75|25|7.48|11.07|9.40|0.00|235.00|187.00|276.75|2.35|0.00|235.00|237.35|48.00| +2451883|30123|6938|18184|770971|111|14098|2|224|75|89|20.73|36.48|26.99|0.00|2402.11|1844.97|3246.72|24.02|0.00|2402.11|2426.13|557.14| +2451883|30123|9905|18184|770971|111|14098|2|291|75|54|63.01|78.76|77.18|0.00|4167.72|3402.54|4253.04|208.38|0.00|4167.72|4376.10|765.18| +2451883|30123|6953|18184|770971|111|14098|2|58|75|71|33.34|57.67|4.03|0.00|286.13|2367.14|4094.57|2.86|0.00|286.13|288.99|-2081.01| +2451883|30123|11039|18184|770971|111|14098|2|79|75|70|87.84|110.67|13.28|0.00|929.60|6148.80|7746.90|74.36|0.00|929.60|1003.96|-5219.20| +2451883|30123|16796|18184|770971|111|14098|2|178|75|93|5.19|6.02|0.66|0.00|61.38|482.67|559.86|1.22|0.00|61.38|62.60|-421.29| +2452143|39901|257|53531|580721|1958|24348|8|284|76|5|8.47|11.34|2.15|0.00|10.75|42.35|56.70|0.10|0.00|10.75|10.85|-31.60| +2452143|39901|1503|53531|580721|1958|24348|8|192|76|97|30.22|32.93|20.41|1207.65|1979.77|2931.34|3194.21|38.60|1207.65|772.12|810.72|-2159.22| +2452143||2869||580721|||||76|32|||||||1564.80|||1267.20|1355.90|| +2452143||465||||||278|76|2|95.45||76.51|0.00|153.02||318.80||0.00|153.02||-37.88| +2452143|39901|4237|53531|580721|1958|24348|8|18|76|48|60.58|81.17|55.19|0.00|2649.12|2907.84|3896.16|158.94|0.00|2649.12|2808.06|-258.72| +|39901|1107|||1958||8|203|76|16||84.68|||||1354.88|4.24||212.12||-715.88| +2452143|39901|10331|53531|580721|1958|24348|8|157|76|3|29.09|57.01|37.05|0.00|111.15|87.27|171.03|6.66|0.00|111.15|117.81|23.88| +2452143|39901|8925|53531|580721|1958|24348|8|31|76|6|80.04|108.05|22.69|0.00|136.14|480.24|648.30|12.25|0.00|136.14|148.39|-344.10| +2452143|39901|4567|53531|580721|1958|24348|8|111|76|40|2.19|3.00|1.29|0.00|51.60|87.60|120.00|1.54|0.00|51.60|53.14|-36.00| +2452143|39901|16463|53531|580721|1958|24348|8|151|76|53|86.06|104.13|27.07|0.00|1434.71|4561.18|5518.89|100.42|0.00|1434.71|1535.13|-3126.47| +2452143|39901|6381|53531|580721|1958|24348|8|156|76|50|34.93|63.92|32.59|0.00|1629.50|1746.50|3196.00|32.59|0.00|1629.50|1662.09|-117.00| +2452143|39901|5529|53531|580721|1958|24348|8|59|76|78|49.64|65.52|53.07|0.00|4139.46|3871.92|5110.56|165.57|0.00|4139.46|4305.03|267.54| +2452143|39901|3363|53531|580721|1958|24348|8|153|76|25|89.90|106.08|77.43|1529.24|1935.75|2247.50|2652.00|20.32|1529.24|406.51|426.83|-1840.99| +2452143|39901|7591|53531|580721|1958|24348|8|207|76|27|61.86|89.69|80.72|0.00|2179.44|1670.22|2421.63|21.79|0.00|2179.44|2201.23|509.22| +2452143|39901|2265|53531|580721|1958|24348|8|18|76|98|89.70|95.08|29.47|0.00|2888.06|8790.60|9317.84|0.00|0.00|2888.06|2888.06|-5902.54| +2451083|53379|13456|19214|1611001|1988|16175|2|171|77|48|85.61|113.86|108.16|0.00|5191.68|4109.28|5465.28|51.91|0.00|5191.68|5243.59|1082.40| +2451083|53379|7448|19214|1611001|1988|16175|2|74|77|95|29.07|54.07|17.84|0.00|1694.80|2761.65|5136.65|84.74|0.00|1694.80|1779.54|-1066.85| +2451083|53379|2713|19214|1611001|1988|16175|2|284|77|85|85.72|129.43|56.94|0.00|4839.90|7286.20|11001.55|387.19|0.00|4839.90|5227.09|-2446.30| +2451083|53379|1399|19214|1611001|1988|16175|2|161|77|34|96.21|182.79|29.24|596.49|994.16|3271.14|6214.86|3.97|596.49|397.67|401.64|-2873.47| +2451083|53379|7339|19214|1611001|1988|16175|2|54|77|88|31.13|59.14|13.60|155.58|1196.80|2739.44|5204.32|10.41|155.58|1041.22|1051.63|-1698.22| +2451083|53379|8686|19214|1611001|1988|16175|2|225|77|38|45.30|84.25|37.91|0.00|1440.58|1721.40|3201.50|72.02|0.00|1440.58|1512.60|-280.82| +2451083|53379|12439|19214|1611001|1988|16175|2|9|77|95|74.32|115.93|31.30|0.00|2973.50|7060.40|11013.35|118.94|0.00|2973.50|3092.44|-4086.90| +2451083|53379|9220|19214|1611001|1988|16175|2|270|77|90|87.41|138.10|75.95|0.00|6835.50|7866.90|12429.00|136.71|0.00|6835.50|6972.21|-1031.40| +2451083|53379|992|19214|1611001|1988|16175|2|295|77|62|21.65|33.34|3.33|0.00|206.46|1342.30|2067.08|10.32|0.00|206.46|216.78|-1135.84| +2451083||7486|19214|1611001|1988||2||77|49||84.08|||535.57|||26.77||535.57|562.34|| +2451083|53379|6289|19214|1611001|1988|16175|2|217|77|30|26.16|32.70|30.41|0.00|912.30|784.80|981.00|54.73|0.00|912.30|967.03|127.50| +2451083|53379|3868|19214|1611001|1988|16175|2|140|77|89|3.72|6.28|1.31|0.00|116.59|331.08|558.92|9.32|0.00|116.59|125.91|-214.49| +2451083|53379|16687|19214|1611001|1988|16175|2|175|77|16|67.63|119.02|86.88|0.00|1390.08|1082.08|1904.32|97.30|0.00|1390.08|1487.38|308.00| +2451083|53379|11918|19214|1611001|1988|16175|2|167|77|36|25.95|42.03|19.75|0.00|711.00|934.20|1513.08|28.44|0.00|711.00|739.44|-223.20| +2451083|53379|14815|19214|1611001|1988|16175|2|277|77|18|69.28|87.29|52.37|0.00|942.66|1247.04|1571.22|84.83|0.00|942.66|1027.49|-304.38| +2451063|30397|6110|10282|743469|2609|19796|4|249|78|43|28.45|40.68|10.98|0.00|472.14|1223.35|1749.24|23.60|0.00|472.14|495.74|-751.21| +2451063|30397|6044|10282|743469|2609|19796|4|213|78|59|15.94|29.01|25.52|240.90|1505.68|940.46|1711.59|50.59|240.90|1264.78|1315.37|324.32| +2451063|30397|11209|10282|743469|2609|19796|4|187|78|7|88.17|166.64|121.64|664.15|851.48|617.19|1166.48|1.87|664.15|187.33|189.20|-429.86| +2451063|30397|6500|10282|743469|2609|19796|4|289|78|76|38.42|68.77|63.26|4375.06|4807.76|2919.92|5226.52|12.98|4375.06|432.70|445.68|-2487.22| +2451063|30397|3304|10282|743469|2609|19796|4|252|78|50|61.49|75.01|72.00|0.00|3600.00|3074.50|3750.50|216.00|0.00|3600.00|3816.00|525.50| +2451063|30397|7576|10282|743469|2609|19796|4|180|78|87|60.62|83.65|15.89|0.00|1382.43|5273.94|7277.55|41.47|0.00|1382.43|1423.90|-3891.51| +2451063|30397|14996|10282|743469|2609|19796|4|213|78|39|36.47|53.24|8.51|0.00|331.89|1422.33|2076.36|3.31|0.00|331.89|335.20|-1090.44| +2451063|30397|901|10282|743469|2609|19796|4|49|78|3|54.50|105.73|88.81|223.80|266.43|163.50|317.19|1.70|223.80|42.63|44.33|-120.87| +2451063|30397|12706|10282|743469|2609|19796|4|124|78|19|81.10|116.78|82.91|0.00|1575.29|1540.90|2218.82|94.51|0.00|1575.29|1669.80|34.39| +2451063|30397|5965|10282|743469|2609|19796|4|8|78|78|7.47|13.22|0.26|0.00|20.28|582.66|1031.16|1.62|0.00|20.28|21.90|-562.38| +2451063|30397|8611|10282|743469|2609|19796|4|40|78|79|95.42|163.16|76.68|0.00|6057.72|7538.18|12889.64|0.00|0.00|6057.72|6057.72|-1480.46| +2451063|30397|3856|10282|743469|2609|19796|4|9|78|27|60.10|87.14|73.19|0.00|1976.13|1622.70|2352.78|158.09|0.00|1976.13|2134.22|353.43| +2451063|30397|10507|10282|743469|2609|19796|4|56|78|6|50.83|57.94|36.50|0.00|219.00|304.98|347.64|6.57|0.00|219.00|225.57|-85.98| +2452280|32066|16519|62017|1062416|1215|43261|7|117|79|9|69.84|98.47|12.80|0.00|115.20|628.56|886.23|4.60|0.00|115.20|119.80|-513.36| +2452280|32066|6864|62017|1062416|1215|43261|7|1|79|1|86.12|105.92|68.84|0.00|68.84|86.12|105.92|1.37|0.00|68.84|70.21|-17.28| +2452280|32066|17269|62017|1062416|1215|43261|7|216|79|81|5.88|9.52|1.61|0.00|130.41|476.28|771.12|11.73|0.00|130.41|142.14|-345.87| +2452280|32066|15109|||1215|43261|7|150|79||45.07|52.73|20.03||620.93|1397.17||||||| +2452280|32066|247|62017||||7||79|||113.96|28.49||227.92|660.64|911.68|||227.92||-432.72| +2452280|32066|16585|62017|1062416|1215|43261|7|252|79|93|70.70|114.53|9.16|0.00|851.88|6575.10|10651.29|0.00|0.00|851.88|851.88|-5723.22| +2452280|32066|10051|62017|1062416|1215|43261|7|156|79|10|94.53|124.77|19.96|0.00|199.60|945.30|1247.70|17.96|0.00|199.60|217.56|-745.70| +2452280|32066|3205|62017|1062416|1215|43261|7|133|79|48|21.34|26.67|15.20|0.00|729.60|1024.32|1280.16|7.29|0.00|729.60|736.89|-294.72| +2452280|32066|12504|62017|1062416|1215|43261|7|141|79|81|14.87|27.80|5.28|0.00|427.68|1204.47|2251.80|12.83|0.00|427.68|440.51|-776.79| +2452280|32066|1063|62017|1062416|1215|43261|7|224|79|85|77.51|148.81|139.88|0.00|11889.80|6588.35|12648.85|118.89|0.00|11889.80|12008.69|5301.45| +2452280|32066|4947|62017|1062416|1215|43261|7|14|79|35|99.77|180.58|74.03|1114.15|2591.05|3491.95|6320.30|44.30|1114.15|1476.90|1521.20|-2015.05| +2452280|32066|10524|62017|1062416|1215|43261|7|202|79|36|21.73|22.81|5.24|0.00|188.64|782.28|821.16|11.31|0.00|188.64|199.95|-593.64| +2452280|32066|14322|62017|1062416|1215|43261|7|18|79|41|58.88|109.51|64.61|0.00|2649.01|2414.08|4489.91|185.43|0.00|2649.01|2834.44|234.93| +2452280|32066|15888|62017|1062416|1215|43261|7|154|79|58|99.75|146.63|23.46|1156.57|1360.68|5785.50|8504.54|18.36|1156.57|204.11|222.47|-5581.39| +2452280|32066|15498|62017|1062416|1215|43261|7|181|79|3|94.64|147.63|87.10|0.00|261.30|283.92|442.89|23.51|0.00|261.30|284.81|-22.62| +2451143|67039|7231|17368|633451|1887|30323|10|27|80|49|76.07|116.38|95.43|0.00|4676.07|3727.43|5702.62|327.32|0.00|4676.07|5003.39|948.64| +2451143|67039|13975|17368|633451|1887|30323|10|162|80|13|83.81|136.61|121.58|0.00|1580.54|1089.53|1775.93|47.41|0.00|1580.54|1627.95|491.01| +2451143|67039|10393|17368|633451|1887|30323|10|283|80|5|71.37|97.06|67.94|0.00|339.70|356.85|485.30|20.38|0.00|339.70|360.08|-17.15| +2451143|67039|1573|17368|633451|1887|30323|10|137|80|40|43.33|83.62|81.94|0.00|3277.60|1733.20|3344.80|196.65|0.00|3277.60|3474.25|1544.40| +2451143|67039|998|17368|633451|1887|30323|10|297|80|93|69.32|75.55|24.93|0.00|2318.49|6446.76|7026.15|208.66|0.00|2318.49|2527.15|-4128.27| +2451143|67039|4040|17368|633451|1887|30323|10|94|80|66|15.01|27.91|3.07|0.00|202.62|990.66|1842.06|14.18|0.00|202.62|216.80|-788.04| +2451143|67039|17755|17368|633451|1887|30323|10|147|80|94|56.04|64.44|12.24|0.00|1150.56|5267.76|6057.36|46.02|0.00|1150.56|1196.58|-4117.20| +2451143|67039|4513||||30323|10||80||76.02||112.82|0.00|9251.24|6233.64|9537.42||0.00||9898.82|3017.60| +2451143|67039|16886|17368|633451|1887|30323|10|189|80|77|89.22|146.32|7.31|135.08|562.87|6869.94|11266.64|4.27|135.08|427.79|432.06|-6442.15| +2451143|67039|1519|17368|633451|1887|30323|10|147|80|25|66.36|130.06|58.52|0.00|1463.00|1659.00|3251.50|0.00|0.00|1463.00|1463.00|-196.00| +2451143|67039|7610|17368|633451|1887|30323|10|201|80|37|24.62|26.09|21.13|0.00|781.81|910.94|965.33|15.63|0.00|781.81|797.44|-129.13| +||12968|17368|633451||||197|80|||28.84|16.15|||2103.93|2797.48|140.98||||| +2451143|67039|4622|17368|633451|1887|30323|10|249|80|1|60.10|110.58|33.17|0.00|33.17|60.10|110.58|2.32|0.00|33.17|35.49|-26.93| +2451143|67039|17308|17368|633451|1887|30323|10|57|80|29|94.38|159.50|71.77|0.00|2081.33|2737.02|4625.50|104.06|0.00|2081.33|2185.39|-655.69| +2451143|67039|13717|17368|633451|1887|30323|10|137|80|91|60.42|116.00|106.72|0.00|9711.52|5498.22|10556.00|97.11|0.00|9711.52|9808.63|4213.30| +2451143|67039|16363|17368|633451|1887|30323|10|262|80|84|84.80|109.39|25.15|0.00|2112.60|7123.20|9188.76|84.50|0.00|2112.60|2197.10|-5010.60| +2451167|61212|5078|24625|258943|3661|29179|7|258|81|75|33.72|35.06|11.21|0.00|840.75|2529.00|2629.50|58.85|0.00|840.75|899.60|-1688.25| +2451167||11368|24625||3661|29179|7||81|26||43.18||582.98|594.88|951.60||0.11|582.98||12.01|-939.70| +2451167|61212|13339|24625|258943|3661|29179|7|261|81|6|4.63|8.19|7.20|36.72|43.20|27.78|49.14|0.25|36.72|6.48|6.73|-21.30| +2451167|61212|9319|24625|258943|3661|29179|7|79|81|36|91.74|143.11|64.39|1831.25|2318.04|3302.64|5151.96|14.60|1831.25|486.79|501.39|-2815.85| +2451167|61212|5216|24625|258943|3661|29179|7|81|81|64|4.59|6.33|6.26|0.00|400.64|293.76|405.12|4.00|0.00|400.64|404.64|106.88| +2451167|61212|4486|24625|258943|3661|29179|7|64|81|31|63.84|79.16|3.95|0.00|122.45|1979.04|2453.96|7.34|0.00|122.45|129.79|-1856.59| +2451167|61212|15793|24625|258943|3661|29179|7|229|81|8|5.61|10.09|10.09|0.00|80.72|44.88|80.72|1.61|0.00|80.72|82.33|35.84| +2451167|61212|5656|24625|258943|3661|29179|7|136|81|24|40.61|76.34|0.00|0.00|0.00|974.64|1832.16|0.00|0.00|0.00|0.00|-974.64| +2451167|61212|8918|24625|258943|3661|29179|7|62|81|37|86.54|154.04|61.61|0.00|2279.57|3201.98|5699.48|205.16|0.00|2279.57|2484.73|-922.41| +2451167|61212|7166|24625|258943|3661|29179|7|259|81|7|22.87|40.93|0.00|0.00|0.00|160.09|286.51|0.00|0.00|0.00|0.00|-160.09| +2451167|61212|11107|24625|258943|3661|29179|7|29|81|36|47.86|86.14|74.08|0.00|2666.88|1722.96|3101.04|160.01|0.00|2666.88|2826.89|943.92| +2451167|61212|7663|24625|258943|3661|29179|7|284|81|79|52.07|59.35|16.61|0.00|1312.19|4113.53|4688.65|104.97|0.00|1312.19|1417.16|-2801.34| +2451363|75450|7862|20896|1350703|157|46411|4|36|82|75|21.65|25.98|21.30|0.00|1597.50|1623.75|1948.50|79.87|0.00|1597.50|1677.37|-26.25| +2451363|75450|14998|20896|1350703|157|46411|4|249|82|49|52.59|57.32|33.81|0.00|1656.69|2576.91|2808.68|149.10|0.00|1656.69|1805.79|-920.22| +2451363|75450|13138|20896|1350703|157|46411|4|125|82|59|31.81|41.67|11.66|564.11|687.94|1876.79|2458.53|6.19|564.11|123.83|130.02|-1752.96| +2451363|75450|17041|20896|1350703|157|46411|4|17|82|18|4.71|7.25|4.93|0.00|88.74|84.78|130.50|6.21|0.00|88.74|94.95|3.96| +2451363|75450|2572|20896|1350703|157|46411|4|154|82|53|55.41|109.71|86.67|0.00|4593.51|2936.73|5814.63|91.87|0.00|4593.51|4685.38|1656.78| +2451363|75450|692|20896|1350703|157|46411|4|168|82|50|12.61|17.40|16.35|0.00|817.50|630.50|870.00|65.40|0.00|817.50|882.90|187.00| +2451363|75450|3538|20896|1350703|157|46411|4|29|82|78|94.47|160.59|105.98|4794.53|8266.44|7368.66|12526.02|173.59|4794.53|3471.91|3645.50|-3896.75| +||14176|20896||157|46411|4|141|82|42||153.22||0.00||||122.26|0.00|6113.10||| +2451363|75450|1700|20896|1350703|157|46411|4|223|82|64|62.61|65.11|32.55|0.00|2083.20|4007.04|4167.04|83.32|0.00|2083.20|2166.52|-1923.84| +2451363|75450|38|20896|1350703|157|46411|4|215|82|29|65.23|99.80|66.86|0.00|1938.94|1891.67|2894.20|77.55|0.00|1938.94|2016.49|47.27| +2451363|75450|16123|20896|1350703|157|46411|4|55|82|30|32.46|59.40|24.35|0.00|730.50|973.80|1782.00|51.13|0.00|730.50|781.63|-243.30| +2452640|38634|3811|42678|1322728|5978|27228|2|57|83|82|92.52|137.85|77.19|1962.16|6329.58|7586.64|11303.70|349.39|1962.16|4367.42|4716.81|-3219.22| +2452640|38634|4539|42678|1322728|5978|27228|2|153|83|33|48.28|51.65|37.18|1067.43|1226.94|1593.24|1704.45|4.78|1067.43|159.51|164.29|-1433.73| +2452640|38634|5376|42678|1322728|5978|27228|2|69|83|78|34.04|56.84|14.77|0.00|1152.06|2655.12|4433.52|92.16|0.00|1152.06|1244.22|-1503.06| +2452640|38634|10884|42678|1322728|5978|27228|2|36|83|82|95.71|178.02|12.46|929.76|1021.72|7848.22|14597.64|6.43|929.76|91.96|98.39|-7756.26| +2452640|38634|396|42678|1322728|5978|27228|2|83|83|8|58.66|115.56|102.84|0.00|822.72|469.28|924.48|24.68|0.00|822.72|847.40|353.44| +2452640|38634|16068|42678|1322728|5978|27228|2|216|83|91|8.40|12.60|8.82|0.00|802.62|764.40|1146.60|40.13|0.00|802.62|842.75|38.22| +2452640|38634|2838|42678|1322728|5978|27228|2|287|83|90|44.17|85.68|62.54|0.00|5628.60|3975.30|7711.20|0.00|0.00|5628.60|5628.60|1653.30| +2452640|38634|15573|42678|1322728|5978|27228|2|189|83|26|68.01|94.53|26.46|20.63|687.96|1768.26|2457.78|60.05|20.63|667.33|727.38|-1100.93| +2452640|38634|11085|42678|1322728|5978|27228|2|161|83|74|27.03|36.76|12.13|0.00|897.62|2000.22|2720.24|35.90|0.00|897.62|933.52|-1102.60| +2452640|38634|14526|42678|1322728|5978|27228|2|182|83|53|15.98|24.92|1.24|0.00|65.72|846.94|1320.76|0.00|0.00|65.72|65.72|-781.22| +2452640|38634|4110|42678|1322728|5978|27228|2|95|83|12|55.64|81.79|65.43|785.16|785.16|667.68|981.48|0.00|785.16|0.00|0.00|-667.68| +2452640|38634|12378|42678|1322728|5978|27228|2|296|83|31|68.51|87.00|54.81|1189.37|1699.11|2123.81|2697.00|10.19|1189.37|509.74|519.93|-1614.07| +2452640|38634|13236|42678|1322728|5978|27228|2|156|83|40|16.44|20.38|0.81|29.16|32.40|657.60|815.20|0.03|29.16|3.24|3.27|-654.36| +2452495|55056|13753|57091|432368|6045|24122|8|166|84|2|95.24|173.33|67.59|0.00|135.18|190.48|346.66|9.46|0.00|135.18|144.64|-55.30| +2452495|55056|8370|57091|432368|6045|24122|8|30|84|100|42.41|46.22|42.06|0.00|4206.00|4241.00|4622.00|252.36|0.00|4206.00|4458.36|-35.00| +2452495|55056|11389|57091|432368|6045|24122|8|74|84|87|38.03|61.98|3.71|0.00|322.77|3308.61|5392.26|6.45|0.00|322.77|329.22|-2985.84| +2452495|55056|6417|57091|432368|6045|24122|8|43|84|12|62.35|97.88|64.60|0.00|775.20|748.20|1174.56|46.51|0.00|775.20|821.71|27.00| +2452495|55056|17155|57091|432368|6045|24122|8|278|84|67|99.29|152.90|27.52|0.00|1843.84|6652.43|10244.30|165.94|0.00|1843.84|2009.78|-4808.59| +2452495|55056|9069|57091|432368|6045|24122|8|193|84|88|67.23|90.08|57.65|0.00|5073.20|5916.24|7927.04|304.39|0.00|5073.20|5377.59|-843.04| +2452495|55056|10191|57091|432368|6045|24122|8|167|84|3|10.51|15.13|10.89|0.00|32.67|31.53|45.39|1.96|0.00|32.67|34.63|1.14| +2452495|55056|8007|57091|432368|6045|24122|8|37|84|54|12.49|16.98|14.09|83.69|760.86|674.46|916.92|13.54|83.69|677.17|690.71|2.71| +2452495|55056|16377|57091|432368|6045|24122|8|3|84|7|67.82|99.01|31.68|0.00|221.76|474.74|693.07|13.30|0.00|221.76|235.06|-252.98| +2452495|55056|10401|57091|432368|6045|24122|8|134|84|54|98.33|106.19|36.10|0.00|1949.40|5309.82|5734.26|175.44|0.00|1949.40|2124.84|-3360.42| +2452495|55056|15336|57091|432368|6045|24122|8|156|84|92|64.65|99.56|77.65|0.00|7143.80|5947.80|9159.52|214.31|0.00|7143.80|7358.11|1196.00| +2452495|55056|12666|57091|432368|6045|24122|8|159|84|78|98.76|113.57|57.92|0.00|4517.76|7703.28|8858.46|316.24|0.00|4517.76|4834.00|-3185.52| +2452495|55056|15769|57091|432368|6045|24122|8|42|84|41|78.58|80.15|48.89|0.00|2004.49|3221.78|3286.15|40.08|0.00|2004.49|2044.57|-1217.29| +2452495|55056|10771|57091|432368|6045|24122|8|297|84|99|3.42|5.57|2.17|0.00|214.83|338.58|551.43|0.00|0.00|214.83|214.83|-123.75| +2451259|29324|9326|63472|1406010|5348|5933|1|76|85|84|98.64|139.08|50.06|0.00|4205.04|8285.76|11682.72|126.15|0.00|4205.04|4331.19|-4080.72| +2451259|29324|704|63472|1406010|5348|5933|1|106|85|33|79.22|82.38|16.47|179.35|543.51|2614.26|2718.54|0.00|179.35|364.16|364.16|-2250.10| +2451259|29324|14168|63472|1406010|5348|5933|1|120|85|73|74.11|128.95|88.97|0.00|6494.81|5410.03|9413.35|64.94|0.00|6494.81|6559.75|1084.78| +2451259|29324|694|63472|1406010|5348|5933|1|253|85|85|46.00|66.70|51.35|0.00|4364.75|3910.00|5669.50|0.00|0.00|4364.75|4364.75|454.75| +2451259|29324|15073|63472|1406010|5348|5933|1|146|85|34|45.19|49.70|32.30|0.00|1098.20|1536.46|1689.80|87.85|0.00|1098.20|1186.05|-438.26| +2451259|29324|7454|63472|1406010|5348|5933|1|159|85|28|78.35|79.91|1.59|0.00|44.52|2193.80|2237.48|2.22|0.00|44.52|46.74|-2149.28| +2451259|29324|446|63472|1406010|5348|5933|1|292|85|37|22.07|24.93|0.74|0.00|27.38|816.59|922.41|1.36|0.00|27.38|28.74|-789.21| +2451259|29324|12434|63472|1406010|5348|5933|1|73|85|80|13.67|13.67|13.12|0.00|1049.60|1093.60|1093.60|10.49|0.00|1049.60|1060.09|-44.00| +2451259|29324|6110|63472|1406010|5348|5933|1|144|85|92|76.94|103.86|64.39|5568.44|5923.88|7078.48|9555.12|3.55|5568.44|355.44|358.99|-6723.04| +2451259|29324|6044|63472|1406010|5348|5933|1|57|85|25|81.13|145.22|97.29|0.00|2432.25|2028.25|3630.50|170.25|0.00|2432.25|2602.50|404.00| +2451259|29324|11209|63472|1406010|5348|5933|1|58|85|65|14.87|18.73|9.17|262.26|596.05|966.55|1217.45|0.00|262.26|333.79|333.79|-632.76| +2451259|29324|6500|63472|1406010|5348|5933|1|84|85|14|38.60|41.30|26.84|0.00|375.76|540.40|578.20|26.30|0.00|375.76|402.06|-164.64| +2451259|29324|3304|63472|1406010|5348|5933|1|155|85|23|74.44|132.50|98.05|0.00|2255.15|1712.12|3047.50|67.65|0.00|2255.15|2322.80|543.03| +2451259||7576|63472|||5933|||85|||2.89||0.00||||2.12|0.00|||49.40| +2451259|29324|14996|63472|1406010|5348|5933|1|55|85|55|30.41|37.70|17.71|0.00|974.05|1672.55|2073.50|0.00|0.00|974.05|974.05|-698.50| +2452641|38153|13411|56359|1905038|2164|27643|10|300|86|88|46.71|86.88|45.17|0.00|3974.96|4110.48|7645.44|317.99|0.00|3974.96|4292.95|-135.52| +2452641||3270|56359|||27643|10|143|86||97.96|191.02|||4541.98|||||4541.98|4587.39|-3490.74| +2452641|38153|3123|56359|1905038|2164|27643|10|202|86|18|84.56|99.78|59.86|851.20|1077.48|1522.08|1796.04|0.00|851.20|226.28|226.28|-1295.80| +2452641|38153|16254|56359|1905038|2164|27643|10|75|86|76|33.46|41.82|6.69|0.00|508.44|2542.96|3178.32|20.33|0.00|508.44|528.77|-2034.52| +2452641|38153|2508|56359|1905038|2164|27643|10|56|86|10|23.91|28.45|16.78|0.00|167.80|239.10|284.50|1.67|0.00|167.80|169.47|-71.30| +2452641|38153|14712|56359|1905038|2164|27643|10|269|86|68|25.97|29.60|14.20|0.00|965.60|1765.96|2012.80|19.31|0.00|965.60|984.91|-800.36| +2452641|38153|14001|56359|1905038|2164|27643|10|153|86|51|8.47|10.07|2.11|0.00|107.61|431.97|513.57|0.00|0.00|107.61|107.61|-324.36| +2452641|38153|643|56359|1905038|2164|27643|10|42|86|84|22.77|44.62|4.46|0.00|374.64|1912.68|3748.08|33.71|0.00|374.64|408.35|-1538.04| +2452641|38153|3379||1905038|2164|27643|||86|||5.42|||63.70||75.88|||63.70||19.32| +2452641|38153|9279|56359|1905038|2164|27643|10|274|86|21|96.24|187.66|31.90|0.00|669.90|2021.04|3940.86|33.49|0.00|669.90|703.39|-1351.14| +2452641|38153|15648|56359|1905038|2164|27643|10|71|86|57|16.66|25.15|4.52|0.00|257.64|949.62|1433.55|7.72|0.00|257.64|265.36|-691.98| +2452641|38153|10341|56359|1905038|2164|27643|10|8|86|36|3.36|5.27|2.00|0.00|72.00|120.96|189.72|5.76|0.00|72.00|77.76|-48.96| +2452641|38153|13413|56359|1905038|2164|27643|10|128|86|46|47.48|69.79|8.37|96.25|385.02|2184.08|3210.34|8.66|96.25|288.77|297.43|-1895.31| +2451500|51805|6217|21438|1209241|1044|47247|10|51|87|15|88.49|138.92|76.40|332.34|1146.00|1327.35|2083.80|0.00|332.34|813.66|813.66|-513.69| +2451500|51805|7462|21438|1209241|1044|47247|10|134|87|75|85.17|160.11|113.67|0.00|8525.25|6387.75|12008.25|511.51|0.00|8525.25|9036.76|2137.50| +2451500|51805|13874|21438|1209241|1044|47247|10|286|87|65|79.59|105.05|99.79|0.00|6486.35|5173.35|6828.25|194.59|0.00|6486.35|6680.94|1313.00| +2451500|51805|14810|21438|1209241|1044|47247|10|275|87|16|22.33|25.00|5.75|0.00|92.00|357.28|400.00|5.52|0.00|92.00|97.52|-265.28| +2451500|51805|6685|21438|1209241|1044|47247|10|83|87|76|63.91|118.87|98.66|0.00|7498.16|4857.16|9034.12|374.90|0.00|7498.16|7873.06|2641.00| +2451500|51805|17146|21438|1209241|1044|47247|10|100|87|50|98.44|176.20|107.48|0.00|5374.00|4922.00|8810.00|322.44|0.00|5374.00|5696.44|452.00| +2451500|51805|15862|21438|1209241|1044|47247|10|6|87|78|13.49|21.98|16.26|0.00|1268.28|1052.22|1714.44|88.77|0.00|1268.28|1357.05|216.06| +2451500|51805|13756|21438|1209241|1044|47247|10|259|87|21|13.40|22.51|13.50|0.00|283.50|281.40|472.71|22.68|0.00|283.50|306.18|2.10| +2451500|51805|12904|21438|1209241|1044|47247|10|144|87|22|33.62|52.78|3.69|0.00|81.18|739.64|1161.16|7.30|0.00|81.18|88.48|-658.46| +2451500|51805|2642|21438|1209241|1044|47247|10|11|87|3|74.37|146.50|52.74|61.70|158.22|223.11|439.50|5.79|61.70|96.52|102.31|-126.59| +2451500|51805|2216|21438|1209241|1044|47247|10|291|87|84|39.15|41.89|5.44|0.00|456.96|3288.60|3518.76|18.27|0.00|456.96|475.23|-2831.64| +2451500|51805|5654|21438|1209241|1044|47247|10|15|87|47|41.72|83.44|36.71|0.00|1725.37|1960.84|3921.68|69.01|0.00|1725.37|1794.38|-235.47| +2451500|51805|15382|21438|1209241|1044|47247|10|188|87|29|81.96|101.63|35.57|0.00|1031.53|2376.84|2947.27|30.94|0.00|1031.53|1062.47|-1345.31| +2451500|51805|13472|21438|1209241|1044|47247|10|93|87|50|62.16|85.15|30.65|0.00|1532.50|3108.00|4257.50|30.65|0.00|1532.50|1563.15|-1575.50| +2451500|51805|10879|21438|1209241|1044|47247|10|3|87|80|70.65|139.18|115.51|0.00|9240.80|5652.00|11134.40|184.81|0.00|9240.80|9425.61|3588.80| +2451500|51805|9500|21438|1209241|1044|47247|10|92|87|47|29.01|46.41|12.06|0.00|566.82|1363.47|2181.27|5.66|0.00|566.82|572.48|-796.65| +2452243|53059|7207|34272|1548428|3384|45940|2|269|88|55|71.42|117.84|18.85|0.00|1036.75|3928.10|6481.20|31.10|0.00|1036.75|1067.85|-2891.35| +2452243|53059|17037|34272|1548428|||2|286|88|37||86.17|4.30|||||0.00||159.10|159.10|| +2452243|53059|14171|34272|1548428|3384|45940|2|267|88|76|42.33|57.56|16.69|0.00|1268.44|3217.08|4374.56|76.10|0.00|1268.44|1344.54|-1948.64| +2452243|53059|5133|34272|1548428|3384|45940|2|143|88|12|34.97|58.74|54.04|0.00|648.48|419.64|704.88|6.48|0.00|648.48|654.96|228.84| +2452243|53059|15457|34272|1548428|3384|45940|2|250|88|36|9.09|16.36|8.34|189.15|300.24|327.24|588.96|3.33|189.15|111.09|114.42|-216.15| +2452243|53059|3653|34272||||2|225|88|||72.14|12.26|0.00|662.04|2434.86|3895.56|46.34|0.00||708.38|| +2452243|53059|17509|34272|1548428|3384|45940|2|249|88|21|46.82|58.99|58.99|0.00|1238.79|983.22|1238.79|74.32|0.00|1238.79|1313.11|255.57| +2452243|53059|17381|34272|1548428|3384|45940|2|255|88|60|44.78|88.21|84.68|0.00|5080.80|2686.80|5292.60|203.23|0.00|5080.80|5284.03|2394.00| +|29627|13400||1278329|4064|32199|4||89||7.99||1.90||11.40|47.94||0.79||11.40||| +2451769|29627|7765|74994|1278329|4064|32199|4|132|89|47|97.15|161.26|8.06|0.00|378.82|4566.05|7579.22|11.36|0.00|378.82|390.18|-4187.23| +|29627|1603||1278329||32199|4||89||80.80||||||6505.65|||2276.52||-2652.28| +2451769|29627|5162|74994|1278329|4064|32199|4|103|89|43|46.23|79.51|27.03|0.00|1162.29|1987.89|3418.93|46.49|0.00|1162.29|1208.78|-825.60| +2451769|29627|2297|74994|1278329|4064|32199|4|258|89|96|2.91|3.25|2.11|0.00|202.56|279.36|312.00|0.00|0.00|202.56|202.56|-76.80| +2451769|29627|17426|74994|1278329|4064|32199|4|78|89|36|41.90|61.17|17.73|0.00|638.28|1508.40|2202.12|19.14|0.00|638.28|657.42|-870.12| +2451769|29627|17600|74994|1278329|4064|32199|4|33|89|2|83.53|131.14|131.14|0.00|262.28|167.06|262.28|0.00|0.00|262.28|262.28|95.22| +2451769|29627|12725|74994|1278329|4064|32199|4|278|89|46|85.58|120.66|4.82|0.00|221.72|3936.68|5550.36|4.43|0.00|221.72|226.15|-3714.96| +2451769|29627|2564|74994|1278329|4064|32199|4|299|89|40|60.84|62.66|1.25|25.00|50.00|2433.60|2506.40|1.75|25.00|25.00|26.75|-2408.60| +2451769|29627|6763|74994|1278329|4064|32199|4|83|89|58|75.88|91.81|15.60|380.01|904.80|4401.04|5324.98|10.49|380.01|524.79|535.28|-3876.25| +2451769|29627|2035|74994|1278329|4064|32199|4|1|89|79|70.39|130.92|44.51|2215.26|3516.29|5560.81|10342.68|39.03|2215.26|1301.03|1340.06|-4259.78| +2451769|29627|1387|74994|1278329|4064|32199|4|223|89|37|30.98|58.55|52.10|0.00|1927.70|1146.26|2166.35|154.21|0.00|1927.70|2081.91|781.44| +2451769|29627|1820|74994|1278329|4064|32199|4|229|89|78|14.85|19.89|12.13|0.00|946.14|1158.30|1551.42|66.22|0.00|946.14|1012.36|-212.16| +2452197|39176|16505|83671|378127|6983|9086|10|239|90|9|27.64|51.68|41.86|0.00|376.74|248.76|465.12|11.30|0.00|376.74|388.04|127.98| +2452197|39176|9951|83671|378127|6983|9086|10|176|90|57|45.91|55.09|1.10|0.00|62.70|2616.87|3140.13|5.01|0.00|62.70|67.71|-2554.17| +2452197|39176|3355|83671|378127|6983|9086|10|276|90|43|97.03|152.33|22.84|481.23|982.12|4172.29|6550.19|35.06|481.23|500.89|535.95|-3671.40| +2452197|39176|3417|83671|378127|6983|9086|10|236|90|38|15.37|16.75|2.84|105.76|107.92|584.06|636.50|0.02|105.76|2.16|2.18|-581.90| +2452197|39176|14861|83671|378127|6983|9086|10|274|90|84|6.95|10.42|4.06|0.00|341.04|583.80|875.28|0.00|0.00|341.04|341.04|-242.76| +2452197|39176|3629|83671|378127|6983|9086|10|37|90|62|16.64|26.12|20.89|323.79|1295.18|1031.68|1619.44|67.99|323.79|971.39|1039.38|-60.29| +2452197|39176|4523|83671|378127|6983|9086|10|105|90|92|83.46|122.68|103.05|0.00|9480.60|7678.32|11286.56|94.80|0.00|9480.60|9575.40|1802.28| +2452197|39176|17325|83671|378127|6983|9086|10|48|90|84|66.73|90.75|0.00|0.00|0.00|5605.32|7623.00|0.00|0.00|0.00|0.00|-5605.32| +||3503|83671|||||240|90||||85.89||||4588.74||||3584.18|-506.61| +2452197|39176|1967|83671|378127|6983|9086|10|284|90|9|81.11|128.15|21.78|0.00|196.02|729.99|1153.35|9.80|0.00|196.02|205.82|-533.97| +2452197|39176|1871|83671|378127|6983|9086|10|49|90|16|29.10|32.59|13.36|0.00|213.76|465.60|521.44|8.55|0.00|213.76|222.31|-251.84| +2452197|39176|12091|83671|378127|6983|9086|10|193|90|39|25.27|37.14|28.96|0.00|1129.44|985.53|1448.46|22.58|0.00|1129.44|1152.02|143.91| +2452197|39176|16043|83671|378127|6983|9086|10|116|90|51|5.35|7.54|1.50|0.00|76.50|272.85|384.54|6.12|0.00|76.50|82.62|-196.35| +2452453|51741|17697|34925|340590|3855|17520|1|227|91|17|90.40|153.68|0.00|0.00|0.00|1536.80|2612.56|0.00|0.00|0.00|0.00|-1536.80| +2452453|51741|1825|34925|340590|3855|17520|1|64|91|50|75.86|108.47|60.74|0.00|3037.00|3793.00|5423.50|242.96|0.00|3037.00|3279.96|-756.00| +2452453|51741|14479|34925|340590|3855|17520|1|53|91|32|79.03|110.64|68.59|0.00|2194.88|2528.96|3540.48|197.53|0.00|2194.88|2392.41|-334.08| +2452453||13333|34925|340590||17520|1|129|91|40|22.51|45.02|17.10|0.00|684.00||1800.80||0.00|684.00||| +2452453|51741|1656|34925|340590|3855|17520|1|181|91|70|83.76|156.63|103.37|4630.97|7235.90|5863.20|10964.10|208.39|4630.97|2604.93|2813.32|-3258.27| +2452453|51741|1695|34925|340590|3855|17520|1|112|91|85|75.83|118.29|39.03|0.00|3317.55|6445.55|10054.65|199.05|0.00|3317.55|3516.60|-3128.00| +2452453|51741|4759|34925|340590|3855|17520|1|51|91|25|49.02|70.09|38.54|0.00|963.50|1225.50|1752.25|86.71|0.00|963.50|1050.21|-262.00| +2452453|51741|498|34925|340590|3855|17520|1|195|91|97|57.83|78.07|31.22|2028.98|3028.34|5609.51|7572.79|0.00|2028.98|999.36|999.36|-4610.15| +2452453|51741|6367|34925|340590|3855|17520|1|102|91|13|91.23|154.17|50.87|0.00|661.31|1185.99|2004.21|6.61|0.00|661.31|667.92|-524.68| +2452453|51741|2311|34925|340590|3855|17520|1|221|91|97|39.85|74.91|35.95|1394.86|3487.15|3865.45|7266.27|188.30|1394.86|2092.29|2280.59|-1773.16| +2452453|51741|2125|34925|340590|3855|17520|1|164|91|36|16.29|21.99|16.05|0.00|577.80|586.44|791.64|23.11|0.00|577.80|600.91|-8.64| +2451091|67194|5210|30306|743793|33|15120|7|6|92|65|44.76|81.01|29.97|0.00|1948.05|2909.40|5265.65|97.40|0.00|1948.05|2045.45|-961.35| +|67194|14059||743793|33||7|296|92|36||136.44||0.00|2750.40|2493.36|4911.84|55.00|0.00|2750.40|2805.40|257.04| +2451091|67194|10510|30306|743793|33|15120|7|220|92|50|73.60|128.06|120.37|0.00|6018.50|3680.00|6403.00|541.66|0.00|6018.50|6560.16|2338.50| +2451091|67194|7498|30306|743793|33|15120|7|71|92|86|29.26|50.32|32.70|1293.61|2812.20|2516.36|4327.52|75.92|1293.61|1518.59|1594.51|-997.77| +2451091|67194|8473|30306|743793|33|15120|7|185|92|34|79.75|130.79|112.47|1797.27|3823.98|2711.50|4446.86|121.60|1797.27|2026.71|2148.31|-684.79| +2451091|67194|15016|30306|743793|33|15120|7|6|92|59|99.52|115.44|2.30|0.00|135.70|5871.68|6810.96|10.85|0.00|135.70|146.55|-5735.98| +||16513||||15120||297|92|30|15.53|21.43||0.00||||20.56|0.00|411.30||| +2451091|67194|12818|30306|743793|33|15120|7|19|92|23|95.39|160.25|1.60|0.00|36.80|2193.97|3685.75|3.31|0.00|36.80|40.11|-2157.17| +2451091|67194|6337|30306|743793|33|15120|7|127|92|9|36.81|70.67|35.33|0.00|317.97|331.29|636.03|15.89|0.00|317.97|333.86|-13.32| +2451091|67194|12476|30306|743793|33|15120|7|142|92|48|5.29|5.76|4.89|0.00|234.72|253.92|276.48|16.43|0.00|234.72|251.15|-19.20| +2451091|67194|10106|30306|743793|33|15120|7|116|92|70|48.90|54.76|44.90|0.00|3143.00|3423.00|3833.20|31.43|0.00|3143.00|3174.43|-280.00| +2451091|67194|10342|30306|743793|33|15120|7|257|92|99|96.69|157.60|113.47|0.00|11233.53|9572.31|15602.40|561.67|0.00|11233.53|11795.20|1661.22| +2451485|42571|11077|52989|885443|1450|14657|8|215|93|85|36.86|72.61|37.75|0.00|3208.75|3133.10|6171.85|0.00|0.00|3208.75|3208.75|75.65| +2451485|42571|1678|52989|885443|1450|14657|8|187|93|50|56.46|81.86|17.19|0.00|859.50|2823.00|4093.00|77.35|0.00|859.50|936.85|-1963.50| +2451485|42571|8038|52989|885443|1450|14657|8|31|93|23|84.90|155.36|17.08|0.00|392.84|1952.70|3573.28|23.57|0.00|392.84|416.41|-1559.86| +2451485|42571|13855|52989|885443|1450|14657|8|226|93|87|60.58|107.83|30.19|0.00|2626.53|5270.46|9381.21|183.85|0.00|2626.53|2810.38|-2643.93| +2451485|42571|13324|52989|885443|1450|14657|8|79|93|34|42.78|82.13|22.17|0.00|753.78|1454.52|2792.42|45.22|0.00|753.78|799.00|-700.74| +2451485|42571|2758|52989|885443|1450|14657|8|298|93|9|78.55|96.61|96.61|0.00|869.49|706.95|869.49|52.16|0.00|869.49|921.65|162.54| +2451485|42571|6658|52989|885443|1450|14657|8|42|93|61|63.45|67.25|16.14|0.00|984.54|3870.45|4102.25|0.00|0.00|984.54|984.54|-2885.91| +2451485|42571|9052|52989|885443|1450|14657|8|28|93|66|33.78|50.67|33.44|0.00|2207.04|2229.48|3344.22|88.28|0.00|2207.04|2295.32|-22.44| +2451485|42571|14786|52989|885443|1450|14657|8|87|93|22|63.19|124.48|44.81|0.00|985.82|1390.18|2738.56|78.86|0.00|985.82|1064.68|-404.36| +2451485|42571|5953|52989|885443|1450|14657|8|2|93|45|1.57|1.89|0.30|0.00|13.50|70.65|85.05|0.00|0.00|13.50|13.50|-57.15| +2452580|66189|4221|73789|1404686|3583|6480|4|15|94|82|79.00|114.55|38.94|0.00|3193.08|6478.00|9393.10|287.37|0.00|3193.08|3480.45|-3284.92| +2452580|66189|17262|73789|1404686|3583|6480|4|21|94|99|53.58|105.55|62.27|0.00|6164.73|5304.42|10449.45|184.94|0.00|6164.73|6349.67|860.31| +2452580|66189|6427|73789|1404686|3583|6480|4|293|94|16|39.12|72.37|56.44|0.00|903.04|625.92|1157.92|36.12|0.00|903.04|939.16|277.12| +2452580|66189|4453|73789|1404686|3583|6480|4|56|94|44|22.11|39.57|30.46|0.00|1340.24|972.84|1741.08|120.62|0.00|1340.24|1460.86|367.40| +2452580||17353|73789|1404686||6480|||94|65|||99.42|0.00|||6875.05|129.24|0.00||6591.54|962.00| +2452580|66189|16455|73789|1404686|3583|6480|4|240|94|89|4.15|6.59|5.33|128.07|474.37|369.35|586.51|3.46|128.07|346.30|349.76|-23.05| +2452580|66189|15121|73789|1404686|3583|6480|4|151|94|16|39.98|59.17|30.76|280.53|492.16|639.68|946.72|2.11|280.53|211.63|213.74|-428.05| +2452580|66189|12979|73789|1404686|3583|6480|4|122|94|57|34.66|51.29|34.36|0.00|1958.52|1975.62|2923.53|19.58|0.00|1958.52|1978.10|-17.10| +2452580|66189|9513|73789|1404686|3583|6480|4|85|94|62|53.30|93.27|46.63|0.00|2891.06|3304.60|5782.74|28.91|0.00|2891.06|2919.97|-413.54| +2451092|30629|2479|20113|1706073|6786|13146|7|215|95|35|62.81|104.26|78.19|0.00|2736.65|2198.35|3649.10|191.56|0.00|2736.65|2928.21|538.30| +2451092|30629|7220|20113|||13146||208|95|76|51.02|81.63||0.00|1922.80|3877.52|6203.88|153.82|0.00||2076.62|| +2451092|30629|3580|20113|1706073|6786|13146|7|54|95|27|30.67|60.72|40.07|0.00|1081.89|828.09|1639.44|54.09|0.00|1081.89|1135.98|253.80| +||1586||1706073|6786|13146||247|95||31.60|57.19|||||1544.13|62.52|||757.23|| +2451092|30629|17767|20113|1706073|6786|13146|7|211|95|35|93.59|99.20|25.79|0.00|902.65|3275.65|3472.00|27.07|0.00|902.65|929.72|-2373.00| +2451092|30629|13382|20113|1706073|6786|13146|7|143|95|66|10.09|18.96|16.68|0.00|1100.88|665.94|1251.36|11.00|0.00|1100.88|1111.88|434.94| +2451092|30629|13364|20113|1706073|6786|13146|7|294|95|48|3.82|5.99|3.17|120.20|152.16|183.36|287.52|1.91|120.20|31.96|33.87|-151.40| +2451092|30629|5072|20113|1706073|6786|13146|7|199|95|74|69.38|105.45|6.32|261.90|467.68|5134.12|7803.30|8.23|261.90|205.78|214.01|-4928.34| +2451092|30629|7372|20113|1706073|6786|13146|7|226|95|88|22.19|22.85|2.05|0.00|180.40|1952.72|2010.80|0.00|0.00|180.40|180.40|-1772.32| +2451092|30629|874|20113|1706073|6786|13146|7|286|95|55|78.40|123.08|29.53|0.00|1624.15|4312.00|6769.40|0.00|0.00|1624.15|1624.15|-2687.85| +2451092|30629|1052|20113|1706073|6786|13146|7|83|95|58|39.53|68.38|56.75|0.00|3291.50|2292.74|3966.04|65.83|0.00|3291.50|3357.33|998.76| +2451092|30629|4903|20113|1706073|6786|13146|7|288|95|29|92.49|184.98|61.04|0.00|1770.16|2682.21|5364.42|88.50|0.00|1770.16|1858.66|-912.05| +2451092|30629|15115|20113|1706073|6786|13146|7|196|95|53|35.19|45.39|44.02|0.00|2333.06|1865.07|2405.67|139.98|0.00|2333.06|2473.04|467.99| +2451092|30629|9343|20113|1706073|6786|13146|7|52|95|15|5.89|8.54|5.20|0.00|78.00|88.35|128.10|3.90|0.00|78.00|81.90|-10.35| +2452189|72786|5555|91907|868252|6644|10589|8|14|96|81|25.32|43.04|9.03|0.00|731.43|2050.92|3486.24|36.57|0.00|731.43|768.00|-1319.49| +2452189|72786|595|91907|868252|6644|10589|8|296|96|73|7.11|8.67|5.20|0.00|379.60|519.03|632.91|3.79|0.00|379.60|383.39|-139.43| +2452189|72786|17611|91907|868252|6644|10589|8|291|96|18|67.01|115.92|67.23|0.00|1210.14|1206.18|2086.56|60.50|0.00|1210.14|1270.64|3.96| +2452189|72786|14383|91907|868252|6644|10589|8|191|96|49|46.24|49.93|32.45|0.00|1590.05|2265.76|2446.57|15.90|0.00|1590.05|1605.95|-675.71| +2452189|72786|3143|91907|868252|6644|10589|8|229|96|6|91.64|106.30|25.51|0.00|153.06|549.84|637.80|7.65|0.00|153.06|160.71|-396.78| +2452189|72786|8837|91907|868252|6644|10589|8|111|96|61|1.30|1.72|1.29|0.00|78.69|79.30|104.92|5.50|0.00|78.69|84.19|-0.61| +2452189|72786|531|91907|868252|6644|10589|8|26|96|61|35.80|64.08|61.51|0.00|3752.11|2183.80|3908.88|75.04|0.00|3752.11|3827.15|1568.31| +2452189|72786|3101|91907|868252|6644|10589|8|40|96|55|38.95|47.51|36.58|804.76|2011.90|2142.25|2613.05|84.49|804.76|1207.14|1291.63|-935.11| +2452189|72786|9073|91907|868252|6644|10589|8|280|96|32|87.28|120.44|83.10|0.00|2659.20|2792.96|3854.08|132.96|0.00|2659.20|2792.16|-133.76| +|45464|16225|6705|1157997|2536|35475|2||97|26|62.63||||620.36|||18.61||||-1008.02| +2452049|45464|7579|6705|1157997|2536|35475|2|260|97|36|89.62|140.70|80.19|0.00|2886.84|3226.32|5065.20|28.86|0.00|2886.84|2915.70|-339.48| +2452049|45464|12297|6705|1157997|2536|35475|2|288|97|50|59.34|105.03|7.35|62.47|367.50|2967.00|5251.50|21.35|62.47|305.03|326.38|-2661.97| +2452049|45464|6599|6705|1157997|2536|35475|2|98|97|4|13.39|22.49|21.36|0.00|85.44|53.56|89.96|3.41|0.00|85.44|88.85|31.88| +2452049|45464|11421|6705|1157997|2536|35475|2|22|97|26|13.87|25.93|9.85|0.00|256.10|360.62|674.18|2.56|0.00|256.10|258.66|-104.52| +2452049|45464|16719|6705|1157997|2536|35475|2|214|97|57|45.95|70.30|50.61|115.39|2884.77|2619.15|4007.10|193.85|115.39|2769.38|2963.23|150.23| +2452049|45464|10057|6705|1157997|2536|35475|2|107|97|1|61.89|117.59|114.06|0.00|114.06|61.89|117.59|6.84|0.00|114.06|120.90|52.17| +2452049|45464|17883|6705|1157997|2536|35475|2|5|97|18|21.69|31.45|30.82|0.00|554.76|390.42|566.10|38.83|0.00|554.76|593.59|164.34| +2452049|45464|8467|6705|1157997|2536|35475|2|52|97|9|48.36|80.76|61.37|508.14|552.33|435.24|726.84|2.20|508.14|44.19|46.39|-391.05| +2452049|45464|6569|6705|1157997|2536|35475|2|80|97|50|3.30|3.43|2.53|0.00|126.50|165.00|171.50|11.38|0.00|126.50|137.88|-38.50| +2452049|45464|12387|6705|1157997|2536|35475|2|113|97|51|29.55|54.07|40.01|0.00|2040.51|1507.05|2757.57|142.83|0.00|2040.51|2183.34|533.46| +2452049|45464|5039|6705|1157997|2536|35475|2|4|97|36|57.21|93.82|86.31|0.00|3107.16|2059.56|3377.52|248.57|0.00|3107.16|3355.73|1047.60| +2452049|45464|1835|6705|1157997|2536|35475|2|58|97|30|46.91|89.12|52.58|0.00|1577.40|1407.30|2673.60|126.19|0.00|1577.40|1703.59|170.10| +2452049|45464|14611|6705|1157997|2536|35475|2|289|97|79|92.88|100.31|47.14|2457.87|3724.06|7337.52|7924.49|12.66|2457.87|1266.19|1278.85|-6071.33| +2452049|45464|6239|6705|1157997|2536|35475|2|78|97|26|11.18|11.40|4.10|0.00|106.60|290.68|296.40|3.19|0.00|106.60|109.79|-184.08| +2452439|29611|4741|2320|232958|2796|43834|7|267|98|29|75.13|148.75|133.87|0.00|3882.23|2178.77|4313.75|232.93|0.00|3882.23|4115.16|1703.46| +2452439|29611|16521|2320|232958|2796|43834|7|216|98|31|55.29|57.50|11.50|0.00|356.50|1713.99|1782.50|3.56|0.00|356.50|360.06|-1357.49| +2452439|29611|9235|2320|232958|2796|43834|7|226|98|79|23.25|40.68|18.30|0.00|1445.70|1836.75|3213.72|115.65|0.00|1445.70|1561.35|-391.05| +2452439|29611|11871|2320|232958|2796|43834|7|139|98|97|3.75|4.20|2.31|0.00|224.07|363.75|407.40|0.00|0.00|224.07|224.07|-139.68| +2452439|29611|3225|2320|232958|2796|43834|7|296|98|100|5.73|7.21|1.94|0.00|194.00|573.00|721.00|17.46|0.00|194.00|211.46|-379.00| +2452439|29611|283|2320|232958|2796|43834|7|91|98|93|75.96|121.53|74.13|0.00|6894.09|7064.28|11302.29|206.82|0.00|6894.09|7100.91|-170.19| +2452439|29611|8799|2320|232958|2796|43834|7|209|98|92|76.56|116.37|65.16|2038.20|5994.72|7043.52|10706.04|39.56|2038.20|3956.52|3996.08|-3087.00| +2452439|29611|13440|2320|232958|2796|43834|7|199|98|57|52.59|65.73|45.35|0.00|2584.95|2997.63|3746.61|51.69|0.00|2584.95|2636.64|-412.68| +2452439|29611|13681|2320|232958|2796|43834|7|224|98|67|70.28|109.63|74.54|4444.82|4994.18|4708.76|7345.21|21.97|4444.82|549.36|571.33|-4159.40| +2452439|64196|903|67232|1472308|3784|460|2|95|99|88|86.79|165.76|59.67|0.00|5250.96|7637.52|14586.88|262.54|0.00|5250.96|5513.50|-2386.56| +2452439|64196|15222|67232|1472308|3784|460|2|194|99|17|90.59|144.03|20.16|41.12|342.72|1540.03|2448.51|21.11|41.12|301.60|322.71|-1238.43| +2452439|64196|11499|67232|1472308|3784|460|2|225|99|84|55.04|99.62|58.77|0.00|4936.68|4623.36|8368.08|246.83|0.00|4936.68|5183.51|313.32| +2452439|64196|15141|67232|1472308|3784|460|2|25|99|88|98.66|156.86|111.37|6664.38|9800.56|8682.08|13803.68|250.89|6664.38|3136.18|3387.07|-5545.90| +2452439|64196|3273|67232|1472308|3784|460|2|186|99|38|44.33|44.77|17.01|0.00|646.38|1684.54|1701.26|38.78|0.00|646.38|685.16|-1038.16| +2452439|64196|4758|67232|1472308|3784|460|2|66|99|69|77.38|149.34|23.89|0.00|1648.41|5339.22|10304.46|115.38|0.00|1648.41|1763.79|-3690.81| +2452439|64196|12801|67232|1472308|3784|460|2|94|99|60|78.44|119.22|112.06|537.88|6723.60|4706.40|7153.20|123.71|537.88|6185.72|6309.43|1479.32| +2452439|64196|537|67232|1472308|3784|460|2|119|99|62|16.05|27.60|12.69|0.00|786.78|995.10|1711.20|47.20|0.00|786.78|833.98|-208.32| +2452439|64196|14931|67232|1472308|3784|460|2|123|99|90|94.40|179.36|150.66|0.00|13559.40|8496.00|16142.40|0.00|0.00|13559.40|13559.40|5063.40| +2452439|64196|3007|67232|1472308|3784|460|2|213|99|91|60.07|89.50|75.18|0.00|6841.38|5466.37|8144.50|615.72|0.00|6841.38|7457.10|1375.01| +2452439|64196|4921|67232|1472308|3784|460|2|116|99|61|40.54|68.10|35.41|0.00|2160.01|2472.94|4154.10|86.40|0.00|2160.01|2246.41|-312.93| +2452326|34293|13537|15875|830977|1279|43581|1|10|100|95|98.78|123.47|59.26|0.00|5629.70|9384.10|11729.65|56.29|0.00|5629.70|5685.99|-3754.40| +2452326|34293|1383|15875|830977|1279|43581|1|257|100|95|31.37|59.91|37.14|0.00|3528.30|2980.15|5691.45|141.13|0.00|3528.30|3669.43|548.15| +2452326|34293|16317|15875|830977|1279|43581|1|167|100|8|66.04|91.79|51.40|0.00|411.20|528.32|734.32|16.44|0.00|411.20|427.64|-117.12| +2452326|34293|12354|15875|830977|1279|43581|1|273|100|31|10.04|17.57|14.75|0.00|457.25|311.24|544.67|0.00|0.00|457.25|457.25|146.01| +2452326|34293|4341|15875|830977|1279|43581|1|35|100|19|62.84|69.12|51.84|719.02|984.96|1193.96|1313.28|10.63|719.02|265.94|276.57|-928.02| +2452326|34293|12343|15875|830977|1279|43581|1|242|100|81|89.21|155.22|65.19|0.00|5280.39|7226.01|12572.82|422.43|0.00|5280.39|5702.82|-1945.62| +2452326|34293|16032|15875|830977|1279|43581|1|94|100|32|10.70|14.23|0.42|0.00|13.44|342.40|455.36|0.53|0.00|13.44|13.97|-328.96| +2452326|34293|11833|15875|830977|1279|43581|1|46|100|36|27.07|50.62|31.89|0.00|1148.04|974.52|1822.32|103.32|0.00|1148.04|1251.36|173.52| +2452326|34293|8661|15875|830977|1279|43581|1|28|100|16|59.60|63.77|3.18|0.00|50.88|953.60|1020.32|3.05|0.00|50.88|53.93|-902.72| +2452326|34293|3966|15875|830977|1279|43581|1|44|100|14|40.69|50.86|28.99|0.00|405.86|569.66|712.04|4.05|0.00|405.86|409.91|-163.80| +2452326|34293|8533|15875|830977|1279||1||100|22|25.82|||0.00|977.02|568.04|977.02|68.39|0.00|||| +2452326|34293|8929|15875|830977|1279|43581|1|110|100|46|92.26|155.91|12.47|0.00|573.62|4243.96|7171.86|34.41|0.00|573.62|608.03|-3670.34| +2452326|34293|2335|15875|830977|1279|43581|1|185|100|24|42.57|84.28|1.68|33.06|40.32|1021.68|2022.72|0.50|33.06|7.26|7.76|-1014.42| +2452326|34293|3349|15875|830977|1279|43581|1|263|100|76|34.63|48.48|43.14|0.00|3278.64|2631.88|3684.48|98.35|0.00|3278.64|3376.99|646.76| +2451220|63410|16136|65242|841963|2684|41758|1|38|101|57|30.54|31.76|13.65|0.00|778.05|1740.78|1810.32|54.46|0.00|778.05|832.51|-962.73| +2451220|63410|12370|65242|841963|2684|41758|1|144|101|83|54.21|70.47|40.16|1333.31|3333.28|4499.43|5849.01|179.99|1333.31|1999.97|2179.96|-2499.46| +2451220|63410|15799|65242|841963|2684|41758|1|16|101|8|77.16|88.73|32.83|0.00|262.64|617.28|709.84|13.13|0.00|262.64|275.77|-354.64| +||14276|||2684|||61|101|77|96.00|175.68|50.94|0.00|||13527.36|353.01|0.00||4275.39|-3469.62| +2451220|63410|13688|65242|841963|2684|41758|1|102|101|90|28.72|30.15|25.02|0.00|2251.80|2584.80|2713.50|202.66|0.00|2251.80|2454.46|-333.00| +2451220|63410|5557|65242|841963|2684|41758|1|116|101|47|8.30|15.85|11.72|0.00|550.84|390.10|744.95|44.06|0.00|550.84|594.90|160.74| +2451220|63410|4264|65242|841963|2684|41758|1|227|101|20|20.95|37.08|7.78|0.00|155.60|419.00|741.60|3.11|0.00|155.60|158.71|-263.40| +2451220|63410|10886|65242|841963|2684|41758|1|245|101|8|33.61|53.10|3.18|0.00|25.44|268.88|424.80|0.00|0.00|25.44|25.44|-243.44| +2451220|63410|5989|65242|841963|2684|41758|1|3|101|15|96.04|186.31|158.36|0.00|2375.40|1440.60|2794.65|118.77|0.00|2375.40|2494.17|934.80| +2451220|63410|6824|65242|841963|2684|41758|1|174|101|55|37.55|45.06|35.59|0.00|1957.45|2065.25|2478.30|58.72|0.00|1957.45|2016.17|-107.80| +2451220|63410|5068|65242|841963|2684|41758|1|152|101|53|27.26|44.16|19.43|0.00|1029.79|1444.78|2340.48|10.29|0.00|1029.79|1040.08|-414.99| +2451220||10826|||2684|41758||33|101||35.64|||0.00|946.83|1318.68|||0.00|946.83||| +2451220|63410|14989|65242|841963|2684|41758|1|17|101|91|38.15|74.77|33.64|0.00|3061.24|3471.65|6804.07|122.44|0.00|3061.24|3183.68|-410.41| +2451220||6706|65242||||1||101|93|95.73|148.38||0.00|||13799.34|137.99|0.00|13799.34||| +2451220|63410|17974|65242|841963|2684|41758|1|10|101|24|30.44|37.44|32.57|703.51|781.68|730.56|898.56|0.78|703.51|78.17|78.95|-652.39| +2451220|63410|2737|65242|841963|2684|41758|1|34|101|48|72.07|121.07|88.38|0.00|4242.24|3459.36|5811.36|42.42|0.00|4242.24|4284.66|782.88| +2452557|66012|5511|42643|880785|4527|1069|1|121|102|31|83.08|101.35|98.30|1797.90|3047.30|2575.48|3141.85|0.00|1797.90|1249.40|1249.40|-1326.08| +|66012|9393|||4527|||146|102||34.88||19.01|||||22.24||741.39||-618.93| +2452557|66012|15348|42643|880785|4527|1069|1|144|102|47|3.00|5.31|0.84|0.00|39.48|141.00|249.57|2.36|0.00|39.48|41.84|-101.52| +2452557|66012|17413|42643|880785|4527|1069|1|120|102|29|53.12|55.24|44.74|687.65|1297.46|1540.48|1601.96|36.58|687.65|609.81|646.39|-930.67| +2452557|66012|3159|42643|880785|4527|1069|1|224|102|57|25.72|50.92|50.41|0.00|2873.37|1466.04|2902.44|229.86|0.00|2873.37|3103.23|1407.33| +2452557|66012|5520|42643|880785|4527|1069|1|285|102|63|10.58|17.88|1.60|0.00|100.80|666.54|1126.44|5.04|0.00|100.80|105.84|-565.74| +2452557|66012|6339|42643|880785|4527|1069|1|83|102|18|79.15|107.64|40.90|0.00|736.20|1424.70|1937.52|29.44|0.00|736.20|765.64|-688.50| +2452557|66012|11563|42643|880785|4527|1069|1|76|102|42|38.50|51.59|32.50|0.00|1365.00|1617.00|2166.78|13.65|0.00|1365.00|1378.65|-252.00| +||14665||880785|||1|275|102|||76.57||0.00|1736.28||4823.91|52.08|0.00|||-843.57| +2451981|42546|8115|69235|1108314|193|35319|7|208|103|26|57.44|74.67|41.06|0.00|1067.56|1493.44|1941.42|10.67|0.00|1067.56|1078.23|-425.88| +2451981|42546|2527|69235|1108314|193|35319|7|264|103|95|62.89|76.09|63.15|0.00|5999.25|5974.55|7228.55|179.97|0.00|5999.25|6179.22|24.70| +2451981|42546|15841|69235|1108314|193|35319|7|185|103|8|69.59|126.65|84.85|0.00|678.80|556.72|1013.20|0.00|0.00|678.80|678.80|122.08| +2451981|42546|11443|69235|1108314|193|35319|7|5|103|69|41.22|77.08|8.47|0.00|584.43|2844.18|5318.52|23.37|0.00|584.43|607.80|-2259.75| +2451981|42546|2017|69235|1108314|193|35319|7|285|103|95|9.21|14.92|9.69|0.00|920.55|874.95|1417.40|82.84|0.00|920.55|1003.39|45.60| +2451981|42546|13285|69235|1108314|193|35319|7|234|103|78|4.59|8.53|3.24|242.61|252.72|358.02|665.34|0.80|242.61|10.11|10.91|-347.91| +2451981|42546|13123|69235|1108314|193|35319|7|192|103|75|89.57|123.60|30.90|0.00|2317.50|6717.75|9270.00|208.57|0.00|2317.50|2526.07|-4400.25| +2451981||13663|69235||193|35319|||103|||107.50|||6454.30|6433.76|||||6841.55|20.54| +2451510|52780|10966|3138|1828712|4333|12707|8|19|104|66|70.83|84.99|41.64|0.00|2748.24|4674.78|5609.34|0.00|0.00|2748.24|2748.24|-1926.54| +2451510|52780|4231|3138|1828712|4333|12707|8|107|104|90|21.26|29.97|13.78|0.00|1240.20|1913.40|2697.30|37.20|0.00|1240.20|1277.40|-673.20| +2451510|52780|4958|3138|1828712|4333|12707|8|86|104|64|20.17|26.82|4.55|0.00|291.20|1290.88|1716.48|23.29|0.00|291.20|314.49|-999.68| +2451510|52780|7346|3138|1828712|4333|12707|8|267|104|90|95.85|130.35|95.15|0.00|8563.50|8626.50|11731.50|599.44|0.00|8563.50|9162.94|-63.00| +2451510|52780|15434|3138|1828712|4333|12707|8|231|104|39|85.11|119.15|106.04|0.00|4135.56|3319.29|4646.85|372.20|0.00|4135.56|4507.76|816.27| +2451510|52780|3034|3138|1828712|4333|12707|8|294|104|70|25.53|41.35|22.74|0.00|1591.80|1787.10|2894.50|0.00|0.00|1591.80|1591.80|-195.30| +2451510|52780|16214|3138|1828712|4333|12707|8|35|104|14|21.02|38.67|15.46|0.00|216.44|294.28|541.38|2.16|0.00|216.44|218.60|-77.84| +2451510|52780|3218|3138|1828712|4333|12707|8|287|104|4|19.25|30.80|20.32|77.21|81.28|77.00|123.20|0.00|77.21|4.07|4.07|-72.93| +2451510|52780|11618|3138|1828712|4333|12707|8|295|104|13|95.32|129.63|7.77|0.00|101.01|1239.16|1685.19|5.05|0.00|101.01|106.06|-1138.15| +2451510|52780|5984|3138||4333|12707|||104|54||86.41|63.94|0.00||4530.60|4666.14|276.22|0.00|||-1077.84| +2451510|52780|13520|3138|1828712|4333|12707|8|170|104|94|92.31|129.23|107.26|0.00|10082.44|8677.14|12147.62|604.94|0.00|10082.44|10687.38|1405.30| +2451510|52780|13544|3138|1828712|4333|12707|8|255|104|75|6.64|7.17|6.02|0.00|451.50|498.00|537.75|31.60|0.00|451.50|483.10|-46.50| +2451510|52780|686|3138|1828712|4333|12707|8|48|104|25|17.32|32.04|4.48|0.00|112.00|433.00|801.00|10.08|0.00|112.00|122.08|-321.00| +2452228|74965|1263|54634|1095567|1653|4586|4|103|105|1|65.23|113.50|107.82|0.00|107.82|65.23|113.50|0.00|0.00|107.82|107.82|42.59| +2452228|74965|15851|54634|1095567|1653|4586|4|81|105|73|52.02|96.75|6.77|0.00|494.21|3797.46|7062.75|9.88|0.00|494.21|504.09|-3303.25| +2452228|74965|7639|54634|1095567|1653|4586|4|82|105|10|71.73|101.85|66.20|0.00|662.00|717.30|1018.50|33.10|0.00|662.00|695.10|-55.30| +|74965|6061||1095567||4586||85|105|||||||244.53|249.21|10.35||129.48|139.83|| +2452228|74965|6791|54634|1095567|1653|4586|4|185|105|31|25.22|42.87|0.42|0.52|13.02|781.82|1328.97|1.12|0.52|12.50|13.62|-769.32| +2452228|74965|11431|54634|1095567|1653|4586|4|206|105|57|82.35|102.11|47.99|0.00|2735.43|4693.95|5820.27|54.70|0.00|2735.43|2790.13|-1958.52| +2452228|74965|12691|54634|1095567|1653|4586|4|29|105|68|99.48|121.36|4.85|0.00|329.80|6764.64|8252.48|0.00|0.00|329.80|329.80|-6434.84| +2452228|74965|17559|54634|1095567|1653|4586|4|119|105|26|20.38|21.80|1.96|0.00|50.96|529.88|566.80|2.03|0.00|50.96|52.99|-478.92| +2452228|74965|13629|54634|1095567|1653|4586|4|253|105|15|86.75|173.50|156.15|0.00|2342.25|1301.25|2602.50|70.26|0.00|2342.25|2412.51|1041.00| +2451935|66128|7933|77533|997877|1774|19271|8|287|106|37|43.69|53.73|34.38|0.00|1272.06|1616.53|1988.01|63.60|0.00|1272.06|1335.66|-344.47| +2451935|66128|2871|77533|997877|1774|19271|8|94|106|81|60.49|76.82|29.95|0.00|2425.95|4899.69|6222.42|0.00|0.00|2425.95|2425.95|-2473.74| +2451935|66128|12267|77533|997877|1774|19271|8|229|106|19|46.52|47.91|26.82|0.00|509.58|883.88|910.29|30.57|0.00|509.58|540.15|-374.30| +2451935|66128|231|77533|997877|1774|19271|8|26|106|9|44.39|58.59|38.08|54.83|342.72|399.51|527.31|25.91|54.83|287.89|313.80|-111.62| +2451935||9187||997877||||99|106|63|32.99|32.99|16.16||||2078.37|||1018.08|1109.70|-1060.29| +2451935|66128|6477|77533|997877|1774|19271|8|191|106|19|25.28|44.24|15.92|0.00|302.48|480.32|840.56|6.04|0.00|302.48|308.52|-177.84| +2451935|66128|1811|77533|997877|1774|19271|8|51|106|92|49.68|53.15|3.18|0.00|292.56|4570.56|4889.80|26.33|0.00|292.56|318.89|-4278.00| +2451935|66128|5171|77533|997877|1774|19271|8|261|106|70|25.14|45.00|28.35|0.00|1984.50|1759.80|3150.00|99.22|0.00|1984.50|2083.72|224.70| +2451935|66128|5999|77533|997877|1774|19271|8|254|106|56|14.38|16.39|1.47|0.00|82.32|805.28|917.84|0.00|0.00|82.32|82.32|-722.96| +2451935|66128|14369|77533|997877|1774|19271|8|250|106|50|24.00|40.56|23.11|0.00|1155.50|1200.00|2028.00|103.99|0.00|1155.50|1259.49|-44.50| +2451954|48538|1181|5780|838019|2258|12497|10|295|107|24|81.42|100.96|32.30|0.00|775.20|1954.08|2423.04|38.76|0.00|775.20|813.96|-1178.88| +2451954|48538|14869|5780|838019|2258|12497|10|69|107|6|8.51|9.19|4.31|0.00|25.86|51.06|55.14|1.03|0.00|25.86|26.89|-25.20| +2451954||12973||||12497|||107||48.81|55.15||0.00|759.39||||0.00|759.39||| +||6363|||||||107||10.93|||0.00|70.20|655.80|1173.60||0.00|||-585.60| +2451954|48538|3019|5780|838019|2258|12497|10|263|107|85|1.38|2.64|2.58|0.00|219.30|117.30|224.40|10.96|0.00|219.30|230.26|102.00| +2451954|48538|16445|5780|838019|2258|12497|10|265|107|55|43.56|87.12|13.06|0.00|718.30|2395.80|4791.60|28.73|0.00|718.30|747.03|-1677.50| +2451954|48538|15653|5780|838019|2258|12497|10|290|107|88|48.45|64.92|16.23|0.00|1428.24|4263.60|5712.96|99.97|0.00|1428.24|1528.21|-2835.36| +2451954||3689||838019|2258|12497||215|107|73||187.04||0.00|12424.60|6826.96|13653.92||0.00|12424.60||| +2451534|35521|13309|55322|1190747|4572|19197|10|186|108|44|12.64|24.39|1.95|0.00|85.80|556.16|1073.16|1.71|0.00|85.80|87.51|-470.36| +2451534|35521|2060|55322|1190747|4572|19197|10|85|108|68|26.25|37.01|20.72|0.00|1408.96|1785.00|2516.68|56.35|0.00|1408.96|1465.31|-376.04| +2451534|35521|14908|55322|1190747|4572|19197|10|221|108|78|67.82|107.15|88.93|0.00|6936.54|5289.96|8357.70|138.73|0.00|6936.54|7075.27|1646.58| +2451534|35521|5356|55322|1190747|4572|19197|10|106|108|100|59.14|83.38|7.50|142.50|750.00|5914.00|8338.00|42.52|142.50|607.50|650.02|-5306.50| +2451534|35521|974|55322|1190747|4572|19197|10|115|108|22|34.28|54.84|3.83|0.00|84.26|754.16|1206.48|5.89|0.00|84.26|90.15|-669.90| +2451534|35521|2576|55322|1190747|4572|19197|10|175|108|9|10.22|16.04|1.28|0.00|11.52|91.98|144.36|0.46|0.00|11.52|11.98|-80.46| +2451534|35521|16975|55322|1190747|4572|19197|10|176|108|73|86.26|100.06|8.00|0.00|584.00|6296.98|7304.38|5.84|0.00|584.00|589.84|-5712.98| +2451534||4102|||||10||108|39|57.84|||||2255.76|2503.80|15.01||||| +2451534|35521|17792|55322|1190747|4572|19197|10|284|108|18|5.50|6.76|3.44|0.00|61.92|99.00|121.68|4.33|0.00|61.92|66.25|-37.08| +2451534|35521|5564|55322|1190747|4572|19197|10|292|108|64|95.83|122.66|120.20|0.00|7692.80|6133.12|7850.24|384.64|0.00|7692.80|8077.44|1559.68| +2451534|35521|5764|55322|1190747|4572|19197|10|9|108|83|31.54|44.15|31.78|1055.09|2637.74|2617.82|3664.45|31.65|1055.09|1582.65|1614.30|-1035.17| +2451534|35521|5563|55322|1190747|4572|19197|10|59|108|65|43.58|50.55|8.59|0.00|558.35|2832.70|3285.75|0.00|0.00|558.35|558.35|-2274.35| +2451534|35521|12907|55322|1190747|4572|19197|10|35|108|34|33.27|40.58|29.62|0.00|1007.08|1131.18|1379.72|90.63|0.00|1007.08|1097.71|-124.10| +2451534|35521|12523|55322|1190747|4572|19197|10|173|108|35|27.74|41.88|9.63|0.00|337.05|970.90|1465.80|20.22|0.00|337.05|357.27|-633.85| +2451534|35521|4420|55322|1190747|4572|19197|10|102|108|55|44.26|67.71|6.77|0.00|372.35|2434.30|3724.05|11.17|0.00|372.35|383.52|-2061.95| +2451534|35521|17950|55322|1190747|4572|19197|10|35|108|79|55.65|92.93|43.67|0.00|3449.93|4396.35|7341.47|206.99|0.00|3449.93|3656.92|-946.42| +2451516|66850|10118|68136|1182996|1856|24669|1|225|109|95|17.43|32.59|6.84|0.00|649.80|1655.85|3096.05|25.99|0.00|649.80|675.79|-1006.05| +2451516|66850|8044|68136|1182996|1856|24669|1|163|109|77|43.85|71.47|2.14|0.00|164.78|3376.45|5503.19|13.18|0.00|164.78|177.96|-3211.67| +2451516|66850|15175|68136|1182996|1856|24669|1|248|109|64|24.34|45.75|13.26|0.00|848.64|1557.76|2928.00|0.00|0.00|848.64|848.64|-709.12| +2451516|66850|13111|68136|1182996|1856|24669|1|74|109|95|72.53|121.12|75.09|0.00|7133.55|6890.35|11506.40|642.01|0.00|7133.55|7775.56|243.20| +2451516|66850|7006|68136|1182996|1856|24669|1|265|109|3|68.36|92.28|16.61|0.00|49.83|205.08|276.84|0.99|0.00|49.83|50.82|-155.25| +2451516|66850|2305|68136|1182996|1856|24669|1|287|109|30|93.80|107.87|83.05|0.00|2491.50|2814.00|3236.10|174.40|0.00|2491.50|2665.90|-322.50| +2451516|66850|745|68136|1182996|1856|24669|1|285|109|30|58.47|107.58|91.44|0.00|2743.20|1754.10|3227.40|27.43|0.00|2743.20|2770.63|989.10| +2451516|66850|4288|68136|1182996|1856|24669|1|184|109|8|16.68|28.52|12.83|0.00|102.64|133.44|228.16|3.07|0.00|102.64|105.71|-30.80| +2451516|66850|11266|68136|1182996|1856|24669|1|191|109|87|80.20|103.45|70.34|0.00|6119.58|6977.40|9000.15|183.58|0.00|6119.58|6303.16|-857.82| +2451516|66850|12260|68136|1182996|1856|24669|1|30|109|60|53.83|64.05|2.56|52.22|153.60|3229.80|3843.00|7.09|52.22|101.38|108.47|-3128.42| +2451631|71018|451|82405|1050977|1149|21321|8|73|110|94|15.90|31.80|4.45|0.00|418.30|1494.60|2989.20|8.36|0.00|418.30|426.66|-1076.30| +2451631|71018|10049|82405|1050977|1149|21321|8|254|110|92|11.74|20.42|8.37|0.00|770.04|1080.08|1878.64|69.30|0.00|770.04|839.34|-310.04| +2451631|71018|12455|82405|1050977|1149|21321|8|125|110|48|26.59|47.06|32.47|0.00|1558.56|1276.32|2258.88|77.92|0.00|1558.56|1636.48|282.24| +2451631|71018|8809|82405|1050977|1149|21321|8|278|110|97|86.67|147.33|1.47|0.00|142.59|8406.99|14291.01|11.40|0.00|142.59|153.99|-8264.40| +2451631|71018|13304|82405|1050977|1149|21321|8|197|110|29|72.04|72.76|8.00|0.00|232.00|2089.16|2110.04|2.32|0.00|232.00|234.32|-1857.16| +2451631|71018|10934|82405|1050977|1149|21321|8|40|110|79|62.63|119.62|105.26|0.00|8315.54|4947.77|9449.98|665.24|0.00|8315.54|8980.78|3367.77| +2451631|71018|7699|82405|1050977|1149|21321|8|256|110|47|78.03|107.68|65.68|0.00|3086.96|3667.41|5060.96|216.08|0.00|3086.96|3303.04|-580.45| +2451631|71018|1403|82405|1050977|1149|21321|8|177|110|79|33.58|51.71|29.99|0.00|2369.21|2652.82|4085.09|71.07|0.00|2369.21|2440.28|-283.61| +2452588|54461|3097|57948|905382|1969|20203|2|41|111|10|74.99|129.73|121.94|0.00|1219.40|749.90|1297.30|12.19|0.00|1219.40|1231.59|469.50| +2452588|54461|2197|57948|905382|1969|20203|2|86|111|70|86.08|136.00|2.72|0.00|190.40|6025.60|9520.00|7.61|0.00|190.40|198.01|-5835.20| +2452588|54461|5991|57948|905382|1969|20203|2|180|111|96|98.64|179.52|55.65|0.00|5342.40|9469.44|17233.92|427.39|0.00|5342.40|5769.79|-4127.04| +2452588|54461|10821|57948|905382|1969|20203|2|215|111|73|57.95|97.93|0.97|0.00|70.81|4230.35|7148.89|4.95|0.00|70.81|75.76|-4159.54| +2452588|54461|16243|57948|905382|1969|20203|2|240|111|20|17.73|35.46|31.20|68.64|624.00|354.60|709.20|33.32|68.64|555.36|588.68|200.76| +||2916||905382||20203|||111||||||504.90|755.92|||||535.19|| +2452588|54461|17575|57948|905382|1969|20203|2|148|111|79|60.57|90.24|55.94|0.00|4419.26|4785.03|7128.96|44.19|0.00|4419.26|4463.45|-365.77| +2452588|54461|6315|57948|905382|1969|20203|2|157|111|66|26.97|31.82|16.54|0.00|1091.64|1780.02|2100.12|10.91|0.00|1091.64|1102.55|-688.38| +2452588|54461|5154|57948|905382|1969|20203|2|88|111|62|10.74|19.97|16.77|0.00|1039.74|665.88|1238.14|31.19|0.00|1039.74|1070.93|373.86| +2452588|54461|16611|57948|905382|1969|20203|2|71|111|79|38.70|39.08|35.17|0.00|2778.43|3057.30|3087.32|0.00|0.00|2778.43|2778.43|-278.87| +2451027|44777|16436|21253|1807375|250|19972|1|80|112|48|22.46|44.69|32.62|0.00|1565.76|1078.08|2145.12|15.65|0.00|1565.76|1581.41|487.68| +2451027|44777|2056|21253|1807375|250|19972|1|242|112|6|73.29|85.74|72.02|0.00|432.12|439.74|514.44|25.92|0.00|432.12|458.04|-7.62| +2451027|44777|7790|21253|1807375|250|19972|1|13|112|47|28.43|48.61|22.84|912.45|1073.48|1336.21|2284.67|12.88|912.45|161.03|173.91|-1175.18| +2451027|44777|17248|21253|1807375|250|19972|1|158|112|68|9.49|16.79|10.40|410.17|707.20|645.32|1141.72|2.97|410.17|297.03|300.00|-348.29| +2451027|44777|6806|21253|1807375|250|19972|1|146|112|17|88.02|166.35|96.48|0.00|1640.16|1496.34|2827.95|32.80|0.00|1640.16|1672.96|143.82| +2451027|44777|6890|21253|1807375|250|19972|1|98|112|20|23.71|41.49|9.54|0.00|190.80|474.20|829.80|13.35|0.00|190.80|204.15|-283.40| +2451027|44777|17780|21253|1807375|250|19972|1|259|112|96|70.47|78.92|21.30|0.00|2044.80|6765.12|7576.32|184.03|0.00|2044.80|2228.83|-4720.32| +2451027|44777|4186|21253|1807375|250|19972|1|285|112|60|82.53|84.18|18.51|766.31|1110.60|4951.80|5050.80|0.00|766.31|344.29|344.29|-4607.51| +2451027|44777|12661|21253|1807375|250|19972|1|272|112|38|65.43|94.87|34.15|0.00|1297.70|2486.34|3605.06|77.86|0.00|1297.70|1375.56|-1188.64| +2451027|44777|6283|21253|1807375|250|19972|1|188|112|50|22.67|23.57|1.88|0.00|94.00|1133.50|1178.50|6.58|0.00|94.00|100.58|-1039.50| +2451027|44777|11252|21253|1807375|250|19972|1|73|112|23|60.65|114.62|108.88|0.00|2504.24|1394.95|2636.26|125.21|0.00|2504.24|2629.45|1109.29| +2451027|44777|4954|21253|1807375|250|19972|1|240|112|2|8.50|14.53|2.17|0.00|4.34|17.00|29.06|0.21|0.00|4.34|4.55|-12.66| +2451027|44777|13849|21253|1807375|250|19972|1|281|112|65|62.75|65.26|23.49|1175.67|1526.85|4078.75|4241.90|24.58|1175.67|351.18|375.76|-3727.57| +2451027|44777|11092|21253|1807375|250|19972|1|170|112|84|32.65|36.89|9.22|0.00|774.48|2742.60|3098.76|46.46|0.00|774.48|820.94|-1968.12| +2451875|40146|2282|3117|666386|2902|35771|7|101|113|57|16.61|26.40|23.76|474.01|1354.32|946.77|1504.80|44.01|474.01|880.31|924.32|-66.46| +2451875|40146|16213|3117|666386|2902|35771|7|289|113|90|30.01|45.31|3.17|0.00|285.30|2700.90|4077.90|2.85|0.00|285.30|288.15|-2415.60| +2451875|40146|12959|3117|666386|2902|35771|7|46|113|80|45.91|72.53|35.53|1932.83|2842.40|3672.80|5802.40|0.00|1932.83|909.57|909.57|-2763.23| +2451875|40146|9733|3117|666386|2902|35771|7|99|113|44|51.89|102.22|95.06|0.00|4182.64|2283.16|4497.68|125.47|0.00|4182.64|4308.11|1899.48| +2451875|40146|9551|3117|666386|2902|35771|7|182|113|3|53.03|66.81|2.00|0.00|6.00|159.09|200.43|0.18|0.00|6.00|6.18|-153.09| +2451875|40146|1843|3117|666386|2902|35771|7|198|113|59|92.82|150.36|73.67|0.00|4346.53|5476.38|8871.24|0.00|0.00|4346.53|4346.53|-1129.85| +2451875|40146|5471||666386|2902|35771|||113|62|13.42||14.88|0.00|||1139.56||0.00|922.56||90.52| +2451875|40146|4556|3117|666386|2902|35771|7|88|113|81|97.32|144.03|79.21|5453.60|6416.01|7882.92|11666.43|67.36|5453.60|962.41|1029.77|-6920.51| +2451875|40146|11299|3117|666386|2902|35771|7|268|113|91|41.08|55.45|18.29|0.00|1664.39|3738.28|5045.95|66.57|0.00|1664.39|1730.96|-2073.89| +2451875|40146|9251|3117|666386|2902|35771|7|158|113|95|82.42|135.99|54.39|0.00|5167.05|7829.90|12919.05|361.69|0.00|5167.05|5528.74|-2662.85| +2451875|40146|16499|||||7|199|113|21|32.82|62.35|57.98|73.05|1217.58|689.22|1309.35|91.56|73.05|1144.53||| +2451875|40146|13|3117|666386|2902|35771|7|298|113|94|26.62|39.93|14.37|0.00|1350.78|2502.28|3753.42|54.03|0.00|1350.78|1404.81|-1151.50| +2451875|40146|1895|3117|666386|2902|35771|7|4|113|46|16.25|18.68|13.63|0.00|626.98|747.50|859.28|37.61|0.00|626.98|664.59|-120.52| +2451875|40146|1736|3117|666386|2902|35771|7|9|113|7|20.28|26.97|9.97|0.00|69.79|141.96|188.79|1.39|0.00|69.79|71.18|-72.17| +2451966|50802|13625|44742|1209636|1730|27071|4|27|114|84|87.76|115.84|67.18|0.00|5643.12|7371.84|9730.56|338.58|0.00|5643.12|5981.70|-1728.72| +2451966|50802|12809|44742|1209636|1730|27071|4|282|114|95|90.64|140.49|18.26|0.00|1734.70|8610.80|13346.55|86.73|0.00|1734.70|1821.43|-6876.10| +2451966|50802|3415|44742|1209636|1730|27071|4|275|114|74|78.65|121.12|20.59|0.00|1523.66|5820.10|8962.88|121.89|0.00|1523.66|1645.55|-4296.44| +2451966|50802|6223|44742|1209636|1730|27071|4|240|114|29|80.29|122.04|59.79|0.00|1733.91|2328.41|3539.16|138.71|0.00|1733.91|1872.62|-594.50| +2451966|50802|4419|44742|1209636|1730|27071|4|182|114|79|17.67|21.02|6.09|0.00|481.11|1395.93|1660.58|24.05|0.00|481.11|505.16|-914.82| +2451966|50802|107|44742|1209636|1730|27071|4|82|114|71|8.31|9.88|2.47|103.46|175.37|590.01|701.48|3.59|103.46|71.91|75.50|-518.10| +2451966|50802|277|44742|1209636|1730|27071|4|188|114|97|46.60|89.47|77.83|0.00|7549.51|4520.20|8678.59|75.49|0.00|7549.51|7625.00|3029.31| +2451966|50802|4343|44742|1209636|1730|27071|4|169|114|100|18.28|32.53|31.55|0.00|3155.00|1828.00|3253.00|94.65|0.00|3155.00|3249.65|1327.00| +2451966|50802|621|44742|1209636|1730|27071|4|119|114|26|32.39|53.76|9.67|183.53|251.42|842.14|1397.76|5.43|183.53|67.89|73.32|-774.25| +2451966|50802|13821|44742|1209636|1730|27071|4|184|114|14|84.20|101.88|75.39|0.00|1055.46|1178.80|1426.32|63.32|0.00|1055.46|1118.78|-123.34| +2451966|50802|10395|44742|1209636|1730|27071|4|171|114|67|41.08|42.31|20.30|0.00|1360.10|2752.36|2834.77|122.40|0.00|1360.10|1482.50|-1392.26| +2452240|47539|4843|9380|1686818|2656|94|1|121|115|24|3.25|4.22|0.29|0.00|6.96|78.00|101.28|0.34|0.00|6.96|7.30|-71.04| +2452240|47539|1589|9380|1686818|2656|94|1|14|115|91|37.54|57.81|30.06|0.00|2735.46|3416.14|5260.71|27.35|0.00|2735.46|2762.81|-680.68| +2452240|47539|6519|9380|1686818|2656|94|1|138|115|45|38.75|46.11|27.20|0.00|1224.00|1743.75|2074.95|97.92|0.00|1224.00|1321.92|-519.75| +2452240|47539|15203|9380|1686818|2656|94|1|237|115|8|57.96|98.53|26.60|0.00|212.80|463.68|788.24|10.64|0.00|212.80|223.44|-250.88| +2452240|47539|14973|9380|1686818|2656|94|1|260|115|47|16.51|31.53|9.14|0.00|429.58|775.97|1481.91|17.18|0.00|429.58|446.76|-346.39| +2452240|47539|5341|9380|1686818|2656|94|1|279|115|71|54.88|66.95|27.44|0.00|1948.24|3896.48|4753.45|77.92|0.00|1948.24|2026.16|-1948.24| +2452240|47539|5777|9380|1686818|2656|94|1|91|115|75|83.76|130.66|86.23|0.00|6467.25|6282.00|9799.50|323.36|0.00|6467.25|6790.61|185.25| +2452240|47539|3403|9380|1686818|2656|94|1|59|115|80|51.96|95.08|43.73|1609.26|3498.40|4156.80|7606.40|37.78|1609.26|1889.14|1926.92|-2267.66| +2452240|47539|4277|9380|1686818|2656|94|1|186|115|21|44.56|53.47|1.06|7.34|22.26|935.76|1122.87|0.89|7.34|14.92|15.81|-920.84| +2452240|47539|6411|9380|1686818|2656|94|1|254|115|14|60.24|77.10|39.32|0.00|550.48|843.36|1079.40|33.02|0.00|550.48|583.50|-292.88| +2452240|47539|2817|9380|1686818|2656|94|1|181|115|60|31.03|33.20|5.64|0.00|338.40|1861.80|1992.00|30.45|0.00|338.40|368.85|-1523.40| +2452240|47539|8407|9380|1686818|2656|94|1|279|115|45|90.37|92.17|67.28|0.00|3027.60|4066.65|4147.65|121.10|0.00|3027.60|3148.70|-1039.05| +2452276|65000|2034|30787|6545|1150|17308|7|17|116|69|97.18|133.13|37.27|0.00|2571.63|6705.42|9185.97|77.14|0.00|2571.63|2648.77|-4133.79| +2452276|65000|10188|30787|6545|1150|17308|7|180|116|43|70.30|125.83|123.31|0.00|5302.33|3022.90|5410.69|371.16|0.00|5302.33|5673.49|2279.43| +2452276|65000|1638|30787|6545|1150|17308|7|10|116|31|49.30|86.27|34.50|0.00|1069.50|1528.30|2674.37|32.08|0.00|1069.50|1101.58|-458.80| +2452276||11917||6545|||7|142|116||||8.71||757.77|2564.76||25.00||500.13|525.13|| +2452276|65000|15462|30787|6545|1150|17308|7|122|116|1|32.08|44.91|38.62|3.86|38.62|32.08|44.91|2.43|3.86|34.76|37.19|2.68| +2452276|65000|11779|30787|6545|1150|17308|7|44|116|11|96.98|145.47|122.19|0.00|1344.09|1066.78|1600.17|40.32|0.00|1344.09|1384.41|277.31| +2452276|65000|16014|30787|6545|1150|17308|7|236|116|30|35.46|37.94|12.89|0.00|386.70|1063.80|1138.20|30.93|0.00|386.70|417.63|-677.10| +2452276|65000|13728|30787|6545|1150|17308|7|207|116|90|78.41|112.91|13.54|0.00|1218.60|7056.90|10161.90|73.11|0.00|1218.60|1291.71|-5838.30| +2452276|65000|16503|30787|6545|1150|17308|7|160|116|26|20.37|32.99|24.74|0.00|643.24|529.62|857.74|38.59|0.00|643.24|681.83|113.62| +2452276|65000|4057|30787|6545|1150|17308|7|183|116|57|13.80|15.59|3.58|0.00|204.06|786.60|888.63|2.04|0.00|204.06|206.10|-582.54| +2452276|65000|6456|30787|6545|1150|17308|7|38|116|43|98.17|124.67|115.94|1645.18|4985.42|4221.31|5360.81|0.00|1645.18|3340.24|3340.24|-881.07| +2452276||5401|30787||1150||7||116|21|65.16||||||2107.14|14.32|||372.37|-1010.31| +2452276|65000|567|30787|6545|1150|17308|7|162|116|74|47.43|75.88|31.11|0.00|2302.14|3509.82|5615.12|23.02|0.00|2302.14|2325.16|-1207.68| +2452276|65000|13455|30787|6545|1150|17308|7|216|116|3|20.75|38.18|22.90|43.96|68.70|62.25|114.54|2.22|43.96|24.74|26.96|-37.51| +2452276|65000|11340|30787|6545|1150|17308|7|198|116|98|89.18|89.18|69.56|0.00|6816.88|8739.64|8739.64|0.00|0.00|6816.88|6816.88|-1922.76| +2451452|39067|11588|57808|67373|6891|32456|7|188|117|93|91.95|115.85|33.59|0.00|3123.87|8551.35|10774.05|0.00|0.00|3123.87|3123.87|-5427.48| +2451452|39067|15170|57808|67373|6891|32456|7|251|117|41|52.60|104.14|46.86|0.00|1921.26|2156.60|4269.74|38.42|0.00|1921.26|1959.68|-235.34| +2451452|39067|6832|57808|67373|6891|32456|7|142|117|84|17.52|33.81|2.70|0.00|226.80|1471.68|2840.04|13.60|0.00|226.80|240.40|-1244.88| +2451452|39067|13159|57808|67373|6891|32456|7|43|117|42|44.10|63.94|28.77|0.00|1208.34|1852.20|2685.48|84.58|0.00|1208.34|1292.92|-643.86| +2451452|39067|1214|57808|67373|6891|32456|7|53|117|83|25.29|47.29|39.25|0.00|3257.75|2099.07|3925.07|228.04|0.00|3257.75|3485.79|1158.68| +2451452|39067|14948|57808|67373|6891|32456|7|296|117|40|97.47|140.35|54.73|0.00|2189.20|3898.80|5614.00|87.56|0.00|2189.20|2276.76|-1709.60| +2451452|39067|17239|57808|67373|6891|32456|7|214|117|6|28.83|50.16|32.10|0.00|192.60|172.98|300.96|15.40|0.00|192.60|208.00|19.62| +2451452|39067|3748|57808|67373|6891|32456|7|298|117|7|48.05|95.13|69.44|0.00|486.08|336.35|665.91|14.58|0.00|486.08|500.66|149.73| +2451452|39067|11593|57808|67373|6891|32456|7|130|117|98|51.17|88.01|58.96|0.00|5778.08|5014.66|8624.98|346.68|0.00|5778.08|6124.76|763.42| +2451452|39067|1339|57808|67373|6891|32456|7|202|117|31|43.40|55.55|25.55|570.27|792.05|1345.40|1722.05|11.08|570.27|221.78|232.86|-1123.62| +2451452|39067|14923|57808|67373|6891|32456|7|6|117|9|65.05|126.84|2.53|0.00|22.77|585.45|1141.56|1.59|0.00|22.77|24.36|-562.68| +2451452|39067|12205|57808|67373|6891|32456|7|100|117|25|72.73|86.54|41.53|0.00|1038.25|1818.25|2163.50|31.14|0.00|1038.25|1069.39|-780.00| +2451452|39067|12170|57808|67373|6891|32456|7|36|117|11|51.79|55.93|24.60|0.00|270.60|569.69|615.23|8.11|0.00|270.60|278.71|-299.09| +2451452|39067|5137|57808|67373|6891|32456|7|242|117|21|15.94|24.70|9.88|190.88|207.48|334.74|518.70|1.49|190.88|16.60|18.09|-318.14| +2451452|39067|3001|57808|67373|6891|32456|7|122|117|94|16.08|24.92|15.45|0.00|1452.30|1511.52|2342.48|0.00|0.00|1452.30|1452.30|-59.22| +2451806|46450|7|20684|992018|2999|12780|4|233|118|27|73.96|79.13|1.58|0.00|42.66|1996.92|2136.51|1.70|0.00|42.66|44.36|-1954.26| +2451806|46450|12632|20684|992018|2999|12780|4|43|118|25|40.81|64.88|43.46|0.00|1086.50|1020.25|1622.00|10.86|0.00|1086.50|1097.36|66.25| +2451806|46450|11945||992018|2999||4||118|||||0.00|2547.96|4018.29||76.43|0.00|||-1470.33| +2451806|46450|16568|20684|992018|2999|12780|4|264|118|24|11.95|16.84|9.26|0.00|222.24|286.80|404.16|11.11|0.00|222.24|233.35|-64.56| +2451806|46450|5078|20684|992018|2999|12780|4|125|118|11|81.23|141.34|117.31|0.00|1290.41|893.53|1554.74|0.00|0.00|1290.41|1290.41|396.88| +2451806|46450|11369|20684|992018|2999|12780|4|12|118|92|79.82|132.50|3.97|0.00|365.24|7343.44|12190.00|10.95|0.00|365.24|376.19|-6978.20| +2451806|46450|13339|20684|992018|2999|12780|4|101|118|37|38.67|47.17|46.22|1556.22|1710.14|1430.79|1745.29|6.15|1556.22|153.92|160.07|-1276.87| +2451806|46450|9319|20684|992018|2999|12780|4|244|118|5|28.03|49.89|5.48|0.00|27.40|140.15|249.45|2.19|0.00|27.40|29.59|-112.75| +2451806|46450|5216|20684|992018|2999|12780|4|199|118|19|58.21|96.62|86.95|710.38|1652.05|1105.99|1835.78|65.91|710.38|941.67|1007.58|-164.32| +2451806|46450|4487|20684|992018|2999|12780|4|168|118|16|44.93|83.12|68.15|0.00|1090.40|718.88|1329.92|87.23|0.00|1090.40|1177.63|371.52| +2451806|46450|15793|20684|992018|2999|12780|4|173|118|88|5.30|8.10|3.96|0.00|348.48|466.40|712.80|31.36|0.00|348.48|379.84|-117.92| +2451806|46450|5657|20684|992018|2999|12780|4|118|118|27|6.49|7.07|0.63|9.18|17.01|175.23|190.89|0.31|9.18|7.83|8.14|-167.40| +2451806|46450|8918|20684|992018|2999|12780|4|37|118|1|17.91|33.84|33.50|0.00|33.50|17.91|33.84|2.68|0.00|33.50|36.18|15.59| +2451806|46450|7166|20684|992018|2999|12780|4|219|118|90|86.94|149.53|19.43|0.00|1748.70|7824.60|13457.70|87.43|0.00|1748.70|1836.13|-6075.90| +2451806|46450|11107|20684|992018|2999|12780|4|291|118|90|17.68|20.33|13.62|0.00|1225.80|1591.20|1829.70|61.29|0.00|1225.80|1287.09|-365.40| +2451806|46450|7663|20684|992018|2999|12780|4|51|118|35|80.21|91.43|0.00|0.00|0.00|2807.35|3200.05|0.00|0.00|0.00|0.00|-2807.35| +2451502|44389|6529|34199|276959|3719|38879|1|89|119|87|11.94|23.16|4.63|0.00|402.81|1038.78|2014.92|28.19|0.00|402.81|431.00|-635.97| +2451502|44389|15632|34199|276959|3719|38879|1|225|119|76|68.20|84.56|6.76|0.00|513.76|5183.20|6426.56|41.10|0.00|513.76|554.86|-4669.44| +2451502|44389|15550|34199|276959|3719|38879|1|205|119|3|57.72|88.31|69.76|2.09|209.28|173.16|264.93|8.28|2.09|207.19|215.47|34.03| +2451502|44389|9937|34199|276959|3719|38879|1|293|119|11|16.06|16.70|9.85|0.00|108.35|176.66|183.70|6.50|0.00|108.35|114.85|-68.31| +2451502|44389|4897|34199|276959|3719|38879|1|285|119|85|51.29|98.47|40.37|0.00|3431.45|4359.65|8369.95|240.20|0.00|3431.45|3671.65|-928.20| +2451502|44389|4135|34199|276959|3719|38879|1|249|119|84|19.84|29.76|11.01|0.00|924.84|1666.56|2499.84|9.24|0.00|924.84|934.08|-741.72| +2451502|44389|17186|34199|276959|3719|38879|1|197|119|43|88.13|139.24|82.15|2331.41|3532.45|3789.59|5987.32|96.08|2331.41|1201.04|1297.12|-2588.55| +2451502|44389|15553|34199|276959|3719|38879|1|154|119|67|88.82|131.45|80.18|0.00|5372.06|5950.94|8807.15|161.16|0.00|5372.06|5533.22|-578.88| +2451502|44389|12368|34199|276959|3719|38879|1|259|119|18|82.99|122.82|82.28|0.00|1481.04|1493.82|2210.76|14.81|0.00|1481.04|1495.85|-12.78| +2451502|44389|14953|34199|276959|3719|38879|1|138|119|87|95.01|111.16|20.00|0.00|1740.00|8265.87|9670.92|34.80|0.00|1740.00|1774.80|-6525.87| +2452222|34466|809|40120|1379144|5015|32486|8|271|120|8|81.51|140.19|137.38|0.00|1099.04|652.08|1121.52|21.98|0.00|1099.04|1121.02|446.96| +2452222|34466|5159|40120|1379144|5015|32486|8|67|120|87|49.78|71.18|22.06|0.00|1919.22|4330.86|6192.66|38.38|0.00|1919.22|1957.60|-2411.64| +2452222|34466|6555|40120|1379144|5015|32486|8|212|120|44|33.50|58.29|8.16|0.00|359.04|1474.00|2564.76|10.77|0.00|359.04|369.81|-1114.96| +2452222|34466|827|40120|1379144|5015|32486|8|279|120|48|85.16|125.18|78.86|0.00|3785.28|4087.68|6008.64|37.85|0.00|3785.28|3823.13|-302.40| +2452222|34466|16327|40120|||||144|120||86.19||11.79|243.46||3016.65|3438.75|8.45|243.46|||| +2452222|34466|14891|40120|1379144|5015|32486|8|188|120|92|65.05|115.78|43.99|0.00|4047.08|5984.60|10651.76|364.23|0.00|4047.08|4411.31|-1937.52| +2452222|34466|14749|40120|1379144|5015|32486|8|178|120|87|90.07|152.21|111.11|0.00|9666.57|7836.09|13242.27|386.66|0.00|9666.57|10053.23|1830.48| +2452222|34466|6093|40120|1379144|5015|32486|8|258|120|9|16.17|31.36|13.48|0.00|121.32|145.53|282.24|6.06|0.00|121.32|127.38|-24.21| +2452222|34466|4999|40120|1379144|5015|32486|8|75|120|8|6.05|8.16|5.63|0.00|45.04|48.40|65.28|1.80|0.00|45.04|46.84|-3.36| +2451149|42172|319|13005|78113|21|2729|8|221|121|93|19.82|31.31|16.28|0.00|1514.04|1843.26|2911.83|45.42|0.00|1514.04|1559.46|-329.22| +2451149|42172|11218|13005|78113|21|2729|8|267|121|31|68.21|68.21|0.00|0.00|0.00|2114.51|2114.51|0.00|0.00|0.00|0.00|-2114.51| +2451149|42172|8413|13005|78113|21|2729|8|170|121|2|63.47|110.43|97.17|0.00|194.34|126.94|220.86|17.49|0.00|194.34|211.83|67.40| +2451149|42172|9097||78113|||||121|||156.34|92.24||||1407.06|74.71|||904.87|| +2451149|42172|11959|13005|78113|21|2729|8|140|121|99|6.94|7.77|1.16|84.98|114.84|687.06|769.23|0.00|84.98|29.86|29.86|-657.20| +2451149|42172|3076|13005|78113|21|2729|8|243|121|78|36.29|52.25|3.13|61.03|244.14|2830.62|4075.50|5.49|61.03|183.11|188.60|-2647.51| +2451149|42172|7|13005|78113|21|2729|8|248|121|14|99.59|127.47|80.30|0.00|1124.20|1394.26|1784.58|22.48|0.00|1124.20|1146.68|-270.06| +2451149|42172|12632|13005|||2729||197|121||85.26|121.92|75.59|0.00|||||0.00|||| +2451149|42172|11944||||2729|||121|72|19.12|21.79|14.59|||||||||| +2451149|42172|16568|13005|78113|21|2729|8|160|121|49|95.05|189.14|54.85|0.00|2687.65|4657.45|9267.86|0.00|0.00|2687.65|2687.65|-1969.80| +2451149|42172|5078|13005|78113|21|2729|8|232|121|52|81.53|118.21|0.00|0.00|0.00|4239.56|6146.92|0.00|0.00|0.00|0.00|-4239.56| +2451149|42172|11368|13005|78113|21|2729|8|32|121|34|34.54|52.50|21.00|0.00|714.00|1174.36|1785.00|64.26|0.00|714.00|778.26|-460.36| +2451149|42172|13339|13005|78113|21|2729|8|255|121|31|71.18|86.83|65.12|0.00|2018.72|2206.58|2691.73|0.00|0.00|2018.72|2018.72|-187.86| +2451149|42172|9319|13005|78113|21|2729|8|217|121|60|4.06|7.42|0.51|0.00|30.60|243.60|445.20|0.30|0.00|30.60|30.90|-213.00| +2451222|65309|3854|24145|841785|4578|5418|10|243|122|1|38.86|59.45|42.80|0.00|42.80|38.86|59.45|0.85|0.00|42.80|43.65|3.94| +2451222|65309|46|24145|841785|4578|5418|10|251|122|65|25.62|44.83|29.13|0.00|1893.45|1665.30|2913.95|113.60|0.00|1893.45|2007.05|228.15| +2451222||6709||841785|||||122||31.79||||||1643.12||||632.01|| +2451222|65309|9008|24145|841785|4578|5418|10|237|122|69|21.53|40.69|22.78|1241.73|1571.82|1485.57|2807.61|29.70|1241.73|330.09|359.79|-1155.48| +2451222|65309|424|24145|841785|4578|5418|10|95|122|27|90.09|119.81|62.30|0.00|1682.10|2432.43|3234.87|16.82|0.00|1682.10|1698.92|-750.33| +2451222|65309|13418|24145|841785|4578|5418|10|71|122|79|2.81|5.36|0.80|0.00|63.20|221.99|423.44|0.63|0.00|63.20|63.83|-158.79| +2451222|65309|15952|24145|841785|4578|5418|10|54|122|86|63.80|121.85|113.32|0.00|9745.52|5486.80|10479.10|682.18|0.00|9745.52|10427.70|4258.72| +2451222|65309|16742|24145|841785|4578|5418|10|228|122|45|28.05|32.53|28.30|0.00|1273.50|1262.25|1463.85|101.88|0.00|1273.50|1375.38|11.25| +|65309|980||841785||||36|122||72.12||116.16|0.00|6737.28||7403.70||0.00|6737.28|7343.63|2554.32| +2451222|65309|3244|24145|841785|4578|5418|10|14|122|20|28.52|53.90|52.82|0.00|1056.40|570.40|1078.00|10.56|0.00|1056.40|1066.96|486.00| +2451222|65309|1606|24145|841785|4578|5418|10|230|122|85|60.16|118.51|117.32|8177.20|9972.20|5113.60|10073.35|89.75|8177.20|1795.00|1884.75|-3318.60| +2451222|65309|8824|24145|841785|4578|5418|10|233|122|4|46.97|80.78|25.04|0.00|100.16|187.88|323.12|9.01|0.00|100.16|109.17|-87.72| +2451222|65309|13436|24145|841785|4578|5418|10|213|122|52|88.05|92.45|89.67|2657.81|4662.84|4578.60|4807.40|160.40|2657.81|2005.03|2165.43|-2573.57| +2451222|65309|9416|24145|841785|4578|5418|10|254|122|84|39.54|62.86|46.51|0.00|3906.84|3321.36|5280.24|312.54|0.00|3906.84|4219.38|585.48| +2451222|65309|1982|24145|841785|4578|5418|10|192|122|80|86.54|149.71|5.98|0.00|478.40|6923.20|11976.80|38.27|0.00|478.40|516.67|-6444.80| +2451511||6904|||1559||4|203|123|95|||22.33|||6800.10|8159.55|3.18||||-6481.89| +2451511|33493|356|82355|1341157|1559|24361|4|296|123|20|71.37|119.18|16.68|0.00|333.60|1427.40|2383.60|3.33|0.00|333.60|336.93|-1093.80| +2451511|33493|2059|82355|1341157|1559|24361|4|169|123|96|66.89|95.65|95.65|0.00|9182.40|6421.44|9182.40|183.64|0.00|9182.40|9366.04|2760.96| +2451511|33493|13676|82355|1341157|1559|24361|4|46|123|9|72.41|88.34|3.53|0.00|31.77|651.69|795.06|0.63|0.00|31.77|32.40|-619.92| +2451511|33493|17590|82355|1341157|1559|24361|4|227|123|62|25.78|47.17|28.77|0.00|1783.74|1598.36|2924.54|124.86|0.00|1783.74|1908.60|185.38| +2451511|33493|14288|82355|1341157|1559|24361|4|268|123|1|11.67|22.28|19.16|0.00|19.16|11.67|22.28|1.14|0.00|19.16|20.30|7.49| +2451511|33493|7756|82355|1341157|1559|24361|4|232|123|40|12.46|13.83|1.10|0.00|44.00|498.40|553.20|3.52|0.00|44.00|47.52|-454.40| +2451511|33493|8240|82355|1341157|1559|24361|4|191|123|70|91.43|107.88|18.33|0.00|1283.10|6400.10|7551.60|89.81|0.00|1283.10|1372.91|-5117.00| +2451511|33493|10270|82355|1341157|1559|24361|4|272|123|81|64.22|100.82|65.53|0.00|5307.93|5201.82|8166.42|371.55|0.00|5307.93|5679.48|106.11| +2451511|33493|11264|82355|1341157|1559|24361|4|292|123|81|47.24|88.81|25.75|0.00|2085.75|3826.44|7193.61|0.00|0.00|2085.75|2085.75|-1740.69| +2451511|33493|4090|82355|1341157|1559|24361|4|181|123|60|6.97|13.94|2.23|0.00|133.80|418.20|836.40|9.36|0.00|133.80|143.16|-284.40| +2451412|58152|14786|91908|1183473|6591|21309|8|212|124|14|51.01|61.72|26.53|0.00|371.42|714.14|864.08|11.14|0.00|371.42|382.56|-342.72| +2451412|58152|5953|91908|1183473|6591|21309|8|194|124|93|11.19|12.53|1.75|0.00|162.75|1040.67|1165.29|9.76|0.00|162.75|172.51|-877.92| +||9085|91908||6591|21309|||124|4|90.29|||||361.16|483.92|3.48||58.04|61.52|-303.12| +2451412|58152|4796|91908|1183473|6591|21309|8|15|124|31|5.64|6.37|0.50|8.68|15.50|174.84|197.47|0.06|8.68|6.82|6.88|-168.02| +2451412|58152|1249|91908|1183473|6591|21309|8|291|124|99|16.30|30.64|8.88|0.00|879.12|1613.70|3033.36|61.53|0.00|879.12|940.65|-734.58| +2451412|58152|5560|91908|1183473|6591|21309|8|22|124|5|15.90|19.71|17.14|36.85|85.70|79.50|98.55|0.97|36.85|48.85|49.82|-30.65| +2451412|58152|844|91908|1183473|6591|21309|8|35|124|92|6.27|8.08|6.22|0.00|572.24|576.84|743.36|0.00|0.00|572.24|572.24|-4.60| +2451412|58152|12883|91908|1183473|6591|21309|8|290|124|77|65.95|131.24|86.61|5068.41|6668.97|5078.15|10105.48|80.02|5068.41|1600.56|1680.58|-3477.59| +2451412|58152|3998|91908|1183473|6591|21309|8|137|124|49|1.22|1.73|0.25|10.29|12.25|59.78|84.77|0.01|10.29|1.96|1.97|-57.82| +2451875|73791|9965|92596|914934|2519|15257|8|146|125|50|95.14|166.49|89.90|0.00|4495.00|4757.00|8324.50|44.95|0.00|4495.00|4539.95|-262.00| +2451875|73791|6749|92596|914934|2519|15257|8|192|125|2|78.73|152.73|132.87|0.00|265.74|157.46|305.46|10.62|0.00|265.74|276.36|108.28| +2451875|73791|15385|92596|914934|2519|15257|8|76|125|2|73.52|80.13|71.31|0.00|142.62|147.04|160.26|5.70|0.00|142.62|148.32|-4.42| +2451875|73791|235|92596|914934|2519|15257|8|99|125|94|28.34|36.55|34.35|0.00|3228.90|2663.96|3435.70|161.44|0.00|3228.90|3390.34|564.94| +2451875|73791|397|92596|914934|2519|15257|8|216|125|7|60.61|73.94|48.06|0.00|336.42|424.27|517.58|20.18|0.00|336.42|356.60|-87.85| +2451875|73791|7099|92596|914934|2519|15257|8|286|125|21|57.14|59.42|21.39|0.00|449.19|1199.94|1247.82|35.93|0.00|449.19|485.12|-750.75| +2451875|73791|8945|92596|914934|2519|15257|8|112|125|34|7.50|13.35|11.48|0.00|390.32|255.00|453.90|19.51|0.00|390.32|409.83|135.32| +2451875|73791|10568|92596|914934|2519|15257|8|108|125|20|62.36|80.44|72.39|0.00|1447.80|1247.20|1608.80|86.86|0.00|1447.80|1534.66|200.60| +2451875|73791|17528|92596|914934|2519|15257|8|191|125|76|38.42|62.24|22.40|0.00|1702.40|2919.92|4730.24|85.12|0.00|1702.40|1787.52|-1217.52| +2451875|73791|4847|92596|914934|2519|15257|8|89|125|2|20.52|31.60|10.11|0.00|20.22|41.04|63.20|1.81|0.00|20.22|22.03|-20.82| +2451875|73791|491|92596|914934|2519|15257|8|249|125|7|72.72|141.07|93.10|0.00|651.70|509.04|987.49|26.06|0.00|651.70|677.76|142.66| +2451875|73791|8015|92596|914934|2519|15257|8|48|125|53|3.90|7.68|4.53|0.00|240.09|206.70|407.04|7.20|0.00|240.09|247.29|33.39| +2451875|73791|13952|92596|914934|2519|15257|8|283|125|72|81.18|125.82|76.75|0.00|5526.00|5844.96|9059.04|165.78|0.00|5526.00|5691.78|-318.96| +2451875|73791|6619|92596|914934|2519|15257|8|126|125|97|99.13|189.33|5.67|0.00|549.99|9615.61|18365.01|38.49|0.00|549.99|588.48|-9065.62| +2451875|73791|10778|92596|914934|2519|15257|8|124|125|85|8.18|11.94|6.56|0.00|557.60|695.30|1014.90|11.15|0.00|557.60|568.75|-137.70| +2452554|48466|11190|84165|152693|4421|23349|1|158|126|97|40.17|67.88|56.34|0.00|5464.98|3896.49|6584.36|218.59|0.00|5464.98|5683.57|1568.49| +2452554|48466|9297|84165|152693|4421|23349|1|197|126|28|20.86|31.29|25.97|29.08|727.16|584.08|876.12|34.90|29.08|698.08|732.98|114.00| +2452554|48466|15679|84165|152693|4421|23349|1|198|126|76|14.99|24.88|9.45|0.00|718.20|1139.24|1890.88|14.36|0.00|718.20|732.56|-421.04| +2452554|48466|17337|84165|152693|4421|23349|1|28|126|24|1.91|3.22|3.12|35.94|74.88|45.84|77.28|0.00|35.94|38.94|38.94|-6.90| +2452554|48466|16675|84165|152693|4421|23349|1|288|126|2|97.08|132.99|73.14|0.00|146.28|194.16|265.98|10.23|0.00|146.28|156.51|-47.88| +2452554|48466|16003|84165|152693|4421|23349|1|234|126|38|11.77|14.00|10.22|120.39|388.36|447.26|532.00|24.11|120.39|267.97|292.08|-179.29| +2452554|48466|13641|84165|152693|4421|23349|1|182|126|60|84.95|111.28|74.55|0.00|4473.00|5097.00|6676.80|357.84|0.00|4473.00|4830.84|-624.00| +2452554|48466|2190|84165|152693|4421|23349|1|262|126|27|48.11|63.50|16.51|374.44|445.77|1298.97|1714.50|0.71|374.44|71.33|72.04|-1227.64| +2452554|48466|15066|84165|152693|4421|23349|1|290|126|76|60.00|97.80|20.53|0.00|1560.28|4560.00|7432.80|0.00|0.00|1560.28|1560.28|-2999.72| +2452554|48466|8589|84165|152693|4421|23349|1|63|126|46|54.00|99.90|99.90|0.00|4595.40|2484.00|4595.40|91.90|0.00|4595.40|4687.30|2111.40| +2452554|48466|16321|84165|152693|4421|23349|1|12|126|2|25.77|50.76|37.05|43.71|74.10|51.54|101.52|0.91|43.71|30.39|31.30|-21.15| +2452554|48466|8490|84165|152693|4421|23349|1|8|126|78|10.00|14.80|7.69|0.00|599.82|780.00|1154.40|17.99|0.00|599.82|617.81|-180.18| +2452554|48466|2503|84165|152693|4421|23349|1|105|126|32|1.52|1.55|0.17|3.86|5.44|48.64|49.60|0.12|3.86|1.58|1.70|-47.06| +2451595|64939|3241|66548|1721030|5912|26794|1|25|127|47|83.16|98.12|40.22|0.00|1890.34|3908.52|4611.64|94.51|0.00|1890.34|1984.85|-2018.18| +2451595|64939|8|66548|1721030|5912|26794|1|207|127|83|35.36|39.60|2.77|0.00|229.91|2934.88|3286.80|6.89|0.00|229.91|236.80|-2704.97| +2451595|64939|2030|66548|1721030|5912|26794|1|192|127|62|12.10|21.17|15.45|0.00|957.90|750.20|1312.54|57.47|0.00|957.90|1015.37|207.70| +2451595|64939|12212|66548|1721030|5912|26794|1|158|127|50|44.61|54.42|32.10|577.80|1605.00|2230.50|2721.00|51.36|577.80|1027.20|1078.56|-1203.30| +2451595|64939|7645|66548|1721030|5912|26794|1|193|127|100|17.66|30.02|22.21|0.00|2221.00|1766.00|3002.00|0.00|0.00|2221.00|2221.00|455.00| +2451595|64939|6428|66548|1721030|5912|26794|1|24|127|91|45.55|46.91|11.25|849.71|1023.75|4145.05|4268.81|13.92|849.71|174.04|187.96|-3971.01| +2451595||953||||26794||96|127|8|46.53|59.55|54.19||||476.40|0.00||433.52||61.28| +2451595|64939|10897|66548|1721030|5912|26794|1|26|127|10|82.14|151.13|136.01|0.00|1360.10|821.40|1511.30|122.40|0.00|1360.10|1482.50|538.70| +2451595|64939|8594|66548|1721030|5912|26794|1|246|127|10|85.32|133.09|129.09|0.00|1290.90|853.20|1330.90|77.45|0.00|1290.90|1368.35|437.70| +|67115|12671|||4364|25820||265|128|45|66.29|74.24||||2983.05|3340.80|0.00||1005.43||| +2452048|67115|3507|77005|1543261|4364|25820|7|169|128|29|45.92|60.61|35.75|0.00|1036.75|1331.68|1757.69|82.94|0.00|1036.75|1119.69|-294.93| +2452048|67115|6753|77005|1543261|4364|25820|7|253|128|41|65.88|103.43|102.39|0.00|4197.99|2701.08|4240.63|125.93|0.00|4197.99|4323.92|1496.91| +2452048|67115|5431|77005|1543261|4364|25820|7|201|128|35|77.20|132.78|102.24|0.00|3578.40|2702.00|4647.30|214.70|0.00|3578.40|3793.10|876.40| +2452048|67115|7871|77005|1543261|4364|25820|7|247|128|99|98.63|103.56|80.77|0.00|7996.23|9764.37|10252.44|479.77|0.00|7996.23|8476.00|-1768.14| +2452048|67115|13579|77005|1543261|4364|25820|7|20|128|77|79.57|129.69|92.07|0.00|7089.39|6126.89|9986.13|496.25|0.00|7089.39|7585.64|962.50| +2452048|67115|3929|77005|1543261|4364|25820|7|67|128|36|74.50|97.59|62.45|2068.34|2248.20|2682.00|3513.24|1.79|2068.34|179.86|181.65|-2502.14| +||1951|77005|1543261||||285|128|14|60.93||68.53|0.00|959.42||1296.54|28.78|0.00|959.42||| +2452048|67115|17349|77005|1543261|4364|25820|7|64|128|47|35.00|49.00|21.07|0.00|990.29|1645.00|2303.00|89.12|0.00|990.29|1079.41|-654.71| +2452048|67115|217|77005|1543261|4364|25820|7|30|128|6|96.56|188.29|122.38|0.00|734.28|579.36|1129.74|58.74|0.00|734.28|793.02|154.92| +2452048|67115|14849|77005|1543261|4364|25820|7|189|128|45|29.49|44.52|28.93|0.00|1301.85|1327.05|2003.40|91.12|0.00|1301.85|1392.97|-25.20| +2452048||17875|||4364|25820||217|128|36|90.59||99.69||3588.84|3261.24|5054.76|||||327.60| +2452048|67115|3239|77005|1543261|4364|25820|7|91|128|79|96.24|163.60|80.16|0.00|6332.64|7602.96|12924.40|189.97|0.00|6332.64|6522.61|-1270.32| +2452048|67115|43|77005|1543261|4364|25820|7|29|128|91|82.53|106.46|8.51|0.00|774.41|7510.23|9687.86|7.74|0.00|774.41|782.15|-6735.82| +2451527|35753|5257|548|84373|3045|17317|7|281|129|23|71.70|72.41|18.82|0.00|432.86|1649.10|1665.43|12.98|0.00|432.86|445.84|-1216.24| +2451527|35753|13597|548|84373|3045|17317|7|35|129|57|43.62|82.87|63.80|0.00|3636.60|2486.34|4723.59|181.83|0.00|3636.60|3818.43|1150.26| +2451527|35753|6913|548|84373|3045|17317|7|154|129|28|91.59|117.23|87.92|0.00|2461.76|2564.52|3282.44|0.00|0.00|2461.76|2461.76|-102.76| +2451527|35753|1489|548|84373|3045|17317|7|141|129|23|54.82|91.54|37.53|0.00|863.19|1260.86|2105.42|25.89|0.00|863.19|889.08|-397.67| +2451527|35753|11455|548|84373|3045|17317|7|212|129|77|37.14|43.45|18.24|0.00|1404.48|2859.78|3345.65|14.04|0.00|1404.48|1418.52|-1455.30| +2451527|35753|10526|548|84373|3045|17317|7|115|129|64|5.24|5.29|3.96|0.00|253.44|335.36|338.56|10.13|0.00|253.44|263.57|-81.92| +2451527|35753|9379|548|84373|3045|17317|7|38|129|53|45.92|76.22|73.17|3179.96|3878.01|2433.76|4039.66|13.96|3179.96|698.05|712.01|-1735.71| +2451527|35753|8374|548|84373|3045|17317|7|122|129|90|74.84|92.80|45.47|0.00|4092.30|6735.60|8352.00|0.00|0.00|4092.30|4092.30|-2643.30| +2451527|35753|16576|548|84373|3045|17317|7|287|129|4|7.82|8.91|6.05|18.63|24.20|31.28|35.64|0.11|18.63|5.57|5.68|-25.71| +2451527|35753|3721|548|84373|3045|17317|7|103|129|56|82.12|149.45|85.18|0.00|4770.08|4598.72|8369.20|190.80|0.00|4770.08|4960.88|171.36| +2451527|35753|12872||84373|3045|17317|||129||10.28|14.49||||||||||-81.46| +2451527|35753|16609|548|84373|3045|17317|7|174|129|9|87.50|135.62|2.71|0.00|24.39|787.50|1220.58|0.48|0.00|24.39|24.87|-763.11| +2451076|58230|17674|87622|1290458|4141|31777|10|112|130|85|76.14|110.40|82.80|985.32|7038.00|6471.90|9384.00|423.68|985.32|6052.68|6476.36|-419.22| +2451076|58230|16915|87622|1290458|4141|31777|10|183|130|64|51.47|88.01|67.76|3079.01|4336.64|3294.08|5632.64|0.00|3079.01|1257.63|1257.63|-2036.45| +2451076|58230|5462|87622|1290458|4141|31777|10|94|130|47|47.16|57.53|52.92|0.00|2487.24|2216.52|2703.91|149.23|0.00|2487.24|2636.47|270.72| +2451076|58230|10945|87622|1290458|4141|31777|10|101|130|39|68.53|111.01|97.68|0.00|3809.52|2672.67|4329.39|304.76|0.00|3809.52|4114.28|1136.85| +2451076|58230|17684|87622|1290458|4141|31777|10|128|130|78|49.00|62.23|42.93|0.00|3348.54|3822.00|4853.94|200.91|0.00|3348.54|3549.45|-473.46| +2451076|58230|2638|87622|1290458|4141|31777|10|137|130|35|72.47|104.35|34.43|0.00|1205.05|2536.45|3652.25|24.10|0.00|1205.05|1229.15|-1331.40| +2451076|58230|9920|87622|1290458|4141|31777|10|41|130|26|24.80|36.70|13.57|0.00|352.82|644.80|954.20|0.00|0.00|352.82|352.82|-291.98| +2451076|58230|596|87622|1290458|4141|31777|10|168|130|91|38.82|43.47|23.47|0.00|2135.77|3532.62|3955.77|170.86|0.00|2135.77|2306.63|-1396.85| +2451076|58230|1255|87622|1290458|4141|31777|10|43|130|73|87.74|110.55|33.16|0.00|2420.68|6405.02|8070.15|0.00|0.00|2420.68|2420.68|-3984.34| +2451076|58230|8446|87622|1290458|4141|31777|10|190|130|81|29.11|36.67|6.23|247.26|504.63|2357.91|2970.27|2.57|247.26|257.37|259.94|-2100.54| +|58230|8060|87622|1290458|4141|31777|10|97|130||||9.02|0.00||2040.22|2958.26||0.00||532.18|-1508.04| +2451076|58230|14750|87622|1290458|4141|31777|10|165|130|8|10.78|14.44|12.85|0.00|102.80|86.24|115.52|3.08|0.00|102.80|105.88|16.56| +2451076|58230|14947|87622|1290458|4141|31777|10|163|130|50|49.36|77.00|75.46|0.00|3773.00|2468.00|3850.00|339.57|0.00|3773.00|4112.57|1305.00| +2451076|58230|9910|87622|1290458|4141|31777|10|28|130|63|26.30|47.07|26.82|0.00|1689.66|1656.90|2965.41|50.68|0.00|1689.66|1740.34|32.76| +|58230|5612|87622|||31777||183|130|21|59.11|109.94|43.97|424.75|||2308.74||424.75|498.62|518.56|| +2451032|34299|7208|61469|188338|5607|9229|10|37|131|83|33.46|37.14|4.08|60.95|338.64|2777.18|3082.62|19.43|60.95|277.69|297.12|-2499.49| +2451032|34299|11938|61469|188338|5607|9229|10|157|131|74|18.32|24.36|16.32|0.00|1207.68|1355.68|1802.64|60.38|0.00|1207.68|1268.06|-148.00| +2451032|34299|4442|61469|188338|5607|9229|10|69|131|4|25.80|48.76|14.14|0.00|56.56|103.20|195.04|4.52|0.00|56.56|61.08|-46.64| +2451032|34299|10774|||5607||10||131|9||28.74|14.37|||||7.75|||137.08|-24.66| +2451032|34299|6718|61469|188338|5607|9229|10|181|131|63|69.42|127.73|42.15|0.00|2655.45|4373.46|8046.99|53.10|0.00|2655.45|2708.55|-1718.01| +2451032|34299|15422|61469|188338|5607|9229|10|102|131|77|32.38|39.50|24.09|0.00|1854.93|2493.26|3041.50|129.84|0.00|1854.93|1984.77|-638.33| +2451032|34299|12248|61469|188338|5607|9229|10|233|131|45|43.99|51.90|2.07|0.00|93.15|1979.55|2335.50|1.86|0.00|93.15|95.01|-1886.40| +||9619|61469||5607|9229||230|131|37||91.61||||2456.43||||||-2151.55| +2451032|34299|6052|61469|188338|5607|9229|10|253|131|57|66.89|66.89|28.09|0.00|1601.13|3812.73|3812.73|48.03|0.00|1601.13|1649.16|-2211.60| +2451032|34299|1934|61469|188338|5607|9229|10|279|131|32|62.03|100.48|14.06|0.00|449.92|1984.96|3215.36|0.00|0.00|449.92|449.92|-1535.04| +2451032|34299|8095|61469|188338|5607|9229|10|226|131|95|97.35|163.54|3.27|0.00|310.65|9248.25|15536.30|6.21|0.00|310.65|316.86|-8937.60| +2451540|45655|388||935362||19917|7||132||63.07|||||1513.68|||||1111.14|-424.32| +2451540|45655|8311|33461|935362|1616|19917|7|154|132|34|10.21|10.72|1.50|0.00|51.00|347.14|364.48|1.53|0.00|51.00|52.53|-296.14| +2451540|45655|5083|33461|935362|1616|19917|7|76|132|56|71.00|122.83|1.22|0.00|68.32|3976.00|6878.48|1.36|0.00|68.32|69.68|-3907.68| +2451540|45655|3052|33461|935362|1616|19917|7|187|132|19|16.14|26.63|21.57|0.00|409.83|306.66|505.97|28.68|0.00|409.83|438.51|103.17| +2451540|45655|3256|33461|935362|1616|19917|7|229|132|71|89.56|172.85|138.28|0.00|9817.88|6358.76|12272.35|883.60|0.00|9817.88|10701.48|3459.12| +2451540|45655|6316|33461|935362|1616|19917|7|90|132|88|43.92|57.53|5.17|0.00|454.96|3864.96|5062.64|31.84|0.00|454.96|486.80|-3410.00| +2451540|45655|17581|33461|935362|1616|19917|7|227|132|68|62.77|75.95|66.07|4178.26|4492.76|4268.36|5164.60|6.29|4178.26|314.50|320.79|-3953.86| +2451540|45655|3434|33461|935362|1616|19917|7|268|132|83|45.35|79.81|19.95|16.55|1655.85|3764.05|6624.23|81.96|16.55|1639.30|1721.26|-2124.75| +2451540|45655|16759|33461|935362|1616|19917|7|242|132|53|60.45|66.49|59.17|0.00|3136.01|3203.85|3523.97|156.80|0.00|3136.01|3292.81|-67.84| +2451540|45655|15655|33461|935362|1616|19917|7|244|132|10|60.95|83.50|55.11|0.00|551.10|609.50|835.00|0.00|0.00|551.10|551.10|-58.40| +2451540|45655|11162|33461|935362|1616|19917|7|192|132|94|55.92|103.45|3.10|0.00|291.40|5256.48|9724.30|11.65|0.00|291.40|303.05|-4965.08| +2451540|45655|12472|33461|935362|1616|19917|7|46|132|67|72.70|141.03|77.56|0.00|5196.52|4870.90|9449.01|0.00|0.00|5196.52|5196.52|325.62| +2451781|58338|13343|60708|282177|3999|1068|2|94|133|25|44.25|45.13|6.76|0.00|169.00|1106.25|1128.25|10.14|0.00|169.00|179.14|-937.25| +2451781|58338|9365|60708|282177|3999|1068|2|270|133|21|97.22|139.02|73.68|0.00|1547.28|2041.62|2919.42|15.47|0.00|1547.28|1562.75|-494.34| +2451781|58338|12767|60708|282177|3999|1068|2|3|133|5|41.09|65.74|42.07|0.00|210.35|205.45|328.70|8.41|0.00|210.35|218.76|4.90| +2451781|58338|1691|60708|282177|3999|1068|2|127|133|44|45.69|55.28|45.32|0.00|1994.08|2010.36|2432.32|179.46|0.00|1994.08|2173.54|-16.28| +2451781|58338|14333|60708|282177|3999|1068|2|296|133|73|3.21|4.91|3.73|0.00|272.29|234.33|358.43|0.00|0.00|272.29|272.29|37.96| +2451781|58338|15373|60708|282177|3999|1068|2|154|133|35|50.52|58.60|31.05|0.00|1086.75|1768.20|2051.00|97.80|0.00|1086.75|1184.55|-681.45| +2451781|58338|8372|60708|282177|3999|1068|2|8|133|95|26.59|30.84|0.30|0.00|28.50|2526.05|2929.80|0.85|0.00|28.50|29.35|-2497.55| +2451781|58338|15805|60708|282177|3999|1068|2|123|133|44|38.11|72.02|16.56|0.00|728.64|1676.84|3168.88|65.57|0.00|728.64|794.21|-948.20| +2451781|58338|5507|60708|282177|3999|1068|2|134|133|5|96.32|163.74|58.94|0.00|294.70|481.60|818.70|0.00|0.00|294.70|294.70|-186.90| +2451781|58338|6907|60708|282177|3999|1068|2|168|133|10|89.17|107.00|5.35|18.19|53.50|891.70|1070.00|2.82|18.19|35.31|38.13|-856.39| +2451781|58338|6103|60708|282177|3999|1068|2|59|133|64|3.28|3.93|3.89|19.91|248.96|209.92|251.52|13.74|19.91|229.05|242.79|19.13| +2452230|35977|9639|58602|1359169|89|37809|1|250|134|4|73.88|110.82|65.38|86.30|261.52|295.52|443.28|0.00|86.30|175.22|175.22|-120.30| +2452230|35977|7717|58602|1359169|89|37809|1|289|134|46|5.61|5.83|0.40|0.00|18.40|258.06|268.18|0.92|0.00|18.40|19.32|-239.66| +2452230|35977|73|58602|1359169|89|37809|1|24|134|84|4.76|7.23|2.16|88.90|181.44|399.84|607.32|8.32|88.90|92.54|100.86|-307.30| +2452230|35977|8271|58602|1359169|89|37809|1|15|134|61|35.72|64.29|61.71|0.00|3764.31|2178.92|3921.69|37.64|0.00|3764.31|3801.95|1585.39| +2452230|35977|23|58602|1359169|89|37809|1|294|134|68|86.13|146.42|77.60|2057.95|5276.80|5856.84|9956.56|96.56|2057.95|3218.85|3315.41|-2637.99| +2452230|35977|17293|58602|1359169|89|37809|1|97|134|37|48.52|94.61|24.59|0.00|909.83|1795.24|3500.57|63.68|0.00|909.83|973.51|-885.41| +2452230|35977|17397|58602|1359169|89|37809|1|282|134|18|67.17|81.94|23.76|0.00|427.68|1209.06|1474.92|17.10|0.00|427.68|444.78|-781.38| +2452230|35977|347|58602|1359169|89|37809|1|152|134|24|28.09|28.37|12.19|0.00|292.56|674.16|680.88|0.00|0.00|292.56|292.56|-381.60| +2452230|35977|11311|58602|1359169|89|37809|1|212|134|10|50.86|62.55|47.53|0.00|475.30|508.60|625.50|19.01|0.00|475.30|494.31|-33.30| +2452230|35977|8571|58602|1359169|89|37809|1|30|134|4|77.89|91.13|71.99|0.00|287.96|311.56|364.52|17.27|0.00|287.96|305.23|-23.60| +2452230|35977|13105|58602|1359169|89|37809|1|233|134|39|46.98|51.20|23.55|0.00|918.45|1832.22|1996.80|36.73|0.00|918.45|955.18|-913.77| +2451788|45482|9440|43197|713915|5080|37398|1|44|135|12|1.29|2.08|1.10|0.00|13.20|15.48|24.96|0.13|0.00|13.20|13.33|-2.28| +2451788|45482|17537|43197|713915|5080|37398|1|198|135|21|52.31|55.44|37.14|561.55|779.94|1098.51|1164.24|13.10|561.55|218.39|231.49|-880.12| +2451788|45482|7445|43197|713915|5080|37398|1|175|135|96|31.28|43.79|17.95|0.00|1723.20|3002.88|4203.84|17.23|0.00|1723.20|1740.43|-1279.68| +2451788|45482|11573|43197|713915|5080|37398|1|165|135|91|96.62|158.45|131.51|0.00|11967.41|8792.42|14418.95|1077.06|0.00|11967.41|13044.47|3174.99| +|45482|1952||||37398||273|135||||||2000.00||2598.00|||2000.00|2160.00|| +2451788|45482|1970|43197|713915|5080|37398|1|240|135|72|27.00|42.66|29.00|0.00|2088.00|1944.00|3071.52|167.04|0.00|2088.00|2255.04|144.00| +2451788|45482|4022|43197|713915|5080|37398|1|268|135|19|50.08|73.11|2.19|0.00|41.61|951.52|1389.09|3.74|0.00|41.61|45.35|-909.91| +2451788|45482|9101|43197|||37398||127|135|4|55.10||64.10|0.00|256.40|220.40||10.25|0.00||266.65|| +2451788|45482|14036|43197|713915|5080|37398|1|43|135|90|90.32|120.12|52.85|0.00|4756.50|8128.80|10810.80|285.39|0.00|4756.50|5041.89|-3372.30| +2451788|45482|16322|43197||5080|37398||195|135|||46.38||||||||||905.28| +2451788|45482|14027|43197|713915|5080|37398|1|109|135|16|51.43|61.71|34.55|0.00|552.80|822.88|987.36|11.05|0.00|552.80|563.85|-270.08| +2451788|45482|667|43197|713915|5080|37398|1|210|135|75|76.57|76.57|52.83|0.00|3962.25|5742.75|5742.75|198.11|0.00|3962.25|4160.36|-1780.50| +2451788|45482|6320|43197|713915|5080|37398|1|98|135|85|14.66|29.32|8.79|0.00|747.15|1246.10|2492.20|7.47|0.00|747.15|754.62|-498.95| +2451788|45482|2294|43197|713915|5080|37398|1|293|135|21|12.86|20.44|15.33|0.00|321.93|270.06|429.24|25.75|0.00|321.93|347.68|51.87| +2451788||12095||713915||37398|1||135|7|33.21||16.36|0.00|||||0.00|114.52||| +2451788|45482|8873|43197|713915|5080|37398|1|52|135|45|62.71|107.23|39.67|0.00|1785.15|2821.95|4825.35|160.66|0.00|1785.15|1945.81|-1036.80| +2452520|35812|4411|17595|751524|3853|15429|2|35|136|40|5.50|9.07|8.61|0.00|344.40|220.00|362.80|17.22|0.00|344.40|361.62|124.40| +2452520|35812|9543|17595|751524|3853|15429|2|160|136|80|14.40|14.40|6.76|0.00|540.80|1152.00|1152.00|32.44|0.00|540.80|573.24|-611.20| +2452520|35812|6117|17595|751524|3853|15429|2|94|136|84|71.98|103.65|29.02|975.07|2437.68|6046.32|8706.60|102.38|975.07|1462.61|1564.99|-4583.71| +2452520|35812|11479|17595|751524|3853|15429|2|18|136|24|98.31|181.87|72.74|0.00|1745.76|2359.44|4364.88|87.28|0.00|1745.76|1833.04|-613.68| +2452520|35812|14311|17595|751524|3853|15429|2|260|136|30|14.26|26.66|16.26|0.00|487.80|427.80|799.80|43.90|0.00|487.80|531.70|60.00| +2452520|35812|8937|17595|751524|3853|15429|2|6|136|75|12.47|14.34|7.45|0.00|558.75|935.25|1075.50|11.17|0.00|558.75|569.92|-376.50| +2452520|35812|12870|17595|751524|3853|15429|2|196|136|86|50.72|89.26|65.15|0.00|5602.90|4361.92|7676.36|448.23|0.00|5602.90|6051.13|1240.98| +2452520|35812|17853|17595|751524|3853|15429|2|142|136|89|9.92|11.40|4.67|0.00|415.63|882.88|1014.60|20.78|0.00|415.63|436.41|-467.25| +2452520|35812|1975|17595|751524|3853|15429|2|205|136|4|80.27|134.85|111.92|0.00|447.68|321.08|539.40|8.95|0.00|447.68|456.63|126.60| +2452520|35812|4002|17595|751524|3853|15429|2|66|136|97|39.56|58.15|55.82|3140.43|5414.54|3837.32|5640.55|90.96|3140.43|2274.11|2365.07|-1563.21| +2452520|35812|4363|17595|751524|3853|15429|2|289|136|53|92.53|161.00|54.74|0.00|2901.22|4904.09|8533.00|116.04|0.00|2901.22|3017.26|-2002.87| +2452520|35812|3631|17595|751524|3853|15429|2|151|136|60|88.93|136.95|6.84|0.00|410.40|5335.80|8217.00|12.31|0.00|410.40|422.71|-4925.40| +2452520|35812|4470|17595|||15429||49|136|27||12.42|10.80|0.00|291.60||||0.00|291.60|317.84|| +2452520|35812|3135|17595|751524|3853|15429|2|42|136|16|36.24|71.75|22.24|0.00|355.84|579.84|1148.00|28.46|0.00|355.84|384.30|-224.00| +2452520|35812|9150|17595|751524|3853|15429|2|35|136|59|57.99|86.40|12.96|0.00|764.64|3421.41|5097.60|15.29|0.00|764.64|779.93|-2656.77| +2452382|68735|9522|67586|548249|112|6775|2|180|137|15|3.68|4.08|1.30|0.00|19.50|55.20|61.20|1.17|0.00|19.50|20.67|-35.70| +2452382|68735|6831|67586|548249|112|6775|2|261|137|53|47.08|56.96|48.41|0.00|2565.73|2495.24|3018.88|25.65|0.00|2565.73|2591.38|70.49| +2452382|68735|2139|67586|548249|112|6775|2|100|137|24|89.69|128.25|88.49|1635.29|2123.76|2152.56|3078.00|4.88|1635.29|488.47|493.35|-1664.09| +2452382|68735|181|67586|548249|112|6775|2|92|137|78|39.50|39.89|35.90|0.00|2800.20|3081.00|3111.42|224.01|0.00|2800.20|3024.21|-280.80| +2452382|68735|13045|67586|548249|112|6775|2|263|137|33|50.72|96.36|70.34|0.00|2321.22|1673.76|3179.88|139.27|0.00|2321.22|2460.49|647.46| +2452382|68735|10155|67586|548249|112|6775|2|65|137|79|28.61|51.78|42.45|0.00|3353.55|2260.19|4090.62|167.67|0.00|3353.55|3521.22|1093.36| +2452382|68735|15219|67586|548249|112|6775|2|207|137|11|58.89|93.63|19.66|0.00|216.26|647.79|1029.93|19.46|0.00|216.26|235.72|-431.53| +2452382|68735|7393|67586|548249|112|6775|2|188|137|62|60.92|95.64|57.38|0.00|3557.56|3777.04|5929.68|142.30|0.00|3557.56|3699.86|-219.48| +2451506|40253|6565|83850|169449|1095|694|7|88|138|33|58.18|111.70|101.64|0.00|3354.12|1919.94|3686.10|234.78|0.00|3354.12|3588.90|1434.18| +||5059|83850||1095||7|142|138|82|85.30|||1283.13|1336.60|6994.60|10281.98||1283.13||57.74|| +2451506|40253|4687|83850|169449|1095|694|7|126|138|62|78.20|101.66|71.16|0.00|4411.92|4848.40|6302.92|220.59|0.00|4411.92|4632.51|-436.48| +2451506|40253|9310|83850|169449|1095|694|7|79|138|39|20.52|22.57|0.22|0.00|8.58|800.28|880.23|0.17|0.00|8.58|8.75|-791.70| +2451506|40253|13198|83850|169449|1095|694|7|19|138|97|99.20|111.10|5.55|0.00|538.35|9622.40|10776.70|5.38|0.00|538.35|543.73|-9084.05| +2451506||10132||169449|||7||138||52.76|99.18|56.53||1074.07|1002.44|||||871.29|| +2451506|40253|4807|83850|169449|1095|694|7|102|138|23|45.30|83.35|37.50|198.37|862.50|1041.90|1917.05|33.20|198.37|664.13|697.33|-377.77| +2451506|40253|15686|83850|169449|1095|694|7|136|138|20|83.81|155.88|6.23|0.00|124.60|1676.20|3117.60|11.21|0.00|124.60|135.81|-1551.60| +2451506|40253|10100|83850|169449|1095|694|7|150|138|77|82.61|142.08|72.46|5133.06|5579.42|6360.97|10940.16|35.70|5133.06|446.36|482.06|-5914.61| +2451506|40253|5504|83850|169449|1095|694|7|57|138|79|86.91|123.41|22.21|0.00|1754.59|6865.89|9749.39|52.63|0.00|1754.59|1807.22|-5111.30| +2451506|40253|12211|83850|169449|1095|694|7|42|138|56|63.68|101.88|3.05|63.19|170.80|3566.08|5705.28|5.38|63.19|107.61|112.99|-3458.47| +2451506|40253|3481|83850|169449|1095|694|7|153|138|56|5.39|9.59|1.15|0.00|64.40|301.84|537.04|4.50|0.00|64.40|68.90|-237.44| +2452028|35084|17865|59057|39821|5032|196|2|145|139|87|92.56|119.40|97.90|0.00|8517.30|8052.72|10387.80|681.38|0.00|8517.30|9198.68|464.58| +2452028|35084|9245|59057|39821|5032|196|2|101|139|79|40.27|53.96|26.44|0.00|2088.76|3181.33|4262.84|0.00|0.00|2088.76|2088.76|-1092.57| +2452028|35084|12811|59057|39821|5032|196|2|193|139|62|29.80|44.70|19.22|0.00|1191.64|1847.60|2771.40|23.83|0.00|1191.64|1215.47|-655.96| +2452028|35084|5129|59057|39821|5032|196|2|40|139|19|62.92|93.75|75.00|470.25|1425.00|1195.48|1781.25|85.92|470.25|954.75|1040.67|-240.73| +2452028|35084|11673|59057|39821|5032|196|2|23|139|6|9.81|11.08|4.98|0.00|29.88|58.86|66.48|1.19|0.00|29.88|31.07|-28.98| +2452028|35084|6387|59057|39821|5032|196|2|225|139|12|97.78|167.20|36.78|436.94|441.36|1173.36|2006.40|0.30|436.94|4.42|4.72|-1168.94| +2452028|35084|1177|59057|39821|5032|196|2|63|139|1|89.94|99.83|43.92|37.77|43.92|89.94|99.83|0.30|37.77|6.15|6.45|-83.79| +2452028|35084|12841|59057|39821|5032|196|2|158|139|10|7.93|9.04|7.59|0.00|75.90|79.30|90.40|0.75|0.00|75.90|76.65|-3.40| +2452028|35084|4175|59057|39821|5032|196|2|25|139|81|5.17|8.89|8.71|141.10|705.51|418.77|720.09|16.93|141.10|564.41|581.34|145.64| +2452028|35084|3799||39821||196|2||139|88|||11.38|0.00||7481.76|7706.16||0.00|1001.44|1051.51|| +2452028|35084|2569|59057|39821|5032|196|2|130|139|61|32.46|62.32|55.46|0.00|3383.06|1980.06|3801.52|135.32|0.00|3383.06|3518.38|1403.00| +2452028|35084|4613|59057|39821|5032|196|2|159|139|48|13.93|18.94|1.13|0.00|54.24|668.64|909.12|1.08|0.00|54.24|55.32|-614.40| +2452028|35084|6177|59057|39821|5032|196|2|14|139|85|10.35|19.56|4.89|0.00|415.65|879.75|1662.60|8.31|0.00|415.65|423.96|-464.10| +2452028|35084|10853|59057|39821|5032|196|2|220|139|11|27.25|33.51|17.76|173.87|195.36|299.75|368.61|1.71|173.87|21.49|23.20|-278.26| +2451973|49689|9295|11738|741197|5409|19268|7|197|140|1|91.58|140.11|33.62|22.52|33.62|91.58|140.11|0.33|22.52|11.10|11.43|-80.48| +2451973||3875|||5409|19268|7||140||||8.22||||2647.72|18.24||608.28|626.52|-1089.28| +2451973|49689|6237|11738|741197|5409|19268|7|163|140|83|6.25|6.43|1.09|0.00|90.47|518.75|533.69|8.14|0.00|90.47|98.61|-428.28| +2451973|49689|4745|11738|741197|5409|19268|7|220|140|73|36.84|43.83|42.51|0.00|3103.23|2689.32|3199.59|0.00|0.00|3103.23|3103.23|413.91| +2451973|49689|11777|11738|741197|5409|19268|7|283|140|57|60.71|115.34|107.26|0.00|6113.82|3460.47|6574.38|0.00|0.00|6113.82|6113.82|2653.35| +2451973|49689|13421|11738|741197|5409|19268|7|145|140|40|98.53|177.35|147.20|0.00|5888.00|3941.20|7094.00|235.52|0.00|5888.00|6123.52|1946.80| +2451973|49689|7963|11738|741197|5409|19268|7|61|140|1|20.94|41.88|40.20|0.00|40.20|20.94|41.88|2.81|0.00|40.20|43.01|19.26| +2451973|49689|6731|11738|741197|5409|19268|7|35|140|85|41.58|49.89|31.43|0.00|2671.55|3534.30|4240.65|240.43|0.00|2671.55|2911.98|-862.75| +|49689|11177|11738||||7||140|15|34.78|50.08|||435.60|||||||-86.10| +2451973|49689|2875|11738|741197|5409|19268|7|133|140|67|52.54|89.31|58.94|0.00|3948.98|3520.18|5983.77|355.40|0.00|3948.98|4304.38|428.80| +2451973|49689|16815|11738|741197|5409|19268|7|272|140|100|8.98|13.55|6.09|0.00|609.00|898.00|1355.00|54.81|0.00|609.00|663.81|-289.00| +2451973|49689|11485|11738|741197|5409|19268|7|71|140|14|72.71|81.43|48.04|0.00|672.56|1017.94|1140.02|40.35|0.00|672.56|712.91|-345.38| +2451850|38400|12629|77161|232000|6839|30478|10|6|141|2|77.96|116.16|98.73|0.00|197.46|155.92|232.32|15.79|0.00|197.46|213.25|41.54| +2451850|38400|13813|77161|232000|6839|30478|10|6|141|91|16.87|31.88|17.85|0.00|1624.35|1535.17|2901.08|0.00|0.00|1624.35|1624.35|89.18| +2451850|38400|12710|77161|232000|6839|30478|10|296|141|2|93.49|145.84|16.04|0.00|32.08|186.98|291.68|2.56|0.00|32.08|34.64|-154.90| +2451850|38400|719|77161|232000|6839|30478|10|76|141|48|68.81|130.73|27.45|461.16|1317.60|3302.88|6275.04|25.69|461.16|856.44|882.13|-2446.44| +2451850|38400|14123|77161|232000|6839|30478|10|84|141|62|15.77|21.76|14.79|467.65|916.98|977.74|1349.12|13.47|467.65|449.33|462.80|-528.41| +2451850|38400|200|77161|232000|6839|30478|10|267|141|56|90.55|97.79|77.25|0.00|4326.00|5070.80|5476.24|216.30|0.00|4326.00|4542.30|-744.80| +2451850|38400|9710|77161|232000|6839|30478|10|25|141|44|21.26|35.07|15.78|0.00|694.32|935.44|1543.08|20.82|0.00|694.32|715.14|-241.12| +2451850|38400|6221|77161|232000|6839|30478|10|83|141|82|11.61|20.31|16.85|0.00|1381.70|952.02|1665.42|110.53|0.00|1381.70|1492.23|429.68| +2451850|38400|7784|77161|232000|6839|30478|10|216|141|69|69.90|70.59|33.17|0.00|2288.73|4823.10|4870.71|183.09|0.00|2288.73|2471.82|-2534.37| +2451777|39047|9782|35668|1714347|579|36411|2|91|142|65|27.87|39.57|14.24|0.00|925.60|1811.55|2572.05|27.76|0.00|925.60|953.36|-885.95| +2451777|39047|5125|35668|1714347|579|36411|2|226|142|23|98.59|146.89|136.60|0.00|3141.80|2267.57|3378.47|62.83|0.00|3141.80|3204.63|874.23| +2451777|39047|1409|35668|1714347|579|36411|2|276|142|55|87.81|100.98|50.49|2693.64|2776.95|4829.55|5553.90|1.66|2693.64|83.31|84.97|-4746.24| +2451777|39047|6529|35668|1714347|579|36411|2|49|142|3|41.25|58.16|21.51|0.00|64.53|123.75|174.48|3.87|0.00|64.53|68.40|-59.22| +2451777|39047|15632|35668|1714347|579|36411|2|119|142|38|43.65|83.37|39.18|0.00|1488.84|1658.70|3168.06|0.00|0.00|1488.84|1488.84|-169.86| +2451777|39047|15551|35668|1714347|579|36411|2|100|142|98|31.88|37.61|10.15|0.00|994.70|3124.24|3685.78|29.84|0.00|994.70|1024.54|-2129.54| +2451777|39047|9937|35668|1714347|579|36411|2|244|142|30|57.95|75.91|50.85|0.00|1525.50|1738.50|2277.30|0.00|0.00|1525.50|1525.50|-213.00| +2451777|39047|4897|35668|1714347|579|36411|2|239|142|80|69.59|78.63|16.51|805.68|1320.80|5567.20|6290.40|46.36|805.68|515.12|561.48|-5052.08| +2451777|39047|4135|35668|1714347|579|36411|2|278|142|99|81.67|158.43|68.12|0.00|6743.88|8085.33|15684.57|134.87|0.00|6743.88|6878.75|-1341.45| +2451777|39047|17186|35668|1714347|579|36411|2|206|142|33|39.68|40.47|4.04|51.99|133.32|1309.44|1335.51|6.50|51.99|81.33|87.83|-1228.11| +2451777|39047|15553|35668|1714347|579|36411|2|141|142|54|42.32|57.97|33.04|0.00|1784.16|2285.28|3130.38|107.04|0.00|1784.16|1891.20|-501.12| +2451777|39047|12368|35668|1714347|579|36411|2|209|142|71|25.98|35.33|31.09|0.00|2207.39|1844.58|2508.43|132.44|0.00|2207.39|2339.83|362.81| +2451777|39047|14953|35668|1714347|579|36411|2|50|142|15|63.86|86.21|41.38|0.00|620.70|957.90|1293.15|18.62|0.00|620.70|639.32|-337.20| +2451777|39047|3643|35668|1714347|579|36411|2|19|142|97|40.65|80.48|38.63|0.00|3747.11|3943.05|7806.56|224.82|0.00|3747.11|3971.93|-195.94| +2451777|39047|3833|35668|1714347|579|36411|2|282|142|26|89.83|112.28|92.06|0.00|2393.56|2335.58|2919.28|191.48|0.00|2393.56|2585.04|57.98| +||6391||1714347||36411|2||142|72|26.81|44.23|20.34|0.00||1930.32|3184.56||0.00|1464.48||-465.84| +2452505|48647|14574|20474|1609192|3957|19875|8|244|143|13|68.81|121.10|26.64|0.00|346.32|894.53|1574.30|20.77|0.00|346.32|367.09|-548.21| +||1845|20474|1609192||19875|||143||||7.59||516.12|||||516.12|526.44|| +2452505|48647|14793|20474|1609192|3957|19875|8|227|143|32|7.36|8.68|3.21|0.00|102.72|235.52|277.76|7.19|0.00|102.72|109.91|-132.80| +2452505||8173||1609192||19875||217|143|49|38.62|64.49||489.72|789.88||3160.01|27.01|489.72|300.16|327.17|| +2452505|48647|3987|20474|1609192|3957|19875|8|103|143|2|36.25|72.50|35.52|0.00|71.04|72.50|145.00|0.00|0.00|71.04|71.04|-1.46| +2452505|48647|12763|20474|1609192|3957|19875|8|149|143|14|36.72|59.11|58.51|0.00|819.14|514.08|827.54|24.57|0.00|819.14|843.71|305.06| +2452505|48647|13743|20474|1609192|3957|19875|8|277|143|12|19.79|30.27|12.71|0.00|152.52|237.48|363.24|1.52|0.00|152.52|154.04|-84.96| +2452505|48647|2721|20474|1609192|3957|19875|8|89|143|78|74.90|147.55|122.46|0.00|9551.88|5842.20|11508.90|477.59|0.00|9551.88|10029.47|3709.68| +2452505|48647|2025|20474|1609192|3957|19875|8|48|143|90|93.74|130.29|26.05|0.00|2344.50|8436.60|11726.10|187.56|0.00|2344.50|2532.06|-6092.10| +2452505|48647|16015|20474|1609192|3957|19875|8|192|143|3|86.59|115.16|29.94|0.00|89.82|259.77|345.48|5.38|0.00|89.82|95.20|-169.95| +2452505|48647|17521|20474|1609192|3957|19875|8|289|143|85|33.77|51.33|16.42|0.00|1395.70|2870.45|4363.05|55.82|0.00|1395.70|1451.52|-1474.75| +2452505|48647|9079|20474|1609192|3957|19875|8|167|143|72|34.15|49.85|47.35|0.00|3409.20|2458.80|3589.20|306.82|0.00|3409.20|3716.02|950.40| +2452505|48647|6090|20474|1609192|3957|19875|8|50|143|48|75.60|117.18|82.02|0.00|3936.96|3628.80|5624.64|0.00|0.00|3936.96|3936.96|308.16| +2452505|48647|16842|20474|1609192|3957|19875|8|235|143|68|92.66|92.66|65.78|0.00|4473.04|6300.88|6300.88|178.92|0.00|4473.04|4651.96|-1827.84| +2452505|48647|12690|20474|1609192|3957|19875|8|163|143|42|13.62|15.52|8.07|0.00|338.94|572.04|651.84|0.00|0.00|338.94|338.94|-233.10| +2451257|39186|11972|93826|1372241|2656|23071|10|267|144|47|47.63|67.15|53.72|0.00|2524.84|2238.61|3156.05|75.74|0.00|2524.84|2600.58|286.23| +2451257|39186|5635|93826|1372241|2656|23071|10|229|144|63|94.77|176.27|118.10|6845.07|7440.30|5970.51|11105.01|47.61|6845.07|595.23|642.84|-5375.28| +2451257|39186|5002|93826|1372241|2656|23071|10|133|144|86|2.55|3.03|1.60|0.00|137.60|219.30|260.58|4.12|0.00|137.60|141.72|-81.70| +|39186|11828||||23071||115|144|||115.88|83.43||||||||2034.02|188.14| +2451257|39186|7478|93826|1372241|2656|23071|10|84|144|6|55.55|76.10|36.52|0.00|219.12|333.30|456.60|8.76|0.00|219.12|227.88|-114.18| +2451257|39186|11312|93826|1372241|2656|23071|10|253|144|39|96.44|144.66|73.77|460.32|2877.03|3761.16|5641.74|48.33|460.32|2416.71|2465.04|-1344.45| +2451257|39186|13010|93826|1372241|2656|23071|10|190|144|68|55.37|101.88|39.73|2026.23|2701.64|3765.16|6927.84|13.50|2026.23|675.41|688.91|-3089.75| +2451257|39186|5744|93826|1372241|2656|23071|10|235|144|10|49.93|89.87|88.97|738.45|889.70|499.30|898.70|6.05|738.45|151.25|157.30|-348.05| +|39186|184|||2656|23071||275|144|||79.86|46.31||231.55|||||231.55|231.55|31.90| +2451257|39186|2120|93826|1372241|2656|23071|10|184|144|98|1.55|1.65|0.82|0.00|80.36|151.90|161.70|4.01|0.00|80.36|84.37|-71.54| +2451257|39186|175|93826|1372241|2656|23071|10|162|144|74|26.07|45.88|19.72|0.00|1459.28|1929.18|3395.12|58.37|0.00|1459.28|1517.65|-469.90| +2451257|39186|15838|||||||144|42|33.02||||1278.90|||38.36||||| +2452623|66262|17010|17576|1775779|382|49785|7|119|145|52|63.34|77.90|67.77|0.00|3524.04|3293.68|4050.80|70.48|0.00|3524.04|3594.52|230.36| +2452623|66262|15030|17576|1775779|382|49785|7|164|145|9|6.03|7.65|3.36|0.00|30.24|54.27|68.85|2.41|0.00|30.24|32.65|-24.03| +2452623|66262|12111|17576|1775779|382|49785|7|186|145|90|59.21|98.28|0.98|0.00|88.20|5328.90|8845.20|4.41|0.00|88.20|92.61|-5240.70| +2452623|66262|17911|17576|1775779|382|49785|7|150|145|98|82.93|84.58|77.81|0.00|7625.38|8127.14|8288.84|457.52|0.00|7625.38|8082.90|-501.76| +2452623|66262|2341|17576|1775779|382|49785|7|141|145|97|7.60|12.99|5.19|0.00|503.43|737.20|1260.03|35.24|0.00|503.43|538.67|-233.77| +2452623|66262|835|17576|1775779|382|49785|7|46|145|40|12.70|17.27|8.63|0.00|345.20|508.00|690.80|6.90|0.00|345.20|352.10|-162.80| +2452623|66262|13935|17576|1775779|382|49785|7|28|145|43|76.49|101.73|50.86|0.00|2186.98|3289.07|4374.39|131.21|0.00|2186.98|2318.19|-1102.09| +2452623|66262|15138|17576|1775779|382|49785|7|192|145|38|26.78|33.47|17.40|0.00|661.20|1017.64|1271.86|39.67|0.00|661.20|700.87|-356.44| +2452623|66262|8671|17576|1775779|382|49785|7|46|145|7|19.91|22.10|20.77|95.95|145.39|139.37|154.70|0.49|95.95|49.44|49.93|-89.93| +2450981|66961|13120|18341|375545|1549|20753|4|35|146|55|24.53|42.19|24.04|171.88|1322.20|1349.15|2320.45|46.01|171.88|1150.32|1196.33|-198.83| +2450981|66961|6004|18341|375545|1549|20753|4|96|146|66|35.63|62.35|39.28|0.00|2592.48|2351.58|4115.10|129.62|0.00|2592.48|2722.10|240.90| +||17656|18341|375545|||4|176|146|61||27.70||||1536.59|1689.70||||843.11|| +|66961|4258|||1549|20753|4|191|146|3|54.66|92.37|37.87||113.61|163.98||6.81|||120.42|-50.37| +2450981|66961|170|18341|375545|1549|20753|4|68|146|52|72.44|76.06|47.15|0.00|2451.80|3766.88|3955.12|147.10|0.00|2451.80|2598.90|-1315.08| +|66961|1070|||1549|20753||247|146||32.84||32.62|0.00||1871.88|2694.96|0.00|0.00||1859.34|-12.54| +2450981|66961|15313|18341|375545|1549|20753|4|179|146|43|89.62|145.18|55.16|1280.81|2371.88|3853.66|6242.74|87.28|1280.81|1091.07|1178.35|-2762.59| +2450981||13945|||||4|54|146|85||||||6853.55|12336.05|||||| +2450981|66961|4304|18341|375545|1549|20753|4|37|146|66|67.79|109.81|96.63|0.00|6377.58|4474.14|7247.46|255.10|0.00|6377.58|6632.68|1903.44| +2451732|63001|13799|58065|1405588|311|47034|2|272|147|99|37.80|75.60|68.04|0.00|6735.96|3742.20|7484.40|0.00|0.00|6735.96|6735.96|2993.76| +2451732|63001|12122|58065|1405588|311|47034|2|100|147|34|4.68|6.41|2.43|0.00|82.62|159.12|217.94|4.13|0.00|82.62|86.75|-76.50| +2451732||17623|58065|1405588|311|47034|2||147|100|61.74|123.48|75.32|0.00|7532.00|6174.00|12348.00||0.00|7532.00|8059.24|| +2451732|63001|8150|58065|1405588|311|47034|2|299|147|50|53.96|99.82|46.91|0.00|2345.50|2698.00|4991.00|23.45|0.00|2345.50|2368.95|-352.50| +2451732|63001|7643|58065|1405588|311|47034|2|203|147|8|52.54|65.67|22.32|0.00|178.56|420.32|525.36|1.78|0.00|178.56|180.34|-241.76| +2451732|63001|14843|58065|1405588|311|47034|2|69|147|64|28.75|40.53|5.26|0.00|336.64|1840.00|2593.92|20.19|0.00|336.64|356.83|-1503.36| +2451732|63001|1991|58065|1405588|311|47034|2|265|147|19|43.01|85.58|46.21|0.00|877.99|817.19|1626.02|70.23|0.00|877.99|948.22|60.80| +2451732|63001|320|58065|1405588|311|47034|2|284|147|71|72.20|101.08|7.07|0.00|501.97|5126.20|7176.68|5.01|0.00|501.97|506.98|-4624.23| +2450980|35022|13078|80402|924366|3530|16694|7|276|148|9|32.79|50.82|16.26|89.26|146.34|295.11|457.38|2.28|89.26|57.08|59.36|-238.03| +2450980|35022|2458|80402|924366|3530|16694|7|21|148|28|2.36|3.72|2.56|0.00|71.68|66.08|104.16|1.43|0.00|71.68|73.11|5.60| +2450980|35022|15349|80402|924366|3530|16694|7|227|148|85|46.82|66.01|44.22|1165.19|3758.70|3979.70|5610.85|155.61|1165.19|2593.51|2749.12|-1386.19| +2450980|35022|3607|80402|924366|3530|16694|7|43|148|97|88.63|161.30|61.29|0.00|5945.13|8597.11|15646.10|59.45|0.00|5945.13|6004.58|-2651.98| +2450980|35022|10153|80402|924366|3530|16694|7|265|148|49|55.07|96.92|84.32|0.00|4131.68|2698.43|4749.08|247.90|0.00|4131.68|4379.58|1433.25| +2450980|35022|14116|80402|924366|3530|16694|7|44|148|45|28.09|32.58|14.98|0.00|674.10|1264.05|1466.10|33.70|0.00|674.10|707.80|-589.95| +2450980|35022|16381|80402|924366|3530|16694|7|137|148|10|37.60|59.40|26.73|0.00|267.30|376.00|594.00|21.38|0.00|267.30|288.68|-108.70| +2450980|35022|11818|80402|924366|3530|16694|7|86|148|74|83.50|116.06|18.56|0.00|1373.44|6179.00|8588.44|27.46|0.00|1373.44|1400.90|-4805.56| +2450980|35022|17698|80402|924366|3530|16694|7|79|148|5|18.77|30.59|2.14|0.00|10.70|93.85|152.95|0.85|0.00|10.70|11.55|-83.15| +2450980|35022|631|80402|924366|3530|16694|7|124|148|15|44.61|48.62|40.84|0.00|612.60|669.15|729.30|6.12|0.00|612.60|618.72|-56.55| +2450980|35022|12704|80402|924366|3530|16694|7|39|148|96|59.23|74.62|28.35|1006.99|2721.60|5686.08|7163.52|68.58|1006.99|1714.61|1783.19|-3971.47| +2450980|35022|11318|80402|924366|3530|16694|7|106|148|53|8.11|13.54|13.26|245.97|702.78|429.83|717.62|4.56|245.97|456.81|461.37|26.98| +2450980|35022|13051|80402|924366|3530|16694|7|116|148|98|13.58|18.46|8.12|0.00|795.76|1330.84|1809.08|23.87|0.00|795.76|819.63|-535.08| +2451072|38356|229|81346|1787868|6837|8559|4|90|149|2|31.47|52.86|24.31|0.00|48.62|62.94|105.72|0.97|0.00|48.62|49.59|-14.32| +2451072|38356|2180|81346|1787868|6837|8559|4|89|149|61|22.88|40.26|24.96|0.00|1522.56|1395.68|2455.86|121.80|0.00|1522.56|1644.36|126.88| +2451072|38356|6158|81346|1787868|6837|8559|4|263|149|26|86.83|86.83|14.76|0.00|383.76|2257.58|2257.58|23.02|0.00|383.76|406.78|-1873.82| +2451072|38356|7321|81346|1787868|6837|8559|4|172|149|35|54.74|74.44|25.30|0.00|885.50|1915.90|2605.40|8.85|0.00|885.50|894.35|-1030.40| +2451072|38356|1711|81346|1787868|6837|8559|4|69|149|69|48.11|53.88|29.09|0.00|2007.21|3319.59|3717.72|80.28|0.00|2007.21|2087.49|-1312.38| +2451072|38356|3716|81346|1787868|6837|8559|4|281|149|75|68.64|114.62|12.60|0.00|945.00|5148.00|8596.50|56.70|0.00|945.00|1001.70|-4203.00| +2451072|38356|10831|81346|1787868|6837|8559|4|221|149|54|19.54|19.73|10.25|0.00|553.50|1055.16|1065.42|16.60|0.00|553.50|570.10|-501.66| +2451072|38356|2762|81346|1787868|6837|8559|4|197|149|77|68.44|88.28|45.90|2085.23|3534.30|5269.88|6797.56|57.96|2085.23|1449.07|1507.03|-3820.81| +||2399|21514||1657|46305|2||150|84|56.03|82.92|78.77|0.00|6616.68|4706.52||264.66|0.00|||1910.16| +2452163|44425|4803|21514|651362|1657|46305|2|251|150|10|48.88|65.98|38.26|0.00|382.60|488.80|659.80|7.65|0.00|382.60|390.25|-106.20| +2452163|44425|4793|21514|651362|1657|46305|2|130|150|26|76.02|117.83|24.74|521.02|643.24|1976.52|3063.58|1.22|521.02|122.22|123.44|-1854.30| +2452163|44425|1813|21514|651362|1657|46305|2|130|150|95|85.83|98.70|72.05|0.00|6844.75|8153.85|9376.50|136.89|0.00|6844.75|6981.64|-1309.10| +2452163|44425|16843|21514|651362|1657|46305|2|187|150|58|2.39|3.80|2.31|0.00|133.98|138.62|220.40|4.01|0.00|133.98|137.99|-4.64| +2452163|44425|13761|21514|651362|1657|46305|2|109|150|47|5.45|8.28|0.41|0.00|19.27|256.15|389.16|0.38|0.00|19.27|19.65|-236.88| +2452163|44425|2301|21514|651362|1657|46305|2|101|150|48|2.53|4.88|2.87|0.00|137.76|121.44|234.24|6.88|0.00|137.76|144.64|16.32| +2452163|44425|16187|21514|651362|1657|46305|2|278|150|91|81.91|105.66|8.45|0.00|768.95|7453.81|9615.06|23.06|0.00|768.95|792.01|-6684.86| +2452163|44425|7673|21514|651362|1657|46305|2|113|150|55|18.56|19.48|0.38|0.00|20.90|1020.80|1071.40|1.67|0.00|20.90|22.57|-999.90| +2452163|44425|13867|21514|651362|1657|46305|2|238|150|84|53.51|86.15|59.44|0.00|4992.96|4494.84|7236.60|299.57|0.00|4992.96|5292.53|498.12| +2452163|44425|5071|21514|651362|1657|46305|2|185|150|80|45.84|57.75|35.80|0.00|2864.00|3667.20|4620.00|0.00|0.00|2864.00|2864.00|-803.20| +2452163|44425|4695|21514|651362|1657|46305|2|4|150|72|94.49|153.07|145.41|0.00|10469.52|6803.28|11021.04|942.25|0.00|10469.52|11411.77|3666.24| +2452163|44425|17635|21514|651362|1657|46305|2|153|150|83|96.21|131.80|61.94|0.00|5141.02|7985.43|10939.40|308.46|0.00|5141.02|5449.48|-2844.41| +2452163|44425|4865|21514|651362|1657|46305|2|89|150|25|97.78|195.56|97.78|782.24|2444.50|2444.50|4889.00|83.11|782.24|1662.26|1745.37|-782.24| +2452163|44425|12939|21514|651362|1657|46305|2|182|150|35|12.78|14.44|5.19|159.85|181.65|447.30|505.40|0.21|159.85|21.80|22.01|-425.50| +2452163|44425|6759|21514|651362|1657|46305|2|82|150|26|33.03|52.84|0.52|0.00|13.52|858.78|1373.84|0.67|0.00|13.52|14.19|-845.26| +2451080|51137|10502|10380|327681|2135|27436|1|102|151|42|39.32|64.09|30.76|0.00|1291.92|1651.44|2691.78|12.91|0.00|1291.92|1304.83|-359.52| +2451080|51137|5360|10380|327681|2135|27436|1|153|151|96|2.63|4.70|4.32|348.36|414.72|252.48|451.20|3.31|348.36|66.36|69.67|-186.12| +2451080|51137|12638|10380|327681|2135|27436|1|192|151|100|85.74|120.03|51.61|0.00|5161.00|8574.00|12003.00|309.66|0.00|5161.00|5470.66|-3413.00| +2451080|51137|2588|10380|327681|2135|27436|1|115|151|91|71.00|89.46|30.41|0.00|2767.31|6461.00|8140.86|193.71|0.00|2767.31|2961.02|-3693.69| +2451080|51137|7460|10380|327681|2135|27436|1|123|151|86|29.93|43.99|36.07|0.00|3102.02|2573.98|3783.14|279.18|0.00|3102.02|3381.20|528.04| +2451080|51137|10490|10380|327681|2135|27436|1|267|151|10|62.35|115.34|21.91|0.00|219.10|623.50|1153.40|0.00|0.00|219.10|219.10|-404.40| +2451080|51137|8212|10380|327681|2135|27436|1|89|151|60|96.56|189.25|164.64|0.00|9878.40|5793.60|11355.00|296.35|0.00|9878.40|10174.75|4084.80| +2451080|51137|2809|10380|327681|2135|27436|1|223|151|55|29.59|57.40|0.57|0.00|31.35|1627.45|3157.00|0.62|0.00|31.35|31.97|-1596.10| +2451080|51137|11570|10380|327681|2135|27436|1|150|151|53|76.38|87.07|87.07|0.00|4614.71|4048.14|4614.71|0.00|0.00|4614.71|4614.71|566.57| +2451080|51137|16832|10380|327681|2135|27436|1|34|151|53|30.72|47.00|31.02|0.00|1644.06|1628.16|2491.00|147.96|0.00|1644.06|1792.02|15.90| +2451080|51137|17414|10380|327681|2135|27436|1|182|151|10|72.13|109.63|29.60|0.00|296.00|721.30|1096.30|14.80|0.00|296.00|310.80|-425.30| +2451080|51137|2947|10380|327681|2135|27436|1|129|151|99|31.21|43.06|25.83|0.00|2557.17|3089.79|4262.94|204.57|0.00|2557.17|2761.74|-532.62| +2451080|51137|15836|10380|327681|2135|27436|1|248|151|100|17.40|17.92|11.11|0.00|1111.00|1740.00|1792.00|11.11|0.00|1111.00|1122.11|-629.00| +2451080|51137|4579|10380|327681|2135|27436|1|107|151|46|24.27|36.16|33.62|0.00|1546.52|1116.42|1663.36|46.39|0.00|1546.52|1592.91|430.10| +2451080|51137|5918|10380|327681|2135|27436|1|218|151|30|23.33|40.82|26.94|0.00|808.20|699.90|1224.60|24.24|0.00|808.20|832.44|108.30| +2452186|64310|14685|20287|1016656|225|26994|4|250|152|50|16.88|19.41|18.82|0.00|941.00|844.00|970.50|0.00|0.00|941.00|941.00|97.00| +2452186|64310|1563|20287|1016656|225|26994|4|100|152|100|34.00|35.02|21.01|0.00|2101.00|3400.00|3502.00|21.01|0.00|2101.00|2122.01|-1299.00| +2452186|64310|10671|20287|1016656|225|26994|4|46|152|90|43.87|56.59|48.66|0.00|4379.40|3948.30|5093.10|43.79|0.00|4379.40|4423.19|431.10| +2452186|64310|2417|20287|1016656|225|26994|4|19|152|83|22.59|35.69|29.97|0.00|2487.51|1874.97|2962.27|74.62|0.00|2487.51|2562.13|612.54| +2452186|64310|7567|20287|1016656|225|26994|4|291|152|5|78.31|90.83|18.16|28.14|90.80|391.55|454.15|1.25|28.14|62.66|63.91|-328.89| +2452186|64310|8103|20287|1016656|225|26994|4|92|152|80|48.80|79.54|33.40|0.00|2672.00|3904.00|6363.20|26.72|0.00|2672.00|2698.72|-1232.00| +2452186|64310|311|20287|1016656|225|26994|4|174|152|64|45.58|54.24|52.61|0.00|3367.04|2917.12|3471.36|202.02|0.00|3367.04|3569.06|449.92| +||3039||1016656||26994|4|192|152|59||||0.00|659.03||941.64|59.31|0.00|659.03|718.34|| +2452186|64310|161|20287|1016656|225|26994|4|259|152|25|70.59|108.70|69.56|0.00|1739.00|1764.75|2717.50|104.34|0.00|1739.00|1843.34|-25.75| +2452186|64310|4389|20287|1016656|225|26994|4|28|152|49|98.37|137.71|75.74|0.00|3711.26|4820.13|6747.79|37.11|0.00|3711.26|3748.37|-1108.87| +2452278|41440|9891|25423|746508|6440|15925|7|133|153|7|31.02|41.56|31.17|0.00|218.19|217.14|290.92|6.54|0.00|218.19|224.73|1.05| +2452278|41440|17676|25423|746508|6440|15925|7|9|153|6|4.82|9.15|6.95|0.00|41.70|28.92|54.90|3.75|0.00|41.70|45.45|12.78| +2452278|41440|16915||||||31|153|40|99.52||34.83|||3980.80||111.45||1393.20|1504.65|-2587.60| +2452278|41440|5463|25423|746508|6440|15925|7|202|153|78|58.26|76.90|67.67|0.00|5278.26|4544.28|5998.20|369.47|0.00|5278.26|5647.73|733.98| +2452278|41440|10945|25423|746508|6440|15925|7|127|153|91|20.77|20.77|5.19|0.00|472.29|1890.07|1890.07|33.06|0.00|472.29|505.35|-1417.78| +2452278|41440|17685|25423|746508|6440|15925|7|46|153|23|85.32|145.89|65.65|0.00|1509.95|1962.36|3355.47|135.89|0.00|1509.95|1645.84|-452.41| +2452278|41440|2640|25423|746508|6440|15925|7|204|153|16|32.15|33.75|29.70|0.00|475.20|514.40|540.00|42.76|0.00|475.20|517.96|-39.20| +2452278|41440|9921|25423|746508|6440|15925|7|193|153|35|83.22|119.83|55.12|0.00|1929.20|2912.70|4194.05|115.75|0.00|1929.20|2044.95|-983.50| +2451848|51847|5597|69252|1126541|6677|14413|4|69|154|93|75.05|98.31|85.52|0.00|7953.36|6979.65|9142.83|715.80|0.00|7953.36|8669.16|973.71| +2451848|51847|5000|69252|1126541|6677|14413|4|275|154|38|3.65|6.97|0.00|0.00|0.00|138.70|264.86|0.00|0.00|0.00|0.00|-138.70| +2451848|51847|1069|69252|1126541|6677|14413|4|158|154|21|93.63|122.65|19.62|0.00|412.02|1966.23|2575.65|32.96|0.00|412.02|444.98|-1554.21| +2451848|51847|1045|69252|1126541|6677|14413|4|204|154|32|53.30|64.49|36.75|0.00|1176.00|1705.60|2063.68|35.28|0.00|1176.00|1211.28|-529.60| +2451848|51847|12344|69252|1126541|6677|14413|4|88|154|70|46.23|79.51|39.75|0.00|2782.50|3236.10|5565.70|222.60|0.00|2782.50|3005.10|-453.60| +2451848|51847|17047|69252|1126541|6677|14413|4|29|154|32|80.96|138.44|83.06|0.00|2657.92|2590.72|4430.08|159.47|0.00|2657.92|2817.39|67.20| +2451848|51847|5786|69252|1126541|6677|14413|4|202|154|21|97.22|154.57|149.93|2865.16|3148.53|2041.62|3245.97|17.00|2865.16|283.37|300.37|-1758.25| +2451848|51847|815|69252|1126541|6677|14413|4|99|154|2|52.21|78.31|78.31|0.00|156.62|104.42|156.62|4.69|0.00|156.62|161.31|52.20| +2451848|51847|629|69252|1126541|6677|14413|4|62|154|61|43.94|76.01|41.04|0.00|2503.44|2680.34|4636.61|100.13|0.00|2503.44|2603.57|-176.90| +2451848|51847|908|69252|1126541|6677|14413|4|15|154|57|80.31|106.00|45.58|0.00|2598.06|4577.67|6042.00|0.00|0.00|2598.06|2598.06|-1979.61| +2451848|51847|6973|69252|1126541|6677|14413|4|172|154|24|71.96|142.48|92.61|0.00|2222.64|1727.04|3419.52|177.81|0.00|2222.64|2400.45|495.60| +2451848|51847|2528|69252|1126541|6677|14413|4|207|154|58|37.54|42.79|29.95|0.00|1737.10|2177.32|2481.82|156.33|0.00|1737.10|1893.43|-440.22| +2451848|51847|1349|69252|1126541|6677|14413|4|185|154|77|63.20|121.34|104.35|5383.41|8034.95|4866.40|9343.18|212.12|5383.41|2651.54|2863.66|-2214.86| +2451955|60898|3481|37327|435886|5998|46510|2|289|155|24|29.90|37.37|29.89|0.00|717.36|717.60|896.88|57.38|0.00|717.36|774.74|-0.24| +2451955|60898|3367|37327|435886|5998|46510|2|95|155|22|9.20|13.98|0.41|0.00|9.02|202.40|307.56|0.45|0.00|9.02|9.47|-193.38| +2451955|60898|5697|37327|435886|5998|46510|2|172|155|93|76.40|147.45|67.82|0.00|6307.26|7105.20|13712.85|63.07|0.00|6307.26|6370.33|-797.94| +2451955|60898|6219|37327|435886|5998|46510|2|52|155|78|86.62|124.73|44.90|0.00|3502.20|6756.36|9728.94|140.08|0.00|3502.20|3642.28|-3254.16| +2451955|60898|3241|37327|435886|5998|46510|2|36|155|31|58.98|75.49|67.18|0.00|2082.58|1828.38|2340.19|145.78|0.00|2082.58|2228.36|254.20| +2451955|60898|9|37327|435886|5998|46510|2|184|155|66|2.31|3.62|0.97|0.00|64.02|152.46|238.92|1.28|0.00|64.02|65.30|-88.44| +2451955|60898|2031|37327|435886|5998|46510|2|262|155|31|32.14|61.06|3.05|0.00|94.55|996.34|1892.86|1.89|0.00|94.55|96.44|-901.79| +2451955|60898|12213|37327|435886|5998|46510|2|292|155|37|56.03|108.69|4.34|30.51|160.58|2073.11|4021.53|5.20|30.51|130.07|135.27|-1943.04| +2451955|60898|7645|37327|435886|5998|46510|2|97|155|59|38.22|65.73|52.58|2512.79|3102.22|2254.98|3878.07|5.89|2512.79|589.43|595.32|-1665.55| +2451955|60898|6429|37327||5998||2||155||||16.88|0.00|236.32|||4.72|0.00|236.32|241.04|| +2451955||953|37327|435886|||2||155|68|22.32|||0.00|901.00|||18.02|0.00||919.02|| +2451955|60898|10897|37327|435886|5998|46510|2|160|155|28|69.78|87.92|40.44|0.00|1132.32|1953.84|2461.76|45.29|0.00|1132.32|1177.61|-821.52| +2451955|60898|8595|37327|435886|5998|46510|2|71|155|55|58.45|64.87|53.84|0.00|2961.20|3214.75|3567.85|148.06|0.00|2961.20|3109.26|-253.55| +2451890|43278|2264||1374375||46642|||156|33||||0.00||1142.79||17.62|0.00|881.10||| +2451890|43278|902|51865|1374375|5685|46642|7|135|156|59|21.26|23.59|4.71|33.34|277.89|1254.34|1391.81|0.00|33.34|244.55|244.55|-1009.79| +2451890|43278|15221|51865|1374375|5685|46642|7|277|156|46|31.55|39.75|9.93|0.00|456.78|1451.30|1828.50|18.27|0.00|456.78|475.05|-994.52| +2451890|43278|11498|51865|1374375|5685|46642|7|160|156|85|74.57|127.51|118.58|4334.09|10079.30|6338.45|10838.35|114.90|4334.09|5745.21|5860.11|-593.24| +2451890|43278|15140|51865|1374375|5685|46642|7|209|156|73|39.35|52.33|32.44|1160.37|2368.12|2872.55|3820.09|84.54|1160.37|1207.75|1292.29|-1664.80| +2451890|43278|3272|51865|1374375|5685|46642|7|119|156|77|52.32|56.50|34.46|796.02|2653.42|4028.64|4350.50|167.16|796.02|1857.40|2024.56|-2171.24| +2451890|43278|4757|51865|1374375|5685|46642|7|94|156|31|96.25|154.00|27.72|0.00|859.32|2983.75|4774.00|68.74|0.00|859.32|928.06|-2124.43| +|43278|12800|||5685||7|27|156|71||100.76||0.00|5722.60||7153.96|0.00|0.00|||| +2451890|43278|536|51865|1374375|5685|46642|7|130|156|97|38.66|63.78|43.37|0.00|4206.89|3750.02|6186.66|42.06|0.00|4206.89|4248.95|456.87| +2451890|43278|14930|51865|1374375|5685|46642|7|174|156|77|24.46|32.04|28.51|0.00|2195.27|1883.42|2467.08|175.62|0.00|2195.27|2370.89|311.85| +2451890|43278|3007|51865|1374375|5685|46642|7|130|156|81|81.78|87.50|54.25|0.00|4394.25|6624.18|7087.50|219.71|0.00|4394.25|4613.96|-2229.93| +2451890|43278|4921|51865|1374375|5685|46642|7|108|156|83|46.64|50.83|34.05|1978.30|2826.15|3871.12|4218.89|50.87|1978.30|847.85|898.72|-3023.27| +2451890|43278|9713|51865|1374375|5685|46642|7|60|156|92|74.47|132.55|45.06|0.00|4145.52|6851.24|12194.60|373.09|0.00|4145.52|4518.61|-2705.72| +2451890|43278|5402|51865|1374375|5685|46642|7|290|156|75|7.92|11.72|2.22|0.00|166.50|594.00|879.00|13.32|0.00|166.50|179.82|-427.50| +2451890|43278|15401|51865|1374375|5685|46642|7|34|156|16|46.99|59.67|50.71|0.00|811.36|751.84|954.72|8.11|0.00|811.36|819.47|59.52| +2452218||3689|9692|989236|780||||157||22.59|||0.00||1874.97|2755.60|49.60|0.00|||| +2452218|75573|8535|9692|989236|780|39124|7|195|157|78|50.48|68.65|52.86|0.00|4123.08|3937.44|5354.70|329.84|0.00|4123.08|4452.92|185.64| +2452218|75573|7369|9692|989236|780|39124|7|58|157|4|8.96|13.44|10.48|5.44|41.92|35.84|53.76|3.28|5.44|36.48|39.76|0.64| +2452218|75573|3227|9692|989236|780|39124|7|228|157|16|6.72|6.92|6.50|0.00|104.00|107.52|110.72|2.08|0.00|104.00|106.08|-3.52| +2452218|75573|13063|9692|989236|780|39124|7|212|157|23|10.36|11.91|6.66|0.00|153.18|238.28|273.93|3.06|0.00|153.18|156.24|-85.10| +2452218|75573|12787||989236|780|39124|7|209|157|42|70.84|87.84|51.82|0.00|2176.44|||195.87|0.00|||| +2452218|75573|14193|9692|989236|780|39124|7|47|157|12|89.12|148.83|62.50|0.00|750.00|1069.44|1785.96|0.00|0.00|750.00|750.00|-319.44| +2452218|75573|4037|9692|989236|780|39124|7|270|157|32|56.34|80.56|80.56|0.00|2577.92|1802.88|2577.92|77.33|0.00|2577.92|2655.25|775.04| +2452218|75573|8169|9692|989236|780|39124|7|264|157|96|31.21|56.17|33.14|0.00|3181.44|2996.16|5392.32|63.62|0.00|3181.44|3245.06|185.28| +2452218|75573|1681|9692|989236|780|39124|7|198|157|71|80.41|105.33|14.74|0.00|1046.54|5709.11|7478.43|83.72|0.00|1046.54|1130.26|-4662.57| +2452218|75573|6679|9692|989236|780|39124|7|19|157|78|53.43|80.14|46.48|0.00|3625.44|4167.54|6250.92|181.27|0.00|3625.44|3806.71|-542.10| +2452218|75573|4587|9692|989236|780|39124|7|141|157|57|6.75|10.26|4.30|0.00|245.10|384.75|584.82|22.05|0.00|245.10|267.15|-139.65| +2452218|75573|57|9692|989236|780|39124|7|189|157|43|63.63|121.53|40.10|0.00|1724.30|2736.09|5225.79|0.00|0.00|1724.30|1724.30|-1011.79| +2452218|75573|2409|9692|989236|780|39124|7|94|157|84|20.09|30.53|27.47|0.00|2307.48|1687.56|2564.52|207.67|0.00|2307.48|2515.15|619.92| +2452218|75573|15827|9692|989236|780|39124|7|11|157|63|41.42|80.76|3.23|0.00|203.49|2609.46|5087.88|0.00|0.00|203.49|203.49|-2405.97| +2451153|61555|7142|73251|807490|1282|35474|7|22|158|97|64.23|111.11|25.55|2354.43|2478.35|6230.31|10777.67|11.15|2354.43|123.92|135.07|-6106.39| +2451153|61555|14968|73251|807490|1282|35474|7|300|158|94|37.97|41.38|7.44|0.00|699.36|3569.18|3889.72|55.94|0.00|699.36|755.30|-2869.82| +2451153|61555|7597|73251|807490|1282|35474|7|51|158|20|25.31|31.63|0.00|0.00|0.00|506.20|632.60|0.00|0.00|0.00|0.00|-506.20| +2451153|61555|4838|73251|807490|1282|35474|7|142|158|38|99.68|185.40|185.40|0.00|7045.20|3787.84|7045.20|281.80|0.00|7045.20|7327.00|3257.36| +2451153|61555|1190|73251|807490|1282|35474|7|108|158|49|99.68|146.52|112.82|0.00|5528.18|4884.32|7179.48|0.00|0.00|5528.18|5528.18|643.86| +2451153|61555|15406|73251|807490|1282|35474|7|164|158|100|76.67|136.47|107.81|0.00|10781.00|7667.00|13647.00|0.00|0.00|10781.00|10781.00|3114.00| +2451153|61555|6160|73251|807490|1282|35474|7|156|158|35|64.32|97.76|37.14|0.00|1299.90|2251.20|3421.60|38.99|0.00|1299.90|1338.89|-951.30| +2451153|61555|10351|73251|807490|1282|35474|7|206|158|8|19.84|30.95|21.04|0.00|168.32|158.72|247.60|15.14|0.00|168.32|183.46|9.60| +2451153|61555|14734|73251|807490|1282|35474|7|104|158|60|85.36|118.65|46.27|0.00|2776.20|5121.60|7119.00|194.33|0.00|2776.20|2970.53|-2345.40| +2451153|61555|5659|73251|807490|1282|35474|7|16|158|22|67.48|122.13|42.74|0.00|940.28|1484.56|2686.86|65.81|0.00|940.28|1006.09|-544.28| +2451153||8437|73251||1282|35474|||158|47|||48.43||2276.21|2701.56|4295.33|0.00|||2276.21|-425.35| +2451153|61555|11104|73251|807490|1282|35474|7|154|158|20|81.88|126.09|117.26|422.13|2345.20|1637.60|2521.80|57.69|422.13|1923.07|1980.76|285.47| +2451153|61555|17368|73251|807490|1282|35474|7|228|158|14|27.75|43.01|12.47|0.00|174.58|388.50|602.14|5.23|0.00|174.58|179.81|-213.92| +2451153|61555|13156|73251|807490|1282|35474|7|57|158|30|56.90|73.97|49.55|0.00|1486.50|1707.00|2219.10|59.46|0.00|1486.50|1545.96|-220.50| +2451153|61555|14264|73251|807490|1282|35474|7|183|158|100|18.66|30.78|24.62|0.00|2462.00|1866.00|3078.00|172.34|0.00|2462.00|2634.34|596.00| +2451515|45995|4072|34823|103215|6360|12352|8|126|159|51|30.76|41.83|26.77|709.94|1365.27|1568.76|2133.33|26.21|709.94|655.33|681.54|-913.43| +2451515|45995|8062|34823|103215|6360|12352|8|132|159|9|52.89|75.10|45.06|0.00|405.54|476.01|675.90|28.38|0.00|405.54|433.92|-70.47| +2451515|45995|9194|34823|103215|6360|12352|8|287|159|84|64.74|95.81|82.39|0.00|6920.76|5438.16|8048.04|69.20|0.00|6920.76|6989.96|1482.60| +2451515|45995|2236|34823|103215|6360|12352|8|237|159|32|59.79|61.58|52.34|0.00|1674.88|1913.28|1970.56|100.49|0.00|1674.88|1775.37|-238.40| +2451515|45995|5368|34823|103215|6360|12352|8|150|159|73|94.72|157.23|69.18|0.00|5050.14|6914.56|11477.79|50.50|0.00|5050.14|5100.64|-1864.42| +2451515|45995|11296|34823|103215|6360|12352|8|252|159|9|30.84|55.20|34.22|0.00|307.98|277.56|496.80|6.15|0.00|307.98|314.13|30.42| +2451515|45995|14965|34823|103215|6360|12352|8|135|159|53|64.61|82.05|12.30|0.00|651.90|3424.33|4348.65|45.63|0.00|651.90|697.53|-2772.43| +2451515|45995|15079|34823|103215|6360|12352|8|269|159|78|95.39|145.94|138.64|6272.07|10813.92|7440.42|11383.32|408.76|6272.07|4541.85|4950.61|-2898.57| +2451515|45995|5908|34823|103215|6360|12352|8|146|159|2|82.07|87.81|20.19|0.00|40.38|164.14|175.62|1.61|0.00|40.38|41.99|-123.76| +2451515|45995|12004|34823|103215|6360|12352|8|105|159|67|81.91|143.34|120.40|3065.38|8066.80|5487.97|9603.78|0.00|3065.38|5001.42|5001.42|-486.55| +2451515|45995|79|34823|103215|6360|12352|8|55|159|32|92.22|159.54|94.12|0.00|3011.84|2951.04|5105.28|180.71|0.00|3011.84|3192.55|60.80| +2451515|45995|4636|34823|103215|6360|12352|8|181|159|16|79.35|121.40|47.34|0.00|757.44|1269.60|1942.40|60.59|0.00|757.44|818.03|-512.16| +||7240||103215||12352|8|153|159|||183.43|||2311.20|1871.80||||2311.20||439.40| +||11257||103215|6360|12352|8|118|159|19|6.68||||96.33|126.92||||96.33||| +2451944|74689|843|76901|1090655|7162|28516|8|251|160|21|34.35|35.38|23.70|0.00|497.70|721.35|742.98|34.83|0.00|497.70|532.53|-223.65| +2451944|74689|17467|76901|1090655|7162|28516|8|110|160|68|66.21|82.10|59.11|1085.25|4019.48|4502.28|5582.80|88.02|1085.25|2934.23|3022.25|-1568.05| +2451944|74689|1865|76901|1090655|7162|28516|8|262|160|20|77.47|80.56|26.58|0.00|531.60|1549.40|1611.20|37.21|0.00|531.60|568.81|-1017.80| +2451944|74689|8055|76901|1090655|7162|28516|8|245|160|66|77.59|85.34|28.16|0.00|1858.56|5120.94|5632.44|37.17|0.00|1858.56|1895.73|-3262.38| +2451944|74689|5233|76901|1090655|7162|28516|8|267|160|64|62.79|67.18|4.03|0.00|257.92|4018.56|4299.52|2.57|0.00|257.92|260.49|-3760.64| +2451944|74689|17173|76901|1090655|7162|28516|8|42|160|95|81.51|142.64|78.45|7080.11|7452.75|7743.45|13550.80|18.63|7080.11|372.64|391.27|-7370.81| +2451944|74689|945|76901|1090655|7162|28516|8|150|160|32|87.05|143.63|103.41|0.00|3309.12|2785.60|4596.16|0.00|0.00|3309.12|3309.12|523.52| +2451944|74689|14683|76901|1090655|7162|28516|8|79|160|1|22.61|27.13|8.95|0.00|8.95|22.61|27.13|0.80|0.00|8.95|9.75|-13.66| +2451944|74689|11663|76901|1090655|7162|28516|8|193|160|46|61.34|120.83|25.37|0.00|1167.02|2821.64|5558.18|70.02|0.00|1167.02|1237.04|-1654.62| +2451944|74689|2551|76901|1090655|7162|28516|8|169|160|49|24.29|40.56|31.23|306.05|1530.27|1190.21|1987.44|85.69|306.05|1224.22|1309.91|34.01| +2451944|74689|15779|76901|1090655|7162|28516|8|148|160|44|38.98|48.72|8.28|0.00|364.32|1715.12|2143.68|7.28|0.00|364.32|371.60|-1350.80| +2451944|74689|1021|76901|1090655|7162|28516|8|93|160|56|26.95|35.30|8.11|0.00|454.16|1509.20|1976.80|0.00|0.00|454.16|454.16|-1055.04| +2451944|74689|17229|76901|1090655|7162|28516|8|113|160|7|5.14|10.28|0.00|0.00|0.00|35.98|71.96|0.00|0.00|0.00|0.00|-35.98| +2451129|46774|7363|7791|573397|6650|43456|7|189|161|86|63.41|101.45|4.05|0.00|348.30|5453.26|8724.70|20.89|0.00|348.30|369.19|-5104.96| +2451129|46774|13132|7791|573397|6650|43456|7|223|161|96|1.92|3.37|2.29|0.00|219.84|184.32|323.52|8.79|0.00|219.84|228.63|35.52| +2451129|46774|5308|7791|573397|6650|43456|7|131|161|51|66.32|130.65|65.32|0.00|3331.32|3382.32|6663.15|299.81|0.00|3331.32|3631.13|-51.00| +2451129|46774|811|7791|573397|6650|43456|7|254|161|43|90.80|138.01|63.48|0.00|2729.64|3904.40|5934.43|109.18|0.00|2729.64|2838.82|-1174.76| +2451129|46774|3265|7791|573397|6650|43456|7|33|161|51|30.98|40.58|19.47|0.00|992.97|1579.98|2069.58|9.92|0.00|992.97|1002.89|-587.01| +2451129|46774|1196|7791|573397|6650|43456|7|38|161|47|27.99|48.70|43.34|1385.14|2036.98|1315.53|2288.90|6.51|1385.14|651.84|658.35|-663.69| +2451129|46774|9487|7791|573397|6650|43456|7|41|161|41|71.07|130.05|79.33|0.00|3252.53|2913.87|5332.05|162.62|0.00|3252.53|3415.15|338.66| +2451129|46774|16105|7791|573397|6650|43456|7|47|161|99|94.25|121.58|70.51|0.00|6980.49|9330.75|12036.42|418.82|0.00|6980.49|7399.31|-2350.26| +2451717|51263|6955|18006|865626|7199|40419|7|241|162|83|65.45|99.48|24.87|0.00|2064.21|5432.35|8256.84|82.56|0.00|2064.21|2146.77|-3368.14| +2451717|51263|2150|18006|865626|7199|40419|7|200|162|16|31.56|51.75|41.91|0.00|670.56|504.96|828.00|26.82|0.00|670.56|697.38|165.60| +2451717|51263|14635|18006|865626|7199|40419|7|83|162|62|43.77|58.65|28.15|0.00|1745.30|2713.74|3636.30|157.07|0.00|1745.30|1902.37|-968.44| +2451717|51263|14489|18006|865626|7199|40419|7|59|162|72|80.96|80.96|37.24|0.00|2681.28|5829.12|5829.12|80.43|0.00|2681.28|2761.71|-3147.84| +2451717|51263|3455|18006|865626|7199|40419|7|134|162|89|61.56|75.71|20.44|0.00|1819.16|5478.84|6738.19|127.34|0.00|1819.16|1946.50|-3659.68| +2451717|51263|10159|18006|865626|7199|40419|7|21|162|75|76.07|104.97|17.84|0.00|1338.00|5705.25|7872.75|80.28|0.00|1338.00|1418.28|-4367.25| +2451717|51263|17753|18006|865626|7199|40419|7|89|162|63|86.48|171.23|29.10|0.00|1833.30|5448.24|10787.49|109.99|0.00|1833.30|1943.29|-3614.94| +2451717|51263|1333|18006|865626|7199|40419|7|52|162|57|81.26|100.76|90.68|0.00|5168.76|4631.82|5743.32|155.06|0.00|5168.76|5323.82|536.94| +2451717|51263|17000|18006|865626|7199|40419|7|75|162|28|77.05|91.68|38.50|0.00|1078.00|2157.40|2567.04|0.00|0.00|1078.00|1078.00|-1079.40| +2451717|51263|5954|18006|865626|7199|40419|7|72|162|58|61.71|62.94|4.40|0.00|255.20|3579.18|3650.52|15.31|0.00|255.20|270.51|-3323.98| +2451543|34093|9658|27551|1587878|1005|22532|4|232|163|39|44.79|63.15|38.52|0.00|1502.28|1746.81|2462.85|105.15|0.00|1502.28|1607.43|-244.53| +2451543|34093|14329|27551|1587878|1005|22532|4|70|163|25|79.69|152.20|85.23|0.00|2130.75|1992.25|3805.00|127.84|0.00|2130.75|2258.59|138.50| +2451543|34093|11470|27551|1587878|1005|22532|4|194|163|32|74.71|100.85|82.69|0.00|2646.08|2390.72|3227.20|26.46|0.00|2646.08|2672.54|255.36| +2451543|34093|214|27551|1587878|1005|22532|4|279|163|36|17.50|29.75|24.99|0.00|899.64|630.00|1071.00|35.98|0.00|899.64|935.62|269.64| +2451543|34093|5977|27551|1587878|1005|22532|4|81|163|52|12.57|17.59|9.14|0.00|475.28|653.64|914.68|0.00|0.00|475.28|475.28|-178.36| +2451543|34093|15967|27551|1587878|1005|22532|4|19|163|20|83.60|96.97|31.03|0.00|620.60|1672.00|1939.40|6.20|0.00|620.60|626.80|-1051.40| +2451543|34093|17281|27551|1587878|1005|22532|4|97|163|78|42.34|42.76|32.92|0.00|2567.76|3302.52|3335.28|25.67|0.00|2567.76|2593.43|-734.76| +2451543|34093|4654|27551|1587878|1005|22532|4|141|163|53|69.62|99.55|85.61|0.00|4537.33|3689.86|5276.15|181.49|0.00|4537.33|4718.82|847.47| +2451543|34093|12928|27551|1587878|1005|22532|4|210|163|37|48.49|89.22|25.87|0.00|957.19|1794.13|3301.14|38.28|0.00|957.19|995.47|-836.94| +2451543|34093|4651|27551|1587878|1005|22532|4|85|163|85|59.85|115.51|77.39|0.00|6578.15|5087.25|9818.35|328.90|0.00|6578.15|6907.05|1490.90| +2451543|34093|16516|27551|1587878|1005|22532|4|294|163|16|97.34|105.12|99.86|0.00|1597.76|1557.44|1681.92|127.82|0.00|1597.76|1725.58|40.32| +2451543|34093|16700|27551|1587878|1005|22532|4|290|163|99|45.56|51.93|20.25|0.00|2004.75|4510.44|5141.07|160.38|0.00|2004.75|2165.13|-2505.69| +2451706|67090|12745|86432|1392299|4566|32880|4|48|164|15|97.22|188.60|49.03|0.00|735.45|1458.30|2829.00|66.19|0.00|735.45|801.64|-722.85| +2451706|67090|1082|86432|1392299|4566|32880|4|187|164|83|75.08|84.84|39.87|860.39|3309.21|6231.64|7041.72|24.48|860.39|2448.82|2473.30|-3782.82| +2451706|67090|767|86432|1392299|4566|32880|4|29|164|71|35.97|49.27|45.82|0.00|3253.22|2553.87|3498.17|65.06|0.00|3253.22|3318.28|699.35| +2451706|67090|16136|86432|1392299|4566|32880|4|248|164|67|63.28|84.16|28.61|0.00|1916.87|4239.76|5638.72|95.84|0.00|1916.87|2012.71|-2322.89| +2451706|67090|12371|86432|1392299|4566|32880|4|89|164|28|16.80|17.30|12.28|0.00|343.84|470.40|484.40|17.19|0.00|343.84|361.03|-126.56| +2451706|67090|15799|86432|1392299|4566|32880|4|121|164|8|73.72|83.30|79.13|0.00|633.04|589.76|666.40|25.32|0.00|633.04|658.36|43.28| +|67090|14276|||4566|32880||25|164|63|||46.75||2945.25|4583.88|5775.21|||||-1638.63| +2451706|67090|13688|86432|1392299|4566|32880|4|80|164|6|43.24|84.31|77.56|0.00|465.36|259.44|505.86|32.57|0.00|465.36|497.93|205.92| +2451706|67090|5557|86432|1392299|4566|32880|4|196|164|98|79.86|114.19|93.63|3853.81|9175.74|7826.28|11190.62|319.31|3853.81|5321.93|5641.24|-2504.35| +2450929|60074|14296|2420|1210344|6621|2145|1|189|165|29|30.12|55.72|25.07|0.00|727.03|873.48|1615.88|50.89|0.00|727.03|777.92|-146.45| +2450929|60074|8521|2420|1210344|6621|2145|1|225|165|1|97.55|191.19|86.03|0.00|86.03|97.55|191.19|7.74|0.00|86.03|93.77|-11.52| +2450929||2710|||6621||1||165|33|86.90|132.95|69.13|0.00|2281.29|2867.70|4387.35|205.31|0.00|||-586.41| +2450929|60074|11527|2420|1210344|6621|2145|1|173|165|100|97.88|165.41|90.97|0.00|9097.00|9788.00|16541.00|0.00|0.00|9097.00|9097.00|-691.00| +2450929|60074|9829|2420|1210344|6621|2145|1|113|165|36|84.18|98.49|24.62|859.73|886.32|3030.48|3545.64|0.00|859.73|26.59|26.59|-3003.89| +2450929|60074|1873|2420|1210344|6621|2145|1|53|165|20|4.12|4.44|0.35|0.00|7.00|82.40|88.80|0.35|0.00|7.00|7.35|-75.40| +2450929|60074|5456|2420|1210344|6621|2145|1|213|165|34|79.85|134.14|116.70|2261.64|3967.80|2714.90|4560.76|153.55|2261.64|1706.16|1859.71|-1008.74| +2450929|60074|2774|2420|1210344|6621|2145|1|193|165|56|18.68|34.55|13.47|0.00|754.32|1046.08|1934.80|45.25|0.00|754.32|799.57|-291.76| +2450929|60074|11666|2420|1210344|6621|2145|1|75|165|68|23.07|44.29|36.31|0.00|2469.08|1568.76|3011.72|24.69|0.00|2469.08|2493.77|900.32| +2450929|60074|11536|2420|1210344|6621|2145|1|300|165|15|40.37|53.69|12.88|0.00|193.20|605.55|805.35|0.00|0.00|193.20|193.20|-412.35| +2450929|60074|10309|2420|1210344|6621|2145|1|133|165|59|4.88|7.27|3.56|0.00|210.04|287.92|428.93|0.00|0.00|210.04|210.04|-77.88| +2451193|47723|15334|19860|838449|3885|48874|7|127|166|54|14.76|15.49|6.19|0.00|334.26|797.04|836.46|16.71|0.00|334.26|350.97|-462.78| +2451193|47723|12664|19860|838449|3885|48874|7|69|166|38|33.78|47.29|5.20|0.00|197.60|1283.64|1797.02|15.80|0.00|197.60|213.40|-1086.04| +2451193|47723|15769|19860|838449|3885|48874|7|34|166|74|34.52|67.65|30.44|0.00|2252.56|2554.48|5006.10|22.52|0.00|2252.56|2275.08|-301.92| +2451193|47723|10771|19860|838449|3885|48874|7|299|166|65|21.85|36.48|20.06|0.00|1303.90|1420.25|2371.20|39.11|0.00|1303.90|1343.01|-116.35| +2451193|47723|14791|19860|838449|3885|48874|7|172|166|31|49.30|66.55|16.63|0.00|515.53|1528.30|2063.05|15.46|0.00|515.53|530.99|-1012.77| +2451193|47723|7916|19860|838449|3885|48874|7|50|166|40|14.17|14.17|2.83|0.00|113.20|566.80|566.80|4.52|0.00|113.20|117.72|-453.60| +2451193|47723|11086|19860|838449|3885|48874|7|24|166|2|89.66|160.49|59.38|0.00|118.76|179.32|320.98|4.75|0.00|118.76|123.51|-60.56| +2451193|47723|1123|19860|838449|3885|48874|7|14|166|53|72.11|134.12|40.23|0.00|2132.19|3821.83|7108.36|170.57|0.00|2132.19|2302.76|-1689.64| +2451193|47723|15394||838449|3885||||166|19|66.94||32.04||608.76|||30.43||||| +2451193|47723|9394|19860|838449|3885|48874|7|3|166|14|98.31|117.97|21.23|0.00|297.22|1376.34|1651.58|20.80|0.00|297.22|318.02|-1079.12| +2451193|47723|17894|19860|838449|3885|48874|7|236|166|34|23.07|45.90|14.22|261.07|483.48|784.38|1560.60|17.79|261.07|222.41|240.20|-561.97| +2451193|47723|728|19860|838449|3885|48874|7|5|166|81|32.41|63.52|3.81|0.00|308.61|2625.21|5145.12|21.60|0.00|308.61|330.21|-2316.60| +2452538|71871|10681|65395|1736234|3873|17512|4|137|167|6|37.01|62.17|14.92|0.00|89.52|222.06|373.02|0.00|0.00|89.52|89.52|-132.54| +2452538|71871|2485|65395|1736234|3873|17512|4|258|167|39|89.60|146.04|11.68|0.00|455.52|3494.40|5695.56|9.11|0.00|455.52|464.63|-3038.88| +2452538|71871|16149|65395|1736234|3873|17512|4|79|167|45|48.52|78.11|66.39|0.00|2987.55|2183.40|3514.95|29.87|0.00|2987.55|3017.42|804.15| +2452538|71871|15895|65395|1736234|3873|17512|4|85|167|53|73.48|135.20|29.74|0.00|1576.22|3894.44|7165.60|47.28|0.00|1576.22|1623.50|-2318.22| +2452538|71871|8610|65395|1736234|3873|17512|4|147|167|15|73.91|145.60|109.20|0.00|1638.00|1108.65|2184.00|114.66|0.00|1638.00|1752.66|529.35| +2452538|71871|2052|65395|1736234|3873|17512|4|141|167|97|50.28|92.51|28.67|2502.89|2780.99|4877.16|8973.47|19.46|2502.89|278.10|297.56|-4599.06| +2452538|71871|10117|65395|1736234|3873|17512|4|239|167|36|21.60|31.32|0.62|18.30|22.32|777.60|1127.52|0.04|18.30|4.02|4.06|-773.58| +2452538|71871|2239|65395|1736234|3873|17512|4|89|167|69|88.82|111.91|60.43|0.00|4169.67|6128.58|7721.79|41.69|0.00|4169.67|4211.36|-1958.91| +2452538|71871|2989|65395|1736234|3873|17512|4|205|167|53|95.07|148.30|130.50|0.00|6916.50|5038.71|7859.90|484.15|0.00|6916.50|7400.65|1877.79| +2452538|71871|2277|65395|1736234|3873|17512|4|288|167|74|74.07|82.95|12.44|0.00|920.56|5481.18|6138.30|9.20|0.00|920.56|929.76|-4560.62| +2452538|71871|5895|65395|1736234|3873|17512|4|42|167|84|33.31|34.97|4.54|186.86|381.36|2798.04|2937.48|0.00|186.86|194.50|194.50|-2603.54| +2452538|71871|2742|65395|1736234|3873|17512|4|117|167|86|43.96|63.74|55.45|572.24|4768.70|3780.56|5481.64|293.75|572.24|4196.46|4490.21|415.90| +2452538|71871|12192|65395|1736234|3873|17512|4|225|167|76|85.49|88.05|22.01|0.00|1672.76|6497.24|6691.80|33.45|0.00|1672.76|1706.21|-4824.48| +2452491|42908|16057|74161|51113|512|12664|2|204|168|41|13.17|20.01|3.20|0.00|131.20|539.97|820.41|3.93|0.00|131.20|135.13|-408.77| +2452491|42908|14581|74161|51113|512|12664|2|173|168|14|88.28|154.49|123.59|0.00|1730.26|1235.92|2162.86|0.00|0.00|1730.26|1730.26|494.34| +2452491|42908|12924|74161|51113|512|12664|2|199|168|18|40.22|80.44|16.08|0.00|289.44|723.96|1447.92|0.00|0.00|289.44|289.44|-434.52| +2452491|42908|8539|74161|51113|512|12664|2|218|168|46|89.08|120.25|52.91|0.00|2433.86|4097.68|5531.50|219.04|0.00|2433.86|2652.90|-1663.82| +2452491|42908|15691|74161|51113|512|12664|2|125|168|82|86.37|166.69|75.01|0.00|6150.82|7082.34|13668.58|369.04|0.00|6150.82|6519.86|-931.52| +2452491|42908|7086|74161|51113|512|12664|2|187|168|48|56.06|101.46|38.55|518.11|1850.40|2690.88|4870.08|0.00|518.11|1332.29|1332.29|-1358.59| +2452491|42908|10825|74161|51113|512|12664|2|56|168|98|10.08|17.84|13.55|0.00|1327.90|987.84|1748.32|119.51|0.00|1327.90|1447.41|340.06| +2452491|42908|15549|74161|51113|512|12664|2|218|168|98|77.17|98.77|34.56|0.00|3386.88|7562.66|9679.46|304.81|0.00|3386.88|3691.69|-4175.78| +2452491|42908|9549|74161|51113|512|12664|2|96|168|89|16.45|16.45|3.78|0.00|336.42|1464.05|1464.05|23.54|0.00|336.42|359.96|-1127.63| +2451140|47889|12625|68742|1817135|6681|23874|2|206|169|56|44.71|49.18|46.72|1648.28|2616.32|2503.76|2754.08|48.40|1648.28|968.04|1016.44|-1535.72| +2451140|47889|3091|68742|1817135|6681|23874|2|164|169|33|90.74|137.01|32.88|0.00|1085.04|2994.42|4521.33|75.95|0.00|1085.04|1160.99|-1909.38| +2451140|47889|422|68742|1817135|6681|23874|2|49|169|75|82.99|121.99|79.29|4757.40|5946.75|6224.25|9149.25|107.04|4757.40|1189.35|1296.39|-5034.90| +2451140|47889|5029|68742|1817135|6681|23874|2|70|169|16|32.33|49.78|31.85|0.00|509.60|517.28|796.48|20.38|0.00|509.60|529.98|-7.68| +2451140|47889|17704|68742|1817135|6681|23874|2|281|169|72|79.17|101.33|33.43|0.00|2406.96|5700.24|7295.76|48.13|0.00|2406.96|2455.09|-3293.28| +2451140|47889|11234|68742|1817135|6681|23874|2|50|169|68|40.72|63.93|22.37|0.00|1521.16|2768.96|4347.24|60.84|0.00|1521.16|1582.00|-1247.80| +2451140|47889|17482|68742|1817135|6681|23874|2|175|169|22|94.37|185.90|109.68|0.00|2412.96|2076.14|4089.80|193.03|0.00|2412.96|2605.99|336.82| +2451140|47889|8384|68742|1817135|6681|23874|2|32|169|15|38.92|68.88|21.35|0.00|320.25|583.80|1033.20|19.21|0.00|320.25|339.46|-263.55| +2451140|47889|6565|68742|1817135|6681|23874|2|55|169|47|43.86|65.35|20.91|0.00|982.77|2061.42|3071.45|49.13|0.00|982.77|1031.90|-1078.65| +2451140|47889|5059|68742|1817135|6681|23874|2|81|169|61|88.31|90.95|17.28|0.00|1054.08|5386.91|5547.95|42.16|0.00|1054.08|1096.24|-4332.83| +2451140|47889|4687|68742|1817135|6681|23874|2|296|169|73|41.03|58.67|27.57|1831.47|2012.61|2995.19|4282.91|7.24|1831.47|181.14|188.38|-2814.05| +2451140|47889|9310|68742|1817135|6681|23874|2|134|169|59|78.41|123.88|96.62|0.00|5700.58|4626.19|7308.92|513.05|0.00|5700.58|6213.63|1074.39| +2451140|47889|13198|68742|1817135|6681|23874|2|5|169|27|69.50|70.19|48.43|0.00|1307.61|1876.50|1895.13|104.60|0.00|1307.61|1412.21|-568.89| +2451140|47889|10132|68742|1817135|6681|23874|2|154|169|9|47.23|79.81|34.31|0.00|308.79|425.07|718.29|9.26|0.00|308.79|318.05|-116.28| +2450935|53371|7738|20701|764989|2430|46318|2|116|170|94|46.93|48.80|8.78|0.00|825.32|4411.42|4587.20|49.51|0.00|825.32|874.83|-3586.10| +2450935|53371|8600|20701|764989|2430|46318|2|129|170|72|93.97|124.98|41.24|0.00|2969.28|6765.84|8998.56|89.07|0.00|2969.28|3058.35|-3796.56| +|53371|8032|20701||2430|||60|170|60|53.82||||1385.40|3229.20||||1385.40||| +2450935|53371|16957|20701|764989|2430|46318|2|193|170|86|48.66|64.71|52.41|0.00|4507.26|4184.76|5565.06|135.21|0.00|4507.26|4642.47|322.50| +2450935|53371|6370|20701|764989|2430|46318|2|184|170|55|61.11|114.88|2.29|0.00|125.95|3361.05|6318.40|5.03|0.00|125.95|130.98|-3235.10| +2450935|53371|15298|20701|764989|2430|46318|2|207|170|13|78.85|91.46|91.46|0.00|1188.98|1025.05|1188.98|107.00|0.00|1188.98|1295.98|163.93| +2450935|53371|13837|20701|764989|2430|46318|2|214|170|24|8.21|15.51|7.75|0.00|186.00|197.04|372.24|1.86|0.00|186.00|187.86|-11.04| +2450935|53371|14536|20701|764989|2430|46318|2|239|170|59|64.21|77.05|69.34|0.00|4091.06|3788.39|4545.95|0.00|0.00|4091.06|4091.06|302.67| +2450935|53371|646|20701|764989|2430|46318|2|95|170|65|65.30|120.80|97.84|0.00|6359.60|4244.50|7852.00|508.76|0.00|6359.60|6868.36|2115.10| +2450935|53371|13592|20701|764989|2430|46318|2|170|170|7|15.09|15.24|13.10|0.00|91.70|105.63|106.68|0.00|0.00|91.70|91.70|-13.93| +2450935|53371|1039|20701|764989|2430|46318|2|281|170|75|63.18|69.49|6.94|0.00|520.50|4738.50|5211.75|36.43|0.00|520.50|556.93|-4218.00| +2450935|53371|14756|20701|764989|2430|46318|2|23|170|20|38.26|39.79|34.21|0.00|684.20|765.20|795.80|41.05|0.00|684.20|725.25|-81.00| +2450935|53371|1270|20701|764989|2430|46318|2|121|170|81|62.81|64.69|27.16|0.00|2199.96|5087.61|5239.89|65.99|0.00|2199.96|2265.95|-2887.65| +2450935|53371|14698|20701|764989|2430|46318|2|158|170|53|59.65|90.07|9.00|0.00|477.00|3161.45|4773.71|0.00|0.00|477.00|477.00|-2684.45| +2451782|38361|6059|72383|492488|2193|15051|2|86|171|35|39.84|56.57|50.34|0.00|1761.90|1394.40|1979.95|70.47|0.00|1761.90|1832.37|367.50| +2451782|38361|12551|72383|492488|2193|15051|2|248|171|96|86.26|156.99|125.59|0.00|12056.64|8280.96|15071.04|602.83|0.00|12056.64|12659.47|3775.68| +2451782|38361|2393|72383|492488|2193|15051|2|200|171|54|3.42|4.65|2.13|52.90|115.02|184.68|251.10|1.86|52.90|62.12|63.98|-122.56| +2451782|38361|15002|72383|492488|2193|15051|2|136|171|79|29.68|41.55|32.40|0.00|2559.60|2344.72|3282.45|179.17|0.00|2559.60|2738.77|214.88| +2451782|38361|6134|72383|492488|2193|15051|2|30|171|66|9.78|11.44|5.83|0.00|384.78|645.48|755.04|15.39|0.00|384.78|400.17|-260.70| +2451782|38361|3613|72383|492488|2193|15051|2|12|171|54|91.84|139.59|131.21|0.00|7085.34|4959.36|7537.86|70.85|0.00|7085.34|7156.19|2125.98| +2451782|38361|9740|72383|492488|2193|15051|2|162|171|56|90.38|137.37|59.06|0.00|3307.36|5061.28|7692.72|264.58|0.00|3307.36|3571.94|-1753.92| +2451782|38361|619|72383|492488|2193|15051|2|156|171|20|60.13|117.25|56.28|157.58|1125.60|1202.60|2345.00|9.68|157.58|968.02|977.70|-234.58| +2451782|38361|1436|72383|492488|2193|15051|2|229|171|88|76.72|140.39|77.21|0.00|6794.48|6751.36|12354.32|0.00|0.00|6794.48|6794.48|43.12| +2452262|33044|17537|73253|301179|7152|36678|4|186|172|76|52.94|94.76|24.63|0.00|1871.88|4023.44|7201.76|93.59|0.00|1871.88|1965.47|-2151.56| +2452262|33044|7445|73253|301179|7152|36678|4|161|172|72|16.37|27.33|23.23|0.00|1672.56|1178.64|1967.76|66.90|0.00|1672.56|1739.46|493.92| +2452262|33044|11573|73253|301179|7152|36678|4|287|172|29|18.38|29.40|20.58|0.00|596.82|533.02|852.60|23.87|0.00|596.82|620.69|63.80| +2452262|33044|1953|73253|301179|7152|36678|4|4|172|23|96.70|157.62|45.70|0.00|1051.10|2224.10|3625.26|0.00|0.00|1051.10|1051.10|-1173.00| +2452262|33044|1971|73253|301179|7152|36678|4|89|172|44|92.94|113.38|9.07|0.00|399.08|4089.36|4988.72|27.93|0.00|399.08|427.01|-3690.28| +2452262|33044|4023|||7152||||172|||89.10||0.00||3048.24|3474.90||0.00|1111.89||-1936.35| +2452262|33044|9101|73253|301179|7152|36678|4|87|172|18|99.11|113.97|109.41|0.00|1969.38|1783.98|2051.46|137.85|0.00|1969.38|2107.23|185.40| +2452262|33044|14037|73253|301179|7152|36678|4|83|172|51|70.71|77.78|67.66|0.00|3450.66|3606.21|3966.78|69.01|0.00|3450.66|3519.67|-155.55| +2452262|33044|16323|73253|301179|7152|36678|4|261|172|2|87.04|128.81|85.01|83.30|170.02|174.08|257.62|0.00|83.30|86.72|86.72|-87.36| +2452262||14027||||36678||9|172|||156.81|18.81|0.00||3818.75|7370.07||0.00|884.07|945.95|-2934.68| +2452262|33044|667|73253|301179|7152|36678|4|296|172|2|10.19|11.51|6.56|0.00|13.12|20.38|23.02|0.13|0.00|13.12|13.25|-7.26| +2452262|33044|6321|73253|301179|7152|36678|4|52|172|44|23.51|45.60|3.19|26.66|140.36|1034.44|2006.40|2.27|26.66|113.70|115.97|-920.74| +2452262|33044|2295|73253|301179|7152|36678|4|10|172|38|34.72|50.69|43.59|0.00|1656.42|1319.36|1926.22|0.00|0.00|1656.42|1656.42|337.06| +2452262|33044|12095|73253|301179|7152|36678|4|256|172|99|41.00|56.99|26.78|0.00|2651.22|4059.00|5642.01|238.60|0.00|2651.22|2889.82|-1407.78| +2452262|33044|8873|73253|301179|7152|36678|4|288|172|13|27.31|50.52|44.96|0.00|584.48|355.03|656.76|23.37|0.00|584.48|607.85|229.45| +2452262|33044|16629|73253|301179|7152|36678|4|216|172|28|65.92|105.47|65.39|0.00|1830.92|1845.76|2953.16|164.78|0.00|1830.92|1995.70|-14.84| +2451059|42142|15772|97090|1738080|5023|36413|1|294|173|32|86.98|162.65|4.87|0.00|155.84|2783.36|5204.80|0.00|0.00|155.84|155.84|-2627.52| +2451059|42142|2696|97090|1738080|5023|36413|1|227|173|49|68.63|126.27|36.61|0.00|1793.89|3362.87|6187.23|107.63|0.00|1793.89|1901.52|-1568.98| +2451059|42142|268|97090|1738080|5023|36413|1|81|173|91|98.27|178.85|51.86|4530.48|4719.26|8942.57|16275.35|11.32|4530.48|188.78|200.10|-8753.79| +|42142|6577|97090|||36413|||173|26|53.82|103.33|||1638.78|1399.32||0.00||1638.78||239.46| +2451059|42142|13555|97090|1738080|5023|36413|1|195|173|99|10.70|21.18|19.27|0.00|1907.73|1059.30|2096.82|76.30|0.00|1907.73|1984.03|848.43| +2451059|42142|12757|97090|1738080|5023|36413|1|18|173|90|90.23|110.08|40.72|0.00|3664.80|8120.70|9907.20|36.64|0.00|3664.80|3701.44|-4455.90| +2451059|42142|2326|97090|1738080|5023|36413|1|242|173|86|90.42|179.03|139.64|0.00|12009.04|7776.12|15396.58|120.09|0.00|12009.04|12129.13|4232.92| +2451059|42142|12604|97090|1738080|5023|36413|1|296|173|40|74.41|104.91|19.93|0.00|797.20|2976.40|4196.40|47.83|0.00|797.20|845.03|-2179.20| +2451059|42142|7405|97090|1738080|5023|36413|1|13|173|32|31.75|60.96|3.04|0.00|97.28|1016.00|1950.72|0.00|0.00|97.28|97.28|-918.72| +2451059|42142|5738|97090|1738080|5023|36413|1|96|173|36|63.54|67.98|48.26|920.80|1737.36|2287.44|2447.28|16.33|920.80|816.56|832.89|-1470.88| +2451059||7960|||5023|||220|173||77.52|93.02|62.32|0.00||||109.06|0.00|||| +2451059|42142|5593|97090|1738080|5023|36413|1|216|173|74|3.64|5.31|0.69|0.00|51.06|269.36|392.94|2.55|0.00|51.06|53.61|-218.30| +2451059|42142|16828|97090|1738080|5023|36413|1|111|173|58|5.96|6.13|4.35|0.00|252.30|345.68|355.54|10.09|0.00|252.30|262.39|-93.38| +2451059|42142|16264|97090|1738080|5023|36413|1|10|173|95|93.59|175.94|63.33|0.00|6016.35|8891.05|16714.30|0.00|0.00|6016.35|6016.35|-2874.70| +2451059|42142|2330|97090|1738080|5023|36413|1|25|173|7|9.05|11.13|2.44|0.00|17.08|63.35|77.91|0.17|0.00|17.08|17.25|-46.27| +2451059|42142|4096|97090|1738080|5023|36413|1|71|173|47|91.96|113.11|49.76|0.00|2338.72|4322.12|5316.17|187.09|0.00|2338.72|2525.81|-1983.40| +2450888|64757|16340|80181|180192|6603|28967|10|157|174|70|61.41|71.84|27.29|0.00|1910.30|4298.70|5028.80|95.51|0.00|1910.30|2005.81|-2388.40| +2450888|64757|278|80181|180192|6603|28967|10|146|174|27|26.28|29.69|10.68|0.00|288.36|709.56|801.63|11.53|0.00|288.36|299.89|-421.20| +2450888|64757|15338|80181|180192|6603|28967|10|258|174|67|25.01|32.01|7.36|0.00|493.12|1675.67|2144.67|4.93|0.00|493.12|498.05|-1182.55| +2450888|64757|1540|80181|180192|6603|28967|10|97|174|51|2.78|3.41|0.17|0.00|8.67|141.78|173.91|0.34|0.00|8.67|9.01|-133.11| +2450888|64757|12577|80181|180192|6603|28967|10|89|174|41|45.06|87.86|2.63|0.00|107.83|1847.46|3602.26|3.23|0.00|107.83|111.06|-1739.63| +2450888|64757|2114|80181|180192|6603|28967|10|147|174|83|15.10|15.40|1.54|0.00|127.82|1253.30|1278.20|5.11|0.00|127.82|132.93|-1125.48| +2450888|64757|7219|80181|180192|6603|28967|10|113|174|59|8.13|13.17|13.17|0.00|777.03|479.67|777.03|46.62|0.00|777.03|823.65|297.36| +2450888|64757|12067|80181|180192|6603|28967|10|258|174|97|56.01|89.05|80.14|0.00|7773.58|5432.97|8637.85|388.67|0.00|7773.58|8162.25|2340.61| +2450888|64757|830|80181|180192|6603|28967|10|299|174|54|73.31|120.22|82.95|2374.02|4479.30|3958.74|6491.88|21.05|2374.02|2105.28|2126.33|-1853.46| +2450888|64757|17302|80181|180192|6603|28967|10|211|174|24|41.31|64.85|42.15|0.00|1011.60|991.44|1556.40|20.23|0.00|1011.60|1031.83|20.16| +2450888|64757|362|80181|180192|6603|28967|10|126|174|39|17.01|30.78|20.93|114.27|816.27|663.39|1200.42|35.10|114.27|702.00|737.10|38.61| +2450888|64757|9344|80181|180192|6603|28967|10|269|174|45|26.38|52.76|50.64|0.00|2278.80|1187.10|2374.20|205.09|0.00|2278.80|2483.89|1091.70| +2450888|64757|12013|80181|180192|6603|28967|10|48|174|66|68.19|113.87|108.17|0.00|7139.22|4500.54|7515.42|428.35|0.00|7139.22|7567.57|2638.68| +2450888|64757|6550|80181|180192|6603|28967|10|156|174|39|85.79|103.80|63.31|1876.50|2469.09|3345.81|4048.20|23.70|1876.50|592.59|616.29|-2753.22| +2450888|64757|16616|80181|180192|6603|28967|10|289|174|96|62.99|93.85|89.15|0.00|8558.40|6047.04|9009.60|427.92|0.00|8558.40|8986.32|2511.36| +2450888|64757|11414|80181|180192|6603|28967|10|72|174|76|86.23|118.99|0.00|0.00|0.00|6553.48|9043.24|0.00|0.00|0.00|0.00|-6553.48| +2452637|38419|16153|68727|1027990|5653|8163|7|56|175|72|47.73|73.50|12.49|0.00|899.28|3436.56|5292.00|62.94|0.00|899.28|962.22|-2537.28| +2452637|38419|10321|68727|1027990|5653|8163|7|116|175|70|37.53|38.65|27.05|1041.42|1893.50|2627.10|2705.50|42.60|1041.42|852.08|894.68|-1775.02| +2452637|38419|2400|68727|1027990|5653|8163|7|157|175|70|54.51|64.86|25.94|0.00|1815.80|3815.70|4540.20|36.31|0.00|1815.80|1852.11|-1999.90| +2452637|38419|4803|68727|1027990|5653|8163|7|215|175|9|15.62|23.43|1.17|0.00|10.53|140.58|210.87|0.84|0.00|10.53|11.37|-130.05| +2452637|38419|4794|68727|1027990|5653|8163|7|151|175|21|94.15|96.03|29.76|456.22|624.96|1977.15|2016.63|5.06|456.22|168.74|173.80|-1808.41| +2452637|38419|1813|68727|1027990|5653|8163|7|269|175|93|28.86|41.26|16.09|0.00|1496.37|2683.98|3837.18|74.81|0.00|1496.37|1571.18|-1187.61| +2452637|38419|16843|68727|1027990|5653|8163|7|10|175|60|93.79|163.19|52.22|0.00|3133.20|5627.40|9791.40|281.98|0.00|3133.20|3415.18|-2494.20| +2452637|38419|13761|68727|1027990|5653|8163|7|7|175|7|96.47|102.25|84.86|0.00|594.02|675.29|715.75|41.58|0.00|594.02|635.60|-81.27| +2452637|38419|2301|68727|1027990|5653|8163|7|252|175|41|88.99|147.72|127.03|2291.62|5208.23|3648.59|6056.52|116.66|2291.62|2916.61|3033.27|-731.98| +2452637|38419|16188|68727|1027990|5653|8163|7|57|175|13|7.29|11.37|11.02|0.00|143.26|94.77|147.81|10.02|0.00|143.26|153.28|48.49| +2452637|38419|7674|68727|1027990|5653|8163|7|149|175|1|29.31|53.63|1.60|0.00|1.60|29.31|53.63|0.08|0.00|1.60|1.68|-27.71| +2452148|43334|16453|5689|1145580|671|44162|4|110|176|12|75.41|110.09|12.10|63.88|145.20|904.92|1321.08|6.50|63.88|81.32|87.82|-823.60| +2452148|43334|17595|5689|1145580|671|44162|4|271|176|93|68.38|90.94|24.55|0.00|2283.15|6359.34|8457.42|0.00|0.00|2283.15|2283.15|-4076.19| +2452148|43334|13217|5689|1145580|671|44162|4|111|176|97|76.61|119.51|43.02|0.00|4172.94|7431.17|11592.47|41.72|0.00|4172.94|4214.66|-3258.23| +2452148|43334|2909|5689|1145580|671|44162|4|25|176|69|69.76|112.31|31.44|0.00|2169.36|4813.44|7749.39|151.85|0.00|2169.36|2321.21|-2644.08| +2452148|43334|6441|5689|1145580|671|44162|4|227|176|9|50.63|71.89||0.00|498.15|455.67||0.00|0.00|498.15|498.15|| +2452148|43334|6795|5689|1145580|671|44162|4|261|176|12|21.71|33.65|23.21|0.00|278.52|260.52|403.80|16.71|0.00|278.52|295.23|18.00| +2452148|43334|17713|5689|1145580|671|44162|4|28|176|83|97.71|125.06|70.03|0.00|5812.49|8109.93|10379.98|348.74|0.00|5812.49|6161.23|-2297.44| +2452148|43334|795|5689|1145580|671|44162|4|80|176|72|71.32|93.42|64.45|0.00|4640.40|5135.04|6726.24|324.82|0.00|4640.40|4965.22|-494.64| +2452148|43334|1129|5689|1145580|671|44162|4|32|176|65|3.45|5.55|1.16|0.00|75.40|224.25|360.75|6.03|0.00|75.40|81.43|-148.85| +2452148|43334|17527|5689|1145580|671|44162|4|61|176|81|55.02|93.53|12.15|580.64|984.15|4456.62|7575.93|32.28|580.64|403.51|435.79|-4053.11| +2452148|43334|6473|5689|1145580|671|44162|4|278|176|91|75.30|124.99|29.99|0.00|2729.09|6852.30|11374.09|27.29|0.00|2729.09|2756.38|-4123.21| +2452148|43334|15489|5689|1145580|671|44162|4|191|176|50|62.07|115.45|71.57|2075.53|3578.50|3103.50|5772.50|90.17|2075.53|1502.97|1593.14|-1600.53| +2452626|39149|990|3824|836462|3500|26538|2|159|177|82|3.09|5.96|4.52|0.00|370.64|253.38|488.72|0.00|0.00|370.64|370.64|117.26| +2452626|39149|10293|3824|836462|3500|26538|2|130|177|22|61.13|63.57|5.72|0.00|125.84|1344.86|1398.54|7.55|0.00|125.84|133.39|-1219.02| +2452626|39149|13434|3824|836462|3500|26538|2|226|177|51|20.69|34.34|29.18|0.00|1488.18|1055.19|1751.34|74.40|0.00|1488.18|1562.58|432.99| +2452626|39149|17346|3824|836462|3500|26538|2|273|177|2|56.26|88.89|82.66|0.00|165.32|112.52|177.78|4.95|0.00|165.32|170.27|52.80| +2452626|39149|17076|3824|836462|3500|26538|2|151|177|66|68.18|76.36|1.52|0.00|100.32|4499.88|5039.76|8.02|0.00|100.32|108.34|-4399.56| +2452626|39149|12717|3824|836462|3500|26538|2|156|177|90|75.15|75.15|57.11|0.00|5139.90|6763.50|6763.50|308.39|0.00|5139.90|5448.29|-1623.60| +2452626|39149|1101|3824|836462|3500|26538|2|75|177|67|33.91|47.13|18.38|332.49|1231.46|2271.97|3157.71|44.94|332.49|898.97|943.91|-1373.00| +2452626|39149|2185|3824|836462|3500|26538|2|259|177|6|68.11|70.83|68.70|0.00|412.20|408.66|424.98|16.48|0.00|412.20|428.68|3.54| +2452626|39149|16161|3824|836462|3500|26538|2|10|177|66|42.18|69.59|20.87|0.00|1377.42|2783.88|4592.94|27.54|0.00|1377.42|1404.96|-1406.46| +2452626|39149|16866|3824|836462|3500|26538|2|242|177|4|15.99|28.30|5.09|0.00|20.36|63.96|113.20|1.83|0.00|20.36|22.19|-43.60| +2452516|66457|17989|73190|802324|6305|9086|7|85|178|68|10.66|11.29|10.49|0.00|713.32|724.88|767.72|42.79|0.00|713.32|756.11|-11.56| +2452516|66457|13569|73190|802324|6305|9086|7|82|178|56|55.00|74.25|49.00|0.00|2744.00|3080.00|4158.00|0.00|0.00|2744.00|2744.00|-336.00| +2452516|66457|6829|73190|802324|6305|9086|7|282|178|2|70.89|79.39|61.13|0.00|122.26|141.78|158.78|3.66|0.00|122.26|125.92|-19.52| +2452516|66457|2184|73190|802324|6305|9086|7|161|178|78|29.10|39.57|28.88|135.15|2252.64|2269.80|3086.46|148.22|135.15|2117.49|2265.71|-152.31| +2452516|66457|1087|73190|802324|6305|9086|7|105|178|1|37.40|37.40|37.40|36.65|37.40|37.40|37.40|0.04|36.65|0.75|0.79|-36.65| +2452516|66457|11187|73190|802324|6305|9086|7|167|178|65|49.71|93.95|55.43|0.00|3602.95|3231.15|6106.75|108.08|0.00|3602.95|3711.03|371.80| +2452516|66457|14281|73190|802324|6305|9086|7|112|178|22|68.26|109.21|36.03|0.00|792.66|1501.72|2402.62|31.70|0.00|792.66|824.36|-709.06| +2452516|66457|7081|73190|802324|6305|9086|7|194|178|16|54.83|60.31|36.78|0.00|588.48|877.28|964.96|52.96|0.00|588.48|641.44|-288.80| +2452559|63108|717|90460|393558|656||||179||76.59|95.73|6.70||556.10||7945.59|16.68||556.10||-5800.87| +2452559|63108|11605|90460|393558|656|44201|4|111|179|86|6.82|10.29|0.00|0.00|0.00|586.52|884.94|0.00|0.00|0.00|0.00|-586.52| +2452559|63108|85|90460|393558|656|44201|4|42|179|60|97.55|155.10|105.46|4745.70|6327.60|5853.00|9306.00|31.63|4745.70|1581.90|1613.53|-4271.10| +|63108|7767||393558|||4||179||||103.68|||||219.80|||5714.84|| +2452559|63108|9606|90460|393558|656|44201|4|241|179|67|81.48|137.70|96.39|0.00|6458.13|5459.16|9225.90|193.74|0.00|6458.13|6651.87|998.97| +||4003|90460|393558|656||4||179||62.45||35.79|1112.35|||5406.44||1112.35|||-3085.19| +2452559|63108|14484|90460|393558|656|44201|4|152|179|96|70.71|137.17|15.08|0.00|1447.68|6788.16|13168.32|72.38|0.00|1447.68|1520.06|-5340.48| +2452559|63108|17472|90460|393558|656|44201|4|97|179|57|22.85|25.36|17.75|0.00|1011.75|1302.45|1445.52|70.82|0.00|1011.75|1082.57|-290.70| +|63108|5583||393558||||7|179||92.99|||0.00||5951.36|11664.64||0.00|232.96|249.26|| +2452559|63108|42|90460|393558|656|44201|4|273|179|17|48.49|89.22|8.92|71.27|151.64|824.33|1516.74|2.41|71.27|80.37|82.78|-743.96| +2452524|65450|8592|96309|457075|5488|27854|7|221|180|81|60.76|80.20|4.81|101.29|389.61|4921.56|6496.20|25.94|101.29|288.32|314.26|-4633.24| +2452524|65450|14697|96309|457075|5488|27854|7|15|180|82|52.10|79.71|75.72|0.00|6209.04|4272.20|6536.22|124.18|0.00|6209.04|6333.22|1936.84| +2452524|65450|600|96309|457075|5488|27854|7|169|180|96|44.47|78.26|16.43|0.00|1577.28|4269.12|7512.96|110.40|0.00|1577.28|1687.68|-2691.84| +2452524|65450|7782|96309|457075|5488|27854|7|174|180|43|78.74|132.28|68.78|2129.42|2957.54|3385.82|5688.04|49.68|2129.42|828.12|877.80|-2557.70| +2452524|65450|7509|96309|457075|5488|27854|7|148|180|27|26.02|28.62|3.14|0.00|84.78|702.54|772.74|4.23|0.00|84.78|89.01|-617.76| +2452524|65450|1941|96309|457075|5488|27854|7|274|180|24|35.87|36.22|16.66|0.00|399.84|860.88|869.28|35.98|0.00|399.84|435.82|-461.04| +2452524|65450|11325|96309|457075|5488|27854|7|127|180|73|20.47|32.95|6.26|0.00|456.98|1494.31|2405.35|13.70|0.00|456.98|470.68|-1037.33| +2452524|65450|14599|96309|457075|5488|27854|7|21|180|60|67.21|112.24|29.18|402.68|1750.80|4032.60|6734.40|80.88|402.68|1348.12|1429.00|-2684.48| +2452524|65450|11592|96309|457075|5488|27854|7|233|180|60|51.60|76.88|16.91|0.00|1014.60|3096.00|4612.80|91.31|0.00|1014.60|1105.91|-2081.40| +2452524|65450|16794|96309|457075|5488|27854|7|60|180|94|90.57|145.81|100.60|0.00|9456.40|8513.58|13706.14|189.12|0.00|9456.40|9645.52|942.82| +2452524|65450|6468|96309|457075|5488|27854|7|73|180|43|9.10|9.91|3.86|0.00|165.98|391.30|426.13|4.97|0.00|165.98|170.95|-225.32| +2452524|65450|15753|96309|457075|5488|27854|7|16|180|16|8.85|14.77|7.08|0.00|113.28|141.60|236.32|5.66|0.00|113.28|118.94|-28.32| +2452383|39523|5349|65464|1607367|3670|3973|10|130|181|2|32.76|39.63|37.25|50.66|74.50|65.52|79.26|1.66|50.66|23.84|25.50|-41.68| +2452383|39523|10872|65464|1607367|3670|3973|10|275|181|17|84.68|102.46|65.57|0.00|1114.69|1439.56|1741.82|0.00|0.00|1114.69|1114.69|-324.87| +2452383|39523|4182|65464|1607367|3670|3973|10|111|181|78|29.33|50.15|45.63|0.00|3559.14|2287.74|3911.70|177.95|0.00|3559.14|3737.09|1271.40| +2452383|39523|2970|65464|1607367|3670|3973|10|69|181|40|73.42|140.96|95.85|0.00|3834.00|2936.80|5638.40|191.70|0.00|3834.00|4025.70|897.20| +2452383|39523|9519|65464|1607367|3670|3973|10|26|181|58|32.51|56.89|29.01|0.00|1682.58|1885.58|3299.62|100.95|0.00|1682.58|1783.53|-203.00| +2452383|39523|9012|65464|1607367|3670|3973|10|118|181|74|58.90|102.48|94.28|0.00|6976.72|4358.60|7583.52|488.37|0.00|6976.72|7465.09|2618.12| +2452383|39523|8625|65464|1607367|||||181|76||||0.00|2965.52|4512.88||29.65|0.00|||| +2452383|39523|10536|65464|1607367|3670|3973|10|194|181|17|77.44|85.95|53.28|0.00|905.76|1316.48|1461.15|72.46|0.00|905.76|978.22|-410.72| +2452383|39523|6001|65464|1607367|3670|3973|10|144|181|34|1.69|1.77|1.25|0.00|42.50|57.46|60.18|2.97|0.00|42.50|45.47|-14.96| +2452383|39523|11569|65464|1607367|3670|3973|10|142|181|57|36.18|55.71|20.61|0.00|1174.77|2062.26|3175.47|58.73|0.00|1174.77|1233.50|-887.49| +2452383|39523|14731|65464|1607367|3670|3973|10|291|181|53|17.31|20.42|3.47|0.00|183.91|917.43|1082.26|3.67|0.00|183.91|187.58|-733.52| +2452383|39523|13062|65464|1607367|3670|3973|10|110|181|56|21.09|39.64|27.74|0.00|1553.44|1181.04|2219.84|108.74|0.00|1553.44|1662.18|372.40| +2452277|65243|14655|62189|531189|3836|11236|1|73|182|65|41.01|65.61|0.65|0.00|42.25|2665.65|4264.65|0.00|0.00|42.25|42.25|-2623.40| +2452277|65243|273|62189|531189|3836|11236|1|84|182|78|66.70|102.05|45.92|3510.12|3581.76|5202.60|7959.90|3.58|3510.12|71.64|75.22|-5130.96| +2452277|65243|9432|62189|531189|3836|11236|1|230|182|21|68.29|124.28|119.30|0.00|2505.30|1434.09|2609.88|0.00|0.00|2505.30|2505.30|1071.21| +2452277|65243|15669|62189|531189|3836|11236|1|25|182|49|64.44|76.68|29.90|0.00|1465.10|3157.56|3757.32|117.20|0.00|1465.10|1582.30|-1692.46| +2452277|65243|13536|62189|531189|3836|11236|1|209|182|46|87.46|168.79|118.15|0.00|5434.90|4023.16|7764.34|380.44|0.00|5434.90|5815.34|1411.74| +2452277|65243|3099|62189|531189|3836|11236|1|207|182|43|92.99|97.63|21.47|0.00|923.21|3998.57|4198.09|83.08|0.00|923.21|1006.29|-3075.36| +2452277|65243|2364|62189|531189|3836|11236|1|214|182|44|12.91|23.49|19.26|0.00|847.44|568.04|1033.56|42.37|0.00|847.44|889.81|279.40| +2452277|65243|9000|62189|531189|3836|11236|1|132|182|45|44.26|71.70|50.90|481.00|2290.50|1991.70|3226.50|0.00|481.00|1809.50|1809.50|-182.20| +2452277|65243|15594|62189|531189|3836|11236|1|24|182|3|71.03|104.41|44.89|0.00|134.67|213.09|313.23|10.77|0.00|134.67|145.44|-78.42| +2452277|65243|5677|62189|531189|3836|11236|1|143|182|12|64.55|72.94|64.91|0.00|778.92|774.60|875.28|23.36|0.00|778.92|802.28|4.32| +2452164|32062|15371|75420|1621264|1508|6552|4|295|183|37|94.08|99.72|36.89|0.00|1364.93|3480.96|3689.64|54.59|0.00|1364.93|1419.52|-2116.03| +2452164|32062|12217|75420|1621264|1508|6552|4|77|183|10|56.80|112.46|67.47|0.00|674.70|568.00|1124.60|60.72|0.00|674.70|735.42|106.70| +2452164|32062|9759|75420|1621264|1508|6552|4|148|183|19|82.78|84.43|16.88|0.00|320.72|1572.82|1604.17|25.65|0.00|320.72|346.37|-1252.10| +2452164|32062|11239|75420|1621264|1508|6552|4|50|183|79|54.74|55.28|2.21|0.00|174.59|4324.46|4367.12|5.23|0.00|174.59|179.82|-4149.87| +2452164|32062|17059|75420|1621264|1508|6552|4|141|183|43|16.30|31.13|10.89|0.00|468.27|700.90|1338.59|14.04|0.00|468.27|482.31|-232.63| +2452164|32062|221|75420|1621264|1508|6552|4|145|183|36|41.97|74.70|14.94|0.00|537.84|1510.92|2689.20|10.75|0.00|537.84|548.59|-973.08| +|32062|13955|||||4||183|49|||||178.85|649.25|1278.90||||186.00|-470.40| +2452164|32062|5367|75420|1621264|1508|6552|4|192|183|98|47.11|62.18|3.73|0.00|365.54|4616.78|6093.64|14.62|0.00|365.54|380.16|-4251.24| +2452164|32062|15677|75420|1621264|1508|6552|4|134|183|45|95.08|184.45|18.44|0.00|829.80|4278.60|8300.25|41.49|0.00|829.80|871.29|-3448.80| +2452164|32062|11763|75420|1621264|1508|6552|4|23|183|59|59.24|65.16|3.25|0.00|191.75|3495.16|3844.44|9.58|0.00|191.75|201.33|-3303.41| +2452164|32062|2015|75420|1621264|1508|6552|4|151|183|53|26.04|39.32|7.86|0.00|416.58|1380.12|2083.96|12.49|0.00|416.58|429.07|-963.54| +2452164|32062|8091|75420|1621264|1508|6552|4|261|183|56|29.31|50.41|15.12|364.08|846.72|1641.36|2822.96|33.78|364.08|482.64|516.42|-1158.72| +2452164|32062|13189|75420|1621264|1508|6552|4|193|183|82|60.48|95.55|18.15|0.00|1488.30|4959.36|7835.10|14.88|0.00|1488.30|1503.18|-3471.06| +2451481|67202|4180|56284|293452|1820|32541|8|69|184|24|40.34|70.19|35.09|0.00|842.16|968.16|1684.56|75.79|0.00|842.16|917.95|-126.00| +2451481|67202|2968|56284|293452|1820|32541|8|156|184|31|14.96|19.29|2.89|0.00|89.59|463.76|597.99|6.27|0.00|89.59|95.86|-374.17| +2451481|67202|9518|56284|293452|1820|32541|8|202|184|2|75.64|103.62|88.07|149.71|176.14|151.28|207.24|1.05|149.71|26.43|27.48|-124.85| +||9010||293452|1820|32541|8|210|184||||0.00||0.00|||0.00||||| +2451481|67202|8624|56284|||32541|8|198|184||11.44|22.53|9.68||609.84|720.72||12.19||609.84|622.03|| +2451481|67202|10534|56284|293452|1820|32541|8|84|184|98|17.19|27.16|27.16|0.00|2661.68|1684.62|2661.68|239.55|0.00|2661.68|2901.23|977.06| +2451481|67202|6001|56284|293452|1820|32541|8|282|184|99|40.75|60.31|49.45|0.00|4895.55|4034.25|5970.69|391.64|0.00|4895.55|5287.19|861.30| +2451481|67202|11569|56284|293452|1820|32541|8|69|184|76|79.83|134.11|97.90|0.00|7440.40|6067.08|10192.36|595.23|0.00|7440.40|8035.63|1373.32| +2451481|67202|14731|56284|293452|1820|32541|8|292|184|1|52.30|78.45|53.34|0.00|53.34|52.30|78.45|0.53|0.00|53.34|53.87|1.04| +2451481|67202|13060|56284|293452|1820|32541|8|122|184|61|91.34|114.17|61.65|0.00|3760.65|5571.74|6964.37|188.03|0.00|3760.65|3948.68|-1811.09| +2451871|65213|4640|22873|316198|4370|40142|10|234|185|41|75.97|144.34|64.95|798.88|2662.95|3114.77|5917.94|111.84|798.88|1864.07|1975.91|-1250.70| +2451871||17933|22873|316198|4370|40142|||185||38.68||54.37||3044.72|||||3044.72||| +2451871|65213|17846|22873|316198|4370|40142|10|291|185|55|94.56|105.90|48.71|133.95|2679.05|5200.80|5824.50|229.05|133.95|2545.10|2774.15|-2655.70| +2451871|65213|13634|22873|316198|4370|40142|10|90|185|31|74.36|75.10|19.52|0.00|605.12|2305.16|2328.10|30.25|0.00|605.12|635.37|-1700.04| +2451871|65213|379|22873|316198|4370|40142|10|131|185|80|74.17|147.59|113.64|0.00|9091.20|5933.60|11807.20|272.73|0.00|9091.20|9363.93|3157.60| +2451871|65213|8228|22873|316198|4370|40142|10|289|185|19|87.54|103.29|7.23|0.00|137.37|1663.26|1962.51|2.74|0.00|137.37|140.11|-1525.89| +2451871|65213|5297|22873|316198|4370|40142|10|103|185|21|44.06|61.24|0.61|0.00|12.81|925.26|1286.04|0.89|0.00|12.81|13.70|-912.45| +2451871|65213|2885|22873|316198|4370|40142|10|88|185|27|60.41|87.59|18.39|0.00|496.53|1631.07|2364.93|34.75|0.00|496.53|531.28|-1134.54| +2451871||11671||316198||40142|10||185||75.27|||0.00|293.40||1544.40||0.00|293.40||| +2451871|65213|8381|22873|316198|4370|40142|10|244|185|69|75.92|110.84|97.53|0.00|6729.57|5238.48|7647.96|0.00|0.00|6729.57|6729.57|1491.09| +2451871|65213|12215|22873|316198|4370|40142|10|163|185|28|19.81|24.56|5.40|48.38|151.20|554.68|687.68|3.08|48.38|102.82|105.90|-451.86| +2451871|65213|13064|22873|316198|4370|40142|10|73|185|36|41.81|57.69|35.76|0.00|1287.36|1505.16|2076.84|64.36|0.00|1287.36|1351.72|-217.80| +2451871|65213|15308|22873|316198|4370|40142|10|246|185|57|91.54|170.26|97.04|3152.82|5531.28|5217.78|9704.82|190.27|3152.82|2378.46|2568.73|-2839.32| +2452610|53055|17859|66861|1214829|5197|47565|1|1|186|77|70.41|95.75|42.13|3016.92|3244.01|5421.57|7372.75|9.08|3016.92|227.09|236.17|-5194.48| +2452610|53055|8730|66861|1214829|5197|47565|1|231|186|9|84.56|144.59|83.86|0.00|754.74|761.04|1301.31|67.92|0.00|754.74|822.66|-6.30| +2452610|53055|14157|66861||5197|||268|186|19||2.00||||29.26|38.00|0.50|||25.96|-3.80| +2452610|53055|8679|66861|1214829|5197|47565|1|213|186|23|49.23|97.96|17.63|239.23|405.49|1132.29|2253.08|1.66|239.23|166.26|167.92|-966.03| +2452610|53055|2142|66861|1214829|5197|47565|1|157|186|72|30.38|42.53|13.18|0.00|948.96|2187.36|3062.16|66.42|0.00|948.96|1015.38|-1238.40| +2452610|53055|17235|66861|1214829|5197|47565|1|110|186|95|54.60|90.63|67.97|0.00|6457.15|5187.00|8609.85|129.14|0.00|6457.15|6586.29|1270.15| +2452610|53055|6336|66861|1214829|5197|47565|1|298|186|87|29.19|40.28|33.02|0.00|2872.74|2539.53|3504.36|258.54|0.00|2872.74|3131.28|333.21| +2452610|53055|1669|66861|1214829|5197|47565|1|23|186|12|61.01|88.46|0.88|0.00|10.56|732.12|1061.52|0.42|0.00|10.56|10.98|-721.56| +2452610|53055|3229|66861|1214829|5197|47565|1|191|186|68|36.25|68.51|34.25|0.00|2329.00|2465.00|4658.68|93.16|0.00|2329.00|2422.16|-136.00| +||16626||||47565|1||186|41|67.40|93.01||0.00|||3813.41|0.00|0.00||0.00|| +2452610|53055|7707|66861|1214829|5197|47565|1|56|186|16|2.33|4.52|3.43|0.00|54.88|37.28|72.32|1.09|0.00|54.88|55.97|17.60| +2452610|53055|8721|66861|1214829|5197|47565|1|299|186|90|20.70|21.73|18.03|0.00|1622.70|1863.00|1955.70|146.04|0.00|1622.70|1768.74|-240.30| +2452610|53055|5595|66861|1214829|5197|47565|1|89|186|75|30.60|31.82|15.91|0.00|1193.25|2295.00|2386.50|0.00|0.00|1193.25|1193.25|-1101.75| +2452610|53055|10038|66861|1214829|5197|47565|1|12|186|4|62.61|71.37|55.66|0.00|222.64|250.44|285.48|20.03|0.00|222.64|242.67|-27.80| +2451946|36487|7191|88272|1513813|4507|2801|8|86|187|23|54.09|56.79|21.01|0.00|483.23|1244.07|1306.17|14.49|0.00|483.23|497.72|-760.84| +2451946|36487|10933|88272|1513813|4507|2801|8|25|187|24|13.73|18.12|13.59|290.28|326.16|329.52|434.88|2.51|290.28|35.88|38.39|-293.64| +2451946|36487|6967|88272|1513813|4507|2801|8|68|187|87|99.58|103.56|103.56|0.00|9009.72|8663.46|9009.72|630.68|0.00|9009.72|9640.40|346.26| +2451946|36487|7893|88272|1513813|4507|2801|8|180|187|27|59.63|75.73|66.64|0.00|1799.28|1610.01|2044.71|71.97|0.00|1799.28|1871.25|189.27| +2451946|36487|13211|88272|1513813|4507|2801|8|93|187|9|25.67|46.97|30.06|0.00|270.54|231.03|422.73|2.70|0.00|270.54|273.24|39.51| +2451946|36487|4735|88272|1513813|4507|2801|8|193|187|11|31.12|62.24|19.29|29.70|212.19|342.32|684.64|14.59|29.70|182.49|197.08|-159.83| +||13679|88272|1513813|4507|2801|8|54|187|||36.88||1360.38|1659.00|3480.00|3688.00||1360.38||304.59|-3181.38| +2451946|36487|9021|88272|1513813|4507|2801|8|269|187|64|64.56|81.34|66.69|0.00|4268.16|4131.84|5205.76|213.40|0.00|4268.16|4481.56|136.32| +2451946|36487|15345|88272|1513813|4507|2801|8|80|187|14|50.05|71.57|35.06|0.00|490.84|700.70|1001.98|19.63|0.00|490.84|510.47|-209.86| +2451946|36487|11195|88272|1513813|4507|2801|8|198|187|48|78.78|149.68|146.68|0.00|7040.64|3781.44|7184.64|0.00|0.00|7040.64|7040.64|3259.20| +2451363|32619|1634|10855|133211|2767|49923|4|154|188|9|30.99|54.23|35.79|0.00|322.11|278.91|488.07|12.88|0.00|322.11|334.99|43.20| +2451363|32619|9830|10855|133211|2767|49923|4|71|188|4|98.55|183.30|111.81|0.00|447.24|394.20|733.20|17.88|0.00|447.24|465.12|53.04| +2451363|32619|2716|10855|133211|2767|49923|4|273|188|20|59.97|72.56|69.65|320.39|1393.00|1199.40|1451.20|42.90|320.39|1072.61|1115.51|-126.79| +||13465|||2767|49923||276|188|95||||0.00||5612.60|9822.05||0.00||6580.65|| +2451363|32619|8050|10855|133211|2767|49923|4|84|188|22|66.73|100.09|64.05|1000.46|1409.10|1468.06|2201.98|24.51|1000.46|408.64|433.15|-1059.42| +2451363|32619|7222|10855|133211|2767|49923|4|159|188|94|55.44|58.21|51.22|0.00|4814.68|5211.36|5471.74|144.44|0.00|4814.68|4959.12|-396.68| +||3854|10855|133211|2767||||188||5.98|10.34|||189.72|215.28|372.24|||189.72|204.89|| +2451363|32619|46|10855|133211|2767|49923|4|104|188|94|52.46|77.64|53.57|3675.97|5035.58|4931.24|7298.16|95.17|3675.97|1359.61|1454.78|-3571.63| +2451363|32619|6709|10855|133211|2767|49923|4|195|188|41|60.00|90.00|9.90|0.00|405.90|2460.00|3690.00|0.00|0.00|405.90|405.90|-2054.10| +2451363|32619|9008|10855|133211|2767|49923|4|145|188|11|46.26|91.13|55.58|397.39|611.38|508.86|1002.43|8.55|397.39|213.99|222.54|-294.87| +2451363|32619|424|10855|133211|2767|49923|4|159|188|57|77.95|81.06|7.29|0.00|415.53|4443.15|4620.42|24.93|0.00|415.53|440.46|-4027.62| +2451363|32619|13418|10855|133211|2767|49923|4|193|188|37|53.63|60.60|15.15|0.00|560.55|1984.31|2242.20|5.60|0.00|560.55|566.15|-1423.76| +2451090|53283|11936|75594|181714|6234|40754|10|103|189|15|89.82|138.32|100.97|0.00|1514.55|1347.30|2074.80|0.00|0.00|1514.55|1514.55|167.25| +2451090|53283|12298||181714|6234||10|5|189|1||35.91|3.23|0.12|3.23|35.91|35.91|0.27|0.12|||-32.80| +2451090|53283|17546|75594|181714|6234|40754|10|76|189|2|95.08|177.79|64.00|0.00|128.00|190.16|355.58|2.56|0.00|128.00|130.56|-62.16| +2451090|53283|3062|75594|181714|6234|40754|10|208|189|55|20.43|27.17|16.84|0.00|926.20|1123.65|1494.35|27.78|0.00|926.20|953.98|-197.45| +2451090|53283|14590|||6234|40754|10|47|189|17|68.28||51.23|0.00|870.91||||0.00||940.58|| +2451090|53283|13196|75594|181714|6234|40754|10|55|189|80|72.12|83.65|39.31|0.00|3144.80|5769.60|6692.00|157.24|0.00|3144.80|3302.04|-2624.80| +2451090|53283|7004|75594|181714|6234|40754|10|108|189|52|52.91|89.41|3.57|0.00|185.64|2751.32|4649.32|16.70|0.00|185.64|202.34|-2565.68| +2451090|53283|9994|75594|181714|6234|40754|10|112|189|32|97.80|170.17|90.19|952.40|2886.08|3129.60|5445.44|135.35|952.40|1933.68|2069.03|-1195.92| +2451090|53283|4024|75594|181714|6234|40754|10|179|189|63|82.89|138.42|103.81|523.20|6540.03|5222.07|8720.46|421.17|523.20|6016.83|6438.00|794.76| +2451090|53283|15158|75594|181714|6234|40754|10|19|189|89|49.98|89.46|57.25|0.00|5095.25|4448.22|7961.94|0.00|0.00|5095.25|5095.25|647.03| +2451090|53283|6076|75594|181714|6234|40754|10|186|189|24|6.99|10.48|4.19|0.00|100.56|167.76|251.52|9.05|0.00|100.56|109.61|-67.20| +2451090|53283|8485|75594|181714|6234|40754|10|254|189|70|26.27|44.13|19.85|0.00|1389.50|1838.90|3089.10|41.68|0.00|1389.50|1431.18|-449.40| +2451090|53283|1441|75594|181714|6234|40754|10|33|189|99|16.20|16.84|4.88|0.00|483.12|1603.80|1667.16|43.48|0.00|483.12|526.60|-1120.68| +2451090|53283|9049|75594|181714|6234|40754|10|20|189|36|42.94|59.68|57.29|0.00|2062.44|1545.84|2148.48|82.49|0.00|2062.44|2144.93|516.60| +2451090|53283|9553|75594|181714|6234|40754|10|156|189|12|15.45|25.64|24.35|0.00|292.20|185.40|307.68|8.76|0.00|292.20|300.96|106.80| +2451090|53283|12736||181714||40754|||189||||4.81|||802.13|874.16|21.21||235.69|256.90|| +2452491|49516|4425|39160|75025|2890|18755|10|168|190|92|5.26|7.99|3.91|0.00|359.72|483.92|735.08|17.98|0.00|359.72|377.70|-124.20| +2452491|49516|10878|39160|75025|2890|18755|10|188|190|45|81.74|149.58|71.79|0.00|3230.55|3678.30|6731.10|161.52|0.00|3230.55|3392.07|-447.75| +2452491|49516|9720|39160|75025|2890|18755|10|222|190|6|98.45|114.20|60.52|0.00|363.12|590.70|685.20|25.41|0.00|363.12|388.53|-227.58| +2452491|49516|1902|39160|75025|2890|18755|10|152|190|40|18.09|32.38|27.84|133.63|1113.60|723.60|1295.20|9.79|133.63|979.97|989.76|256.37| +2452491|49516|15234|39160|75025|2890|18755|10|267|190|15|8.79|10.46|3.45|0.00|51.75|131.85|156.90|3.10|0.00|51.75|54.85|-80.10| +2452491|49516|14046|39160|75025|2890|18755|10|165|190|91|40.67|69.13|64.29|0.00|5850.39|3700.97|6290.83|468.03|0.00|5850.39|6318.42|2149.42| +2452491|49516|6348|39160|75025|2890|18755|10|132|190|82|37.53|64.92|36.35|0.00|2980.70|3077.46|5323.44|268.26|0.00|2980.70|3248.96|-96.76| +2452491|49516|7260|39160|75025|2890|18755|10|64|190|90|88.56|102.72|36.97|0.00|3327.30|7970.40|9244.80|99.81|0.00|3327.30|3427.11|-4643.10| +2452491|49516|7665|39160|75025|2890|18755|10|74|190|6|36.05|67.41|20.22|86.13|121.32|216.30|404.46|0.35|86.13|35.19|35.54|-181.11| +2452491|49516|6433|39160|75025|2890|18755|10|8|190|5|82.45|127.79|23.00|0.00|115.00|412.25|638.95|5.75|0.00|115.00|120.75|-297.25| +2452491|49516|5325|39160|75025|2890|18755|10|149|190|13|42.01|44.53|10.24|0.00|133.12|546.13|578.89|5.32|0.00|133.12|138.44|-413.01| +2452491|49516|3331|39160|75025|2890|18755|10|6|190|64|42.28|63.42|32.34|0.00|2069.76|2705.92|4058.88|41.39|0.00|2069.76|2111.15|-636.16| +2452491|49516|7123|39160|75025|2890|18755|10|126|190|34|72.87|85.98|41.27|0.00|1403.18|2477.58|2923.32|42.09|0.00|1403.18|1445.27|-1074.40| +2452491|49516|5094|39160|75025|2890|18755|10|123|190|23|35.58|40.91|18.40|0.00|423.20|818.34|940.93|29.62|0.00|423.20|452.82|-395.14| +2452491|49516|13941|39160|75025|2890|18755|10|269|190|18|57.89|90.88|84.51|0.00|1521.18|1042.02|1635.84|76.05|0.00|1521.18|1597.23|479.16| +2452508|65090|1602|81037|1008360|930|28300|1|108|191|47|37.73|69.04|44.87|0.00|2108.89|1773.31|3244.88|0.00|0.00|2108.89|2108.89|335.58| +2452508|65090|15426|81037|1008360|930|28300|1|179|191|19|64.68|126.77|32.96|0.00|626.24|1228.92|2408.63|56.36|0.00|626.24|682.60|-602.68| +2452508|65090|15561|81037|1008360|930|28300|1|2|191|97|44.73|72.46|48.54|0.00|4708.38|4338.81|7028.62|141.25|0.00|4708.38|4849.63|369.57| +2452508|65090|11821|81037|1008360|930|28300|1|117|191|12|87.59|151.53|27.27|0.00|327.24|1051.08|1818.36|6.54|0.00|327.24|333.78|-723.84| +2452508|65090|12313|81037|1008360|930|28300||226|191||95.41|122.12|45.18|||||26.20|||2646.64|-2913.34| +2452508|65090|3588|81037|1008360|930|28300|1|213|191|42|68.97|114.49|82.43|0.00|3462.06|2896.74|4808.58|69.24|0.00|3462.06|3531.30|565.32| +2452508|65090|2259|81037|1008360|930|28300|1|261|191|3|34.84|43.20|40.60|0.00|121.80|104.52|129.60|8.52|0.00|121.80|130.32|17.28| +2452508|65090|6787|81037|1008360|930|28300|1|90|191|36|75.99|120.82|85.78|0.00|3088.08|2735.64|4349.52|277.92|0.00|3088.08|3366.00|352.44| +2452508|65090|3486|81037|1008360|930|28300|1|148|191|41|51.18|66.02|12.54|0.00|514.14|2098.38|2706.82|46.27|0.00|514.14|560.41|-1584.24| +2452508|65090|5257|81037|1008360|930|28300|1|95|191|93|59.93|76.71|75.94|0.00|7062.42|5573.49|7134.03|70.62|0.00|7062.42|7133.04|1488.93| +2452508|65090|13597|81037|1008360|930|28300|1|241|191|37|30.81|38.51|10.78|0.00|398.86|1139.97|1424.87|19.94|0.00|398.86|418.80|-741.11| +2452508|65090|6913|81037|1008360|930|28300|1|135|191|100|66.43|98.31|89.46|0.00|8946.00|6643.00|9831.00|447.30|0.00|8946.00|9393.30|2303.00| +2452508|65090|1489|81037|1008360|930|28300|1|181|191|29|26.84|26.84|3.75|0.00|108.75|778.36|778.36|0.00|0.00|108.75|108.75|-669.61| +||12565|||1622|||122|192|70|4.83|5.94|1.36||95.20|338.10||0.00||95.20||| +2451995|65741|4533|52228|1615325|1622|42719|4|294|192|80|52.55|74.62|58.20|3538.56|4656.00|4204.00|5969.60|0.00|3538.56|1117.44|1117.44|-3086.56| +2451995|65741|15823|52228|1615325|1622|42719|4|300|192|14|3.30|5.11|1.58|0.00|22.12|46.20|71.54|0.22|0.00|22.12|22.34|-24.08| +2451995|65741|7827|52228|1615325|1622|42719|4|80|192|74|76.28|129.67|71.31|0.00|5276.94|5644.72|9595.58|52.76|0.00|5276.94|5329.70|-367.78| +2451995|65741|11317|52228|1615325|1622|42719|4|202|192|6|75.02|145.53|18.91|0.00|113.46|450.12|873.18|3.40|0.00|113.46|116.86|-336.66| +2451995|65741|16693|52228|1615325|1622|42719|4|260|192|21|71.75|89.68|51.11|0.00|1073.31|1506.75|1883.28|96.59|0.00|1073.31|1169.90|-433.44| +2451995|65741|6799|52228|1615325|1622|42719|4|121|192|13|87.71|107.88|99.24|0.00|1290.12|1140.23|1402.44|38.70|0.00|1290.12|1328.82|149.89| +2451995|65741|13077|52228|1615325|1622|42719|4|228|192|96|12.71|24.65|15.77|590.42|1513.92|1220.16|2366.40|9.23|590.42|923.50|932.73|-296.66| +2451995|65741|15871|52228|1615325|1622|42719|4|157|192|81|24.84|48.68|31.64|410.05|2562.84|2012.04|3943.08|150.69|410.05|2152.79|2303.48|140.75| +2451995||16561|52228|||42719|4||192|78||||0.00||||29.54|0.00|||| +2451995|65741|13167|52228|1615325|1622|42719|4|9|192|13|73.96|92.45|74.88|0.00|973.44|961.48|1201.85|68.14|0.00|973.44|1041.58|11.96| +2451995|65741|3981|52228|1615325|1622|42719|4|45|192|51|55.21|70.66|33.21|0.00|1693.71|2815.71|3603.66|152.43|0.00|1693.71|1846.14|-1122.00| +2451995|65741|287|52228|1615325|1622|42719|4|246|192|97|68.21|72.98|10.94|0.00|1061.18|6616.37|7079.06|63.67|0.00|1061.18|1124.85|-5555.19| +2451995|65741|7769|52228|1615325|1622|42719|4|247|192|28|60.83|86.98|55.66|0.00|1558.48|1703.24|2435.44|46.75|0.00|1558.48|1605.23|-144.76| +2451995|65741|14461|52228|1615325|1622|42719|4|245|192|93|12.49|14.98|3.59|0.00|333.87|1161.57|1393.14|30.04|0.00|333.87|363.91|-827.70| +2451995|65741|1575|52228|1615325|1622|42719|4|119|192|69|56.78|70.40|14.78|0.00|1019.82|3917.82|4857.60|50.99|0.00|1019.82|1070.81|-2898.00| +2451856|55407|14213|72739|1882087|1388|6289|4|20|193|14|54.39|89.19|46.37|0.00|649.18|761.46|1248.66|58.42|0.00|649.18|707.60|-112.28| +2451856|55407|2690|72739|1882087|1388|6289|4|42|193|67|60.87|91.91|87.31|0.00|5849.77|4078.29|6157.97|292.48|0.00|5849.77|6142.25|1771.48| +2451856|55407|2012|72739|1882087|1388|6289|4|36|193|47|51.17|52.19|40.18|0.00|1888.46|2404.99|2452.93|56.65|0.00|1888.46|1945.11|-516.53| +2451856|55407|12815|72739|1882087|1388|6289|4|259|193|35|80.85|102.67|99.58|0.00|3485.30|2829.75|3593.45|278.82|0.00|3485.30|3764.12|655.55| +2451856|55407|697|72739|1882087|1388|6289|4|3|193|89|79.21|151.29|45.38|0.00|4038.82|7049.69|13464.81|323.10|0.00|4038.82|4361.92|-3010.87| +2451856|55407|17048|72739|1882087|1388|||169|193||84.62|93.92|21.60|0.00||423.10|||0.00||114.48|-315.10| +2451856|55407|2255|72739|1882087|1388|6289|4|46|193|22|79.06|95.66|58.35|0.00|1283.70|1739.32|2104.52|102.69|0.00|1283.70|1386.39|-455.62| +2451856|55407|3365|72739|1882087|1388|6289|4|161|193|86|18.52|25.55|9.96|0.00|856.56|1592.72|2197.30|59.95|0.00|856.56|916.51|-736.16| +2451856|55407|1883|72739|1882087|1388|6289|4|68|193|41|73.17|77.56|29.47|930.36|1208.27|2999.97|3179.96|25.01|930.36|277.91|302.92|-2722.06| +2451856|55407|6368|72739|1882087|1388|6289|4|132|193|52|75.93|78.20|60.99|0.00|3171.48|3948.36|4066.40|190.28|0.00|3171.48|3361.76|-776.88| +2451856|55407|13087|72739|1882087|1388|6289|4|86|193|24|9.16|10.62|0.10|2.11|2.40|219.84|254.88|0.01|2.11|0.29|0.30|-219.55| +2451856|55407|809|72739|1882087|1388|6289|4|197|193|85|78.64|83.35|0.83|0.00|70.55|6684.40|7084.75|3.52|0.00|70.55|74.07|-6613.85| +2451856|55407|5159|72739|1882087|1388|6289|4|133|193|40|63.64|78.27|7.04|0.00|281.60|2545.60|3130.80|19.71|0.00|281.60|301.31|-2264.00| +2451181|53887|9304|10411|191837|2510|49936|1|244|194|15|67.39|93.67|83.36|0.00|1250.40|1010.85|1405.05|112.53|0.00|1250.40|1362.93|239.55| +2451181|53887|4952|10411|191837|2510|49936|1|177|194|61|55.72|57.39|30.41|0.00|1855.01|3398.92|3500.79|111.30|0.00|1855.01|1966.31|-1543.91| +2451181|53887|17054|10411|191837|2510|49936|1|283|194|16|86.35|145.06|132.00|0.00|2112.00|1381.60|2320.96|42.24|0.00|2112.00|2154.24|730.40| +2451181|53887|12301|10411|191837|2510|49936|1|294|194|37|28.06|38.44|31.13|0.00|1151.81|1038.22|1422.28|69.10|0.00|1151.81|1220.91|113.59| +2451181|53887|4843|10411|191837|2510|49936|1|59|194|76|84.50|119.14|103.65|0.00|7877.40|6422.00|9054.64|630.19|0.00|7877.40|8507.59|1455.40| +2451181|53887|1588|10411|191837|2510|49936|1|211|194|52|95.84|101.59|65.01|0.00|3380.52|4983.68|5282.68|270.44|0.00|3380.52|3650.96|-1603.16| +2451181|53887|6518|10411|191837|2510|49936|1|228|194|58|19.49|37.81|30.24|0.00|1753.92|1130.42|2192.98|105.23|0.00|1753.92|1859.15|623.50| +2451181|53887|15202|10411||2510|||276|194||38.73|60.03||0.00|82.80|||3.31|0.00||86.11|-149.58| +2451181|53887|14972|10411|191837|2510|49936|1|98|194|5|64.16|94.31|12.26|1.22|61.30|320.80|471.55|3.60|1.22|60.08|63.68|-260.72| +2451181|53887|5341|10411|191837|2510|49936|1|223|194|22|21.04|29.45|2.65|0.58|58.30|462.88|647.90|4.61|0.58|57.72|62.33|-405.16| +2451181|53887|5776|10411|191837|2510|49936|1|277|194|13|79.95|93.54|32.73|0.00|425.49|1039.35|1216.02|38.29|0.00|425.49|463.78|-613.86| +2451181|53887|3403|10411|191837|2510|49936|1|162|194|15|79.18|101.35|45.60|0.00|684.00|1187.70|1520.25|0.00|0.00|684.00|684.00|-503.70| +|53887|4276||191837|2510|49936|||194|56|70.95|||0.00|2425.36|||97.01|0.00|2425.36||-1547.84| +2451181|53887|6410|10411|191837|2510|49936|1|81|194|4|23.18|44.27|5.75|0.00|23.00|92.72|177.08|1.15|0.00|23.00|24.15|-69.72| +2451181|53887|2816|10411|191837|2510|49936|1|90|194|65|30.99|37.18|32.71|0.00|2126.15|2014.35|2416.70|85.04|0.00|2126.15|2211.19|111.80| +2451181|53887|8407|10411|191837|2510|49936|1|240|194|64|22.33|22.77|6.37|0.00|407.68|1429.12|1457.28|36.69|0.00|407.68|444.37|-1021.44| +2452132|74561|1527|16349|1243475|4648|25421|10|121|195|46|58.43|75.95|12.91|0.00|593.86|2687.78|3493.70|5.93|0.00|593.86|599.79|-2093.92| +2452132|74561|12017|16349|1243475|4648|25421|10|19|195|27|24.25|34.43|31.33|549.84|845.91|654.75|929.61|26.64|549.84|296.07|322.71|-358.68| +2452132|74561|16325|16349|1243475|4648|25421|10|223|195|66|11.37|13.18|5.14|0.00|339.24|750.42|869.88|6.78|0.00|339.24|346.02|-411.18| +2452132|74561|4119|16349|1243475|4648|25421|10|214|195|38|33.95|65.18|7.82|0.00|297.16|1290.10|2476.84|5.94|0.00|297.16|303.10|-992.94| +2452132|74561|5197|16349|1243475|4648|25421|10|195|195|37|17.20|25.62|7.42|0.00|274.54|636.40|947.94|19.21|0.00|274.54|293.75|-361.86| +2452132|74561|2313|16349|1243475|4648|25421|10|5|195|18|16.27|17.40|7.48|0.00|134.64|292.86|313.20|12.11|0.00|134.64|146.75|-158.22| +2452132|74561|13815|16349|1243475|4648|25421|10|262|195|34|83.19|162.22|16.22|0.00|551.48|2828.46|5515.48|38.60|0.00|551.48|590.08|-2276.98| +|74561|5449||1243475||||30|195|52|||5.53||||364.52||||296.18|| +2452132|74561|1683|16349|1243475|4648|25421|10|124|195|6|62.07|100.55|30.16|99.52|180.96|372.42|603.30|4.88|99.52|81.44|86.32|-290.98| +2452132|74561|6631|16349|1243475|4648|25421|10|113|195|57|73.14|146.28|143.35|0.00|8170.95|4168.98|8337.96|163.41|0.00|8170.95|8334.36|4001.97| +2452132|74561|4667|16349|1243475|4648|25421|10|133|195|65|3.89|5.29|3.86|0.00|250.90|252.85|343.85|15.05|0.00|250.90|265.95|-1.95| +2452132|74561|10497|16349|1243475|4648|25421|10|213|195|70|97.92|152.75|54.99|0.00|3849.30|6854.40|10692.50|76.98|0.00|3849.30|3926.28|-3005.10| +2452132|74561|8399|16349|1243475|4648|25421|10|250|195|97|89.83|121.27|67.91|0.00|6587.27|8713.51|11763.19|65.87|0.00|6587.27|6653.14|-2126.24| +2452132|74561|4141|16349|1243475|4648|25421|10|36|195|67|27.41|41.93|20.12|0.00|1348.04|1836.47|2809.31|13.48|0.00|1348.04|1361.52|-488.43| +2451910|63464|9326|13009|1610331|6853|11728|10|154|196|36|49.50|72.76|50.93|0.00|1833.48|1782.00|2619.36|146.67|0.00|1833.48|1980.15|51.48| +2451910|63464|704|13009|1610331|6853|11728|10|223|196|56|25.73|37.56|5.25|0.00|294.00|1440.88|2103.36|23.52|0.00|294.00|317.52|-1146.88| +2451910|63464|14168|13009|1610331|6853|11728|10|279|196|71|43.87|55.71|51.25|0.00|3638.75|3114.77|3955.41|109.16|0.00|3638.75|3747.91|523.98| +2451910|63464|695|13009|1610331|6853|11728|10|124|196|26|23.22|37.15|4.08|0.00|106.08|603.72|965.90|6.36|0.00|106.08|112.44|-497.64| +2451910|63464|15073|13009|1610331|6853|11728|10|184|196|42|20.24|24.08|10.83|100.06|454.86|850.08|1011.36|17.74|100.06|354.80|372.54|-495.28| +2451910|63464|7454|13009|1610331|6853|11728|10|67|196|81|69.43|113.17|30.55|0.00|2474.55|5623.83|9166.77|0.00|0.00|2474.55|2474.55|-3149.28| +2451910||446|13009|1610331||11728|10|63|196|23|||70.78||1627.94||1789.17|16.27||1627.94||442.98| +2451910|63464|12434|13009|1610331|6853|11728|10|110|196|42|14.26|23.95|19.16|0.00|804.72|598.92|1005.90|56.33|0.00|804.72|861.05|205.80| +2451910|63464|6110|13009|1610331|6853|11728|10|261|196|82|79.58|87.53|85.77|140.66|7033.14|6525.56|7177.46|275.69|140.66|6892.48|7168.17|366.92| +2451910|63464|6044|13009|1610331|6853|11728|10|290|196|55|8.00|9.68|5.61|0.00|308.55|440.00|532.40|24.68|0.00|308.55|333.23|-131.45| +2451910|63464|11209|13009|1610331|6853|11728|10|185|196|40|35.24|51.09|12.77|0.00|510.80|1409.60|2043.60|10.21|0.00|510.80|521.01|-898.80| +2451910|63464|6500|13009|1610331|6853|11728|10|151|196|64|84.74|116.09|42.95|0.00|2748.80|5423.36|7429.76|54.97|0.00|2748.80|2803.77|-2674.56| +2451418|45157|13456|38627|1489827|7074|15474|10|25|197|36|81.12|96.53|55.98|0.00|2015.28|2920.32|3475.08|161.22|0.00|2015.28|2176.50|-905.04| +2451418|45157|7448|38627|1489827|7074|15474|10|4|197|19|20.79|28.89|19.64|0.00|373.16|395.01|548.91|18.65|0.00|373.16|391.81|-21.85| +2451418|45157|2713|38627|1489827|7074|15474|10|11|197|65|47.94|48.89|11.73|0.00|762.45|3116.10|3177.85|68.62|0.00|762.45|831.07|-2353.65| +2451418|45157|1399|38627|1489827|7074|15474|10|93|197|74|46.78|76.25|4.57|121.74|338.18|3461.72|5642.50|19.47|121.74|216.44|235.91|-3245.28| +2451418|45157|7339|38627|1489827|7074|15474|10|147|197|71|17.72|18.42|2.21|14.12|156.91|1258.12|1307.82|11.42|14.12|142.79|154.21|-1115.33| +2451418|45157|8686|38627|1489827|7074|15474|10|19|197|32|12.06|17.96|3.41|0.00|109.12|385.92|574.72|9.82|0.00|109.12|118.94|-276.80| +2451418|45157|12439|38627|1489827|7074|15474|10|9|197|26|19.13|32.13|17.02|194.70|442.52|497.38|835.38|4.95|194.70|247.82|252.77|-249.56| +2451418|45157|9220|38627|1489827|7074|15474|10|16|197|71|18.09|22.97|7.58|0.00|538.18|1284.39|1630.87|48.43|0.00|538.18|586.61|-746.21| +2451418|45157|992|38627|1489827|7074|15474|10|47|197|62|52.97|105.41|4.21|125.28|261.02|3284.14|6535.42|2.71|125.28|135.74|138.45|-3148.40| +2451418|45157|7486|38627|1489827|7074|15474|10|276|197|34|72.29|125.06|38.76|0.00|1317.84|2457.86|4252.04|105.42|0.00|1317.84|1423.26|-1140.02| +2451418|45157|6289|38627|1489827|7074|15474|10|291|197|32|10.19|10.69|0.85|0.00|27.20|326.08|342.08|0.54|0.00|27.20|27.74|-298.88| +2451418|45157|3868|38627|1489827|7074|15474|10|53|197|2|44.33|77.57|39.56|0.00|79.12|88.66|155.14|4.74|0.00|79.12|83.86|-9.54| +2451418|45157|16687|38627|1489827|7074|15474|10|44|197|54|45.26|58.38|6.42|0.00|346.68|2444.04|3152.52|3.46|0.00|346.68|350.14|-2097.36| +2451418|45157|11918|38627|1489827|7074|15474|10|243|197|51|94.18|105.48|61.17|842.31|3119.67|4803.18|5379.48|68.32|842.31|2277.36|2345.68|-2525.82| +2452518|67579|17395|27178|740045|5359|3026|8|252|198|60|67.98|112.16|41.49|0.00|2489.40|4078.80|6729.60|149.36|0.00|2489.40|2638.76|-1589.40| +2452518|67579|157|27178|740045|5359|3026|8|84|198|74|4.52|5.74|5.28|0.00|390.72|334.48|424.76|19.53|0.00|390.72|410.25|56.24| +2452518|67579|10699|27178|740045|5359|3026|8|213|198|10|20.58|38.89|29.94|0.00|299.40|205.80|388.90|17.96|0.00|299.40|317.36|93.60| +2452518|67579|15157|27178|740045|5359|3026|8|206|198|40|55.68|89.64|73.50|0.00|2940.00|2227.20|3585.60|117.60|0.00|2940.00|3057.60|712.80| +2452518|67579|17226|27178|740045|5359|3026|8|4|198|17|66.28|116.65|99.15|0.00|1685.55|1126.76|1983.05|134.84|0.00|1685.55|1820.39|558.79| +2452518|67579|11991|27178|740045|5359|3026|8|181|198|78|5.27|8.43|2.36|0.00|184.08|411.06|657.54|7.36|0.00|184.08|191.44|-226.98| +2452518|67579|9901|27178|740045|5359|3026|8|297|198|70|25.92|45.36|29.48|660.35|2063.60|1814.40|3175.20|56.13|660.35|1403.25|1459.38|-411.15| +2452518|67579|7608|27178|740045|5359|3026|8|257|198|41|33.31|56.62|28.87|0.00|1183.67|1365.71|2321.42|82.85|0.00|1183.67|1266.52|-182.04| +2452518|67579|15649|27178|740045|5359|3026|8|97|198|69|10.73|21.35|2.34|0.00|161.46|740.37|1473.15|6.45|0.00|161.46|167.91|-578.91| +2452518|67579|11490|27178|740045|5359|3026|8|274|198|25|41.78|58.90|34.75|0.00|868.75|1044.50|1472.50|17.37|0.00|868.75|886.12|-175.75| +2452518|67579|2421|27178|740045|5359|3026|8|4|198|20|57.40|107.91|64.74|0.00|1294.80|1148.00|2158.20|51.79|0.00|1294.80|1346.59|146.80| +2452518|67579|16647|27178|740045|5359|3026|8|98|198|22|22.28|39.43|36.66|0.00|806.52|490.16|867.46|32.26|0.00|806.52|838.78|316.36| +2452518|67579|17457|27178|740045|5359|3026|8|150|198|38|62.07|65.79|46.71|0.00|1774.98|2358.66|2500.02|17.74|0.00|1774.98|1792.72|-583.68| +2452518|67579|6099|27178|740045|5359|3026|8|105|198|21|10.84|19.40|12.99|0.00|272.79|227.64|407.40|5.45|0.00|272.79|278.24|45.15| +2451581|72460|1574|89187|624875|1716|19804|10|170|199|11|48.61|93.81|18.76|0.00|206.36|534.71|1031.91|10.31|0.00|206.36|216.67|-328.35| +2451581|72460|15536|89187|624875|1716|19804|10|142|199|66|63.43|116.71|23.34|0.00|1540.44|4186.38|7702.86|107.83|0.00|1540.44|1648.27|-2645.94| +2451581|72460|9169|89187|624875|1716|19804|10|135|199|63|88.91|112.02|89.61|0.00|5645.43|5601.33|7057.26|338.72|0.00|5645.43|5984.15|44.10| +2451581|72460|1733|89187|624875|1716|19804|10|74|199|93|21.36|32.04|5.44|0.00|505.92|1986.48|2979.72|25.29|0.00|505.92|531.21|-1480.56| +2451581|72460|1670|89187|624875|1716|19804|10|228|199|39|80.98|130.37|74.31|0.00|2898.09|3158.22|5084.43|28.98|0.00|2898.09|2927.07|-260.13| +2451581|72460|5227|89187|624875|1716|19804|10|122|199|82|92.38|130.25|33.86|0.00|2776.52|7575.16|10680.50|138.82|0.00|2776.52|2915.34|-4798.64| +2451581|72460|2516|89187|624875|1716|19804|10|290|199|80|66.03|81.21|47.10|2373.84|3768.00|5282.40|6496.80|0.00|2373.84|1394.16|1394.16|-3888.24| +2451581|72460|11480|89187|624875|1716|19804|10|17|199|64|51.95|63.37|44.35|0.00|2838.40|3324.80|4055.68|85.15|0.00|2838.40|2923.55|-486.40| +2451581|72460|12893|89187|624875|1716|19804|10|62|199|33|65.57|100.97|4.03|0.00|132.99|2163.81|3332.01|0.00|0.00|132.99|132.99|-2030.82| +2451581|72460|2533|89187|624875|1716|19804|10|122|199|76|52.83|99.32|90.38|0.00|6868.88|4015.08|7548.32|412.13|0.00|6868.88|7281.01|2853.80| +2451581|72460|15865|89187|624875|1716|19804|10|110|199|49|36.62|47.60|5.23|225.51|256.27|1794.38|2332.40|2.46|225.51|30.76|33.22|-1763.62| +2451581|72460|5339|89187|624875|1716|19804|10|280|199|100|79.94|117.51|42.30|0.00|4230.00|7994.00|11751.00|211.50|0.00|4230.00|4441.50|-3764.00| +2451776|65514|13688|73203|311434|5665|37943|10|100|200|21|11.05|19.55|13.68|0.00|287.28|232.05|410.55|17.23|0.00|287.28|304.51|55.23| +2451776|65514|5557|73203|311434|5665|37943|10|175|200|3|36.84|47.89|17.24|0.00|51.72|110.52|143.67|3.10|0.00|51.72|54.82|-58.80| +2451776|65514|4265|73203|311434|5665|37943|10|264|200|39|13.28|18.06|13.18|0.00|514.02|517.92|704.34|0.00|0.00|514.02|514.02|-3.90| +2451776|65514|10886|73203|311434|5665|37943|10|264|200|73|11.83|21.17|7.62|311.50|556.26|863.59|1545.41|9.79|311.50|244.76|254.55|-618.83| +2451776|65514|5989|73203|311434|5665|37943|10|171|200|52|58.54|74.34|68.39|0.00|3556.28|3044.08|3865.68|177.81|0.00|3556.28|3734.09|512.20| +2451776|65514|6824|73203|311434|5665|37943|10|188|200|33|40.98|57.37|47.61|0.00|1571.13|1352.34|1893.21|109.97|0.00|1571.13|1681.10|218.79| +2451776|65514|5069|73203|311434|5665|37943|10|238|200|88|17.61|25.88|15.78|0.00|1388.64|1549.68|2277.44|111.09|0.00|1388.64|1499.73|-161.04| +2451776|65514|10826|73203|311434|5665|37943|10|285|200|13|49.67|92.88|61.30|0.00|796.90|645.71|1207.44|71.72|0.00|796.90|868.62|151.19| +2451776|65514|14989|73203|311434|5665|37943|10|52|200|78|25.97|51.94|49.86|2061.21|3889.08|2025.66|4051.32|73.11|2061.21|1827.87|1900.98|-197.79| +2451776|65514|6707|73203|311434|5665|37943|10|201|200|72|63.74|115.36|72.67|0.00|5232.24|4589.28|8305.92|209.28|0.00|5232.24|5441.52|642.96| +2451776|65514|17975|73203|311434|5665|37943|10|69|200|66|10.13|13.57|0.27|0.00|17.82|668.58|895.62|1.06|0.00|17.82|18.88|-650.76| +||2737|73203||5665|37943||130|200||97.06|||||582.36||||211.38||-370.98| +2451255|55991|2614|99060|939028|596|10611|1|61|201|4|1.33|2.60|1.92|5.14|7.68|5.32|10.40|0.12|5.14|2.54|2.66|-2.78| +2451255|55991|8354|99060|939028|596|10611|1|227|201|69|69.90|105.54|93.93|0.00|6481.17|4823.10|7282.26|64.81|0.00|6481.17|6545.98|1658.07| +2451255|55991|6478|99060|939028|596|10611|1|272|201|43|95.20|150.41|12.03|0.00|517.29|4093.60|6467.63|20.69|0.00|517.29|537.98|-3576.31| +2451255|55991|9866|99060|939028|596|10611|1|281|201|81|79.91|114.27|14.85|0.00|1202.85|6472.71|9255.87|0.00|0.00|1202.85|1202.85|-5269.86| +2451255|55991|14911|99060|939028|596|10611|1|135|201|13|8.77|11.83|10.41|0.00|135.33|114.01|153.79|12.17|0.00|135.33|147.50|21.32| +2451255|55991|14071|99060|939028|596|10611|1|134|201|47|61.70|101.80|31.55|0.00|1482.85|2899.90|4784.60|59.31|0.00|1482.85|1542.16|-1417.05| +2451255|55991|4318|||596|10611|1|123|201|28|41.95|72.57||0.00|||2031.96||0.00|1076.88|1119.95|-97.72| +2451255|55991|11731|99060|939028|596|10611|1|300|201|90|1.42|1.80|0.00|0.00|0.00|127.80|162.00|0.00|0.00|0.00|0.00|-127.80| +2451255|55991|15931|99060|939028|596|10611|1|214|201|52|35.94|42.40|1.27|0.00|66.04|1868.88|2204.80|0.00|0.00|66.04|66.04|-1802.84| +2451255|55991|3859|99060|939028|596|10611|1|300|201|25|14.49|16.37|14.40|0.00|360.00|362.25|409.25|0.00|0.00|360.00|360.00|-2.25| +2451255|55991|343|99060|939028|596|10611|1|82|201|83|62.76|86.60|26.84|913.36|2227.72|5209.08|7187.80|0.00|913.36|1314.36|1314.36|-3894.72| +2451255|55991|10642|99060|939028|596|10611|1|160|201|13|86.34|138.14|73.21|0.00|951.73|1122.42|1795.82|47.58|0.00|951.73|999.31|-170.69| +2451255|55991|9889|99060|939028|596|10611|1|82|201|59|91.24|145.98|80.28|0.00|4736.52|5383.16|8612.82|47.36|0.00|4736.52|4783.88|-646.64| +2451255|55991|3310|99060|939028|596|10611|1|160|201|60|67.63|77.77|70.77|0.00|4246.20|4057.80|4666.20|0.00|0.00|4246.20|4246.20|188.40| +2451255|55991|4309|99060|939028|596|10611|1|141|201|5|81.06|84.30|33.72|0.00|168.60|405.30|421.50|15.17|0.00|168.60|183.77|-236.70| +2451255|55991|140|99060|939028|596|10611|1|195|201|21|14.64|28.40|12.21|0.00|256.41|307.44|596.40|5.12|0.00|256.41|261.53|-51.03| +2452011|32774|14457|23204|35877|2150|39270|8|1|202|1|98.26|189.64|109.99|0.00|109.99|98.26|189.64|6.59|0.00|109.99|116.58|11.73| +2452011|32774|5665|23204|35877|2150|39270|8|225|202|39|36.45|63.42|61.51|0.00|2398.89|1421.55|2473.38|191.91|0.00|2398.89|2590.80|977.34| +2452011|32774|10243|23204|35877|2150|39270|8|29|202|65|15.48|18.26|2.00|100.10|130.00|1006.20|1186.90|0.89|100.10|29.90|30.79|-976.30| +2452011|32774|7899|23204|35877|2150|39270|8|264|202|21|90.46|155.59|40.45|0.00|849.45|1899.66|3267.39|16.98|0.00|849.45|866.43|-1050.21| +2452011|32774|9127|23204|35877|2150|39270|8|46|202|4|31.09|60.00|54.60|0.00|218.40|124.36|240.00|0.00|0.00|218.40|218.40|94.04| +2452011|32774|2123|23204|35877|2150|39270|8|148|202|63|35.04|63.07|23.96|0.00|1509.48|2207.52|3973.41|120.75|0.00|1509.48|1630.23|-698.04| +2452011|32774|2357|23204|35877|2150|39270|8|30|202|25|16.86|17.87|0.00|0.00|0.00|421.50|446.75|0.00|0.00|0.00|0.00|-421.50| +2452011||17829|23204|35877||39270||8|202|49||158.99|84.26||4128.74|||||||-650.72| +2452011|32774|13623|23204|35877|2150|39270|8|78|202|89|78.42|93.31|79.31|0.00|7058.59|6979.38|8304.59|0.00|0.00|7058.59|7058.59|79.21| +2451519|38227|14222|63323|504620|317|11214|10|252|203|90|35.03|57.09|52.52|0.00|4726.80|3152.70|5138.10|47.26|0.00|4726.80|4774.06|1574.10| +2451519|38227|14096|63323|504620|317|11214|10|228|203|80|80.30|121.25|109.12|0.00|8729.60|6424.00|9700.00|785.66|0.00|8729.60|9515.26|2305.60| +2451519|38227|15196|63323|504620|317|11214|10|75|203|51|38.51|65.46|21.60|0.00|1101.60|1964.01|3338.46|11.01|0.00|1101.60|1112.61|-862.41| +2451519|38227|3172|63323|504620|317|11214|10|91|203|90|85.84|93.56|14.96|0.00|1346.40|7725.60|8420.40|0.00|0.00|1346.40|1346.40|-6379.20| +2451519|38227|17552|63323|504620|317|11214|10|76|203|6|96.35|114.65|2.29|0.00|13.74|578.10|687.90|0.41|0.00|13.74|14.15|-564.36| +2451519|38227|13000|63323|504620|317|11214|10|162|203|39|81.02|88.31|72.41|2739.27|2823.99|3159.78|3444.09|5.08|2739.27|84.72|89.80|-3075.06| +2451519|38227|7388|63323|504620|317|11214|10|25|203|97|80.71|147.69|53.16|567.21|5156.52|7828.87|14325.93|45.89|567.21|4589.31|4635.20|-3239.56| +2451519|38227|12596|63323|504620|317|11214|10|146|203|96|9.21|14.18|11.20|0.00|1075.20|884.16|1361.28|10.75|0.00|1075.20|1085.95|191.04| +2451519|38227|4933|63323|504620|317|11214|10|101|203|17|35.92|36.99|26.26|0.00|446.42|610.64|628.83|17.85|0.00|446.42|464.27|-164.22| +2451519|38227|15182|63323|504620|317|11214|10|78|203|28|85.90|153.76|96.86|0.00|2712.08|2405.20|4305.28|81.36|0.00|2712.08|2793.44|306.88| +2451533|63407|15476|2818|1208644|4423|5849|8|221|204|88|77.06|114.04|42.19|0.00|3712.72|6781.28|10035.52|259.89|0.00|3712.72|3972.61|-3068.56| +2451533|63407|13576|2818|1208644|4423|5849|8|206|204|94|82.67|156.24|123.42|0.00|11601.48|7770.98|14686.56|116.01|0.00|11601.48|11717.49|3830.50| +2451533|63407|5959|2818|1208644|4423|5849|8|295|204|100|95.61|122.38|22.02|0.00|2202.00|9561.00|12238.00|22.02|0.00|2202.00|2224.02|-7359.00| +2451533|63407|1640|2818|1208644|4423|5849|8|35|204|47|36.04|70.99|37.62|0.00|1768.14|1693.88|3336.53|0.00|0.00|1768.14|1768.14|74.26| +2451533|63407|10603|2818|1208644|4423|5849|8|275|204|32|5.38|9.14|0.00|0.00|0.00|172.16|292.48|0.00|0.00|0.00|0.00|-172.16| +2451533|63407|16774|2818|1208644|4423|5849|8|217|204|56|72.80|93.91|1.87|0.00|104.72|4076.80|5258.96|4.18|0.00|104.72|108.90|-3972.08| +2451533|63407|9241|2818|1208644|4423|5849|8|174|204|74|27.46|37.07|10.37|0.00|767.38|2032.04|2743.18|15.34|0.00|767.38|782.72|-1264.66| +2451533|63407|9388|2818|1208644|4423|5849|8|85|204|36|21.63|21.63|0.21|0.00|7.56|778.68|778.68|0.22|0.00|7.56|7.78|-771.12| +2451533|63407|625|2818|1208644|4423|5849|8|252|204|1|18.14|31.92|25.85|0.00|25.85|18.14|31.92|0.00|0.00|25.85|25.85|7.71| +|66305|12273||||11060|2|28|205||34.06|57.22||1296.59||1294.28|2174.36|22.22|1296.59|246.97|269.19|| +2452203|66305|3263|25272|1024370|4715|11060|2|27|205|7|63.25|98.03|59.79|0.00|418.53|442.75|686.21|33.48|0.00|418.53|452.01|-24.22| +2452203|66305|2193|25272|1024370|4715|11060|2|4|205|38|82.17|83.81|5.02|0.00|190.76|3122.46|3184.78|5.72|0.00|190.76|196.48|-2931.70| +2452203|66305|10199|25272|1024370|4715|11060|2|43|205|48|5.50|10.67|9.38|0.00|450.24|264.00|512.16|31.51|0.00|450.24|481.75|186.24| +2452203|66305|9931|25272|1024370|4715|11060|2|152|205|91|58.96|117.33|16.42|0.00|1494.22|5365.36|10677.03|89.65|0.00|1494.22|1583.87|-3871.14| +2452203|66305|14497|25272|1024370|4715|11060|2|218|205|32|34.91|37.00|14.80|0.00|473.60|1117.12|1184.00|33.15|0.00|473.60|506.75|-643.52| +2452203|66305|9635|25272|1024370|4715|11060|2|192|205|88|29.44|41.21|14.01|0.00|1232.88|2590.72|3626.48|0.00|0.00|1232.88|1232.88|-1357.84| +2452203|66305|8935|25272|1024370|4715|11060|2|272|205|52|1.39|2.55|2.11|0.00|109.72|72.28|132.60|5.48|0.00|109.72|115.20|37.44| +2452203|66305|5685|25272|1024370|4715|11060|2|158|205|68|54.61|91.19|88.45|0.00|6014.60|3713.48|6200.92|421.02|0.00|6014.60|6435.62|2301.12| +2452203|66305|17703|25272|1024370|4715|11060|2|244|205|70|58.66|99.72|9.97|0.00|697.90|4106.20|6980.40|6.97|0.00|697.90|704.87|-3408.30| +2452203|66305|5695|25272|1024370|4715|11060|2|53|205|22|57.34|101.49|40.59|741.17|892.98|1261.48|2232.78|3.03|741.17|151.81|154.84|-1109.67| +2452203|66305|9967|25272|1024370|4715|11060|2|24|205|76|67.84|111.25|1.11|0.00|84.36|5155.84|8455.00|5.90|0.00|84.36|90.26|-5071.48| +2452203|66305|3901|25272|1024370|4715|11060|2|300|205|96|71.22|126.05|39.07|0.00|3750.72|6837.12|12100.80|300.05|0.00|3750.72|4050.77|-3086.40| +2452203|66305|8189|25272|1024370|4715|11060|2|256|205|58|70.90|86.49|13.83|200.53|802.14|4112.20|5016.42|48.12|200.53|601.61|649.73|-3510.59| +2452203|66305|325|25272|1024370|4715|11060|2|270|205|46|69.57|73.04|58.43|0.00|2687.78|3200.22|3359.84|134.38|0.00|2687.78|2822.16|-512.44| +2452335|68874|4614|13614|792571|5128|45685|10|112|206|89|28.37|34.89|16.04|0.00|1427.56|2524.93|3105.21|85.65|0.00|1427.56|1513.21|-1097.37| +2452335|68874|6177|13614|792571|5128|45685|10|106|206|61|46.27|69.40|25.67|0.00|1565.87|2822.47|4233.40|140.92|0.00|1565.87|1706.79|-1256.60| +2452335|68874|10854|13614|792571|5128|45685|10|278|206|17|84.39|151.90|45.57|0.00|774.69|1434.63|2582.30|7.74|0.00|774.69|782.43|-659.94| +2452335|68874|6213|13614|792571|5128|45685|10|134|206|75|34.30|57.62|13.82|0.00|1036.50|2572.50|4321.50|93.28|0.00|1036.50|1129.78|-1536.00| +2452335|68874|7971|||5128||10|242|206|57|33.07||6.21||353.97|1884.99||||353.97||-1531.02| +2452335|68874|9181|13614|792571|5128|45685|10|8|206|63|89.77|137.34|63.17|2427.62|3979.71|5655.51|8652.42|0.00|2427.62|1552.09|1552.09|-4103.42| +2452335|68874|6757|13614|792571|5128|45685|10|177|206|34|31.82|59.50|36.89|0.00|1254.26|1081.88|2023.00|62.71|0.00|1254.26|1316.97|172.38| +2452335|68874|345|13614|792571|5128|45685|10|156|206|9|2.58|4.20|1.55|0.00|13.95|23.22|37.80|0.00|0.00|13.95|13.95|-9.27| +2451534|64270|17539|57984|1309149|3961|44726|4|199|207|50|32.93|39.84|14.34|0.00|717.00|1646.50|1992.00|28.68|0.00|717.00|745.68|-929.50| +2451534|64270|16330|57984|1309149|3961|44726|4|195|207|66|90.94|143.68|45.97|0.00|3034.02|6002.04|9482.88|212.38|0.00|3034.02|3246.40|-2968.02| +2451534|64270|7814|57984|1309149|3961|44726|4|243|207|12|25.12|36.67|21.63|0.00|259.56|301.44|440.04|20.76|0.00|259.56|280.32|-41.88| +2451534||10558||1309149||||63|207||14.87|||||1070.64||20.44||||| +||15256|||||||207||||5.57|0.00|94.69|633.42|||0.00||95.63|-538.73| +2451534|64270|8786|57984|1309149|3961|44726|4|198|207|8|59.91|115.62|54.34|0.00|434.72|479.28|924.96|4.34|0.00|434.72|439.06|-44.56| +2451534|64270|4940|57984|1309149|3961|44726|4|150|207|100|6.81|10.75|0.43|0.00|43.00|681.00|1075.00|2.58|0.00|43.00|45.58|-638.00| +2451534|64270|15056|57984|1309149|3961|44726|4|70|207|7|44.92|73.21|66.62|0.00|466.34|314.44|512.47|13.99|0.00|466.34|480.33|151.90| +2451534|64270|6331|57984|1309149|3961|44726|4|186|207|37|65.15|85.99|85.99|0.00|3181.63|2410.55|3181.63|190.89|0.00|3181.63|3372.52|771.08| +2451534|64270|8263|57984|1309149|3961|44726|4|257|207|71|35.95|52.84|12.15|0.00|862.65|2552.45|3751.64|69.01|0.00|862.65|931.66|-1689.80| +2451534|64270|12106|57984|1309149|3961|44726|4|90|207|94|45.99|52.42|3.66|0.00|344.04|4323.06|4927.48|0.00|0.00|344.04|344.04|-3979.02| +2451534|64270|17012|57984|1309149|3961|44726|4|195|207|75|58.02|79.48|21.45|0.00|1608.75|4351.50|5961.00|48.26|0.00|1608.75|1657.01|-2742.75| +2451534|64270|9568|57984|1309149|3961|44726|4|92|207|66|2.36|3.42|3.04|0.00|200.64|155.76|225.72|18.05|0.00|200.64|218.69|44.88| +|62083|5575||||45689|||208|2|52.07|54.67|34.98|0.00||104.14|109.34||0.00|69.96||-34.18| +2451406|62083|4753|58476|1880747|7018|45689|4|278|208|20|46.39|46.39|45.46|672.80|909.20|927.80|927.80|9.45|672.80|236.40|245.85|-691.40| +2451406|62083|17812|58476|1880747|7018|45689|4|221|208|23|86.56|169.65|139.11|0.00|3199.53|1990.88|3901.95|63.99|0.00|3199.53|3263.52|1208.65| +2451406|62083|16981|58476|1880747|7018|45689|4|273|208|93|76.09|149.13|52.19|0.00|4853.67|7076.37|13869.09|291.22|0.00|4853.67|5144.89|-2222.70| +||5660|58476|1880747||||130|208||84.12|127.86|121.46|||4710.72||||6801.76|7413.91|| +2451406|62083|4732|58476|1880747|7018|45689|4|4|208|87|94.44|107.66|34.45|0.00|2997.15|8216.28|9366.42|0.00|0.00|2997.15|2997.15|-5219.13| +2451406|62083|3314|58476|1880747|7018|45689|4|200|208|86|61.28|117.65|45.88|0.00|3945.68|5270.08|10117.90|197.28|0.00|3945.68|4142.96|-1324.40| +2451406|62083|4438|58476|1880747|7018|45689|4|215|208|93|59.63|95.40|32.43|0.00|3015.99|5545.59|8872.20|90.47|0.00|3015.99|3106.46|-2529.60| +2451406|62083|6818|58476|1880747|7018|45689|4|113|208|78|23.80|28.08|25.27|0.00|1971.06|1856.40|2190.24|98.55|0.00|1971.06|2069.61|114.66| +2451406|62083|940|58476|1880747|7018|45689|4|30|208|26|2.33|3.89|2.41|0.00|62.66|60.58|101.14|5.01|0.00|62.66|67.67|2.08| +2451406|62083|12200|58476|1880747|7018|45689|4|245|208|56|18.24|25.90|12.43|563.82|696.08|1021.44|1450.40|7.93|563.82|132.26|140.19|-889.18| +2451406|62083|12422|58476|1880747|7018|45689|4|23|208|65|73.15|133.13|6.65|0.00|432.25|4754.75|8653.45|17.29|0.00|432.25|449.54|-4322.50| +2451406|62083|2101|58476|1880747|7018|45689|4|15|208|14|60.41|61.61|1.84|15.45|25.76|845.74|862.54|0.92|15.45|10.31|11.23|-835.43| +2451406|62083|15940|58476|1880747|7018|45689|4|25|208|10|87.58|151.51|99.99|0.00|999.90|875.80|1515.10|9.99|0.00|999.90|1009.89|124.10| +2451406|62083|6584|58476|1880747|7018|45689|4|51|208|34|41.52|44.01|11.00|0.00|374.00|1411.68|1496.34|14.96|0.00|374.00|388.96|-1037.68| +2451406|62083|10627|58476|1880747|7018|45689|4|159|208|6|73.53|114.70|47.02|0.00|282.12|441.18|688.20|19.74|0.00|282.12|301.86|-159.06| +2451824|70187|4466|23683|1817102|2521|40740|4|298|209|71|54.06|104.33|62.59|0.00|4443.89|3838.26|7407.43|177.75|0.00|4443.89|4621.64|605.63| +2451824|70187|7679|23683|1817102|2521|40740|4|93|209|87|74.00|121.36|121.36|8974.57|10558.32|6438.00|10558.32|47.51|8974.57|1583.75|1631.26|-4854.25| +2451824|70187|14279|23683|1817102|2521||4|192|209|62|||19.89|0.00||1020.52|1561.16|24.66|0.00||1257.84|212.66| +2451824|70187|3470|23683|1817102|2521|40740|4|11|209|97|54.42|78.36|12.53|1203.25|1215.41|5278.74|7600.92|1.09|1203.25|12.16|13.25|-5266.58| +2451824|70187|3392|23683|1817102|2521|40740|4|63|209|9|95.80|161.90|0.00|0.00|0.00|862.20|1457.10|0.00|0.00|0.00|0.00|-862.20| +2451824|70187|9056|23683|1817102|2521|40740|4|63|209|22|47.36|62.98|7.55|0.00|166.10|1041.92|1385.56|14.94|0.00|166.10|181.04|-875.82| +||10994||1817102|2521|40740|4|114|209||27.41||||317.25|411.15||12.69||317.25|329.94|| +2451824|70187|6997|23683|1817102|2521|40740|4|76|209|46|31.40|57.14|32.56|0.00|1497.76|1444.40|2628.44|74.88|0.00|1497.76|1572.64|53.36| +2451824|70187|1919|23683|1817102|2521|40740|4|103|209|53|37.59|71.04|18.47|293.67|978.91|1992.27|3765.12|13.70|293.67|685.24|698.94|-1307.03| +2451824|70187|1538|23683|1817102|2521|40740|4|136|209|97|76.25|104.46|16.71|0.00|1620.87|7396.25|10132.62|97.25|0.00|1620.87|1718.12|-5775.38| +2451824|70187|59|23683||2521|40740|||209||||16.22|492.43|1492.24||2369.00|59.98|492.43|999.81||-729.79| +2451824|70187|12524|23683|1817102|2521|40740|4|296|209|90|66.45|84.39|29.53|0.00|2657.70|5980.50|7595.10|159.46|0.00|2657.70|2817.16|-3322.80| +2451824|70187|296|23683|1817102|2521|40740|4|217|209|79|93.40|186.80|22.41|0.00|1770.39|7378.60|14757.20|70.81|0.00|1770.39|1841.20|-5608.21| +2451824|70187|7175|23683|1817102|2521|40740|4|209|209|87|65.40|75.86|44.75|3503.92|3893.25|5689.80|6599.82|7.78|3503.92|389.33|397.11|-5300.47| +2451824|70187|17714|23683|1817102|2521|40740|4|186|209|81|88.12|157.73|74.13|0.00|6004.53|7137.72|12776.13|240.18|0.00|6004.53|6244.71|-1133.19| +2451824|70187|12338|23683|1817102|2521|40740|4|152|209|48|65.89|67.86|59.71|0.00|2866.08|3162.72|3257.28|171.96|0.00|2866.08|3038.04|-296.64| +2451572|65189|6301|54822|1448103|231|1159|1|101|210|37|78.53|119.36|82.35|0.00|3046.95|2905.61|4416.32|30.46|0.00|3046.95|3077.41|141.34| +2451572|65189|12173|54822|1448103|231|1159|1|128|210|42|12.86|19.41|16.11|0.00|676.62|540.12|815.22|60.89|0.00|676.62|737.51|136.50| +2451572|65189|6809|54822|1448103|231|1159|1|285|210|35|83.72|161.57|135.71|0.00|4749.85|2930.20|5654.95|427.48|0.00|4749.85|5177.33|1819.65| +2451572|65189|2851|54822|1448103|231|1159|1|35|210|67|86.45|95.09|72.26|387.31|4841.42|5792.15|6371.03|356.32|387.31|4454.11|4810.43|-1338.04| +2451572|65189|194|54822|1448103|231|1159|1|297|210|40|34.14|61.45|36.25|0.00|1450.00|1365.60|2458.00|14.50|0.00|1450.00|1464.50|84.40| +2451572|65189|14300|54822|1448103|231|1159|1|89|210|5|75.41|139.50|19.53|67.37|97.65|377.05|697.50|0.00|67.37|30.28|30.28|-346.77| +2451572|65189|9455|54822|1448103|231|1159|1|246|210|80|38.59|77.18|60.97|0.00|4877.60|3087.20|6174.40|97.55|0.00|4877.60|4975.15|1790.40| +2451572|65189|12347|54822|1448103|231|1159|1|208|210|39|69.19|111.39|47.89|0.00|1867.71|2698.41|4344.21|74.70|0.00|1867.71|1942.41|-830.70| +2451572|65189|7589|54822|1448103|231|1159|1|41|210|84|10.44|15.66|1.72|0.00|144.48|876.96|1315.44|7.22|0.00|144.48|151.70|-732.48| +2451572|65189|931|54822|1448103|231|1159|1|264|210|12|31.30|32.23|2.57|0.00|30.84|375.60|386.76|1.23|0.00|30.84|32.07|-344.76| +2451572|65189|338|54822|1448103|231|1159|1|105|210|87|39.59|60.17|54.15|0.00|4711.05|3444.33|5234.79|47.11|0.00|4711.05|4758.16|1266.72| +2451572|65189|8855|54822|1448103|231|1159|1|24|210|8|86.98|166.13|68.11|0.00|544.88|695.84|1329.04|21.79|0.00|544.88|566.67|-150.96| +2451572|65189|15731|54822|1448103|231|1159|1|228|210|29|81.98|94.27|11.31|0.00|327.99|2377.42|2733.83|19.67|0.00|327.99|347.66|-2049.43| +2451572|65189|15110|54822|1448103|231|1159|1|53|210|90|19.62|29.03|0.87|0.00|78.30|1765.80|2612.70|5.48|0.00|78.30|83.78|-1687.50| +2451572|65189|5957|54822|1448103|231|1159|1|14|210|45|62.49|107.48|59.11|0.00|2659.95|2812.05|4836.60|0.00|0.00|2659.95|2659.95|-152.10| +2451145|62041|2191|74219|1724558|5297|49989|7|97|211|37|91.84|135.92|77.47|1719.83|2866.39|3398.08|5029.04|45.86|1719.83|1146.56|1192.42|-2251.52| +2451145|62041|7723|74219|1724558|5297|49989|7|38|211|61|85.61|146.39|23.42|0.00|1428.62|5222.21|8929.79|114.28|0.00|1428.62|1542.90|-3793.59| +2451145|62041|2108|74219|1724558|5297|49989|7|31|211|48|96.43|122.46|115.11|5028.00|5525.28|4628.64|5878.08|34.80|5028.00|497.28|532.08|-4131.36| +2451145|62041|11758|74219|1724558|5297|49989|7|295|211|18|94.74|155.37|127.40|894.34|2293.20|1705.32|2796.66|13.98|894.34|1398.86|1412.84|-306.46| +||4598||||49989|7|253|211||||49.89|729.39|2145.27||||729.39|||| +2451145|62041|14047|74219|1724558|5297|49989|7|26|211|99|20.10|39.79|16.71|0.00|1654.29|1989.90|3939.21|99.25|0.00|1654.29|1753.54|-335.61| +2451145|62041|16273|74219|1724558|5297|49989|7|188|211|73|77.35|83.53|18.37|0.00|1341.01|5646.55|6097.69|26.82|0.00|1341.01|1367.83|-4305.54| +2451145|62041|16696|74219|1724558|5297|49989|7|71|211|52|45.07|71.66|24.36|0.00|1266.72|2343.64|3726.32|63.33|0.00|1266.72|1330.05|-1076.92| +2451145|62041|1345|74219|1724558|5297|49989|7|184|211|91|43.88|50.02|1.00|61.88|91.00|3993.08|4551.82|0.58|61.88|29.12|29.70|-3963.96| +2451145||12034||||49989||123|211|||63.23||0.00||1288.10|1391.06|62.59|0.00|1251.80||-36.30| +2451543|62844|4658|77193|438725|1057|29389|4|26|212|5|77.60|90.01|61.20|0.00|306.00|388.00|450.05|27.54|0.00|306.00|333.54|-82.00| +2451543|62844|9721|77193|438725|1057|29389|4|277|212|57|94.79|149.76|13.47|0.00|767.79|5403.03|8536.32|69.10|0.00|767.79|836.89|-4635.24| +2451543|62844|8806|77193|438725|1057|29389|||212||81.87|129.35||0.00|||11512.15|99.00|0.00||5049.18|-2336.25| +2451543|62844|13711|77193|438725|1057|29389|4|192|212|50|54.30|77.10|60.90|0.00|3045.00|2715.00|3855.00|30.45|0.00|3045.00|3075.45|330.00| +2451543|62844|13648|77193|438725|1057|29389|4|246|212|85|70.54|107.92|85.25|0.00|7246.25|5995.90|9173.20|652.16|0.00|7246.25|7898.41|1250.35| +2451543|62844|19|77193|438725|1057|29389|4|296|212|5|40.54|62.02|60.77|0.00|303.85|202.70|310.10|12.15|0.00|303.85|316.00|101.15| +2451543|62844|9451|77193|438725|1057|29389|4|197|212|85|14.85|28.80|21.88|1729.61|1859.80|1262.25|2448.00|2.60|1729.61|130.19|132.79|-1132.06| +2451543|62844|4243|77193|438725|1057|29389|4|48|212|56|13.92|16.98|0.16|0.00|8.96|779.52|950.88|0.53|0.00|8.96|9.49|-770.56| +2451543|62844|8296|77193|438725|1057|29389|4|62|212|2|15.15|24.08|15.65|0.00|31.30|30.30|48.16|0.00|0.00|31.30|31.30|1.00| +2452171||12601|||3722|||25|213|99||109.33||0.00|107.91||||0.00|107.91|111.14|| +|37860|8179||20478||49366|||213|84|||2.94|0.00||598.08|687.12||0.00||269.18|-351.12| +|37860|4725||20478||49366|10|247|213|||113.25|50.96|0.00|2649.92||5889.00|132.49|0.00|2649.92||| +2452171|37860|14783|26034|20478|3722|49366|10|44|213|3|2.27|3.94|1.22|0.00|3.66|6.81|11.82|0.29|0.00|3.66|3.95|-3.15| +2452171|37860|15767|26034|20478|3722|49366|10|158|213|50|74.38|81.81|31.08|279.72|1554.00|3719.00|4090.50|12.74|279.72|1274.28|1287.02|-2444.72| +2452171|37860|10337|26034|20478|3722|49366|10|124|213|63|25.60|38.40|14.97|0.00|943.11|1612.80|2419.20|18.86|0.00|943.11|961.97|-669.69| +2452171|37860|17557|26034|20478|3722|49366|10|166|213|48|96.72|153.78|99.95|0.00|4797.60|4642.56|7381.44|431.78|0.00|4797.60|5229.38|155.04| +2452171|37860|14719|26034|20478|3722|49366|10|133|213|36|38.30|65.49|45.84|0.00|1650.24|1378.80|2357.64|0.00|0.00|1650.24|1650.24|271.44| +||8495|26034|20478|3722||10|168|213|17|||||448.46|||||||| +2452171|37860|2637|26034|20478|3722|49366|10|281|213|1|41.31|54.94|30.76|0.00|30.76|41.31|54.94|0.30|0.00|30.76|31.06|-10.55| +2452171|37860|11159|26034|20478|3722|49366|10|277|213|88|73.90|120.45|103.58|0.00|9115.04|6503.20|10599.60|91.15|0.00|9115.04|9206.19|2611.84| +2451839|32644|7573|3237|1301986|3935|11884|8|88|214|84|21.17|39.16|27.02|0.00|2269.68|1778.28|3289.44|68.09|0.00|2269.68|2337.77|491.40| +2451839|32644|7520|3237|1301986|3935|11884|8|209|214|11|58.12|59.28|21.93|0.00|241.23|639.32|652.08|4.82|0.00|241.23|246.05|-398.09| +2451839|32644|5795|3237|1301986|3935|11884|8|231|214|65|27.60|30.36|20.03|1262.89|1301.95|1794.00|1973.40|2.34|1262.89|39.06|41.40|-1754.94| +2451839||15590|||3935|11884|||214|||||503.34|1572.96|||32.08|503.34|1069.62||-4999.50| +2451839|32644|4721|3237|1301986|3935|11884|8|155|214|23|71.78|99.77|18.95|0.00|435.85|1650.94|2294.71|39.22|0.00|435.85|475.07|-1215.09| +2451839|32644|8701|3237|1301986|3935|11884|8|167|214|66|19.30|29.91|8.07|0.00|532.62|1273.80|1974.06|37.28|0.00|532.62|569.90|-741.18| +2451839|32644|10031|3237|1301986|3935|11884|8|48|214|62|40.52|57.13|7.42|0.00|460.04|2512.24|3542.06|0.00|0.00|460.04|460.04|-2052.20| +2451839|32644|12379|3237|1301986|3935|11884|8|239|214|17|35.12|54.78|1.09|0.00|18.53|597.04|931.26|0.18|0.00|18.53|18.71|-578.51| +2451839|32644|10861|3237|1301986|3935|11884|8|203|214|19|9.11|18.03|5.58|0.00|106.02|173.09|342.57|4.24|0.00|106.02|110.26|-67.07| +2451899|37791|10169|82892|1796756|2173|28837|1|151|215|33|76.23|96.04|61.46|486.76|2028.18|2515.59|3169.32|77.07|486.76|1541.42|1618.49|-974.17| +2451899|37791|10723|82892|1796756|2173|28837|1|154|215|95|43.44|76.02|15.20|0.00|1444.00|4126.80|7221.90|28.88|0.00|1444.00|1472.88|-2682.80| +2451899|37791|4016|82892|1796756|2173|28837|1|168|215|79|85.81|93.53|75.75|2214.17|5984.25|6778.99|7388.87|75.40|2214.17|3770.08|3845.48|-3008.91| +2451899|37791|3919|82892|1796756|2173|28837|1|46|215|29|76.89|83.81|20.95|0.00|607.55|2229.81|2430.49|0.00|0.00|607.55|607.55|-1622.26| +2451899|37791|9482|82892|1796756|2173|28837|1|211|215|58|94.77|151.63|34.87|0.00|2022.46|5496.66|8794.54|141.57|0.00|2022.46|2164.03|-3474.20| +2451899|37791|9434|82892|1796756|2173|28837|1|225|215|68|68.62|72.05|19.45|0.00|1322.60|4666.16|4899.40|119.03|0.00|1322.60|1441.63|-3343.56| +2451899|37791|11330|82892|1796756|2173|28837|1|173|215|21|15.26|16.32|12.72|0.00|267.12|320.46|342.72|2.67|0.00|267.12|269.79|-53.34| +||17273||1796756|2173|28837|1||215||||3.61||310.46||6210.92|12.41||||-3571.58| +2451899|37791|4778|82892|1796756|2173|28837|1|285|215|39|68.67|136.65|106.58|3948.78|4156.62|2678.13|5329.35|18.70|3948.78|207.84|226.54|-2470.29| +2451899|37791|11329|82892|1796756|2173|28837|1|81|215|4|22.61|25.54|13.53|0.00|54.12|90.44|102.16|4.32|0.00|54.12|58.44|-36.32| +2451899|37791|7532|82892|1796756|2173|28837|1|58|215|10|3.35|3.35|0.73|0.00|7.30|33.50|33.50|0.43|0.00|7.30|7.73|-26.20| +2451899|37791|1676|82892|1796756|2173|28837|1|63|215|99|92.21|113.41|77.11|0.00|7633.89|9128.79|11227.59|381.69|0.00|7633.89|8015.58|-1494.90| +2451899|37791|9812|82892|1796756|2173|28837|1|41|215|16|72.66|116.25|106.95|0.00|1711.20|1162.56|1860.00|154.00|0.00|1711.20|1865.20|548.64| +2451899|37791|11840|82892|1796756|2173|28837|1|88|215|25|78.55|118.61|112.67|0.00|2816.75|1963.75|2965.25|56.33|0.00|2816.75|2873.08|853.00| +2451899|37791|7661|82892|1796756|2173|28837|1|54|215|95|83.58|113.66|7.95|241.68|755.25|7940.10|10797.70|35.94|241.68|513.57|549.51|-7426.53| +2451899|37791|12812|82892|1796756|2173|28837|1|107|215|83|45.58|80.22|67.38|0.00|5592.54|3783.14|6658.26|0.00|0.00|5592.54|5592.54|1809.40| +2452383|59317|16657|52082|501962|7042|26122|2|257|216|45|29.45|42.40|42.40|0.00|1908.00|1325.25|1908.00|38.16|0.00|1908.00|1946.16|582.75| +2452383||6447||||26122|||216|||53.63|||733.40|1819.82|2037.94|0.00||||| +2452383|59317|14868|52082|501962|7042|26122|2|157|216|37|78.10|94.50|11.34|193.00|419.58|2889.70|3496.50|18.12|193.00|226.58|244.70|-2663.12| +2452383|59317|16008|52082|501962|7042|26122|2|207|216|86|77.84|112.86|111.73|480.43|9608.78|6694.24|9705.96|91.28|480.43|9128.35|9219.63|2434.11| +2452383|59317|1927|52082|501962|7042|26122|2|260|216|89|4.95|8.36|3.09|123.75|275.01|440.55|744.04|7.56|123.75|151.26|158.82|-289.29| +2452383|59317|14541|52082|501962|7042|26122|2|242|216|96|70.23|79.35|71.41|0.00|6855.36|6742.08|7617.60|548.42|0.00|6855.36|7403.78|113.28| +2452383|59317|3951|52082|501962|7042|26122|2|77|216|56|1.49|1.87|0.11|0.00|6.16|83.44|104.72|0.18|0.00|6.16|6.34|-77.28| +2452383|59317|3648|52082|501962|7042|26122|2|123|216|38|79.42|117.54|78.75|478.80|2992.50|3017.96|4466.52|25.13|478.80|2513.70|2538.83|-504.26| +2452383|59317|14971|52082|501962|7042|26122|2|269|216|65|58.69|72.18|37.53|0.00|2439.45|3814.85|4691.70|97.57|0.00|2439.45|2537.02|-1375.40| +2452383|59317|11973|52082|501962|7042|26122|2|66|216|75|63.38|122.32|40.36|0.00|3027.00|4753.50|9174.00|242.16|0.00|3027.00|3269.16|-1726.50| +2452383|59317|5635|52082|501962|7042|26122|2|77|216|33|21.73|39.11|21.11|0.00|696.63|717.09|1290.63|55.73|0.00|696.63|752.36|-20.46| +2452383|59317|5004|52082|501962|7042|26122|2|118|216|13|56.83|88.08|11.45|0.00|148.85|738.79|1145.04|5.95|0.00|148.85|154.80|-589.94| +2452383|59317|11829|52082|501962|7042|26122|2|50|216|53|66.94|129.86|106.48|0.00|5643.44|3547.82|6882.58|451.47|0.00|5643.44|6094.91|2095.62| +|59317|7479||||26122|||216|||90.18|66.73||||6132.24|272.25||||| +2452383|59317|11313|52082|501962|7042|26122|2|107|216|57|14.54|21.66|8.66|0.00|493.62|828.78|1234.62|4.93|0.00|493.62|498.55|-335.16| +2452383|59317|13011|52082|501962|7042|26122|2|268|216|51|99.61|108.57|49.94|0.00|2546.94|5080.11|5537.07|25.46|0.00|2546.94|2572.40|-2533.17| +2451476|53864|10570|32349|123432|2868|20751|10|119|217|75|64.34|71.41|58.55|1932.15|4391.25|4825.50|5355.75|98.36|1932.15|2459.10|2557.46|-2366.40| +2451476|53864|5443|32349|123432|2868|20751|10|234|217|59|65.75|106.51|18.10|341.72|1067.90|3879.25|6284.09|0.00|341.72|726.18|726.18|-3153.07| +2451476|53864|6010|32349|123432|2868|20751|10|30|217|76|58.57|91.36|24.66|0.00|1874.16|4451.32|6943.36|168.67|0.00|1874.16|2042.83|-2577.16| +2451476|53864|7466|32349|123432|2868|20751|10|102|217|61|66.20|94.66|46.38|0.00|2829.18|4038.20|5774.26|198.04|0.00|2829.18|3027.22|-1209.02| +2451476|53864|17068|32349|123432|2868|20751|10|224|217|92|51.63|79.51|65.99|2185.58|6071.08|4749.96|7314.92|77.71|2185.58|3885.50|3963.21|-864.46| +2451476|53864|16096|32349|123432|2868|20751|10|171|217|37|1.01|1.65|0.90|0.00|33.30|37.37|61.05|0.33|0.00|33.30|33.63|-4.07| +2451476||9932|32349|||20751|||217|7|41.21|71.70|||||501.90||||464.53|158.20| +2451476|53864|11132|32349|123432|2868|20751|10|171|217|80|3.35|3.78|2.45|0.00|196.00|268.00|302.40|15.68|0.00|196.00|211.68|-72.00| +2451476|53864|440|32349|123432|2868|20751|10|240|217|90|24.33|40.38|13.32|0.00|1198.80|2189.70|3634.20|35.96|0.00|1198.80|1234.76|-990.90| +2451476|53864|10873|32349|123432|2868|20751|10|297|217|58|41.29|57.80|43.35|0.00|2514.30|2394.82|3352.40|0.00|0.00|2514.30|2514.30|119.48| +2451476|53864|832|32349|123432|2868|20751|10|228|217|78|54.46|84.41|30.38|0.00|2369.64|4247.88|6583.98|71.08|0.00|2369.64|2440.72|-1878.24| +2451476|53864|1240|32349|123432|2868|20751|10|217|217|64|62.73|84.05|58.83|0.00|3765.12|4014.72|5379.20|301.20|0.00|3765.12|4066.32|-249.60| +2451476|53864|1543|32349|123432|2868|20751|10|291|217|24|68.91|126.79|92.55|0.00|2221.20|1653.84|3042.96|111.06|0.00|2221.20|2332.26|567.36| +||3698|||3296|28919||117|218||68.68|107.82|61.45|0.00||412.08|646.92|11.06|0.00|368.70||-43.38| +2451536|38033|16762|18097|1875273|3296|28919||43|218||79.02|||0.00|||6759.07|54.06|0.00||2757.50|| +2451536|38033|14198|18097|1875273|3296|28919|7|193|218|58|48.80|68.80|36.46|1332.24|2114.68|2830.40|3990.40|46.94|1332.24|782.44|829.38|-2047.96| +2451536|38033|15452|18097|1875273|3296|28919|7|151|218|78|55.52|66.62|8.66|0.00|675.48|4330.56|5196.36|20.26|0.00|675.48|695.74|-3655.08| +2451536|38033|14774|18097|1875273|3296|28919|7|268|218|2|51.20|64.00|10.88|0.00|21.76|102.40|128.00|0.65|0.00|21.76|22.41|-80.64| +2451536|38033|1855|18097|1875273|3296|28919|7|245|218|1|59.20|65.71|44.68|0.00|44.68|59.20|65.71|1.78|0.00|44.68|46.46|-14.52| +2451536|38033|5323|18097|1875273|3296|28919|7|53|218|85|98.13|127.56|110.97|0.00|9432.45|8341.05|10842.60|188.64|0.00|9432.45|9621.09|1091.40| +||1615|18097|1875273|3296||||218|5|||29.78|0.00||152.10|252.45||0.00|||| +2451536|38033|11930|18097|1875273|3296|28919|7|45|218|17|21.46|37.98|14.81|0.00|251.77|364.82|645.66|15.10|0.00|251.77|266.87|-113.05| +2451536|38033|2212|18097|1875273|3296|28919|7|223|218|52|65.68|95.89|43.15|0.00|2243.80|3415.36|4986.28|157.06|0.00|2243.80|2400.86|-1171.56| +2451536|38033|17120|18097|1875273|3296|28919|7|248|218|7|38.01|43.71|0.87|4.32|6.09|266.07|305.97|0.10|4.32|1.77|1.87|-264.30| +2451536|38033|10948|18097|1875273|3296|28919|7|25|218|22|14.41|16.71|4.34|46.78|95.48|317.02|367.62|0.48|46.78|48.70|49.18|-268.32| +2451022|52870|13628|5023|1803937|3651|44118|2|277|219|100|74.19|98.67|68.08|0.00|6808.00|7419.00|9867.00|408.48|0.00|6808.00|7216.48|-611.00| +|52870|13519|5023|||||110|219|92||33.32|||||3065.44|139.47||2789.44||213.44| +2451022|52870|13982|5023|1803937|3651|44118|2|66|219|91|15.99|29.10|15.42|589.35|1403.22|1455.09|2648.10|16.27|589.35|813.87|830.14|-641.22| +2451022|52870|1831|5023|1803937|3651|44118|2|168|219|33|11.26|20.94|1.04|4.11|34.32|371.58|691.02|2.71|4.11|30.21|32.92|-341.37| +2451022|52870|10960|5023|1803937|3651|44118|2|89|219|42|26.47|43.94|20.21|0.00|848.82|1111.74|1845.48|8.48|0.00|848.82|857.30|-262.92| +2451022|52870|1852|5023|1803937|3651|44118|2|192|219|29|78.31|110.41|107.09|0.00|3105.61|2270.99|3201.89|279.50|0.00|3105.61|3385.11|834.62| +2451022|52870|13672|5023|1803937|3651|44118|2|172|219|96|42.08|70.69|54.43|0.00|5225.28|4039.68|6786.24|470.27|0.00|5225.28|5695.55|1185.60| +2451022||9928||1803937||||52|219|82|5.80|||0.00|368.18|||3.68|0.00|||-107.42| +2451022|52870|3368|5023|1803937|3651|44118|2|253|219|15|25.49|30.58|7.95|0.00|119.25|382.35|458.70|9.54|0.00|119.25|128.79|-263.10| +2451022|52870|15806|5023|1803937|3651|44118|2|141|219|95|53.35|74.69|69.46|0.00|6598.70|5068.25|7095.55|0.00|0.00|6598.70|6598.70|1530.45| +2451022|52870|5395|5023|1803937|3651|44118|2|226|219|15|80.10|128.16|47.41|0.00|711.15|1201.50|1922.40|49.78|0.00|711.15|760.93|-490.35| +2451022|52870|7000|5023|1803937|3651|44118|2|199|219|28|62.93|105.72|32.77|385.37|917.56|1762.04|2960.16|5.32|385.37|532.19|537.51|-1229.85| +2451400|44235|3536|87909|455354|399|14244|4|230|220|20|69.54|106.39|6.38|0.00|127.60|1390.80|2127.80|7.65|0.00|127.60|135.25|-1263.20| +2451400|44235|15427|87909|455354|399|14244|4|94|220|67|6.72|9.07|5.80|0.00|388.60|450.24|607.69|31.08|0.00|388.60|419.68|-61.64| +2451400|44235|12574|87909|455354|399|14244|4|266|220|89|83.31|166.62|31.65|0.00|2816.85|7414.59|14829.18|0.00|0.00|2816.85|2816.85|-4597.74| +2451400|44235|3898|87909|455354|399|14244|4|79|220|94|59.54|116.10|41.79|0.00|3928.26|5596.76|10913.40|157.13|0.00|3928.26|4085.39|-1668.50| +2451400|44235|523|87909|455354|399|14244|4|53|220|48|59.42|76.65|74.35|0.00|3568.80|2852.16|3679.20|214.12|0.00|3568.80|3782.92|716.64| +2451400|44235|361|87909|455354|399|14244|4|208|220|15|14.87|15.61|7.33|0.00|109.95|223.05|234.15|3.29|0.00|109.95|113.24|-113.10| +2451400|44235|4465|87909|455354|399|14244|4|42|220|78|96.92|157.01|131.88|9257.97|10286.64|7559.76|12246.78|41.14|9257.97|1028.67|1069.81|-6531.09| +2451400|44235|9427|87909|455354|399|14244|4|253|220|28|7.80|13.10|8.77|0.00|245.56|218.40|366.80|7.36|0.00|245.56|252.92|27.16| +2451400|44235|16009|87909|455354|399|14244|4|15|220|88|44.21|72.50|47.85|0.00|4210.80|3890.48|6380.00|336.86|0.00|4210.80|4547.66|320.32| +2451400|44235|4790|87909|455354|399|14244|4|93|220|65|99.23|118.08|25.97|0.00|1688.05|6449.95|7675.20|101.28|0.00|1688.05|1789.33|-4761.90| +2451400|44235|13459|87909|455354|399|14244|4|46|220|100|26.54|29.45|21.79|0.00|2179.00|2654.00|2945.00|65.37|0.00|2179.00|2244.37|-475.00| +2451400|44235|11146|87909|455354|399|14244|4|236|220|63|60.67|72.19|57.03|0.00|3592.89|3822.21|4547.97|215.57|0.00|3592.89|3808.46|-229.32| +2451400|44235|7741|87909|455354|399|14244|4|29|220|72|20.01|28.41|2.84|0.00|204.48|1440.72|2045.52|10.22|0.00|204.48|214.70|-1236.24| +2451463|52257|17518|24985|1807869|5953|36037|2|242|221|87|56.91|84.79|79.70|0.00|6933.90|4951.17|7376.73|554.71|0.00|6933.90|7488.61|1982.73| +2451463|52257|7111|24985|1807869|5953|36037|2|136|221|21|66.40|125.49|35.13|0.00|737.73|1394.40|2635.29|7.37|0.00|737.73|745.10|-656.67| +2451463|52257|12208|24985|1807869|5953|36037|2|8|221|67|33.77|65.51|32.75|1141.01|2194.25|2262.59|4389.17|21.06|1141.01|1053.24|1074.30|-1209.35| +2451463|52257|14023|24985|1807869|5953|36037|2|286|221|35|20.70|36.01|27.72|0.00|970.20|724.50|1260.35|9.70|0.00|970.20|979.90|245.70| +|52257|2899|24985||||2|204|221|48|90.31|101.14|||3349.44||||||3516.91|-985.44| +2451463|52257|15068|24985|1807869|5953|36037|2|167|221|34|15.68|27.91|12.00|0.00|408.00|533.12|948.94|24.48|0.00|408.00|432.48|-125.12| +2451463|52257|1280|24985|1807869|5953|36037|2|14|221|89|31.25|54.06|22.70|0.00|2020.30|2781.25|4811.34|181.82|0.00|2020.30|2202.12|-760.95| +||16081||1807869|5953|36037|||221|73|||||||4370.51|119.26||||-994.26| +2451463|52257|1312|24985|1807869|5953|36037|2|125|221|31|21.88|29.10|14.55|266.11|451.05|678.28|902.10|16.64|266.11|184.94|201.58|-493.34| +2451463|52257|7135|24985|1807869|5953|36037|2|139|221|80|71.42|108.55|42.33|3352.53|3386.40|5713.60|8684.00|0.00|3352.53|33.87|33.87|-5679.73| +2451463|52257|1984|24985|1807869|5953|36037|2|55|221|26|47.09|71.57|61.55|1600.30|1600.30|1224.34|1860.82|0.00|1600.30|0.00|0.00|-1224.34| +2451494|47301|6265|12062|1846462|4862|45262|10|7|222|39|61.56|98.49|4.92|0.00|191.88|2400.84|3841.11|1.91|0.00|191.88|193.79|-2208.96| +2451494|47301|13072|12062|1846462|4862|45262|10|158|222|17|73.25|89.36|33.06|0.00|562.02|1245.25|1519.12|50.58|0.00|562.02|612.60|-683.23| +2451494|47301|4466|12062|1846462|4862|45262|10|73|222|94|4.24|4.57|3.24|271.05|304.56|398.56|429.58|0.33|271.05|33.51|33.84|-365.05| +2451494|47301|7678|12062|1846462|4862|45262|10|217|222|12|51.13|89.47|61.73|0.00|740.76|613.56|1073.64|0.00|0.00|740.76|740.76|127.20| +2451494|47301|14278|12062|1846462|4862|45262|10|44|222|32|53.43|59.84|0.00|0.00|0.00|1709.76|1914.88|0.00|0.00|0.00|0.00|-1709.76| +2451494|47301|3470|12062|1846462|4862|45262|10|28|222|83|45.65|51.12|6.64|0.00|551.12|3788.95|4242.96|16.53|0.00|551.12|567.65|-3237.83| +2451494|47301|3392|12062|1846462|4862|45262|10|273|222|41|50.76|73.60|27.96|0.00|1146.36|2081.16|3017.60|0.00|0.00|1146.36|1146.36|-934.80| +2451494|47301|9056|12062|1846462|4862|45262|10|174|222|96|66.31|100.79|78.61|0.00|7546.56|6365.76|9675.84|452.79|0.00|7546.56|7999.35|1180.80| +2452521|66369|7309|672|177589|5706|3423|2|248|223|42|26.11|31.33|9.71|0.00|407.82|1096.62|1315.86|20.39|0.00|407.82|428.21|-688.80| +2452521|66369|12409|672|177589|5706|3423|2|293|223|55|25.78|37.38|37.38|0.00|2055.90|1417.90|2055.90|102.79|0.00|2055.90|2158.69|638.00| +2452521|66369|5569|672|177589|5706|3423|2|300|223|100|60.46|67.71|52.13|0.00|5213.00|6046.00|6771.00|469.17|0.00|5213.00|5682.17|-833.00| +2452521|66369|9559|672|177589|5706|3423|2|127|223|97|91.08|117.49|45.82|0.00|4444.54|8834.76|11396.53|311.11|0.00|4444.54|4755.65|-4390.22| +2452521|66369|16789|672|177589|5706|3423|2|288|223|76|59.06|89.18|22.29|1592.39|1694.04|4488.56|6777.68|3.04|1592.39|101.65|104.69|-4386.91| +2452521|66369|13113|672|177589|5706|3423|2|109|223|15|3.46|4.04|1.05|0.00|15.75|51.90|60.60|1.10|0.00|15.75|16.85|-36.15| +2452521|66369|133|672|177589|5706|3423|2|177|223|99|95.17|106.59|38.37|2848.97|3798.63|9421.83|10552.41|18.99|2848.97|949.66|968.65|-8472.17| +2452521|66369|16393|672|177589|5706|3423|2|98|223|96|13.55|26.82|9.92|0.00|952.32|1300.80|2574.72|85.70|0.00|952.32|1038.02|-348.48| +2452521|66369|9943|672|177589|5706|3423|2|251|223|60|82.55|163.44|138.92|6501.45|8335.20|4953.00|9806.40|110.02|6501.45|1833.75|1943.77|-3119.25| +2452521|66369|553|672|177589|5706|3423|2|153|223|62|42.21|65.42|58.22|0.00|3609.64|2617.02|4056.04|288.77|0.00|3609.64|3898.41|992.62| +2452521|66369|5490|672|177589|5706|3423|2|109|223|73|27.27|38.72|7.35|0.00|536.55|1990.71|2826.56|48.28|0.00|536.55|584.83|-1454.16| +2452186|65191|6053|90791|1095821|2004|5434|4|53|224|46|82.36|131.77|85.65|1457.76|3939.90|3788.56|6061.42|99.28|1457.76|2482.14|2581.42|-1306.42| +2452186|65191|1935|90791|1095821|2004|5434|4|216|224|24|35.22|66.56|49.92|0.00|1198.08|845.28|1597.44|71.88|0.00|1198.08|1269.96|352.80| +2452186|65191|8095|90791|1095821|2004|5434|4|74|224|39|76.03|146.73|101.24|0.00|3948.36|2965.17|5722.47|236.90|0.00|3948.36|4185.26|983.19| +2452186|65191|16777|90791|1095821|2004|5434|4|195|224|91|86.42|171.97|134.13|6469.08|12205.83|7864.22|15649.27|57.36|6469.08|5736.75|5794.11|-2127.47| +2452186|65191|9403|90791|1095821|2004|5434|4|89|224|66|47.71|62.50|42.50|0.00|2805.00|3148.86|4125.00|112.20|0.00|2805.00|2917.20|-343.86| +2452186|65191|12715|90791|1095821|2004|5434|4|183|224|51|80.51|107.07|16.06|0.00|819.06|4106.01|5460.57|24.57|0.00|819.06|843.63|-3286.95| +2452186|65191|1233|90791|1095821|2004|5434|4|202|224|77|45.16|83.99|26.03|0.00|2004.31|3477.32|6467.23|20.04|0.00|2004.31|2024.35|-1473.01| +2452186|65191|415|90791|1095821|2004|5434|4|268|224|52|92.54|122.15|80.61|0.00|4191.72|4812.08|6351.80|335.33|0.00|4191.72|4527.05|-620.36| +2452186|65191|15439|90791|1095821|2004|5434|4|198|224|10|84.63|87.16|74.08|22.22|740.80|846.30|871.60|21.55|22.22|718.58|740.13|-127.72| +2452186|65191|8313|90791|1095821|2004|5434|4|65|224|20|92.03|161.97|76.12|0.00|1522.40|1840.60|3239.40|121.79|0.00|1522.40|1644.19|-318.20| +2452186|65191|11175|90791|1095821|2004|5434|4|258|224|12|7.07|12.79|6.90|0.00|82.80|84.84|153.48|4.14|0.00|82.80|86.94|-2.04| +2452186|65191|14211|90791|1095821|2004|5434|4|12|224|73|22.49|31.71|31.39|0.00|2291.47|1641.77|2314.83|137.48|0.00|2291.47|2428.95|649.70| +2452186|65191|9637|90791|1095821|2004|5434|4|78|224|55|47.38|82.44|61.83|0.00|3400.65|2605.90|4534.20|68.01|0.00|3400.65|3468.66|794.75| +2452186|65191|10707|90791|1095821|2004|5434|4|210|224|47|33.34|51.67|42.36|0.00|1990.92|1566.98|2428.49|79.63|0.00|1990.92|2070.55|423.94| +2452186|65191|9643|90791|1095821|2004|5434|4|132|224|93|8.08|9.13|4.19|0.00|389.67|751.44|849.09|0.00|0.00|389.67|389.67|-361.77| +2452186|65191|2543|90791|1095821|2004|5434|4|189|224|31|10.63|14.13|0.56|0.00|17.36|329.53|438.03|1.56|0.00|17.36|18.92|-312.17| +2451829|47109|17675|16452|341765|3878|22744|8|233|225|30|61.78|100.70|16.11|367.30|483.30|1853.40|3021.00|2.32|367.30|116.00|118.32|-1737.40| +2451829|47109|16915|16452|341765|3878|22744|8|76|225|55|35.92|39.87|28.30|793.81|1556.50|1975.60|2192.85|53.38|793.81|762.69|816.07|-1212.91| +2451829|47109|5462|16452|341765|3878|22744|8|300|225|29|90.22|94.73|38.83|0.00|1126.07|2616.38|2747.17|101.34|0.00|1126.07|1227.41|-1490.31| +2451829|47109|10945|16452|341765|3878|22744|8|181|225|18|38.21|55.78|41.27|572.00|742.86|687.78|1004.04|5.12|572.00|170.86|175.98|-516.92| +2451829|47109|17684|16452|341765|3878|22744|8|51|225|20|93.60|107.64|49.51|445.59|990.20|1872.00|2152.80|32.67|445.59|544.61|577.28|-1327.39| +2451829|47109|2639|16452|341765|3878|22744|8|294|225|57|88.28|92.69|88.98|1369.40|5071.86|5031.96|5283.33|296.19|1369.40|3702.46|3998.65|-1329.50| +2451829|47109|9920|16452|341765|3878|22744|8|44|225|83|99.74|174.54|8.72|0.00|723.76|8278.42|14486.82|65.13|0.00|723.76|788.89|-7554.66| +2451829|47109|596|16452|341765|3878|22744|8|104|225|27|66.48|96.39|13.49|0.00|364.23|1794.96|2602.53|18.21|0.00|364.23|382.44|-1430.73| +2451829|47109|1255|16452|341765|3878|22744|8|286|225|52|43.19|63.48|21.58|0.00|1122.16|2245.88|3300.96|89.77|0.00|1122.16|1211.93|-1123.72| +2451829|47109|8447|16452|341765|3878|22744|8|297|225|22|26.95|39.88|37.48|0.00|824.56|592.90|877.36|65.96|0.00|824.56|890.52|231.66| +|37694|17879||1069439||13218||2|226|19|||90.72|0.00||1702.40||155.13|0.00||1878.81|21.28| +2451865|37694|17737|8326|1069439|6105|13218|7|74|226|12|6.69|12.44|6.96|0.00|83.52|80.28|149.28|4.17|0.00|83.52|87.69|3.24| +2451865|37694|6872||1069439|6105|13218|7|168|226|59|34.84||12.43|0.00||2055.56|2445.55|36.66|0.00|||| +2451865|37694|16979|8326|1069439|6105|13218|7|43|226|79|77.75|144.61|141.71|0.00|11195.09|6142.25|11424.19|0.00|0.00|11195.09|11195.09|5052.84| +2451865|37694|7103|8326|1069439|6105|13218|7|96|226|64|76.24|123.50|79.04|2023.42|5058.56|4879.36|7904.00|60.70|2023.42|3035.14|3095.84|-1844.22| +2451865|37694|12077|8326|1069439|6105|13218|7|288|226|95|43.79|49.04|23.53|0.00|2235.35|4160.05|4658.80|201.18|0.00|2235.35|2436.53|-1924.70| +2451865|37694|914|8326|1069439|6105|13218|7|221|226|92|32.55|51.42|8.74|0.00|804.08|2994.60|4730.64|72.36|0.00|804.08|876.44|-2190.52| +||1891|8326|1069439|6105||7||226|39|44.79|89.58|73.45|0.00|2864.55|1746.81|3493.62||0.00|2864.55||1117.74| +2451865|37694|1471|8326|1069439|6105|13218|7|267|226|57|22.50|44.32|38.55|0.00|2197.35|1282.50|2526.24|65.92|0.00|2197.35|2263.27|914.85| +2451865|37694|3475|8326|1069439|6105|13218|7|183|226|99|69.75|96.95|31.99|0.00|3167.01|6905.25|9598.05|126.68|0.00|3167.01|3293.69|-3738.24| +2451865|37694|661|8326|1069439|6105|13218|7|210|226|42|30.12|51.80|5.18|0.00|217.56|1265.04|2175.60|10.87|0.00|217.56|228.43|-1047.48| +2451865|37694|17438|8326|1069439|6105|13218|7|274|226|71|17.92|34.22|29.42|1775.49|2088.82|1272.32|2429.62|12.53|1775.49|313.33|325.86|-958.99| +2451865|37694|17149|8326|1069439|6105|13218|7|178|226|91|95.72|99.54|75.65|0.00|6884.15|8710.52|9058.14|137.68|0.00|6884.15|7021.83|-1826.37| +2451865|37694|4501|8326|1069439|6105|13218|7|4|226|82|76.80|100.60|70.42|0.00|5774.44|6297.60|8249.20|0.00|0.00|5774.44|5774.44|-523.16| +2451865|37694|10430|8326|1069439|6105|13218|7|153|226|3|9.76|9.76|3.51|5.05|10.53|29.28|29.28|0.21|5.05|5.48|5.69|-23.80| +2451865|37694|12281|8326|1069439|6105|13218|7|217|226|67|10.54|19.60|17.24|0.00|1155.08|706.18|1313.20|11.55|0.00|1155.08|1166.63|448.90| +2451383|36603|13423|80855|1647581|1060|48586|7|288|227|15|23.69|40.50|0.81|0.00|12.15|355.35|607.50|0.24|0.00|12.15|12.39|-343.20| +2451383|36603|10426|80855|1647581|1060|48586|7|65|227|45|78.36|115.97|13.91|0.00|625.95|3526.20|5218.65|18.77|0.00|625.95|644.72|-2900.25| +2451383|36603|6112|80855|1647581|1060|48586|7|131|227|30|88.80|124.32|59.67|0.00|1790.10|2664.00|3729.60|35.80|0.00|1790.10|1825.90|-873.90| +2451383|36603|9860|80855|1647581|1060|48586|7|43|227|96|97.41|108.12|89.73|7235.82|8614.08|9351.36|10379.52|96.47|7235.82|1378.26|1474.73|-7973.10| +2451383|36603|10753|80855|1647581|1060|48586|7|33|227|88|92.44|113.70|53.43|2727.06|4701.84|8134.72|10005.60|78.99|2727.06|1974.78|2053.77|-6159.94| +2451383|36603|12788|80855|1647581|1060|48586|7|61|227|32|90.69|148.73|22.30|0.00|713.60|2902.08|4759.36|21.40|0.00|713.60|735.00|-2188.48| +2451383|36603|5917|80855|1647581|1060|48586|7|271|227|88|51.21|101.39|26.36|0.00|2319.68|4506.48|8922.32|208.77|0.00|2319.68|2528.45|-2186.80| +2451383|36603|13904|80855|1647581|1060|48586|7|88|227|35|43.35|79.76|42.27|0.00|1479.45|1517.25|2791.60|133.15|0.00|1479.45|1612.60|-37.80| +2451383|36603|15586|80855|1647581|1060|48586|7|77|227|99|31.72|38.38|37.61|0.00|3723.39|3140.28|3799.62|260.63|0.00|3723.39|3984.02|583.11| +2451383|36603|11740|80855|1647581|1060|48586|7|168|227|28|96.53|100.39|94.36|1664.51|2642.08|2702.84|2810.92|48.87|1664.51|977.57|1026.44|-1725.27| +2451383|36603|1534|80855|1647581|1060|48586|7|167|227|7|47.59|69.00|35.88|0.00|251.16|333.13|483.00|5.02|0.00|251.16|256.18|-81.97| +2451383|36603|16765|80855|1647581|1060|48586|7|261|227|52|33.65|59.22|39.08|0.00|2032.16|1749.80|3079.44|20.32|0.00|2032.16|2052.48|282.36| +2451383|36603|12901|80855|1647581|1060|48586|7|29|227|22|43.58|44.01|16.28|103.86|358.16|958.76|968.22|10.17|103.86|254.30|264.47|-704.46| +2451383|36603|6670|80855|1647581|1060|48586|7|97|227|1|94.86|105.29|91.60|0.00|91.60|94.86|105.29|0.91|0.00|91.60|92.51|-3.26| +2451383|36603|13084|80855|1647581|1060|48586|7|213|227|41|87.29|121.33|32.75|0.00|1342.75|3578.89|4974.53|26.85|0.00|1342.75|1369.60|-2236.14| +2451383|36603|116|80855|1647581|1060|48586|7|25|227|59|15.16|16.97|15.27|0.00|900.93|894.44|1001.23|81.08|0.00|900.93|982.01|6.49| +2451537|49083|1837|52289|393032|2575|43318|1|98|228|74|54.48|76.81|72.96|3671.34|5399.04|4031.52|5683.94|120.93|3671.34|1727.70|1848.63|-2303.82| +2451537|49083|8666|52289|393032|2575|43318|1|292|228|35|94.49|108.66|56.50|0.00|1977.50|3307.15|3803.10|138.42|0.00|1977.50|2115.92|-1329.65| +2451537|49083|12572|52289|393032|2575|43318|1|228|228|42|73.12|143.31|54.45|0.00|2286.90|3071.04|6019.02|45.73|0.00|2286.90|2332.63|-784.14| +2451537|49083|12670|52289|393032|2575|43318|1|183|228|46|15.88|16.99|3.05|0.00|140.30|730.48|781.54|1.40|0.00|140.30|141.70|-590.18| +2451537|49083|3506|52289|393032|2575|43318|1|206|228|67|31.31|49.15|44.23|1926.21|2963.41|2097.77|3293.05|41.48|1926.21|1037.20|1078.68|-1060.57| +2451537|49083|6752|52289|393032|2575|43318|1|188|228|9|20.44|36.38|10.55|0.00|94.95|183.96|327.42|6.64|0.00|94.95|101.59|-89.01| +2451537|49083|5431|52289|393032|2575|43318|1|291|228|14|80.58|149.07|81.98|0.00|1147.72|1128.12|2086.98|80.34|0.00|1147.72|1228.06|19.60| +2451537|49083|7870|52289|393032|2575|43318|1|75|228|24|54.66|85.81|11.15|0.00|267.60|1311.84|2059.44|24.08|0.00|267.60|291.68|-1044.24| +2451537|49083|13579|52289|393032|2575|43318|1|12|228|25|42.25|54.92|36.24|0.00|906.00|1056.25|1373.00|72.48|0.00|906.00|978.48|-150.25| +2451537|49083|3928|52289|393032|2575|43318|1|5|228|59|32.17|36.99|16.27|0.00|959.93|1898.03|2182.41|67.19|0.00|959.93|1027.12|-938.10| +2451537|49083|1951|52289|393032|2575|43318|1|259|228|17|97.72|102.60|25.65|0.00|436.05|1661.24|1744.20|39.24|0.00|436.05|475.29|-1225.19| +2452215|61694|15351|1438|334160|6082|36804|7|195|229|37|77.76|134.52|4.03|0.00|149.11|2877.12|4977.24|0.00|0.00|149.11|149.11|-2728.01| +2452215|61694|1261|1438|334160|6082|36804|7|204|229|9|78.40|78.40|59.58|0.00|536.22|705.60|705.60|16.08|0.00|536.22|552.30|-169.38| +2452215|61694|4515|1438|334160|6082|36804|7|250|229|93|85.78|147.54|101.80|0.00|9467.40|7977.54|13721.22|662.71|0.00|9467.40|10130.11|1489.86| +2452215|61694|13323|1438|334160|6082|36804|7|90|229|28|86.99|91.33|67.58|0.00|1892.24|2435.72|2557.24|0.00|0.00|1892.24|1892.24|-543.48| +|61694|8251|1438|334160|6082||7||229|19|47.13|55.61|||||||||330.64|-568.10| +2452215|61694|14895|1438|334160|6082|36804|7|81|229|6|31.34|33.53|24.81|0.00|148.86|188.04|201.18|2.97|0.00|148.86|151.83|-39.18| +2452215|61694|16615|1438|334160|6082|36804|7|134|229|1|82.17|84.63|18.61|0.00|18.61|82.17|84.63|0.74|0.00|18.61|19.35|-63.56| +2452215|61694|9623|1438|334160|6082|36804|7|158|229|72|36.13|66.47|22.59|0.00|1626.48|2601.36|4785.84|113.85|0.00|1626.48|1740.33|-974.88| +2452215|61694|5149|1438|334160|6082|36804|7|94|229|39|49.55|66.39|19.25|45.04|750.75|1932.45|2589.21|7.05|45.04|705.71|712.76|-1226.74| +2452215|61694|8709|1438|334160|6082|36804|7|138|229|87|26.49|41.32|20.66|0.00|1797.42|2304.63|3594.84|17.97|0.00|1797.42|1815.39|-507.21| +2452215|61694|9295|1438|334160|6082|36804|7|5|229|60|39.29|74.25|44.55|0.00|2673.00|2357.40|4455.00|26.73|0.00|2673.00|2699.73|315.60| +2451719|50610|4613|52268|1833479|6243|49007|4|183|230|88|53.48|88.24|44.12|0.00|3882.56|4706.24|7765.12|77.65|0.00|3882.56|3960.21|-823.68| +2451719|50610|6176|52268|1833479|6243|49007|4|287|230|19|22.01|27.51|19.53|0.00|371.07|418.19|522.69|0.00|0.00|371.07|371.07|-47.12| +2451719|50610|10853|52268|1833479|6243|49007|4|259|230|43|22.06|42.35|13.97|0.00|600.71|948.58|1821.05|12.01|0.00|600.71|612.72|-347.87| +2451719|50610|6212|52268|1833479|6243|49007|4|61|230|94|66.05|108.32|56.32|5294.08|5294.08|6208.70|10182.08|0.00|5294.08|0.00|0.00|-6208.70| +2451719|50610|7970|52268|1833479|6243|49007|4|253|230|1|5.78|5.78|3.81|0.00|3.81|5.78|5.78|0.34|0.00|3.81|4.15|-1.97| +2451719|50610|9181|52268|1833479|6243|49007|4|299|230|12|69.11|137.52|49.50|0.00|594.00|829.32|1650.24|41.58|0.00|594.00|635.58|-235.32| +2451719|50610|6757|52268|1833479|6243|49007|4|110|230|42|9.90|19.40|10.08|0.00|423.36|415.80|814.80|4.23|0.00|423.36|427.59|7.56| +2451719|50610|344|52268|1833479|6243|49007|4|196|230|56|27.42|42.77|39.34|0.00|2203.04|1535.52|2395.12|154.21|0.00|2203.04|2357.25|667.52| +2450951|69511|10873|89768|573061|1954|18047|10|145|231|52|35.42|63.75|12.75|0.00|663.00|1841.84|3315.00|19.89|0.00|663.00|682.89|-1178.84| +2450951|69511|832|89768|573061|1954|18047|10|296|231|18|72.88|120.98|48.39|0.00|871.02|1311.84|2177.64|43.55|0.00|871.02|914.57|-440.82| +2450951|69511|1240|89768|573061|1954|18047|10|33|231|9|15.75|23.46|19.23|13.84|173.07|141.75|211.14|4.77|13.84|159.23|164.00|17.48| +2450951|69511|1543|89768|573061|1954|18047|10|193|231|82|53.46|104.78|37.72|1113.49|3093.04|4383.72|8591.96|158.36|1113.49|1979.55|2137.91|-2404.17| +2450951|69511|15514|89768|573061|1954|18047|10|269|231|84|91.96|135.18|74.34|0.00|6244.56|7724.64|11355.12|312.22|0.00|6244.56|6556.78|-1480.08| +2450951|69511|3031|89768|573061|1954|18047|10|169|231|70|70.36|90.76|88.94|2179.03|6225.80|4925.20|6353.20|40.46|2179.03|4046.77|4087.23|-878.43| +2450951|69511|16676|89768|573061|1954|18047|10|26|231|17|6.28|11.61|0.69|11.37|11.73|106.76|197.37|0.02|11.37|0.36|0.38|-106.40| +2450951|69511|4525|89768|573061|1954|18047|10|83|231|12|2.41|3.51|1.43|0.00|17.16|28.92|42.12|1.20|0.00|17.16|18.36|-11.76| +2450951|69511|4214|89768|573061|1954|18047|10|161|231|38|22.37|26.17|1.83|0.00|69.54|850.06|994.46|4.86|0.00|69.54|74.40|-780.52| +2451791|45624|8126|32624|1355812|1047|44635|8|43|232|95|2.81|3.68|2.76|0.00|262.20|266.95|349.60|15.73|0.00|262.20|277.93|-4.75| +2451791|45624|9527||||44635|8||232|98||66.68|23.33|0.00||3867.08|6534.64||0.00|2286.34||| +2451791|45624|17582|32624|1355812|1047|44635|8|172|232|4|56.20|86.54|74.42|0.00|297.68|224.80|346.16|0.00|0.00|297.68|297.68|72.88| +2451791|45624|12326|32624|1355812|1047|44635|8|248|232|49|48.57|81.59|27.74|0.00|1359.26|2379.93|3997.91|54.37|0.00|1359.26|1413.63|-1020.67| +2451791|45624|16063||1355812|1047|||105|232||||90.04||6032.68|6200.85|8618.88|||6032.68||-168.17| +2451791|45624|1507|32624|1355812|1047|44635|8|201|232|79|75.98|90.41|31.64|0.00|2499.56|6002.42|7142.39|224.96|0.00|2499.56|2724.52|-3502.86| +2451791|45624|1508|32624|1355812|1047|44635|8|266|232|55|62.79|82.88|53.04|933.50|2917.20|3453.45|4558.40|178.53|933.50|1983.70|2162.23|-1469.75| +2451791|45624|2689|32624|1355812|1047|44635|8|189|232|37|83.29|134.92|72.85|0.00|2695.45|3081.73|4992.04|53.90|0.00|2695.45|2749.35|-386.28| +2451791|45624|9464|32624|1355812|1047|44635|8|70|232|94|50.61|98.18|34.36|0.00|3229.84|4757.34|9228.92|226.08|0.00|3229.84|3455.92|-1527.50| +||2605||225763|1913||2|222|233||91.88|135.98|121.02||||5983.12|||4952.14|5348.31|| +2452012|30039|15505|30766|225763|1913|2110|2|129|233|68|10.14|14.29|8.71|0.00|592.28|689.52|971.72|35.53|0.00|592.28|627.81|-97.24| +2452012|30039|10735|30766|225763|1913|2110|2|60|233|20|48.29|59.39|11.28|47.37|225.60|965.80|1187.80|5.34|47.37|178.23|183.57|-787.57| +2452012|30039|3993|30766|225763|1913|2110|2|208|233|91|59.08|65.57|1.96|0.00|178.36|5376.28|5966.87|16.05|0.00|178.36|194.41|-5197.92| +2452012|30039|11631|30766|225763|1913|2110|2|111|233|3|38.46|52.30|4.70|2.67|14.10|115.38|156.90|0.11|2.67|11.43|11.54|-103.95| +2452012||15683||||2110||45|233||56.14||||5322.24|3536.82|6189.12||||5588.35|| +2452012|30039|6801|30766|225763|1913|2110|2|241|233|30|80.39|91.64|36.65|0.00|1099.50|2411.70|2749.20|0.00|0.00|1099.50|1099.50|-1312.20| +2452012|30039|16447|30766|225763|1913|2110|2|287|233|25|60.74|116.01|15.08|0.00|377.00|1518.50|2900.25|15.08|0.00|377.00|392.08|-1141.50| +2452012|30039|12219|30766|225763|1913|2110|2|107|233|76|2.92|4.67|3.08|0.00|234.08|221.92|354.92|16.38|0.00|234.08|250.46|12.16| +2452012|30039|17979|30766|225763|1913|2110|2|75|233|85|93.51|141.20|121.43|6502.57|10321.55|7948.35|12002.00|114.56|6502.57|3818.98|3933.54|-4129.37| +2452012|30039|7067|30766|225763|1913|2110|2|220|233|14|68.42|74.57|21.62|0.00|302.68|957.88|1043.98|6.05|0.00|302.68|308.73|-655.20| +2452012|30039|9687|30766|225763|1913|2110|2|198|233|23|2.53|4.47|4.33|0.00|99.59|58.19|102.81|3.98|0.00|99.59|103.57|41.40| +2451295|53341|16141|12516|100793|3917|23441|10|141|234|81|12.77|15.06|11.89|0.00|963.09|1034.37|1219.86|86.67|0.00|963.09|1049.76|-71.28| +2451295|53341|12088|12516|100793|3917|23441|10|69|234|21|56.34|85.07|61.25|0.00|1286.25|1183.14|1786.47|12.86|0.00|1286.25|1299.11|103.11| +2451295|53341|11413|12516|100793|3917|23441|10|117|234|33|22.85|42.95|8.59|0.00|283.47|754.05|1417.35|0.00|0.00|283.47|283.47|-470.58| +2451295|53341|5540|12516|100793|3917|23441|10|241|234|75|64.30|96.45|46.29|0.00|3471.75|4822.50|7233.75|0.00|0.00|3471.75|3471.75|-1350.75| +2451295|53341|13108|12516|100793|3917|23441|10|64|234|9|82.58|141.21|120.02|0.00|1080.18|743.22|1270.89|97.21|0.00|1080.18|1177.39|336.96| +2451295|53341|9862|12516|100793|3917|23441|10|118|234|63|14.14|17.10|1.71|0.00|107.73|890.82|1077.30|2.15|0.00|107.73|109.88|-783.09| +2451295|53341|16999|12516|100793|3917|23441|10|96|234|15|45.33|58.92|53.02|0.00|795.30|679.95|883.80|55.67|0.00|795.30|850.97|115.35| +2451295|53341|10538|12516|100793|3917|23441|10|71|234|56|15.48|21.98|20.44|0.00|1144.64|866.88|1230.88|45.78|0.00|1144.64|1190.42|277.76| +2451295|53341|448|12516|100793|3917|23441|10|149|234|32|72.93|81.68|50.64|372.71|1620.48|2333.76|2613.76|112.29|372.71|1247.77|1360.06|-1085.99| +2451295|53341|15559|12516|100793|3917|23441|10|226|234|34|11.66|22.85|13.48|0.00|458.32|396.44|776.90|13.74|0.00|458.32|472.06|61.88| +2451295|53341|16939|12516|100793|3917|23441|10|182|234|21|1.40|1.76|0.70|0.00|14.70|29.40|36.96|1.17|0.00|14.70|15.87|-14.70| +2452524|58587|5031|43608|1522643|5678|37066|1|233|235|42|9.26|15.55|15.08|0.00|633.36|388.92|653.10|12.66|0.00|633.36|646.02|244.44| +2452524|58587|552|43608|1522643|5678|37066|1|15|235|90|39.39|66.56|18.63|0.00|1676.70|3545.10|5990.40|83.83|0.00|1676.70|1760.53|-1868.40| +2452524|58587|7326|43608|1522643|5678|37066|1|150|235|43|48.99|48.99|18.61|0.00|800.23|2106.57|2106.57|64.01|0.00|800.23|864.24|-1306.34| +2452524|58587|11868|43608|1522643|5678|37066|1|114|235|86|36.77|66.92|55.54|0.00|4776.44|3162.22|5755.12|95.52|0.00|4776.44|4871.96|1614.22| +2452524|58587|2893|43608|1522643|5678|37066|1|16|235|98|23.91|33.23|32.23|94.75|3158.54|2343.18|3256.54|153.18|94.75|3063.79|3216.97|720.61| +2452524|58587|7485|43608|1522643|5678|37066|1|94|235|97|46.49|55.32|23.23|0.00|2253.31|4509.53|5366.04|202.79|0.00|2253.31|2456.10|-2256.22| +2452524|58587|7419|43608|1522643|5678|37066|1|225|235|21|67.67|115.03|47.16|0.00|990.36|1421.07|2415.63|59.42|0.00|990.36|1049.78|-430.71| +2452524|58587|10093|43608|1522643|5678|37066|1|172|235|43|50.09|50.59|8.09|76.53|347.87|2153.87|2175.37|10.85|76.53|271.34|282.19|-1882.53| +2452524|58587|11089|43608|1522643|5678|37066|1|65|235|71|44.49|81.41|6.51|0.00|462.21|3158.79|5780.11|32.35|0.00|462.21|494.56|-2696.58| +2452524|58587|13911|43608|1522643|5678|37066|1|135|235|46|25.67|39.01|30.03|787.38|1381.38|1180.82|1794.46|17.82|787.38|594.00|611.82|-586.82| +2452524|58587|10203|43608|1522643|5678|37066|1|250|235|57|48.28|50.69|5.57|0.00|317.49|2751.96|2889.33|19.04|0.00|317.49|336.53|-2434.47| +2451127|53411|3496|79134|1489899|1817|17782|2|259|236|19|68.47|88.32|9.71|0.00|184.49|1300.93|1678.08|5.53|0.00|184.49|190.02|-1116.44| +2451127|53411|4183|79134|1489899|1817|17782|2|49|236|6|10.81|13.18|7.38|0.00|44.28|64.86|79.08|0.00|0.00|44.28|44.28|-20.58| +2451127|53411|13426|79134|1489899|1817|17782|2|131|236|63|96.75|100.62|63.39|0.00|3993.57|6095.25|6339.06|0.00|0.00|3993.57|3993.57|-2101.68| +2451127|53411|664|79134|1489899|1817|17782|2|210|236|40|16.55|32.60|31.62|0.00|1264.80|662.00|1304.00|113.83|0.00|1264.80|1378.63|602.80| +2451127|53411|11893|79134|1489899|1817|17782|2|193|236|93|9.81|14.61|9.49|0.00|882.57|912.33|1358.73|79.43|0.00|882.57|962.00|-29.76| +2451127|53411|11365|79134|1489899|1817|17782|2|79|236|6|87.36|131.04|14.41|0.00|86.46|524.16|786.24|0.00|0.00|86.46|86.46|-437.70| +2451127|53411|7114|79134|1489899|1817|17782|2|85|236|69|78.80|107.95|97.15|6368.18|6703.35|5437.20|7448.55|26.81|6368.18|335.17|361.98|-5102.03| +2451127|53411|12040|79134|1489899|1817|17782|2|34|236|36|73.35|121.02|116.17|0.00|4182.12|2640.60|4356.72|376.39|0.00|4182.12|4558.51|1541.52| +2451127|53411|5929|79134|1489899|1817|17782|2|157|236|73|83.24|86.56|37.22|0.00|2717.06|6076.52|6318.88|0.00|0.00|2717.06|2717.06|-3359.46| +2451127|53411|16670|79134|1489899|1817|17782|2|190|236|45|60.07|112.93|82.43|0.00|3709.35|2703.15|5081.85|148.37|0.00|3709.35|3857.72|1006.20| +2451127|53411|17119|79134|1489899|1817|17782|2|225|236|1|68.79|125.88|39.02|0.00|39.02|68.79|125.88|2.73|0.00|39.02|41.75|-29.77| +2451127|53411|17545|79134|1489899|1817|17782|2|274|236|25|43.31|55.43|44.89|0.00|1122.25|1082.75|1385.75|0.00|0.00|1122.25|1122.25|39.50| +||14419|||||||237||64.69|71.80|||2623.34|3752.02||||2623.34||-1128.68| +2451445|37659|4820|58516|1258378|5793|5655|1|244|237|85|44.27|70.83|11.33|0.00|963.05|3762.95|6020.55|77.04|0.00|963.05|1040.09|-2799.90| +2451445|37659|1513|58516|1258378|5793|5655|1|275|237|95|20.32|36.98|35.87|0.00|3407.65|1930.40|3513.10|204.45|0.00|3407.65|3612.10|1477.25| +2451445|37659|17786|58516|1258378|5793|5655|1|189|237|2|8.73|13.35|3.47|0.00|6.94|17.46|26.70|0.00|0.00|6.94|6.94|-10.52| +2451445|37659|10478|58516|1258378|5793|5655|1|103|237|44|94.90|148.04|137.67|0.00|6057.48|4175.60|6513.76|484.59|0.00|6057.48|6542.07|1881.88| +2451445|37659|2504|58516|1258378|5793|5655|1|42|237|98|9.95|13.23|3.43|315.97|336.14|975.10|1296.54|1.00|315.97|20.17|21.17|-954.93| +2451445|37659|14644|58516|1258378|5793|5655|1|71|237|23|93.36|140.04|60.21|692.41|1384.83|2147.28|3220.92|55.39|692.41|692.42|747.81|-1454.86| +2451445|37659|7528|58516|1258378|5793|5655|1|121|237|89|73.51|87.47|53.35|4605.70|4748.15|6542.39|7784.83|11.39|4605.70|142.45|153.84|-6399.94| +2451445|37659|15535|58516|1258378|5793|5655|1|48|237|97|80.55|152.23|45.66|1328.70|4429.02|7813.35|14766.31|31.00|1328.70|3100.32|3131.32|-4713.03| +2451445|37659|32|58516|1258378|5793|5655|1|54|237|60|97.47|121.83|40.20|0.00|2412.00|5848.20|7309.80|120.60|0.00|2412.00|2532.60|-3436.20| +2451445|37659|1922|58516|1258378|5793|5655|1|32|237|43|12.94|22.90|12.59|135.34|541.37|556.42|984.70|4.06|135.34|406.03|410.09|-150.39| +2451445|37659|1708|58516|1258378|5793|5655|1|188|237|39|2.32|4.40|0.96|0.00|37.44|90.48|171.60|0.00|0.00|37.44|37.44|-53.04| +2451445|37659|8899|58516|1258378|5793|5655|1|282|237|3|35.55|64.70|23.93|0.00|71.79|106.65|194.10|6.46|0.00|71.79|78.25|-34.86| +2451445|37659|16580|58516|1258378|5793|5655|1|222|237|40|69.35|99.86|81.88|0.00|3275.20|2774.00|3994.40|229.26|0.00|3275.20|3504.46|501.20| +2452000|31885|6335|69761|1831070|6332|4441|10|159|238|95|28.59|53.17|15.41|819.81|1463.95|2716.05|5051.15|32.20|819.81|644.14|676.34|-2071.91| +2452000|31885|1669|69761|1831070|6332|4441|10|224|238|80|2.61|4.30|2.02|92.11|161.60|208.80|344.00|5.55|92.11|69.49|75.04|-139.31| +2452000|31885|3229|69761|1831070|6332|4441|10|50|238|90|40.35|69.40|20.12|0.00|1810.80|3631.50|6246.00|126.75|0.00|1810.80|1937.55|-1820.70| +2452000|31885|16625|69761|1831070|6332|4441|10|207|238|97|36.59|68.78|33.70|3072.76|3268.90|3549.23|6671.66|7.84|3072.76|196.14|203.98|-3353.09| +2452000|31885|7707|69761|1831070|6332|4441|10|179|238|94|72.04|92.93|12.08|0.00|1135.52|6771.76|8735.42|34.06|0.00|1135.52|1169.58|-5636.24| +2452000|31885|8721|69761|1831070|6332|4441|10|165|238|98|5.88|6.70|1.34|0.00|131.32|576.24|656.60|5.25|0.00|131.32|136.57|-444.92| +2452000|31885|5595|69761|1831070|6332|4441|10|210|238|58|43.73|53.35|51.21|0.00|2970.18|2536.34|3094.30|59.40|0.00|2970.18|3029.58|433.84| +2452000|31885|10037|69761|1831070|6332|4441|10|102|238|19|16.33|28.57|15.71|41.78|298.49|310.27|542.83|15.40|41.78|256.71|272.11|-53.56| +2452000|31885|5421|69761|1831070|6332|4441|10|121|238|96|69.04|82.15|4.10|0.00|393.60|6627.84|7886.40|7.87|0.00|393.60|401.47|-6234.24| +2452000|31885|10989|69761|1831070|6332|4441|10|88|238|3|93.27|118.45|82.91|0.00|248.73|279.81|355.35|14.92|0.00|248.73|263.65|-31.08| +2452000|31885|6965|69761|1831070|6332|4441|10|170|238|97|57.46|68.37|24.61|1121.96|2387.17|5573.62|6631.89|63.26|1121.96|1265.21|1328.47|-4308.41| +2452000|31885|17209|69761|1831070|6332|4441|10|74|238|92|2.72|3.10|2.60|0.00|239.20|250.24|285.20|0.00|0.00|239.20|239.20|-11.04| +|31885|453|69761||||10|163|238|49||||118.74|565.46|510.09||26.80|118.74|||-63.37| +2452000||4267|||6332||10||238|89||||0.00|3898.20||||0.00|3898.20||-4460.68| +2452000|31885|7823|69761|1831070|6332|4441|10|81|238|19|22.40|40.54|6.48|0.00|123.12|425.60|770.26|9.84|0.00|123.12|132.96|-302.48| +2452000|31885|323|69761|1831070|6332|4441|10|191|238|95|72.12|72.12|62.74|0.00|5960.30|6851.40|6851.40|0.00|0.00|5960.30|5960.30|-891.10| +2451218|43540|14398|71550|1664599|1295|3591|2|27|239|38|24.17|34.32|13.38|0.00|508.44|918.46|1304.16|15.25|0.00|508.44|523.69|-410.02| +2451218|43540|10022|71550|1664599|1295|3591|2|162|239|88|72.50|124.70|54.86|0.00|4827.68|6380.00|10973.60|289.66|0.00|4827.68|5117.34|-1552.32| +2451218|43540|9502|71550|1664599|1295|3591|2|169|239|44|54.13|64.95|6.49|0.00|285.56|2381.72|2857.80|19.98|0.00|285.56|305.54|-2096.16| +2451218|43540|5587|71550|1664599|1295|3591|2|123|239|55|26.33|44.76|22.82|0.00|1255.10|1448.15|2461.80|87.85|0.00|1255.10|1342.95|-193.05| +2451218|43540|7852|71550|1664599|1295|3591|2|241|239|23|13.32|21.71|4.99|12.62|114.77|306.36|499.33|0.00|12.62|102.15|102.15|-204.21| +2451218|43540|2629|71550|1664599|1295|3591|2|151|239|17|90.26|91.16|82.95|0.00|1410.15|1534.42|1549.72|28.20|0.00|1410.15|1438.35|-124.27| +2451218|43540|8389|71550|1664599|1295|3591|2|134|239|86|13.22|25.38|13.95|0.00|1199.70|1136.92|2182.68|35.99|0.00|1199.70|1235.69|62.78| +2451218|43540|7820|71550|1664599|1295|3591|2|245|239|87|51.24|65.58|13.11|0.00|1140.57|4457.88|5705.46|79.83|0.00|1140.57|1220.40|-3317.31| +2451218|43540|8722|71550|1664599|1295|3591|2|123|239|47|91.53|142.78|127.07|0.00|5972.29|4301.91|6710.66|179.16|0.00|5972.29|6151.45|1670.38| +2451218|43540|14326|71550|1664599|1295|3591|2|77|239|52|44.25|76.11|73.06|0.00|3799.12|2301.00|3957.72|0.00|0.00|3799.12|3799.12|1498.12| +2451218|43540|15946|71550|1664599|1295|3591|2|98|239|78|55.62|105.67|56.00|0.00|4368.00|4338.36|8242.26|393.12|0.00|4368.00|4761.12|29.64| +2451218|43540|17311|71550|1664599|1295|3591|2|108|239|30|59.94|83.91|26.01|148.25|780.30|1798.20|2517.30|50.56|148.25|632.05|682.61|-1166.15| +2451119|70676|9781|36394|1334981|5121|38108|1|290|240|44|47.24|83.61|49.32|0.00|2170.08|2078.56|3678.84|86.80|0.00|2170.08|2256.88|91.52| +2451119||16268|36394|1334981|5121||1||240|41|||76.95||3154.95|2714.20|4207.01|||3154.95|3344.24|| +2451119|70676|13526|36394|1334981|5121|38108|1|121|240|23|1.46|1.48|1.21|0.00|27.83|33.58|34.04|0.27|0.00|27.83|28.10|-5.75| +2451119|70676|7810|36394|1334981|5121|38108|1|28|240|16|39.89|62.22|49.77|0.00|796.32|638.24|995.52|39.81|0.00|796.32|836.13|158.08| +2451119|70676|248|36394|1334981|5121|38108|1|222|240|26|55.19|93.82|52.53|0.00|1365.78|1434.94|2439.32|81.94|0.00|1365.78|1447.72|-69.16| +2451119|70676|6745|36394|1334981|5121|38108|1|214|240|50|21.07|34.34|4.12|82.40|206.00|1053.50|1717.00|3.70|82.40|123.60|127.30|-929.90| +2451119|70676|8203|36394|1334981|5121|38108|1|9|240|33|75.42|116.14|31.35|0.00|1034.55|2488.86|3832.62|51.72|0.00|1034.55|1086.27|-1454.31| +2451119|70676|4204|36394|1334981|5121|38108|1|277|240|100|5.70|6.27|2.88|0.00|288.00|570.00|627.00|25.92|0.00|288.00|313.92|-282.00| +2451119|70676|14474|36394|1334981|5121|38108|1|181|240|28|61.32|68.67|32.27|0.00|903.56|1716.96|1922.76|9.03|0.00|903.56|912.59|-813.40| +2451119|70676|9415|36394|1334981|5121|38108|1|278|240|8|66.22|121.18|84.82|0.00|678.56|529.76|969.44|27.14|0.00|678.56|705.70|148.80| +2451119|70676|5821|36394|1334981|5121|38108|1|21|240|20|87.00|152.25|47.19|0.00|943.80|1740.00|3045.00|9.43|0.00|943.80|953.23|-796.20| +2452321|51239|10185|97255|107318|2181|17444|1|228|241|56|25.28|47.52|25.18|0.00|1410.08|1415.68|2661.12|98.70|0.00|1410.08|1508.78|-5.60| +2452321|51239|10284|97255|107318|2181|17444|1|211|241|32|33.68|42.77|11.97|0.00|383.04|1077.76|1368.64|34.47|0.00|383.04|417.51|-694.72| +2452321|51239|17965|97255|107318|2181|17444|1|100|241|31|51.03|57.66|34.59|525.42|1072.29|1581.93|1787.46|10.93|525.42|546.87|557.80|-1035.06| +2452321|51239|12799|97255|107318|2181|17444|1|96|241|56|7.73|11.59|8.69|0.00|486.64|432.88|649.04|9.73|0.00|486.64|496.37|53.76| +2452321|51239|8067|97255|107318|2181|17444|1|155|241|83|18.56|24.12|0.96|0.00|79.68|1540.48|2001.96|2.39|0.00|79.68|82.07|-1460.80| +2452321|51239|16239|97255|107318|2181|17444|1|267|241|5|7.68|10.13|6.98|0.00|34.90|38.40|50.65|2.09|0.00|34.90|36.99|-3.50| +2452321|51239|17739|97255|107318|2181|17444|1|251|241|42|34.18|66.65|30.65|0.00|1287.30|1435.56|2799.30|38.61|0.00|1287.30|1325.91|-148.26| +2452321|51239|9882|97255|107318|2181|17444|1|16|241|65|64.78|127.61|98.25|0.00|6386.25|4210.70|8294.65|574.76|0.00|6386.25|6961.01|2175.55| +2452321|51239|16458|97255|107318|2181|17444|1|19|241|36|81.82|106.36|82.96|0.00|2986.56|2945.52|3828.96|0.00|0.00|2986.56|2986.56|41.04| +2452321|51239|2337|97255|107318|2181|17444|1|125|241|28|1.17|1.93|0.21|0.00|5.88|32.76|54.04|0.05|0.00|5.88|5.93|-26.88| +2452321|51239|9571|97255|107318|2181|17444|1|27|241|81|57.76|76.82|18.43|0.00|1492.83|4678.56|6222.42|0.00|0.00|1492.83|1492.83|-3185.73| +2452321|51239|10219|97255|107318|2181|17444|1|85|241|50|35.28|53.62|14.47|0.00|723.50|1764.00|2681.00|50.64|0.00|723.50|774.14|-1040.50| +2451762|34183|7868|54202|559993|4984|16022|1|239|242|38|39.84|49.00|14.70|0.00|558.60|1513.92|1862.00|5.58|0.00|558.60|564.18|-955.32| +2451762|34183|4489|54202|559993|4984|16022|1|183|242|49|60.41|79.74|28.70|0.00|1406.30|2960.09|3907.26|70.31|0.00|1406.30|1476.61|-1553.79| +2451762|34183|8425|54202|559993|4984|16022|1|55|242|86|68.45|130.05|7.80|0.00|670.80|5886.70|11184.30|60.37|0.00|670.80|731.17|-5215.90| +2451762|34183|3695|54202|559993|4984|16022|1|215|242|79|1.03|1.52|0.30|0.00|23.70|81.37|120.08|0.71|0.00|23.70|24.41|-57.67| +2451762|34183|7130|54202|559993|4984|16022|1|42|242|32|20.38|32.20|6.11|0.00|195.52|652.16|1030.40|3.91|0.00|195.52|199.43|-456.64| +2451762|34183|2546|54202|559993|4984|16022|1|41|242|25|87.15|125.49|91.60|0.00|2290.00|2178.75|3137.25|137.40|0.00|2290.00|2427.40|111.25| +2451762||10747|||||1||242|||||291.98|512.25|2205.00|||291.98|220.27|222.47|| +2451762|34183|10499|54202|||16022||264|242||57.37|66.54||0.00||2352.17||10.90|0.00||556.20|-1806.87| +2451762|34183|10679|54202|559993|4984|16022|1|272|242|65|94.91|170.83|97.37|4113.88|6329.05|6169.15|11103.95|66.45|4113.88|2215.17|2281.62|-3953.98| +2451762|34183|12914|54202|559993|4984|16022|1|206|242|75|23.17|40.08|5.21|0.00|390.75|1737.75|3006.00|31.26|0.00|390.75|422.01|-1347.00| +2451762|34183|9601|54202|559993|4984|16022|1|179|242|97|14.38|17.11|1.88|0.00|182.36|1394.86|1659.67|12.76|0.00|182.36|195.12|-1212.50| +2451762|34183|10880|54202|559993|4984|16022|1|238|242|14|50.17|69.73|56.48|0.00|790.72|702.38|976.22|0.00|0.00|790.72|790.72|88.34| +2451762|34183|16772|54202|559993|4984|16022|1|179|242|62|41.75|81.41|52.91|0.00|3280.42|2588.50|5047.42|196.82|0.00|3280.42|3477.24|691.92| +2451762|34183|16933|54202|559993|4984|16022|1|285|242|28|2.17|4.03|2.33|0.00|65.24|60.76|112.84|5.87|0.00|65.24|71.11|4.48| +2452610|45728|13053|33227|1016073|2919|22212|7|207|243|59|43.72|75.63|65.04|0.00|3837.36|2579.48|4462.17|345.36|0.00|3837.36|4182.72|1257.88| +2452610|45728|6282|33227|1016073|2919|22212|7|146|243|71|45.74|64.49|52.88|0.00|3754.48|3247.54|4578.79|37.54|0.00|3754.48|3792.02|506.94| +2452610|45728|12453|33227|1016073|2919|22212|7|295|243|60|45.72|66.75|58.07|836.20|3484.20|2743.20|4005.00|105.92|836.20|2648.00|2753.92|-95.20| +2452610|45728|3669|33227|1016073|2919|22212|7|237|243|21|49.12|52.55|26.80|0.00|562.80|1031.52|1103.55|33.76|0.00|562.80|596.56|-468.72| +2452610|45728|17481|33227|1016073|2919|22212|7|81|243|40|60.64|86.10|40.46|0.00|1618.40|2425.60|3444.00|145.65|0.00|1618.40|1764.05|-807.20| +2452610|45728|14061|33227|1016073|2919|22212|7|300|243|40|8.78|10.44|4.38|0.00|175.20|351.20|417.60|10.51|0.00|175.20|185.71|-176.00| +2452610|45728|5886|33227|1016073|2919|22212|7|185|243|86|53.54|104.93|3.14|0.00|270.04|4604.44|9023.98|16.20|0.00|270.04|286.24|-4334.40| +2452610|45728|6211|33227|1016073|2919|22212|7|5|243|1|57.59|59.31|53.97|0.00|53.97|57.59|59.31|2.69|0.00|53.97|56.66|-3.62| +2452610|45728|7290|33227|1016073|2919|22212|7|230|243|3|11.50|23.00|8.51|0.00|25.53|34.50|69.00|0.76|0.00|25.53|26.29|-8.97| +|45728|8073|33227|1016073|2919|22212||97|243||43.49|46.09||15.03|17.48|1652.62|1751.42||15.03|2.45|2.45|| +2452610|45728|1765|33227|1016073|2919|22212|7|219|243|34|63.40|86.85|40.81|0.00|1387.54|2155.60|2952.90|13.87|0.00|1387.54|1401.41|-768.06| +2452610|45728|9291|33227|1016073|2919|22212|7|141|243|47|59.99|92.98|43.70|0.00|2053.90|2819.53|4370.06|184.85|0.00|2053.90|2238.75|-765.63| +2452610|45728|13807|33227|1016073|2919|22212|7|237|243|28|38.24|51.62|28.90|0.00|809.20|1070.72|1445.36|64.73|0.00|809.20|873.93|-261.52| +2452610|45728|265|33227|1016073|2919||||243||57.61|102.54|60.49||2964.01|2822.89||48.90||||| +2452610|45728|11256|33227|1016073|2919|22212|7|76|243|68|57.61|110.03|83.62|0.00|5686.16|3917.48|7482.04|227.44|0.00|5686.16|5913.60|1768.68| +2451843|50201|15722|67342|1205311|3727|18143|8|258|244|93|15.38|20.14|8.05|0.00|748.65|1430.34|1873.02|0.00|0.00|748.65|748.65|-681.69| +2451843|50201|5678|67342|1205311|3727|18143|8|121|244|62|2.18|4.22|0.37|0.00|22.94|135.16|261.64|1.83|0.00|22.94|24.77|-112.22| +2451843|50201|16391|67342|1205311|3727|18143|8|295|244|59|97.25|134.20|34.89|1976.16|2058.51|5737.75|7917.80|0.00|1976.16|82.35|82.35|-5655.40| +2451843|50201|6073|67342|1205311|3727|18143|8|97|244|65|31.93|47.89|36.87|0.00|2396.55|2075.45|3112.85|143.79|0.00|2396.55|2540.34|321.10| +|50201|5021|67342|1205311||18143|8|93|244|24|20.40||||583.92|489.60|739.20|15.41||256.93|272.34|| +2451843|50201|14072|67342|1205311|3727|18143|8|149|244|35|95.72|137.83|130.93|0.00|4582.55|3350.20|4824.05|274.95|0.00|4582.55|4857.50|1232.35| +2451843|50201|17417|67342|1205311|3727|18143|8|277|244|49|88.87|154.63|71.12|3310.63|3484.88|4354.63|7576.87|5.22|3310.63|174.25|179.47|-4180.38| +2451843|50201|2768|67342|1205311|3727|18143|8|169|244|47|23.97|30.44|18.56|662.96|872.32|1126.59|1430.68|12.56|662.96|209.36|221.92|-917.23| +2451843|50201|17984|67342|1205311|3727|18143|8|163|244|29|82.98|117.00|76.05|0.00|2205.45|2406.42|3393.00|132.32|0.00|2205.45|2337.77|-200.97| +2451843|50201|2999|67342|1205311|3727|18143|8|78|244|2|51.37|82.70|52.92|0.00|105.84|102.74|165.40|3.17|0.00|105.84|109.01|3.10| +2451843|50201|3713|67342|1205311|3727|18143|8|17|244|53|46.98|85.50|17.95|304.43|951.35|2489.94|4531.50|19.40|304.43|646.92|666.32|-1843.02| +2451843|50201|6182|67342|1205311|3727|18143|8|57|244|39|56.13|111.13|37.78|0.00|1473.42|2189.07|4334.07|132.60|0.00|1473.42|1606.02|-715.65| +2452566|39582|12897|67416|674271|2373|10828|7|131|245|4|92.63|159.32|111.52|0.00|446.08|370.52|637.28|31.22|0.00|446.08|477.30|75.56| +2452566|39582|6342|67416|674271|2373|10828|7|5|245|57|41.30|61.95|30.97|0.00|1765.29|2354.10|3531.15|17.65|0.00|1765.29|1782.94|-588.81| +2452566|39582|10297|67416|674271|2373|10828|7|44|245|2|92.14|121.62|96.07|0.00|192.14|184.28|243.24|17.29|0.00|192.14|209.43|7.86| +2452566|39582|7905|67416|674271|2373|10828|7|158|245|97|65.08|116.49|18.63|0.00|1807.11|6312.76|11299.53|18.07|0.00|1807.11|1825.18|-4505.65| +2452566|39582|14448|67416|674271|2373|10828|7|130|245|92|7.17|8.10|8.10|0.00|745.20|659.64|745.20|7.45|0.00|745.20|752.65|85.56| +2452566|39582|4482|67416|674271|2373|10828|7|67|245|12|77.28|108.19|84.38|749.29|1012.56|927.36|1298.28|23.69|749.29|263.27|286.96|-664.09| +||14667|||2373|||189|245||||5.26||||1578.42|4.51||150.55||| +2452566|39582|12421|67416|674271|2373|10828|7|74|245|99|13.70|26.85|10.47|0.00|1036.53|1356.30|2658.15|62.19|0.00|1036.53|1098.72|-319.77| +2452566|39582|15859|67416|674271|2373|10828|7|173|245|98|96.21|114.48|57.24|0.00|5609.52|9428.58|11219.04|280.47|0.00|5609.52|5889.99|-3819.06| +2452566|39582|6996|67416|674271|2373|10828|7|207|245|46|47.31|54.87|30.17|0.00|1387.82|2176.26|2524.02|83.26|0.00|1387.82|1471.08|-788.44| +2452566|39582|505|67416|674271|2373|10828|7|287|245|48|98.92|116.72|88.70|0.00|4257.60|4748.16|5602.56|255.45|0.00|4257.60|4513.05|-490.56| +2452566|39582|10071|67416|674271|2373|10828|7|287|245|100|11.56|22.65|2.03|0.00|203.00|1156.00|2265.00|12.18|0.00|203.00|215.18|-953.00| +2452566|39582|12168|67416|674271|2373|10828|7|10|245|16|98.16|121.71|102.23|0.00|1635.68|1570.56|1947.36|0.00|0.00|1635.68|1635.68|65.12| +2452566|39582|7818|67416|674271|2373|10828|7|203|245|73|58.31|81.05|30.79|1708.22|2247.67|4256.63|5916.65|37.76|1708.22|539.45|577.21|-3717.18| +2452566|39582|6985|67416|674271|2373|10828|7|253|245|14|20.08|29.11|11.35|0.00|158.90|281.12|407.54|11.12|0.00|158.90|170.02|-122.22| +2452149|52322|16677|70315|1608702|336|9570|8|284|246|11|76.86|100.68|92.62|81.50|1018.82|845.46|1107.48|56.23|81.50|937.32|993.55|91.86| +2452149|52322|4525|70315|1608702|336|9570|8|29|246|37|88.82|170.53|66.50|0.00|2460.50|3286.34|6309.61|73.81|0.00|2460.50|2534.31|-825.84| +2452149|52322|4215|70315|1608702|336|9570|8|198|246|79|27.86|33.71|19.21|0.00|1517.59|2200.94|2663.09|45.52|0.00|1517.59|1563.11|-683.35| +2452149|52322|991|70315|1608702|336|9570|8|130|246|82|12.73|23.04|6.22|0.00|510.04|1043.86|1889.28|30.60|0.00|510.04|540.64|-533.82| +2452149|52322|16279|70315|1608702|336|9570|8|228|246|17|43.96|81.76|43.33|0.00|736.61|747.32|1389.92|29.46|0.00|736.61|766.07|-10.71| +2452149|52322|16489|70315|1608702|336|9570|8|234|246|23|81.11|137.07|2.74|0.00|63.02|1865.53|3152.61|5.67|0.00|63.02|68.69|-1802.51| +2452149|52322|1817|70315|1608702|336|9570|8|152|246|36|66.96|72.31|33.98|0.00|1223.28|2410.56|2603.16|97.86|0.00|1223.28|1321.14|-1187.28| +2452149|52322|7573|70315|1608702|336|9570|8|22|246|36|74.51|125.17|92.62|0.00|3334.32|2682.36|4506.12|133.37|0.00|3334.32|3467.69|651.96| +2451157|39778|6812|63981|1732465|5749|5033|2|112|247|67|85.25|88.66|22.16|103.93|1484.72|5711.75|5940.22|41.42|103.93|1380.79|1422.21|-4330.96| +|39778|3892|63981||5749|||297|247|88||59.50||0.00|3665.20|4717.68|5236.00|73.30|0.00|3665.20||| +2451157|39778|17732|63981|1732465|5749|5033|2|165|247|85|64.86|79.77|75.78|0.00|6441.30|5513.10|6780.45|128.82|0.00|6441.30|6570.12|928.20| +2451157|39778|10432|63981|1732465|5749|5033|2|137|247|64|28.79|38.86|32.25|0.00|2064.00|1842.56|2487.04|123.84|0.00|2064.00|2187.84|221.44| +2451157|39778|4172|63981|1732465|5749|5033|2|193|247|58|37.77|40.79|40.79|0.00|2365.82|2190.66|2365.82|70.97|0.00|2365.82|2436.79|175.16| +2451157|39778|1622|63981|1732465|5749|5033|2|161|247|59|35.58|65.82|15.79|0.00|931.61|2099.22|3883.38|9.31|0.00|931.61|940.92|-1167.61| +2451157|39778|17200|63981|1732465|5749|5033|2|261|247|66|36.05|41.09|10.68|289.00|704.88|2379.30|2711.94|37.42|289.00|415.88|453.30|-1963.42| +2451157|39778|17818|63981|1732465|5749|5033|2|201|247|92|38.44|60.73|59.51|0.00|5474.92|3536.48|5587.16|109.49|0.00|5474.92|5584.41|1938.44| +2451157|39778|5144|63981|1732465|5749|5033|2|149|247|99|72.22|144.44|116.99|0.00|11582.01|7149.78|14299.56|694.92|0.00|11582.01|12276.93|4432.23| +2451794|38092|1892|38147|125144|4941|16338|1|106|248|25|19.45|37.14|9.65|0.00|241.25|486.25|928.50|21.71|0.00|241.25|262.96|-245.00| +2451794|38092|15437|38147|125144|4941|16338|1|123|248|86|25.44|25.69|2.82|0.00|242.52|2187.84|2209.34|21.82|0.00|242.52|264.34|-1945.32| +2451794|38092|3317|38147||4941||1||248||||169.38|0.00||||955.30|0.00||16877.02|7173.14| +2451794|38092|2210|38147|125144|4941|16338|1|158|248|75|86.82|133.70|53.48|561.54|4011.00|6511.50|10027.50|103.48|561.54|3449.46|3552.94|-3062.04| +2451794|38092|12805|38147|125144|4941|16338|1|228|248|91|86.86|118.12|25.98|0.00|2364.18|7904.26|10748.92|141.85|0.00|2364.18|2506.03|-5540.08| +2451794|38092|7249|38147|125144|4941|16338|1|10|248|17|23.92|40.66|21.95|0.00|373.15|406.64|691.22|26.12|0.00|373.15|399.27|-33.49| +2451794|38092|3161|38147|125144|4941|16338|1|26|248|29|56.68|62.91|4.40|0.00|127.60|1643.72|1824.39|0.00|0.00|127.60|127.60|-1516.12| +2451794|38092|10141|38147|125144|4941|16338|1|79|248|50|82.86|127.60|38.28|1780.02|1914.00|4143.00|6380.00|6.69|1780.02|133.98|140.67|-4009.02| +2451794|38092|1301|38147|125144|4941|16338|1|10|248|40|57.23|73.25|34.42|0.00|1376.80|2289.20|2930.00|123.91|0.00|1376.80|1500.71|-912.40| +2451794|38092|14155|38147|125144|4941|16338|1|36|248|72|23.68|40.96|35.22|1217.20|2535.84|1704.96|2949.12|0.00|1217.20|1318.64|1318.64|-386.32| +2451794|38092|4463|38147|125144|4941|16338|1|294|248|96|49.85|84.74|16.10|0.00|1545.60|4785.60|8135.04|15.45|0.00|1545.60|1561.05|-3240.00| +2451794|38092|4201|38147|125144|4941|16338|1|58|248|23|80.08|92.89|24.15|0.00|555.45|1841.84|2136.47|38.88|0.00|555.45|594.33|-1286.39| +2451799|39448|16625|15004|139659|4761|7463|1|237|249|11|33.77|48.62|35.97|0.00|395.67|371.47|534.82|35.61|0.00|395.67|431.28|24.20| +2451799|39448|7706|15004|139659|4761|7463|1|285|249|12|69.66|112.15|76.26|9.15|915.12|835.92|1345.80|63.41|9.15|905.97|969.38|70.05| +2451799|39448|8720|15004|139659|4761|7463|1|192|249|22|96.77|185.79|61.31|0.00|1348.82|2128.94|4087.38|13.48|0.00|1348.82|1362.30|-780.12| +2451799|39448|5594|15004|139659|4761|7463|1|229|249|67|38.96|66.23|1.32|0.00|88.44|2610.32|4437.41|4.42|0.00|88.44|92.86|-2521.88| +2451799|39448|10037|15004|139659|4761|7463|1|223|249|54|38.63|73.39|65.31|0.00|3526.74|2086.02|3963.06|211.60|0.00|3526.74|3738.34|1440.72| +2451799|39448|5420|15004|139659|4761|7463|1|201|249|84|68.63|73.43|55.07|0.00|4625.88|5764.92|6168.12|323.81|0.00|4625.88|4949.69|-1139.04| +2451799|39448|10988|15004|139659|4761|7463|1|146|249|74|70.31|134.29|59.08|0.00|4371.92|5202.94|9937.46|262.31|0.00|4371.92|4634.23|-831.02| +2451799|39448|6965|15004|139659|4761|7463|1|86|249|40|11.60|14.50|13.19|0.00|527.60|464.00|580.00|21.10|0.00|527.60|548.70|63.60| +2451799|39448|17209|15004|139659|4761|7463|1|25|249|2|34.58|52.21|20.36|0.00|40.72|69.16|104.42|1.22|0.00|40.72|41.94|-28.44| +2451799|39448|452|15004|139659|4761|7463|1|24|249|35|98.74|183.65|126.71|0.00|4434.85|3455.90|6427.75|88.69|0.00|4434.85|4523.54|978.95| +2451799|39448|4267|15004|139659|4761|7463|1|4|249|55|49.00|76.93|73.08|0.00|4019.40|2695.00|4231.15|200.97|0.00|4019.40|4220.37|1324.40| +2451799|39448|7823|15004|139659|4761|7463|1|200|249|100|78.06|106.94|66.30|0.00|6630.00|7806.00|10694.00|265.20|0.00|6630.00|6895.20|-1176.00| +2451799|39448|323||||7463|1|138|249|60|48.00|||0.00|||5702.40|233.79|0.00|4675.80|4909.59|1795.80| +2451799|39448|11137|15004|139659|4761|7463|1|33|249|17|48.47|66.40|32.53|0.00|553.01|823.99|1128.80|5.53|0.00|553.01|558.54|-270.98| +2451799|39448|13793|15004|139659|4761|7463|1|256|249|38|62.19|69.65|18.80|0.00|714.40|2363.22|2646.70|14.28|0.00|714.40|728.68|-1648.82| +2452577|40872|2385|88951|264554|1310|4113|1|213|250|23|22.16|25.26|13.89|281.13|319.47|509.68|580.98|0.00|281.13|38.34|38.34|-471.34| +2452577|40872|8005|88951|264554|1310|4113|1|21|250|89|50.89|95.67|79.40|282.66|7066.60|4529.21|8514.63|610.55|282.66|6783.94|7394.49|2254.73| +2452577|40872|787|88951|264554|1310|4113|1|290|250|94|63.20|120.71|71.21|0.00|6693.74|5940.80|11346.74|200.81|0.00|6693.74|6894.55|752.94| +2452577|40872|8109|88951|264554|1310|4113|1|175|250|4|36.44|44.82|42.57|0.00|170.28|145.76|179.28|11.91|0.00|170.28|182.19|24.52| +2452577|40872|6613|88951|264554|1310|4113|1|161|250|14|33.33|55.66|35.06|0.00|490.84|466.62|779.24|39.26|0.00|490.84|530.10|24.22| +2452577|40872|7209|88951|264554|1310|4113|1|268|250|56|65.77|98.65|16.77|0.00|939.12|3683.12|5524.40|65.73|0.00|939.12|1004.85|-2744.00| +2452577|40872|11940|88951|264554|1310|4113|1|187|250|87|87.33|92.56|46.28|4026.36|4026.36|7597.71|8052.72|0.00|4026.36|0.00|0.00|-7597.71| +2452577|40872|4443|88951|264554|1310|4113|1|94|250|40|78.76|143.34|117.53|0.00|4701.20|3150.40|5733.60|47.01|0.00|4701.20|4748.21|1550.80| +2452577|40872|10776|88951|264554|1310|4113|1|89|250|81|25.64|30.25|30.25|0.00|2450.25|2076.84|2450.25|73.50|0.00|2450.25|2523.75|373.41| +2452577|40872|6720|88951|264554|1310|4113|1|243|250|90|66.74|79.42|22.23|220.07|2000.70|6006.60|7147.80|35.61|220.07|1780.63|1816.24|-4225.97| +2452577|40872|15423|88951|264554|1310|4113|1|105|250|87|74.84|115.25|26.50|0.00|2305.50|6511.08|10026.75|69.16|0.00|2305.50|2374.66|-4205.58| +2452577|40872|12249|88951|264554|1310|4113|1|72|250|13|35.99|42.46|36.51|0.00|474.63|467.87|551.98|9.49|0.00|474.63|484.12|6.76| +2452577||9619|88951|264554|1310||1||250|70|95.63||34.42|||||0.00||2409.40|2409.40|-4284.70| +2452577|40872|6054|88951|264554|1310|4113|1|113|250|98|27.42|31.80|6.36|0.00|623.28|2687.16|3116.40|49.86|0.00|623.28|673.14|-2063.88| +2452577|40872|1935|88951|264554|1310|4113|1|151|250|59|27.10|42.54|16.59|0.00|978.81|1598.90|2509.86|9.78|0.00|978.81|988.59|-620.09| +2452577|40872|8095|88951|264554|1310|4113|1|3|250|51|37.90|43.20|32.83|502.29|1674.33|1932.90|2203.20|46.88|502.29|1172.04|1218.92|-760.86| +2452045|37556|8867|52127|1281990|3406|39669|7|273|251|26|92.32|183.71|25.71|213.90|668.46|2400.32|4776.46|18.18|213.90|454.56|472.74|-1945.76| +2452045|37556|12385|52127|1281990|3406|39669|7|220|251|57|19.99|33.98|5.43|0.00|309.51|1139.43|1936.86|21.66|0.00|309.51|331.17|-829.92| +2452045|37556|14929|52127|1281990|3406|39669|7|79|251|12|24.27|33.00|21.45|0.00|257.40|291.24|396.00|15.44|0.00|257.40|272.84|-33.84| +2452045|37556|6881|52127|1281990|3406|39669|7|114|251|6|58.66|114.38|40.03|163.32|240.18|351.96|686.28|0.00|163.32|76.86|76.86|-275.10| +2452045|37556|17287|52127|1281990|3406|39669|7|87|251|78|47.19|66.53|52.55|0.00|4098.90|3680.82|5189.34|286.92|0.00|4098.90|4385.82|418.08| +2452045||9309|52127||||7|286|251|65|34.37|61.86||0.00|3136.25|2234.05|||0.00|3136.25||| +2452045|37556|1043|52127|1281990|3406|39669|7|106|251|98|12.10|17.90|5.90|0.00|578.20|1185.80|1754.20|34.69|0.00|578.20|612.89|-607.60| +2452045|37556|15959|52127|1281990|3406|39669|7|242|251|29|86.86|148.53|111.39|0.00|3230.31|2518.94|4307.37|193.81|0.00|3230.31|3424.12|711.37| +2452045|37556|3443|52127|1281990|3406|39669|7|265|251|53|85.06|85.91|57.55|0.00|3050.15|4508.18|4553.23|244.01|0.00|3050.15|3294.16|-1458.03| +2452045|37556|16465|52127|1281990|3406|39669|7|226|251|66|56.05|111.53|88.10|0.00|5814.60|3699.30|7360.98|0.00|0.00|5814.60|5814.60|2115.30| +2452045||5135|52127|1281990|3406||7|188|251|10||||0.00||314.90|497.50||0.00|139.30||-175.60| +2452045|37556|4703|52127|1281990|3406|39669|7|10|251|35|71.44|71.44|41.43|0.00|1450.05|2500.40|2500.40|29.00|0.00|1450.05|1479.05|-1050.35| +2452045|37556|4581|52127|1281990|3406|39669|7|109|251|29|67.51|104.64|97.31|0.00|2821.99|1957.79|3034.56|197.53|0.00|2821.99|3019.52|864.20| +2452045|37556|10041|52127|1281990|3406|39669|7|248|251|13|94.87|102.45|63.51|0.00|825.63|1233.31|1331.85|74.30|0.00|825.63|899.93|-407.68| +2452045|37556|12141|52127|1281990|3406|39669|7|52|251|50|44.95|75.51|36.99|0.00|1849.50|2247.50|3775.50|73.98|0.00|1849.50|1923.48|-398.00| +2452045|37556|3069|52127|1281990|3406|39669|7|125|251|64|8.62|10.25|6.76|168.72|432.64|551.68|656.00|5.27|168.72|263.92|269.19|-287.76| +||14664|93856||45||2|79|252|||24.94|||1329.56||2144.84|||1329.56||-91.16| +2452593|64459|6171|93856|906374|45|26163|2|68|252|17|31.39|37.35|22.41|0.00|380.97|533.63|634.95|22.85|0.00|380.97|403.82|-152.66| +2452593|64459|16597|93856|906374|45|26163|2|208|252|59|64.75|69.28|12.47|0.00|735.73|3820.25|4087.52|51.50|0.00|735.73|787.23|-3084.52| +2452593|64459|5295|93856|906374|45|26163|2|34|252|69|65.77|108.52|33.64|0.00|2321.16|4538.13|7487.88|0.00|0.00|2321.16|2321.16|-2216.97| +2452593|64459|1911|93856|906374|45|26163|2|66|252|85|68.22|107.78|35.56|0.00|3022.60|5798.70|9161.30|151.13|0.00|3022.60|3173.73|-2776.10| +2452593|64459|15033|93856|906374|45|26163|2|228|252|2|46.72|71.01|63.90|0.00|127.80|93.44|142.02|7.66|0.00|127.80|135.46|34.36| +2452593|64459|1326|93856|906374|45|26163|2|236|252|85|49.87|76.79|14.59|0.00|1240.15|4238.95|6527.15|37.20|0.00|1240.15|1277.35|-2998.80| +2452593|64459|5497|93856|906374|45|26163|2|70|252|61|31.96|58.80|20.58|0.00|1255.38|1949.56|3586.80|100.43|0.00|1255.38|1355.81|-694.18| +2452593|64459|7974|93856|906374|45|26163|2|248|252|79|92.78|137.31|12.35|0.00|975.65|7329.62|10847.49|9.75|0.00|975.65|985.40|-6353.97| +2452593|64459|3111|93856|906374|45|26163|2|38|252|58|95.87|182.15|67.39|0.00|3908.62|5560.46|10564.70|351.77|0.00|3908.62|4260.39|-1651.84| +2452593|64459|15789|93856|906374|45|26163|2|41|252|56|81.16|126.60|53.17|446.62|2977.52|4544.96|7089.60|75.92|446.62|2530.90|2606.82|-2014.06| +|64459|781|93856|||26163||228|252||39.62|45.16|23.03||||2032.20||||1088.16|-746.55| +2452593|64459|15792|93856|906374|45|26163|2|247|252|91|33.56|46.98|12.68|726.94|1153.88|3053.96|4275.18|17.07|726.94|426.94|444.01|-2627.02| +2452593|64459|4602|93856|906374|45|26163|2|28|252|6|67.76|107.73|6.46|32.55|38.76|406.56|646.38|0.37|32.55|6.21|6.58|-400.35| +2451521||5962|25000|315309|6418||10||253||||3.83|0.00||706.18|1398.10||0.00|237.46||| +2451521|29884|11012|25000|315309|6418|49127|10|156|253|41|21.95|30.29|29.38|277.05|1204.58|899.95|1241.89|27.82|277.05|927.53|955.35|27.58| +2451521|29884|2143|25000|315309|6418|49127|10|46|253|8|71.27|131.84|76.46|0.00|611.68|570.16|1054.72|12.23|0.00|611.68|623.91|41.52| +2451521|29884|15148|25000|315309|6418|49127|10|23|253|35|69.95|91.63|14.66|0.00|513.10|2448.25|3207.05|0.00|0.00|513.10|513.10|-1935.15| +2451521|29884|14770|25000|315309|6418|49127|10|26|253|56|40.31|47.56|35.19|0.00|1970.64|2257.36|2663.36|0.00|0.00|1970.64|1970.64|-286.72| +2451521|29884|11755|25000|315309|6418|49127|10|205|253|66|17.99|21.04|14.30|0.00|943.80|1187.34|1388.64|28.31|0.00|943.80|972.11|-243.54| +2451521|29884|9583|25000|315309|6418|49127|10|124|253|37|85.79|171.58|156.13|0.00|5776.81|3174.23|6348.46|173.30|0.00|5776.81|5950.11|2602.58| +2451521|29884|7286|25000|315309|6418|49127|10|268|253|65|13.02|22.91|18.55|0.00|1205.75|846.30|1489.15|60.28|0.00|1205.75|1266.03|359.45| +2451521|29884|3013|25000|315309|6418|49127|10|274|253|15|18.94|32.38|1.94|0.00|29.10|284.10|485.70|0.87|0.00|29.10|29.97|-255.00| +2451521|29884|14030|25000|315309|6418|49127|10|121|253|27|95.69|168.41|122.93|0.00|3319.11|2583.63|4547.07|298.71|0.00|3319.11|3617.82|735.48| +2451521|29884|6524|25000|315309|6418|49127|10|171|253|66|18.68|28.95|18.81|0.00|1241.46|1232.88|1910.70|24.82|0.00|1241.46|1266.28|8.58| +2451521|29884|10024|25000|315309|6418|49127|10|37|253|11|92.18|98.63|30.57|0.00|336.27|1013.98|1084.93|20.17|0.00|336.27|356.44|-677.71| +2451521|29884|14341|25000|315309|6418|49127|10|44|253|33|5.56|10.06|1.10|0.00|36.30|183.48|331.98|0.00|0.00|36.30|36.30|-147.18| +2451521|29884|8510|25000|315309|6418|49127|10|13|253|16|90.03|147.64|140.25|0.00|2244.00|1440.48|2362.24|157.08|0.00|2244.00|2401.08|803.52| +2451521|29884|904|25000|315309|6418|49127|10|54|253|80|74.98|126.71|108.97|0.00|8717.60|5998.40|10136.80|348.70|0.00|8717.60|9066.30|2719.20| +2452535|62578|3109|47017|1823832|5894|39509|8|300|254|97|49.45|98.40|34.44|0.00|3340.68|4796.65|9544.80|200.44|0.00|3340.68|3541.12|-1455.97| +2452535|62578|2436|47017|1823832|5894|39509|8|207|254|78|28.33|51.27|19.99|0.00|1559.22|2209.74|3999.06|0.00|0.00|1559.22|1559.22|-650.52| +2452535|62578|513|47017|1823832|5894|39509|8|286|254|60|34.31|65.18|14.33|0.00|859.80|2058.60|3910.80|8.59|0.00|859.80|868.39|-1198.80| +||5748|47017|||||198|254|||38.13||0.00||2328.80|||0.00||1827.56|-620.80| +2452535|62578|13029|47017|1823832|5894|39509|8|76|254|73|21.72|26.28|9.46|89.77|690.58|1585.56|1918.44|24.03|89.77|600.81|624.84|-984.75| +2452535|62578|15601|47017|1823832|5894|39509|8|46|254|73|56.56|92.75|22.26|0.00|1624.98|4128.88|6770.75|146.24|0.00|1624.98|1771.22|-2503.90| +2452535|62578|14851|47017|1823832|5894|39509|8|17|254|60|4.07|6.47|6.27|210.67|376.20|244.20|388.20|14.89|210.67|165.53|180.42|-78.67| +2452535|62578|7788|47017|1823832|5894|39509|8|152|254|28|55.00|78.10|67.94|741.90|1902.32|1540.00|2186.80|104.43|741.90|1160.42|1264.85|-379.58| +2452535|62578|9870|47017|1823832|5894|39509|8|255|254|35|81.66|119.22|21.45|0.00|750.75|2858.10|4172.70|67.56|0.00|750.75|818.31|-2107.35| +2450824|38102|4994|98663|1666751|439|35727|4|73|255|87|63.85|99.60|61.75|0.00|5372.25|5554.95|8665.20|268.61|0.00|5372.25|5640.86|-182.70| +2450824|38102|16754|98663|1666751|439|35727|4|1|255|12|79.38|148.44|106.87|0.00|1282.44|952.56|1781.28|25.64|0.00|1282.44|1308.08|329.88| +2450824|38102|5632|98663|1666751|439|35727|4|169|255|63|68.83|105.99|105.99|0.00|6677.37|4336.29|6677.37|467.41|0.00|6677.37|7144.78|2341.08| +2450824|38102|17726|98663|1666751|439|35727|4|116|255|29|26.19|48.97|39.66|0.00|1150.14|759.51|1420.13|57.50|0.00|1150.14|1207.64|390.63| +2450824|38102|7702|98663|1666751|439|35727|4|105|255|92|8.15|10.10|1.31|0.00|120.52|749.80|929.20|9.64|0.00|120.52|130.16|-629.28| +2450824|38102|10904|98663|1666751|439|35727|4|49|255|95|24.29|28.90|2.89|0.00|274.55|2307.55|2745.50|13.72|0.00|274.55|288.27|-2033.00| +2450824|38102|9691|98663|1666751|439|35727|4|208|255|33|38.08|67.40|55.94|0.00|1846.02|1256.64|2224.20|18.46|0.00|1846.02|1864.48|589.38| +2450824|38102|8206|98663|1666751|439|35727|4|55|255|97|63.95|124.06|64.51|3379.03|6257.47|6203.15|12033.82|230.27|3379.03|2878.44|3108.71|-3324.71| +2450824|38102|1876|98663|1666751|439|35727|4|288|255|48|6.53|6.72|4.56|0.00|218.88|313.44|322.56|13.13|0.00|218.88|232.01|-94.56| +2450824|38102|3028|98663|1666751|439|35727|4|44|255|50|85.50|155.61|80.91|0.00|4045.50|4275.00|7780.50|161.82|0.00|4045.50|4207.32|-229.50| +2450824|38102|10702|98663|1666751|439|35727|4|185|255|88|28.78|38.56|17.35|0.00|1526.80|2532.64|3393.28|15.26|0.00|1526.80|1542.06|-1005.84| +2450824|38102|3907|98663|1666751|439|35727|4|67|255|98|46.91|50.19|22.08|0.00|2163.84|4597.18|4918.62|108.19|0.00|2163.84|2272.03|-2433.34| +2451255|30459|122|48420|543719|971|26223|1|242|256|65|41.15|79.00|71.10|0.00|4621.50|2674.75|5135.00|46.21|0.00|4621.50|4667.71|1946.75| +||5108||543719||||79|256|45|1.91|3.13||||85.95||1.30||130.95||| +2451255|30459|5893|48420|543719|971|26223|1|154|256|41|34.37|37.11|36.36|0.00|1490.76|1409.17|1521.51|119.26|0.00|1490.76|1610.02|81.59| +2451255|30459|15901|48420|543719|971|26223|1|110|256|80|44.79|80.62|75.78|0.00|6062.40|3583.20|6449.60|424.36|0.00|6062.40|6486.76|2479.20| +||4417|48420|543719||26223|||256|34|25.79|30.43|27.99|0.00|951.66||1034.62||0.00||970.69|74.80| +2451255|30459|15736|48420|543719|971|26223|1|46|256|60|63.75|120.48|12.04|0.00|722.40|3825.00|7228.80|21.67|0.00|722.40|744.07|-3102.60| +2451255|30459|11128|48420|543719|971|26223|1|41|256|82|93.93|156.86|116.07|0.00|9517.74|7702.26|12862.52|95.17|0.00|9517.74|9612.91|1815.48| +2451255|30459|8092|48420|543719|971|26223|1|21|256|79|83.73|144.85|137.60|543.52|10870.40|6614.67|11443.15|309.80|543.52|10326.88|10636.68|3712.21| +2451255|30459|10444|48420|543719|971|26223|1|255|256|46|90.47|159.22|154.44|0.00|7104.24|4161.62|7324.12|213.12|0.00|7104.24|7317.36|2942.62| +2451255|30459|11750|48420|543719|971|26223|1|253|256|6|62.32|120.27|0.00|0.00|0.00|373.92|721.62|0.00|0.00|0.00|0.00|-373.92| +2451485|47069|10406|42884|37976|1515|44654|7|264|257|19|61.35|107.97|50.74|0.00|964.06|1165.65|2051.43|48.20|0.00|964.06|1012.26|-201.59| +2451485|47069|8876|42884|37976|1515|44654|7|46|257|17|38.74|76.31|58.75|0.00|998.75|658.58|1297.27|69.91|0.00|998.75|1068.66|340.17| +2451485|47069|11683|42884|37976|1515|44654|7|143|257|94|92.84|158.75|128.58|0.00|12086.52|8726.96|14922.50|846.05|0.00|12086.52|12932.57|3359.56| +||2407|42884||1515|44654|||257|14|46.41|51.51|22.66||317.24|||||||-519.67| +2451485|47069|15520|42884|37976|1515|44654|7|93|257|20|45.78|59.05|14.17|0.00|283.40|915.60|1181.00|5.66|0.00|283.40|289.06|-632.20| +2451485|47069|7070|42884|37976|1515|44654|7|153|257|97|40.27|44.69|22.79|0.00|2210.63|3906.19|4334.93|198.95|0.00|2210.63|2409.58|-1695.56| +2451485|47069|6104|42884|37976|1515|44654|7|25|257|75|94.93|145.24|132.16|0.00|9912.00|7119.75|10893.00|594.72|0.00|9912.00|10506.72|2792.25| +2451485|47069|14623|42884|37976|1515|44654|7|211|257|66|74.86|99.56|22.89|0.00|1510.74|4940.76|6570.96|15.10|0.00|1510.74|1525.84|-3430.02| +2451485|47069|5854|42884|37976|1515|44654|7|11|257|63|61.44|82.32|46.09|0.00|2903.67|3870.72|5186.16|261.33|0.00|2903.67|3165.00|-967.05| +|47069|17131||||||281|257||37.79|54.79|34.51|0.00||2494.14||113.88|0.00|2277.66|2391.54|| +2451485|47069|11996|42884|37976|1515|44654|7|215|257|22|62.58|100.12|29.03|0.00|638.66|1376.76|2202.64|51.09|0.00|638.66|689.75|-738.10| +2451485|47069|8794|42884|37976|1515|44654|7|150|257|23|73.87|81.99|9.01|0.00|207.23|1699.01|1885.77|18.65|0.00|207.23|225.88|-1491.78| +2451485|47069|3166|42884|37976|1515|44654|7|87|257|80|88.49|135.38|75.81|0.00|6064.80|7079.20|10830.40|485.18|0.00|6064.80|6549.98|-1014.40| +2451485|47069|13255|42884|37976|1515|44654|7|110|257|70|47.48|78.81|9.45|0.00|661.50|3323.60|5516.70|0.00|0.00|661.50|661.50|-2662.10| +2451485|47069|13852|42884|37976|1515|44654|7|244|257|39|80.58|138.59|83.15|0.00|3242.85|3142.62|5405.01|259.42|0.00|3242.85|3502.27|100.23| +2451485|47069|2438|42884|37976|1515|44654|7|62|257|86|33.51|44.90|18.40|0.00|1582.40|2881.86|3861.40|94.94|0.00|1582.40|1677.34|-1299.46| +2452535|57567|4506|10558|447560|2162|21564|10|54|258|96|78.12|127.33|43.29|0.00|4155.84|7499.52|12223.68|83.11|0.00|4155.84|4238.95|-3343.68| +2452535|57567|4242|10558|447560|2162|21564|10|123|258|14|22.49|32.61|27.71|0.00|387.94|314.86|456.54|31.03|0.00|387.94|418.97|73.08| +2452535|57567|17331|10558|447560|2162|21564|10|140|258|46|71.38|138.47|74.77|0.00|3439.42|3283.48|6369.62|171.97|0.00|3439.42|3611.39|155.94| +2452535|57567|7392|10558|447560|2162|21564|10|161|258|100|10.49|13.63|8.85|300.90|885.00|1049.00|1363.00|23.36|300.90|584.10|607.46|-464.90| +2452535|57567|8337|10558|447560|2162|21564|10|137|258|27|35.03|37.48|5.62|0.00|151.74|945.81|1011.96|9.10|0.00|151.74|160.84|-794.07| +2452535|57567|17565|10558|447560|2162|21564|10|95|258|36|76.25|147.92|31.06|0.00|1118.16|2745.00|5325.12|78.27|0.00|1118.16|1196.43|-1626.84| +2452535|57567|13173|10558|447560|2162|21564|10|97|258|66|80.57|98.29|61.92|0.00|4086.72|5317.62|6487.14|81.73|0.00|4086.72|4168.45|-1230.90| +2452535|57567|15301|10558|447560|2162|21564|10|211|258|14|65.87|79.70|62.16|861.53|870.24|922.18|1115.80|0.78|861.53|8.71|9.49|-913.47| +2452535|57567|5997|10558|447560|2162|21564|10|23|258|19|34.87|40.44|26.28|414.43|499.32|662.53|768.36|0.84|414.43|84.89|85.73|-577.64| +2452535|57567|4743|10558|447560|2162|21564|10|37|258|99|2.15|4.19|1.71|10.15|169.29|212.85|414.81|4.77|10.15|159.14|163.91|-53.71| +2452535|57567|15144|10558|447560|2162|21564|10|300|258|76|51.36|96.04|73.95|0.00|5620.20|3903.36|7299.04|281.01|0.00|5620.20|5901.21|1716.84| +2452535|57567|8628|10558|447560|2162|21564|10|143|258|63|73.55|86.05|17.21|0.00|1084.23|4633.65|5421.15|10.84|0.00|1084.23|1095.07|-3549.42| +2452535|57567|5119|10558|447560|2162|21564|10|49|258|59|9.67|16.53|1.81|0.00|106.79|570.53|975.27|9.61|0.00|106.79|116.40|-463.74| +2452535||13707|10558||2162|||30|258||90.79|177.04|159.33||6851.19|3903.97|7612.72|||6851.19||2947.22| +2451658|41345|17819|55596|1549335|3874|30413|7|213|259|24|62.95|81.83|36.00|0.00|864.00|1510.80|1963.92|8.64|0.00|864.00|872.64|-646.80| +2451658|41345|5144|55596|1549335|3874|30413|7|123|259|93|23.91|26.30|2.36|0.00|219.48|2223.63|2445.90|2.19|0.00|219.48|221.67|-2004.15| +2451658|41345|5873|55596|1549335|3874|30413|7|33|259|7|77.32|132.99|111.71|0.00|781.97|541.24|930.93|15.63|0.00|781.97|797.60|240.73| +2451658|41345|3955|55596|1549335|3874|30413|7|50|259|22|18.01|23.05|7.60|0.00|167.20|396.22|507.10|5.01|0.00|167.20|172.21|-229.02| +2451658|41345|14822|55596|1549335|3874|30413|7|34|259|38|22.69|42.65|27.29|0.00|1037.02|862.22|1620.70|20.74|0.00|1037.02|1057.76|174.80| +2451658|41345|7171|55596|1549335|3874|30413|7|183|259|12|75.20|78.96|16.58|0.00|198.96|902.40|947.52|17.90|0.00|198.96|216.86|-703.44| +2451658|41345|17105|55596|1549335|3874|30413|7|159|259|17|76.63|105.74|65.55|144.86|1114.35|1302.71|1797.58|0.00|144.86|969.49|969.49|-333.22| +2451658|41345|12128|55596|1549335|3874|30413|7|239|259|93|22.77|35.74|7.50|0.00|697.50|2117.61|3323.82|55.80|0.00|697.50|753.30|-1420.11| +2451658|41345|2036|55596|1549335|3874|30413|7|191|259|34|47.76|53.01|22.26|0.00|756.84|1623.84|1802.34|52.97|0.00|756.84|809.81|-867.00| +2451658|41345|2525|55596|1549335|3874|30413|7|280|259|89|18.01|23.23|3.48|0.00|309.72|1602.89|2067.47|21.68|0.00|309.72|331.40|-1293.17| +2451658|41345|7280|55596|1549335|3874|30413|7|235|259|8|82.58|106.52|23.43|0.00|187.44|660.64|852.16|1.87|0.00|187.44|189.31|-473.20| +||2957|55596|1549335|3874|30413|7|45|259|17|6.23|12.27|||||||||95.47|| +2451658|41345|16435|55596|1549335|3874|30413|7|50|259|69|17.94|33.18|30.52|0.00|2105.88|1237.86|2289.42|168.47|0.00|2105.88|2274.35|868.02| +2452262||1991|41218||3052|35922||81|260|7||14.96|6.43|0.00|45.01|57.26|104.72|2.70|0.00|45.01||-12.25| +2452262|59756|321|41218|1192535|3052|35922|8|102|260|86|2.45|3.47|3.47|0.00|298.42|210.70|298.42|2.98|0.00|298.42|301.40|87.72| +2452262|59756|14901|41218|1192535|3052|35922|8|57|260|64|35.04|42.74|19.23|0.00|1230.72|2242.56|2735.36|24.61|0.00|1230.72|1255.33|-1011.84| +2452262|59756|17831|41218|1192535|3052|35922|8|78|260|91|55.67|91.29|0.91|0.00|82.81|5065.97|8307.39|5.79|0.00|82.81|88.60|-4983.16| +2452262|59756|17145|41218|1192535|3052|35922|8|275|260|2|89.39|99.22|45.64|4.56|91.28|178.78|198.44|0.00|4.56|86.72|86.72|-92.06| +2452262|59756|16037|41218|1192535|3052|35922|8|108|260|37|7.79|13.47|13.47|0.00|498.39|288.23|498.39|34.88|0.00|498.39|533.27|210.16| +||12265|41218|||||60|260|74|||58.51||4329.74|3775.48||||2034.98||-1740.50| +2452262|59756|7309|41218|1192535|3052|35922|8|56|260|87|25.06|34.33|32.95|0.00|2866.65|2180.22|2986.71|57.33|0.00|2866.65|2923.98|686.43| +2452262|59756|12409|41218|1192535|3052|35922|8|133|260|84|10.38|11.52|10.25|0.00|861.00|871.92|967.68|17.22|0.00|861.00|878.22|-10.92| +2452262|59756|5569|41218|1192535|3052|35922|8|287|260|81|20.41|31.02|19.85|0.00|1607.85|1653.21|2512.62|64.31|0.00|1607.85|1672.16|-45.36| +2452262|59756|9559|41218|1192535|3052|35922|8|201|260|67|30.33|38.51|1.54|0.00|103.18|2032.11|2580.17|7.22|0.00|103.18|110.40|-1928.93| +2452262|59756|16789|41218|1192535|3052|35922|8|235|260|15|42.76|44.04|5.72|0.00|85.80|641.40|660.60|3.43|0.00|85.80|89.23|-555.60| +2452262|59756|13113|41218|1192535|3052|35922|8|73|260|59|7.39|11.52|2.76|0.00|162.84|436.01|679.68|14.65|0.00|162.84|177.49|-273.17| +2452262|59756|133|41218|1192535|3052|35922|8|264|260|10|49.15|73.23|24.89|0.00|248.90|491.50|732.30|0.00|0.00|248.90|248.90|-242.60| +2452262|59756|16393|41218|1192535|3052|35922|8|33|260|6|91.10|112.05|11.20|0.00|67.20|546.60|672.30|3.36|0.00|67.20|70.56|-479.40| +2452262|59756|9943|41218|1192535|3052|35922|8|249|260|79|28.10|28.38|10.50|0.00|829.50|2219.90|2242.02|8.29|0.00|829.50|837.79|-1390.40| +2452525|39324|14979|81093|1388634|6128|2422|2|279|261|13|62.24|77.80|17.89|0.00|232.57|809.12|1011.40|13.95|0.00|232.57|246.52|-576.55| +2452525|39324|127|81093|1388634|6128|2422|2|162|261|14|78.29|117.43|32.88|0.00|460.32|1096.06|1644.02|27.61|0.00|460.32|487.93|-635.74| +2452525|39324|17725|81093|1388634|6128|2422|2|256|261|14|40.24|54.32|1.08|0.00|15.12|563.36|760.48|0.45|0.00|15.12|15.57|-548.24| +2452525|39324|4257|81093|1388634|6128|2422|2|272|261|61|43.22|76.93|49.23|0.00|3003.03|2636.42|4692.73|180.18|0.00|3003.03|3183.21|366.61| +2452525|39324|8785|81093|1388634|6128|2422|2|108|261|35|72.65|87.90|75.59|423.30|2645.65|2542.75|3076.50|111.11|423.30|2222.35|2333.46|-320.40| +2452525|39324|1207|81093|1388634|6128|2422|2|126|261|85|19.91|35.83|3.94|0.00|334.90|1692.35|3045.55|0.00|0.00|334.90|334.90|-1357.45| +2452525||9699|81093|1388634||2422|2||261||68.74|||12.67|97.50|||2.54|12.67||87.37|| +2452525|39324|7056|81093|1388634|6128|2422|2|274|261|4|40.40|65.44|46.46|0.00|185.84|161.60|261.76|13.00|0.00|185.84|198.84|24.24| +|39324|9147|81093|||||297|261|||11.49||22.04|68.88||137.88|1.87|22.04||48.71|| +2452525|39324|11280|81093|1388634|6128|2422|2|11|261|11|40.71|49.25|47.77|336.30|525.47|447.81|541.75|13.24|336.30|189.17|202.41|-258.64| +2452525|39324|7248|81093|1388634|6128|2422|2|48|261|85|29.55|46.98|46.98|0.00|3993.30|2511.75|3993.30|159.73|0.00|3993.30|4153.03|1481.55| +2452525|39324|11521|81093|1388634|6128|2422|2|118|261|87|46.56|56.33|19.71|548.72|1714.77|4050.72|4900.71|81.62|548.72|1166.05|1247.67|-2884.67| +2452525|39324|11971|81093|1388634|6128|2422|2|123|261|81|22.57|34.08|32.37|0.00|2621.97|1828.17|2760.48|183.53|0.00|2621.97|2805.50|793.80| +2452525|39324|17227|81093|1388634|6128|2422|2|206|261|58|70.28|131.42|98.56|0.00|5716.48|4076.24|7622.36|285.82|0.00|5716.48|6002.30|1640.24| +2452525|39324|13881|81093|1388634|6128|2422|2|261|261|32|73.53|81.61|19.58|0.00|626.56|2352.96|2611.52|56.39|0.00|626.56|682.95|-1726.40| +2451494|71019|7147|72780|1701020|2060|2300|7|260|262|38|14.35|28.26|0.84|0.00|31.92|545.30|1073.88|2.87|0.00|31.92|34.79|-513.38| +2451494|71019|15595|72780|1701020|2060|2300|7|81|262|54|99.97|168.94|97.98|0.00|5290.92|5398.38|9122.76|423.27|0.00|5290.92|5714.19|-107.46| +2451494|71019|17384|72780|1701020|2060|2300|7|14|262|33|34.29|42.17|30.36|0.00|1001.88|1131.57|1391.61|60.11|0.00|1001.88|1061.99|-129.69| +2451494|71019|17150|72780|1701020|2060|2300|7|260|262|2|60.38|87.55|0.00|0.00|0.00|120.76|175.10|0.00|0.00|0.00|0.00|-120.76| +2451494|71019|1960|72780|1701020|2060|2300|7|216|262|96|44.69|88.48|13.27|0.00|1273.92|4290.24|8494.08|50.95|0.00|1273.92|1324.87|-3016.32| +2451494|71019|7040|72780|1701020|2060|2300|7|65|262|29|43.73|65.59|51.81|0.00|1502.49|1268.17|1902.11|45.07|0.00|1502.49|1547.56|234.32| +2451494|71019|16540|72780|1701020|2060|2300|7|23|262|39|45.24|86.86|72.96|0.00|2845.44|1764.36|3387.54|0.00|0.00|2845.44|2845.44|1081.08| +2451494|71019|13042|72780|1701020|2060|2300|7|16|262|44|36.64|51.29|41.03|0.00|1805.32|1612.16|2256.76|144.42|0.00|1805.32|1949.74|193.16| +2451494|71019|8065|72780|1701020|2060|2300|7|34|262|35|16.09|21.88|9.40|0.00|329.00|563.15|765.80|29.61|0.00|329.00|358.61|-234.15| +2451494|71019|3010|72780|1701020|2060|2300|7|9|262|76|91.60|180.45|176.84|0.00|13439.84|6961.60|13714.20|134.39|0.00|13439.84|13574.23|6478.24| +2451494|71019|11426|72780|1701020|2060|2300|7|237|262|47|31.08|34.18|31.44|0.00|1477.68|1460.76|1606.46|132.99|0.00|1477.68|1610.67|16.92| +2451494|71019|15715|72780|1701020|2060|2300|7|28|262|76|24.14|29.69|29.39|982.80|2233.64|1834.64|2256.44|87.55|982.80|1250.84|1338.39|-583.80| +2451494|71019|8101|72780|1701020|2060|2300|7|155|262|30|72.46|126.80|103.97|0.00|3119.10|2173.80|3804.00|93.57|0.00|3119.10|3212.67|945.30| +2452272|65124|16595|40105|284291|2877|45463|10|31|263|93|31.80|34.34|12.01|0.00|1116.93|2957.40|3193.62|55.84|0.00|1116.93|1172.77|-1840.47| +2452272|65124|10917|40105|284291|2877|45463|10|88|263|44|20.58|27.98|0.00|0.00|0.00|905.52|1231.12|0.00|0.00|0.00|0.00|-905.52| +2452272|65124|17367|40105|284291|2877|45463|10|103|263|86|33.66|45.77|23.80|0.00|2046.80|2894.76|3936.22|184.21|0.00|2046.80|2231.01|-847.96| +2452272|65124|5245|40105|284291|2877|45463|10|166|263|33|88.19|119.05|82.14|0.00|2710.62|2910.27|3928.65|81.31|0.00|2710.62|2791.93|-199.65| +2452272|65124|12703|40105|284291|2877|45463|10|205|263|90|94.45|94.45|4.72|0.00|424.80|8500.50|8500.50|0.00|0.00|424.80|424.80|-8075.70| +2452272|65124|209|40105|284291|2877|45463|10|59|263|87|29.95|57.80|29.47|0.00|2563.89|2605.65|5028.60|102.55|0.00|2563.89|2666.44|-41.76| +2452272|65124|8317|40105|284291|2877|45463|10|109|263|70|18.43|31.14|29.89|2008.60|2092.30|1290.10|2179.80|5.85|2008.60|83.70|89.55|-1206.40| +2452272|65124|2499|40105|284291|2877|45463|10|156|263|55|61.60|73.30|29.32|0.00|1612.60|3388.00|4031.50|64.50|0.00|1612.60|1677.10|-1775.40| +2452272|65124|16343|40105|284291|2877|45463|10|267|263|31|70.29|104.73|21.99|0.00|681.69|2178.99|3246.63|27.26|0.00|681.69|708.95|-1497.30| +2452272|65124|7349|40105|284291|2877|45463|10|47|263|2|84.98|104.52|39.71|69.09|79.42|169.96|209.04|0.51|69.09|10.33|10.84|-159.63| +2452272|65124|6129|40105|284291|2877|45463|10|111|263|48|75.57|123.17|3.69|0.00|177.12|3627.36|5912.16|0.00|0.00|177.12|177.12|-3450.24| +2451856|56196|12005|4471|1544102|3733|20917|10|4|264|4|57.80|115.02|93.16|152.78|372.64|231.20|460.08|13.19|152.78|219.86|233.05|-11.34| +2451856|56196|79|4471|1544102|3733|20917|10|83|264|82|43.48|70.43|54.23|2356.83|4446.86|3565.36|5775.26|104.50|2356.83|2090.03|2194.53|-1475.33| +2451856|56196|4637|4471|1544102|3733|20917|10|57|264|54|56.79|60.19|53.56|0.00|2892.24|3066.66|3250.26|202.45|0.00|2892.24|3094.69|-174.42| +2451856|56196|7241|4471|1544102|3733|20917|10|83|264|94|89.84|94.33|47.16|3280.44|4433.04|8444.96|8867.02|0.00|3280.44|1152.60|1152.60|-7292.36| +2451856|56196|11257|4471|1544102|3733|20917|10|83|264|6|13.05|16.57|9.27|0.00|55.62|78.30|99.42|1.11|0.00|55.62|56.73|-22.68| +2451856||12397|4471|1544102||20917|10||264|||10.78||35.04|106.21||463.54|2.13|35.04|71.17|73.30|| +2451856|56196|9092|4471|1544102|3733|20917|10|154|264|15|17.43|23.00|13.57|0.00|203.55|261.45|345.00|8.14|0.00|203.55|211.69|-57.90| +2451856|56196|15403|4471|1544102|3733|20917|10|115|264|66|9.33|13.99|8.53|0.00|562.98|615.78|923.34|33.77|0.00|562.98|596.75|-52.80| +2451856|56196|8180|4471|1544102|3733|20917|10|235|264|9|4.97|5.91|5.25|0.00|47.25|44.73|53.19|3.78|0.00|47.25|51.03|2.52| +2451856|56196|17630|4471|1544102|3733|20917|10|227|264|71|47.96|79.61|74.83|0.00|5312.93|3405.16|5652.31|106.25|0.00|5312.93|5419.18|1907.77| +2451856|56196|319|4471|1544102|3733|20917|10|208|264|34|96.40|161.95|80.97|0.00|2752.98|3277.60|5506.30|82.58|0.00|2752.98|2835.56|-524.62| +2451856|56196|11219|4471|1544102|3733|20917|10|260|264|87|11.69|20.57|12.75|0.00|1109.25|1017.03|1789.59|55.46|0.00|1109.25|1164.71|92.22| +2452084|48845|9587|37586|439670|2907|42991|7|184|265|70|47.51|54.63|18.02|0.00|1261.40|3325.70|3824.10|25.22|0.00|1261.40|1286.62|-2064.30| +2452084|48845|3719|37586|439670|2907|42991|7|258|265|97|88.74|136.65|2.73|0.00|264.81|8607.78|13255.05|18.53|0.00|264.81|283.34|-8342.97| +2452084|48845|7057|37586|439670|2907|42991|7|258|265|8|4.15|6.22|0.68|0.00|5.44|33.20|49.76|0.27|0.00|5.44|5.71|-27.76| +2452084|48845|2831|37586|439670|2907|42991|7|118|265|98|40.82|80.82|10.50|0.00|1029.00|4000.36|7920.36|10.29|0.00|1029.00|1039.29|-2971.36| +2452084|48845|12401|37586|439670|2907|42991|7|267|265|77|57.32|94.57|59.57|0.00|4586.89|4413.64|7281.89|229.34|0.00|4586.89|4816.23|173.25| +2452084|48845|4457|37586|439670|2907|42991|7|190|265|1|3.76|4.02|0.76|0.00|0.76|3.76|4.02|0.05|0.00|0.76|0.81|-3.00| +2452084|48845|14633|37586|439670|2907|42991|7|252|265|63|90.65|180.39|16.23|204.49|1022.49|5710.95|11364.57|8.18|204.49|818.00|826.18|-4892.95| +2452084|48845|10669||439670|2907|42991|||265|||12.83|||156.96||230.94|||156.96||| +2452084|48845|3431|37586|439670|2907|42991|7|182|265|27|41.40|66.65|57.98|485.29|1565.46|1117.80|1799.55|10.80|485.29|1080.17|1090.97|-37.63| +2452084|48845|12099|37586|439670|2907|42991|7|248|265|28|80.38|100.47|82.38|0.00|2306.64|2250.64|2813.16|92.26|0.00|2306.64|2398.90|56.00| +2452084|48845|7931|37586|439670|2907|42991|7|279|265|26|82.37|129.32|82.76|0.00|2151.76|2141.62|3362.32|129.10|0.00|2151.76|2280.86|10.14| +2452084|48845|9755|37586|439670|2907|42991|7|187|265|5|11.08|12.85|0.89|0.00|4.45|55.40|64.25|0.00|0.00|4.45|4.45|-50.95| +2452084|48845|15667|37586|439670|2907|42991|7|121|265|20|62.35|83.54|28.40|488.48|568.00|1247.00|1670.80|0.79|488.48|79.52|80.31|-1167.48| +2452084|48845|5397|37586|439670|2907|42991|7|197|265|76|27.39|40.81|37.13|0.00|2821.88|2081.64|3101.56|225.75|0.00|2821.88|3047.63|740.24| +||1685|37586|439670||42991|7|264|265||28.96|56.18||0.00||||25.18|0.00||528.88|| +2452298|48146|12741|65737|1209636|6335|11834|1|52|266|10|39.58|69.66|3.48|33.06|34.80|395.80|696.60|0.15|33.06|1.74|1.89|-394.06| +2452298|48146|4029|65737|1209636|6335|11834|1|207|266|78|42.23|48.56|23.79|0.00|1855.62|3293.94|3787.68|92.78|0.00|1855.62|1948.40|-1438.32| +2452298|48146|17730|65737|1209636|6335|11834|1|229|266|23|5.46|6.98|3.49|0.00|80.27|125.58|160.54|2.40|0.00|80.27|82.67|-45.31| +2452298|48146|17994|65737|1209636|6335|11834|1|172|266|39|63.25|104.99|80.84|599.02|3152.76|2466.75|4094.61|178.76|599.02|2553.74|2732.50|86.99| +2452298|48146|684|65737|1209636|6335|11834|1|286|266|76|1.85|3.34|2.60|0.00|197.60|140.60|253.84|13.83|0.00|197.60|211.43|57.00| +2452298|48146|3753||1209636|6335|11834||65|266||4.89|5.42||||239.61|265.58|||||-178.33| +2452298|48146|10035|65737|1209636|6335|11834|1|34|266|89|30.87|60.19|30.69|0.00|2731.41|2747.43|5356.91|0.00|0.00|2731.41|2731.41|-16.02| +2452298|48146|7359|65737|1209636|6335|11834|1|212|266|89|32.78|35.07|30.86|0.00|2746.54|2917.42|3121.23|192.25|0.00|2746.54|2938.79|-170.88| +2452298|48146|15585|65737|1209636|6335|11834|1|135|266|17|1.73|2.54|0.43|0.00|7.31|29.41|43.18|0.65|0.00|7.31|7.96|-22.10| +2452298|48146|6397|65737|1209636|6335|11834|1|188|266|29|2.46|4.57|2.83|0.00|82.07|71.34|132.53|4.92|0.00|82.07|86.99|10.73| +2452298|48146|17305|65737|1209636|6335|11834|1|134|266|27|3.81|5.60|0.95|0.00|25.65|102.87|151.20|2.05|0.00|25.65|27.70|-77.22| +2452298|48146|7315|65737|1209636|6335|11834|1|245|266|84|48.95|69.01|65.55|0.00|5506.20|4111.80|5796.84|220.24|0.00|5506.20|5726.44|1394.40| +2452298|48146|17544|65737|1209636|6335|11834|1|160|266|99|1.77|2.61|0.57|0.00|56.43|175.23|258.39|2.82|0.00|56.43|59.25|-118.80| +2451796|64568|7075|43915|1624140|2318|6543|10|58|267|53|13.68|23.11|15.94|270.34|844.82|725.04|1224.83|40.21|270.34|574.48|614.69|-150.56| +2451796|64568|4199|43915|1624140|2318|6543|10|242|267|1|67.05|100.57|41.23|0.00|41.23|67.05|100.57|1.23|0.00|41.23|42.46|-25.82| +2451796|64568|12227|43915|1624140|2318|6543|10|120|267|4|61.13|113.70|100.05|0.00|400.20|244.52|454.80|8.00|0.00|400.20|408.20|155.68| +2451796|64568|3301|43915|1624140|2318|6543|10|110|267|98|29.07|49.70|18.38|0.00|1801.24|2848.86|4870.60|0.00|0.00|1801.24|1801.24|-1047.62| +2451796|64568|12659|43915|1624140|2318|6543|10|51|267|65|65.36|84.31|40.46|0.00|2629.90|4248.40|5480.15|105.19|0.00|2629.90|2735.09|-1618.50| +2451796|64568|7837|43915|1624140|2318|6543|10|202|267|4|35.72|40.36|5.24|0.00|20.96|142.88|161.44|1.25|0.00|20.96|22.21|-121.92| +2451796|64568|17881|43915|1624140|2318|6543|10|227|267|35|94.09|168.42|151.57|0.00|5304.95|3293.15|5894.70|477.44|0.00|5304.95|5782.39|2011.80| +2451796|64568|3302|43915|1624140|2318|6543|10|266|267|64|71.75|112.64|94.61|0.00|6055.04|4592.00|7208.96|544.95|0.00|6055.04|6599.99|1463.04| +2451796|64568|17551|43915|1624140|2318|6543|10|201|267|31|84.62|149.77|31.45|0.00|974.95|2623.22|4642.87|19.49|0.00|974.95|994.44|-1648.27| +2451796|64568|13903|43915|1624140|2318|6543|10|140|267|99|9.92|13.59|5.57|0.00|551.43|982.08|1345.41|11.02|0.00|551.43|562.45|-430.65| +2451563|38495|3698|99323|1510372|621|6117|7|297|268|85|22.71|37.01|31.82|135.23|2704.70|1930.35|3145.85|231.25|135.23|2569.47|2800.72|639.12| +2451563|38495|16763|99323|1510372|621|6117|7|139|268|81|48.58|90.35|79.50|0.00|6439.50|3934.98|7318.35|64.39|0.00|6439.50|6503.89|2504.52| +2451563|38495|14198|99323|1510372|621|6117|7|97|268|72|1.09|1.80|1.71|0.00|123.12|78.48|129.60|4.92|0.00|123.12|128.04|44.64| +2451563|38495|15452|99323|1510372|621|6117|7|111|268|94|68.18|96.13|37.49|1057.21|3524.06|6408.92|9036.22|222.01|1057.21|2466.85|2688.86|-3942.07| +2451563|38495|14774|99323|1510372|621|6117|7|291|268|41|94.60|159.87|129.49|0.00|5309.09|3878.60|6554.67|318.54|0.00|5309.09|5627.63|1430.49| +2451563|38495|1855|99323|1510372|621|6117|7|45|268|59|85.91|87.62|34.17|0.00|2016.03|5068.69|5169.58|60.48|0.00|2016.03|2076.51|-3052.66| +2451563|38495|5323|99323|1510372|621|6117|7|96|268|4|33.45|37.12|15.96|0.00|63.84|133.80|148.48|0.63|0.00|63.84|64.47|-69.96| +2451563|38495|1615||||6117|||268|51||14.83|5.93|238.91|||756.33||238.91|||-415.37| +2451563|38495|11930|99323|1510372|621|6117|7|56|268|81|56.49|59.31|0.59|0.00|47.79|4575.69|4804.11|1.91|0.00|47.79|49.70|-4527.90| +2451563|38495|2213|99323|1510372|621|6117|7|270|268|22|93.71|137.75|99.18|0.00|2181.96|2061.62|3030.50|65.45|0.00|2181.96|2247.41|120.34| +2451563|38495|17120|99323|1510372|621|6117|7|209|268|40|70.67|126.49|101.19|0.00|4047.60|2826.80|5059.60|121.42|0.00|4047.60|4169.02|1220.80| +2451182|44029|4999|17432|1626054|1876|31298|8|265|269|45|82.88|116.03|30.16|0.00|1357.20|3729.60|5221.35|108.57|0.00|1357.20|1465.77|-2372.40| +2451182|44029|11138|17432|1626054|1876|31298|8|256|269|18|99.02|117.83|103.69|0.00|1866.42|1782.36|2120.94|18.66|0.00|1866.42|1885.08|84.06| +2451182|44029|10411|17432|1626054|1876|31298|8|78|269|21|18.43|31.69|19.96|41.91|419.16|387.03|665.49|11.31|41.91|377.25|388.56|-9.78| +2451182|44029|2342|17432|1626054|1876|31298|8|264|269|89|44.80|52.41|20.43|0.00|1818.27|3987.20|4664.49|109.09|0.00|1818.27|1927.36|-2168.93| +2451182|44029|11000|17432|1626054|1876|31298|8|122|269|43|23.36|36.20|1.44|0.00|61.92|1004.48|1556.60|1.23|0.00|61.92|63.15|-942.56| +2451182|44029|9362|17432|1626054|1876|31298|8|8|269|15|34.22|65.70|24.30|0.00|364.50|513.30|985.50|29.16|0.00|364.50|393.66|-148.80| +2451182|44029|61|17432|1626054|1876|31298|8|130|269|8|17.07|18.60|6.51|0.00|52.08|136.56|148.80|3.12|0.00|52.08|55.20|-84.48| +2451182|44029|7976|17432|1626054|1876|31298|8|7|269|93|65.83|73.07|48.95|0.00|4552.35|6122.19|6795.51|45.52|0.00|4552.35|4597.87|-1569.84| +2451182||5836||1626054||31298|||269|10|98.73||31.79|0.00|||1382.20|0.00|0.00|||-669.40| +2451182|44029|4879|17432|1626054|1876|31298|8|262|269|7|31.91|53.60|17.15|0.00|120.05|223.37|375.20|0.00|0.00|120.05|120.05|-103.32| +2451182|44029|15643|17432|1626054|1876|31298|8|4|269|90|30.76|38.75|12.01|0.00|1080.90|2768.40|3487.50|43.23|0.00|1080.90|1124.13|-1687.50| +2451558|71711|3469|24666|894434|3849|4417|10|28|270|48|25.70|35.98|33.82|0.00|1623.36|1233.60|1727.04|97.40|0.00|1623.36|1720.76|389.76| +2451558|71711|14105|24666|894434|3849|4417|10|1|270|53|86.14|147.29|10.31|0.00|546.43|4565.42|7806.37|5.46|0.00|546.43|551.89|-4018.99| +2451558|71711|4328|24666|894434|3849|4417|10|142|270|26|98.98|151.43|46.94|0.00|1220.44|2573.48|3937.18|0.00|0.00|1220.44|1220.44|-1353.04| +2451558|71711|14779|24666|894434|3849|4417|10|147|270|44|93.58|107.61|23.67|0.00|1041.48|4117.52|4734.84|10.41|0.00|1041.48|1051.89|-3076.04| +2451558|71711|5165|24666|894434|3849|4417|10|59|270|94|38.14|70.55|66.31|0.00|6233.14|3585.16|6631.70|249.32|0.00|6233.14|6482.46|2647.98| +2451558|71711|9170|24666|894434|3849|4417|10|137|270|91|91.22|99.42|25.84|2304.41|2351.44|8301.02|9047.22|3.29|2304.41|47.03|50.32|-8253.99| +2451558|71711|16387|24666|894434|3849|4417|10|121|270|83|40.76|48.50|45.59|0.00|3783.97|3383.08|4025.50|302.71|0.00|3783.97|4086.68|400.89| +2451558|71711|1868|24666|894434|3849|4417|10|240|270|93|96.71|101.54|69.04|0.00|6420.72|8994.03|9443.22|256.82|0.00|6420.72|6677.54|-2573.31| +2451558|71711|6499|24666|894434|3849|4417|10|110|270|9|67.79|124.73|97.28|0.00|875.52|610.11|1122.57|43.77|0.00|875.52|919.29|265.41| +2451558|71711|12283|24666|894434|3849|4417|10|220|270|20|97.26|119.62|107.65|0.00|2153.00|1945.20|2392.40|150.71|0.00|2153.00|2303.71|207.80| +2451558|71711|7523|24666|894434|3849|4417|10|187|270|47|28.81|43.50|4.78|0.00|224.66|1354.07|2044.50|6.73|0.00|224.66|231.39|-1129.41| +2451558|71711|12746|24666|894434|3849|4417|10|7|270|21|30.13|59.35|20.77|0.00|436.17|632.73|1246.35|34.89|0.00|436.17|471.06|-196.56| +2450893|39456|542|881|1120568|1055|30389|7|184|271|14|43.13|73.75|23.60|0.00|330.40|603.82|1032.50|26.43|0.00|330.40|356.83|-273.42| +2450893|39456|17192|881|1120568|1055|30389|7|157|271|58|51.00|74.97|59.22|0.00|3434.76|2958.00|4348.26|68.69|0.00|3434.76|3503.45|476.76| +2450893|39456|9484|881|1120568|1055|30389|7|110|271|98|1.72|2.58|2.42|0.00|237.16|168.56|252.84|21.34|0.00|237.16|258.50|68.60| +2450893||961|881|1120568|1055|30389|7||271||||119.53|0.00||9029.37|||0.00|||| +2450893|39456|14392|881|1120568|1055|30389|7|157|271|10|48.23|64.62|2.58|0.00|25.80|482.30|646.20|0.51|0.00|25.80|26.31|-456.50| +2450893|39456|14161|881|1120568|1055|30389|7|115|271|77|11.78|16.25|15.92|0.00|1225.84|907.06|1251.25|98.06|0.00|1225.84|1323.90|318.78| +2450893|39456|7048|881|1120568|1055|30389|7|56|271|66|13.08|18.18|0.00|0.00|0.00|863.28|1199.88|0.00|0.00|0.00|0.00|-863.28| +2450893|39456|12181|881|1120568|1055|30389|7|299|271|63|28.85|45.29|3.62|0.00|228.06|1817.55|2853.27|2.28|0.00|228.06|230.34|-1589.49| +2450893|39456|5344|881|1120568|1055|30389|7|42|271|89|26.62|37.80|15.87|607.34|1412.43|2369.18|3364.20|16.10|607.34|805.09|821.19|-1564.09| +2450893|39456|9844|881|1120568|1055|30389|7|158|271|56|29.24|50.58|3.54|0.00|198.24|1637.44|2832.48|17.84|0.00|198.24|216.08|-1439.20| +2450893|39456|470|881|1120568|1055|30389|7|174|271|33|87.72|163.15|32.63|936.80|1076.79|2894.76|5383.95|2.79|936.80|139.99|142.78|-2754.77| +2450893|39456|17902|881|1120568|1055|30389|7|65|271|30|59.87|100.58|71.41|0.00|2142.30|1796.10|3017.40|107.11|0.00|2142.30|2249.41|346.20| +2452435|61237|8749|44046|8386|7126|24666|7|44|272|10|79.05|140.70|121.00|0.00|1210.00|790.50|1407.00|0.00|0.00|1210.00|1210.00|419.50| +2452435|61237|13417|44046|8386|7126|24666|7|214|272|17|37.18|39.41|13.39|0.00|227.63|632.06|669.97|2.27|0.00|227.63|229.90|-404.43| +2452435|61237|6775|44046|8386|7126|24666|7|73|272|89|81.21|86.08|43.04|0.00|3830.56|7227.69|7661.12|38.30|0.00|3830.56|3868.86|-3397.13| +2452435|61237|12204|44046|8386|7126|24666|7|165|272|84|56.38|69.34|5.54|162.87|465.36|4735.92|5824.56|9.07|162.87|302.49|311.56|-4433.43| +2452435|61237|13377||8386|||7|86|272||53.41|65.69|40.07|||1068.20||||||| +2452435|61237|7152|44046|8386|7126|24666|7|257|272|44|1.51|2.76|1.24|0.00|54.56|66.44|121.44|3.81|0.00|54.56|58.37|-11.88| +2452435|61237|13317|44046|8386|7126|24666|7|29|272|31|34.37|38.15|24.03|0.00|744.93|1065.47|1182.65|44.69|0.00|744.93|789.62|-320.54| +2452435|61237|11719|44046|8386|7126|24666|7|175|272|58|55.84|98.83|53.36|0.00|3094.88|3238.72|5732.14|92.84|0.00|3094.88|3187.72|-143.84| +2452435|61237|7735|44046|8386|7126|24666|7|173|272|32|67.04|121.34|58.24|1621.40|1863.68|2145.28|3882.88|21.80|1621.40|242.28|264.08|-1903.00| +2452435|61237|5550|44046|8386|7126|24666|7|159|272|97|76.27|114.40|41.18|0.00|3994.46|7398.19|11096.80|199.72|0.00|3994.46|4194.18|-3403.73| +2452435|61237|15519|44046|8386|7126|24666|7|2|272|38|52.94|59.82|52.64|0.00|2000.32|2011.72|2273.16|0.00|0.00|2000.32|2000.32|-11.40| +2451882|71984|5246|32880|722375|670|35365|2|31|273|76|96.15|153.84|123.07|0.00|9353.32|7307.40|11691.84|93.53|0.00|9353.32|9446.85|2045.92| +2451882|71984|17597|32880|722375|670|35365|2|8|273|76|16.36|16.36|3.59|0.00|272.84|1243.36|1243.36|19.09|0.00|272.84|291.93|-970.52| +2451882|71984|3824|32880|722375|670|35365|2|272|273|43|65.69|104.44|101.30|0.00|4355.90|2824.67|4490.92|0.00|0.00|4355.90|4355.90|1531.23| +2451882|71984|2965|32880|722375|670|35365|2|83|273|40|58.78|88.17|56.42|0.00|2256.80|2351.20|3526.80|67.70|0.00|2256.80|2324.50|-94.40| +2451882|71984|12287|32880|722375|670|35365|2|73|273|15|29.62|43.83|39.00|23.40|585.00|444.30|657.45|39.31|23.40|561.60|600.91|117.30| +2451882|71984|5810|32880|722375|670|35365|2|52|273|86|73.13|76.78|50.67|0.00|4357.62|6289.18|6603.08|261.45|0.00|4357.62|4619.07|-1931.56| +2451882|71984|3611|32880|722375|670|35365|2|52|273|40|62.40|109.20|17.47|0.00|698.80|2496.00|4368.00|48.91|0.00|698.80|747.71|-1797.20| +2451882|71984|1742|32880|722375|670|35365|2|135|273|86|61.25|109.02|75.22|0.00|6468.92|5267.50|9375.72|452.82|0.00|6468.92|6921.74|1201.42| +2451882|71984|1112|32880|722375|670|35365|2|51|273|10|54.51|92.12|58.95|0.00|589.50|545.10|921.20|0.00|0.00|589.50|589.50|44.40| +2451882|71984|9674|32880|722375|670|35365|2|266|273|94|69.91|81.79|40.07|0.00|3766.58|6571.54|7688.26|150.66|0.00|3766.58|3917.24|-2804.96| +2451882|71984|9656|32880|722375|670|35365|2|52|273|91|22.30|30.77|7.38|0.00|671.58|2029.30|2800.07|60.44|0.00|671.58|732.02|-1357.72| +2452125|43987|1749|70136|866189|5089|47589|8|295|274|33|69.53|72.31|57.84|0.00|1908.72|2294.49|2386.23|0.00|0.00|1908.72|1908.72|-385.77| +2452125|43987|14259|70136|866189|5089|47589|8|99|274|71|99.84|133.78|68.22|0.00|4843.62|7088.64|9498.38|242.18|0.00|4843.62|5085.80|-2245.02| +2452125|43987|14213|70136|866189|5089|47589|8|292|274|62|20.54|21.15|1.69|0.00|104.78|1273.48|1311.30|5.23|0.00|104.78|110.01|-1168.70| +2452125|43987|2691|70136|866189|5089|47589|8|11|274|51|97.93|135.14|5.40|0.00|275.40|4994.43|6892.14|0.00|0.00|275.40|275.40|-4719.03| +2452125|43987|2013|70136|866189|5089|47589|8|274|274|51|37.56|67.98|9.51|0.00|485.01|1915.56|3466.98|29.10|0.00|485.01|514.11|-1430.55| +2452125||12815||866189||47589|||274||1.29|2.06|0.39||39.00||206.00|3.51||39.00||-90.00| +2452125|43987|697|70136|866189|5089|47589|8|14|274|12|98.65|176.58|104.18|387.54|1250.16|1183.80|2118.96|34.50|387.54|862.62|897.12|-321.18| +2452125|43987|17049|70136|866189|5089|47589|8|132|274|38|66.98|97.12|67.98|154.99|2583.24|2545.24|3690.56|218.54|154.99|2428.25|2646.79|-116.99| +2452125|43987|2255|70136|866189|5089|47589|8|258|274|1|95.99|121.90|102.39|0.00|102.39|95.99|121.90|7.16|0.00|102.39|109.55|6.40| +2452125|43987|3365|70136|866189|5089|47589|8|271|274|96|62.91|107.57|80.67|0.00|7744.32|6039.36|10326.72|387.21|0.00|7744.32|8131.53|1704.96| +2452125|43987|1883|70136|866189|5089|47589|8|242|274|60|33.17|46.76|1.40|0.00|84.00|1990.20|2805.60|7.56|0.00|84.00|91.56|-1906.20| +2452125|43987|6369|70136|866189|5089|47589|8|150|274|59|69.12|71.88|40.97|0.00|2417.23|4078.08|4240.92|24.17|0.00|2417.23|2441.40|-1660.85| +2451162|43415|8935|62086|358238|1143|31295|4|8|275|24|10.92|21.40|13.48|0.00|323.52|262.08|513.60|22.64|0.00|323.52|346.16|61.44| +|43415|5684|||1143|31295|||275|44|7.86||3.46|||||12.17||||-193.60| +2451162|43415|17702|62086|358238|1143|31295|4|17|275|64|15.80|21.01|10.50|0.00|672.00|1011.20|1344.64|33.60|0.00|672.00|705.60|-339.20| +2451162|43415|5695|62086|358238|1143|31295|4|33|275|79|39.30|40.87|9.80|0.00|774.20|3104.70|3228.73|7.74|0.00|774.20|781.94|-2330.50| +2451162|43415|9967|||1143||4|289|275|40|94.61|113.53||0.00||3784.40||44.95|0.00||4540.55|711.20| +2451162|43415|3901|62086|358238|1143|31295|4|193|275|7|59.75|67.51|47.25|0.00|330.75|418.25|472.57|23.15|0.00|330.75|353.90|-87.50| +2451162|43415|8188|62086|358238|1143|31295|4|196|275|92|8.95|12.35|11.97|374.42|1101.24|823.40|1136.20|43.60|374.42|726.82|770.42|-96.58| +2451162|43415|325|62086|358238|1143|31295|4|218|275|73|53.76|53.76|23.65|0.00|1726.45|3924.48|3924.48|17.26|0.00|1726.45|1743.71|-2198.03| +2451162|43415|13384|62086|358238|1143|31295|4|293|275|75|72.15|103.17|13.41|0.00|1005.75|5411.25|7737.75|30.17|0.00|1005.75|1035.92|-4405.50| +2451162|43415|11305|62086|358238|1143|31295|4|89|275|58|60.84|90.65|10.87|0.00|630.46|3528.72|5257.70|31.52|0.00|630.46|661.98|-2898.26| +2451162|43415|11954|62086|358238|1143|31295|4|152|275|75|68.07|68.07|12.25|643.12|918.75|5105.25|5105.25|24.80|643.12|275.63|300.43|-4829.62| +2451162|43415|15415|62086|358238|1143|31295|4|218|275|96|98.25|132.63|62.33|0.00|5983.68|9432.00|12732.48|538.53|0.00|5983.68|6522.21|-3448.32| +2451903|61718|746|75100|1169970|5193|10416|1|1|276|49|25.80|30.44|23.13|0.00|1133.37|1264.20|1491.56|79.33|0.00|1133.37|1212.70|-130.83| +2451903|61718|17540|75100|1169970|5193|10416|1|271|276|56|74.02|117.69|75.32|0.00|4217.92|4145.12|6590.64|253.07|0.00|4217.92|4470.99|72.80| +2451903|61718|1379|75100|1169970|5193|10416|1|87|276|98|48.71|64.78|58.30|0.00|5713.40|4773.58|6348.44|285.67|0.00|5713.40|5999.07|939.82| +2451903|61718|12631|75100|1169970|5193|10416|1|274|276|91|33.96|58.07|17.42|0.00|1585.22|3090.36|5284.37|0.00|0.00|1585.22|1585.22|-1505.14| +2451903|61718|9061|75100|1169970|5193|10416|1|7|276|91|85.15|131.98|1.31|0.00|119.21|7748.65|12010.18|7.15|0.00|119.21|126.36|-7629.44| +2451903|61718|13355|75100|1169970|5193|10416|1|66|276|54|28.09|51.12|20.44|540.84|1103.76|1516.86|2760.48|11.25|540.84|562.92|574.17|-953.94| +2451903|61718|5791|75100|1169970|5193|10416|1|22|276|18|38.12|72.42|44.17|55.65|795.06|686.16|1303.56|0.00|55.65|739.41|739.41|53.25| +2451903|61718|623|75100||5193||1|165|276|||174.15||1265.40||5395.05|||1265.40|421.80||| +2451903|61718|8347|75100|1169970|5193|10416|1|133|276|67|20.55|24.45|4.64|0.00|310.88|1376.85|1638.15|15.54|0.00|310.88|326.42|-1065.97| +2452174|39760|7319|73965|334166|4767|40568|8|43|277|16|30.78|43.39|42.52|0.00|680.32|492.48|694.24|61.22|0.00|680.32|741.54|187.84| +2452174|39760|15091|73965|334166|4767|40568|8|43|277|25|76.71|116.59|18.65|0.00|466.25|1917.75|2914.75|23.31|0.00|466.25|489.56|-1451.50| +2452174|39760|12585|73965|334166|4767|40568|8|153|277|43|27.11|32.26|4.83|4.15|207.69|1165.73|1387.18|2.03|4.15|203.54|205.57|-962.19| +2452174|39760|17459|73965|334166|4767|40568|8|26|277|23|76.11|99.70|50.84|0.00|1169.32|1750.53|2293.10|81.85|0.00|1169.32|1251.17|-581.21| +2452174|39760|8659|73965|334166|4767|40568|8|136|277|100|4.12|7.12|2.56|0.00|256.00|412.00|712.00|17.92|0.00|256.00|273.92|-156.00| +2452174|39760|8577|73965|334166|4767|40568|8|287|277|52|5.26|7.83|7.67|0.00|398.84|273.52|407.16|15.95|0.00|398.84|414.79|125.32| +2452174|39760|8441|73965|334166|4767|40568|8|165|277|41|25.99|47.30|0.94|0.00|38.54|1065.59|1939.30|0.00|0.00|38.54|38.54|-1027.05| +2452174|39760|6663|73965|334166|4767|40568|8|144|277|52|73.95|96.13|48.06|0.00|2499.12|3845.40|4998.76|24.99|0.00|2499.12|2524.11|-1346.28| +2452174|39760|16203|73965|334166|4767|40568|8|138|277|75|49.42|82.53|15.68|987.84|1176.00|3706.50|6189.75|15.05|987.84|188.16|203.21|-3518.34| +2452174|39760|13145|73965|334166|4767|40568|8|142|277|93|73.01|144.55|10.11|0.00|940.23|6789.93|13443.15|56.41|0.00|940.23|996.64|-5849.70| +2451479|72152|4004|38327|1230301|322|23496|1|7|278|42|69.70|108.03|19.44|0.00|816.48|2927.40|4537.26|73.48|0.00|816.48|889.96|-2110.92| +2451479|72152|2665|38327|1230301|322|23496|1|200|278|94|73.86|127.03|19.05|0.00|1790.70|6942.84|11940.82|17.90|0.00|1790.70|1808.60|-5152.14| +2451479|72152|12878|38327|1230301|322|23496|1|198|278|71|16.31|26.42|20.87|0.00|1481.77|1158.01|1875.82|133.35|0.00|1481.77|1615.12|323.76| +2451479|72152|7750|38327|1230301|322|23496|1|178|278|56|35.21|69.01|20.70|0.00|1159.20|1971.76|3864.56|34.77|0.00|1159.20|1193.97|-812.56| +2451479|72152|7646|38327|1230301|322|23496|1|192|278|95|77.86|143.26|78.79|0.00|7485.05|7396.70|13609.70|673.65|0.00|7485.05|8158.70|88.35| +2451479||11300|||322|||237|278|27||38.04|26.24||708.48|917.19|1027.08|49.59|||758.07|| +2451479|72152|7544|38327|1230301|322|23496|1|9|278|43|77.98|152.06|50.17|0.00|2157.31|3353.14|6538.58|107.86|0.00|2157.31|2265.17|-1195.83| +2451479|72152|14983|38327|1230301|322|23496|1|209|278|33|19.36|27.49|11.82|0.00|390.06|638.88|907.17|7.80|0.00|390.06|397.86|-248.82| +2450964|37945|1516|153|1759856|2312|35215|10|288|279|33|35.35|37.11|21.89|187.81|722.37|1166.55|1224.63|42.76|187.81|534.56|577.32|-631.99| +2450964|37945|14371|153|1759856|2312|35215|10|284|279|34|24.51|37.74|22.26|211.91|756.84|833.34|1283.16|16.34|211.91|544.93|561.27|-288.41| +2450964|37945|2149|153|1759856|2312|35215|10|281|279|47|13.21|19.15|12.83|0.00|603.01|620.87|900.05|12.06|0.00|603.01|615.07|-17.86| +2450964|37945|15254|153|1759856|||||279|||||706.49||||148.36|706.49|2119.47|2267.83|-5936.15| +2450964|37945|15055|153|1759856|2312|35215|10|24|279|39|99.38|164.97|118.77|0.00|4632.03|3875.82|6433.83|92.64|0.00|4632.03|4724.67|756.21| +2450964|37945|6505|153|1759856|2312|35215|10|255|279|33|23.59|29.48|22.99|0.00|758.67|778.47|972.84|45.52|0.00|758.67|804.19|-19.80| +2450964|37945|14234|153|1759856|2312|35215|10|24|279|5|19.68|38.96|8.96|42.11|44.80|98.40|194.80|0.02|42.11|2.69|2.71|-95.71| +2450964|37945|10237|153|1759856|2312|35215|10|112|279|55|61.04|98.27|11.79|0.00|648.45|3357.20|5404.85|58.36|0.00|648.45|706.81|-2708.75| +2450964|37945|2522|153|1759856|2312|35215|10|271|279|10|72.77|74.95|19.48|75.97|194.80|727.70|749.50|3.56|75.97|118.83|122.39|-608.87| +2450964|37945|8584|153|1759856|2312|35215|10|80|279|97|21.42|27.41|10.68|0.00|1035.96|2077.74|2658.77|93.23|0.00|1035.96|1129.19|-1041.78| +2450964|37945|2090|153|1759856|2312|35215|10|268|279|65|85.60|164.35|126.54|3536.79|8225.10|5564.00|10682.75|375.06|3536.79|4688.31|5063.37|-875.69| +2450990|33037|8101|95014|855476|1891|21553|4|197|280|56|19.48|28.83|2.88|0.00|161.28|1090.88|1614.48|1.61|0.00|161.28|162.89|-929.60| +2450990|33037|8024|95014|855476|1891|21553|4|75|280|62|85.47|145.29|17.43|0.00|1080.66|5299.14|9007.98|10.80|0.00|1080.66|1091.46|-4218.48| +2450990|33037|8605|95014|855476|1891|21553|4|115|280|38|33.57|53.37|51.76|0.00|1966.88|1275.66|2028.06|0.00|0.00|1966.88|1966.88|691.22| +2450990|33037|2888|95014|855476|1891|21553|4|160|280|24|15.57|29.11|28.23|0.00|677.52|373.68|698.64|47.42|0.00|677.52|724.94|303.84| +2450990|33037|13922|95014|855476|1891|21553|4|224|280|32|32.63|45.35|41.26|0.00|1320.32|1044.16|1451.20|0.00|0.00|1320.32|1320.32|276.16| +2450990|33037|14360|95014||1891|||252|280|72||39.85||0.00||1965.60|2869.20|2.27|0.00|56.88|59.15|-1908.72| +2450990|33037|4675|95014|855476|1891|21553|4|216|280|42|48.03|59.07|51.98|0.00|2183.16|2017.26|2480.94|152.82|0.00|2183.16|2335.98|165.90| +2450990|33037|6350|95014|855476|1891|21553|4|212|280|28|23.49|25.13|9.04|0.00|253.12|657.72|703.64|17.71|0.00|253.12|270.83|-404.60| +2451176|67834|12184|62016|958817|4418|40019|7|266|281|27|61.06|61.67|24.66|0.00|665.82|1648.62|1665.09|53.26|0.00|665.82|719.08|-982.80| +2451176|67834|13744|62016|958817|4418|40019|7|74|281|15|55.98|62.13|49.08|0.00|736.20|839.70|931.95|7.36|0.00|736.20|743.56|-103.50| +2451176|67834|7624|62016|958817|4418|40019|7|19|281|94|36.79|43.78|30.64|0.00|2880.16|3458.26|4115.32|115.20|0.00|2880.16|2995.36|-578.10| +2451176|67834|9284|62016|958817|4418|40019|7|257|281|51|93.83|153.88|29.23|193.79|1490.73|4785.33|7847.88|51.87|193.79|1296.94|1348.81|-3488.39| +2451176||14498|62016|958817||40019|7||281|40|57.08|||0.00|978.00|||0.00|0.00|978.00||| +2451176|67834|7880|62016|958817|4418|40019|7|250|281|41|30.92|42.05|16.82|0.00|689.62|1267.72|1724.05|41.37|0.00|689.62|730.99|-578.10| +2451176|67834|16543|62016|958817|4418|40019|7|173|281|52|26.83|46.14|43.83|934.45|2279.16|1395.16|2399.28|80.68|934.45|1344.71|1425.39|-50.45| +2451176|67834|14605||958817||40019|7||281|11||119.02||0.00|497.42|872.85|1309.22||0.00|497.42||-375.43| +2451176|67834|14233|62016|958817|4418|40019|7|182|281|90|11.44|14.30|13.44|0.00|1209.60|1029.60|1287.00|84.67|0.00|1209.60|1294.27|180.00| +2451176|67834|10759|62016|958817|4418|40019|7|74|281|28|35.33|50.16|18.55|0.00|519.40|989.24|1404.48|0.00|0.00|519.40|519.40|-469.84| +2451176|67834|10504|62016|958817|4418|40019|7|265|281|63|75.55|146.56|133.36|6469.29|8401.68|4759.65|9233.28|57.97|6469.29|1932.39|1990.36|-2827.26| +2451176|67834|5212|62016|958817|4418|40019|7|236|281|69|23.71|42.91|42.91|2339.02|2960.79|1635.99|2960.79|24.87|2339.02|621.77|646.64|-1014.22| +2451176|67834|7540|62016|958817|4418|40019|7|62|281|1|39.18|77.18|60.97|0.00|60.97|39.18|77.18|1.82|0.00|60.97|62.79|21.79| +2451176|67834|14722|62016|958817|4418|40019|7|126|281|33|92.67|101.93|9.17|0.00|302.61|3058.11|3363.69|0.00|0.00|302.61|302.61|-2755.50| +2451176|67834|16352|62016|958817|4418|40019|7|90|281|10|19.22|27.48|17.31|0.00|173.10|192.20|274.80|8.65|0.00|173.10|181.75|-19.10| +2451176|67834|13225|62016|958817|4418|40019|7|232|281|52|12.06|13.62|1.36|48.08|70.72|627.12|708.24|1.81|48.08|22.64|24.45|-604.48| +2452055|45725|8945|927|614037|5900|20563|2|124|282|8|28.58|30.86|13.57|0.00|108.56|228.64|246.88|1.08|0.00|108.56|109.64|-120.08| +2452055|45725|10569|927|614037|5900|20563|2|90|282|73|90.11|168.50|141.54|0.00|10332.42|6578.03|12300.50|929.91|0.00|10332.42|11262.33|3754.39| +2452055|45725|17529|927|614037|5900|20563|2|57|282|59|26.88|29.29|22.26|0.00|1313.34|1585.92|1728.11|0.00|0.00|1313.34|1313.34|-272.58| +2452055|45725|4847|927|614037|5900|20563|2|199|282|18|24.45|38.63|31.29|0.00|563.22|440.10|695.34|28.16|0.00|563.22|591.38|123.12| +2452055|45725|491|927|614037|5900|20563|2|32|282|94|8.33|10.07|0.30|0.00|28.20|783.02|946.58|1.69|0.00|28.20|29.89|-754.82| +2452055|45725|8015|927|614037|5900|20563|2|227|282|26|75.85|81.91|31.94|0.00|830.44|1972.10|2129.66|16.60|0.00|830.44|847.04|-1141.66| +2452055|45725|13953|927||5900||2||282|57||||0.00|6162.84|4509.84|7801.59||0.00|6162.84|6286.09|| +2452055|45725|6619|927|614037|5900|20563|2|83|282|54|48.83|64.45|57.36|0.00|3097.44|2636.82|3480.30|61.94|0.00|3097.44|3159.38|460.62| +|45725|10779||614037||||267|282|22|87.78|162.39||0.00|535.70||3572.58||0.00|||-1395.46| +2452055|45725|16017|927|614037|5900|20563|2|142|282|32|34.10|51.49|2.05|22.96|65.60|1091.20|1647.68|1.70|22.96|42.64|44.34|-1048.56| +2451524|54118|12679|51887|687347|5207|12714|7|6|283|69|51.12|54.18|24.38|0.00|1682.22|3527.28|3738.42|84.11|0.00|1682.22|1766.33|-1845.06| +2451524|54118|11173|51887|687347|5207|12714|7|223|283|45|20.75|31.95|12.14|0.00|546.30|933.75|1437.75|27.31|0.00|546.30|573.61|-387.45| +2451524|54118|17401|51887|687347|5207|12714|7|97|283|48|97.58|126.85|107.82|0.00|5175.36|4683.84|6088.80|414.02|0.00|5175.36|5589.38|491.52| +2451524|54118|932|51887|687347|5207|12714|7|55|283|64|12.27|15.46|15.46|0.00|989.44|785.28|989.44|69.26|0.00|989.44|1058.70|204.16| +2451524|54118|17914|51887|687347|5207|12714|7|189|283|20|12.63|24.62|12.80|0.00|256.00|252.60|492.40|0.00|0.00|256.00|256.00|3.40| +2451524|54118|14870|51887|687347|5207|12714|7|158|283|49|50.36|65.97|30.34|0.00|1486.66|2467.64|3232.53|14.86|0.00|1486.66|1501.52|-980.98| +2451524|54118|2011|||5207||7||283||70.94||83.02|59.77|249.06||283.05|3.78|59.77|||| +2451524|54118|9376|51887|687347|5207|12714|7|69|283|69|75.42|120.67|51.88|1038.11|3579.72|5203.98|8326.23|203.32|1038.11|2541.61|2744.93|-2662.37| +2451524|54118|17108|51887|687347|5207|12714|7|97|283|1|21.52|23.88|2.62|0.00|2.62|21.52|23.88|0.15|0.00|2.62|2.77|-18.90| +2451524|54118|7261|51887|687347|5207|12714|7|181|283|79|78.10|107.77|5.38|0.00|425.02|6169.90|8513.83|0.00|0.00|425.02|425.02|-5744.88| +2451524|54118|883|51887|687347|5207|12714|7|10|283|23|84.16|134.65|131.95|0.00|3034.85|1935.68|3096.95|212.43|0.00|3034.85|3247.28|1099.17| +2451524|54118|16750|51887|687347|5207|12714|7|255|283|47|35.25|37.36|14.19|600.23|666.93|1656.75|1755.92|6.00|600.23|66.70|72.70|-1590.05| +2451524|54118|4711|51887|687347|5207|12714|7|14|283|11|99.24|192.52|86.63|0.00|952.93|1091.64|2117.72|76.23|0.00|952.93|1029.16|-138.71| +2452590|41632|5406|19453|261432|1458|31037|2|43|284|73|43.69|59.85|0.00|0.00|0.00|3189.37|4369.05|0.00|0.00|0.00|0.00|-3189.37| +2452590|41632|6582|19453|261432|1458|31037|2|229|284|100|18.86|29.61|23.09|0.00|2309.00|1886.00|2961.00|46.18|0.00|2309.00|2355.18|423.00| +||4111|19453||||||284|46|36.58|53.40|22.96||1056.16||2456.40|||264.04||-1418.64| +2452590|41632|17419|19453|261432|1458|31037|2|16|284|24|43.16|54.38|41.87|954.63|1004.88|1035.84|1305.12|4.02|954.63|50.25|54.27|-985.59| +2452590|41632|13201|19453|261432|1458|31037|2|170|284|15|90.31|101.14|81.92|0.00|1228.80|1354.65|1517.10|0.00|0.00|1228.80|1228.80|-125.85| +2452590|41632|727|19453|261432|1458|31037|2|93|284|63|64.68|129.36|121.59|0.00|7660.17|4074.84|8149.68|229.80|0.00|7660.17|7889.97|3585.33| +2452590|41632|11809|19453|261432|1458|31037|2|279|284|8|39.27|59.29|58.10|60.42|464.80|314.16|474.32|12.13|60.42|404.38|416.51|90.22| +2452590|41632|2065|19453|261432|1458|31037|2|259|284|70|36.08|45.82|5.49|149.87|384.30|2525.60|3207.40|11.72|149.87|234.43|246.15|-2291.17| +||8493|||||2||284|13|96.89||8.83|34.43|114.79||1435.85||34.43|80.36|83.57|| +2451486|62473|2216|70345|1638245|3745|30845|1|81|285|6|94.29|158.40|104.54|0.00|627.24|565.74|950.40|6.27|0.00|627.24|633.51|61.50| +2451486|62473|5654|70345|1638245|3745|30845|1|287|285|15|61.81|81.58|7.34|0.00|110.10|927.15|1223.70|1.10|0.00|110.10|111.20|-817.05| +2451486|62473|15382|70345|1638245|3745|30845|1|144|285|27|83.72|132.27|54.23|1390.99|1464.21|2260.44|3571.29|0.73|1390.99|73.22|73.95|-2187.22| +2451486|62473|13472|70345|1638245|3745|30845|1|206|285|7|12.91|22.33|8.93|57.50|62.51|90.37|156.31|0.05|57.50|5.01|5.06|-85.36| +2451486|62473|10879|70345|1638245|3745|30845|1|207|285|70|93.52|126.25|0.00|0.00|0.00|6546.40|8837.50|0.00|0.00|0.00|0.00|-6546.40| +2451486|62473|9500|70345|1638245|3745|30845|1|286|285|42|89.98|178.16|117.58|0.00|4938.36|3779.16|7482.72|197.53|0.00|4938.36|5135.89|1159.20| +2451486|62473|11932|70345|1638245|3745|30845|||285||17.27|||99.72|174.96|414.48||2.25|99.72|75.24|77.49|-339.24| +||962||1638245|3745|30845|1||285||||5.71||531.03|||||||-4100.37| +2451486|62473|6142|70345|1638245|3745|30845|1|205|285|63|27.64|39.52|11.06|90.58|696.78|1741.32|2489.76|48.49|90.58|606.20|654.69|-1135.12| +2451486|62473|10684|70345|1638245|3745|30845|1|100|285|33|27.01|33.22|6.64|0.00|219.12|891.33|1096.26|0.00|0.00|219.12|219.12|-672.21| +2451486|62473|16450|70345|1638245|3745|30845|1|91|285|13|96.80|132.61|102.10|0.00|1327.30|1258.40|1723.93|106.18|0.00|1327.30|1433.48|68.90| +2451320|74389|14084|52550|644054|4805|41156|7|130|286|69|26.15|30.59|23.24|0.00|1603.56|1804.35|2110.71|64.14|0.00|1603.56|1667.70|-200.79| +2451320|74389|2617|52550|644054|4805|41156|7|113|286|36|8.70|12.52|2.37|0.00|85.32|313.20|450.72|5.11|0.00|85.32|90.43|-227.88| +2451320|74389|2744|52550|644054|4805|41156|7|269|286|29|91.78|164.28|133.06|0.00|3858.74|2661.62|4764.12|154.34|0.00|3858.74|4013.08|1197.12| +2451320|74389|17768|52550|644054|4805|41156|7|299|286|8|69.86|121.55|12.15|0.00|97.20|558.88|972.40|0.97|0.00|97.20|98.17|-461.68| +2451320|74389|7316|52550|644054|4805|41156|7|212|286|46|63.64|112.00|68.32|0.00|3142.72|2927.44|5152.00|157.13|0.00|3142.72|3299.85|215.28| +2451320|74389|14563|52550|644054|4805|41156|7|64|286|1|64.58|83.95|45.33|0.00|45.33|64.58|83.95|1.35|0.00|45.33|46.68|-19.25| +2451320|74389|6844|52550|644054|4805|41156|7||286|64|70.03|102.94|56.61||||6588.16|326.07|||3949.11|| +2451320|74389|3794|52550|644054|4805|41156|7|267|286|73|96.79|131.63|35.54|0.00|2594.42|7065.67|9608.99|129.72|0.00|2594.42|2724.14|-4471.25| +2451320|74389|11978|52550|644054|4805|41156|7|84|286|23|56.90|108.11|24.86|0.00|571.78|1308.70|2486.53|17.15|0.00|571.78|588.93|-736.92| +2451320|74389|9463|52550|644054|4805|41156|7|71|286|39|50.69|66.91|28.77|0.00|1122.03|1976.91|2609.49|78.54|0.00|1122.03|1200.57|-854.88| +2451320|74389|11552|52550|644054|4805|41156|7|220|286|58|65.99|83.14|32.42|0.00|1880.36|3827.42|4822.12|131.62|0.00|1880.36|2011.98|-1947.06| +2451148|39855|1771|58576|1618504|1632|3160|1|161|287|88|70.02|138.63|55.45|0.00|4879.60|6161.76|12199.44|48.79|0.00|4879.60|4928.39|-1282.16| +2451148|39855|3782|58576|1618504|1632|3160|1|71|287|31|74.07|143.69|44.54|0.00|1380.74|2296.17|4454.39|82.84|0.00|1380.74|1463.58|-915.43| +2451148|39855|14522|58576|1618504|1632|3160|1|278|287|16|46.99|85.99|0.85|10.74|13.60|751.84|1375.84|0.11|10.74|2.86|2.97|-748.98| +2451148|39855|2443|58576|1618504|1632|3160|1|2|287|63|39.05|53.49|36.90|0.00|2324.70|2460.15|3369.87|185.97|0.00|2324.70|2510.67|-135.45| +2451148|39855|17774|58576|1618504|1632|3160|1|117|287|12|42.91|80.67|70.98|59.62|851.76|514.92|968.04|0.00|59.62|792.14|792.14|277.22| +2451148|39855|3374|58576|1618504|1632|3160|1|253|287|56|68.99|75.88|46.28|0.00|2591.68|3863.44|4249.28|181.41|0.00|2591.68|2773.09|-1271.76| +2451148|39855|10958|58576|1618504|1632|3160|1|275|287|36|7.37|11.27|7.77|151.04|279.72|265.32|405.72|0.00|151.04|128.68|128.68|-136.64| +2451148|39855|8974|58576|1618504|1632|3160|1|260|287|52|43.63|65.88|23.05|47.94|1198.60|2268.76|3425.76|92.05|47.94|1150.66|1242.71|-1118.10| +2451148|39855|12175|58576|1618504|1632|3160|1|61|287|23|14.91|26.24|15.48|0.00|356.04|342.93|603.52|0.00|0.00|356.04|356.04|13.11| +2451148|39855|4168|58576|1618504|1632|3160|1|2|287|32|85.02|99.47|40.78|0.00|1304.96|2720.64|3183.04|13.04|0.00|1304.96|1318.00|-1415.68| +2451148|39855|2626|58576|1618504|1632|3160|1|101|287|53|51.13|55.73|30.65|0.00|1624.45|2709.89|2953.69|32.48|0.00|1624.45|1656.93|-1085.44| +2451148|39855|7940|58576|1618504|1632|3160|1|273|287|63|38.33|60.56|42.99|0.00|2708.37|2414.79|3815.28|81.25|0.00|2708.37|2789.62|293.58| +2451148|39855|17696|58576|1618504|1632|3160|1|170|287|92|36.03|58.00|10.44|0.00|960.48|3314.76|5336.00|57.62|0.00|960.48|1018.10|-2354.28| +2451148|39855|1825|58576|1618504|1632|3160|1|28|287|25|4.42|5.48|5.48|13.70|137.00|110.50|137.00|1.23|13.70|123.30|124.53|12.80| +||14711||269034||6030||83|288||36.16|||838.68||||124.60|838.68|1557.56|1682.16|| +2451958|62223|14001|51783|269034|1132|6030|8|72|288|1|18.13|25.56|5.11|0.00|5.11|18.13|25.56|0.25|0.00|5.11|5.36|-13.02| +2451958|62223|643|51783|269034|1132|6030|8|80|288|43|47.06|50.35|15.60|0.00|670.80|2023.58|2165.05|60.37|0.00|670.80|731.17|-1352.78| +2451958|62223|3379|51783|269034|1132|6030|8|240|288|32|39.09|42.21|3.37|0.00|107.84|1250.88|1350.72|9.70|0.00|107.84|117.54|-1143.04| +2451958|62223|9279|51783|269034|1132|6030|8|106|288|56|68.10|127.34|105.69|0.00|5918.64|3813.60|7131.04|177.55|0.00|5918.64|6096.19|2105.04| +2451958|62223|15647|51783|269034|1132|6030|8|123|288|59|65.21|127.81|83.07|2940.67|4901.13|3847.39|7540.79|0.00|2940.67|1960.46|1960.46|-1886.93| +2451958|62223|10341|51783|269034|1132|6030|8|209|288|6|40.17|44.18|7.06|0.00|42.36|241.02|265.08|0.42|0.00|42.36|42.78|-198.66| +2451958|62223|13413|51783|269034|1132|6030|8|221|288|67|54.11|69.80|69.10|0.00|4629.70|3625.37|4676.60|416.67|0.00|4629.70|5046.37|1004.33| +2451958|62223|9925|51783|269034|1132|6030|8|51|288|34|9.80|17.05|2.55|69.36|86.70|333.20|579.70|0.00|69.36|17.34|17.34|-315.86| +2451958|62223|2161|51783|269034|1132|6030|8|254|288|42|49.54|61.42|38.69|0.00|1624.98|2080.68|2579.64|0.00|0.00|1624.98|1624.98|-455.70| +2451084|52103|3505|49526|996650|6621|23792|1|30|289|49|91.15|154.04|86.26|0.00|4226.74|4466.35|7547.96|253.60|0.00|4226.74|4480.34|-239.61| +||14572||||23792|1|250|289|59||42.96|29.21|0.00||1362.90|2534.64||0.00|1723.39||360.49| +2451084|52103|1844|49526|996650|6621|23792|1|203|289|30|64.61|74.94|59.20|88.80|1776.00|1938.30|2248.20|16.87|88.80|1687.20|1704.07|-251.10| +2451084|52103|14792|49526|996650|6621|23792|1|19|289|78|93.93|154.04|23.10|0.00|1801.80|7326.54|12015.12|162.16|0.00|1801.80|1963.96|-5524.74| +2451084|52103|8173|49526|996650|6621|23792|1|66|289|45|83.33|141.66|94.91|939.60|4270.95|3749.85|6374.70|99.94|939.60|3331.35|3431.29|-418.50| +|52103|3986|||6621|23792||292|289|45|24.87|43.77|0.43||19.35|1119.15||1.74||||| +2451084|52103|12763|49526|996650|6621|23792|1|290|289|97|69.78|117.23|83.23|0.00|8073.31|6768.66|11371.31|726.59|0.00|8073.31|8799.90|1304.65| +2451084|52103|13742|49526|996650|6621|23792|1|70|289|83|34.52|63.86|7.02|0.00|582.66|2865.16|5300.38|29.13|0.00|582.66|611.79|-2282.50| +2452248|45661|2329|12059|109804|4702|800|10|159|290|89|68.08|133.43|14.67|0.00|1305.63|6059.12|11875.27|0.00|0.00|1305.63|1305.63|-4753.49| +2452248|45661|10893|12059|109804|4702|800|10|155|290|90|93.89|101.40|73.00|0.00|6570.00|8450.10|9126.00|65.70|0.00|6570.00|6635.70|-1880.10| +2452248|45661|3419|12059|109804|4702|800|10|84|290|52|23.16|37.98|24.68|0.00|1283.36|1204.32|1974.96|102.66|0.00|1283.36|1386.02|79.04| +2452248|45661|16699|12059|109804|4702|800|10|35|290|97|67.03|85.79|72.06|0.00|6989.82|6501.91|8321.63|419.38|0.00|6989.82|7409.20|487.91| +2452248|45661|8507|12059|109804|4702|800|10|197|290|29|6.60|11.74|6.80|0.00|197.20|191.40|340.46|13.80|0.00|197.20|211.00|5.80| +2452248|45661|4875|12059|109804|4702|800|10|102|290|91|98.77|163.95|14.75|0.00|1342.25|8988.07|14919.45|80.53|0.00|1342.25|1422.78|-7645.82| +2452248|45661|17825|12059|109804|4702|800|10|180|290|76|64.86|87.56|84.93|0.00|6454.68|4929.36|6654.56|64.54|0.00|6454.68|6519.22|1525.32| +2452248|45661|7023|12059|109804|4702|800|10|203|290|56|59.00|106.79|5.33|0.00|298.48|3304.00|5980.24|5.96|0.00|298.48|304.44|-3005.52| +2452248|45661|17065|12059|109804|4702|800|10|234|290|14|38.09|65.51|41.27|0.00|577.78|533.26|917.14|28.88|0.00|577.78|606.66|44.52| +2450864|44189|4174|84620|1079386|2505|71|7|264|291|37|15.57|19.30|17.56|0.00|649.72|576.09|714.10|32.48|0.00|649.72|682.20|73.63| +2450864|44189|3799|84620|1079386|2505|71|7|150|291|51|96.93|109.53|62.43|350.23|3183.93|4943.43|5586.03|85.01|350.23|2833.70|2918.71|-2109.73| +2450864|44189|2569|84620|1079386|2505|71|7|5|291|51|97.18|187.55|133.16|1969.43|6791.16|4956.18|9565.05|337.52|1969.43|4821.73|5159.25|-134.45| +2450864|44189|4612|84620|1079386|2505|71|7|59|291|58|9.48|18.10|13.93|807.94|807.94|549.84|1049.80|0.00|807.94|0.00|0.00|-549.84| +2450864|44189|6176|84620|1079386|2505|71|7|42|291|4|82.70|123.22|65.30|0.00|261.20|330.80|492.88|18.28|0.00|261.20|279.48|-69.60| +|44189|10852|84620|1079386||71|||291|45|85.08|145.48|96.01||4320.45||6546.60|||||491.85| +2450864|44189|6212|84620|1079386|2505|71|7|175|291|33|2.40|4.05|1.62|0.00|53.46|79.20|133.65|2.67|0.00|53.46|56.13|-25.74| +2450864|44189|7970|84620|1079386|2505|71|7|291|291|7|50.94|93.22|27.96|0.00|195.72|356.58|652.54|7.82|0.00|195.72|203.54|-160.86| +2450864|44189|9181|84620|1079386|2505|71|7|292|291|7|21.31|21.31|8.95|0.00|62.65|149.17|149.17|1.87|0.00|62.65|64.52|-86.52| +2450864|44189|6757|84620|1079386|2505|71|7|52|291|81|45.56|82.46|70.91|0.00|5743.71|3690.36|6679.26|344.62|0.00|5743.71|6088.33|2053.35| +2450864|44189|344|84620|1079386|2505|71|7|227|291|84|17.38|29.19|15.76|0.00|1323.84|1459.92|2451.96|13.23|0.00|1323.84|1337.07|-136.08| +2450864|44189|4009|84620|1079386|2505|71|7|74|291||||55.06||3138.42||||||3326.72|| +2450940|29527|7760|17143|166857|1371|38685|10|227|292|66|69.37|110.99|22.19|0.00|1464.54|4578.42|7325.34|131.80|0.00|1464.54|1596.34|-3113.88| +2450940|29527|14107|17143|166857|1371|38685|10|37|292|3|28.87|53.98|41.56|0.00|124.68|86.61|161.94|2.49|0.00|124.68|127.17|38.07| +2450940|29527|10924|17143|166857|1371|38685|10|217|292|95|17.82|31.89|30.61|0.00|2907.95|1692.90|3029.55|232.63|0.00|2907.95|3140.58|1215.05| +||15260|17143|166857||38685|||292|23|7.05|11.06|9.40||||||||11.67|| +2450940|29527|2848|17143|166857|1371|38685|10|149|292|33|51.90|89.78|20.64|0.00|681.12|1712.70|2962.74|27.24|0.00|681.12|708.36|-1031.58| +2450940|29527|6674|17143|166857|1371|38685|10|9|292|29|9.81|16.48|8.73|0.00|253.17|284.49|477.92|10.12|0.00|253.17|263.29|-31.32| +||12430|17143||1371||10|182|292|||80.69|64.55|||||||3743.90|4043.41|1143.76| +2450940|29527|10712|17143|166857|1371|38685|10|192|292|19|78.71|116.49|27.95|0.00|531.05|1495.49|2213.31|26.55|0.00|531.05|557.60|-964.44| +2450940|29527|982|17143|166857|1371|38685|10|213|292|52|90.99|93.71|86.21|0.00|4482.92|4731.48|4872.92|224.14|0.00|4482.92|4707.06|-248.56| +2450940|29527|7748|17143|166857|1371|38685|10|181|292|95|77.32|142.26|119.49|8627.17|11351.55|7345.40|13514.70|190.70|8627.17|2724.38|2915.08|-4621.02| +2450940|29527|16448|17143|166857|1371|38685|10|296|292|73|59.40|59.99|0.00|0.00|0.00|4336.20|4379.27|0.00|0.00|0.00|0.00|-4336.20| +2452233|36390|143|78712|364230|2548|42466|2|28|293|88|71.17|85.40|18.78|0.00|1652.64|6262.96|7515.20|49.57|0.00|1652.64|1702.21|-4610.32| +2452233|36390|15921|78712|364230|2548|42466|2|80|293|50|56.38|91.89|60.64|0.00|3032.00|2819.00|4594.50|212.24|0.00|3032.00|3244.24|213.00| +2452233|36390|3255|78712|364230|2548|42466|2|286|293|70|27.74|47.99|26.87|601.88|1880.90|1941.80|3359.30|102.32|601.88|1279.02|1381.34|-662.78| +2452233|36390|15885|78712|364230|2548|42466|2|165|293|43|70.56|110.77|3.32|0.00|142.76|3034.08|4763.11|1.42|0.00|142.76|144.18|-2891.32| +2452233|36390|11513|78712|364230|2548|42466|2|42|293|22|96.09|167.19|43.46|0.00|956.12|2113.98|3678.18|57.36|0.00|956.12|1013.48|-1157.86| +2452233|36390|14085|78712|364230|2548|42466|2|260|293|34|61.41|66.32|35.81|0.00|1217.54|2087.94|2254.88|109.57|0.00|1217.54|1327.11|-870.40| +2452233|36390|2617|78712|364230|2548|42466|2|170|293|48|85.66|92.51|46.25|0.00|2220.00|4111.68|4440.48|44.40|0.00|2220.00|2264.40|-1891.68| +||2745||364230|2548||||293|66||131.08||0.00|1297.56||||0.00||1362.43|| +2452233|36390|17769|78712|364230|2548|42466|2|155|293|92|52.65|74.76|27.66|0.00|2544.72|4843.80|6877.92|203.57|0.00|2544.72|2748.29|-2299.08| +2452233|36390|7317|78712|364230|2548|42466|2|173|293|12|47.03|50.32|39.75|0.00|477.00|564.36|603.84|0.00|0.00|477.00|477.00|-87.36| +2452233|36390|14563|78712|364230|2548|42466|2|206|293|82|89.79|126.60|29.11|190.96|2387.02|7362.78|10381.20|21.96|190.96|2196.06|2218.02|-5166.72| +2452233|36390|6845|78712|364230|2548|42466|2|132|293|47|49.79|49.79|8.46|0.00|397.62|2340.13|2340.13|3.97|0.00|397.62|401.59|-1942.51| +2452233|36390|3795|78712|364230|2548|42466|2|204|293|62|23.80|24.03|10.09|0.00|625.58|1475.60|1489.86|12.51|0.00|625.58|638.09|-850.02| +2452233|36390|11979|78712|364230|2548|42466|2|266|293|74|79.83|96.59|73.40|0.00|5431.60|5907.42|7147.66|217.26|0.00|5431.60|5648.86|-475.82| +2452233|36390|9463|78712|364230|2548|42466|2|292|293|70|18.90|27.97|10.62|0.00|743.40|1323.00|1957.90|14.86|0.00|743.40|758.26|-579.60| +|50074|14273|21102|730752||16899|4||294||||36.77|0.00||2575.80|||0.00|3309.30||| +2451671|50074|15761|21102|730752|849|16899|4|137|294|46|51.63|84.67|27.94|0.00|1285.24|2374.98|3894.82|12.85|0.00|1285.24|1298.09|-1089.74| +2451671|50074|15152|21102|730752|849|16899|4|270|294|35|63.71|70.08|5.60|121.52|196.00|2229.85|2452.80|6.70|121.52|74.48|81.18|-2155.37| +2451671|50074|16496|21102|730752|849|16899|4|110|294|25|84.23|90.96|44.57|0.00|1114.25|2105.75|2274.00|100.28|0.00|1114.25|1214.53|-991.50| +2451671|50074|4751|21102|730752|849|16899|4|297|294|32|32.95|62.27|7.47|0.00|239.04|1054.40|1992.64|4.78|0.00|239.04|243.82|-815.36| +2451671|50074|17629|21102|730752|849|16899|4|134|294|49|92.16|148.37|114.24|0.00|5597.76|4515.84|7270.13|111.95|0.00|5597.76|5709.71|1081.92| +2451671|50074|8773|21102|730752|849|16899|4|300|294|60|88.37|88.37|14.13|0.00|847.80|5302.20|5302.20|59.34|0.00|847.80|907.14|-4454.40| +2451671|50074|12935|21102|730752|849|16899|4|106|294|58|47.89|85.72|43.71|0.00|2535.18|2777.62|4971.76|228.16|0.00|2535.18|2763.34|-242.44| +2451671|50074|9572|21102|730752|849|16899|4|26|294|30|43.97|43.97|5.71|0.00|171.30|1319.10|1319.10|8.56|0.00|171.30|179.86|-1147.80| +2451671|50074|6233|21102|730752|849|16899|4|206|294|31|28.75|30.18|0.90|16.74|27.90|891.25|935.58|0.89|16.74|11.16|12.05|-880.09| +2452525|62940|7237|68512|275814|4363|21665|8|61|295|80|48.38|53.21|50.01|1880.37|4000.80|3870.40|4256.80|42.40|1880.37|2120.43|2162.83|-1749.97| +2452525|62940|17100|68512|275814|4363|21665|8|121|295|68|89.15|97.17|78.70|0.00|5351.60|6062.20|6607.56|160.54|0.00|5351.60|5512.14|-710.60| +2452525|62940|6837|68512|275814|4363|21665|8|232|295|40|15.99|20.94|3.97|0.00|158.80|639.60|837.60|4.76|0.00|158.80|163.56|-480.80| +2452525|62940|14424|68512|275814|4363|21665|8|242|295|50|54.57|92.22|4.61|129.08|230.50|2728.50|4611.00|6.08|129.08|101.42|107.50|-2627.08| +2452525|62940|17556|68512|275814|4363|21665|8|250|295|61|86.39|162.41|1.62|0.00|98.82|5269.79|9907.01|0.00|0.00|98.82|98.82|-5170.97| +2452525|62940|12186|68512|275814|4363|21665|8|93|295|71|76.02|76.78|7.67|0.00|544.57|5397.42|5451.38|27.22|0.00|544.57|571.79|-4852.85| +2452525|62940|13746|68512|275814|4363|21665|8|236|295|54|33.47|51.20|11.26|0.00|608.04|1807.38|2764.80|42.56|0.00|608.04|650.60|-1199.34| +2452525|62940|7626|68512|275814|4363|21665|8|143|295|3|9.40|11.75|8.57|0.00|25.71|28.20|35.25|1.02|0.00|25.71|26.73|-2.49| +2452525|62940|9285|68512|275814|4363|21665|8|255|295|55|42.04|81.55|24.46|0.00|1345.30|2312.20|4485.25|0.00|0.00|1345.30|1345.30|-966.90| +2452525||14499||275814||21665|8||295|||3.69|||3.96|||||2.62|2.62|-11.96| +||7881||||21665||119|295||91.90||18.28|0.00|1681.76||16824.96|151.35|0.00|1681.76||-6773.04| +2452525|62940|16543|68512|275814|4363|21665|8|276|295|98|97.17|132.15|13.21|0.00|1294.58|9522.66|12950.70|38.83|0.00|1294.58|1333.41|-8228.08| +2452525|62940|14605|68512|275814|4363|21665|8|222|295|52|86.55|134.15|40.24|0.00|2092.48|4500.60|6975.80|20.92|0.00|2092.48|2113.40|-2408.12| +2452525|62940|14233|68512|275814|4363|21665|8|102|295|29|79.90|146.21|128.66|1082.03|3731.14|2317.10|4240.09|158.94|1082.03|2649.11|2808.05|332.01| +2452525|62940|10759|68512|275814|4363|21665|8|37|295|92|27.10|51.76|39.85|1979.74|3666.20|2493.20|4761.92|33.72|1979.74|1686.46|1720.18|-806.74| +2452525|62940|10506|68512|275814|4363|21665|8|1|295|62|84.58|167.46|1.67|0.00|103.54|5243.96|10382.52|2.07|0.00|103.54|105.61|-5140.42| +2452542|37975|6981|70451|434118|5432|47146|10|136|296|40|23.31|43.12|31.90|0.00|1276.00|932.40|1724.80|51.04|0.00|1276.00|1327.04|343.60| +2452542|37975|7836|70451|434118|5432|47146|10|283|296|7|50.63|83.03|82.19|0.00|575.33|354.41|581.21|5.75|0.00|575.33|581.08|220.92| +2452542|37975|3817|70451|434118|5432|47146|10|273|296|75|60.47|75.58|10.58|0.00|793.50|4535.25|5668.50|23.80|0.00|793.50|817.30|-3741.75| +2452542|37975|6511|70451|434118|5432|47146|10|116|296|80|61.12|105.12|31.53|0.00|2522.40|4889.60|8409.60|227.01|0.00|2522.40|2749.41|-2367.20| +2452542|37975|10863|70451|434118|5432|47146|10|45|296|16|9.98|12.47|7.60|0.00|121.60|159.68|199.52|4.86|0.00|121.60|126.46|-38.08| +2452542|37975|15912|70451|434118|5432|47146|10|31|296|97|26.52|47.73|19.56|0.00|1897.32|2572.44|4629.81|113.83|0.00|1897.32|2011.15|-675.12| +2452542||8592|70451||5432|47146||80|296|||19.94|||17.91||179.46|||17.91||-100.98| +2452542|37975|14697|70451|434118|5432|47146|10|299|296|1|32.92|37.52|28.89|2.60|28.89|32.92|37.52|2.10|2.60|26.29|28.39|-6.63| +2452542|37975|600|70451|434118|5432|47146|10|139|296|6|98.98|193.01|55.97|0.00|335.82|593.88|1158.06|10.07|0.00|335.82|345.89|-258.06| +2452542|37975|7782|70451|434118|5432|47146|10|47|296|68|65.67|130.68|45.73|2767.57|3109.64|4465.56|8886.24|30.78|2767.57|342.07|372.85|-4123.49| +2452542|37975|7509|70451|434118|5432|47146|10|226|296|94|26.76|51.37|6.67|0.00|626.98|2515.44|4828.78|43.88|0.00|626.98|670.86|-1888.46| +2452542|37975|1941|70451|434118|5432|47146|10|62|296|79|34.62|42.92|14.16|0.00|1118.64|2734.98|3390.68|44.74|0.00|1118.64|1163.38|-1616.34| +2452542|37975|11325|70451|434118|5432|47146|10|246|296|26|70.26|107.49|21.49|0.00|558.74|1826.76|2794.74|0.00|0.00|558.74|558.74|-1268.02| +2451419|62407|3728|25644|717255|866|24477|7|197|297|20|9.21|15.19|1.06|0.00|21.20|184.20|303.80|1.69|0.00|21.20|22.89|-163.00| +2451419|62407|17446|25644|717255|866|24477|7|2|297|58|57.42|103.93|9.35|0.00|542.30|3330.36|6027.94|10.84|0.00|542.30|553.14|-2788.06| +2451419|62407|4610|25644|717255|866|24477|7|19|297|26|48.44|53.76|12.90|0.00|335.40|1259.44|1397.76|3.35|0.00|335.40|338.75|-924.04| +2451419|62407|5174|25644|717255|866|24477|7|289|297|12|14.31|16.17|11.80|120.36|141.60|171.72|194.04|1.69|120.36|21.24|22.93|-150.48| +2451419|62407|11467|25644|717255|866|24477|7|6|297|35|20.06|38.11|31.63|0.00|1107.05|702.10|1333.85|99.63|0.00|1107.05|1206.68|404.95| +2451419|62407|12686|25644|717255|866|24477|7|39|297|33|20.33|20.53|1.43|0.00|47.19|670.89|677.49|0.94|0.00|47.19|48.13|-623.70| +2451419|62407|15092|25644|717255|866|24477|7|198|297|4|2.53|3.23|2.61|0.00|10.44|10.12|12.92|0.31|0.00|10.44|10.75|0.32| +2451419|62407|1856|25644|717255|866|24477|7|108|297|15|5.54|7.86|1.02|0.00|15.30|83.10|117.90|0.45|0.00|15.30|15.75|-67.80| +|62407|3127|25644|717255||24477|||297|||126.42|||||10492.86|||8708.36||| +2451419|62407|10561|25644|717255|866|24477|7|39|297|51|61.35|66.25|36.43|836.06|1857.93|3128.85|3378.75|40.87|836.06|1021.87|1062.74|-2106.98| +2451419|62407|14624|25644|717255|866|24477|7|79|297|24|37.40|40.01|34.80|175.39|835.20|897.60|960.24|52.78|175.39|659.81|712.59|-237.79| +2451419|62407|11812|25644|717255|866|24477|7|262|297|44|89.74|156.14|18.73|0.00|824.12|3948.56|6870.16|8.24|0.00|824.12|832.36|-3124.44| +2451419|62407|1996|25644|717255|866|24477|7|262|297|2|83.80|159.22|119.41|0.00|238.82|167.60|318.44|11.94|0.00|238.82|250.76|71.22| +|62612|1035|||||8||298|61||55.65|10.01||610.61|2108.77|3394.65|30.53|||641.14|-1498.16| +2452578|62612|4770|37107|1045882|5057|20232|8|113|298|39|63.15|76.41|34.38|0.00|1340.82|2462.85|2979.99|53.63|0.00|1340.82|1394.45|-1122.03| +2452578|62612|504|37107|1045882|5057|20232|8|122|298|62|73.10|100.14|2.00|0.00|124.00|4532.20|6208.68|4.96|0.00|124.00|128.96|-4408.20| +|62612|1357|||5057|||7|298|36|||16.52||594.72||3130.56|||594.72|600.66|-2104.20| +2452578|62612|15510|37107|1045882|5057|20232|8|101|298|93|64.35|70.14|39.97|0.00|3717.21|5984.55|6523.02|334.54|0.00|3717.21|4051.75|-2267.34| +2452578|62612|15822|37107|1045882|5057|20232|8|198|298|37|23.82|36.44|33.88|676.92|1253.56|881.34|1348.28|11.53|676.92|576.64|588.17|-304.70| +2452578|62612|6649|37107|1045882|5057|20232|8|207|298|88|66.25|113.95|42.16|0.00|3710.08|5830.00|10027.60|296.80|0.00|3710.08|4006.88|-2119.92| +2452578|62612|15523|37107|1045882|5057|20232|8|72|298|48|89.42|166.32|19.95|746.92|957.60|4292.16|7983.36|8.42|746.92|210.68|219.10|-4081.48| +2452578|62612|14335|37107|1045882|5057|20232|8|100|298|14|82.84|150.76|48.24|493.01|675.36|1159.76|2110.64|1.82|493.01|182.35|184.17|-977.41| +2452578|62612|16189|37107|1045882|5057|20232|8|127|298|19|41.43|59.65|16.10|0.00|305.90|787.17|1133.35|12.23|0.00|305.90|318.13|-481.27| +2452578|62612|1675|37107|1045882|5057|20232|8|83|298|96|78.83|119.03|73.79|0.00|7083.84|7567.68|11426.88|495.86|0.00|7083.84|7579.70|-483.84| +2452578|62612|5629|37107|1045882|5057|20232|8|46|298|93|18.26|28.30|1.98|0.00|184.14|1698.18|2631.90|3.68|0.00|184.14|187.82|-1514.04| +2452578|62612|16951|37107|1045882|5057|20232|8|212|298|86|68.93|130.96|61.55|0.00|5293.30|5927.98|11262.56|105.86|0.00|5293.30|5399.16|-634.68| +2452578|62612|8557|37107|1045882|5057|20232|8|90|298|89|66.77|99.48|70.63|0.00|6286.07|5942.53|8853.72|502.88|0.00|6286.07|6788.95|343.54| +2452578||528|37107|1045882|5057|20232|8|225|298|||44.19||0.00|2651.25|3041.25|3314.25||0.00|2651.25|2730.78|-390.00| +2452623|69343|8043|56734|1624461|4988|37162|1|96|299|89|39.02|53.06|28.65|0.00|2549.85|3472.78|4722.34|0.00|0.00|2549.85|2549.85|-922.93| +||14827|56734|||37162|||299||46.01|64.87|33.08|||2208.48|3113.76|||1333.79||-874.69| +2452623|69343|14833|56734|1624461|4988|37162|1|18|299|78|85.23|150.85|60.34|0.00|4706.52|6647.94|11766.30|376.52|0.00|4706.52|5083.04|-1941.42| +2452623|69343|11202|56734|1624461|4988|37162|1|156|299|91|14.87|29.29|21.38|0.00|1945.58|1353.17|2665.39|0.00|0.00|1945.58|1945.58|592.41| +2452623|69343|17095|56734|1624461|4988|37162|1|112|299|51|9.82|16.00|11.84|0.00|603.84|500.82|816.00|54.34|0.00|603.84|658.18|103.02| +2452623|69343|12745|56734|1624461|4988|37162|1|216|299|42|27.84|52.89|45.48|0.00|1910.16|1169.28|2221.38|133.71|0.00|1910.16|2043.87|740.88| +2452623|69343|1083|56734|1624461|4988|37162|1|237|299|95|50.73|80.66|41.13|0.00|3907.35|4819.35|7662.70|234.44|0.00|3907.35|4141.79|-912.00| +2452623|69343|768|56734|1624461|4988|37162|1|263|299|45|33.07|54.23|48.26|0.00|2171.70|1488.15|2440.35|43.43|0.00|2171.70|2215.13|683.55| +2452623|69343|16137|56734|1624461|4988|37162|1|106|299|25|49.93|69.40|67.31|0.00|1682.75|1248.25|1735.00|16.82|0.00|1682.75|1699.57|434.50| +2452623|69343|12372|56734|1624461|4988|37162|1|135|299|91|62.90|98.75|0.98|0.00|89.18|5723.90|8986.25|4.45|0.00|89.18|93.63|-5634.72| +2451867|58368|3710|51038|1415744|6050|40756|2|47|300|4|36.69|50.63|7.08|0.00|28.32|146.76|202.52|0.28|0.00|28.32|28.60|-118.44| +2451867|58368|17762|51038|1415744|6050|40756|2|15|300|41|10.94|17.94|15.60|0.00|639.60|448.54|735.54|0.00|0.00|639.60|639.60|191.06| +2451867|58368|16709|51038|1415744|6050|40756|2|48|300|18|46.51|86.50|22.49|97.15|404.82|837.18|1557.00|0.00|97.15|307.67|307.67|-529.51| +2451867|58368|7705|51038|1415744|6050|40756|2|29|300|43|1.58|2.22|0.75|4.51|32.25|67.94|95.46|1.38|4.51|27.74|29.12|-40.20| +2451867|58368|10987|51038|1415744|6050|40756|2|120|300|14|4.18|5.30|0.05|0.00|0.70|58.52|74.20|0.05|0.00|0.70|0.75|-57.82| +2451867|58368|865|51038|1415744|6050|40756|2|18|300|24|97.15|145.72|106.37|0.00|2552.88|2331.60|3497.28|0.00|0.00|2552.88|2552.88|221.28| +2451867|58368|5845|51038|1415744|6050|40756|2|128|300|37|22.46|35.03|4.90|19.94|181.30|831.02|1296.11|14.52|19.94|161.36|175.88|-669.66| +||7849|51038|1415744|6050|40756|2|276|300|||||||||137.42|||2100.58|354.96| +2451867|58368|7910|51038|1415744|6050|40756|2|99|300|12|27.85|37.87|11.73|0.00|140.76|334.20|454.44|7.03|0.00|140.76|147.79|-193.44| +2451457|71213|4114|88697|572392|6840|47335|1|147|301|2|39.81|47.37|11.36|18.85|22.72|79.62|94.74|0.23|18.85|3.87|4.10|-75.75| +|71213|9578||572392||47335|1||301||||7.80||772.20|472.23||||540.54||| +2451457|71213|11342|88697|572392|6840|47335|1|74|301|66|52.62|69.98|45.48|0.00|3001.68|3472.92|4618.68|60.03|0.00|3001.68|3061.71|-471.24| +2451457|71213|17912|88697|572392|6840|47335|1|186|301|58|54.84|87.19|39.23|0.00|2275.34|3180.72|5057.02|91.01|0.00|2275.34|2366.35|-905.38| +2451457|71213|6139|88697|572392|6840|47335|1|59|301|78|68.38|82.73|43.01|1207.72|3354.78|5333.64|6452.94|128.82|1207.72|2147.06|2275.88|-3186.58| +2451457|71213|5248|88697|572392|6840|47335|1|170|301|90|11.13|12.35|1.35|0.00|121.50|1001.70|1111.50|1.21|0.00|121.50|122.71|-880.20| +2451457|71213|15166|88697|572392|6840|47335|1|9|301|15|46.58|78.72|65.33|0.00|979.95|698.70|1180.80|0.00|0.00|979.95|979.95|281.25| +2451457|71213|5797|88697|572392|6840|47335|1|218|301|67|85.76|145.79|99.13|0.00|6641.71|5745.92|9767.93|199.25|0.00|6641.71|6840.96|895.79| +2451457|71213|9470||572392|6840|47335||222|301|18||60.45|||1000.98||1088.10|||1000.98|1031.00|| +2451457|71213|2474|88697|572392|6840|47335|1|264|301|72|95.97|144.91|73.90|0.00|5320.80|6909.84|10433.52|319.24|0.00|5320.80|5640.04|-1589.04| +2451457||16934|88697|||47335|||301|||154.17|15.41|488.18||2952.36||3.32|488.18||69.90|| +2451457|71213|9145||||47335|1|238|301|||||0.00|707.58|618.66|760.86|35.37|0.00||742.95|| +2451457|71213|5222|88697|572392|6840|47335|1|294|301|23|80.59|126.52|80.97|595.93|1862.31|1853.57|2909.96|12.66|595.93|1266.38|1279.04|-587.19| +2451457|71213|17708|88697|572392|6840|47335|1|6|301|8|65.54|102.24|0.00|0.00|0.00|524.32|817.92|0.00|0.00|0.00|0.00|-524.32| +2451457|71213|3535|88697|572392|6840|47335|1|161|301|77|75.81|85.66|21.41|0.00|1648.57|5837.37|6595.82|131.88|0.00|1648.57|1780.45|-4188.80| +2451539|44068|4195|76691|1032044|3844|46797|4|167|302|93|25.46|44.30|25.25|0.00|2348.25|2367.78|4119.90|46.96|0.00|2348.25|2395.21|-19.53| +2451539|44068|13231|76691|1032044|3844|46797|4|107|302|43|92.00|122.36|29.36|0.00|1262.48|3956.00|5261.48|37.87|0.00|1262.48|1300.35|-2693.52| +2451539|44068|2098|76691|1032044|3844|46797|4|164|302|18|99.32|171.82|146.04|446.88|2628.72|1787.76|3092.76|109.09|446.88|2181.84|2290.93|394.08| +2451539|44068|9274|76691|1032044|3844|46797|4|267|302|96|19.55|27.56|1.10|0.00|105.60|1876.80|2645.76|4.22|0.00|105.60|109.82|-1771.20| +2451539|44068|10288|76691|1032044|3844|46797|4|200|302|42|44.15|63.13|37.24|0.00|1564.08|1854.30|2651.46|46.92|0.00|1564.08|1611.00|-290.22| +2451539|44068|5858|76691|1032044|3844|46797|4|7|302|98|88.52|173.49|170.02|0.00|16661.96|8674.96|17002.02|1166.33|0.00|16661.96|17828.29|7987.00| +2451539|44068|10108|76691|1032044|3844|46797|4|57|302|63|5.07|7.50|3.97|0.00|250.11|319.41|472.50|22.50|0.00|250.11|272.61|-69.30| +2451539|44068|9997|76691|1032044|3844|46797|4|154|302|79|81.62|110.18|51.78|0.00|4090.62|6447.98|8704.22|81.81|0.00|4090.62|4172.43|-2357.36| +2451539|44068|17246|76691|1032044|3844|46797|4|73|302|54|34.15|37.90|30.69|0.00|1657.26|1844.10|2046.60|132.58|0.00|1657.26|1789.84|-186.84| +2451489|52000|12847|82789|1407849|109|29569|1|240|303|29|49.38|58.76|27.02|0.00|783.58|1432.02|1704.04|62.68|0.00|783.58|846.26|-648.44| +2451489|52000|6206|82789|1407849|109|29569|1|294|303|75|55.23|96.65|11.59|0.00|869.25|4142.25|7248.75|26.07|0.00|869.25|895.32|-3273.00| +2451489|52000|1544|82789|1407849|109|29569|1|7|303|100|91.21|123.13|84.95|0.00|8495.00|9121.00|12313.00|764.55|0.00|8495.00|9259.55|-626.00| +2451489|52000|14054|82789|1407849|109|29569|1|5|303|80|89.77|154.40|41.68|0.00|3334.40|7181.60|12352.00|200.06|0.00|3334.40|3534.46|-3847.20| +2451489|52000|7010|82789|1407849|109|29569|1|115|303|95|68.04|77.56|58.94|503.93|5599.30|6463.80|7368.20|305.72|503.93|5095.37|5401.09|-1368.43| +2451489|52000|4232|82789|1407849|109|29569|1|44|303|61|59.26|62.81|42.08|564.71|2566.88|3614.86|3831.41|80.08|564.71|2002.17|2082.25|-1612.69| +2451489|52000|17252|82789|1407849|109|29569|1|191|303|60|78.30|86.91|6.95|0.00|417.00|4698.00|5214.60|12.51|0.00|417.00|429.51|-4281.00| +2451489|52000|11587|82789|1407849|109|29569|1|81|303|9|73.44|116.03|54.53|0.00|490.77|660.96|1044.27|39.26|0.00|490.77|530.03|-170.19| +2451967|61482|5153|38380|1285238|6849|42907|1|139|304|31|25.07|36.85|11.05|0.00|342.55|777.17|1142.35|6.85|0.00|342.55|349.40|-434.62| +2451967|61482|16611|38380|1285238|6849|42907|1|13|304|82|71.98|141.08|31.03|0.00|2544.46|5902.36|11568.56|127.22|0.00|2544.46|2671.68|-3357.90| +2451967|61482|13937|38380|1285238|6849|42907|1|35|304|33|60.32|114.60|114.60|2760.71|3781.80|1990.56|3781.80|81.68|2760.71|1021.09|1102.77|-969.47| +2451967|61482|16205|38380|1285238|6849|42907|1|216|304|1|88.94|128.96|81.24|0.00|81.24|88.94|128.96|3.24|0.00|81.24|84.48|-7.70| +2451967|61482|16135|38380|1285238|6849|42907|1|257|304|35|78.57|100.56|84.47|0.00|2956.45|2749.95|3519.60|29.56|0.00|2956.45|2986.01|206.50| +2451967|61482|1897|38380|1285238|6849|42907|1|69|304|38|12.59|22.53|0.90|0.00|34.20|478.42|856.14|1.02|0.00|34.20|35.22|-444.22| +2451967|61482|5409|38380|1285238|6849|42907|1|225|304|77|69.19|124.54|11.20|0.00|862.40|5327.63|9589.58|43.12|0.00|862.40|905.52|-4465.23| +2451967|61482|7613|38380|1285238|6849|42907|1|243|304|89|56.05|58.29|11.07|0.00|985.23|4988.45|5187.81|78.81|0.00|985.23|1064.04|-4003.22| +2451967|61482|3527|38380|1285238|6849|42907|1|183|304|8|13.61|24.49|20.08|114.05|160.64|108.88|195.92|2.32|114.05|46.59|48.91|-62.29| +2451967|61482|6993|38380|1285238|6849|42907|1|200|304|90|8.31|11.96|2.63|0.00|236.70|747.90|1076.40|16.56|0.00|236.70|253.26|-511.20| +2451967|61482|7031|38380|1285238|6849|42907|1|233|304|38|44.20|65.85|29.63|0.00|1125.94|1679.60|2502.30|101.33|0.00|1125.94|1227.27|-553.66| +2451963|36242|11313|23057|982545|2033|10274|2|133|305|77|40.61|44.26|0.44|0.00|33.88|3126.97|3408.02|2.03|0.00|33.88|35.91|-3093.09| +2451963|36242|13011|23057|982545|2033|10274|2|131|305|47|5.86|8.14|1.38|0.00|64.86|275.42|382.58|1.29|0.00|64.86|66.15|-210.56| +2451963|36242|5745|23057|982545|2033|10274|2|8|305|90|76.23|78.51|58.88|1801.72|5299.20|6860.70|7065.90|34.97|1801.72|3497.48|3532.45|-3363.22| +2451963|36242|185|23057|982545|2033|10274|2|214|305|37|97.79|128.10|105.04|0.00|3886.48|3618.23|4739.70|272.05|0.00|3886.48|4158.53|268.25| +2451963|36242|2121|23057|982545|2033|10274|2|34|305|24|39.08|66.04|21.13|0.00|507.12|937.92|1584.96|10.14|0.00|507.12|517.26|-430.80| +2451963|36242|175|23057|982545|2033|10274|2|87|305|14|31.49|35.26|33.84|0.00|473.76|440.86|493.64|37.90|0.00|473.76|511.66|32.90| +2451963|36242|15839|23057|982545|2033|10274|2|6|305|22|81.49|102.67|101.64|0.00|2236.08|1792.78|2258.74|111.80|0.00|2236.08|2347.88|443.30| +2451963|36242|16487|23057|982545|2033|10274|2|292|305|72|60.47|96.75|35.79|128.84|2576.88|4353.84|6966.00|24.48|128.84|2448.04|2472.52|-1905.80| +2451963|36242|2987|23057|982545|2033|10274|2|152|305|62|95.33|144.90|33.32|0.00|2065.84|5910.46|8983.80|41.31|0.00|2065.84|2107.15|-3844.62| +2451963|36242|14553|23057|982545|2033|10274|2|16|305|98|81.21|103.13|35.06|0.00|3435.88|7958.58|10106.74|34.35|0.00|3435.88|3470.23|-4522.70| +2451963|36242|7879|23057|982545|2033|10274|2|293|305|37|54.76|86.52|22.49|0.00|832.13|2026.12|3201.24|41.60|0.00|832.13|873.73|-1193.99| +2451963|36242|4159|23057|982545|2033|10274|2|251|305|82|5.78|7.28|1.09|0.00|89.38|473.96|596.96|0.00|0.00|89.38|89.38|-384.58| +2452444|52590|4231|26936|590507|2878|25602|4|242|306|49|56.64|80.42|74.79|0.00|3664.71|2775.36|3940.58|0.00|0.00|3664.71|3664.71|889.35| +||4959|||||4||306|||63.20|42.34|0.00|381.06|507.87|568.80||0.00|||| +2452444||7347|26936||||4||306||85.99|105.76|3.17|||6879.20|8460.80|||218.10||| +2452444|52590|15435|26936|590507|2878|25602|4|96|306|90|10.59|16.62|11.46|721.98|1031.40|953.10|1495.80|0.00|721.98|309.42|309.42|-643.68| +2452444|52590|3036|26936|590507|2878|25602|4|118|306|9|38.21|51.96|20.78|130.91|187.02|343.89|467.64|3.36|130.91|56.11|59.47|-287.78| +2452444|52590|16215|26936|590507|2878|25602|4|238|306|60|47.84|86.11|32.72|451.53|1963.20|2870.40|5166.60|136.05|451.53|1511.67|1647.72|-1358.73| +2452444|52590|3219|26936|590507|2878|25602|4|240|306|95|25.82|29.69|13.65|505.73|1296.75|2452.90|2820.55|63.28|505.73|791.02|854.30|-1661.88| +2452444|52590|11619|26936|590507|2878|25602|4|209|306|4|24.28|39.33|33.03|0.00|132.12|97.12|157.32|3.96|0.00|132.12|136.08|35.00| +2452444|52590|5985|26936|590507|2878|25602|4|233|306|28|10.18|10.89|8.82|0.00|246.96|285.04|304.92|7.40|0.00|246.96|254.36|-38.08| +2452444|52590|13521|26936|590507|2878|25602|4|79|306|96|69.09|81.52|57.06|0.00|5477.76|6632.64|7825.92|328.66|0.00|5477.76|5806.42|-1154.88| +2452444|52590|13545|26936|590507|2878|25602|4|289|306|75|46.45|60.84|19.46|0.00|1459.50|3483.75|4563.00|131.35|0.00|1459.50|1590.85|-2024.25| +2452444|52590|687|26936|590507|2878|25602|4|297|306|22|67.82|95.62|12.43|0.00|273.46|1492.04|2103.64|24.61|0.00|273.46|298.07|-1218.58| +2452444|52590|7117|26936|590507|2878|25602|4|211|306|16|10.09|11.70|5.38|0.00|86.08|161.44|187.20|3.44|0.00|86.08|89.52|-75.36| +2452444|52590|7062|26936|590507|2878|25602|4|51|306|40|37.71|47.51|29.45|0.00|1178.00|1508.40|1900.40|58.90|0.00|1178.00|1236.90|-330.40| +2452444|52590|12871|26936|590507|2878|25602|4|145|306|59|3.92|4.31|4.09|0.00|241.31|231.28|254.29|16.89|0.00|241.31|258.20|10.03| +2452444|52590|16941|26936|590507|2878|25602|4|294|306|21|79.15|151.17|80.12|0.00|1682.52|1662.15|3174.57|67.30|0.00|1682.52|1749.82|20.37| +2452311|38610|6703|24006|1119629|6805|25981|1|248|307|61|81.13|142.78|39.97|0.00|2438.17|4948.93|8709.58|48.76|0.00|2438.17|2486.93|-2510.76| +2452311|38610|10483|24006|1119629|6805|25981|1|7|307|95|91.38|142.55|15.68|0.00|1489.60|8681.10|13542.25|59.58|0.00|1489.60|1549.18|-7191.50| +2452311|38610|14893|24006|1119629|6805|25981|1|256|307|15|4.51|7.48|1.34|0.00|20.10|67.65|112.20|0.40|0.00|20.10|20.50|-47.55| +2452311|38610|1569|24006|1119629|6805|25981|1|265|307|2|26.60|35.64|3.56|0.71|7.12|53.20|71.28|0.44|0.71|6.41|6.85|-46.79| +2452311|38610|15216|24006|1119629|6805|25981|1|175|307|89|62.64|110.87|55.43|0.00|4933.27|5574.96|9867.43|295.99|0.00|4933.27|5229.26|-641.69| +2452311|38610|13071|24006|1119629|6805|25981|1|181|307|97|69.52|98.02|94.09|0.00|9126.73|6743.44|9507.94|730.13|0.00|9126.73|9856.86|2383.29| +2452311|38610|1933|24006|1119629|6805|25981|1|50|307|40|31.95|63.58|29.24|0.00|1169.60|1278.00|2543.20|93.56|0.00|1169.60|1263.16|-108.40| +2452311|38610|13015|24006|1119629|6805|25981|1|256|307|15|22.90|44.19|39.32|0.00|589.80|343.50|662.85|11.79|0.00|589.80|601.59|246.30| +2452311|38610|11721|24006|1119629|6805|25981|1|55|307|80|14.79|19.67|9.24|0.00|739.20|1183.20|1573.60|0.00|0.00|739.20|739.20|-444.00| +2452311|38610|9495|24006|1119629|6805|25981|1|129|307|68|42.51|45.06|37.39|0.00|2542.52|2890.68|3064.08|177.97|0.00|2542.52|2720.49|-348.16| +2452311|38610|11689|24006|1119629|6805|25981|1|236|307|95|37.21|68.09|10.21|0.00|969.95|3534.95|6468.55|38.79|0.00|969.95|1008.74|-2565.00| +2452200|71638|865|94379|86421|6463|29752|7|50|308|72|19.72|23.46|4.22|0.00|303.84|1419.84|1689.12|15.19|0.00|303.84|319.03|-1116.00| +2452200|71638|5845|94379|86421|6463|29752|7|145|308|20|17.00|23.29|19.33|0.00|386.60|340.00|465.80|0.00|0.00|386.60|386.60|46.60| +2452200|71638|7849|94379|86421|6463|29752|7|187|308|61|92.93|113.37|52.15|0.00|3181.15|5668.73|6915.57|159.05|0.00|3181.15|3340.20|-2487.58| +2452200|71638|7911|94379|86421|6463|29752|7|45|308|41|65.69|122.18|103.85|0.00|4257.85|2693.29|5009.38|170.31|0.00|4257.85|4428.16|1564.56| +2452200|71638|12277|94379|86421|6463|29752|7|5|308|25|16.25|24.53|12.51|168.88|312.75|406.25|613.25|10.07|168.88|143.87|153.94|-262.38| +2452200|71638|14551|94379|86421|6463|29752|7|152|308|96|45.76|84.19|77.45|3420.19|7435.20|4392.96|8082.24|361.35|3420.19|4015.01|4376.36|-377.95| +2452200|71638|16359|94379|86421|6463|29752|7|57|308|84|79.48|127.96|69.09|5165.16|5803.56|6676.32|10748.64|19.15|5165.16|638.40|657.55|-6037.92| +2452200|71638|13027|94379|86421|6463|29752|7|233|308|20|61.73|113.58|71.55|0.00|1431.00|1234.60|2271.60|71.55|0.00|1431.00|1502.55|196.40| +2452200|71638|3325|94379|86421|6463|29752|7|41|308|47|87.23|134.33|18.80|247.40|883.60|4099.81|6313.51|6.36|247.40|636.20|642.56|-3463.61| +2452200|71638|5637|94379|86421|6463|29752|7|54|308|74|16.89|32.25|22.25|0.00|1646.50|1249.86|2386.50|49.39|0.00|1646.50|1695.89|396.64| +2451312|65775|5764|34893|1371032|6371|2510|4|160|309|19|71.78|109.10|17.45|0.00|331.55|1363.82|2072.90|9.94|0.00|331.55|341.49|-1032.27| +2451312|65775|5563|34893|1371032|6371|2510|4|290|309|52|78.53|124.86|96.14|0.00|4999.28|4083.56|6492.72|199.97|0.00|4999.28|5199.25|915.72| +2451312|65775|12907|34893|1371032|6371|2510|4|103|309|10|17.97|25.69|11.04|0.00|110.40|179.70|256.90|6.62|0.00|110.40|117.02|-69.30| +|65775|12523|||6371|||290|309||||1.10|0.00|86.90|6178.59|||0.00||91.24|-6091.69| +2451312|65775|4420|34893|1371032|6371|2510|4|43|309|77|37.18|71.75|21.52|0.00|1657.04|2862.86|5524.75|49.71|0.00|1657.04|1706.75|-1205.82| +2451312|65775|17950|34893|1371032|6371|2510|4|19|309|15|91.37|92.28|48.90|0.00|733.50|1370.55|1384.20|44.01|0.00|733.50|777.51|-637.05| +2451312|65775|5536|34893|1371032|6371|2510|4|299|309|3|46.27|59.68|31.63|0.00|94.89|138.81|179.04|7.59|0.00|94.89|102.48|-43.92| +2451312|65775|11581|34893|1371032|6371|2510|4|57|309|56|93.48|147.69|56.12|0.00|3142.72|5234.88|8270.64|219.99|0.00|3142.72|3362.71|-2092.16| +2451312|65775|3904|34893|1371032|6371|2510|4|22|309|50|11.59|20.51|14.56|0.00|728.00|579.50|1025.50|50.96|0.00|728.00|778.96|148.50| +2451312|65775|10672|34893|1371032|6371|2510|4|277|309|35|27.66|30.42|25.85|0.00|904.75|968.10|1064.70|54.28|0.00|904.75|959.03|-63.35| +2451312|65775|15770|34893|1371032|6371|2510|4|295|309|3|15.38|17.84|3.21|0.00|9.63|46.14|53.52|0.28|0.00|9.63|9.91|-36.51| +2451312|65775|12116|34893|1371032|6371|2510|4|13|309|21|83.87|166.90|20.02|0.00|420.42|1761.27|3504.90|4.20|0.00|420.42|424.62|-1340.85| +2451312|65775|15430|34893|1371032|6371|2510|4|223|309|80|3.31|4.40|0.44|0.00|35.20|264.80|352.00|0.00|0.00|35.20|35.20|-229.60| +2451312|65775|14191|34893|1371032|6371|2510|4|44|309|48|80.96|93.10|27.93|0.00|1340.64|3886.08|4468.80|26.81|0.00|1340.64|1367.45|-2545.44| +2451312|65775|14302|34893|1371032|6371|2510|4|212|309|27|49.89|70.84|6.37|0.00|171.99|1347.03|1912.68|8.59|0.00|171.99|180.58|-1175.04| +2452598|63709|12942|43765|1013598|1031|36864|8|18|310|26|87.30|140.55|64.65|0.00|1680.90|2269.80|3654.30|84.04|0.00|1680.90|1764.94|-588.90| +2452598|63709|12021|43765|1013598|1031|36864|8|298|310|68|80.05|83.25|52.44|0.00|3565.92|5443.40|5661.00|178.29|0.00|3565.92|3744.21|-1877.48| +2452598|63709|9055|43765|1013598|1031|36864|8|168|310|24|91.53|161.09|115.98|0.00|2783.52|2196.72|3866.16|167.01|0.00|2783.52|2950.53|586.80| +2452598|63709|7272|43765|1013598|1031|36864|8|200|310|55|96.18|100.02|34.00|0.00|1870.00|5289.90|5501.10|130.90|0.00|1870.00|2000.90|-3419.90| +||11377|43765|1013598||36864|8||310|60||31.38||0.00|1468.20|1088.40||117.45|0.00|||| +2452598|63709|13381|43765|1013598|1031|36864|8|66|310|44|8.97|12.01|10.68|0.00|469.92|394.68|528.44|4.69|0.00|469.92|474.61|75.24| +2452598|63709|12493|43765|1013598|1031|36864|8|134|310|86|80.71|152.54|117.45|0.00|10100.70|6941.06|13118.44|101.00|0.00|10100.70|10201.70|3159.64| +2452598|63709|9750|43765|1013598|1031|36864|8|133|310|72|26.81|48.52|14.07|0.00|1013.04|1930.32|3493.44|81.04|0.00|1013.04|1094.08|-917.28| +2452598|63709|9409|43765|1013598|1031|36864|8|10|310|14|32.49|48.73|45.80|0.00|641.20|454.86|682.22|12.82|0.00|641.20|654.02|186.34| +2452598|63709|6606|43765|1013598|1031|36864|8|262|310|18|59.13|107.02|8.56|0.00|154.08|1064.34|1926.36|12.32|0.00|154.08|166.40|-910.26| +2452598|63709|7128|43765|1013598|1031|36864|8|56|310|49|46.48|73.90|20.69|0.00|1013.81|2277.52|3621.10|10.13|0.00|1013.81|1023.94|-1263.71| +2452598|63709|17779|43765|1013598|1031|36864|8|121|310|84|7.53|8.05|3.70|0.00|310.80|632.52|676.20|3.10|0.00|310.80|313.90|-321.72| +2451757|43124|1520|54711|1683697|6283|30469|4|293|311|27|33.06|40.99|40.17|0.00|1084.59|892.62|1106.73|65.07|0.00|1084.59|1149.66|191.97| +2451757|43124|12374|54711|1683697|6283|30469|4|163|311|95|80.49|80.49|25.75|0.00|2446.25|7646.55|7646.55|122.31|0.00|2446.25|2568.56|-5200.30| +2451757|43124|5234|54711|1683697|6283|30469|4|247|311|95|93.67|181.71|165.35|11781.18|15708.25|8898.65|17262.45|157.08|11781.18|3927.07|4084.15|-4971.58| +2451757|43124|15362|54711|1683697|6283|30469|4|281|311|96|52.21|99.19|36.70|0.00|3523.20|5012.16|9522.24|211.39|0.00|3523.20|3734.59|-1488.96| +|43124|16817|54711|1683697|6283||4||311||81.41|155.49|||||5442.15|84.89||||| +2451757|43124|6038|54711|1683697|6283|30469|4|278|311|98|62.03|98.62|62.13|0.00|6088.74|6078.94|9664.76|426.21|0.00|6088.74|6514.95|9.80| +2451757|43124|15745|54711|1683697|6283|30469|4|263|311|8|70.92|96.45|17.36|0.00|138.88|567.36|771.60|12.49|0.00|138.88|151.37|-428.48| +2451757|43124|1754|54711|1683697|6283|30469|4|113|311|91|12.55|14.30|13.29|72.56|1209.39|1142.05|1301.30|45.47|72.56|1136.83|1182.30|-5.22| +2451757|43124|2555|54711|1683697|6283|30469|4|4|311|51|85.96|133.23|91.92|2437.71|4687.92|4383.96|6794.73|202.51|2437.71|2250.21|2452.72|-2133.75| +2451757|43124|13067|54711|1683697|6283|30469|4|66|311|27|13.93|13.93|12.67|0.00|342.09|376.11|376.11|3.42|0.00|342.09|345.51|-34.02| +2451757|43124|7187|54711|1683697|6283|30469|4|231|311|36|41.93|68.34|53.30|0.00|1918.80|1509.48|2460.24|134.31|0.00|1918.80|2053.11|409.32| +2451757|43124|16399|54711|1683697|6283|30469|4|162|311|96|34.80|40.02|18.00|0.00|1728.00|3340.80|3841.92|138.24|0.00|1728.00|1866.24|-1612.80| +2451106|48706|7376|70317|1608636|5355|1274|1|123|312|2|35.74|45.74|6.86|0.00|13.72|71.48|91.48|0.96|0.00|13.72|14.68|-57.76| +2451106|48706|13279|70317|1608636|5355|1274|1|53|312|74|73.16|82.67|81.01|1258.89|5994.74|5413.84|6117.58|284.15|1258.89|4735.85|5020.00|-677.99| +2451106|48706|3944|70317|1608636|5355|1274|1|99|312|18|92.01|107.65|20.45|191.41|368.10|1656.18|1937.70|3.53|191.41|176.69|180.22|-1479.49| +2451106|48706|14180|70317|1608636|5355|1274|1|63|312|33|71.54|108.02|81.01|0.00|2673.33|2360.82|3564.66|53.46|0.00|2673.33|2726.79|312.51| +2451106|48706|1414|70317|1608636|5355|1274|1|33|312|4|93.61|137.60|13.76|37.97|55.04|374.44|550.40|0.68|37.97|17.07|17.75|-357.37| +2451106|48706|14806|70317|1608636|5355|1274|1|164|312|20|22.01|28.61|9.15|0.00|183.00|440.20|572.20|10.98|0.00|183.00|193.98|-257.20| +2451106|48706|11197|70317|1608636|5355|1274|1|71|312|18|97.59|148.33|115.69|0.00|2082.42|1756.62|2669.94|62.47|0.00|2082.42|2144.89|325.80| +2451106|48706|5347|70317|1608636|5355|1274|1|150|312|27|5.44|8.92|8.11|0.00|218.97|146.88|240.84|15.32|0.00|218.97|234.29|72.09| +2451106|48706|11353|70317|1608636|5355|1274|1|237|312|56|44.89|67.33|48.47|0.00|2714.32|2513.84|3770.48|81.42|0.00|2714.32|2795.74|200.48| +2452128|33683|5895|86226|633839|5005|34218|8|258|313|94|9.31|10.79|9.38|0.00|881.72|875.14|1014.26|52.90|0.00|881.72|934.62|6.58| +2452128|33683|2741|86226|633839|5005|34218|8|274|313|39|81.11|111.93|31.34|0.00|1222.26|3163.29|4365.27|48.89|0.00|1222.26|1271.15|-1941.03| +2452128|33683|12191|86226|633839|5005|34218|8|223|313|23|38.63|63.35|60.18|0.00|1384.14|888.49|1457.05|0.00|0.00|1384.14|1384.14|495.65| +2452128|33683|8253|86226|633839|5005|34218||93|313|39|||33.28|973.44||3278.34||22.71|973.44|324.48|347.19|| +2452128|33683|6667|86226|633839|5005|34218|8|184|313|24|66.49|130.98|78.58|0.00|1885.92|1595.76|3143.52|75.43|0.00|1885.92|1961.35|290.16| +2452128|33683|5363|86226|633839|5005|34218|8|171|313|81|4.10|6.02|5.65|0.00|457.65|332.10|487.62|41.18|0.00|457.65|498.83|125.55| +2452128|33683|13359|86226|633839|5005|34218|8|279|313|40|8.45|14.78|5.17|0.00|206.80|338.00|591.20|18.61|0.00|206.80|225.41|-131.20| +2452128|33683|9081|86226|633839|5005|34218|8|232|313|61|10.10|10.30|10.19|0.00|621.59|616.10|628.30|0.00|0.00|621.59|621.59|5.49| +2452128|33683|951|86226|633839|5005|34218|8|218|313|31|8.83|16.24|4.22|92.88|130.82|273.73|503.44|1.89|92.88|37.94|39.83|-235.79| +2451877|56686|1295|75578|923380|782|21784|7|165|314|94|84.98|115.57|1.15|0.00|108.10|7988.12|10863.58|8.64|0.00|108.10|116.74|-7880.02| +2451877|56686|8096|75578|923380|782|21784|7|119|314|95|74.37|90.73|61.69|0.00|5860.55|7065.15|8619.35|293.02|0.00|5860.55|6153.57|-1204.60| +2451877|56686|13394|75578|923380|782|21784|7|180|314|54|69.80|133.31|41.32|1071.01|2231.28|3769.20|7198.74|34.80|1071.01|1160.27|1195.07|-2608.93| +|56686|17455|||||7|181|314|79|||||1248.99||2840.05||||1273.96|-560.11| +2451877|56686|2353|75578|923380|782|21784|7|109|314|25|5.92|11.48|1.60|0.00|40.00|148.00|287.00|0.40|0.00|40.00|40.40|-108.00| +|56686|3811||923380|||7|194|314|||60.04|31.82|||3594.80||140.00||||-794.64| +2451877|56686|4538|75578|923380|782|21784|7|265|314|63|96.67|184.63|160.62|0.00|10119.06|6090.21|11631.69|505.95|0.00|10119.06|10625.01|4028.85| +2451877|56686|5375|75578|923380|782|21784|7|291|314|86|97.34|185.91|94.81|0.00|8153.66|8371.24|15988.26|570.75|0.00|8153.66|8724.41|-217.58| +2451877|56686|10883|75578|923380|782|21784|7|188|314|2|39.05|71.85|10.05|14.07|20.10|78.10|143.70|0.24|14.07|6.03|6.27|-72.07| +2451877|56686|395|75578|923380|782|21784|7|119|314|21|51.59|55.20|33.12|0.00|695.52|1083.39|1159.20|34.77|0.00|695.52|730.29|-387.87| +2451877|56686|16067|75578|923380|782|21784|7|181|314|53|37.86|73.44|4.40|86.28|233.20|2006.58|3892.32|4.40|86.28|146.92|151.32|-1859.66| +2451877|56686|2837|75578|923380|782|21784|7|66|314|71|14.65|25.34|16.21|34.52|1150.91|1040.15|1799.14|89.31|34.52|1116.39|1205.70|76.24| +2451877|56686|15572|75578|923380|782|21784|7|212|314|86|88.86|103.07|29.89|0.00|2570.54|7641.96|8864.02|0.00|0.00|2570.54|2570.54|-5071.42| +2451282|50539|6586|85958|1569947|6736|29125|2|99|315|84|72.91|119.57|93.26|0.00|7833.84|6124.44|10043.88|705.04|0.00|7833.84|8538.88|1709.40| +2451282|50539|3187|85958|1569947|6736|29125|2|247|315|61|38.82|60.94|33.51|0.00|2044.11|2368.02|3717.34|143.08|0.00|2044.11|2187.19|-323.91| +2451282|50539|3008|85958|1569947|6736|29125|2|269|315|26|19.56|28.75|6.32|129.81|164.32|508.56|747.50|3.10|129.81|34.51|37.61|-474.05| +2451282|50539|13195|85958|1569947|6736|29125|2|297|315|68|22.74|26.37|17.66|0.00|1200.88|1546.32|1793.16|48.03|0.00|1200.88|1248.91|-345.44| +2451282|50539|11611|85958|1569947|6736|29125|2|208|315|3|62.79|75.34|73.83|0.00|221.49|188.37|226.02|13.28|0.00|221.49|234.77|33.12| +2451282|50539|8593|85958|1569947|6736|29125|2|8|315|29|83.37|111.71|90.48|0.00|2623.92|2417.73|3239.59|236.15|0.00|2623.92|2860.07|206.19| +2451282|50539|1216|85958|1569947|6736|29125|2|14|315|4|28.95|30.39|23.09|0.00|92.36|115.80|121.56|4.61|0.00|92.36|96.97|-23.44| +2451282|50539|11668|85958|1569947|6736|29125|2|122|315|13|62.54|96.31|46.22|360.51|600.86|813.02|1252.03|16.82|360.51|240.35|257.17|-572.67| +2451282|50539|8242|85958|1569947|6736|29125|2|147|315|51|68.21|131.64|22.37|0.00|1140.87|3478.71|6713.64|22.81|0.00|1140.87|1163.68|-2337.84| +2451282|50539|613|85958|1569947|6736|29125|2|76|315|96|20.60|34.40|17.54|0.00|1683.84|1977.60|3302.40|84.19|0.00|1683.84|1768.03|-293.76| +2451282||8338|85958|1569947||29125|2|262|315|48|71.66|105.34|||4651.68||5056.32|||4651.68|4930.78|| +2451282|50539|11522|85958|1569947|6736|29125|2|134|315|54|6.35|7.87|2.51|0.00|135.54|342.90|424.98|12.19|0.00|135.54|147.73|-207.36| +2451282|50539|14899|85958|1569947|6736|29125|2|190|315|24|35.16|54.49|0.54|0.00|12.96|843.84|1307.76|0.90|0.00|12.96|13.86|-830.88| +2451282|50539|12484|85958|1569947|6736|29125|2|35|315|80|99.75|142.64|92.71|6155.94|7416.80|7980.00|11411.20|75.65|6155.94|1260.86|1336.51|-6719.14| +2451282|50539|5461|85958|1569947|6736|29125|2|203|315|23|35.32|67.81|63.06|0.00|1450.38|812.36|1559.63|14.50|0.00|1450.38|1464.88|638.02| +2451282|50539|734|85958|1569947|6736|29125|2|153|315|37|68.45|73.92|18.48|0.00|683.76|2532.65|2735.04|47.86|0.00|683.76|731.62|-1848.89| +2452141|36302|10649|79361|819087|4766|20092|8|71|316|12|40.37|48.44|2.90|0.00|34.80|484.44|581.28|2.08|0.00|34.80|36.88|-449.64| +2452141|36302|4917|79361|819087|4766|20092|8|78|316|35|50.52|80.32|30.52|0.00|1068.20|1768.20|2811.20|53.41|0.00|1068.20|1121.61|-700.00| +2452141|36302|17489|79361|819087|4766|20092|8|270|316|33|38.42|54.55|53.45|0.00|1763.85|1267.86|1800.15|88.19|0.00|1763.85|1852.04|495.99| +2452141|36302|13117|79361|819087|4766|20092|8|45|316|11|80.81|145.45|37.81|0.00|415.91|888.91|1599.95|29.11|0.00|415.91|445.02|-473.00| +2452141|36302|8035|79361|819087|4766|20092|8|118|316|16|11.23|12.91|1.67|2.40|26.72|179.68|206.56|0.00|2.40|24.32|24.32|-155.36| +2452141|36302|17327|79361|819087|4766|20092|8|221|316|88|64.55|100.05|14.00|0.00|1232.00|5680.40|8804.40|12.32|0.00|1232.00|1244.32|-4448.40| +2452141|36302|577|79361|819087|4766|20092|8|13|316|69|94.66|124.95|89.96|0.00|6207.24|6531.54|8621.55|434.50|0.00|6207.24|6641.74|-324.30| +2452141|36302|14319|79361|819087|4766|20092|8|191|316|42|79.93|129.48|98.40|2810.30|4132.80|3357.06|5438.16|26.45|2810.30|1322.50|1348.95|-2034.56| +2452141|36302|12781|79361|819087|4766|20092|8|207|316|62|84.70|125.35|48.88|0.00|3030.56|5251.40|7771.70|0.00|0.00|3030.56|3030.56|-2220.84| +2452141|36302|16073|79361|819087|4766|20092|8|125|316|84|32.90|62.83|11.30|170.85|949.20|2763.60|5277.72|38.91|170.85|778.35|817.26|-1985.25| +2452141|36302|15903|79361|819087|4766|20092|8|240|316|3|36.04|46.49|23.24|0.00|69.72|108.12|139.47|2.78|0.00|69.72|72.50|-38.40| +2452141|36302|17865|79361|819087|4766|20092|8|214|316|8|88.05|169.05|106.50|0.00|852.00|704.40|1352.40|42.60|0.00|852.00|894.60|147.60| +2451634|71009|2669|11168|592011|4854|25297|4|135|317|70|84.65|105.81|33.85|0.00|2369.50|5925.50|7406.70|23.69|0.00|2369.50|2393.19|-3556.00| +2451634|71009|10502|11168|592011|4854|25297|4|128|317|69|55.19|109.27|13.11|0.00|904.59|3808.11|7539.63|45.22|0.00|904.59|949.81|-2903.52| +2451634|71009|5360|11168|592011|4854|25297|4|268|317|82|49.50|80.68|73.41|0.00|6019.62|4059.00|6615.76|300.98|0.00|6019.62|6320.60|1960.62| +2451634|71009|12638|11168|592011|4854|25297|4|219|317|9|38.83|67.56|2.70|0.00|24.30|349.47|608.04|1.70|0.00|24.30|26.00|-325.17| +2451634|71009|2588|11168|592011|4854|25297|4|184|317|25|73.64|145.80|48.11|0.00|1202.75|1841.00|3645.00|36.08|0.00|1202.75|1238.83|-638.25| +2451634|71009|7460|11168|592011|4854|25297|4|255|317|73|32.34|45.92|4.13|217.07|301.49|2360.82|3352.16|3.37|217.07|84.42|87.79|-2276.40| +2451634|71009|10490|11168|592011|4854|25297|4|242|317|69|22.82|31.49|1.25|0.00|86.25|1574.58|2172.81|3.45|0.00|86.25|89.70|-1488.33| +2451634|71009|8213|11168|592011|4854|25297|4|125|317|28|38.58|69.05|62.14|0.00|1739.92|1080.24|1933.40|0.00|0.00|1739.92|1739.92|659.68| +2451634|71009|2809|11168|592011|4854|25297|4|33|317|2|29.63|56.00|22.96|0.00|45.92|59.26|112.00|4.13|0.00|45.92|50.05|-13.34| +2451634|71009|11570|11168|592011|4854|25297|4|213|317|73|14.19|24.12|6.27|0.00|457.71|1035.87|1760.76|13.73|0.00|457.71|471.44|-578.16| +2451634|71009|16832|11168|592011|4854|25297|4|7|317|47|13.88|15.96|5.90|110.92|277.30|652.36|750.12|11.64|110.92|166.38|178.02|-485.98| +2451634|71009|17414|11168|592011|4854|25297|4|294|317|20|57.02|101.49|23.34|0.00|466.80|1140.40|2029.80|28.00|0.00|466.80|494.80|-673.60| +2451634|71009|2947|11168|592011|4854|25297|4|176|317|32|48.09|49.05|45.61|0.00|1459.52|1538.88|1569.60|87.57|0.00|1459.52|1547.09|-79.36| +2451634|71009|15836|11168|592011|4854|25297|4|34|317|28|48.98|75.91|3.03|0.00|84.84|1371.44|2125.48|6.78|0.00|84.84|91.62|-1286.60| +2451634|71009|4579|11168|592011|4854|25297|4|155|317|40|90.79|155.25|145.93|3502.32|5837.20|3631.60|6210.00|186.79|3502.32|2334.88|2521.67|-1296.72| +2451972|68105|15385|42477|1089941|3005|4422|7|39|318|27|92.34|144.97|121.77|0.00|3287.79|2493.18|3914.19|295.90|0.00|3287.79|3583.69|794.61| +2451972|68105|235|42477|1089941|3005|4422|7|181|318|51|95.34|151.59|150.07|0.00|7653.57|4862.34|7731.09|535.74|0.00|7653.57|8189.31|2791.23| +2451972|68105|397||1089941|||||318|||12.29|||474.72|565.34|565.34|18.98||474.72|493.70|-90.62| +2451972|68105|7099|42477|1089941|3005|4422|7|259|318|55|58.46|78.92|48.14|0.00|2647.70|3215.30|4340.60|158.86|0.00|2647.70|2806.56|-567.60| +2451972|68105|8945|42477|1089941||4422||35|318||60.46||76.17|||1753.34||110.44||2208.93||455.59| +2451972||10569|42477||3005||7||318|||57.66|52.47|4247.97|||||4247.97|||-3130.37| +2451972|68105|17529|42477|1089941|3005|4422|7|248|318|19|64.42|109.51|98.55|0.00|1872.45|1223.98|2080.69|74.89|0.00|1872.45|1947.34|648.47| +2451972|68105|4847|42477|1089941|3005|4422|7|273|318|24|59.32|63.47|17.77|98.09|426.48|1423.68|1523.28|16.41|98.09|328.39|344.80|-1095.29| +2451972|68105|491|42477|1089941|3005|4422|7|254|318|31|96.00|157.44|89.74|0.00|2781.94|2976.00|4880.64|222.55|0.00|2781.94|3004.49|-194.06| +2451972|68105|8015|42477|1089941|3005|4422|7|25|318|54|2.15|3.11|2.70|34.99|145.80|116.10|167.94|3.32|34.99|110.81|114.13|-5.29| +2451972|68105|13953|42477|1089941|3005|4422|7|21|318|52|72.97|123.31|118.37|2154.33|6155.24|3794.44|6412.12|40.00|2154.33|4000.91|4040.91|206.47| +2451972|68105|6619|42477|1089941|3005|4422|7|8|318|23|76.31|89.28|5.35|0.00|123.05|1755.13|2053.44|11.07|0.00|123.05|134.12|-1632.08| +2451972|68105|10779|42477|1089941|3005|4422|7|97|318|100|85.15|142.20|105.22|0.00|10522.00|8515.00|14220.00|420.88|0.00|10522.00|10942.88|2007.00| +2451972|68105|16017|42477|1089941|3005|4422|7|21|318|30|86.43|150.38|79.70|0.00|2391.00|2592.90|4511.40|167.37|0.00|2391.00|2558.37|-201.90| +2451972|68105|12655|42477|1089941|3005|4422|7|259|318|81|33.65|38.36|35.67|0.00|2889.27|2725.65|3107.16|260.03|0.00|2889.27|3149.30|163.62| +2451972|68105|6733|42477|1089941|3005|4422|7|64|318|37|23.66|26.97|0.26|0.00|9.62|875.42|997.89|0.00|0.00|9.62|9.62|-865.80| +2450884|44899|3319|67173|1411760|1500|1687|2|24|319|57|89.21|98.13|76.54|916.18|4362.78|5084.97|5593.41|34.46|916.18|3446.60|3481.06|-1638.37| +2450884|44899|3044|67173|1411760|1500|1687|2|228|319|98|42.25|80.69|75.84|0.00|7432.32|4140.50|7907.62|222.96|0.00|7432.32|7655.28|3291.82| +2450884|44899|14542|67173|1411760|1500|1687|2|233|319|15|57.92|115.84|47.49|0.00|712.35|868.80|1737.60|56.98|0.00|712.35|769.33|-156.45| +2450884|44899|4390|67173|1411760|1500|1687|2|77|319|48|45.50|47.77|27.22|0.00|1306.56|2184.00|2292.96|91.45|0.00|1306.56|1398.01|-877.44| +2450884|44899|14876|67173|1411760|1500|1687|2|40|319|26|4.71|5.13|4.36|12.46|113.36|122.46|133.38|0.00|12.46|100.90|100.90|-21.56| +2450884|44899|5018|67173|1411760|1500|1687|2|228|319|43|83.68|142.25|112.37|0.00|4831.91|3598.24|6116.75|48.31|0.00|4831.91|4880.22|1233.67| +2450884|44899|2432|67173|1411760|1500|1687|2|162|319|99|85.26|98.90|85.05|0.00|8419.95|8440.74|9791.10|168.39|0.00|8419.95|8588.34|-20.79| +2450884|44899|7285|67173|1411760|1500|1687|2|173|319|22|61.76|113.63|104.53|0.00|2299.66|1358.72|2499.86|45.99|0.00|2299.66|2345.65|940.94| +2450884|44899|196|67173|1411760|1500|1687|2|29|319|22|9.69|11.72|1.99|0.00|43.78|213.18|257.84|1.75|0.00|43.78|45.53|-169.40| +2450884|44899|9544|67173|1411760|1500|1687|2|126|319|93|57.54|95.51|82.13|0.00|7638.09|5351.22|8882.43|381.90|0.00|7638.09|8019.99|2286.87| +2452640|64580|12495|79275|1428991|660|41529|8|87|320|27|33.83|49.39|43.95|0.00|1186.65|913.41|1333.53|35.59|0.00|1186.65|1222.24|273.24| +2452640|64580|10620|||660|41529|||320||52.29|87.84||0.00|||6851.52|10.95|0.00|273.78||| +2452640|64580|12061|79275|1428991|660|41529|8|134|320|34|33.09|52.61|37.87|0.00|1287.58|1125.06|1788.74|90.13|0.00|1287.58|1377.71|162.52| +2452640|64580|13566|79275|1428991|660|41529|8|103|320|9|53.44|56.11|8.41|0.00|75.69|480.96|504.99|0.00|0.00|75.69|75.69|-405.27| +2452640|64580|14943|79275|1428991|660|41529|8|259|320|54|59.03|74.37|63.21|102.40|3413.34|3187.62|4015.98|99.32|102.40|3310.94|3410.26|123.32| +2452640|64580|15366|79275|1428991|660|41529|8|195|320|29|44.10|45.42|35.42|0.00|1027.18|1278.90|1317.18|20.54|0.00|1027.18|1047.72|-251.72| +2452640|64580|13081|79275|1428991|660|41529|8|105|320|23|81.19|100.67|6.04|38.89|138.92|1867.37|2315.41|7.00|38.89|100.03|107.03|-1767.34| +2452640|64580|8316|79275|1428991|660|41529|8|54|320|41|15.93|16.72|11.03|0.00|452.23|653.13|685.52|36.17|0.00|452.23|488.40|-200.90| +2452640|64580|17535|79275|1428991|660|41529|8|126|320|14|2.64|3.59|1.72|10.11|24.08|36.96|50.26|0.27|10.11|13.97|14.24|-22.99| +2452640|64580|11251||1428991||41529|||320||77.74|||0.00|||1110.06||0.00|122.08||-966.28| +2452640|64580|10551|79275|1428991|660|41529|8|87|320|67|82.21|122.49|69.81|0.00|4677.27|5508.07|8206.83|0.00|0.00|4677.27|4677.27|-830.80| +2452640|64580|5100|79275|1428991|660|41529|8|125|320|12|53.00|91.69|59.59|250.27|715.08|636.00|1100.28|32.53|250.27|464.81|497.34|-171.19| +2452640|64580|10095|79275|1428991|660|41529|8|45|320|89|33.73|50.93|44.81|0.00|3988.09|3001.97|4532.77|159.52|0.00|3988.09|4147.61|986.12| +2451675|40774|8255|89370|670114|3658|32049|1|151|321|9|28.83|48.14|2.88|25.40|25.92|259.47|433.26|0.03|25.40|0.52|0.55|-258.95| +2451675|40774|16685|89370|670114|3658|32049|1|237|321|16|62.79|68.44|13.00|0.00|208.00|1004.64|1095.04|16.64|0.00|208.00|224.64|-796.64| +2451675|40774|16937|89370|670114|3658|32049|1|209|321|26|9.86|12.12|3.39|84.61|88.14|256.36|315.12|0.00|84.61|3.53|3.53|-252.83| +2451675|40774|17954|89370|670114|3658|32049|1|253|321|98|42.77|72.70|12.35|0.00|1210.30|4191.46|7124.60|96.82|0.00|1210.30|1307.12|-2981.16| +2451675|40774|15883|89370|670114|3658|32049|1|296|321|63|85.10|86.80|68.57|0.00|4319.91|5361.30|5468.40|43.19|0.00|4319.91|4363.10|-1041.39| +2451675|40774|383|89370|670114|3658|32049|1|260|321|59|15.72|29.23|9.64|0.00|568.76|927.48|1724.57|0.00|0.00|568.76|568.76|-358.72| +2451675|40774|3685|89370|670114|3658|32049|1|265|321|88|8.49|12.22|3.66|302.75|322.08|747.12|1075.36|0.96|302.75|19.33|20.29|-727.79| +2451675|40774|5282|89370|670114|3658|32049|1|155|321|13|12.31|15.51|9.61|0.00|124.93|160.03|201.63|4.99|0.00|124.93|129.92|-35.10| +2451675|40774|5435|89370|670114|3658|32049|1|269|321|29|90.44|179.07|69.83|0.00|2025.07|2622.76|5193.03|81.00|0.00|2025.07|2106.07|-597.69| +2451675|40774|10367|89370|670114|3658|32049|1|76|321|59|18.04|24.89|8.96|216.74|528.64|1064.36|1468.51|28.07|216.74|311.90|339.97|-752.46| +2451675|40774|13541|89370|670114|3658|32049|1|12|321|69|43.33|62.82|55.90|0.00|3857.10|2989.77|4334.58|38.57|0.00|3857.10|3895.67|867.33| +2451675|40774|10586|89370|670114|3658|32049|1|221|321|34|39.41|40.98|3.68|0.00|125.12|1339.94|1393.32|8.75|0.00|125.12|133.87|-1214.82| +2451675|40774|2906|89370|670114|3658|32049|1|111|321|17|32.21|61.52|38.14|0.00|648.38|547.57|1045.84|0.00|0.00|648.38|648.38|100.81| +2451675|40774|11576|89370|670114|3658|32049|1|214|321|23|75.11|82.62|26.43|0.00|607.89|1727.53|1900.26|24.31|0.00|607.89|632.20|-1119.64| +2451937|52863|14103|98601|1549667|2780|25893|4|161|322|80|74.16|106.79|59.80|2535.52|4784.00|5932.80|8543.20|134.90|2535.52|2248.48|2383.38|-3684.32| +2451937|52863|7627|98601|1549667|2780|25893|4|110|322|18|89.77|141.83|15.60|0.00|280.80|1615.86|2552.94|11.23|0.00|280.80|292.03|-1335.06| +2451937|52863|45|98601|1549667|2780|25893|4|62|322|75|54.81|64.12|53.86|0.00|4039.50|4110.75|4809.00|40.39|0.00|4039.50|4079.89|-71.25| +2451937|52863|14819|98601|1549667|2780|25893|4|232|322|85|55.40|90.85|13.62|0.00|1157.70|4709.00|7722.25|69.46|0.00|1157.70|1227.16|-3551.30| +2451937|52863|2127|98601|1549667|2780|25893|4|210|322|4|30.23|45.04|34.23|0.00|136.92|120.92|180.16|5.47|0.00|136.92|142.39|16.00| +2451937|52863|6147|98601|1549667|2780|25893|4|126|322|44|33.92|47.48|20.89|73.53|919.16|1492.48|2089.12|67.65|73.53|845.63|913.28|-646.85| +2451937|52863|9141|98601|1549667|2780|25893|4|167|322|79|14.71|19.85|18.65|530.40|1473.35|1162.09|1568.15|0.00|530.40|942.95|942.95|-219.14| +2451937|52863|2797|98601|1549667|2780|25893|4|132|322|81|1.88|2.36|1.20|0.00|97.20|152.28|191.16|2.91|0.00|97.20|100.11|-55.08| +2451937|52863|13703|98601|1549667|2780|25893|4|123|322|76|48.90|50.36|10.57|441.82|803.32|3716.40|3827.36|32.53|441.82|361.50|394.03|-3354.90| +2451937|52863|10547|98601|1549667|2780|25893|4|274|322|65|59.76|103.98|96.70|0.00|6285.50|3884.40|6758.70|125.71|0.00|6285.50|6411.21|2401.10| +2451937|52863|3193|98601|1549667|2780|25893|4|282|322|70|26.88|34.40|27.52|0.00|1926.40|1881.60|2408.00|115.58|0.00|1926.40|2041.98|44.80| +||16011|98601|1549667|2780||||322|100|||76.15|0.00|7615.00|||380.75|0.00|7615.00||2981.00| +||2594|28196||639||2||323|61|84.91||49.62|||||211.87||||-2152.69| +2451702|43016|2783|28196|239394|639|20237|2|90|323|85|57.77|75.10|73.59|0.00|6255.15|4910.45|6383.50|0.00|0.00|6255.15|6255.15|1344.70| +2451702|43016|10165|28196|239394|639|20237|2|287|323|43|48.08|78.85|25.23|0.00|1084.89|2067.44|3390.55|75.94|0.00|1084.89|1160.83|-982.55| +2451702|43016|14773|28196|239394|639|20237|2|118|323|99|19.72|29.38|23.79|0.00|2355.21|1952.28|2908.62|23.55|0.00|2355.21|2378.76|402.93| +2451702|43016|8528||239394||20237|2||323||97.21|142.89|2.85|0.00|28.50||||0.00|28.50||-943.60| +2451702|43016|12164|28196|||||28|323|5|16.97||18.21||91.05|||||91.05|93.78|6.20| +2451702|43016|12007|28196|239394|639|20237|2|184|323|84|21.16|26.23|13.11|0.00|1101.24|1777.44|2203.32|55.06|0.00|1101.24|1156.30|-676.20| +2451702|43016|13943|28196|239394|639|20237|2|184|323|5|54.74|82.11|23.81|0.00|119.05|273.70|410.55|2.38|0.00|119.05|121.43|-154.65| +2451702|43016|9019|28196|239394|639|20237|2|209|323|92|10.61|11.98|11.98|0.00|1102.16|976.12|1102.16|77.15|0.00|1102.16|1179.31|126.04| +2451702||3512|28196|239394|639|||1|323|60||106.08|21.21|0.00|1272.60|||0.00|0.00|1272.60||| +2451702|43016|9445|28196|239394|639|20237|2|172|323|15|36.98|41.04|1.64|0.00|24.60|554.70|615.60|0.73|0.00|24.60|25.33|-530.10| +2452196|62069|2081|83934|1883222|5357|31092|7|253|324|38|73.08|140.31|56.12|0.00|2132.56|2777.04|5331.78|170.60|0.00|2132.56|2303.16|-644.48| +2452196|62069|1753|83934|1883222|5357|31092|7|295|324|65|30.64|49.33|42.91|0.00|2789.15|1991.60|3206.45|223.13|0.00|2789.15|3012.28|797.55| +2452196|62069|1323|||5357|31092|||324|4||37.06|16.67|0.00|||148.24||0.00|66.68||-64.52| +2452196|62069|16471|83934|1883222|5357|31092|7|21|324|42|52.36|95.81|13.41|0.00|563.22|2199.12|4024.02|45.05|0.00|563.22|608.27|-1635.90| +2452196|62069|1455|83934|1883222|5357|31092|7|177|324|36|42.54|65.08|29.28|411.09|1054.08|1531.44|2342.88|57.86|411.09|642.99|700.85|-888.45| +2452196|62069|8309|83934|1883222|5357|31092|7|80|324|26|4.28|6.50|5.39|0.00|140.14|111.28|169.00|12.61|0.00|140.14|152.75|28.86| +2452196|62069|9917|83934|1883222|5357|31092|7|4|324|80|21.99|37.82|33.28|1091.58|2662.40|1759.20|3025.60|62.83|1091.58|1570.82|1633.65|-188.38| +2452196|62069|15051|83934|1883222|5357|31092|7|151|324|90|8.22|13.80|0.55|0.00|49.50|739.80|1242.00|0.00|0.00|49.50|49.50|-690.30| +2452196|62069|3223|83934|1883222|5357|31092|7|220|324|74|31.03|31.65|0.00|0.00|0.00|2296.22|2342.10|0.00|0.00|0.00|0.00|-2296.22| +2452196|62069|2537|83934|1883222|5357|31092|7|190|324|47|38.11|72.02|56.89|0.00|2673.83|1791.17|3384.94|187.16|0.00|2673.83|2860.99|882.66| +2452196|62069|10335|83934|1883222|5357|31092|7|173|324|95|18.16|25.24|17.41|0.00|1653.95|1725.20|2397.80|0.00|0.00|1653.95|1653.95|-71.25| +2451720|49110|6518|65326|1474708|2490|36225|8|257|325|75|44.60|49.06|36.30|0.00|2722.50|3345.00|3679.50|217.80|0.00|2722.50|2940.30|-622.50| +2451720|49110|15203|65326|1474708|2490|36225|8|290|325|89|36.19|40.17|27.71|0.00|2466.19|3220.91|3575.13|147.97|0.00|2466.19|2614.16|-754.72| +2451720|49110|14972|65326|1474708|2490|36225|8|205|325|14|80.67|127.45|31.86|0.00|446.04|1129.38|1784.30|13.38|0.00|446.04|459.42|-683.34| +2451720|49110|5341|||2490|36225|8|37|325||22.91|43.98|35.62|0.00|1389.18|893.49|1715.22|69.45|0.00|||| +2451720|49110|5777||||36225|||325|47|69.41|117.99|69.61||||5545.53|6.54||||| +2451720|49110|3403|65326|1474708|2490|36225|8|200|325|56|74.11|103.01|44.29|0.00|2480.24|4150.16|5768.56|223.22|0.00|2480.24|2703.46|-1669.92| +2451720|49110|4277|65326|1474708|2490|36225|8|78|325|29|5.21|8.33|3.66|0.00|106.14|151.09|241.57|5.30|0.00|106.14|111.44|-44.95| +2451720|49110|6410|65326|1474708|2490|36225|8|295|325|71|20.42|38.59|21.61|0.00|1534.31|1449.82|2739.89|107.40|0.00|1534.31|1641.71|84.49| +2452108|52030|2905|58699|102025|4104|33830|8|124|326|8|45.85|53.64|24.67|0.00|197.36|366.80|429.12|7.89|0.00|197.36|205.25|-169.44| +2452108|52030|1827|58699|102025|4104|33830|8|79|326|38|58.91|95.43|34.35|0.00|1305.30|2238.58|3626.34|65.26|0.00|1305.30|1370.56|-933.28| +2452108|52030|16713|58699|102025|4104|33830|8|176|326|27|7.85|14.67|6.01|0.00|162.27|211.95|396.09|4.86|0.00|162.27|167.13|-49.68| +2452108|52030|6919|58699|102025|4104|33830|8|54|326|68|84.16|162.42|56.84|0.00|3865.12|5722.88|11044.56|309.20|0.00|3865.12|4174.32|-1857.76| +2452108|52030|17507|58699|102025|4104|33830|8|89|326|25|39.84|52.19|47.49|0.00|1187.25|996.00|1304.75|106.85|0.00|1187.25|1294.10|191.25| +2452108|52030|13397|58699|102025|4104|33830|8|79|326|54|42.18|54.41|10.88|0.00|587.52|2277.72|2938.14|35.25|0.00|587.52|622.77|-1690.20| +2452108|52030|2757|58699|102025|4104|33830|8|171|326|74|96.49|168.85|54.03|0.00|3998.22|7140.26|12494.90|119.94|0.00|3998.22|4118.16|-3142.04| +2452108|52030|14227|58699||||8||326|||125.12|||30.02||125.12|1.20||30.02||-44.02| +2452108|52030|16567|58699|102025|4104|33830|8|10|326|63|28.06|52.47|49.84|0.00|3139.92|1767.78|3305.61|219.79|0.00|3139.92|3359.71|1372.14| +2452108|52030|15303|58699|102025|4104|33830|8|119|326|1|32.20|58.92|7.07|0.00|7.07|32.20|58.92|0.07|0.00|7.07|7.14|-25.13| +2452108|52030|5331|58699|102025|4104|33830|8|103|326|44|73.48|124.91|38.72|0.00|1703.68|3233.12|5496.04|68.14|0.00|1703.68|1771.82|-1529.44| +2452108|52030|1703|58699|102025|4104|33830|8|143|326|72|2.51|3.36|2.58|0.00|185.76|180.72|241.92|3.71|0.00|185.76|189.47|5.04| +2452108|52030|1775|58699|102025|4104|33830|8|253|326|53|1.88|3.45|3.34|0.00|177.02|99.64|182.85|12.39|0.00|177.02|189.41|77.38| +2452108|52030|13701|58699|102025|4104|33830|8|63|326|81|20.23|27.31|17.20|0.00|1393.20|1638.63|2212.11|27.86|0.00|1393.20|1421.06|-245.43| +2452515|63955|1824|77925|221568|4411|47254|4|99|327|22|33.26|61.53|15.38|287.60|338.36|731.72|1353.66|0.00|287.60|50.76|50.76|-680.96| +2452515|63955|6181|77925|221568|4411|47254|4|20|327|14|27.27|39.81|25.08|0.00|351.12|381.78|557.34|10.53|0.00|351.12|361.65|-30.66| +2452515|63955|6198|77925|221568|4411|47254|4|36|327|8|70.33|73.14|13.16|11.58|105.28|562.64|585.12|2.81|11.58|93.70|96.51|-468.94| +2452515|63955|3711|77925|221568|4411|47254|4|15|327|51|45.53|84.68|83.83|0.00|4275.33|2322.03|4318.68|213.76|0.00|4275.33|4489.09|1953.30| +2452515|63955|17763||221568||||127|327|||||1991.82|3758.16||5966.52||1991.82|||-2720.10| +2452515|63955|16710|77925|221568|4411|47254|4|89|327|66|28.98|34.77|11.12|0.00|733.92|1912.68|2294.82|44.03|0.00|733.92|777.95|-1178.76| +2452515|63955|7705|77925|221568|4411|47254|4|103|327|67|36.60|47.21|21.24|0.00|1423.08|2452.20|3163.07|56.92|0.00|1423.08|1480.00|-1029.12| +2452515|63955|10987|77925|221568|4411|47254|4|128|327|43|6.84|8.55|1.19|0.00|51.17|294.12|367.65|3.07|0.00|51.17|54.24|-242.95| +2452515|63955|865|77925|221568|4411|47254|4|227|327|77|32.05|41.34|19.01|307.39|1463.77|2467.85|3183.18|0.00|307.39|1156.38|1156.38|-1311.47| +2452515|63955|5845|77925|221568|4411|47254|4|207|327|30|41.13|61.69|0.61|0.00|18.30|1233.90|1850.70|1.28|0.00|18.30|19.58|-1215.60| +2452515|63955|7849|77925|221568|4411|47254|4|248|327|8|86.83|122.43|47.74|0.00|381.92|694.64|979.44|30.55|0.00|381.92|412.47|-312.72| +2451167|38840|700|31383|1767129|475|27421|8|208|328|97|45.74|52.60|1.57|0.00|152.29|4436.78|5102.20|12.18|0.00|152.29|164.47|-4284.49| +2451167|38840|3860|31383|1767129|475|27421|8|297|328|99|73.41|92.49|14.79|0.00|1464.21|7267.59|9156.51|87.85|0.00|1464.21|1552.06|-5803.38| +2451167|38840|14737|31383|1767129|475|27421|8|102|328|32|87.50|157.50|51.97|0.00|1663.04|2800.00|5040.00|49.89|0.00|1663.04|1712.93|-1136.96| +2451167|38840|16972|31383|1767129|475|27421|8|104|328|31|65.72|122.23|29.33|0.00|909.23|2037.32|3789.13|63.64|0.00|909.23|972.87|-1128.09| +2451167|38840|12268|31383|1767129|475|27421|8|215|328|35|10.32|20.02|17.01|0.00|595.35|361.20|700.70|11.90|0.00|595.35|607.25|234.15| +2451167|38840|8224|31383|1767129|475|27421|8|220|328|34|43.58|57.08|45.09|91.98|1533.06|1481.72|1940.72|14.41|91.98|1441.08|1455.49|-40.64| +2451167|38840|14570|31383|1767129|475|27421|8|300|328|28|23.58|34.42|14.45|295.35|404.60|660.24|963.76|7.64|295.35|109.25|116.89|-550.99| +2451167|38840|2006|31383|1767129|475|27421|8|63|328|6|87.62|104.26|4.17|0.00|25.02|525.72|625.56|0.50|0.00|25.02|25.52|-500.70| +2451167|38840|4334|31383|1767129|475|27421|8|255|328|28|76.11|115.68|2.31|0.00|64.68|2131.08|3239.04|0.00|0.00|64.68|64.68|-2066.40| +2451167|38840|6188|31383|1767129|475|27421|8|1|328|81|98.10|114.77|78.04|0.00|6321.24|7946.10|9296.37|63.21|0.00|6321.24|6384.45|-1624.86| +2451167|38840|1903|31383|1767129|475|27421|8|76|328|75|26.60|51.60|40.24|0.00|3018.00|1995.00|3870.00|30.18|0.00|3018.00|3048.18|1023.00| +2451167|38840|1771|31383|1767129|475|27421|8|87|328|20|67.83|113.27|57.76|0.00|1155.20|1356.60|2265.40|11.55|0.00|1155.20|1166.75|-201.40| +2451167|38840|3782|31383|1767129|475|27421|8|145|328|30|27.12|51.79|40.39|0.00|1211.70|813.60|1553.70|72.70|0.00|1211.70|1284.40|398.10| +2451167|38840|14522|31383|1767129|475|27421|8|249|328|57|49.06|97.62|77.11|3428.31|4395.27|2796.42|5564.34|77.35|3428.31|966.96|1044.31|-1829.46| +2451167|38840|2443|31383|1767129|475|27421|8|103|328|85|48.79|81.47|80.65|0.00|6855.25|4147.15|6924.95|411.31|0.00|6855.25|7266.56|2708.10| +2451167|38840|17774|31383|1767129|475|27421|8|250|328|19|2.48|4.76|3.04|0.00|57.76|47.12|90.44|2.31|0.00|57.76|60.07|10.64| +2451507|50946|12838|40746|1637013|4927|17994|7|54|329|85|92.96|114.34|94.90|0.00|8066.50|7901.60|9718.90|161.33|0.00|8066.50|8227.83|164.90| +2451507|50946|14017|40746|1637013|4927|17994|7|136|329|25|8.07|13.23|2.51|0.00|62.75|201.75|330.75|3.13|0.00|62.75|65.88|-139.00| +2451507|50946|2692|40746|1637013|4927|17994|7|124|329|55|81.18|152.61|125.14|3097.21|6882.70|4464.90|8393.55|151.41|3097.21|3785.49|3936.90|-679.41| +2451507|50946|8329|40746|1637013|4927|17994|7|246|329|3|71.56|118.07|28.33|0.00|84.99|214.68|354.21|6.79|0.00|84.99|91.78|-129.69| +2451507|50946|7438|40746|1637013|4927|17994|7|20|329|31|43.42|66.43|11.29|0.00|349.99|1346.02|2059.33|10.49|0.00|349.99|360.48|-996.03| +2451507|50946|13886|40746|1637013|4927|17994|7|282|329|98|4.19|8.25|2.55|0.00|249.90|410.62|808.50|0.00|0.00|249.90|249.90|-160.72| +2451507|50946|4699|40746|1637013|4927|17994|7|269|329|84|95.70|185.65|12.99|0.00|1091.16|8038.80|15594.60|0.00|0.00|1091.16|1091.16|-6947.64| +2451507|50946|5204|40746|1637013|4927|17994|7|160|329|13|2.82|3.35|0.00|0.00|0.00|36.66|43.55|0.00|0.00|0.00|0.00|-36.66| +2450839|48122|9164|95762|824982|7002|28669|8|297|330|93|35.17|59.43|45.76|0.00|4255.68|3270.81|5526.99|212.78|0.00|4255.68|4468.46|984.87| +2450839|48122|1435|95762|824982|7002|28669|8|227|330|73|97.77|166.20|74.79|0.00|5459.67|7137.21|12132.60|0.00|0.00|5459.67|5459.67|-1677.54| +2450839|48122|4132|95762|824982|7002|28669|8|121|330|86|19.05|35.62|18.16|0.00|1561.76|1638.30|3063.32|78.08|0.00|1561.76|1639.84|-76.54| +2450839|48122|10472|95762|824982|7002|28669|8|222|330|4|62.53|85.66|22.27|16.03|89.08|250.12|342.64|5.84|16.03|73.05|78.89|-177.07| +2450839|48122|10598|95762|824982|7002|28669|8|64|330|43|17.75|22.01|1.54|0.00|66.22|763.25|946.43|4.63|0.00|66.22|70.85|-697.03| +2450839|48122|3974|95762|824982|7002|28669|8|272|330|2|24.25|46.07|25.79|4.12|51.58|48.50|92.14|2.37|4.12|47.46|49.83|-1.04| +2450839|48122|15493|95762|824982|7002|28669|8|199|330|14|47.70|77.75|44.31|0.00|620.34|667.80|1088.50|18.61|0.00|620.34|638.95|-47.46| +2450839|48122|5618|95762|824982|7002|28669|8|48|330|4|18.01|32.23|0.64|0.00|2.56|72.04|128.92|0.10|0.00|2.56|2.66|-69.48| +2452478|70912|4578|37108|1881342|2023|15104|7|142|331|49|24.15|45.16|7.67|0.00|375.83|1183.35|2212.84|11.27|0.00|375.83|387.10|-807.52| +2452478|70912|17655|37108|1881342|2023|15104|7|162|331|94|62.19|71.51|39.33|0.00|3697.02|5845.86|6721.94|0.00|0.00|3697.02|3697.02|-2148.84| +2452478|70912|4845|37108|1881342|2023|15104|7|252|331|8|90.57|103.24|75.36|0.00|602.88|724.56|825.92|6.02|0.00|602.88|608.90|-121.68| +2452478|70912|7362|37108|1881342|2023|15104|7|10|331|24|97.67|98.64|74.96|0.00|1799.04|2344.08|2367.36|0.00|0.00|1799.04|1799.04|-545.04| +2452478|70912|13951|37108|1881342|2023|15104|7|214|331|84|23.35|30.35|2.73|0.00|229.32|1961.40|2549.40|13.75|0.00|229.32|243.07|-1732.08| +2452478|70912|13773|37108|1881342|2023|15104|7|67|331|68|21.31|42.19|5.48|137.87|372.64|1449.08|2868.92|2.34|137.87|234.77|237.11|-1214.31| +2452478|70912|2953|37108|1881342|2023|15104|7|129|331|3|72.76|106.22|29.74|0.00|89.22|218.28|318.66|7.13|0.00|89.22|96.35|-129.06| +2452478|70912|17307|37108|1881342|2023|15104|7|287|331|10|10.88|18.06|2.70|0.00|27.00|108.80|180.60|2.16|0.00|27.00|29.16|-81.80| +2452478|70912|8988|37108|1881342|2023|15104|7|277|331|60|38.91|74.70|40.33|0.00|2419.80|2334.60|4482.00|48.39|0.00|2419.80|2468.19|85.20| +2452177|69001|8271|59293|320101|462|13270|7|92|332|89|27.61|43.34|37.70|0.00|3355.30|2457.29|3857.26|167.76|0.00|3355.30|3523.06|898.01| +2452177|69001|23|59293|320101|462|13270|7|282|332|32|72.15|111.11|5.55|0.00|177.60|2308.80|3555.52|14.20|0.00|177.60|191.80|-2131.20| +2452177|69001|17293|59293|320101|462|13270|7|123|332|93|91.51|155.56|115.11|0.00|10705.23|8510.43|14467.08|642.31|0.00|10705.23|11347.54|2194.80| +2452177|69001|17397|59293|320101|462|13270|7|74|332|69|67.13|120.83|8.45|0.00|583.05|4631.97|8337.27|11.66|0.00|583.05|594.71|-4048.92| +2452177|69001|347|59293|320101|462|13270|7|13|332|50|32.69|54.26|15.19|0.00|759.50|1634.50|2713.00|37.97|0.00|759.50|797.47|-875.00| +2452177|69001|11311|59293|320101|462|13270|7|248|332|20|89.12|124.76|111.03|0.00|2220.60|1782.40|2495.20|199.85|0.00|2220.60|2420.45|438.20| +2452177|69001|8571|59293|320101|462|13270|7|205|332|86|9.78|10.56|9.29|439.41|798.94|841.08|908.16|3.59|439.41|359.53|363.12|-481.55| +||13105||320101|462|||200|332|43|92.46|||0.00|5559.04|3975.78|6042.79||0.00|5559.04|5670.22|1583.26| +2450969|42453|5678|94312|464097|282|39667|7|199|333|38|56.04|60.52|20.57|711.31|781.66|2129.52|2299.76|0.70|711.31|70.35|71.05|-2059.17| +2450969|42453|16390|94312|464097|282|39667|7|212|333|76|99.98|170.96|129.92|0.00|9873.92|7598.48|12992.96|0.00|0.00|9873.92|9873.92|2275.44| +2450969|42453|6073|94312|464097|282|39667|7|157|333|8|52.10|92.73|12.98|0.00|103.84|416.80|741.84|2.07|0.00|103.84|105.91|-312.96| +2450969|42453|5020|94312|464097|282|39667|7|131|333|81|76.58|98.02|24.50|0.00|1984.50|6202.98|7939.62|99.22|0.00|1984.50|2083.72|-4218.48| +2450969|42453|14072|94312|464097|282|39667|7|272|333|53|11.97|17.83|3.56|175.47|188.68|634.41|944.99|1.05|175.47|13.21|14.26|-621.20| +2450969|42453|17416|94312|464097|282|39667|7|240|333|92|4.37|4.85|4.46|196.95|410.32|402.04|446.20|4.26|196.95|213.37|217.63|-188.67| +2450969|42453|2768|94312|464097|282|39667|7|186|333|74|36.70|61.28|60.05|0.00|4443.70|2715.80|4534.72|133.31|0.00|4443.70|4577.01|1727.90| +2450969|42453|17984|94312|464097|282|39667|7|81|333|38|86.35|168.38|18.52|703.76|703.76|3281.30|6398.44|0.00|703.76|0.00|0.00|-3281.30| +2450969|42453|2998|94312|464097|282|39667|7|225|333|24|3.51|4.52|3.97|0.00|95.28|84.24|108.48|0.00|0.00|95.28|95.28|11.04| +2452080|39177|8393|76672|152519|4719|23319|8|257|334|21|48.58|86.47|33.72|0.00|708.12|1020.18|1815.87|21.24|0.00|708.12|729.36|-312.06| +2452080|39177|13095|76672|152519|4719|23319|8|108|334|11|21.05|38.94|17.91|45.31|197.01|231.55|428.34|12.13|45.31|151.70|163.83|-79.85| +2452080|39177|6823|76672|152519|4719|23319|8|227|334|19|8.35|16.53|16.03|0.00|304.57|158.65|314.07|24.36|0.00|304.57|328.93|145.92| +2452080|39177|7089|76672|152519|4719|23319|8|260|334|57|13.93|25.90|7.77|380.88|442.89|794.01|1476.30|5.58|380.88|62.01|67.59|-732.00| +2452080|39177|8367|76672|152519|4719|23319|8|169|334|76|53.71|62.84|27.02|0.00|2053.52|4081.96|4775.84|0.00|0.00|2053.52|2053.52|-2028.44| +2452080|39177|9371|76672|152519|4719|23319|8|300|334|61|45.66|69.40|22.90|0.00|1396.90|2785.26|4233.40|69.84|0.00|1396.90|1466.74|-1388.36| +2452080|39177|13287|76672|152519|4719|23319|8|200|334|64|83.69|155.66|1.55|0.00|99.20|5356.16|9962.24|2.97|0.00|99.20|102.17|-5256.96| +2452080|39177|15693|76672|152519|4719|23319|8|271|334|89|46.86|82.00|13.94|719.58|1240.66|4170.54|7298.00|36.47|719.58|521.08|557.55|-3649.46| +2452080|39177|11675|76672|152519|4719|23319|8|150|334|22|60.12|85.37|56.34|0.00|1239.48|1322.64|1878.14|111.55|0.00|1239.48|1351.03|-83.16| +2452080|39177|3519|76672|152519|4719|23319|8|86|334|60|97.09|137.86|64.79|0.00|3887.40|5825.40|8271.60|349.86|0.00|3887.40|4237.26|-1938.00| +2452080|39177|5707|76672|152519|4719|23319|8|221|334|55|45.39|86.24|22.42|0.00|1233.10|2496.45|4743.20|61.65|0.00|1233.10|1294.75|-1263.35| +2452080|39177|15101|76672|152519|4719|23319|8|65|334|64|88.16|90.80|5.44|100.96|348.16|5642.24|5811.20|22.24|100.96|247.20|269.44|-5395.04| +2452080|39177|16409|76672|152519|4719|23319|8|257|334|70|10.68|14.31|14.31|701.19|1001.70|747.60|1001.70|24.04|701.19|300.51|324.55|-447.09| +2452080|39177|14185|76672|152519|4719|23319|8|213|334|25|10.48|13.41|12.06|262.30|301.50|262.00|335.25|1.17|262.30|39.20|40.37|-222.80| +2452552|42005|10267|72493|202474|455|37636|1|107|335|13|24.50|25.23|14.88|0.00|193.44|318.50|327.99|11.60|0.00|193.44|205.04|-125.06| +2452552|42005|16116|72493|202474|455|37636|1|20|335|70|17.29|21.26|2.97|0.00|207.90|1210.30|1488.20|18.71|0.00|207.90|226.61|-1002.40| +2452552|42005|14778|72493|202474|455|37636|1|181|335|18|92.81|160.56|126.84|2214.62|2283.12|1670.58|2890.08|3.42|2214.62|68.50|71.92|-1602.08| +||13884|72493|202474|455|37636|1|167|335|||||540.94||1435.14|2180.85|10.00|540.94||210.08|-1235.06| +2452552|42005|11151|72493|202474|455|37636|1|49|335|24|74.28|146.33|124.38|0.00|2985.12|1782.72|3511.92|268.66|0.00|2985.12|3253.78|1202.40| +2452552|42005|17017|72493|202474|455|37636|1|218|335|57|57.09|82.78|73.67|0.00|4199.19|3254.13|4718.46|335.93|0.00|4199.19|4535.12|945.06| +2452552|42005|15835|72493|202474|455|37636|1|49|335|100|53.69|68.18|14.31|0.00|1431.00|5369.00|6818.00|85.86|0.00|1431.00|1516.86|-3938.00| +2452552|42005|9895|72493|202474|455|37636|1|286|335|80|93.24|96.96|81.44|0.00|6515.20|7459.20|7756.80|65.15|0.00|6515.20|6580.35|-944.00| +2452552|42005|7351|72493|202474|455|37636|1|4|335|34|31.33|36.96|20.69|0.00|703.46|1065.22|1256.64|56.27|0.00|703.46|759.73|-361.76| +2452552|42005|15433|72493|202474|455|37636|1|275|335|23|24.13|43.91|2.19|0.00|50.37|554.99|1009.93|1.51|0.00|50.37|51.88|-504.62| +2452552|42005|4773|72493|202474|455|37636|1|23|335|81|96.96|161.92|6.47|0.00|524.07|7853.76|13115.52|41.92|0.00|524.07|565.99|-7329.69| +2452552|42005|17898|72493|202474|455|37636|1|3|335|57|77.83|127.64|8.93|0.00|509.01|4436.31|7275.48|40.72|0.00|509.01|549.73|-3927.30| +2452552|42005|15624|72493|202474|455|37636|1|19|335|38|9.91|10.10|9.09|0.00|345.42|376.58|383.80|6.90|0.00|345.42|352.32|-31.16| +2451882|40407|11227|6836|1225823|5963|34950|10|116|336|93|71.27|138.97|11.11|185.98|1033.23|6628.11|12924.21|16.94|185.98|847.25|864.19|-5780.86| +2451882|40407|1310|6836|1225823|5963|34950|10|64|336|74|44.57|66.40|63.74|0.00|4716.76|3298.18|4913.60|283.00|0.00|4716.76|4999.76|1418.58| +2451882|40407|12113|6836|1225823|5963|34950|10|173|336|47|92.08|116.94|54.96|0.00|2583.12|4327.76|5496.18|180.81|0.00|2583.12|2763.93|-1744.64| +2451882|40407|6770|6836|1225823|5963|34950|10|195|336|60|22.99|43.45|20.85|0.00|1251.00|1379.40|2607.00|87.57|0.00|1251.00|1338.57|-128.40| +2451882|40407|13861|6836|1225823|5963|34950|10|160|336|19|65.48|78.57|45.57|0.00|865.83|1244.12|1492.83|17.31|0.00|865.83|883.14|-378.29| +2451882|40407|11041|6836|1225823|5963|34950|10|169|336|17|98.49|107.35|72.99|0.00|1240.83|1674.33|1824.95|111.67|0.00|1240.83|1352.50|-433.50| +2451882|40407|2882|6836|1225823|5963|34950|10|199|336|49|66.07|72.67|40.69|0.00|1993.81|3237.43|3560.83|79.75|0.00|1993.81|2073.56|-1243.62| +2451882|40407|7382|6836|1225823|5963|34950|10|232|336|47|44.36|48.35|5.80|0.00|272.60|2084.92|2272.45|8.17|0.00|272.60|280.77|-1812.32| +2451882|40407|15377|6836|1225823|5963|34950|10|124|336|67|66.43|120.90|114.85|0.00|7694.95|4450.81|8100.30|76.94|0.00|7694.95|7771.89|3244.14| +2451882|40407|16085|6836||5963|34950||41|336|36||||||2737.80|2956.68|||||-2235.24| +2451882|40407|6884|6836|1225823|5963|34950|10|105|336|37|45.52|66.00|2.64|0.00|97.68|1684.24|2442.00|8.79|0.00|97.68|106.47|-1586.56| +2451882|40407|11429|6836|1225823|5963|34950|10|31|336|23|94.63|109.77|81.22|0.00|1868.06|2176.49|2524.71|130.76|0.00|1868.06|1998.82|-308.43| +2450952|68002|14840|99819|525755|5738|34474|10|155|337|77|43.92|46.11|26.74|0.00|2058.98|3381.84|3550.47|20.58|0.00|2058.98|2079.56|-1322.86| +2450952|68002|3386|99819|525755|5738|34474|10|202|337|86|88.09|155.03|136.42|0.00|11732.12|7575.74|13332.58|117.32|0.00|11732.12|11849.44|4156.38| +2450952|68002|13801|99819|525755|5738|34474|10|35|337|90|6.95|7.29|3.13|56.34|281.70|625.50|656.10|2.25|56.34|225.36|227.61|-400.14| +2450952|68002|17290|99819|525755|5738|34474|10|110|337|68|68.99|133.15|5.32|54.26|361.76|4691.32|9054.20|6.15|54.26|307.50|313.65|-4383.82| +2450952|68002|400|99819|525755|5738|34474|10|40|337|88|65.31|112.33|37.06|0.00|3261.28|5747.28|9885.04|260.90|0.00|3261.28|3522.18|-2486.00| +2450952|68002|11425|99819|525755|5738|34474|10|287|337|10|60.51|113.15|10.18|67.18|101.80|605.10|1131.50|1.73|67.18|34.62|36.35|-570.48| +2450952|68002|13532|99819|525755|5738|34474|10|36|337|36|78.27|90.01|82.80|0.00|2980.80|2817.72|3240.36|89.42|0.00|2980.80|3070.22|163.08| +2450952|68002|16528|99819|525755|5738|34474|10|74|337|57|59.72|102.12|7.14|0.00|406.98|3404.04|5820.84|24.41|0.00|406.98|431.39|-2997.06| +2450952|68002|16222|99819|525755|5738|34474|10|40|337|1|97.91|111.61|10.04|0.00|10.04|97.91|111.61|0.00|0.00|10.04|10.04|-87.87| +2450952|68002|13327|99819|525755|5738|34474|10|109|337|74|83.48|154.43|115.82|0.00|8570.68|6177.52|11427.82|514.24|0.00|8570.68|9084.92|2393.16| +2450952||13591||525755||34474|10|249|337||||0.00|0.00|0.00||528.57||0.00|||| +2451383|61154|200|74364|337001|4438|8275|4|231|338|99|5.98|8.31|5.56|0.00|550.44|592.02|822.69|27.52|0.00|550.44|577.96|-41.58| +2451383|61154|9710|74364|337001|4438|8275|4|75|338|68|97.46|166.65|39.99|0.00|2719.32|6627.28|11332.20|135.96|0.00|2719.32|2855.28|-3907.96| +2451383|61154|6220|74364|337001|4438|8275|4|183|338|7|8.82|9.08|1.36|0.00|9.52|61.74|63.56|0.47|0.00|9.52|9.99|-52.22| +2451383|61154|7784|74364|337001|4438|8275|4|67|338|68|53.50|60.45|1.81|0.00|123.08|3638.00|4110.60|3.69|0.00|123.08|126.77|-3514.92| +2451383|61154|10441|74364|337001|4438|8275|4|20|338|40|34.40|43.00|13.76|0.00|550.40|1376.00|1720.00|11.00|0.00|550.40|561.40|-825.60| +2451383|61154|2164|74364|337001|4438|8275|4|1|338|49|85.71|99.42|67.60|2550.54|3312.40|4199.79|4871.58|22.85|2550.54|761.86|784.71|-3437.93| +2451383|61154|757|74364|337001|4438|8275|4|60|338|27|70.52|71.22|10.68|0.00|288.36|1904.04|1922.94|17.30|0.00|288.36|305.66|-1615.68| +2451383|61154|8870|74364|337001|4438|8275|4|270|338|86|23.76|30.65|2.14|130.66|184.04|2043.36|2635.90|4.80|130.66|53.38|58.18|-1989.98| +2451383||3439||337001||8275||79|338||41.85|83.28|35.81|0.00|3294.52|3850.20|7661.76|230.61|0.00||3525.13|-555.68| +2451383|61154|7018|74364|337001|4438|8275|4|211|338|74|48.49|66.91|29.44|0.00|2178.56|3588.26|4951.34|21.78|0.00|2178.56|2200.34|-1409.70| +2451383|61154|5113|74364|337001|4438|8275|4|208|338|62|8.21|9.44|5.28|0.00|327.36|509.02|585.28|16.36|0.00|327.36|343.72|-181.66| +2451383|61154|4154|74364|337001|4438|8275|4|291|338|64|93.88|159.59|143.63|0.00|9192.32|6008.32|10213.76|827.30|0.00|9192.32|10019.62|3184.00| +2451383|61154|15698|74364|337001|4438|8275|4|43|338|19|18.96|19.33|14.30|0.00|271.70|360.24|367.27|8.15|0.00|271.70|279.85|-88.54| +2451383|61154|12086|74364|337001|4438|8275|4|138|338|56|44.26|53.99|28.61|0.00|1602.16|2478.56|3023.44|144.19|0.00|1602.16|1746.35|-876.40| +2451383|61154|7507|74364|337001|4438|8275|4|16|338|98|77.16|81.78|37.61|1732.31|3685.78|7561.68|8014.44|0.00|1732.31|1953.47|1953.47|-5608.21| +2451383|61154|8468|74364|337001|4438|8275|4|122|338|89|52.63|59.99|17.39|0.00|1547.71|4684.07|5339.11|139.29|0.00|1547.71|1687.00|-3136.36| +2451366|38922|12182|95992|1709535|3078|29821|1|284|339|72|12.00|13.20|4.75|0.00|342.00|864.00|950.40|0.00|0.00|342.00|342.00|-522.00| +2451366|38922|890|95992|1709535|3078|29821|1|187|339|100|61.54|89.84|9.88|0.00|988.00|6154.00|8984.00|69.16|0.00|988.00|1057.16|-5166.00| +2451366|38922|16738|95992|1709535|3078|29821|1|237|339|46|76.59|76.59|33.69|0.00|1549.74|3523.14|3523.14|108.48|0.00|1549.74|1658.22|-1973.40| +2451366|38922|2083|95992|1709535|3078|29821|1|145|339|7|66.50|98.42|60.03|0.00|420.21|465.50|688.94|0.00|0.00|420.21|420.21|-45.29| +2451366|38922|7804|95992|1709535||29821||68|339|27||73.73||||||||1433.16|1562.14|-86.67| +2451366|38922|10456|95992|1709535|3078|29821|1|21|339|47|35.53|68.92|31.70|0.00|1489.90|1669.91|3239.24|29.79|0.00|1489.90|1519.69|-180.01| +2451366|38922|5074|95992|1709535|3078|29821|1|253|339|33|62.89|77.98|42.10|0.00|1389.30|2075.37|2573.34|97.25|0.00|1389.30|1486.55|-686.07| +2451366|38922|16768|95992|1709535|3078|29821|1|127|339|29|71.66|123.97|84.29|0.00|2444.41|2078.14|3595.13|122.22|0.00|2444.41|2566.63|366.27| +2451366|38922|9358|95992|1709535|3078|29821|1|77|339|9|56.82|84.66|11.85|0.00|106.65|511.38|761.94|7.46|0.00|106.65|114.11|-404.73| +2451366|38922|3823|95992|1709535|3078|29821|1|147|339|93|73.69|99.48|38.79|0.00|3607.47|6853.17|9251.64|324.67|0.00|3607.47|3932.14|-3245.70| +2451366|38922|2323|95992|1709535|3078|29821|1|182|339|68|79.52|94.62|39.74|2432.08|2702.32|5407.36|6434.16|8.10|2432.08|270.24|278.34|-5137.12| +2451366|38922|3169|95992|1709535|3078|29821|1|186|339|91|44.11|63.95|14.06|0.00|1279.46|4014.01|5819.45|38.38|0.00|1279.46|1317.84|-2734.55| +2451366|38922|7480|95992|1709535|3078|29821|1|60|339|96|38.43|68.02|14.28|0.00|1370.88|3689.28|6529.92|95.96|0.00|1370.88|1466.84|-2318.40| +2451366|38922|2674|95992|1709535|3078|29821|1|62|339|85|15.49|25.24|3.53|0.00|300.05|1316.65|2145.40|0.00|0.00|300.05|300.05|-1016.60| +2452167|33438|417|21461|1143282|5550|1371|8|176|340|57|97.21|116.65|116.65|0.00|6649.05|5540.97|6649.05|531.92|0.00|6649.05|7180.97|1108.08| +2452167|33438|14421|21461|1143282|5550|1371|8|72|340|96|64.57|72.96|29.91|0.00|2871.36|6198.72|7004.16|28.71|0.00|2871.36|2900.07|-3327.36| +2452167|33438|6949|21461|1143282|5550|1371|8|287|340|20|41.73|68.85|53.70|0.00|1074.00|834.60|1377.00|32.22|0.00|1074.00|1106.22|239.40| +2452167|33438|7825|21461|1143282|5550|1371|8|154|340|10|58.85|97.69|93.78|0.00|937.80|588.50|976.90|0.00|0.00|937.80|937.80|349.30| +2452167||14351|21461|1143282|5550|1371|8|70|340|10|84.30|165.22||0.00|280.80||||0.00|||-562.20| +2452167|33438|8719|21461|1143282|5550|1371|8|209|340|68|84.63|160.79|91.65|0.00|6232.20|5754.84|10933.72|62.32|0.00|6232.20|6294.52|477.36| +2452167|33438|10555|21461|1143282|5550|1371|8|79|340|32|78.92|95.49|29.60|0.00|947.20|2525.44|3055.68|18.94|0.00|947.20|966.14|-1578.24| +2452167|33438|15119|21461|1143282|5550|1371|8|233|340|99|65.17|65.82|26.98|0.00|2671.02|6451.83|6516.18|160.26|0.00|2671.02|2831.28|-3780.81| +|52831|17132|||6162||||341|18|||34.93|||1050.84|1103.22|||||| +2451512|52831|10930|80091|1206503|6162|26174|7|169|341|34|12.75|23.58|10.13|0.00|344.42|433.50|801.72|17.22|0.00|344.42|361.64|-89.08| +2451512||1738||1206503||||263|341|||83.76|8.37|0.00|200.88|||18.07|0.00|200.88||| +2451512|52831|3356|80091|1206503|6162|26174|7|102|341|91|50.80|53.34|13.33|0.00|1213.03|4622.80|4853.94|48.52|0.00|1213.03|1261.55|-3409.77| +||9368||||26174|7|148|341|54|67.30|106.33|63.79|0.00|3444.66|||68.89|0.00||3513.55|-189.54| +2451512|52831|3200|80091|1206503|6162|26174|7|41|341|76|76.87|93.78|83.46|0.00|6342.96|5842.12|7127.28|570.86|0.00|6342.96|6913.82|500.84| +2451512|52831|5290|80091|1206503|6162|26174|7|115|341|34|70.69|94.72|75.77|0.00|2576.18|2403.46|3220.48|231.85|0.00|2576.18|2808.03|172.72| +2451512|52831|11528|80091|1206503|6162|26174|7|282|341|60|99.23|128.99|50.30|0.00|3018.00|5953.80|7739.40|150.90|0.00|3018.00|3168.90|-2935.80| +2451512|52831|1604|80091|1206503|6162|26174|7|52|341|34|93.17|183.54|55.06|0.00|1872.04|3167.78|6240.36|131.04|0.00|1872.04|2003.08|-1295.74| +2451512|52831|12307|80091|1206503|6162|26174|7|152|341|11|15.67|26.63|15.71|0.00|172.81|172.37|292.93|10.36|0.00|172.81|183.17|0.44| +2451512|52831|7238|80091|1206503|6162|26174|7|155|341|89|50.75|57.34|34.40|0.00|3061.60|4516.75|5103.26|122.46|0.00|3061.60|3184.06|-1455.15| +2451512|52831|11185|80091|1206503|6162|26174|7|81|341|65|24.77|37.89|3.41|0.00|221.65|1610.05|2462.85|19.94|0.00|221.65|241.59|-1388.40| +2451512|52831|10616|80091|1206503|6162|26174|7|249|341|35|86.56|156.67|144.13|0.00|5044.55|3029.60|5483.45|201.78|0.00|5044.55|5246.33|2014.95| +2451512|52831|2510|80091|1206503|6162|26174|7|288|341|61|22.73|29.32|24.04|982.51|1466.44|1386.53|1788.52|38.71|982.51|483.93|522.64|-902.60| +2451512|52831|17858|80091|1206503|6162|26174|7|230|341|50|61.07|90.38|40.67|1138.76|2033.50|3053.50|4519.00|80.52|1138.76|894.74|975.26|-2158.76| +2451512|52831|8728|80091|1206503|6162|26174|7|20|341|82|45.13|78.07|62.45|0.00|5120.90|3700.66|6401.74|256.04|0.00|5120.90|5376.94|1420.24| +2451775|47416|8993|12041|1311657|6920|19383|1|214|342|51|96.61|128.49|96.36|0.00|4914.36|4927.11|6552.99|442.29|0.00|4914.36|5356.65|-12.75| +2451775|47416|895|12041|1311657|6920|19383|1|33|342|98|91.82|99.16|1.98|0.00|194.04|8998.36|9717.68|15.52|0.00|194.04|209.56|-8804.32| +2451775|47416|2659|12041|1311657|6920|19383|1|206|342|44|77.59|117.93|4.71|0.00|207.24|3413.96|5188.92|18.65|0.00|207.24|225.89|-3206.72| +2451775|47416|8053|12041|1311657|6920|19383|1|52|342|52|36.69|49.16|37.36|1593.03|1942.72|1907.88|2556.32|6.99|1593.03|349.69|356.68|-1558.19| +2451775|47416|3473|12041|1311657|6920|19383|1|82|342|48|36.31|39.57|24.13|0.00|1158.24|1742.88|1899.36|23.16|0.00|1158.24|1181.40|-584.64| +2451775|47416|1729|12041|1311657|6920|19383|1|16|342|57|5.86|10.02|6.31|0.00|359.67|334.02|571.14|10.79|0.00|359.67|370.46|25.65| +2451775|47416|15746|12041|1311657|6920|19383|1|162|342|47|74.70|121.76|70.62|0.00|3319.14|3510.90|5722.72|132.76|0.00|3319.14|3451.90|-191.76| +2451775|47416|5246|12041|1311657|6920|19383|1|232|342|86|50.47|69.14|49.08|0.00|4220.88|4340.42|5946.04|379.87|0.00|4220.88|4600.75|-119.54| +2451775|47416|17597|12041|1311657|6920|19383|1|19|342|42|48.61|62.70|3.76|0.00|157.92|2041.62|2633.40|12.63|0.00|157.92|170.55|-1883.70| +2451775|47416|3824|12041|1311657|6920|19383|1|20|342|14|19.58|36.41|1.45|0.00|20.30|274.12|509.74|1.62|0.00|20.30|21.92|-253.82| +2451550|62977|9557|73030|1331342|6454|5166|7|188|343|90|97.88|110.60|54.19|0.00|4877.10|8809.20|9954.00|243.85|0.00|4877.10|5120.95|-3932.10| +2451550|62977|13193|73030|1331342|6454|5166|7|275|343|52|10.67|19.63|15.50|0.00|806.00|554.84|1020.76|0.00|0.00|806.00|806.00|251.16| +2451550|62977|9955|73030|1331342|6454|5166|7|236|343|37|97.32|127.48|109.63|1460.27|4056.31|3600.84|4716.76|51.92|1460.27|2596.04|2647.96|-1004.80| +2451550|62977|16703|73030|1331342|6454|5166|7|65|343|51|67.65|81.85|62.20|0.00|3172.20|3450.15|4174.35|31.72|0.00|3172.20|3203.92|-277.95| +2451550|62977|15815|73030|1331342|6454|5166|7|4|343|63|2.52|4.43|1.86|0.00|117.18|158.76|279.09|1.17|0.00|117.18|118.35|-41.58| +2451550|62977|11123|73030|1331342|6454|5166|7|182|343|59|69.50|97.99|95.05|0.00|5607.95|4100.50|5781.41|504.71|0.00|5607.95|6112.66|1507.45| +2451550|62977|11083|73030|1331342|6454|5166|7|27|343|49|94.81|156.43|17.20|665.81|842.80|4645.69|7665.07|7.07|665.81|176.99|184.06|-4468.70| +2451550|62977|13163|73030|1331342|6454|5166|7|107|343|75|67.96|80.87|21.83|720.39|1637.25|5097.00|6065.25|82.51|720.39|916.86|999.37|-4180.14| +2451550|62977|8966|73030|1331342|6454|5166|7|231|343|9|59.28|102.55|86.14|0.00|775.26|533.52|922.95|0.00|0.00|775.26|775.26|241.74| +2451550||10556|||6454|||141|343|4|5.86||2.57||10.28|23.44|46.88|0.25||||-17.06| +2451550|62977|16423|73030|1331342|6454|5166|7|277|343|57|58.53|78.43|69.01|3815.56|3933.57|3336.21|4470.51|10.62|3815.56|118.01|128.63|-3218.20| +2451550|62977|10807|73030|1331342|6454|5166|7|229|343|23|16.42|31.69|9.50|0.00|218.50|377.66|728.87|15.29|0.00|218.50|233.79|-159.16| +2451550|62977|15809|73030|1331342|6454|5166|7|185|343|60|94.88|110.06|77.04|0.00|4622.40|5692.80|6603.60|46.22|0.00|4622.40|4668.62|-1070.40| +2451550|62977|1697|73030|1331342|6454|5166|7|155|343|59|65.95|129.26|95.65|0.00|5643.35|3891.05|7626.34|225.73|0.00|5643.35|5869.08|1752.30| +2451298|36840|2276|48579|644346|5713|30280|7|108|344|11|62.99|69.28|16.62|0.00|182.82|692.89|762.08|3.65|0.00|182.82|186.47|-510.07| +2451298|36840|5894|48579|644346|5713|30280|7|252|344|30|88.56|102.72|50.33|0.00|1509.90|2656.80|3081.60|75.49|0.00|1509.90|1585.39|-1146.90| +2451298|36840|2740|48579|644346|5713|30280|7|273|344|30|21.17|38.52|24.65|0.00|739.50|635.10|1155.60|59.16|0.00|739.50|798.66|104.40| +|36840|12190||644346|5713|30280|7||344||87.72|148.24|||1589.12|1403.52|2371.84||||527.11|| +2451298|36840|8252|48579|644346|5713|30280|7|57|344|15|56.72|108.33|48.74|0.00|731.10|850.80|1624.95|65.79|0.00|731.10|796.89|-119.70| +2451298|36840|6667|48579|644346|5713|30280|7|268|344|100|48.26|80.11|20.82|0.00|2082.00|4826.00|8011.00|62.46|0.00|2082.00|2144.46|-2744.00| +2451298|36840|5362|48579|644346|5713|30280|7|213|344|29|7.80|9.90|6.83|0.00|198.07|226.20|287.10|7.92|0.00|198.07|205.99|-28.13| +2451298|36840|13358|48579|644346|5713|30280|7|228|344|6|40.06|56.48|37.27|0.00|223.62|240.36|338.88|13.41|0.00|223.62|237.03|-16.74| +2451298|36840|9080|48579|644346|5713|30280|7|21|344|78|26.43|52.06|13.01|0.00|1014.78|2061.54|4060.68|81.18|0.00|1014.78|1095.96|-1046.76| +2451298|36840|950|48579|644346|5713|30280|7|257|344|14|49.51|86.14|60.29|0.00|844.06|693.14|1205.96|42.20|0.00|844.06|886.26|150.92| +2451298|36840|17890|48579|644346|5713|30280|7|42|344|79|93.07|111.68|1.11|0.00|87.69|7352.53|8822.72|4.38|0.00|87.69|92.07|-7264.84| +2451298|36840|1912|48579|644346|5713|30280|7|66|344|24|58.69|105.05|19.95|0.00|478.80|1408.56|2521.20|0.00|0.00|478.80|478.80|-929.76| +2451298|36840|9088|48579|644346|5713|30280|7|113|344|64|53.16|59.53|2.38|0.00|152.32|3402.24|3809.92|3.04|0.00|152.32|155.36|-3249.92| +2451298|36840|4966|48579|644346|5713|30280|7|265|344|53|72.15|122.65|42.92|0.00|2274.76|3823.95|6500.45|113.73|0.00|2274.76|2388.49|-1549.19| +2451298|36840|4876|48579|644346|5713|30280|7|242|344|42|59.93|76.71|28.38|0.00|1191.96|2517.06|3221.82|11.91|0.00|1191.96|1203.87|-1325.10| +2451298|36840|2071|48579|644346|5713|30280|7|133|344|33|4.86|6.46|4.19|0.00|138.27|160.38|213.18|0.00|0.00|138.27|138.27|-22.11| +2451098|48845|5606|76229|1730381|980|47932|7|70|345|19|2.37|4.66|1.86|20.14|35.34|45.03|88.54|0.15|20.14|15.20|15.35|-29.83| +2451098|48845|9070|76229|1730381|980|47932|7|58|345|83|15.90|20.19|5.45|0.00|452.35|1319.70|1675.77|13.57|0.00|452.35|465.92|-867.35| +||8710||1730381||47932|||345|||105.46|32.69|0.00|621.11|||49.68|0.00|||| +2451098|48845|15583|76229|1730381|980|47932|7|261|345|37|41.13|44.42|36.42|0.00|1347.54|1521.81|1643.54|80.85|0.00|1347.54|1428.39|-174.27| +2451098|48845|6385|76229|1730381|980|47932|7|278|345|68|80.85|158.46|3.16|58.01|214.88|5497.80|10775.28|9.41|58.01|156.87|166.28|-5340.93| +2451098|48845|307|76229|1730381|980|47932|7|210|345|55|37.31|55.21|31.46|1366.93|1730.30|2052.05|3036.55|25.43|1366.93|363.37|388.80|-1688.68| +|48845|16639||1730381||47932|||345|||66.60|37.96||||1465.20||||876.87|-615.78| +2451098|48845|12058|76229|1730381|980|47932|7|256|345|55|99.04|185.20|33.33|274.97|1833.15|5447.20|10186.00|109.07|274.97|1558.18|1667.25|-3889.02| +2450895|72850|15772|54810|743167|6724|10234|4|16|346|35|4.80|4.80|3.31|0.00|115.85|168.00|168.00|6.95|0.00|115.85|122.80|-52.15| +2450895|72850|2696|54810|||10234|4|170|346|91||||||2855.58|3283.28|18.70||1870.96||-984.62| +2450895|72850|268|54810|743167|6724|10234|4|208|346|4|50.49|97.95|76.40|0.00|305.60|201.96|391.80|6.11|0.00|305.60|311.71|103.64| +2450895|72850|6577|54810|743167|6724|10234|4|13|346|83|57.13|75.41|26.39|2037.04|2190.37|4741.79|6259.03|3.06|2037.04|153.33|156.39|-4588.46| +||13555||743167|||4||346|67||60.69|9.10|0.00|609.70|3910.12|||0.00|||| +2450895||12757|54810|743167|6724|10234||41|346|||48.84|||1464.75|2796.75|3663.00|58.59|||1523.34|-1332.00| +2450895|72850|2326|54810|743167|6724|10234|4|132|346|97|4.61|9.03|5.14|154.55|498.58|447.17|875.91|6.88|154.55|344.03|350.91|-103.14| +2450895|72850|12604|54810|743167|6724|10234|4|133|346|21|60.70|89.22|10.70|0.00|224.70|1274.70|1873.62|15.72|0.00|224.70|240.42|-1050.00| +2450895|72850|7405|54810|743167|6724|10234|4|132|346|79|91.32|111.41|22.28|1003.26|1760.12|7214.28|8801.39|0.00|1003.26|756.86|756.86|-6457.42| +||5738||743167|6724|10234|4||346|78||7.87|2.12|||404.04|613.86||||165.36|-238.68| +2450895|72850|7960|54810|743167|6724|10234|4|202|346|2|31.54|31.54|27.75|7.21|55.50|63.08|63.08|1.93|7.21|48.29|50.22|-14.79| +2450895|72850|5593|54810|743167|6724|10234|4|281|346|9|95.20|154.22|32.38|0.00|291.42|856.80|1387.98|23.31|0.00|291.42|314.73|-565.38| +2451509|55298|11204|86460|1699979|4698|32264|1|221|347|63|81.04|102.11|72.49|1370.06|4566.87|5105.52|6432.93|95.90|1370.06|3196.81|3292.71|-1908.71| +2451509|55298|14246|86460|1699979|4698|32264|1|9|347|97|86.72|116.20|18.59|1388.48|1803.23|8411.84|11271.40|16.59|1388.48|414.75|431.34|-7997.09| +2451509|55298|7474|86460|1699979|4698|32264|1|30|347|78|34.18|66.99|1.33|0.00|103.74|2666.04|5225.22|9.33|0.00|103.74|113.07|-2562.30| +2451509|55298|17060|86460|1699979|4698|32264|1|163|347|48|15.44|28.40|10.22|0.00|490.56|741.12|1363.20|14.71|0.00|490.56|505.27|-250.56| +2451509|55298|14078|86460|1699979|4698|32264|1|205|347|83|29.81|33.68|21.55|697.57|1788.65|2474.23|2795.44|10.91|697.57|1091.08|1101.99|-1383.15| +2451509|55298|1792|86460|1699979|4698|32264|1|86|347|22|48.94|85.64|85.64|0.00|1884.08|1076.68|1884.08|18.84|0.00|1884.08|1902.92|807.40| +2451509|55298|16303|86460|1699979|4698|32264|1|202|347|100|39.74|66.36|31.85|0.00|3185.00|3974.00|6636.00|127.40|0.00|3185.00|3312.40|-789.00| +2451509|55298|14713|86460|1699979|4698|32264|1|110|347|38|49.27|89.67|16.14|0.00|613.32|1872.26|3407.46|24.53|0.00|613.32|637.85|-1258.94| +2451678|50040|15698|31073|1490854|6188|8722|4|134|348|90|77.12|145.75|48.09|0.00|4328.10|6940.80|13117.50|216.40|0.00|4328.10|4544.50|-2612.70| +2451678|50040|12086|31073|1490854|6188|8722|4|282|348|11|47.92|51.75|11.38|0.00|125.18|527.12|569.25|5.00|0.00|125.18|130.18|-401.94| +2451678|50040|7507|31073|1490854|6188|8722|4|105|348|41|7.36|11.48|8.61|0.00|353.01|301.76|470.68|24.71|0.00|353.01|377.72|51.25| +2451678|50040|8468|31073|1490854|6188|8722|4|80|348|97|11.57|14.80|3.25|0.00|315.25|1122.29|1435.60|18.91|0.00|315.25|334.16|-807.04| +2451678|50040|2132|31073|1490854|6188|8722|4|250|348|100|91.84|108.37|44.43|133.29|4443.00|9184.00|10837.00|301.67|133.29|4309.71|4611.38|-4874.29| +2451678|50040|12151|31073|1490854|6188|8722|4|193|348|34|67.92|104.59|17.78|0.00|604.52|2309.28|3556.06|0.00|0.00|604.52|604.52|-1704.76| +2451678|50040|10406|31073|1490854|6188|8722|4|248|348|15|55.26|110.52|96.15|0.00|1442.25|828.90|1657.80|115.38|0.00|1442.25|1557.63|613.35| +2451678|50040|8876|31073|1490854|6188|8722|4|247|348|33|88.03|125.00|25.00|0.00|825.00|2904.99|4125.00|16.50|0.00|825.00|841.50|-2079.99| +2451678|50040|11683|31073|1490854|6188|8722|4|13|348|59|43.36|44.22|17.68|0.00|1043.12|2558.24|2608.98|0.00|0.00|1043.12|1043.12|-1515.12| +2451678|50040|2407|||6188|8722|||348|||9.49|||35.67||275.21|1.78||||-146.74| +2451678|50040|15521|31073|1490854|6188|8722|4|246|348|44|14.98|25.76|19.83|0.00|872.52|659.12|1133.44|69.80|0.00|872.52|942.32|213.40| +2451678|50040|7070|31073|1490854|6188|8722|4|66|348|16|90.53|132.17|91.19|0.00|1459.04|1448.48|2114.72|0.00|0.00|1459.04|1459.04|10.56| +2451678|50040|6104|31073|1490854|6188|8722|4|266|348|30|86.65|124.77|101.06|0.00|3031.80|2599.50|3743.10|151.59|0.00|3031.80|3183.39|432.30| +2451678|50040|14623|31073|1490854|6188|8722|4|111|348|30|41.84|69.03|12.42|0.00|372.60|1255.20|2070.90|22.35|0.00|372.60|394.95|-882.60| +2451806|65883|6229|73970|733293|4969|20692|7|173|349|73|23.76|27.08|9.20|208.19|671.60|1734.48|1976.84|41.70|208.19|463.41|505.11|-1271.07| +2451806|65883|11141|73970|733293|4969|20692|7|258|349|6|23.95|41.43|26.51|0.00|159.06|143.70|248.58|7.95|0.00|159.06|167.01|15.36| +2451806|65883|3737|73970|733293|4969|20692|7|165|349|95|41.83|79.05|26.87|178.68|2552.65|3973.85|7509.75|142.43|178.68|2373.97|2516.40|-1599.88| +2451806|65883|13471|73970|733293|4969|20692|7|92|349|21|34.35|49.80|17.43|0.00|366.03|721.35|1045.80|14.64|0.00|366.03|380.67|-355.32| +2451806||3877|73970|733293||20692||221|349|66|44.88||15.61|0.00||2962.08|4294.62|61.81|0.00|1030.26||| +2451806|65883|16412|73970|733293|4969|20692|7|114|349|54|82.15|134.72|28.29|0.00|1527.66|4436.10|7274.88|61.10|0.00|1527.66|1588.76|-2908.44| +2451806|65883|13751|73970|733293|4969|20692|7|85|349|30|98.24|126.72|69.69|0.00|2090.70|2947.20|3801.60|146.34|0.00|2090.70|2237.04|-856.50| +2451806|65883|12329|73970|733293|4969|20692|7|36|349|97|88.49|152.20|114.15|0.00|11072.55|8583.53|14763.40|221.45|0.00|11072.55|11294.00|2489.02| +2451806|65883|8471|73970|733293|4969|20692|7|265|349|29|32.38|32.70|14.71|0.00|426.59|939.02|948.30|25.59|0.00|426.59|452.18|-512.43| +2451806|65883|9175|73970|733293|4969|20692|7|237|349|33|38.50|52.36|32.98|0.00|1088.34|1270.50|1727.88|54.41|0.00|1088.34|1142.75|-182.16| +2451806|65883|1496|73970|733293|4969|20692|7|290|349|6|35.78|64.40|11.59|0.00|69.54|214.68|386.40|5.56|0.00|69.54|75.10|-145.14| +2452558|52247|7773|52959|90652|1945|38391|10|64|350|38|83.15|117.24|70.34|0.00|2672.92|3159.70|4455.12|0.00|0.00|2672.92|2672.92|-486.78| +2452558|52247|751|52959|90652|1945|38391|10|33|350|87|60.86|115.02|64.41|0.00|5603.67|5294.82|10006.74|168.11|0.00|5603.67|5771.78|308.85| +2452558|52247|17959|52959|90652|1945|38391|10|17|350|80|37.82|70.72|23.33|0.00|1866.40|3025.60|5657.60|149.31|0.00|1866.40|2015.71|-1159.20| +2452558|52247|1135|52959|90652|1945|38391|10|137|350|10|61.07|101.37|41.56|0.00|415.60|610.70|1013.70|4.15|0.00|415.60|419.75|-195.10| +2452558|52247|2541|52959|90652|1945|38391|10|203|350|40|6.29|11.25|10.23|0.00|409.20|251.60|450.00|16.36|0.00|409.20|425.56|157.60| +2452558|52247|11685|52959|90652|1945|38391|10|110|350|66|18.85|36.19|28.95|0.00|1910.70|1244.10|2388.54|95.53|0.00|1910.70|2006.23|666.60| +||2005||90652|||10|280|350|20|||5.35|57.78||101.00|187.80|2.46|57.78|49.22|51.68|-51.78| +2452558|52247|8443|52959|90652|1945|38391|10|268|350|19|49.17|85.06|84.20|0.00|1599.80|934.23|1616.14|31.99|0.00|1599.80|1631.79|665.57| +2452558|52247|10777||90652||38391|||350|24||116.49|||1481.52|2219.04|||||1614.85|| +2451032|60808|745|26878|484227|7176|24972|4|100|351|75|63.14|91.55|90.63|0.00|6797.25|4735.50|6866.25|0.00|0.00|6797.25|6797.25|2061.75| +2451032|60808|4288|26878|484227|7176|24972|4|48|351|85|23.45|29.07|25.00|0.00|2125.00|1993.25|2470.95|85.00|0.00|2125.00|2210.00|131.75| +2451032|60808|11266|26878|484227|7176|24972|4|159|351|39|81.69|96.39|66.50|881.79|2593.50|3185.91|3759.21|0.00|881.79|1711.71|1711.71|-1474.20| +2451032|60808|12260|26878|484227|7176|24972|4|25|351|54|6.36|11.70|9.82|0.00|530.28|343.44|631.80|0.00|0.00|530.28|530.28|186.84| +2451032|60808|1666|26878|484227|7176|24972|4|174|351|9|26.03|39.30|6.68|0.00|60.12|234.27|353.70|0.60|0.00|60.12|60.72|-174.15| +2451032|60808|13442|26878|484227|7176|24972|4|35|351|96|49.09|61.36|56.45|3305.71|5419.20|4712.64|5890.56|63.40|3305.71|2113.49|2176.89|-2599.15| +2451032|60808|820|26878|484227|7176|24972|4|289|351|36|88.19|110.23|36.37|65.46|1309.32|3174.84|3968.28|62.19|65.46|1243.86|1306.05|-1930.98| +2451032|60808|6583|26878|484227|7176|24972|4|116|351|14|64.75|105.54|43.27|424.04|605.78|906.50|1477.56|10.90|424.04|181.74|192.64|-724.76| +2451032|60808|3176|26878|484227|7176|24972|4|77|351|100|39.07|61.73|16.04|0.00|1604.00|3907.00|6173.00|80.20|0.00|1604.00|1684.20|-2303.00| +2451032|60808|2371|26878|484227|7176|24972|4|93|351|90|6.47|12.74|9.42|0.00|847.80|582.30|1146.60|76.30|0.00|847.80|924.10|265.50| +2451032|60808|10423|26878|484227|7176|24972|4|199|351|79|84.04|86.56|34.62|1367.49|2734.98|6639.16|6838.24|68.37|1367.49|1367.49|1435.86|-5271.67| +2451032|60808|2332|26878|484227|7176|24972|4|115|351|10|38.80|49.27|23.64|170.20|236.40|388.00|492.70|4.63|170.20|66.20|70.83|-321.80| +2451032|60808|7429|26878|484227|7176|24972|4|64|351|52|85.92|171.84|56.70|0.00|2948.40|4467.84|8935.68|147.42|0.00|2948.40|3095.82|-1519.44| +||8182|26878||7176||4||351||72.98||87.96|0.00|||429.12|28.14|0.00|351.84||59.92| +2451032|60808|2452|26878|484227|7176|24972|4|57|351|11|60.14|117.27|28.14|0.00|309.54|661.54|1289.97|21.66|0.00|309.54|331.20|-352.00| +2451032|60808|13645||484227||24972|4|273|351|58||||0.00|600.88|1105.48|1624.58||0.00||642.94|-504.60| +2452039|61660|14017|1484|1337582|2833|6531|1|124|352|37|3.25|3.25|2.99|0.00|110.63|120.25|120.25|4.42|0.00|110.63|115.05|-9.62| +2452039|61660|2693|1484|1337582|2833|6531|1|55|352|1|17.82|19.42|13.20|0.00|13.20|17.82|19.42|0.66|0.00|13.20|13.86|-4.62| +2452039|61660|8329|1484|1337582|2833|6531|1|232|352|63|16.51|21.62|11.89|0.00|749.07|1040.13|1362.06|22.47|0.00|749.07|771.54|-291.06| +2452039|61660|7439|1484|1337582|2833|6531|1|99|352|21|14.79|17.60|3.69|0.00|77.49|310.59|369.60|4.64|0.00|77.49|82.13|-233.10| +2452039|61660|13887|1484|1337582|2833|6531|1|189|352|4|22.84|44.53|0.44|0.00|1.76|91.36|178.12|0.14|0.00|1.76|1.90|-89.60| +2452039|61660|4699|1484|1337582|2833|6531|1|250|352|93|95.99|115.18|47.22|0.00|4391.46|8927.07|10711.74|0.00|0.00|4391.46|4391.46|-4535.61| +2452039|61660|5205|1484|1337582|2833|6531|1|193|352|88|46.15|60.91|43.85|0.00|3858.80|4061.20|5360.08|38.58|0.00|3858.80|3897.38|-202.40| +2452039|61660|12649|1484|1337582|2833|6531|1|76|352|67|75.49|77.75|33.43|918.32|2239.81|5057.83|5209.25|39.64|918.32|1321.49|1361.13|-3736.34| +2452039|61660|509|1484|1337582|2833|6531|1|199|352|90|35.06|53.29|38.90|0.00|3501.00|3155.40|4796.10|105.03|0.00|3501.00|3606.03|345.60| +2452039|61660|5417|1484|1337582|2833|6531|1|77|352|21|19.50|24.96|14.22|0.00|298.62|409.50|524.16|23.88|0.00|298.62|322.50|-110.88| +2452039|61660|9259|1484|1337582|2833|6531|1|275|352|42|10.76|15.27|7.63|0.00|320.46|451.92|641.34|3.20|0.00|320.46|323.66|-131.46| +2452039|61660|7535|1484|1337582|2833|6531|1|196|352|95|34.48|56.89|29.01|0.00|2755.95|3275.60|5404.55|0.00|0.00|2755.95|2755.95|-519.65| +2452039|61660|2773|1484|1337582|2833|6531|1|241|352|46|51.76|70.39|49.27|0.00|2266.42|2380.96|3237.94|135.98|0.00|2266.42|2402.40|-114.54| +2451039|50242|1525|9799|704837|||8||353|||115.26||0.00|6186.40|||371.18|0.00|||-233.20| +2451039|50242|4555|9799|704837|5475|29713|8|112|353|6|95.54|138.53|22.16|0.00|132.96|573.24|831.18|6.64|0.00|132.96|139.60|-440.28| +2451039|50242|15020|9799|704837|5475|29713|8|4|353|68|52.24|70.52|48.65|0.00|3308.20|3552.32|4795.36|132.32|0.00|3308.20|3440.52|-244.12| +2451039|50242|7868|9799|704837|5475|29713|8|165|353|43|97.68|153.35|62.87|0.00|2703.41|4200.24|6594.05|27.03|0.00|2703.41|2730.44|-1496.83| +2451039|50242|4489|9799|704837|5475|29713|8|257|353|40|41.96|70.49|21.85|0.00|874.00|1678.40|2819.60|43.70|0.00|874.00|917.70|-804.40| +2451039|50242|8425|9799|704837|5475|29713|8|77|353|78|16.21|27.23|4.62|0.00|360.36|1264.38|2123.94|14.41|0.00|360.36|374.77|-904.02| +2451039|50242|3694|9799|704837|5475|29713|8|282|353|23|29.38|56.11|44.88|0.00|1032.24|675.74|1290.53|20.64|0.00|1032.24|1052.88|356.50| +2451039|50242|7130|9799|704837|5475|29713|8|5|353|48|76.54|93.37|1.86|0.00|89.28|3673.92|4481.76|5.35|0.00|89.28|94.63|-3584.64| +2451039|50242|2546|9799|704837|5475|29713|8|31|353|48|72.54|102.28|70.57|0.00|3387.36|3481.92|4909.44|203.24|0.00|3387.36|3590.60|-94.56| +2451039|50242|10747|9799|704837|5475|29713|8|258|353|24|70.19|105.98|3.17|0.00|76.08|1684.56|2543.52|5.32|0.00|76.08|81.40|-1608.48| +2451039|50242|10498|9799|704837|5475|29713|8|229|353|78|73.88|81.26|58.50|0.00|4563.00|5762.64|6338.28|0.00|0.00|4563.00|4563.00|-1199.64| +2451039|50242|10678|9799|704837|5475|29713|8|192|353|4|62.56|77.57|26.37|0.00|105.48|250.24|310.28|2.10|0.00|105.48|107.58|-144.76| +2451074|51295|13718|17299|432797|4648|10146|7|10|354|70|13.07|24.57|3.19|0.00|223.30|914.90|1719.90|0.00|0.00|223.30|223.30|-691.60| +2451074|51295|16237|17299|432797|4648|10146|7|13|354|46|49.10|53.51|39.59|0.00|1821.14|2258.60|2461.46|0.00|0.00|1821.14|1821.14|-437.46| +2451074|51295|17122|17299|432797|4648|10146|7|33|354|61|49.87|79.29|7.92|0.00|483.12|3042.07|4836.69|0.00|0.00|483.12|483.12|-2558.95| +2451074|51295|2824|17299|432797|4648|10146|7|54|354|65|95.17|100.88|82.72|0.00|5376.80|6186.05|6557.20|215.07|0.00|5376.80|5591.87|-809.25| +2451074|51295|6986|17299|432797|4648|10146|7|105|354|9|8.74|17.30|8.82|53.18|79.38|78.66|155.70|0.26|53.18|26.20|26.46|-52.46| +2451074|51295|14714|17299|432797|4648|10146|7|48|354|16|72.49|144.98|13.04|0.00|208.64|1159.84|2319.68|0.00|0.00|208.64|208.64|-951.20| +2451074|51295|8002|17299|432797|4648|10146|7|198|354|41|71.29|139.01|79.23|0.00|3248.43|2922.89|5699.41|259.87|0.00|3248.43|3508.30|325.54| +2451074|51295|2048|17299|432797|4648|10146|7|182|354|18|1.50|2.04|1.12|3.42|20.16|27.00|36.72|1.00|3.42|16.74|17.74|-10.26| +2451074|51295|15872|17299|432797|4648|10146|7|108|354|25|72.06|88.63|13.29|0.00|332.25|1801.50|2215.75|0.00|0.00|332.25|332.25|-1469.25| +2451074|51295|2666|17299|432797|4648|10146|7|79|354|34|18.57|30.08|5.11|0.00|173.74|631.38|1022.72|15.63|0.00|173.74|189.37|-457.64| +2451074|51295|15998|17299|432797|4648|10146|7|58|354|41|12.43|17.27|12.08|0.00|495.28|509.63|708.07|29.71|0.00|495.28|524.99|-14.35| +2451074|51295|15728|17299|432797|4648|10146|7|48|354|21|71.86|141.56|21.23|0.00|445.83|1509.06|2972.76|40.12|0.00|445.83|485.95|-1063.23| +2451074|51295|3902|17299|432797|4648|10146|7|149|354|55|19.63|26.10|5.74|0.00|315.70|1079.65|1435.50|0.00|0.00|315.70|315.70|-763.95| +2451074|51295|14204|17299|432797|4648|10146|7|264|354|68|24.19|43.54|33.52|0.00|2279.36|1644.92|2960.72|136.76|0.00|2279.36|2416.12|634.44| +2451074|51295|5198|17299|432797|4648|10146|7|18|354|84|80.54|98.25|77.61|0.00|6519.24|6765.36|8253.00|195.57|0.00|6519.24|6714.81|-246.12| +2451074|51295|5089|17299|432797|4648|10146|7|11|354|8|58.17|73.87|34.71|41.65|277.68|465.36|590.96|16.52|41.65|236.03|252.55|-229.33| +2452637|62653|10287|9321|154938|6955|25372|8|16|355|48|5.85|9.41|6.21|0.00|298.08|280.80|451.68|2.98|0.00|298.08|301.06|17.28| +2452637|62653|12607|9321|154938|6955|25372|8|170|355|33|42.70|73.01|43.80|1069.59|1445.40|1409.10|2409.33|18.79|1069.59|375.81|394.60|-1033.29| +2452637|62653|3429|9321|154938|6955|25372|8|30|355|33|37.77|40.03|8.80|0.00|290.40|1246.41|1320.99|11.61|0.00|290.40|302.01|-956.01| +2452637|62653|9024|9321|154938|6955|25372|8|272|355|95|45.68|84.50|32.95|0.00|3130.25|4339.60|8027.50|219.11|0.00|3130.25|3349.36|-1209.35| +2452637|62653|6495|9321|154938|6955|25372|8|168|355|22|91.40|169.09|47.34|0.00|1041.48|2010.80|3719.98|0.00|0.00|1041.48|1041.48|-969.32| +2452637|62653|11607|9321|154938|6955|25372|8|49|355|87|5.22|5.68|1.64|0.00|142.68|454.14|494.16|9.98|0.00|142.68|152.66|-311.46| +2452637|62653|17821|||6955|25372|8|264|355|88||84.71|||||7454.48|||2459.60||| +2452637|62653|1344|9321|154938|6955|25372|8|144|355|48|84.36|100.38|87.33|0.00|4191.84|4049.28|4818.24|0.00|0.00|4191.84|4191.84|142.56| +2452637|62653|10941|9321|154938|6955|25372|8|106|355|89|70.53|83.22|61.58|0.00|5480.62|6277.17|7406.58|493.25|0.00|5480.62|5973.87|-796.55| +2452637|62653|16134|9321|154938|6955|25372|8|32|355|47|3.42|4.71|2.49|80.75|117.03|160.74|221.37|1.45|80.75|36.28|37.73|-124.46| +2452273|40715|14799|4305|458605|105|20052|7|15|356|71|8.10|10.53|6.31|0.00|448.01|575.10|747.63|0.00|0.00|448.01|448.01|-127.09| +2452273|40715|15719|4305|458605|105|20052|7|299|356|81|81.06|153.20|72.00|0.00|5832.00|6565.86|12409.20|233.28|0.00|5832.00|6065.28|-733.86| +2452273|40715|5535|4305|458605|105|20052|7|93|356|19|2.43|2.81|0.53|0.00|10.07|46.17|53.39|0.30|0.00|10.07|10.37|-36.10| +2452273|40715|5803|4305|458605|105|20052|7|253|356|37|35.67|65.63|32.81|0.00|1213.97|1319.79|2428.31|60.69|0.00|1213.97|1274.66|-105.82| +2452273|40715|8357|4305|458605|105|20052|7|157|356|17|77.91|84.92|16.13|0.00|274.21|1324.47|1443.64|13.71|0.00|274.21|287.92|-1050.26| +2452273|40715|13683|4305|458605|105|20052|7|115|356|60|34.57|40.10|15.23|0.00|913.80|2074.20|2406.00|18.27|0.00|913.80|932.07|-1160.40| +2452273|40715|9807|4305|458605|105|20052|7|193|356|4|7.88|10.32|2.27|0.00|9.08|31.52|41.28|0.27|0.00|9.08|9.35|-22.44| +2452273|40715|17589|4305|458605|105|20052|7|204|356|64|39.45|48.52|38.33|0.00|2453.12|2524.80|3105.28|49.06|0.00|2453.12|2502.18|-71.68| +2452359|42492|12570|56418|1210565|6312|26059|7|74|357|49|31.21|57.42|13.20|0.00|646.80|1529.29|2813.58|25.87|0.00|646.80|672.67|-882.49| +2452359|42492|17073|56418|1210565|6312|26059|7|99|357|69|67.37|88.25|2.64|0.00|182.16|4648.53|6089.25|16.39|0.00|182.16|198.55|-4466.37| +2452359|42492|10734|56418|1210565|6312|26059|7|241|357|49|60.10|87.74|36.85|1805.65|1805.65|2944.90|4299.26|0.00|1805.65|0.00|0.00|-2944.90| +2452359|42492|13458|56418|1210565|6312|26059|7|13|357|34|78.70|126.70|50.68|1688.65|1723.12|2675.80|4307.80|0.34|1688.65|34.47|34.81|-2641.33| +2452359|42492|7449|56418|1210565|6312|26059|7|174|357|31|66.36|108.16|42.18|0.00|1307.58|2057.16|3352.96|91.53|0.00|1307.58|1399.11|-749.58| +2452359|42492|2713|56418|1210565|6312|26059|7|286|357|14|93.67|170.47|73.30|0.00|1026.20|1311.38|2386.58|20.52|0.00|1026.20|1046.72|-285.18| +2452359|42492|1399|56418|1210565|6312|26059|7|256|357|30|7.56|12.85|9.25|0.00|277.50|226.80|385.50|2.77|0.00|277.50|280.27|50.70| +2452359|42492|7339|56418|1210565|6312|26059|7|59|357|37|18.16|32.50|11.37|0.00|420.69|671.92|1202.50|8.41|0.00|420.69|429.10|-251.23| +2452359|42492|8688|56418|1210565||26059|7|114|357||27.99||||1914.78||3682.92|||||| +2452359|42492|12439|56418|1210565|6312|26059|7|38|357|24|21.30|34.08|6.13|0.00|147.12|511.20|817.92|7.35|0.00|147.12|154.47|-364.08| +2452359|42492|9222|56418|1210565|6312|26059|7|292|357|56|76.69|151.84|28.84|0.00|1615.04|4294.64|8503.04|48.45|0.00|1615.04|1663.49|-2679.60| +2452359|42492|993|56418|1210565|6312|26059|7|104|357|67|5.22|9.44|5.75|0.00|385.25|349.74|632.48|26.96|0.00|385.25|412.21|35.51| +2452359|42492|7488|56418|1210565|6312|26059|7|54|357|90|31.38|51.14|2.04|179.92|183.60|2824.20|4602.60|0.33|179.92|3.68|4.01|-2820.52| +2452359|42492|6289|56418|1210565|6312|26059|7|80|357|64|20.25|27.13|21.16|0.00|1354.24|1296.00|1736.32|94.79|0.00|1354.24|1449.03|58.24| +2452359|42492|3870|56418|1210565|6312|26059|7|15|357|70|81.74|140.59|119.50|4935.35|8365.00|5721.80|9841.30|102.88|4935.35|3429.65|3532.53|-2292.15| +2451923|47503|8287|27487|1707076|2218|33807|10|20|358|98|90.47|161.94|87.44|0.00|8569.12|8866.06|15870.12|771.22|0.00|8569.12|9340.34|-296.94| +2451923||11511|27487|1707076|2218||10||358|||90.94|23.64|||||||141.84|153.18|-221.94| +2451923|47503|5389|27487|1707076|2218|33807|10|95|358|37|40.46|64.33|15.43|62.80|570.91|1497.02|2380.21|35.56|62.80|508.11|543.67|-988.91| +2451923|47503|5875|27487|1707076|2218|33807|10|27|358|30|99.18|161.66|30.71|322.45|921.30|2975.40|4849.80|47.90|322.45|598.85|646.75|-2376.55| +|47503|4569|27487||2218||10||358||75.44|||0.00||4526.40|||0.00|537.60||-3988.80| +2451923|47503|2257|27487|1707076|2218|33807|10|260|358|5|63.62|75.07|51.04|63.80|255.20|318.10|375.35|1.91|63.80|191.40|193.31|-126.70| +||10665|27487|||33807||224|358||14.70|28.07|||||1291.22|24.51||490.36|514.87|-185.84| +2451923|47503|15445|27487|1707076|2218|33807|10|251|358|13|21.61|23.98|15.82|0.00|205.66|280.93|311.74|14.39|0.00|205.66|220.05|-75.27| +2451923|47503|4981|27487|1707076|2218|33807|10|4|358|75|98.49|187.13|24.32|0.00|1824.00|7386.75|14034.75|36.48|0.00|1824.00|1860.48|-5562.75| +2451923|47503|17375|27487|1707076|2218|33807|10|166|358|35|43.13|83.24|29.96|0.00|1048.60|1509.55|2913.40|62.91|0.00|1048.60|1111.51|-460.95| +2451923|47503|16509|27487|1707076|2218|33807|10|38|358|53|67.27|90.81|69.92|0.00|3705.76|3565.31|4812.93|0.00|0.00|3705.76|3705.76|140.45| +2451923|47503|11827|27487|1707076|2218|33807|10|128|358|67|69.25|126.03|10.08|0.00|675.36|4639.75|8444.01|60.78|0.00|675.36|736.14|-3964.39| +2451923|47503|8895|27487|1707076|2218|33807|10|112|358|72|42.67|57.17|13.72|0.00|987.84|3072.24|4116.24|49.39|0.00|987.84|1037.23|-2084.40| +|47503|10353|||2218|33807|10||358|59||81.33|45.54|0.00|||4798.47||0.00|2686.86||-512.12| +2450835|71743|13915|24866|1832958|2650|41142|10|49|359|93|73.23|117.16|99.58|0.00|9260.94|6810.39|10895.88|277.82|0.00|9260.94|9538.76|2450.55| +2450835|71743|14144|24866|1832958|2650|41142|10|173|359|53|99.87|152.80|39.72|0.00|2105.16|5293.11|8098.40|21.05|0.00|2105.16|2126.21|-3187.95| +2450835|71743|8404|24866|1832958|2650|41142|10|60|359|12|20.89|26.32|6.31|0.00|75.72|250.68|315.84|4.54|0.00|75.72|80.26|-174.96| +2450835|71743|5498|24866|1832958|2650|41142|10|135|359|21|83.09|145.40|50.89|0.00|1068.69|1744.89|3053.40|53.43|0.00|1068.69|1122.12|-676.20| +2450835|71743|4124|24866|1832958|2650|41142|10|300|359|10|67.78|124.03|101.70|162.72|1017.00|677.80|1240.30|51.25|162.72|854.28|905.53|176.48| +2450835|71743|16910|24866|1832958|2650|41142|10|87|359|71|23.08|41.54|17.86|0.00|1268.06|1638.68|2949.34|12.68|0.00|1268.06|1280.74|-370.62| +2450835|71743|11773|24866|1832958|2650|41142|10|248|359|27|7.63|15.10|10.57|0.00|285.39|206.01|407.70|11.41|0.00|285.39|296.80|79.38| +2450835|71743|17158|24866|1832958|2650|41142|10|75|359|95|84.85|124.72|68.59|0.00|6516.05|8060.75|11848.40|195.48|0.00|6516.05|6711.53|-1544.70| +2450835|71743|9202|24866|1832958|2650|41142|10|231|359|45|54.61|106.48|9.58|275.90|431.10|2457.45|4791.60|9.31|275.90|155.20|164.51|-2302.25| +2452099|37881|2801|34066|1841797|1081|43165|10|157|360|100|3.67|7.00|2.24|0.00|224.00|367.00|700.00|2.24|0.00|224.00|226.24|-143.00| +2452099|37881|14249|34066|1841797|1081|43165|10|280|360|44|62.94|68.60|63.11|0.00|2776.84|2769.36|3018.40|249.91|0.00|2776.84|3026.75|7.48| +2452099|37881|13505|34066|1841797|1081|43165|10|203|360|51|20.31|21.32|0.63|0.00|32.13|1035.81|1087.32|2.89|0.00|32.13|35.02|-1003.68| +2452099|37881|17855|34066|1841797|1081|43165|10|74|360|42|9.95|19.00|18.81|0.00|790.02|417.90|798.00|47.40|0.00|790.02|837.42|372.12| +2452099|37881|3017|34066|1841797|1081|43165|10|119|360|100|92.51|162.81|148.15|9926.05|14815.00|9251.00|16281.00|97.77|9926.05|4888.95|4986.72|-4362.05| +2452099|37881|4015|34066|1841797|1081|43165|10|5|360|73|60.57|66.02|41.59|971.54|3036.07|4421.61|4819.46|0.00|971.54|2064.53|2064.53|-2357.08| +2452099|37881|4549|34066|1841797|1081|43165|10|67|360|45|63.12|98.46|42.33|1371.49|1904.85|2840.40|4430.70|32.00|1371.49|533.36|565.36|-2307.04| +2452099|37881|9249|34066|1841797|1081|43165|10|274|360|44|13.04|20.21|18.59|114.51|817.96|573.76|889.24|42.20|114.51|703.45|745.65|129.69| +2452099|37881|15357|34066|1841797|1081|43165|10|217|360|88|23.90|46.12|11.99|0.00|1055.12|2103.20|4058.56|0.00|0.00|1055.12|1055.12|-1048.08| +2452099|37881|6625|34066|1841797|1081|43165|10|111|360|22|74.62|102.22|101.19|0.00|2226.18|1641.64|2248.84|89.04|0.00|2226.18|2315.22|584.54| +2452099|37881|13345|34066|1841797|1081|43165|10|57|360|73|20.57|36.61|9.15|80.15|667.95|1501.61|2672.53|17.63|80.15|587.80|605.43|-913.81| +2452099|37881|16907|34066|1841797|1081|43165|10|160|360|17|71.41|128.53|1.28|0.00|21.76|1213.97|2185.01|0.43|0.00|21.76|22.19|-1192.21| +2452099|37881|14951|34066|1841797|1081|43165|10|223|360|10|93.16|127.62|74.01|0.00|740.10|931.60|1276.20|66.60|0.00|740.10|806.70|-191.50| +2452099|37881|14143|34066|1841797|1081|43165|10|125|360|55|95.77|124.50|51.04|589.51|2807.20|5267.35|6847.50|199.59|589.51|2217.69|2417.28|-3049.66| +2452099|37881|14119|34066|1841797|1081|43165|10|190|360|7|83.25|100.73|94.68|503.69|662.76|582.75|705.11|9.54|503.69|159.07|168.61|-423.68| +2451505|39146|11143|13768|1779171|4221|28024|10|119|361|40|10.30|18.54|4.63|122.23|185.20|412.00|741.60|3.14|122.23|62.97|66.11|-349.03| +2451505|39146|14041|13768|1779171|4221|28024|10|147|361|30|51.96|54.55|15.81|0.00|474.30|1558.80|1636.50|37.94|0.00|474.30|512.24|-1084.50| +2451505|39146|13291|13768|1779171|4221|28024|10|232|361|91|28.21|53.59|22.50|0.00|2047.50|2567.11|4876.69|81.90|0.00|2047.50|2129.40|-519.61| +2451505|39146|12331|13768|1779171|4221|28024|10|294|361|37|67.46|105.91|20.12|0.00|744.44|2496.02|3918.67|44.66|0.00|744.44|789.10|-1751.58| +2451505|39146|650|13768|1779171|4221|28024|10|128|361|35|79.00|143.78|125.08|0.00|4377.80|2765.00|5032.30|43.77|0.00|4377.80|4421.57|1612.80| +2451505|39146|9224|13768|1779171|4221|28024|10|81|361|17|74.06|128.12|71.74|0.00|1219.58|1259.02|2178.04|36.58|0.00|1219.58|1256.16|-39.44| +2451505|39146|15919|13768|1779171|4221|28024|10|54|361|99|19.50|29.83|22.37|0.00|2214.63|1930.50|2953.17|110.73|0.00|2214.63|2325.36|284.13| +2451505|39146|12718|13768|1779171|4221|28024|10|86|361|88|46.58|50.77|37.06|0.00|3261.28|4099.04|4467.76|130.45|0.00|3261.28|3391.73|-837.76| +2451505|39146|4663|13768|1779171|4221|28024|10|239|361|59|84.08|126.96|66.01|0.00|3894.59|4960.72|7490.64|350.51|0.00|3894.59|4245.10|-1066.13| +2451505|39146|9763|13768|1779171|4221|28024|10|241|361|99|88.27|136.81|110.81|0.00|10970.19|8738.73|13544.19|548.50|0.00|10970.19|11518.69|2231.46| +2452275|66157|8783|24378|399005|5632|38306|4|9|362|78|75.34|143.89|30.21|0.00|2356.38|5876.52|11223.42|141.38|0.00|2356.38|2497.76|-3520.14| +2452275|66157|17441|24378|399005|5632|38306|4|46|362|86|76.10|126.32|51.79|0.00|4453.94|6544.60|10863.52|0.00|0.00|4453.94|4453.94|-2090.66| +2452275|66157|11461|24378|399005|5632|38306|4|226|362|63|38.15|50.35|18.62|0.00|1173.06|2403.45|3172.05|0.00|0.00|1173.06|1173.06|-1230.39| +2452275|66157|6541|24378|399005|5632|38306|4|123|362|56|58.71|93.93|13.15|301.92|736.40|3287.76|5260.08|21.72|301.92|434.48|456.20|-2853.28| +2452275|66157|17205|24378|399005|5632|38306|4|249|362|91|61.98|96.68|95.71|0.00|8709.61|5640.18|8797.88|261.28|0.00|8709.61|8970.89|3069.43| +2452275|66157|11519|24378|399005|5632|38306|4|132|362|58|81.53|93.75|24.37|56.53|1413.46|4728.74|5437.50|81.41|56.53|1356.93|1438.34|-3371.81| +2452275|66157|9313|24378|399005|5632|38306|4|159|362|15|4.34|5.20|0.10|0.00|1.50|65.10|78.00|0.01|0.00|1.50|1.51|-63.60| +2452275|66157|11987|24378|399005|5632|38306|4|90|362|68|40.11|76.61|3.06|187.27|208.08|2727.48|5209.48|1.66|187.27|20.81|22.47|-2706.67| +2452120|64600|9417|51844|1279476|1955|25234|7|246|363|21|16.79|17.79|7.82|0.00|164.22|352.59|373.59|8.21|0.00|164.22|172.43|-188.37| +2452120|64600|1983|51844|1279476|1955|25234|7|42|363|31|7.46|8.05|3.30|0.00|102.30|231.26|249.55|3.06|0.00|102.30|105.36|-128.96| +2452120|64600|6797|51844|1279476|1955|25234|7|105|363|31|92.51|150.79|125.15|0.00|3879.65|2867.81|4674.49|310.37|0.00|3879.65|4190.02|1011.84| +2452120|64600|14659|51844|1279476|1955|25234|7|33|363|77|70.11|93.24|76.45|0.00|5886.65|5398.47|7179.48|58.86|0.00|5886.65|5945.51|488.18| +2452120|64600|12685|51844|1279476|1955|25234|7|55|363|62|65.50|121.83|1.21|0.00|75.02|4061.00|7553.46|0.75|0.00|75.02|75.77|-3985.98| +2452120|64600|3723|51844|1279476|1955|25234|7|54|363|24|75.98|83.57|0.00|0.00|0.00|1823.52|2005.68|0.00|0.00|0.00|0.00|-1823.52| +2452120|64600|11057|51844|1279476|1955|25234|7|168|363|61|38.63|48.28|4.82|0.00|294.02|2356.43|2945.08|8.82|0.00|294.02|302.84|-2062.41| +2452120|64600|5387|51844|1279476|1955|25234|7|246|363|100|98.74|163.90|62.28|0.00|6228.00|9874.00|16390.00|311.40|0.00|6228.00|6539.40|-3646.00| +2452120|64600|14987|51844|1279476|1955|25234|7|56|363|39|49.75|74.12|42.98|0.00|1676.22|1940.25|2890.68|67.04|0.00|1676.22|1743.26|-264.03| +2452120|64600|2879|51844|1279476|1955|25234|7|112|363|33|1.40|1.65|0.92|0.00|30.36|46.20|54.45|2.42|0.00|30.36|32.78|-15.84| +2452120|64600|12157|51844|1279476|1955|25234|7|252|363|29|38.82|41.92|34.79|433.83|1008.91|1125.78|1215.68|5.75|433.83|575.08|580.83|-550.70| +2452120|64600|1295|51844|1279476|1955|25234|7|139|363|91|54.35|60.87|14.60|0.00|1328.60|4945.85|5539.17|26.57|0.00|1328.60|1355.17|-3617.25| +2452120|64600|8097|51844|1279476|1955|25234|7|289|363|84|27.32|50.81|11.68|0.00|981.12|2294.88|4268.04|9.81|0.00|981.12|990.93|-1313.76| +2452120|64600|13395|51844|1279476|1955|25234|7|265|363|23|5.38|7.42|5.93|0.00|136.39|123.74|170.66|0.00|0.00|136.39|136.39|12.65| +2452633|38201|6114|83959|1770540|6274|30055|7|88|364|70|11.90|13.56|1.62|97.52|113.40|833.00|949.20|0.15|97.52|15.88|16.03|-817.12| +2452633|38201|9861|83959|1770540|6274|30055|7|257|364|81|32.02|63.39|55.14|0.00|4466.34|2593.62|5134.59|223.31|0.00|4466.34|4689.65|1872.72| +2452633||10753|83959|1770540||30055||112|364||82.35|129.28|69.81|||741.15|1163.52|||628.29||-112.86| +2452633|38201|12789|83959|1770540|6274|30055|7|39|364|90|46.60|56.38|19.16|327.63|1724.40|4194.00|5074.20|55.87|327.63|1396.77|1452.64|-2797.23| +2452633|38201|5917|83959|1770540|6274|30055|7|57|364|18|55.73|96.41|59.77|0.00|1075.86|1003.14|1735.38|75.31|0.00|1075.86|1151.17|72.72| +||13905|83959|1770540||30055|||364|||||0.00|286.44||363.44||0.00|286.44|300.76|-41.58| +2452633|38201|15588|83959|1770540|6274|30055|7|261|364|62|22.22|29.55|9.16|215.80|567.92|1377.64|1832.10|10.56|215.80|352.12|362.68|-1025.52| +2452633|38201|11742|83959|1770540|6274|30055|7|184|364|50|15.95|30.46|10.35|0.00|517.50|797.50|1523.00|41.40|0.00|517.50|558.90|-280.00| +2452633|38201|1536|83959|1770540|6274|30055|7|162|364|97|50.60|86.02|6.02|0.00|583.94|4908.20|8343.94|52.55|0.00|583.94|636.49|-4324.26| +2452633|38201|16765|83959|1770540|6274|30055|7|111|364|37|11.81|21.02|19.12|516.43|707.44|436.97|777.74|15.28|516.43|191.01|206.29|-245.96| +2452633|38201|12901|83959|1770540|6274|30055|7|183|364|61|93.23|125.86|120.82|0.00|7370.02|5687.03|7677.46|221.10|0.00|7370.02|7591.12|1682.99| +2452633|38201|6672|83959|1770540|6274|30055|7|17|364|88|96.19|178.91|135.97|0.00|11965.36|8464.72|15744.08|119.65|0.00|11965.36|12085.01|3500.64| +2452633|38201|13086|83959|1770540|6274|30055|7|80|364|1|29.32|30.78|28.93|0.00|28.93|29.32|30.78|1.44|0.00|28.93|30.37|-0.39| +2452633||117|||6274||7||364||52.62|63.14||14.06|39.06|1631.22||2.00|14.06||27.00|-1606.22| +||12822|83959|1770540||||231|364|90||50.57|||181.80|4376.70||||181.80|194.52|| +2452633|38201|13596|83959|1770540|6274|30055|7|179|364|10|90.13|100.94|23.21|0.00|232.10|901.30|1009.40|16.24|0.00|232.10|248.34|-669.20| +2452553|53984|10338|29988|535981|5974|29167|10|167|365|25|74.75|80.73|25.83|0.00|645.75|1868.75|2018.25|45.20|0.00|645.75|690.95|-1223.00| +2452553|53984|17557|29988|535981|5974|29167|10|172|365|86|87.28|157.97|36.33|0.00|3124.38|7506.08|13585.42|93.73|0.00|3124.38|3218.11|-4381.70| +2452553|53984|14719|29988|535981|5974|29167|10|160|365|54|45.80|84.73|46.60|0.00|2516.40|2473.20|4575.42|0.00|0.00|2516.40|2516.40|43.20| +2452553|53984|8496|29988|535981|5974|29167|10|229|365|76|72.02|121.71|92.49|0.00|7029.24|5473.52|9249.96|70.29|0.00|7029.24|7099.53|1555.72| +2452553|53984|2637|29988|535981|5974|29167|10|281|365|85|97.43|145.17|136.45|0.00|11598.25|8281.55|12339.45|347.94|0.00|11598.25|11946.19|3316.70| +2452553|53984|11160|29988||5974|29167|||365|73||||0.00||||375.45|0.00|||194.91| +2452553|53984|17910|29988|535981|5974|29167|10|66|365|3|98.84|184.83|177.43|0.00|532.29|296.52|554.49|21.29|0.00|532.29|553.58|235.77| +2452553|53984|15414|29988|535981|5974|29167|10|278|365|10|81.96|118.02|21.24|0.00|212.40|819.60|1180.20|16.99|0.00|212.40|229.39|-607.20| +2452553|53984|5413|29988|535981|5974|29167|10|170|365|24|59.76|79.48|77.89|0.00|1869.36|1434.24|1907.52|112.16|0.00|1869.36|1981.52|435.12| +2452553|53984|6673|29988|535981|5974|29167|10|294|365|62|43.83|47.33|34.07|0.00|2112.34|2717.46|2934.46|190.11|0.00|2112.34|2302.45|-605.12| +2452553|53984|14679|29988|535981|5974|29167|10|92|365|79|15.78|28.87|4.61|14.56|364.19|1246.62|2280.73|10.48|14.56|349.63|360.11|-896.99| +2452553|53984|13638|29988|535981|5974|29167|10|115|365|16|89.80|149.96|38.98|0.00|623.68|1436.80|2399.36|12.47|0.00|623.68|636.15|-813.12| +2452553|53984|17988|29988|535981|5974|29167|10|137|365|12|64.48|78.02|6.24|38.18|74.88|773.76|936.24|0.00|38.18|36.70|36.70|-737.06| +2452553|53984|15169|29988|535981|5974|29167|10|219|365|44|6.79|9.70|3.58|56.70|157.52|298.76|426.80|7.05|56.70|100.82|107.87|-197.94| +2452553|53984|11337|29988|535981|5974|29167|10|96|365|37|32.82|47.26|0.94|0.00|34.78|1214.34|1748.62|0.69|0.00|34.78|35.47|-1179.56| +2451024|43995|15032|15564|237338|3112|13686|7|5|366|10|94.62|139.09|47.29|0.00|472.90|946.20|1390.90|9.45|0.00|472.90|482.35|-473.30| +2451024|43995|1324|15564|237338|3112|13686|7|37|366|47|17.82|20.84|0.20|0.00|9.40|837.54|979.48|0.37|0.00|9.40|9.77|-828.14| +2451024|43995|5497|15564|237338|3112|13686|7|93|366|66|3.80|4.21|0.04|0.00|2.64|250.80|277.86|0.13|0.00|2.64|2.77|-248.16| +2451024|43995|7972|15564|237338|3112|13686|7|85|366|29|97.83|134.02|131.33|0.00|3808.57|2837.07|3886.58|38.08|0.00|3808.57|3846.65|971.50| +2451024|43995|3110|15564|237338|3112|13686|7|72|366|29|20.04|29.25|18.72|0.00|542.88|581.16|848.25|16.28|0.00|542.88|559.16|-38.28| +2451024|43995|15788|15564|237338|3112|13686|7|44|366|75|88.43|145.90|67.11|0.00|5033.25|6632.25|10942.50|100.66|0.00|5033.25|5133.91|-1599.00| +||781|||3112||||366||83.21||73.03|0.00|6864.82|7821.74|12045.16||0.00||7070.76|-956.92| +2451024|43995|15790|15564|237338|3112|13686|7|229|366|1|68.28|101.73|4.06|0.00|4.06|68.28|101.73|0.28|0.00|4.06|4.34|-64.22| +2451024|43995|4600|15564|237338|3112|13686|7|236|366|70|83.92|107.41|52.63|0.00|3684.10|5874.40|7518.70|147.36|0.00|3684.10|3831.46|-2190.30| +2451024|43995|14978|15564|237338|3112|13686|7|47|366|64|29.33|41.35|0.82|0.00|52.48|1877.12|2646.40|4.19|0.00|52.48|56.67|-1824.64| +2451024|43995|127|15564|237338|3112|13686|7|54|366|46|62.06|68.26|27.98|0.00|1287.08|2854.76|3139.96|0.00|0.00|1287.08|1287.08|-1567.68| +||17725|||3112|13686|||366||6.12|7.77|1.08|27.21|60.48||435.12|1.33|27.21|33.27||| +2451024|43995|4256|15564|237338|3112|13686|7|14|366|38|48.27|73.37|13.20|0.00|501.60|1834.26|2788.06|25.08|0.00|501.60|526.68|-1332.66| +|43995|8785|||3112|13686|||366||1.05|1.98||0.00|||||0.00|39.68|40.87|6.08| +2451024||1207|15564|237338|||7||366||||19.57|0.00||6449.00|8512.00||0.00|1957.00||-4492.00| +2451024|43995|9698|15564|237338|3112|13686|7|18|366|2|3.80|7.37|1.17|0.11|2.34|7.60|14.74|0.06|0.11|2.23|2.29|-5.37| +2451769|50946|3253|20134|914832|1883|25781|8|102|367|32|79.39|150.84|126.70|0.00|4054.40|2540.48|4826.88|202.72|0.00|4054.40|4257.12|1513.92| +2451769|50946|12721|20134|914832|1883|25781|8|101|367|32|39.30|56.98|54.13|0.00|1732.16|1257.60|1823.36|34.64|0.00|1732.16|1766.80|474.56| +2451769|50946|12740|20134|914832|1883|25781|8|148|367|39|3.18|5.75|3.39|0.00|132.21|124.02|224.25|3.96|0.00|132.21|136.17|8.19| +2451769|50946|4028|20134|914832|1883|25781|8|64|367|78|17.80|29.19|14.01|0.00|1092.78|1388.40|2276.82|65.56|0.00|1092.78|1158.34|-295.62| +2451769|50946|17729|20134|914832|1883|25781|8|42|367|99|62.69|125.38|119.11|0.00|11791.89|6206.31|12412.62|943.35|0.00|11791.89|12735.24|5585.58| +2451769|50946|17993|20134|914832|1883|25781|8|149|367|31|61.28|93.14|29.80|831.42|923.80|1899.68|2887.34|0.92|831.42|92.38|93.30|-1807.30| +2451769|50946|683|20134|914832|1883|25781|8|58|367|32|63.22|81.55|61.16|0.00|1957.12|2023.04|2609.60|97.85|0.00|1957.12|2054.97|-65.92| +2451769|50946|3752|20134|914832|1883|25781|8|212|367|69|58.35|112.61|83.33|0.00|5749.77|4026.15|7770.09|402.48|0.00|5749.77|6152.25|1723.62| +2451769|50946|10034|20134|914832|1883|25781|8|198|367|88|44.22|64.56|58.10|0.00|5112.80|3891.36|5681.28|51.12|0.00|5112.80|5163.92|1221.44| +2451769|50946|7358|20134|914832|1883|25781|8|60|367|38|16.66|23.65|15.84|0.00|601.92|633.08|898.70|18.05|0.00|601.92|619.97|-31.16| +2451769|50946|15584|20134|914832|1883|25781|8|3|367|5|88.60|122.26|114.92|0.00|574.60|443.00|611.30|28.73|0.00|574.60|603.33|131.60| +2451769|50946|6397|20134|914832|1883|25781|8|28|367|93|66.91|82.96|14.10|0.00|1311.30|6222.63|7715.28|39.33|0.00|1311.30|1350.63|-4911.33| +2451769|50946|17305|20134|914832|1883|25781|8|250|367|42|99.07|120.86|32.63|0.00|1370.46|4160.94|5076.12|123.34|0.00|1370.46|1493.80|-2790.48| +2451769|50946|7315|20134|914832|1883|25781|8|188|367|88|86.53|89.99|7.19|0.00|632.72|7614.64|7919.12|50.61|0.00|632.72|683.33|-6981.92| +2451879|44249|6539|10878|1291292|953|35367|10|153|368|36|66.85|71.52|52.92|1124.02|1905.12|2406.60|2574.72|15.62|1124.02|781.10|796.72|-1625.50| +2451879|44249|392|10878|1291292|953|35367|10|179|368|95|21.94|30.49|10.67|91.22|1013.65|2084.30|2896.55|36.89|91.22|922.43|959.32|-1161.87| +2451879|44249|1243|10878|1291292|953|35367|10|34|368|40|32.03|49.00|2.45|0.00|98.00|1281.20|1960.00|6.86|0.00|98.00|104.86|-1183.20| +2451879|44249|17636|10878|1291292|953|35367|10|32|368|4|63.44|93.89|20.65|0.00|82.60|253.76|375.56|6.60|0.00|82.60|89.20|-171.16| +2451879|44249|16274|10878|1291292|953|35367|10|156|368|59|2.79|4.01|2.96|0.00|174.64|164.61|236.59|3.49|0.00|174.64|178.13|10.03| +2451879|44249|11477|10878|1291292|953|35367|10|151|368|52|93.09|155.46|104.15|216.63|5415.80|4840.68|8083.92|0.00|216.63|5199.17|5199.17|358.49| +2451879|44249|9665||1291292|953|||256|368||37.73|60.74||69.92|998.90||||69.92|928.98|975.42|| +2451879|44249|10742|10878|1291292|953|35367|10|254|368|79|53.43|68.92|0.68|0.00|53.72|4220.97|5444.68|4.29|0.00|53.72|58.01|-4167.25| +2451879|44249|6959|10878|1291292|953|35367|10|153|368|76|46.73|86.91|6.08|388.14|462.08|3551.48|6605.16|2.21|388.14|73.94|76.15|-3477.54| +2451879|44249|14957|10878|1291292|953|35367|10|254|368|4|49.73|66.63|42.64|0.00|170.56|198.92|266.52|5.11|0.00|170.56|175.67|-28.36| +2451879|44249|4670|10878|1291292|953|35367|10|274|368|95|31.92|52.98|44.50|0.00|4227.50|3032.40|5033.10|126.82|0.00|4227.50|4354.32|1195.10| +2451879|44249|12695|10878|1291292|953|35367|10|151|368|98|60.70|117.15|28.11|0.00|2754.78|5948.60|11480.70|27.54|0.00|2754.78|2782.32|-3193.82| +2451879|44249|16171|10878|1291292|953|35367|10|18|368|66|43.73|48.97|48.97|1066.56|3232.02|2886.18|3232.02|173.23|1066.56|2165.46|2338.69|-720.72| +2452473|49148|4020|20535|1726409|1883|49357|4|31|369|100|51.54|103.08|40.20|0.00|4020.00|5154.00|10308.00|241.20|0.00|4020.00|4261.20|-1134.00| +2452473||9223|20535|||49357|4|86|369||75.94||||210.50|151.88|300.72|||105.25|114.72|| +2452473|49148|5371|20535|1726409|1883|49357|4|182|369|50|99.39|187.84|116.46|58.23|5823.00|4969.50|9392.00|172.94|58.23|5764.77|5937.71|795.27| +2452473|49148|13215|20535|1726409|1883|49357|4|60|369|61|39.22|64.32|40.52|0.00|2471.72|2392.42|3923.52|173.02|0.00|2471.72|2644.74|79.30| +2452473|49148|1347|20535|1726409|1883|49357|4|41|369|91|29.75|52.36|25.13|1097.67|2286.83|2707.25|4764.76|35.67|1097.67|1189.16|1224.83|-1518.09| +2452473|49148|15673|20535|1726409|1883|49357|4|104|369|41|87.49|162.73|86.24|0.00|3535.84|3587.09|6671.93|282.86|0.00|3535.84|3818.70|-51.25| +2452473||12097|20535||1883|49357|4|4|369|71||135.28|110.92|0.00|7875.32||9604.88||0.00|7875.32|7954.07|| +2452473||15660||1726409|||||369|84||||0.00|5350.80||8494.08||0.00|||-589.68| +2452473|49148|16117|20535|1726409|1883|49357|4|271|369|9|61.29|96.22|0.96|0.00|8.64|551.61|865.98|0.34|0.00|8.64|8.98|-542.97| +2452473|49148|5586|20535|1726409|1883|49357|4|289|369|36|25.68|43.39|10.84|0.00|390.24|924.48|1562.04|31.21|0.00|390.24|421.45|-534.24| +2452473|49148|3780|20535|1726409|1883|49357|4|156|369|4|23.67|26.98|16.99|45.53|67.96|94.68|107.92|1.34|45.53|22.43|23.77|-72.25| +2452473|49148|3835|20535|1726409|1883|49357|4|105|369|88|3.17|3.26|2.70|0.00|237.60|278.96|286.88|16.63|0.00|237.60|254.23|-41.36| +2452473|49148|925|20535|1726409|1883|49357|4|167|369|8|83.52|161.19|14.50|0.00|116.00|668.16|1289.52|6.96|0.00|116.00|122.96|-552.16| +2452473|49148|3699|20535|1726409|1883|49357|4|7|369|82|94.96|112.05|29.13|0.00|2388.66|7786.72|9188.10|143.31|0.00|2388.66|2531.97|-5398.06| +2452473|49148|16764|20535|1726409|1883|49357|4|107|369|62|58.69|101.53|60.91|0.00|3776.42|3638.78|6294.86|151.05|0.00|3776.42|3927.47|137.64| +2452473|49148|14199|20535|1726409|1883|49357|4|16|369|82|55.54|98.86|82.05|0.00|6728.10|4554.28|8106.52|0.00|0.00|6728.10|6728.10|2173.82| +2451375|75079|1078|50435|1603869|6103|34290|7|112|370|78|84.72|151.64|100.08|0.00|7806.24|6608.16|11827.92|624.49|0.00|7806.24|8430.73|1198.08| +2451375|75079|3085|50435|1603869|6103|34290|7|28|370|100|83.44|160.20|140.97|0.00|14097.00|8344.00|16020.00|281.94|0.00|14097.00|14378.94|5753.00| +2451375|75079|13477|50435|1603869|6103|34290|7|244|370|74|1.95|3.80|3.64|121.21|269.36|144.30|281.20|1.48|121.21|148.15|149.63|3.85| +2451375|75079|5468|50435|1603869|6103|34290|7|137|370|31|48.91|89.99|41.39|0.00|1283.09|1516.21|2789.69|25.66|0.00|1283.09|1308.75|-233.12| +2451375|75079|5305|50435|1603869|6103|34290|7|71|370|90|40.93|78.58|24.35|1205.32|2191.50|3683.70|7072.20|19.72|1205.32|986.18|1005.90|-2697.52| +2451375|75079|1132|50435|1603869|6103|34290|7|278|370|87|10.73|18.56|12.06|0.00|1049.22|933.51|1614.72|20.98|0.00|1049.22|1070.20|115.71| +2451375|75079|4774|50435|1603869|6103|34290|7|116|370|84|73.68|90.62|47.12|0.00|3958.08|6189.12|7612.08|356.22|0.00|3958.08|4314.30|-2231.04| +2451375|75079|5008|50435|1603869|6103|34290|7|286|370|21|7.51|12.61|9.83|0.00|206.43|157.71|264.81|2.06|0.00|206.43|208.49|48.72| +2451917|29876|7101|84319|162602|1685|36368|7|37|371|79|14.07|25.04|12.01|0.00|948.79|1111.53|1978.16|56.92|0.00|948.79|1005.71|-162.74| +2451917|29876|11633|||1685|36368||283|371|20|78.92||||1155.20|||||1155.20||-423.20| +2451917|29876|15485|84319|162602|1685|36368|7|101|371|46|26.25|41.47|1.24|0.00|57.04|1207.50|1907.62|5.13|0.00|57.04|62.17|-1150.46| +2451917|29876|12419|84319|162602|1685|36368|7|239|371|46|25.50|32.13|4.17|0.00|191.82|1173.00|1477.98|13.42|0.00|191.82|205.24|-981.18| +2451917|29876|13307|84319|162602|1685|36368|7|300|371|47|59.40|94.44|83.10|78.11|3905.70|2791.80|4438.68|344.48|78.11|3827.59|4172.07|1035.79| +2451917|29876|17067|84319|162602|1685|36368|7|129|371|86|97.29|145.93|21.88|0.00|1881.68|8366.94|12549.98|169.35|0.00|1881.68|2051.03|-6485.26| +2451917|29876|10323|84319|162602|1685|36368|7|111|371|5|40.98|77.45|17.03|0.00|85.15|204.90|387.25|0.85|0.00|85.15|86.00|-119.75| +2451917|29876|12993|84319|162602|1685|36368|7|123|371|53|6.96|11.55|0.11|4.72|5.83|368.88|612.15|0.04|4.72|1.11|1.15|-367.77| +2451917|29876|539|84319|162602|1685|36368|7|129|371|29|16.90|17.06|8.53|0.00|247.37|490.10|494.74|2.47|0.00|247.37|249.84|-242.73| +2451917|29876|1119|84319|162602|1685|36368|7|108|371|47|88.50|140.71|61.91|0.00|2909.77|4159.50|6613.37|116.39|0.00|2909.77|3026.16|-1249.73| +2451917|29876|17609|84319|162602|1685|36368|7|188|371|5|89.68|171.28|11.98|0.00|59.90|448.40|856.40|4.79|0.00|59.90|64.69|-388.50| +2451969|47535|2603|93120|421952|6784|35616|7|188|372|83|45.13|64.98|30.54|887.18|2534.82|3745.79|5393.34|82.38|887.18|1647.64|1730.02|-2098.15| +2451969|47535|7901|93120|421952|6784|35616|7|110|372|34|45.91|81.26|34.94|0.00|1187.96|1560.94|2762.84|95.03|0.00|1187.96|1282.99|-372.98| +2451969|47535|10063|93120|421952|6784|35616|7|4|372|99|80.58|150.68|81.36|0.00|8054.64|7977.42|14917.32|241.63|0.00|8054.64|8296.27|77.22| +|47535|1841||||35616|7||372||46.23||||0.00|1941.66||||0.00|0.00|| +2451969|47535|4179|93120|421952|6784|35616|7|15|372|65|17.05|28.81|14.98|0.00|973.70|1108.25|1872.65|87.63|0.00|973.70|1061.33|-134.55| +2451969|47535|4013|93120|421952|6784|35616|7|40|372|93|73.11|89.19|2.67|0.00|248.31|6799.23|8294.67|22.34|0.00|248.31|270.65|-6550.92| +2451969|47535|13057|93120|421952|6784|35616|7|286|372|20||46.74|17.76||355.20|||24.86||||-285.20| +2451969|47535|14689|93120|421952|6784|35616|7|251|372|56|58.18|115.19|51.83|0.00|2902.48|3258.08|6450.64|0.00|0.00|2902.48|2902.48|-355.60| +2451969|47535|12697|93120|421952|6784|35616|7|241|372|25|84.37|96.18|71.17|0.00|1779.25|2109.25|2404.50|53.37|0.00|1779.25|1832.62|-330.00| +2451969|47535|6813|93120|421952|6784|35616|7|273|372|84|80.89|105.15|88.32|0.00|7418.88|6794.76|8832.60|370.94|0.00|7418.88|7789.82|624.12| +2451969|47535|3893|93120|421952|6784|35616|7|124|372|92|6.20|8.37|6.86|0.00|631.12|570.40|770.04|25.24|0.00|631.12|656.36|60.72| +2451960|68811|7291|86172|1358346|3673|49000|2|199|373|55|50.82|55.39|45.97|0.00|2528.35|2795.10|3046.45|151.70|0.00|2528.35|2680.05|-266.75| +|68811|13851|86172||3673|49000|||373||11.74|21.71|10.85|0.00|716.10|774.84|1432.86|57.28|0.00|716.10|773.38|| +2451960|68811|109|86172|1358346|3673|49000|2|258|373|28|33.51|43.89|26.77|712.08|749.56|938.28|1228.92|1.12|712.08|37.48|38.60|-900.80| +2451960|68811|2283|86172|1358346|3673|49000|2|27|373|34|50.74|99.45|0.99|0.00|33.66|1725.16|3381.30|1.00|0.00|33.66|34.66|-1691.50| +2451960|68811|16213|86172|1358346|3673|49000|2|167|373|27|7.74|9.28|2.96|2.39|79.92|208.98|250.56|2.32|2.39|77.53|79.85|-131.45| +2451960|68811|12959|86172|1358346|3673|49000|2|284|373|47|1.86|3.60|3.13|0.00|147.11|87.42|169.20|11.76|0.00|147.11|158.87|59.69| +2451960|68811|9733|86172|1358346|3673|49000|2|97|373|44|97.69|114.29|72.00|0.00|3168.00|4298.36|5028.76|0.00|0.00|3168.00|3168.00|-1130.36| +2451960|68811|9551|86172|1358346|3673|49000|2|73|373|41|7.35|11.90|11.90|48.79|487.90|301.35|487.90|30.73|48.79|439.11|469.84|137.76| +2451960|68811|1843|86172|1358346|3673|49000|2|206|373|94|58.96|90.79|31.77|0.00|2986.38|5542.24|8534.26|209.04|0.00|2986.38|3195.42|-2555.86| +2450842|74916|15262|24545|1881150|7002|22807|7|42|374|37|55.07|77.64|48.91|995.31|1809.67|2037.59|2872.68|0.00|995.31|814.36|814.36|-1223.23| +2450842|74916|8738|24545|1881150|7002|22807|7|193|374|92|59.44|83.21|19.13|0.00|1759.96|5468.48|7655.32|35.19|0.00|1759.96|1795.15|-3708.52| +2450842|74916|6277|24545|1881150|7002|22807|7|44|374|60|3.89|7.70|3.54|0.00|212.40|233.40|462.00|8.49|0.00|212.40|220.89|-21.00| +2450842|74916|10714|24545|1881150|7002|22807|7|283|374|87|14.51|17.41|2.08|0.00|180.96|1262.37|1514.67|5.42|0.00|180.96|186.38|-1081.41| +2450842|74916|16970|24545|1881150|7002|22807|7|290|374|67|45.03|51.33|8.72|0.00|584.24|3017.01|3439.11|23.36|0.00|584.24|607.60|-2432.77| +2450842|74916|10928|24545|1881150|7002|22807|7|40|374|64|80.66|129.05|43.87|0.00|2807.68|5162.24|8259.20|28.07|0.00|2807.68|2835.75|-2354.56| +||2996|24545||7002|||236|374|98|48.56|87.40||0.00|||||0.00|||550.76| +2450842|74916|13171|24545|1881150|7002|22807|7|215|374|68|55.41|62.05|37.85|0.00|2573.80|3767.88|4219.40|51.47|0.00|2573.80|2625.27|-1194.08| +2450842|74916|7282|24545|1881150|7002|22807|7|173|374|92|96.63|157.50|155.92|0.00|14344.64|8889.96|14490.00|1004.12|0.00|14344.64|15348.76|5454.68| +2450842|74916|15337|24545|1881150|7002|22807|7|189|374|78|51.72|71.37|28.54|0.00|2226.12|4034.16|5566.86|44.52|0.00|2226.12|2270.64|-1808.04| +2450842|74916|17125|24545|1881150|7002|22807|7|39|374|30|92.78|108.55|60.78|0.00|1823.40|2783.40|3256.50|164.10|0.00|1823.40|1987.50|-960.00| +2450842|74916|14780|24545|1881150|7002|22807|7|277|374|24|73.59|80.94|61.51|782.40|1476.24|1766.16|1942.56|62.44|782.40|693.84|756.28|-1072.32| +2450842|74916|17449|24545|1881150|7002|22807|7|47|374|50|90.51|177.39|177.39|0.00|8869.50|4525.50|8869.50|177.39|0.00|8869.50|9046.89|4344.00| +2451464|52192|15308|16316|620319|5922|17525|7|224|375|85|3.12|4.24|4.15|0.00|352.75|265.20|360.40|3.52|0.00|352.75|356.27|87.55| +2451464|52192|14708|16316|620319|5922|17525|7|183|375|87|72.26|112.00|10.08|0.00|876.96|6286.62|9744.00|26.30|0.00|876.96|903.26|-5409.66| +2451464|52192|13160|16316|620319|5922|17525|7|101|375|7|47.80|82.69|78.55|0.00|549.85|334.60|578.83|27.49|0.00|549.85|577.34|215.25| +2451464|52192|7747|16316|620319|5922|17525|7|97|375|43|16.82|26.07|23.98|0.00|1031.14|723.26|1121.01|82.49|0.00|1031.14|1113.63|307.88| +2451464|52192|4432|16316|620319|5922|17525|7|202|375|56|32.96|62.29|31.14|0.00|1743.84|1845.76|3488.24|87.19|0.00|1743.84|1831.03|-101.92| +2451464|52192|14428|16316|620319|5922|17525|7|297|375|62|16.33|16.49|0.98|0.00|60.76|1012.46|1022.38|1.82|0.00|60.76|62.58|-951.70| +2451464|52192|9547|16316|620319|5922|17525|7|89|375|90|2.04|2.83|2.68|0.00|241.20|183.60|254.70|16.88|0.00|241.20|258.08|57.60| +2451464|52192|1532|16316|620319|5922|17525|7|170|375|15|65.37|78.44|48.63|0.00|729.45|980.55|1176.60|7.29|0.00|729.45|736.74|-251.10| +2451905|39044|2318|26646|1059659|2625|46838|4|177|376|90|53.39|87.02|22.62|0.00|2035.80|4805.10|7831.80|142.50|0.00|2035.80|2178.30|-2769.30| +2451905|39044|1328|26646|1059659|2625|46838|4|225|376|63|48.04|72.06|10.80|0.00|680.40|3026.52|4539.78|27.21|0.00|680.40|707.61|-2346.12| +2451905|39044|7877|26646|1059659|2625|46838|4|202|376|48|26.04|51.81|46.62|0.00|2237.76|1249.92|2486.88|179.02|0.00|2237.76|2416.78|987.84| +2451905|39044|7159|26646|1059659|2625|46838|4|53|376|90|3.33|4.56|2.69|0.00|242.10|299.70|410.40|4.84|0.00|242.10|246.94|-57.60| +2451905|39044|14012||1059659|2625||4|83|376||81.87|||0.00||6303.99|11346.72||0.00|7375.06||| +2451905|39044|13507|26646|1059659|2625|46838|4|27|376|33|16.16|27.31|12.83|0.00|423.39|533.28|901.23|8.46|0.00|423.39|431.85|-109.89| +2451905|39044|7016|26646|1059659|2625|46838|4|186|376|65|24.69|29.62|18.95|0.00|1231.75|1604.85|1925.30|12.31|0.00|1231.75|1244.06|-373.10| +2451905|39044|2389|26646|1059659|2625|46838|4|295|376|83|10.38|13.28|1.06|0.00|87.98|861.54|1102.24|4.39|0.00|87.98|92.37|-773.56| +2451905|39044|1477|26646|1059659|2625|46838|4|290|376|18|49.67|61.09|42.76|0.00|769.68|894.06|1099.62|7.69|0.00|769.68|777.37|-124.38| +2451905|39044|11432|26646|1059659|2625|46838|4|116|376|58|64.38|88.84|38.20|0.00|2215.60|3734.04|5152.72|132.93|0.00|2215.60|2348.53|-1518.44| +2451905|39044|10567|26646|1059659|2625|46838|4|63|376|15|81.97|112.29|99.93|0.00|1498.95|1229.55|1684.35|89.93|0.00|1498.95|1588.88|269.40| +2451905|39044|6740|26646|1059659|2625|46838|4|84|376|53|41.68|70.85|68.01|0.00|3604.53|2209.04|3755.05|288.36|0.00|3604.53|3892.89|1395.49| +|39044|13481|26646||2625|46838|||376|||76.45|75.68|0.00|1210.88|1082.56||60.54|0.00|1210.88||128.32| +2451905|39044|2845|26646|1059659|2625|46838|4|4|376|72|68.73|112.71|37.19|0.00|2677.68|4948.56|8115.12|0.00|0.00|2677.68|2677.68|-2270.88| +2451905|39044|3401|26646|1059659|2625|46838|4|132|376|14|72.14|102.43|57.36|473.79|803.04|1009.96|1434.02|9.87|473.79|329.25|339.12|-680.71| +||11390||1881814|853||||377|||||||3872.61|3872.61|||||| +2451398|42971|6928|60375|1881814|853|28339|2|6|377|87|3.66|5.27|3.37|249.21|293.19|318.42|458.49|0.87|249.21|43.98|44.85|-274.44| +2451398|42971|13927|60375|1881814|853|28339|2|261|377|54|11.18|13.30|6.51|0.00|351.54|603.72|718.20|31.63|0.00|351.54|383.17|-252.18| +2451398|42971|10768|60375|1881814|853|28339|2|181|377|89|64.08|116.62|113.12|0.00|10067.68|5703.12|10379.18|302.03|0.00|10067.68|10369.71|4364.56| +2451398|42971|14701|60375|1881814|853|28339|2|217|377|54|96.56|144.84|134.70|0.00|7273.80|5214.24|7821.36|436.42|0.00|7273.80|7710.22|2059.56| +2451398|42971|17420|60375|1881814|853|28339|2|18|377|41|24.92|45.10|25.70|158.05|1053.70|1021.72|1849.10|8.95|158.05|895.65|904.60|-126.07| +2451398|42971|9436|60375|1881814|853|28339|2|148|377|16|72.81|112.12|108.75|435.00|1740.00|1164.96|1793.92|52.20|435.00|1305.00|1357.20|140.04| +2451398|42971|7654|60375|1881814|853|28339|2|148|377|95|97.90|169.36|152.42|0.00|14479.90|9300.50|16089.20|1158.39|0.00|14479.90|15638.29|5179.40| +2451398|42971|12223|60375|1881814|853|28339|2|31|377|79|25.72|37.55|26.66|0.00|2106.14|2031.88|2966.45|63.18|0.00|2106.14|2169.32|74.26| +2451398|42971|15085|60375|1881814|853|28339|2|283|377|78|65.74|131.48|120.96|0.00|9434.88|5127.72|10255.44|0.00|0.00|9434.88|9434.88|4307.16| +2451534|63060|6560|67578|283525|6225|2972|4|73|378|64|28.78|42.30|35.95|0.00|2300.80|1841.92|2707.20|46.01|0.00|2300.80|2346.81|458.88| +2451534|63060|12838|67578|283525|6225|2972|4|32|378|39|9.97|15.55|13.68|0.00|533.52|388.83|606.45|32.01|0.00|533.52|565.53|144.69| +||14017|67578|283525||||211|378||41.74||21.96|0.00||||147.57|0.00|2108.16||-1898.88| +2451534|63060|2692|67578|283525|6225|2972|4|202|378|79|53.91|90.56|46.18|2918.57|3648.22|4258.89|7154.24|51.07|2918.57|729.65|780.72|-3529.24| +2451534|63060|8329|||6225|2972|4|116|378|36||95.97||0.00|2349.00||3454.92||0.00||2419.47|471.24| +2451534|63060|7438|67578|283525|6225|2972|4|65|378|10|46.19|48.49|5.81|0.00|58.10|461.90|484.90|0.58|0.00|58.10|58.68|-403.80| +2451534|63060|13886|67578|283525|6225|2972|4|17|378|94|17.13|25.18|12.33|46.36|1159.02|1610.22|2366.92|100.13|46.36|1112.66|1212.79|-497.56| +||4699|67578||||||378||||14.87||609.67|||24.38||609.67|634.05|| +2451534|63060|5204|67578|283525|6225|2972|4|131|378|42|59.67|62.05|31.02|0.00|1302.84|2506.14|2606.10|52.11|0.00|1302.84|1354.95|-1203.30| +2451534|63060|12649|67578|283525|6225|2972|4|17|378|83|73.01|137.98|135.22|0.00|11223.26|6059.83|11452.34|448.93|0.00|11223.26|11672.19|5163.43| +2451534|63060|508|67578|283525|6225|2972|4|245|378|88|31.84|58.58|4.10|0.00|360.80|2801.92|5155.04|10.82|0.00|360.80|371.62|-2441.12| +2451534|63060|5416|||6225|||130|378|14|77.24|79.55|54.88|0.00||1081.36|||0.00||783.68|| +|63060|9259|67578|283525|||4|189|378|77|45.13||||2876.72|3475.01||230.13||2876.72||| +2451534|63060|7534|67578|283525|6225|2972|4|57|378|43|26.51|51.16|29.67|0.00|1275.81|1139.93|2199.88|25.51|0.00|1275.81|1301.32|135.88| +2451534|63060|2773|67578|283525|6225|2972|4|226|378|18|47.59|79.95|77.55|0.00|1395.90|856.62|1439.10|125.63|0.00|1395.90|1521.53|539.28| +2452592|39015|10809|31405|404787|2616|7598|4|200|379|15|13.48|14.42|7.78|63.01|116.70|202.20|216.30|0.53|63.01|53.69|54.22|-148.51| +2452592|39015|1032|31405|404787|2616|7598|4|242|379|9|21.58|40.57|28.80|0.00|259.20|194.22|365.13|23.32|0.00|259.20|282.52|64.98| +2452592|39015|11065|31405|404787|2616|7598|4|67|379|69|30.83|36.37|1.09|0.00|75.21|2127.27|2509.53|5.26|0.00|75.21|80.47|-2052.06| +2452592|39015|900|31405|404787|2616|7598|4|231|379|42|7.93|11.49|8.27|0.00|347.34|333.06|482.58|17.36|0.00|347.34|364.70|14.28| +2452592|39015|13311|31405|404787|2616|7598|4|87|379|57|10.64|20.32|20.32|289.56|1158.24|606.48|1158.24|69.49|289.56|868.68|938.17|262.20| +2452592|39015|660|31405|404787|2616|7598|4|192|379|37|79.12|142.41|61.23|0.00|2265.51|2927.44|5269.17|0.00|0.00|2265.51|2265.51|-661.93| +2452592|39015|5491|31405|404787|2616|7598|4|47|379|43|51.04|65.84|48.06|0.00|2066.58|2194.72|2831.12|185.99|0.00|2066.58|2252.57|-128.14| +2452592|39015|14847|31405|404787|2616|7598|4|187|379|70|24.69|26.66|5.33|0.00|373.10|1728.30|1866.20|26.11|0.00|373.10|399.21|-1355.20| +2452592|39015|8865|31405|404787|2616|7598|4|266|379|100|61.08|92.84|23.21|0.00|2321.00|6108.00|9284.00|69.63|0.00|2321.00|2390.63|-3787.00| +|39015|763|||||4|92|379|76||47.06|||||3576.56|63.08||700.97|764.05|-1354.83| +2452107|38378|6409|13752|89646|3940|22681|1|225|380|47|25.80|35.86|22.59|913.08|1061.73|1212.60|1685.42|11.89|913.08|148.65|160.54|-1063.95| +2452107|38378|13661|13752|89646|3940|22681|1|113|380|67|14.47|18.52|2.59|0.00|173.53|969.49|1240.84|8.67|0.00|173.53|182.20|-795.96| +2452107|38378|10763|13752|89646|3940|22681|1|290|380|84|34.09|59.31|52.19|0.00|4383.96|2863.56|4982.04|87.67|0.00|4383.96|4471.63|1520.40| +2452107|38378|10197|13752|89646|3940|22681|1|112|380|90|57.86|95.46|83.05|0.00|7474.50|5207.40|8591.40|448.47|0.00|7474.50|7922.97|2267.10| +2452107||7301|13752||3940|22681||120|380||51.67|68.72||2810.73|3268.30|||9.15|2810.73|457.57|466.72|| +2452107|38378|15479|13752|89646|3940|22681|1|66|380|6|95.69|117.69|87.09|438.93|522.54|574.14|706.14|0.83|438.93|83.61|84.44|-490.53| +2452107|38378|9649|13752|89646|3940|22681|1|176|380|4|7.28|8.22|0.65|0.00|2.60|29.12|32.88|0.00|0.00|2.60|2.60|-26.52| +2452107|38378|12965|13752|89646|3940|22681|1|18|380|59|84.98|139.36|65.49|0.00|3863.91|5013.82|8222.24|270.47|0.00|3863.91|4134.38|-1149.91| +2452107|38378|14693|13752|89646|3940|22681|1|49|380|6|72.86|77.96|10.91|0.00|65.46|437.16|467.76|1.30|0.00|65.46|66.76|-371.70| +2452107||12761|13752|||22681||121|380|10||98.09||0.00|539.40|628.80||21.57|0.00|539.40|560.97|-89.40| +2452107|38378|11119|13752|89646|3940|22681|1|175|380|71|85.45|155.51|124.40|0.00|8832.40|6066.95|11041.21|618.26|0.00|8832.40|9450.66|2765.45| +2452107|38378|11757|13752|89646|3940|22681|1|103|380|19|51.22|89.63|14.34|0.00|272.46|973.18|1702.97|16.34|0.00|272.46|288.80|-700.72| +2452107|38378|14125|13752|89646|3940|22681|1|82|380|26|43.91|68.49|21.91|0.00|569.66|1141.66|1780.74|28.48|0.00|569.66|598.14|-572.00| +2451028|69979|10130|15667|125610|3106|44649|7|151|381|60|23.18|24.57|15.23|9.13|913.80|1390.80|1474.20|27.14|9.13|904.67|931.81|-486.13| +2451028||3680|15667|125610||44649||90|381|||125.26|||1064.54|2535.04||85.16||||-1470.50| +2451028|69979|3562|15667|125610|3106|44649|7|85|381|28|38.69|65.77|53.27|0.00|1491.56|1083.32|1841.56|74.57|0.00|1491.56|1566.13|408.24| +2451028|69979|7742|15667|125610|3106|44649|7|199|381|78|32.26|39.35|39.35|2056.43|3069.30|2516.28|3069.30|0.00|2056.43|1012.87|1012.87|-1503.41| +2451028|69979|6130|15667|125610|3106|44649|7|154|381|7|10.41|17.17|10.64|0.00|74.48|72.87|120.19|5.21|0.00|74.48|79.69|1.61| +2451028||4400||125610||44649|||381||55.22||7.34|0.00|7.34||104.91||0.00|7.34||-47.88| +2451028|69979|13498|15667|125610|3106|44649|7|133|381|22|25.48|26.75|19.79|0.00|435.38|560.56|588.50|26.12|0.00|435.38|461.50|-125.18| +2451028|69979|15163|15667|125610|3106|44649|7|258|381|41|83.38|100.05|93.04|0.00|3814.64|3418.58|4102.05|305.17|0.00|3814.64|4119.81|396.06| +2451028|69979|11326|15667|125610|3106|44649|7|15|381|53|32.46|43.49|10.43|60.80|552.79|1720.38|2304.97|19.67|60.80|491.99|511.66|-1228.39| +2451028|69979|2653|15667|125610|3106|44649|7|2|381|41|34.99|45.48|23.64|329.54|969.24|1434.59|1864.68|19.19|329.54|639.70|658.89|-794.89| +2451028|69979|12373|15667|||44649|7|41|381||98.35||154.05||3697.20|||||3697.20||1336.80| +2451028|69979|9944|15667|125610|3106|44649|7|252|381|46|92.12|133.57|16.02|0.00|736.92|4237.52|6144.22|7.36|0.00|736.92|744.28|-3500.60| +2451028|69979|8770|15667|125610|3106|44649|7|31|381|42|18.28|22.30|0.22|7.29|9.24|767.76|936.60|0.13|7.29|1.95|2.08|-765.81| +2451028|69979|14534|15667|125610|3106|44649|7|133|381|41|40.72|58.63|26.96|0.00|1105.36|1669.52|2403.83|77.37|0.00|1105.36|1182.73|-564.16| +2451160|60337|6962|58244|47397|4197|24950|8|18|382|85|15.22|18.72|3.36|0.00|285.60|1293.70|1591.20|19.99|0.00|285.60|305.59|-1008.10| +2451160|60337|12862|58244|47397|4197|24950|8|141|382|81|40.84|46.14|7.38|0.00|597.78|3308.04|3737.34|35.86|0.00|597.78|633.64|-2710.26| +2451160|60337|4219|58244|47397|4197|24950|8|30|382|70|6.30|12.15|8.99|0.00|629.30|441.00|850.50|44.05|0.00|629.30|673.35|188.30| +2451160|60337|4268|58244|47397|4197|24950|8|149|382|96|1.71|2.27|2.27|0.00|217.92|164.16|217.92|8.71|0.00|217.92|226.63|53.76| +2451160|60337|1108|58244|47397|4197||8|20|382|||46.55|32.11|0.00|2344.03|1788.50|3398.15|70.32|0.00|2344.03||555.53| +2451160|60337|11227|58244|47397|4197|24950|8|92|382|41|44.40|72.81|66.98|0.00|2746.18|1820.40|2985.21|192.23|0.00|2746.18|2938.41|925.78| +2451160|60337|1310|58244|47397|4197|24950|8|160|382|4|86.00|116.10|84.75|0.00|339.00|344.00|464.40|6.78|0.00|339.00|345.78|-5.00| +2451160|60337|12112|58244|47397|4197|24950|8|30|382|100|70.48|105.72|4.22|0.00|422.00|7048.00|10572.00|4.22|0.00|422.00|426.22|-6626.00| +2451160|60337|6770|58244|47397|4197|24950|8|274|382|44|59.87|65.25|1.95|0.00|85.80|2634.28|2871.00|2.57|0.00|85.80|88.37|-2548.48| +2452191|69956|7381|26687|1880336|3938|39987|4|200|383|25|33.05|50.89|19.84|0.00|496.00|826.25|1272.25|29.76|0.00|496.00|525.76|-330.25| +2452191|69956|13343|26687|1880336|3938|39987|4|266|383|41|32.86|56.84|48.31|0.00|1980.71|1347.26|2330.44|19.80|0.00|1980.71|2000.51|633.45| +2452191|69956|9365|26687|1880336|3938|39987|4|155|383|81|11.20|18.14|4.89|0.00|396.09|907.20|1469.34|31.68|0.00|396.09|427.77|-511.11| +2452191||12767|26687|1880336|3938||4||383||||84.44|0.00|4222.00|||168.88|0.00|||| +2452191|69956|1691|26687|1880336|3938|39987|4|102|383|64|23.24|37.64|18.44|0.00|1180.16|1487.36|2408.96|0.00|0.00|1180.16|1180.16|-307.20| +2452191|69956|14333|26687|1880336|3938|39987|4|197|383|84|91.38|165.39|99.23|0.00|8335.32|7675.92|13892.76|666.82|0.00|8335.32|9002.14|659.40| +2452191|69956|15373|26687|1880336|3938|39987|4|200|383|39|69.48|97.96|70.53|0.00|2750.67|2709.72|3820.44|0.00|0.00|2750.67|2750.67|40.95| +2452191|69956|8373|26687|1880336|3938|39987|4|162|383|26|4.48|7.34|1.39|16.98|36.14|116.48|190.84|0.57|16.98|19.16|19.73|-97.32| +2451491|63444|4270|58957|1178788|6972|36803|4|123|384|43|57.11|94.80|22.75|0.00|978.25|2455.73|4076.40|0.00|0.00|978.25|978.25|-1477.48| +2451491|63444|14857|58957|1178788|6972|36803|4|95|384|38|74.22|100.19|41.07|0.00|1560.66|2820.36|3807.22|62.42|0.00|1560.66|1623.08|-1259.70| +2451491|63444|10624|58957|1178788|6972|36803|4|164|384|76|66.90|105.70|96.18|0.00|7309.68|5084.40|8033.20|219.29|0.00|7309.68|7528.97|2225.28| +2451491|63444|13996|58957|1178788|6972|36803|4|69|384|43|55.93|71.59|32.21|0.00|1385.03|2404.99|3078.37|0.00|0.00|1385.03|1385.03|-1019.96| +2451491|63444|262|58957|1178788|6972|36803|4|61|384|78|60.73|81.98|59.02|0.00|4603.56|4736.94|6394.44|368.28|0.00|4603.56|4971.84|-133.38| +2451491|63444|10099|58957|1178788|6972|36803|4|170|384|25|13.75|16.22|5.51|0.00|137.75|343.75|405.50|8.26|0.00|137.75|146.01|-206.00| +2451491|63444|7843|58957|1178788|6972|36803|4|143|384|57|48.50|77.60|31.81|1704.37|1813.17|2764.50|4423.20|0.00|1704.37|108.80|108.80|-2655.70| +2451491|63444|16600|58957|1178788|6972|36803|4|207|384|70|85.06|148.00|31.08|0.00|2175.60|5954.20|10360.00|65.26|0.00|2175.60|2240.86|-3778.60| +2451491|63444|14|58957|1178788|6972|36803|4|269|384|100|23.54|25.89|12.16|0.00|1216.00|2354.00|2589.00|24.32|0.00|1216.00|1240.32|-1138.00| +2451491|63444|13129|58957|1178788|6972|36803|4|223|384|52|32.66|57.80|56.64|0.00|2945.28|1698.32|3005.60|0.00|0.00|2945.28|2945.28|1246.96| +2451491|63444|14728|58957|1178788|6972|36803|4|267|384|61|38.03|40.69|19.53|0.00|1191.33|2319.83|2482.09|23.82|0.00|1191.33|1215.15|-1128.50| +2451491|63444|10228|58957|1178788|6972|36803|4|203|384|37|55.60|80.62|43.53|0.00|1610.61|2057.20|2982.94|128.84|0.00|1610.61|1739.45|-446.59| +2451491|63444|10291|58957|1178788|6972|36803|4|149|384|28|90.61|173.97|34.79|0.00|974.12|2537.08|4871.16|9.74|0.00|974.12|983.86|-1562.96| +2451880|41637|5413|62143|1791883|1623|37763|4|193|385|71|1.51|2.56|1.84|60.09|130.64|107.21|181.76|2.82|60.09|70.55|73.37|-36.66| +2451880|41637|6673||||37763||149|385|||9.60|0.09||5.04|512.40|537.60|||5.04|5.29|-507.36| +2451880|41637|14678|62143|1791883|1623|37763|4|188|385|56|88.46|135.34|21.65|557.70|1212.40|4953.76|7579.04|39.28|557.70|654.70|693.98|-4299.06| +2451880|41637|13637||1791883|1623||4||385|48|||||3926.88||6438.24|||||94.56| +2451880|41637|17987||1791883|1623|37763||165|385||61.43||53.85|0.00|3123.30||||0.00||3310.69|| +2451880||15169|||1623|37763||243|385||||||1500.00|1002.75||75.00||1500.00|1575.00|497.25| +2451880|41637|11336|62143|1791883|1623|37763|4|191|385|7|63.05|109.07|69.80|0.00|488.60|441.35|763.49|14.65|0.00|488.60|503.25|47.25| +2451880|41637|8105|62143|1791883|1623|37763|4|163|385|46|42.20|48.10|33.67|356.22|1548.82|1941.20|2212.60|35.77|356.22|1192.60|1228.37|-748.60| +2451880|41637|16196|62143|1791883|1623|37763|4|218|385|42|43.01|51.18|33.26|0.00|1396.92|1806.42|2149.56|13.96|0.00|1396.92|1410.88|-409.50| +2451880|41637|5570|62143|1791883|1623|37763|4|127|385|48|39.19|63.48|55.86|670.32|2681.28|1881.12|3047.04|100.54|670.32|2010.96|2111.50|129.84| +2451880|41637|11923|62143|1791883|1623|37763|4|114|385|52|17.05|21.65|15.15|0.00|787.80|886.60|1125.80|15.75|0.00|787.80|803.55|-98.80| +2451880|41637|9047|62143|1791883|1623|37763|4|152|385|35|41.68|62.10|37.88|1232.99|1325.80|1458.80|2173.50|8.35|1232.99|92.81|101.16|-1365.99| +2451880|41637|4693|62143|1791883|1623|37763|4|26|385|37|8.01|13.53|12.71|183.40|470.27|296.37|500.61|17.21|183.40|286.87|304.08|-9.50| +2451880|41637|16459|62143|1791883|1623|37763|4|269|385|57|52.09|79.69|0.79|23.41|45.03|2969.13|4542.33|1.51|23.41|21.62|23.13|-2947.51| +2451880|41637|8579|62143|1791883|1623|37763|4|253|385|10|37.35|63.12|44.81|0.00|448.10|373.50|631.20|8.96|0.00|448.10|457.06|74.60| +2451880|41637|10909|62143|1791883|1623|37763|4|294|385|86|84.55|122.59|91.94|0.00|7906.84|7271.30|10542.74|474.41|0.00|7906.84|8381.25|635.54| +2452191|44081|11347|22927|1536310|2846|36629|2|125|386|2|67.66|92.69|56.54|0.00|113.08|135.32|185.38|1.13|0.00|113.08|114.21|-22.24| +2452191|44081|9035|22927|1536310|2846|36629|2|221|386|69|18.03|18.21|14.93|0.00|1030.17|1244.07|1256.49|61.81|0.00|1030.17|1091.98|-213.90| +|44081|15189||1536310|2846|36629|2||386|||||||2602.24||17.98||199.88|217.86|| +2452191|44081|4783|22927|1536310|2846|36629|2|90|386|56|68.50|80.83|66.28|705.21|3711.68|3836.00|4526.48|210.45|705.21|3006.47|3216.92|-829.53| +2452191|44081|1275|22927|1536310|2846|36629|2|265|386|35|47.24|78.89|74.15|1219.76|2595.25|1653.40|2761.15|123.79|1219.76|1375.49|1499.28|-277.91| +2452191|44081|13965|22927|1536310|2846|36629|2|10|386|19|8.57|9.76|5.46|0.00|103.74|162.83|185.44|0.00|0.00|103.74|103.74|-59.09| +2452191|44081|11815|22927|1536310|2846|36629|2|233|386|75|94.16|135.59|23.05|1573.16|1728.75|7062.00|10169.25|9.33|1573.16|155.59|164.92|-6906.41| +2452191|44081|3839|22927|1536310|2846|36629|2|294|386|67|20.75|31.54|27.43|0.00|1837.81|1390.25|2113.18|110.26|0.00|1837.81|1948.07|447.56| +2451086|61509|11240|58602|||41046|||387|6||14.94||2.18||79.38|89.64|0.25|2.18|||| +2451086|61509|13916|58602|214750|1298|41046|10|31|387|5|4.02|5.70|3.02|0.00|15.10|20.10|28.50|0.00|0.00|15.10|15.10|-5.00| +2451086|61509|13148|58602|214750|1298|41046|10|154|387|94|50.38|73.55|63.25|0.00|5945.50|4735.72|6913.70|356.73|0.00|5945.50|6302.23|1209.78| +2451086|61509|3152|58602|214750|1298|41046|10|263|387|37|65.48|106.73|99.25|0.00|3672.25|2422.76|3949.01|73.44|0.00|3672.25|3745.69|1249.49| +2451086|61509|11582|58602|214750|1298|41046|10|187|387|89|24.35|47.23|44.86|1597.01|3992.54|2167.15|4203.47|119.77|1597.01|2395.53|2515.30|228.38| +2451086|61509|3649|58602|214750|1298|41046|10|17|387|27|24.61|47.98|30.22|0.00|815.94|664.47|1295.46|16.31|0.00|815.94|832.25|151.47| +2451086|61509|12913|58602|214750|1298|41046|10|37|387|74|47.13|81.53|57.07|1942.66|4223.18|3487.62|6033.22|136.83|1942.66|2280.52|2417.35|-1207.10| +2451086|61509|4502|58602|214750|1298|41046|10|271|387|4|75.81|102.34|89.03|0.00|356.12|303.24|409.36|32.05|0.00|356.12|388.17|52.88| +2451086|61509|1687|58602|214750|1298|41046|10|93|387|100|57.62|99.68|3.98|0.00|398.00|5762.00|9968.00|3.98|0.00|398.00|401.98|-5364.00| +2451086|61509|8200|58602|214750|1298|41046|10|7|387|97|23.26|28.60|10.01|0.00|970.97|2256.22|2774.20|58.25|0.00|970.97|1029.22|-1285.25| +2451086|61509|5704|58602|214750|1298|41046|10|294|387|19|83.48|136.07|119.74|0.00|2275.06|1586.12|2585.33|0.00|0.00|2275.06|2275.06|688.94| +2451086|61509|2018|58602|214750|1298|41046|10|75|387|33|94.96|145.28|113.31|0.00|3739.23|3133.68|4794.24|224.35|0.00|3739.23|3963.58|605.55| +2452400|59292|8193|79936|791007|4031|255|7|192|388|64|28.76|30.48|27.73|0.00|1774.72|1840.64|1950.72|0.00|0.00|1774.72|1774.72|-65.92| +2452400|59292|2262|79936|791007|4031|255|7|113|388|45|55.58|72.25|61.41|1188.28|2763.45|2501.10|3251.25|47.25|1188.28|1575.17|1622.42|-925.93| +2452400|59292|6408|79936|791007|4031|255|7|42|388|7|26.61|40.71|15.06|0.00|105.42|186.27|284.97|0.00|0.00|105.42|105.42|-80.85| +2452400|59292|11292|79936|791007|4031|255|7|297|388|74|2.73|5.18|2.43|0.00|179.82|202.02|383.32|12.58|0.00|179.82|192.40|-22.20| +2452400|59292|13722|79936|791007|4031|255|7|262|388|15|24.25|46.80|4.68|0.00|70.20|363.75|702.00|2.10|0.00|70.20|72.30|-293.55| +2452400|59292|2047|79936|791007|4031|255|7|296|388|26|74.80|114.44|111.00|0.00|2886.00|1944.80|2975.44|202.02|0.00|2886.00|3088.02|941.20| +2452400|59292|5622|79936|791007|4031|255|7|12|388|68|36.15|57.47|1.72|0.00|116.96|2458.20|3907.96|3.50|0.00|116.96|120.46|-2341.24| +2452400|59292|14985|79936|791007|4031|255|7|36|388|5|34.08|56.91|46.66|0.00|233.30|170.40|284.55|13.99|0.00|233.30|247.29|62.90| +2452400|59292|17178|79936|791007|4031|255|7|163|388|19|64.17|104.59|100.40|0.00|1907.60|1219.23|1987.21|133.53|0.00|1907.60|2041.13|688.37| +2451894|58377|12368|61442|4037|5032|30827|8|255|389|71|42.75|84.21|31.15|0.00|2211.65|3035.25|5978.91|176.93|0.00|2211.65|2388.58|-823.60| +2451894|58377|14953|61442|4037|5032|30827|8|223|389|40|13.73|18.12|10.87|0.00|434.80|549.20|724.80|13.04|0.00|434.80|447.84|-114.40| +2451894|58377|3643|61442|4037|5032|30827|8|96|389|5|54.79|106.84|26.71|0.00|133.55|273.95|534.20|2.67|0.00|133.55|136.22|-140.40| +2451894|58377|3833|61442|4037|5032|30827|8|164|389|85|27.38|43.53|1.30|46.41|110.50|2327.30|3700.05|4.48|46.41|64.09|68.57|-2263.21| +2451894|58377|6391|61442|4037|5032|30827|8|132|389|12|70.77|113.23|49.82|508.16|597.84|849.24|1358.76|0.00|508.16|89.68|89.68|-759.56| +2451894|58377|11270|61442|4037|5032|30827|8|224|389|55|28.11|30.63|11.94|0.00|656.70|1546.05|1684.65|52.53|0.00|656.70|709.23|-889.35| +2451894|58377|1001|61442|4037|5032|30827|8|15|389|2|43.52|67.45|30.35|0.00|60.70|87.04|134.90|1.82|0.00|60.70|62.52|-26.34| +2451894|58377|11221|61442|4037|5032|30827|8|31|389|18|73.30|117.28|76.23|0.00|1372.14|1319.40|2111.04|54.88|0.00|1372.14|1427.02|52.74| +2451171|42294|14534|80465|1232840|4415|48390|8|21|390|58|88.90|130.68|88.86|0.00|5153.88|5156.20|7579.44|257.69|0.00|5153.88|5411.57|-2.32| +2451171|42294|2492|80465|1232840|4415|48390|8|232|390|93|52.20|66.29|54.35|0.00|5054.55|4854.60|6164.97|202.18|0.00|5054.55|5256.73|199.95| +2451171|42294|13549|80465|1232840|4415|48390|8|241|390|54|17.78|21.51|10.53|0.00|568.62|960.12|1161.54|34.11|0.00|568.62|602.73|-391.50| +2451171|42294|16826|80465|1232840|4415|48390|8|81|390|80|39.91|70.24|10.53|0.00|842.40|3192.80|5619.20|42.12|0.00|842.40|884.52|-2350.40| +2451171|42294|7312|80465|1232840|4415|48390|8|128|390|19|96.55|126.48|21.50|0.00|408.50|1834.45|2403.12|0.00|0.00|408.50|408.50|-1425.95| +2451171|42294|6688|80465|1232840|4415|48390|8|191|390|79|47.87|85.68|80.53|1908.56|6361.87|3781.73|6768.72|267.19|1908.56|4453.31|4720.50|671.58| +2451171|42294|6392|80465|1232840|4415|48390|8|83|390|1|55.99|76.70|74.39|72.90|74.39|55.99|76.70|0.11|72.90|1.49|1.60|-54.50| +2451171|42294|9913|80465|1232840||48390|||390|84|78.74|117.32||0.00|9066.12|6614.16|9854.88|362.64|0.00|||| +2451171|42294|1183|80465|1232840|4415|48390|8|205|390|42|59.07|103.37|23.77|908.48|998.34|2480.94|4341.54|5.39|908.48|89.86|95.25|-2391.08| +2451171|42294|6571|80465|1232840|4415|48390|8|63|390|1|78.96|126.33|54.32|0.00|54.32|78.96|126.33|3.25|0.00|54.32|57.57|-24.64| +2451171|42294|4495|80465|1232840|4415|48390|8|80|390|26|10.59|15.56|3.26|0.00|84.76|275.34|404.56|0.00|0.00|84.76|84.76|-190.58| +2451990|61951|12797|30804|163593|5182|27218|4|162|391|15|7.79|13.16|7.89|0.00|118.35|116.85|197.40|0.00|0.00|118.35|118.35|1.50| +2451990|61951|5617|30804|163593|5182|27218|4|26|391|76|39.23|50.60|0.00|0.00|0.00|2981.48|3845.60|0.00|0.00|0.00|0.00|-2981.48| +2451990|61951|15247|30804|163593|5182|27218|4|54|391|85|32.12|40.47|3.64|108.29|309.40|2730.20|3439.95|2.01|108.29|201.11|203.12|-2529.09| +2451990|61951|13137|30804|163593|5182|27218|4|259|391|28|93.97|174.78|33.20|0.00|929.60|2631.16|4893.84|65.07|0.00|929.60|994.67|-1701.56| +2451990|61951|10867|30804|163593|5182|27218|4|133|391|40|60.84|90.65|14.50|0.00|580.00|2433.60|3626.00|29.00|0.00|580.00|609.00|-1853.60| +2451990||17569|30804|163593|||4||391||5.32||3.30|0.00|260.70|420.28|768.67||0.00|260.70|284.16|| +2451990|61951|7263|30804|163593|5182|27218|4|223|391|29|23.39|34.14|1.36|23.26|39.44|678.31|990.06|0.00|23.26|16.18|16.18|-662.13| +2451990|61951|8233|30804|163593|5182|27218|4|106|391|80|49.56|92.67|59.30|0.00|4744.00|3964.80|7413.60|332.08|0.00|4744.00|5076.08|779.20| +2451990|61951|271|30804|163593|5182|27218|4|171|391|84|90.27|125.47|82.81|5217.03|6956.04|7582.68|10539.48|121.73|5217.03|1739.01|1860.74|-5843.67| +2451990|61951|16591|30804|163593|5182|27218|4|289|391|59|81.93|122.07|122.07|0.00|7202.13|4833.87|7202.13|288.08|0.00|7202.13|7490.21|2368.26| +2451990|61951|17511|30804|163593|5182|27218|4|140|391|11|41.83|57.72|18.47|0.00|203.17|460.13|634.92|2.03|0.00|203.17|205.20|-256.96| +2451990|61951|2621|30804|163593|5182|27218|4|259|391|91|18.23|30.62|15.92|0.00|1448.72|1658.93|2786.42|86.92|0.00|1448.72|1535.64|-210.21| +2452189|73981|5467|83449|599641|3033|20825|4|118|392|83|83.44|106.80|88.64|0.00|7357.12|6925.52|8864.40|294.28|0.00|7357.12|7651.40|431.60| +2452189|73981|8817|83449|599641|3033|20825|4|4|392|19|45.65|56.60|5.09|0.00|96.71|867.35|1075.40|6.76|0.00|96.71|103.47|-770.64| +2452189||7029|83449|599641||20825|||392||29.16||17.47||663.86|||||||| +2452189|73981|2155|83449|599641|3033|20825|4|297|392|78|14.57|28.70|19.22|0.00|1499.16|1136.46|2238.60|44.97|0.00|1499.16|1544.13|362.70| +2452189|73981|15389|83449|599641|3033|20825|4|48|392|82|96.05|170.96|100.86|0.00|8270.52|7876.10|14018.72|165.41|0.00|8270.52|8435.93|394.42| +2452189|73981|6265|83449|599641|3033|20825|4|243|392|66|84.22|153.28|76.64|0.00|5058.24|5558.52|10116.48|252.91|0.00|5058.24|5311.15|-500.28| +2452189|73981|13073|83449|599641|3033|20825|4|6|392|51|80.23|139.60|62.82|2018.40|3203.82|4091.73|7119.60|23.70|2018.40|1185.42|1209.12|-2906.31| +2452189|73981|4467|83449|599641|3033|20825|4|274|392|11|86.97|112.19|6.73|0.00|74.03|956.67|1234.09|2.22|0.00|74.03|76.25|-882.64| +2452189|73981|7679|83449|599641|3033|20825|4|266|392|34|37.91|70.13|4.90|0.00|166.60|1288.94|2384.42|4.99|0.00|166.60|171.59|-1122.34| +2452189|73981|14279|83449|599641|3033|20825|4|288|392|73|89.72|114.84|55.12|2172.83|4023.76|6549.56|8383.32|55.52|2172.83|1850.93|1906.45|-4698.63| +2452418|29815|5910|66639|283745|498|22004|4|156|393|63|30.98|55.14|14.33|0.00|902.79|1951.74|3473.82|54.16|0.00|902.79|956.95|-1048.95| +2452418|29815|12006|66639|283745|498|22004|4|189|393|3|30.20|39.86|14.74|0.00|44.22|90.60|119.58|0.88|0.00|44.22|45.10|-46.38| +2452418||79|66639|283745|498||4||393|||57.66|||8.07|36.04|57.66|0.14||3.64|3.78|| +2452418|29815|4638|66639|283745|498|22004|4|216|393|4|8.34|9.25|9.25|0.00|37.00|33.36|37.00|1.48|0.00|37.00|38.48|3.64| +2452418|29815|7242|66639|283745|498|22004|4|144|393|31|81.64|90.62|34.43|0.00|1067.33|2530.84|2809.22|96.05|0.00|1067.33|1163.38|-1463.51| +2452418|29815|11257|66639|283745|498|22004|4|99|393|47|92.18|175.14|143.61|0.00|6749.67|4332.46|8231.58|67.49|0.00|6749.67|6817.16|2417.21| +2452418|29815|12397|66639|283745|498|22004|4|256|393|18|29.88|33.46|5.35|0.00|96.30|537.84|602.28|0.96|0.00|96.30|97.26|-441.54| +2452418|29815|9093|66639|283745|498|22004|4|180|393|67|52.03|92.61|2.77|59.38|185.59|3486.01|6204.87|6.31|59.38|126.21|132.52|-3359.80| +2452418|29815|15403|66639|283745|498|22004|4|96|393|6|74.08|111.86|51.45|148.17|308.70|444.48|671.16|3.21|148.17|160.53|163.74|-283.95| +2452418|29815|8181|66639|283745|498|22004|4|227|393|31|95.88|168.74|133.30|123.96|4132.30|2972.28|5230.94|240.50|123.96|4008.34|4248.84|1036.06| +2452256|72949|12265|70303|693852|1848|4597|7|283|394|20|37.26|48.06|5.28|0.00|105.60|745.20|961.20|0.00|0.00|105.60|105.60|-639.60| +||7309||693852|||7|29|394|54|82.44|108.82||23.50|293.76|||10.81|23.50||281.07|| +2452256|72949|12409|70303|693852|1848|4597|7|132|394|84|64.29|124.07|106.70|0.00|8962.80|5400.36|10421.88|537.76|0.00|8962.80|9500.56|3562.44| +2452256|72949|5569|70303|693852|1848|4597|7|255|394|58|52.14|74.56|12.67|0.00|734.86|3024.12|4324.48|58.78|0.00|734.86|793.64|-2289.26| +2452256|72949|9559|70303|693852|1848|4597|7|271|394|25|19.83|31.52|8.82|0.00|220.50|495.75|788.00|15.43|0.00|220.50|235.93|-275.25| +2452256|72949|16789|70303|693852|1848|4597|7|157|394|86|31.39|61.83|57.50|0.00|4945.00|2699.54|5317.38|49.45|0.00|4945.00|4994.45|2245.46| +|72949|13113||||4597||63|394|20|16.32|31.98||||326.40|639.60|||63.80||-262.60| +2452256|72949|133|70303|693852|1848|4597|7|169|394|75|74.57|85.00|56.95|3587.85|4271.25|5592.75|6375.00|47.83|3587.85|683.40|731.23|-4909.35| +2452256|72949|16393|70303|693852|1848|4597|7|143|394|28|48.55|50.49|30.29|0.00|848.12|1359.40|1413.72|42.40|0.00|848.12|890.52|-511.28| +2452256|72949|9943|70303|693852|1848|4597|7|168|394|40|34.74|36.12|3.97|0.00|158.80|1389.60|1444.80|7.94|0.00|158.80|166.74|-1230.80| +2452256|72949|553|70303|693852|1848|4597|7|19|394|9|70.61|122.15|114.82|0.00|1033.38|635.49|1099.35|10.33|0.00|1033.38|1043.71|397.89| +2452256|72949|5489|70303|693852|1848|4597|7|196|394|41|96.70|167.29|78.62|0.00|3223.42|3964.70|6858.89|64.46|0.00|3223.42|3287.88|-741.28| +2452256|72949|15849|70303|693852|1848|4597|7|6|394|6|4.34|7.42|0.37|0.00|2.22|26.04|44.52|0.04|0.00|2.22|2.26|-23.82| +2452256|72949|4459|70303|693852|1848|4597|7|185|394|5|2.41|4.74|4.21|20.83|21.05|12.05|23.70|0.01|20.83|0.22|0.23|-11.83| +2452256|72949|13823|70303|693852|1848|4597|7|30|394|60|57.06|76.46|52.75|664.65|3165.00|3423.60|4587.60|200.02|664.65|2500.35|2700.37|-923.25| +2452256|72949|9137|70303|693852|1848|4597|7|28|394|65|8.10|13.12|12.33|0.00|801.45|526.50|852.80|0.00|0.00|801.45|801.45|274.95| +2452453|52743|5317|12834|1843754|3827|13100|8|163|395|39|56.14|87.57|30.64|0.00|1194.96|2189.46|3415.23|47.79|0.00|1194.96|1242.75|-994.50| +2452453|52743|1081|12834|1843754|3827|13100|8|28|395|73|60.37|63.99|7.67|0.00|559.91|4407.01|4671.27|50.39|0.00|559.91|610.30|-3847.10| +2452453|52743|14589|12834||3827|||4|395|||136.06|66.66|0.00|||8571.78|251.97|0.00|||-1631.70| +2452453|52743|17085|12834|1843754|3827|13100|8|71|395|93|78.92|134.95|134.95|0.00|12550.35|7339.56|12550.35|1004.02|0.00|12550.35|13554.37|5210.79| +2452453|52743|12180|12834|1843754|3827|13100|8|9|395|26|57.80|64.15|59.65|0.00|1550.90|1502.80|1667.90|31.01|0.00|1550.90|1581.91|48.10| +2452453|52743|888|12834|1843754|3827|13100|8|61|395|53|42.75|61.13|45.84|0.00|2429.52|2265.75|3239.89|97.18|0.00|2429.52|2526.70|163.77| +2452453|52743|10453|12834|1843754|3827|13100|8|77|395|75|95.61|171.14|13.69|0.00|1026.75|7170.75|12835.50|71.87|0.00|1026.75|1098.62|-6144.00| +2452453|52743|8748|12834|1843754|3827|13100|8|164|395|32|3.58|5.19|2.59|0.00|82.88|114.56|166.08|0.00|0.00|82.88|82.88|-31.68| +2452453|52743|11031|12834|1843754|3827|13100|8|76|395|60|43.56|55.32|8.29|0.00|497.40|2613.60|3319.20|14.92|0.00|497.40|512.32|-2116.20| +2452453|52743|17922|12834|1843754|3827|13100|8|46|395|84|74.48|123.63|19.78|0.00|1661.52|6256.32|10384.92|49.84|0.00|1661.52|1711.36|-4594.80| +|33652|5627||517253||39984|7|248|396||3.41||0.00||||203.85|0.00||0.00||-153.45| +2452109|33652|5709|69911|517253|595|39984|7|91|396|49|3.66|3.91|2.50|0.00|122.50|179.34|191.59|3.67|0.00|122.50|126.17|-56.84| +2452109|33652|5381|69911|517253|595|39984|7|297|396|10|31.59|52.75|18.99|0.00|189.90|315.90|527.50|5.69|0.00|189.90|195.59|-126.00| +2452109|33652|2563|69911|517253|595|39984|7|158|396|10|25.94|28.27|19.78|0.00|197.80|259.40|282.70|7.91|0.00|197.80|205.71|-61.60| +2452109|33652|4271|69911|517253|595|39984|7|197|396|21|74.68|128.44|35.96|0.00|755.16|1568.28|2697.24|15.10|0.00|755.16|770.26|-813.12| +2452109|33652|14857|69911|517253|595|39984|7|60|396|1|24.41|30.51|10.06|0.00|10.06|24.41|30.51|0.30|0.00|10.06|10.36|-14.35| +|33652|10625|69911|517253||39984||189|396||79.76|90.12|||||180.24|0.00||52.26||| +2452109||13997|||||||396||52.52|81.93|||1394.26|||||1394.26||| +2452109|33652|263|69911|517253|595|39984|7|124|396|12|7.13|11.83|3.78|0.00|45.36|85.56|141.96|4.08|0.00|45.36|49.44|-40.20| +2452109|33652|10099|69911|517253|595|39984|7|276|396|63|62.13|88.84|58.63|0.00|3693.69|3914.19|5596.92|73.87|0.00|3693.69|3767.56|-220.50| +2452109|33652|7843|69911|517253|595|39984|7|259|396|33|21.13|24.51|15.68|0.00|517.44|697.29|808.83|20.69|0.00|517.44|538.13|-179.85| +2452566|43283|17292|42975|464464|5953|7442|4|117|397|64|16.15|26.80|0.53|0.00|33.92|1033.60|1715.20|0.33|0.00|33.92|34.25|-999.68| +2452566|43283|402|42975|464464|5953|7442|4|22|397|12|39.28|67.16|57.08|0.00|684.96|471.36|805.92|47.94|0.00|684.96|732.90|213.60| +2452566|43283|11425|42975|464464|5953|7442|4|231|397|46|49.03|70.11|44.16|0.00|2031.36|2255.38|3225.06|60.94|0.00|2031.36|2092.30|-224.02| +2452566|43283|13533|42975|464464|5953|7442|4|93|397|80|13.58|26.48|11.65|0.00|932.00|1086.40|2118.40|18.64|0.00|932.00|950.64|-154.40| +2452566|43283|16530|42975|464464|5953|7442|4|47|397|94|37.11|63.45|35.53|0.00|3339.82|3488.34|5964.30|100.19|0.00|3339.82|3440.01|-148.52| +2452566|43283|16224|42975|464464|5953|7442|4|163|397|49|2.55|4.25|1.31|35.30|64.19|124.95|208.25|1.15|35.30|28.89|30.04|-96.06| +2452566|43283|13327|42975|464464|5953|7442|4|182|397|92|97.44|143.23|101.69|0.00|9355.48|8964.48|13177.16|93.55|0.00|9355.48|9449.03|391.00| +2452566||13591|||5953||||397|60||25.14||0.00||1069.80|||0.00|||| +|43283|258|42975||||||397|37|||21.02||777.74|1714.95|2880.82|38.88|||816.62|| +2452566|43283|1503|42975|464464|5953|7442|4|37|397|90|9.67|14.11|5.64|0.00|507.60|870.30|1269.90|20.30|0.00|507.60|527.90|-362.70| +2452566|43283|2869|42975|464464|5953|7442|4|171|397|4|89.29|132.14|96.46|0.00|385.84|357.16|528.56|19.29|0.00|385.84|405.13|28.68| +2452566|43283|465|42975|464464|5953|7442|4|276|397|60|21.81|43.18|12.09|0.00|725.40|1308.60|2590.80|50.77|0.00|725.40|776.17|-583.20| +2451532|61929|15824|46188|1506875|5515|23802|2|189|398|10|10.13|12.56|5.02|47.69|50.20|101.30|125.60|0.10|47.69|2.51|2.61|-98.79| +2451532|61929|15818|46188|1506875|5515|23802|2|10|398|13|70.03|86.83|12.15|47.38|157.95|910.39|1128.79|3.31|47.38|110.57|113.88|-799.82| +2451532|61929|9986|46188|1506875|5515|23802|2|221|398|84|98.03|104.89|98.59|0.00|8281.56|8234.52|8810.76|0.00|0.00|8281.56|8281.56|47.04| +2451532|61929|16382|46188|1506875|5515|23802|2|246|398|100|7.89|8.20|0.98|0.00|98.00|789.00|820.00|6.86|0.00|98.00|104.86|-691.00| +2451532|61929|3961|46188|1506875|5515|23802|2|11|398|44|4.92|5.31|0.37|0.00|16.28|216.48|233.64|1.30|0.00|16.28|17.58|-200.20| +2451532|61929|7442|46188|1506875|5515|23802|2|103|398|34|20.08|24.29|13.11|0.00|445.74|682.72|825.86|4.45|0.00|445.74|450.19|-236.98| +2451532|61929|10447|46188|1506875|5515|23802|2|2|398|45|86.22|167.26|138.82|0.00|6246.90|3879.90|7526.70|499.75|0.00|6246.90|6746.65|2367.00| +2451532|61929|4436|46188|1506875|5515|23802|2|171|398|7|75.46|116.20|82.50|11.55|577.50|528.22|813.40|33.95|11.55|565.95|599.90|37.73| +2451532|61929|4196|46188|1506875|5515|23802|2|152|398|45|23.61|35.65|15.68|0.00|705.60|1062.45|1604.25|35.28|0.00|705.60|740.88|-356.85| +2451532|61929|12001|46188|1506875|5515|23802|2|101|398|85|72.09|98.04|1.96|0.00|166.60|6127.65|8333.40|0.00|0.00|166.60|166.60|-5961.05| +2450945|37892|10714|24949|1322308|6514|29473|7|71|399|86|39.40|49.25|45.80|157.55|3938.80|3388.40|4235.50|37.81|157.55|3781.25|3819.06|392.85| +2450945|37892|16970|24949|1322308|6514|29473|7|193|399|15|42.95|68.29|33.46|326.23|501.90|644.25|1024.35|8.78|326.23|175.67|184.45|-468.58| +2450945|37892|10928|24949|1322308|6514|29473|7|295|399|59|48.02|66.26|1.98|0.00|116.82|2833.18|3909.34|3.50|0.00|116.82|120.32|-2716.36| +2450945|37892|2996|24949|1322308|6514|29473|7|142|399|72|81.35|130.97|13.09|0.00|942.48|5857.20|9429.84|65.97|0.00|942.48|1008.45|-4914.72| +2450945|37892|13171|24949|1322308|6514|29473|7|163|399|6|82.27|111.88|86.14|196.39|516.84|493.62|671.28|22.43|196.39|320.45|342.88|-173.17| +2450945|37892|7282|24949|1322308|6514|29473|7|84|399|35|98.84|100.81|64.51|248.36|2257.85|3459.40|3528.35|140.66|248.36|2009.49|2150.15|-1449.91| +||15337|24949|1322308||29473|7|206|399||29.31|53.05||||||36.07||3607.20||1262.40| +2450945|37892|17125|24949|1322308|6514|29473|7|292|399|85|31.96|33.87|21.33|0.00|1813.05|2716.60|2878.95|18.13|0.00|1813.05|1831.18|-903.55| +2450945|37892|14780|24949|1322308|6514|29473|7|109|399|70|88.47|157.47|33.06|0.00|2314.20|6192.90|11022.90|46.28|0.00|2314.20|2360.48|-3878.70| +2450945|37892|17449|24949|1322308|6514|29473|7|199|399|14|30.90|48.51|40.74|0.00|570.36|432.60|679.14|39.92|0.00|570.36|610.28|137.76| +2450945|37892|12764|24949|1322308|6514|29473|7|157|399|61|38.18|54.59|4.36|0.00|265.96|2328.98|3329.99|18.61|0.00|265.96|284.57|-2063.02| +2450945|37892|5936|24949|1322308|6514|29473|7|79|399|58|30.12|42.77|4.27|0.00|247.66|1746.96|2480.66|14.85|0.00|247.66|262.51|-1499.30| +2451444|33108|16948|57408|1722378|4948|41518|8|10|400|44|4.02|4.90|3.33|0.00|146.52|176.88|215.60|1.46|0.00|146.52|147.98|-30.36| +2451444|33108|10802|57408|1722378|4948|41518|8|260|400|36|23.03|40.76|39.94|0.00|1437.84|829.08|1467.36|129.40|0.00|1437.84|1567.24|608.76| +2451444|33108|13183|57408|1722378|4948|41518|8|176|400|17|79.01|96.39|38.55|0.00|655.35|1343.17|1638.63|6.55|0.00|655.35|661.90|-687.82| +2451444|33108|12064|57408|1722378|4948|41518|8|287|400|49|20.97|25.79|18.31|0.00|897.19|1027.53|1263.71|8.97|0.00|897.19|906.16|-130.34| +2451444|33108|11701|57408|1722378|4948|41518|8|280|400|12|44.09|76.27|0.76|0.00|9.12|529.08|915.24|0.72|0.00|9.12|9.84|-519.96| +2451444|33108|4646|57408|1722378|4948|41518|8|215|400|64|22.02|34.13|9.89|0.00|632.96|1409.28|2184.32|25.31|0.00|632.96|658.27|-776.32| +2451444|33108|15829|57408|1722378|4948|41518|8|296|400|40|73.01|135.06|25.66|0.00|1026.40|2920.40|5402.40|92.37|0.00|1026.40|1118.77|-1894.00| +2451444|33108|7039|57408|1722378|4948|41518|8|191|400|45|95.45|178.49|35.69|0.00|1606.05|4295.25|8032.05|80.30|0.00|1606.05|1686.35|-2689.20| +2452219|67892|15333|57962|79562|3562|44362|7|294|401|56|20.95|39.17|1.95|60.06|109.20|1173.20|2193.52|3.43|60.06|49.14|52.57|-1124.06| +2452219|67892|4833|57962|79562|3562|44362|7|12|401|68|54.58|104.79|82.78|2195.32|5629.04|3711.44|7125.72|171.68|2195.32|3433.72|3605.40|-277.72| +||10487|57962|79562|||||401|47|99.04|||4130.46|4172.19|4654.88||2.92|4130.46|41.73||| +2452219|67892|5905|57962|79562|3562|44362|7|25|401|76|21.11|29.76|2.38|177.26|180.88|1604.36|2261.76|0.10|177.26|3.62|3.72|-1600.74| +2452219|67892|8955|57962|79562|3562|44362|7|45|401|66|94.96|171.87|65.31|0.00|4310.46|6267.36|11343.42|43.10|0.00|4310.46|4353.56|-1956.90| +2452219|67892|15179|57962|79562|3562|44362|7|242|401|43|53.49|83.44|41.72|0.00|1793.96|2300.07|3587.92|161.45|0.00|1793.96|1955.41|-506.11| +2452219|67892|4795|57962|79562|3562|44362|7|95|401|94|6.02|8.36|6.68|0.00|627.92|565.88|785.84|12.55|0.00|627.92|640.47|62.04| +2452219|67892|17255|57962|79562|3562|44362|7|157|401|23|86.66|121.32|25.47|0.00|585.81|1993.18|2790.36|41.00|0.00|585.81|626.81|-1407.37| +2452219|67892|12571|57962|79562|3562|44362|7|265|401|100|31.51|55.14|51.83|0.00|5183.00|3151.00|5514.00|0.00|0.00|5183.00|5183.00|2032.00| +2452219|67892|10177|57962|79562|3562|44362|7|131|401|24|41.97|73.02|51.11|0.00|1226.64|1007.28|1752.48|49.06|0.00|1226.64|1275.70|219.36| +2452219|67892|8597|57962|79562|3562|44362|7|118|401|26|50.41|53.93|20.49|0.00|532.74|1310.66|1402.18|21.30|0.00|532.74|554.04|-777.92| +2452219|67892|3621|57962|79562|3562|44362|7|185|401|50|13.63|21.53|10.33|0.00|516.50|681.50|1076.50|46.48|0.00|516.50|562.98|-165.00| +2451530|59297|11002|26512|1020325|3385|4445|8|256|402|97|61.88|97.77|64.52|0.00|6258.44|6002.36|9483.69|312.92|0.00|6258.44|6571.36|256.08| +2451530|59297|4904|26512|1020325|3385|4445|8|204|402|71|72.05|97.98|7.83|366.91|555.93|5115.55|6956.58|9.45|366.91|189.02|198.47|-4926.53| +2451530|59297|4540|26512|1020325|3385|4445|8|286|402|44|90.05|140.47|28.09|0.00|1235.96|3962.20|6180.68|98.87|0.00|1235.96|1334.83|-2726.24| +2451530|59297|9050|26512|1020325|3385|4445|8|32|402|66|97.27|147.85|59.14|0.00|3903.24|6419.82|9758.10|312.25|0.00|3903.24|4215.49|-2516.58| +2451530|59297|1561|26512|1020325|3385|4445|8|11|402|3|64.16|121.26|72.75|181.14|218.25|192.48|363.78|0.74|181.14|37.11|37.85|-155.37| +2451530|59297|3040|26512|1020325|3385|4445|8|163|402|35|85.70|154.26|57.07|0.00|1997.45|2999.50|5399.10|179.77|0.00|1997.45|2177.22|-1002.05| +2451530|59297|14858|26512|1020325|3385|4445|8|133|402|83|14.57|18.21|3.09|153.88|256.47|1209.31|1511.43|3.07|153.88|102.59|105.66|-1106.72| +2451530|59297|2788|26512|1020325|3385|4445|8|74|402|88|48.52|50.46|49.45|0.00|4351.60|4269.76|4440.48|348.12|0.00|4351.60|4699.72|81.84| +2451530|59297|17918|26512|1020325|3385|4445|8|282|402|67|41.79|43.46|31.72|722.58|2125.24|2799.93|2911.82|0.00|722.58|1402.66|1402.66|-1397.27| +2451530|59297|3140|26512|1020325|3385|4445|8|202|402|17|33.94|50.91|3.56|1.81|60.52|576.98|865.47|2.93|1.81|58.71|61.64|-518.27| +2451530|59297|6956|26512|1020325|3385|4445|8|52|402|24|86.27|115.60|110.97|0.00|2663.28|2070.48|2774.40|133.16|0.00|2663.28|2796.44|592.80| +2451530|59297|3352|26512|1020325|3385|4445|8|138|402|8|31.69|38.02|20.53|0.00|164.24|253.52|304.16|13.13|0.00|164.24|177.37|-89.28| +2451530|59297|358|26512|1020325|3385|4445|8|269|402|41|96.95|171.60|130.41|4651.72|5346.81|3974.95|7035.60|34.75|4651.72|695.09|729.84|-3279.86| +|59297|8420|||3385|4445|||402|84|48.20|92.54|79.58||6684.72|4048.80||133.69||6684.72|6818.41|2635.92| +2451530|59297|1724|26512|1020325|3385|4445|8|171|402|50|83.82|151.71|4.55|184.27|227.50|4191.00|7585.50|3.89|184.27|43.23|47.12|-4147.77| +2451530|59297|17827|26512|1020325|3385|4445|8|296|402|58|43.52|66.58|11.98|0.00|694.84|2524.16|3861.64|6.94|0.00|694.84|701.78|-1829.32| +2451855|34148|15269|51007|1394084|4361|37838|1|11|403|47|10.29|13.58|10.99|0.00|516.53|483.63|638.26|20.66|0.00|516.53|537.19|32.90| +2451855|34148|4781|51007|1394084|4361|37838|1|61|403|97|77.20|148.22|29.64|0.00|2875.08|7488.40|14377.34|258.75|0.00|2875.08|3133.83|-4613.32| +2451855|34148|14462|51007|1394084|4361|37838|1|256|403|82|75.97|151.94|129.14|0.00|10589.48|6229.54|12459.08|0.00|0.00|10589.48|10589.48|4359.94| +2451855|34148|16406|51007|1394084|4361|37838|1|69|403|14|47.00|66.74|36.70|0.00|513.80|658.00|934.36|20.55|0.00|513.80|534.35|-144.20| +2451855||10313|51007|1394084|4361||1|203|403|6|10.10|||0.00|||98.16||0.00|20.58||| +2451855|34148|9113|51007|1394084|4361|37838|1|187|403|89|6.85|9.59|8.24|528.01|733.36|609.65|853.51|4.10|528.01|205.35|209.45|-404.30| +2451855|34148|8245|51007|1394084|4361|37838|1|237|403|57|98.16|161.96|157.10|0.00|8954.70|5595.12|9231.72|179.09|0.00|8954.70|9133.79|3359.58| +2451855|34148|11648|51007|1394084|4361|37838|1|107|403|61|35.95|41.70|12.92|94.57|788.12|2192.95|2543.70|48.54|94.57|693.55|742.09|-1499.40| +2451855|34148|13154|51007|||37838|||403|67|34.86||55.51|0.00|3719.17|2335.62||111.57|0.00||3830.74|1383.55| +||15469|51007|1394084|||||403|||113.94|5.69||324.33||6494.58|19.45||324.33||| +2451855|34148|13249|51007|1394084|4361|37838|1|68|403|93|33.76|50.30|30.18|0.00|2806.74|3139.68|4677.90|196.47|0.00|2806.74|3003.21|-332.94| +||4058|51007|||37838|1||403||64.82|85.56||0.00|680.10||1283.40|61.20|0.00|680.10|741.30|| +2451855|34148|10303|51007|1394084|4361|37838|1|151|403|50|13.49|21.58|11.86|0.00|593.00|674.50|1079.00|11.86|0.00|593.00|604.86|-81.50| +2451556|52411|1082|27157|333548|5539|8777|7|180|404|78|37.13|58.29|39.05|0.00|3045.90|2896.14|4546.62|182.75|0.00|3045.90|3228.65|149.76| +2451556|52411|767|27157|333548|5539|8777|7|14|404|72|93.71|115.26|28.81|0.00|2074.32|6747.12|8298.72|20.74|0.00|2074.32|2095.06|-4672.80| +2451556|52411|16136|27157|333548|5539|8777|7|116|404|33|28.69|30.41|0.00|0.00|0.00|946.77|1003.53|0.00|0.00|0.00|0.00|-946.77| +2451556|52411|12371|27157|333548|5539|8777|7|105|404|38|76.36|104.61|20.92|0.00|794.96|2901.68|3975.18|47.69|0.00|794.96|842.65|-2106.72| +2451556|52411|15799|27157|333548|5539|8777|7|2|404|81|2.53|4.47|4.11|243.02|332.91|204.93|362.07|8.09|243.02|89.89|97.98|-115.04| +2451556|52411|14276|27157|333548|5539|8777|7|203|404|43|27.58|41.37|10.75|147.92|462.25|1185.94|1778.91|15.71|147.92|314.33|330.04|-871.61| +2451556|52411|13688|27157|333548|5539|8777|7|154|404|87|95.77|143.65|93.37|0.00|8123.19|8331.99|12497.55|487.39|0.00|8123.19|8610.58|-208.80| +2451556|52411|5557|27157|333548|5539|8777|7|50|404|38|14.82|28.00|14.00|0.00|532.00|563.16|1064.00|31.92|0.00|532.00|563.92|-31.16| +2451556|52411|4265|27157|333548|5539|8777|7|93|404|72|48.90|78.72|40.93|0.00|2946.96|3520.80|5667.84|265.22|0.00|2946.96|3212.18|-573.84| +2451556|52411|10886|27157|333548|5539|8777|7|32|404|61|91.92|164.53|154.65|3584.78|9433.65|5607.12|10036.33|292.44|3584.78|5848.87|6141.31|241.75| +2451141|56243|6610|72256|1880547|1991|35882|7|154|405|94|75.15|136.02|129.21|0.00|12145.74|7064.10|12785.88|0.00|0.00|12145.74|12145.74|5081.64| +2451141|56243|7801|72256|1880547|1991|35882|7|68|405|72|58.77|101.67|88.45|0.00|6368.40|4231.44|7320.24|191.05|0.00|6368.40|6559.45|2136.96| +2451141|56243|6284|72256|1880547|1991|35882|7|192|405|98|4.42|8.75|6.82|0.00|668.36|433.16|857.50|53.46|0.00|668.36|721.82|235.20| +2451141|56243|5839|72256|1880547|1991|35882|7|35|405|84|64.86|128.42|109.15|0.00|9168.60|5448.24|10787.28|183.37|0.00|9168.60|9351.97|3720.36| +2451141|56243|5689|72256|1880547|1991|35882|7|132|405|32|56.28|93.42|70.06|0.00|2241.92|1800.96|2989.44|22.41|0.00|2241.92|2264.33|440.96| +2451141|56243|9610|72256|1880547|1991|35882|7|175|405|33|14.99|18.28|12.43|0.00|410.19|494.67|603.24|20.50|0.00|410.19|430.69|-84.48| +2451141|56243|15997||1880547|||7|2|405||31.82||16.92|0.00|592.20|1113.70|2115.75||0.00|592.20|592.20|-521.50| +2451141|56243|8821|72256|1880547|1991|35882|7|101|405|54|21.76|42.64|24.73|0.00|1335.42|1175.04|2302.56|53.41|0.00|1335.42|1388.83|160.38| +|56243|3242|72256|||35882|7|247|405|83|20.49|29.50||0.00|2227.72|1700.67|||0.00|2227.72|2272.27|| +2451141|56243|17776|72256|1880547|1991|35882|7|86|405|79|47.17|91.50|17.38|54.92|1373.02|3726.43|7228.50|26.36|54.92|1318.10|1344.46|-2408.33| +2451141|56243|8122|72256|1880547|1991|35882|7|123|405|25|54.27|98.77|67.16|0.00|1679.00|1356.75|2469.25|16.79|0.00|1679.00|1695.79|322.25| +2451141|56243|17740|72256|1880547|1991|35882|7|219|405|80|98.55|193.15|104.30|0.00|8344.00|7884.00|15452.00|166.88|0.00|8344.00|8510.88|460.00| +2451141|56243|4327|72256|1880547|1991|35882|7|59|405|29|93.74|104.98|89.23|0.00|2587.67|2718.46|3044.42|232.89|0.00|2587.67|2820.56|-130.79| +2451141|56243|4346|72256|1880547|1991|35882|7|54|405|100|99.13|157.61|9.45|0.00|945.00|9913.00|15761.00|85.05|0.00|945.00|1030.05|-8968.00| +2451141|56243|17872|72256|1880547|1991|35882|7|134|405|74|88.32|141.31|8.47|0.00|626.78|6535.68|10456.94|50.14|0.00|626.78|676.92|-5908.90| +2451141|56243|2401|72256|1880547|1991|35882|7|7|405|16|47.39|50.70|41.06|0.00|656.96|758.24|811.20|45.98|0.00|656.96|702.94|-101.28| +2451536|36780|15788|7409|1042776|1790|32757|8|145|406|18|30.91|53.16|9.56|0.00|172.08|556.38|956.88|15.48|0.00|172.08|187.56|-384.30| +||781|7409|1042776||32757||241|406|57|64.73|68.61|0.00|0.00|0.00||3910.77|0.00|0.00|0.00||-3689.61| +2451536|36780|15790|7409|1042776|1790|32757|8|167|406|26|90.12|123.46|69.13|0.00|1797.38|2343.12|3209.96|0.00|0.00|1797.38|1797.38|-545.74| +2451536|36780|4600|7409|1042776|1790|32757|8|106|406|76|59.57|66.12|59.50|0.00|4522.00|4527.32|5025.12|361.76|0.00|4522.00|4883.76|-5.32| +2451536|36780|14978|7409|1042776|1790|32757|8|169|406|54|31.35|34.79|20.52|0.00|1108.08|1692.90|1878.66|88.64|0.00|1108.08|1196.72|-584.82| +2451536|36780|127|7409|1042776|1790|32757|8|126|406|13|73.82|115.89|82.28|374.37|1069.64|959.66|1506.57|27.81|374.37|695.27|723.08|-264.39| +2451536|36780|17725|7409|1042776|1790|32757|8|33|406|22|77.50|140.27|124.84|0.00|2746.48|1705.00|3085.94|219.71|0.00|2746.48|2966.19|1041.48| +2451536|36780|4256|7409|1042776|1790|32757|8|179|406|16|27.42|54.56|40.92|373.19|654.72|438.72|872.96|5.63|373.19|281.53|287.16|-157.19| +2451536|36780|8785|7409|1042776|1790|32757|8|41|406|13|2.75|5.50|0.93|0.12|12.09|35.75|71.50|0.59|0.12|11.97|12.56|-23.78| +2452515|63665|14601|43504|402142|5133|22971|8|71|407|57|72.67|103.91|44.68|2241.14|2546.76|4142.19|5922.87|6.11|2241.14|305.62|311.73|-3836.57| +2452515|63665|11274|43504|402142|5133|22971|8|49|407|66|58.94|67.19|34.93|2028.73|2305.38|3890.04|4434.54|24.89|2028.73|276.65|301.54|-3613.39| +2452515|63665|4075|43504|402142|5133|22971|8|250|407|25|29.12|50.08|11.51|46.04|287.75|728.00|1252.00|14.50|46.04|241.71|256.21|-486.29| +2452515|63665|9132|43504|402142|5133|22971|8|25|407|55|19.36|31.75|6.66|0.00|366.30|1064.80|1746.25|14.65|0.00|366.30|380.95|-698.50| +2452515|63665|2496|43504|402142|5133|22971|8|157|407|86|20.40|28.76|2.87|0.00|246.82|1754.40|2473.36|0.00|0.00|246.82|246.82|-1507.58| +2452515|63665|7651|43504|402142|5133|22971|8|11|407|69|90.02|152.13|45.63|0.00|3148.47|6211.38|10496.97|0.00|0.00|3148.47|3148.47|-3062.91| +2452515|63665|3483|43504|402142|5133|22971|8|183|407|79|89.37|99.20|55.55|0.00|4388.45|7060.23|7836.80|87.76|0.00|4388.45|4476.21|-2671.78| +2452515|63665|4050|43504|402142|5133|22971|8|94|407|90|27.70|31.85|1.59|0.00|143.10|2493.00|2866.50|5.72|0.00|143.10|148.82|-2349.90| +2452515|63665|10189|43504|402142|5133|22971|8|263|407|59|61.83|88.41|49.50|0.00|2920.50|3647.97|5216.19|175.23|0.00|2920.50|3095.73|-727.47| +2452515|63665|147|43504|402142|5133|22971|8|284|407|96|89.35|150.10|9.00|0.00|864.00|8577.60|14409.60|60.48|0.00|864.00|924.48|-7713.60| +2452515|63665|12105|43504|402142|5133|22971|8|120|407|26|45.54|59.65|35.79|0.00|930.54|1184.04|1550.90|55.83|0.00|930.54|986.37|-253.50| +2452515|63665|16207|43504|402142|5133|22971|8|228|407|65|76.27|106.77|23.48|1312.53|1526.20|4957.55|6940.05|19.23|1312.53|213.67|232.90|-4743.88| +2452515|63665|5532|43504|402142|5133|22971|8|73|407|52|33.55|45.62|5.01|0.00|260.52|1744.60|2372.24|10.42|0.00|260.52|270.94|-1484.08| +2451080|68431|1921|65390|198581|2172|33410|1|146|408|82|37.53|43.91|36.00|0.00|2952.00|3077.46|3600.62|29.52|0.00|2952.00|2981.52|-125.46| +2451080|68431|16234|65390|198581|2172|33410|1|46|408|50|80.36|85.98|74.80|0.00|3740.00|4018.00|4299.00|187.00|0.00|3740.00|3927.00|-278.00| +2451080|68431|11905|65390|198581|2172|33410|1|166|408|65|21.54|37.91|36.77|501.91|2390.05|1400.10|2464.15|169.93|501.91|1888.14|2058.07|488.04| +2451080|68431|5215|65390|198581|2172|33410|1|113|408|78|74.16|83.80|56.98|0.00|4444.44|5784.48|6536.40|44.44|0.00|4444.44|4488.88|-1340.04| +2451080|68431|15010|65390|198581|2172|33410|1|91|408|53|18.18|24.36|3.41|101.20|180.73|963.54|1291.08|1.59|101.20|79.53|81.12|-884.01| +2451080|68431|9205|65390|198581|2172|33410|1|219|408|15|91.26|92.17|3.68|0.00|55.20|1368.90|1382.55|1.65|0.00|55.20|56.85|-1313.70| +2451080|68431|8278|65390|198581|2172|33410|1|244|408|96|74.78|100.95|70.66|0.00|6783.36|7178.88|9691.20|135.66|0.00|6783.36|6919.02|-395.52| +2451080|68431|928|65390|198581|2172|33410|1|81|408|17|31.74|61.25|50.83|0.00|864.11|539.58|1041.25|0.00|0.00|864.11|864.11|324.53| +2451080|68431|16048|65390|198581|2172|33410|1|283|408|20|58.23|100.73|14.10|0.00|282.00|1164.60|2014.60|5.64|0.00|282.00|287.64|-882.60| +2451080|68431|14755|65390|198581|2172|33410|1|95|408|10|27.76|29.98|22.48|38.21|224.80|277.60|299.80|3.73|38.21|186.59|190.32|-91.01| +2451080|68431|10477|65390|198581|2172|33410|1|185|408|43|11.21|11.21|0.89|0.00|38.27|482.03|482.03|0.76|0.00|38.27|39.03|-443.76| +2450959|71343|6128||1473819||7109|10||409|82|27.64|43.11|15.95|0.00|1307.90|||117.71|0.00|||| +2450959|71343|14402|63774|1473819|3064|7109|10|117|409|54|35.25|38.07|6.09|144.69|328.86|1903.50|2055.78|9.20|144.69|184.17|193.37|-1719.33| +2450959|71343|12980|63774|1473819|3064|7109|10|104|409|75|3.03|6.06|4.78|0.00|358.50|227.25|454.50|28.68|0.00|358.50|387.18|131.25| +2450959|71343|151|63774|1473819|3064|7109|10|79|409|37|23.08|26.54|9.55|0.00|353.35|853.96|981.98|3.53|0.00|353.35|356.88|-500.61| +2450959|71343|1072|63774|1473819|3064|7109|10|42|409|6|53.46|85.00|58.65|0.00|351.90|320.76|510.00|24.63|0.00|351.90|376.53|31.14| +2450959|71343|11011|63774|1473819|3064|7109|10|76|409|25|49.59|51.57|1.03|1.80|25.75|1239.75|1289.25|1.43|1.80|23.95|25.38|-1215.80| +2450959|71343|17947|63774|1473819|3064|7109|10|109|409|49|70.47|109.93|94.53|0.00|4631.97|3453.03|5386.57|0.00|0.00|4631.97|4631.97|1178.94| +2450959|71343|1330|63774|1473819|3064|7109|10|290|409|17|75.90|100.94|5.04|0.00|85.68|1290.30|1715.98|1.71|0.00|85.68|87.39|-1204.62| +2450959|71343|6980|63774|1473819|3064|7109|10|240|409|29|95.28|110.52|32.05|0.00|929.45|2763.12|3205.08|65.06|0.00|929.45|994.51|-1833.67| +2451230|39403|4874|97273|549580|5440|49347|4|177|410|14|6.10|7.13|4.77|0.00|66.78|85.40|99.82|0.00|0.00|66.78|66.78|-18.62| +2451230|39403|17824|97273|549580|5440|49347|4|247|410|90|37.25|53.64|46.13|0.00|4151.70|3352.50|4827.60|124.55|0.00|4151.70|4276.25|799.20| +2451230|39403|7022|97273|549580|5440|49347|4|120|410|30|30.04|48.66|3.40|44.88|102.00|901.20|1459.80|0.57|44.88|57.12|57.69|-844.08| +2451230|39403|17065|97273|549580|5440|49347|4|36|410|10|2.77|3.73|2.61|0.00|26.10|27.70|37.30|0.26|0.00|26.10|26.36|-1.60| +2451230|39403|17473|97273|549580|5440|49347|4|7|410|66|80.81|138.99|119.53|0.00|7888.98|5333.46|9173.34|236.66|0.00|7888.98|8125.64|2555.52| +2451230|39403|2269|97273|549580|5440|49347|4|134|410|91|76.82|104.47|38.65|0.00|3517.15|6990.62|9506.77|35.17|0.00|3517.15|3552.32|-3473.47| +2451230|39403|11276|97273|549580|5440|49347|4|254|410|88|45.17|62.78|53.99|3420.80|4751.12|3974.96|5524.64|39.90|3420.80|1330.32|1370.22|-2644.64| +2451230|39403|2096|97273|549580|5440|49347|4|285|410|35|80.94|136.78|27.35|0.00|957.25|2832.90|4787.30|0.00|0.00|957.25|957.25|-1875.65| +2451230|39403|9457|97273|549580|5440|49347|4|230|410|38|51.78|98.89|45.48|0.00|1728.24|1967.64|3757.82|51.84|0.00|1728.24|1780.08|-239.40| +2451230|39403|15278||549580|5440|||193|410|73||28.14||||||||1724.99||| +|39403|17906||||49347|4|201|410||16.37||||118.20|163.70||||||| +2451230|39403|12830|97273|549580|5440|49347|4|129|410|25|29.26|45.64|1.82|0.00|45.50|731.50|1141.00|1.36|0.00|45.50|46.86|-686.00| +2451230|39403|6122|97273|549580|5440|49347|4|137|410|99|10.20|18.56|11.32|0.00|1120.68|1009.80|1837.44|78.44|0.00|1120.68|1199.12|110.88| +2452071|63155|7373|95080|1085999|1271|28990|7|150|411|22|27.81|38.65|27.82|140.76|612.04|611.82|850.30|37.70|140.76|471.28|508.98|-140.54| +2452071|63155|875|95080|||28990|7||411||||69.68||2926.56|2651.04||234.12||||275.52| +2452071|63155|1053|95080|1085999|1271|28990|7|287|411|80|82.47|136.90|56.12|0.00|4489.60|6597.60|10952.00|224.48|0.00|4489.60|4714.08|-2108.00| +2452071|63155|4903|95080|1085999|1271|28990|7|158|411|15|25.26|46.47|28.34|0.00|425.10|378.90|697.05|21.25|0.00|425.10|446.35|46.20| +2452071|63155|15115|95080|1085999|1271|28990|7|225|411|48|74.55|122.26|48.90|1408.32|2347.20|3578.40|5868.48|18.77|1408.32|938.88|957.65|-2639.52| +2452071|63155|9343|95080|1085999|1271|28990|7|269|411|23|74.33|141.97|105.05|0.00|2416.15|1709.59|3265.31|48.32|0.00|2416.15|2464.47|706.56| +2452071|63155|2951|95080|1085999|1271|28990|7|222|411|23|39.93|65.88|47.43|0.00|1090.89|918.39|1515.24|10.90|0.00|1090.89|1101.79|172.50| +2452071|63155|2289||||28990||260|411||53.86|96.94|74.64||3433.44||4459.24|||3433.44||955.88| +2452071|63155|16613|95080|1085999|1271|28990|7|49|411|75|17.68|25.45|17.56|0.00|1317.00|1326.00|1908.75|105.36|0.00|1317.00|1422.36|-9.00| +2452071|63155|17671|95080||1271|28990|7|112|411|97||94.40||0.00||7044.14|9156.80||0.00|4578.40||| +2452071|63155|6653|95080|1085999|1271|28990|7|184|411|55|15.69|30.59|14.37|0.00|790.35|862.95|1682.45|7.90|0.00|790.35|798.25|-72.60| +2452071|63155|7635|95080|1085999|1271|28990|7|26|411|83|14.42|21.63|0.86|0.00|71.38|1196.86|1795.29|1.42|0.00|71.38|72.80|-1125.48| +2452071|63155|2625|95080|1085999|1271|28990|7|133|411|78|55.38|78.63|22.01|0.00|1716.78|4319.64|6133.14|34.33|0.00|1716.78|1751.11|-2602.86| +2452071|63155|2003|95080|1085999|1271|28990|7|104|411|6|12.60|16.63|2.32|0.00|13.92|75.60|99.78|0.00|0.00|13.92|13.92|-61.68| +2452071|63155|10521|95080|1085999|1271|28990|7|178|411|68|75.55|146.56|24.91|0.00|1693.88|5137.40|9966.08|152.44|0.00|1693.88|1846.32|-3443.52| +2452071|63155|10969|95080|1085999|1271|28990|7|180|411|33|38.01|47.89|42.14|458.90|1390.62|1254.33|1580.37|18.63|458.90|931.72|950.35|-322.61| +2452187|48439|16293|47578|1064306|681|41456|7|114|412|39|4.55|7.23|3.90|0.00|152.10|177.45|281.97|10.64|0.00|152.10|162.74|-25.35| +2452187|48439|12559|47578|1064306|681|41456|7|51|412|20|82.80|149.86|100.40|0.00|2008.00|1656.00|2997.20|80.32|0.00|2008.00|2088.32|352.00| +|48439|15071||1064306||||266|412|76|96.00|174.72|17.47||1327.72||13278.72|119.49|||1447.21|-5968.28| +2452187|48439|15511|47578|1064306|681|41456|7|48|412|71|4.59|7.11|6.18|0.00|438.78|325.89|504.81|26.32|0.00|438.78|465.10|112.89| +2452187|48439|1279|47578|1064306|681|41456|7|218|412|48|26.37|51.15|41.43|0.00|1988.64|1265.76|2455.20|19.88|0.00|1988.64|2008.52|722.88| +2452187|48439|9339|47578|1064306|681|41456|7|43|412|9|28.92|52.92|1.05|0.00|9.45|260.28|476.28|0.37|0.00|9.45|9.82|-250.83| +|48439|6595|47578|1064306|||7|264|412|||72.17|31.75||3143.25||7144.83|||3143.25|3237.54|-1407.78| +2452187|48439|14019|47578|1064306|681|41456|7|63|412|87|90.51|94.13|19.76|361.01|1719.12|7874.37|8189.31|54.32|361.01|1358.11|1412.43|-6516.26| +2452187|48439|15317|47578|1064306|681|41456|7|6|412|38|78.08|106.96|23.53|89.41|894.14|2967.04|4064.48|48.28|89.41|804.73|853.01|-2162.31| +2452187|48439|17283|47578|1064306|681|41456|7|110|412|36|80.66|107.27|65.43|0.00|2355.48|2903.76|3861.72|70.66|0.00|2355.48|2426.14|-548.28| +2452187|48439|17435|47578|1064306|681|41456|7|98|412|52|37.43|38.55|8.09|0.00|420.68|1946.36|2004.60|25.24|0.00|420.68|445.92|-1525.68| +2452187|48439|16819|47578|1064306|681|41456|7|117|412|86|20.59|32.94|28.98|99.69|2492.28|1770.74|2832.84|95.70|99.69|2392.59|2488.29|621.85| +2451176|46428|12062|31435|286564|6046|19928|1|148|413|90|25.29|34.90|34.55|1337.08|3109.50|2276.10|3141.00|35.44|1337.08|1772.42|1807.86|-503.68| +2451176|46428|10256|31435|286564|6046|19928|1|284|413|33|70.77|129.50|117.84|0.00|3888.72|2335.41|4273.50|311.09|0.00|3888.72|4199.81|1553.31| +2451176|46428|14888|31435|286564|6046|19928|1|296|413|73|53.26|90.54|90.54|0.00|6609.42|3887.98|6609.42|330.47|0.00|6609.42|6939.89|2721.44| +2451176|46428|7165|31435|286564|6046|19928|1|48|413|53|43.70|62.49|8.12|0.00|430.36|2316.10|3311.97|34.42|0.00|430.36|464.78|-1885.74| +2451176|46428|17810|31435|286564|6046|19928|1|56|413|2|55.79|104.88|30.41|0.00|60.82|111.58|209.76|0.60|0.00|60.82|61.42|-50.76| +2451176|46428|13018|31435|286564|6046|19928|1|43|413|4|48.04|86.47|4.32|0.00|17.28|192.16|345.88|0.51|0.00|17.28|17.79|-174.88| +2451176|46428|15502|31435|286564|6046|19928|1|142|413|17|22.77|38.70|28.25|0.00|480.25|387.09|657.90|38.42|0.00|480.25|518.67|93.16| +2451176|46428|2042|31435|286564|6046|19928|1|137|413|70|91.13|179.52|30.51|0.00|2135.70|6379.10|12566.40|106.78|0.00|2135.70|2242.48|-4243.40| +2451176|46428|1372|31435|286564|6046|19928|1|121|413|100|74.62|79.09|71.97|0.00|7197.00|7462.00|7909.00|71.97|0.00|7197.00|7268.97|-265.00| +2451176|46428|13987|31435|286564|6046|19928|1|158|413|53|75.39|139.47|47.41|0.00|2512.73|3995.67|7391.91|150.76|0.00|2512.73|2663.49|-1482.94| +2451176|46428|16831|31435|286564|6046|19928|1|227|413|81|8.02|8.98|3.32|0.00|268.92|649.62|727.38|5.37|0.00|268.92|274.29|-380.70| +2451176|46428|8636|31435|286564|6046|19928|1|185|413|38|61.39|118.48|99.52|2836.32|3781.76|2332.82|4502.24|9.45|2836.32|945.44|954.89|-1387.38| +2451176||1981||286564|||1||413||30.64|||0.00||1378.80|1433.70||0.00|1089.45||| +2451176|46428|14569|31435|286564|6046|19928|1|91|413|30|15.94|24.22|15.74|0.00|472.20|478.20|726.60|33.05|0.00|472.20|505.25|-6.00| +2451176|46428|4970|31435|286564|6046|19928|1|129|413|19|30.51|32.95|4.61|46.42|87.59|579.69|626.05|2.05|46.42|41.17|43.22|-538.52| +2451176|46428|2512|31435|286564|6046|19928|1|102|413|2|90.50|109.50|73.36|0.00|146.72|181.00|219.00|8.80|0.00|146.72|155.52|-34.28| +2451084|40310|2059|46839|1007476|6960|29997|4|277|414|66|76.03|143.69|94.83|0.00|6258.78|5017.98|9483.54|62.58|0.00|6258.78|6321.36|1240.80| +2451084|40310|13676|46839|1007476|||||414||81.29||79.64|0.00|6610.12||9580.69|66.10|0.00||6676.22|| +2451084|40310|17590|46839|1007476|6960|29997|4|160|414|86|7.02|7.37|3.53|0.00|303.58|603.72|633.82|24.28|0.00|303.58|327.86|-300.14| +2451084|40310|14288|46839|1007476|6960|29997|4|107|414|35|2.89|4.01|1.96|0.00|68.60|101.15|140.35|2.74|0.00|68.60|71.34|-32.55| +2451084|40310|7756|46839|1007476|6960|29997|4|288|414|80|13.82|27.36|12.58|0.00|1006.40|1105.60|2188.80|90.57|0.00|1006.40|1096.97|-99.20| +2451084|40310|8240|46839|1007476|6960|29997|4|227|414|78|91.03|167.49|118.91|0.00|9274.98|7100.34|13064.22|741.99|0.00|9274.98|10016.97|2174.64| +2451084|40310|10270|46839|1007476|6960|29997|4|242|414|84|44.75|58.17|30.83|0.00|2589.72|3759.00|4886.28|155.38|0.00|2589.72|2745.10|-1169.28| +2451084|40310|11264|46839|1007476|6960|29997|4|286|414|85|6.27|10.78|3.01|0.00|255.85|532.95|916.30|15.35|0.00|255.85|271.20|-277.10| +2451084|40310|4090|46839|1007476|6960|29997|4|74|414|39|55.87|59.78|57.98|0.00|2261.22|2178.93|2331.42|67.83|0.00|2261.22|2329.05|82.29| +2451084|40310|7387|46839|1007476|6960|29997|4|119|414|88|24.27|40.28|8.05|0.00|708.40|2135.76|3544.64|7.08|0.00|708.40|715.48|-1427.36| +2451638|62185|8179|61949|252660|6571|19729|2|49|415|42|64.73|85.44|22.21|0.00|932.82|2718.66|3588.48|46.64|0.00|932.82|979.46|-1785.84| +2451638|62185|4724|61949|252660|6571|19729|2|275|415|16|47.29|56.27|41.07|0.00|657.12|756.64|900.32|32.85|0.00|657.12|689.97|-99.52| +2451638|62185|14783|61949|252660|6571|19729|2|111|415|47|73.65|86.17|58.59|0.00|2753.73|3461.55|4049.99|110.14|0.00|2753.73|2863.87|-707.82| +2451638|62185|15767|61949|252660|6571|19729|2|60|415|23|23.10|23.10|2.77|41.41|63.71|531.30|531.30|0.00|41.41|22.30|22.30|-509.00| +2451638|62185|10337|61949|252660|6571|19729|2|298|415|76|93.47|112.16|6.72|0.00|510.72|7103.72|8524.16|20.42|0.00|510.72|531.14|-6593.00| +2451638|62185|17557|61949|252660|6571|19729|2|65|415|99|58.75|77.55|11.63|0.00|1151.37|5816.25|7677.45|69.08|0.00|1151.37|1220.45|-4664.88| +2451638|62185|14719|61949|252660|6571|19729|2|15|415|88|72.75|144.77|101.33|0.00|8917.04|6402.00|12739.76|356.68|0.00|8917.04|9273.72|2515.04| +2451638|62185|8495|61949|252660|6571|19729|2|204|415|100|38.10|57.53|32.79|0.00|3279.00|3810.00|5753.00|295.11|0.00|3279.00|3574.11|-531.00| +2451638|62185|2636|61949|252660|6571|19729|2|231|415|92|36.81|48.58|6.31|29.02|580.52|3386.52|4469.36|11.03|29.02|551.50|562.53|-2835.02| +2451638|62185|11159|61949|252660|6571|19729|2|118|415|48|30.68|35.28|16.58|0.00|795.84|1472.64|1693.44|63.66|0.00|795.84|859.50|-676.80| +2451638|62185|17909|61949|252660|||2|295|415|||95.45|83.04||8054.88|||241.64||8054.88|8296.52|| +2451638|62185|15413|61949|252660|6571|19729|2|186|415|66|90.87|126.30|107.35|0.00|7085.10|5997.42|8335.80|283.40|0.00|7085.10|7368.50|1087.68| +2451638|62185|5413|61949|252660|6571|19729|2|151|415|60|26.46|27.51|12.37|0.00|742.20|1587.60|1650.60|37.11|0.00|742.20|779.31|-845.40| +|67462|7065|93947|511502|1410|18918|||416||92.71||||3655.23|3430.27||292.41||3655.23|3947.64|| +2451931|67462|2229|93947|511502|1410|18918|8|26|416|87|12.98|20.24|11.94|0.00|1038.78|1129.26|1760.88|62.32|0.00|1038.78|1101.10|-90.48| +2451931|67462|17885|93947|511502|1410|18918|8|114|416|10|96.11|125.90|52.87|0.00|528.70|961.10|1259.00|0.00|0.00|528.70|528.70|-432.40| +2451931|67462|7863|93947|511502|1410|18918|8|48|416|63|96.96|155.13|141.16|0.00|8893.08|6108.48|9773.19|800.37|0.00|8893.08|9693.45|2784.60| +2451931|67462|14999|93947|511502|1410|18918|8|256|416|94|37.88|67.04|28.82|0.00|2709.08|3560.72|6301.76|27.09|0.00|2709.08|2736.17|-851.64| +|67462|13139||511502|1410|||87|416|90||45.52|28.22||2539.80|3443.40|||||2641.39|-903.60| +2451931||17041|93947|511502||||16|416|8|72.91||71.01|0.00|||||0.00|568.08||-15.20| +2451931|67462|2573|93947|511502|1410|18918|8|256|416|96|10.59|18.74|6.37|0.00|611.52|1016.64|1799.04|55.03|0.00|611.52|666.55|-405.12| +2451510|54850|3854|72010|931421|79|19937|10|155|417|90|55.03|97.40|0.00|0.00|0.00|4952.70|8766.00|0.00|0.00|0.00|0.00|-4952.70| +2451510|54850|46|72010|931421|79|19937|10|116|417|77|37.24|40.59|10.95|0.00|843.15|2867.48|3125.43|16.86|0.00|843.15|860.01|-2024.33| +2451510|54850|6709|72010|931421|79|19937|10|227|417|100|25.67|49.02|46.07|0.00|4607.00|2567.00|4902.00|0.00|0.00|4607.00|4607.00|2040.00| +2451510|54850|9008|72010|931421|79|19937|10|5|417|78|17.07|32.09|16.68|0.00|1301.04|1331.46|2503.02|13.01|0.00|1301.04|1314.05|-30.42| +2451510|54850|424|72010|931421|79|19937|10|167|417|13|68.82|92.21|71.00|913.77|923.00|894.66|1198.73|0.18|913.77|9.23|9.41|-885.43| +2451510|54850|13418|72010|931421|79|19937|10|233|417|44|30.55|39.10|18.37|0.00|808.28|1344.20|1720.40|48.49|0.00|808.28|856.77|-535.92| +2451510|54850|15952|72010|931421|79|19937|10|272|417|52|9.93|19.76|5.73|0.00|297.96|516.36|1027.52|17.87|0.00|297.96|315.83|-218.40| +2451510|54850|16742|72010|931421|79|19937|10|125|417|19|85.83|156.21|31.24|0.00|593.56|1630.77|2967.99|29.67|0.00|593.56|623.23|-1037.21| +2451510|54850|980|72010|931421|79|19937|10|206|417|68|57.65|68.60|26.75|0.00|1819.00|3920.20|4664.80|18.19|0.00|1819.00|1837.19|-2101.20| +2451510|54850|3244|72010|931421|79|19937|10|110|417|92|73.25|84.97|27.19|50.02|2501.48|6739.00|7817.24|49.02|50.02|2451.46|2500.48|-4287.54| +2451510||1606||931421|||||417|87|94.65||||4219.50||10292.97|||||-4015.05| +2451510|54850|8824|72010|931421|79|19937|10|186|417|84|43.37|85.87|32.63|0.00|2740.92|3643.08|7213.08|137.04|0.00|2740.92|2877.96|-902.16| +2451510||13436|||||||417||50.04|62.55||0.00|||688.05||0.00|144.43||-406.01| +2451510|54850|9416|72010|931421|79|19937|10|283|417|40|16.15|20.51|0.61|0.00|24.40|646.00|820.40|1.70|0.00|24.40|26.10|-621.60| +2451510|54850|1982|72010|931421|79|19937|10|92|417|6|30.46|52.08|47.91|0.00|287.46|182.76|312.48|11.49|0.00|287.46|298.95|104.70| +2451510|54850|6796|72010|931421|79|19937|10|159|417|69|88.90|118.23|105.22|0.00|7260.18|6134.10|8157.87|435.61|0.00|7260.18|7695.79|1126.08| +2452243|52650|1743|44949|180753|1710|37914|4|225|418|41|74.18|126.10|46.65|0.00|1912.65|3041.38|5170.10|133.88|0.00|1912.65|2046.53|-1128.73| +2452243|52650|1113|44949|180753|1710|37914|4|162|418|95|90.81|165.27|72.71|0.00|6907.45|8626.95|15700.65|414.44|0.00|6907.45|7321.89|-1719.50| +2452243|52650|9675|44949|180753|1710|37914|4|277|418|43|72.15|125.54|123.02|0.00|5289.86|3102.45|5398.22|0.00|0.00|5289.86|5289.86|2187.41| +2452243|52650|9657|44949|180753|1710|37914|4|168|418|36|51.04|91.87|68.90|0.00|2480.40|1837.44|3307.32|99.21|0.00|2480.40|2579.61|642.96| +2452243|52650|14517|44949|180753|1710|37914|4|204|418|7|31.61|40.14|18.06|92.28|126.42|221.27|280.98|0.68|92.28|34.14|34.82|-187.13| +2452243|52650|9263|44949|180753|1710|37914|4|162|418|47|62.64|99.59|49.79|0.00|2340.13|2944.08|4680.73|70.20|0.00|2340.13|2410.33|-603.95| +2452243|52650|6245|44949|180753|1710|37914|4|240|418|97|79.99|80.78|3.23|0.00|313.31|7759.03|7835.66|0.00|0.00|313.31|313.31|-7445.72| +2452243|52650|11393|44949|180753|1710|37914|4|180|418|11|37.44|67.01|60.97|0.00|670.67|411.84|737.11|26.82|0.00|670.67|697.49|258.83| +2452243|52650|3601|44949|180753|1710|37914|4|174|418|74|75.02|77.27|27.04|1140.54|2000.96|5551.48|5717.98|17.20|1140.54|860.42|877.62|-4691.06| +2452243|52650|2275|44949|180753|1710|37914|4|41|418|39|20.23|20.83|20.20|0.00|787.80|788.97|812.37|31.51|0.00|787.80|819.31|-1.17| +2452234|52389|7735|73535|1198756|5930|46304|2|83|419|9|69.68|117.75|111.86|0.00|1006.74|627.12|1059.75|0.00|0.00|1006.74|1006.74|379.62| +2452234|52389|5549|73535|1198756|5930|46304|2|151|419|5|57.35|61.36|43.56|0.00|217.80|286.75|306.80|13.06|0.00|217.80|230.86|-68.95| +2452234|52389|15519|73535|1198756|5930|46304|2|186|419|60|43.75|78.31|3.91|0.00|234.60|2625.00|4698.60|18.76|0.00|234.60|253.36|-2390.40| +2452234|52389|13767|73535|1198756|5930|46304|2|218|419|29|93.78|158.48|103.01|0.00|2987.29|2719.62|4595.92|209.11|0.00|2987.29|3196.40|267.67| +2452234|52389|7669|73535|1198756|5930|46304|2|136|419|25|76.42|99.34|64.57|0.00|1614.25|1910.50|2483.50|32.28|0.00|1614.25|1646.53|-296.25| +2452234|52389|5017|73535|1198756|5930|46304|2|277|419|94|45.68|83.59|49.31|0.00|4635.14|4293.92|7857.46|417.16|0.00|4635.14|5052.30|341.22| +2452234|52389|4521|73535|1198756|5930|46304|2|176|419|23|36.28|60.22|40.94|0.00|941.62|834.44|1385.06|65.91|0.00|941.62|1007.53|107.18| +2452234|52389|5751|73535|1198756|5930|46304|2|218|419|87|19.26|24.07|19.25|0.00|1674.75|1675.62|2094.09|16.74|0.00|1674.75|1691.49|-0.87| +2452234|52389|10781|73535|1198756|5930|46304|2|228|419|43|64.66|106.04|89.07|727.70|3830.01|2780.38|4559.72|186.13|727.70|3102.31|3288.44|321.93| +2452234|52389|11643|73535|1198756|5930|46304|2|243|419|47|24.05|34.39|10.66|0.00|501.02|1130.35|1616.33|5.01|0.00|501.02|506.03|-629.33| +2452234|52389|16519|73535|1198756|5930|46304|2|10|419|91|51.01|56.62|34.53|0.00|3142.23|4641.91|5152.42|125.68|0.00|3142.23|3267.91|-1499.68| +2452234|52389|6863|73535|1198756|5930|46304|2|229|419|25|7.50|14.32|4.58|0.00|114.50|187.50|358.00|3.43|0.00|114.50|117.93|-73.00| +2452234|52389|17269|73535|1198756|5930|46304|2|70|419|81|1.00|1.16|0.75|0.00|60.75|81.00|93.96|3.64|0.00|60.75|64.39|-20.25| +2452234|52389|15109|73535|1198756|5930|46304|2|16|419|31|41.39|75.74|35.59|0.00|1103.29|1283.09|2347.94|33.09|0.00|1103.29|1136.38|-179.80| +2452234|52389|247|73535|1198756|5930|46304|2|75|419|29|80.30|84.31|67.44|0.00|1955.76|2328.70|2444.99|19.55|0.00|1955.76|1975.31|-372.94| +2451935|34662|1013|41075|1563441|4560|27326|4|116|420|35|46.36|64.44|14.82|0.00|518.70|1622.60|2255.40|5.18|0.00|518.70|523.88|-1103.90| +2451935|34662|13809|41075|||27326|4|204|420||36.00|||0.00|655.60|||32.78|0.00|655.60||-1324.40| +||2811|41075||||4|52|420|||||0.00||2938.32|3702.16||0.00||38.06|-2901.36| +2451935|34662|943|41075|1563441|4560|27326|4|120|420|17|85.88|170.04|71.41|582.70|1213.97|1459.96|2890.68|44.18|582.70|631.27|675.45|-828.69| +2451935|34662|6019|41075|1563441|4560|27326|4|119|420|55|64.20|64.20|14.76|0.00|811.80|3531.00|3531.00|32.47|0.00|811.80|844.27|-2719.20| +2451935|34662|14241|41075|1563441|4560|27326|4|56|420|75|84.71|142.31|83.96|0.00|6297.00|6353.25|10673.25|377.82|0.00|6297.00|6674.82|-56.25| +2451935|34662|717|41075|1563441|4560|27326|4|275|420|80|5.48|10.30|0.41|0.00|32.80|438.40|824.00|2.29|0.00|32.80|35.09|-405.60| +2451935|34662|11605|41075|1563441|4560|27326|4|247|420|92|76.99|85.45|29.05|0.00|2672.60|7083.08|7861.40|26.72|0.00|2672.60|2699.32|-4410.48| +2451935|34662|85|41075|1563441|4560|27326|4|186|420|89|15.59|25.56|6.39|0.00|568.71|1387.51|2274.84|11.37|0.00|568.71|580.08|-818.80| +2451935|34662|7767|41075|1563441|4560|27326|4|180|420|5|19.82|36.07|20.55|0.00|102.75|99.10|180.35|4.11|0.00|102.75|106.86|3.65| +2451935|34662|9605|41075|1563441|4560|27326|4|185|420|92|19.18|35.48|29.44|0.00|2708.48|1764.56|3264.16|216.67|0.00|2708.48|2925.15|943.92| +2451935|34662|4003|41075|1563441|4560|27326|4|33|420|15|82.78|165.56|135.75|1771.53|2036.25|1241.70|2483.40|21.17|1771.53|264.72|285.89|-976.98| +2451935|34662|14483|41075|1563441|4560|27326|4|250|420|71|47.49|91.65|47.65|0.00|3383.15|3371.79|6507.15|169.15|0.00|3383.15|3552.30|11.36| +2451935|34662|17471|41075|1563441|4560|27326|4|181|420|38|4.92|7.13|3.49|0.00|132.62|186.96|270.94|5.30|0.00|132.62|137.92|-54.34| +2451935|34662|5583|41075|1563441|4560|27326|4|41|420|3|86.36|91.54|33.86|56.88|101.58|259.08|274.62|0.89|56.88|44.70|45.59|-214.38| +2451935|34662|41|41075|1563441|4560|27326|4|184|420|66|11.35|19.97|17.77|0.00|1172.82|749.10|1318.02|23.45|0.00|1172.82|1196.27|423.72| +2452326|52876|17784|97932|74266|6247|46228|1|2|421|76|40.93|76.94|47.70|0.00|3625.20|3110.68|5847.44|0.00|0.00|3625.20|3625.20|514.52| +2452326|52876|7170|97932|74266|6247|46228|1|290|421|13|53.24|57.49|31.04|310.71|403.52|692.12|747.37|8.35|310.71|92.81|101.16|-599.31| +2452326|52876|12469|97932|74266|6247|46228|1|56|421|17|65.32|77.07|3.08|0.00|52.36|1110.44|1310.19|3.14|0.00|52.36|55.50|-1058.08| +2452326|52876|17940|97932||6247||||421||||68.66|0.00|5492.80||7524.80||0.00|||1653.60| +2452326|52876|2994|97932|74266|6247|46228|1|27|421|99|35.99|45.70|29.70|0.00|2940.30|3563.01|4524.30|117.61|0.00|2940.30|3057.91|-622.71| +2452326|52876|5940|97932|74266|6247|46228|1|97|421|95|54.37|70.13|27.35|0.00|2598.25|5165.15|6662.35|77.94|0.00|2598.25|2676.19|-2566.90| +2452326|52876|985|97932|74266|6247|46228|1|8|421|88|80.46|102.18|48.02|0.00|4225.76|7080.48|8991.84|84.51|0.00|4225.76|4310.27|-2854.72| +2452326|52876|5664|97932|74266|6247|46228|1|95|421|92|72.10|90.12|19.82|0.00|1823.44|6633.20|8291.04|91.17|0.00|1823.44|1914.61|-4809.76| +2452326|52876|13971|97932|74266|6247|46228|1|20|421|93|7.14|11.56|2.19|30.55|203.67|664.02|1075.08|1.73|30.55|173.12|174.85|-490.90| +2452326|52876|5820|97932|74266|6247|46228|1|74|421|97|30.67|32.81|29.85|781.77|2895.45|2974.99|3182.57|169.09|781.77|2113.68|2282.77|-861.31| +2452326|52876|9177|97932|74266|6247|46228|1|30|421|29|39.70|62.32|60.45|0.00|1753.05|1151.30|1807.28|70.12|0.00|1753.05|1823.17|601.75| +2452326|52876|15621|97932|74266|6247|46228|1|161|421|82|66.19|121.78|41.40|0.00|3394.80|5427.58|9985.96|237.63|0.00|3394.80|3632.43|-2032.78| +2452326|52876|15744|97932|74266|6247|46228|1|15|421|86|12.65|23.27|10.93|0.00|939.98|1087.90|2001.22|37.59|0.00|939.98|977.57|-147.92| +2452326|52876|5373|97932|74266|6247|46228|1|11|421|94|74.38|77.35|23.20|915.93|2180.80|6991.72|7270.90|12.64|915.93|1264.87|1277.51|-5726.85| +2452326|52876|639|97932|74266|6247|46228|1|202|421|42|7.69|15.22|10.34|0.00|434.28|322.98|639.24|17.37|0.00|434.28|451.65|111.30| +2452326|52876|16219|97932|74266|6247|46228|1|273|421|65|89.35|126.87|21.56|0.00|1401.40|5807.75|8246.55|126.12|0.00|1401.40|1527.52|-4406.35| +2451149|71217|1909|31925|1490843|6208|47173|8|75|422|32|72.17|142.17|34.12|0.00|1091.84|2309.44|4549.44|10.91|0.00|1091.84|1102.75|-1217.60| +2451149|71217|3097|31925|1490843|6208|47173|8|48|422|4|36.21|60.47|47.77|0.00|191.08|144.84|241.88|0.00|0.00|191.08|191.08|46.24| +2451149|71217|2197|31925|1490843|6208|47173|8|90|422|65|98.34|186.84|113.97|0.00|7408.05|6392.10|12144.60|666.72|0.00|7408.05|8074.77|1015.95| +|71217|5990|31925|1490843|6208|47173|||422|||||1344.38|2068.28|4892.88|7388.04||1344.38||738.37|| +2451149|71217|10820|31925|1490843|6208|47173|8|10|422|18|60.19|104.12|76.00|0.00|1368.00|1083.42|1874.16|123.12|0.00|1368.00|1491.12|284.58| +2451149|71217|16243|31925|1490843|6208|47173|8|57|422|64|13.91|24.34|23.60|0.00|1510.40|890.24|1557.76|15.10|0.00|1510.40|1525.50|620.16| +2451149|71217|2914|31925|1490843|6208|47173|8|259|422|28|2.45|2.49|1.74|0.00|48.72|68.60|69.72|1.94|0.00|48.72|50.66|-19.88| +2451149|71217|17575|31925|1490843|6208|47173|8|188|422|75|62.16|85.78|48.03|0.00|3602.25|4662.00|6433.50|180.11|0.00|3602.25|3782.36|-1059.75| +2451747|43734|5725|86637|10941|3598|48302|1|292|423|65|96.27|130.92|24.87|0.00|1616.55|6257.55|8509.80|16.16|0.00|1616.55|1632.71|-4641.00| +2451747|43734|11282|86637|10941|3598|48302|1|223|423|65|59.91|95.25|12.38|8.04|804.70|3894.15|6191.25|7.96|8.04|796.66|804.62|-3097.49| +2451747|43734|5870|86637|10941|3598|48302|1|208|423|28|9.19|11.02|1.65|0.00|46.20|257.32|308.56|3.69|0.00|46.20|49.89|-211.12| +2451747|43734|16688|86637|10941|3598|48302|1|136|423|42|58.49|89.48|41.16|795.21|1728.72|2456.58|3758.16|9.33|795.21|933.51|942.84|-1523.07| +2451747|43734|3841|86637|10941|3598|48302|1|296|423|18|7.88|13.55|8.67|0.00|156.06|141.84|243.90|9.36|0.00|156.06|165.42|14.22| +2451747|43734|14221|86637|10941|3598|48302|1|133|423|64|95.50|121.28|84.89|0.00|5432.96|6112.00|7761.92|217.31|0.00|5432.96|5650.27|-679.04| +2451747|43734|10178|86637|10941|3598|48302|1|171|423|83|98.76|113.57|23.84|0.00|1978.72|8197.08|9426.31|98.93|0.00|1978.72|2077.65|-6218.36| +2451747|43734|17719|86637|10941|3598|48302|1|83|423|37|17.76|24.33|0.24|0.00|8.88|657.12|900.21|0.35|0.00|8.88|9.23|-648.24| +2451747|43734|3881|86637|10941|3598|48302|1|162|423|49|19.96|22.15|3.54|0.00|173.46|978.04|1085.35|0.00|0.00|173.46|173.46|-804.58| +2451747|43734|14267|86637|10941|3598|48302|1|272|423|20|20.15|37.07|5.56|0.00|111.20|403.00|741.40|0.00|0.00|111.20|111.20|-291.80| +2451747|43734|1475|86637|10941|3598|48302|1|179|423|32|64.48|103.16|7.22|0.00|231.04|2063.36|3301.12|20.79|0.00|231.04|251.83|-1832.32| +2452609|31732|7557|29266|1704100|3287|27321|8|271|424|95|57.69|95.76|95.76|0.00|9097.20|5480.55|9097.20|454.86|0.00|9097.20|9552.06|3616.65| +2452609|31732|12823|29266|1704100|3287|27321|8|199|424|17|45.57|61.06|7.32|0.00|124.44|774.69|1038.02|3.73|0.00|124.44|128.17|-650.25| +2452609|31732|7701|29266|1704100|3287|27321|8|155|424|63|67.41|126.05|52.94|0.00|3335.22|4246.83|7941.15|0.00|0.00|3335.22|3335.22|-911.61| +2452609|31732|444|29266|1704100|3287|27321|8|7|424|95|30.18|59.75|45.41|0.00|4313.95|2867.10|5676.25|0.00|0.00|4313.95|4313.95|1446.85| +2452609|31732|2805|29266|1704100|3287|27321|8|165|424|19|43.98|86.64|31.19|0.00|592.61|835.62|1646.16|5.92|0.00|592.61|598.53|-243.01| +2452609|31732|7371|29266|1704100|3287|27321|8|65|424|89|84.20|127.98|79.34|0.00|7061.26|7493.80|11390.22|70.61|0.00|7061.26|7131.87|-432.54| +2452609|31732|14355|29266|1704100|3287|27321|8|110|424|72|1.87|2.97|0.80|0.00|57.60|134.64|213.84|0.00|0.00|57.60|57.60|-77.04| +2452609|31732|3027|29266|1704100|3287|27321|8|206|424|40|52.87|79.83|20.75|33.20|830.00|2114.80|3193.20|31.87|33.20|796.80|828.67|-1318.00| +2452609|31732|5407|29266|1704100|3287|27321|8|158|424|13|88.63|169.28|74.48|0.00|968.24|1152.19|2200.64|48.41|0.00|968.24|1016.65|-183.95| +2452609|31732|1278|29266|1704100|3287|27321|8|89|424|43|47.37|68.21|40.92|844.58|1759.56|2036.91|2933.03|54.89|844.58|914.98|969.87|-1121.93| +2452609|31732|7404|29266|1704100|3287|27321|8|131|424|21|55.28|56.38|16.91|0.00|355.11|1160.88|1183.98|28.40|0.00|355.11|383.51|-805.77| +2452609|31732|3381|29266|1704100|3287|27321|8|83|424|28|28.44|55.45|13.30|0.00|372.40|796.32|1552.60|22.34|0.00|372.40|394.74|-423.92| +2452609|31732|15243|29266|1704100|3287|27321|8|109|424|38|80.97|92.30|6.46|0.00|245.48|3076.86|3507.40|0.00|0.00|245.48|245.48|-2831.38| +2452609|31732|12|29266|1704100|3287|27321|8|57|424|2|21.20|36.88|35.03|0.00|70.06|42.40|73.76|0.00|0.00|70.06|70.06|27.66| +2451128|47959|4346|57229|497633|790|48794|7|82|425|5|25.57|34.77|4.86|0.00|24.30|127.85|173.85|1.45|0.00|24.30|25.75|-103.55| +||17872|57229|497633|790|48794|7||425||38.08||57.98|1982.91|3304.86|2170.56|||1982.91|1321.95||-848.61| +2451128|47959|2401|57229|497633|790|48794|7|215|425|100|73.71|123.09|40.61|0.00|4061.00|7371.00|12309.00|162.44|0.00|4061.00|4223.44|-3310.00| +2451128|47959|3866|57229|497633|790|48794|7|177|425|6|13.37|20.99|14.48|0.00|86.88|80.22|125.94|7.81|0.00|86.88|94.69|6.66| +2451128|47959|14294|57229|497633|790|48794|7|287|425|45|96.63|165.23|132.18|0.00|5948.10|4348.35|7435.35|535.32|0.00|5948.10|6483.42|1599.75| +2451128|47959|5200|57229|497633|790|48794|7|137|425|40|33.49|44.20|24.75|0.00|990.00|1339.60|1768.00|29.70|0.00|990.00|1019.70|-349.60| +2451128|47959|7894|57229|497633|790|48794|7|229|425|20|12.38|17.57|6.67|0.00|133.40|247.60|351.40|5.33|0.00|133.40|138.73|-114.20| +2451128|47959|9872|57229|497633|790|48794|7|90|425|13|51.97|70.67|61.48|0.00|799.24|675.61|918.71|0.00|0.00|799.24|799.24|123.63| +2451128|47959|8299|57229|497633|790|48794|7|273|425|3|72.30|100.49|48.23|0.00|144.69|216.90|301.47|0.00|0.00|144.69|144.69|-72.21| +2451128|47959|17162|57229|497633|790|48794|7|137|425|60|89.99|119.68|39.49|0.00|2369.40|5399.40|7180.80|118.47|0.00|2369.40|2487.87|-3030.00| +2451128|47959|6217|57229|497633|790|48794|7|234|425|38|91.07|129.31|81.46|0.00|3095.48|3460.66|4913.78|30.95|0.00|3095.48|3126.43|-365.18| +2451128|47959|7462|57229|497633|790|48794|7|234|425|45|40.26|61.19|54.45|0.00|2450.25|1811.70|2753.55|98.01|0.00|2450.25|2548.26|638.55| +2451128|47959|13874|57229|497633|790|48794|7|238|425|93|49.12|91.36|55.72|0.00|5181.96|4568.16|8496.48|414.55|0.00|5181.96|5596.51|613.80| +2450953|70399|8618|24858|933380|2702|14377|8|298|426|79|14.74|24.02|1.44|58.01|113.76|1164.46|1897.58|2.78|58.01|55.75|58.53|-1108.71| +2450953|70399|8476|24858|933380|2702|14377|8|142|426|52|97.61|170.81|70.03|0.00|3641.56|5075.72|8882.12|327.74|0.00|3641.56|3969.30|-1434.16| +2450953|70399|2425|24858|933380|2702|14377|8|146|426|84|20.10|31.55|10.72|0.00|900.48|1688.40|2650.20|81.04|0.00|900.48|981.52|-787.92| +2450953|70399|8564|24858|933380|2702|14377|8|275|426|76|36.60|50.50|31.81|0.00|2417.56|2781.60|3838.00|217.58|0.00|2417.56|2635.14|-364.04| +2450953|70399|1444|24858|933380|2702|14377|8|34|426|77|13.94|19.93|15.54|969.22|1196.58|1073.38|1534.61|18.18|969.22|227.36|245.54|-846.02| +2450953|70399|3565|24858|933380|2702|14377|8|184|426|97|1.14|1.39|0.50|0.00|48.50|110.58|134.83|3.88|0.00|48.50|52.38|-62.08| +2450953|70399|14560|24858|933380|2702|14377|8|298|426|42|85.43|140.95|129.67|54.46|5446.14|3588.06|5919.90|107.83|54.46|5391.68|5499.51|1803.62| +2450953|70399|6661|24858|933380|2702|14377|8|164|426|38|47.00|72.38|13.02|0.00|494.76|1786.00|2750.44|19.79|0.00|494.76|514.55|-1291.24| +2450953|70399|17893|24858|933380|2702|14377|8|162|426|99|30.90|53.45|29.39|2735.03|2909.61|3059.10|5291.55|3.49|2735.03|174.58|178.07|-2884.52| +|70399|847|24858|||||250|426|||||0.00|1495.48|4550.14|||0.00||1585.20|-3054.66| +2450953|70399|16178|24858|933380|2702|14377|8|224|426|1|55.70|62.94|50.35|0.00|50.35|55.70|62.94|3.52|0.00|50.35|53.87|-5.35| +2450953|70399|872|24858|933380|2702|14377|8|100|426|16|98.47|154.59|72.65|0.00|1162.40|1575.52|2473.44|23.24|0.00|1162.40|1185.64|-413.12| +2451121|51404|1189|57891|39906|6721|47489|2|146|427|47|30.20|40.46|16.18|0.00|760.46|1419.40|1901.62|30.41|0.00|760.46|790.87|-658.94| +2451121|51404|12742|57891|39906|6721|47489|2|211|427|82|6.84|9.91|3.66|0.00|300.12|560.88|812.62|3.00|0.00|300.12|303.12|-260.76| +2451121|51404|1520|57891|39906|6721|47489|2|193|427|62|4.93|7.34|6.38|284.80|395.56|305.66|455.08|7.75|284.80|110.76|118.51|-194.90| +2451121|51404|12374|57891|39906|6721|47489|2|126|427|23|49.63|60.05|4.80|0.00|110.40|1141.49|1381.15|8.83|0.00|110.40|119.23|-1031.09| +2451121|51404|5234|57891|39906|6721|47489|2|105|427|40|47.29|66.67|30.00|0.00|1200.00|1891.60|2666.80|24.00|0.00|1200.00|1224.00|-691.60| +2451121|51404|15362|57891|39906|6721|47489|2|227|427|77|32.69|47.07|22.59|0.00|1739.43|2517.13|3624.39|0.00|0.00|1739.43|1739.43|-777.70| +2451121|51404|16816|57891|39906|6721|47489|2|136|427|41|89.63|176.57|97.11|0.00|3981.51|3674.83|7239.37|238.89|0.00|3981.51|4220.40|306.68| +2451121|51404|6038|57891|39906|6721|47489|2|10|427|58|23.62|39.68|18.64|0.00|1081.12|1369.96|2301.44|10.81|0.00|1081.12|1091.93|-288.84| +2451121|51404|15745|57891|39906|6721|47489|2|37|427|46|85.50|156.46|95.44|0.00|4390.24|3933.00|7197.16|351.21|0.00|4390.24|4741.45|457.24| +2451121|51404|1754|57891|39906|6721|47489|2|14|427|47|93.33|117.59|83.48|0.00|3923.56|4386.51|5526.73|313.88|0.00|3923.56|4237.44|-462.95| +2451121|51404|2554|57891|39906|6721|47489|2|272|427|87|52.15|75.09|61.57|0.00|5356.59|4537.05|6532.83|214.26|0.00|5356.59|5570.85|819.54| +2451121|51404|13066|57891|39906|6721|47489|2|14|427|31|92.95|114.32|11.43|0.00|354.33|2881.45|3543.92|21.25|0.00|354.33|375.58|-2527.12| +2451121|51404|7186|57891||6721|47489|||427||46.08|76.03|39.53|||||||||-517.45| +2452474|50283|17229|64120|270481|3242|42003|7|173|428|23|49.61|92.77|13.91|0.00|319.93|1141.03|2133.71|6.39|0.00|319.93|326.32|-821.10| +2452474|50283|9511|||3242|||41|428|20|73.43||37.83|0.00|||1644.80|7.56|0.00||764.16|-712.00| +2452474|50283|16579|64120|270481|3242|42003|7|212|428|36|48.32|86.97|66.09|0.00|2379.24|1739.52|3130.92|190.33|0.00|2379.24|2569.57|639.72| +2452474|50283|6715|64120|270481|3242|42003|7|241|428|35|83.42|150.15|22.52|0.00|788.20|2919.70|5255.25|31.52|0.00|788.20|819.72|-2131.50| +|50283|6175|64120|270481|||||428||||||67.05|85.56|119.76|0.52||17.44||| +2452474|50283|13788|64120|270481|3242|42003|7|153|428|18|72.49|95.68|22.96|0.00|413.28|1304.82|1722.24|37.19|0.00|413.28|450.47|-891.54| +2452474|50283|7947|64120|270481|3242|42003|7|163|428|46|43.05|72.75|36.37|1204.57|1673.02|1980.30|3346.50|37.47|1204.57|468.45|505.92|-1511.85| +2452474|50283|3438|64120|270481|3242|42003|7|117|428|69|83.71|113.00|38.42|0.00|2650.98|5775.99|7797.00|0.00|0.00|2650.98|2650.98|-3125.01| +2452474|50283|16077|64120|270481|3242|42003|7|39|428|75|41.57|47.38|1.89|0.00|141.75|3117.75|3553.50|1.41|0.00|141.75|143.16|-2976.00| +2452474|50283|5833|64120|270481|3242|42003|7|258|428|56|58.28|71.10|63.27|0.00|3543.12|3263.68|3981.60|35.43|0.00|3543.12|3578.55|279.44| +2451464|52891|5144|80740|1139222|2504|35339|8|163|429|17|16.43|23.16|6.25|0.00|106.25|279.31|393.72|0.00|0.00|106.25|106.25|-173.06| +2451464|52891|5872|80740|1139222|2504|35339|8|284|429|98|87.11|166.38|158.06|0.00|15489.88|8536.78|16305.24|1084.29|0.00|15489.88|16574.17|6953.10| +2451464|52891|3955|80740|1139222||||212|429||3.18||4.95||163.35||187.77||||158.15|| +2451464|52891|14822|80740|1139222|2504|35339|8|244|429|84|52.32|102.54|102.54|0.00|8613.36|4394.88|8613.36|516.80|0.00|8613.36|9130.16|4218.48| +2451464|52891|7171|80740|1139222|2504|35339|8|23|429|55|33.37|64.07|41.64|0.00|2290.20|1835.35|3523.85|45.80|0.00|2290.20|2336.00|454.85| +2451464|52891|17104|80740|1139222|2504|35339|8|243|429|9|46.74|78.52|18.84|0.00|169.56|420.66|706.68|1.69|0.00|169.56|171.25|-251.10| +2451464|52891|12128|80740|1139222|2504|35339|8|274|429|21|45.46|48.64|28.21|0.00|592.41|954.66|1021.44|47.39|0.00|592.41|639.80|-362.25| +2451464|52891|2036|80740|1139222|2504|35339|8|269|429|7|60.51|66.56|0.00|0.00|0.00|423.57|465.92|0.00|0.00|0.00|0.00|-423.57| +2451464|52891|2524|80740|1139222|2504|35339|8|292|429|80|7.41|13.63|4.49|0.00|359.20|592.80|1090.40|17.96|0.00|359.20|377.16|-233.60| +2452521|40744|11475|10205|556588|4218|16201|10|46|430|43|52.26|55.39|49.29|0.00|2119.47|2247.18|2381.77|0.00|0.00|2119.47|2119.47|-127.71| +2452521|40744|15177|10205|556588|4218|16201|10|91|430|28|46.47|85.96|14.61|0.00|409.08|1301.16|2406.88|0.00|0.00|409.08|409.08|-892.08| +2452521|40744|13897|10205|556588|4218|16201|10|27|430|31|53.81|107.08|86.73|0.00|2688.63|1668.11|3319.48|188.20|0.00|2688.63|2876.83|1020.52| +2452521|40744|14905|10205|556588|4218|16201|10|58|430|97|55.94|67.68|38.57|3591.63|3741.29|5426.18|6564.96|0.00|3591.63|149.66|149.66|-5276.52| +2452521|40744|4227|10205|556588|4218|16201|10|298|430|70|77.60|85.36|18.77|0.00|1313.90|5432.00|5975.20|13.13|0.00|1313.90|1327.03|-4118.10| +2452521|40744|9625|10205|556588|4218|16201|10|125|430|80|63.39|86.84|40.81|0.00|3264.80|5071.20|6947.20|32.64|0.00|3264.80|3297.44|-1806.40| +|40744|17025|||4218|16201||101|430|45|41.36|51.28|2.05|||||||||-1768.95| +2452521|40744|2287|10205|556588|4218|16201|10|90|430|30|60.55|94.45|15.11|0.00|453.30|1816.50|2833.50|13.59|0.00|453.30|466.89|-1363.20| +2451518|65897|13447|82545|34534|476|18865|10|112|431|5|9.86|18.93|4.16|0.00|20.80|49.30|94.65|0.62|0.00|20.80|21.42|-28.50| +2451518|65897|3136|82545|34534|476|18865|10|102|431|43|81.37|85.43|41.00|0.00|1763.00|3498.91|3673.49|0.00|0.00|1763.00|1763.00|-1735.91| +2451518|65897|3289|82545|34534|476|18865|10|15|431|16|35.71|52.85|22.19|0.00|355.04|571.36|845.60|17.75|0.00|355.04|372.79|-216.32| +2451518||9226|||||10||431||33.46|||||2844.10||27.60||2760.80||| +2451518|65897|9746|82545|34534|476|18865|10|287|431|64|62.54|105.69|26.42|0.00|1690.88|4002.56|6764.16|0.00|0.00|1690.88|1690.88|-2311.68| +2451518|65897|7873|82545|34534|476|18865|10|101|431|87|90.04|108.04|88.59|0.00|7707.33|7833.48|9399.48|77.07|0.00|7707.33|7784.40|-126.15| +2451518|65897|8084|82545|34534|476|18865|10|121|431|29|32.45|36.99|29.96|0.00|868.84|941.05|1072.71|8.68|0.00|868.84|877.52|-72.21| +2451518|65897|226|82545|34534|476|18865|10|171|431|75|71.80|95.49|92.62|0.00|6946.50|5385.00|7161.75|208.39|0.00|6946.50|7154.89|1561.50| +2451518|65897|17014|82545|34534|476|18865|10|13|431|69|64.41|117.22|117.22|0.00|8088.18|4444.29|8088.18|323.52|0.00|8088.18|8411.70|3643.89| +2451518|65897|5086|82545|34534|476|18865|10|217|431|27|95.55|103.19|75.32|0.00|2033.64|2579.85|2786.13|81.34|0.00|2033.64|2114.98|-546.21| +2451518|65897|17338|82545|34534|476|18865|10|59|431|100|81.02|113.42|7.93|0.00|793.00|8102.00|11342.00|23.79|0.00|793.00|816.79|-7309.00| +2451518|65897|13654|82545|34534|476|18865|10|159|431|85|70.35|135.77|74.67|0.00|6346.95|5979.75|11540.45|444.28|0.00|6346.95|6791.23|367.20| +2451518|65897|15578|82545|34534|476|18865|10|205|431|28|77.80|112.81|84.60|0.00|2368.80|2178.40|3158.68|142.12|0.00|2368.80|2510.92|190.40| +2451899|35551|15701|77017|182290|4716|47189|8|117|432|86|82.45|152.53|22.87|0.00|1966.82|7090.70|13117.58|0.00|0.00|1966.82|1966.82|-5123.88| +2451899|35551|10796|77017|182290|4716|47189|8|297|432|7|96.10|190.27|156.02|688.04|1092.14|672.70|1331.89|36.36|688.04|404.10|440.46|-268.60| +2451899|35551|13205|77017|182290|4716|47189|8|84|432|90|34.94|44.72|29.06|0.00|2615.40|3144.60|4024.80|52.30|0.00|2615.40|2667.70|-529.20| +2451899|35551|15985|77017|182290|4716|47189|8|281|432|58|97.28|135.21|108.16|0.00|6273.28|5642.24|7842.18|564.59|0.00|6273.28|6837.87|631.04| +2451899|35551|14831|77017|182290|4716|47189|8|217|432|77|32.18|55.67|33.95|0.00|2614.15|2477.86|4286.59|104.56|0.00|2614.15|2718.71|136.29| +2451899|35551|8941|77017|182290|4716|47189|8|283|432|45|88.72|159.69|132.54|0.00|5964.30|3992.40|7186.05|536.78|0.00|5964.30|6501.08|1971.90| +2451899|35551|5065|77017|182290|4716|47189|8|224|432|37|69.87|114.58|111.14|0.00|4112.18|2585.19|4239.46|370.09|0.00|4112.18|4482.27|1526.99| +2451899|35551|5107|77017|182290|4716|47189|8|103|432|23|6.52|10.75|3.22|18.51|74.06|149.96|247.25|1.66|18.51|55.55|57.21|-94.41| +2452225|71003|10651|62360|1664464|7015|43966|1|229|433|67|24.49|31.83|14.00|0.00|938.00|1640.83|2132.61|37.52|0.00|938.00|975.52|-702.83| +2452225|71003|3133|62360|1664464|7015|43966|1|24|433|66|8.62|14.99|2.99|0.00|197.34|568.92|989.34|3.94|0.00|197.34|201.28|-371.58| +2452225|71003|7087|62360|1664464|7015|43966|1|196|433|89|50.65|70.91|67.36|0.00|5995.04|4507.85|6310.99|359.70|0.00|5995.04|6354.74|1487.19| +2452225|71003|9347|62360|1664464|7015|43966|1|262|433|7|32.56|33.53|9.72|0.00|68.04|227.92|234.71|2.04|0.00|68.04|70.08|-159.88| +2452225|71003|6773|62360|1664464|7015|43966|1|77|433|90|55.21|103.79|65.38|0.00|5884.20|4968.90|9341.10|58.84|0.00|5884.20|5943.04|915.30| +2452225|71003|9095|62360|1664464|7015|43966|1|191|433|81|58.32|103.80|46.71|2307.94|3783.51|4723.92|8407.80|14.75|2307.94|1475.57|1490.32|-3248.35| +2452225|71003|1095|62360|1664464|7015|43966|1|18|433|83|61.55|91.09|11.84|0.00|982.72|5108.65|7560.47|78.61|0.00|982.72|1061.33|-4125.93| +2452225|71003|3377|62360|1664464|7015|43966|1|272|433|97|35.47|70.58|51.52|3648.13|4997.44|3440.59|6846.26|121.43|3648.13|1349.31|1470.74|-2091.28| +2452225|71003|4883|62360|1664464|7015|43966|1|22|433|68|10.30|15.86|9.99|0.00|679.32|700.40|1078.48|61.13|0.00|679.32|740.45|-21.08| +2452225|71003|8663|62360|1664464|7015|43966|1|77|433|91|31.54|57.40|43.05|0.00|3917.55|2870.14|5223.40|352.57|0.00|3917.55|4270.12|1047.41| +2452225|71003|17539|62360|1664464|7015|43966|1|135|433|26|98.73|175.73|3.51|0.00|91.26|2566.98|4568.98|2.73|0.00|91.26|93.99|-2475.72| +2452640|43716|8647|59712|1076059|2006|27153|1|282|434|22|19.81|22.78|17.54|0.00|385.88|435.82|501.16|11.57|0.00|385.88|397.45|-49.94| +2452640|43716|16821|59712|1076059|2006|27153|1|292|434|49|47.76|52.05|7.28|0.00|356.72|2340.24|2550.45|17.83|0.00|356.72|374.55|-1983.52| +2452640|43716|1057|59712|1076059|2006|27153|1|258|434|21|88.83|132.35|34.41|0.00|722.61|1865.43|2779.35|36.13|0.00|722.61|758.74|-1142.82| +2452640|43716|15037|59712|1076059|2006|27153|1|19|434|37|86.79|145.80|37.90|0.00|1402.30|3211.23|5394.60|42.06|0.00|1402.30|1444.36|-1808.93| +2452640|43716|10653|59712|1076059|2006|27153|1|145|434|84|69.81|118.67|79.50|0.00|6678.00|5864.04|9968.28|267.12|0.00|6678.00|6945.12|813.96| +2452640|43716|4051|59712|1076059|2006|27153|1|121|434|83|73.23|109.11|42.55|0.00|3531.65|6078.09|9056.13|141.26|0.00|3531.65|3672.91|-2546.44| +2452640|43716|14077|59712|1076059|2006|27153|1|89|434|53|98.64|180.51|70.39|261.14|3730.67|5227.92|9567.03|277.56|261.14|3469.53|3747.09|-1758.39| +2452640|43716|12031|59712|1076059|2006|27153|1|183|434|68|96.27|126.11|63.05|0.00|4287.40|6546.36|8575.48|171.49|0.00|4287.40|4458.89|-2258.96| +2452640|43716|13962|59712|1076059|2006|27153|1|112|434|41|42.94|48.95|23.49|0.00|963.09|1760.54|2006.95|28.89|0.00|963.09|991.98|-797.45| +2452640|43716|132|59712|1076059|2006|27153|1|101|434|85|51.16|90.04|71.13|2297.49|6046.05|4348.60|7653.40|262.39|2297.49|3748.56|4010.95|-600.04| +2452640|43716|14745|59712|1076059|2006|27153|1|128|434|72|78.07|142.08|71.04|0.00|5114.88|5621.04|10229.76|358.04|0.00|5114.88|5472.92|-506.16| +2452640|43716|10809|59712|1076059|2006|27153|1|125|434|50|6.42|10.01|3.10|0.00|155.00|321.00|500.50|6.20|0.00|155.00|161.20|-166.00| +||1032||1076059|2006||||434|11|||64.27||706.97||2079.44|28.27||706.97||-353.98| +2452640|43716|11065|59712|1076059|2006|27153|1|150|434|23|65.52|129.07|29.68|0.00|682.64|1506.96|2968.61|40.95|0.00|682.64|723.59|-824.32| +2451546|53415|91|79762|1582108|1701|28847|4|23|435|98|69.47|95.17|94.21|0.00|9232.58|6808.06|9326.66|646.28|0.00|9232.58|9878.86|2424.52| +2451546|53415|10772|79762|1582108|1701|28847|4|251|435|26|23.45|27.20|17.40|330.25|452.40|609.70|707.20|10.99|330.25|122.15|133.14|-487.55| +2451546|53415|2387|79762|1582108|1701|28847|4|35|435|88|39.77|62.43|17.48|0.00|1538.24|3499.76|5493.84|30.76|0.00|1538.24|1569.00|-1961.52| +2451546|53415|15275|79762|1582108|1701|28847|4|52|435|57|82.65|145.46|84.36|2548.51|4808.52|4711.05|8291.22|0.00|2548.51|2260.01|2260.01|-2451.04| +2451546|53415|10285|79762|1582108|1701|28847|4|254|435|85|95.43|162.23|157.36|6019.02|13375.60|8111.55|13789.55|147.13|6019.02|7356.58|7503.71|-754.97| +2451546||16723|79762|1582108|1701|28847|4|31|435||||||0.00|||0.00||||| +2451546|53415|11995|79762|1582108|1701|28847|4|84|435|16|15.03|15.48|1.23|0.00|19.68|240.48|247.68|1.18|0.00|19.68|20.86|-220.80| +2451546|53415|605|79762|1582108|1701|28847|4|15|435|66|67.03|79.76|37.48|0.00|2473.68|4423.98|5264.16|222.63|0.00|2473.68|2696.31|-1950.30| +2452109|34048|9653|55736|1678834|4685|26136|2|214|436|20|77.35|100.55|64.35|733.59|1287.00|1547.00|2011.00|27.67|733.59|553.41|581.08|-993.59| +2452109|34048|5801|55736|1678834|4685|26136|2|236|436|85|97.24|149.74|98.82|0.00|8399.70|8265.40|12727.90|587.97|0.00|8399.70|8987.67|134.30| +2452109|34048|5063|55736|1678834|4685|26136|2|42|436|38|95.06|189.16|17.02|0.00|646.76|3612.28|7188.08|25.87|0.00|646.76|672.63|-2965.52| +2452109|34048|8673|55736|1678834|4685|26136|2|4|436|95|40.15|64.24|16.06|381.42|1525.70|3814.25|6102.80|57.21|381.42|1144.28|1201.49|-2669.97| +2452109|34048|6471|55736|1678834|4685|26136|2|33|436|42|15.61|20.91|6.90|0.00|289.80|655.62|878.22|14.49|0.00|289.80|304.29|-365.82| +2452109|34048|10143|55736|1678834|4685|26136|2|173|436|71|16.48|25.54|10.72|0.00|761.12|1170.08|1813.34|7.61|0.00|761.12|768.73|-408.96| +2452109|34048|3575|55736|1678834|4685|26136|2|127|436|76|15.04|22.40|8.73|0.00|663.48|1143.04|1702.40|59.71|0.00|663.48|723.19|-479.56| +2452109|34048|12507|55736|1678834|4685|26136|2|202|436|26|13.34|16.40|2.46|0.00|63.96|346.84|426.40|5.75|0.00|63.96|69.71|-282.88| +2452296|51865|15853|18333|1580719|1058|28997|7|231|437|38|7.55|8.68|0.95|0.00|36.10|286.90|329.84|0.36|0.00|36.10|36.46|-250.80| +2452296|51865|10819|18333||1058||7||437||85.62||77.43|0.00|6504.12|7192.08||65.04|0.00|6504.12||| +2452296|51865|12955|18333|1580719|1058|28997|7|290|437|24|94.10|96.92|39.73|0.00|953.52|2258.40|2326.08|47.67|0.00|953.52|1001.19|-1304.88| +2452296|51865|7213|18333|1580719|1058|28997|7|195|437|84|93.66|116.13|108.00|1179.36|9072.00|7867.44|9754.92|394.63|1179.36|7892.64|8287.27|25.20| +2452296|51865|8805|18333|1580719|1058|28997|7|21|437|7|70.83|128.91|99.26|270.97|694.82|495.81|902.37|16.95|270.97|423.85|440.80|-71.96| +2452296|51865|15951|18333|1580719|1058|28997|7|175|437|71|2.60|4.42|4.37|0.00|310.27|184.60|313.82|18.61|0.00|310.27|328.88|125.67| +2452296|51865|11568|18333|1580719|1058|28997|7|211|437|40|53.60|54.13|42.22|0.00|1688.80|2144.00|2165.20|0.00|0.00|1688.80|1688.80|-455.20| +2452296|51865|5067|18333|1580719|1058|28997|7|225|437|49|78.94|135.77|81.46|0.00|3991.54|3868.06|6652.73|359.23|0.00|3991.54|4350.77|123.48| +2452296|51865|8556|18333|1580719|1058|28997|7|90|437|93|29.94|29.94|24.85|0.00|2311.05|2784.42|2784.42|23.11|0.00|2311.05|2334.16|-473.37| +2452296|51865|5151|18333|1580719|1058|28997|7|235|437|1|50.31|94.58|13.24|0.00|13.24|50.31|94.58|0.66|0.00|13.24|13.90|-37.07| +2451502|51834|11830|28482|272911|1340|42169|7|297|438|1|72.17|108.25|28.14|0.00|28.14|72.17|108.25|1.68|0.00|28.14|29.82|-44.03| +2451502|51834|17156|28482|272911|1340|42169|7|267|438|89|27.45|40.07|32.85|0.00|2923.65|2443.05|3566.23|116.94|0.00|2923.65|3040.59|480.60| +2451502|51834|8960|28482|272911|1340|42169|7|40|438|91|76.15|100.51|25.12|0.00|2285.92|6929.65|9146.41|182.87|0.00|2285.92|2468.79|-4643.73| +2451502|51834|7435|28482|272911|1340|42169|7|293|438|11|3.76|4.51|2.20|8.22|24.20|41.36|49.61|0.31|8.22|15.98|16.29|-25.38| +2451502|51834|3847|28482|272911|1340|42169|7|167|438|26|77.60|133.47|54.72|0.00|1422.72|2017.60|3470.22|128.04|0.00|1422.72|1550.76|-594.88| +2451502|51834|5737|28482|272911|1340|42169|7|189|438|85|75.03|93.03|76.28|0.00|6483.80|6377.55|7907.55|324.19|0.00|6483.80|6807.99|106.25| +||12865|28482|272911|1340|42169||130|438|94|95.05||94.66|0.00|8898.04||14831.32||0.00||9609.88|| +2451502|51834|3271|28482|272911|1340|42169|7|249|438|1|51.74|81.23|6.49|0.00|6.49|51.74|81.23|0.45|0.00|6.49|6.94|-45.25| +2451502|51834|3541|28482|272911|1340|42169|7|266|438|85|14.29|16.29|1.14|0.00|96.90|1214.65|1384.65|1.93|0.00|96.90|98.83|-1117.75| +2451502|51834|17038|28482|272911|1340|42169|7|32|438|68|32.71|34.34|5.15|0.00|350.20|2224.28|2335.12|14.00|0.00|350.20|364.20|-1874.08| +2450896|34879|13604|23731|891902|2026|4597|8|84|439|1|71.56|74.42|73.67|0.00|73.67|71.56|74.42|5.15|0.00|73.67|78.82|2.11| +2450896|34879|17377|23731|891902|2026|4597|8|289|439|58|4.20|4.74|1.70|0.00|98.60|243.60|274.92|0.98|0.00|98.60|99.58|-145.00| +2450896|34879|10120|23731|891902|2026|4597|8|16|439|98|76.50|88.74|44.37|0.00|4348.26|7497.00|8696.52|304.37|0.00|4348.26|4652.63|-3148.74| +2450896|34879|4100|23731|891902|2026|4597|8|121|439|5|15.67|26.79|10.98|0.00|54.90|78.35|133.95|4.39|0.00|54.90|59.29|-23.45| +2450896|34879|6490|23731|891902|2026|4597|8|25|439|34|67.80|116.61|6.99|0.00|237.66|2305.20|3964.74|11.88|0.00|237.66|249.54|-2067.54| +2450896|34879|12308|23731|891902|2026|4597|8|134|439|47|85.81|110.69|70.84|0.00|3329.48|4033.07|5202.43|166.47|0.00|3329.48|3495.95|-703.59| +2450896|34879|2324|23731|891902|2026|4597|8|151|439|33|98.63|187.39|54.34|0.00|1793.22|3254.79|6183.87|53.79|0.00|1793.22|1847.01|-1461.57| +2450896|34879|880|23731|891902|2026|4597|8|109|439|19|76.25|118.95|23.79|9.04|452.01|1448.75|2260.05|26.57|9.04|442.97|469.54|-1005.78| +2450896|34879|12514|23731|891902|2026|4597|8|9|439|89|60.51|83.50|67.63|0.00|6019.07|5385.39|7431.50|300.95|0.00|6019.07|6320.02|633.68| +2450896|34879|9037|23731|891902|2026|4597|8|192|439|95|21.27|27.86|17.27|0.00|1640.65|2020.65|2646.70|32.81|0.00|1640.65|1673.46|-380.00| +2450896|34879|17347|23731|891902|2026|4597|8|121|439|96|73.32|112.91|27.09|0.00|2600.64|7038.72|10839.36|182.04|0.00|2600.64|2782.68|-4438.08| +2450896|34879|11791|23731|891902|2026|4597|8|219|439|15|18.56|22.64|1.13|0.00|16.95|278.40|339.60|1.52|0.00|16.95|18.47|-261.45| +2450896|34879|8158|23731|891902|2026|4597|8|225|439|91|26.44|29.08|20.06|0.00|1825.46|2406.04|2646.28|36.50|0.00|1825.46|1861.96|-580.58| +2450896|34879|3883|23731|891902|2026|4597|8|65|439|89|68.32|99.06|88.16|0.00|7846.24|6080.48|8816.34|470.77|0.00|7846.24|8317.01|1765.76| +2450896|34879|1448|23731|891902|2026|4597|8|108|439|22|9.23|10.89|3.59|0.00|78.98|203.06|239.58|2.36|0.00|78.98|81.34|-124.08| +2450896|34879|4060|23731|891902|2026|4597|8|178|439|38|58.88|64.17|8.34|0.00|316.92|2237.44|2438.46|19.01|0.00|316.92|335.93|-1920.52| +2451838|44492|8023|20647|1227029|611|33440|10|87|440|67|51.25|52.78|24.80|1080.04|1661.60|3433.75|3536.26|23.26|1080.04|581.56|604.82|-2852.19| +2451838|44492|5429|20647|1227029|611|33440|10|177|440|53|31.20|38.68|17.40|0.00|922.20|1653.60|2050.04|27.66|0.00|922.20|949.86|-731.40| +2451838|44492|2657|20647|1227029|611|33440|10|52|440|27|53.00|104.41|8.35|0.00|225.45|1431.00|2819.07|9.01|0.00|225.45|234.46|-1205.55| +2451838|44492|10339|20647|1227029|611|33440|10|41|440|90|95.42|103.05|69.04|0.00|6213.60|8587.80|9274.50|310.68|0.00|6213.60|6524.28|-2374.20| +2451838|44492|4909|20647|1227029|611|33440|10|44|440|50|93.51|127.17|89.01|0.00|4450.50|4675.50|6358.50|0.00|0.00|4450.50|4450.50|-225.00| +2451838|44492|9566|20647|1227029|611|33440|10|228|440|35|31.01|31.01|22.32|0.00|781.20|1085.35|1085.35|70.30|0.00|781.20|851.50|-304.15| +2451838|44492|1442|20647|1227029|611|33440|10|48|440|40|48.33|68.62|26.76|0.00|1070.40|1933.20|2744.80|42.81|0.00|1070.40|1113.21|-862.80| +2451838|44492|6287|20647|1227029|611|33440|10|195|440|100|49.85|58.32|47.82|0.00|4782.00|4985.00|5832.00|47.82|0.00|4782.00|4829.82|-203.00| +2451838|44492|5255|20647|1227029|611|33440|10|288|440|87|88.28|162.43|34.11|0.00|2967.57|7680.36|14131.41|267.08|0.00|2967.57|3234.65|-4712.79| +2451838|44492|13261|20647|1227029|611|33440|10|238|440|75|37.93|44.37|23.51|0.00|1763.25|2844.75|3327.75|0.00|0.00|1763.25|1763.25|-1081.50| +2451838|44492|1421|20647|1227029|611|33440|10|87|440|79|87.65|95.53|67.82|0.00|5357.78|6924.35|7546.87|107.15|0.00|5357.78|5464.93|-1566.57| +2451838|44492|8917||||33440|||440||64.20|110.42||2425.13|2755.84||||2425.13|330.71||| +2451838|44492|3623|20647|1227029|611|33440|10|257|440|81|53.89|106.70|39.47|0.00|3197.07|4365.09|8642.70|127.88|0.00|3197.07|3324.95|-1168.02| +2451172|65698|4238|84518|366250|4335|11267|8|162|441|86|83.25|89.91|9.89|0.00|850.54|7159.50|7732.26|51.03|0.00|850.54|901.57|-6308.96| +2451172|65698|8527|84518|366250|4335|11267|8|73|441|59|80.51|145.72|20.40|0.00|1203.60|4750.09|8597.48|72.21|0.00|1203.60|1275.81|-3546.49| +2451172|65698|15880|84518|366250|4335|11267|8|282|441|39|16.06|22.32|4.46|0.00|173.94|626.34|870.48|6.95|0.00|173.94|180.89|-452.40| +2451172|65698|5264||||11267|||441||41.46|||2318.29|2341.71|2031.54|||2318.29||23.65|| +2451172|65698|11035|84518|366250|4335|11267|8|7|441|42|27.94|52.24|3.13|0.00|131.46|1173.48|2194.08|0.00|0.00|131.46|131.46|-1042.02| +2451172|65698|7195|84518|366250|4335|11267|8|179|441|58|26.30|36.03|24.86|576.75|1441.88|1525.40|2089.74|51.90|576.75|865.13|917.03|-660.27| +2451172|65698|14995|84518|366250|4335|11267|8|99|441|55|58.09|66.22|63.57|0.00|3496.35|3194.95|3642.10|104.89|0.00|3496.35|3601.24|301.40| +2451172|65698|1699|84518|366250|4335|11267|8|178|441|21|63.39|95.08|77.96|1637.16|1637.16|1331.19|1996.68|0.00|1637.16|0.00|0.00|-1331.19| +2451432|37881|5701|29242|1787781|5073|15565|8|120|442|1|22.64|39.84|33.86|0.00|33.86|22.64|39.84|2.37|0.00|33.86|36.23|11.22| +2451432|37881|3604|29242|1787781|5073|15565|8|110|442|71|56.97|106.53|84.15|0.00|5974.65|4044.87|7563.63|59.74|0.00|5974.65|6034.39|1929.78| +2451432|37881|10298|29242|1787781|5073|15565|8|232|442|64|73.56|128.73|2.57|0.00|164.48|4707.84|8238.72|3.28|0.00|164.48|167.76|-4543.36| +2451432|37881|6904|29242|1787781|5073|15565|8|23|442|5|2.39|3.51|0.00|0.00|0.00|11.95|17.55|0.00|0.00|0.00|0.00|-11.95| +2451432|37881|356|29242|1787781|5073|15565|8|25|442|38|12.78|15.33|4.13|0.00|156.94|485.64|582.54|7.84|0.00|156.94|164.78|-328.70| +2451432|37881|2059|29242|1787781|5073|15565|8|229|442|93|93.66|110.51|99.45|0.00|9248.85|8710.38|10277.43|739.90|0.00|9248.85|9988.75|538.47| +2451432|37881|13676|29242|1787781|5073|15565|8|114|442|18|39.68|77.77|38.10|0.00|685.80|714.24|1399.86|34.29|0.00|685.80|720.09|-28.44| +2451432|37881|17590|29242|1787781|5073|15565|8|299|442|30|98.87|106.77|61.92|0.00|1857.60|2966.10|3203.10|167.18|0.00|1857.60|2024.78|-1108.50| +2451432|37881|14288|29242|1787781|5073|15565|8|261|442|47|42.01|49.99|20.99|0.00|986.53|1974.47|2349.53|0.00|0.00|986.53|986.53|-987.94| +2451432|37881|7756|29242|1787781|5073|15565|8|13|442|80|1.02|1.48|0.04|0.00|3.20|81.60|118.40|0.19|0.00|3.20|3.39|-78.40| +2451432|37881|8240|29242|1787781|5073|15565|8|15|442|15|39.73|68.33|54.66|336.15|819.90|595.95|1024.95|19.35|336.15|483.75|503.10|-112.20| +2451432|37881|10270|29242|1787781|5073|15565|8|29|442|87|59.73|93.77|66.57|0.00|5791.59|5196.51|8157.99|289.57|0.00|5791.59|6081.16|595.08| +2451432|37881|11264|29242|1787781|5073|15565|8|32|442|81|73.96|119.81|41.93|815.11|3396.33|5990.76|9704.61|25.81|815.11|2581.22|2607.03|-3409.54| +2451432|37881|4090|29242|1787781|5073|15565|8|274|442|91|3.58|6.01|1.44|0.00|131.04|325.78|546.91|9.17|0.00|131.04|140.21|-194.74| +2451432|37881|7387|29242|1787781|5073|15565|8|244|442|36|79.92|155.04|72.86|0.00|2622.96|2877.12|5581.44|78.68|0.00|2622.96|2701.64|-254.16| +2451502|46364|6814|22814|1622496|5731|4566|10|269|443|90|12.80|22.91|13.74|321.51|1236.60|1152.00|2061.90|0.00|321.51|915.09|915.09|-236.91| +2451502|46364|3464|22814|1622496|5731|4566|10|218|443|88|55.45|107.01|17.12|0.00|1506.56|4879.60|9416.88|120.52|0.00|1506.56|1627.08|-3373.04| +2451502|46364|12404|22814|1622496|5731|4566|10|299|443|11|24.56|38.80|23.28|0.00|256.08|270.16|426.80|12.80|0.00|256.08|268.88|-14.08| +2451502|46364|3422|22814|1622496|5731|4566|10|286|443|65|63.86|111.75|74.87|0.00|4866.55|4150.90|7263.75|145.99|0.00|4866.55|5012.54|715.65| +2451502|46364|11126|22814|1622496|5731|4566|10|33|443|82|4.28|5.82|4.77|0.00|391.14|350.96|477.24|7.82|0.00|391.14|398.96|40.18| +2451502|46364|16183|22814|1622496|5731|4566|10|112|443|6|54.52|81.23|4.06|0.00|24.36|327.12|487.38|0.48|0.00|24.36|24.84|-302.76| +||7858|22814||5731|4566|10|74|443||28.33|43.34|22.97|0.00||||3.44|0.00|68.91|72.35|| +2451502|46364|11113|22814|1622496|5731|4566|10|133|443|57|58.45|78.90|48.91|0.00|2787.87|3331.65|4497.30|195.15|0.00|2787.87|2983.02|-543.78| +2451502|46364|14120|22814|1622496|5731|4566|10|4|443|24|65.47|110.64|49.78|322.57|1194.72|1571.28|2655.36|17.44|322.57|872.15|889.59|-699.13| +2451502|46364|7228|22814|1622496|5731|4566|10|39|443|52|75.48|84.53|44.80|0.00|2329.60|3924.96|4395.56|93.18|0.00|2329.60|2422.78|-1595.36| +2451502||2806|22814|1622496|5731||||443|||9.41|5.17||||564.60|27.91||||-156.60| +2451502|46364|17332|22814|1622496|5731|4566|10|59|443|22|12.15|15.67|4.38|57.81|96.36|267.30|344.74|2.31|57.81|38.55|40.86|-228.75| +2451502|46364|3806|22814|1622496|5731|4566|10|281|443|25|27.81|41.43|6.21|0.00|155.25|695.25|1035.75|10.86|0.00|155.25|166.11|-540.00| +2451502|46364|10010|22814|1622496|5731|4566|10|283|443|100|48.64|57.39|30.99|0.00|3099.00|4864.00|5739.00|154.95|0.00|3099.00|3253.95|-1765.00| +2452521|71475|421|3617|1438751|5085|39466|10|1|444|3|80.95|131.94|102.91|308.73|308.73|242.85|395.82|0.00|308.73|0.00|0.00|-242.85| +2452521|71475|12804|3617|1438751|5085|39466|10|259|444|18|15.40|30.18|13.27|181.53|238.86|277.20|543.24|0.57|181.53|57.33|57.90|-219.87| +2452521|71475|5763|3617|1438751|5085|39466|10|19|444|21|25.11|39.42|24.04|0.00|504.84|527.31|827.82|35.33|0.00|504.84|540.17|-22.47| +2452521|71475|4350|3617|1438751|5085|39466|10|207|444|31|67.27|106.28|60.57|0.00|1877.67|2085.37|3294.68|168.99|0.00|1877.67|2046.66|-207.70| +2452521|71475|3061|3617|1438751|5085|39466|10|31|444|71|19.33|29.76|25.89|0.00|1838.19|1372.43|2112.96|147.05|0.00|1838.19|1985.24|465.76| +2452521|71475|6900|3617|1438751|5085|39466|10|131|444|100|43.57|79.73|70.16|0.00|7016.00|4357.00|7973.00|491.12|0.00|7016.00|7507.12|2659.00| +2452521|71475|15417|3617|1438751|5085|39466|10|49|444|39|43.36|69.37|46.47|0.00|1812.33|1691.04|2705.43|72.49|0.00|1812.33|1884.82|121.29| +2452521|71475|17295|3617|1438751|5085|39466|10|151|444|90|55.81|72.55|67.47|3825.54|6072.30|5022.90|6529.50|179.74|3825.54|2246.76|2426.50|-2776.14| +2452521|71475|7425|3617|1438751|5085|39466|10|152|444|17|71.01|100.83|89.73|0.00|1525.41|1207.17|1714.11|76.27|0.00|1525.41|1601.68|318.24| +2451473|62889|6625|6753|995762|1924|30533|1|23|445|91|78.53|155.48|135.26|0.00|12308.66|7146.23|14148.68|492.34|0.00|12308.66|12801.00|5162.43| +2451473|62889|13345|6753|995762|1924|30533|1|57|445|11|73.70|137.81|71.66|0.00|788.26|810.70|1515.91|15.76|0.00|788.26|804.02|-22.44| +2451473|62889|16906|6753|995762|1924|30533|1|292|445|39|38.14|47.29|29.31|0.00|1143.09|1487.46|1844.31|68.58|0.00|1143.09|1211.67|-344.37| +2451473|62889|14950|6753|995762|1924|30533|1|265|445|31|91.73|161.44|20.98|0.00|650.38|2843.63|5004.64|39.02|0.00|650.38|689.40|-2193.25| +2451473|62889|14143|6753|995762|1924|30533|1|11|445|72|68.83|74.33|2.97|0.00|213.84|4955.76|5351.76|19.24|0.00|213.84|233.08|-4741.92| +2451473|62889|14119|6753|995762|1924|30533|1|107|445|32|15.97|16.28|5.53|0.00|176.96|511.04|520.96|5.30|0.00|176.96|182.26|-334.08| +2451473|62889|6154|6753|995762|1924|30533|1|287|445|36|52.77|86.54|82.21|0.00|2959.56|1899.72|3115.44|118.38|0.00|2959.56|3077.94|1059.84| +2451473|62889|15082|6753|995762|1924|30533|1|75|445|92|52.53|94.55|79.42|0.00|7306.64|4832.76|8698.60|219.19|0.00|7306.64|7525.83|2473.88| +|34874|3584|||2032||1||446|1|62.18|103.21|62.95||62.95|62.18|103.21|||62.95||0.77| +2451524|34874|6452|77401|1039940|2032|37371|1|250|446|27|42.85|66.41|3.98|0.00|107.46|1156.95|1793.07|7.52|0.00|107.46|114.98|-1049.49| +2451524|34874|9044|77401|1039940|2032|37371|1|109|446|49|98.67|121.36|83.73|0.00|4102.77|4834.83|5946.64|328.22|0.00|4102.77|4430.99|-732.06| +2451524|34874|14785|77401|1039940|2032|37371|1|119|446|56|75.14|105.94|52.97|0.00|2966.32|4207.84|5932.64|266.96|0.00|2966.32|3233.28|-1241.52| +2451524|34874|7436|77401|1039940|2032|37371|1|243|446|81|46.35|61.18|1.83|10.37|148.23|3754.35|4955.58|6.89|10.37|137.86|144.75|-3616.49| +2451524|34874|5782|77401|1039940|2032|37371|1|227|446|62|58.70|78.07|46.84|2671.75|2904.08|3639.40|4840.34|4.64|2671.75|232.33|236.97|-3407.07| +2451524|34874|4855|77401|1039940|2032|37371|1|138|446|71|31.49|50.38|31.23|0.00|2217.33|2235.79|3576.98|44.34|0.00|2217.33|2261.67|-18.46| +2451524|34874|6722|77401|1039940|2032|37371|1|79|446|5|17.63|32.26|12.90|0.00|64.50|88.15|161.30|1.93|0.00|64.50|66.43|-23.65| +2450979|71152|1976|61212|1546201|1943|30885|10|263|447|46|6.82|7.50|7.12|0.00|327.52|313.72|345.00|26.20|0.00|327.52|353.72|13.80| +2450979|71152|5024|61212|1546201|1943|30885|10|226|447|2|96.05|129.66|16.85|0.00|33.70|192.10|259.32|0.33|0.00|33.70|34.03|-158.40| +2450979|71152|6664|61212|1546201|1943|30885|10|74|447|55|82.85|84.50|65.91|0.00|3625.05|4556.75|4647.50|253.75|0.00|3625.05|3878.80|-931.70| +2450979|71152|10609|61212|1546201|1943|30885|10|39|447|9|73.77|146.80|137.99|0.00|1241.91|663.93|1321.20|37.25|0.00|1241.91|1279.16|577.98| +2450979|71152|1285|61212|1546201|1943|30885|10|35|447|37|66.06|99.09|53.50|0.00|1979.50|2444.22|3666.33|158.36|0.00|1979.50|2137.86|-464.72| +2450979|71152|1166|61212|1546201|1943|30885|10|62|447|17|57.62|88.15|86.38|0.00|1468.46|979.54|1498.55|58.73|0.00|1468.46|1527.19|488.92| +2450979|71152|1957|61212|1546201|1943|30885|10|145|447|16|70.23|82.16|40.25|309.12|644.00|1123.68|1314.56|26.79|309.12|334.88|361.67|-788.80| +2450979|71152|7808|61212|1546201|1943|30885|10|17|447|85|19.22|37.28|23.48|1337.18|1995.80|1633.70|3168.80|59.27|1337.18|658.62|717.89|-975.08| +2450979|71152|10357|61212|1546201|1943|30885|10|277|447|100|32.45|61.00|11.59|0.00|1159.00|3245.00|6100.00|11.59|0.00|1159.00|1170.59|-2086.00| +2450979|71152|6190|61212|1546201|1943|30885|10|98|447|24|36.08|46.18|24.47|0.00|587.28|865.92|1108.32|35.23|0.00|587.28|622.51|-278.64| +2450979|71152|11708|61212|1546201|1943|30885|10|47|447|5|3.00|4.68|1.45|0.00|7.25|15.00|23.40|0.00|0.00|7.25|7.25|-7.75| +2451406|36440|3967|34691|1134664|6990|36563|8|163|448|85|59.43|79.63|33.44|0.00|2842.40|5051.55|6768.55|56.84|0.00|2842.40|2899.24|-2209.15| +2451406|36440|8896|34691|1134664|6990|36563|8|229|448|37|43.75|54.25|4.34|0.00|160.58|1618.75|2007.25|4.81|0.00|160.58|165.39|-1458.17| +2451406|36440|9208|34691|1134664|6990|36563|8|121|448|68|9.09|16.54|4.46|0.00|303.28|618.12|1124.72|0.00|0.00|303.28|303.28|-314.84| +|36440|968|34691||6990||8||448|||44.58|8.47|93.33|160.93|756.39|||93.33|||-688.79| +2451406|36440|3211|34691|1134664|6990|36563|8|233|448|56|99.34|181.79|152.70|0.00|8551.20|5563.04|10180.24|769.60|0.00|8551.20|9320.80|2988.16| +2451406|36440|2552|34691|1134664|6990|36563|8|92|448|76|19.36|28.26|0.28|0.00|21.28|1471.36|2147.76|0.63|0.00|21.28|21.91|-1450.08| +2451406|36440|14251|34691|1134664|6990|36563|8|22|448|33|46.78|83.73|79.54|0.00|2624.82|1543.74|2763.09|236.23|0.00|2624.82|2861.05|1081.08| +2451406|36440|17356|34691|1134664|6990|36563|8|37|448|34|74.34|80.28|52.98|216.15|1801.32|2527.56|2729.52|15.85|216.15|1585.17|1601.02|-942.39| +2451406|36440|2078|34691|1134664|6990|36563|8|290|448|27|93.81|131.33|40.71|0.00|1099.17|2532.87|3545.91|0.00|0.00|1099.17|1099.17|-1433.70| +2451406|36440|8845|34691|1134664|6990|36563|8|185|448|36|80.05|126.47|103.70|0.00|3733.20|2881.80|4552.92|37.33|0.00|3733.20|3770.53|851.40| +2451406|36440|13892|34691|1134664|6990|36563|8|75|448|85|38.36|62.91|50.95|0.00|4330.75|3260.60|5347.35|259.84|0.00|4330.75|4590.59|1070.15| +2451406|36440|10694|34691|1134664|6990|36563|8|76|448|48|27.77|34.71|24.64|685.97|1182.72|1332.96|1666.08|4.96|685.97|496.75|501.71|-836.21| +2451406|36440|4898|34691|1134664|6990|36563|8|3|448|5|28.80|41.47|2.90|0.00|14.50|144.00|207.35|1.30|0.00|14.50|15.80|-129.50| +2451406|36440|9328|34691|1134664|6990|36563|8|240|448|21|12.91|19.49|8.18|0.00|171.78|271.11|409.29|8.58|0.00|171.78|180.36|-99.33| +2451172||16480|27603||3616|29846|1|8|449|||||||||34.70||||-6933.00| +2451172|65010|15013|27603|1028986|3616|29846|1|78|449|91|53.67|89.09|73.94|0.00|6728.54|4883.97|8107.19|0.00|0.00|6728.54|6728.54|1844.57| +2451172|65010|4994|27603|1028986|3616|29846|1|46|449|84|93.03|149.77|98.84|0.00|8302.56|7814.52|12580.68|0.00|0.00|8302.56|8302.56|488.04| +2451172|65010|16754|27603||3616|29846|||449|16|||||120.32||802.40|7.21||120.32|127.53|-644.00| +||5632||1028986|3616|||54|449||83.10|132.12||0.00|1405.60||||0.00||1461.82|-3248.00| +2451172|65010|17726|27603|1028986|3616|29846|1|213|449|63|28.46|44.11|33.96|0.00|2139.48|1792.98|2778.93|149.76|0.00|2139.48|2289.24|346.50| +2451172|65010|7702|27603|1028986|3616|29846|1|296|449|35|37.67|68.93|28.26|415.42|989.10|1318.45|2412.55|17.21|415.42|573.68|590.89|-744.77| +2451172|65010|10904|27603|1028986|3616|29846|1|298|449|12|15.64|29.09|23.27|0.00|279.24|187.68|349.08|8.37|0.00|279.24|287.61|91.56| +2451172|65010|9691|27603|1028986|3616|29846|1|109|449|82|14.57|17.77|13.14|991.28|1077.48|1194.74|1457.14|6.03|991.28|86.20|92.23|-1108.54| +2451172|65010|8206|27603||3616|29846|||449||52.22|||65.74|||2267.82|35.50|65.74|||| +2451172|65010|1876|27603|1028986|3616|29846|1|93|449|48|91.90|98.33|60.96|0.00|2926.08|4411.20|4719.84|204.82|0.00|2926.08|3130.90|-1485.12| +2451172|65010|3028|27603|1028986|3616|29846|1|282|449|29|6.93|12.05|5.30|0.00|153.70|200.97|349.45|10.75|0.00|153.70|164.45|-47.27| +2451172|65010|10702|27603|1028986|3616|29846|1|172|449|66|32.70|49.05|41.69|0.00|2751.54|2158.20|3237.30|165.09|0.00|2751.54|2916.63|593.34| +2451172|65010|3907|27603|1028986|3616|29846|1|226|449|50|28.53|43.08|40.92|0.00|2046.00|1426.50|2154.00|20.46|0.00|2046.00|2066.46|619.50| +2451172|65010|9422|27603|1028986|3616|29846|1|95|449|45|87.75|93.89|76.05|0.00|3422.25|3948.75|4225.05|171.11|0.00|3422.25|3593.36|-526.50| +2452434|44874|2100|17429|1810965|4501|15684|10|87|450|61|6.56|7.15|5.29|0.00|322.69|400.16|436.15|0.00|0.00|322.69|322.69|-77.47| +2452434|44874|9276|17429|1810965|4501|15684|10|280|450|71|15.11|30.22|23.87|0.00|1694.77|1072.81|2145.62|33.89|0.00|1694.77|1728.66|621.96| +2452434|44874|10290|17429|1810965|4501|15684|10|178|450|88|98.42|175.18|84.08|0.00|7399.04|8660.96|15415.84|369.95|0.00|7399.04|7768.99|-1261.92| +2452434|44874|5859|17429|1810965|4501|15684|10|157|450|9|5.34|6.40|6.33|55.26|56.97|48.06|57.60|0.11|55.26|1.71|1.82|-46.35| +2452434|44874|10110|17429|1810965|4501|15684|10|263|450|53|99.61|130.48|10.43|0.00|552.79|5279.33|6915.44|49.75|0.00|552.79|602.54|-4726.54| +2452434|44874|9997|17429|1810965|4501|15684|10|155|450|81|17.08|30.40|14.59|0.00|1181.79|1383.48|2462.40|0.00|0.00|1181.79|1181.79|-201.69| +2452434|44874|17247|17429|1810965|4501|15684|10|105|450|66|13.82|20.73|15.75|0.00|1039.50|912.12|1368.18|41.58|0.00|1039.50|1081.08|127.38| +2452434|44874|9792|17429|1810965|4501|15684|10|174|450|49|64.91|103.20|92.88|0.00|4551.12|3180.59|5056.80|273.06|0.00|4551.12|4824.18|1370.53| +2452434|44874|4146|17429|1810965|4501|15684|10|264|450|74|50.22|60.76|30.38|22.48|2248.12|3716.28|4496.24|200.30|22.48|2225.64|2425.94|-1490.64| +2452434|44874|8887|17429|1810965|4501|15684|10|112|450|34|98.68|131.24|36.74|0.00|1249.16|3355.12|4462.16|49.96|0.00|1249.16|1299.12|-2105.96| +2452434|44874|6009|17429|1810965|4501|15684|10|298|450|25|19.39|31.99|16.95|0.00|423.75|484.75|799.75|38.13|0.00|423.75|461.88|-61.00| +2452434|44874|457|17429|1810965|4501|15684|10|3|450|53|94.85|141.32|120.12|1655.25|6366.36|5027.05|7489.96|376.88|1655.25|4711.11|5087.99|-315.94| +2452434|44874|11169|17429|1810965|4501|15684|10|284|450|15|9.16|16.30|7.66|0.00|114.90|137.40|244.50|10.34|0.00|114.90|125.24|-22.50| +2452434|44874|9769|17429|1810965|4501|15684|10|163|450|12|38.46|64.99|40.29|0.00|483.48|461.52|779.88|0.00|0.00|483.48|483.48|21.96| +2451115|74154|17446|39699|519717|2935|43753|1|282|451|66|41.64|63.70|63.70|0.00|4204.20|2748.24|4204.20|42.04|0.00|4204.20|4246.24|1455.96| +2451115|74154|4610|39699|519717|2935|43753|1|228|451|82|2.70|3.59|1.11|0.00|91.02|221.40|294.38|0.91|0.00|91.02|91.93|-130.38| +2451115|74154|5174|39699|519717|2935|43753|1|121|451|15|21.10|26.79|20.36|0.00|305.40|316.50|401.85|9.16|0.00|305.40|314.56|-11.10| +2451115|74154|11467|39699|519717|2935|43753|1|159|451|88|32.08|36.25|11.96|0.00|1052.48|2823.04|3190.00|10.52|0.00|1052.48|1063.00|-1770.56| +2451115|74154|12686|39699|519717|2935|43753|1|245|451|77|78.38|123.84|60.68|0.00|4672.36|6035.26|9535.68|280.34|0.00|4672.36|4952.70|-1362.90| +2451115|74154|15092|39699|519717|2935|43753|1|269|451|66|61.55|70.78|36.09|0.00|2381.94|4062.30|4671.48|95.27|0.00|2381.94|2477.21|-1680.36| +|74154|1856|39699|||||167|451||7.99|14.06||||551.31|970.14|||135.24||| +2451115|74154|3127|39699|519717|2935|43753|1|21|451|55|22.59|40.66|35.37|0.00|1945.35|1242.45|2236.30|175.08|0.00|1945.35|2120.43|702.90| +2451115|74154|10561|39699|519717|2935|43753|1|45|451|2|69.21|71.97|0.00|0.00|0.00|138.42|143.94|0.00|0.00|0.00|0.00|-138.42| +2451115|74154|14624|39699|519717|2935|43753|1|156|451|72|81.63|136.32|20.44|0.00|1471.68|5877.36|9815.04|14.71|0.00|1471.68|1486.39|-4405.68| +2451115||11812|39699|||43753||37|451||74.39|97.45||||||||3803.59||| +2451115|74154|1996|39699|519717|2935|43753|1|33|451|68|50.19|55.71|47.91|0.00|3257.88|3412.92|3788.28|260.63|0.00|3257.88|3518.51|-155.04| +2451115|74154|15463|39699|519717|2935|43753|1|238|451|31|49.24|69.42|42.34|0.00|1312.54|1526.44|2152.02|105.00|0.00|1312.54|1417.54|-213.90| +2451115|74154|3865|39699|519717|2935|43753|1|107|451|87|9.16|12.45|6.34|0.00|551.58|796.92|1083.15|11.03|0.00|551.58|562.61|-245.34| +2450832|34803|13354|77669|1164609|3996|7736|1|287|452|99|29.49|48.36|42.07|3873.38|4164.93|2919.51|4787.64|26.23|3873.38|291.55|317.78|-2627.96| +2450832|34803|5791|77669|1164609|3996|7736|1|185|452|80|61.23|120.01|28.80|0.00|2304.00|4898.40|9600.80|23.04|0.00|2304.00|2327.04|-2594.40| +2450832|34803|622|77669|1164609|3996|7736|1|86|452|21|55.37|94.68|4.73|0.00|99.33|1162.77|1988.28|4.96|0.00|99.33|104.29|-1063.44| +2450832|34803|8347|77669|1164609|3996|7736|1|261|452|79|28.00|53.20|29.26|0.00|2311.54|2212.00|4202.80|46.23|0.00|2311.54|2357.77|99.54| +2450832|34803|7210|77669|1164609|3996|7736|1|265|452|25|39.89|76.58|73.51|0.00|1837.75|997.25|1914.50|55.13|0.00|1837.75|1892.88|840.50| +2450832|34803|17407|77669|1164609|3996|7736|1|106|452|69|2.62|4.42|2.65|0.00|182.85|180.78|304.98|7.31|0.00|182.85|190.16|2.07| +2450832|34803|4669|77669|1164609|3996|7736|1|99|452|2|82.45|145.93|96.31|0.00|192.62|164.90|291.86|1.92|0.00|192.62|194.54|27.72| +2450832|34803|16813|77669|1164609|3996|7736|1|109|452|68|91.81|134.04|36.19|0.00|2460.92|6243.08|9114.72|73.82|0.00|2460.92|2534.74|-3782.16| +2450832|34803|13120|77669|1164609|3996|7736|1|198|452|52|26.11|31.07|21.74|0.00|1130.48|1357.72|1615.64|56.52|0.00|1130.48|1187.00|-227.24| +2450832|34803|6004|77669|1164609|3996|7736|1|13|452|64|53.55|72.82|60.44|0.00|3868.16|3427.20|4660.48|77.36|0.00|3868.16|3945.52|440.96| +2450832|34803|17656|77669|1164609|3996|7736|1|103|452|35|17.57|32.32|16.48|351.84|576.80|614.95|1131.20|0.00|351.84|224.96|224.96|-389.99| +2450832|34803|4258|77669|1164609|3996|7736|1|231|452|3|37.58|64.26|24.41|0.00|73.23|112.74|192.78|6.59|0.00|73.23|79.82|-39.51| +2451939|62824|13519|56905|1914253|48|643|4|200|453|49|73.66|92.07|85.62|0.00|4195.38|3609.34|4511.43|293.67|0.00|4195.38|4489.05|586.04| +2451939|62824|13983|56905|1914253|48|643|4|272|453|52|51.96|82.61|0.82|0.00|42.64|2701.92|4295.72|0.42|0.00|42.64|43.06|-2659.28| +2451939|62824|1831|56905|1914253|48|643|4|169|453|27|67.14|118.83|76.05|0.00|2053.35|1812.78|3208.41|184.80|0.00|2053.35|2238.15|240.57| +2451939|62824|10961|56905|1914253|48|643|4|146|453|52|45.52|54.62|13.10|0.00|681.20|2367.04|2840.24|54.49|0.00|681.20|735.69|-1685.84| +2451939|62824|1853|56905|1914253|48|643|4|76|453|60|70.69|106.03|92.24|110.68|5534.40|4241.40|6361.80|433.89|110.68|5423.72|5857.61|1182.32| +2451939|62824|13673|56905|1914253|48|643|4|244|453|100|76.54|146.95|111.68|0.00|11168.00|7654.00|14695.00|893.44|0.00|11168.00|12061.44|3514.00| +2451939|62824|9929|56905|1914253|48|643|4|193|453|62|79.50|105.73|12.68|0.00|786.16|4929.00|6555.26|0.00|0.00|786.16|786.16|-4142.84| +2451939|62824|3369|56905|1914253|48|643|4|205|453|30|3.24|4.66|3.30|0.00|99.00|97.20|139.80|3.96|0.00|99.00|102.96|1.80| +2451939|62824|15807|56905|1914253|48|643|4|45|453|59|33.25|61.18|53.83|0.00|3175.97|1961.75|3609.62|0.00|0.00|3175.97|3175.97|1214.22| +2451939|62824|5395|56905|1914253|48|643|4|172|453|24|12.27|15.70|10.20|0.00|244.80|294.48|376.80|7.34|0.00|244.80|252.14|-49.68| +2451939|62824|7001|56905|1914253|48|643|4|105|453|9|23.82|39.54|38.74|0.00|348.66|214.38|355.86|3.48|0.00|348.66|352.14|134.28| +2451882|33061|3698|16391|1817061|5731|8801|10|200|454|47|7.67|12.50|2.37|0.00|111.39|360.49|587.50|0.00|0.00|111.39|111.39|-249.10| +2451882|33061|16763|16391|1817061|5731|8801||286|454|89||71.75|17.22|||5505.54|6385.75||||1624.53|| +2451882|33061|14198|16391|1817061|5731|8801|10|172|454|55|37.39|42.99|18.91|644.83|1040.05|2056.45|2364.45|27.66|644.83|395.22|422.88|-1661.23| +2451882|33061|15452|16391|1817061|5731|8801|10|48|454|1|84.38|163.69|117.85|0.00|117.85|84.38|163.69|0.00|0.00|117.85|117.85|33.47| +2451882|33061|14774|16391|1817061|5731|8801|10|115|454|11|23.46|34.01|31.28|0.00|344.08|258.06|374.11|10.32|0.00|344.08|354.40|86.02| +2451882|33061|1855|16391|1817061|5731|8801|10|220|454|56|39.44|39.83|1.19|0.00|66.64|2208.64|2230.48|1.99|0.00|66.64|68.63|-2142.00| +|33061|5323||1817061|||||454|97|97.51|108.23|73.59|||9458.47|10498.31|142.76|||7280.99|-2320.24| +|33061|1615|||||10|290|454|83||96.64|||3207.95|||177.07||2213.49||-2099.19| +2451882|33061|11930|16391|1817061|5731|8801|10|159|454|48|27.98|47.00|30.55|0.00|1466.40|1343.04|2256.00|0.00|0.00|1466.40|1466.40|123.36| +2451882|33061|2213|16391|1817061|5731|8801|10|9|454|75|28.20|47.09|36.73|1515.11|2754.75|2115.00|3531.75|74.37|1515.11|1239.64|1314.01|-875.36| +2451882|33061|17120|16391|1817061|5731|8801|10|287|454|85|41.32|60.32|46.44|0.00|3947.40|3512.20|5127.20|236.84|0.00|3947.40|4184.24|435.20| +2452207|40532|4387|12799|71777|4708|14086|10|54|455|76|25.58|48.09|6.73|0.00|511.48|1944.08|3654.84|46.03|0.00|511.48|557.51|-1432.60| +2452207|40532|4849|12799|71777|4708|14086|10|9|455|45|44.48|62.27|55.42|0.00|2493.90|2001.60|2802.15|149.63|0.00|2493.90|2643.53|492.30| +2452207|40532|6925|12799|71777|4708|14086|10|266|455|70|25.65|43.60|25.72|0.00|1800.40|1795.50|3052.00|162.03|0.00|1800.40|1962.43|4.90| +2452207|40532|2219|12799|71777|4708|14086|10|11|455|37|82.70|165.40|28.11|0.00|1040.07|3059.90|6119.80|52.00|0.00|1040.07|1092.07|-2019.83| +2452207|40532|1273|12799|71777|4708|14086|10|190|455|79|46.04|58.47|22.80|0.00|1801.20|3637.16|4619.13|162.10|0.00|1801.20|1963.30|-1835.96| +2452207|40532|17063|12799|71777|4708|14086|10|282|455|12|61.50|83.64|25.92|0.00|311.04|738.00|1003.68|9.33|0.00|311.04|320.37|-426.96| +2452207|40532|3199|12799|71777|4708|14086|10|153|455|67|90.83|139.87|110.49|518.19|7402.83|6085.61|9371.29|619.61|518.19|6884.64|7504.25|799.03| +2452207|40532|7153|12799|71777|4708|14086|10|183|455|75|13.55|21.81|19.84|0.00|1488.00|1016.25|1635.75|119.04|0.00|1488.00|1607.04|471.75| +2451020|38644|6781|4419|1627710|954|49045|1|158|456|52|29.43|36.49|1.45|0.00|75.40|1530.36|1897.48|0.00|0.00|75.40|75.40|-1454.96| +2451020|38644|3758|4419|1627710|954|49045|1|1|456|2|33.04|64.09|29.48|0.00|58.96|66.08|128.18|4.71|0.00|58.96|63.67|-7.12| +2451020|38644|10610|4419|1627710|954|49045|1|248|456|97|37.37|49.32|17.75|0.00|1721.75|3624.89|4784.04|86.08|0.00|1721.75|1807.83|-1903.14| +2451020|38644|11926|4419|1627710|954|49045|1|109|456|83|36.41|36.41|31.67|0.00|2628.61|3022.03|3022.03|78.85|0.00|2628.61|2707.46|-393.42| +2451020|38644|4360|4419|1627710|954|49045|1|148|456|29|78.25|151.80|112.33|2378.02|3257.57|2269.25|4402.20|26.38|2378.02|879.55|905.93|-1389.70| +2451020|38644|1597|4419|1627710|954|49045|1|142|456|49|95.28|166.74|110.04|0.00|5391.96|4668.72|8170.26|323.51|0.00|5391.96|5715.47|723.24| +2451020|38644|3283|4419|1627710|954|49045|1|215|456|76|96.13|179.76|122.23|0.00|9289.48|7305.88|13661.76|836.05|0.00|9289.48|10125.53|1983.60| +2451020|38644|10414|4419|1627710|954|49045|1|2|456|73|81.46|92.04|54.30|0.00|3963.90|5946.58|6718.92|277.47|0.00|3963.90|4241.37|-1982.68| +2451020|38644|13126|4419|1627710|954|49045|1|195|456|11|53.31|76.23|27.44|0.00|301.84|586.41|838.53|3.01|0.00|301.84|304.85|-284.57| +2451057|64456|10015|43274|1905497|4073|3449|10|79|457|14|63.93|68.40|41.72|0.00|584.08|895.02|957.60|29.20|0.00|584.08|613.28|-310.94| +2451057|64456|11690|43274|1905497|4073|3449|10|134|457|78|36.43|66.66|30.66|0.00|2391.48|2841.54|5199.48|167.40|0.00|2391.48|2558.88|-450.06| +2451057|64456|14852|43274|1905497|4073|3449|10|137|457|54|82.38|88.97|73.84|0.00|3987.36|4448.52|4804.38|79.74|0.00|3987.36|4067.10|-461.16| +2451057|64456|1262|43274|1905497|4073|3449|10|190|457|37|59.29|86.56|32.89|0.00|1216.93|2193.73|3202.72|97.35|0.00|1216.93|1314.28|-976.80| +2451057|64456|15850|43274|1905497|4073|3449|10|231|457|59|8.62|9.13|0.73|0.00|43.07|508.58|538.67|1.72|0.00|43.07|44.79|-465.51| +2451057|64456|7639|43274|1905497|4073|3449|10|274|457|50|18.06|34.31|25.38|203.04|1269.00|903.00|1715.50|0.00|203.04|1065.96|1065.96|162.96| +2451057|64456|6061|43274|1905497|4073|3449|10|18|457|7|51.36|100.15|60.09|0.00|420.63|359.52|701.05|8.41|0.00|420.63|429.04|61.11| +2451057|64456|6790|43274|1905497|4073|3449|10|55|457|14|61.37|84.07|40.35|0.00|564.90|859.18|1176.98|16.94|0.00|564.90|581.84|-294.28| +2451057|64456|11431|43274|1905497|4073|3449|10|237|457|57|9.96|13.04|1.56|0.00|88.92|567.72|743.28|8.00|0.00|88.92|96.92|-478.80| +2451057|64456|12691|43274|1905497|4073|3449|10|199|457|67|10.82|17.42|9.05|0.00|606.35|724.94|1167.14|42.44|0.00|606.35|648.79|-118.59| +2451057|64456|17558|43274|1905497|4073|3449|10|34|457|27|73.78|120.26|14.43|0.00|389.61|1992.06|3247.02|27.27|0.00|389.61|416.88|-1602.45| +2451057|64456|13628|43274|1905497|4073|3449|10|177|457|58|43.55|63.14|35.35|1271.18|2050.30|2525.90|3662.12|54.53|1271.18|779.12|833.65|-1746.78| +||13519|43274|1905497|4073||||457||67.13||18.13|0.00|652.68||||0.00||704.89|| +2451470|52500|6698|11407|1765463|2719|5975|2|80|458|52|9.76|9.85|8.76|0.00|455.52|507.52|512.20|36.44|0.00|455.52|491.96|-52.00| +2451470|52500|640|11407|1765463|2719|5975|2|221|458|83|81.89|83.52|43.43|2451.18|3604.69|6796.87|6932.16|57.67|2451.18|1153.51|1211.18|-5643.36| +2451470|52500|15290|11407|1765463|2719|5975|2|144|458|85|6.79|8.01|7.68|0.00|652.80|577.15|680.85|39.16|0.00|652.80|691.96|75.65| +2451470|52500|9488|11407|1765463|2719|5975|2|120|458|58|47.32|52.05|7.28|0.00|422.24|2744.56|3018.90|0.00|0.00|422.24|422.24|-2322.32| +2451470|52500|9385|11407|1765463|2719|5975|2|283|458|90|14.15|20.51|6.56|0.00|590.40|1273.50|1845.90|29.52|0.00|590.40|619.92|-683.10| +2451470|52500|13447|11407|1765463|2719|5975|2|157|458|94|82.30|115.22|103.69|0.00|9746.86|7736.20|10830.68|292.40|0.00|9746.86|10039.26|2010.66| +2451470|52500|3136|11407|1765463|2719|5975|2|5|458|78|16.67|24.33|16.05|0.00|1251.90|1300.26|1897.74|87.63|0.00|1251.90|1339.53|-48.36| +2451470|52500|3289|11407|1765463|2719|5975|2|265|458|75|30.97|40.88|4.08|0.00|306.00|2322.75|3066.00|9.18|0.00|306.00|315.18|-2016.75| +2451746|69898|8678|1393|1204137|1281|41625|10|179|459|51|13.36|15.76|14.18|0.00|723.18|681.36|803.76|50.62|0.00|723.18|773.80|41.82| +2451746|69898|2141|1393|1204137|1281|41625|10|282|459|73|94.38|156.67|23.50|0.00|1715.50|6889.74|11436.91|68.62|0.00|1715.50|1784.12|-5174.24| +2451746|69898|17234|1393|1204137|1281|41625|10|116|459|73|2.02|3.73|2.08|0.00|151.84|147.46|272.29|12.14|0.00|151.84|163.98|4.38| +2451746|69898|6335|1393|1204137|1281|41625|10|267|459|5|76.49|97.90|59.71|0.00|298.55|382.45|489.50|26.86|0.00|298.55|325.41|-83.90| +2451746|69898|1669|1393|1204137|1281|41625|10|92|459|87|44.49|48.49|20.36|0.00|1771.32|3870.63|4218.63|35.42|0.00|1771.32|1806.74|-2099.31| +||3229||1204137|1281|41625|||459|53||16.73|10.53||||886.69|||22.33||| +2451746|69898|16625|1393|1204137|1281|41625|10|291|459|26|56.05|99.20|39.68|0.00|1031.68|1457.30|2579.20|30.95|0.00|1031.68|1062.63|-425.62| +2451746|69898|7706|1393|1204137|1281|41625|10|244|459|97|59.74|83.63|63.55|0.00|6164.35|5794.78|8112.11|369.86|0.00|6164.35|6534.21|369.57| +|69898|8720|1393|1204137|||10|37|459||||21.44|||3370.84|4381.81|30.23|||1037.91|-2363.16| +2451746|69898|5594|1393|1204137|1281|41625|10|190|459|90|46.84|66.51|36.58|98.76|3292.20|4215.60|5985.90|223.54|98.76|3193.44|3416.98|-1022.16| +2451746|69898|10037|1393|1204137|||10||459||55.31|||133.26||4977.90|||133.26|650.64|683.17|| +2451969|58548|123|17725|460235|5592|40502|8|194|460|79|78.05|109.27|2.18|0.00|172.22|6165.95|8632.33|6.88|0.00|172.22|179.10|-5993.73| +2451969|58548|5109|17725|460235|5592|40502|8|96|460|78|99.45|190.94|124.11|0.00|9680.58|7757.10|14893.32|0.00|0.00|9680.58|9680.58|1923.48| +2451969|58548|5893|17725|460235|5592|40502|8|275|460|76|65.41|90.91|77.27|0.00|5872.52|4971.16|6909.16|234.90|0.00|5872.52|6107.42|901.36| +2451969|58548|15901|17725|460235|5592|40502|8|119|460|74|13.18|23.59|3.06|0.00|226.44|975.32|1745.66|2.26|0.00|226.44|228.70|-748.88| +2451969|58548|4417|17725|460235|5592|40502|8|147|460|93|16.69|16.85|4.38|0.00|407.34|1552.17|1567.05|28.51|0.00|407.34|435.85|-1144.83| +2451969|58548|15737|17725|460235|5592|40502|8|31|460|6|43.23|52.30|51.77|0.00|310.62|259.38|313.80|18.63|0.00|310.62|329.25|51.24| +2451969|58548|11129|17725|460235|5592|40502|8|30|460|88|31.94|48.86|26.38|0.00|2321.44|2810.72|4299.68|69.64|0.00|2321.44|2391.08|-489.28| +2451969|58548|8093|17725|460235|5592|40502|8|148|460|37|56.68|87.28|41.89|0.00|1549.93|2097.16|3229.36|0.00|0.00|1549.93|1549.93|-547.23| +2452178|62548|17601|33183|939054|3772|33643|4|101|461|15|74.09|91.13|39.18|0.00|587.70|1111.35|1366.95|11.75|0.00|587.70|599.45|-523.65| +2452178|62548|12725|33183|939054|3772|33643|4|71|461|47|71.28|119.03|39.27|0.00|1845.69|3350.16|5594.41|0.00|0.00|1845.69|1845.69|-1504.47| +2452178|62548|2565|33183|939054|3772|33643|4|193|461|47|91.97|91.97|49.66|0.00|2334.02|4322.59|4322.59|23.34|0.00|2334.02|2357.36|-1988.57| +2452178|62548|6763|33183|939054|3772|33643|4|152|461|53|49.51|57.92|16.21|0.00|859.13|2624.03|3069.76|17.18|0.00|859.13|876.31|-1764.90| +||2035|33183|||33643|||461|75|||33.43||2507.25||3742.50|||||-225.00| +2452178|62548|1387|33183|939054|3772|33643|4|274|461|89|87.13|146.37|134.66|0.00|11984.74|7754.57|13026.93|719.08|0.00|11984.74|12703.82|4230.17| +2452178|62548|1821|33183|939054|3772|33643|4|249|461|9|12.14|22.33|16.97|0.00|152.73|109.26|200.97|0.00|0.00|152.73|152.73|43.47| +2452178|62548|12527|33183|939054|3772|33643|4|53|461|70|46.93|89.16|64.19|0.00|4493.30|3285.10|6241.20|314.53|0.00|4493.30|4807.83|1208.20| +2452178|62548|2315|33183|939054|3772|33643|4|280|461|3|85.37|88.78|27.52|0.00|82.56|256.11|266.34|5.77|0.00|82.56|88.33|-173.55| +2452178|62548|10017|33183|939054|3772|33643|4|196|461|74|14.60|24.67|15.29|0.00|1131.46|1080.40|1825.58|33.94|0.00|1131.46|1165.40|51.06| +2452178|62548|3347|33183|939054|3772|33643|4|207|461|71|13.10|24.49|7.59|355.66|538.89|930.10|1738.79|3.66|355.66|183.23|186.89|-746.87| +2452178|62548|977|33183|939054|3772|33643|4|165|461|95|10.59|13.66|3.96|0.00|376.20|1006.05|1297.70|30.09|0.00|376.20|406.29|-629.85| +2451035|38290|4766|71982|1821664|2899|10534|7|47|462|43|61.57|100.35|36.12|0.00|1553.16|2647.51|4315.05|0.00|0.00|1553.16|1553.16|-1094.35| +2451035|38290|16093|71982|1821664|2899|10534|7|179|462|43|97.32|124.56|28.64|0.00|1231.52|4184.76|5356.08|73.89|0.00|1231.52|1305.41|-2953.24| +2451035|38290|7051|71982|1821664|2899|10534|7|218|462|31|18.19|25.46|20.11|0.00|623.41|563.89|789.26|6.23|0.00|623.41|629.64|59.52| +2451035|38290|12146|71982|1821664|2899|10534|7|168|462|66|91.37|120.60|78.39|1810.80|5173.74|6030.42|7959.60|67.25|1810.80|3362.94|3430.19|-2667.48| +2451035||13357|71982||2899|10534|7|277|462|||8.38||0.00|110.22||||0.00|||-192.06| +2451035|38290|9280|71982|1821664|2899|10534|7|258|462|51|50.50|92.92|26.01|0.00|1326.51|2575.50|4738.92|39.79|0.00|1326.51|1366.30|-1248.99| +2451035|38290|4162|71982|1821664|2899|10534|7|73|462|92|74.35|117.47|81.05|0.00|7456.60|6840.20|10807.24|521.96|0.00|7456.60|7978.56|616.40| +||16184|71982||||||462|||||0.00|||||0.00|||-1.65| +2451035|38290|13828|71982|1821664|2899|10534|7|216|462|38|25.87|41.65|17.07|0.00|648.66|983.06|1582.70|6.48|0.00|648.66|655.14|-334.40| +2451035|38290|11900|71982|1821664|2899|10534|7|56|462|48|79.32|111.04|65.51|0.00|3144.48|3807.36|5329.92|62.88|0.00|3144.48|3207.36|-662.88| +2451035|38290|11539|71982|1821664|2899|10534|7|287|462|100|13.93|27.30|3.27|0.00|327.00|1393.00|2730.00|26.16|0.00|327.00|353.16|-1066.00| +2451035|38290|12304|71982|1821664|2899|10534|7|44|462|23|22.48|33.49|0.00|0.00|0.00|517.04|770.27|0.00|0.00|0.00|0.00|-517.04| +2451035|38290|2404||||10534|7||462|99||158.04|107.46|||8597.16|15645.96|||10638.54||| +2452374|61328|4416|9180|748154|4375|7349|4|48|463|63|6.39|7.73|6.72|0.00|423.36|402.57|486.99|8.46|0.00|423.36|431.82|20.79| +2452374|61328|3885|9180|748154|4375|7349|4|100|463|99|99.25|129.02|119.98|0.00|11878.02|9825.75|12772.98|1069.02|0.00|11878.02|12947.04|2052.27| +2452374|61328|1617|9180|748154|4375|7349|4|237|463|35|79.70|106.00|47.70|0.00|1669.50|2789.50|3710.00|16.69|0.00|1669.50|1686.19|-1120.00| +2452374|61328|5439|9180|748154|4375|7349|4|40|463|54|68.29|94.24|74.44|0.00|4019.76|3687.66|5088.96|321.58|0.00|4019.76|4341.34|332.10| +2452374|61328|14532|9180|748154|4375|7349|4|251|463|28|91.64|136.54|42.32|0.00|1184.96|2565.92|3823.12|11.84|0.00|1184.96|1196.80|-1380.96| +2452374|61328|3645|9180|748154|4375|7349|4|195|463|33|37.67|59.89|3.59|0.00|118.47|1243.11|1976.37|9.47|0.00|118.47|127.94|-1124.64| +2452374|61328|937|9180|748154|4375|7349|4|290|463|91|72.11|98.06|95.11|0.00|8655.01|6562.01|8923.46|0.00|0.00|8655.01|8655.01|2093.00| +2452374|61328|11023|9180|748154|4375|7349|4|86|463|94|55.66|79.59|66.85|1193.94|6283.90|5232.04|7481.46|50.89|1193.94|5089.96|5140.85|-142.08| +2452374|61328|13281|9180|748154|4375|7349|4|184|463|40|61.09|80.63|36.28|0.00|1451.20|2443.60|3225.20|87.07|0.00|1451.20|1538.27|-992.40| +2452374||16104|9180||||4|88|463|||||||129.94|248.18|6.03||||| +2452374|61328|4500|9180|748154|4375|7349|4|265|463|55|52.41|91.19|51.97|2372.43|2858.35|2882.55|5015.45|14.57|2372.43|485.92|500.49|-2396.63| +2452374|61328|14491|9180|748154|4375|7349|4|241|463|66|95.53|95.53|32.48|0.00|2143.68|6304.98|6304.98|0.00|0.00|2143.68|2143.68|-4161.30| +2452374|61328|8797|9180|748154|4375|7349|4|142|463|16|87.55|168.09|107.57|1187.57|1721.12|1400.80|2689.44|10.67|1187.57|533.55|544.22|-867.25| +2452374|61328|1819|9180|748154|4375|7349|4|61|463|58|25.58|41.69|38.77|0.00|2248.66|1483.64|2418.02|179.89|0.00|2248.66|2428.55|765.02| +2452374|61328|11121|9180|748154|4375|7349|4|32|463|80|51.59|83.57|83.57|0.00|6685.60|4127.20|6685.60|267.42|0.00|6685.60|6953.02|2558.40| +2451142|45936|8323|24604|1471070|736|42002|4|43|464|54|99.22|124.02|96.73|4910.01|5223.42|5357.88|6697.08|6.26|4910.01|313.41|319.67|-5044.47| +2451142||7490||1471070||42002|4|162|464|38|||91.34|0.00||2606.04|||0.00||3470.92|864.88| +2451142|45936|5926|24604|1471070|736|42002|4|194|464|94|6.03|7.95|3.25|0.00|305.50|566.82|747.30|6.11|0.00|305.50|311.61|-261.32| +2451142|45936|15320|24604|1471070|736|42002|4|142|464|35|89.42|128.76|29.61|580.35|1036.35|3129.70|4506.60|18.24|580.35|456.00|474.24|-2673.70| +2451142|45936|9496|24604|1471070|736|42002|4|296|464|99|36.33|45.04|24.32|168.53|2407.68|3596.67|4458.96|89.56|168.53|2239.15|2328.71|-1357.52| +2451142|45936|4126|24604|1471070|736|42002|4|134|464|56|69.97|128.04|33.29|0.00|1864.24|3918.32|7170.24|37.28|0.00|1864.24|1901.52|-2054.08| +||1256||1471070|736|42002|4|250|464|||||0.00|1128.63|819.40||0.00|0.00|1128.63||| +2451142|45936|17071|24604|1471070|736|42002|4|251|464|85|60.26|90.39|55.13|421.74|4686.05|5122.10|7683.15|42.64|421.74|4264.31|4306.95|-857.79| +2451527|46769|13436|31625|1680639|1300|24361|8|139|465|15|53.15|63.78|47.83|265.45|717.45|797.25|956.70|36.16|265.45|452.00|488.16|-345.25| +2451527|46769|9416|31625|1680639|1300|24361|8|207|465|15|92.15|167.71|78.82|0.00|1182.30|1382.25|2515.65|106.40|0.00|1182.30|1288.70|-199.95| +2451527|46769|1982|31625|1680639|1300|24361|8|30|465|74|29.18|52.81|38.02|0.00|2813.48|2159.32|3907.94|196.94|0.00|2813.48|3010.42|654.16| +2451527|46769|6796|31625|1680639|1300|24361|8|195|465|33|99.92|145.88|128.37|0.00|4236.21|3297.36|4814.04|254.17|0.00|4236.21|4490.38|938.85| +2451527|46769|14659|31625|1680639|1300|24361|8|43|465|87|6.44|8.24|7.74|0.00|673.38|560.28|716.88|26.93|0.00|673.38|700.31|113.10| +||12685||||24361||28|465||||||839.26|4581.42||2.51||||-4531.06| +2451527|46769|3722|31625|1680639|1300|24361|8|43|465|90|14.85|26.13|8.36|0.00|752.40|1336.50|2351.70|45.14|0.00|752.40|797.54|-584.10| +2451527|46769|11056|31625|1680639|1300|24361|8|132|465|97|4.56|5.28|5.17|65.19|501.49|442.32|512.16|30.54|65.19|436.30|466.84|-6.02| +2451527|46769|5386|31625|1680639|1300|24361|8|49|465|6|67.11|71.13|0.71|0.00|4.26|402.66|426.78|0.34|0.00|4.26|4.60|-398.40| +2451527|46769|14986|31625|1680639|1300|24361|8|149|465|77|45.53|69.66|67.57|2601.44|5202.89|3505.81|5363.82|52.02|2601.44|2601.45|2653.47|-904.36| +2451527|46769|2878|31625|1680639|1300|24361|8|130|465|58|68.91|79.93|53.55|0.00|3105.90|3996.78|4635.94|217.41|0.00|3105.90|3323.31|-890.88| +2451546|65844|6965|89778|1813647|716|20185|8|67|466|63|58.73|80.46|78.04|0.00|4916.52|3699.99|5068.98|98.33|0.00|4916.52|5014.85|1216.53| +2451546|65844|17209|89778|1813647|716|20185|8|287|466|76|80.78|109.05|85.05|0.00|6463.80|6139.28|8287.80|64.63|0.00|6463.80|6528.43|324.52| +2451546|65844|452|89778|1813647|716|20185|8|279|466|83|92.01|176.65|128.95|963.25|10702.85|7636.83|14661.95|681.77|963.25|9739.60|10421.37|2102.77| +2451546||4267|89778|1813647|716|||112|466||4.43|4.43||||407.56||||142.60||| +2451546|65844|7823|89778|1813647|716|20185|8|191|466|58|43.72|55.52|53.29|0.00|3090.82|2535.76|3220.16|247.26|0.00|3090.82|3338.08|555.06| +2451546|65844|323|89778|1813647|716|20185|8|76|466|89|24.45|25.67|18.99|1233.78|1690.11|2176.05|2284.63|31.94|1233.78|456.33|488.27|-1719.72| +2451546|65844|11137|89778|1813647|716|20185|8|13|466|57|92.96|168.25|131.23|0.00|7480.11|5298.72|9590.25|0.00|0.00|7480.11|7480.11|2181.39| +2451546|65844|13793|89778|1813647|716|20185|8|235|466|50|83.82|88.01|66.88|0.00|3344.00|4191.00|4400.50|167.20|0.00|3344.00|3511.20|-847.00| +2451546|65844|13489|89778|1813647|716|20185|8|150|466|96|18.23|32.81|23.29|514.24|2235.84|1750.08|3149.76|103.29|514.24|1721.60|1824.89|-28.48| +2451546|65844|15449|89778|1813647|716|20185|8|114|466|18|65.77|82.87|70.43|0.00|1267.74|1183.86|1491.66|114.09|0.00|1267.74|1381.83|83.88| +2451546|65844|6137|89778|1813647|716|20185|8|86|466|12|14.91|24.30|20.89|0.00|250.68|178.92|291.60|17.54|0.00|250.68|268.22|71.76| +2451876|36146|8021|23005|964907|5209|27911|8|141|467|60|5.27|6.27|2.63|0.00|157.80|316.20|376.20|0.00|0.00|157.80|157.80|-158.40| +|36146|17021|23005|||27911||261|467|22|88.56|154.98|122.43|53.86|2693.46|1948.32|||53.86|2639.60||691.28| +2451876|36146|1832|23005|964907|5209|27911|8|46|467|29|33.86|54.17|2.70|20.35|78.30|981.94|1570.93|1.15|20.35|57.95|59.10|-923.99| +2451876|36146|14834|23005|964907|5209|27911|8|265|467|68|72.21|119.86|62.32|0.00|4237.76|4910.28|8150.48|84.75|0.00|4237.76|4322.51|-672.52| +2451876|36146|2735|23005|964907|5209|27911|8|111|467|28|99.35|131.14|22.29|230.92|624.12|2781.80|3671.92|0.00|230.92|393.20|393.20|-2388.60| +2451876|36146|1466|23005|964907|5209|27911|8|72|467|10|40.07|40.47|16.18|0.00|161.80|400.70|404.70|3.23|0.00|161.80|165.03|-238.90| +2451876|36146|5864|23005|964907|5209|27911|8|85|467|100|11.72|14.06|0.14|0.00|14.00|1172.00|1406.00|0.98|0.00|14.00|14.98|-1158.00| +2451876|36146|15319|23005|964907|5209|27911|8|291|467|18|66.94|108.44|76.99|0.00|1385.82|1204.92|1951.92|69.29|0.00|1385.82|1455.11|180.90| +2451602|63717|8657|77376|1470593|6210|13427|2|209|468|48|14.59|28.59|11.72|0.00|562.56|700.32|1372.32|45.00|0.00|562.56|607.56|-137.76| +2451602|63717|11911|77376|1470593|6210|13427|2|155|468|47|41.75|43.42|23.44|0.00|1101.68|1962.25|2040.74|22.03|0.00|1101.68|1123.71|-860.57| +2451602|63717|857|77376|1470593|6210|13427|2|191|468|69|80.45|90.90|65.44|0.00|4515.36|5551.05|6272.10|45.15|0.00|4515.36|4560.51|-1035.69| +2451602|63717|4592|77376|1470593|6210|13427|2|171|468|27|27.67|42.88|8.14|72.52|219.78|747.09|1157.76|2.94|72.52|147.26|150.20|-599.83| +2451602|63717|5321||||13427|2|29|468||53.97||48.29||724.35||1270.95|||492.56|502.41|| +2451602|63717|11161|77376|1470593|6210|13427|2|140|468|7|28.16|30.13|19.28|0.00|134.96|197.12|210.91|5.39|0.00|134.96|140.35|-62.16| +2451602|63717|6860|77376|1470593|6210|13427|2|246|468|14|62.68|105.30|43.17|0.00|604.38|877.52|1474.20|42.30|0.00|604.38|646.68|-273.14| +2451602|63717|5303|77376|1470593|6210|13427|2|160|468|18|20.96|27.45|2.47|0.00|44.46|377.28|494.10|3.11|0.00|44.46|47.57|-332.82| +2452599|53510|4975|96265|501091|764|23990|10|47|469|43|81.70|152.77|39.72|0.00|1707.96|3513.10|6569.11|136.63|0.00|1707.96|1844.59|-1805.14| +2452599|53510|4333|96265|501091|764|23990|10|270|469|51|45.14|52.81|45.94|0.00|2342.94|2302.14|2693.31|140.57|0.00|2342.94|2483.51|40.80| +2452599|53510|17868|96265|501091|764|23990|10|191|469|21|84.49|103.07|5.15|0.00|108.15|1774.29|2164.47|4.32|0.00|108.15|112.47|-1666.14| +2452599|53510|2461|96265|501091|764|23990|10|42|469|40|69.45|130.56|11.75|0.00|470.00|2778.00|5222.40|4.70|0.00|470.00|474.70|-2308.00| +2452599|53510|4867|96265|501091|764|23990|10|152|469|61|43.87|77.64|51.24|0.00|3125.64|2676.07|4736.04|93.76|0.00|3125.64|3219.40|449.57| +2452599|53510|8031|96265|501091|764|23990|10|286|469|10|55.99|71.66|45.14|0.00|451.40|559.90|716.60|4.51|0.00|451.40|455.91|-108.50| +2452599|53510|7867|96265|501091|764|23990|10|193|469|59|27.09|31.96|18.85|0.00|1112.15|1598.31|1885.64|77.85|0.00|1112.15|1190.00|-486.16| +2452599|53510|469|96265|501091|764|23990|10|105|469|27|26.84|48.58|48.58|0.00|1311.66|724.68|1311.66|26.23|0.00|1311.66|1337.89|586.98| +2452599|53510|8359|96265|501091|764|23990|10|217|469|15|11.45|14.88|4.76|0.00|71.40|171.75|223.20|0.00|0.00|71.40|71.40|-100.35| +2452599|53510|849|96265|501091|764|23990|10|263|469|11|5.73|10.25|4.51|36.21|49.61|63.03|112.75|0.67|36.21|13.40|14.07|-49.63| +2452599||16555|96265||764|23990|10|23|469|93||||||2156.67||75.88|||1024.48|-1208.07| +2452599|53510|9693|96265|501091|764|23990|10|136|469|40|7.83|9.63|5.68|0.00|227.20|313.20|385.20|2.27|0.00|227.20|229.47|-86.00| +2451137|50436|5630|44999|349028|610|15439|1|116|470|74|38.02|50.94|43.80|0.00|3241.20|2813.48|3769.56|97.23|0.00|3241.20|3338.43|427.72| +2451137|50436|17750|44999|349028|610|15439|1|250|470|53|97.02|106.72|2.13|86.92|112.89|5142.06|5656.16|2.07|86.92|25.97|28.04|-5116.09| +2451137|50436|4754|44999|349028|610|15439|1|250|470|8|4.17|4.79|2.25|0.00|18.00|33.36|38.32|1.62|0.00|18.00|19.62|-15.36| +|50436|6607|||610|15439|1|74|470||90.95|||||2455.65|2701.08|156.92||2241.81|2398.73|| +2451137|50436|7046|44999|349028|610|15439|1|84|470|15|24.24|39.99|20.39|103.98|305.85|363.60|599.85|4.03|103.98|201.87|205.90|-161.73| +2451137|50436|4972|44999|349028|610|15439|1|257|470|8|9.42|13.37|6.95|0.00|55.60|75.36|106.96|3.33|0.00|55.60|58.93|-19.76| +2451137|50436|1472|44999|||15439||175|470|||30.13|22.29|0.00||256.44|||0.00||278.17|11.04| +2451137|50436|11906|44999|349028|610|15439|1|231|470|15|75.00|107.25|93.30|0.00|1399.50|1125.00|1608.75|55.98|0.00|1399.50|1455.48|274.50| +2451137|50436|17570|44999|349028|610|15439|1|219|470|38|76.66|150.25|58.59|155.84|2226.42|2913.08|5709.50|165.64|155.84|2070.58|2236.22|-842.50| +2451137|50436|14677|44999|349028|610|15439|1|156|470|41|35.20|51.39|23.12|0.00|947.92|1443.20|2106.99|75.83|0.00|947.92|1023.75|-495.28| +2451137|50436|11395|44999|349028|610|15439|1|178|470|47|82.44|104.69|9.42|0.00|442.74|3874.68|4920.43|39.84|0.00|442.74|482.58|-3431.94| +2451137|50436|14510|44999|349028|610|15439|1|135|470|60|86.90|166.84|135.14|0.00|8108.40|5214.00|10010.40|567.58|0.00|8108.40|8675.98|2894.40| +2451137|50436|13508|44999|349028|610|15439|1|212|470|82|90.36|142.76|14.27|0.00|1170.14|7409.52|11706.32|23.40|0.00|1170.14|1193.54|-6239.38| +2451137|50436|9998|44999|349028|610|15439|1|105|470|70|29.21|48.78|5.85|0.00|409.50|2044.70|3414.60|4.09|0.00|409.50|413.59|-1635.20| +2451519|66249|5656|82172|1843830|4335|9993|1|101|471|69|20.76|26.98|23.74|0.00|1638.06|1432.44|1861.62|131.04|0.00|1638.06|1769.10|205.62| +2451519|66249|8918|82172|1843830|4335|9993|1|45|471|76|78.94|104.20|87.52|0.00|6651.52|5999.44|7919.20|598.63|0.00|6651.52|7250.15|652.08| +2451519|66249|7166|82172|1843830|4335|9993|1|77|471|74|29.91|48.15|12.51|0.00|925.74|2213.34|3563.10|37.02|0.00|925.74|962.76|-1287.60| +2451519|66249|11107|82172|1843830|4335|9993|1|14|471|71|53.98|54.51|19.62|0.00|1393.02|3832.58|3870.21|13.93|0.00|1393.02|1406.95|-2439.56| +2451519|66249|7663|82172|1843830|4335|9993|1|55|471|80|5.95|9.10|3.36|0.00|268.80|476.00|728.00|8.06|0.00|268.80|276.86|-207.20| +2451519|66249|13574|82172||||1||471||23.16|44.69|18.32|0.00|||||0.00|256.48||| +2451519|66249|10480|82172|1843830|4335|9993|1|62|471|26|56.99|66.67|30.00|0.00|780.00|1481.74|1733.42|0.00|0.00|780.00|780.00|-701.74| +2451519|66249|10849|82172|1843830|4335|9993|1|123|471|10|15.75|30.71|19.34|0.00|193.40|157.50|307.10|9.67|0.00|193.40|203.07|35.90| +2451519|66249|16852|82172|1843830|4335|9993|1|67|471|63|1.16|2.16|2.05|0.00|129.15|73.08|136.08|9.04|0.00|129.15|138.19|56.07| +2451519|66249|13142|82172|1843830|4335|9993|1|190|471|76|28.97|52.72|17.39|0.00|1321.64|2201.72|4006.72|0.00|0.00|1321.64|1321.64|-880.08| +2451519|66249|302|82172|1843830|4335|||129|471|15|56.87|||||853.05||10.36||||| +2451519|66249|4454|82172|1843830|4335|9993|1|47|471|22|37.54|45.79|1.37|25.61|30.14|825.88|1007.38|0.36|25.61|4.53|4.89|-821.35| +2451519|66249|7045|82172|1843830|4335|9993|1|36|471|94|94.61|177.86|85.37|2006.19|8024.78|8893.34|16718.84|240.74|2006.19|6018.59|6259.33|-2874.75| +2451436|64060|5800|43160|1418034|908|10182|10|111|472|96|68.21|108.45|15.18|0.00|1457.28|6548.16|10411.20|131.15|0.00|1457.28|1588.43|-5090.88| +2451436|64060|5062|43160|1418034|908|10182|10|215|472|92|56.57|78.06|48.39|2760.16|4451.88|5204.44|7181.52|67.66|2760.16|1691.72|1759.38|-3512.72| +2451436|64060|8672|43160|1418034|908|10182|10|266|472|25|72.83|91.76|45.88|0.00|1147.00|1820.75|2294.00|68.82|0.00|1147.00|1215.82|-673.75| +2451436|64060|6470|43160|1418034|908|10182|10|233|472|55|7.97|8.20|8.11|0.00|446.05|438.35|451.00|17.84|0.00|446.05|463.89|7.70| +2451436|64060|10142|43160|1418034|908|10182|10|261|472|2|55.46|68.77|26.13|0.00|52.26|110.92|137.54|2.61|0.00|52.26|54.87|-58.66| +2451436|64060|3574|43160|1418034|908|10182|10|124|472|99|32.35|48.20|5.78|0.00|572.22|3202.65|4771.80|11.44|0.00|572.22|583.66|-2630.43| +2451436|64060|12506|43160|1418034|908|10182|10|121|472|55|4.05|7.33|4.03|0.00|221.65|222.75|403.15|8.86|0.00|221.65|230.51|-1.10| +2451436|64060|15856|43160|1418034|908|10182|10|227|472|80|96.36|164.77|95.56|0.00|7644.80|7708.80|13181.60|305.79|0.00|7644.80|7950.59|-64.00| +2452640|36381|1302|64250|1489142|81|5373|2|12|473|58|24.84|45.95|33.54|0.00|1945.32|1440.72|2665.10|77.81|0.00|1945.32|2023.13|504.60| +2452640|36381|14155|64250|1489142|81|5373|2|256|473|47|74.90|117.59|3.52|0.00|165.44|3520.30|5526.73|6.61|0.00|165.44|172.05|-3354.86| +2452640|36381|4464|64250|1489142|81|5373|2|195|473|88|28.23|31.89|0.31|0.00|27.28|2484.24|2806.32|0.54|0.00|27.28|27.82|-2456.96| +2452640|36381|4201|64250|1489142|81|5373|2|21|473|94|59.65|113.33|48.73|3710.30|4580.62|5607.10|10653.02|0.00|3710.30|870.32|870.32|-4736.78| +2452640|36381|4705|64250|1489142|81|5373|2|85|473|46|81.76|112.82|1.12|0.00|51.52|3760.96|5189.72|3.09|0.00|51.52|54.61|-3709.44| +2452640|36381|6909|64250|1489142|81|5373|2|43|473|46|41.56|63.17|51.16|2259.22|2353.36|1911.76|2905.82|6.58|2259.22|94.14|100.72|-1817.62| +2452640|36381|841|64250|1489142|81|5373|2|119|473|34|79.78|110.89|87.60|0.00|2978.40|2712.52|3770.26|119.13|0.00|2978.40|3097.53|265.88| +2452640|36381|5142|64250|1489142|81|5373|2|255|473|23|94.90|108.18|97.36|0.00|2239.28|2182.70|2488.14|179.14|0.00|2239.28|2418.42|56.58| +2452640|36381|8901|||||||473|||91.68|9.16|0.00|732.80|3742.40|7334.40||0.00|732.80|791.42|| +2452640|36381|10260|64250|1489142|81|5373|2|212|473|7|49.66|65.05|44.88|0.00|314.16|347.62|455.35|15.70|0.00|314.16|329.86|-33.46| +2452640|36381|17169|64250|1489142|81|5373|2|142|473|38|84.77|144.95|140.60|0.00|5342.80|3221.26|5508.10|160.28|0.00|5342.80|5503.08|2121.54| +2452640|36381|336|64250|1489142|81|5373|2|236|473|20|59.99|108.58|19.54|0.00|390.80|1199.80|2171.60|19.54|0.00|390.80|410.34|-809.00| +2452640|36381|9325|64250|1489142|81|5373|2|33|473|66|39.73|42.90|35.17|0.00|2321.22|2622.18|2831.40|92.84|0.00|2321.22|2414.06|-300.96| +2452640|36381|9768|64250|1489142|81|5373|2|120|473|63|87.77|100.05|80.04|4689.54|5042.52|5529.51|6303.15|31.76|4689.54|352.98|384.74|-5176.53| +|36381|12513|64250|1489142||||59|473|42||37.94|36.04|0.00|1513.68||1593.48|121.09|0.00|1513.68|1634.77|| +2452640|36381|1717|64250|1489142|81|5373|2|100|473|97|2.68|3.24|1.65|0.00|160.05|259.96|314.28|1.60|0.00|160.05|161.65|-99.91| +2452185|52541|5595|26197|98242|1134|3579|1|23|474|19|19.43|24.67|19.24|14.62|365.56|369.17|468.73|17.54|14.62|350.94|368.48|-18.23| +2452185|52541|10037|26197|98242|1134|3579|1|102|474|5|39.35|74.37|26.02|93.67|130.10|196.75|371.85|2.55|93.67|36.43|38.98|-160.32| +||5421|26197|98242|1134|||162|474|||39.56||0.00|90.06|1138.86||6.30|0.00|90.06||| +2452185|52541|10989|26197|98242|1134|3579|1|67|474|30|80.28|128.44|77.06|0.00|2311.80|2408.40|3853.20|23.11|0.00|2311.80|2334.91|-96.60| +2452185|52541|6965|26197|98242|1134|3579|1|251|474|34|16.77|18.95|10.42|0.00|354.28|570.18|644.30|28.34|0.00|354.28|382.62|-215.90| +2452185|52541|17209|26197|98242|1134|3579|1|54|474|24|5.91|10.81|9.40|20.30|225.60|141.84|259.44|18.47|20.30|205.30|223.77|63.46| +2452185|52541|453|26197|98242|1134|3579|1|4|474|84|22.07|28.91|23.99|0.00|2015.16|1853.88|2428.44|60.45|0.00|2015.16|2075.61|161.28| +2452185|52541|4267|26197|98242|1134|3579|1|247|474|57|74.43|146.62|54.24|0.00|3091.68|4242.51|8357.34|92.75|0.00|3091.68|3184.43|-1150.83| +2452185|52541|7823|26197|98242|1134|3579|1|214|474|84|6.10|11.04|10.92|0.00|917.28|512.40|927.36|55.03|0.00|917.28|972.31|404.88| +2452185|52541|323|26197|98242|1134|3579|1|208|474|73|61.73|112.34|46.05|0.00|3361.65|4506.29|8200.82|67.23|0.00|3361.65|3428.88|-1144.64| +2452185|52541|11137|26197|98242|1134|3579|1|106|474|13|13.27|15.92|7.32|0.00|95.16|172.51|206.96|7.61|0.00|95.16|102.77|-77.35| +2452185|52541|13793|26197|98242|1134|3579|1|156|474|11|72.17|132.07|38.30|181.15|421.30|793.87|1452.77|14.40|181.15|240.15|254.55|-553.72| +2452616|70504|8533|99711|626192|1120|31248|8|38|475|98|29.95|56.00|3.36|0.00|329.28|2935.10|5488.00|16.46|0.00|329.28|345.74|-2605.82| +2452616|70504|8929|99711|626192|1120|31248|8|231|475|66|91.03|124.71|34.91|0.00|2304.06|6007.98|8230.86|0.00|0.00|2304.06|2304.06|-3703.92| +|70504|2335|99711||1120||8||475|26|23.83|40.74||0.00|412.88||1059.24||0.00||417.00|| +2452616|70504|3349|99711|626192|1120|31248|8|88|475|70|57.86|87.94|58.91|0.00|4123.70|4050.20|6155.80|247.42|0.00|4123.70|4371.12|73.50| +2452616|70504|1629|99711|626192|1120|31248|8|204|475|39|63.80|125.68|115.62|496.00|4509.18|2488.20|4901.52|321.05|496.00|4013.18|4334.23|1524.98| +2452616|70504|4122|99711|626192|1120|31248|8|244|475|84|55.27|55.27|35.37|0.00|2971.08|4642.68|4642.68|0.00|0.00|2971.08|2971.08|-1671.60| +2452616|70504|4261|99711|626192|1120|31248|8|215|475|98|33.18|57.40|6.31|451.41|618.38|3251.64|5625.20|8.34|451.41|166.97|175.31|-3084.67| +2452616|70504|15474|99711|626192|1120|31248|8|198|475|27|50.32|94.60|46.35|0.00|1251.45|1358.64|2554.20|62.57|0.00|1251.45|1314.02|-107.19| +2452616|70504|2821|99711|626192|1120|31248|8|267|475|37|92.53|175.80|151.18|0.00|5593.66|3423.61|6504.60|111.87|0.00|5593.66|5705.53|2170.05| +2452616|70504|17803|99711|626192|1120|31248|8|51|475|44|75.13|139.74|20.96|0.00|922.24|3305.72|6148.56|36.88|0.00|922.24|959.12|-2383.48| +2451771|53891|8765|77281|610555|2229|1503|4|156|476|84|70.74|82.76|2.48|0.00|208.32|5942.16|6951.84|14.58|0.00|208.32|222.90|-5733.84| +2451771|53891|764|77281|610555|2229|1503|4|15|476|36|19.29|25.26|14.90|0.00|536.40|694.44|909.36|16.09|0.00|536.40|552.49|-158.04| +2451771|53891|3293|77281|610555|2229|1503|4|94|476|3|27.69|33.78|22.97|68.22|68.91|83.07|101.34|0.00|68.22|0.69|0.69|-82.38| +2451771|53891|11263|77281|610555|2229|1503|4|122|476|63|60.35|66.38|41.15|0.00|2592.45|3802.05|4181.94|207.39|0.00|2592.45|2799.84|-1209.60| +2451771|53891|10979|77281|610555|2229|1503|4|220|476|7|96.97|165.81|124.35|478.74|870.45|678.79|1160.67|19.58|478.74|391.71|411.29|-287.08| +2451771|53891|15277|77281|610555|2229|1503|4|3|476|85|33.63|44.72|8.94|0.00|759.90|2858.55|3801.20|45.59|0.00|759.90|805.49|-2098.65| +2451771|53891|16225|77281|610555|2229|1503|4|116|476|22|17.18|32.47|3.89|0.00|85.58|377.96|714.34|1.71|0.00|85.58|87.29|-292.38| +2451771|53891|7579|77281|610555|2229|1503|4|40|476|89|62.27|84.06|54.63|0.00|4862.07|5542.03|7481.34|437.58|0.00|4862.07|5299.65|-679.96| +2451771|53891|12296|77281|610555|2229|1503|4|211|476|61|58.60|100.79|45.35|0.00|2766.35|3574.60|6148.19|110.65|0.00|2766.35|2877.00|-808.25| +2451771|53891|6599|77281|610555|2229|1503|4|294|476|85|30.14|52.44|44.04|0.00|3743.40|2561.90|4457.40|0.00|0.00|3743.40|3743.40|1181.50| +2451927|63038|14123|56618|32110|2802|36294|2|199|477|24|89.76|93.35|64.41|0.00|1545.84|2154.24|2240.40|46.37|0.00|1545.84|1592.21|-608.40| +2451927|63038|201|56618|32110|2802|36294|2|109|477|46|99.30|112.20|2.24|0.00|103.04|4567.80|5161.20|5.15|0.00|103.04|108.19|-4464.76| +2451927|63038|9711|56618|32110|2802|36294|2|258|477|38|65.71|108.42|87.82|0.00|3337.16|2496.98|4119.96|66.74|0.00|3337.16|3403.90|840.18| +2451927|63038|6221|56618|32110|2802|36294|2|128|477|24|67.96|79.51|30.21|0.00|725.04|1631.04|1908.24|7.25|0.00|725.04|732.29|-906.00| +2451927|63038|7785|56618|32110|2802|36294|2|275|477|59|62.78|124.30|39.77|0.00|2346.43|3704.02|7333.70|70.39|0.00|2346.43|2416.82|-1357.59| +2451927|63038|10441|56618|32110|2802|36294|2|8|477|68|81.99|150.04|124.53|0.00|8468.04|5575.32|10202.72|84.68|0.00|8468.04|8552.72|2892.72| +2451927|63038|2165|56618|32110|2802|36294|2|7|477|93|61.76|95.72|54.56|0.00|5074.08|5743.68|8901.96|50.74|0.00|5074.08|5124.82|-669.60| +2451927|63038|757|56618|32110|2802|36294|2|15|477|42|87.69|97.33|56.45|0.00|2370.90|3682.98|4087.86|0.00|0.00|2370.90|2370.90|-1312.08| +|48080|16909||||17155||267|478|||155.97|146.61||7916.94|||0.00||||-5251.71| +2451850|48080|11707|36676|726420|3011|17155|2|110|478|99|48.47|76.09|9.13|0.00|903.87|4798.53|7532.91|63.27|0.00|903.87|967.14|-3894.66| +2451850|48080|13448|36676|726420|3011|17155|2|153|478|61|30.27|38.44|22.29|489.48|1359.69|1846.47|2344.84|60.91|489.48|870.21|931.12|-976.26| +2451850|48080|16837|36676|726420|3011|17155|2|148|478|34|70.57|128.43|59.07|0.00|2008.38|2399.38|4366.62|120.50|0.00|2008.38|2128.88|-391.00| +2451850|48080|7303|36676|726420|3011|17155|2|212|478|49|36.94|55.77|15.05|0.00|737.45|1810.06|2732.73|66.37|0.00|737.45|803.82|-1072.61| +2451850|48080|8449|36676|726420|3011|17155|2|75|478|15|39.69|67.86|30.53|0.00|457.95|595.35|1017.90|36.63|0.00|457.95|494.58|-137.40| +|48080|16429||||17155|||478|21|18.58|34.18|25.97||545.37||717.78|49.08|||594.45|155.19| +2451850|48080|5329|36676|726420|3011|17155|2|201|478|15|22.41|38.09|7.61|0.00|114.15|336.15|571.35|1.14|0.00|114.15|115.29|-222.00| +2451850|48080|16550|36676|726420|3011|17155|2|23|478|61|64.18|87.28|13.09|359.32|798.49|3914.98|5324.08|21.95|359.32|439.17|461.12|-3475.81| +2451191|39484|11492|12205|831963|5674|32632|10|297|479|52|1.37|1.80|0.77|0.00|40.04|71.24|93.60|0.00|0.00|40.04|40.04|-31.20| +2451191|39484|3988|12205|831963|5674|32632|10|16|479|1|14.36|21.10|17.72|0.00|17.72|14.36|21.10|0.17|0.00|17.72|17.89|3.36| +2451191|39484|13556|12205|831963|5674|32632|10|109|479|54|29.16|36.15|5.42|0.00|292.68|1574.64|1952.10|26.34|0.00|292.68|319.02|-1281.96| +2451191|39484|3067|12205|831963|5674|32632|10|46|479|56|96.36|119.48|75.27|0.00|4215.12|5396.16|6690.88|0.00|0.00|4215.12|4215.12|-1181.04| +2451191|39484|8350|12205|831963|5674|32632|10|103|479|29|74.08|90.37|60.54|0.00|1755.66|2148.32|2620.73|122.89|0.00|1755.66|1878.55|-392.66| +2451191|39484|2455|12205|831963|5674|32632|10|265|479|39|26.06|32.31|25.20|0.00|982.80|1016.34|1260.09|0.00|0.00|982.80|982.80|-33.54| +2451191|39484|13274|12205|831963|5674|32632|10|34|479|5|78.52|127.98|35.83|0.00|179.15|392.60|639.90|10.74|0.00|179.15|189.89|-213.45| +2451191|39484|15556|12205|831963|5674|32632|10|136|479|94|18.60|21.57|12.07|0.00|1134.58|1748.40|2027.58|22.69|0.00|1134.58|1157.27|-613.82| +2451191|39484|3686|12205|831963|5674|32632|10|267|479|18|95.07|137.85|34.46|0.00|620.28|1711.26|2481.30|43.41|0.00|620.28|663.69|-1090.98| +2452357|37840|13905|98042|1478148|5454|25960|2|46|480|19|86.93|132.13|109.66|0.00|2083.54|1651.67|2510.47|41.67|0.00|2083.54|2125.21|431.87| +2452357|37840|15588|98042|1478148|5454|25960|2|295|480|60|6.71|11.74|4.57|0.00|274.20|402.60|704.40|10.96|0.00|274.20|285.16|-128.40| +||11742|98042|1478148|5454|25960|2|148|480|3|6.45|||0.00|17.01|19.35|||0.00|||| +2452357|37840|1536|98042||5454|25960|||480|||85.25||0.00|||5115.00||0.00|766.80||-3760.20| +2452357|37840|16765|98042|1478148|||2||480|47|83.00||||4622.45||6163.58|||4622.45||721.45| +2452357|37840|12901|98042|1478148|5454|25960|2|183|480|32|21.03|34.48|28.96|324.35|926.72|672.96|1103.36|42.16|324.35|602.37|644.53|-70.59| +||6672||1478148|5454|||23|480|||112.80||0.00|4399.20|4130.88|5865.60|0.00|0.00|||| +2452357|37840|13086|98042|1478148|5454|25960|2|180|480|56|55.49|92.66|2.77|69.80|155.12|3107.44|5188.96|2.55|69.80|85.32|87.87|-3022.12| +2451943|58056|8669|68653|454505|3557|41409|8|277|481|69|16.38|31.77|6.67|0.00|460.23|1130.22|2192.13|0.00|0.00|460.23|460.23|-669.99| +2451943|58056|12463|68653|454505|3557|41409|8|181|481|36|50.40|56.44|32.17|0.00|1158.12|1814.40|2031.84|104.23|0.00|1158.12|1262.35|-656.28| +2451943|58056|16293|68653|454505|3557|41409|8|124|481|19|3.61|5.19|0.72|0.00|13.68|68.59|98.61|0.13|0.00|13.68|13.81|-54.91| +2451943|58056|12559|68653|454505|3557|41409|8|231|481|57|30.78|38.47|2.30|0.00|131.10|1754.46|2192.79|5.24|0.00|131.10|136.34|-1623.36| +2451943|58056|15071|68653|454505|3557|41409|8|108|481|42|21.71|40.16|18.87|0.00|792.54|911.82|1686.72|47.55|0.00|792.54|840.09|-119.28| +2451943|58056|15511|68653|454505|3557|41409|8|39|481|66|53.51|87.22|58.43|0.00|3856.38|3531.66|5756.52|269.94|0.00|3856.38|4126.32|324.72| +2451943|58056|1279|68653|454505|3557|41409|8|246|481|72|27.85|32.58|28.99|125.23|2087.28|2005.20|2345.76|0.00|125.23|1962.05|1962.05|-43.15| +2451943|58056|9339|68653|454505|3557|41409|8|37|481|86|6.37|11.72|1.17|0.00|100.62|547.82|1007.92|3.01|0.00|100.62|103.63|-447.20| +2451943|58056|6595|68653|454505|3557|41409|8|256|481|62|10.68|13.45|8.47|283.57|525.14|662.16|833.90|21.74|283.57|241.57|263.31|-420.59| +2451263|38241|12860|69961|706545|729|42113|8|185|482|91|11.20|21.61|15.55|141.50|1415.05|1019.20|1966.51|63.67|141.50|1273.55|1337.22|254.35| +2451263|38241|12826|69961|706545|729|42113|8|294|482|17|49.04|83.36|62.52|0.00|1062.84|833.68|1417.12|74.39|0.00|1062.84|1137.23|229.16| +2451263|38241|14377|69961|706545|729|42113|8|210|482|32|24.44|31.52|18.28|0.00|584.96|782.08|1008.64|52.64|0.00|584.96|637.60|-197.12| +2451263|38241|2440|69961|706545|729|42113|8|53|482|35|82.08|104.24|48.99|0.00|1714.65|2872.80|3648.40|0.00|0.00|1714.65|1714.65|-1158.15| +2451263|38241|8779|69961|706545|729|42113|8|109|482|7|37.43|67.37|29.64|0.00|207.48|262.01|471.59|10.37|0.00|207.48|217.85|-54.53| +2451263|38241|14372|69961|706545|729|42113|8|188|482|12|45.76|72.75|5.82|8.38|69.84|549.12|873.00|1.22|8.38|61.46|62.68|-487.66| +2451263|38241|5251|69961|706545|729|42113|8|107|482|21|54.71|54.71|29.54|490.06|620.34|1148.91|1148.91|9.11|490.06|130.28|139.39|-1018.63| +2451263|38241|6697|69961|706545|729|42113|8|214|482|56|64.72|81.54|44.03|0.00|2465.68|3624.32|4566.24|147.94|0.00|2465.68|2613.62|-1158.64| +2451263|38241|12355|69961|706545|729|42113|8|190|482|37|7.71|9.40|0.75|0.00|27.75|285.27|347.80|1.38|0.00|27.75|29.13|-257.52| +2451263|38241|3343|69961|706545|729|42113|8|10|482|53|60.01|106.81|40.58|0.00|2150.74|3180.53|5660.93|64.52|0.00|2150.74|2215.26|-1029.79| +2451263|38241|16666|69961|706545|729|42113|8|158|482|55|74.17|140.92|125.41|0.00|6897.55|4079.35|7750.60|620.77|0.00|6897.55|7518.32|2818.20| +2451263|38241|8990|69961|706545|729|42113|8|264|482|27|22.43|30.72|27.03|0.00|729.81|605.61|829.44|65.68|0.00|729.81|795.49|124.20| +2451263|38241|1555|69961|706545|729|42113|8|264|482|46|1.57|2.11|0.16|3.23|7.36|72.22|97.06|0.16|3.23|4.13|4.29|-68.09| +2452433|48516|8283|19957|237930|3456|15029|2|61|483|1|7.68|13.74|4.12|0.00|4.12|7.68|13.74|0.12|0.00|4.12|4.24|-3.56| +2452433|48516|8239|19957|237930|3456|15029|2|205|483|28|23.77|47.06|3.76|0.00|105.28|665.56|1317.68|6.31|0.00|105.28|111.59|-560.28| +2452433|48516|3769|19957|237930|3456|15029|2|91|483|43|37.12|56.05|37.55|0.00|1614.65|1596.16|2410.15|113.02|0.00|1614.65|1727.67|18.49| +2452433|48516|6405|19957|237930|3456|15029|2|137|483|73|96.08|184.47|75.63|0.00|5520.99|7013.84|13466.31|331.25|0.00|5520.99|5852.24|-1492.85| +2452433|48516|13431|19957|237930|3456|15029|2|186|483|72|79.39|123.05|100.90|0.00|7264.80|5716.08|8859.60|435.88|0.00|7264.80|7700.68|1548.72| +2452433|48516|13125|19957|237930|3456|15029|2|157|483|99|6.09|6.88|4.67|0.00|462.33|602.91|681.12|27.73|0.00|462.33|490.06|-140.58| +2452433|48516|16416|19957|237930|3456|15029|2|110|483|65|9.81|12.16|10.21|0.00|663.65|637.65|790.40|46.45|0.00|663.65|710.10|26.00| +2452433|48516|1317|19957|237930|3456|15029|2|204|483|54|7.75|10.15|6.19|0.00|334.26|418.50|548.10|10.02|0.00|334.26|344.28|-84.24| +2451088|65682|6032|23192|974296|6200|26457|1|185|484|11|36.41|48.06|8.17|0.00|89.87|400.51|528.66|8.08|0.00|89.87|97.95|-310.64| +2451088|65682|17935|23192|974296|6200|26457|1|217|484|50|79.09|90.16|18.93|359.67|946.50|3954.50|4508.00|52.81|359.67|586.83|639.64|-3367.67| +2451088|65682|3398|23192|974296|6200|26457|1|214|484|100|35.16|40.78|39.14|0.00|3914.00|3516.00|4078.00|313.12|0.00|3914.00|4227.12|398.00| +2451088|65682|4195|23192|974296|6200|26457|1|288|484|97|35.30|36.35|35.25|0.00|3419.25|3424.10|3525.95|102.57|0.00|3419.25|3521.82|-4.85| +2451088|65682|13231|23192|974296|6200|26457|1|70|484|15|30.10|33.41|11.69|103.45|175.35|451.50|501.15|3.59|103.45|71.90|75.49|-379.60| +2451088|65682|2098|23192|974296|6200|26457|1|3|484|8|31.59|52.12|2.60|0.00|20.80|252.72|416.96|0.20|0.00|20.80|21.00|-231.92| +2451088|65682|9274|23192|974296|6200|26457|1|231|484|49|69.80|80.27|69.03|0.00|3382.47|3420.20|3933.23|270.59|0.00|3382.47|3653.06|-37.73| +2451088|65682|10288|23192|974296|6200|26457|1|193|484|1|93.39|107.39|99.87|0.00|99.87|93.39|107.39|0.99|0.00|99.87|100.86|6.48| +2451088|65682|5858|23192|974296|6200|26457|1|208|484|69|38.80|60.52|44.78|0.00|3089.82|2677.20|4175.88|247.18|0.00|3089.82|3337.00|412.62| +2451088|65682|10108|23192|974296|6200|26457|1|207|484|89|66.23|129.14|104.60|1489.50|9309.40|5894.47|11493.46|312.79|1489.50|7819.90|8132.69|1925.43| +2451088|65682|9997|23192|974296|6200|26457|1|290|484|18|5.67|9.24|5.08|16.45|91.44|102.06|166.32|5.24|16.45|74.99|80.23|-27.07| +2452233|63875|10061|50551|1465557|4563|22722|10|111|485|27|10.32|17.75|5.68|0.00|153.36|278.64|479.25|9.20|0.00|153.36|162.56|-125.28| +2452233||1169|50551|1465557||22722||292|485||30.60||16.85|0.00|438.10||1288.82|21.90|0.00||460.00|-357.50| +2452233||10397|50551|||22722|||485|||60.79|48.02||3505.46|2323.59||||3505.46||1181.87| +2452233||9027|50551||4563|22722|10|280|485|15|99.90||68.47||1027.05|1498.50||0.00|||893.54|| +2452233|63875|2201|50551|1465557|4563|22722|10|177|485|77|42.19|56.95|2.84|0.00|218.68|3248.63|4385.15|15.30|0.00|218.68|233.98|-3029.95| +2452233|63875|16953|50551|1465557|4563|22722|10|134|485|78|54.51|99.20|27.77|0.00|2166.06|4251.78|7737.60|64.98|0.00|2166.06|2231.04|-2085.72| +2452233|63875|10585|50551|1465557|4563|22722|10|124|485|32|12.90|17.41|13.05|0.00|417.60|412.80|557.12|29.23|0.00|417.60|446.83|4.80| +2452233|63875|10921|50551|1465557|4563|22722|10|123|485|40|71.05|108.70|75.00|2760.00|3000.00|2842.00|4348.00|7.20|2760.00|240.00|247.20|-2602.00| +2452233|63875|11613|50551|1465557|4563|22722|10|161|485|60|11.40|18.81|15.61|0.00|936.60|684.00|1128.60|65.56|0.00|936.60|1002.16|252.60| +2452233|63875|709|50551|1465557|4563|22722|10|150|485|41|91.15|159.51|70.18|2848.60|2877.38|3737.15|6539.91|2.01|2848.60|28.78|30.79|-3708.37| +2452233|63875|14577|50551|1465557|4563|22722|10|239|485|76|89.51|175.43|56.13|0.00|4265.88|6802.76|13332.68|383.92|0.00|4265.88|4649.80|-2536.88| +2452230|52504|4587|5389|1702466|3096|10173|8|203|486|37|68.40|69.76|33.48|0.00|1238.76|2530.80|2581.12|99.10|0.00|1238.76|1337.86|-1292.04| +2452230|52504|57|5389|1702466|3096|10173|8|213|486|46|39.27|46.73|29.90|1141.58|1375.40|1806.42|2149.58|9.35|1141.58|233.82|243.17|-1572.60| +2452230|52504|2409|5389|1702466|3096|10173|8|108|486|8|18.91|26.28|7.35|53.50|58.80|151.28|210.24|0.31|53.50|5.30|5.61|-145.98| +2452230|52504|15827|5389|1702466|3096|10173|8|146|486|18|57.44|98.22|57.94|0.00|1042.92|1033.92|1767.96|31.28|0.00|1042.92|1074.20|9.00| +2452230|52504|15475|5389|1702466|3096|10173|8|121|486|27|73.42|146.84|24.96|0.00|673.92|1982.34|3964.68|40.43|0.00|673.92|714.35|-1308.42| +2452230|52504|15917|5389|1702466|3096|10173|8|159|486|40|34.77|69.19|59.50|0.00|2380.00|1390.80|2767.60|95.20|0.00|2380.00|2475.20|989.20| +2452230|52504|8647|5389|1702466|3096|10173|8|211|486|63|50.17|61.20|34.88|0.00|2197.44|3160.71|3855.60|21.97|0.00|2197.44|2219.41|-963.27| +2452230|52504|16821|5389|1702466|3096|10173|8|76|486|15|66.65|132.63|6.63|0.00|99.45|999.75|1989.45|7.95|0.00|99.45|107.40|-900.30| +2452535|63103|13266|47138|570980|1575|49370|8|27|487|70|71.44|113.58|87.45|0.00|6121.50|5000.80|7950.60|122.43|0.00|6121.50|6243.93|1120.70| +2452535|63103|7225|47138|570980|1575|49370|8|288|487|34|92.50|182.22|52.84|0.00|1796.56|3145.00|6195.48|125.75|0.00|1796.56|1922.31|-1348.44| +2452535|63103|16737|47138|570980|1575|49370|8|57|487|19|31.93|34.16|7.51|69.91|142.69|606.67|649.04|2.18|69.91|72.78|74.96|-533.89| +2452535|63103|150|47138|570980|1575|49370|8|206|487|15|46.62|62.47|2.49|0.00|37.35|699.30|937.05|1.86|0.00|37.35|39.21|-661.95| +2452535|63103|1689|47138|570980|1575|49370|8|198|487|65|17.92|31.53|25.22|803.25|1639.30|1164.80|2049.45|66.88|803.25|836.05|902.93|-328.75| +2452535|63103|17005|47138|570980|1575|49370|8|200|487|48|31.71|47.56|12.36|0.00|593.28|1522.08|2282.88|47.46|0.00|593.28|640.74|-928.80| +2452535|63103|14814|47138|570980|1575|49370|8|29|487|22|96.96|146.40|115.65|0.00|2544.30|2133.12|3220.80|25.44|0.00|2544.30|2569.74|411.18| +2452535|63103|11352|47138|570980|1575|49370|8|212|487|83|4.69|8.34|3.41|0.00|283.03|389.27|692.22|8.49|0.00|283.03|291.52|-106.24| +2452535|63103|414|47138|570980|1575|49370|8|126|487|74|87.20|170.91|82.03|0.00|6070.22|6452.80|12647.34|60.70|0.00|6070.22|6130.92|-382.58| +2452535|63103|12909|47138|570980|1575|49370|8|69|487|84|38.70|71.98|55.42|0.00|4655.28|3250.80|6046.32|325.86|0.00|4655.28|4981.14|1404.48| +2452535|63103|14961|47138|570980|1575|49370|8|52|487|74|97.39|141.21|83.31|0.00|6164.94|7206.86|10449.54|308.24|0.00|6164.94|6473.18|-1041.92| +2452535|63103|5775|47138|570980|1575|49370|8|98|487|74|39.00|58.50|7.02|0.00|519.48|2886.00|4329.00|41.55|0.00|519.48|561.03|-2366.52| +2451436|45932|8734|22046|1003968|6337|24659|1|253|488|21|65.24|73.06|16.07|0.00|337.47|1370.04|1534.26|6.74|0.00|337.47|344.21|-1032.57| +2451436|45932|16100|22046|1003968|6337|24659|1|170|488|18|10.29|20.06|7.62|0.00|137.16|185.22|361.08|9.60|0.00|137.16|146.76|-48.06| +2451436|45932|2266|22046|1003968|6337|24659|1|296|488|97|48.36|71.57|57.97|899.69|5623.09|4690.92|6942.29|236.17|899.69|4723.40|4959.57|32.48| +2451436|45932|16078|22046|1003968|6337|24659|1|10|488|96|93.94|131.51|69.70|0.00|6691.20|9018.24|12624.96|133.82|0.00|6691.20|6825.02|-2327.04| +2451436|45932|15662|22046|1003968|6337|24659|1|269|488|87|15.13|20.42|1.63|59.56|141.81|1316.31|1776.54|0.00|59.56|82.25|82.25|-1234.06| +2451436|45932|13405|22046|1003968|6337|24659|1|293|488|32|59.39|89.08|84.62|0.00|2707.84|1900.48|2850.56|54.15|0.00|2707.84|2761.99|807.36| +2451436|45932|11524|22046|1003968|6337|24659|1|48|488|12|17.90|30.78|22.16|0.00|265.92|214.80|369.36|2.65|0.00|265.92|268.57|51.12| +2451436|45932|17614|22046|1003968|6337|24659|1|33|488|71|76.01|96.53|64.67|0.00|4591.57|5396.71|6853.63|183.66|0.00|4591.57|4775.23|-805.14| +2451216|48716|7213|27327|918704|3681|42611|10|6|489|28|1.59|2.25|1.10|13.24|30.80|44.52|63.00|0.17|13.24|17.56|17.73|-26.96| +2451216|48716|8804|27327|918704|3681|42611|10|93|489|28|25.48|44.08|5.73|0.00|160.44|713.44|1234.24|1.60|0.00|160.44|162.04|-553.00| +2451216|48716|15950|27327|918704|3681|42611|10|229|489|34|27.61|35.61|3.56|84.72|121.04|938.74|1210.74|2.54|84.72|36.32|38.86|-902.42| +2451216|48716|11566|27327|918704|3681|42611|10|237|489|5|78.89|88.35|9.71|0.00|48.55|394.45|441.75|4.36|0.00|48.55|52.91|-345.90| +2451216|48716|5066|27327|918704|3681|42611|10|88|489|65|79.22|150.51|3.01|0.00|195.65|5149.30|9783.15|13.69|0.00|195.65|209.34|-4953.65| +2451216|48716|8554|27327|918704|3681|42611|10|187|489|84|32.26|45.48|40.02|0.00|3361.68|2709.84|3820.32|201.70|0.00|3361.68|3563.38|651.84| +2451216|48716|5150|27327|918704|3681|42611|10|62|489|1|34.00|44.88|41.73|0.00|41.73|34.00|44.88|0.83|0.00|41.73|42.56|7.73| +2451216|48716|3020|27327|918704|3681|42611|10|222|489|62|24.58|42.52|25.08|0.00|1554.96|1523.96|2636.24|62.19|0.00|1554.96|1617.15|31.00| +2451216|48716|5287|27327|918704|3681|42611|10|132|489|92|5.16|8.66|2.68|0.00|246.56|474.72|796.72|2.46|0.00|246.56|249.02|-228.16| +2451216|48716|10249|27327|918704|3681|42611|10|49|489|28|65.19|98.43|11.81|0.00|330.68|1825.32|2756.04|3.30|0.00|330.68|333.98|-1494.64| +2451216|48716|13570|27327|918704|3681|42611|10|50|489|37|55.76|103.15|84.58|0.00|3129.46|2063.12|3816.55|125.17|0.00|3129.46|3254.63|1066.34| +2451216|48716|10081|27327|918704|3681|42611|10|185|489|35|48.42|92.96|14.87|0.00|520.45|1694.70|3253.60|41.63|0.00|520.45|562.08|-1174.25| +2451216|48716|16375|27327|918704|3681|42611|10|113|489|33|94.40|104.78|98.49|0.00|3250.17|3115.20|3457.74|162.50|0.00|3250.17|3412.67|134.97| +2451145|38923|10786|31917|1240331|1098|32712|4|53|490|10|72.65|140.21|123.38|0.00|1233.80|726.50|1402.10|0.00|0.00|1233.80|1233.80|507.30| +2451145|38923|9163|31917|1240331|1098|32712|4|23|490|3|97.10|151.47|69.67|0.00|209.01|291.30|454.41|14.63|0.00|209.01|223.64|-82.29| +2451145|38923|16388|31917|1240331|1098|32712|4|273|490|4|11.91|19.05|10.09|27.84|40.36|47.64|76.20|0.37|27.84|12.52|12.89|-35.12| +2451145|38923|17968|31917|1240331|1098|32712|4|232|490|10|74.14|140.12|127.50|0.00|1275.00|741.40|1401.20|25.50|0.00|1275.00|1300.50|533.60| +2451145|38923|488|31917|1240331|1098|32712|4|118|490|41|6.12|8.93|8.21|0.00|336.61|250.92|366.13|26.92|0.00|336.61|363.53|85.69| +2451145|38923|2224|31917|1240331|1098|32712|4|149|490|5|50.26|79.41|54.79|0.00|273.95|251.30|397.05|2.73|0.00|273.95|276.68|22.65| +2451145|38923|1012|31917|1240331|1098|32712|4|193|490|6|51.14|58.29|57.12|0.00|342.72|306.84|349.74|17.13|0.00|342.72|359.85|35.88| +2451145|38923|13808|31917|1240331|1098|32712|4|128|490|63|97.29|148.85|26.79|0.00|1687.77|6129.27|9377.55|101.26|0.00|1687.77|1789.03|-4441.50| +2451145|38923|2810|31917|1240331|1098|32712|4|191|490|6|78.61|148.57|19.31|0.00|115.86|471.66|891.42|5.79|0.00|115.86|121.65|-355.80| +2451145|38923|943|31917|1240331|1098|32712|4|42|490|63|46.00|60.26|17.47|0.00|1100.61|2898.00|3796.38|11.00|0.00|1100.61|1111.61|-1797.39| +2451145|38923|6019|31917|1240331|1098|32712|4|92|490|53|21.82|26.40|16.36|0.00|867.08|1156.46|1399.20|78.03|0.00|867.08|945.11|-289.38| +2451145|38923|14240|31917|1240331|1098|32712|4|271|490|81|89.42|93.89|40.37|0.00|3269.97|7243.02|7605.09|0.00|0.00|3269.97|3269.97|-3973.05| +2451145|38923|716|31917|1240331|1098|32712|4|22|490|22|89.56|120.90|32.64|445.20|718.08|1970.32|2659.80|19.10|445.20|272.88|291.98|-1697.44| +2451145|38923|11605|31917|1240331|1098|32712|4|86|490|76|47.75|83.56|55.98|0.00|4254.48|3629.00|6350.56|297.81|0.00|4254.48|4552.29|625.48| +2451145|38923|85|31917|1240331|1098|32712|4|150|490|53|33.48|50.55|3.53|0.00|187.09|1774.44|2679.15|0.00|0.00|187.09|187.09|-1587.35| +2451145|38923|7766|31917|1240331|1098|32712|4|294|490|49|61.55|75.70|63.58|373.85|3115.42|3015.95|3709.30|219.32|373.85|2741.57|2960.89|-274.38| +2451742|52780|10291|61118|998251|1784|46117|10|165|491|24|64.16|105.22|56.81|1145.28|1363.44|1539.84|2525.28|8.72|1145.28|218.16|226.88|-1321.68| +2451742|52780|16441|61118|998251|1784|46117|10|50|491|16|69.82|85.18|36.62|0.00|585.92|1117.12|1362.88|5.85|0.00|585.92|591.77|-531.20| +2451742|52780|11408|61118|998251|1784|46117|10|20|491|83|66.16|99.90|32.96|0.00|2735.68|5491.28|8291.70|246.21|0.00|2735.68|2981.89|-2755.60| +2451742|52780|15529|61118|998251|1784|46117|10|197|491|28|85.90|99.64|23.91|394.99|669.48|2405.20|2789.92|2.74|394.99|274.49|277.23|-2130.71| +2451742|52780|6313|61118|998251|1784|46117|10|225|491|100|41.62|44.53|28.49|0.00|2849.00|4162.00|4453.00|256.41|0.00|2849.00|3105.41|-1313.00| +2451742|52780|4043|61118|998251|1784|46117|10|158|491|84|4.89|5.03|2.06|0.00|173.04|410.76|422.52|8.65|0.00|173.04|181.69|-237.72| +2451742|52780|5599|61118|998251|1784|46117|10|21|491|34|16.04|22.61|7.00|0.00|238.00|545.36|768.74|21.42|0.00|238.00|259.42|-307.36| +2451742|52780|17749|61118|998251|1784|46117|10|118|491|28|7.86|13.44|12.90|0.00|361.20|220.08|376.32|18.06|0.00|361.20|379.26|141.12| +2450864|47378|10676|17343|1778653|2363|46073|4|47|492|79|49.28|80.32|4.01|53.85|316.79|3893.12|6345.28|13.14|53.85|262.94|276.08|-3630.18| +2450864|47378|13573|17343|1778653|2363|46073|4|259|492|84|79.60|87.56|47.28|0.00|3971.52|6686.40|7355.04|238.29|0.00|3971.52|4209.81|-2714.88| +2450864|47378|2338|17343|1778653|2363|46073|4|207|492|45|11.12|13.90|13.48|0.00|606.60|500.40|625.50|30.33|0.00|606.60|636.93|106.20| +2450864|47378|5272|17343|1778653|2363|46073|4|5|492|50|41.23|75.86|65.23|0.00|3261.50|2061.50|3793.00|163.07|0.00|3261.50|3424.57|1200.00| +2450864|47378|14518|17343|1778653|2363|46073|4|132|492|74|43.36|57.66|27.67|0.00|2047.58|3208.64|4266.84|184.28|0.00|2047.58|2231.86|-1161.06| +2450864|47378|15062|17343|1778653|2363|46073|4|228|492|88|7.83|8.45|0.08|3.80|7.04|689.04|743.60|0.19|3.80|3.24|3.43|-685.80| +2450864||12259||||46073||123|492|82|85.27|||0.00|||11956.42||0.00|||-6275.46| +2450864|47378|14647|17343|1778653|2363|46073|4|241|492|72|48.04|91.75|20.18|0.00|1452.96|3458.88|6606.00|130.76|0.00|1452.96|1583.72|-2005.92| +2450864|47378|12712|17343|1778653|2363|46073|4|190|492|38|77.61|119.51|52.58|0.00|1998.04|2949.18|4541.38|19.98|0.00|1998.04|2018.02|-951.14| +2450864|47378|6326|17343|1778653|2363|46073|4|270|492|7|66.02|96.38|15.42|0.00|107.94|462.14|674.66|1.07|0.00|107.94|109.01|-354.20| +2451133|60735|8822|30590|809675|3607|30692|7|266|493|16|24.85|32.30|24.22|85.25|387.52|397.60|516.80|9.06|85.25|302.27|311.33|-95.33| +2451133|60735|94|30590|809675|3607|30692|7|260|493|20|32.17|55.33|29.32|0.00|586.40|643.40|1106.60|11.72|0.00|586.40|598.12|-57.00| +2451133|60735|5692|30590|809675|3607|30692|7|159|493|43|35.95|55.00|4.40|79.46|189.20|1545.85|2365.00|3.29|79.46|109.74|113.03|-1436.11| +2451133|60735|1003|30590|809675|3607|30692|7|184|493|77|37.98|68.36|48.53|0.00|3736.81|2924.46|5263.72|112.10|0.00|3736.81|3848.91|812.35| +2451133|60735|1390|30590|809675|3607|30692|7|18|493|46|86.24|98.31|33.42|0.00|1537.32|3967.04|4522.26|30.74|0.00|1537.32|1568.06|-2429.72| +2451133||14449||||30692|||493||60.23||101.65|||5059.32|8803.20|||6916.27||1856.95| +2451133|60735|13762|30590|809675|3607|30692|7|215|493|33|21.11|36.09|7.57|0.00|249.81|696.63|1190.97|17.48|0.00|249.81|267.29|-446.82| +2451133|60735|4726|30590|809675|3607|30692|7|185|493|33|44.02|83.19|6.65|0.00|219.45|1452.66|2745.27|10.97|0.00|219.45|230.42|-1233.21| +2452617|67763|1050|50712|638555|6845|3088|1|4|494|12|72.55|121.15|115.09|0.00|1381.08|870.60|1453.80|124.29|0.00|1381.08|1505.37|510.48| +2452617|67763|7179|50712|638555|6845|3088|1|282|494|48|24.70|30.13|20.18|0.00|968.64|1185.60|1446.24|87.17|0.00|968.64|1055.81|-216.96| +2452617|67763|16563|50712|638555|6845|3088|1|231|494|50|75.80|144.02|64.80|2721.60|3240.00|3790.00|7201.00|5.18|2721.60|518.40|523.58|-3271.60| +2452617|67763|16633|50712|638555|6845|3088|1|63|494|3|75.32|115.23|51.85|0.00|155.55|225.96|345.69|1.55|0.00|155.55|157.10|-70.41| +2452617|67763|11407|50712|638555|6845|3088|1|65|494|75|30.63|44.41|34.63|0.00|2597.25|2297.25|3330.75|207.78|0.00|2597.25|2805.03|300.00| +2452617|67763|7074|50712|638555|6845|3088|1|298|494|42|90.32|135.48|20.32|0.00|853.44|3793.44|5690.16|42.67|0.00|853.44|896.11|-2940.00| +2452617|67763|15195|50712|638555|6845|3088|1|166|494|73|55.29|102.83|25.70|0.00|1876.10|4036.17|7506.59|0.00|0.00|1876.10|1876.10|-2160.07| +2452617|67763|6483|50712|638555|6845|3088|1|264|494|16|52.54|56.74|9.64|0.00|154.24|840.64|907.84|13.88|0.00|154.24|168.12|-686.40| +2451892|69065|1484|5693|666259|425|42991|4|81|495|55|88.52|120.38|49.35|2279.97|2714.25|4868.60|6620.90|21.71|2279.97|434.28|455.99|-4434.32| +2451892|69065|9269|5693|666259|425|42991|4|64|495|5|88.16|120.77|80.91|93.04|404.55|440.80|603.85|9.34|93.04|311.51|320.85|-129.29| +2451892|69065|15323|5693|666259|425|42991|4|194|495|7|24.33|31.62|24.34|0.00|170.38|170.31|221.34|15.33|0.00|170.38|185.71|0.07| +2451892|69065|13723|5693|666259|425|42991|4|24|495|43|20.07|36.52|17.89|0.00|769.27|863.01|1570.36|0.00|0.00|769.27|769.27|-93.74| +2451892|69065|13141|5693|666259|425|42991|4|259|495|32|51.67|61.48|43.65|0.00|1396.80|1653.44|1967.36|111.74|0.00|1396.80|1508.54|-256.64| +2451892|69065|13079|5693|666259|425|42991|4|108|495|41|95.20|149.46|65.76|0.00|2696.16|3903.20|6127.86|0.00|0.00|2696.16|2696.16|-1207.04| +2451892|69065|2459|5693|666259|425|42991|4|230|495|73|21.52|33.78|19.93|0.00|1454.89|1570.96|2465.94|29.09|0.00|1454.89|1483.98|-116.07| +2451892|69065|15349|5693|666259|425|42991|4|142|495|20|6.06|11.81|7.20|0.00|144.00|121.20|236.20|5.76|0.00|144.00|149.76|22.80| +2451892|69065|3607|5693|666259|425|42991|4|298|495|51|36.00|72.00|9.36|329.37|477.36|1836.00|3672.00|1.47|329.37|147.99|149.46|-1688.01| +2451864|53616|8297|36845|1582021|6844|1620|10|125|496|86|37.00|38.48|35.01|511.84|3010.86|3182.00|3309.28|74.97|511.84|2499.02|2573.99|-682.98| +2451864|53616|12877|36845|1582021|6844|1620|10|50|496|33|81.25|100.75|91.68|0.00|3025.44|2681.25|3324.75|211.78|0.00|3025.44|3237.22|344.19| +2451864|53616|6356|36845|1582021|6844|1620|10|73|496|83|57.36|101.52|35.53|0.00|2948.99|4760.88|8426.16|147.44|0.00|2948.99|3096.43|-1811.89| +2451864|53616|7999|36845|1582021|6844|1620|10|252|496|33|77.53|132.57|23.86|661.39|787.38|2558.49|4374.81|5.03|661.39|125.99|131.02|-2432.50| +2451864|53616|12433|36845|1582021|6844|1620|10|161|496|76|70.53|134.71|119.89|0.00|9111.64|5360.28|10237.96|728.93|0.00|9111.64|9840.57|3751.36| +2451864|53616|5405|36845|1582021|6844|1620|10|279|496|67|51.92|62.82|4.39|273.54|294.13|3478.64|4208.94|1.64|273.54|20.59|22.23|-3458.05| +2451864|53616|6581|36845|1582021|6844|1620|10|60|496|50|1.35|1.86|1.74|33.93|87.00|67.50|93.00|3.18|33.93|53.07|56.25|-14.43| +2451864|53616|4111|36845|1582021|6844|1620|10|17|496|44|86.38|107.11|36.41|304.38|1602.04|3800.72|4712.84|12.97|304.38|1297.66|1310.63|-2503.06| +2451864|53616|17419|36845|1582021|6844|1620|10|42|496|31|68.03|71.43|53.57|0.00|1660.67|2108.93|2214.33|49.82|0.00|1660.67|1710.49|-448.26| +2451864|53616|13201|36845|1582021|6844|1620|10|174|496|47|20.29|32.46|1.62|0.00|76.14|953.63|1525.62|4.56|0.00|76.14|80.70|-877.49| +2452606|63741|11425|86799|821310|3374|39085|10|171|497|65|36.82|55.59|37.80|0.00|2457.00|2393.30|3613.35|98.28|0.00|2457.00|2555.28|63.70| +2452606|63741|13533|86799|821310|3374|39085|10|229|497|46|33.01|48.85|40.54|951.06|1864.84|1518.46|2247.10|36.55|951.06|913.78|950.33|-604.68| +2452606|63741|16530|86799|821310|3374|39085|10|4|497|94|38.74|59.27|43.85|0.00|4121.90|3641.56|5571.38|0.00|0.00|4121.90|4121.90|480.34| +2452606|63741|16224|86799|821310|3374|39085|10|136|497|58|47.18|81.14|4.05|218.45|234.90|2736.44|4706.12|0.32|218.45|16.45|16.77|-2719.99| +|63741|13327|86799|821310|3374|39085|10|73|497|||||0.00||1297.14|1647.36||0.00|||-1247.74| +2452606|63741|13591|86799|821310|3374|39085|10|208|497|2|12.57|18.10|17.91|0.00|35.82|25.14|36.20|2.14|0.00|35.82|37.96|10.68| +2452606|63741|258|86799|821310|3374|39085|10|200|497|61|46.83|59.00|10.03|0.00|611.83|2856.63|3599.00|24.47|0.00|611.83|636.30|-2244.80| +2452606|63741|1503|86799|821310|3374|39085|10|160|497|46|22.07|44.14|13.24|0.00|609.04|1015.22|2030.44|12.18|0.00|609.04|621.22|-406.18| +2452606|63741|2869|86799|821310|3374|39085|10|157|497|70|93.35|177.36|104.64|0.00|7324.80|6534.50|12415.20|0.00|0.00|7324.80|7324.80|790.30| +2452606|63741|465|86799|821310|3374|39085|10|199|497|96|6.35|11.62|3.60|0.00|345.60|609.60|1115.52|13.82|0.00|345.60|359.42|-264.00| +|63741|4237|86799|821310||||228|497||23.46|||0.00|2306.60||3343.05|0.00|0.00|2306.60||77.90| +2452606|63741|1107|86799|821310|3374|39085|10|212|497|95|84.09|121.08|24.21|0.00|2299.95|7988.55|11502.60|91.99|0.00|2299.95|2391.94|-5688.60| +2452606|63741|10332|86799|821310|3374|39085|10|168|497|53|60.26|88.58|31.00|722.92|1643.00|3193.78|4694.74|0.00|722.92|920.08|920.08|-2273.70| +2452606|63741|8925|86799|821310|3374|39085|10|221|497|62|25.29|48.05|9.61|0.00|595.82|1567.98|2979.10|53.62|0.00|595.82|649.44|-972.16| +2452606|63741|4567|86799|821310|3374|39085|10|39|497|20|46.58|63.34|8.23|0.00|164.60|931.60|1266.80|3.29|0.00|164.60|167.89|-767.00| +2452606|63741|16464|86799|821310|3374|39085|10|261|497|59|72.38|86.85|63.40|0.00|3740.60|4270.42|5124.15|37.40|0.00|3740.60|3778.00|-529.82| +2451810|45147|3385|52762|787442|75|24355|4|148|498|88|51.35|85.75|10.29|0.00|905.52|4518.80|7546.00|81.49|0.00|905.52|987.01|-3613.28| +2451810|45147|3104|52762|787442|75|24355|4|55|498|77|26.29|39.17|6.65|0.00|512.05|2024.33|3016.09|0.00|0.00|512.05|512.05|-1512.28| +2451810|45147|16694|52762|787442|75|24355|4|18|498|76|6.01|7.57|2.57|0.00|195.32|456.76|575.32|5.85|0.00|195.32|201.17|-261.44| +2451810|45147|326|52762|787442|75|24355|4|277|498|9|53.76|76.33|0.76|0.00|6.84|483.84|686.97|0.06|0.00|6.84|6.90|-477.00| +2451810|45147|5492|52762|787442|75|24355|4|289|498|86|12.43|19.01|17.48|0.00|1503.28|1068.98|1634.86|105.22|0.00|1503.28|1608.50|434.30| +2451810|45147|9620|52762|787442|75|24355|4|144|498|49|32.71|62.14|38.52|0.00|1887.48|1602.79|3044.86|94.37|0.00|1887.48|1981.85|284.69| +2451810|45147|8653|52762|787442|75|24355|4|207|498|93|17.25|24.84|24.34|0.00|2263.62|1604.25|2310.12|90.54|0.00|2263.62|2354.16|659.37| +2451810|45147|13208|52762|787442|75|24355|4|13|498|45|1.86|2.71|2.41|0.00|108.45|83.70|121.95|5.42|0.00|108.45|113.87|24.75| +2451810|45147|3175|52762|787442|75|24355|4|18|498|67|98.88|112.72|33.81|0.00|2265.27|6624.96|7552.24|158.56|0.00|2265.27|2423.83|-4359.69| +2451810|45147|10069|52762|787442|75|24355|4|293|498|18|52.57|67.81|66.45|0.00|1196.10|946.26|1220.58|95.68|0.00|1196.10|1291.78|249.84| +2451810|45147|3755|52762|787442|75|24355|4|250|498|11|6.84|11.28|7.55|78.06|83.05|75.24|124.08|0.04|78.06|4.99|5.03|-70.25| +2451810|45147|3278|52762|787442|75|24355|4|287|498|9|69.63|135.08|20.26|0.00|182.34|626.67|1215.72|7.29|0.00|182.34|189.63|-444.33| +||6074|66301||||7||499|||12.34|||175.68|105.60||14.05||||| +2450965|30014|2714|66301|1678562|2376|33437|7|167|499|77|23.59|41.51|2.90|0.00|223.30|1816.43|3196.27|4.46|0.00|223.30|227.76|-1593.13| +2450965|30014|3412|66301|1678562|2376|33437|7|113|499|70|13.67|18.86|14.52|0.00|1016.40|956.90|1320.20|0.00|0.00|1016.40|1016.40|59.50| +2450965|30014|14924|66301|1678562|2376|33437|7|80|499|85|32.11|32.11|19.26|0.00|1637.10|2729.35|2729.35|81.85|0.00|1637.10|1718.95|-1092.25| +2450965|30014|4063|66301|1678562|2376|33437|7|104|499|76|67.07|67.07|6.70|0.00|509.20|5097.32|5097.32|20.36|0.00|509.20|529.56|-4588.12| +2450965|30014|6328|66301|1678562|2376|33437|7|116|499|7|57.15|100.58|92.53|0.00|647.71|400.05|704.06|25.90|0.00|647.71|673.61|247.66| +2450965|30014|14608|66301|1678562|2376|33437|7|153|499|91|40.80|78.33|58.74|0.00|5345.34|3712.80|7128.03|374.17|0.00|5345.34|5719.51|1632.54| +2450965|30014|15352|66301|1678562|2376|33437|7|91|499|3|37.83|51.82|17.10|0.00|51.30|113.49|155.46|1.02|0.00|51.30|52.32|-62.19| +2450965|30014|3079|66301|1678562|2376|33437|7|27|499|60|1.08|1.84|1.71|0.00|102.60|64.80|110.40|5.13|0.00|102.60|107.73|37.80| +2450965|30014|14500|66301|1678562|2376|33437|7|107|499|54|27.97|40.55|15.81|0.00|853.74|1510.38|2189.70|8.53|0.00|853.74|862.27|-656.64| +2450965|30014|5026|66301|1678562|2376|33437|7|243|499|81|80.42|128.67|45.03|583.58|3647.43|6514.02|10422.27|0.00|583.58|3063.85|3063.85|-3450.17| +2452543|74066|1533|81767|1870014|3061|49308|1|96|500|18|18.37|19.28|11.18|0.00|201.24|330.66|347.04|6.03|0.00|201.24|207.27|-129.42| +2452543|74066|14245|81767|1870014|3061|49308|1|46|500|35|49.63|80.89|76.84|0.00|2689.40|1737.05|2831.15|215.15|0.00|2689.40|2904.55|952.35| +2452543|74066|7957|81767|1870014|3061|49308|1|39|500|100|63.95|95.92|66.18|0.00|6618.00|6395.00|9592.00|0.00|0.00|6618.00|6618.00|223.00| +2452543|74066|8959|81767|1870014|3061|49308|1|241|500|23|2.18|3.35|1.37|0.00|31.51|50.14|77.05|2.52|0.00|31.51|34.03|-18.63| +2452543|74066|11238|81767|1870014|3061|49308|1|82|500|8|25.89|51.00|27.54|0.00|220.32|207.12|408.00|6.60|0.00|220.32|226.92|13.20| +2452543|74066|6738|81767|1870014|3061|49308|1|123|500|1|3.83|5.93|3.79|0.26|3.79|3.83|5.93|0.31|0.26|3.53|3.84|-0.30| +2452543|74066|5359|81767|1870014|3061|49308|1|268|500|23|21.82|40.80|15.91|0.00|365.93|501.86|938.40|25.61|0.00|365.93|391.54|-135.93| +2452543|74066|2203|81767|1870014|3061|49308|1|190|500|28|34.66|58.22|40.17|0.00|1124.76|970.48|1630.16|0.00|0.00|1124.76|1124.76|154.28| +2452543|74066|2169|81767|1870014|3061|49308|1|55|500|73|63.95|83.13|52.37|1987.96|3823.01|4668.35|6068.49|73.40|1987.96|1835.05|1908.45|-2833.30| +2452543|74066|17031|81767|1870014|3061|49308|1|242|500|37|96.25|121.27|37.59|0.00|1390.83|3561.25|4486.99|69.54|0.00|1390.83|1460.37|-2170.42| +2452543|74066|10305|81767|1870014|3061|49308|1|218|500|97|98.37|183.95|84.61|0.00|8207.17|9541.89|17843.15|164.14|0.00|8207.17|8371.31|-1334.72| +2452543|74066|433|81767|1870014|3061|49308|1|239|500|19|4.47|8.04|0.24|4.10|4.56|84.93|152.76|0.02|4.10|0.46|0.48|-84.47| +2452543|74066|9729|81767|1870014|3061|49308|1|162|500|76|79.20|143.35|131.88|0.00|10022.88|6019.20|10894.60|0.00|0.00|10022.88|10022.88|4003.68| +2452543|74066|2029|81767|1870014|3061|49308|1|164|500|20|25.14|48.77|15.11|3.02|302.20|502.80|975.40|11.96|3.02|299.18|311.14|-203.62| +2452493|53282|1675|92812|1851835|5705|10728|7|50|501|17|65.51|71.40|10.71|0.00|182.07|1113.67|1213.80|9.10|0.00|182.07|191.17|-931.60| +||5629||1851835|||||501||||113.56|115.83|5791.56||||115.83||5732.48|1480.47| +2452493||16951||||10728|||501||||||||2023.35|||||| +2452493|53282|8557|92812|1851835|5705|10728|7|64|501|51|16.27|19.84|15.67|0.00|799.17|829.77|1011.84|0.00|0.00|799.17|799.17|-30.60| +2452493|53282|528|92812|1851835|5705|10728|7|258|501|76|82.16|137.20|127.59|0.00|9696.84|6244.16|10427.20|581.81|0.00|9696.84|10278.65|3452.68| +2452493|53282|11863|92812|1851835|5705|10728|7|100|501|29|66.90|94.99|89.29|0.00|2589.41|1940.10|2754.71|233.04|0.00|2589.41|2822.45|649.31| +2452493|53282|15825|92812|1851835|5705|10728|7|10|501|7|99.30|169.80|169.80|0.00|1188.60|695.10|1188.60|71.31|0.00|1188.60|1259.91|493.50| +2452493|53282|15819|92812|1851835|5705|10728|7|190|501|25|4.10|7.50|7.20|0.00|180.00|102.50|187.50|1.80|0.00|180.00|181.80|77.50| +2452493|53282|9987|92812|1851835|5705|10728|7|199|501|34|80.34|140.59|68.88|0.00|2341.92|2731.56|4780.06|187.35|0.00|2341.92|2529.27|-389.64| +2452493|53282|16383|92812|1851835|5705|10728|7|10|501|67|60.49|71.37|59.95|0.00|4016.65|4052.83|4781.79|240.99|0.00|4016.65|4257.64|-36.18| +2450963|62877|17977|80216|381922|75|42581|8|168|502|51|91.36|155.31|51.25|0.00|2613.75|4659.36|7920.81|156.82|0.00|2613.75|2770.57|-2045.61| +2450963|62877|2770|80216|381922|75|42581|8|276|502|9|63.79|113.54|101.05|0.00|909.45|574.11|1021.86|72.75|0.00|909.45|982.20|335.34| +2450963|62877|4550|80216|381922|75|42581|8|118|502|22|8.25|8.25|1.32|0.00|29.04|181.50|181.50|0.58|0.00|29.04|29.62|-152.46| +2450963|62877|10519|80216|381922|75|42581|8|271|502|85|40.92|74.06|38.51|2847.81|3273.35|3478.20|6295.10|4.25|2847.81|425.54|429.79|-3052.66| +2450963|62877|649|80216|381922|75|42581|8|4|502|95|18.85|23.18|9.96|321.70|946.20|1790.75|2202.10|6.24|321.70|624.50|630.74|-1166.25| +2450963|62877|15956|80216|381922|75|42581|8|52|502|32|19.22|20.56|9.45|0.00|302.40|615.04|657.92|15.12|0.00|302.40|317.52|-312.64| +2450963|62877|9988|80216|381922|75|42581|8|296|502|24|51.05|76.57|29.86|0.00|716.64|1225.20|1837.68|57.33|0.00|716.64|773.97|-508.56| +2450963|62877|6046|80216|381922|75|42581|8|99|502|90|74.14|124.55|37.36|0.00|3362.40|6672.60|11209.50|134.49|0.00|3362.40|3496.89|-3310.20| +2450963|62877|4630|80216|381922|75|42581|8|39|502|81|31.69|48.80|21.47|0.00|1739.07|2566.89|3952.80|104.34|0.00|1739.07|1843.41|-827.82| +2450963|62877|17942|80216|381922|75|42581|8|44|502|93|13.65|24.43|7.81|0.00|726.33|1269.45|2271.99|29.05|0.00|726.33|755.38|-543.12| +2450963|62877|9842|80216|381922|75|42581|8|237|502|16|43.62|74.59|2.98|0.00|47.68|697.92|1193.44|0.95|0.00|47.68|48.63|-650.24| +2450963|62877|17587|80216|381922|75|42581|8|196|502|36|34.27|34.27|12.67|0.00|456.12|1233.72|1233.72|31.92|0.00|456.12|488.04|-777.60| +2451479|36892|698|46284|661611|938|21018|1|37|503|7|74.46|131.04|61.58|0.00|431.06|521.22|917.28|0.00|0.00|431.06|431.06|-90.16| +2451479|36892|11600|46284|661611|938|21018|1|148|503|49|93.27|104.46|60.58|0.00|2968.42|4570.23|5118.54|237.47|0.00|2968.42|3205.89|-1601.81| +2451479|36892|5182|46284|661611||21018|1||503|93||152.88|85.61|1990.43|7961.73|8124.48|14217.84|537.41|1990.43|||-2153.18| +2451479|36892|4627|46284|661611|938|21018|1|33|503|54|98.05|168.64|23.60|0.00|1274.40|5294.70|9106.56|38.23|0.00|1274.40|1312.63|-4020.30| +2451479|36892|9751|46284|661611|938|21018|1|171|503|46|24.30|37.66|6.77|0.00|311.42|1117.80|1732.36|12.45|0.00|311.42|323.87|-806.38| +2451479|36892|15332|46284|661611|938|21018|1|261|503|5|92.70|180.76|1.80|0.00|9.00|463.50|903.80|0.09|0.00|9.00|9.09|-454.50| +2451479|36892|4832|46284|661611|938|21018|1|86|503|33|89.52|102.94|66.91|0.00|2208.03|2954.16|3397.02|132.48|0.00|2208.03|2340.51|-746.13| +2451479|36892|10486|46284|661611|938|21018|1|161|503|53|87.48|143.46|123.37|915.40|6538.61|4636.44|7603.38|449.85|915.40|5623.21|6073.06|986.77| +2451479|36892|5905|46284|661611|938|21018|1|228|503|14|67.48|91.77|69.74|0.00|976.36|944.72|1284.78|78.10|0.00|976.36|1054.46|31.64| +2451479|36892|8954|46284|661611|938|21018|1|95|503|98|88.03|135.56|130.13|0.00|12752.74|8626.94|13284.88|1147.74|0.00|12752.74|13900.48|4125.80| +2451479|36892|15178|46284|661611|938|21018|1|53|503|85|56.24|69.17|60.86|0.00|5173.10|4780.40|5879.45|465.57|0.00|5173.10|5638.67|392.70| +2451479|36892|4795|46284|661611|938|21018|1|87|503|45|97.92|182.13|114.74|0.00|5163.30|4406.40|8195.85|206.53|0.00|5163.30|5369.83|756.90| +2451479|36892|17254|46284|661611|938|21018|1|217|503|62|14.31|16.45|7.23|0.00|448.26|887.22|1019.90|22.41|0.00|448.26|470.67|-438.96| +2451479|36892|12571|46284|661611|938|21018|1|257|503|41|30.61|36.11|0.00|0.00|0.00|1255.01|1480.51|0.00|0.00|0.00|0.00|-1255.01| +2451479|36892|10177|46284|661611|938|21018|1|177|503|85|97.37|139.23|129.48|0.00|11005.80|8276.45|11834.55|550.29|0.00|11005.80|11556.09|2729.35| +2451479|36892|8596|46284|661611|938|21018|1|146|503|97|54.13|100.14|99.13|0.00|9615.61|5250.61|9713.58|769.24|0.00|9615.61|10384.85|4365.00| +2452565|35480|13933|73671|230053|835|19617|4|296|504|94|78.51|117.76|44.74|0.00|4205.56|7379.94|11069.44|126.16|0.00|4205.56|4331.72|-3174.38| +2452565|35480|11589|73671|230053|835|19617|4|176|504|4|42.18|66.64|27.98|0.00|111.92|168.72|266.56|6.71|0.00|111.92|118.63|-56.80| +|35480|15171|73671|||||116|504||9.35|14.11|3.80||||973.59|0.00|||0.00|-645.15| +2452565|35480|6834|73671|230053|835|19617|4|134|504|8|75.78|85.63|60.79|466.86|486.32|606.24|685.04|1.75|466.86|19.46|21.21|-586.78| +2452565|35480|13159|73671|230053|835|19617|4|278|504|56|68.25|82.58|61.93|0.00|3468.08|3822.00|4624.48|0.00|0.00|3468.08|3468.08|-353.92| +|35480|1215|73671|230053|835||||504||||7.41||||3481.27|31.64||452.01||| +2452565|35480|14949|73671|230053|835|19617|4|297|504|100|77.57|83.77|38.53|0.00|3853.00|7757.00|8377.00|0.00|0.00|3853.00|3853.00|-3904.00| +2452565|35480|17239|73671|230053|835|19617|4|216|504|15|58.10|106.90|48.10|0.00|721.50|871.50|1603.50|36.07|0.00|721.50|757.57|-150.00| +2452565|35480|3750|73671|230053|835|19617|4|32|504|89|33.56|52.01|3.12|155.50|277.68|2986.84|4628.89|6.10|155.50|122.18|128.28|-2864.66| +2451497|67917|8392|91588|290857|5773|20689|7|294|505|89|91.87|163.52|150.43|0.00|13388.27|8176.43|14553.28|401.64|0.00|13388.27|13789.91|5211.84| +2451497|67917|13094|91588|290857|5773|20689|7|266|505|55|39.89|65.41|61.48|0.00|3381.40|2193.95|3597.55|101.44|0.00|3381.40|3482.84|1187.45| +2451497|67917|6823|91588|290857|5773|20689|7|9|505|14|50.73|67.97|5.43|33.44|76.02|710.22|951.58|2.12|33.44|42.58|44.70|-667.64| +2451497|67917|7088|91588|290857|5773|20689|7|297|505|43|78.37|89.34|32.16|0.00|1382.88|3369.91|3841.62|69.14|0.00|1382.88|1452.02|-1987.03| +2451497|67917|8366|91588|290857|5773|20689|7|194|505|30|88.99|95.21|64.74|0.00|1942.20|2669.70|2856.30|38.84|0.00|1942.20|1981.04|-727.50| +2451497|67917|9370|91588|290857|5773|20689|7|254|505|86|43.04|65.42|64.76|0.00|5569.36|3701.44|5626.12|0.00|0.00|5569.36|5569.36|1867.92| +2451497|67917|13286|91588|290857|5773|20689|7|123|505|58|4.76|7.09|4.46|0.00|258.68|276.08|411.22|0.00|0.00|258.68|258.68|-17.40| +2451497|67917|15692|91588|290857||20689|||505|||23.84|21.93|0.00|||||0.00|||| +2451497|67917|11674|91588|290857|5773|20689|7|273|505|20|32.04|54.14|39.52|150.17|790.40|640.80|1082.80|12.80|150.17|640.23|653.03|-0.57| +2451497|67917|3518|91588|290857|5773|20689|7|135|505|98|88.68|123.26|85.04|0.00|8333.92|8690.64|12079.48|500.03|0.00|8333.92|8833.95|-356.72| +2451497|67917|5707|91588|290857|5773|20689|7|96|505|71|70.25|135.58|109.81|3820.28|7796.51|4987.75|9626.18|198.81|3820.28|3976.23|4175.04|-1011.52| +2452314|29316|5013|11269|1517808|4700|1192|10|113|506|42|97.79|110.50|36.46|229.69|1531.32|4107.18|4641.00|78.09|229.69|1301.63|1379.72|-2805.55| +2452314|29316|14695|11269|1517808|4700|1192|10|148|506|48|45.81|63.67|19.73|0.00|947.04|2198.88|3056.16|85.23|0.00|947.04|1032.27|-1251.84| +2452314|29316|15325|11269|1517808|4700|1192|10|291|506|11|24.29|35.94|19.40|0.00|213.40|267.19|395.34|10.67|0.00|213.40|224.07|-53.79| +2452314|29316|12679|11269|1517808|4700|1192|10|173|506|53|3.18|5.21|0.20|0.00|10.60|168.54|276.13|0.00|0.00|10.60|10.60|-157.94| +2452314|29316|11173|11269|1517808|4700|1192|10|265|506|89|46.32|66.70|57.36|2909.87|5105.04|4122.48|5936.30|0.00|2909.87|2195.17|2195.17|-1927.31| +2452314|29316|17401|11269|1517808|4700|1192|10|149|506|39|46.04|90.69|32.64|0.00|1272.96|1795.56|3536.91|76.37|0.00|1272.96|1349.33|-522.60| +2452314|29316|933|11269|1517808|4700|1192|10|147|506|59|24.61|46.51|45.11|0.00|2661.49|1451.99|2744.09|0.00|0.00|2661.49|2661.49|1209.50| +2452314|29316|17916|11269|1517808|4700|1192|10|102|506|35|34.43|53.02|6.36|0.00|222.60|1205.05|1855.70|8.90|0.00|222.60|231.50|-982.45| +2452314|29316|14871|11269|1517808|4700|1192|10|239|506|96|41.92|53.23|14.90|0.00|1430.40|4024.32|5110.08|14.30|0.00|1430.40|1444.70|-2593.92| +2452314|29316|2011|11269|1517808|4700|1192|10|76|506|25|56.59|80.35|24.90|0.00|622.50|1414.75|2008.75|49.80|0.00|622.50|672.30|-792.25| +2452314|29316|9378|11269|1517808|4700|1192|10|35|506|45|10.36|12.95|11.65|0.00|524.25|466.20|582.75|20.97|0.00|524.25|545.22|58.05| +||17109|11269|||1192|10||506|72|92.46|162.72|||10778.40|6657.12|11715.84|754.48||10778.40||| +2452314|29316|7261|11269|1517808|4700|1192|10|226|506|95|80.76|84.79|15.26|173.96|1449.70|7672.20|8055.05|102.05|173.96|1275.74|1377.79|-6396.46| +2452314|29316|883|11269|1517808|4700|1192|10|224|506|75|27.67|40.39|23.42|0.00|1756.50|2075.25|3029.25|140.52|0.00|1756.50|1897.02|-318.75| +2452314|29316|16752|11269|1517808|4700|1192|10|62|506|82|99.88|193.76|34.87|0.00|2859.34|8190.16|15888.32|57.18|0.00|2859.34|2916.52|-5330.82| +2451938|45457|13481|70466|1868287|5661|23082|4|79|507|17|84.99|127.48|127.48|0.00|2167.16|1444.83|2167.16|173.37|0.00|2167.16|2340.53|722.33| +2451938||2845|70466||5661|23082|4||507|67|||||||5585.12|||341.70|351.95|| +2451938|45457|3401|70466|1868287|5661|23082|4|121|507|97|71.53|97.99|52.91|0.00|5132.27|6938.41|9505.03|0.00|0.00|5132.27|5132.27|-1806.14| +2451938|45457|7025|70466|1868287|5661|23082|4|16|507|14|30.84|46.26|39.32|176.15|550.48|431.76|647.64|18.71|176.15|374.33|393.04|-57.43| +2451938|45457|9349|70466|1868287|5661|23082|4|138|507|8|40.20|63.51|34.93|0.00|279.44|321.60|508.08|25.14|0.00|279.44|304.58|-42.16| +2451938|45457|13657|70466|1868287|5661|23082|4|282|507|23|36.47|37.92|8.34|0.00|191.82|838.81|872.16|7.67|0.00|191.82|199.49|-646.99| +2451938|45457|5253|70466|1868287|5661|23082|4|84|507|87|88.63|124.08|34.74|0.00|3022.38|7710.81|10794.96|30.22|0.00|3022.38|3052.60|-4688.43| +2451938|45457|5575|70466|1868287|5661|23082|4|263|507|45|5.52|7.23|1.51|0.00|67.95|248.40|325.35|3.39|0.00|67.95|71.34|-180.45| +2451938|45457|4753|70466|1868287|5661|23082|4|68|507|20|9.19|14.52|7.26|0.00|145.20|183.80|290.40|13.06|0.00|145.20|158.26|-38.60| +2451938|45457|17813|70466|1868287|5661|23082|4|236|507|37|35.65|44.91|39.07|0.00|1445.59|1319.05|1661.67|0.00|0.00|1445.59|1445.59|126.54| +2451938|45457|16981|70466|1868287|5661|23082|4|23|507|93|2.75|4.56|1.23|0.00|114.39|255.75|424.08|3.43|0.00|114.39|117.82|-141.36| +2451938|45457|5661|70466|1868287|5661|23082|4|108|507|84|71.91|117.21|10.54|433.82|885.36|6040.44|9845.64|22.57|433.82|451.54|474.11|-5588.90| +2451938|45457|4733|70466|1868287|5661|23082|4|182|507|65|8.21|12.56|8.16|0.00|530.40|533.65|816.40|15.91|0.00|530.40|546.31|-3.25| +2451938|45457|3315|70466|1868287|5661|23082|4|262|507|91|42.16|43.84|11.83|0.00|1076.53|3836.56|3989.44|0.00|0.00|1076.53|1076.53|-2760.03| +2451938|45457|4439|70466|1868287|5661|23082|4|263|507|50|40.56|54.35|30.97|154.85|1548.50|2028.00|2717.50|97.55|154.85|1393.65|1491.20|-634.35| +2451780|63821|7523|23637|260966|4695|49624|8|174|508|8|41.25|77.96|56.91|0.00|455.28|330.00|623.68|4.55|0.00|455.28|459.83|125.28| +2451780|63821|12746|23637|260966|4695|49624|8|48|508|47|7.25|13.99|1.11|17.21|52.17|340.75|657.53|0.69|17.21|34.96|35.65|-305.79| +2451780|63821|6746|23637|260966|4695|49624|8|14|508|42|17.47|28.30|28.01|0.00|1176.42|733.74|1188.60|0.00|0.00|1176.42|1176.42|442.68| +2451780|63821|4382|23637|260966|4695|49624|8|71|508|56|12.40|22.44|5.16|0.00|288.96|694.40|1256.64|8.66|0.00|288.96|297.62|-405.44| +2451780|63821|13022|23637|260966|4695|49624|8|104|508|76|48.74|61.89|25.37|0.00|1928.12|3704.24|4703.64|134.96|0.00|1928.12|2063.08|-1776.12| +2451780|63821|8069|23637|260966|4695|49624|8|239|508|58|95.55|185.36|44.48|0.00|2579.84|5541.90|10750.88|25.79|0.00|2579.84|2605.63|-2962.06| +2451780|63821|9469|23637|260966|4695|49624|8|146|508|82|74.10|97.07|69.89|2865.49|5730.98|6076.20|7959.74|200.58|2865.49|2865.49|3066.07|-3210.71| +2451780|63821|10003|23637|260966|4695|49624|8|134|508|4|30.93|38.35|31.83|0.00|127.32|123.72|153.40|2.54|0.00|127.32|129.86|3.60| +2452231|35524|13999|82312|338159|406|653|10|237|509|6|15.37|16.75|5.52|0.00|33.12|92.22|100.50|0.99|0.00|33.12|34.11|-59.10| +2452231|35524|13947|82312|338159|406|653|10|209|509|23|33.71|54.61|6.55|0.00|150.65|775.33|1256.03|10.54|0.00|150.65|161.19|-624.68| +2452231|35524|9897|82312|338159|406|653|10|148|509|1|3.99|6.22|3.11|0.00|3.11|3.99|6.22|0.21|0.00|3.11|3.32|-0.88| +2452231|35524|15589|82312|338159|406|653|10|108|509|84|67.28|127.83|0.00|0.00|0.00|5651.52|10737.72|0.00|0.00|0.00|0.00|-5651.52| +2452231|35524|1145|82312|338159|406|653|10|267|509|11|25.76|29.62|15.99|0.00|175.89|283.36|325.82|8.79|0.00|175.89|184.68|-107.47| +2452231|35524|7969|82312|338159|406|653|10|235|509|84|44.56|73.07|67.95|0.00|5707.80|3743.04|6137.88|228.31|0.00|5707.80|5936.11|1964.76| +2452231|35524|8135|82312|338159|406|653|10|168|509|62|56.40|85.16|84.30|0.00|5226.60|3496.80|5279.92|0.00|0.00|5226.60|5226.60|1729.80| +2452231|35524|253|82312|338159|406|653|10|76|509|46|88.67|128.57|128.57|0.00|5914.22|4078.82|5914.22|473.13|0.00|5914.22|6387.35|1835.40| +2452231|35524|3509|82312|338159|406|653|10|199|509|82|23.73|33.45|8.02|0.00|657.64|1945.86|2742.90|52.61|0.00|657.64|710.25|-1288.22| +2452231|35524|14391|82312|338159|406|653|10|133|509|87|88.68|124.15|64.55|0.00|5615.85|7715.16|10801.05|112.31|0.00|5615.85|5728.16|-2099.31| +2452231|35524|4929|82312|338159|406|653|10|133|509|26|44.71|74.21|28.94|338.59|752.44|1162.46|1929.46|0.00|338.59|413.85|413.85|-748.61| +2452231|35524|5441|82312|338159|406|653|10|125|509|66|89.41|155.57|107.34|0.00|7084.44|5901.06|10267.62|637.59|0.00|7084.44|7722.03|1183.38| +2452231|35524|15019|82312|||653|10||509|12|83.00|99.60||||996.00||18.16||454.08||-541.92| +2451830|38038|15793|71370|880027|4684|8494|1|247|510|81|84.32|86.84|82.49|0.00|6681.69|6829.92|7034.04|200.45|0.00|6681.69|6882.14|-148.23| +2451830|38038|5657|71370|880027|4684|8494|1|17|510|20|37.51|38.26|15.30|0.00|306.00|750.20|765.20|3.06|0.00|306.00|309.06|-444.20| +2451830|38038|8918|71370|880027|4684|8494|1|236|510|32|97.56|117.07|38.63|0.00|1236.16|3121.92|3746.24|98.89|0.00|1236.16|1335.05|-1885.76| +2451830|38038|7166|71370|880027|4684|8494|1|162|510|46|93.74|132.17|118.95|0.00|5471.70|4312.04|6079.82|164.15|0.00|5471.70|5635.85|1159.66| +2451830|38038|11107|71370|880027|4684|8494|1|280|510|58|69.80|113.07|40.70|0.00|2360.60|4048.40|6558.06|94.42|0.00|2360.60|2455.02|-1687.80| +2451830|38038|7663|71370|880027|4684|8494|1|300|510|32|3.78|5.74|3.09|0.00|98.88|120.96|183.68|4.94|0.00|98.88|103.82|-22.08| +2451830|38038|13574|71370|880027|4684|8494|1|130|510|30|26.89|36.30|3.63|46.82|108.90|806.70|1089.00|4.96|46.82|62.08|67.04|-744.62| +2451830|38038|10481|71370|880027|4684|8494|1|57|510|34|53.72|92.39|49.89|0.00|1696.26|1826.48|3141.26|67.85|0.00|1696.26|1764.11|-130.22| +2451830|38038|10849|71370|880027|4684|8494|1|131|510|82|98.23|194.49|70.01|0.00|5740.82|8054.86|15948.18|0.00|0.00|5740.82|5740.82|-2314.04| +2451582|70566|12200|74922|309535|4595|28855|4|48|511|86|99.78|155.65|49.80|0.00|4282.80|8581.08|13385.90|42.82|0.00|4282.80|4325.62|-4298.28| +2451582|70566|12422|74922|309535|4595|28855|4|261|511|44|4.24|6.27|1.19|19.37|52.36|186.56|275.88|0.32|19.37|32.99|33.31|-153.57| +2451582|70566|2101|74922|309535|4595|28855|4|151|511|98|44.83|80.24|70.61|4982.24|6919.78|4393.34|7863.52|135.62|4982.24|1937.54|2073.16|-2455.80| +2451582|70566|15941|74922|309535|4595|28855|4|287|511|31|71.47|132.93|10.63|0.00|329.53|2215.57|4120.83|16.47|0.00|329.53|346.00|-1886.04| +|70566|6584|74922|309535||28855|4||511|97||8.08|3.95||383.15|443.29|783.76|||383.15||| +||10627|||4595||4||511|12||||0.00|120.96||||0.00|120.96|130.63|| +||4088|74922||4595|28855|4|111|511|50||93.78||||||236.32|||3190.32|| +2451582|70566|6062|74922|309535|4595|28855|4|87|511|85|14.11|15.09|9.50|0.00|807.50|1199.35|1282.65|0.00|0.00|807.50|807.50|-391.85| +2451582|70566|5623|74922|309535|4595|28855|4|209|511|92|61.25|115.76|70.61|0.00|6496.12|5635.00|10649.92|194.88|0.00|6496.12|6691.00|861.12| +2451582|70566|15989|74922|309535|4595|28855|4|11|511|100|41.22|72.95|48.87|0.00|4887.00|4122.00|7295.00|146.61|0.00|4887.00|5033.61|765.00| +2451582|70566|7327|74922|309535|4595|28855|4|118|511|51|76.79|89.84|3.59|0.00|183.09|3916.29|4581.84|0.00|0.00|183.09|183.09|-3733.20| +2451582|70566|8360|74922|309535|4595|28855|4|91|511|19|73.79|116.58|109.58|0.00|2082.02|1402.01|2215.02|124.92|0.00|2082.02|2206.94|680.01| +2451582|70566|11681|74922|309535|4595|28855|4|82|511|97|46.10|52.09|33.33|0.00|3233.01|4471.70|5052.73|226.31|0.00|3233.01|3459.32|-1238.69| +2452518|40190|3672|29022|31503|384|23712|8|38|512|12|98.56|162.62|160.99|0.00|1931.88|1182.72|1951.44|96.59|0.00|1931.88|2028.47|749.16| +2452518|40190|3390|29022|31503|384|23712|8|108|512|53|74.54|98.39|74.77|0.00|3962.81|3950.62|5214.67|118.88|0.00|3962.81|4081.69|12.19| +2452518|40190|12925|29022|31503|384|23712|8|143|512|51|58.26|79.81|16.76|0.00|854.76|2971.26|4070.31|42.73|0.00|854.76|897.49|-2116.50| +2452518|40190|5857|29022|31503|384|23712|8|152|512|36|21.71|41.90|2.51|3.61|90.36|781.56|1508.40|1.73|3.61|86.75|88.48|-694.81| +2452518|40190|16200|29022|31503|384|23712|8|226|512|56|39.05|75.36|39.18|0.00|2194.08|2186.80|4220.16|131.64|0.00|2194.08|2325.72|7.28| +2452518|40190|4993|29022|31503|384|23712|8|60|512|54|89.94|168.18|146.31|0.00|7900.74|4856.76|9081.72|632.05|0.00|7900.74|8532.79|3043.98| +2452518|40190|4767|29022|31503|384|23712|8|228|512|70|84.06|139.53|136.73|861.39|9571.10|5884.20|9767.10|783.87|861.39|8709.71|9493.58|2825.51| +2452518||16093||31503|||8||512|||104.60|67.99|3131.61||8476.92||97.78|3131.61|3259.45|3357.23|| +2452518|40190|7051|29022|31503|384|23712|8|240|512|27|90.72|96.16|73.08|0.00|1973.16|2449.44|2596.32|78.92|0.00|1973.16|2052.08|-476.28| +2452518|40190|12147|29022|31503|384|23712|8|46|512|48|72.02|112.35|105.60|0.00|5068.80|3456.96|5392.80|456.19|0.00|5068.80|5524.99|1611.84| +||13357||31503|384||8||512|83||104.67|16.74||||8687.61|||||-6165.24| +2452518|40190|9282|29022|31503|384|23712|8|151|512|14|57.67|58.24|23.29|221.72|326.06|807.38|815.36|3.13|221.72|104.34|107.47|-703.04| +2452518|40190|4164|29022|31503|384|23712|8|244|512|50|86.72|150.02|106.51|5059.22|5325.50|4336.00|7501.00|21.30|5059.22|266.28|287.58|-4069.72| +2452518|40190|16185|29022|31503|384|23712|8|174|512|52|44.17|65.37|26.14|0.00|1359.28|2296.84|3399.24|0.00|0.00|1359.28|1359.28|-937.56| +2452518|40190|13830|29022|31503|384|23712|8|76|512|57|83.80|138.27|37.33|0.00|2127.81|4776.60|7881.39|106.39|0.00|2127.81|2234.20|-2648.79| +2452518|40190|11901|29022|31503|384|23712|8|215|512|39|89.88|154.59|111.30|0.00|4340.70|3505.32|6029.01|43.40|0.00|4340.70|4384.10|835.38| +2451561|58650|7849|93484|517629|2221|5166|8|248|513|70|64.20|107.85|19.41|0.00|1358.70|4494.00|7549.50|108.69|0.00|1358.70|1467.39|-3135.30| +2451561|58650|7910|93484||||8||513||||56.96|0.00||3750.66|5963.00||0.00|3816.32|4083.46|| +2451561|58650|12277|93484|517629|2221|5166|8|174|513|58|54.24|91.12|19.13|0.00|1109.54|3145.92|5284.96|77.66|0.00|1109.54|1187.20|-2036.38| +2451561|58650|14551|93484|517629|2221|5166|8|54|513|27|82.20|129.05|54.20|0.00|1463.40|2219.40|3484.35|117.07|0.00|1463.40|1580.47|-756.00| +2451561|58650|16358|93484|517629|2221|5166|8|77|513|81|91.18|133.12|34.61|0.00|2803.41|7385.58|10782.72|140.17|0.00|2803.41|2943.58|-4582.17| +2451561||13027||517629||||54|513|39||71.24||608.40||||42.58|608.40||1462.18|-1278.03| +|58650|3325|93484|517629|||8||513|56|||||4434.64|4576.88||||||-3290.83| +2451561|58650|5636|93484|517629|2221|5166|8|36|513|56|22.18|29.49|26.83|0.00|1502.48|1242.08|1651.44|45.07|0.00|1502.48|1547.55|260.40| +2451561|58650|8759|||2221|5166|8||513|15||140.22|11.21|||||6.72||||-1000.35| +2451561|58650|1367|93484|517629|2221|5166|8|80|513|42|43.21|48.39|6.29|129.44|264.18|1814.82|2032.38|12.12|129.44|134.74|146.86|-1680.08| +2451561|58650|17096|93484|517629|2221|5166|8|60|513|41|42.21|66.69|23.34|0.00|956.94|1730.61|2734.29|9.56|0.00|956.94|966.50|-773.67| +2451561|58650|14963|93484|517629|2221|5166|8|120|513|8|81.32|95.95|3.83|0.00|30.64|650.56|767.60|2.75|0.00|30.64|33.39|-619.92| +2451561|58650|113|93484|517629|2221|5166|8|43|513|71|61.71|112.92|13.55|0.00|962.05|4381.41|8017.32|86.58|0.00|962.05|1048.63|-3419.36| +2451561|58650|8665|93484|517629|2221|5166|8|282|513|88|61.88|105.81|44.44|0.00|3910.72|5445.44|9311.28|117.32|0.00|3910.72|4028.04|-1534.72| +2451712|53372|1177|27928|1505357|6786|37015|10|1|514|53|50.27|81.43|48.85|0.00|2589.05|2664.31|4315.79|181.23|0.00|2589.05|2770.28|-75.26| +2451712|53372|12841|27928|1505357|6786|37015|10|74|514|15|56.53|95.53|91.70|192.57|1375.50|847.95|1432.95|0.00|192.57|1182.93|1182.93|334.98| +2451712|53372|4175|27928|1505357|6786|37015|10|146|514|39|2.86|4.51|0.58|22.39|22.62|111.54|175.89|0.01|22.39|0.23|0.24|-111.31| +2451712|53372|3799|27928|1505357|6786|37015|10|197|514|64|21.10|31.22|20.91|0.00|1338.24|1350.40|1998.08|26.76|0.00|1338.24|1365.00|-12.16| +2451712|53372|2569|27928|1505357|6786|37015|10|21|514|19|87.82|147.53|38.35|0.00|728.65|1668.58|2803.07|58.29|0.00|728.65|786.94|-939.93| +2451712|53372|4613|27928|1505357|6786|37015|10|280|514|85|34.62|62.31|16.20|1142.91|1377.00|2942.70|5296.35|11.70|1142.91|234.09|245.79|-2708.61| +2451712|53372|6176|27928|1505357|6786|37015|10|199|514|62|44.09|50.70|48.67|0.00|3017.54|2733.58|3143.40|60.35|0.00|3017.54|3077.89|283.96| +2451712|53372|10853|27928|1505357|6786|37015|10|200|514|93|32.43|64.21|14.12|0.00|1313.16|3015.99|5971.53|118.18|0.00|1313.16|1431.34|-1702.83| +2451712|53372|6212|27928|1505357|6786|37015|10|216|514|16|77.41|113.01|25.99|0.00|415.84|1238.56|1808.16|29.10|0.00|415.84|444.94|-822.72| +2451712|53372|7970|27928|1505357|6786|37015|10|117|514|27|25.67|29.52|21.54|0.00|581.58|693.09|797.04|17.44|0.00|581.58|599.02|-111.51| +2451712|53372|9181|27928|1505357|6786|37015|10|206|514|73|49.82|96.15|61.53|0.00|4491.69|3636.86|7018.95|179.66|0.00|4491.69|4671.35|854.83| +2451712|53372|6757|27928|1505357|6786|37015|10|228|514|48|75.65|114.23|33.12|429.23|1589.76|3631.20|5483.04|58.02|429.23|1160.53|1218.55|-2470.67| +2451712|53372|344|27928|1505357|6786|37015|10|263|514|69|23.47|30.74|17.52|1124.25|1208.88|1619.43|2121.06|3.38|1124.25|84.63|88.01|-1534.80| +2451712|53372|4009|27928|1505357|6786|37015|10|162|514|53|74.14|121.58|64.43|1058.58|3414.79|3929.42|6443.74|70.68|1058.58|2356.21|2426.89|-1573.21| +2451712|53372|7172|27928|1505357|6786|37015|10|218|514|54|66.34|79.60|78.00|0.00|4212.00|3582.36|4298.40|379.08|0.00|4212.00|4591.08|629.64| +2451712|53372|4055|27928|1505357|6786|37015|10|147|514|71|57.43|63.17|7.58|0.00|538.18|4077.53|4485.07|48.43|0.00|538.18|586.61|-3539.35| +2452197|70305|2763|43346|704014|682|16950|1|153|515|3|68.03|113.61|72.71|0.00|218.13|204.09|340.83|6.54|0.00|218.13|224.67|14.04| +2452197|70305|6101|43346|704014|682|16950|1|54|515|63|53.45|95.14|92.28|0.00|5813.64|3367.35|5993.82|465.09|0.00|5813.64|6278.73|2446.29| +2452197|70305|12889|43346|704014|682|16950|1|218|515|86|38.16|53.42|10.68|339.83|918.48|3281.76|4594.12|40.50|339.83|578.65|619.15|-2703.11| +2452197|70305|5947|43346|704014|682|16950|1|130|515|57|97.49|154.03|77.01|0.00|4389.57|5556.93|8779.71|351.16|0.00|4389.57|4740.73|-1167.36| +2452197|70305|11049|43346|704014|682|16950|1|106|515|13|58.92|63.63|6.99|0.00|90.87|765.96|827.19|7.26|0.00|90.87|98.13|-675.09| +2452197|70305|8127|43346|704014|682|16950|1|125|515|72|21.11|39.47|22.89|1186.61|1648.08|1519.92|2841.84|13.84|1186.61|461.47|475.31|-1058.45| +2452197|70305|9527|43346|704014|682|16950|1|258|515|92|85.00|134.30|68.49|0.00|6301.08|7820.00|12355.60|63.01|0.00|6301.08|6364.09|-1518.92| +2452197|70305|17583|43346|704014|682|16950|1|241|515|41|37.58|38.33|13.03|0.00|534.23|1540.78|1571.53|37.39|0.00|534.23|571.62|-1006.55| +2452197|70305|12327|43346|704014|682|16950|1|94|515|34|71.21|76.19|3.04|0.00|103.36|2421.14|2590.46|6.20|0.00|103.36|109.56|-2317.78| +2452197|70305|16063|43346|704014|682|16950|1|210|515|87|14.26|25.38|6.09|0.00|529.83|1240.62|2208.06|10.59|0.00|529.83|540.42|-710.79| +2452197|70305|1507|43346|704014|682|16950|1|57|515|94|52.81|58.09|49.37|928.15|4640.78|4964.14|5460.46|111.37|928.15|3712.63|3824.00|-1251.51| +2452197|70305|1509|43346|704014|682|16950|1|103|515|88|25.74|35.52|17.76|0.00|1562.88|2265.12|3125.76|109.40|0.00|1562.88|1672.28|-702.24| +2452197|70305|2689|43346|704014|682|16950|1|141|515|17|7.98|14.28|4.85|0.00|82.45|135.66|242.76|2.47|0.00|82.45|84.92|-53.21| +2452623|71640|8301|47112|1448131|703|15275|8|36|516|66|5.53|9.78|2.24|76.87|147.84|364.98|645.48|2.12|76.87|70.97|73.09|-294.01| +2452623|71640|13092|47112|1448131|703|15275|8|271|516|91|93.93|105.20|35.76|0.00|3254.16|8547.63|9573.20|292.87|0.00|3254.16|3547.03|-5293.47| +2452623|71640|1663|47112|1448131|703|15275|8|271|516|98|65.63|91.22|74.80|3152.07|7330.40|6431.74|8939.56|292.48|3152.07|4178.33|4470.81|-2253.41| +2452623|71640|15963|47112|1448131|703|15275|8|249|516|11|4.02|7.03|2.31|13.21|25.41|44.22|77.33|0.24|13.21|12.20|12.44|-32.02| +2452623|71640|9949|47112|1448131|703|15275|8|265|516|40|26.60|34.58|4.49|0.00|179.60|1064.00|1383.20|5.38|0.00|179.60|184.98|-884.40| +2452623|71640|6945|47112|1448131|703|15275|8|110|516|24|17.04|33.39|8.34|0.00|200.16|408.96|801.36|2.00|0.00|200.16|202.16|-208.80| +2452623|71640|14142|47112|1448131|703|15275|8|113|516|4|74.09|130.39|125.17|0.00|500.68|296.36|521.56|0.00|0.00|500.68|500.68|204.32| +2452623||13488|47112||703|||72|516|||||994.17||5753.88|11277.09||994.17|583.89||| +2452623|71640|17520|47112|1448131|703|15275|8|254|516|27|86.77|142.30|69.72|0.00|1882.44|2342.79|3842.10|0.00|0.00|1882.44|1882.44|-460.35| +2452623||7111||1448131||||1|516|||2.41||0.00|194.74|166.53|219.31|5.84|0.00|||28.21| +2452623|71640|12210|47112|1448131|703|15275|8|294|516|36|44.17|63.60|50.24|0.00|1808.64|1590.12|2289.60|72.34|0.00|1808.64|1880.98|218.52| +2452623|71640|14023|47112|1448131|703|15275|8|165|516|38|83.74|136.49|95.54|0.00|3630.52|3182.12|5186.62|145.22|0.00|3630.52|3775.74|448.40| +2452623|71640|2899|47112|1448131|703|15275|8|184|516|6|95.82|138.93|50.01|99.01|300.06|574.92|833.58|12.06|99.01|201.05|213.11|-373.87| +2452623|71640|15069|47112|1448131|703|15275|8|121|516|45|9.11|16.30|2.60|93.60|117.00|409.95|733.50|0.23|93.60|23.40|23.63|-386.55| +2451463|46967|8830|61845|33754|1663|26772|4|220|517|27|56.99|63.25|46.17|0.00|1246.59|1538.73|1707.75|37.39|0.00|1246.59|1283.98|-292.14| +2451463|46967|16|61845|33754|1663|26772|4|235|517|90|94.37|134.94|20.24|0.00|1821.60|8493.30|12144.60|54.64|0.00|1821.60|1876.24|-6671.70| +2451463|46967|11233|61845|33754|1663|26772|4|120|517|24|13.29|23.12|3.46|0.00|83.04|318.96|554.88|6.64|0.00|83.04|89.68|-235.92| +2451463|46967|15976|61845|33754|1663|26772|4|14|517|30|51.83|55.45|33.27|0.00|998.10|1554.90|1663.50|9.98|0.00|998.10|1008.08|-556.80| +2451463|46967|11744|61845|33754|1663|26772|4|241|517|99|88.60|136.44|84.59|5527.11|8374.41|8771.40|13507.56|28.47|5527.11|2847.30|2875.77|-5924.10| +2451463|46967|11746|61845|33754|1663|26772|4|56|517|17|5.06|6.83|3.96|0.00|67.32|86.02|116.11|2.01|0.00|67.32|69.33|-18.70| +2451463|46967|11044|61845|33754|1663|26772|4|208|517|51|83.82|113.15|21.49|0.00|1095.99|4274.82|5770.65|65.75|0.00|1095.99|1161.74|-3178.83| +2451463|46967|9160|61845|33754|1663|26772|4|40|517|12|89.97|137.65|118.37|0.00|1420.44|1079.64|1651.80|113.63|0.00|1420.44|1534.07|340.80| +2451463|46967|4868|61845|33754|1663|26772|4|16|517|83|53.75|64.50|20.64|0.00|1713.12|4461.25|5353.50|154.18|0.00|1713.12|1867.30|-2748.13| +2451403|35225|5504|46327|427594|3708|14575|4|52|518|88|8.93|14.19|8.23|369.36|724.24|785.84|1248.72|3.54|369.36|354.88|358.42|-430.96| +2451403|35225|12211|46327|427594|3708|14575|4|237|518|59|49.94|94.38|73.61|2779.51|4342.99|2946.46|5568.42|93.80|2779.51|1563.48|1657.28|-1382.98| +2451403|35225|3481|46327|427594|3708|14575|4|217|518|92|8.10|14.66|7.47|0.00|687.24|745.20|1348.72|54.97|0.00|687.24|742.21|-57.96| +2451403|35225|3367|46327|427594|3708|14575|4|256|518|81|81.32|84.57|64.27|0.00|5205.87|6586.92|6850.17|312.35|0.00|5205.87|5518.22|-1381.05| +2451403|35225|5696|46327|427594|3708|14575|4|268|518|53|6.56|10.29|6.37|91.15|337.61|347.68|545.37|22.18|91.15|246.46|268.64|-101.22| +2451403|35225|6218|46327|427594|3708|14575|4|275|518|3|69.16|132.78|17.26|4.14|51.78|207.48|398.34|4.28|4.14|47.64|51.92|-159.84| +2451403|35225|3241|46327|427594|3708|14575|4|259|518|67|98.95|115.77|54.41|0.00|3645.47|6629.65|7756.59|218.72|0.00|3645.47|3864.19|-2984.18| +2451403|35225|8|46327|427594|3708|14575|4|103|518|49|49.53|80.73|62.16|0.00|3045.84|2426.97|3955.77|121.83|0.00|3045.84|3167.67|618.87| +2451403|35225|2030|46327|427594|3708|14575|4|88|518|21|28.44|30.99|22.31|342.01|468.51|597.24|650.79|10.12|342.01|126.50|136.62|-470.74| +2451403|35225|12212|46327|427594|3708|14575|4|206|518|72|13.52|14.06|6.04|0.00|434.88|973.44|1012.32|39.13|0.00|434.88|474.01|-538.56| +2451403|35225|7645|46327|427594|3708|14575|4|226|518|32|27.33|53.84|24.76|0.00|792.32|874.56|1722.88|55.46|0.00|792.32|847.78|-82.24| +2451403|35225|6428|46327|427594|3708|14575|4|216|518|83|80.80|150.28|37.57|0.00|3118.31|6706.40|12473.24|249.46|0.00|3118.31|3367.77|-3588.09| +2451403|35225|952|46327|427594|3708|14575|4|87|518|6|79.75|92.51|7.40|0.00|44.40|478.50|555.06|2.66|0.00|44.40|47.06|-434.10| +2451403|35225|10897|46327|427594|3708|14575|4|68|518|90|27.71|44.89|8.52|575.10|766.80|2493.90|4040.10|11.50|575.10|191.70|203.20|-2302.20| +2451403|35225|8594|46327|427594|3708|14575|4|56|518|24|61.35|95.09|68.46|0.00|1643.04|1472.40|2282.16|131.44|0.00|1643.04|1774.48|170.64| +2451403|35225|5096|46327|427594|3708|14575|4|56|518|73|64.95|77.94|73.26|0.00|5347.98|4741.35|5689.62|320.87|0.00|5347.98|5668.85|606.63| +2451862|45730|569|9996|218584|3172|48113|1|22|519|16|22.13|25.00|11.25|10.80|180.00|354.08|400.00|15.22|10.80|169.20|184.42|-184.88| +2451862|45730|12998|9996|218584|3172|48113|1|144|519|75|8.30|15.10|1.81|0.00|135.75|622.50|1132.50|4.07|0.00|135.75|139.82|-486.75| +2451862|45730|11051|9996|218584|3172|48113|1|259|519|65|92.73|102.00|25.50|0.00|1657.50|6027.45|6630.00|99.45|0.00|1657.50|1756.95|-4369.95| +2451862|45730|2309|9996|218584|3172|48113|1|270|519|93|15.72|24.05|4.32|0.00|401.76|1461.96|2236.65|24.10|0.00|401.76|425.86|-1060.20| +2451862|45730|7142|9996|218584|3172|48113|1|17|519|12|83.70|107.13|71.77|0.00|861.24|1004.40|1285.56|34.44|0.00|861.24|895.68|-143.16| +2451862|45730|14969|9996|218584|3172|48113|1|150|519|87|10.14|15.41|4.93|351.70|428.91|882.18|1340.67|4.63|351.70|77.21|81.84|-804.97| +2451862|45730|7597|9996|218584|3172|48113|1|165|519|41|14.99|17.23|3.10|106.76|127.10|614.59|706.43|0.81|106.76|20.34|21.15|-594.25| +2451862||4838|9996||3172||1|300|519||||42.24|0.00||1495.53|1914.00|12.24|0.00|1224.96||-270.57| +2451862|45730|1190|9996|218584|3172|48113|1|100|519|8|45.36|75.75|72.72|0.00|581.76|362.88|606.00|29.08|0.00|581.76|610.84|218.88| +2451862|45730|15407|9996|218584|3172|48113|1|19|519|39|85.50|114.57|19.47|0.00|759.33|3334.50|4468.23|53.15|0.00|759.33|812.48|-2575.17| +2451862|45730|6161|9996|218584|3172|48113|1|246|519|69|98.02|142.12|85.27|0.00|5883.63|6763.38|9806.28|0.00|0.00|5883.63|5883.63|-879.75| +2451613|67975|11021|25103|704976|6449|37122|2|235|520|20|74.93|143.11|32.91|0.00|658.20|1498.60|2862.20|59.23|0.00|658.20|717.43|-840.40| +2451613|67975|8815|25103|704976|6449|37122|2|100|520|5|29.84|38.79|13.57|0.00|67.85|149.20|193.95|4.74|0.00|67.85|72.59|-81.35| +2451613|67975|13082|25103|704976|6449|37122|2|24|520|76|94.23|172.44|93.11|0.00|7076.36|7161.48|13105.44|566.10|0.00|7076.36|7642.46|-85.12| +2451613|67975|5459|25103|704976|6449|37122|2|190|520|43|3.83|4.05|3.56|0.00|153.08|164.69|174.15|3.06|0.00|153.08|156.14|-11.61| +2451613|67975|5221|25103|704976|6449|37122|2|283|520|13|90.76|98.02|79.39|0.00|1032.07|1179.88|1274.26|72.24|0.00|1032.07|1104.31|-147.81| +2451613|67975|2021|25103|704976|6449|37122|2|32|520|43|74.92|105.63|41.19|0.00|1771.17|3221.56|4542.09|35.42|0.00|1771.17|1806.59|-1450.39| +2451613|67975|1556|25103|704976|6449|37122|2|47|520|99|60.53|95.63|86.06|0.00|8519.94|5992.47|9467.37|681.59|0.00|8519.94|9201.53|2527.47| +2451613|67975|17233|25103|704976|6449|37122|2|14|520|16|55.80|89.28|24.99|79.96|399.84|892.80|1428.48|28.78|79.96|319.88|348.66|-572.92| +2451613|67975|2131|25103|704976|6449|37122|2|265|520|24|64.41|120.44|77.08|0.00|1849.92|1545.84|2890.56|110.99|0.00|1849.92|1960.91|304.08| +2451613|67975|17515|25103|704976|6449|37122|2|164|520|22|53.48|60.43|58.61|657.60|1289.42|1176.56|1329.46|44.22|657.60|631.82|676.04|-544.74| +2452248|61661|13741|31373|862249|1478|11554|10|262|521|48|6.97|7.94|4.92|0.00|236.16|334.56|381.12|2.36|0.00|236.16|238.52|-98.40| +2452248|61661|15713|31373|862249|1478|11554|10|254|521|12|23.90|33.46|20.41|0.00|244.92|286.80|401.52|14.69|0.00|244.92|259.61|-41.88| +2452248|61661|16351|31373|862249|1478|11554|10|140|521|11|17.73|25.17|10.31|95.26|113.41|195.03|276.87|1.08|95.26|18.15|19.23|-176.88| +2452248|61661|11849|31373|862249|1478|11554|10|247|521|71|33.50|66.33|8.62|0.00|612.02|2378.50|4709.43|6.12|0.00|612.02|618.14|-1766.48| +2452248|61661|11505|31373|862249|1478|11554|10|261|521|46|95.98|173.72|27.79|0.00|1278.34|4415.08|7991.12|76.70|0.00|1278.34|1355.04|-3136.74| +2452248|61661|6843|31373|862249|1478|11554|10|102|521|23|41.40|48.02|7.68|0.00|176.64|952.20|1104.46|8.83|0.00|176.64|185.47|-775.56| +2452248|61661|4935|31373|862249|1478|11554|10|193|521|59|48.64|72.47|70.29|0.00|4147.11|2869.76|4275.73|0.00|0.00|4147.11|4147.11|1277.35| +2452248|61661|9083|31373|862249|1478|11554|10|125|521|38|59.39|101.55|42.65|0.00|1620.70|2256.82|3858.90|16.20|0.00|1620.70|1636.90|-636.12| +2452277|56561|15573|21404|1891667|7083|49346|7|89|522|52|36.86|56.76|22.70|0.00|1180.40|1916.72|2951.52|23.60|0.00|1180.40|1204.00|-736.32| +2452277|56561|11085|21404|1891667|7083|49346|7|154|522|44|17.37|18.58|4.83|108.38|212.52|764.28|817.52|3.12|108.38|104.14|107.26|-660.14| +2452277|56561|14526|21404|1891667|7083|49346|7|19|522|2|87.01|112.24|25.81|0.00|51.62|174.02|224.48|3.09|0.00|51.62|54.71|-122.40| +2452277|56561|4110|21404|1891667|7083|49346|7|39|522|45|44.67|75.93|63.78|0.00|2870.10|2010.15|3416.85|258.30|0.00|2870.10|3128.40|859.95| +2452277|56561|12378|21404|1891667|7083|49346|7|170|522|55|76.85|86.07|40.45|0.00|2224.75|4226.75|4733.85|177.98|0.00|2224.75|2402.73|-2002.00| +2452277|56561|13236|21404|1891667|7083|49346|7|154|522|27|47.66|51.47|33.97|0.00|917.19|1286.82|1389.69|36.68|0.00|917.19|953.87|-369.63| +2452277|56561|8574|21404|1891667|7083|49346|7|27|522|45|60.24|82.52|48.68|21.90|2190.60|2710.80|3713.40|151.80|21.90|2168.70|2320.50|-542.10| +2452277|56561|7621|21404|1891667|7083|49346|7|148|522|54|95.66|135.83|29.88|0.00|1613.52|5165.64|7334.82|0.00|0.00|1613.52|1613.52|-3552.12| +2452277|56561|10999|21404|1891667|7083|49346|7|248|522|46|39.09|77.78|35.00|0.00|1610.00|1798.14|3577.88|80.50|0.00|1610.00|1690.50|-188.14| +2452277|56561|7551|21404|1891667|7083|49346|7|77|522|7|26.89|34.41|32.68|0.00|228.76|188.23|240.87|6.86|0.00|228.76|235.62|40.53| +2452277|56561|6481|21404|1891667|7083|49346|7|208|522|29|1.25|2.06|1.11|0.00|32.19|36.25|59.74|1.60|0.00|32.19|33.79|-4.06| +2452277|56561|8077|21404|1891667|7083|49346|7|282|522|95|5.50|7.70|7.54|0.00|716.30|522.50|731.50|21.48|0.00|716.30|737.78|193.80| +2452584|58065|12696|27261|1521294|6527|36160|1|9|523|43|26.60|38.30|24.89|0.00|1070.27|1143.80|1646.90|0.00|0.00|1070.27|1070.27|-73.53| +2452584|58065|16171|27261|1521294|6527|36160|1|293|523|92|22.79|33.04|20.81|0.00|1914.52|2096.68|3039.68|57.43|0.00|1914.52|1971.95|-182.16| +2452584|58065|9984|27261|1521294|6527|36160|1|275|523|32|26.54|34.23|25.67|106.78|821.44|849.28|1095.36|50.02|106.78|714.66|764.68|-134.62| +2452584|58065|6361|27261|1521294|6527|36160|1|183|523|52|6.86|7.61|2.51|0.00|130.52|356.72|395.72|11.74|0.00|130.52|142.26|-226.20| +2452584|58065|5961|27261|1521294|6527|36160|1|173|523|61|24.59|34.91|17.45|0.00|1064.45|1499.99|2129.51|31.93|0.00|1064.45|1096.38|-435.54| +2452584|58065|12325|27261|1521294|6527|36160|1|224|523|89|27.07|43.58|30.50|1954.44|2714.50|2409.23|3878.62|7.60|1954.44|760.06|767.66|-1649.17| +2452584|58065|7555|27261|1521294|6527|36160|1|164|523|79|80.46|153.67|12.29|0.00|970.91|6356.34|12139.93|77.67|0.00|970.91|1048.58|-5385.43| +2452584|58065|1365||1521294|6527|36160|||523|75||56.48|41.79|0.00|3134.25|3959.25|||0.00||3228.27|| +2452584|58065|15007|27261|1521294|6527|36160|1|299|523|85|54.84|64.71|15.53|0.00|1320.05|4661.40|5500.35|39.60|0.00|1320.05|1359.65|-3341.35| +2452584|58065|15205||||36160||211|523|||122.46||7083.92||7642.64||100.00|7083.92||1350.11|| +2452584|58065|9384|27261|1521294|6527|36160|1|11|523|87|98.46|170.33|109.01|2845.16|9483.87|8566.02|14818.71|66.38|2845.16|6638.71|6705.09|-1927.31| +2452584|58065|3426|27261|1521294|6527|36160|1|95|523|39|65.93|123.28|81.36|1618.25|3173.04|2571.27|4807.92|124.38|1618.25|1554.79|1679.17|-1016.48| +2452584|58065|16572|27261|1521294|6527|36160|1|291|523|4|99.04|154.50|49.44|0.00|197.76|396.16|618.00|3.95|0.00|197.76|201.71|-198.40| +2452584|58065|12951|27261|1521294|6527|36160|1|112|523|25|37.26|46.57|6.98|0.00|174.50|931.50|1164.25|1.74|0.00|174.50|176.24|-757.00| +2452584|58065|13059|27261|1521294|6527|36160|1|266|523|48|7.32|12.22|1.71|0.00|82.08|351.36|586.56|2.46|0.00|82.08|84.54|-269.28| +2452584|58065|5281|27261|1521294|6527|36160|1|47|523|58|3.62|6.11|2.93|0.00|169.94|209.96|354.38|3.39|0.00|169.94|173.33|-40.02| +2451520|36675|1442|78322|1565973|5624|6475|7|39|524|31|11.69|18.58|1.30|34.25|40.30|362.39|575.98|0.06|34.25|6.05|6.11|-356.34| +2451520|36675|6286|78322|1565973|5624|6475|7|57|524|24|90.75|114.34|93.75|0.00|2250.00|2178.00|2744.16|112.50|0.00|2250.00|2362.50|72.00| +2451520|36675|5254|78322|1565973|5624|6475|7|201|524|99|82.98|152.68|35.11|0.00|3475.89|8215.02|15115.32|243.31|0.00|3475.89|3719.20|-4739.13| +2451520|36675|13261|78322|1565973|5624|6475|7|163|524|71|29.64|37.64|36.13|0.00|2565.23|2104.44|2672.44|25.65|0.00|2565.23|2590.88|460.79| +2451520|36675|1420|78322|1565973|5624|6475|7|105|524|81|79.04|128.83|86.31|0.00|6991.11|6402.24|10435.23|629.19|0.00|6991.11|7620.30|588.87| +2451520|36675|8917|78322|1565973|5624|6475|7|179|524|12|98.18|171.81|25.77|0.00|309.24|1178.16|2061.72|15.46|0.00|309.24|324.70|-868.92| +2451520|36675|3622|78322|1565973|5624|6475|7|197|524|45|23.75|46.78|0.93|29.71|41.85|1068.75|2105.10|0.72|29.71|12.14|12.86|-1056.61| +2451520|36675|12614|78322|1565973|5624|6475|7|276|524|9|86.89|152.92|9.17|0.00|82.53|782.01|1376.28|1.65|0.00|82.53|84.18|-699.48| +2451520|36675|11090|78322|1565973|5624|6475|7|178|524|8|98.74|152.05|138.36|0.00|1106.88|789.92|1216.40|44.27|0.00|1106.88|1151.15|316.96| +2451520|36675|14629|78322|1565973|5624|6475|7|244|524|18|53.12|74.36|28.25|0.00|508.50|956.16|1338.48|35.59|0.00|508.50|544.09|-447.66| +2451520|36675|8956|78322|1565973|5624|6475|7|48|524|57|55.52|58.85|47.66|0.00|2716.62|3164.64|3354.45|217.32|0.00|2716.62|2933.94|-448.02| +2451520|36675|6901|78322|1565973|5624|6475|7|2|524|83|70.03|72.13|44.72|2709.58|3711.76|5812.49|5986.79|40.08|2709.58|1002.18|1042.26|-4810.31| +2451520|36675|12118|78322|1565973|5624|6475|7|254|524|90|73.37|78.50|13.34|0.00|1200.60|6603.30|7065.00|36.01|0.00|1200.60|1236.61|-5402.70| +2452328|49458|7803|21019|1854085|5198|25064|7|97|525|88|70.47|84.56|48.19|0.00|4240.72|6201.36|7441.28|127.22|0.00|4240.72|4367.94|-1960.64| +2452328|49458|16899|21019|1854085|5198|25064|7|210|525|62|37.43|40.42|19.80|0.00|1227.60|2320.66|2506.04|73.65|0.00|1227.60|1301.25|-1093.06| +2452328|49458|6576|21019|1854085|5198|25064|7|80|525|39|11.55|22.17|15.96|0.00|622.44|450.45|864.63|24.89|0.00|622.44|647.33|171.99| +2452328||7395||1854085|||||525||69.66||10.53|0.00|||131.65|0.00|0.00|||-59.13| +2452328|49458|9337|21019|1854085|5198|25064|7|236|525|32|57.33|62.48|36.86|0.00|1179.52|1834.56|1999.36|35.38|0.00|1179.52|1214.90|-655.04| +2452328|49458|12039|21019|1854085|5198|25064|7|285|525|96|30.83|37.61|13.91|0.00|1335.36|2959.68|3610.56|66.76|0.00|1335.36|1402.12|-1624.32| +2452328|49458|16221|21019|1854085|5198|25064|7|53|525|97|28.70|39.89|23.93|0.00|2321.21|2783.90|3869.33|116.06|0.00|2321.21|2437.27|-462.69| +2452328|49458|17388|21019|1854085|5198|25064|7|177|525|17|29.23|49.39|2.96|0.00|50.32|496.91|839.63|1.50|0.00|50.32|51.82|-446.59| +2452328|49458|17127|21019|1854085|5198|25064|7|233|525|79|42.58|68.55|21.25|0.00|1678.75|3363.82|5415.45|67.15|0.00|1678.75|1745.90|-1685.07| +2452328|49458|11823|21019|1854085|5198|25064|7|71|525|3|92.71|103.83|31.14|0.00|93.42|278.13|311.49|8.40|0.00|93.42|101.82|-184.71| +2452328|49458|16173|21019|1854085|5198|25064|7|223|525|4|22.71|29.06|25.28|86.96|101.12|90.84|116.24|1.13|86.96|14.16|15.29|-76.68| +2452328||4590|21019|||25064|||525||78.56|139.83||||||||||| +2452328|49458|17445|21019|1854085|5198|25064|7|145|525|5|67.96|69.99|28.69|0.00|143.45|339.80|349.95|12.91|0.00|143.45|156.36|-196.35| +2452547|63817|1851|63178|1377181|1584|32787|2|138|526|67|20.50|29.31|2.34|0.00|156.78|1373.50|1963.77|0.00|0.00|156.78|156.78|-1216.72| +||3925|63178||1584|||240|526|90|57.64|95.68|84.19|6288.99||5187.60||0.00|6288.99|||| +2452547|63817|4047|63178|1377181|1584|32787|2|137|526|13|10.09|10.79|7.44|46.42|96.72|131.17|140.27|1.50|46.42|50.30|51.80|-80.87| +2452547|63817|1716|63178|1377181|1584|32787|2|103|526|88|85.44|144.39|140.05|6901.66|12324.40|7518.72|12706.32|325.36|6901.66|5422.74|5748.10|-2095.98| +2452547|63817|36|63178|1377181|1584|32787|2|39|526|66|62.73|107.26|98.67|0.00|6512.22|4140.18|7079.16|455.85|0.00|6512.22|6968.07|2372.04| +2452547|63817|4345|63178|1377181|1584|32787|2|130|526|43|80.48|93.35|31.73|504.82|1364.39|3460.64|4014.05|77.36|504.82|859.57|936.93|-2601.07| +2452547|63817|4068|63178|1377181|1584|32787|2|248|526|10|50.79|56.37|10.71|0.00|107.10|507.90|563.70|8.56|0.00|107.10|115.66|-400.80| +2452547|63817|4563|63178|1377181|1584|32787|2|285|526|91|24.74|27.21|20.67|0.00|1880.97|2251.34|2476.11|94.04|0.00|1880.97|1975.01|-370.37| +2452547|63817|10836|63178|1377181|1584|32787|2|263|526|46|17.39|27.12|13.56|0.00|623.76|799.94|1247.52|31.18|0.00|623.76|654.94|-176.18| +2452547|63817|17748|63178|1377181|1584|32787|2|124|526|70|50.07|91.62|21.07|0.00|1474.90|3504.90|6413.40|58.99|0.00|1474.90|1533.89|-2030.00| +2452547|63817|11629|63178|1377181|1584|32787|2|266|526|78|20.16|26.61|5.05|0.00|393.90|1572.48|2075.58|23.63|0.00|393.90|417.53|-1178.58| +2452547|63817|11079|63178|1377181|1584|32787|2|105|526|7|6.68|11.75|8.93|0.00|62.51|46.76|82.25|4.37|0.00|62.51|66.88|15.75| +2452547|63817|2844|63178|1377181|1584|32787|2|179|526|82|84.10|163.15|13.05|0.00|1070.10|6896.20|13378.30|64.20|0.00|1070.10|1134.30|-5826.10| +2452547|63817|3672|63178|1377181|1584|32787|2|134|526|67|52.76|71.22|29.91|0.00|2003.97|3534.92|4771.74|100.19|0.00|2003.97|2104.16|-1530.95| +2452505|39265|5556|15366|435617|2268|47719|2|205|527|23|44.14|60.47|15.72|0.00|361.56|1015.22|1390.81|14.46|0.00|361.56|376.02|-653.66| +2452505|39265|595|15366|435617|2268|47719|2|229|527|37|51.51|101.47|21.30|299.47|788.10|1905.87|3754.39|0.00|299.47|488.63|488.63|-1417.24| +2452505|39265|17611|15366|435617|2268|47719|2|171|527|28|39.30|55.02|0.00|0.00|0.00|1100.40|1540.56|0.00|0.00|0.00|0.00|-1100.40| +2452505|39265|14383|15366|435617|2268|47719|2|160|527|27|28.69|42.46|14.43|7.79|389.61|774.63|1146.42|0.00|7.79|381.82|381.82|-392.81| +2452505|39265|3144|15366|435617|2268|47719|2|233|527|94|95.97|119.00|7.14|0.00|671.16|9021.18|11186.00|53.69|0.00|671.16|724.85|-8350.02| +2452505|39265|8838|15366|435617|2268|47719|2|190|527|52|60.04|107.47|35.46|0.00|1843.92|3122.08|5588.44|18.43|0.00|1843.92|1862.35|-1278.16| +2452505|39265|531|15366|435617|2268|47719|2|274|527|55|76.29|82.39|36.25|0.00|1993.75|4195.95|4531.45|119.62|0.00|1993.75|2113.37|-2202.20| +2452505|39265|3102|15366|435617|2268|47719|2|121|527|35|11.10|18.98|16.13|146.78|564.55|388.50|664.30|25.06|146.78|417.77|442.83|29.27| +2452505|39265|9073|15366|435617|2268|47719|2|223|527|59|22.69|43.56|34.84|0.00|2055.56|1338.71|2570.04|61.66|0.00|2055.56|2117.22|716.85| +2452505|39265|4605|15366|435617|2268|47719|2|22|527|29|56.13|85.87|52.38|0.00|1519.02|1627.77|2490.23|106.33|0.00|1519.02|1625.35|-108.75| +2452505|39265|15295|15366|435617|2268|47719|2|208|527|89|37.72|60.72|5.46|0.00|485.94|3357.08|5404.08|43.73|0.00|485.94|529.67|-2871.14| +2452533|61474|6486|76826|1007717|5808|43946|4|145|528|68|64.84|127.08|102.93|0.00|6999.24|4409.12|8641.44|139.98|0.00|6999.24|7139.22|2590.12| +2452533|61474|10638|76826|1007717|5808|43946|4|67|528|82|66.56|95.18|19.98|573.42|1638.36|5457.92|7804.76|42.59|573.42|1064.94|1107.53|-4392.98| +2452533|61474|14917|76826|1007717|5808|43946|4|21|528|21|82.23|136.50|99.64|1381.01|2092.44|1726.83|2866.50|7.11|1381.01|711.43|718.54|-1015.40| +2452533|61474|3597|76826|1007717|5808|43946|4|143|528|90|1.45|1.91|0.85|0.00|76.50|130.50|171.90|6.12|0.00|76.50|82.62|-54.00| +2452533|61474|16662|76826|1007717|5808|43946|4|201|528|23|21.22|32.25|24.51|0.00|563.73|488.06|741.75|0.00|0.00|563.73|563.73|75.67| +2452533|61474|2106|76826|1007717|5808|43946|4|110|528|56|51.91|97.07|6.79|228.14|380.24|2906.96|5435.92|6.08|228.14|152.10|158.18|-2754.86| +2452533|61474|14991|76826|1007717|5808|43946|4|285|528|49|68.51|96.59|83.06|0.00|4069.94|3356.99|4732.91|81.39|0.00|4069.94|4151.33|712.95| +2452533|61474|4771|76826|1007717|5808|43946|4|197|528|1|67.05|117.33|36.37|0.00|36.37|67.05|117.33|2.18|0.00|36.37|38.55|-30.68| +2452533|61474|3991|76826|1007717|5808|43946|4|142|528|9|98.09|159.88|158.28|1025.65|1424.52|882.81|1438.92|0.00|1025.65|398.87|398.87|-483.94| +2452533|61474|9186|76826|1007717|5808|43946|4|86|528|21|92.56|144.39|1.44|2.72|30.24|1943.76|3032.19|2.47|2.72|27.52|29.99|-1916.24| +2452533|61474|4296|76826|1007717|5808|43946|4|127|528|48|87.75|139.52|115.80|889.34|5558.40|4212.00|6696.96|233.45|889.34|4669.06|4902.51|457.06| +2452533|61474|16525|76826|1007717|5808|43946|4|97|528|27|57.13|86.26|46.58|0.00|1257.66|1542.51|2329.02|12.57|0.00|1257.66|1270.23|-284.85| +2452533|61474|2118|76826|1007717|5808|43946|4|44|528|48|99.43|188.91|130.34|0.00|6256.32|4772.64|9067.68|125.12|0.00|6256.32|6381.44|1483.68| +2451338|36507|1522|79556|218810|6064|19718|8|263|529|37|65.95|110.13|94.71|0.00|3504.27|2440.15|4074.81|35.04|0.00|3504.27|3539.31|1064.12| +2451338|36507|6637|79556|218810|6064|19718|8|87|529|73|24.55|44.43|0.44|0.00|32.12|1792.15|3243.39|2.24|0.00|32.12|34.36|-1760.03| +2451338|36507|15832|79556|218810|6064|19718|8|267|529|76|62.49|118.73|106.85|0.00|8120.60|4749.24|9023.48|406.03|0.00|8120.60|8526.63|3371.36| +2451338|36507|2348|79556|218810|6064|19718|8|114|529|15|74.15|110.48|83.96|0.00|1259.40|1112.25|1657.20|113.34|0.00|1259.40|1372.74|147.15| +2451338|36507|4004|79556|218810|6064|19718|8|299|529|15|38.88|49.76|27.86|0.00|417.90|583.20|746.40|0.00|0.00|417.90|417.90|-165.30| +2451338|36507|2665|79556|218810|6064|19718|8|130|529|43|79.08|79.87|53.51|0.00|2300.93|3400.44|3434.41|115.04|0.00|2300.93|2415.97|-1099.51| +2451338|36507|12878|79556|218810|6064|19718|8|151|529|100|11.01|18.71|14.03|0.00|1403.00|1101.00|1871.00|84.18|0.00|1403.00|1487.18|302.00| +2451338|36507|7750|79556|218810|6064|19718|8|171|529|83|93.82|167.93|95.72|0.00|7944.76|7787.06|13938.19|715.02|0.00|7944.76|8659.78|157.70| +2451338|36507|7646|79556|218810|6064|19718|8|125|529|4|55.38|82.51|75.08|0.00|300.32|221.52|330.04|9.00|0.00|300.32|309.32|78.80| +2451338|36507|11300|79556|218810|6064|19718|8|219|529|53|12.33|16.15|4.84|246.25|256.52|653.49|855.95|0.71|246.25|10.27|10.98|-643.22| +2451338|36507|7544|79556|218810|6064|19718|8|143|529|53|31.22|34.02|10.20|0.00|540.60|1654.66|1803.06|48.65|0.00|540.60|589.25|-1114.06| +2451338|36507|14983|79556|218810|6064|19718|8|75|529|88|67.03|126.01|70.56|2111.15|6209.28|5898.64|11088.88|368.83|2111.15|4098.13|4466.96|-1800.51| +2451338|36507|1352|79556|218810|6064|19718|8|265|529|18|72.25|109.82|35.14|354.21|632.52|1300.50|1976.76|19.48|354.21|278.31|297.79|-1022.19| +2451338|36507|9367|79556|218810|6064|19718|8|291|529|76|79.48|149.42|144.93|4626.16|11014.68|6040.48|11355.92|255.54|4626.16|6388.52|6644.06|348.04| +2451338|36507|2086|79556|218810|6064|19718|8|202|529|96|29.77|52.99|13.24|0.00|1271.04|2857.92|5087.04|88.97|0.00|1271.04|1360.01|-1586.88| +2452623|32519|17241|36249|1919727|805|35883|1|182|530|88|32.42|45.06|16.22|899.23|1427.36|2852.96|3965.28|36.96|899.23|528.13|565.09|-2324.83| +2452623|32519|16896|36249|1919727|805|35883|1|162|530|50|18.69|20.93|0.41|0.00|20.50|934.50|1046.50|0.00|0.00|20.50|20.50|-914.00| +2452623|32519|6141|36249|1919727|805|35883|1|68|530|96|83.18|92.32|79.39|0.00|7621.44|7985.28|8862.72|457.28|0.00|7621.44|8078.72|-363.84| +2452623|32519|2191|36249|1919727|805|35883|1|156|530|67|10.37|12.75|0.00|0.00|0.00|694.79|854.25|0.00|0.00|0.00|0.00|-694.79| +2452623|32519|7723|36249|1919727|805|35883|1|13|530|98|48.05|48.05|11.05|0.00|1082.90|4708.90|4708.90|54.14|0.00|1082.90|1137.04|-3626.00| +2452623|32519|2109|36249|1919727|805|35883|1|298|530|98|34.96|67.47|65.44|0.00|6413.12|3426.08|6612.06|513.04|0.00|6413.12|6926.16|2987.04| +2452623|32519|11760|36249|1919727|805|35883|1|1|530|20|3.39|4.10|2.50|0.00|50.00|67.80|82.00|1.00|0.00|50.00|51.00|-17.80| +2452623|32519|4599|36249|1919727|805|35883|1|141|530|95|45.17|85.37|17.92|1430.01|1702.40|4291.15|8110.15|21.79|1430.01|272.39|294.18|-4018.76| +2452623|32519|14047|36249|1919727|805|35883|1|255|530|58|67.08|116.71|19.84|138.08|1150.72|3890.64|6769.18|40.50|138.08|1012.64|1053.14|-2878.00| +2452623|32519|16273|36249|1919727|805|35883|1|62|530|93|2.35|3.61|2.49|0.00|231.57|218.55|335.73|4.63|0.00|231.57|236.20|13.02| +2452623|32519|16698|36249|1919727|805|35883|1|247|530|45|16.68|20.85|16.05|93.89|722.25|750.60|938.25|43.98|93.89|628.36|672.34|-122.24| +2452623|32519|1345|36249|1919727|805|35883|1|133|530|84|33.97|55.03|2.20|166.32|184.80|2853.48|4622.52|0.55|166.32|18.48|19.03|-2835.00| +2452075||15811|30524|||44838|||531|||28.54|7.99|0.00||1334.00|||0.00|399.50||| +2452075|53232|9783|30524|1697460|3149|44838|1|86|531|17|37.49|73.10|38.74|0.00|658.58|637.33|1242.70|0.00|0.00|658.58|658.58|21.25| +2452075|53232|5125|30524|1697460|3149|44838|1|292|531|87|21.16|23.27|6.51|0.00|566.37|1840.92|2024.49|5.66|0.00|566.37|572.03|-1274.55| +2452075|53232|1409|30524|1697460|3149|44838|1|79|531|50|93.14|150.88|101.08|0.00|5054.00|4657.00|7544.00|353.78|0.00|5054.00|5407.78|397.00| +2452075|53232|6529|30524|1697460|3149|44838|1|149|531|72|25.50|44.37|15.97|0.00|1149.84|1836.00|3194.64|91.98|0.00|1149.84|1241.82|-686.16| +||15633|30524||||1||531||98.79|162.01||136.46|262.44|592.74|||136.46|125.98|137.31|| +2452075|53232|15551|30524|1697460|3149|44838|1|135|531|65|54.72|103.96|13.51|641.04|878.15|3556.80|6757.40|9.48|641.04|237.11|246.59|-3319.69| +2452075|53232|9937|30524|1697460|3149|44838|1|161|531|47|85.43|124.72|72.33|0.00|3399.51|4015.21|5861.84|169.97|0.00|3399.51|3569.48|-615.70| +2452075|53232|4897|30524|1697460|3149|44838|1|86|531|14|10.14|18.04|7.21|0.00|100.94|141.96|252.56|5.04|0.00|100.94|105.98|-41.02| +|53232|4135|30524||3149||1||531||70.84|99.17||||495.88|694.19|||||| +2452075|53232|17187|30524|1697460|3149|44838|1|16|531|52|48.03|72.04|71.31|0.00|3708.12|2497.56|3746.08|259.56|0.00|3708.12|3967.68|1210.56| +2451892|33593|1103|45480|644923|1166|22656|8|260|532|44|51.50|90.12|45.96|0.00|2022.24|2266.00|3965.28|60.66|0.00|2022.24|2082.90|-243.76| +2451892|33593|2972|45480|644923|1166|22656|8|63|532|15|16.25|22.75|21.15|0.00|317.25|243.75|341.25|15.86|0.00|317.25|333.11|73.50| +2451892|33593|2978|45480|644923|1166|22656|8|57|532|58|53.25|61.23|0.00|0.00|0.00|3088.50|3551.34|0.00|0.00|0.00|0.00|-3088.50| +2451892|33593|6728|45480|644923|1166|22656|8|117|532|31|20.99|38.62|37.46|0.00|1161.26|650.69|1197.22|46.45|0.00|1161.26|1207.71|510.57| +2451892|33593|15704|45480|644923|1166|22656|8|219|532|28|68.11|94.67|31.24|0.00|874.72|1907.08|2650.76|78.72|0.00|874.72|953.44|-1032.36| +2451892|33593|10513|45480|644923|1166|22656|8|248|532|37|52.98|96.95|32.96|0.00|1219.52|1960.26|3587.15|85.36|0.00|1219.52|1304.88|-740.74| +2451892|33593|16961|45480|644923|1166|22656|8|136|532|80|31.30|61.34|34.96|0.00|2796.80|2504.00|4907.20|195.77|0.00|2796.80|2992.57|292.80| +2451892|33593|1219|45480|644923|1166|22656|8|73|532|100|93.64|156.37|28.14|0.00|2814.00|9364.00|15637.00|253.26|0.00|2814.00|3067.26|-6550.00| +2451892|33593|331|45480|644923|1166|22656|8|144|532|11|44.62|46.40|17.16|0.00|188.76|490.82|510.40|15.10|0.00|188.76|203.86|-302.06| +2451892|33593|4375|45480|644923|1166|22656|8|29|532|2|98.41|150.56|97.86|0.00|195.72|196.82|301.12|9.78|0.00|195.72|205.50|-1.10| +2451892|33593|13871|45480|644923|1166|22656|8|243|532|6|11.13|21.81|10.25|0.00|61.50|66.78|130.86|5.53|0.00|61.50|67.03|-5.28| +2451892|33593|1769|45480|644923|1166|22656|8|25|532|37|62.91|79.26|64.99|0.00|2404.63|2327.67|2932.62|24.04|0.00|2404.63|2428.67|76.96| +2451707|43295|1139|25104|348594|4437|33480|2|50|533|36|16.49|21.93|15.35|259.72|552.60|593.64|789.48|14.64|259.72|292.88|307.52|-300.76| +2451707|43295|1886|25104|348594|4437|33480|2|266|533|51|23.34|24.74|11.62|0.00|592.62|1190.34|1261.74|53.33|0.00|592.62|645.95|-597.72| +2451707|43295|9031|25104|348594|4437|33480|2|249|533|60|91.04|145.66|21.84|0.00|1310.40|5462.40|8739.60|26.20|0.00|1310.40|1336.60|-4152.00| +2451707|43295|13424|25104|348594|4437|33480|2|143|533|54|80.16|96.99|65.95|0.00|3561.30|4328.64|5237.46|35.61|0.00|3561.30|3596.91|-767.34| +2451707|43295|6503|25104|348594|4437|33480|2|23|533|80|7.21|12.47|11.97|0.00|957.60|576.80|997.60|9.57|0.00|957.60|967.17|380.80| +2451707|43295|13865|25104|348594|4437|33480|2|262|533|25|35.46|46.45|39.94|0.00|998.50|886.50|1161.25|39.94|0.00|998.50|1038.44|112.00| +2451707|43295|11575|25104|348594|4437|33480|2|216|533|71|39.00|58.50|50.31|0.00|3572.01|2769.00|4153.50|214.32|0.00|3572.01|3786.33|803.01| +2451707|43295|6332|25104|348594|4437|33480|2|202|533|84|93.75|135.00|122.85|0.00|10319.40|7875.00|11340.00|103.19|0.00|10319.40|10422.59|2444.40| +2451707|43295|9421|25104|348594|4437|33480|2|105|533|56|96.99|108.62|103.18|4738.02|5778.08|5431.44|6082.72|62.40|4738.02|1040.06|1102.46|-4391.38| +2451707|43295|4379|25104|348594|4437|33480|2|270|533|67|90.20|156.94|80.03|0.00|5362.01|6043.40|10514.98|53.62|0.00|5362.01|5415.63|-681.39| +2451707|43295|8047|25104|348594|4437|33480|2|271|533|91|51.93|57.64|39.19|0.00|3566.29|4725.63|5245.24|0.00|0.00|3566.29|3566.29|-1159.34| +2451707|43295|12553|25104|348594|4437|33480|2|159|533|18|75.86|93.30|70.90|0.00|1276.20|1365.48|1679.40|38.28|0.00|1276.20|1314.48|-89.28| +2451707|43295|15533|25104|348594|4437|33480|2|4|533|15|21.77|36.13|11.92|0.00|178.80|326.55|541.95|12.51|0.00|178.80|191.31|-147.75| +2451707|43295|11936|25104|348594|4437|33480|2|105|533|67|61.04|108.04|57.26|1304.38|3836.42|4089.68|7238.68|25.32|1304.38|2532.04|2557.36|-1557.64| +2451911|51280|15621|56955|1037501|5694|26287|2|128|534|22|97.70|161.20|98.33|0.00|2163.26|2149.40|3546.40|151.42|0.00|2163.26|2314.68|13.86| +2451911|51280|15743|56955|1037501|5694|26287|2|247|534|4|59.29|94.86|33.20|79.68|132.80|237.16|379.44|3.71|79.68|53.12|56.83|-184.04| +2451911|51280|5373|56955|1037501|5694|26287|2|12|534|5|8.38|12.65|10.75|0.00|53.75|41.90|63.25|4.30|0.00|53.75|58.05|11.85| +2451911|51280|639|56955|1037501|5694|26287|2|80|534|55|51.44|57.61|9.79|0.00|538.45|2829.20|3168.55|43.07|0.00|538.45|581.52|-2290.75| +2451911|51280|16219|56955|1037501|5694|26287|2|204|534|32|99.60|109.56|79.97|0.00|2559.04|3187.20|3505.92|204.72|0.00|2559.04|2763.76|-628.16| +2451911|51280|8921|56955|1037501|5694|26287|2|244|534|60|61.03|78.72|42.50|0.00|2550.00|3661.80|4723.20|229.50|0.00|2550.00|2779.50|-1111.80| +2451911|51280|3983|56955|1037501|5694|26287|2|70|534|73|38.22|65.35|0.65|0.00|47.45|2790.06|4770.55|0.94|0.00|47.45|48.39|-2742.61| +2451911|51280|12817|56955|1037501|5694|26287|2|209|534|13|30.33|35.18|20.05|0.00|260.65|394.29|457.34|7.81|0.00|260.65|268.46|-133.64| +2450903|63593|8648|47354|397320|6039|48005|8|46|535|30|43.77|57.33|26.37|0.00|791.10|1313.10|1719.90|71.19|0.00|791.10|862.29|-522.00| +2450903||2038|47354|397320|6039|||79|535||||50.83||||423.64|2.03||203.32||| +2450903|63593|11533|47354|397320|6039|48005|8|263|535|97|76.37|118.37|27.22|0.00|2640.34|7407.89|11481.89|211.22|0.00|2640.34|2851.56|-4767.55| +2450903|63593|1228|47354|397320|6039|48005|8|69|535|65|21.96|23.49|20.67|0.00|1343.55|1427.40|1526.85|13.43|0.00|1343.55|1356.98|-83.85| +2450903|63593|6742|47354|397320|6039|48005|8|184|535|10|13.43|25.78|10.31|0.00|103.10|134.30|257.80|8.24|0.00|103.10|111.34|-31.20| +2450903|63593|14918|47354|397320|6039|48005|8|296|535|34|94.00|141.94|38.32|0.00|1302.88|3196.00|4825.96|65.14|0.00|1302.88|1368.02|-1893.12| +2450903|63593|10789|47354|397320|6039|48005|8|172|535|45|55.57|95.58|3.82|0.00|171.90|2500.65|4301.10|10.31|0.00|171.90|182.21|-2328.75| +2450903|63593|15466|47354|397320|6039|48005|8|226|535|19|12.35|16.42|12.47|0.00|236.93|234.65|311.98|11.84|0.00|236.93|248.77|2.28| +2450903|63593|6016|47354|397320|6039|48005|8|189|535|41|56.01|83.45|40.05|0.00|1642.05|2296.41|3421.45|98.52|0.00|1642.05|1740.57|-654.36| +2450903|63593|17707|47354|397320|6039|48005|8|24|535|62|60.50|68.97|58.62|2289.69|3634.44|3751.00|4276.14|40.34|2289.69|1344.75|1385.09|-2406.25| +2450903|63593|2662|47354|397320|6039|48005|8|175|535|81|44.95|71.92|65.44|0.00|5300.64|3640.95|5825.52|371.04|0.00|5300.64|5671.68|1659.69| +2451540|35755|14450|74934|242600|2619|46638|1|288|536|18|77.19|92.62|69.46|0.00|1250.28|1389.42|1667.16|12.50|0.00|1250.28|1262.78|-139.14| +2451540|35755|8818|74934|242600|2619|46638|1|67|536|33|25.95|49.82|6.47|125.97|213.51|856.35|1644.06|7.87|125.97|87.54|95.41|-768.81| +2451540|35755|17270|74934|242600|2619|46638|1|263|536|24|79.83|95.79|42.14|0.00|1011.36|1915.92|2298.96|20.22|0.00|1011.36|1031.58|-904.56| +2451540|35755|12751|74934|242600|2619|46638|1|289|536|67|55.34|89.09|89.09|0.00|5969.03|3707.78|5969.03|298.45|0.00|5969.03|6267.48|2261.25| +2451540|35755|12127|74934|242600|2619|46638|1|250|536|86|66.20|129.75|92.12|1584.46|7922.32|5693.20|11158.50|63.37|1584.46|6337.86|6401.23|644.66| +2451540|35755|8197|74934|242600|2619|46638|1|259|536|8|63.08|74.43|27.53|0.00|220.24|504.64|595.44|19.82|0.00|220.24|240.06|-284.40| +2451540|35755|6506|74934|242600|2619|46638|1|4|536|1|56.77|84.58|84.58|32.98|84.58|56.77|84.58|4.12|32.98|51.60|55.72|-5.17| +2451540|35755|9236|74934|242600|2619|46638|1|126|536|79|41.86|72.41|72.41|4576.31|5720.39|3306.94|5720.39|11.44|4576.31|1144.08|1155.52|-2162.86| +2451540|35755|7226|74934|242600|2619|46638|1|5|536|60|87.76|113.21|101.88|183.38|6112.80|5265.60|6792.60|118.58|183.38|5929.42|6048.00|663.82| +2451540|35755|15628|74934|242600|2619|46638|1|282|536|76|26.77|42.29|20.72|0.00|1574.72|2034.52|3214.04|141.72|0.00|1574.72|1716.44|-459.80| +2451540|35755|11419|74934|242600|2619|46638|1|260|536|32|73.33|89.46|36.67|222.95|1173.44|2346.56|2862.72|19.00|222.95|950.49|969.49|-1396.07| +2451540|35755|10018|74934|242600|2619|46638|1|279|536|63|6.39|9.39|3.94|183.68|248.22|402.57|591.57|0.64|183.68|64.54|65.18|-338.03| +2451540|35755|7033|74934|242600|2619|46638|1|269|536|54|68.24|108.50|94.39|0.00|5097.06|3684.96|5859.00|254.85|0.00|5097.06|5351.91|1412.10| +2452105|62341|7297|25761|740174|1787|25000|8|249|537|66|57.44|62.03|36.59|1255.76|2414.94|3791.04|4093.98|11.59|1255.76|1159.18|1170.77|-2631.86| +2452105|62341|8651|25761|740174|1787|25000|8|249|537|90|93.98|139.09|79.28|0.00|7135.20|8458.20|12518.10|0.00|0.00|7135.20|7135.20|-1323.00| +2452105|62341|11679|25761|740174|1787|25000|8|193|537|29|37.78|60.07|43.85|0.00|1271.65|1095.62|1742.03|50.86|0.00|1271.65|1322.51|176.03| +2452105|62341|14009|25761|740174|1787|25000|8|134|537|98|60.30|106.73|58.70|2473.61|5752.60|5909.40|10459.54|229.52|2473.61|3278.99|3508.51|-2630.41| +2452105|62341|12139|25761|740174|1787|25000|8|30|537|95|55.87|103.91|103.91|0.00|9871.45|5307.65|9871.45|197.42|0.00|9871.45|10068.87|4563.80| +2452105|62341|6871|25761|740174|1787|25000|8|81|537|20|73.39|140.17|134.56|0.00|2691.20|1467.80|2803.40|161.47|0.00|2691.20|2852.67|1223.40| +2452105|62341|10391|25761|740174|1787|25000|8|41|537|69|55.56|100.56|20.11|0.00|1387.59|3833.64|6938.64|27.75|0.00|1387.59|1415.34|-2446.05| +2452105|62341|1463|25761|740174|1787|25000|8|221|537|96|61.89|113.87|15.94|0.00|1530.24|5941.44|10931.52|137.72|0.00|1530.24|1667.96|-4411.20| +2452105|62341|11625|25761|740174|1787|25000|8|231|537|25|23.46|40.58|28.81|0.00|720.25|586.50|1014.50|50.41|0.00|720.25|770.66|133.75| +2452105|62341|14579|25761|740174|1787|25000|8|139|537|12|99.79|186.60|126.88|0.00|1522.56|1197.48|2239.20|0.00|0.00|1522.56|1522.56|325.08| +2452105|62341|8403|25761|740174|1787|25000|8|283|537|30|62.77|65.28|0.00|0.00|0.00|1883.10|1958.40|0.00|0.00|0.00|0.00|-1883.10| +2452105|62341|14733|25761|740174|1787|25000|8|185|537|57|31.28|53.48|22.46|115.21|1280.22|1782.96|3048.36|69.90|115.21|1165.01|1234.91|-617.95| +2452105|62341|5713|25761|740174|1787|25000|8|101|537|82|4.87|5.64|1.97|0.00|161.54|399.34|462.48|8.07|0.00|161.54|169.61|-237.80| +2452533|52780|13437|29851|624385|4519|33853|1|32|538|65|81.26|157.64|156.06|1927.34|10143.90|5281.90|10246.60|82.16|1927.34|8216.56|8298.72|2934.66| +2452533|52780|9417|29851|624385|4519|33853|1|162|538|30|89.30|163.41|0.00|0.00|0.00|2679.00|4902.30|0.00|0.00|0.00|0.00|-2679.00| +2452533|52780|1983|29851|624385|4519|33853|1|84|538|60|92.77|179.04|3.58|0.00|214.80|5566.20|10742.40|12.88|0.00|214.80|227.68|-5351.40| +2452533|52780|6798|29851|624385|4519|33853|1|101|538|94|34.84|58.53|2.34|131.97|219.96|3274.96|5501.82|0.87|131.97|87.99|88.86|-3186.97| +2452533|52780|14659|29851|624385|4519|33853|1|19|538|8|31.78|61.33|11.03|0.00|88.24|254.24|490.64|3.52|0.00|88.24|91.76|-166.00| +2452533|52780|12685|29851|624385|4519|33853|1|268|538|4|62.12|87.58|64.80|0.00|259.20|248.48|350.32|20.73|0.00|259.20|279.93|10.72| +2452533|52780|3723|29851|624385|4519|33853|1|56|538|61|61.08|81.23|18.68|0.00|1139.48|3725.88|4955.03|45.57|0.00|1139.48|1185.05|-2586.40| +2452533|52780|11058|29851|624385|4519|33853|1|134|538|52|90.82|112.61|31.53|0.00|1639.56|4722.64|5855.72|114.76|0.00|1639.56|1754.32|-3083.08| +2452269|61393|3731|67256|291664|3272|4527|1|278|539|70|52.29|90.46|64.22|3731.18|4495.40|3660.30|6332.20|0.00|3731.18|764.22|764.22|-2896.08| +2452269|61393|12183|67256|291664|3272|4527|1|190|539|75|83.59|97.80|73.35|0.00|5501.25|6269.25|7335.00|330.07|0.00|5501.25|5831.32|-768.00| +2452269|61393|891|67256|291664|3272|4527|1|133|539|20|79.87|154.14|18.49|0.00|369.80|1597.40|3082.80|33.28|0.00|369.80|403.08|-1227.60| +2452269|61393|16739|67256|291664|3272|4527|1|249|539|95|78.93|116.81|7.00|0.00|665.00|7498.35|11096.95|33.25|0.00|665.00|698.25|-6833.35| +2452269|61393|2083|67256|291664|3272|4527|1|268|539|82|94.68|177.05|8.85|0.00|725.70|7763.76|14518.10|7.25|0.00|725.70|732.95|-7038.06| +2452269|61393|7805|67256|291664|3272|4527|1|264|539|16|18.97|26.55|16.19|0.00|259.04|303.52|424.80|5.18|0.00|259.04|264.22|-44.48| +2452269|61393|10457|67256|291664|3272|4527|1|157|539|9|91.42|128.90|64.45|0.00|580.05|822.78|1160.10|34.80|0.00|580.05|614.85|-242.73| +2452269|61393|5075|67256|291664|3272|4527|1|295|539|94|35.51|59.65|41.15|0.00|3868.10|3337.94|5607.10|0.00|0.00|3868.10|3868.10|530.16| +2452269|61393|16769|67256|291664|3272|4527|1|6|539|18|37.88|41.66|32.07|0.00|577.26|681.84|749.88|17.31|0.00|577.26|594.57|-104.58| +2452269|61393|9359|67256|291664|3272|4527|1|281|539|19|85.33|104.95|101.80|0.00|1934.20|1621.27|1994.05|154.73|0.00|1934.20|2088.93|312.93| +2452269|61393|3823|67256|291664|3272|4527|1|143|539|42|49.16|82.58|31.38|0.00|1317.96|2064.72|3468.36|0.00|0.00|1317.96|1317.96|-746.76| +2452269|61393|2323|67256|291664|3272|4527|1|126|539|12|43.23|85.16|68.12|0.00|817.44|518.76|1021.92|49.04|0.00|817.44|866.48|298.68| +2452269|61393|3169|67256|291664|3272|4527|1|258|539|77|9.65|18.52|5.55|0.00|427.35|743.05|1426.04|4.27|0.00|427.35|431.62|-315.70| +2452269|61393|7481|67256|291664|3272|4527|1|245|539|29|76.66|147.95|105.04|0.00|3046.16|2223.14|4290.55|0.00|0.00|3046.16|3046.16|823.02| +2452269|61393|2675|67256|291664|3272|4527|1|197|539|8|98.35|138.67|120.64|675.58|965.12|786.80|1109.36|5.79|675.58|289.54|295.33|-497.26| +2452269|61393|1787|67256|291664|3272|4527|1|148|539|63|9.14|10.87|6.30|222.26|396.90|575.82|684.81|10.47|222.26|174.64|185.11|-401.18| +2451504|47818|10378|59704|608500|6543|7739|1|277|540|2|1.55|2.85|1.56|0.00|3.12|3.10|5.70|0.12|0.00|3.12|3.24|0.02| +2451504|47818|10837|59704|608500|6543|7739|1|124|540|35|80.53|144.14|20.17|0.00|705.95|2818.55|5044.90|7.05|0.00|705.95|713.00|-2112.60| +2451504|47818|13522|59704|608500|6543|7739|1|222|540|44|10.77|20.35|9.15|0.00|402.60|473.88|895.40|28.18|0.00|402.60|430.78|-71.28| +2451504|47818|8011|59704|608500|6543|7739|1|56|540|51|72.08|92.26|11.07|62.10|564.57|3676.08|4705.26|30.14|62.10|502.47|532.61|-3173.61| +2451504|47818|8977|59704|608500|6543|7739|1|78|540|87|50.11|58.12|18.01|0.00|1566.87|4359.57|5056.44|109.68|0.00|1566.87|1676.55|-2792.70| +2451504|47818|15094|59704|608500|6543|7739|1|29|540|95|68.30|68.98|62.08|0.00|5897.60|6488.50|6553.10|471.80|0.00|5897.60|6369.40|-590.90| +2451504|47818|16730|59704|608500|6543|7739|1|25|540|85|82.71|148.05|39.97|0.00|3397.45|7030.35|12584.25|0.00|0.00|3397.45|3397.45|-3632.90| +2451504|47818|1162|59704|608500|6543|7739|1|234|540|31|88.06|160.26|16.02|0.00|496.62|2729.86|4968.06|4.96|0.00|496.62|501.58|-2233.24| +2451504||7942||608500|6543|7739|1|148|540||80.86|151.20|140.61|||5660.20|10584.00|0.00||||4182.50| +2451504|47818|11506|59704|608500|6543|7739|1|129|540|15|52.24|85.15|14.47|0.00|217.05|783.60|1277.25|6.51|0.00|217.05|223.56|-566.55| +2451504|47818|3094|59704|608500|6543|7739|1|3|540|100|45.11|52.77|13.72|0.00|1372.00|4511.00|5277.00|82.32|0.00|1372.00|1454.32|-3139.00| +2451504|47818|224|59704|608500|6543|7739|1|229|540|8|50.62|96.17|15.38|0.00|123.04|404.96|769.36|9.84|0.00|123.04|132.88|-281.92| +2451504|47818|16477|59704|608500|6543|7739|1|126|540|31|69.92|123.05|30.76|0.00|953.56|2167.52|3814.55|38.14|0.00|953.56|991.70|-1213.96| +2451504|47818|2896|59704|608500|6543|7739|1|169|540|64|8.05|13.68|13.40|0.00|857.60|515.20|875.52|34.30|0.00|857.60|891.90|342.40| +2451504|47818|13862|59704|608500|6543|7739|1|93|540|98|42.12|48.85|25.89|0.00|2537.22|4127.76|4787.30|202.97|0.00|2537.22|2740.19|-1590.54| +2451504|47818|3025|59704|608500|6543|7739|1|194|540|48|87.79|151.87|10.63|0.00|510.24|4213.92|7289.76|0.00|0.00|510.24|510.24|-3703.68| +2450974|70556|3856|11684|522215|4393|19724|4|29|541|50|6.07|11.04|0.33|0.00|16.50|303.50|552.00|1.15|0.00|16.50|17.65|-287.00| +2450974|70556|10507|11684|522215|4393|19724|4|158|541|38|11.60|20.06|17.85|0.00|678.30|440.80|762.28|13.56|0.00|678.30|691.86|237.50| +2450974|70556|4471|11684|522215|4393|19724|4|147|541|69|11.84|21.31|11.93|0.00|823.17|816.96|1470.39|74.08|0.00|823.17|897.25|6.21| +||1972|11684||4393||4|292|541|79||12.91|||||1019.89|7.74||387.10|394.84|| +2450974|70556|10939|11684|522215|4393|19724|4|84|541|90|91.07|152.99|82.61|4758.33|7434.90|8196.30|13769.10|53.53|4758.33|2676.57|2730.10|-5519.73| +2450974|70556|817|11684|522215|4393|19724|4|206|541|65|60.65|62.46|12.49|0.00|811.85|3942.25|4059.90|24.35|0.00|811.85|836.20|-3130.40| +2450974|70556|14444|11684|522215|4393|19724|4|221|541|36|29.19|54.29|49.40|0.00|1778.40|1050.84|1954.44|160.05|0.00|1778.40|1938.45|727.56| +2450974|70556|7468|11684|522215|4393|19724|4|55|541|94|87.77|169.39|44.04|3725.78|4139.76|8250.38|15922.66|4.13|3725.78|413.98|418.11|-7836.40| +2450974|70556|374|11684|522215|4393|19724|4|43|541|1|14.80|28.71|13.20|0.00|13.20|14.80|28.71|0.13|0.00|13.20|13.33|-1.60| +2450974|70556|14557|11684|522215|4393|19724|4|5|541|85|58.16|101.19|23.27|0.00|1977.95|4943.60|8601.15|138.45|0.00|1977.95|2116.40|-2965.65| +2450974|70556|3320|11684|522215|4393|19724|4|83|541|6|29.07|54.94|28.01|0.00|168.06|174.42|329.64|10.08|0.00|168.06|178.14|-6.36| +2450974|70556|10384|11684|522215|4393|19724|4|18|541|98|98.27|123.82|34.66|0.00|3396.68|9630.46|12134.36|67.93|0.00|3396.68|3464.61|-6233.78| +2451962|48206|7015|63606|1606857|6108|26206|4|112|542|61|5.25|8.87|7.27|0.00|443.47|320.25|541.07|22.17|0.00|443.47|465.64|123.22| +2451962|48206|10955|63606|1606857|6108|26206|4|52|542|88|34.95|40.89|7.36|284.97|647.68|3075.60|3598.32|21.76|284.97|362.71|384.47|-2712.89| +2451962|48206|16139|63606|1606857|6108|26206|4|77|542|68|88.33|140.44|18.25|0.00|1241.00|6006.44|9549.92|99.28|0.00|1241.00|1340.28|-4765.44| +2451962|48206|11737|63606|1606857|6108|26206|4|90|542|2|7.80|11.38|3.86|0.30|7.72|15.60|22.76|0.07|0.30|7.42|7.49|-8.18| +2451962|48206|11543|63606|1606857|6108|26206|4|172|542|7|8.71|11.67|5.48|0.00|38.36|60.97|81.69|2.68|0.00|38.36|41.04|-22.61| +2451962|48206|11293|63606|1606857|6108|26206|4|219|542|17|8.82|13.40|0.26|0.00|4.42|149.94|227.80|0.35|0.00|4.42|4.77|-145.52| +2451962|48206|13025|63606|1606857|6108|26206|4|166|542|77|40.37|68.22|24.55|0.00|1890.35|3108.49|5252.94|94.51|0.00|1890.35|1984.86|-1218.14| +2451962|48206|12079|63606|1606857|6108|26206|4|183|542|92|93.08|160.09|97.65|0.00|8983.80|8563.36|14728.28|628.86|0.00|8983.80|9612.66|420.44| +2451962|48206|12449|63606|1606857|6108|26206|4|168|542|56|33.84|34.85|17.07|0.00|955.92|1895.04|1951.60|28.67|0.00|955.92|984.59|-939.12| +2451872|52432|6025||1004090||29581|||543|||13.43|||401.28||1181.84|||401.28||-352.00| +2451872|52432|16501|23568|1004090|2659|29581|1|281|543|4|62.52|113.78|79.64|0.00|318.56|250.08|455.12|0.00|0.00|318.56|318.56|68.48| +2451872|52432|17653|23568|1004090|2659|29581|1|219|543|94|40.26|41.87|3.76|0.00|353.44|3784.44|3935.78|10.60|0.00|353.44|364.04|-3431.00| +2451872|52432|7993|23568|1004090|2659|29581|1|200|543|43|99.29|158.86|63.54|136.61|2732.22|4269.47|6830.98|0.00|136.61|2595.61|2595.61|-1673.86| +2451872|52432|15067|23568|1004090|2659|29581|1|92|543|69|21.92|33.53|29.84|0.00|2058.96|1512.48|2313.57|0.00|0.00|2058.96|2058.96|546.48| +2451872|52432|907|23568|1004090|2659|29581|1|242|543|59|61.32|119.57|13.15|0.00|775.85|3617.88|7054.63|23.27|0.00|775.85|799.12|-2842.03| +2451872|52432|1406|23568|1004090|2659|29581|1|17|543|86|62.69|73.97|10.35|0.00|890.10|5391.34|6361.42|8.90|0.00|890.10|899.00|-4501.24| +2451872|52432|17785|23568|1004090|2659|29581|1|264|543|30|69.26|133.67|57.47|0.00|1724.10|2077.80|4010.10|34.48|0.00|1724.10|1758.58|-353.70| +2451872|52432|9626|23568|1004090|2659|29581|1|108|543|78|22.95|44.29|30.56|0.00|2383.68|1790.10|3454.62|166.85|0.00|2383.68|2550.53|593.58| +2451872|52432|17599|23568|1004090|2659|29581|1|277|543|27|22.06|26.03|12.23|0.00|330.21|595.62|702.81|0.00|0.00|330.21|330.21|-265.41| +2451872|52432|10007|23568|1004090|2659|29581|||543|||||17.80|61.38|62.60||0.00|17.80|43.58||| +2451872|52432|2177|23568|1004090|2659|29581|1|146|543|1|43.76|46.38|5.56|0.00|5.56|43.76|46.38|0.05|0.00|5.56|5.61|-38.20| +2451872|52432|6521|23568|1004090|2659|29581|1|27|543|10|41.56|66.49|15.95|0.00|159.50|415.60|664.90|0.00|0.00|159.50|159.50|-256.10| +|52432|8450||1004090||29581|||543|32|16.59|20.23|||504.64|530.88||||504.64||-26.24| +2451872|52432|16157|23568|1004090|2659|29581|1|105|543|98|60.32|63.33|7.59|0.00|743.82|5911.36|6206.34|7.43|0.00|743.82|751.25|-5167.54| +2451401|57404|17086|35957|197493|6676|47412|7|191|544|12|90.59|154.00|40.04|172.97|480.48|1087.08|1848.00|18.45|172.97|307.51|325.96|-779.57| +2451401|57404|2917|35957|197493|6676|47412|7|146|544|45|56.89|110.36|52.97|0.00|2383.65|2560.05|4966.20|23.83|0.00|2383.65|2407.48|-176.40| +2451401|57404|15122|35957|197493|6676|47412|7|264|544|10|16.59|30.02|29.71|103.98|297.10|165.90|300.20|7.72|103.98|193.12|200.84|27.22| +2451401|57404|760|||||7|203|544||36.47||19.41|0.00|1028.73||2338.36||0.00|||| +2451401|57404|8992|35957|197493|6676|47412|7|183|544|8|79.23|94.28|86.73|666.08|693.84|633.84|754.24|0.55|666.08|27.76|28.31|-606.08| +2451401|57404|895|35957|197493|6676|47412|7|100|544|91|14.03|14.31|12.73|0.00|1158.43|1276.73|1302.21|81.09|0.00|1158.43|1239.52|-118.30| +2451401|57404|2659|35957|197493|6676|47412|7|74|544|77|12.14|12.38|3.34|0.00|257.18|934.78|953.26|12.85|0.00|257.18|270.03|-677.60| +2451401|57404|8053|35957|197493|6676|47412|7|100|544|83|83.68|99.57|65.71|0.00|5453.93|6945.44|8264.31|272.69|0.00|5453.93|5726.62|-1491.51| +2451401|57404|3472|35957|197493|6676|47412|7|43|544|47|23.84|30.99|13.01|0.00|611.47|1120.48|1456.53|42.80|0.00|611.47|654.27|-509.01| +2451474|32727|3388|14440|464891|3789|45087|2|211|545|37|29.57|57.07|41.66|0.00|1541.42|1094.09|2111.59|46.24|0.00|1541.42|1587.66|447.33| +2451474|32727|12925|14440|464891|3789|45087|2|291|545|80|48.38|76.44|28.28|0.00|2262.40|3870.40|6115.20|90.49|0.00|2262.40|2352.89|-1608.00| +2451474|32727|5857|14440|464891|3789|45087|2|186|545|73|11.16|12.72|12.72|380.70|928.56|814.68|928.56|0.00|380.70|547.86|547.86|-266.82| +2451474|32727|16198|14440|464891|3789|45087|2|294|545|17|58.63|100.25|86.21|0.00|1465.57|996.71|1704.25|58.62|0.00|1465.57|1524.19|468.86| +2451474|32727|4993|14440|464891|3789|45087|2|207|545|51|49.72|78.55|15.71|0.00|801.21|2535.72|4006.05|56.08|0.00|801.21|857.29|-1734.51| +2451474|32727|4766|14440|464891|3789|45087|2|56|545|9|8.36|16.21|11.18|0.00|100.62|75.24|145.89|2.01|0.00|100.62|102.63|25.38| +2451474||16093|14440|||45087||250|545|1||96.77|43.54|3.04|43.54||96.77|1.62|3.04|40.50||| +2451474|32727|7051|14440|464891|3789|45087|2|227|545|79|10.44|16.07|9.15|0.00|722.85|824.76|1269.53|57.82|0.00|722.85|780.67|-101.91| +2451474|32727|12146|14440|464891|3789|45087|2|138|545|91|75.34|83.62|39.30|0.00|3576.30|6855.94|7609.42|178.81|0.00|3576.30|3755.11|-3279.64| +2452256|37915|8559|94905|1152165|5425|39485|2|132|546|49|10.80|16.63|13.47|389.41|660.03|529.20|814.87|16.23|389.41|270.62|286.85|-258.58| +2452256|37915|17497|94905|1152165|5425|39485|2|271|546|39|26.89|30.38|2.12|59.52|82.68|1048.71|1184.82|1.62|59.52|23.16|24.78|-1025.55| +2452256|37915|13783|94905|1152165|5425|39485|2|141|546|96|36.39|62.59|36.92|0.00|3544.32|3493.44|6008.64|70.88|0.00|3544.32|3615.20|50.88| +2452256|37915|3461|94905|1152165|5425|39485|2|263|546|86|60.70|68.59|49.38|0.00|4246.68|5220.20|5898.74|127.40|0.00|4246.68|4374.08|-973.52| +2452256|37915|691|94905|1152165|5425|39485|2|203|546|93|53.73|65.01|24.05|0.00|2236.65|4996.89|6045.93|89.46|0.00|2236.65|2326.11|-2760.24| +2452256|37915|3195|94905|1152165|5425|39485|2|232|546|19|12.99|14.54|9.45|0.00|179.55|246.81|276.26|7.18|0.00|179.55|186.73|-67.26| +2452256|37915|10053|94905|1152165|5425|39485|2|118|546|84|94.53|152.19|77.61|2803.27|6519.24|7940.52|12783.96|37.15|2803.27|3715.97|3753.12|-4224.55| +2452256|37915|14273|94905|1152165|5425|39485|2|114|546|82|94.20|185.57|83.50|0.00|6847.00|7724.40|15216.74|68.47|0.00|6847.00|6915.47|-877.40| +2451412|40507|11947|3297|1890578|6659|7140|1|173|547|99|43.38|84.15|30.29|0.00|2998.71|4294.62|8330.85|269.88|0.00|2998.71|3268.59|-1295.91| +2451412|40507|13894|3297|1890578|6659|7140|1|178|547|72|47.54|61.32|45.37|0.00|3266.64|3422.88|4415.04|293.99|0.00|3266.64|3560.63|-156.24| +2451412|40507|9794|3297|1890578|6659|7140|1|26|547|57|24.82|42.69|16.64|0.00|948.48|1414.74|2433.33|47.42|0.00|948.48|995.90|-466.26| +2451412||17791|3297|||7140|||547|76|93.92|180.32|12.62|||7137.92|13704.32||||1035.84|-6178.80| +2451412|40507|7771|3297|1890578|6659|7140|1|40|547|21|81.15|162.30|77.90|0.00|1635.90|1704.15|3408.30|114.51|0.00|1635.90|1750.41|-68.25| +2451412|40507|4282|3297|1890578|6659|7140|1|122|547|28|82.87|95.30|73.38|0.00|2054.64|2320.36|2668.40|82.18|0.00|2054.64|2136.82|-265.72| +2451412|40507|2660|3297|1890578|6659|7140|1|276|547|16|32.59|59.31|19.57|0.00|313.12|521.44|948.96|0.00|0.00|313.12|313.12|-208.32| +2451412|40507|8828|3297|1890578|6659|7140|1|288|547|64|63.42|66.59|39.95|0.00|2556.80|4058.88|4261.76|178.97|0.00|2556.80|2735.77|-1502.08| +2451412|40507|15026|3297|1890578|6659|7140|1|108|547|77|64.69|80.86|2.42|163.97|186.34|4981.13|6226.22|0.22|163.97|22.37|22.59|-4958.76| +2451412|40507|9715|3297|||7140|1|244|547||50.53|69.22||0.00|||3184.12|19.73|0.00|||| +2451412|40507|2846|3297|1890578|6659|7140|1|234|547|13|80.73|81.53|57.07|0.00|741.91|1049.49|1059.89|7.41|0.00|741.91|749.32|-307.58| +2451412|40507|242|3297|1890578|6659|7140|1|224|547|29|41.77|58.89|20.02|0.00|580.58|1211.33|1707.81|23.22|0.00|580.58|603.80|-630.75| +2451412|40507|12739|3297|1890578|6659|7140|1|234|547|96|74.95|107.92|48.56|0.00|4661.76|7195.20|10360.32|233.08|0.00|4661.76|4894.84|-2533.44| +2451412|40507|15283||1890578|6659|7140|1||547|||53.13|21.25|||2399.60|3719.10|133.87||||| +2451412|40507|9836|3297|1890578|6659|7140|1|230|547|43|72.44|126.04|100.83|0.00|4335.69|3114.92|5419.72|0.00|0.00|4335.69|4335.69|1220.77| +2451412|40507|11114|3297|1890578|6659|7140|1|19|547|38|17.79|19.39|1.93|73.34|73.34|676.02|736.82|0.00|73.34|0.00|0.00|-676.02| +2451041|42430|10315|55173|1106389|2758|15174|8|177|548|57|17.23|22.05|2.42|0.00|137.94|982.11|1256.85|2.75|0.00|137.94|140.69|-844.17| +2451041|42430|15664|55173|1106389|2758|15174|8|26|548|98|98.84|114.65|74.52|0.00|7302.96|9686.32|11235.70|219.08|0.00|7302.96|7522.04|-2383.36| +2451041|42430|5734|55173|1106389|2758|15174|8|76|548|91|52.82|99.82|94.82|0.00|8628.62|4806.62|9083.62|86.28|0.00|8628.62|8714.90|3822.00| +2451041|42430|17485|55173|1106389|2758|15174|8|281|548|2|28.38|43.98|22.86|0.00|45.72|56.76|87.96|0.91|0.00|45.72|46.63|-11.04| +2451041|42430|3871|55173|1106389|2758|15174|8|189|548|18|4.33|4.76|0.04|0.00|0.72|77.94|85.68|0.01|0.00|0.72|0.73|-77.22| +2451041|42430|6412|55173|1106389|2758|15174|8|244|548|94|2.15|2.92|2.01|0.00|188.94|202.10|274.48|13.22|0.00|188.94|202.16|-13.16| +2451041|42430|5848|55173|1106389|2758|15174|8|153|548|17|14.02|17.10|12.14|0.00|206.38|238.34|290.70|0.00|0.00|206.38|206.38|-31.96| +2451041|42430|11468|55173|1106389|2758|15174|8|63|548|53|27.66|35.40|2.83|32.99|149.99|1465.98|1876.20|9.36|32.99|117.00|126.36|-1348.98| +2451041|42430|1784|55173|1106389|2758|15174|8|268|548|41|60.89|79.76|46.26|0.00|1896.66|2496.49|3270.16|18.96|0.00|1896.66|1915.62|-599.83| +2451041|42430|9818|55173|1106389|2758|15174|8|89|548|45|89.82|158.08|52.16|0.00|2347.20|4041.90|7113.60|211.24|0.00|2347.20|2558.44|-1694.70| +2452315|35224|13401|66237|1150262|240|3465|1|64|549|92|38.34|52.52|51.46|0.00|4734.32|3527.28|4831.84|284.05|0.00|4734.32|5018.37|1207.04| +2452315|35224|7765|66237|1150262|240|3465|1|74|549|22|51.71|54.81|33.43|0.00|735.46|1137.62|1205.82|51.48|0.00|735.46|786.94|-402.16| +2452315|35224|1603|66237|1150262|240|3465|1|79|549|73|58.65|66.27|20.54|0.00|1499.42|4281.45|4837.71|59.97|0.00|1499.42|1559.39|-2782.03| +2452315|35224|5163|66237|1150262|240|3465|1|51|549|27|24.08|39.00|29.25|0.00|789.75|650.16|1053.00|63.18|0.00|789.75|852.93|139.59| +||2298|||240|3465|||549|93||164.77|120.28|0.00|||15323.61|0.00|0.00|11186.04|11186.04|3327.54| +2452315|35224|17427|66237|1150262|240|3465|1|67|549|43|52.29|59.08|56.71|0.00|2438.53|2248.47|2540.44|0.00|0.00|2438.53|2438.53|190.06| +2452315|35224|17601|66237|1150262|240|3465|1|100|549|98|68.70|75.57|46.85|0.00|4591.30|6732.60|7405.86|183.65|0.00|4591.30|4774.95|-2141.30| +2452315|35224|12726|66237|1150262|240|3465|1|61|549|8|33.74|33.74|12.82|0.00|102.56|269.92|269.92|1.02|0.00|102.56|103.58|-167.36| +2452315|35224|2565|66237|1150262|240|3465|1|249|549|42|36.26|45.32|14.50|0.00|609.00|1522.92|1903.44|0.00|0.00|609.00|609.00|-913.92| +2452315|35224|6763|66237|1150262|240|3465|1|141|549|72|27.63|36.74|31.22|0.00|2247.84|1989.36|2645.28|202.30|0.00|2247.84|2450.14|258.48| +2452298|69594|5274|69903|1522123|4743|47363|2|101|550|36|8.67|10.49|6.39|0.00|230.04|312.12|377.64|0.00|0.00|230.04|230.04|-82.08| +2452298|69594|14520|69903|1522123|4743|47363|2|104|550|59|77.30|92.76|62.14|0.00|3666.26|4560.70|5472.84|146.65|0.00|3666.26|3812.91|-894.44| +2452298|69594|15063|69903|1522123|4743|47363|2|255|550|44|82.26|95.42|22.90|937.06|1007.60|3619.44|4198.48|3.52|937.06|70.54|74.06|-3548.90| +2452298|69594|12259|69903|1522123|4743|47363|2|210|550|75|89.35|151.89|50.12|0.00|3759.00|6701.25|11391.75|75.18|0.00|3759.00|3834.18|-2942.25| +2452298|69594|14647|69903|1522123|4743|47363|2|266|550|62|65.15|104.89|97.54|0.00|6047.48|4039.30|6503.18|483.79|0.00|6047.48|6531.27|2008.18| +2452298|69594|12714|69903|1522123|4743|47363|2|94|550|62|15.40|15.70|4.86|0.00|301.32|954.80|973.40|0.00|0.00|301.32|301.32|-653.48| +2452298|69594|6327|69903|1522123|4743|47363|2|67|550|83|78.78|79.56|35.00|0.00|2905.00|6538.74|6603.48|203.35|0.00|2905.00|3108.35|-3633.74| +2452298|69594|165|69903|1522123|4743|47363|2|208|550|98|56.60|71.31|47.77|0.00|4681.46|5546.80|6988.38|280.88|0.00|4681.46|4962.34|-865.34| +2452298|69594|16122|69903|1522123|4743|47363|2|37|550|15|75.51|125.34|41.36|421.87|620.40|1132.65|1880.10|0.00|421.87|198.53|198.53|-934.12| +2452298|69594|11904|69903|1522123|4743|47363|2|45|550|14|81.81|103.89|79.99|313.56|1119.86|1145.34|1454.46|64.50|313.56|806.30|870.80|-339.04| +2452298|69594|6025|69903|1522123|4743|47363|2|173|550|89|62.06|67.02|18.09|805.00|1610.01|5523.34|5964.78|64.40|805.00|805.01|869.41|-4718.33| +2452298|69594|16501|69903|1522123|4743|47363|2|175|550|73|98.09|134.38|18.81|0.00|1373.13|7160.57|9809.74|27.46|0.00|1373.13|1400.59|-5787.44| +2452298|69594|17653|69903|1522123|4743|47363|2|174|550|47|71.33|79.88|3.19|0.00|149.93|3352.51|3754.36|7.49|0.00|149.93|157.42|-3202.58| +2452298|69594|7993|69903|1522123|4743|47363|2|37|550|72|62.99|125.98|28.97|0.00|2085.84|4535.28|9070.56|20.85|0.00|2085.84|2106.69|-2449.44| +2452298|69594|15067|69903|1522123|4743|47363|2|196|550|100|41.78|60.58|28.47|0.00|2847.00|4178.00|6058.00|0.00|0.00|2847.00|2847.00|-1331.00| +2452298|69594|907|69903|1522123|4743|47363|2|291|550|94|36.44|48.46|47.97|3381.88|4509.18|3425.36|4555.24|90.18|3381.88|1127.30|1217.48|-2298.06| +2451790|30080|15139|17263|218356|1402|33248|10|83|551|98|90.43|119.36|77.58|0.00|7602.84|8862.14|11697.28|532.19|0.00|7602.84|8135.03|-1259.30| +2451790|30080|4087|17263|218356|1402|33248|10|104|551|69|9.90|10.39|8.41|0.00|580.29|683.10|716.91|46.42|0.00|580.29|626.71|-102.81| +2451790|30080|926|17263|218356|1402|33248|10|60|551|57|52.52|88.23|13.23|0.00|754.11|2993.64|5029.11|22.62|0.00|754.11|776.73|-2239.53| +2451790|30080|11864|17263|218356|1402|33248|10|179|551|15|26.31|28.94|4.91|67.75|73.65|394.65|434.10|0.05|67.75|5.90|5.95|-388.75| +2451790|30080|14738|17263|218356|1402|33248|10|200|551|99|65.51|68.13|10.90|431.64|1079.10|6485.49|6744.87|6.47|431.64|647.46|653.93|-5838.03| +2451790|30080|3164|17263|218356|1402|33248|10|49|551|98|60.64|111.57|82.56|5987.25|8090.88|5942.72|10933.86|0.00|5987.25|2103.63|2103.63|-3839.09| +2451790|30080|6721|17263|218356|1402|33248|10|162|551|19|73.24|131.83|43.50|0.00|826.50|1391.56|2504.77|24.79|0.00|826.50|851.29|-565.06| +2451790|30080|8881|17263|218356|1402|33248|10|143|551|18|97.96|102.85|83.30|0.00|1499.40|1763.28|1851.30|14.99|0.00|1499.40|1514.39|-263.88| +2451790|30080|2467|17263|218356|1402|33248|10|30|551|39|50.20|84.83|25.44|0.00|992.16|1957.80|3308.37|39.68|0.00|992.16|1031.84|-965.64| +2451790|30080|1058|17263|218356|1402|33248|10|287|551|85|65.38|76.49|27.53|0.00|2340.05|5557.30|6501.65|187.20|0.00|2340.05|2527.25|-3217.25| +2451272|44209|16640|62919|991272|3272|20472|1|146|552|85|7.27|12.64|3.66|0.00|311.10|617.95|1074.40|12.44|0.00|311.10|323.54|-306.85| +2451272|44209|16945|62919|991272|3272|20472|1|232|552|25|13.19|16.22|12.32|197.12|308.00|329.75|405.50|4.43|197.12|110.88|115.31|-218.87| +2451272|44209|3250|62919|991272|3272|20472|1|160|552|20|96.42|161.98|82.60|0.00|1652.00|1928.40|3239.60|148.68|0.00|1652.00|1800.68|-276.40| +2451272|44209|7586|62919|991272|3272|20472|1|131|552|99|30.05|49.88|26.43|0.00|2616.57|2974.95|4938.12|130.82|0.00|2616.57|2747.39|-358.38| +2451272|44209|9493|62919|991272|3272|20472|1|246|552|60|76.06|134.62|102.31|0.00|6138.60|4563.60|8077.20|552.47|0.00|6138.60|6691.07|1575.00| +2451272|44209|4244|62919|991272|3272|20472|1|230|552|58|6.34|7.79|2.57|0.00|149.06|367.72|451.82|2.98|0.00|149.06|152.04|-218.66| +2451272|44209|12580|62919|991272|3272|20472|1|162|552|44|74.82|145.15|33.38|0.00|1468.72|3292.08|6386.60|88.12|0.00|1468.72|1556.84|-1823.36| +2451272|44209|11977||991272|3272||1||552|84||41.31||0.00||||16.30|0.00||1646.74|| +||3070|62919||||||552|3||12.17|1.09|0.00|3.27|||0.16|0.00|||| +2451272|44209|8780|62919|991272|3272|20472|1|199|552|49|8.86|12.31|6.52|0.00|319.48|434.14|603.19|6.38|0.00|319.48|325.86|-114.66| +2451272|44209|5767|62919|991272|3272|20472|1|158|552|7|72.92|85.31|40.09|0.00|280.63|510.44|597.17|11.22|0.00|280.63|291.85|-229.81| +2451991|47911|2413|4600|1528963|1773|6136|8|223|553|99|81.42|159.58|118.08|0.00|11689.92|8060.58|15798.42|701.39|0.00|11689.92|12391.31|3629.34| +2451991|47911|7525|4600|1528963|1773|6136|8|79|553|28|38.26|38.64|19.32|0.00|540.96|1071.28|1081.92|43.27|0.00|540.96|584.23|-530.32| +2451991|47911|17167|4600|1528963|1773|6136|8|222|553|46|74.93|138.62|67.92|2780.64|3124.32|3446.78|6376.52|27.49|2780.64|343.68|371.17|-3103.10| +2451991|47911|6567|4600|1528963|1773|6136|8|228|553|46|6.69|8.83|7.32|0.00|336.72|307.74|406.18|20.20|0.00|336.72|356.92|28.98| +2451991|47911|12613|4600|1528963|1773|6136|8|266|553|7|46.84|63.70|9.55|51.47|66.85|327.88|445.90|0.30|51.47|15.38|15.68|-312.50| +2451991|47911|5049|4600|1528963|1773|6136|8|15|553|8|90.67|131.47|31.55|0.00|252.40|725.36|1051.76|22.71|0.00|252.40|275.11|-472.96| +2451991|47911|2939|4600|1528963|1773|6136|8|284|553|54|10.26|10.36|9.01|0.00|486.54|554.04|559.44|9.73|0.00|486.54|496.27|-67.50| +2451991|47911|6199|4600|1528963|1773|6136|8|68|553|11|3.71|3.85|3.15|34.65|34.65|40.81|42.35|0.00|34.65|0.00|0.00|-40.81| +||1875|||1773|||131|553|||||0.00|5403.84|||54.03|0.00||5457.87|| +2451514|54881|2858|37221|1406751|1881|7811|8|232|554|38|17.80|33.10|8.93|0.00|339.34|676.40|1257.80|6.78|0.00|339.34|346.12|-337.06| +2451514|54881|12673|37221|1406751|1881|7811|8|243|554|53|39.40|73.28|1.46|0.00|77.38|2088.20|3883.84|6.96|0.00|77.38|84.34|-2010.82| +2451514|54881|1|37221|1406751|1881|7811|8|48|554|35|5.40|9.34|3.64|78.98|127.40|189.00|326.90|1.93|78.98|48.42|50.35|-140.58| +2451514|54881|589|37221|1406751|1881|7811|8|229|554|88|49.47|67.77|63.70|0.00|5605.60|4353.36|5963.76|56.05|0.00|5605.60|5661.65|1252.24| +2451514|54881|7459|37221|1406751|1881|7811|8|7|554|29|9.07|13.24|9.66|0.00|280.14|263.03|383.96|11.20|0.00|280.14|291.34|17.11| +2451514|54881|6421|37221|1406751|1881|7811|8|210|554|62|70.89|119.09|19.05|0.00|1181.10|4395.18|7383.58|82.67|0.00|1181.10|1263.77|-3214.08| +2451514|54881|17650|37221|1406751|1881|7811|8|182|554|57|62.86|72.91|56.14|0.00|3199.98|3583.02|4155.87|127.99|0.00|3199.98|3327.97|-383.04| +2451514|54881|2812|37221|1406751|1881|7811|8|173|554|3|70.71|98.28|87.46|0.00|262.38|212.13|294.84|23.61|0.00|262.38|285.99|50.25| +2450828|35208|7477|6196|920618|2593|21815|10|120|555|25|90.07|145.91|142.99|0.00|3574.75|2251.75|3647.75|214.48|0.00|3574.75|3789.23|1323.00| +|35208|11053|6196|920618||||97|555||55.44||71.51|0.00|214.53|166.32|||0.00|||48.21| +2450828|35208|10004|6196|920618|2593|21815|10|289|555|24|62.45|84.93|21.23|0.00|509.52|1498.80|2038.32|20.38|0.00|509.52|529.90|-989.28| +2450828|35208|11950|6196|920618|2593|21815|10|169|555|70|91.15|91.15|6.38|366.21|446.60|6380.50|6380.50|4.01|366.21|80.39|84.40|-6300.11| +2450828|35208|13351|6196|920618|2593|21815|10|259|555|57|4.56|5.06|2.98|164.76|169.86|259.92|288.42|0.25|164.76|5.10|5.35|-254.82| +2450828|35208|7396|6196|920618|2593|21815|10|248|555|27|50.39|81.12|1.62|0.00|43.74|1360.53|2190.24|0.87|0.00|43.74|44.61|-1316.79| +2450828|35208|7981|6196|920618|2593|21815|10|2|555|92|16.83|20.19|3.23|0.00|297.16|1548.36|1857.48|17.82|0.00|297.16|314.98|-1251.20| +2450828|35208|10114|6196|920618|2593|21815|10|249|555|88|38.54|46.63|17.25|0.00|1518.00|3391.52|4103.44|15.18|0.00|1518.00|1533.18|-1873.52| +2450828|35208|16348|6196|920618|2593|21815|10|210|555|88|35.84|51.25|46.63|3775.16|4103.44|3153.92|4510.00|0.00|3775.16|328.28|328.28|-2825.64| +2450828|35208|11095|6196|920618|2593|21815|10|284|555|55|55.85|106.11|44.56|0.00|2450.80|3071.75|5836.05|196.06|0.00|2450.80|2646.86|-620.95| +2450828|35208|1478|6196|920618|2593|21815|10|198|555|19|34.82|48.05|11.05|0.00|209.95|661.58|912.95|18.89|0.00|209.95|228.84|-451.63| +|35208|7363||920618|2593||||555|||||0.00|375.68|501.12||26.29|0.00||401.97|-125.44| +2451070||17330|||126|15931|||556||34.89|43.96||0.00|1799.85|||89.99|0.00|||| +2451070|45567|7390|66916|991921|126|15931|2|202|556|97|34.63|37.40|1.12|0.00|108.64|3359.11|3627.80|6.51|0.00|108.64|115.15|-3250.47| +2451070|45567|8336|66916|991921|126|15931|2|158|556|63|24.47|41.84|15.06|0.00|948.78|1541.61|2635.92|37.95|0.00|948.78|986.73|-592.83| +2451070|45567|17564|66916|991921|126|15931|2|9|556|12|79.67|144.20|92.28|0.00|1107.36|956.04|1730.40|88.58|0.00|1107.36|1195.94|151.32| +2451070|45567|13172|66916|991921|126|15931|2|201|556|23|42.45|45.84|8.25|0.00|189.75|976.35|1054.32|0.00|0.00|189.75|189.75|-786.60| +2451070|45567|15301|66916|991921|126|15931|2|215|556|96|68.71|76.26|33.55|0.00|3220.80|6596.16|7320.96|193.24|0.00|3220.80|3414.04|-3375.36| +2451070|45567|5996|66916|991921|126|15931|2|68|556|48|99.61|141.44|87.69|0.00|4209.12|4781.28|6789.12|294.63|0.00|4209.12|4503.75|-572.16| +2451070|45567|4742|66916|991921|126|15931|2|102|556|43|92.52|155.43|83.93|2887.19|3608.99|3978.36|6683.49|7.21|2887.19|721.80|729.01|-3256.56| +2451070|45567|15142|66916|991921|126|15931|2|57|556|99|10.02|17.03|5.79|91.71|573.21|991.98|1685.97|28.89|91.71|481.50|510.39|-510.48| +2451070|45567|8626|66916|991921|126|15931|2|120|556|37|35.84|56.62|27.17|0.00|1005.29|1326.08|2094.94|80.42|0.00|1005.29|1085.71|-320.79| +2451070|45567|5119|66916|991921|126|15931|2|33|556|52|63.22|109.37|40.46|0.00|2103.92|3287.44|5687.24|63.11|0.00|2103.92|2167.03|-1183.52| +2451070|45567|13706|66916|991921|126|15931|2|300|556|52|20.58|20.99|15.53|0.00|807.56|1070.16|1091.48|48.45|0.00|807.56|856.01|-262.60| +|45567|12692|||126|15931|2||556|81|72.67||12.61||1021.41|5886.27|7298.91||||1041.83|| +2451070|45567|10840|66916|991921|126|15931|2|215|556|21|28.36|33.46|4.34|0.00|91.14|595.56|702.66|7.29|0.00|91.14|98.43|-504.42| +2451070|45567|15229|66916|991921|126|15931|2|80|556|52|35.28|62.79|1.25|0.00|65.00|1834.56|3265.08|3.90|0.00|65.00|68.90|-1769.56| +2451070|45567|3466|66916|991921|126|15931|2|279|556|13|11.61|20.66|14.66|80.04|190.58|150.93|268.58|3.31|80.04|110.54|113.85|-40.39| +2452563|40419|10669|47349|1292989|4229|36669|10|176|557|3|89.03|162.03|61.57|0.00|184.71|267.09|486.09|12.92|0.00|184.71|197.63|-82.38| +2452563|40419|3432|47349|1292989|4229|36669|10|26|557|89|79.11|113.91|9.11|0.00|810.79|7040.79|10137.99|16.21|0.00|810.79|827.00|-6230.00| +2452563|40419|12099|47349|1292989|4229|36669|10|35|557|88|48.35|66.72|48.03|0.00|4226.64|4254.80|5871.36|380.39|0.00|4226.64|4607.03|-28.16| +2452563|40419|7932|47349|1292989|4229|36669|10|7|557|22|86.08|139.44|86.45|0.00|1901.90|1893.76|3067.68|95.09|0.00|1901.90|1996.99|8.14| +2452563|40419|9756|47349|1292989|4229|36669|10|8|557|21|37.79|48.37|11.60|0.00|243.60|793.59|1015.77|12.18|0.00|243.60|255.78|-549.99| +2452563|40419|15667|47349|1292989|4229|36669|10|19|557|94|29.47|55.40|54.29|0.00|5103.26|2770.18|5207.60|102.06|0.00|5103.26|5205.32|2333.08| +2452563||5397|47349|||36669|10||557||14.97|21.25||0.00||||1.16|0.00|||| +2452563|40419|1686|47349|1292989|4229|36669|10|5|557|53|7.77|13.98|11.32|0.00|599.96|411.81|740.94|11.99|0.00|599.96|611.95|188.15| +2452563|40419|16771|47349|1292989|4229|36669|10|217|557|17|40.50|62.77|48.33|0.00|821.61|688.50|1067.09|0.00|0.00|821.61|821.61|133.11| +2452563|40419|8851|47349|1292989|4229|36669|10|64|557|46|50.72|92.81|75.17|0.00|3457.82|2333.12|4269.26|242.04|0.00|3457.82|3699.86|1124.70| +2452563|40419|6444|47349|1292989|4229|36669|10|261|557|2|85.02|102.02|27.54|0.00|55.08|170.04|204.04|0.00|0.00|55.08|55.08|-114.96| +||16434|47349|1292989||36669|||557||66.94|85.68|0.85|0.00|77.35|6091.54||1.54|0.00|77.35||-6014.19| +2452579|41845|17433|65883|303275|1020|45212|10|35|558|68|27.15|32.85|11.49|0.00|781.32|1846.20|2233.80|0.00|0.00|781.32|781.32|-1064.88| +2452579|41845|2877|65883|303275|1020|45212|10|7|558|82|39.29|49.11|35.85|734.92|2939.70|3221.78|4027.02|88.19|734.92|2204.78|2292.97|-1017.00| +|41845|17323|65883|||45212|||558||41.10|48.49|16.00|0.00|1120.00|2877.00|||0.00|||-1757.00| +2452579|41845|4657|65883|303275|1020|45212|10|25|558|62|2.42|4.71|1.74|0.00|107.88|150.04|292.02|8.63|0.00|107.88|116.51|-42.16| +2452579|41845|14365|65883|303275|1020|45212|10|276|558|23|78.74|92.91|33.44|0.00|769.12|1811.02|2136.93|0.00|0.00|769.12|769.12|-1041.90| +|41845|10104|65883||||||558|67|62.49||25.34|0.00|1697.78|4186.83|6531.16||0.00|1697.78|1850.58|-2489.05| +2452579|41845|15657|65883|303275|1020|45212|10|112|558|24|25.02|34.77|21.55|0.00|517.20|600.48|834.48|36.20|0.00|517.20|553.40|-83.28| +2452579|41845|4609|65883||||10|288|558||76.80|||0.00||5145.60||51.21|0.00||782.85|-4413.96| +2452579|41845|17112|65883|303275|1020|45212|10|69|558|5|41.10|55.48|41.05|67.73|205.25|205.50|277.40|11.00|67.73|137.52|148.52|-67.98| +|41845|17268|65883|303275||||184|558||3.78||1.78||26.70|56.70||0.80||26.70|27.50|| +2452579|41845|11659|65883|303275|1020|45212|10|98|558|73|92.69|164.98|34.64|0.00|2528.72|6766.37|12043.54|25.28|0.00|2528.72|2554.00|-4237.65| +2452579|41845|17857|65883|303275|1020|45212|10|85|558|38|31.29|35.35|25.80|0.00|980.40|1189.02|1343.30|29.41|0.00|980.40|1009.81|-208.62| +2452578|49199|1479|76762|1500187|7165|5343|10|32|559|15|42.03|63.04|20.80|0.00|312.00|630.45|945.60|9.36|0.00|312.00|321.36|-318.45| +2452578|49199|7363|76762|1500187|7165|5343|10|150|559|40|99.47|138.26|121.66|0.00|4866.40|3978.80|5530.40|145.99|0.00|4866.40|5012.39|887.60| +2452578|49199|13134|76762|1500187|7165|5343|10|249|559|24|20.78|32.41|2.91|0.00|69.84|498.72|777.84|2.09|0.00|69.84|71.93|-428.88| +2452578|49199|5310|76762|1500187|7165|5343|10|280|559|47|91.41|170.02|163.21|0.00|7670.87|4296.27|7990.94|460.25|0.00|7670.87|8131.12|3374.60| +2452578|49199|811|76762|1500187|7165|5343|10|103|559|89|94.56|133.32|5.33|0.00|474.37|8415.84|11865.48|4.74|0.00|474.37|479.11|-7941.47| +2452578|49199|3265|76762|1500187|7165|5343|10|253|559|16|71.90|131.57|128.93|0.00|2062.88|1150.40|2105.12|165.03|0.00|2062.88|2227.91|912.48| +2452578|49199|1197|76762|1500187|7165|5343|10|175|559|46|97.09|98.06|1.96|0.00|90.16|4466.14|4510.76|3.60|0.00|90.16|93.76|-4375.98| +2452578|49199|9487|76762|1500187|7165|5343|10|94|559|96|14.09|20.28|11.96|0.00|1148.16|1352.64|1946.88|80.37|0.00|1148.16|1228.53|-204.48| +2452578|49199|16105|76762|1500187|7165|5343|10|231|559|99|36.33|70.11|8.41|0.00|832.59|3596.67|6940.89|41.62|0.00|832.59|874.21|-2764.08| +2452578|49199|4027|76762|1500187|7165|5343|10|9|559|7|94.69|131.61|5.26|0.00|36.82|662.83|921.27|1.84|0.00|36.82|38.66|-626.01| +2452578|49199|144|76762|1500187|7165|5343|10|87|559|70|10.48|19.38|7.75|0.00|542.50|733.60|1356.60|0.00|0.00|542.50|542.50|-191.10| +|53685|2729|48449||||10|4|560||12.98||10.94|0.00|||1676.75|20.78|0.00||1060.08|| +2452024|53685|8491|48449|1039978|1907|37036|10|30|560|37|22.35|40.90|25.76|0.00|953.12|826.95|1513.30|38.12|0.00|953.12|991.24|126.17| +2452024|53685|3453|48449|1039978|1907|37036|10|220|560|30|69.10|127.14|115.69|0.00|3470.70|2073.00|3814.20|242.94|0.00|3470.70|3713.64|1397.70| +2452024|53685|6821|48449|1039978|1907|37036|10|70|560|45|30.45|42.63|0.00|0.00|0.00|1370.25|1918.35|0.00|0.00|0.00|0.00|-1370.25| +2452024|53685|12561|48449|1039978|1907|37036|10|81|560|66|32.21|51.53|44.83|532.58|2958.78|2125.86|3400.98|169.83|532.58|2426.20|2596.03|300.34| +2452024|53685|4969|48449|1039978|1907|37036|10|149|560|51|39.81|46.57|36.79|0.00|1876.29|2030.31|2375.07|37.52|0.00|1876.29|1913.81|-154.02| +2452024|53685|5445|48449|1039978|1907|37036|10|71|560|41|37.48|51.34|26.18|0.00|1073.38|1536.68|2104.94|96.60|0.00|1073.38|1169.98|-463.30| +2452024|53685|14621|48449|1039978|1907|37036|10|184|560|22|57.12|113.09|55.41|0.00|1219.02|1256.64|2487.98|24.38|0.00|1219.02|1243.40|-37.62| +2452024|53685|211|48449|1039978|1907|37036|10|98|560|99|64.89|75.27|39.89|0.00|3949.11|6424.11|7451.73|39.49|0.00|3949.11|3988.60|-2475.00| +2452024|53685|10907|48449|1039978|1907|37036|10|223|560|64|28.03|53.53|19.80|925.05|1267.20|1793.92|3425.92|30.79|925.05|342.15|372.94|-1451.77| +2452024|53685|475|48449|1039978|1907|37036|10|294|560|44|14.80|23.68|4.97|0.00|218.68|651.20|1041.92|8.74|0.00|218.68|227.42|-432.52| +2452024|53685|12475|48449|1039978|1907|37036|10|42|560|78|67.03|90.49|76.91|0.00|5998.98|5228.34|7058.22|59.98|0.00|5998.98|6058.96|770.64| +2452024|53685|16557|48449|1039978|1907|37036|10|122|560|36|9.70|11.54|0.46|8.44|16.56|349.20|415.44|0.32|8.44|8.12|8.44|-341.08| +2452024|53685|12919|48449|1039978|1907|37036|10|257|560|86|20.49|33.39|9.34|0.00|803.24|1762.14|2871.54|0.00|0.00|803.24|803.24|-958.90| +2452024||13735|48449|1039978|1907||10||560||||3.49|0.00||878.08|1150.24|9.77|0.00|||-682.64| +2451468|59586|4774|67977|550347|5070|25793|1|60|561|42|86.93|98.23|79.56|0.00|3341.52|3651.06|4125.66|200.49|0.00|3341.52|3542.01|-309.54| +2451468|59586|5008|67977|550347|5070|25793|1|123|561|87|36.00|51.84|15.55|0.00|1352.85|3132.00|4510.08|67.64|0.00|1352.85|1420.49|-1779.15| +2451468|59586|254|67977|550347|5070|25793|1|55|561|91|55.46|76.53|76.53|1741.05|6964.23|5046.86|6964.23|156.69|1741.05|5223.18|5379.87|176.32| +2451468|59586|3202|67977|550347|5070|25793|1|77|561|33|23.67|39.05|29.28|0.00|966.24|781.11|1288.65|0.00|0.00|966.24|966.24|185.13| +2451468|59586|1916|67977|550347|5070|25793|1|288|561|1|96.70|112.17|95.34|0.00|95.34|96.70|112.17|5.72|0.00|95.34|101.06|-1.36| +2451468|59586|481|67977|550347|5070|25793|1|84|561|75|8.33|14.57|11.94|0.00|895.50|624.75|1092.75|62.68|0.00|895.50|958.18|270.75| +2451468|59586|9770|67977|550347|5070|25793|1|292|561|47|12.09|19.70|15.36|0.00|721.92|568.23|925.90|14.43|0.00|721.92|736.35|153.69| +2451468|59586|3878|67977|550347|5070|25793|1|166|561|74|53.27|76.17|73.88|0.00|5467.12|3941.98|5636.58|0.00|0.00|5467.12|5467.12|1525.14| +2451468|59586|7214|67977|550347|5070|25793|1|239|561|20|32.89|51.30|32.83|164.15|656.60|657.80|1026.00|4.92|164.15|492.45|497.37|-165.35| +2451468|59586|13246|67977|550347|5070|25793|1|281|561|38|55.42|67.05|46.93|1301.83|1783.34|2105.96|2547.90|43.33|1301.83|481.51|524.84|-1624.45| +2451468|59586|12529|67977|550347|5070|25793|1|209|561|35|28.90|39.59|36.02|0.00|1260.70|1011.50|1385.65|12.60|0.00|1260.70|1273.30|249.20| +2451468|59586|8119|67977|550347|5070|25793|1|193|561|83|48.83|56.15|41.55|0.00|3448.65|4052.89|4660.45|34.48|0.00|3448.65|3483.13|-604.24| +2451468|59586|13588|67977|550347|5070|25793|1|243|561|15|73.74|112.08|98.63|0.00|1479.45|1106.10|1681.20|29.58|0.00|1479.45|1509.03|373.35| +2451468|59586|2515|67977|550347|5070|25793|1|241|561|36|63.43|107.19|65.38|0.00|2353.68|2283.48|3858.84|164.75|0.00|2353.68|2518.43|70.20| +2450919|64552|2996|50857|1582168|4679|46390|10|110|562|14|52.60|78.90|0.00|0.00|0.00|736.40|1104.60|0.00|0.00|0.00|0.00|-736.40| +2450919||13171||1582168||46390||242|562|50||21.32|||330.00|||23.10||||| +2450919|64552|7282|50857|1582168|4679|46390|10|49|562|24|2.02|3.65|3.28|0.00|78.72|48.48|87.60|0.78|0.00|78.72|79.50|30.24| +2450919|64552|15337|50857|1582168|4679|46390|10|237|562|4|2.87|3.47|1.45|3.19|5.80|11.48|13.88|0.07|3.19|2.61|2.68|-8.87| +2450919|64552|17125|50857|1582168|4679|46390|10|257|562|71|57.07|93.59|80.48|0.00|5714.08|4051.97|6644.89|228.56|0.00|5714.08|5942.64|1662.11| +2450919|64552|14780|50857|1582168|4679|46390|10|2|562|9|83.61|87.79|79.88|0.00|718.92|752.49|790.11|14.37|0.00|718.92|733.29|-33.57| +2450919|64552|17449|50857|1582168|4679|46390|10|233|562|7|8.36|9.94|0.09|0.32|0.63|58.52|69.58|0.02|0.32|0.31|0.33|-58.21| +2450919|64552|12764|50857|1582168|4679|46390|10|87|562|56|33.27|56.89|29.58|0.00|1656.48|1863.12|3185.84|49.69|0.00|1656.48|1706.17|-206.64| +2450919|64552|5936|50857|1582168|4679|46390|10|163|562|38|84.77|93.24|71.79|0.00|2728.02|3221.26|3543.12|0.00|0.00|2728.02|2728.02|-493.24| +2450919|64552|14263|50857|1582168|4679|46390|10|221|562|36|25.99|36.90|29.52|584.49|1062.72|935.64|1328.40|19.12|584.49|478.23|497.35|-457.41| +||742|50857||4679|46390|10|278|562|||130.62||0.00||358.85||0.00|0.00|||255.05| +2450919|64552|12361|50857|1582168|4679|46390|10|192|562|90|90.21|92.91|18.58|1504.98|1672.20|8118.90|8361.90|3.34|1504.98|167.22|170.56|-7951.68| +2450919|64552|7366|50857|1582168|4679|46390|10|143|562|25|47.02|82.28|13.98|0.00|349.50|1175.50|2057.00|0.00|0.00|349.50|349.50|-826.00| +2452346|54984|12349|19974|1154947|3967|20987|8|80|563|96|66.38|112.18|75.16|0.00|7215.36|6372.48|10769.28|288.61|0.00|7215.36|7503.97|842.88| +2452346|54984|3109|19974|1154947|3967|20987|8|182|563|79|62.68|72.08|67.75|0.00|5352.25|4951.72|5694.32|53.52|0.00|5352.25|5405.77|400.53| +2452346|54984|2436|19974|1154947|3967|20987|8|90|563|30|1.06|1.48|1.31|0.00|39.30|31.80|44.40|1.17|0.00|39.30|40.47|7.50| +2452346|54984|513|19974|1154947|3967|20987|8|277|563|31|32.37|48.87|25.41|0.00|787.71|1003.47|1514.97|63.01|0.00|787.71|850.72|-215.76| +2452346|54984|5748|19974|1154947|3967|20987|8|231|563|32|64.99|123.48|35.80|446.78|1145.60|2079.68|3951.36|20.96|446.78|698.82|719.78|-1380.86| +2452346|54984|13029|19974|1154947|3967|20987|8|146|563|46|30.61|45.30|23.55|0.00|1083.30|1408.06|2083.80|32.49|0.00|1083.30|1115.79|-324.76| +2452346|54984|15601|19974|1154947|3967|20987|8|125|563|75|95.91|182.22|176.75|0.00|13256.25|7193.25|13666.50|1060.50|0.00|13256.25|14316.75|6063.00| +2452346|54984|14851|19974|1154947|3967|20987|8|138|563|79|98.68|176.63|60.05|0.00|4743.95|7795.72|13953.77|142.31|0.00|4743.95|4886.26|-3051.77| +2452346||7788||1154947||||264|563|27||107.91|94.96|0.00||2081.16|||0.00|2563.92||482.76| +2452346|54984|9870|19974|1154947|3967|20987|8|143|563|49|61.62|122.00|0.00|0.00|0.00|3019.38|5978.00|0.00|0.00|0.00|0.00|-3019.38| +2452346|54984|7320|19974|1154947|3967|20987|8|7|563|72|91.17|133.10|79.86|0.00|5749.92|6564.24|9583.20|229.99|0.00|5749.92|5979.91|-814.32| +2452346|54984|15091|19974|1154947|3967|20987|8|120|563|79|59.68|102.05|26.53|880.26|2095.87|4714.72|8061.95|60.78|880.26|1215.61|1276.39|-3499.11| +2451172|68056|8731|97665|218411|404|38397|4|84|564|32|20.56|24.87|18.90|0.00|604.80|657.92|795.84|0.00|0.00|604.80|604.80|-53.12| +2451172|68056|2044|97665|218411|404|38397|4|213|564|4|58.30|109.02|71.95|0.00|287.80|233.20|436.08|20.14|0.00|287.80|307.94|54.60| +2451172|68056|15740|97665|218411|404|38397|4|21|564|31|94.72|101.35|42.56|778.42|1319.36|2936.32|3141.85|48.68|778.42|540.94|589.62|-2395.38| +2451172|68056|6226|97665|218411|404|38397|4|17|564|9|98.79|102.74|78.08|407.57|702.72|889.11|924.66|5.90|407.57|295.15|301.05|-593.96| +2451172|68056|17182|97665|218411|404|38397|4|292|564|37|75.01|83.26|68.27|0.00|2525.99|2775.37|3080.62|0.00|0.00|2525.99|2525.99|-249.38| +2451172|68056|746|97665|218411|404|38397|4|216|564|39|85.11|111.49|41.25|0.00|1608.75|3319.29|4348.11|96.52|0.00|1608.75|1705.27|-1710.54| +2451172|68056|17540|97665|218411|404|38397|4|242|564|58|87.50|175.00|49.00|0.00|2842.00|5075.00|10150.00|85.26|0.00|2842.00|2927.26|-2233.00| +2451172|68056|1378|97665|218411|404|38397|4|25|564|19|91.03|117.42|46.96|35.68|892.24|1729.57|2230.98|68.52|35.68|856.56|925.08|-873.01| +2451172|68056|12631|97665|218411|404|38397|4|30|564|86|64.19|70.60|60.01|0.00|5160.86|5520.34|6071.60|464.47|0.00|5160.86|5625.33|-359.48| +2451174|66423|10696|26529|168812|7095|578|8|85|565|58|32.21|64.09|21.14|993.15|1226.12|1868.18|3717.22|13.97|993.15|232.97|246.94|-1635.21| +2451174|66423|14558||168812|7095||||565|88||100.68|44.29|||5716.48||||||-1818.96| +2451174|66423|17695|26529|168812|7095|578|8|48|565|7|70.12|76.43|16.05|0.00|112.35|490.84|535.01|7.86|0.00|112.35|120.21|-378.49| +2451174|66423|8702|26529|168812|7095|578|8|229|565|23|90.01|174.61|171.11|0.00|3935.53|2070.23|4016.03|196.77|0.00|3935.53|4132.30|1865.30| +2451174|66423|8275|26529|168812|7095|578|8|45|565|98|1.47|1.52|0.80|0.00|78.40|144.06|148.96|4.70|0.00|78.40|83.10|-65.66| +2451174|66423|10952|26529|168812|7095|578|8|74|565|47|95.05|171.09|160.82|4535.12|7558.54|4467.35|8041.23|30.23|4535.12|3023.42|3053.65|-1443.93| +2451174|66423|2990|26529|168812|7095|578|8|292|565|32|60.87|73.65|30.19|0.00|966.08|1947.84|2356.80|9.66|0.00|966.08|975.74|-981.76| +2451174|66423|3154|26529|168812|7095|578|8|286|565|32|1.45|2.71|0.18|0.00|5.76|46.40|86.72|0.51|0.00|5.76|6.27|-40.64| +2452230|45634|16665|19727|115531|2535|5139|8|68|566|25|29.25|48.26|32.33|0.00|808.25|731.25|1206.50|72.74|0.00|808.25|880.99|77.00| +2452230|45634|12471|19727|115531|2535|5139|8|81|566|87|17.62|28.19|20.57|268.43|1789.59|1532.94|2452.53|30.42|268.43|1521.16|1551.58|-11.78| +2452230|45634|10469|19727|115531|2535|5139|8|164|566|23|85.10|92.75|27.82|0.00|639.86|1957.30|2133.25|44.79|0.00|639.86|684.65|-1317.44| +2452230|45634|11861|19727|115531|2535|5139|8|147|566|11|21.02|31.95|21.08|0.00|231.88|231.22|351.45|13.91|0.00|231.88|245.79|0.66| +2452230|45634|16337|19727|115531|2535|5139|8|49|566|67|22.54|34.48|1.72|0.00|115.24|1510.18|2310.16|9.21|0.00|115.24|124.45|-1394.94| +2452230|45634|17461|19727|115531|2535|5139|8|191|566|1|50.83|62.52|41.88|16.75|41.88|50.83|62.52|0.00|16.75|25.13|25.13|-25.70| +2452230|45634|5979|19727|115531|2535|5139|8|123|566|1|12.24|22.15|9.30|0.00|9.30|12.24|22.15|0.55|0.00|9.30|9.85|-2.94| +2452230|45634|16065|19727|115531|2535|5139|8|4|566|45|43.66|50.20|30.12|0.00|1355.40|1964.70|2259.00|67.77|0.00|1355.40|1423.17|-609.30| +2451335|63001|4210|48561|908025|6242|22059|7|142|567|66|91.06|102.89|86.42|0.00|5703.72|6009.96|6790.74|399.26|0.00|5703.72|6102.98|-306.24| +||11989|||6242|22059|||567||43.65|||0.00|||5104.76||0.00|||| +2451335|63001|13694|48561|908025|6242|22059|7|123|567|9|26.49|32.58|3.58|18.68|32.22|238.41|293.22|0.00|18.68|13.54|13.54|-224.87| +2451335|63001|14050|48561|908025|6242|22059|7|226|567|88|1.30|1.33|0.31|18.55|27.28|114.40|117.04|0.26|18.55|8.73|8.99|-105.67| +2451335|63001|11210|48561|908025|6242|22059|7||567||96.57||||2982.05|4152.51||||2982.05||-1170.46| +2451335|63001|7562|48561|908025|6242|22059|7|39|567|36|70.95|112.81|90.24|0.00|3248.64|2554.20|4061.16|162.43|0.00|3248.64|3411.07|694.44| +2451335|63001|6496|48561|908025|6242|22059|7|281|567|76|6.15|11.68|5.37|69.38|408.12|467.40|887.68|3.38|69.38|338.74|342.12|-128.66| +2451335|63001|3577|48561|908025|6242|22059|7|233|567|96|92.13|182.41|140.45|0.00|13483.20|8844.48|17511.36|943.82|0.00|13483.20|14427.02|4638.72| +2451335|63001|800|48561|908025|6242|22059|7|252|567|35|56.34|74.36|34.20|0.00|1197.00|1971.90|2602.60|59.85|0.00|1197.00|1256.85|-774.90| +2451335|63001|11110|48561|908025|6242|22059|7|115|567|19|93.42|183.10|91.55|0.00|1739.45|1774.98|3478.90|86.97|0.00|1739.45|1826.42|-35.53| +2451335|63001|8098|48561|908025|6242|22059|7|28|567|48|37.66|73.81|0.00|0.00|0.00|1807.68|3542.88|0.00|0.00|0.00|0.00|-1807.68| +2452553|53503|2977|40416|1873784|1199|4614|2|285|568|84|98.03|133.32|66.66|0.00|5599.44|8234.52|11198.88|503.94|0.00|5599.44|6103.38|-2635.08| +2452553|53503|14433|40416|1873784|1199|4614|2|258|568|51|52.96|53.48|17.64|0.00|899.64|2700.96|2727.48|80.96|0.00|899.64|980.60|-1801.32| +2452553|53503|13716|40416||1199|4614|2|11|568||11.52||||614.66||992.07|||614.66|626.95|-226.30| +2452553|53503|7515|40416|1873784|1199|4614|2|86|568|34|18.15|30.85|12.34|0.00|419.56|617.10|1048.90|25.17|0.00|419.56|444.73|-197.54| +2452553|53503|11334|40416|1873784|1199|4614|2|146|568|11|32.50|65.00|29.90|0.00|328.90|357.50|715.00|16.44|0.00|328.90|345.34|-28.60| +2452553|53503|13993|40416|1873784|1199|4614|2|203|568|45|5.11|7.61|5.85|0.00|263.25|229.95|342.45|0.00|0.00|263.25|263.25|33.30| +2452553|53503|5581|40416|1873784|1199|4614|2|192|568|99|18.27|31.24|23.11|0.00|2287.89|1808.73|3092.76|0.00|0.00|2287.89|2287.89|479.16| +2452553|53503|3852|40416|1873784|1199|4614|2|94|568|65|35.46|60.63|45.47|0.00|2955.55|2304.90|3940.95|236.44|0.00|2955.55|3191.99|650.65| +2452553||15282||1873784||||83|568|87|||71.42|||5854.23||497.08||6213.54||| +2452553|53503|1341|40416|1873784|1199|4614|2|221|568|63|88.16|94.33|83.01|0.00|5229.63|5554.08|5942.79|52.29|0.00|5229.63|5281.92|-324.45| +2452553|53503|2748|40416|1873784|1199|4614|2|213|568|41|20.02|39.23|15.69|0.00|643.29|820.82|1608.43|19.29|0.00|643.29|662.58|-177.53| +2452553|53503|17335|40416|1873784|1199|4614|2|171|568|40|8.82|10.76|2.25|54.90|90.00|352.80|430.40|2.80|54.90|35.10|37.90|-317.70| +2451214|59428|12518|88085|123217|2349|38281|4|95|569|74|15.28|23.37|4.44|315.41|328.56|1130.72|1729.38|0.65|315.41|13.15|13.80|-1117.57| +2451214|59428|2803|88085|123217|2349|38281|4|47|569|95|43.29|81.38|31.73|0.00|3014.35|4112.55|7731.10|150.71|0.00|3014.35|3165.06|-1098.20| +2451214|59428|13207|88085|123217|2349|38281|4|183|569|75|33.94|45.47|25.91|0.00|1943.25|2545.50|3410.25|155.46|0.00|1943.25|2098.71|-602.25| +|59428|3451||||38281|4|248|569|85|6.91|13.05|||443.70||1109.25|35.49||443.70|479.19|| +2451214|59428|4189|88085|123217|2349|38281|4|81|569|16|7.35|13.23|11.37|0.00|181.92|117.60|211.68|12.73|0.00|181.92|194.65|64.32| +2451214|59428|9410|88085|123217|2349|38281|4|172|569|61|71.16|74.00|24.42|0.00|1489.62|4340.76|4514.00|89.37|0.00|1489.62|1578.99|-2851.14| +2451214|59428|8326|88085|123217|2349|38281|4|298|569|19|41.82|43.91|32.93|0.00|625.67|794.58|834.29|31.28|0.00|625.67|656.95|-168.91| +2451214|59428|6434|88085|123217|2349|38281|4|37|569|30|57.33|95.16|19.03|0.00|570.90|1719.90|2854.80|5.70|0.00|570.90|576.60|-1149.00| +2451833|48702|13907|71263|1065034|4405|23023|7|160|570|4|68.91|84.75|68.64|0.00|274.56|275.64|339.00|19.21|0.00|274.56|293.77|-1.08| +2451833|48702|8017|71263|1065034|4405|23023|7|84|570|42|47.14|63.16|42.31|0.00|1777.02|1979.88|2652.72|88.85|0.00|1777.02|1865.87|-202.86| +2451833|48702|1010|71263|1065034|4405|23023|7|172|570|27|67.87|69.22|33.22|0.00|896.94|1832.49|1868.94|35.87|0.00|896.94|932.81|-935.55| +2451833|48702|7675|71263|1065034|4405|23023|7|148|570|54|72.32|132.34|51.61|0.00|2786.94|3905.28|7146.36|222.95|0.00|2786.94|3009.89|-1118.34| +2451833|48702|4478|71263|1065034|4405|23023|7|134|570|28|97.04|97.04|12.61|0.00|353.08|2717.12|2717.12|3.53|0.00|353.08|356.61|-2364.04| +2451833|48702|14533|71263|1065034|4405|23023|7|67|570|84|44.62|46.40|19.02|0.00|1597.68|3748.08|3897.60|111.83|0.00|1597.68|1709.51|-2150.40| +2451833|48702|6401|71263|1065034|4405|23023|7|45|570|17|95.43|181.31|116.03|0.00|1972.51|1622.31|3082.27|78.90|0.00|1972.51|2051.41|350.20| +2451833|48702|4130|71263|1065034|4405|23023|7|124|570|27|35.73|70.38|54.19|1463.13|1463.13|964.71|1900.26|0.00|1463.13|0.00|0.00|-964.71| +2451833|48702|17618|71263|1065034|4405|23023|7|42|570|83|65.84|88.22|56.46|0.00|4686.18|5464.72|7322.26|281.17|0.00|4686.18|4967.35|-778.54| +2451833|48702|7903|71263|1065034|4405|23023|7|118|570|44|87.35|152.86|123.81|1906.67|5447.64|3843.40|6725.84|318.68|1906.67|3540.97|3859.65|-302.43| +2451833|48702|1088|71263|1065034|4405|23023|7|259|570|85|44.41|63.06|56.75|0.00|4823.75|3774.85|5360.10|48.23|0.00|4823.75|4871.98|1048.90| +2452128|50567|17803|45287|934582|1030|41525|10|231|571|100|34.59|51.88|30.60|0.00|3060.00|3459.00|5188.00|183.60|0.00|3060.00|3243.60|-399.00| +2452128|50567|16191|45287|934582|1030|41525|10|270|571|60|90.12|103.63|74.61|0.00|4476.60|5407.20|6217.80|223.83|0.00|4476.60|4700.43|-930.60| +2452128|50567|9555|45287|934582|1030|41525|10|167|571|60|88.93|113.83|36.42|0.00|2185.20|5335.80|6829.80|65.55|0.00|2185.20|2250.75|-3150.60| +2452128|50567|8545|45287|934582|1030|41525|10|297|571|52|50.19|84.31|0.84|0.00|43.68|2609.88|4384.12|3.05|0.00|43.68|46.73|-2566.20| +2452128|50567|15443|45287|934582|1030|41525|10|126|571|64|29.28|38.64|5.40|0.00|345.60|1873.92|2472.96|10.36|0.00|345.60|355.96|-1528.32| +2452128|50567|11545|45287|934582|1030|41525|10|137|571|77|38.93|50.99|32.12|0.00|2473.24|2997.61|3926.23|148.39|0.00|2473.24|2621.63|-524.37| +2452128|50567|4185|45287|934582|1030|41525|10|235|571|78|74.46|116.90|22.21|0.00|1732.38|5807.88|9118.20|0.00|0.00|1732.38|1732.38|-4075.50| +||7715|45287|||41525||294|571||27.87||16.03|0.00|||28.14||0.00|16.03|16.35|-11.84| +2452128|50567|12053|45287|934582|1030|41525|10|132|571|61|70.58|81.16|81.16|0.00|4950.76|4305.38|4950.76|148.52|0.00|4950.76|5099.28|645.38| +2452128|50567|8349|45287|934582|1030|41525|10|16|571|68|93.64|130.15|19.52|0.00|1327.36|6367.52|8850.20|26.54|0.00|1327.36|1353.90|-5040.16| +2452128|50567|3763|45287|934582|1030|41525|10|186|571|33|31.88|42.08|1.26|0.00|41.58|1052.04|1388.64|0.41|0.00|41.58|41.99|-1010.46| +2452128|50567|10681|45287|934582|1030|41525|10|191|571|71|62.45|79.31|27.75|0.00|1970.25|4433.95|5631.01|59.10|0.00|1970.25|2029.35|-2463.70| +2451582|44788|7517|38321|836863|6605|26394|7|21|572|80|2.82|4.73|4.25|0.00|340.00|225.60|378.40|0.00|0.00|340.00|340.00|114.40| +2451582|44788|7094|38321|836863|6605|26394|7|49|572|60|32.72|48.42|32.44|0.00|1946.40|1963.20|2905.20|58.39|0.00|1946.40|2004.79|-16.80| +2451582|44788|7919|38321|836863|6605|26394|7|166|572|20|92.17|117.05|42.13|0.00|842.60|1843.40|2341.00|67.40|0.00|842.60|910.00|-1000.80| +2451582|44788|11887|38321|836863|6605|26394|7|155|572|21|59.91|94.05|92.16|0.00|1935.36|1258.11|1975.05|174.18|0.00|1935.36|2109.54|677.25| +2451582|44788|17213|38321|836863|6605|26394|7|175|572|11|86.17|125.80|81.77|0.00|899.47|947.87|1383.80|44.97|0.00|899.47|944.44|-48.40| +2451582|44788|1267|38321|836863|6605|26394|7|99|572|67|2.71|3.25|1.98|0.00|132.66|181.57|217.75|9.28|0.00|132.66|141.94|-48.91| +|44788|2396|38321||6605|26394||228|572||||3.61|||473.04||11.69||292.41|304.10|-180.63| +2451582|44788|9425|38321|836863|6605|26394|7|124|572|5|26.80|42.88|19.72|0.00|98.60|134.00|214.40|6.90|0.00|98.60|105.50|-35.40| +2451582|44788|3764|38321|836863|6605|26394|7|145|572|86|63.26|76.54|71.94|0.00|6186.84|5440.36|6582.44|371.21|0.00|6186.84|6558.05|746.48| +2452638|53725|1735|7582|265835|2135|4788|10|69|573|58|56.15|71.31|0.71|0.00|41.18|3256.70|4135.98|3.70|0.00|41.18|44.88|-3215.52| +||9534|7582|265835|||||573||||17.12||1575.04|2131.64||15.75|||1590.79|-556.60| +2452638|53725|1224|7582|265835|2135|4788|10|188|573|66|30.41|33.14|21.54|0.00|1421.64|2007.06|2187.24|99.51|0.00|1421.64|1521.15|-585.42| +2452638|53725|12144|7582|265835|2135|4788|10|222|573|77|74.33|146.43|128.85|0.00|9921.45|5723.41|11275.11|694.50|0.00|9921.45|10615.95|4198.04| +2452638|53725|13251|7582|265835|2135|4788|10|71|573|77|10.34|19.02|18.44|0.00|1419.88|796.18|1464.54|0.00|0.00|1419.88|1419.88|623.70| +2452638|53725|12336|7582|265835|2135|4788|10|131|573|69|54.51|74.13|30.39|0.00|2096.91|3761.19|5114.97|62.90|0.00|2096.91|2159.81|-1664.28| +2452638|53725|16927|7582|265835|2135|4788|10|100|573|85|98.32|131.74|113.29|0.00|9629.65|8357.20|11197.90|770.37|0.00|9629.65|10400.02|1272.45| +2452638|53725|8364|7582|265835|2135|4788|10|22|573|32|32.49|55.23|28.71|0.00|918.72|1039.68|1767.36|64.31|0.00|918.72|983.03|-120.96| +2452638|53725|14347|7582|265835|2135|4788|10|298|573|37|77.06|82.45|61.83|0.00|2287.71|2851.22|3050.65|0.00|0.00|2287.71|2287.71|-563.51| +2452638|53725|11517|7582|265835|2135|4788|10|17|573|15|30.61|53.26|34.08|0.00|511.20|459.15|798.90|46.00|0.00|511.20|557.20|52.05| +2452638|53725|1099|7582|265835|2135|4788|10|41|573|75|7.67|14.34|5.30|0.00|397.50|575.25|1075.50|31.80|0.00|397.50|429.30|-177.75| +2451176|48934|6637|33950|750844|5691|6938|10|208|574|74|91.11|125.73|28.91|0.00|2139.34|6742.14|9304.02|128.36|0.00|2139.34|2267.70|-4602.80| +2451176|48934|15832|33950|750844|5691|6938|10|102|574|53|48.66|61.79|42.01|0.00|2226.53|2578.98|3274.87|22.26|0.00|2226.53|2248.79|-352.45| +2451176|48934|2348|33950|750844|5691|6938|10|188|574|11|5.15|9.78|3.22|0.00|35.42|56.65|107.58|2.83|0.00|35.42|38.25|-21.23| +2451176|48934|4004|33950|750844|5691|6938|10|13|574|95|71.86|117.13|56.22|0.00|5340.90|6826.70|11127.35|0.00|0.00|5340.90|5340.90|-1485.80| +2451176|48934|2665|33950|750844|5691|6938|10|276|574|95|3.67|6.31|0.69|0.00|65.55|348.65|599.45|5.89|0.00|65.55|71.44|-283.10| +2451176|48934|12878|33950|750844|5691|6938|10|297|574|67|18.55|30.23|18.44|284.16|1235.48|1242.85|2025.41|19.02|284.16|951.32|970.34|-291.53| +2451176|48934|7750|33950|750844|5691|6938|10|214|574|46|95.78|136.00|9.52|271.51|437.92|4405.88|6256.00|13.31|271.51|166.41|179.72|-4239.47| +2451176|48934|7646|33950|750844|5691|6938|10|44|574|1|5.80|7.25|0.94|0.00|0.94|5.80|7.25|0.00|0.00|0.94|0.94|-4.86| +2451176|48934|11300|33950|||6938|||574|40||128.97||0.00|4075.20|||40.75|0.00||4115.95|76.00| +2451455|33717|14216|15482|619881|633|28585|10|218|575|39|6.33|7.15|5.64|0.00|219.96|246.87|278.85|4.39|0.00|219.96|224.35|-26.91| +2451455|33717|2834|15482|619881|633|28585|10|126|575|27|41.90|57.82|57.82|0.00|1561.14|1131.30|1561.14|62.44|0.00|1561.14|1623.58|429.84| +2451455|33717|4117|15482|619881|633|28585|10|283|575|76|76.59|83.48|29.21|0.00|2219.96|5820.84|6344.48|133.19|0.00|2219.96|2353.15|-3600.88| +2451455|33717|5035|15482|619881|633|28585|10|195|575|17|84.00|127.68|22.98|0.00|390.66|1428.00|2170.56|11.71|0.00|390.66|402.37|-1037.34| +2451455|33717|662|15482|619881|633|28585|10|100|575|98|79.90|128.63|48.87|0.00|4789.26|7830.20|12605.74|95.78|0.00|4789.26|4885.04|-3040.94| +2451455|33717|10804|15482|619881|633|28585|10|23|575|47|16.94|19.98|10.38|0.00|487.86|796.18|939.06|39.02|0.00|487.86|526.88|-308.32| +||7345|15482|619881|633|28585||162|575||70.03|107.14|4.28||303.88|||5.59|||285.16|-4692.56| +2451455|33717|11378|15482|619881|633|28585|10|134|575|12|79.66|90.81|22.70|0.00|272.40|955.92|1089.72|16.34|0.00|272.40|288.74|-683.52| +2451455|33717|11164|15482|619881|633|28585|10|82|575|17|61.41|66.32|15.25|18.14|259.25|1043.97|1127.44|12.05|18.14|241.11|253.16|-802.86| +2451455|33717|13778|15482|619881|633|28585|10|286|575|12|11.26|12.49|11.36|76.33|136.32|135.12|149.88|2.99|76.33|59.99|62.98|-75.13| +2451455|33717|9850|15482|619881|633|28585|10|193|575|50|27.82|33.38|21.69|0.00|1084.50|1391.00|1669.00|65.07|0.00|1084.50|1149.57|-306.50| +|33717|13666||||28585|||575||1.97||2.18|4.77|159.14|143.81|181.04||4.77|154.37|160.54|10.56| +2451455|33717|15526|15482|619881|633|28585|10|249|575|31|24.69|28.64|16.89|0.00|523.59|765.39|887.84|41.88|0.00|523.59|565.47|-241.80| +2451455|33717|9292|15482|619881|633|28585|10|164|575|19|33.10|56.60|34.52|400.08|655.88|628.90|1075.40|23.02|400.08|255.80|278.82|-373.10| +2451455|33717|12583|15482|619881|633|28585|10|20|575|77|97.16|117.56|27.03|0.00|2081.31|7481.32|9052.12|83.25|0.00|2081.31|2164.56|-5400.01| +2451442|62975|6434|94402|247691|4980|41116|8|296|576|61|68.14|101.52|87.30|0.00|5325.30|4156.54|6192.72|213.01|0.00|5325.30|5538.31|1168.76| +2451442|62975|14005|94402|247691|4980|41116|8|194|576|78|57.48|62.07|14.27|768.01|1113.06|4483.44|4841.46|6.90|768.01|345.05|351.95|-4138.39| +2451442|62975|4477|94402|247691|4980|41116|8|128|576|50|68.63|137.26|41.17|0.00|2058.50|3431.50|6863.00|102.92|0.00|2058.50|2161.42|-1373.00| +2451442|62975|9350|94402|247691|4980|41116|8|144|576|36|47.90|64.66|21.98|0.00|791.28|1724.40|2327.76|71.21|0.00|791.28|862.49|-933.12| +2451442|62975|4148|94402|247691|4980|41116|8|97|576|60|44.37|59.89|46.11|0.00|2766.60|2662.20|3593.40|165.99|0.00|2766.60|2932.59|104.40| +2451442|62975|10231|94402|247691|4980|41116|8|57|576|87|94.00|151.34|143.77|0.00|12507.99|8178.00|13166.58|1000.63|0.00|12507.99|13508.62|4329.99| +2451442|62975|7978|94402|247691|4980|41116|8|197|576|74|13.91|18.91|13.80|0.00|1021.20|1029.34|1399.34|71.48|0.00|1021.20|1092.68|-8.14| +2451442|62975|10327|94402|247691|4980|41116|8|23|576|90|83.83|83.83|23.47|1077.27|2112.30|7544.70|7544.70|10.35|1077.27|1035.03|1045.38|-6509.67| +2451442|62975|8839|94402|247691|4980|41116|8|164|576|58|70.26|131.38|105.10|0.00|6095.80|4075.08|7620.04|304.79|0.00|6095.80|6400.59|2020.72| +2451442|62975|15367|94402|247691|4980|41116|8|118|576|9|55.39|78.09|50.75|0.00|456.75|498.51|702.81|36.54|0.00|456.75|493.29|-41.76| +2451442|62975|2558|94402|247691|4980|41116|8|194|576|74|45.18|75.90|37.95|0.00|2808.30|3343.32|5616.60|252.74|0.00|2808.30|3061.04|-535.02| +2451442|62975|9166|94402|247691|4980|41116|8|258|576|22|40.23|74.42|15.62|0.00|343.64|885.06|1637.24|17.18|0.00|343.64|360.82|-541.42| +2451442|62975|16370|94402|247691|4980|41116|8|123|576|17|34.34|67.64|30.43|0.00|517.31|583.78|1149.88|20.69|0.00|517.31|538.00|-66.47| +2451442|62975|14053|94402|247691|4980|41116|8|186|576|34|1.08|1.94|1.00|0.00|34.00|36.72|65.96|2.04|0.00|34.00|36.04|-2.72| +2451442|62975|12544|94402|247691|4980|41116|8|13|576|11|80.34|152.64|91.58|221.62|1007.38|883.74|1679.04|0.00|221.62|785.76|785.76|-97.98| +2451442|62975|17758|94402|247691|4980|41116|8|297|576|68|85.95|134.94|99.85|0.00|6789.80|5844.60|9175.92|611.08|0.00|6789.80|7400.88|945.20| +2451497|42925|13960|45489|1439405|6121|48884|7|100|577|72|80.78|106.62|77.83|0.00|5603.76|5816.16|7676.64|392.26|0.00|5603.76|5996.02|-212.40| +2451497|42925|130|45489|1439405|6121|48884|7|1|577|98|24.63|45.81|25.19|0.00|2468.62|2413.74|4489.38|0.00|0.00|2468.62|2468.62|54.88| +2451497|42925|14744|45489|1439405|6121|48884|7|261|577|44|51.11|88.93|9.78|0.00|430.32|2248.84|3912.92|34.42|0.00|430.32|464.74|-1818.52| +2451497|42925|10808|45489|1439405|6121|48884|7|124|577|26|58.68|93.88|46.94|0.00|1220.44|1525.68|2440.88|61.02|0.00|1220.44|1281.46|-305.24| +2451497|42925|1030|45489|1439405|6121|48884|7|274|577|54|61.39|85.33|19.62|0.00|1059.48|3315.06|4607.82|63.56|0.00|1059.48|1123.04|-2255.58| +2451497|42925|11065|45489|1439405|6121|48884|7|255|577|5|58.80|91.14|80.20|184.46|401.00|294.00|455.70|4.33|184.46|216.54|220.87|-77.46| +2451497|42925|898|45489|1439405|6121|48884|7|2|577|24|8.48|15.34|5.52|0.00|132.48|203.52|368.16|7.94|0.00|132.48|140.42|-71.04| +2451497|42925|13310|45489|1439405|6121|48884|7|276|577|70|54.55|89.46|67.98|0.00|4758.60|3818.50|6262.20|333.10|0.00|4758.60|5091.70|940.10| +2451497|42925|658|45489|1439405|6121|48884|7|39|577|48|3.98|7.16|4.51|0.00|216.48|191.04|343.68|19.48|0.00|216.48|235.96|25.44| +2451497|42925|5491|45489|1439405|6121|48884|7|55|577|66|19.83|38.86|12.82|389.21|846.12|1308.78|2564.76|27.41|389.21|456.91|484.32|-851.87| +2451556|29705|2707|93540|1034211|129|25600|7|243|578|97|95.55|164.34|115.03|5355.79|11157.91|9268.35|15940.98|232.08|5355.79|5802.12|6034.20|-3466.23| +2451556|29705|15217|93540|1034211|129|25600|7|225|578|52|56.19|74.73|20.17|20.97|1048.84|2921.88|3885.96|0.00|20.97|1027.87|1027.87|-1894.01| +||5486|93540||129|25600|7||578|8|80.68||||721.04||1265.04|0.00||721.04||75.60| +2451556|29705|3608|93540|1034211|129|25600|7|233|578|98|88.28|132.42|71.50|2452.45|7007.00|8651.44|12977.16|409.90|2452.45|4554.55|4964.45|-4096.89| +2451556|29705|10364|93540|1034211|129|25600|7|152|578|26|62.22|102.04|63.26|0.00|1644.76|1617.72|2653.04|148.02|0.00|1644.76|1792.78|27.04| +2451556|29705|584|93540|1034211|129|25600|7|285|578|42|21.86|36.50|8.76|0.00|367.92|918.12|1533.00|18.39|0.00|367.92|386.31|-550.20| +2451556|29705|17363|93540|1034211|129|25600|7|262|578|54|76.13|98.96|10.88|0.00|587.52|4111.02|5343.84|17.62|0.00|587.52|605.14|-3523.50| +2451556|29705|8147|93540|1034211|129|25600|7|42|578|3|26.00|52.00|29.12|0.00|87.36|78.00|156.00|3.49|0.00|87.36|90.85|9.36| +2451556|29705|8273|93540|1034211|129|25600|7|239|578|98|46.88|85.32|59.72|4038.26|5852.56|4594.24|8361.36|18.14|4038.26|1814.30|1832.44|-2779.94| +2452455|52320|11647|73996|1146473|1703|28533|1|91|579|70|50.73|97.40|36.03|1008.84|2522.10|3551.10|6818.00|45.39|1008.84|1513.26|1558.65|-2037.84| +2452455|52320|15637|73996|1146473|1703|28533|1|111|579|38|42.40|83.95|44.49|0.00|1690.62|1611.20|3190.10|67.62|0.00|1690.62|1758.24|79.42| +2452455|52320|853|73996|1146473|1703|28533|1|7|579|95|56.36|70.45|22.54|0.00|2141.30|5354.20|6692.75|21.41|0.00|2141.30|2162.71|-3212.90| +2452455|52320|2041|73996|1146473|1703|28533|1|237|579|44|9.25|16.74|10.37|0.00|456.28|407.00|736.56|22.81|0.00|456.28|479.09|49.28| +2452455|52320|5143|73996|1146473|1703|28533|1|20|579|63|62.45|86.80|54.68|0.00|3444.84|3934.35|5468.40|172.24|0.00|3444.84|3617.08|-489.51| +2452455|52320|6267|73996|1146473|1703|28533|1|172|579|49|38.53|43.15|35.38|849.47|1733.62|1887.97|2114.35|70.73|849.47|884.15|954.88|-1003.82| +2452455|52320|14481|73996|1146473|1703|28533|1|115|579|42|89.33|128.63|36.01|0.00|1512.42|3751.86|5402.46|60.49|0.00|1512.42|1572.91|-2239.44| +2452455|52320|3276|73996|1146473|1703|28533|1|59|579|50|93.46|175.70|86.09|0.00|4304.50|4673.00|8785.00|43.04|0.00|4304.50|4347.54|-368.50| +2452455|52320|14418|73996|1146473|1703|28533|1|274|579|1|42.36|44.90|21.10|0.00|21.10|42.36|44.90|1.05|0.00|21.10|22.15|-21.26| +2452455|52320|10371|73996|1146473|1703|28533|1|298|579|50|79.39|84.15|16.83|0.00|841.50|3969.50|4207.50|50.49|0.00|841.50|891.99|-3128.00| +2452455|52320|12159|73996|1146473|1703|28533|1|120|579|16|73.83|76.78|3.83|0.00|61.28|1181.28|1228.48|4.90|0.00|61.28|66.18|-1120.00| +2452455|52320|1335|73996|1146473|1703|28533|1|58|579|36|36.09|70.01|29.40|0.00|1058.40|1299.24|2520.36|31.75|0.00|1058.40|1090.15|-240.84| +2452455|52320|16801||1146473|||||579||39.82||58.53||||6689.28|||5618.88||| +2452455|52320|8814|73996|1146473|1703|28533|1|163|579|98|50.33|99.65|7.97|187.45|781.06|4932.34|9765.70|17.80|187.45|593.61|611.41|-4338.73| +2451780|71542|2831|62242|120732|1622|41312|2|14|580|11|51.46|58.66|14.66|0.00|161.26|566.06|645.26|8.06|0.00|161.26|169.32|-404.80| +2451780|71542|12401|62242|120732|1622|41312|2|161|580|40|23.26|28.37|24.39|868.28|975.60|930.40|1134.80|7.51|868.28|107.32|114.83|-823.08| +2451780|71542|4457|62242|120732|1622|41312|2|255|580|36|6.80|11.69|0.00|0.00|0.00|244.80|420.84|0.00|0.00|0.00|0.00|-244.80| +2451780|71542|14633|62242|120732|1622|41312|2|188|580|25|13.90|23.07|5.53|53.91|138.25|347.50|576.75|6.74|53.91|84.34|91.08|-263.16| +2451780|71542|10669|62242|120732||41312|||580||1.22|1.81||3.51|||||3.51||49.51|| +2451780|71542|3431|62242|120732|1622|41312|2|205|580|42|38.11|51.06|38.80|0.00|1629.60|1600.62|2144.52|81.48|0.00|1629.60|1711.08|28.98| +2451780|71542|12098|62242|120732|1622|41312|2|14|580|6|77.27|127.49|7.64|0.00|45.84|463.62|764.94|0.45|0.00|45.84|46.29|-417.78| +2451780|71542|7931|62242|120732|1622|41312|2|54|580|53|35.45|42.54|39.56|0.00|2096.68|1878.85|2254.62|41.93|0.00|2096.68|2138.61|217.83| +2451780|71542|9755|62242|120732|1622|41312|2|40|580|66|73.85|98.95|77.18|0.00|5093.88|4874.10|6530.70|305.63|0.00|5093.88|5399.51|219.78| +2451780|71542|15667|62242|120732|1622|41312|2|52|580|6|94.86|120.47|112.03|0.00|672.18|569.16|722.82|20.16|0.00|672.18|692.34|103.02| +2451780|71542|5396|62242|120732|1622|41312|2|30|580|31|56.46|105.58|73.90|0.00|2290.90|1750.26|3272.98|137.45|0.00|2290.90|2428.35|540.64| +2451780|71542|1685|62242|120732|1622|41312|2|248|580|9|20.53|28.94|4.34|10.93|39.06|184.77|260.46|1.12|10.93|28.13|29.25|-156.64| +2451780|71542|16771|62242|120732|1622|41312|2|266|580|7|8.04|12.62|4.29|0.30|30.03|56.28|88.34|0.59|0.30|29.73|30.32|-26.55| +2451780|71542|8851|62242|120732|1622|41312|2|251|580|9|94.94|153.80|67.67|0.00|609.03|854.46|1384.20|36.54|0.00|609.03|645.57|-245.43| +2451780|71542|6443|62242|120732|1622|41312|2|8|580|11|24.78|32.70|17.00|0.00|187.00|272.58|359.70|3.74|0.00|187.00|190.74|-85.58| +2451615|42027|1991|64617|1086933|6049|21165|4|117|581|75|22.64|24.45|12.46|0.00|934.50|1698.00|1833.75|46.72|0.00|934.50|981.22|-763.50| +2451615|42027|320|64617|1086933|6049|21165|4|116|581|98|38.93|69.29|30.48|0.00|2987.04|3815.14|6790.42|238.96|0.00|2987.04|3226.00|-828.10| +2451615|42027|14900|64617|1086933|6049|21165|4|290|581|9|50.61|74.39|15.62|0.00|140.58|455.49|669.51|7.02|0.00|140.58|147.60|-314.91| +2451615|42027|17831|64617|1086933|6049|21165|4|12|581|83|15.63|28.29|4.52|0.00|375.16|1297.29|2348.07|0.00|0.00|375.16|375.16|-922.13| +2451615|42027|17144|64617|1086933|6049|21165|4|67|581|74|26.16|41.33|11.15|0.00|825.10|1935.84|3058.42|16.50|0.00|825.10|841.60|-1110.74| +2451615|42027|16037|64617|1086933|6049|21165|4|68|581|74|22.84|36.77|20.22|0.00|1496.28|1690.16|2720.98|0.00|0.00|1496.28|1496.28|-193.88| +2451615|42027|12265|64617|1086933|6049|21165|4|269|581|2|59.49|105.29|86.33|0.00|172.66|118.98|210.58|6.90|0.00|172.66|179.56|53.68| +2451615|42027|7309|64617|1086933|6049|21165|4|174|581|21|91.24|100.36|89.32|0.00|1875.72|1916.04|2107.56|168.81|0.00|1875.72|2044.53|-40.32| +2451615|42027|12409|64617|1086933|6049|21165|4|194|581|67|52.03|92.09|70.90|0.00|4750.30|3486.01|6170.03|142.50|0.00|4750.30|4892.80|1264.29| +2451615|42027|5569|64617|1086933|6049|21165|4|251|581|57|92.40|147.84|76.87|0.00|4381.59|5266.80|8426.88|43.81|0.00|4381.59|4425.40|-885.21| +2451615|42027|9559|64617||6049|21165|4||581|90||114.60||0.00|||||0.00|6084.90|6510.84|| +2452245|68169|5209|46889|956415|2388|29914|4|210|582|25|10.38|20.76|18.26|415.41|456.50|259.50|519.00|0.00|415.41|41.09|41.09|-218.41| +2452245|68169|7865|46889|956415|2388|29914|4|178|582|61|86.00|104.06|12.48|0.00|761.28|5246.00|6347.66|0.00|0.00|761.28|761.28|-4484.72| +2452245|68169|15697|46889|956415|2388|29914|4|21|582|21|7.55|12.23|5.74|0.00|120.54|158.55|256.83|9.64|0.00|120.54|130.18|-38.01| +2452245|68169|6875|46889|956415|2388|29914|4|95|582|77|19.48|31.55|25.24|0.00|1943.48|1499.96|2429.35|0.00|0.00|1943.48|1943.48|443.52| +2452245|68169|10981|46889|956415|2388|29914|4|244|582|65|20.82|23.31|12.82|0.00|833.30|1353.30|1515.15|8.33|0.00|833.30|841.63|-520.00| +2452245|68169|561|46889|956415|2388|29914|4|99|582|100|75.00|85.50|27.36|0.00|2736.00|7500.00|8550.00|246.24|0.00|2736.00|2982.24|-4764.00| +2452245|68169|11231|46889|956415|2388|29914|4|290|582|75|61.94|68.75|67.37|0.00|5052.75|4645.50|5156.25|101.05|0.00|5052.75|5153.80|407.25| +2452245|68169|1189|46889|956415|2388|29914|4|53|582|52|15.01|28.66|20.06|0.00|1043.12|780.52|1490.32|52.15|0.00|1043.12|1095.27|262.60| +2452245||12743|46889|956415|2388|29914|4||582|71|29.30|31.93|15.64|0.00|1110.44|2080.30|||0.00|1110.44|1210.37|-969.86| +2452245|68169|1521|46889|956415|2388|29914|4|187|582|55|25.03|46.05|2.30|0.00|126.50|1376.65|2532.75|8.85|0.00|126.50|135.35|-1250.15| +2452245|68169|12375|46889|956415|2388|29914|4|149|582|6|91.58|163.01|32.60|0.00|195.60|549.48|978.06|5.86|0.00|195.60|201.46|-353.88| +2452245|68169|5235|46889|956415|2388|29914|4|222|582|56|50.06|82.59|66.89|0.00|3745.84|2803.36|4625.04|262.20|0.00|3745.84|4008.04|942.48| +2452245|68169|15363|46889|956415|2388|29914|4|289|582|57|72.73|141.82|38.29|720.23|2182.53|4145.61|8083.74|29.24|720.23|1462.30|1491.54|-2683.31| +2452332|37900|16788|55634|571400|319|22485|4|112|583|75|96.62|164.25|146.18|0.00|10963.50|7246.50|12318.75|767.44|0.00|10963.50|11730.94|3717.00| +2452332|37900|4851|55634|571400|319|22485|4|114|583|50|78.98|88.45|22.99|0.00|1149.50|3949.00|4422.50|11.49|0.00|1149.50|1160.99|-2799.50| +2452332|37900|3768|55634|571400|319|22485|4|207|583|83|65.69|89.33|85.75|0.00|7117.25|5452.27|7414.39|142.34|0.00|7117.25|7259.59|1664.98| +2452332|37900|477|55634|571400|319|22485|4|47|583|10|66.16|74.09|20.00|0.00|200.00|661.60|740.90|18.00|0.00|200.00|218.00|-461.60| +2452332|37900|15114|55634|571400|319|22485|4|177|583|90|28.92|54.94|7.69|0.00|692.10|2602.80|4944.60|62.28|0.00|692.10|754.38|-1910.70| +2452332|37900|169|55634|571400|319|22485|4|276|583|29|28.58|39.72|15.88|0.00|460.52|828.82|1151.88|0.00|0.00|460.52|460.52|-368.30| +2452332|37900|5943|55634|571400|319|22485|4|205|583|81|49.00|67.62|22.31|469.84|1807.11|3969.00|5477.22|13.37|469.84|1337.27|1350.64|-2631.73| +2452332|37900|8487|55634|571400|319|22485|4|58|583|1|94.64|105.99|100.69|0.00|100.69|94.64|105.99|1.00|0.00|100.69|101.69|6.05| +2452332|37900|12048|55634|571400|319|22485|4|274|583|31|11.09|19.40|10.28|0.00|318.68|343.79|601.40|6.37|0.00|318.68|325.05|-25.11| +2452332|37900|12933|55634|571400|319|22485|4|42|583|75|24.27|31.55|1.57|0.00|117.75|1820.25|2366.25|5.88|0.00|117.75|123.63|-1702.50| +2452332|37900|11199|55634|571400|319|22485|4|233|583|100|57.89|59.04|8.26|0.00|826.00|5789.00|5904.00|0.00|0.00|826.00|826.00|-4963.00| +2452332|37900|7269|55634|571400|319|22485|4|114|583|40|81.54|159.00|36.57|0.00|1462.80|3261.60|6360.00|0.00|0.00|1462.80|1462.80|-1798.80| +2452411|62470|2925|38491|949349|5454|32712|8|41|584|70|88.03|129.40|90.58|0.00|6340.60|6162.10|9058.00|507.24|0.00|6340.60|6847.84|178.50| +2452411|62470|12709|38491|949349|5454|32712|8|172|584|73|80.68|127.47|63.73|0.00|4652.29|5889.64|9305.31|46.52|0.00|4652.29|4698.81|-1237.35| +2452411|62470|16242|38491|949349|5454|32712|8|157|584|1|28.50|47.88|17.23|0.00|17.23|28.50|47.88|1.55|0.00|17.23|18.78|-11.27| +2452411|62470|17917|38491|949349|5454|32712|8|55|584|25|40.94|69.59|54.28|0.00|1357.00|1023.50|1739.75|81.42|0.00|1357.00|1438.42|333.50| +2452411||6295|||||8||584|38||||0.00||3081.04|||0.00||2283.87|| +2452411|62470|6859|38491|949349|5454|32712|8|29|584|68|52.80|85.53|29.08|0.00|1977.44|3590.40|5816.04|79.09|0.00|1977.44|2056.53|-1612.96| +2452411|62470|657|38491|949349|5454|32712|8|170|584|89|92.88|185.76|26.00|0.00|2314.00|8266.32|16532.64|138.84|0.00|2314.00|2452.84|-5952.32| +2452411|62470|11316|38491|949349|5454|32712|8|16|584|38|74.41|148.07|63.67|0.00|2419.46|2827.58|5626.66|0.00|0.00|2419.46|2419.46|-408.12| +2451174|34572|17701|44243|217853|7016|10979|2|294|585|41|65.93|109.44|31.73|0.00|1300.93|2703.13|4487.04|65.04|0.00|1300.93|1365.97|-1402.20| +2451174|34572|1994|44243|217853|7016|10979|2|45|585|80|23.00|26.68|7.47|0.00|597.60|1840.00|2134.40|41.83|0.00|597.60|639.43|-1242.40| +2451174|34572|9212|44243|217853|7016|10979|2|271|585|45|74.83|99.52|88.57|0.00|3985.65|3367.35|4478.40|79.71|0.00|3985.65|4065.36|618.30| +2451174|34572|2593|44243|217853|7016|10979|2|222|585|13|53.27|68.18|26.59|0.00|345.67|692.51|886.34|0.00|0.00|345.67|345.67|-346.84| +2451174|34572|16714|44243|217853|7016|10979|2|280|585|79|73.24|92.28|43.37|0.00|3426.23|5785.96|7290.12|239.83|0.00|3426.23|3666.06|-2359.73| +2451174|34572|313|44243|217853|7016|10979|2|126|585|6|38.18|41.99|37.79|163.25|226.74|229.08|251.94|5.71|163.25|63.49|69.20|-165.59| +2451174|34572|17876|44243|217853|7016|10979|2|250|585|25|49.55|80.76|0.00|0.00|0.00|1238.75|2019.00|0.00|0.00|0.00|0.00|-1238.75| +2451174|34572|8744|44243|217853|7016|10979|2|205|585|84|78.58|102.15|75.59|0.00|6349.56|6600.72|8580.60|126.99|0.00|6349.56|6476.55|-251.16| +2451174|34572|14545|44243|217853|7016|10979|2|106|585|64|22.04|22.48|20.23|0.00|1294.72|1410.56|1438.72|51.78|0.00|1294.72|1346.50|-115.84| +2451174|34572|16034|44243|217853|7016|10979|2|92|585|76|53.05|64.72|43.36|0.00|3295.36|4031.80|4918.72|131.81|0.00|3295.36|3427.17|-736.44| +2451174|34572|4837|44243|217853|7016|10979|2|130|585|42|27.28|43.64|40.14|1685.88|1685.88|1145.76|1832.88|0.00|1685.88|0.00|0.00|-1145.76| +2451174|34572|9752|44243|217853|7016|10979|2|227|585|19|38.91|70.42|16.90|0.00|321.10|739.29|1337.98|28.89|0.00|321.10|349.99|-418.19| +2451174|34572|4220|44243|217853|7016|10979|2|287|585|11|53.83|88.28|37.07|0.00|407.77|592.13|971.08|32.62|0.00|407.77|440.39|-184.36| +2452544|49268|8563|41265|1707702|4539|37376|2|149|586|77|92.17|94.93|15.18|946.77|1168.86|7097.09|7309.61|2.22|946.77|222.09|224.31|-6875.00| +||1956|41265|1707702|4539||2||586|77||58.34||0.00|||4492.18|40.42|0.00||2061.67|| +2452544|49268|4095|41265|1707702|4539|37376|2|274|586|64|51.72|80.16|61.72|1185.02|3950.08|3310.08|5130.24|138.25|1185.02|2765.06|2903.31|-545.02| +2452544|49268|4518|41265|1707702|4539|37376|2|56|586|57|47.26|83.17|75.68|3321.59|4313.76|2693.82|4740.69|9.92|3321.59|992.17|1002.09|-1701.65| +2452544|49268|7182|41265|1707702|4539|37376|2|268|586|88|56.50|88.70|74.50|0.00|6556.00|4972.00|7805.60|590.04|0.00|6556.00|7146.04|1584.00| +2452544||9645||1707702|||2|35|586|75||137.44|123.69|3432.39|||10308.00|292.21|3432.39|5844.36|6136.57|| +2452544|49268|17889|41265|1707702|4539|37376|2|240|586|100|30.72|39.32|20.44|0.00|2044.00|3072.00|3932.00|122.64|0.00|2044.00|2166.64|-1028.00| +2452544|49268|11886|41265|||||20|586||4.43|6.42|2.50|72.07|232.50|411.99|597.06|8.02|72.07|||-251.56| +2452544|49268|421|41265|1707702|4539|37376|2|16|586|54|7.26|8.78|3.24|0.00|174.96|392.04|474.12|13.99|0.00|174.96|188.95|-217.08| +2452544|49268|12804|41265|1707702|4539|37376|2|99|586|42|30.72|42.39|31.36|0.00|1317.12|1290.24|1780.38|39.51|0.00|1317.12|1356.63|26.88| +2451622|51775|3853|57001|58151|5254|42804|7|14|587|96|25.13|40.96|33.99|0.00|3263.04|2412.48|3932.16|195.78|0.00|3263.04|3458.82|850.56| +2451622|51775|14801|57001|58151|5254|42804|7|113|587|44|87.49|113.73|105.76|0.00|4653.44|3849.56|5004.12|93.06|0.00|4653.44|4746.50|803.88| +|51775|9655||||42804||23|587||45.85|||||2338.35|4442.61|||||-72.93| +2451622|51775|5510|57001|58151|5254|42804|7|21|587|48|90.07|161.22|120.91|0.00|5803.68|4323.36|7738.56|0.00|0.00|5803.68|5803.68|1480.32| +2451622|51775|9392|57001|58151|5254|42804|7|49|587|3|27.53|29.45|15.60|0.00|46.80|82.59|88.35|1.40|0.00|46.80|48.20|-35.79| +2451622|51775|15347|57001|58151|5254|42804|7|178|587|6|76.73|125.06|68.78|0.00|412.68|460.38|750.36|16.50|0.00|412.68|429.18|-47.70| +2451622|51775|17413|57001|58151|5254|42804|7|290|587|20|91.86|146.97|98.46|0.00|1969.20|1837.20|2939.40|59.07|0.00|1969.20|2028.27|132.00| +2451622|51775|3158|57001|58151|5254|42804|7|121|587|89|2.68|4.28|2.18|0.00|194.02|238.52|380.92|15.52|0.00|194.02|209.54|-44.50| +2451622|51775|5519|57001|58151|5254|42804|7|160|587|66|4.17|6.71|5.23|0.00|345.18|275.22|442.86|6.90|0.00|345.18|352.08|69.96| +2451622|51775|6338|57001|58151|5254|42804|7|184|587|43|44.36|70.53|5.64|0.00|242.52|1907.48|3032.79|21.82|0.00|242.52|264.34|-1664.96| +2451622|51775|11563|57001|58151|5254|42804|7|164|587|62|9.78|12.22|8.55|0.00|530.10|606.36|757.64|42.40|0.00|530.10|572.50|-76.26| +2451622|51775|14665|57001||||7|232|587|87||||||688.17||||441.82||| +2451622|51775|9491|57001|58151|5254|42804|7|227|587|65|63.49|76.82|59.15|2499.08|3844.75|4126.85|4993.30|80.74|2499.08|1345.67|1426.41|-2781.18| +2452076|44582|17425|82413|1023135|4304|12673|7|168|588|96|42.37|74.14|41.51|0.00|3984.96|4067.52|7117.44|318.79|0.00|3984.96|4303.75|-82.56| +2452076|44582|11193|82413|1023135|4304|12673|7|43|588|82|38.70|53.01|51.41|3667.58|4215.62|3173.40|4346.82|32.88|3667.58|548.04|580.92|-2625.36| +2452076|44582|7279|82413|1023135|4304|12673|7|57|588|95|94.02|113.76|91.00|0.00|8645.00|8931.90|10807.20|778.05|0.00|8645.00|9423.05|-286.90| +2452076|44582|15075|82413|1023135|4304|12673|7|112|588|5|94.32|128.27|114.16|0.00|570.80|471.60|641.35|11.41|0.00|570.80|582.21|99.20| +2452076|44582|9481|82413|1023135|4304|12673|7|127|588|89|98.16|108.95|34.86|558.45|3102.54|8736.24|9696.55|76.32|558.45|2544.09|2620.41|-6192.15| +2452076|44582|8607|82413|1023135|4304|12673|7|53|588|9|97.84|127.19|109.38|0.00|984.42|880.56|1144.71|9.84|0.00|984.42|994.26|103.86| +2452076|44582|9907||||12673|7||588|18|31.92|56.81||0.00|695.34|574.56|||0.00||730.10|| +2452076|44582|6531|82413|1023135|4304|12673|7|25|588|47|99.33|125.15|82.59|0.00|3881.73|4668.51|5882.05|310.53|0.00|3881.73|4192.26|-786.78| +2452076|44582|997|82413|1023135|4304|12673|7|226|588|78|74.71|125.51|77.81|0.00|6069.18|5827.38|9789.78|485.53|0.00|6069.18|6554.71|241.80| +2452076|44582|419|82413|1023135|4304|12673|7|185|588|12|17.86|18.75|10.12|0.00|121.44|214.32|225.00|0.00|0.00|121.44|121.44|-92.88| +2452076|44582|2779|82413|1023135|4304|12673|7|153|588|18|34.95|54.87|13.71|0.00|246.78|629.10|987.66|2.46|0.00|246.78|249.24|-382.32| +2452076|44582|3709|82413|1023135|4304|12673|7|192|588|24|65.48|115.24|111.78|0.00|2682.72|1571.52|2765.76|187.79|0.00|2682.72|2870.51|1111.20| +2451293|75570|17587|21665|465762|1506|30050|2|296|589|87|98.96|171.20|66.76|1858.59|5808.12|8609.52|14894.40|315.96|1858.59|3949.53|4265.49|-4659.99| +2451293|75570|2900|21665|465762|1506|30050|2|3|589|59|97.38|167.49|60.29|0.00|3557.11|5745.42|9881.91|248.99|0.00|3557.11|3806.10|-2188.31| +2451293|75570|5545|21665|465762|1506|30050|2|268|589|43|46.05|61.24|52.66|0.00|2264.38|1980.15|2633.32|203.79|0.00|2264.38|2468.17|284.23| +2451293|75570|12562|21665|465762|1506|30050|2|130|589|31|49.23|53.16|22.85|0.00|708.35|1526.13|1647.96|7.08|0.00|708.35|715.43|-817.78| +2451293|75570|16885|21665|465762|1506|30050|2|51|589|49|33.60|37.96|6.07|56.51|297.43|1646.40|1860.04|21.68|56.51|240.92|262.60|-1405.48| +2451293|75570|7333|21665|465762|1506|30050|2|167|589|11|64.17|89.19|50.83|0.00|559.13|705.87|981.09|11.18|0.00|559.13|570.31|-146.74| +2451293|75570|9973|21665|465762|1506|30050|2|66|589|22|44.86|54.72|8.75|136.67|192.50|986.92|1203.84|4.46|136.67|55.83|60.29|-931.09| +2451293|75570|12758|21665|465762|1506|30050|2|28|589|48|32.87|63.43|2.53|0.00|121.44|1577.76|3044.64|1.21|0.00|121.44|122.65|-1456.32| +2451293|75570|10375|21665|465762|1506|30050|2|179|589|8|8.76|15.06|9.78|0.00|78.24|70.08|120.48|0.78|0.00|78.24|79.02|8.16| +2451293|75570|3829|21665|465762|1506|30050|2|147|589|42|37.74|64.15|18.60|0.00|781.20|1585.08|2694.30|39.06|0.00|781.20|820.26|-803.88| +2452297|66517|7419|58617|1723255|4246|10084|10|225|590|44|38.95|57.64|8.64|364.95|380.16|1713.80|2536.16|0.91|364.95|15.21|16.12|-1698.59| +2452297|66517|10093|58617|1723255|4246|10084|10|269|590|79|26.98|46.67|43.40|754.29|3428.60|2131.42|3686.93|53.48|754.29|2674.31|2727.79|542.89| +2452297|66517|11089|58617|1723255|4246|10084|10|279|590|4|37.47|41.96|29.79|44.08|119.16|149.88|167.84|5.25|44.08|75.08|80.33|-74.80| +2452297|66517|13911|58617|1723255|4246|10084|10|141|590|3|98.20|136.49|79.16|40.37|237.48|294.60|409.47|5.91|40.37|197.11|203.02|-97.49| +||10203|58617|1723255|||10||590||41.27||33.39||1368.99|1692.07||68.44|||1437.43|-323.08| +2452297|66517|7257|58617|1723255|4246|10084|10|82|590|62|31.70|59.91|5.99|0.00|371.38|1965.40|3714.42|22.28|0.00|371.38|393.66|-1594.02| +2452297|66517|9600|58617|1723255|4246|10084|10|96|590|97|19.27|19.84|2.97|0.00|288.09|1869.19|1924.48|2.88|0.00|288.09|290.97|-1581.10| +2452297|66517|10140|58617|1723255|4246|10084|10|288|590|57|91.86|150.65|43.68|0.00|2489.76|5236.02|8587.05|49.79|0.00|2489.76|2539.55|-2746.26| +2452297|66517|5139|58617|1723255|4246|10084|10|247|590|65|78.72|90.52|57.93|0.00|3765.45|5116.80|5883.80|75.30|0.00|3765.45|3840.75|-1351.35| +2451990|46471|10963|80973|1615349|4125|24948|2|238|591|97|6.46|11.17|1.11|69.98|107.67|626.62|1083.49|2.26|69.98|37.69|39.95|-588.93| +2451990|46471|10059|80973|1615349|4125|24948|2|276|591|11|46.06|47.90|3.83|0.00|42.13|506.66|526.90|2.94|0.00|42.13|45.07|-464.53| +2451990|46471|7951|80973|1615349|4125|24948|2|78|591|73|17.16|30.88|26.55|0.00|1938.15|1252.68|2254.24|155.05|0.00|1938.15|2093.20|685.47| +2451990|46471|16411|80973|1615349|4125|24948|2|7|591|50|89.63|98.59|30.56|397.28|1528.00|4481.50|4929.50|79.15|397.28|1130.72|1209.87|-3350.78| +2451990|46471|13625|80973|1615349|4125|24948|2|241|591|12|76.40|126.06|41.59|0.00|499.08|916.80|1512.72|24.95|0.00|499.08|524.03|-417.72| +2451990|46471|12809|80973|1615349|4125|24948|2|4|591|76|91.02|144.72|70.91|0.00|5389.16|6917.52|10998.72|485.02|0.00|5389.16|5874.18|-1528.36| +2451990|46471|3415|80973|1615349|4125|24948|2|206|591|49|45.88|69.73|69.03|913.26|3382.47|2248.12|3416.77|172.84|913.26|2469.21|2642.05|221.09| +2451990|46471|6223|80973|1615349|4125|24948|2|290|591|87|21.43|27.43|23.86|290.61|2075.82|1864.41|2386.41|107.11|290.61|1785.21|1892.32|-79.20| +2451990|46471|4419|80973|1615349|4125|24948|2|130|591|81|31.31|44.77|16.56|0.00|1341.36|2536.11|3626.37|26.82|0.00|1341.36|1368.18|-1194.75| +2451990|46471|107|80973|1615349|4125|24948|2|76|591|4|19.81|32.29|19.69|0.00|78.76|79.24|129.16|1.57|0.00|78.76|80.33|-0.48| +2451990|46471|277|80973|1615349|4125|24948|2|246|591|64|96.76|132.56|68.93|0.00|4411.52|6192.64|8483.84|88.23|0.00|4411.52|4499.75|-1781.12| +2451990|46471|4343|80973|1615349|4125|24948|2|289|591|79|24.50|39.44|33.12|0.00|2616.48|1935.50|3115.76|26.16|0.00|2616.48|2642.64|680.98| +2451990|46471|621|80973|1615349|4125|24948|2|166|591|19|25.95|39.96|13.18|197.83|250.42|493.05|759.24|0.52|197.83|52.59|53.11|-440.46| +2451887|35374|15254|77377|703501|3225|37749|10|97|592|7|28.70|42.76|24.37|0.00|170.59|200.90|299.32|10.23|0.00|170.59|180.82|-30.31| +2451887|35374|15055|77377|703501|3225|37749|10|248|592|75|89.76|151.69|50.05|0.00|3753.75|6732.00|11376.75|75.07|0.00|3753.75|3828.82|-2978.25| +2451887|35374|6505|77377|703501|3225|37749|10|256|592|31|97.04|146.53|46.88|0.00|1453.28|3008.24|4542.43|14.53|0.00|1453.28|1467.81|-1554.96| +2451887|35374|14234|77377|703501|3225|37749|10|155|592|14|66.01|112.21|108.84|0.00|1523.76|924.14|1570.94|15.23|0.00|1523.76|1538.99|599.62| +2451887|35374|10237|77377|703501|3225|37749|10|113|592|75|46.81|58.04|3.48|75.69|261.00|3510.75|4353.00|0.00|75.69|185.31|185.31|-3325.44| +2451887|35374|2522|77377|703501|3225|37749|10|31|592|50|50.16|77.74|61.41|0.00|3070.50|2508.00|3887.00|30.70|0.00|3070.50|3101.20|562.50| +2451887|35374|8585|77377|703501|3225|37749|10|136|592|61|45.67|80.83|20.20|0.00|1232.20|2785.87|4930.63|24.64|0.00|1232.20|1256.84|-1553.67| +2451887|35374|2090|77377|703501|3225|37749|10|238|592|11|51.92|72.68|21.80|0.00|239.80|571.12|799.48|14.38|0.00|239.80|254.18|-331.32| +||15940|74705|988145|338||7||593|||||0.00||5932.08|||0.00|||| +2451079|67684|6584|74705|988145|338|8978|7|139|593|66|33.67|62.28|52.31|0.00|3452.46|2222.22|4110.48|34.52|0.00|3452.46|3486.98|1230.24| +2451079|67684|10627|74705|988145|338|8978|7|159|593|88|17.20|29.75|23.50|0.00|2068.00|1513.60|2618.00|103.40|0.00|2068.00|2171.40|554.40| +2451079|67684|4088|74705|988145|338|8978|7|86|593|45|3.17|4.97|1.68|0.00|75.60|142.65|223.65|3.02|0.00|75.60|78.62|-67.05| +2451079|67684|6062|74705|988145|338|8978|7|228|593|69|91.40|91.40|4.57|0.00|315.33|6306.60|6306.60|3.15|0.00|315.33|318.48|-5991.27| +2451079|67684|5623|74705|988145|338|8978|7|120|593|30|93.56|150.63|16.56|0.00|496.80|2806.80|4518.90|39.74|0.00|496.80|536.54|-2310.00| +2451079|67684|15988|74705|988145|338|8978|7|77|593|38|90.51|106.80|58.74|0.00|2232.12|3439.38|4058.40|133.92|0.00|2232.12|2366.04|-1207.26| +2451079||7327|74705|988145||8978|7||593|||||0.00|1578.08|1325.52||126.24|0.00|1578.08|1704.32|252.56| +2451079|67684|8360|74705|988145|338|8978|7|249|593|20|96.49|178.50|169.57|0.00|3391.40|1929.80|3570.00|237.39|0.00|3391.40|3628.79|1461.60| +2451079|67684|11680|74705|988145|338|8978|7|100|593|13|65.44|83.76|26.80|0.00|348.40|850.72|1088.88|6.96|0.00|348.40|355.36|-502.32| +2451079|67684|5972|74705|988145|338|8978|7|250|593|50|52.76|66.47|65.80|0.00|3290.00|2638.00|3323.50|164.50|0.00|3290.00|3454.50|652.00| +2451079|67684|8980|74705|988145|338|8978|7|42|593|44|6.23|8.03|4.01|0.00|176.44|274.12|353.32|10.58|0.00|176.44|187.02|-97.68| +2451079|67684|6034|74705|988145|338|8978|7|107|593|70|90.97|130.99|43.22|2995.14|3025.40|6367.90|9169.30|0.00|2995.14|30.26|30.26|-6337.64| +2451079|67684|9139|74705|||||236|593|9||||0.00|||380.25|10.03|0.00||260.95|| +2451079|67684|12676|74705|988145|338|8978|7|192|593|27|94.69|128.77|77.26|0.00|2086.02|2556.63|3476.79|83.44|0.00|2086.02|2169.46|-470.61| +2451532|32660|4537|28784|1571219|407|24042|7|130|594|2|8.52|14.56|2.32|0.00|4.64|17.04|29.12|0.23|0.00|4.64|4.87|-12.40| +2451532|32660|4808|28784|1571219|407|24042|7|253|594|38|32.11|34.67|15.60|0.00|592.80|1220.18|1317.46|35.56|0.00|592.80|628.36|-627.38| +2451532|32660|5798|28784|1571219|407|24042|7|161|594|13|57.65|115.30|87.62|0.00|1139.06|749.45|1498.90|22.78|0.00|1139.06|1161.84|389.61| +2451532|32660|17659|28784|1571219|407|24042|7|245|594|82|80.47|136.79|71.13|0.00|5832.66|6598.54|11216.78|466.61|0.00|5832.66|6299.27|-765.88| +2451532|32660|15001|28784|1571219|407|24042|7|68|594|51|21.89|24.51|6.61|0.00|337.11|1116.39|1250.01|3.37|0.00|337.11|340.48|-779.28| +2451532|32660|11005|28784|1571219|407|24042|7|278|594|30|28.37|32.05|29.80|733.08|894.00|851.10|961.50|6.43|733.08|160.92|167.35|-690.18| +2451532|32660|15644|28784|1571219|407|24042|7|213|594|58|99.02|186.15|74.46|0.00|4318.68|5743.16|10796.70|215.93|0.00|4318.68|4534.61|-1424.48| +2451532|32660|16420|28784|1571219|||7||594|83||53.59|9.64||800.12||4447.97|||800.12||| +2451532|32660|2095|28784|1571219|407|24042|7|2|594|11|77.87|123.03|6.15|16.23|67.65|856.57|1353.33|4.11|16.23|51.42|55.53|-805.15| +2451532|32660|7546|28784|1571219|407|24042|7|13|594|40|33.52|35.19|27.80|44.48|1112.00|1340.80|1407.60|53.37|44.48|1067.52|1120.89|-273.28| +2451126|66204|11875|30219|1855365|2911|35610|2|204|595|4|16.24|29.23|7.59|10.01|30.36|64.96|116.92|1.42|10.01|20.35|21.77|-44.61| +2451126|66204|10214|30219|1855365|2911|35610|2|256|595|92|41.73|65.09|27.98|0.00|2574.16|3839.16|5988.28|25.74|0.00|2574.16|2599.90|-1265.00| +2451126|66204|12446|30219|1855365|2911|35610|2|239|595|69|41.16|62.97|41.56|0.00|2867.64|2840.04|4344.93|229.41|0.00|2867.64|3097.05|27.60| +2451126|66204|244|30219|1855365|2911|35610|2|167|595|57|4.64|8.02|4.89|0.00|278.73|264.48|457.14|11.14|0.00|278.73|289.87|14.25| +2451126|66204|1531|30219|1855365|2911|35610|2|270|595|37|77.95|116.92|0.00|0.00|0.00|2884.15|4326.04|0.00|0.00|0.00|0.00|-2884.15| +2451126|66204|3524|30219|1855365|2911|35610|2|39|595|44|71.71|93.94|67.63|0.00|2975.72|3155.24|4133.36|208.30|0.00|2975.72|3184.02|-179.52| +2451126|66204|13021|30219|1855365|2911|35610|2|260|595|75|75.39|147.01|45.57|0.00|3417.75|5654.25|11025.75|136.71|0.00|3417.75|3554.46|-2236.50| +2451126|66204|17263|30219|1855365|2911|35610|2|53|595|74|75.46|107.90|89.55|927.73|6626.70|5584.04|7984.60|512.90|927.73|5698.97|6211.87|114.93| +2451126|66204|17971|30219|1855365|2911|35610|2|11|595|70|96.44|173.59|69.43|0.00|4860.10|6750.80|12151.30|48.60|0.00|4860.10|4908.70|-1890.70| +2451126|66204|14443|30219|1855365|2911|35610|2|124|595|49|89.81|152.67|67.17|0.00|3291.33|4400.69|7480.83|65.82|0.00|3291.33|3357.15|-1109.36| +2451126|66204|14098|30219|1855365|2911|35610|2|258|595|17|78.13|151.57|39.40|308.10|669.80|1328.21|2576.69|3.61|308.10|361.70|365.31|-966.51| +2451126|66204|15784|30219|1855365|2911|35610|2|120|595|32|12.00|20.28|19.26|0.00|616.32|384.00|648.96|18.48|0.00|616.32|634.80|232.32| +2451126|66204|10693|30219|1855365|2911|35610|2|172|595|92|67.29|103.62|81.85|0.00|7530.20|6190.68|9533.04|0.00|0.00|7530.20|7530.20|1339.52| +2451133|71842|17174|70467|560994|5921|12048|2|231|596|96|1.24|2.28|1.91|0.00|183.36|119.04|218.88|9.16|0.00|183.36|192.52|64.32| +2451133|71842|115|70467|560994|5921|12048|2|245|596|20|57.76|94.72|26.52|0.00|530.40|1155.20|1894.40|10.60|0.00|530.40|541.00|-624.80| +2451133|71842|12895|70467|560994|5921|12048|2|6|596|84|37.77|67.23|65.21|0.00|5477.64|3172.68|5647.32|54.77|0.00|5477.64|5532.41|2304.96| +2451133|71842|13264|70467|560994|5921|12048|2|289|596|47|25.12|37.42|26.94|0.00|1266.18|1180.64|1758.74|25.32|0.00|1266.18|1291.50|85.54| +2451133|71842|7225|70467|560994|5921|12048|2|260|596|91|20.84|26.46|13.49|1178.48|1227.59|1896.44|2407.86|1.96|1178.48|49.11|51.07|-1847.33| +2451133|71842|16736|70467|560994|5921|12048|2|286|596|46|89.78|102.34|1.02|0.00|46.92|4129.88|4707.64|0.46|0.00|46.92|47.38|-4082.96| +2451133|71842|148|70467|560994|5921|12048|2|158|596|25|9.80|12.83|0.25|0.00|6.25|245.00|320.75|0.25|0.00|6.25|6.50|-238.75| +2451133|71842|1688|70467|560994|5921|12048|2|221|596|70|49.17|96.86|90.07|0.00|6304.90|3441.90|6780.20|378.29|0.00|6304.90|6683.19|2863.00| +2451133|71842|17005|70467|560994|5921|12048|2|146|596|81|96.70|127.64|102.11|0.00|8270.91|7832.70|10338.84|744.38|0.00|8270.91|9015.29|438.21| +2451133|71842|14812|70467|560994|5921|12048|2|184|596|37|77.74|104.94|32.53|0.00|1203.61|2876.38|3882.78|108.32|0.00|1203.61|1311.93|-1672.77| +2451133|71842|11350|70467|560994|5921|12048|2|120|596|50|8.40|11.92|3.33|0.00|166.50|420.00|596.00|9.99|0.00|166.50|176.49|-253.50| +2451133|71842|412|70467|560994|5921|12048|2|258|596|73|21.87|39.80|31.44|0.00|2295.12|1596.51|2905.40|91.80|0.00|2295.12|2386.92|698.61| +2451133||12908||||12048|2|152|596|96||||0.00|1519.68||||0.00|1519.68|1641.25|182.40| +2451186|64573|5858|17452|239855|6258|47414|4|64|597|57|68.87|96.41|95.44|0.00|5440.08|3925.59|5495.37|0.00|0.00|5440.08|5440.08|1514.49| +2451186|64573|10108|17452|239855|6258|47414|4|41|597|51|35.78|50.09|4.00|0.00|204.00|1824.78|2554.59|4.08|0.00|204.00|208.08|-1620.78| +2451186|64573|9997|17452|239855|6258|47414|4|33|597|26|63.00|97.02|51.42|0.00|1336.92|1638.00|2522.52|0.00|0.00|1336.92|1336.92|-301.08| +2451186|64573|17246|17452|239855|6258|47414|4|99|597|75|1.22|1.85|1.77|0.00|132.75|91.50|138.75|2.65|0.00|132.75|135.40|41.25| +2451186|64573|9790|17452|239855|6258|47414|4|102|597|20|64.18|120.65|10.85|0.00|217.00|1283.60|2413.00|10.85|0.00|217.00|227.85|-1066.60| +2451186|64573|4144|17452||||4||597||84.25||||||7006.23|61.65|||3144.24|| +2451186|64573|8887|17452|239855|6258|47414|4|129|597|74|16.61|31.06|10.24|750.18|757.76|1229.14|2298.44|0.07|750.18|7.58|7.65|-1221.56| +2451186|64573|6008|17452|||||133|597|62||||0.00|3668.54|||110.05|0.00|||| +2451186|64573|457|17452|239855|6258|47414|4|175|597|90|19.86|38.92|28.41|0.00|2556.90|1787.40|3502.80|204.55|0.00|2556.90|2761.45|769.50| +2451186|64573|11168|17452|239855|6258|47414|4|139|597|24|11.07|19.04|11.23|0.00|269.52|265.68|456.96|8.08|0.00|269.52|277.60|3.84| +2451186|64573|9769|17452|239855|6258|47414|4|147|597|27|65.69|77.51|10.07|0.00|271.89|1773.63|2092.77|24.47|0.00|271.89|296.36|-1501.74| +2451186|64573|7504|17452|239855|6258|47414|4|8|597|67|75.84|75.84|51.57|0.00|3455.19|5081.28|5081.28|138.20|0.00|3455.19|3593.39|-1626.09| +2451186|64573|1888|17452|239855|6258|47414|4|67|597|48|28.85|55.39|7.20|0.00|345.60|1384.80|2658.72|3.45|0.00|345.60|349.05|-1039.20| +2451275|32707|12272|90510|771893|2076|40659|1|197|598|27|74.17|83.81|61.18|0.00|1651.86|2002.59|2262.87|16.51|0.00|1651.86|1668.37|-350.73| +|32707|3262|90510|771893|2076||1||598||||2.95||26.55||156.33|1.32||||-119.61| +2451275|32707|2192|90510|771893|2076|40659|1|187|598|27|61.83|105.72|24.31|0.00|656.37|1669.41|2854.44|52.50|0.00|656.37|708.87|-1013.04| +2451275|32707|10198|90510|771893|2076|40659|1|148|598|30|76.00|100.32|5.01|0.00|150.30|2280.00|3009.60|12.02|0.00|150.30|162.32|-2129.70| +2451275|32707|9931|90510|771893|2076|40659|1|113|598|51|81.90|93.36|16.80|0.00|856.80|4176.90|4761.36|34.27|0.00|856.80|891.07|-3320.10| +2451275|32707|14497|90510|771893|2076|40659|1|169|598|67|92.74|136.32|106.32|0.00|7123.44|6213.58|9133.44|0.00|0.00|7123.44|7123.44|909.86| +2451275|32707|9634|90510|771893|2076|40659|1|163|598|77|9.70|17.65|6.88|0.00|529.76|746.90|1359.05|5.29|0.00|529.76|535.05|-217.14| +2451275|32707|8935|90510|771893|2076|40659|1|49|598|92|65.91|108.75|8.70|0.00|800.40|6063.72|10005.00|48.02|0.00|800.40|848.42|-5263.32| +2451275|32707|5684|90510|771893|2076|40659|1|66|598|16|64.21|84.11|62.24|0.00|995.84|1027.36|1345.76|69.70|0.00|995.84|1065.54|-31.52| +2451275|32707|17702|90510|771893|2076|40659|1|157|598|15|55.94|107.40|70.88|0.00|1063.20|839.10|1611.00|10.63|0.00|1063.20|1073.83|224.10| +2451907|38142|17792|80992|66679|6319|47206|4|64|599|89|4.81|5.91|0.29|0.00|25.81|428.09|525.99|0.51|0.00|25.81|26.32|-402.28| +2451907|38142|5564|80992|66679|6319|47206|4|275|599|100|87.12|94.08|51.74|0.00|5174.00|8712.00|9408.00|413.92|0.00|5174.00|5587.92|-3538.00| +2451907|38142|5765|80992|66679|6319|47206|4|133|599|37|91.93|116.75|10.50|0.00|388.50|3401.41|4319.75|19.42|0.00|388.50|407.92|-3012.91| +2451907|38142|5563|80992|66679|6319|47206|4|235|599|43|96.92|168.64|129.85|0.00|5583.55|4167.56|7251.52|390.84|0.00|5583.55|5974.39|1415.99| +2451907|38142|12907|80992|66679|6319|47206|4|290|599|34|51.76|75.56|50.62|0.00|1721.08|1759.84|2569.04|34.42|0.00|1721.08|1755.50|-38.76| +2451907|38142|12523|80992|66679|6319|47206|4|2|599|36|21.05|31.99|20.79|0.00|748.44|757.80|1151.64|7.48|0.00|748.44|755.92|-9.36| +2451907|38142|4421|80992|66679|6319|47206|4|238|599|99|63.80|65.71|42.71|0.00|4228.29|6316.20|6505.29|169.13|0.00|4228.29|4397.42|-2087.91| +2451907|38142|17951|80992|66679|6319|47206|4|149|599|13|44.57|71.75|33.72|0.00|438.36|579.41|932.75|13.15|0.00|438.36|451.51|-141.05| +2451885|41771|1733|76235|535968|4501|30634|1|77|600|76|29.17|49.88|21.94|0.00|1667.44|2216.92|3790.88|66.69|0.00|1667.44|1734.13|-549.48| +2451885|41771|1670|76235|535968|4501|30634|1|72|600|40|55.13|105.84|95.25|0.00|3810.00|2205.20|4233.60|304.80|0.00|3810.00|4114.80|1604.80| +2451885|41771|5227|76235|535968|4501|30634|1|27|600|83|34.29|53.49|28.88|0.00|2397.04|2846.07|4439.67|119.85|0.00|2397.04|2516.89|-449.03| +2451885|41771|2516|76235|535968|4501|30634|1|31|600|62|41.64|66.62|47.96|0.00|2973.52|2581.68|4130.44|178.41|0.00|2973.52|3151.93|391.84| +2451885|41771|11480|76235|535968|4501|30634|1|29|600|33|74.12|74.86|43.41|515.71|1432.53|2445.96|2470.38|36.67|515.71|916.82|953.49|-1529.14| +2451885|41771|12893|76235|535968|4501|30634|1|37|600|60|94.18|139.38|33.45|983.43|2007.00|5650.80|8362.80|0.00|983.43|1023.57|1023.57|-4627.23| +2451885|41771|2533|76235|535968|4501|30634|1|291|600|3|37.02|64.41|3.86|0.00|11.58|111.06|193.23|0.69|0.00|11.58|12.27|-99.48| +2451885|41771|15865|76235|535968|4501|30634|1|227|600|36|61.03|65.30|3.91|0.00|140.76|2197.08|2350.80|9.85|0.00|140.76|150.61|-2056.32| +2451885|41771|5339|76235|535968|4501|30634|1|60|600|58|75.64|98.33|11.79|0.00|683.82|4387.12|5703.14|13.67|0.00|683.82|697.49|-3703.30| +2451885|41771|12359|76235|535968|4501|30634|1|171|600|14|3.87|4.37|2.49|0.00|34.86|54.18|61.18|3.13|0.00|34.86|37.99|-19.32| +2451885|41771|10267|76235|535968|4501|30634|1|202|600|53|87.38|150.29|22.54|0.00|1194.62|4631.14|7965.37|71.67|0.00|1194.62|1266.29|-3436.52| +2451885|41771|16115|76235|535968|4501|30634|1|203|600|60|60.35|63.36|47.52|0.00|2851.20|3621.00|3801.60|171.07|0.00|2851.20|3022.27|-769.80| +2451885|41771|14777|76235|535968|4501|30634|1|117|600|54|22.02|43.15|27.61|0.00|1490.94|1189.08|2330.10|134.18|0.00|1490.94|1625.12|301.86| +2451885|41771|13883|76235|535968|4501|30634|1|113|600|20|68.99|126.25|124.98|0.00|2499.60|1379.80|2525.00|49.99|0.00|2499.60|2549.59|1119.80| +2451168|55795|8299|51630|104627|6424|9021|7|88|601|68|48.49|92.61|30.56|1246.84|2078.08|3297.32|6297.48|0.00|1246.84|831.24|831.24|-2466.08| +2451168|55795|17162|51630|104627|6424|9021|7|196|601|94|98.69|113.49|79.44|0.00|7467.36|9276.86|10668.06|224.02|0.00|7467.36|7691.38|-1809.50| +2451168|55795|6217|51630|104627|6424|9021|7|31|601|17|23.37|29.21|6.13|58.35|104.21|397.29|496.57|1.83|58.35|45.86|47.69|-351.43| +2451168|55795|7462|51630|104627|6424|9021|7|45|601|95|8.12|12.42|3.10|0.00|294.50|771.40|1179.90|8.83|0.00|294.50|303.33|-476.90| +2451168|55795|13874|51630|104627|6424|9021|7|148|601|2|40.84|81.27|77.20|92.64|154.40|81.68|162.54|3.08|92.64|61.76|64.84|-19.92| +2451168|55795|14810|51630|104627|6424|9021|7|154|601|100|53.18|75.51|72.48|0.00|7248.00|5318.00|7551.00|0.00|0.00|7248.00|7248.00|1930.00| +2451168|55795|6685|51630|104627|6424|9021|7|235|601|88|78.04|135.00|68.85|0.00|6058.80|6867.52|11880.00|121.17|0.00|6058.80|6179.97|-808.72| +2451168|55795|17146|51630|104627|6424|9021|7|37|601|14|84.86|140.86|73.24|0.00|1025.36|1188.04|1972.04|71.77|0.00|1025.36|1097.13|-162.68| +2452104|39739|12645|7990|248731|5919|21107|8|72|602|27|77.93|87.28|10.47|0.00|282.69|2104.11|2356.56|25.44|0.00|282.69|308.13|-1821.42| +2452104|39739|3181|7990|248731|5919|21107|8|291|602|37|43.21|54.44|4.89|94.08|180.93|1598.77|2014.28|5.21|94.08|86.85|92.06|-1511.92| +2452104|39739|3397|7990|248731|5919|21107|8|258|602|85|57.53|78.24|76.67|0.00|6516.95|4890.05|6650.40|260.67|0.00|6516.95|6777.62|1626.90| +2452104|39739|9959|7990|248731|5919|21107|8|288|602|71|66.34|105.48|72.78|0.00|5167.38|4710.14|7489.08|103.34|0.00|5167.38|5270.72|457.24| +2452104|39739|9507|7990|248731|5919|21107|8|285|602|36|88.38|129.91|2.59|0.00|93.24|3181.68|4676.76|7.45|0.00|93.24|100.69|-3088.44| +2452104|39739|2359|7990|248731|5919|21107|8|149|602|8|9.61|13.16|3.02|0.00|24.16|76.88|105.28|1.20|0.00|24.16|25.36|-52.72| +2452104|39739|17647|7990|248731|5919|21107|8|167|602|88|56.20|86.54|81.34|0.00|7157.92|4945.60|7615.52|644.21|0.00|7157.92|7802.13|2212.32| +2452104|39739|12465|7990|248731|5919|21107|8|274|602|54|37.73|52.06|34.35|370.98|1854.90|2037.42|2811.24|29.67|370.98|1483.92|1513.59|-553.50| +2452104|39739|8569|7990|248731|5919|21107|8|150|602|31|79.71|109.20|41.49|0.00|1286.19|2471.01|3385.20|90.03|0.00|1286.19|1376.22|-1184.82| +2452104|39739|16385|7990|248731|5919|21107|8|249|602|86|94.95|116.78|49.04|0.00|4217.44|8165.70|10043.08|84.34|0.00|4217.44|4301.78|-3948.26| +2452104|39739|11371|7990|248731|5919|21107|8|184|602|100|41.53|44.02|20.68|951.28|2068.00|4153.00|4402.00|89.33|951.28|1116.72|1206.05|-3036.28| +2452104|39739|15151|7990|248731|5919|21107|8|58|602|7|47.84|93.76|70.32|0.00|492.24|334.88|656.32|34.45|0.00|492.24|526.69|157.36| +2452104|39739|9615|7990|248731|5919|21107|8|64|602|79|52.16|73.54|17.64|1045.17|1393.56|4120.64|5809.66|27.87|1045.17|348.39|376.26|-3772.25| +2452630|53395|12714|78894|306347|1000|3253|10|119|603|14|63.83|105.95|82.64|0.00|1156.96|893.62|1483.30|11.56|0.00|1156.96|1168.52|263.34| +2452630|53395|6327|78894|306347|1000|3253|10|164|603|19|37.12|44.17|9.27|142.66|176.13|705.28|839.23|2.67|142.66|33.47|36.14|-671.81| +2452630|53395|165|78894|306347|1000|3253|10|241|603|73|98.17|144.30|93.79|0.00|6846.67|7166.41|10533.90|410.80|0.00|6846.67|7257.47|-319.74| +2452630|53395|16122|78894|306347|1000|3253|10|156|603|80|75.78|124.27|88.23|0.00|7058.40|6062.40|9941.60|282.33|0.00|7058.40|7340.73|996.00| +2452630|53395|11904|78894|306347|1000|3253|10|49|603|67|90.67|111.52|3.34|0.00|223.78|6074.89|7471.84|17.90|0.00|223.78|241.68|-5851.11| +2452630|53395|6025|78894|306347|1000|3253|10|118|603|26|58.83|68.24|12.96|0.00|336.96|1529.58|1774.24|16.84|0.00|336.96|353.80|-1192.62| +2452630|53395|16501|78894|306347|1000|3253|10|47|603|34|63.41|110.33|41.92|0.00|1425.28|2155.94|3751.22|14.25|0.00|1425.28|1439.53|-730.66| +2452630|53395|17653|78894|306347|1000|3253|10|5|603|91|64.93|105.18|51.53|0.00|4689.23|5908.63|9571.38|234.46|0.00|4689.23|4923.69|-1219.40| +2452630|53395|7993|78894|306347|1000|3253|10|40|603|3|51.59|81.51|71.72|0.00|215.16|154.77|244.53|10.75|0.00|215.16|225.91|60.39| +2452630|53395|15067|78894|306347|1000|3253|10|83|603|80|90.75|104.36|15.65|0.00|1252.00|7260.00|8348.80|75.12|0.00|1252.00|1327.12|-6008.00| +2452630|53395|907|78894|306347|1000|3253|10|60|603|46|62.79|73.46|59.50|1423.24|2737.00|2888.34|3379.16|65.68|1423.24|1313.76|1379.44|-1574.58| +2452630|53395|1407|78894|306347|1000|3253|10|117|603|5|2.92|5.60|4.08|0.00|20.40|14.60|28.00|1.02|0.00|20.40|21.42|5.80| +2452630|53395|17785|78894|306347|1000|3253|10|63|603|27|92.51|172.06|147.97|0.00|3995.19|2497.77|4645.62|119.85|0.00|3995.19|4115.04|1497.42| +2452630|53395|9627|78894|306347|1000|3253|10|79|603|98|83.28|125.75|23.89|0.00|2341.22|8161.44|12323.50|187.29|0.00|2341.22|2528.51|-5820.22| +2451460|52640|1543|84887|668449|2058|41220|4|200|604|68|51.90|93.42|69.13|0.00|4700.84|3529.20|6352.56|94.01|0.00|4700.84|4794.85|1171.64| +2451460|52640|15514|84887|668449|2058|41220|4|275|604|53|19.49|22.80|1.59|0.00|84.27|1032.97|1208.40|5.05|0.00|84.27|89.32|-948.70| +2451460|52640|3031|84887|668449|2058|41220|4|196|604|32|91.05|172.99|17.29|0.00|553.28|2913.60|5535.68|27.66|0.00|553.28|580.94|-2360.32| +2451460|52640|16676|84887|668449|2058|41220|4|261|604|55|65.91|94.25|49.95|0.00|2747.25|3625.05|5183.75|109.89|0.00|2747.25|2857.14|-877.80| +2451460|52640|4525|84887|668449|2058|41220|4|215|604|20|85.02|121.57|105.76|719.16|2115.20|1700.40|2431.40|27.92|719.16|1396.04|1423.96|-304.36| +2451460|52640|4214|||2058|||71|604||22.88|26.08|4.69|0.00||91.52|||0.00|18.76|18.76|| +2451460|52640|991|84887|668449|2058|41220|4|45|604|42|87.37|120.57|12.05|0.00|506.10|3669.54|5063.94|45.54|0.00|506.10|551.64|-3163.44| +2451460|52640|16279|84887|668449|2058|41220|4|96|604|15|35.85|52.69|52.16|0.00|782.40|537.75|790.35|0.00|0.00|782.40|782.40|244.65| +2451460|52640|16489|84887|668449|2058|41220|4|254|604|16|64.78|98.46|11.81|0.00|188.96|1036.48|1575.36|7.55|0.00|188.96|196.51|-847.52| +2451460|52640|1816|84887|668449|2058|41220|4|91|604|80|16.96|27.81|6.39|194.25|511.20|1356.80|2224.80|9.50|194.25|316.95|326.45|-1039.85| +2451460|52640|7573|84887|668449|2058|41220|4|187|604|3|79.74|101.26|97.20|0.00|291.60|239.22|303.78|20.41|0.00|291.60|312.01|52.38| +2451460|52640|7520|84887|668449|2058|41220|4|212|604|54|40.50|59.94|37.16|0.00|2006.64|2187.00|3236.76|160.53|0.00|2006.64|2167.17|-180.36| +2451460|52640|5794|84887|668449|2058|41220|4|127|604|1|6.95|9.52|7.80|0.00|7.80|6.95|9.52|0.31|0.00|7.80|8.11|0.85| +2451769|30261|2516|34912|278766|3420|33627|4|113|605|14|30.65|38.92|3.11|0.00|43.54|429.10|544.88|3.91|0.00|43.54|47.45|-385.56| +2451769|30261|11480|34912|278766|3420|33627|4|226|605|36|67.75|120.59|54.26|1191.54|1953.36|2439.00|4341.24|0.00|1191.54|761.82|761.82|-1677.18| +2451769|30261|12893|34912|278766|3420|33627|4|65|605|98|66.76|118.16|30.72|0.00|3010.56|6542.48|11579.68|210.73|0.00|3010.56|3221.29|-3531.92| +2451769|30261|2533|34912|278766|3420|33627|4|25|605|76|5.00|7.35|4.18|212.84|317.68|380.00|558.60|5.24|212.84|104.84|110.08|-275.16| +2451769|30261|15865|34912|278766|3420|33627|4|15|605|16|23.88|39.40|18.91|0.00|302.56|382.08|630.40|18.15|0.00|302.56|320.71|-79.52| +2451769|30261|5339|34912|278766|3420|33627|4|129|605|53|3.99|7.22|5.41|0.00|286.73|211.47|382.66|0.00|0.00|286.73|286.73|75.26| +2451769|30261|12359|34912|278766|3420|33627|4|90|605|47|30.96|33.74|3.71|0.00|174.37|1455.12|1585.78|3.48|0.00|174.37|177.85|-1280.75| +2451769|30261|10267|34912|278766|3420|33627|4|174|605|19|2.92|5.46|3.27|60.26|62.13|55.48|103.74|0.03|60.26|1.87|1.90|-53.61| +|30261|16115||||33627|||605||79.36|156.33||0.00|||||0.00|||| +2451769||14777||||33627|||605|14||||0.00|188.58|137.76||16.97|0.00|||50.82| +2450883|62689|13528|90374|1801337|3947|28910|1|11|606|18|52.71|54.29|51.03|0.00|918.54|948.78|977.22|55.11|0.00|918.54|973.65|-30.24| +2450883|62689|17300|90374|1801337|3947|28910|1|182|606|52|8.26|10.57|5.60|81.53|291.20|429.52|549.64|12.58|81.53|209.67|222.25|-219.85| +2450883|62689|9592|90374|1801337|3947|28910|1|299|606|61|79.82|125.31|61.40|0.00|3745.40|4869.02|7643.91|0.00|0.00|3745.40|3745.40|-1123.62| +2450883|62689|938|90374|1801337|3947|28910|1|42|606|82|6.40|11.96|2.51|0.00|205.82|524.80|980.72|18.52|0.00|205.82|224.34|-318.98| +2450883|62689|205|90374|1801337|3947|28910|1|243|606|97|85.40|126.39|77.09|5234.41|7477.73|8283.80|12259.83|112.16|5234.41|2243.32|2355.48|-6040.48| +2450883|62689|12194|90374|1801337|3947|28910|1|273|606|87|6.97|8.99|0.53|0.00|46.11|606.39|782.13|3.68|0.00|46.11|49.79|-560.28| +2450883|62689|6418|90374|1801337|3947|28910|1|181|606|53|69.99|90.98|40.94|0.00|2169.82|3709.47|4821.94|65.09|0.00|2169.82|2234.91|-1539.65| +2450883|62689|12254|90374|1801337|3947|28910|1|63|606|37|72.78|144.10|37.46|0.00|1386.02|2692.86|5331.70|69.30|0.00|1386.02|1455.32|-1306.84| +2451897|71251|2378|34423|321846|6869|37492|8|251|607|100|82.24|153.78|72.27|0.00|7227.00|8224.00|15378.00|361.35|0.00|7227.00|7588.35|-997.00| +2451897|71251|16873|34423|321846|6869|37492|8|259|607|67|83.88|136.72|68.36|0.00|4580.12|5619.96|9160.24|45.80|0.00|4580.12|4625.92|-1039.84| +2451897|71251|12644|34423|321846|6869|37492|8|278|607|77|80.81|151.11|89.15|5354.34|6864.55|6222.37|11635.47|0.00|5354.34|1510.21|1510.21|-4712.16| +2451897|71251|3181|34423|321846|6869|37492|8|215|607|9|34.35|57.70|27.69|0.00|249.21|309.15|519.30|17.44|0.00|249.21|266.65|-59.94| +|71251|3397|34423|321846|6869||8|160|607||5.28||||38.50||560.56|3.08|||41.58|| +2451897|71251|9959|34423|321846|6869|37492|8|18|607|54|93.68|126.46|59.43|0.00|3209.22|5058.72|6828.84|256.73|0.00|3209.22|3465.95|-1849.50| +2451897|71251|9506|34423|321846|6869|37492|8|296|607|19|82.29|109.44|63.47|0.00|1205.93|1563.51|2079.36|108.53|0.00|1205.93|1314.46|-357.58| +2451897|71251|2359|34423|321846|6869|37492|8|15|607|96|64.54|128.43|120.72|0.00|11589.12|6195.84|12329.28|811.23|0.00|11589.12|12400.35|5393.28| +2451897|71251|17647|34423|321846|6869|37492|8|120|607|52|15.96|28.24|25.69|0.00|1335.88|829.92|1468.48|66.79|0.00|1335.88|1402.67|505.96| +2451511|53896|5342|66709|1177369|1982|45795|2|85|608|51|49.47|66.28|17.89|538.31|912.39|2522.97|3380.28|0.00|538.31|374.08|374.08|-2148.89| +2451511|53896|5852|66709|1177369|1982|45795|2|230|608|46|35.74|62.54|21.88|0.00|1006.48|1644.04|2876.84|60.38|0.00|1006.48|1066.86|-637.56| +2451511|53896|14668|66709|1177369|1982|45795|2|110|608|96|21.07|38.76|37.59|0.00|3608.64|2022.72|3720.96|36.08|0.00|3608.64|3644.72|1585.92| +2451511|53896|3910|66709|1177369|1982|45795|2|266|608|88|45.34|52.59|22.61|0.00|1989.68|3989.92|4627.92|99.48|0.00|1989.68|2089.16|-2000.24| +2451511|53896|2923|66709|1177369|1982|45795|2|211|608|56|97.34|104.15|46.86|0.00|2624.16|5451.04|5832.40|52.48|0.00|2624.16|2676.64|-2826.88| +2451511|53896|3344|66709|1177369|1982|45795|2|143|608|88|51.78|103.56|77.67|0.00|6834.96|4556.64|9113.28|273.39|0.00|6834.96|7108.35|2278.32| +2451511|53896|11770|66709|1177369|1982|45795|2|92|608|42|5.06|8.70|6.78|0.00|284.76|212.52|365.40|25.62|0.00|284.76|310.38|72.24| +2451511|53896|5773|66709|1177369|1982|45795|2|259|608|96|41.64|79.11|58.54|0.00|5619.84|3997.44|7594.56|112.39|0.00|5619.84|5732.23|1622.40| +2451511|53896|4690|66709|1177369|1982|45795|2|42|608|80|53.43|58.23|44.83|1255.24|3586.40|4274.40|4658.40|23.31|1255.24|2331.16|2354.47|-1943.24| +2451511|53896|4364|66709|1177369|1982|45795|2|211|608|61|26.27|35.20|10.91|0.00|665.51|1602.47|2147.20|59.89|0.00|665.51|725.40|-936.96| +2451511|53896|15451|66709|1177369|1982|45795|2|108|608|19|89.63|169.40|101.64|0.00|1931.16|1702.97|3218.60|115.86|0.00|1931.16|2047.02|228.19| +2451511|53896|13843|66709|1177369|1982|45795|2|282|608|37|45.75|60.84|35.28|0.00|1305.36|1692.75|2251.08|52.21|0.00|1305.36|1357.57|-387.39| +2451511|53896|4738|66709|1177369|1982|45795|2|30|608|37|14.23|26.18|20.42|0.00|755.54|526.51|968.66|15.11|0.00|755.54|770.65|229.03| +2451511|53896|16333|66709|1177369|1982|45795|2|153|608|4|68.96|133.78|1.33|0.00|5.32|275.84|535.12|0.15|0.00|5.32|5.47|-270.52| +2451511|53896|266|66709|1177369|1982|45795|2|3|608|92|63.68|121.62|10.94|0.00|1006.48|5858.56|11189.04|80.51|0.00|1006.48|1086.99|-4852.08| +2451511|53896|2863|66709|1177369|1982|45795|2|72|608|13|19.11|28.09|22.47|0.00|292.11|248.43|365.17|5.84|0.00|292.11|297.95|43.68| +2452642|48766|5281|56011|1301732|1053|10821|10|126|609|69|99.61|109.57|48.21|0.00|3326.49|6873.09|7560.33|166.32|0.00|3326.49|3492.81|-3546.60| +2452642|48766|9786|56011|1301732|1053|10821|10|44|609|66|36.43|56.83|43.19|0.00|2850.54|2404.38|3750.78|171.03|0.00|2850.54|3021.57|446.16| +2452642|48766|7989|56011|1301732|1053|10821|10|49|609|28|1.65|2.92|2.01|0.00|56.28|46.20|81.76|4.50|0.00|56.28|60.78|10.08| +2452642|48766|17871|56011|1301732|1053|10821|10|30|609|29|13.26|15.77|6.62|0.00|191.98|384.54|457.33|15.35|0.00|191.98|207.33|-192.56| +2452642|48766|6457|56011|1301732||||28|609|93|53.96|79.32||0.00||5018.28|7376.76|11.75|0.00|||-4871.34| +2452642|48766|2076|56011|1301732|1053|10821|10|291|609|9|15.62|30.61|19.28|0.00|173.52|140.58|275.49|1.73|0.00|173.52|175.25|32.94| +2452642|48766|10839|56011|1301732|1053|10821|10|263|609|21|47.48|72.64|0.00|0.00|0.00|997.08|1525.44|0.00|0.00|0.00|0.00|-997.08| +2452642|48766|14412|56011|1301732|1053|10821|10|28|609|55|69.52|132.08|58.11|0.00|3196.05|3823.60|7264.40|255.68|0.00|3196.05|3451.73|-627.55| +2452642|48766|17689|56011|1301732|1053|10821|10|193|609|7|14.58|23.47|9.38|0.00|65.66|102.06|164.29|3.28|0.00|65.66|68.94|-36.40| +2452642|48766|11004|56011|1301732|1053|10821|10|270|609|80|29.10|29.97|2.09|0.00|167.20|2328.00|2397.60|6.68|0.00|167.20|173.88|-2160.80| +2452642|48766|4905|56011|1301732|1053|10821|10|219|609|87|16.72|33.44|14.71|0.00|1279.77|1454.64|2909.28|0.00|0.00|1279.77|1279.77|-174.87| +2452642|48766|4542|56011|1301732|1053|10821|10|135|609|82|92.95|122.69|76.06|0.00|6236.92|7621.90|10060.58|374.21|0.00|6236.92|6611.13|-1384.98| +2452642|48766|9051|56011|1301732|1053|10821|10|112|609|74|46.21|57.30|19.48|0.00|1441.52|3419.54|4240.20|57.66|0.00|1441.52|1499.18|-1978.02| +2452642|48766|1561|56011|1301732|1053|10821|10|90|609|14|23.71|39.59|19.79|0.00|277.06|331.94|554.26|0.00|0.00|277.06|277.06|-54.88| +2452642|48766|3042|56011|1301732|1053|10821|10|229|609|19|92.24|121.75|107.14|814.26|2035.66|1752.56|2313.25|109.92|814.26|1221.40|1331.32|-531.16| +2452642|48766|14859|56011|1301732|1053|10821|10|23|609|51|60.34|111.62|109.38|0.00|5578.38|3077.34|5692.62|167.35|0.00|5578.38|5745.73|2501.04| +2450893|50782|13216|92557|1790561|4118|19275|7|245|610|69|74.84|81.57|66.07|0.00|4558.83|5163.96|5628.33|45.58|0.00|4558.83|4604.41|-605.13| +2450893|50782|2908|92557|1790561|4118|19275|7|28|610|35|7.87|9.75|0.09|0.00|3.15|275.45|341.25|0.22|0.00|3.15|3.37|-272.30| +2450893|50782|6440|92557|1790561|4118|19275|7|133|610|93|85.90|98.78|55.31|0.00|5143.83|7988.70|9186.54|102.87|0.00|5143.83|5246.70|-2844.87| +2450893|50782|6794|92557|||19275|||610|78||31.75|24.76|0.00||||135.18|0.00|1931.28|2066.46|| +2450893|50782|17713|92557|1790561|4118|19275|7|151|610|85|35.37|50.57|28.82|0.00|2449.70|3006.45|4298.45|0.00|0.00|2449.70|2449.70|-556.75| +2450893|50782|794|92557|1790561|4118|19275|7|87|610|50|83.14|91.45|52.12|0.00|2606.00|4157.00|4572.50|208.48|0.00|2606.00|2814.48|-1551.00| +2450893||1129|92557||4118||7|12|610||11.53|22.02||0.00|||||0.00|0.00||-1083.82| +2450893|50782|17527|92557|1790561|4118|19275|7|232|610|73|2.33|4.38|3.72|0.00|271.56|170.09|319.74|10.86|0.00|271.56|282.42|101.47| +2450893|50782|6472|92557|1790561|4118|19275|7|265|610|43|69.86|85.92|16.32|0.00|701.76|3003.98|3694.56|49.12|0.00|701.76|750.88|-2302.22| +2450893|50782|15488|92557|1790561|4118|19275|7|27|610|88|40.10|55.33|27.11|0.00|2385.68|3528.80|4869.04|95.42|0.00|2385.68|2481.10|-1143.12| +2450893|50782|2473|92557|1790561|4118|19275|7|100|610|26|60.00|120.00|46.80|0.00|1216.80|1560.00|3120.00|0.00|0.00|1216.80|1216.80|-343.20| +2450893|50782|1459|92557|1790561|4118|19275|7|74|610|45|3.74|6.95|1.52|0.00|68.40|168.30|312.75|1.36|0.00|68.40|69.76|-99.90| +2450893|50782|1142|92557|1790561|4118|19275|7|113|610|13|89.00|110.36|91.59|464.36|1190.67|1157.00|1434.68|14.52|464.36|726.31|740.83|-430.69| +2450893|50782|17956|92557|1790561|4118|19275|7|60|610|19|40.90|78.93|69.45|501.42|1319.55|777.10|1499.67|32.72|501.42|818.13|850.85|41.03| +2450893|50782|13868|92557|1790561|4118|19275|7|72|610|76|60.87|94.34|43.39|2868.94|3297.64|4626.12|7169.84|12.86|2868.94|428.70|441.56|-4197.42| +2450893|50782|6151|92557|1790561|4118|19275|7|222|610|58|31.46|62.60|10.01|0.00|580.58|1824.68|3630.80|17.41|0.00|580.58|597.99|-1244.10| +2451987|36046|8007|78290|581159|381|27250|8|131|611|22|66.61|66.61|66.61|0.00|1465.42|1465.42|1465.42|87.92|0.00|1465.42|1553.34|0.00| +2451987|36046|16377|78290|581159|381|27250|8|233|611|69|27.39|47.11|46.16|0.00|3185.04|1889.91|3250.59|191.10|0.00|3185.04|3376.14|1295.13| +2451987|36046|10401|78290|581159|381|27250|8|40|611|17|44.21|53.93|30.74|0.00|522.58|751.57|916.81|26.12|0.00|522.58|548.70|-228.99| +2451987|36046|15335|78290|581159|381|27250|8|52|611|25|39.42|40.99|13.93|0.00|348.25|985.50|1024.75|17.41|0.00|348.25|365.66|-637.25| +2451987|36046|12665|78290|581159|381|27250|8|1|611|43|86.68|151.69|83.42|0.00|3587.06|3727.24|6522.67|322.83|0.00|3587.06|3909.89|-140.18| +2451987|36046|15769|78290|581159|381|27250|8|124|611|61|72.85|75.76|36.36|0.00|2217.96|4443.85|4621.36|44.35|0.00|2217.96|2262.31|-2225.89| +2451987|36046|10771|78290|581159|381|27250|8|41|611|81|69.75|131.82|18.45|0.00|1494.45|5649.75|10677.42|0.00|0.00|1494.45|1494.45|-4155.30| +2451987|36046|14791|78290|581159|381|27250|8|246|611|96|48.23|73.30|43.24|0.00|4151.04|4630.08|7036.80|83.02|0.00|4151.04|4234.06|-479.04| +2451987|36046|7917|78290|581159|381|27250|8|107|611|95|18.90|25.70|23.64|0.00|2245.80|1795.50|2441.50|0.00|0.00|2245.80|2245.80|450.30| +2451987|36046|11087|78290|581159|381|27250|8|200|611|4|66.37|117.47|35.24|0.00|140.96|265.48|469.88|0.00|0.00|140.96|140.96|-124.52| +2452194|45603|14023|20558|1705918|4878|3438|2|289|612|34|5.98|8.91|4.72|99.49|160.48|203.32|302.94|3.65|99.49|60.99|64.64|-142.33| +2452194|45603|2899|20558|1705918|4878|3438|2|203|612|17|68.43|131.38|131.38|0.00|2233.46|1163.31|2233.46|44.66|0.00|2233.46|2278.12|1070.15| +2452194|45603|15069|20558|1705918|4878|3438|2|281|612|78|96.41|112.79|108.27|0.00|8445.06|7519.98|8797.62|760.05|0.00|8445.06|9205.11|925.08| +2452194|45603|1281|20558|1705918|4878|3438|2|195|612|48|71.90|80.52|58.77|2313.18|2820.96|3451.20|3864.96|40.62|2313.18|507.78|548.40|-2943.42| +2452194|45603|16081|20558|1705918|4878|3438|2|259|612|57|78.21|93.85|73.20|0.00|4172.40|4457.97|5349.45|166.89|0.00|4172.40|4339.29|-285.57| +2452194|45603|1313|20558|1705918|4878|3438|2|106|612|97|11.37|17.96|9.33|0.00|905.01|1102.89|1742.12|27.15|0.00|905.01|932.16|-197.88| +2452194|45603|7135|20558|1705918|4878|3438|2|292|612|5|64.35|128.05|64.02|291.29|320.10|321.75|640.25|2.30|291.29|28.81|31.11|-292.94| +2452194|45603|1985|20558|1705918|4878|3438|2|181|612|58|39.48|41.84|22.59|0.00|1310.22|2289.84|2426.72|26.20|0.00|1310.22|1336.42|-979.62| +2452194|45603|5023|20558|1705918|4878|3438|2|19|612|33|46.57|49.82|44.83|1346.24|1479.39|1536.81|1644.06|2.66|1346.24|133.15|135.81|-1403.66| +2452194|45603|1909|20558|1705918|4878|3438|2|278|612|76|66.58|81.22|24.36|0.00|1851.36|5060.08|6172.72|18.51|0.00|1851.36|1869.87|-3208.72| +||3097|20558|||||143|612|6||22.18|2.88|0.00|17.28|122.10|||0.00|||-104.82| +2452143|32809|4375|65930|793708|2286|18017|8|4|613|12|98.25|121.83|113.30|0.00|1359.60|1179.00|1461.96|81.57|0.00|1359.60|1441.17|180.60| +2452143|32809|13871|65930|793708|2286|18017|8|245|613|17|83.94|83.94|49.52|404.08|841.84|1426.98|1426.98|21.88|404.08|437.76|459.64|-989.22| +2452143|32809|1769|65930|793708|2286|18017|8|235|613|67|9.64|17.35|16.13|0.00|1080.71|645.88|1162.45|75.64|0.00|1080.71|1156.35|434.83| +2452143|32809|261|65930|793708|2286|18017|8|109|613|73|41.11|81.80|12.27|0.00|895.71|3001.03|5971.40|44.78|0.00|895.71|940.49|-2105.32| +2452143|32809|9341|65930|793708|2286|18017|8|106|613|88|83.63|147.18|66.23|0.00|5828.24|7359.44|12951.84|116.56|0.00|5828.24|5944.80|-1531.20| +2452143|32809|4247|65930|793708|2286|18017|8|74|613|57|54.89|85.07|55.29|0.00|3151.53|3128.73|4848.99|0.00|0.00|3151.53|3151.53|22.80| +2452143|32809|17525|65930|793708|2286|18017|8|199|613|49|59.54|118.48|118.48|0.00|5805.52|2917.46|5805.52|0.00|0.00|5805.52|5805.52|2888.06| +2452143|32809|16969|65930|793708|2286|18017|8|5|613|46|28.48|49.84|12.95|434.86|595.70|1310.08|2292.64|12.86|434.86|160.84|173.70|-1149.24| +2452143|32809|7063|65930|793708|2286|18017|8|272|613|46|78.03|119.38|66.85|0.00|3075.10|3589.38|5491.48|184.50|0.00|3075.10|3259.60|-514.28| +2452143|32809|1625|65930|793708|2286|18017|8|272|613|99|36.04|43.60|41.42|0.00|4100.58|3567.96|4316.40|205.02|0.00|4100.58|4305.60|532.62| +2452143|32809|17371|65930|793708|2286|18017|8|290|613|6|91.62|106.27|59.51|0.00|357.06|549.72|637.62|10.71|0.00|357.06|367.77|-192.66| +2451630|40341|10418|68219|555070|2274|27078|2|269|614|63|55.75|83.06|73.92|0.00|4656.96|3512.25|5232.78|232.84|0.00|4656.96|4889.80|1144.71| +2451630|40341|16574|68219|555070|2274|27078|2|263|614|63|45.31|45.76|6.40|0.00|403.20|2854.53|2882.88|8.06|0.00|403.20|411.26|-2451.33| +||7565|68219|555070|||2|167|614||43.25|||0.00|1712.21|||51.36|0.00|1712.21||-320.54| +||17927||555070||27078|||614||7.94||||||140.50|||||| +2451630|40341|12350|68219|555070|2274|27078|2|131|614|93|71.17|116.71|17.50|0.00|1627.50|6618.81|10854.03|113.92|0.00|1627.50|1741.42|-4991.31| +2451630|40341|1439|68219|555070|2274|27078|2|112|614|65|12.96|17.62|8.63|0.00|560.95|842.40|1145.30|50.48|0.00|560.95|611.43|-281.45| +2451630|40341|4387|68219|555070|2274|27078|2|97|614|46|7.75|10.07|2.31|0.00|106.26|356.50|463.22|4.25|0.00|106.26|110.51|-250.24| +2451630|40341|4849|68219|555070|2274|27078|2|84|614|99|12.42|24.09|5.05|314.96|499.95|1229.58|2384.91|7.39|314.96|184.99|192.38|-1044.59| +2451630|40341|6925|68219|555070|2274|27078|2|141|614|64|26.76|28.36|11.91|457.34|762.24|1712.64|1815.04|12.19|457.34|304.90|317.09|-1407.74| +2452620|30892|8845|99054|726734|2414|45694|7|211|615|8|50.07|71.09|9.24|0.00|73.92|400.56|568.72|2.95|0.00|73.92|76.87|-326.64| +2452620|30892|13893|99054|726734|2414|45694|7|227|615|68|6.70|7.57|6.20|0.00|421.60|455.60|514.76|29.51|0.00|421.60|451.11|-34.00| +2452620|30892|10695|99054|726734|2414|45694|7|166|615|61|15.65|15.80|12.95|0.00|789.95|954.65|963.80|39.49|0.00|789.95|829.44|-164.70| +2452620|30892|4899|99054|726734|2414|45694|7|267|615|26|47.02|74.76|44.10|0.00|1146.60|1222.52|1943.76|0.00|0.00|1146.60|1146.60|-75.92| +2452620|30892|9330|99054|726734|2414|45694|7|7|615|69|60.04|75.65|37.06|0.00|2557.14|4142.76|5219.85|102.28|0.00|2557.14|2659.42|-1585.62| +2452620|30892|14269|99054|726734|2414|45694|7|153|615|75|92.71|114.03|28.50|0.00|2137.50|6953.25|8552.25|106.87|0.00|2137.50|2244.37|-4815.75| +2452620|30892|2067|99054|726734|2414|45694|7|82|615|2|96.75|107.39|94.50|0.00|189.00|193.50|214.78|7.56|0.00|189.00|196.56|-4.50| +2452620|30892|16005|99054|726734|2414|45694|7|181|615|69|18.48|22.73|15.22|0.00|1050.18|1275.12|1568.37|84.01|0.00|1050.18|1134.19|-224.94| +2452620|30892|259|99054|726734|2414|45694|7|80|615|54|30.75|31.36|3.13|0.00|169.02|1660.50|1693.44|0.00|0.00|169.02|169.02|-1491.48| +2452620|30892|17688|99054|726734|2414|45694|7|51|615|49|74.66|79.88|73.48|0.00|3600.52|3658.34|3914.12|72.01|0.00|3600.52|3672.53|-57.82| +2452620|30892|11281|99054|726734|2414|45694|7|112|615|51|33.70|47.85|17.22|570.84|878.22|1718.70|2440.35|18.44|570.84|307.38|325.82|-1411.32| +2452620|30892|1359|99054|726734|2414|45694|7|128|615|53|9.33|13.15|7.36|0.00|390.08|494.49|696.95|27.30|0.00|390.08|417.38|-104.41| +2452620|30892|8205|99054|726734|2414|45694|7|234|615|41|22.23|31.56|23.98|0.00|983.18|911.43|1293.96|39.32|0.00|983.18|1022.50|71.75| +2451511|67458|15884|81555|995713|5071|12829|7|190|616|18|10.14|17.23|6.37|0.00|114.66|182.52|310.14|4.58|0.00|114.66|119.24|-67.86| +|67458|11512|||5071|||12|616|23|1.04|2.02|1.69||38.87|23.92||0.38|||39.25|14.95| +2451511|67458|14084|81555|995713|5071|12829|7|25|616|15|49.75|89.05|69.45|0.00|1041.75|746.25|1335.75|31.25|0.00|1041.75|1073.00|295.50| +2451511|67458|2617|81555|995713|5071|12829|7|293|616|36|46.60|64.77|26.55|76.46|955.80|1677.60|2331.72|43.96|76.46|879.34|923.30|-798.26| +2451511|67458|2744|81555|995713|5071|12829|7|189|616|38|81.89|140.03|100.82|612.98|3831.16|3111.82|5321.14|96.54|612.98|3218.18|3314.72|106.36| +2451511|67458|17768|81555|995713|5071|12829|7|75|616|12|24.38|38.27|8.80|0.00|105.60|292.56|459.24|9.50|0.00|105.60|115.10|-186.96| +2451511|67458|7316|81555|995713|5071|12829|7|101|616|84|4.69|5.95|1.01|0.00|84.84|393.96|499.80|7.63|0.00|84.84|92.47|-309.12| +2451511|67458|14563|81555|995713|5071|12829|7|236|616|24|31.63|61.04|35.40|0.00|849.60|759.12|1464.96|59.47|0.00|849.60|909.07|90.48| +2451511|67458|6844|81555||5071|12829|7|219|616||73.94||||137.72|||12.39||137.72||-675.62| +2451511|67458|3794|81555|995713|5071|12829|7|116|616|45|27.98|36.65|30.05|1311.68|1352.25|1259.10|1649.25|3.24|1311.68|40.57|43.81|-1218.53| +2451511|67458|11978|81555|995713|5071|12829|7|193|616|82|82.34|88.10|34.35|0.00|2816.70|6751.88|7224.20|140.83|0.00|2816.70|2957.53|-3935.18| +2451511|67458|9463|81555|995713|5071|12829|7|86|616|18|60.79|110.63|55.31|0.00|995.58|1094.22|1991.34|79.64|0.00|995.58|1075.22|-98.64| +2451511|67458|11552|81555|995713|5071|12829|7|276|616|87|31.55|32.49|26.31|0.00|2288.97|2744.85|2826.63|160.22|0.00|2288.97|2449.19|-455.88| +2451511|67458|430|||5071|12829|||616|96||147.76|||850.56||14184.96|||850.56||-6986.88| +2451511|67458|7144|81555|995713|5071|12829|7|8|616|58|10.22|16.14|8.07|0.00|468.06|592.76|936.12|37.44|0.00|468.06|505.50|-124.70| +2451882|64705|4364|18024|1275696|6615|4734|4|234|617|57|4.17|7.92|0.95|0.00|54.15|237.69|451.44|4.87|0.00|54.15|59.02|-183.54| +2451882|64705|15451|18024|1275696|6615|4734|4|189|617|48|70.25|74.46|25.31|0.00|1214.88|3372.00|3574.08|36.44|0.00|1214.88|1251.32|-2157.12| +2451882|64705|13843|18024|1275696|6615|4734|4|298|617|29|52.23|60.06|9.60|0.00|278.40|1514.67|1741.74|0.00|0.00|278.40|278.40|-1236.27| +2451882|64705|4739|18024|1275696|6615|4734|4|67|617|33|15.91|29.27|3.51|0.00|115.83|525.03|965.91|3.47|0.00|115.83|119.30|-409.20| +2451882|64705|16333|18024|1275696|6615|4734|4|139|617|60|78.76|110.26|97.02|0.00|5821.20|4725.60|6615.60|232.84|0.00|5821.20|6054.04|1095.60| +2451882|64705|266|18024|1275696|6615|4734|4|16|617|32|62.36|87.30|2.61|0.00|83.52|1995.52|2793.60|4.17|0.00|83.52|87.69|-1912.00| +2451882|64705|2863|18024|1275696|6615|4734|4|299|617|22|20.31|23.15|12.50|0.00|275.00|446.82|509.30|11.00|0.00|275.00|286.00|-171.82| +2451882|64705|14339|18024|1275696|6615|4734|4|176|617|70|71.63|95.98|23.99|0.00|1679.30|5014.10|6718.60|151.13|0.00|1679.30|1830.43|-3334.80| +2451882|64705|3707|18024|1275696|6615|4734|4|274|617|44|88.22|163.20|3.26|0.00|143.44|3881.68|7180.80|10.04|0.00|143.44|153.48|-3738.24| +2451882|64705|8198|18024|1275696|6615|4734|4|285|617|39|32.84|63.70|62.42|0.00|2434.38|1280.76|2484.30|0.00|0.00|2434.38|2434.38|1153.62| +2451882|64705|14450|18024|1275696|6615|4734|4|51|617|57|41.42|56.33|31.54|0.00|1797.78|2360.94|3210.81|161.80|0.00|1797.78|1959.58|-563.16| +2451882|64705|8819|18024|1275696|6615|4734|4|96|617|6|74.97|109.45|20.79|0.00|124.74|449.82|656.70|2.49|0.00|124.74|127.23|-325.08| +2451882|64705|17270|18024|1275696|6615|4734|4|202|617|54|78.86|145.10|74.00|0.00|3996.00|4258.44|7835.40|79.92|0.00|3996.00|4075.92|-262.44| +2451882|64705|12751|18024|1275696|6615|4734|4|53|617|100|1.37|2.52|2.31|0.00|231.00|137.00|252.00|13.86|0.00|231.00|244.86|94.00| +2451882|64705|12127|18024|1275696|6615|4734|4|125|617|84|62.67|119.69|41.89|0.00|3518.76|5264.28|10053.96|105.56|0.00|3518.76|3624.32|-1745.52| +2451882|64705|8197|18024|1275696|6615|4734|4|60|617|4|62.21|71.54|11.44|0.00|45.76|248.84|286.16|0.91|0.00|45.76|46.67|-203.08| +2451656|55896|14042|53238|1808369|6347|21078|8|215|618|79|39.83|43.41|1.73|0.00|136.67|3146.57|3429.39|8.20|0.00|136.67|144.87|-3009.90| +||15875||1808369||21078|||618|||112.36||0.00|151.47||3033.72||0.00|||| +2451656|55896|3584|53238|1808369|6347|21078|8|67|618|97|60.24|88.55|47.81|0.00|4637.57|5843.28|8589.35|324.62|0.00|4637.57|4962.19|-1205.71| +2451656|55896|6452|53238|1808369|6347|21078|8|92|618|84|2.13|4.21|2.31|0.00|194.04|178.92|353.64|7.76|0.00|194.04|201.80|15.12| +2451656|55896|9044|53238|1808369|6347|21078|8|247|618|69|14.27|19.26|9.24|0.00|637.56|984.63|1328.94|12.75|0.00|637.56|650.31|-347.07| +2451656|55896|14785|53238|1808369|6347|21078|8|105|618|19|29.01|50.76|14.72|0.00|279.68|551.19|964.44|22.37|0.00|279.68|302.05|-271.51| +2451656|55896|7436|53238|1808369|6347|21078|8|77|618|65|79.71|112.39|1.12|0.00|72.80|5181.15|7305.35|2.91|0.00|72.80|75.71|-5108.35| +2451656|55896|5783|53238|1808369|6347|21078|8|278|618|80|21.92|32.44|24.65|0.00|1972.00|1753.60|2595.20|98.60|0.00|1972.00|2070.60|218.40| +2451656|55896|4855|53238|1808369|6347|21078|8|67|618|37|86.69|115.29|115.29|0.00|4265.73|3207.53|4265.73|255.94|0.00|4265.73|4521.67|1058.20| +||6722|53238||||8|256|618|||||0.00|502.35|347.31|||0.00||507.37|| +2451656|55896|12775|53238|1808369|6347|21078|8|64|618|69|98.92|137.49|122.36|7345.27|8442.84|6825.48|9486.81|98.78|7345.27|1097.57|1196.35|-5727.91| +2451656|55896|13190|53238|1808369|6347|21078|8|41|618|91|89.96|143.03|68.65|0.00|6247.15|8186.36|13015.73|562.24|0.00|6247.15|6809.39|-1939.21| +2451656|55896|13739|53238|1808369|6347|21078|8|265|618|89|52.11|94.84|2.84|0.00|252.76|4637.79|8440.76|7.58|0.00|252.76|260.34|-4385.03| +2451656|55896|6932|53238|1808369|6347|21078|8|192|618|7|27.03|28.38|20.71|0.00|144.97|189.21|198.66|8.69|0.00|144.97|153.66|-44.24| +2451656|55896|13847|53238|1808369|6347|21078|8|132|618|56|56.64|75.33|24.10|0.00|1349.60|3171.84|4218.48|121.46|0.00|1349.60|1471.06|-1822.24| +2451656|55896|17027|53238|1808369|6347|21078|8|205|618|57|87.46|94.45|24.55|0.00|1399.35|4985.22|5383.65|13.99|0.00|1399.35|1413.34|-3585.87| +2451061|32660|13324|6694|394202|910|22818|2|238|619|78|70.83|75.07|5.25|0.00|409.50|5524.74|5855.46|36.85|0.00|409.50|446.35|-5115.24| +2451061|32660|2758|6694|394202|910|22818|2|225|619|62|51.99|83.70|46.87|0.00|2905.94|3223.38|5189.40|87.17|0.00|2905.94|2993.11|-317.44| +2451061|32660|6658|6694|394202|910|22818|2|261|619|16|17.18|31.61|4.74|0.00|75.84|274.88|505.76|2.27|0.00|75.84|78.11|-199.04| +2451061|32660|9052|6694|394202|910|22818|2|236|619|75|74.90|104.86|57.67|0.00|4325.25|5617.50|7864.50|346.02|0.00|4325.25|4671.27|-1292.25| +2451061|32660|14786|6694|394202|910|22818|2|175|619|83|74.20|148.40|40.06|0.00|3324.98|6158.60|12317.20|66.49|0.00|3324.98|3391.47|-2833.62| +2451061|32660|5953|6694|394202|910|22818|2|1|619|33|95.03|125.43|112.88|0.00|3725.04|3135.99|4139.19|37.25|0.00|3725.04|3762.29|589.05| +2451061|32660|9085|6694|394202|910|22818|2|101|619|21|21.01|22.90|10.99|0.00|230.79|441.21|480.90|20.77|0.00|230.79|251.56|-210.42| +2451061|32660|4796|6694|394202|910|22818|2|16|619|80|66.53|94.47|75.57|0.00|6045.60|5322.40|7557.60|0.00|0.00|6045.60|6045.60|723.20| +2451061|32660|1249||394202|910|22818||10|619|||72.45|||586.44|2949.21||17.41||193.53|210.94|| +2451061|32660|5560|6694|394202|910|22818|2|234|619|23|26.50|51.67|30.48|609.90|701.04|609.50|1188.41|6.37|609.90|91.14|97.51|-518.36| +2452097|30521|5473|40289|1188955|3668|18734|7|119|620|46|42.22|55.73|11.14|0.00|512.44|1942.12|2563.58|10.24|0.00|512.44|522.68|-1429.68| +2452097|30521|637|40289|1188955|3668|18734|7|60|620|90|84.33|164.44|55.90|0.00|5031.00|7589.70|14799.60|0.00|0.00|5031.00|5031.00|-2558.70| +2452097|30521|15499|40289|1188955|3668|18734|7|65|620|5|56.12|82.49|39.59|0.00|197.95|280.60|412.45|17.81|0.00|197.95|215.76|-82.65| +2452097|30521|349|40289|1188955|3668|18734|7|100|620|16|8.68|14.66|1.90|3.95|30.40|138.88|234.56|1.85|3.95|26.45|28.30|-112.43| +2452097|30521|351|40289|1188955|3668|||223|620||20.84||13.86||||666.75|31.18||346.50||| +2452097|30521|14797|40289|1188955|3668|18734|7|40|620|2|52.22|84.59|38.91|0.00|77.82|104.44|169.18|0.77|0.00|77.82|78.59|-26.62| +2452097|30521|3889|40289|1188955|3668|18734|7|118|620|29|94.32|174.49|80.26|0.00|2327.54|2735.28|5060.21|93.10|0.00|2327.54|2420.64|-407.74| +2452097|30521|9539|40289|1188955|3668|18734|7|52|620|52|14.14|14.42|2.73|114.98|141.96|735.28|749.84|1.61|114.98|26.98|28.59|-708.30| +2451883|35447|16027|69441|477570|5252|44581|8|293|621|38|9.06|13.59|9.64|0.00|366.32|344.28|516.42|29.30|0.00|366.32|395.62|22.04| +2451883|35447|4273|69441|477570|5252|44581|8|252|621|6|69.35|135.23|2.70|0.00|16.20|416.10|811.38|1.13|0.00|16.20|17.33|-399.90| +2451883|35447|14863|69441|477570|5252|44581|8|28|621|77|91.16|98.45|40.36|0.00|3107.72|7019.32|7580.65|31.07|0.00|3107.72|3138.79|-3911.60| +2451883|35447|9757|69441|477570|5252|44581|8|52|621|41|30.72|47.30|3.78|0.00|154.98|1259.52|1939.30|0.00|0.00|154.98|154.98|-1104.54| +2451883|35447|8875|69441|477570|5252|44581|8|158|621|67|54.70|104.47|54.32|0.00|3639.44|3664.90|6999.49|218.36|0.00|3639.44|3857.80|-25.46| +2451883|35447|10967|69441|477570|5252|44581|8|187|621|85|56.58|96.75|35.79|0.00|3042.15|4809.30|8223.75|121.68|0.00|3042.15|3163.83|-1767.15| +2451883|35447|4231|69441|477570|5252|44581|8|235|621|48|33.76|34.77|25.38|0.00|1218.24|1620.48|1668.96|12.18|0.00|1218.24|1230.42|-402.24| +2451883|35447|4958|69441|477570|5252|44581|8|1|621|1|12.15|22.96|0.91|0.86|0.91|12.15|22.96|0.00|0.86|0.05|0.05|-12.10| +2451883|35447|7346|69441|477570|5252|44581|8|150|621|36|44.79|45.23|13.56|0.00|488.16|1612.44|1628.28|0.00|0.00|488.16|488.16|-1124.28| +2452341|53773|6888|2235|1321590|2545|25500|2|147|622|95|91.26|93.08|75.39|0.00|7162.05|8669.70|8842.60|214.86|0.00|7162.05|7376.91|-1507.65| +2452341|53773|3495|2235|1321590|2545|25500|2|194|622|95|66.49|92.42|84.10|2476.74|7989.50|6316.55|8779.90|330.76|2476.74|5512.76|5843.52|-803.79| +2452341|53773|13393|2235|1321590|2545|25500|2|73|622|87|26.35|27.93|16.19|0.00|1408.53|2292.45|2429.91|0.00|0.00|1408.53|1408.53|-883.92| +2452341|53773|14841|2235|1321590|2545|25500|2|261|622|3|59.89|62.28|55.42|0.00|166.26|179.67|186.84|1.66|0.00|166.26|167.92|-13.41| +2452341|53773|3387|2235|1321590|2545|25500|2|194|622|65|29.79|38.13|24.02|0.00|1561.30|1936.35|2478.45|62.45|0.00|1561.30|1623.75|-375.05| +2452341|53773|13801|2235|1321590|2545|25500|2|158|622|79|10.18|20.36|20.15|0.00|1591.85|804.22|1608.44|111.42|0.00|1591.85|1703.27|787.63| +2452341|53773|17292|2235|1321590|2545|25500|2|129|622|82|56.63|104.76|43.99|0.00|3607.18|4643.66|8590.32|36.07|0.00|3607.18|3643.25|-1036.48| +2452341|53773|402||1321590|||2|278|622|28||||412.52|453.32|474.04||2.85|412.52||43.65|| +2452341|53773|11425|2235|1321590|2545|25500|2|43|622|97|31.57|54.61|16.38|0.00|1588.86|3062.29|5297.17|0.00|0.00|1588.86|1588.86|-1473.43| +2452341|53773|13533|2235|1321590|2545|25500|2|16|622|23|39.51|71.11|40.53|0.00|932.19|908.73|1635.53|83.89|0.00|932.19|1016.08|23.46| +2452341|53773|16530|2235|1321590|2545|25500|2|54|622|52|45.34|65.28|40.47|0.00|2104.44|2357.68|3394.56|84.17|0.00|2104.44|2188.61|-253.24| +2452341|53773|16224|2235|1321590|2545|25500|2|1|622|6|35.24|52.50|48.82|0.00|292.92|211.44|315.00|2.92|0.00|292.92|295.84|81.48| +2452341|53773|13327|2235|1321590|2545|25500|2|152|622|52|45.94|53.29|35.70|1225.22|1856.40|2388.88|2771.08|0.00|1225.22|631.18|631.18|-1757.70| +2452341|53773|13591|2235|1321590|2545||2|71|622||8.95||3.60||338.40|||||||| +2451853|36266|7211|15472|489811|4597|20936|7|263|623|29|33.38|52.40|14.14|0.00|410.06|968.02|1519.60|28.70|0.00|410.06|438.76|-557.96| +2451853|36266|17407|15472|489811|4597|20936|7|197|623|97|20.26|24.71|1.48|0.00|143.56|1965.22|2396.87|1.43|0.00|143.56|144.99|-1821.66| +2451853|36266|4669|15472|489811||20936||212|623|84|74.00||||11147.64||11375.28||||12150.92|| +2451853|36266|16813|15472|489811|4597|20936|7|125|623|95|79.80|126.08|0.00|0.00|0.00|7581.00|11977.60|0.00|0.00|0.00|0.00|-7581.00| +2451853|36266|13121|15472|489811|4597|20936|7|201|623|62|85.37|151.95|19.75|0.00|1224.50|5292.94|9420.90|61.22|0.00|1224.50|1285.72|-4068.44| +2451853|36266|6005|15472|489811|4597|20936|7|295|623|20|94.87|147.04|76.46|0.00|1529.20|1897.40|2940.80|30.58|0.00|1529.20|1559.78|-368.20| +2451853|36266|17657|15472|489811|4597|20936|7|160|623|92|22.27|40.75|30.56|0.00|2811.52|2048.84|3749.00|112.46|0.00|2811.52|2923.98|762.68| +2451853|36266|4259|15472|489811|4597|20936|7|294|623|8|86.24|124.18|42.22|0.00|337.76|689.92|993.44|20.26|0.00|337.76|358.02|-352.16| +2451103|70433|11882|25258|816910|1009|33461|1|185|624|13|94.11|127.04|52.08|0.00|677.04|1223.43|1651.52|33.85|0.00|677.04|710.89|-546.39| +2451103|70433|15914|25258|816910|1009|33461|1|198|624|12|78.79|109.51|73.37|0.00|880.44|945.48|1314.12|26.41|0.00|880.44|906.85|-65.04| +2451103|70433|3298|25258|816910|1009|33461|1|12|624|85|49.80|75.69|16.65|0.00|1415.25|4233.00|6433.65|113.22|0.00|1415.25|1528.47|-2817.75| +2451103|70433|4828|25258|816910|1009|33461|1|174|624|31|9.49|13.09|11.12|0.00|344.72|294.19|405.79|0.00|0.00|344.72|344.72|50.53| +2451103|70433|685|25258|816910|1009|33461|1|212|624|49|7.73|10.43|4.38|0.00|214.62|378.77|511.07|10.73|0.00|214.62|225.35|-164.15| +2451103|70433|104|25258|816910|1009|33461|1|162|624|86|24.32|25.29|4.29|0.00|368.94|2091.52|2174.94|18.44|0.00|368.94|387.38|-1722.58| +2451103|70433|10105|||1009||1||624||||||3757.60|3329.60||||3757.60||428.00| +2451103|70433|70|25258|816910|1009|33461|1|35|624|83|12.54|14.04|0.28|0.00|23.24|1040.82|1165.32|0.00|0.00|23.24|23.24|-1017.58| +2451103|70433|9211|25258|816910|1009|33461|1|25|624|46|39.30|44.40|38.62|0.00|1776.52|1807.80|2042.40|88.82|0.00|1776.52|1865.34|-31.28| +2451516|49221|4435|50721|225123|1478|10633|1|181|625|54|24.83|35.25|32.43|0.00|1751.22|1340.82|1903.50|52.53|0.00|1751.22|1803.75|410.40| +2451516|49221|3182|50721|225123|1478|10633|1|208|625|54|37.14|67.59|2.70|0.00|145.80|2005.56|3649.86|0.00|0.00|145.80|145.80|-1859.76| +2451516|49221|1268|50721|225123|1478|10633|1|120|625|76|18.48|22.54|9.24|0.00|702.24|1404.48|1713.04|21.06|0.00|702.24|723.30|-702.24| +2451516|49221|12886|50721|225123|1478|10633|1|85|625|47|43.69|48.49|45.09|656.96|2119.23|2053.43|2279.03|29.24|656.96|1462.27|1491.51|-591.16| +2451516|49221|13610|50721|225123|1478|10633|1|155|625|7|85.60|114.70|8.02|0.00|56.14|599.20|802.90|5.05|0.00|56.14|61.19|-543.06| +2451516|49221|13360|50721|225123|1478|10633|1|116|625|51|36.83|59.29|42.09|0.00|2146.59|1878.33|3023.79|42.93|0.00|2146.59|2189.52|268.26| +2451516|49221|16297|50721|225123|1478|10633|1|140|625|89|74.43|138.43|98.28|0.00|8746.92|6624.27|12320.27|699.75|0.00|8746.92|9446.67|2122.65| +2451516|49221|16480|50721|225123|1478|10633|1|94|625|99|89.24|91.02|13.65|0.00|1351.35|8834.76|9010.98|94.59|0.00|1351.35|1445.94|-7483.41| +2451516|49221|15013|50721|225123|1478|10633|1|295|625|81|85.46|87.16|57.52|0.00|4659.12|6922.26|7059.96|139.77|0.00|4659.12|4798.89|-2263.14| +2452599|65719|10743|9964|103504|2644|2984|10|172|626|17|63.88|79.21|60.99|0.00|1036.83|1085.96|1346.57|62.20|0.00|1036.83|1099.03|-49.13| +2452599|65719|6960|9964|103504|2644|2984|10|18|626|77|83.01|161.86|30.75|0.00|2367.75|6391.77|12463.22|47.35|0.00|2367.75|2415.10|-4024.02| +2452599|65719|14958|9964|103504|2644|2984|10|194|626|1|45.79|59.06|19.48|0.00|19.48|45.79|59.06|0.38|0.00|19.48|19.86|-26.31| +2452599|65719|4671|9964|103504|2644|2984|10|99|626|40|67.24|88.75|49.70|0.00|1988.00|2689.60|3550.00|59.64|0.00|1988.00|2047.64|-701.60| +2452599|65719|12696|9964|103504|2644|2984|10|180|626|44|33.36|59.38|8.90|0.00|391.60|1467.84|2612.72|23.49|0.00|391.60|415.09|-1076.24| +2452599|65719|16171|9964||2644|2984||73|626|||42.55||0.00|803.70||||0.00|803.70||| +2452599|65719|9984|9964|103504|2644|2984|10|40|626|76|62.74|105.40|105.40|3444.47|8010.40|4768.24|8010.40|365.27|3444.47|4565.93|4931.20|-202.31| +||6361|||2644|2984|||626||58.82|85.87|67.83|||3882.12||0.00||||| +2452599|65719|5961|9964|103504|2644|2984|10|49|626|45|63.53|111.17|103.38|0.00|4652.10|2858.85|5002.65|372.16|0.00|4652.10|5024.26|1793.25| +2452599|65719|12325|9964|103504|2644|2984|10|265|626|51|8.83|13.77|1.23|0.00|62.73|450.33|702.27|3.13|0.00|62.73|65.86|-387.60| +2452599|65719|7555|9964|103504|2644|2984|10|155|626|83|84.42|86.95|59.99|0.00|4979.17|7006.86|7216.85|298.75|0.00|4979.17|5277.92|-2027.69| +2452599|65719|1365|9964|103504|2644|2984|10|219|626|27|85.48|93.17|20.49|0.00|553.23|2307.96|2515.59|11.06|0.00|553.23|564.29|-1754.73| +2452599|65719|15007|9964|103504|2644|2984|10|168|626|37|45.68|79.48|20.66|0.00|764.42|1690.16|2940.76|0.00|0.00|764.42|764.42|-925.74| +2452599|65719|15205|9964|103504|2644|2984|10|260|626|74|49.50|55.93|31.88|0.00|2359.12|3663.00|4138.82|188.72|0.00|2359.12|2547.84|-1303.88| +2452599|65719|9384|9964|103504|2644|2984|10|199|626|84|10.58|11.00|1.43|0.00|120.12|888.72|924.00|10.81|0.00|120.12|130.93|-768.60| +2451915|51947|13723|83157|1106431|6152|29426|10|207|627|81|11.34|16.89|6.24|0.00|505.44|918.54|1368.09|35.38|0.00|505.44|540.82|-413.10| +2451915|51947|13141|83157|1106431|6152|29426|10|88|627|24|78.07|143.64|74.69|0.00|1792.56|1873.68|3447.36|161.33|0.00|1792.56|1953.89|-81.12| +2451915||13079||1106431||29426||261|627|49|72.41||68.90|0.00||3548.09|6492.99||0.00|||| +2451915|51947|2459|83157|1106431|6152|29426|10|157|627|12|63.52|115.60|52.02|0.00|624.24|762.24|1387.20|18.72|0.00|624.24|642.96|-138.00| +2451915|51947|15349|83157|1106431|6152|29426|10|269|627|81|27.78|52.78|19.00|0.00|1539.00|2250.18|4275.18|15.39|0.00|1539.00|1554.39|-711.18| +2451915|51947|3607|83157|1106431|6152|29426|10|234|627|21|11.03|21.06|0.21|0.00|4.41|231.63|442.26|0.30|0.00|4.41|4.71|-227.22| +2451915|51947|10153|83157|1106431|6152|29426|10|146|627|7|50.94|89.65|19.72|0.00|138.04|356.58|627.55|5.52|0.00|138.04|143.56|-218.54| +2451915|51947|14117|83157|1106431|6152|29426|10|94|627|99|43.49|79.58|44.56|0.00|4411.44|4305.51|7878.42|132.34|0.00|4411.44|4543.78|105.93| +2451915|51947|16381|83157|1106431|6152|29426|10|80|627|79|1.10|1.52|1.44|94.42|113.76|86.90|120.08|0.96|94.42|19.34|20.30|-67.56| +2451431|48342|12385|79843|76621|4599|25824|10|194|628|5|13.79|21.37|6.62|0.00|33.10|68.95|106.85|0.99|0.00|33.10|34.09|-35.85| +2451431|48342|14929|79843|76621|4599|25824|10|119|628|89|57.93|81.10|62.44|0.00|5557.16|5155.77|7217.90|0.00|0.00|5557.16|5557.16|401.39| +2451431|48342|6880|79843|76621|4599|25824|10|186|628|34|59.05|72.04|10.08|0.00|342.72|2007.70|2449.36|13.70|0.00|342.72|356.42|-1664.98| +2451431|48342|17287|79843|76621|4599|25824|10|38|628|88|36.96|68.74|12.37|0.00|1088.56|3252.48|6049.12|10.88|0.00|1088.56|1099.44|-2163.92| +2451431|48342|9308|79843|76621|4599|25824|10|112|628|8|89.84|91.63|43.98|256.84|351.84|718.72|733.04|6.65|256.84|95.00|101.65|-623.72| +2451431|48342|1042|79843|76621|4599|25824|10|34|628|65|78.13|83.59|18.38|585.40|1194.70|5078.45|5433.35|48.74|585.40|609.30|658.04|-4469.15| +|48342|15958|79843|||25824|10|122|628|||||||2076.32|3052.16|156.56|||1896.20|-336.68| +2451431|48342|3442|79843|76621|4599|25824|10|41|628|55|52.31|84.74|36.43|0.00|2003.65|2877.05|4660.70|60.10|0.00|2003.65|2063.75|-873.40| +2451431|48342|16465|79843|76621|4599|25824|10|39|628|14|84.77|157.67|137.17|1747.54|1920.38|1186.78|2207.38|1.72|1747.54|172.84|174.56|-1013.94| +2451431|48342|5134|79843|76621|4599|25824|10|151|628|46|92.36|141.31|135.65|0.00|6239.90|4248.56|6500.26|561.59|0.00|6239.90|6801.49|1991.34| +2451431|48342|4702|79843|76621|4599|25824|10|212|628|56|95.22|112.35|10.11|0.00|566.16|5332.32|6291.60|28.30|0.00|566.16|594.46|-4766.16| +2451431|48342|4580|79843|76621|4599|25824|10|104|628|6|59.61|68.55|0.00|0.00|0.00|357.66|411.30|0.00|0.00|0.00|0.00|-357.66| +2452498|67926|13867|58153|1819631|2709|2187|7|236|629|53|90.61|140.44|140.44|0.00|7443.32|4802.33|7443.32|297.73|0.00|7443.32|7741.05|2640.99| +|67926|5071|58153|||2187|||629||51.66|||0.00||568.26|568.26|20.28|0.00|289.74||| +||4695||1819631||2187|||629|92||112.07||190.62|515.20|6828.24|||190.62||344.05|| +2452498|67926|17635|58153|1819631|2709|2187|7|150|629|95|71.11|104.53|75.26|0.00|7149.70|6755.45|9930.35|285.98|0.00|7149.70|7435.68|394.25| +2452498|67926|4866|58153|1819631|2709|2187|7|218|629|35|64.16|99.44|17.89|250.46|626.15|2245.60|3480.40|11.27|250.46|375.69|386.96|-1869.91| +|67926|12939|58153|1819631|2709|2187|7|287|629||19.85|35.53||459.48|||||459.48||696.11|-144.48| +2452498|67926|6759|58153|1819631|2709|2187|7|26|629|73|33.73|45.87|5.50|0.00|401.50|2462.29|3348.51|24.09|0.00|401.50|425.59|-2060.79| +2452498|67926|9696|58153|1819631|2709|2187|7|200|629|28|14.06|15.18|15.18|0.00|425.04|393.68|425.04|29.75|0.00|425.04|454.79|31.36| +2452498|67926|11562|58153|1819631|2709|2187|7|206|629|56|64.73|67.31|41.73|0.00|2336.88|3624.88|3769.36|186.95|0.00|2336.88|2523.83|-1288.00| +2452498|67926|2172|58153|1819631|2709|2187|7|271|629|35|40.93|41.74|30.88|0.00|1080.80|1432.55|1460.90|64.84|0.00|1080.80|1145.64|-351.75| +2452498|67926|12549|58153|1819631|2709|2187|7|15|629|14|29.08|44.49|41.37|0.00|579.18|407.12|622.86|46.33|0.00|579.18|625.51|172.06| +2452498|67926|13554|58153|1819631|2709|2187|7|95|629|30|39.88|63.40|22.19|0.00|665.70|1196.40|1902.00|39.94|0.00|665.70|705.64|-530.70| +2452498|67926|12357|58153|1819631|2709|2187|7|35|629|51|81.86|134.25|42.96|0.00|2190.96|4174.86|6846.75|21.90|0.00|2190.96|2212.86|-1983.90| +2452498|67926|17004|58153|1819631|2709|2187|7|65|629|43|1.29|2.05|0.79|0.00|33.97|55.47|88.15|2.71|0.00|33.97|36.68|-21.50| +2451866|74243|11071|3604|475183|4715|4647|4|6|630|77|18.83|20.52|3.69|238.66|284.13|1449.91|1580.04|3.18|238.66|45.47|48.65|-1404.44| +2451866|74243|11435|3604|475183|4715|4647|4|157|630|100|78.78|118.95|113.00|0.00|11300.00|7878.00|11895.00|678.00|0.00|11300.00|11978.00|3422.00| +2451866|74243|5239|3604|475183|4715|4647|4|255|630|43|75.74|106.03|56.19|1932.93|2416.17|3256.82|4559.29|19.32|1932.93|483.24|502.56|-2773.58| +2451866|74243|11786|3604|475183|4715|4647|4|230|630|1|78.22|154.09|90.91|40.00|90.91|78.22|154.09|1.01|40.00|50.91|51.92|-27.31| +2451866|74243|11999|3604|475183|4715|4647|4|10|630|58|90.77|130.70|36.59|0.00|2122.22|5264.66|7580.60|127.33|0.00|2122.22|2249.55|-3142.44| +2451866|74243|9697|3604|475183|4715|4647|4|260|630|93|70.92|112.05|108.68|1313.94|10107.24|6595.56|10420.65|263.79|1313.94|8793.30|9057.09|2197.74| +2451866|74243|17549|3604|475183|4715|4647|4|74|630|5|51.56|92.80|56.60|0.00|283.00|257.80|464.00|16.98|0.00|283.00|299.98|25.20| +2451866|74243|16856|3604|475183|4715|4647|4|122|630|15|6.50|12.80|2.04|0.00|30.60|97.50|192.00|2.44|0.00|30.60|33.04|-66.90| +2451866|74243|13411|3604|475183|4715|4647|4|273|630|3|19.01|34.78|4.52|0.00|13.56|57.03|104.34|0.27|0.00|13.56|13.83|-43.47| +2452562|48450|9523|34562|259010|5505|31553|8|285|631|28|73.09|136.67|62.86|897.64|1760.08|2046.52|3826.76|34.49|897.64|862.44|896.93|-1184.08| +2452562|48450|16788|34562|259010|5505|31553|8|142|631|3|13.99|24.06|0.96|1.69|2.88|41.97|72.18|0.09|1.69|1.19|1.28|-40.78| +2452562|48450|4851|34562|259010|5505|31553|8|182|631|86|10.95|14.67|12.17|1036.15|1046.62|941.70|1261.62|0.31|1036.15|10.47|10.78|-931.23| +2452562|48450|3768|34562|259010|5505|31553|8|246|631|36|23.12|31.90|17.22|0.00|619.92|832.32|1148.40|55.79|0.00|619.92|675.71|-212.40| +2452562|48450|477|34562|259010|5505|31553|8|175|631|2|98.83|120.57|112.13|0.00|224.26|197.66|241.14|0.00|0.00|224.26|224.26|26.60| +2452562|48450|15114|34562|259010|5505|31553|8|141|631|18|83.65|127.14|7.62|0.00|137.16|1505.70|2288.52|2.74|0.00|137.16|139.90|-1368.54| +2452562|48450|169|34562|259010|5505|31553|8|221|631|71|94.94|143.35|25.80|787.67|1831.80|6740.74|10177.85|31.32|787.67|1044.13|1075.45|-5696.61| +2452562|48450|5943|34562|259010|5505|31553|8|274|631|47|64.74|86.10|59.40|2038.01|2791.80|3042.78|4046.70|0.00|2038.01|753.79|753.79|-2288.99| +2452562|48450|8487|34562|259010|5505|31553|8|221|631|35|22.32|39.50|17.77|0.00|621.95|781.20|1382.50|0.00|0.00|621.95|621.95|-159.25| +2452354|51405|1740|49012|1116590|5321|20499|10|211|632|59|32.83|50.22|31.13|0.00|1836.67|1936.97|2962.98|36.73|0.00|1836.67|1873.40|-100.30| +2452354|51405|3357|49012|1116590|5321|20499|10|41|632|35|98.72|192.50|94.32|1749.63|3301.20|3455.20|6737.50|46.54|1749.63|1551.57|1598.11|-1903.63| +2452354|51405|9369|49012|1116590|5321|20499|10|213|632|72|79.24|139.46|122.72|0.00|8835.84|5705.28|10041.12|0.00|0.00|8835.84|8835.84|3130.56| +2452354|51405|3201|49012|1116590|5321|20499|10|16|632|98|16.52|30.72|16.89|0.00|1655.22|1618.96|3010.56|99.31|0.00|1655.22|1754.53|36.26| +2452354|51405|5292|49012|1116590|5321|20499|10|26|632|67|22.06|34.41|22.02|0.00|1475.34|1478.02|2305.47|103.27|0.00|1475.34|1578.61|-2.68| +2452354||11529||1116590|||||632||||33.56|0.00|1141.04||1934.26||0.00|||78.20| +2452354|51405|1605|49012|1116590|5321|20499|10|252|632|96|52.52|71.42|22.14|0.00|2125.44|5041.92|6856.32|63.76|0.00|2125.44|2189.20|-2916.48| +2452354|51405|12307|49012|1116590|5321|20499|10|254|632|69|6.63|11.40|2.96|0.00|204.24|457.47|786.60|10.21|0.00|204.24|214.45|-253.23| +2452354|51405|7239|49012|1116590|5321|20499|10|115|632|87|21.94|32.03|16.97|0.00|1476.39|1908.78|2786.61|132.87|0.00|1476.39|1609.26|-432.39| +2452354|51405|11185|49012|1116590|5321|20499|10|237|632|73|78.50|91.84|73.47|0.00|5363.31|5730.50|6704.32|321.79|0.00|5363.31|5685.10|-367.19| +2452354|51405|10617|49012|1116590|5321|20499|10|74|632|59|6.91|9.46|0.94|0.00|55.46|407.69|558.14|0.00|0.00|55.46|55.46|-352.23| +2452354|51405|2511|49012|1116590|5321|20499|10|154|632|92|66.22|127.80|111.18|0.00|10228.56|6092.24|11757.60|715.99|0.00|10228.56|10944.55|4136.32| +2452354|51405|17859|49012|1116590|5321|20499|10|196|632|9|76.92|149.22|62.67|0.00|564.03|692.28|1342.98|33.84|0.00|564.03|597.87|-128.25| +2452354|51405|8730|49012|1116590|5321|20499|10|263|632|69|6.61|8.85|4.24|0.00|292.56|456.09|610.65|14.62|0.00|292.56|307.18|-163.53| +2451474|41756|1531|66457|1633190|5523|10256|4|189|633|47|61.84|117.49|2.34|0.00|109.98|2906.48|5522.03|3.29|0.00|109.98|113.27|-2796.50| +2451474|41756|3524|66457|1633190|5523|10256|4|192|633|86|87.28|164.95|107.21|0.00|9220.06|7506.08|14185.70|737.60|0.00|9220.06|9957.66|1713.98| +2451474|41756|13021|66457|1633190|5523|10256|4|247|633|50|42.79|77.87|10.12|0.00|506.00|2139.50|3893.50|10.12|0.00|506.00|516.12|-1633.50| +2451474|41756|17263|66457|1633190|5523|10256|4|105|633|16|45.57|89.77|39.49|0.00|631.84|729.12|1436.32|18.95|0.00|631.84|650.79|-97.28| +2451474||17971||1633190|5523||||633||||26.29|0.00||974.28|||0.00||1245.62|235.06| +2451474|41756|14443|66457|1633190|5523|10256|4|204|633|95|58.73|76.34|16.79|0.00|1595.05|5579.35|7252.30|79.75|0.00|1595.05|1674.80|-3984.30| +2451474|41756|14098|66457|1633190|5523|10256|4|96|633|5|14.82|18.96|16.68|0.00|83.40|74.10|94.80|7.50|0.00|83.40|90.90|9.30| +|41756|15784||||10256|||633|||132.75|25.22||2017.60|5966.40|10620.00|20.17||||| +2451474|41756|10693|66457|1633190|5523|10256|4|211|633|91|93.53|97.27|63.22|0.00|5753.02|8511.23|8851.57|57.53|0.00|5753.02|5810.55|-2758.21| +2451474|41756|16868|66457|1633190|5523|10256|4|235|633|53|24.45|34.71|20.47|0.00|1084.91|1295.85|1839.63|0.00|0.00|1084.91|1084.91|-210.94| +2451474|41756|5354|66457|1633190|5523|10256|4|277|633|37|96.77|114.18|103.90|0.00|3844.30|3580.49|4224.66|345.98|0.00|3844.30|4190.28|263.81| +|41756|4696|66457|1633190|5523|10256|||633|||||0.00||7695.87|13005.72|85.83|0.00|8583.12||| +2451497|35096|3496|57775|970534|3740|17699|2|41|634|66|75.05|124.58|37.37|1060.56|2466.42|4953.30|8222.28|98.41|1060.56|1405.86|1504.27|-3547.44| +2451497|35096|4183|57775|970534|3740|17699|2|15|634|67|69.22|81.67|43.28|0.00|2899.76|4637.74|5471.89|28.99|0.00|2899.76|2928.75|-1737.98| +2451497|35096|13426|57775|970534|3740|17699|2|42|634|84|83.66|162.30|3.24|0.00|272.16|7027.44|13633.20|0.00|0.00|272.16|272.16|-6755.28| +2451497|35096|664|57775|970534|3740|17699|2|269|634|57|8.19|11.30|4.06|0.00|231.42|466.83|644.10|4.62|0.00|231.42|236.04|-235.41| +2451497|35096|11893|57775|970534|3740|17699|2|135|634|53|12.75|21.93|13.59|0.00|720.27|675.75|1162.29|50.41|0.00|720.27|770.68|44.52| +2451497|35096|11365|57775|970534|3740|17699|2|238|634|59|82.65|133.06|11.97|0.00|706.23|4876.35|7850.54|21.18|0.00|706.23|727.41|-4170.12| +2451497|35096|7114|57775|970534|3740|17699|2|176|634|99|84.19|145.64|49.51|784.23|4901.49|8334.81|14418.36|247.03|784.23|4117.26|4364.29|-4217.55| +2451497|35096|12040|57775|970534|3740|17699|2|290|634|62|16.43|30.06|24.34|0.00|1509.08|1018.66|1863.72|15.09|0.00|1509.08|1524.17|490.42| +2451497|35096|5929|57775|970534|3740|17699|2|273|634|88|45.87|62.84|18.85|0.00|1658.80|4036.56|5529.92|132.70|0.00|1658.80|1791.50|-2377.76| +2451497|35096|16670|57775|970534|3740|17699|2|85|634|34|19.38|24.03|12.49|0.00|424.66|658.92|817.02|12.73|0.00|424.66|437.39|-234.26| +2451497||17119||970534|3740||2|292|634|47||24.73||||945.17||37.76|||793.05|-189.88| +2451497|35096|17545|57775|970534|3740|17699|2|287|634|30|64.97|109.14|64.39|0.00|1931.70|1949.10|3274.20|173.85|0.00|1931.70|2105.55|-17.40| +2451497|35096|14126|57775|970534|3740|17699|2|96|634|97|21.71|31.47|19.82|0.00|1922.54|2105.87|3052.59|153.80|0.00|1922.54|2076.34|-183.33| +2451497|35096|7628|57775|970534|3740|17699|2|156|634|88|53.96|75.54|71.00|0.00|6248.00|4748.48|6647.52|562.32|0.00|6248.00|6810.32|1499.52| +2451497||8725|57775|970534||17699||92|634|67|98.23|129.66|54.45|2334.81|3648.15|6581.41|8687.22|0.00|2334.81|1313.34||-5268.07| +2451497|35096|1672|57775|970534|3740|17699|2|82|634|83|31.29|33.16|24.20|0.00|2008.60|2597.07|2752.28|0.00|0.00|2008.60|2008.60|-588.47| +2450954|30147|3079|80162|1454076|4343|7857|10|230|635|32|32.09|35.94|7.18|0.00|229.76|1026.88|1150.08|0.00|0.00|229.76|229.76|-797.12| +2450954|30147|14500|80162|1454076|4343|7857|10|70|635|2|86.73|150.91|84.50|0.00|169.00|173.46|301.82|0.00|0.00|169.00|169.00|-4.46| +2450954|30147|5026|80162|1454076|4343|7857|10|156|635|47|40.88|56.41|48.51|1823.97|2279.97|1921.36|2651.27|9.12|1823.97|456.00|465.12|-1465.36| +2450954|30147|12626|80162|1454076|4343|7857|10|143|635|28|6.53|8.09|7.28|0.00|203.84|182.84|226.52|0.00|0.00|203.84|203.84|21.00| +2450954|30147|10514|80162|1454076|4343|7857|10|211|635|78|67.63|99.41|21.87|0.00|1705.86|5275.14|7753.98|17.05|0.00|1705.86|1722.91|-3569.28| +2450954|30147|16094|80162|1454076|4343|7857|10|225|635|91|24.48|28.39|7.38|0.00|671.58|2227.68|2583.49|26.86|0.00|671.58|698.44|-1556.10| +|30147|1111||1454076||7857|10|98|635||14.11||15.21|0.00|1460.16||2519.04||0.00|||105.60| +2450954|30147|292|80162|1454076|4343|7857|10|40|635|40|68.01|99.97|6.99|0.00|279.60|2720.40|3998.80|5.59|0.00|279.60|285.19|-2440.80| +2450954|30147|12926|80162|1454076|4343|7857|10|233|635|73|53.01|75.27|4.51|0.00|329.23|3869.73|5494.71|16.46|0.00|329.23|345.69|-3540.50| +2450954|30147|10972|80162|1454076|4343|7857|10|176|635|38|91.60|157.55|122.88|0.00|4669.44|3480.80|5986.90|186.77|0.00|4669.44|4856.21|1188.64| +2450954|30147|4123|80162|1454076|4343|7857|10|264|635|27|28.55|45.68|31.51|0.00|850.77|770.85|1233.36|25.52|0.00|850.77|876.29|79.92| +2450954|30147|3760|80162|1454076|4343|7857|10|236|635|86|11.31|12.66|0.88|0.00|75.68|972.66|1088.76|3.78|0.00|75.68|79.46|-896.98| +2450954|30147|4192|80162|1454076|4343|7857|10|196|635|78|8.24|13.67|13.67|0.00|1066.26|642.72|1066.26|21.32|0.00|1066.26|1087.58|423.54| +2450954|30147|5650|80162|1454076|4343|7857|10|16|635|94|4.36|4.79|0.52|0.00|48.88|409.84|450.26|0.48|0.00|48.88|49.36|-360.96| +2451878|33596|17072|74391|1730310|1429|48595|2|162|636|54|45.55|87.91|32.52|0.00|1756.08|2459.70|4747.14|105.36|0.00|1756.08|1861.44|-703.62| +2451878|33596|10733|74391|1730310|1429|48595|2|247|636|25|45.72|71.78|2.87|0.00|71.75|1143.00|1794.50|2.15|0.00|71.75|73.90|-1071.25| +2451878|33596|13457|74391|1730310|1429|48595|2|234|636|51|9.52|13.80|5.52|0.00|281.52|485.52|703.80|11.26|0.00|281.52|292.78|-204.00| +2451878|33596|7448|74391|1730310|1429|48595|2|7|636|63|52.54|90.36|38.85|0.00|2447.55|3310.02|5692.68|171.32|0.00|2447.55|2618.87|-862.47| +2451878|33596|2713|74391|1730310|1429|48595|2|68|636|96|57.08|73.06|2.19|113.52|210.24|5479.68|7013.76|2.90|113.52|96.72|99.62|-5382.96| +2451878|33596|1399|74391|1730310|1429|48595|2|268|636|84|65.92|100.85|9.07|0.00|761.88|5537.28|8471.40|60.95|0.00|761.88|822.83|-4775.40| +2451878|33596|7339|74391|1730310|1429|48595|2|234|636|23|63.22|95.46|68.73|885.24|1580.79|1454.06|2195.58|13.91|885.24|695.55|709.46|-758.51| +|33596|8687|||1429|48595||24|636|73||||3582.02|5596.91|4598.27|||3582.02|||-2583.38| +2451878|33596|12439|74391|1730310|1429|48595|2|238|636|8|30.83|50.86|30.51|192.82|244.08|246.64|406.88|4.10|192.82|51.26|55.36|-195.38| +2451878|33596|9221|74391|1730310|1429|48595|2|94|636|83|40.50|75.73|73.45|0.00|6096.35|3361.50|6285.59|426.74|0.00|6096.35|6523.09|2734.85| +2452239|55232|10091|44866|1754031|3196|5561|10|281|637|62|35.03|44.13|38.39|0.00|2380.18|2171.86|2736.06|95.20|0.00|2380.18|2475.38|208.32| +2452239|55232|17029|44866|1754031|3196|5561|10|33|637|34|80.61|98.34|98.34|0.00|3343.56|2740.74|3343.56|200.61|0.00|3343.56|3544.17|602.82| +2452239|55232|2731|44866|1754031|3196|5561|10|44|637|46|36.48|72.23|62.11|1199.96|2857.06|1678.08|3322.58|99.42|1199.96|1657.10|1756.52|-20.98| +2452239|55232|10287|44866|1754031|3196|5561|10|208|637|73|99.59|107.55|6.45|0.00|470.85|7270.07|7851.15|0.00|0.00|470.85|470.85|-6799.22| +2452239|55232|12607|44866|1754031|3196|5561|10|59|637|9|80.97|134.41|60.48|0.00|544.32|728.73|1209.69|21.77|0.00|544.32|566.09|-184.41| +2452239|55232|3429|44866|1754031|3196|5561|10|273|637|12|80.24|82.64|25.61|0.00|307.32|962.88|991.68|0.00|0.00|307.32|307.32|-655.56| +2452239|55232|9023|44866|1754031|3196|5561|10|206|637|42|83.46|87.63|85.87|0.00|3606.54|3505.32|3680.46|288.52|0.00|3606.54|3895.06|101.22| +2452239|55232|6495|44866|1754031|3196|5561|10|212|637|31|40.87|41.27|7.42|0.00|230.02|1266.97|1279.37|13.80|0.00|230.02|243.82|-1036.95| +2452239||11607|44866|1754031|3196|5561||25|637|||||0.00|1472.72|3256.22||14.72|0.00||1487.44|| +2452175|68163|5727|49500|843800|53|8353|1|84|638|29|69.54|130.73|41.83|921.93|1213.07|2016.66|3791.17|2.91|921.93|291.14|294.05|-1725.52| +2452175|68163|3691|49500|843800|53|8353|1|69|638|36|80.29|118.82|68.91|0.00|2480.76|2890.44|4277.52|198.46|0.00|2480.76|2679.22|-409.68| +2452175|68163|9849|49500|843800|53|8353|1|272|638|79|19.26|35.43|10.98|0.00|867.42|1521.54|2798.97|8.67|0.00|867.42|876.09|-654.12| +2452175|68163|12931|49500|843800|53|8353|1|195|638|97|18.23|24.97|23.72|552.20|2300.84|1768.31|2422.09|0.00|552.20|1748.64|1748.64|-19.67| +2452175|68163|12833|49500|843800|53|8353|1|141|638|63|96.48|189.10|83.20|0.00|5241.60|6078.24|11913.30|104.83|0.00|5241.60|5346.43|-836.64| +2452175|68163|13587|49500|843800|53|8353|1|203|638|91|98.08|118.67|20.17|1394.95|1835.47|8925.28|10798.97|26.43|1394.95|440.52|466.95|-8484.76| +2452175|68163|17981|49500|843800|53|8353|1|155|638|97|86.01|116.97|36.26|0.00|3517.22|8342.97|11346.09|246.20|0.00|3517.22|3763.42|-4825.75| +2452175|68163|179|49500|843800|53|8353|1|258|638|78|7.34|13.28|8.76|0.00|683.28|572.52|1035.84|61.49|0.00|683.28|744.77|110.76| +2452175|68163|6597|49500|843800|53|8353|1|243|638|6|55.64|87.91|77.36|0.00|464.16|333.84|527.46|18.56|0.00|464.16|482.72|130.32| +2452175|68163|17989|49500|843800|53|8353|1|113|638|82|92.60|138.90|134.73|2761.96|11047.86|7593.20|11389.80|662.87|2761.96|8285.90|8948.77|692.70| +2452175|68163|13569|49500|843800|53|8353|1|97|638|42|17.56|21.95|10.97|55.28|460.74|737.52|921.90|16.21|55.28|405.46|421.67|-332.06| +2452175|68163|6829|49500|843800|53|8353|1|69|638|18|52.08|73.43|73.43|0.00|1321.74|937.44|1321.74|79.30|0.00|1321.74|1401.04|384.30| +2452175|68163|2183||||||161|638|54||76.35|26.72||1442.88|2329.56|4122.90|||||-886.68| +2452175|68163|1087||843800|53|8353|||638|66||85.14|||5506.38|||297.34||||1282.85| +2452175|68163|11187|49500|843800|53|8353|1|111|638|89|42.78|47.91|5.27|0.00|469.03|3807.42|4263.99|32.83|0.00|469.03|501.86|-3338.39| +2452175|68163|14281|49500|843800|53|8353|1|45|638|71|39.27|62.83|35.81|0.00|2542.51|2788.17|4460.93|76.27|0.00|2542.51|2618.78|-245.66| +2451789|32828|2267|54121|1174869|2817|12|2|220|639|12|6.11|8.73|4.36|0.00|52.32|73.32|104.76|2.61|0.00|52.32|54.93|-21.00| +2451789|32828|16079|54121|1174869|2817|12|2|45|639|42|45.56|52.84|28.53|0.00|1198.26|1913.52|2219.28|95.86|0.00|1198.26|1294.12|-715.26| +2451789|32828|15662|54121|1174869|2817|12|2|200|639|3|76.25|140.30|56.12|0.00|168.36|228.75|420.90|13.46|0.00|168.36|181.82|-60.39| +|32828|13405||1174869|2817||2||639||74.09|80.01||385.72|||560.07|5.92|385.72|84.68|90.60|| +2451789|32828|11525|54121|1174869|2817|12|2|27|639|49|84.29|115.47|47.34|115.98|2319.66|4130.21|5658.03|22.03|115.98|2203.68|2225.71|-1926.53| +2451789|32828|17615|54121|1174869|2817|12|2|135|639|63|66.71|120.74|51.91|0.00|3270.33|4202.73|7606.62|196.21|0.00|3270.33|3466.54|-932.40| +2451789|32828|1034|54121|1174869|2817|12|2|9|639|2|73.82|129.92|3.89|0.00|7.78|147.64|259.84|0.31|0.00|7.78|8.09|-139.86| +2451789|32828|4769|54121|1174869|2817|12|2|34|639|30|44.31|85.96|22.34|0.00|670.20|1329.30|2578.80|20.10|0.00|670.20|690.30|-659.10| +2451789|32828|503|54121|1174869|2817|12|2|174|639|31|67.16|117.53|37.60|0.00|1165.60|2081.96|3643.43|81.59|0.00|1165.60|1247.19|-916.36| +2451789|32828|1357|54121|1174869|2817|12|2|298|639|59|10.15|10.45|1.04|0.00|61.36|598.85|616.55|3.06|0.00|61.36|64.42|-537.49| +2451789|32828|15509|54121|1174869|2817|12|2|102|639|25|24.48|34.51|33.12|0.00|828.00|612.00|862.75|0.00|0.00|828.00|828.00|216.00| +2451789|32828|15821|54121|1174869|2817|12|2|191|639|26|99.51|136.32|16.35|0.00|425.10|2587.26|3544.32|4.25|0.00|425.10|429.35|-2162.16| +2451789|32828|6649|54121|1174869|2817|12|2|161|639|25|54.95|107.70|52.77|0.00|1319.25|1373.75|2692.50|52.77|0.00|1319.25|1372.02|-54.50| +2451789|32828|15523|54121|1174869|2817|12|2|238|639|67|79.42|132.63|5.30|0.00|355.10|5321.14|8886.21|31.95|0.00|355.10|387.05|-4966.04| +2451881||15703|60595|||17216|4|42|640||36.36|||0.00||2545.20||97.83|0.00||2543.63|| +2451881|41024|6007|60595|990802|3409|17216|4|224|640|98|60.32|104.95|10.49|904.65|1028.02|5911.36|10285.10|3.70|904.65|123.37|127.07|-5787.99| +2451881|41024|11725|60595|990802|3409|17216|4|18|640|74|47.58|76.12|57.85|0.00|4280.90|3520.92|5632.88|0.00|0.00|4280.90|4280.90|759.98| +2451881|41024|16589|60595|990802|3409|17216|4|266|640|11|87.70|125.41|43.89|0.00|482.79|964.70|1379.51|14.48|0.00|482.79|497.27|-481.91| +||17960||||17216|4||640|18|51.89|95.47|0.95||||||||17.78|| +2451881|41024|14720|60595|990802||17216|||640|48|95.74|||0.00|4269.12||7490.40||0.00|||-326.40| +2451881|41024|17216|60595|990802|3409|17216|4|267|640|78|42.21|49.38|30.61|0.00|2387.58|3292.38|3851.64|47.75|0.00|2387.58|2435.33|-904.80| +2451881|41024|3617|60595|990802|3409|17216|4|30|640|62|97.47|179.34|32.28|0.00|2001.36|6043.14|11119.08|40.02|0.00|2001.36|2041.38|-4041.78| +2451881|41024|13283|60595|990802|3409|17216|4|284|640|1|4.53|7.15|0.57|0.00|0.57|4.53|7.15|0.00|0.00|0.57|0.57|-3.96| +2451881|41024|13631|60595|990802|3409|17216|4|249|640|21|25.95|43.33|12.56|0.00|263.76|544.95|909.93|0.00|0.00|263.76|263.76|-281.19| +2451881|41024|5981|60595|990802|3409|17216|4|9|640|21|80.87|85.72|31.71|0.00|665.91|1698.27|1800.12|0.00|0.00|665.91|665.91|-1032.36| +2451881|41024|4553|60595|990802|3409|17216|4|164|640|10|97.27|120.61|37.38|0.00|373.80|972.70|1206.10|7.47|0.00|373.80|381.27|-598.90| +2451881|41024|10993|60595|990802|3409|17216|4|272|640|43|78.49|108.31|74.73|0.00|3213.39|3375.07|4657.33|0.00|0.00|3213.39|3213.39|-161.68| +2451881|41024|49|60595|990802|3409|17216|4|71|640|52|74.46|100.52|14.07|0.00|731.64|3871.92|5227.04|58.53|0.00|731.64|790.17|-3140.28| +2451881|41024|4583|60595|990802|3409|17216|4|156|640|47|26.60|35.11|1.75|0.00|82.25|1250.20|1650.17|2.46|0.00|82.25|84.71|-1167.95| +2451421||3850|1542|1407676||48391|||641||68.87|77.13|77.13|1392.96||2961.41|3316.59|115.41|1392.96|1923.63||| +||15280|1542||||4||641|44|98.38|189.87||501.23|4176.92||8354.28||501.23|3675.69|3932.98|| +2451421|67410|1340|1542|1407676|6257|48391|4|174|641|34|17.82|20.31|4.67|0.00|158.78|605.88|690.54|7.93|0.00|158.78|166.71|-447.10| +2451421|67410|2746|1542|1407676|6257|48391|4|146|641|2|13.80|22.90|2.51|0.00|5.02|27.60|45.80|0.30|0.00|5.02|5.32|-22.58| +2451421|67410|17335|1542|1407676|6257|48391|4|255|641|23|58.95|70.74|67.20|0.00|1545.60|1355.85|1627.02|15.45|0.00|1545.60|1561.05|189.75| +2451421|67410|730|1542|1407676|6257|48391|4|269|641|70|46.26|55.04|47.33|0.00|3313.10|3238.20|3852.80|298.17|0.00|3313.10|3611.27|74.90| +2451421|67410|2680|1542|1407676|6257|48391|4|30|641|20|77.73|104.93|20.98|0.00|419.60|1554.60|2098.60|12.58|0.00|419.60|432.18|-1135.00| +2451421|67410|10418|1542|1407676|6257|48391|4|249|641|9|91.06|122.02|34.16|0.00|307.44|819.54|1098.18|24.59|0.00|307.44|332.03|-512.10| +2451421|67410|16574|1542|1407676|6257|48391|4|287|641|91|44.90|87.10|36.58|1497.95|3328.78|4085.90|7926.10|18.30|1497.95|1830.83|1849.13|-2255.07| +2451421|67410|7564|1542|1407676|6257|48391|4|60|641|100|4.62|6.74|1.68|0.00|168.00|462.00|674.00|1.68|0.00|168.00|169.68|-294.00| +||17926|1542|1407676||48391|4|269|641||38.17||55.78|0.00|5075.98|||456.83|0.00||5532.81|| +2451421|67410|12350|1542|1407676|6257|48391|4|200|641|15|25.06|41.09|6.57|95.59|98.55|375.90|616.35|0.17|95.59|2.96|3.13|-372.94| +2452621|62118|1333|41211|1682276|2894|3512|10|108|642|56|40.89|59.69|19.10|0.00|1069.60|2289.84|3342.64|0.00|0.00|1069.60|1069.60|-1220.24| +2452621|62118|17001|41211|1682276|2894|3512|10|252|642|43|8.61|8.86|1.24|25.06|53.32|370.23|380.98|2.54|25.06|28.26|30.80|-341.97| +2452621|62118|5955|41211|1682276|2894|3512|10|3|642|24|27.24|53.93|25.34|0.00|608.16|653.76|1294.32|54.73|0.00|608.16|662.89|-45.60| +2452621|62118|2610|41211|1682276|2894|3512|10|210|642|2|37.74|61.51|39.36|0.00|78.72|75.48|123.02|0.78|0.00|78.72|79.50|3.24| +2452621|62118|10329|41211|1682276|2894|3512|10|193|642|39|16.55|30.12|19.57|0.00|763.23|645.45|1174.68|15.26|0.00|763.23|778.49|117.78| +2452621|62118|7377|41211|1682276|2894|3512|10|278|642|81|24.15|42.02|28.15|0.00|2280.15|1956.15|3403.62|91.20|0.00|2280.15|2371.35|324.00| +2452621||13279|41211||||10||642|55|42.46||62.41|2711.71||2335.30||50.45|2711.71|720.84||| +2452621|62118|3945|41211|1682276|2894|3512|10|111|642|59|83.85|154.28|154.28|0.00|9102.52|4947.15|9102.52|546.15|0.00|9102.52|9648.67|4155.37| +2452621|62118|14181|41211|1682276|2894|3512|10|171|642|3|61.40|100.69|37.25|0.00|111.75|184.20|302.07|4.47|0.00|111.75|116.22|-72.45| +2452621|62118|1416|41211|1682276|2894|3512|10|224|642|37|40.05|76.49|31.36|0.00|1160.32|1481.85|2830.13|46.41|0.00|1160.32|1206.73|-321.53| +2452621|62118|14808|41211|1682276|2894|3512|10|79|642|5|82.91|126.85|112.89|293.51|564.45|414.55|634.25|16.25|293.51|270.94|287.19|-143.61| +2452621|62118|11197|41211|1682276|2894|3512|10|79|642|39|26.74|28.07|4.77|0.00|186.03|1042.86|1094.73|3.72|0.00|186.03|189.75|-856.83| +2452326|65483|12102|66696|1625265|3834|48003|10|84|643|14|30.95|50.13|31.08|356.79|435.12|433.30|701.82|3.91|356.79|78.33|82.24|-354.97| +2452326|65483|11695|66696|1625265|3834|48003|10|32|643|34|41.26|76.33|24.42|0.00|830.28|1402.84|2595.22|0.00|0.00|830.28|830.28|-572.56| +2452326|65483|17523|66696|1625265|3834|48003|10|91|643|82|43.05|44.77|21.93|1420.62|1798.26|3530.10|3671.14|0.00|1420.62|377.64|377.64|-3152.46| +2452326|65483|4321|66696|1625265|3834|48003|10|3|643|61|19.38|27.32|3.00|0.00|183.00|1182.18|1666.52|5.49|0.00|183.00|188.49|-999.18| +2452326|65483|4645|66696|1625265|3834|48003|10|61|643|83|81.68|107.81|67.92|0.00|5637.36|6779.44|8948.23|0.00|0.00|5637.36|5637.36|-1142.08| +2452326|65483|6573|66696|1625265|3834|48003|10|245|643|53|37.69|68.97|49.65|1210.46|2631.45|1997.57|3655.41|14.20|1210.46|1420.99|1435.19|-576.58| +2452326|65483|10146|66696|1625265|3834|48003|10|271|643|79|67.23|95.46|53.45|0.00|4222.55|5311.17|7541.34|295.57|0.00|4222.55|4518.12|-1088.62| +2452326|65483|5742|66696|1625265|3834|48003|10|78|643|19|47.78|75.01|30.75|0.00|584.25|907.82|1425.19|5.84|0.00|584.25|590.09|-323.57| +2452326|65483|14763|66696|1625265|3834|48003|10|147|643|31|82.75|118.33|87.56|0.00|2714.36|2565.25|3668.23|244.29|0.00|2714.36|2958.65|149.11| +2452326|65483|13737|66696|1625265|3834|48003|10|276|643|15|60.53|110.16|56.18|0.00|842.70|907.95|1652.40|0.00|0.00|842.70|842.70|-65.25| +2452326|65483|5601|66696|1625265|3834|48003|10|87|643|17|36.65|64.87|18.16|154.36|308.72|623.05|1102.79|13.89|154.36|154.36|168.25|-468.69| +2452326|65483|13914|66696|1625265|3834|48003|10|162|643|68|36.33|66.48|53.84|183.05|3661.12|2470.44|4520.64|139.12|183.05|3478.07|3617.19|1007.63| +2452326|65483|9091|66696|1625265|3834|48003|10|144|643|76|49.66|60.58|8.48|0.00|644.48|3774.16|4604.08|12.88|0.00|644.48|657.36|-3129.68| +2451541|35163|6622|45052|167946|4793|44487|4|206|644|47|76.34|96.95|41.68|0.00|1958.96|3587.98|4556.65|78.35|0.00|1958.96|2037.31|-1629.02| +2451541|35163|13130|45052|167946|4793|44487|4|287|644|38|91.05|125.64|48.99|55.84|1861.62|3459.90|4774.32|72.23|55.84|1805.78|1878.01|-1654.12| +2451541|35163|3520|45052|167946|4793|44487|4|201|644|80|99.89|104.88|19.92|0.00|1593.60|7991.20|8390.40|143.42|0.00|1593.60|1737.02|-6397.60| +2451541|35163|16846|45052|167946|4793|44487|4|214|644|16|23.78|24.96|4.49|2.15|71.84|380.48|399.36|4.18|2.15|69.69|73.87|-310.79| +2451541|35163|3139|45052|167946|4793|44487|4|250|644|79|87.31|119.61|13.15|0.00|1038.85|6897.49|9449.19|0.00|0.00|1038.85|1038.85|-5858.64| +2451541|35163|6632|45052|167946|4793|44487|4|14|644|93|69.47|82.66|25.62|0.00|2382.66|6460.71|7687.38|166.78|0.00|2382.66|2549.44|-4078.05| +2451541|35163|8383|45052|167946|4793|44487|4|186|644|35|73.30|105.55|33.77|0.00|1181.95|2565.50|3694.25|70.91|0.00|1181.95|1252.86|-1383.55| +2451541|35163|10717|45052|167946|4793|44487|4|19|644|80|61.73|98.15|5.88|277.53|470.40|4938.40|7852.00|17.35|277.53|192.87|210.22|-4745.53| +2451541|35163|11764|45052|167946|4793|44487|4|208|644|76|18.42|33.70|33.70|0.00|2561.20|1399.92|2561.20|51.22|0.00|2561.20|2612.42|1161.28| +2451541|35163|14006|45052|167946|4793|44487|4|249|644|88|82.82|144.93|17.39|1515.01|1530.32|7288.16|12753.84|1.07|1515.01|15.31|16.38|-7272.85| +2452240|61813|17255|49358|668947|2936|47278|8|21|645|35|15.43|21.44|6.86|0.00|240.10|540.05|750.40|2.40|0.00|240.10|242.50|-299.95| +2452240|61813|12571|49358|668947|2936|47278|8|174|645|14|56.42|62.62|16.28|0.00|227.92|789.88|876.68|13.67|0.00|227.92|241.59|-561.96| +2452240|61813|10177|49358|668947|2936|47278|8|276|645|47|92.73|140.02|35.00|0.00|1645.00|4358.31|6580.94|148.05|0.00|1645.00|1793.05|-2713.31| +2452240|61813|8597|49358|668947|2936|47278|8|176|645|48|88.86|112.85|107.20|411.64|5145.60|4265.28|5416.80|236.69|411.64|4733.96|4970.65|468.68| +2452240|61813|3621|49358|668947|2936|47278|8|263|645|96|70.33|109.01|95.92|0.00|9208.32|6751.68|10464.96|552.49|0.00|9208.32|9760.81|2456.64| +2452240|61813|12487|49358|668947|2936|47278|8|174|645|75|69.99|81.18|0.00|0.00|0.00|5249.25|6088.50|0.00|0.00|0.00|0.00|-5249.25| +2452240|61813|713|49358|668947|2936|47278|8|235|645|69|20.50|40.59|39.77|0.00|2744.13|1414.50|2800.71|219.53|0.00|2744.13|2963.66|1329.63| +2452240|61813|14617|49358|668947|2936|47278|8|163|645|97|20.10|24.52|6.86|0.00|665.42|1949.70|2378.44|6.65|0.00|665.42|672.07|-1284.28| +2452240|61813|5013|49358|668947|2936|47278|8|236|645|55|81.80|108.79|1.08|0.00|59.40|4499.00|5983.45|2.97|0.00|59.40|62.37|-4439.60| +2452240||14695|49358||2936||||645|7|35.75|||0.00|||290.29||0.00|||-125.44| +2452240|61813|15325|49358|668947|2936|47278|8|48|645|77|14.63|25.60|11.77|0.00|906.29|1126.51|1971.20|36.25|0.00|906.29|942.54|-220.22| +2452240|61813|12679|49358|668947|2936|47278|8|101|645|48|42.08|63.12|47.97|0.00|2302.56|2019.84|3029.76|69.07|0.00|2302.56|2371.63|282.72| +2452240|61813|11173|49358|668947|2936|47278|8|199|645|48|31.15|59.80|24.51|0.00|1176.48|1495.20|2870.40|70.58|0.00|1176.48|1247.06|-318.72| +2452240|61813|17401|49358|668947|2936|47278|8|262|645|73|97.66|143.56|142.12|0.00|10374.76|7129.18|10479.88|414.99|0.00|10374.76|10789.75|3245.58| +2452240|61813|933|49358|668947|2936|47278|8|166|645|32|23.86|38.65|30.14|424.37|964.48|763.52|1236.80|43.20|424.37|540.11|583.31|-223.41| +2452240|61813|17915|49358|668947|2936|47278|8|110|645|16|42.93|85.86|60.96|224.33|975.36|686.88|1373.76|0.00|224.33|751.03|751.03|64.15| +2451172|45288|15574|76841|956953|5865|47177|2|7|646|60|22.49|41.83|23.84|0.00|1430.40|1349.40|2509.80|114.43|0.00|1430.40|1544.83|81.00| +2451172|45288|6760|76841|956953|5865|47177|2|113|646|5|68.64|107.07|66.38|159.31|331.90|343.20|535.35|12.08|159.31|172.59|184.67|-170.61| +2451172|45288|6644|76841|956953|5865|47177|2|187|646|65|65.86|81.00|36.45|0.00|2369.25|4280.90|5265.00|23.69|0.00|2369.25|2392.94|-1911.65| +2451172|45288|15961|76841|956953|5865|47177|2|284|646|3|47.00|92.59|60.18|0.00|180.54|141.00|277.77|9.02|0.00|180.54|189.56|39.54| +2451172||11912||||47177|2|28|646|||119.89|89.91||2787.21||3716.59||||3038.05|| +2451172|45288|15904|76841|956953|5865|47177|2|296|646|89|6.10|6.28|4.14|0.00|368.46|542.90|558.92|14.73|0.00|368.46|383.19|-174.44| +||4714||956953|5865||2||646|||37.16||0.00|1792.08||2675.52|107.52|0.00|||140.40| +2451172|45288|15397|76841|956953|5865|47177|2|217|646|18|83.59|88.60|81.51|0.00|1467.18|1504.62|1594.80|73.35|0.00|1467.18|1540.53|-37.44| +2451172|45288|17936|76841|956953|5865|47177|2|207|646|46|29.10|45.39|28.14|232.99|1294.44|1338.60|2087.94|42.45|232.99|1061.45|1103.90|-277.15| +||14458|||5865||2|215|646||||||196.94|||||196.94|206.78|-2842.30| +2451172|45288|11984|76841|956953|5865|47177|2|202|646|2|19.59|29.38|14.69|0.00|29.38|39.18|58.76|1.46|0.00|29.38|30.84|-9.80| +2451172|45288|11384|76841|956953|5865|47177|2|213|646|75|88.09|101.30|73.94|0.00|5545.50|6606.75|7597.50|0.00|0.00|5545.50|5545.50|-1061.25| +2452608|52937|10992|81613|1911755|4297|31192|2|238|647|41|62.64|69.53|13.21|0.00|541.61|2568.24|2850.73|43.32|0.00|541.61|584.93|-2026.63| +2452608|52937|11149|81613|1911755|4297|31192|2|71|647|96|36.50|49.27|7.88|0.00|756.48|3504.00|4729.92|45.38|0.00|756.48|801.86|-2747.52| +2452608|52937|456|81613|1911755|4297|31192|2|211|647|41|35.51|41.54|23.67|427.00|970.47|1455.91|1703.14|38.04|427.00|543.47|581.51|-912.44| +2452608|52937|6031|81613|1911755|4297|31192|2|186|647|73|47.68|88.68|0.88|0.00|64.24|3480.64|6473.64|4.49|0.00|64.24|68.73|-3416.40| +2452608|52937|9927|81613|1911755|4297|31192|2|87|647|26|12.95|16.83|12.11|0.00|314.86|336.70|437.58|12.59|0.00|314.86|327.45|-21.84| +2452608|52937|16345|81613|1911755|4297|31192|2|229|647|76|61.30|101.14|1.01|0.00|76.76|4658.80|7686.64|2.30|0.00|76.76|79.06|-4582.04| +2452608|52937|6270|81613|1911755|4297|31192|2|42|647|43|73.75|94.40|68.91|0.00|2963.13|3171.25|4059.20|207.41|0.00|2963.13|3170.54|-208.12| +2452608|52937|2937|||||2|213|647|25|||||||309.00||||33.21|-247.75| +2452608|52937|6589|81613|1911755|4297|31192|2|107|647|22|13.11|18.35|13.39|176.74|294.58|288.42|403.70|4.71|176.74|117.84|122.55|-170.58| +2452608|52937|13267|81613|1911755|4297|31192|2|202|647|97|53.82|69.42|11.80|0.00|1144.60|5220.54|6733.74|34.33|0.00|1144.60|1178.93|-4075.94| +2452608|52937|5913|81613|1911755|4297|31192|2|77|647|89|78.48|103.59|15.53|0.00|1382.17|6984.72|9219.51|27.64|0.00|1382.17|1409.81|-5602.55| +2452608|52937|7657|81613|1911755|4297|31192|2|143|647|65|93.92|129.60|60.91|0.00|3959.15|6104.80|8424.00|118.77|0.00|3959.15|4077.92|-2145.65| +2452608|52937|7308|81613|1911755|4297|31192|2|228|647|68|97.52|177.48|133.11|7603.24|9051.48|6631.36|12068.64|28.96|7603.24|1448.24|1477.20|-5183.12| +2452608|52937|3114|81613|1911755|4297|31192|2|209|647|11|95.30|136.27|136.27|344.76|1498.97|1048.30|1498.97|103.87|344.76|1154.21|1258.08|105.91| +2452208|51258|8693|26202|1348268|3934|30800|8|244|648|54|95.32|113.43|44.23|0.00|2388.42|5147.28|6125.22|95.53|0.00|2388.42|2483.95|-2758.86| +2452208|51258|5625|26202|1348268|3934|30800|8|59|648|49|78.30|79.86|76.66|0.00|3756.34|3836.70|3913.14|300.50|0.00|3756.34|4056.84|-80.36| +2452208|51258|3209|26202|1348268|3934|30800|8|124|648|2|71.80|99.08|96.10|0.00|192.20|143.60|198.16|7.68|0.00|192.20|199.88|48.60| +2452208|51258|10705|26202|1348268|3934|30800|8|26|648|37|87.26|166.66|76.66|28.36|2836.42|3228.62|6166.42|168.48|28.36|2808.06|2976.54|-420.56| +2452208|51258|15263|26202|1348268|3934|30800|8|13|648|41|68.93|136.48|106.45|0.00|4364.45|2826.13|5595.68|392.80|0.00|4364.45|4757.25|1538.32| +2452208|51258|8739|26202|1348268|3934|30800|8|14|648|79|46.48|65.07|49.45|0.00|3906.55|3671.92|5140.53|39.06|0.00|3906.55|3945.61|234.63| +2452208|51258|6277|26202|1348268|3934|30800|8|180|648|49|82.76|139.86|64.33|0.00|3152.17|4055.24|6853.14|126.08|0.00|3152.17|3278.25|-903.07| +2452208|51258|10715|26202|1348268|3934|30800|8|257|648|26|6.26|7.69|1.61|0.00|41.86|162.76|199.94|0.83|0.00|41.86|42.69|-120.90| +||16971|26202|1348268|3934|||191|648|||46.41||||||0.00|||2536.83|| +2452208|51258|10929|26202|1348268|3934|30800|8|94|648|83|82.16|114.20|106.20|0.00|8814.60|6819.28|9478.60|0.00|0.00|8814.60|8814.60|1995.32| +2452208|51258|2997|26202|1348268|3934|30800|8|3|648|28|16.61|24.58|20.40|114.24|571.20|465.08|688.24|22.84|114.24|456.96|479.80|-8.12| +2452158||11773|62867|||||215|649||69.73|113.65|56.82|0.00||||145.45|0.00|1818.24|1963.69|-413.12| +2452158|32077|17159|62867|1825907|2994|40481|8|213|649|71|15.13|23.14|20.82|0.00|1478.22|1074.23|1642.94|103.47|0.00|1478.22|1581.69|403.99| +2452158|32077|9203|62867|1825907|2994|40481|8|36|649|62|82.54|143.61|110.57|0.00|6855.34|5117.48|8903.82|274.21|0.00|6855.34|7129.55|1737.86| +2452158|32077|16305|62867|1825907|2994|40481|8|223|649|56|94.35|144.35|121.25|0.00|6790.00|5283.60|8083.60|135.80|0.00|6790.00|6925.80|1506.40| +2452158|32077|7763|62867|1825907|2994|40481|8|143|649|19|56.04|100.31|65.20|0.00|1238.80|1064.76|1905.89|111.49|0.00|1238.80|1350.29|174.04| +2452158|32077|17437|62867|1825907|2994|40481|8|251|649|34|85.00|151.30|24.20|0.00|822.80|2890.00|5144.20|24.68|0.00|822.80|847.48|-2067.20| +2452158|32077|3319|62867|1825907|2994|40481|8|295|649|40|21.89|31.08|1.86|0.00|74.40|875.60|1243.20|1.48|0.00|74.40|75.88|-801.20| +2452158|32077|3045|62867|1825907|2994|40481|8|298|649|40|99.47|167.10|11.69|388.10|467.60|3978.80|6684.00|6.36|388.10|79.50|85.86|-3899.30| +2452158|32077|14543|62867|1825907|2994|40481|8|26|649|38|49.15|98.30|42.26|0.00|1605.88|1867.70|3735.40|16.05|0.00|1605.88|1621.93|-261.82| +2451181||1676|64920||1176||8||650|||51.37|5.65||395.50|||||395.50|403.41|| +2451181|67842|9812|64920|1479048|1176|857|8|109|650|62|19.84|23.80|14.04|835.66|870.48|1230.08|1475.60|0.00|835.66|34.82|34.82|-1195.26| +2451181|67842|11840|64920|1479048|1176|857|8|56|650|1|68.38|92.31|44.30|0.00|44.30|68.38|92.31|3.98|0.00|44.30|48.28|-24.08| +2451181|67842|7660|64920|1479048|1176|857|8|136|650|20|61.82|78.51|21.19|0.00|423.80|1236.40|1570.20|16.95|0.00|423.80|440.75|-812.60| +2451181|67842|12812|64920|1479048|1176|857|8|177|650|11|70.94|127.69|74.06|0.00|814.66|780.34|1404.59|73.31|0.00|814.66|887.97|34.32| +2451181|67842|7819|64920|1479048|1176|857|8|202|650|60|59.71|115.83|44.01|0.00|2640.60|3582.60|6949.80|52.81|0.00|2640.60|2693.41|-942.00| +2451181|67842|8558|64920|1479048|1176|857|8|8|650|63|79.82|110.15|19.82|0.00|1248.66|5028.66|6939.45|37.45|0.00|1248.66|1286.11|-3780.00| +2451181|67842|17497|64920|1479048|1176|857|8|295|650|98|35.58|51.23|39.95|0.00|3915.10|3486.84|5020.54|352.35|0.00|3915.10|4267.45|428.26| +2451181|67842|13783|64920|1479048|1176|857|8|32|650|94|68.53|82.23|10.68|20.07|1003.92|6441.82|7729.62|49.19|20.07|983.85|1033.04|-5457.97| +2451181|67842|3460|64920|1479048|1176|857|8|74|650|56|63.20|98.59|97.60|0.00|5465.60|3539.20|5521.04|163.96|0.00|5465.60|5629.56|1926.40| +2451181|67842|691|64920|1479048|1176|857|8|251|650|74|26.39|51.46|1.02|0.00|75.48|1952.86|3808.04|3.01|0.00|75.48|78.49|-1877.38| +2452568|45166|17877|10722|894758|394|17230|2|169|651|96|94.98|135.82|96.43|1018.30|9257.28|9118.08|13038.72|494.33|1018.30|8238.98|8733.31|-879.10| +2452568|45166|8745|10722|894758|394|17230|2|272|651|30|33.09|64.85|14.26|0.00|427.80|992.70|1945.50|12.83|0.00|427.80|440.63|-564.90| +2452568|45166|14545|10722|894758|394|17230|2|95|651|34|86.83|134.58|121.12|0.00|4118.08|2952.22|4575.72|205.90|0.00|4118.08|4323.98|1165.86| +2452568|45166|16035|10722|894758|394|17230|2|103|651|96|70.25|89.21|49.06|0.00|4709.76|6744.00|8564.16|423.87|0.00|4709.76|5133.63|-2034.24| +2452568|45166|4837|10722|894758|394|17230|2|172|651|45|6.40|6.72|0.60|0.00|27.00|288.00|302.40|0.54|0.00|27.00|27.54|-261.00| +2452568|45166|9753|10722|894758|394|17230|2|171|651|100|10.28|16.75|7.37|0.00|737.00|1028.00|1675.00|0.00|0.00|737.00|737.00|-291.00| +2452568|45166|4221|10722|894758|394|17230|2|57|651|82|35.11|67.41|32.35|0.00|2652.70|2879.02|5527.62|79.58|0.00|2652.70|2732.28|-226.32| +2452568|45166|17262|10722|894758|394|17230|2|89|651|20|89.41|130.53|70.48|0.00|1409.60|1788.20|2610.60|98.67|0.00|1409.60|1508.27|-378.60| +2452568|45166|6427|10722|894758|394|17230|2|122|651|60|44.39|65.69|61.74|0.00|3704.40|2663.40|3941.40|259.30|0.00|3704.40|3963.70|1041.00| +|45166|4453|10722||||2||651|||22.82|0.22|0.00|||1643.04|0.31|0.00||16.15|-1051.20| +2452452|68790|5905|60299|1228786|4920|33054|7|200|652|43|20.00|37.60|0.75|0.00|32.25|860.00|1616.80|2.25|0.00|32.25|34.50|-827.75| +2452452|68790|8955|60299|1228786|4920|33054|7|281|652|30|82.69|152.14|71.50|0.00|2145.00|2480.70|4564.20|150.15|0.00|2145.00|2295.15|-335.70| +2452452|68790|15180|60299|1228786|4920|33054|7|223|652|72|87.67|111.34|34.51|1739.30|2484.72|6312.24|8016.48|52.17|1739.30|745.42|797.59|-5566.82| +2452452|68790|4795|60299|1228786|4920|33054|7|79|652|5|58.19|62.26|41.09|0.00|205.45|290.95|311.30|2.05|0.00|205.45|207.50|-85.50| +2452452|68790|17256|60299|1228786|4920|33054|7|230|652|15|66.92|75.61|74.09|0.00|1111.35|1003.80|1134.15|66.68|0.00|1111.35|1178.03|107.55| +2452452|68790|12571|60299|1228786|4920|33054|7|150|652|58|86.14|98.19|41.23|215.22|2391.34|4996.12|5695.02|130.56|215.22|2176.12|2306.68|-2820.00| +2452452|68790|10177|60299|1228786|4920|33054|7|27|652|49|61.49|67.63|62.89|0.00|3081.61|3013.01|3313.87|123.26|0.00|3081.61|3204.87|68.60| +2452452|68790|8598|60299|1228786|4920|33054|7|162|652|5|29.06|57.82|24.28|115.33|121.40|145.30|289.10|0.00|115.33|6.07|6.07|-139.23| +2452452|68790|3621|60299|1228786|4920|33054|7|139|652|24|37.37|42.97|30.07|0.00|721.68|896.88|1031.28|0.00|0.00|721.68|721.68|-175.20| +2451811|34557|10567|20358|1405473|4988|35656|4|274|653|31|66.12|94.55|51.05|0.00|1582.55|2049.72|2931.05|110.77|0.00|1582.55|1693.32|-467.17| +2451811|34557|6740|20358|1405473|4988|35656|4|189|653|99|81.70|155.23|96.24|8384.42|9527.76|8088.30|15367.77|45.73|8384.42|1143.34|1189.07|-6944.96| +2451811|34557|13481|20358|1405473|4988|35656|4|74|653|75|99.14|172.50|72.45|597.71|5433.75|7435.50|12937.50|48.36|597.71|4836.04|4884.40|-2599.46| +2451811|34557|2845|20358|1405473|4988|35656|4|107|653|92|44.38|80.32|66.66|0.00|6132.72|4082.96|7389.44|61.32|0.00|6132.72|6194.04|2049.76| +|34557|3401||1405473||||248|653||||||526.68|1640.10|2508.66|5.26|||531.94|| +2451811|34557|7025|20358|1405473|4988|35656|4|156|653|36|92.46|137.76|9.64|0.00|347.04|3328.56|4959.36|6.94|0.00|347.04|353.98|-2981.52| +2451811|34557|9349|20358|1405473|4988|35656|4|174|653|100|47.02|81.34|50.43|0.00|5043.00|4702.00|8134.00|403.44|0.00|5043.00|5446.44|341.00| +2451811|34557|13657|20358|1405473|4988|35656|4|63|653|48|34.27|58.94|34.77|0.00|1668.96|1644.96|2829.12|33.37|0.00|1668.96|1702.33|24.00| +2451811|34557|5252|20358|1405473|4988|35656|4|252|653|41|93.19|121.14|27.86|79.95|1142.26|3820.79|4966.74|53.11|79.95|1062.31|1115.42|-2758.48| +2451811|34557|5575|20358|1405473|4988|35656|4|176|653|54|77.20|82.60|21.47|0.00|1159.38|4168.80|4460.40|69.56|0.00|1159.38|1228.94|-3009.42| +2451811|34557|4753|20358|1405473|4988|35656|4|49|653|37|76.19|98.28|53.07|451.62|1963.59|2819.03|3636.36|15.11|451.62|1511.97|1527.08|-1307.06| +2451811|34557|17813|20358|1405473|4988|35656|4|216|653|76|41.26|62.71|48.28|0.00|3669.28|3135.76|4765.96|293.54|0.00|3669.28|3962.82|533.52| +2451811|34557|16981|20358|1405473|4988|35656|4|134|653|15|18.16|23.24|6.04|0.00|90.60|272.40|348.60|8.15|0.00|90.60|98.75|-181.80| +2451811|34557|5660|20358|1405473|4988|35656|4|116|653|45|33.02|62.40|16.22|0.00|729.90|1485.90|2808.00|36.49|0.00|729.90|766.39|-756.00| +|34557|4733|20358|1405473||||182|653|||137.82||0.00||4853.00|||0.00||788.32|| +2452260|51570|13747|1309|23009|4262|46362|8|61|654|58|62.13|96.92|75.59|2630.53|4384.22|3603.54|5621.36|35.07|2630.53|1753.69|1788.76|-1849.85| +2452260|51570|11925|1309|23009|4262|46362|8|2|654|91|8.61|11.10|8.54|0.00|777.14|783.51|1010.10|15.54|0.00|777.14|792.68|-6.37| +2452260|51570|4371|1309|23009|4262|46362|8|13|654|49|66.06|114.28|20.57|0.00|1007.93|3236.94|5599.72|70.55|0.00|1007.93|1078.48|-2229.01| +2452260|51570|6533|1309|23009|4262|46362|8|159|654|77|4.64|7.47|3.73|0.00|287.21|357.28|575.19|2.87|0.00|287.21|290.08|-70.07| +2452260|51570|1289|1309|23009|4262|46362|8|120|654|88|78.90|126.24|65.64|2772.63|5776.32|6943.20|11109.12|180.22|2772.63|3003.69|3183.91|-3939.51| +2452260|51570|13797|1309|23009|4262|46362|8|239|654|99|99.50|134.32|96.71|0.00|9574.29|9850.50|13297.68|478.71|0.00|9574.29|10053.00|-276.21| +2452260|51570|5647|1309|23009|4262|46362|8|97|654|14|68.14|68.14|1.36|0.00|19.04|953.96|953.96|1.33|0.00|19.04|20.37|-934.92| +2452260|51570|291|1309|23009|4262|46362|8|56|654|35|3.55|6.78|1.01|0.00|35.35|124.25|237.30|1.76|0.00|35.35|37.11|-88.90| +||17285|1309|23009||46362|||654|||||0.00|7137.76||9035.60|285.51|0.00|7137.76|7423.27|1694.56| +2452260|51570|10591|1309|23009|4262|46362|8|27|654|40|57.30|103.14|69.10|0.00|2764.00|2292.00|4125.60|0.00|0.00|2764.00|2764.00|472.00| +2452260|51570|11463|1309|23009|4262|46362|8|111|654|35|42.74|58.98|57.21|0.00|2002.35|1495.90|2064.30|80.09|0.00|2002.35|2082.44|506.45| +2452260|51570|12363|1309|23009|4262|46362|8|73|654|87|71.91|127.99|84.47|0.00|7348.89|6256.17|11135.13|0.00|0.00|7348.89|7348.89|1092.72| +2452260|51570|10553|1309|23009|4262|46362|8|158|654|7|50.70|71.48|15.72|63.82|110.04|354.90|500.36|1.38|63.82|46.22|47.60|-308.68| +2452260|51570|2129|1309|23009|4262|46362|8|120|654|98|38.18|69.10|26.94|0.00|2640.12|3741.64|6771.80|211.20|0.00|2640.12|2851.32|-1101.52| +2452260|51570|4275|1309|23009|4262|46362|8|253|654|62|45.33|45.78|24.26|0.00|1504.12|2810.46|2838.36|135.37|0.00|1504.12|1639.49|-1306.34| +2452189|43976|1951|77145|324934|6323|3712|2|239|655|90|10.78|13.15|9.73|0.00|875.70|970.20|1183.50|43.78|0.00|875.70|919.48|-94.50| +2452189|43976|17349|77145|324934|6323|3712|2|221|655|89|2.00|2.30|2.20|135.10|195.80|178.00|204.70|1.82|135.10|60.70|62.52|-117.30| +2452189|43976|217|77145|324934|6323|3712|2|268|655|51|60.53|101.69|30.50|0.00|1555.50|3087.03|5186.19|62.22|0.00|1555.50|1617.72|-1531.53| +2452189|43976|14849|77145|324934|6323|3712|2|51|655|100|27.58|36.68|16.87|691.67|1687.00|2758.00|3668.00|79.62|691.67|995.33|1074.95|-1762.67| +2452189|43976|17875|77145|324934|6323|3712|2|197|655|6|12.77|18.89|12.08|0.00|72.48|76.62|113.34|2.17|0.00|72.48|74.65|-4.14| +2452189|43976|3239||324934|6323|3712|2|110|655|||81.46||||||||||| +2452189|43976|43|77145|324934|6323|3712|2|298|655|98|27.86|32.03|23.06|0.00|2259.88|2730.28|3138.94|180.79|0.00|2259.88|2440.67|-470.40| +2452189|43976|4817|77145|324934|6323|3712|2|61|655|39|66.43|115.58|25.42|951.72|991.38|2590.77|4507.62|2.37|951.72|39.66|42.03|-2551.11| +2451068|55662|2576|76516|1612777|2638|2899|1|248|656|81|85.66|129.34|38.80|0.00|3142.80|6938.46|10476.54|125.71|0.00|3142.80|3268.51|-3795.66| +2451068|55662|16975|76516|1612777|2638|2899|1|137|656|70|61.92|105.88|99.52|0.00|6966.40|4334.40|7411.60|626.97|0.00|6966.40|7593.37|2632.00| +2451068|55662|4102|76516|1612777|2638|2899|1|3|656|7|20.38|31.58|31.58|0.00|221.06|142.66|221.06|15.47|0.00|221.06|236.53|78.40| +2451068|55662|17792|76516|1612777|2638|2899|1|21|656|38|25.85|44.46|11.11|0.00|422.18|982.30|1689.48|0.00|0.00|422.18|422.18|-560.12| +2451068|55662|5564|76516|1612777|2638|2899|1|54|656|52|12.88|23.18|22.25|0.00|1157.00|669.76|1205.36|0.00|0.00|1157.00|1157.00|487.24| +2451068|55662|5764|76516|1612777|2638|2899|1|224|656|78|49.76|55.73|20.06|0.00|1564.68|3881.28|4346.94|15.64|0.00|1564.68|1580.32|-2316.60| +2451068|55662|5563|76516|1612777|2638|2899|1|128|656|73|54.57|79.12|31.64|0.00|2309.72|3983.61|5775.76|23.09|0.00|2309.72|2332.81|-1673.89| +2451068|55662|12907|||||||656|51|66.99||85.20|0.00|4345.20||5500.35|217.26|0.00||4562.46|| +2451068|55662|12523|76516|1612777|2638|2899|1|157|656|75|73.09|145.44|74.17|0.00|5562.75|5481.75|10908.00|445.02|0.00|5562.75|6007.77|81.00| +2451068|55662|4420|76516|1612777|2638|2899|1|134|656|16|12.82|13.97|7.54|19.30|120.64|205.12|223.52|4.05|19.30|101.34|105.39|-103.78| +2451068|55662|17950|76516|1612777|2638|2899|1|198|656|84|21.36|22.21|13.99|0.00|1175.16|1794.24|1865.64|11.75|0.00|1175.16|1186.91|-619.08| +2451068|55662|5536|76516|1612777|2638|2899|1|295|656|52|36.22|72.44|44.91|0.00|2335.32|1883.44|3766.88|210.17|0.00|2335.32|2545.49|451.88| +2451821|62058|8759|11383|1591619|7055|46091|8|217|657|38|54.61|101.57|78.20|0.00|2971.60|2075.18|3859.66|208.01|0.00|2971.60|3179.61|896.42| +2451821|62058|1367|11383|1591619|7055|46091|8|5|657|79|17.66|29.84|23.27|0.00|1838.33|1395.14|2357.36|0.00|0.00|1838.33|1838.33|443.19| +2451821|62058|17096|11383|1591619|7055|46091|8|150|657|94|98.77|129.38|111.26|7007.15|10458.44|9284.38|12161.72|207.07|7007.15|3451.29|3658.36|-5833.09| +2451821|62058|14963|11383|1591619|7055|46091|8|141|657|95|2.08|3.36|2.45|0.00|232.75|197.60|319.20|18.62|0.00|232.75|251.37|35.15| +2451821|62058|113|11383|1591619|7055|46091|8|288|657|90|87.33|108.28|34.64|0.00|3117.60|7859.70|9745.20|280.58|0.00|3117.60|3398.18|-4742.10| +2451821|62058|8665|11383|1591619|7055|46091|8|254|657|65|18.51|30.54|25.65|0.00|1667.25|1203.15|1985.10|0.00|0.00|1667.25|1667.25|464.10| +2451821|62058|14425|11383|1591619|7055|46091|8|118|657|87|75.23|89.52|61.76|5373.12|5373.12|6545.01|7788.24|0.00|5373.12|0.00|0.00|-6545.01| +2451821|62058|11657|11383|1591619|7055|46091|8|75|657|3|62.74|67.13|6.71|0.00|20.13|188.22|201.39|0.40|0.00|20.13|20.53|-168.09| +2451821|62058|9716|11383|1591619|7055|46091|8|199|657|98|14.59|16.77|12.91|0.00|1265.18|1429.82|1643.46|25.30|0.00|1265.18|1290.48|-164.64| +2451821|62058|13819|11383|1591619|7055|46091|8|134|657|70|80.36|108.48|11.93|0.00|835.10|5625.20|7593.60|25.05|0.00|835.10|860.15|-4790.10| +2451821|62058|5521|11383|1591619|7055|46091|8|160|657|63|30.77|44.61|36.13|0.00|2276.19|1938.51|2810.43|182.09|0.00|2276.19|2458.28|337.68| +2451522|34121|17215|25120|1745830|4493|30803|8|57|658|50|25.39|34.02|17.35|746.05|867.50|1269.50|1701.00|9.71|746.05|121.45|131.16|-1148.05| +2451522|34121|4681|25120|1745830|4493|30803|8|108|658|38|20.51|35.48|12.77|0.00|485.26|779.38|1348.24|29.11|0.00|485.26|514.37|-294.12| +2451522|34121|1579|25120|1745830|4493|30803|8|1|658|89|70.38|80.23|76.21|0.00|6782.69|6263.82|7140.47|474.78|0.00|6782.69|7257.47|518.87| +2451522|34121|17983|25120|1745830|4493|30803|8|194|658|14|41.67|72.08|20.90|225.30|292.60|583.38|1009.12|2.69|225.30|67.30|69.99|-516.08| +2451522|34121|5378|25120|1745830|4493|30803|8|276|658|86|90.93|131.84|96.24|0.00|8276.64|7819.98|11338.24|248.29|0.00|8276.64|8524.93|456.66| +2451522|34121|9530|25120|1745830|4493|30803|8|112|658|48|60.02|82.22|53.44|0.00|2565.12|2880.96|3946.56|153.90|0.00|2565.12|2719.02|-315.84| +2451522|34121|12902|25120|1745830|4493|30803|8|127|658|29|18.13|21.03|0.00|0.00|0.00|525.77|609.87|0.00|0.00|0.00|0.00|-525.77| +2451522|34121|16340|25120|1745830|4493|30803|8|167|658|2|52.61|80.49|70.83|0.00|141.66|105.22|160.98|8.49|0.00|141.66|150.15|36.44| +2451522|34121|278|25120|1745830|4493|30803|8|86|658|33|16.56|32.12|0.32|0.00|10.56|546.48|1059.96|0.10|0.00|10.56|10.66|-535.92| +2451522|34121|15338|25120|1745830|4493|30803|8|144|658|63|28.85|45.58|39.19|0.00|2468.97|1817.55|2871.54|222.20|0.00|2468.97|2691.17|651.42| +2451522|34121|1540|25120|1745830|4493|30803|8|147|658|18|39.38|48.43|43.10|0.00|775.80|708.84|871.74|0.00|0.00|775.80|775.80|66.96| +2451522|34121|12577|25120|1745830|4493|30803|8|210|658|29|8.10|10.12|7.79|0.00|225.91|234.90|293.48|4.51|0.00|225.91|230.42|-8.99| +2451522|34121|2114|25120|1745830|4493|30803|8|225|658|99|14.36|14.79|1.92|148.26|190.08|1421.64|1464.21|3.34|148.26|41.82|45.16|-1379.82| +2451522|34121|7219|25120|1745830|4493|30803|8|191|658|82|67.40|80.88|52.57|0.00|4310.74|5526.80|6632.16|301.75|0.00|4310.74|4612.49|-1216.06| +2452245|40424|6693|69457|1069512|1120|9528|8|153|659|91|64.36|74.01|28.86|0.00|2626.26|5856.76|6734.91|26.26|0.00|2626.26|2652.52|-3230.50| +2452245|40424|10731|69457|1069512|1120|9528|8|122|659|7|18.83|37.66|9.79|0.00|68.53|131.81|263.62|6.16|0.00|68.53|74.69|-63.28| +2452245|40424|17839|69457|1069512|1120|9528|8|77|659|76|53.59|76.63|72.79|0.00|5532.04|4072.84|5823.88|221.28|0.00|5532.04|5753.32|1459.20| +2452245|40424|10997|69457|1069512|1120|9528|8|62|659|59|29.83|46.23|15.25|0.00|899.75|1759.97|2727.57|80.97|0.00|899.75|980.72|-860.22| +2452245|40424|15933|69457|1069512|1120|9528|8|110|659|98|46.83|69.30|48.51|0.00|4753.98|4589.34|6791.40|190.15|0.00|4753.98|4944.13|164.64| +2452245|40424|9631|69457|1069512|1120|9528|8|184|659|22|39.37|48.81|16.59|0.00|364.98|866.14|1073.82|3.64|0.00|364.98|368.62|-501.16| +2452245|40424|10633|69457|1069512|1120|9528|8|89|659|46|73.03|97.86|36.20|0.00|1665.20|3359.38|4501.56|133.21|0.00|1665.20|1798.41|-1694.18| +2452245|40424|2103||1069512|1120|9528|8|143|659|33|||61.23|||2800.71|5461.17|||2020.59|2121.61|| +2452245|40424|2749|69457|1069512|1120|9528|8|54|659|42|19.96|20.15|12.69|85.27|532.98|838.32|846.30|13.43|85.27|447.71|461.14|-390.61| +2452245|40424|7255|69457|1069512|1120|9528|8|54|659|84|80.49|116.71|103.87|0.00|8725.08|6761.16|9803.64|0.00|0.00|8725.08|8725.08|1963.92| +2452245|40424|15259|69457|1069512|1120|9528|8|102|659|96|80.12|130.59|57.45|0.00|5515.20|7691.52|12536.64|496.36|0.00|5515.20|6011.56|-2176.32| +2452245||11019|69457||1120|9528||277|659|72|84.53||41.68||3000.96|6086.16|8337.60|||||-3085.20| +2452245|40424|14555|69457|1069512|1120|9528|8|252|659|78|43.02|84.31|37.09|0.00|2893.02|3355.56|6576.18|202.51|0.00|2893.02|3095.53|-462.54| +2452245|40424|2223|69457|1069512|1120|9528|8|19|659|22|48.48|80.96|59.91|0.00|1318.02|1066.56|1781.12|13.18|0.00|1318.02|1331.20|251.46| +2452245|40424|3557|69457|1069512|1120|9528|8|282|659|76|50.12|93.22|33.55|0.00|2549.80|3809.12|7084.72|50.99|0.00|2549.80|2600.79|-1259.32| +2452245|40424|8459|69457|1069512|1120|9528|8|212|659|2|53.72|100.99|16.15|0.00|32.30|107.44|201.98|2.58|0.00|32.30|34.88|-75.14| +2450927|38518|11632|6351|577665|4179|45493|4|273|660|47|17.34|19.24|18.66|87.70|877.02|814.98|904.28|47.35|87.70|789.32|836.67|-25.66| +2450927|38518|15484|6351|577665|4179|45493|4|61|660|27|33.85|40.95|30.30|261.79|818.10|913.95|1105.65|33.37|261.79|556.31|589.68|-357.64| +2450927|38518|12418|6351|577665|4179|45493|4|17|660|63|71.49|139.40|71.09|0.00|4478.67|4503.87|8782.20|223.93|0.00|4478.67|4702.60|-25.20| +2450927|38518|13306|6351|577665|4179|45493|4|183|660|91|99.97|167.94|87.32|0.00|7946.12|9097.27|15282.54|238.38|0.00|7946.12|8184.50|-1151.15| +2450927|38518|17066|6351|577665|4179|45493|4|144|660|65|87.68|104.33|10.43|0.00|677.95|5699.20|6781.45|33.89|0.00|677.95|711.84|-5021.25| +2450927|38518|10322|6351|577665|4179|45493|4|232|660|15|99.39|143.12|83.00|0.00|1245.00|1490.85|2146.80|37.35|0.00|1245.00|1282.35|-245.85| +2450927|38518|12992|6351|577665|4179|45493|4|79|660|44|57.73|95.83|31.62|0.00|1391.28|2540.12|4216.52|0.00|0.00|1391.28|1391.28|-1148.84| +2450927|38518|538|6351|577665|4179|45493|4|234|660|76|13.78|14.60|9.49|0.00|721.24|1047.28|1109.60|28.84|0.00|721.24|750.08|-326.04| +2450927|38518|1118|6351|577665|4179|45493|4|141|660|95|50.59|52.10|46.89|0.00|4454.55|4806.05|4949.50|178.18|0.00|4454.55|4632.73|-351.50| +2450927|38518|17608|6351|577665|4179|45493|4|261|660|62|70.60|134.14|111.33|0.00|6902.46|4377.20|8316.68|483.17|0.00|6902.46|7385.63|2525.26| +2450927|38518|4006|6351|577665|4179|45493|4|112|660|1|34.05|49.71|31.31|0.00|31.31|34.05|49.71|1.25|0.00|31.31|32.56|-2.74| +2450927|38518|12103|6351|577665|4179|45493|4|150|660|50|82.32|126.77|54.51|0.00|2725.50|4116.00|6338.50|218.04|0.00|2725.50|2943.54|-1390.50| +|38518|11486||577665||45493|4|225|660|||145.08|37.72||1056.16|||||570.33|581.73|-2033.67| +2451505|53598|14347|93638|1281033|3628|42223|10|225|661|22|46.11|60.40|28.99|0.00|637.78|1014.42|1328.80|12.75|0.00|637.78|650.53|-376.64| +2451505|53598|11516|93638|1281033|3628|42223|10|203|661|98|68.29|119.50|21.51|0.00|2107.98|6692.42|11711.00|84.31|0.00|2107.98|2192.29|-4584.44| +2451505|53598|1099|93638|1281033|3628|42223|10|83|661|85|92.50|153.55|52.20|0.00|4437.00|7862.50|13051.75|399.33|0.00|4437.00|4836.33|-3425.50| +|53598|15541|||3628|42223|||661|15||114.36|53.74|620.69||||0.00|620.69|||-689.84| +2451505|53598|2242|93638|1281033|3628|42223|10|137|661|58|49.61|98.72|0.00|0.00|0.00|2877.38|5725.76|0.00|0.00|0.00|0.00|-2877.38| +2451505|53598|5846|93638|1281033|3628|42223|10|220|661|12|4.36|8.10|0.56|0.00|6.72|52.32|97.20|0.47|0.00|6.72|7.19|-45.60| +2451505|53598|9908|93638|1281033|3628|42223|10|81|661|78|11.22|15.25|5.33|0.00|415.74|875.16|1189.50|29.10|0.00|415.74|444.84|-459.42| +2451505|53598|13741|93638|1281033|3628|42223|10|290|661|34|30.64|40.44|0.40|0.00|13.60|1041.76|1374.96|1.08|0.00|13.60|14.68|-1028.16| +2451505|53598|15712|93638|1281033|3628|42223|10|113|661|80|42.81|75.34|67.05|0.00|5364.00|3424.80|6027.20|429.12|0.00|5364.00|5793.12|1939.20| +2451505|53598|16351|93638|1281033|3628|42223|10|240|661|98|65.51|113.33|3.39|0.00|332.22|6419.98|11106.34|23.25|0.00|332.22|355.47|-6087.76| +2451505|53598|11848|93638|1281033|3628|42223|10|75|661|17|3.66|5.63|5.46|0.00|92.82|62.22|95.71|1.85|0.00|92.82|94.67|30.60| +2451505|53598|11504|93638|1281033|3628|42223|10|121|661|58|97.69|188.54|64.10|0.00|3717.80|5666.02|10935.32|223.06|0.00|3717.80|3940.86|-1948.22| +2451303|38917|11108|36789|277174|6712|31097|1|233|662|99|45.95|51.46|14.40|0.00|1425.60|4549.05|5094.54|85.53|0.00|1425.60|1511.13|-3123.45| +2451303|38917|6626|36789|277174|6712|31097|1|180|662|39|54.23|71.04|33.38|0.00|1301.82|2114.97|2770.56|39.05|0.00|1301.82|1340.87|-813.15| +2451303|38917|7832|36789|277174|6712|31097|1|224|662|15|56.24|83.23|60.75|0.00|911.25|843.60|1248.45|45.56|0.00|911.25|956.81|67.65| +2451303|38917|11062|36789|277174|6712|31097|1|76|662|70|88.91|139.58|75.37|0.00|5275.90|6223.70|9770.60|211.03|0.00|5275.90|5486.93|-947.80| +2451303||11665||277174|||1||662|12|84.31|143.32||0.00||1011.72||1.20|0.00||121.56|-891.36| +2451303|38917|5446|36789|277174|6712|31097|1|200|662|93|24.27|41.98|13.85|0.00|1288.05|2257.11|3904.14|115.92|0.00|1288.05|1403.97|-969.06| +2451303|38917|1154|36789|||||253|662||5.16|||0.00||350.88||0.35|0.00|17.68|18.03|| +2451303|38917|6931|36789|277174|6712|31097|1|167|662|85|92.63|150.06|111.04|4907.96|9438.40|7873.55|12755.10|317.13|4907.96|4530.44|4847.57|-3343.11| +2451303|38917|6094|36789|277174|6712|31097|1|187|662|90|45.94|65.69|64.37|0.00|5793.30|4134.60|5912.10|173.79|0.00|5793.30|5967.09|1658.70| +2451303|38917|9524|36789|277174|6712|31097|1|233|662|96|21.00|40.95|38.08|1316.04|3655.68|2016.00|3931.20|210.56|1316.04|2339.64|2550.20|323.64| +2451303|38917|8281|36789|277174|6712|31097|1|60|662|1|95.89|190.82|108.76|0.00|108.76|95.89|190.82|7.61|0.00|108.76|116.37|12.87| +2451363|74395|946|64431|252674|3317|15913|2|160|663|19|20.07|26.89|13.17|0.00|250.23|381.33|510.91|22.52|0.00|250.23|272.75|-131.10| +2451363|74395|14164|64431|252674|3317|15913|2|2|663|34|64.18|127.71|74.07|0.00|2518.38|2182.12|4342.14|125.91|0.00|2518.38|2644.29|336.26| +2451363|74395|13213|64431|252674|3317|15913|2|240|663|30|31.38|58.99|48.37|551.41|1451.10|941.40|1769.70|80.97|551.41|899.69|980.66|-41.71| +2451363|74395|15700|64431|252674|3317|15913|2|105|663|23|97.96|110.69|33.20|0.00|763.60|2253.08|2545.87|15.27|0.00|763.60|778.87|-1489.48| +2451363|74395|10796|64431|252674|3317|15913|2|288|663|82|34.06|48.02|38.89|0.00|3188.98|2792.92|3937.64|191.33|0.00|3188.98|3380.31|396.06| +2451363|74395|13204|64431|252674|3317|15913|2|164|663|64|55.72|81.90|18.01|0.00|1152.64|3566.08|5241.60|34.57|0.00|1152.64|1187.21|-2413.44| +2451363|74395|15985|64431|252674|3317|15913|2|99|663|68|6.63|7.62|3.50|0.00|238.00|450.84|518.16|7.14|0.00|238.00|245.14|-212.84| +2451363|74395|14830|64431|252674|3317|15913|2|108|663|42|27.63|51.94|19.73|0.00|828.66|1160.46|2181.48|74.57|0.00|828.66|903.23|-331.80| +2451363|74395|8941|64431|252674|3317|15913|2|240|663|63|51.63|99.64|54.80|0.00|3452.40|3252.69|6277.32|172.62|0.00|3452.40|3625.02|199.71| +2451363|74395|5065|64431|252674|3317|15913|2|223|663|91|26.58|35.35|31.46|0.00|2862.86|2418.78|3216.85|85.88|0.00|2862.86|2948.74|444.08| +2451363|74395|5107|64431|252674|3317|15913|2|65|663|11|87.38|89.12|74.86|0.00|823.46|961.18|980.32|74.11|0.00|823.46|897.57|-137.72| +2451363|74395|16267|64431|252674|3317|15913|2|134|663|61|7.12|13.24|6.22|49.32|379.42|434.32|807.64|23.10|49.32|330.10|353.20|-104.22| +2451103|54001|15280|16969|516529|2136|44096|8|292|664|66|22.71|28.84|1.44|0.00|95.04|1498.86|1903.44|8.55|0.00|95.04|103.59|-1403.82| +2451103|54001|1340|16969|516529|2136|44096|8|254|664|38|53.98|78.81|38.61|0.00|1467.18|2051.24|2994.78|102.70|0.00|1467.18|1569.88|-584.06| +2451103|54001|2746|16969|516529|2136|44096|8|179|664|65|95.54|98.40|76.75|0.00|4988.75|6210.10|6396.00|448.98|0.00|4988.75|5437.73|-1221.35| +2451103|54001|17335|16969||2136||||664||49.88|84.29||2980.20||4090.16|6911.78||2980.20||3830.93|-297.16| +2451103|54001|730|16969|516529|2136|44096|8|99|664|90|21.61|32.19|15.12|0.00|1360.80|1944.90|2897.10|108.86|0.00|1360.80|1469.66|-584.10| +2451103|54001|2680|16969|516529|2136|44096|8|129|664|80|63.72|91.11|56.48|0.00|4518.40|5097.60|7288.80|406.65|0.00|4518.40|4925.05|-579.20| +2451103|54001|10418|16969|516529|2136|44096|8|212|664|29|50.40|50.90|45.30|0.00|1313.70|1461.60|1476.10|0.00|0.00|1313.70|1313.70|-147.90| +2451103|54001|16574|16969|516529|2136|44096|8|154|664|62|86.58|111.68|59.19|0.00|3669.78|5367.96|6924.16|293.58|0.00|3669.78|3963.36|-1698.18| +2451103|54001|7564|16969|516529|2136|44096|8|20|664|47|60.63|72.75|58.20|0.00|2735.40|2849.61|3419.25|109.41|0.00|2735.40|2844.81|-114.21| +2451103|54001|17926|16969|516529|2136|44096|8|297|664|39|23.50|45.59|4.55|0.00|177.45|916.50|1778.01|0.00|0.00|177.45|177.45|-739.05| +2451103|54001|12350|16969|516529|2136|44096|8|179|664|78|99.97|107.96|80.97|0.00|6315.66|7797.66|8420.88|0.00|0.00|6315.66|6315.66|-1482.00| +||1438||516529|2136||8||664||67.30|67.97||||||7.09||||| +2451103|54001|4387|16969|516529|2136|44096|8|75|664|52|83.17|108.95|46.84|0.00|2435.68|4324.84|5665.40|97.42|0.00|2435.68|2533.10|-1889.16| +2452038|52386|9421|76312|1281602|3245|3288|2|176|665|79|9.25|18.31|0.18|6.25|14.22|730.75|1446.49|0.39|6.25|7.97|8.36|-722.78| +||4379||1281602|||2|188|665|||173.47||0.00||9277.00||1530.00|0.00|||| +2452038|52386|8047|||3245|||46|665|||38.12||0.00|788.26|729.30|838.64||0.00|||58.96| +2452038|52386|12553|76312|1281602|3245|3288|2|50|665|60|4.61|5.16|2.21|0.00|132.60|276.60|309.60|7.95|0.00|132.60|140.55|-144.00| +2452038|52386|15533|76312|1281602|3245|3288|2|165|665|23|9.10|14.74|14.29|0.00|328.67|209.30|339.02|26.29|0.00|328.67|354.96|119.37| +2452038|52386|11937|76312|1281602|3245|3288|2|191|665|34|69.67|135.15|118.93|0.00|4043.62|2368.78|4595.10|0.00|0.00|4043.62|4043.62|1674.84| +2452038|52386|12299|76312|1281602|3245|3288|2|109|665|59|80.41|135.08|135.08|0.00|7969.72|4744.19|7969.72|557.88|0.00|7969.72|8527.60|3225.53| +2452038|52386|17547|76312|1281602|3245|3288|2|165|665|43|52.92|69.85|55.18|0.00|2372.74|2275.56|3003.55|23.72|0.00|2372.74|2396.46|97.18| +2452038|52386|3063|76312|1281602|3245|3288|2|233|665|71|19.57|24.65|20.95|0.00|1487.45|1389.47|1750.15|118.99|0.00|1487.45|1606.44|97.98| +2452038|52386|14591|76312|1281602|3245|3288|2|261|665|25|73.15|132.40|39.72|0.00|993.00|1828.75|3310.00|39.72|0.00|993.00|1032.72|-835.75| +2452038|52386|13197|76312|1281602|3245|3288|2|264|665|52|19.50|38.61|37.06|1002.10|1927.12|1014.00|2007.72|64.75|1002.10|925.02|989.77|-88.98| +2451601|51235|3275|26245|698164|2436|31155|2|153|666|21|84.66|102.43|56.33|0.00|1182.93|1777.86|2151.03|94.63|0.00|1182.93|1277.56|-594.93| +2451601|51235|14417|26245|698164|2436|31155|2|239|666|74|96.12|172.05|30.96|0.00|2291.04|7112.88|12731.70|183.28|0.00|2291.04|2474.32|-4821.84| +2451601|51235|10370|26245|698164|2436|31155|2|211|666|37|79.27|151.40|27.25|0.00|1008.25|2932.99|5601.80|70.57|0.00|1008.25|1078.82|-1924.74| +2451601|51235|12158|26245|698164|2436|31155|2|277|666|68|45.05|50.90|34.61|1600.36|2353.48|3063.40|3461.20|45.18|1600.36|753.12|798.30|-2310.28| +2451601|51235|1334|26245|698164|2436|31155|2|229|666|31|47.10|71.12|59.02|0.00|1829.62|1460.10|2204.72|54.88|0.00|1829.62|1884.50|369.52| +2451601|51235|16801|26245|698164|2436|31155|2|91|666|3|91.71|160.49|104.31|0.00|312.93|275.13|481.47|12.51|0.00|312.93|325.44|37.80| +2451601|51235|8813|26245|698164|2436|31155|2|110|666|15|13.90|15.70|0.47|0.00|7.05|208.50|235.50|0.63|0.00|7.05|7.68|-201.45| +2451601|51235|1781|26245|698164|2436|31155|2|175|666|29|68.76|119.64|16.74|0.00|485.46|1994.04|3469.56|14.56|0.00|485.46|500.02|-1508.58| +2451601|51235|1652|26245|698164|2436|31155|2|81|666|84|98.84|161.10|91.82|0.00|7712.88|8302.56|13532.40|385.64|0.00|7712.88|8098.52|-589.68| +2451601|51235|7112|26245|698164|2436|31155|2|282|666|82|54.45|104.54|42.86|0.00|3514.52|4464.90|8572.28|246.01|0.00|3514.52|3760.53|-950.38| +2451305|69776|10939|21638|282272|2906|7940|10|85|667|95|48.59|75.31|1.50|0.00|142.50|4616.05|7154.45|5.70|0.00|142.50|148.20|-4473.55| +2451305|69776|817|21638|282272|2906|7940|10|248|667|93|53.30|98.60|14.79|0.00|1375.47|4956.90|9169.80|123.79|0.00|1375.47|1499.26|-3581.43| +2451305|69776|14444|21638|282272|2906|7940|10|57|667|18|4.70|5.68|3.57|0.00|64.26|84.60|102.24|0.64|0.00|64.26|64.90|-20.34| +2451305|69776|7468|21638|282272|2906|7940|10|52|667|91|26.75|41.99|32.75|0.00|2980.25|2434.25|3821.09|29.80|0.00|2980.25|3010.05|546.00| +2451305|69776|374|21638|282272|2906|7940|10|198|667|74|80.87|84.10|35.32|0.00|2613.68|5984.38|6223.40|182.95|0.00|2613.68|2796.63|-3370.70| +2451305|69776|14557|21638|282272|2906|7940|10|248|667|42|47.11|83.38|51.69|1150.61|2170.98|1978.62|3501.96|0.00|1150.61|1020.37|1020.37|-958.25| +2451305|69776|3320|21638|282272|2906|7940|10|281|667|57|9.87|16.28|5.69|0.00|324.33|562.59|927.96|0.00|0.00|324.33|324.33|-238.26| +2451305|69776|10384|21638|282272|2906|7940|10|191|667|4|23.31|33.56|2.01|0.00|8.04|93.24|134.24|0.16|0.00|8.04|8.20|-85.20| +2451305|69776|5500|21638|282272|2906|7940|10|173|667|61|38.84|48.16|29.85|0.00|1820.85|2369.24|2937.76|127.45|0.00|1820.85|1948.30|-548.39| +2451305|69776|10562|21638|282272|2906|7940|10|190|667|80|64.13|84.65|27.08|1234.84|2166.40|5130.40|6772.00|37.26|1234.84|931.56|968.82|-4198.84| +2451305|69776|15358|21638|282272|2906|7940|10|268|667|73|36.14|48.78|5.36|0.00|391.28|2638.22|3560.94|11.73|0.00|391.28|403.01|-2246.94| +|31165|10714|5155||6779|||58|668|34|37.17||||||1693.20|48.07||||| +2451076|31165|16970|5155|40767|6779|17944|2|78|668|28|10.76|13.77|10.05|0.00|281.40|301.28|385.56|19.69|0.00|281.40|301.09|-19.88| +|31165|10928|5155|40767||||168|668|38|2.89|||0.00||109.82||12.68|0.00|||| +2451076|31165|2996|5155|40767|6779|17944|2|209|668|34|47.41|84.38|13.50|165.24|459.00|1611.94|2868.92|26.43|165.24|293.76|320.19|-1318.18| +2451076|31165|13171|5155|40767|6779|17944|2|199|668|55|50.57|60.17|15.04|0.00|827.20|2781.35|3309.35|0.00|0.00|827.20|827.20|-1954.15| +2451076|31165|7282|5155|40767|6779|17944|2|88|668|61|55.00|85.80|29.17|1530.25|1779.37|3355.00|5233.80|12.45|1530.25|249.12|261.57|-3105.88| +2451076|31165|15337|5155|40767|6779|17944|2|226|668|96|3.43|5.48|3.39|0.00|325.44|329.28|526.08|9.76|0.00|325.44|335.20|-3.84| +2451076|31165|17125|5155|40767|6779|17944|2|57|668|42|27.49|36.28|29.74|499.63|1249.08|1154.58|1523.76|22.48|499.63|749.45|771.93|-405.13| +2451076|31165|14780|5155|40767|6779|17944|2|156|668|16|9.39|18.59|12.45|0.00|199.20|150.24|297.44|1.99|0.00|199.20|201.19|48.96| +2451920|32404|13331|7734|1605090|2093|9096|2|14|669|86|37.33|46.28|33.32|0.00|2865.52|3210.38|3980.08|229.24|0.00|2865.52|3094.76|-344.86| +2451920||16909|7734|||9096|2||669|17||||||||||||| +2451920|32404|11707|7734|1605090|2093|9096|2|238|669|8|61.16|81.95|28.68|0.00|229.44|489.28|655.60|16.06|0.00|229.44|245.50|-259.84| +2451920|32404|13449|7734|1605090|2093|9096|2|76|669|18|54.83|82.79|54.64|0.00|983.52|986.94|1490.22|19.67|0.00|983.52|1003.19|-3.42| +2451920|32404|16837|7734|1605090|2093|9096|2|43|669|72|41.00|70.11|65.20|0.00|4694.40|2952.00|5047.92|375.55|0.00|4694.40|5069.95|1742.40| +2451920|32404|7303|7734|1605090|2093|9096|2|57|669|59|84.54|114.97|97.72|0.00|5765.48|4987.86|6783.23|230.61|0.00|5765.48|5996.09|777.62| +2451920|32404|8449|7734|1605090|2093|9096|2|128|669|13|78.75|151.20|21.16|0.00|275.08|1023.75|1965.60|2.75|0.00|275.08|277.83|-748.67| +2451920|32404|16429|7734|1605090|2093|9096|2|292|669|37|26.84|28.98|11.59|55.74|428.83|993.08|1072.26|7.46|55.74|373.09|380.55|-619.99| +2451920|32404|5329|7734|1605090|2093|9096|2|55|669|20|68.73|82.47|14.84|0.00|296.80|1374.60|1649.40|20.77|0.00|296.80|317.57|-1077.80| +2451920|32404|16551|7734|1605090|2093|9096|2|146|669|15|27.23|27.77|7.77|0.00|116.55|408.45|416.55|8.15|0.00|116.55|124.70|-291.90| +2451920|32404|51|7734|1605090|2093|9096|2|165|669|39|57.93|78.78|71.68|0.00|2795.52|2259.27|3072.42|139.77|0.00|2795.52|2935.29|536.25| +2451920|32404|13799|7734|1605090|2093|9096|2|201|669|89|44.25|56.64|34.55|0.00|3074.95|3938.25|5040.96|122.99|0.00|3074.95|3197.94|-863.30| +2451920|32404|12123|7734|1605090|2093|9096|2|202|669|40|73.29|75.48|57.36|0.00|2294.40|2931.60|3019.20|206.49|0.00|2294.40|2500.89|-637.20| +2451920|32404|17623|7734|1605090|2093|9096|2|282|669|11|41.57|53.62|27.34|0.00|300.74|457.27|589.82|12.02|0.00|300.74|312.76|-156.53| +2451920|32404|8151|7734|1605090|2093|9096|2|13|669|13|73.79|109.94|84.65|0.00|1100.45|959.27|1429.22|55.02|0.00|1100.45|1155.47|141.18| +2451920|32404|7643|7734|1605090|2093|9096|2|234|669|7|26.15|30.07|25.86|0.00|181.02|183.05|210.49|0.00|0.00|181.02|181.02|-2.03| +2451317|59298|2809|52213|1551093|6420|36100|2|134|670|78|62.82|114.33|61.73|0.00|4814.94|4899.96|8917.74|0.00|0.00|4814.94|4814.94|-85.02| +2451317|59298|11570|52213|1551093|6420|36100|2|279|670|98|85.57|154.02|23.10|0.00|2263.80|8385.86|15093.96|158.46|0.00|2263.80|2422.26|-6122.06| +2451317|59298|16832|52213|1551093|6420|36100|2|42|670|98|42.39|83.93|58.75|1842.40|5757.50|4154.22|8225.14|352.35|1842.40|3915.10|4267.45|-239.12| +2451317|59298|17414|52213|1551093|6420|36100|2|97|670|40|29.37|49.04|0.00|0.00|0.00|1174.80|1961.60|0.00|0.00|0.00|0.00|-1174.80| +2451317|59298|2947|52213|1551093|6420|36100|2|69|670|17|45.14|54.61|16.92|0.00|287.64|767.38|928.37|25.88|0.00|287.64|313.52|-479.74| +|59298|15836|52213||6420||||670||||73.38|0.00|4109.28|||287.64|0.00|||-669.20| +2451317|59298|4579|52213|1551093|6420|36100|2|20|670|56|37.39|56.08|49.35|0.00|2763.60|2093.84|3140.48|165.81|0.00|2763.60|2929.41|669.76| +2451317|59298|5918|52213|1551093|6420|36100|2|18|670|25|31.68|33.89|5.08|0.00|127.00|792.00|847.25|7.62|0.00|127.00|134.62|-665.00| +2451317|59298|6229|52213|1551093|6420|36100|2|147|670|26|4.71|5.98|3.76|0.00|97.76|122.46|155.48|6.84|0.00|97.76|104.60|-24.70| +2452217|43193|6283|59598|1282456|4514|37325|4|47|671|87|42.83|85.23|52.84|0.00|4597.08|3726.21|7415.01|183.88|0.00|4597.08|4780.96|870.87| +||11253||1282456||37325|4|207|671|80|||13.40||||4874.40|||||| +2452217|43193|4955|59598|1282456|4514|37325|4|52|671|4|94.38|168.94|20.27|0.00|81.08|377.52|675.76|2.43|0.00|81.08|83.51|-296.44| +2452217|43193|13849|59598|1282456|4514|37325|4|124|671|44|58.40|65.99|38.93|0.00|1712.92|2569.60|2903.56|119.90|0.00|1712.92|1832.82|-856.68| +2452217|43193|11093|59598|1282456|4514|37325|4|262|671|18|35.49|42.23|40.11|0.00|721.98|638.82|760.14|0.00|0.00|721.98|721.98|83.16| +2452217|43193|6865|59598|1282456|4514|37325|4|136|671|19|38.14|42.33|11.85|0.00|225.15|724.66|804.27|20.26|0.00|225.15|245.41|-499.51| +2452217|43193|1647|59598|1282456|4514|37325|4|180|671|60|81.59|152.57|134.26|0.00|8055.60|4895.40|9154.20|161.11|0.00|8055.60|8216.71|3160.20| +2452217|43193|5825|59598|1282456|4514|37325|4|17|671|50|48.90|49.38|11.35|0.00|567.50|2445.00|2469.00|34.05|0.00|567.50|601.55|-1877.50| +2452217|43193|11171|59598|1282456|4514|37325|4|252|671|81|89.80|165.23|117.31|0.00|9502.11|7273.80|13383.63|0.00|0.00|9502.11|9502.11|2228.31| +2452217|43193|3891|59598|1282456|4514|37325|4|221|671|90|47.92|52.71|34.78|0.00|3130.20|4312.80|4743.90|250.41|0.00|3130.20|3380.61|-1182.60| +2452217|43193|4509|59598|1282456|4514|37325|4|190|671|88|24.49|27.42|14.25|438.90|1254.00|2155.12|2412.96|40.75|438.90|815.10|855.85|-1340.02| +2452217||17503|||||4||671|49|14.60|27.44|||94.08||1344.56|0.00||||-621.32| +2452217|43193|10689|59598|1282456|4514|37325|4|280|671|47|74.82|125.69|106.83|0.00|5021.01|3516.54|5907.43|301.26|0.00|5021.01|5322.27|1504.47| +2452217|43193|7235|59598|1282456|4514|37325|4|227|671|74|32.89|51.30|38.47|0.00|2846.78|2433.86|3796.20|56.93|0.00|2846.78|2903.71|412.92| +2451797|63860|17798|40218|42923|28|7086|4|161|672|90|60.07|86.50|51.90|0.00|4671.00|5406.30|7785.00|326.97|0.00|4671.00|4997.97|-735.30| +2451797|63860|2828|40218|42923|28|7086|4|148|672|81|43.00|58.05|2.32|46.98|187.92|3483.00|4702.05|8.45|46.98|140.94|149.39|-3342.06| +2451797|63860|5971|40218|42923|28|7086|4|45|672|7|16.71|18.88|17.74|0.00|124.18|116.97|132.16|0.00|0.00|124.18|124.18|7.21| +2451797|63860|6683|40218|42923|28|7086|4|48|672|41|3.31|6.45|1.41|0.00|57.81|135.71|264.45|2.89|0.00|57.81|60.70|-77.90| +2451797|63860|4430|40218|42923|28|7086|4|257|672|2|28.63|38.36|34.90|69.10|69.80|57.26|76.72|0.02|69.10|0.70|0.72|-56.56| +2451797|63860|6085|40218|42923|28|7086|4|95|672|31|42.40|48.33|37.21|0.00|1153.51|1314.40|1498.23|0.00|0.00|1153.51|1153.51|-160.89| +2451797|63860|1760|40218|42923|28|7086|4|201|672|89|17.83|30.84|9.25|0.00|823.25|1586.87|2744.76|16.46|0.00|823.25|839.71|-763.62| +2451797|63860|10097|40218|42923|28|7086|4|227|672|37|35.01|57.41|45.92|0.00|1699.04|1295.37|2124.17|118.93|0.00|1699.04|1817.97|403.67| +2451797|63860|1898|40218|42923|28|7086|4|17|672|65|13.76|18.57|9.28|0.00|603.20|894.40|1207.05|0.00|0.00|603.20|603.20|-291.20| +2451797|63860|7640|40218|42923|28|7086|4|261|672|7|20.60|32.34|16.49|0.00|115.43|144.20|226.38|1.15|0.00|115.43|116.58|-28.77| +2452242|39885|5521|41683|961256|29|35555|10|218|673|60|48.99|73.97|49.55|2705.43|2973.00|2939.40|4438.20|18.72|2705.43|267.57|286.29|-2671.83| +2452242|39885|4281|41683|961256|29|35555|10|240|673|22|47.93|75.72|7.57|9.99|166.54|1054.46|1665.84|3.13|9.99|156.55|159.68|-897.91| +2452242|39885|1907|41683|961256|29|35555|10|216|673|87|32.03|38.75|21.31|0.00|1853.97|2786.61|3371.25|111.23|0.00|1853.97|1965.20|-932.64| +2452242|39885|15759|41683|961256|29|35555|10|218|673|85|76.04|106.45|72.38|0.00|6152.30|6463.40|9048.25|430.66|0.00|6152.30|6582.96|-311.10| +2452242|39885|11289|41683|961256|29|35555|10|5|673|4|47.49|52.23|39.17|0.00|156.68|189.96|208.92|0.00|0.00|156.68|156.68|-33.28| +2452242|39885|473|41683|961256|29|35555|10|220|673|2|14.58|24.64|10.10|0.00|20.20|29.16|49.28|0.60|0.00|20.20|20.80|-8.96| +2452242||13607|||29||10||673|||52.51||0.00|2744.04||4095.78||0.00|2744.04||-382.98| +2452242|39885|3625|41683|961256|29|35555|10|50|673|31|94.88|135.67|132.95|3462.01|4121.45|2941.28|4205.77|6.59|3462.01|659.44|666.03|-2281.84| +2452242|39885|3967|41683|961256|29|35555|10|206|673|80|66.10|126.25|49.23|0.00|3938.40|5288.00|10100.00|157.53|0.00|3938.40|4095.93|-1349.60| +2452242|39885|8897|41683|961256|29|35555|10|5|673|26|31.29|41.30|20.65|0.00|536.90|813.54|1073.80|32.21|0.00|536.90|569.11|-276.64| +2452242|39885|9209|41683|961256|29|35555|10|298|673|10|44.40|77.70|66.04|0.00|660.40|444.00|777.00|19.81|0.00|660.40|680.21|216.40| +2451084|47115|1532|91437|434554|5318|3635|10|146|674|49|85.71|137.13|122.04|0.00|5979.96|4199.79|6719.37|0.00|0.00|5979.96|5979.96|1780.17| +2451084|47115|14245|91437|434554|5318|3635|10|99|674|15|2.12|4.15|0.91|0.00|13.65|31.80|62.25|0.54|0.00|13.65|14.19|-18.15| +2451084|47115|7957|91437|434554|5318|3635|10|47|674|5|76.85|140.63|85.78|0.00|428.90|384.25|703.15|21.44|0.00|428.90|450.34|44.65| +2451084|47115|8959|91437|434554|5318|3635|10|13|674|79|68.94|68.94|62.73|0.00|4955.67|5446.26|5446.26|346.89|0.00|4955.67|5302.56|-490.59| +2451084|47115|11236|91437|434554|5318|3635|10|135|674|55|63.61|110.04|57.22|0.00|3147.10|3498.55|6052.20|0.00|0.00|3147.10|3147.10|-351.45| +2451084|47115|6736|91437|434554|5318|3635|10|116|674|86|21.78|35.71|27.85|0.00|2395.10|1873.08|3071.06|47.90|0.00|2395.10|2443.00|522.02| +2451084|47115|5359|91437|434554|5318|3635|10|70|674|100|86.38|101.06|73.77|4131.12|7377.00|8638.00|10106.00|0.00|4131.12|3245.88|3245.88|-5392.12| +2451084|47115|2203|91437|434554|5318|3635|10|266|674|30|88.04|124.13|35.99|0.00|1079.70|2641.20|3723.90|43.18|0.00|1079.70|1122.88|-1561.50| +2451084|47115|2168|91437|434554|5318|3635|10|300|674|79|86.28|150.12|132.10|0.00|10435.90|6816.12|11859.48|730.51|0.00|10435.90|11166.41|3619.78| +2451084|47115|17030|91437|434554|5318|3635|10|65|674|47|97.35|130.44|70.43|0.00|3310.21|4575.45|6130.68|165.51|0.00|3310.21|3475.72|-1265.24| +2451084|47115|10304|91437|434554|5318|3635|10|70|674|42|38.39|66.79|65.45|0.00|2748.90|1612.38|2805.18|219.91|0.00|2748.90|2968.81|1136.52| +2451084|47115|433|91437|434554|5318|3635|10|218|674|1|66.06|108.99|92.64|13.89|92.64|66.06|108.99|5.51|13.89|78.75|84.26|12.69| +2451084|47115|9728|91437|434554|5318|3635|10|91|674|19|24.82|48.15|13.96|0.00|265.24|471.58|914.85|18.56|0.00|265.24|283.80|-206.34| +2451084|47115|2029|91437|434554|5318|3635|10|8|674|17|24.53|37.77|29.08|0.00|494.36|417.01|642.09|39.54|0.00|494.36|533.90|77.35| +2451109|73415|16748|89850|894891|2332|5773|2|223|675|13|62.35|72.32|49.17|0.00|639.21|810.55|940.16|25.56|0.00|639.21|664.77|-171.34| +2451109|73415|9787|89850|894891|2332|5773|2|272|675|85|95.14|124.63|22.43|1372.71|1906.55|8086.90|10593.55|37.36|1372.71|533.84|571.20|-7553.06| +2451109|73415|11452|89850|894891|2332|5773|2|71|675|1|93.44|167.25|112.05|0.00|112.05|93.44|167.25|8.96|0.00|112.05|121.01|18.61| +2451109|73415|14152|89850|894891|2332|5773|2|244|675|17|33.48|58.25|39.61|0.00|673.37|569.16|990.25|53.86|0.00|673.37|727.23|104.21| +2451109|73415|12424|89850|894891|2332|5773|2|240|675|70|44.89|61.49|3.07|0.00|214.90|3142.30|4304.30|2.14|0.00|214.90|217.04|-2927.40| +2451109|73415|188|89850|894891|2332|5773|2|180|675|59|70.01|128.11|52.52|0.00|3098.68|4130.59|7558.49|123.94|0.00|3098.68|3222.62|-1031.91| +2451109|73415|10687|89850|894891|2332|5773|2|164|675|57|94.48|112.43|82.07|0.00|4677.99|5385.36|6408.51|93.55|0.00|4677.99|4771.54|-707.37| +2451109|73415|14083|89850|894891|2332|5773|2|206|675|9|44.29|59.79|40.05|0.00|360.45|398.61|538.11|18.02|0.00|360.45|378.47|-38.16| +2451109|73415|668|89850|894891|2332|5773|2|55|675|92|93.66|121.75|53.57|0.00|4928.44|8616.72|11201.00|197.13|0.00|4928.44|5125.57|-3688.28| +2451109|73415|212|89850|894891|2332|5773|2|74|675|87|89.05|166.52|108.23|0.00|9416.01|7747.35|14487.24|376.64|0.00|9416.01|9792.65|1668.66| +2451109|73415|2708|89850|894891|2332|5773|2|7|675|44|1.91|1.96|0.99|0.00|43.56|84.04|86.24|0.00|0.00|43.56|43.56|-40.48| +2451109|73415|10156|89850|894891|2332|5773|2|72|675|45|10.93|17.26|12.08|0.00|543.60|491.85|776.70|0.00|0.00|543.60|543.60|51.75| +2451109|73415|9706|89850|894891|2332|5773|2|19|675|30|3.48|5.95|5.65|0.00|169.50|104.40|178.50|8.47|0.00|169.50|177.97|65.10| +2451886|37936|11963|46710|662464|1065|32019|8|25|676|34|74.99|138.73|131.79|0.00|4480.86|2549.66|4716.82|358.46|0.00|4480.86|4839.32|1931.20| +2451886|37936|9725|46710|662464|1065|32019|8|19|676|58|85.86|140.81|0.00|0.00|0.00|4979.88|8166.98|0.00|0.00|0.00|0.00|-4979.88| +2451886|37936|8846|46710|662464|1065|32019|8|232|676|24|4.37|8.34|7.08|0.00|169.92|104.88|200.16|15.29|0.00|169.92|185.21|65.04| +2451886|37936|10435|46710|662464|1065|32019|8|45|676|41|31.66|39.89|5.98|0.00|245.18|1298.06|1635.49|12.25|0.00|245.18|257.43|-1052.88| +2451886|37936|15650|46710|662464|1065|32019|8|4|676|65|64.60|106.59|33.04|0.00|2147.60|4199.00|6928.35|171.80|0.00|2147.60|2319.40|-2051.40| +2451886|37936|17999|46710|662464|1065|32019|8|78|676|41|1.07|1.88|0.31|9.27|12.71|43.87|77.08|0.24|9.27|3.44|3.68|-40.43| +2451886|37936|3614|46710|662464|1065|32019|8|10|676|34|13.29|24.05|9.13|18.62|310.42|451.86|817.70|5.83|18.62|291.80|297.63|-160.06| +|37936|9266|||1065|32019||8|676|60|99.14||46.49||2789.40|5948.40||||2789.40|2928.87|-3159.00| +||15943|46710||1065||8||676|32|86.19||32.31|0.00|1033.92|||82.71|0.00||1116.63|| +2451886|37936|14189|46710|662464|1065|32019|8|72|676|83|77.15|83.32|64.98|0.00|5393.34|6403.45|6915.56|377.53|0.00|5393.34|5770.87|-1010.11| +2451886|37936|11651|46710|662464|1065|32019|8|6|676|56|32.06|54.82|15.34|0.00|859.04|1795.36|3069.92|77.31|0.00|859.04|936.35|-936.32| +2451886|37936|3440|46710|662464|1065|32019|8|149|676|52|94.04|154.22|50.89|0.00|2646.28|4890.08|8019.44|0.00|0.00|2646.28|2646.28|-2243.80| +2451870|65735|2318|13251|187622|3240|33258|2|82|677|24|70.77|119.60|114.81|0.00|2755.44|1698.48|2870.40|0.00|0.00|2755.44|2755.44|1056.96| +2451870|65735|1328|13251|187622|3240|33258|2|292|677|2|9.54|9.73|8.27|0.00|16.54|19.08|19.46|0.49|0.00|16.54|17.03|-2.54| +2451870|65735|7877|13251|187622|3240|33258|2|15|677|24|53.11|92.41|62.83|0.00|1507.92|1274.64|2217.84|45.23|0.00|1507.92|1553.15|233.28| +2451870|65735|7159|13251|187622|3240|33258|2|45|677|76|15.22|22.83|10.95|0.00|832.20|1156.72|1735.08|74.89|0.00|832.20|907.09|-324.52| +2451870|65735|14012|13251|187622|3240|33258|2|268|677|46|56.54|79.15|4.74|0.00|218.04|2600.84|3640.90|6.54|0.00|218.04|224.58|-2382.80| +2451870|65735|13507|13251|187622|3240|33258|2|43|677|40|48.70|64.28|43.71|0.00|1748.40|1948.00|2571.20|139.87|0.00|1748.40|1888.27|-199.60| +2451870|65735|7016|13251|187622|3240|33258|2|52|677|95|17.26|21.57|0.64|0.00|60.80|1639.70|2049.15|3.04|0.00|60.80|63.84|-1578.90| +2451870|65735|2389|13251|187622|3240|33258|2|90|677|25|3.73|4.06|1.21|0.00|30.25|93.25|101.50|1.51|0.00|30.25|31.76|-63.00| +2451870|65735|1477|13251|187622|3240|33258|2|96|677|99|80.40|129.44|34.94|0.00|3459.06|7959.60|12814.56|69.18|0.00|3459.06|3528.24|-4500.54| +2451870|65735|11432|13251|187622|3240|33258|2|9|677|92|94.24|148.89|38.71|0.00|3561.32|8670.08|13697.88|320.51|0.00|3561.32|3881.83|-5108.76| +2451870|65735|10567|13251|187622|3240|33258|2|257|677|79|55.78|84.22|77.48|0.00|6120.92|4406.62|6653.38|183.62|0.00|6120.92|6304.54|1714.30| +2451870|65735|6740|13251|187622|3240|33258|2|69|677|66|90.21|165.08|66.03|1263.81|4357.98|5953.86|10895.28|0.00|1263.81|3094.17|3094.17|-2859.69| +2451870|65735|13481|13251|187622|3240|33258|2|153|677|31|34.43|42.34|8.46|0.00|262.26|1067.33|1312.54|23.60|0.00|262.26|285.86|-805.07| +2451870|65735|2845|13251|187622|3240|33258|2|142|677|87|8.38|15.92|5.25|0.00|456.75|729.06|1385.04|31.97|0.00|456.75|488.72|-272.31| +2451870|65735|3401|13251|187622|3240|33258|2|42|677|43|49.00|90.16|7.21|279.02|310.03|2107.00|3876.88|0.00|279.02|31.01|31.01|-2075.99| +2451870|65735|7025|13251|187622|3240|33258|2|54|677|88|33.36|65.05|37.72|531.09|3319.36|2935.68|5724.40|250.94|531.09|2788.27|3039.21|-147.41| +2452095|47882|9151|49907|1861152|1644|6136|8|300|678|20|48.52|88.30|30.02|0.00|600.40|970.40|1766.00|6.00|0.00|600.40|606.40|-370.00| +2452095|47882|91|49907|1861152|1644|6136|8|263|678|54|30.63|52.07|8.33|0.00|449.82|1654.02|2811.78|26.98|0.00|449.82|476.80|-1204.20| +2452095|47882|10773|49907|1861152|1644|6136|8|69|678|61|43.68|46.73|4.20|0.00|256.20|2664.48|2850.53|7.68|0.00|256.20|263.88|-2408.28| +2452095|47882|2387|49907|1861152|1644|6136|8|291|678|73|3.70|4.69|0.42|0.00|30.66|270.10|342.37|0.30|0.00|30.66|30.96|-239.44| +||15275||||6136||134|678|76||153.02|||10582.24|5903.68|11629.52|317.46||10582.24||| +2452095|47882|10285|49907|1861152|1644|6136|8|192|678|58|98.22|158.13|94.87|0.00|5502.46|5696.76|9171.54|440.19|0.00|5502.46|5942.65|-194.30| +2452095|47882|16723|49907|1861152|1644|6136|8|82|678|48|38.58|76.77|61.41|795.87|2947.68|1851.84|3684.96|129.10|795.87|2151.81|2280.91|299.97| +2452095|47882|11995|49907|1861152|1644|6136|8|133|678|21|92.18|117.99|95.57|0.00|2006.97|1935.78|2477.79|180.62|0.00|2006.97|2187.59|71.19| +2452095|47882|605|49907|1861152|1644|6136|8|190|678|99|54.81|74.54|64.84|192.57|6419.16|5426.19|7379.46|311.32|192.57|6226.59|6537.91|800.40| +2452095|47882|10691|49907|1861152|1644|6136|8|101|678|82|79.21|114.85|84.98|0.00|6968.36|6495.22|9417.70|209.05|0.00|6968.36|7177.41|473.14| +2452095|47882|565|49907|1861152|1644|6136|8|222|678|69|5.05|6.91|6.56|0.00|452.64|348.45|476.79|18.10|0.00|452.64|470.74|104.19| +2452095|47882|2431|49907|1861152|1644|6136|8|268|678|52|42.54|61.68|42.55|0.00|2212.60|2212.08|3207.36|0.00|0.00|2212.60|2212.60|0.52| +2450917|33370|8990|46858|1015881|||1||679|75|||19.49||||12183.75|||1461.75||-5790.75| +2450917|33370|1555|46858|1015881|1351|16374|1|294|679|6|75.01|113.26|98.53|0.00|591.18|450.06|679.56|53.20|0.00|591.18|644.38|141.12| +2450917|33370|1504|46858|1015881|1351|16374|1|222|679|46|58.94|110.21|47.39|850.17|2179.94|2711.24|5069.66|79.78|850.17|1329.77|1409.55|-1381.47| +2450917|33370|10072|46858|1015881|1351|16374|1|53|679|85|9.55|18.81|11.09|0.00|942.65|811.75|1598.85|84.83|0.00|942.65|1027.48|130.90| +2450917|33370|14236|46858|1015881|1351|16374|1|297|679|31|37.27|47.33|2.83|0.00|87.73|1155.37|1467.23|5.26|0.00|87.73|92.99|-1067.64| +2450917|33370|2293|46858|1015881|1351|16374|1|274|679|44|27.50|40.42|21.42|0.00|942.48|1210.00|1778.48|47.12|0.00|942.48|989.60|-267.52| +2450917|33370|4286|46858|1015881|1351|16374|1|271|679|94|96.60|188.37|7.53|0.00|707.82|9080.40|17706.78|56.62|0.00|707.82|764.44|-8372.58| +2450917||12752|||1351|||74|679|11||128.17||0.00||819.72||5.07|0.00|84.59|89.66|-735.13| +2451527|65907|15802|54819|1780956|3446|12736|4|71|680|93|77.99|121.66|91.24|6109.43|8485.32|7253.07|11314.38|71.27|6109.43|2375.89|2447.16|-4877.18| +2451527|65907|16490|54819|1780956|3446|12736|4|113|680|41|48.21|66.04|3.30|0.00|135.30|1976.61|2707.64|5.41|0.00|135.30|140.71|-1841.31| +2451527|65907|13330|54819|1780956|3446|12736|4|219|680|90|83.82|167.64|135.78|0.00|12220.20|7543.80|15087.60|488.80|0.00|12220.20|12709.00|4676.40| +2451527|65907|16909|54819|1780956|3446|12736|4|208|680|41|61.17|82.57|41.28|0.00|1692.48|2507.97|3385.37|152.32|0.00|1692.48|1844.80|-815.49| +2451527|65907|11707|54819|1780956|3446|12736|4|83|680|5|99.71|165.51|21.51|0.00|107.55|498.55|827.55|4.30|0.00|107.55|111.85|-391.00| +2451527||13448|54819|||12736|4||680|||46.64|43.37|||||||173.48||| +2451527|65907|16837|54819|1780956|3446|12736|4|42|680|77|51.98|68.61|24.01|0.00|1848.77|4002.46|5282.97|0.00|0.00|1848.77|1848.77|-2153.69| +2451527|65907|7303|54819|1780956|3446|12736|4|10|680|90|24.70|26.42|19.28|0.00|1735.20|2223.00|2377.80|86.76|0.00|1735.20|1821.96|-487.80| +2451527|65907|8449|54819|1780956|3446|12736|4|222|680|49|55.61|95.64|25.82|0.00|1265.18|2724.89|4686.36|50.60|0.00|1265.18|1315.78|-1459.71| +2451527|65907|16429|54819|1780956|3446|12736|4|106|680|97|45.20|61.02|15.86|0.00|1538.42|4384.40|5918.94|0.00|0.00|1538.42|1538.42|-2845.98| +2451871|47623|17300|46521|1047630|4739|13848|1|67|681|19|74.64|146.29|136.04|0.00|2584.76|1418.16|2779.51|155.08|0.00|2584.76|2739.84|1166.60| +2451871|47623|9593|46521|1047630|4739|13848|1|233|681|46|33.62|42.36|21.60|0.00|993.60|1546.52|1948.56|19.87|0.00|993.60|1013.47|-552.92| +2451871|47623|938|46521|1047630|4739|13848|1|240|681|34|2.86|4.11|1.31|0.00|44.54|97.24|139.74|0.89|0.00|44.54|45.43|-52.70| +2451871|47623|205|46521|1047630|4739|13848|1|287|681|99|2.85|3.59|0.00|0.00|0.00|282.15|355.41|0.00|0.00|0.00|0.00|-282.15| +2451871|47623|12194|46521|1047630|4739|13848|1|96|681|97|67.52|95.87|41.22|0.00|3998.34|6549.44|9299.39|199.91|0.00|3998.34|4198.25|-2551.10| +2451871|47623|6419|46521|1047630|4739|13848|1|73|681|78|83.21|157.26|61.33|0.00|4783.74|6490.38|12266.28|191.34|0.00|4783.74|4975.08|-1706.64| +2451871||12254|46521|1047630|4739||||681||98.70|124.36|43.52|0.00|1436.16|3257.10||57.44|0.00||1493.60|| +2451871|47623|593|46521|1047630|4739|13848|1|29|681|47|22.44|40.16|4.41|0.00|207.27|1054.68|1887.52|2.07|0.00|207.27|209.34|-847.41| +2451616|63046|10079|11650|1506212|5221|37383|10|94|682|94|67.35|94.96|81.66|3530.97|7676.04|6330.90|8926.24|290.15|3530.97|4145.07|4435.22|-2185.83| +2451616|63046|4927|11650|1506212|5221|37383|10|33|682|85|17.78|22.75|2.04|0.00|173.40|1511.30|1933.75|12.13|0.00|173.40|185.53|-1337.90| +2451616|63046|14875|11650|1506212|5221|37383|10|135|682|57|54.86|59.79|39.46|1776.88|2249.22|3127.02|3408.03|28.34|1776.88|472.34|500.68|-2654.68| +2451616|63046|14186|11650|1506212|5221|37383|10|241|682|55|70.42|114.08|87.84|0.00|4831.20|3873.10|6274.40|144.93|0.00|4831.20|4976.13|958.10| +2451616|63046|8726|11650|1506212|5221|37383|10|267|682|71|9.21|15.01|2.10|0.00|149.10|653.91|1065.71|5.96|0.00|149.10|155.06|-504.81| +2451616|63046|607|11650|1506212|5221|37383|10|48|682|48|86.45|168.57|74.17|0.00|3560.16|4149.60|8091.36|142.40|0.00|3560.16|3702.56|-589.44| +2451616|63046|4625|11650|1506212|5221|37383|10|75|682|95|30.91|33.38|20.69|0.00|1965.55|2936.45|3171.10|58.96|0.00|1965.55|2024.51|-970.90| +2451616|63046|10217|11650|1506212|5221|37383|10|192|682|99|38.47|55.01|33.00|0.00|3267.00|3808.53|5445.99|98.01|0.00|3267.00|3365.01|-541.53| +2451616|63046|2948|11650|1506212|5221|37383|10|43|682|97|28.09|53.65|28.43|0.00|2757.71|2724.73|5204.05|27.57|0.00|2757.71|2785.28|32.98| +2451616|63046|6938|11650|1506212|5221|37383|10|224|682|47|71.37|99.20|11.90|0.00|559.30|3354.39|4662.40|22.37|0.00|559.30|581.67|-2795.09| +2451616|63046|9905|11650|1506212|5221|37383|10|127|682|60|98.89|150.31|106.72|0.00|6403.20|5933.40|9018.60|576.28|0.00|6403.20|6979.48|469.80| +2451616|63046|6953|11650|1506212|5221|37383|10|163|682|65|37.77|38.52|17.71|0.00|1151.15|2455.05|2503.80|57.55|0.00|1151.15|1208.70|-1303.90| +2451616|63046|11039|11650|1506212|5221|37383|10|296|682|53|93.91|171.85|103.11|0.00|5464.83|4977.23|9108.05|491.83|0.00|5464.83|5956.66|487.60| +2451616|63046|16796|11650|1506212|5221|37383|10|8|682|66|28.24|49.70|34.29|0.00|2263.14|1863.84|3280.20|90.52|0.00|2263.14|2353.66|399.30| +2451616|63046|1550|11650|1506212|5221|37383|10|94|682|31|71.66|80.97|22.67|0.00|702.77|2221.46|2510.07|21.08|0.00|702.77|723.85|-1518.69| +2450943|37582|1729|31370|656710|450|17390|4|210|683|50|83.27|122.40|111.38|0.00|5569.00|4163.50|6120.00|389.83|0.00|5569.00|5958.83|1405.50| +2450943|37582|15746|31370|656710|450|17390|4|168|683|76|13.98|19.15|5.74|0.00|436.24|1062.48|1455.40|34.89|0.00|436.24|471.13|-626.24| +2450943|37582|5246|31370|656710|450|17390|4|153|683|90|3.40|4.08|1.26|0.00|113.40|306.00|367.20|4.53|0.00|113.40|117.93|-192.60| +|37582|17596|31370||||4|101|683|87|36.32|70.46|||||6130.02|235.38||||| +2450943|37582|3824|31370|656710|450|17390|4|111|683|60|50.52|91.94|18.38|0.00|1102.80|3031.20|5516.40|22.05|0.00|1102.80|1124.85|-1928.40| +2450943|37582|2965|31370|656710|450|17390|4|275|683|43|91.39|112.40|67.44|318.99|2899.92|3929.77|4833.20|77.42|318.99|2580.93|2658.35|-1348.84| +2450943|37582|12286|31370|656710|450|17390|4|110|683|83|65.40|125.56|47.71|3286.74|3959.93|5428.20|10421.48|0.00|3286.74|673.19|673.19|-4755.01| +2450943|37582|5810|31370|656710|450|17390|4|261|683|57|88.12|134.82|1.34|0.00|76.38|5022.84|7684.74|0.00|0.00|76.38|76.38|-4946.46| +2450943|37582|3610|31370|656710|450|17390|4|112|683|77|91.55|142.81|92.82|0.00|7147.14|7049.35|10996.37|214.41|0.00|7147.14|7361.55|97.79| +2451753|70926|2099|56816|671892|6968|31644|1|272|684|38|53.83|81.82|60.54|0.00|2300.52|2045.54|3109.16|46.01|0.00|2300.52|2346.53|254.98| +2451753|70926|9275|56816|671892|6968|31644|1|117|684|43|7.25|10.65|0.00|0.00|0.00|311.75|457.95|0.00|0.00|0.00|0.00|-311.75| +2451753|70926|10289||671892||31644|1|181|684|17|62.08||12.06|0.00||||2.05|0.00||207.07|| +2451753|70926|5858|56816|671892|6968|31644|1|251|684|25|94.35|159.45|55.80|0.00|1395.00|2358.75|3986.25|125.55|0.00|1395.00|1520.55|-963.75| +2451753|70926|10109|56816|671892|6968|31644|1|229|684|87|19.22|36.32|28.69|0.00|2496.03|1672.14|3159.84|24.96|0.00|2496.03|2520.99|823.89| +2451753|70926|9997|56816|671892|6968|31644|1|97|684|14|72.29|127.23|7.63|0.00|106.82|1012.06|1781.22|6.40|0.00|106.82|113.22|-905.24| +2451753|70926|17246|56816|671892|6968|31644|1|149|684|45|74.95|143.15|57.26|0.00|2576.70|3372.75|6441.75|25.76|0.00|2576.70|2602.46|-796.05| +2451753|70926|9791|56816|671892|6968|31644|1|152|684|79|62.75|101.02|37.37|0.00|2952.23|4957.25|7980.58|0.00|0.00|2952.23|2952.23|-2005.02| +2451753|70926|4145|56816|671892|6968|31644|1|102|684|78|66.35|97.53|31.20|0.00|2433.60|5175.30|7607.34|146.01|0.00|2433.60|2579.61|-2741.70| +2451753|70926|8887|56816|671892|6968|31644|1|105|684|76|28.65|50.13|39.60|0.00|3009.60|2177.40|3809.88|240.76|0.00|3009.60|3250.36|832.20| +2451753|70926|6008|56816|671892|6968|31644|1|214|684|49|71.77|75.35|42.94|0.00|2104.06|3516.73|3692.15|168.32|0.00|2104.06|2272.38|-1412.67| +2451753|70926|457|56816|671892|6968|31644|1|260|684|39|55.68|64.03|17.92|0.00|698.88|2171.52|2497.17|34.94|0.00|698.88|733.82|-1472.64| +2451753|70926|11168|56816|671892|6968|31644|1|158|684|59|30.34|36.71|12.84|0.00|757.56|1790.06|2165.89|7.57|0.00|757.56|765.13|-1032.50| +2451753|70926|9769|56816|671892|6968|31644|1|283|684|46|69.38|81.86|40.93|0.00|1882.78|3191.48|3765.56|75.31|0.00|1882.78|1958.09|-1308.70| +2451753|70926|7505|56816|671892|6968|31644|1|56|684|43|92.22|180.75|133.75|0.00|5751.25|3965.46|7772.25|402.58|0.00|5751.25|6153.83|1785.79| +2451040|30776|16579|46585|843115|5692|48622|7|200|685|29|65.98|77.19|32.41|0.00|939.89|1913.42|2238.51|0.00|0.00|939.89|939.89|-973.53| +2451040|30776|6715|46585|843115|5692|48622|7|255|685|47|84.27|133.98|65.65|0.00|3085.55|3960.69|6297.06|123.42|0.00|3085.55|3208.97|-875.14| +2451040|30776|6175|46585|843115|5692|48622|7|19|685|77|90.20|178.59|82.15|0.00|6325.55|6945.40|13751.43|126.51|0.00|6325.55|6452.06|-619.85| +|30776|13786|46585|843115||48622|7|235|685||64.75||||||5180.00|0.00||1558.55|1558.55|| +2451040|30776|7946|46585|843115|5692|48622|7|121|685|33|26.24|45.65|30.58|0.00|1009.14|865.92|1506.45|90.82|0.00|1009.14|1099.96|143.22| +2451040|30776|3436|46585|843115|5692|48622|7|235|685|77|1.66|2.29|1.23|0.00|94.71|127.82|176.33|4.73|0.00|94.71|99.44|-33.11| +2451040|30776|16076|46585|843115|5692|48622|7|201|685|96|42.42|70.84|63.04|0.00|6051.84|4072.32|6800.64|302.59|0.00|6051.84|6354.43|1979.52| +2451040|30776|5833|46585|843115|5692|48622|7|257|685|3|20.86|30.87|3.39|0.00|10.17|62.58|92.61|0.30|0.00|10.17|10.47|-52.41| +2451040|30776|17188|46585|843115|5692|48622|7|246|685|46|1.29|2.32|0.48|0.00|22.08|59.34|106.72|0.88|0.00|22.08|22.96|-37.26| +2451040|30776|11383|46585|843115|5692|48622|7|51|685|56|82.15|85.43|79.44|0.00|4448.64|4600.40|4784.08|311.40|0.00|4448.64|4760.04|-151.76| +2451040|30776|9877|46585|843115|5692|48622|7|147|685|82|55.31|61.39|44.81|0.00|3674.42|4535.42|5033.98|330.69|0.00|3674.42|4005.11|-861.00| +2451040|30776|5210|46585|843115|5692|48622|7|5|685|19|86.72|88.45|83.14|0.00|1579.66|1647.68|1680.55|47.38|0.00|1579.66|1627.04|-68.02| +|50786|16027||36215||13424|10||686|||10.42|||11.55|||||11.55|11.89|-10.02| +2451459|50786|4273|11387|36215|2196|13424|10|214|686|45|56.57|61.09|44.59|0.00|2006.55|2545.65|2749.05|120.39|0.00|2006.55|2126.94|-539.10| +2451459|50786|14863|11387|36215|2196|13424|10|11|686|46|9.93|9.93|4.26|0.00|195.96|456.78|456.78|7.83|0.00|195.96|203.79|-260.82| +2451459|50786|9757|11387|36215|2196|13424|10|265|686|42|20.03|36.85|5.52|39.41|231.84|841.26|1547.70|1.92|39.41|192.43|194.35|-648.83| +2451459|50786|8875|11387|36215|2196|13424|10|188|686|40|23.96|32.58|29.64|0.00|1185.60|958.40|1303.20|71.13|0.00|1185.60|1256.73|227.20| +2451459|50786|10966|11387|36215|2196|13424|10|83|686|96|16.40|23.61|2.83|0.00|271.68|1574.40|2266.56|21.73|0.00|271.68|293.41|-1302.72| +|50786|4231|11387|36215|||10||686|59|80.87|118.87|28.52|0.00|1682.68||7013.33||0.00||1716.33|-3088.65| +2451459|50786|4958|11387|36215|2196|13424|10|223|686|35|55.68|75.72|57.54|0.00|2013.90|1948.80|2650.20|120.83|0.00|2013.90|2134.73|65.10| +2451459|50786|7346|11387|36215|2196|13424|10|166|686|58|62.19|74.00|5.92|0.00|343.36|3607.02|4292.00|3.43|0.00|343.36|346.79|-3263.66| +2451459|50786|15434|11387|36215|2196|13424|10|92|686|89|36.97|69.13|64.29|0.00|5721.81|3290.33|6152.57|228.87|0.00|5721.81|5950.68|2431.48| +2451459|50786|3034|11387|36215|2196|13424|10|33|686|88|73.54|77.95|67.03|0.00|5898.64|6471.52|6859.60|471.89|0.00|5898.64|6370.53|-572.88| +2451459|50786|16214|11387|36215|2196|13424|10|290|686|47|70.92|83.68|76.14|0.00|3578.58|3333.24|3932.96|214.71|0.00|3578.58|3793.29|245.34| +2451926|44614|15337|16379|432009|6839|8469|7|247|687|16|7.32|13.02|0.00|0.00|0.00|117.12|208.32|0.00|0.00|0.00|0.00|-117.12| +2451926|44614|17125|16379|432009|6839|8469|7|2|687|76|80.48|90.94|0.00|0.00|0.00|6116.48|6911.44|0.00|0.00|0.00|0.00|-6116.48| +2451926|44614|14781|16379|432009|6839|8469|7|140|687|93|18.39|28.50|23.08|0.00|2146.44|1710.27|2650.50|107.32|0.00|2146.44|2253.76|436.17| +2451926|44614|17449|16379|432009|6839|8469|7|30|687|56|2.98|3.87|2.63|132.55|147.28|166.88|216.72|1.03|132.55|14.73|15.76|-152.15| +2451926|44614|12765|16379|432009|6839|8469|7|169|687|76|76.64|93.50|19.63|119.35|1491.88|5824.64|7106.00|82.35|119.35|1372.53|1454.88|-4452.11| +2451926|44614|5937|16379|432009|6839|8469|7|279|687|81|59.27|109.05|99.23|0.00|8037.63|4800.87|8833.05|241.12|0.00|8037.63|8278.75|3236.76| +2451926|44614|14263|16379|432009|6839|8469|7|297|687|94|57.81|74.57|61.14|0.00|5747.16|5434.14|7009.58|114.94|0.00|5747.16|5862.10|313.02| +2451926|44614|743|16379|432009|6839|8469|7|164|687|17|50.11|99.71|95.72|0.00|1627.24|851.87|1695.07|48.81|0.00|1627.24|1676.05|775.37| +2451926|44614|12361|16379|432009|6839|8469|7|244|687|53|41.19|69.19|68.49|0.00|3629.97|2183.07|3667.07|217.79|0.00|3629.97|3847.76|1446.90| +2451926|44614|7367|16379|432009|6839|8469|7|39|687|20|5.74|7.28|0.65|3.51|13.00|114.80|145.60|0.56|3.51|9.49|10.05|-105.31| +2451926|44614|15775|16379|432009|6839|8469|7|226|687|10|45.97|88.26|45.01|0.00|450.10|459.70|882.60|31.50|0.00|450.10|481.60|-9.60| +2451926|44614|4381|16379|432009|6839|8469|7|157|687|56|12.03|14.07|9.00|0.00|504.00|673.68|787.92|10.08|0.00|504.00|514.08|-169.68| +|49546|211||382643|2177|32511|1|266|688|56|59.22||46.13|0.00|2583.28||4874.80||0.00|||-733.04| +2452632|49546|10908|66309|382643|2177|32511|1|107|688|87|29.11|30.56|2.13|0.00|185.31|2532.57|2658.72|11.11|0.00|185.31|196.42|-2347.26| +2452632|49546|475|66309|382643|2177|32511|1|198|688|66|42.95|60.98|23.17|0.00|1529.22|2834.70|4024.68|107.04|0.00|1529.22|1636.26|-1305.48| +2452632|49546|12475|66309|382643|2177|32511|1|107|688|64|77.29|108.20|57.34|0.00|3669.76|4946.56|6924.80|220.18|0.00|3669.76|3889.94|-1276.80| +2452632|49546|16557|66309|382643|2177|32511|1|195|688|51|33.33|55.32|11.06|0.00|564.06|1699.83|2821.32|50.76|0.00|564.06|614.82|-1135.77| +2452632|49546|12919|66309|382643|2177|32511|1|151|688|51|65.97|104.23|71.91|0.00|3667.41|3364.47|5315.73|183.37|0.00|3667.41|3850.78|302.94| +2452632|49546|13735|66309|382643|2177|32511|1|268|688|2|70.74|132.28|116.40|0.00|232.80|141.48|264.56|9.31|0.00|232.80|242.11|91.32| +2452632|49546|2802|66309|382643|2177|32511|1|64|688|43|80.63|90.30|69.53|0.00|2989.79|3467.09|3882.90|239.18|0.00|2989.79|3228.97|-477.30| +2452632|49546|14250|66309|382643|2177|32511|1|177|688|29|88.22|148.20|130.41|1361.48|3781.89|2558.38|4297.80|121.02|1361.48|2420.41|2541.43|-137.97| +2452632|49546|13506|66309|382643|2177|32511|1|52|688|38|17.82|27.26|17.99|0.00|683.62|677.16|1035.88|54.68|0.00|683.62|738.30|6.46| +2452632|49546|17856|66309|382643|2177|32511|1|244|688|9|28.00|38.08|36.93|0.00|332.37|252.00|342.72|19.94|0.00|332.37|352.31|80.37| +2451894|42763|9811|90229|10244|6612|16627|2|245|689|87|2.97|4.78|0.00|0.00|0.00|258.39|415.86|0.00|0.00|0.00|0.00|-258.39| +2451894|42763|3932|90229|10244|6612|16627|2|179|689|81|53.42|66.77|20.69|0.00|1675.89|4327.02|5408.37|117.31|0.00|1675.89|1793.20|-2651.13| +2451894|42763|5863|90229|10244|6612|16627|2|283|689|34|27.29|42.02|6.30|0.00|214.20|927.86|1428.68|10.71|0.00|214.20|224.91|-713.66| +2451894|42763|4357|90229|10244|6612|16627|2|240|689|53|19.28|22.75|11.83|0.00|626.99|1021.84|1205.75|37.61|0.00|626.99|664.60|-394.85| +2451894|42763|15446|90229|10244|6612|16627|2|270|689|95|31.92|36.38|20.73|0.00|1969.35|3032.40|3456.10|39.38|0.00|1969.35|2008.73|-1063.05| +2451894|42763|3433|90229|10244|6612|16627|2|207|689|25|25.29|38.44|7.68|0.00|192.00|632.25|961.00|9.60|0.00|192.00|201.60|-440.25| +2451894|42763|755|90229|10244|6612|16627|2|176|689|19|9.35|10.84|6.17|53.92|117.23|177.65|205.96|4.43|53.92|63.31|67.74|-114.34| +2451894|42763|1424|90229|10244|6612|16627|2|155|689|28|89.74|139.09|15.29|0.00|428.12|2512.72|3894.52|38.53|0.00|428.12|466.65|-2084.60| +2451894|42763|1511|90229|10244|6612|16627|2|180|689|43|54.55|98.73|29.61|0.00|1273.23|2345.65|4245.39|101.85|0.00|1273.23|1375.08|-1072.42| +2451894|42763|13802|90229|10244|6612|16627|2|130|689|67|62.12|114.30|77.72|0.00|5207.24|4162.04|7658.10|208.28|0.00|5207.24|5415.52|1045.20| +|42763|12569|||||2|39|689|58|8.29||4.85|0.00|281.30||879.86||0.00||281.30|-199.52| +2451894|42763|17072|90229|10244|6612|16627|2|259|689|10|12.24|17.99|7.01|0.00|70.10|122.40|179.90|0.00|0.00|70.10|70.10|-52.30| +2451894|42763|10733|90229|10244|6612|16627|2|144|689|3|62.96|72.40|39.09|0.00|117.27|188.88|217.20|3.51|0.00|117.27|120.78|-71.61| +2451894|42763|13457|90229|10244|6612|16627|2|22|689|11|46.96|59.16|44.37|0.00|488.07|516.56|650.76|24.40|0.00|488.07|512.47|-28.49| +2451894|42763|7448|90229|10244|6612|16627|2|89|689|30|49.36|60.21|12.64|0.00|379.20|1480.80|1806.30|15.16|0.00|379.20|394.36|-1101.60| +2451894|42763|2713|90229|10244|6612|16627|2|51|689|34|10.45|16.51|15.51|0.00|527.34|355.30|561.34|0.00|0.00|527.34|527.34|172.04| +2452236|57356|16917|71807|677111|6768|8072|7|290|690|19|56.20|87.11|20.90|0.00|397.10|1067.80|1655.09|19.85|0.00|397.10|416.95|-670.70| +2452236|57356|15763|71807|677111|6768|8072|7|221|690|62|84.69|140.58|119.49|0.00|7408.38|5250.78|8715.96|148.16|0.00|7408.38|7556.54|2157.60| +2452236|57356|16669|71807|677111|6768|8072|7|175|690|68|43.80|59.13|28.97|0.00|1969.96|2978.40|4020.84|59.09|0.00|1969.96|2029.05|-1008.44| +2452236|57356|1633|71807|677111|6768|8072|7|111|690|3|81.01|116.65|109.65|325.66|328.95|243.03|349.95|0.13|325.66|3.29|3.42|-239.74| +2452236|57356|8191|71807|677111|6768|8072|7|274|690|19|41.66|64.15|16.67|0.00|316.73|791.54|1218.85|22.17|0.00|316.73|338.90|-474.81| +2452236|57356|673|71807|677111|6768|8072|7|136|690|89|37.00|54.39|3.26|0.00|290.14|3293.00|4840.71|0.00|0.00|290.14|290.14|-3002.86| +||15861|71807||6768||||690|8|||47.52||380.16|405.44||||380.16||-25.28| +2452236|57356|16485|71807|677111|6768|8072|7|3|690|57|98.81|142.28|132.32|0.00|7542.24|5632.17|8109.96|678.80|0.00|7542.24|8221.04|1910.07| +2452236|57356|4621|71807|677111|6768|8072|7|24|690|28|40.22|50.27|37.19|0.00|1041.32|1126.16|1407.56|72.89|0.00|1041.32|1114.21|-84.84| +2452236|57356|1861|71807|677111|6768|8072|7|29|690|77|45.11|48.26|19.30|0.00|1486.10|3473.47|3716.02|29.72|0.00|1486.10|1515.82|-1987.37| +2452236|57356|9287|71807|677111|6768|8072|7|28|690|94|24.04|40.14|24.08|0.00|2263.52|2259.76|3773.16|203.71|0.00|2263.52|2467.23|3.76| +2452236|57356|2077|||||||690|40||98.46|0.00||||3938.40|||0.00|0.00|-2477.20| +2452236|57356|9641|71807|677111|6768|8072|7|27|690|24|13.00|14.43|4.32|0.00|103.68|312.00|346.32|3.11|0.00|103.68|106.79|-208.32| +2452236|57356|4853|71807|677111|6768|8072|7|297|690|97|36.40|72.80|59.69|0.00|5789.93|3530.80|7061.60|463.19|0.00|5789.93|6253.12|2259.13| +2451136|41808|9184|42188|1734879|4723|5367|8|54|691|40|62.33|103.46|32.07|0.00|1282.80|2493.20|4138.40|89.79|0.00|1282.80|1372.59|-1210.40| +2451136|41808|4294|42188|1734879|4723|5367|8|111|691|90|91.65|147.55|33.93|0.00|3053.70|8248.50|13279.50|0.00|0.00|3053.70|3053.70|-5194.80| +2451136|41808|16525|42188|1734879|4723|5367|8|42|691|71|89.53|160.25|153.84|0.00|10922.64|6356.63|11377.75|436.90|0.00|10922.64|11359.54|4566.01| +2451136|41808|2116|42188|1734879|4723|5367|8|251|691|53|16.53|31.07|0.93|0.00|49.29|876.09|1646.71|0.49|0.00|49.29|49.78|-826.80| +2451136|41808|11768|42188|1734879|4723|5367|8|81|691|84|66.85|112.97|7.90|0.00|663.60|5615.40|9489.48|13.27|0.00|663.60|676.87|-4951.80| +2451136|41808|3692|42188|1734879|4723|5367|8|206|691|56|76.76|129.72|27.24|0.00|1525.44|4298.56|7264.32|15.25|0.00|1525.44|1540.69|-2773.12| +2451136|41808|16729|42188|1734879|4723|5367|8|181|691|77|32.56|56.32|11.26|0.00|867.02|2507.12|4336.64|8.67|0.00|867.02|875.69|-1640.10| +2451136|41808|14707|42188|1734879|4723|5367|8|191|691|37|77.75|127.51|44.62|0.00|1650.94|2876.75|4717.87|99.05|0.00|1650.94|1749.99|-1225.81| +2451706|40009|11174|39985|1117552|2319|38239|7|252|692|87|41.99|48.70|20.94|0.00|1821.78|3653.13|4236.90|54.65|0.00|1821.78|1876.43|-1831.35| +2451706|40009|14210|39985|1117552|2319|38239|7|18|692|81|93.76|149.07|138.63|5726.80|11229.03|7594.56|12074.67|275.11|5726.80|5502.23|5777.34|-2092.33| +2451706|40009|9637|39985|1117552|2319|38239|7|104|692|93|93.92|108.94|101.31|0.00|9421.83|8734.56|10131.42|188.43|0.00|9421.83|9610.26|687.27| +||10706|39985|1117552|||7|123|692|76||149.23|20.89|0.00||7317.28|||0.00|1587.64||| +2451706|40009|9643|39985|1117552|2319|38239|7|13|692|57|25.67|48.00|38.88|0.00|2216.16|1463.19|2736.00|199.45|0.00|2216.16|2415.61|752.97| +2451706|40009|2543|39985|1117552|2319|38239|7|151|692|4|93.89|168.06|48.73|130.59|194.92|375.56|672.24|0.64|130.59|64.33|64.97|-311.23| +2451706|40009|3635|39985|1117552|2319|38239|7|272|692|60|57.85|89.08|41.86|0.00|2511.60|3471.00|5344.80|150.69|0.00|2511.60|2662.29|-959.40| +|40009|14627||1117552|||||692||41.53||||3218.00||5191.00|193.08|||3411.08|-935.00| +2451706|40009|14222|39985|1117552|2319|38239|7|231|692|50|3.39|6.10|0.61|0.00|30.50|169.50|305.00|0.30|0.00|30.50|30.80|-139.00| +||14096|39985|1117552|2319||7|212|692|20|65.53||60.43|||1310.60|2083.80|||1208.60||-102.00| +||15197|||||7|106|692||69.45|125.70||0.00|4689.23|6319.95|11438.70|422.03|0.00|||-1630.72| +2451706|40009|3173|39985|1117552|2319|38239|7|120|692|20|66.19|122.45|121.22|0.00|2424.40|1323.80|2449.00|24.24|0.00|2424.40|2448.64|1100.60| +2451706|40009|17552|39985|1117552|2319|38239|7|127|692|61|37.14|41.59|39.09|0.00|2384.49|2265.54|2536.99|166.91|0.00|2384.49|2551.40|118.95| +2450914|69689|10892|58716|1812917|6671|12072|10|243|693|25|41.13|75.26|21.07|0.00|526.75|1028.25|1881.50|42.14|0.00|526.75|568.89|-501.50| +2450914|69689|3418|58716|1812917|6671|12072|10|24|693|82|33.28|56.24|36.55|2397.68|2997.10|2728.96|4611.68|23.97|2397.68|599.42|623.39|-2129.54| +2450914|69689|16699|58716|1812917|6671|12072|10|172|693|80|29.30|44.82|3.58|0.00|286.40|2344.00|3585.60|2.86|0.00|286.40|289.26|-2057.60| +2450914|69689|8506|58716|1812917|6671|12072|10|65|693|56|42.04|42.46|2.54|0.00|142.24|2354.24|2377.76|5.68|0.00|142.24|147.92|-2212.00| +2450914|69689|4874|58716|1812917|6671|12072|10|130|693|59|9.79|15.95|2.39|0.00|141.01|577.61|941.05|0.00|0.00|141.01|141.01|-436.60| +2450914|69689|17824|58716|1812917|6671|12072|10|178|693|61|97.58|113.19|50.93|310.67|3106.73|5952.38|6904.59|167.76|310.67|2796.06|2963.82|-3156.32| +2450914|69689|7022|58716|1812917|6671|12072|10|25|693|40|53.37|67.77|54.89|0.00|2195.60|2134.80|2710.80|21.95|0.00|2195.60|2217.55|60.80| +|69689|17065||1812917||12072||123|693|1|78.26|79.82||10.72|12.77||79.82||10.72|2.05|2.05|| +2450914|69689|17473|58716|1812917|6671|12072|10|281|693|15|98.35|115.06|109.30|0.00|1639.50|1475.25|1725.90|0.00|0.00|1639.50|1639.50|164.25| +2450914|69689|2269|58716|1812917|6671|12072|10|121|693|74|54.63|82.49|25.57|0.00|1892.18|4042.62|6104.26|132.45|0.00|1892.18|2024.63|-2150.44| +2450914|69689|11276|58716|1812917|6671|12072|10|254|693|10|12.88|15.84|9.50|0.00|95.00|128.80|158.40|7.60|0.00|95.00|102.60|-33.80| +2451866|37799|16271|96731|1446400|3716|44632|2|162|694|90|28.61|43.20|0.43|0.00|38.70|2574.90|3888.00|0.00|0.00|38.70|38.70|-2536.20| +2451866|37799|1849|96731|1446400|3716|44632|2|3|694|48|52.91|73.54|10.29|0.00|493.92|2539.68|3529.92|4.93|0.00|493.92|498.85|-2045.76| +2451866|37799|9043|96731|1446400|3716|44632|2|293|694|65|10.09|16.44|11.17|0.00|726.05|655.85|1068.60|36.30|0.00|726.05|762.35|70.20| +2451866|37799|7934|96731|1446400|3716|44632|2|157|694|30|35.35|59.03|16.52|0.00|495.60|1060.50|1770.90|39.64|0.00|495.60|535.24|-564.90| +2451866|37799|13460|96731|1446400|3716|44632|2|46|694|31|60.32|60.32|16.28|0.00|504.68|1869.92|1869.92|30.28|0.00|504.68|534.96|-1365.24| +2451866|37799|16735|96731|1446400||||210|694|95|34.04||||2264.80|3233.80||67.94||2264.80|2332.74|| +2451866|37799|12875|96731|1446400|3716|44632|2|274|694|33|89.27|138.36|37.35|0.00|1232.55|2945.91|4565.88|61.62|0.00|1232.55|1294.17|-1713.36| +2451866|37799|7955|96731|1446400|3716|44632|2|204|694|83|72.61|72.61|20.33|0.00|1687.39|6026.63|6026.63|33.74|0.00|1687.39|1721.13|-4339.24| +2451866|37799|8324|96731|1446400|3716|44632|2|36|694|17|92.54|158.24|28.48|0.00|484.16|1573.18|2690.08|29.04|0.00|484.16|513.20|-1089.02| +2451866|37799|10436|96731|1446400|3716|44632|2|254|694|75|79.51|154.24|41.64|0.00|3123.00|5963.25|11568.00|124.92|0.00|3123.00|3247.92|-2840.25| +2451866|37799|3157|96731|1446400|3716|44632|2|174|694|31|10.16|10.87|2.06|0.00|63.86|314.96|336.97|3.19|0.00|63.86|67.05|-251.10| +2451866|37799|2732|96731|1446400|3716|44632|2|131|694|68|37.23|72.59|49.36|0.00|3356.48|2531.64|4936.12|67.12|0.00|3356.48|3423.60|824.84| +2451177|38017|9559|48573|610588|3003|26665|2|175|695|52|13.36|21.24|12.31|0.00|640.12|694.72|1104.48|51.20|0.00|640.12|691.32|-54.60| +2451177|38017|16789|48573|610588|3003|26665|2|80|695|61|39.74|46.89|3.75|0.00|228.75|2424.14|2860.29|13.72|0.00|228.75|242.47|-2195.39| +2451177|38017|13112|48573|610588|3003|26665|2|178|695|59|9.91|14.76|8.85|0.00|522.15|584.69|870.84|26.10|0.00|522.15|548.25|-62.54| +2451177|38017|133|48573|610588|3003|26665|2|191|695|31|81.33|152.08|48.66|0.00|1508.46|2521.23|4714.48|120.67|0.00|1508.46|1629.13|-1012.77| +2451177|38017|16393|48573|610588|3003|26665|2|245|695|16|20.97|38.79|33.74|0.00|539.84|335.52|620.64|5.39|0.00|539.84|545.23|204.32| +2451177|38017|9943|48573|610588|3003|26665|2|286|695|59|14.65|18.31|6.59|0.00|388.81|864.35|1080.29|7.77|0.00|388.81|396.58|-475.54| +2451177|38017|553|48573|610588|3003|26665|2|133|695|89|6.21|7.82|3.20|0.00|284.80|552.69|695.98|8.54|0.00|284.80|293.34|-267.89| +2451177|38017|5488|48573|610588|3003|26665|2|41|695|60|90.19|178.57|46.42|0.00|2785.20|5411.40|10714.20|194.96|0.00|2785.20|2980.16|-2626.20| +2451177|38017|15848|48573|610588|3003|26665|2|114|695|40|50.36|62.44|26.84|0.00|1073.60|2014.40|2497.60|42.94|0.00|1073.60|1116.54|-940.80| +2451177|38017|4459|48573|610588|3003|26665|2|57|695|79|82.17|89.56|76.12|0.00|6013.48|6491.43|7075.24|360.80|0.00|6013.48|6374.28|-477.95| +2452135|61367|7425|68494|1843097|1347|11950|4|168|696|77|40.33|77.83|55.25|0.00|4254.25|3105.41|5992.91|127.62|0.00|4254.25|4381.87|1148.84| +2452135|61367|6365|68494|1843097|1347|11950|4|189|696|5|25.36|46.66|19.59|0.00|97.95|126.80|233.30|6.85|0.00|97.95|104.80|-28.85| +2452135|61367|14035|68494|1843097|1347|11950|4|38|696|53|41.93|80.92|26.70|0.00|1415.10|2222.29|4288.76|99.05|0.00|1415.10|1514.15|-807.19| +2452135|61367|4451|68494|1843097|1347|11950|4|59|696|49|85.55|169.38|88.07|1639.86|4315.43|4191.95|8299.62|240.80|1639.86|2675.57|2916.37|-1516.38| +2452135||3427|68494||||4|249|696|2|||136.46|264.73|272.92|199.78|275.68|0.00|264.73|8.19|8.19|-191.59| +2452135|61367|10201|68494|1843097|1347|11950|4|247|696|52|80.55|124.85|58.67|0.00|3050.84|4188.60|6492.20|244.06|0.00|3050.84|3294.90|-1137.76| +2452135|61367|16033|68494|1843097|1347|11950|4|184|696|69|47.28|50.11|29.06|0.00|2005.14|3262.32|3457.59|180.46|0.00|2005.14|2185.60|-1257.18| +2452135|61367|6133|68494|1843097|1347|11950|4|274|696|13|53.80|80.16|54.50|0.00|708.50|699.40|1042.08|0.00|0.00|708.50|708.50|9.10| +2452271|71201|11001|1081|1160254|3074|46647|1|75|697|62|70.23|71.63|49.42|0.00|3064.04|4354.26|4441.06|91.92|0.00|3064.04|3155.96|-1290.22| +2452271|71201|9363|1081|1160254|3074|46647|1|144|697|99|6.36|10.81|10.48|0.00|1037.52|629.64|1070.19|31.12|0.00|1037.52|1068.64|407.88| +2452271|71201|61|1081|1160254|3074|46647|1|252|697|14|86.03|111.83|80.51|0.00|1127.14|1204.42|1565.62|56.35|0.00|1127.14|1183.49|-77.28| +2452271|71201|7977|1081|1160254|3074|46647|1|78|697|59|64.63|83.37|81.70|0.00|4820.30|3813.17|4918.83|0.00|0.00|4820.30|4820.30|1007.13| +2452271|71201|5837|1081|1160254|3074|46647|1|118|697|2|37.03|55.54|12.77|0.00|25.54|74.06|111.08|1.27|0.00|25.54|26.81|-48.52| +2452271|71201|4879|1081|1160254|3074|46647|1|269|697|3|77.76|139.96|128.76|0.00|386.28|233.28|419.88|3.86|0.00|386.28|390.14|153.00| +2452271|71201|15643|1081|1160254|3074|46647|1|31|697|7|70.10|94.63|47.31|0.00|331.17|490.70|662.41|16.55|0.00|331.17|347.72|-159.53| +2452271|71201|12881|1081|1160254|3074|46647|1|8|697|40|55.02|99.03|20.79|0.00|831.60|2200.80|3961.20|33.26|0.00|831.60|864.86|-1369.20| +2452271|71201|14101|1081|1160254|3074|46647|1|3|697|56|93.79|185.70|157.84|0.00|8839.04|5252.24|10399.20|795.51|0.00|8839.04|9634.55|3586.80| +2451822|39379|857|45191|1460021|2051|37328|10|154|698|38|16.56|28.48|25.91|0.00|984.58|629.28|1082.24|68.92|0.00|984.58|1053.50|355.30| +2451822|39379|4592|45191|1460021|2051|37328|10|22|698|51|84.72|156.73|53.28|0.00|2717.28|4320.72|7993.23|244.55|0.00|2717.28|2961.83|-1603.44| +2451822|39379|5321|45191|1460021|2051|37328|10|129|698|5|7.80|8.26|0.24|0.24|1.20|39.00|41.30|0.04|0.24|0.96|1.00|-38.04| +2451822|39379|11161|45191|1460021|2051|37328|10|194|698|92|1.71|2.37|1.84|0.00|169.28|157.32|218.04|15.23|0.00|169.28|184.51|11.96| +2451822|39379|6860|45191|1460021|2051|37328|10|183|698|84|23.12|42.30|15.65|0.00|1314.60|1942.08|3553.20|118.31|0.00|1314.60|1432.91|-627.48| +2451822|39379|5303|45191|1460021|2051|37328|10|45|698|58|38.23|51.61|49.02|0.00|2843.16|2217.34|2993.38|227.45|0.00|2843.16|3070.61|625.82| +2451822|39379|7687|45191|1460021|2051|37328|10|126|698|90|91.05|116.54|10.48|0.00|943.20|8194.50|10488.60|75.45|0.00|943.20|1018.65|-7251.30| +2451822|39379|1097|45191|1460021|2051|37328|10|295|698|20|84.79|168.73|107.98|0.00|2159.60|1695.80|3374.60|64.78|0.00|2159.60|2224.38|463.80| +2451822|39379|8081|45191|1460021|2051|37328|10|140|698|46|66.48|93.07|65.14|0.00|2996.44|3058.08|4281.22|59.92|0.00|2996.44|3056.36|-61.64| +2451822|39379|9133|45191|1460021|2051|37328|10|136|698|10|1.83|2.12|1.95|0.00|19.50|18.30|21.20|1.75|0.00|19.50|21.25|1.20| +2451822|39379|7856|45191|1460021|2051|37328|10|197|698|97|59.34|67.64|12.17|0.00|1180.49|5755.98|6561.08|94.43|0.00|1180.49|1274.92|-4575.49| +2450866|39151|2924|20223|1156060|697|6581|7|211|699|88|72.90|103.51|65.21|0.00|5738.48|6415.20|9108.88|344.30|0.00|5738.48|6082.78|-676.72| +2450866|39151|12709|20223|1156060|697|6581|7|29|699|72|67.25|99.53|76.63|0.00|5517.36|4842.00|7166.16|496.56|0.00|5517.36|6013.92|675.36| +2450866|39151|16240|20223|1156060|697|6581|7|196|699|83|42.07|76.56|39.81|0.00|3304.23|3491.81|6354.48|132.16|0.00|3304.23|3436.39|-187.58| +2450866|39151|17917|20223|1156060|697|6581|7|280|699|77|26.49|36.02|18.37|0.00|1414.49|2039.73|2773.54|14.14|0.00|1414.49|1428.63|-625.24| +2450866|39151|6295|20223|1156060|697|6581|7|96|699|95|33.63|60.53|37.52|0.00|3564.40|3194.85|5750.35|320.79|0.00|3564.40|3885.19|369.55| +2450866|39151|6859|20223|1156060|697|6581|7|21|699|27|52.12|82.87|0.82|13.06|22.14|1407.24|2237.49|0.00|13.06|9.08|9.08|-1398.16| +2450866|39151|656|20223|1156060|697|6581|7|168|699|53|6.99|12.23|6.48|0.00|343.44|370.47|648.19|13.73|0.00|343.44|357.17|-27.03| +2450866|39151|11314|20223|1156060|697|6581|7|27|699|56|33.23|45.52|12.29|0.00|688.24|1860.88|2549.12|41.29|0.00|688.24|729.53|-1172.64| +2450866|39151|17354|20223|1156060|697|6581|7|18|699|81|28.63|35.21|11.26|0.00|912.06|2319.03|2852.01|0.00|0.00|912.06|912.06|-1406.97| +2450866|39151|14114|20223|1156060|697|6581|7|212|699|31|66.56|77.87|54.50|0.00|1689.50|2063.36|2413.97|33.79|0.00|1689.50|1723.29|-373.86| +2450866|39151|15877|20223|1156060|697|6581|7|168|699|31|1.49|1.49|1.19|0.00|36.89|46.19|46.19|2.95|0.00|36.89|39.84|-9.30| +2450866|39151|4336|20223|1156060|697|6581|7|133|699|2|28.06|33.11|11.25|22.27|22.50|56.12|66.22|0.00|22.27|0.23|0.23|-55.89| +2450866|39151|3004|20223|1156060|697|6581|7|141|699|85|53.47|94.64|88.01|0.00|7480.85|4544.95|8044.40|598.46|0.00|7480.85|8079.31|2935.90| +2450866|39151|6724|20223|1156060|697|6581|7|15|699|64|12.99|17.79|15.12|0.00|967.68|831.36|1138.56|67.73|0.00|967.68|1035.41|136.32| +2450866|39151|5726|20223|1156060|697|6581|7|127|699|56|95.33|189.70|140.37|0.00|7860.72|5338.48|10623.20|157.21|0.00|7860.72|8017.93|2522.24| +2450866|39151|3691|20223|1156060|697|6581|7|110|699|4|85.73|166.31|161.32|0.00|645.28|342.92|665.24|12.90|0.00|645.28|658.18|302.36| +2451546|45565|16286|32668|345478|6641|5051|8|94|700|67|43.33|43.33|16.03|526.26|1074.01|2903.11|2903.11|5.47|526.26|547.75|553.22|-2355.36| +2451546|45565|2491|32668|345478|6641|5051|8|245|700|89|6.38|10.14|3.14|0.00|279.46|567.82|902.46|16.76|0.00|279.46|296.22|-288.36| +2451546|45565|7295|32668|345478|6641|5051|8|83|700|46|45.26|70.60|16.23|0.00|746.58|2081.96|3247.60|37.32|0.00|746.58|783.90|-1335.38| +2451546|45565|14408|32668|345478|6641|5051|8|188|700|18|84.24|140.68|47.83|0.00|860.94|1516.32|2532.24|25.82|0.00|860.94|886.76|-655.38| +2451546|45565|16849|32668|345478|6641|5051|8|41|700|12|28.79|54.70|54.15|0.00|649.80|345.48|656.40|19.49|0.00|649.80|669.29|304.32| +2451546|45565|7136|32668|345478|6641|5051|8|76|700|59|84.22|118.75|79.56|422.46|4694.04|4968.98|7006.25|0.00|422.46|4271.58|4271.58|-697.40| +2451546|45565|13387|32668|345478|6641|5051|8|38|700|95|79.04|82.99|63.07|0.00|5991.65|7508.80|7884.05|359.49|0.00|5991.65|6351.14|-1517.15| +2451546|45565|14873|32668|345478|6641|5051|8|92|700|1|17.72|21.97|12.74|8.91|12.74|17.72|21.97|0.22|8.91|3.83|4.05|-13.89| +2451546|45565|9947|32668|345478|6641|5051|8|30|700|10|90.00|111.60|79.23|285.22|792.30|900.00|1116.00|20.28|285.22|507.08|527.36|-392.92| +2451546|45565|7775|32668|345478|6641|5051|8|29|700|82|98.89|145.36|50.87|1209.68|4171.34|8108.98|11919.52|59.23|1209.68|2961.66|3020.89|-5147.32| +2451546|45565|3407|32668|345478|6641|5051|8|160|700|5|49.62|69.46|68.07|0.00|340.35|248.10|347.30|6.80|0.00|340.35|347.15|92.25| +2451546|45565|17843|32668|345478|6641|5051|8|184|700|71|37.11|63.08|27.75|0.00|1970.25|2634.81|4478.68|118.21|0.00|1970.25|2088.46|-664.56| +2452144|43332|5713|19152|1291744|1489|881|8|181|701|92|85.84|107.30|85.84|0.00|7897.28|7897.28|9871.60|236.91|0.00|7897.28|8134.19|0.00| +2452144||2329|19152||1489|881|||701||35.95||28.70||2611.70|3271.45|||||2690.05|-659.75| +2452144|43332|10893|19152|1291744|1489|881|8|162|701|42|18.51|36.83|6.62|0.00|278.04|777.42|1546.86|16.68|0.00|278.04|294.72|-499.38| +2452144|43332|3419|19152|1291744|1489|881|8|216|701|32|93.34|111.07|48.87|0.00|1563.84|2986.88|3554.24|62.55|0.00|1563.84|1626.39|-1423.04| +2452144|43332|16699|19152|1291744|1489|881|8|159|701|14|21.83|30.78|11.69|78.55|163.66|305.62|430.92|5.10|78.55|85.11|90.21|-220.51| +2452144|43332|8507|19152|1291744|1489|881|8|98|701|67|1.74|3.34|1.80|65.12|120.60|116.58|223.78|0.55|65.12|55.48|56.03|-61.10| +2452144|43332|4875||1291744|1489|881||178|701|60|15.23|||0.00|835.20|913.80|||0.00|835.20||| +2452144|43332|17825|19152|1291744|1489|881|8|240|701|76|75.50|145.71|36.42|0.00|2767.92|5738.00|11073.96|27.67|0.00|2767.92|2795.59|-2970.08| +2452144|43332|7023|19152|1291744|1489|881|8|180|701|49|81.50|117.36|89.19|0.00|4370.31|3993.50|5750.64|87.40|0.00|4370.31|4457.71|376.81| +2452144|43332|17065|19152|1291744|1489|881|8|149|701|85|25.18|27.44|5.21|0.00|442.85|2140.30|2332.40|4.42|0.00|442.85|447.27|-1697.45| +2452144|43332|17473|19152|1291744|1489|881|8|31|701|44|38.38|42.98|15.04|0.00|661.76|1688.72|1891.12|46.32|0.00|661.76|708.08|-1026.96| +2452144|43332|2269|19152|1291744|1489|881|8|234|701|90|13.49|20.10|2.41|0.00|216.90|1214.10|1809.00|4.33|0.00|216.90|221.23|-997.20| +2452144|43332|11277|19152|1291744|1489|881|8|205|701|60|45.66|55.24|18.22|0.00|1093.20|2739.60|3314.40|21.86|0.00|1093.20|1115.06|-1646.40| +2452144|43332|2097|19152|1291744|1489|881|8|12|701|98|99.21|189.49|13.26|0.00|1299.48|9722.58|18570.02|25.98|0.00|1299.48|1325.46|-8423.10| +2452144|43332|9457|19152|1291744|1489|881|8|276|701|11|10.89|14.37|9.48|0.00|104.28|119.79|158.07|2.08|0.00|104.28|106.36|-15.51| +2452144|43332|15279|19152|1291744|1489|881|8|270|701|69|60.87|71.21|14.24|923.60|982.56|4200.03|4913.49|4.71|923.60|58.96|63.67|-4141.07| +2452549|35950|3918|24177|418661|345|44429|8|69|702|86|24.00|31.68|29.77|0.00|2560.22|2064.00|2724.48|153.61|0.00|2560.22|2713.83|496.22| +2452549|35950|15241|24177|418661|345|44429|8|106|702|27|51.48|85.45|6.83|145.68|184.41|1389.96|2307.15|1.54|145.68|38.73|40.27|-1351.23| +|35950|3793|||345|44429|8|109|702|37|79.37|81.75||1346.38||2936.69||23.08|1346.38|||-2680.23| +2452549|35950|15690|24177|418661|345|44429|8|260|702|70|47.21|51.45|9.26|550.97|648.20|3304.70|3601.50|0.97|550.97|97.23|98.20|-3207.47| +||12279|24177||345|44429|8||702|83||123.54|||4613.97||10253.82|138.41||4613.97||-3524.18| +2452549|35950|7983|24177|418661|345|44429|8|174|702|54|99.43|140.19|21.02|692.39|1135.08|5369.22|7570.26|8.85|692.39|442.69|451.54|-4926.53| +2452549|35950|11718|24177|418661|345|44429|8|255|702|51|38.84|75.34|6.02|0.00|307.02|1980.84|3842.34|15.35|0.00|307.02|322.37|-1673.82| +2452549|35950|6057|24177|418661|345|44429|8|72|702|41|97.04|115.47|8.08|0.00|331.28|3978.64|4734.27|9.93|0.00|331.28|341.21|-3647.36| +2452549|35950|14593|24177|418661|345|44429|8|40|702|58|6.19|10.70|0.53|10.14|30.74|359.02|620.60|1.23|10.14|20.60|21.83|-338.42| +2452549|35950|6439|24177|418661|345|44429|8|178|702|39|94.41|117.06|81.94|0.00|3195.66|3681.99|4565.34|95.86|0.00|3195.66|3291.52|-486.33| +2452549|35950|10815|24177|418661|345|44429|8|18|702|24|46.63|64.81|17.49|0.00|419.76|1119.12|1555.44|4.19|0.00|419.76|423.95|-699.36| +2452549|35950|14179|24177|418661|345|44429|8|245|702|92|69.21|76.82|24.58|0.00|2261.36|6367.32|7067.44|203.52|0.00|2261.36|2464.88|-4105.96| +2452549|35950|16867|24177|418661|345|44429|8|278|702|34|92.90|156.07|129.53|0.00|4404.02|3158.60|5306.38|308.28|0.00|4404.02|4712.30|1245.42| +2452549|35950|7797|24177|418661|345|44429|8|91|702|99|80.84|136.61|69.67|0.00|6897.33|8003.16|13524.39|0.00|0.00|6897.33|6897.33|-1105.83| +2452549|35950|9903|24177|418661|345|44429|8|291|702|54|3.90|4.52|3.70|0.00|199.80|210.60|244.08|15.98|0.00|199.80|215.78|-10.80| +2452549|35950|13777|24177|418661|345|44429|8|170|702|11|44.86|66.84|28.74|0.00|316.14|493.46|735.24|22.12|0.00|316.14|338.26|-177.32| +2452118|45398|5373|19459|656107|3602|2297|8|210|703|85|57.66|58.23|40.76|0.00|3464.60|4901.10|4949.55|0.00|0.00|3464.60|3464.60|-1436.50| +2452118|45398|639|19459|656107|3602|2297|8|6|703|33|88.97|111.21|98.97|0.00|3266.01|2936.01|3669.93|97.98|0.00|3266.01|3363.99|330.00| +2452118|45398|16219|19459|656107|3602|2297|8|80|703|73|47.91|55.57|43.90|0.00|3204.70|3497.43|4056.61|224.32|0.00|3204.70|3429.02|-292.73| +2452118|45398|8921|19459|656107|3602|2297|8|225|703|84|38.99|67.45|62.05|0.00|5212.20|3275.16|5665.80|156.36|0.00|5212.20|5368.56|1937.04| +2452118|45398|3983|19459|656107|3602|2297|8|41|703|29|71.17|86.11|67.16|1538.63|1947.64|2063.93|2497.19|0.00|1538.63|409.01|409.01|-1654.92| +2452118|45398|12817|19459|656107|3602|2297|8|257|703|14|5.25|6.24|2.68|0.00|37.52|73.50|87.36|0.00|0.00|37.52|37.52|-35.98| +2452118|45398|15041|19459|656107|3602|2297|8|136|703|62|62.78|103.58|65.25|0.00|4045.50|3892.36|6421.96|161.82|0.00|4045.50|4207.32|153.14| +2452118|45398|17141|19459|656107|3602|2297|8|129|703|52|76.08|111.07|51.09|0.00|2656.68|3956.16|5775.64|106.26|0.00|2656.68|2762.94|-1299.48| +2452118|45398|1915|19459|656107|3602|2297|8|29|703|53|60.51|120.41|13.24|0.00|701.72|3207.03|6381.73|28.06|0.00|701.72|729.78|-2505.31| +2452118|45398|16705|19459|656107|3602|2297|8|215|703|73|98.98|186.08|66.98|0.00|4889.54|7225.54|13583.84|293.37|0.00|4889.54|5182.91|-2336.00| +2452633|48935|14715|4751|1614343|1088|29880|1|4|704|1|2.48|4.68|3.04|0.00|3.04|2.48|4.68|0.27|0.00|3.04|3.31|0.56| +2452633|48935|8004|4751|1614343|1088|29880|1|254|704|89|25.80|35.34|22.61|0.00|2012.29|2296.20|3145.26|140.86|0.00|2012.29|2153.15|-283.91| +|48935|2049||||29880|1|242|704|71|||||||1914.87||||309.78|| +2452633|48935|15873|4751|1614343|1088|29880|1|22|704|21|90.29|138.14|78.73|0.00|1653.33|1896.09|2900.94|132.26|0.00|1653.33|1785.59|-242.76| +2452633|48935|2667|4751|1614343|1088|29880|1|194|704|55|87.68|105.21|36.82|0.00|2025.10|4822.40|5786.55|162.00|0.00|2025.10|2187.10|-2797.30| +2452633|48935|15999|4751|1614343|1088|29880|1|202|704|53|17.38|31.28|12.19|0.00|646.07|921.14|1657.84|38.76|0.00|646.07|684.83|-275.07| +2452633|48935|15729|4751|1614343|1088|29880|1|104|704|61|70.21|93.37|25.20|0.00|1537.20|4282.81|5695.57|122.97|0.00|1537.20|1660.17|-2745.61| +2452633|48935|3903|4751|1614343|1088|29880|1|240|704|82|20.42|31.65|6.64|0.00|544.48|1674.44|2595.30|21.77|0.00|544.48|566.25|-1129.96| +2452633|48935|14205|4751|1614343|1088|29880|1|3|704|32|46.85|72.61|55.18|971.16|1765.76|1499.20|2323.52|63.56|971.16|794.60|858.16|-704.60| +2452633|48935|5199|4751|1614343|1088|29880|1|235|704|30|20.64|29.30|21.68|117.07|650.40|619.20|879.00|21.33|117.07|533.33|554.66|-85.87| +2452633|48935|5089|4751|1614343|1088|29880|1|190|704|55|24.46|45.74|38.87|0.00|2137.85|1345.30|2515.70|64.13|0.00|2137.85|2201.98|792.55| +2452633|48935|15139|4751|1614343|1088|29880|1|22|704|81|76.15|95.18|54.25|2328.95|4394.25|6168.15|7709.58|41.30|2328.95|2065.30|2106.60|-4102.85| +2452633|48935|4087|4751|1614343|1088|29880|1|15|704|36|58.86|91.82|69.78|0.00|2512.08|2118.96|3305.52|125.60|0.00|2512.08|2637.68|393.12| +2451499|29863|8227|53418|156850|7035|44067|2|76|705|40|50.09|75.13|21.78|331.05|871.20|2003.60|3005.20|10.80|331.05|540.15|550.95|-1463.45| +2451499|29863|7708|53418|156850|7035|44067|2|140|705|66|83.93|151.91|34.93|0.00|2305.38|5539.38|10026.06|92.21|0.00|2305.38|2397.59|-3234.00| +2451499|29863|11956|53418|156850|7035|44067|2|30|705|73|38.20|62.26|59.76|0.00|4362.48|2788.60|4544.98|0.00|0.00|4362.48|4362.48|1573.88| +2451499|29863|7100|53418|156850|7035|44067|2|128|705|88|23.48|41.32|1.23|0.00|108.24|2066.24|3636.16|0.00|0.00|108.24|108.24|-1958.00| +2451499|29863|11632|53418|156850|7035|44067|2|236|705|17|38.58|42.05|42.05|0.00|714.85|655.86|714.85|0.00|0.00|714.85|714.85|58.99| +2451499|29863|15484|53418|156850|7035|44067|2|28|705|80|10.35|12.73|8.14|0.00|651.20|828.00|1018.40|45.58|0.00|651.20|696.78|-176.80| +2451499|29863|12418|53418|156850|7035|44067|2|85|705|66|48.28|64.21|34.03|0.00|2245.98|3186.48|4237.86|0.00|0.00|2245.98|2245.98|-940.50| +|29863|13306|||7035|44067|2|278|705||95.41|166.96|45.07|0.00||4007.22||18.92|0.00||1911.86|-2114.28| +2451795|38121|2309|53426|338594|4562|10177|1|263|706|69|35.66|49.21|23.12|0.00|1595.28|2460.54|3395.49|47.85|0.00|1595.28|1643.13|-865.26| +2451795|38121|7142|53426|338594|4562|10177|1|291|706|9|95.83|104.45|25.06|196.21|225.54|862.47|940.05|2.05|196.21|29.33|31.38|-833.14| +2451795|38121|14969|||||||706|85|66.26|108.66|35.85||3047.25||9236.10|||||| +2451795|38121|7597|53426|338594|4562|10177|1|85|706|25|45.36|68.49|48.62|0.00|1215.50|1134.00|1712.25|24.31|0.00|1215.50|1239.81|81.50| +2451795|38121|4838|53426|338594|4562|10177|1|63|706|91|77.97|113.05|99.48|0.00|9052.68|7095.27|10287.55|633.68|0.00|9052.68|9686.36|1957.41| +2451795|38121|1190||||||203|706|100|||||1568.00||5410.00|||1568.00|1646.40|-2063.00| +2451795|38121|15407|53426|338594|4562|10177|1|239|706|69|45.19|81.34|17.89|0.00|1234.41|3118.11|5612.46|86.40|0.00|1234.41|1320.81|-1883.70| +2451795|38121|6161|53426|338594|4562|10177|1|180|706|66|43.61|54.07|30.27|0.00|1997.82|2878.26|3568.62|119.86|0.00|1997.82|2117.68|-880.44| +2451795|38121|10351|53426|338594|4562|10177|1|131|706|77|9.44|14.06|9.27|0.00|713.79|726.88|1082.62|57.10|0.00|713.79|770.89|-13.09| +2451795|38121|14735|53426|338594|4562|10177|1|52|706|17|61.79|87.74|2.63|0.00|44.71|1050.43|1491.58|0.00|0.00|44.71|44.71|-1005.72| +2452615|35623|309|91838|1689608|978|47195|8|109|707|91|22.21|24.43|19.29|0.00|1755.39|2021.11|2223.13|140.43|0.00|1755.39|1895.82|-265.72| +2452615|35623|15187|91838|1689608|978|47195|8|26|707|52|48.69|58.42|49.65|0.00|2581.80|2531.88|3037.84|103.27|0.00|2581.80|2685.07|49.92| +2452615|35623|14413|91838|1689608|978|47195|8|85|707|18|87.27|131.77|21.08|0.00|379.44|1570.86|2371.86|18.97|0.00|379.44|398.41|-1191.42| +2452615|35623|10629|91838|1689608|978|47195|8|258|707|85|55.17|104.82|36.68|0.00|3117.80|4689.45|8909.70|124.71|0.00|3117.80|3242.51|-1571.65| +2452615|35623|15165|91838|1689608|978|47195|8|103|707|7|21.75|29.14|24.18|0.00|169.26|152.25|203.98|6.77|0.00|169.26|176.03|17.01| +2452615|35623|17107|91838|1689608|978|47195|8|200|707|88|28.49|54.70|23.52|0.00|2069.76|2507.12|4813.60|0.00|0.00|2069.76|2069.76|-437.36| +2452615|35623|13107|91838|1689608|978|47195|8|114|707|67|50.19|50.19|24.09|0.00|1614.03|3362.73|3362.73|0.00|0.00|1614.03|1614.03|-1748.70| +2452615|35623|8304|91838|1689608|978|47195|8|180|707|41|56.43|87.46|42.85|0.00|1756.85|2313.63|3585.86|105.41|0.00|1756.85|1862.26|-556.78| +2452615|35623|10014|91838|1689608|978|47195|8|132|707|55|80.48|95.77|57.46|0.00|3160.30|4426.40|5267.35|189.61|0.00|3160.30|3349.91|-1266.10| +2452615|35623|15847|91838|1689608|978|47195|8|16|707|8|13.42|15.29|6.57|0.00|52.56|107.36|122.32|1.05|0.00|52.56|53.61|-54.80| +2451906|47736|9847|51117|387804|6091|37178|4|57|708|93|4.88|7.61|4.64|168.29|431.52|453.84|707.73|15.79|168.29|263.23|279.02|-190.61| +2451906|47736|4279|51117|387804|6091|37178|4|236|708|10|5.24|9.90|6.53|0.00|65.30|52.40|99.00|1.30|0.00|65.30|66.60|12.90| +2451906|47736|17605|51117|387804|6091|37178|4|279|708|45|63.99|72.94|26.98|0.00|1214.10|2879.55|3282.30|97.12|0.00|1214.10|1311.22|-1665.45| +2451906|47736|11621|51117|387804|6091|37178|4|235|708|18|75.89|114.59|104.27|0.00|1876.86|1366.02|2062.62|168.91|0.00|1876.86|2045.77|510.84| +2451906|47736|7681|51117|387804|6091|37178|4|243|708|8|59.91|85.67|47.97|0.00|383.76|479.28|685.36|15.35|0.00|383.76|399.11|-95.52| +||16285||387804|||4||708||53.26|||0.00||852.16|||0.00|245.60|257.88|| +2451906|47736|10813|51117|387804|6091|37178|4|153|708|52|75.55|106.52|74.56|0.00|3877.12|3928.60|5539.04|38.77|0.00|3877.12|3915.89|-51.48| +2451906|47736|3809|51117|387804|6091|37178|4|132|708|94|67.67|92.03|19.32|0.00|1816.08|6360.98|8650.82|36.32|0.00|1816.08|1852.40|-4544.90| +2451906|47736|6041|51117|387804|6091|37178|4|118|708|21|4.19|6.53|1.95|0.00|40.95|87.99|137.13|0.40|0.00|40.95|41.35|-47.04| +2451906|47736|1457|51117|387804|6091|37178|4|240|708|6|96.32|190.71|173.54|0.00|1041.24|577.92|1144.26|72.88|0.00|1041.24|1114.12|463.32| +2451906|47736|5822|51117|387804|6091|37178|4|299|708|60|78.91|95.48|82.11|0.00|4926.60|4734.60|5728.80|49.26|0.00|4926.60|4975.86|192.00| +2451906|47736|17816|51117|387804|6091|37178|4|194|708|20|54.26|65.65|17.72|205.55|354.40|1085.20|1313.00|7.44|205.55|148.85|156.29|-936.35| +2451906|47736|11101|51117|387804|6091|37178|4|234|708|53|6.42|11.81|4.36|0.00|231.08|340.26|625.93|4.62|0.00|231.08|235.70|-109.18| +2451906|47736|15979|51117|387804|6091|37178|4|147|708|33|7.14|10.99|9.34|0.00|308.22|235.62|362.67|21.57|0.00|308.22|329.79|72.60| +|47736|17425||387804|6091||4||708|88|36.41||6.51|||||||||| +2451906|47736|11192|51117|387804|6091|37178|4|278|708|1|84.42|167.15|96.94|0.00|96.94|84.42|167.15|1.93|0.00|96.94|98.87|12.52| +2452584||9084||887166|6997||2||709||73.09|106.71|53.35|||6578.10|||||5185.62|-1776.60| +2452584|43908|819|23335|887166|6997|31675|2|104|709|70|13.16|18.81|15.42|0.00|1079.40|921.20|1316.70|53.97|0.00|1079.40|1133.37|158.20| +2452584|43908|738|23335|887166||31675|||709|33|||26.41||871.53||||||923.82|| +2452584|43908|16986|23335|887166|6997|31675|2|162|709|78|39.73|49.66|8.44|539.82|658.32|3098.94|3873.48|0.00|539.82|118.50|118.50|-2980.44| +2452584|43908|14959|23335|887166|6997|31675|2|75|709|22|82.55|107.31|103.01|430.58|2266.22|1816.10|2360.82|110.13|430.58|1835.64|1945.77|19.54| +2452584|43908|14215|23335|887166|6997|31675|2|206|709|92|12.52|12.77|10.21|0.00|939.32|1151.84|1174.84|46.96|0.00|939.32|986.28|-212.52| +2452584|43908|7413|23335|887166|6997|31675|2|166|709|67|76.59|142.45|8.54|503.51|572.18|5131.53|9544.15|0.68|503.51|68.67|69.35|-5062.86| +2452584|43908|4374|23335|887166|6997|31675|2|132|709|32|24.71|41.75|12.52|4.00|400.64|790.72|1336.00|3.96|4.00|396.64|400.60|-394.08| +2452584|43908|16548|23335|887166|6997|31675|2|279|709|81|7.89|12.30|0.36|0.00|29.16|639.09|996.30|0.00|0.00|29.16|29.16|-609.93| +2452584|43908|17007|23335|887166|6997|31675|2|128|709|82|56.07|72.33|38.33|0.00|3143.06|4597.74|5931.06|125.72|0.00|3143.06|3268.78|-1454.68| +2452584|43908|9589|23335|887166|6997|31675|2|139|709|93|21.04|40.60|11.36|0.00|1056.48|1956.72|3775.80|42.25|0.00|1056.48|1098.73|-900.24| +2452584|43908|12145|23335|887166|6997|31675|2|185|709|36|6.92|10.31|3.91|0.00|140.76|249.12|371.16|5.63|0.00|140.76|146.39|-108.36| +2452584|43908|5209|23335|887166|6997|31675|2|241|709|75|84.85|123.88|112.73|0.00|8454.75|6363.75|9291.00|760.92|0.00|8454.75|9215.67|2091.00| +2452584||7866||887166||31675||185|709||11.72|||0.00|886.20|984.48|1122.24|35.44|0.00|||| +||15697|||||2||709|38||88.91|26.67||1013.46|||10.13||1013.46||| +2451130|48922|17618|98793|1017639|3685|40531|7|89|710|57|3.15|4.41|2.07|0.00|117.99|179.55|251.37|8.25|0.00|117.99|126.24|-61.56| +2451130|48922|7903|98793|1017639|3685|40531|7|82|710|60|69.45|113.89|107.05|0.00|6423.00|4167.00|6833.40|64.23|0.00|6423.00|6487.23|2256.00| +2451130|48922|1088|98793|1017639|3685|40531|7|155|710|34|94.43|113.31|11.33|0.00|385.22|3210.62|3852.54|11.55|0.00|385.22|396.77|-2825.40| +2451130|48922|7666|98793|1017639|3685|40531|7|288|710|31|23.88|39.40|33.09|0.00|1025.79|740.28|1221.40|61.54|0.00|1025.79|1087.33|285.51| +2451130|48922|218|98793|1017639|3685|40531|7|193|710|7|15.14|19.68|11.80|0.00|82.60|105.98|137.76|0.00|0.00|82.60|82.60|-23.38| +2451130|48922|13994|98793|1017639|3685|40531|7|95|710|26|35.74|45.74|21.95|176.91|570.70|929.24|1189.24|19.68|176.91|393.79|413.47|-535.45| +2451130|48922|13276|98793|1017639|3685|40531|7|196|710|66|37.67|62.15|19.88|1272.71|1312.08|2486.22|4101.90|0.00|1272.71|39.37|39.37|-2446.85| +2451130|48922|11096||1017639||40531|||710|71|80.29|98.75||||5700.59||85.52||4276.33||-1424.26| +2451130|48922|16075|98793|1017639|3685|40531|7|53|710|82|59.32|73.55|24.27|1273.68|1990.14|4864.24|6031.10|57.31|1273.68|716.46|773.77|-4147.78| +2451130|48922|10160|98793|1017639|3685|40531|7|279|710|61|26.70|45.65|8.21|0.00|500.81|1628.70|2784.65|20.03|0.00|500.81|520.84|-1127.89| +2451866|66535|8639|52319|1708864|6209|3554|1|279|711|26|8.71|13.50|10.39|0.00|270.14|226.46|351.00|13.50|0.00|270.14|283.64|43.68| +2451866||16753||1708864|6209|3554|1|132|711|48|||104.58|4266.86|5019.84|3873.60|||4266.86||820.74|| +2451866|66535|9962|52319|1708864|6209|3554|1|239|711|92|13.07|25.22|13.11|0.00|1206.12|1202.44|2320.24|24.12|0.00|1206.12|1230.24|3.68| +2451866|66535|2233|52319|1708864|6209|3554|1|114|711|72|8.34|10.67|10.34|0.00|744.48|600.48|768.24|14.88|0.00|744.48|759.36|144.00| +2451866|66535|2881|52319|1708864|6209|3554|1|299|711|30|48.49|63.03|22.69|0.00|680.70|1454.70|1890.90|27.22|0.00|680.70|707.92|-774.00| +2451866|66535|10532|52319|1708864|6209|3554|1|135|711|79|48.07|77.39|61.91|0.00|4890.89|3797.53|6113.81|293.45|0.00|4890.89|5184.34|1093.36| +2451866|66535|12074|52319|1708864|6209|3554|1|137|711|79|61.82|103.85|90.34|0.00|7136.86|4883.78|8204.15|428.21|0.00|7136.86|7565.07|2253.08| +2451866|66535|725|52319|1708864|6209|3554|1|110|711|4|91.84|124.90|119.90|0.00|479.60|367.36|499.60|14.38|0.00|479.60|493.98|112.24| +2451866|66535|17809|52319|1708864|6209|3554|1|231|711|23|47.59|83.28|66.62|0.00|1532.26|1094.57|1915.44|137.90|0.00|1532.26|1670.16|437.69| +2451866|66535|10613|52319|1708864|6209|3554|1|153|711|58|89.11|114.06|79.84|2732.12|4630.72|5168.38|6615.48|94.93|2732.12|1898.60|1993.53|-3269.78| +2451866|66535|7907|52319|1708864|6209|3554|1|50|711|67|25.24|38.86|37.69|0.00|2525.23|1691.08|2603.62|75.75|0.00|2525.23|2600.98|834.15| +2451866|66535|13415|52319|1708864|6209|3554|1|24|711|44|85.55|127.46|5.09|0.00|223.96|3764.20|5608.24|17.91|0.00|223.96|241.87|-3540.24| +2451866|66535|989|52319|1708864|6209|3554|1|283|711|95|43.92|83.88|7.54|0.00|716.30|4172.40|7968.60|21.48|0.00|716.30|737.78|-3456.10| +2452623|49064|6162|87767|1203116|2465|40167|7|257|712|65|30.39|30.39|2.73|0.00|177.45|1975.35|1975.35|1.77|0.00|177.45|179.22|-1797.90| +2452623|49064|10351|87767|1203116|2465|40167|7|60|712|77|65.37|108.51|56.42|477.87|4344.34|5033.49|8355.27|347.98|477.87|3866.47|4214.45|-1167.02| +2452623|49064|14736|87767|1203116|2465|40167|7|118|712|65|30.96|30.96|21.05|0.00|1368.25|2012.40|2012.40|27.36|0.00|1368.25|1395.61|-644.15| +2452623|49064|5659|87767|1203116|2465|40167|7|63|712|33|72.00|143.28|75.93|0.00|2505.69|2376.00|4728.24|175.39|0.00|2505.69|2681.08|129.69| +2452623|49064|8437|87767|1203116|2465|40167|7|287|712|26|47.22|83.57|31.75|0.00|825.50|1227.72|2172.82|16.51|0.00|825.50|842.01|-402.22| +2452623|49064|11106|87767|1203116|2465|40167|7|34|712|91|22.75|24.11|13.50|0.00|1228.50|2070.25|2194.01|98.28|0.00|1228.50|1326.78|-841.75| +2452623|49064|17370|87767|1203116|2465|40167|7|293|712|21|29.36|45.50|39.13|0.00|821.73|616.56|955.50|57.52|0.00|821.73|879.25|205.17| +2452623|49064|13158|87767|1203116|2465|40167|7|15|712|54|78.16|117.24|101.99|0.00|5507.46|4220.64|6330.96|110.14|0.00|5507.46|5617.60|1286.82| +2452623|49064|14265|87767|1203116|2465|40167|7|26|712|46|79.35|144.41|121.30|0.00|5579.80|3650.10|6642.86|167.39|0.00|5579.80|5747.19|1929.70| +2452623|49064|11361|87767|1203116|2465|40167|7|54|712|52|91.69|120.11|103.29|0.00|5371.08|4767.88|6245.72|322.26|0.00|5371.08|5693.34|603.20| +2452623|49064|7596|87767|1203116|2465|40167|7|254|712|78|59.31|95.48|39.14|0.00|3052.92|4626.18|7447.44|213.70|0.00|3052.92|3266.62|-1573.26| +2452623|49064|16417|87767|1203116|2465|40167|7|96|712|32|66.82|101.56|72.10|1038.24|2307.20|2138.24|3249.92|76.13|1038.24|1268.96|1345.09|-869.28| +2452623|49064|9307|87767|1203116|2465|40167|7|218|712|25|98.75|141.21|11.29|0.00|282.25|2468.75|3530.25|2.82|0.00|282.25|285.07|-2186.50| +2452623|49064|1423|87767|1203116|2465|40167|7|162|712|70|5.54|9.91|5.54|0.00|387.80|387.80|693.70|0.00|0.00|387.80|387.80|0.00| +2452591|70640|12180|52004|1491443|129|24768|1|55|713|12|5.17|8.78|4.47|0.00|53.64|62.04|105.36|1.07|0.00|53.64|54.71|-8.40| +2452591|70640|888|52004|1491443|129|24768|1|40|713|8|26.69|44.57|41.00|0.00|328.00|213.52|356.56|13.12|0.00|328.00|341.12|114.48| +2452591|70640|10453|52004|1491443|129|24768|1|8|713|24|24.04|33.17|25.87|434.61|620.88|576.96|796.08|1.86|434.61|186.27|188.13|-390.69| +2452591|70640|8748|52004|1491443|129|24768|1|40|713|27|91.01|149.25|125.37|3114.19|3384.99|2457.27|4029.75|8.12|3114.19|270.80|278.92|-2186.47| +2452591|70640|11031|52004|1491443|129|24768|1|292|713|44|87.32|158.92|84.22|0.00|3705.68|3842.08|6992.48|185.28|0.00|3705.68|3890.96|-136.40| +2452591|70640|17922|52004|1491443|129|24768|1|86|713|31|22.59|25.07|20.55|0.00|637.05|700.29|777.17|0.00|0.00|637.05|637.05|-63.24| +2452591|70640|11457|52004|1491443|129|24768|1|201|713|58|18.65|19.02|1.90|0.00|110.20|1081.70|1103.16|3.30|0.00|110.20|113.50|-971.50| +2452591|70640|10597|52004|1491443|129|24768|1|277|713|33|40.39|73.91|53.21|0.00|1755.93|1332.87|2439.03|52.67|0.00|1755.93|1808.60|423.06| +2452591|70640|10971|52004|1491443|129|24768|1|91|713|22|44.64|58.47|17.54|0.00|385.88|982.08|1286.34|34.72|0.00|385.88|420.60|-596.20| +2452601||17239|47605||||2|297|714|55|82.63|87.58||0.00|2648.80|4544.65||105.95|0.00||2754.75|| +2452601|44505|3750|47605|1375961|1117|14273|2|8|714|54|81.51|110.03|41.81|0.00|2257.74|4401.54|5941.62|67.73|0.00|2257.74|2325.47|-2143.80| +2452601|44505|11593|47605|1375961|1117|14273|2|40|714|99|58.48|70.17|39.99|3246.38|3959.01|5789.52|6946.83|21.37|3246.38|712.63|734.00|-5076.89| +2452601|44505|1339|47605|1375961|1117|14273|2|132|714|86|22.65|33.29|17.64|0.00|1517.04|1947.90|2862.94|45.51|0.00|1517.04|1562.55|-430.86| +2452601|44505|14923|47605|1375961|1117|14273|2|31|714|22|68.22|112.56|108.05|0.00|2377.10|1500.84|2476.32|118.85|0.00|2377.10|2495.95|876.26| +2452601|44505|12205|47605|1375961|1117|14273|2|178|714|2|21.89|43.12|7.76|0.00|15.52|43.78|86.24|1.24|0.00|15.52|16.76|-28.26| +2452601|44505|12171|47605|1375961|1117|14273|2|33|714|9|74.03|133.99|68.33|0.00|614.97|666.27|1205.91|30.74|0.00|614.97|645.71|-51.30| +2452601|44505|5137|47605|1375961|1117|14273|2|99|714|32|39.82|41.01|2.05|0.00|65.60|1274.24|1312.32|4.59|0.00|65.60|70.19|-1208.64| +2452601|44505|3001|47605|1375961|1117|14273|2|9|714|3|26.27|31.78|13.66|0.00|40.98|78.81|95.34|1.63|0.00|40.98|42.61|-37.83| +2452601|44505|9703|47605|1375961|1117|14273|2|248|714|80|50.67|95.25|50.48|0.00|4038.40|4053.60|7620.00|282.68|0.00|4038.40|4321.08|-15.20| +2452601|44505|14136|47605|1375961|1117|14273|2|1|714|5|35.01|54.26|17.36|0.00|86.80|175.05|271.30|1.73|0.00|86.80|88.53|-88.25| +2452601|44505|17677|47605|1375961|1117|14273|2|126|714|32|27.62|45.84|32.54|0.00|1041.28|883.84|1466.88|93.71|0.00|1041.28|1134.99|157.44| +2452601||5808||1375961|1117|14273|||714|||17.88|9.47||359.86|543.78||||||-183.92| +2452601|44505|10233|47605|1375961|1117|14273|2|252|714|55|17.55|30.01|12.60|498.96|693.00|965.25|1650.55|11.64|498.96|194.04|205.68|-771.21| +2452601|44505|15145|47605|1375961|1117|14273|2|146|714|81|64.13|91.06|17.30|0.00|1401.30|5194.53|7375.86|98.09|0.00|1401.30|1499.39|-3793.23| +2452163|50173|17731|69133|732449|918|46234|1|274|715|1|29.08|39.54|34.39|0.00|34.39|29.08|39.54|3.09|0.00|34.39|37.48|5.31| +2452163|50173|8237|69133|732449|918|46234|1|49|715|85|90.98|136.47|84.61|0.00|7191.85|7733.30|11599.95|575.34|0.00|7191.85|7767.19|-541.45| +2452163|50173|4411|69133|732449|918|46234|1|129|715|71|19.03|34.82|24.37|0.00|1730.27|1351.13|2472.22|0.00|0.00|1730.27|1730.27|379.14| +2452163||9543|69133|732449|918||1|174|715||8.60||8.78|0.00|||||0.00||271.30|| +2452163|50173|6117|69133|732449|918|46234|1|254|715|76|61.22|104.07|34.34|0.00|2609.84|4652.72|7909.32|78.29|0.00|2609.84|2688.13|-2042.88| +2452163|50173|11479|69133|732449|918|46234|1|79|715|27|49.43|64.75|29.13|0.00|786.51|1334.61|1748.25|0.00|0.00|786.51|786.51|-548.10| +2452163|50173|14311|69133|732449|918|46234|1|179|715|99|76.69|141.87|112.07|0.00|11094.93|7592.31|14045.13|443.79|0.00|11094.93|11538.72|3502.62| +2452163|50173|8937|69133|732449|918|46234|1|184|715|99|52.84|103.56|4.14|0.00|409.86|5231.16|10252.44|8.19|0.00|409.86|418.05|-4821.30| +2452163|50173|12869|69133|732449|918|46234|1|33|715|7|74.63|145.52|42.20|0.00|295.40|522.41|1018.64|20.67|0.00|295.40|316.07|-227.01| +2451870|49296|356|33157|1819007|6010|36213|8|241|716|44|14.50|20.30|3.85|0.00|169.40|638.00|893.20|13.55|0.00|169.40|182.95|-468.60| +2451870|49296|2059|33157|1819007|6010|36213|8|21|716|41|95.67|168.37|28.62|46.93|1173.42|3922.47|6903.17|56.32|46.93|1126.49|1182.81|-2795.98| +2451870|49296|13676|33157|1819007|6010|36213|8|248|716|99|59.44|60.62|2.42|0.00|239.58|5884.56|6001.38|4.79|0.00|239.58|244.37|-5644.98| +2451870|49296|17591|33157|1819007|6010|36213|8|161|716|97|17.14|31.02|1.86|0.00|180.42|1662.58|3008.94|10.82|0.00|180.42|191.24|-1482.16| +2451870|49296|14288|33157|1819007|6010|36213|8|283|716|85|32.85|44.34|10.64|0.00|904.40|2792.25|3768.90|9.04|0.00|904.40|913.44|-1887.85| +2451870|49296|7757|33157|1819007|6010|36213|8|136|716|30|57.53|92.04|1.84|18.21|55.20|1725.90|2761.20|2.58|18.21|36.99|39.57|-1688.91| +2451870|49296|8240|33157|1819007|6010|36213|8|233|716|81|57.26|77.87|39.71|0.00|3216.51|4638.06|6307.47|0.00|0.00|3216.51|3216.51|-1421.55| +2451870|49296|10271|33157|1819007|6010|36213|8|280|716|32|51.28|86.15|37.04|0.00|1185.28|1640.96|2756.80|23.70|0.00|1185.28|1208.98|-455.68| +2451870|49296|11264|33157|1819007|6010|36213|8|275|716|86|33.29|39.61|27.72|0.00|2383.92|2862.94|3406.46|214.55|0.00|2383.92|2598.47|-479.02| +2451870|49296|4091|33157|1819007|6010|36213|8|15|716|20|52.37|71.22|71.22|0.00|1424.40|1047.40|1424.40|0.00|0.00|1424.40|1424.40|377.00| +2452547|38175|1009|497|1680749|1448|46900|10|251|717|2|95.92|134.28|45.65|0.00|91.30|191.84|268.56|0.91|0.00|91.30|92.21|-100.54| +2452547|38175|13692|497|1680749|1448|46900|10|248|717|34|2.73|4.17|2.75|22.44|93.50|92.82|141.78|2.84|22.44|71.06|73.90|-21.76| +2452547|38175|4213|497|1680749|1448|46900|10|220|717|33|43.07|67.18|34.93|0.00|1152.69|1421.31|2216.94|0.00|0.00|1152.69|1152.69|-268.62| +2452547|38175|12492|497|1680749|1448|46900|10|239|717|83|76.43|116.93|112.25|0.00|9316.75|6343.69|9705.19|279.50|0.00|9316.75|9596.25|2973.06| +2452547||15127|497||1448|||132|717|85|||58.79|0.00||4059.60||49.97|0.00||5047.12|| +2452547|38175|63|497|1680749|1448|46900|10|269|717|53|31.98|45.73|17.37|598.39|920.61|1694.94|2423.69|19.33|598.39|322.22|341.55|-1372.72| +||15297|497||1448|||112|717|||126.31|||3500.91|||||||-732.06| +2452547|38175|9433|497|1680749|1448|46900|10|227|717|5|61.55|89.86|57.51|57.51|287.55|307.75|449.30|11.50|57.51|230.04|241.54|-77.71| +2452547|38175|11451|497|1680749|1448|46900|10|130|717|51|47.58|85.16|28.10|0.00|1433.10|2426.58|4343.16|71.65|0.00|1433.10|1504.75|-993.48| +2452547|38175|14419|497|1680749|1448|46900|10|54|717|81|29.98|44.37|8.87|0.00|718.47|2428.38|3593.97|21.55|0.00|718.47|740.02|-1709.91| +2452547||4821||||46900||37|717|36||125.04|||2115.36|2344.68|4501.44|||2115.36|2157.66|-229.32| +2452547|38175|1513|497|1680749|1448|46900|10|298|717|12|37.88|51.51|49.44|35.59|593.28|454.56|618.12|11.15|35.59|557.69|568.84|103.13| +2452547|38175|17787|497|1680749|1448|46900|10|92|717|68|26.11|28.98|15.93|0.00|1083.24|1775.48|1970.64|43.32|0.00|1083.24|1126.56|-692.24| +2452547|38175|10479|497|1680749|1448|46900|10|181|717|94|54.19|61.23|17.14|0.00|1611.16|5093.86|5755.62|112.78|0.00|1611.16|1723.94|-3482.70| +2451488|68120|2194|75283|921829|3222|45368|7|210|718|16|66.63|117.26|24.62|0.00|393.92|1066.08|1876.16|27.57|0.00|393.92|421.49|-672.16| +2451488|68120|13093||||45368|7|23|718|20||36.16||146.64|282.00|463.60|||146.64||142.12|-328.24| +2451488|68120|17840|75283|921829|3222|45368|7|58|718|36|18.19|27.83|25.32|0.00|911.52|654.84|1001.88|63.80|0.00|911.52|975.32|256.68| +2451488|68120|460|75283|921829|3222|45368|7|220|718|57|93.23|145.43|31.99|0.00|1823.43|5314.11|8289.51|109.40|0.00|1823.43|1932.83|-3490.68| +2451488|68120|3560|75283|921829|3222|45368|7|87|718|40|72.55|94.31|45.26|0.00|1810.40|2902.00|3772.40|90.52|0.00|1810.40|1900.92|-1091.60| +2451488|68120|103|75283|921829|3222|45368|7|24|718|83|58.32|90.39|75.02|0.00|6226.66|4840.56|7502.37|498.13|0.00|6226.66|6724.79|1386.10| +2451488|68120|5674|75283|921829|3222|45368|7|274|718|66|18.84|19.02|0.95|0.00|62.70|1243.44|1255.32|1.88|0.00|62.70|64.58|-1180.74| +|68120|10765|75283||3222|45368|7|283|718||97.91|101.82|||420.08|||||420.08||-5356.61| +2451488|68120|8056|75283|921829|3222|45368|7|239|718|29|85.19|88.59|47.83|0.00|1387.07|2470.51|2569.11|69.35|0.00|1387.07|1456.42|-1083.44| +2451488|68120|1634|75283|921829|3222|45368|7|275|718|34|82.67|118.21|85.11|0.00|2893.74|2810.78|4019.14|0.00|0.00|2893.74|2893.74|82.96| +2452261|69067|1363|57624|1832348|1472|14596|10|166|719|37|67.53|76.30|66.38|0.00|2456.06|2498.61|2823.10|73.68|0.00|2456.06|2529.74|-42.55| +2452261|69067|12069|57624|1832348|1472|14596|10|215|719|48|28.73|53.43|37.93|0.00|1820.64|1379.04|2564.64|127.44|0.00|1820.64|1948.08|441.60| +2452261|69067|13341|57624|1832348|1472|14596|10|200|719|25|95.78|153.24|111.86|0.00|2796.50|2394.50|3831.00|55.93|0.00|2796.50|2852.43|402.00| +2452261|69067|2931|57624|1832348|1472|14596|10|26|719|90|30.65|33.40|8.01|0.00|720.90|2758.50|3006.00|7.20|0.00|720.90|728.10|-2037.60| +2452261|69067|15783|57624|1832348|1472|14596|10|51|719|45|13.32|15.58|1.24|0.00|55.80|599.40|701.10|0.55|0.00|55.80|56.35|-543.60| +2452261|69067|21|57624|1832348|1472|14596|10|203|719|37|53.37|59.24|35.54|0.00|1314.98|1974.69|2191.88|118.34|0.00|1314.98|1433.32|-659.71| +2452261|69067|15627|57624|1832348|1472|14596|10|139|719|77|20.48|29.49|20.93|0.00|1611.61|1576.96|2270.73|0.00|0.00|1611.61|1611.61|34.65| +2452261|69067|9459|57624|1832348|1472|14596|10|233|719|61|21.43|25.28|22.49|0.00|1371.89|1307.23|1542.08|123.47|0.00|1371.89|1495.36|64.66| +2452261|69067|10795|57624|1832348|1472|14596|10|258|719|9|7.77|10.87|7.39|0.00|66.51|69.93|97.83|0.00|0.00|66.51|66.51|-3.42| +2452261|69067|13693|57624|1832348|1472|14596|10|219|719|22|59.26|65.18|20.20|0.00|444.40|1303.72|1433.96|22.22|0.00|444.40|466.62|-859.32| +2452261|69067|12921|57624|1832348|1472|14596|10|270|719|12|73.03|125.61|95.46|57.27|1145.52|876.36|1507.32|21.76|57.27|1088.25|1110.01|211.89| +2452261|69067|4177|57624|1832348|1472|14596|10|108|719|51|68.55|76.77|42.22|0.00|2153.22|3496.05|3915.27|193.78|0.00|2153.22|2347.00|-1342.83| +2452261|69067|2361|57624|1832348|1472|14596|10|243|719|24|85.66|116.49|101.34|0.00|2432.16|2055.84|2795.76|97.28|0.00|2432.16|2529.44|376.32| +2452518|62784|4194|95979|434804|6876|46501|1|211|720|54|91.79|115.65|99.45|0.00|5370.30|4956.66|6245.10|268.51|0.00|5370.30|5638.81|413.64| +2452518||5652|95979||6876|46501|||720|68|69.04|102.86|90.51||6154.68|4694.72||108.32||2708.06|2816.38|| +2452518|62784|8137|95979|434804|6876|46501|1|61|720|10|20.79|40.54|31.62|0.00|316.20|207.90|405.40|0.00|0.00|316.20|316.20|108.30| +2452518|62784|4137|95979|434804|6876|46501|1|29|720|80|9.09|16.54|8.43|0.00|674.40|727.20|1323.20|0.00|0.00|674.40|674.40|-52.80| +2452518|62784|15597|95979|434804|6876|46501|1|20|720|15|54.00|98.28|23.58|0.00|353.70|810.00|1474.20|28.29|0.00|353.70|381.99|-456.30| +2452518|62784|13567|95979|434804|6876|46501|1|225|720|44|52.55|87.23|72.40|0.00|3185.60|2312.20|3838.12|191.13|0.00|3185.60|3376.73|873.40| +2452518|62784|9099|95979|434804|6876|46501|1|185|720|64|61.92|92.88|29.72|0.00|1902.08|3962.88|5944.32|38.04|0.00|1902.08|1940.12|-2060.80| +2452518|62784|12885|95979|434804|6876|46501|1|269|720|82|34.96|60.83|6.69|0.00|548.58|2866.72|4988.06|5.48|0.00|548.58|554.06|-2318.14| +2452518|62784|3703|95979|434804|6876|46501|1|277|720|66|62.81|88.56|61.10|2500.21|4032.60|4145.46|5844.96|107.26|2500.21|1532.39|1639.65|-2613.07| +2452518|62784|13188|95979|434804|6876|46501|1|278|720|46|57.41|111.94|19.02|0.00|874.92|2640.86|5149.24|61.24|0.00|874.92|936.16|-1765.94| +2452518||8187||434804||46501||31|720|10||12.57|5.65|||||0.40||40.68|41.08|-28.42| +2452518|62784|11007|95979|434804|6876|46501|1|109|720|54|72.29|111.32|37.84|0.00|2043.36|3903.66|6011.28|81.73|0.00|2043.36|2125.09|-1860.30| +2452518|62784|6591|95979|434804|6876|46501|1|198|720|39|68.94|119.26|77.51|0.00|3022.89|2688.66|4651.14|181.37|0.00|3022.89|3204.26|334.23| +2452518|62784|5610|95979|434804|6876|46501|1|241|720|42|15.19|16.25|1.95|0.00|81.90|637.98|682.50|4.09|0.00|81.90|85.99|-556.08| +2452518|62784|11460|95979|434804|6876|46501|1|66|720|81|44.12|84.26|47.18|0.00|3821.58|3573.72|6825.06|152.86|0.00|3821.58|3974.44|247.86| +2450822|44378|7210|60804|332475|6546|4264|1|190|721|98|91.74|176.14|112.72|2098.84|11046.56|8990.52|17261.72|805.29|2098.84|8947.72|9753.01|-42.80| +2450822|44378|17407|60804|332475|6546|4264|1|193|721|92|27.00|34.83|14.97|0.00|1377.24|2484.00|3204.36|55.08|0.00|1377.24|1432.32|-1106.76| +2450822|44378|4669|60804|332475|6546|4264|1|258|721|1|30.07|39.99|37.99|21.65|37.99|30.07|39.99|0.32|21.65|16.34|16.66|-13.73| +2450822|44378|16813|60804|332475|6546|4264|||721||63.46|102.17|||3065.00|6346.00||||3065.00|3310.20|-3281.00| +2450822|44378|13120|60804|332475|6546|4264|1|109|721|33|62.98|95.09|77.02|0.00|2541.66|2078.34|3137.97|101.66|0.00|2541.66|2643.32|463.32| +2450822|44378|6004|60804|332475|6546|4264|1|237|721|25|45.48|54.57|36.01|0.00|900.25|1137.00|1364.25|54.01|0.00|900.25|954.26|-236.75| +2450822|44378|17656|60804|332475|6546|4264|1|230|721|94|59.47|115.37|13.84|455.33|1300.96|5590.18|10844.78|67.65|455.33|845.63|913.28|-4744.55| +2450822|44378|4258|60804|332475|6546|4264|1|169|721|72|40.90|51.94|4.15|53.78|298.80|2944.80|3739.68|22.05|53.78|245.02|267.07|-2699.78| +2450822|44378|170|60804|332475|6546|4264|1|274|721|26|2.98|4.26|3.02|0.00|78.52|77.48|110.76|0.78|0.00|78.52|79.30|1.04| +2450822|44378|1070|60804|332475|6546|4264|1|124|721|54|11.82|23.64|19.14|0.00|1033.56|638.28|1276.56|31.00|0.00|1033.56|1064.56|395.28| +2450822|44378|15313|60804|332475|6546|4264|1|125|721|30|29.56|58.52|25.16|0.00|754.80|886.80|1755.60|0.00|0.00|754.80|754.80|-132.00| +2450822|44378|13945|60804|332475|6546|4264|1|110|721|40|37.21|48.00|41.28|0.00|1651.20|1488.40|1920.00|49.53|0.00|1651.20|1700.73|162.80| +2452216|48475|10559|49297|1474819|694|48969|8|230|722|74|67.76|106.38|40.42|0.00|2991.08|5014.24|7872.12|209.37|0.00|2991.08|3200.45|-2023.16| +2452216|48475|15257|49297|1474819|694|48969|8|164|722|12|34.36|66.65|5.33|59.48|63.96|412.32|799.80|0.31|59.48|4.48|4.79|-407.84| +2452216|48475|8787|49297|1474819|694|48969|8|36|722|62|50.21|62.26|53.54|663.89|3319.48|3113.02|3860.12|53.11|663.89|2655.59|2708.70|-457.43| +2452216|48475|4941|49297|1474819|694|48969|8|61|722|35|16.20|22.35|13.85|0.00|484.75|567.00|782.25|38.78|0.00|484.75|523.53|-82.25| +2452216|48475|15057|49297|1474819|694|48969|8|143|722|92|86.49|172.98|140.11|6445.06|12890.12|7957.08|15914.16|128.90|6445.06|6445.06|6573.96|-1512.02| +2452216|48475|6331|49297|1474819|694|48969|8|62|722|76|57.45|59.17|37.27|0.00|2832.52|4366.20|4496.92|198.27|0.00|2832.52|3030.79|-1533.68| +|48475|8263||1474819|694|48969|||722|75||||0.00|||5792.25||0.00|5792.25||2302.50| +2452216|48475|12107|49297|1474819|694|48969|8|256|722|70|27.83|48.98|33.30|0.00|2331.00|1948.10|3428.60|46.62|0.00|2331.00|2377.62|382.90| +2452216|48475|17013|49297|1474819|694|48969|8|174|722|35|77.77|87.10|33.96|879.56|1188.60|2721.95|3048.50|12.36|879.56|309.04|321.40|-2412.91| +2452216|48475|9569|49297|1474819|694|48969|8|179|722|69|92.10|165.78|56.36|0.00|3888.84|6354.90|11438.82|272.21|0.00|3888.84|4161.05|-2466.06| +2452216|48475|16125|49297|1474819|694|48969|8|237|722|39|43.94|83.04|68.92|322.54|2687.88|1713.66|3238.56|47.30|322.54|2365.34|2412.64|651.68| +2452216|48475|14437|49297|1474819|694|48969|8|237|722|79|73.93|138.24|17.97|0.00|1419.63|5840.47|10920.96|14.19|0.00|1419.63|1433.82|-4420.84| +2452216|48475|4311|49297|1474819|694|48969|8|100|722|47|87.00|163.56|19.62|0.00|922.14|4089.00|7687.32|46.10|0.00|922.14|968.24|-3166.86| +2450992|63807|13028|42235|1613064|3540|22369|4|93|723|72|86.98|157.43|135.38|0.00|9747.36|6262.56|11334.96|194.94|0.00|9747.36|9942.30|3484.80| +2450992|63807|15601|42235|1613064|3540|22369|4|154|723|1|21.03|24.39|20.73|0.00|20.73|21.03|24.39|1.65|0.00|20.73|22.38|-0.30| +2450992|63807|14851|42235|1613064|3540|22369|4|138|723|100|50.64|51.14|23.01|0.00|2301.00|5064.00|5114.00|161.07|0.00|2301.00|2462.07|-2763.00| +2450992|63807|7786|42235|1613064|3540|22369|4|160|723|1|81.82|139.91|76.95|0.00|76.95|81.82|139.91|1.53|0.00|76.95|78.48|-4.87| +2450992|63807|9868|42235|1613064|3540|22369|4|18|723|59|61.91|61.91|19.81|0.00|1168.79|3652.69|3652.69|46.75|0.00|1168.79|1215.54|-2483.90| +2450992|63807|7318|42235|1613064|3540|22369|4|164|723|32|4.13|4.21|1.05|0.00|33.60|132.16|134.72|1.00|0.00|33.60|34.60|-98.56| +2450992|63807|15091|42235|1613064|3540|22369|4|116|723|82|58.33|77.57|34.90|0.00|2861.80|4783.06|6360.74|57.23|0.00|2861.80|2919.03|-1921.26| +2450992|63807|12584|42235|1613064|3540|22369|4|51|723|46|31.88|55.79|39.05|0.00|1796.30|1466.48|2566.34|35.92|0.00|1796.30|1832.22|329.82| +2450992|63807|17458|42235|1613064|3540|22369|4|158|723|62|59.20|98.27|26.53|773.08|1644.86|3670.40|6092.74|8.71|773.08|871.78|880.49|-2798.62| +2450992|63807|8659|42235|1613064|3540|22369|4|199|723|88|81.81|110.44|34.23|0.00|3012.24|7199.28|9718.72|120.48|0.00|3012.24|3132.72|-4187.04| +2450992|63807|8576|42235|1613064|3540|22369|4|206|723|74|17.74|33.52|12.40|220.22|917.60|1312.76|2480.48|20.92|220.22|697.38|718.30|-615.38| +2450992|63807|8440|42235|1613064|3540|22369|4|61|723|67|35.72|71.08|51.88|0.00|3475.96|2393.24|4762.36|312.83|0.00|3475.96|3788.79|1082.72| +2450992|63807|6662|42235|1613064|3540|22369|4|157|723|77|85.83|127.02|81.29|0.00|6259.33|6608.91|9780.54|250.37|0.00|6259.33|6509.70|-349.58| +2451711|50425|4351|16166|504370|3421|27225|4|33|724|16|9.74|15.09|4.67|0.00|74.72|155.84|241.44|1.49|0.00|74.72|76.21|-81.12| +2451711|50425|9446|16166|504370|3421|27225|4|49|724|30|55.52|106.04|59.38|0.00|1781.40|1665.60|3181.20|89.07|0.00|1781.40|1870.47|115.80| +2451711|50425|10123|16166|504370|3421|27225|4|246|724|81|92.69|185.38|63.02|0.00|5104.62|7507.89|15015.78|51.04|0.00|5104.62|5155.66|-2403.27| +2451711|50425|3589|16166|504370|3421|27225|4|206|724|89|91.71|152.23|124.82|5443.40|11108.98|8162.19|13548.47|283.27|5443.40|5665.58|5948.85|-2496.61| +2451711|50425|2705|16166|504370|3421|27225|4|198|724|20|65.61|74.13|15.56|0.00|311.20|1312.20|1482.60|6.22|0.00|311.20|317.42|-1001.00| +2451711|50425|15355|16166|504370|3421|27225|4|48|724|55|19.79|27.90|25.66|0.00|1411.30|1088.45|1534.50|70.56|0.00|1411.30|1481.86|322.85| +2451711|50425|3548|16166|504370|3421|27225|4|210|724|31|95.39|188.87|5.66|0.00|175.46|2957.09|5854.97|14.03|0.00|175.46|189.49|-2781.63| +2451711|50425|12953|16166|504370|3421|27225|4|222|724|69|27.91|46.88|25.31|0.00|1746.39|1925.79|3234.72|52.39|0.00|1746.39|1798.78|-179.40| +2451711|50425|5192|16166|504370|3421|27225|4|191|724|60|16.50|27.06|19.75|0.00|1185.00|990.00|1623.60|0.00|0.00|1185.00|1185.00|195.00| +2451711|50425|11594|16166|504370|3421|27225|4|129|724|11|75.26|110.63|68.59|0.00|754.49|827.86|1216.93|45.26|0.00|754.49|799.75|-73.37| +2450964|57827|17851|40338|495045|5305|26012|2|38|725|88|54.34|55.42|39.34|0.00|3461.92|4781.92|4876.96|276.95|0.00|3461.92|3738.87|-1320.00| +2450964|57827|9109|40338|495045|5305|26012|2|58|725|81|92.91|123.57|51.89|0.00|4203.09|7525.71|10009.17|210.15|0.00|4203.09|4413.24|-3322.62| +2450964|57827|12187|40338|495045|5305|26012|2|28|725|45|98.82|141.31|137.07|0.00|6168.15|4446.90|6358.95|308.40|0.00|6168.15|6476.55|1721.25| +2450964|57827|4894|40338|495045|5305|26012|2|77|725|58|73.10|128.65|37.30|0.00|2163.40|4239.80|7461.70|129.80|0.00|2163.40|2293.20|-2076.40| +2450964|57827|8858|40338|495045|5305|26012|2|174|725|12|68.57|122.05|1.22|5.12|14.64|822.84|1464.60|0.57|5.12|9.52|10.09|-813.32| +2450964|57827|6049|40338|495045|5305|26012|2|104|725|75|96.60|118.81|81.97|3811.60|6147.75|7245.00|8910.75|163.53|3811.60|2336.15|2499.68|-4908.85| +2450964|57827|14546|40338|495045|5305|26012|2|244|725|24|48.26|80.59|63.66|0.00|1527.84|1158.24|1934.16|122.22|0.00|1527.84|1650.06|369.60| +2450964|57827|6902|40338|495045|5305|26012|2|55|725|93|96.38|157.09|141.38|0.00|13148.34|8963.34|14609.37|1183.35|0.00|13148.34|14331.69|4185.00| +2450964|57827|8012|40338|495045|5305|26012|2|91|725|75|14.38|15.53|6.83|0.00|512.25|1078.50|1164.75|10.24|0.00|512.25|522.49|-566.25| +2450964|57827|6866|40338|495045|5305|26012|2|275|725|40|76.17|97.49|20.47|0.00|818.80|3046.80|3899.60|65.50|0.00|818.80|884.30|-2228.00| +2450964|57827|1706|40338|495045|5305|26012|2|10|725|71|85.93|163.26|88.16|0.00|6259.36|6101.03|11591.46|500.74|0.00|6259.36|6760.10|158.33| +2450964|57827|12910|40338|495045|5305|26012|2|102|725|70|6.41|7.75|0.46|0.00|32.20|448.70|542.50|0.32|0.00|32.20|32.52|-416.50| +2450964|57827|5335|40338|495045|5305|26012|2|62|725|88|57.82|111.01|108.78|0.00|9572.64|5088.16|9768.88|765.81|0.00|9572.64|10338.45|4484.48| +2451839|46276|5276|18239|1865411|3430|8649|7|33|726|3|64.37|75.95|63.03|0.00|189.09|193.11|227.85|9.45|0.00|189.09|198.54|-4.02| +2451839|46276|16346|18239|1865411|3430|8649|7|109|726|43|92.52|178.56|151.77|0.00|6526.11|3978.36|7678.08|391.56|0.00|6526.11|6917.67|2547.75| +2451839|46276|239|18239|1865411|3430|8649|7|208|726|76|41.32|46.27|35.16|0.00|2672.16|3140.32|3516.52|53.44|0.00|2672.16|2725.60|-468.16| +2451839|46276|6847|18239|1865411|3430|8649|7|74|726|70|36.84|46.41|46.41|0.00|3248.70|2578.80|3248.70|0.00|0.00|3248.70|3248.70|669.90| +|46276|9659||1865411||8649|7|23|726|46|79.06||40.00|220.80||3636.76|4181.86|129.53|220.80|||-2017.56| +2451839|46276|14329|18239|1865411|3430|8649|7|30|726|95|51.90|54.49|51.22|4525.28|4865.90|4930.50|5176.55|13.62|4525.28|340.62|354.24|-4589.88| +2451839|46276|11471|18239|1865411|3430|8649|7|9|726|47|17.13|26.89|10.75|0.00|505.25|805.11|1263.83|15.15|0.00|505.25|520.40|-299.86| +2451839|46276|215|18239|1865411|3430|8649|7|72|726|69|33.84|34.51|23.81|854.30|1642.89|2334.96|2381.19|0.00|854.30|788.59|788.59|-1546.37| +2451839|46276|5977|18239|1865411|3430|8649|7|116|726|55|52.88|74.56|2.98|0.00|163.90|2908.40|4100.80|6.55|0.00|163.90|170.45|-2744.50| +2451839|46276|15967|18239|1865411|3430|8649|7|231|726|74|9.46|13.90|10.70|0.00|791.80|700.04|1028.60|39.59|0.00|791.80|831.39|91.76| +2450961|43867|17365|11580|15211|3642|49673|2|170|727|68|23.22|28.09|4.21|0.00|286.28|1578.96|1910.12|5.72|0.00|286.28|292.00|-1292.68| +2450961|43867|9704|11580|15211|3642|49673|2|218|727|97|98.75|192.56|159.82|14417.36|15502.54|9578.75|18678.32|21.70|14417.36|1085.18|1106.88|-8493.57| +2450961|43867|752|11580|15211|3642|49673|2|254|727|90|42.34|55.46|26.62|0.00|2395.80|3810.60|4991.40|143.74|0.00|2395.80|2539.54|-1414.80| +2450961|43867|409|11580|15211|3642|49673|2|80|727|58|59.91|62.90|52.83|0.00|3064.14|3474.78|3648.20|0.00|0.00|3064.14|3064.14|-410.64| +2450961|43867|10166|11580|15211|3642|49673|2|98|727|34|19.77|28.46|14.23|0.00|483.82|672.18|967.64|24.19|0.00|483.82|508.01|-188.36| +2450961|43867|4448|11580|15211|3642|49673|2|80|727|93|78.54|142.15|100.92|0.00|9385.56|7304.22|13219.95|0.00|0.00|9385.56|9385.56|2081.34| +2450961|43867|2446|11580|15211|3642|49673|2|284|727|4|35.39|53.43|39.00|0.00|156.00|141.56|213.72|4.68|0.00|156.00|160.68|14.44| +2450961|43867|10754|11580|15211|3642|49673|2|163|727|78|85.50|138.51|56.78|0.00|4428.84|6669.00|10803.78|177.15|0.00|4428.84|4605.99|-2240.16| +2450961|43867|6764|11580|15211|3642|49673|2|46|727|38|5.89|8.01|4.40|0.00|167.20|223.82|304.38|13.37|0.00|167.20|180.57|-56.62| +2450961|43867|4222|11580|15211|3642|49673|2|284|727|71|9.94|11.33|10.53|0.00|747.63|705.74|804.43|59.81|0.00|747.63|807.44|41.89| +2450961|43867|3908|11580|15211|3642|49673|2|223|727|4|36.03|70.61|7.76|0.00|31.04|144.12|282.44|1.55|0.00|31.04|32.59|-113.08| +2450961|43867|8764|11580|15211|3642|49673|2|42|727|75|94.10|151.50|107.56|0.00|8067.00|7057.50|11362.50|161.34|0.00|8067.00|8228.34|1009.50| +2450961|43867|764|11580|15211|3642|49673|2|95|727|51|22.99|31.72|29.18|0.00|1488.18|1172.49|1617.72|29.76|0.00|1488.18|1517.94|315.69| +2450961|43867|3292|11580|15211|3642|49673|2|93|727|80|50.16|79.25|15.85|0.00|1268.00|4012.80|6340.00|12.68|0.00|1268.00|1280.68|-2744.80| +2451079|37943|1069|37234|862503|1819|48335|8|208|728|84|75.69|129.42|119.06|0.00|10001.04|6357.96|10871.28|200.02|0.00|10001.04|10201.06|3643.08| +2451079|37943|1045|37234|862503|1819|48335|8|143|728|83|14.04|22.74|19.10|729.23|1585.30|1165.32|1887.42|68.48|729.23|856.07|924.55|-309.25| +2451079|37943|12344|37234|862503|1819|48335|8|189|728|22|5.05|6.61|3.96|0.00|87.12|111.10|145.42|7.84|0.00|87.12|94.96|-23.98| +||17047|37234|862503|1819|48335|8|256|728|10|46.26|||0.00|||||0.00|||-120.30| +2451079|37943|5786||862503|1819|48335||155|728||64.63|100.82|55.45|0.00|3992.40|||239.54|0.00|3992.40|4231.94|| +2451079|37943|814|37234|862503|1819|48335|8|295|728|9|21.40|38.94|35.43|0.00|318.87|192.60|350.46|0.00|0.00|318.87|318.87|126.27| +2451079|37943|628|37234|862503|1819|48335|8|249|728|93|26.22|34.87|33.12|0.00|3080.16|2438.46|3242.91|61.60|0.00|3080.16|3141.76|641.70| +2451079|37943|908|37234|862503|1819|48335|8|278|728|30|80.11|124.17|73.26|879.12|2197.80|2403.30|3725.10|92.30|879.12|1318.68|1410.98|-1084.62| +2451079|37943|6973|37234|862503|1819|48335|8|59|728|29|55.20|85.00|66.30|0.00|1922.70|1600.80|2465.00|115.36|0.00|1922.70|2038.06|321.90| +2451079|37943|2528|37234|862503|1819|48335|8|97|728|32|34.32|38.78|9.69|0.00|310.08|1098.24|1240.96|6.20|0.00|310.08|316.28|-788.16| +2451079||1348|37234|862503||48335||216|728||89.78|163.39||0.00||1526.26|||0.00|666.57||-859.69| +2451079|37943|13336|37234|862503|1819|48335|8|132|728|42|36.40|64.42|61.19|0.00|2569.98|1528.80|2705.64|128.49|0.00|2569.98|2698.47|1041.18| +2451324|63135|17824|81478|1474542|2453|47164|8|30|729|46|3.27|5.19|0.72|0.00|33.12|150.42|238.74|2.64|0.00|33.12|35.76|-117.30| +2451324|63135|7022|81478|1474542|2453|47164|8|188|729|50|16.15|17.11|15.91|0.00|795.50|807.50|855.50|0.00|0.00|795.50|795.50|-12.00| +2451324|63135|17065|81478|1474542|2453|47164|8|101|729|21|49.95|67.43|17.53|0.00|368.13|1048.95|1416.03|14.72|0.00|368.13|382.85|-680.82| +2451324|63135|17473|81478|1474542|2453|47164|8|224|729|48|84.22|150.75|66.33|0.00|3183.84|4042.56|7236.00|286.54|0.00|3183.84|3470.38|-858.72| +2451324|63135|2269|81478|1474542|2453|47164|8|164|729|60|42.37|64.40|16.74|0.00|1004.40|2542.20|3864.00|10.04|0.00|1004.40|1014.44|-1537.80| +2451324|63135|11276|81478|1474542|2453|47164|8|92|729|43|99.88|160.80|32.16|0.00|1382.88|4294.84|6914.40|110.63|0.00|1382.88|1493.51|-2911.96| +||2096|||2453||8||729||20.48||10.72|0.00|943.36|1802.24|2360.16||0.00|943.36||-858.88| +2451324||9457|81478||||8||729|51||40.99|15.16|0.00|773.16|||69.58|0.00||842.74|-277.44| +2451324|63135|15278|81478|1474542|2453|47164|8|173|729|42|49.97|86.44|31.11|0.00|1306.62|2098.74|3630.48|26.13|0.00|1306.62|1332.75|-792.12| +2452638|54936|14413|15002|1515087|6372|48904|8|84|730|26|38.47|49.62|9.92|0.00|257.92|1000.22|1290.12|23.21|0.00|257.92|281.13|-742.30| +2452638|54936|10629|15002|1515087|6372|48904|8|203|730|16|68.63|121.47|115.39|0.00|1846.24|1098.08|1943.52|36.92|0.00|1846.24|1883.16|748.16| +2452638|54936|15165|15002|1515087|6372|48904|8|77|730|71|88.92|117.37|116.19|0.00|8249.49|6313.32|8333.27|577.46|0.00|8249.49|8826.95|1936.17| +2452638|54936|17107|15002|1515087|6372|48904|8|67|730|40|52.77|88.65|7.97|156.21|318.80|2110.80|3546.00|4.87|156.21|162.59|167.46|-1948.21| +2452638|54936|13107|15002|1515087|6372|48904|8|168|730|22|46.87|57.18|25.73|0.00|566.06|1031.14|1257.96|50.94|0.00|566.06|617.00|-465.08| +2452638|54936|8304|15002|1515087|6372|48904|8|253|730|99|80.90|147.23|35.33|0.00|3497.67|8009.10|14575.77|139.90|0.00|3497.67|3637.57|-4511.43| +2452638|54936|10014|15002|1515087|6372|48904|8|110|730|9|45.46|50.46|28.76|0.00|258.84|409.14|454.14|2.58|0.00|258.84|261.42|-150.30| +2452638|54936|15847|15002|1515087|6372|48904|8|50|730|74|67.90|98.45|12.79|0.00|946.46|5024.60|7285.30|0.00|0.00|946.46|946.46|-4078.14| +2452638|54936|777|15002|1515087|6372|48904|8|222|730|76|5.02|9.63|2.40|0.00|182.40|381.52|731.88|10.94|0.00|182.40|193.34|-199.12| +2452638|54936|1491|15002||||8||730|97|||92.77|0.00||5687.11||539.92|0.00|||3311.58| +2452638|54936|8113|15002|1515087|6372|48904|8|171|730|99|92.75|173.44|156.09|0.00|15452.91|9182.25|17170.56|463.58|0.00|15452.91|15916.49|6270.66| +||1026|15002|1515087||48904|||730|90|39.21|73.32||0.00|197.10|||17.73|0.00|||-3331.80| +2452638|54936|14406|15002|1515087|6372|48904|8|31|730|65|22.42|30.71|1.53|68.62|99.45|1457.30|1996.15|0.92|68.62|30.83|31.75|-1426.47| +2452638|54936|6783|15002|1515087|6372|48904|8|145|730|12|27.40|33.15|31.16|0.00|373.92|328.80|397.80|18.69|0.00|373.92|392.61|45.12| +2452545||9792||725387|||||731||||12.03||673.68|851.20|||||346.00|| +2452545||4146||725387||||174|731|35|||35.05|||1281.00|2011.10||||1263.55|| +2452545|46716|8887|41020|725387|4954|35321|10|205|731|95|50.99|80.56|48.33|734.61|4591.35|4844.05|7653.20|231.40|734.61|3856.74|4088.14|-987.31| +|46716|6009|41020||4954||10|24|731|20|7.82|12.12|9.81|||156.40|242.40|||196.20||| +2452545|46716|457|41020|725387|4954|35321|10|272|731|86|76.99|111.63|6.69|0.00|575.34|6621.14|9600.18|28.76|0.00|575.34|604.10|-6045.80| +2452545|46716|11169|41020|725387|4954|35321|10|201|731|65|78.80|155.23|144.36|0.00|9383.40|5122.00|10089.95|656.83|0.00|9383.40|10040.23|4261.40| +2452545|46716|9769|41020|725387|4954|35321|10|63|731|99|12.53|20.17|10.08|29.93|997.92|1240.47|1996.83|48.39|29.93|967.99|1016.38|-272.48| +2452545|46716|7506|41020|725387|4954|35321|10|118|731|39|21.03|33.85|17.60|0.00|686.40|820.17|1320.15|13.72|0.00|686.40|700.12|-133.77| +2452545|46716|1890|41020|725387|4954|35321|10|259|731|54|26.22|42.73|30.33|0.00|1637.82|1415.88|2307.42|114.64|0.00|1637.82|1752.46|221.94| +2452545|46716|16227|41020|725387|4954|35321|10|220|731|76|96.54|177.63|165.19|4519.59|12554.44|7337.04|13499.88|562.43|4519.59|8034.85|8597.28|697.81| +2452545|46716|11922|41020|725387|4954|35321|10|207|731|89|11.09|17.63|13.04|0.00|1160.56|987.01|1569.07|46.42|0.00|1160.56|1206.98|173.55| +2450929|69609|1988|44455|1317061|4867|11073|8|85|732|43|99.89|198.78|101.37|4315.32|4358.91|4295.27|8547.54|0.43|4315.32|43.59|44.02|-4251.68| +2450929|69609|9679|44455|1317061|4867|11073|8|253|732|10|89.50|99.34|69.53|556.24|695.30|895.00|993.40|2.78|556.24|139.06|141.84|-755.94| +2450929|69609|652|44455|1317061|4867|11073|8|58|732|63|49.05|91.23|89.40|0.00|5632.20|3090.15|5747.49|506.89|0.00|5632.20|6139.09|2542.05| +2450929|69609|8906|44455|1317061|4867|11073|8|282|732|67|98.99|160.36|62.54|0.00|4190.18|6632.33|10744.12|251.41|0.00|4190.18|4441.59|-2442.15| +2450929|69609|11830|44455|1317061|4867|11073|8|149|732|56|14.00|23.94|3.35|0.00|187.60|784.00|1340.64|1.87|0.00|187.60|189.47|-596.40| +2450929|69609|17156|44455|1317061|4867|11073|8|226|732|48|64.48|74.79|4.48|0.00|215.04|3095.04|3589.92|10.75|0.00|215.04|225.79|-2880.00| +2450929|69609|8960|44455|1317061|4867|11073|8|215|732|77|89.39|101.01|11.11|0.00|855.47|6883.03|7777.77|76.99|0.00|855.47|932.46|-6027.56| +2450929|69609|7435|44455|1317061|4867|11073|8|267|732|51|23.86|27.20|2.99|0.00|152.49|1216.86|1387.20|12.19|0.00|152.49|164.68|-1064.37| +2450929|69609|3847|44455|1317061|4867|11073|8|87|732|1|12.10|24.07|5.05|0.00|5.05|12.10|24.07|0.10|0.00|5.05|5.15|-7.05| +2450929|69609|5737|44455|1317061|4867|11073|8|28|732|56|31.26|61.58|6.77|0.00|379.12|1750.56|3448.48|30.32|0.00|379.12|409.44|-1371.44| +2450929|69609|12865|44455|1317061|4867|11073|8|52|732|89|68.25|124.89|63.69|0.00|5668.41|6074.25|11115.21|340.10|0.00|5668.41|6008.51|-405.84| +2450929|69609|3271|44455|1317061|4867|11073|8|281|732|82|87.25|123.89|95.39|0.00|7821.98|7154.50|10158.98|156.43|0.00|7821.98|7978.41|667.48| +2450929|69609|3541|44455|1317061|4867|11073|8|271|732|87|75.00|149.25|138.80|11471.82|12075.60|6525.00|12984.75|48.30|11471.82|603.78|652.08|-5921.22| +2450929|69609|17038|44455|1317061|4867|11073|8|1|732|36|47.28|52.00|10.92|326.28|393.12|1702.08|1872.00|1.33|326.28|66.84|68.17|-1635.24| +2450929|69609|11500|44455|1317061|4867|11073|8|186|732|46|66.26|127.88|20.46|545.87|941.16|3047.96|5882.48|27.67|545.87|395.29|422.96|-2652.67| +2450929|69609|6826|44455|1317061|4867|11073|8|184|732|94|95.53|160.49|86.66|3258.41|8146.04|8979.82|15086.06|293.25|3258.41|4887.63|5180.88|-4092.19| +2452400||13857||||39968||133|733|100|36.46|45.21|4.97|||||||||| +2452400|72941|2581|16772|178653|5334|39968|1|248|733|50|82.32|121.01|0.00|0.00|0.00|4116.00|6050.50|0.00|0.00|0.00|0.00|-4116.00| +2452400|72941|13782|16772|178653|5334|39968|1|148|733|54|6.97|13.24|8.20|0.00|442.80|376.38|714.96|26.56|0.00|442.80|469.36|66.42| +2452400|72941|5427|16772|178653|5334|39968|1|228|733|56|47.18|62.74|39.52|0.00|2213.12|2642.08|3513.44|22.13|0.00|2213.12|2235.25|-428.96| +2452400|72941|5191|16772|178653|5334|39968|1|94|733|36|34.41|42.66|5.97|0.00|214.92|1238.76|1535.76|2.14|0.00|214.92|217.06|-1023.84| +2452400|72941|9135|16772|178653|5334|39968|1|118|733|6|91.50|153.72|115.29|0.00|691.74|549.00|922.32|34.58|0.00|691.74|726.32|142.74| +||10944|||||1|237|733|67|25.46|46.33|33.35|||1705.82|3104.11|44.68|||2279.13|| +||11677|16772|||39968|||733|24||||0.00|||1541.28|122.05|0.00|1525.68|1647.73|| +2452400|72941|3705|16772|178653|5334|39968|1|121|733|79|89.50|104.71|56.54|0.00|4466.66|7070.50|8272.09|44.66|0.00|4466.66|4511.32|-2603.84| +2452400|72941|17136|16772|178653|5334|39968|1|6|733|93|72.03|131.81|52.72|0.00|4902.96|6698.79|12258.33|294.17|0.00|4902.96|5197.13|-1795.83| +2452400|72941|8143|16772|178653|5334|39968|1|280|733|10|26.43|46.51|0.93|6.60|9.30|264.30|465.10|0.00|6.60|2.70|2.70|-261.60| +2452400|72941|3739|16772|178653|5334|39968|1|224|733|23|54.81|94.27|32.99|0.00|758.77|1260.63|2168.21|37.93|0.00|758.77|796.70|-501.86| +2451995|39039|10769|82837|73182|2374|27119|2|203|734|51|75.74|104.52|43.89|0.00|2238.39|3862.74|5330.52|44.76|0.00|2238.39|2283.15|-1624.35| +2451995|39039|14701|82837|73182|2374|27119|2|207|734|53|35.92|40.23|8.44|0.00|447.32|1903.76|2132.19|17.89|0.00|447.32|465.21|-1456.44| +2451995|39039|17421|82837|73182|2374|27119|2|54|734|31|86.65|97.91|46.01|0.00|1426.31|2686.15|3035.21|99.84|0.00|1426.31|1526.15|-1259.84| +2451995|39039|9437|82837|73182|2374|27119|2|257|734|4|70.63|124.30|88.25|0.00|353.00|282.52|497.20|21.18|0.00|353.00|374.18|70.48| +2451995|39039|7655|82837|73182|2374|27119|2|201|734|97|92.66|154.74|57.25|0.00|5553.25|8988.02|15009.78|444.26|0.00|5553.25|5997.51|-3434.77| +2451995|39039|12223|82837|73182|2374|27119|2|173|734|84|79.64|84.41|35.45|0.00|2977.80|6689.76|7090.44|148.89|0.00|2977.80|3126.69|-3711.96| +2451995|39039|15085|82837|73182|2374|27119|2|151|734|42|73.57|89.75|52.95|600.45|2223.90|3089.94|3769.50|146.11|600.45|1623.45|1769.56|-1466.49| +||3741|||2374|27119|||734||43.08|||||4308.00||||2056.00||-2252.00| +2451995|39039|1363|82837|73182|2374|27119|2|53|734|84|67.01|86.44|30.25|0.00|2541.00|5628.84|7260.96|0.00|0.00|2541.00|2541.00|-3087.84| +2451995|39039|12069|82837|73182|2374|27119|2|236|734|18|55.27|100.59|4.02|0.00|72.36|994.86|1810.62|5.78|0.00|72.36|78.14|-922.50| +2451995|39039|13341|82837|73182|2374|27119|2|70|734|37|7.83|9.39|2.34|7.79|86.58|289.71|347.43|1.57|7.79|78.79|80.36|-210.92| +2451995|39039|2931|82837|73182|2374|27119|2|189|734|76|73.76|132.03|106.94|0.00|8127.44|5605.76|10034.28|81.27|0.00|8127.44|8208.71|2521.68| +2451995|39039|15783|82837|73182|2374|27119|2|30|734|49|13.45|25.55|4.59|0.00|224.91|659.05|1251.95|6.74|0.00|224.91|231.65|-434.14| +2451995|39039|21|82837|73182|2374|27119|2|193|734|53|91.36|96.84|8.71|36.93|461.63|4842.08|5132.52|38.22|36.93|424.70|462.92|-4417.38| +2451861|67432|10765|5852|1027207|4915|16623|1|108|735|82|48.52|95.58|0.00|0.00|0.00|3978.64|7837.56|0.00|0.00|0.00|0.00|-3978.64| +2451861|67432|8057|5852|1027207|4915|16623|1|162|735|29|56.28|86.10|40.46|0.00|1173.34|1632.12|2496.90|11.73|0.00|1173.34|1185.07|-458.78| +2451861||1634|||||||735||75.62|85.45|61.52|||||61.52||||-1410.00| +2451861|67432|9830|5852|1027207|4915|16623|1|25|735|7|76.47|103.23|11.35|0.00|79.45|535.29|722.61|1.58|0.00|79.45|81.03|-455.84| +|67432|2717||||16623|||735|13|30.31|57.89||0.00|30.03|394.03||1.50|0.00|30.03||-364.00| +2451861|67432|13465|5852|1027207|4915|16623|1|171|735|15|81.10|98.13|51.02|344.38|765.30|1216.50|1471.95|21.04|344.38|420.92|441.96|-795.58| +2451861|67432|8051|5852|1027207|4915|16623|1|236|735|28|73.58|90.50|19.91|0.00|557.48|2060.24|2534.00|5.57|0.00|557.48|563.05|-1502.76| +2451861|67432|7223|5852|1027207|4915|16623|1|37|735|70|60.53|100.47|50.23|0.00|3516.10|4237.10|7032.90|0.00|0.00|3516.10|3516.10|-721.00| +2451861|67432|3854|5852|1027207|4915|16623|1|30|735|70|9.11|10.65|1.06|31.90|74.20|637.70|745.50|3.38|31.90|42.30|45.68|-595.40| +2451861|67432|47|5852|1027207|4915|16623|1|21|735|59|33.70|67.06|22.80|0.00|1345.20|1988.30|3956.54|13.45|0.00|1345.20|1358.65|-643.10| +2451861|67432|6709|5852|1027207|4915|16623|1|259|735|56|67.06|77.11|16.96|0.00|949.76|3755.36|4318.16|75.98|0.00|949.76|1025.74|-2805.60| +2451076|64382|15452|44751|1655425|6099|19526|7|196|736|22|35.40|61.95|50.79|0.00|1117.38|778.80|1362.90|67.04|0.00|1117.38|1184.42|338.58| +2451076|64382|14774|44751|1655425|6099|19526|7|143|736|14|17.92|25.26|1.01|0.00|14.14|250.88|353.64|0.98|0.00|14.14|15.12|-236.74| +2451076|64382|1855|44751|1655425|6099|19526|7|220|736|42|96.87|133.68|4.01|0.00|168.42|4068.54|5614.56|6.73|0.00|168.42|175.15|-3900.12| +2451076|64382|5323|44751|1655425|6099|19526|7|189|736|59|48.11|79.86|63.08|0.00|3721.72|2838.49|4711.74|74.43|0.00|3721.72|3796.15|883.23| +2451076|64382|1615|44751|1655425|6099|19526|7|149|736|94|57.34|72.24|14.44|0.00|1357.36|5389.96|6790.56|40.72|0.00|1357.36|1398.08|-4032.60| +2451076|64382|11930|44751|1655425|6099|19526|7|259|736|86|16.76|18.43|12.71|754.21|1093.06|1441.36|1584.98|0.00|754.21|338.85|338.85|-1102.51| +2451076|64382|2212|44751|1655425|6099|19526|7|112|736|34|60.72|117.18|37.49|433.38|1274.66|2064.48|3984.12|67.30|433.38|841.28|908.58|-1223.20| +2451076|64382|17120|44751|1655425|6099|19526|7|3|736|54|85.39|166.51|113.22|0.00|6113.88|4611.06|8991.54|305.69|0.00|6113.88|6419.57|1502.82| +2451076|64382|10948|44751|1655425|6099|19526|7|243|736|32|96.56|155.46|133.69|1668.45|4278.08|3089.92|4974.72|52.19|1668.45|2609.63|2661.82|-480.29| +2451076|64382|17383|44751|1655425|6099|19526|7|50|736|34|86.86|118.99|22.60|0.00|768.40|2953.24|4045.66|7.68|0.00|768.40|776.08|-2184.84| +2451076|64382|11380|44751|1655425|6099|19526|7|240|736|15|21.82|42.33|17.35|0.00|260.25|327.30|634.95|0.00|0.00|260.25|260.25|-67.05| +2451076|64382|13684|44751|1655425|6099|19526|7|260|736|5|68.01|122.41|122.41|563.08|612.05|340.05|612.05|3.42|563.08|48.97|52.39|-291.08| +2451076|64382|11390|44751|1655425|6099|19526|7|14|736|76|56.38|89.64|43.92|0.00|3337.92|4284.88|6812.64|66.75|0.00|3337.92|3404.67|-946.96| +2451078|48111|8962|75072|1742063|5501|16492|7|69|737|71|83.33|97.49|39.97|0.00|2837.87|5916.43|6921.79|255.40|0.00|2837.87|3093.27|-3078.56| +2451078|48111|9014|75072|1742063|5501|16492|7|118|737|46|12.08|14.37|3.01|137.07|138.46|555.68|661.02|0.00|137.07|1.39|1.39|-554.29| +2451078|48111|3332|75072|1742063|5501|16492|7|286|737|38|1.49|2.87|2.35|0.00|89.30|56.62|109.06|3.57|0.00|89.30|92.87|32.68| +2451078|48111|2698|75072|1742063|5501|16492|7|81|737|19|39.15|66.16|46.97|0.00|892.43|743.85|1257.04|62.47|0.00|892.43|954.90|148.58| +2451078|48111|2785|75072|1742063|5501|16492|7|271|737|6|96.79|120.01|24.00|113.76|144.00|580.74|720.06|1.81|113.76|30.24|32.05|-550.50| +2451078|48111|7064|75072|1742063|5501|16492|7|9|737|80|3.13|4.44|1.02|0.00|81.60|250.40|355.20|2.44|0.00|81.60|84.04|-168.80| +2451078|48111|2228|75072|1742063|5501|16492|7|96|737|21|80.37|139.84|111.87|0.00|2349.27|1687.77|2936.64|164.44|0.00|2349.27|2513.71|661.50| +2451078|48111|17884|75072|1742063|5501|16492|7|23|737|49|31.96|43.14|34.51|0.00|1690.99|1566.04|2113.86|101.45|0.00|1690.99|1792.44|124.95| +2451078|48111|7862|75072|1742063|5501|16492|7|235|737|35|71.49|142.98|62.91|330.27|2201.85|2502.15|5004.30|149.72|330.27|1871.58|2021.30|-630.57| +2451078|48111|14998|75072|1742063|5501|16492|7|72|737|38|46.50|58.12|58.12|0.00|2208.56|1767.00|2208.56|22.08|0.00|2208.56|2230.64|441.56| +2451078|48111|13138|75072|1742063|5501|16492|7|246|737|66|44.67|69.68|24.38|32.18|1609.08|2948.22|4598.88|110.38|32.18|1576.90|1687.28|-1371.32| +2451078|48111|17041|75072|1742063|5501|16492|7|46|737|39|91.19|125.84|124.58|0.00|4858.62|3556.41|4907.76|437.27|0.00|4858.62|5295.89|1302.21| +2451078|48111|2572|75072|1742063|5501|16492|7|222|737|7|98.14|128.56|97.70|0.00|683.90|686.98|899.92|41.03|0.00|683.90|724.93|-3.08| +2451078|48111|692|75072|1742063|5501|16492|7|175|737|34|8.04|13.58|8.01|0.00|272.34|273.36|461.72|10.89|0.00|272.34|283.23|-1.02| +2451078|48111|3538|75072|1742063|5501|16492|7|68|737|24|73.73|108.38|2.16|0.00|51.84|1769.52|2601.12|2.07|0.00|51.84|53.91|-1717.68| +2451078|48111|14176|75072|1742063|5501|16492|7|107|737|66|80.82|135.77|4.07|0.00|268.62|5334.12|8960.82|13.43|0.00|268.62|282.05|-5065.50| +2451838|63719|8725|66864|1527945|1826|43307|1|159|738|60|67.91|92.35|55.41|0.00|3324.60|4074.60|5541.00|166.23|0.00|3324.60|3490.83|-750.00| +2451838|63719|1673|66864|1527945|1826|43307|1|67|738|97|58.72|115.67|32.38|0.00|3140.86|5695.84|11219.99|62.81|0.00|3140.86|3203.67|-2554.98| +2451838|63719|7949|66864|1527945|1826|43307|1|24|738|44|75.00|109.50|33.94|0.00|1493.36|3300.00|4818.00|29.86|0.00|1493.36|1523.22|-1806.64| +2451838|63719|3146|66864|1527945|1826|43307|1|42|738|92|48.20|51.09|32.18|0.00|2960.56|4434.40|4700.28|236.84|0.00|2960.56|3197.40|-1473.84| +2451838|63719|10783|66864|1527945|1826|43307|1|158|738|50|82.49|108.88|31.57|0.00|1578.50|4124.50|5444.00|78.92|0.00|1578.50|1657.42|-2546.00| +2451838|63719|15703|66864|1527945|1826|43307|1|38|738|11|27.30|36.58|33.28|205.00|366.08|300.30|402.38|1.61|205.00|161.08|162.69|-139.22| +2451838|63719|6007|66864|1527945|1826|43307|1|26|738|44|42.57|56.19|25.28|0.00|1112.32|1873.08|2472.36|88.98|0.00|1112.32|1201.30|-760.76| +2451838|63719|11725|66864|1527945|1826|43307|1|43|738|73|29.10|53.83|27.99|0.00|2043.27|2124.30|3929.59|102.16|0.00|2043.27|2145.43|-81.03| +2451838|63719|16589|66864|1527945|1826|43307|1|49|738|73|68.56|111.75|87.16|0.00|6362.68|5004.88|8157.75|190.88|0.00|6362.68|6553.56|1357.80| +2451838|63719|17960|66864|1527945|1826|43307|1|296|738|46|1.84|3.31|1.58|0.00|72.68|84.64|152.26|2.90|0.00|72.68|75.58|-11.96| +2451838|63719|14720|66864|1527945|1826|43307|1|7|738|1|56.51|87.59|49.05|0.00|49.05|56.51|87.59|0.00|0.00|49.05|49.05|-7.46| +2451838|63719|17216|66864|1527945|1826|43307|1|138|738|28|28.86|32.32|21.33|0.00|597.24|808.08|904.96|0.00|0.00|597.24|597.24|-210.84| +2451838|63719|3617|66864|1527945|1826|43307|1|190|738|42|81.07|126.46|84.72|0.00|3558.24|3404.94|5311.32|177.91|0.00|3558.24|3736.15|153.30| +2451838|63719|13283|66864|1527945|1826|43307|1|266|738|72|16.64|25.95|4.93|0.00|354.96|1198.08|1868.40|17.74|0.00|354.96|372.70|-843.12| +2452530|48380|9049|31823|621759|2095|33984|7|8|739|23|28.32|53.80|51.64|0.00|1187.72|651.36|1237.40|95.01|0.00|1187.72|1282.73|536.36| +|48380|9553|31823||2095|33984|7|256|739|53||||2479.17|||7258.35|38.95|2479.17|1947.92||| +2452530|48380|12738|31823|621759|2095|33984|7|131|739|35|74.87|147.49|48.67|0.00|1703.45|2620.45|5162.15|102.20|0.00|1703.45|1805.65|-917.00| +2452530|48380|7141|31823|621759|2095|33984|7|20|739|16|91.53|172.99|107.25|600.60|1716.00|1464.48|2767.84|78.07|600.60|1115.40|1193.47|-349.08| +2452530|48380|750|31823|621759|2095|33984|7|300|739|83|27.66|55.04|37.42|310.58|3105.86|2295.78|4568.32|111.81|310.58|2795.28|2907.09|499.50| +2452530|48380|6805|31823|621759|2095|33984|7|228|739|68|20.87|27.33|25.41|0.00|1727.88|1419.16|1858.44|0.00|0.00|1727.88|1727.88|308.72| +2452530|48380|16362|31823|621759|2095|33984|7|292|739|99|15.47|19.18|0.57|41.75|56.43|1531.53|1898.82|0.73|41.75|14.68|15.41|-1516.85| +2452530|48380|2136|31823|621759|2095|33984|7|271|739|30|48.90|90.95|21.82|0.00|654.60|1467.00|2728.50|39.27|0.00|654.60|693.87|-812.40| +2452530|48380|15156|31823|621759|2095|33984|7|65|739|61|86.45|115.84|71.82|0.00|4381.02|5273.45|7066.24|394.29|0.00|4381.02|4775.31|-892.43| +2452530||5343||621759|||7|223|739|23|||105.61||||||||1554.58|-172.49| +2452530|48380|5853|31823|621759|2095|33984|7|258|739|82|11.57|15.73|15.41|732.89|1263.62|948.74|1289.86|21.22|732.89|530.73|551.95|-418.01| +2452530|48380|14670|31823|621759|2095|33984|7|113|739|86|99.70|125.62|79.14|0.00|6806.04|8574.20|10803.32|272.24|0.00|6806.04|7078.28|-1768.16| +2452530|48380|3912|31823|621759|2095|33984|7|208|739|73|68.50|74.66|67.94|2380.61|4959.62|5000.50|5450.18|180.53|2380.61|2579.01|2759.54|-2421.49| +2452530|48380|2923|31823|621759|2095|33984|7|186|739|69|60.04|69.04|9.66|0.00|666.54|4142.76|4763.76|6.66|0.00|666.54|673.20|-3476.22| +2452530|48380|3345|31823|621759|2095|33984|7|174|739|78|85.37|116.95|76.01|0.00|5928.78|6658.86|9122.10|474.30|0.00|5928.78|6403.08|-730.08| +2451540|56119|1507|3355|1252916|4077|46838|10|134|740|91|12.89|16.49|13.19|0.00|1200.29|1172.99|1500.59|0.00|0.00|1200.29|1200.29|27.30| +2451540|56119|1508|3355|1252916|4077|46838|10|245|740|23|33.85|57.54|56.96|0.00|1310.08|778.55|1323.42|104.80|0.00|1310.08|1414.88|531.53| +2451540|56119|2689|3355|1252916|4077|46838|10|56|740|71|37.90|60.64|12.12|0.00|860.52|2690.90|4305.44|0.00|0.00|860.52|860.52|-1830.38| +2451540|56119|9464|3355|1252916|4077|46838|10|236|740|19|72.88|74.33|55.74|0.00|1059.06|1384.72|1412.27|95.31|0.00|1059.06|1154.37|-325.66| +2451540|56119|9448|3355|1252916|4077|46838|10|74|740|82|10.59|19.69|6.30|0.00|516.60|868.38|1614.58|30.99|0.00|516.60|547.59|-351.78| +2451540|56119|10696|3355|1252916|4077|46838|10|170|740|21|24.24|31.26|27.50|0.00|577.50|509.04|656.46|17.32|0.00|577.50|594.82|68.46| +2451540|56119|14558|3355|1252916|4077|46838|10|170|740|83|34.17|63.89|6.38|0.00|529.54|2836.11|5302.87|21.18|0.00|529.54|550.72|-2306.57| +2451540|56119|17695|3355|1252916|4077|46838|10|70|740|94|2.64|3.77|1.24|109.56|116.56|248.16|354.38|0.42|109.56|7.00|7.42|-241.16| +2451540|56119|8702|3355|1252916|4077|46838|10|33|740|43|67.60|76.38|1.52|0.00|65.36|2906.80|3284.34|0.00|0.00|65.36|65.36|-2841.44| +2451540|56119|8275|3355|1252916|4077|46838|10|252|740|49|17.99|23.20|6.03|0.00|295.47|881.51|1136.80|26.59|0.00|295.47|322.06|-586.04| +2451540|56119|10952|3355|1252916|4077|46838|10|264|740|83|16.50|31.18|27.75|0.00|2303.25|1369.50|2587.94|161.22|0.00|2303.25|2464.47|933.75| +2451540|56119|2990|3355|1252916|4077|46838|10|75|740|58|60.04|100.26|97.25|0.00|5640.50|3482.32|5815.08|507.64|0.00|5640.50|6148.14|2158.18| +2451540|56119|3154|3355|1252916|4077|46838|10|282|740|49|66.64|123.95|39.66|0.00|1943.34|3265.36|6073.55|0.00|0.00|1943.34|1943.34|-1322.02| +2451540|56119|13363|3355|1252916|4077|46838|10|279|740|48|31.65|33.86|1.01|23.27|48.48|1519.20|1625.28|0.00|23.27|25.21|25.21|-1493.99| +2451789|60426|13352|10478|250690|55|27467|1|220|741|10|34.21|51.65|51.13|0.00|511.30|342.10|516.50|35.79|0.00|511.30|547.09|169.20| +2451789|60426|13226|10478|250690|55|27467|1|20|741|17|26.43|36.47|33.18|0.00|564.06|449.31|619.99|0.00|0.00|564.06|564.06|114.75| +2451789|60426|12701|10478|250690|55|27467|1|174|741|99|69.06|75.96|31.14|0.00|3082.86|6836.94|7520.04|246.62|0.00|3082.86|3329.48|-3754.08| +2451789|60426|1175|10478|250690|55|27467|1|219|741|10|60.91|108.41|3.25|0.00|32.50|609.10|1084.10|0.97|0.00|32.50|33.47|-576.60| +2451789|60426|14378|10478|250690|55|27467|1|108|741|62|61.87|98.37|16.72|0.00|1036.64|3835.94|6098.94|82.93|0.00|1036.64|1119.57|-2799.30| +|60426|17807|10478||55|27467|1|79|741|||113.32|43.06|0.00||9262.76|10652.08||0.00|4047.64||-5215.12| +|60426|458|||55||1|62|741||25.36||11.92|0.00|727.12|1546.96|2598.60||0.00|727.12|741.66|-819.84| +2451789|60426|1208|10478|250690|55|27467|1|5|741|3|91.07|109.28|18.57|32.31|55.71|273.21|327.84|0.70|32.31|23.40|24.10|-249.81| +2451789|60426|3119|10478|250690|55|27467|1|203|741|72|22.68|34.02|31.29|0.00|2252.88|1632.96|2449.44|0.00|0.00|2252.88|2252.88|619.92| +2451789|60426|3701|10478|250690|55|27467|1|87|741|41|72.83|93.95|29.12|0.00|1193.92|2986.03|3851.95|47.75|0.00|1193.92|1241.67|-1792.11| +2451789||12230|10478||55||1||741|16|51.52||67.97|||824.32|1195.20|||1087.52||| +2451789|60426|16130|10478|250690|55|27467|1|190|741|14|91.43|168.23|43.73|0.00|612.22|1280.02|2355.22|30.61|0.00|612.22|642.83|-667.80| +2451789|60426|7472|10478|250690|55|27467|1|276|741|43|76.45|144.49|106.92|0.00|4597.56|3287.35|6213.07|91.95|0.00|4597.56|4689.51|1310.21| +2451249|57663|1888|61092|94323|354|37585|2|77|742|62|4.78|6.69|2.54|0.00|157.48|296.36|414.78|4.72|0.00|157.48|162.20|-138.88| +2451249|57663|16226|61092|94323|354|37585|2|50|742|56|68.88|126.73|84.90|0.00|4754.40|3857.28|7096.88|95.08|0.00|4754.40|4849.48|897.12| +2451249|57663|11920|61092|94323|354|37585|2|27|742|53|94.02|104.36|49.04|0.00|2599.12|4983.06|5531.08|181.93|0.00|2599.12|2781.05|-2383.94| +2451249|57663|16369|61092|94323||37585|2|160|742||55.53||||211.24|||12.67||211.24|223.91|-10.88| +2451249|57663|3602|61092|94323|354|37585|2|218|742|47|69.47|90.31|70.44|0.00|3310.68|3265.09|4244.57|0.00|0.00|3310.68|3310.68|45.59| +2451249|57663|7552|61092|94323|354|37585|2|265|742|44|54.77|56.41|48.51|0.00|2134.44|2409.88|2482.04|192.09|0.00|2134.44|2326.53|-275.44| +|57663|7496||||||58|742|6|19.16|19.73|5.32|0.00||114.96|118.38|0.00|0.00|||-83.04| +2451249|57663|9476|61092|94323|354|37585|2|272|742|93|90.77|108.92|99.11|0.00|9217.23|8441.61|10129.56|184.34|0.00|9217.23|9401.57|775.62| +2451249|57663|11614|61092|94323|354|37585|2|152|742|14|43.35|62.85|35.82|0.00|501.48|606.90|879.90|45.13|0.00|501.48|546.61|-105.42| +2451249|57663|10454|61092|94323|354|37585|2|201|742|56|81.77|128.37|0.00|0.00|0.00|4579.12|7188.72|0.00|0.00|0.00|0.00|-4579.12| +2451249|57663|5077|61092|94323|354|37585|2|44|742|31|10.69|17.85|4.64|79.11|143.84|331.39|553.35|1.29|79.11|64.73|66.02|-266.66| +2451249|57663|12232|61092|94323|354|37585|2|93|742|7|95.56|141.42|48.08|168.28|336.56|668.92|989.94|13.46|168.28|168.28|181.74|-500.64| +||15542|61092|94323||||25|742|88||143.20||322.57||||148.38|322.57|||-4118.01| +2451249|57663|13321|61092|94323|354|37585|2|265|742|12|88.87|163.52|53.96|0.00|647.52|1066.44|1962.24|38.85|0.00|647.52|686.37|-418.92| +2451249|57663|10874|61092|94323|354|37585|2|249|742|89|31.65|31.96|24.28|0.00|2160.92|2816.85|2844.44|43.21|0.00|2160.92|2204.13|-655.93| +2451249|57663|15580||||37585|||742|40|69.11|69.80|50.25||2010.00|2764.40|2792.00|||||-754.40| +2452154|67281|1439|92246|398719|6792|20465|8|210|743|62|14.45|26.87|7.79|0.00|482.98|895.90|1665.94|0.00|0.00|482.98|482.98|-412.92| +2452154|67281|4387|92246|398719|6792|20465|8|173|743|5|60.01|88.21|22.93|83.69|114.65|300.05|441.05|1.85|83.69|30.96|32.81|-269.09| +2452154|67281|4849|92246|398719|6792|20465|8|32|743|29|77.34|136.11|1.36|27.21|39.44|2242.86|3947.19|0.85|27.21|12.23|13.08|-2230.63| +2452154|67281|6925|92246|398719|6792|20465|8|71|743|83|47.85|72.25|0.00|0.00|0.00|3971.55|5996.75|0.00|0.00|0.00|0.00|-3971.55| +2452154|67281|2219|92246|398719|6792|20465|8|111|743|49|1.36|1.70|1.12|0.00|54.88|66.64|83.30|2.19|0.00|54.88|57.07|-11.76| +2452154|67281|1273|92246|398719|6792|20465|8|50|743|48|35.58|46.60|25.16|0.00|1207.68|1707.84|2236.80|96.61|0.00|1207.68|1304.29|-500.16| +2452154|67281|17063|92246|398719|6792|20465|8|18|743|89|38.96|45.19|0.45|20.42|40.05|3467.44|4021.91|0.19|20.42|19.63|19.82|-3447.81| +2452154|67281|3199|92246|398719|6792|20465|8|295|743|13|50.04|51.54|13.91|162.74|180.83|650.52|670.02|0.00|162.74|18.09|18.09|-632.43| +2452154|67281|7153|92246|398719|6792|20465|8|261|743|4|74.14|86.74|50.30|0.00|201.20|296.56|346.96|10.06|0.00|201.20|211.26|-95.36| +2452154|67281|14817|92246|398719|6792|20465|8|182|743|36|92.50|184.07|180.38|844.17|6493.68|3330.00|6626.52|225.98|844.17|5649.51|5875.49|2319.51| +2452154|67281|17077|92246|398719|6792|20465|8|140|743|23|12.63|23.74|20.41|0.00|469.43|290.49|546.02|18.77|0.00|469.43|488.20|178.94| +2452154|67281|12303|92246|398719|6792|20465|8|243|743|97|22.68|36.06|29.56|0.00|2867.32|2199.96|3497.82|114.69|0.00|2867.32|2982.01|667.36| +2451979|30211|2497|51871|843815|6959|27703|8|89|744|59|46.51|54.41|26.66|0.00|1572.94|2744.09|3210.19|0.00|0.00|1572.94|1572.94|-1171.15| +2451979|30211|4749|51871|843815|6959|27703|8|229|744|31|96.16|129.81|6.49|0.00|201.19|2980.96|4024.11|4.02|0.00|201.19|205.21|-2779.77| +2451979|30211|6751|51871|843815|6959|27703|8|124|744|69|25.35|45.88|15.59|914.35|1075.71|1749.15|3165.72|6.45|914.35|161.36|167.81|-1587.79| +2451979|30211|6043|51871|843815|6959|27703|8|224|744|67|51.81|69.42|36.79|0.00|2464.93|3471.27|4651.14|0.00|0.00|2464.93|2464.93|-1006.34| +2451979|30211|13303|51871|843815|6959|27703|8|172|744|18|91.01|182.02|176.55|0.00|3177.90|1638.18|3276.36|127.11|0.00|3177.90|3305.01|1539.72| +2451979|30211|7375|51871|843815|6959|27703|8|291|744|5|70.01|122.51|75.95|0.00|379.75|350.05|612.55|7.59|0.00|379.75|387.34|29.70| +2451979|30211|10265|51871|843815|6959|27703|8|116|744|19|12.14|14.08|4.92|0.00|93.48|230.66|267.52|6.54|0.00|93.48|100.02|-137.18| +||1795|||||||744||97.27|182.86|177.37|3277.79|3902.14|2139.94|||3277.79|||| +2451979|30211|11551|51871|843815|6959|27703|8|137|744|40|59.51|69.62|29.93|1005.64|1197.20|2380.40|2784.80|15.32|1005.64|191.56|206.88|-2188.84| +2451979|30211|5653|51871|843815|6959|27703|8|36|744|81|95.38|111.59|24.54|0.00|1987.74|7725.78|9038.79|159.01|0.00|1987.74|2146.75|-5738.04| +2451979|30211|407|51871|843815|6959|27703|8|28|744|59|16.75|30.65|26.66|0.00|1572.94|988.25|1808.35|0.00|0.00|1572.94|1572.94|584.69| +2451979|30211|1389|51871|843815|6959|27703|8|66|744|3|23.19|42.66|7.25|0.00|21.75|69.57|127.98|0.21|0.00|21.75|21.96|-47.82| +2451979|30211|3103|51871|843815|6959|27703|8|59|744|29|69.21|134.26|26.85|0.00|778.65|2007.09|3893.54|54.50|0.00|778.65|833.15|-1228.44| +2451042|59154|10837|59824|1157692|1607|27922|10|164|745|60|39.05|66.77|24.70|1348.62|1482.00|2343.00|4006.20|9.33|1348.62|133.38|142.71|-2209.62| +2451042|59154|13522|59824|1157692|1607|27922|10|119|745|41|14.41|20.02|3.00|0.00|123.00|590.81|820.82|7.38|0.00|123.00|130.38|-467.81| +2451042|59154|8011|59824|1157692|1607|27922|10|272|745|3|87.55|155.83|112.19|0.00|336.57|262.65|467.49|13.46|0.00|336.57|350.03|73.92| +2451042|59154|8977|59824|1157692|1607|27922|10|288|745|36|97.09|158.25|4.74|0.00|170.64|3495.24|5697.00|13.65|0.00|170.64|184.29|-3324.60| +2451042|59154|15094|59824|1157692|1607|27922|10|75|745|86|41.20|53.14|31.35|0.00|2696.10|3543.20|4570.04|107.84|0.00|2696.10|2803.94|-847.10| +2451042|59154|16730|59824|1157692|1607|27922|10|275|745|12|64.61|108.54|65.12|562.63|781.44|775.32|1302.48|2.18|562.63|218.81|220.99|-556.51| +2451042|59154|1162|59824|1157692|1607|27922|10|271|745|82|97.67|140.64|123.76|0.00|10148.32|8008.94|11532.48|507.41|0.00|10148.32|10655.73|2139.38| +2451042|59154|7942|59824|1157692|1607|27922|10|135|745|4|27.07|53.59|27.33|0.00|109.32|108.28|214.36|2.18|0.00|109.32|111.50|1.04| +2451042|59154|11506|59824|1157692|1607|27922|10|31|745|56|42.38|76.70|69.79|3712.82|3908.24|2373.28|4295.20|3.90|3712.82|195.42|199.32|-2177.86| +2451042|59154|3094|59824|1157692|1607|27922|10|74|745|70|5.89|7.48|4.48|0.00|313.60|412.30|523.60|28.22|0.00|313.60|341.82|-98.70| +2451042|59154|224|59824|1157692|1607|||241|745||25.00|27.00||||1600.00||23.50||293.76|317.26|-1306.24| +2451042|59154|16477|59824|1157692|1607|27922|10|94|745|79|7.23|7.23|6.72|0.00|530.88|571.17|571.17|5.30|0.00|530.88|536.18|-40.29| +2451655|52488|16109|26122|581330|6585|18640|4|228|746|6|71.94|138.84|105.51|0.00|633.06|431.64|833.04|6.33|0.00|633.06|639.39|201.42| +2451655|52488|4915|26122|581330|6585|18640|4|153|746|71|39.28|57.34|1.72|0.00|122.12|2788.88|4071.14|3.66|0.00|122.12|125.78|-2666.76| +2451655|52488|8789|26122|581330|6585|18640|4|267|746|77|92.57|130.52|71.78|110.54|5527.06|7127.89|10050.04|433.32|110.54|5416.52|5849.84|-1711.37| +2451655|52488|14795|26122|581330|6585|18640|4|260|746|95|45.44|62.25|41.08|1365.91|3902.60|4316.80|5913.75|228.30|1365.91|2536.69|2764.99|-1780.11| +2451655|52488|4811|26122|581330|6585|18640|4|253|746|77|79.38|101.60|10.16|70.40|782.32|6112.26|7823.20|7.11|70.40|711.92|719.03|-5400.34| +2451655|52488|11183|26122|581330|6585|18640|4|105|746|34|66.87|88.26|38.83|0.00|1320.22|2273.58|3000.84|92.41|0.00|1320.22|1412.63|-953.36| +2451655|52488|7493|26122|581330|6585|18640|4|199|746|61|51.81|54.40|14.68|0.00|895.48|3160.41|3318.40|35.81|0.00|895.48|931.29|-2264.93| +2451655|52488|3938|26122|581330|6585|18640|4|226|746|6|93.61|131.05|89.11|0.00|534.66|561.66|786.30|26.73|0.00|534.66|561.39|-27.00| +2451655|52488|14150|26122|581330|6585|18640|4|267|746|54|64.01|96.65|8.69|0.00|469.26|3456.54|5219.10|9.38|0.00|469.26|478.64|-2987.28| +2451655|52488|14167|26122|581330|6585|18640|4|296|746|55|2.04|2.85|1.36|0.00|74.80|112.20|156.75|5.23|0.00|74.80|80.03|-37.40| +2451655|52488|9452|26122|581330|6585|18640|4|71|746|98|37.21|64.37|0.64|0.00|62.72|3646.58|6308.26|5.01|0.00|62.72|67.73|-3583.86| +2451655|52488|3715|26122|581330|6585|18640|4|255|746|27|92.12|107.78|64.66|0.00|1745.82|2487.24|2910.06|87.29|0.00|1745.82|1833.11|-741.42| +2452366|54417|17175|86082|1784624|4766|508|7|243|747|21|6.25|7.06|1.34|0.00|28.14|131.25|148.26|1.96|0.00|28.14|30.10|-103.11| +2452366|54417|115|86082|1784624|4766|508|7|158|747|17|57.96|61.43|2.45|0.00|41.65|985.32|1044.31|0.41|0.00|41.65|42.06|-943.67| +2452366|54417|12895|86082|1784624|4766|508|7|86|747|70|31.42|49.64|14.89|0.00|1042.30|2199.40|3474.80|52.11|0.00|1042.30|1094.41|-1157.10| +2452366|54417|13266|86082|1784624|4766|508|7|296|747|71|21.28|26.17|13.34|217.84|947.14|1510.88|1858.07|7.29|217.84|729.30|736.59|-781.58| +2452366|54417|7225|86082|1784624|4766|508|7|51|747|91|83.98|83.98|81.46|0.00|7412.86|7642.18|7642.18|518.90|0.00|7412.86|7931.76|-229.32| +2452366|54417|16737|86082|1784624|4766|508|7|89|747|24|82.52|137.80|86.81|0.00|2083.44|1980.48|3307.20|145.84|0.00|2083.44|2229.28|102.96| +2452366|54417|150|86082|1784624|4766|508|7|18|747|74|87.27|141.37|124.40|0.00|9205.60|6457.98|10461.38|92.05|0.00|9205.60|9297.65|2747.62| +2452366|54417|1689|86082|1784624|4766|508|7|190|747|99|11.55|22.40|6.49|539.70|642.51|1143.45|2217.60|9.25|539.70|102.81|112.06|-1040.64| +2452366|54417|17005|86082|1784624|4766|508|7|154|747|26|3.61|7.07|5.72|11.89|148.72|93.86|183.82|5.47|11.89|136.83|142.30|42.97| +2452366|54417|14814|86082|1784624|4766|508|7|284|747|37|3.99|5.10|0.20|0.00|7.40|147.63|188.70|0.59|0.00|7.40|7.99|-140.23| +2452366|54417|11352|86082|1784624|4766|508|7|152|747|15|6.57|8.47|4.57|0.00|68.55|98.55|127.05|2.74|0.00|68.55|71.29|-30.00| +2452366|54417|414|86082|1784624|4766|508|7|280|747|23|20.00|32.80|18.36|0.00|422.28|460.00|754.40|8.44|0.00|422.28|430.72|-37.72| +2452366|54417|12909|86082|1784624|4766|508|7|207|747|64|68.61|70.66|21.19|0.00|1356.16|4391.04|4522.24|81.36|0.00|1356.16|1437.52|-3034.88| +2452366|54417|14961|86082|1784624|4766|508|7|47|747|92|79.07|127.30|109.47|0.00|10071.24|7274.44|11711.60|805.69|0.00|10071.24|10876.93|2796.80| +2452366|54417|5775|86082|1784624|4766|508|7|81|747|57|57.89|93.78|93.78|4864.36|5345.46|3299.73|5345.46|4.81|4864.36|481.10|485.91|-2818.63| +2452366|54417|2995|86082|1784624|4766|508|7|260|747|100|52.86|89.86|67.39|0.00|6739.00|5286.00|8986.00|202.17|0.00|6739.00|6941.17|1453.00| +2451725|38984|10928|67020|1161848|6723|34751|7|22|748|55|86.28|88.86|6.22|0.00|342.10|4745.40|4887.30|10.26|0.00|342.10|352.36|-4403.30| +2451725|38984|2996|67020|1161848|6723|34751|7|187|748|97|43.54|67.48|26.31|0.00|2552.07|4223.38|6545.56|229.68|0.00|2552.07|2781.75|-1671.31| +2451725|38984|13171|67020|1161848|6723|34751|7|150|748|51|81.46|110.78|21.04|0.00|1073.04|4154.46|5649.78|0.00|0.00|1073.04|1073.04|-3081.42| +2451725|38984|7283|67020|1161848|6723|34751|7|18|748|22|36.09|51.24|26.13|0.00|574.86|793.98|1127.28|22.99|0.00|574.86|597.85|-219.12| +2451725|38984|15337|67020|1161848|6723|34751|7|200|748|19|2.19|3.67|2.05|0.00|38.95|41.61|69.73|0.77|0.00|38.95|39.72|-2.66| +2451725|38984|17125|67020|1161848|6723|34751|7|254|748|18|50.78|54.84|30.16|0.00|542.88|914.04|987.12|27.14|0.00|542.88|570.02|-371.16| +2451725|38984|14780|67020|1161848|6723|34751|7|122|748|55|50.62|68.33|63.54|0.00|3494.70|2784.10|3758.15|279.57|0.00|3494.70|3774.27|710.60| +2451725|38984|17449|67020|1161848|6723|34751|7|249|748|48|50.68|65.37|59.48|0.00|2855.04|2432.64|3137.76|85.65|0.00|2855.04|2940.69|422.40| +2451725|38984|12764|67020|1161848|6723|34751|7|175|748|25|33.78|59.79|16.14|246.13|403.50|844.50|1494.75|1.57|246.13|157.37|158.94|-687.13| +2451725|38984|5936|67020|1161848|6723|34751|7|256|748|22|87.42|100.53|82.43|0.00|1813.46|1923.24|2211.66|126.94|0.00|1813.46|1940.40|-109.78| +2451725|38984|14263|67020|1161848|6723|34751|7|104|748|79|42.64|53.30|50.10|0.00|3957.90|3368.56|4210.70|197.89|0.00|3957.90|4155.79|589.34| +2451725|38984|743|67020|1161848|6723|34751|7|199|748|23|68.52|91.13|36.45|503.01|838.35|1575.96|2095.99|23.47|503.01|335.34|358.81|-1240.62| +2451725|38984|12361|67020|1161848|6723|34751|7|285|748|10|98.39|120.03|31.20|0.00|312.00|983.90|1200.30|6.24|0.00|312.00|318.24|-671.90| +2451725|38984|7367|67020|1161848|6723|34751|7|208|748|1|82.32|135.00|24.30|0.00|24.30|82.32|135.00|1.94|0.00|24.30|26.24|-58.02| +2451725|38984|15775|67020|1161848|6723|34751|7|274|748|52|76.43|134.51|117.02|5659.08|6085.04|3974.36|6994.52|34.07|5659.08|425.96|460.03|-3548.40| +2451059|44700|6998|11156|1284035|914|4993|8|272|749|24|32.77|57.67|30.56|0.00|733.44|786.48|1384.08|36.67|0.00|733.44|770.11|-53.04| +2451059|44700|17851|11156|1284035|914|4993|8|31|749|83|93.15|179.77|97.07|0.00|8056.81|7731.45|14920.91|725.11|0.00|8056.81|8781.92|325.36| +2451059|44700|9109|11156|1284035|914|4993|8|204|749|43|23.93|42.11|18.10|0.00|778.30|1028.99|1810.73|7.78|0.00|778.30|786.08|-250.69| +2451059|44700|12187|11156|1284035|914|4993|8|124|749|89|39.62|44.37|43.48|0.00|3869.72|3526.18|3948.93|154.78|0.00|3869.72|4024.50|343.54| +2451059|44700|4894|11156||914|4993|8|213|749||93.48|||244.30|370.16||673.04||244.30||127.11|| +2451059|44700|8858|11156|1284035|914|4993|8|8|749|12|68.47|103.38|40.31|0.00|483.72|821.64|1240.56|24.18|0.00|483.72|507.90|-337.92| +2451059|44700|6049|11156|1284035|914|4993|8|227|749|74|84.97|114.70|89.46|3574.82|6620.04|6287.78|8487.80|213.16|3574.82|3045.22|3258.38|-3242.56| +2451059|44700|14546|11156|1284035|914|4993|8|126|749|68|85.40|111.02|27.75|0.00|1887.00|5807.20|7549.36|94.35|0.00|1887.00|1981.35|-3920.20| +2451059|44700|6902|11156|1284035|914|4993|8|151|749|66|41.66|60.40|10.87|0.00|717.42|2749.56|3986.40|0.00|0.00|717.42|717.42|-2032.14| +2451059|44700|8012|11156|1284035|914|4993|8|276|749|78|73.55|119.15|115.57|0.00|9014.46|5736.90|9293.70|721.15|0.00|9014.46|9735.61|3277.56| +2451059|44700|6866|11156|1284035|914|4993|8|274|749|71|99.95|189.90|153.81|0.00|10920.51|7096.45|13482.90|109.20|0.00|10920.51|11029.71|3824.06| +2451059|44700|1706|11156|1284035|914|4993|8|238|749|65|60.02|84.62|26.23|0.00|1704.95|3901.30|5500.30|51.14|0.00|1704.95|1756.09|-2196.35| +2451059|44700|12910|11156|1284035|914|4993|8|291|749|64|29.84|51.92|37.90|0.00|2425.60|1909.76|3322.88|121.28|0.00|2425.60|2546.88|515.84| +2451059|44700|5335|11156|1284035|914|4993|8|117|749|72|38.15|44.63|12.05|546.58|867.60|2746.80|3213.36|3.21|546.58|321.02|324.23|-2425.78| +2451473|52864|14629|96414|1803301|5388|31875|1|262|750|52|86.67|124.80|99.84|1142.16|5191.68|4506.84|6489.60|364.45|1142.16|4049.52|4413.97|-457.32| +2451473|52864|8956|96414|1803301|5388|31875|1|56|750|5|71.94|142.44|133.89|0.00|669.45|359.70|712.20|13.38|0.00|669.45|682.83|309.75| +2451473|52864|6901|96414|1803301|5388|31875|1|257|750|46|93.07|177.76|106.65|0.00|4905.90|4281.22|8176.96|49.05|0.00|4905.90|4954.95|624.68| +||12118||1803301|5388|31875|1|38|750||95.77|164.72||0.00||5267.35|9059.60|24.45|0.00|||| +2451473|52864|10412|96414|1803301|5388|31875|1|56|750|71|85.72|109.72|49.37|0.00|3505.27|6086.12|7790.12|0.00|0.00|3505.27|3505.27|-2580.85| +2451473|52864|1964|96414|1803301|5388|31875|1|190|750|75|65.54|72.09|18.02|0.00|1351.50|4915.50|5406.75|40.54|0.00|1351.50|1392.04|-3564.00| +2451473|52864|1946|96414|1803301|5388|31875|1|31|750|19|3.01|3.25|1.98|19.18|37.62|57.19|61.75|0.00|19.18|18.44|18.44|-38.75| +2451473|52864|3230|96414|1803301|5388|31875|1|54|750|71|43.99|72.58|21.04|0.00|1493.84|3123.29|5153.18|104.56|0.00|1493.84|1598.40|-1629.45| +2451670|36393|6853|32427|483580|333|4333|2|179|751|45|50.15|58.17|35.48|1213.41|1596.60|2256.75|2617.65|3.83|1213.41|383.19|387.02|-1873.56| +2451670|36393|1019|32427|483580|333|4333|2|177|751|3|40.80|64.05|7.04|0.00|21.12|122.40|192.15|1.90|0.00|21.12|23.02|-101.28| +2451670|36393|4297|32427|483580|333|4333|2|261|751|16|67.92|117.50|52.87|516.01|845.92|1086.72|1880.00|16.49|516.01|329.91|346.40|-756.81| +2451670|36393|14503|32427|483580|333|4333|2|245|751|43|80.11|124.17|18.62|0.00|800.66|3444.73|5339.31|56.04|0.00|800.66|856.70|-2644.07| +2451670|36393|5117|32427|483580|333|4333|2|191|751|14|7.17|12.97|10.76|25.60|150.64|100.38|181.58|3.75|25.60|125.04|128.79|24.66| +2451670|36393|15635|32427|483580|333|4333|2|268|751|35|73.86|81.24|40.62|0.00|1421.70|2585.10|2843.40|99.51|0.00|1421.70|1521.21|-1163.40| +2451670|36393|3848|32427|483580|333|4333|2|298|751|24|83.44|152.69|42.75|0.00|1026.00|2002.56|3664.56|0.00|0.00|1026.00|1026.00|-976.56| +2451670|36393|7649|32427|483580|333|4333|2|157|751|62|56.06|109.87|0.00|0.00|0.00|3475.72|6811.94|0.00|0.00|0.00|0.00|-3475.72| +2451670|36393|1829|32427|483580|333|4333|2|258|751|34|72.05|119.60|49.03|616.79|1667.02|2449.70|4066.40|0.00|616.79|1050.23|1050.23|-1399.47| +2451670|36393|16472|32427|483580|333|4333|2|294|751|53|7.28|8.00|6.56|0.00|347.68|385.84|424.00|20.86|0.00|347.68|368.54|-38.16| +2451670|36393|9743|32427|483580|333|4333|2|39|751|97|33.60|44.01|34.76|0.00|3371.72|3259.20|4268.97|101.15|0.00|3371.72|3472.87|112.52| +2451670|36393|16949|32427|483580|333|4333|2|8|751|57|57.30|67.04|34.19|0.00|1948.83|3266.10|3821.28|77.95|0.00|1948.83|2026.78|-1317.27| +2451670|36393|10802|32427|483580|333|4333|2|15|751|61|8.08|13.57|3.79|0.00|231.19|492.88|827.77|4.62|0.00|231.19|235.81|-261.69| +2451182|58994|5764|44769|170231|2893|45955|2|203|752|16|30.22|41.40|4.96|0.00|79.36|483.52|662.40|2.38|0.00|79.36|81.74|-404.16| +2451182|58994|5563|44769|170231|2893|45955|2|201|752|60|34.74|59.75|11.95|0.00|717.00|2084.40|3585.00|28.68|0.00|717.00|745.68|-1367.40| +2451182|58994|12907|44769|170231|2893|45955|2|30|752|89|72.79|92.44|48.99|0.00|4360.11|6478.31|8227.16|348.80|0.00|4360.11|4708.91|-2118.20| +2451182|58994|12523|44769|170231|2893|45955|2|228|752|14|64.34|117.74|76.53|0.00|1071.42|900.76|1648.36|53.57|0.00|1071.42|1124.99|170.66| +|58994|4420||||45955|2|71|752||44.21||20.79|0.00|644.49||||0.00||702.49|-726.02| +2451182|58994|17950|44769|170231|2893|45955|2|32|752|50|61.90|78.61|11.00|0.00|550.00|3095.00|3930.50|33.00|0.00|550.00|583.00|-2545.00| +2451182|58994|5536|44769|170231|2893|45955|2|180|752|1|23.25|31.15|24.29|14.81|24.29|23.25|31.15|0.00|14.81|9.48|9.48|-13.77| +|58994|11581|44769||2893||2||752|52||26.42|||1304.68||1373.84|65.23||||| +2451182|58994|3904|44769|170231|2893|45955|2|210|752|36|60.77|65.63|25.59|0.00|921.24|2187.72|2362.68|27.63|0.00|921.24|948.87|-1266.48| +2451182|58994|10672|44769|170231|2893|45955|2|126|752|53|26.16|36.36|0.36|0.00|19.08|1386.48|1927.08|0.38|0.00|19.08|19.46|-1367.40| +2451182|58994|15770|44769|170231|2893|45955|2|252|752|4|45.69|52.54|38.87|0.00|155.48|182.76|210.16|10.88|0.00|155.48|166.36|-27.28| +2451531|62777|12859|7960|695429|4945|42861|1|259|753|52|93.00|108.81|3.26|0.00|169.52|4836.00|5658.12|5.08|0.00|169.52|174.60|-4666.48| +2451531|62777|6235|7960|695429|4945|42861|1|84|753|50|40.88|73.58|37.52|0.00|1876.00|2044.00|3679.00|18.76|0.00|1876.00|1894.76|-168.00| +2451531|62777|8497|7960|695429|4945|42861|1|132|753|42|99.04|125.78|12.57|0.00|527.94|4159.68|5282.76|26.39|0.00|527.94|554.33|-3631.74| +2451531|62777|5728|7960|695429|4945|42861|1|230|753|89|81.00|95.58|0.95|0.00|84.55|7209.00|8506.62|1.69|0.00|84.55|86.24|-7124.45| +2451531|62777|7987|7960|695429|4945|42861|1|7|753|8|24.99|39.48|17.76|0.00|142.08|199.92|315.84|11.36|0.00|142.08|153.44|-57.84| +||5300|||4945||1|93|753|25||16.45|||407.00|326.50||23.44||||| +2451531|62777|2860|7960|695429|4945|42861|1|102|753|55|34.55|43.87|14.03|138.89|771.65|1900.25|2412.85|37.96|138.89|632.76|670.72|-1267.49| +2451531|62777|11108|7960|695429|4945|42861|1|155|753|79|76.77|105.17|51.53|0.00|4070.87|6064.83|8308.43|0.00|0.00|4070.87|4070.87|-1993.96| +2451531|62777|6626|7960|695429|4945|42861|1|28|753|69|54.34|95.09|25.67|0.00|1771.23|3749.46|6561.21|53.13|0.00|1771.23|1824.36|-1978.23| +2451531|62777|7832|7960|695429|4945|42861|1|167|753|61|30.52|36.01|10.44|0.00|636.84|1861.72|2196.61|31.84|0.00|636.84|668.68|-1224.88| +2451531|62777|11062|7960|695429|4945|42861|1|235|753|52|93.26|104.45|100.27|0.00|5214.04|4849.52|5431.40|52.14|0.00|5214.04|5266.18|364.52| +2451531|62777|11665|7960|695429|4945|42861|1|54|753|5|38.96|75.58|36.27|103.36|181.35|194.80|377.90|2.33|103.36|77.99|80.32|-116.81| +2451531|62777|5446|7960|695429|4945|42861|1|60|753|18|63.27|77.18|62.51|202.53|1125.18|1138.86|1389.24|0.00|202.53|922.65|922.65|-216.21| +2451908|53610|8939|33126|413803|4433|34710|8|167|754|27|20.67|25.01|21.00|0.00|567.00|558.09|675.27|39.69|0.00|567.00|606.69|8.91| +2451908|53610|3626|33126|413803|4433|34710|8|102|754|11|42.92|59.22|53.29|0.00|586.19|472.12|651.42|0.00|0.00|586.19|586.19|114.07| +2451908|53610|8444|33126|413803|4433|34710|8|160|754|31|40.66|69.52|32.67|0.00|1012.77|1260.46|2155.12|30.38|0.00|1012.77|1043.15|-247.69| +2451908|53610|4826|33126|413803|4433|34710|8|263|754|29|87.52|102.39|28.66|0.00|831.14|2538.08|2969.31|49.86|0.00|831.14|881.00|-1706.94| +||7915||413803|4433||8||754||||13.26||716.04||1791.18|||||-382.86| +2451908|53610|6086|33126|413803|4433|34710|8|91|754|1|95.97|149.71|23.95|0.00|23.95|95.97|149.71|0.71|0.00|23.95|24.66|-72.02| +||2107|||4433||8|114|754|37|17.78|33.24||86.09|614.94|657.86|||86.09||539.42|| +2451908|53610|16958|33126|413803|4433|34710|8|241|754|5|4.78|9.22|4.79|18.68|23.95|23.90|46.10|0.26|18.68|5.27|5.53|-18.63| +2451908|53610|521|33126|413803|4433|34710|8|86|754|93|35.26|56.76|50.51|0.00|4697.43|3279.18|5278.68|93.94|0.00|4697.43|4791.37|1418.25| +2451908|53610|5759|33126|413803|4433||||754|52|84.43|97.93|||2851.68|||127.75||1596.95||| +2451908|53610|8912|33126|413803|4433|34710|8|13|754|12|46.85|77.77|2.33|0.00|27.96|562.20|933.24|0.27|0.00|27.96|28.23|-534.24| +2451908|53610|4367|33126|413803|4433|34710|8|63|754|62|61.90|122.56|15.93|0.00|987.66|3837.80|7598.72|49.38|0.00|987.66|1037.04|-2850.14| +2451905|70863|10145|24744|1673802|4460|31118|2|193|755|5|80.37|156.72|31.34|0.00|156.70|401.85|783.60|12.53|0.00|156.70|169.23|-245.15| +2451905|70863|5741|24744|1673802|4460|31118|2|256|755|9|41.82|49.34|48.84|0.00|439.56|376.38|444.06|17.58|0.00|439.56|457.14|63.18| +2451905|70863|14762|24744|1673802|4460|31118|2|276|755|18|12.03|18.52|1.29|0.00|23.22|216.54|333.36|1.62|0.00|23.22|24.84|-193.32| +2451905|70863|13736|24744|1673802|4460|31118|2|168|755|10|42.45|42.87|36.86|0.00|368.60|424.50|428.70|14.74|0.00|368.60|383.34|-55.90| +2451905|70863|5600|24744|1673802|4460|31118|2|236|755|61|28.32|41.91|30.59|0.00|1865.99|1727.52|2556.51|149.27|0.00|1865.99|2015.26|138.47| +2451905|70863|13913|24744|1673802|4460|31118|2|131|755|68|34.14|54.96|25.28|0.00|1719.04|2321.52|3737.28|85.95|0.00|1719.04|1804.99|-602.48| +2451905|70863|9091|24744|1673802|4460|31118|2|16|755|79|43.01|49.46|0.98|54.96|77.42|3397.79|3907.34|1.12|54.96|22.46|23.58|-3375.33| +2451905|70863|703|24744|1673802|4460|31118|2|127|755|32|73.95|129.41|120.35|0.00|3851.20|2366.40|4141.12|77.02|0.00|3851.20|3928.22|1484.80| +2450834|38403|13045|35993|1512459|2525|7391|1|66|756|53|60.99|65.86|16.46|0.00|872.38|3232.47|3490.58|69.79|0.00|872.38|942.17|-2360.09| +2450834|38403|10154|35993|1512459|2525|7391|1|150|756|84|91.37|107.81|57.13|0.00|4798.92|7675.08|9056.04|335.92|0.00|4798.92|5134.84|-2876.16| +||15218|35993|1512459||||177|756|4|16.94||1.66||||133.48|0.44|||5.36|-62.84| +||7393|||||||756|58||||0.00||2977.72||155.41|0.00||2745.69|-387.44| +2450834|38403|6445|35993|1512459|2525|7391|1|70|756|76|38.89|66.50|5.98|0.00|454.48|2955.64|5054.00|9.08|0.00|454.48|463.56|-2501.16| +2450834|38403|15230|35993|1512459|2525|7391|1|185|756|90|81.75|152.87|9.17|0.00|825.30|7357.50|13758.30|33.01|0.00|825.30|858.31|-6532.20| +2450834|38403|9853|35993|1512459|2525|7391|1|243|756|100|23.95|39.03|20.29|1014.50|2029.00|2395.00|3903.00|71.01|1014.50|1014.50|1085.51|-1380.50| +2450834|38403|8266|35993|1512459|2525|7391|1|97|756|2|14.53|22.08|12.58|13.83|25.16|29.06|44.16|0.11|13.83|11.33|11.44|-17.73| +2450834|38403|7844|35993|1512459|2525|7391|1|217|756|58|16.57|31.98|8.95|0.00|519.10|961.06|1854.84|41.52|0.00|519.10|560.62|-441.96| +2450834|38403|12622|35993|1512459|2525|7391|1|216|756|96|2.72|4.27|0.25|0.00|24.00|261.12|409.92|1.44|0.00|24.00|25.44|-237.12| +2450940|63662|12631|94445|1327210|4775|3140|7|117|757|28|88.18|173.71|55.58|871.49|1556.24|2469.04|4863.88|41.08|871.49|684.75|725.83|-1784.29| +2450940|63662|9061|94445|1327210|4775|3140|7|100|757|62|62.15|90.11|56.76|0.00|3519.12|3853.30|5586.82|316.72|0.00|3519.12|3835.84|-334.18| +2450940|63662|13354|94445|1327210|4775|3140|7|43|757|46|97.37|160.66|118.88|0.00|5468.48|4479.02|7390.36|109.36|0.00|5468.48|5577.84|989.46| +2450940|63662|5791|94445|1327210|4775|3140|7|68|757|30|21.81|32.49|30.54|0.00|916.20|654.30|974.70|64.13|0.00|916.20|980.33|261.90| +2450940|63662|622|94445|1327210|4775|3140|7|54|757|64|65.55|112.09|102.00|2937.60|6528.00|4195.20|7173.76|71.80|2937.60|3590.40|3662.20|-604.80| +2450940|63662|8347|94445|1327210|4775|3140|7|28|757|71|89.71|89.71|21.53|0.00|1528.63|6369.41|6369.41|76.43|0.00|1528.63|1605.06|-4840.78| +2450940|63662|7210|94445|1327210|4775|3140|7|235|757|99|34.52|57.99|27.83|854.10|2755.17|3417.48|5741.01|57.03|854.10|1901.07|1958.10|-1516.41| +2450940|63662|17407|94445|1327210|4775|3140|7|263|757|39|29.99|51.58|23.21|0.00|905.19|1169.61|2011.62|54.31|0.00|905.19|959.50|-264.42| +2450859|58017|14582|13608|1615751|7047|19543|8|237|758|85|53.18|64.87|19.46|0.00|1654.10|4520.30|5513.95|16.54|0.00|1654.10|1670.64|-2866.20| +2450859|58017|10976|13608|1615751|7047|19543|8|111|758|2|6.51|8.72|8.45|2.70|16.90|13.02|17.44|1.13|2.70|14.20|15.33|1.18| +2450859|58017|7570|13608|1615751|7047|19543|8|256|758|19|36.71|63.87|19.16|0.00|364.04|697.49|1213.53|32.76|0.00|364.04|396.80|-333.45| +2450859|58017|1250|13608|1615751|7047|19543|8|2|758|7|25.07|28.57|17.71|0.00|123.97|175.49|199.99|6.19|0.00|123.97|130.16|-51.52| +2450859|58017|5785|13608|1615751|7047|19543|8|293|758|52|56.82|104.54|68.99|0.00|3587.48|2954.64|5436.08|0.00|0.00|3587.48|3587.48|632.84| +2450859|58017|2719|13608|1615751|7047|19543|8|119|758|66|78.41|93.30|68.10|0.00|4494.60|5175.06|6157.80|134.83|0.00|4494.60|4629.43|-680.46| +2450859|58017|10898|13608|1615751|7047|19543|8|194|758|16|18.81|24.45|18.58|0.00|297.28|300.96|391.20|2.97|0.00|297.28|300.25|-3.68| +2450859|58017|4306|13608|1615751|7047|19543|8|231|758|87|21.47|22.11|19.67|1437.48|1711.29|1867.89|1923.57|0.00|1437.48|273.81|273.81|-1594.08| +2450859|58017|9004|13608|1615751|7047|19543|8|196|758|75|95.33|171.59|82.36|3026.73|6177.00|7149.75|12869.25|189.01|3026.73|3150.27|3339.28|-3999.48| +2450859|58017|2984|13608|1615751|7047|19543|8|88|758|35|47.79|61.17|9.17|0.00|320.95|1672.65|2140.95|12.83|0.00|320.95|333.78|-1351.70| +2450859|58017|11248|13608|1615751|7047|19543|8|123|758|54|32.98|57.38|14.91|0.00|805.14|1780.92|3098.52|8.05|0.00|805.14|813.19|-975.78| +2450859|58017|9520|13608|1615751|7047|19543|8|62|758|67|63.25|101.83|97.75|0.00|6549.25|4237.75|6822.61|523.94|0.00|6549.25|7073.19|2311.50| +2450859|58017|6830|13608|1615751|7047|19543|8|103|758|97|90.80|148.91|93.81|0.00|9099.57|8807.60|14444.27|636.96|0.00|9099.57|9736.53|291.97| +2450859||2138|||7047|19543|8|219|758|41||18.56||||738.82||||364.90|386.79|| +2450859|58017|181|13608|1615751|7047|19543|8|206|758|79|5.20|7.07|4.52|0.00|357.08|410.80|558.53|0.00|0.00|357.08|357.08|-53.72| +2451436|71375|10906|78937|1088298|3706|14667|7|142|759|3|76.08|152.16|114.12|0.00|342.36|228.24|456.48|30.81|0.00|342.36|373.17|114.12| +2451436|71375|475|78937|1088298|3706|14667|7|65|759|90|93.95|139.04|136.25|245.25|12262.50|8455.50|12513.60|120.17|245.25|12017.25|12137.42|3561.75| +|71375|12475|78937|1088298|3706||7|260|759|33|89.32|164.34||0.00|||5423.22|28.74|0.00|2874.30|2903.04|-73.26| +|71375|16556|78937|1088298|3706|14667|||759|||36.24||0.00|474.72|1667.04|2500.56|4.74|0.00|474.72||| +2451436|71375|12919|78937|1088298|3706|14667|7|227|759|98|27.50|33.27|29.61|0.00|2901.78|2695.00|3260.46|87.05|0.00|2901.78|2988.83|206.78| +2451436|71375|13735|78937|1088298|3706|14667|7|155|759|11|15.08|23.82|6.90|38.70|75.90|165.88|262.02|2.23|38.70|37.20|39.43|-128.68| +2451436|71375|2800|78937|1088298|3706|14667|7|119|759|75|56.53|96.66|79.26|0.00|5944.50|4239.75|7249.50|237.78|0.00|5944.50|6182.28|1704.75| +2451436|71375|14248|78937|1088298|3706|14667|7|113|759|14|46.41|90.96|80.04|0.00|1120.56|649.74|1273.44|0.00|0.00|1120.56|1120.56|470.82| +2451436|71375|13504|78937|1088298|3706|14667|7|94|759|47|51.27|80.49|44.26|0.00|2080.22|2409.69|3783.03|166.41|0.00|2080.22|2246.63|-329.47| +2451436|71375|17854|78937|1088298|3706|14667|7|197|759|37|58.05|73.72|51.60|477.30|1909.20|2147.85|2727.64|85.91|477.30|1431.90|1517.81|-715.95| +2451436|71375|3016|78937|1088298|3706|14667|7|83|759|47|98.85|130.48|123.95|0.00|5825.65|4645.95|6132.56|466.05|0.00|5825.65|6291.70|1179.70| +2451157|34206|12073|47287|1118741|6548|46002|2|65|760|65|17.32|17.49|14.86|0.00|965.90|1125.80|1136.85|38.63|0.00|965.90|1004.53|-159.90| +2451157|34206|1321|47287|1118741|6548|46002|2|154|760|17|80.84|151.17|117.91|1643.66|2004.47|1374.28|2569.89|7.21|1643.66|360.81|368.02|-1013.47| +2451157|34206|15890|47287|1118741|6548|46002|2|258|760|50|65.93|96.25|32.72|0.00|1636.00|3296.50|4812.50|130.88|0.00|1636.00|1766.88|-1660.50| +2451157|34206|7933|47287|1118741|6548|46002|2|186|760|4|53.30|75.68|71.13|0.00|284.52|213.20|302.72|22.76|0.00|284.52|307.28|71.32| +2451157|34206|2870|47287|1118741|6548|46002|2|3|760|67|3.88|4.61|1.52|0.00|101.84|259.96|308.87|5.09|0.00|101.84|106.93|-158.12| +2451157|34206|12266|47287|1118741|6548|46002|2|9|760|97|36.20|57.55|51.79|0.00|5023.63|3511.40|5582.35|150.70|0.00|5023.63|5174.33|1512.23| +2451157|34206|230|47287|1118741|6548|46002|2|216|760|29|67.11|134.22|5.36|0.00|155.44|1946.19|3892.38|3.10|0.00|155.44|158.54|-1790.75| +2451157|34206|9187|47287|1118741|6548|46002|2|147|760|7|51.54|84.52|73.53|0.00|514.71|360.78|591.64|25.73|0.00|514.71|540.44|153.93| +2451157|34206|6476|47287|1118741|6548|46002|2|82|760|29|49.15|49.15|41.28|0.00|1197.12|1425.35|1425.35|35.91|0.00|1197.12|1233.03|-228.23| +2451157|34206|1810|47287|1118741|6548|46002|2|56|760|19|92.05|164.76|141.69|2611.34|2692.11|1748.95|3130.44|7.26|2611.34|80.77|88.03|-1668.18| +2451157|34206|5170|47287|1118741|6548|46002|2|46|760|43|43.06|84.39|54.85|2334.96|2358.55|1851.58|3628.77|2.12|2334.96|23.59|25.71|-1827.99| +2451157|34206|5998|47287|1118741|6548|46002|2|141|760|21|75.84|94.80|49.29|0.00|1035.09|1592.64|1990.80|51.75|0.00|1035.09|1086.84|-557.55| +2451157|34206|14368|47287|1118741|6548|46002|2|115|760|7|58.56|101.30|62.80|74.73|439.60|409.92|709.10|10.94|74.73|364.87|375.81|-45.05| +2451157|34206|14396|47287|1118741|6548|46002|2|109|760|82|28.17|44.22|0.88|0.00|72.16|2309.94|3626.04|0.00|0.00|72.16|72.16|-2237.78| +2451157|34206|11492|47287|1118741|6548|46002|2|127|760|82|24.72|40.54|6.89|0.00|564.98|2027.04|3324.28|28.24|0.00|564.98|593.22|-1462.06| +2452603|37505|486|84823|1796810|5150|35090|4|137|761|27|65.74|84.14|0.84|0.00|22.68|1774.98|2271.78|0.22|0.00|22.68|22.90|-1752.30| +2452603|37505|8910|84823|1796810|5150|35090|4|145|761|4|12.47|16.95|6.44|0.00|25.76|49.88|67.80|1.03|0.00|25.76|26.79|-24.12| +2452603|37505|5892|84823|1796810|5150|35090|4|286|761|83|28.16|30.69|14.73|0.00|1222.59|2337.28|2547.27|48.90|0.00|1222.59|1271.49|-1114.69| +2452603|37505|17299|84823|1796810|5150|35090|4|181|761|45|52.72|66.42|51.80|0.00|2331.00|2372.40|2988.90|116.55|0.00|2331.00|2447.55|-41.40| +2452603|37505|4423|84823|1796810|5150|35090|4|238|761|15|53.74|82.75|80.26|0.00|1203.90|806.10|1241.25|36.11|0.00|1203.90|1240.01|397.80| +2452603|37505|14103|84823|1796810|5150|35090|4|193|761|60|11.71|20.25|6.88|0.00|412.80|702.60|1215.00|0.00|0.00|412.80|412.80|-289.80| +2452603|37505|7627|84823|1796810|5150|35090|4|107|761|40|92.43|129.40|20.70|0.00|828.00|3697.20|5176.00|8.28|0.00|828.00|836.28|-2869.20| +2452603|37505|45|84823|1796810|5150|35090|4|286|761|79|99.68|126.59|82.28|0.00|6500.12|7874.72|10000.61|0.00|0.00|6500.12|6500.12|-1374.60| +2452603|37505|14820|84823|1796810|5150|35090|4|43|761|7|26.83|53.66|25.22|0.00|176.54|187.81|375.62|1.76|0.00|176.54|178.30|-11.27| +2452603|37505|2127|84823|1796810|5150|35090|4|201|761|30|65.33|78.39|61.92|0.00|1857.60|1959.90|2351.70|55.72|0.00|1857.60|1913.32|-102.30| +2452603|37505|6147|84823|1796810|5150|35090|4|246|761|63|14.62|27.33|5.73|0.00|360.99|921.06|1721.79|32.48|0.00|360.99|393.47|-560.07| +2452603|37505|9141|84823|1796810|5150|35090|4|232|761|18|14.50|16.96|16.28|0.00|293.04|261.00|305.28|23.44|0.00|293.04|316.48|32.04| +2452603|37505|2797|84823|1796810|5150|35090|4|43|761|32|61.63|69.02|57.97|0.00|1855.04|1972.16|2208.64|55.65|0.00|1855.04|1910.69|-117.12| +2452603|37505|13704|84823|1796810|5150|35090|4|212|761|84|10.66|21.21|2.33|0.00|195.72|895.44|1781.64|17.61|0.00|195.72|213.33|-699.72| +2451154|38427|116|58002|913213|3977|15188|1|186|762|93|75.72|122.66|45.38|0.00|4220.34|7041.96|11407.38|379.83|0.00|4220.34|4600.17|-2821.62| +2451154|38427|12820|58002|913213|3977|15188|1|247|762|61|66.79|78.14|0.00|0.00|0.00|4074.19|4766.54|0.00|0.00|0.00|0.00|-4074.19| +2451154|38427|13594|58002|913213|3977|15188|1|252|762|55|77.25|98.10|9.81|0.00|539.55|4248.75|5395.50|43.16|0.00|539.55|582.71|-3709.20| +2451154|38427|878|58002|913213|3977|15188|1|279|762|88|49.71|54.68|18.04|0.00|1587.52|4374.48|4811.84|127.00|0.00|1587.52|1714.52|-2786.96| +2451154|38427|601|58002|913213|3977|15188|1|281|762|1|8.65|12.62|5.42|0.00|5.42|8.65|12.62|0.10|0.00|5.42|5.52|-3.23| +2451154|38427|7754|58002|913213|3977|15188|1|256|762|100|83.15|146.34|52.68|0.00|5268.00|8315.00|14634.00|210.72|0.00|5268.00|5478.72|-3047.00| +2451154|38427|10663|58002|913213|3977|15188|1|188|762|54|80.58|151.49|140.88|0.00|7607.52|4351.32|8180.46|456.45|0.00|7607.52|8063.97|3256.20| +2451154|38427|10495|58002|913213|3977|15188|1|20|762|85|70.00|107.80|28.02|0.00|2381.70|5950.00|9163.00|119.08|0.00|2381.70|2500.78|-3568.30| +2451154|38427|6592|58002|913213|3977|||10|762|45|39.12||51.25||2306.25|1760.40||||322.88|339.02|-1437.52| +2451154|38427|17648|58002|913213|3977|15188|1|79|762|53|83.26|103.24|25.81|0.00|1367.93|4412.78|5471.72|13.67|0.00|1367.93|1381.60|-3044.85| +2452586|52952|11781|13232|136262|5245|31428|8|86|763|87|5.83|6.41|4.03|0.00|350.61|507.21|557.67|14.02|0.00|350.61|364.63|-156.60| +2452586|52952|11323|13232|136262|5245|31428|8|118|763|20|12.23|20.30|4.87|0.00|97.40|244.60|406.00|8.76|0.00|97.40|106.16|-147.20| +2452586|52952|15735|13232|136262|5245|31428|8|124|763|19|75.28|115.93|19.70|0.00|374.30|1430.32|2202.67|26.20|0.00|374.30|400.50|-1056.02| +2452586|52952|1356|13232|136262|5245|31428|8|254|763|14|18.65|36.55|28.14|0.00|393.96|261.10|511.70|0.00|0.00|393.96|393.96|132.86| +2452586|52952|9277|13232|136262|5245|31428|8|96|763|97|50.25|65.32|36.57|0.00|3547.29|4874.25|6336.04|177.36|0.00|3547.29|3724.65|-1326.96| +2452586|52952|12795|13232|136262|5245|31428|8|131|763|15|85.19|142.26|122.34|0.00|1835.10|1277.85|2133.90|0.00|0.00|1835.10|1835.10|557.25| +2452586|52952|579|13232|136262|5245|31428|8|102|763|47|24.09|31.55|22.08|799.07|1037.76|1132.23|1482.85|11.93|799.07|238.69|250.62|-893.54| +2452586|52952|55|13232|136262|5245|31428|8|77|763|16|58.30|105.52|97.07|0.00|1553.12|932.80|1688.32|46.59|0.00|1553.12|1599.71|620.32| +2452586|52952|12943|13232|136262|5245|31428|8|208|763|20|90.27|158.87|41.30|0.00|826.00|1805.40|3177.40|57.82|0.00|826.00|883.82|-979.40| +2452586|52952|11304|13232|136262|5245|31428|8|299|763|31|48.59|92.80|68.67|0.00|2128.77|1506.29|2876.80|149.01|0.00|2128.77|2277.78|622.48| +2452586|52952|16440|13232|136262|5245|31428|8|114|763|67|23.45|25.79|3.61|0.00|241.87|1571.15|1727.93|21.76|0.00|241.87|263.63|-1329.28| +2452586|52952|16533|13232|136262|5245|31428|8|286|763|69|17.96|20.65|20.44|0.00|1410.36|1239.24|1424.85|0.00|0.00|1410.36|1410.36|171.12| +2452586|52952|13699|13232|136262|5245|31428|8|277|763|38|38.36|48.33|42.53|0.00|1616.14|1457.68|1836.54|0.00|0.00|1616.14|1616.14|158.46| +2452586|52952|5715|13232|136262|5245|31428|8|228|763|15|82.36|107.06|2.14|14.76|32.10|1235.40|1605.90|0.00|14.76|17.34|17.34|-1218.06| +2452491|49805|3801||1337706|4162|14971|||764||58.38|86.98|2.60||137.80|3094.14|4609.94|1.37||137.80||-2956.34| +2452491|49805|4011|69650|1337706|4162|14971|1|124|764|46|34.58|41.15|38.26|0.00|1759.96|1590.68|1892.90|158.39|0.00|1759.96|1918.35|169.28| +2452491|49805|4561|69650|1337706|4162|14971|1|198|764|23|72.87|120.96|58.06|0.00|1335.38|1676.01|2782.08|53.41|0.00|1335.38|1388.79|-340.63| +2452491|49805|780|69650|1337706|4162|14971|1|269|764|27|44.06|57.27|19.47|278.61|525.69|1189.62|1546.29|9.88|278.61|247.08|256.96|-942.54| +2452491|49805|11059|69650|1337706|4162|14971|1|32|764|45|52.78|99.22|8.92|0.00|401.40|2375.10|4464.90|32.11|0.00|401.40|433.51|-1973.70| +||2082||1337706|4162|14971|1|239|764|61|89.92||||392.23|5485.12|9817.95|3.92||392.23|396.15|| +2452491|49805|1753|69650|1337706|4162|14971|1|197|764|58|46.27|85.13|83.42|0.00|4838.36|2683.66|4937.54|338.68|0.00|4838.36|5177.04|2154.70| +2452491|49805|1323|69650|1337706|4162|14971|1|190|764|43|74.76|90.45|83.21|0.00|3578.03|3214.68|3889.35|0.00|0.00|3578.03|3578.03|363.35| +2452491|49805|16471|69650|1337706|4162|14971|1|72|764|52|43.44|56.90|44.95|1355.69|2337.40|2258.88|2958.80|88.35|1355.69|981.71|1070.06|-1277.17| +2452491|49805|1455|69650|1337706|4162|14971|1|10|764|15|7.90|12.48|3.49|0.00|52.35|118.50|187.20|1.04|0.00|52.35|53.39|-66.15| +2452491|49805|8310|69650|1337706|4162|14971|1|161|764|96|33.35|66.36|33.18|0.00|3185.28|3201.60|6370.56|254.82|0.00|3185.28|3440.10|-16.32| +2452491|49805|9918|69650|1337706|4162|14971|1|10|764|32|64.29|100.93|65.60|0.00|2099.20|2057.28|3229.76|0.00|0.00|2099.20|2099.20|41.92| +2452040|52036|9923|44625|39818|7011|16251|10|144|765|79|74.61|143.99|21.59|0.00|1705.61|5894.19|11375.21|17.05|0.00|1705.61|1722.66|-4188.58| +2452040|52036|14257|44625|39818|7011|16251|10|47|765|42|46.32|81.52|48.91|0.00|2054.22|1945.44|3423.84|102.71|0.00|2054.22|2156.93|108.78| +2452040|52036|10089|44625|39818|7011|16251|10|184|765|6|97.92|165.48|137.34|0.00|824.04|587.52|992.88|49.44|0.00|824.04|873.48|236.52| +2452040|52036|12535|44625|39818|7011|16251|10|103|765|42|12.81|18.70|9.35|0.00|392.70|538.02|785.40|35.34|0.00|392.70|428.04|-145.32| +2452040|52036|15285|44625|39818|7011|16251|10|138|765|85|27.54|36.62|34.42|0.00|2925.70|2340.90|3112.70|29.25|0.00|2925.70|2954.95|584.80| +2452040|52036|8603|44625|39818|7011|16251|10|234|765|86|93.98|170.10|78.24|0.00|6728.64|8082.28|14628.60|336.43|0.00|6728.64|7065.07|-1353.64| +2452040|52036|17745|44625|39818|7011|16251|10|33|765|43|36.72|52.87|21.14|0.00|909.02|1578.96|2273.41|63.63|0.00|909.02|972.65|-669.94| +2452040|52036|7659|44625|39818|7011|16251|10|42|765|73|80.87|105.93|65.67|0.00|4793.91|5903.51|7732.89|383.51|0.00|4793.91|5177.42|-1109.60| +2452040|52036|14921|44625|39818|7011|16251|10|12|765|64|16.57|21.20|15.26|341.82|976.64|1060.48|1356.80|57.13|341.82|634.82|691.95|-425.66| +2452040|52036|15897|44625|39818|7011|16251|10|12|765|85|67.28|107.64|35.52|0.00|3019.20|5718.80|9149.40|241.53|0.00|3019.20|3260.73|-2699.60| +2451983|41534|10257|62751|122748|4345|9542|2|200|766|34|18.10|25.15|22.38|0.00|760.92|615.40|855.10|53.26|0.00|760.92|814.18|145.52| +2451983|41534|14889|62751|122748|4345|9542|2|291|766|36|75.22|100.79|9.07|0.00|326.52|2707.92|3628.44|19.59|0.00|326.52|346.11|-2381.40| +2451983|41534|7165|62751|122748|4345|9542|2|133|766|67|82.86|145.00|21.75|0.00|1457.25|5551.62|9715.00|87.43|0.00|1457.25|1544.68|-4094.37| +||17811||122748||9542|||766|33||||0.00|1179.09|1871.76|||0.00|1179.09||| +2451983|41534|13019|62751|122748|4345|9542|2|277|766|71|4.04|6.58|1.71|0.00|121.41|286.84|467.18|10.92|0.00|121.41|132.33|-165.43| +2451983|41534|15503|62751|122748|4345|9542|2|16|766|14|45.46|72.28|72.28|607.15|1011.92|636.44|1011.92|24.28|607.15|404.77|429.05|-231.67| +2451983|41534|2043|62751|122748|4345|9542|2|249|766|69|32.50|43.87|23.25|96.25|1604.25|2242.50|3027.03|0.00|96.25|1508.00|1508.00|-734.50| +2451983|41534|1373|62751|122748|4345|9542|2|2|766|64|24.07|37.30|20.14|0.00|1288.96|1540.48|2387.20|103.11|0.00|1288.96|1392.07|-251.52| +2451983|41534|13987|62751|122748|4345|9542|2|196|766|87|46.02|84.21|63.15|0.00|5494.05|4003.74|7326.27|329.64|0.00|5494.05|5823.69|1490.31| +2451983|41534|16831|62751|122748|4345|9542|2|154|766|31|20.92|39.95|15.98|178.33|495.38|648.52|1238.45|12.68|178.33|317.05|329.73|-331.47| +2451983|41534|8637|62751|122748|4345|9542|2|167|766|60|44.71|46.94|30.51|0.00|1830.60|2682.60|2816.40|36.61|0.00|1830.60|1867.21|-852.00| +2451796|51540|14066|87101|1869041|1028|5368|2|161|767|65|16.84|30.14|23.20|0.00|1508.00|1094.60|1959.10|90.48|0.00|1508.00|1598.48|413.40| +2451796|51540|17191|87101|1869041|1028|5368|2|81|767|32|19.08|34.91|6.98|147.41|223.36|610.56|1117.12|2.27|147.41|75.95|78.22|-534.61| +2451796|51540|7925|87101|1869041|1028|5368|2|100|767|73|46.70|56.04|15.69|0.00|1145.37|3409.10|4090.92|11.45|0.00|1145.37|1156.82|-2263.73| +2451796|51540|1789|87101|1869041|1028|5368|2|196|767|79|58.24|62.31|39.87|1669.35|3149.73|4600.96|4922.49|44.41|1669.35|1480.38|1524.79|-3120.58| +2451796|51540|11143|87101|1869041|1028|5368|2|102|767|4|52.11|60.96|40.23|0.00|160.92|208.44|243.84|8.04|0.00|160.92|168.96|-47.52| +2451796|51540|14041|87101|1869041|1028|5368|2|212|767|29|43.03|71.42|65.70|0.00|1905.30|1247.87|2071.18|95.26|0.00|1905.30|2000.56|657.43| +2451796|51540|13291|87101|1869041|1028|5368|2|3|767|61|72.64|127.84|16.61|729.51|1013.21|4431.04|7798.24|11.34|729.51|283.70|295.04|-4147.34| +2451796|51540|12331|87101|1869041|1028|5368|2|220|767|54|41.12|41.53|2.07|0.00|111.78|2220.48|2242.62|1.11|0.00|111.78|112.89|-2108.70| +2451796|51540|650|87101|1869041|1028|5368|2|235|767|92|86.63|136.00|42.16|0.00|3878.72|7969.96|12512.00|271.51|0.00|3878.72|4150.23|-4091.24| +2451796|51540|9224|87101|1869041|1028|5368|2|195|767|93|93.88|104.20|38.55|0.00|3585.15|8730.84|9690.60|179.25|0.00|3585.15|3764.40|-5145.69| +2451796|51540|15919|87101|1869041|1028|5368|2|225|767|62|7.13|11.90|8.68|0.00|538.16|442.06|737.80|48.43|0.00|538.16|586.59|96.10| +2451796|51540|12719|87101|1869041|1028|5368|2|131|767|74|68.11|100.12|48.05|248.89|3555.70|5040.14|7408.88|264.54|248.89|3306.81|3571.35|-1733.33| +2451796|51540|4663|87101|1869041|1028|5368|2|12|767|70|12.84|20.54|17.45|0.00|1221.50|898.80|1437.80|48.86|0.00|1221.50|1270.36|322.70| +2451796|51540|9763|87101|1869041|1028|5368|2|258|767|42|30.26|46.29|3.70|0.00|155.40|1270.92|1944.18|1.55|0.00|155.40|156.95|-1115.52| +2451796|51540|9461|87101|1869041|1028|5368|2|70|767|48|65.34|69.26|41.55|0.00|1994.40|3136.32|3324.48|19.94|0.00|1994.40|2014.34|-1141.92| +2451796|51540|12521|87101|1869041|1028|5368|2|28|767|35|14.84|28.78|23.31|0.00|815.85|519.40|1007.30|16.31|0.00|815.85|832.16|296.45| +2451563|69498|9505|1639|794936|1033|21930|1|169|768|81|99.76|156.62|112.76|0.00|9133.56|8080.56|12686.22|548.01|0.00|9133.56|9681.57|1053.00| +2451563|69498|4397|1639|794936|1033|21930|1|145|768|37|2.79|2.79|1.00|0.00|37.00|103.23|103.23|1.85|0.00|37.00|38.85|-66.23| +2451563|69498|689|1639|794936|1033|21930|1|222|768|58|21.70|41.88|38.52|0.00|2234.16|1258.60|2429.04|44.68|0.00|2234.16|2278.84|975.56| +2451563|69498|1195|1639|794936|1033|21930|1|274|768|62|28.53|54.49|17.43|0.00|1080.66|1768.86|3378.38|43.22|0.00|1080.66|1123.88|-688.20| +2451563|69498|7495|1639|794936|1033|21930|1|83|768|80|7.81|11.01|5.39|0.00|431.20|624.80|880.80|38.80|0.00|431.20|470.00|-193.60| +2451563|69498|1238|1639|794936|1033|21930|1|41|768|11|22.25|42.72|26.91|183.52|296.01|244.75|469.92|4.49|183.52|112.49|116.98|-132.26| +2451563||10085|1639|794936|1033|21930|1||768|||||0.00|5733.42|||286.67|0.00|5733.42||-784.63| +2451563|69498|14261|1639|794936|1033|21930|1|20|768|4|3.06|5.99|1.55|0.00|6.20|12.24|23.96|0.00|0.00|6.20|6.20|-6.04| +2451055|62124|12151|44911|1086078|4845|26412|8|58|769|86|26.81|29.75|24.39|608.28|2097.54|2305.66|2558.50|119.14|608.28|1489.26|1608.40|-816.40| +2451055|62124|10406|44911|1086078|4845|26412|8|160|769|47|50.09|70.62|66.38|0.00|3119.86|2354.23|3319.14|187.19|0.00|3119.86|3307.05|765.63| +2451055|62124|8876|44911|1086078|4845|26412|8|6|769|29|48.90|95.84|70.92|0.00|2056.68|1418.10|2779.36|0.00|0.00|2056.68|2056.68|638.58| +2451055|62124|11683|44911|1086078|4845|26412|8|80|769|37|63.11|108.54|45.58|893.82|1686.46|2335.07|4015.98|63.41|893.82|792.64|856.05|-1542.43| +2451055|62124|2407|44911|1086078|4845|26412|8|130|769|40|40.60|63.74|44.61|0.00|1784.40|1624.00|2549.60|124.90|0.00|1784.40|1909.30|160.40| +||15520|44911|1086078|||8||769|80|87.05|140.15||0.00|0.00|6964.00||0.00|0.00||0.00|-6964.00| +2451055|62124|7070|44911|1086078|4845|26412|8|79|769|99|3.60|4.21|0.16|0.00|15.84|356.40|416.79|1.10|0.00|15.84|16.94|-340.56| +2451055|62124|6104|44911|1086078|4845|26412|8|289|769|19|68.81|90.82|76.28|0.00|1449.32|1307.39|1725.58|86.95|0.00|1449.32|1536.27|141.93| +2451055|62124|14623|44911|1086078|4845|26412|8|34|769|32|26.43|47.30|24.59|306.88|786.88|845.76|1513.60|0.00|306.88|480.00|480.00|-365.76| +2451055|62124|5854|44911|1086078|4845|26412|8|259|769|44|25.02|32.02|20.81|0.00|915.64|1100.88|1408.88|9.15|0.00|915.64|924.79|-185.24| +2451055|62124|17131|44911|1086078|4845|26412|8|46|769|38|13.52|14.73|13.10|328.54|497.80|513.76|559.74|13.54|328.54|169.26|182.80|-344.50| +2451447|42224|13826|142|1650641|1493|37145|8|285|770|73|28.03|33.63|29.93|0.00|2184.89|2046.19|2454.99|65.54|0.00|2184.89|2250.43|138.70| +2451447|42224|15781|142|1650641|1493|37145|8|138|770|88|89.52|118.16|5.90|0.00|519.20|7877.76|10398.08|36.34|0.00|519.20|555.54|-7358.56| +2451447|42224|9214|142|1650641|1493|37145|8|124|770|67|39.84|75.29|59.47|0.00|3984.49|2669.28|5044.43|358.60|0.00|3984.49|4343.09|1315.21| +2451447|42224|10543|142|1650641|1493|37145|8|168|770|43|82.84|147.45|120.90|0.00|5198.70|3562.12|6340.35|207.94|0.00|5198.70|5406.64|1636.58| +2451447|42224|3583|142|1650641|1493|37145|8|296|770|99|89.82|126.64|36.72|1090.58|3635.28|8892.18|12537.36|127.23|1090.58|2544.70|2671.93|-6347.48| +2451447|42224|2173|142|1650641|1493|37145|8|104|770|55|76.39|124.51|92.13|0.00|5067.15|4201.45|6848.05|304.02|0.00|5067.15|5371.17|865.70| +2451447|42224|4228|142|1650641|1493|37145|8|49|770|81|74.19|87.54|12.25|0.00|992.25|6009.39|7090.74|89.30|0.00|992.25|1081.55|-5017.14| +2451447|42224|15392|142|1650641|1493|37145|8|34|770|65|91.29|127.80|11.50|0.00|747.50|5933.85|8307.00|67.27|0.00|747.50|814.77|-5186.35| +2451447|42224|740|142|1650641|1493|37145|8|220|770|2|20.13|38.44|22.29|0.00|44.58|40.26|76.88|2.67|0.00|44.58|47.25|4.32| +2452360|70810|2757|866|822353|3393|32480|7|32|771|41|47.14|69.76|10.46|390.26|428.86|1932.74|2860.16|3.08|390.26|38.60|41.68|-1894.14| +2452360|70810|14227|866|822353|3393|32480|7|39|771|65|90.18|162.32|87.65|0.00|5697.25|5861.70|10550.80|341.83|0.00|5697.25|6039.08|-164.45| +2452360||16567|866|822353||32480||211|771||91.25|||258.92|2353.86|||62.84|258.92|2094.94|2157.78|| +2452360|70810|15303|866|822353|3393|32480|7|255|771|45|22.36|33.31|29.97|0.00|1348.65|1006.20|1498.95|80.91|0.00|1348.65|1429.56|342.45| +2452360|70810|5331|866|822353|3393|32480|7|84|771|31|66.81|101.55|34.52|620.66|1070.12|2071.11|3148.05|13.48|620.66|449.46|462.94|-1621.65| +|70810|1704||||32480|7||771|||107.48|||||8490.92|360.83||||2626.75| +2452360|70810|1776|866|822353|3393|32480|7|95|771|47|95.05|176.79|132.59|0.00|6231.73|4467.35|8309.13|0.00|0.00|6231.73|6231.73|1764.38| +2452360||13701|866|||32480|7|60|771||42.93|51.08|4.08||106.08||1328.08|4.24|||110.32|-1010.10| +2452360|70810|15513|866|822353|3393|32480|7|69|771|57|64.30|128.60|102.88|0.00|5864.16|3665.10|7330.20|527.77|0.00|5864.16|6391.93|2199.06| +2451808|47438|10075|69942|799672|1849|10779|1|69|772|67|56.44|71.67|29.38|0.00|1968.46|3781.48|4801.89|19.68|0.00|1968.46|1988.14|-1813.02| +2451808|47438|7082|69942|799672|1849|10779|1|3|772|77|66.27|111.99|103.03|0.00|7933.31|5102.79|8623.23|475.99|0.00|7933.31|8409.30|2830.52| +2451808|47438|9541|69942|799672|1849|10779|1|118|772|49|27.75|37.46|11.98|0.00|587.02|1359.75|1835.54|11.74|0.00|587.02|598.76|-772.73| +|47438|10067||799672|1849||||772|70|93.94|171.91||450.97|||12033.70||450.97|150.33||| +2451808|47438|353|69942|799672|1849|10779|1|187|772|78|69.92|85.30|23.88|1098.95|1862.64|5453.76|6653.40|68.73|1098.95|763.69|832.42|-4690.07| +2451808|47438|12656|69942|799672|1849|10779|1|236|772|30|6.98|10.26|9.23|171.67|276.90|209.40|307.80|4.20|171.67|105.23|109.43|-104.17| +2451808|47438|10766|69942|799672|1849|10779|1|116|772|53|62.49|76.23|53.36|707.02|2828.08|3311.97|4040.19|84.84|707.02|2121.06|2205.90|-1190.91| +2451808|47438|16394|69942|799672|1849|10779|1|224|772|75|31.97|47.63|41.43|0.00|3107.25|2397.75|3572.25|124.29|0.00|3107.25|3231.54|709.50| +2451808|47438|5276|69942|799672|1849|10779|1|174|772|45|88.52|125.69|52.78|878.78|2375.10|3983.40|5656.05|44.88|878.78|1496.32|1541.20|-2487.08| +2451808|47438|16346|69942|799672|1849|10779|1|286|772|90|9.93|10.82|2.48|194.18|223.20|893.70|973.80|2.32|194.18|29.02|31.34|-864.68| +2451808|47438|239|69942|799672|1849|10779|1|86|772|19|57.46|95.95|80.59|0.00|1531.21|1091.74|1823.05|15.31|0.00|1531.21|1546.52|439.47| +2451808|47438|6847|69942|799672|1849|10779|1|264|772|2|98.22|178.76|153.73|0.00|307.46|196.44|357.52|6.14|0.00|307.46|313.60|111.02| +2451808|47438|9659|69942|799672|1849|10779|1|54|772|27|35.98|42.81|24.40|250.34|658.80|971.46|1155.87|8.16|250.34|408.46|416.62|-563.00| +2451808|47438|14329|69942|799672|1849|10779|1|7|772|41|81.95|131.12|41.95|0.00|1719.95|3359.95|5375.92|68.79|0.00|1719.95|1788.74|-1640.00| +2451808|47438|11471|69942|799672|1849|10779|1|164|772|28|39.71|50.82|41.67|0.00|1166.76|1111.88|1422.96|81.67|0.00|1166.76|1248.43|54.88| +2451877|67916|16387|51638|22372|5906|38640|7|192|773|96|44.80|47.93|28.75|0.00|2760.00|4300.80|4601.28|220.80|0.00|2760.00|2980.80|-1540.80| +2451877|67916|1868|51638|22372|5906|38640|7|289|773|98|90.33|123.75|87.86|0.00|8610.28|8852.34|12127.50|688.82|0.00|8610.28|9299.10|-242.06| +2451877|67916|6499|51638|22372|5906|38640|7|112|773|36|33.84|49.74|4.97|0.00|178.92|1218.24|1790.64|16.10|0.00|178.92|195.02|-1039.32| +2451877|67916|12283|51638|22372|5906|38640|7|1|773|19|39.97|50.36|45.32|0.00|861.08|759.43|956.84|60.27|0.00|861.08|921.35|101.65| +2451877|67916|7523|51638|22372|5906|38640|7|203|773|37|7.68|8.90|3.91|0.00|144.67|284.16|329.30|7.23|0.00|144.67|151.90|-139.49| +2451877|67916|12746|51638|22372|5906|38640|7|30|773|37|45.61|62.48|24.99|0.00|924.63|1687.57|2311.76|18.49|0.00|924.63|943.12|-762.94| +2451877|67916|6746|51638|22372|5906|38640|7|265|773|10|52.08|52.60|19.98|0.00|199.80|520.80|526.00|7.99|0.00|199.80|207.79|-321.00| +2451877|67916|4382|51638|22372|5906|38640|7|26|773|91|79.82|146.86|8.81|601.28|801.71|7263.62|13364.26|18.03|601.28|200.43|218.46|-7063.19| +2451877|67916|13022|51638|22372|5906|38640|7|92|773|40|2.26|3.88|3.37|0.00|134.80|90.40|155.20|2.69|0.00|134.80|137.49|44.40| +2451877|67916|8069|51638|22372|5906|38640|7|132|773|64|17.71|21.78|19.16|0.00|1226.24|1133.44|1393.92|24.52|0.00|1226.24|1250.76|92.80| +|67916|9469||||38640||62|773||13.51|24.72|22.49||||2249.52|||2046.59|2148.91|817.18| +2451877|67916|10003|51638|22372|5906|38640|7|86|773|26|81.66|133.10|93.17|0.00|2422.42|2123.16|3460.60|145.34|0.00|2422.42|2567.76|299.26| +2451877|67916|12617|51638|22372|5906|38640|7|237|773|5|64.69|97.68|31.25|151.56|156.25|323.45|488.40|0.00|151.56|4.69|4.69|-318.76| +2451755|39139|14047|54354|454823|6045|36867|10|58|774|60|21.31|36.44|24.05|0.00|1443.00|1278.60|2186.40|43.29|0.00|1443.00|1486.29|164.40| +2451755|39139|16273|54354|454823|6045|36867|10|264|774|78|28.57|39.14|31.31|0.00|2442.18|2228.46|3052.92|219.79|0.00|2442.18|2661.97|213.72| +2451755|39139|16697|54354|454823|6045|36867|10|264|774|14|13.41|19.84|15.87|0.00|222.18|187.74|277.76|19.99|0.00|222.18|242.17|34.44| +2451755|39139|1345|54354|454823|6045|36867|10|156|774|39|89.68|141.69|41.09|0.00|1602.51|3497.52|5525.91|80.12|0.00|1602.51|1682.63|-1895.01| +2451755|39139|12035|54354|454823|6045|36867|10|244|774|54|40.14|52.58|22.60|0.00|1220.40|2167.56|2839.32|48.81|0.00|1220.40|1269.21|-947.16| +2451755||14231|||6045|||169|774||51.06||6.33|0.00|183.57|1480.74|2295.06|16.52|0.00|183.57||-1297.17| +2451755|39139|14375|54354|454823|6045|36867|10|78|774|16|92.77|145.64|1.45|0.00|23.20|1484.32|2330.24|2.08|0.00|23.20|25.28|-1461.12| +2451755|39139|11341|54354|454823|6045|36867|10|86|774|46|28.01|45.09|40.13|0.00|1845.98|1288.46|2074.14|36.91|0.00|1845.98|1882.89|557.52| +2451755|39139|2582|54354|454823|6045|36867|10|140|774|39|79.85|104.60|34.51|403.76|1345.89|3114.15|4079.40|9.42|403.76|942.13|951.55|-2172.02| +2451755|39139|5285|54354|454823|6045|36867|10|243|774|84|10.49|14.79|12.57|0.00|1055.88|881.16|1242.36|42.23|0.00|1055.88|1098.11|174.72| +2451755|39139|14654|54354|454823|6045|36867|10|186|774|19|17.70|34.51|12.76|0.00|242.44|336.30|655.69|9.69|0.00|242.44|252.13|-93.86| +2451755|39139|272|54354|454823|6045|36867|10|82|774|8|1.74|1.98|0.89|0.00|7.12|13.92|15.84|0.35|0.00|7.12|7.47|-6.80| +2451755|39139|9431|54354|454823|6045|36867|10|168|774|34|85.32|127.12|124.57|0.00|4235.38|2900.88|4322.08|296.47|0.00|4235.38|4531.85|1334.50| +2451755|39139|15668|54354|454823|6045|36867|10|106|774|48|48.58|67.04|14.74|0.00|707.52|2331.84|3217.92|56.60|0.00|707.52|764.12|-1624.32| +2451755|39139|13535|54354|454823|6045|36867|10|212|774|73|89.20|160.56|38.53|1912.62|2812.69|6511.60|11720.88|27.00|1912.62|900.07|927.07|-5611.53| +2451755|39139|3098|54354|454823|6045|36867|10|176|774|79|38.02|48.66|32.11|0.00|2536.69|3003.58|3844.14|101.46|0.00|2536.69|2638.15|-466.89| +2451339|35178|7262|60335|1351296|3057|11618|4|222|775|89|65.48|113.93|1.13|0.00|100.57|5827.72|10139.77|6.03|0.00|100.57|106.60|-5727.15| +2451339|35178|8233|60335|1351296|3057|11618|4|48|775|92|64.61|123.40|108.59|1798.25|9990.28|5944.12|11352.80|655.36|1798.25|8192.03|8847.39|2247.91| +2451339|35178|271|60335|1351296|3057|11618|4|40|775|1|10.46|20.71|9.11|0.00|9.11|10.46|20.71|0.63|0.00|9.11|9.74|-1.35| +2451339|35178|16591|60335|1351296|3057|11618|4|176|775|45|91.67|111.83|29.07|0.00|1308.15|4125.15|5032.35|65.40|0.00|1308.15|1373.55|-2817.00| +2451339|35178|17510|60335|1351296|3057|11618|4|6|775|85|63.95|125.98|39.05|0.00|3319.25|5435.75|10708.30|199.15|0.00|3319.25|3518.40|-2116.50| +2451339|35178|2620|60335|1351296|3057|11618|4|107|775|52|33.24|33.90|33.22|0.00|1727.44|1728.48|1762.80|69.09|0.00|1727.44|1796.53|-1.04| +2451339|35178|1885|60335|1351296|3057|11618|4|176|775|96|85.44|152.08|21.29|0.00|2043.84|8202.24|14599.68|102.19|0.00|2043.84|2146.03|-6158.40| +2451339|35178|2272|60335|1351296|3057|11618|4|74|775|27|46.25|86.48|16.43|0.00|443.61|1248.75|2334.96|0.00|0.00|443.61|443.61|-805.14| +2451339|35178|14275|60335|1351296|3057|11618|4|89|775|88|89.35|95.60|59.27|0.00|5215.76|7862.80|8412.80|260.78|0.00|5215.76|5476.54|-2647.04| +2451339|35178|3148|60335|1351296|3057|11618|4|104|775|19|48.58|57.81|32.37|0.00|615.03|923.02|1098.39|43.05|0.00|615.03|658.08|-307.99| +2451339|35178|100|60335|1351296|3057|11618|4|218|775|75|34.39|68.78|17.19|747.76|1289.25|2579.25|5158.50|37.90|747.76|541.49|579.39|-2037.76| +2451339|35178|17206|60335|1351296|3057|11618|4|33|775|78|5.74|9.12|4.74|0.00|369.72|447.72|711.36|7.39|0.00|369.72|377.11|-78.00| +2451339|35178|4303|60335|1351296|3057|11618|4|229|775|2|55.56|89.45|25.94|0.00|51.88|111.12|178.90|2.07|0.00|51.88|53.95|-59.24| +2451854|58113|11546|49037|1076062|4334|24959|8|277|776|54|15.81|16.75|4.35|0.00|234.90|853.74|904.50|11.74|0.00|234.90|246.64|-618.84| +2451854|58113|12236|49037|1076062|4334|24959|8|203|776|96|36.26|65.63|13.78|0.00|1322.88|3480.96|6300.48|66.14|0.00|1322.88|1389.02|-2158.08| +2451854|58113|2671|49037|1076062|4334|24959|8|238|776|92|92.11|127.11|49.57|0.00|4560.44|8474.12|11694.12|0.00|0.00|4560.44|4560.44|-3913.68| +2451854|58113|15025|49037|1076062|4334|24959|8|105|776|94|21.44|34.94|3.84|0.00|360.96|2015.36|3284.36|0.00|0.00|360.96|360.96|-1654.40| +2451854|58113|2137|49037|1076062|4334|24959|8|126|776|55|37.35|62.37|55.50|0.00|3052.50|2054.25|3430.35|244.20|0.00|3052.50|3296.70|998.25| +2451854|58113|16028|49037|1076062|4334|24959|8|95|776|30|19.06|21.34|9.17|0.00|275.10|571.80|640.20|24.75|0.00|275.10|299.85|-296.70| +||7139||1076062||||223|776||13.08||7.53|302.32|549.69||1222.02||302.32|||| +2451854|58113|11441|49037|1076062|4334|24959|8|31|776|43|9.72|10.20|10.09|0.00|433.87|417.96|438.60|26.03|0.00|433.87|459.90|15.91| +2451854|58113|10757|49037|1076062|4334|24959|8|189|776|48|78.34|119.07|14.28|0.00|685.44|3760.32|5715.36|34.27|0.00|685.44|719.71|-3074.88| +2451854|58113|1304|49037|1076062|4334|24959|8|258|776|80|82.01|152.53|132.70|849.28|10616.00|6560.80|12202.40|586.00|849.28|9766.72|10352.72|3205.92| +2451854|58113|12749|49037|1076062|4334|24959|8|78|776|1|42.44|81.90|71.25|0.00|71.25|42.44|81.90|2.13|0.00|71.25|73.38|28.81| +2451854|58113|11509|49037|1076062|4334|24959|8|9|776|85|46.43|49.21|45.76|0.00|3889.60|3946.55|4182.85|272.27|0.00|3889.60|4161.87|-56.95| +2451854|58113|16418|49037|1076062|4334|24959|8|275|776|68|62.76|115.47|31.17|0.00|2119.56|4267.68|7851.96|127.17|0.00|2119.56|2246.73|-2148.12| +2451854|58113|13885|49037|1076062|4334|24959|8|245|776|29|16.31|26.25|4.72|0.00|136.88|472.99|761.25|12.31|0.00|136.88|149.19|-336.11| +2451854|58113|8167|49037|1076062|4334|24959|8|263|776|46|96.99|158.09|26.87|0.00|1236.02|4461.54|7272.14|37.08|0.00|1236.02|1273.10|-3225.52| +2452273|60471|6893|59456|1144701|5946|44791|10|238|777|66|78.84|104.06|30.17|0.00|1991.22|5203.44|6867.96|0.00|0.00|1991.22|1991.22|-3212.22| +2452273|60471|5289|59456|1144701|5946|44791|10|220|777|48|98.12|116.76|58.38|0.00|2802.24|4709.76|5604.48|28.02|0.00|2802.24|2830.26|-1907.52| +2452273|60471|15225|59456|1144701|5946|44791|10|15|777|80|34.47|56.18|6.74|37.74|539.20|2757.60|4494.40|25.07|37.74|501.46|526.53|-2256.14| +2452273|60471|2487|59456|1144701|5946|44791|10|125|777|94|53.33|94.39|79.28|0.00|7452.32|5013.02|8872.66|372.61|0.00|7452.32|7824.93|2439.30| +2452273|60471|7341|59456|1144701|5946|44791|10|151|777|36|63.10|120.52|21.69|0.00|780.84|2271.60|4338.72|46.85|0.00|780.84|827.69|-1490.76| +2452273|60471|3145|59456|1144701|5946|44791|10|288|777|13|36.55|52.99|38.68|0.00|502.84|475.15|688.87|0.00|0.00|502.84|502.84|27.69| +|60471|15271|59456||5946|||245|777|18||117.78|5.88||105.84|||2.11||||-1590.30| +2452273|60471|1549|59456|1144701|5946|44791|10|298|777|60|48.80|49.77|29.86|0.00|1791.60|2928.00|2986.20|35.83|0.00|1791.60|1827.43|-1136.40| +2452273|60471|17607|59456|1144701|5946|44791|10|298|777|62|85.14|130.26|67.73|0.00|4199.26|5278.68|8076.12|0.00|0.00|4199.26|4199.26|-1079.42| +|60471|11889|59456||||10||777|34||||0.00|||3908.64||0.00|77.86||-2443.92| +2452273|60471|9953|59456|1144701|5946|44791|10|226|777|18|11.66|22.73|17.04|0.00|306.72|209.88|409.14|21.47|0.00|306.72|328.19|96.84| +2452273|60471|17723|59456|1144701|5946|44791|10|110|777|90|80.60|152.33|85.30|3454.65|7677.00|7254.00|13709.70|337.78|3454.65|4222.35|4560.13|-3031.65| +2452273|60471|11261|59456|1144701|5946|44791|10|68|777|56|66.39|113.52|85.14|0.00|4767.84|3717.84|6357.12|333.74|0.00|4767.84|5101.58|1050.00| +2452273|60471|11803|59456|1144701|5946|44791|10|168|777|22|67.14|127.56|35.71|0.00|785.62|1477.08|2806.32|39.28|0.00|785.62|824.90|-691.46| +2452273|60471|2921|59456|1144701|5946|44791|10|72|777|3|99.31|138.04|93.86|0.00|281.58|297.93|414.12|5.63|0.00|281.58|287.21|-16.35| +2452273|60471|4911|59456|1144701|5946|44791|10|265|777|67|37.03|49.24|39.39|0.00|2639.13|2481.01|3299.08|158.34|0.00|2639.13|2797.47|158.12| +2451446|70512|6571|27961|583176|2391|22759|2|107|778|91|75.12|146.48|10.25|410.41|932.75|6835.92|13329.68|0.00|410.41|522.34|522.34|-6313.58| +2451446|70512|4495|27961|583176|2391|22759|2|44|778|49|34.28|55.87|33.52|0.00|1642.48|1679.72|2737.63|49.27|0.00|1642.48|1691.75|-37.24| +2451446|70512|1720|27961|583176|2391|22759|2|168|778|51|1.53|1.77|0.53|4.59|27.03|78.03|90.27|0.44|4.59|22.44|22.88|-55.59| +2451446|70512|10466|27961|583176|2391|22759|2|15|778|9|33.90|64.74|60.85|0.00|547.65|305.10|582.66|43.81|0.00|547.65|591.46|242.55| +2451446|70512|7670|27961||||2|67|778|23|20.97|21.80||||482.31|501.40|5.12||||| +2451446|70512|11638|27961|583176|2391|22759|2|281|778|80|33.16|48.74|7.79|0.00|623.20|2652.80|3899.20|56.08|0.00|623.20|679.28|-2029.60| +2451446|70512|8752|27961|583176|2391|22759|2|197|778|57|55.16|68.95|42.74|0.00|2436.18|3144.12|3930.15|170.53|0.00|2436.18|2606.71|-707.94| +2451446|70512|12499|27961|583176|2391|22759|2|76|778|42|81.11|94.89|93.94|0.00|3945.48|3406.62|3985.38|39.45|0.00|3945.48|3984.93|538.86| +2451446|70512|14906|27961|583176|2391|22759|2|54|778|69|32.42|44.73|30.41|0.00|2098.29|2236.98|3086.37|146.88|0.00|2098.29|2245.17|-138.69| +2451446|70512|4712|27961|583176|2391|22759|2|203|778|87|53.09|87.06|79.22|0.00|6892.14|4618.83|7574.22|0.00|0.00|6892.14|6892.14|2273.31| +2451446|70512|10045|27961|583176|2391|22759|2|94|778|24|5.90|10.44|5.84|0.00|140.16|141.60|250.56|12.61|0.00|140.16|152.77|-1.44| +2451446|70512|13652|27961|583176|2391|22759|2|276|778|41|39.55|51.01|31.11|1173.46|1275.51|1621.55|2091.41|8.16|1173.46|102.05|110.21|-1519.50| +2451446|70512|799|27961|583176|2391|22759|2|65|778|41|77.34|98.22|3.92|40.18|160.72|3170.94|4027.02|4.82|40.18|120.54|125.36|-3050.40| +2451446|70512|16027|27961|583176|2391|22759|2|164|778|92|64.75|86.76|34.70|0.00|3192.40|5957.00|7981.92|95.77|0.00|3192.40|3288.17|-2764.60| +2451446|70512|4273|27961|583176|2391|22759|2|19|778|34|94.60|109.73|68.03|531.99|2313.02|3216.40|3730.82|71.24|531.99|1781.03|1852.27|-1435.37| +2452521|44554|17277|11659|1396628|4444|19060|7|155|779|38|97.64|181.61|103.51|0.00|3933.38|3710.32|6901.18|236.00|0.00|3933.38|4169.38|223.06| +2452521|44554|15570|11659|1396628|4444|19060|7|175|779|29|73.55|114.73|71.13|0.00|2062.77|2132.95|3327.17|41.25|0.00|2062.77|2104.02|-70.18| +2452521|44554|15507|11659|1396628|4444|19060|7|30|779|74|12.13|23.04|9.21|0.00|681.54|897.62|1704.96|40.89|0.00|681.54|722.43|-216.08| +2452521|44554|6324|11659|1396628|4444|19060|7|34|779|77|67.87|107.91|69.06|0.00|5317.62|5225.99|8309.07|478.58|0.00|5317.62|5796.20|91.63| +2452521|44554|7585|11659|1396628|4444|19060|7|148|779|13|96.93|132.79|95.60|0.00|1242.80|1260.09|1726.27|74.56|0.00|1242.80|1317.36|-17.29| +2452521|44554|5671|11659|1396628|4444|19060|7|274|779|9|54.59|86.79|36.45|0.00|328.05|491.31|781.11|29.52|0.00|328.05|357.57|-163.26| +2452521|44554|15642|11659|1396628|4444|19060|7|136|779|44|28.67|38.99|10.91|0.00|480.04|1261.48|1715.56|0.00|0.00|480.04|480.04|-781.44| +2452521|44554|9193|11659|1396628|4444|19060|7|169|779|41|88.70|124.18|110.52|0.00|4531.32|3636.70|5091.38|362.50|0.00|4531.32|4893.82|894.62| +2452521|44554|16443|11659|1396628|4444|19060|7|121|779|19|58.00|66.12|17.85|0.00|339.15|1102.00|1256.28|3.39|0.00|339.15|342.54|-762.85| +2452521|44554|12684|11659|1396628|4444|19060|7|171|779|1|66.86|94.27|55.61|0.00|55.61|66.86|94.27|0.55|0.00|55.61|56.16|-11.25| +2452521|44554|11373|11659|1396628|4444|19060|7|210|779|90|8.70|17.31|4.50|0.00|405.00|783.00|1557.90|16.20|0.00|405.00|421.20|-378.00| +2452521|44554|4429|11659|1396628|4444|19060|7|184|779|90|61.74|96.31|36.59|0.00|3293.10|5556.60|8667.90|98.79|0.00|3293.10|3391.89|-2263.50| +|44554|10171|11659|1396628|||7|130|779||4.04|6.10||||||9.88|||119.68|| +2451053|30488|15506|36195|874145|6048|27198|10|209|780|19|37.68|47.85|22.48|0.00|427.12|715.92|909.15|38.44|0.00|427.12|465.56|-288.80| +2451053|30488|6322|36195|874145|6048|27198|10|164|780|70|74.57|124.53|41.09|0.00|2876.30|5219.90|8717.10|0.00|0.00|2876.30|2876.30|-2343.60| +2451053|30488|7585|36195|874145|6048|27198|10|177|780|41|89.88|178.86|55.44|0.00|2273.04|3685.08|7333.26|90.92|0.00|2273.04|2363.96|-1412.04| +2451053|30488|5671|36195|874145|6048|27198|10|131|780|37|70.17|117.18|2.34|0.00|86.58|2596.29|4335.66|1.73|0.00|86.58|88.31|-2509.71| +2451053|30488|15640|36195|874145|6048|27198|10|61|780|40|5.09|6.26|2.25|0.00|90.00|203.60|250.40|3.60|0.00|90.00|93.60|-113.60| +2451053|30488|9193|36195|874145|6048|27198|10|195|780|1|75.35|140.90|18.31|0.00|18.31|75.35|140.90|0.00|0.00|18.31|18.31|-57.04| +2451053|30488|16442|36195|874145|6048|27198|10|289|780|2|41.16|56.38|3.94|0.00|7.88|82.32|112.76|0.07|0.00|7.88|7.95|-74.44| +2451053|30488|12682|36195|874145|6048|27198|10|285|780|10|23.90|40.86|11.44|0.00|114.40|239.00|408.60|1.14|0.00|114.40|115.54|-124.60| +2451053|30488|11372|36195|874145|6048|27198|10|58|780|27|35.41|44.26|26.99|0.00|728.73|956.07|1195.02|58.29|0.00|728.73|787.02|-227.34| +2451053|30488|4429|36195|874145|6048|27198|10|237|780|71|11.81|16.29|11.89|0.00|844.19|838.51|1156.59|42.20|0.00|844.19|886.39|5.68| +2451053|30488|10171|36195|874145|6048|27198|10|2|780|6|89.60|167.55|55.29|0.00|331.74|537.60|1005.30|29.85|0.00|331.74|361.59|-205.86| +||14080||874145|6048||10|80|780|||33.15|10.60|0.00|593.60|||11.87|0.00|593.60||| +2451862|48824|6383|33811|925257|3550|21796|4|171|781|58|56.88|72.23|2.88|0.00|167.04|3299.04|4189.34|1.67|0.00|167.04|168.71|-3132.00| +|48824|16307||925257|||||781|1||||||45.64|56.59|||||-29.80| +2451862|48824|11531|33811|925257|3550|21796|4|213|781|11|95.18|102.79|30.83|0.00|339.13|1046.98|1130.69|20.34|0.00|339.13|359.47|-707.85| +2451862|48824|1661|33811|925257|3550|21796|4|286|781|19|62.73|85.31|45.21|0.00|858.99|1191.87|1620.89|77.30|0.00|858.99|936.29|-332.88| +2451862|48824|17203|33811|925257|3550|21796|4|272|781|43|6.43|11.57|1.38|0.00|59.34|276.49|497.51|1.78|0.00|59.34|61.12|-217.15| +2451862|48824|7526|33811|925257|3550|21796|4|35|781|6|30.01|45.61|8.20|0.00|49.20|180.06|273.66|0.98|0.00|49.20|50.18|-130.86| +2451862|48824|2351|33811|925257|3550|21796|4|18|781|39|34.89|45.35|15.87|0.00|618.93|1360.71|1768.65|12.37|0.00|618.93|631.30|-741.78| +2451862|48824|2623|33811|925257|3550|21796|4|2|781|93|36.92|45.41|39.50|0.00|3673.50|3433.56|4223.13|330.61|0.00|3673.50|4004.11|239.94| +2451862|48824|9059|33811|925257|3550|21796|4|107|781|79|7.67|8.36|2.17|0.00|171.43|605.93|660.44|3.42|0.00|171.43|174.85|-434.50| +2451862|48824|12341|33811|925257|3550|21796|4|264|781|40|14.15|28.30|21.79|453.23|871.60|566.00|1132.00|16.73|453.23|418.37|435.10|-147.63| +2451862|48824|14149|33811|925257|3550|21796|4|284|781|75|49.95|60.93|8.53|0.00|639.75|3746.25|4569.75|31.98|0.00|639.75|671.73|-3106.50| +|48824|15803|||||4|239|781|||136.58||||8212.84|12565.36|18.82||||-7585.40| +2451862|48824|16490|33811|925257|3550|21796|4|85|781|66|90.49|152.92|32.11|275.50|2119.26|5972.34|10092.72|147.50|275.50|1843.76|1991.26|-4128.58| +2451862|48824|13331|33811|925257|3550|21796|4|207|781|51|78.92|102.59|35.90|0.00|1830.90|4024.92|5232.09|128.16|0.00|1830.90|1959.06|-2194.02| +2451862|48824|16909|33811|925257|3550|21796|4|80|781|30|15.07|20.04|2.00|5.40|60.00|452.10|601.20|4.91|5.40|54.60|59.51|-397.50| +2451489||17620|27721|216382|||4||782|38|88.74|109.15||0.00|||||0.00|2115.08|2157.38|| +2451489|51682|11968|27721|216382|6643|2104|4|68|782|36|49.46|88.53|39.83|0.00|1433.88|1780.56|3187.08|100.37|0.00|1433.88|1534.25|-346.68| +2451489|51682|16276|27721|216382|6643|2104|4|222|782|67|97.05|116.46|82.68|0.00|5539.56|6502.35|7802.82|276.97|0.00|5539.56|5816.53|-962.79| +||9355||216382|6643|||120|782||92.16||75.47|||1843.20||45.28||1509.40|1554.68|| +2451489|51682|4789|27721|216382|6643|2104|4|265|782|70|95.70|139.72|83.83|0.00|5868.10|6699.00|9780.40|528.12|0.00|5868.10|6396.22|-830.90| +2451489|51682|7447|27721|216382|6643|2104|4|30|782|75|28.18|49.59|0.49|0.00|36.75|2113.50|3719.25|1.47|0.00|36.75|38.22|-2076.75| +2451489|51682|10634|27721|216382|6643|2104|4|242|782|96|40.82|69.39|17.34|0.00|1664.64|3918.72|6661.44|149.81|0.00|1664.64|1814.45|-2254.08| +2451489|51682|5950|27721|216382|6643|2104|4|286|782|90|51.71|70.84|53.83|2083.22|4844.70|4653.90|6375.60|220.91|2083.22|2761.48|2982.39|-1892.42| +2451489|51682|15982|27721|216382|6643|2104|4|191|782|54|89.35|164.40|19.72|0.00|1064.88|4824.90|8877.60|85.19|0.00|1064.88|1150.07|-3760.02| +2451489|51682|6802|27721|216382|6643|2104|4|173|782|34|91.31|159.79|95.87|0.00|3259.58|3104.54|5432.86|195.57|0.00|3259.58|3455.15|155.04| +2451489|51682|6926|27721|216382|6643|2104|4|161|782|4|32.12|53.31|25.05|0.00|100.20|128.48|213.24|6.01|0.00|100.20|106.21|-28.28| +2451489|51682|3658|27721|216382|6643|2104|4|183|782|54|15.20|17.78|14.40|15.55|777.60|820.80|960.12|68.58|15.55|762.05|830.63|-58.75| +2452624|36117|873|99689|1248631|6726|21|4|246|783|17|71.11|113.77|55.74|0.00|947.58|1208.87|1934.09|0.00|0.00|947.58|947.58|-261.29| +2452624|36117|571|99689|1248631|6726|21|4|135|783|46|90.72|170.55|64.80|0.00|2980.80|4173.12|7845.30|178.84|0.00|2980.80|3159.64|-1192.32| +2452624|36117|16470|99689|1248631|6726|21|4|293|783|39|62.28|99.64|91.66|214.48|3574.74|2428.92|3885.96|100.80|214.48|3360.26|3461.06|931.34| +2452624|36117|15135|99689|1248631|6726|21|4|254|783|2|15.65|31.14|30.20|0.00|60.40|31.30|62.28|2.41|0.00|60.40|62.81|29.10| +2452624|36117|17257|99689|1248631|6726|21|4|178|783|11|24.97|35.70|34.62|0.00|380.82|274.67|392.70|11.42|0.00|380.82|392.24|106.15| +2452624|36117|16623|99689|1248631|6726|21|4|133|783|7|35.30|62.12|15.53|0.00|108.71|247.10|434.84|0.00|0.00|108.71|108.71|-138.39| +2452624|36117|16260|99689|1248631|6726|21|4|225|783|76|31.49|58.25|51.26|2688.07|3895.76|2393.24|4427.00|108.69|2688.07|1207.69|1316.38|-1185.55| +2452624|36117|2977|99689|1248631|6726|21|4|146|783|82|12.30|23.37|19.39|0.00|1589.98|1008.60|1916.34|127.19|0.00|1589.98|1717.17|581.38| +2452624|36117|14433|99689|1248631|6726|21|4|38|783|96|86.48|95.99|54.71|0.00|5252.16|8302.08|9215.04|210.08|0.00|5252.16|5462.24|-3049.92| +2452624|36117|13716|99689|1248631|6726|21|4|67|783|15|21.90|28.03|18.78|0.00|281.70|328.50|420.45|19.71|0.00|281.70|301.41|-46.80| +2452624|36117|7515|99689|1248631|6726|21|4|59|783|52|89.93|123.20|45.58|0.00|2370.16|4676.36|6406.40|118.50|0.00|2370.16|2488.66|-2306.20| +2452624|36117|11334|99689|1248631|6726|21|4|88|783|90|45.79|87.91|9.67|0.00|870.30|4121.10|7911.90|26.10|0.00|870.30|896.40|-3250.80| +|53206|1243|51256|||||44|784|37|58.52||62.70|||||69.59||2319.90|2389.49|| +2452197|53206|17637|51256|919293|4130|42520|1|266|784|52|53.99|65.32|0.00|0.00|0.00|2807.48|3396.64|0.00|0.00|0.00|0.00|-2807.48| +2452197|53206|16275|51256|919293|4130|42520|1|114|784|67|54.94|55.48|22.19|0.00|1486.73|3680.98|3717.16|133.80|0.00|1486.73|1620.53|-2194.25| +2452197|53206|11477|51256|919293|4130|42520|1|109|784|12|48.12|56.78|32.93|0.00|395.16|577.44|681.36|11.85|0.00|395.16|407.01|-182.28| +2452197|53206|9665|51256|919293|4130|42520|1|80|784|18|90.74|101.62|95.52|0.00|1719.36|1633.32|1829.16|0.00|0.00|1719.36|1719.36|86.04| +2452197|53206|10743|51256|919293|4130|42520|1|215|784|37|75.40|122.90|17.20|0.00|636.40|2789.80|4547.30|25.45|0.00|636.40|661.85|-2153.40| +2452197|53206|6959|51256|919293|4130|42520|1|94|784|84|36.08|55.92|50.88|0.00|4273.92|3030.72|4697.28|213.69|0.00|4273.92|4487.61|1243.20| +2452197|53206|14957|51256|919293|4130|42520|1|6|784|79|25.39|39.10|11.33|0.00|895.07|2005.81|3088.90|53.70|0.00|895.07|948.77|-1110.74| +2452197|53206|4671|51256|919293|4130|42520|1|62|784|43|38.38|56.03|47.62|1228.59|2047.66|1650.34|2409.29|16.38|1228.59|819.07|835.45|-831.27| +2452197||12695|51256||||||784||90.91|112.72|15.78|0.00|1104.60||||0.00|1104.60|1126.69|-5259.10| +2452197|53206|16171|51256|919293|4130|42520|1|91|784|98|99.67|178.40|12.48|635.98|1223.04|9767.66|17483.20|46.96|635.98|587.06|634.02|-9180.60| +2451881|72619|17069|1189|366908|6227|42393|7|143|785|83|8.55|14.53|6.68|216.23|554.44|709.65|1205.99|13.52|216.23|338.21|351.73|-371.44| +2451881|72619|16097|1189|366908|6227|42393|7|61|785|86|54.95|73.08|55.54|47.76|4776.44|4725.70|6284.88|425.58|47.76|4728.68|5154.26|2.98| +2451881|72619|9932|1189|366908|6227|42393|7|96|785|75|81.50|135.29|96.05|504.26|7203.75|6112.50|10146.75|602.95|504.26|6699.49|7302.44|586.99| +2451881|72619|11132|1189|366908|6227|42393|7|231|785|23|72.61|111.09|28.88|0.00|664.24|1670.03|2555.07|26.56|0.00|664.24|690.80|-1005.79| +2451881|72619|440|1189|366908|6227|42393|7|21|785|77|97.53|108.25|14.07|0.00|1083.39|7509.81|8335.25|75.83|0.00|1083.39|1159.22|-6426.42| +2451881|72619|10873|1189|366908|6227|42393|7|149|785|64|28.04|40.09|10.82|0.00|692.48|1794.56|2565.76|20.77|0.00|692.48|713.25|-1102.08| +2451881|72619|833|1189|366908|6227|42393|7|61|785|30|4.58|9.06|0.36|0.00|10.80|137.40|271.80|0.43|0.00|10.80|11.23|-126.60| +2451881|72619|1241|1189|366908|6227|42393|7|140|785|54|94.44|98.21|90.35|0.00|4878.90|5099.76|5303.34|292.73|0.00|4878.90|5171.63|-220.86| +2451881|72619|1543|1189|366908|6227|42393|7|268|785|46|85.63|122.45|71.02|0.00|3266.92|3938.98|5632.70|163.34|0.00|3266.92|3430.26|-672.06| +2451094|29486|1036|90840|262905|3417|23529|1|189|786|95|14.89|26.80|2.14|0.00|203.30|1414.55|2546.00|0.00|0.00|203.30|203.30|-1211.25| +2451094|29486|5185|90840|262905|3417|23529|1|241|786|5|90.85|107.20|70.75|0.00|353.75|454.25|536.00|10.61|0.00|353.75|364.36|-100.50| +2451094|29486|634|90840|262905|3417|23529|1|154|786|93|87.80|99.21|21.82|385.55|2029.26|8165.40|9226.53|147.93|385.55|1643.71|1791.64|-6521.69| +2451094|29486|556|90840|262905|3417|23529|1|21|786|81|90.17|146.07|138.76|0.00|11239.56|7303.77|11831.67|224.79|0.00|11239.56|11464.35|3935.79| +2451094|29486|12940|90840|262905|3417|23529|1|273|786|45|93.54|96.34|9.63|0.00|433.35|4209.30|4335.30|39.00|0.00|433.35|472.35|-3775.95| +2451094|29486|12020|90840|262905|3417|23529|1|103|786|79|27.17|32.87|2.30|0.00|181.70|2146.43|2596.73|9.08|0.00|181.70|190.78|-1964.73| +2451094|29486|9055|90840|262905|3417|23529|1|101|786|67|17.88|22.70|6.81|0.00|456.27|1197.96|1520.90|27.37|0.00|456.27|483.64|-741.69| +2451094|29486|7270|90840|262905|3417|23529|1|274|786|73|2.96|3.22|2.15|0.00|156.95|216.08|235.06|7.84|0.00|156.95|164.79|-59.13| +2451094|29486|11377|90840|262905|3417|23529|1|101|786|50|22.52|41.21|33.79|1385.39|1689.50|1126.00|2060.50|3.04|1385.39|304.11|307.15|-821.89| +2451094|29486|13381|90840|262905|3417|23529|1|94|786|77|26.55|28.67|4.58|0.00|352.66|2044.35|2207.59|21.15|0.00|352.66|373.81|-1691.69| +2451094|29486|12493|90840|262905|3417|23529|1|156|786|40|83.70|159.86|75.13|2824.88|3005.20|3348.00|6394.40|10.81|2824.88|180.32|191.13|-3167.68| +2451513|39745|158|95227|1060995|810|15378|1|36|787|6|26.89|43.56|38.33|68.99|229.98|161.34|261.36|4.82|68.99|160.99|165.81|-0.35| +2451513|39745|10033|95227|1060995|810|15378|1|131|787|88|3.55|4.57|0.36|0.00|31.68|312.40|402.16|2.85|0.00|31.68|34.53|-280.72| +2451513|39745|5761|95227|1060995|810|15378|1|227|787|98|11.67|13.53|5.27|0.00|516.46|1143.66|1325.94|25.82|0.00|516.46|542.28|-627.20| +||8290|95227||810||1|246|787||91.64|153.95|72.35|0.00|3906.90|4948.56|||0.00||4180.38|| +2451513|39745|14389|95227|1060995|810|15378|1|93|787|14|37.40|40.76|16.30|0.00|228.20|523.60|570.64|20.53|0.00|228.20|248.73|-295.40| +2451513|39745|3673|95227|1060995|810|15378|1|62|787|62|38.72|56.14|51.08|0.00|3166.96|2400.64|3480.68|126.67|0.00|3166.96|3293.63|766.32| +2451513|39745|8479|95227|1060995|810|15378|1|142|787|20|37.79|41.94|24.32|0.00|486.40|755.80|838.80|43.77|0.00|486.40|530.17|-269.40| +2451513|39745|16720|95227|1060995|810|15378|1|242|787|41|3.55|5.71|4.51|116.49|184.91|145.55|234.11|6.15|116.49|68.42|74.57|-77.13| +||860||1060995|810||||787|88|15.68|22.26||0.00|1174.80||1958.88||0.00|||| +2452027|49493|1181|32085|1204029|5220|10364|8|293|788|43|11.79|19.21|10.56|40.86|454.08|506.97|826.03|33.05|40.86|413.22|446.27|-93.75| +2452027|49493|14869|32085|1204029|5220|10364|8|141|788|8|39.09|56.68|41.37|0.00|330.96|312.72|453.44|3.30|0.00|330.96|334.26|18.24| +2452027|49493|12973|32085|1204029|5220|10364|8|154|788|64|50.62|77.44|27.10|0.00|1734.40|3239.68|4956.16|86.72|0.00|1734.40|1821.12|-1505.28| +2452027|49493|6363|32085|1204029|5220|10364|8|73|788|95|49.05|71.61|62.30|0.00|5918.50|4659.75|6802.95|59.18|0.00|5918.50|5977.68|1258.75| +2452027|49493|3019|32085|1204029|5220|10364|8|109|788|61|18.90|20.79|7.90|0.00|481.90|1152.90|1268.19|33.73|0.00|481.90|515.63|-671.00| +2452027|49493|16445|32085|1204029|5220|10364|8|275|788|41|15.65|25.04|23.53|0.00|964.73|641.65|1026.64|0.00|0.00|964.73|964.73|323.08| +2452027|49493|15653|32085|1204029|5220|10364|8|164|788|28|72.29|128.67|27.02|0.00|756.56|2024.12|3602.76|15.13|0.00|756.56|771.69|-1267.56| +2452027|49493|3689|32085|1204029|5220|10364|8|76|788|98|30.20|51.34|31.31|2178.54|3068.38|2959.60|5031.32|62.28|2178.54|889.84|952.12|-2069.76| +2452027|49493|8535|32085|1204029|5220|10364|8|30|788|54|49.80|91.63|76.05|0.00|4106.70|2689.20|4948.02|164.26|0.00|4106.70|4270.96|1417.50| +2452027|49493|7369|32085|1204029|5220|10364|8|65|788|65|33.49|42.19|13.07|0.00|849.55|2176.85|2742.35|50.97|0.00|849.55|900.52|-1327.30| +2452317|52959|11040|27999|368298|2644|25743|7|64|789|29|88.01|145.21|43.56|0.00|1263.24|2552.29|4211.09|50.52|0.00|1263.24|1313.76|-1289.05| +2452317|52959|16797|27999||2644|25743|||789||37.13||||1250.90|2599.10|4314.10|||1250.90|1275.91|-1348.20| +2452317|52959|1551|27999|368298|2644|25743|7|96|789|55|29.95|47.32|17.98|0.00|988.90|1647.25|2602.60|69.22|0.00|988.90|1058.12|-658.35| +2452317|52959|12774|27999|368298|2644|25743|7|271|789|18|55.16|89.91|88.11|0.00|1585.98|992.88|1618.38|63.43|0.00|1585.98|1649.41|593.10| +2452317|52959|5467|27999|368298|2644|25743|7|14|789|4|57.77|112.07|15.68|15.05|62.72|231.08|448.28|1.43|15.05|47.67|49.10|-183.41| +2452317|52959|8817|27999|368298|2644|25743|7|265|789|74|68.88|88.85|82.63|0.00|6114.62|5097.12|6574.90|244.58|0.00|6114.62|6359.20|1017.50| +2452317|52959|7029|27999|368298|2644|25743|7|109|789|95|79.93|85.52|62.42|0.00|5929.90|7593.35|8124.40|355.79|0.00|5929.90|6285.69|-1663.45| +2452317|52959|2155|27999|368298|2644|25743|7|173|789|10|8.73|14.57|5.97|0.00|59.70|87.30|145.70|4.17|0.00|59.70|63.87|-27.60| +2452317|52959|15390|27999|368298|2644|25743|7|219|789|66|69.13|71.89|27.31|0.00|1802.46|4562.58|4744.74|162.22|0.00|1802.46|1964.68|-2760.12| +2452317||6265|27999|368298||25743|7||789|62||||||||||0.00|0.00|| +2452317|52959|13074|27999|368298|2644|25743|7|22|789|2|6.91|12.02|5.04|0.00|10.08|13.82|24.04|0.30|0.00|10.08|10.38|-3.74| +2452317|52959|4467|27999|368298|2644|25743|7|187|789|82|94.59|104.04|78.03|0.00|6398.46|7756.38|8531.28|63.98|0.00|6398.46|6462.44|-1357.92| +2452317|52959|7680|27999|368298|2644|25743|7|4|789|69|90.07|93.67|83.36|0.00|5751.84|6214.83|6463.23|115.03|0.00|5751.84|5866.87|-462.99| +2452317|52959|14280|27999|368298|2644|25743|7|65|789|92|50.82|63.01|18.27|1512.75|1680.84|4675.44|5796.92|8.40|1512.75|168.09|176.49|-4507.35| +2452317|52959|3471|27999|368298|2644|25743|7|265|789|96|50.83|92.00|68.08|0.00|6535.68|4879.68|8832.00|65.35|0.00|6535.68|6601.03|1656.00| +2451460|63204|2389|15783|1480079|2422|16718|10|89|790|39|83.18|158.04|9.48|0.00|369.72|3244.02|6163.56|25.88|0.00|369.72|395.60|-2874.30| +2451460|63204|1477|15783|1480079|2422|16718|10|147|790|2|10.38|18.99|1.70|0.00|3.40|20.76|37.98|0.06|0.00|3.40|3.46|-17.36| +2451460|63204|11432|15783|1480079|2422|16718|10|258|790|53|4.47|8.49|3.73|15.81|197.69|236.91|449.97|9.09|15.81|181.88|190.97|-55.03| +2451460|63204|10567|15783||||||790|||||||||192.72||2141.34||725.35| +||6740|||2422|||118|790|||||0.00|||||0.00|||-748.84| +2451460|63204|13480|15783|1480079|2422|16718|10|191|790|6|53.95|99.26|29.77|0.00|178.62|323.70|595.56|1.78|0.00|178.62|180.40|-145.08| +2451460|63204|2845|15783|1480079|2422|16718|10|295|790|46|51.62|68.65|63.84|0.00|2936.64|2374.52|3157.90|117.46|0.00|2936.64|3054.10|562.12| +2451460|63204|3400|15783|1480079|2422|16718|10|11|790|3|22.30|23.63|3.54|0.00|10.62|66.90|70.89|0.42|0.00|10.62|11.04|-56.28| +2451460|63204|7024|15783|1480079|2422|16718|10|189|790|2|67.90|130.36|93.85|7.50|187.70|135.80|260.72|9.01|7.50|180.20|189.21|44.40| +2451460|63204|9349|15783|1480079|2422|16718|10|122|790|6|81.60|140.35|58.94|229.86|353.64|489.60|842.10|4.95|229.86|123.78|128.73|-365.82| +2451460|63204|13657|15783|1480079|2422|16718|10|142|790|6|66.32|102.79|43.17|0.00|259.02|397.92|616.74|20.72|0.00|259.02|279.74|-138.90| +2451460|63204|5252|15783|1480079|2422|16718|10|37|790|89|84.26|164.30|62.43|0.00|5556.27|7499.14|14622.70|111.12|0.00|5556.27|5667.39|-1942.87| +2451460|63204|5575|15783|1480079|2422|16718|10|6|790|20|73.97|109.47|88.67|0.00|1773.40|1479.40|2189.40|159.60|0.00|1773.40|1933.00|294.00| +2451855|62288|1273|86796|572014|6032|29650|2|137|791|14|98.42|120.07|93.65|0.00|1311.10|1377.88|1680.98|13.11|0.00|1311.10|1324.21|-66.78| +2451855|62288|17063|86796|572014|6032|29650|2|17|791|75|48.79|81.47|36.66|0.00|2749.50|3659.25|6110.25|109.98|0.00|2749.50|2859.48|-909.75| +2451855|62288|3199|86796|572014|6032|29650|2|42|791|73|95.30|99.11|97.12|0.00|7089.76|6956.90|7235.03|354.48|0.00|7089.76|7444.24|132.86| +2451855||7153|86796|572014||29650|2||791||76.24|77.00|50.82||4014.78||6083.00||||4215.51|| +2451855|62288|14816|86796|572014|6032|29650|2|285|791|82|36.21|69.16|4.84|190.50|396.88|2969.22|5671.12|14.44|190.50|206.38|220.82|-2762.84| +2451855||17077|86796|||||199|791|13||3.03||0.00|3.12|27.95|||0.00|||-24.83| +2451855|62288|12302|86796|572014|6032|29650|2|262|791|17|66.59|105.87|52.93|0.00|899.81|1132.03|1799.79|44.99|0.00|899.81|944.80|-232.22| +2451855|62288|14414|86796|572014|6032|29650|2|239|791|87|9.57|10.81|8.64|435.97|751.68|832.59|940.47|18.94|435.97|315.71|334.65|-516.88| +2451855|62288|13603|86796|572014|6032|29650|2|266|791|8|74.08|145.19|31.94|0.00|255.52|592.64|1161.52|2.55|0.00|255.52|258.07|-337.12| +2451855|62288|8963|86796|572014|6032|29650|2|174|791|26|12.28|21.98|5.05|0.00|131.30|319.28|571.48|0.00|0.00|131.30|131.30|-187.98| +2451855||9014||||||215|791|||||0.00|555.36||5054.31|27.76|0.00|||-2300.65| +2451855|62288|3332|86796|572014|6032|29650|2|183|791|24|23.34|37.11|15.58|0.00|373.92|560.16|890.64|22.43|0.00|373.92|396.35|-186.24| +2451855|62288|2699|86796|572014|6032|29650|2|220|791|30|39.75|42.53|31.47|0.00|944.10|1192.50|1275.90|0.00|0.00|944.10|944.10|-248.40| +2451855|62288|2785|86796|572014|6032|29650|2|122|791|16|84.54|119.20|82.24|0.00|1315.84|1352.64|1907.20|118.42|0.00|1315.84|1434.26|-36.80| +2451855|62288|7064|86796|572014|6032|29650|2|63|791|34|45.82|46.27|21.28|0.00|723.52|1557.88|1573.18|28.94|0.00|723.52|752.46|-834.36| +2452534|36073|17517|61684|1454276|2068|260|1|81|792|33|32.33|50.43|41.35|136.45|1364.55|1066.89|1664.19|12.28|136.45|1228.10|1240.38|161.21| +2452534|36073|11943|61684|1454276|2068|260|1|277|792|82|93.29|102.61|51.30|0.00|4206.60|7649.78|8414.02|252.39|0.00|4206.60|4458.99|-3443.18| +2452534|36073|1797|61684|1454276|2068|260|1|289|792|76|1.07|1.97|1.73|0.00|131.48|81.32|149.72|6.57|0.00|131.48|138.05|50.16| +2452534|36073|15471|61684|1454276|2068|260|1|226|792|48|12.39|21.18|2.96|0.00|142.08|594.72|1016.64|12.78|0.00|142.08|154.86|-452.64| +2452534|36073|4983|61684|1454276|2068|260|1|53|792|19|94.76|130.76|40.53|0.00|770.07|1800.44|2484.44|30.80|0.00|770.07|800.87|-1030.37| +2452534|36073|8676|61684|1454276|2068|260|1|38|792|91|79.83|112.56|87.79|4633.55|7988.89|7264.53|10242.96|167.76|4633.55|3355.34|3523.10|-3909.19| +2452534|36073|17139|61684|1454276|2068|260|1|234|792|99|60.60|71.50|32.17|286.63|3184.83|5999.40|7078.50|57.96|286.63|2898.20|2956.16|-3101.20| +2452534|36073|15240|61684|1454276|2068|260|1|124|792|8|40.32|77.01|20.79|0.00|166.32|322.56|616.08|13.30|0.00|166.32|179.62|-156.24| +2452534|36073|8631|61684|1454276|2068|260|1|270|792|9|64.05|69.81|24.43|140.71|219.87|576.45|628.29|4.74|140.71|79.16|83.90|-497.29| +2452534|36073|12198|61684|1454276|2068|260|1|216|792|48|30.69|40.51|10.93|0.00|524.64|1473.12|1944.48|0.00|0.00|524.64|524.64|-948.48| +2452556|63804|1627|65347|341099|6004|40956|1|157|793|27|42.04|42.04|36.99|639.18|998.73|1135.08|1135.08|0.00|639.18|359.55|359.55|-775.53| +2452556|63804|16902|65347|341099|6004|40956|1|182|793|27|89.85|177.90|126.30|0.00|3410.10|2425.95|4803.30|102.30|0.00|3410.10|3512.40|984.15| +2452556|63804|10608|65347|341099|6004|40956|1|281|793|27|12.42|12.66|8.60|62.69|232.20|335.34|341.82|13.56|62.69|169.51|183.07|-165.83| +2452556|63804|399|65347|341099|6004|40956|1|105|793|11|51.48|67.43|2.02|0.00|22.22|566.28|741.73|0.66|0.00|22.22|22.88|-544.06| +2452556|63804|7543|65347|341099|6004|40956|1|265|793|85|54.08|102.21|77.67|0.00|6601.95|4596.80|8687.85|0.00|0.00|6601.95|6601.95|2005.15| +2452556|63804|11310|65347|341099|6004|40956|1|262|793|93|81.94|108.98|11.98|0.00|1114.14|7620.42|10135.14|100.27|0.00|1114.14|1214.41|-6506.28| +2452556|63804|16537|65347|341099|6004|40956|1|133|793|25|81.00|87.48|75.23|150.46|1880.75|2025.00|2187.00|86.51|150.46|1730.29|1816.80|-294.71| +2452556|63804|17469|65347|341099|6004|40956|1|244|793|34|74.26|100.25|90.22|0.00|3067.48|2524.84|3408.50|184.04|0.00|3067.48|3251.52|542.64| +2452556|63804|13315|65347|341099|6004|40956|1|240|793|42|31.32|38.83|17.47|0.00|733.74|1315.44|1630.86|58.69|0.00|733.74|792.43|-581.70| +2452556|63804|11929|65347|341099|6004|40956|1|191|793|25|32.47|43.18|2.15|22.03|53.75|811.75|1079.50|0.63|22.03|31.72|32.35|-780.03| +2452556|63804|13599|65347|341099|6004|40956|1|152|793|20|79.05|113.83|22.76|0.00|455.20|1581.00|2276.60|22.76|0.00|455.20|477.96|-1125.80| +|63804|13116|65347|||40956|1|162|793|1|5.28|10.19|9.37|||||||6.47|6.47|| +2452556|63804|11226|65347|341099|6004|40956|1|21|793|21|38.89|61.44|35.02|125.02|735.42|816.69|1290.24|0.00|125.02|610.40|610.40|-206.29| +2452556|63804|9129|65347|341099|6004|40956|1|137|793|17|47.03|79.95|79.15|0.00|1345.55|799.51|1359.15|0.00|0.00|1345.55|1345.55|546.04| +2451880|53538|15631|91687|1761112|1737|21162|8|267|794|52|86.29|90.60|10.87|531.32|565.24|4487.08|4711.20|1.69|531.32|33.92|35.61|-4453.16| +2451880|53538|3977|91687|1761112|1737|21162|8|117|794|38|95.19|107.56|77.44|0.00|2942.72|3617.22|4087.28|205.99|0.00|2942.72|3148.71|-674.50| +|53538|17731||1761112|1737|21162||147|794|86|68.42||19.02|0.00||5884.12||49.07|0.00|1635.72|1684.79|| +2451880|53538|8237|91687|1761112|1737|21162|8|177|794|31|66.16|80.05|16.81|416.88|521.11|2050.96|2481.55|7.29|416.88|104.23|111.52|-1946.73| +2451880|53538|4411|91687|1761112|1737|21162|8|88|794|13|17.51|32.04|4.16|44.88|54.08|227.63|416.52|0.09|44.88|9.20|9.29|-218.43| +2451880|53538|9542|91687|1761112|1737|21162|8|203|794|87|93.02|118.13|99.22|6042.49|8632.14|8092.74|10277.31|181.27|6042.49|2589.65|2770.92|-5503.09| +2451880|53538|6116|91687|1761112|1737|21162|8|29|794|6|58.95|63.07|28.38|0.00|170.28|353.70|378.42|13.62|0.00|170.28|183.90|-183.42| +2451880|53538|11479|91687|1761112|1737|21162|8|63|794|92|5.04|8.21|2.21|0.00|203.32|463.68|755.32|16.26|0.00|203.32|219.58|-260.36| +2451880|53538|14311|91687|1761112|1737|21162|8|289|794|85|82.52|142.75|29.97|0.00|2547.45|7014.20|12133.75|152.84|0.00|2547.45|2700.29|-4466.75| +2451880|53538|8936|91687|1761112|1737|21162|8|102|794|67|90.27|148.04|48.85|0.00|3272.95|6048.09|9918.68|98.18|0.00|3272.95|3371.13|-2775.14| +2451880|53538|12869|91687|1761112|1737|21162|8|182|794|22|55.92|88.91|56.01|0.00|1232.22|1230.24|1956.02|0.00|0.00|1232.22|1232.22|1.98| +2451880|53538|17852|91687|1761112|1737|21162|8|28|794|65|78.09|106.20|47.79|0.00|3106.35|5075.85|6903.00|62.12|0.00|3106.35|3168.47|-1969.50| +2451880|53538|1975|91687|1761112|1737|21162|8|220|794|96|74.74|136.02|53.04|0.00|5091.84|7175.04|13057.92|101.83|0.00|5091.84|5193.67|-2083.20| +||4001|91687|1761112||||106|794|13||25.36|10.90|||255.58|329.68|5.66||141.70|147.36|| +2452474|30332|2155|19095|1563083|1118|14753|1|290|795|65|19.07|26.69|1.60|90.48|104.00|1239.55|1734.85|0.54|90.48|13.52|14.06|-1226.03| +2452474|30332|15390|19095|1563083|1118|14753|1|299|795|4|21.90|22.11|5.96|0.00|23.84|87.60|88.44|0.71|0.00|23.84|24.55|-63.76| +||6265|19095||||1|165|795|2||19.58|15.27|0.00|30.54|21.52|39.16|0.61|0.00|30.54|31.15|| +2452474|30332|13074|19095|1563083|1118|14753|1|211|795|79|63.67|76.40|2.29|0.00|180.91|5029.93|6035.60|5.42|0.00|180.91|186.33|-4849.02| +2452474|30332|4467|19095|1563083|1118|14753|1|111|795|37|54.46|89.31|10.71|0.00|396.27|2015.02|3304.47|19.81|0.00|396.27|416.08|-1618.75| +2452474|30332|7680|19095|1563083|1118|14753|1|147|795|100|65.93|129.22|54.27|4450.14|5427.00|6593.00|12922.00|29.30|4450.14|976.86|1006.16|-5616.14| +||14280|19095||1118||1||795|55|1.08|1.19|0.47|0.00||59.40|65.45||0.00|25.85||-33.55| +2452474|30332|3471|19095|1563083|1118|14753|1|287|795|46|15.98|21.73|15.42|0.00|709.32|735.08|999.58|14.18|0.00|709.32|723.50|-25.76| +2452474|30332|3393|19095|1563083|1118|14753|1|114|795|63|28.32|33.98|16.99|0.00|1070.37|1784.16|2140.74|10.70|0.00|1070.37|1081.07|-713.79| +2452474|30332|9057|19095|1563083|1118|14753|1|200|795|78|77.37|123.01|78.72|0.00|6140.16|6034.86|9594.78|184.20|0.00|6140.16|6324.36|105.30| +2452474|30332|10995|19095|1563083|1118|14753|1|89|795|87|48.84|53.23|48.43|0.00|4213.41|4249.08|4631.01|168.53|0.00|4213.41|4381.94|-35.67| +2452621|39032|9027|35966|1051184|6402|10020|7|12|796|25|56.62|97.38|46.74|151.90|1168.50|1415.50|2434.50|81.32|151.90|1016.60|1097.92|-398.90| +2452621|39032|2202|35966|1051184|6402|10020|7|82|796|1|69.36|81.84|67.10|0.00|67.10|69.36|81.84|4.02|0.00|67.10|71.12|-2.26| +2452621|39032|16953|35966|1051184|6402|10020|7|13|796|74|49.86|71.29|10.69|0.00|791.06|3689.64|5275.46|71.19|0.00|791.06|862.25|-2898.58| +2452621|39032|10585|35966|1051184|6402|10020|7|26|796|76|24.22|43.59|30.94|0.00|2351.44|1840.72|3312.84|188.11|0.00|2351.44|2539.55|510.72| +2452621|39032|10921|35966|1051184|6402|10020|7|128|796|45|45.53|66.92|26.76|0.00|1204.20|2048.85|3011.40|36.12|0.00|1204.20|1240.32|-844.65| +||11613||1051184|6402||7|133|796|||83.81|||4505.60|3649.28|5363.84|0.00||4505.60||856.32| +2452621|39032|709|35966|1051184|6402|10020|7|292|796|89|5.43|7.60|4.86|0.00|432.54|483.27|676.40|30.27|0.00|432.54|462.81|-50.73| +2452621|39032|14577|35966|1051184|6402|10020|7|93|796|71|8.25|9.07|8.97|178.32|636.87|585.75|643.97|32.09|178.32|458.55|490.64|-127.20| +2452621|39032|3093|35966|1051184|6402|10020|7|184|796|11|83.39|108.40|6.50|50.05|71.50|917.29|1192.40|0.42|50.05|21.45|21.87|-895.84| +2452621|39032|17925|35966|1051184|6402|10020|7|51|796|90|5.53|7.57|0.60|0.00|54.00|497.70|681.30|4.86|0.00|54.00|58.86|-443.70| +2452621|39032|7105|35966|1051184|6402|10020|7|107|796|76|37.81|57.09|44.53|0.00|3384.28|2873.56|4338.84|169.21|0.00|3384.28|3553.49|510.72| +2452621|39032|11437|35966|1051184|6402|10020|7|142|796|78|72.76|88.03|75.70|0.00|5904.60|5675.28|6866.34|413.32|0.00|5904.60|6317.92|229.32| +2451841|51721|14924|31604|130671|138|38052|4|250|797|46|14.27|15.83|4.90|0.00|225.40|656.42|728.18|6.76|0.00|225.40|232.16|-431.02| +2451841|51721|4063|31604|130671|138|38052|4|148|797|65|68.28|77.83|10.11|0.00|657.15|4438.20|5058.95|46.00|0.00|657.15|703.15|-3781.05| +2451841|51721|6329|31604|130671|138|38052|4|223|797|92|88.42|129.97|103.97|0.00|9565.24|8134.64|11957.24|765.21|0.00|9565.24|10330.45|1430.60| +2451841|51721|14609|31604|130671|138|38052|4|191|797|68|22.28|29.40|11.76|0.00|799.68|1515.04|1999.20|39.98|0.00|799.68|839.66|-715.36| +2451841|51721|15353|31604|130671|138|38052|4|287|797|59|26.95|30.99|17.97|180.23|1060.23|1590.05|1828.41|17.60|180.23|880.00|897.60|-710.05| +2451841|51721|3079|31604|130671|138|38052|4|172|797|14|52.16|55.28|50.85|0.00|711.90|730.24|773.92|49.83|0.00|711.90|761.73|-18.34| +2451841|51721|14501|31604|130671|138|38052|4|209|797|46|56.65|113.30|11.33|0.00|521.18|2605.90|5211.80|20.84|0.00|521.18|542.02|-2084.72| +2451841|51721|5027|31604|130671|138|38052|4|259|797|40|69.20|125.94|100.75|0.00|4030.00|2768.00|5037.60|282.10|0.00|4030.00|4312.10|1262.00| +2451841|51721|12626|31604|130671|138|38052|4|167|797|25|66.47|75.77|46.21|0.00|1155.25|1661.75|1894.25|80.86|0.00|1155.25|1236.11|-506.50| +2451841|51721|10514|31604|130671|138|38052|4|290|797|27|17.97|28.39|17.88|0.00|482.76|485.19|766.53|24.13|0.00|482.76|506.89|-2.43| +2451841|51721|16094|31604|||38052|4|274|797||4.43|||165.41||||16.19|165.41|||| +2451841|51721|1111|31604|130671|138|38052|4|74|797|76|28.53|29.10|3.20|0.00|243.20|2168.28|2211.60|7.29|0.00|243.20|250.49|-1925.08| +2451841|51721|293|31604|130671|138|38052|4|130|797|49|27.35|35.00|17.15|0.00|840.35|1340.15|1715.00|33.61|0.00|840.35|873.96|-499.80| +2451792|63198|5657|95935||5530||||798||21.15|31.30||0.00|0.00|2051.55|3036.10||0.00||0.00|-2051.55| +2451792|63198|8918|95935|1277309|5530|6104|1|143|798|26|44.92|79.05|75.88|1953.15|1972.88|1167.92|2055.30|1.18|1953.15|19.73|20.91|-1148.19| +2451792|63198|7166|95935|1277309|5530|6104|1|162|798|76|95.75|165.64|28.15|1839.88|2139.40|7277.00|12588.64|20.96|1839.88|299.52|320.48|-6977.48| +2451792|63198|11107|95935|1277309|5530|6104|1|266|798|74|60.27|67.50|39.15|956.04|2897.10|4459.98|4995.00|116.46|956.04|1941.06|2057.52|-2518.92| +2451792|63198|7663|95935|1277309|5530|6104|1|164|798|75|68.66|89.94|8.09|0.00|606.75|5149.50|6745.50|54.60|0.00|606.75|661.35|-4542.75| +2451792|63198|13574|95935|1277309|5530|6104|1|48|798|67|78.18|101.63|32.52|0.00|2178.84|5238.06|6809.21|0.00|0.00|2178.84|2178.84|-3059.22| +2451792|63198|10481|95935|1277309|5530|6104|1|208|798|92|2.55|3.59|1.86|0.00|171.12|234.60|330.28|8.55|0.00|171.12|179.67|-63.48| +2451792|63198|10849|95935|1277309|5530|6104|1|102|798|67|21.86|43.72|31.04|1746.93|2079.68|1464.62|2929.24|9.98|1746.93|332.75|342.73|-1131.87| +2451792|63198|16853|95935|1277309|5530|6104|1|225|798|71|86.49|89.08|15.14|0.00|1074.94|6140.79|6324.68|96.74|0.00|1074.94|1171.68|-5065.85| +2451792|63198|13142|95935|1277309|5530|6104|1|126|798|78|53.02|84.30|26.13|0.00|2038.14|4135.56|6575.40|20.38|0.00|2038.14|2058.52|-2097.42| +2451792|63198|302|95935|1277309|5530|6104|1|5|798|95|59.27|82.97|67.20|4404.96|6384.00|5630.65|7882.15|19.79|4404.96|1979.04|1998.83|-3651.61| +2451127|36026|5876|74600|1502899|2045|25991|8|128|799|54|98.71|120.42|12.04|0.00|650.16|5330.34|6502.68|45.51|0.00|650.16|695.67|-4680.18| +2451127|36026|8330|74600|1502899|2045|25991|8|79|799|43|78.65|93.59|73.93|0.00|3178.99|3381.95|4024.37|254.31|0.00|3178.99|3433.30|-202.96| +2451127|36026|12028|74600|1502899|2045|25991|8|222|799|47|87.84|169.53|162.74|0.00|7648.78|4128.48|7967.91|305.95|0.00|7648.78|7954.73|3520.30| +2451127|36026|16564|74600|1502899|2045|25991|8|139|799|3|80.80|122.81|40.52|0.00|121.56|242.40|368.43|3.64|0.00|121.56|125.20|-120.84| +2451127|36026|9964|74600|1502899|2045|25991|8|39|799|49|98.38|148.55|101.01|4899.99|4949.49|4820.62|7278.95|1.48|4899.99|49.50|50.98|-4771.12| +2451127|36026|6748|74600|1502899|2045|25991|8|201|799|82|69.90|69.90|39.84|0.00|3266.88|5731.80|5731.80|294.01|0.00|3266.88|3560.89|-2464.92| +2451127|36026|15385|74600|1502899|2045|25991|8|200|799|35|19.87|36.36|7.63|0.00|267.05|695.45|1272.60|18.69|0.00|267.05|285.74|-428.40| +2451127|36026|235|74600|1502899|2045|25991|8|299|799|33|67.81|78.65|39.32|51.90|1297.56|2237.73|2595.45|24.91|51.90|1245.66|1270.57|-992.07| +2451127|36026|397|74600|1502899|2045|25991|8|41|799|81|35.78|43.29|15.15|0.00|1227.15|2898.18|3506.49|36.81|0.00|1227.15|1263.96|-1671.03| +|36026|7099|||2045||||799||74.70||||975.27||5420.92|68.26||975.27|1043.53|-3432.03| +2451127|36026|8944|74600|1502899|2045|25991|8|171|799|17|54.37|71.76|46.64|0.00|792.88|924.29|1219.92|47.57|0.00|792.88|840.45|-131.41| +2451127|36026|10568|74600|1502899|2045|25991|8|12|799|66|57.47|81.03|38.89|0.00|2566.74|3793.02|5347.98|51.33|0.00|2566.74|2618.07|-1226.28| +2451127|36026|17528|74600|1502899|2045|25991|8|153|799|90|54.37|92.42|60.07|1513.76|5406.30|4893.30|8317.80|194.62|1513.76|3892.54|4087.16|-1000.76| +2451127|36026|4846|74600|1502899|2045|25991|8|78|799|43|79.15|146.42|98.10|0.00|4218.30|3403.45|6296.06|379.64|0.00|4218.30|4597.94|814.85| +2451127|36026|490|74600|1502899|2045|25991|8|263|799|93|80.56|128.89|10.31|0.00|958.83|7492.08|11986.77|57.52|0.00|958.83|1016.35|-6533.25| +2451475|62535|5744|8574|667683|1639|395|7|234|800|1|18.96|33.36|19.01|0.00|19.01|18.96|33.36|1.71|0.00|19.01|20.72|0.05| +2451475|62535|184|8574|667683|1639|395|7|54|800|7|62.68|68.32|31.42|0.00|219.94|438.76|478.24|13.19|0.00|219.94|233.13|-218.82| +2451475|62535|2120|8574|667683|1639|395|7|114|800|7|38.62|50.59|40.47|223.79|283.29|270.34|354.13|5.35|223.79|59.50|64.85|-210.84| +2451475|62535|175|8574|667683|1639|395|7|236|800|62|18.80|33.27|8.98|0.00|556.76|1165.60|2062.74|44.54|0.00|556.76|601.30|-608.84| +2451475|62535|15838|8574|667683|1639|395|7|191|800|60|14.89|15.18|7.43|0.00|445.80|893.40|910.80|17.83|0.00|445.80|463.63|-447.60| +2451475|62535|16486|8574|667683|1639|395|7|268|800|84|89.05|178.10|106.86|0.00|8976.24|7480.20|14960.40|89.76|0.00|8976.24|9066.00|1496.04| +2451475|62535|2986|8574|667683|1639|395|7|271|800|59|26.21|48.22|26.52|1236.09|1564.68|1546.39|2844.98|26.28|1236.09|328.59|354.87|-1217.80| +2451475|62535|14552|8574|667683|1639|395|7|147|800|90|78.32|90.06|18.01|0.00|1620.90|7048.80|8105.40|97.25|0.00|1620.90|1718.15|-5427.90| +2451475|62535|7879|8574|667683|1639|395|7|155|800|55|3.33|4.76|0.85|0.00|46.75|183.15|261.80|1.87|0.00|46.75|48.62|-136.40| +2451475|62535|4159|8574|667683|1639|395|7|263|800|78|26.89|42.48|3.82|0.00|297.96|2097.42|3313.44|2.97|0.00|297.96|300.93|-1799.46| +2451475|62535|15925|8574|667683|1639|395|7|151|800|23|72.04|73.48|64.66|0.00|1487.18|1656.92|1690.04|14.87|0.00|1487.18|1502.05|-169.74| +2451475|62535|8923|8574|667683|1639|395|7|293|800|83|35.64|69.85|50.29|0.00|4174.07|2958.12|5797.55|0.00|0.00|4174.07|4174.07|1215.95| +2452129|39774|17961|27296|1404153|4657|46361|8|180|801|58|47.12|49.00|34.30|0.00|1989.40|2732.96|2842.00|39.78|0.00|1989.40|2029.18|-743.56| +2452129|39774|14721|27296|1404153|4657|46361|8|245|801|29|17.82|17.82|4.45|0.00|129.05|516.78|516.78|6.45|0.00|129.05|135.50|-387.73| +2452129|39774|17217|27296|1404153|4657|46361|8|7|801|65|47.88|61.28|57.60|0.00|3744.00|3112.20|3983.20|37.44|0.00|3744.00|3781.44|631.80| +2452129|39774|3617|27296|1404153|4657|46361|8|40|801|72|5.76|6.45|2.32|0.00|167.04|414.72|464.40|5.01|0.00|167.04|172.05|-247.68| +2452129|39774|13283|27296|1404153|4657|46361|8|283|801|77|92.92|117.07|36.29|0.00|2794.33|7154.84|9014.39|139.71|0.00|2794.33|2934.04|-4360.51| +2452129|39774|13631|27296|1404153|4657|46361|8|189|801|83|96.12|174.93|31.48|0.00|2612.84|7977.96|14519.19|104.51|0.00|2612.84|2717.35|-5365.12| +2452129|39774|5981|27296|1404153|4657|46361|8|279|801|24|13.58|18.60|14.69|126.92|352.56|325.92|446.40|0.00|126.92|225.64|225.64|-100.28| +2452129|39774|4553|27296|1404153|4657|46361|8|256|801|15|19.89|33.21|6.30|0.00|94.50|298.35|498.15|2.83|0.00|94.50|97.33|-203.85| +2452129|39774|10993|27296|1404153|4657|46361|8|208|801|33|78.59|102.16|83.77|0.00|2764.41|2593.47|3371.28|193.50|0.00|2764.41|2957.91|170.94| +2452129|39774|49|27296|1404153|4657|46361|8|187|801|42|21.05|31.15|15.57|0.00|653.94|884.10|1308.30|45.77|0.00|653.94|699.71|-230.16| +2451515|56640|13934|12512|1647398|1131|49217|8|137|802|6|3.45|6.90|0.55|0.00|3.30|20.70|41.40|0.23|0.00|3.30|3.53|-17.40| +2451515|56640|15136|12512|1647398|1131|49217|8|10|802|27|89.45|146.69|137.88|2531.47|3722.76|2415.15|3960.63|59.56|2531.47|1191.29|1250.85|-1223.86| +2451515|56640|8671|12512|1647398|1131|49217|8|260|802|94|3.78|6.72|5.51|0.00|517.94|355.32|631.68|15.53|0.00|517.94|533.47|162.62| +2451515|56640|1369|12512|1647398|1131|49217|8|51|802|14|15.84|23.91|1.43|0.00|20.02|221.76|334.74|1.40|0.00|20.02|21.42|-201.74| +2451515|56640|14095|12512|1647398|1131|49217|8|40|802|45|98.65|197.30|183.48|0.00|8256.60|4439.25|8878.50|0.00|0.00|8256.60|8256.60|3817.35| +2451515|56640|2630|12512|1647398|1131|49217|8|256|802|14|44.04|71.78|0.00|0.00|0.00|616.56|1004.92|0.00|0.00|0.00|0.00|-616.56| +2451515||10459|12512|1647398|||8|91|802|||||||||116.04||1657.84||430.10| +2451515|56640|1202|12512|1647398|1131|49217|8|64|802|98|84.85|135.76|27.15|0.00|2660.70|8315.30|13304.48|26.60|0.00|2660.70|2687.30|-5654.60| +2451515|56640|775|12512|1647398|1131|49217|8|246|802|29|41.67|46.25|26.36|0.00|764.44|1208.43|1341.25|61.15|0.00|764.44|825.59|-443.99| +2451515|56640|16328|12512|1647398|1131|49217|8|152|802|64|68.97|119.31|1.19|0.00|76.16|4414.08|7635.84|6.85|0.00|76.16|83.01|-4337.92| +2451515|56640|8408|12512|1647398|1131|49217|8|231|802|25|5.13|8.51|1.95|0.00|48.75|128.25|212.75|4.38|0.00|48.75|53.13|-79.50| +2451515|56640|7406|12512|1647398|1131|49217|8|260|802|29|1.62|2.94|2.26|0.00|65.54|46.98|85.26|4.58|0.00|65.54|70.12|18.56| +|50454|14995|||||4||803|72||102.60||1256.47|5983.20||7387.20||1256.47|4726.73|5010.33|| +2451740|50454|1699|68271|76550|5629|33908|4|97|803|59|24.53|37.77|36.63|0.00|2161.17|1447.27|2228.43|129.67|0.00|2161.17|2290.84|713.90| +2451740|50454|1400|68271|76550|5629|33908|4|245|803|62|55.02|101.23|72.88|0.00|4518.56|3411.24|6276.26|361.48|0.00|4518.56|4880.04|1107.32| +2451740||17411|68271|76550||33908||14|803|61||66.09||1061.00||2900.55|4031.49||1061.00|833.66||| +2451740|50454|4892|68271|76550|5629|33908|4|153|803|79|31.03|53.99|12.41|0.00|980.39|2451.37|4265.21|78.43|0.00|980.39|1058.82|-1470.98| +2451740|50454|16592|68271|76550|5629|33908|4|151|803|2|13.70|16.30|2.93|0.00|5.86|27.40|32.60|0.35|0.00|5.86|6.21|-21.54| +2451740|50454|793|68271|76550|5629|33908|4|150|803|40|58.25|92.03|26.68|0.00|1067.20|2330.00|3681.20|53.36|0.00|1067.20|1120.56|-1262.80| +2451740|50454|15817|68271|76550|5629|33908|4|111|803|51|36.94|39.52|32.80|0.00|1672.80|1883.94|2015.52|66.91|0.00|1672.80|1739.71|-211.14| +2451740|50454|17264|68271|76550|5629|33908|4|113|803|5|92.29|159.66|124.53|0.00|622.65|461.45|798.30|24.90|0.00|622.65|647.55|161.20| +2451740|50454|11444|68271|76550|5629|33908|4|298|803|60|25.88|49.17|15.24|0.00|914.40|1552.80|2950.20|54.86|0.00|914.40|969.26|-638.40| +2451740|50454|9809|68271|76550|5629|33908|4|203|803|58|57.47|108.61|84.71|786.10|4913.18|3333.26|6299.38|206.35|786.10|4127.08|4333.43|793.82| +2451740|50454|16903|68271|76550|5629|33908|4|60|803|32|90.89|140.87|26.76|0.00|856.32|2908.48|4507.84|68.50|0.00|856.32|924.82|-2052.16| +2451047|46832|2150|27076|128918|3207|21049|7|291|804|40|56.08|60.56|1.21|0.00|48.40|2243.20|2422.40|0.00|0.00|48.40|48.40|-2194.80| +2451047|46832|14635|27076|128918|3207|21049|7|226|804|44|37.89|46.60|38.67|0.00|1701.48|1667.16|2050.40|51.04|0.00|1701.48|1752.52|34.32| +2451047|46832|14488|27076|128918|3207|21049|7|183|804|10|33.88|48.10|41.84|359.82|418.40|338.80|481.00|1.17|359.82|58.58|59.75|-280.22| +2451047|46832|3454|27076|128918|3207|21049|7|59|804|96|11.59|12.74|6.11|0.00|586.56|1112.64|1223.04|35.19|0.00|586.56|621.75|-526.08| +2451047|46832|10159|27076|128918|3207|21049|7|193|804|20|2.76|5.35|3.79|0.00|75.80|55.20|107.00|6.82|0.00|75.80|82.62|20.60| +2451047|46832|17752|27076|128918|3207|21049|7|254|804|18|21.02|21.44|5.57|0.00|100.26|378.36|385.92|2.00|0.00|100.26|102.26|-278.10| +2451047|46832|1333||||21049|||804|68|8.89|15.73|8.65|0.00|||1069.64|29.41|0.00||617.61|| +2451047|46832|17000|27076|128918|3207|21049|7|169|804|23|35.84|69.17|42.88|0.00|986.24|824.32|1590.91|78.89|0.00|986.24|1065.13|161.92| +2451047|46832|5954|27076|128918|3207|21049|7|261|804|46|21.11|30.39|24.00|0.00|1104.00|971.06|1397.94|99.36|0.00|1104.00|1203.36|132.94| +||2608||128918|3207|||78|804|||102.37|||1970.10|3656.40|5630.35|||1970.10||-1686.30| +2451047|46832|10328|27076|128918|3207|21049|7|257|804|41|75.10|105.14|29.43|0.00|1206.63|3079.10|4310.74|0.00|0.00|1206.63|1206.63|-1872.47| +2451047|46832|7376|27076|128918|3207|21049|7|144|804|28|28.09|52.52|41.49|0.00|1161.72|786.52|1470.56|104.55|0.00|1161.72|1266.27|375.20| +2451047|46832|13279|27076|128918|3207|21049|7|6|804|27|42.49|50.56|1.01|0.00|27.27|1147.23|1365.12|2.18|0.00|27.27|29.45|-1119.96| +2451047|46832|3944|27076|128918|3207|21049|7|232|804|74|95.16|126.56|48.09|2028.43|3558.66|7041.84|9365.44|107.11|2028.43|1530.23|1637.34|-5511.61| +2451047|46832|14180|27076|128918|3207|21049|7|87|804|79|56.04|69.48|8.33|230.32|658.07|4427.16|5488.92|21.38|230.32|427.75|449.13|-3999.41| +2452046|61318|13565|80665|327368|4824|26727|2|127|805|52|42.66|51.19|34.80|0.00|1809.60|2218.32|2661.88|72.38|0.00|1809.60|1881.98|-408.72| +2452046|61318|14943|80665|327368|4824|26727|2|290|805|56|90.24|128.14|3.84|0.00|215.04|5053.44|7175.84|0.00|0.00|215.04|215.04|-4838.40| +2452046|61318|15365|80665|327368|4824|26727|2|207|805|54|39.40|72.89|72.16|0.00|3896.64|2127.60|3936.06|233.79|0.00|3896.64|4130.43|1769.04| +2452046|61318|13081|80665|327368|4824|26727|2|232|805|51|83.54|105.26|27.36|0.00|1395.36|4260.54|5368.26|0.00|0.00|1395.36|1395.36|-2865.18| +2452046|61318|8315|80665|327368|4824|26727|2|113|805|100|67.35|114.49|6.86|0.00|686.00|6735.00|11449.00|0.00|0.00|686.00|686.00|-6049.00| +2452046|61318|17535|80665|327368|4824|26727|2|53|805|24|85.90|98.78|4.93|0.00|118.32|2061.60|2370.72|1.18|0.00|118.32|119.50|-1943.28| +2452046|61318|11251|80665|327368|4824|26727|2|217|805|82|86.31|89.76|59.24|0.00|4857.68|7077.42|7360.32|97.15|0.00|4857.68|4954.83|-2219.74| +2452046|61318|10551|80665|327368|4824|26727|2|120|805|64|41.92|49.88|9.97|0.00|638.08|2682.88|3192.32|51.04|0.00|638.08|689.12|-2044.80| +2452046|61318|5099|80665|327368|4824|26727|2|57|805|30|4.35|8.09|5.98|0.00|179.40|130.50|242.70|12.55|0.00|179.40|191.95|48.90| +2452046|61318|10095|80665|327368|4824|26727|2|289|805|40|57.90|69.48|6.25|0.00|250.00|2316.00|2779.20|2.50|0.00|250.00|252.50|-2066.00| +2451738|52769|8005|11043|1831290|6737|24184|2|41|806|13|81.81|97.35|93.45|413.04|1214.85|1063.53|1265.55|72.16|413.04|801.81|873.97|-261.72| +2451738|52769|787|11043|1831290|6737|24184|2|151|806|85|7.01|13.31|10.51|0.00|893.35|595.85|1131.35|0.00|0.00|893.35|893.35|297.50| +2451738|52769|8108|11043|1831290|6737|24184|2|21|806|15|92.16|120.72|47.08|0.00|706.20|1382.40|1810.80|14.12|0.00|706.20|720.32|-676.20| +2451738|52769|6613|11043|1831290|6737|24184|2|120|806|86|42.84|52.26|10.45|0.00|898.70|3684.24|4494.36|0.00|0.00|898.70|898.70|-2785.54| +2451738|52769|7208|11043|1831290|6737|24184|2|34|806|48|49.72|65.63|11.81|402.48|566.88|2386.56|3150.24|11.50|402.48|164.40|175.90|-2222.16| +2451738|52769|11939|11043|1831290|6737|24184|2|127|806|72|60.94|89.58|17.91|0.00|1289.52|4387.68|6449.76|64.47|0.00|1289.52|1353.99|-3098.16| +|52769|4442|11043|||24184|2||806|83|9.35||3.39||281.37|776.05||||||| +2451738|52769|10775|11043|1831290|6737|24184|2|142|806|76|4.52|7.23|6.57|0.00|499.32|343.52|549.48|9.98|0.00|499.32|509.30|155.80| +2451738|52769|6719|11043|1831290|6737|24184|2|77|806|10|2.68|2.70|0.45|0.00|4.50|26.80|27.00|0.18|0.00|4.50|4.68|-22.30| +2451738|52769|15422|11043|1831290|6737|24184|2|98|806|100|13.16|19.08|1.14|92.34|114.00|1316.00|1908.00|1.73|92.34|21.66|23.39|-1294.34| +2451738|52769|12248|11043|1831290|6737|24184|2|252|806|8|32.46|40.89|12.26|0.00|98.08|259.68|327.12|3.92|0.00|98.08|102.00|-161.60| +2451738|52769|9619|11043|1831290|6737|24184|2|162|806|68|86.36|117.44|51.67|632.44|3513.56|5872.48|7985.92|201.67|632.44|2881.12|3082.79|-2991.36| +2451738|52769|6053|11043|1831290|6737|24184|2|47|806|18|21.99|32.10|9.63|0.00|173.34|395.82|577.80|1.73|0.00|173.34|175.07|-222.48| +|53708|17373|||6922|||233|807|94|19.53|31.83||0.00|0.00|1835.82|2992.02||0.00||0.00|| +2452243|53708|17251|24341|737408|6922|11189|2|95|807|88|32.16|48.88|14.17|0.00|1246.96|2830.08|4301.44|12.46|0.00|1246.96|1259.42|-1583.12| +2452243|53708|13687|24341|737408|6922|11189|2|143|807|36|10.32|12.79|9.97|236.88|358.92|371.52|460.44|4.88|236.88|122.04|126.92|-249.48| +2452243|53708|13841|24341|737408|6922|11189|2|221|807|52|20.87|32.97|13.18|657.94|685.36|1085.24|1714.44|2.46|657.94|27.42|29.88|-1057.82| +2452243|53708|14003|24341|737408|6922|11189|2|30|807|39|63.98|77.41|61.15|0.00|2384.85|2495.22|3018.99|95.39|0.00|2384.85|2480.24|-110.37| +2452243|53708|17133|24341|737408|6922|11189|2|114|807|47|96.82|154.91|10.84|45.85|509.48|4550.54|7280.77|9.27|45.85|463.63|472.90|-4086.91| +2452243|53708|10931|24341|737408|6922|11189|2|88|807|81|27.47|29.94|0.29|13.62|23.49|2225.07|2425.14|0.19|13.62|9.87|10.06|-2215.20| +2452243|53708|1739|24341|737408|6922|11189|2|149|807|86|81.86|81.86|42.56|0.00|3660.16|7039.96|7039.96|183.00|0.00|3660.16|3843.16|-3379.80| +2452243|53708|3357||||11189|2|68|807|||||0.00||2411.70||18.73|0.00|1873.50|1892.23|| +2452243|53708|9369|24341|737408|6922|11189|2|276|807|86|49.46|88.53|69.05|0.00|5938.30|4253.56|7613.58|178.14|0.00|5938.30|6116.44|1684.74| +2452243|53708|3201|24341|737408|6922|11189|2|15|807|86|44.45|87.56|82.30|0.00|7077.80|3822.70|7530.16|283.11|0.00|7077.80|7360.91|3255.10| +2452353|72465|4008|3329|845164|2999|44081|7|247|808|6|14.93|26.27|0.78|3.93|4.68|89.58|157.62|0.01|3.93|0.75|0.76|-88.83| +2452353|72465|12103|3329|845164|2999|44081|7|58|808|86|66.78|82.80|3.31|0.00|284.66|5743.08|7120.80|17.07|0.00|284.66|301.73|-5458.42| +2452353|72465|11487|3329|845164|2999|44081|7|180|808|17|7.95|15.34|7.51|0.00|127.67|135.15|260.78|10.21|0.00|127.67|137.88|-7.48| +2452353|72465|14427|3329|845164|2999|44081|||808|83|80.77||63.02|0.00|5230.66|6703.91||261.53|0.00|||-1473.25| +2452353|72465|9408|3329|845164|2999|44081|7|161|808|42|75.46|94.32|11.31|0.00|475.02|3169.32|3961.44|9.50|0.00|475.02|484.52|-2694.30| +2452353|72465|16890|3329|845164|2999|44081|7|93|808|37|29.22|37.10|26.71|88.94|988.27|1081.14|1372.70|0.00|88.94|899.33|899.33|-181.81| +2452353|72465|14514|3329|845164|2999|44081|7|235|808|82|89.91|110.58|33.17|0.00|2719.94|7372.62|9067.56|244.79|0.00|2719.94|2964.73|-4652.68| +2452353|72465|16087|3329|845164|2999|44081|7|34|808|49|54.81|99.75|25.93|0.00|1270.57|2685.69|4887.75|12.70|0.00|1270.57|1283.27|-1415.12| +2452353|72465|16807|3329|845164|2999|44081|7|213|808|43|22.89|23.11|6.70|0.00|288.10|984.27|993.73|5.76|0.00|288.10|293.86|-696.17| +2452353|72465|7620|3329|845164|2999|44081|7|165|808|64|5.74|10.50|7.14|242.18|456.96|367.36|672.00|17.18|242.18|214.78|231.96|-152.58| +2452353|72465|16129|3329|845164|2999|44081|7|112|808|8|42.21|56.98|27.92|0.00|223.36|337.68|455.84|0.00|0.00|223.36|223.36|-114.32| +2452353|72465|17757|3329|845164||44081|7|77|808|||48.87|||2242.98|1743.18|2492.37|157.00||||| +2452353|72465|9039|3329|845164|2999|44081|7|185|808|60|34.48|62.40|19.96|455.08|1197.60|2068.80|3744.00|0.00|455.08|742.52|742.52|-1326.28| +2452353|72465|6|3329|845164|2999|44081|7|78|808|31|31.50|44.41|23.09|0.00|715.79|976.50|1376.71|57.26|0.00|715.79|773.05|-260.71| +2452353|72465|13515|3329|845164|2999|44081|7|136|808|80|43.47|57.81|52.60|0.00|4208.00|3477.60|4624.80|42.08|0.00|4208.00|4250.08|730.40| +2452353|72465|177|3329|845164|2999|44081|7|157|808|32|17.20|20.12|1.60|0.00|51.20|550.40|643.84|3.07|0.00|51.20|54.27|-499.20| +2452557|61896|3012|44134|233672|5257|30042|8|234|809|81|72.60|72.60|48.64|1575.93|3939.84|5880.60|5880.60|141.83|1575.93|2363.91|2505.74|-3516.69| +2452557|61896|11427|44134|233672|5257|30042|8|277|809|92|1.55|1.61|1.44|80.81|132.48|142.60|148.12|2.06|80.81|51.67|53.73|-90.93| +2452557|61896|15715|44134|233672|5257|30042|8|260|809|81|68.32|127.07|125.79|0.00|10188.99|5533.92|10292.67|0.00|0.00|10188.99|10188.99|4655.07| +2452557|61896|8101|44134|233672|5257|30042|8|75|809|73|87.08|112.33|74.13|0.00|5411.49|6356.84|8200.09|54.11|0.00|5411.49|5465.60|-945.35| +||8025||233672|||8||809||60.58||73.11||5848.80||6881.60|||||| +2452557|61896|8605|44134|233672|5257|30042|8|261|809|72|53.65|100.32|98.31|0.00|7078.32|3862.80|7223.04|353.91|0.00|7078.32|7432.23|3215.52| +2452557|61896|2889|44134|233672|5257|30042|8|32|809|93|98.55|124.17|39.73|0.00|3694.89|9165.15|11547.81|184.74|0.00|3694.89|3879.63|-5470.26| +2452557|61896|13923|44134|233672|5257|30042|8|299|809|2|41.18|63.82|35.73|0.00|71.46|82.36|127.64|5.71|0.00|71.46|77.17|-10.90| +2452557|61896|14361|44134|233672|5257|30042|8|291|809|96|29.12|55.32|14.38|0.00|1380.48|2795.52|5310.72|41.41|0.00|1380.48|1421.89|-1415.04| +2452212|48908|435|17684|1164051|2771|21891|1|27|810|90|67.94|87.64|41.19|2854.46|3707.10|6114.60|7887.60|42.63|2854.46|852.64|895.27|-5261.96| +2452212|48908|14941|17684|1164051|2771|21891|1|213|810|47|37.06|60.77|17.01|0.00|799.47|1741.82|2856.19|55.96|0.00|799.47|855.43|-942.35| +2452212|48908|11637|17684|1164051|2771|21891|1|172|810|53|8.43|12.47|8.60|0.00|455.80|446.79|660.91|9.11|0.00|455.80|464.91|9.01| +2452212|48908|2501|17684|1164051|2771|21891|1|277|810|18|26.96|42.05|1.26|0.00|22.68|485.28|756.90|0.22|0.00|22.68|22.90|-462.60| +2452212|48908|14485|17684|1164051|2771|21891|1|288|810|84|45.84|78.84|53.61|0.00|4503.24|3850.56|6622.56|0.00|0.00|4503.24|4503.24|652.68| +2452212|48908|5899|17684|1164051|2771|21891|1|150|810|38|82.49|87.43|79.56|0.00|3023.28|3134.62|3322.34|272.09|0.00|3023.28|3295.37|-111.34| +2452212|48908|8989|17684|1164051|2771|21891|1|109|810|23|30.57|46.77|8.88|0.00|204.24|703.11|1075.71|18.38|0.00|204.24|222.62|-498.87| +|48908|1061|17684|1164051|2771|21891|||810|53|||46.84|0.00|2482.52||7092.99|173.77|0.00|2482.52||-1547.60| +2452212|48908|5639|17684|1164051|2771|21891|1|158|810|61|11.12|17.68|5.12|0.00|312.32|678.32|1078.48|24.98|0.00|312.32|337.30|-366.00| +2452212|48908|17009|17684|1164051|2771|21891|1|185|810|46|56.02|106.43|17.02|7.82|782.92|2576.92|4895.78|62.00|7.82|775.10|837.10|-1801.82| +2452212|48908|15029|17684|1164051|2771|21891|1|86|810|51|66.84|78.87|29.97|0.00|1528.47|3408.84|4022.37|137.56|0.00|1528.47|1666.03|-1880.37| +2451184|38928|12583|32616|668488|4360|39432|4|293|811|82|99.53|112.46|35.98|0.00|2950.36|8161.46|9221.72|265.53|0.00|2950.36|3215.89|-5211.10| +2451184|38928|7120|32616|668488|4360|39432|4|293|811|78|9.62|14.04|9.54|0.00|744.12|750.36|1095.12|29.76|0.00|744.12|773.88|-6.24| +2451184|38928|11851|32616|668488|4360|39432|4|192|811|43|26.24|26.76|16.59|0.00|713.37|1128.32|1150.68|42.80|0.00|713.37|756.17|-414.95| +2451184|38928|17678|32616|668488|4360|39432|4|58|811|67|56.71|72.58|2.90|0.00|194.30|3799.57|4862.86|3.88|0.00|194.30|198.18|-3605.27| +2451184|38928|5479|32616|668488|4360|39432|4|174|811|34|9.86|10.84|5.09|25.95|173.06|335.24|368.56|11.76|25.95|147.11|158.87|-188.13| +2451184|38928|3826|32616|668488|4360|39432|4|173|811|71|49.38|75.55|24.93|0.00|1770.03|3505.98|5364.05|53.10|0.00|1770.03|1823.13|-1735.95| +|38928|2476||||39432|4|188|811|13||107.85|||1331.85|801.19||106.54|||1438.39|530.66| +2451184|38928|3568|32616|668488|4360|39432|4|144|811|30|87.02|123.56|46.95|0.00|1408.50|2610.60|3706.80|56.34|0.00|1408.50|1464.84|-1202.10| +2451184|38928|8110|32616|668488|4360|39432|4|182|811|83|14.96|25.73|15.95|0.00|1323.85|1241.68|2135.59|79.43|0.00|1323.85|1403.28|82.17| +2451184|38928|4948|32616|668488|4360|39432|4|15|811|17|37.99|47.10|37.68|0.00|640.56|645.83|800.70|0.00|0.00|640.56|640.56|-5.27| +2451184|38928|9002|32616|668488|4360|39432|4|298|811|61|6.02|9.87|8.29|0.00|505.69|367.22|602.07|0.00|0.00|505.69|505.69|138.47| +2451184|38928|5551|32616|668488|4360|39432|4|87|811|23|46.32|71.79|43.79|0.00|1007.17|1065.36|1651.17|50.35|0.00|1007.17|1057.52|-58.19| +2451184|38928|4708|32616|668488|4360|39432|4|34|811|13|88.95|175.23|31.54|28.70|410.02|1156.35|2277.99|15.25|28.70|381.32|396.57|-775.03| +2451371|52209|3247|51958|557123|3535|7900|7|72|812|17|30.39|54.70|16.41|0.00|278.97|516.63|929.90|25.10|0.00|278.97|304.07|-237.66| +2451371|52209|9356|51958|557123|3535|7900|7|250|812|40|68.69|107.15|28.93|0.00|1157.20|2747.60|4286.00|23.14|0.00|1157.20|1180.34|-1590.40| +2451371|52209|2488|51958|557123|3535|7900|7|285|812|19|5.06|7.28|4.00|40.28|76.00|96.14|138.32|0.00|40.28|35.72|35.72|-60.42| +2451371|52209|6850|51958|557123|3535|7900|7|216|812|1|65.61|92.51|44.40|0.00|44.40|65.61|92.51|1.77|0.00|44.40|46.17|-21.21| +2451371||250|||3535|7900||204|812|87|50.48|||0.00||4391.76|||0.00|||| +2451371|52209|16453|51958|557123|3535|7900|7|239|812|67|86.93|157.34|108.56|0.00|7273.52|5824.31|10541.78|290.94|0.00|7273.52|7564.46|1449.21| +2451371|52209|17594|51958|557123|3535|7900|7|33|812|71|19.78|23.14|9.48|592.31|673.08|1404.38|1642.94|4.84|592.31|80.77|85.61|-1323.61| +2451371||13216|||||7|70|812||94.12|114.82|||9093.15|9317.88|11367.18|||9093.15||-224.73| +2451371|52209|2908|51958|557123|3535|7900|7|59|812|70|75.77|121.23|32.73|710.24|2291.10|5303.90|8486.10|63.23|710.24|1580.86|1644.09|-3723.04| +|52209|6440|51958|557123|3535||||812|18||42.86|40.71|||602.82||||732.78|784.07|129.96| +2451371|52209|6794|51958|557123|3535|7900|7|116|812|85|73.79|88.54|28.33|0.00|2408.05|6272.15|7525.90|24.08|0.00|2408.05|2432.13|-3864.10| +2451371|52209|17713|51958|557123|3535|7900|7|68|812|83|83.10|138.77|5.55|0.00|460.65|6897.30|11517.91|9.21|0.00|460.65|469.86|-6436.65| +2451371|52209|794|51958|557123|3535|7900|7|92|812|82|80.53|143.34|34.40|0.00|2820.80|6603.46|11753.88|112.83|0.00|2820.80|2933.63|-3782.66| +2451371|52209|1129|51958|557123|3535|7900|7|205|812|14|68.20|83.20|38.27|0.00|535.78|954.80|1164.80|21.43|0.00|535.78|557.21|-419.02| +2452134|42500|14343|5534|1358988|2489|48343|4|31|813|73|47.74|78.29|44.62|0.00|3257.26|3485.02|5715.17|0.00|0.00|3257.26|3257.26|-227.76| +2452134|42500|12253|5534|1358988|2489|48343|4|246|813|13|56.15|65.13|26.70|0.00|347.10|729.95|846.69|31.23|0.00|347.10|378.33|-382.85| +2452134|42500|4741|5534|1358988|2489|48343|4|30|813|38|10.23|17.90|15.39|0.00|584.82|388.74|680.20|29.24|0.00|584.82|614.06|196.08| +|42500|16521|||2489||4|59|813|41|79.99||||3935.18||3935.18||||4171.29|655.59| +2452134|42500|9235|5534|1358988|2489|48343|4|74|813|84|71.23|113.96|103.70|0.00|8710.80|5983.32|9572.64|522.64|0.00|8710.80|9233.44|2727.48| +2452134|42500|11871|5534|1358988|2489|48343|4|216|813|84|91.32|182.64|91.32|0.00|7670.88|7670.88|15341.76|690.37|0.00|7670.88|8361.25|0.00| +2452134||3225|||2489||4|176|813||||||515.28|||10.30||||-141.93| +2452134|42500|283|5534|1358988|2489|48343|4|140|813|68|29.11|55.30|51.98|989.69|3534.64|1979.48|3760.40|0.00|989.69|2544.95|2544.95|565.47| +|42500|8799||1358988|2489||||813||12.34|22.70|||471.87|259.14||9.43||||212.73| +2452134|42500|13439|5534|1358988|2489|48343|4|30|813|35|23.20|40.13|30.09|0.00|1053.15|812.00|1404.55|84.25|0.00|1053.15|1137.40|241.15| +2452134|42500|13681|5534|1358988|2489|48343|4|153|813|41|24.50|37.97|34.93|0.00|1432.13|1004.50|1556.77|0.00|0.00|1432.13|1432.13|427.63| +2452134|42500|8323|5534|1358988|2489|48343|4|2|813|18|87.40|119.73|14.36|0.00|258.48|1573.20|2155.14|2.58|0.00|258.48|261.06|-1314.72| +2452134|42500|7491|5534|1358988|2489|48343|4|1|813|1|87.43|108.41|27.10|26.01|27.10|87.43|108.41|0.00|26.01|1.09|1.09|-86.34| +2452134|42500|5927|5534|1358988|2489|48343|4|96|813|5|54.41|70.18|63.86|271.40|319.30|272.05|350.90|2.39|271.40|47.90|50.29|-224.15| +2452134|42500|15321|5534|1358988|2489|48343|4|285|813|56|19.57|23.48|8.92|0.00|499.52|1095.92|1314.88|14.98|0.00|499.52|514.50|-596.40| +||13982|40909|||950||166|814||16.04||7.80|0.00|522.60|1074.68|||0.00|522.60|527.82|-552.08| +2451550|70547|1831|40909|13307|552|950|2|4|814|85|12.20|22.93|2.06|0.00|175.10|1037.00|1949.05|0.00|0.00|175.10|175.10|-861.90| +2451550|70547|10961|40909|13307|552|950|2|16|814|73|54.97|100.59|86.50|6314.50|6314.50|4012.81|7343.07|0.00|6314.50|0.00|0.00|-4012.81| +||1853||13307||950|2|227|814|64|3.74||1.55|0.00||239.36|||0.00||104.16|| +2451550|70547|13673|40909|13307|552|950|2|207|814|2|96.32|133.88|133.88|0.00|267.76|192.64|267.76|5.35|0.00|267.76|273.11|75.12| +2451550|70547|9929|40909|13307|552|950|2|38|814|5|58.97|88.45|44.22|50.85|221.10|294.85|442.25|1.70|50.85|170.25|171.95|-124.60| +2451550|70547|3368|40909|13307|552|950|2|75|814|25|2.90|4.00|1.12|0.00|28.00|72.50|100.00|1.96|0.00|28.00|29.96|-44.50| +2451550|70547|15806|40909|13307|552|950|2|290|814|59|12.47|16.08|11.89|0.00|701.51|735.73|948.72|42.09|0.00|701.51|743.60|-34.22| +2451550|70547|5395|40909|13307|552|950|2|6|814|27|44.35|72.73|9.45|0.00|255.15|1197.45|1963.71|15.30|0.00|255.15|270.45|-942.30| +2451550|70547|7001|40909|13307|552|950|2|8|814|7|17.73|27.65|5.25|0.00|36.75|124.11|193.55|0.73|0.00|36.75|37.48|-87.36| +2451550|70547|1585|40909|13307|552|950|2|163|814|68|46.64|59.23|11.84|692.40|805.12|3171.52|4027.64|0.00|692.40|112.72|112.72|-3058.80| +2451550|70547|12530|40909|13307|552|950|2|38|814|79|33.21|43.83|6.57|342.55|519.03|2623.59|3462.57|12.35|342.55|176.48|188.83|-2447.11| +2452064|36490|15145|234|1256405|5376|22747|7|178|815|96|44.17|70.23|50.56|0.00|4853.76|4240.32|6742.08|145.61|0.00|4853.76|4999.37|613.44| +2452064|36490|12793|234|1256405|5376|22747|7|133|815|79|18.19|34.01|22.78|0.00|1799.62|1437.01|2686.79|89.98|0.00|1799.62|1889.60|362.61| +2452064|36490|715|234|1256405|5376|22747|7|172|815|45|44.69|67.92|61.80|1362.69|2781.00|2011.05|3056.40|99.28|1362.69|1418.31|1517.59|-592.74| +2452064|36490|16091|234|1256405|5376|22747|7|166|815|93|28.30|28.86|25.10|0.00|2334.30|2631.90|2683.98|116.71|0.00|2334.30|2451.01|-297.60| +2452064|36490|1601|234|1256405|5376|22747|7|153|815|53|26.80|30.82|5.85|58.90|310.05|1420.40|1633.46|15.06|58.90|251.15|266.21|-1169.25| +2452064|36490|15425|234|1256405|5376|22747|7|258|815|92|32.88|60.82|24.93|0.00|2293.56|3024.96|5595.44|137.61|0.00|2293.56|2431.17|-731.40| +||15561|||5376||||815|4||||0.00||378.16||3.24|0.00|||-297.16| +2452064|36490|11821|234|1256405|5376|22747|7|245|815|29|32.90|49.02|1.96|0.00|56.84|954.10|1421.58|0.56|0.00|56.84|57.40|-897.26| +2452064|36490|12313|234|1256405|5376|22747|7|192|815|88|75.41|85.96|55.01|0.00|4840.88|6636.08|7564.48|338.86|0.00|4840.88|5179.74|-1795.20| +||3587|||5376||||815|||68.74|8.24||692.16|2887.08|5774.16|||519.12|550.26|-2367.96| +2452064|36490|2259|234|1256405|5376|22747|7|88|815|91|90.71|109.75|57.07|0.00|5193.37|8254.61|9987.25|103.86|0.00|5193.37|5297.23|-3061.24| +2452064|36490|6787|234|1256405|5376|22747|7|241|815|28|10.47|19.47|13.82|0.00|386.96|293.16|545.16|30.95|0.00|386.96|417.91|93.80| +2452264|37488|7277|58929|205401|3426|7098|8|117|816|62|41.02|70.55|53.61|0.00|3323.82|2543.24|4374.10|199.42|0.00|3323.82|3523.24|780.58| +2452264|37488|16229|58929|205401|3426|7098|8|117|816|1|80.57|94.26|59.38|0.00|59.38|80.57|94.26|0.00|0.00|59.38|59.38|-21.19| +2452264|37488|6711|58929|205401|3426|7098|8|88|816|71|47.90|55.08|53.97|0.00|3831.87|3400.90|3910.68|0.00|0.00|3831.87|3831.87|430.97| +2452264|37488|12837|58929|205401|3426|7098|8|160|816|36|61.01|79.92|68.73|123.71|2474.28|2196.36|2877.12|164.53|123.71|2350.57|2515.10|154.21| +||8461||||7098|||816||||24.49|||117.90||11.02||||| +2452264|37488|16535|58929|205401|3426|7098|8|212|816|22|75.13|82.64|22.31|0.00|490.82|1652.86|1818.08|29.44|0.00|490.82|520.26|-1162.04| +2452264|37488|16099|58929|205401|3426|7098|8|209|816|99|48.37|48.85|9.77|0.00|967.23|4788.63|4836.15|77.37|0.00|967.23|1044.60|-3821.40| +2452264|37488|5781|58929|205401|3426|7098|8|264|816|39|18.29|34.01|8.16|0.00|318.24|713.31|1326.39|0.00|0.00|318.24|318.24|-395.07| +2452264|37488|3651|58929|205401|3426|7098|8|56|816|25|3.77|6.55|0.19|0.00|4.75|94.25|163.75|0.38|0.00|4.75|5.13|-89.50| +||10571||205401||7098|||816|||||0.00||||73.18|0.00|3659.30||621.23| +2452264|37488|5443|58929|205401|3426|7098|8|198|816|51|72.74|73.46|52.89|0.00|2697.39|3709.74|3746.46|80.92|0.00|2697.39|2778.31|-1012.35| +2452264|37488|6011|58929|205401|3426|7098|8|164|816|71|19.95|25.93|18.41|0.00|1307.11|1416.45|1841.03|117.63|0.00|1307.11|1424.74|-109.34| +2452264|37488|7467|58929|205401|3426|7098|8|172|816|68|61.40|111.13|87.79|0.00|5969.72|4175.20|7556.84|417.88|0.00|5969.72|6387.60|1794.52| +2452264|37488|17069|58929|205401|3426|7098|8|118|816|46|85.06|119.08|88.11|0.00|4053.06|3912.76|5477.68|364.77|0.00|4053.06|4417.83|140.30| +2452264|37488|16097|58929|205401|3426|7098|8|125|816|81|97.59|165.90|23.22|0.00|1880.82|7904.79|13437.90|37.61|0.00|1880.82|1918.43|-6023.97| +2451486|37446|2348|10517|1818329|3294|13906|10|185|817|74|51.32|94.94|47.47|0.00|3512.78|3797.68|7025.56|105.38|0.00|3512.78|3618.16|-284.90| +2451486|37446|4004|10517|1818329|3294|13906|10|193|817|6|30.90|46.04|0.00|0.00|0.00|185.40|276.24|0.00|0.00|0.00|0.00|-185.40| +2451486|37446|2665|10517|1818329|3294|13906|10|128|817|60|40.60|60.90|9.13|0.00|547.80|2436.00|3654.00|10.95|0.00|547.80|558.75|-1888.20| +2451486|37446|12878|10517|1818329|3294|13906|10|19|817|98|30.79|53.57|31.07|0.00|3044.86|3017.42|5249.86|121.79|0.00|3044.86|3166.65|27.44| +2451486|37446|7750|10517|1818329|3294|13906|10|43|817|73|84.13|146.38|27.81|0.00|2030.13|6141.49|10685.74|0.00|0.00|2030.13|2030.13|-4111.36| +2451486|37446|7646|10517|1818329|3294|13906|10|215|817|34|17.56|21.24|17.62|0.00|599.08|597.04|722.16|35.94|0.00|599.08|635.02|2.04| +2451486|37446|11300|10517|1818329|3294|13906|10|55|817|92|74.21|141.74|121.89|0.00|11213.88|6827.32|13040.08|784.97|0.00|11213.88|11998.85|4386.56| +2451486|37446|7544|10517|1818329|3294|13906|10|183|817|91|11.76|14.93|7.16|0.00|651.56|1070.16|1358.63|58.64|0.00|651.56|710.20|-418.60| +2451486|37446|14983|10517|1818329|3294|13906|10|168|817|21|77.33|153.88|120.02|1335.82|2520.42|1623.93|3231.48|59.23|1335.82|1184.60|1243.83|-439.33| +2451486|37446|1352|10517|1818329|3294|13906|10|300|817|84|12.73|16.67|13.16|762.75|1105.44|1069.32|1400.28|10.28|762.75|342.69|352.97|-726.63| +2451486|37446|9367|10517|1818329|3294|13906|10|1|817|21|41.77|76.02|55.49|0.00|1165.29|877.17|1596.42|104.87|0.00|1165.29|1270.16|288.12| +2451486|37446|2086|10517|1818329|3294|13906|10|8|817|59|72.27|96.84|80.37|0.00|4741.83|4263.93|5713.56|0.00|0.00|4741.83|4741.83|477.90| +2451486|37446|8546|10517|1818329|3294|13906|10|10|817|9|71.37|114.19|31.97|0.00|287.73|642.33|1027.71|8.63|0.00|287.73|296.36|-354.60| +2451486|37446|13510|10517|1818329|3294|13906|10|134|817|68|4.81|7.31|0.36|0.00|24.48|327.08|497.08|1.46|0.00|24.48|25.94|-302.60| +2451486|37446|16640|10517|1818329|3294|13906|10|5|817|39|20.30|22.93|13.52|0.00|527.28|791.70|894.27|10.54|0.00|527.28|537.82|-264.42| +2451486|37446|16945|10517|1818329|3294|13906|10|71|817|17|81.88|135.10|86.46|0.00|1469.82|1391.96|2296.70|0.00|0.00|1469.82|1469.82|77.86| +2450942|72596|4687|80268|334977|2839|42301|8|217|818|18|59.14|116.50|26.79|0.00|482.22|1064.52|2097.00|14.46|0.00|482.22|496.68|-582.30| +2450942|72596|9310|80268|||42301|||818||||35.51|909.05|1136.32|1516.16|1804.16|6.81|909.05|227.27||| +2450942|72596|13198|80268|334977|2839|42301|8|268|818|73|77.48|109.24|81.93|0.00|5980.89|5656.04|7974.52|358.85|0.00|5980.89|6339.74|324.85| +2450942|72596|10132|80268|334977|2839|42301|8|125|818|90|14.60|15.62|13.74|0.00|1236.60|1314.00|1405.80|98.92|0.00|1236.60|1335.52|-77.40| +2450942|72596|4807|80268|334977|2839|42301|8|211|818|18|86.39|100.21|88.18|0.00|1587.24|1555.02|1803.78|31.74|0.00|1587.24|1618.98|32.22| +2450942|72596|15686|80268|334977|2839|42301|8|232|818|42|43.40|80.72|70.22|0.00|2949.24|1822.80|3390.24|29.49|0.00|2949.24|2978.73|1126.44| +2450942|72596|10100|80268|334977|2839|42301|8|50|818|98|71.17|120.27|61.33|0.00|6010.34|6974.66|11786.46|120.20|0.00|6010.34|6130.54|-964.32| +2450942|72596|5504|80268|334977|2839|42301|8|180|818|29|31.04|46.87|5.62|0.00|162.98|900.16|1359.23|14.66|0.00|162.98|177.64|-737.18| +2450942|72596|12211|80268|334977|2839|42301|8|84|818|26|92.78|150.30|69.13|0.00|1797.38|2412.28|3907.80|35.94|0.00|1797.38|1833.32|-614.90| +2450942|72596|3481|80268|334977|2839|42301|8|191|818|92|37.57|55.60|51.15|0.00|4705.80|3456.44|5115.20|329.40|0.00|4705.80|5035.20|1249.36| +2450942|72596|3367|80268|334977|2839|42301|8|233|818|41|91.59|177.68|39.08|0.00|1602.28|3755.19|7284.88|80.11|0.00|1602.28|1682.39|-2152.91| +2450942|72596|5696|80268|334977|2839|42301|8|25|818|50|77.75|84.74|9.32|0.00|466.00|3887.50|4237.00|41.94|0.00|466.00|507.94|-3421.50| +2450942|72596|6218|80268|334977|2839|42301|8|151|818|13|16.92|23.51|8.46|0.00|109.98|219.96|305.63|0.00|0.00|109.98|109.98|-109.98| +2452151|35335|6753|2887|560870|6406|35989|8|210|819|8|17.83|33.69|7.74|0.00|61.92|142.64|269.52|4.33|0.00|61.92|66.25|-80.72| +2452151|35335|5431|2887|560870|6406|35989|8|131|819|41|56.72|64.66|53.02|0.00|2173.82|2325.52|2651.06|195.64|0.00|2173.82|2369.46|-151.70| +2452151|35335|7871|2887|560870|6406|35989|8|55|819|68|86.09|99.86|88.87|1752.51|6043.16|5854.12|6790.48|300.34|1752.51|4290.65|4590.99|-1563.47| +2452151|35335|13579|2887|560870|6406|35989|8|229|819|10|96.57|123.60|64.27|0.00|642.70|965.70|1236.00|0.00|0.00|642.70|642.70|-323.00| +2452151|35335|3929|2887|560870|6406|35989|8|290|819|72|21.74|33.47|9.37|0.00|674.64|1565.28|2409.84|53.97|0.00|674.64|728.61|-890.64| +2452151|35335|1951|2887|560870|6406|35989|8|22|819|84|53.90|96.48|42.45|0.00|3565.80|4527.60|8104.32|178.29|0.00|3565.80|3744.09|-961.80| +2452151|35335|17349|2887|560870|6406|35989|8|197|819|31|82.40|145.02|95.71|0.00|2967.01|2554.40|4495.62|237.36|0.00|2967.01|3204.37|412.61| +2452151|35335|217|2887|560870|6406|35989|8|118|819|33|7.33|9.74|2.62|0.00|86.46|241.89|321.42|0.00|0.00|86.46|86.46|-155.43| +|35335|14849|2887||6406|||284|819|18||||156.34|781.74||1281.60|6.25|156.34|625.40|631.65|| +2452151|35335|17875|2887|560870|6406|35989|8|236|819|1|27.86|40.95|1.22|0.00|1.22|27.86|40.95|0.03|0.00|1.22|1.25|-26.64| +2452151|35335|3239|2887|560870|6406|35989|8|264|819|53|4.97|5.71|0.51|0.00|27.03|263.41|302.63|0.27|0.00|27.03|27.30|-236.38| +2452151|35335|43|2887|560870|6406|35989|8|93|819|95|65.66|124.75|9.98|0.00|948.10|6237.70|11851.25|85.32|0.00|948.10|1033.42|-5289.60| +2452151|35335|4817|2887|560870|6406|35989|8|56|819|43|83.79|86.30|79.39|1706.88|3413.77|3602.97|3710.90|85.34|1706.88|1706.89|1792.23|-1896.08| +2452151|35335|8211|2887|560870|6406|35989|8|8|819|57|22.73|30.91|6.49|0.00|369.93|1295.61|1761.87|11.09|0.00|369.93|381.02|-925.68| +2452151|35335|11983|2887|560870|6406|35989|8|18|819|1|96.00|111.36|80.17|0.00|80.17|96.00|111.36|1.60|0.00|80.17|81.77|-15.83| +2452151|35335|16299|2887|560870|6406|35989|8|143|819|27|19.20|24.19|6.77|0.00|182.79|518.40|653.13|3.65|0.00|182.79|186.44|-335.61| +2452168|45748|1227|89539|1920630|3721|22812|1|38|820|19|2.54|4.92|3.98|0.00|75.62|48.26|93.48|5.29|0.00|75.62|80.91|27.36| +2452168|45748|9761|89539|1920630|3721|22812|1|100|820|57|74.47|117.66|109.42|0.00|6236.94|4244.79|6706.62|561.32|0.00|6236.94|6798.26|1992.15| +|45748|14805|89539|1920630|||||820||91.68||50.52|||8159.52||||||| +2452168|45748|10493|89539|1920630|3721|22812|1|241|820|50|32.57|56.02|10.08|0.00|504.00|1628.50|2801.00|25.20|0.00|504.00|529.20|-1124.50| +||6897|89539||3721|22812|1|67|820||38.58|38.96|35.06||3260.58||3623.28|293.45||3260.58||| +2452168|45748|2585|89539|1920630|3721|22812|1|241|820|62|96.01|99.85|36.94|0.00|2290.28|5952.62|6190.70|206.12|0.00|2290.28|2496.40|-3662.34| +2452168|45748|1745|89539|1920630|3721|22812|1|223|820|29|15.09|18.71|1.12|0.00|32.48|437.61|542.59|0.64|0.00|32.48|33.12|-405.13| +|45748|3785|89539|1920630|3721||1|3|820|45||85.18||1046.38|||3833.10||1046.38|2441.57||413.42| +2452168|45748|9013|89539|1920630|3721|22812|1|242|820|10|19.42|36.31|18.88|0.00|188.80|194.20|363.10|15.10|0.00|188.80|203.90|-5.40| +2452168|45748|1993|89539|1920630|3721|22812|1|290|820|31|46.17|54.01|43.74|0.00|1355.94|1431.27|1674.31|13.55|0.00|1355.94|1369.49|-75.33| +2452168|45748|8769|89539|1920630|3721|22812|1|127|820|18|52.12|53.16|11.16|164.72|200.88|938.16|956.88|0.00|164.72|36.16|36.16|-902.00| +2452168|45748|2445|89539|1920630|3721|22812|1|253|820|26|36.37|55.64|9.45|149.87|245.70|945.62|1446.64|0.95|149.87|95.83|96.78|-849.79| +2452168|45748|12849|89539|1920630|3721|22812|1|243|820|24|85.95|126.34|101.07|0.00|2425.68|2062.80|3032.16|24.25|0.00|2425.68|2449.93|362.88| +2451883|39978|4220|92962|324432|4702|30525|1|244|821|99|48.91|96.35|43.35|0.00|4291.65|4842.09|9538.65|343.33|0.00|4291.65|4634.98|-550.44| +2451883|39978|17261|92962|324432|4702|30525|1|286|821|94|55.64|69.55|24.34|0.00|2287.96|5230.16|6537.70|22.87|0.00|2287.96|2310.83|-2942.20| +2451883||6427|||||1|37|821|||28.29|13.29|0.00||1046.52||28.70|0.00||746.36|-328.86| +2451883|39978|4453|92962|324432|4702|30525|1|114|821|59|92.85|107.70|43.08|0.00|2541.72|5478.15|6354.30|50.83|0.00|2541.72|2592.55|-2936.43| +2451883|39978|17353|92962|324432|4702|30525|1|10|821|53|43.12|50.45|19.67|0.00|1042.51|2285.36|2673.85|20.85|0.00|1042.51|1063.36|-1242.85| +2451883|39978|16454|92962|324432|4702|30525|1|195|821|32|43.00|67.08|16.09|0.00|514.88|1376.00|2146.56|46.33|0.00|514.88|561.21|-861.12| +2451883|39978|15121|92962|324432|4702|30525|1|21|821|27|38.03|75.67|10.59|0.00|285.93|1026.81|2043.09|8.57|0.00|285.93|294.50|-740.88| +2451883|39978|12979|92962|324432|4702|30525|1|181|821|93|39.58|73.22|69.55|0.00|6468.15|3680.94|6809.46|517.45|0.00|6468.15|6985.60|2787.21| +2451883|39978|9512|92962|324432|4702|30525|1|100|821|22|87.02|148.80|41.66|0.00|916.52|1914.44|3273.60|18.33|0.00|916.52|934.85|-997.92| +2451883|39978|1259|92962|324432|4702|30525|1|187|821|23|6.65|11.10|8.76|0.00|201.48|152.95|255.30|16.11|0.00|201.48|217.59|48.53| +2451883|39978|7399|92962|324432|4702|30525|1|224|821|77|50.32|74.47|23.83|0.00|1834.91|3874.64|5734.19|146.79|0.00|1834.91|1981.70|-2039.73| +2451883|39978|533|92962|324432|4702|30525|1|124|821|96|88.40|127.29|42.00|0.00|4032.00|8486.40|12219.84|161.28|0.00|4032.00|4193.28|-4454.40| +2451460|35378|3578|61367|1535249|5931|35460|2|18|822|47|11.15|16.05|15.08|49.61|708.76|524.05|754.35|0.00|49.61|659.15|659.15|135.10| +2451460|35378|17443|61367|1535249|5931|35460|2|5|822|24|19.11|32.86|9.20|0.00|220.80|458.64|788.64|19.87|0.00|220.80|240.67|-237.84| +|35378|5155||1535249||35460|2||822|61|86.86|110.31|||1749.48|||55.10||787.27|842.37|| +2451460|35378|11785|61367|1535249|5931|35460|2|53|822|87|40.65|64.63|62.69|0.00|5454.03|3536.55|5622.81|490.86|0.00|5454.03|5944.89|1917.48| +2451460|35378|1627|61367|1535249|5931|35460|2|248|822|32|95.01|145.36|30.52|283.22|976.64|3040.32|4651.52|20.80|283.22|693.42|714.22|-2346.90| +2451460|35378|16900|61367|1535249|5931|35460|2|17|822|48|62.85|111.87|90.61|0.00|4349.28|3016.80|5369.76|304.44|0.00|4349.28|4653.72|1332.48| +2451460|35378|10606|61367|1535249|5931|35460|2|287|822|84|84.18|117.85|32.99|0.00|2771.16|7071.12|9899.40|249.40|0.00|2771.16|3020.56|-4299.96| +2451460||398|61367|||35460|2||822|||145.57|||||9898.76||||4256.12|-2048.84| +2451460|35378|7543|61367|1535249|5931|35460|2|282|822|17|83.65|104.56|56.46|0.00|959.82|1422.05|1777.52|76.78|0.00|959.82|1036.60|-462.23| +2451460|35378|11308|61367|1535249|5931|35460|2|106|822|34|28.62|30.05|1.50|0.00|51.00|973.08|1021.70|2.55|0.00|51.00|53.55|-922.08| +2451460|35378|16537|61367|1535249|5931|35460|2|62|822|29|23.30|36.11|5.41|0.00|156.89|675.70|1047.19|9.41|0.00|156.89|166.30|-518.81| +2451460|35378|17468|61367|1535249|5931|35460|2|33|822|75|72.26|114.89|34.46|1628.23|2584.50|5419.50|8616.75|0.00|1628.23|956.27|956.27|-4463.23| +2451460||13315|61367|1535249|5931|||168|822|67|||76.75|||5955.63|9349.85|411.38||||-813.38| +2451460|35378|11929|61367|1535249|5931|35460|2|244|822|33|41.25|70.53|38.08|0.00|1256.64|1361.25|2327.49|113.09|0.00|1256.64|1369.73|-104.61| +2451911|46704|3415|62960|1160317|1674|24231|7|209|823|52|80.41|87.64|34.17|0.00|1776.84|4181.32|4557.28|35.53|0.00|1776.84|1812.37|-2404.48| +2451911|46704|6223|62960|1160317|1674|24231|7|10|823|63|57.37|82.61|70.21|4202.06|4423.23|3614.31|5204.43|4.42|4202.06|221.17|225.59|-3393.14| +2451911|46704|4419|62960|1160317|1674|24231|7|53|823|13|67.37|93.64|14.04|0.00|182.52|875.81|1217.32|12.77|0.00|182.52|195.29|-693.29| +2451911|46704|107|62960|1160317|1674|24231|7|224|823|67|34.37|38.49|37.33|0.00|2501.11|2302.79|2578.83|175.07|0.00|2501.11|2676.18|198.32| +2451911|46704|277|62960|1160317|1674|24231|7|250|823|31|91.68|116.43|50.06|0.00|1551.86|2842.08|3609.33|93.11|0.00|1551.86|1644.97|-1290.22| +2451911|46704|4343|62960|1160317|1674|24231|7|211|823|12|74.77|128.60|41.15|217.27|493.80|897.24|1543.20|13.82|217.27|276.53|290.35|-620.71| +2451911|46704|621|62960|1160317|1674|24231|7|154|823|74|93.77|112.52|73.13|0.00|5411.62|6938.98|8326.48|487.04|0.00|5411.62|5898.66|-1527.36| +2451911|46704|13821|62960|1160317|1674|24231|7|245|823|15|41.99|80.62|49.17|0.00|737.55|629.85|1209.30|66.37|0.00|737.55|803.92|107.70| +2451911|46704|10395|62960|1160317|1674|24231|7|9|823|82|78.29|156.58|72.02|0.00|5905.64|6419.78|12839.56|295.28|0.00|5905.64|6200.92|-514.14| +2451911|46704|10075|62960|1160317|1674|24231|7|228|823|13|36.55|63.96|0.63|0.00|8.19|475.15|831.48|0.08|0.00|8.19|8.27|-466.96| +2451911|46704|7083|62960|1160317|1674|24231|7|117|823|54|73.20|98.82|47.43|0.00|2561.22|3952.80|5336.28|25.61|0.00|2561.22|2586.83|-1391.58| +2451911|46704|9541|62960|1160317|1674|24231|7|92|823|53|20.68|21.92|7.01|0.00|371.53|1096.04|1161.76|22.29|0.00|371.53|393.82|-724.51| +2451666|31437|6331|3967|1744754|7141|44281|10|66|824|40|9.47|13.16|8.15|0.00|326.00|378.80|526.40|22.82|0.00|326.00|348.82|-52.80| +2451666|31437|8263|3967|1744754|7141|44281|10|212|824|27|21.19|35.59|5.69|116.75|153.63|572.13|960.93|1.84|116.75|36.88|38.72|-535.25| +2451666|31437|12107|3967|1744754|7141|44281|10|173|824|8|53.77|95.17|24.74|0.00|197.92|430.16|761.36|0.00|0.00|197.92|197.92|-232.24| +|31437|17012|3967|||44281||218|824|96|||13.17||1264.32|1896.00|2691.84|||||| +2451666|31437|9569|3967|1744754|7141|44281|10|205|824|42|25.54|43.67|23.14|252.68|971.88|1072.68|1834.14|21.57|252.68|719.20|740.77|-353.48| +2451666|31437|16124|3967|1744754|7141|44281|10|253|824|33|20.94|32.66|17.63|0.00|581.79|691.02|1077.78|0.00|0.00|581.79|581.79|-109.23| +2451666|31437|14437|3967|1744754|7141|44281|10|95|824|7|98.05|136.28|42.24|0.00|295.68|686.35|953.96|26.61|0.00|295.68|322.29|-390.67| +||4310|3967|||44281||232|824|68|||||10175.52|6330.80||203.51||10175.52|10379.03|| +2451666|31437|15811|3967|1744754|7141|44281|10|177|824|69|68.35|86.12|75.78|0.00|5228.82|4716.15|5942.28|313.72|0.00|5228.82|5542.54|512.67| +2451666|31437|9782|3967|1744754|7141|44281|10|110|824|17|13.27|15.52|4.96|0.00|84.32|225.59|263.84|0.00|0.00|84.32|84.32|-141.27| +2451666|31437|5125|3967|1744754|7141|44281|10|248|824|3|52.06|81.21|24.36|0.00|73.08|156.18|243.63|5.84|0.00|73.08|78.92|-83.10| +2451666|31437|1409|3967|1744754|7141|44281|10|52|824|46|54.44|108.88|14.15|0.00|650.90|2504.24|5008.48|26.03|0.00|650.90|676.93|-1853.34| +2451551|51364|11701|43389|1158289|6789|2842|7|57|825|27|13.07|17.64|16.93|187.41|457.11|352.89|476.28|24.27|187.41|269.70|293.97|-83.19| +2451551|51364|4646|43389|1158289|6789|2842|7|180|825|34|11.39|14.46|5.06|0.00|172.04|387.26|491.64|0.00|0.00|172.04|172.04|-215.22| +2451551|51364|15829|43389|1158289|6789|2842|7|66|825|77|54.75|101.28|23.29|0.00|1793.33|4215.75|7798.56|71.73|0.00|1793.33|1865.06|-2422.42| +2451551|51364|7039|43389|1158289|6789|2842|7|190|825|86|28.16|45.61|23.26|0.00|2000.36|2421.76|3922.46|120.02|0.00|2000.36|2120.38|-421.40| +2451551|51364|11975|43389|1158289|6789|2842|7|193|825|20|7.56|14.96|0.59|0.00|11.80|151.20|299.20|1.06|0.00|11.80|12.86|-139.40| +2451551|51364|13451|43389|1158289|6789|2842|7|252|825|70|83.69|130.55|39.16|0.00|2741.20|5858.30|9138.50|246.70|0.00|2741.20|2987.90|-3117.10| +2451551|51364|7622|43389|1158289|6789|2842|7|76|825|28|69.67|133.06|79.83|0.00|2235.24|1950.76|3725.68|22.35|0.00|2235.24|2257.59|284.48| +||15089|43389|1158289|6789|2842|7||825|75||6.94|1.73||||520.50|9.08||129.75||-283.50| +2451551|51364|13790|43389||6789|2842|7||825||58.86||54.57|1680.75||2354.40|||1680.75|||-1852.35| +2451551|51364|12383|43389|1158289|6789|2842|7|194|825|21|47.14|88.62|40.76|0.00|855.96|989.94|1861.02|59.91|0.00|855.96|915.87|-133.98| +2451551|51364|12611|43389|1158289|6789|2842|7|40|825|39|76.64|150.21|64.59|0.00|2519.01|2988.96|5858.19|226.71|0.00|2519.01|2745.72|-469.95| +2451551|51364|6509|43389|1158289|6789|2842|7|14|825|92|38.10|45.33|41.25|796.95|3795.00|3505.20|4170.36|179.88|796.95|2998.05|3177.93|-507.15| +2451551|51364|8861|43389|1158289|6789|2842|7|227|825|82|17.91|29.37|8.51|0.00|697.82|1468.62|2408.34|55.82|0.00|697.82|753.64|-770.80| +2451119|48406|3218|58760|308702|1491|21880|7|217|826|6|87.82|151.92|60.76|0.00|364.56|526.92|911.52|0.00|0.00|364.56|364.56|-162.36| +2451119|48406|11618|58760|308702|1491|21880|7|252|826|89|13.07|17.25|6.90|104.39|614.10|1163.23|1535.25|40.77|104.39|509.71|550.48|-653.52| +2451119|48406|5984|58760|308702|1491|21880|7|282|826|1|93.00|118.11|12.99|12.86|12.99|93.00|118.11|0.00|12.86|0.13|0.13|-92.87| +2451119|48406|13520|58760|308702|1491|21880|7|229|826|88|44.99|75.58|25.69|0.00|2260.72|3959.12|6651.04|67.82|0.00|2260.72|2328.54|-1698.40| +2451119||13544||308702||21880|7||826|||3.87||||||||123.59||| +2451119|48406|686|58760|308702|1491|21880|7|52|826|22|16.13|20.16|5.84|0.00|128.48|354.86|443.52|7.70|0.00|128.48|136.18|-226.38| +2451119|48406|7117|58760|308702|1491|21880|7|71|826|41|55.15|93.75|70.31|2565.61|2882.71|2261.15|3843.75|19.02|2565.61|317.10|336.12|-1944.05| +2451119|48406|7060|58760|308702|1491|21880|7|192|826|39|34.66|64.46|7.73|0.00|301.47|1351.74|2513.94|0.00|0.00|301.47|301.47|-1050.27| +2451119|48406|12871|58760|308702|1491|21880|7|16|826|82|80.52|119.97|45.58|0.00|3737.56|6602.64|9837.54|37.37|0.00|3737.56|3774.93|-2865.08| +2451119|48406|16940|58760|308702|1491|21880|7|98|826|47|21.14|39.74|15.89|0.00|746.83|993.58|1867.78|37.34|0.00|746.83|784.17|-246.75| +2451119|48406|9565|58760|308702|1491|21880|7|12|826|22|1.15|1.92|1.86|0.00|40.92|25.30|42.24|0.81|0.00|40.92|41.73|15.62| +2451119|48406|17023|58760|308702|1491|21880|7|189|826|67|41.84|59.83|16.75|0.00|1122.25|2803.28|4008.61|11.22|0.00|1122.25|1133.47|-1681.03| +2451322|54595|10016|22064|656254|5197|21787|4|142|827|34|58.73|80.46|36.20|0.00|1230.80|1996.82|2735.64|98.46|0.00|1230.80|1329.26|-766.02| +2451322|54595|3346|22064|656254|5197|21787|4|24|827|12|11.82|18.79|4.69|13.50|56.28|141.84|225.48|2.99|13.50|42.78|45.77|-99.06| +2451322|54595|976|22064|656254|5197|21787|4|113|827|53|74.50|93.12|32.59|552.72|1727.27|3948.50|4935.36|93.96|552.72|1174.55|1268.51|-2773.95| +2451322|54595|10471|22064|656254|5197|21787|4|254|827|77|92.89|150.48|19.56|0.00|1506.12|7152.53|11586.96|105.42|0.00|1506.12|1611.54|-5646.41| +2451322|54595|6058|22064|656254|5197|21787|4|118|827|26|58.04|108.53|94.42|0.00|2454.92|1509.04|2821.78|49.09|0.00|2454.92|2504.01|945.88| +2451322|54595|12550|22064|656254|5197|21787|4|215|827|99|81.06|103.75|36.31|0.00|3594.69|8024.94|10271.25|323.52|0.00|3594.69|3918.21|-4430.25| +2451322|54595|2392|22064|656254|5197|21787|4|53|827|52|30.17|41.03|40.20|209.04|2090.40|1568.84|2133.56|56.44|209.04|1881.36|1937.80|312.52| +2451322|54595|15002|22064|656254|5197|21787|4|81|827|36|13.68|19.42|13.98|0.00|503.28|492.48|699.12|45.29|0.00|503.28|548.57|10.80| +2451322|54595|6134|22064|656254|5197|21787|4|85|827|18|13.83|24.89|8.21|0.00|147.78|248.94|448.02|11.82|0.00|147.78|159.60|-101.16| +2451322|54595|3613|22064|656254|5197|21787|4|286|827|38|25.94|49.80|18.92|0.00|718.96|985.72|1892.40|57.51|0.00|718.96|776.47|-266.76| +2451322|54595|9740|22064|656254|5197|21787|4|223|827|46|27.36|42.68|34.14|0.00|1570.44|1258.56|1963.28|78.52|0.00|1570.44|1648.96|311.88| +2451322|54595|619|22064|656254|5197|21787|4|145|827|58|79.83|123.73|33.40|38.74|1937.20|4630.14|7176.34|18.98|38.74|1898.46|1917.44|-2731.68| +2451322||1436|||5197||4||827|78|58.83||44.01|2814.87||4588.74|||2814.87|617.91||-3970.83| +2451322|54595|8462|22064|656254|5197|21787|4|91|827|36|41.61|67.82|24.41|0.00|878.76|1497.96|2441.52|70.30|0.00|878.76|949.06|-619.20| +2451425|46928|17224|32366|990370|4821|10472|10|27|828|75|48.21|56.40|24.81|0.00|1860.75|3615.75|4230.00|0.00|0.00|1860.75|1860.75|-1755.00| +2451425|46928|11990|32366|990370|4821|10472|10|236|828|96|58.01|98.03|86.26|0.00|8280.96|5568.96|9410.88|248.42|0.00|8280.96|8529.38|2712.00| +2451425|46928|9901|32366|990370|4821|10472|10|4|828|7|76.44|139.88|65.74|354.33|460.18|535.08|979.16|2.11|354.33|105.85|107.96|-429.23| +2451425|46928|7606|32366|990370|4821|10472|10|238|828|92|31.29|38.48|12.31|0.00|1132.52|2878.68|3540.16|79.27|0.00|1132.52|1211.79|-1746.16| +2451425|46928|15649|32366|990370|4821|10472|10|38|828|62|36.18|45.94|19.29|1004.62|1195.98|2243.16|2848.28|1.91|1004.62|191.36|193.27|-2051.80| +2451425|46928|11488|32366|990370|4821|10472|10|164|828|9|29.37|57.27|8.01|0.00|72.09|264.33|515.43|2.16|0.00|72.09|74.25|-192.24| +2451425|46928|2420|32366|990370|4821|10472|10|83|828|67|8.93|17.41|13.92|0.00|932.64|598.31|1166.47|65.28|0.00|932.64|997.92|334.33| +2451425|46928|16646|32366|990370|4821|10472|10|148|828|33|99.31|159.88|135.89|0.00|4484.37|3277.23|5276.04|134.53|0.00|4484.37|4618.90|1207.14| +2451425|46928|17456|32366|990370|4821|10472|10|280|828|5|58.84|108.85|51.15|0.00|255.75|294.20|544.25|17.90|0.00|255.75|273.65|-38.45| +2451425|46928|6098|32366|990370|4821|10472|10|222|828|25|52.86|67.66|54.12|0.00|1353.00|1321.50|1691.50|0.00|0.00|1353.00|1353.00|31.50| +2451996|39930|17437|19149|763760|5171|30182|1|71|829|75|97.44|182.21|96.57|0.00|7242.75|7308.00|13665.75|434.56|0.00|7242.75|7677.31|-65.25| +2451996|39930|3319|19149|763760|5171|30182|1|109|829|47|76.28|110.60|3.31|0.00|155.57|3585.16|5198.20|12.44|0.00|155.57|168.01|-3429.59| +2451996|39930|3045|19149|763760|5171|30182|1|239|829|87|53.04|101.30|10.13|44.06|881.31|4614.48|8813.10|75.35|44.06|837.25|912.60|-3777.23| +2451996|39930|14543|19149|763760|5171|30182|1|112|829|99|3.52|4.71|1.74|0.00|172.26|348.48|466.29|1.72|0.00|172.26|173.98|-176.22| +2451996|39930|4391|19149|763760|5171|30182|1|292|829|59|48.59|95.72|44.03|1428.77|2597.77|2866.81|5647.48|93.52|1428.77|1169.00|1262.52|-1697.81| +2451996|39930|14877|19149|763760|5171|30182|1|124|829|55|48.16|56.34|11.83|0.00|650.65|2648.80|3098.70|26.02|0.00|650.65|676.67|-1998.15| +2451996|39930|5019|19149|763760|5171|30182|1|254|829|35|92.77|139.15|34.78|0.00|1217.30|3246.95|4870.25|73.03|0.00|1217.30|1290.33|-2029.65| +2451996|39930|2433|19149|763760|5171|30182|1|177|829|36|7.29|10.35|0.82|11.21|29.52|262.44|372.60|1.46|11.21|18.31|19.77|-244.13| +2451996|39930|7285|19149|763760|5171|30182|1|151|829|74|13.21|21.53|12.48|0.00|923.52|977.54|1593.22|27.70|0.00|923.52|951.22|-54.02| +2451996|39930|197|19149|763760|5171|30182|1|267|829|62|80.28|124.43|57.23|0.00|3548.26|4977.36|7714.66|141.93|0.00|3548.26|3690.19|-1429.10| +2451505|34186|8503|42944|440476|4855|28387|10|20|830|31|1.76|2.48|2.40|0.00|74.40|54.56|76.88|4.46|0.00|74.40|78.86|19.84| +2451505|34186|2227|42944|440476|4855|28387|10|199|830|99|46.24|73.98|63.62|0.00|6298.38|4577.76|7324.02|62.98|0.00|6298.38|6361.36|1720.62| +2451505|34186|1351|42944|440476|4855|28387|10|20|830|17|98.25|183.72|91.86|93.69|1561.62|1670.25|3123.24|58.71|93.69|1467.93|1526.64|-202.32| +2451505|34186|4255|42944|440476|4855|28387|10|82|830|47|42.75|74.38|46.11|0.00|2167.17|2009.25|3495.86|0.00|0.00|2167.17|2167.17|157.92| +2451505|34186|2726|42944|440476|4855|28387|10|247|830|60|35.91|37.34|25.76|0.00|1545.60|2154.60|2240.40|15.45|0.00|1545.60|1561.05|-609.00| +2451505|34186|12806|42944|440476|4855|28387|10|294|830|35|76.88|129.92|41.57|0.00|1454.95|2690.80|4547.20|116.39|0.00|1454.95|1571.34|-1235.85| +2451505|34186|17690|42944|440476|4855|28387|10|292|830|27|21.54|40.49|37.65|0.00|1016.55|581.58|1093.23|0.00|0.00|1016.55|1016.55|434.97| +2451505|34186|2612|42944|440476|4855|28387|10|250|830|15|4.28|4.66|3.12|0.00|46.80|64.20|69.90|2.34|0.00|46.80|49.14|-17.40| +2451505|34186|11167|42944|440476|4855|28387|10|171|830|24|85.34|151.90|136.71|0.00|3281.04|2048.16|3645.60|262.48|0.00|3281.04|3543.52|1232.88| +2452619|36992|6841|13636|1395734|3785|29483|8|81|831|56|51.33|58.51|5.85|0.00|327.60|2874.48|3276.56|13.10|0.00|327.60|340.70|-2546.88| +2452619|36992|10501|13636|1395734|3785|29483|8|220|831|36|40.18|45.80|25.19|0.00|906.84|1446.48|1648.80|45.34|0.00|906.84|952.18|-539.64| +2452619|36992|8971|13636|1395734|3785|29483|8|24|831|65|20.64|38.80|25.22|0.00|1639.30|1341.60|2522.00|131.14|0.00|1639.30|1770.44|297.70| +2452619|36992|6651|13636|1395734|3785|29483|8|112|831|55|27.18|47.83|33.48|0.00|1841.40|1494.90|2630.65|128.89|0.00|1841.40|1970.29|346.50| +2452619|36992|7633|13636|1395734|3785|29483|8|171|831|44|25.90|41.44|12.01|0.00|528.44|1139.60|1823.36|31.70|0.00|528.44|560.14|-611.16| +2452619|36992|8617|13636|1395734|3785|29483|8|75|831|31|97.14|141.82|73.74|0.00|2285.94|3011.34|4396.42|22.85|0.00|2285.94|2308.79|-725.40| +2452619|36992|11347|13636|1395734|3785|29483|8|265|831|12|40.20|67.13|18.79|0.00|225.48|482.40|805.56|9.01|0.00|225.48|234.49|-256.92| +2452619|36992|9036|13636|1395734|3785|29483|8|211|831|73|40.00|50.40|3.02|0.00|220.46|2920.00|3679.20|4.40|0.00|220.46|224.86|-2699.54| +2452619|36992|15189|13636|1395734|3785|29483|8|271|831|17|84.34|151.81|22.77|0.00|387.09|1433.78|2580.77|3.87|0.00|387.09|390.96|-1046.69| +2452619|36992|4783|13636|1395734|3785|29483|8|144|831|2|74.02|86.60|14.72|0.00|29.44|148.04|173.20|0.88|0.00|29.44|30.32|-118.60| +|36992|1275|13636||||||831||7.33|10.84|||238.63|359.17|||||238.63|-120.54| +2452624|46644|607|60182|1797718|6454|12790|10|235|832|68|9.13|10.49|8.91|0.00|605.88|620.84|713.32|0.00|0.00|605.88|605.88|-14.96| +2452624|46644|4626|60182|1797718|6454|12790|10|163|832|29|95.95|160.23|155.42|0.00|4507.18|2782.55|4646.67|45.07|0.00|4507.18|4552.25|1724.63| +2452624|46644|10218|60182|1797718|6454|12790|10|48|832|61|58.77|62.88|16.34|0.00|996.74|3584.97|3835.68|9.96|0.00|996.74|1006.70|-2588.23| +2452624|46644|2949|60182|1797718|6454|12790|10|147|832|50|89.64|143.42|119.03|4523.14|5951.50|4482.00|7171.00|28.56|4523.14|1428.36|1456.92|-3053.64| +2452624||6939|60182||6454||10|258|832||72.44|||0.00||1521.24||161.64|0.00|||| +2452624|46644|9906|60182|1797718|6454|12790|10|203|832|74|31.40|33.28|7.65|0.00|566.10|2323.60|2462.72|16.98|0.00|566.10|583.08|-1757.50| +2452624|46644|6954|60182|1797718|6454|12790|10|41|832|45|84.50|152.94|61.17|0.00|2752.65|3802.50|6882.30|55.05|0.00|2752.65|2807.70|-1049.85| +2452624|46644|11040|60182|1797718|6454|12790|10|165|832|64|59.33|72.97|16.05|0.00|1027.20|3797.12|4670.08|0.00|0.00|1027.20|1027.20|-2769.92| +2452624|46644|16797|60182|1797718|6454|12790|10|267|832|23|66.27|69.58|68.18|0.00|1568.14|1524.21|1600.34|141.13|0.00|1568.14|1709.27|43.93| +2451057|43833|7390|72201|473996|2638|4480|8|179|833|62|74.98|86.22|36.21|0.00|2245.02|4648.76|5345.64|157.15|0.00|2245.02|2402.17|-2403.74| +2451057|43833|8336|72201|473996|2638|4480|8|35|833|80|34.85|42.16|13.49|0.00|1079.20|2788.00|3372.80|97.12|0.00|1079.20|1176.32|-1708.80| +2451057|43833|17564|72201|473996|2638|4480|8|38|833|23|45.83|49.95|25.97|0.00|597.31|1054.09|1148.85|47.78|0.00|597.31|645.09|-456.78| +2451057|43833|13172|72201|473996|2638|4480|8|244|833|30|70.01|112.71|81.15|0.00|2434.50|2100.30|3381.30|219.10|0.00|2434.50|2653.60|334.20| +2451057|43833|15301|72201|473996|2638|4480|8|210|833|43|60.24|110.84|62.07|0.00|2669.01|2590.32|4766.12|53.38|0.00|2669.01|2722.39|78.69| +2451057|43833|5996|72201|473996|2638|4480|8|117|833|24|21.53|23.89|10.75|0.00|258.00|516.72|573.36|5.16|0.00|258.00|263.16|-258.72| +2451057|43833|4742|72201|473996|2638|4480|8|26|833|82|24.51|38.23|4.58|0.00|375.56|2009.82|3134.86|30.04|0.00|375.56|405.60|-1634.26| +2451057|43833|15142|72201|473996|2638|4480|8|116|833|80|81.02|144.21|87.96|0.00|7036.80|6481.60|11536.80|281.47|0.00|7036.80|7318.27|555.20| +|43833|8626|72201|473996||4480|8||833||68.03||5.81|0.00|52.29|612.27|1046.97||0.00|52.29||| +2451057|43833|5119|72201|473996|2638|4480|8|128|833|69|40.66|41.47|2.90|198.09|200.10|2805.54|2861.43|0.08|198.09|2.01|2.09|-2803.53| +2451057|43833|13706|72201|473996|2638|4480|8|65|833|26|55.05|95.78|0.95|0.00|24.70|1431.30|2490.28|0.00|0.00|24.70|24.70|-1406.60| +2451057|43833|12692|72201|473996|2638|4480|8|279|833|39|5.73|9.16|7.87|52.17|306.93|223.47|357.24|10.19|52.17|254.76|264.95|31.29| +2451057||10840|72201||2638||8||833|||59.21||||||||3267.84|3365.87|1150.92| +2451057|43833|15229|72201|473996|2638|4480|8|226|833|3|32.12|49.46|0.00|0.00|0.00|96.36|148.38|0.00|0.00|0.00|0.00|-96.36| +2452260|53488|599|41839|1482533|827|2321|8|148|834|84|14.94|17.03|16.34|0.00|1372.56|1254.96|1430.52|68.62|0.00|1372.56|1441.18|117.60| +2452260|53488|7781|41839|1482533|827|2321|8|59|834|46|87.04|162.76|122.07|0.00|5615.22|4003.84|7486.96|224.60|0.00|5615.22|5839.82|1611.38| +2452260|53488|7509|||827||8||834||49.21|||0.00||3838.38|||0.00||2141.75|-1738.62| +2452260||1941|41839|||2321|||834|93||||0.00|||||0.00|2861.61||-3586.08| +2452260|53488|11325|41839|1482533|827|2321|8|248|834|92|64.70|75.05|64.54|0.00|5937.68|5952.40|6904.60|534.39|0.00|5937.68|6472.07|-14.72| +2452260|53488|14599|41839|1482533|827|2321|8|143|834|26|60.19|75.23|11.28|0.00|293.28|1564.94|1955.98|26.39|0.00|293.28|319.67|-1271.66| +2452260|53488|11591|41839|1482533|827|2321|8|172|834|95|42.17|54.39|22.84|1800.93|2169.80|4006.15|5167.05|22.13|1800.93|368.87|391.00|-3637.28| +2452260|53488|16793|41839|1482533|827|2321|8|162|834|4|45.33|50.31|25.65|0.00|102.60|181.32|201.24|5.13|0.00|102.60|107.73|-78.72| +2452260|53488|6467|41839|1482533|827|2321|8|215|834|90|6.80|10.74|5.90|0.00|531.00|612.00|966.60|10.62|0.00|531.00|541.62|-81.00| +2451886|51821|1483|49537|729675|6966|26257|10|284|835|6|24.52|39.72|30.18|0.00|181.08|147.12|238.32|3.62|0.00|181.08|184.70|33.96| +2451886|51821|2369|49537|729675|6966|26257|10|155|835|80|5.64|9.02|3.78|217.72|302.40|451.20|721.60|7.62|217.72|84.68|92.30|-366.52| +2451886|51821|3593|49537|729675|6966|26257|10|184|835|67|3.88|5.54|2.54|0.00|170.18|259.96|371.18|11.91|0.00|170.18|182.09|-89.78| +2451886|51821|14201|49537|729675|6966|26257|||835||60.40|93.62||0.00|1314.30||||0.00|1314.30||-256.10| +2451886|51821|10657|49537|729675|6966|26257|10|178|835|79|16.24|21.43|14.35|0.00|1133.65|1282.96|1692.97|90.69|0.00|1133.65|1224.34|-149.31| +2451886|51821|5522|49537|729675|6966|26257|10|266|835|71|48.28|84.97|81.57|0.00|5791.47|3427.88|6032.87|173.74|0.00|5791.47|5965.21|2363.59| +2451886|51821|5711|49537|729675|6966|26257|10|28|835|100|63.03|117.23|35.16|0.00|3516.00|6303.00|11723.00|210.96|0.00|3516.00|3726.96|-2787.00| +2451886|51821|5474|49537|729675|6966|26257|10|76|835|99|86.94|97.37|94.44|467.47|9349.56|8607.06|9639.63|88.82|467.47|8882.09|8970.91|275.03| +2452522|45594|11127|28915|574047|1579|30311|1|108|836|14|85.47|119.65|104.09|0.00|1457.26|1196.58|1675.10|87.43|0.00|1457.26|1544.69|260.68| +2452522|45594|16183|28915|574047|1579|30311|1|10|836|2|91.92|131.44|88.06|0.00|176.12|183.84|262.88|8.80|0.00|176.12|184.92|-7.72| +2452522|45594|7860|28915|574047|1579|30311|1|258|836|91|11.81|17.24|17.06|0.00|1552.46|1074.71|1568.84|15.52|0.00|1552.46|1567.98|477.75| +2452522|45594|11113|28915|574047|1579|30311|1|112|836|28|16.43|28.58|19.14|0.00|535.92|460.04|800.24|48.23|0.00|535.92|584.15|75.88| +2452522|45594|14121|28915|574047|1579|30311|1|268|836|24|90.84|129.90|9.09|0.00|218.16|2180.16|3117.60|0.00|0.00|218.16|218.16|-1962.00| +2452522|45594|7230|28915|574047|1579|30311|1|56|836|51|46.22|85.96|49.85|0.00|2542.35|2357.22|4383.96|228.81|0.00|2542.35|2771.16|185.13| +2452522|45594|2808|28915|574047|1579|30311|1|284|836|55|69.76|73.24|32.95|90.61|1812.25|3836.80|4028.20|0.00|90.61|1721.64|1721.64|-2115.16| +2452522|45594|17334|28915|574047|1579|30311|1|250|836|18|63.09|82.64|19.00|225.72|342.00|1135.62|1487.52|0.00|225.72|116.28|116.28|-1019.34| +2452522|45594|3807|28915|574047|1579|30311|1|265|836|43|99.81|132.74|80.97|0.00|3481.71|4291.83|5707.82|278.53|0.00|3481.71|3760.24|-810.12| +2452522|45594|10011|28915|574047|1579|30311|1|156|836|21|27.15|53.21|30.32|0.00|636.72|570.15|1117.41|44.57|0.00|636.72|681.29|66.57| +2452522|45594|2761|28915|574047|1579|30311|1|110|836|34|16.25|17.22|15.15|0.00|515.10|552.50|585.48|36.05|0.00|515.10|551.15|-37.40| +2452522|45594|843|28915|574047|1579|30311|1|6|836|92|96.08|184.47|178.93|0.00|16461.56|8839.36|16971.24|1481.54|0.00|16461.56|17943.10|7622.20| +2452522|45594|17467|28915|574047|1579|30311|1|253|836|31|80.26|100.32|74.23|0.00|2301.13|2488.06|3109.92|92.04|0.00|2301.13|2393.17|-186.93| +2451035|62609|3799|72597|1532892|5540|7035|4|191|837|41|3.35|3.98|0.75|0.00|30.75|137.35|163.18|0.61|0.00|30.75|31.36|-106.60| +2451035|62609|2569|72597|1532892|5540|7035|4|161|837|8|18.52|24.07|22.86|18.28|182.88|148.16|192.56|8.23|18.28|164.60|172.83|16.44| +2451035|62609|4612|72597|1532892|5540|7035|4|17|837|49|44.46|45.79|13.73|0.00|672.77|2178.54|2243.71|60.54|0.00|672.77|733.31|-1505.77| +2451035|62609|6176|72597|1532892|5540|7035|4|158|837|46|89.74|89.74|82.56|0.00|3797.76|4128.04|4128.04|303.82|0.00|3797.76|4101.58|-330.28| +2451035|62609|10852|72597|1532892|5540|7035|4|233|837|49|88.24|165.89|46.44|0.00|2275.56|4323.76|8128.61|182.04|0.00|2275.56|2457.60|-2048.20| +2451035|62609|6212|72597|1532892|5540|7035|4|290|837|50|55.14|58.44|16.94|0.00|847.00|2757.00|2922.00|59.29|0.00|847.00|906.29|-1910.00| +2451035|62609|7970|72597|1532892|5540|7035|4|132|837|32|10.39|17.45|10.12|0.00|323.84|332.48|558.40|9.71|0.00|323.84|333.55|-8.64| +2451035|62609|9181|72597|1532892|5540|7035|4|97|837|62|25.27|40.68|38.23|2322.85|2370.26|1566.74|2522.16|4.26|2322.85|47.41|51.67|-1519.33| +2451035|62609|6757|72597|1532892|5540|7035|4|246|837|61|54.51|67.04|48.93|0.00|2984.73|3325.11|4089.44|0.00|0.00|2984.73|2984.73|-340.38| +2451398|49623|5566|70151|84655|5912|44816|1|274|838|25|76.45|84.85|78.06|0.00|1951.50|1911.25|2121.25|78.06|0.00|1951.50|2029.56|40.25| +2451398|49623|9322|70151|84655|5912|44816|1|85|838|63|40.99|74.19|25.22|0.00|1588.86|2582.37|4673.97|63.55|0.00|1588.86|1652.41|-993.51| +2451398|49623|16514|70151|84655|5912|44816|1|68|838|78|51.01|90.79|36.31|0.00|2832.18|3978.78|7081.62|113.28|0.00|2832.18|2945.46|-1146.60| +2451398|49623|15328|70151|84655|5912|44816|1|3|838|87|88.68|175.58|24.58|0.00|2138.46|7715.16|15275.46|0.00|0.00|2138.46|2138.46|-5576.70| +2451398|49623|7603|70151|84655|5912|44816|1|253|838|95|65.58|109.51|87.60|0.00|8322.00|6230.10|10403.45|748.98|0.00|8322.00|9070.98|2091.90| +2451398|49623|6310|70151|84655|5912|44816|1|265|838|49|40.07|55.29|43.67|0.00|2139.83|1963.43|2709.21|192.58|0.00|2139.83|2332.41|176.40| +2451398|49623|14788|70151|84655|5912|44816|1|102|838|23||16.67|||7.59|241.27||0.30||7.59||| +2451398|49623|64|70151|84655|5912|44816|1|196|838|58|16.88|27.17|11.41|0.00|661.78|979.04|1575.86|26.47|0.00|661.78|688.25|-317.26| +2451398|49623|568|70151|84655|5912|44816|1|222|838|57|72.71|74.16|74.16|0.00|4227.12|4144.47|4227.12|338.16|0.00|4227.12|4565.28|82.65| +2451398|49623|12998|70151|84655|5912|44816|1|237|838|19|53.33|71.99|33.11|0.00|629.09|1013.27|1367.81|37.74|0.00|629.09|666.83|-384.18| +2451398|49623|11050|70151|84655|5912|44816|1|64|838|61|27.04|37.58|11.64|0.00|710.04|1649.44|2292.38|56.80|0.00|710.04|766.84|-939.40| +2451398|49623|2308|70151|84655|5912|44816|1|283|838|68|59.73|60.32|19.30|0.00|1312.40|4061.64|4101.76|26.24|0.00|1312.40|1338.64|-2749.24| +2451398|49623|7142|70151|84655|5912|44816|1|87|838|39|73.04|142.42|106.81|458.21|4165.59|2848.56|5554.38|333.66|458.21|3707.38|4041.04|858.82| +2451395|54133|4156|23555|923577|399|13574|2|285|839|34|39.37|43.70|10.05|0.00|341.70|1338.58|1485.80|23.91|0.00|341.70|365.61|-996.88| +2451395|54133|13174|23555|923577|399|13574|2|154|839|95|57.76|105.70|61.30|0.00|5823.50|5487.20|10041.50|349.41|0.00|5823.50|6172.91|336.30| +2451395|54133|145|23555|923577|399|13574|2|96|839|64|55.20|80.59|0.00|0.00|0.00|3532.80|5157.76|0.00|0.00|0.00|0.00|-3532.80| +2451395|54133|7243|23555|923577|399|13574|2|130|839|1|72.59|129.21|5.16|0.00|5.16|72.59|129.21|0.46|0.00|5.16|5.62|-67.43| +2451395|54133|15748|23555|923577|399|13574|2|92|839|13|44.40|64.38|14.80|75.03|192.40|577.20|836.94|1.17|75.03|117.37|118.54|-459.83| +2451395|54133|14761|23555|923577||13574|2||839||13.35||||23.50|133.50|261.60|1.64||23.50||-110.00| +2451395|54133|9332|23555|923577|399|13574|2|46|839|33|76.41|110.03|63.81|0.00|2105.73|2521.53|3630.99|63.17|0.00|2105.73|2168.90|-415.80| +2451395|54133|7192|23555|923577|399|13574|2|102|839|50|72.03|97.96|51.91|0.00|2595.50|3601.50|4898.00|103.82|0.00|2595.50|2699.32|-1006.00| +2451395|54133|7850|23555|923577|399|13574|2|209|839|83|89.28|176.77|5.30|351.92|439.90|7410.24|14671.91|1.75|351.92|87.98|89.73|-7322.26| +2451395|54133|17641|23555|923577|399|13574|2|274|839|68|52.31|97.81|36.18|0.00|2460.24|3557.08|6651.08|172.21|0.00|2460.24|2632.45|-1096.84| +2451395|54133|4747|23555|923577|399|13574|2|271|839|64|7.39|13.59|0.54|0.00|34.56|472.96|869.76|2.41|0.00|34.56|36.97|-438.40| +2451395|54133|3064|23555|923577|399|13574|2|85|839|22|89.74|171.40|167.97|0.00|3695.34|1974.28|3770.80|36.95|0.00|3695.34|3732.29|1721.06| +2450837|32658|17156|98692|165883|274|23065|4|205|840|36|16.67|29.00|8.99|0.00|323.64|600.12|1044.00|29.12|0.00|323.64|352.76|-276.48| +2450837||8960|98692||||4||840|58|||5.75||||1451.74|||333.50|343.50|-929.16| +2450837|32658|7435|98692|165883|274|23065|4|253|840|65|53.17|99.42|10.93|0.00|710.45|3456.05|6462.30|0.00|0.00|710.45|710.45|-2745.60| +2450837|32658|3847|98692|165883|274|23065|4|93|840|56|42.93|73.41|19.08|0.00|1068.48|2404.08|4110.96|64.10|0.00|1068.48|1132.58|-1335.60| +2450837|32658|5737|98692|165883|274|23065|4|231|840|78|14.63|15.94|10.99|0.00|857.22|1141.14|1243.32|17.14|0.00|857.22|874.36|-283.92| +2450837|32658|12865|98692|165883|274|23065|4|128|840|31|16.97|26.47|21.70|0.00|672.70|526.07|820.57|40.36|0.00|672.70|713.06|146.63| +2450837|32658|3271|98692|165883|274|23065|4|123|840|37|93.16|111.79|5.58|43.35|206.46|3446.92|4136.23|9.78|43.35|163.11|172.89|-3283.81| +2450837|32658|3541|98692|165883|274|23065|4|232|840|40|64.97|81.86|9.00|313.20|360.00|2598.80|3274.40|2.80|313.20|46.80|49.60|-2552.00| +2450837|32658|17038|98692|165883|274|23065|4|267|840|73|66.32|116.72|67.69|0.00|4941.37|4841.36|8520.56|345.89|0.00|4941.37|5287.26|100.01| +2450837|32658|11500|98692|165883|274|23065|4|90|840|50|40.30|44.33|15.07|0.00|753.50|2015.00|2216.50|22.60|0.00|753.50|776.10|-1261.50| +2450837|32658|6826|98692|165883|274|23065|4|209|840|29|15.75|21.89|6.34|0.00|183.86|456.75|634.81|14.70|0.00|183.86|198.56|-272.89| +2450916|31236|3982|81287|690806|325|20244|1|20|841|14|6.09|10.59|1.90|0.00|26.60|85.26|148.26|0.53|0.00|26.60|27.13|-58.66| +2450916|31236|12817|81287|690806|325|20244|1|270|841|30|26.67|28.00|1.96|0.00|58.80|800.10|840.00|1.76|0.00|58.80|60.56|-741.30| +2450916|31236|15040|81287|690806|325|20244|1|52|841|68|5.94|6.83|2.25|0.00|153.00|403.92|464.44|9.18|0.00|153.00|162.18|-250.92| +2450916|31236|17140|81287|690806|325|20244|1|147|841|23|70.04|70.04|64.43|0.00|1481.89|1610.92|1610.92|44.45|0.00|1481.89|1526.34|-129.03| +2450916||1915||||||225|841|58|59.55|79.20||1398.75|||4593.60||1398.75|||| +2450916||16705|||325|20244|1|36|841|60|50.94||34.76|604.82||3056.40|5348.40||604.82|||-1575.62| +2450916|31236|5948|81287|690806|325|20244|1|11|841|100|63.29|68.98|26.90|0.00|2690.00|6329.00|6898.00|53.80|0.00|2690.00|2743.80|-3639.00| +2450916|31236|6781|81287|690806|325|20244|1|201|841|86|8.11|9.97|4.08|0.00|350.88|697.46|857.42|3.50|0.00|350.88|354.38|-346.58| +2450916|31236|3758|81287|690806|325|20244|1|258|841|60|56.87|80.75|31.49|0.00|1889.40|3412.20|4845.00|94.47|0.00|1889.40|1983.87|-1522.80| +2450916|31236|10610|81287|690806|325|20244|1|241|841|95|72.29|120.72|33.80|96.33|3211.00|6867.55|11468.40|280.32|96.33|3114.67|3394.99|-3752.88| +2452137|49693|10915|22015|1158984|6353|34922|8|196|842|5|8.88|12.60|1.38|0.00|6.90|44.40|63.00|0.13|0.00|6.90|7.03|-37.50| +2452137|49693|5567|22015|1158984|6353|34922|8|46|842|63|3.68|6.47|2.39|69.26|150.57|231.84|407.61|4.87|69.26|81.31|86.18|-150.53| +2452137|49693|9323|22015|1158984|6353|34922|8|228|842|60|9.84|14.26|0.42|0.00|25.20|590.40|855.60|1.51|0.00|25.20|26.71|-565.20| +2452137|49693|16515|22015|1158984|6353|34922|8|101|842|32|68.85|111.53|13.38|0.00|428.16|2203.20|3568.96|38.53|0.00|428.16|466.69|-1775.04| +||15329|22015||6353||8|148|842|47|68.69|||0.00|32.90|3228.43|3292.82|0.98|0.00||33.88|| +2452137|49693|7603|22015|1158984|6353|34922|8|237|842|3|91.73|151.35|92.32|0.00|276.96|275.19|454.05|2.76|0.00|276.96|279.72|1.77| +2452137|49693|6311|22015|1158984|6353|34922|8|106|842|7|18.50|21.83|6.33|0.00|44.31|129.50|152.81|2.21|0.00|44.31|46.52|-85.19| +2452137|49693|14789|22015|1158984|6353|34922|8|272|842|6|39.14|55.57|53.90|48.51|323.40|234.84|333.42|24.74|48.51|274.89|299.63|40.05| +|49693|65|22015|1158984|6353||8|78|842|41|43.25|||||1773.25||57.74|||1982.69|151.70| +2452137|49693|569|22015|1158984|6353|34922|8|37|842|76|22.82|42.21|14.77|0.00|1122.52|1734.32|3207.96|0.00|0.00|1122.52|1122.52|-611.80| +2452137|49693|12999|22015|1158984|6353|34922|8|135|842|80|38.84|42.72|18.36|0.00|1468.80|3107.20|3417.60|58.75|0.00|1468.80|1527.55|-1638.40| +2452137|49693|11051|22015|1158984|6353|34922|8|220|842|96|92.88|136.53|120.14|0.00|11533.44|8916.48|13106.88|807.34|0.00|11533.44|12340.78|2616.96| +2450834|51119|9752|23086|1669081|1476|48318|4|220|843|34|16.74|21.92|10.30|0.00|350.20|569.16|745.28|0.00|0.00|350.20|350.20|-218.96| +2450834|51119|4220|23086|1669081|1476|48318|4|221|843|60|94.23|168.67|3.37|0.00|202.20|5653.80|10120.20|16.17|0.00|202.20|218.37|-5451.60| +2450834|51119|17260|23086|1669081|1476|48318|4|185|843|41|48.40|54.20|47.69|0.00|1955.29|1984.40|2222.20|136.87|0.00|1955.29|2092.16|-29.11| +2450834|51119|6427|23086|1669081|1476|48318|4|141|843|68|58.80|99.37|15.89|0.00|1080.52|3998.40|6757.16|21.61|0.00|1080.52|1102.13|-2917.88| +2450834|51119|4453|23086|1669081|1476|48318|4|242|843|91|52.14|62.56|55.67|0.00|5065.97|4744.74|5692.96|253.29|0.00|5065.97|5319.26|321.23| +2450834|51119|17353|23086|1669081|1476|48318|4|211|843|78|26.93|41.47|5.39|0.00|420.42|2100.54|3234.66|21.02|0.00|420.42|441.44|-1680.12| +2450834|51119|16454|23086|1669081|1476|48318|4|113|843|25|87.66|112.20|2.24|0.00|56.00|2191.50|2805.00|2.24|0.00|56.00|58.24|-2135.50| +2450834|51119|15121|23086|1669081|1476|48318|4|17|843|24|63.21|71.42|2.85|0.00|68.40|1517.04|1714.08|0.68|0.00|68.40|69.08|-1448.64| +2450834|51119|12979|23086|1669081|1476|48318|4|234|843|89|39.91|49.08|30.92|385.26|2751.88|3551.99|4368.12|141.99|385.26|2366.62|2508.61|-1185.37| +2451664|69394|2923|15382|1717711|5193|19475|8|48|844|20|78.38|126.19|44.16|0.00|883.20|1567.60|2523.80|17.66|0.00|883.20|900.86|-684.40| +2451664|69394|3344|15382|1717711|5193|19475|8|296|844|91|3.05|4.05|0.48|5.24|43.68|277.55|368.55|1.15|5.24|38.44|39.59|-239.11| +2451664|69394|11771|15382|1717711|5193|19475|8|6|844|28|86.84|136.33|77.70|0.00|2175.60|2431.52|3817.24|65.26|0.00|2175.60|2240.86|-255.92| +2451664|69394|5773|15382|1717711|5193|19475|8|168|844|49|76.24|105.21|22.09|0.00|1082.41|3735.76|5155.29|43.29|0.00|1082.41|1125.70|-2653.35| +2451664|69394|4691|15382|1717711|5193|19475|8|297|844|93|47.16|69.79|46.75|0.00|4347.75|4385.88|6490.47|43.47|0.00|4347.75|4391.22|-38.13| +2451664|69394|4364|15382|1717711|5193|19475|8|240|844|92|30.55|52.54|6.30|0.00|579.60|2810.60|4833.68|17.38|0.00|579.60|596.98|-2231.00| +2451664|69394|15451|15382|1717711|5193|19475|8|72|844|60|4.75|6.27|5.45|0.00|327.00|285.00|376.20|3.27|0.00|327.00|330.27|42.00| +2451664|69394|13843|15382|1717711|5193|19475|8|137|844|55|41.36|78.58|26.71|0.00|1469.05|2274.80|4321.90|117.52|0.00|1469.05|1586.57|-805.75| +2451664|69394|4739|15382|1717711|5193|19475|8|1|844|40|28.40|47.99|25.91|0.00|1036.40|1136.00|1919.60|41.45|0.00|1036.40|1077.85|-99.60| +2451664|69394|16333|15382|1717711|5193|19475|8|93|844|19|94.04|131.65|122.43|0.00|2326.17|1786.76|2501.35|46.52|0.00|2326.17|2372.69|539.41| +2451664|69394|266|15382|1717711|5193|19475|8|199|844|76|25.58|50.90|37.15|0.00|2823.40|1944.08|3868.40|0.00|0.00|2823.40|2823.40|879.32| +2451664|69394|2863|15382|1717711|5193|19475|8|97|844|41|77.24|89.59|28.66|0.00|1175.06|3166.84|3673.19|94.00|0.00|1175.06|1269.06|-1991.78| +2451664|69394|14339|15382|1717711|5193|19475|8|132|844|87|23.99|29.02|15.38|0.00|1338.06|2087.13|2524.74|107.04|0.00|1338.06|1445.10|-749.07| +2451664|69394|3707|15382|1717711|5193|19475|8|215|844|95|10.49|12.06|6.27|0.00|595.65|996.55|1145.70|29.78|0.00|595.65|625.43|-400.90| +2451503|66945|6404|81939|1170338|1490|38242|7|83|845|5|93.47|185.07|172.11|0.00|860.55|467.35|925.35|43.02|0.00|860.55|903.57|393.20| +2451503|66945|13430|81939|1170338|1490|38242|7|235|845|71|31.57|45.46|15.91|0.00|1129.61|2241.47|3227.66|45.18|0.00|1129.61|1174.79|-1111.86| +2451503|66945|13124|81939|1170338|1490|38242|7|34|845|35|55.01|82.51|3.30|0.00|115.50|1925.35|2887.85|0.00|0.00|115.50|115.50|-1809.85| +2451503|66945|16414|81939|1170338|1490|38242|7|182|845|61|19.15|24.32|11.67|0.00|711.87|1168.15|1483.52|14.23|0.00|711.87|726.10|-456.28| +2451503|66945|1316|81939|1170338|1490|38242|7|164|845|3|46.30|57.87|53.81|0.00|161.43|138.90|173.61|8.07|0.00|161.43|169.50|22.53| +2451503|66945|13708|81939|1170338|1490|38242|7|20|845|29|71.88|93.44|20.55|0.00|595.95|2084.52|2709.76|23.83|0.00|595.95|619.78|-1488.57| +2451503|66945|416|81939|1170338|1490|38242|7|93|845|52|13.92|27.42|9.87|92.38|513.24|723.84|1425.84|25.25|92.38|420.86|446.11|-302.98| +2451503|66945|14420|81939|1170338|1490|38242|7|283|845|49|38.88|59.09|43.13|0.00|2113.37|1905.12|2895.41|169.06|0.00|2113.37|2282.43|208.25| +2451503|66945|6949|81939|1170338|1490|38242|7|50|845|21|67.52|93.85|64.75|0.00|1359.75|1417.92|1970.85|122.37|0.00|1359.75|1482.12|-58.17| +2451503|66945|7825|81939|1170338|1490|38242|7|179|845|44|73.56|79.44|50.84|268.43|2236.96|3236.64|3495.36|59.05|268.43|1968.53|2027.58|-1268.11| +2451503|66945|14350|81939|1170338|1490|38242|7|29|845|6|42.27|44.38|9.76|0.00|58.56|253.62|266.28|0.00|0.00|58.56|58.56|-195.06| +2451105|73754|4513|71743|1235547|951|31155|2|274|846|41|61.63|97.37|34.07|0.00|1396.87|2526.83|3992.17|125.71|0.00|1396.87|1522.58|-1129.96| +2451105|73754|16886|71743|1235547|951|31155|2|49|846|49|17.98|33.80|27.71|0.00|1357.79|881.02|1656.20|108.62|0.00|1357.79|1466.41|476.77| +2451105|73754|1519|71743|1235547|951|31155|2|109|846|70|13.57|24.15|3.62|0.00|253.40|949.90|1690.50|17.73|0.00|253.40|271.13|-696.50| +2451105|73754|7610|71743|1235547|951|31155|2|193|846|9|10.51|15.65|11.73|0.00|105.57|94.59|140.85|2.11|0.00|105.57|107.68|10.98| +2451105|73754|12968|71743|1235547|951|31155|2|80|846|100|98.53|134.00|113.90|0.00|11390.00|9853.00|13400.00|569.50|0.00|11390.00|11959.50|1537.00| +2451105|73754|4622|71743|1235547|951|31155|2|271|846|75|64.95|83.78|77.07|0.00|5780.25|4871.25|6283.50|289.01|0.00|5780.25|6069.26|909.00| +2451105|73754|17308|71743|1235547|951|31155|2|253|846|18|68.98|100.02|39.00|372.06|702.00|1241.64|1800.36|23.09|372.06|329.94|353.03|-911.70| +2451105|73754|13717|71743|1235547|951|31155|2|180|846|87|67.27|71.30|54.90|0.00|4776.30|5852.49|6203.10|0.00|0.00|4776.30|4776.30|-1076.19| +2451105|73754|16363|71743|1235547|951|31155|2|139|846|58|18.24|30.46|17.05|0.00|988.90|1057.92|1766.68|0.00|0.00|988.90|988.90|-69.02| +2451105|73754|68|71743|1235547|951|31155|2|196|846|87|20.63|36.10|6.49|0.00|564.63|1794.81|3140.70|45.17|0.00|564.63|609.80|-1230.18| +2451105|73754|16460|71743|1235547|951|31155|2|241|846|43|29.96|39.54|33.60|0.00|1444.80|1288.28|1700.22|86.68|0.00|1444.80|1531.48|156.52| +2451105|73754|889|71743|1235547|951|31155|2|60|846|61|65.16|107.51|72.03|0.00|4393.83|3974.76|6558.11|43.93|0.00|4393.83|4437.76|419.07| +2451105|73754|12026|71743|1235547|951|31155|2|40|846|52|67.23|118.32|50.87|0.00|2645.24|3495.96|6152.64|185.16|0.00|2645.24|2830.40|-850.72| +2451105|73754|1184|71743|1235547|951|31155|2|263|846|94|11.64|21.99|21.33|1243.11|2005.02|1094.16|2067.06|60.95|1243.11|761.91|822.86|-332.25| +2451105|73754|16082|71743|1235547|951|31155|2|18|846|19|6.81|12.25|6.86|0.00|130.34|129.39|232.75|7.82|0.00|130.34|138.16|0.95| +2451105|73754|7945|71743|1235547|951|31155|2|102|846|61|70.19|91.24|71.16|0.00|4340.76|4281.59|5565.64|303.85|0.00|4340.76|4644.61|59.17| +2451859|68153|13868|9472|1221483|2741|19952|7|201|847|73|94.43|183.19|152.04|0.00|11098.92|6893.39|13372.87|110.98|0.00|11098.92|11209.90|4205.53| +2451859|68153|6151|9472|1221483|2741|19952|7|173|847|99|46.00|59.34|58.74|2035.34|5815.26|4554.00|5874.66|113.39|2035.34|3779.92|3893.31|-774.08| +2451859|68153|8885|9472|1221483|2741|19952|7|235|847|14|60.80|95.45|10.49|80.77|146.86|851.20|1336.30|0.00|80.77|66.09|66.09|-785.11| +2451859|68153|11585|9472|1221483|2741|19952|7|111|847|30|46.09|82.04|11.48|0.00|344.40|1382.70|2461.20|27.55|0.00|344.40|371.95|-1038.30| +2451859|68153|11891|9472|1221483|2741|19952|7|196|847|5|91.03|140.18|60.27|0.00|301.35|455.15|700.90|24.10|0.00|301.35|325.45|-153.80| +2451859|68153|12602||1221483|2741|19952|7|221|847||||23.69|0.00|1397.71||||0.00|1397.71|1439.64|-639.56| +2451859|68153|7379|9472|1221483|2741|19952|7|235|847|97|83.35|163.36|129.05|0.00|12517.85|8084.95|15845.92|250.35|0.00|12517.85|12768.20|4432.90| +2451859|68153|9887|9472|1221483|2741|19952|7|117|847|100|98.02|165.65|89.45|0.00|8945.00|9802.00|16565.00|0.00|0.00|8945.00|8945.00|-857.00| +2451859|68153|14011|9472|1221483|2741|19952|7|81|847|56|55.79|59.13|10.05|157.58|562.80|3124.24|3311.28|36.46|157.58|405.22|441.68|-2719.02| +2451859|68153|17995|9472|1221483|2741|19952|7|110|847|62|62.99|85.66|14.56|0.00|902.72|3905.38|5310.92|45.13|0.00|902.72|947.85|-3002.66| +2451901|36499|3949|77254|310638|3575|3359|2|203|848|98|78.20|132.15|79.29|0.00|7770.42|7663.60|12950.70|543.92|0.00|7770.42|8314.34|106.82| +2451901|36499|14147|77254|310638|3575|3359|2|126|848|26|61.67|80.78|51.69|1169.22|1343.94|1603.42|2100.28|13.97|1169.22|174.72|188.69|-1428.70| +2451901|36499|16595|77254|310638|3575|3359|2|148|848|49|60.66|94.62|3.78|0.00|185.22|2972.34|4636.38|3.70|0.00|185.22|188.92|-2787.12| +2451901|36499|10916|77254|310638|3575|3359|2|243|848|25|62.75|88.47|11.50|0.00|287.50|1568.75|2211.75|14.37|0.00|287.50|301.87|-1281.25| +2451901|36499|17366|77254||3575||2|165|848|52|55.03|106.75|55.51|0.00|||5551.00|144.32|0.00||3030.84|24.96| +2451901|36499|5245|77254|310638|3575|3359|2|241|848|22|74.28|81.70|59.64|0.00|1312.08|1634.16|1797.40|65.60|0.00|1312.08|1377.68|-322.08| +2451901|36499|12703|77254|310638|3575|3359|2|37|848|26|70.90|97.13|92.27|0.00|2399.02|1843.40|2525.38|191.92|0.00|2399.02|2590.94|555.62| +2451901|36499|209|77254|310638|3575|3359|2|110|848|78|59.85|92.16|23.04|0.00|1797.12|4668.30|7188.48|53.91|0.00|1797.12|1851.03|-2871.18| +2451901|36499|8317|77254|310638|3575|3359|2|179|848|27|33.08|35.39|10.61|0.00|286.47|893.16|955.53|11.45|0.00|286.47|297.92|-606.69| +2451901|36499|2498|77254|310638|3575|3359|2|258|848|75|86.96|173.92|166.96|8765.40|12522.00|6522.00|13044.00|112.69|8765.40|3756.60|3869.29|-2765.40| +2451901|36499|16343|77254|310638|3575|3359|2|236|848|4|72.98|140.85|91.55|0.00|366.20|291.92|563.40|25.63|0.00|366.20|391.83|74.28| +2451901|36499|7349|77254|310638|3575|3359|2|144|848|67|66.23|111.26|57.85|0.00|3875.95|4437.41|7454.42|310.07|0.00|3875.95|4186.02|-561.46| +2451537|71766|4861|94882|1394749|3063|47330|1|178|849|92|34.87|46.37|20.86|0.00|1919.12|3208.04|4266.04|115.14|0.00|1919.12|2034.26|-1288.92| +2451537|71766|8419|94882|1394749|3063|47330|1|18|849|48|72.50|120.35|78.22|2140.09|3754.56|3480.00|5776.80|80.72|2140.09|1614.47|1695.19|-1865.53| +2451537|71766|8342|94882|1394749|3063|47330|1|127|849|17|69.95|139.20|114.14|0.00|1940.38|1189.15|2366.40|0.00|0.00|1940.38|1940.38|751.23| +|71766|7777|94882|1394749|3063|||156|849||23.48|||||680.92||||125.57||| +2451537|71766|6620|94882|1394749|3063|47330|1|84|849|5|95.51|138.48|106.62|426.48|533.10|477.55|692.40|4.26|426.48|106.62|110.88|-370.93| +2451537|71766|8048|94882|1394749|3063|47330|1|209|849|96|74.84|98.04|50.98|3181.15|4894.08|7184.64|9411.84|0.00|3181.15|1712.93|1712.93|-5471.71| +2451537|71766|7744|94882|1394749|3063|47330|1|288|849|47|16.97|30.88|18.83|0.00|885.01|797.59|1451.36|35.40|0.00|885.01|920.41|87.42| +2451537|71766|16882|94882|1394749|3063|47330|1|225|849|6|22.01|25.09|6.77|0.00|40.62|132.06|150.54|1.21|0.00|40.62|41.83|-91.44| +2451537|71766|10682|94882|1394749|3063|47330|1|136|849|75|27.79|49.46|48.96|0.00|3672.00|2084.25|3709.50|73.44|0.00|3672.00|3745.44|1587.75| +2452591|57497|5431|69544|1181783|4775|46967|7|214|850|31|29.42|58.54|0.58|0.00|17.98|912.02|1814.74|0.71|0.00|17.98|18.69|-894.04| +2452591|57497|7872|69544|1181783|4775|46967|7|147|850|7|30.70|58.02|6.96|0.00|48.72|214.90|406.14|0.00|0.00|48.72|48.72|-166.18| +2452591|57497|13579|69544|1181783|4775|46967|7|286|850|36|46.08|48.84|0.00|0.00|0.00|1658.88|1758.24|0.00|0.00|0.00|0.00|-1658.88| +2452591|57497|3930|69544|1181783|4775|46967|7|66|850|84|78.30|131.54|127.59|0.00|10717.56|6577.20|11049.36|107.17|0.00|10717.56|10824.73|4140.36| +2452591|57497|1951|69544|1181783|4775|46967|7|120|850|54|75.61|140.63|42.18|0.00|2277.72|4082.94|7594.02|45.55|0.00|2277.72|2323.27|-1805.22| +2452591|57497|17349|69544|1181783|4775|46967|7|247|850|89|65.75|99.28|54.60|0.00|4859.40|5851.75|8835.92|145.78|0.00|4859.40|5005.18|-992.35| +2452591|57497|217|69544|1181783|4775|46967|7|195|850|11|95.47|187.12|162.79|1414.64|1790.69|1050.17|2058.32|18.80|1414.64|376.05|394.85|-674.12| +2452591|57497|14850|69544|1181783|4775|46967|7|271|850|91|24.30|43.25|37.19|0.00|3384.29|2211.30|3935.75|236.90|0.00|3384.29|3621.19|1172.99| +2452591|57497|17875|69544|1181783|4775|46967|7|75|850|34|78.05|149.85|55.44|885.93|1884.96|2653.70|5094.90|39.96|885.93|999.03|1038.99|-1654.67| +2452591|57497|3240|69544|1181783|4775|46967|7|252|850|70|69.13|81.57|41.60|0.00|2912.00|4839.10|5709.90|145.60|0.00|2912.00|3057.60|-1927.10| +2451739|45512|12011|62810|1677642|474|4459|8|62|851|91|63.68|105.70|57.07|0.00|5193.37|5794.88|9618.70|311.60|0.00|5193.37|5504.97|-601.51| +2451739|45512|6050|62810|1677642|474|4459|8|3|851|6|39.05|41.00|30.75|154.98|184.50|234.30|246.00|0.00|154.98|29.52|29.52|-204.78| +2451739|45512|1943|62810|1677642|474|4459|8|238|851|51|10.07|15.00|14.70|0.00|749.70|513.57|765.00|7.49|0.00|749.70|757.19|236.13| +2451739|45512|4406|62810|1677642|474|4459|8|13|851|63|83.37|95.04|58.92|0.00|3711.96|5252.31|5987.52|111.35|0.00|3711.96|3823.31|-1540.35| +2451739|45512|9335|62810|1677642|474|4459|8|284|851|34|53.09|64.76|0.64|0.00|21.76|1805.06|2201.84|1.52|0.00|21.76|23.28|-1783.30| +2451739|45512|377|62810|1677642|474|4459|8|232|851|41|78.64|144.69|54.98|0.00|2254.18|3224.24|5932.29|90.16|0.00|2254.18|2344.34|-970.06| +2451739|45512|16058|62810|1677642|474|4459|8|258|851|12|11.67|11.67|10.96|53.92|131.52|140.04|140.04|0.00|53.92|77.60|77.60|-62.44| +2451739|45512|6517|62810|1677642|474|4459|8|125|851|94|76.08|117.16|103.10|0.00|9691.40|7151.52|11013.04|872.22|0.00|9691.40|10563.62|2539.88| +2451739|45512|9158|62810|1677642|474|4459|8|84|851|96|43.87|64.48|26.43|2004.45|2537.28|4211.52|6190.08|10.65|2004.45|532.83|543.48|-3678.69| +2451739|45512|1165|62810|1677642|474|4459|8|192|851|78|50.76|50.76|18.27|0.00|1425.06|3959.28|3959.28|28.50|0.00|1425.06|1453.56|-2534.22| +2451739|45512|17479|62810|1677642|474|4459|8|78|851|46|53.33|102.92|11.32|0.00|520.72|2453.18|4734.32|5.20|0.00|520.72|525.92|-1932.46| +2452426|67675|13548|78064|740696|928|3960|2|148|852|60|74.03|137.69|72.97|700.51|4378.20|4441.80|8261.40|257.43|700.51|3677.69|3935.12|-764.11| +2452426|67675|17118||||3960|2|59|852||||0.00|0.00||4378.80||0.00|0.00|||| +2452426|67675|10056|78064|740696|928|3960|2|265|852|48|19.74|25.66|7.95|145.00|381.60|947.52|1231.68|16.56|145.00|236.60|253.16|-710.92| +2452426|67675|13969|78064|740696|928|3960|2|275|852|48|83.20|94.01|7.52|0.00|360.96|3993.60|4512.48|7.21|0.00|360.96|368.17|-3632.64| +2452426|67675|1749|78064|740696|928|3960|2|153|852|72|44.19|53.02|35.52|0.00|2557.44|3181.68|3817.44|204.59|0.00|2557.44|2762.03|-624.24| +2452426|67675|14259|78064|740696|928|3960|2|289|852|51|8.03|9.79|4.89|0.00|249.39|409.53|499.29|12.46|0.00|249.39|261.85|-160.14| +2452426|67675|14214|78064|740696|928|3960|2|122|852|30|14.98|24.11|20.49|0.00|614.70|449.40|723.30|49.17|0.00|614.70|663.87|165.30| +2452426|67675|2691|78064|740696|928|3960|2|24|852|39|1.10|1.82|1.09|0.00|42.51|42.90|70.98|3.82|0.00|42.51|46.33|-0.39| +2452426|67675|2013|78064|740696|928|3960|2|236|852|100|27.44|38.14|10.67|480.15|1067.00|2744.00|3814.00|11.73|480.15|586.85|598.58|-2157.15| +2452426|67675|12816|78064|740696|928|3960|2|44|852|89|5.08|8.78|6.58|0.00|585.62|452.12|781.42|5.85|0.00|585.62|591.47|133.50| +2452426|67675|697|78064|740696|928|3960|2|53|852|44|99.63|148.44|0.00|0.00|0.00|4383.72|6531.36|0.00|0.00|0.00|0.00|-4383.72| +2452426|67675|17049|78064|740696|928|3960|2|162|852|92|91.88|132.30|75.41|3746.36|6937.72|8452.96|12171.60|63.82|3746.36|3191.36|3255.18|-5261.60| +2452426|67675|2256|78064|740696|928|3960|2|213|852|79|87.08|148.90|105.71|6096.29|8351.09|6879.32|11763.10|112.74|6096.29|2254.80|2367.54|-4624.52| +2452426|67675|3366|78064|740696|928|3960|2|21|852|39|54.10|67.62|8.11|161.30|316.29|2109.90|2637.18|0.00|161.30|154.99|154.99|-1954.91| +2452426|67675|1884|78064|740696|928|3960|2|52|852|28|57.73|92.36|47.10|0.00|1318.80|1616.44|2586.08|26.37|0.00|1318.80|1345.17|-297.64| +2452426|67675|6369|78064|740696|928|3960|2|144|852|68|32.22|33.83|18.60|0.00|1264.80|2190.96|2300.44|63.24|0.00|1264.80|1328.04|-926.16| +2451780|49127|15113|98495|1648318|6141|39564|1|157|853|93|45.16|57.80|51.44|0.00|4783.92|4199.88|5375.40|191.35|0.00|4783.92|4975.27|584.04| +2451780|49127|169|98495|1648318|6141|39564|1|9|853|52|54.22|65.60|61.66|1186.33|3206.32|2819.44|3411.20|121.19|1186.33|2019.99|2141.18|-799.45| +2451780|49127|5942|98495|1648318|6141|39564|1|24|853|90|10.73|19.31|10.42|0.00|937.80|965.70|1737.90|75.02|0.00|937.80|1012.82|-27.90| +2451780|49127|8486|98495|1648318|6141|39564|1|286|853|47|7.96|14.48|4.63|0.00|217.61|374.12|680.56|0.00|0.00|217.61|217.61|-156.51| +2451780|49127|12047|98495|1648318|6141|39564|1|116|853|92|36.34|69.77|33.48|0.00|3080.16|3343.28|6418.84|92.40|0.00|3080.16|3172.56|-263.12| +2451780|49127|12932|98495|1648318|6141|39564|1|144|853|84|62.10|113.02|111.88|0.00|9397.92|5216.40|9493.68|281.93|0.00|9397.92|9679.85|4181.52| +2451780|49127|11198|98495|1648318|6141|39564|1|127|853|78|54.36|66.86|64.85|0.00|5058.30|4240.08|5215.08|455.24|0.00|5058.30|5513.54|818.22| +2451780|49127|7268|98495|1648318|6141|39564|1|18|853|73|50.97|83.59|22.56|0.00|1646.88|3720.81|6102.07|131.75|0.00|1646.88|1778.63|-2073.93| +2451780|49127|8075|98495|1648318|6141|39564|1|77|853|6|22.03|30.40|19.45|0.00|116.70|132.18|182.40|9.33|0.00|116.70|126.03|-15.48| +2451780|49127|5935|98495|1648318|6141|39564|1|187|853|12|4.12|7.37|2.35|0.00|28.20|49.44|88.44|1.97|0.00|28.20|30.17|-21.24| +2451780|49127|3554|98495|1648318|6141|39564|1|178|853|88|63.42|88.15|46.71|2507.39|4110.48|5580.96|7757.20|144.27|2507.39|1603.09|1747.36|-3977.87| +2451780|49127|6029|98495|1648318|6141|39564|1|68|853|43|62.79|113.02|48.59|0.00|2089.37|2699.97|4859.86|20.89|0.00|2089.37|2110.26|-610.60| +2451780|49127|1751|98495|1648318|6141|39564|1|39|853|36|4.66|6.89|2.75|26.73|99.00|167.76|248.04|0.72|26.73|72.27|72.99|-95.49| +2451780|49127|9653|98495|1648318|6141|39564|1|262|853|39|55.40|101.93|93.77|0.00|3657.03|2160.60|3975.27|255.99|0.00|3657.03|3913.02|1496.43| +2451780|49127|5801|98495||6141|39564||42|853|70|45.34||69.41|||3173.80|5585.30|||||| +2451099|52840|11834|49741|1014956|6924|41692|7|120|854|74|37.68|70.46|9.86|0.00|729.64|2788.32|5214.04|0.00|0.00|729.64|729.64|-2058.68| +2451099||1213||1014956||||298|854|48||||0.00||2820.96||33.36|0.00|3336.96||516.00| +2451099|52840|15934|49741|1014956|6924|41692|7|213|854|5|53.89|100.77|92.70|0.00|463.50|269.45|503.85|13.90|0.00|463.50|477.40|194.05| +2451099|52840|13222|49741|1014956|6924|41692|7|145|854|92|9.81|12.36|4.44|0.00|408.48|902.52|1137.12|32.67|0.00|408.48|441.15|-494.04| +2451099||16921|||6924||||854||||43.60||1613.20||1716.43|132.12||1468.02||| +2451099|52840|17617|49741|1014956|6924|41692|7|16|854|3|33.24|34.56|31.44|46.21|94.32|99.72|103.68|0.48|46.21|48.11|48.59|-51.61| +2451099|52840|3962|49741|1014956|6924|41692|7|37|854|21|54.26|85.73|71.15|0.00|1494.15|1139.46|1800.33|29.88|0.00|1494.15|1524.03|354.69| +2451099|52840|17584|49741|1014956|6924|41692|7|187|854|48|73.84|77.53|54.27|0.00|2604.96|3544.32|3721.44|208.39|0.00|2604.96|2813.35|-939.36| +2451099|52840|3088|49741|1014956|6924|41692|7|254|854|70|62.80|71.59|0.71|0.00|49.70|4396.00|5011.30|1.49|0.00|49.70|51.19|-4346.30| +|52840|5887||1014956|6924|41692|||854|27||||0.00|438.21|608.85||21.91|0.00|438.21|460.12|-170.64| +2451099|52840|7874|49741|1014956|6924|41692|7|145|854|50|79.00|142.99|90.08|0.00|4504.00|3950.00|7149.50|225.20|0.00|4504.00|4729.20|554.00| +2451099|52840|9472|49741|1014956|6924|41692|7|164|854|6|90.32|158.96|23.84|0.00|143.04|541.92|953.76|5.72|0.00|143.04|148.76|-398.88| +2451099|52840|4880|49741|1014956|6924|41692|7|186|854|38|65.06|83.27|79.93|0.00|3037.34|2472.28|3164.26|91.12|0.00|3037.34|3128.46|565.06| +2451099|52840|6955|49741|1014956|6924|41692|7|251|854|63|43.35|80.63|60.47|0.00|3809.61|2731.05|5079.69|342.86|0.00|3809.61|4152.47|1078.56| +2451099|52840|2150|49741|1014956|6924|41692|7|5|854|99|1.49|2.53|0.35|0.00|34.65|147.51|250.47|0.00|0.00|34.65|34.65|-112.86| +2451531|62815|9602|18429|1444448|6467|18886|10|243|855|41|80.66|92.75|60.28|0.00|2471.48|3307.06|3802.75|24.71|0.00|2471.48|2496.19|-835.58| +2451531|62815|5590|18429|1444448|6467|18886|10|75|855|100|60.77|75.35|65.55|4391.85|6555.00|6077.00|7535.00|173.05|4391.85|2163.15|2336.20|-3913.85| +2451531|62815|13165|18429|1444448|6467|18886|10|136|855|87|29.42|57.66|40.36|0.00|3511.32|2559.54|5016.42|0.00|0.00|3511.32|3511.32|951.78| +2451531|62815|6460|18429|1444448|6467|18886|10|57|855|64|6.50|10.66|1.81|77.61|115.84|416.00|682.24|3.44|77.61|38.23|41.67|-377.77| +2451531|62815|9016|18429|1444448|6467|18886|10|197|855|66|58.46|64.89|36.33|0.00|2397.78|3858.36|4282.74|191.82|0.00|2397.78|2589.60|-1460.58| +2451531|62815|7090|18429|1444448|6467|18886|10|152|855|27|81.59|139.51|129.74|0.00|3502.98|2202.93|3766.77|175.14|0.00|3502.98|3678.12|1300.05| +2451531|62815|10573|18429|1444448|6467|18886|10|44|855|79|9.79|15.76|8.19|0.00|647.01|773.41|1245.04|58.23|0.00|647.01|705.24|-126.40| +2451531|62815|10856|18429|1444448|6467|18886|10|12|855|8|94.89|117.66|112.95|0.00|903.60|759.12|941.28|0.00|0.00|903.60|903.60|144.48| +2451531|62815|14407|18429|1444448|6467|18886|10|175|855|43|23.20|42.22|28.70|0.00|1234.10|997.60|1815.46|0.00|0.00|1234.10|1234.10|236.50| +2451531|62815|15866|18429|1444448|6467|18886|10|121|855|27|6.24|9.04|3.07|0.00|82.89|168.48|244.08|5.80|0.00|82.89|88.69|-85.59| +2452138|55696|8085|78793|1416671|2902|28780|2|284|856|25|81.87|134.26|69.81|0.00|1745.25|2046.75|3356.50|157.07|0.00|1745.25|1902.32|-301.50| +2452138|55696|227|78793|1416671|2902|28780|2|122|856|47|55.04|87.51|66.50|0.00|3125.50|2586.88|4112.97|31.25|0.00|3125.50|3156.75|538.62| +2452138|55696|17015|78793|1416671|2902|28780|2|144|856|56|40.86|79.67|64.53|0.00|3613.68|2288.16|4461.52|144.54|0.00|3613.68|3758.22|1325.52| +2452138|55696|5087||1416671|2902|||24|856|87||99.35||0.00||||178.91|0.00||6142.76|| +|55696|17339|78793|||28780|2|132|856||||1.55|||||||29.45|31.51|-267.52| +2452138|55696|13655|78793|1416671|2902|28780|2|298|856|59|46.61|92.75|1.85|89.50|109.15|2749.99|5472.25|1.57|89.50|19.65|21.22|-2730.34| +2452138|55696|15579|78793|1416671|2902|28780|2|127|856|7|67.41|99.76|58.85|0.00|411.95|471.87|698.32|12.35|0.00|411.95|424.30|-59.92| +2452138|55696|6587|78793|1416671|2902|28780|2|229|856|43|29.98|47.36|43.57|0.00|1873.51|1289.14|2036.48|93.67|0.00|1873.51|1967.18|584.37| +2452138|55696|3187|78793|1416671|2902|28780|2|246|856|47|59.57|80.41|4.82|0.00|226.54|2799.79|3779.27|15.85|0.00|226.54|242.39|-2573.25| +2452138|55696|3009|78793|1416671|2902|28780|2|281|856|63|1.58|1.76|0.93|0.00|58.59|99.54|110.88|2.92|0.00|58.59|61.51|-40.95| +|55696|13195|78793||2902||2||856|72|74.85||||1524.24|5389.20|6628.32|45.72|||1569.96|| +2452138|55696|11611|78793|1416671|2902|28780|2|37|856|15|95.56|163.40|132.35|1488.93|1985.25|1433.40|2451.00|19.85|1488.93|496.32|516.17|-937.08| +2452138|55696|8593|78793|1416671|2902|28780|2|239|856|36|39.23|41.19|28.00|0.00|1008.00|1412.28|1482.84|0.00|0.00|1008.00|1008.00|-404.28| +2452138|55696|1217|78793|1416671|2902|28780|2|191|856|95|60.86|115.02|48.30|3120.18|4588.50|5781.70|10926.90|0.00|3120.18|1468.32|1468.32|-4313.38| +2452138|55696|11669|78793|1416671|2902|28780|2|112|856|9|81.78|86.68|2.60|0.00|23.40|736.02|780.12|0.70|0.00|23.40|24.10|-712.62| +2452259|28925|10163|35904|1116952|5200|25801|1|264|857|61|3.00|5.19|5.13|0.00|312.93|183.00|316.59|21.90|0.00|312.93|334.83|129.93| +2452259|28925|5425|35904|1116952|5200|25801|1|117|857|53|47.32|88.48|71.66|0.00|3797.98|2507.96|4689.44|75.95|0.00|3797.98|3873.93|1290.02| +2452259|28925|6027|35904|1116952|5200|25801|1|295|857|78|50.86|59.50|28.56|0.00|2227.68|3967.08|4641.00|133.66|0.00|2227.68|2361.34|-1739.40| +2452259|28925|13601|35904|1116952|5200|25801|1|19|857|35|57.19|62.90|49.69|0.00|1739.15|2001.65|2201.50|156.52|0.00|1739.15|1895.67|-262.50| +||2619|35904||5200|25801|||857||21.25|||||255.00||20.49|||430.41|| +2452259|28925|15131|35904|1116952|5200|25801|1|114|857|83|10.47|15.28|12.98|0.00|1077.34|869.01|1268.24|86.18|0.00|1077.34|1163.52|208.33| +2452259|28925|9801|35904|1116952|5200|25801|1|126|857|27|1.87|3.51|1.82|0.00|49.14|50.49|94.77|0.00|0.00|49.14|49.14|-1.35| +2452259|28925|14935|35904|1116952|5200|25801|1|167|857|44|71.76|72.47|40.58|1285.57|1785.52|3157.44|3188.68|19.99|1285.57|499.95|519.94|-2657.49| +2451485|41138|16762|52011|203547|3883|31866|7|124|858|92|69.41|113.13|93.89|0.00|8637.88|6385.72|10407.96|518.27|0.00|8637.88|9156.15|2252.16| +2451485|41138|14198|52011|203547|3883|31866|7|89|858|33|41.60|49.50|38.61|1095.75|1274.13|1372.80|1633.50|12.48|1095.75|178.38|190.86|-1194.42| +2451485|41138|15452|52011|203547|3883|31866|7|19|858|6|57.47|109.76|96.58|0.00|579.48|344.82|658.56|0.00|0.00|579.48|579.48|234.66| +2451485|41138|14774|52011|203547|3883|31866|7|159|858|71|53.94|80.91|73.62|0.00|5227.02|3829.74|5744.61|104.54|0.00|5227.02|5331.56|1397.28| +2451485|41138|1855|52011|203547|3883|31866|7|144|858|38|44.65|83.04|33.21|0.00|1261.98|1696.70|3155.52|25.23|0.00|1261.98|1287.21|-434.72| +2451485|41138|5323|52011|203547|3883|31866|7|132|858|32|80.09|157.77|66.26|0.00|2120.32|2562.88|5048.64|127.21|0.00|2120.32|2247.53|-442.56| +2451485|41138|1615|52011|203547|3883|31866|7|116|858|83|84.24|117.09|36.29|0.00|3012.07|6991.92|9718.47|90.36|0.00|3012.07|3102.43|-3979.85| +2451485|41138|11930|52011|203547|3883|31866|7|4|858|35|84.87|140.03|88.21|0.00|3087.35|2970.45|4901.05|277.86|0.00|3087.35|3365.21|116.90| +2451485|41138|2212|52011|203547|3883|31866|7|94|858|60|67.22|86.04|7.74|0.00|464.40|4033.20|5162.40|4.64|0.00|464.40|469.04|-3568.80| +|41138|17120||203547|||7|63|858|||8.06|6.93||||386.88|29.93||332.64||| +2451485|41138|10948|52011|203547|3883|31866|7|146|858|61|52.96|95.85|92.97|0.00|5671.17|3230.56|5846.85|56.71|0.00|5671.17|5727.88|2440.61| +2451485|41138|17383|52011|203547|3883|31866|7|29|858|23|32.88|47.34|0.94|16.43|21.62|756.24|1088.82|0.05|16.43|5.19|5.24|-751.05| +2451485|41138|11380|52011|203547|3883|31866|7|177|858|100|69.32|130.32|115.98|0.00|11598.00|6932.00|13032.00|0.00|0.00|11598.00|11598.00|4666.00| +2451485|41138|13684|52011|203547|3883|31866|7|119|858|94|84.15|99.29|38.72|0.00|3639.68|7910.10|9333.26|291.17|0.00|3639.68|3930.85|-4270.42| +2451485|41138|11390|52011|203547|3883|31866|7|240|858|14|5.73|10.77|3.98|0.00|55.72|80.22|150.78|3.34|0.00|55.72|59.06|-24.50| +2451485|41138|6928|52011|203547|3883|31866|7|174|858|41|19.75|20.14|7.45|0.00|305.45|809.75|825.74|15.27|0.00|305.45|320.72|-504.30| +2452135|35574|157|56094|1824076|58|24393|1|74|859|52|22.86|36.80|27.60|0.00|1435.20|1188.72|1913.60|71.76|0.00|1435.20|1506.96|246.48| +2452135|35574|10699|56094|1824076|58|24393|1|122|859|23|13.74|25.14|15.08|0.00|346.84|316.02|578.22|10.40|0.00|346.84|357.24|30.82| +2452135|35574|15157|56094|1824076|58|24393|1|102|859|7|55.59|66.15|29.10|144.62|203.70|389.13|463.05|4.72|144.62|59.08|63.80|-330.05| +2452135|35574|17225|56094|1824076|58|24393|1|106|859|31|90.28|130.00|15.60|0.00|483.60|2798.68|4030.00|14.50|0.00|483.60|498.10|-2315.08| +2452135|35574|11991|56094|1824076|58|24393|1|40|859|69|36.61|56.01|21.84|0.00|1506.96|2526.09|3864.69|15.06|0.00|1506.96|1522.02|-1019.13| +2452135|35574|9901|56094|1824076|58|24393|1|229|859|21|76.12|105.04|6.30|97.90|132.30|1598.52|2205.84|1.03|97.90|34.40|35.43|-1564.12| +2452135|35574|7607|56094|1824076||24393|||859||70.92||113.32|0.00||2198.52|||0.00|3512.92|3829.08|| +2452135|35574|15649|56094|1824076|58|24393|1|133|859|55|78.46|104.35|36.52|0.00|2008.60|4315.30|5739.25|160.68|0.00|2008.60|2169.28|-2306.70| +2452135|35574|11489|56094|1824076|58|24393|1|242|859|48|57.46|108.59|14.11|0.00|677.28|2758.08|5212.32|0.00|0.00|677.28|677.28|-2080.80| +2451440|52344|4177|87877|377843|7009|40575|8|41|860|82|34.05|65.37|36.60|0.00|3001.20|2792.10|5360.34|150.06|0.00|3001.20|3151.26|209.10| +2451440|52344|2360|87877|377843|7009|40575|8|23|860|43|96.69|183.71|130.43|0.00|5608.49|4157.67|7899.53|504.76|0.00|5608.49|6113.25|1450.82| +2451440|52344|4322|87877|377843|7009|40575|8|137|860|87|5.94|8.73|3.57|0.00|310.59|516.78|759.51|27.95|0.00|310.59|338.54|-206.19| +2451440|52344|6835|87877|377843|7009|40575|8|232|860|83|74.14|93.41|91.54|0.00|7597.82|6153.62|7753.03|151.95|0.00|7597.82|7749.77|1444.20| +2451440|52344|7792|87877|377843|7009|40575|8|150|860|13|97.51|121.88|12.18|0.00|158.34|1267.63|1584.44|12.66|0.00|158.34|171.00|-1109.29| +2451440|52344|14881|87877|377843|7009|40575|8|167|860|22|72.02|82.82|78.67|1073.05|1730.74|1584.44|1822.04|0.00|1073.05|657.69|657.69|-926.75| +2451440|52344|3949|87877|377843|7009|40575|8|248|860|25|5.03|8.09|4.69|86.76|117.25|125.75|202.25|1.52|86.76|30.49|32.01|-95.26| +2451440|52344|14146|87877|377843|7009|40575|8|29|860|92|21.92|26.52|5.56|0.00|511.52|2016.64|2439.84|0.00|0.00|511.52|511.52|-1505.12| +2451440|52344|16594|87877|377843|7009|40575|8|69|860|39|45.93|73.02|5.84|0.00|227.76|1791.27|2847.78|15.94|0.00|227.76|243.70|-1563.51| +2451440|52344|10916|87877|377843|7009|40575|8|165|860|94|20.66|36.15|15.90|0.00|1494.60|1942.04|3398.10|119.56|0.00|1494.60|1614.16|-447.44| +2451501|65200|16546|95936|487960|6031|24055|8|111|861|48|7.03|7.38|3.02|0.00|144.96|337.44|354.24|1.44|0.00|144.96|146.40|-192.48| +2451501|65200|17006|95936|487960|6031|24055|8|91|861|76|80.56|85.39|83.68|0.00|6359.68|6122.56|6489.64|445.17|0.00|6359.68|6804.85|237.12| +2451501|65200|9589|95936|487960|6031|24055|8|84|861|84|48.27|70.95|48.24|486.25|4052.16|4054.68|5959.80|106.97|486.25|3565.91|3672.88|-488.77| +2451501|65200|12145|95936|487960|6031|24055|8|84|861|15|88.97|93.41|9.34|0.00|140.10|1334.55|1401.15|12.60|0.00|140.10|152.70|-1194.45| +2451501|65200|5209|95936|487960|6031|24055|8|5|861|30|13.37|18.58|8.54|179.34|256.20|401.10|557.40|0.00|179.34|76.86|76.86|-324.24| +2451501|65200|7864|95936|487960|6031|24055|8|23|861|61|38.83|74.16|48.20|1764.12|2940.20|2368.63|4523.76|35.28|1764.12|1176.08|1211.36|-1192.55| +2451501|65200|15697|95936|487960|6031|24055|8|242|861|82|26.50|30.74|4.61|215.47|378.02|2173.00|2520.68|1.62|215.47|162.55|164.17|-2010.45| +2451501|65200|6874|95936|487960|6031|24055|8|212|861|74|95.69|185.63|144.79|0.00|10714.46|7081.06|13736.62|857.15|0.00|10714.46|11571.61|3633.40| +2451501|65200|10981|95936|487960|6031|24055|8|290|861|55|41.08|50.11|17.53|0.00|964.15|2259.40|2756.05|48.20|0.00|964.15|1012.35|-1295.25| +2451501|65200|560|95936|487960|6031|24055|8|193|861|63|1.98|2.23|0.15|7.18|9.45|124.74|140.49|0.00|7.18|2.27|2.27|-122.47| +2451501|65200|11230|95936|487960|6031|24055|8|18|861|17|1.71|2.51|1.22|0.00|20.74|29.07|42.67|1.86|0.00|20.74|22.60|-8.33| +2450846|39591|6602|89383|942949|868|30834|4|199|862|21|61.59|104.08|87.42|0.00|1835.82|1293.39|2185.68|73.43|0.00|1835.82|1909.25|542.43| +2450846|39591|11054|89383|942949|868|30834|4|245|862|51|71.83|74.70|4.48|0.00|228.48|3663.33|3809.70|18.27|0.00|228.48|246.75|-3434.85| +2450846|39591|3872|89383|942949|868|30834|4|253|862|48|31.58|46.73|29.90|0.00|1435.20|1515.84|2243.04|86.11|0.00|1435.20|1521.31|-80.64| +2450846|39591|15661|89383|942949|868|30834|4|58|862|47|74.87|110.80|72.02|0.00|3384.94|3518.89|5207.60|236.94|0.00|3384.94|3621.88|-133.95| +2450846||3796|89383|942949||30834|4|12|862|97|63.13|||0.00|||10225.74|521.47|0.00|||2567.59| +2450846|39591|12152|89383|942949|868|30834|4|50|862|83|17.18|20.61|17.72|338.27|1470.76|1425.94|1710.63|22.64|338.27|1132.49|1155.13|-293.45| +2450846|39591|4142|89383|942949|868|30834|4|122|862|65|3.43|5.76|0.34|0.00|22.10|222.95|374.40|0.44|0.00|22.10|22.54|-200.85| +2450846|39591|16573|89383|942949|868|30834|4|253|862|74|65.40|105.29|15.79|0.00|1168.46|4839.60|7791.46|70.10|0.00|1168.46|1238.56|-3671.14| +2450846|39591|8512|89383|942949|868|30834|4|222|862|27|82.04|159.15|41.37|0.00|1116.99|2215.08|4297.05|78.18|0.00|1116.99|1195.17|-1098.09| +2450846|39591|4394|89383|942949|868|30834|4|132|862|61|90.39|147.33|29.46|0.00|1797.06|5513.79|8987.13|17.97|0.00|1797.06|1815.03|-3716.73| +2450846|39591|16552|89383|942949|868|30834|4|7|862|59|22.10|37.12|19.67|0.00|1160.53|1303.90|2190.08|34.81|0.00|1160.53|1195.34|-143.37| +2450846|39591|2596|89383|942949|868|30834|4|262|862|96|11.99|12.22|5.49|0.00|527.04|1151.04|1173.12|15.81|0.00|527.04|542.85|-624.00| +2450846|39591|1712|89383|942949|868|30834|4|100|862|87|59.77|113.56|81.76|0.00|7113.12|5199.99|9879.72|213.39|0.00|7113.12|7326.51|1913.13| +2450846|39591|15544|89383|942949|868|30834|4|286|862|41|81.86|103.96|0.00|0.00|0.00|3356.26|4262.36|0.00|0.00|0.00|0.00|-3356.26| +2450846|39591|12970|89383|942949|868|30834|4|97|862|30|76.55|110.23|69.44|0.00|2083.20|2296.50|3306.90|83.32|0.00|2083.20|2166.52|-213.30| +2451582|37536|5449|87111|1024086|1414|42389|8|149|863|47|55.94|70.48|58.49|0.00|2749.03|2629.18|3312.56|192.43|0.00|2749.03|2941.46|119.85| +2451582|37536|1682|87111|1024086|1414|42389|8|201|863|100|88.82|103.91|67.54|0.00|6754.00|8882.00|10391.00|67.54|0.00|6754.00|6821.54|-2128.00| +2451582|37536|6631|87111|1024086|1414|42389|8|275|863|92|18.37|26.82|1.87|0.00|172.04|1690.04|2467.44|5.16|0.00|172.04|177.20|-1518.00| +2451582|37536|4667|87111|1024086|1414|42389|8|117|863|90|22.56|36.99|32.18|0.00|2896.20|2030.40|3329.10|260.65|0.00|2896.20|3156.85|865.80| +2451582|37536|10496|87111|1024086|1414|42389|8|185|863|59|45.54|47.81|35.85|0.00|2115.15|2686.86|2820.79|126.90|0.00|2115.15|2242.05|-571.71| +2451582|37536|8399|87111|1024086|1414|42389|8|144|863|52|87.15|166.45|139.81|0.00|7270.12|4531.80|8655.40|654.31|0.00|7270.12|7924.43|2738.32| +2451582|37536|4141|87111|1024086|1414|42389|8|181|863|15|4.17|5.96|3.87|0.00|58.05|62.55|89.40|3.48|0.00|58.05|61.53|-4.50| +2451582|37536|295|87111|1024086|1414|42389|8|162|863|98|72.75|82.93|4.97|0.00|487.06|7129.50|8127.14|4.87|0.00|487.06|491.93|-6642.44| +2451582|37536|12284|87111|1024086|1414|42389|8|51|863|29|92.90|107.76|39.87|0.00|1156.23|2694.10|3125.04|92.49|0.00|1156.23|1248.72|-1537.87| +2451582|37536|4253|87111|1024086|1414|42389|8|78|863|65|48.75|60.93|56.66|0.00|3682.90|3168.75|3960.45|110.48|0.00|3682.90|3793.38|514.15| +2451582|37536|4339|87111|1024086|1414|42389|8|198|863|39|13.57|17.50|9.27|307.30|361.53|529.23|682.50|3.79|307.30|54.23|58.02|-475.00| +2451136|41987|8089|35868|413626|4984|3942|2|199|864|94|21.88|43.76|19.25|0.00|1809.50|2056.72|4113.44|126.66|0.00|1809.50|1936.16|-247.22| +2451136|41987|482|35868|413626|4984|3942|2|270|864|70|28.09|30.33|15.16|986.91|1061.20|1966.30|2123.10|2.97|986.91|74.29|77.26|-1892.01| +2451136|41987|3382|35868|413626|4984|3942|2|61|864|64|56.47|101.64|62.00|0.00|3968.00|3614.08|6504.96|317.44|0.00|3968.00|4285.44|353.92| +2451136|41987|16690|35868|413626|4984|3942|2|253|864|18|30.79|41.87|14.65|0.00|263.70|554.22|753.66|23.73|0.00|263.70|287.43|-290.52| +2451136|41987|1292|35868|413626|4984|3942|2|180|864|61|9.80|18.03|10.45|0.00|637.45|597.80|1099.83|25.49|0.00|637.45|662.94|39.65| +2451136|41987|14032|35868|413626|4984|3942|2|85|864|34|73.78|147.56|73.78|0.00|2508.52|2508.52|5017.04|150.51|0.00|2508.52|2659.03|0.00| +2451136|41987|1492|35868|413626|4984|3942|2|43|864|69|14.36|20.24|16.19|178.73|1117.11|990.84|1396.56|84.45|178.73|938.38|1022.83|-52.46| +2451136|41987|5611|35868|413626|4984|3942|2|282|864|46|15.74|29.74|20.52|519.15|943.92|724.04|1368.04|4.24|519.15|424.77|429.01|-299.27| +2451136|41987|559||||||287|864||79.74|||1774.48|3696.84||8037.12||1774.48|1922.36|2076.14|-4775.80| +||559|36240||||||865|42|||66.46||2791.32|2318.82|3246.18|55.82|||2847.14|| +2452159|43434|16147|36240|1377962|1637|1759|8|6|865|26|52.60|91.52|22.88|0.00|594.88|1367.60|2379.52|17.84|0.00|594.88|612.72|-772.72| +2452159|43434|11875|36240|1377962|1637|1759|8|49|865|35|80.26|126.81|101.44|0.00|3550.40|2809.10|4438.35|0.00|0.00|3550.40|3550.40|741.30| +2452159|43434|10215|36240|1377962|1637|1759|8|79|865|94|99.88|173.79|72.99|0.00|6861.06|9388.72|16336.26|617.49|0.00|6861.06|7478.55|-2527.66| +2452159|43434|12447|36240|1377962|1637|1759|8|106|865|22|51.61|74.31|4.45|85.17|97.90|1135.42|1634.82|0.12|85.17|12.73|12.85|-1122.69| +2452159|43434|245|36240|1377962|1637|1759|8|202|865|44|97.67|135.76|86.88|0.00|3822.72|4297.48|5973.44|76.45|0.00|3822.72|3899.17|-474.76| +2452159|43434|1531|36240|1377962|1637|1759|8|153|865|22|57.99|79.44|3.97|0.00|87.34|1275.78|1747.68|5.24|0.00|87.34|92.58|-1188.44| +2452159|43434|3525|36240|1377962|1637|1759|8|276|865|95|64.38|81.76|18.80|0.00|1786.00|6116.10|7767.20|142.88|0.00|1786.00|1928.88|-4330.10| +2452159|43434|13021|36240|1377962|1637|1759|8|263|865|27|78.87|108.84|68.56|0.00|1851.12|2129.49|2938.68|18.51|0.00|1851.12|1869.63|-278.37| +2452159|43434|17263|36240|1377962|1637|1759|8|117|865|69|97.70|166.09|46.50|0.00|3208.50|6741.30|11460.21|160.42|0.00|3208.50|3368.92|-3532.80| +2452159|43434|17971|36240|1377962|1637|1759|8|141|865|25|43.06|49.94|37.45|0.00|936.25|1076.50|1248.50|65.53|0.00|936.25|1001.78|-140.25| +2452159|43434|14443|36240|1377962|1637|1759|8|96|865|26|60.56|78.72|10.23|0.00|265.98|1574.56|2046.72|0.00|0.00|265.98|265.98|-1308.58| +2451829|61430|1205|25957|1551610|6510|23763|4|15|866|78|4.52|8.90|4.62|0.00|360.36|352.56|694.20|14.41|0.00|360.36|374.77|7.80| +2451829|61430|13493|25957|1551610|6510|23763|4|122|866|37|89.28|126.77|92.54|0.00|3423.98|3303.36|4690.49|273.91|0.00|3423.98|3697.89|120.62| +2451829|61430|8483|25957|1551610|6510|23763|4|33|866|51|43.14|56.94|38.71|0.00|1974.21|2200.14|2903.94|19.74|0.00|1974.21|1993.95|-225.93| +2451829|61430|2113|25957|1551610|6510|23763|4|31|866|99|90.25|112.81|38.35|0.00|3796.65|8934.75|11168.19|303.73|0.00|3796.65|4100.38|-5138.10| +|61430|13616|25957|||23763|4|88|866|||120.53|||2566.65|||153.99||2566.65|2720.64|-3417.94| +2451829|61430|4358|25957|1551610|6510|23763|4|273|866|84|69.54|124.47|12.44|0.00|1044.96|5841.36|10455.48|73.14|0.00|1044.96|1118.10|-4796.40| +||13921|25957|1551610||23763|||866||25.30||8.33|||177.10|324.03|||||| +2451829|61430|17531|25957|1551610|6510|23763|4|119|866|21|78.83|115.09|63.29|0.00|1329.09|1655.43|2416.89|66.45|0.00|1329.09|1395.54|-326.34| +2451829|61430|3553|25957|1551610|6510|23763|4|126|866|73|38.67|75.79|6.06|0.00|442.38|2822.91|5532.67|17.69|0.00|442.38|460.07|-2380.53| +2451162|31178|326|83535|1365385|835|27085|1|187|867|5|62.74|72.15|5.77|0.00|28.85|313.70|360.75|1.15|0.00|28.85|30.00|-284.85| +2451162|31178|5492|83535|1365385|835|27085|1|97|867|79|48.93|93.45|73.82|0.00|5831.78|3865.47|7382.55|233.27|0.00|5831.78|6065.05|1966.31| +2451162|31178|9620|83535|1365385|835|27085|1|283|867|78|35.19|51.02|18.36|0.00|1432.08|2744.82|3979.56|14.32|0.00|1432.08|1446.40|-1312.74| +2451162|31178|8653|83535|1365385|835|27085|1|128|867|89|63.95|92.08|51.56|0.00|4588.84|5691.55|8195.12|137.66|0.00|4588.84|4726.50|-1102.71| +2451162|31178|13208|83535|1365385|835|27085|1|283|867|56|11.89|17.35|12.31|0.00|689.36|665.84|971.60|48.25|0.00|689.36|737.61|23.52| +2451162|31178|3175|83535|1365385|835|27085|1|222|867|53|23.15|38.42|34.19|833.55|1812.07|1226.95|2036.26|39.14|833.55|978.52|1017.66|-248.43| +2451162|31178|10069|83535|1365385|835|27085|1|40|867|99|88.47|94.66|85.19|0.00|8433.81|8758.53|9371.34|590.36|0.00|8433.81|9024.17|-324.72| +|31178|3754||1365385||27085||277|867|||107.30||||1129.50|1931.40|25.10|||1280.42|| +2451510|34127|17101|32054|294846|1638|35983|8|234|868|22|40.54|46.62|25.17|0.00|553.74|891.88|1025.64|38.76|0.00|553.74|592.50|-338.14| +2451510|34127|3056|32054|294846|1638|35983|8|106|868|43|90.40|105.76|63.45|0.00|2728.35|3887.20|4547.68|54.56|0.00|2728.35|2782.91|-1158.85| +2451510|34127|9301|32054|294846|1638|35983|8|27|868|71|52.34|71.70|24.37|0.00|1730.27|3716.14|5090.70|86.51|0.00|1730.27|1816.78|-1985.87| +2451510|34127|3476|32054|294846|1638|35983|8|212|868|72|18.75|29.25|9.65|0.00|694.80|1350.00|2106.00|55.58|0.00|694.80|750.38|-655.20| +2451510|34127|7696|32054|294846|1638|35983|8|286|868|44|75.33|106.96|71.66|0.00|3153.04|3314.52|4706.24|189.18|0.00|3153.04|3342.22|-161.48| +2451510|34127|4190|32054|294846|1638|35983|8|64|868|87|50.06|66.57|35.28|0.00|3069.36|4355.22|5791.59|245.54|0.00|3069.36|3314.90|-1285.86| +2451510|34127|386|32054|294846|1638|35983|8|79|868|45|61.94|120.78|94.20|0.00|4239.00|2787.30|5435.10|169.56|0.00|4239.00|4408.56|1451.70| +2451510|34127|8410|32054|294846|1638|35983|8|239|868|84|28.29|53.18|48.39|0.00|4064.76|2376.36|4467.12|243.88|0.00|4064.76|4308.64|1688.40| +2451510|34127|7096|32054|294846|1638|35983|8|184|868|69|4.48|4.52|4.42|0.00|304.98|309.12|311.88|15.24|0.00|304.98|320.22|-4.14| +2451510|34127|1598|32054|294846|1638|35983|8|125|868|59|10.82|18.06|6.86|0.00|404.74|638.38|1065.54|32.37|0.00|404.74|437.11|-233.64| +2451510||8078|||1638|||176|868|||84.90|||||6622.20|||6555.90||1215.24| +2451510|34127|4957|32054|294846|1638|35983|8|293|868|66|5.13|9.18|7.61|0.00|502.26|338.58|605.88|30.13|0.00|502.26|532.39|163.68| +2451063|33020|4633|54263|1859770|2877|2831|2|76|869|32|90.02|141.33|35.33|983.58|1130.56|2880.64|4522.56|5.87|983.58|146.98|152.85|-2733.66| +2451063|33020|12842|54263|1859770|2877|2831|2|201|869|8|50.29|50.29|43.75|0.00|350.00|402.32|402.32|21.00|0.00|350.00|371.00|-52.32| +2451063|33020|5719|54263|1859770|2877|2831|2|219|869|75|40.95|70.02|62.31|0.00|4673.25|3071.25|5251.50|280.39|0.00|4673.25|4953.64|1602.00| +2451063|33020|8834|54263|1859770|2877|2831|2|53|869|30|15.55|22.23|17.33|213.15|519.90|466.50|666.90|15.33|213.15|306.75|322.08|-159.75| +2451063|33020|14317|54263|1859770|2877|2831|2|158|869|62|45.12|60.00|28.80|0.00|1785.60|2797.44|3720.00|89.28|0.00|1785.60|1874.88|-1011.84| +2451063|33020|3836|54263|1859770|2877|2831|2|209|869|95|88.57|89.45|30.41|895.57|2888.95|8414.15|8497.75|139.53|895.57|1993.38|2132.91|-6420.77| +2451063|33020|5041|54263|1859770|2877|2831|2|121|869|21|61.14|84.37|68.33|0.00|1434.93|1283.94|1771.77|71.74|0.00|1434.93|1506.67|150.99| +2451063|33020|4534|54263|1859770|2877|2831|2|282|869|84|77.16|152.00|117.04|0.00|9831.36|6481.44|12768.00|786.50|0.00|9831.36|10617.86|3349.92| +2451063|33020|4594|54263|1859770|2877|2831|2|239|869|64|54.48|84.98|71.38|0.00|4568.32|3486.72|5438.72|411.14|0.00|4568.32|4979.46|1081.60| +2451063|33020|11546|54263|1859770|2877|2831|2|243|869|14|30.94|60.33|27.75|27.19|388.50|433.16|844.62|10.83|27.19|361.31|372.14|-71.85| +2451063|33020|12236|54263|1859770|2877|2831|2|2|869|8|53.65|98.71|11.84|0.00|94.72|429.20|789.68|8.52|0.00|94.72|103.24|-334.48| +2451063|33020|2671|54263|1859770|2877|2831|2|231|869|42|31.37|62.74|52.70|0.00|2213.40|1317.54|2635.08|44.26|0.00|2213.40|2257.66|895.86| +2451063|33020|15025|54263|1859770|2877|2831|2|179|869|92|77.07|122.54|45.33|0.00|4170.36|7090.44|11273.68|125.11|0.00|4170.36|4295.47|-2920.08| +2452527|75187|1393|88247|932486|6325|36964|4|47|870|95|47.98|69.09|37.99|1407.52|3609.05|4558.10|6563.55|154.10|1407.52|2201.53|2355.63|-2356.57| +2452527|75187|15174|88247|932486|6325|36964|4|88|870|41|98.08|99.06|26.74|0.00|1096.34|4021.28|4061.46|0.00|0.00|1096.34|1096.34|-2924.94| +2452527|75187|7278|88247|932486|6325|36964|4|275|870|50|30.79|57.88|43.41|0.00|2170.50|1539.50|2894.00|21.70|0.00|2170.50|2192.20|631.00| +2452527|75187|16230|88247|932486|6325|36964|4|157|870|27|77.48|88.32|58.29|188.85|1573.83|2091.96|2384.64|41.54|188.85|1384.98|1426.52|-706.98| +2452527|75187|6711|88247|932486|6325|36964|4|50|870|84|19.33|36.72|24.23|0.00|2035.32|1623.72|3084.48|101.76|0.00|2035.32|2137.08|411.60| +2452527|75187|12837|88247|932486|6325|36964|4|128|870|62|89.54|145.95|137.19|0.00|8505.78|5551.48|9048.90|340.23|0.00|8505.78|8846.01|2954.30| +2452527|75187|8461|88247|932486|6325|36964|4|300|870|86|15.59|26.34|2.89|0.00|248.54|1340.74|2265.24|14.91|0.00|248.54|263.45|-1092.20| +2452527|75187|16536|88247|932486|6325|36964|4|285|870|80|91.51|103.40|5.17|0.00|413.60|7320.80|8272.00|16.54|0.00|413.60|430.14|-6907.20| +2452527|75187|16099|88247|932486|6325|36964|4|252|870|10|2.38|3.11|0.18|0.70|1.80|23.80|31.10|0.05|0.70|1.10|1.15|-22.70| +2452527|75187|5781|88247|932486|6325|36964|4|21|870|47|57.87|81.01|12.15|0.00|571.05|2719.89|3807.47|17.13|0.00|571.05|588.18|-2148.84| +2452527|75187|3651||932486|6325|36964|4||870||51.57|67.55||0.00|2721.55|4074.03|5336.45|190.50|0.00|||| +2452527|75187|10572|88247|932486|6325|36964|4|166|870|22|44.00|51.48|0.00|0.00|0.00|968.00|1132.56|0.00|0.00|0.00|0.00|-968.00| +2451607|59331|7742|72445|1777163|2681|45399|2|11|871|60|25.17|47.57|10.46|326.35|627.60|1510.20|2854.20|15.06|326.35|301.25|316.31|-1208.95| +2451607|59331|6131|72445|1777163|2681|45399|2|77|871|28|1.75|3.29|1.57|42.64|43.96|49.00|92.12|0.06|42.64|1.32|1.38|-47.68| +2451607|59331|4400|72445|1777163|2681|45399|2|39|871|82|24.72|24.72|15.32|0.00|1256.24|2027.04|2027.04|113.06|0.00|1256.24|1369.30|-770.80| +2451607|59331|13499|72445|1777163|2681|45399|2|26|871|52|71.12|81.78|77.69|0.00|4039.88|3698.24|4252.56|201.99|0.00|4039.88|4241.87|341.64| +2451607|59331|15163|72445|1777163|2681|45399|2|201|871|66|31.22|43.70|4.37|0.00|288.42|2060.52|2884.20|8.65|0.00|288.42|297.07|-1772.10| +2451607|59331|11327|72445|1777163|2681|45399|2|16|871|36|19.40|30.84|18.50|0.00|666.00|698.40|1110.24|39.96|0.00|666.00|705.96|-32.40| +2451607|59331|2653|72445|1777163|2681|45399|2|285|871|55|59.75|112.92|15.80|278.08|869.00|3286.25|6210.60|11.81|278.08|590.92|602.73|-2695.33| +2451607|59331|12373|72445|1777163|2681|45399|2|224|871|82|49.15|95.84|67.08|0.00|5500.56|4030.30|7858.88|385.03|0.00|5500.56|5885.59|1470.26| +2451288|72418|17222|64138|1261606|5474|17742|4|267|872|68|99.71|99.71|89.73|0.00|6101.64|6780.28|6780.28|488.13|0.00|6101.64|6589.77|-678.64| +2451288|72418|4376|64138|1261606|5474|17742|4|151|872|13|87.20|124.69|36.16|0.00|470.08|1133.60|1620.97|14.10|0.00|470.08|484.18|-663.52| +2451288|72418|884|64138|1261606|5474|17742|4|121|872|50|71.48|80.77|16.96|0.00|848.00|3574.00|4038.50|16.96|0.00|848.00|864.96|-2726.00| +2451288|72418|4760|64138|1261606|5474|17742|4|156|872|50|89.35|107.22|47.17|1886.80|2358.50|4467.50|5361.00|28.30|1886.80|471.70|500.00|-3995.80| +2451288|72418|17372|64138|1261606|5474|17742|4|102|872|33|65.05|70.25|35.82|626.49|1182.06|2146.65|2318.25|22.22|626.49|555.57|577.79|-1591.08| +2451288|72418|17251|64138|1261606|5474|17742|4|95|872|77|59.67|75.18|0.00|0.00|0.00|4594.59|5788.86|0.00|0.00|0.00|0.00|-4594.59| +2451288|72418|13687|64138|1261606|5474|17742|4|199|872|98|93.90|140.85|60.56|0.00|5934.88|9202.20|13803.30|118.69|0.00|5934.88|6053.57|-3267.32| +2451288|72418|13840|64138|1261606|5474|17742|4|142|872|63|56.50|81.36|29.28|0.00|1844.64|3559.50|5125.68|129.12|0.00|1844.64|1973.76|-1714.86| +2451288|72418|14002|64138|1261606|5474|17742|4|8|872|16|49.97|99.44|76.56|0.00|1224.96|799.52|1591.04|61.24|0.00|1224.96|1286.20|425.44| +2451026|44977|7603|70800|922349|3051|33400|1|203|873|31|27.38|40.52|10.53|0.00|326.43|848.78|1256.12|9.79|0.00|326.43|336.22|-522.35| +2451026|44977|6310|70800|922349|3051|33400|1|94|873|65|26.39|50.14|48.63|0.00|3160.95|1715.35|3259.10|31.60|0.00|3160.95|3192.55|1445.60| +2451026|44977|14788|70800|922349||33400|1||873|||||1561.23||||28.05|1561.23||729.48|| +2451026|44977|64|70800|922349|3051|33400|1|215|873|47|78.55|80.12|37.65|0.00|1769.55|3691.85|3765.64|70.78|0.00|1769.55|1840.33|-1922.30| +2451026|44977|568|70800|922349|3051|33400|1|255|873|48|39.40|61.85|42.67|1413.23|2048.16|1891.20|2968.80|50.79|1413.23|634.93|685.72|-1256.27| +2451026|44977|12998|70800|922349|3051|33400|1|93|873|93|13.96|19.54|2.34|0.00|217.62|1298.28|1817.22|15.23|0.00|217.62|232.85|-1080.66| +2451026|44977|11050|70800|922349|3051|33400|1|234|873|42|56.79|63.03|29.62|0.00|1244.04|2385.18|2647.26|111.96|0.00|1244.04|1356.00|-1141.14| +2451026|44977|2308|70800|922349|3051|33400|1|176|873|26|92.12|172.26|36.17|0.00|940.42|2395.12|4478.76|65.82|0.00|940.42|1006.24|-1454.70| +2451790|56228|12817|22119|1045269|419|47922|10|237|874|63|55.70|104.15|35.41|0.00|2230.83|3509.10|6561.45|156.15|0.00|2230.83|2386.98|-1278.27| +2451790|56228|15041|22119|1045269|419|47922|10|189|874|7|17.23|21.19|2.33|0.00|16.31|120.61|148.33|0.97|0.00|16.31|17.28|-104.30| +2451790||17141|||||||874|||155.20|130.36|||||||10428.80||3568.80| +2451790|56228|1915|22119|1045269|419|47922|10|16|874|39|78.48|109.87|45.04|0.00|1756.56|3060.72|4284.93|17.56|0.00|1756.56|1774.12|-1304.16| +2451790|56228|16705|22119|1045269|419|47922|10|294|874|46|99.22|177.60|58.60|0.00|2695.60|4564.12|8169.60|215.64|0.00|2695.60|2911.24|-1868.52| +2451790|56228|5948|22119|1045269|419|47922|10|162|874|68|27.79|37.23|9.67|420.83|657.56|1889.72|2531.64|2.36|420.83|236.73|239.09|-1652.99| +2451790|56228|6781|22119|1045269|419|47922|10|41|874|11|78.27|90.01|74.70|805.26|821.70|860.97|990.11|1.15|805.26|16.44|17.59|-844.53| +2451790|56228|3758|22119|1045269|419|47922|10|30|874|73|20.45|29.44|28.26|1320.30|2062.98|1492.85|2149.12|37.13|1320.30|742.68|779.81|-750.17| +2451790|56228|10610|22119|1045269|419|47922|10|44|874|79|84.17|93.42|7.47|0.00|590.13|6649.43|7380.18|41.30|0.00|590.13|631.43|-6059.30| +2451076|36480|9698|55150|38248|5793|31338|8|40|875|9|77.32|90.46|1.80|0.00|16.20|695.88|814.14|1.45|0.00|16.20|17.65|-679.68| +2451076|36480|7054|55150|38248|5793|31338|8|254|875|62|59.39|99.77|20.95|0.00|1298.90|3682.18|6185.74|38.96|0.00|1298.90|1337.86|-2383.28| +2451076|36480|9146|55150|38248|5793|31338|8|56|875|58|68.70|79.00|36.34|84.30|2107.72|3984.60|4582.00|121.40|84.30|2023.42|2144.82|-1961.18| +2451076|36480|11278|55150|38248|5793|31338|8|219|875|10|16.92|22.67|10.65|63.90|106.50|169.20|226.70|3.40|63.90|42.60|46.00|-126.60| +2451076|36480|7246|55150|38248|5793|31338|8|130|875|86|45.03|52.68|51.09|0.00|4393.74|3872.58|4530.48|131.81|0.00|4393.74|4525.55|521.16| +|36480|11521||38248|||8|41|875|||38.19|8.40|||173.61||0.00||||| +2451076|36480|11971|55150|38248|5793|31338|8|274|875|17|20.76|31.14|11.52|0.00|195.84|352.92|529.38|15.66|0.00|195.84|211.50|-157.08| +2451076|36480|17227|55150|38248|5793|31338|8|65|875|96|11.79|15.09|10.26|0.00|984.96|1131.84|1448.64|78.79|0.00|984.96|1063.75|-146.88| +2451076|36480|13880|55150|38248|5793|31338|8|261|875|5|50.93|62.64|25.05|102.70|125.25|254.65|313.20|1.57|102.70|22.55|24.12|-232.10| +2451076|36480|9811|55150|38248|5793|31338|8|145|875|8|42.31|42.31|27.50|0.00|220.00|338.48|338.48|17.60|0.00|220.00|237.60|-118.48| +2452444|52437|6489|49837|43238|5439|39240|1|255|876|69|33.86|51.46|32.41|0.00|2236.29|2336.34|3550.74|0.00|0.00|2236.29|2236.29|-100.05| +2452444|52437|14349|49837|43238|5439|39240|1|229|876|56|96.82|122.96|46.72|0.00|2616.32|5421.92|6885.76|235.46|0.00|2616.32|2851.78|-2805.60| +2452444|52437|37|49837||5439|39240||16|876||||||656.53|2958.12||4.59||459.58|464.17|-2498.54| +2452444|52437|5055|49837|43238||39240|1|158|876|29|31.09|54.09|20.55|||901.61||||470.81|470.81|-430.80| +2452444|52437|11016|49837|43238|5439|39240|1|141|876|83|20.76|23.66|11.59|0.00|961.97|1723.08|1963.78|67.33|0.00|961.97|1029.30|-761.11| +2452444|52437|3445|49837|43238|5439|39240|1|86|876|66|47.11|76.78|50.67|0.00|3344.22|3109.26|5067.48|33.44|0.00|3344.22|3377.66|234.96| +2452444|52437|13933|49837|43238|5439|39240|1|166|876|9|44.11|52.93|50.28|194.58|452.52|396.99|476.37|20.63|194.58|257.94|278.57|-139.05| +2452444|52437|11589|49837|43238|5439|39240|1|208|876|72|48.47|81.91|81.91|0.00|5897.52|3489.84|5897.52|58.97|0.00|5897.52|5956.49|2407.68| +2452444|52437|15171|49837|43238|5439|39240|1|245|876|52|36.80|61.82|38.32|0.00|1992.64|1913.60|3214.64|159.41|0.00|1992.64|2152.05|79.04| +2451851|45994|16844|26101|1057122|3343|21964|2|144|877|70|3.22|5.28|2.00|0.00|140.00|225.40|369.60|9.80|0.00|140.00|149.80|-85.40| +2451851|45994|16757|26101|1057122|3343|21964|2|227|877|89|56.95|60.36|51.90|0.00|4619.10|5068.55|5372.04|0.00|0.00|4619.10|4619.10|-449.45| +2451851|45994|17627|26101|1057122|3343|21964|2|184|877|52|66.07|105.71|59.19|0.00|3077.88|3435.64|5496.92|123.11|0.00|3077.88|3200.99|-357.76| +||2635|26101||3343||2||877|60|87.61||72.27|4162.75||5256.60|8673.00||4162.75||173.45|-5083.15| +2451851|45994|2426|26101|1057122|3343|21964|2|36|877|65|82.11|101.81|76.35|248.13|4962.75|5337.15|6617.65|141.43|248.13|4714.62|4856.05|-622.53| +|45994|15146|26101||3343|21964|2|67|877|29||144.68|||3776.09|||||3776.09||| +2451851|45994|12320|26101|1057122|3343|21964|2|14|877|95|26.27|28.37|15.60|0.00|1482.00|2495.65|2695.15|29.64|0.00|1482.00|1511.64|-1013.65| +2451851|45994|7831|26101|1057122|3343|21964|2|232|877|7|58.12|75.55|0.75|0.00|5.25|406.84|528.85|0.36|0.00|5.25|5.61|-401.59| +2451851|45994|12989|26101|1057122|3343|21964|2|136|877|35|56.23|105.15|82.01|0.00|2870.35|1968.05|3680.25|143.51|0.00|2870.35|3013.86|902.30| +2451851|45994|12791|26101|1057122|3343|21964|2|292|877|3|12.95|15.41|8.47|0.00|25.41|38.85|46.23|2.28|0.00|25.41|27.69|-13.44| +2451851|45994|5515|26101|1057122|3343|21964|2|199|877|56|21.82|23.78|21.87|379.66|1224.72|1221.92|1331.68|76.05|379.66|845.06|921.11|-376.86| +2451851|45994|7991|26101|1057122|3343|21964|2|293|877|62|42.92|60.08|29.43|0.00|1824.66|2661.04|3724.96|18.24|0.00|1824.66|1842.90|-836.38| +2450958|38391|7982|10116|804534|3399|49817|4|95|878|42|90.37|164.47|0.00|0.00|0.00|3795.54|6907.74|0.00|0.00|0.00|0.00|-3795.54| +2450958|38391|11716|10116|804534|3399|49817|4|36|878|27|86.37|157.19|48.72|0.00|1315.44|2331.99|4244.13|0.00|0.00|1315.44|1315.44|-1016.55| +2450958|38391|6056|10116|804534|3399|49817|4|7|878|66|51.01|77.53|37.98|0.00|2506.68|3366.66|5116.98|200.53|0.00|2506.68|2707.21|-859.98| +2450958|38391|14593|10116|804534|3399|49817|4|142|878|50|92.43|120.15|30.03|0.00|1501.50|4621.50|6007.50|0.00|0.00|1501.50|1501.50|-3120.00| +2450958|38391|6439|10116|804534|3399|49817|4|56|878|84|93.81|141.65|128.90|0.00|10827.60|7880.04|11898.60|0.00|0.00|10827.60|10827.60|2947.56| +2450958|38391|10814|10116|804534|3399|49817|4|53|878|64|6.39|8.88|7.28|0.00|465.92|408.96|568.32|4.65|0.00|465.92|470.57|56.96| +2450958|38391|14179|10116|804534|3399|49817|4|215|878|5|97.70|144.59|117.11|0.00|585.55|488.50|722.95|29.27|0.00|585.55|614.82|97.05| +2450958|38391|16867|10116|804534|3399|49817|4|191|878|1|19.13|26.97|12.40|0.00|12.40|19.13|26.97|0.24|0.00|12.40|12.64|-6.73| +2450958|38391|7796|10116|804534|3399|49817|4|6|878|85|7.32|9.22|6.91|0.00|587.35|622.20|783.70|17.62|0.00|587.35|604.97|-34.85| +2450958|38391|9902|10116|804534|3399|49817|4|249|878|35|58.65|80.35|77.93|0.00|2727.55|2052.75|2812.25|163.65|0.00|2727.55|2891.20|674.80| +2450958|38391|13777|10116|804534|3399|49817|4|90|878|65|75.09|139.66|138.26|0.00|8986.90|4880.85|9077.90|0.00|0.00|8986.90|8986.90|4106.05| +2450958|38391|15361|10116|804534|3399|49817|4|88|878|58|87.91|170.54|126.19|0.00|7319.02|5098.78|9891.32|439.14|0.00|7319.02|7758.16|2220.24| +2450958||3842|||||4|72|878|96|23.53|41.88|2.51|0.00||||21.68|0.00|||-2017.92| +2451394|35754|1022|52904|1022693|6529|2365|1|84|879|9|99.25|111.16|58.91|0.00|530.19|893.25|1000.44|10.60|0.00|530.19|540.79|-363.06| +2451394|35754|9142|52904|1022693|6529|2365|1|30|879|96|69.65|133.72|106.97|0.00|10269.12|6686.40|12837.12|616.14|0.00|10269.12|10885.26|3582.72| +2451394|35754|1976|52904|1022693|6529|2365|1|249|879|57|16.21|29.66|8.89|0.00|506.73|923.97|1690.62|25.33|0.00|506.73|532.06|-417.24| +2451394|35754|5024|52904|1022693|6529|2365|1|30|879|30|74.71|137.46|26.11|0.00|783.30|2241.30|4123.80|15.66|0.00|783.30|798.96|-1458.00| +2451394|35754|6664|52904|1022693|6529|2365|1|259|879|28|58.84|101.20|28.33|0.00|793.24|1647.52|2833.60|23.79|0.00|793.24|817.03|-854.28| +2451394|35754|10609|52904|1022693|6529|2365|1|222|879|33|39.79|77.98|38.99|12.86|1286.67|1313.07|2573.34|0.00|12.86|1273.81|1273.81|-39.26| +2451394|35754|1285|52904|1022693|6529|2365|1|2|879|55|42.54|62.10|44.09|1406.47|2424.95|2339.70|3415.50|10.18|1406.47|1018.48|1028.66|-1321.22| +2451394|35754|1166|52904|1022693|6529|2365|1|10|879|20|10.90|10.90|1.09|0.00|21.80|218.00|218.00|0.00|0.00|21.80|21.80|-196.20| +2451394|35754|1957|52904|1022693|6529|2365|1|40|879|73|62.38|69.86|53.09|0.00|3875.57|4553.74|5099.78|193.77|0.00|3875.57|4069.34|-678.17| +2451394|35754|7808|52904|1022693|6529|2365|1|235|879|56|37.86|56.41|38.35|0.00|2147.60|2120.16|3158.96|193.28|0.00|2147.60|2340.88|27.44| +2451775|68349|17845|42388|784559|683|16454|10|256|880|10|11.65|19.92|18.12|0.00|181.20|116.50|199.20|5.43|0.00|181.20|186.63|64.70| +2451775|68349|15326|42388|784559|683|16454|10|292|880|50|44.77|89.09|50.78|0.00|2539.00|2238.50|4454.50|101.56|0.00|2539.00|2640.56|300.50| +2451775|68349|7937|42388|784559|683|16454|10|247|880|27|7.63|7.63|0.00|0.00|0.00|206.01|206.01|0.00|0.00|0.00|0.00|-206.01| +2451775|68349|9523|42388|784559|683|16454|10|10|880|31|6.34|12.17|7.30|24.89|226.30|196.54|377.27|8.05|24.89|201.41|209.46|4.87| +2451775|68349|16787|42388|784559|683|16454|10|54|880|72|84.10|106.80|64.08|2353.01|4613.76|6055.20|7689.60|135.64|2353.01|2260.75|2396.39|-3794.45| +2451775|68349|4850|42388|784559|683|16454|10|147|880|93|7.76|11.25|9.78|0.00|909.54|721.68|1046.25|63.66|0.00|909.54|973.20|187.86| +2451775|68349|3767|42388|784559|683|16454|10|208|880|58|29.86|31.05|23.90|0.00|1386.20|1731.88|1800.90|27.72|0.00|1386.20|1413.92|-345.68| +2451775|68349|476|42388|784559|683|16454|10|129|880|66|85.79|102.94|53.52|0.00|3532.32|5662.14|6794.04|282.58|0.00|3532.32|3814.90|-2129.82| +2452223|45929|14855|48204|1360838|1025|41659|7|249|881|42|36.29|63.87|26.82|0.00|1126.44|1524.18|2682.54|101.37|0.00|1126.44|1227.81|-397.74| +2452223|45929|13609|48204|1360838|1025|41659|7|1|881|28|68.79|90.11|63.07|0.00|1765.96|1926.12|2523.08|17.65|0.00|1765.96|1783.61|-160.16| +2452223|45929|4639|48204|1360838|1025|41659|7|152|881|8|28.84|54.50|10.35|0.00|82.80|230.72|436.00|0.00|0.00|82.80|82.80|-147.92| +2452223|45929|12229|48204|1360838|1025|41659|7|22|881|89|90.97|165.56|155.62|0.00|13850.18|8096.33|14734.84|277.00|0.00|13850.18|14127.18|5753.85| +2452223|45929|8761|48204|1360838|1025|41659|7|288|881|29|22.59|29.81|25.63|683.80|743.27|655.11|864.49|4.75|683.80|59.47|64.22|-595.64| +2452223|45929|12567|48204|1360838|1025|41659|7|16|881|73|50.31|63.39|6.97|0.00|508.81|3672.63|4627.47|20.35|0.00|508.81|529.16|-3163.82| +2452223|45929|8983|48204|1360838|1025|41659|7|157|881|94|50.94|80.48|45.87|0.00|4311.78|4788.36|7565.12|258.70|0.00|4311.78|4570.48|-476.58| +2452223|45929|10001|48204|1360838|1025|41659|7|158|881|36|63.21|123.25|29.58|0.00|1064.88|2275.56|4437.00|63.89|0.00|1064.88|1128.77|-1210.68| +2452223|45929|12427|48204|1360838|1025|41659|7|13|881|65|15.54|17.71|8.67|0.00|563.55|1010.10|1151.15|11.27|0.00|563.55|574.82|-446.55| +2452223|45929|6713|48204|1360838|1025|41659|7|78|881|5|57.30|67.04|13.40|0.00|67.00|286.50|335.20|3.35|0.00|67.00|70.35|-219.50| +2452223|45929|7365|48204|1360838|1025|41659|7|167|881|87|35.54|40.16|19.27|0.00|1676.49|3091.98|3493.92|50.29|0.00|1676.49|1726.78|-1415.49| +2452223|45929|10387|48204|1360838|1025|41659|7|188|881|97|35.25|70.14|51.20|0.00|4966.40|3419.25|6803.58|347.64|0.00|4966.40|5314.04|1547.15| +2452223|45929|1225|48204|1360838|1025|41659|7|85|881|26|78.86|133.27|74.63|1726.93|1940.38|2050.36|3465.02|12.80|1726.93|213.45|226.25|-1836.91| +2452223|45929|7725|48204|1360838|1025|41659|7|296|881|12|57.11|91.94|88.26|0.00|1059.12|685.32|1103.28|42.36|0.00|1059.12|1101.48|373.80| +2452223|45929|12319|48204|1360838|1025|41659|7|209|881|73|93.26|162.27|150.91|0.00|11016.43|6807.98|11845.71|771.15|0.00|11016.43|11787.58|4208.45| +2452223|45929|281|48204|1360838|1025|41659|7|131|881|8|5.67|6.35|5.71|0.00|45.68|45.36|50.80|3.65|0.00|45.68|49.33|0.32| +2451521|71034|10189|42449|326471|1246|30021|10|293|882|19|59.10|60.28|18.08|0.00|343.52|1122.90|1145.32|0.00|0.00|343.52|343.52|-779.38| +2451521|71034|146|42449|326471|1246|30021|10|71|882|76|95.16|163.67|119.47|0.00|9079.72|7232.16|12438.92|272.39|0.00|9079.72|9352.11|1847.56| +2451521|71034|12104|42449|326471|1246|30021|10|212|882|84|5.46|6.16|1.54|0.00|129.36|458.64|517.44|0.00|0.00|129.36|129.36|-329.28| +2451521|71034|16207|42449|326471|1246|30021|10|14|882|31|52.12|56.81|35.22|0.00|1091.82|1615.72|1761.11|98.26|0.00|1091.82|1190.08|-523.90| +2451521|71034|5530|42449|326471|1246|30021|10|264|882|26|76.24|108.26|62.79|0.00|1632.54|1982.24|2814.76|0.00|0.00|1632.54|1632.54|-349.70| +2451521|71034|15031|42449|326471|1246|30021|10|36|882|40|50.20|74.79|70.30|0.00|2812.00|2008.00|2991.60|84.36|0.00|2812.00|2896.36|804.00| +2451521|71034|15190|42449|326471|1246|30021|10|17|882|49|9.44|14.91|4.02|0.00|196.98|462.56|730.59|11.81|0.00|196.98|208.79|-265.58| +2451521|71034|1580|42449|326471|1246|30021|10|206|882|72|89.02|125.51|84.09|5570.12|6054.48|6409.44|9036.72|14.53|5570.12|484.36|498.89|-5925.08| +2451521|71034|6242|42449|326471|1246|30021|10|150|882|40|23.66|42.11|10.10|0.00|404.00|946.40|1684.40|32.32|0.00|404.00|436.32|-542.40| +2451521|71034|13466|42449|326471|1246|30021|10|282|882|69|81.21|141.30|39.56|0.00|2729.64|5603.49|9749.70|218.37|0.00|2729.64|2948.01|-2873.85| +2451521|71034|9178|42449|326471|1246|30021|10|274|882|78|92.93|173.77|67.77|0.00|5286.06|7248.54|13554.06|317.16|0.00|5286.06|5603.22|-1962.48| +2451521|71034|8149|42449|326471|1246|30021|10|255|882|39|44.56|72.63|8.71|0.00|339.69|1737.84|2832.57|0.00|0.00|339.69|339.69|-1398.15| +2451521|71034|3259|42449|||30021|10|28|882||89.52|149.49|70.26|0.00||6534.96|10912.77|51.28|0.00|5128.98|5180.26|| +2451521|71034|4412|42449|326471|1246|30021|10|36|882|9|8.38|9.88|1.67|13.52|15.03|75.42|88.92|0.01|13.52|1.51|1.52|-73.91| +2451521|71034|562|42449|326471|1246|30021|10|152|882|45|32.44|42.49|11.89|0.00|535.05|1459.80|1912.05|10.70|0.00|535.05|545.75|-924.75| +2452112|52021|10645|65118|926520|722|3324|7|19|883|45|97.17|111.74|82.68|0.00|3720.60|4372.65|5028.30|37.20|0.00|3720.60|3757.80|-652.05| +2452112|52021|10225|65118|926520|722|3324|7|5|883|95|97.59|188.34|124.30|10037.22|11808.50|9271.05|17892.30|70.85|10037.22|1771.28|1842.13|-7499.77| +2452112|52021|16311|65118|926520|722|3324|7|291|883|76|21.43|38.14|28.22|2101.82|2144.72|1628.68|2898.64|3.43|2101.82|42.90|46.33|-1585.78| +2452112|52021|6961|65118|926520|722|3324|7|109|883|66|47.35|54.45|42.47|0.00|2803.02|3125.10|3593.70|168.18|0.00|2803.02|2971.20|-322.08| +2452112|52021|4545|65118|926520|722|3324|7|281|883|3|12.22|20.89|14.41|0.00|43.23|36.66|62.67|1.72|0.00|43.23|44.95|6.57| +2452112|52021|5037|||722|3324||8|883||7.22|11.69||0.00|218.40||911.82|10.92|0.00|||-344.76| +2452112|52021|12629|65118|926520|722|3324|7|85|883|13|24.73|39.07|13.28|0.00|172.64|321.49|507.91|12.08|0.00|172.64|184.72|-148.85| +2452112|52021|13813|65118|926520|722|3324|7|285|883|50|97.35|131.42|124.84|2496.80|6242.00|4867.50|6571.00|37.45|2496.80|3745.20|3782.65|-1122.30| +2452112|52021|12711|65118|926520|722|3324|7|54|883|71|94.94|165.19|61.12|0.00|4339.52|6740.74|11728.49|0.00|0.00|4339.52|4339.52|-2401.22| +2452112|52021|719|65118|926520|722|3324|7|175|883|27|8.25|8.66|6.23|65.60|168.21|222.75|233.82|6.15|65.60|102.61|108.76|-120.14| +2452406|64898|7305|65610|811674|2538|11402|7|224|884|90|82.83|122.58|110.32|0.00|9928.80|7454.70|11032.20|99.28|0.00|9928.80|10028.08|2474.10| +2452406|64898|11245|65610|811674|2538|11402|7|149|884|6|55.90|109.00|81.75|0.00|490.50|335.40|654.00|9.81|0.00|490.50|500.31|155.10| +2452406|64898|7297|65610|811674|2538|11402|7|230|884|1|30.63|52.37|40.84|0.00|40.84|30.63|52.37|3.26|0.00|40.84|44.10|10.21| +2452406|64898|8652|65610|811674|2538|11402|7|276|884|23|60.59|89.06|43.63|0.00|1003.49|1393.57|2048.38|70.24|0.00|1003.49|1073.73|-390.08| +2452406|64898|11679|65610|811674|2538|11402|7|15|884|38|57.25|66.98|44.87|0.00|1705.06|2175.50|2545.24|136.40|0.00|1705.06|1841.46|-470.44| +|64898|14010|65610|811674|2538|11402|7|267|884|88|69.91|128.63|30.87||||11319.44|0.00||2716.56|2716.56|-3435.52| +2452406|64898|12139|65610|811674|2538|11402|7|118|884|87|47.40|47.87|3.35|0.00|291.45|4123.80|4164.69|0.00|0.00|291.45|291.45|-3832.35| +2452406|64898|6871|65610|811674|2538|11402|7|291|884|70|69.15|134.15|9.39|0.00|657.30|4840.50|9390.50|19.71|0.00|657.30|677.01|-4183.20| +2452406|64898|10392|65610|811674|2538|11402|7|8|884|68|27.84|34.24|16.43|0.00|1117.24|1893.12|2328.32|33.51|0.00|1117.24|1150.75|-775.88| +||1464||811674|2538||||884|11||||0.00||||3.47|0.00||61.44|| +2452406|64898|11625|65610|811674|2538|11402|7|273|884|37|14.67|17.89|9.48|0.00|350.76|542.79|661.93|10.52|0.00|350.76|361.28|-192.03| +2452406|64898|14580|65610|811674|2538|11402|7|266|884|92|4.49|7.54|4.52|0.00|415.84|413.08|693.68|4.15|0.00|415.84|419.99|2.76| +2452406|64898|8403|65610|811674|2538|11402|7|46|884|23|88.39|145.84|30.62|0.00|704.26|2032.97|3354.32|63.38|0.00|704.26|767.64|-1328.71| +2452406|64898|14733|65610|811674|2538|11402|7|288|884|8|36.98|37.71|16.59|0.00|132.72|295.84|301.68|11.94|0.00|132.72|144.66|-163.12| +2452406|64898|5713|65610|811674|2538|11402|7|63|884|42|33.80|35.49|20.93|0.00|879.06|1419.60|1490.58|35.16|0.00|879.06|914.22|-540.54| +2450888|65845|7318|85495|1484426|957|36688|4|246|885|52|50.05|84.58|7.61|0.00|395.72|2602.60|4398.16|0.00|0.00|395.72|395.72|-2206.88| +2450888|65845|15091|85495|1484426|957|36688|4|54|885|69|22.74|26.37|13.18|0.00|909.42|1569.06|1819.53|63.65|0.00|909.42|973.07|-659.64| +2450888|65845|12584|85495|1484426|957|36688|4|290|885|25|31.65|32.28|16.14|0.00|403.50|791.25|807.00|12.10|0.00|403.50|415.60|-387.75| +2450888|65845|17458|85495|1484426|957|36688|4|154|885|76|54.91|55.45|31.05|0.00|2359.80|4173.16|4214.20|188.78|0.00|2359.80|2548.58|-1813.36| +2450888|65845|8659|85495|1484426|957|36688|4|262|885|39|44.30|63.34|58.27|1158.99|2272.53|1727.70|2470.26|0.00|1158.99|1113.54|1113.54|-614.16| +2450888|65845|8576|85495|1484426|957|36688|4|234|885|72|72.96|113.08|89.33|0.00|6431.76|5253.12|8141.76|64.31|0.00|6431.76|6496.07|1178.64| +2450888|65845|8440|85495|1484426|957|36688|4|299|885|7|52.54|93.52|29.92|0.00|209.44|367.78|654.64|14.66|0.00|209.44|224.10|-158.34| +2450888|65845|6662|85495|1484426|957|36688|4|49|885|94|26.57|48.88|34.21|0.00|3215.74|2497.58|4594.72|289.41|0.00|3215.74|3505.15|718.16| +2450888|65845|16202|85495|1484426|957|36688|4|12|885|52|58.14|106.39|10.63|0.00|552.76|3023.28|5532.28|44.22|0.00|552.76|596.98|-2470.52| +2450888|65845|13144|85495|1484426|957|36688|4|91|885|41|25.39|43.92|43.04|0.00|1764.64|1040.99|1800.72|70.58|0.00|1764.64|1835.22|723.65| +2452591|64689|5941|17029|1081961|6893|46978|2|18|886|67|98.61|153.83|38.45|0.00|2576.15|6606.87|10306.61|231.85|0.00|2576.15|2808.00|-4030.72| +||8439|||6893|46978|2|132|886|80||76.86||0.00||4841.60|||0.00|2397.60|2565.43|| +2452591|64689|15711|17029|1081961|6893|46978|2|80|886|32|16.31|16.79|16.11|20.62|515.52|521.92|537.28|4.94|20.62|494.90|499.84|-27.02| +2452591|64689|1236|17029|1081961|6893|46978|2|102|886|62|44.79|45.68|26.03|0.00|1613.86|2776.98|2832.16|32.27|0.00|1613.86|1646.13|-1163.12| +2452591|64689|13389|17029|1081961|6893|46978|2|193|886|37|72.88|100.57|41.23|0.00|1525.51|2696.56|3721.09|91.53|0.00|1525.51|1617.04|-1171.05| +2452591|64689|1524|17029|1081961|6893|46978|2|134|886|59|78.91|117.57|35.27|1810.40|2080.93|4655.69|6936.63|2.70|1810.40|270.53|273.23|-4385.16| +2452591|64689|6637|17029|1081961|6893|46978|2|136|886|47|19.87|20.46|3.47|0.00|163.09|933.89|961.62|0.00|0.00|163.09|163.09|-770.80| +2452591|64689|15834|17029|1081961|6893|46978|2|139|886|97|99.64|163.40|132.35|3081.10|12837.95|9665.08|15849.80|682.97|3081.10|9756.85|10439.82|91.77| +2452591|64689|2349|17029|1081961|6893|46978|2|280|886|34|39.72|50.44|13.61|0.00|462.74|1350.48|1714.96|23.13|0.00|462.74|485.87|-887.74| +2452591|64689|4005|17029|1081961|6893|46978|2|269|886|66|54.26|73.79|48.70|0.00|3214.20|3581.16|4870.14|192.85|0.00|3214.20|3407.05|-366.96| +2452591|64689|2665|17029|1081961|6893|46978|2|85|886|87|55.18|61.24|0.61|0.00|53.07|4800.66|5327.88|1.06|0.00|53.07|54.13|-4747.59| +2452591|64689|12879|17029|1081961|6893|46978|2|232|886|99|26.79|45.81|21.98|913.92|2176.02|2652.21|4535.19|37.86|913.92|1262.10|1299.96|-1390.11| +2452591|64689|7752|17029|1081961|6893|46978|2|222|886|67|48.26|62.25|15.56|0.00|1042.52|3233.42|4170.75|72.97|0.00|1042.52|1115.49|-2190.90| +2452591||7647||1081961|||||886||70.80||||||10248.00|||4328.64|4674.93|| +2452591|64689|11301|17029|1081961|6893|46978|2|209|886|38|40.85|78.84|26.80|0.00|1018.40|1552.30|2995.92|20.36|0.00|1018.40|1038.76|-533.90| +2452591|64689|7545|17029|1081961|6893|46978|2|82|886|34|95.80|134.12|83.15|0.00|2827.10|3257.20|4560.08|0.00|0.00|2827.10|2827.10|-430.10| +2451908|71647|5233|87807|304816|6625|6834|1|159|887|5|91.69|106.36|30.84|0.00|154.20|458.45|531.80|4.62|0.00|154.20|158.82|-304.25| +2451908|71647|17173|87807|304816|6625|6834|1|172|887|52|52.00|101.92|28.53|0.00|1483.56|2704.00|5299.84|118.68|0.00|1483.56|1602.24|-1220.44| +2451908|71647|944|87807|304816|6625|6834|1|132|887|86|71.72|94.67|56.80|1611.98|4884.80|6167.92|8141.62|196.36|1611.98|3272.82|3469.18|-2895.10| +2451908|71647|14683|87807|304816|6625|6834|1|225|887|50|87.27|172.79|36.28|1469.34|1814.00|4363.50|8639.50|31.01|1469.34|344.66|375.67|-4018.84| +2451908|71647|11663|87807|304816|6625|6834|1|196|887|30|3.58|5.87|4.16|89.85|124.80|107.40|176.10|1.04|89.85|34.95|35.99|-72.45| +||2551|87807|||6834|1|15|887|36|50.62|90.10||0.00|||3243.60||0.00|129.60|130.89|| +2451908|71647|15779|87807|304816|6625|6834|1|222|887|18|81.13|126.56|84.79|0.00|1526.22|1460.34|2278.08|91.57|0.00|1526.22|1617.79|65.88| +2451908|71647|1021|87807|304816|6625|6834|1|47|887|51|92.30|140.29|28.05|514.99|1430.55|4707.30|7154.79|64.08|514.99|915.56|979.64|-3791.74| +2451908|71647|17228|87807|304816|6625|6834|1|173|887|19|20.74|21.77|6.74|0.00|128.06|394.06|413.63|0.00|0.00|128.06|128.06|-266.00| +2451908|71647|9511|87807|304816|6625|6834|1|203|887|19|20.89|20.89|9.81|0.00|186.39|396.91|396.91|16.77|0.00|186.39|203.16|-210.52| +2451908|71647|16579|87807|304816|6625|6834|1|13|887|80|68.75|127.18|64.86|0.00|5188.80|5500.00|10174.40|363.21|0.00|5188.80|5552.01|-311.20| +2451908|71647|6715|87807|304816|6625|6834|1|79|887|20|16.30|29.17|13.70|0.00|274.00|326.00|583.40|5.48|0.00|274.00|279.48|-52.00| +2452634|41743|337|58364|906655|3299|9143|10|66|888|17|92.82|115.09|11.50|23.46|195.50|1577.94|1956.53|12.04|23.46|172.04|184.08|-1405.90| +2452634|41743|8599|58364|906655|3299|9143|10|57|888|20|80.05|92.05|0.00|0.00|0.00|1601.00|1841.00|0.00|0.00|0.00|0.00|-1601.00| +2452634|41743|7266|58364|906655|3299|9143|10|62|888|59|29.29|51.25|4.61|0.00|271.99|1728.11|3023.75|16.31|0.00|271.99|288.30|-1456.12| +2452634|41743|4209|58364|906655|3299|9143|10|209|888|24|53.53|71.19|42.00|252.00|1008.00|1284.72|1708.56|52.92|252.00|756.00|808.92|-528.72| +2452634|41743|7129|58364|906655|3299|9143|10|135|888|8|65.57|71.47|22.15|0.00|177.20|524.56|571.76|3.54|0.00|177.20|180.74|-347.36| +2452634|41743|2646|58364|906655|3299|9143|10|27|888|51|9.07|17.14|4.62|230.90|235.62|462.57|874.14|0.18|230.90|4.72|4.90|-457.85| +2452634|41743|8833|58364|906655|3299|9143|10|36|888|39|85.48|119.67|59.83|0.00|2333.37|3333.72|4667.13|23.33|0.00|2333.37|2356.70|-1000.35| +2452634||627|58364||||10||888||92.10|||0.00|||160.25||0.00|62.49|68.11|-29.61| +2452634|41743|2571|58364|906655|3299|9143|10|125|888|3|20.65|35.51|8.16|0.00|24.48|61.95|106.53|0.00|0.00|24.48|24.48|-37.47| +2452634|41743|10369|58364|906655|3299|9143|10|69|888|93|19.05|36.00|28.08|0.00|2611.44|1771.65|3348.00|156.68|0.00|2611.44|2768.12|839.79| +2452634|41743|3774|58364|906655|3299|9143|10|239|888|73|54.99|75.33|9.03|0.00|659.19|4014.27|5499.09|0.00|0.00|659.19|659.19|-3355.08| +||11558|77107|1374922||30952|||889|92|3.72|4.61|1.42|0.00||342.24|||0.00|130.64|139.78|| +2451432|33851|15721|77107|1374922|4737|30952|8|41|889|66|98.21|150.26|76.63|0.00|5057.58|6481.86|9917.16|303.45|0.00|5057.58|5361.03|-1424.28| +2451432|33851|4574|77107|1374922|4737|30952|8|122|889|57|13.76|19.95|3.99|0.00|227.43|784.32|1137.15|6.82|0.00|227.43|234.25|-556.89| +2451432|33851|6352|77107|1374922|4737|30952|8|60|889|31|90.97|116.44|116.44|0.00|3609.64|2820.07|3609.64|36.09|0.00|3609.64|3645.73|789.57| +2451432|33851|6298|77107|1374922|4737|30952|8|193|889|71|76.68|82.04|1.64|0.00|116.44|5444.28|5824.84|9.31|0.00|116.44|125.75|-5327.84| +2451432|33851|10903|77107|1374922|4737|30952|8|123|889|88|54.17|108.34|76.92|0.00|6768.96|4766.96|9533.92|0.00|0.00|6768.96|6768.96|2002.00| +2451432|33851|6784|77107|1374922|4737|30952|8|55|889|41|96.23|117.40|59.87|196.37|2454.67|3945.43|4813.40|67.74|196.37|2258.30|2326.04|-1687.13| +2451432|33851|2578|77107|1374922|4737|30952|8|98|889|18|23.56|36.51|3.28|0.00|59.04|424.08|657.18|2.36|0.00|59.04|61.40|-365.04| +2452541|30269|16117|16848|293474|2453|4232|10|77|890|71|8.44|9.36|2.62|0.00|186.02|599.24|664.56|3.72|0.00|186.02|189.74|-413.22| +2452541|30269|5586|16848|293474|2453|4232|10|146|890|46|54.99|58.83|38.82|0.00|1785.72|2529.54|2706.18|17.85|0.00|1785.72|1803.57|-743.82| +2452541|30269|3780|16848|293474|2453|4232|10|52|890|32|59.31|114.46|65.24|772.44|2087.68|1897.92|3662.72|13.15|772.44|1315.24|1328.39|-582.68| +2452541|30269|3835|16848|293474|2453|4232|10|193|890|42|67.55|131.04|87.79|0.00|3687.18|2837.10|5503.68|36.87|0.00|3687.18|3724.05|850.08| +2452541|30269|925|16848|293474|2453|4232|10|286|890|81|17.04|24.70|4.44|0.00|359.64|1380.24|2000.70|25.17|0.00|359.64|384.81|-1020.60| +2452541|30269|3699|16848|293474|2453|4232|10|115|890|65|64.04|85.81|25.74|0.00|1673.10|4162.60|5577.65|83.65|0.00|1673.10|1756.75|-2489.50| +2452541|30269|16764|16848|293474|2453|4232|10|266|890|49|6.04|8.45|8.36|0.00|409.64|295.96|414.05|4.09|0.00|409.64|413.73|113.68| +2452541|30269|14199|16848|293474|2453|4232|10|26|890|82|33.35|43.68|16.16|0.00|1325.12|2734.70|3581.76|13.25|0.00|1325.12|1338.37|-1409.58| +2452541|30269|15453|16848|293474|2453|4232|10|300|890|86|23.00|40.94|4.09|0.00|351.74|1978.00|3520.84|28.13|0.00|351.74|379.87|-1626.26| +2452541|30269|14775|16848|293474|2453|4232|10|143|890|78|48.34|49.79|32.36|1186.31|2524.08|3770.52|3883.62|93.64|1186.31|1337.77|1431.41|-2432.75| +2451709|39331|13367|71738|964257|3623|29742|2|10|891|58|25.72|39.86|27.90|0.00|1618.20|1491.76|2311.88|80.91|0.00|1618.20|1699.11|126.44| +2451709|39331|13537|71738|964257|3623|29742|2|272|891|75|42.82|73.65|49.34|1517.20|3700.50|3211.50|5523.75|196.49|1517.20|2183.30|2379.79|-1028.20| +2451709|39331|1382|71738|964257|3623|29742|2|258|891|74|95.40|138.33|5.53|0.00|409.22|7059.60|10236.42|32.73|0.00|409.22|441.95|-6650.38| +2451709|39331|16316|71738|964257|3623|29742|2|287|891|98|93.90|108.92|21.78|0.00|2134.44|9202.20|10674.16|128.06|0.00|2134.44|2262.50|-7067.76| +2451709|39331|12353|71738|964257|3623|29742|2|35|891|89|47.14|58.92|34.76|0.00|3093.64|4195.46|5243.88|185.61|0.00|3093.64|3279.25|-1101.82| +2451709|39331|4340|71738|964257|3623|29742|2|300|891|7|23.55|39.79|13.92|25.33|97.44|164.85|278.53|5.76|25.33|72.11|77.87|-92.74| +2451709|39331|12343|71738|964257|3623|29742|2|230|891|4|70.70|77.06|73.97|0.00|295.88|282.80|308.24|23.67|0.00|295.88|319.55|13.08| +2451709|39331|16031|71738|964257|3623|29742|2|207|891|62|23.76|36.82|13.99|0.00|867.38|1473.12|2282.84|17.34|0.00|867.38|884.72|-605.74| +2451709|39331|11833|71738|964257|3623|29742|2|172|891|48|73.00|130.67|23.52|0.00|1128.96|3504.00|6272.16|33.86|0.00|1128.96|1162.82|-2375.04| +2452133|61134|8801|74030|1130231|6785|42186|1|184|892|56|87.00|130.50|82.21|0.00|4603.76|4872.00|7308.00|184.15|0.00|4603.76|4787.91|-268.24| +2452133|61134|11715|74030|1130231|6785|42186|1|154|892|6|91.71|150.40|148.89|0.00|893.34|550.26|902.40|44.66|0.00|893.34|938.00|343.08| +2452133|61134|12517|74030|1130231|6785|42186|1|172|892|62|92.72|166.89|46.72|0.00|2896.64|5748.64|10347.18|202.76|0.00|2896.64|3099.40|-2852.00| +2452133|61134|223|74030|1130231|6785|42186|1|42|892|63|42.04|44.14|40.60|0.00|2557.80|2648.52|2780.82|179.04|0.00|2557.80|2736.84|-90.72| +2452133|61134|15501|74030|1130231|6785|42186|1|60|892|77|13.30|24.73|18.79|0.00|1446.83|1024.10|1904.21|130.21|0.00|1446.83|1577.04|422.73| +2452133|61134|2299|74030|1130231|6785|42186|1|193|892|68|39.55|66.44|29.89|0.00|2032.52|2689.40|4517.92|142.27|0.00|2032.52|2174.79|-656.88| +2452133|61134|1131|74030|1130231|6785|42186|1|85|892|24|14.94|25.84|2.06|11.37|49.44|358.56|620.16|3.04|11.37|38.07|41.11|-320.49| +2452133|61134|2163|74030|1130231|6785|42186|1|88|892|23|13.96|27.36|19.69|0.00|452.87|321.08|629.28|27.17|0.00|452.87|480.04|131.79| +2452133|61134|5481|74030|1130231|6785|42186|1|94|892|43|19.64|25.13|20.85|0.00|896.55|844.52|1080.59|17.93|0.00|896.55|914.48|52.03| +2452133|61134|9121|74030|1130231|6785|42186|1|56|892|15|26.30|29.98|7.49|0.00|112.35|394.50|449.70|5.61|0.00|112.35|117.96|-282.15| +2452133|61134|1419|74030|1130231|6785|42186|1|234|892|33|28.17|35.77|18.60|0.00|613.80|929.61|1180.41|42.96|0.00|613.80|656.76|-315.81| +2452133|61134|4961|74030|1130231|6785|42186|1|37|892|36|13.33|13.86|4.29|71.04|154.44|479.88|498.96|5.83|71.04|83.40|89.23|-396.48| +2452133|61134|7559|74030|1130231|6785|42186|1|255|892|25|96.73|184.75|133.02|0.00|3325.50|2418.25|4618.75|199.53|0.00|3325.50|3525.03|907.25| +2452133|61134|12461|74030|1130231|6785|42186|1|139|892|71|49.74|61.18|35.48|0.00|2519.08|3531.54|4343.78|151.14|0.00|2519.08|2670.22|-1012.46| +2452133|61134|4403|74030|1130231|6785|42186|1|276|892|76|74.04|77.74|74.63|0.00|5671.88|5627.04|5908.24|113.43|0.00|5671.88|5785.31|44.84| +2451432|66513|3530|47054|480588|1769|48150|8|129|893|2|75.18|102.24|100.19|0.00|200.38|150.36|204.48|16.03|0.00|200.38|216.41|50.02| +2451432|66513|9536|47054|480588|1769|48150|8|97|893|13|26.64|53.28|19.18|0.00|249.34|346.32|692.64|17.45|0.00|249.34|266.79|-96.98| +2451432|66513|2284|47054|480588|1769|48150|8|213|893|68|41.57|41.57|24.52|0.00|1667.36|2826.76|2826.76|50.02|0.00|1667.36|1717.38|-1159.40| +2451432|66513|16052|47054|480588|1769|48150|8|50|893|57|30.64|31.55|18.93|0.00|1079.01|1746.48|1798.35|0.00|0.00|1079.01|1079.01|-667.47| +2451432|66513|3733|47054|480588|1769|48150|8|201|893|30|50.98|60.66|51.56|1515.86|1546.80|1529.40|1819.80|1.54|1515.86|30.94|32.48|-1498.46| +2451432|66513|7840|47054|480588|1769|48150|8|135|893|34|18.14|35.55|30.57|0.00|1039.38|616.76|1208.70|72.75|0.00|1039.38|1112.13|422.62| +2451432|66513|5293|47054|480588|1769|48150|8|16|893|21|52.60|74.16|15.57|0.00|326.97|1104.60|1557.36|6.53|0.00|326.97|333.50|-777.63| +2451432|66513|6886|47054|480588|1769|48150|8|105|893|81|17.27|25.73|15.95|0.00|1291.95|1398.87|2084.13|77.51|0.00|1291.95|1369.46|-106.92| +2451432|66513|3494|47054|480588|1769|48150|8|243|893|76|38.28|65.45|45.16|0.00|3432.16|2909.28|4974.20|102.96|0.00|3432.16|3535.12|522.88| +2451432|66513|13393|47054|480588|1769|48150|8|131|893|52|18.77|33.97|25.81|550.26|1342.12|976.04|1766.44|23.75|550.26|791.86|815.61|-184.18| +2451432|66513|14840|47054|480588|1769|48150|8|95|893|67|50.31|61.88|41.45|138.85|2777.15|3370.77|4145.96|79.14|138.85|2638.30|2717.44|-732.47| +2451432|66513|3386|47054|480588|1769|48150|8|18|893|97|74.96|135.67|39.34|0.00|3815.98|7271.12|13159.99|0.00|0.00|3815.98|3815.98|-3455.14| +2451432|66513|13801|47054|480588|1769|48150|8|51|893|92|92.10|180.51|81.22|1270.28|7472.24|8473.20|16606.92|62.01|1270.28|6201.96|6263.97|-2271.24| +2451432|66513|17290|47054|480588|1769|48150|8|225|893|26|13.95|18.27|10.77|0.00|280.02|362.70|475.02|2.80|0.00|280.02|282.82|-82.68| +2451088|34401|10327|59840|459160|5327|21847|1|107|894|36|10.42|10.94|7.76|0.00|279.36|375.12|393.84|22.34|0.00|279.36|301.70|-95.76| +2451088|34401|8839|59840|459160|5327|21847|1|34|894|85|7.20|11.16|3.23|0.00|274.55|612.00|948.60|19.21|0.00|274.55|293.76|-337.45| +2451088|34401|15367|59840|459160|5327|21847|1|83|894|93|16.06|18.46|17.16|0.00|1595.88|1493.58|1716.78|31.91|0.00|1595.88|1627.79|102.30| +2451088|34401|2558|59840|459160|5327|21847|1|9|894|13|66.60|103.23|87.74|0.00|1140.62|865.80|1341.99|68.43|0.00|1140.62|1209.05|274.82| +2451088|34401|9166|59840|459160|5327|21847|1|235|894|66|83.53|152.02|51.68|0.00|3410.88|5512.98|10033.32|102.32|0.00|3410.88|3513.20|-2102.10| +2451088|34401|16370|59840|459160|5327|21847|1|77|894|12|25.25|31.05|25.77|0.00|309.24|303.00|372.60|0.00|0.00|309.24|309.24|6.24| +2451088|34401|14053|59840|459160|5327|21847|1|215|894|24|5.02|5.22|4.33|0.00|103.92|120.48|125.28|5.19|0.00|103.92|109.11|-16.56| +2451088|34401|12544|59840|459160|5327|21847|1|188|894|62|12.41|12.65|3.03|50.72|187.86|769.42|784.30|12.34|50.72|137.14|149.48|-632.28| +2451088|34401|17758|59840|459160|5327|21847|1|76|894|62|25.63|46.90|11.72|0.00|726.64|1589.06|2907.80|65.39|0.00|726.64|792.03|-862.42| +2451088|34401|15370|59840|459160|5327|21847|1|298|894|17|67.37|86.23|6.89|99.56|117.13|1145.29|1465.91|0.70|99.56|17.57|18.27|-1127.72| +2451088|34401|12217|59840|459160|5327|21847|1|220|894|16|90.70|160.53|35.31|0.00|564.96|1451.20|2568.48|16.94|0.00|564.96|581.90|-886.24| +2451088|34401|9758|59840|459160|5327|21847|1|296|894|69|34.07|59.62|43.52|0.00|3002.88|2350.83|4113.78|180.17|0.00|3002.88|3183.05|652.05| +|34401|11239|||5327||||894||||39.40|0.00|1891.20||||0.00||2023.58|| +2451088|34401|17059|59840|459160|5327|21847|1|101|894|37|30.35|34.90|24.08|151.46|890.96|1122.95|1291.30|29.58|151.46|739.50|769.08|-383.45| +||7982|||||10||895|59||81.36|7.32|0.00|431.88|||25.91|0.00|431.88|457.79|| +2451464|51576|11716|54681|1389064|3894|37077|10|2|895|67|18.80|19.92|0.39|0.00|26.13|1259.60|1334.64|0.00|0.00|26.13|26.13|-1233.47| +|51576|6056||1389064|3894||10|167|895|||||58.80|98.00||3274.18|1.96|58.80|39.20|41.16|| +2451464|51576|14593|54681|1389064|3894|37077|10|105|895|55|55.33|90.18|88.37|0.00|4860.35|3043.15|4959.90|340.22|0.00|4860.35|5200.57|1817.20| +2451464|51576|6439|54681|1389064|3894|37077|10|101|895|66|82.48|104.74|54.46|0.00|3594.36|5443.68|6912.84|143.77|0.00|3594.36|3738.13|-1849.32| +2451464|51576|10814|54681|1389064|3894|37077|10|203|895|96|24.09|31.31|20.97|0.00|2013.12|2312.64|3005.76|20.13|0.00|2013.12|2033.25|-299.52| +2451464|51576|14179|54681|1389064|3894|37077|10|195|895|65|62.35|74.19|0.00|0.00|0.00|4052.75|4822.35|0.00|0.00|0.00|0.00|-4052.75| +2451464|51576|16867|54681|1389064|3894|37077|10|291|895|61|52.61|98.38|16.72|0.00|1019.92|3209.21|6001.18|61.19|0.00|1019.92|1081.11|-2189.29| +2451464|51576|7796|54681|1389064|3894|37077|10|179|895|81|33.83|56.15|42.67|0.00|3456.27|2740.23|4548.15|34.56|0.00|3456.27|3490.83|716.04| +2451464|51576|9902|54681|1389064|3894|37077|10|239|895|49|32.83|45.63|3.19|0.00|156.31|1608.67|2235.87|12.50|0.00|156.31|168.81|-1452.36| +2451464|51576|13777|54681|1389064|3894|37077|10|43|895|1|17.87|31.27|8.13|5.36|8.13|17.87|31.27|0.08|5.36|2.77|2.85|-15.10| +2451464|51576|15361|54681|1389064|3894|37077|10|280|895|18|51.54|53.08|48.30|0.00|869.40|927.72|955.44|34.77|0.00|869.40|904.17|-58.32| +2451464|51576|3842|54681|1389064|3894|37077|10|222|895|59|23.90|34.89|15.35|0.00|905.65|1410.10|2058.51|54.33|0.00|905.65|959.98|-504.45| +2451464|51576|949|54681|1389064|3894|37077|10|19|895|58|3.67|4.25|3.95|0.00|229.10|212.86|246.50|2.29|0.00|229.10|231.39|16.24| +2451464||5336|54681|1389064|3894|37077|||895|||||0.00|52.20||5266.11|1.04|0.00|52.20||-3736.65| +2451464|51576|15244|54681|1389064|3894|37077|10|250|895|82|14.18|23.96|1.19|0.00|97.58|1162.76|1964.72|5.85|0.00|97.58|103.43|-1065.18| +2451950|42402|4021|44170|132658|6746|24034|2|300|896|48|58.78|59.95|4.79|147.14|229.92|2821.44|2877.60|1.65|147.14|82.78|84.43|-2738.66| +2451950|42402|8083|44170|132658|6746|24034|2|56|896|61|93.98|180.44|131.72|0.00|8034.92|5732.78|11006.84|482.09|0.00|8034.92|8517.01|2302.14| +2451950|42402|13651|44170|132658|6746|24034|2|129|896|23|98.30|122.87|108.12|1069.30|2486.76|2260.90|2826.01|56.69|1069.30|1417.46|1474.15|-843.44| +2451950|42402|6343|44170|132658|6746|24034|2|28|896|17|51.90|93.42|38.30|0.00|651.10|882.30|1588.14|13.02|0.00|651.10|664.12|-231.20| +2451950|42402|5559|44170|132658|6746|24034|2|188|896|81|52.57|66.76|14.01|0.00|1134.81|4258.17|5407.56|90.78|0.00|1134.81|1225.59|-3123.36| +2451950|42402|11749|44170|132658|6746|24034|2|292|896|93|97.50|127.72|3.83|0.00|356.19|9067.50|11877.96|10.68|0.00|356.19|366.87|-8711.31| +2451950|42402|13075|44170|132658|6746|24034|2|141|896|86|57.86|109.93|69.25|0.00|5955.50|4975.96|9453.98|119.11|0.00|5955.50|6074.61|979.54| +2451950|42402|8831|44170|132658|6746|24034|2|143|896|61|50.13|61.65|21.57|0.00|1315.77|3057.93|3760.65|0.00|0.00|1315.77|1315.77|-1742.16| +2451135|44273|3794|67431|493056|5881|48306|7|180|897|61|22.97|37.21|19.72|0.00|1202.92|1401.17|2269.81|0.00|0.00|1202.92|1202.92|-198.25| +2451135|44273|11978|67431|493056|5881|48306|7|100|897|7|39.86|52.61|30.51|0.00|213.57|279.02|368.27|2.13|0.00|213.57|215.70|-65.45| +2451135|44273|9463|67431|493056|5881|48306|7|60|897|56|9.72|18.07|4.51|0.00|252.56|544.32|1011.92|7.57|0.00|252.56|260.13|-291.76| +2451135|44273|11552|67431|493056|5881|48306|7|80|897|72|20.50|38.13|1.90|0.00|136.80|1476.00|2745.36|1.36|0.00|136.80|138.16|-1339.20| +2451135|44273|430|67431|493056|5881|48306|7|118|897|3|28.66|30.37|28.24|0.00|84.72|85.98|91.11|1.69|0.00|84.72|86.41|-1.26| +2451135|44273|7144|67431|493056|5881|48306|7|206|897|51|74.29|75.77|0.00|0.00|0.00|3788.79|3864.27|0.00|0.00|0.00|0.00|-3788.79| +2451135|44273|6610|67431|493056|5881|48306|7|103|897|80|77.16|102.62|89.27|0.00|7141.60|6172.80|8209.60|499.91|0.00|7141.60|7641.51|968.80| +2451135|44273|7801||493056|||7||897||38.90|||0.00|1437.80||4793.72|43.13|0.00|||-1635.30| +||6284|67431||5881||||897||56.53||27.70|132.96|332.40|678.36|||132.96|199.44|209.41|| +2451135|44273|5839|67431|493056|5881|48306|7|214|897|87|56.99|84.34|16.86|0.00|1466.82|4958.13|7337.58|14.66|0.00|1466.82|1481.48|-3491.31| +2451135|44273|5689|67431|493056|5881|48306|7|108|897|18|10.80|15.87|15.39|0.00|277.02|194.40|285.66|11.08|0.00|277.02|288.10|82.62| +2451135|44273|9610|67431|493056|5881|48306|7|168|897|16|23.01|28.53|21.11|0.00|337.76|368.16|456.48|3.37|0.00|337.76|341.13|-30.40| +2451135|44273|15997|67431|493056|5881|48306|7|119|897|65|41.78|58.07|17.42|0.00|1132.30|2715.70|3774.55|90.58|0.00|1132.30|1222.88|-1583.40| +2451135|44273|8821|67431|493056|5881|48306|7|197|897|67|78.15|137.54|97.65|0.00|6542.55|5236.05|9215.18|196.27|0.00|6542.55|6738.82|1306.50| +2451135|44273|3242|67431|493056|5881|48306|7|213|897|32|34.42|65.05|44.88|0.00|1436.16|1101.44|2081.60|0.00|0.00|1436.16|1436.16|334.72| +2451825|71662|5390|85573|91654|935|17816|1|270|898|21|10.79|20.39|19.37|0.00|406.77|226.59|428.19|32.54|0.00|406.77|439.31|180.18| +2451825|71662|2743|85573|91654|935|17816|1|18|898|61|89.30|123.23|101.04|2588.64|6163.44|5447.30|7517.03|142.99|2588.64|3574.80|3717.79|-1872.50| +2451825|71662|1495|85573|91654|935|17816|1|45|898|80|13.89|22.36|4.69|0.00|375.20|1111.20|1788.80|22.51|0.00|375.20|397.71|-736.00| +2451825|71662|14048|85573|91654|935|17816|1|15|898|99|18.58|32.14|13.17|0.00|1303.83|1839.42|3181.86|52.15|0.00|1303.83|1355.98|-535.59| +2451825|71662|9833|85573|91654|935|17816|1|35|898|41|11.92|18.95|11.55|0.00|473.55|488.72|776.95|42.61|0.00|473.55|516.16|-15.17| +2451825|71662|15044|85573|91654|935|17816|1|202|898|23|9.35|12.24|0.48|0.00|11.04|215.05|281.52|0.00|0.00|11.04|11.04|-204.01| +2451825|71662|8287|85573|91654|935|17816|1|282|898|24|5.62|5.95|1.78|29.47|42.72|134.88|142.80|0.92|29.47|13.25|14.17|-121.63| +2451825|71662|11510|85573|91654|935|17816|1|291|898|78|81.05|106.17|79.62|0.00|6210.36|6321.90|8281.26|0.00|0.00|6210.36|6210.36|-111.54| +2451825|71662|5389|85573|91654|935|17816|1|107|898|34|70.03|77.73|0.00|0.00|0.00|2381.02|2642.82|0.00|0.00|0.00|0.00|-2381.02| +2451825|71662|5875|85573|91654|935|17816|1|123|898|67|49.15|82.08|32.83|0.00|2199.61|3293.05|5499.36|197.96|0.00|2199.61|2397.57|-1093.44| +2451825|71662|4568|85573|91654|935|17816|1|287|898|91|36.40|60.06|59.45|0.00|5409.95|3312.40|5465.46|270.49|0.00|5409.95|5680.44|2097.55| +2451122|33344|5629|70399|1072008|2417|34073|2|102|899|50|28.15|35.46|23.40|35.10|1170.00|1407.50|1773.00|102.14|35.10|1134.90|1237.04|-272.60| +2451122|33344|16951|70399|1072008|2417|34073|2|163|899|73|17.19|30.94|11.44|0.00|835.12|1254.87|2258.62|58.45|0.00|835.12|893.57|-419.75| +2451122|33344|8557|70399|1072008|2417|34073|2|99|899|70|78.93|94.71|42.61|0.00|2982.70|5525.10|6629.70|0.00|0.00|2982.70|2982.70|-2542.40| +2451122|33344|526|70399|1072008|2417|34073|2|189|899|26|68.00|91.12|78.36|0.00|2037.36|1768.00|2369.12|183.36|0.00|2037.36|2220.72|269.36| +2451122|33344|11863|70399|1072008|2417|34073|2|194|899|21|67.08|72.44|62.29|0.00|1308.09|1408.68|1521.24|104.64|0.00|1308.09|1412.73|-100.59| +||15824|70399|||34073|||899|39|35.92|||0.00|||1792.83|1.79|0.00|||| +2451122|33344|15818|70399|1072008|2417|34073|2|241|899|21|14.06|19.68|3.34|14.02|70.14|295.26|413.28|2.80|14.02|56.12|58.92|-239.14| +2451122|33344|9986|70399|1072008|2417|34073|2|299|899|1|40.65|69.10|35.93|0.00|35.93|40.65|69.10|2.15|0.00|35.93|38.08|-4.72| +||16382||1072008|2417|||128|899|||46.57||0.00||323.04|558.84||0.00|273.72|298.35|-49.32| +||11419|63022|659919|5969||7||900|6|31.01|51.78|48.15||288.90||310.68|20.22||||102.84| +2451817|47135|10019|63022|659919|5969|21666|7|190|900|22|35.41|35.41|5.31|57.24|116.82|779.02|779.02|4.17|57.24|59.58|63.75|-719.44| +|47135|7033|63022||5969||7||900|8|55.16||||||728.08|4.44||444.08|448.52|| +2451817|47135|16795|63022|659919|5969|21666|7|98|900|37|74.84|147.43|60.44|0.00|2236.28|2769.08|5454.91|178.90|0.00|2236.28|2415.18|-532.80| +2451817|47135|16745|63022|659919|5969|21666|7|28|900|88|9.30|17.11|10.60|0.00|932.80|818.40|1505.68|18.65|0.00|932.80|951.45|114.40| +2451817|47135|4301|63022|659919|5969|21666|7|32|900|35|21.76|32.85|11.82|0.00|413.70|761.60|1149.75|37.23|0.00|413.70|450.93|-347.90| +||16850|63022||||7||900||||36.39||1965.06||3708.18|19.65||1965.06||| +2451817|47135|4801|63022|659919|5969|21666|7|268|900|78|26.60|38.83|18.63|130.78|1453.14|2074.80|3028.74|0.00|130.78|1322.36|1322.36|-752.44| +2451817|47135|3179|63022|659919|5969|21666|7|160|900|95|85.19|117.56|91.69|0.00|8710.55|8093.05|11168.20|435.52|0.00|8710.55|9146.07|617.50| +2451817|47135|11804|63022|659919|5969|21666|7|175|900|85|93.56|180.57|79.45|6483.12|6753.25|7952.60|15348.45|5.40|6483.12|270.13|275.53|-7682.47| +2451530|49162|13430|4978|1875568|6158|32748|2|287|901|67|38.48|42.71|24.77|0.00|1659.59|2578.16|2861.57|132.76|0.00|1659.59|1792.35|-918.57| +2451530|49162|13124|4978|1875568|6158|32748|2|195|901|84|50.73|100.95|92.87|2262.31|7801.08|4261.32|8479.80|166.16|2262.31|5538.77|5704.93|1277.45| +2451530|49162|16414|4978|1875568|6158|32748|2|197|901|50|86.31|87.17|61.89|0.00|3094.50|4315.50|4358.50|247.56|0.00|3094.50|3342.06|-1221.00| +2451530|49162|1316|4978|1875568|6158|32748|2|5|901|53|83.14|127.20|73.77|430.07|3909.81|4406.42|6741.60|104.39|430.07|3479.74|3584.13|-926.68| +2451530|49162|13708|4978|1875568|6158|32748|2|33|901|18|29.13|29.42|7.64|12.37|137.52|524.34|529.56|11.26|12.37|125.15|136.41|-399.19| +2451530|49162|416|4978|1875568|6158|32748|2|188|901|63|2.30|4.23|0.12|0.00|7.56|144.90|266.49|0.22|0.00|7.56|7.78|-137.34| +2451530|49162|14420|4978|1875568|6158|32748|2|67|901|37|89.14|91.81|12.85|0.00|475.45|3298.18|3396.97|14.26|0.00|475.45|489.71|-2822.73| +2451530|49162|6949|4978|1875568|6158|32748|2|237|901|67|73.47|76.40|70.28|0.00|4708.76|4922.49|5118.80|141.26|0.00|4708.76|4850.02|-213.73| +2451530|49162|7825|4978|1875568|6158|32748|2|23|901|86|67.26|77.34|71.92|0.00|6185.12|5784.36|6651.24|123.70|0.00|6185.12|6308.82|400.76| +2451530|49162|14350|4978|1875568|6158|32748|2|18|901|70|79.69|130.69|74.49|0.00|5214.30|5578.30|9148.30|52.14|0.00|5214.30|5266.44|-364.00| +2451530|49162|8719|4978|1875568|6158|32748|2|194|901|43|39.58|48.68|18.01|0.00|774.43|1701.94|2093.24|30.97|0.00|774.43|805.40|-927.51| +2451530|49162|10555|4978|1875568|6158|32748|2|255|901|94|8.93|17.14|3.94|0.00|370.36|839.42|1611.16|3.70|0.00|370.36|374.06|-469.06| +2451530|49162|15118|4978|1875568|6158|32748|2|17|901|54|37.84|73.78|28.77|0.00|1553.58|2043.36|3984.12|0.00|0.00|1553.58|1553.58|-489.78| +2451530|49162|11216|4978|1875568|6158|32748|2|2|901|95|17.19|30.08|13.53|1169.66|1285.35|1633.05|2857.60|5.78|1169.66|115.69|121.47|-1517.36| +2451530|49162|16657|4978|1875568|6158|32748|2|134|901|42|52.15|58.40|9.34|0.00|392.28|2190.30|2452.80|15.69|0.00|392.28|407.97|-1798.02| +||6446|4978|1875568||32748|2||901|56||||0.00|||7750.40|6.19|0.00|309.68||| +2451909|52134|5384|61933|1804222|||2||902||44.93||64.61||5750.29|3998.77|||||6152.81|1751.52| +2451909|52134|12851|61933|1804222|3311|25647|2|250|902|36|76.83|79.90|61.52|0.00|2214.72|2765.88|2876.40|110.73|0.00|2214.72|2325.45|-551.16| +2451909|52134|6647|61933|1804222|3311|25647|2|164|902|35|96.53|140.93|12.68|0.00|443.80|3378.55|4932.55|31.06|0.00|443.80|474.86|-2934.75| +2451909|52134|5053|61933|1804222|3311|25647|2|127|902|71|62.55|116.96|40.93|0.00|2906.03|4441.05|8304.16|232.48|0.00|2906.03|3138.51|-1535.02| +2451909|52134|7337|61933|1804222|3311|25647|2|270|902|95|49.12|51.08|25.02|0.00|2376.90|4666.40|4852.60|213.92|0.00|2376.90|2590.82|-2289.50| +2451909|52134|1988|61933|1804222|3311|25647|2|205|902|51|87.50|124.25|16.15|625.97|823.65|4462.50|6336.75|0.00|625.97|197.68|197.68|-4264.82| +2451909|52134|9679|61933|1804222|3311|25647|2|45|902|58|5.58|9.37|3.37|0.00|195.46|323.64|543.46|15.63|0.00|195.46|211.09|-128.18| +2451909|52134|653|61933|1804222|3311|25647|2|120|902|68|29.18|33.55|9.05|0.00|615.40|1984.24|2281.40|49.23|0.00|615.40|664.63|-1368.84| +2451909|52134|8906|61933|1804222|3311|25647|2|198|902|26|21.91|39.65|38.46|0.00|999.96|569.66|1030.90|79.99|0.00|999.96|1079.95|430.30| +2451909|52134|11831|61933|1804222|3311|25647|2|14|902|57|78.12|146.86|57.27|0.00|3264.39|4452.84|8371.02|32.64|0.00|3264.39|3297.03|-1188.45| +2451909|52134|17156|61933|1804222|3311|25647|2|238|902|4|77.51|134.09|0.00|0.00|0.00|310.04|536.36|0.00|0.00|0.00|0.00|-310.04| +||625||||||271|903|80||45.45|15.90||1272.00|3162.40||||||| +2451149|48114|10318||833584|||||903||89.12||37.20|||7307.84|12204.06|30.50||3050.40|3080.90|| +2451149|48114|5683|76092|833584|5476|28568|2|176|903|17|67.74|131.41|57.82|0.00|982.94|1151.58|2233.97|78.63|0.00|982.94|1061.57|-168.64| +2451149|48114|6868|76092|833584|5476|28568|2|241|903|32|11.98|17.97|3.77|0.00|120.64|383.36|575.04|2.41|0.00|120.64|123.05|-262.72| +2451149|48114|11074|76092|833584|5476|28568|2|46|903|63|47.78|57.33|40.70|0.00|2564.10|3010.14|3611.79|153.84|0.00|2564.10|2717.94|-446.04| +2451149|48114|5986|76092|833584|5476|28568|2|185|903|3|75.03|108.04|5.40|0.00|16.20|225.09|324.12|0.16|0.00|16.20|16.36|-208.89| +2451149|48114|7600|76092|833584|5476|28568|2|82|903|82|27.18|40.22|22.52|0.00|1846.64|2228.76|3298.04|147.73|0.00|1846.64|1994.37|-382.12| +2451149||9775|||5476||2|52|903|||115.86||0.00||3778.20||38.92|0.00|486.60|525.52|| +2451149|48114|2833|76092|833584|5476|28568|2|113|903|45|91.01|167.45|144.00|0.00|6480.00|4095.45|7535.25|453.60|0.00|6480.00|6933.60|2384.55| +2451149|48114|4819|76092|833584|5476|28568|2|174|903|46|53.59|101.28|63.80|0.00|2934.80|2465.14|4658.88|117.39|0.00|2934.80|3052.19|469.66| +2451149|48114|2929|76092|833584|5476|28568|2|40|903|23|19.70|31.12|31.12|443.77|715.76|453.10|715.76|13.59|443.77|271.99|285.58|-181.11| +2451149|48114|11696|76092|833584|5476|28568|2|179|903|80|27.61|48.59|28.66|0.00|2292.80|2208.80|3887.20|206.35|0.00|2292.80|2499.15|84.00| +2451149|48114|17743|76092|833584|5476|28568|2|262|903|34|68.98|75.18|47.36|402.56|1610.24|2345.32|2556.12|72.46|402.56|1207.68|1280.14|-1137.64| +2452218|34690|14609|5382|1870671|7160|36096|8|239|904|12|67.73|98.88|58.33|0.00|699.96|812.76|1186.56|34.99|0.00|699.96|734.95|-112.80| +2452218|34690|15353|5382|1870671|7160|36096|8|146|904|74|84.41|121.55|46.18|0.00|3417.32|6246.34|8994.70|34.17|0.00|3417.32|3451.49|-2829.02| +2452218|34690|3079|5382|1870671|7160|36096|8|211|904|75|28.94|44.27|17.26|0.00|1294.50|2170.50|3320.25|0.00|0.00|1294.50|1294.50|-876.00| +2452218|34690|14501|5382|1870671|7160|36096|8|161|904|7|46.17|49.86|18.44|127.78|129.08|323.19|349.02|0.01|127.78|1.30|1.31|-321.89| +2452218|34690|5027|5382|1870671|7160|36096|8|120|904|54|10.84|19.83|18.24|0.00|984.96|585.36|1070.82|9.84|0.00|984.96|994.80|399.60| +2452218|34690|12627|5382|1870671|7160|36096|8|14|904|42|99.82|155.71|24.91|0.00|1046.22|4192.44|6539.82|73.23|0.00|1046.22|1119.45|-3146.22| +2452218|34690|10515|5382|1870671|7160|36096|8|281|904|39|18.78|19.53|14.64|0.00|570.96|732.42|761.67|0.00|0.00|570.96|570.96|-161.46| +2452218|34690|16095|5382|1870671|7160|36096|8|295|904|3|34.94|39.13|30.52|0.00|91.56|104.82|117.39|7.32|0.00|91.56|98.88|-13.26| +2452598|68791|17790|20179|536980|6581|1829|2|48|905|76|53.77|75.27|46.66|0.00|3546.16|4086.52|5720.52|141.84|0.00|3546.16|3688.00|-540.36| +2452598|68791|1950|20179|536980|6581|1829|2|268|905|52|60.51|119.80|93.44|0.00|4858.88|3146.52|6229.60|145.76|0.00|4858.88|5004.64|1712.36| +2452598|68791|9822|20179|536980|6581|1829|2|68|905|47|42.15|46.36|1.39|0.00|65.33|1981.05|2178.92|5.87|0.00|65.33|71.20|-1915.72| +2452598|68791|168|20179|536980|6581|1829|2|233|905|31|75.01|132.76|1.32|0.00|40.92|2325.31|4115.56|2.86|0.00|40.92|43.78|-2284.39| +2452598|68791|3735|20179|536980|6581|1829|2|7|905|43|60.74|68.02|58.49|704.21|2515.07|2611.82|2924.86|0.00|704.21|1810.86|1810.86|-800.96| +2452598|68791|16584|20179|536980|6581|1829|2|229|905|89|31.65|37.03|31.47|0.00|2800.83|2816.85|3295.67|28.00|0.00|2800.83|2828.83|-16.02| +2452598|68791|6249|20179|536980|6581|1829|2|50|905|6|30.92|45.45|1.81|0.00|10.86|185.52|272.70|0.10|0.00|10.86|10.96|-174.66| +2452598|68791|14946|20179|536980|6581|1829|2|214|905|25|39.80|72.43|70.98|0.00|1774.50|995.00|1810.75|88.72|0.00|1774.50|1863.22|779.50| +2452598|68791|9789|20179|536980|6581|1829|2|208|905|11|15.67|23.66|3.07|0.00|33.77|172.37|260.26|1.35|0.00|33.77|35.12|-138.60| +2452598|68791|373|20179|536980|6581|1829|2|62|905|43|54.70|103.93|75.86|0.00|3261.98|2352.10|4468.99|260.95|0.00|3261.98|3522.93|909.88| +2452598|68791|9618|20179|536980|6581|1829|2|296|905|2|43.21|50.12|8.52|0.00|17.04|86.42|100.24|1.36|0.00|17.04|18.40|-69.38| +2452598|68791|4890|20179|536980|6581|1829|2|197|905|55|47.84|48.31|10.14|111.54|557.70|2631.20|2657.05|4.46|111.54|446.16|450.62|-2185.04| +2452598|68791|14748|20179|536980|6581|1829|2|252|905|71|89.36|160.84|152.79|0.00|10848.09|6344.56|11419.64|542.40|0.00|10848.09|11390.49|4503.53| +2452598|68791|1591|20179|536980|6581|1829|2|278|905|24|78.53|124.07|58.31|0.00|1399.44|1884.72|2977.68|83.96|0.00|1399.44|1483.40|-485.28| +2452218|37649|9567|59715|339960|5005|14050|8|300|906|84|70.18|110.18|93.65|0.00|7866.60|5895.12|9255.12|550.66|0.00|7866.60|8417.26|1971.48| +2452218|37649|1443|59715|339960|5005|14050|8|286|906|13|73.24|119.38|107.44|0.00|1396.72|952.12|1551.94|69.83|0.00|1396.72|1466.55|444.60| +2452218|37649|6287|59715|339960|5005|14050|8|80|906|74|99.64|175.36|156.07|0.00|11549.18|7373.36|12976.64|346.47|0.00|11549.18|11895.65|4175.82| +2452218||5255|59715|339960||14050|8||906|55|64.83|120.58||||3565.65|6631.90|||5836.05||2270.40| +|37649|13261|||5005||8||906|95|3.68||3.59|0.00|341.05||||0.00||347.87|-8.55| +2452218||1421|59715||5005|||156|906|93|83.46|||||7761.78||486.27|||6564.75|-1683.30| +2452218|37649|8917|59715|339960|5005|14050|8|59|906|93|98.65|112.46|66.35|3764.03|6170.55|9174.45|10458.78|96.26|3764.03|2406.52|2502.78|-6767.93| +2452218|37649|3623|59715|339960|5005|14050|8|143|906|37|88.39|128.16|87.14|1225.18|3224.18|3270.43|4741.92|19.99|1225.18|1999.00|2018.99|-1271.43| +2452218||12615|||5005|14050||215|906|||||||1618.78||||984.55|1073.15|| +2452218|37649|11091|59715|339960|5005|14050|8|268|906|97|55.71|109.19|19.65|0.00|1906.05|5403.87|10591.43|76.24|0.00|1906.05|1982.29|-3497.82| +2452218|37649|14629|59715|339960|5005|14050|8|240|906|8|44.14|56.94|14.80|0.00|118.40|353.12|455.52|10.65|0.00|118.40|129.05|-234.72| +2452218|37649|8957|59715|339960|5005|14050|8|9|906|33|37.43|48.28|37.65|1093.35|1242.45|1235.19|1593.24|11.92|1093.35|149.10|161.02|-1086.09| +2452218|37649|6901|59715|339960|5005|14050|8|64|906|8|72.59|109.61|0.00|0.00|0.00|580.72|876.88|0.00|0.00|0.00|0.00|-580.72| +2452559|70285|17463|81659|560036|2413|33590|4|196|907|17|39.40|66.98|2.67|30.41|45.39|669.80|1138.66|0.14|30.41|14.98|15.12|-654.82| +2452559|70285|16245|81659|560036|2413|33590|4|242|907|54|6.43|12.66|4.93|23.95|266.22|347.22|683.64|21.80|23.95|242.27|264.07|-104.95| +2452559||10425||560036|2413|33590|4|221|907|82|||2.16||177.12|1536.68||6.58||164.73|171.31|| +2452559|70285|15609|81659|560036|2413|33590|4|136|907|43|33.47|63.59|54.68|0.00|2351.24|1439.21|2734.37|117.56|0.00|2351.24|2468.80|912.03| +|70285|9253|||2413||4||907||87.53|139.17|122.46|4283.65||5776.98||341.88|4283.65|3798.71||| +2452559|70285|17502|81659|560036|2413|33590|4|101|907|29|27.52|35.22|16.90|465.59|490.10|798.08|1021.38|0.73|465.59|24.51|25.24|-773.57| +2452559|70285|289|81659|560036|2413|33590|4|55|907|47|80.11|151.40|145.34|0.00|6830.98|3765.17|7115.80|136.61|0.00|6830.98|6967.59|3065.81| +2452559|70285|9993|81659|560036|2413|33590|4|174|907|73|3.37|4.51|1.26|0.00|91.98|246.01|329.23|0.91|0.00|91.98|92.89|-154.03| +2451614|51268|10661|44284|803742|1953|5411|7|139|908|50|44.07|78.44|54.90|0.00|2745.00|2203.50|3922.00|247.05|0.00|2745.00|2992.05|541.50| +2451614|51268|5792|44284|803742|1953|5411|7|15|908|89|94.94|168.99|35.48|0.00|3157.72|8449.66|15040.11|0.00|0.00|3157.72|3157.72|-5291.94| +2451614|51268|9074|44284|803742|1953|5411|7|215|908|44|24.05|46.65|3.26|100.40|143.44|1058.20|2052.60|0.00|100.40|43.04|43.04|-1015.16| +2451614|51268|11846|44284|803742|1953|5411|7|35|908|79|81.35|81.35|32.54|0.00|2570.66|6426.65|6426.65|231.35|0.00|2570.66|2802.01|-3855.99| +2451614|51268|9662|44284|803742|1953|5411|7|223|908|85|70.09|125.46|62.73|0.00|5332.05|5957.65|10664.10|0.00|0.00|5332.05|5332.05|-625.60| +2451614|51268|493|44284|803742|1953|5411|7|260|908|61|74.47|142.98|17.15|0.00|1046.15|4542.67|8721.78|20.92|0.00|1046.15|1067.07|-3496.52| +2451614|51268|11899|44284|803742|1953|5411|7|39|908|12|9.27|13.34|1.06|0.00|12.72|111.24|160.08|1.14|0.00|12.72|13.86|-98.52| +|51268|17275|||1953|||2|908||4.11||0.14|0.00|||277.78|0.42|0.00|5.32|5.74|| +2451614|51268|6449|44284|803742|1953|5411|7|55|908|29|91.76|179.84|158.25|2661.76|4589.25|2661.04|5215.36|19.27|2661.76|1927.49|1946.76|-733.55| +2451614|51268|13049|44284|803742|1953|5411|7|60|908|41|60.33|84.46|44.76|0.00|1835.16|2473.53|3462.86|18.35|0.00|1835.16|1853.51|-638.37| +2451614|51268|7400|44284|803742|1953|5411|7|233|908|36|82.67|118.21|59.10|0.00|2127.60|2976.12|4255.56|127.65|0.00|2127.60|2255.25|-848.52| +2451614|51268|17683|44284|803742|1953|5411|7|254|908|37|10.58|19.99|15.39|0.00|569.43|391.46|739.63|5.69|0.00|569.43|575.12|177.97| +2452538|71640|9661|86304|1732370|3236|24102|8|173|909|26|82.63|148.73|148.73|0.00|3866.98|2148.38|3866.98|38.66|0.00|3866.98|3905.64|1718.60| +2452538|71640|14769|86304|1732370|3236|24102|8|84|909|84|21.99|24.40|6.83|361.44|573.72|1847.16|2049.60|4.24|361.44|212.28|216.52|-1634.88| +2452538|71640|15685|86304|1732370|3236|24102|8|69|909|11|21.96|35.13|29.15|0.00|320.65|241.56|386.43|16.03|0.00|320.65|336.68|79.09| +2452538|71640|3447|86304|1732370|3236|24102|8|187|909|67|12.87|22.00|2.86|0.00|191.62|862.29|1474.00|9.58|0.00|191.62|201.20|-670.67| +2452538|71640|15639||||24102|8|181|909|38|19.26|32.74|15.38|||||52.59||584.44||-147.44| +2452538|71640|10440|86304|1732370|3236|24102|8|106|909|74|21.42|38.77|10.85|0.00|802.90|1585.08|2868.98|8.02|0.00|802.90|810.92|-782.18| +|71640|5688|86304|1732370|3236|24102|8||909||43.17||12.25||208.25||1225.53|0.54|||54.69|-679.74| +2452538|71640|6655|86304|1732370|3236|24102|8|134|909|33|73.65|104.58|47.06|155.29|1552.98|2430.45|3451.14|27.95|155.29|1397.69|1425.64|-1032.76| +2452538|71640|5419|86304|1732370|3236|24102|8|80|909|91|63.15|71.99|2.87|0.00|261.17|5746.65|6551.09|0.00|0.00|261.17|261.17|-5485.48| +2452538|71640|17887|86304|1732370|3236|24102|8|60|909|30|36.08|68.91|28.25|0.00|847.50|1082.40|2067.30|0.00|0.00|847.50|847.50|-234.90| +2452538|71640|8670|86304|1732370|3236|24102|8|111|909|55|30.16|47.65|0.47|0.00|25.85|1658.80|2620.75|1.03|0.00|25.85|26.88|-1632.95| +2451479|40237|625|30314|842828|2542|15456|2|229|910|62|22.27|42.09|11.36|0.00|704.32|1380.74|2609.58|42.25|0.00|704.32|746.57|-676.42| +2451479|40237|10318|30314|842828|2542|15456|2|232|910|27|4.83|8.40|4.45|0.00|120.15|130.41|226.80|7.20|0.00|120.15|127.35|-10.26| +2451479|40237|5683|30314|842828|2542|15456|2|231|910|53|14.51|21.76|3.91|0.00|207.23|769.03|1153.28|12.43|0.00|207.23|219.66|-561.80| +2451479|40237|6868|30314|842828|2542|15456|2|172|910|46|14.19|18.16|17.61|0.00|810.06|652.74|835.36|16.20|0.00|810.06|826.26|157.32| +2451479|40237|11074||||||10|910||67.46|99.16|94.20|0.00|2826.00||||0.00|||| +2451479|40237|5986|30314|842828|2542|15456|2|172|910|31|36.84|65.57|45.89|0.00|1422.59|1142.04|2032.67|113.80|0.00|1422.59|1536.39|280.55| +2451479|40237|7600|30314|842828|2542|15456|2|166|910|76|24.71|24.95|8.98|614.23|682.48|1877.96|1896.20|3.41|614.23|68.25|71.66|-1809.71| +2451479|40237|9775|30314|842828|2542|15456|2|51|910|14|76.37|134.41|126.34|0.00|1768.76|1069.18|1881.74|88.43|0.00|1768.76|1857.19|699.58| +2451479|40237|2833|30314|842828|2542|15456|2|128|910|51|87.76|134.27|89.96|0.00|4587.96|4475.76|6847.77|45.87|0.00|4587.96|4633.83|112.20| +2451888|73033|182|30227|608002|1330|49505|8|193|911|42|49.25|77.32|48.71|0.00|2045.82|2068.50|3247.44|81.83|0.00|2045.82|2127.65|-22.68| +2451888|73033|15878|30227|608002|1330|49505|8|219|911|96|77.66|105.61|43.30|0.00|4156.80|7455.36|10138.56|41.56|0.00|4156.80|4198.36|-3298.56| +2451888|73033|16061|30227|608002|1330|49505|8|235|911|7|47.50|50.35|20.64|0.00|144.48|332.50|352.45|13.00|0.00|144.48|157.48|-188.02| +2451888|73033|8827|30227|608002|1330|49505|8|284|911|66|55.82|94.89|60.72|0.00|4007.52|3684.12|6262.74|40.07|0.00|4007.52|4047.59|323.40| +2451888|73033|2867|30227|608002|1330|49505|8|273|911|76|61.80|88.37|88.37|0.00|6716.12|4696.80|6716.12|268.64|0.00|6716.12|6984.76|2019.32| +2451888|73033|4165|30227|608002|1330|49505|8|173|911|91|92.30|152.29|60.91|0.00|5542.81|8399.30|13858.39|55.42|0.00|5542.81|5598.23|-2856.49| +2451888|73033|9116|30227|608002|1330|49505|8|72|911|7|81.75|118.53|93.63|0.00|655.41|572.25|829.71|45.87|0.00|655.41|701.28|83.16| +2451888|73033|3941|30227|608002|1330|49505|8|136|911|47|68.20|107.75|82.96|0.00|3899.12|3205.40|5064.25|0.00|0.00|3899.12|3899.12|693.72| +2451888|73033|12337|30227|608002|1330|49505|8|94|911|45|77.50|81.37|19.52|0.00|878.40|3487.50|3661.65|35.13|0.00|878.40|913.53|-2609.10| +2451888|73033|1309|30227|608002|1330|49505|8|188|911|69|51.22|71.70|36.56|0.00|2522.64|3534.18|4947.30|176.58|0.00|2522.64|2699.22|-1011.54| +2452366|62191|12393|73918|1739331|391|43804|2|93|912|86|40.89|44.57|37.43|0.00|3218.98|3516.54|3833.02|289.70|0.00|3218.98|3508.68|-297.56| +2452366|62191|11223|73918|1739331|391|43804|2|2|912|27|67.37|122.61|67.43|0.00|1820.61|1818.99|3310.47|18.20|0.00|1820.61|1838.81|1.62| +2452366|62191|10596|73918|1739331|391|43804|2|57|912|12|93.42|94.35|54.72|0.00|656.64|1121.04|1132.20|19.69|0.00|656.64|676.33|-464.40| +2452366|62191|4723|73918|1739331|391|43804|2|29|912|18|27.11|29.27|20.48|0.00|368.64|487.98|526.86|33.17|0.00|368.64|401.81|-119.34| +2452366|62191|15631|73918|1739331|391|43804|2|279|912|45|70.00|133.70|86.90|0.00|3910.50|3150.00|6016.50|39.10|0.00|3910.50|3949.60|760.50| +2452366|62191|3978|73918|1739331|391|43804|2|34|912|25|79.98|112.77|40.59|0.00|1014.75|1999.50|2819.25|10.14|0.00|1014.75|1024.89|-984.75| +2452366|62191|17731|73918|1739331|391|43804|2|123|912|87|77.23|136.69|117.55|0.00|10226.85|6719.01|11892.03|511.34|0.00|10226.85|10738.19|3507.84| +2452366|62191|8238|73918|1739331|391|43804|2|198|912|59|72.20|126.35|117.50|0.00|6932.50|4259.80|7454.65|415.95|0.00|6932.50|7348.45|2672.70| +2452366|62191|4411|73918|1739331|391|43804|2|6|912|72|64.13|109.02|82.85|0.00|5965.20|4617.36|7849.44|238.60|0.00|5965.20|6203.80|1347.84| +2452366|62191|9543|73918|1739331|391|43804|2|191|912|69|50.40|93.74|64.68|0.00|4462.92|3477.60|6468.06|312.40|0.00|4462.92|4775.32|985.32| +2452366|62191|6117|73918|1739331|391|43804|2|205|912|65|3.60|6.30|0.00|0.00|0.00|234.00|409.50|0.00|0.00|0.00|0.00|-234.00| +2452366|62191|11479|73918|1739331|391|43804|2|50|912|34|41.90|49.86|41.88|0.00|1423.92|1424.60|1695.24|56.95|0.00|1423.92|1480.87|-0.68| +2452366||14311|73918|||43804|2|197|912|87||||0.00|||||0.00|8238.90|8980.40|2728.32| +2452366|62191|8937|73918|1739331|391|43804|2|95|912|67|79.49|127.97|3.83|2.56|256.61|5325.83|8573.99|10.16|2.56|254.05|264.21|-5071.78| +2452366|62191|12870|73918|1739331|391|43804|2|73|912|89|28.29|36.77|28.31|0.00|2519.59|2517.81|3272.53|50.39|0.00|2519.59|2569.98|1.78| +2452366|62191|17853|73918|1739331|391|43804|2|197|912|97|50.62|94.15|47.07|2739.47|4565.79|4910.14|9132.55|54.78|2739.47|1826.32|1881.10|-3083.82| +2451898|30876|7202|84375|822899|1870|7194|8|238|913|85|86.37|123.50|43.22|0.00|3673.70|7341.45|10497.50|183.68|0.00|3673.70|3857.38|-3667.75| +2451898|30876|5123|84375|822899|1870|7194|8|63|913|66|58.97|98.47|87.63|3527.98|5783.58|3892.02|6499.02|0.00|3527.98|2255.60|2255.60|-1636.42| +2451898|30876|3787|84375|822899|1870|7194|8|214|913|12|25.08|43.13|14.66|0.00|175.92|300.96|517.56|8.79|0.00|175.92|184.71|-125.04| +2451898|30876|3547|84375|822899|1870|7194|8|114|913|54|26.13|32.40|20.08|0.00|1084.32|1411.02|1749.60|21.68|0.00|1084.32|1106.00|-326.70| +2451898|30876|8623|84375|822899|1870|7194|8|200|913|12|24.63|42.85|9.85|0.00|118.20|295.56|514.20|5.91|0.00|118.20|124.11|-177.36| +2451898|30876|17963|84375|822899|1870|7194|8|151|913|37|50.33|53.34|52.80|0.00|1953.60|1862.21|1973.58|97.68|0.00|1953.60|2051.28|91.39| +2451898|30876|15971|84375|822899|1870|7194|8|147|913|11|38.11|41.92|36.47|308.90|401.17|419.21|461.12|0.92|308.90|92.27|93.19|-326.94| +2451898|30876|6409|84375|822899|1870|7194|8|263|913|62|27.01|35.11|21.76|0.00|1349.12|1674.62|2176.82|107.92|0.00|1349.12|1457.04|-325.50| +2451898|30876|13661|84375|822899|1870|7194|8|101|913|74|99.68|161.48|96.88|0.00|7169.12|7376.32|11949.52|215.07|0.00|7169.12|7384.19|-207.20| +2451392|36931|13442|50211|131735|4812|14289|10|106|914|83|8.98|10.23|0.51|0.00|42.33|745.34|849.09|2.53|0.00|42.33|44.86|-703.01| +2451392|36931|820|50211|131735|4812|14289|10|190|914|17|21.12|41.60|30.78|0.00|523.26|359.04|707.20|47.09|0.00|523.26|570.35|164.22| +2451392|36931|6583|50211|131735|4812|14289|10|128|914|46|80.23|141.20|83.30|0.00|3831.80|3690.58|6495.20|191.59|0.00|3831.80|4023.39|141.22| +2451392|36931|3176|50211|131735|4812|14289|10|166|914|69|89.13|132.80|53.12|0.00|3665.28|6149.97|9163.20|183.26|0.00|3665.28|3848.54|-2484.69| +2451392|36931|2371|50211|131735|4812|14289|10|3|914|94|15.33|18.39|17.65|0.00|1659.10|1441.02|1728.66|149.31|0.00|1659.10|1808.41|218.08| +2451392|36931|10423|50211|131735|4812|14289|10|15|914|48|40.89|56.42|35.54|0.00|1705.92|1962.72|2708.16|136.47|0.00|1705.92|1842.39|-256.80| +2451392|36931|2332|50211|131735|4812|14289|10|246|914|86|10.17|14.33|7.16|0.00|615.76|874.62|1232.38|43.10|0.00|615.76|658.86|-258.86| +2451392|36931|7429|50211|131735||14289|||914|37|38.43||21.03|0.00|778.11|1421.91||7.78|0.00|778.11|785.89|| +2451392|36931|8182|50211|131735|4812|14289|10|123|914|30|5.79|7.70|2.23|0.00|66.90|173.70|231.00|5.35|0.00|66.90|72.25|-106.80| +2451392|36931|2452|50211|131735|4812|14289|10|268|914|34|59.67|106.21|91.34|3012.39|3105.56|2028.78|3611.14|8.38|3012.39|93.17|101.55|-1935.61| +2452607|50953|8593|32090|305406|1122|47638|2|59|915|94|52.07|66.64|53.97|0.00|5073.18|4894.58|6264.16|202.92|0.00|5073.18|5276.10|178.60| +2452607|50953|1218|32090|305406|1122|47638|2|220|915|4|85.59|95.00|53.20|29.79|212.80|342.36|380.00|7.32|29.79|183.01|190.33|-159.35| +2452607|50953|11670|32090|305406|1122|47638|2|204|915|45|68.95|102.04|19.38|0.00|872.10|3102.75|4591.80|43.60|0.00|872.10|915.70|-2230.65| +2452607|50953|8244|32090|305406|1122|47638|2|72|915|79|81.30|140.64|36.56|0.00|2888.24|6422.70|11110.56|0.00|0.00|2888.24|2888.24|-3534.46| +|50953|613||305406||||236|915|59|81.60|||0.00||||5.96|0.00|596.49||| +2452607|50953|8340|32090|305406|1122|47638|2|241|915|13|65.24|118.73|56.99|0.00|740.87|848.12|1543.49|59.26|0.00|740.87|800.13|-107.25| +|50953|11523|32090|305406|||2|159|915||85.29||40.29||2941.17|||||500.00|510.00|| +2452607|50953|14899|32090|305406|1122|47638|2|139|915|21|45.84|76.55|57.41|0.00|1205.61|962.64|1607.55|24.11|0.00|1205.61|1229.72|242.97| +2452607|50953|12486|32090|305406|1122|47638|2|3|915|7|9.61|13.35|11.74|0.00|82.18|67.27|93.45|7.39|0.00|82.18|89.57|14.91| +2452602|63794|6157|45075|1129562|6402|8889|10|82|916|94|25.53|42.12|3.79|192.38|356.26|2399.82|3959.28|11.47|192.38|163.88|175.35|-2235.94| +||1266|45075|1129562|6402|||170|916|81|62.62||84.68|||5072.22|9941.13|||2263.50|2421.94|-2808.72| +2452602|63794|16159|45075|1129562|6402|8889|10|200|916|46|81.55|141.08|100.16|0.00|4607.36|3751.30|6489.68|322.51|0.00|4607.36|4929.87|856.06| +2452602|63794|306|45075|1129562|6402|8889|10|54|916|34|81.65|160.03|75.21|0.00|2557.14|2776.10|5441.02|178.99|0.00|2557.14|2736.13|-218.96| +2452602|63794|15477|45075|1129562|6402|8889|10|44|916|65|37.11|69.39|12.49|0.00|811.85|2412.15|4510.35|8.11|0.00|811.85|819.96|-1600.30| +2452602|63794|13578|45075|1129562|6402|8889|10|245|916|95|36.78|50.38|18.13|499.48|1722.35|3494.10|4786.10|97.82|499.48|1222.87|1320.69|-2271.23| +2452602|63794|5959|45075|1129562|6402|8889|10|277|916|34|56.34|95.77|50.75|0.00|1725.50|1915.56|3256.18|0.00|0.00|1725.50|1725.50|-190.06| +2452602|63794|1641|45075|1129562|6402|8889|10|108|916|22|92.15|145.59|69.88|0.00|1537.36|2027.30|3202.98|46.12|0.00|1537.36|1583.48|-489.94| +2452602|63794|10603|45075|1129562|6402|8889|10|142|916|41|64.91|87.62|2.62|0.00|107.42|2661.31|3592.42|5.37|0.00|107.42|112.79|-2553.89| +2452602|63794|16776|45075|1129562|6402|8889|10|228|916|41|4.51|7.39|1.84|0.00|75.44|184.91|302.99|4.52|0.00|75.44|79.96|-109.47| +2451630|45801|1405|17353|1734807|5061|32468|2|241|917|6|14.27|16.69|16.18|0.00|97.08|85.62|100.14|6.79|0.00|97.08|103.87|11.46| +2451630|45801|6451|17353|1734807|5061|32468|2|122|917|80|54.89|63.12|35.34|56.54|2827.20|4391.20|5049.60|83.11|56.54|2770.66|2853.77|-1620.54| +2451630|45801|10844|17353|1734807|5061|32468|2|3|917|60|69.13|76.04|75.27|0.00|4516.20|4147.80|4562.40|90.32|0.00|4516.20|4606.52|368.40| +2451630|45801|4139|17353|1734807|5061|32468|2|26|917|16|14.27|16.69|15.68|0.00|250.88|228.32|267.04|20.07|0.00|250.88|270.95|22.56| +2451630||2023|17353||5061|32468|2||917|40||||||||||||-1891.20| +2451630|45801|6734|17353|1734807|5061|32468|2|8|917|48|25.01|30.26|15.43|622.13|740.64|1200.48|1452.48|3.55|622.13|118.51|122.06|-1081.97| +2451630|45801|11033|17353|1734807|5061|32468|2|295|917|41|94.76|170.56|69.92|2694.71|2866.72|3885.16|6992.96|5.16|2694.71|172.01|177.17|-3713.15| +2451630|45801|3515|17353|1734807|5061|32468|2|37|917|92|81.66|107.79|70.06|0.00|6445.52|7512.72|9916.68|515.64|0.00|6445.52|6961.16|-1067.20| +2451630|45801|16001|17353|1734807|5061|32468|2|49|917|17|11.97|14.72|5.59|0.00|95.03|203.49|250.24|3.80|0.00|95.03|98.83|-108.46| +2451630|45801|7519|17353|1734807|5061|32468|2|109|917|89|63.62|110.06|39.62|0.00|3526.18|5662.18|9795.34|105.78|0.00|3526.18|3631.96|-2136.00| +2451630|45801|1315|17353|1734807|5061|32468|2|219|917|78|45.12|75.80|40.17|0.00|3133.26|3519.36|5912.40|281.99|0.00|3133.26|3415.25|-386.10| +2451630|45801|12437|17353|1734807|5061|32468|2|172|917|94|51.34|59.55|47.04|0.00|4421.76|4825.96|5597.70|265.30|0.00|4421.76|4687.06|-404.20| +2451172|66748|3926|62702|842480|7058|41636|1|280|918|77|19.21|32.65|26.77|0.00|2061.29|1479.17|2514.05|144.29|0.00|2061.29|2205.58|582.12| +2451172|66748|7609|62702|842480|7058|41636|1|200|918|80|94.01|151.35|54.48|0.00|4358.40|7520.80|12108.00|130.75|0.00|4358.40|4489.15|-3162.40| +2451172|66748|6124|62702|842480|7058|41636|1|4|918|77|67.19|128.33|19.24|266.66|1481.48|5173.63|9881.41|72.88|266.66|1214.82|1287.70|-3958.81| +2451172|66748|4099|62702|842480|7058|41636|1|29|918|55|28.55|32.54|31.56|1562.22|1735.80|1570.25|1789.70|0.00|1562.22|173.58|173.58|-1396.67| +2451172|66748|17221|62702|842480|7058|41636|1|6|918|39|20.10|30.15|15.07|0.00|587.73|783.90|1175.85|41.14|0.00|587.73|628.87|-196.17| +2451172||11713|62702|842480||41636|||918|1||76.10|50.22|4.51|50.22|||0.91|4.51|||| +2451172|66748|15268|62702|842480|7058|41636|1|21|918|90|79.04|85.36|45.24|0.00|4071.60|7113.60|7682.40|203.58|0.00|4071.60|4275.18|-3042.00| +2451172|66748|4780|62702|842480|7058|41636|1|77|918|43|78.21|87.59|56.05|1397.88|2410.15|3363.03|3766.37|0.00|1397.88|1012.27|1012.27|-2350.76| +2451172|66748|14462|62702|842480|7058|41636|1|3|918|23|79.47|97.74|44.96|0.00|1034.08|1827.81|2248.02|41.36|0.00|1034.08|1075.44|-793.73| +2451172|66748|16406|62702|842480|7058|41636|1|150|918|26|66.72|118.09|98.01|0.00|2548.26|1734.72|3070.34|0.00|0.00|2548.26|2548.26|813.54| +2451172|66748|10312|62702|842480|7058|41636|1|37|918|86|78.63|152.54|50.33|0.00|4328.38|6762.18|13118.44|346.27|0.00|4328.38|4674.65|-2433.80| +2451172|66748|9112|62702|842480|7058|41636|1|272|918|22|20.93|41.23|24.32|0.00|535.04|460.46|907.06|0.00|0.00|535.04|535.04|74.58| +2451172|66748|8245|62702|842480|7058|41636|1|146|918|36|24.08|27.45|5.21|0.00|187.56|866.88|988.20|16.88|0.00|187.56|204.44|-679.32| +2451172|66748|11648|62702|842480|7058|41636|1|92|918|3|4.72|5.99|2.27|0.00|6.81|14.16|17.97|0.40|0.00|6.81|7.21|-7.35| +2451172|66748|13154|62702|842480|7058|41636|1|289|918|85|39.08|73.47|16.16|0.00|1373.60|3321.80|6244.95|96.15|0.00|1373.60|1469.75|-1948.20| +2451439|68263|6049|98454|1832625|5643|45814|4|108|919|64|37.38|42.23|27.02|881.93|1729.28|2392.32|2702.72|67.78|881.93|847.35|915.13|-1544.97| +2451439|68263|14546|98454|1832625|5643|45814|4|84|919|53|48.59|76.28|37.37|1762.74|1980.61|2575.27|4042.84|8.71|1762.74|217.87|226.58|-2357.40| +2451439|68263|6902|98454|1832625|5643|45814|4|259|919|58|29.29|46.86|9.84|0.00|570.72|1698.82|2717.88|22.82|0.00|570.72|593.54|-1128.10| +2451439|68263|8012|98454|1832625|5643|45814|4|229|919|44|91.76|165.16|133.77|0.00|5885.88|4037.44|7267.04|529.72|0.00|5885.88|6415.60|1848.44| +2451439|68263|6866|98454|1832625|5643|45814|4|39|919|75|9.32|15.00|7.80|0.00|585.00|699.00|1125.00|29.25|0.00|585.00|614.25|-114.00| +2451439|68263|1706|98454|1832625|5643|45814|4|295|919|45|15.34|24.08|16.85|0.00|758.25|690.30|1083.60|68.24|0.00|758.25|826.49|67.95| +2451439|68263|12910|98454|1832625|5643|45814|4|223|919|26|29.75|48.79|38.54|0.00|1002.04|773.50|1268.54|30.06|0.00|1002.04|1032.10|228.54| +2451439|68263|5335|98454|1832625|5643|45814|4|75|919|96|14.32|25.63|9.99|0.00|959.04|1374.72|2460.48|76.72|0.00|959.04|1035.76|-415.68| +2451439|68263|15604|98454|1832625|5643|45814|4|230|919|78|79.29|153.02|90.28|0.00|7041.84|6184.62|11935.56|352.09|0.00|7041.84|7393.93|857.22| +2451439|68263|3566|98454|1832625|5643|45814|4|47|919|98|62.84|76.66|62.86|0.00|6160.28|6158.32|7512.68|123.20|0.00|6160.28|6283.48|1.96| +2451439|68263|12982||1832625|5643||4|80|919|||145.00||0.00|2985.55||||0.00|2985.55|3194.53|| +2451439|68263|9115|98454|1832625|5643|45814|4|20|919|70|17.98|28.40|17.89|0.00|1252.30|1258.60|1988.00|112.70|0.00|1252.30|1365.00|-6.30| +2451439|68263|854|98454|1832625|5643|45814|4|222|919|76|47.62|85.23|24.71|0.00|1877.96|3619.12|6477.48|131.45|0.00|1877.96|2009.41|-1741.16| +2451439|68263|17113|98454|1832625|5643|45814|4|48|919|82|36.74|37.84|12.10|595.32|992.20|3012.68|3102.88|35.71|595.32|396.88|432.59|-2615.80| +2451788|67793|8435|64972|827560|4195|24783|7|191|920|81|75.37|128.88|55.41|0.00|4488.21|6104.97|10439.28|89.76|0.00|4488.21|4577.97|-1616.76| +2451788|67793|12944|64972|827560|4195|24783|7|95|920|73|9.18|12.76|2.55|150.78|186.15|670.14|931.48|2.12|150.78|35.37|37.49|-634.77| +2451788|67793|14471|64972|827560|4195|24783|7|171|920|76|3.91|7.15|2.28|0.00|173.28|297.16|543.40|10.39|0.00|173.28|183.67|-123.88| +2451788|67793|9157|64972|827560|4195|24783|7|238|920|15|28.04|41.21|0.82|0.00|12.30|420.60|618.15|0.98|0.00|12.30|13.28|-408.30| +2451788|67793|7298|64972|827560|4195|24783|7|188|920|25|63.45|119.28|52.48|0.00|1312.00|1586.25|2982.00|39.36|0.00|1312.00|1351.36|-274.25| +2451788|67793|15047|||4195|24783||89|920||||15.24|0.00||||0.00|0.00|381.00|381.00|| +2451788|67793|3775|64972|827560|4195|24783|7|69|920|63|78.18|138.37|78.87|0.00|4968.81|4925.34|8717.31|149.06|0.00|4968.81|5117.87|43.47| +2451788|67793|1433|64972|827560|4195|24783|7|109|920|94|1.47|2.80|2.66|37.50|250.04|138.18|263.20|14.87|37.50|212.54|227.41|74.36| +2451788|67793|6535|64972|827560|4195|24783|7|99|920|47|37.61|50.77|20.81|0.00|978.07|1767.67|2386.19|58.68|0.00|978.07|1036.75|-789.60| +2451788|67793|2725|64972|827560|4195|24783|7|82|920|63|89.58|113.76|65.98|0.00|4156.74|5643.54|7166.88|374.10|0.00|4156.74|4530.84|-1486.80| +2451788|67793|3055|64972|827560|4195|24783|7|141|920|23|25.75|28.06|0.84|0.19|19.32|592.25|645.38|0.00|0.19|19.13|19.13|-573.12| +2451788|67793|1727|64972|827560|4195|24783|7|96|920|5|52.34|74.32|17.09|0.00|85.45|261.70|371.60|1.70|0.00|85.45|87.15|-176.25| +2451788|67793|1028|64972|827560|4195|24783|7|51|920|22|60.36|108.04|68.06|0.00|1497.32|1327.92|2376.88|29.94|0.00|1497.32|1527.26|169.40| +2451788|67793|13469|64972|827560|4195|24783|7|150|920|38|89.65|162.26|63.28|0.00|2404.64|3406.70|6165.88|0.00|0.00|2404.64|2404.64|-1002.06| +2451088|49292|2125|20767|950664|4561|36505|2|40|921|51|96.99|130.93|86.41|0.00|4406.91|4946.49|6677.43|176.27|0.00|4406.91|4583.18|-539.58| +2451088|49292|14684|20767|950664|4561|36505|2|257|921|4|54.89|77.39|51.07|0.00|204.28|219.56|309.56|6.12|0.00|204.28|210.40|-15.28| +2451088|49292|1562|20767|950664|4561|36505|2|119|921|2|37.20|54.31|1.08|0.00|2.16|74.40|108.62|0.10|0.00|2.16|2.26|-72.24| +|49292|10670|20767|950664||36505|2||921|6|14.22|||0.00|48.84||162.96||0.00|48.84||-36.48| +2451088|49292|2416|20767|950664|4561|36505|2|248|921|42|80.44|146.40|54.16|1797.02|2274.72|3378.48|6148.80|4.77|1797.02|477.70|482.47|-2900.78| +2451088|49292|7567|20767|950664|4561|36505|2|87|921|93|60.63|100.64|8.05|0.00|748.65|5638.59|9359.52|14.97|0.00|748.65|763.62|-4889.94| +2451088|49292|8102|20767|950664|4561|36505|2|68|921|14|2.61|3.52|0.00|0.00|0.00|36.54|49.28|0.00|0.00|0.00|0.00|-36.54| +2451088|49292|310|20767|950664|4561|36505|2|273|921|1|84.74|91.51|20.13|0.00|20.13|84.74|91.51|1.00|0.00|20.13|21.13|-64.61| +2451088|49292|3038|20767|950664|4561|36505|2|127|921|34|27.67|37.07|15.94|0.00|541.96|940.78|1260.38|48.77|0.00|541.96|590.73|-398.82| +2451088|49292|160|20767|950664|4561|36505|2|88|921|71|62.28|120.82|16.91|0.00|1200.61|4421.88|8578.22|108.05|0.00|1200.61|1308.66|-3221.27| +2451088|49292|4388|20767|950664|4561|36505|2|227|921|89|50.43|67.57|35.81|1179.22|3187.09|4488.27|6013.73|0.00|1179.22|2007.87|2007.87|-2480.40| +2451088|49292|16918|20767|950664|4561|36505|2|170|921|19|53.27|100.68|55.37|0.00|1052.03|1012.13|1912.92|0.00|0.00|1052.03|1052.03|39.90| +2451088|49292|14636|20767|950664|4561|36505|2|284|921|53|49.99|77.98|20.27|0.00|1074.31|2649.47|4132.94|42.97|0.00|1074.31|1117.28|-1575.16| +2451088|49292|9562|20767|950664|4561|36505|2|209|921|18|14.43|19.04|18.84|0.00|339.12|259.74|342.72|0.00|0.00|339.12|339.12|79.38| +2451088|49292|5509|20767|950664|4561|36505|2|144|921|49|98.52|176.35|142.84|0.00|6999.16|4827.48|8641.15|0.00|0.00|6999.16|6999.16|2171.68| +2450980|32467|12721|85925|1094366|54|22942|7|21|922|8|76.61|94.99|2.84|0.00|22.72|612.88|759.92|0.45|0.00|22.72|23.17|-590.16| +2450980|32467|12740|85925|1094366|54|22942|7|97|922|52|3.08|3.41|1.22|0.00|63.44|160.16|177.32|1.26|0.00|63.44|64.70|-96.72| +2450980|32467|4028|85925|1094366|54|22942|7|214|922|71|39.64|60.25|5.42|0.00|384.82|2814.44|4277.75|15.39|0.00|384.82|400.21|-2429.62| +2450980|32467|17728|85925|1094366|54|22942|7|176|922|63|88.90|160.90|64.36|0.00|4054.68|5600.70|10136.70|0.00|0.00|4054.68|4054.68|-1546.02| +2450980|32467|17992|85925|1094366|54|22942|7|283|922|84|98.74|186.61|59.71|0.00|5015.64|8294.16|15675.24|250.78|0.00|5015.64|5266.42|-3278.52| +2450980|32467|682|85925|1094366|54|22942|7|107|922|6|30.86|60.48|21.16|0.00|126.96|185.16|362.88|6.34|0.00|126.96|133.30|-58.20| +2450980|32467|3752|85925|1094366|54|22942|7|147|922|17|55.09|84.28|0.00|0.00|0.00|936.53|1432.76|0.00|0.00|0.00|0.00|-936.53| +2450980|32467|10034|85925|1094366|54|22942|7|287|922|75|2.52|4.58|3.84|0.00|288.00|189.00|343.50|11.52|0.00|288.00|299.52|99.00| +2452522|67751|6349|46602|270293|5184|47275|2|45|923|65|91.84|114.80|64.28|0.00|4178.20|5969.60|7462.00|292.47|0.00|4178.20|4470.67|-1791.40| +2452522|67751|2419|46602|270293|5184|47275|2|84|923|34|41.70|77.97|45.22|753.36|1537.48|1417.80|2650.98|39.20|753.36|784.12|823.32|-633.68| +2452522|67751|11617|46602|270293|5184|47275|2|216|923|30|27.76|33.86|30.47|0.00|914.10|832.80|1015.80|27.42|0.00|914.10|941.52|81.30| +2452522|67751|11811|46602|270293|5184|47275|2|130|923|100|28.84|42.39|30.94|0.00|3094.00|2884.00|4239.00|247.52|0.00|3094.00|3341.52|210.00| +2452522|67751|11839|46602|270293|5184|47275|2|16|923|96|11.93|15.62|13.90|0.00|1334.40|1145.28|1499.52|26.68|0.00|1334.40|1361.08|189.12| +2452522|67751|5881|46602|270293|5184|47275|2|270|923|89|78.71|90.51|12.67|0.00|1127.63|7005.19|8055.39|78.93|0.00|1127.63|1206.56|-5877.56| +2452522|67751|16401|46602|270293|5184|47275|2|42|923|34|13.14|17.21|13.42|0.00|456.28|446.76|585.14|13.68|0.00|456.28|469.96|9.52| +2452522|67751|4476|46602|270293|5184|47275|2|291|923|37|91.85|131.34|17.07|0.00|631.59|3398.45|4859.58|6.31|0.00|631.59|637.90|-2766.86| +2452522|67751|3573|46602|270293|5184|47275|2|166|923|50|45.71|46.62|19.58|0.00|979.00|2285.50|2331.00|88.11|0.00|979.00|1067.11|-1306.50| +2452522|67751|591|46602|270293|5184|47275|2|131|923|56|69.46|90.29|10.83|0.00|606.48|3889.76|5056.24|42.45|0.00|606.48|648.93|-3283.28| +2452522|67751|6355|46602|270293|5184|47275|2|132|923|46|68.48|82.86|51.37|0.00|2363.02|3150.08|3811.56|47.26|0.00|2363.02|2410.28|-787.06| +2452522|67751|17313|46602|270293|5184|47275|2|54|923|5|47.42|91.99|84.63|29.62|423.15|237.10|459.95|7.87|29.62|393.53|401.40|156.43| +2452522|67751|8196|46602|270293|5184|47275|2|35|923|6|26.75|26.75|18.99|0.00|113.94|160.50|160.50|9.11|0.00|113.94|123.05|-46.56| +2452522|67751|8997|46602|270293|5184|47275|2|186|923|69|51.83|62.71|1.25|0.00|86.25|3576.27|4326.99|4.31|0.00|86.25|90.56|-3490.02| +2452522|67751|17905|46602|270293|5184|47275|2|34|923|88|36.38|40.01|7.20|0.00|633.60|3201.44|3520.88|0.00|0.00|633.60|633.60|-2567.84| +2452522|67751|11229|46602|270293|5184|47275|2|104|923|79|82.08|98.49|65.98|2658.33|5212.42|6484.32|7780.71|76.62|2658.33|2554.09|2630.71|-3930.23| +2452281|44829|10825|23739|947663|4034|23958|10|69|924|92|41.24|68.45|55.44|0.00|5100.48|3794.08|6297.40|459.04|0.00|5100.48|5559.52|1306.40| +2452281|44829|15549|23739|947663|4034|23958|10|222|924|99|95.20|189.44|130.71|0.00|12940.29|9424.80|18754.56|776.41|0.00|12940.29|13716.70|3515.49| +2452281|44829|9549|23739|947663|4034|23958|10|272|924|1|95.13|111.30|46.74|0.00|46.74|95.13|111.30|3.27|0.00|46.74|50.01|-48.39| +2452281|44829|1650|23739|947663|4034|23958|10|154|924|94|78.95|128.68|9.00|0.00|846.00|7421.30|12095.92|59.22|0.00|846.00|905.22|-6575.30| +2452281|44829|7044|23739|947663|4034|23958|10|152|924|48|32.27|63.89|32.58|0.00|1563.84|1548.96|3066.72|62.55|0.00|1563.84|1626.39|14.88| +2452281|44829|5262|23739|947663|4034|23958|10|173|924|8|99.01|113.86|67.17|0.00|537.36|792.08|910.88|21.49|0.00|537.36|558.85|-254.72| +2452281|44829|12121|23739|947663|4034|23958|10|23|924|52|34.50|43.47|41.73|0.00|2169.96|1794.00|2260.44|195.29|0.00|2169.96|2365.25|375.96| +2452281|44829|5604|23739|947663|4034|23958|10|48|924|43|51.75|84.35|20.24|0.00|870.32|2225.25|3627.05|43.51|0.00|870.32|913.83|-1354.93| +2452281|44829|14359|23739|947663|4034|23958|10|253|924|77|93.87|178.35|1.78|0.00|137.06|7227.99|13732.95|9.59|0.00|137.06|146.65|-7090.93| +2452281|44829|1465|23739|947663|4034|23958|10||924||||9.91||||1486.80|||51.14||-735.58| +2452281|44829|13098|23739|947663|4034|23958|10|91|924|91|77.64|140.52|99.76|0.00|9078.16|7065.24|12787.32|90.78|0.00|9078.16|9168.94|2012.92| +2452281|44829|11244|23739|947663|4034|23958|10|184|924|92|4.99|5.98|2.39|0.00|219.88|459.08|550.16|6.59|0.00|219.88|226.47|-239.20| +2452281|44829|16875|23739|947663|4034|23958|10|48|924|3|57.31|83.67|43.50|0.00|130.50|171.93|251.01|9.13|0.00|130.50|139.63|-41.43| +2452281|44829|4083|23739|947663|4034|23958|10|195|924|35|5.80|10.61|4.24|0.00|148.40|203.00|371.35|13.35|0.00|148.40|161.75|-54.60| +2451838|36711|7817|65883|803439|3395|10598|2|261|925|100|74.44|121.33|101.91|2140.11|10191.00|7444.00|12133.00|0.00|2140.11|8050.89|8050.89|606.89| +2451838|36711|6985|65883|803439|3395|10598|2|62|925|21|13.51|22.83|5.02|0.00|105.42|283.71|479.43|2.10|0.00|105.42|107.52|-178.29| +2451838|36711|12847|65883|803439|3395|10598|2|135|925|31|25.81|50.84|47.78|0.00|1481.18|800.11|1576.04|74.05|0.00|1481.18|1555.23|681.07| +2451838|36711|6206|65883|803439|3395|10598|2|223|925|57|34.77|57.02|1.71|0.00|97.47|1981.89|3250.14|8.77|0.00|97.47|106.24|-1884.42| +2451838|36711|1544|65883|803439|3395|10598|2|79|925|4|43.33|78.86|48.89|0.00|195.56|173.32|315.44|5.86|0.00|195.56|201.42|22.24| +2451838|36711|14054|65883|803439|3395|10598|2|186|925|60|88.35|103.36|75.45|0.00|4527.00|5301.00|6201.60|271.62|0.00|4527.00|4798.62|-774.00| +2451838|36711|7010|65883|803439|3395|10598|2|259|925|55|25.42|35.33|13.42|0.00|738.10|1398.10|1943.15|36.90|0.00|738.10|775.00|-660.00| +2451838||4232||803439|3395|10598|||925|||46.01||0.00||613.58|1012.22||0.00|465.52|507.41|| +2451838|36711|17252|65883|803439|3395|10598|2|20|925|30|48.00|71.04|49.72|0.00|1491.60|1440.00|2131.20|29.83|0.00|1491.60|1521.43|51.60| +2451838|36711|11587|65883|803439|3395|10598|2|87|925|52|42.88|75.89|2.27|0.00|118.04|2229.76|3946.28|7.08|0.00|118.04|125.12|-2111.72| +2451838|36711|15619|65883|803439|3395|10598|2|173|925|100|46.79|82.35|26.35|0.00|2635.00|4679.00|8235.00|210.80|0.00|2635.00|2845.80|-2044.00| +2451838|36711|17450|65883|803439|3395|10598|2|195|925|86|19.37|33.70|23.25|0.00|1999.50|1665.82|2898.20|0.00|0.00|1999.50|1999.50|333.68| +2451838|36711|2377|65883|803439|3395|10598|2|140|925|43|53.04|84.33|16.86|587.23|724.98|2280.72|3626.19|5.51|587.23|137.75|143.26|-2142.97| +2451838|36711|5179|65883|803439|3395|10598|2|8|925|90|59.68|73.40|60.92|0.00|5482.80|5371.20|6606.00|383.79|0.00|5482.80|5866.59|111.60| +2451875|69821|12253|4480|193032|1231|12183|10|205|926|85|81.06|149.96|2.99|0.00|254.15|6890.10|12746.60|10.16|0.00|254.15|264.31|-6635.95| +2451875|69821|4741|4480|193032|1231|12183|10|212|926|35|49.57|59.48|43.42|0.00|1519.70|1734.95|2081.80|15.19|0.00|1519.70|1534.89|-215.25| +2451875|69821|16520|4480|193032|1231|12183|10|57|926|98|88.07|96.87|61.02|0.00|5979.96|8630.86|9493.26|358.79|0.00|5979.96|6338.75|-2650.90| +2451875|69821|9235|4480|193032|1231|12183|10|265|926|81|86.63|97.02|11.64|0.00|942.84|7017.03|7858.62|65.99|0.00|942.84|1008.83|-6074.19| +2451875|69821|11870|4480|193032|1231|12183|10|119|926|75|34.82|61.28|23.89|0.00|1791.75|2611.50|4596.00|71.67|0.00|1791.75|1863.42|-819.75| +2451875|69821|3224|4480|193032|1231|12183|10|93|926|25|47.73|79.70|1.59|0.00|39.75|1193.25|1992.50|0.00|0.00|39.75|39.75|-1153.50| +2451875|69821|283|4480|193032|1231|12183|10|189|926|44|25.09|45.66|2.73|0.00|120.12|1103.96|2009.04|1.20|0.00|120.12|121.32|-983.84| +2451875|69821|8798|4480|193032|1231|12183|10|206|926|59|21.61|29.60|27.52|0.00|1623.68|1274.99|1746.40|16.23|0.00|1623.68|1639.91|348.69| +2451875|69821|13439|4480|193032|1231|12183|10|106|926|54|33.31|39.30|27.51|1307.27|1485.54|1798.74|2122.20|16.04|1307.27|178.27|194.31|-1620.47| +2451456|49983|8503|78392|1203419|446|7473|2|170|927|5|97.96|145.96|135.74|0.00|678.70|489.80|729.80|47.50|0.00|678.70|726.20|188.90| +2451456|49983|2227|78392|1203419|446|7473|2|266|927|56|31.36|31.98|20.14|135.34|1127.84|1756.16|1790.88|59.55|135.34|992.50|1052.05|-763.66| +2451456|49983|1351|78392|1203419|446|7473|2|264|927|25|12.01|14.89|14.89|0.00|372.25|300.25|372.25|26.05|0.00|372.25|398.30|72.00| +2451456|49983|4255|||||||927||36.17|53.53||||1699.99||24.40||2440.24|2464.64|740.25| +2451456|49983|2726|78392|1203419|446|7473|2|124|927|14|66.57|119.82|46.72|196.22|654.08|931.98|1677.48|36.62|196.22|457.86|494.48|-474.12| +2451456|49983|12806|78392|1203419|446|7473|2|195|927|82|8.93|16.07|3.05|0.00|250.10|732.26|1317.74|22.50|0.00|250.10|272.60|-482.16| +2451456|49983|17690|78392|1203419|446|7473|2|141|927|87|46.59|57.30|6.30|0.00|548.10|4053.33|4985.10|16.44|0.00|548.10|564.54|-3505.23| +2451456|49983|2612|78392|1203419|446|7473|2|170|927|18|38.67|63.80|49.76|0.00|895.68|696.06|1148.40|71.65|0.00|895.68|967.33|199.62| +|49983|11167|78392||446|7473|2|282|927|||21.46|||1111.79|1127.12||87.05||||| +2451456|49983|17086|78392|1203419|446|7473|2|207|927|33|50.08|84.13|47.95|0.00|1582.35|1652.64|2776.29|79.11|0.00|1582.35|1661.46|-70.29| +2451456|49983|2917|78392|1203419|446|7473|2|124|927|44|40.17|52.62|34.72|0.00|1527.68|1767.48|2315.28|45.83|0.00|1527.68|1573.51|-239.80| +2451456|49983|15122|78392|1203419|446|7473|2|36|927|19|39.72|61.56|16.62|0.00|315.78|754.68|1169.64|12.63|0.00|315.78|328.41|-438.90| +2452124|59342|12283|7477|71647|3370|30340|10|200|928|19|53.22|87.81|13.17|12.51|250.23|1011.18|1668.39|11.88|12.51|237.72|249.60|-773.46| +2452124|59342|7523|7477|71647|3370|30340|10|191|928|23|24.71|48.18|46.25|0.00|1063.75|568.33|1108.14|31.91|0.00|1063.75|1095.66|495.42| +2452124|59342|12747|7477|71647|3370|30340|10|52|928|93|93.23|140.77|18.30|0.00|1701.90|8670.39|13091.61|136.15|0.00|1701.90|1838.05|-6968.49| +2452124|59342|6747|7477|71647|3370|30340|10|219|928|69|4.57|5.30|4.61|0.00|318.09|315.33|365.70|9.54|0.00|318.09|327.63|2.76| +2452124|59342|4383|7477|71647|3370|30340|10|17|928|17|65.42|70.65|24.72|0.00|420.24|1112.14|1201.05|12.60|0.00|420.24|432.84|-691.90| +2452124|59342|13023|7477|71647|3370|30340|10|291|928|2|35.01|37.11|35.62|0.00|71.24|70.02|74.22|0.00|0.00|71.24|71.24|1.22| +2452124|59342|8069|7477|71647|3370|30340|10|108|928|12|24.56|47.89|3.83|0.00|45.96|294.72|574.68|0.45|0.00|45.96|46.41|-248.76| +2452124|59342|9469|7477|71647|3370|30340|10|205|928|12|61.57|78.80|52.79|0.00|633.48|738.84|945.60|19.00|0.00|633.48|652.48|-105.36| +2452124|59342|10003|7477|71647|3370|30340|10|229|928|5|64.37|83.03|69.74|0.00|348.70|321.85|415.15|0.00|0.00|348.70|348.70|26.85| +2452124|59342|12617|7477|71647|3370|30340|10|285|928|33|76.64|91.20|21.88|0.00|722.04|2529.12|3009.60|28.88|0.00|722.04|750.92|-1807.08| +2452124|59342|5931|7477|71647|3370|30340|10|265|928|84|99.20|182.52|5.47|0.00|459.48|8332.80|15331.68|18.37|0.00|459.48|477.85|-7873.32| +2452124|59342|1931|7477|71647|3370|30340|10|237|928|97|65.53|102.88|96.70|0.00|9379.90|6356.41|9979.36|844.19|0.00|9379.90|10224.09|3023.49| +2452124|59342|16863|7477|71647|3370|30340|10|273|928|20|99.85|165.75|56.35|0.00|1127.00|1997.00|3315.00|33.81|0.00|1127.00|1160.81|-870.00| +2452124|59342|5377|7477|71647|3370|30340|10|142|928|95|65.21|103.03|51.51|0.00|4893.45|6194.95|9787.85|244.67|0.00|4893.45|5138.12|-1301.50| +2452124|59342|9479|7477|||30340|||928|95|11.92||4.65|353.40|441.75||1924.70||353.40|88.35||-1044.05| +2452567|59112|6091|57384|1572034|2331|27381|2|24|929|88|87.78|174.68|66.37|0.00|5840.56|7724.64|15371.84|58.40|0.00|5840.56|5898.96|-1884.08| +2452567|59112|13981|57384|1572034|2331|27381|2|293|929|1|94.21|133.77|56.18|0.00|56.18|94.21|133.77|2.80|0.00|56.18|58.98|-38.03| +2452567|59112|9063|57384|1572034|2331|27381|2|282|929|92|88.19|146.39|0.00|0.00|0.00|8113.48|13467.88|0.00|0.00|0.00|0.00|-8113.48| +2452567|59112|8436|57384|1572034|2331|27381|2|248|929|29|50.67|53.20|1.06|0.00|30.74|1469.43|1542.80|1.84|0.00|30.74|32.58|-1438.69| +2452567|59112|12945|57384|1572034|2331|27381|2|81|929|48|5.81|8.88|8.52|0.00|408.96|278.88|426.24|32.71|0.00|408.96|441.67|130.08| +2452567|59112|14472|57384|1572034|2331|27381|2|133|929|3|3.58|5.29|1.90|0.00|5.70|10.74|15.87|0.39|0.00|5.70|6.09|-5.04| +2452567|59112|9157|57384|1572034|2331|27381|2|202|929|80|64.81|84.90|0.00|0.00|0.00|5184.80|6792.00|0.00|0.00|0.00|0.00|-5184.80| +2452567|59112|7299|57384|1572034|2331|27381|2|162|929|95|82.03|109.92|48.36|0.00|4594.20|7792.85|10442.40|413.47|0.00|4594.20|5007.67|-3198.65| +2452567|59112|15048|57384|1572034|2331|27381|2|203|929|15|56.73|74.88|34.44|0.00|516.60|850.95|1123.20|36.16|0.00|516.60|552.76|-334.35| +2452567|59112|3775|57384|1572034|2331|27381|2|3|929|43|35.19|36.24|18.12|0.00|779.16|1513.17|1558.32|0.00|0.00|779.16|779.16|-734.01| +2452567|59112|1434|57384|1572034|2331|27381|2|277|929|79|20.56|22.20|21.75|0.00|1718.25|1624.24|1753.80|137.46|0.00|1718.25|1855.71|94.01| +2452417|67040|10723|10488|1377994|2653|18512|2|53|930|32|79.98|157.56|69.32|0.00|2218.24|2559.36|5041.92|199.64|0.00|2218.24|2417.88|-341.12| +2452417|67040|4017|10488|1377994|2653|18512|2|27|930|3|8.33|8.49|2.80|0.00|8.40|24.99|25.47|0.00|0.00|8.40|8.40|-16.59| +2452417|67040|3919|10488|1377994|2653|18512|2|247|930|28|7.35|8.82|5.02|2.81|140.56|205.80|246.96|8.26|2.81|137.75|146.01|-68.05| +2452417|67040|9483|10488|1377994|2653|18512|2|225|930|93|42.61|84.36|23.62|0.00|2196.66|3962.73|7845.48|87.86|0.00|2196.66|2284.52|-1766.07| +2452417|67040|9435|10488|1377994|2653|18512|2|199|930|11|49.31|55.22|45.83|0.00|504.13|542.41|607.42|30.24|0.00|504.13|534.37|-38.28| +2452417|67040|11331|10488|1377994|2653|18512|2|108|930|61|24.35|46.75|10.28|0.00|627.08|1485.35|2851.75|56.43|0.00|627.08|683.51|-858.27| +2452417|67040|17274|10488|1377994|2653|18512|2|62|930|47|23.76|44.90|27.38|0.00|1286.86|1116.72|2110.30|115.81|0.00|1286.86|1402.67|170.14| +2452417|67040|4779|10488|1377994|2653|18512|2|190|930|44|77.48|151.08|146.54|0.00|6447.76|3409.12|6647.52|64.47|0.00|6447.76|6512.23|3038.64| +2452417|67040|11329|10488|1377994|2653|18512|2|205|930|54|14.98|28.76|28.47|0.00|1537.38|808.92|1553.04|30.74|0.00|1537.38|1568.12|728.46| +2452417|67040|7533|10488|1377994|2653|18512|2|105|930|64|51.56|75.79|31.83|0.00|2037.12|3299.84|4850.56|61.11|0.00|2037.12|2098.23|-1262.72| +2452417|67040|1677|10488|1377994|2653|18512|2|280|930|25|25.37|25.87|25.87|0.00|646.75|634.25|646.75|6.46|0.00|646.75|653.21|12.50| +2452417|67040|9813|10488|1377994|2653|18512|2|250|930|63|19.51|32.58|14.66|0.00|923.58|1229.13|2052.54|83.12|0.00|923.58|1006.70|-305.55| +2452417|67040|11841|10488|1377994|2653|18512|2|247|930|54|56.11|88.09|67.82|0.00|3662.28|3029.94|4756.86|146.49|0.00|3662.28|3808.77|632.34| +||7662||||18512|2||930|12|41.88||44.27||531.24|502.56||47.81||||28.68| +2452417|67040|12813|10488|1377994|2653|18512|2|148|930|44|6.05|9.01|7.11|0.00|312.84|266.20|396.44|18.77|0.00|312.84|331.61|46.64| +2452583|35639|11383|37052|1839287|3122|11757|10|107|931|24|33.60|34.94|12.92|0.00|310.08|806.40|838.56|21.70|0.00|310.08|331.78|-496.32| +2452583|35639|9877|37052|1839287|3122|11757|10|204|931|85|86.73|113.61|12.49|0.00|1061.65|7372.05|9656.85|0.00|0.00|1061.65|1061.65|-6310.40| +2452583|35639|5211|37052|1839287|3122|11757|10|56|931|41|31.12|45.43|24.07|0.00|986.87|1275.92|1862.63|29.60|0.00|986.87|1016.47|-289.05| +|35639|14059|37052|||||161|931||89.78|115.81||||||||||| +2452583|35639|10512|37052|1839287|3122|11757|10|243|931|71|62.66|62.66|62.03|0.00|4404.13|4448.86|4448.86|352.33|0.00|4404.13|4756.46|-44.73| +2452583|35639|7500|37052|1839287|3122|11757|10|59|931|6|14.80|19.24|0.57|0.00|3.42|88.80|115.44|0.20|0.00|3.42|3.62|-85.38| +2452583|35639|8473|37052|1839287|3122|11757|10|10|931|74|97.43|177.32|74.47|0.00|5510.78|7209.82|13121.68|495.97|0.00|5510.78|6006.75|-1699.04| +2452583|35639|15018|37052|1839287|3122|11757|10|98|931|76|36.78|41.56|1.66|118.59|126.16|2795.28|3158.56|0.60|118.59|7.57|8.17|-2787.71| +2451638|47029|1051|||3587|7299|||932|||55.14||0.00|4267.80|3703.50|4962.60|256.06|0.00|4267.80|4523.86|564.30| +2451638|47029|1075|75716|576275|3587|7299|7|164|932|11|75.03|108.04|88.59|945.25|974.49|825.33|1188.44|1.46|945.25|29.24|30.70|-796.09| +2451638|47029|4160|75716|576275|3587|7299|7|11|932|55|86.13|156.75|12.54|0.00|689.70|4737.15|8621.25|48.27|0.00|689.70|737.97|-4047.45| +2451638|47029|17672|75716|576275|3587|7299|7|81|932|25|80.16|81.76|49.87|1084.67|1246.75|2004.00|2044.00|12.96|1084.67|162.08|175.04|-1841.92| +2451638||7580|75716||3587||7||932|63|20.58|||0.00|1629.18|1296.54||0.00|0.00|1629.18|1629.18|332.64| +2451638|47029|9991|75716|576275|3587|7299|7|239|932|14|29.18|42.60|8.09|0.00|113.26|408.52|596.40|2.26|0.00|113.26|115.52|-295.26| +2451638|47029|4597|75716|576275|3587|7299|7|87|932|73|35.63|55.93|42.50|0.00|3102.50|2600.99|4082.89|0.00|0.00|3102.50|3102.50|501.51| +2451638|47029|7331|75716|576275|3587|7299|7|97|932|81|65.34|77.10|53.19|4049.88|4308.39|5292.54|6245.10|10.34|4049.88|258.51|268.85|-5034.03| +2451638|47029|15973|75716|576275|3587|7299|7|104|932|71|41.18|55.59|46.69|0.00|3314.99|2923.78|3946.89|132.59|0.00|3314.99|3447.58|391.21| +2451638|47029|6515|75716|576275|3587|7299|7|268|932|8|49.55|96.62|32.85|0.00|262.80|396.40|772.96|0.00|0.00|262.80|262.80|-133.60| +2451638|47029|15617|75716|576275|3587|7299|7|143|932|89|15.70|29.04|23.81|0.00|2119.09|1397.30|2584.56|148.33|0.00|2119.09|2267.42|721.79| +2451638|47029|10658|75716|576275|3587|7299|7|127|932|26|74.01|108.79|105.52|0.00|2743.52|1924.26|2828.54|219.48|0.00|2743.52|2963.00|819.26| +|47029|2605||576275|3587|7299||104|932||32.09|51.34||99.75||1058.97|||99.75|425.28||| +2451638|47029|15505|75716|576275|3587|7299|7|53|932|73|63.49|89.52|14.32|0.00|1045.36|4634.77|6534.96|20.90|0.00|1045.36|1066.26|-3589.41| +2451638|47029|10735|75716|576275|3587|7299|7|54|932|72|5.01|9.26|5.55|0.00|399.60|360.72|666.72|3.99|0.00|399.60|403.59|38.88| +2451638|47029|3992|75716|576275|3587|7299|7|138|932|31|68.62|72.05|37.46|0.00|1161.26|2127.22|2233.55|23.22|0.00|1161.26|1184.48|-965.96| +2452505|34228|16086|2249|51393|6584|34928|4|28|933|25|7.96|14.80|7.25|0.00|181.25|199.00|370.00|0.00|0.00|181.25|181.25|-17.75| +2452505|34228|6885|2249|51393|6584|34928|4|178|933|88|9.86|10.94|1.20|57.02|105.60|867.68|962.72|4.37|57.02|48.58|52.95|-819.10| +2452505|34228|11430|2249|51393|6584|34928|4|37|933|6|84.19|143.12|74.42|0.00|446.52|505.14|858.72|8.93|0.00|446.52|455.45|-58.62| +2452505|34228|13369|2249|51393|6584|34928|4|99|933|84|5.11|5.77|5.30|0.00|445.20|429.24|484.68|35.61|0.00|445.20|480.81|15.96| +2452505|34228|12009||51393|||4|97|933|||3.69|2.58||196.08|||||11.77|12.71|| +2452505|34228|12510|2249|51393|6584|34928|4|256|933|40|39.39|64.99|55.24|0.00|2209.60|1575.60|2599.60|22.09|0.00|2209.60|2231.69|634.00| +2452505|34228|16680|2249|51393|6584|34928|4|226|933|14|87.04|143.61|139.30|0.00|1950.20|1218.56|2010.54|19.50|0.00|1950.20|1969.70|731.64| +2452505|34228|499|2249|51393|6584|34928|4|162|933|26|78.91|101.79|34.60|0.00|899.60|2051.66|2646.54|44.98|0.00|899.60|944.58|-1152.06| +2452505|34228|13353|2249|51393|6584|34928|4|162|933|52|48.38|96.27|0.00|0.00|0.00|2515.76|5006.04|0.00|0.00|0.00|0.00|-2515.76| +2452505|34228|13227|2249|51393|6584|34928|4|205|933|64|16.73|32.12|25.37|0.00|1623.68|1070.72|2055.68|32.47|0.00|1623.68|1656.15|552.96| +2452505|34228|12702||51393|6584|34928||115|933||79.41|134.99||||||141.85||3546.46||| +2452505|34228|1176|2249|51393|6584|34928|4|91|933|28|30.57|50.74|43.12|0.00|1207.36|855.96|1420.72|96.58|0.00|1207.36|1303.94|351.40| +2452505|34228|14379|2249|51393|6584|34928|4|135|933|92|55.63|81.77|46.60|0.00|4287.20|5117.96|7522.84|342.97|0.00|4287.20|4630.17|-830.76| +2452505|34228|17808|2249|51393|6584|34928|4|110|933|41|99.58|154.34|132.73|0.00|5441.93|4082.78|6327.94|108.83|0.00|5441.93|5550.76|1359.15| +2452505|34228|459|2249|51393|6584|34928|4|255|933|33|75.28|137.76|5.51|30.91|181.83|2484.24|4546.08|12.07|30.91|150.92|162.99|-2333.32| +2452505||1209|2249|51393||34928||275|933||77.63|143.61|110.57|425.69|1216.27|||7.90|425.69|790.58||| +2451138|42280|344|71722|965686|1697|38861|10|80|934|91|13.45|24.07|13.96|0.00|1270.36|1223.95|2190.37|0.00|0.00|1270.36|1270.36|46.41| +2451138|42280|4009|71722|965686|1697|38861|10|259|934|36|71.39|114.22|113.07|0.00|4070.52|2570.04|4111.92|366.34|0.00|4070.52|4436.86|1500.48| +2451138|42280|7172|71722|965686|1697|38861|10|99|934|6|67.60|117.62|62.33|355.28|373.98|405.60|705.72|1.49|355.28|18.70|20.19|-386.90| +2451138|42280|4054|71722|965686|1697|38861|10|257|934|78|26.21|43.77|13.56|0.00|1057.68|2044.38|3414.06|95.19|0.00|1057.68|1152.87|-986.70| +2451138|42280|16046|71722|965686|1697|38861|10|188|934|2|75.32|146.87|96.93|0.00|193.86|150.64|293.74|17.44|0.00|193.86|211.30|43.22| +2451138|42280|10111|71722|965686|1697|38861|10|188|934|78|7.65|11.78|6.47|0.00|504.66|596.70|918.84|5.04|0.00|504.66|509.70|-92.04| +2451138|42280|8395|71722|965686|1697|38861|10|170|934|38|90.53|92.34|19.39|486.30|736.82|3440.14|3508.92|0.00|486.30|250.52|250.52|-3189.62| +2451138|42280|9968|71722|965686|1697|38861|10|57|934|91|53.53|56.20|39.90|0.00|3630.90|4871.23|5114.20|0.00|0.00|3630.90|3630.90|-1240.33| +2451138|42280|4384|71722|965686|1697|38861|10|89|934|37|3.49|5.23|3.71|15.09|137.27|129.13|193.51|0.00|15.09|122.18|122.18|-6.95| +2451138|42280|6578|71722|965686|1697|38861|10|197|934|46|4.88|6.58|0.46|0.00|21.16|224.48|302.68|0.00|0.00|21.16|21.16|-203.32| +2451138|42280|11722|71722|965686|1697|38861|10|62|934|67|74.65|125.41|10.03|127.68|672.01|5001.55|8402.47|48.98|127.68|544.33|593.31|-4457.22| +|42280|7789|71722||1697||10||934||||21.10||2110.00|2984.00|3103.00|||||-874.00| +2451138|42280|1880|71722|965686|1697|38861|10|80|934|46|30.55|32.38|27.19|0.00|1250.74|1405.30|1489.48|112.56|0.00|1250.74|1363.30|-154.56| +2451858|35674|13507|39456|135321|6386|48651|7|59|935|2|20.59|28.82|21.32|0.00|42.64|41.18|57.64|1.27|0.00|42.64|43.91|1.46| +2451858|35674|7016|39456|135321|6386|48651|7|295|935|10|66.73|128.78|57.95|0.00|579.50|667.30|1287.80|23.18|0.00|579.50|602.68|-87.80| +2451858|35674|2389|39456|135321|6386|48651|7|182|935|60|69.69|125.44|56.44|0.00|3386.40|4181.40|7526.40|304.77|0.00|3386.40|3691.17|-795.00| +2451858|35674|1477|39456|135321|6386|48651|7|245|935|72|75.01|111.01|51.06|0.00|3676.32|5400.72|7992.72|294.10|0.00|3676.32|3970.42|-1724.40| +2451858|35674|11432|39456|135321|6386|48651|7|154|935|73|42.32|62.63|33.82|0.00|2468.86|3089.36|4571.99|98.75|0.00|2468.86|2567.61|-620.50| +2451858|35674|10567|39456|135321|6386|48651|7|9|935|57|89.82|179.64|86.22|0.00|4914.54|5119.74|10239.48|393.16|0.00|4914.54|5307.70|-205.20| +2451858|35674|6740|39456|135321|6386|48651|7|38|935|19|36.99|66.58|9.98|0.00|189.62|702.81|1265.02|7.58|0.00|189.62|197.20|-513.19| +2451858|35674|13481|39456|135321|6386|48651|7|252|935|38|99.59|191.21|187.38|0.00|7120.44|3784.42|7265.98|284.81|0.00|7120.44|7405.25|3336.02| +2451858|35674|2845|39456|135321|6386|48651|7|272|935|28|91.15|113.93|110.51|0.00|3094.28|2552.20|3190.04|61.88|0.00|3094.28|3156.16|542.08| +2451858|35674|3401|39456|135321|6386|48651|7|289|935|61|53.81|82.86|37.28|0.00|2274.08|3282.41|5054.46|181.92|0.00|2274.08|2456.00|-1008.33| +2451858|35674|7025|39456|135321|6386|48651|7|11|935|31|52.40|96.94|84.33|0.00|2614.23|1624.40|3005.14|130.71|0.00|2614.23|2744.94|989.83| +2451858||9349|39456|||48651|7|300|935|||40.59||||300.72||||||-198.04| +2451868|70395|3715|63191|63351|311|22824|2|278|936|98|65.72|88.06|73.08|5084.90|7161.84|6440.56|8629.88|166.15|5084.90|2076.94|2243.09|-4363.62| +2451868||2534||63351|||2||936|25||57.09||9.13|456.50||||9.13|447.37|487.63|-467.63| +2451868|70395|15965|63191|63351|311|22824|2|179|936|82|5.62|10.05|8.64|467.59|708.48|460.84|824.10|0.00|467.59|240.89|240.89|-219.95| +2451868|70395|15764|63191|63351|311|22824|2|77|936|2|57.05|70.74|48.10|0.00|96.20|114.10|141.48|8.65|0.00|96.20|104.85|-17.90| +2451868|70395|9638|63191|63351|311|22824|2|91|936|39|87.85|93.12|57.73|0.00|2251.47|3426.15|3631.68|112.57|0.00|2251.47|2364.04|-1174.68| +2451868|70395|7717|63191|63351|311|22824|2|147|936|27|17.88|29.50|21.83|0.00|589.41|482.76|796.50|17.68|0.00|589.41|607.09|106.65| +2451868|70395|73|63191|63351|311|22824|2|271|936|||||0.00|4072.88|5085.36|||0.00||4235.79|| +2451868|70395|8270|63191|63351|311|22824|2|224|936|84|93.78|118.16|30.72|0.00|2580.48|7877.52|9925.44|103.21|0.00|2580.48|2683.69|-5297.04| +2451868|70395|23|63191|63351|311|22824|2|118|936|3|24.76|47.78|23.41|0.00|70.23|74.28|143.34|4.21|0.00|70.23|74.44|-4.05| +2451461|43062|2938|4727|1902811|4895|26979|4|238|937|27|62.40|84.86|79.76|0.00|2153.52|1684.80|2291.22|193.81|0.00|2153.52|2347.33|468.72| +2451461|43062|6199|4727|1902811|4895|26979|4|149|937|14|24.42|27.35|9.02|117.44|126.28|341.88|382.90|0.08|117.44|8.84|8.92|-333.04| +2451461|43062|1874|4727|1902811|4895|26979|4|205|937|32|44.49|80.97|9.71|0.00|310.72|1423.68|2591.04|9.32|0.00|310.72|320.04|-1112.96| +2451461|43062|10615|4727|1902811|4895|26979|4|133|937|87|51.71|89.45|6.26|0.00|544.62|4498.77|7782.15|10.89|0.00|544.62|555.51|-3954.15| +2451461|43062|1429|4727|1902811|4895|26979|4|162|937|58|74.16|96.40|13.49|0.00|782.42|4301.28|5591.20|70.41|0.00|782.42|852.83|-3518.86| +2451461|43062|3260|4727|1902811|4895|26979|4|253|937|78|47.92|91.04|2.73|0.00|212.94|3737.76|7101.12|2.12|0.00|212.94|215.06|-3524.82| +2451461|43062|11344|4727|1902811|4895|26979|4|276|937|31|30.49|55.18|22.07|0.00|684.17|945.19|1710.58|61.57|0.00|684.17|745.74|-261.02| +2451461|43062|13642|4727|1902811|4895|26979|4|124|937|50|93.91|179.36|114.79|0.00|5739.50|4695.50|8968.00|172.18|0.00|5739.50|5911.68|1044.00| +2451461|43062|17032|4727|1902811|4895|26979|4|72|937|92|92.66|112.11|59.41|0.00|5465.72|8524.72|10314.12|109.31|0.00|5465.72|5575.03|-3059.00| +2451167|31796|1462|8846|9937|1428|35319|2|9|938|13|89.98|137.66|0.00|0.00|0.00|1169.74|1789.58|0.00|0.00|0.00|0.00|-1169.74| +2451167|31796|11624|8846|9937|1428|35319|2|71|938|74|91.00|135.59|117.96|7943.42|8729.04|6734.00|10033.66|62.84|7943.42|785.62|848.46|-5948.38| +2451167|31796|14578|8846|9937|1428|35319|2|62|938|44|79.44|136.63|51.91|0.00|2284.04|3495.36|6011.72|68.52|0.00|2284.04|2352.56|-1211.32| +2451167|31796|8402|8846|9937|1428|35319|2|169|938|89|54.03|80.50|8.05|107.46|716.45|4808.67|7164.50|36.53|107.46|608.99|645.52|-4199.68| +2451167|31796|14732|8846|9937|1428|35319|2|171|938|47|79.82|159.64|15.96|0.00|750.12|3751.54|7503.08|15.00|0.00|750.12|765.12|-3001.42| +2451167|31796|5713|8846|9937|1428|35319|2|47|938|69|74.03|115.48|3.46|0.00|238.74|5108.07|7968.12|11.93|0.00|238.74|250.67|-4869.33| +2451167|31796|2329|8846|9937|1428|35319|2|282|938|13|68.06|114.34|35.44|0.00|460.72|884.78|1486.42|13.82|0.00|460.72|474.54|-424.06| +2451167|31796|10892|8846|9937|1428|35319|2|44|938|89|95.79|174.33|55.78|1141.81|4964.42|8525.31|15515.37|191.13|1141.81|3822.61|4013.74|-4702.70| +2451167|31796|3418|8846|9937|1428|35319|2|144|938|12|69.87|126.46|70.81|0.00|849.72|838.44|1517.52|59.48|0.00|849.72|909.20|11.28| +2451167||16699|8846|||35319||148|938||37.04|45.92||0.00|1261.16|1518.64|1882.72||0.00|1261.16|1273.77|-257.48| +2451167|31796|8506|8846|9937|1428|35319|2|128|938|18|34.33|44.97|31.92|0.00|574.56|617.94|809.46|0.00|0.00|574.56|574.56|-43.38| +2451167|31796|4874|8846|9937|1428|35319|2|27|938|63|18.36|30.11|22.88|0.00|1441.44|1156.68|1896.93|0.00|0.00|1441.44|1441.44|284.76| +2451167|31796|17824|8846|9937|1428|35319|2|137|938|45|28.89|46.51|1.39|45.03|62.55|1300.05|2092.95|0.35|45.03|17.52|17.87|-1282.53| +2451167|31796|7022|8846|9937|1428|35319|2|89|938|16|92.36|179.17|73.45|152.77|1175.20|1477.76|2866.72|51.12|152.77|1022.43|1073.55|-455.33| +2451102|62909|931|2803|419350|1850|43997|8|289|939|38|22.25|41.16|30.87|0.00|1173.06|845.50|1564.08|46.92|0.00|1173.06|1219.98|327.56| +2451102|62909|338|2803|419350|1850|43997|8|25|939|80|94.68|143.91|11.51|0.00|920.80|7574.40|11512.80|73.66|0.00|920.80|994.46|-6653.60| +2451102|62909|8854|2803|419350|1850|43997|8|78|939|72|55.12|93.15|75.45|2118.63|5432.40|3968.64|6706.80|132.55|2118.63|3313.77|3446.32|-654.87| +2451102|62909|15730|2803|419350|1850|43997|8|231|939|58|13.34|22.81|14.37|0.00|833.46|773.72|1322.98|25.00|0.00|833.46|858.46|59.74| +2451102|62909|15110|2803|419350|1850|43997|8|228|939|13|73.73|82.57|54.49|0.00|708.37|958.49|1073.41|21.25|0.00|708.37|729.62|-250.12| +2451102|62909|5956|2803|419350|1850|43997|8|146|939|57|89.84|160.81|133.47|0.00|7607.79|5120.88|9166.17|684.70|0.00|7607.79|8292.49|2486.91| +2451102|62909|11962|2803|419350|1850|43997|8|240|939|42|10.45|11.70|1.63|0.00|68.46|438.90|491.40|3.42|0.00|68.46|71.88|-370.44| +2451102|62909|9724|2803|419350|||||939|16|44.28||||||1013.12|||347.43||| +2451102|62909|8846|2803|419350|1850|43997|8|198|939|92|94.40|123.66|59.35|0.00|5460.20|8684.80|11376.72|491.41|0.00|5460.20|5951.61|-3224.60| +2451140|40942|16676|60478|847322|119|42516|7|181|940|2|10.78|15.63|10.47|6.07|20.94|21.56|31.26|0.74|6.07|14.87|15.61|-6.69| +2451140|40942|4525|60478|847322|119|42516|7|202|940|61|3.99|4.06|2.15|1.31|131.15|243.39|247.66|9.08|1.31|129.84|138.92|-113.55| +2451140|40942|4214|60478|847322|119|42516|7|145|940|20|5.58|8.37|1.42|0.00|28.40|111.60|167.40|0.56|0.00|28.40|28.96|-83.20| +2451140|40942|991|60478|847322|119|42516|7|228|940|34|74.32|126.34|37.90|0.00|1288.60|2526.88|4295.56|25.77|0.00|1288.60|1314.37|-1238.28| +2451140|40942|16279|60478|847322|119|42516|7|259|940|22|20.11|26.14|2.09|22.07|45.98|442.42|575.08|0.95|22.07|23.91|24.86|-418.51| +2451140|40942|16489|60478|847322|119|42516|7|245|940|44|52.21|80.92|50.97|0.00|2242.68|2297.24|3560.48|89.70|0.00|2242.68|2332.38|-54.56| +2451140|40942|1816|60478|847322|119|42516|7|196|940|10|49.65|61.56|11.08|0.00|110.80|496.50|615.60|1.10|0.00|110.80|111.90|-385.70| +2451140|40942|7573|60478|847322|119|42516|7|276|940|24|21.40|38.30|12.25|147.00|294.00|513.60|919.20|5.88|147.00|147.00|152.88|-366.60| +2451140|40942|7520|60478|847322|119|42516|7|4|940|48|70.06|134.51|127.78|0.00|6133.44|3362.88|6456.48|552.00|0.00|6133.44|6685.44|2770.56| +2451140|40942|5794|60478|847322|119|42516|7|279|940|33|48.48|93.56|60.81|0.00|2006.73|1599.84|3087.48|160.53|0.00|2006.73|2167.26|406.89| +2452240|61143|16931|51607|819267|2927|44187|8|6|941|89|5.95|9.40|6.48|0.00|576.72|529.55|836.60|23.06|0.00|576.72|599.78|47.17| +2452240||8613|51607|819267||44187|8|105|941|60|||||107.40||3594.60|1.70||||-3181.87| +2452240|61143|9191|51607|819267|2927|44187|8|273|941|94|17.45|32.98|7.25|640.61|681.50|1640.30|3100.12|2.04|640.61|40.89|42.93|-1599.41| +2452240|61143|10787|51607|819267|2927|44187|8|79|941|27|74.27|122.54|0.00|0.00|0.00|2005.29|3308.58|0.00|0.00|0.00|0.00|-2005.29| +2452240|61143|9163|51607|819267|2927|44187|8|200|941|53|94.40|176.52|49.42|0.00|2619.26|5003.20|9355.56|104.77|0.00|2619.26|2724.03|-2383.94| +2452240|61143|16389|51607|819267|2927|44187|8|250|941|40|48.11|80.34|53.02|466.57|2120.80|1924.40|3213.60|82.71|466.57|1654.23|1736.94|-270.17| +2452240|61143|17969|51607|819267|2927|44187|8|235|941|60|41.89|71.21|71.21|2563.56|4272.60|2513.40|4272.60|68.36|2563.56|1709.04|1777.40|-804.36| +2452240|61143|489|51607|819267|2927|44187|8|169|941|9|73.65|119.31|17.89|69.23|161.01|662.85|1073.79|0.91|69.23|91.78|92.69|-571.07| +2452240|61143|2225|51607|819267||44187|8|53|941|45|9.10||||593.55|409.50||9.97||||| +2452240|61143|1013|51607|819267|2927|44187|8|126|941|47|54.41|93.58|25.26|0.00|1187.22|2557.27|4398.26|35.61|0.00|1187.22|1222.83|-1370.05| +2452240|61143|13809|51607|819267|2927|44187|8|281|941|75|13.70|18.90|8.31|0.00|623.25|1027.50|1417.50|0.00|0.00|623.25|623.25|-404.25| +2452240|61143|2811|51607|819267|2927|44187|8|161|941|37|11.82|19.73|14.79|0.00|547.23|437.34|730.01|10.94|0.00|547.23|558.17|109.89| +||943||819267|2927|44187|||941|57||||0.00|||||0.00|||149.34| +2452240|61143|6019|51607|819267|2927|44187|8|248|941|35|22.12|38.48|20.39|0.00|713.65|774.20|1346.80|49.95|0.00|713.65|763.60|-60.55| +2452240|61143|14241|51607|819267|2927|44187|8|203|941|80|15.90|23.69|3.31|198.60|264.80|1272.00|1895.20|3.31|198.60|66.20|69.51|-1205.80| +2452240|61143|717|51607|819267|2927|44187|8|176|941|78|57.58|99.03|37.63|0.00|2935.14|4491.24|7724.34|29.35|0.00|2935.14|2964.49|-1556.10| +2451544|34984|16840|22888|147207|6726|27845|7|249|942|52|34.93|68.11|38.82|1796.58|2018.64|1816.36|3541.72|0.00|1796.58|222.06|222.06|-1594.30| +2451544|34984|12688|22888|147207|6726|27845|7|101|942|31|86.07|120.49|73.49|0.00|2278.19|2668.17|3735.19|45.56|0.00|2278.19|2323.75|-389.98| +2451544|34984|4825|22888|147207|6726|27845|7|75|942|60|29.29|31.63|12.33|0.00|739.80|1757.40|1897.80|44.38|0.00|739.80|784.18|-1017.60| +2451544|34984|17018|22888|147207|6726|27845|7|271|942|66|20.68|32.46|23.37|0.00|1542.42|1364.88|2142.36|46.27|0.00|1542.42|1588.69|177.54| +2451544|34984|15128|22888|147207|6726|27845|7|275|942|6|47.78|84.09|14.29|0.00|85.74|286.68|504.54|7.71|0.00|85.74|93.45|-200.94| +2451544|34984|2437|22888|147207|6726|27845|7|181|942|45|66.84|108.28|70.38|1330.18|3167.10|3007.80|4872.60|55.10|1330.18|1836.92|1892.02|-1170.88| +2451544|34984|580|22888|147207|6726|27845|7|135|942|96|68.10|108.27|10.82|903.68|1038.72|6537.60|10393.92|8.10|903.68|135.04|143.14|-6402.56| +2451544|34984|9680|22888|147207|6726|27845|7|105|942|24|35.63|64.49|23.21|0.00|557.04|855.12|1547.76|27.85|0.00|557.04|584.89|-298.08| +2451544|34984|17624|22888|147207|6726|27845|7|190|942|62|30.70|47.27|28.83|0.00|1787.46|1903.40|2930.74|53.62|0.00|1787.46|1841.08|-115.94| +2451544|34984|7718|22888|147207|6726|27845|7|195|942|55|12.34|16.65|4.16|0.00|228.80|678.70|915.75|11.44|0.00|228.80|240.24|-449.90| +2451544|34984|11644|22888|147207|6726|27845|7|236|942|73|68.67|69.35|67.26|0.00|4909.98|5012.91|5062.55|0.00|0.00|4909.98|4909.98|-102.93| +2451544|34984|15908|22888|147207|6726|27845|7|299|942|84|80.21|110.68|47.59|0.00|3997.56|6737.64|9297.12|159.90|0.00|3997.56|4157.46|-2740.08| +2452572|52732|12246|89693|1077023|706|10436|4|93|943|2|39.62|48.73|35.08|0.00|70.16|79.24|97.46|0.00|0.00|70.16|70.16|-9.08| +2452572|52732|7737|89693|1077023|706|10436|4|180|943|66|83.87|155.15|7.75|485.92|511.50|5535.42|10239.90|0.51|485.92|25.58|26.09|-5509.84| +2452572|52732|16128|89693|1077023|706|10436|4|40|943|22|69.83|128.48|48.82|225.54|1074.04|1536.26|2826.56|50.91|225.54|848.50|899.41|-687.76| +2452572|52732|7027|89693|1077023|706|10436|4|238|943|28|49.46|85.07|9.35|0.00|261.80|1384.88|2381.96|5.23|0.00|261.80|267.03|-1123.08| +2452572|52732|7615|89693|1077023|706|10436|4|287|943|60|18.38|29.22|4.96|0.00|297.60|1102.80|1753.20|17.85|0.00|297.60|315.45|-805.20| +2452572|52732|14913|89693|1077023|706|10436|4|136|943|85|14.73|16.20|1.13|0.00|96.05|1252.05|1377.00|2.88|0.00|96.05|98.93|-1156.00| +2452572|52732|5268|89693|1077023|706|10436|4|263|943|74|40.43|69.94|67.14|0.00|4968.36|2991.82|5175.56|298.10|0.00|4968.36|5266.46|1976.54| +2452572|52732|2070|89693|1077023|706|10436|4|20|943|20|74.55|77.53|41.86|0.00|837.20|1491.00|1550.60|33.48|0.00|837.20|870.68|-653.80| +2452572|52732|2232|89693|1077023|706|10436|4|105|943|41|74.49|93.11|4.65|0.00|190.65|3054.09|3817.51|7.62|0.00|190.65|198.27|-2863.44| +2452356|52864|1191|60865|274867|1669|27624|7|37|944|30|33.63|61.54|50.46|0.00|1513.80|1008.90|1846.20|105.96|0.00|1513.80|1619.76|504.90| +2452356|52864|15408|60865|274867|1669|27624|7|166|944|94|65.28|114.24|29.70|0.00|2791.80|6136.32|10738.56|251.26|0.00|2791.80|3043.06|-3344.52| +2452356|52864|6162|60865|274867|1669|27624|7|89|944|96|91.57|155.66|94.95|2825.71|9115.20|8790.72|14943.36|566.05|2825.71|6289.49|6855.54|-2501.23| +2452356|52864|10351|60865|274867|1669|27624|7|61|944|32|9.66|12.26|11.40|0.00|364.80|309.12|392.32|0.00|0.00|364.80|364.80|55.68| +2452356|52864|14736|60865|274867|1669|27624|7|33|944|19|57.22|84.11|63.08|0.00|1198.52|1087.18|1598.09|11.98|0.00|1198.52|1210.50|111.34| +2452356|52864|5659|60865|274867|1669|27624|7|43|944|81|92.00|116.84|70.10|0.00|5678.10|7452.00|9464.04|113.56|0.00|5678.10|5791.66|-1773.90| +2452356|52864|8437|60865|274867|1669|27624|7|45|944|3|68.09|130.73|90.20|0.00|270.60|204.27|392.19|2.70|0.00|270.60|273.30|66.33| +2452356|52864|11106|60865|274867|1669|27624|7|127|944|68|81.36|157.83|148.36|0.00|10088.48|5532.48|10732.44|605.30|0.00|10088.48|10693.78|4556.00| +2452356|52864|17370|60865|274867|1669|27624|7|64|944|50|85.98|120.37|111.94|0.00|5597.00|4299.00|6018.50|223.88|0.00|5597.00|5820.88|1298.00| +2452356|52864|13158|60865|274867|1669|27624|7|290|944|74|98.99|170.26|37.45|0.00|2771.30|7325.26|12599.24|166.27|0.00|2771.30|2937.57|-4553.96| +2452356|52864|14265|60865|274867|1669|27624|7|42|944|81|36.15|70.49|67.67|2411.75|5481.27|2928.15|5709.69|184.17|2411.75|3069.52|3253.69|141.37| +2452356|52864|11361|60865|274867|1669|27624|7|231|944|68|9.86|19.52|7.80|0.00|530.40|670.48|1327.36|42.43|0.00|530.40|572.83|-140.08| +2452356|52864|7596|60865|274867|1669|27624|7|202|944|72|46.07|75.55|27.95|0.00|2012.40|3317.04|5439.60|181.11|0.00|2012.40|2193.51|-1304.64| +2452356|52864|16417|60865|274867|1669|27624|7|50|944|39|14.89|17.57|9.66|0.00|376.74|580.71|685.23|7.53|0.00|376.74|384.27|-203.97| +2452356|52864|9307||274867|||7|269|944||70.01|80.51|42.67|0.00|||7004.37|222.73|0.00|3712.29|3935.02|| +2452506|29675|17635|32085|1899892|946|25553|4|161|945|48|5.63|8.50|7.39|340.53|354.72|270.24|408.00|1.27|340.53|14.19|15.46|-256.05| +2452506|29675|4866|32085|1899892|946|25553|4|136|945|2|41.56|56.10|33.09|0.00|66.18|83.12|112.20|0.00|0.00|66.18|66.18|-16.94| +2452506|29675|12939|32085|1899892|946|25553|4|129|945|17|89.74|157.04|34.54|0.00|587.18|1525.58|2669.68|17.61|0.00|587.18|604.79|-938.40| +2452506|29675|6759|32085|1899892|946|25553|4|273|945|34|97.98|154.80|0.00|0.00|0.00|3331.32|5263.20|0.00|0.00|0.00|0.00|-3331.32| +2452506|29675|9696|32085|1899892|946|25553|4|260|945|9|2.44|2.95|1.26|0.00|11.34|21.96|26.55|0.79|0.00|11.34|12.13|-10.62| +2452506||11562|32085||||4|153|945|75|75.23||68.54|||||51.40||5140.50|5191.90|-501.75| +2452506|29675|2172|32085|1899892|946|25553|4|294|945|79|50.64|86.59|29.44|0.00|2325.76|4000.56|6840.61|93.03|0.00|2325.76|2418.79|-1674.80| +2452506|29675|12549|32085|1899892|946|25553|4|55|945|86|53.74|103.71|71.55|0.00|6153.30|4621.64|8919.06|369.19|0.00|6153.30|6522.49|1531.66| +2452506|29675|13554|32085|1899892|946|25553|4|47|945|67|37.28|52.93|0.00|0.00|0.00|2497.76|3546.31|0.00|0.00|0.00|0.00|-2497.76| +|29675|12357|||946||4||945|||66.90|||||2943.60|||323.40|342.80|| +2452506|29675|17004|32085|1899892|946|25553|4|221|945|60|4.44|7.41|3.48|0.00|208.80|266.40|444.60|18.79|0.00|208.80|227.59|-57.60| +2452506|29675|12733|32085|1899892|946|25553|4|280|945|22|21.08|38.99|23.39|0.00|514.58|463.76|857.78|5.14|0.00|514.58|519.72|50.82| +2451124|65740|10288|71408|499168|4081|14594|8|30|946|69|72.05|90.06|27.91|0.00|1925.79|4971.45|6214.14|115.54|0.00|1925.79|2041.33|-3045.66| +2451124|65740|5858|71408|499168|4081|14594|8|201|946|62|76.72|128.88|86.34|0.00|5353.08|4756.64|7990.56|481.77|0.00|5353.08|5834.85|596.44| +2451124|65740|10108|71408|499168|4081|14594|8|148|946|66|12.01|24.02|13.45|656.89|887.70|792.66|1585.32|6.92|656.89|230.81|237.73|-561.85| +2451124|65740|9997|71408|499168|4081|14594|8|164|946|77|91.17|140.40|84.24|0.00|6486.48|7020.09|10810.80|64.86|0.00|6486.48|6551.34|-533.61| +2451124|65740|17246|71408|499168|4081|14594|8|32|946|94|15.00|26.85|5.90|404.85|554.60|1410.00|2523.90|10.48|404.85|149.75|160.23|-1260.25| +2451124|65740|9790|71408|499168|4081|14594|8|15|946|26|78.85|148.23|117.10|1096.05|3044.60|2050.10|3853.98|175.36|1096.05|1948.55|2123.91|-101.55| +2451124|65740|4144|71408|499168|4081|14594|8|276|946|72|80.77|151.03|110.25|0.00|7938.00|5815.44|10874.16|0.00|0.00|7938.00|7938.00|2122.56| +2451124|65740|8887|71408|499168|4081|14594|8|51|946|46|54.57|76.94|56.16|0.00|2583.36|2510.22|3539.24|0.00|0.00|2583.36|2583.36|73.14| +2451124|65740|6008|71408|499168|4081|14594|8|212|946|38|83.94|104.92|48.26|0.00|1833.88|3189.72|3986.96|55.01|0.00|1833.88|1888.89|-1355.84| +||457||||||109|946|47|45.93||61.75|0.00||2158.71|3583.28|116.09|0.00|2902.25|3018.34|743.54| +2451171|40420|11797|27582|346275|6215|21708|10|103|947|49|28.81|34.57|15.55|0.00|761.95|1411.69|1693.93|15.23|0.00|761.95|777.18|-649.74| +2451171|40420|4990|27582|346275|6215|21708|10|60|947|3|31.93|47.25|31.18|0.00|93.54|95.79|141.75|0.93|0.00|93.54|94.47|-2.25| +2451171|40420|12859|27582|346275|6215|21708|10|197|947|17|6.00|11.52|4.14|0.00|70.38|102.00|195.84|4.92|0.00|70.38|75.30|-31.62| +2451171|40420|6235|27582|346275|6215|21708|10|223|947|57|39.75|68.37|62.21|0.00|3545.97|2265.75|3897.09|212.75|0.00|3545.97|3758.72|1280.22| +2451171|40420|8497|27582|346275|6215|21708|10|241|947|62|53.90|75.46|15.84|0.00|982.08|3341.80|4678.52|9.82|0.00|982.08|991.90|-2359.72| +2451171|40420|5728|27582|346275|6215|21708|10|279|947|34|40.69|56.96|36.45|0.00|1239.30|1383.46|1936.64|24.78|0.00|1239.30|1264.08|-144.16| +2451171|40420|7987|27582|346275|6215|21708|10|266|947|24|96.80|137.45|68.72|0.00|1649.28|2323.20|3298.80|32.98|0.00|1649.28|1682.26|-673.92| +2451171|40420|5300|27582|346275|6215|21708|10|99|947|26|43.35|82.79|78.65|695.26|2044.90|1127.10|2152.54|26.99|695.26|1349.64|1376.63|222.54| +|40420|2860||346275|||10|7|947||80.03||21.13|0.00|||1223.64|11.62|0.00|232.43|244.05|-647.90| +2451171|40420|11108|27582|346275|6215|21708|10|99|947|3|64.03|124.21|110.54|112.75|331.62|192.09|372.63|6.56|112.75|218.87|225.43|26.78| +2451171|40420|6626|27582|346275|6215|21708|10|291|947|17|23.20|24.82|14.64|174.21|248.88|394.40|421.94|2.24|174.21|74.67|76.91|-319.73| +2452337|42423|17520|61307|1365066|1625|43384|2|42|948|23|18.50|34.59|6.22|0.00|143.06|425.50|795.57|10.01|0.00|143.06|153.07|-282.44| +2452337|42423|7111|61307|1365066|1625|43384|2|136|948|20|15.67|19.74|19.54|0.00|390.80|313.40|394.80|0.00|0.00|390.80|390.80|77.40| +2452337|42423|12210|61307|1365066|1625|43384|2|295|948|88|20.84|39.59|17.02|0.00|1497.76|1833.92|3483.92|104.84|0.00|1497.76|1602.60|-336.16| +2452337|42423|14023|61307|1365066|1625|43384|2|71|948|64|35.94|52.83|42.26|1379.36|2704.64|2300.16|3381.12|106.02|1379.36|1325.28|1431.30|-974.88| +||2899|61307|1365066|||2||948|||||||||15.31||||-1379.60| +2452337|42423|15069|61307|1365066|1625|43384|2|289|948|49|3.46|3.87|1.23|0.00|60.27|169.54|189.63|3.61|0.00|60.27|63.88|-109.27| +2452337|42423|1281|61307|1365066|1625|43384|2|74|948|62|16.79|18.30|2.19|0.00|135.78|1040.98|1134.60|8.14|0.00|135.78|143.92|-905.20| +2452337|42423|16081|61307|1365066|1625|43384|2|21|948|81|92.20|111.56|78.09|0.00|6325.29|7468.20|9036.36|442.77|0.00|6325.29|6768.06|-1142.91| +2450898|53697|9409|99585|194430|475|2120|4|139|949|79|27.34|53.58|1.60|0.00|126.40|2159.86|4232.82|3.79|0.00|126.40|130.19|-2033.46| +2450898|53697|6604|99585|194430|475|2120|4|161|949|49|98.32|112.08|34.74|0.00|1702.26|4817.68|5491.92|85.11|0.00|1702.26|1787.37|-3115.42| +2450898|53697|7126|99585|194430|475|2120|4|35|949|27|41.77|83.54|72.67|0.00|1962.09|1127.79|2255.58|19.62|0.00|1962.09|1981.71|834.30| +||17779||||2120|4||949|66||145.88||0.00|6161.76||9628.08|0.00|0.00|||| +2450898|53697|2890|99585|194430|475|2120|4|41|949|63|89.55|146.86|38.18|0.00|2405.34|5641.65|9252.18|24.05|0.00|2405.34|2429.39|-3236.31| +2450898|53697|859|99585|194430|475|2120|4|118|949|96|74.02|103.62|70.46|0.00|6764.16|7105.92|9947.52|270.56|0.00|6764.16|7034.72|-341.76| +2450898|53697|16168|99585|194430|475|2120|4|137|949|11|15.42|17.88|8.40|0.00|92.40|169.62|196.68|0.00|0.00|92.40|92.40|-77.22| +2450898|53697|16930|99585|194430|475|2120|4|206|949|98|75.08|108.86|94.70|0.00|9280.60|7357.84|10668.28|278.41|0.00|9280.60|9559.01|1922.76| +2450898|53697|8612|99585|194430|475|2120|4|213|949|32|43.42|58.61|20.51|393.79|656.32|1389.44|1875.52|7.87|393.79|262.53|270.40|-1126.91| +2450898|53697|9190|99585|194430|475|2120|4|284|949|74|5.12|5.42|0.48|0.00|35.52|378.88|401.08|0.35|0.00|35.52|35.87|-343.36| +2450898|53697|10786|99585|194430|475|2120|4|73|949|85|36.89|56.07|54.38|1848.92|4622.30|3135.65|4765.95|194.13|1848.92|2773.38|2967.51|-362.27| +2450898|53697|9163|99585|194430|475|2120|4|227|949|27|81.39|162.78|74.87|0.00|2021.49|2197.53|4395.06|161.71|0.00|2021.49|2183.20|-176.04| +2450898|53697|16388|99585|194430|475|2120|4|247|949|71|39.59|48.69|10.71|0.00|760.41|2810.89|3456.99|22.81|0.00|760.41|783.22|-2050.48| +2450898|53697|17968|99585|194430|475|2120|4|16|949|36|33.94|61.43|22.11|0.00|795.96|1221.84|2211.48|0.00|0.00|795.96|795.96|-425.88| +2450898|53697|488|99585|194430|475|2120|4|172|949|95|78.87|154.58|119.02|0.00|11306.90|7492.65|14685.10|0.00|0.00|11306.90|11306.90|3814.25| +2452308|73254|17811|80609|1465414|1866|22318|4|204|950|75|98.04|186.27|40.97|0.00|3072.75|7353.00|13970.25|92.18|0.00|3072.75|3164.93|-4280.25| +2452308|73254|13020|80609|1465414|1866|22318|4|229|950|39|98.31|122.88|18.43|0.00|718.77|3834.09|4792.32|14.37|0.00|718.77|733.14|-3115.32| +2452308|73254|15504|80609|1465414|1866|22318|4|141|950|27|55.01|86.36|45.77|0.00|1235.79|1485.27|2331.72|24.71|0.00|1235.79|1260.50|-249.48| +2452308|73254|2043|80609|1465414|1866|22318|4|200|950|50|7.11|14.14|1.13|0.00|56.50|355.50|707.00|3.39|0.00|56.50|59.89|-299.00| +2452308|73254|1374|80609|1465414|1866|22318|4|296|950|80|56.97|56.97|22.21|0.00|1776.80|4557.60|4557.60|35.53|0.00|1776.80|1812.33|-2780.80| +2452308|73254|13987|80609|1465414|1866|22318|4|125|950|69|82.05|130.45|7.82|0.00|539.58|5661.45|9001.05|43.16|0.00|539.58|582.74|-5121.87| +2452308|73254|16831|80609|1465414|1866|22318|4|251|950|63|33.08|52.26|39.71|0.00|2501.73|2084.04|3292.38|150.10|0.00|2501.73|2651.83|417.69| +2452308|73254|8637|80609|1465414|1866|22318|4|55|950|55|59.01|87.33|81.21|2188.60|4466.55|3245.55|4803.15|113.89|2188.60|2277.95|2391.84|-967.60| +2452308|73254|1981|80609|1465414|1866|22318|4|104|950|33|65.56|126.53|77.18|0.00|2546.94|2163.48|4175.49|76.40|0.00|2546.94|2623.34|383.46| +2452308|73254|14569|80609|1465414|1866|22318|4|255|950|52|18.07|28.18|6.76|0.00|351.52|939.64|1465.36|24.60|0.00|351.52|376.12|-588.12| +2452308|73254|4971|80609|1465414|1866|22318|4|170|950|89|96.90|184.11|114.14|4571.30|10158.46|8624.10|16385.79|391.10|4571.30|5587.16|5978.26|-3036.94| +2452308|73254|2514|80609|1465414|1866|22318|4|280|950|1|60.27|72.92|1.45|0.00|1.45|60.27|72.92|0.13|0.00|1.45|1.58|-58.82| +2452308|73254|3684|80609|1465414|1866|22318|4|58|950|43|3.40|6.05|1.08|24.61|46.44|146.20|260.15|1.52|24.61|21.83|23.35|-124.37| +2452308|73254|5970|80609|1465414|1866|22318|4|78|950|40|40.83|55.52|31.64|658.11|1265.60|1633.20|2220.80|0.00|658.11|607.49|607.49|-1025.71| +2451108|65559|12958|75004|1008205|7006|24607|7|144|951|27|17.89|27.01|2.70|0.00|72.90|483.03|729.27|6.56|0.00|72.90|79.46|-410.13| +2451108|65559|9733|75004|1008205|7006|24607|7|177|951|74|38.62|40.55|27.57|0.00|2040.18|2857.88|3000.70|102.00|0.00|2040.18|2142.18|-817.70| +|65559|9550||1008205|7006|24607|7|280|951|42|78.46|101.21|40.48|||||0.00|||1700.16|-1595.16| +2451108|65559|1843|75004|1008205|7006|24607|7|159|951|95|94.25|121.58|46.20|0.00|4389.00|8953.75|11550.10|87.78|0.00|4389.00|4476.78|-4564.75| +2451108|65559|5470|75004|1008205|7006|24607|7|9|951|36|41.35|71.94|3.59|0.00|129.24|1488.60|2589.84|2.58|0.00|129.24|131.82|-1359.36| +2451108|65559|4556|75004|1008205|7006|24607|7|145|951|86|28.50|57.00|37.62|0.00|3235.32|2451.00|4902.00|97.05|0.00|3235.32|3332.37|784.32| +2451108|65559|11299|75004|1008205|7006|24607|7|209|951|19|30.49|57.01|37.05|0.00|703.95|579.31|1083.19|56.31|0.00|703.95|760.26|124.64| +2451108|65559|9250|75004|1008205|7006|24607|7|235|951|48|98.27|183.76|47.77|0.00|2292.96|4716.96|8820.48|206.36|0.00|2292.96|2499.32|-2424.00| +2451108|65559|16498|75004|1008205|7006|24607|7|40|951|76|4.28|7.87|2.04|110.07|155.04|325.28|598.12|2.24|110.07|44.97|47.21|-280.31| +2451108|65559|13|75004|1008205|7006|24607|7|130|951|1|71.89|143.78|97.77|0.00|97.77|71.89|143.78|6.84|0.00|97.77|104.61|25.88| +2451108|65559|1894|75004|1008205|7006|24607|7|158|951|47|71.82|94.80|27.49|426.36|1292.03|3375.54|4455.60|25.97|426.36|865.67|891.64|-2509.87| +2451108|65559|1736|75004|1008205|7006|24607|7|224|951|19|14.85|24.20|5.32|0.00|101.08|282.15|459.80|3.03|0.00|101.08|104.11|-181.07| +2451108|65559|9188|75004|1008205|7006|24607|7|254|951|89|93.24|142.65|142.65|0.00|12695.85|8298.36|12695.85|507.83|0.00|12695.85|13203.68|4397.49| +2451449|53100|13672|23538|1597314|1937|28513|7|228|952|39|42.66|68.68|42.58|0.00|1660.62|1663.74|2678.52|83.03|0.00|1660.62|1743.65|-3.12| +2451449|53100|9928|23538|1597314|1937|28513|7|293|952|66|96.81|175.22|113.89|3157.03|7516.74|6389.46|11564.52|261.58|3157.03|4359.71|4621.29|-2029.75| +2451449|53100|3368|23538|1597314|1937|28513|7|292|952|10|76.01|145.93|128.41|0.00|1284.10|760.10|1459.30|77.04|0.00|1284.10|1361.14|524.00| +2451449|53100|15806|23538|1597314|1937|28513|7|13|952|52|62.54|73.79|28.04|0.00|1458.08|3252.08|3837.08|116.64|0.00|1458.08|1574.72|-1794.00| +2451449|53100|5395|23538|1597314|1937|28513|7|232|952|69|89.39|164.47|72.36|3245.34|4992.84|6167.91|11348.43|104.85|3245.34|1747.50|1852.35|-4420.41| +2451449|53100|7000|23538|1597314|1937|28513|7|289|952|75|82.07|123.10|52.93|0.00|3969.75|6155.25|9232.50|79.39|0.00|3969.75|4049.14|-2185.50| +2451449|53100|1585|23538|1597314|1937|28513|7|86|952|96|72.80|145.60|101.92|4305.10|9784.32|6988.80|13977.60|493.12|4305.10|5479.22|5972.34|-1509.58| +2451449|53100|12530|23538|1597314|1937|28513|7|281|952|93|27.33|28.42|12.50|0.00|1162.50|2541.69|2643.06|58.12|0.00|1162.50|1220.62|-1379.19| +2451449||11872||1597314||28513|7|233|952|||||0.00|||8592.30|474.28|0.00||8378.98|| +2451449|53100|2852|23538|1597314|1937|28513|7|191|952|24|53.06|93.38|28.94|0.00|694.56|1273.44|2241.12|55.56|0.00|694.56|750.12|-578.88| +2451793||13069||163202|45|4135|8|23|953|||||626.24|782.80|578.80|1151.60|1.56|626.24|156.56|158.12|| +2451793|46965|4507|66712|163202|45|4135|8|3|953|52|79.04|124.88|24.97|0.00|1298.44|4110.08|6493.76|0.00|0.00|1298.44|1298.44|-2811.64| +2451793|46965|13844|66712|163202|45|4135|8|277|953|84|60.53|76.26|44.99|0.00|3779.16|5084.52|6405.84|37.79|0.00|3779.16|3816.95|-1305.36| +2451793|46965|10175|66712|163202|45|4135|8|77|953|74|37.01|62.17|57.81|0.00|4277.94|2738.74|4600.58|256.67|0.00|4277.94|4534.61|1539.20| +2451793|46965|769|66712|163202|45|4135|8|161|953|11|15.82|22.30|11.37|0.00|125.07|174.02|245.30|10.00|0.00|125.07|135.07|-48.95| +2451793|46965|11417|66712|163202|45|4135|8|87|953|60|3.78|6.69|1.33|51.07|79.80|226.80|401.40|0.86|51.07|28.73|29.59|-198.07| +2451793|46965|7253|66712|163202|45|4135|8|245|953|69|37.31|72.75|66.93|0.00|4618.17|2574.39|5019.75|46.18|0.00|4618.17|4664.35|2043.78| +2451793|46965|6247|66712|163202|45|4135|8|268|953|1|36.80|46.00|12.42|0.00|12.42|36.80|46.00|0.12|0.00|12.42|12.54|-24.38| +2451793|46965|2167|66712|163202|45|4135|8|104|953|13|46.69|77.50|2.32|15.98|30.16|606.97|1007.50|0.85|15.98|14.18|15.03|-592.79| +2451793|46965|17941|66712|163202|45|4135|8|29|953|51|27.36|33.37|17.01|0.00|867.51|1395.36|1701.87|26.02|0.00|867.51|893.53|-527.85| +2451793|46965|9440|66712|163202|45|4135|8|232|953|41|16.93|29.62|22.80|0.00|934.80|694.13|1214.42|9.34|0.00|934.80|944.14|240.67| +2451793|46965|17537|66712|163202|45|4135|8|172|953|74|58.06|116.12|2.32|0.00|171.68|4296.44|8592.88|8.58|0.00|171.68|180.26|-4124.76| +2451793|46965|7445|66712|163202|45|4135|8|168|953|43|30.96|52.32|24.06|0.00|1034.58|1331.28|2249.76|51.72|0.00|1034.58|1086.30|-296.70| +2451793|46965|11573|66712|163202|45|4135|8|244|953|40|29.52|48.41|0.48|0.00|19.20|1180.80|1936.40|0.76|0.00|19.20|19.96|-1161.60| +2451793|46965|1952|66712|163202|45|4135|8|6|953|39|98.20|140.42|134.80|0.00|5257.20|3829.80|5476.38|210.28|0.00|5257.20|5467.48|1427.40| +2451793|46965|1970|66712|163202|45|4135|8|183|953|60|47.57|75.63|45.37|244.99|2722.20|2854.20|4537.80|74.31|244.99|2477.21|2551.52|-376.99| +2451129|51818|15026|65409|1583594|979|33564|4|87|954|13|91.04|162.96|37.48|82.83|487.24|1183.52|2118.48|24.26|82.83|404.41|428.67|-779.11| +2451129|51818|9715|65409|1583594|979|33564|4|229|954|10|51.65|82.12|35.31|123.58|353.10|516.50|821.20|4.59|123.58|229.52|234.11|-286.98| +2451129|51818|2846|65409|1583594|979|33564|4|50|954|81|56.52|61.04|7.93|0.00|642.33|4578.12|4944.24|6.42|0.00|642.33|648.75|-3935.79| +2451129||242|65409|1583594||33564||208|954|56|66.07||56.25||3150.00|3699.92||220.50||||| +2451129|51818|12739|65409|1583594|979|33564|4|155|954|7|26.22|42.21|10.97|0.00|76.79|183.54|295.47|3.83|0.00|76.79|80.62|-106.75| +2451129|51818|15283|65409|1583594|979|33564|4|79|954|44|85.35|121.19|95.74|0.00|4212.56|3755.40|5332.36|252.75|0.00|4212.56|4465.31|457.16| +2451129|51818|9836|65409|1583594|979|33564|4|91|954|48|76.59|119.48|76.46|0.00|3670.08|3676.32|5735.04|36.70|0.00|3670.08|3706.78|-6.24| +2451129|51818|11114|65409|1583594|979|33564|4|49|954|100|63.83|116.17|17.42|0.00|1742.00|6383.00|11617.00|52.26|0.00|1742.00|1794.26|-4641.00| +2451351|52289|8306|33224|1698239|5207|32975|2|82|955|95|17.07|25.94|0.00|0.00|0.00|1621.65|2464.30|0.00|0.00|0.00|0.00|-1621.65| +2451351|52289|13615|33224|1698239|5207|32975|2|4|955|28|4.32|6.69|4.61|0.00|129.08|120.96|187.32|3.87|0.00|129.08|132.95|8.12| +2451351|52289|5044|33224|1698239|5207|32975|2|286|955|86|82.81|164.79|1.64|0.00|141.04|7121.66|14171.94|7.05|0.00|141.04|148.09|-6980.62| +2451351|52289|13270|33224|1698239|5207|32975|2|215|955|30|70.94|133.36|117.35|0.00|3520.50|2128.20|4000.80|211.23|0.00|3520.50|3731.73|1392.30| +2451351|52289|13891|33224|1698239|5207|32975|2|19|955|22|26.40|44.35|36.81|194.35|809.82|580.80|975.70|36.92|194.35|615.47|652.39|34.67| +2451351|52289|5944|33224|1698239|5207|32975|2||955|75|42.54||||||5934.00||||4236.75|844.50| +2451351|52289|9475|33224|1698239|5207|32975|2|118|955|27|19.83|29.34|1.17|18.00|31.59|535.41|792.18|0.00|18.00|13.59|13.59|-521.82| +2451351|52289|12193|33224|1698239|5207|32975|2|239|955|76|40.47|48.15|45.26|928.73|3439.76|3075.72|3659.40|75.33|928.73|2511.03|2586.36|-564.69| +2451351|52289|7501|33224|1698239|5207|32975|2|241|955|73|97.71|117.25|22.27|1089.22|1625.71|7132.83|8559.25|42.91|1089.22|536.49|579.40|-6596.34| +2451351|52289|13825|||5207||2||955||88.24|143.83|||||3164.26|||1044.12|1117.20|-897.16| +2451351|52289|6469|33224|1698239|5207|32975|2|177|955|81|94.20|144.12|23.05|0.00|1867.05|7630.20|11673.72|0.00|0.00|1867.05|1867.05|-5763.15| +2451351|52289|11602|33224|1698239|5207|32975|2|187|955|9|3.03|4.87|3.50|14.49|31.50|27.27|43.83|1.19|14.49|17.01|18.20|-10.26| +2452574|56474|1017|56397|1186908|2666|34024|2|106|956|40|6.85|11.78|6.00|0.00|240.00|274.00|471.20|19.20|0.00|240.00|259.20|-34.00| +2452574|56474|9289|56397|1186908|2666|34024|2|142|956|97|7.32|12.95|7.77|0.00|753.69|710.04|1256.15|15.07|0.00|753.69|768.76|43.65| +2452574|56474|6463|56397|1186908|2666|34024|2|39|956|87|86.59|108.23|96.32|0.00|8379.84|7533.33|9416.01|418.99|0.00|8379.84|8798.83|846.51| +2452574|56474|1705|56397|1186908|2666|34024|2|36|956|48|65.57|104.25|32.31|0.00|1550.88|3147.36|5004.00|62.03|0.00|1550.88|1612.91|-1596.48| +2452574|56474|11364|56397|1186908|2666|34024|2|236|956|28|92.57|171.25|29.11|0.00|815.08|2591.96|4795.00|8.15|0.00|815.08|823.23|-1776.88| +2452574|56474|7080|56397|1186908|2666|34024|2|31|956|54|4.52|7.09|4.75|94.90|256.50|244.08|382.86|0.00|94.90|161.60|161.60|-82.48| +2452574|56474|10605|56397|1186908|2666|34024|2|232|956|54|30.60|56.30|9.00|0.00|486.00|1652.40|3040.20|9.72|0.00|486.00|495.72|-1166.40| +2452574|56474|10213|56397|1186908|2666|34024|2|164|956|90|88.40|122.87|31.94|0.00|2874.60|7956.00|11058.30|258.71|0.00|2874.60|3133.31|-5081.40| +2452624|48591|3048|56178|955810|242|43474|2|247|957|15|86.43|146.93|49.95|0.00|749.25|1296.45|2203.95|59.94|0.00|749.25|809.19|-547.20| +2452624|48591|2251|56178|955810|242|43474|2|126|957|61|47.17|65.09|20.17|0.00|1230.37|2877.37|3970.49|86.12|0.00|1230.37|1316.49|-1647.00| +2452624|48591|17058|56178|955810|242|43474|2|277|957|80|2.81|5.45|4.03|19.34|322.40|224.80|436.00|18.18|19.34|303.06|321.24|78.26| +2452624|48591|10975|56178||||||957|29|11.93|||0.00|||||0.00|||218.95| +2452624|48591|1893|56178|955810|242|43474|2|33|957|95|19.42|29.51|10.62|0.00|1008.90|1844.90|2803.45|60.53|0.00|1008.90|1069.43|-836.00| +2452624|48591|15438|56178|955810|242|43474|2|288|957|100|42.54|82.95|11.61|0.00|1161.00|4254.00|8295.00|46.44|0.00|1161.00|1207.44|-3093.00| +2452624|48591|3318|56178|955810|242|43474|2|38|957|100|96.28|150.19|75.09|0.00|7509.00|9628.00|15019.00|150.18|0.00|7509.00|7659.18|-2119.00| +2452624|48591|2211|56178|955810|242|43474|2|16|957|57|54.21|69.38|39.54|0.00|2253.78|3089.97|3954.66|180.30|0.00|2253.78|2434.08|-836.19| +2452624|48591|12805|56178|955810|242|43474|2|3|957|18|4.31|4.99|1.54|0.00|27.72|77.58|89.82|1.10|0.00|27.72|28.82|-49.86| +2452624|48591|7249|56178|955810|242|43474|2|86|957|6|70.83|71.53|71.53|184.54|429.18|424.98|429.18|0.00|184.54|244.64|244.64|-180.34| +2452624|48591|3162|56178|955810|242|43474|2|209|957|64|18.13|22.66|9.51|353.01|608.64|1160.32|1450.24|5.11|353.01|255.63|260.74|-904.69| +2452624|48591|10141|56178|955810|242|43474|2|126|957|84|55.20|77.83|14.78|0.00|1241.52|4636.80|6537.72|74.49|0.00|1241.52|1316.01|-3395.28| +2452624|48591|1302|56178|955810|242|43474|2|111|957|97|77.17|132.73|71.67|0.00|6951.99|7485.49|12874.81|347.59|0.00|6951.99|7299.58|-533.50| +2452005|68266|13767|64796|1605028|5260|2692|10|129|958|79|80.27|141.27|120.07|0.00|9485.53|6341.33|11160.33|189.71|0.00|9485.53|9675.24|3144.20| +2452005|68266|7669|64796|1605028|5260|2692|10|59|958|84|2.21|3.22|2.41|0.00|202.44|185.64|270.48|0.00|0.00|202.44|202.44|16.80| +2452005|68266|5017|64796|1605028|5260|2692|10|136|958|12|10.84|14.85|4.60|0.00|55.20|130.08|178.20|3.86|0.00|55.20|59.06|-74.88| +2452005|68266|4521|64796|1605028|5260|2692|10|134|958|37|32.43|55.45|7.76|0.00|287.12|1199.91|2051.65|25.84|0.00|287.12|312.96|-912.79| +2452005|68266|5751|64796|1605028|5260|2692|10|282|958|100|44.12|82.94|21.56|0.00|2156.00|4412.00|8294.00|129.36|0.00|2156.00|2285.36|-2256.00| +2452005|68266|10781|64796|1605028|5260|2692|10|234|958|5|41.13|56.34|29.29|0.00|146.45|205.65|281.70|13.18|0.00|146.45|159.63|-59.20| +|68266|11643|64796||5260|||109|958|5||30.41|||68.40|118.80|152.05|2.05||68.40||-50.40| +2452005|68266|16519|64796|1605028|5260|2692|10|279|958|32|84.06|163.07|53.81|981.49|1721.92|2689.92|5218.24|22.21|981.49|740.43|762.64|-1949.49| +2452005|68266|6863|64796|1605028|5260|2692|10|275|958|100|95.59|157.72|130.90|0.00|13090.00|9559.00|15772.00|785.40|0.00|13090.00|13875.40|3531.00| +2452005|68266|17269|64796|1605028|5260|2692|10|136|958|83|15.92|21.01|16.38|367.07|1359.54|1321.36|1743.83|69.47|367.07|992.47|1061.94|-328.89| +2452005|68266|15109|64796|1605028|5260|2692|10|65|958|42|94.63|95.57|93.65|0.00|3933.30|3974.46|4013.94|196.66|0.00|3933.30|4129.96|-41.16| +2452005|68266|247|64796|1605028|5260|2692|10|44|958|68|49.48|67.78|58.29|0.00|3963.72|3364.64|4609.04|237.82|0.00|3963.72|4201.54|599.08| +2452005|68266|16585|64796|1605028|5260|2692|10|259|958|33|12.37|24.12|21.46|0.00|708.18|408.21|795.96|0.00|0.00|708.18|708.18|299.97| +2451052|43343|10586|65860|1709930|6800|9270|4|269|959|65|32.66|58.46|19.87|0.00|1291.55|2122.90|3799.90|25.83|0.00|1291.55|1317.38|-831.35| +2451052|43343|2906|65860|1709930|6800|9270|4|191|959|25|21.54|32.52|3.25|0.00|81.25|538.50|813.00|0.00|0.00|81.25|81.25|-457.25| +2451052|43343|11576|65860|1709930|6800|9270|4|115|959|18|21.87|34.33|22.65|0.00|407.70|393.66|617.94|16.30|0.00|407.70|424.00|14.04| +||4862||1709930||9270||298|959|||32.26||||||25.54||851.40|876.94|| +2451052|43343|4129|65860|1709930|6800|9270|4|16|959|14|40.43|63.47|9.52|0.00|133.28|566.02|888.58|1.33|0.00|133.28|134.61|-432.74| +2451052|43343|1480|65860|1709930|6800|9270|4|98|959|87|11.97|20.82|20.61|0.00|1793.07|1041.39|1811.34|161.37|0.00|1793.07|1954.44|751.68| +2451052|43343|5974|65860|1709930|6800|9270|4|5|959|41|78.43|106.66|21.33|0.00|874.53|3215.63|4373.06|43.72|0.00|874.53|918.25|-2341.10| +|43343|14068|65860|1709930|||||959||||20.74||1099.22|2776.67|3053.86|32.97||1099.22||| +2451052|43343|823|65860|1709930|6800|9270|4|297|959|74|20.63|41.05|24.21|0.00|1791.54|1526.62|3037.70|35.83|0.00|1791.54|1827.37|264.92| +2451052|43343|14656|65860|1709930|6800|9270|4|132|959|11|98.45|141.76|94.97|0.00|1044.67|1082.95|1559.36|41.78|0.00|1044.67|1086.45|-38.28| +2451052|43343|10592|65860|1709930|6800|9270|4|254|959|81|97.56|119.99|61.19|0.00|4956.39|7902.36|9719.19|148.69|0.00|4956.39|5105.08|-2945.97| +2451052|43343|12554|65860|1709930|6800|9270|4|263|959|81|64.92|73.35|55.01|0.00|4455.81|5258.52|5941.35|178.23|0.00|4455.81|4634.04|-802.71| +2451534|35112|6352|22749|63803|4857|17511|8|86|960|24|18.26|18.44|9.40|0.00|225.60|438.24|442.56|4.51|0.00|225.60|230.11|-212.64| +2451534|35112|6298|22749|63803|4857|17511|8|169|960|99|74.29|109.94|82.45|0.00|8162.55|7354.71|10884.06|653.00|0.00|8162.55|8815.55|807.84| +2451534|35112|10903|22749|63803|4857|17511|8|263|960|99|63.54|95.94|75.79|0.00|7503.21|6290.46|9498.06|0.00|0.00|7503.21|7503.21|1212.75| +2451534|35112|6784|22749|63803|4857|17511|8|234|960|35|54.19|93.20|69.90|0.00|2446.50|1896.65|3262.00|48.93|0.00|2446.50|2495.43|549.85| +2451534|35112|2578|22749|63803|4857|17511|8|126|960|3|32.50|45.50|20.93|0.00|62.79|97.50|136.50|3.76|0.00|62.79|66.55|-34.71| +2451534|35112|10784|22749|63803|4857|17511|8|35|960|89|38.00|69.54|0.00|0.00|0.00|3382.00|6189.06|0.00|0.00|0.00|0.00|-3382.00| +2451534|35112|11335|22749|63803|4857|17511|8|247|960|66|65.05|80.01|20.00|0.00|1320.00|4293.30|5280.66|66.00|0.00|1320.00|1386.00|-2973.30| +2451534|35112|6940|22749|63803|4857|17511|8|20|960|40|62.13|122.39|6.11|0.00|244.40|2485.20|4895.60|17.10|0.00|244.40|261.50|-2240.80| +2451534|35112|4460|22749|63803|4857|17511|8|260|960|33|84.81|117.03|95.96|0.00|3166.68|2798.73|3861.99|95.00|0.00|3166.68|3261.68|367.95| +2451534|35112|15464|22749|63803|4857|17511|8|140|960|87|54.35|72.28|54.93|0.00|4778.91|4728.45|6288.36|238.94|0.00|4778.91|5017.85|50.46| +2451534|35112|782|22749|63803|4857|17511|8|73|960|11|67.98|106.04|85.89|0.00|944.79|747.78|1166.44|9.44|0.00|944.79|954.23|197.01| +2451534|35112|7058|22749|63803|4857|17511|8|23|960|51|76.69|134.97|44.54|181.72|2271.54|3911.19|6883.47|104.49|181.72|2089.82|2194.31|-1821.37| +2451534|35112|13378|22749|63803|4857|17511|8|218|960|72|29.09|54.39|16.86|0.00|1213.92|2094.48|3916.08|48.55|0.00|1213.92|1262.47|-880.56| +2451534|35112|10964|22749|63803|4857|17511|8|241|960|90|80.13|159.45|9.56|0.00|860.40|7211.70|14350.50|0.00|0.00|860.40|860.40|-6351.30| +2452111|43448|15419|170|56499|4475|20804|7|196|961|48|66.04|116.89|72.47|3130.70|3478.56|3169.92|5610.72|13.91|3130.70|347.86|361.77|-2822.06| +2452111|43448|12137|170|56499|4475|20804|7|67|961|56|5.51|10.68|2.02|0.00|113.12|308.56|598.08|2.26|0.00|113.12|115.38|-195.44| +||13675|||4475||||961|65|9.64|14.07|6.61|382.38|429.65|626.60|||382.38|47.27|50.10|| +2452111|43448|10963|170|56499|4475|20804|7|104|961|53|24.23|26.89|3.76|0.00|199.28|1284.19|1425.17|15.94|0.00|199.28|215.22|-1084.91| +2452111|43448|10059|170|56499|4475|20804|7|38|961|77|16.47|31.78|13.34|0.00|1027.18|1268.19|2447.06|71.90|0.00|1027.18|1099.08|-241.01| +2452111|43448|7951|170|56499|4475|20804|7|110|961|54|46.29|46.75|14.02|0.00|757.08|2499.66|2524.50|52.99|0.00|757.08|810.07|-1742.58| +2452111|43448|16411|170|56499|4475|20804|7|152|961|61|6.25|6.50|5.65|0.00|344.65|381.25|396.50|31.01|0.00|344.65|375.66|-36.60| +2452111|43448|13625|170|56499|4475|20804|7|204|961|31|10.69|15.92|7.32|181.53|226.92|331.39|493.52|0.00|181.53|45.39|45.39|-286.00| +2452111|43448|12809|170|56499|4475|20804|7|86|961|16|40.30|79.79|48.67|669.69|778.72|644.80|1276.64|2.18|669.69|109.03|111.21|-535.77| +2452111|43448|3415||56499||||285|961||53.61|||0.00|393.82|3109.38|5627.74||0.00|393.82||-2715.56| +2452111|43448|6223|170|56499|4475|20804|7|205|961|37|10.66|16.20|0.32|0.00|11.84|394.42|599.40|0.00|0.00|11.84|11.84|-382.58| +2452111|43448|4419|170|56499|4475|20804|7|6|961|65|18.17|29.07|7.26|0.00|471.90|1181.05|1889.55|23.59|0.00|471.90|495.49|-709.15| +2452111|43448|107|170|56499|4475|20804|7|27|961|56|90.93|128.21|71.79|0.00|4020.24|5092.08|7179.76|120.60|0.00|4020.24|4140.84|-1071.84| +2452111|43448|277|170|56499|4475|20804|7|216|961|7|96.00|183.36|86.17|0.00|603.19|672.00|1283.52|36.19|0.00|603.19|639.38|-68.81| +2452111|43448|4343|170|56499|4475|20804|7|157|961|46|42.27|56.21|43.84|0.00|2016.64|1944.42|2585.66|20.16|0.00|2016.64|2036.80|72.22| +2452111|43448|621|170|56499|4475|20804|7|68|961|77|21.27|29.13|14.27|0.00|1098.79|1637.79|2243.01|32.96|0.00|1098.79|1131.75|-539.00| +2451092|49396|8222|49632|651643|407|38818|8|123|962|72|15.91|23.22|2.32|0.00|167.04|1145.52|1671.84|8.35|0.00|167.04|175.39|-978.48| +2451092|49396|13621|49632|651643|407|38818|8|297|962|74|78.46|105.13|6.30|9.32|466.20|5806.04|7779.62|0.00|9.32|456.88|456.88|-5349.16| +2451092|49396|14314|49632|651643|407|38818|8|135|962|84|47.47|79.74|32.69|0.00|2745.96|3987.48|6698.16|247.13|0.00|2745.96|2993.09|-1241.52| +2451092|49396|8968|49632|651643|407|38818|8|239|962|15|26.56|37.71|15.83|0.00|237.45|398.40|565.65|11.87|0.00|237.45|249.32|-160.95| +2451092|49396|8392|49632|651643|407|38818|8|110|962|85|45.12|59.10|28.36|0.00|2410.60|3835.20|5023.50|216.95|0.00|2410.60|2627.55|-1424.60| +2451092|49396|13094|49632|651643|407|38818|8|130|962|42|30.66|49.36|5.92|0.00|248.64|1287.72|2073.12|4.97|0.00|248.64|253.61|-1039.08| +2451092|49396|6823|49632|651643|407|38818|8|257|962|24|93.31|138.09|24.85|0.00|596.40|2239.44|3314.16|0.00|0.00|596.40|596.40|-1643.04| +2451092|49396|7088|49632|651643|407|38818|8|115|962|63|93.01|97.66|58.59|0.00|3691.17|5859.63|6152.58|73.82|0.00|3691.17|3764.99|-2168.46| +|49396|8366|||407||||962|14|64.22|103.39||0.00|463.12||1447.46||0.00|463.12||-435.96| +2451092|49396|9370|49632|651643|407|38818|8|296|962|12|44.37|67.44|12.81|0.00|153.72|532.44|809.28|12.29|0.00|153.72|166.01|-378.72| +2451092|49396|13286|49632|651643|407|38818|8|259|962|24|83.62|107.86|19.41|0.00|465.84|2006.88|2588.64|13.97|0.00|465.84|479.81|-1541.04| +2451092|49396|15692|49632|651643|407|38818|8|6|962|70|6.73|8.68|7.03|0.00|492.10|471.10|607.60|19.68|0.00|492.10|511.78|21.00| +2451092|49396|11674|49632|651643|407|38818|8|238|962|8|53.90|67.91|14.94|0.00|119.52|431.20|543.28|10.75|0.00|119.52|130.27|-311.68| +2451092|49396|3518|49632|651643|407|38818|8|162|962|5|49.40|83.98|23.51|0.00|117.55|247.00|419.90|4.70|0.00|117.55|122.25|-129.45| +2452636|63470|14733|72195|679831|6667|30703|1|155|963|34|67.82|80.02|65.61|0.00|2230.74|2305.88|2720.68|133.84|0.00|2230.74|2364.58|-75.14| +2452636|63470|5713|72195|679831|6667|30703|1|20|963|35|22.26|34.05|22.81|0.00|798.35|779.10|1191.75|31.93|0.00|798.35|830.28|19.25| +2452636|63470|2329|72195|679831|6667|30703|1|228|963|94|77.60|151.32|16.64|0.00|1564.16|7294.40|14224.08|0.00|0.00|1564.16|1564.16|-5730.24| +2452636|63470|10893|72195|679831|6667|30703|1|165|963|93|93.38|118.59|84.19|0.00|7829.67|8684.34|11028.87|156.59|0.00|7829.67|7986.26|-854.67| +2452636|63470|3420|72195|679831|6667|30703|1|75|963|17|18.36|32.31|23.58|92.19|400.86|312.12|549.27|3.08|92.19|308.67|311.75|-3.45| +2452636|63470|16699|72195|679831|6667|30703|1|235|963|64|61.84|71.11|26.31|1380.74|1683.84|3957.76|4551.04|6.06|1380.74|303.10|309.16|-3654.66| +2452636|63470|8508|72195|679831|6667|30703|1|87|963|91|53.54|83.52|77.67|0.00|7067.97|4872.14|7600.32|353.39|0.00|7067.97|7421.36|2195.83| +2452636|63470|4875|72195|679831|6667|30703|1|160|963|27|41.59|67.37|43.79|697.57|1182.33|1122.93|1818.99|38.78|697.57|484.76|523.54|-638.17| +2452636|63470|17826|72195|679831|6667|30703|1|151|963|3|64.05|107.60|38.73|0.00|116.19|192.15|322.80|10.45|0.00|116.19|126.64|-75.96| +2452636|63470|7023|72195|679831|6667|30703|1|241|963|70|13.37|21.79|7.19|407.67|503.30|935.90|1525.30|8.60|407.67|95.63|104.23|-840.27| +2452636|63470|17065|72195|679831|6667|30703|1|172|963|8|52.55|105.10|45.19|0.00|361.52|420.40|840.80|3.61|0.00|361.52|365.13|-58.88| +2451838|44794|16562|79453|1228282|5111|32340|2|154|964|46|39.41|57.53|51.77|0.00|2381.42|1812.86|2646.38|95.25|0.00|2381.42|2476.67|568.56| +2451838|44794|16633|79453|1228282|5111|32340|2|261|964|78|58.59|91.98|24.83|0.00|1936.74|4570.02|7174.44|58.10|0.00|1936.74|1994.84|-2633.28| +2451838|44794|11407|79453|1228282|5111|32340|2|257|964|4|6.71|8.38|8.38|0.00|33.52|26.84|33.52|2.34|0.00|33.52|35.86|6.68| +2451838|44794|7073|79453|1228282|5111|32340|2|75|964|24|77.43|150.21|75.10|0.00|1802.40|1858.32|3605.04|0.00|0.00|1802.40|1802.40|-55.92| +2451838|44794|15194|79453|1228282|5111|32340|2|85|964|12|51.20|71.16|14.94|0.00|179.28|614.40|853.92|7.17|0.00|179.28|186.45|-435.12| +2451838|44794|6482|79453|1228282|5111|32340|2|242|964|73|60.53|61.74|38.89|0.00|2838.97|4418.69|4507.02|198.72|0.00|2838.97|3037.69|-1579.72| +2451838|44794|2245|79453|1228282|5111|32340|2|270|964|96|92.09|134.45|44.36|0.00|4258.56|8840.64|12907.20|127.75|0.00|4258.56|4386.31|-4582.08| +2451838|44794|10279|79453|1228282|5111|32340|2|291|964|97|23.37|38.56|9.25|0.00|897.25|2266.89|3740.32|53.83|0.00|897.25|951.08|-1369.64| +2451838|44794|6253|79453|1228282|5111|32340|2|210|964|95|59.36|64.10|8.97|357.90|852.15|5639.20|6089.50|19.77|357.90|494.25|514.02|-5144.95| +2451838|44794|2198|79453|1228282|5111|32340|2|38|964|93|24.76|45.80|40.76|3676.95|3790.68|2302.68|4259.40|2.27|3676.95|113.73|116.00|-2188.95| +2451838|44794|5641|79453|1228282|5111|32340|2|38|964|92|50.84|54.90|25.25|0.00|2323.00|4677.28|5050.80|162.61|0.00|2323.00|2485.61|-2354.28| +2451838|44794|2471|79453|1228282|5111|32340|2|203|964|7|46.08|47.46|4.74|0.00|33.18|322.56|332.22|2.32|0.00|33.18|35.50|-289.38| +2451838|44794|17171|79453|1228282|5111|32340|2|299|964|92|78.99|99.52|68.66|0.00|6316.72|7267.08|9155.84|315.83|0.00|6316.72|6632.55|-950.36| +2452164|50877|5177|14819|1013107|1348|32036|10|201|965|5|44.98|54.42|50.61|0.00|253.05|224.90|272.10|5.06|0.00|253.05|258.11|28.15| +2452164|50877|9591|14819|1013107|1348|32036|10|149|965|47|62.24|124.48|77.17|0.00|3626.99|2925.28|5850.56|181.34|0.00|3626.99|3808.33|701.71| +2452164|50877|17359|14819|1013107|1348|32036|10|141|965|82|83.54|145.35|15.98|1244.84|1310.36|6850.28|11918.70|2.62|1244.84|65.52|68.14|-6784.76| +2452164|50877|17279|14819||1348||||965|60|42.71|||0.00|||||0.00|||| +2452164|50877|15547|14819|1013107|1348|32036|10|33|965|59|92.76|174.38|125.55|1703.71|7407.45|5472.84|10288.42|0.00|1703.71|5703.74|5703.74|230.90| +2452164|50877|4519|14819|1013107|1348|32036|10|101|965|99|41.25|48.67|18.98|0.00|1879.02|4083.75|4818.33|56.37|0.00|1879.02|1935.39|-2204.73| +2452164|50877|7465|14819|1013107|1348|32036|10|210|965|74|43.11|58.62|16.41|0.00|1214.34|3190.14|4337.88|109.29|0.00|1214.34|1323.63|-1975.80| +2452164|50877|5779|14819|1013107|1348|32036|10|204|965|76|3.08|3.44|2.92|0.00|221.92|234.08|261.44|11.09|0.00|221.92|233.01|-12.16| +2452164|50877|3287|14819|1013107|1348|32036|10|20|965|7|20.40|37.33|4.85|0.00|33.95|142.80|261.31|3.05|0.00|33.95|37.00|-108.85| +2452164|50877|13009|14819|1013107|1348|32036|10|204|965|47|58.51|85.42|34.16|0.00|1605.52|2749.97|4014.74|16.05|0.00|1605.52|1621.57|-1144.45| +2452164|50877|16425|14819|1013107|1348|32036|10|72|965|75|72.79|140.48|26.69|0.00|2001.75|5459.25|10536.00|40.03|0.00|2001.75|2041.78|-3457.50| +2452164|50877|9505|14819|1013107|1348|32036|10|291|965|36|58.53|104.76|79.61|0.00|2865.96|2107.08|3771.36|114.63|0.00|2865.96|2980.59|758.88| +2452164|50877|4397|14819|1013107|1348|32036|10|101|965|36|40.58|71.82|3.59|0.00|129.24|1460.88|2585.52|2.58|0.00|129.24|131.82|-1331.64| +2452164|50877|689|14819|1013107|1348|32036|10|271|965|16|5.08|9.34|3.54|0.00|56.64|81.28|149.44|1.69|0.00|56.64|58.33|-24.64| +2452164|50877|1195|14819|1013107|1348|32036|10|202|965|6|69.40|76.34|16.79|0.00|100.74|416.40|458.04|6.04|0.00|100.74|106.78|-315.66| +2451403|44516|17402|53335|1527344|6506|47315|10|176|966|54|16.82|22.37|12.07|0.00|651.78|908.28|1207.98|26.07|0.00|651.78|677.85|-256.50| +2451403|44516|10408|53335|1527344|6506|47315|10|221|966|51|3.23|4.74|1.37|0.00|69.87|164.73|241.74|4.89|0.00|69.87|74.76|-94.86| +2451403|44516|9793|53335|1527344|6506|47315|10|233|966|35|90.64|151.36|119.57|0.00|4184.95|3172.40|5297.60|83.69|0.00|4184.95|4268.64|1012.55| +|44516|9373|53335|1527344|6506|47315||265|966|77|79.37|153.97|||8061.13|||||||1949.64| +2451403|44516|2974|53335|1527344|6506|47315|10|215|966|23|82.65|157.03|6.28|0.00|144.44|1900.95|3611.69|7.22|0.00|144.44|151.66|-1756.51| +2451403|44516|5203|53335|1527344|6506|47315|10|242|966|22|32.64|45.36|37.19|0.00|818.18|718.08|997.92|49.09|0.00|818.18|867.27|100.10| +|44516|1054||1527344|6506|47315|10||966||19.31|36.88||0.00|42.78|38.62|||0.00||46.63|4.16| +2451403|44516|11275|53335|1527344|6506|47315|10|139|966|54|27.49|54.15|42.77|0.00|2309.58|1484.46|2924.10|46.19|0.00|2309.58|2355.77|825.12| +2451162|37530|373|58991|728348|6952|46756|2|188|967|19|12.86|13.37|13.23|0.00|251.37|244.34|254.03|2.51|0.00|251.37|253.88|7.03| +2451162|37530|9616|58991|728348|6952|46756|2|251|967|76|54.14|82.29|19.74|240.03|1500.24|4114.64|6254.04|75.61|240.03|1260.21|1335.82|-2854.43| +2451162|37530|4888|58991|728348|6952|46756|2|195|967|93|37.49|66.73|8.67|475.72|806.31|3486.57|6205.89|3.30|475.72|330.59|333.89|-3155.98| +2451162|37530|14746|58991|728348|6952|46756|2|177|967|12|73.18|140.50|108.18|220.68|1298.16|878.16|1686.00|64.64|220.68|1077.48|1142.12|199.32| +2451162|37530|1591|58991|728348|6952|46756|2|178|967|35|19.88|30.61|14.69|0.00|514.15|695.80|1071.35|10.28|0.00|514.15|524.43|-181.65| +2451162|37530|6148|58991|728348|6952|46756|2|197|967|91|87.06|92.28|64.59|0.00|5877.69|7922.46|8397.48|58.77|0.00|5877.69|5936.46|-2044.77| +2451162|37530|15440|58991|728348|6952|46756|2|147|967|50|10.18|14.65|8.35|54.27|417.50|509.00|732.50|25.42|54.27|363.23|388.65|-145.77| +2451162|37530|13399|58991|728348|6952|46756|2|8|967|81|49.43|62.77|59.63|0.00|4830.03|4003.83|5084.37|193.20|0.00|4830.03|5023.23|826.20| +2451162||12844|58991||6952||||967|||99.57|59.74|0.00|5555.82|8343.03|9260.01|277.79|0.00|||-2787.21| +2451162|37530|10750|58991|728348|6952|46756|2|65|967|19|73.58|76.52|3.82|0.00|72.58|1398.02|1453.88|3.62|0.00|72.58|76.20|-1325.44| +2451162|37530|10112|58991|728348|6952|46756|2|289|967|67|47.78|62.59|35.05|0.00|2348.35|3201.26|4193.53|164.38|0.00|2348.35|2512.73|-852.91| +2451162|37530|15524|58991|728348|6952|46756|2|4|967|43|98.51|194.06|65.98|879.51|2837.14|4235.93|8344.58|39.15|879.51|1957.63|1996.78|-2278.30| +2451162|37530|7012|58991|728348|6952|46756|2|253|967|62|30.67|49.07|30.91|0.00|1916.42|1901.54|3042.34|57.49|0.00|1916.42|1973.91|14.88| +2451162|37530|12730|58991|||46756|2||967|2|69.88|113.90|89.98|0.00||139.76|||0.00|179.96||| +2451162|37530|4528|58991|728348|6952|46756|2|63|967|57|77.98|82.65|57.02|0.00|3250.14|4444.86|4711.05|0.00|0.00|3250.14|3250.14|-1194.72| +2451868|41610|11515|33981|232777|7126|18675|2|190|968|22|26.35|40.57|1.21|11.97|26.62|579.70|892.54|1.17|11.97|14.65|15.82|-565.05| +2451868|41610|8264|33981|232777|7126|18675|2|10|968|17|59.53|102.39|71.67|0.00|1218.39|1012.01|1740.63|97.47|0.00|1218.39|1315.86|206.38| +2451868|41610|9182|33981|232777|7126|18675|2|104|968|40|24.64|28.82|4.32|160.70|172.80|985.60|1152.80|0.00|160.70|12.10|12.10|-973.50| +2451868|41610|16781|33981|232777|7126|18675|2|65|968|33|97.76|130.02|9.10|0.00|300.30|3226.08|4290.66|15.01|0.00|300.30|315.31|-2925.78| +2451868|41610|299|33981|232777|7126|18675|2|223|968|7|4.19|4.60|1.74|0.00|12.18|29.33|32.20|0.00|0.00|12.18|12.18|-17.15| +2451868|41610|2413|33981|232777|7126|18675|2|8|968|50|35.34|51.59|42.81|0.00|2140.50|1767.00|2579.50|0.00|0.00|2140.50|2140.50|373.50| +2451868|41610|7525|33981|232777|7126|18675|2|155|968|31|27.42|35.92|7.54|0.00|233.74|850.02|1113.52|16.36|0.00|233.74|250.10|-616.28| +2451868|41610|17167|33981|232777|7126|18675|2|209|968|50|97.74|158.33|72.83|0.00|3641.50|4887.00|7916.50|182.07|0.00|3641.50|3823.57|-1245.50| +2451868|41610|6566|33981|232777|7126|18675|2|92|968|23|91.30|170.73|46.09|0.00|1060.07|2099.90|3926.79|84.80|0.00|1060.07|1144.87|-1039.83| +|41610|12613||232777||18675||102|968||34.20||||698.88|1333.80|1427.01|||698.88|754.79|-634.92| +2451123|39888|10312|93748|1336809|5023|42084|7|105|969|83|8.48|8.98|5.38|0.00|446.54|703.84|745.34|35.72|0.00|446.54|482.26|-257.30| +2451123|39888|9112|93748|1336809|5023|42084|7|192|969|51|81.10|122.46|12.24|580.54|624.24|4136.10|6245.46|2.18|580.54|43.70|45.88|-4092.40| +2451123|39888|8245|93748|1336809|5023|42084|7|64|969|89|76.16|112.71|30.43|0.00|2708.27|6778.24|10031.19|54.16|0.00|2708.27|2762.43|-4069.97| +2451123|39888|11648|93748|1336809|5023|42084|7|196|969|3|59.91|62.30|8.72|0.00|26.16|179.73|186.90|0.78|0.00|26.16|26.94|-153.57| +||13154|93748|||42084|7||969||||13.00|0.00|455.00||1198.05||0.00|455.00||| +2451123|39888|15469|93748|1336809|5023|42084|7|285|969|91|88.28|160.66|149.41|0.00|13596.31|8033.48|14620.06|815.77|0.00|13596.31|14412.08|5562.83| +2451123|39888|13249|93748|1336809|5023|42084|7|6|969|84|78.50|79.28|37.26|0.00|3129.84|6594.00|6659.52|156.49|0.00|3129.84|3286.33|-3464.16| +2451123|39888|4058|93748|1336809|5023|42084|7|77|969|47|21.15|24.53|1.71|73.94|80.37|994.05|1152.91|0.00|73.94|6.43|6.43|-987.62| +2451123|39888|10303|93748|1336809|5023|42084|7|39|969|32|26.10|27.40|18.63|0.00|596.16|835.20|876.80|0.00|0.00|596.16|596.16|-239.04| +2451123|39888|11366|93748|1336809|5023|42084|7|55|969|45|78.89|127.80|53.67|0.00|2415.15|3550.05|5751.00|193.21|0.00|2415.15|2608.36|-1134.90| +2451123|39888|7513|93748|1336809|5023|42084|7|286|969|45|41.69|52.11|41.16|0.00|1852.20|1876.05|2344.95|166.69|0.00|1852.20|2018.89|-23.85| +2452242|34407|11419|26374|823223|4872|43721|10|153|970|3|44.17|52.12|21.36|0.00|64.08|132.51|156.36|1.28|0.00|64.08|65.36|-68.43| +2452242|34407|10019|26374|823223|4872|43721|10|122|970|6|82.75|146.46|130.34|0.00|782.04|496.50|878.76|39.10|0.00|782.04|821.14|285.54| +2452242|34407|7033|26374|823223|4872|43721|10|149|970|81|69.50|100.08|18.01|0.00|1458.81|5629.50|8106.48|72.94|0.00|1458.81|1531.75|-4170.69| +2452242|34407|16795|26374|823223|4872|43721|10|286|970|8|8.38|10.22|10.11|0.00|80.88|67.04|81.76|0.80|0.00|80.88|81.68|13.84| +2452242|34407|16745|26374|823223|4872|43721|10|72|970|7|59.92|66.51|43.89|0.00|307.23|419.44|465.57|9.21|0.00|307.23|316.44|-112.21| +2452242|34407|4301|26374|823223|4872|43721|10|32|970|9|90.09|99.09|2.97|0.00|26.73|810.81|891.81|1.06|0.00|26.73|27.79|-784.08| +2452242|34407|16851|26374|823223|4872|43721|10|47|970|45|70.76|89.15|45.46|900.10|2045.70|3184.20|4011.75|0.00|900.10|1145.60|1145.60|-2038.60| +2452242||4801||||||170|970|42||91.06|71.93||3021.06|||||||| +2452242|34407|3179|26374|823223|4872|43721|10|210|970|92|56.85|112.56|76.54|0.00|7041.68|5230.20|10355.52|492.91|0.00|7041.68|7534.59|1811.48| +2452600|34283|6987|27558|1479311|3864|24817|2|299|971|69|5.46|8.40|2.85|0.00|196.65|376.74|579.60|7.86|0.00|196.65|204.51|-180.09| +2452600|34283|14715|27558|1479311|3864|24817|2|11|971|93|98.86|118.63|60.50|0.00|5626.50|9193.98|11032.59|393.85|0.00|5626.50|6020.35|-3567.48| +||8004|27558||||||971||98.39|146.60|137.80|0.00|9508.20|6788.91||760.65|0.00|9508.20|10268.85|| +2452600|34283|2049|27558|1479311|3864|24817|2|191|971|42|51.58|70.14|41.38|0.00|1737.96|2166.36|2945.88|52.13|0.00|1737.96|1790.09|-428.40| +2452600|34283|15873|27558|1479311|3864|24817|2|1|971|42|82.11|114.95|54.02|0.00|2268.84|3448.62|4827.90|68.06|0.00|2268.84|2336.90|-1179.78| +2452600|34283|2667|27558|1479311|3864|24817|2|101|971|10|74.31|114.43|10.29|0.00|102.90|743.10|1144.30|8.23|0.00|102.90|111.13|-640.20| +2452600|34283|15999|27558|1479311|3864|24817|2|25|971|8|33.25|38.23|2.67|0.00|21.36|266.00|305.84|0.00|0.00|21.36|21.36|-244.64| +2452600|34283|15729|27558|1479311|3864|24817|2|24|971|64|31.31|46.33|10.65|558.91|681.60|2003.84|2965.12|9.81|558.91|122.69|132.50|-1881.15| +2452600|34283|3903|27558|1479311|3864|24817|2|297|971|36|92.06|100.34|16.05|329.34|577.80|3314.16|3612.24|4.96|329.34|248.46|253.42|-3065.70| +2452600|34283|14205|27558|1479311|3864|24817|2|23|971|39|91.55|182.18|158.49|4326.77|6181.11|3570.45|7105.02|55.63|4326.77|1854.34|1909.97|-1716.11| +2452600|34283|5199|27558|1479311|3864|24817|2|140|971|50|57.21|59.49|16.06|337.26|803.00|2860.50|2974.50|18.62|337.26|465.74|484.36|-2394.76| +2452600|34283|5089|27558|1479311|3864|24817|2|234|971|88|4.62|7.29|5.61|0.00|493.68|406.56|641.52|4.93|0.00|493.68|498.61|87.12| +2452600|34283|15139|27558|1479311|3864|24817|2|143|971|86|29.95|56.60|5.66|0.00|486.76|2575.70|4867.60|14.60|0.00|486.76|501.36|-2088.94| +2452600|34283|4087|27558|1479311|3864|24817|2|297|971|16|47.39|55.92|6.15|0.00|98.40|758.24|894.72|7.87|0.00|98.40|106.27|-659.84| +2452222|53323|2745|41544|492687|3830|21773|7|222|972|86|20.02|20.82|0.20|0.00|17.20|1721.72|1790.52|0.86|0.00|17.20|18.06|-1704.52| +2452222|53323|17769|41544|492687|3830|21773|7|81|972|33|47.72|85.41|49.53|0.00|1634.49|1574.76|2818.53|98.06|0.00|1634.49|1732.55|59.73| +2452222|53323|7317|41544|492687|3830|21773|7|278|972|60|41.25|75.07|48.79|0.00|2927.40|2475.00|4504.20|263.46|0.00|2927.40|3190.86|452.40| +2452222|53323|14563|41544|492687|3830|21773|7|44|972|22|38.07|70.81|5.66|0.00|124.52|837.54|1557.82|2.49|0.00|124.52|127.01|-713.02| +2452222|53323|6845|41544|492687|3830|21773|7|171|972|88|11.39|18.90|0.37|0.00|32.56|1002.32|1663.20|0.00|0.00|32.56|32.56|-969.76| +2452222|53323|3795|41544|492687|3830|21773|7|228|972|60|4.05|5.18|3.05|51.24|183.00|243.00|310.80|0.00|51.24|131.76|131.76|-111.24| +2452222||11979||||21773|7||972|||23.68|||||2060.16|0.40||40.08|40.48|-1431.96| +2452222|53323|9463|||||||972|18|38.72|||0.00|0.00|696.96|||0.00|0.00|0.00|| +2452222|53323|11553|41544|492687|3830|21773|7|292|972|30|64.05|78.78|30.72|0.00|921.60|1921.50|2363.40|55.29|0.00|921.60|976.89|-999.90| +2451756|39565|15991|38757|1202100|5443|15939|2|114|973|2|57.16|87.45|17.49|27.98|34.98|114.32|174.90|0.49|27.98|7.00|7.49|-107.32| +2451756|39565|2786|38757|1202100|5443|15939|2|255|973|11|79.91|104.68|30.35|0.00|333.85|879.01|1151.48|26.70|0.00|333.85|360.55|-545.16| +2451756|39565|6677|38757|1202100|5443|15939|2|89|973|72|92.28|113.50|97.61|0.00|7027.92|6644.16|8172.00|140.55|0.00|7027.92|7168.47|383.76| +2451756|39565|12161|38757|1202100|5443|15939|2|111|973|9|62.23|80.27|54.58|0.00|491.22|560.07|722.43|44.20|0.00|491.22|535.42|-68.85| +2451756|39565|8432|38757|1202100|5443|15939|2|234|973|69|72.59|134.29|91.31|0.00|6300.39|5008.71|9266.01|567.03|0.00|6300.39|6867.42|1291.68| +2451756|39565|10220|38757|1202100|5443|15939|2|96|973|77|69.38|138.06|16.56|0.00|1275.12|5342.26|10630.62|114.76|0.00|1275.12|1389.88|-4067.14| +||10043|38757||5443||2||973||||16.06|0.00|176.66|746.13||15.89|0.00|||| +2451756|39565|10922|38757|1202100|5443|15939|2|117|973|47|19.78|32.83|10.17|0.00|477.99|929.66|1543.01|28.67|0.00|477.99|506.66|-451.67| +2450848|35957|17695|52145|1811089|433|19330|10|3|974|47|86.27|124.22|104.34|0.00|4903.98|4054.69|5838.34|0.00|0.00|4903.98|4903.98|849.29| +2450848|35957|8702|52145|1811089|433|19330|10|207|974|75|25.66|39.00|37.05|0.00|2778.75|1924.50|2925.00|166.72|0.00|2778.75|2945.47|854.25| +2450848|35957|8275|52145|1811089|433|19330|10|24|974|23|88.63|168.39|26.94|0.00|619.62|2038.49|3872.97|0.00|0.00|619.62|619.62|-1418.87| +2450848|35957|10952|52145|1811089|433|19330|10|35|974|85|31.99|46.38|29.21|0.00|2482.85|2719.15|3942.30|0.00|0.00|2482.85|2482.85|-236.30| +2450848|35957|2990|52145|1811089|433|19330|10|100|974|22|15.45|26.26|26.26|0.00|577.72|339.90|577.72|40.44|0.00|577.72|618.16|237.82| +2450848|35957|3154|52145|1811089|433|19330|10|259|974|81|34.46|42.04|14.29|0.00|1157.49|2791.26|3405.24|92.59|0.00|1157.49|1250.08|-1633.77| +2450848|35957|13363|52145|1811089|433|19330|10|214|974|7|8.22|11.75|1.17|0.00|8.19|57.54|82.25|0.73|0.00|8.19|8.92|-49.35| +2450848|35957|10621|52145|1811089|433|19330|10|130|974|83|87.87|138.83|2.77|0.00|229.91|7293.21|11522.89|16.09|0.00|229.91|246.00|-7063.30| +2450848|35957|8306|52145|1811089|433|19330|10|145|974|3|38.37|66.76|57.41|0.00|172.23|115.11|200.28|5.16|0.00|172.23|177.39|57.12| +2450848|35957|13615|52145|1811089|433|19330|10|113|974|1|17.62|18.50|3.51|0.00|3.51|17.62|18.50|0.28|0.00|3.51|3.79|-14.11| +2450848|35957|5044|52145|1811089|433|19330|10|195|974|34|50.47|78.73|31.49|0.00|1070.66|1715.98|2676.82|10.70|0.00|1070.66|1081.36|-645.32| +2450848|35957|13270|52145|1811089|433|19330|10|162|974|4|61.44|101.37|25.34|0.00|101.36|245.76|405.48|6.08|0.00|101.36|107.44|-144.40| +2452258|68217|14677|21518|155057|949|48841|1|279|975|62|73.43|82.24|13.15|0.00|815.30|4552.66|5098.88|0.00|0.00|815.30|815.30|-3737.36| +|68217|11395|21518|155057||48841||235|975|81|58.05|101.58|14.22||1151.82|4702.05||11.51||1151.82||| +2452258|68217|14511|21518|155057|949|48841|1|279|975|32|77.70|120.43|44.55|0.00|1425.60|2486.40|3853.76|114.04|0.00|1425.60|1539.64|-1060.80| +2452258|68217|13509|21518|155057|949|48841|1|143|975|6|59.45|107.60|73.16|0.00|438.96|356.70|645.60|4.38|0.00|438.96|443.34|82.26| +2452258|68217|9999|21518|155057|949|48841|1|144|975|48|92.76|159.54|31.90|0.00|1531.20|4452.48|7657.92|15.31|0.00|1531.20|1546.51|-2921.28| +2452258|68217|6291|21518|155057|949|48841|1|218|975|75|83.81|90.51|76.02|0.00|5701.50|6285.75|6788.25|228.06|0.00|5701.50|5929.56|-584.25| +2452258|68217|11515|21518|155057|949|48841|1|21|975|16|63.19|106.79|106.79|0.00|1708.64|1011.04|1708.64|34.17|0.00|1708.64|1742.81|697.60| +2452258|68217|8265|21518|155057|949|48841|1|247|975|39|96.79|193.58|3.87|0.00|150.93|3774.81|7549.62|6.03|0.00|150.93|156.96|-3623.88| +2452258|68217|9183|21518|155057|949|48841|1|78|975|28|21.20|37.10|5.56|0.00|155.68|593.60|1038.80|3.11|0.00|155.68|158.79|-437.92| +2452258|68217|16781|21518|155057|949|48841|1|280|975|61|43.42|43.42|36.47|0.00|2224.67|2648.62|2648.62|88.98|0.00|2224.67|2313.65|-423.95| +2452258|68217|299|21518|155057|949|48841|1|71|975|68|64.04|97.34|21.41|0.00|1455.88|4354.72|6619.12|0.00|0.00|1455.88|1455.88|-2898.84| +2452258|68217|2413|21518|155057|949|48841|1|95|975|98|58.26|60.00|31.20|703.24|3057.60|5709.48|5880.00|70.63|703.24|2354.36|2424.99|-3355.12| +2452258|68217|7525|21518|155057|949|48841|1|7|975|50|57.74|65.82|65.82|0.00|3291.00|2887.00|3291.00|131.64|0.00|3291.00|3422.64|404.00| +2452258|68217|17167|21518|155057|949|48841|1|278|975|20|93.40|108.34|54.17|0.00|1083.40|1868.00|2166.80|32.50|0.00|1083.40|1115.90|-784.60| +2452258|68217|6567|21518|155057|949|48841|1|191|975|29|38.08|54.83|40.02|0.00|1160.58|1104.32|1590.07|34.81|0.00|1160.58|1195.39|56.26| +2451780|62324|10292|49594|174801|6055|25262|10|120|976|26|40.95|79.85|12.77|0.00|332.02|1064.70|2076.10|19.92|0.00|332.02|351.94|-732.68| +2451780|62324|13433|49594|174801|6055|25262|10|296|976|2|35.54|57.93|57.93|0.00|115.86|71.08|115.86|3.47|0.00|115.86|119.33|44.78| +2451780|62324|17345|49594|174801|6055|25262|10|217|976|33|27.91|51.35|7.70|0.00|254.10|921.03|1694.55|7.62|0.00|254.10|261.72|-666.93| +2451780|62324|17075|49594|174801|6055|25262|10|67|976|39|73.20|121.51|100.85|0.00|3933.15|2854.80|4738.89|39.33|0.00|3933.15|3972.48|1078.35| +2451780|62324|12716|49594|174801|6055|25262|10|26|976|91|2.88|5.32|4.68|0.00|425.88|262.08|484.12|0.00|0.00|425.88|425.88|163.80| +||1100|||6055|25262|10||976||96.28||32.34||485.10|1444.20||33.95||485.10||| +2451780|62324|2185|49594|174801|6055|25262|10|239|976|56|97.19|113.71|2.27|0.00|127.12|5442.64|6367.76|10.16|0.00|127.12|137.28|-5315.52| +2451780|62324|16160|49594|174801|6055|25262|10|83|976|45|37.92|65.60|17.71|0.00|796.95|1706.40|2952.00|7.96|0.00|796.95|804.91|-909.45| +2451780|62324|16865||174801|6055|25262|10|118|976|||||0.00||1784.09|2943.43|135.07|0.00|||| +2451780|62324|5399|49594|174801|6055|25262|10|185|976|1|14.08|16.05|3.85|0.00|3.85|14.08|16.05|0.26|0.00|3.85|4.11|-10.23| +2451780|62324|17899|49594|174801|6055|25262|10|239|976|73|93.70|122.74|39.27|0.00|2866.71|6840.10|8960.02|143.33|0.00|2866.71|3010.04|-3973.39| +2451780|62324|6344|49594|174801|6055|25262|10|214|976|13|18.98|37.20|28.64|0.00|372.32|246.74|483.60|22.33|0.00|372.32|394.65|125.58| +2451780|62324|1921|49594|174801|6055|25262|10|294|976|6|69.25|137.80|49.60|0.00|297.60|415.50|826.80|23.80|0.00|297.60|321.40|-117.90| +2451780|62324|16235|49594|174801|6055|25262|10|181|976|50|37.81|45.75|36.14|0.00|1807.00|1890.50|2287.50|144.56|0.00|1807.00|1951.56|-83.50| +2451854|71058|14888|83069|1530783|4097|41811|4|166|977|74|94.79|99.52|25.87|0.00|1914.38|7014.46|7364.48|57.43|0.00|1914.38|1971.81|-5100.08| +2451854|71058|7165|83069|1530783|4097|41811|4|49|977|60|55.25|59.11|23.64|0.00|1418.40|3315.00|3546.60|127.65|0.00|1418.40|1546.05|-1896.60| +2451854|71058|17810|83069|1530783|4097|41811|4|142|977|58|10.78|15.30|3.51|0.00|203.58|625.24|887.40|2.03|0.00|203.58|205.61|-421.66| +|71058|13019||1530783|4097|||83|977||||0.00|0.00|0.00||3256.38|0.00|0.00||0.00|| +2451854|71058|15503|83069|1530783|4097|41811|4|69|977|74|34.84|56.44|22.01|0.00|1628.74|2578.16|4176.56|130.29|0.00|1628.74|1759.03|-949.42| +2451854|71058|2042|83069|1530783|4097|41811|4|220|977|94|83.39|86.72|37.28|0.00|3504.32|7838.66|8151.68|105.12|0.00|3504.32|3609.44|-4334.34| +2451854|71058|1373|83069|1530783|4097|41811|4|237|977|4|3.70|5.88|5.58|0.00|22.32|14.80|23.52|1.11|0.00|22.32|23.43|7.52| +2451854|71058|13987|83069|1530783|4097|41811|4|47|977|71|29.32|36.06|22.71|0.00|1612.41|2081.72|2560.26|128.99|0.00|1612.41|1741.40|-469.31| +2451854|71058|16831|83069|1530783|4097|41811|4|182|977|41|95.48|128.89|114.71|0.00|4703.11|3914.68|5284.49|47.03|0.00|4703.11|4750.14|788.43| +2451854|71058|8636|83069|1530783|4097|41811|4|15|977|32|16.65|31.30|5.63|0.00|180.16|532.80|1001.60|12.61|0.00|180.16|192.77|-352.64| +2451854|71058|1981|83069|1530783|4097|41811|4|282|977|9|64.64|122.16|78.18|0.00|703.62|581.76|1099.44|56.28|0.00|703.62|759.90|121.86| +2452525|32742|13339|67948|65787|5226|3465|2|130|978|5|23.92|27.74|25.52|0.00|127.60|119.60|138.70|7.65|0.00|127.60|135.25|8.00| +2452525|32742|9319|67948|65787|5226|3465|2|240|978|29|17.14|32.05|25.96|0.00|752.84|497.06|929.45|0.00|0.00|752.84|752.84|255.78| +2452525|32742|5217|67948|65787|5226|3465|2|183|978|66|44.65|45.09|39.22|1164.83|2588.52|2946.90|2975.94|113.89|1164.83|1423.69|1537.58|-1523.21| +2452525|32742|4488|67948|65787|5226|3465|2|50|978|84|2.49|3.21|1.79|0.00|150.36|209.16|269.64|7.51|0.00|150.36|157.87|-58.80| +|32742|15793||65787|5226|3465||109|978|54||||0.00|3108.78|4771.44|8636.22|93.26|0.00|3108.78|3202.04|| +2452525|32742|5658|67948|65787|5226|3465|2|290|978|61|52.50|81.37|15.46|0.00|943.06|3202.50|4963.57|18.86|0.00|943.06|961.92|-2259.44| +2452525|32742|8919|67948|65787|5226|3465|2|274|978|40|83.52|122.77|116.63|0.00|4665.20|3340.80|4910.80|0.00|0.00|4665.20|4665.20|1324.40| +2452525|32742|7167|67948|65787|5226|3465|2|34|978|82|72.45|100.70|73.51|0.00|6027.82|5940.90|8257.40|361.66|0.00|6027.82|6389.48|86.92| +2452525|32742|11107|67948|65787|5226|3465|2|124|978|70|75.21|100.78|98.76|0.00|6913.20|5264.70|7054.60|345.66|0.00|6913.20|7258.86|1648.50| +2452525|32742|7663|67948|65787|5226|3465|2|167|978|22|93.56|144.08|46.10|0.00|1014.20|2058.32|3169.76|81.13|0.00|1014.20|1095.33|-1044.12| +2452525|32742|13575|67948|65787|5226|3465|2|177|978|86|51.73|61.55|43.70|0.00|3758.20|4448.78|5293.30|150.32|0.00|3758.20|3908.52|-690.58| +2451792|61036|11983|3107|1343649|4991|33533|2|179|979|14|97.61|178.62|33.93|0.00|475.02|1366.54|2500.68|42.75|0.00|475.02|517.77|-891.52| +2451792|61036|16298||1343649||33533|||979|||39.70|||104.61||1310.10|||104.61||-911.13| +2451792|61036|16964|3107|1343649|4991|33533|2|115|979|32|14.38|24.73|8.65|0.00|276.80|460.16|791.36|0.00|0.00|276.80|276.80|-183.36| +2451792|61036|5051|3107|1343649|4991|33533|2|227|979|41|62.79|120.55|18.08|0.00|741.28|2574.39|4942.55|37.06|0.00|741.28|778.34|-1833.11| +2451792|61036|1297|3107|1343649|4991|33533|2|163|979|50|79.67|88.43|1.76|54.56|88.00|3983.50|4421.50|2.34|54.56|33.44|35.78|-3950.06| +2451792|61036|8953|3107|1343649|4991|33533|2|6|979|97|93.08|105.18|66.26|5270.32|6427.22|9028.76|10202.46|11.56|5270.32|1156.90|1168.46|-7871.86| +2451792|61036|4511|3107|1343649|4991|33533|2|177|979|60|96.52|101.34|22.29|0.00|1337.40|5791.20|6080.40|93.61|0.00|1337.40|1431.01|-4453.80| +2451792|61036|13367|3107|1343649|4991|33533|2|174|979|68|38.54|58.96|10.61|0.00|721.48|2620.72|4009.28|50.50|0.00|721.48|771.98|-1899.24| +2451792|61036|13537|3107|1343649|4991|33533|2|32|979|63|7.69|11.68|10.27|0.00|647.01|484.47|735.84|45.29|0.00|647.01|692.30|162.54| +2451792|61036|1382|3107|1343649|4991|33533|2|63|979|16|19.73|36.10|21.29|0.00|340.64|315.68|577.60|30.65|0.00|340.64|371.29|24.96| +2451792|61036|16316|3107|1343649|4991|33533|2|34|979|53|28.22|53.33|39.46|0.00|2091.38|1495.66|2826.49|104.56|0.00|2091.38|2195.94|595.72| +2451804|67164|12689|32216|155109|3155|26801|4|285|980|55|20.04|26.45|0.79|0.00|43.45|1102.20|1454.75|1.30|0.00|43.45|44.75|-1058.75| +2451804|67164|4825|32216|155109|3155|26801|4|170|980|73|28.65|47.84|42.09|2089.34|3072.57|2091.45|3492.32|39.32|2089.34|983.23|1022.55|-1108.22| +2451804|67164|17018|32216|155109|3155|26801|4|49|980|90|83.56|107.79|56.05|0.00|5044.50|7520.40|9701.10|151.33|0.00|5044.50|5195.83|-2475.90| +2451804|67164|15128|32216|155109|3155|26801|4|269|980|23|99.71|170.50|95.48|0.00|2196.04|2293.33|3921.50|0.00|0.00|2196.04|2196.04|-97.29| +2451804|67164|2437|32216|155109|3155|26801|4|134|980|42|13.68|18.33|8.98|0.00|377.16|574.56|769.86|30.17|0.00|377.16|407.33|-197.40| +2451804|67164|581|32216|155109|3155|26801|4|243|980|5|76.16|150.03|9.00|0.00|45.00|380.80|750.15|0.00|0.00|45.00|45.00|-335.80| +2451804|67164|9680|32216|155109|3155|26801|4|131|980|27|43.24|63.13|44.82|0.00|1210.14|1167.48|1704.51|12.10|0.00|1210.14|1222.24|42.66| +2451804|67164|17624|32216|155109|3155|26801|4|211|980|6|80.28|103.56|79.74|0.00|478.44|481.68|621.36|14.35|0.00|478.44|492.79|-3.24| +2451874|55797|14864|22219|1380725|3231|9737|4|175|981|52|94.99|97.83|89.02|0.00|4629.04|4939.48|5087.16|138.87|0.00|4629.04|4767.91|-310.44| +2451874|55797|5923|22219|1380725|3231|9737|4|102|981|71|15.73|18.71|8.41|0.00|597.11|1116.83|1328.41|23.88|0.00|597.11|620.99|-519.72| +2451874|55797|12619|22219|1380725|3231|9737|4|20|981|3|3.81|4.41|3.70|0.00|11.10|11.43|13.23|0.00|0.00|11.10|11.10|-0.33| +2451874|55797|13919|22219|1380725|3231|9737|4|248|981|79|99.03|192.11|146.00|0.00|11534.00|7823.37|15176.69|922.72|0.00|11534.00|12456.72|3710.63| +2451874|55797|6023|22219|1380725|3231|9737|4|193|981|51|35.93|48.50|41.71|0.00|2127.21|1832.43|2473.50|21.27|0.00|2127.21|2148.48|294.78| +2451874|55797|6383|22219|1380725|3231|9737|4|231|981|37|60.63|83.06|5.81|0.00|214.97|2243.31|3073.22|0.00|0.00|214.97|214.97|-2028.34| +2451874|55797|16307|22219|1380725|3231|9737|4|227|981|89|84.57|116.70|59.51|0.00|5296.39|7526.73|10386.30|52.96|0.00|5296.39|5349.35|-2230.34| +2451874|55797|11531|22219|1380725|3231|9737|4|90|981|31|15.27|18.32|2.93|63.58|90.83|473.37|567.92|0.54|63.58|27.25|27.79|-446.12| +2451874|55797|1661|22219|1380725|3231|9737|4|240|981|79|64.24|68.09|64.00|151.68|5056.00|5074.96|5379.11|294.25|151.68|4904.32|5198.57|-170.64| +2451874|55797|17203|22219|1380725|3231|9737|4|300|981|40|71.36|119.88|119.88|0.00|4795.20|2854.40|4795.20|431.56|0.00|4795.20|5226.76|1940.80| +2451874|55797|7526|22219|1380725|3231|9737|4|127|981|77|21.72|26.93|8.88|0.00|683.76|1672.44|2073.61|34.18|0.00|683.76|717.94|-988.68| +2451874|55797|2351|22219|1380725|3231|9737|4|248|981|95|1.99|2.38|1.11|0.00|105.45|189.05|226.10|2.10|0.00|105.45|107.55|-83.60| +2451901|46139|5455|89770|372488|4806|3924|7|255|982|75|18.97|26.36|21.87|0.00|1640.25|1422.75|1977.00|65.61|0.00|1640.25|1705.86|217.50| +2451901|46139|1499|89770|372488|4806|3924|7|118|982|1|91.07|123.85|28.48|0.00|28.48|91.07|123.85|2.56|0.00|28.48|31.04|-62.59| +2451901|46139|9401|89770|372488|4806|3924|7|217|982|10|92.59|164.81|56.03|0.00|560.30|925.90|1648.10|50.42|0.00|560.30|610.72|-365.60| +2451901|46139|12083|89770|372488|4806|3924|7|25|982|59|99.50|138.30|134.15|0.00|7914.85|5870.50|8159.70|0.00|0.00|7914.85|7914.85|2044.35| +2451901|46139|11009|89770|372488|4806|3924|7|234|982|7|1.80|2.44|0.56|0.00|3.92|12.60|17.08|0.35|0.00|3.92|4.27|-8.68| +2451901|46139|15305|89770|372488|4806|3924|7|228|982|58|41.47|78.37|55.64|0.00|3227.12|2405.26|4545.46|290.44|0.00|3227.12|3517.56|821.86| +2451901|46139|9317|89770|372488|4806|3924|7|279|982|95|48.88|67.45|40.47|307.57|3844.65|4643.60|6407.75|176.85|307.57|3537.08|3713.93|-1106.52| +2451901|46139|5126|89770|372488|4806|3924|7|45|982|12|1.44|1.92|0.84|0.00|10.08|17.28|23.04|0.90|0.00|10.08|10.98|-7.20| +2451901|46139|16319|89770|372488|4806|3924|7|252|982|23|21.50|24.29|18.46|63.68|424.58|494.50|558.67|0.00|63.68|360.90|360.90|-133.60| +2451901|46139|10709|89770|372488|4806|3924|7|46|982|37|48.10|58.68|11.73|0.00|434.01|1779.70|2171.16|21.70|0.00|434.01|455.71|-1345.69| +2451901|46139|4997|89770|372488|4806|3924|7|295|982|10|40.46|51.38|14.38|17.25|143.80|404.60|513.80|3.79|17.25|126.55|130.34|-278.05| +2452209|70971|16925|10248|755052|4656|46260|4|258|983|26|9.01|11.89|3.32|32.80|86.32|234.26|309.14|0.53|32.80|53.52|54.05|-180.74| +2452209|70971|1009|10248|755052|4656|46260|4|186|983|78|63.51|71.76|50.23|3173.53|3917.94|4953.78|5597.28|14.88|3173.53|744.41|759.29|-4209.37| +2452209|70971|13691|10248|755052|4656|46260|4|260|983|69|29.03|50.51|43.43|0.00|2996.67|2003.07|3485.19|89.90|0.00|2996.67|3086.57|993.60| +2452209|70971|4213|10248|755052|4656|46260|4|147|983|81|20.20|21.61|11.66|0.00|944.46|1636.20|1750.41|75.55|0.00|944.46|1020.01|-691.74| +2452209|70971|12491|10248|755052|4656|46260|4|26|983|98|1.01|1.93|1.17|0.00|114.66|98.98|189.14|0.00|0.00|114.66|114.66|15.68| +2452209|70971|15127|10248|755052|4656|46260|4|68|983|91|34.24|66.42|36.53|0.00|3324.23|3115.84|6044.22|199.45|0.00|3324.23|3523.68|208.39| +2452209|70971|63|10248|755052|4656|46260|4|101|983|50|31.10|59.40|26.73|0.00|1336.50|1555.00|2970.00|120.28|0.00|1336.50|1456.78|-218.50| +2452209|70971|15297|10248|755052|4656|46260|4|146|983|53|10.20|14.07|4.22|0.00|223.66|540.60|745.71|15.65|0.00|223.66|239.31|-316.94| +2452209|70971|9433|10248|755052|4656|46260|4|89|983|19|70.79|135.20|89.23|0.00|1695.37|1345.01|2568.80|84.76|0.00|1695.37|1780.13|350.36| +2451139|32300|4489|81704|1275290|5676|23615|1|108|984|74|60.55|83.55|3.34|0.00|247.16|4480.70|6182.70|7.41|0.00|247.16|254.57|-4233.54| +2451139|32300|8425|81704|1275290|5676|23615|1|203|984|22|60.53|81.11|60.02|0.00|1320.44|1331.66|1784.42|105.63|0.00|1320.44|1426.07|-11.22| +2451139|32300|3694|81704|1275290|5676|23615|1|290|984|35|74.03|139.91|9.79|0.00|342.65|2591.05|4896.85|20.55|0.00|342.65|363.20|-2248.40| +2451139|32300|7130|81704|1275290|5676|23615|1|116|984|19|13.34|14.67|1.76|0.00|33.44|253.46|278.73|0.33|0.00|33.44|33.77|-220.02| +2451139|32300|2546|81704|1275290|5676|23615|1|194|984|16|61.19|72.81|0.72|6.22|11.52|979.04|1164.96|0.00|6.22|5.30|5.30|-973.74| +2451139|32300|10747|81704|1275290|5676|23615|1|81|984|17|43.09|79.28|45.98|0.00|781.66|732.53|1347.76|23.44|0.00|781.66|805.10|49.13| +2451139|32300|10498|81704|1275290|5676|23615|1|134|984|23|71.05|82.41|33.78|116.54|776.94|1634.15|1895.43|26.41|116.54|660.40|686.81|-973.75| +2451139|32300|10678|81704|1275290|5676|23615|1|157|984|84|94.30|116.93|35.07|0.00|2945.88|7921.20|9822.12|235.67|0.00|2945.88|3181.55|-4975.32| +2451139|32300|12914|81704|1275290|5676|23615|1|133|984|63|20.70|27.94|5.30|3.33|333.90|1304.10|1760.22|3.30|3.33|330.57|333.87|-973.53| +2451139|32300|9601|81704|1275290|5676|23615|1|52|984|1|15.40|24.02|1.92|0.00|1.92|15.40|24.02|0.15|0.00|1.92|2.07|-13.48| +2451139|32300|10880|81704|1275290|5676|23615|1|246|984|3|63.80|77.19|44.77|0.00|134.31|191.40|231.57|12.08|0.00|134.31|146.39|-57.09| +2451139||16772|81704||5676|23615|1|167|984|||92.17|||||3963.31|12.65||158.24||-2777.80| +2451139|32300|16933|81704|1275290|5676|23615|1|274|984|71|45.41|60.84|56.58|0.00|4017.18|3224.11|4319.64|200.85|0.00|4017.18|4218.03|793.07| +2451139|32300|17788|81704|1275290|5676|23615|1|68|984|99|92.86|113.28|80.42|0.00|7961.58|9193.14|11214.72|477.69|0.00|7961.58|8439.27|-1231.56| +2451139|32300|1948|81704|1275290|5676|23615|1|194|984|44|27.37|54.74|33.39|0.00|1469.16|1204.28|2408.56|29.38|0.00|1469.16|1498.54|264.88| +2451709|30696|11303|94603|532548|2140|26885|2|152|985|24|75.11|116.42|76.83|0.00|1843.92|1802.64|2794.08|129.07|0.00|1843.92|1972.99|41.28| +2451709|30696|16439|94603|532548|2140|26885|2|238|985|2|55.31|59.73|56.14|0.00|112.28|110.62|119.46|8.98|0.00|112.28|121.26|1.66| +||16532|94603|532548|2140||2|109|985||||0.58|0.00|28.42|1334.27||0.85|0.00|28.42|29.27|-1305.85| +2451709|30696|13699|94603|532548|2140|26885|2|2|985|68|93.43|140.14|0.00|0.00|0.00|6353.24|9529.52|0.00|0.00|0.00|0.00|-6353.24| +2451709|30696|5714|94603|532548|2140|26885|2|272|985|26|83.34|159.17|93.91|0.00|2441.66|2166.84|4138.42|195.33|0.00|2441.66|2636.99|274.82| +2451709|30696|8750|94603|532548|2140|26885|2|149|985|86|17.82|21.74|16.52|0.00|1420.72|1532.52|1869.64|113.65|0.00|1420.72|1534.37|-111.80| +2451709|30696|4987|94603|532548|2140|26885|2|167|985|57|35.43|52.08|19.79|947.54|1128.03|2019.51|2968.56|7.21|947.54|180.49|187.70|-1839.02| +2451709||26||532548|2140|||155|985|10||139.47|25.10||251.00|||||45.18|48.34|-666.42| +2451709|30696|14195|94603|532548|2140|26885|2|65|985|59|19.33|36.92|1.10|62.95|64.90|1140.47|2178.28|0.01|62.95|1.95|1.96|-1138.52| +2451709|30696|6701|94603|532548|2140|26885|2|116|985|76|31.47|45.94|12.86|0.00|977.36|2391.72|3491.44|87.96|0.00|977.36|1065.32|-1414.36| +2451709|30696|12133|94603|532548|2140|26885|2|176|985|31|74.71|92.64|8.33|0.00|258.23|2316.01|2871.84|18.07|0.00|258.23|276.30|-2057.78| +2451815|53773|11167|98010|1791847|6484|19568|8|153|986|44|57.84|80.97|10.52|0.00|462.88|2544.96|3562.68|4.62|0.00|462.88|467.50|-2082.08| +2451815|53773|17087|98010|1791847|6484|19568|8|274|986|51|15.08|16.13|12.09|61.65|616.59|769.08|822.63|16.64|61.65|554.94|571.58|-214.14| +2451815|53773|2917|98010|1791847|6484|19568|8|176|986|97|37.62|39.12|27.38|0.00|2655.86|3649.14|3794.64|26.55|0.00|2655.86|2682.41|-993.28| +2451815|53773|15122|98010|1791847|6484|19568|8|294|986|65|13.01|20.42|1.83|0.00|118.95|845.65|1327.30|3.56|0.00|118.95|122.51|-726.70| +2451815|53773|761|98010|1791847|6484|19568|8|132|986|66|84.34|167.83|135.94|0.00|8972.04|5566.44|11076.78|0.00|0.00|8972.04|8972.04|3405.60| +2451815|53773|8993|98010|1791847|6484|19568|8|110|986|62|2.61|3.05|1.67|0.00|103.54|161.82|189.10|3.10|0.00|103.54|106.64|-58.28| +2451815|53773|895|98010|1791847|6484|19568|8|216|986|96|25.77|36.85|23.21|1336.89|2228.16|2473.92|3537.60|35.65|1336.89|891.27|926.92|-1582.65| +2451815|53773|2659|98010|1791847|6484|19568|8|221|986|50|11.46|13.06|6.92|0.00|346.00|573.00|653.00|6.92|0.00|346.00|352.92|-227.00| +2451815|53773|8053|98010|1791847|6484|19568|8|192|986|8|87.44|97.05|51.43|0.00|411.44|699.52|776.40|4.11|0.00|411.44|415.55|-288.08| +2451815|53773|3473|98010|1791847|6484|19568|8|196|986|35|69.59|103.68|98.49|0.00|3447.15|2435.65|3628.80|172.35|0.00|3447.15|3619.50|1011.50| +2451815|53773|1729|98010|1791847|6484|19568|8|284|986|65|32.61|42.71|13.66|0.00|887.90|2119.65|2776.15|8.87|0.00|887.90|896.77|-1231.75| +2451815|53773|15746|98010|1791847|6484|19568|8|81|986|46|3.14|4.74|3.22|0.00|148.12|144.44|218.04|2.96|0.00|148.12|151.08|3.68| +2450900|68733|16669|89879|1663653|2674|41648|1|85|987|83|85.88|161.45|95.25|4427.22|7905.75|7128.04|13400.35|208.71|4427.22|3478.53|3687.24|-3649.51| +2450900|68733|1633|89879|1663653|2674|41648|1|187|987|8|97.05|124.22|86.95|125.20|695.60|776.40|993.76|34.22|125.20|570.40|604.62|-206.00| +2450900|68733|8191|89879|1663653|2674|41648|1|207|987|74|98.10|167.75|87.23|0.00|6455.02|7259.40|12413.50|387.30|0.00|6455.02|6842.32|-804.38| +2450900|68733|673|89879|1663653|2674|41648|1|289|987|12|81.45|103.44|37.23|0.00|446.76|977.40|1241.28|22.33|0.00|446.76|469.09|-530.64| +2450900|68733|15860|89879|1663653|2674|41648|1|288|987|93|10.97|18.31|3.29|0.00|305.97|1020.21|1702.83|6.11|0.00|305.97|312.08|-714.24| +2450900|68733|16484|89879|1663653|2674|41648|1|262|987|21|25.98|37.67|3.01|25.28|63.21|545.58|791.07|2.27|25.28|37.93|40.20|-507.65| +2450900|68733|4621|89879|1663653|2674|41648|1|104|987|34|25.94|48.24|3.85|0.00|130.90|881.96|1640.16|3.92|0.00|130.90|134.82|-751.06| +2450900|68733|1861|89879|1663653|2674|41648|1|143|987|66|20.01|31.81|6.04|0.00|398.64|1320.66|2099.46|0.00|0.00|398.64|398.64|-922.02| +2450900|68733|9286|89879|1663653|2674|41648|1|52|987|39|61.63|116.48|41.93|0.00|1635.27|2403.57|4542.72|0.00|0.00|1635.27|1635.27|-768.30| +2450900|68733|2077|89879|1663653|2674|41648|1|260|987|33|51.41|78.14|22.66|501.01|747.78|1696.53|2578.62|2.46|501.01|246.77|249.23|-1449.76| +2452363|53330|9342|1842|1580034|1873|45712|7|132|988|83|94.07|159.91|35.18|0.00|2919.94|7807.81|13272.53|0.00|0.00|2919.94|2919.94|-4887.87| +2452363|53330|4248|1842|1580034|1873|45712|7|60|988|41|65.38|130.10|50.73|1871.93|2079.93|2680.58|5334.10|18.72|1871.93|208.00|226.72|-2472.58| +2452363|53330|17526|1842|1580034|1873|45712|7|69|988|80|25.95|36.58|20.85|0.00|1668.00|2076.00|2926.40|83.40|0.00|1668.00|1751.40|-408.00| +|53330|16969|1842||1873|45712||230|988|71||45.93||0.00|||3261.03|15.32|0.00||1547.50|| +2452363|53330|7063|1842|1580034|1873|45712|7|91|988|5|71.75|93.27|11.19|0.00|55.95|358.75|466.35|2.23|0.00|55.95|58.18|-302.80| +2452363|53330|1626|1842|1580034|1873|45712|7|134|988|15|7.54|15.08|9.65|0.00|144.75|113.10|226.20|10.13|0.00|144.75|154.88|31.65| +2452363|53330|17371|1842|1580034|1873|45712|7|94|988|24|98.54|168.50|3.37|0.00|80.88|2364.96|4044.00|4.04|0.00|80.88|84.92|-2284.08| +2452363|53330|17215|1842|1580034|1873|45712|7|214|988|95|21.82|28.58|16.57|0.00|1574.15|2072.90|2715.10|110.19|0.00|1574.15|1684.34|-498.75| +2452363|53330|4681|1842|1580034|1873|45712|7|31|988|45|75.02|129.03|38.70|1393.20|1741.50|3375.90|5806.35|17.41|1393.20|348.30|365.71|-3027.60| +2452363|53330|1579|1842|1580034|1873|45712|7|49|988|66|62.65|103.37|29.97|0.00|1978.02|4134.90|6822.42|39.56|0.00|1978.02|2017.58|-2156.88| +2452363|53330|17983|1842|1580034|1873|45712|7|178|988|2|79.65|130.62|100.57|0.00|201.14|159.30|261.24|0.00|0.00|201.14|201.14|41.84| +2452363|53330|5379|1842|1580034|1873|45712|7|171|988|19|76.25|89.97|74.67|0.00|1418.73|1448.75|1709.43|85.12|0.00|1418.73|1503.85|-30.02| +2452363|53330|9531|1842|1580034|1873|45712|7|177|988|48|15.84|18.69|4.67|221.91|224.16|760.32|897.12|0.20|221.91|2.25|2.45|-758.07| +2452363|53330|12903|1842|||45712|7||988||14.72|27.52|26.69|0.00|2455.48||||0.00||2627.36|1101.24| +2452363|53330|16341|1842|1580034|1873|45712|7|293|988|92|99.93|139.90|46.16|0.00|4246.72|9193.56|12870.80|0.00|0.00|4246.72|4246.72|-4946.84| +2451076|54271|9922|90223|342014|652|29643|4|13|989|69|94.70|98.48|55.14|989.21|3804.66|6534.30|6795.12|28.15|989.21|2815.45|2843.60|-3718.85| +2451076|54271|14257|90223|342014|652|29643|4|84|989|38|52.21|79.35|40.46|0.00|1537.48|1983.98|3015.30|122.99|0.00|1537.48|1660.47|-446.50| +2451076|54271|10088|90223|342014|652|29643|4|152|989|75|19.04|31.03|29.16|0.00|2187.00|1428.00|2327.25|131.22|0.00|2187.00|2318.22|759.00| +2451076|54271|12535|90223|342014|652|29643|4|159|989|46|2.17|2.79|0.75|0.00|34.50|99.82|128.34|0.69|0.00|34.50|35.19|-65.32| +2451076|54271|15284|90223|342014|652|29643|4|148|989|83|3.95|5.96|5.12|0.00|424.96|327.85|494.68|29.74|0.00|424.96|454.70|97.11| +2451076|54271|8602|90223|342014|652|29643|4|210|989|7|49.35|57.73|6.35|0.00|44.45|345.45|404.11|1.33|0.00|44.45|45.78|-301.00| +2451076|54271|17744|90223|342014|652|29643|4|190|989|31|60.26|62.06|6.82|0.00|211.42|1868.06|1923.86|0.00|0.00|211.42|211.42|-1656.64| +2451076|54271|7658|90223|342014|652|29643|4|190|989|20|75.18|126.30|102.30|736.56|2046.00|1503.60|2526.00|13.09|736.56|1309.44|1322.53|-194.16| +2451534|36889|10198|10841|1702505|3769|25033|2|53|990|99|43.02|55.49|2.21|0.00|218.79|4258.98|5493.51|15.31|0.00|218.79|234.10|-4040.19| +2451534|36889|9931|10841|1702505|3769|25033|2|218|990|98|48.61|72.42|23.89|351.18|2341.22|4763.78|7097.16|119.40|351.18|1990.04|2109.44|-2773.74| +2451534|36889|14497|10841|1702505|3769|25033|2|270|990|15|58.30|110.77|46.52|0.00|697.80|874.50|1661.55|48.84|0.00|697.80|746.64|-176.70| +2451534|36889|9634|10841|1702505|3769|25033|2|150|990|83|28.36|47.07|7.06|345.72|585.98|2353.88|3906.81|9.61|345.72|240.26|249.87|-2113.62| +2451534|36889|8935|10841|1702505|3769|25033|2|268|990|47|59.31|73.54|36.03|0.00|1693.41|2787.57|3456.38|118.53|0.00|1693.41|1811.94|-1094.16| +2451534|36889|5684|10841|1702505|3769|25033|2|267|990|34|96.93|144.42|53.43|1780.28|1816.62|3295.62|4910.28|2.54|1780.28|36.34|38.88|-3259.28| +2451534|36889|17702|10841|1702505|3769|25033|2|94|990|25|50.13|56.64|41.91|0.00|1047.75|1253.25|1416.00|94.29|0.00|1047.75|1142.04|-205.50| +2451534|36889|5695|10841|1702505|3769|25033|2|105|990|73|9.22|17.61|5.81|0.00|424.13|673.06|1285.53|4.24|0.00|424.13|428.37|-248.93| +2451534|36889|9967|10841|1702505|3769|25033|2|104|990|2|14.52|28.31|9.05|0.00|18.10|29.04|56.62|1.62|0.00|18.10|19.72|-10.94| +2451534|36889|3901|10841|1702505|3769|25033|2|244|990|51|79.51|158.22|7.91|201.70|403.41|4055.01|8069.22|16.13|201.70|201.71|217.84|-3853.30| +2451534||8188|||3769|25033|||990||58.63|||||5804.37||279.86|||3389.45|| +2451534|36889|325|10841|1702505|3769|25033|2|101|990|28|92.02|137.10|15.08|0.00|422.24|2576.56|3838.80|29.55|0.00|422.24|451.79|-2154.32| +2451900||8615|||||10|253|991|14|||106.58|0.00||||0.00|0.00|||| +2451900|71503|541|26802|1789425|3248|3871|10|33|991|75|69.12|126.48|11.38|0.00|853.50|5184.00|9486.00|59.74|0.00|853.50|913.24|-4330.50| +2451900|71503|16201|26802|1789425|3248|3871|10|179|991|83|45.84|45.84|32.08|0.00|2662.64|3804.72|3804.72|239.63|0.00|2662.64|2902.27|-1142.08| +2451900|71503|3743|26802|1789425|3248|3871|10|186|991|70|70.28|106.82|65.16|0.00|4561.20|4919.60|7477.40|182.44|0.00|4561.20|4743.64|-358.40| +2451900|71503|8618|26802|1789425|3248|3871|10|45|991|36|24.10|44.82|3.58|0.00|128.88|867.60|1613.52|1.28|0.00|128.88|130.16|-738.72| +2451900|71503|8477|26802|1789425|3248|3871|10|96|991|64|53.25|92.65|29.64|0.00|1896.96|3408.00|5929.60|113.81|0.00|1896.96|2010.77|-1511.04| +2451900|71503|2425|26802|1789425|3248|3871|10|136|991|4|56.40|89.67|0.89|0.74|3.56|225.60|358.68|0.00|0.74|2.82|2.82|-222.78| +2451900|71503|8564|26802|1789425|3248|3871|10|44|991|30|13.91|24.62|18.95|0.00|568.50|417.30|738.60|22.74|0.00|568.50|591.24|151.20| +2452557|50533|6781|24333|1349691|7049|4760|10|95|992|11|69.45|87.50|25.37|128.37|279.07|763.95|962.50|12.05|128.37|150.70|162.75|-613.25| +2452557|50533|3759|24333|1349691|7049|4760|10|32|992|75|65.65|91.91|63.41|0.00|4755.75|4923.75|6893.25|428.01|0.00|4755.75|5183.76|-168.00| +2452557|50533|10611|24333|1349691|7049|4760|10|241|992|11|33.00|47.85|33.97|0.00|373.67|363.00|526.35|26.15|0.00|373.67|399.82|10.67| +2452557|50533|11928|24333|1349691|7049|4760|10|276|992|62|8.76|9.37|7.77|0.00|481.74|543.12|580.94|28.90|0.00|481.74|510.64|-61.38| +2452557|50533|4362|24333|1349691|7049|4760|10|86|992|83|59.04|72.61|55.18|0.00|4579.94|4900.32|6026.63|137.39|0.00|4579.94|4717.33|-320.38| +2452557|50533|1597|24333|1349691|7049|4760|10|183|992|29|87.84|132.63|123.34|2933.02|3576.86|2547.36|3846.27|6.43|2933.02|643.84|650.27|-1903.52| +2452557|50533|3283|24333|1349691|7049|4760|10|235|992|13|34.73|59.38|36.81|0.00|478.53|451.49|771.94|33.49|0.00|478.53|512.02|27.04| +2452557|50533|10416|24333|1349691|7049|4760|10|280|992|18|22.72|43.16|9.49|0.00|170.82|408.96|776.88|13.66|0.00|170.82|184.48|-238.14| +2452557|50533|13128|24333|1349691|7049|4760|10|118|992|8|46.70|84.52|56.62|0.00|452.96|373.60|676.16|4.52|0.00|452.96|457.48|79.36| +2452557|50533|9607|24333|1349691|7049|4760|10|223|992|27|36.16|44.11|38.81|0.00|1047.87|976.32|1190.97|83.82|0.00|1047.87|1131.69|71.55| +2452557|50533|573|24333|1349691|7049|4760|10|241|992|95|37.58|41.33|38.02|0.00|3611.90|3570.10|3926.35|36.11|0.00|3611.90|3648.01|41.80| +2452557||2925|24333|1349691|7049|4760|10||992|||||||||||49.98||-819.40| +2452557|50533|12709|24333|1349691|7049|4760|10|290|992|10|71.36|72.78|48.76|0.00|487.60|713.60|727.80|39.00|0.00|487.60|526.60|-226.00| +2452557|50533|16242|24333|1349691|7049|4760|10|81|992|59|82.52|100.67|1.00|0.00|59.00|4868.68|5939.53|4.72|0.00|59.00|63.72|-4809.68| +2452557|50533|17917|24333|1349691|7049|4760|10|117|992|32|94.24|186.59|139.94|0.00|4478.08|3015.68|5970.88|358.24|0.00|4478.08|4836.32|1462.40| +2452139|69517|9123|45215|1685935|1855|10175|4|70|993|82|11.61|15.09|2.56|0.00|209.92|952.02|1237.38|0.00|0.00|209.92|209.92|-742.10| +2452139|69517|7245|45215|1685935|1855|10175|4|86|993|5|26.53|33.42|31.41|0.00|157.05|132.65|167.10|14.13|0.00|157.05|171.18|24.40| +2452139|69517|7489|45215|1685935|1855|10175|4|133|993|40|16.10|22.54|12.84|118.12|513.60|644.00|901.60|3.95|118.12|395.48|399.43|-248.52| +|69517|1657||1685935|1855|||18|993||73.58|134.65|||||5520.65|||||847.47| +2452139|69517|3591|45215|1685935|1855|10175|4|38|993|21|59.89|66.47|66.47|0.00|1395.87|1257.69|1395.87|41.87|0.00|1395.87|1437.74|138.18| +2452139|69517|10537|45215|1685935|1855|10175|4|88|993|99|26.98|27.78|11.38|0.00|1126.62|2671.02|2750.22|45.06|0.00|1126.62|1171.68|-1544.40| +2452139|69517|153|45215|1685935|1855|10175|4|292|993|79|39.09|65.67|31.52|0.00|2490.08|3088.11|5187.93|74.70|0.00|2490.08|2564.78|-598.03| +2452139|69517|15273|45215|1685935|1855|10175|4|63|993|26|14.36|26.56|0.26|0.00|6.76|373.36|690.56|0.06|0.00|6.76|6.82|-366.60| +2452139|69517|8219|45215|1685935|1855|10175|4|65|993|29|76.48|81.83|39.27|0.00|1138.83|2217.92|2373.07|34.16|0.00|1138.83|1172.99|-1079.09| +2452139|69517|5513|45215|1685935|1855|10175|4|241|993|50|33.14|60.31|31.36|0.00|1568.00|1657.00|3015.50|109.76|0.00|1568.00|1677.76|-89.00| +2452139|69517|10227|45215|1685935|1855|10175|4|186|993|28|21.17|22.86|21.25|0.00|595.00|592.76|640.08|29.75|0.00|595.00|624.75|2.24| +2452139|69517|8391|45215|1685935|1855|10175|4|74|993|64|31.64|37.65|18.44|896.92|1180.16|2024.96|2409.60|14.16|896.92|283.24|297.40|-1741.72| +2452139|69517|4701|45215|1685935|1855|10175|4|96|993|93|94.98|183.31|47.66|0.00|4432.38|8833.14|17047.83|221.61|0.00|4432.38|4653.99|-4400.76| +2452139|69517|10379|45215|1685935|1855|10175|4|102|993|88|13.57|21.57|1.07|0.00|94.16|1194.16|1898.16|7.53|0.00|94.16|101.69|-1100.00| +2452139|69517|10837|45215|1685935|1855|10175|4|61|993|63|92.29|147.66|59.06|0.00|3720.78|5814.27|9302.58|111.62|0.00|3720.78|3832.40|-2093.49| +2452321|59926|17574|42746|54648|4950|15652|4|147|994|88|54.61|103.75|33.20|0.00|2921.60|4805.68|9130.00|204.51|0.00|2921.60|3126.11|-1884.08| +2452321|59926|2616|42746|54648|4950|15652|4|102|994|30|98.86|170.03|25.50|313.65|765.00|2965.80|5100.90|9.02|313.65|451.35|460.37|-2514.45| +2452321|59926|8355|42746|54648|4950|15652|4|105|994|86|9.64|10.50|1.15|0.00|98.90|829.04|903.00|5.93|0.00|98.90|104.83|-730.14| +2452321|59926|6480|42746|54648|4950|15652|4|112|994|6|39.09|50.81|40.64|173.12|243.84|234.54|304.86|5.65|173.12|70.72|76.37|-163.82| +2452321|59926|9867|42746|54648|4950|15652|4|280|994|60|17.58|21.44|0.42|2.26|25.20|1054.80|1286.40|0.00|2.26|22.94|22.94|-1031.86| +2452321|59926|14911|42746|54648|4950|15652|4|13|994|51|32.33|56.57|2.26|0.00|115.26|1648.83|2885.07|6.91|0.00|115.26|122.17|-1533.57| +2452321|59926|14071|42746|54648|4950|15652|4|123|994|33|41.87|72.43|58.66|0.00|1935.78|1381.71|2390.19|116.14|0.00|1935.78|2051.92|554.07| +2452321|59926|4320|42746|54648|4950|15652|4|159|994|54|24.33|42.09|31.14|0.00|1681.56|1313.82|2272.86|100.89|0.00|1681.56|1782.45|367.74| +2452321|59926|11731||||15652||84|994||7.76|15.05|5.71|0.00|||617.05||0.00|234.11||| +2452321|59926|15931|42746|54648|4950|15652|4|225|994|65|95.50|113.64|10.22|0.00|664.30|6207.50|7386.60|39.85|0.00|664.30|704.15|-5543.20| +2452321|59926|3859|42746|54648|4950|15652|4|30|994|96|11.07|13.28|2.39|4.58|229.44|1062.72|1274.88|13.49|4.58|224.86|238.35|-837.86| +2451875|70157|16949|9431|359377|4523|37669|8|93|995|36|38.70|60.75|18.83|0.00|677.88|1393.20|2187.00|33.89|0.00|677.88|711.77|-715.32| +2451875|70157|10802|9431|359377|4523|37669|8|293|995|73|8.25|9.81|5.10|0.00|372.30|602.25|716.13|26.06|0.00|372.30|398.36|-229.95| +2451875|70157|13183|9431|359377|4523|37669|8|39|995|22|35.97|50.35|44.81|512.62|985.82|791.34|1107.70|28.39|512.62|473.20|501.59|-318.14| +2451875|70157|12065|9431|359377|4523|37669|8|294|995|73|74.16|93.44|14.95|851.25|1091.35|5413.68|6821.12|16.80|851.25|240.10|256.90|-5173.58| +2451875|70157|11701|9431||||||995||64.17||52.18||||||||450.83|-95.92| +2451875|70157|4646|9431|359377|4523|37669|8|178|995|97|46.28|86.54|14.71|0.00|1426.87|4489.16|8394.38|99.88|0.00|1426.87|1526.75|-3062.29| +2451875|70157|15829|9431|359377|4523|37669|8|171|995|49|68.03|72.79|28.38|1237.65|1390.62|3333.47|3566.71|7.64|1237.65|152.97|160.61|-3180.50| +2451875|70157|7039|9431|359377|4523|37669|8|10|995|91|88.15|98.72|95.75|0.00|8713.25|8021.65|8983.52|261.39|0.00|8713.25|8974.64|691.60| +2451875|70157|11975|9431|359377|4523|37669|8|176|995|72|60.06|111.71|100.53|0.00|7238.16|4324.32|8043.12|217.14|0.00|7238.16|7455.30|2913.84| +2451875|70157|13451|9431|359377|4523|37669|8|89|995|47|16.22|32.27|24.52|0.00|1152.44|762.34|1516.69|69.14|0.00|1152.44|1221.58|390.10| +2451875|70157|7622|9431|359377|4523|37669|8|295|995|80|93.84|180.17|108.10|0.00|8648.00|7507.20|14413.60|778.32|0.00|8648.00|9426.32|1140.80| +||15089||359377||37669|8||995||36.83|61.13||0.00||331.47||14.02|0.00|467.64||136.17| +2451756|68700|14276|23194|851770|7193|10616|2|293|996|49|57.50|60.37|35.01|0.00|1715.49|2817.50|2958.13|137.23|0.00|1715.49|1852.72|-1102.01| +2451756|68700|13688|23194|851770|7193|10616|2|264|996|21|31.88|33.15|13.26|0.00|278.46|669.48|696.15|2.78|0.00|278.46|281.24|-391.02| +2451756|68700|5557|23194|851770|7193|10616|2|90|996|19|75.41|107.08|66.38|0.00|1261.22|1432.79|2034.52|75.67|0.00|1261.22|1336.89|-171.57| +2451756|68700|4265|23194|851770|7193|10616|2|221|996|54|75.90|86.52|6.05|0.00|326.70|4098.60|4672.08|16.33|0.00|326.70|343.03|-3771.90| +2451756|68700|10886|23194|851770|7193|10616|2|298|996|8|42.00|70.14|17.53|0.00|140.24|336.00|561.12|12.62|0.00|140.24|152.86|-195.76| +2451756|68700|5989|23194|851770|7193|10616|2|116|996|56|78.00|124.02|13.64|114.57|763.84|4368.00|6945.12|6.49|114.57|649.27|655.76|-3718.73| +2451756|68700|6824|23194|851770|7193|10616|2|245|996|92|37.25|50.66|7.09|0.00|652.28|3427.00|4660.72|13.04|0.00|652.28|665.32|-2774.72| +2451756|68700|5069|23194|851770|7193|10616|2|231|996|49|18.64|26.65|3.19|0.00|156.31|913.36|1305.85|0.00|0.00|156.31|156.31|-757.05| +2451756|68700|10826|23194|851770|7193|10616|2|85|996|88|51.69|85.80|71.21|0.00|6266.48|4548.72|7550.40|501.31|0.00|6266.48|6767.79|1717.76| +2451756|68700|14989|23194|851770|7193|10616|2|1|996|72|21.34|34.35|10.30|0.00|741.60|1536.48|2473.20|51.91|0.00|741.60|793.51|-794.88| +2451756|68700|6707|23194|851770|7193|10616|2|171|996|33|13.34|21.74|2.17|15.03|71.61|440.22|717.42|0.56|15.03|56.58|57.14|-383.64| +2451756|68700|17975|23194|851770|7193|10616|2|85|996|27|33.59|60.46|47.76|0.00|1289.52|906.93|1632.42|103.16|0.00|1289.52|1392.68|382.59| +2451756|68700|2737|23194|851770|7193|10616|2|183|996||||55.65|||||||||-35.19| +|68700|2354||||10616||278|996|26|67.25||||2316.34|1748.50|3217.24|185.30||2316.34|2501.64|567.84| +2451756|68700|14663|23194|851770|7193|10616|2|203|996|71|4.00|4.20|3.40|0.00|241.40|284.00|298.20|16.89|0.00|241.40|258.29|-42.60| +2451756|68700|6170|23194|851770|7193|10616|2|134|996|52|42.45|60.70|31.56|0.00|1641.12|2207.40|3156.40|147.70|0.00|1641.12|1788.82|-566.28| +2452502|52725|5791|24018|1521000|3135|29717|10|249|997|99|79.40|121.48|38.87|0.00|3848.13|7860.60|12026.52|0.00|0.00|3848.13|3848.13|-4012.47| +2452502|52725|624|24018|1521000|3135|29717|10|98|997|27|66.27|120.61|28.94|0.00|781.38|1789.29|3256.47|62.51|0.00|781.38|843.89|-1007.91| +2452502|52725|8347|24018|1521000|3135|29717|10|270|997|62|99.74|155.59|85.57|0.00|5305.34|6183.88|9646.58|424.42|0.00|5305.34|5729.76|-878.54| +2452502|52725|7212|24018|1521000|3135|29717|10|250|997|51|62.57|100.73|64.46|0.00|3287.46|3191.07|5137.23|262.99|0.00|3287.46|3550.45|96.39| +2452502|52725|17407|24018|1521000|3135|29717|10|219|997|22|43.05|69.31|68.61|0.00|1509.42|947.10|1524.82|120.75|0.00|1509.42|1630.17|562.32| +2452502|52725|4669|24018|1521000|3135|29717|10|216|997|63|68.49|128.76|1.28|0.00|80.64|4314.87|8111.88|2.41|0.00|80.64|83.05|-4234.23| +2452502|52725|16813|24018|1521000|3135|29717|10|297|997|51|51.54|85.55|55.60|1587.93|2835.60|2628.54|4363.05|24.95|1587.93|1247.67|1272.62|-1380.87| +2452502|52725|13122|24018|1521000|3135|29717|10|99|997|95|23.49|46.98|7.04|0.00|668.80|2231.55|4463.10|6.68|0.00|668.80|675.48|-1562.75| +2452502|52725|6006|24018|1521000|3135|29717|10|28|997|58|85.06|162.46|102.34|0.00|5935.72|4933.48|9422.68|0.00|0.00|5935.72|5935.72|1002.24| +2452502|52725|17658|24018|1521000|3135|29717|10|160|997|8|35.33|69.24|15.92|0.00|127.36|282.64|553.92|1.27|0.00|127.36|128.63|-155.28| +2452502|52725|4260|24018|1521000|3135|29717|10|265|997|5|60.46|84.64|20.31|0.00|101.55|302.30|423.20|7.10|0.00|101.55|108.65|-200.75| +2452502|52725|171|24018|1521000|3135|29717|10|90|997|62|95.18|98.03|68.62|0.00|4254.44|5901.16|6077.86|340.35|0.00|4254.44|4594.79|-1646.72| +2452502|52725|1071|24018|1521000|3135|29717|10|34|997|38|57.13|110.83|4.43|0.00|168.34|2170.94|4211.54|10.10|0.00|168.34|178.44|-2002.60| +2452502|52725|15313|24018|1521000|3135|29717|10|1|997|26|77.50|139.50|139.50|2901.60|3627.00|2015.00|3627.00|21.76|2901.60|725.40|747.16|-1289.60| +2452502|52725|13945|24018|1521000|3135|29717|10|27|997|81|78.49|97.32|14.59|0.00|1181.79|6357.69|7882.92|35.45|0.00|1181.79|1217.24|-5175.90| +2452502|52725|4305|24018|1521000|3135|29717|10|150|997|81|13.54|17.60|12.49|0.00|1011.69|1096.74|1425.60|50.58|0.00|1011.69|1062.27|-85.05| +2452409|52500|5461|64704|433625|6193|6714|8|209|998|82|58.98|99.67|12.95|0.00|1061.90|4836.36|8172.94|21.23|0.00|1061.90|1083.13|-3774.46| +||735|64704|433625|6193|6714|8|29|998|||||||||3.74||187.32|191.06|| +2452409|52500|7218|64704|433625|6193|6714|8|164|998|3|7.79|13.63|6.13|0.00|18.39|23.37|40.89|0.73|0.00|18.39|19.12|-4.98| +2452409|52500|11853|64704|433625|6193|6714|8|196|998|57|23.19|31.07|20.50|0.00|1168.50|1321.83|1770.99|70.11|0.00|1168.50|1238.61|-153.33| +2452409|52500|13968|64704|433625|6193|6714|8|179|998|62|82.60|146.20|54.09|1542.64|3353.58|5121.20|9064.40|108.65|1542.64|1810.94|1919.59|-3310.26| +||17823||433625||6714|8||998|81|13.25|21.99||0.00|88.29|1073.25|1781.19||0.00||90.05|| +2452409|52500|8139|64704|433625|6193|6714|8|186|998|5|60.45|110.62|58.62|0.00|293.10|302.25|553.10|26.37|0.00|293.10|319.47|-9.15| +2452409|52500|14029|64704|433625|6193||||998|94|91.47||10.31|||8598.18|12123.18|67.83|||1036.97|| +2451152|40988|10858|70130|429042|5107|45172|1|41|999|91|56.43|105.52|67.53|0.00|6145.23|5135.13|9602.32|553.07|0.00|6145.23|6698.30|1010.10| +2451152|40988|6553|70130|429042|5107|45172|1|33|999|95|58.32|79.31|42.82|0.00|4067.90|5540.40|7534.45|325.43|0.00|4067.90|4393.33|-1472.50| +2451152|40988|5756|70130|429042|5107|45172|1|134|999|84|67.33|133.98|52.25|0.00|4389.00|5655.72|11254.32|175.56|0.00|4389.00|4564.56|-1266.72| +2451152|40988|4441|70130|429042|5107|45172|1|185|999|67|48.80|93.69|74.01|3074.37|4958.67|3269.60|6277.23|113.05|3074.37|1884.30|1997.35|-1385.30| +2451152|40988|13724|70130|429042|5107|45172|1|102|999|74|98.18|179.66|46.71|1831.96|3456.54|7265.32|13294.84|146.21|1831.96|1624.58|1770.79|-5640.74| +2451152|40988|76|70130|429042|5107|45172|1|76|999|10|47.64|74.79|30.66|303.53|306.60|476.40|747.90|0.18|303.53|3.07|3.25|-473.33| +2451152|40988|15097|70130|429042|5107|45172|1|248|999|5|91.77|141.32|59.35|0.00|296.75|458.85|706.60|17.80|0.00|296.75|314.55|-162.10| +2451152|40988|17710|70130|429042|5107|45172|1|197|999|94|26.15|41.84|6.27|0.00|589.38|2458.10|3932.96|53.04|0.00|589.38|642.42|-1868.72| +2451152|40988|13312|70130|429042|5107|45172|1|69|999|96|51.08|82.74|52.95|2185.77|5083.20|4903.68|7943.04|231.79|2185.77|2897.43|3129.22|-2006.25| +2451152|40988|6853|70130|429042|5107|45172|1|244|999|74|84.07|130.30|88.60|0.00|6556.40|6221.18|9642.20|131.12|0.00|6556.40|6687.52|335.22| +2451152|40988|1018|70130|429042|5107|45172|1|17|999|78|41.93|67.50|62.10|0.00|4843.80|3270.54|5265.00|242.19|0.00|4843.80|5085.99|1573.26| +2451471|37905|10436|18428|1245785|3989|33746|8|102|1000|18|54.90|83.99|78.95|0.00|1421.10|988.20|1511.82|14.21|0.00|1421.10|1435.31|432.90| +2451471|37905|3157|18428|1245785|3989|33746|8|194|1000|5|41.45|64.66|40.08|0.00|200.40|207.25|323.30|2.00|0.00|200.40|202.40|-6.85| +2451471|37905|2732|18428|1245785|3989|33746|8|87|1000|11|38.77|39.54|3.95|39.53|43.45|426.47|434.94|0.07|39.53|3.92|3.99|-422.55| +2451471|37905|4945|18428|1245785|3989|33746|8|122|1000|92|97.04|120.32|45.72|0.00|4206.24|8927.68|11069.44|168.24|0.00|4206.24|4374.48|-4721.44| +2451471|37905|17053|18428|1245785|3989|33746|8|154|1000|14|2.70|4.48|0.49|5.00|6.86|37.80|62.72|0.13|5.00|1.86|1.99|-35.94| +2451471|37905|8683|18428|1245785|3989|33746|8|42|1000|48|84.54|92.99|79.97|0.00|3838.56|4057.92|4463.52|345.47|0.00|3838.56|4184.03|-219.36| +2451471|37905|15200|18428|1245785|3989|33746|8|217|1000|64|3.53|3.95|3.35|0.00|214.40|225.92|252.80|19.29|0.00|214.40|233.69|-11.52| +2451471|37905|6560|18428|1245785|3989|33746|8|172|1000|67|55.36|104.07|23.93|0.00|1603.31|3709.12|6972.69|0.00|0.00|1603.31|1603.31|-2105.81| +|37905|12838|18428|1245785|3989|33746|||1000|||97.94|||634.56|687.36||||||| +2451471|37905|14017|18428|1245785|3989|33746|8|163|1000|52|35.45|66.29|62.97|0.00|3274.44|1843.40|3447.08|32.74|0.00|3274.44|3307.18|1431.04| +2451471|37905|2692|18428|1245785|3989|33746|8|218|1000|26|14.24|20.36|16.69|195.27|433.94|370.24|529.36|11.93|195.27|238.67|250.60|-131.57| +2451471|37905|8329|18428|1245785|3989|33746|8|121|1000|67|43.80|58.69|38.14|1635.44|2555.38|2934.60|3932.23|36.79|1635.44|919.94|956.73|-2014.66| +2451471||7438|18428|1245785|3989||||1000|53|98.30|||0.00|2969.59||9898.81||0.00|2969.59||-2240.31| +2452215|47735|11465|97390|1368660|3130|20847|1|145|1001|37|81.46|153.95|69.27|0.00|2562.99|3014.02|5696.15|153.77|0.00|2562.99|2716.76|-451.03| +2452215|47735|10919|97390|1368660|3130|20847|1|87|1001|18|37.65|42.16|36.25|0.00|652.50|677.70|758.88|13.05|0.00|652.50|665.55|-25.20| +2452215|47735|5827|97390|1368660|3130|20847|1|231|1001|24|2.95|4.63|1.94|0.00|46.56|70.80|111.12|1.39|0.00|46.56|47.95|-24.24| +2452215|47735|5011|97390|1368660|3130|20847|1|34|1001|96|4.98|7.07|5.58|0.00|535.68|478.08|678.72|16.07|0.00|535.68|551.75|57.60| +2452215|47735|14243|97390|1368660|3130|20847|1|110|1001|67|67.91|108.65|41.28|0.00|2765.76|4549.97|7279.55|0.00|0.00|2765.76|2765.76|-1784.21| +2452215|47735|14939|97390|1368660|3130|20847|1|116|1001|94|94.40|158.59|34.88|0.00|3278.72|8873.60|14907.46|65.57|0.00|3278.72|3344.29|-5594.88| +2452215|47735|8803|97390|1368660|3130|20847|1|149|1001|19|29.93|32.62|16.31|0.00|309.89|568.67|619.78|0.00|0.00|309.89|309.89|-258.78| +2452215|47735|16683|97390|1368660|3130|20847|1|113|1001|67|25.79|34.30|33.95|0.00|2274.65|1727.93|2298.10|0.00|0.00|2274.65|2274.65|546.72| +2452215|47735|5503|97390|1368660|3130|20847|1|239|1001|70|3.69|5.86|0.87|0.00|60.90|258.30|410.20|3.04|0.00|60.90|63.94|-197.40| +2452215|47735|9979|97390|1368660|3130|20847|1|290|1001|17|84.84|159.49|19.13|136.58|325.21|1442.28|2711.33|3.77|136.58|188.63|192.40|-1253.65| +2452573|69156|3205|51073|1300168|6063|30758|2|1|1002|11|6.94|9.16|8.15|0.00|89.65|76.34|100.76|6.27|0.00|89.65|95.92|13.31| +2452573|69156|12504|51073|1300168|6063|30758|2|199|1002|68|58.76|82.26|76.50|0.00|5202.00|3995.68|5593.68|364.14|0.00|5202.00|5566.14|1206.32| +2452573|69156|1063|51073|1300168|6063|30758|2|179|1002|58|14.59|19.98|2.59|0.00|150.22|846.22|1158.84|0.00|0.00|150.22|150.22|-696.00| +2452573|69156|4947|51073|1300168|6063|30758|2|11|1002|100|43.61|75.88|51.59|3456.53|5159.00|4361.00|7588.00|136.19|3456.53|1702.47|1838.66|-2658.53| +2452573|69156|10524|51073|1300168|6063|30758|2|57|1002|5|36.58|55.23|52.46|0.00|262.30|182.90|276.15|10.49|0.00|262.30|272.79|79.40| +|69156|14322|51073||6063|||124|1002|75|56.34|105.91||935.65||4225.50|7943.25||935.65|||| +2452573|69156|15888|51073|1300168|6063|30758|2|32|1002|74|14.15|20.23|19.21|0.00|1421.54|1047.10|1497.02|127.93|0.00|1421.54|1549.47|374.44| +2452573|69156|15498|51073|1300168|6063|30758|2|46|1002|33|75.15|133.01|55.86|0.00|1843.38|2479.95|4389.33|55.30|0.00|1843.38|1898.68|-636.57| +2452573|69156|13185|51073|1300168|6063|30758|2|187|1002|88|99.52|119.42|62.09|0.00|5463.92|8757.76|10508.96|109.27|0.00|5463.92|5573.19|-3293.84| +2452573|69156|9799|51073|1300168|6063|30758|2|155|1002|49|78.44|109.03|4.36|0.00|213.64|3843.56|5342.47|19.22|0.00|213.64|232.86|-3629.92| +2452573|69156|13233|51073|1300168|6063|30758|2|286|1002|98|36.35|46.16|4.15|0.00|406.70|3562.30|4523.68|4.06|0.00|406.70|410.76|-3155.60| +2452573|69156|17869|51073|1300168|6063|30758|2|271|1002|11|3.21|3.43|0.92|0.00|10.12|35.31|37.73|0.10|0.00|10.12|10.22|-25.19| +2452573|69156|8293|51073|1300168|6063|30758|2|78|1002|38|26.34|48.72|30.20|700.03|1147.60|1000.92|1851.36|31.32|700.03|447.57|478.89|-553.35| +2452573|69156|11286|51073|1300168|6063|30758|2|133|1002|62|66.05|94.45|70.83|0.00|4391.46|4095.10|5855.90|0.00|0.00|4391.46|4391.46|296.36| +2452573|69156|525|51073|1300168|6063|30758|2|296|1002|75|72.79|130.29|115.95|2608.87|8696.25|5459.25|9771.75|0.00|2608.87|6087.38|6087.38|628.13| +2451470|48417|3151|69768|1109486|1204|22075|4|284|1003|7|93.77|187.54|15.00|95.55|105.00|656.39|1312.78|0.00|95.55|9.45|9.45|-646.94| +2451470|48417|5102|69768|1109486|1204|22075|4|163|1003|44|99.50|125.37|87.75|656.37|3861.00|4378.00|5516.28|128.18|656.37|3204.63|3332.81|-1173.37| +2451470|48417|13402|69768|1109486|1204|22075|4|223|1003|76|51.87|73.65|13.99|0.00|1063.24|3942.12|5597.40|21.26|0.00|1063.24|1084.50|-2878.88| +2451470|48417|15772|69768|1109486|1204|22075|4|42|1003|17|69.89|100.64|41.26|617.24|701.42|1188.13|1710.88|0.84|617.24|84.18|85.02|-1103.95| +2451470|48417|2696|69768|1109486|1204|22075|4|95|1003|22|31.82|52.18|7.82|0.00|172.04|700.04|1147.96|12.04|0.00|172.04|184.08|-528.00| +2451470|48417|268|69768|1109486|1204|22075|4|43|1003|98|67.24|119.01|21.42|0.00|2099.16|6589.52|11662.98|0.00|0.00|2099.16|2099.16|-4490.36| +2451470|48417|6577|69768|1109486|1204|22075|4|209|1003|3|61.68|93.13|33.52|0.00|100.56|185.04|279.39|5.02|0.00|100.56|105.58|-84.48| +2451470||13555||1109486||22075|4||1003|||51.54|||||257.70|0.00||0.00||-145.60| +2451470|48417|12757|69768|1109486|1204|22075|4|226|1003|31|42.46|70.05|38.52|0.00|1194.12|1316.26|2171.55|11.94|0.00|1194.12|1206.06|-122.14| +|48417|2326|||||4||1003||4.00||||55.20|276.00|463.68|1.10||55.20|56.30|| +2451470|48417|12604|69768|1109486|1204|22075|4|121|1003|100|89.79|115.82|79.91|0.00|7991.00|8979.00|11582.00|159.82|0.00|7991.00|8150.82|-988.00| +2451470|48417|7405|69768|1109486|1204|22075|4|238|1003|11|34.32|57.31|42.98|359.31|472.78|377.52|630.41|7.94|359.31|113.47|121.41|-264.05| +||5738||1109486|1204|22075||137|1003|89||||||8153.29||43.78||4378.80|4422.58|| +2451489|51951|12715|63948|511130|4265|17047|10|172|1004|23|50.05|53.05|4.77|0.00|109.71|1151.15|1220.15|8.77|0.00|109.71|118.48|-1041.44| +2451489|51951|1232|63948|511130|4265|17047|10|175|1004|91|15.40|26.64|25.30|828.82|2302.30|1401.40|2424.24|29.46|828.82|1473.48|1502.94|72.08| +2451489|51951|415|63948|511130|4265|17047|10|42|1004|65|23.99|32.86|8.87|0.00|576.55|1559.35|2135.90|5.76|0.00|576.55|582.31|-982.80| +||15439|63948||4265|17047||195|1004||||35.85|||||200.04|||2422.74|-1697.56| +2451489|51951|8312|63948|511130|4265|17047|10|110|1004|77|57.90|98.43|90.55|0.00|6972.35|4458.30|7579.11|209.17|0.00|6972.35|7181.52|2514.05| +2451489|51951|11174|63948|511130|4265|17047|10|199|1004|66|89.50|179.00|17.90|0.00|1181.40|5907.00|11814.00|82.69|0.00|1181.40|1264.09|-4725.60| +2451489|51951|14210|63948|511130|4265|17047|10|72|1004|76|62.93|111.38|25.61|0.00|1946.36|4782.68|8464.88|136.24|0.00|1946.36|2082.60|-2836.32| +2451489|51951|9637|63948|511130|4265|17047|10|137|1004|90|31.20|39.31|23.58|0.00|2122.20|2808.00|3537.90|148.55|0.00|2122.20|2270.75|-685.80| +2451489|51951|10706|63948|511130|4265|17047|10|120|1004|77|97.50|99.45|63.64|931.05|4900.28|7507.50|7657.65|277.84|931.05|3969.23|4247.07|-3538.27| +2451489|51951|9643|63948|511130|4265|17047|10|1|1004|3|83.43|102.61|62.59|0.00|187.77|250.29|307.83|7.51|0.00|187.77|195.28|-62.52| +2451489|51951|2542|63948|511130|4265|17047|10|143|1004|67|24.35|46.75|40.67|0.00|2724.89|1631.45|3132.25|163.49|0.00|2724.89|2888.38|1093.44| +2452018|37438|8247|36959||||10||1005||95.19||||5051.61|||50.51||5051.61||1529.58| +2452018|37438|3213|36959|1269904|5696|29441|10|47|1005|95|26.35|45.58|19.14|0.00|1818.30|2503.25|4330.10|54.54|0.00|1818.30|1872.84|-684.95| +2452018|37438|13775|36959|1269904|5696|29441|10|19|1005|90|59.01|76.71|37.58|2164.60|3382.20|5310.90|6903.90|48.70|2164.60|1217.60|1266.30|-4093.30| +2452018|37438|11641|36959||||10|179|1005|42|80.04|124.86|14.98||629.16|3361.68||||629.16||| +2452018|37438|1801|36959|1269904|5696|29441|10|85|1005|54|82.07|88.63|9.74|0.00|525.96|4431.78|4786.02|47.33|0.00|525.96|573.29|-3905.82| +2452018|37438|8261||||29441|10|236|1005||14.11|||0.00|1615.41|1312.23|2099.01||0.00|1615.41|1696.18|| +2452018||14527||1269904|5696|29441||152|1005||48.62|93.83|||4538.70|3986.84||||||| +2452018|37438|5415|36959|1269904|5696|29441|10|287|1005|15|98.20|140.42|88.46|145.95|1326.90|1473.00|2106.30|106.28|145.95|1180.95|1287.23|-292.05| +2452018|37438|13299|36959|1269904|5696|29441|10|59|1005|68|83.06|145.35|36.33|0.00|2470.44|5648.08|9883.80|222.33|0.00|2470.44|2692.77|-3177.64| +2452018||5473|36959|||29441|10|118|1005||66.27|114.64||||3976.20|6878.40|||1617.70|1666.23|-2358.50| +2452018|37438|637|36959|1269904|5696|29441|10|222|1005|61|1.35|2.06|1.75|0.00|106.75|82.35|125.66|6.40|0.00|106.75|113.15|24.40| +2452018|37438|15499|36959|1269904|5696|29441|10|80|1005|24|24.42|45.17|32.97|0.00|791.28|586.08|1084.08|7.91|0.00|791.28|799.19|205.20| +2452018|37438|349|36959|1269904|5696|29441|10|69|1005|16|61.88|79.20|11.08|0.00|177.28|990.08|1267.20|0.00|0.00|177.28|177.28|-812.80| +2452018|37438|351|36959|1269904|5696|29441|10|155|1005|13|32.70|43.81|34.17|111.05|444.21|425.10|569.53|6.66|111.05|333.16|339.82|-91.94| +2452018|37438|14797|||5696||10|135|1005||94.06|162.72|60.20|0.00|5297.60|8277.28||317.85|0.00|5297.60|5615.45|| +2451411|34343|2209|54165|1091001|4190|8626|7|180|1006|62|82.14|93.63|3.74|0.00|231.88|5092.68|5805.06|2.31|0.00|231.88|234.19|-4860.80| +2451411|34343|11800|54165|1091001|4190|8626|7|132|1006|46|25.04|46.82|20.13|0.00|925.98|1151.84|2153.72|74.07|0.00|925.98|1000.05|-225.86| +2451411|34343|8000|54165|1091001|4190|8626|7|21|1006|59|87.03|126.19|29.02|0.00|1712.18|5134.77|7445.21|34.24|0.00|1712.18|1746.42|-3422.59| +2451411|34343|12734|54165|1091001|4190|8626|7|249|1006|78|43.55|44.42|43.08|0.00|3360.24|3396.90|3464.76|100.80|0.00|3360.24|3461.04|-36.66| +2451411||1846||||||184|1006|32|||||115.52||2889.92|||115.52||| +2451411|34343|9154|54165|1091001|4190|8626|7|111|1006|74|66.93|87.00|44.37|0.00|3283.38|4952.82|6438.00|197.00|0.00|3283.38|3480.38|-1669.44| +2451411|34343|16405|54165|1091001|4190|8626|7|125|1006|47|33.35|43.68|23.15|0.00|1088.05|1567.45|2052.96|65.28|0.00|1088.05|1153.33|-479.40| +2451411|34343|17848|54165|1091001|4190|8626|7|218|1006|97|33.65|55.85|21.78|1415.48|2112.66|3264.05|5417.45|6.97|1415.48|697.18|704.15|-2566.87| +2451411|34343|2194|54165|1091001|4190|8626|7|282|1006|32|5.13|5.74|4.24|0.00|135.68|164.16|183.68|8.14|0.00|135.68|143.82|-28.48| +2451411|34343|13093|54165|1091001|4190|8626|7|6|1006|14|62.87|113.16|113.16|0.00|1584.24|880.18|1584.24|0.00|0.00|1584.24|1584.24|704.06| +2451411|34343|17840|54165|1091001|4190|8626|7|20|1006|44|40.79|59.14|45.53|0.00|2003.32|1794.76|2602.16|20.03|0.00|2003.32|2023.35|208.56| +2451411|34343|460|54165|1091001|4190|8626|7|279|1006|52|60.19|100.51|92.46|0.00|4807.92|3129.88|5226.52|192.31|0.00|4807.92|5000.23|1678.04| +2451411|34343|3560|54165|1091001|4190|8626|7|46|1006|44|49.80|56.27|9.56|0.00|420.64|2191.20|2475.88|8.41|0.00|420.64|429.05|-1770.56| +2451411|34343|103|54165|1091001|4190|8626|7|123|1006|18|48.31|93.72|12.18|0.00|219.24|869.58|1686.96|15.34|0.00|219.24|234.58|-650.34| +2451411|34343|5674|54165|1091001|4190|8626|7|235|1006|41|71.24|117.54|28.20|0.00|1156.20|2920.84|4819.14|104.05|0.00|1156.20|1260.25|-1764.64| +2452391|32533|17431|35430|1098138|399|25907|7|209|1007|60|85.51|148.78|72.90|0.00|4374.00|5130.60|8926.80|131.22|0.00|4374.00|4505.22|-756.60| +2452391|32533|12015|35430|1098138|399|25907|7|90|1007|51|30.63|54.82|37.82|327.89|1928.82|1562.13|2795.82|64.03|327.89|1600.93|1664.96|38.80| +2452391|32533|8629|35430|1098138|399|25907|7|184|1007|20|16.58|18.23|5.83|0.00|116.60|331.60|364.60|0.00|0.00|116.60|116.60|-215.00| +2452391|32533|5365|35430|1098138|399|25907|7|129|1007|31|14.62|26.60|21.81|0.00|676.11|453.22|824.60|47.32|0.00|676.11|723.43|222.89| +2452391|32533|9030|35430|1098138|399|25907|7|152|1007|57|30.45|55.11|52.35|1820.20|2983.95|1735.65|3141.27|69.82|1820.20|1163.75|1233.57|-571.90| +2452391|32533|14299|35430|1098138|399|25907|7|160|1007|90|3.41|5.49|1.48|0.00|133.20|306.90|494.10|3.99|0.00|133.20|137.19|-173.70| +2452391|32533|17043|35430|1098138|399|25907|7|255|1007|74|37.08|57.10|32.54|0.00|2407.96|2743.92|4225.40|216.71|0.00|2407.96|2624.67|-335.96| +2452391|32533|13423|35430|1098138|399|25907|7|120|1007|14|69.20|117.64|97.64|574.12|1366.96|968.80|1646.96|23.78|574.12|792.84|816.62|-175.96| +2452391|32533|10428|35430||399||||1007||28.34||3.96|210.98|285.12|2040.48|3570.48||210.98|74.14|77.10|-1966.34| +2452391|32533|6114|35430|1098138|399|25907|7|113|1007|40|97.50|109.20|104.83|0.00|4193.20|3900.00|4368.00|377.38|0.00|4193.20|4570.58|293.20| +2452391|32533|9861|35430|1098138|399|25907|7|240|1007|30|86.37|131.28|115.52|0.00|3465.60|2591.10|3938.40|34.65|0.00|3465.60|3500.25|874.50| +2452391|32533|10753|35430|1098138|399|25907|7|111|1007|60|99.83|187.68|31.90|210.54|1914.00|5989.80|11260.80|0.00|210.54|1703.46|1703.46|-4286.34| +2452182|40284|10381|5483|907684|285|27686|4|141|1008|62|30.62|33.06|24.46|0.00|1516.52|1898.44|2049.72|75.82|0.00|1516.52|1592.34|-381.92| +2452182|40284|10517|5483|907684|285|27686|4|263|1008|99|27.12|39.32|8.65|0.00|856.35|2684.88|3892.68|51.38|0.00|856.35|907.73|-1828.53| +2452182|40284|12199|5483|907684|285|27686|4|257|1008|87|66.27|75.54|18.88|0.00|1642.56|5765.49|6571.98|16.42|0.00|1642.56|1658.98|-4122.93| +2452182|40284|12407|5483|907684|285|27686|4|285|1008|30|69.05|73.19|70.26|2044.56|2107.80|2071.50|2195.70|2.52|2044.56|63.24|65.76|-2008.26| +2452182|40284|8457|5483|907684|285|27686|4|146|1008|60|86.63|117.81|115.45|0.00|6927.00|5197.80|7068.60|346.35|0.00|6927.00|7273.35|1729.20| +2452182|40284|16151|5483|907684|285|27686|4|216|1008|17|37.33|39.94|27.95|218.56|475.15|634.61|678.98|15.39|218.56|256.59|271.98|-378.02| +2452182|40284|15733|5483|907684|285|27686|4|287|1008|41|38.02|60.07|11.41|0.00|467.81|1558.82|2462.87|42.10|0.00|467.81|509.91|-1091.01| +|40284|6403||907684|285|27686|4|12|1008||45.79|||||||||323.16||-226.32| +2452182|40284|4093|5483|907684|285|27686|4|228|1008|35|3.10|4.86|1.36|0.00|47.60|108.50|170.10|3.80|0.00|47.60|51.40|-60.90| +2452182|40284|12163|5483|907684|285|27686|4|300|1008|61|65.60|66.25|56.31|0.00|3434.91|4001.60|4041.25|34.34|0.00|3434.91|3469.25|-566.69| +2452182|40284|15399|5483|907684|285|27686|4|95|1008|7|25.27|31.08|10.87|0.00|76.09|176.89|217.56|1.52|0.00|76.09|77.61|-100.80| +2452182|40284|13559|5483|907684|285|27686|4|102|1008|78|55.69|93.55|64.54|0.00|5034.12|4343.82|7296.90|302.04|0.00|5034.12|5336.16|690.30| +2452182|40284|10149|5483|907684|285|27686|4|291|1008|89|46.62|79.72|3.18|0.00|283.02|4149.18|7095.08|22.64|0.00|283.02|305.66|-3866.16| +2452182|40284|16663|5483|907684|285|27686|4|138|1008|38|68.81|103.90|6.23|0.00|236.74|2614.78|3948.20|9.46|0.00|236.74|246.20|-2378.04| +2452182|40284|14897|5483|907684|285|27686|4|254|1008|56|97.60|173.72|31.26|0.00|1750.56|5465.60|9728.32|35.01|0.00|1750.56|1785.57|-3715.04| +2452182|40284|2159|5483|907684|285|27686|4|72|1008|44|8.50|15.81|2.05|0.00|90.20|374.00|695.64|3.60|0.00|90.20|93.80|-283.80| +2452144|61893|13457|61572|1808364|309|38254|8|55|1009|34|18.91|32.52|21.46|0.00|729.64|642.94|1105.68|7.29|0.00|729.64|736.93|86.70| +2452144|61893|7449|61572|1808364|309|38254|8|299|1009|56|93.34|166.14|154.51|951.78|8652.56|5227.04|9303.84|0.00|951.78|7700.78|7700.78|2473.74| +2452144|61893|2713|61572|1808364|309|38254|8|21|1009|52|57.07|67.91|52.29|0.00|2719.08|2967.64|3531.32|135.95|0.00|2719.08|2855.03|-248.56| +2452144|61893|1399|61572|1808364|309|38254|8|43|1009|34|70.79|102.64|5.13|115.11|174.42|2406.86|3489.76|3.55|115.11|59.31|62.86|-2347.55| +2452144|61893|7339|61572|1808364|309|38254|8|65|1009|12|96.47|158.21|42.71|0.00|512.52|1157.64|1898.52|30.75|0.00|512.52|543.27|-645.12| +2452144|61893|8687|61572|1808364|309|38254|8|136|1009|87|53.35|79.49|74.72|0.00|6500.64|4641.45|6915.63|195.01|0.00|6500.64|6695.65|1859.19| +2452144|61893|12439|61572|1808364|309|38254|8|177|1009|13|19.83|31.72|23.15|0.00|300.95|257.79|412.36|18.05|0.00|300.95|319.00|43.16| +2452144|61893|9221|61572|1808364|309|38254|8|117|1009|18|55.38|72.54|12.33|0.00|221.94|996.84|1305.72|0.00|0.00|221.94|221.94|-774.90| +2452144|61893|993|61572|1808364|309|38254|8|208|1009|100|99.20|132.92|110.32|0.00|11032.00|9920.00|13292.00|110.32|0.00|11032.00|11142.32|1112.00| +2452144|61893|7487|61572|1808364|309|38254|8|238|1009|80|78.87|89.91|69.23|0.00|5538.40|6309.60|7192.80|0.00|0.00|5538.40|5538.40|-771.20| +2452144|61893|6289|61572|1808364|309|38254|8|157|1009|39|56.75|77.74|45.08|0.00|1758.12|2213.25|3031.86|105.48|0.00|1758.12|1863.60|-455.13| +2452144|61893|3869|61572|1808364|309|38254|8|241|1009|1|7.89|15.62|9.21|0.00|9.21|7.89|15.62|0.00|0.00|9.21|9.21|1.32| +2451413|72112|8863|10393|607607|1621|6035|8|103|1010|35|89.32|126.83|40.58|0.00|1420.30|3126.20|4439.05|14.20|0.00|1420.30|1434.50|-1705.90| +2451413|72112|8428|10393|607607|1621|6035|8|14|1010|95|16.09|25.58|20.71|0.00|1967.45|1528.55|2430.10|157.39|0.00|1967.45|2124.84|438.90| +2451413|72112|9326|10393|607607|1621|6035|8|298|1010|90|38.40|72.19|62.80|0.00|5652.00|3456.00|6497.10|339.12|0.00|5652.00|5991.12|2196.00| +2451413|72112|704|10393|607607|1621|6035|8|64|1010|17|24.86|29.83|5.36|0.00|91.12|422.62|507.11|4.55|0.00|91.12|95.67|-331.50| +2451413|72112|14168|10393|607607|1621|6035|8|291|1010|25|79.86|101.42|47.66|548.09|1191.50|1996.50|2535.50|57.90|548.09|643.41|701.31|-1353.09| +2451413|72112|694|10393|607607|1621|6035|8|117|1010|35|92.34|139.43|41.82|0.00|1463.70|3231.90|4880.05|117.09|0.00|1463.70|1580.79|-1768.20| +2451413|72112|15073|10393|607607|1621|6035|8|248|1010|100|1.64|3.24|2.59|0.00|259.00|164.00|324.00|20.72|0.00|259.00|279.72|95.00| +2451413|72112|7454|10393|607607|1621|6035|8|125|1010|85|14.22|23.74|13.53|0.00|1150.05|1208.70|2017.90|57.50|0.00|1150.05|1207.55|-58.65| +2451413|72112|446|10393|607607|1621|6035|8|300|1010|75|11.09|14.41|2.16|0.00|162.00|831.75|1080.75|9.72|0.00|162.00|171.72|-669.75| +2451413|72112|12434|10393|607607|1621|6035|8|152|1010|48|29.32|52.48|10.49|0.00|503.52|1407.36|2519.04|25.17|0.00|503.52|528.69|-903.84| +2451413|72112|6110|10393|607607|1621|6035|8|239|1010|27|22.23|28.89|5.20|120.74|140.40|600.21|780.03|0.58|120.74|19.66|20.24|-580.55| +2451413|72112|6044|10393|607607|1621|6035|8|160|1010|7|5.03|8.65|4.23|0.00|29.61|35.21|60.55|2.36|0.00|29.61|31.97|-5.60| +2451413|72112|11209|10393|607607|1621|6035|8|19|1010|44|78.07|118.66|109.16|0.00|4803.04|3435.08|5221.04|432.27|0.00|4803.04|5235.31|1367.96| +2452371|67008|13449|19519|||6815|7||1011|||58.48||0.00||2312.34||41.15|0.00|685.95|727.10|| +2452371|67008|16837|19519|1427551|2783|6815|7|281|1011|99|78.88|108.85|67.48|0.00|6680.52|7809.12|10776.15|0.00|0.00|6680.52|6680.52|-1128.60| +2452371|67008|7303|19519|1427551|2783|6815|7|3|1011|87|5.47|9.95|7.06|0.00|614.22|475.89|865.65|55.27|0.00|614.22|669.49|138.33| +2452371|67008|8449|19519|1427551|2783|6815|7|105|1011|72|49.98|74.97|70.47|0.00|5073.84|3598.56|5397.84|202.95|0.00|5073.84|5276.79|1475.28| +2452371|67008|16429|19519|1427551|2783|6815|7|195|1011|10|54.46|108.37|81.27|0.00|812.70|544.60|1083.70|24.38|0.00|812.70|837.08|268.10| +2452371|67008|5329|19519|1427551|2783|6815|7|8|1011|46|96.41|121.47|110.53|0.00|5084.38|4434.86|5587.62|152.53|0.00|5084.38|5236.91|649.52| +2452371|67008|16551|19519|1427551|2783|6815|7|119|1011|17|38.70|51.85|9.33|0.00|158.61|657.90|881.45|14.27|0.00|158.61|172.88|-499.29| +2452371|67008|51|19519|1427551|2783|6815|7|18|1011|48|95.38|157.37|111.73|0.00|5363.04|4578.24|7553.76|214.52|0.00|5363.04|5577.56|784.80| +2452371|67008|13800||||6815|7||1011||17.06|18.42||0.00|215.73|870.06|939.42|17.25|0.00||232.98|-654.33| +2452371|67008|12123|19519|1427551|2783|6815|7|241|1011|85|78.28|104.89|96.49|0.00|8201.65|6653.80|8915.65|328.06|0.00|8201.65|8529.71|1547.85| +2452371|67008|17623|19519|1427551|2783|6815|7|287|1011|68|99.46|163.11|53.82|0.00|3659.76|6763.28|11091.48|182.98|0.00|3659.76|3842.74|-3103.52| +2452371|67008|8151|19519|1427551|2783|6815|7|12|1011|8|9.11|17.76|8.70|0.00|69.60|72.88|142.08|3.48|0.00|69.60|73.08|-3.28| +2452371|67008|7644|19519|1427551|2783|6815|7|214|1011|78|27.14|42.88|25.29|0.00|1972.62|2116.92|3344.64|39.45|0.00|1972.62|2012.07|-144.30| +2452371|67008|14844|19519|1427551|2783|6815|7|31|1011|1|5.99|11.14|4.23|0.00|4.23|5.99|11.14|0.21|0.00|4.23|4.44|-1.76| +2451074|61024|7720|84957|1271930|5750|32370|8|245|1012|60|39.91|56.67|53.83|1356.51|3229.80|2394.60|3400.20|112.39|1356.51|1873.29|1985.68|-521.31| +2451074|61024|10372|84957|1271930|5750|32370|8|193|1012|29|17.93|21.15|21.15|263.74|613.35|519.97|613.35|17.48|263.74|349.61|367.09|-170.36| +2451074|61024|16681|84957|1271930|5750|32370|8|152|1012|61|71.23|78.35|37.60|0.00|2293.60|4345.03|4779.35|68.80|0.00|2293.60|2362.40|-2051.43| +2451074|61024|10333|84957|1271930|5750|32370|8|112|1012|24|51.39|52.41|46.64|0.00|1119.36|1233.36|1257.84|11.19|0.00|1119.36|1130.55|-114.00| +2451074|61024|9841|84957|1271930|5750|32370|8|276|1012|48|21.04|40.81|13.87|0.00|665.76|1009.92|1958.88|13.31|0.00|665.76|679.07|-344.16| +2451074|61024|16118|84957|1271930|5750|32370|8|190|1012|70|85.32|126.27|35.35|2053.83|2474.50|5972.40|8838.90|0.00|2053.83|420.67|420.67|-5551.73| +2451074|61024|16990|84957|1271930|5750|32370|8|237|1012|42|56.03|61.07|42.13|53.08|1769.46|2353.26|2564.94|34.32|53.08|1716.38|1750.70|-636.88| +2451074|61024|4210|84957|1271930|5750|32370|8|229|1012|23|86.39|91.57|17.39|0.00|399.97|1986.97|2106.11|0.00|0.00|399.97|399.97|-1587.00| +2451074|61024|11989|84957|1271930|5750|32370|8|40|1012|43|23.32|46.64|20.98|0.00|902.14|1002.76|2005.52|81.19|0.00|902.14|983.33|-100.62| +2451074|61024|13694|84957|1271930|5750|32370|8|158|1012|58|62.41|81.13|58.41|643.67|3387.78|3619.78|4705.54|192.08|643.67|2744.11|2936.19|-875.67| +2451147|72663|17888|70329|1398224|2566|3261|2|24|1013|86|38.47|51.54|41.74|0.00|3589.64|3308.42|4432.44|287.17|0.00|3589.64|3876.81|281.22| +2451147|72663|11884|70329|1398224|2566|3261|2|130|1013|59|57.56|104.18|27.08|1358.06|1597.72|3396.04|6146.62|9.58|1358.06|239.66|249.24|-3156.38| +2451147|72663|421|70329|1398224|2566|3261|2|177|1013|46|38.22|48.53|1.94|24.09|89.24|1758.12|2232.38|5.86|24.09|65.15|71.01|-1692.97| +2451147|72663|12802|70329|1398224|2566|3261|2|75|1013|55|80.14|157.87|148.39|0.00|8161.45|4407.70|8682.85|571.30|0.00|8161.45|8732.75|3753.75| +2451147|72663|5762|70329|1398224|2566|3261|2|257|1013|97|30.75|33.51|18.09|0.00|1754.73|2982.75|3250.47|17.54|0.00|1754.73|1772.27|-1228.02| +2451147|72663|4348|70329|1398224|2566|3261|2|134|1013|59|80.86|130.18|24.73|204.26|1459.07|4770.74|7680.62|75.28|204.26|1254.81|1330.09|-3515.93| +2451147|72663|3061|70329|1398224|2566|3261|2|4|1013|31|11.64|14.20|6.95|0.00|215.45|360.84|440.20|4.30|0.00|215.45|219.75|-145.39| +2451147|72663|6898|70329|1398224|2566|3261|2|14|1013|43|17.78|22.04|3.52|0.00|151.36|764.54|947.72|9.08|0.00|151.36|160.44|-613.18| +2451147|72663|15416|70329|1398224|2566|3261|2|37|1013|89|54.72|56.36|27.05|0.00|2407.45|4870.08|5016.04|96.29|0.00|2407.45|2503.74|-2462.63| +2451147|72663|17294|70329|1398224|2566|3261|2|213|1013|47|50.10|50.10|37.57|0.00|1765.79|2354.70|2354.70|123.60|0.00|1765.79|1889.39|-588.91| +2451147|72663|7424|70329|1398224|2566|3261|2|97|1013|54|46.71|63.05|42.87|0.00|2314.98|2522.34|3404.70|115.74|0.00|2314.98|2430.72|-207.36| +2451147|72663|6364|70329|1398224|2566|3261|2|247|1013|4|60.32|91.08|49.18|0.00|196.72|241.28|364.32|17.70|0.00|196.72|214.42|-44.56| +2451147|72663|14035|70329|1398224|2566|3261|2|133|1013|60|49.21|73.32|10.99|0.00|659.40|2952.60|4399.20|52.75|0.00|659.40|712.15|-2293.20| +2452509|72148|6939|65842|1476634|3873|3065|4|187|1014|76|10.79|10.79|2.69|155.37|204.44|820.04|820.04|0.49|155.37|49.07|49.56|-770.97| +2452509|72148|9906|65842|1476634|3873|3065|4|56|1014|76|23.18|30.59|16.21|0.00|1231.96|1761.68|2324.84|12.31|0.00|1231.96|1244.27|-529.72| +2452509|72148|6954|65842|1476634|3873|3065|4|144|1014|34|92.36|132.99|130.33|0.00|4431.22|3140.24|4521.66|398.80|0.00|4431.22|4830.02|1290.98| +2452509|72148|11040|65842|1476634|3873|3065|4|29|1014|65|88.62|172.80|60.48|0.00|3931.20|5760.30|11232.00|0.00|0.00|3931.20|3931.20|-1829.10| +2452509|72148|16797|65842|1476634|3873|3065|4|8|1014|11|39.50|52.14|2.60|0.00|28.60|434.50|573.54|0.00|0.00|28.60|28.60|-405.90| +|72148|1551|65842|1476634|3873|3065||204|1014|18|59.37||58.18|0.00|1047.24|1068.66|1292.94||0.00|1047.24||| +2452509|72148|12774|65842|1476634|3873|3065|4|241|1014|90|95.76|109.16|104.79|0.00|9431.10|8618.40|9824.40|660.17|0.00|9431.10|10091.27|812.70| +2452509|72148|5467|65842|1476634|3873|3065|4|153|1014|15|46.89|59.55|17.26|0.00|258.90|703.35|893.25|2.58|0.00|258.90|261.48|-444.45| +2452509|72148|8817|65842|1476634|3873|3065|4|240|1014|90|80.05|108.06|97.25|0.00|8752.50|7204.50|9725.40|262.57|0.00|8752.50|9015.07|1548.00| +2452509|72148|7029|65842|1476634|3873|3065|4|185|1014|99|19.76|24.30|21.62|1155.80|2140.38|1956.24|2405.70|68.92|1155.80|984.58|1053.50|-971.66| +||2155||||3065||153|1014||93.75|171.56||0.00||2250.00||13.17|0.00|658.56|671.73|| +2452509|72148|15390|65842|1476634|3873|3065|4|29|1014|35|4.43|8.28|7.61|0.00|266.35|155.05|289.80|15.98|0.00|266.35|282.33|111.30| +2452509|72148|6265|65842|1476634|3873|3065|4|104|1014|40|36.07|50.85|31.01|0.00|1240.40|1442.80|2034.00|86.82|0.00|1240.40|1327.22|-202.40| +2452509|72148|13074|||||4|135|1014||20.82||17.48|||1124.28||37.75||943.92||-180.36| +2451893|61983|14654|58910|1300278|5999|21767|1|252|1015|18|11.52|14.16|9.48|0.00|170.64|207.36|254.88|15.35|0.00|170.64|185.99|-36.72| +2451893|61983|272|58910|1300278|5999|21767|1|197|1015|52|41.84|58.99|11.79|0.00|613.08|2175.68|3067.48|18.39|0.00|613.08|631.47|-1562.60| +2451893|61983|9431|58910|1300278|5999|21767|1|154|1015|50|60.00|64.80|36.93|0.00|1846.50|3000.00|3240.00|147.72|0.00|1846.50|1994.22|-1153.50| +2451893|61983|15668|58910|1300278|5999|21767|1|188|1015|4|15.72|22.16|18.83|57.24|75.32|62.88|88.64|1.26|57.24|18.08|19.34|-44.80| +2451893|61983|13535|58910|1300278|5999|21767|1|98|1015|11|57.61|108.88|33.75|133.65|371.25|633.71|1197.68|9.50|133.65|237.60|247.10|-396.11| +||3098|58910||5999|||69|1015||98.12|||0.00||||61.64|0.00|1027.36||| +2451893|61983|2363|58910|1300278|5999|21767|1|241|1015|16|20.73|27.98|27.98|0.00|447.68|331.68|447.68|31.33|0.00|447.68|479.01|116.00| +2451893|61983|8999|58910|1300278|5999|21767|1|119|1015|11|9.97|18.94|11.93|0.00|131.23|109.67|208.34|9.18|0.00|131.23|140.41|21.56| +2451893|61983|15593|58910|1300278|5999|21767|1|172|1015|15|1.17|1.38|0.42|1.19|6.30|17.55|20.70|0.00|1.19|5.11|5.11|-12.44| +2451893|61983|5677|58910|1300278|5999|21767|1|258|1015|39|95.75|181.92|112.79|0.00|4398.81|3734.25|7094.88|175.95|0.00|4398.81|4574.76|664.56| +2451893|61983|16706|58910|1300278|5999|21767|1|162|1015|97|51.18|75.23|4.51|0.00|437.47|4964.46|7297.31|8.74|0.00|437.47|446.21|-4526.99| +2451893|61983|16466|58910|1300278|5999|21767|1|281|1015|9|78.54|141.37|81.99|0.00|737.91|706.86|1272.33|7.37|0.00|737.91|745.28|31.05| +2451893|61983|17639|58910|1300278|5999|21767|1|167|1015|5|5.43|8.25|4.04|0.00|20.20|27.15|41.25|1.41|0.00|20.20|21.61|-6.95| +2451893|61983|11144|58910|1300278|5999|21767|1|107|1015|84|56.14|67.36|54.56|0.00|4583.04|4715.76|5658.24|0.00|0.00|4583.04|4583.04|-132.72| +2451893|61983|2153|58910|1300278|5999|21767|1|193|1015|15|55.99|68.86|56.46|0.00|846.90|839.85|1032.90|33.87|0.00|846.90|880.77|7.05| +2451893||1777||1300278|5999||1|117|1015|30|||15.97|||803.70|1141.20|||479.10||-324.60| +2452602|63194|11268|34393|84902|564|36673|4|107|1016|81|39.01|57.73|35.21|0.00|2852.01|3159.81|4676.13|114.08|0.00|2852.01|2966.09|-307.80| +2452602|63194|12261|34393|84902|564|36673|4|241|1016|28|84.60|137.89|37.23|0.00|1042.44|2368.80|3860.92|83.39|0.00|1042.44|1125.83|-1326.36| +2452602|63194|1668|34393|84902|564|36673|4|161|1016|95|15.85|30.27|2.11|0.00|200.45|1505.75|2875.65|16.03|0.00|200.45|216.48|-1305.30| +2452602|63194|13443|34393|84902|564|36673|4|44|1016|41|65.63|116.16|46.46|0.00|1904.86|2690.83|4762.56|0.00|0.00|1904.86|1904.86|-785.97| +2452602|63194|822|34393|84902|564|36673|4|279|1016|12|58.45|77.15|66.34|0.00|796.08|701.40|925.80|63.68|0.00|796.08|859.76|94.68| +2452602|63194|6583|34393|84902|564|36673|4|125|1016|7|85.28|162.03|87.49|0.00|612.43|596.96|1134.21|6.12|0.00|612.43|618.55|15.47| +2452602|63194|3177|34393|84902|564|36673|4|136|1016|18|93.63|98.31|37.35|0.00|672.30|1685.34|1769.58|60.50|0.00|672.30|732.80|-1013.04| +2452602|63194|2371|34393|84902|564|36673|4|148|1016|96|8.83|9.35|3.08|0.00|295.68|847.68|897.60|23.65|0.00|295.68|319.33|-552.00| +2452602|63194|10423|34393|84902|564|36673|4|46|1016|80|57.95|92.72|89.01|0.00|7120.80|4636.00|7417.60|71.20|0.00|7120.80|7192.00|2484.80| +2452602|63194|2334|34393|84902|564|36673|4|176|1016|76|88.65|140.06|22.40|0.00|1702.40|6737.40|10644.56|51.07|0.00|1702.40|1753.47|-5035.00| +2452602|63194|7429|34393|84902|564|36673|4|207|1016|73|78.98|136.63|98.37|0.00|7181.01|5765.54|9973.99|359.05|0.00|7181.01|7540.06|1415.47| +2452602|63194|8184|34393|84902|564|36673|4|204|1016|38|69.92|132.14|0.00|0.00|0.00|2656.96|5021.32|0.00|0.00|0.00|0.00|-2656.96| +2452602|63194|2454|34393|84902|564|36673|4|71|1016|35|30.12|53.01|6.36|0.00|222.60|1054.20|1855.35|6.67|0.00|222.60|229.27|-831.60| +2452602|63194|13645|34393|84902|564|36673|4|113|1016|3|76.71|140.37|50.53|0.00|151.59|230.13|421.11|3.03|0.00|151.59|154.62|-78.54| +2452602|63194|723|34393|84902|564|36673|4|116|1016|83|74.45|78.17|32.04|0.00|2659.32|6179.35|6488.11|79.77|0.00|2659.32|2739.09|-3520.03| +2452602|63194|3048|34393||564||||1016||90.66|135.08||0.00|1215.60||2701.60||0.00|1215.60|1288.53|-597.60| +2451848|32910|7825|47982|958968|4354|41740|2|112|1017|79|84.20|153.24|53.63|0.00|4236.77|6651.80|12105.96|338.94|0.00|4236.77|4575.71|-2415.03| +2451848|32910|14351|47982|958968|4354|41740|2|54|1017|74|21.34|21.55|16.59|0.00|1227.66|1579.16|1594.70|49.10|0.00|1227.66|1276.76|-351.50| +2451848|32910|8719|47982|958968|4354|41740|2|140|1017|19|37.34|62.35|40.52|0.00|769.88|709.46|1184.65|38.49|0.00|769.88|808.37|60.42| +2451848|32910|10555|47982|958968|4354|41740|2|266|1017|26|50.63|69.36|63.11|0.00|1640.86|1316.38|1803.36|98.45|0.00|1640.86|1739.31|324.48| +2451848|32910|15119|47982|958968|4354|41740|2|249|1017|36|92.85|115.13|96.70|0.00|3481.20|3342.60|4144.68|0.00|0.00|3481.20|3481.20|138.60| +2451848|32910|11216|47982|958968|4354|41740|2|260|1017|21|54.80|63.56|61.01|0.00|1281.21|1150.80|1334.76|102.49|0.00|1281.21|1383.70|130.41| +2451848|32910|16657|47982|958968|4354|41740|2|284|1017|75|99.13|161.58|9.69|625.00|726.75|7434.75|12118.50|0.00|625.00|101.75|101.75|-7333.00| +2451848|32910|6446|47982|958968|4354|41740|2|97|1017|68|85.03|111.38|111.38|0.00|7573.84|5782.04|7573.84|0.00|0.00|7573.84|7573.84|1791.80| +2451848|32910|14867|47982|958968|4354|41740|2|11|1017|38|92.88|124.45|97.07|0.00|3688.66|3529.44|4729.10|147.54|0.00|3688.66|3836.20|159.22| +2452613|45805|1050|27013|988357|3301|16645|8|154|1018|91|8.76|15.06|3.46|0.00|314.86|797.16|1370.46|22.04|0.00|314.86|336.90|-482.30| +2452613|45805|7179|27013|988357|3301|16645|8|17|1018|83|82.84|109.34|57.95|3751.68|4809.85|6875.72|9075.22|52.90|3751.68|1058.17|1111.07|-5817.55| +2452613|45805|16563|27013|988357|3301|16645|8|111|1018|81|12.26|13.97|6.42|447.21|520.02|993.06|1131.57|2.18|447.21|72.81|74.99|-920.25| +2452613|45805|16633|27013|988357|3301|16645|8|38|1018|89|4.20|6.09|0.24|0.00|21.36|373.80|542.01|1.70|0.00|21.36|23.06|-352.44| +2452613|45805|11407|27013|988357|3301|16645|8|249|1018|89|9.91|18.72|13.47|0.00|1198.83|881.99|1666.08|0.00|0.00|1198.83|1198.83|316.84| +2452613|45805|7074|27013|988357|3301|16645|8|232|1018|60|86.77|124.08|18.61|0.00|1116.60|5206.20|7444.80|55.83|0.00|1116.60|1172.43|-4089.60| +2452613|45805|15195|27013|988357|3301|16645|8|213|1018|50|51.20|81.40|32.56|0.00|1628.00|2560.00|4070.00|32.56|0.00|1628.00|1660.56|-932.00| +2452613|45805|6483|27013|988357|3301|16645|8|104|1018|48|55.65|111.30|90.15|0.00|4327.20|2671.20|5342.40|173.08|0.00|4327.20|4500.28|1656.00| +2452613|45805|2245|27013|988357|3301|16645|8|2|1018|16|84.77|166.99|96.85|0.00|1549.60|1356.32|2671.84|61.98|0.00|1549.60|1611.58|193.28| +2452613|45805|10279|27013|988357|3301|16645|8|221|1018|87|61.65|115.90|62.58|0.00|5444.46|5363.55|10083.30|272.22|0.00|5444.46|5716.68|80.91| +2451799|45692|14132|83607|1462126|5445|27611|7|176|1019|65|63.13|86.48|25.94|0.00|1686.10|4103.45|5621.20|0.00|0.00|1686.10|1686.10|-2417.35| +2451799|45692|8948|83607|1462126|5445|27611|7|296|1019|60|8.47|9.23|7.47|0.00|448.20|508.20|553.80|22.41|0.00|448.20|470.61|-60.00| +2451799|45692|2317|83607|1462126|5445|27611|7|71|1019|90|86.92|121.68|64.49|0.00|5804.10|7822.80|10951.20|0.00|0.00|5804.10|5804.10|-2018.70| +2451799|45692|1999|83607|1462126|5445|27611|7|78|1019|28|6.73|13.12|0.65|17.10|18.20|188.44|367.36|0.03|17.10|1.10|1.13|-187.34| +2451799|45692|17516|83607|1462126|5445|27611|7|150|1019|9|10.85|16.92|6.93|0.00|62.37|97.65|152.28|5.61|0.00|62.37|67.98|-35.28| +2451799|45692|11942|83607|1462126|5445|27611|7|283|1019|37|49.59|95.21|52.36|0.00|1937.32|1834.83|3522.77|0.00|0.00|1937.32|1937.32|102.49| +2451799|45692|1796|83607|1462126|5445|27611|7|287|1019|70|82.88|165.76|149.18|0.00|10442.60|5801.60|11603.20|104.42|0.00|10442.60|10547.02|4641.00| +2451799|45692|15470|83607|1462126|5445|27611|7|217|1019|10|5.03|9.20|8.37|0.00|83.70|50.30|92.00|7.53|0.00|83.70|91.23|33.40| +2451799|45692|4982|83607|1462126|5445|27611|7|278|1019|17|9.70|19.20|11.90|0.00|202.30|164.90|326.40|0.00|0.00|202.30|202.30|37.40| +2451907|41803|10907|50313|1012897|3034|39171|10|102|1020|71|60.22|95.14|46.61|0.00|3309.31|4275.62|6754.94|33.09|0.00|3309.31|3342.40|-966.31| +2451907|41803|475|50313|1012897|3034|39171|10|155|1020|33|7.86|9.98|2.89|0.00|95.37|259.38|329.34|6.67|0.00|95.37|102.04|-164.01| +2451907|41803|12475|50313|1012897|3034|39171|10|151|1020|23|81.38|113.11|99.53|0.00|2289.19|1871.74|2601.53|45.78|0.00|2289.19|2334.97|417.45| +2451907|41803|16556|50313|1012897|3034|39171|10|166|1020|98|98.59|110.42|106.00|8725.92|10388.00|9661.82|10821.16|33.24|8725.92|1662.08|1695.32|-7999.74| +|41803|12919|||3034|39171|10|167|1020|10|90.35||139.77|0.00|||1725.60|55.90|0.00|1397.70|1453.60|494.20| +2451907|41803|13735|50313|1012897|3034|39171|10|137|1020|40|99.08|198.16|95.11|0.00|3804.40|3963.20|7926.40|38.04|0.00|3804.40|3842.44|-158.80| +2451907|41803|2801|50313|1012897|3034|39171|10|232|1020|78|1.99|2.00|1.34|95.11|104.52|155.22|156.00|0.56|95.11|9.41|9.97|-145.81| +2451907|41803|14249|50313|1012897|3034|39171|10|154|1020|11|8.49|8.49|5.26|0.00|57.86|93.39|93.39|5.20|0.00|57.86|63.06|-35.53| +|41803|13505|50313|1012897|3034|39171||25|1020|68|38.25|63.11||0.00||2601.00|4291.48|51.48|0.00||909.64|-1742.84| +2451907|41803|17855|50313|1012897|3034|39171|10|139|1020|51|81.44|120.53|107.27|3063.63|5470.77|4153.44|6147.03|48.14|3063.63|2407.14|2455.28|-1746.30| +2451907|41803|3017|50313|1012897|3034|39171|10|246|1020|27|13.24|16.94|11.01|0.00|297.27|357.48|457.38|23.78|0.00|297.27|321.05|-60.21| +2451907|41803|4015|50313|1012897|3034|39171|10|157|1020|33|89.89|167.19|115.36|2893.22|3806.88|2966.37|5517.27|73.09|2893.22|913.66|986.75|-2052.71| +2451907|41803|4549|50313|1012897|3034|39171|10|8|1020|98|2.96|3.99|0.99|0.00|97.02|290.08|391.02|7.76|0.00|97.02|104.78|-193.06| +2451063|51671|14278|42994|1400769|6097|38625|7|179|1021|60|62.96|74.92|35.21|0.00|2112.60|3777.60|4495.20|0.00|0.00|2112.60|2112.60|-1665.00| +2451063|51671|3470|42994|1400769|6097|38625|7|226|1021|4|51.98|53.01|48.76|0.00|195.04|207.92|212.04|13.65|0.00|195.04|208.69|-12.88| +2451063|51671|3392|42994|1400769|6097|38625|7|90|1021|69|69.20|130.09|106.67|0.00|7360.23|4774.80|8976.21|294.40|0.00|7360.23|7654.63|2585.43| +2451063|51671|9056|42994|1400769|6097|38625|7|292|1021|36|58.85|116.52|75.73|0.00|2726.28|2118.60|4194.72|163.57|0.00|2726.28|2889.85|607.68| +2451063|51671|10994|42994|1400769|6097|38625|7|174|1021|73|82.02|136.15|28.59|897.44|2087.07|5987.46|9938.95|95.17|897.44|1189.63|1284.80|-4797.83| +2451063|51671|6997|42994|1400769|6097|38625|7|104|1021|78|32.19|60.51|47.19|0.00|3680.82|2510.82|4719.78|110.42|0.00|3680.82|3791.24|1170.00| +2451063|51671|1918|42994|1400769|6097|38625|7|75|1021|33|68.08|87.82|39.51|0.00|1303.83|2246.64|2898.06|78.22|0.00|1303.83|1382.05|-942.81| +2451063|51671|1538|42994|1400769|6097|38625|7|102|1021|88|16.08|23.79|23.07|0.00|2030.16|1415.04|2093.52|162.41|0.00|2030.16|2192.57|615.12| +2451063|51671|58|42994|1400769|6097|38625|7|86|1021|92|68.82|98.41|70.85|0.00|6518.20|6331.44|9053.72|325.91|0.00|6518.20|6844.11|186.76| +2451063|51671|12524|42994|1400769|6097|38625|7|100|1021|66|35.31|69.20|3.46|100.47|228.36|2330.46|4567.20|6.39|100.47|127.89|134.28|-2202.57| +2451063|51671|296|42994|1400769|6097|38625|7|283|1021|82|59.10|71.51|29.31|0.00|2403.42|4846.20|5863.82|192.27|0.00|2403.42|2595.69|-2442.78| +2451063|51671|7174|42994|1400769|6097|38625|7|210|1021|42|33.87|64.69|16.81|0.00|706.02|1422.54|2716.98|14.12|0.00|706.02|720.14|-716.52| +2451063|51671|17714|42994|1400769|6097|38625|7|232|1021|46|36.38|61.11|21.99|0.00|1011.54|1673.48|2811.06|0.00|0.00|1011.54|1011.54|-661.94| +2451063|51671|12338|42994|1400769|6097|38625|7|262|1021|15|28.83|57.08|17.12|0.00|256.80|432.45|856.20|15.40|0.00|256.80|272.20|-175.65| +2451063|51671|10306|42994|1400769|6097|38625|7|186|1021|91|77.27|88.86|67.53|184.35|6145.23|7031.57|8086.26|417.26|184.35|5960.88|6378.14|-1070.69| +2451063|51671|4030|42994|1400769|6097|38625|7|190|1021|96|7.31|9.28|1.57|66.31|150.72|701.76|890.88|4.22|66.31|84.41|88.63|-617.35| +2452636|64585|14040|59821|1640448|3475|17349|2|188|1022|66|10.38|14.63|2.34|0.00|154.44|685.08|965.58|3.08|0.00|154.44|157.52|-530.64| +2452636|64585|9414|59821|1640448|3475|17349|2|199|1022|66|21.43|21.43|10.07|245.90|664.62|1414.38|1414.38|4.18|245.90|418.72|422.90|-995.66| +2452636|64585|1159|59821|1640448|3475|17349|2|245|1022|3|2.29|3.27|1.56|0.74|4.68|6.87|9.81|0.00|0.74|3.94|3.94|-2.93| +||17862||1640448|3475|17349|||1022|17||70.68||||993.14||10.09||1009.29|1019.38|| +2452636|64585|2755|59821|1640448|3475|17349|2|88|1022|38|50.09|56.60|31.69|0.00|1204.22|1903.42|2150.80|72.25|0.00|1204.22|1276.47|-699.20| +2452636|64585|15288|59821|1640448|3475|17349|2|54|1022|79|21.47|33.06|13.88|0.00|1096.52|1696.13|2611.74|98.68|0.00|1096.52|1195.20|-599.61| +2452636|64585|3309|59821|1640448|3475|17349|2|125|1022|56|7.09|9.35|9.25|274.54|518.00|397.04|523.60|17.04|274.54|243.46|260.50|-153.58| +2452636|64585|7914|59821|1640448|3475|17349|2|194|1022|6|1.22|1.93|1.35|0.00|8.10|7.32|11.58|0.00|0.00|8.10|8.10|0.78| +2452636|64585|13734|59821|1640448|3475|17349|2|201|1022|49|64.91|112.94|108.42|0.00|5312.58|3180.59|5534.06|318.75|0.00|5312.58|5631.33|2131.99| +2452636|64585|9499|59821|1640448|3475|17349|2|158|1022|99|37.63|55.31|41.48|0.00|4106.52|3725.37|5475.69|287.45|0.00|4106.52|4393.97|381.15| +2452636|64585|1731|59821|1640448|3475|17349|2|232|1022|58|29.03|43.25|8.65|0.00|501.70|1683.74|2508.50|35.11|0.00|501.70|536.81|-1182.04| +2452636|64585|13039|59821|1640448|3475|17349|2|294|1022|76|32.86|61.77|31.50|0.00|2394.00|2497.36|4694.52|0.00|0.00|2394.00|2394.00|-103.36| +2451867|65680|2197|39393|1757871|3902|30333|10|271|1023|53|83.18|101.47|52.76|0.00|2796.28|4408.54|5377.91|0.00|0.00|2796.28|2796.28|-1612.26| +2451867|65680|5990|39393|1757871|3902|30333|10|66|1023|90|6.42|6.74|0.00|0.00|0.00|577.80|606.60|0.00|0.00|0.00|0.00|-577.80| +2451867|65680|10820|39393|1757871|3902|30333|10|194|1023|32|3.95|6.00|0.30|0.00|9.60|126.40|192.00|0.86|0.00|9.60|10.46|-116.80| +2451867|65680|16243|39393|1757871|3902|30333|10|259|1023|16|52.86|84.57|11.83|0.00|189.28|845.76|1353.12|5.67|0.00|189.28|194.95|-656.48| +2451867|65680|2915|39393|1757871|3902|30333|10|232|1023|21|67.15|104.75|86.94|0.00|1825.74|1410.15|2199.75|127.80|0.00|1825.74|1953.54|415.59| +2451867|65680|17575|39393|1757871|3902|30333|10|222|1023|85|2.05|3.75|0.33|0.00|28.05|174.25|318.75|1.68|0.00|28.05|29.73|-146.20| +2451867|65680|6314|39393|1757871|3902|30333|10|241|1023|54|10.08|13.00|2.08|11.23|112.32|544.32|702.00|6.06|11.23|101.09|107.15|-443.23| +2451867|65680|5153|39393|1757871|3902|30333|10|266|1023|24|43.21|46.23|37.44|0.00|898.56|1037.04|1109.52|8.98|0.00|898.56|907.54|-138.48| +2451867|65680|16610|39393|1757871|3902|30333|10|253|1023|80|10.16|12.49|3.74|0.00|299.20|812.80|999.20|17.95|0.00|299.20|317.15|-513.60| +2451867|65680|13937|39393|1757871|3902|30333|10|290|1023|42|94.46|128.46|33.39|0.00|1402.38|3967.32|5395.32|42.07|0.00|1402.38|1444.45|-2564.94| +2451867|65680|16205|39393|1757871|3902|30333|10|198|1023|61|55.05|89.18|34.78|0.00|2121.58|3358.05|5439.98|169.72|0.00|2121.58|2291.30|-1236.47| +2451867|65680|16135|39393|1757871|3902|30333|10|43|1023|71|24.68|47.63|22.38|0.00|1588.98|1752.28|3381.73|15.88|0.00|1588.98|1604.86|-163.30| +2451867|65680|1897|39393|1757871|3902|30333|10|58|1023|7|71.55|138.80|55.52|0.00|388.64|500.85|971.60|23.31|0.00|388.64|411.95|-112.21| +2451867|65680|5408|39393|1757871|3902|30333|10|158|1023|60|22.69|34.48|22.41|0.00|1344.60|1361.40|2068.80|107.56|0.00|1344.60|1452.16|-16.80| +2451867|65680|7613|39393|1757871|3902|30333|10|96|1023|64|9.77|12.89|0.25|0.00|16.00|625.28|824.96|1.44|0.00|16.00|17.44|-609.28| +2451530|47712|17581|55341|1297316|891|66|4|189|1024|17|28.60|56.62|50.95|0.00|866.15|486.20|962.54|25.98|0.00|866.15|892.13|379.95| +2451530|47712|3434|55341|1297316|891|66|4|42|1024|56|13.47|21.28|15.53|0.00|869.68|754.32|1191.68|17.39|0.00|869.68|887.07|115.36| +2451530|47712|16759|55341|1297316|891|66|4|265|1024|94|49.23|49.72|37.29|0.00|3505.26|4627.62|4673.68|210.31|0.00|3505.26|3715.57|-1122.36| +2451530|47712|15655|55341|1297316|891|66|4|189|1024|43|99.32|150.96|52.83|2180.82|2271.69|4270.76|6491.28|4.54|2180.82|90.87|95.41|-4179.89| +2451530|47712|11162|55341|1297316|891|66|4|9|1024|95|25.91|48.71|4.87|0.00|462.65|2461.45|4627.45|4.62|0.00|462.65|467.27|-1998.80| +2451530|47712|12472|55341|1297316|891|66|4|33|1024|49|74.38|122.72|29.45|995.70|1443.05|3644.62|6013.28|31.31|995.70|447.35|478.66|-3197.27| +2451530|47712|15184|55341|1297316|891|66|4|289|1024|69|72.58|110.32|95.97|0.00|6621.93|5008.02|7612.08|397.31|0.00|6621.93|7019.24|1613.91| +2451530|47712|364|55341|1297316|891|66|4|169|1024|76|48.00|61.92|40.24|1315.04|3058.24|3648.00|4705.92|122.02|1315.04|1743.20|1865.22|-1904.80| +2451530|47712|11935|55341|1297316|891|66|4|138|1024|90|9.70|12.90|1.03|0.00|92.70|873.00|1161.00|4.63|0.00|92.70|97.33|-780.30| +2451131|75259|4885|81542|959854|2519|49231|7|91|1025|43|41.32|52.06|40.08|0.00|1723.44|1776.76|2238.58|17.23|0.00|1723.44|1740.67|-53.32| +2451131|75259|11557|81542|959854|2519|49231|7|63|1025|84|76.58|129.42|128.12|3121.00|10762.08|6432.72|10871.28|534.87|3121.00|7641.08|8175.95|1208.36| +2451131|75259|15343|81542|959854|2519|49231|7|103|1025|9|78.42|113.70|87.54|0.00|787.86|705.78|1023.30|7.87|0.00|787.86|795.73|82.08| +2451131|75259|15530|81542|959854|2519|49231|7|17|1025|43|34.73|46.53|35.82|0.00|1540.26|1493.39|2000.79|123.22|0.00|1540.26|1663.48|46.87| +2451131|75259|6634|81542|959854|2519|49231|7|220|1025|67|4.83|7.53|3.91|0.00|261.97|323.61|504.51|15.71|0.00|261.97|277.68|-61.64| +2451131|75259|10381|81542|959854|2519|49231|7|221|1025|31|95.47|114.56|89.35|0.00|2769.85|2959.57|3551.36|166.19|0.00|2769.85|2936.04|-189.72| +2451131|75259|10516|81542|959854|2519|49231|7|37|1025|24|6.76|12.91|5.93|0.00|142.32|162.24|309.84|7.11|0.00|142.32|149.43|-19.92| +||12199|81542||2519|49231||42|1025|27||||||1379.97||10.59|||187.17|-1203.39| +2451131|75259|12406|81542|959854|2519|49231|7|238|1025|63|24.10|33.49|9.71|0.00|611.73|1518.30|2109.87|12.23|0.00|611.73|623.96|-906.57| +2451131|75259|8456|81542|959854|2519|49231|7|55|1025|19|73.39|124.02|54.56|51.83|1036.64|1394.41|2356.38|59.08|51.83|984.81|1043.89|-409.60| +2451131|75259|16150|81542|959854|2519|49231|7|294|1025|68|87.77|122.87|11.05|0.00|751.40|5968.36|8355.16|15.02|0.00|751.40|766.42|-5216.96| +2451131|75259|15733|81542|959854|2519|49231|7|116|1025|76|62.69|94.66|81.40|247.45|6186.40|4764.44|7194.16|356.33|247.45|5938.95|6295.28|1174.51| +2451131|75259|6403|81542|959854|2519|49231|7|252|1025|6|28.36|37.15|35.29|129.16|211.74|170.16|222.90|4.95|129.16|82.58|87.53|-87.58| +2451867|49359|235|81071|1210834|1044|36854|10|250|1026|45|24.50|41.16|17.28|0.00|777.60|1102.50|1852.20|31.10|0.00|777.60|808.70|-324.90| +2451867|49359|397|81071|1210834|1044|36854|10|289|1026|84|16.03|24.36|5.60|0.00|470.40|1346.52|2046.24|28.22|0.00|470.40|498.62|-876.12| +2451867|49359|7099|81071|1210834|1044|36854|10|28|1026|36|99.51|124.38|119.40|0.00|4298.40|3582.36|4477.68|343.87|0.00|4298.40|4642.27|716.04| +2451867|49359|8945|81071|1210834|1044|36854|10|64|1026|15|65.32|118.22|106.39|0.00|1595.85|979.80|1773.30|15.95|0.00|1595.85|1611.80|616.05| +2451867|49359|10568|81071|1210834|1044|36854|10|99|1026|76|10.54|13.59|2.85|0.00|216.60|801.04|1032.84|15.16|0.00|216.60|231.76|-584.44| +2451867|49359|17528|81071|1210834|1044|36854|10|81|1026|64|11.40|17.89|0.53|23.06|33.92|729.60|1144.96|0.97|23.06|10.86|11.83|-718.74| +2451867|49359|4847|81071|1210834|1044|36854|10|8|1026|65|91.15|121.22|109.09|4254.51|7090.85|5924.75|7879.30|28.36|4254.51|2836.34|2864.70|-3088.41| +2451867|49359|491|81071|1210834|1044|36854|10|204|1026|94|63.85|91.30|80.34|0.00|7551.96|6001.90|8582.20|151.03|0.00|7551.96|7702.99|1550.06| +2451867|49359|8015|81071|1210834|1044|36854|10|216|1026|100|64.28|82.92|48.09|0.00|4809.00|6428.00|8292.00|336.63|0.00|4809.00|5145.63|-1619.00| +||14395||411056|||||1027|||105.49|73.84|||3577.17|6223.91|||4356.56||| +2451076|45506|5900|70689|411056|4223|543|7|42|1027|22|42.64|83.14|5.81|0.00|127.82|938.08|1829.08|11.50|0.00|127.82|139.32|-810.26| +2451076|45506|8120|70689|411056|4223|543|7|208|1027|2|16.50|24.58|9.83|12.58|19.66|33.00|49.16|0.63|12.58|7.08|7.71|-25.92| +2451076|45506|5804|70689|411056|4223|543|7|288|1027|84|55.41|68.70|13.05|0.00|1096.20|4654.44|5770.80|43.84|0.00|1096.20|1140.04|-3558.24| +2451076|45506|15374|70689|411056|4223|543|7|135|1027|77|76.34|105.34|83.21|0.00|6407.17|5878.18|8111.18|512.57|0.00|6407.17|6919.74|528.99| +2451076|45506|11752|70689|411056|4223|543|7|212|1027|45|97.33|175.19|61.31|0.00|2758.95|4379.85|7883.55|110.35|0.00|2758.95|2869.30|-1620.90| +2451076|45506|12505|70689|411056|4223|543|7|36|1027|81|71.00|120.70|114.66|7337.09|9287.46|5751.00|9776.70|78.01|7337.09|1950.37|2028.38|-3800.63| +||5090||411056||543||28|1027|||23.28||||||0.28|||29.27|| +2452574|53633|5599|52162|15333|104|11275|8|287|1028|59|68.75|130.62|118.86|0.00|7012.74|4056.25|7706.58|210.38|0.00|7012.74|7223.12|2956.49| +2452574|53633|17749|52162|15333|104|11275|8|150|1028|10|85.28|159.47|36.67|0.00|366.70|852.80|1594.70|29.33|0.00|366.70|396.03|-486.10| +2452574|53633|10830|52162|15333|104|11275|8|10|1028|12|65.78|127.61|24.24|0.00|290.88|789.36|1531.32|17.45|0.00|290.88|308.33|-498.48| +2452574|53633|1237|52162|15333|104|11275|8|93|1028|93|22.06|33.31|10.32|134.36|959.76|2051.58|3097.83|16.50|134.36|825.40|841.90|-1226.18| +2452574|53633|9007|52162|15333|104|11275|8|264|1028|84|22.49|22.71|8.40|0.00|705.60|1889.16|1907.64|0.00|0.00|705.60|705.60|-1183.56| +2452574|53633|14287|52162||||8||1028||29.62||24.73|||533.16|||||190.52|-355.10| +2452574|53633|14742|52162|15333|104|11275|8|214|1028|59|69.26|94.88|52.18|0.00|3078.62|4086.34|5597.92|215.50|0.00|3078.62|3294.12|-1007.72| +2452574|53633|3421|52162|15333|104|11275|8|141|1028|80|7.98|13.72|10.29|181.10|823.20|638.40|1097.60|12.84|181.10|642.10|654.94|3.70| +2452574|53633|16603|52162|15333|104|11275|8|122|1028|76|34.45|55.80|49.10|0.00|3731.60|2618.20|4240.80|0.00|0.00|3731.60|3731.60|1113.40| +2452574|53633|2557|52162|15333|104|11275|8|222|1028|21|55.99|101.90|75.40|0.00|1583.40|1175.79|2139.90|0.00|0.00|1583.40|1583.40|407.61| +2452574|53633|3295|52162|15333|104|11275|8|51|1028|48|86.18|122.37|24.47|0.00|1174.56|4136.64|5873.76|35.23|0.00|1174.56|1209.79|-2962.08| +2452574|53633|12918|52162|15333|104|11275|8|111|1028|89|13.03|20.32|0.00|0.00|0.00|1159.67|1808.48|0.00|0.00|0.00|0.00|-1159.67| +2452574|53633|3805|52162|15333|104|11275|8|114|1028|26|23.41|25.75|22.40|0.00|582.40|608.66|669.50|40.76|0.00|582.40|623.16|-26.26| +2452574|53633|12663|52162|15333|104|11275|8|180|1028|98|33.72|39.45|36.68|0.00|3594.64|3304.56|3866.10|323.51|0.00|3594.64|3918.15|290.08| +2451123|51519|16171|80369|670309|249|24589|1|104|1029|100|89.01|109.48|49.26|0.00|4926.00|8901.00|10948.00|443.34|0.00|4926.00|5369.34|-3975.00| +2451123|51519|9982|80369|670309|249|24589|1|240|1029|6|62.41|87.99|22.87|0.00|137.22|374.46|527.94|1.37|0.00|137.22|138.59|-237.24| +2451123|51519|6361|80369|670309|249|24589|1|100|1029|38|81.26|147.08|123.54|0.00|4694.52|3087.88|5589.04|140.83|0.00|4694.52|4835.35|1606.64| +2451123|51519|5960|80369|670309|249|24589|1|145|1029|94|38.63|69.14|44.94|0.00|4224.36|3631.22|6499.16|253.46|0.00|4224.36|4477.82|593.14| +2451123|51519|12325|80369|670309|249|24589|1|9|1029|49|10.31|16.49|1.48|0.00|72.52|505.19|808.01|4.35|0.00|72.52|76.87|-432.67| +2451123|51519|7555|80369|670309|249|24589|1|177|1029|43|53.17|95.17|25.69|828.50|1104.67|2286.31|4092.31|24.85|828.50|276.17|301.02|-2010.14| +2451123|51519|1364|80369|670309|249|24589|1|66|1029|49|12.35|15.93|5.09|0.00|249.41|605.15|780.57|17.45|0.00|249.41|266.86|-355.74| +2451123|51519|15007|80369|670309|249|24589|1|42|1029|51|80.54|161.08|33.82|569.19|1724.82|4107.54|8215.08|34.66|569.19|1155.63|1190.29|-2951.91| +2451123|51519|15205|80369|670309|249|24589|1|118|1029|23|64.74|116.53|17.47|0.00|401.81|1489.02|2680.19|4.01|0.00|401.81|405.82|-1087.21| +2451123|51519|9382|80369|670309|249|24589|1|10|1029|99|72.92|138.54|9.69|47.96|959.31|7219.08|13715.46|63.79|47.96|911.35|975.14|-6307.73| +2451123|51519|3424|80369|670309|249|24589|1|193|1029|58|72.13|105.30|98.98|0.00|5740.84|4183.54|6107.40|401.85|0.00|5740.84|6142.69|1557.30| +2451123|51519|16570||||24589|||1029||54.86|58.70|4.10|60.68||1097.20|1174.00||60.68||21.74|-1075.88| +2452070|55044|9405|61553|1761153|4753|23553|8|146|1030|93|78.50|102.83|1.02|0.00|94.86|7300.50|9563.19|2.84|0.00|94.86|97.70|-7205.64| +2452070|55044|2919|61553|1761153|4753|23553|8|164|1030|20|39.62|66.56|6.65|0.00|133.00|792.40|1331.20|3.99|0.00|133.00|136.99|-659.40| +2452070|55044|16309|61553|1761153|4753|23553|8|206|1030|16|4.17|7.25|5.29|0.00|84.64|66.72|116.00|0.00|0.00|84.64|84.64|17.92| +2452070|55044|13429|61553|1761153|4753|23553|8|113|1030|14|38.36|45.64|15.51|0.00|217.14|537.04|638.96|19.54|0.00|217.14|236.68|-319.90| +2452070|55044|9103|61553|1761153|4753|23553|8|64|1030|77|13.25|17.22|7.74|0.00|595.98|1020.25|1325.94|0.00|0.00|595.98|595.98|-424.27| +2452070|55044|17093|61553|1761153|4753|23553|8|125|1030|7|63.88|89.43|50.08|161.25|350.56|447.16|626.01|0.00|161.25|189.31|189.31|-257.85| +2452070|55044|14597|61553|1761153|4753|23553|8|184|1030|61|78.13|107.03|100.60|0.00|6136.60|4765.93|6528.83|61.36|0.00|6136.60|6197.96|1370.67| +2452070|55044|4685|61553|1761153|4753|23553|8|155|1030|83|88.16|149.87|79.43|0.00|6592.69|7317.28|12439.21|197.78|0.00|6592.69|6790.47|-724.59| +2452070|55044|6669|61553|1761153|4753|23553|8|28|1030|99|63.85|95.13|72.29|0.00|7156.71|6321.15|9417.87|357.83|0.00|7156.71|7514.54|835.56| +2452070|55044|13877|61553|1761153|4753|23553|8|262|1030|34|44.03|53.71|42.96|0.00|1460.64|1497.02|1826.14|0.00|0.00|1460.64|1460.64|-36.38| +2451043|63797|5281|95497|368551|5|15059|8|177|1031|42|54.14|61.17|8.56|147.40|359.52|2273.88|2569.14|0.00|147.40|212.12|212.12|-2061.76| +2451043|63797|9784|95497|368551|5|15059|8|9|1031|46|4.48|7.79|3.73|0.00|171.58|206.08|358.34|1.71|0.00|171.58|173.29|-34.50| +2451043|63797|7988|95497|368551||15059|8||1031||65.62|120.74||0.00|3390.12|2362.32|4346.64|169.50|0.00|3390.12||| +2451043|63797|17870|95497|368551|5|15059|8|280|1031|48|39.52|60.86|14.60|56.06|700.80|1896.96|2921.28|45.13|56.06|644.74|689.87|-1252.22| +2451043|63797|6457|95497|368551|5|15059|8|278|1031|36|59.24|62.20|36.07|0.00|1298.52|2132.64|2239.20|64.92|0.00|1298.52|1363.44|-834.12| +2451043|63797|2074|95497|368551|5|15059|8|158|1031|88|91.05|142.94|28.58|0.00|2515.04|8012.40|12578.72|25.15|0.00|2515.04|2540.19|-5497.36| +2451043|63797|10838|95497|368551|5|15059|8|129|1031|18|41.72|42.97|5.15|0.00|92.70|750.96|773.46|0.00|0.00|92.70|92.70|-658.26| +2451043|63797|14410|95497|368551|5|15059|8|161|1031|76|28.16|52.09|44.79|0.00|3404.04|2140.16|3958.84|68.08|0.00|3404.04|3472.12|1263.88| +2451043|63797|17689|95497|368551|5|15059|8|117|1031|23|48.82|54.19|40.10|0.00|922.30|1122.86|1246.37|64.56|0.00|922.30|986.86|-200.56| +2451043|63797|11002|95497|368551|5|15059|8|167|1031|8|42.48|73.91|37.69|108.54|301.52|339.84|591.28|5.78|108.54|192.98|198.76|-146.86| +2451043|63797|4904|95497|368551|5|15059|8|202|1031|56|48.11|57.25|21.75|0.00|1218.00|2694.16|3206.00|60.90|0.00|1218.00|1278.90|-1476.16| +2451043|63797|4540|95497|368551|5|15059|8|294|1031|96|93.62|140.43|91.27|0.00|8761.92|8987.52|13481.28|613.33|0.00|8761.92|9375.25|-225.60| +2451043|63797|9050|95497|368551|5|15059|8|24|1031|57|64.59|125.30|103.99|3675.00|5927.43|3681.63|7142.10|67.57|3675.00|2252.43|2320.00|-1429.20| +2452611|47313|13542|98290|1630391|539|18640|4|264|1032|86|2.14|2.61|1.85|108.18|159.10|184.04|224.46|1.52|108.18|50.92|52.44|-133.12| +2452611|47313|10587|98290|1630391|539|18640|4|260|1032|70|99.46|198.92|137.25|768.60|9607.50|6962.20|13924.40|176.77|768.60|8838.90|9015.67|1876.70| +2452611|47313|2907|98290|1630391|539|18640|4|30|1032|41|25.45|38.93|17.12|0.00|701.92|1043.45|1596.13|35.09|0.00|701.92|737.01|-341.53| +2452611|47313|11577|98290|1630391|539|18640|4|13|1032|91|76.33|114.49|85.86|0.00|7813.26|6946.03|10418.59|390.66|0.00|7813.26|8203.92|867.23| +2452611|47313|4863|98290|1630391|539|18640|4|103|1032|19|55.23|98.86|66.23|0.00|1258.37|1049.37|1878.34|113.25|0.00|1258.37|1371.62|209.00| +2452611|47313|4129|98290|1630391|539|18640|4|82|1032|35|3.35|5.86|3.86|0.00|135.10|117.25|205.10|9.45|0.00|135.10|144.55|17.85| +2452611|47313|1482|98290|1630391|539|18640|4|154|1032|95|17.80|34.53|27.62|0.00|2623.90|1691.00|3280.35|131.19|0.00|2623.90|2755.09|932.90| +2452611|47313|5976|98290|1630391|539|18640|4|206|1032|53|15.04|18.95|3.41|0.00|180.73|797.12|1004.35|1.80|0.00|180.73|182.53|-616.39| +2452611|47313|14070|98290|1630391|539|18640|4|195|1032|83|7.87|11.25|3.60|0.00|298.80|653.21|933.75|20.91|0.00|298.80|319.71|-354.41| +2452611|47313|823|98290|1630391|539|18640|4|261|1032|96|40.61|41.01|22.55|0.00|2164.80|3898.56|3936.96|86.59|0.00|2164.80|2251.39|-1733.76| +2452611|47313|14658|98290|1630391|539|18640|4|102|1032|96|10.74|18.04|5.23|0.00|502.08|1031.04|1731.84|45.18|0.00|502.08|547.26|-528.96| +2452611|47313|10593|98290|1630391|539|18640|4|164|1032|92|50.09|77.63|17.85|0.00|1642.20|4608.28|7141.96|131.37|0.00|1642.20|1773.57|-2966.08| +2452481|46753|10723|63664|1455946|1381|16863|10|104|1033|53|15.23|21.47|11.37|0.00|602.61|807.19|1137.91|42.18|0.00|602.61|644.79|-204.58| +2452481|46753|4017|63664|1455946|1381|16863|10|15|1033|81|62.94|120.21|38.46|0.00|3115.26|5098.14|9737.01|124.61|0.00|3115.26|3239.87|-1982.88| +2452481|46753|3919|63664|1455946|1381|16863|10|92|1033|18|11.07|19.59|18.61|0.00|334.98|199.26|352.62|23.44|0.00|334.98|358.42|135.72| +2452481|46753|9483|63664|1455946|1381|16863|10|23|1033|29|1.43|2.45|0.63|0.00|18.27|41.47|71.05|0.54|0.00|18.27|18.81|-23.20| +2452481|46753|9435|63664|1455946|1381|16863|10|55|1033|89|41.49|56.84|55.70|0.00|4957.30|3692.61|5058.76|0.00|0.00|4957.30|4957.30|1264.69| +2452481|46753|11331|63664|1455946|1381|16863|10|61|1033|51|77.16|137.34|48.06|0.00|2451.06|3935.16|7004.34|220.59|0.00|2451.06|2671.65|-1484.10| +2452481|46753|17274|63664|1455946|1381|16863|10|242|1033|54|80.76|121.94|78.04|0.00|4214.16|4361.04|6584.76|252.84|0.00|4214.16|4467.00|-146.88| +2452481|46753|4779|63664|1455946|1381|16863|10|221|1033|15|91.99|130.62|69.22|0.00|1038.30|1379.85|1959.30|93.44|0.00|1038.30|1131.74|-341.55| +2452481|46753|11329|63664|1455946|1381|16863|10|245|1033|35|87.12|141.13|53.62|0.00|1876.70|3049.20|4939.55|150.13|0.00|1876.70|2026.83|-1172.50| +2451381|37431|7093|48938|1395676|2861|32046|8|223|1034|30|69.67|105.89|85.77|0.00|2573.10|2090.10|3176.70|154.38|0.00|2573.10|2727.48|483.00| +2451381|37431|16378|48938|1395676|2861|32046|8|18|1034|33|38.00|66.88|8.69|0.00|286.77|1254.00|2207.04|2.86|0.00|286.77|289.63|-967.23| +2451381|37431|2791|48938|1395676|2861|32046|8|249|1034|9|98.16|138.40|40.13|299.77|361.17|883.44|1245.60|1.84|299.77|61.40|63.24|-822.04| +2451381|37431|92|48938|1395676|2861|32046|8|40|1034|13|53.48|90.38|75.91|0.00|986.83|695.24|1174.94|29.60|0.00|986.83|1016.43|291.59| +2451381|37431|529|48938|1395676|2861|32046|8|190|1034|48|36.17|37.25|22.35|0.00|1072.80|1736.16|1788.00|96.55|0.00|1072.80|1169.35|-663.36| +2451381|37431|14674|48938|1395676|2861|32046|8|163|1034|57|82.35|152.34|114.25|0.00|6512.25|4693.95|8683.38|325.61|0.00|6512.25|6837.86|1818.30| +2451381|37431|8656|48938|1395676|2861|32046|8|76|1034|46|82.39|157.36|86.54|0.00|3980.84|3789.94|7238.56|119.42|0.00|3980.84|4100.26|190.90| +|37431|11911|||2861||||1034|34|98.17||119.02|0.00|4046.68|||364.20|0.00|4046.68||| +2452274|61407|11703|23820|1467173|2879|30585|4|14|1035|39|39.51|67.56|62.15|0.00|2423.85|1540.89|2634.84|121.19|0.00|2423.85|2545.04|882.96| +2452274|61407|13239|23820|1467173|2879|30585|4|61|1035|34|2.39|3.39|1.01|21.63|34.34|81.26|115.26|0.38|21.63|12.71|13.09|-68.55| +2452274|61407|9443|23820|1467173|2879|30585|4|286|1035|26|32.05|35.25|25.02|0.00|650.52|833.30|916.50|13.01|0.00|650.52|663.53|-182.78| +2452274|61407|8849|23820|1467173|2879|30585|4|180|1035|38|7.54|8.67|6.15|0.00|233.70|286.52|329.46|4.67|0.00|233.70|238.37|-52.82| +2452274|61407|4053|23820|1467173|2879|30585|4|77|1035|36|59.74|105.14|65.18|0.00|2346.48|2150.64|3785.04|93.85|0.00|2346.48|2440.33|195.84| +2452274|61407|15161|23820|1467173|2879|30585|4|138|1035|91|40.11|44.12|4.85|0.00|441.35|3650.01|4014.92|22.06|0.00|441.35|463.41|-3208.66| +2452274|61407|13135|23820|1467173|2879|30585|4|118|1035|40|34.53|40.40|13.33|0.00|533.20|1381.20|1616.00|31.99|0.00|533.20|565.19|-848.00| +2452274|61407|16835|23820|1467173|2879|30585|4|49|1035|67|44.79|84.20|9.26|0.00|620.42|3000.93|5641.40|24.81|0.00|620.42|645.23|-2380.51| +2452274|61407|13729||1467173|2879||||1035|52|||24.88|0.00|1293.76|2061.28|3318.64||0.00|||| +2452274||12637|||2879|30585|4|203|1035||||81.53|1898.01||1751.52|2574.72|1.76|1898.01||60.47|-1692.81| +2452274|61407|10131|23820|1467173|2879|30585|4|298|1035|84|29.62|45.91|21.57|1032.77|1811.88|2488.08|3856.44|54.53|1032.77|779.11|833.64|-1708.97| +2452274|61407|3681|23820|1467173|2879|30585|4|70|1035|81|55.86|94.40|39.64|0.00|3210.84|4524.66|7646.40|64.21|0.00|3210.84|3275.05|-1313.82| +2452276|45307|9783|62046|1083856|866|26276|1|177|1036|76|12.28|20.38|15.89|0.00|1207.64|933.28|1548.88|84.53|0.00|1207.64|1292.17|274.36| +2452276|45307|5125|62046|1083856|866|26276|1|152|1036|47|13.20|13.99|7.69|0.00|361.43|620.40|657.53|25.30|0.00|361.43|386.73|-258.97| +2452276|45307|1410|62046|1083856|866|26276|1|266|1036|100|34.25|39.38|29.53|0.00|2953.00|3425.00|3938.00|177.18|0.00|2953.00|3130.18|-472.00| +2452276|45307|6529|62046|1083856|866|26276|1|47|1036|42|12.14|19.66|14.74|0.00|619.08|509.88|825.72|0.00|0.00|619.08|619.08|109.20| +2452276|45307|15633|62046|1083856|866|26276|1|126|1036|70|29.16|41.69|14.17|0.00|991.90|2041.20|2918.30|39.67|0.00|991.90|1031.57|-1049.30| +2452276|45307|15552|62046|1083856|866|26276|1|199|1036|24|43.60|85.02|29.75|0.00|714.00|1046.40|2040.48|7.14|0.00|714.00|721.14|-332.40| +2452276|45307|9937|62046|1083856|866|26276|1|182|1036|12|22.29|26.74|2.13|0.00|25.56|267.48|320.88|0.00|0.00|25.56|25.56|-241.92| +2452276|45307|4897|62046|1083856|866|26276|1|177|1036|40|63.20|82.16|3.28|0.00|131.20|2528.00|3286.40|3.93|0.00|131.20|135.13|-2396.80| +2452276|45307|4135|62046|1083856|866|26276|1|39|1036|54|46.70|56.97|7.97|0.00|430.38|2521.80|3076.38|17.21|0.00|430.38|447.59|-2091.42| +2452276|45307|17187|62046|1083856|866|26276|1|278|1036|11|81.36|135.05|45.91|0.00|505.01|894.96|1485.55|30.30|0.00|505.01|535.31|-389.95| +2452276|45307|15553|62046|1083856|866|26276|1|179|1036|81|1.27|1.86|1.45|0.00|117.45|102.87|150.66|10.57|0.00|117.45|128.02|14.58| +2452276|45307|12369|62046|1083856|866|26276|1|247|1036|71|92.42|143.25|65.89|0.00|4678.19|6561.82|10170.75|0.00|0.00|4678.19|4678.19|-1883.63| +2450971|68243|5398|87360|1214677|2441|13005|1|142|1037|26|68.83|112.19|78.53|0.00|2041.78|1789.58|2916.94|20.41|0.00|2041.78|2062.19|252.20| +2450971|68243|17899|87360|1214677|2441|13005|1|203|1037|76|60.66|89.17|8.91|0.00|677.16|4610.16|6776.92|13.54|0.00|677.16|690.70|-3933.00| +2450971|68243|6344|87360|1214677|2441|13005|1|126|1037|37|43.11|85.78|17.15|0.00|634.55|1595.07|3173.86|19.03|0.00|634.55|653.58|-960.52| +2450971|68243|1921|87360|1214677|2441|13005|1|241|1037|82|1.70|2.66|0.31|0.00|25.42|139.40|218.12|2.28|0.00|25.42|27.70|-113.98| +2450971|68243|16234|87360|1214677|2441|13005|1|62|1037|21|9.83|18.77|18.01|0.00|378.21|206.43|394.17|0.00|0.00|378.21|378.21|171.78| +2450971|68243|11905|87360|1214677|2441|13005|1|296|1037|59|35.77|54.37|7.06|0.00|416.54|2110.43|3207.83|29.15|0.00|416.54|445.69|-1693.89| +2450971|68243|5215|87360|1214677|2441|13005|1|254|1037|100|3.70|4.25|2.38|0.00|238.00|370.00|425.00|16.66|0.00|238.00|254.66|-132.00| +2450971|68243|15010|87360|1214677|2441|13005|1|287|1037|93|81.58|161.52|142.13|0.00|13218.09|7586.94|15021.36|793.08|0.00|13218.09|14011.17|5631.15| +2450971|68243|9205|87360|1214677|2441|13005|1|103|1037|17|16.50|19.63|10.40|0.00|176.80|280.50|333.71|3.53|0.00|176.80|180.33|-103.70| +2450971|68243|8278|87360|1214677|2441|13005|1|298|1037|34|87.25|147.45|128.28|0.00|4361.52|2966.50|5013.30|305.30|0.00|4361.52|4666.82|1395.02| +2450971|68243|928|87360|1214677|2441|13005|1|145|1037|53|95.94|125.68|106.82|0.00|5661.46|5084.82|6661.04|56.61|0.00|5661.46|5718.07|576.64| +2450971|68243|16048|87360|1214677|2441|13005|1|144|1037|56|12.57|17.34|2.42|0.00|135.52|703.92|971.04|6.77|0.00|135.52|142.29|-568.40| +2450971|68243|14755|87360|1214677|2441|13005|1|73|1037|81|50.58|78.90|7.10|0.00|575.10|4096.98|6390.90|23.00|0.00|575.10|598.10|-3521.88| +2450971|68243|10477|87360|1214677|2441|13005|1|221|1037|41|33.07|58.86|8.82|0.00|361.62|1355.87|2413.26|10.84|0.00|361.62|372.46|-994.25| +2450971|68243|15889|87360|1214677|2441|13005|1|79|1037|80|52.73|104.40|58.46|0.00|4676.80|4218.40|8352.00|233.84|0.00|4676.80|4910.64|458.40| +2450971|68243|14014|87360|1214677|2441|13005|1|49|1037|62|62.34|97.25|85.58|0.00|5305.96|3865.08|6029.50|318.35|0.00|5305.96|5624.31|1440.88| +2451094|68031|4574|81304|1661467|4477|25440|2|245|1038|75|29.32|54.82|25.76|0.00|1932.00|2199.00|4111.50|96.60|0.00|1932.00|2028.60|-267.00| +2451094|68031|6352|81304|1661467|4477|25440|2|180|1038|47|12.39|16.10|8.05|0.00|378.35|582.33|756.70|18.91|0.00|378.35|397.26|-203.98| +2451094|68031|6298|81304|1661467|4477|25440|2|170|1038|97|14.82|21.04|10.94|0.00|1061.18|1437.54|2040.88|84.89|0.00|1061.18|1146.07|-376.36| +2451094|68031|10903|81304|1661467|4477|25440|2|268|1038|24|25.76|50.74|39.57|0.00|949.68|618.24|1217.76|66.47|0.00|949.68|1016.15|331.44| +2451094|68031|6784|81304|1661467|4477|25440|2|169|1038|70|60.47|67.72|0.67|0.00|46.90|4232.90|4740.40|0.00|0.00|46.90|46.90|-4186.00| +2451094|68031|2578|81304|1661467|4477|25440|2|168|1038|81|48.83|84.96|61.17|0.00|4954.77|3955.23|6881.76|198.19|0.00|4954.77|5152.96|999.54| +2451094|68031|10784|81304|1661467|4477|25440|2|168|1038|39|27.17|47.27|27.88|0.00|1087.32|1059.63|1843.53|21.74|0.00|1087.32|1109.06|27.69| +2451094|68031|11335|81304|1661467|4477|25440|2|68|1038|35|29.36|49.91|21.96|15.37|768.60|1027.60|1746.85|67.79|15.37|753.23|821.02|-274.37| +2451094|68031|6940|81304|1661467|4477|25440|2|143|1038|55|63.48|80.61|33.05|0.00|1817.75|3491.40|4433.55|163.59|0.00|1817.75|1981.34|-1673.65| +2451094|68031|4460|81304|1661467|4477|25440|2|147|1038|89|28.54|57.08|35.96|0.00|3200.44|2540.06|5080.12|32.00|0.00|3200.44|3232.44|660.38| +2451094|68031|15464|81304|1661467|4477|25440|2|152|1038|41|27.55|32.78|30.81|0.00|1263.21|1129.55|1343.98|50.52|0.00|1263.21|1313.73|133.66| +2451094|68031|782|81304|1661467|4477|25440|2|287|1038|6|49.96|49.96|28.47|80.28|170.82|299.76|299.76|4.52|80.28|90.54|95.06|-209.22| +2451094|68031|7058|81304|1661467|4477|25440|2|240|1038|44|7.12|8.54|0.59|0.00|25.96|313.28|375.76|2.07|0.00|25.96|28.03|-287.32| +2452438|58703|12877|82756|1121642|2128|14614|10|148|1039|33|40.95|49.95|22.97|0.00|758.01|1351.35|1648.35|68.22|0.00|758.01|826.23|-593.34| +2452438|58703|6357|82756|1121642|2128|14614|10|152|1039|3|97.20|151.63|89.46|0.00|268.38|291.60|454.89|24.15|0.00|268.38|292.53|-23.22| +2452438|58703|7999|82756|1121642|2128|14614|10|295|1039|53|18.57|20.98|2.09|0.00|110.77|984.21|1111.94|7.75|0.00|110.77|118.52|-873.44| +2452438|58703|12433|82756|1121642|2128|14614|10|117|1039|78|14.14|22.90|13.28|0.00|1035.84|1102.92|1786.20|72.50|0.00|1035.84|1108.34|-67.08| +2452438|58703|5406|82756|1121642|2128|14614|10|220|1039|76|69.99|86.78|68.55|0.00|5209.80|5319.24|6595.28|52.09|0.00|5209.80|5261.89|-109.44| +2452438|58703|6582|82756|1121642|2128|14614|10|102|1039|91|69.28|94.91|25.62|0.00|2331.42|6304.48|8636.81|209.82|0.00|2331.42|2541.24|-3973.06| +|58703|4111||1121642||14614|10|78|1039|7|83.29|121.60|||212.80|583.03||19.15||||| +2452438|58703|17419|82756|1121642|2128|14614|10|192|1039|7|25.94|40.72|28.91|0.00|202.37|181.58|285.04|12.14|0.00|202.37|214.51|20.79| +2452438|58703|13201|82756|1121642|2128|14614|10|164|1039|12|84.27|132.30|19.84|0.00|238.08|1011.24|1587.60|2.38|0.00|238.08|240.46|-773.16| +2452438|58703|727|82756|1121642|2128|14614|10|151|1039|65|47.29|83.23|2.49|33.98|161.85|3073.85|5409.95|7.67|33.98|127.87|135.54|-2945.98| +2452438|58703|11809|82756|1121642|2128|14614|10|284|1039|84|63.00|114.03|35.34|0.00|2968.56|5292.00|9578.52|89.05|0.00|2968.56|3057.61|-2323.44| +2452438|58703|2065|82756|1121642|2128|14614|10|83|1039|13|25.42|32.53|16.26|84.55|211.38|330.46|422.89|10.14|84.55|126.83|136.97|-203.63| +2452438|58703|8493|82756|1121642|2128|14614|10|136|1039|87|47.40|56.40|0.00|0.00|0.00|4123.80|4906.80|0.00|0.00|0.00|0.00|-4123.80| +2452438|58703|3663|82756|1121642|2128|14614|10|239|1039|99|53.17|75.50|64.17|0.00|6352.83|5263.83|7474.50|317.64|0.00|6352.83|6670.47|1089.00| +2452126|38683|615|71845|249556|4067|30178|4|131|1040|46|9.29|11.89|9.63|181.62|442.98|427.34|546.94|13.06|181.62|261.36|274.42|-165.98| +2452126|38683|17773|71845|249556|4067|30178|4|204|1040|24|85.77|107.21|28.94|0.00|694.56|2058.48|2573.04|27.78|0.00|694.56|722.34|-1363.92| +2452126|38683|15409|71845|249556|4067|30178|4|170|1040|68|19.79|37.40|4.86|0.00|330.48|1345.72|2543.20|0.00|0.00|330.48|330.48|-1015.24| +2452126|38683|11767|71845|249556|4067|30178|4|174|1040|22|66.46|101.01|2.02|0.00|44.44|1462.12|2222.22|0.88|0.00|44.44|45.32|-1417.68| +2452126|38683|16527|71845|249556|4067|30178|4|217|1040|86|60.29|119.37|60.87|0.00|5234.82|5184.94|10265.82|418.78|0.00|5234.82|5653.60|49.88| +2452126|38683|14505|71845|249556|4067|30178|4|87|1040|7|66.67|71.33|26.39|66.50|184.73|466.69|499.31|3.54|66.50|118.23|121.77|-348.46| +2452126|38683|10865|71845|249556|4067|30178|4|257|1040|94|46.39|72.36|60.05|0.00|5644.70|4360.66|6801.84|56.44|0.00|5644.70|5701.14|1284.04| +2452126|38683|11205|71845|249556|4067|30178|4|67|1040|90|17.35|34.52|26.58|0.00|2392.20|1561.50|3106.80|47.84|0.00|2392.20|2440.04|830.70| +2452126|38683|14247|71845|249556|4067|30178|4|290|1040|74|9.87|19.44|16.71|0.00|1236.54|730.38|1438.56|74.19|0.00|1236.54|1310.73|506.16| +2452126|38683|7475|71845|249556|4067|30178|4|89|1040|43|2.80|4.76|0.33|8.08|14.19|120.40|204.68|0.48|8.08|6.11|6.59|-114.29| +2452126|38683|17061|71845|249556|4067|30178|4|267|1040|30|87.79|136.07|87.08|0.00|2612.40|2633.70|4082.10|156.74|0.00|2612.40|2769.14|-21.30| +2452126|38683|14079|71845|249556|4067|30178|4|33|1040|88|6.47|9.64|9.44|0.00|830.72|569.36|848.32|8.30|0.00|830.72|839.02|261.36| +2452126|38683|1793|71845|249556|4067|30178|4|116|1040|4|26.11|51.17|39.40|52.00|157.60|104.44|204.68|2.11|52.00|105.60|107.71|1.16| +2452126|38683|16303|71845|249556|4067|30178|4|29|1040|4|36.46|42.65|31.56|0.00|126.24|145.84|170.60|8.83|0.00|126.24|135.07|-19.60| +2452126|38683|14713|71845|249556|4067|30178|4|21|1040|3|38.24|68.44|33.53|0.00|100.59|114.72|205.32|0.00|0.00|100.59|100.59|-14.13| +2450819|48084|15334|76474|709524|6574|13624|7|246|1041|82|38.58|66.74|61.40|0.00|5034.80|3163.56|5472.68|100.69|0.00|5034.80|5135.49|1871.24| +2450819|48084|12664|76474|709524|6574|13624|7|42|1041|23|11.11|18.44|11.98|60.61|275.54|255.53|424.12|12.89|60.61|214.93|227.82|-40.60| +2450819|48084|15769|76474|709524|6574|13624|7|65|1041|21|45.68|89.98|35.99|0.00|755.79|959.28|1889.58|37.78|0.00|755.79|793.57|-203.49| +2450819|48084|10771|76474|709524|6574|13624|7|250|1041|65|21.19|37.71|0.00|0.00|0.00|1377.35|2451.15|0.00|0.00|0.00|0.00|-1377.35| +||14791|||||||1041|25|80.30|85.11||0.00|1042.50|2007.50|||0.00|1042.50|1084.20|| +2450819|48084|7916|76474|709524|6574|13624|7|186|1041|77|6.24|9.79|3.42|0.00|263.34|480.48|753.83|0.00|0.00|263.34|263.34|-217.14| +2450819|48084|11086|76474|709524|6574|13624|7|198|1041|10|19.54|27.55|12.67|24.07|126.70|195.40|275.50|2.05|24.07|102.63|104.68|-92.77| +2450819|48084|1123|76474|709524|6574|13624|7|197|1041|35|42.52|76.53|61.98|0.00|2169.30|1488.20|2678.55|195.23|0.00|2169.30|2364.53|681.10| +2450819|48084|15394|76474|709524|6574|13624|7|208|1041|39|64.42|81.16|22.72|0.00|886.08|2512.38|3165.24|70.88|0.00|886.08|956.96|-1626.30| +2450819|48084|9394|76474|709524|6574|13624|7|69|1041|34|55.78|80.88|8.89|0.00|302.26|1896.52|2749.92|27.20|0.00|302.26|329.46|-1594.26| +2450819|48084|17894|76474|709524|6574|13624|7|81|1041|94|95.18|119.92|55.16|2229.56|5185.04|8946.92|11272.48|88.66|2229.56|2955.48|3044.14|-5991.44| +2450819|48084|728|76474|709524|6574|13624|7|63|1041|60|49.09|82.47|28.86|0.00|1731.60|2945.40|4948.20|121.21|0.00|1731.60|1852.81|-1213.80| +2450819|48084|11564|76474|709524|6574|13624|7|273|1041|19|95.56|140.47|43.54|0.00|827.26|1815.64|2668.93|49.63|0.00|827.26|876.89|-988.38| +2450819|48084|11188|76474|709524|6574|13624|7|201|1041|35|22.45|23.79|18.08|0.00|632.80|785.75|832.65|0.00|0.00|632.80|632.80|-152.95| +2450819|48084|9296|76474|709524|6574|13624|7|61|1041|6|10.95|10.95|1.31|0.39|7.86|65.70|65.70|0.07|0.39|7.47|7.54|-58.23| +2450819|48084|15679|76474|709524|6574|13624|7|17|1041|74|39.83|50.98|45.37|0.00|3357.38|2947.42|3772.52|302.16|0.00|3357.38|3659.54|409.96| +2451817|62088|13015|88012|651987|315|40690|8|81|1042|78|91.22|110.37|39.73|0.00|3098.94|7115.16|8608.86|61.97|0.00|3098.94|3160.91|-4016.22| +2451817|62088|11720|88012|651987|315|40690|8|206|1042|15|67.04|89.16|30.31|231.87|454.65|1005.60|1337.40|15.59|231.87|222.78|238.37|-782.82| +2451817|62088|9494|88012|651987|315|40690|8|255|1042|46|64.78|106.23|58.42|0.00|2687.32|2979.88|4886.58|107.49|0.00|2687.32|2794.81|-292.56| +2451817|62088|11689|88012|651987|315|40690|8|212|1042|12|87.32|156.30|21.88|0.00|262.56|1047.84|1875.60|7.87|0.00|262.56|270.43|-785.28| +2451817|62088|11894|88012|651987|315|40690|8|78|1042|35|3.43|5.35|5.35|101.11|187.25|120.05|187.25|4.30|101.11|86.14|90.44|-33.91| +2451817|62088|15038|88012|651987|315|40690|8|100|1042|81|91.27|120.47|33.73|1721.24|2732.13|7392.87|9758.07|90.98|1721.24|1010.89|1101.87|-6381.98| +2451817|62088|5720|88012|651987|315|40690|8|195|1042|52|46.09|72.36|23.87|0.00|1241.24|2396.68|3762.72|62.06|0.00|1241.24|1303.30|-1155.44| +2451817|62088|5384|88012|651987|315|40690|8|61|1042|42|87.88|109.85|46.13|1763.08|1937.46|3690.96|4613.70|6.97|1763.08|174.38|181.35|-3516.58| +2451817|62088|12851|88012|651987|315|40690|8|54|1042|63|94.80|121.34|8.49|0.00|534.87|5972.40|7644.42|0.00|0.00|534.87|534.87|-5437.53| +2451817|62088|6647|88012|651987|315|40690|8|45|1042|10|30.63|44.10|3.96|0.00|39.60|306.30|441.00|2.77|0.00|39.60|42.37|-266.70| +2451817|62088|5053|88012|651987|315|40690|8|196|1042|70|99.86|199.72|127.82|3936.85|8947.40|6990.20|13980.40|350.73|3936.85|5010.55|5361.28|-1979.65| +2451817|62088|7337|88012|651987|315|40690|8|95|1042|12|86.73|101.47|57.83|0.00|693.96|1040.76|1217.64|48.57|0.00|693.96|742.53|-346.80| +2451817|62088|1988|88012|651987|315|40690|8|43|1042|9|83.34|128.34|35.93|0.00|323.37|750.06|1155.06|12.93|0.00|323.37|336.30|-426.69| +2452271|66248|3411|79198|185190|5935|32727|8|140|1043|53|57.04|67.30|4.71|0.00|249.63|3023.12|3566.90|19.97|0.00|249.63|269.60|-2773.49| +2452271|66248|14653|79198|185190|5935|32727|8|107|1043|1|6.85|7.26|3.84|0.00|3.84|6.85|7.26|0.00|0.00|3.84|3.84|-3.01| +2452271|66248|1815|79198|185190|5935|32727|8|299|1043|3|64.11|92.95|20.44|0.00|61.32|192.33|278.85|0.00|0.00|61.32|61.32|-131.01| +2452271|66248|5789|79198|185190|5935|32727|8|137|1043|83|40.18|60.67|15.77|0.00|1308.91|3334.94|5035.61|117.80|0.00|1308.91|1426.71|-2026.03| +2452271|66248|6395|79198|185190|5935|32727|8|79|1043|22|85.64|112.18|80.76|0.00|1776.72|1884.08|2467.96|53.30|0.00|1776.72|1830.02|-107.36| +2452271|66248|501|79198|185190|5935|32727|8|38|1043|21|29.45|32.98|12.53|0.00|263.13|618.45|692.58|0.00|0.00|263.13|263.13|-355.32| +2452271|66248|10791|79198|185190|5935|32727|8|294|1043|95|5.69|8.81|0.88|0.00|83.60|540.55|836.95|7.52|0.00|83.60|91.12|-456.95| +2452271|66248|11727|79198|185190|5935|32727|8|195|1043|73|63.17|83.38|74.20|2329.13|5416.60|4611.41|6086.74|277.87|2329.13|3087.47|3365.34|-1523.94| +2452271|66248|12251|79198|185190|5935|32727|8|51|1043|22|58.08|72.01|51.84|0.00|1140.48|1277.76|1584.22|79.83|0.00|1140.48|1220.31|-137.28| +2452271|66248|7109|79198|185190|5935|32727|8|107|1043|90|56.59|85.45|61.52|0.00|5536.80|5093.10|7690.50|166.10|0.00|5536.80|5702.90|443.70| +2452271|66248|13229|79198|185190|5935|32727|8|285|1043|60|30.42|59.92|49.73|924.97|2983.80|1825.20|3595.20|41.17|924.97|2058.83|2100.00|233.63| +2452271|66248|14855|79198|185190|5935|32727|8|104|1043|29|50.88|54.95|43.96|369.70|1274.84|1475.52|1593.55|36.20|369.70|905.14|941.34|-570.38| +2452222|67001|13139|83173|1579055|3384|26866|4|181|1044|62|31.30|40.69|25.63|0.00|1589.06|1940.60|2522.78|0.00|0.00|1589.06|1589.06|-351.54| +2452222|67001|17041|83173|1579055|3384|26866|4|35|1044|85|3.31|5.32|3.24|179.01|275.40|281.35|452.20|0.00|179.01|96.39|96.39|-184.96| +2452222|67001|2573|83173|1579055|3384|26866|4|168|1044|33|83.44|127.66|98.29|32.43|3243.57|2753.52|4212.78|289.00|32.43|3211.14|3500.14|457.62| +2452222|67001|693|83173|1579055|3384|26866|4|44|1044|24|38.82|58.23|54.15|0.00|1299.60|931.68|1397.52|12.99|0.00|1299.60|1312.59|367.92| +2452222|67001|3539|83173|1579055|3384|26866|4|15|1044|61|30.31|32.12|22.16|1216.58|1351.76|1848.91|1959.32|1.35|1216.58|135.18|136.53|-1713.73| +2452222|67001|14177|83173|||26866|||1044|55|48.10|92.83||1745.67||2645.50|5105.65|0.00|1745.67|91.88||-2553.62| +2452222|67001|1701|83173|1579055|3384|26866|4|168|1044|10|63.33|63.33|62.06|0.00|620.60|633.30|633.30|0.00|0.00|620.60|620.60|-12.70| +2452222|67001|39|83173|1579055|3384|26866|4|18|1044|95|88.91|170.70|1.70|0.00|161.50|8446.45|16216.50|4.84|0.00|161.50|166.34|-8284.95| +2452222|67001|16123|83173|1579055|3384|26866|4|171|1044|87|90.73|106.15|40.33|736.82|3508.71|7893.51|9235.05|0.00|736.82|2771.89|2771.89|-5121.62| +2452222|67001|9429|83173|1579055|3384|26866|4|160|1044|53|7.90|9.32|5.68|0.00|301.04|418.70|493.96|15.05|0.00|301.04|316.09|-117.66| +2450970|31028|16606|75557|1874125|4693|||244|1045||30.06|53.50|32.63||||2835.50|||1729.39|1781.27|| +2450970|31028|11404|75557|1874125|4693|10061|7|15|1045|31|83.57|111.14|36.67|863.94|1136.77|2590.67|3445.34|21.82|863.94|272.83|294.65|-2317.84| +2450970|31028|12416|75557|1874125|4693|10061|7|51|1045|8|84.28|111.24|97.89|0.00|783.12|674.24|889.92|39.15|0.00|783.12|822.27|108.88| +2450970|31028|8041|75557|1874125|4693|10061|7|28|1045|75|35.15|38.66|25.12|715.92|1884.00|2636.25|2899.50|93.44|715.92|1168.08|1261.52|-1468.17| +2450970|31028|14764|75557|1874125|4693|10061|7|133|1045|22|11.55|20.32|14.42|34.89|317.24|254.10|447.04|5.64|34.89|282.35|287.99|28.25| +2450970|31028|6937|75557|1874125|4693|10061|7|13|1045|49|22.86|26.28|12.35|248.11|605.15|1120.14|1287.72|10.71|248.11|357.04|367.75|-763.10| +|31028|5902|||4693|10061|7|117|1045|||66.58||||3444.00|4993.50|||4893.00|5137.65|| +2450970|31028|15724|75557|1874125|4693|10061|7|14|1045|25|92.52|108.24|6.49|0.00|162.25|2313.00|2706.00|1.62|0.00|162.25|163.87|-2150.75| +2450970|31028|12769|75557|1874125|4693|10061|7|95|1045|19|55.08|76.56|74.26|0.00|1410.94|1046.52|1454.64|126.98|0.00|1410.94|1537.92|364.42| +2450970|31028|2000|75557|1874125|4693|10061|7|259|1045|80|27.13|42.32|24.54|0.00|1963.20|2170.40|3385.60|117.79|0.00|1963.20|2080.99|-207.20| +2450970|31028|1664|75557|1874125|4693|10061|7|105|1045|15|38.58|50.15|27.08|0.00|406.20|578.70|752.25|32.49|0.00|406.20|438.69|-172.50| +2450970||4490|75557||4693||7|292|1045|91|||136.57|||||745.67||||4574.57| +2450970|31028|16364|75557|1874125|4693|10061|7|151|1045|52|33.66|58.56|29.86|0.00|1552.72|1750.32|3045.12|108.69|0.00|1552.72|1661.41|-197.60| +2450970|31028|12314|75557|1874125|4693|10061|7|48|1045|41|20.30|34.30|17.49|0.00|717.09|832.30|1406.30|50.19|0.00|717.09|767.28|-115.21| +2450970|31028|8131|75557|1874125|4693|10061|7|260|1045|92|77.20|91.09|66.49|0.00|6117.08|7102.40|8380.28|0.00|0.00|6117.08|6117.08|-985.32| +2452545|62700|3835|98431|978437|3596|3202|1|258|1046|49|38.95|47.90|4.31|0.00|211.19|1908.55|2347.10|6.33|0.00|211.19|217.52|-1697.36| +2452545|62700|925|98431|978437|3596|3202|1|269|1046|44|31.08|45.68|17.81|0.00|783.64|1367.52|2009.92|47.01|0.00|783.64|830.65|-583.88| +2452545|62700|3699|98431|978437|3596|3202|1|96|1046|15|79.83|134.11|134.11|0.00|2011.65|1197.45|2011.65|100.58|0.00|2011.65|2112.23|814.20| +2452545|62700|16764|98431|978437|||1||1046|||44.06|33.92|||859.17|||||1402.25|| +2452545|62700|14199|98431|978437|3596|3202|1|134|1046|8|66.41|104.26|12.51|0.00|100.08|531.28|834.08|2.00|0.00|100.08|102.08|-431.20| +2452545|62700|15453|98431|978437|3596|3202|1|29|1046|60|65.56|85.88|66.98|0.00|4018.80|3933.60|5152.80|80.37|0.00|4018.80|4099.17|85.20| +2452545|62700|14775|98431|978437|3596|3202|1|25|1046|35|54.68|73.27|70.33|0.00|2461.55|1913.80|2564.45|24.61|0.00|2461.55|2486.16|547.75| +2452545|62700|1855|98431|978437|3596|3202|1|36|1046|20|46.11|69.62|65.44|0.00|1308.80|922.20|1392.40|0.00|0.00|1308.80|1308.80|386.60| +2452545|62700|5323|98431|978437|3596|3202|1|44|1046|10|84.32|84.32|19.39|0.00|193.90|843.20|843.20|7.75|0.00|193.90|201.65|-649.30| +2452545|62700|1615|98431|978437|3596|3202|1|134|1046|42|34.88|68.71|46.03|0.00|1933.26|1464.96|2885.82|115.99|0.00|1933.26|2049.25|468.30| +2452545|62700|11931|98431|978437|3596|3202|1|264|1046|35|33.88|37.26|19.37|0.00|677.95|1185.80|1304.10|6.77|0.00|677.95|684.72|-507.85| +2452561|35051|6537|42511|933832|3906|37627|4|231|1047|53|41.75|53.85|3.76|0.00|199.28|2212.75|2854.05|13.94|0.00|199.28|213.22|-2013.47| +2452561|35051|16893|42511|933832|3906|37627|4|99|1047|8|78.30|129.97|68.88|0.00|551.04|626.40|1039.76|5.51|0.00|551.04|556.55|-75.36| +|35051|14487||933832|||4||1047||48.56||27.09|0.00|948.15|1699.60|3161.20||0.00|||-751.45| +2452561|35051|8857|42511|933832|3906|37627|4|293|1047|75|93.28|138.98|137.59|0.00|10319.25|6996.00|10423.50|722.34|0.00|10319.25|11041.59|3323.25| +2452561|35051|13297|42511|933832|3906|37627|4|255|1047|88|41.54|44.44|37.77|0.00|3323.76|3655.52|3910.72|99.71|0.00|3323.76|3423.47|-331.76| +2452561|35051|7813|42511|933832|3906|37627|4|174|1047|20|96.18|118.30|87.54|0.00|1750.80|1923.60|2366.00|157.57|0.00|1750.80|1908.37|-172.80| +2452561|35051|16653|42511|933832|3906|37627|4|81|1047|15|21.44|40.73|24.03|25.23|360.45|321.60|610.95|0.00|25.23|335.22|335.22|13.62| +2452561|35051|6204|42511|933832|3906|37627|4|280|1047|50|57.40|110.20|68.32|0.00|3416.00|2870.00|5510.00|170.80|0.00|3416.00|3586.80|546.00| +2452561|35051|8550|42511|933832|3906|37627|4|201|1047|94|70.75|79.94|30.37|799.33|2854.78|6650.50|7514.36|184.99|799.33|2055.45|2240.44|-4595.05| +2452561|35051|3126|42511|933832|3906|37627|4|231|1047|58|92.69|171.47|126.88|0.00|7359.04|5376.02|9945.26|220.77|0.00|7359.04|7579.81|1983.02| +2452561|35051|16296|42511|933832|3906|37627|4|31|1047|50|66.28|67.60|33.80|0.00|1690.00|3314.00|3380.00|33.80|0.00|1690.00|1723.80|-1624.00| +2452561|35051|11994|42511|933832|3906|37627|4|115|1047|75|94.48|125.65|13.82|82.92|1036.50|7086.00|9423.75|47.67|82.92|953.58|1001.25|-6132.42| +2452561|35051|4116|42511|933832|3906|37627|4|216|1047|66|3.69|7.34|3.15|0.00|207.90|243.54|484.44|2.07|0.00|207.90|209.97|-35.64| +2452561|35051|9579|42511|933832|3906|37627|4|61|1047|28|85.45|164.91|84.10|0.00|2354.80|2392.60|4617.48|164.83|0.00|2354.80|2519.63|-37.80| +2452226|39721|16855|53282|1189385|3088|4791|7|263|1048|86|68.29|105.16|1.05|0.00|90.30|5872.94|9043.76|7.22|0.00|90.30|97.52|-5782.64| +2452226|39721|8641|53282|1189385|3088|4791|7|296|1048|44|72.69|122.11|54.94|0.00|2417.36|3198.36|5372.84|48.34|0.00|2417.36|2465.70|-781.00| +|39721|13053|53282|1189385||||130|1048|88||138.89|58.33||5133.04|8730.48||153.99|||5287.03|| +2452226|39721|6281|53282|1189385|3088|4791|7|271|1048|38|28.58|33.15|27.18|299.52|1032.84|1086.04|1259.70|0.00|299.52|733.32|733.32|-352.72| +2452226|39721|12453|53282|1189385|3088|4791|7|49|1048|53|21.26|28.06|12.62|0.00|668.86|1126.78|1487.18|60.19|0.00|668.86|729.05|-457.92| +2452226|39721|3669|53282|1189385|3088|4791|7|171|1048|99|55.88|87.17|36.61|0.00|3624.39|5532.12|8629.83|36.24|0.00|3624.39|3660.63|-1907.73| +2452226|39721|17481|53282|1189385|3088|4791|7|191|1048|26|1.42|2.17|1.60|0.00|41.60|36.92|56.42|2.91|0.00|41.60|44.51|4.68| +2452226|39721|14061|53282|1189385|3088|4791|7|218|1048|94|95.80|134.12|64.37|0.00|6050.78|9005.20|12607.28|0.00|0.00|6050.78|6050.78|-2954.42| +2452202|63992|3883|2000|1861980|7113|12693|4|221|1049|94|29.70|49.00|16.66|0.00|1566.04|2791.80|4606.00|0.00|0.00|1566.04|1566.04|-1225.76| +2452202|63992|1449|2000|1861980|7113|12693|4|185|1049|38|34.19|42.39|28.40|733.85|1079.20|1299.22|1610.82|6.90|733.85|345.35|352.25|-953.87| +2452202|63992|4061||1861980|7113||4|33|1049|||||0.00||1048.80|1457.70||0.00|1428.30|1471.14|379.50| +2452202|63992|11687|2000|1861980|7113|12693|4|62|1049|37|78.49|149.91|17.98|0.00|665.26|2904.13|5546.67|26.61|0.00|665.26|691.87|-2238.87| +||17499||1861980||12693|4||1049||69.33||88.31|||2703.87|3514.68||||3192.68|| +2452202|63992|871|2000|1861980|7113|12693|4|107|1049|17|80.52|115.14|74.84|0.00|1272.28|1368.84|1957.38|89.05|0.00|1272.28|1361.33|-96.56| +2452202|63992|10901|2000|1861980|7113|12693|4|262|1049|3|98.23|169.93|50.97|0.00|152.91|294.69|509.79|3.05|0.00|152.91|155.96|-141.78| +2452202|63992|4577|2000|1861980|7113|12693|4|198|1049|51|53.14|105.74|62.38|0.00|3181.38|2710.14|5392.74|0.00|0.00|3181.38|3181.38|471.24| +2452202|63992|17655|2000|1861980|7113|12693|4|2|1049|25|76.82|101.40|68.95|1344.52|1723.75|1920.50|2535.00|26.54|1344.52|379.23|405.77|-1541.27| +2452202|63992|4845|2000|1861980|7113|12693|4|136|1049|96|93.12|148.99|120.68|0.00|11585.28|8939.52|14303.04|347.55|0.00|11585.28|11932.83|2645.76| +2451625|73919|16478|62971|1820574|6530|12486|2|279|1050|8|21.51|27.74|22.74|149.17|181.92|172.08|221.92|2.29|149.17|32.75|35.04|-139.33| +2451625|73919|11947|62971|1820574|6530|12486|2|136|1050|78|68.49|78.76|71.67|0.00|5590.26|5342.22|6143.28|167.70|0.00|5590.26|5757.96|248.04| +2451625|73919|13895|62971|1820574|6530|12486|2|287|1050|13|94.30|173.51|150.95|0.00|1962.35|1225.90|2255.63|39.24|0.00|1962.35|2001.59|736.45| +2451625|73919|9794|62971|1820574||12486|2|116|1050|31||72.12|32.45||1005.95|1152.58|2235.72|70.41|||1076.36|-146.63| +2451625|73919|17791|62971|1820574|6530|12486|2|51|1050|77|21.76|38.29|21.82|151.21|1680.14|1675.52|2948.33|76.44|151.21|1528.93|1605.37|-146.59| +2451625|73919|7771|62971|1820574|6530|12486|2|82|1050|53|36.49|44.15|44.15|0.00|2339.95|1933.97|2339.95|46.79|0.00|2339.95|2386.74|405.98| +||4283||1820574|||||1050||37.91|||0.00|583.20||1296.36|11.66|0.00|583.20||-99.18| +2451625|73919|2660|62971|1820574|6530|12486|2|191|1050|95|77.48|113.89|77.44|0.00|7356.80|7360.60|10819.55|0.00|0.00|7356.80|7356.80|-3.80| +2451625|73919|8828|62971|1820574|6530|12486|2|89|1050|39|88.76|157.99|6.31|0.00|246.09|3461.64|6161.61|14.76|0.00|246.09|260.85|-3215.55| +2451625|73919|15026|62971|1820574|6530|12486|2|71|1050|8|79.07|98.83|80.05|0.00|640.40|632.56|790.64|25.61|0.00|640.40|666.01|7.84| +2451625|73919|9715|62971|1820574|6530|12486|2|113|1050|4|54.60|90.63|21.75|0.00|87.00|218.40|362.52|5.22|0.00|87.00|92.22|-131.40| +2451625|73919|2846|62971|1820574|6530|12486|2|194|1050|14|87.92|136.27|96.75|0.00|1354.50|1230.88|1907.78|81.27|0.00|1354.50|1435.77|123.62| +2451625|73919|242|62971|1820574|6530|12486|2|9|1050|75|26.82|53.37|42.16|0.00|3162.00|2011.50|4002.75|63.24|0.00|3162.00|3225.24|1150.50| +2451310|65534|8804|68527|320409|4738|41467|7|198|1051|15|62.42|82.39|43.66|170.27|654.90|936.30|1235.85|33.92|170.27|484.63|518.55|-451.67| +2451310|65534|15950|68527|320409|4738|41467|7|136|1051|94|53.96|77.16|44.75|0.00|4206.50|5072.24|7253.04|378.58|0.00|4206.50|4585.08|-865.74| +2451310|65534|11566|68527|320409|4738|41467|7|71|1051|35|28.30|35.94|12.93|0.00|452.55|990.50|1257.90|18.10|0.00|452.55|470.65|-537.95| +2451310|65534|5066|68527|320409|4738|41467|7|207|1051|84|18.43|19.35|8.32|0.00|698.88|1548.12|1625.40|27.95|0.00|698.88|726.83|-849.24| +2451310|65534|8554|68527|320409|4738|41467|7|233|1051|74|78.21|94.63|77.59|0.00|5741.66|5787.54|7002.62|0.00|0.00|5741.66|5741.66|-45.88| +2451310|65534|5150|68527|320409|4738|41467|7|194|1051|34|89.72|178.54|14.28|0.00|485.52|3050.48|6070.36|4.85|0.00|485.52|490.37|-2564.96| +2451310|65534|3020|68527|320409|4738|41467|7|215|1051|9|14.03|17.81|10.50|0.00|94.50|126.27|160.29|7.56|0.00|94.50|102.06|-31.77| +2451310|65534|5287|68527|320409|4738|41467|7|142|1051|25|76.61|136.36|27.27|0.00|681.75|1915.25|3409.00|40.90|0.00|681.75|722.65|-1233.50| +2452584|38994|9045|83557|1784450|6943|20989|10|135|1052|85|40.28|77.74|27.20|0.00|2312.00|3423.80|6607.90|161.84|0.00|2312.00|2473.84|-1111.80| +2452584|38994|14785|83557|1784450|6943|20989|10|179|1052|10|57.47|97.12|67.98|0.00|679.80|574.70|971.20|61.18|0.00|679.80|740.98|105.10| +2452584|38994|7437|83557|1784450|6943|20989|10|46|1052|91|79.04|135.94|4.07|0.00|370.37|7192.64|12370.54|14.81|0.00|370.37|385.18|-6822.27| +2452584|38994|5784|83557|1784450|6943|20989|10|123|1052|36|93.77|123.77|53.22|76.63|1915.92|3375.72|4455.72|73.57|76.63|1839.29|1912.86|-1536.43| +2452584|38994|4855|83557|1784450|6943|20989|10|160|1052|33|99.66|118.59|55.73|0.00|1839.09|3288.78|3913.47|55.17|0.00|1839.09|1894.26|-1449.69| +2452584|38994|6723|83557|1784450|6943|20989|10|220|1052|95|31.21|34.01|1.02|0.00|96.90|2964.95|3230.95|8.72|0.00|96.90|105.62|-2868.05| +2452584|38994|12775||1784450|6943|20989|10|142|1052||||29.79|0.00||4520.16|5107.68||0.00||1458.51|| +2452584|38994|13191|83557|1784450|6943|20989|10|73|1052|57|52.92|59.27|47.41|0.00|2702.37|3016.44|3378.39|81.07|0.00|2702.37|2783.44|-314.07| +||13740|83557||6943||10||1052||||70.81|3370.55||4826.64|7626.08|11.89|3370.55|||| +2452584|38994|6933|83557|1784450|6943|20989|10|24|1052|57|35.87|63.13|58.07|1158.49|3309.99|2044.59|3598.41|129.09|1158.49|2151.50|2280.59|106.91| +2452584|38994|13848|83557|1784450|6943|20989|10|244|1052|63|71.85|132.20|132.20|0.00|8328.60|4526.55|8328.60|416.43|0.00|8328.60|8745.03|3802.05| +2452584|38994|17028|83557|1784450|6943|20989|10|38|1052|98|6.02|7.64|2.90|28.42|284.20|589.96|748.72|15.34|28.42|255.78|271.12|-334.18| +2452584|38994|10848|83557|1784450|6943|20989|10|118|1052|20|81.17|138.80|26.37|0.00|527.40|1623.40|2776.00|26.37|0.00|527.40|553.77|-1096.00| +2452584|38994|6513|83557|1784450|6943|20989|10|253|1052|27|32.36|38.50|30.80|0.00|831.60|873.72|1039.50|33.26|0.00|831.60|864.86|-42.12| +2452584|38994|4765|83557|1784450|6943|20989|10|45|1052|15|91.07|164.83|42.85|0.00|642.75|1366.05|2472.45|51.42|0.00|642.75|694.17|-723.30| +2451082|40511|15142|7708|1072297|439|15428|2|142|1053|49|35.50|35.50|8.16|0.00|399.84|1739.50|1739.50|3.99|0.00|399.84|403.83|-1339.66| +2451082|40511|8626|7708|1072297|439|15428|2|197|1053|67|4.69|5.95|0.83|0.00|55.61|314.23|398.65|3.89|0.00|55.61|59.50|-258.62| +2451082|40511|5119|7708|1072297|439|15428|2|225|1053|24|72.33|75.22|0.75|0.00|18.00|1735.92|1805.28|1.62|0.00|18.00|19.62|-1717.92| +2451082|40511|13706|7708|1072297|439|15428|2|159|1053|88|13.22|17.31|14.88|0.00|1309.44|1163.36|1523.28|117.84|0.00|1309.44|1427.28|146.08| +2451082|40511|12692|7708|1072297|439|15428|2|189|1053|98|72.20|125.62|2.51|0.00|245.98|7075.60|12310.76|4.91|0.00|245.98|250.89|-6829.62| +2451082|40511|10840|7708|1072297|439|15428|2|93|1053|8|55.33|110.10|41.83|107.08|334.64|442.64|880.80|6.82|107.08|227.56|234.38|-215.08| +2451082|40511|15229|7708|1072297|439|15428|2|208|1053|17|35.57|60.82|3.04|0.00|51.68|604.69|1033.94|3.61|0.00|51.68|55.29|-553.01| +2451082|40511|3466|7708|1072297|439|15428|2|35|1053|19|40.01|65.21|44.99|0.00|854.81|760.19|1238.99|17.09|0.00|854.81|871.90|94.62| +2451082|40511|8638|7708|1072297|439|15428|2|272|1053|11|95.66|135.83|70.63|0.00|776.93|1052.26|1494.13|38.84|0.00|776.93|815.77|-275.33| +2451082|40511|16753|7708|1072297|439|15428|2|177|1053|75|5.78|7.74|0.23|3.79|17.25|433.50|580.50|0.40|3.79|13.46|13.86|-420.04| +2451082|40511|9962|7708|1072297|439|15428|2|43|1053|88|72.39|133.19|13.31|0.00|1171.28|6370.32|11720.72|81.98|0.00|1171.28|1253.26|-5199.04| +2451082|40511|2233|7708||439||2||1053|||109.72|97.65||||8338.72|||3116.99|3366.34|-1390.57| +2451082|40511|2881|7708|1072297|439|15428|2|233|1053|19|71.20|77.60|3.10|0.00|58.90|1352.80|1474.40|0.00|0.00|58.90|58.90|-1293.90| +2452511|69679|4543|88441|592052|4016|33227|8|283|1054|76|82.05|90.25|69.49|0.00|5281.24|6235.80|6859.00|369.68|0.00|5281.24|5650.92|-954.56| +2452511|69679|8466|88441|592052|4016|33227|8|52|1054|49|76.57|137.06|102.79|3424.96|5036.71|3751.93|6715.94|128.94|3424.96|1611.75|1740.69|-2140.18| +2452511|69679|13713|88441|592052|4016|33227|8|127|1054|63|73.77|87.78|27.21|0.00|1714.23|4647.51|5530.14|154.28|0.00|1714.23|1868.51|-2933.28| +2452511|69679|2497|88441|592052|4016|33227|8|8|1054|65|35.56|41.24|15.25|0.00|991.25|2311.40|2680.60|19.82|0.00|991.25|1011.07|-1320.15| +2452511|69679|4749|88441|592052|4016|33227|8|102|1054|33|9.66|14.87|7.13|0.00|235.29|318.78|490.71|21.17|0.00|235.29|256.46|-83.49| +2452511|69679|6751|88441|592052|4016|33227|8|285|1054|76|85.43|133.27|3.99|0.00|303.24|6492.68|10128.52|15.16|0.00|303.24|318.40|-6189.44| +2452511|69679|6043|88441|592052|4016|33227|8|9|1054|81|25.29|43.24|7.35|0.00|595.35|2048.49|3502.44|53.58|0.00|595.35|648.93|-1453.14| +2452511|69679|13303|88441|592052|4016|33227|8|285|1054|51|68.01|90.45|15.37|0.00|783.87|3468.51|4612.95|70.54|0.00|783.87|854.41|-2684.64| +2452511|69679|7375|88441|592052|4016|33227|8|253|1054|98|20.80|36.60|23.42|0.00|2295.16|2038.40|3586.80|137.70|0.00|2295.16|2432.86|256.76| +2452511|69679|10266|88441|592052|4016|33227|8|209|1054|84|75.57|146.60|92.35|0.00|7757.40|6347.88|12314.40|310.29|0.00|7757.40|8067.69|1409.52| +2452511|69679|1795|88441|592052|4016|33227|8|191|1054|34|16.68|21.01|7.14|0.00|242.76|567.12|714.34|4.85|0.00|242.76|247.61|-324.36| +2452511|69679|11551|88441|592052|4016|33227|8|104|1054|74|10.30|20.29|14.00|0.00|1036.00|762.20|1501.46|72.52|0.00|1036.00|1108.52|273.80| +2451395|39114|12686|3953|316068|4536|37051|7|143|1055|30|24.77|33.19|6.63|0.00|198.90|743.10|995.70|3.97|0.00|198.90|202.87|-544.20| +2451395|39114|15092||316068|4536|37051|||1055|10||170.83|47.83|||876.10|1708.30|23.91||478.30||-397.80| +2451395|39114|1856|3953|316068|4536|37051|7|86|1055|49|20.08|20.08|1.80|0.00|88.20|983.92|983.92|2.64|0.00|88.20|90.84|-895.72| +2451395|39114|3127|3953|316068|4536|37051|7|5|1055|28|98.60|118.32|48.51|0.00|1358.28|2760.80|3312.96|0.00|0.00|1358.28|1358.28|-1402.52| +2451395|39114|10561|3953|316068|4536|37051|7|222|1055|51|20.09|35.96|8.99|0.00|458.49|1024.59|1833.96|18.33|0.00|458.49|476.82|-566.10| +2451395|39114|14624|3953||4536|37051||92|1055||||||||7145.04|||5858.16|5858.16|| +2451395|39114|11812|3953|316068|4536|37051|7|57|1055|92|68.92|77.87|1.55|5.70|142.60|6340.64|7164.04|12.32|5.70|136.90|149.22|-6203.74| +2451395|39114|1996|3953|316068|4536|37051|7|97|1055|22|79.08|87.77|29.84|0.00|656.48|1739.76|1930.94|6.56|0.00|656.48|663.04|-1083.28| +2451395|39114|15463|3953|316068|4536|37051|7|233|1055|34|8.13|13.17|10.93|0.00|371.62|276.42|447.78|7.43|0.00|371.62|379.05|95.20| +2451395|39114|3865|3953|316068|4536|37051|7|112|1055|79|36.66|52.05|31.23|0.00|2467.17|2896.14|4111.95|74.01|0.00|2467.17|2541.18|-428.97| +2451395||8932|3953|||37051|7|275|1055|65|||46.42||3017.30|1622.40|3244.80|||||| +2451395|39114|8938|3953|316068|4536|37051|7|9|1055|84|92.93|136.60|54.64|0.00|4589.76|7806.12|11474.40|229.48|0.00|4589.76|4819.24|-3216.36| +2451395|39114|3626|3953|316068|4536|37051|7|104|1055|24|2.69|3.41|1.60|0.00|38.40|64.56|81.84|3.45|0.00|38.40|41.85|-26.16| +2451395|39114|8444|3953|316068|4536|37051|7|91|1055|71|22.89|45.32|4.07|0.00|288.97|1625.19|3217.72|17.33|0.00|288.97|306.30|-1336.22| +||4322|351|||12465|8|175|1056|96|||18.53||||3786.24|124.52||||| +2451181|31574|6835||||12465|||1056||||85.85|||3244.33||140.79|||3660.64|275.52| +2451181|31574|7792|351|38789|6027|12465|8|218|1056|70|84.73|94.89|25.62|0.00|1793.40|5931.10|6642.30|35.86|0.00|1793.40|1829.26|-4137.70| +2451181|31574|14881|351|38789|6027|12465|8|89|1056|25|94.11|109.16|67.67|0.00|1691.75|2352.75|2729.00|118.42|0.00|1691.75|1810.17|-661.00| +2451181|31574|3949|351|38789|6027|12465|8|281|1056|43|28.36|48.77|26.33|0.00|1132.19|1219.48|2097.11|33.96|0.00|1132.19|1166.15|-87.29| +2451181|31574|14146|351|38789|6027|12465|8|45|1056|34|40.26|76.89|16.91|0.00|574.94|1368.84|2614.26|45.99|0.00|574.94|620.93|-793.90| +2451181|31574|16594|351|38789|6027|12465|8|261|1056|12|51.64|66.09|50.22|0.00|602.64|619.68|793.08|6.02|0.00|602.64|608.66|-17.04| +2451181|31574|10916|351|38789|6027|12465|8|114|1056|37|35.57|51.93|51.93|0.00|1921.41|1316.09|1921.41|57.64|0.00|1921.41|1979.05|605.32| +|31574|17366|351||||||1056||63.07|||||1702.89|2877.66|||1122.12|1178.22|| +2451181|31574|5245|351|38789|6027|12465|8|138|1056|89|88.02|175.15|8.75|0.00|778.75|7833.78|15588.35|70.08|0.00|778.75|848.83|-7055.03| +2451181|31574|12703|351|38789|6027|12465|8|64|1056|12|69.63|73.80|32.47|0.00|389.64|835.56|885.60|0.00|0.00|389.64|389.64|-445.92| +2451181||208|||6027||||1056||59.18|115.99||0.00||1301.96||2.03|0.00|101.86|103.89|-1200.10| +2451181|31574|8317|351|38789|6027|12465|8|43|1056|3|42.39|47.47|43.67|0.00|131.01|127.17|142.41|9.17|0.00|131.01|140.18|3.84| +2451181|31574|2498|351|38789|6027|12465|8|92|1056|97|7.19|8.34|7.83|0.00|759.51|697.43|808.98|30.38|0.00|759.51|789.89|62.08| +2451771|39001|6307|76539|35345|4904|33663|2|170|1057|38|82.84|125.91|11.33|0.00|430.54|3147.92|4784.58|0.00|0.00|430.54|430.54|-2717.38| +2451771|39001|7328|76539|35345|4904|33663|2|272|1057|49|96.86|186.93|48.60|0.00|2381.40|4746.14|9159.57|95.25|0.00|2381.40|2476.65|-2364.74| +2451771|39001|10262|76539|35345|4904|33663|2|131|1057|58|44.90|55.22|19.87|0.00|1152.46|2604.20|3202.76|103.72|0.00|1152.46|1256.18|-1451.74| +2451771|39001|12949|76539|35345|4904|33663|2|242|1057|31|60.89|73.67|5.15|0.00|159.65|1887.59|2283.77|11.17|0.00|159.65|170.82|-1727.94| +||11191||||33663|2|258|1057||||49.49|0.00|4553.08||||0.00|4553.08|4689.67|| +2451771|39001|11156|76539|35345|4904|33663|2|6|1057|84|20.74|41.27|9.90|0.00|831.60|1742.16|3466.68|16.63|0.00|831.60|848.23|-910.56| +2451771|39001|4238|76539|35345|4904|33663|2|247|1057|71|89.73|111.26|46.72|0.00|3317.12|6370.83|7899.46|99.51|0.00|3317.12|3416.63|-3053.71| +2451771|39001|8527|76539|35345|4904|33663|2|66|1057|58|22.81|32.84|4.26|2.47|247.08|1322.98|1904.72|0.00|2.47|244.61|244.61|-1078.37| +2451771|39001|15881|76539|35345|4904|33663|2|15|1057|14|58.35|95.11|66.57|316.87|931.98|816.90|1331.54|24.60|316.87|615.11|639.71|-201.79| +2451771|39001|5264|76539|35345|4904|33663|2|10|1057|86|31.97|37.40|24.31|0.00|2090.66|2749.42|3216.40|125.43|0.00|2090.66|2216.09|-658.76| +2451771|39001|11035|76539|35345|4904|33663|2|72|1057|91|95.04|119.75|87.41|7556.59|7954.31|8648.64|10897.25|15.90|7556.59|397.72|413.62|-8250.92| +2451771|39001|7195|76539|35345|4904|33663|2|41|1057|24|59.50|115.43|31.16|0.00|747.84|1428.00|2770.32|59.82|0.00|747.84|807.66|-680.16| +2451771|39001|14995|76539|35345|4904|33663|2|1|1057|5|85.78|134.67|0.00|0.00|0.00|428.90|673.35|0.00|0.00|0.00|0.00|-428.90| +|39001|1699|||4904|33663||37|1057|51|37.03|37.40|||419.22||1907.40|8.38||419.22||| +2451771|39001|1400|76539|35345|4904|33663|2|61|1057|77|13.01|22.11|17.02|0.00|1310.54|1001.77|1702.47|91.73|0.00|1310.54|1402.27|308.77| +2451771|39001|17411|76539|35345|4904|33663|2|188|1057|88|48.99|65.64|3.28|0.00|288.64|4311.12|5776.32|23.09|0.00|288.64|311.73|-4022.48| +2451761|47110|12746|98910||4559|22360|8||1058||42.67|80.21||2320.36|4936.95|3456.27|6497.01||2320.36|2616.59|2695.08|-839.68| +2451761|47110|6746|98910|1522358|4559|22360|8|74|1058|56|70.74|96.20|68.30|0.00|3824.80|3961.44|5387.20|344.23|0.00|3824.80|4169.03|-136.64| +2451761|47110|4382|98910|1522358|4559|22360|8|92|1058|68|54.22|66.14|52.91|0.00|3597.88|3686.96|4497.52|251.85|0.00|3597.88|3849.73|-89.08| +2451761|47110|13022|98910|1522358|4559|22360|8|165|1058|25|57.88|107.07|13.91|0.00|347.75|1447.00|2676.75|0.00|0.00|347.75|347.75|-1099.25| +2451761|47110|8069|98910|1522358|4559|22360|8|178|1058|64|1.04|1.38|0.12|1.30|7.68|66.56|88.32|0.31|1.30|6.38|6.69|-60.18| +2451761|47110|9469|98910|1522358|4559|22360|8|111|1058|67|79.78|97.33|75.91|2695.56|5085.97|5345.26|6521.11|47.80|2695.56|2390.41|2438.21|-2954.85| +2451761|47110|10003|98910|1522358|4559|22360|8|105|1058|31|88.19|155.21|20.17|0.00|625.27|2733.89|4811.51|37.51|0.00|625.27|662.78|-2108.62| +2451761|47110|12617|98910|1522358|4559|22360|8|11|1058|51|60.80|105.79|40.20|0.00|2050.20|3100.80|5395.29|123.01|0.00|2050.20|2173.21|-1050.60| +2451761|47110|5930|98910|1522358|4559|22360|8|97|1058|46|85.55|111.21|63.38|903.79|2915.48|3935.30|5115.66|181.05|903.79|2011.69|2192.74|-1923.61| +2451761|47110|1931|98910|1522358|4559|22360|8|220|1058|98|73.01|105.86|12.70|0.00|1244.60|7154.98|10374.28|87.12|0.00|1244.60|1331.72|-5910.38| +2451761|47110|16862|98910|1522358|4559|22360|8|187|1058|7|12.38|21.29|3.83|9.91|26.81|86.66|149.03|1.35|9.91|16.90|18.25|-69.76| +2451761|47110|5377|98910|1522358|4559|22360|8|211|1058|74|65.42|110.55|57.48|0.00|4253.52|4841.08|8180.70|255.21|0.00|4253.52|4508.73|-587.56| +2451761|47110|9479|98910|1522358|4559|22360|8|168|1058|37|52.79|76.54|31.38|267.04|1161.06|1953.23|2831.98|53.64|267.04|894.02|947.66|-1059.21| +2451447|28944|3902|27017|1211221|1025|31629|8|39|1059|5|80.56|87.00|72.21|0.00|361.05|402.80|435.00|3.61|0.00|361.05|364.66|-41.75| +2451447|28944|14204|27017|1211221|1025|31629|8|2|1059|29|69.30|110.88|3.32|0.00|96.28|2009.70|3215.52|4.81|0.00|96.28|101.09|-1913.42| +2451447|28944|5198|27017|1211221|1025|31629|8|105|1059|3|66.40|86.32|15.53|0.00|46.59|199.20|258.96|0.00|0.00|46.59|46.59|-152.61| +2451447||5089|27017||||||1059||17.26|26.58|18.87|0.00|811.41|742.18||16.22|0.00|||69.23| +2451447|28944|15139|27017|1211221|1025|31629|8|130|1059|81|33.75|66.82|26.05|0.00|2110.05|2733.75|5412.42|0.00|0.00|2110.05|2110.05|-623.70| +2451447|28944|4087|27017|1211221|1025|31629|8|100|1059|68|72.36|91.17|72.02|0.00|4897.36|4920.48|6199.56|342.81|0.00|4897.36|5240.17|-23.12| +2451447|28944|926|27017|1211221|1025|31629|8|289|1059|46|4.54|4.63|1.43|0.00|65.78|208.84|212.98|0.00|0.00|65.78|65.78|-143.06| +2451447|28944|11864|27017|1211221|1025|31629|8|22|1059|64|7.81|10.93|7.86|503.04|503.04|499.84|699.52|0.00|503.04|0.00|0.00|-499.84| +2451447|28944|14738|27017|1211221|1025|31629|8|280|1059|75|72.15|88.02|69.53|0.00|5214.75|5411.25|6601.50|0.00|0.00|5214.75|5214.75|-196.50| +2451447|28944|3164|27017|1211221|1025|31629|8|16|1059|32|58.67|116.75|98.07|2447.82|3138.24|1877.44|3736.00|48.32|2447.82|690.42|738.74|-1187.02| +2451447|28944|6721|27017|1211221|1025|31629|8|115|1059|76|81.51|149.97|61.48|1775.54|4672.48|6194.76|11397.72|231.75|1775.54|2896.94|3128.69|-3297.82| +2451447|28944|8881|27017|1211221|1025|31629|8|217|1059|1|41.92|42.75|40.18|7.63|40.18|41.92|42.75|0.65|7.63|32.55|33.20|-9.37| +2451460|54691|688|6968|739192|4663|44887|1|48|1060|9|87.01|114.85|103.36|0.00|930.24|783.09|1033.65|46.51|0.00|930.24|976.75|147.15| +2451460|54691|1195|6968|739192|4663|44887|1|47|1060|17|84.12|150.57|24.09|0.00|409.53|1430.04|2559.69|16.38|0.00|409.53|425.91|-1020.51| +2451460|54691|7495|6968|||44887|||1060|38|96.12|162.44|||5184.72||6172.72|||2229.43|2318.60|| +2451460|54691|1238|6968|739192|4663|44887|1|263|1060|50|27.22|54.44|51.17|0.00|2558.50|1361.00|2722.00|51.17|0.00|2558.50|2609.67|1197.50| +2451460|54691|10084|6968|739192|4663|44887|1|110|1060|4|57.83|69.39|34.00|76.16|136.00|231.32|277.56|1.79|76.16|59.84|61.63|-171.48| +2451460|54691|14260|6968|739192|4663|44887|1|253|1060|47|35.78|56.17|43.81|0.00|2059.07|1681.66|2639.99|41.18|0.00|2059.07|2100.25|377.41| +2451460|54691|4978|6968|739192|4663|44887|1|141|1060|2|45.07|80.67|23.39|0.00|46.78|90.14|161.34|2.80|0.00|46.78|49.58|-43.36| +||6698|6968|739192||||148|1060|2|20.49|38.31|||42.90|40.98|76.62|||42.90||| +2451460|54691|640|6968|739192|4663|44887|1|225|1060|33|11.08|20.71|14.08|0.00|464.64|365.64|683.43|9.29|0.00|464.64|473.93|99.00| +2451460|54691|15290|6968|739192|4663|44887|1|125|1060|100|40.23|48.67|30.66|0.00|3066.00|4023.00|4867.00|183.96|0.00|3066.00|3249.96|-957.00| +2451460|54691|9488|6968|739192|4663|44887|1|282|1060|70|94.57|106.86|73.73|0.00|5161.10|6619.90|7480.20|361.27|0.00|5161.10|5522.37|-1458.80| +2451460|54691|9385|6968|739192|4663|44887|1|25|1060|66|15.99|25.42|18.04|773.91|1190.64|1055.34|1677.72|8.33|773.91|416.73|425.06|-638.61| +2451460|54691|13447|6968|739192|||1|92|1060||57.30|98.55|42.37|0.00|||2266.65|9.74|0.00||984.25|-343.39| +2451460||3136||739192|4663||1||1060||5.39|8.30|2.82|0.00||377.30|||0.00|||| +2451460|54691|3289|6968|739192|4663|44887|1|128|1060|88|37.91|39.04|3.12|233.37|274.56|3336.08|3435.52|3.70|233.37|41.19|44.89|-3294.89| +2452159|51529|12533|60794|1197064|6934|12804|2|212|1061|99|1.11|2.13|0.27|0.00|26.73|109.89|210.87|0.80|0.00|26.73|27.53|-83.16| +2452159|51529|10347|60794|1197064|6934|12804|2|6|1061|79|24.93|28.91|15.61|0.00|1233.19|1969.47|2283.89|0.00|0.00|1233.19|1233.19|-736.28| +2452159|51529|4035|60794|1197064|6934|12804|2|182|1061|39|91.19|182.38|69.30|1297.29|2702.70|3556.41|7112.82|84.32|1297.29|1405.41|1489.73|-2151.00| +2452159|51529|12825|60794|1197064|6934|12804|2|108|1061|45|44.11|55.13|41.89|0.00|1885.05|1984.95|2480.85|37.70|0.00|1885.05|1922.75|-99.90| +2452159|51529|4923|60794|1197064|6934|12804|2|34|1061|22|11.56|14.68|7.63|0.00|167.86|254.32|322.96|3.35|0.00|167.86|171.21|-86.46| +|51529|2209||1197064|||2|108|1061|88||64.38||0.00|3965.28|3752.32|5665.44||0.00|3965.28||212.96| +2452159|51529|11801|60794|1197064|6934|12804|2|244|1061|73|99.73|176.52|81.19|0.00|5926.87|7280.29|12885.96|177.80|0.00|5926.87|6104.67|-1353.42| +2452159|51529|8001|60794|1197064|6934|12804|2|142|1061|31|29.13|55.34|27.11|0.00|840.41|903.03|1715.54|50.42|0.00|840.41|890.83|-62.62| +2452612|34079|6015|61626|1481246|2019|44439|7|36|1062|21|1.97|3.23|3.00|0.00|63.00|41.37|67.83|5.67|0.00|63.00|68.67|21.63| +2452612|34079|4285|61626|1481246|2019|44439|7|146|1062|67|11.02|15.09|14.93|0.00|1000.31|738.34|1011.03|0.00|0.00|1000.31|1000.31|261.97| +2452612|34079|12033|61626|1481246|2019|44439|7|230|1062|81|73.18|141.23|96.03|0.00|7778.43|5927.58|11439.63|233.35|0.00|7778.43|8011.78|1850.85| +2452612|34079|8130|61626|1481246|2019|44439|7|153|1062|14|46.88|59.53|38.69|0.00|541.66|656.32|833.42|27.08|0.00|541.66|568.74|-114.66| +2452612|34079|3459|61626|1481246|2019|44439|7|251|1062|17|5.17|5.99|3.95|24.84|67.15|87.89|101.83|0.84|24.84|42.31|43.15|-45.58| +2452612|34079|10947|61626|1481246|2019|44439|7|113|1062|49|83.82|160.93|96.55|0.00|4730.95|4107.18|7885.57|47.30|0.00|4730.95|4778.25|623.77| +2452612|34079|403|61626|1481246|2019|44439|7|58|1062|3|3.39|3.89|0.89|0.00|2.67|10.17|11.67|0.05|0.00|2.67|2.72|-7.50| +2452612|34079|2575|61626|1481246|2019|44439|7|200|1062|35|39.02|49.16|3.93|0.00|137.55|1365.70|1720.60|1.37|0.00|137.55|138.92|-1228.15| +2452612|34079|4659|61626|1481246|2019|44439|7|141|1062|21|71.55|89.43|18.78|0.00|394.38|1502.55|1878.03|3.94|0.00|394.38|398.32|-1108.17| +2452612|34079|9721|61626|1481246|2019|44439|7|158|1062|47|4.60|6.85|5.48|0.00|257.56|216.20|321.95|2.57|0.00|257.56|260.13|41.36| +2452612|34079|8808|61626|1481246|2019|44439|7|198|1062|68|31.47|45.63|27.83|1684.27|1892.44|2139.96|3102.84|0.00|1684.27|208.17|208.17|-1931.79| +2452612|34079|13711|61626|1481246|2019|44439|7|105|1062|85|40.35|72.22|59.22|0.00|5033.70|3429.75|6138.70|352.35|0.00|5033.70|5386.05|1603.95| +2452612|34079|13650|61626|1481246|2019|44439|7|252|1062|51|31.33|42.60|2.55|0.00|130.05|1597.83|2172.60|0.00|0.00|130.05|130.05|-1467.78| +2452612|34079|19|61626|1481246|2019|44439|7|248|1062|7|69.86|78.94|6.31|0.00|44.17|489.02|552.58|2.65|0.00|44.17|46.82|-444.85| +2452612|34079|9451|61626|1481246|2019|44439|7|31|1062|7|18.79|20.48|14.13|0.00|98.91|131.53|143.36|0.98|0.00|98.91|99.89|-32.62| +2452600|36829|10395|48510|1161809|1155|24054|4|246|1063|36|55.72|70.76|25.47|0.00|916.92|2005.92|2547.36|9.16|0.00|916.92|926.08|-1089.00| +2452600|36829|10075|48510|1161809|1155|24054|4|32|1063|73|25.89|33.39|1.66|41.20|121.18|1889.97|2437.47|1.59|41.20|79.98|81.57|-1809.99| +2452600|36829|7083|48510|1161809|1155|24054|4|74|1063|39|80.70|91.19|1.82|0.00|70.98|3147.30|3556.41|3.54|0.00|70.98|74.52|-3076.32| +2452600|36829|9541|48510|1161809|1155|24054|4|30|1063|39|44.69|84.46|5.06|0.00|197.34|1742.91|3293.94|17.76|0.00|197.34|215.10|-1545.57| +2452600|36829|10068|48510|1161809|1155|24054|4|156|1063|32|82.37|98.84|50.40|0.00|1612.80|2635.84|3162.88|145.15|0.00|1612.80|1757.95|-1023.04| +2452600||354||1161809|||4||1063|78|57.30|80.22||0.00|5443.62|4469.40|||0.00|5443.62|5552.49|974.22| +2452600|36829|12657|48510|1161809|1155|24054|4|100|1063|56|53.72|83.26|32.47|0.00|1818.32|3008.32|4662.56|109.09|0.00|1818.32|1927.41|-1190.00| +2452600|36829|10767|48510|1161809|1155|24054|4|190|1063|83|54.75|64.05|44.83|0.00|3720.89|4544.25|5316.15|223.25|0.00|3720.89|3944.14|-823.36| +2452600|36829|16395|48510|1161809|1155|24054|4|168|1063|16|33.34|50.01|19.00|0.00|304.00|533.44|800.16|18.24|0.00|304.00|322.24|-229.44| +2452600|36829|5277|48510|1161809|1155|24054|4|237|1063|30|38.80|67.51|43.20|0.00|1296.00|1164.00|2025.30|90.72|0.00|1296.00|1386.72|132.00| +2451530|64831|15493|98161|1259268|1975|11835|7|31|1064|6|52.08|94.78|67.29|0.00|403.74|312.48|568.68|4.03|0.00|403.74|407.77|91.26| +|64831|5618||1259268||||54|1064|63||173.61||9967.92|10171.35||10937.43||9967.92|203.43|221.73|| +2451530|64831|8020|98161|1259268|1975|11835|7|268|1064|45|65.47|108.68|103.24|1161.45|4645.80|2946.15|4890.60|243.90|1161.45|3484.35|3728.25|538.20| +2451530|64831|17020|98161|1259268|1975|11835|7|214|1064|87|36.08|44.01|18.48|0.00|1607.76|3138.96|3828.87|0.00|0.00|1607.76|1607.76|-1531.20| +2451530|64831|1832|98161|1259268|1975|11835|7|116|1064|4|28.38|35.47|22.34|0.00|89.36|113.52|141.88|1.78|0.00|89.36|91.14|-24.16| +2451530|64831|14834|98161|1259268|1975|11835|7|170|1064|76|55.38|79.19|62.56|0.00|4754.56|4208.88|6018.44|427.91|0.00|4754.56|5182.47|545.68| +2451530|64831|2734|98161|1259268|1975|11835|7|79|1064|61|71.55|100.17|20.03|0.00|1221.83|4364.55|6110.37|61.09|0.00|1221.83|1282.92|-3142.72| +2451530|64831|1466|98161||1975|11835|||1064|7||17.11|2.39|0.00|16.73|62.72|119.77|0.33|0.00|16.73|17.06|| +2451002|38393|1288|32035|1579857|6432|30607|1|84|1065|99|52.51|102.39|34.81|0.00|3446.19|5198.49|10136.61|103.38|0.00|3446.19|3549.57|-1752.30| +2451002|38393|13796|32035|1579857|6432|30607|1|220|1065|91|89.00|98.79|98.79|0.00|8989.89|8099.00|8989.89|629.29|0.00|8989.89|9619.18|890.89| +2451002|38393|5647|32035|1579857|6432|30607|1|4|1065|20|32.82|57.10|41.68|0.00|833.60|656.40|1142.00|66.68|0.00|833.60|900.28|177.20| +2451002|38393|290|32035|1579857|6432|30607|1|118|1065|76|17.34|19.59|9.40|0.00|714.40|1317.84|1488.84|7.14|0.00|714.40|721.54|-603.44| +||17284|32035|1579857|6432|30607||5|1065|96|||||||9874.56||||629.00|| +2451002|38393|10591|32035|1579857|6432|30607|1|186|1065|47|89.87|150.98|146.45|0.00|6883.15|4223.89|7096.06|550.65|0.00|6883.15|7433.80|2659.26| +2451002|38393|11462|32035|1579857|6432|30607|1|46|1065|3|34.28|61.36|44.17|0.00|132.51|102.84|184.08|10.60|0.00|132.51|143.11|29.67| +2451002|38393|12362|32035|1579857|6432|30607|1|22|1065|37|43.76|60.82|5.47|111.31|202.39|1619.12|2250.34|0.91|111.31|91.08|91.99|-1528.04| +2451002|38393|10552|32035|1579857|6432|30607|1|207|1065|78|74.22|124.68|1.24|0.00|96.72|5789.16|9725.04|5.80|0.00|96.72|102.52|-5692.44| +2451002|38393|2128|32035|1579857|6432|30607|1|139|1065|10|62.60|92.02|64.41|0.00|644.10|626.00|920.20|32.20|0.00|644.10|676.30|18.10| +2451002|38393|4274|32035|1579857|6432|30607|1|204|1065|69|49.28|72.93|70.01|0.00|4830.69|3400.32|5032.17|144.92|0.00|4830.69|4975.61|1430.37| +2451002|38393|7888|32035|1579857|6432|30607|1|111|1065|4|69.00|93.84|63.81|0.00|255.24|276.00|375.36|12.76|0.00|255.24|268.00|-20.76| +2451002|38393|16724|32035|1579857|6432|30607|1|220|1065|66|30.31|49.70|18.38|606.54|1213.08|2000.46|3280.20|36.39|606.54|606.54|642.93|-1393.92| +2451002|38393|17164|32035|1579857|6432|30607|1|163|1065|62|19.74|29.01|8.70|0.00|539.40|1223.88|1798.62|43.15|0.00|539.40|582.55|-684.48| +2451869|60365|2659|98664|1352701|3985|40869|8|216|1066|2|83.21|102.34|30.70|0.00|61.40|166.42|204.68|3.07|0.00|61.40|64.47|-105.02| +2451869|60365|8053|98664|1352701|3985|40869|8|126|1066|14|6.42|12.45|6.22|49.63|87.08|89.88|174.30|2.99|49.63|37.45|40.44|-52.43| +2451869|60365|3473|98664|1352701|3985|40869|8|121|1066|50|97.02|104.78|46.10|1290.80|2305.00|4851.00|5239.00|81.13|1290.80|1014.20|1095.33|-3836.80| +2451869||1729|||3985||||1066|38|||71.03|1727.44|2699.14|1704.30|||1727.44|971.70|1020.28|-732.60| +2451869|60365|15746|98664|1352701|3985|40869|8|173|1066|45|73.10|119.88|75.52|0.00|3398.40|3289.50|5394.60|169.92|0.00|3398.40|3568.32|108.90| +|60365|5246||1352701||40869|8|186|1066|14|69.21|||152.17||968.94|1598.66||152.17|||-34.15| +2451869|60365|17597|98664|1352701|3985|40869|8|77|1066|42|24.01|36.01|31.68|0.00|1330.56|1008.42|1512.42|26.61|0.00|1330.56|1357.17|322.14| +2451869|60365|3824|98664|1352701|3985|40869|8|160|1066|29|17.70|28.85|2.88|0.00|83.52|513.30|836.65|0.00|0.00|83.52|83.52|-429.78| +2451869|60365|2965|98664|1352701|3985|40869|8|294|1066|8|18.91|29.68|11.87|0.00|94.96|151.28|237.44|1.89|0.00|94.96|96.85|-56.32| +2451869|60365|12287|98664|1352701|3985|40869|8|147|1066|20|2.10|2.14|0.40|0.00|8.00|42.00|42.80|0.64|0.00|8.00|8.64|-34.00| +2451869|60365|5810|98664|1352701|3985|40869|8|121|1066|82|28.28|38.74|29.05|0.00|2382.10|2318.96|3176.68|166.74|0.00|2382.10|2548.84|63.14| +2451869|60365|3611|98664|1352701|3985|40869|8|110|1066|5|49.06|56.90|54.62|0.00|273.10|245.30|284.50|10.92|0.00|273.10|284.02|27.80| +2451869|60365|1742|98664|1352701|3985|40869|8|183|1066|68|67.73|75.18|27.81|0.00|1891.08|4605.64|5112.24|170.19|0.00|1891.08|2061.27|-2714.56| +2451869|60365|1112|98664|1352701|3985|40869|8|249|1066|86|40.64|66.24|2.64|222.49|227.04|3495.04|5696.64|0.22|222.49|4.55|4.77|-3490.49| +2451869|60365|9674|98664|1352701|3985|40869|8|193|1066|33|28.77|37.97|22.78|0.00|751.74|949.41|1253.01|22.55|0.00|751.74|774.29|-197.67| +2451869|60365|9656|98664|1352701|3985|40869|8|269|1066|84|79.11|106.00|20.14|152.25|1691.76|6645.24|8904.00|30.79|152.25|1539.51|1570.30|-5105.73| +2451870||6101|||4610||1||1067|||37.56||105.44|||1464.84|4.80|105.44|480.34||| +2451870|36849|12889|10256|91033|4610|10587|1|264|1067|75|36.78|70.24|60.40|0.00|4530.00|2758.50|5268.00|45.30|0.00|4530.00|4575.30|1771.50| +2451870|36849|5947|10256|91033|4610|10587|1|147|1067|41|63.61|66.15|50.27|0.00|2061.07|2608.01|2712.15|144.27|0.00|2061.07|2205.34|-546.94| +2451870|36849|11048|10256|91033|4610|10587|1|222|1067|3|83.69|97.08|49.51|0.00|148.53|251.07|291.24|11.88|0.00|148.53|160.41|-102.54| +2451870|36849|8126|10256|91033|||1||1067||||2.44|125.24|143.96|92.63||0.00|125.24|||-73.91| +2451870|36849|9527|10256|91033|4610|10587|1|138|1067|60|48.43|84.75|56.78|0.00|3406.80|2905.80|5085.00|170.34|0.00|3406.80|3577.14|501.00| +2451870|36849|17582|10256|91033|4610|10587|1|9|1067|60|12.77|21.96|1.75|0.00|105.00|766.20|1317.60|9.45|0.00|105.00|114.45|-661.20| +|36849|12326|10256|91033|||1|194|1067||41.56||34.28||2365.32|||||2365.32|2365.32|| +2451870|36849|16063|10256|91033|4610|10587|1|228|1067|69|23.74|30.14|28.93|1756.62|1996.17|1638.06|2079.66|11.97|1756.62|239.55|251.52|-1398.51| +2451870|36849|1507|10256|91033|4610|10587|1|81|1067|66|47.42|93.41|18.68|0.00|1232.88|3129.72|6165.06|49.31|0.00|1232.88|1282.19|-1896.84| +2451870|36849|1508|10256|91033|4610|10587|1|239|1067|70|10.59|14.93|14.63|798.79|1024.10|741.30|1045.10|11.26|798.79|225.31|236.57|-515.99| +2451870|36849|2689|10256|91033|4610|10587|1|249|1067|25|50.95|99.86|0.99|0.00|24.75|1273.75|2496.50|0.24|0.00|24.75|24.99|-1249.00| +2451901|64335|12181|21921|324641|1761|8881|8|185|1068|46|20.84|24.17|22.23|0.00|1022.58|958.64|1111.82|30.67|0.00|1022.58|1053.25|63.94| +2451901|64335|5345|21921|324641|1761|8881|8|289|1068|15|53.20|78.20|43.79|0.00|656.85|798.00|1173.00|19.70|0.00|656.85|676.55|-141.15| +2451901|64335|9845|21921|324641|1761|8881|8|141|1068|4|18.43|29.67|9.49|0.00|37.96|73.72|118.68|1.51|0.00|37.96|39.47|-35.76| +2451901|64335|470|21921|324641|1761|8881|8|191|1068|78|79.16|103.69|0.00|0.00|0.00|6174.48|8087.82|0.00|0.00|0.00|0.00|-6174.48| +2451901|64335|17903|21921|324641|1761|8881|8|251|1068|89|18.50|18.87|1.50|52.06|133.50|1646.50|1679.43|4.88|52.06|81.44|86.32|-1565.06| +2451901|64335|518|21921|324641|1761|8881|8|240|1068|48|6.78|9.01|2.70|0.00|129.60|325.44|432.48|5.18|0.00|129.60|134.78|-195.84| +2451901|64335|15014|21921|324641|1761|8881|8|244|1068|86|4.70|7.42|1.48|0.00|127.28|404.20|638.12|6.36|0.00|127.28|133.64|-276.92| +2451901|64335|4291|21921|324641|1761|8881|8|111|1068|78|31.96|33.23|10.30|0.00|803.40|2492.88|2591.94|16.06|0.00|803.40|819.46|-1689.48| +2451901|64335|14717|21921|324641|1761|8881|8|214|1068|97|48.32|49.28|41.39|2689.93|4014.83|4687.04|4780.16|66.24|2689.93|1324.90|1391.14|-3362.14| +2451901|64335|851|21921|324641|1761|8881|8|273|1068|95|52.18|53.74|12.36|0.00|1174.20|4957.10|5105.30|82.19|0.00|1174.20|1256.39|-3782.90| +2451901|64335|4603|21921|324641|1761|8881|8|104|1068|30|35.90|54.56|44.73|0.00|1341.90|1077.00|1636.80|0.00|0.00|1341.90|1341.90|264.90| +2451854|44915|896|30512|1872586|3907|28828|7|287|1069|26|1.71|2.05|1.43|0.00|37.18|44.46|53.30|0.00|0.00|37.18|37.18|-7.28| +2451854|44915|10184|30512|1872586|3907|28828|7|245|1069|58|39.28|50.67|18.74|54.34|1086.92|2278.24|2938.86|72.28|54.34|1032.58|1104.86|-1245.66| +2451854|44915|10283|30512|1872586|3907|28828|7|225|1069|89|59.74|69.29|11.77|0.00|1047.53|5316.86|6166.81|73.32|0.00|1047.53|1120.85|-4269.33| +2451854|44915|17965|30512|1872586|3907|28828|7|202|1069|48|29.07|50.29|46.76|0.00|2244.48|1395.36|2413.92|202.00|0.00|2244.48|2446.48|849.12| +2451854|44915|12799|30512|1872586|3907|28828|7|180|1069|97|48.71|53.09|52.02|0.00|5045.94|4724.87|5149.73|302.75|0.00|5045.94|5348.69|321.07| +2451854|44915|8066|30512|1872586|3907|28828|7|176|1069|68|13.22|21.81|21.37|0.00|1453.16|898.96|1483.08|43.59|0.00|1453.16|1496.75|554.20| +2451854|44915|16238|30512|1872586|3907|28828|7|43|1069|26|7.66|12.02|1.08|0.00|28.08|199.16|312.52|0.84|0.00|28.08|28.92|-171.08| +2451854|44915|17738|30512|1872586|3907|28828|7|81|1069|73|91.69|116.44|107.12|0.00|7819.76|6693.37|8500.12|703.77|0.00|7819.76|8523.53|1126.39| +2451854|44915|9881|30512|1872586|3907|28828|7|39|1069|62|59.06|99.81|26.94|0.00|1670.28|3661.72|6188.22|116.91|0.00|1670.28|1787.19|-1991.44| +||16457|||3907||7||1069|93|||26.88|||4025.04|4386.81|||2449.85|2523.34|| +2451854|44915|2336|30512|1872586|3907|28828|7|271|1069|34|6.32|8.72|6.88|0.00|233.92|214.88|296.48|14.03|0.00|233.92|247.95|19.04| +2451854|44915|9571|30512|1872586|3907|28828|7|204|1069|74|83.64|127.96|33.26|2264.34|2461.24|6189.36|9469.04|17.72|2264.34|196.90|214.62|-5992.46| +2451951|63808|2617|18406|525012|418|42224|1|4|1070|86|30.83|48.71|48.71|628.35|4189.06|2651.38|4189.06|249.24|628.35|3560.71|3809.95|909.33| +2451951|63808|2745|18406|525012|418|42224|1|88|1070|5|92.23|96.84|78.44|0.00|392.20|461.15|484.20|11.76|0.00|392.20|403.96|-68.95| +2451951|63808|17769|18406|525012|418|42224|1|203|1070|51|37.85|57.15|27.43|0.00|1398.93|1930.35|2914.65|97.92|0.00|1398.93|1496.85|-531.42| +2451951|63808|7317|18406|525012|418|42224|1|43|1070|77|20.90|27.79|24.73|0.00|1904.21|1609.30|2139.83|133.29|0.00|1904.21|2037.50|294.91| +2451951|63808|14563|18406|525012|418|42224|1|44|1070|23|20.84|37.30|11.19|0.00|257.37|479.32|857.90|23.16|0.00|257.37|280.53|-221.95| +2451951|63808|6845|18406|525012|418|42224|1|153|1070|16|73.53|94.85|11.38|0.00|182.08|1176.48|1517.60|12.74|0.00|182.08|194.82|-994.40| +2451951|63808|3795|18406|525012|418|42224|1|210|1070|55|1.81|2.49|1.86|0.00|102.30|99.55|136.95|4.09|0.00|102.30|106.39|2.75| +2451951|63808|11979|18406|525012|418|42224|1|47|1070|96|86.37|125.23|37.56|0.00|3605.76|8291.52|12022.08|180.28|0.00|3605.76|3786.04|-4685.76| +2451951|63808|9463|18406|525012|418|42224|1|44|1070|67|7.53|12.95|12.82|0.00|858.94|504.51|867.65|34.35|0.00|858.94|893.29|354.43| +2451951|63808|11553|18406|525012|418|42224|1|114|1070|92|28.06|47.70|28.62|0.00|2633.04|2581.52|4388.40|236.97|0.00|2633.04|2870.01|51.52| +2451951|63808|431|18406|525012|418|42224|1|216|1070|61|82.21|151.26|22.68|0.00|1383.48|5014.81|9226.86|41.50|0.00|1383.48|1424.98|-3631.33| +2451951|63808|7145|18406|525012|418|42224|1|296|1070|25|35.89|61.37|55.23|538.49|1380.75|897.25|1534.25|16.84|538.49|842.26|859.10|-54.99| +2451951|63808|6611|18406|525012|418|42224|1|85|1070|94|96.27|169.43|118.60|5685.68|11148.40|9049.38|15926.42|382.39|5685.68|5462.72|5845.11|-3586.66| +2451951|63808|7801|18406|525012|418|42224|1|170|1070|100|77.92|143.37|123.29|0.00|12329.00|7792.00|14337.00|986.32|0.00|12329.00|13315.32|4537.00| +2451951|63808|6285|18406|525012|418|42224|1|182|1070|30|91.93|94.68|14.20|0.00|426.00|2757.90|2840.40|4.26|0.00|426.00|430.26|-2331.90| +2451951||5839||||||86|1070|40||114.35|||2195.20|||57.29||1909.83||| +2451774|74348|14671|43793|1374111|76|29194|4|269|1071|72|77.99|83.44|29.20|0.00|2102.40|5615.28|6007.68|63.07|0.00|2102.40|2165.47|-3512.88| +2451774|74348|2765|43793|1374111|76|29194|4|78|1071|77|11.70|21.06|19.79|975.25|1523.83|900.90|1621.62|38.40|975.25|548.58|586.98|-352.32| +2451774|74348|3595|43793|1374111|76|29194|4|74|1071|30|60.65|88.54|3.54|0.00|106.20|1819.50|2656.20|2.12|0.00|106.20|108.32|-1713.30| +2451774|74348|11411|43793|1374111|76|29194|4|220|1071|72|15.09|24.74|15.33|0.00|1103.76|1086.48|1781.28|44.15|0.00|1103.76|1147.91|17.28| +2451774|74348|1141|43793|1374111|76|29194|4|126|1071|74|11.20|19.48|8.18|0.00|605.32|828.80|1441.52|12.10|0.00|605.32|617.42|-223.48| +2451774|74348|3638|43793|1374111|76|29194|4|79|1071|44|54.33|76.60|36.00|0.00|1584.00|2390.52|3370.40|0.00|0.00|1584.00|1584.00|-806.52| +2451774|74348|16799|43793|1374111|76|29194|4|202|1071|62|28.35|40.54|35.26|0.00|2186.12|1757.70|2513.48|109.30|0.00|2186.12|2295.42|428.42| +2451774|74348|4472|43793|1374111|76|29194|4|289|1071|80|70.19|135.46|90.75|0.00|7260.00|5615.20|10836.80|72.60|0.00|7260.00|7332.60|1644.80| +2451774|74348|12115|43793|1374111|76|29194|4|267|1071|39|63.50|66.04|27.07|0.00|1055.73|2476.50|2575.56|63.34|0.00|1055.73|1119.07|-1420.77| +2451774|74348|15613|43793|1374111|76|29194|4|97|1071|24|92.24|121.75|119.31|0.00|2863.44|2213.76|2922.00|143.17|0.00|2863.44|3006.61|649.68| +2451774|74348|8696|43793|1374111|76|29194|4|14|1071|11|82.44|117.88|80.15|0.00|881.65|906.84|1296.68|79.34|0.00|881.65|960.99|-25.19| +2451774|74348|12023|43793|1374111|76|29194|4|15|1071|79|51.53|70.59|48.70|0.00|3847.30|4070.87|5576.61|346.25|0.00|3847.30|4193.55|-223.57| +2451774|74348|14255|43793||||4||1071|7||4.16|0.87|0.00|6.09|||0.00|0.00||6.09|-17.22| +2451774|74348|8365|43793|1374111|76|29194|4|215|1071|19|61.05|79.97|29.58|5.62|562.02|1159.95|1519.43|33.38|5.62|556.40|589.78|-603.55| +2451774|74348|5189|43793|1374111|76|29194|4|43|1071|13|49.91|65.38|24.84|164.68|322.92|648.83|849.94|14.24|164.68|158.24|172.48|-490.59| +2450825|50897|5005|70098|791869|3240|25411|2|235|1072|48|83.52|132.79|11.95|57.36|573.60|4008.96|6373.92|0.00|57.36|516.24|516.24|-3492.72| +2450825|50897|12274|70098|791869|3240|25411|2|155|1072|1|61.05|73.26|69.59|34.09|69.59|61.05|73.26|1.77|34.09|35.50|37.27|-25.55| +2450825|50897|7250|70098|791869|3240|25411|2|181|1072|99|43.35|51.58|31.46|2086.74|3114.54|4291.65|5106.42|41.11|2086.74|1027.80|1068.91|-3263.85| +2450825|50897|3956|70098|791869|3240|25411|2|15|1072|56|59.26|105.48|32.69|0.00|1830.64|3318.56|5906.88|91.53|0.00|1830.64|1922.17|-1487.92| +2450825||6877|70098|||25411|2|129|1072||||122.94||5286.42||6446.99|422.91|||5709.33|| +2450825|50897|13180|70098|791869|3240|25411|2|136|1072|22|59.74|75.27|28.60|0.00|629.20|1314.28|1655.94|12.58|0.00|629.20|641.78|-685.08| +2450825|50897|4156|70098|791869|3240|25411|2|3|1072|64|87.12|148.97|111.72|0.00|7150.08|5575.68|9534.08|286.00|0.00|7150.08|7436.08|1574.40| +2450825|50897|13174|70098|791869|3240|25411|2|176|1072|95|76.75|84.42|48.96|0.00|4651.20|7291.25|8019.90|372.09|0.00|4651.20|5023.29|-2640.05| +2450825|50897|145|70098|791869|3240|25411|2|153|1072|57|42.41|82.69|35.55|0.00|2026.35|2417.37|4713.33|101.31|0.00|2026.35|2127.66|-391.02| +2452141|31120|9411|56896|1243128|2513|2745|7|42|1073|61|79.32|145.15|143.69|0.00|8765.09|4838.52|8854.15|701.20|0.00|8765.09|9466.29|3926.57| +2452141|31120|8327|56896|1243128|2513|2745|7|121|1073|63|5.02|8.38|2.93|0.00|184.59|316.26|527.94|1.84|0.00|184.59|186.43|-131.67| +2452141|31120|6435|56896|1243128|2513|2745|7|154|1073|22|45.81|50.84|11.69|0.00|257.18|1007.82|1118.48|0.00|0.00|257.18|257.18|-750.64| +2452141|31120|14005|56896|1243128|2513|2745|7|152|1073|8|37.56|44.32|29.25|0.00|234.00|300.48|354.56|7.02|0.00|234.00|241.02|-66.48| +2452141|31120|4477|56896|1243128|2513|2745|7|130|1073|96|84.46|113.17|36.21|0.00|3476.16|8108.16|10864.32|312.85|0.00|3476.16|3789.01|-4632.00| +2452141|31120|9351|56896|1243128|2513|2745|7|184|1073|55|51.16|96.69|5.80|0.00|319.00|2813.80|5317.95|0.00|0.00|319.00|319.00|-2494.80| +2452141|31120|4149|56896|1243128|2513|2745|7|182|1073|34|59.41|79.01|23.70|701.04|805.80|2019.94|2686.34|1.04|701.04|104.76|105.80|-1915.18| +2452141|31120|10231|56896|1243128|2513|2745|7|15|1073|11|69.91|74.80|67.32|0.00|740.52|769.01|822.80|44.43|0.00|740.52|784.95|-28.49| +2452141|31120|7979|56896|1243128|2513|2745|7|79|1073|59|65.63|85.31|31.56|0.00|1862.04|3872.17|5033.29|0.00|0.00|1862.04|1862.04|-2010.13| +2452141|31120|10327|56896|1243128|2513|2745|7|130|1073|8|62.37|123.49|90.14|28.84|721.12|498.96|987.92|41.53|28.84|692.28|733.81|193.32| +2452141|31120|8839|56896|1243128|2513|2745|7|142|1073|87|46.87|91.39|60.31|0.00|5246.97|4077.69|7950.93|0.00|0.00|5246.97|5246.97|1169.28| +2452141|31120|15367|56896|1243128|2513|2745|7|279|1073|74|99.88|179.78|70.11|0.00|5188.14|7391.12|13303.72|103.76|0.00|5188.14|5291.90|-2202.98| +2452141|31120|2559|56896|1243128|2513|2745|7|127|1073|52|49.20|82.65|57.02|0.00|2965.04|2558.40|4297.80|88.95|0.00|2965.04|3053.99|406.64| +2452141|31120|9167|56896|1243128|2513|2745|7|220|1073|1|17.82|35.28|1.05|0.00|1.05|17.82|35.28|0.08|0.00|1.05|1.13|-16.77| +2452141||16371||1243128|2513||||1073|97||35.80|30.43|0.00|||3472.60|59.03|0.00||3010.74|128.04| +2452141|31120|14053|56896|1243128|2513|2745|7|156|1073|70|51.69|89.42|35.76|0.00|2503.20|3618.30|6259.40|150.19|0.00|2503.20|2653.39|-1115.10| +2451126|45730|12664|70555|1801485|4768|5298|10|21|1074|29|68.51|136.33|85.88|0.00|2490.52|1986.79|3953.57|174.33|0.00|2490.52|2664.85|503.73| +2451126|45730|15769|70555|1801485|4768|5298|10|266|1074|51|93.02|148.83|147.34|0.00|7514.34|4744.02|7590.33|0.00|0.00|7514.34|7514.34|2770.32| +2451126|45730|10771|70555|1801485|4768|5298|10|200|1074|27|53.73|73.07|24.84|0.00|670.68|1450.71|1972.89|46.94|0.00|670.68|717.62|-780.03| +2451126|45730|14791|70555|1801485|4768|5298|10|161|1074|55|98.40|146.61|133.41|0.00|7337.55|5412.00|8063.55|0.00|0.00|7337.55|7337.55|1925.55| +2451126|45730|7916|70555|1801485|4768|5298|10|63|1074|99|4.43|5.89|5.12|167.27|506.88|438.57|583.11|16.98|167.27|339.61|356.59|-98.96| +2451126|45730|11086|70555|1801485|4768|5298|10|118|1074|27|69.42|81.22|19.49|0.00|526.23|1874.34|2192.94|5.26|0.00|526.23|531.49|-1348.11| +2451126|45730|1123|70555|1801485|4768|5298|10|189|1074|67|50.74|93.86|55.37|0.00|3709.79|3399.58|6288.62|259.68|0.00|3709.79|3969.47|310.21| +2451126|45730|15394|70555|1801485|4768|5298|10|47|1074|23|22.73|34.09|12.95|0.00|297.85|522.79|784.07|26.80|0.00|297.85|324.65|-224.94| +2451126|45730|9394|70555|1801485|4768|5298|10|188|1074|38|57.44|77.54|13.18|340.57|500.84|2182.72|2946.52|12.82|340.57|160.27|173.09|-2022.45| +2451126|45730|17894|70555|1801485||||40|1074||33.17|||0.00||1326.80|2016.40|8.86|0.00|221.60||| +2451126|45730|728|70555|1801485|4768|5298|10|86|1074|49|79.69|90.04|65.72|0.00|3220.28|3904.81|4411.96|128.81|0.00|3220.28|3349.09|-684.53| +2451126|45730|11564|70555|1801485|4768|5298|10|6|1074|91|46.51|81.85|33.55|0.00|3053.05|4232.41|7448.35|244.24|0.00|3053.05|3297.29|-1179.36| +2451126|45730|11188|70555|1801485|4768|5298|10|264|1074|27|99.38|117.26|73.87|0.00|1994.49|2683.26|3166.02|119.66|0.00|1994.49|2114.15|-688.77| +2451126|45730|9296|70555|1801485|4768|5298|10|286|1074|53|30.89|42.62|7.67|0.00|406.51|1637.17|2258.86|28.45|0.00|406.51|434.96|-1230.66| +2451100|43423|15200|4200|1668170|1740|20110|7|74|1075|64|95.82|112.10|89.68|0.00|5739.52|6132.48|7174.40|401.76|0.00|5739.52|6141.28|-392.96| +2451100|43423|6560|4200|1668170|1740|20110|7|141|1075|68|89.19|106.13|16.98|0.00|1154.64|6064.92|7216.84|23.09|0.00|1154.64|1177.73|-4910.28| +2451100|43423|12838|4200|1668170|1740|20110|7|144|1075|69|13.40|20.63|18.97|1256.57|1308.93|924.60|1423.47|4.18|1256.57|52.36|56.54|-872.24| +2451100|43423|14017|4200|1668170|1740|20110|7|100|1075|33|97.77|114.39|8.00|0.00|264.00|3226.41|3774.87|5.28|0.00|264.00|269.28|-2962.41| +2451100|43423|2692|4200|1668170|1740|20110|7|195|1075|71|74.53|127.44|79.01|0.00|5609.71|5291.63|9048.24|224.38|0.00|5609.71|5834.09|318.08| +2451100|43423|8329|4200|1668170|1740|20110|7|89|1075|21|43.21|68.27|37.54|0.00|788.34|907.41|1433.67|31.53|0.00|788.34|819.87|-119.07| +2451100|43423|7438|4200|1668170|1740|20110|7|91|1075|84|53.71|53.71|39.74|0.00|3338.16|4511.64|4511.64|66.76|0.00|3338.16|3404.92|-1173.48| +2451100|43423|13886|4200|1668170|1740|20110|7|250|1075|48|85.14|143.88|57.55|0.00|2762.40|4086.72|6906.24|82.87|0.00|2762.40|2845.27|-1324.32| +2451248|44760|7351|49059|611598|6937|39523|10|299|1076|49|86.41|148.62|43.09|0.00|2111.41|4234.09|7282.38|0.00|0.00|2111.41|2111.41|-2122.68| +2451248|44760|15433|49059|611598|6937|39523|10|130|1076|45|24.47|32.05|8.33|0.00|374.85|1101.15|1442.25|33.73|0.00|374.85|408.58|-726.30| +2451248|44760|4772|49059|611598|6937|39523|10|145|1076|86|92.91|136.57|118.81|0.00|10217.66|7990.26|11745.02|204.35|0.00|10217.66|10422.01|2227.40| +2451248|44760|17896|49059|611598|6937|39523|10|174|1076|26|27.39|39.44|16.95|0.00|440.70|712.14|1025.44|13.22|0.00|440.70|453.92|-271.44| +2451248|44760|15622|49059|611598|6937|39523|10|228|1076|82|37.80|69.17|41.50|0.00|3403.00|3099.60|5671.94|272.24|0.00|3403.00|3675.24|303.40| +2451248|44760|16586|49059|611598|6937|39523|10|194|1076|27|45.30|59.34|58.74|1506.68|1585.98|1223.10|1602.18|6.34|1506.68|79.30|85.64|-1143.80| +|44760|478|49059||6937|39523|10||1076|58|35.51||31.81||||2883.18|166.04|||2011.02|| +2451248|44760|6187|49059|611598|6937|39523|10|249|1076|62|3.98|4.25|3.78|0.00|234.36|246.76|263.50|18.74|0.00|234.36|253.10|-12.40| +2452175|38837|2349|98833|260008|4736|37464|4|253|1077|98|29.80|47.68|31.94|0.00|3130.12|2920.40|4672.64|93.90|0.00|3130.12|3224.02|209.72| +2452175|38837|4005|98833|260008|4736|37464|4|19|1077|36|36.45|60.87|53.56|0.00|1928.16|1312.20|2191.32|154.25|0.00|1928.16|2082.41|615.96| +2452175|38837|2665|98833|260008|4736|37464|4|84|1077|71|59.27|100.75|71.53|3961.33|5078.63|4208.17|7153.25|0.00|3961.33|1117.30|1117.30|-3090.87| +2452175|38837|12879|98833|260008|4736|37464|4|185|1077|66|6.87|8.10|3.48|0.00|229.68|453.42|534.60|9.18|0.00|229.68|238.86|-223.74| +2452175|38837|7751|98833|260008|4736|37464|4|60|1077|54|68.62|130.37|18.25|364.63|985.50|3705.48|7039.98|49.66|364.63|620.87|670.53|-3084.61| +2452175|38837|7647|98833|260008|4736|37464|4|286|1077|28|99.83|155.73|101.22|0.00|2834.16|2795.24|4360.44|85.02|0.00|2834.16|2919.18|38.92| +||11301|98833|260008|4736|37464|4|29|1077|82|12.27|15.58|14.95|0.00||1006.14|1277.56||0.00|1225.90||219.76| +2452175|38837|7545|98833|260008|4736|37464|4|148|1077|27|34.42|67.11|28.85|0.00|778.95|929.34|1811.97|0.00|0.00|778.95|778.95|-150.39| +2452175|38837|14983|98833|260008|4736|37464|4|114|1077|58|13.36|25.51|15.56|0.00|902.48|774.88|1479.58|81.22|0.00|902.48|983.70|127.60| +2452175|38837|1353|98833|260008|4736|37464|4|255|1077|78|3.20|3.87|3.48|0.00|271.44|249.60|301.86|5.42|0.00|271.44|276.86|21.84| +2452434|65418|3321|80132|795301|2470|839|2|286|1078|81|11.16|20.64|11.97|0.00|969.57|903.96|1671.84|0.00|0.00|969.57|969.57|65.61| +2452434|65418|10386|80132|795301|2470|839|2|18|1078|53|20.96|26.40|15.04|0.00|797.12|1110.88|1399.20|63.76|0.00|797.12|860.88|-313.76| +2452434|65418|5502|80132|795301|2470|839|2|263|1078|51|26.47|30.17|4.82|0.00|245.82|1349.97|1538.67|19.66|0.00|245.82|265.48|-1104.15| +2452434|65418|10563|80132|795301|2470|839|2|184|1078|43|71.81|79.70|40.64|0.00|1747.52|3087.83|3427.10|69.90|0.00|1747.52|1817.42|-1340.31| +2452434|65418|15360|80132|795301|2470|839|2|191|1078|62|3.36|3.79|2.16|0.00|133.92|208.32|234.98|5.35|0.00|133.92|139.27|-74.40| +2452434|65418|17593|80132|795301|2470|839|2|140|1078|2|20.16|34.07|7.83|0.00|15.66|40.32|68.14|0.15|0.00|15.66|15.81|-24.66| +|65418|3789|80132|795301|2470||||1078||13.78|18.18||0.00||1019.72||11.29|0.00|||| +2452434|65418|15181|80132|795301|2470|839|2|29|1078|37|25.13|33.42|16.71|0.00|618.27|929.81|1236.54|6.18|0.00|618.27|624.45|-311.54| +2452434|65418|12243|80132|795301|2470|839|2|176|1078|36|11.93|20.16|2.21|0.00|79.56|429.48|725.76|3.18|0.00|79.56|82.74|-349.92| +2452434|65418|9577|80132|795301|2470|839|2|96|1078|80|37.78|64.22|1.92|0.00|153.60|3022.40|5137.60|7.68|0.00|153.60|161.28|-2868.80| +2451175|74915|1958|76739|360664|6189|36114|4|108|1079|68|95.53|187.23|80.50|0.00|5474.00|6496.04|12731.64|0.00|0.00|5474.00|5474.00|-1022.04| +2451175|74915|9151|76739|360664|6189|36114|4|31|1079|42|26.51|40.03|4.80|139.10|201.60|1113.42|1681.26|5.00|139.10|62.50|67.50|-1050.92| +2451175|74915|91|76739|360664|6189|36114|4|89|1079|92|45.09|83.86|72.95|0.00|6711.40|4148.28|7715.12|134.22|0.00|6711.40|6845.62|2563.12| +2451175|74915|10772|76739|360664|6189|36114|4|107|1079|43|27.88|55.20|38.08|0.00|1637.44|1198.84|2373.60|130.99|0.00|1637.44|1768.43|438.60| +||2386|||6189|36114|||1079|96|47.51|89.79||||4560.96||20.67||||| +2451175||15274|||6189|36114|4||1079||13.81|21.12|16.47||148.23|124.29|190.08|1.77||29.65|31.42|| +2451175|74915|10285|76739|360664|6189|36114|4|85|1079|41|55.65|89.59|84.21|0.00|3452.61|2281.65|3673.19|138.10|0.00|3452.61|3590.71|1170.96| +2451175|74915|16723|76739|360664|6189|36114|4|253|1079|35|23.91|45.66|43.83|0.00|1534.05|836.85|1598.10|0.00|0.00|1534.05|1534.05|697.20| +2451175|74915|11995|76739|360664|6189|36114|4|262|1079|27|91.98|149.92|56.96|0.00|1537.92|2483.46|4047.84|92.27|0.00|1537.92|1630.19|-945.54| +2451175|74915|604|76739|360664|6189|36114|4|24|1079|56|77.60|94.67|63.42|0.00|3551.52|4345.60|5301.52|213.09|0.00|3551.52|3764.61|-794.08| +||10690||360664|6189||4|29|1079|||20.65|18.17||417.91||474.95|12.53||||40.94| +2451175|74915|565|76739|360664|6189|36114|4|86|1079|30|22.11|37.36|36.98|0.00|1109.40|663.30|1120.80|33.28|0.00|1109.40|1142.68|446.10| +2451175|74915|2431|76739|360664|6189|36114|4|103|1079|14|29.97|56.04|6.72|27.28|94.08|419.58|784.56|4.67|27.28|66.80|71.47|-352.78| +2451175|74915|14305|76739|360664|6189|36114|4|35|1079|10|54.62|65.54|25.56|0.00|255.60|546.20|655.40|15.33|0.00|255.60|270.93|-290.60| +2451175|74915|15331|76739|360664|6189|36114|4|287|1079|81|82.13|157.68|154.52|0.00|12516.12|6652.53|12772.08|0.00|0.00|12516.12|12516.12|5863.59| +2451175|74915|1076|76739|360664|6189|36114|4|83|1079|4|58.14|113.37|12.47|0.00|49.88|232.56|453.48|1.99|0.00|49.88|51.87|-182.68| +2450829|44523|6847|4627|1671984|6988|25192|4|190|1080|80|74.62|108.19|80.06|192.14|6404.80|5969.60|8655.20|559.13|192.14|6212.66|6771.79|243.06| +2450829|44523|9658|4627|1671984|6988|25192|4|26|1080|99|79.16|92.61|57.41|3353.31|5683.59|7836.84|9168.39|46.60|3353.31|2330.28|2376.88|-5506.56| +2450829|44523|14329|4627|1671984|6988|25192|4|125|1080|45|55.39|74.22|69.76|2009.08|3139.20|2492.55|3339.90|90.40|2009.08|1130.12|1220.52|-1362.43| +2450829|44523|11470|4627|1671984|6988|25192|4|123|1080|86|25.18|33.48|19.75|594.47|1698.50|2165.48|2879.28|44.16|594.47|1104.03|1148.19|-1061.45| +2450829|44523|214|4627|1671984|6988|25192|4|67|1080|27|55.33|61.96|22.92|0.00|618.84|1493.91|1672.92|6.18|0.00|618.84|625.02|-875.07| +2450829|44523|5977|4627|1671984|6988|25192|4|281|1080|52|84.34|113.85|19.35|0.00|1006.20|4385.68|5920.20|60.37|0.00|1006.20|1066.57|-3379.48| +2450829|44523|15967|4627|1671984|6988|25192|4|47|1080|63|58.97|94.35|85.85|0.00|5408.55|3715.11|5944.05|108.17|0.00|5408.55|5516.72|1693.44| +2450829|44523|17281|4627|1671984|6988|25192|4|198|1080|48|17.21|23.40|0.93|0.00|44.64|826.08|1123.20|4.01|0.00|44.64|48.65|-781.44| +2450829|44523|4654|4627|1671984|6988|25192|4|140|1080|25|54.53|55.62|33.92|839.52|848.00|1363.25|1390.50|0.33|839.52|8.48|8.81|-1354.77| +2450829|44523|12928|4627|1671984|6988|25192|4|133|1080|93|8.46|13.95|5.44|0.00|505.92|786.78|1297.35|45.53|0.00|505.92|551.45|-280.86| +2450829|44523|4651|4627|1671984|6988|25192|4|63|1080|97|42.26|81.98|50.00|1455.00|4850.00|4099.22|7952.06|271.60|1455.00|3395.00|3666.60|-704.22| +2450829|44523|16516||1671984||||206|1080||90.05||37.31|||3782.10||||||| +2450829|44523|16700|4627|1671984|6988|25192|4|66|1080|35|76.67|84.33|46.38|0.00|1623.30|2683.45|2951.55|64.93|0.00|1623.30|1688.23|-1060.15| +2451426|62784|1298|874|1534265|4456|27800|8|63|1081|63|18.33|27.67|11.06|0.00|696.78|1154.79|1743.21|13.93|0.00|696.78|710.71|-458.01| +2451426|62784|17078|874|1534265|4456|27800|8|243|1081|67|85.59|88.15|53.77|0.00|3602.59|5734.53|5906.05|144.10|0.00|3602.59|3746.69|-2131.94| +2451426|62784|5084|874|1534265|4456|27800|8|109|1081|90|17.49|28.50|21.94|0.00|1974.60|1574.10|2565.00|59.23|0.00|1974.60|2033.83|400.50| +2451426|62784|16544|874|1534265|4456|27800|8|140|1081|22|11.91|14.53|8.71|170.54|191.62|262.02|319.66|1.47|170.54|21.08|22.55|-240.94| +2451426|62784|7984|874|1534265|4456|27800|8|123|1081|30|14.22|17.06|11.25|0.00|337.50|426.60|511.80|16.87|0.00|337.50|354.37|-89.10| +2451426||7183||1534265|4456|27800|||1081||76.36|101.55|10.15|0.00|||||0.00|233.45||| +||13696|874|1534265||27800||206|1081||89.64|159.55|74.98|0.00|4648.76|5557.68||418.38|0.00|4648.76||-908.92| +2451426|62784|14902|874|1534265|4456|27800|8|226|1081|44|22.48|43.38|42.07|0.00|1851.08|989.12|1908.72|0.00|0.00|1851.08|1851.08|861.96| +2451426|62784|2858|874|1534265|4456|27800|8|83|1081|32|33.77|53.35|0.53|0.00|16.96|1080.64|1707.20|1.52|0.00|16.96|18.48|-1063.68| +|62784|12673|||4456|27800||98|1081|||3.35|2.41||||90.45|0.65||65.07||| +2451426|62784|1|874|1534265|4456|27800|8|187|1081|97|19.83|21.21|16.11|0.00|1562.67|1923.51|2057.37|140.64|0.00|1562.67|1703.31|-360.84| +2451537|66571|15454|36896|1041042|388|42181|8|161|1082|6|63.33|116.52|113.02|61.03|678.12|379.98|699.12|6.17|61.03|617.09|623.26|237.11| +2451537|66571|12206|36896|1041042|388|42181|8|201|1082|83|96.58|190.26|133.18|0.00|11053.94|8016.14|15791.58|773.77|0.00|11053.94|11827.71|3037.80| +2451537|66571|16994|36896|1041042|388|42181|8|166|1082|28|51.49|88.56|48.70|981.79|1363.60|1441.72|2479.68|19.09|981.79|381.81|400.90|-1059.91| +2451537|66571|14024|36896|1041042|388|42181|8|161|1082|58|75.13|77.38|23.21|323.08|1346.18|4357.54|4488.04|20.46|323.08|1023.10|1043.56|-3334.44| +2451537|66571|14353|36896|1041042|388|42181|8|256|1082|46|30.24|38.70|26.31|0.00|1210.26|1391.04|1780.20|24.20|0.00|1210.26|1234.46|-180.78| +2451537|66571|9199|36896|1041042|388|42181|8|38|1082|78|89.24|165.98|54.77|0.00|4272.06|6960.72|12946.44|170.88|0.00|4272.06|4442.94|-2688.66| +2451537|66571|12466|36896|1041042|388|42181|8|53|1082|24|38.22|69.56|55.64|0.00|1335.36|917.28|1669.44|80.12|0.00|1335.36|1415.48|418.08| +2451537|66571|13153|36896|1041042|388|42181|8|256|1082|87|69.03|71.79|20.10|0.00|1748.70|6005.61|6245.73|34.97|0.00|1748.70|1783.67|-4256.91| +2451537|66571|2818|36896|1041042|388|42181|8|56|1082|85|62.76|119.24|44.11|0.00|3749.35|5334.60|10135.40|37.49|0.00|3749.35|3786.84|-1585.25| +2451537|66571|11258|36896|1041042|388|42181|8|10|1082|53|6.37|7.26|4.42|0.00|234.26|337.61|384.78|0.00|0.00|234.26|234.26|-103.35| +2451537|66571|17798|36896|1041042|388|42181|8|216|1082|43|16.18|30.90|7.41|0.00|318.63|695.74|1328.70|12.74|0.00|318.63|331.37|-377.11| +2451537|66571|2828|36896|1041042|388|42181|8|292|1082|82|10.90|19.07|16.20|0.00|1328.40|893.80|1563.74|92.98|0.00|1328.40|1421.38|434.60| +2451537|66571|5971|36896|1041042|388|42181|8|226|1082|83|6.20|8.06|4.75|0.00|394.25|514.60|668.98|0.00|0.00|394.25|394.25|-120.35| +2451086|49810|374|47431|1906348|2418|27335|1|114|1083|96|74.69|91.86|18.37|0.00|1763.52|7170.24|8818.56|35.27|0.00|1763.52|1798.79|-5406.72| +2451086|49810|14557|47431|1906348|2418|27335|1|3|1083|65|72.28|91.79|51.40|0.00|3341.00|4698.20|5966.35|100.23|0.00|3341.00|3441.23|-1357.20| +2451086|49810|3320|47431|1906348|2418|27335|1|159|1083|78|38.76|76.35|34.35|0.00|2679.30|3023.28|5955.30|0.00|0.00|2679.30|2679.30|-343.98| +2451086|49810|10384|47431|1906348|2418|27335|1|166|1083|79|78.73|151.16|46.85|0.00|3701.15|6219.67|11941.64|111.03|0.00|3701.15|3812.18|-2518.52| +2451086|49810|5500|47431|1906348|2418|27335|1|174|1083|23|34.12|65.16|62.55|0.00|1438.65|784.76|1498.68|86.31|0.00|1438.65|1524.96|653.89| +2451086|49810|10562|47431|1906348|2418|27335|1|280|1083|61|54.74|82.65|0.82|0.00|50.02|3339.14|5041.65|4.50|0.00|50.02|54.52|-3289.12| +2451086|49810|15358|47431|1906348|2418|27335|1|27|1083|93|42.21|48.96|32.31|0.00|3004.83|3925.53|4553.28|150.24|0.00|3004.83|3155.07|-920.70| +2451086|49810|17593|47431|1906348|2418|27335|1|204|1083|85|54.27|83.57|32.59|0.00|2770.15|4612.95|7103.45|138.50|0.00|2770.15|2908.65|-1842.80| +2451086|49810|3788|47431|1906348|2418|27335|1|23|1083|47|88.24|176.48|174.71|6322.75|8211.37|4147.28|8294.56|18.88|6322.75|1888.62|1907.50|-2258.66| +2451086|49810|15181|47431|1906348|2418|27335|1|111|1083|42|26.21|45.60|11.85|0.00|497.70|1100.82|1915.20|24.88|0.00|497.70|522.58|-603.12| +2451086|49810|12242|47431|1906348|2418|27335|1|107|1083|49|27.05|43.00|14.62|0.00|716.38|1325.45|2107.00|21.49|0.00|716.38|737.87|-609.07| +2451086|49810|9577|47431|1906348|2418|27335|1|74|1083|95|52.02|99.35|29.80|1330.57|2831.00|4941.90|9438.25|135.03|1330.57|1500.43|1635.46|-3441.47| +2451086|49810|5030|47431|1906348|2418|27335|1|109|1083|78|41.31|82.62|56.18|0.00|4382.04|3222.18|6444.36|219.10|0.00|4382.04|4601.14|1159.86| +2451086|49810|550|47431|1906348|2418|27335|1|223|1083|70|41.69|52.52|47.79|0.00|3345.30|2918.30|3676.40|66.90|0.00|3345.30|3412.20|427.00| +2451086|49810|7324|47431|1906348|2418|27335|1|210|1083|86|37.31|56.33|50.13|0.00|4311.18|3208.66|4844.38|388.00|0.00|4311.18|4699.18|1102.52| +2451086|49810|11866|47431|1906348|2418|27335|1|123|1083|95|15.28|16.80|15.96|0.00|1516.20|1451.60|1596.00|75.81|0.00|1516.20|1592.01|64.60| +2451524|39357|11797|21877|88112|1100|20535|2|92|1084|4|58.47|97.06|68.91|275.64|275.64|233.88|388.24|0.00|275.64|0.00|0.00|-233.88| +2451524|39357|4990|21877|88112|1100|20535|2|279|1084|69|61.79|90.83|19.07|0.00|1315.83|4263.51|6267.27|52.63|0.00|1315.83|1368.46|-2947.68| +2451524||12859|21877|88112|1100|20535|2||1084|18|11.12|||0.00|145.98|200.16|292.14|11.67|0.00|145.98||| +2451524|39357|6235|21877|88112|1100|20535|2|50|1084|80|22.55|30.66|17.78|1351.28|1422.40|1804.00|2452.80|4.97|1351.28|71.12|76.09|-1732.88| +2451524||8497|21877|||20535|2||1084|55||138.30||0.00||5357.00|||0.00||5248.10|| +2451524|39357|5728|21877|88112|1100|20535|2|109|1084|35|79.42|105.62|86.60|0.00|3031.00|2779.70|3696.70|151.55|0.00|3031.00|3182.55|251.30| +2451524|39357|7987|21877|88112|1100|20535|2|16|1084|98|25.01|43.26|4.75|0.00|465.50|2450.98|4239.48|18.62|0.00|465.50|484.12|-1985.48| +2451524|39357|5300|21877|88112|1100|20535|2|6|1084|100|72.07|85.04|56.97|0.00|5697.00|7207.00|8504.00|284.85|0.00|5697.00|5981.85|-1510.00| +2451524|39357|2860|21877|88112|1100|20535|2|153|1084|92|92.65|129.71|86.90|0.00|7994.80|8523.80|11933.32|479.68|0.00|7994.80|8474.48|-529.00| +2451430|44165|12842|80401|1458998|3995|19792|4|212|1085|78|24.08|34.43|27.19|0.00|2120.82|1878.24|2685.54|42.41|0.00|2120.82|2163.23|242.58| +2451430|44165|5719|80401|1458998|3995|19792|4|171|1085|77|78.92|122.32|59.93|2907.20|4614.61|6076.84|9418.64|136.59|2907.20|1707.41|1844.00|-4369.43| +2451430|44165|8834|80401|1458998|3995|19792|4|94|1085|74|65.66|108.33|20.58|0.00|1522.92|4858.84|8016.42|60.91|0.00|1522.92|1583.83|-3335.92| +2451430|44165|14317|80401|1458998|3995|19792|4|279|1085|50|38.30|38.68|9.67|0.00|483.50|1915.00|1934.00|9.67|0.00|483.50|493.17|-1431.50| +2451430|44165|3836|80401|1458998|3995|19792|4|51|1085|70|96.07|179.65|10.77|595.58|753.90|6724.90|12575.50|0.00|595.58|158.32|158.32|-6566.58| +2451430|44165|5041|80401|1458998|3995|19792|4|291|1085|94|21.94|24.79|14.37|1323.76|1350.78|2062.36|2330.26|2.43|1323.76|27.02|29.45|-2035.34| +2451430|44165|4534|80401|1458998|3995|19792|4|19|1085|17|41.38|82.76|0.00|0.00|0.00|703.46|1406.92|0.00|0.00|0.00|0.00|-703.46| +2451430|44165|4594|80401|1458998|3995|19792|4|226|1085|92|55.06|98.55|96.57|2221.11|8884.44|5065.52|9066.60|199.89|2221.11|6663.33|6863.22|1597.81| +2451080|48773|1844|36728|9252|3023|7594|2|228|1086|75|97.35|159.65|79.82|2993.25|5986.50|7301.25|11973.75|239.46|2993.25|2993.25|3232.71|-4308.00| +2451080|48773|14792|36728|9252|3023|7594|2|166|1086|29|4.29|6.43|1.15|8.33|33.35|124.41|186.47|0.00|8.33|25.02|25.02|-99.39| +2451080|48773|8173|36728|9252|3023|7594|2|65|1086|16|90.28|152.57|102.22|0.00|1635.52|1444.48|2441.12|32.71|0.00|1635.52|1668.23|191.04| +2451080|48773|3986|36728|9252|3023|7594|2|292|1086|11||94.46|||103.84||1039.06|7.26|||111.10|| +2451080|48773|12763|36728|9252|3023|7594|2|93|1086|95|99.74|156.59|120.57|0.00|11454.15|9475.30|14876.05|687.24|0.00|11454.15|12141.39|1978.85| +2451080|48773|13742|36728|9252|3023|7594|2|73|1086|66|64.96|85.09|34.03|0.00|2245.98|4287.36|5615.94|112.29|0.00|2245.98|2358.27|-2041.38| +2451080|48773|2720|36728|9252|3023|7594|2|227|1086|64|61.73|91.97|41.38|1933.27|2648.32|3950.72|5886.08|7.15|1933.27|715.05|722.20|-3235.67| +2451080|48773|2024|36728|9252|3023|7594|2|185|1086|81|93.80|141.63|29.74|0.00|2408.94|7597.80|11472.03|24.08|0.00|2408.94|2433.02|-5188.86| +2451080|48773|16015|36728|9252|3023|7594|2|259|1086|1|27.84|40.36|5.24|0.00|5.24|27.84|40.36|0.47|0.00|5.24|5.71|-22.60| +2451080|48773|17521|36728|9252|3023|7594|2|24|1086|53|65.98|77.19|20.06|0.00|1063.18|3496.94|4091.07|63.79|0.00|1063.18|1126.97|-2433.76| +2451080|48773|9079|36728|9252|3023|7594|2|97|1086|83|35.66|68.11|27.24|0.00|2260.92|2959.78|5653.13|67.82|0.00|2260.92|2328.74|-698.86| +2451080|48773|6088|36728|9252|3023|7594|2|283|1086|78|64.92|92.83|79.83|0.00|6226.74|5063.76|7240.74|62.26|0.00|6226.74|6289.00|1162.98| +2451080|48773|16840|36728|9252|3023|7594|2|196|1086|96|67.63|86.56|35.48|0.00|3406.08|6492.48|8309.76|170.30|0.00|3406.08|3576.38|-3086.40| +2451080|48773|12688|36728|9252|3023|7594|2|104|1086|77|66.28|107.37|38.65|0.00|2976.05|5103.56|8267.49|59.52|0.00|2976.05|3035.57|-2127.51| +2451080|48773|4825|36728|9252|3023|7594|2|49|1086|75|88.88|160.87|65.95|2522.58|4946.25|6666.00|12065.25|72.71|2522.58|2423.67|2496.38|-4242.33| +2451424|37514|11956||||31331||78|1087|64|||67.60|||6070.40||||||-1744.00| +2451424|37514|7100|76809|999383|1957|31331|4|257|1087|91|26.63|43.67|40.61|0.00|3695.51|2423.33|3973.97|0.00|0.00|3695.51|3695.51|1272.18| +2451424|37514|11632|76809|999383|1957|31331|4|175|1087|52|89.00|175.33|40.32|0.00|2096.64|4628.00|9117.16|0.00|0.00|2096.64|2096.64|-2531.36| +2451424|37514|15484|76809|999383|1957|31331|4|82|1087|87|33.65|53.16|47.84|0.00|4162.08|2927.55|4624.92|208.10|0.00|4162.08|4370.18|1234.53| +2451424|37514|12418|76809|999383|1957|31331|4|282|1087|53|75.15|121.74|45.04|1814.21|2387.12|3982.95|6452.22|5.72|1814.21|572.91|578.63|-3410.04| +2451424|37514|13306|76809|999383|1957|31331|4|234|1087|40|34.15|47.46|40.81|0.00|1632.40|1366.00|1898.40|65.29|0.00|1632.40|1697.69|266.40| +2451424|37514|17066|76809|999383|1957|31331|4|106|1087|75|69.82|113.10|6.78|0.00|508.50|5236.50|8482.50|25.42|0.00|508.50|533.92|-4728.00| +2451424|37514|10322|76809|999383|1957|31331|4|11|1087|83|83.36|130.04|81.92|0.00|6799.36|6918.88|10793.32|475.95|0.00|6799.36|7275.31|-119.52| +2451424|37514|12992|76809|999383|1957|31331|4|245|1087|72|10.84|20.05|0.80|0.00|57.60|780.48|1443.60|2.30|0.00|57.60|59.90|-722.88| +2451900|53636|5383|28452|1573599|4346|49846|1|48|1088|24|7.12|11.53|3.92|0.00|94.08|170.88|276.72|3.76|0.00|94.08|97.84|-76.80| +2451900|53636|7829|28452|1573599|4346|49846|1|5|1088|74|22.84|44.53|37.40|0.00|2767.60|1690.16|3295.22|138.38|0.00|2767.60|2905.98|1077.44| +2451900|53636|12349|28452|1573599|4346|49846|1|259|1088|25|37.01|38.86|36.52|0.00|913.00|925.25|971.50|9.13|0.00|913.00|922.13|-12.25| +2451900|53636|3109|28452|1573599|4346|49846|1|138|1088|60|14.77|17.72|7.97|0.00|478.20|886.20|1063.20|38.25|0.00|478.20|516.45|-408.00| +2451900|53636|2435|28452|1573599|4346|49846|1|137|1088|2|82.41|149.98|100.48|0.00|200.96|164.82|299.96|14.06|0.00|200.96|215.02|36.14| +2451900|53636|512|||||1|102|1088||17.98|32.72||0.00||1078.80|1963.20||0.00|||334.20| +2451900|53636|5747|28452|1573599|4346|49846|1|217|1088|88|40.74|72.51|21.02|0.00|1849.76|3585.12|6380.88|129.48|0.00|1849.76|1979.24|-1735.36| +2451900|53636|13028|28452|1573599|4346|49846|1|169|1088|64|82.54|89.14|76.66|0.00|4906.24|5282.56|5704.96|0.00|0.00|4906.24|4906.24|-376.32| +2451900|53636|15601|28452|1573599|4346|49846|1|300|1088|69|15.92|30.56|18.33|0.00|1264.77|1098.48|2108.64|63.23|0.00|1264.77|1328.00|166.29| +2451900|53636|14851|28452|1573599|4346|49846|1|103|1088|2|87.13|123.72|35.87|0.00|71.74|174.26|247.44|0.71|0.00|71.74|72.45|-102.52| +2451785|50078|1645|37983|736094|4958|26269|10|83|1089|63|56.14|88.13|85.48|3823.52|5385.24|3536.82|5552.19|15.61|3823.52|1561.72|1577.33|-1975.10| +2451785|50078|12853|37983|736094|4958|26269|10|68|1089|38|87.04|103.57|51.78|295.14|1967.64|3307.52|3935.66|150.52|295.14|1672.50|1823.02|-1635.02| +2451785|50078|8621|37983|736094|4958|26269|10|272|1089|37|46.14|51.67|29.96|0.00|1108.52|1707.18|1911.79|88.68|0.00|1108.52|1197.20|-598.66| +2451785|50078|3049|37983|736094|4958|26269|10|92|1089|28|44.93|81.77|12.26|209.40|343.28|1258.04|2289.56|4.01|209.40|133.88|137.89|-1124.16| +2451785|50078|2549|37983|736094|4958|26269|10|241|1089|69|6.36|11.76|6.58|0.00|454.02|438.84|811.44|31.78|0.00|454.02|485.80|15.18| +2451785|50078|10895|37983|736094|4958|26269|10|1|1089|37|77.90|126.97|79.99|0.00|2959.63|2882.30|4697.89|207.17|0.00|2959.63|3166.80|77.33| +2451785|50078|11705|37983|736094|4958|26269|10|28|1089|27|89.41|136.79|91.64|0.00|2474.28|2414.07|3693.33|148.45|0.00|2474.28|2622.73|60.21| +2451785|50078|13513|37983|736094|4958|26269|10|77|1089|38|25.64|26.40|19.27|329.51|732.26|974.32|1003.20|0.00|329.51|402.75|402.75|-571.57| +2451785|50078|3290|37983|736094|4958|26269|10|138|1089|5|7.05|8.67|0.52|0.00|2.60|35.25|43.35|0.02|0.00|2.60|2.62|-32.65| +2451785|50078|13244|37983|736094|4958|26269|10|136|1089|46|55.87|106.71|24.54|0.00|1128.84|2570.02|4908.66|79.01|0.00|1128.84|1207.85|-1441.18| +2451073|52382|1058|69371|24469|5167|33065|10|297|1090|20|57.12|70.82|24.07|0.00|481.40|1142.40|1416.40|19.25|0.00|481.40|500.65|-661.00| +2451073||9584||24469|||10||1090|64|60.47|||||3870.08||493.45||5482.88|5976.33|| +2451073|52382|9196|69371|24469|5167|33065|10|141|1090|63|53.31|85.82|30.89|0.00|1946.07|3358.53|5406.66|155.68|0.00|1946.07|2101.75|-1412.46| +2451073|52382|2032|69371|24469|5167|33065|10|120|1090|1|70.30|111.77|84.94|0.00|84.94|70.30|111.77|2.54|0.00|84.94|87.48|14.64| +2451073|52382|10186|69371|24469|5167|33065|10|292|1090|64|26.36|32.68|6.53|0.00|417.92|1687.04|2091.52|20.89|0.00|417.92|438.81|-1269.12| +2451073|52382|1636|69371|24469|5167|33065|10|248|1090|93|52.39|104.78|85.91|0.00|7989.63|4872.27|9744.54|159.79|0.00|7989.63|8149.42|3117.36| +2451073|52382|11917|69371|24469|5167|33065|10|207|1090|1|97.42|104.23|104.23|0.00|104.23|97.42|104.23|5.21|0.00|104.23|109.44|6.81| +2451073|52382|15460|69371|24469|5167|33065|10|295|1090|94|63.60|68.68|62.49|3406.95|5874.06|5978.40|6455.92|24.67|3406.95|2467.11|2491.78|-3511.29| +2451073|52382|11779|69371|24469|5167|33065|10|150|1090|52|7.67|11.12|9.56|0.00|497.12|398.84|578.24|0.00|0.00|497.12|497.12|98.28| +2451073|52382|16012|69371|24469|5167|33065|10|1|1090|86|37.51|48.38|18.38|1106.47|1580.68|3225.86|4160.68|14.22|1106.47|474.21|488.43|-2751.65| +2451073|52382|13726|69371|24469|5167|33065|10|70|1090|20|82.46|163.27|86.53|0.00|1730.60|1649.20|3265.40|0.00|0.00|1730.60|1730.60|81.40| +2451073|52382|16502|69371|24469|5167|33065|10|131|1090|16|40.83|64.51|45.15|0.00|722.40|653.28|1032.16|50.56|0.00|722.40|772.96|69.12| +2451073|52382|4057|69371|24469|5167|33065|10|8|1090|40|66.86|101.62|67.06|429.18|2682.40|2674.40|4064.80|157.72|429.18|2253.22|2410.94|-421.18| +||6454|69371|24469|5167|33065|10|103|1090|||17.36|14.06|0.00||1152.61|||0.00||1199.59|-41.87| +2451161|30313|14504|46262|715985|1355|33478|7|159|1091|79|6.30|11.02|8.59|0.00|678.61|497.70|870.58|0.00|0.00|678.61|678.61|180.91| +||10864|||1355|33478|||1091|7|||43.83|0.00|306.81||639.31|0.00|0.00|||-19.39| +2451161|30313|11204|46262||||7||1091|100||12.60|10.58||1058.00|1248.00||29.09|||610.99|| +2451161|30313|14246|46262|715985|1355|33478|7|232|1091|76|83.19|114.80|51.66|0.00|3926.16|6322.44|8724.80|235.56|0.00|3926.16|4161.72|-2396.28| +||7474|46262|||33478|7|117|1091||89.11||149.70|||8465.45|14814.30|142.21||||5756.05| +2451161|30313|17060|46262|715985|1355|33478|7|173|1091|97|87.30|147.53|89.99|0.00|8729.03|8468.10|14310.41|698.32|0.00|8729.03|9427.35|260.93| +2451161|30313|14078|46262|715985|1355|33478|7|184|1091|21|4.54|5.03|4.67|39.22|98.07|95.34|105.63|1.17|39.22|58.85|60.02|-36.49| +2451161|30313|1792|46262|715985|1355|33478|7|195|1091|100|51.15|83.37|10.00|520.00|1000.00|5115.00|8337.00|19.20|520.00|480.00|499.20|-4635.00| +2451161|30313|16303|46262|715985|1355|33478|7|20|1091|28|21.59|25.26|24.24|115.38|678.72|604.52|707.28|11.26|115.38|563.34|574.60|-41.18| +2451161|30313|14713|46262|715985|1355|33478|7|181|1091|59|36.55|45.68|34.71|0.00|2047.89|2156.45|2695.12|163.83|0.00|2047.89|2211.72|-108.56| +2451161||772|||1355|33478|||1091|13|10.27|20.54|6.77|0.00|88.01||267.02|7.92|0.00|||| +2451161|30313|12394|46262|715985|1355|33478|7|54|1091|69|81.70|103.75|12.45|283.48|859.05|5637.30|7158.75|51.80|283.48|575.57|627.37|-5061.73| +2451161|30313|9874|46262|715985|1355|33478|7|221|1091|33|48.73|59.45|46.96|1115.76|1549.68|1608.09|1961.85|34.71|1115.76|433.92|468.63|-1174.17| +2451161|30313|3637|46262|715985|1355|33478|7|280|1091|53|73.75|125.37|60.17|829.14|3189.01|3908.75|6644.61|141.59|829.14|2359.87|2501.46|-1548.88| +2451161|30313|12590|46262|715985|1355|33478|7|176|1091|7|55.76|89.21|38.36|0.00|268.52|390.32|624.47|10.74|0.00|268.52|279.26|-121.80| +2451161|30313|13294|46262|715985|1355|33478|7|10|1091|6|10.99|21.21|1.27|3.04|7.62|65.94|127.26|0.27|3.04|4.58|4.85|-61.36| +2452249|52898|16195|84808|1425454|1300|7146|2|7|1092|60|67.83|109.20|84.08|504.48|5044.80|4069.80|6552.00|227.01|504.48|4540.32|4767.33|470.52| +2452249|52898|15939|84808|1425454|1300|7146|2|206|1092|14|39.90|57.85|49.75|0.00|696.50|558.60|809.90|34.82|0.00|696.50|731.32|137.90| +2452249|52898|5237|84808|1425454|1300|7146|2|121|1092|31|89.09|92.65|57.44|0.00|1780.64|2761.79|2872.15|89.03|0.00|1780.64|1869.67|-981.15| +2452249|52898|3|84808|1425454|1300|7146|2|96|1092|64|78.01|105.31|26.32|1145.44|1684.48|4992.64|6739.84|43.12|1145.44|539.04|582.16|-4453.60| +2452249|52898|14885||1425454|1300||2|3|1092|46|46.40|86.76|21.69|0.00||2134.40|||0.00|997.74|1037.64|| +2452249|52898|5271|84808|1425454|1300|7146|2|198|1092|11|71.28|135.43|98.86|250.11|1087.46|784.08|1489.73|75.36|250.11|837.35|912.71|53.27| +2452249|52898|123|84808|1425454|1300|7146|2|281|1092|89|11.78|18.25|7.48|0.00|665.72|1048.42|1624.25|13.31|0.00|665.72|679.03|-382.70| +2452249|52898|5109|84808|1425454|1300|7146|2|152|1092|28|25.48|42.04|21.02|0.00|588.56|713.44|1177.12|5.88|0.00|588.56|594.44|-124.88| +2452249|52898|5893|84808|1425454|1300|7146|2|232|1092|83|27.22|37.29|30.95|1361.49|2568.85|2259.26|3095.07|60.36|1361.49|1207.36|1267.72|-1051.90| +2452249|52898|15901|84808|1425454|1300|7146|2|238|1092|1|17.40|22.44|3.36|0.00|3.36|17.40|22.44|0.26|0.00|3.36|3.62|-14.04| +2452249|52898|4417|84808|1425454|1300|7146|2|225|1092|17|88.51|159.31|74.87|1107.32|1272.79|1504.67|2708.27|9.92|1107.32|165.47|175.39|-1339.20| +2452249|52898|15737|84808|1425454|1300|7146|2|98|1092|47|10.97|15.57|8.40|379.00|394.80|515.59|731.79|0.63|379.00|15.80|16.43|-499.79| +2452249|52898|11129|84808|1425454|1300|7146|2|151|1092|20|16.59|19.74|12.83|0.00|256.60|331.80|394.80|0.00|0.00|256.60|256.60|-75.20| +2452249|52898|8093|84808|1425454|1300|7146|2|202|1092|76|44.59|82.49|18.14|0.00|1378.64|3388.84|6269.24|27.57|0.00|1378.64|1406.21|-2010.20| +2452249|52898|10445|84808|1425454|1300|7146|2|75|1092|66|10.85|14.64|12.00|87.12|792.00|716.10|966.24|28.19|87.12|704.88|733.07|-11.22| +2452249|52898|11751|84808|1425454|1300|7146|2|178|1092|25|41.88|44.81|39.88|0.00|997.00|1047.00|1120.25|89.73|0.00|997.00|1086.73|-50.00| +2451059|52167|9574|54240|1530197|2154|17311|10|234|1093|61|15.15|28.48|22.49|0.00|1371.89|924.15|1737.28|109.75|0.00|1371.89|1481.64|447.74| +2451059|52167|3697|54240|1530197|2154|17311|10|86|1093|27|33.21|62.10|50.92|0.00|1374.84|896.67|1676.70|123.73|0.00|1374.84|1498.57|478.17| +2451059|52167|11102|54240|1530197|2154|17311|10|219|1093|95|64.75|111.37|23.38|0.00|2221.10|6151.25|10580.15|111.05|0.00|2221.10|2332.15|-3930.15| +2451059|52167|8635|54240|1530197|2154|17311|10|81|1093|88|17.74|32.28|4.51|0.00|396.88|1561.12|2840.64|19.84|0.00|396.88|416.72|-1164.24| +2451059|52167|14515|54240|1530197|2154|17311|10|209|1093|69|83.10|96.39|4.81|185.85|331.89|5733.90|6650.91|2.92|185.85|146.04|148.96|-5587.86| +2451059|52167|6032|54240|1530197|2154|17311|10|37|1093|70|37.79|56.68|12.46|0.00|872.20|2645.30|3967.60|0.00|0.00|872.20|872.20|-1773.10| +2451059|52167|17935|54240|1530197|2154|17311|10|212|1093|21|27.87|49.32|37.48|0.00|787.08|585.27|1035.72|62.96|0.00|787.08|850.04|201.81| +2451059|52167|3398|54240|1530197|2154|17311|10|70|1093|36|35.80|69.09|33.16|0.00|1193.76|1288.80|2487.24|23.87|0.00|1193.76|1217.63|-95.04| +2451059|52167|4195|54240|1530197|2154|17311|10|37|1093|18|96.80|158.75|53.97|738.30|971.46|1742.40|2857.50|6.99|738.30|233.16|240.15|-1509.24| +2451059|52167|13231|54240|1530197|2154|17311|10|281|1093|28|49.36|71.07|3.55|12.92|99.40|1382.08|1989.96|0.00|12.92|86.48|86.48|-1295.60| +2451059|52167|2098|54240|1530197|2154|17311|10|82|1093|46|38.01|58.15|51.17|0.00|2353.82|1748.46|2674.90|211.84|0.00|2353.82|2565.66|605.36| +2451059|52167|9274|54240|1530197|2154|17311|10|297|1093|80|55.85|99.41|57.65|2905.56|4612.00|4468.00|7952.80|136.51|2905.56|1706.44|1842.95|-2761.56| +2451059|52167|10288|54240|1530197|2154|17311|10|279|1093|34|50.07|97.13|61.19|0.00|2080.46|1702.38|3302.42|187.24|0.00|2080.46|2267.70|378.08| +2451059|52167|5858|54240|1530197|2154|17311|10|156|1093|72|8.82|11.81|5.90|0.00|424.80|635.04|850.32|25.48|0.00|424.80|450.28|-210.24| +2451059|52167|10108|54240|1530197|2154|17311|10|210|1093|83|40.32|65.31|46.37|0.00|3848.71|3346.56|5420.73|269.40|0.00|3848.71|4118.11|502.15| +2451059|52167|9997|54240|1530197|2154|17311|10|38|1093|53|98.73|196.47|74.65|0.00|3956.45|5232.69|10412.91|316.51|0.00|3956.45|4272.96|-1276.24| +2451446|51873|4081|78482|193469|6056|16243|4|134|1094|71|15.31|16.68|9.67|0.00|686.57|1087.01|1184.28|34.32|0.00|686.57|720.89|-400.44| +2451446|51873|2521|78482|193469|6056|16243|4|149|1094|84|51.93|83.60|65.20|0.00|5476.80|4362.12|7022.40|164.30|0.00|5476.80|5641.10|1114.68| +2451446|51873|9628|78482|193469|6056|16243|4|219|1094|66|73.81|123.26|104.77|0.00|6914.82|4871.46|8135.16|207.44|0.00|6914.82|7122.26|2043.36| +2451446|51873|14206|78482|193469|6056|16243|4|58|1094|92|14.62|24.12|10.37|57.24|954.04|1345.04|2219.04|53.80|57.24|896.80|950.60|-448.24| +2451446|51873|10711|78482|193469|6056|16243|4|113|1094|47|11.42|15.64|3.12|0.00|146.64|536.74|735.08|13.19|0.00|146.64|159.83|-390.10| +2451446|51873|14062|78482|193469|6056|16243|4|242|1094|11|30.49|57.62|14.40|0.00|158.40|335.39|633.82|11.08|0.00|158.40|169.48|-176.99| +2451446|51873|10027|78482|193469|6056|16243|4|106|1094|69|46.97|53.07|41.92|0.00|2892.48|3240.93|3661.83|0.00|0.00|2892.48|2892.48|-348.45| +||2008||||16243|4|99|1094||24.18|38.92||||||16.25||541.68||| +2451446|51873|16372|78482|193469|6056|16243|4|102|1094|74|80.91|87.38|15.72|0.00|1163.28|5987.34|6466.12|69.79|0.00|1163.28|1233.07|-4824.06| +2451446|51873|1204|78482|193469|6056|16243|4|130|1094|67|51.78|69.90|7.68|0.00|514.56|3469.26|4683.30|15.43|0.00|514.56|529.99|-2954.70| +2451446|51873|13492|78482|193469|6056|16243|4|86|1094|96|28.67|55.61|52.27|0.00|5017.92|2752.32|5338.56|0.00|0.00|5017.92|5017.92|2265.60| +2451446|51873|8482|78482|193469|6056|16243|4|19|1094|10|84.37|86.05|36.14|0.00|361.40|843.70|860.50|14.45|0.00|361.40|375.85|-482.30| +2451446|51873|2113|78482|193469|6056|16243|4|93|1094|9|39.30|68.77|7.56|0.00|68.04|353.70|618.93|1.36|0.00|68.04|69.40|-285.66| +|51873|13616|78482||6056||4|217|1094|3||67.35|47.81||143.43|102.57|202.05|4.30||143.43|147.73|| +2451446|51873|4358|78482|193469|6056|16243|4|218|1094|1|61.74|112.98|107.33|105.18|107.33|61.74|112.98|0.04|105.18|2.15|2.19|-59.59| +2451446|51873|13921||||16243||99|1094|||82.73|57.91||||4301.96||||3011.32|44.20| +2452638|37697|17121|6837|975590|4790||2|13|1095||87.10||49.21|114.16|196.84|348.40||5.78|114.16|82.68|88.46|| +2452638|37697|10950|6837|975590|4790|35143|2|52|1095|98|2.98|4.52|2.62|0.00|256.76|292.04|442.96|7.70|0.00|256.76|264.46|-35.28| +2452638|37697|17383|6837|975590|4790|35143|2|129|1095|39|51.26|68.17|42.94|0.00|1674.66|1999.14|2658.63|100.47|0.00|1674.66|1775.13|-324.48| +2452638|37697|11382|6837|975590|4790|35143|2|160|1095|94|41.09|52.18|19.30|0.00|1814.20|3862.46|4904.92|90.71|0.00|1814.20|1904.91|-2048.26| +2452638|37697|13686|6837|975590|4790|35143|2|208|1095|1|93.19|147.24|94.23|0.00|94.23|93.19|147.24|7.53|0.00|94.23|101.76|1.04| +2452638|37697|11391|6837|975590|4790|35143|2|4|1095|78|55.85|97.17|1.94|0.00|151.32|4356.30|7579.26|7.56|0.00|151.32|158.88|-4204.98| +2452638|37697|6930|6837|975590|4790|35143|2|80|1095|56|67.13|131.57|11.84|0.00|663.04|3759.28|7367.92|53.04|0.00|663.04|716.08|-3096.24| +2452638|37697|13927|6837|975590|4790|35143|2|287|1095|51|11.45|18.32|0.54|0.00|27.54|583.95|934.32|2.20|0.00|27.54|29.74|-556.41| +2452638|37697|10770|6837|975590|4790|35143|2|150|1095|48|54.22|100.84|83.69|0.00|4017.12|2602.56|4840.32|321.36|0.00|4017.12|4338.48|1414.56| +2452638|37697|14701|6837|975590|4790|35143|2|208|1095|16|6.54|11.18|7.04|0.00|112.64|104.64|178.88|2.25|0.00|112.64|114.89|8.00| +2452638|37697|17421|6837|975590|4790|35143|2|117|1095|49|34.73|59.04|49.00|1416.59|2401.00|1701.77|2892.96|39.37|1416.59|984.41|1023.78|-717.36| +2452638|37697|9438|6837|975590|4790|35143|2|230|1095|5|21.70|39.92|10.77|0.00|53.85|108.50|199.60|0.00|0.00|53.85|53.85|-54.65| +2452638|37697|7656|6837|975590|4790|35143|2|3|1095|83|88.97|173.49|72.86|0.00|6047.38|7384.51|14399.67|544.26|0.00|6047.38|6591.64|-1337.13| +2452638|37697|12223|6837|975590|4790|35143|2|79|1095|61|9.24|9.51|1.33|19.47|81.13|563.64|580.11|0.61|19.47|61.66|62.27|-501.98| +2452638|37697|15085|6837|975590|4790|35143|2|144|1095|70|83.69|92.05|0.92|57.31|64.40|5858.30|6443.50|0.35|57.31|7.09|7.44|-5851.21| +2452638|37697|3741|6837|975590|4790|35143|2|183|1095|80|66.98|120.56|40.99|0.00|3279.20|5358.40|9644.80|0.00|0.00|3279.20|3279.20|-2079.20| +2451722|61238|589|77004|252914|24|8287|4|201|1096|30|60.52|75.04|11.25|0.00|337.50|1815.60|2251.20|13.50|0.00|337.50|351.00|-1478.10| +2451722|61238|7459|77004|252914|24|8287|||1096|12|25.95|29.06|15.98||191.76|||||151.50||-159.90| +2451722|61238|6421|77004|252914|24|8287|4|294|1096|3|54.99|73.68|33.15|0.00|99.45|164.97|221.04|7.95|0.00|99.45|107.40|-65.52| +2451722|61238|17651|77004|252914|24|8287|4|297|1096|5|70.36|99.20|8.92|0.00|44.60|351.80|496.00|0.89|0.00|44.60|45.49|-307.20| +2451722|61238|2813||252914|||4|9|1096||5.39|10.40|7.28|102.35||199.43|384.80||102.35||180.37|-32.42| +2451722|61238|17378|77004|252914|24|8287|4|20|1096|97|64.29|120.22|97.37|0.00|9444.89|6236.13|11661.34|283.34|0.00|9444.89|9728.23|3208.76| +2451722|61238|10087|77004|252914|24|8287|4|104|1096|26|75.15|142.78|127.07|0.00|3303.82|1953.90|3712.28|132.15|0.00|3303.82|3435.97|1349.92| +2451722|61238|5888|77004|252914|24|8287|4|12|1096|13|12.55|13.80|9.93|0.00|129.09|163.15|179.40|10.32|0.00|129.09|139.41|-34.06| +2451722|61238|16627|77004|252914|24|8287|4|40|1096|80|41.11|53.44|47.56|0.00|3804.80|3288.80|4275.20|114.14|0.00|3804.80|3918.94|516.00| +2451722|61238|15455|77004|252914|24|8287|4|187|1096|20|86.14|123.18|32.02|0.00|640.40|1722.80|2463.60|0.00|0.00|640.40|640.40|-1082.40| +2451722|61238|12206|77004|252914|24|8287|4|52|1096|59|51.98|92.52|5.55|0.00|327.45|3066.82|5458.68|16.37|0.00|327.45|343.82|-2739.37| +2451722|61238|16994|77004|252914|24|8287|4|260|1096|19|73.33|107.06|98.49|0.00|1871.31|1393.27|2034.14|18.71|0.00|1871.31|1890.02|478.04| +2451722|61238|14024|77004|252914|24|8287|4|143|1096|55|88.34|93.64|71.16|0.00|3913.80|4858.70|5150.20|352.24|0.00|3913.80|4266.04|-944.90| +2451722|61238|14353|77004|252914|24|8287|4|163|1096|36|2.46|4.92|1.52|0.00|54.72|88.56|177.12|3.83|0.00|54.72|58.55|-33.84| +2452394|38534|2682|21716|423418|1120|18422|10|253|1097|63|50.84|58.46|14.61|0.00|920.43|3202.92|3682.98|64.43|0.00|920.43|984.86|-2282.49| +2452394|38534|10419|21716|423418|1120|18422|10|109|1097|50|45.48|61.39|21.48|0.00|1074.00|2274.00|3069.50|42.96|0.00|1074.00|1116.96|-1200.00| +2452394|38534|16575|21716|423418|1120|18422|10|64|1097|89|52.01|98.29|68.80|0.00|6123.20|4628.89|8747.81|489.85|0.00|6123.20|6613.05|1494.31| +2452394|38534|7566|21716|423418|1120|18422|10|197|1097|85|98.12|156.01|23.40|0.00|1989.00|8340.20|13260.85|159.12|0.00|1989.00|2148.12|-6351.20| +2452394|38534|17928|21716|423418|1120|18422|10|190|1097|9|15.91|24.50|14.70|63.50|132.30|143.19|220.50|2.75|63.50|68.80|71.55|-74.39| +2452394|38534|12351|21716|423418|1120|18422|10|269|1097|66|20.23|37.62|35.36|0.00|2333.76|1335.18|2482.92|70.01|0.00|2333.76|2403.77|998.58| +2452394|38534|1440|21716|423418|1120|18422|10|21|1097|22|56.30|99.65|2.98|0.00|65.56|1238.60|2192.30|5.90|0.00|65.56|71.46|-1173.04| +2452394|38534|4387|21716|423418|1120|18422|10|192|1097|9|70.29|78.72|30.70|0.00|276.30|632.61|708.48|24.86|0.00|276.30|301.16|-356.31| +2451479|41672|1790|45228|1093653|6878|23656|4|163|1098|3|55.70|94.13|53.65|0.00|160.95|167.10|282.39|12.87|0.00|160.95|173.82|-6.15| +2451479|41672|3571|45228|1093653|6878|23656|4|145|1098|84|61.09|70.86|38.26|32.13|3213.84|5131.56|5952.24|254.53|32.13|3181.71|3436.24|-1949.85| +2451479|41672|13879|45228|1093653|6878|23656|4|3|1098|100|62.57|112.62|2.25|123.75|225.00|6257.00|11262.00|4.05|123.75|101.25|105.30|-6155.75| +2451479|41672|14113|45228|1093653|6878|23656|4|41|1098|63|46.85|81.98|30.33|0.00|1910.79|2951.55|5164.74|133.75|0.00|1910.79|2044.54|-1040.76| +2451479|41672|16726|45228|1093653|6878|23656|4|240|1098|39|75.33|86.62|11.26|0.00|439.14|2937.87|3378.18|0.00|0.00|439.14|439.14|-2498.73| +2451479|41672|16549|45228|1093653|6878|23656|4|93|1098|71|27.64|50.58|8.09|0.00|574.39|1962.44|3591.18|28.71|0.00|574.39|603.10|-1388.05| +2451479|41672|16904|45228|1093653|6878|23656|4|258|1098|24|95.00|143.45|25.82|0.00|619.68|2280.00|3442.80|18.59|0.00|619.68|638.27|-1660.32| +2451479|41672|1969|45228|1093653|6878|23656|4|193|1098|76|14.66|14.95|10.16|0.00|772.16|1114.16|1136.20|38.60|0.00|772.16|810.76|-342.00| +2451479|41672|3770|45228|1093653|6878|23656|4|136|1098|98|49.75|98.50|69.93|5551.04|6853.14|4875.50|9653.00|104.16|5551.04|1302.10|1406.26|-3573.40| +2451479|41672|14618|45228|1093653|6878|23656|4|170|1098|53|9.23|10.98|7.68|236.08|407.04|489.19|581.94|1.70|236.08|170.96|172.66|-318.23| +2451479|41672|7274|45228|1093653|6878|23656|4|183|1098|34|50.87|62.57|37.54|0.00|1276.36|1729.58|2127.38|25.52|0.00|1276.36|1301.88|-453.22| +2451808|67884|9068||972473|||8|166|1099||||||2395.60|1871.96||||2395.60||523.64| +2451808|67884|10190|16146|972473|3200|10680|8|159|1099|4|33.38|57.74|39.26|0.00|157.04|133.52|230.96|7.85|0.00|157.04|164.89|23.52| +2451808|67884|8006|16146|972473|3200|10680|8|279|1099|98|72.64|124.94|31.23|0.00|3060.54|7118.72|12244.12|61.21|0.00|3060.54|3121.75|-4058.18| +2451808|67884|16376|16146|972473|3200|10680|8|12|1099|35|23.12|44.39|23.08|0.00|807.80|809.20|1553.65|64.62|0.00|807.80|872.42|-1.40| +2451808|67884|10400|16146|972473|3200|10680|8|75|1099|18|24.23|36.34|35.97|0.00|647.46|436.14|654.12|32.37|0.00|647.46|679.83|211.32| +2451808|67884|15335|16146|972473|3200|10680|8|273|1099|44|17.90|26.49|0.26|0.00|11.44|787.60|1165.56|0.00|0.00|11.44|11.44|-776.16| +2451808|67884|12665|16146|972473|3200|10680|8|156|1099|24|31.07|32.93|20.08|0.00|481.92|745.68|790.32|9.63|0.00|481.92|491.55|-263.76| +|67884|15769|16146|972473|||8|40|1099|||59.95|59.35||1958.55|1221.33|1978.35|97.34||1390.58||| +2451808|67884|10771|16146|972473|3200|10680|8|154|1099|11|79.69|129.89|14.28|127.23|157.08|876.59|1428.79|0.59|127.23|29.85|30.44|-846.74| +2451808|67884|14791|16146|972473|3200|10680|8|66|1099|42|86.08|171.29|164.43|0.00|6906.06|3615.36|7194.18|0.00|0.00|6906.06|6906.06|3290.70| +2451140|43911|11162|92698|349826|3754|48482|4|226|1100|33|50.78|77.18|33.95|627.39|1120.35|1675.74|2546.94|0.00|627.39|492.96|492.96|-1182.78| +2451140|43911|12472|92698|349826|3754|48482|4|299|1100|75|75.29|110.67|39.84|0.00|2988.00|5646.75|8300.25|268.92|0.00|2988.00|3256.92|-2658.75| +2451140|43911|15184|92698|349826|3754|48482|4|239|1100|9|35.24|68.36|47.16|106.11|424.44|317.16|615.24|25.46|106.11|318.33|343.79|1.17| +2451140|43911|364|92698|349826|3754|48482|4|204|1100|29|66.79|91.50|45.75|0.00|1326.75|1936.91|2653.50|53.07|0.00|1326.75|1379.82|-610.16| +2451140|43911|11935|92698|349826|3754|48482|4|164|1100|36|60.11|62.51|7.50|243.00|270.00|2163.96|2250.36|0.00|243.00|27.00|27.00|-2136.96| +2451140|43911|9320|92698|349826|3754|48482|4|254|1100|45|72.14|111.81|76.03|376.34|3421.35|3246.30|5031.45|182.70|376.34|3045.01|3227.71|-201.29| +2451140|43911|6037|92698|349826|3754|48482|4|193|1100|21|66.46|71.11|41.95|0.00|880.95|1395.66|1493.31|70.47|0.00|880.95|951.42|-514.71| +2451140|43911|16861|92698|349826|3754|48482|4|253|1100|26|13.83|24.89|2.98|0.00|77.48|359.58|647.14|5.42|0.00|77.48|82.90|-282.10| +2451140|43911|16436|92698|349826|3754|48482|4|208|1100|41|37.73|61.49|56.57|0.00|2319.37|1546.93|2521.09|139.16|0.00|2319.37|2458.53|772.44| +2451140|43911|2056|92698|349826|3754|48482|4|101|1100|55|16.90|19.77|7.31|0.00|402.05|929.50|1087.35|32.16|0.00|402.05|434.21|-527.45| +2451140|43911|7790|92698|349826|3754|48482|4|185|1100|63|49.20|50.18|7.02|358.23|442.26|3099.60|3161.34|7.56|358.23|84.03|91.59|-3015.57| +2452223|66384|7953|11037|740281|1972|34816|7|224|1101|56|84.34|101.20|45.54|0.00|2550.24|4723.04|5667.20|178.51|0.00|2550.24|2728.75|-2172.80| +2452223|66384|15853|11037|740281|1972|34816|7|222|1101|77|69.47|124.35|19.89|0.00|1531.53|5349.19|9574.95|137.83|0.00|1531.53|1669.36|-3817.66| +2452223|66384|10819|11037|740281|1972|34816|7|81|1101|48|87.58|158.51|11.09|0.00|532.32|4203.84|7608.48|15.96|0.00|532.32|548.28|-3671.52| +|66384|12955||||34816|7|269|1101||||||746.58|1097.33|||||813.77|| +2452223|66384|7213|11037|740281|1972|34816|7|267|1101|51|97.05|105.78|52.89|0.00|2697.39|4949.55|5394.78|0.00|0.00|2697.39|2697.39|-2252.16| +2452223|66384|8805|11037|740281|1972|34816|7|208|1101|16|19.01|29.46|0.88|0.00|14.08|304.16|471.36|0.98|0.00|14.08|15.06|-290.08| +2452223|66384|15951|11037|740281|1972|34816|7|232|1101|62|69.87|85.24|16.19|0.00|1003.78|4331.94|5284.88|20.07|0.00|1003.78|1023.85|-3328.16| +2452223|66384|11567|11037|740281|1972|34816|7|92|1101|31|12.58|15.34|7.05|0.00|218.55|389.98|475.54|2.18|0.00|218.55|220.73|-171.43| +2452223|66384|5067|11037|740281|1972|34816|7|296|1101|2|14.52|25.41|8.89|0.00|17.78|29.04|50.82|0.17|0.00|17.78|17.95|-11.26| +2452223|66384|8555|11037|740281|1972|34816|7|264|1101|97|38.53|68.58|60.35|0.00|5853.95|3737.41|6652.26|234.15|0.00|5853.95|6088.10|2116.54| +2452223|66384|5151|11037|740281|1972|34816|7|286|1101|46|40.96|65.53|26.86|0.00|1235.56|1884.16|3014.38|0.00|0.00|1235.56|1235.56|-648.60| +2452223|66384|3021|11037|740281|1972|34816|7|188|1101|66|36.80|51.52|48.94|0.00|3230.04|2428.80|3400.32|290.70|0.00|3230.04|3520.74|801.24| +2452163|66148|10359|15005|1218420|7082|29744|1|135|1102|53|19.88|31.80|23.21|0.00|1230.13|1053.64|1685.40|86.10|0.00|1230.13|1316.23|176.49| +2452163|66148|6779|15005|1218420|7082|29744|1|88|1102|48|34.39|53.64|27.89|0.00|1338.72|1650.72|2574.72|66.93|0.00|1338.72|1405.65|-312.00| +2452163|66148|4805|15005|1218420|7082|29744|1|179|1102|19|80.97|131.17|51.15|612.26|971.85|1538.43|2492.23|14.38|612.26|359.59|373.97|-1178.84| +2452163|66148|8423|15005|1218420|7082|29744|1|101|1102|33|65.24|129.17|52.95|0.00|1747.35|2152.92|4262.61|104.84|0.00|1747.35|1852.19|-405.57| +2452163|66148|15251|15005|1218420|7082|29744|1|124|1102|3|68.06|131.35|65.67|0.00|197.01|204.18|394.05|5.91|0.00|197.01|202.92|-7.17| +2452163||13033|||7082||1|203|1102||78.39|117.58||1397.59||3605.94||108.51|1397.59|2712.97|2821.48|-892.97| +2452163|66148|6705|15005|1218420|7082|29744|1|4|1102|60|59.11|88.66|66.49|0.00|3989.40|3546.60|5319.60|159.57|0.00|3989.40|4148.97|442.80| +2452163|66148|14975|15005|1218420|7082|29744|1|72|1102|82|93.50|144.92|24.63|0.00|2019.66|7667.00|11883.44|20.19|0.00|2019.66|2039.85|-5647.34| +2452163|66148|6153|15005|1218420|7082|29744|1|122|1102|68|16.93|29.28|3.51|0.00|238.68|1151.24|1991.04|21.48|0.00|238.68|260.16|-912.56| +2452163|66148|15307|15005|1218420|7082|29744|1|257|1102|57|82.60|108.20|12.98|0.00|739.86|4708.20|6167.40|0.00|0.00|739.86|739.86|-3968.34| +2452163|66148|7803|15005|1218420|7082|29744|1|99|1102|54|59.17|90.53|18.10|0.00|977.40|3195.18|4888.62|87.96|0.00|977.40|1065.36|-2217.78| +2452626|39480|16944|43475|236170|418|17113|2|153|1103|63|77.94|149.64|82.30|0.00|5184.90|4910.22|9427.32|207.39|0.00|5184.90|5392.29|274.68| +2452626|39480|2466|43475|236170|418|17113|2|83|1103|7|41.83|76.54|47.45|0.00|332.15|292.81|535.78|0.00|0.00|332.15|332.15|39.34| +2452626|39480|8517|43475|236170|418|17113|2|156|1103|63|96.30|117.48|3.52|164.10|221.76|6066.90|7401.24|0.57|164.10|57.66|58.23|-6009.24| +2452626|39480|14364|43475|236170|418|17113|2|225|1103|32|77.38|98.27|4.91|67.56|157.12|2476.16|3144.64|1.79|67.56|89.56|91.35|-2386.60| +2452626|39480|2562|43475|236170|418|17113|2|226|1103|45|81.85|129.32|82.76|0.00|3724.20|3683.25|5819.40|0.00|0.00|3724.20|3724.20|40.95| +2452626|39480|14706|43475|236170|418|17113|2|133|1103|31|56.01|104.17|77.08|0.00|2389.48|1736.31|3229.27|215.05|0.00|2389.48|2604.53|653.17| +2452626|39480|3973|43475|236170|418|17113|2|124|1103|94|15.51|16.90|14.19|0.00|1333.86|1457.94|1588.60|120.04|0.00|1333.86|1453.90|-124.08| +2452626|39480|6072|43475|236170|418|17113|2|161|1103|41|63.38|90.63|27.18|0.00|1114.38|2598.58|3715.83|22.28|0.00|1114.38|1136.66|-1484.20| +2452626|39480|6883|43475|236170|418|17113|2|299|1103|6|68.18|105.67|44.38|263.61|266.28|409.08|634.02|0.10|263.61|2.67|2.77|-406.41| +||381||236170|||2|233|1103|13||134.99||103.84||1014.39|1754.87||103.84|176.83|189.20|| +2452626|39480|14325|43475|236170|418|17113|2|160|1103|77|37.35|38.09|3.80|114.11|292.60|2875.95|2932.93|1.78|114.11|178.49|180.27|-2697.46| +2452626|39480|9661|43475|236170|418|17113|2|46|1103|40|55.10|88.71|18.62|0.00|744.80|2204.00|3548.40|67.03|0.00|744.80|811.83|-1459.20| +2452266|37682|8221|45181|330243|6952|31614|8|237|1104|97|34.81|66.48|48.53|0.00|4707.41|3376.57|6448.56|188.29|0.00|4707.41|4895.70|1330.84| +2452266|37682|8541|45181|330243|6952|31614|8|140|1104|34|22.84|45.22|40.69|0.00|1383.46|776.56|1537.48|0.00|0.00|1383.46|1383.46|606.90| +2452266|37682|4251|45181|330243|6952|31614|8|3|1104|54|91.54|119.91|34.77|0.00|1877.58|4943.16|6475.14|56.32|0.00|1877.58|1933.90|-3065.58| +2452266|37682|16599|45181|330243|6952|31614|8|69|1104|23|80.50|129.60|49.24|0.00|1132.52|1851.50|2980.80|90.60|0.00|1132.52|1223.12|-718.98| +2452266|37682|12891|45181|330243|6952|31614|8|114|1104|58|23.99|40.06|7.21|405.63|418.18|1391.42|2323.48|1.12|405.63|12.55|13.67|-1378.87| +||4543||330243|||||1104|12|25.56|31.69||0.00|144.48||380.28||0.00||154.59|-162.24| +2452266|37682|8465|45181|330243|6952|31614|8|181|1104|93|18.42|24.68|21.47|0.00|1996.71|1713.06|2295.24|119.80|0.00|1996.71|2116.51|283.65| +2452266|37682|13713|45181|330243|6952|31614|8|147|1104|18|65.23|86.75|28.62|458.49|515.16|1174.14|1561.50|0.56|458.49|56.67|57.23|-1117.47| +2452266|37682|2497|45181|330243|6952|31614|8|158|1104|26|30.51|47.59|13.80|129.16|358.80|793.26|1237.34|6.88|129.16|229.64|236.52|-563.62| +2452266|37682|4749|45181|330243|6952|31614|8|129|1104|27|76.79|85.23|6.81|0.00|183.87|2073.33|2301.21|1.83|0.00|183.87|185.70|-1889.46| +2452266|37682|6751|45181|330243|6952|31614|8|296|1104|86|63.35|76.65|37.55|0.00|3229.30|5448.10|6591.90|290.63|0.00|3229.30|3519.93|-2218.80| +2452266|37682|6043|45181|330243|6952|31614|8|138|1104|49|76.83|129.84|122.04|3468.37|5979.96|3764.67|6362.16|75.34|3468.37|2511.59|2586.93|-1253.08| +2451764|57462|17861|76946|496895|844|45193|10|154|1105|29|78.29|126.04|114.69|0.00|3326.01|2270.41|3655.16|66.52|0.00|3326.01|3392.53|1055.60| +2451764|57462|2755|76946|496895|844|45193|10|19|1105|24|47.30|51.08|0.51|10.15|12.24|1135.20|1225.92|0.10|10.15|2.09|2.19|-1133.11| +2451764|57462|15287|76946|496895|844|45193|10|262|1105|15|14.78|22.90|13.96|0.00|209.40|221.70|343.50|6.28|0.00|209.40|215.68|-12.30| +2451764|57462|3308|76946|496895|844|45193|10|131|1105|19|90.71|146.95|14.69|0.00|279.11|1723.49|2792.05|5.58|0.00|279.11|284.69|-1444.38| +2451764|57462|7913|76946|496895|844|45193|10|233|1105|82|63.48|107.28|36.47|0.00|2990.54|5205.36|8796.96|179.43|0.00|2990.54|3169.97|-2214.82| +2451764|57462|13733|76946|496895|844|45193|10|57|1105|69|20.78|30.54|5.49|0.00|378.81|1433.82|2107.26|18.94|0.00|378.81|397.75|-1055.01| +2451764|57462|9499|76946|496895|844|45193|10|35|1105|32|91.14|91.14|86.58|2216.44|2770.56|2916.48|2916.48|22.16|2216.44|554.12|576.28|-2362.36| +||1730|||||10||1105|70||129.24|29.72||2080.40||9046.80|0.00||||-5158.66| +2451764|57462|13039|76946|496895|844|45193|10|147|1105|65|72.26|115.61|61.27|836.33|3982.55|4696.90|7514.65|157.31|836.33|3146.22|3303.53|-1550.68| +2451764|57462|14660|76946|496895|844|45193|10|90|1105|52|3.00|3.48|1.80|0.00|93.60|156.00|180.96|1.87|0.00|93.60|95.47|-62.40| +2451764|57462|6817|76946|496895|844|45193|10|83|1105|59|70.04|99.45|64.64|0.00|3813.76|4132.36|5867.55|228.82|0.00|3813.76|4042.58|-318.60| +2451550|37306|3041|18649|1380841|4986|7978|8|147|1106|33|93.52|177.68|138.59|0.00|4573.47|3086.16|5863.44|274.40|0.00|4573.47|4847.87|1487.31| +2451550|37306|14858|18649|1380841|4986|7978|8|78|1106|80|60.78|89.34|33.05|0.00|2644.00|4862.40|7147.20|211.52|0.00|2644.00|2855.52|-2218.40| +2451550|37306|2789|18649|1380841|4986|7978|8|55|1106|88|95.64|133.89|24.10|0.00|2120.80|8416.32|11782.32|190.87|0.00|2120.80|2311.67|-6295.52| +|37306|17918|18649|1380841|4986|7978|8||1106||10.85|||0.00||||0.00|0.00|||| +2451550|37306|3140|18649|1380841|4986|7978|8|236|1106|46|14.05|25.14|17.59|388.38|809.14|646.30|1156.44|12.62|388.38|420.76|433.38|-225.54| +2451550|37306|6956|18649|1380841|4986|7978|8|48|1106|82|33.43|59.83|15.55|0.00|1275.10|2741.26|4906.06|38.25|0.00|1275.10|1313.35|-1466.16| +2451550|37306|3353|18649|1380841|4986|7978|8|149|1106|5|29.08|43.91|6.58|0.00|32.90|145.40|219.55|1.64|0.00|32.90|34.54|-112.50| +2451550|37306|359|18649|1380841|4986|7978|8|149|1106|64|15.92|31.68|16.47|0.00|1054.08|1018.88|2027.52|10.54|0.00|1054.08|1064.62|35.20| +2451550|37306|8420|18649|1380841|4986|7978|8|171|1106|13|96.95|135.73|29.86|0.00|388.18|1260.35|1764.49|15.52|0.00|388.18|403.70|-872.17| +2451550|37306|1724|18649|1380841|4986|7978|8|230|1106|73|90.02|145.83|71.45|0.00|5215.85|6571.46|10645.59|156.47|0.00|5215.85|5372.32|-1355.61| +2451550|37306|17827|18649|1380841|4986|7978|8|30|1106|80|79.86|132.56|106.04|0.00|8483.20|6388.80|10604.80|339.32|0.00|8483.20|8822.52|2094.40| +2451550|37306|956|18649|1380841|4986|7978|8|138|1106|87|37.88|62.12|16.15|0.00|1405.05|3295.56|5404.44|14.05|0.00|1405.05|1419.10|-1890.51| +2452249|47822|349|72142|1480258|821|36137|7|127|1107|79|87.05|90.53|32.59|308.95|2574.61|6876.95|7151.87|45.31|308.95|2265.66|2310.97|-4611.29| +2452249|47822|351|72142|1480258|821|36137|7|221|1107|12|97.96|117.55|49.37|0.00|592.44|1175.52|1410.60|53.31|0.00|592.44|645.75|-583.08| +2452249|47822|14797|72142|1480258|821|36137|7|13|1107|47|67.66|82.54|37.96|0.00|1784.12|3180.02|3879.38|35.68|0.00|1784.12|1819.80|-1395.90| +2452249|47822|3889|72142|1480258|821|36137|7|129|1107|70|85.92|98.80|73.11|0.00|5117.70|6014.40|6916.00|409.41|0.00|5117.70|5527.11|-896.70| +2452249|47822|9539|72142|1480258|821|36137|7|225|1107|95|4.80|5.85|0.99|0.00|94.05|456.00|555.75|4.70|0.00|94.05|98.75|-361.95| +2452249|47822|14821|72142|1480258|821|36137|7|56|1107|89|66.60|129.87|59.74|478.51|5316.86|5927.40|11558.43|0.00|478.51|4838.35|4838.35|-1089.05| +2452249|47822|75|72142|1480258|821|36137|7|163|1107|33|26.42|51.78|13.46|0.00|444.18|871.86|1708.74|0.00|0.00|444.18|444.18|-427.68| +2452249|47822|617|72142|1480258|821|36137|7|20|1107|79|96.64|120.80|60.40|0.00|4771.60|7634.56|9543.20|429.44|0.00|4771.60|5201.04|-2862.96| +||1779|72142|1480258|821||7||1107|73|||2.14|0.00|||||0.00||162.46|| +2452249|47822|15993|72142|1480258|821|36137|7|191|1107|30|75.23|97.79|46.93|0.00|1407.90|2256.90|2933.70|42.23|0.00|1407.90|1450.13|-849.00| +2452249|47822|17493|72142|1480258|821|36137|7|84|1107|59|6.25|9.62|8.08|0.00|476.72|368.75|567.58|33.37|0.00|476.72|510.09|107.97| +2452249|47822|3189|72142|1480258|821|36137|7|107|1107|32|65.06|74.16|34.11|0.00|1091.52|2081.92|2373.12|43.66|0.00|1091.52|1135.18|-990.40| +|47822|5327|72142|||36137|7||1107|||110.04||||1347.48|1980.72||||1360.22|| +2451448|62213|10150|45219|393764|709|10054|1|241|1108|53|12.78|20.57|1.23|0.00|65.19|677.34|1090.21|0.65|0.00|65.19|65.84|-612.15| +2451448|62213|8480|45219|393764|709|10054|1|26|1108|18|44.21|55.70|36.76|0.00|661.68|795.78|1002.60|52.93|0.00|661.68|714.61|-134.10| +2451448|62213|16210|45219|393764|709|10054|1|64|1108|78|75.52|136.69|114.81|0.00|8955.18|5890.56|10661.82|358.20|0.00|8955.18|9313.38|3064.62| +2451448|62213|17180|45219|393764|709|10054|1|223|1108|95|81.80|104.70|0.00|0.00|0.00|7771.00|9946.50|0.00|0.00|0.00|0.00|-7771.00| +2451448|62213|14290|45219|393764|709|10054|1|108|1108|1|96.86|189.84|159.46|0.00|159.46|96.86|189.84|11.16|0.00|159.46|170.62|62.60| +2451448|62213|9418|45219|393764|709|10054|1|53|1108|58|79.70|144.25|89.43|1192.99|5186.94|4622.60|8366.50|319.51|1192.99|3993.95|4313.46|-628.65| +||1484||||10054|1||1108|43||106.99|0.00||0.00|||||0.00||| +2451448|62213|9268|45219|393764|709|10054|1|29|1108|72|61.12|89.23|6.24|0.00|449.28|4400.64|6424.56|13.47|0.00|449.28|462.75|-3951.36| +2451448|62213|15322|45219|393764|709|10054|1|234|1108|96|24.46|35.95|8.98|0.00|862.08|2348.16|3451.20|68.96|0.00|862.08|931.04|-1486.08| +2451448|62213|13723|45219|393764|709|10054|1|235|1108|17|28.37|41.70|35.02|0.00|595.34|482.29|708.90|5.95|0.00|595.34|601.29|113.05| +2451448|62213|13141|45219|393764|709|10054|1|73|1108|51|8.61|9.38|6.84|0.00|348.84|439.11|478.38|31.39|0.00|348.84|380.23|-90.27| +2451824|35934|1993|48445|844939|3033|39527|8|22|1109|50|96.44|102.22|22.48|0.00|1124.00|4822.00|5111.00|89.92|0.00|1124.00|1213.92|-3698.00| +2451824|35934|8768|48445|844939|3033|39527|8|94|1109|39|26.96|29.11|4.07|77.77|158.73|1051.44|1135.29|4.85|77.77|80.96|85.81|-970.48| +||2444||||39527|8||1109||7.79||9.57|||||5.14||85.75|90.89|-163.53| +2451824|35934|12848|48445|844939|3033|39527|8|246|1109|66|10.29|14.71|6.03|0.00|397.98|679.14|970.86|35.81|0.00|397.98|433.79|-281.16| +2451824|35934|12997|48445|844939|3033|39527|8|155|1109|86|20.01|32.41|32.41|0.00|2787.26|1720.86|2787.26|83.61|0.00|2787.26|2870.87|1066.40| +2451824|35934|5576|48445|844939|3033|39527|8|123|1109|55|17.29|22.47|2.24|0.00|123.20|950.95|1235.85|7.39|0.00|123.20|130.59|-827.75| +2451824|35934|2873|48445|844939|3033|39527|8|217|1109|15|52.68|96.93|7.75|10.46|116.25|790.20|1453.95|8.46|10.46|105.79|114.25|-684.41| +2451824|35934|9764|48445|844939|3033|39527|8|82|1109|62|46.75|92.09|30.38|0.00|1883.56|2898.50|5709.58|75.34|0.00|1883.56|1958.90|-1014.94| +2451824|35934|14138|48445|844939|3033|39527|8|183|1109|97|22.58|34.32|1.02|0.00|98.94|2190.26|3329.04|6.92|0.00|98.94|105.86|-2091.32| +2451898|63659|17383|24210|206087|6364|23041|2|150|1110|45|76.97|113.91|27.33|860.89|1229.85|3463.65|5125.95|0.00|860.89|368.96|368.96|-3094.69| +2451898|63659|11381|24210|206087|6364|23041|2|101|1110|2|7.68|12.82|8.84|0.00|17.68|15.36|25.64|1.06|0.00|17.68|18.74|2.32| +2451898|63659|13685|24210|206087|6364|23041|2|264|1110|64|3.70|5.10|4.74|0.00|303.36|236.80|326.40|12.13|0.00|303.36|315.49|66.56| +2451898|63659|11390|24210|206087|6364|23041|2|162|1110|62|39.50|74.65|44.04|0.00|2730.48|2449.00|4628.30|27.30|0.00|2730.48|2757.78|281.48| +2451898|63659|6929|24210|206087|6364|23041|2|233|1110|63|38.10|38.48|27.32|0.00|1721.16|2400.30|2424.24|137.69|0.00|1721.16|1858.85|-679.14| +2451898|63659|13927|24210|206087|6364|23041|2|69|1110|48|90.70|131.51|90.74|0.00|4355.52|4353.60|6312.48|261.33|0.00|4355.52|4616.85|1.92| +||10769|||6364||||1110||85.22||116.91|0.00|||2147.40|126.26|0.00||2230.64|| +2451898|63659|14701|24210|206087|6364|23041|2|61|1110|60|69.16|118.26|100.52|241.24|6031.20|4149.60|7095.60|289.49|241.24|5789.96|6079.45|1640.36| +2451383|50930|5671|63710|522925|3387|24602|2|103|1111|94|77.01|140.92|22.54|0.00|2118.76|7238.94|13246.48|169.50|0.00|2118.76|2288.26|-5120.18| +2451383|50930|15640|63710|522925|3387|24602|2|229|1111|1|41.57|46.97|11.74|11.62|11.74|41.57|46.97|0.00|11.62|0.12|0.12|-41.45| +2451383|50930|9193|63710|522925|3387|24602|2|288|1111|54|61.02|95.80|44.06|0.00|2379.24|3295.08|5173.20|142.75|0.00|2379.24|2521.99|-915.84| +2451383|50930|16442|63710|522925|3387|24602|2|55|1111|86|50.98|97.88|97.88|0.00|8417.68|4384.28|8417.68|168.35|0.00|8417.68|8586.03|4033.40| +2451383|50930|12682|63710|522925|3387|24602|2|72|1111|70|10.17|14.94|14.94|0.00|1045.80|711.90|1045.80|94.12|0.00|1045.80|1139.92|333.90| +2451383|50930|11372|63710|522925|3387|24602|2|44|1111|79|93.58|104.80|102.70|0.00|8113.30|7392.82|8279.20|405.66|0.00|8113.30|8518.96|720.48| +2451383|50930|4429|63710|522925|3387|24602|2|61|1111|33|71.86|114.97|14.94|0.00|493.02|2371.38|3794.01|39.44|0.00|493.02|532.46|-1878.36| +2451383|50930|10171|63710|522925|3387|24602|2|176|1111|77|20.05|36.29|5.44|0.00|418.88|1543.85|2794.33|4.18|0.00|418.88|423.06|-1124.97| +2451383|50930|14080|63710|522925|3387|24602|2|169|1111|60|34.32|52.85|2.64|0.00|158.40|2059.20|3171.00|12.67|0.00|158.40|171.07|-1900.80| +2451383|50930|11654|63710|522925|3387|24602|2|280|1111|66|41.86|50.23|48.22|0.00|3182.52|2762.76|3315.18|159.12|0.00|3182.52|3341.64|419.76| +2451383|50930|14468|63710|522925|3387|24602|2|94|1111|81|94.90|175.56|35.11|0.00|2843.91|7686.90|14220.36|0.00|0.00|2843.91|2843.91|-4842.99| +||6182|82353|1496122|3491||||1112|60|5.65|6.78|2.57||154.20||||||155.74|| +2451595|62129|16166|82353|1496122|3491|32507|10|25|1112|41|81.82|154.63|40.20|0.00|1648.20|3354.62|6339.83|0.00|0.00|1648.20|1648.20|-1706.42| +2451595|62129|10489|82353|1496122|3491|32507|10|82|1112|32|67.72|129.34|40.09|0.00|1282.88|2167.04|4138.88|89.80|0.00|1282.88|1372.68|-884.16| +2451595|62129|11375|82353|1496122|3491|32507|10|228|1112|36|32.02|32.98|19.78|0.00|712.08|1152.72|1187.28|42.72|0.00|712.08|754.80|-440.64| +2451595|62129|1181|82353|1496122|3491|32507|10|14|1112|73|89.89|176.18|153.27|0.00|11188.71|6561.97|12861.14|447.54|0.00|11188.71|11636.25|4626.74| +2451595|62129|14869|82353|1496122|3491|32507|10|195|1112|44|84.36|152.69|126.73|0.00|5576.12|3711.84|6718.36|111.52|0.00|5576.12|5687.64|1864.28| +2451595|62129|12973|82353|1496122|3491|32507|10|27|1112|41|51.93|102.82|8.22|0.00|337.02|2129.13|4215.62|16.85|0.00|337.02|353.87|-1792.11| +2451595|62129|6362|82353|1496122|3491|32507|10|274|1112|63|90.03|135.94|72.04|0.00|4538.52|5671.89|8564.22|136.15|0.00|4538.52|4674.67|-1133.37| +2451595|62129|3019|82353|1496122|3491|32507|10|162|1112|67|97.23|146.81|64.59|2380.14|4327.53|6514.41|9836.27|116.84|2380.14|1947.39|2064.23|-4567.02| +2451595|62129|16445|82353|1496122|3491|32507|10|94|1112|21|33.10|47.33|17.98|0.00|377.58|695.10|993.93|7.55|0.00|377.58|385.13|-317.52| +2451595|62129|15653|82353|1496122|3491|32507|10|199|1112|87|41.38|77.38|31.72|0.00|2759.64|3600.06|6732.06|55.19|0.00|2759.64|2814.83|-840.42| +2451595|62129|3689|82353|1496122|3491|32507|10|225|1112|80|55.87|111.18|26.68|896.44|2134.40|4469.60|8894.40|61.89|896.44|1237.96|1299.85|-3231.64| +2451595|62129|8534|82353|1496122|3491|32507|10|85|1112|18|84.51|106.48|66.01|0.00|1188.18|1521.18|1916.64|106.93|0.00|1188.18|1295.11|-333.00| +2451595|62129|7369|82353|1496122|3491|32507|10|112|1112|86|38.09|47.99|1.91|0.00|164.26|3275.74|4127.14|14.78|0.00|164.26|179.04|-3111.48| +2452553|30139|13632|87963|538129|295|3661|4|287|1113|39|32.88|39.12|8.21|0.00|320.19|1282.32|1525.68|6.40|0.00|320.19|326.59|-962.13| +2452553|30139|5982|87963|538129|295|3661|4|265|1113|12|12.99|20.26|4.25|0.00|51.00|155.88|243.12|1.02|0.00|51.00|52.02|-104.88| +2452553|30139|4554|87963|538129|295|3661|4|31|1113|18|9.15|12.26|11.40|0.00|205.20|164.70|220.68|6.15|0.00|205.20|211.35|40.50| +2452553|30139|10993|87963|538129|295|3661|4|74|1113|61|49.65|93.83|59.11|0.00|3605.71|3028.65|5723.63|108.17|0.00|3605.71|3713.88|577.06| +2452553|30139|49|87963|538129|295|3661|4|69|1113|76|20.67|37.82|35.17|0.00|2672.92|1570.92|2874.32|26.72|0.00|2672.92|2699.64|1102.00| +2452553|30139|4584|87963|538129|295|3661|4|150|1113|1|39.93|71.87|8.62|0.00|8.62|39.93|71.87|0.51|0.00|8.62|9.13|-31.31| +2452553|30139|13539|87963|538129|295|3661|4|234|1113|7|10.83|17.76|8.52|0.00|59.64|75.81|124.32|5.36|0.00|59.64|65.00|-16.17| +2452553|30139|3249|87963|538129|295|3661|4|235|1113|7|28.72|56.29|45.03|0.00|315.21|201.04|394.03|18.91|0.00|315.21|334.12|114.17| +2452553|30139|4683|87963|538129|295|3661|4|84|1113|30|27.74|53.81|36.05|0.00|1081.50|832.20|1614.30|21.63|0.00|1081.50|1103.13|249.30| +2452170|37249|11795|71050|998765|2480|18195|7|21|1114|95|52.49|53.01|8.48|209.45|805.60|4986.55|5035.95|17.88|209.45|596.15|614.03|-4390.40| +2452170|37249|11597|71050|998765|2480|18195|7|267|1114|1|16.93|22.00|15.40|0.00|15.40|16.93|22.00|1.23|0.00|15.40|16.63|-1.53| +2452170|37249|587|71050|998765|2480|18195|7|232|1114|1|69.55|88.32|33.56|0.00|33.56|69.55|88.32|0.67|0.00|33.56|34.23|-35.99| +2452170|37249|15945|71050|998765|2480|18195|7|199|1114|98|22.24|27.35|17.50|0.00|1715.00|2179.52|2680.30|102.90|0.00|1715.00|1817.90|-464.52| +2452170|37249|11399|71050|998765|2480|18195|7|202|1114|55|58.19|62.26|26.14|0.00|1437.70|3200.45|3424.30|57.50|0.00|1437.70|1495.20|-1762.75| +2452170|37249|9299|71050|998765|2480|18195|7|215|1114|97|57.61|84.68|77.90|3022.52|7556.30|5588.17|8213.96|90.67|3022.52|4533.78|4624.45|-1054.39| +2452170|37249|2089|71050|998765|2480|18195|7|23|1114|1|95.96|97.87|67.53|52.67|67.53|95.96|97.87|0.00|52.67|14.86|14.86|-81.10| +2452170|37249|12681|71050|998765|2480|18195|7|29|1114|99|5.82|6.86|5.96|0.00|590.04|576.18|679.14|35.40|0.00|590.04|625.44|13.86| +2452170|37249|13833|71050|998765|2480|18195|7|13|1114|13|40.57|69.37|69.37|0.00|901.81|527.41|901.81|27.05|0.00|901.81|928.86|374.40| +2452170|37249|8397|71050|998765|2480||7|104|1114||14.70||17.26|0.00||294.00|426.20||0.00||369.36|| +2452170|37249|6975|71050|998765|2480|18195|7|112|1114|35|25.44|45.28|7.69|0.00|269.15|890.40|1584.80|21.53|0.00|269.15|290.68|-621.25| +2452170|37249|14567|71050|998765|2480|18195|7|281|1114|37|88.83|172.33|110.29|1713.90|4080.73|3286.71|6376.21|142.00|1713.90|2366.83|2508.83|-919.88| +2452170|37249|6487|71050|998765|2480|18195|7|31|1114|34|94.09|127.02|45.72|808.32|1554.48|3199.06|4318.68|7.46|808.32|746.16|753.62|-2452.90| +2452170|37249|11295|71050|998765|2480|18195|7|139|1114|25|34.65|54.74|12.04|0.00|301.00|866.25|1368.50|6.02|0.00|301.00|307.02|-565.25| +2452170|37249|12057|71050|998765|2480|18195|7|260|1114|27|82.10|133.00|22.61|0.00|610.47|2216.70|3591.00|12.20|0.00|610.47|622.67|-1606.23| +2452170|37249|3487|71050|998765|2480|18195|7|15|1114|92|38.13|56.43|20.87|787.21|1920.04|3507.96|5191.56|11.32|787.21|1132.83|1144.15|-2375.13| +2451061|37632|15716|46095|867630|5518|16253|10|100|1115|32|88.43|155.63|138.51|0.00|4432.32|2829.76|4980.16|221.61|0.00|4432.32|4653.93|1602.56| +2451061|37632|1525|46095|867630|5518|16253|10|151|1115|18|17.35|31.40|23.23|0.00|418.14|312.30|565.20|4.18|0.00|418.14|422.32|105.84| +2451061|37632|4555|46095|867630|5518|16253|10|235|1115|93|3.44|4.50|0.76|0.00|70.68|319.92|418.50|0.00|0.00|70.68|70.68|-249.24| +2451061|37632|15020|46095|867630|5518|16253|10|155|1115|61|80.61|141.06|122.72|0.00|7485.92|4917.21|8604.66|299.43|0.00|7485.92|7785.35|2568.71| +2451061|37632|7868|46095|867630|5518|16253|10|199|1115|57|5.23|5.23|0.15|0.00|8.55|298.11|298.11|0.17|0.00|8.55|8.72|-289.56| +2451061|37632|4489|46095|867630|5518|16253|10|119|1115|76|67.77|83.35|66.68|0.00|5067.68|5150.52|6334.60|253.38|0.00|5067.68|5321.06|-82.84| +2451061|37632|8425|46095|867630|5518|16253|10|83|1115|93|6.31|7.00|4.27|0.00|397.11|586.83|651.00|0.00|0.00|397.11|397.11|-189.72| +2451061|37632|3694|46095|867630|5518|16253|10|265|1115|91|69.85|124.33|11.18|0.00|1017.38|6356.35|11314.03|91.56|0.00|1017.38|1108.94|-5338.97| +2451061|37632|7130|46095|867630|5518|16253|10|281|1115|48|30.85|42.57|20.43|0.00|980.64|1480.80|2043.36|78.45|0.00|980.64|1059.09|-500.16| +2451061|37632|2546|46095|867630|5518|16253|10|84|1115|26|6.34|11.60|8.58|82.53|223.08|164.84|301.60|4.21|82.53|140.55|144.76|-24.29| +2451061|37632|10747|46095|867630|5518|16253|10|165|1115|11|65.71|90.67|32.64|0.00|359.04|722.81|997.37|10.77|0.00|359.04|369.81|-363.77| +2451061|37632|10498|46095|867630|5518|16253|10|289|1115|32|16.97|26.13|22.73|436.41|727.36|543.04|836.16|14.54|436.41|290.95|305.49|-252.09| +2451061|37632|10678|46095|867630|5518|16253|10|143|1115|33|96.88|156.94|26.67|0.00|880.11|3197.04|5179.02|8.80|0.00|880.11|888.91|-2316.93| +2451061|37632|12914|46095|867630|5518|16253|10|104|1115|6|56.98|99.71|82.75|0.00|496.50|341.88|598.26|24.82|0.00|496.50|521.32|154.62| +2451061|37632|9601|46095|867630|5518|16253|10|65|1115|27|31.59|42.33|23.28|358.27|628.56|852.93|1142.91|8.10|358.27|270.29|278.39|-582.64| +2451737|65407|17300|17284|1240156|6675|27194|1|2|1116|44|43.60|67.14|29.54|0.00|1299.76|1918.40|2954.16|116.97|0.00|1299.76|1416.73|-618.64| +2451737|65407|9593|17284|1240156|6675|27194|1|46|1116|53|46.44|79.41|23.02|0.00|1220.06|2461.32|4208.73|12.20|0.00|1220.06|1232.26|-1241.26| +2451737|65407|938|17284|1240156|6675|27194|1|12|1116|44|50.95|81.52|48.09|0.00|2115.96|2241.80|3586.88|126.95|0.00|2115.96|2242.91|-125.84| +2451737|65407|205|17284|1240156|6675|27194|1|188|1116|65|77.09|111.00|89.91|0.00|5844.15|5010.85|7215.00|292.20|0.00|5844.15|6136.35|833.30| +2451737|65407|12194|17284|1240156|6675|27194|1|15|1116|92|79.64|110.69|49.81|0.00|4582.52|7326.88|10183.48|45.82|0.00|4582.52|4628.34|-2744.36| +2451737|65407|6419|17284|1240156|6675|27194|1|226|1116|96|2.68|4.23|0.54|4.14|51.84|257.28|406.08|0.00|4.14|47.70|47.70|-209.58| +2451737|65407|12254|17284|1240156|6675|27194|1|67|1116|55|86.27|135.44|43.34|0.00|2383.70|4744.85|7449.20|143.02|0.00|2383.70|2526.72|-2361.15| +|65407|593|17284|1240156||27194|1|104|1116||||||98.42|118.94||||||| +2451737|65407|1411|17284|1240156|6675|27194|1|61|1116|13|20.95|29.53|4.13|0.00|53.69|272.35|383.89|3.22|0.00|53.69|56.91|-218.66| +2451737|65407|13915|17284|1240156|6675|27194|1|69|1116|81|22.12|43.79|38.53|873.86|3120.93|1791.72|3546.99|67.41|873.86|2247.07|2314.48|455.35| +2451737|65407|14144|17284|1240156|6675|27194|1|72|1116|11|60.47|117.31|19.94|0.00|219.34|665.17|1290.41|13.16|0.00|219.34|232.50|-445.83| +2451737|65407|8405|17284|1240156|6675|27194|1|21|1116|15|34.05|37.79|9.44|0.00|141.60|510.75|566.85|8.49|0.00|141.60|150.09|-369.15| +2451737|65407|5498|17284|1240156|6675|27194|1|111|1116|10|99.01|158.41|136.23|735.64|1362.30|990.10|1584.10|43.86|735.64|626.66|670.52|-363.44| +2451737|65407|4124|17284|1240156|6675|27194|1|212|1116|83|88.88|119.98|115.18|0.00|9559.94|7377.04|9958.34|382.39|0.00|9559.94|9942.33|2182.90| +2451818|58007|1895|61480|594453|2273|48893|8|201|1117|70|10.69|14.32|12.31|0.00|861.70|748.30|1002.40|68.93|0.00|861.70|930.63|113.40| +2451818|58007|1736|61480|594453|2273|48893|8|221|1117|2|82.27|85.56|52.19|0.00|104.38|164.54|171.12|9.39|0.00|104.38|113.77|-60.16| +2451818|58007|9188|61480|594453|2273|48893|8|54|1117|97|45.51|47.33|43.07|0.00|4177.79|4414.47|4591.01|41.77|0.00|4177.79|4219.56|-236.68| +2451818|58007|16261|61480|594453|2273|48893|8|163|1117|86|80.46|97.35|76.90|0.00|6613.40|6919.56|8372.10|462.93|0.00|6613.40|7076.33|-306.16| +2451818|58007|8504|61480|594453|2273|48893|8|21|1117|19|44.41|54.18|25.46|0.00|483.74|843.79|1029.42|43.53|0.00|483.74|527.27|-360.05| +2451818|58007|14473|61480|594453|2273|48893|8|263|1117|1|55.94|71.60|64.44|0.00|64.44|55.94|71.60|3.86|0.00|64.44|68.30|8.50| +2451818|58007|6889|61480|594453|2273|48893|8|202|1117|98|64.67|107.99|56.15|0.00|5502.70|6337.66|10583.02|110.05|0.00|5502.70|5612.75|-834.96| +2451818|58007|4171|61480|594453|2273|48893|8|89|1117|18|79.82|88.60|2.65|0.00|47.70|1436.76|1594.80|0.47|0.00|47.70|48.17|-1389.06| +2451818|58007|16790|61480|594453|2273|48893|8|158|1117|93|33.06|42.97|12.03|0.00|1118.79|3074.58|3996.21|0.00|0.00|1118.79|1118.79|-1955.79| +2451818|58007|2960|61480|594453|2273|48893|8|205|1117|97|57.90|88.58|25.68|0.00|2490.96|5616.30|8592.26|124.54|0.00|2490.96|2615.50|-3125.34| +2451818|58007|15428|61480|594453|2273|48893|8|288|1117|92|51.96|103.40|74.44|0.00|6848.48|4780.32|9512.80|0.00|0.00|6848.48|6848.48|2068.16| +2451362|61710|9410|60269|142180|6138|9296|4|285|1118|52|38.85|66.82|36.08|0.00|1876.16|2020.20|3474.64|0.00|0.00|1876.16|1876.16|-144.04| +2451362|61710|8326|60269|142180|6138|9296|4|121|1118|72|47.89|92.42|27.72|0.00|1995.84|3448.08|6654.24|39.91|0.00|1995.84|2035.75|-1452.24| +2451362|61710|6434|60269|142180|6138|9296|4|204|1118|96|49.14|92.38|3.69|0.00|354.24|4717.44|8868.48|7.08|0.00|354.24|361.32|-4363.20| +2451362|61710|14005|60269|142180|6138|9296|4|108|1118|57|15.39|26.93|26.39|0.00|1504.23|877.23|1535.01|30.08|0.00|1504.23|1534.31|627.00| +2451362|61710|4477|60269|142180|6138|9296|4|291|1118|94|2.89|3.41|1.43|37.63|134.42|271.66|320.54|3.87|37.63|96.79|100.66|-174.87| +2451362|61710|9350|60269|142180|6138|9296|4|16|1118|7|20.57|32.50|15.92|0.00|111.44|143.99|227.50|5.57|0.00|111.44|117.01|-32.55| +2451362||4148||142180||9296|||1118|||60.25||22.55||790.24|964.00||22.55|353.29|367.42|| +2451362|61710|10231|60269|142180|6138|9296|4|158|1118|26|78.58|154.80|65.01|0.00|1690.26|2043.08|4024.80|67.61|0.00|1690.26|1757.87|-352.82| +2451362|61710|7978|60269|142180|6138|9296|4|267|1118|20|85.94|146.95|26.45|0.00|529.00|1718.80|2939.00|10.58|0.00|529.00|539.58|-1189.80| +2451362|61710|10327|60269|142180|6138|9296|4|276|1118|24|50.79|87.35|10.48|0.00|251.52|1218.96|2096.40|15.09|0.00|251.52|266.61|-967.44| +|61710|8839|60269|142180|6138||4|159|1118|99|70.87|||||7016.13|8068.50|306.60||6132.06||-884.07| +2451917|29725|4333|78589|1200632|7028|5871|1|267|1119|9|86.17|155.96|20.27|0.00|182.43|775.53|1403.64|3.64|0.00|182.43|186.07|-593.10| +2451917|29725|17867|78589|1200632|7028|5871|1|39|1119|74|53.31|53.84|30.68|0.00|2270.32|3944.94|3984.16|68.10|0.00|2270.32|2338.42|-1674.62| +2451917|29725|2461|78589|1200632|7028|5871|1|55|1119|71|41.72|73.84|59.07|0.00|4193.97|2962.12|5242.64|335.51|0.00|4193.97|4529.48|1231.85| +2451917|29725|4867|78589|1200632|7028|5871|1|52|1119|36|38.52|61.24|25.72|0.00|925.92|1386.72|2204.64|9.25|0.00|925.92|935.17|-460.80| +2451917|29725|8031|78589|1200632|7028|5871|1|205|1119|37|82.64|140.48|57.59|0.00|2130.83|3057.68|5197.76|149.15|0.00|2130.83|2279.98|-926.85| +2451917|29725|7867|78589|1200632|7028|5871|1|85|1119|7|47.96|58.03|35.39|0.00|247.73|335.72|406.21|2.47|0.00|247.73|250.20|-87.99| +2451917|29725|469|78589|1200632|7028|5871|1|137|1119|6|6.31|7.76|1.55|4.65|9.30|37.86|46.56|0.00|4.65|4.65|4.65|-33.21| +2451917|29725|8359|78589|1200632|7028|5871|1|5|1119|94|98.51|120.18|12.01|0.00|1128.94|9259.94|11296.92|101.60|0.00|1128.94|1230.54|-8131.00| +2451917|29725|849|78589|1200632|7028|5871|1|290|1119|67|8.86|12.75|5.99|0.00|401.33|593.62|854.25|20.06|0.00|401.33|421.39|-192.29| +2451917|29725|16555|78589|1200632|7028|5871|1|202|1119|62|46.85|47.78|15.28|0.00|947.36|2904.70|2962.36|0.00|0.00|947.36|947.36|-1957.34| +2451917|29725|9693|78589|1200632|7028|5871|1|48|1119|75|21.45|34.10|10.57|0.00|792.75|1608.75|2557.50|47.56|0.00|792.75|840.31|-816.00| +2451917|29725|10315|78589|1200632|7028|5871|1|255|1119|11|19.10|35.71|27.85|0.00|306.35|210.10|392.81|6.12|0.00|306.35|312.47|96.25| +2451917|29725|15665|78589|1200632|7028|5871|1|279|1119|19|28.24|55.91|21.80|0.00|414.20|536.56|1062.29|8.28|0.00|414.20|422.48|-122.36| +2451564|41115|8107|||4057||10||1120|67|2.15||0.98|0.00|65.66|144.05||0.65|0.00|||-78.39| +2451564|41115|4907|18495|815276|4057|47238|10|70|1120|7|64.53|109.05|87.24|439.68|610.68|451.71|763.35|10.26|439.68|171.00|181.26|-280.71| +2451564|41115|16355|18495|815276|4057|47238|10|101|1120|84|75.06|93.82|70.36|0.00|5910.24|6305.04|7880.88|413.71|0.00|5910.24|6323.95|-394.80| +2451564|41115|16637|18495|815276|4057|47238|10|290|1120|10|11.37|19.67|15.73|0.00|157.30|113.70|196.70|4.71|0.00|157.30|162.01|43.60| +2451564|41115|14197|18495|815276|4057|47238|10|141|1120|2|67.51|68.86|47.51|0.00|95.02|135.02|137.72|0.95|0.00|95.02|95.97|-40.00| +2451564|41115|12770|18495|815276|4057|47238|10|143|1120|20|49.47|79.64|26.28|5.25|525.60|989.40|1592.80|46.83|5.25|520.35|567.18|-469.05| +|41115|8525|18495|815276|4057|47238||76|1120|||112.19|85.26|||||||6991.32|6991.32|| +2451564|41115|10151|18495|815276|4057|47238|10|217|1120|61|64.51|84.50|29.57|1100.29|1803.77|3935.11|5154.50|7.03|1100.29|703.48|710.51|-3231.63| +2451564|41115|8480|18495|815276|4057|47238|10|124|1120|80|86.00|101.48|71.03|0.00|5682.40|6880.00|8118.40|227.29|0.00|5682.40|5909.69|-1197.60| +2451564|41115|16211|18495|815276|4057|47238|10|113|1120|13|1.13|1.61|0.49|0.00|6.37|14.69|20.93|0.19|0.00|6.37|6.56|-8.32| +2451564|41115|17180|18495|815276|4057|47238|10|230|1120|41|30.05|59.79|31.09|0.00|1274.69|1232.05|2451.39|38.24|0.00|1274.69|1312.93|42.64| +2451913|70673|321|85230|1838711|192|11673|1|236|1121|37|92.24|106.07|83.79|0.00|3100.23|3412.88|3924.59|93.00|0.00|3100.23|3193.23|-312.65| +2451913|70673|14901|85230|1838711|192|11673|1|26|1121|70|8.00|11.52|5.64|0.00|394.80|560.00|806.40|15.79|0.00|394.80|410.59|-165.20| +2451913|70673|17831|85230|1838711|192|11673|1|80|1121|18|48.41|77.45|20.91|0.00|376.38|871.38|1394.10|22.58|0.00|376.38|398.96|-495.00| +2451913|70673|17145|85230|1838711|192|11673|1|79|1121|77|75.82|78.09|71.06|0.00|5471.62|5838.14|6012.93|218.86|0.00|5471.62|5690.48|-366.52| +2451913|70673|16037|85230|1838711|192|11673|1|232|1121|74|7.57|13.32|0.66|0.00|48.84|560.18|985.68|4.39|0.00|48.84|53.23|-511.34| +2451913|70673|12265|85230|1838711|192|11673|1|261|1121|11|84.86|140.86|78.88|0.00|867.68|933.46|1549.46|17.35|0.00|867.68|885.03|-65.78| +2451913|70673|7309|85230|1838711|192|11673|1|111|1121|33|78.91|130.20|79.42|0.00|2620.86|2604.03|4296.60|131.04|0.00|2620.86|2751.90|16.83| +2451913|70673|12409|85230|1838711|192|11673|1|95|1121|10|81.80|94.88|11.38|0.00|113.80|818.00|948.80|10.24|0.00|113.80|124.04|-704.20| +2451913|70673|5569|85230|1838711|192|11673|1|34|1121|52|8.05|15.05|8.87|0.00|461.24|418.60|782.60|0.00|0.00|461.24|461.24|42.64| +2451913|70673|9559|85230|1838711|192|11673|1|133|1121|30|8.00|9.52|9.13|0.00|273.90|240.00|285.60|13.69|0.00|273.90|287.59|33.90| +2451913|70673|16789|85230|1838711|192|11673|1|193|1121|58|32.61|33.58|19.47|0.00|1129.26|1891.38|1947.64|101.63|0.00|1129.26|1230.89|-762.12| +2451913|70673|13113|85230|1838711|192|11673|1|235|1121|7|45.23|46.58|45.18|300.44|316.26|316.61|326.06|0.79|300.44|15.82|16.61|-300.79| +2451913|70673|133|85230|1838711|192|11673|1|165|1121|73|55.75|73.03|11.68|0.00|852.64|4069.75|5331.19|34.10|0.00|852.64|886.74|-3217.11| +2451913|70673|16393|85230|1838711|192|11673|1|167|1121|55|22.05|22.93|19.49|0.00|1071.95|1212.75|1261.15|64.31|0.00|1071.95|1136.26|-140.80| +2451913|70673|9943|85230|1838711|192|11673|1|158|1121|4|63.08|68.12|67.43|0.00|269.72|252.32|272.48|18.88|0.00|269.72|288.60|17.40| +2451913|70673|553|85230|1838711|192|11673|1|21|1121|43|13.44|20.69|0.20|0.00|8.60|577.92|889.67|0.00|0.00|8.60|8.60|-569.32| +2451906|32558|6775|89876|421597|3365|16686|7|188|1122|63|37.41|66.21|6.62|179.33|417.06|2356.83|4171.23|0.00|179.33|237.73|237.73|-2119.10| +2451906|32558|12203|89876|421597|3365|16686|7|261|1122|98|95.24|186.67|100.80|0.00|9878.40|9333.52|18293.66|691.48|0.00|9878.40|10569.88|544.88| +2451906|32558|13376|89876|421597|3365|16686|7|117|1122|34|9.73|15.56|2.64|0.00|89.76|330.82|529.04|5.38|0.00|89.76|95.14|-241.06| +2451906|32558|7151|89876|421597|3365|16686|7|245|1122|71|67.35|89.57|22.39|0.00|1589.69|4781.85|6359.47|15.89|0.00|1589.69|1605.58|-3192.16| +|32558|13316|89876||3365|16686|7||1122|4||123.27||0.00||352.20|493.08||0.00|310.64|332.38|| +2451906|32558|11719|89876|421597|3365|16686|7|108|1122|97|45.79|91.58|53.11|0.00|5151.67|4441.63|8883.26|154.55|0.00|5151.67|5306.22|710.04| +2451906|32558|7735|89876|421597|3365|16686|7|213|1122|71|66.86|133.72|49.47|0.00|3512.37|4747.06|9494.12|35.12|0.00|3512.37|3547.49|-1234.69| +2451906|32558|5549|89876|421597|3365|16686|7|126|1122|95|27.30|31.66|12.66|0.00|1202.70|2593.50|3007.70|24.05|0.00|1202.70|1226.75|-1390.80| +2451906|32558|15518|89876|421597|3365|16686|7|218|1122|33|20.42|34.71|15.61|0.00|515.13|673.86|1145.43|36.05|0.00|515.13|551.18|-158.73| +2451906|32558|13766|89876|421597|3365|16686|7|168|1122|69|67.49|97.18|48.59|0.00|3352.71|4656.81|6705.42|234.68|0.00|3352.71|3587.39|-1304.10| +2451906|32558|7669|89876|421597|3365|16686|7|83|1122|96|86.68|169.89|101.93|0.00|9785.28|8321.28|16309.44|489.26|0.00|9785.28|10274.54|1464.00| +2451906|32558|5017|89876|421597|3365|16686|7|284|1122|47|68.08|99.39|30.81|0.00|1448.07|3199.76|4671.33|86.88|0.00|1448.07|1534.95|-1751.69| +2451906|32558|4520|89876|421597|3365|16686|7|278|1122|82|34.47|60.66|3.63|0.00|297.66|2826.54|4974.12|0.00|0.00|297.66|297.66|-2528.88| +2451906|32558|5750|89876|421597|3365|16686|7|271|1122|84|13.32|16.78|12.24|0.00|1028.16|1118.88|1409.52|20.56|0.00|1028.16|1048.72|-90.72| +2451906|32558|10781|89876|421597|3365|16686|7|92|1122|73|92.50|112.85|64.32|0.00|4695.36|6752.50|8238.05|187.81|0.00|4695.36|4883.17|-2057.14| +||11642|89876|421597|3365|16686|7|92|1122||||9.65|0.00|||3199.08|10.22|0.00|511.45||| +2451823|38208|15017|20878|674932|5339|17393|1|200|1123|88|54.10|91.97|55.18|0.00|4855.84|4760.80|8093.36|145.67|0.00|4855.84|5001.51|95.04| +2451823|38208|16513|20878|674932|5339|17393|1|229|1123|20|99.04|130.73|122.88|0.00|2457.60|1980.80|2614.60|73.72|0.00|2457.60|2531.32|476.80| +2451823|38208|12818|20878|674932|5339|17393|1|228|1123|91|87.77|157.10|149.24|0.00|13580.84|7987.07|14296.10|407.42|0.00|13580.84|13988.26|5593.77| +2451823|38208|6337|20878|674932|5339|17393|1|244|1123|4|25.66|33.10|24.82|28.79|99.28|102.64|132.40|4.93|28.79|70.49|75.42|-32.15| +2451823|38208|12476|20878|674932|5339|17393|1|11|1123|14|85.25|160.27|56.09|0.00|785.26|1193.50|2243.78|70.67|0.00|785.26|855.93|-408.24| +2451823|38208|10106|20878|674932|5339|17393|1|123|1123|9|62.34|81.66|2.44|0.00|21.96|561.06|734.94|0.65|0.00|21.96|22.61|-539.10| +2451823|38208|10343|20878|674932|5339|17393|1|188|1123|51|5.05|6.21|0.24|0.00|12.24|257.55|316.71|0.73|0.00|12.24|12.97|-245.31| +2451823|38208|15421|20878|674932|5339|17393|1|179|1123|100|72.80|104.10|12.49|0.00|1249.00|7280.00|10410.00|74.94|0.00|1249.00|1323.94|-6031.00| +2451823|38208|11066|20878|674932|5339|17393|1|89|1123|99|82.89|84.54|30.43|0.00|3012.57|8206.11|8369.46|120.50|0.00|3012.57|3133.07|-5193.54| +2451823|38208|11287|20878|674932|5339|17393|1|197|1123|46|38.02|62.35|3.11|0.00|143.06|1748.92|2868.10|5.72|0.00|143.06|148.78|-1605.86| +2451823|38208|7568|20878|674932|5339|17393|1|183|1123|93|58.15|91.29|65.72|0.00|6111.96|5407.95|8489.97|61.11|0.00|6111.96|6173.07|704.01| +2451823|38208|17245|20878|674932|5339|17393|1|264|1123|91|68.09|68.77|2.06|0.00|187.46|6196.19|6258.07|9.37|0.00|187.46|196.83|-6008.73| +2451823|38208|6305|20878|674932|5339|17393|1|199|1123|100|87.42|125.88|50.35|0.00|5035.00|8742.00|12588.00|251.75|0.00|5035.00|5286.75|-3707.00| +2452611|55787|2118|74894|1253088|4082|47829|4|163|1124|89|35.31|66.73|8.67|0.00|771.63|3142.59|5938.97|30.86|0.00|771.63|802.49|-2370.96| +2452611|55787|11769|74894|1253088|4082|47829|4|216|1124|47|75.26|141.48|94.79|0.00|4455.13|3537.22|6649.56|0.00|0.00|4455.13|4455.13|917.91| +2452611|55787|3693|74894|1253088|4082|47829|4|157|1124|52|3.06|4.68|3.27|0.00|170.04|159.12|243.36|8.50|0.00|170.04|178.54|10.92| +2452611|55787|16729|74894|1253088|4082|47829|4|294|1124|57|76.06|111.80|0.00|0.00|0.00|4335.42|6372.60|0.00|0.00|0.00|0.00|-4335.42| +2452611|55787|14707|74894|1253088|4082|47829|4|161|1124|15|92.06|147.29|73.64|0.00|1104.60|1380.90|2209.35|22.09|0.00|1104.60|1126.69|-276.30| +2452611|55787|3657|74894|1253088|4082|47829|4|293|1124|17|28.98|29.26|2.92|0.00|49.64|492.66|497.42|3.47|0.00|49.64|53.11|-443.02| +2452611|55787|7093|74894|1253088|4082|47829|4|166|1124|30|87.52|89.27|50.88|0.00|1526.40|2625.60|2678.10|106.84|0.00|1526.40|1633.24|-1099.20| +2452611|55787|16380|74894|1253088|4082|47829|4|100|1124|95|87.69|175.38|99.96|4178.32|9496.20|8330.55|16661.10|265.89|4178.32|5317.88|5583.77|-3012.67| +2452611|55787|2791|74894|1253088|4082|47829|4|136|1124|72|87.11|99.30|64.54|0.00|4646.88|6271.92|7149.60|371.75|0.00|4646.88|5018.63|-1625.04| +2452611|55787|93|74894|1253088|4082|47829|4|231|1124|41|71.25|134.66|43.09|0.00|1766.69|2921.25|5521.06|35.33|0.00|1766.69|1802.02|-1154.56| +2452611|55787|529|74894|1253088|4082|47829|4|89|1124|21|61.76|104.99|52.49|407.84|1102.29|1296.96|2204.79|62.50|407.84|694.45|756.95|-602.51| +2450977|55523|10579|34815|95491|2214|11290|2|264|1125|36|42.22|61.21|28.76|610.86|1035.36|1519.92|2203.56|0.00|610.86|424.50|424.50|-1095.42| +2450977|55523|14401|34815|95491|2214|11290|2|156|1125|45|99.93|145.89|5.83|0.00|262.35|4496.85|6565.05|23.61|0.00|262.35|285.96|-4234.50| +2450977|55523|16621|34815|95491|2214|11290|2|159|1125|62|68.44|124.56|79.71|0.00|4942.02|4243.28|7722.72|345.94|0.00|4942.02|5287.96|698.74| +2450977|55523|14494|34815|95491|2214|11290|2|232|1125|16|78.43|87.84|19.32|0.00|309.12|1254.88|1405.44|15.45|0.00|309.12|324.57|-945.76| +2450977|55523|10358|34815|95491|2214|11290|2|259|1125|26|41.63|52.87|16.91|0.00|439.66|1082.38|1374.62|4.39|0.00|439.66|444.05|-642.72| +2450977|55523|6778|34815|95491|2214|11290|2|52|1125|73|56.08|97.57|92.69|0.00|6766.37|4093.84|7122.61|405.98|0.00|6766.37|7172.35|2672.53| +2450977|55523|4804|34815|95491|2214|11290|2|135|1125|71|61.38|112.93|68.88|0.00|4890.48|4357.98|8018.03|293.42|0.00|4890.48|5183.90|532.50| +2450977|55523|8422|34815|95491|2214|11290|2|143|1125|48|34.68|48.20|32.77|0.00|1572.96|1664.64|2313.60|110.10|0.00|1572.96|1683.06|-91.68| +2450977|55523|15250|34815|95491|2214|11290|2|69|1125|50|27.24|49.84|14.45|0.00|722.50|1362.00|2492.00|36.12|0.00|722.50|758.62|-639.50| +2450977|55523|13033|34815|95491|2214|11290|2|88|1125|4|62.68|73.33|44.73|0.00|178.92|250.72|293.32|12.52|0.00|178.92|191.44|-71.80| +||6704|34815|95491||11290|2||1125|85|||4.02|||505.75|||||269.19|-239.22| +2450977|55523|14974|34815|95491|2214|11290|2|167|1125|14|46.43|52.46|23.60|85.90|330.40|650.02|734.44|22.00|85.90|244.50|266.50|-405.52| +2450977|55523|6152|34815|95491|2214|11290|2|15|1125|34|31.07|37.28|0.74|0.00|25.16|1056.38|1267.52|2.01|0.00|25.16|27.17|-1031.22| +2450977|55523|15307|34815|95491|2214|11290|2|125|1125|38|71.95|94.97|52.23|0.00|1984.74|2734.10|3608.86|39.69|0.00|1984.74|2024.43|-749.36| +2452641|34445|3811|3377|803773|4969|32965|1|163|1126|17|73.15|85.58|3.42|23.25|58.14|1243.55|1454.86|0.34|23.25|34.89|35.23|-1208.66| +2452641|34445|4539|3377|803773|4969|32965|1|160|1126|22|38.62|68.35|27.34|0.00|601.48|849.64|1503.70|42.10|0.00|601.48|643.58|-248.16| +2452641|34445|5376|3377|803773|4969|32965|1|112|1126|7|37.91|61.03|6.71|0.00|46.97|265.37|427.21|3.75|0.00|46.97|50.72|-218.40| +2452641|34445|10884|3377|803773|4969|32965|1|136|1126|35|9.44|11.13|1.78|0.00|62.30|330.40|389.55|4.36|0.00|62.30|66.66|-268.10| +2452641|34445|396|3377|803773|4969|32965|1|164|1126|57|46.59|72.21|42.60|1165.53|2428.20|2655.63|4115.97|25.25|1165.53|1262.67|1287.92|-1392.96| +2452641|34445|16068|3377|803773|4969|32965|1|248|1126|36|35.02|46.57|14.90|0.00|536.40|1260.72|1676.52|37.54|0.00|536.40|573.94|-724.32| +2452641|34445|2838|3377|803773|4969|32965|1|110|1126|21|90.64|158.62|149.10|0.00|3131.10|1903.44|3331.02|31.31|0.00|3131.10|3162.41|1227.66| +2452641|34445|15573|3377|803773|4969|32965|1|184|1126|55|8.74|9.00|5.76|0.00|316.80|480.70|495.00|15.84|0.00|316.80|332.64|-163.90| +2452641|34445|11085|3377|803773|4969|32965|1|13|1126|30|41.79|75.63|2.26|0.00|67.80|1253.70|2268.90|4.74|0.00|67.80|72.54|-1185.90| +2452641|34445|14526|3377|803773|4969|32965|1|204|1126|80|7.46|9.77|0.00|0.00|0.00|596.80|781.60|0.00|0.00|0.00|0.00|-596.80| +2452641|34445|4110|3377|803773|4969|32965|1|290|1126|58|37.41|53.49|16.04|0.00|930.32|2169.78|3102.42|65.12|0.00|930.32|995.44|-1239.46| +2452641|34445|12378|||4969|32965|||1126|4||38.53|26.97|0.00|107.88|124.32||6.47|0.00||114.35|-16.44| +2452641|34445|13236|3377|803773|4969|32965|1|175|1126|65|49.75|56.71|42.53|82.93|2764.45|3233.75|3686.15|80.44|82.93|2681.52|2761.96|-552.23| +2451939|42198|8533|75879|1232575|3671|39567|4|206|1127|19|25.76|35.29|29.64|61.94|563.16|489.44|670.51|25.06|61.94|501.22|526.28|11.78| +2451939|42198|8929|75879|1232575|3671|39567|4|146|1127|92|60.69|62.51|11.87|0.00|1092.04|5583.48|5750.92|87.36|0.00|1092.04|1179.40|-4491.44| +2451939|42198|2335|75879|1232575|3671|39567|4|252|1127|12|26.86|51.57|47.96|74.81|575.52|322.32|618.84|15.02|74.81|500.71|515.73|178.39| +2451939|42198|3349|75879|1232575|3671|39567|4|111|1127|66|46.52|57.21|41.19|0.00|2718.54|3070.32|3775.86|190.29|0.00|2718.54|2908.83|-351.78| +2451939|42198|1629|75879|1232575|3671|39567|4|285|1127|80|13.79|26.06|20.58|0.00|1646.40|1103.20|2084.80|16.46|0.00|1646.40|1662.86|543.20| +||4121|75879||3671|39567|||1127||35.73|50.73||0.00||2215.26||107.52|0.00|||| +2451939|42198|4261|75879|1232575|3671|39567|4|204|1127|5|6.99|11.39|8.42|0.00|42.10|34.95|56.95|1.26|0.00|42.10|43.36|7.15| +2451939|42198|15473|75879|1232575||39567|||1127||||0.42|30.24||272.25|351.00||30.24||1.32|-270.99| +2451939|42198|2821|75879|1232575|3671|39567|4|14|1127|94|25.94|30.09|2.10|0.00|197.40|2438.36|2828.46|11.84|0.00|197.40|209.24|-2240.96| +2451939|42198|17803|75879|1232575|3671|39567|4|148|1127|53|16.90|33.46|11.04|392.03|585.12|895.70|1773.38|9.65|392.03|193.09|202.74|-702.61| +2451939|42198|16191|75879|1232575|3671|39567|4|50|1127|59|44.99|56.68|12.46|0.00|735.14|2654.41|3344.12|0.00|0.00|735.14|735.14|-1919.27| +2451939|42198|9555|75879|1232575|3671|39567|4|207|1127|46|75.41|92.75|18.55|0.00|853.30|3468.86|4266.50|17.06|0.00|853.30|870.36|-2615.56| +2451939|42198|8545|75879|1232575|3671|39567|4|104|1127|61|15.52|29.95|3.59|0.00|218.99|946.72|1826.95|6.56|0.00|218.99|225.55|-727.73| +2451939|42198|15443|75879|1232575|3671|39567|4|268|1127|77|32.23|41.89|10.05|0.00|773.85|2481.71|3225.53|46.43|0.00|773.85|820.28|-1707.86| +2452602|36080|2889|90206|1074057|2708|28606|1|5|1128|13|22.07|29.35|0.00|0.00|0.00|286.91|381.55|0.00|0.00|0.00|0.00|-286.91| +2452602|36080|13923|90206|1074057|2708|28606|1|291|1128|2|58.97|64.27|46.91|0.00|93.82|117.94|128.54|4.69|0.00|93.82|98.51|-24.12| +2452602|36080|14361|90206|1074057|2708|28606|1|104|1128|19|46.05|48.35|2.90|0.00|55.10|874.95|918.65|3.85|0.00|55.10|58.95|-819.85| +2452602||4675|90206|1074057|2708||1|256|1128||65.62||106.53||9161.58||9254.46|||||3518.26| +2452602|36080|6351|90206|1074057|2708|28606|1|225|1128|33|8.19|15.97|3.19|0.00|105.27|270.27|527.01|5.26|0.00|105.27|110.53|-165.00| +|36080|5880|90206||2708|28606|1||1128|97|||||3310.61|1844.94||||||| +2452602|36080|16897|90206|1074057|2708|28606|1|101|1128|70|55.96|61.55|60.93|0.00|4265.10|3917.20|4308.50|383.85|0.00|4265.10|4648.95|347.90| +2452602|36080|10656|90206|1074057|2708|28606|1|3|1128|65|93.89|132.38|38.39|0.00|2495.35|6102.85|8604.70|149.72|0.00|2495.35|2645.07|-3607.50| +2452602||4167||||||149|1128|4||||0.00|261.04||532.76|15.66|0.00|261.04||-101.40| +2452602|36080|6373|90206|1074057|2708|28606|1|283|1128|6|87.56|121.70|41.37|0.00|248.22|525.36|730.20|19.85|0.00|248.22|268.07|-277.14| +2452602|36080|2247|||2708|||215|1128|31|51.07|87.84|35.13||1089.03|||||||-494.14| +2452602|36080|4435|90206|1074057|2708|28606|1|129|1128|87|15.03|20.59|12.35|0.00|1074.45|1307.61|1791.33|85.95|0.00|1074.45|1160.40|-233.16| +2452602|36080|3183|90206|1074057|2708|28606|1|201|1128|4|32.09|59.36|24.93|0.00|99.72|128.36|237.44|6.98|0.00|99.72|106.70|-28.64| +2452602|36080|1269|90206|1074057|2708|28606|1|261|1128|31|77.92|103.63|4.14|0.00|128.34|2415.52|3212.53|8.98|0.00|128.34|137.32|-2287.18| +2452602|36080|12888|90206|1074057|2708|28606|1|15|1128|72|73.66|110.49|35.35|0.00|2545.20|5303.52|7955.28|50.90|0.00|2545.20|2596.10|-2758.32| +2451507|41532|2362|30460|1440572|2237|37949|4|48|1129|56|48.36|81.24|77.17|0.00|4321.52|2708.16|4549.44|86.43|0.00|4321.52|4407.95|1613.36| +2451507|41532|8998|30460|1440572|2237|37949|4|289|1129|49|3.85|4.54|1.54|0.00|75.46|188.65|222.46|0.00|0.00|75.46|75.46|-113.19| +2451507|41532|15592|30460|1440572|2237|37949|4|181|1129|28|14.77|16.24|0.64|0.00|17.92|413.56|454.72|0.35|0.00|17.92|18.27|-395.64| +2451507|41532|5677|30460|1440572|2237|37949|4|231|1129|97|39.22|46.27|11.10|0.00|1076.70|3804.34|4488.19|0.00|0.00|1076.70|1076.70|-2727.64| +2451507|41532|16706|30460|1440572|2237|37949|4|227|1129|93|84.03|168.06|40.33|3263.10|3750.69|7814.79|15629.58|4.87|3263.10|487.59|492.46|-7327.20| +2451507|41532|16466|30460|1440572|2237|37949|4|144|1129|67|74.05|77.01|61.60|0.00|4127.20|4961.35|5159.67|330.17|0.00|4127.20|4457.37|-834.15| +2451507|41532|17638|30460|1440572|2237|37949|4|125|1129|11|75.88|116.09|91.71|0.00|1008.81|834.68|1276.99|0.00|0.00|1008.81|1008.81|174.13| +|41532|11144|30460|1440572|2237||4||1129|52|62.28||20.14||||4760.60|||||| +2451507|41532|2152|30460|1440572|2237|37949|4|280|1129|41|48.24|55.95|19.02|0.00|779.82|1977.84|2293.95|54.58|0.00|779.82|834.40|-1198.02| +2451507|41532|1777|30460|1440572|2237|37949|4|30|1129|15|60.89|84.63|13.54|0.00|203.10|913.35|1269.45|12.18|0.00|203.10|215.28|-710.25| +2451507|41532|10910|30460|1440572|2237|37949|4|202|1129|58|54.31|69.51|18.76|0.00|1088.08|3149.98|4031.58|43.52|0.00|1088.08|1131.60|-2061.90| +2451507|41532|11029|30460|1440572|2237|37949|4|171|1129|68|74.52|74.52|43.22|0.00|2938.96|5067.36|5067.36|58.77|0.00|2938.96|2997.73|-2128.40| +2451507|41532|388|30460|1440572|2237|37949|4|170|1129|38|65.65|109.63|10.96|0.00|416.48|2494.70|4165.94|37.48|0.00|416.48|453.96|-2078.22| +2451507|41532|8311|30460|1440572|2237|37949|4|109|1129|3|88.20|105.84|69.85|0.00|209.55|264.60|317.52|4.19|0.00|209.55|213.74|-55.05| +||5083|30460|1440572|||4|10|1129||||1.15|||905.64||1.10|||14.90|| +2451507|41532|3052|30460|1440572||37949|4|297|1129|45|91.78|159.69|||||7186.05|93.81||||-2566.44| +2451993|43562|3405|50831|1262058|1285|32465|2|296|1130|74|91.00|148.33|90.48|2410.38|6695.52|6734.00|10976.42|299.95|2410.38|4285.14|4585.09|-2448.86| +|43562|13273|50831|1262058|||2||1130|76||6.62||0.00||479.56|||0.00|54.72||-424.84| +2451993|43562|17317|50831|1262058|1285|32465|2|90|1130|63|64.71|69.23|44.30|1618.72|2790.90|4076.73|4361.49|11.72|1618.72|1172.18|1183.90|-2904.55| +2451993|43562|14759|50831|1262058|1285|32465|2|286|1130|24|65.20|126.48|21.50|0.00|516.00|1564.80|3035.52|15.48|0.00|516.00|531.48|-1048.80| +2451993|43562|16427|50831|1262058|1285|32465|2|222|1130|66|78.95|112.10|34.75|0.00|2293.50|5210.70|7398.60|137.61|0.00|2293.50|2431.11|-2917.20| +2451993|43562|679|50831|1262058|1285|32465|2|87|1130|1|91.87|166.28|49.88|6.98|49.88|91.87|166.28|0.00|6.98|42.90|42.90|-48.97| +|43562|2375|50831||1285||||1130||53.75|83.85|37.73|0.00|3357.97||||0.00||3559.44|| +2451993|43562|12481|50831|1262058|1285|32465|2|56|1130|65|46.44|91.95|85.51|0.00|5558.15|3018.60|5976.75|500.23|0.00|5558.15|6058.38|2539.55| +2452613|66545|16644|49347|553076|5605|10087|10|7|1131|97|98.16|107.97|107.97|9425.78|10473.09|9521.52|10473.09|62.83|9425.78|1047.31|1110.14|-8474.21| +2452613|66545|3501|49347|553076|5605|10087|10|184|1131|45|42.24|71.38|68.52|0.00|3083.40|1900.80|3212.10|61.66|0.00|3083.40|3145.06|1182.60| +2452613|66545|10581|49347|553076|5605|10087|10|174|1131|67|57.00|77.52|23.25|358.28|1557.75|3819.00|5193.84|95.95|358.28|1199.47|1295.42|-2619.53| +2452613|66545|17082|49347|553076|5605|10087|10|166|1131|7|15.02|19.97|8.18|0.00|57.26|105.14|139.79|1.14|0.00|57.26|58.40|-47.88| +2452613|66545|3517|49347|553076|5605|10087|10|269|1131|2|66.06|79.27|77.68|32.62|155.36|132.12|158.54|2.45|32.62|122.74|125.19|-9.38| +||6258|||5605||10|270|1131|82|46.01|80.05||0.00|787.20||6564.10||0.00|||| +2452613|66545|11484|49347|553076|5605|10087|10|178|1131|2|14.76|16.38|10.31|0.00|20.62|29.52|32.76|1.64|0.00|20.62|22.26|-8.90| +2452613|66545|8731|49347|553076|5605|10087|10|128|1131|31|9.07|14.14|13.99|0.00|433.69|281.17|438.34|30.35|0.00|433.69|464.04|152.52| +2452613|66545|2046|49347|553076|5605|10087|10|6|1131|38|15.83|21.37|3.20|105.79|121.60|601.54|812.06|1.26|105.79|15.81|17.07|-585.73| +2452613|66545|15741|49347|553076|5605|10087|10|248|1131|53|87.68|128.01|97.28|0.00|5155.84|4647.04|6784.53|360.90|0.00|5155.84|5516.74|508.80| +2452613||6228|49347|553076|5605||||1131||82.12||||||1708.00|||1417.60||-224.80| +2452613|66545|17184|49347|553076|5605|10087|10|12|1131|3|6.83|7.51|5.85|0.00|17.55|20.49|22.53|0.00|0.00|17.55|17.55|-2.94| +2451830|41701|5627|74703|1271990|3324|37858|7|185|1132|31|21.70|29.72|25.85|0.00|801.35|672.70|921.32|56.09|0.00|801.35|857.44|128.65| +2451830|41701|5708|74703|1271990|3324|37858|7|262|1132|6|30.56|43.08|19.81|0.00|118.86|183.36|258.48|3.56|0.00|118.86|122.42|-64.50| +2451830|41701|5381|74703|1271990|3324|37858|7|71|1132|6|45.00|79.20|60.19|0.00|361.14|270.00|475.20|3.61|0.00|361.14|364.75|91.14| +2451830|41701|2563|74703|1271990|3324|37858|7|109|1132|99|93.47|107.49|52.67|0.00|5214.33|9253.53|10641.51|156.42|0.00|5214.33|5370.75|-4039.20| +2451830|41701|4271|74703|1271990|3324|37858|7|8|1132|37|86.72|133.54|37.39|0.00|1383.43|3208.64|4940.98|27.66|0.00|1383.43|1411.09|-1825.21| +2451830|41701|14857|74703|1271990|3324|37858|7|49|1132|13|87.62|156.83|97.23|0.00|1263.99|1139.06|2038.79|25.27|0.00|1263.99|1289.26|124.93| +2451830|41701|10625|74703|1271990|3324|37858|7|213|1132|17|46.40|87.23|10.46|0.00|177.82|788.80|1482.91|16.00|0.00|177.82|193.82|-610.98| +2451830|41701|13997|74703|1271990|3324|37858|7|274|1132|35|96.08|167.17|120.36|1263.78|4212.60|3362.80|5850.95|0.00|1263.78|2948.82|2948.82|-413.98| +2452402|35805|15427|41450|590235|3820|7758|4|202|1133|74|41.32|42.14|10.95|0.00|810.30|3057.68|3118.36|72.92|0.00|810.30|883.22|-2247.38| +2452402|35805|12576|41450|590235|3820|7758|4|13|1133|6|75.24|94.05|76.18|27.42|457.08|451.44|564.30|12.88|27.42|429.66|442.54|-21.78| +|35805|3900|41450|590235||||278|1133|64|7.77||||||586.24|18.62||||-186.88| +2452402|35805|523|41450|590235|3820|7758|4|121|1133|72|67.80|92.88|39.93|0.00|2874.96|4881.60|6687.36|201.24|0.00|2874.96|3076.20|-2006.64| +2452402|35805|361|41450|590235|3820|7758|4|40|1133|18|19.18|37.01|5.55|74.92|99.90|345.24|666.18|1.49|74.92|24.98|26.47|-320.26| +2452402|35805|4465|41450|590235|3820|7758|4|247|1133|88|49.02|50.00|1.00|38.72|88.00|4313.76|4400.00|1.47|38.72|49.28|50.75|-4264.48| +||9427||590235|||||1133|76|||||570.76|819.28||||570.76||| +2452402|35805|16009|41450|590235|3820|7758|4|113|1133|27|77.14|111.85|104.02|589.79|2808.54|2082.78|3019.95|44.37|589.79|2218.75|2263.12|135.97| +2452402|35805|4791|41450|590235|3820|7758|4|148|1133|41|10.78|12.72|0.38|0.00|15.58|441.98|521.52|0.15|0.00|15.58|15.73|-426.40| +2452402|35805|13459|41450|590235|3820|7758|4|201|1133|22|99.45|118.34|35.50|0.00|781.00|2187.90|2603.48|31.24|0.00|781.00|812.24|-1406.90| +2452402|35805|11148|41450|590235|3820|7758|4|195|1133|16|20.28|34.47|12.75|0.00|204.00|324.48|551.52|14.28|0.00|204.00|218.28|-120.48| +2452402|35805|7741|41450|590235|3820|7758|4|200|1133|59|91.67|128.33|59.03|0.00|3482.77|5408.53|7571.47|208.96|0.00|3482.77|3691.73|-1925.76| +2452402|35805|11694|41450|590235|3820|7758|4|96|1133|85|79.10|130.51|3.91|0.00|332.35|6723.50|11093.35|9.97|0.00|332.35|342.32|-6391.15| +2451339|62722|1720|22094|835840|3986|26932|2|173|1134|72|83.02|121.20|76.35|0.00|5497.20|5977.44|8726.40|494.74|0.00|5497.20|5991.94|-480.24| +2451339|62722|10466|22094|835840|3986|26932|2|36|1134|69|10.29|17.08|5.80|0.00|400.20|710.01|1178.52|36.01|0.00|400.20|436.21|-309.81| +||7670|22094||||2|13|1134||31.20|54.60||0.00||1591.20|2784.60||0.00||2310.81|719.61| +2451339|62722|11638|22094|835840|3986|26932|2|297|1134|29|12.31|12.92|9.17|0.00|265.93|356.99|374.68|5.31|0.00|265.93|271.24|-91.06| +2451339|62722|8752|22094|835840|3986|26932|2|202|1134|33|41.61|58.67|38.13|905.96|1258.29|1373.13|1936.11|17.61|905.96|352.33|369.94|-1020.80| +2451339|62722|12499|22094|835840|3986|26932|2|73|1134|31|78.88|142.77|69.95|0.00|2168.45|2445.28|4425.87|0.00|0.00|2168.45|2168.45|-276.83| +2451339|62722|14906|22094|835840|3986|26932|2|119|1134|13|84.14|128.73|59.21|0.00|769.73|1093.82|1673.49|53.88|0.00|769.73|823.61|-324.09| +2451339|62722|4712|22094|835840|3986|26932|2|10|1134|79|53.06|67.91|5.43|0.00|428.97|4191.74|5364.89|25.73|0.00|428.97|454.70|-3762.77| +2451339|62722|10045|22094|835840|3986|26932|2|134|1134|45|49.80|87.15|8.71|0.00|391.95|2241.00|3921.75|19.59|0.00|391.95|411.54|-1849.05| +2451339|62722|13652|22094|835840|3986|26932|2|27|1134|85|55.21|69.56|13.91|0.00|1182.35|4692.85|5912.60|106.41|0.00|1182.35|1288.76|-3510.50| +2451339|62722|799|22094|835840|3986|26932|2|84|1134|31|24.46|39.38|2.75|0.00|85.25|758.26|1220.78|4.26|0.00|85.25|89.51|-673.01| +2451339|62722|16027|22094|835840|3986|26932|2|228|1134|99|14.39|28.34|13.31|0.00|1317.69|1424.61|2805.66|0.00|0.00|1317.69|1317.69|-106.92| +2451455|43206|8096|18549|1519721|4893|39665|4|222|1135|78|20.66|39.46|10.25|0.00|799.50|1611.48|3077.88|23.98|0.00|799.50|823.48|-811.98| +2451455|43206|13394|18549|1519721|4893|39665|4|287|1135|82|22.97|39.73|10.32|338.49|846.24|1883.54|3257.86|30.46|338.49|507.75|538.21|-1375.79| +2451455|43206|17455|18549|1519721|4893|39665|4|254|1135|87|26.43|29.33|21.70|0.00|1887.90|2299.41|2551.71|113.27|0.00|1887.90|2001.17|-411.51| +2451455|43206|2353|18549|1519721|4893|39665|4|19|1135|62|80.57|161.14|24.17|0.00|1498.54|4995.34|9990.68|0.00|0.00|1498.54|1498.54|-3496.80| +2451455|43206|3811|18549|1519721|4893|39665|4|269|1135|39|41.78|72.27|59.26|0.00|2311.14|1629.42|2818.53|46.22|0.00|2311.14|2357.36|681.72| +2451455|43206|4538|18549|1519721|4893|39665|4|283|1135|42|4.69|5.29|0.68|22.84|28.56|196.98|222.18|0.17|22.84|5.72|5.89|-191.26| +|43206|5374|18549|1519721|4893||4|94|1135||26.85||||346.58|349.05|450.19|20.79||||-2.47| +2451455|43206|10882|18549|1519721|4893|39665|4|274|1135|56|42.98|44.26|40.71|478.74|2279.76|2406.88|2478.56|0.00|478.74|1801.02|1801.02|-605.86| +2451455|43206|394|18549|1519721|4893|39665|4|17|1135|29|41.61|73.64|50.07|0.00|1452.03|1206.69|2135.56|0.00|0.00|1452.03|1452.03|245.34| +2451455|43206|16066|18549|1519721|4893|39665|4|279|1135|28|4.54|5.17|3.20|88.70|89.60|127.12|144.76|0.07|88.70|0.90|0.97|-126.22| +2451455|43206|2836|18549|1519721|4893|39665|4|10|1135|54|96.87|146.27|93.61|0.00|5054.94|5230.98|7898.58|303.29|0.00|5054.94|5358.23|-176.04| +2451455|43206|15572|18549|1519721|4893|39665|4|294|1135|6|82.61|128.87|55.41|0.00|332.46|495.66|773.22|0.00|0.00|332.46|332.46|-163.20| +2451670|64243|2689|98135|1159874|6766|13305|10|79|1136|5|23.49|46.27|3.23|0.00|16.15|117.45|231.35|0.80|0.00|16.15|16.95|-101.30| +2451670|64243|9464|98135|1159874|6766|13305|10|221|1136|56|60.55|68.42|34.89|0.00|1953.84|3390.80|3831.52|19.53|0.00|1953.84|1973.37|-1436.96| +2451670|64243|9449|98135|1159874|6766|13305|10|172|1136|48|55.77|102.61|75.93|0.00|3644.64|2676.96|4925.28|255.12|0.00|3644.64|3899.76|967.68| +2451670|64243|10697|98135|1159874|6766|13305|10|16|1136|94|36.06|58.05|23.80|0.00|2237.20|3389.64|5456.70|67.11|0.00|2237.20|2304.31|-1152.44| +2451670|64243|14558|98135|1159874|6766|13305|10|209|1136|19|49.69|88.44|61.02|0.00|1159.38|944.11|1680.36|23.18|0.00|1159.38|1182.56|215.27| +2451670|64243|17695|98135|1159874|6766|13305|10|55|1136|66|9.72|16.91|12.51|0.00|825.66|641.52|1116.06|74.30|0.00|825.66|899.96|184.14| +2451670|64243|8702|98135|1159874|6766|13305|10|158|1136|22|98.71|175.70|70.28|0.00|1546.16|2171.62|3865.40|77.30|0.00|1546.16|1623.46|-625.46| +2451670|64243|8275|98135|1159874|6766|13305|10|76|1136|39|35.00|57.40|17.79|0.00|693.81|1365.00|2238.60|48.56|0.00|693.81|742.37|-671.19| +2451670|64243|10952|98135|1159874|6766|13305|10|101|1136|87|13.36|24.31|18.96|0.00|1649.52|1162.32|2114.97|65.98|0.00|1649.52|1715.50|487.20| +2451670|64243|2990|98135|1159874|6766||||1136|3|||41.12|14.80||231.06||9.77|14.80||118.33|| +2451670|64243|3155|98135|1159874|6766|13305|10|191|1136|82|85.91|161.51|71.06|0.00|5826.92|7044.62|13243.82|349.61|0.00|5826.92|6176.53|-1217.70| +2451670|64243|13363|98135|1159874|6766|13305|10|224|1136|61|96.66|129.52|94.54|1557.07|5766.94|5896.26|7900.72|294.69|1557.07|4209.87|4504.56|-1686.39| +2451670|64243|10621|98135|1159874|6766|13305|10|20|1136|77|68.65|82.38|25.53|157.26|1965.81|5286.05|6343.26|144.68|157.26|1808.55|1953.23|-3477.50| +2451670|64243|8306|98135|1159874|6766|13305|10|55|1136|64|44.43|80.41|69.95|0.00|4476.80|2843.52|5146.24|402.91|0.00|4476.80|4879.71|1633.28| +2452448|42149|11292|28317|1154166|4815|13064|10|154|1137|70|31.58|54.31|26.06|0.00|1824.20|2210.60|3801.70|0.00|0.00|1824.20|1824.20|-386.40| +2452448|42149|13722|28317|1154166|4815|13064|10|1|1137|19|30.16|57.60|24.19|0.00|459.61|573.04|1094.40|22.98|0.00|459.61|482.59|-113.43| +2452448|42149|2047|28317|1154166|4815|13064|10|224|1137|82|43.31|61.50|43.05|1094.33|3530.10|3551.42|5043.00|97.43|1094.33|2435.77|2533.20|-1115.65| +2452448|42149|5622|28317|1154166|4815|13064|10|38|1137|24|8.80|13.55|11.78|79.16|282.72|211.20|325.20|0.00|79.16|203.56|203.56|-7.64| +2452448|42149|14985|28317|1154166|4815|13064|10|13|1137|25|88.89|106.66|103.46|0.00|2586.50|2222.25|2666.50|77.59|0.00|2586.50|2664.09|364.25| +|42149|17178|28317||4815||10|136|1137|||126.50||0.00||5921.52|11132.00||0.00||11354.64|5210.48| +2452448|42149|15060|28317|1154166|4815|13064|10|121|1137|45|9.81|17.36|3.29|146.56|148.05|441.45|781.20|0.07|146.56|1.49|1.56|-439.96| +2452448|42149|1767|28317|1154166|4815|13064|10|124|1137|48|48.08|78.85|39.42|0.00|1892.16|2307.84|3784.80|18.92|0.00|1892.16|1911.08|-415.68| +2452448|42149|10119|28317|1154166|4815|13064|10|284|1137|15|59.44|69.54|36.16|0.00|542.40|891.60|1043.10|37.96|0.00|542.40|580.36|-349.20| +2452448|42149|8046|28317|1154166|4815|13064|10|242|1137|37|83.32|121.64|17.02|0.00|629.74|3082.84|4500.68|37.78|0.00|629.74|667.52|-2453.10| +2452448|42149|15175|28317|1154166|4815|13064|10|214|1137|19|28.31|56.62|56.05|0.00|1064.95|537.89|1075.78|95.84|0.00|1064.95|1160.79|527.06| +2452448|42149|13111|28317|1154166|4815|13064|10|295|1137|11|86.79|91.12|16.40|0.00|180.40|954.69|1002.32|0.00|0.00|180.40|180.40|-774.29| +2452448|42149|7008|28317|1154166|4815|13064|10|235|1137|31|97.45|168.58|92.71|0.00|2874.01|3020.95|5225.98|86.22|0.00|2874.01|2960.23|-146.94| +2452448|42149|2305|28317|1154166|4815|13064|10|71|1137|91|50.22|85.37|47.80|0.00|4349.80|4570.02|7768.67|130.49|0.00|4349.80|4480.29|-220.22| +2451275|48488|4598|20033|1612735|637|10939|8|215|1138|47|69.76|103.24|7.22|0.00|339.34|3278.72|4852.28|10.18|0.00|339.34|349.52|-2939.38| +2451275|48488|14047|20033|1612735|637|10939|8|60|1138|77|8.94|12.87|8.75|0.00|673.75|688.38|990.99|13.47|0.00|673.75|687.22|-14.63| +2451275|48488|16273|20033|1612735|637|10939|8|288|1138|6|63.91|100.33|29.09|0.00|174.54|383.46|601.98|5.23|0.00|174.54|179.77|-208.92| +2451275|48488|16696|20033|1612735|637|10939|8|42|1138|71|64.08|124.95|37.48|0.00|2661.08|4549.68|8871.45|79.83|0.00|2661.08|2740.91|-1888.60| +2451275|48488|1345|20033|1612735|637|10939|8|100|1138|32|97.50|176.47|105.88|0.00|3388.16|3120.00|5647.04|237.17|0.00|3388.16|3625.33|268.16| +2451275|48488|12034|20033|1612735|637|10939|8|170|1138|58|93.89|183.08|119.00|0.00|6902.00|5445.62|10618.64|207.06|0.00|6902.00|7109.06|1456.38| +2451275|48488|14230|20033|1612735|637|10939|8|180|1138|36|56.79|86.32|75.96|0.00|2734.56|2044.44|3107.52|27.34|0.00|2734.56|2761.90|690.12| +2451275|48488|14374|20033|1612735|637|10939|8|61|1138|88|41.47|51.42|7.19|512.50|632.72|3649.36|4524.96|10.81|512.50|120.22|131.03|-3529.14| +2451275|48488|11341|20033|1612735|637|10939|8|58|1138|16|72.63|135.09|36.47|0.00|583.52|1162.08|2161.44|40.84|0.00|583.52|624.36|-578.56| +2451275||2582|20033||637||8||1138||69.78|||27.31|||3126.08||27.31|3.73||-2229.23| +|52651|471|55214||3940|26750|8||1139||55.54|||||277.70|324.90|||||| +2451980|52651|17903|55214|603977|3940|26750|8|202|1139|42|91.80|133.11|127.78|5098.42|5366.76|3855.60|5590.62|18.78|5098.42|268.34|287.12|-3587.26| +2451980|52651|519|55214|603977|3940|26750|8|120|1139|61|95.40|186.98|162.67|0.00|9922.87|5819.40|11405.78|793.82|0.00|9922.87|10716.69|4103.47| +|52651|15015||603977|3940|26750||255|1139|32|10.51|21.02|17.23|363.89||336.32|672.64||363.89|||| +2451980|52651|4291|55214|603977|3940|26750|8|265|1139|69|50.68|75.00|46.50|0.00|3208.50|3496.92|5175.00|96.25|0.00|3208.50|3304.75|-288.42| +2451980|52651|14717|55214|603977|3940|26750|8|281|1139|80|2.21|3.38|0.40|0.00|32.00|176.80|270.40|0.00|0.00|32.00|32.00|-144.80| +2451980|52651|851|55214|603977||26750||150|1139|59|54.92|57.66|47.85|1778.58|2823.15||3401.94||1778.58|1044.57||-2195.71| +2451980|52651|4603|55214|603977|3940|26750|8|259|1139|24|34.97|63.99|20.47|103.16|491.28|839.28|1535.76|3.88|103.16|388.12|392.00|-451.16| +2451980|52651|275|55214|603977|3940|26750|8|173|1139|27|52.72|78.02|76.45|0.00|2064.15|1423.44|2106.54|165.13|0.00|2064.15|2229.28|640.71| +2451980|52651|17621|55214|603977|3940|26750|8|200|1139|18|64.43|126.28|1.26|0.00|22.68|1159.74|2273.04|0.90|0.00|22.68|23.58|-1137.06| +2451980|52651|11969|55214|603977|3940|26750|8|229|1139|90|69.58|139.16|111.32|0.00|10018.80|6262.20|12524.40|0.00|0.00|10018.80|10018.80|3756.60| +2451980|52651|16277|55214|603977|3940|26750|8|87|1139|87|25.81|51.10|2.04|0.00|177.48|2245.47|4445.70|1.77|0.00|177.48|179.25|-2067.99| +2451197|63929|12290|85247|826272|3744|8016|10|299|1140|70|2.76|3.86|3.70|0.00|259.00|193.20|270.20|0.00|0.00|259.00|259.00|65.80| +2451197|63929|1526|85247|826272|3744|8016|10|118|1140|49|25.38|43.14|3.01|0.00|147.49|1243.62|2113.86|5.89|0.00|147.49|153.38|-1096.13| +||12016|85247||3744||10|264|1140|36|35.75|||0.00|742.68|1287.00|||0.00||787.24|-544.32| +2451197|63929|16324|85247|826272|3744|8016|10|43|1140|39|83.44|166.04|116.22|1223.79|4532.58|3254.16|6475.56|0.00|1223.79|3308.79|3308.79|54.63| +2451197|63929|4118|85247|826272|3744|8016|10|227|1140|62|67.13|107.40|37.59|0.00|2330.58|4162.06|6658.80|93.22|0.00|2330.58|2423.80|-1831.48| +2451197|63929|5197|85247|826272|3744|8016|10|284|1140|41|12.31|14.15|1.27|0.00|52.07|504.71|580.15|4.68|0.00|52.07|56.75|-452.64| +2451197|63929|2312|85247|826272|3744|8016|10|162|1140|65|46.84|79.15|59.36|0.00|3858.40|3044.60|5144.75|270.08|0.00|3858.40|4128.48|813.80| +2451197|63929|13814|85247|826272|3744|8016|10|51|1140|40|34.41|40.94|18.83|0.00|753.20|1376.40|1637.60|7.53|0.00|753.20|760.73|-623.20| +2451197|63929|5449|85247|826272|3744|8016|10|148|1140|93|82.36|91.41|45.70|0.00|4250.10|7659.48|8501.13|212.50|0.00|4250.10|4462.60|-3409.38| +2451197|63929|1682|85247|826272|3744|8016|10|217|1140|69|37.59|40.22|0.40|0.00|27.60|2593.71|2775.18|0.00|0.00|27.60|27.60|-2566.11| +2451197|63929|6631|85247|826272|3744|8016|10|23|1140|26|93.29|166.98|125.23|0.00|3255.98|2425.54|4341.48|195.35|0.00|3255.98|3451.33|830.44| +2451197|63929|4666|85247|826272|3744|8016|10|215|1140|30|38.64|59.50|48.19|28.91|1445.70|1159.20|1785.00|85.00|28.91|1416.79|1501.79|257.59| +2451197|63929|10496|85247|826272|3744|8016|10|91|1140|39|60.28|118.75|108.06|0.00|4214.34|2350.92|4631.25|0.00|0.00|4214.34|4214.34|1863.42| +2451346|62107|493|6746|194367|6435|33303|10|199|1141|63|64.24|120.12|92.49|0.00|5826.87|4047.12|7567.56|291.34|0.00|5826.87|6118.21|1779.75| +2451346|62107|11899|6746|194367|6435|33303|10|146|1141|4|79.00|92.43|17.56|0.00|70.24|316.00|369.72|0.70|0.00|70.24|70.94|-245.76| +|62107|17275|6746|194367||33303||231|1141|34|88.56|154.98|54.24||1844.16|3011.04||73.76||1844.16|1917.92|-1166.88| +2451346|62107|6448|6746|194367|6435|33303|10|297|1141|94|61.26|113.33|84.99|2316.82|7989.06|5758.44|10653.02|397.05|2316.82|5672.24|6069.29|-86.20| +2451346|62107|13048|6746|194367|6435|33303|10|110|1141|49|52.52|101.88|14.26|0.00|698.74|2573.48|4992.12|62.88|0.00|698.74|761.62|-1874.74| +2451346|62107|7400|6746|194367|6435|33303|10|187|1141|38|20.47|39.71|27.00|0.00|1026.00|777.86|1508.98|0.00|0.00|1026.00|1026.00|248.14| +2451346|62107|17683||194367|6435|33303|||1141|31|46.69|75.63||0.00|375.10||||0.00|||| +2451346|62107|4678|6746|194367|6435|33303|10|174|1141|94|83.61|155.51|82.42|0.00|7747.48|7859.34|14617.94|619.79|0.00|7747.48|8367.27|-111.86| +2451346|62107|6811|6746|194367|6435|33303|10|266|1141|57|51.93|81.53|40.76|0.00|2323.32|2960.01|4647.21|69.69|0.00|2323.32|2393.01|-636.69| +2451346|62107|13177|6746|194367|6435|33303|10|26|1141|1|61.46|122.92|63.91|0.00|63.91|61.46|122.92|4.47|0.00|63.91|68.38|2.45| +2451737|38020|12182|4989|1611976|2963|34750|1|186|1142|11|37.83|58.25|28.54|0.00|313.94|416.13|640.75|18.83|0.00|313.94|332.77|-102.19| +2451737|38020|890||1611976||34750|1||1142||12.33|22.07|12.35|0.00||937.08|||0.00|||1.52| +2451737|38020|16739|4989|1611976|2963|34750|1|64|1142|8|7.34|11.81|3.07|0.00|24.56|58.72|94.48|0.00|0.00|24.56|24.56|-34.16| +2451737|38020|2083|4989|1611976|2963|34750|1|204|1142|26|65.26|73.09|45.31|0.00|1178.06|1696.76|1900.34|70.68|0.00|1178.06|1248.74|-518.70| +2451737|38020|7805|4989|1611976|2963|34750|1|181|1142|23|29.43|44.73|0.00|0.00|0.00|676.89|1028.79|0.00|0.00|0.00|0.00|-676.89| +2451737|38020|10457|4989|1611976|2963|34750|1|44|1142|51|8.85|17.08|1.36|11.79|69.36|451.35|871.08|4.02|11.79|57.57|61.59|-393.78| +2451737|38020|5075|4989|1611976|2963|34750|1|65|1142|41|51.46|78.73|68.49|0.00|2808.09|2109.86|3227.93|56.16|0.00|2808.09|2864.25|698.23| +2451737|38020|16769|4989|1611976|2963|34750|1|79|1142|59|76.23|88.42|78.69|0.00|4642.71|4497.57|5216.78|185.70|0.00|4642.71|4828.41|145.14| +2451737|38020|9359|4989|1611976|2963|34750|1|155|1142|21|69.77|89.30|51.79|0.00|1087.59|1465.17|1875.30|21.75|0.00|1087.59|1109.34|-377.58| +2451737|38020|3823|4989|1611976|2963|34750|1|257|1142|63|1.31|2.18|1.96|0.00|123.48|82.53|137.34|0.00|0.00|123.48|123.48|40.95| +2451737|38020|2323|4989|1611976|2963|34750|1|37|1142|14|2.09|4.07|3.21|0.00|44.94|29.26|56.98|0.00|0.00|44.94|44.94|15.68| +2451737|38020|3169|4989|1611976|2963|34750|1|84|1142|94|32.59|40.41|21.41|1650.28|2012.54|3063.46|3798.54|14.49|1650.28|362.26|376.75|-2701.20| +2451737|38020|7481|4989|1611976|2963|34750|1|67|1142|56|58.89|109.53|101.86|0.00|5704.16|3297.84|6133.68|342.24|0.00|5704.16|6046.40|2406.32| +2452320|35226|14001|50189|1620492|4395|30298|1|55|1143|74|68.21|70.25|11.94|0.00|883.56|5047.54|5198.50|26.50|0.00|883.56|910.06|-4163.98| +2452320|35226|643|50189|1620492||30298|1|233|1143|30|||20.84|0.00||1364.10||18.75|0.00|625.20||| +2452320|35226|3379|50189|1620492|4395|30298|1|204|1143|90|27.80|45.31|12.23|0.00|1100.70|2502.00|4077.90|88.05|0.00|1100.70|1188.75|-1401.30| +||9279|||||1|53|1143|53|80.31|||||4256.43|7916.61|53.85|||823.26|-3487.02| +2452320|35226|15648|50189|1620492|4395|30298|1|131|1143|70|49.57|60.47|42.32|0.00|2962.40|3469.90|4232.90|236.99|0.00|2962.40|3199.39|-507.50| +2452320|35226|10341|50189|1620492|4395|30298|1|258|1143|53|73.73|115.01|52.90|0.00|2803.70|3907.69|6095.53|28.03|0.00|2803.70|2831.73|-1103.99| +2452320|35226|13413|50189|1620492|4395|30298|1|91|1143|81|96.67|122.77|65.06|0.00|5269.86|7830.27|9944.37|52.69|0.00|5269.86|5322.55|-2560.41| +2452320|35226|9925|50189|1620492|4395|30298|1|189|1143|93|87.05|138.40|60.89|0.00|5662.77|8095.65|12871.20|113.25|0.00|5662.77|5776.02|-2432.88| +2452320|35226|2161|50189|1620492|4395|30298|1|169|1143|98|17.72|21.61|12.53|0.00|1227.94|1736.56|2117.78|36.83|0.00|1227.94|1264.77|-508.62| +2452320|35226|6145|50189|1620492|4395|30298|1|279|1143|70|7.39|11.23|11.23|0.00|786.10|517.30|786.10|0.00|0.00|786.10|786.10|268.80| +|35226|15727||1620492|4395|||259|1143||||||995.52|1167.36||79.64||995.52|1075.16|-171.84| +2452320|35226|5226|50189|1620492|4395|30298|1|112|1143|9|2.73|4.61|3.45|0.00|31.05|24.57|41.49|1.24|0.00|31.05|32.29|6.48| +2452320|35226|3853|50189|1620492|4395|30298|1|212|1143|41|22.11|26.53|2.12|0.00|86.92|906.51|1087.73|6.95|0.00|86.92|93.87|-819.59| +2452320|35226|14802|50189|1620492|4395|30298|1|269|1143|45|93.00|157.17|77.01|0.00|3465.45|4185.00|7072.65|0.00|0.00|3465.45|3465.45|-719.55| +2452320|35226|9655|50189|1620492|4395|30298|1|228|1143|40|84.20|91.77|80.75|0.00|3230.00|3368.00|3670.80|290.70|0.00|3230.00|3520.70|-138.00| +2452320|35226|5511|50189|1620492|4395|30298|1|156|1143|11|21.40|35.95|15.09|131.13|165.99|235.40|395.45|0.00|131.13|34.86|34.86|-200.54| +2451249|35147|17035|1515|1475478|1088|28251|10|228|1144|28|90.01|112.51|103.50|0.00|2898.00|2520.28|3150.28|260.82|0.00|2898.00|3158.82|377.72| +2451249|35147|284|1515|1475478|1088|28251|10|114|1144|32|21.40|31.03|24.51|321.57|784.32|684.80|992.96|18.51|321.57|462.75|481.26|-222.05| +2451249|35147|16946|1515|1475478|1088|28251|10|258|1144|62|40.89|72.78|40.02|1389.49|2481.24|2535.18|4512.36|0.00|1389.49|1091.75|1091.75|-1443.43| +2451249|35147|12154|1515|1475478|1088|28251|10|169|1144|84|25.17|31.21|19.35|0.00|1625.40|2114.28|2621.64|0.00|0.00|1625.40|1625.40|-488.88| +2451249|35147|4033|1515|1475478|1088|28251|10|120|1144|95|78.49|87.90|29.88|0.00|2838.60|7456.55|8350.50|113.54|0.00|2838.60|2952.14|-4617.95| +2451249|35147|17258|1515|1475478|1088|28251|10|183|1144|54|7.35|13.81|5.93|0.00|320.22|396.90|745.74|6.40|0.00|320.22|326.62|-76.68| +2451249|35147|2306|1515|1475478|1088|28251|10|48|1144|99|22.63|39.14|39.14|0.00|3874.86|2240.37|3874.86|348.73|0.00|3874.86|4223.59|1634.49| +2451249|35147|10729|1515|1475478|1088|28251|10|291|1144|36|40.17|63.06|18.28|0.00|658.08|1446.12|2270.16|32.90|0.00|658.08|690.98|-788.04| +2451249|35147|10405|1515|1475478|1088|28251|10|88|1144|60|27.61|43.89|25.89|0.00|1553.40|1656.60|2633.40|46.60|0.00|1553.40|1600.00|-103.20| +2452233|51326|9785|77735|1827473|1564|45655|8|125|1145|56|52.57|68.86|19.96|0.00|1117.76|2943.92|3856.16|22.35|0.00|1117.76|1140.11|-1826.16| +2452233|51326|7989|77735|1827473|1564|45655|8|105|1145|7|80.33|120.49|87.95|0.00|615.65|562.31|843.43|18.46|0.00|615.65|634.11|53.34| +2452233|51326|17871|77735|1827473|1564|45655|8|234|1145|65|70.13|110.10|99.09|0.00|6440.85|4558.45|7156.50|257.63|0.00|6440.85|6698.48|1882.40| +2452233|51326|6457|77735|1827473|1564|45655|8|72|1145|17|56.55|59.37|21.37|0.00|363.29|961.35|1009.29|25.43|0.00|363.29|388.72|-598.06| +2452233|51326|2075|77735|1827473|1564|45655|8|30|1145|31|55.42|96.98|7.75|0.00|240.25|1718.02|3006.38|21.62|0.00|240.25|261.87|-1477.77| +2452233|51326|10839|77735|1827473|1564|45655|8|176|1145|80|87.15|173.42|31.21|0.00|2496.80|6972.00|13873.60|124.84|0.00|2496.80|2621.64|-4475.20| +2452233|51326|14411|77735|1827473|1564|45655|8|65|1145|20|79.91|136.64|94.28|0.00|1885.60|1598.20|2732.80|150.84|0.00|1885.60|2036.44|287.40| +2452233|51326|17689|77735|1827473|1564|45655|8|141|1145|93|40.55|70.96|1.41|0.00|131.13|3771.15|6599.28|7.86|0.00|131.13|138.99|-3640.02| +2452233|51326|11003|77735|1827473|1564|45655|8|119|1145|75|64.49|107.05|81.35|0.00|6101.25|4836.75|8028.75|244.05|0.00|6101.25|6345.30|1264.50| +2452233||4905||1827473|||||1145|48||13.35|||||||||63.84|| +2452233|51326|4541|77735|1827473|1564|45655|8|74|1145|76|1.88|3.60|3.45|0.00|262.20|142.88|273.60|20.97|0.00|262.20|283.17|119.32| +|51326|9051|77735||1564||8||1145||||||379.89|935.97|1310.19|||||-556.08| +2452039|37234|3611|48656|1616121|5986|49451|2|188|1146|37|8.93|15.35|14.42|0.00|533.54|330.41|567.95|16.00|0.00|533.54|549.54|203.13| +2452039|37234|1743|48656|1616121|5986|49451|2|296|1146|51|68.28|96.95|45.56|0.00|2323.56|3482.28|4944.45|46.47|0.00|2323.56|2370.03|-1158.72| +2452039|37234|1113|48656|1616121|5986|49451|2|224|1146|79|62.17|107.55|103.24|0.00|8155.96|4911.43|8496.45|489.35|0.00|8155.96|8645.31|3244.53| +2452039|37234|9675|48656|1616121|5986|49451|2|160|1146|45|50.54|80.86|25.06|191.70|1127.70|2274.30|3638.70|18.72|191.70|936.00|954.72|-1338.30| +2452039|37234|9657|48656|1616121|5986|49451|2|231|1146|13|61.43|93.37|42.01|0.00|546.13|798.59|1213.81|0.00|0.00|546.13|546.13|-252.46| +2452039|37234|14517|48656|1616121|5986|49451|2|247|1146|57|88.69|119.73|27.53|0.00|1569.21|5055.33|6824.61|125.53|0.00|1569.21|1694.74|-3486.12| +2452039|37234|9263|48656|1616121|5986|49451|2|43|1146|72|90.27|136.30|49.06|0.00|3532.32|6499.44|9813.60|70.64|0.00|3532.32|3602.96|-2967.12| +2452039||6245|48656||5986|||136|1146|62|56.76|70.38||0.00|2137.76|3519.12|4363.56|106.88|0.00|2137.76||-1381.36| +2452039|37234|11393|48656|1616121|5986|49451|2|52|1146|92|97.35|100.27|78.21|5108.67|7195.32|8956.20|9224.84|20.86|5108.67|2086.65|2107.51|-6869.55| +2452039|37234|3601|48656|1616121|5986|49451|2|110|1146|37|78.16|106.29|27.63|0.00|1022.31|2891.92|3932.73|71.56|0.00|1022.31|1093.87|-1869.61| +2452039|37234|2275|48656|1616121|5986|49451|2|285|1146|7|4.63|5.78|2.36|0.00|16.52|32.41|40.46|1.48|0.00|16.52|18.00|-15.89| +2452039|37234|17835|48656|1616121|5986|49451|2|80|1146|21|8.67|15.86|2.06|0.00|43.26|182.07|333.06|0.86|0.00|43.26|44.12|-138.81| +2452039|37234|611|48656|1616121|5986|49451|2|276|1146|48|64.30|124.74|1.24|0.00|59.52|3086.40|5987.52|5.35|0.00|59.52|64.87|-3026.88| +2451140|53545|5662|7215|1026672|4082|339|10|108|1147|64|86.51|135.82|29.88|0.00|1912.32|5536.64|8692.48|57.36|0.00|1912.32|1969.68|-3624.32| +2451140|53545|13970|7215|1026672|4082|339|10|89|1147|12|93.84|137.94|44.14|0.00|529.68|1126.08|1655.28|10.59|0.00|529.68|540.27|-596.40| +2451140|53545|5818|7215|1026672|4082|339|10|117|1147|1|43.44|54.73|49.25|0.00|49.25|43.44|54.73|3.44|0.00|49.25|52.69|5.81| +2451140|53545|9176|7215|1026672|4082|339|10|136|1147|94|13.26|25.72|15.68|29.47|1473.92|1246.44|2417.68|101.11|29.47|1444.45|1545.56|198.01| +2451140|53545|15620|7215|1026672|4082|339|10|282|1147|74|99.07|183.27|109.96|0.00|8137.04|7331.18|13561.98|244.11|0.00|8137.04|8381.15|805.86| +2451140|53545|15742||1026672|||10|34|1147|56|91.84||48.48|0.00|2714.88||||0.00|||| +2451140|53545|5372|7215|1026672|4082|339|10|294|1147|59|27.49|51.13|31.70|0.00|1870.30|1621.91|3016.67|112.21|0.00|1870.30|1982.51|248.39| +2451140|53545|638|7215|1026672|4082|339|10|261|1147|19|20.81|31.42|0.94|0.00|17.86|395.39|596.98|0.00|0.00|17.86|17.86|-377.53| +2451140|53545|16219|7215|1026672|4082|339|10|243|1147|39|14.85|15.14|11.20|0.00|436.80|579.15|590.46|34.94|0.00|436.80|471.74|-142.35| +2451140|53545|8920|7215|1026672|4082|339|10|168|1147|93|60.32|63.93|20.45|0.00|1901.85|5609.76|5945.49|38.03|0.00|1901.85|1939.88|-3707.91| +2451140|53545|3982|7215|1026672|4082|339|10|67|1147|13|82.89|138.42|137.03|0.00|1781.39|1077.57|1799.46|124.69|0.00|1781.39|1906.08|703.82| +2451140|53545|12817|7215|1026672|4082|339|10|146|1147|78|36.88|64.17|60.96|0.00|4754.88|2876.64|5005.26|0.00|0.00|4754.88|4754.88|1878.24| +2451140|53545|15040|7215|1026672|4082|339|10|228|1147|17|10.17|12.81|10.24|134.04|174.08|172.89|217.77|1.60|134.04|40.04|41.64|-132.85| +2451140|53545|17140|7215|1026672|4082|339|10|179|1147|42|11.02|21.26|17.22|0.00|723.24|462.84|892.92|57.85|0.00|723.24|781.09|260.40| +2451140|53545|1915|7215|1026672|4082|339|10|252|1147|1|65.66|66.31|33.15|0.00|33.15|65.66|66.31|1.98|0.00|33.15|35.13|-32.51| +2451140|53545|16705|7215|1026672|4082|339|10|293|1147|35|12.54|16.55|12.24|0.00|428.40|438.90|579.25|17.13|0.00|428.40|445.53|-10.50| +2451157|41448|13400|51183|1673107|184|23308|7|106|1148|13|82.03|95.97|95.01|0.00|1235.13|1066.39|1247.61|111.16|0.00|1235.13|1346.29|168.74| +2451157|41448|7765|51183|1673107|184|23308|7|53|1148|97|50.21|85.85|8.58|0.00|832.26|4870.37|8327.45|58.25|0.00|832.26|890.51|-4038.11| +2451157|41448|1603|51183|1673107|184|23308|7|188|1148|94|15.68|21.16|9.31|0.00|875.14|1473.92|1989.04|0.00|0.00|875.14|875.14|-598.78| +2451157|41448|5162|51183|1673107|184|23308|7|40|1148|29|14.82|22.23|21.56|0.00|625.24|429.78|644.67|56.27|0.00|625.24|681.51|195.46| +2451157|41448|2296|51183|1673107|184|23308|7|140|1148|7|55.64|91.80|11.93|0.00|83.51|389.48|642.60|4.17|0.00|83.51|87.68|-305.97| +2451157|41448|17426|51183|1673107|184|23308|7|60|1148|34|92.86|105.86|51.87|0.00|1763.58|3157.24|3599.24|88.17|0.00|1763.58|1851.75|-1393.66| +2451157|41448|17600|51183|1673107|184|23308|7|28|1148|75|70.38|99.93|95.93|0.00|7194.75|5278.50|7494.75|575.58|0.00|7194.75|7770.33|1916.25| +2451157|41448|12724|51183|1673107|184|23308|7|230|1148|30|88.19|118.17|46.08|0.00|1382.40|2645.70|3545.10|96.76|0.00|1382.40|1479.16|-1263.30| +2451157|41448|2564|51183|1673107|184|23308|7|141|1148|84|53.90|75.99|36.47|0.00|3063.48|4527.60|6383.16|30.63|0.00|3063.48|3094.11|-1464.12| +2451157|41448|6763|51183|1673107|184|23308|7|107|1148|60|25.52|30.87|7.40|261.96|444.00|1531.20|1852.20|14.56|261.96|182.04|196.60|-1349.16| +||2035|51183||184|23308|||1148|81||||0.00|5181.57||9253.44|207.26|0.00|5181.57|5388.83|| +2451157|41448|1387|51183|1673107|184|23308|7|115|1148|56|31.41|43.03|3.87|0.00|216.72|1758.96|2409.68|0.00|0.00|216.72|216.72|-1542.24| +2451157|41448|1820|51183|1673107|184|23308|7|12|1148|9|96.20|150.07|141.06|558.59|1269.54|865.80|1350.63|63.98|558.59|710.95|774.93|-154.85| +2451157|41448|12526|51183|1673107|184|23308|7|297|1148|80|80.44|102.96|98.84|0.00|7907.20|6435.20|8236.80|158.14|0.00|7907.20|8065.34|1472.00| +2451644|62558|17479|13011|1334340|916|42645|2|17|1149|77|37.47|61.07|36.64|2144.17|2821.28|2885.19|4702.39|54.16|2144.17|677.11|731.27|-2208.08| +2451644|62558|824|13011|1334340|916|42645|2|179|1149|35|17.24|28.61|8.58|222.22|300.30|603.40|1001.35|2.34|222.22|78.08|80.42|-525.32| +2451644|62558|701|13011|1334340|916|42645|2|5|1149|65|75.76|93.94|42.27|0.00|2747.55|4924.40|6106.10|0.00|0.00|2747.55|2747.55|-2176.85| +2451644|62558|3860|13011|1334340|916|42645|2|64|1149|85|52.43|94.37|0.94|27.96|79.90|4456.55|8021.45|4.67|27.96|51.94|56.61|-4404.61| +2451644|62558|14737|13011|1334340|916|42645|2|146|1149|87|32.15|50.47|48.95|0.00|4258.65|2797.05|4390.89|0.00|0.00|4258.65|4258.65|1461.60| +2451644|62558|16973|13011|1334340|916|42645|2|6|1149|64|44.17|57.42|55.69|2673.12|3564.16|2826.88|3674.88|44.55|2673.12|891.04|935.59|-1935.84| +2451644|62558|12269|13011|1334340|916|42645|2|18|1149|98|57.47|60.34|52.49|1131.68|5144.02|5632.06|5913.32|80.24|1131.68|4012.34|4092.58|-1619.72| +2451644|62558|8225|13011|1334340|916|42645|2|102|1149|67|66.94|68.27|30.03|0.00|2012.01|4484.98|4574.09|120.72|0.00|2012.01|2132.73|-2472.97| +2451644|62558|14570|13011|1334340|916|42645|2|103|1149|75|26.11|51.43|44.22|0.00|3316.50|1958.25|3857.25|232.15|0.00|3316.50|3548.65|1358.25| +2451644|62558|2006|13011|1334340|916|42645|2|58|1149|5|81.75|104.64|24.06|0.00|120.30|408.75|523.20|6.01|0.00|120.30|126.31|-288.45| +2451644|62558|4334|13011|1334340|916|42645|2|116|1149|44|34.70|46.84|33.72|0.00|1483.68|1526.80|2060.96|14.83|0.00|1483.68|1498.51|-43.12| +2451531|63845|8216|44461|961609|1957|1552|10|263|1150|18|21.10|34.60|23.52|0.00|423.36|379.80|622.80|29.63|0.00|423.36|452.99|43.56| +2451531|63845|9871|44461|961609|1957|1552|10|79|1150|35|51.53|77.29|68.78|0.00|2407.30|1803.55|2705.15|168.51|0.00|2407.30|2575.81|603.75| +2451531|63845|3661|44461|961609|1957|1552|10|212|1150|55|38.17|45.80|33.89|0.00|1863.95|2099.35|2519.00|55.91|0.00|1863.95|1919.86|-235.40| +2451531|63845|5114|44461|961609|1957|1552|10|273|1150|66|19.98|22.17|6.42|0.00|423.72|1318.68|1463.22|25.42|0.00|423.72|449.14|-894.96| +2451531|63845|5896|44461|961609|1957|1552|10|278|1150|34|17.08|19.30|0.57|0.00|19.38|580.72|656.20|0.96|0.00|19.38|20.34|-561.34| +2451531|63845|6214|44461|961609|1957|1552|10|289|1150|15|54.24|93.29|89.55|0.00|1343.25|813.60|1399.35|94.02|0.00|1343.25|1437.27|529.65| +||13219|||1957||||1150|||||0.00|||1696.24||0.00|||| +2451531|63845|9229|44461|961609|1957|1552|10|85|1150|74|26.94|40.67|25.21|0.00|1865.54|1993.56|3009.58|18.65|0.00|1865.54|1884.19|-128.02| +2451531|63845|15614|44461|961609|1957|1552|10|284|1150|17|29.55|38.71|8.90|9.07|151.30|502.35|658.07|8.53|9.07|142.23|150.76|-360.12| +2451531|63845|8156|44461|961609|1957|1552|10|209|1150|99|85.33|95.56|83.13|0.00|8229.87|8447.67|9460.44|82.29|0.00|8229.87|8312.16|-217.80| +2451531|63845|14131|44461|961609|1957|1552|10|48|1150|98|69.47|95.17|4.75|0.00|465.50|6808.06|9326.66|13.96|0.00|465.50|479.46|-6342.56| +2451531|63845|16111|44461|961609|1957|1552|10|120|1150|30|38.53|38.53|38.53|0.00|1155.90|1155.90|1155.90|69.35|0.00|1155.90|1225.25|0.00| +2451109|69827|2449|23845|1256479|4666|22453|2|108|1151|61|54.26|62.39|24.33|0.00|1484.13|3309.86|3805.79|44.52|0.00|1484.13|1528.65|-1825.73| +2451109|69827|10240|23845|1256479|4666|22453|2|99|1151|28|88.02|169.87|93.42|0.00|2615.76|2464.56|4756.36|26.15|0.00|2615.76|2641.91|151.20| +2451109|69827|5698|23845|1256479|4666|22453|2|170|1151|75|47.76|70.20|22.46|0.00|1684.50|3582.00|5265.00|134.76|0.00|1684.50|1819.26|-1897.50| +2451109|69827|16630|23845|1256479|4666|22453|2|24|1151|16|24.23|24.47|23.24|0.00|371.84|387.68|391.52|29.74|0.00|371.84|401.58|-15.84| +2451109|69827|10195|23845|1256479|4666|22453|2|187|1151|6|82.80|161.46|72.65|200.51|435.90|496.80|968.76|11.76|200.51|235.39|247.15|-261.41| +||11422|23845||4666|22453||206|1151|86|89.44|114.48|90.43|||||||7776.98||85.14| +2451109|69827|11449|23845|1256479|4666|22453|2|86|1151|7|91.85|123.99|8.67|0.00|60.69|642.95|867.93|1.82|0.00|60.69|62.51|-582.26| +2451109|69827|8152|23845|1256479|4666|22453|2|159|1151|81|66.48|91.74|62.38|0.00|5052.78|5384.88|7430.94|252.63|0.00|5052.78|5305.41|-332.10| +2451631|73154|2441|93078|216082|5133|38984|8|108|1152|20|6.68|10.08|6.85|0.00|137.00|133.60|201.60|8.22|0.00|137.00|145.22|3.40| +2451631|73154|8779|93078|216082|5133|38984|8|175|1152|85|42.49|72.65|69.01|0.00|5865.85|3611.65|6175.25|117.31|0.00|5865.85|5983.16|2254.20| +2451631|73154|14372|93078|216082|5133|38984|8|34|1152|69|82.73|90.17|17.13|0.00|1181.97|5708.37|6221.73|106.37|0.00|1181.97|1288.34|-4526.40| +2451631|73154|5251|||5133||8||1152||||||||8981.88|33.22||||-1723.18| +2451631|73154|6697|93078|216082|5133|38984|8|221|1152|62|4.49|4.49|2.28|60.78|141.36|278.38|278.38|7.25|60.78|80.58|87.83|-197.80| +2451631|73154|12355|93078|216082|5133|38984|8|232|1152|16|71.83|119.95|101.95|0.00|1631.20|1149.28|1919.20|97.87|0.00|1631.20|1729.07|481.92| +2451631|73154|3343|93078|216082|5133|38984|8|27|1152|18|96.57|112.02|41.44|0.00|745.92|1738.26|2016.36|67.13|0.00|745.92|813.05|-992.34| +2451631|73154|16667|93078|216082|5133|38984|8|3|1152|51|25.11|36.40|2.91|0.00|148.41|1280.61|1856.40|1.48|0.00|148.41|149.89|-1132.20| +2451631|73154|8990|93078|216082|5133|38984|8|298|1152|61|45.50|75.98|27.35|0.00|1668.35|2775.50|4634.78|83.41|0.00|1668.35|1751.76|-1107.15| +2451631|73154|1555|93078|216082|5133|38984|8|188|1152|38|77.12|79.43|35.74|0.00|1358.12|2930.56|3018.34|13.58|0.00|1358.12|1371.70|-1572.44| +2452640|59424|17005|31165|105229|5246|10110|1|32|1153|15|2.26|2.93|0.17|0.15|2.55|33.90|43.95|0.04|0.15|2.40|2.44|-31.50| +2452640|59424|14814|31165|105229|5246|10110|1|174|1153|93|83.31|150.79|39.20|2260.27|3645.60|7747.83|14023.47|27.70|2260.27|1385.33|1413.03|-6362.50| +2452640|59424|11352|31165|105229|5246|10110|1|21|1153|16|66.42|75.71|40.12|0.00|641.92|1062.72|1211.36|6.41|0.00|641.92|648.33|-420.80| +2452640||414||105229|5246||||1153|97||15.54|8.70|0.00|843.90|1370.61|1507.38||0.00|843.90|886.09|| +2452640|59424|12909|31165|105229|5246|10110|1|57|1153|77|98.91|151.33|111.98|0.00|8622.46|7616.07|11652.41|776.02|0.00|8622.46|9398.48|1006.39| +2452640|59424|14961|31165|105229|5246|10110|1|238|1153|44|98.69|100.66|8.05|0.00|354.20|4342.36|4429.04|0.00|0.00|354.20|354.20|-3988.16| +2452640|59424|5775|31165|105229|5246|10110|1|262|1153|79|59.20|84.06|12.60|358.34|995.40|4676.80|6640.74|50.96|358.34|637.06|688.02|-4039.74| +2452640|59424|2995|31165|105229|5246|10110|1|114|1153|99|44.20|63.64|7.63|0.00|755.37|4375.80|6300.36|67.98|0.00|755.37|823.35|-3620.43| +2452640|59424|4633|31165|105229|5246|10110|1|224|1153|29|7.90|12.24|4.03|0.00|116.87|229.10|354.96|10.51|0.00|116.87|127.38|-112.23| +2452640|59424|12843|31165|105229|5246|10110|1|216|1153|94|99.43|133.23|35.97|0.00|3381.18|9346.42|12523.62|169.05|0.00|3381.18|3550.23|-5965.24| +2451940|69058|6971|967|1354764|561|36005|7|172|1154|49|92.91|124.49|112.04|0.00|5489.96|4552.59|6100.01|219.59|0.00|5489.96|5709.55|937.37| +2451940|69058|3129|967|1354764|561|36005|7|173|1154|19|41.86|66.13|38.35|0.00|728.65|795.34|1256.47|14.57|0.00|728.65|743.22|-66.69| +2451940|69058|16249|967|1354764|561|36005|7|192|1154|44|42.34|83.83|51.13|0.00|2249.72|1862.96|3688.52|134.98|0.00|2249.72|2384.70|386.76| +2451940|69058|15723|967|1354764|561|36005|7|241|1154|57|31.96|52.41|45.07|102.75|2568.99|1821.72|2987.37|24.66|102.75|2466.24|2490.90|644.52| +2451940|69058|5679|967|1354764|561|36005|7|95|1154|31|44.81|57.35|27.52|0.00|853.12|1389.11|1777.85|8.53|0.00|853.12|861.65|-535.99| +2451940|69058|16391|967|1354764|561|36005|7|156|1154|95|44.29|60.67|57.63|0.00|5474.85|4207.55|5763.65|437.98|0.00|5474.85|5912.83|1267.30| +2451940|69058|6073|967|1354764|561|36005|7|289|1154|42|28.28|36.19|26.05|0.00|1094.10|1187.76|1519.98|98.46|0.00|1094.10|1192.56|-93.66| +||5021|967|1354764|561||||1154|||160.23|104.14|0.00||4413.70|||0.00|4790.44|4790.44|376.74| +2451062|32655|1063|96256|1308532|2408|45484|1|211|1155|62|17.59|32.89|16.11|539.36|998.82|1090.58|2039.18|9.18|539.36|459.46|468.64|-631.12| +2451062|32655|4946|96256|1308532|2408|45484|1|270|1155|48|78.11|123.41|4.93|0.00|236.64|3749.28|5923.68|18.93|0.00|236.64|255.57|-3512.64| +2451062|32655|10522|96256|1308532|2408|45484|1|2|1155|26|54.57|60.02|34.21|0.00|889.46|1418.82|1560.52|0.00|0.00|889.46|889.46|-529.36| +2451062|32655|14320|96256|1308532|2408|45484|1|155|1155|70|51.15|90.53|57.03|0.00|3992.10|3580.50|6337.10|39.92|0.00|3992.10|4032.02|411.60| +2451062|32655|15886|96256|1308532|2408|45484|1|208|1155|51|6.97|8.01|2.08|0.00|106.08|355.47|408.51|3.18|0.00|106.08|109.26|-249.39| +|32655|15496||1308532||45484|1||1155|2|51.24|||0.00||102.48|||0.00|||| +2451062|32655|13184|96256|1308532|2408|45484|1|88|1155|45|94.07|95.01|68.40|0.00|3078.00|4233.15|4275.45|61.56|0.00|3078.00|3139.56|-1155.15| +2451062|32655|9799|96256|1308532|2408|45484|1|251|1155|91|67.22|119.65|39.48|2407.09|3592.68|6117.02|10888.15|71.13|2407.09|1185.59|1256.72|-4931.43| +2451062|32655|13232|96256|1308532|2408|45484|1|106|1155|73|71.24|92.61|18.52|0.00|1351.96|5200.52|6760.53|13.51|0.00|1351.96|1365.47|-3848.56| +2451062|32655|17869|96256|1308532|2408|45484|1|297|1155|36|31.08|33.87|21.33|215.00|767.88|1118.88|1219.32|27.64|215.00|552.88|580.52|-566.00| +2451062|32655|8293|96256|1308532|2408|45484|1|272|1155|61|31.79|54.36|25.00|0.00|1525.00|1939.19|3315.96|76.25|0.00|1525.00|1601.25|-414.19| +2451062|32655|11284|96256|1308532|2408|45484|1|101|1155|73|12.94|17.98|0.89|0.00|64.97|944.62|1312.54|4.54|0.00|64.97|69.51|-879.65| +2451062|32655|524|96256|1308532|2408|45484|1|166|1155|79|72.21|77.98|53.80|0.00|4250.20|5704.59|6160.42|340.01|0.00|4250.20|4590.21|-1454.39| +2451062|32655|3247|96256|1308532|2408|45484|1|144|1155|83|38.54|58.58|7.61|208.43|631.63|3198.82|4862.14|25.39|208.43|423.20|448.59|-2775.62| +2451062|32655|9356|96256|1308532|2408|45484|1|298|1155|94|34.29|66.52|41.24|1628.15|3876.56|3223.26|6252.88|134.90|1628.15|2248.41|2383.31|-974.85| +2450907|32758|1105|9889|833604|1672|16987|7|247|1156|22|47.69|91.56|29.29|0.00|644.38|1049.18|2014.32|51.55|0.00|644.38|695.93|-404.80| +2450907|32758|15289|9889|833604|1672|16987|7|222|1156|24|87.01|162.70|3.25|0.00|78.00|2088.24|3904.80|6.24|0.00|78.00|84.24|-2010.24| +2450907|32758|1090|9889|833604|1672|16987|7|265|1156|2|29.72|36.25|24.65|0.00|49.30|59.44|72.50|0.98|0.00|49.30|50.28|-10.14| +2450907|32758|1838|9889|833604|1672|16987|7|253|1156|9|69.63|137.17|26.06|0.00|234.54|626.67|1234.53|2.34|0.00|234.54|236.88|-392.13| +2450907|32758|7292|9889|833604|1672|16987|7|64|1156|81|64.60|64.60|24.54|0.00|1987.74|5232.60|5232.60|119.26|0.00|1987.74|2107.00|-3244.86| +2450907|32758|10549|9889|833604|1672|16987|7|198|1156|35|71.20|83.30|59.97|0.00|2098.95|2492.00|2915.50|167.91|0.00|2098.95|2266.86|-393.05| +2450907|32758|11798|9889|833604||16987|||1156|60|37.62||14.45|190.74|867.00||4130.40|20.28|190.74||696.54|| +2450907|32758|2606|9889|833604|1672|16987|7|222|1156|64|4.52|6.19|1.36|45.26|87.04|289.28|396.16|1.67|45.26|41.78|43.45|-247.50| +2450907||3448|||1672|16987|7||1156|||88.54|42.49|0.00|934.78||1947.88||0.00||1000.21|-95.92| +2450907|32758|7531|9889|833604|1672|16987|7|23|1156|43|62.30|62.30|15.57|0.00|669.51|2678.90|2678.90|33.47|0.00|669.51|702.98|-2009.39| +2450907|32758|9560|9889|833604|1672|16987|7|173|1156|23|74.51|116.98|52.64|108.96|1210.72|1713.73|2690.54|88.14|108.96|1101.76|1189.90|-611.97| +2450907|32758|15404|9889|833604|1672|16987|7|245|1156|33|5.43|10.37|1.76|0.00|58.08|179.19|342.21|2.32|0.00|58.08|60.40|-121.11| +2450907|32758|9884|9889|833604|1672|16987|7|116|1156|35|64.15|67.99|46.23|0.00|1618.05|2245.25|2379.65|16.18|0.00|1618.05|1634.23|-627.20| +2450907|32758|5104|9889|833604|1672|16987|7|219|1156|30|75.20|104.52|58.53|0.00|1755.90|2256.00|3135.60|35.11|0.00|1755.90|1791.01|-500.10| +|32758|6064|||1672||7|268|1156|83||39.55|||2396.21||||||2444.13|649.89| +2450907|32758|9172|9889|833604|1672|16987|7|232|1156|94|98.88|172.05|87.74|0.00|8247.56|9294.72|16172.70|659.80|0.00|8247.56|8907.36|-1047.16| +2451040|33137|11893|14292|995611|4930|41402|4|8|1157|11|78.14|132.83|87.66|0.00|964.26|859.54|1461.13|86.78|0.00|964.26|1051.04|104.72| +2451040|33137|11365|14292|995611|4930|41402|4|190|1157|48|51.01|101.50|12.18|0.00|584.64|2448.48|4872.00|11.69|0.00|584.64|596.33|-1863.84| +2451040|33137|7114|14292|995611|4930|41402|4|18|1157|97|36.94|54.67|1.09|0.00|105.73|3583.18|5302.99|8.45|0.00|105.73|114.18|-3477.45| +2451040|33137|12040|14292|995611|4930|41402|4|27|1157|24|85.76|110.63|24.33|0.00|583.92|2058.24|2655.12|0.00|0.00|583.92|583.92|-1474.32| +2451040|33137|5929|14292|995611|4930|41402|4|222|1157|75|74.61|127.58|30.61|0.00|2295.75|5595.75|9568.50|22.95|0.00|2295.75|2318.70|-3300.00| +2451040|33137|16670|14292|995611|4930|41402|4|94|1157|56|10.85|21.15|7.61|0.00|426.16|607.60|1184.40|17.04|0.00|426.16|443.20|-181.44| +2451040|33137|17119|14292|995611|4930|41402|4|213|1157|21|21.70|35.15|24.25|0.00|509.25|455.70|738.15|20.37|0.00|509.25|529.62|53.55| +2451040|33137|17545|14292|995611|4930|41402|4|104|1157|19|90.15|134.32|102.08|0.00|1939.52|1712.85|2552.08|77.58|0.00|1939.52|2017.10|226.67| +2451040|33137|14126|14292|995611|4930|41402|4|184|1157|29|2.43|3.18|0.63|0.00|18.27|70.47|92.22|1.09|0.00|18.27|19.36|-52.20| +2451040|33137|7628|14292|995611|4930|41402|4|78|1157|100|34.15|54.98|1.64|0.00|164.00|3415.00|5498.00|3.28|0.00|164.00|167.28|-3251.00| +2451405|67162|15326|57009|802353|497|641|2|35|1158|88|90.89|176.32|14.10|0.00|1240.80|7998.32|15516.16|24.81|0.00|1240.80|1265.61|-6757.52| +2451405|67162|7936|57009|802353|497|641|2|6|1158|26|86.23|136.24|31.33|0.00|814.58|2241.98|3542.24|24.43|0.00|814.58|839.01|-1427.40| +2451405|67162|9523|57009|802353|497|641|2|82|1158|28|84.81|128.06|98.60|0.00|2760.80|2374.68|3585.68|110.43|0.00|2760.80|2871.23|386.12| +2451405|67162|16786|57009|802353|497|641|2|192|1158|87|18.28|21.75|9.57|691.04|832.59|1590.36|1892.25|5.66|691.04|141.55|147.21|-1448.81| +2451405|67162|4850|57009|802353|497|641|2|214|1158|13|26.30|42.86|18.42|0.00|239.46|341.90|557.18|16.76|0.00|239.46|256.22|-102.44| +2451405|67162|3766|57009|802353|497|641|2|271|1158|1|78.93|87.61|26.28|0.00|26.28|78.93|87.61|0.26|0.00|26.28|26.54|-52.65| +2451405|67162|476|57009|802353|497|641|2|204|1158|60|61.01|117.74|44.74|0.00|2684.40|3660.60|7064.40|214.75|0.00|2684.40|2899.15|-976.20| +2451405|67162|15112|57009|802353|497|641|2|279|1158|4|11.14|14.14|10.18|9.77|40.72|44.56|56.56|2.16|9.77|30.95|33.11|-13.61| +2452503|68571|12963|85460|87718|1176|17034|10|165|1159|72|69.18|75.40|74.64|0.00|5374.08|4980.96|5428.80|483.66|0.00|5374.08|5857.74|393.12| +2452503|68571|739|85460|87718|1176|17034|10|164|1159|12|84.96|117.24|64.48|0.00|773.76|1019.52|1406.88|0.00|0.00|773.76|773.76|-245.76| +2452503|68571|6615|85460|87718|1176|17034|10|290|1159|20|43.31|83.15|41.57|773.20|831.40|866.20|1663.00|1.74|773.20|58.20|59.94|-808.00| +2452503|68571|4857|85460|87718|1176|17034|10|284|1159|69|46.67|67.67|24.36|0.00|1680.84|3220.23|4669.23|117.65|0.00|1680.84|1798.49|-1539.39| +2452503|68571|4113|85460|87718|1176|17034|10|58|1159|67|57.06|85.01|47.60|0.00|3189.20|3823.02|5695.67|0.00|0.00|3189.20|3189.20|-633.82| +2452503|68571|5877|85460|87718|1176|17034|10|295|1159|64|5.74|10.50|5.25|0.00|336.00|367.36|672.00|6.72|0.00|336.00|342.72|-31.36| +2452503||8331||87718|||10|99|1159||||82.21|0.00||1333.36||0.00|0.00||2301.88|| +2452503|68571|12030|85460|87718|1176|17034|10|181|1159|19|49.96|94.92|37.96|0.00|721.24|949.24|1803.48|43.27|0.00|721.24|764.51|-228.00| +2452503|68571|16566|85460|87718|1176|17034|10|164|1159|44|6.72|9.40|5.35|0.00|235.40|295.68|413.60|4.70|0.00|235.40|240.10|-60.28| +2452503|68571|9966|85460|87718|1176|17034|10|177|1159|90|52.56|86.19|8.61|0.00|774.90|4730.40|7757.10|54.24|0.00|774.90|829.14|-3955.50| +2452503|68571|6750|85460|87718|1176|17034|10|144|1159|26|67.91|118.16|64.98|0.00|1689.48|1765.66|3072.16|152.05|0.00|1689.48|1841.53|-76.18| +2452503|68571|15385|85460|87718|1176|17034|10|157|1159|39|26.40|45.93|13.31|0.00|519.09|1029.60|1791.27|0.00|0.00|519.09|519.09|-510.51| +2452503|68571|235|85460|87718|1176|17034|10|195|1159|27|41.41|59.63|56.64|0.00|1529.28|1118.07|1610.01|45.87|0.00|1529.28|1575.15|411.21| +2451963|40576|2649|61779|876697|4768|38886|7|72|1160|25|4.27|7.00|2.66|0.00|66.50|106.75|175.00|0.00|0.00|66.50|66.50|-40.25| +2451963|40576|17101|61779|876697|4768|38886|7|169|1160|74|2.97|4.42|1.50|0.00|111.00|219.78|327.08|1.11|0.00|111.00|112.11|-108.78| +2451963|40576|3057|61779|876697|4768|38886|7|20|1160|92|23.76|45.85|23.84|0.00|2193.28|2185.92|4218.20|131.59|0.00|2193.28|2324.87|7.36| +2451963|40576|9301|61779|876697|4768|38886|7|175|1160|71|52.31|66.95|61.59|0.00|4372.89|3714.01|4753.45|262.37|0.00|4372.89|4635.26|658.88| +2451963|40576|3477|61779|876697|4768|38886|7|270|1160|6|44.84|46.63|31.24|0.00|187.44|269.04|279.78|9.37|0.00|187.44|196.81|-81.60| +2451963|40576|7697|61779|876697|4768|38886|7|131|1160|48|90.67|93.39|28.95|0.00|1389.60|4352.16|4482.72|83.37|0.00|1389.60|1472.97|-2962.56| +2451963|40576|4191|61779|876697|4768|38886|7|35|1160|82|31.54|52.98|25.96|0.00|2128.72|2586.28|4344.36|85.14|0.00|2128.72|2213.86|-457.56| +2451963|40576|387|61779|876697|4768|38886|7|183|1160|99|40.54|55.94|0.55|22.32|54.45|4013.46|5538.06|2.89|22.32|32.13|35.02|-3981.33| +|40576|8411||876697||38886|7|287|1160||3.69|7.30|2.77|||214.02||1.60||160.66||| +||7097||876697||38886||265|1160||||||284.76|285.60||||284.76|310.38|-0.84| +2451963|40576|1599|61779|876697|4768|38886|7|90|1160|33|20.30|34.30|24.35|0.00|803.55|669.90|1131.90|56.24|0.00|803.55|859.79|133.65| +2452352|66563|14263|8735|207962|342|43169|2|65|1161|62|50.25|73.36|55.75|0.00|3456.50|3115.50|4548.32|69.13|0.00|3456.50|3525.63|341.00| +2452352|66563|744|8735|207962|342|43169|2|203|1161|66|91.88|147.92|20.70|0.00|1366.20|6064.08|9762.72|109.29|0.00|1366.20|1475.49|-4697.88| +2452352|66563|12361|8735|207962|342|43169|2|19|1161|91|31.68|33.89|9.15|0.00|832.65|2882.88|3083.99|41.63|0.00|832.65|874.28|-2050.23| +2452352|66563|7368|8735|207962|342|43169|2|254|1161|35|30.13|44.29|34.54|689.07|1208.90|1054.55|1550.15|10.39|689.07|519.83|530.22|-534.72| +2452352|66563|15775|8735|207962|342|43169|2|112|1161|27|23.90|30.83|21.58|0.00|582.66|645.30|832.41|40.78|0.00|582.66|623.44|-62.64| +2452352|66563|4381|8735|207962|342|43169|2|156|1161|71|2.67|4.83|4.83|0.00|342.93|189.57|342.93|30.86|0.00|342.93|373.79|153.36| +2452352|66563|2739|8735|207962|342|43169|2|127|1161|92|7.58|10.46|8.57|646.52|788.44|697.36|962.32|4.25|646.52|141.92|146.17|-555.44| +2452352|66563|4351|8735|207962|342|43169|2|20|1161|57|42.01|78.55|40.84|0.00|2327.88|2394.57|4477.35|46.55|0.00|2327.88|2374.43|-66.69| +||9447|8735|207962|||2||1161|16||174.97||0.00|2771.52|1590.72|2799.52||0.00|2771.52|2993.24|1180.80| +2452352|66563|10123|8735|207962|342|43169|2|223|1161|93|78.59|90.37|37.05|0.00|3445.65|7308.87|8404.41|206.73|0.00|3445.65|3652.38|-3863.22| +2451635|39683|16979|86004|103992|5980|49568|7|183|1162|76|70.45|126.81|102.71|0.00|7805.96|5354.20|9637.56|390.29|0.00|7805.96|8196.25|2451.76| +2451635|39683|7103|86004|103992|5980|49568|7|294|1162|82|87.81|122.05|74.45|0.00|6104.90|7200.42|10008.10|549.44|0.00|6104.90|6654.34|-1095.52| +2451635|39683|12077|86004|103992|5980|49568|7|213|1162|39|73.36|94.63|64.34|0.00|2509.26|2861.04|3690.57|125.46|0.00|2509.26|2634.72|-351.78| +2451635|39683|914|86004|103992|5980|49568|7|90|1162|55|88.21|112.90|84.67|0.00|4656.85|4851.55|6209.50|93.13|0.00|4656.85|4749.98|-194.70| +2451635|39683|1891|86004|103992|5980|49568|7|184|1162|73|13.63|16.62|14.29|52.15|1043.17|994.99|1213.26|19.82|52.15|991.02|1010.84|-3.97| +2451635|39683|1471|86004|103992|5980|49568|7|151|1162|56|22.17|42.12|1.68|0.00|94.08|1241.52|2358.72|1.88|0.00|94.08|95.96|-1147.44| +2451635|39683|3475|86004|103992|5980|49568|7|32|1162|13|64.01|116.49|95.52|0.00|1241.76|832.13|1514.37|0.00|0.00|1241.76|1241.76|409.63| +2451635|39683|661|86004|103992|5980|49568|7|5|1162|57|43.72|85.25|62.23|0.00|3547.11|2492.04|4859.25|248.29|0.00|3547.11|3795.40|1055.07| +2451156|60566|14978|40660|1160321|3982|26043|1|215|1163|72|40.25|68.02|10.88|0.00|783.36|2898.00|4897.44|70.50|0.00|783.36|853.86|-2114.64| +2451156|60566|127|40660|1160321|3982|26043|1|16|1163|54|42.30|76.56|68.90|1302.21|3720.60|2284.20|4134.24|96.73|1302.21|2418.39|2515.12|134.19| +2451156|60566|17725|40660|1160321|3982|26043|1|127|1163|31|9.15|11.71|7.96|0.00|246.76|283.65|363.01|22.20|0.00|246.76|268.96|-36.89| +2451156|60566|4256|40660|1160321|3982|26043|1|174|1163|98|50.73|90.29|5.41|0.00|530.18|4971.54|8848.42|37.11|0.00|530.18|567.29|-4441.36| +||8785|40660|1160321|||1|48|1163||||16.45|||||7.10||||-1573.56| +2451156|60566|1207|40660|1160321|3982|26043|1|75|1163|38|25.55|44.20|13.70|447.71|520.60|970.90|1679.60|2.91|447.71|72.89|75.80|-898.01| +2451156|60566|9698|40660|1160321|3982|26043|1|243|1163|20|43.35|47.68|39.09|367.44|781.80|867.00|953.60|4.14|367.44|414.36|418.50|-452.64| +2451156|60566|7054|40660|1160321|3982|26043|1|14|1163|30|98.79|154.11|33.90|0.00|1017.00|2963.70|4623.30|40.68|0.00|1017.00|1057.68|-1946.70| +2451156|60566|9146|40660|1160321|3982|26043|1|162|1163|38|7.59|14.49|12.02|0.00|456.76|288.42|550.62|13.70|0.00|456.76|470.46|168.34| +2451156|60566|11278|40660|1160321|3982|26043|1|72|1163|84|18.02|26.66|25.32|595.52|2126.88|1513.68|2239.44|61.25|595.52|1531.36|1592.61|17.68| +2451156|60566|7246|40660|1160321|3982|26043|1|56|1163|46|6.76|11.15|1.22|0.00|56.12|310.96|512.90|2.80|0.00|56.12|58.92|-254.84| +2451156||11521|||||1||1163||||15.48|0.00|1362.24|1638.56||13.62|0.00||1375.86|-276.32| +2451156|60566|11971|40660|1160321|3982|26043|1|128|1163|66|71.98|87.09|11.32|0.00|747.12|4750.68|5747.94|67.24|0.00|747.12|814.36|-4003.56| +2451156|60566|17227|40660|1160321|3982|26043|1|193|1163|66|43.61|83.29|64.96|0.00|4287.36|2878.26|5497.14|85.74|0.00|4287.36|4373.10|1409.10| +2451156|60566|13880|40660|1160321|3982|26043|1|98|1163|43|70.89|137.52|57.75|0.00|2483.25|3048.27|5913.36|124.16|0.00|2483.25|2607.41|-565.02| +2452261|57661|12125|67814|803636|7105|14127|8|1|1164|27|74.13|136.39|2.72|0.00|73.44|2001.51|3682.53|2.93|0.00|73.44|76.37|-1928.07| +2452261|57661|1067|67814|803636|7105|14127|8|214|1164|76|46.98|64.83|35.00|0.00|2660.00|3570.48|4927.08|106.40|0.00|2660.00|2766.40|-910.48| +2452261|57661|10817|67814|803636|7105|14127|8|90|1164|49|29.98|53.36|50.15|0.00|2457.35|1469.02|2614.64|122.86|0.00|2457.35|2580.21|988.33| +2452261|57661|2175|67814|803636|7105|14127|8|152|1164|61|18.93|33.50|32.83|0.00|2002.63|1154.73|2043.50|20.02|0.00|2002.63|2022.65|847.90| +2452261|57661|9107|67814|803636|7105|14127|8|189|1164|75|80.62|130.60|19.59|0.00|1469.25|6046.50|9795.00|44.07|0.00|1469.25|1513.32|-4577.25| +2452261|57661|4665|67814|803636|7105|14127|8|106|1164|1|96.95|183.23|128.26|0.00|128.26|96.95|183.23|6.41|0.00|128.26|134.67|31.31| +2452261|57661|1639|67814|803636|7105|14127|8|291|1164|100|49.57|75.34|73.83|0.00|7383.00|4957.00|7534.00|590.64|0.00|7383.00|7973.64|2426.00| +2452261|57661|3493|67814|803636|7105|14127|8|197|1164|40|84.25|149.96|10.49|0.00|419.60|3370.00|5998.40|16.78|0.00|419.60|436.38|-2950.40| +2452261|57661|9119|67814|803636|7105|14127|8|28|1164|3|78.37|143.41|86.04|0.00|258.12|235.11|430.23|0.00|0.00|258.12|258.12|23.01| +2452261|57661|12857|67814|803636|7105|14127|8|28|1164|29|84.64|127.80|90.73|1052.46|2631.17|2454.56|3706.20|126.29|1052.46|1578.71|1705.00|-875.85| +2452261|57661|14529|67814|803636|7105|14127|8|192|1164|19|12.49|12.61|9.33|0.00|177.27|237.31|239.59|7.09|0.00|177.27|184.36|-60.04| +2451176|47091|9034|36875|22005|4735|15065|8|30|1165|19|96.26|162.67|157.78|59.95|2997.82|1828.94|3090.73|29.37|59.95|2937.87|2967.24|1108.93| +2451176|47091|15188|36875|22005|4735|15065|8|181|1165|17|88.69|141.01|136.77|0.00|2325.09|1507.73|2397.17|69.75|0.00|2325.09|2394.84|817.36| +2451176|47091|4783|36875|22005|4735|15065|8|108|1165|96|97.66|146.49|143.56|0.00|13781.76|9375.36|14063.04|964.72|0.00|13781.76|14746.48|4406.40| +2451176||1274|||||8|257|1165||37.14|59.05|8.85|0.00|566.40|2376.96||33.98|0.00|566.40|600.38|-1810.56| +2451176|47091|13964|36875|22005|4735|15065|8|88|1165|65|36.34|65.04|45.52|0.00|2958.80|2362.10|4227.60|236.70|0.00|2958.80|3195.50|596.70| +2451176|47091|11815|36875|22005|4735|15065|8|139|1165|92|33.45|57.19|25.16|0.00|2314.72|3077.40|5261.48|46.29|0.00|2314.72|2361.01|-762.68| +2451176|47091|3838|36875|22005|4735|15065|8|131|1165|92|3.71|3.85|3.00|0.00|276.00|341.32|354.20|11.04|0.00|276.00|287.04|-65.32| +2451176|47091|9247|36875||4735||8|2|1165||23.55|37.68|||259.95|353.25|565.20|0.00||259.95||| +||12062|||4735||8||1165||||92.87||8636.91|6164.04|10663.38|0.00||8636.91||2472.87| +2451176|47091|10256|36875|22005|4735|15065|8|123|1165|40|33.81|38.54|36.22|0.00|1448.80|1352.40|1541.60|72.44|0.00|1448.80|1521.24|96.40| +||16667||||14107|4|64|1166|13|18.70|31.60||0.00|217.62|243.10|||0.00||230.67|-25.48| +2452055|30077|8991|22614|1597794|974|14107|4|104|1166|74|20.24|40.27|8.85|0.00|654.90|1497.76|2979.98|26.19|0.00|654.90|681.09|-842.86| +2452055|30077|1555|22614|1597794|974|14107|4|209|1166|90|74.73|120.31|111.88|0.00|10069.20|6725.70|10827.90|302.07|0.00|10069.20|10371.27|3343.50| +2452055|30077|1505|22614|1597794|974|14107|4|271|1166|12|4.05|4.29|0.98|0.00|11.76|48.60|51.48|1.05|0.00|11.76|12.81|-36.84| +2452055|30077|10073|22614|1597794|974|14107|4|229|1166|53|71.01|123.55|81.54|0.00|4321.62|3763.53|6548.15|216.08|0.00|4321.62|4537.70|558.09| +2452055|30077|14237|22614|1597794|974|14107|4|96|1166|59|63.84|114.91|96.52|0.00|5694.68|3766.56|6779.69|56.94|0.00|5694.68|5751.62|1928.12| +2452055|30077|2293||1597794|974|14107|||1166||3.36|3.76|0.82|||||||||| +2452055|30077|4287|22614|1597794|974|14107|4|239|1166|49|87.54|164.57|49.37|1669.19|2419.13|4289.46|8063.93|0.00|1669.19|749.94|749.94|-3539.52| +2452055|30077|12753|22614|1597794|974|14107|4|65|1166|84|53.28|61.80|58.09|0.00|4879.56|4475.52|5191.20|146.38|0.00|4879.56|5025.94|404.04| +2452002|69057|15217|30313|1007556|3646|11689|4|218|1167|37|3.01|5.74|5.33|0.00|197.21|111.37|212.38|13.80|0.00|197.21|211.01|85.84| +2452002|69057|5487|30313|1007556|3646|11689|4|150|1167|32|28.11|52.00|8.32|98.50|266.24|899.52|1664.00|5.03|98.50|167.74|172.77|-731.78| +2452002|69057|3609|30313|1007556|3646|11689|4|81|1167|78|65.63|128.63|2.57|0.00|200.46|5119.14|10033.14|0.00|0.00|200.46|200.46|-4918.68| +2452002|69057|10365|30313|1007556|3646|11689|4|142|1167|64|80.76|138.09|122.90|6213.82|7865.60|5168.64|8837.76|115.62|6213.82|1651.78|1767.40|-3516.86| +2452002||585|||3646|11689|4||1167|||40.57|4.86||||3732.44|||447.12||-2612.80| +2452002|69057|17363|30313|1007556|3646|11689|4|65|1167|25|67.59|68.94|24.12|0.00|603.00|1689.75|1723.50|18.09|0.00|603.00|621.09|-1086.75| +2452002|69057|8147|30313|1007556|3646|11689|4|176|1167|70|31.10|58.46|48.52|0.00|3396.40|2177.00|4092.20|305.67|0.00|3396.40|3702.07|1219.40| +2452002|69057|8273|30313|1007556|3646|11689|4|10|1167|69|90.36|121.98|39.03|0.00|2693.07|6234.84|8416.62|80.79|0.00|2693.07|2773.86|-3541.77| +2452002|69057|12991|30313|1007556|3646|11689|4|278|1167|85|72.83|75.74|67.40|0.00|5729.00|6190.55|6437.90|286.45|0.00|5729.00|6015.45|-461.55| +2452002||4729|30313|1007556|3646|||48|1167|19||24.23||0.00|||460.37|3.86|0.00||68.27|-259.92| +2452002|69057|451|30313|1007556|3646|11689|4|38|1167|67|40.26|80.52|9.66|388.33|647.22|2697.42|5394.84|10.35|388.33|258.89|269.24|-2438.53| +2452002|69057|10049|30313|1007556|3646|11689|4|156|1167|42|38.33|69.76|6.27|165.90|263.34|1609.86|2929.92|3.89|165.90|97.44|101.33|-1512.42| +2452161|74478|3651|26459|1513053|800|8194|7|184|1168|78|28.68|32.12|11.56|243.45|901.68|2237.04|2505.36|52.65|243.45|658.23|710.88|-1578.81| +2452161|74478|10571|26459|1513053|800|8194|7|6|1168|94|78.51|132.68|115.43|0.00|10850.42|7379.94|12471.92|651.02|0.00|10850.42|11501.44|3470.48| +2452161|74478|5443|26459|1513053|800|8194|7|33|1168|61|66.91|88.32|80.37|0.00|4902.57|4081.51|5387.52|294.15|0.00|4902.57|5196.72|821.06| +2452161|74478|6011|26459|1513053|800|8194|7|268|1168|44|82.18|86.28|56.94|75.16|2505.36|3615.92|3796.32|48.60|75.16|2430.20|2478.80|-1185.72| +2452161|74478|7467|26459|1513053|800|8194|7|185|1168|28|9.23|11.62|6.85|38.36|191.80|258.44|325.36|9.20|38.36|153.44|162.64|-105.00| +2452161|74478|17069|26459|1513053|800|8194|7|78|1168|55|40.90|80.16|12.82|0.00|705.10|2249.50|4408.80|14.10|0.00|705.10|719.20|-1544.40| +||16097|26459|1513053||8194|7|198|1168||80.08|||0.00|||2769.00||0.00|719.94||| +2452161|74478|9933|26459|1513053|800|8194|7|267|1168|47|7.40|10.58|6.24|0.00|293.28|347.80|497.26|23.46|0.00|293.28|316.74|-54.52| +2452161|74478|11133|26459|1513053|800|8194|7|82|1168|48|66.34|78.94|41.04|0.00|1969.92|3184.32|3789.12|0.00|0.00|1969.92|1969.92|-1214.40| +2452161|74478|441|26459|1513053|800|8194|7|135|1168|17|41.41|48.86|34.69|0.00|589.73|703.97|830.62|11.79|0.00|589.73|601.52|-114.24| +2452161|74478|10873|26459|1513053|800|8194|7|181|1168|83|8.72|9.59|9.30|0.00|771.90|723.76|795.97|0.00|0.00|771.90|771.90|48.14| +2452161|74478|833|26459|1513053|800|8194|7|108|1168|43|79.69|144.23|33.17|0.00|1426.31|3426.67|6201.89|128.36|0.00|1426.31|1554.67|-2000.36| +2452161|74478|1241|26459|1513053|800|8194|7|117|1168|1|48.27|63.23|20.23|0.00|20.23|48.27|63.23|1.21|0.00|20.23|21.44|-28.04| +2451542|66627|4975|66775|639511|4694|14677|10|285|1169|89|58.95|61.89|11.14|0.00|991.46|5246.55|5508.21|89.23|0.00|991.46|1080.69|-4255.09| +|66627|4333|||4694||10||1169||52.83||33.19|0.00|||4407.21|241.95|0.00|2688.39|2930.34|-1590.84| +2451542|66627|17866|66775|639511|4694|14677|10|43|1169|19|43.90|79.89|56.72|0.00|1077.68|834.10|1517.91|64.66|0.00|1077.68|1142.34|243.58| +2451542|66627|2461|66775|639511|4694|14677|10|64|1169|68|94.45|103.89|25.97|1200.85|1765.96|6422.60|7064.52|39.55|1200.85|565.11|604.66|-5857.49| +2451542|66627|4867|66775|639511|4694|14677|10|237|1169|79|87.94|92.33|77.55|0.00|6126.45|6947.26|7294.07|183.79|0.00|6126.45|6310.24|-820.81| +2451542|66627|8030|66775|639511|4694|14677|10|276|1169|28|92.30|115.37|56.53|0.00|1582.84|2584.40|3230.36|79.14|0.00|1582.84|1661.98|-1001.56| +2451542|66627|7867|66775|639511|4694|14677|10|211|1169|70|45.26|77.39|51.07|0.00|3574.90|3168.20|5417.30|107.24|0.00|3574.90|3682.14|406.70| +2451542|66627|469|66775|639511|4694|14677|10|50|1169|3|39.65|56.30|47.29|0.00|141.87|118.95|168.90|7.09|0.00|141.87|148.96|22.92| +2451542|66627|8359|66775|639511|4694|14677|10|171|1169|47|12.41|23.08|12.46|345.51|585.62|583.27|1084.76|2.40|345.51|240.11|242.51|-343.16| +2451542|66627|848|66775|639511|4694|14677|10|168|1169|82|46.09|69.13|17.97|1281.97|1473.54|3779.38|5668.66|9.57|1281.97|191.57|201.14|-3587.81| +2451542|66627|16555|66775|639511|4694|14677|10|228|1169|9|24.07|46.21|38.81|0.00|349.29|216.63|415.89|10.47|0.00|349.29|359.76|132.66| +2451542|66627|9692|66775|639511|4694|14677|10|288|1169|70|30.82|61.33|40.47|0.00|2832.90|2157.40|4293.10|28.32|0.00|2832.90|2861.22|675.50| +2451542|66627|10315|66775|639511|4694|14677|10|2|1169|87|23.11|25.65|3.07|128.20|267.09|2010.57|2231.55|8.33|128.20|138.89|147.22|-1871.68| +2451727|39719|10411|52345|783631|4964|650|2|62|1170|78|32.82|42.33|35.13|0.00|2740.14|2559.96|3301.74|164.40|0.00|2740.14|2904.54|180.18| +2451727||2342|||4964|650|||1170|6||108.32|94.23|0.00|565.38|||28.26|0.00|||| +2451727|39719|11000|52345|783631|4964|650|2|288|1170|96|63.36|80.46|54.71|0.00|5252.16|6082.56|7724.16|52.52|0.00|5252.16|5304.68|-830.40| +2451727|39719|9362|52345|783631|4964|650|2|101|1170|51|1.76|3.06|1.83|0.00|93.33|89.76|156.06|5.59|0.00|93.33|98.92|3.57| +2451727|39719|61|52345|783631|4964|650|2|208|1170|31|66.19|72.14|16.59|0.00|514.29|2051.89|2236.34|15.42|0.00|514.29|529.71|-1537.60| +2451727|39719|7976|52345|783631|4964|650|2|122|1170|72|13.05|22.18|3.77|0.00|271.44|939.60|1596.96|8.14|0.00|271.44|279.58|-668.16| +2451727|39719|5837|52345|783631|4964|650|2|168|1170|8|3.71|6.82|3.27|25.89|26.16|29.68|54.56|0.00|25.89|0.27|0.27|-29.41| +2451727|39719|4879|52345|783631|4964|650|2|5|1170|31|92.21|138.31|106.49|2046.73|3301.19|2858.51|4287.61|62.72|2046.73|1254.46|1317.18|-1604.05| +2451727|39719|15643|52345|783631|4964|650|2|185|1170|62|27.01|41.86|4.18|0.00|259.16|1674.62|2595.32|2.59|0.00|259.16|261.75|-1415.46| +2451727|39719|12881|52345|783631|4964|650|2|186|1170|51|4.59|5.37|1.39|0.00|70.89|234.09|273.87|4.25|0.00|70.89|75.14|-163.20| +2451727|39719|14101|52345|783631|4964|650|2|101|1170|93|70.34|137.86|75.82|0.00|7051.26|6541.62|12820.98|493.58|0.00|7051.26|7544.84|509.64| +2451727|39719|11855|52345|783631|4964|650|2|282|1170|22|62.19|108.21|50.85|0.00|1118.70|1368.18|2380.62|67.12|0.00|1118.70|1185.82|-249.48| +2451504|47670|6242|37977|121472|167|18462|7|200|1171|30|55.78|84.22|73.27|0.00|2198.10|1673.40|2526.60|43.96|0.00|2198.10|2242.06|524.70| +2451504|47670|13466|37977|121472|167|18462|7|24|1171|85|32.90|51.32|50.80|0.00|4318.00|2796.50|4362.20|129.54|0.00|4318.00|4447.54|1521.50| +2451504|47670|9178|37977|121472|167|18462|7|139|1171|95|88.55|154.07|52.38|0.00|4976.10|8412.25|14636.65|99.52|0.00|4976.10|5075.62|-3436.15| +2451504|47670|8149|37977|121472|167|18462|7|37|1171|11|50.75|58.87|20.01|0.00|220.11|558.25|647.57|8.80|0.00|220.11|228.91|-338.14| +2451504|47670|3259|37977|121472|167|18462|7|229|1171|8|95.77|129.28|31.02|0.00|248.16|766.16|1034.24|12.40|0.00|248.16|260.56|-518.00| +2451504|47670|4412|37977|121472|167|18462|7|115|1171|4|34.27|46.26|4.62|0.00|18.48|137.08|185.04|0.18|0.00|18.48|18.66|-118.60| +2451504|47670|562|37977|121472|167|18462|7|6|1171|65|41.15|44.03|34.78|0.00|2260.70|2674.75|2861.95|135.64|0.00|2260.70|2396.34|-414.05| +2451504|47670|5476|37977|121472|167|18462|7|270|1171|78|64.26|91.89|51.45|601.96|4013.10|5012.28|7167.42|307.00|601.96|3411.14|3718.14|-1601.14| +2451504|47670|17491|37977|121472|167|18462|7|41|1171|81|91.81|172.60|0.00|0.00|0.00|7436.61|13980.60|0.00|0.00|0.00|0.00|-7436.61| +2452244|50755|14793|15494|487230|1719|655|1|4|1172|25|17.37|25.01|6.00|16.50|150.00|434.25|625.25|9.34|16.50|133.50|142.84|-300.75| +2452244|50755|8173|15494|487230|1719|655|1|258|1172|9|49.21|50.19|17.56|0.00|158.04|442.89|451.71|7.90|0.00|158.04|165.94|-284.85| +|50755|3987|15494|487230|1719|655||100|1172|||77.08||0.00|3022.59||4085.24||0.00|3022.59||| +2452244|50755|12763|15494|487230|1719|655|1|105|1172|61|30.16|45.84|28.87|0.00|1761.07|1839.76|2796.24|17.61|0.00|1761.07|1778.68|-78.69| +2452244|50755|13743|15494|487230|1719|655|1|76|1172|88|77.53|80.63|20.15|0.00|1773.20|6822.64|7095.44|53.19|0.00|1773.20|1826.39|-5049.44| +||2721|15494|487230||655||117|1172|||8.53||||55.04|68.24|0.52|||6.36|-49.20| +2452244|50755|2025|15494|487230|1719|655|1|166|1172|22|55.46|79.86|21.56|0.00|474.32|1220.12|1756.92|14.22|0.00|474.32|488.54|-745.80| +2452244|50755|16015|15494|487230|1719|655|1|186|1172|20|38.93|59.95|21.58|0.00|431.60|778.60|1199.00|30.21|0.00|431.60|461.81|-347.00| +2452244|50755|17521|15494|487230|1719|655|1|89|1172|40|12.31|23.14|9.48|0.00|379.20|492.40|925.60|26.54|0.00|379.20|405.74|-113.20| +2452244|50755|9079|15494|487230|1719|655|1|59|1172|84|63.28|112.00|67.20|0.00|5644.80|5315.52|9408.00|282.24|0.00|5644.80|5927.04|329.28| +2452244|50755|6089|15494|487230|1719|655|1|286|1172|91|48.13|54.86|36.75|0.00|3344.25|4379.83|4992.26|300.98|0.00|3344.25|3645.23|-1035.58| +2452244|50755|16841|15494|487230|1719|655|1|33|1172|44|10.53|17.16|6.69|0.00|294.36|463.32|755.04|11.77|0.00|294.36|306.13|-168.96| +2452244|50755|12689|15494|487230|1719|655|1|21|1172|60|43.57|51.41|4.62|0.00|277.20|2614.20|3084.60|13.86|0.00|277.20|291.06|-2337.00| +2452244||4825|||1719||||1172|17||13.29|6.91|||113.56||4.69||117.47|122.16|| +2452540|37654|207|558|1048939|4520|21348|1|123|1173|25|75.57|133.00|1.33|0.00|33.25|1889.25|3325.00|0.99|0.00|33.25|34.24|-1856.00| +2452540|37654|5178|558|1048939|4520|21348|1|207|1173|89|73.78|110.67|108.45|0.00|9652.05|6566.42|9849.63|772.16|0.00|9652.05|10424.21|3085.63| +2452540|37654|9591|558|1048939|4520|21348|1|62|1173|7|72.07|80.71|29.86|0.00|209.02|504.49|564.97|18.81|0.00|209.02|227.83|-295.47| +2452540|37654|17359|558|1048939|4520|21348|1|127|1173|67|63.31|83.56|23.39|0.00|1567.13|4241.77|5598.52|47.01|0.00|1567.13|1614.14|-2674.64| +2452540|37654|17280|558|1048939|4520|21348|1|160|1173|34|2.06|2.76|0.93|0.00|31.62|70.04|93.84|0.31|0.00|31.62|31.93|-38.42| +2452540|37654|15547|558|1048939|4520|21348|1|212|1173|40|51.60|93.39|25.21|857.14|1008.40|2064.00|3735.60|12.10|857.14|151.26|163.36|-1912.74| +2452540|37654|4519|558|1048939|4520|21348|1|138|1173|77|33.21|35.86|18.64|0.00|1435.28|2557.17|2761.22|114.82|0.00|1435.28|1550.10|-1121.89| +2452540|37654|7465|558|1048939|4520|21348|1|226|1173|59|44.61|53.97|53.97|1560.27|3184.23|2631.99|3184.23|16.23|1560.27|1623.96|1640.19|-1008.03| +2452540|37654|5779|558|1048939|4520|21348|1|256|1173|98|21.46|24.24|14.54|0.00|1424.92|2103.08|2375.52|128.24|0.00|1424.92|1553.16|-678.16| +2452540|37654|3288|558|1048939|4520|21348|1|199|1173|65|87.25|101.21|42.50|0.00|2762.50|5671.25|6578.65|110.50|0.00|2762.50|2873.00|-2908.75| +||13009|558|||21348|||1173|73||117.77|73.01|0.00|||8597.21|373.08|0.00||5702.81|631.45| +2452540|37654|16425|558|1048939|4520|21348|1|10|1173|46|34.08|61.68|17.27|0.00|794.42|1567.68|2837.28|0.00|0.00|794.42|794.42|-773.26| +2452449|50823|1722|26873|626398|7170|14695|7|218|1174|74|33.38|40.72|16.69|0.00|1235.06|2470.12|3013.28|49.40|0.00|1235.06|1284.46|-1235.06| +2452449|50823|10467|26873|626398|7170|14695|7|126|1174|5|21.84|42.80|21.82|0.00|109.10|109.20|214.00|0.00|0.00|109.10|109.10|-0.10| +|50823|7671||626398|7170|14695|||1174|||||0.00||4389.12||192.11|0.00|4802.88|4994.99|413.76| +2452449|50823|11640|26873|626398|7170|14695|7|15|1174|28|56.66|84.42|43.89|1106.02|1228.92|1586.48|2363.76|0.00|1106.02|122.90|122.90|-1463.58| +2452449|50823|8754|26873|626398|7170|14695|7|227|1174|39|8.82|14.28|10.28|0.00|400.92|343.98|556.92|24.05|0.00|400.92|424.97|56.94| +2452449|50823|12499|26873|626398|7170|14695|7|291|1174|43|73.58|102.27|83.86|0.00|3605.98|3163.94|4397.61|252.41|0.00|3605.98|3858.39|442.04| +2452449|50823|14907|26873|626398|7170|14695|7|2|1174|50|66.19|79.42|29.38|0.00|1469.00|3309.50|3971.00|14.69|0.00|1469.00|1483.69|-1840.50| +2452449|50823|4713|26873|626398|7170|14695|7|167|1174|9|32.60|39.12|10.95|0.00|98.55|293.40|352.08|3.94|0.00|98.55|102.49|-194.85| +2452449|50823|10045|26873|626398|7170|14695|7|193|1174|87|2.01|2.61|0.05|0.00|4.35|174.87|227.07|0.00|0.00|4.35|4.35|-170.52| +2452449|50823|13653|26873|626398|7170|14695|7|201|1174|87|46.81|52.89|33.84|0.00|2944.08|4072.47|4601.43|117.76|0.00|2944.08|3061.84|-1128.39| +2452555|49733|301|40314|1703180|446|18559|1|285|1175|3|10.73|17.27|9.15|1.37|27.45|32.19|51.81|1.56|1.37|26.08|27.64|-6.11| +2452555|49733|9361|40314|1703180|446|18559|1|299|1175|74|96.12|150.90|108.64|0.00|8039.36|7112.88|11166.60|723.54|0.00|8039.36|8762.90|926.48| +2452555|49733|12093|40314|1703180|446|18559|1|34|1175|62|23.79|32.35|22.96|1124.58|1423.52|1474.98|2005.70|23.91|1124.58|298.94|322.85|-1176.04| +2452555|49733|17083|40314|1703180|446|18559|1|225|1175|98|44.51|70.32|66.80|0.00|6546.40|4361.98|6891.36|458.24|0.00|6546.40|7004.64|2184.42| +2452555||13932|||446|18559||127|1175|60|27.70|39.61||0.00||1662.00|2376.60||0.00||2065.72|286.80| +2452555|49733|4218|40314|1703180|446|18559|1|265|1175|2|78.11|108.57|49.94|0.00|99.88|156.22|217.14|6.99|0.00|99.88|106.87|-56.34| +2452555|49733|15846|40314|1703180|446|18559|1|273|1175|42|13.78|14.05|4.07|0.00|170.94|578.76|590.10|11.96|0.00|170.94|182.90|-407.82| +2452555|49733|17634|40314|1703180|446|18559|1|164|1175|75|95.33|101.04|56.58|0.00|4243.50|7149.75|7578.00|127.30|0.00|4243.50|4370.80|-2906.25| +2452555|49733|12169|40314|1703180|446|18559|1|115|1175|99|50.89|68.19|6.81|0.00|674.19|5038.11|6750.81|33.70|0.00|674.19|707.89|-4363.92| +2452555|49733|16209|40314|1703180|446|18559|1|113|1175|55|36.48|72.23|13.72|0.00|754.60|2006.40|3972.65|37.73|0.00|754.60|792.33|-1251.80| +2452555|49733|837|40314|1703180|446|18559|1|38|1175|95|69.65|114.22|49.11|0.00|4665.45|6616.75|10850.90|233.27|0.00|4665.45|4898.72|-1951.30| +2452555|49733|11628|40314|1703180|446|18559|1|7|1175|87|21.06|36.22|12.67|0.00|1102.29|1832.22|3151.14|22.04|0.00|1102.29|1124.33|-729.93| +||17880||1703180|446||||1175|6|70.47|95.83|38.33||||574.98|||||-192.84| +2452555|49733|17737|40314|1703180|446|18559|1|123|1175|69|22.10|31.38|30.43|0.00|2099.67|1524.90|2165.22|62.99|0.00|2099.67|2162.66|574.77| +2452555|49733|6873|40314|1703180|446|18559|1|204|1175|58|86.34|104.47|91.93|3145.84|5331.94|5007.72|6059.26|196.74|3145.84|2186.10|2382.84|-2821.62| +2452555|49733|16980|40314|1703180|446|18559|1|118|1175|80|87.24|88.98|88.09|0.00|7047.20|6979.20|7118.40|563.77|0.00|7047.20|7610.97|68.00| +2452608|64825|12775|25651|1748785|719|9406|2|192|1176|40|26.29|34.17|22.89|466.95|915.60|1051.60|1366.80|31.40|466.95|448.65|480.05|-602.95| +2452608|64825|13191|25651|1748785|719|9406|2|66|1176|2|72.65|74.10|60.76|0.00|121.52|145.30|148.20|4.86|0.00|121.52|126.38|-23.78| +||13740|||719|9406||90|1176|26||140.82||0.00|2855.58|2440.88||142.77|0.00||2998.35|414.70| +2452608||6933||1748785|||2||1176||93.55|||266.54|||||266.54|13060.58|13321.79|| +2452608|64825|13848|25651|1748785|719|9406|2|192|1176|50|83.64|116.25|49.98|1699.32|2499.00|4182.00|5812.50|71.97|1699.32|799.68|871.65|-3382.32| +2452608|64825|17028|25651|1748785|719|9406|2|138|1176|20|28.93|48.60|12.63|5.05|252.60|578.60|972.00|4.95|5.05|247.55|252.50|-331.05| +2452608|64825|10848|25651|1748785|719|9406|2|190|1176|5|43.88|86.88|17.37|0.00|86.85|219.40|434.40|3.47|0.00|86.85|90.32|-132.55| +2452608|64825|6513|25651|1748785|719|9406|2|245|1176|89|84.34|103.73|17.63|0.00|1569.07|7506.26|9231.97|47.07|0.00|1569.07|1616.14|-5937.19| +2452608|64825|4765|25651|1748785|719|9406|2|212|1176|70|36.84|36.84|36.84|0.00|2578.80|2578.80|2578.80|51.57|0.00|2578.80|2630.37|0.00| +2452608|64825|1221|25651|1748785|719|9406|2|182|1176|43|16.13|16.29|14.66|0.00|630.38|693.59|700.47|6.30|0.00|630.38|636.68|-63.21| +2452608|64825|6543|25651|1748785|719|9406|2|238|1176|53|18.94|29.54|28.65|0.00|1518.45|1003.82|1565.62|91.10|0.00|1518.45|1609.55|514.63| +2452608|64825|6912|25651|1748785|719|9406|2|285|1176|29|90.53|107.73|100.18|1772.18|2905.22|2625.37|3124.17|11.33|1772.18|1133.04|1144.37|-1492.33| +2452608|64825|11910|25651|1748785|719|9406|2|209|1176|75|48.98|54.85|23.03|0.00|1727.25|3673.50|4113.75|17.27|0.00|1727.25|1744.52|-1946.25| +2452608|64825|6768|25651|1748785|719|9406|2|131|1176|94|16.19|24.77|2.47|0.00|232.18|1521.86|2328.38|2.32|0.00|232.18|234.50|-1289.68| +2452608|64825|708|25651|1748785|719|9406|2|185|1176|1|21.46|36.26|36.26|11.24|36.26|21.46|36.26|0.00|11.24|25.02|25.02|3.56| +2452608|64825|2424|25651|1748785|719|9406|2|51|1176|11|7.62|14.17|9.77|0.00|107.47|83.82|155.87|5.37|0.00|107.47|112.84|23.65| +2451017|64258|11623|58162|1517293|2471|18002|4|278|1177|5|88.41|128.19|121.78|0.00|608.90|442.05|640.95|18.26|0.00|608.90|627.16|166.85| +|64258|3895||1517293|||4||1177|||124.55||0.00|1046.00|3299.60||52.30|0.00|1046.00|1098.30|-2253.60| +2451017|64258|1282|58162|1517293|2471|18002|4|116|1177|64|17.16|23.68|2.60|0.00|166.40|1098.24|1515.52|13.31|0.00|166.40|179.71|-931.84| +2451017|64258|4393|58162|1517293|2471|18002|4|236|1177|48|7.41|8.44|7.42|0.00|356.16|355.68|405.12|7.12|0.00|356.16|363.28|0.48| +2451017|64258|8914|58162|1517293|2471|18002|4|71|1177|10|59.47|113.58|88.59|0.00|885.90|594.70|1135.80|0.00|0.00|885.90|885.90|291.20| +2451017|64258|8029|58162|1517293|2471|18002|4|151|1177|68|84.43|140.15|128.93|1490.43|8767.24|5741.24|9530.20|72.76|1490.43|7276.81|7349.57|1535.57| +|64258|16022|58162|1517293|2471|||277|1177|||154.48|||9564.92||13285.28|||9564.92|10330.11|| +2451017|64258|13034|58162|1517293|2471|18002|4|292|1177|46|10.62|12.53|7.01|0.00|322.46|488.52|576.38|0.00|0.00|322.46|322.46|-166.06| +2451017|64258|3370|58162|1517293|2471|18002|4|106|1177|85|31.35|37.62|27.08|0.00|2301.80|2664.75|3197.70|115.09|0.00|2301.80|2416.89|-362.95| +2451606|65472|6497|7929|1480908|1087|46866|1|251|1178|99|42.53|85.06|71.45|0.00|7073.55|4210.47|8420.94|0.00|0.00|7073.55|7073.55|2863.08| +2451606|65472|3577|7929|1480908|1087|46866|1|136|1178|41|66.35|118.10|94.48|0.00|3873.68|2720.35|4842.10|77.47|0.00|3873.68|3951.15|1153.33| +2451606|65472|800|7929|1480908|1087|46866|1|233|1178|74|50.71|67.95|27.85|0.00|2060.90|3752.54|5028.30|185.48|0.00|2060.90|2246.38|-1691.64| +2451606|65472|11111|7929|1480908|1087|46866|1|148|1178|55|1.61|2.25|1.80|0.00|99.00|88.55|123.75|4.95|0.00|99.00|103.95|10.45| +2451606|65472|8099|7929|1480908|1087|46866|1|32|1178|52|82.24|111.84|44.73|0.00|2325.96|4276.48|5815.68|0.00|0.00|2325.96|2325.96|-1950.52| +2451606|65472|7015|7929|1480908|1087|46866|1|86|1178|31|5.57|10.52|2.73|27.08|84.63|172.67|326.12|1.72|27.08|57.55|59.27|-115.12| +2451606|65472|10955|7929|1480908|1087|46866|1|12|1178|58|21.36|28.40|4.54|0.00|263.32|1238.88|1647.20|2.63|0.00|263.32|265.95|-975.56| +2451606|65472|16139|7929|1480908|1087|46866|1|219|1178|43|32.18|43.12|7.33|0.00|315.19|1383.74|1854.16|9.45|0.00|315.19|324.64|-1068.55| +2451606||11737|7929||1087|46866|1||1178|12|25.88||12.88||154.56|310.56|515.52|||154.56||-156.00| +2451606|65472|11543|7929|1480908|1087|46866|1|290|1178|42|22.76|36.41|12.74|0.00|535.08|955.92|1529.22|16.05|0.00|535.08|551.13|-420.84| +2451606|65472|11293|7929|1480908|1087|46866|1|181|1178|36|65.65|117.51|71.68|1393.45|2580.48|2363.40|4230.36|94.96|1393.45|1187.03|1281.99|-1176.37| +2451606|65472|13025|7929|1480908|1087|46866|1|255|1178|62|60.80|114.91|29.87|0.00|1851.94|3769.60|7124.42|18.51|0.00|1851.94|1870.45|-1917.66| +2451606|65472|12079|7929|1480908|1087|46866|1|90|1178|6|61.70|78.35|9.40|0.00|56.40|370.20|470.10|3.38|0.00|56.40|59.78|-313.80| +2451606|65472|12449|7929|1480908|1087|46866|1|217|1178|1|76.00|77.52|37.20|2.60|37.20|76.00|77.52|0.34|2.60|34.60|34.94|-41.40| +2451843|36610|9278|40143|935150|3757|32811|7|175|1179|15|38.30|40.21|11.66|0.00|174.90|574.50|603.15|12.24|0.00|174.90|187.14|-399.60| +2451843|36610|15647|40143|935150|3757|32811|7|192|1179|60|42.50|42.92|7.29|188.08|437.40|2550.00|2575.20|14.95|188.08|249.32|264.27|-2300.68| +2451843|36610|10340|40143|935150|3757|32811|7|27|1179|25|55.66|106.31|92.48|0.00|2312.00|1391.50|2657.75|208.08|0.00|2312.00|2520.08|920.50| +2451843|36610|13412|40143|935150|3757|32811|7|43|1179|20|62.21|68.43|62.95|0.00|1259.00|1244.20|1368.60|113.31|0.00|1259.00|1372.31|14.80| +2451843|36610|9925|40143|935150|3757|32811|7|107|1179|42|43.35|81.93|17.20|0.00|722.40|1820.70|3441.06|36.12|0.00|722.40|758.52|-1098.30| +2451843|36610|2161|40143|935150|3757|32811|7|63|1179|87|92.03|134.36|134.36|0.00|11689.32|8006.61|11689.32|0.00|0.00|11689.32|11689.32|3682.71| +2451843|36610|6145|40143|935150|3757|32811|7|97|1179|64|80.98|136.85|62.95|2497.85|4028.80|5182.72|8758.40|137.78|2497.85|1530.95|1668.73|-3651.77| +2451843|36610|15727|40143|935150|3757|32811|7|57|1179|43|42.32|75.75|74.23|0.00|3191.89|1819.76|3257.25|223.43|0.00|3191.89|3415.32|1372.13| +2451843|36610|5225|40143|935150|3757|32811|7|22|1179|18|41.72|63.41|34.24|0.00|616.32|750.96|1141.38|24.65|0.00|616.32|640.97|-134.64| +2451843||3853|40143|935150|||||1179|51|34.35|38.12|||408.00|||12.24|||420.24|| +2451843|36610|14801|40143|935150|3757|32811|7|295|1179|52|88.57|139.05|116.80|425.15|6073.60|4605.64|7230.60|282.42|425.15|5648.45|5930.87|1042.81| +2451843|36610|9655|40143|935150|3757|32811|7|8|1179|35|32.63|39.15|34.06|107.28|1192.10|1142.05|1370.25|43.39|107.28|1084.82|1128.21|-57.23| +2451843|36610|5510|40143|935150|3757|32811|7|213|1179|89|49.23|52.67|16.32|0.00|1452.48|4381.47|4687.63|116.19|0.00|1452.48|1568.67|-2928.99| +2451843|36610|9392|40143|935150|3757|32811|7|124|1179|4|53.87|80.26|13.64|6.00|54.56|215.48|321.04|2.42|6.00|48.56|50.98|-166.92| +2452602|34858|1053|5008|183053|1545|49990|4|249|1180|38|68.83|87.41|12.23|0.00|464.74|2615.54|3321.58|27.88|0.00|464.74|492.62|-2150.80| +2452602|34858|4903|5008|183053|1545|49990|4|191|1180|80|41.23|62.66|41.98|0.00|3358.40|3298.40|5012.80|0.00|0.00|3358.40|3358.40|60.00| +2452602|34858|15115|5008|183053|1545|49990|4|119|1180|46|52.28|98.80|33.59|0.00|1545.14|2404.88|4544.80|139.06|0.00|1545.14|1684.20|-859.74| +2452602|34858|9343|5008|183053|1545|49990|4|298|1180|44|39.93|67.08|21.46|623.19|944.24|1756.92|2951.52|25.68|623.19|321.05|346.73|-1435.87| +2452602|34858|2952|5008|183053|1545|49990|4|2|1180|64|79.01|136.68|27.33|0.00|1749.12|5056.64|8747.52|0.00|0.00|1749.12|1749.12|-3307.52| +2452602|34858|2289|5008|183053|1545|49990|4|59|1180|22|62.15|110.62|96.23|0.00|2117.06|1367.30|2433.64|21.17|0.00|2117.06|2138.23|749.76| +2452602||16614|5008|183053|1545|49990||13|1180|||81.70||0.00|||490.20||0.00|132.30||-266.28| +|34858|17671|5008|183053|||4|27|1180||72.14|111.09||0.00|||||0.00|2332.40|2402.37|-2717.40| +2452602|34858|6654|5008|183053|1545|49990|4|231|1180|7|25.52|31.64|29.10|177.21|203.70|178.64|221.48|0.79|177.21|26.49|27.28|-152.15| +2452602|34858|7635|5008|183053|1545|49990|4|224|1180|34|90.21|111.86|52.57|482.59|1787.38|3067.14|3803.24|39.14|482.59|1304.79|1343.93|-1762.35| +2452602|34858|2625|5008|183053|1545|49990|4|219|1180|59|54.85|75.14|21.03|0.00|1240.77|3236.15|4433.26|86.85|0.00|1240.77|1327.62|-1995.38| +2452119|61025|1985|53227|977049|871|33093|4|198|1181|45|60.82|80.28|6.42|127.11|288.90|2736.90|3612.60|12.94|127.11|161.79|174.73|-2575.11| +2452119|61025|5023|53227|977049|871|33093|4|270|1181|77|12.69|19.16|9.96|0.00|766.92|977.13|1475.32|61.35|0.00|766.92|828.27|-210.21| +2452119|61025|1909|53227|977049|871|33093|4|63|1181|22|1.28|1.70|1.20|12.67|26.40|28.16|37.40|0.68|12.67|13.73|14.41|-14.43| +2452119|61025|3097|53227|977049|871|33093|4|98|1181|71|72.82|99.03|24.75|0.00|1757.25|5170.22|7031.13|70.29|0.00|1757.25|1827.54|-3412.97| +2452119|61025|2197|53227|977049|871|33093|4|18|1181|14|86.72|130.94|43.21|0.00|604.94|1214.08|1833.16|48.39|0.00|604.94|653.33|-609.14| +2452119|61025|5991|53227|977049|871|33093|4|90|1181|9|83.15|98.94|40.56|14.60|365.04|748.35|890.46|14.01|14.60|350.44|364.45|-397.91| +2452119|61025|10821|53227|977049|871|33093|4|257|1181|100|57.70|63.47|58.39|0.00|5839.00|5770.00|6347.00|175.17|0.00|5839.00|6014.17|69.00| +2452119|61025|16243|53227|977049|871|33093|4|229|1181|74|61.80|71.68|43.72|0.00|3235.28|4573.20|5304.32|194.11|0.00|3235.28|3429.39|-1337.92| +2452119|61025|2915|53227|977049|871|33093|4|274|1181|16|19.68|32.47|27.59|154.50|441.44|314.88|519.52|8.60|154.50|286.94|295.54|-27.94| +2452119|61025|17575|53227|977049|871|33093|4|18|1181|21|39.72|71.49|43.60|0.00|915.60|834.12|1501.29|18.31|0.00|915.60|933.91|81.48| +2452119|61025|6315|53227|977049|871|33093|4|200|1181|38|60.53|71.42|26.42|0.00|1003.96|2300.14|2713.96|30.11|0.00|1003.96|1034.07|-1296.18| +2452119|61025|5153|53227|977049|871|33093|4|189|1181|84|25.58|44.50|2.67|0.00|224.28|2148.72|3738.00|11.21|0.00|224.28|235.49|-1924.44| +2452391|69044|3895|6345|1506705|4218|37378|8|70|1182|84|15.80|23.54|12.00|0.00|1008.00|1327.20|1977.36|30.24|0.00|1008.00|1038.24|-319.20| +2452391|69044|1284|6345|1506705|4218|37378|8|80|1182|21|47.58|87.07|57.46|0.00|1206.66|999.18|1828.47|36.19|0.00|1206.66|1242.85|207.48| +2452391|69044|4393|6345|1506705|4218|37378|8|9|1182|48|25.94|32.16|31.51|0.00|1512.48|1245.12|1543.68|45.37|0.00|1512.48|1557.85|267.36| +2452391|69044|8916|6345|1506705|4218|37378|8|57|1182|23|60.61|115.15|26.48|140.07|609.04|1394.03|2648.45|18.75|140.07|468.97|487.72|-925.06| +2452391|69044|8029|6345|1506705|4218|37378|8|86|1182|11|43.18|69.08|18.65|0.00|205.15|474.98|759.88|16.41|0.00|205.15|221.56|-269.83| +2452391|69044|16023|6345|1506705|4218|37378|8|99|1182|2|11.77|14.71|7.94|0.00|15.88|23.54|29.42|0.79|0.00|15.88|16.67|-7.66| +2452391|69044|13035|6345|1506705|4218|37378|8|237|1182|34|1.90|3.55|2.66|0.00|90.44|64.60|120.70|5.42|0.00|90.44|95.86|25.84| +2452391|69044|3372|6345|1506705|4218|37378|8|179|1182|93|81.81|139.07|126.55|3766.12|11769.15|7608.33|12933.51|640.24|3766.12|8003.03|8643.27|394.70| +2452391|69044|13633|6345|1506705|4218|37378|8|97|1182|57|30.88|48.48|22.30|0.00|1271.10|1760.16|2763.36|12.71|0.00|1271.10|1283.81|-489.06| +2452391|69044|14586|6345|1506705|4218|37378|8|203|1182|80|69.96|131.52|118.36|0.00|9468.80|5596.80|10521.60|568.12|0.00|9468.80|10036.92|3872.00| +2452391|69044|9865|6345|1506705|4218|37378|8|271|1182|84|37.87|66.65|50.65|0.00|4254.60|3181.08|5598.60|170.18|0.00|4254.60|4424.78|1073.52| +2452391|69044|6879|6345|1506705|4218|37378|8|114|1182|60|16.32|20.40|17.54|0.00|1052.40|979.20|1224.00|31.57|0.00|1052.40|1083.97|73.20| +2452391|69044|17487||||37378|8|37|1182|||147.32|66.29||729.19|852.94|1620.52||||736.48|-123.75| +2452391|69044|1149|6345|1506705|4218|37378|8|133|1182|83|83.76|145.74|62.66|0.00|5200.78|6952.08|12096.42|104.01|0.00|5200.78|5304.79|-1751.30| +2452420|39661|2277|43414|202416|4618|9620|2|110|1183|14|99.89|175.80|38.67|0.00|541.38|1398.46|2461.20|10.82|0.00|541.38|552.20|-857.08| +2452420|39661|5895|43414|202416|4618|9620|2|138|1183|38|91.72|124.73|69.84|0.00|2653.92|3485.36|4739.74|0.00|0.00|2653.92|2653.92|-831.44| +2452420|39661|2742|43414|202416|4618|9620|2|58|1183|71|43.28|78.33|64.23|2690.59|4560.33|3072.88|5561.43|149.57|2690.59|1869.74|2019.31|-1203.14| +2452420|39661|12192|43414|202416|4618|9620|2|163|1183|33|96.47|96.47|73.31|0.00|2419.23|3183.51|3183.51|217.73|0.00|2419.23|2636.96|-764.28| +2452420|39661|8253|43414|202416|4618|9620|2|251|1183|13|16.14|29.05|10.45|0.00|135.85|209.82|377.65|6.79|0.00|135.85|142.64|-73.97| +2452420|39661|6667|43414|202416|4618|9620|2|62|1183|51|31.81|41.98|39.88|0.00|2033.88|1622.31|2140.98|20.33|0.00|2033.88|2054.21|411.57| +2452420|39661|5364|43414|202416|4618|9620|2|191|1183|97|39.57|77.95|41.31|0.00|4007.07|3838.29|7561.15|160.28|0.00|4007.07|4167.35|168.78| +2452420|39661|13359|43414|202416|4618|9620|2|244|1183|34|20.39|34.86|27.88|255.93|947.92|693.26|1185.24|34.59|255.93|691.99|726.58|-1.27| +2452420|39661|9081|43414|202416|4618|9620|2|106|1183|68|65.83|121.78|81.59|0.00|5548.12|4476.44|8281.04|221.92|0.00|5548.12|5770.04|1071.68| +2452420|39661|951|43414|202416|4618|9620|2|186|1183|34|40.52|49.83|0.00|0.00|0.00|1377.68|1694.22|0.00|0.00|0.00|0.00|-1377.68| +2452420|39661|17892|43414|202416|4618|9620|2|186|1183|13|84.05|160.53|130.02|0.00|1690.26|1092.65|2086.89|84.51|0.00|1690.26|1774.77|597.61| +2452420|39661|1914|43414|202416|4618|9620|2|296|1183|54|32.07|42.65|38.38|0.00|2072.52|1731.78|2303.10|62.17|0.00|2072.52|2134.69|340.74| +2452420|39661|9090|43414|202416|4618|9620|2|73|1183|78|88.74|120.68|6.03|0.00|470.34|6921.72|9413.04|18.81|0.00|470.34|489.15|-6451.38| +2451069|32805|9182|13446|321105|6364|8823|4|299|1184|19|36.93|67.21|17.47|0.00|331.93|701.67|1276.99|13.27|0.00|331.93|345.20|-369.74| +2451069|32805|16780|13446|321105|6364|8823|4|135|1184|100|92.40|177.40|51.44|0.00|5144.00|9240.00|17740.00|360.08|0.00|5144.00|5504.08|-4096.00| +|32805|298||321105|6364||4|167|1184|35||||0.00|2881.90|3245.90|||0.00||3054.81|| +2451069|32805|2413|13446|321105|6364|8823|4|185|1184|14|74.59|99.20|37.69|0.00|527.66|1044.26|1388.80|0.00|0.00|527.66|527.66|-516.60| +2451069|32805|7525|13446|321105|6364|8823|4|193|1184|1|95.91|99.74|28.92|28.92|28.92|95.91|99.74|0.00|28.92|0.00|0.00|-95.91| +|32805|17167|13446||6364|8823|4||1184|19||9.63|||78.66||182.97|||||| +2451069|32805|6566|13446|321105|6364|8823|4|146|1184|30|89.81|124.83|98.61|0.00|2958.30|2694.30|3744.90|266.24|0.00|2958.30|3224.54|264.00| +2451069|32805|12613|13446|321105|6364|8823|4|44|1184|11|45.37|54.44|18.50|0.00|203.50|499.07|598.84|4.07|0.00|203.50|207.57|-295.57| +2451069|32805|5048|13446|321105|6364|8823|4|55|1184|9|67.38|128.69|33.45|0.00|301.05|606.42|1158.21|12.04|0.00|301.05|313.09|-305.37| +2451069|32805|2938|13446|321105|6364|8823|4|9|1184|92|47.43|62.13|10.56|0.00|971.52|4363.56|5715.96|68.00|0.00|971.52|1039.52|-3392.04| +2451069|32805|6199|13446|321105|6364|8823|4|277|1184|67|4.56|5.88|5.05|0.00|338.35|305.52|393.96|16.91|0.00|338.35|355.26|32.83| +2451069|32805|1874|13446|321105|6364|8823|4|294|1184|51|28.05|37.58|19.54|0.00|996.54|1430.55|1916.58|79.72|0.00|996.54|1076.26|-434.01| +2451069|32805|10615|13446|321105|6364|8823|4|201|1184|78|23.83|29.78|22.93|0.00|1788.54|1858.74|2322.84|53.65|0.00|1788.54|1842.19|-70.20| +2451495|37070|7372|36340|1621919|1098|14677|7|87|1185|29|4.31|4.95|4.40|52.31|127.60|124.99|143.55|0.00|52.31|75.29|75.29|-49.70| +2451495|37070|874|36340|1621919|1098|14677|7|124|1185|23|28.19|54.68|39.36|0.00|905.28|648.37|1257.64|0.00|0.00|905.28|905.28|256.91| +2451495|37070|1052|36340|1621919|1098|14677|7|291|1185|56|32.54|40.02|0.80|21.05|44.80|1822.24|2241.12|0.23|21.05|23.75|23.98|-1798.49| +2451495|37070|4903|36340|1621919|1098|14677|7|87|1185|33|65.31|65.96|36.27|0.00|1196.91|2155.23|2176.68|83.78|0.00|1196.91|1280.69|-958.32| +2451495|37070|15115|36340|1621919|1098|14677|7|128|1185|43|76.25|132.67|59.70|2002.33|2567.10|3278.75|5704.81|33.88|2002.33|564.77|598.65|-2713.98| +2451495|37070|9343|36340|1621919|1098|14677|7|149|1185|52|46.76|92.11|13.81|0.00|718.12|2431.52|4789.72|64.63|0.00|718.12|782.75|-1713.40| +2451495|37070|2950|36340|1621919|1098|14677|7|181|1185|19|8.66|9.69|4.65|0.00|88.35|164.54|184.11|0.00|0.00|88.35|88.35|-76.19| +2451495|37070|2288|36340|1621919|1098|14677|7|85|1185|6|51.29|70.26|10.53|0.00|63.18|307.74|421.56|3.79|0.00|63.18|66.97|-244.56| +2451495|37070|16612|36340|1621919|1098|14677|7|193|1185|90|30.92|51.63|40.78|0.00|3670.20|2782.80|4646.70|220.21|0.00|3670.20|3890.41|887.40| +2452148|29800|2141|92109|270121|6533|22545|8|76|1186|97|92.41|159.86|97.51|0.00|9458.47|8963.77|15506.42|94.58|0.00|9458.47|9553.05|494.70| +2452148|29800|17235|92109|270121|6533|22545|8|156|1186|91|47.61|52.84|30.64|0.00|2788.24|4332.51|4808.44|195.17|0.00|2788.24|2983.41|-1544.27| +2452148|29800|6335|92109|270121|6533|22545|8|158|1186|11|97.38|125.62|59.04|0.00|649.44|1071.18|1381.82|19.48|0.00|649.44|668.92|-421.74| +2452148|29800|1669|92109|270121|6533|22545|8|87|1186|13|70.54|79.71|73.33|0.00|953.29|917.02|1036.23|0.00|0.00|953.29|953.29|36.27| +2452148|29800|3229|92109|270121|6533|22545|8|136|1186|80|40.20|77.98|18.71|0.00|1496.80|3216.00|6238.40|134.71|0.00|1496.80|1631.51|-1719.20| +2452148|29800|16625|92109|270121|6533|22545|8|79|1186|12|89.10|170.18|158.26|0.00|1899.12|1069.20|2042.16|75.96|0.00|1899.12|1975.08|829.92| +2452148|29800|7707|92109|270121|6533|22545|8|269|1186|45|31.23|54.96|17.03|574.76|766.35|1405.35|2473.20|17.24|574.76|191.59|208.83|-1213.76| +2452148|29800|8721|92109|270121|6533|22545|8|227|1186|70|92.78|119.68|39.49|0.00|2764.30|6494.60|8377.60|221.14|0.00|2764.30|2985.44|-3730.30| +2451091|68556|6944|99283|1613268|3606|15944|8|221|1187|50|68.67|125.66|23.87|0.00|1193.50|3433.50|6283.00|83.54|0.00|1193.50|1277.04|-2240.00| +2451091|68556|14140|99283|1613268|3606|15944|8|233|1187|80|38.72|59.62|53.06|0.00|4244.80|3097.60|4769.60|297.13|0.00|4244.80|4541.93|1147.20| +2451091|68556|13486|99283|1613268|3606|15944|8|184|1187|37|19.08|29.19|28.31|0.00|1047.47|705.96|1080.03|0.00|0.00|1047.47|1047.47|341.51| +2451091|68556|17518|99283|1613268|3606|15944|8|14|1187|99|42.14|67.84|2.71|0.00|268.29|4171.86|6716.16|0.00|0.00|268.29|268.29|-3903.57| +2451091|68556|7111|99283|1613268|3606|15944|8|87|1187|60|36.77|40.44|15.77|0.00|946.20|2206.20|2426.40|37.84|0.00|946.20|984.04|-1260.00| +2451091|68556|12208|99283|1613268|3606|15944|8|220|1187|9|80.77|84.00|57.12|0.00|514.08|726.93|756.00|25.70|0.00|514.08|539.78|-212.85| +2451091|68556|14023|99283|1613268|3606|15944|8|210|1187|37|69.69|137.98|107.62|0.00|3981.94|2578.53|5105.26|278.73|0.00|3981.94|4260.67|1403.41| +2451091|68556|2899|99283|1613268|3606|15944|8|196|1187|46|3.00|4.14|2.23|0.00|102.58|138.00|190.44|9.23|0.00|102.58|111.81|-35.42| +2451091|68556|15068|99283|1613268|3606|15944|8|174|1187|88|8.12|12.42|3.47|0.00|305.36|714.56|1092.96|3.05|0.00|305.36|308.41|-409.20| +2451091|68556|1280|99283|1613268|3606|15944|8|265|1187|79|85.33|162.98|21.18|0.00|1673.22|6741.07|12875.42|0.00|0.00|1673.22|1673.22|-5067.85| +2451091|68556|16081|99283|1613268|3606|15944|8|35|1187|16|17.22|23.76|3.08|0.00|49.28|275.52|380.16|0.00|0.00|49.28|49.28|-226.24| +2451091|68556|1312|99283|1613268|3606|15944|8|256|1187|8|60.10|68.51|17.81|11.39|142.48|480.80|548.08|0.00|11.39|131.09|131.09|-349.71| +2451091|68556|7135|99283|1613268|3606|15944|8|168|1187|3|98.21|148.29|5.93|0.00|17.79|294.63|444.87|1.42|0.00|17.79|19.21|-276.84| +2451572|61449|4673|48647|1772687|5880|12673|7|208|1188|78|33.01|36.31|5.80|0.00|452.40|2574.78|2832.18|4.52|0.00|452.40|456.92|-2122.38| +2451572|61449|9971|48647|1772687|5880|12673|7|244|1188|85|82.93|142.63|15.68|866.32|1332.80|7049.05|12123.55|37.31|866.32|466.48|503.79|-6582.57| +2451572|61449|12451|48647|1772687|5880|12673|7|71|1188|20|32.82|42.00|39.48|0.00|789.60|656.40|840.00|47.37|0.00|789.60|836.97|133.20| +2451572|61449|3197|48647|1772687|5880|12673|7|180|1188|39|38.20|75.25|50.41|98.29|1965.99|1489.80|2934.75|112.06|98.29|1867.70|1979.76|377.90| +2451572|61449|9854|48647|1772687|5880|12673|7|18|1188|53|35.90|57.08|42.23|0.00|2238.19|1902.70|3025.24|0.00|0.00|2238.19|2238.19|335.49| +2451572|61449|5648|48647|1772687|5880|12673|7|210|1188|82|57.23|64.09|0.64|24.14|52.48|4692.86|5255.38|2.26|24.14|28.34|30.60|-4664.52| +2451572|61449|241|48647|1772687|5880|12673|7|20|1188|25|39.43|55.59|3.89|0.00|97.25|985.75|1389.75|8.75|0.00|97.25|106.00|-888.50| +2451572|61449|9839|48647|1772687|5880|12673|7|253|1188|60|97.03|120.31|4.81|0.00|288.60|5821.80|7218.60|14.43|0.00|288.60|303.03|-5533.20| +2451572|61449|13406|48647|1772687|5880|12673|7|131|1188|57|65.49|113.95|34.18|0.00|1948.26|3732.93|6495.15|97.41|0.00|1948.26|2045.67|-1784.67| +2451572|61449|11306|48647|1772687|5880|12673|7|160|1188|96|38.64|59.11|56.74|0.00|5447.04|3709.44|5674.56|163.41|0.00|5447.04|5610.45|1737.60| +2451572|61449|6107|48647|1772687|5880|12673|7|254|1188|86|11.95|12.06|10.85|0.00|933.10|1027.70|1037.16|46.65|0.00|933.10|979.75|-94.60| +2451572|61449|13999|48647|1772687|5880|12673|7|289|1188|62|39.72|71.89|23.00|0.00|1426.00|2462.64|4457.18|128.34|0.00|1426.00|1554.34|-1036.64| +2451550|37263|17378|73714|677529|2497|19112|1|277|1189|13|54.64|63.92|3.19|0.00|41.47|710.32|830.96|3.31|0.00|41.47|44.78|-668.85| +2451550|37263|10087|73714|677529|2497|19112|1|124|1189|55|99.76|149.64|68.83|2952.80|3785.65|5486.80|8230.20|0.00|2952.80|832.85|832.85|-4653.95| +2451550|37263|5888|73714|677529|2497|19112|1|129|1189|9|42.17|81.80|67.89|0.00|611.01|379.53|736.20|18.33|0.00|611.01|629.34|231.48| +2451550|37263|16627|73714|677529|2497|19112|1|271|1189|88|9.62|19.14|6.12|0.00|538.56|846.56|1684.32|43.08|0.00|538.56|581.64|-308.00| +2451550|37263|15455|73714|677529|2497|19112|1|218|1189|30|45.11|75.33|42.93|0.00|1287.90|1353.30|2259.90|77.27|0.00|1287.90|1365.17|-65.40| +||12206|73714|677529|2497||1|141|1189|||||||7527.60|14001.30|1019.23|||13759.63|5212.80| +2451550|37263|16994|73714|677529|2497|19112|1|282|1189|62|33.90|58.64|7.03|0.00|435.86|2101.80|3635.68|8.71|0.00|435.86|444.57|-1665.94| +2451550|37263|14024|73714|677529|2497|19112|1|254|1189|42|68.37|73.83|0.00|0.00|0.00|2871.54|3100.86|0.00|0.00|0.00|0.00|-2871.54| +2452582|66913|9669|85124|140360|396|26879|4|170|1190|51|8.30|14.77|2.95|0.00|150.45|423.30|753.27|13.54|0.00|150.45|163.99|-272.85| +2452582|66913|14493|85124|140360|396|26879|4|188|1190|19|22.10|35.80|32.93|0.00|625.67|419.90|680.20|12.51|0.00|625.67|638.18|205.77| +2452582|66913|15996|85124|140360|396|26879|4|241|1190|78|44.81|63.63|27.99|0.00|2183.22|3495.18|4963.14|0.00|0.00|2183.22|2183.22|-1311.96| +2452582|66913|3931|85124|140360|396|26879|4|94|1190|4|90.66|117.85|60.10|0.00|240.40|362.64|471.40|19.23|0.00|240.40|259.63|-122.24| +2452582|66913|7291|85124|140360|396|26879|4|268|1190|17|2.19|3.28|1.04|0.00|17.68|37.23|55.76|0.17|0.00|17.68|17.85|-19.55| +2452582|66913|13851|85124|140360|396|26879|4|75|1190|40|78.11|142.16|28.43|0.00|1137.20|3124.40|5686.40|0.00|0.00|1137.20|1137.20|-1987.20| +2452582|66913|109|85124|140360|396|26879|4|138|1190|50|55.30|100.09|34.03|765.67|1701.50|2765.00|5004.50|84.22|765.67|935.83|1020.05|-1829.17| +2452582|66913|2283|85124|140360|396|26879|4|267|1190|95|52.22|97.65|38.08|144.70|3617.60|4960.90|9276.75|312.56|144.70|3472.90|3785.46|-1488.00| +2452582|66913|16213|85124|140360|396|26879|4|62|1190|64|76.76|118.21|9.45|0.00|604.80|4912.64|7565.44|24.19|0.00|604.80|628.99|-4307.84| +2452582|66913|12960|85124|140360|396|26879|4|291|1190|38|41.67|76.67|0.76|0.00|28.88|1583.46|2913.46|2.31|0.00|28.88|31.19|-1554.58| +2452582|66913|9733|85124|140360|396|26879|4|251|1190|19|42.63|50.30|12.57|0.00|238.83|809.97|955.70|11.94|0.00|238.83|250.77|-571.14| +2452582|66913|9552|85124|140360|396|26879|4|107|1190|38|13.45|24.74|20.53|296.45|780.14|511.10|940.12|0.00|296.45|483.69|483.69|-27.41| +2452582|66913|1843|85124|140360|396|26879|4|4|1190|33|41.70|70.47|61.30|1982.44|2022.90|1376.10|2325.51|1.21|1982.44|40.46|41.67|-1335.64| +2452582|66913|5472|85124|140360|396|26879|4|29|1190|65|45.97|78.60|16.50|0.00|1072.50|2988.05|5109.00|32.17|0.00|1072.50|1104.67|-1915.55| +2452582|66913|4557|85124|140360|396|26879|4|167|1190|67|99.39|105.35|32.65|0.00|2187.55|6659.13|7058.45|131.25|0.00|2187.55|2318.80|-4471.58| +2452582||11299||140360|||||1190|||81.60|39.98|0.00|||||0.00|||| +2452429|43166|16188|60060|537894|4975|22409|1|47|1191|33|96.78|153.88|95.40|975.94|3148.20|3193.74|5078.04|65.16|975.94|2172.26|2237.42|-1021.48| +2452429|43166|7674|60060|537894|4975|22409|1|97|1191|93|62.00|100.44|94.41|0.00|8780.13|5766.00|9340.92|790.21|0.00|8780.13|9570.34|3014.13| +2452429|43166|13867|||||||1191|65|53.25||||4765.80|3461.25|5883.80|285.94||||| +2452429|43166|5071|60060|537894|4975|22409|1|205|1191|82|98.38|120.02|50.40|371.95|4132.80|8067.16|9841.64|263.25|371.95|3760.85|4024.10|-4306.31| +2452429|43166|4695|60060|537894|4975|22409|1|69|1191|26|51.81|66.83|2.67|63.17|69.42|1347.06|1737.58|0.37|63.17|6.25|6.62|-1340.81| +2452429|43166|17635|60060|537894|4975|22409|1|176|1191|2|42.57|61.30|2.45|4.26|4.90|85.14|122.60|0.01|4.26|0.64|0.65|-84.50| +2452429|43166|4866|60060|537894|4975|22409|1|152|1191|10|19.98|37.16|17.83|0.00|178.30|199.80|371.60|1.78|0.00|178.30|180.08|-21.50| +2452429|43166|12939|60060|537894|4975|22409|1|11|1191|20|64.48|115.41|78.47|1428.15|1569.40|1289.60|2308.20|7.06|1428.15|141.25|148.31|-1148.35| +2452429||6759||537894|4975|22409||242|1191||1.82|||0.00|289.71|158.34|||0.00|||131.37| +2452429|43166|9696|60060|537894|4975|22409|1|61|1191|38|57.65|101.46|62.90|0.00|2390.20|2190.70|3855.48|71.70|0.00|2390.20|2461.90|199.50| +2452429|43166|11562|60060|537894|4975|22409|1|75|1191|69|63.94|108.05|19.44|0.00|1341.36|4411.86|7455.45|80.48|0.00|1341.36|1421.84|-3070.50| +2452429|43166|2172|60060|537894|4975|22409|1|234|1191|27|1.06|1.93|0.90|0.00|24.30|28.62|52.11|0.97|0.00|24.30|25.27|-4.32| +2452429|43166|12549|60060|537894|4975|22409|1|81|1191|60|22.43|26.46|18.25|0.00|1095.00|1345.80|1587.60|76.65|0.00|1095.00|1171.65|-250.80| +2452429|43166|13554|60060|537894|4975|22409|1|183|1191|24|77.16|132.71|74.31|0.00|1783.44|1851.84|3185.04|0.00|0.00|1783.44|1783.44|-68.40| +2452429|43166|12357|60060|537894|4975|22409|1|208|1191|44|9.92|14.38|8.62|0.00|379.28|436.48|632.72|0.00|0.00|379.28|379.28|-57.20| +2451485|46716|868|14050|1007577|4657|35384|4|112|1192|28|2.66|3.56|2.17|0.00|60.76|74.48|99.68|4.86|0.00|60.76|65.62|-13.72| +2451485|46716|17240|14050|1007577|4657|35384|4|124|1192|41|28.55|55.10|15.42|0.00|632.22|1170.55|2259.10|50.57|0.00|632.22|682.79|-538.33| +2451485|46716|16894|14050|1007577|4657|35384|4|205|1192|17|84.25|117.10|36.30|0.00|617.10|1432.25|1990.70|24.68|0.00|617.10|641.78|-815.15| +2451485|46716|6140||1007577|4657||4||1192|85|35.35|45.95|41.81||3553.85|3004.75|3905.75|||||| +2451485|46716|2191|14050|1007577|4657|35384|4|260|1192|92|94.39|165.18|3.30|291.45|303.60|8683.88|15196.56|0.48|291.45|12.15|12.63|-8671.73| +2451485|46716|7723|14050|1007577|4657|35384|4|1|1192|69|84.95|156.30|31.26|884.34|2156.94|5861.55|10784.70|50.90|884.34|1272.60|1323.50|-4588.95| +2451485|46716|2108|14050|1007577|4657|35384|4|113|1192|56|19.51|26.92|17.76|0.00|994.56|1092.56|1507.52|49.72|0.00|994.56|1044.28|-98.00| +2451485|46716|11758|14050|1007577|4657|35384|4|116|1192|12|75.54|124.64|12.46|0.00|149.52|906.48|1495.68|7.47|0.00|149.52|156.99|-756.96| +2451485|46716|4598|14050|1007577|4657|35384|4|101|1192|20|66.80|78.82|62.26|0.00|1245.20|1336.00|1576.40|0.00|0.00|1245.20|1245.20|-90.80| +2451485|46716|14047|14050|1007577|4657|35384|4|279|1192|54|39.01|53.05|12.20|0.00|658.80|2106.54|2864.70|13.17|0.00|658.80|671.97|-1447.74| +2451485|46716|16273|14050|1007577|4657|35384|4|109|1192|77|91.07|172.12|70.56|0.00|5433.12|7012.39|13253.24|54.33|0.00|5433.12|5487.45|-1579.27| +2451511|75012|14797|56532|1006636|2693|14397|4|211|1193|63|77.16|109.56|25.19|0.00|1586.97|4861.08|6902.28|142.82|0.00|1586.97|1729.79|-3274.11| +2451511|75012|3889|56532|1006636|2693|14397|4|270|1193|90|85.22|123.56|49.42|0.00|4447.80|7669.80|11120.40|44.47|0.00|4447.80|4492.27|-3222.00| +2451511|75012|9538|56532|1006636|2693|14397|4|102|1193|89|53.32|80.51|27.37|0.00|2435.93|4745.48|7165.39|194.87|0.00|2435.93|2630.80|-2309.55| +2451511|75012|14821|56532|1006636|2693|14397|4|20|1193|48|27.56|53.74|48.90|0.00|2347.20|1322.88|2579.52|164.30|0.00|2347.20|2511.50|1024.32| +2451511|75012|74|56532|1006636|2693|14397|4|142|1193|32|30.97|49.86|42.87|288.08|1371.84|991.04|1595.52|54.18|288.08|1083.76|1137.94|92.72| +2451511|75012|616|56532|1006636|2693|14397|4|27|1193|85|10.63|19.55|16.42|0.00|1395.70|903.55|1661.75|69.78|0.00|1395.70|1465.48|492.15| +2451511|75012|1778|56532|1006636|2693|14397|4|25|1193|48|75.56|118.62|112.68|0.00|5408.64|3626.88|5693.76|378.60|0.00|5408.64|5787.24|1781.76| +2451511|75012|15992|56532|1006636|2693|14397|4|224|1193|70|68.52|119.22|69.14|0.00|4839.80|4796.40|8345.40|290.38|0.00|4839.80|5130.18|43.40| +2451511|75012|17492|56532|1006636|2693|14397|4|295|1193|98|25.58|37.85|11.73|0.00|1149.54|2506.84|3709.30|57.47|0.00|1149.54|1207.01|-1357.30| +2451511|75012|3188|56532|1006636|2693|14397|4|149|1193|33|83.25|87.41|8.74|0.00|288.42|2747.25|2884.53|17.30|0.00|288.42|305.72|-2458.83| +2451511|75012|5326|56532|1006636|2693|14397|4|60|1193|12|43.49|52.62|51.04|0.00|612.48|521.88|631.44|0.00|0.00|612.48|612.48|90.60| +2451511|75012|15776|56532|1006636|2693|14397|4|67|1193|96|83.09|158.70|11.10|0.00|1065.60|7976.64|15235.20|42.62|0.00|1065.60|1108.22|-6911.04| +2451511|75012|17770|56532||||4||1193|83|69.18|70.56|19.05|||5741.94||0.00||1581.15|1581.15|| +2452246|58968|9569|11041|991298|1955|6565|10|211|1194|65|31.05|37.57|4.13|0.00|268.45|2018.25|2442.05|2.68|0.00|268.45|271.13|-1749.80| +2452246|58968|16125|11041|991298|1955|6565|10|200|1194|4|76.97|104.67|50.24|0.00|200.96|307.88|418.68|6.02|0.00|200.96|206.98|-106.92| +2452246|58968|14437|11041|991298|1955|6565|10|46|1194|85|14.12|16.94|4.57|0.00|388.45|1200.20|1439.90|23.30|0.00|388.45|411.75|-811.75| +2452246|58968|4311|11041|991298|1955|6565|10|269|1194|17|1.58|2.57|0.84|8.42|14.28|26.86|43.69|0.00|8.42|5.86|5.86|-21.00| +2452246|58968|15811|11041|991298|1955|6565|10|44|1194|44|77.81|128.38|50.06|0.00|2202.64|3423.64|5648.72|176.21|0.00|2202.64|2378.85|-1221.00| +||9783|11041|||6565|||1194|94|60.11|84.15|56.38||5299.72|||52.99|||5352.71|| +2452246|58968|5125|11041|991298|1955|6565|10|113|1194|84|92.63|150.06|114.04|6609.75|9579.36|7780.92|12605.04|59.39|6609.75|2969.61|3029.00|-4811.31| +2452246|58968|1409|11041|991298|1955|6565|10|130|1194|39|63.78|108.42|102.99|1325.48|4016.61|2487.42|4228.38|53.82|1325.48|2691.13|2744.95|203.71| +2452246|58968|6529|11041|991298|1955|6565|10|28|1194|15|19.28|25.25|6.31|45.43|94.65|289.20|378.75|3.93|45.43|49.22|53.15|-239.98| +2452246|58968|15633|11041|991298|1955|6565|10|113|1194|8|11.29|17.95|0.35|0.00|2.80|90.32|143.60|0.11|0.00|2.80|2.91|-87.52| +2452246||15551||991298||6565|||1194|69|71.31|||0.00|5788.41||||0.00|5788.41||| +2451521|60067|17162|39359|1401884|4056|8581|7|271|1195|30|77.63|83.84|83.84|0.00|2515.20|2328.90|2515.20|176.06|0.00|2515.20|2691.26|186.30| +2451521|60067|6217|39359|1401884|4056|8581|7|100|1195|23|95.68|112.90|97.09|0.00|2233.07|2200.64|2596.70|66.99|0.00|2233.07|2300.06|32.43| +2451521|60067|7462|39359|1401884|4056|8581|7|71|1195|39|48.14|66.43|47.16|0.00|1839.24|1877.46|2590.77|110.35|0.00|1839.24|1949.59|-38.22| +2451521|60067|13874|39359|1401884|4056|8581|7|67|1195|3|2.18|4.01|0.16|0.00|0.48|6.54|12.03|0.00|0.00|0.48|0.48|-6.06| +2451521|60067|14810|39359|1401884|4056|8581|7|24|1195|14|40.65|69.10|56.66|0.00|793.24|569.10|967.40|63.45|0.00|793.24|856.69|224.14| +2451521|60067|6685|39359|1401884|4056|8581|7|272|1195|22|9.72|16.03|8.33|0.00|183.26|213.84|352.66|14.66|0.00|183.26|197.92|-30.58| +2451521|60067|17146|39359|1401884|4056|8581|7|257|1195|66|4.78|8.69|3.12|0.00|205.92|315.48|573.54|14.41|0.00|205.92|220.33|-109.56| +2451521|60067|15862|39359|1401884|4056|8581|7|286|1195|3|54.79|107.38|18.25|0.00|54.75|164.37|322.14|1.64|0.00|54.75|56.39|-109.62| +2451521|60067|13756|39359|1401884|4056|8581|7|7|1195|13|43.82|78.87|61.51|0.00|799.63|569.66|1025.31|0.00|0.00|799.63|799.63|229.97| +2451521|60067|12904|39359|1401884|4056|8581|7|233|1195|72|71.66|128.98|77.38|0.00|5571.36|5159.52|9286.56|445.70|0.00|5571.36|6017.06|411.84| +2451521|60067|2642|39359|1401884|4056|8581|7|183|1195|75|76.14|143.14|123.10|0.00|9232.50|5710.50|10735.50|276.97|0.00|9232.50|9509.47|3522.00| +|60067|2216|39359||4056|8581||296|1195||65.19||61.95|418.16||1629.75|||418.16|1130.59||-499.16| +2451521|60067|5654|39359|1401884|4056|8581|7|181|1195|91|72.87|83.80|59.49|0.00|5413.59|6631.17|7625.80|324.81|0.00|5413.59|5738.40|-1217.58| +2452405|45748|8367|10197|1422965|886|32054|8|235|1196|95|70.93|116.32|90.72|0.00|8618.40|6738.35|11050.40|430.92|0.00|8618.40|9049.32|1880.05| +2452405|45748|9372|10197|1422965|886|32054|8|271|1196|92|65.58|71.48|57.18|0.00|5260.56|6033.36|6576.16|210.42|0.00|5260.56|5470.98|-772.80| +2452405|45748|13287|10197|1422965|886|32054|8|40|1196|39|72.55|137.84|8.27|112.88|322.53|2829.45|5375.76|0.00|112.88|209.65|209.65|-2619.80| +2452405|45748|15693|10197|1422965|886|32054|8|119|1196|53|77.14|103.36|50.64|1341.96|2683.92|4088.42|5478.08|0.00|1341.96|1341.96|1341.96|-2746.46| +2452405|45748|11676|10197|1422965|886|32054|8|267|1196|30|17.07|20.31|1.62|0.00|48.60|512.10|609.30|0.48|0.00|48.60|49.08|-463.50| +2452405|45748|3519|10197|1422965|886|32054|8|5|1196|78|63.47|87.58|22.77|799.22|1776.06|4950.66|6831.24|87.91|799.22|976.84|1064.75|-3973.82| +2452405|45748|5707|10197|1422965|886|32054|8|261|1196|87|62.15|67.12|24.16|0.00|2101.92|5407.05|5839.44|63.05|0.00|2101.92|2164.97|-3305.13| +2452405|45748|15102|10197|1422965|886|32054|8|125|1196|63|71.87|122.17|7.33|0.00|461.79|4527.81|7696.71|36.94|0.00|461.79|498.73|-4066.02| +2452405|45748|16410|10197|1422965|886|32054|8|194|1196|97|80.34|94.80|47.40|229.89|4597.80|7792.98|9195.60|218.39|229.89|4367.91|4586.30|-3425.07| +2452405|45748|14185|10197|1422965|886|32054|8|182|1196|82|66.46|89.05|23.15|0.00|1898.30|5449.72|7302.10|151.86|0.00|1898.30|2050.16|-3551.42| +2452405|45748|10545|10197|1422965|886|32054|8|166|1196|22|12.11|16.83|4.54|0.00|99.88|266.42|370.26|3.99|0.00|99.88|103.87|-166.54| +2452405|45748|6075|10197|1422965|886|32054|8|120|1196|94|14.74|21.81|3.92|0.00|368.48|1385.56|2050.14|25.79|0.00|368.48|394.27|-1017.08| +2452405|45748|2715|10197|1422965|886|32054|8|224|1196|32|47.15|51.39|0.00|0.00|0.00|1508.80|1644.48|0.00|0.00|0.00|0.00|-1508.80| +2451068|43104|11950|86780|1640736|3139|3495|2|122|1197|96|39.94|59.51|29.15|0.00|2798.40|3834.24|5712.96|27.98|0.00|2798.40|2826.38|-1035.84| +2451068|43104|13351|86780|1640736|3139|3495|2|245|1197|4|21.87|32.58|19.22|0.00|76.88|87.48|130.32|0.76|0.00|76.88|77.64|-10.60| +2451068||7396|86780|1640736|3139|3495|||1197||62.88||49.64||198.56||422.52|13.89||198.56||-52.96| +2451068|43104|7981|86780|1640736|3139|3495|2|289|1197|55|38.37|65.22|41.08|0.00|2259.40|2110.35|3587.10|22.59|0.00|2259.40|2281.99|149.05| +2451068|43104|10114|86780|1640736|3139|3495|2|213|1197|74|77.66|86.97|17.39|0.00|1286.86|5746.84|6435.78|77.21|0.00|1286.86|1364.07|-4459.98| +2451068|43104|16348|86780|1640736|3139|3495|2|261|1197|98|39.02|42.92|4.72|0.00|462.56|3823.96|4206.16|32.37|0.00|462.56|494.93|-3361.40| +2451068|43104|11095|86780|1640736|3139|3495|2|289|1197|36|20.04|35.27|9.17|0.00|330.12|721.44|1269.72|26.40|0.00|330.12|356.52|-391.32| +2451068|43104|1478|86780|1640736|3139|3495|2|203|1197|43|59.01|101.49|8.11|278.98|348.73|2537.43|4364.07|0.00|278.98|69.75|69.75|-2467.68| +2451068|43104|7363|86780|1640736|3139|3495|2|6|1197|85|99.37|104.33|54.25|3412.32|4611.25|8446.45|8868.05|47.95|3412.32|1198.93|1246.88|-7247.52| +2451068|43104|13132|86780|1640736|3139|3495|2|12|1197|38|88.11|94.27|88.61|0.00|3367.18|3348.18|3582.26|202.03|0.00|3367.18|3569.21|19.00| +2451068|43104|5308|86780|1640736|3139|3495|2|258|1197|33|23.14|40.49|8.50|134.64|280.50|763.62|1336.17|8.75|134.64|145.86|154.61|-617.76| +2451068|43104|811|86780|1640736|3139|3495|2|234|1197|78|52.30|91.00|28.21|0.00|2200.38|4079.40|7098.00|110.01|0.00|2200.38|2310.39|-1879.02| +2451068|43104|3265|86780|1640736|3139|3495|2|17|1197|2|6.80|10.60|8.16|0.00|16.32|13.60|21.20|0.32|0.00|16.32|16.64|2.72| +2451529|29108|10490|61578|610672|2966|32507|4|300|1198|48|72.78|101.16|43.49|0.00|2087.52|3493.44|4855.68|83.50|0.00|2087.52|2171.02|-1405.92| +2451529|29108|8212|61578|610672|2966|32507|4|99|1198|91|92.90|178.36|24.97|0.00|2272.27|8453.90|16230.76|45.44|0.00|2272.27|2317.71|-6181.63| +2451529|29108|2809|61578|610672|2966|32507|4|261|1198|25|11.35|12.59|6.79|0.00|169.75|283.75|314.75|1.69|0.00|169.75|171.44|-114.00| +2451529|29108|11570|61578|610672|2966|32507|4|31|1198|20|9.88|18.37|4.59|0.00|91.80|197.60|367.40|1.83|0.00|91.80|93.63|-105.80| +2451529|29108|16832|61578|610672|2966|32507|4|195|1198|18|98.44|156.51|136.16|0.00|2450.88|1771.92|2817.18|24.50|0.00|2450.88|2475.38|678.96| +2451529|29108|17414|61578|610672|2966|32507|4|52|1198|13|45.52|90.58|64.31|0.00|836.03|591.76|1177.54|0.00|0.00|836.03|836.03|244.27| +2451529|29108|2947|61578|610672|2966|32507|4|49|1198|13|89.82|157.18|0.00|0.00|0.00|1167.66|2043.34|0.00|0.00|0.00|0.00|-1167.66| +2451529|29108|15836|61578|610672|2966|32507|4|59|1198|84|43.68|44.55|25.83|0.00|2169.72|3669.12|3742.20|130.18|0.00|2169.72|2299.90|-1499.40| +2451529|29108|4579|61578|610672|2966|32507|4|281|1198|98|54.46|77.87|42.04|0.00|4119.92|5337.08|7631.26|370.79|0.00|4119.92|4490.71|-1217.16| +2451529|29108|5918|61578|610672|2966|32507|4|57|1198|65|74.27|79.46|72.30|2913.69|4699.50|4827.55|5164.90|53.57|2913.69|1785.81|1839.38|-3041.74| +2451529|29108|6229|61578|610672|2966|32507|4|60|1198|57|81.20|144.53|119.95|3486.94|6837.15|4628.40|8238.21|234.51|3486.94|3350.21|3584.72|-1278.19| +2451529|29108|11140|61578|610672|2966|32507|4|189|1198|85|78.84|88.30|83.88|0.00|7129.80|6701.40|7505.50|570.38|0.00|7129.80|7700.18|428.40| +2451529|29108|3736|61578|610672|2966|32507|4|229|1198|57|73.09|87.70|52.62|0.00|2999.34|4166.13|4998.90|59.98|0.00|2999.34|3059.32|-1166.79| +2451529||13471||||32507|4||1198||76.22||77.06|||381.10||15.41||385.30||| +2451529|29108|3877|61578|610672|2966|32507|4|29|1198|85|41.06|58.30|23.32|0.00|1982.20|3490.10|4955.50|0.00|0.00|1982.20|1982.20|-1507.90| +2451529|29108|16412|61578|610672|2966|32507|4|144|1198|64|58.99|58.99|27.72|798.33|1774.08|3775.36|3775.36|19.51|798.33|975.75|995.26|-2799.61| +2451356|61390|7453|80022|362396|5217|40791|8|119|1199|64|10.01|12.31|2.95|0.00|188.80|640.64|787.84|11.32|0.00|188.80|200.12|-451.84| +2451356|61390|15193|80022|362396|5217|40791|8|184|1199|100|29.63|52.74|49.04|0.00|4904.00|2963.00|5274.00|0.00|0.00|4904.00|4904.00|1941.00| +||3086|80022|362396|5217|40791|||1199|76|59.89||55.09|3810.02||4551.64|||3810.02||406.96|-4174.82| +2451356|61390|6002|80022|362396|5217|40791|8|274|1199|42|61.65|96.17|93.28|0.00|3917.76|2589.30|4039.14|274.24|0.00|3917.76|4192.00|1328.46| +2451356|61390|1618|80022|362396|5217|40791|8|81|1199|71|86.92|112.99|99.43|0.00|7059.53|6171.32|8022.29|494.16|0.00|7059.53|7553.69|888.21| +2451356|61390|4414|80022|||40791|||1199||88.36|||||||||||-359.89| +2451356|61390|3884|80022|362396|5217|40791|8|98|1199|25|43.40|74.64|9.70|0.00|242.50|1085.00|1866.00|9.70|0.00|242.50|252.20|-842.50| +2451356|61390|1616|80022|362396|5217|40791|8|261|1199|48|63.63|101.17|101.17|2719.44|4856.16|3054.24|4856.16|42.73|2719.44|2136.72|2179.45|-917.52| +2451356||5438|80022|362396|5217||||1199||76.06||38.10|||3346.64||||1676.40||| +2451550|65326|7255|83944|1016630|516|10802|10|126|1200|46|83.82|134.11|52.30|0.00|2405.80|3855.72|6169.06|24.05|0.00|2405.80|2429.85|-1449.92| +2451550|65326|15259|83944|1016630|516|10802|10|29|1200|21|88.74|173.93|154.79|0.00|3250.59|1863.54|3652.53|130.02|0.00|3250.59|3380.61|1387.05| +2451550|65326|11018|83944|1016630|516|10802|10|41|1200|56|27.72|43.52|37.86|0.00|2120.16|1552.32|2437.12|21.20|0.00|2120.16|2141.36|567.84| +2451550|65326|14555|83944|1016630|516|10802|10|31|1200|24|89.91|154.64|44.84|0.00|1076.16|2157.84|3711.36|96.85|0.00|1076.16|1173.01|-1081.68| +2451550|65326|2222|83944|1016630|516|10802|10|73|1200|37|79.45|137.44|87.96|0.00|3254.52|2939.65|5085.28|195.27|0.00|3254.52|3449.79|314.87| +2451550|65326|3557|83944|1016630|516|10802|10|129|1200|60|22.06|30.88|3.08|0.00|184.80|1323.60|1852.80|12.93|0.00|184.80|197.73|-1138.80| +2451550|65326|8459|83944|1016630|516|10802|10|103|1200|29|36.86|68.55|3.42|0.00|99.18|1068.94|1987.95|5.95|0.00|99.18|105.13|-969.76| +2451550|65326|9701|83944|1016630|516|10802|10|95|1200|33|82.46|108.02|101.53|770.61|3350.49|2721.18|3564.66|103.19|770.61|2579.88|2683.07|-141.30| +2451550|65326|6415|83944|1016630|516|10802|10|126|1200|87|57.13|66.84|38.09|0.00|3313.83|4970.31|5815.08|298.24|0.00|3313.83|3612.07|-1656.48| +||12500||1016630|516|||127|1200|||38.59|6.17||302.33||1890.91|||||| +2451364|43963|8950|33554|45300|3446|14392|10|198|1201|58|81.91|130.23|27.34|206.14|1585.72|4750.78|7553.34|110.36|206.14|1379.58|1489.94|-3371.20| +||994||45300||14392||284|1201|64||66.01|34.98|||4023.68|4224.64|44.77||2238.72|2283.49|| +2451364|43963|6643|33554|45300|3446|14392|10|134|1201|5|83.48|84.31|12.64|39.81|63.20|417.40|421.55|0.70|39.81|23.39|24.09|-394.01| +2451364|43963|9722|33554|45300|3446|14392|10|106|1201|96|68.68|108.51|15.19|0.00|1458.24|6593.28|10416.96|131.24|0.00|1458.24|1589.48|-5135.04| +2451364|43963|1747|33554|45300|3446|14392|10|66|1201|72|24.30|24.54|10.55|0.00|759.60|1749.60|1766.88|68.36|0.00|759.60|827.96|-990.00| +2451364|43963|7738|33554|45300|3446|14392|10|54|1201|57|38.86|45.46|19.09|0.00|1088.13|2215.02|2591.22|65.28|0.00|1088.13|1153.41|-1126.89| +2451364|43963|8600|33554|45300|3446|14392|10|294|1201|75|67.82|85.45|74.34|2118.69|5575.50|5086.50|6408.75|138.27|2118.69|3456.81|3595.08|-1629.69| +2451364|43963|8032|33554|45300|3446|14392|10|142|1201|68|27.86|49.86|20.44|0.00|1389.92|1894.48|3390.48|27.79|0.00|1389.92|1417.71|-504.56| +2451364|43963|16957|33554|45300|3446|14392|10|99|1201|26|76.08|142.26|41.25|0.00|1072.50|1978.08|3698.76|32.17|0.00|1072.50|1104.67|-905.58| +2451364|43963|6370|33554|45300|3446|14392|10|244|1201|70|12.70|22.98|22.75|0.00|1592.50|889.00|1608.60|95.55|0.00|1592.50|1688.05|703.50| +2451364|43963|15298|33554|45300|3446|14392|10|231|1201|62|8.11|14.19|10.35|0.00|641.70|502.82|879.78|51.33|0.00|641.70|693.03|138.88| +2451364|43963|13837|33554|45300|3446|14392|10|8|1201|34|31.20|57.72|55.98|1065.85|1903.32|1060.80|1962.48|25.12|1065.85|837.47|862.59|-223.33| +2451364|43963|14536|33554|45300|3446|14392|10|173|1201|16|71.39|125.64|33.92|0.00|542.72|1142.24|2010.24|5.42|0.00|542.72|548.14|-599.52| +2451364|43963|646|33554|45300|3446|14392|10|131|1201|83|53.75|61.81|60.57|0.00|5027.31|4461.25|5130.23|50.27|0.00|5027.31|5077.58|566.06| +2450884|53752|5491|1768|125354|5387|40973|4|109|1202|81|87.62|115.65|60.13|0.00|4870.53|7097.22|9367.65|389.64|0.00|4870.53|5260.17|-2226.69| +||14846||125354|5387|40973|4|82|1202|99||46.82|20.60|1672.30||3287.79||7.34|1672.30|367.10||-2920.69| +2450884|53752|8864|1768|125354|5387|40973|4|287|1202|33|92.66|156.59|101.78|0.00|3358.74|3057.78|5167.47|302.28|0.00|3358.74|3661.02|300.96| +2450884|53752|763|1768|125354|5387|40973|4|206|1202|97|52.78|69.66|62.69|0.00|6080.93|5119.66|6757.02|304.04|0.00|6080.93|6384.97|961.27| +2450884|53752|82|1768|125354|5387|40973|4|147|1202|50|61.54|100.31|89.27|0.00|4463.50|3077.00|5015.50|44.63|0.00|4463.50|4508.13|1386.50| +2450884|53752|8284|1768|125354|5387|40973|4|279|1202|59|60.47|96.14|27.88|0.00|1644.92|3567.73|5672.26|148.04|0.00|1644.92|1792.96|-1922.81| +|53752|6157||125354|5387|40973|||1202||24.40|26.35|10.27|318.37|1027.00||||318.37|708.63|736.97|| +2450884|53752|1264|1768|125354|5387|40973|4|236|1202|55|30.75|33.82|19.61|0.00|1078.55|1691.25|1860.10|53.92|0.00|1078.55|1132.47|-612.70| +2450884|53752|16159|1768|125354|5387|40973|4|205|1202|99|71.43|134.28|116.82|1272.16|11565.18|7071.57|13293.72|205.86|1272.16|10293.02|10498.88|3221.45| +2450884|53752|304|1768|125354|5387|40973|4|29|1202|89|91.37|125.17|76.35|0.00|6795.15|8131.93|11140.13|611.56|0.00|6795.15|7406.71|-1336.78| +2450884|53752|15476|1768|125354|5387|40973|4|49|1202|96|33.56|48.66|48.66|0.00|4671.36|3221.76|4671.36|140.14|0.00|4671.36|4811.50|1449.60| +2450884|53752|13576|1768|125354|5387|40973|4|125|1202|54|40.02|78.03|70.22|0.00|3791.88|2161.08|4213.62|0.00|0.00|3791.88|3791.88|1630.80| +2450989|65125|7190|96184|1261590|4596|32802|7|42|1203|45|16.11|16.11|14.98|0.00|674.10|724.95|724.95|26.96|0.00|674.10|701.06|-50.85| +2450989|65125|10933|96184|1261590|4596|32802|7|172|1203|95|38.41|59.53|51.79|0.00|4920.05|3648.95|5655.35|147.60|0.00|4920.05|5067.65|1271.10| +2450989|65125|6967|96184|1261590|4596|32802|7|270|1203|91|72.93|78.76|37.01|0.00|3367.91|6636.63|7167.16|33.67|0.00|3367.91|3401.58|-3268.72| +2450989|65125|7892|96184|1261590|4596|32802|7|273|1203|9|51.10|67.45|63.40|0.00|570.60|459.90|607.05|17.11|0.00|570.60|587.71|110.70| +2450989|65125|13210|96184|1261590|4596|32802|7|263|1203|87|77.08|131.80|80.39|0.00|6993.93|6705.96|11466.60|489.57|0.00|6993.93|7483.50|287.97| +2450989|65125|4735|96184|1261590|4596|32802|7|50|1203|31|65.70|68.32|9.56|0.00|296.36|2036.70|2117.92|2.96|0.00|296.36|299.32|-1740.34| +2450989|65125|13678|96184|1261590|4596|32802|7|73|1203|59|33.61|50.75|30.95|0.00|1826.05|1982.99|2994.25|36.52|0.00|1826.05|1862.57|-156.94| +2450989|65125|9020|96184|1261590|4596|32802|7|157|1203|4|11.21|21.74|13.04|0.00|52.16|44.84|86.96|0.00|0.00|52.16|52.16|7.32| +2450989|65125|15344|96184|1261590|4596|32802|7|109|1203|24|56.60|66.22|25.16|0.00|603.84|1358.40|1589.28|54.34|0.00|603.84|658.18|-754.56| +2450989|65125|11194|96184|1261590|4596|32802|7|242|1203|14|1.12|1.80|1.78|0.00|24.92|15.68|25.20|1.49|0.00|24.92|26.41|9.24| +2450989|65125|8632|96184|1261590|4596|32802|7|223|1203|2|44.47|81.82|79.36|0.00|158.72|88.94|163.64|6.34|0.00|158.72|165.06|69.78| +2450989|65125|8185|96184|1261590|4596|32802|7|188|1203|33|29.68|48.67|0.97|0.00|32.01|979.44|1606.11|0.64|0.00|32.01|32.65|-947.43| +2452135|51169|1383|35408|971842|3476|16710|2|207|1204|47|75.02|135.03|49.96|0.00|2348.12|3525.94|6346.41|140.88|0.00|2348.12|2489.00|-1177.82| +2452135|51169|16317|35408|971842|3476|16710|2|184|1204|95|71.57|132.40|63.55|0.00|6037.25|6799.15|12578.00|362.23|0.00|6037.25|6399.48|-761.90| +2452135|51169|12353|35408|971842|3476|16710|2|72|1204|15|50.23|52.23|52.23|250.70|783.45|753.45|783.45|37.29|250.70|532.75|570.04|-220.70| +2452135|51169|4341|35408|971842|3476|16710|2|112|1204|71|88.22|113.80|88.76|0.00|6301.96|6263.62|8079.80|567.17|0.00|6301.96|6869.13|38.34| +2452135|51169|12343|35408|971842|3476|16710|2|200|1204|50|55.92|107.36|5.36|259.96|268.00|2796.00|5368.00|0.72|259.96|8.04|8.76|-2787.96| +2452135|51169|16031|35408|971842|3476|16710|2|220|1204|84|72.29|122.89|24.57|0.00|2063.88|6072.36|10322.76|123.83|0.00|2063.88|2187.71|-4008.48| +2452135|51169|11833|35408|971842|3476|16710|2|95|1204|10|4.27|7.85|1.33|0.00|13.30|42.70|78.50|0.13|0.00|13.30|13.43|-29.40| +2452135|51169|8661|35408|971842|3476|16710|2|187|1204|36|13.31|19.83|12.29|79.63|442.44|479.16|713.88|32.65|79.63|362.81|395.46|-116.35| +2452135|51169|3965|35408|971842|3476|16710|2|93|1204|62|33.90|37.62|23.70|0.00|1469.40|2101.80|2332.44|102.85|0.00|1469.40|1572.25|-632.40| +2452135|51169|8533|35408|971842|3476|16710|2|34|1204|80|72.45|120.99|99.21|0.00|7936.80|5796.00|9679.20|317.47|0.00|7936.80|8254.27|2140.80| +2452135||8929|35408|971842||||203|1204|33|91.63|109.03|32.70||1079.10|3023.79||75.53|||1154.63|-1944.69| +2452135|51169|2335|35408|971842|3476|16710|2|178|1204|83|34.91|58.64|35.77|0.00|2968.91|2897.53|4867.12|178.13|0.00|2968.91|3147.04|71.38| +2452135|51169|3349|35408|971842|3476|16710|2|69|1204|1|54.46|78.42|24.31|0.00|24.31|54.46|78.42|2.18|0.00|24.31|26.49|-30.15| +2452135|51169|1629|35408|971842|3476|16710|2|169|1204|31|48.51|78.10|13.27|0.00|411.37|1503.81|2421.10|24.68|0.00|411.37|436.05|-1092.44| +2452135|51169|4121|35408|971842|3476|16710|2|133|1204|81|78.99|134.28|73.85|0.00|5981.85|6398.19|10876.68|239.27|0.00|5981.85|6221.12|-416.34| +2451920|66501|16079|67049|664724|446|39721|2|37|1205|50|42.22|53.61|28.94|1027.37|1447.00|2111.00|2680.50|0.00|1027.37|419.63|419.63|-1691.37| +2451920|66501|15663|67049|664724|446|39721|2|119|1205|30|79.07|116.23|109.25|0.00|3277.50|2372.10|3486.90|65.55|0.00|3277.50|3343.05|905.40| +2451920|66501|13405|67049|664724|446|39721|2|30|1205|98|51.94|70.63|66.39|4684.47|6506.22|5090.12|6921.74|91.08|4684.47|1821.75|1912.83|-3268.37| +2451920|66501|11525|67049|664724|446|39721|2|142|1205|32|21.62|31.56|21.14|155.59|676.48|691.84|1009.92|46.88|155.59|520.89|567.77|-170.95| +2451920|66501|17615|67049|664724|446|39721|2|15|1205|47|93.91|97.66|2.92|0.00|137.24|4413.77|4590.02|2.74|0.00|137.24|139.98|-4276.53| +2451920|66501|1035|67049|664724|446|39721|2|98|1205|65|16.63|31.09|14.30|0.00|929.50|1080.95|2020.85|46.47|0.00|929.50|975.97|-151.45| +2451920|66501|4769|67049|664724|446|39721|2|105|1205|49|82.74|101.77|40.70|0.00|1994.30|4054.26|4986.73|59.82|0.00|1994.30|2054.12|-2059.96| +2451920|66501|503|67049|664724|446|39721|2|24|1205|11|64.95|111.06|34.42|0.00|378.62|714.45|1221.66|15.14|0.00|378.62|393.76|-335.83| +2451920|66501|1357|67049|664724|446|39721|2|205|1205|66|19.66|26.54|1.85|0.00|122.10|1297.56|1751.64|2.44|0.00|122.10|124.54|-1175.46| +2451920|66501|15509|67049|664724|446|39721|2|253|1205|86|74.80|146.60|73.30|4223.54|6303.80|6432.80|12607.60|83.21|4223.54|2080.26|2163.47|-4352.54| +2451920|66501|15821|67049|664724|446|39721|2|230|1205|4|74.82|148.14|145.17|0.00|580.68|299.28|592.56|0.00|0.00|580.68|580.68|281.40| +2451920|66501|6649|67049|664724|446|39721|2|128|1205|41|18.44|30.24|13.60|105.94|557.60|756.04|1239.84|27.09|105.94|451.66|478.75|-304.38| +2451920|66501|15523|67049|664724|446|39721|2|229|1205|81|72.71|92.34|68.33|0.00|5534.73|5889.51|7479.54|498.12|0.00|5534.73|6032.85|-354.78| +2451194|57413|7436|78877|513641|4758|4322|10|27|1206|34|11.72|13.24|8.34|0.00|283.56|398.48|450.16|22.68|0.00|283.56|306.24|-114.92| +2451194|57413|5782|78877|513641|4758|4322|10|182|1206|31|98.79|186.71|14.93|0.00|462.83|3062.49|5788.01|9.25|0.00|462.83|472.08|-2599.66| +2451194|57413|4855|78877|513641|4758|4322|10|52|1206|27|37.35|50.79|42.66|0.00|1151.82|1008.45|1371.33|69.10|0.00|1151.82|1220.92|143.37| +||6722|78877|513641|||10||1206|||||||||||747.63||189.41| +2451194|57413|12775|78877|513641|4758|4322|10|94|1206|75|71.97|114.43|69.80|0.00|5235.00|5397.75|8582.25|418.80|0.00|5235.00|5653.80|-162.75| +|57413|13190|78877||4758|4322|||1206|||75.25||4197.09|6082.74|6489.76||169.70|4197.09|1885.65||| +2451194|57413|13738|78877|513641|4758|4322|10|58|1206|26|39.60|40.39|11.71|0.00|304.46|1029.60|1050.14|0.00|0.00|304.46|304.46|-725.14| +2451194|57413|6932|78877|513641|4758|4322|10|43|1206|13|70.53|107.20|32.16|0.00|418.08|916.89|1393.60|29.26|0.00|418.08|447.34|-498.81| +2451194|57413|13846|78877|513641|4758|4322|10|248|1206|76|10.85|18.66|0.93|0.00|70.68|824.60|1418.16|2.12|0.00|70.68|72.80|-753.92| +2451194|57413|17026|78877|513641|4758|4322|10|183|1206|73|66.15|130.31|79.48|0.00|5802.04|4828.95|9512.63|116.04|0.00|5802.04|5918.08|973.09| +2451194|57413|10846|78877|513641|4758|4322|10|21|1206|30|82.81|90.26|19.85|0.00|595.50|2484.30|2707.80|11.91|0.00|595.50|607.41|-1888.80| +2451194|57413|6512|78877|513641|4758|4322|10|169|1206|38|94.53|109.65|35.08|0.00|1333.04|3592.14|4166.70|106.64|0.00|1333.04|1439.68|-2259.10| +2452245|40187|17989|25766|735876|833|27743|7|266|1207|40|76.03|136.85|56.10|0.00|2244.00|3041.20|5474.00|179.52|0.00|2244.00|2423.52|-797.20| +2452245|40187|13569|25766|735876|833|27743|7|112|1207|21|69.59|91.16|29.17|0.00|612.57|1461.39|1914.36|6.12|0.00|612.57|618.69|-848.82| +2452245||6829||735876|833|27743|||1207||||||1228.92|||61.44||1228.92||| +2452245|40187|2183|25766|735876|833|27743|7|264|1207|5|62.62|107.70|76.46|0.00|382.30|313.10|538.50|11.46|0.00|382.30|393.76|69.20| +2452245|40187|1087|25766|735876|833|27743|7|13|1207|49|80.41|86.84|78.15|0.00|3829.35|3940.09|4255.16|268.05|0.00|3829.35|4097.40|-110.74| +2452245|40187|11187|25766|735876|833|27743|7|213|1207|33|62.73|104.75|63.89|0.00|2108.37|2070.09|3456.75|105.41|0.00|2108.37|2213.78|38.28| +2452245|40187|14281|25766|735876|833|27743|7|271|1207|29|83.40|140.11|137.30|318.53|3981.70|2418.60|4063.19|73.26|318.53|3663.17|3736.43|1244.57| +2452245|40187|7081|25766|735876|833|27743|7|230|1207|22|4.58|8.33|1.49|0.00|32.78|100.76|183.26|0.65|0.00|32.78|33.43|-67.98| +2452245|40187|6603|25766|735876|833|27743|7|108|1207|98|18.87|25.47|8.91|0.00|873.18|1849.26|2496.06|17.46|0.00|873.18|890.64|-976.08| +2452245|40187|11055|25766|735876||27743|7||1207||||21.97|0.00|43.94||||0.00|||-110.78| +2452245|40187|3873|25766|735876|833|27743|7|294|1207|63|43.98|72.56|68.20|0.00|4296.60|2770.74|4571.28|257.79|0.00|4296.60|4554.39|1525.86| +2452245|40187|15661|25766|735876|833|27743|7|12|1207|5|67.14|91.98|22.99|0.00|114.95|335.70|459.90|3.44|0.00|114.95|118.39|-220.75| +2452245|40187|3797|25766|735876|833|27743|7|192|1207|29|1.64|2.19|1.70|0.00|49.30|47.56|63.51|1.97|0.00|49.30|51.27|1.74| +2451609|32733|6143|47065|269877|242|3054|2|262|1208|49|64.42|78.59|14.14|0.00|692.86|3156.58|3850.91|34.64|0.00|692.86|727.50|-2463.72| +2451609|32733|10685|47065|269877|242|3054|2|283|1208|98|82.54|161.77|100.29|3439.94|9828.42|8088.92|15853.46|511.07|3439.94|6388.48|6899.55|-1700.44| +2451609|32733|16451|47065|269877|242|3054|2|256|1208|59|88.07|95.11|49.45|0.00|2917.55|5196.13|5611.49|58.35|0.00|2917.55|2975.90|-2278.58| +2451609|32733|8132|47065|269877|242|3054|2|22|1208|26|39.01|47.20|6.13|0.00|159.38|1014.26|1227.20|3.18|0.00|159.38|162.56|-854.88| +2451609|32733|15751|47065|269877|242|3054|2|160|1208|21|54.49|81.73|29.42|0.00|617.82|1144.29|1716.33|30.89|0.00|617.82|648.71|-526.47| +2451609|32733|14509|47065|269877|242|3054|2|148|1208|88|67.87|69.22|49.14|0.00|4324.32|5972.56|6091.36|345.94|0.00|4324.32|4670.26|-1648.24| +||6458|47065||242||2||1208|89||12.85|5.78|||800.11|1143.65|||514.42|529.85|| +2451609|32733|1772|47065|269877|242|3054|2|211|1208|2|75.23|111.34|5.56|1.22|11.12|150.46|222.68|0.00|1.22|9.90|9.90|-140.56| +2451609|32733|2648|47065|269877|242|3054|2|79|1208|93|15.21|16.42|10.01|0.00|930.93|1414.53|1527.06|46.54|0.00|930.93|977.47|-483.60| +2451609|32733|17101|47065|269877|242|3054|2|147|1208|35|89.29|100.00|92.00|2318.40|3220.00|3125.15|3500.00|72.12|2318.40|901.60|973.72|-2223.55| +2451609|32733|3056|47065|269877|242|3054|2|263|1208|78|48.82|95.19|1.90|0.00|148.20|3807.96|7424.82|10.37|0.00|148.20|158.57|-3659.76| +2451609|32733|9301|47065|269877|242|3054|2|199|1208|80|9.37|9.46|1.79|0.00|143.20|749.60|756.80|10.02|0.00|143.20|153.22|-606.40| +2451609|32733|3476|47065|269877|242|3054|2|291|1208|88|25.70|38.55|14.26|0.00|1254.88|2261.60|3392.40|87.84|0.00|1254.88|1342.72|-1006.72| +||7697|47065||242||||1208||12.90|22.83|||451.88|283.80||10.12||126.53||-157.27| +2452099|63229|15435|26586|1518851|4716|37781|10|31|1209|31|32.53|49.77|25.38|0.00|786.78|1008.43|1542.87|31.47|0.00|786.78|818.25|-221.65| +2452099|63229|3035|26586|1518851|4716|37781|10|229|1209|77|97.15|111.72|5.58|0.00|429.66|7480.55|8602.44|8.59|0.00|429.66|438.25|-7050.89| +2452099|63229|16215|26586|1518851|4716|37781|10|120|1209|5|50.91|94.18|45.20|0.00|226.00|254.55|470.90|6.78|0.00|226.00|232.78|-28.55| +2452099|63229|3219|26586|1518851|4716|37781|10|21|1209|88|91.51|102.49|97.36|0.00|8567.68|8052.88|9019.12|771.09|0.00|8567.68|9338.77|514.80| +2452099|63229|11619|26586|1518851|4716|37781|10|191|1209|83|71.60|97.37|59.39|0.00|4929.37|5942.80|8081.71|0.00|0.00|4929.37|4929.37|-1013.43| +2452099|63229|5985|26586|1518851|4716|37781|10|275|1209|77|58.88|73.01|54.02|0.00|4159.54|4533.76|5621.77|124.78|0.00|4159.54|4284.32|-374.22| +2452099|63229|13521|26586|1518851|4716|37781|10|264|1209|4|6.13|6.49|5.25|0.00|21.00|24.52|25.96|1.89|0.00|21.00|22.89|-3.52| +2452099|63229|13545||||37781||255|1209|51||108.72|||||5544.72||||4466.43|| +2452099|63229|687|26586|1518851|4716|37781|10|119|1209|51|92.04|145.42|98.88|3479.58|5042.88|4694.04|7416.42|62.53|3479.58|1563.30|1625.83|-3130.74| +2452099|63229|7117|26586|1518851|4716|37781|10|28|1209|49|19.08|19.84|4.56|0.00|223.44|934.92|972.16|17.87|0.00|223.44|241.31|-711.48| +2451874|47519|13682|48911|841659|2407|12755|4|220|1210|45|79.98|88.77|0.00|0.00|0.00|3599.10|3994.65|0.00|0.00|0.00|0.00|-3599.10| +2451874|47519|9806|48911|841659|2407|12755|4|192|1210|25|2.68|2.70|1.83|0.00|45.75|67.00|67.50|2.74|0.00|45.75|48.49|-21.25| +2451874|47519|17588|48911|841659|2407|12755|4|260|1210|52|12.63|13.26|7.69|259.92|399.88|656.76|689.52|11.19|259.92|139.96|151.15|-516.80| +2451874|47519|2723|48911|841659|2407|12755|4|40|1210|71|78.72|144.84|7.24|0.00|514.04|5589.12|10283.64|10.28|0.00|514.04|524.32|-5075.08| +2451874|47519|1958|48911|841659|2407|12755|4|286|1210|69|2.00|3.78|1.66|0.00|114.54|138.00|260.82|10.30|0.00|114.54|124.84|-23.46| +2451874|47519|9151|48911|841659|2407|12755|4|241|1210|16|99.63|122.54|96.80|526.59|1548.80|1594.08|1960.64|71.55|526.59|1022.21|1093.76|-571.87| +2451874|47519|91|48911|841659|2407|12755|4|135|1210|92|44.81|66.76|51.40|0.00|4728.80|4122.52|6141.92|189.15|0.00|4728.80|4917.95|606.28| +2451874|47519|10772|48911|841659|2407|12755|4|74|1210|74|37.48|67.08|40.91|0.00|3027.34|2773.52|4963.92|272.46|0.00|3027.34|3299.80|253.82| +2451874|47519|2387|48911|841659|2407|12755|4|200|1210|85|78.77|92.16|17.51|0.00|1488.35|6695.45|7833.60|59.53|0.00|1488.35|1547.88|-5207.10| +2451874|47519|15275|48911|841659|2407|12755|4|143|1210|75|73.12|84.81|33.07|0.00|2480.25|5484.00|6360.75|124.01|0.00|2480.25|2604.26|-3003.75| +2451874|47519|10285|48911|841659|2407|12755|4|204|1210|35|28.17|49.01|13.23|0.00|463.05|985.95|1715.35|9.26|0.00|463.05|472.31|-522.90| +2451874|47519|16723|48911|841659|2407|12755|4|91|1210|83|57.41|99.31|15.88|0.00|1318.04|4765.03|8242.73|26.36|0.00|1318.04|1344.40|-3446.99| +2451143|58988|14096|43722|865970|3065|20513|8|30|1211|72|24.01|36.97|0.00|0.00|0.00|1728.72|2661.84|0.00|0.00|0.00|0.00|-1728.72| +2451143|58988|15196|||||8|44|1211|33|||129.35||4268.55|||||4268.55||1296.57| +2451143|58988|3172|43722|865970|3065|20513|8|18|1211|93|38.18|46.19|4.61|0.00|428.73|3550.74|4295.67|0.00|0.00|428.73|428.73|-3122.01| +2451143|58988|17552|43722|865970|3065|20513|8|43|1211|53|58.76|86.96|59.13|0.00|3133.89|3114.28|4608.88|62.67|0.00|3133.89|3196.56|19.61| +2451143|58988|13000|43722|865970|3065|20513|8|68|1211|44|22.45|32.55|22.13|0.00|973.72|987.80|1432.20|68.16|0.00|973.72|1041.88|-14.08| +2451143|58988|7388|43722|865970|3065|20513|8|172|1211|39|7.70|9.93|7.34|271.94|286.26|300.30|387.27|1.14|271.94|14.32|15.46|-285.98| +2451143|58988|12596|43722|865970|3065|20513|8|151|1211|6|41.36|69.07|15.19|0.00|91.14|248.16|414.42|6.37|0.00|91.14|97.51|-157.02| +2451143|58988|4933|43722|865970|3065|20513|8|261|1211|20|53.68|98.23|74.65|0.00|1493.00|1073.60|1964.60|134.37|0.00|1493.00|1627.37|419.40| +2451143|58988|15182|43722|865970|3065|20513|8|288|1211|89|27.42|46.61|7.92|0.00|704.88|2440.38|4148.29|56.39|0.00|704.88|761.27|-1735.50| +2451143|58988|3253|43722|865970|3065|20513|8|99|1211|95|19.23|35.96|14.74|0.00|1400.30|1826.85|3416.20|112.02|0.00|1400.30|1512.32|-426.55| +2451143|58988|12721|43722|865970|3065|20513|8|111|1211|67|19.38|22.86|1.37|27.53|91.79|1298.46|1531.62|1.28|27.53|64.26|65.54|-1234.20| +2451143|58988|12740|43722|865970|3065|20513|8|191|1211|4|25.81|48.00|42.72|0.00|170.88|103.24|192.00|15.37|0.00|170.88|186.25|67.64| +2451143|58988|4028|43722|865970|3065|20513|8|230|1211|87|22.58|41.99|1.25|0.00|108.75|1964.46|3653.13|5.43|0.00|108.75|114.18|-1855.71| +2451143|58988|17728|43722|865970|3065|20513|8|297|1211|63|70.97|90.84|37.24|0.00|2346.12|4471.11|5722.92|23.46|0.00|2346.12|2369.58|-2124.99| +2451143||17992|43722|865970||20513|8||1211||2.23|4.32|2.76|118.34|||276.48||118.34|||| +2451143|58988|682||||20513||116|1211||7.94||0.53|0.55|||53.00||0.55|||| +2451129|53860|16735|62503|966833|5916|6646|8|285|1212|96|51.67|72.33|63.65|0.00|6110.40|4960.32|6943.68|427.72|0.00|6110.40|6538.12|1150.08| +2451129|53860|12874|62503|966833|5916|6646|8|70|1212|80|69.02|125.61|27.63|0.00|2210.40|5521.60|10048.80|154.72|0.00|2210.40|2365.12|-3311.20| +2451129|53860|7954|62503|966833|5916|6646|8|132|1212|61|69.93|97.90|0.00|0.00|0.00|4265.73|5971.90|0.00|0.00|0.00|0.00|-4265.73| +2451129|53860|8324|62503|966833|5916|6646|8|181|1212|6|81.65|138.80|31.92|0.00|191.52|489.90|832.80|0.00|0.00|191.52|191.52|-298.38| +2451129|53860|10436|62503|966833|5916|6646|8|65|1212|21|67.03|109.92|26.38|493.04|553.98|1407.63|2308.32|4.87|493.04|60.94|65.81|-1346.69| +2451129|53860|3157|62503|966833|5916|6646|8|68|1212|12|90.87|180.83|153.70|0.00|1844.40|1090.44|2169.96|129.10|0.00|1844.40|1973.50|753.96| +2451129|53860|2732|||5916|6646||92|1212|40|27.75|30.24||||1110.00|1209.60|||774.00|797.22|-336.00| +2451129|53860|4945|62503|966833|5916|6646|8|80|1212|23|43.86|79.82|26.34|0.00|605.82|1008.78|1835.86|42.40|0.00|605.82|648.22|-402.96| +2452162|34766|8525|64653|505833|3813|9793|4|16|1213|12|11.37|18.07|4.87|0.00|58.44|136.44|216.84|2.33|0.00|58.44|60.77|-78.00| +2452162|34766|10151|64653|505833|3813|9793|4|172|1213|62|23.48|44.61|36.58|0.00|2267.96|1455.76|2765.82|22.67|0.00|2267.96|2290.63|812.20| +2452162|34766|8481|64653|505833|3813|9793|4|149|1213|37|85.06|98.66|6.90|0.00|255.30|3147.22|3650.42|10.21|0.00|255.30|265.51|-2891.92| +2452162|34766|16211|64653|505833|3813|9793|4|24|1213|97|5.85|10.82|4.76|0.00|461.72|567.45|1049.54|36.93|0.00|461.72|498.65|-105.73| +|34766|17181|64653|505833|||4|152|1213|39||||0.00|||430.17||0.00|||17.94| +2452162|34766|14291|64653|505833|3813|9793|4|233|1213|39|84.34|145.06|68.17|0.00|2658.63|3289.26|5657.34|159.51|0.00|2658.63|2818.14|-630.63| +2452162|34766|9419|64653|505833|3813|9793|4|268|1213|2|74.52|106.56|59.67|0.00|119.34|149.04|213.12|4.77|0.00|119.34|124.11|-29.70| +2452162|34766|1485|64653|505833|3813|9793|4|71|1213|35|48.52|61.13|9.16|0.00|320.60|1698.20|2139.55|16.03|0.00|320.60|336.63|-1377.60| +2452162|34766|9269|64653|505833|3813|9793|4|199|1213|92|24.16|30.92|5.56|0.00|511.52|2222.72|2844.64|15.34|0.00|511.52|526.86|-1711.20| +2452162|34766|15323|64653|505833|3813|9793|4|270|1213|56|49.98|91.46|64.93|0.00|3636.08|2798.88|5121.76|0.00|0.00|3636.08|3636.08|837.20| +2452162|34766|13723|64653|505833|3813|9793|4|294|1213|49|12.17|14.84|10.38|0.00|508.62|596.33|727.16|0.00|0.00|508.62|508.62|-87.71| +|34766|13141||505833|||||1213|||||0.00|3777.02||||0.00|||-3333.18| +2452162|34766|13079|64653|505833|3813|9793|4|147|1213|12|84.50|148.72|120.46|0.00|1445.52|1014.00|1784.64|43.36|0.00|1445.52|1488.88|431.52| +2452162|34766|2459|64653|505833|3813|9793|4|235|1213|50|53.23|97.94|15.67|0.00|783.50|2661.50|4897.00|0.00|0.00|783.50|783.50|-1878.00| +2452162|34766|15349|64653|505833|3813|9793|4|117|1213|36|17.40|23.49|10.80|139.96|388.80|626.40|845.64|4.97|139.96|248.84|253.81|-377.56| +||3607|64653||||4|127|1213|87||42.96|17.18||1494.66||3737.52|0.00||||| +2451097|45119|17251|11345|1433368|191|40261|7|216|1214|94|58.12|69.74|32.77|0.00|3080.38|5463.28|6555.56|215.62|0.00|3080.38|3296.00|-2382.90| +2451097|45119|13687|11345|1433368|191|40261|7|136|1214|65|32.44|61.31|52.72|0.00|3426.80|2108.60|3985.15|274.14|0.00|3426.80|3700.94|1318.20| +2451097|45119|13840|11345|1433368|191|40261|7|212|1214|7|95.27|96.22|18.28|0.00|127.96|666.89|673.54|7.67|0.00|127.96|135.63|-538.93| +||14002|11345||191|40261|7|245|1214|72|30.68||24.88|||2208.96||89.56||1791.36|1880.92|-417.60| +2451097|45119|17132|11345|1433368|191|40261|7|132|1214|88|29.53|37.50|24.75|0.00|2178.00|2598.64|3300.00|87.12|0.00|2178.00|2265.12|-420.64| +2451097|45119|10930||1433368|191||||1214||20.89|||0.00|722.50||||0.00|722.50|751.40|12.24| +2451097|45119|1738|11345|1433368|191|40261|7|97|1214|30|46.26|77.25|8.49|0.00|254.70|1387.80|2317.50|0.00|0.00|254.70|254.70|-1133.10| +2451097|45119|3356|11345|1433368|191|40261|7|14|1214|68|80.02|159.23|125.79|4704.54|8553.72|5441.36|10827.64|38.49|4704.54|3849.18|3887.67|-1592.18| +2451097|45119|9368|11345|1433368|191|40261|7|4|1214|75|78.66|106.97|4.27|0.00|320.25|5899.50|8022.75|9.60|0.00|320.25|329.85|-5579.25| +2451097|45119|3200|11345|1433368|191|40261|7|257|1214|40|43.66|57.63|10.37|0.00|414.80|1746.40|2305.20|0.00|0.00|414.80|414.80|-1331.60| +2451097|45119|5290|11345|1433368|191|40261|7|247|1214|47|59.03|69.65|30.64|0.00|1440.08|2774.41|3273.55|0.00|0.00|1440.08|1440.08|-1334.33| +2451097|45119|11528|11345|1433368|191|40261|7|106|1214|79|42.29|65.12|22.79|288.06|1800.41|3340.91|5144.48|45.37|288.06|1512.35|1557.72|-1828.56| +2451097|45119|1604|11345|1433368|191|40261|7|118|1214|92|62.29|64.78|33.68|0.00|3098.56|5730.68|5959.76|247.88|0.00|3098.56|3346.44|-2632.12| +2451407|38386|16876|33365|670345|925|37905|2|160|1215|64|5.35|9.41|6.86|0.00|439.04|342.40|602.24|8.78|0.00|439.04|447.82|96.64| +2451407|38386|12073|33365|670345|925|37905|2|74|1215|37|64.22|82.20|27.12|0.00|1003.44|2376.14|3041.40|50.17|0.00|1003.44|1053.61|-1372.70| +2451407|38386|1321|33365|670345|925|37905|2|232|1215|35|95.61|174.01|139.20|0.00|4872.00|3346.35|6090.35|97.44|0.00|4872.00|4969.44|1525.65| +2451407|38386|15890|33365|670345|925|37905|2|62|1215|48|85.40|104.18|7.29|0.00|349.92|4099.20|5000.64|20.99|0.00|349.92|370.91|-3749.28| +2451407|38386|7933|33365|670345|925|37905|2|300|1215|99|86.87|94.68|2.84|0.00|281.16|8600.13|9373.32|14.05|0.00|281.16|295.21|-8318.97| +2451407|38386|2870|33365|670345|925|37905|2|141|1215|31|21.72|30.62|26.02|274.25|806.62|673.32|949.22|31.94|274.25|532.37|564.31|-140.95| +2451407|38386|12266|33365|670345|925|37905|2|260|1215|6|71.81|106.99|57.77|0.00|346.62|430.86|641.94|24.26|0.00|346.62|370.88|-84.24| +2451407|38386|230|33365|670345|925|37905|2|62|1215|70|89.70|114.81|12.62|803.89|883.40|6279.00|8036.70|5.56|803.89|79.51|85.07|-6199.49| +2451407|38386|9187|33365|670345|925|37905|2|233|1215|4|76.07|151.37|60.54|0.00|242.16|304.28|605.48|12.10|0.00|242.16|254.26|-62.12| +2451407|38386|6476|33365|670345|925|37905|2|8|1215|35|60.66|74.00|22.94|0.00|802.90|2123.10|2590.00|24.08|0.00|802.90|826.98|-1320.20| +2451407|38386|1810|33365|670345|925|37905|2|193|1215|21|90.53|156.61|45.41|0.00|953.61|1901.13|3288.81|66.75|0.00|953.61|1020.36|-947.52| +2450891|62954|17524|82113|1833221|1129|21437|8|154|1216|22|59.07|115.18|69.10|0.00|1520.20|1299.54|2533.96|136.81|0.00|1520.20|1657.01|220.66| +||16969|82113||1129|21437||67|1216||85.34|158.73||0.00||3498.94|6507.93||0.00|1952.01|2030.09|-1546.93| +2450891|62954|7063|82113|1833221|1129|21437|8|284|1216|73|29.64|46.23|0.92|0.00|67.16|2163.72|3374.79|4.70|0.00|67.16|71.86|-2096.56| +2450891|62954|1624|82113|1833221|1129|21437|8|295|1216|67|48.71|50.65|7.09|0.00|475.03|3263.57|3393.55|0.00|0.00|475.03|475.03|-2788.54| +2450891|62954|17371||||21437|8||1216||96.82|||0.00||8036.06|||0.00||9491.20|671.47| +2450891|62954|17215|82113|1833221|1129|21437|8|215|1216|35|67.62|72.35|61.49|0.00|2152.15|2366.70|2532.25|0.00|0.00|2152.15|2152.15|-214.55| +2450891|62954|4681|82113|1833221|1129|21437|8|189|1216|86|35.92|41.30|38.40|0.00|3302.40|3089.12|3551.80|99.07|0.00|3302.40|3401.47|213.28| +2450891|62954|1579|82113|1833221|1129|21437|8|255|1216|69|43.59|84.56|52.42|0.00|3616.98|3007.71|5834.64|217.01|0.00|3616.98|3833.99|609.27| +2450891|62954|17983|82113|1833221|1129|21437|8|264|1216|49|47.85|76.56|60.48|0.00|2963.52|2344.65|3751.44|237.08|0.00|2963.52|3200.60|618.87| +2450891|62954|5378|82113|1833221|1129|21437|8|236|1216|50|74.55|141.64|69.40|2602.50|3470.00|3727.50|7082.00|26.02|2602.50|867.50|893.52|-2860.00| +2451282|52816|7520|23336|686583|6630|6277|2|107|1217|49|17.11|19.50|15.60|0.00|764.40|838.39|955.50|38.22|0.00|764.40|802.62|-73.99| +2451282|52816|5794|23336|686583|6630|6277|2|210|1217|96|57.39|91.82|84.47|0.00|8109.12|5509.44|8814.72|648.72|0.00|8109.12|8757.84|2599.68| +2451282|52816|15590|23336|686583|6630|6277|2|43|1217|71|23.13|33.30|7.65|0.00|543.15|1642.23|2364.30|5.43|0.00|543.15|548.58|-1099.08| +2451282|52816|4720|23336|686583||6277|||1217||19.47||16.59|||1908.06|2804.76|||1105.56||-802.50| +||8701|||||2|18|1217||28.27||35.03|0.00|||2980.53||0.00|1996.71||| +2451282|52816|10030|23336|686583|6630|6277|2|60|1217|3|20.45|25.97|25.97|64.66|77.91|61.35|77.91|0.53|64.66|13.25|13.78|-48.10| +2451282|52816|12379|23336|686583|6630|6277|2|271|1217|62|22.13|33.41|4.00|0.00|248.00|1372.06|2071.42|19.84|0.00|248.00|267.84|-1124.06| +2451282||10861|23336|686583|||2||1217|||74.78|68.79|0.00||267.10|373.90|17.19|0.00|343.95|361.14|| +2451282|52816|2864|23336|686583|6630|6277|2|96|1217|28|98.71|188.53|47.13|0.00|1319.64|2763.88|5278.84|13.19|0.00|1319.64|1332.83|-1444.24| +2451282|52816|15235|23336|686583|6630|6277|2|67|1217|84|36.64|62.65|15.66|131.54|1315.44|3077.76|5262.60|106.55|131.54|1183.90|1290.45|-1893.86| +2451282|52816|5455|23336|686583|6630|6277|2|13|1217|12|29.95|49.11|30.44|36.52|365.28|359.40|589.32|16.43|36.52|328.76|345.19|-30.64| +2451975|30119|1255|86327|955707|5534|34514|10|121|1218|14|49.86|78.28|15.65|0.00|219.10|698.04|1095.92|13.14|0.00|219.10|232.24|-478.94| +2451975|30119|8447|86327|955707|5534|34514|10|88|1218|12|29.81|49.48|14.84|0.00|178.08|357.72|593.76|1.78|0.00|178.08|179.86|-179.64| +2451975|30119|8061||955707|5534|||199|1218|56||||0.00||||285.10|0.00|4072.88||1513.68| +2451975|30119|14751|86327|955707|5534|34514|10|64|1218|6|84.87|140.88|77.48|0.00|464.88|509.22|845.28|18.59|0.00|464.88|483.47|-44.34| +2451975|30119|14947|86327|955707|5534|34514|10|139|1218|10|57.79|87.84|7.02|28.78|70.20|577.90|878.40|2.89|28.78|41.42|44.31|-536.48| +2451975|30119|9911|86327|955707|5534|34514|10|259|1218|61|78.05|124.09|67.00|0.00|4087.00|4761.05|7569.49|286.09|0.00|4087.00|4373.09|-674.05| +2451975|30119|5613|86327|955707|5534|34514|10|138|1218|87|36.63|63.73|12.10|0.00|1052.70|3186.81|5544.51|84.21|0.00|1052.70|1136.91|-2134.11| +2451975|30119|2215|86327|955707|5534|34514|10|61|1218|63|54.81|82.21|13.97|0.00|880.11|3453.03|5179.23|61.60|0.00|880.11|941.71|-2572.92| +2451975|30119|99|86327|955707|5534|34514|10|76|1218|55|73.29|78.42|58.03|0.00|3191.65|4030.95|4313.10|0.00|0.00|3191.65|3191.65|-839.30| +2452052|36959|10507|47711|688115|5319|18476|7|158|1219|84|47.68|65.32|28.74|1955.46|2414.16|4005.12|5486.88|9.17|1955.46|458.70|467.87|-3546.42| +2452052|36959|4471|47711|688115|5319|18476|7|36|1219|57|75.98|85.09|6.80|0.00|387.60|4330.86|4850.13|27.13|0.00|387.60|414.73|-3943.26| +2452052|36959|1973|47711|688115|5319|18476|7|156|1219|53|90.88|109.96|54.98|0.00|2913.94|4816.64|5827.88|174.83|0.00|2913.94|3088.77|-1902.70| +2452052|36959|10939|47711|688115|5319|18476|7|283|1219|73|49.51|67.82|57.64|0.00|4207.72|3614.23|4950.86|294.54|0.00|4207.72|4502.26|593.49| +2452052|36959|817|47711|688115|5319|18476|7|170|1219|96|79.50|112.89|102.72|0.00|9861.12|7632.00|10837.44|295.83|0.00|9861.12|10156.95|2229.12| +2452052|36959|14445|47711|688115|5319|18476|7|197|1219|74|58.90|63.02|49.15|0.00|3637.10|4358.60|4663.48|327.33|0.00|3637.10|3964.43|-721.50| +2452052|36959|7469|47711|688115|5319|18476|7|25|1219|13|15.55|29.54|2.95|0.00|38.35|202.15|384.02|0.38|0.00|38.35|38.73|-163.80| +2452052|36959|375|47711|688115|5319|18476|7|221|1219|28|11.77|22.36|11.40|143.64|319.20|329.56|626.08|12.28|143.64|175.56|187.84|-154.00| +2452052||14557|||5319|||168|1219|74|48.22|||||||||1177.34|1212.66|| +2452265|67354|6787|95412|1205703|6841|45132|2|158|1220|63|86.47|147.86|90.19|0.00|5681.97|5447.61|9315.18|511.37|0.00|5681.97|6193.34|234.36| +|67354|3485||1205703|6841||2|43|1220|89||129.49||0.00|6453.39||11524.61||0.00|6453.39||121.04| +2452265|67354|5257|95412|1205703|6841|45132|2|199|1220|24|72.66|95.18|59.96|0.00|1439.04|1743.84|2284.32|28.78|0.00|1439.04|1467.82|-304.80| +2452265|67354|13597||1205703|6841|45132|||1220||||||||1921.48|63.78||||| +2452265|67354|6913|95412|1205703|6841|45132|2|103|1220|79|7.63|11.59|8.34|0.00|658.86|602.77|915.61|39.53|0.00|658.86|698.39|56.09| +2452265|67354|1489|95412|1205703|6841|45132|2|234|1220|68|91.19|144.08|12.96|396.57|881.28|6200.92|9797.44|4.84|396.57|484.71|489.55|-5716.21| +2452265|67354|11455|95412|1205703|6841|45132|2|288|1220|81|66.70|106.72|22.41|1198.03|1815.21|5402.70|8644.32|0.00|1198.03|617.18|617.18|-4785.52| +2452265|67354|10527|95412|1205703|6841|45132|2|93|1220|8|55.67|106.32|44.65|0.00|357.20|445.36|850.56|32.14|0.00|357.20|389.34|-88.16| +2452265|67354|9379|95412|1205703|6841|45132|2|196|1220|77|72.81|129.60|0.00|0.00|0.00|5606.37|9979.20|0.00|0.00|0.00|0.00|-5606.37| +2452265|67354|8375|95412|1205703|6841|45132|2|41|1220|95|70.36|127.35|92.96|0.00|8831.20|6684.20|12098.25|353.24|0.00|8831.20|9184.44|2147.00| +2452265|67354|16577|95412|1205703|6841|45132|2|246|1220|78|33.52|45.58|26.43|0.00|2061.54|2614.56|3555.24|82.46|0.00|2061.54|2144.00|-553.02| +2452265|67354|3721|95412|1205703|6841|45132|2|56|1220|61|96.99|125.11|92.58|0.00|5647.38|5916.39|7631.71|282.36|0.00|5647.38|5929.74|-269.01| +2452265|67354|12873|95412|1205703|6841|45132|2|72|1220|20|8.69|9.81|2.55|0.00|51.00|173.80|196.20|0.00|0.00|51.00|51.00|-122.80| +2452128|64291|745|91790|511004|6665|36679|1|120|1221|86|60.19|68.61|48.71|0.00|4189.06|5176.34|5900.46|335.12|0.00|4189.06|4524.18|-987.28| +2452128||4289|91790|511004|6665|36679|1|119|1221|||66.57|38.61|0.00||4161.00|6657.00||0.00||3899.61|| +2452128|64291|11267|91790|511004|6665|36679|1|197|1221|48|73.00|81.76|54.77|0.00|2628.96|3504.00|3924.48|184.02|0.00|2628.96|2812.98|-875.04| +2452128|64291|12261|91790|511004|6665|36679|1|63|1221|70|18.70|27.11|16.53|0.00|1157.10|1309.00|1897.70|11.57|0.00|1157.10|1168.67|-151.90| +2452128|64291|1667|91790|511004|6665|36679|1|268|1221|66|19.32|29.75|21.71|0.00|1432.86|1275.12|1963.50|85.97|0.00|1432.86|1518.83|157.74| +2452128|64291|13443|91790|511004|6665|36679|1|11|1221|36|17.61|27.82|7.23|0.00|260.28|633.96|1001.52|13.01|0.00|260.28|273.29|-373.68| +2452128|64291|821|91790|511004|6665|36679|1|52|1221|21|44.49|68.06|15.65|0.00|328.65|934.29|1429.26|3.28|0.00|328.65|331.93|-605.64| +2452128|64291|6583|91790|511004|6665|36679|1|148|1221|14|17.99|19.24|1.73|0.00|24.22|251.86|269.36|1.69|0.00|24.22|25.91|-227.64| +2452193|50012|2697|57755|20609|4263|4717|8|13|1222|86|52.70|94.33|10.37|0.00|891.82|4532.20|8112.38|53.50|0.00|891.82|945.32|-3640.38| +2452193|50012|269|57755|20609|4263|4717|8|73|1222|72|35.96|43.15|43.15|2951.46|3106.80|2589.12|3106.80|3.10|2951.46|155.34|158.44|-2433.78| +2452193|50012|6577|57755|20609|4263|4717|8|190|1222|73|62.77|74.69|65.72|0.00|4797.56|4582.21|5452.37|0.00|0.00|4797.56|4797.56|215.35| +2452193||13555||20609|||8|219|1222||75.89|113.83|69.43||||9675.55|||||-5329.35| +2452193|50012|12757|57755|20609|4263|4717|8|237|1222|43|76.39|124.51|57.27|0.00|2462.61|3284.77|5353.93|172.38|0.00|2462.61|2634.99|-822.16| +2452193|50012|2327|57755|20609|4263|||62|1222||||31.56|0.00|||7196.25|89.94|0.00|1798.92|1888.86|| +2452193|50012|12605|57755|20609|4263|4717|8|183|1222|45|34.36|53.25|17.04|0.00|766.80|1546.20|2396.25|0.00|0.00|766.80|766.80|-779.40| +2452193|50012|7405|57755|20609|4263|4717|8|204|1222|43|37.38|71.76|12.91|0.00|555.13|1607.34|3085.68|22.20|0.00|555.13|577.33|-1052.21| +2451834|38548|7501|77248|1381879|5263|694|10|277|1223|18|28.40|46.29|17.12|224.95|308.16|511.20|833.22|3.32|224.95|83.21|86.53|-427.99| +2451834|38548|13825|77248|1381879|5263|694|10|284|1223|62|12.60|25.07|5.51|0.00|341.62|781.20|1554.34|17.08|0.00|341.62|358.70|-439.58| +2451834|38548|6469|77248|1381879|5263|694|10|21|1223|31|96.77|133.54|48.07|0.00|1490.17|2999.87|4139.74|14.90|0.00|1490.17|1505.07|-1509.70| +2451834|38548|11603|77248|1381879|5263|694|10|167|1223|31|46.83|81.01|26.73|0.00|828.63|1451.73|2511.31|66.29|0.00|828.63|894.92|-623.10| +2451834|38548|2912|77248|1381879|5263|694|10|250|1223|38|57.15|106.29|61.64|0.00|2342.32|2171.70|4039.02|23.42|0.00|2342.32|2365.74|170.62| +2451834|38548|11966|77248|1381879|5263|694|10|263|1223|53|36.13|43.35|0.00|0.00|0.00|1914.89|2297.55|0.00|0.00|0.00|0.00|-1914.89| +2451834|38548|10417|77248|1381879|5263|694|10|155|1223|44|67.72|96.16|54.81|0.00|2411.64|2979.68|4231.04|192.93|0.00|2411.64|2604.57|-568.04| +2451834|38548|2822|77248|1381879|5263|694|10|90|1223|89|63.91|63.91|52.40|0.00|4663.60|5687.99|5687.99|419.72|0.00|4663.60|5083.32|-1024.39| +2451834|38548|6083|77248|1381879|5263|694|10|8|1223|97|91.01|138.33|13.83|0.00|1341.51|8827.97|13418.01|120.73|0.00|1341.51|1462.24|-7486.46| +2451834|38548|3751|77248|1381879|5263|694|10|141|1223|9|38.98|49.50|21.28|0.00|191.52|350.82|445.50|0.00|0.00|191.52|191.52|-159.30| +2451834|38548|16511|77248|1381879|5263|694|10|239|1223|24|31.70|62.44|32.46|0.00|779.04|760.80|1498.56|31.16|0.00|779.04|810.20|18.24| +2451834|38548|12647|77248|1381879|5263|694|10|288|1223|71|21.14|21.77|15.89|0.00|1128.19|1500.94|1545.67|33.84|0.00|1128.19|1162.03|-372.75| +2451834||10885||1381879||694|||1223|28|64.93|83.11||0.00|535.08|1818.04|||0.00|||-1282.96| +2451351|29045|1705|37680|77518|3756|26794|7|43|1224|18|29.36|40.51|33.21|0.00|597.78|528.48|729.18|53.80|0.00|597.78|651.58|69.30| +2451351|29045|11362|37680|77518|3756|26794|7|138|1224|95|12.02|19.35|14.51|0.00|1378.45|1141.90|1838.25|124.06|0.00|1378.45|1502.51|236.55| +2451351|29045|7078|37680|77518|3756|26794|7|288|1224|80|9.47|16.38|4.25|0.00|340.00|757.60|1310.40|23.80|0.00|340.00|363.80|-417.60| +2451351|29045|10604|37680|77518|3756|26794|7|239|1224|19|20.07|40.14|30.10|0.00|571.90|381.33|762.66|22.87|0.00|571.90|594.77|190.57| +2451351|29045|10213|37680|77518|3756|26794|7|128|1224|36|8.84|14.40|0.14|1.15|5.04|318.24|518.40|0.00|1.15|3.89|3.89|-314.35| +2451351|29045|2677|37680|77518|3756|26794|7|75|1224|81|30.53|52.20|46.45|0.00|3762.45|2472.93|4228.20|300.99|0.00|3762.45|4063.44|1289.52| +2451351|29045|16808|37680|77518|3756|26794|7|23|1224|13|24.56|39.78|7.95|0.00|103.35|319.28|517.14|6.20|0.00|103.35|109.55|-215.93| +2451351|29045|7798|37680|77518|3756|26794|7|291|1224|25|56.95|72.89|16.76|0.00|419.00|1423.75|1822.25|12.57|0.00|419.00|431.57|-1004.75| +2451351|29045|15022|37680|77518|3756|26794|7|162|1224|95|27.09|32.77|18.02|0.00|1711.90|2573.55|3113.15|0.00|0.00|1711.90|1711.90|-861.65| +2451351|29045|7202|37680|77518|3756|26794|7|179|1224|17|77.22|91.89|79.02|107.46|1343.34|1312.74|1562.13|61.79|107.46|1235.88|1297.67|-76.86| +2451351|29045|5122|37680|77518|3756|26794|7|92|1224|77|11.18|20.12|17.30|0.00|1332.10|860.86|1549.24|66.60|0.00|1332.10|1398.70|471.24| +2451351|29045|3787|37680|77518|3756|26794|7|144|1224|36|25.89|47.37|43.10|0.00|1551.60|932.04|1705.32|62.06|0.00|1551.60|1613.66|619.56| +|39803|17557|92787|444531|6322|9713|||1225|||||||918.72|1561.76|||140.48|151.71|-778.24| +2452236|39803|14719|92787|444531|6322|9713|7|131|1225|13|5.03|7.59|4.62|0.00|60.06|65.39|98.67|1.80|0.00|60.06|61.86|-5.33| +2452236|39803|8495|92787|444531|6322|9713|7|70|1225|79|67.87|113.34|60.07|0.00|4745.53|5361.73|8953.86|427.09|0.00|4745.53|5172.62|-616.20| +2452236|39803|2637|92787|444531|6322|9713|7|271|1225|24|20.14|29.80|12.21|67.39|293.04|483.36|715.20|9.02|67.39|225.65|234.67|-257.71| +|39803|11159|92787|444531|6322|9713|||1225|||180.00|162.00|0.00|1458.00|826.56|1620.00||0.00||1574.64|| +|39803|17909|92787|444531||9713|7|285|1225|||54.94|34.06||34.06|||||34.06|34.40|-13.31| +2452236|39803|15413|92787|444531|6322|9713|7|180|1225|27|48.56|53.41|26.17|0.00|706.59|1311.12|1442.07|42.39|0.00|706.59|748.98|-604.53| +2452236|39803|5413|92787|444531|6322|9713|7|241|1225|40|5.89|6.12|5.14|0.00|205.60|235.60|244.80|16.44|0.00|205.60|222.04|-30.00| +2451048|63098|1918|26134|1481196|5058|14167|4|20|1226|67|25.95|49.82|40.85|0.00|2736.95|1738.65|3337.94|82.10|0.00|2736.95|2819.05|998.30| +2451048|63098|1538|26134|1481196|5058|14167|4|134|1226|82|55.94|56.49|33.89|0.00|2778.98|4587.08|4632.18|222.31|0.00|2778.98|3001.29|-1808.10| +2451048|63098|58|26134|1481196|5058|14167|4|12|1226|100|76.12|102.76|8.22|0.00|822.00|7612.00|10276.00|65.76|0.00|822.00|887.76|-6790.00| +2451048|63098|12524|26134|1481196|5058|14167|4|202|1226|62|97.09|98.06|97.07|0.00|6018.34|6019.58|6079.72|240.73|0.00|6018.34|6259.07|-1.24| +2451048|63098|296|26134|1481196|5058|14167|4|273|1226|19|70.23|125.71|96.79|404.58|1839.01|1334.37|2388.49|28.68|404.58|1434.43|1463.11|100.06| +2451048|63098|7174|26134|1481196|5058|14167|4|263|1226|20|90.37|178.02|160.21|0.00|3204.20|1807.40|3560.40|288.37|0.00|3204.20|3492.57|1396.80| +2451048|63098|17714|26134|1481196|5058|14167|4|136|1226|71|15.91|25.93|1.55|0.00|110.05|1129.61|1841.03|8.80|0.00|110.05|118.85|-1019.56| +2451048|63098|12338|26134|1481196|5058|14167|4|175|1226|99|14.46|20.67|14.05|0.00|1390.95|1431.54|2046.33|69.54|0.00|1390.95|1460.49|-40.59| +2451048|63098|10306|26134|1481196|5058|14167|4|206|1226|6|22.05|25.13|5.27|7.27|31.62|132.30|150.78|0.00|7.27|24.35|24.35|-107.95| +2451048|63098|4030|26134|1481196|5058|14167|4|271|1226|40|64.99|69.53|44.49|0.00|1779.60|2599.60|2781.20|124.57|0.00|1779.60|1904.17|-820.00| +2451048|63098|17462|26134|1481196|5058|14167|4|233|1226|65|95.88|138.06|84.21|0.00|5473.65|6232.20|8973.90|383.15|0.00|5473.65|5856.80|-758.55| +2451048|63098|16244|26134|1481196|5058|14167|4|208|1226|68|41.99|74.74|10.46|0.00|711.28|2855.32|5082.32|64.01|0.00|711.28|775.29|-2144.04| +2451173|36881|2692|9388|652895|3249|7556|1|272|1227|27|78.43|141.17|26.82|188.27|724.14|2117.61|3811.59|10.71|188.27|535.87|546.58|-1581.74| +2451173|36881|8329|9388|652895|3249|7556|1|268|1227|12|18.69|33.08|32.08|0.00|384.96|224.28|396.96|0.00|0.00|384.96|384.96|160.68| +2451173|36881|7438|9388|652895|3249|7556|1|162|1227|83|71.19|109.63|84.41|0.00|7006.03|5908.77|9099.29|140.12|0.00|7006.03|7146.15|1097.26| +2451173|36881|13886|9388|652895|3249|7556|1|267|1227|46|52.88|80.90|50.15|669.00|2306.90|2432.48|3721.40|114.65|669.00|1637.90|1752.55|-794.58| +2451173|36881|4699|9388|652895|3249|7556|1|7|1227|99|58.49|97.67|78.13|0.00|7734.87|5790.51|9669.33|464.09|0.00|7734.87|8198.96|1944.36| +2451173|36881|5204|9388|652895|3249|7556|1|212|1227|12|2.07|2.27|1.06|10.43|12.72|24.84|27.24|0.09|10.43|2.29|2.38|-22.55| +2451173|36881|12649|9388|652895|3249|7556|1|240|1227|84|61.81|88.38|1.76|0.00|147.84|5192.04|7423.92|10.34|0.00|147.84|158.18|-5044.20| +2451173||508||652895|||1||1227|13||78.19|22.67|0.00||726.05||11.78|0.00||306.49|-431.34| +2451173|36881|5416|9388|652895|3249|7556|1|165|1227|11|11.89|12.48|7.23|0.00|79.53|130.79|137.28|3.97|0.00|79.53|83.50|-51.26| +2451173|36881|9259|9388|652895|3249|7556|1|104|1227|64|5.82|9.48|7.86|0.00|503.04|372.48|606.72|20.12|0.00|503.04|523.16|130.56| +2451173|36881|7534|9388|652895|3249|7556|1|67|1227|39|82.91|149.23|22.38|0.00|872.82|3233.49|5819.97|69.82|0.00|872.82|942.64|-2360.67| +2451173|36881|2773|9388|652895|3249|7556|1|276|1227|50|44.92|89.39|75.08|0.00|3754.00|2246.00|4469.50|112.62|0.00|3754.00|3866.62|1508.00| +2451173||190|9388|||7556|1||1227||39.87|||0.00||398.70|470.40||0.00|||| +2451064||7202||1739563||12228||65|1228|62|96.93|153.14|133.23|0.00||6009.66||660.82|0.00|8260.26|8921.08|| +2451064|56479|5122|28207|1739563|519|12228|2|18|1228|1|18.60|21.76|4.56|0.00|4.56|18.60|21.76|0.22|0.00|4.56|4.78|-14.04| +2451064|56479|3787|28207|1739563|519|12228|2|277|1228|60|20.05|21.85|17.91|0.00|1074.60|1203.00|1311.00|64.47|0.00|1074.60|1139.07|-128.40| +2451064|56479|3547|28207|1739563|519|12228|2|247|1228|85|90.45|93.16|88.50|7071.15|7522.50|7688.25|7918.60|18.05|7071.15|451.35|469.40|-7236.90| +2451064|56479|8623|28207|1739563|519|12228|2|190|1228|23|32.53|38.38|17.27|0.00|397.21|748.19|882.74|15.88|0.00|397.21|413.09|-350.98| +2451064|56479|17962|28207|1739563|519|12228|2|192|1228|52|1.25|2.37|1.91|0.00|99.32|65.00|123.24|8.93|0.00|99.32|108.25|34.32| +2451064|56479|15970|28207|1739563|519|12228|2|288|1228|47|91.21|124.95|71.22|0.00|3347.34|4286.87|5872.65|200.84|0.00|3347.34|3548.18|-939.53| +||6409|28207|1739563|519||2|241|1228|||34.88||0.00||2351.44|||0.00|1935.57|1954.92|-415.87| +2451064|56479|13660|28207|1739563|519|12228|2|127|1228|1|90.20|92.90|18.58|0.00|18.58|90.20|92.90|0.92|0.00|18.58|19.50|-71.62| +||10762|||||||1228|29||84.43||0.00|1248.45||2448.47|74.90|0.00|1248.45|1323.35|-183.57| +2451064|56479|10196|28207|1739563|519|12228|2|189|1228|100|56.07|89.71|7.17|0.00|717.00|5607.00|8971.00|64.53|0.00|717.00|781.53|-4890.00| +2451064|56479|7300|28207|1739563|519|12228|2|91|1228|10|76.38|130.60|95.33|0.00|953.30|763.80|1306.00|9.53|0.00|953.30|962.83|189.50| +2451064|56479|15478|28207|1739563|519|12228|2|94|1228|96|84.52|165.65|137.48|0.00|13198.08|8113.92|15902.40|395.94|0.00|13198.08|13594.02|5084.16| +2451064|56479|9649|28207|1739563|519|12228|2|105|1228|74|47.98|94.52|92.62|0.00|6853.88|3550.52|6994.48|548.31|0.00|6853.88|7402.19|3303.36| +2451064|56479|12964|28207|1739563|519|12228|2|263|1228|48|41.15|62.54|6.25|36.00|300.00|1975.20|3001.92|21.12|36.00|264.00|285.12|-1711.20| +2452537|54212|1863|47632|1880370|2708|20246|7|151|1229|75|94.85|142.27|51.21|0.00|3840.75|7113.75|10670.25|192.03|0.00|3840.75|4032.78|-3273.00| +2452537|54212|9883|47632|1880370|2708|20246|7|198|1229|55|35.16|68.91|59.26|0.00|3259.30|1933.80|3790.05|32.59|0.00|3259.30|3291.89|1325.50| +2452537|54212|10531|47632|1880370|2708|20246|7|113|1229|94|58.61|93.18|30.74|1560.36|2889.56|5509.34|8758.92|119.62|1560.36|1329.20|1448.82|-4180.14| +2452537|54212|3151|47632|1880370|2708|20246|7|101|1229|98|18.45|19.92|6.77|0.00|663.46|1808.10|1952.16|13.26|0.00|663.46|676.72|-1144.64| +2452537|54212|5103|47632|1880370|2708|20246|7|84|1229|14|42.00|68.88|13.08|0.00|183.12|588.00|964.32|16.48|0.00|183.12|199.60|-404.88| +2452537|54212|13404|47632|1880370|2708|20246|7|173|1229|97|98.34|146.52|0.00|0.00|0.00|9538.98|14212.44|0.00|0.00|0.00|0.00|-9538.98| +2452537|54212|15774|47632|1880370|2708|20246|7|87|1229|50|62.23|111.39|90.22|0.00|4511.00|3111.50|5569.50|0.00|0.00|4511.00|4511.00|1399.50| +||2697|47632|1880370|||7||1229|4|9.42|13.94||0.00|16.72||55.76|0.66|0.00|16.72|17.38|-20.96| +2452537|54212|270|47632|1880370|2708|20246|7|111|1229|23|56.48|75.68|31.78|0.00|730.94|1299.04|1740.64|0.00|0.00|730.94|730.94|-568.10| +2452537|54212|6577|47632|1880370|2708|20246|7|177|1229|32|68.83|78.46|70.61|338.92|2259.52|2202.56|2510.72|38.41|338.92|1920.60|1959.01|-281.96| +2452537|54212|13555|47632|1880370|2708|20246|7|245|1229|71|19.19|36.26|27.55|0.00|1956.05|1362.49|2574.46|136.92|0.00|1956.05|2092.97|593.56| +2452537|54212|12757|47632|1880370|2708|20246|7|210|1229|61|13.02|25.38|1.01|0.00|61.61|794.22|1548.18|4.31|0.00|61.61|65.92|-732.61| +2451814|49534|5312|90978|360797|2263|6482|1|18|1230|1|41.90|54.05|44.32|0.00|44.32|41.90|54.05|0.88|0.00|44.32|45.20|2.42| +2451814|49534|1451|90978|360797|2263|6482|1|204|1230|62|35.59|59.43|18.42|0.00|1142.04|2206.58|3684.66|22.84|0.00|1142.04|1164.88|-1064.54| +||5483|90978|360797|2263|6482|||1230|||75.23||0.00|126.00||4212.88||0.00|||| +2451814|49534|8714|90978|360797|2263|6482|1|20|1230|80|67.16|97.38|68.16|4471.29|5452.80|5372.80|7790.40|19.63|4471.29|981.51|1001.14|-4391.29| +2451814|49534|7207|90978|360797|2263|6482|1|107|1230|37|69.76|126.96|105.37|0.00|3898.69|2581.12|4697.52|155.94|0.00|3898.69|4054.63|1317.57| +2451814|49534|17036|90978|360797|2263|6482|1|93|1230|2|88.37|156.41|43.79|0.00|87.58|176.74|312.82|7.88|0.00|87.58|95.46|-89.16| +2451814|49534|14171|90978|360797|2263|6482|1|98|1230|82|36.82|51.54|31.95|0.00|2619.90|3019.24|4226.28|235.79|0.00|2619.90|2855.69|-399.34| +2451814|49534|5132|90978|360797|2263|6482|1|63|1230|41|78.05|155.31|105.61|0.00|4330.01|3200.05|6367.71|259.80|0.00|4330.01|4589.81|1129.96| +2451814|49534|15457|90978|360797|2263|6482|1|193|1230|57|67.38|90.96|52.75|0.00|3006.75|3840.66|5184.72|270.60|0.00|3006.75|3277.35|-833.91| +2451835|47521|902|74536|257716|6143|14970|7|13|1231|22|20.34|20.95|18.85|0.00|414.70|447.48|460.90|20.73|0.00|414.70|435.43|-32.78| +2451835|47521|15221|74536|257716|6143|14970|7|7|1231|48|36.64|65.21|24.12|0.00|1157.76|1758.72|3130.08|104.19|0.00|1157.76|1261.95|-600.96| +2451835|47521|11498|74536|257716|6143|14970|7|153|1231|63|36.40|41.86|20.09|0.00|1265.67|2293.20|2637.18|12.65|0.00|1265.67|1278.32|-1027.53| +2451835|47521|15140|74536|257716|6143|14970|7|102|1231|9|56.84|105.15|91.48|0.00|823.32|511.56|946.35|8.23|0.00|823.32|831.55|311.76| +2451835|47521|3272|74536|257716|6143|14970|7|20|1231|2|59.05|114.55|72.16|0.00|144.32|118.10|229.10|10.10|0.00|144.32|154.42|26.22| +2451835|47521|4757|74536|257716|6143|14970|7|161|1231|6|39.14|49.70|10.43|0.00|62.58|234.84|298.20|3.75|0.00|62.58|66.33|-172.26| +2451835|47521|12800|||6143|14970|||1231|||85.77|60.89|0.00||5062.32||306.88|0.00|4384.08|4690.96|| +2451835|47521|536|74536|257716|6143|14970|7|251|1231|74|60.66|94.02|0.94|30.60|69.56|4488.84|6957.48|0.77|30.60|38.96|39.73|-4449.88| +2451835|47521|14930|74536|257716|6143|14970|7|21|1231|89|8.60|11.61|9.52|0.00|847.28|765.40|1033.29|76.25|0.00|847.28|923.53|81.88| +2451835|47521|3007|74536|257716|6143|14970|7|194|1231|14|67.82|131.57|119.72|0.00|1676.08|949.48|1841.98|16.76|0.00|1676.08|1692.84|726.60| +2451835|47521|4921|74536|257716|6143|14970|7|15|1231|71|14.82|18.82|4.14|0.00|293.94|1052.22|1336.22|14.69|0.00|293.94|308.63|-758.28| +2451835|47521|9713|74536|257716|6143|14970|7|46|1231|51|42.86|42.86|36.43|0.00|1857.93|2185.86|2185.86|74.31|0.00|1857.93|1932.24|-327.93| +2451835|47521|5402|74536|257716|6143|14970|7|282|1231|96|75.09|134.41|63.17|5821.74|6064.32|7208.64|12903.36|9.70|5821.74|242.58|252.28|-6966.06| +2451835|47521|15401|74536|257716|6143|14970|7|173|1231|60|52.42|91.73|35.77|0.00|2146.20|3145.20|5503.80|64.38|0.00|2146.20|2210.58|-999.00| +2452228|49078|16377|47832|1494842|1052|48062|8|233|1232|79|54.14|108.28|71.46|0.00|5645.34|4277.06|8554.12|395.17|0.00|5645.34|6040.51|1368.28| +2452228|49078|10401|47832|1494842|1052|48062|8|176|1232|75|83.77|116.44|1.16|0.00|87.00|6282.75|8733.00|6.09|0.00|87.00|93.09|-6195.75| +2452228|49078|15335|47832|1494842|1052|48062|8|262|1232|22|75.84|136.51|103.74|0.00|2282.28|1668.48|3003.22|68.46|0.00|2282.28|2350.74|613.80| +2452228|49078|12665|47832|1494842|1052|48062|8|285|1232|72|46.75|48.62|0.97|0.00|69.84|3366.00|3500.64|2.79|0.00|69.84|72.63|-3296.16| +2452228|49078|15769|47832|1494842|1052|48062|8|65|1232|62|50.04|84.56|7.61|0.00|471.82|3102.48|5242.72|4.71|0.00|471.82|476.53|-2630.66| +2452228|49078|10771|47832|1494842|1052|48062|8|130|1232|82|86.45|127.08|29.22|0.00|2396.04|7088.90|10420.56|191.68|0.00|2396.04|2587.72|-4692.86| +2452228|49078|14791|47832|1494842|1052|48062|8|221|1232|85|74.33|112.23|3.36|237.04|285.60|6318.05|9539.55|1.45|237.04|48.56|50.01|-6269.49| +2452228|49078|7917|47832|1494842|1052|48062|8|184|1232|30|55.06|56.71|36.29|0.00|1088.70|1651.80|1701.30|21.77|0.00|1088.70|1110.47|-563.10| +2452228|49078|11087|47832|1494842|1052|48062|8|259|1232|90|40.18|57.45|30.44|0.00|2739.60|3616.20|5170.50|109.58|0.00|2739.60|2849.18|-876.60| +2452228|49078|1123|47832|1494842|1052|48062|8|297|1232|15|83.27|111.58|63.60|0.00|954.00|1249.05|1673.70|38.16|0.00|954.00|992.16|-295.05| +2452269|38353|4497|41798|1309160|2569|35578|10|222|1233|65|61.22|110.80|37.67|0.00|2448.55|3979.30|7202.00|24.48|0.00|2448.55|2473.03|-1530.75| +2452269|38353|9331|41798|1309160|2569|35578|10|89|1233|38|49.80|79.18|19.00|0.00|722.00|1892.40|3008.84|21.66|0.00|722.00|743.66|-1170.40| +2452269|38353|9231|41798|1309160|2569|35578|10|118|1233|20|91.98|175.68|128.24|0.00|2564.80|1839.60|3513.60|51.29|0.00|2564.80|2616.09|725.20| +2452269|38353|11541|41798|1309160|2569|35578|10|196|1233|88|15.67|25.22|20.42|0.00|1796.96|1378.96|2219.36|161.72|0.00|1796.96|1958.68|418.00| +2452269|38353|7953|41798|1309160|2569|35578|10|167|1233|69|22.80|28.95|11.29|0.00|779.01|1573.20|1997.55|15.58|0.00|779.01|794.59|-794.19| +2452269|38353|15853|41798|1309160|2569|35578|10|158|1233|35|14.72|28.26|15.26|0.00|534.10|515.20|989.10|48.06|0.00|534.10|582.16|18.90| +2452269|38353|10819|41798|1309160|2569|35578|10|196|1233|42|98.39|127.90|2.55|51.40|107.10|4132.38|5371.80|0.55|51.40|55.70|56.25|-4076.68| +2452269|38353|12955|41798|1309160|2569|35578|10|41|1233|85|47.14|49.96|27.97|0.00|2377.45|4006.90|4246.60|142.64|0.00|2377.45|2520.09|-1629.45| +2452269|38353|7213|41798|1309160|2569|35578|10|28|1233|32|10.36|19.78|6.72|0.00|215.04|331.52|632.96|15.05|0.00|215.04|230.09|-116.48| +2452269|38353|8805|41798|1309160|2569|35578|10|242|1233|92|66.28|103.39|66.16|6086.72|6086.72|6097.76|9511.88|0.00|6086.72|0.00|0.00|-6097.76| +2452269|38353|15951|41798|1309160|2569|35578|10|161|1233|60|49.79|64.72|58.24|0.00|3494.40|2987.40|3883.20|69.88|0.00|3494.40|3564.28|507.00| +2452269|38353|11567|41798|1309160|2569|35578|10|253|1233|47|39.01|45.25|28.05|514.15|1318.35|1833.47|2126.75|48.25|514.15|804.20|852.45|-1029.27| +2452269|38353|5067|41798|1309160|2569|35578|10|39|1233|100|94.58|120.11|50.44|0.00|5044.00|9458.00|12011.00|100.88|0.00|5044.00|5144.88|-4414.00| +2451634|40695|13340|29545|763588|2431|23627|10|110|1234|49|20.38|37.09|17.06|0.00|835.94|998.62|1817.41|25.07|0.00|835.94|861.01|-162.68| +2451634|40695|2930|29545|763588|2431|23627|10|137|1234|77|86.65|116.11|88.24|0.00|6794.48|6672.05|8940.47|271.77|0.00|6794.48|7066.25|122.43| +2451634|40695|15782|29545|763588|2431|23627|10|108|1234|50|98.33|189.77|187.87|0.00|9393.50|4916.50|9488.50|0.00|0.00|9393.50|9393.50|4477.00| +2451634|40695|20|29545|763588|2431|23627|10|239|1234|35|13.11|17.30|10.38|0.00|363.30|458.85|605.50|25.43|0.00|363.30|388.73|-95.55| +2451634|40695|15626|29545|763588|2431|23627|10|288|1234|5|5.74|9.12|0.18|0.00|0.90|28.70|45.60|0.05|0.00|0.90|0.95|-27.80| +2451634|40695|9458|29545|763588|2431|23627|10|267|1234|51|43.80|59.13|1.18|0.00|60.18|2233.80|3015.63|5.41|0.00|60.18|65.59|-2173.62| +2451634|40695|10795|29545|763588|2431|23627|10|239|1234|18|97.82|131.07|111.40|842.18|2005.20|1760.76|2359.26|104.67|842.18|1163.02|1267.69|-597.74| +2451634|40695|13693|29545|763588|2431|23627|10|117|1234|31|91.30|100.43|63.27|0.00|1961.37|2830.30|3113.33|156.90|0.00|1961.37|2118.27|-868.93| +2451634|40695|12920|29545|763588|2431|23627|10|45|1234|85|80.00|134.40|127.68|0.00|10852.80|6800.00|11424.00|759.69|0.00|10852.80|11612.49|4052.80| +2451634|40695|4177|29545|763588|2431|23627|10|6|1234|14|30.96|34.67|10.05|0.00|140.70|433.44|485.38|4.22|0.00|140.70|144.92|-292.74| +||2360||763588|2431||10||1234|78|9.17||4.80|67.39|374.40||815.10||67.39|||-408.25| +2451634|40695|4322|29545|763588|2431|23627|10|266|1234|92|75.40|97.26|72.94|0.00|6710.48|6936.80|8947.92|402.62|0.00|6710.48|7113.10|-226.32| +2451897|71578|3551|34990|3980|4170|13207|8|223|1235|45|18.66|19.77|12.85|0.00|578.25|839.70|889.65|28.91|0.00|578.25|607.16|-261.45| +2451897|71578|301|34990|3980|4170|13207|8|272|1235|55|86.84|107.68|103.37|0.00|5685.35|4776.20|5922.40|170.56|0.00|5685.35|5855.91|909.15| +2451897|71578|9361|34990|3980|4170|13207|8|139|1235|77|29.00|30.16|18.69|0.00|1439.13|2233.00|2322.32|115.13|0.00|1439.13|1554.26|-793.87| +2451897|71578|12092|34990|3980|4170|13207|8|52|1235|62|92.26|113.47|41.98|0.00|2602.76|5720.12|7035.14|0.00|0.00|2602.76|2602.76|-3117.36| +2451897|71578|17083|34990|3980|4170|13207|8|247|1235|40|1.41|2.42|1.01|0.00|40.40|56.40|96.80|3.63|0.00|40.40|44.03|-16.00| +2451897|71578|13931|34990|3980|4170|13207|8|75|1235|60|30.57|56.55|41.84|0.00|2510.40|1834.20|3393.00|50.20|0.00|2510.40|2560.60|676.20| +2451897|71578|4217|34990|3980|4170|13207|8|162|1235|71|24.39|44.14|37.96|0.00|2695.16|1731.69|3133.94|0.00|0.00|2695.16|2695.16|963.47| +2451897|71578|15845|34990|3980|4170|13207|8|248|1235|79|22.18|28.61|8.58|0.00|677.82|1752.22|2260.19|47.44|0.00|677.82|725.26|-1074.40| +2451897|71578|17633|34990|3980|4170|13207|8|155|1235|62|80.61|120.10|44.43|2479.19|2754.66|4997.82|7446.20|0.00|2479.19|275.47|275.47|-4722.35| +2451897|71578|12169|34990|3980|4170|13207|8|49|1235|16|41.34|66.97|4.68|0.00|74.88|661.44|1071.52|5.99|0.00|74.88|80.87|-586.56| +2452558|42043|13188|42335|1157307|726|8898|8|109|1236|61|82.86|125.11|15.01|0.00|915.61|5054.46|7631.71|64.09|0.00|915.61|979.70|-4138.85| +2452558|42043|8187|42335|1157307|726|8898|8|221|1236|95|3.45|6.72|3.96|0.00|376.20|327.75|638.40|33.85|0.00|376.20|410.05|48.45| +2452558|42043|11007|42335|1157307|726|8898|8|272|1236|46|35.02|35.02|12.95|0.00|595.70|1610.92|1610.92|35.74|0.00|595.70|631.44|-1015.22| +2452558|42043|6591|42335|1157307|726|8898|8|267|1236|84|25.25|41.66|37.07|1806.05|3113.88|2121.00|3499.44|13.07|1806.05|1307.83|1320.90|-813.17| +2452558|42043|5610|42335|1157307|726|8898|8|109|1236|7|73.05|86.19|62.91|0.00|440.37|511.35|603.33|26.42|0.00|440.37|466.79|-70.98| +2452558|42043|11460|42335|1157307|726|8898|8|69|1236|97|39.15|64.98|44.83|0.00|4348.51|3797.55|6303.06|217.42|0.00|4348.51|4565.93|550.96| +||5451|42335||726|8898|8||1236|80||76.67|1.53||122.40||6133.60|||122.40|123.62|| +2452558|42043|5454|42335|1157307|726|8898|8|115|1236|71|41.60|69.47|57.66|0.00|4093.86|2953.60|4932.37|163.75|0.00|4093.86|4257.61|1140.26| +2452558|42043|9672|42335|1157307|726|8898|8|188|1236|98|29.57|30.75|3.69|0.00|361.62|2897.86|3013.50|3.61|0.00|361.62|365.23|-2536.24| +2452558|42043|16914|42335|1157307|726|8898|8|287|1236|14|69.93|87.41|72.55|0.00|1015.70|979.02|1223.74|91.41|0.00|1015.70|1107.11|36.68| +2452558||9690|||726||8||1236|21|19.37|30.60||||||||||-374.64| +2452167|33262|6639|89465|1902193|2921|2449|10|26|1237|53|93.94|99.57|47.79|0.00|2532.87|4978.82|5277.21|25.32|0.00|2532.87|2558.19|-2445.95| +2452167|33262|6841|89465|1902193|2921|2449|10|94|1237|14|4.64|6.35|4.00|0.00|56.00|64.96|88.90|1.12|0.00|56.00|57.12|-8.96| +2452167|33262|10501|89465|1902193|2921|2449|10|46|1237|84|33.13|61.29|27.58|0.00|2316.72|2782.92|5148.36|162.17|0.00|2316.72|2478.89|-466.20| +2452167|33262|8971|89465|1902193|2921|2449|10|6|1237|65|46.53|89.33|4.46|0.00|289.90|3024.45|5806.45|11.59|0.00|289.90|301.49|-2734.55| +2452167|33262|6651|89465|1902193|2921|2449|10|102|1237|72|76.04|114.82|42.48|0.00|3058.56|5474.88|8267.04|152.92|0.00|3058.56|3211.48|-2416.32| +2452167|33262|7633|89465|1902193|2921|2449|10|262|1237|97|51.68|101.80|90.60|0.00|8788.20|5012.96|9874.60|175.76|0.00|8788.20|8963.96|3775.24| +2452167|33262|8617|89465|1902193|2921|2449|10|65|1237|80|22.90|44.42|4.88|0.00|390.40|1832.00|3553.60|31.23|0.00|390.40|421.63|-1441.60| +2452167|33262|11347|89465|1902193|2921|2449|10|12|1237|7|45.14|65.90|44.81|0.00|313.67|315.98|461.30|15.68|0.00|313.67|329.35|-2.31| +2452167|33262|9035|89465|1902193|2921|2449|10|21|1237|93|99.01|196.03|90.17|0.00|8385.81|9207.93|18230.79|503.14|0.00|8385.81|8888.95|-822.12| +2452167|33262|15189|89465|1902193|2921|2449|10|184|1237|36|73.37|126.93|29.19|0.00|1050.84|2641.32|4569.48|0.00|0.00|1050.84|1050.84|-1590.48| +2452167|33262|4783|89465|1902193|2921|2449|10|82|1237|82|16.89|30.23|12.39|0.00|1015.98|1384.98|2478.86|40.63|0.00|1015.98|1056.61|-369.00| +||1275||1902193|||10||1237|64|65.26|95.27||0.00|2804.48||||0.00|2804.48||| +2452167|33262|13965|89465|1902193|2921|2449|10|36|1237|31|52.39|77.01|64.68|0.00|2005.08|1624.09|2387.31|140.35|0.00|2005.08|2145.43|380.99| +2451607|50559|8|50179|1273891|2050|22962|1|71|1238|85|52.97|97.99|13.71|0.00|1165.35|4502.45|8329.15|93.22|0.00|1165.35|1258.57|-3337.10| +2451607|50559|2030|50179|1273891|2050|22962|1|148|1238|3|28.85|44.14|4.41|0.00|13.23|86.55|132.42|0.13|0.00|13.23|13.36|-73.32| +2451607|50559|12212|50179||2050||1||1238||62.13|87.60|||1913.04|1491.12|2102.40|||1913.04|2008.69|| +2451607|50559|7645|50179|1273891|2050|22962|1|204|1238|70|53.81|103.31|103.31|0.00|7231.70|3766.70|7231.70|361.58|0.00|7231.70|7593.28|3465.00| +2451607|50559|6428|50179|1273891|2050|22962|1|246|1238|83|43.58|57.96|0.57|0.00|47.31|3617.14|4810.68|3.31|0.00|47.31|50.62|-3569.83| +2451607|50559|953|50179|1273891|2050|22962|1|114|1238|85|6.81|7.83|1.09|41.69|92.65|578.85|665.55|1.52|41.69|50.96|52.48|-527.89| +2451607|50559|10897|50179|1273891|2050|22962|1|128|1238|62|30.22|44.42|18.21|0.00|1129.02|1873.64|2754.04|11.29|0.00|1129.02|1140.31|-744.62| +2451607|50559|8594|50179|1273891|2050|22962|1|129|1238|96|86.27|169.08|158.93|9459.51|15257.28|8281.92|16231.68|405.84|9459.51|5797.77|6203.61|-2484.15| +2451607|50559|5096|50179|1273891|2050|22962|1|162|1238|72|39.43|58.35|36.17|2421.94|2604.24|2838.96|4201.20|9.11|2421.94|182.30|191.41|-2656.66| +2451607|50559|12593|50179|1273891|2050|22962|1|24|1238|56|5.28|6.60|4.29|0.00|240.24|295.68|369.60|14.41|0.00|240.24|254.65|-55.44| +2451607|50559|2600|50179|1273891|2050|22962|1|53|1238|9|22.86|45.72|31.54|0.00|283.86|205.74|411.48|19.87|0.00|283.86|303.73|78.12| +2451965|46855|7095|59612|280291|170|45202|8|82|1239|||80.38|20.09|||||||1406.30|1532.86|| +2451965|46855|7919|59612|280291|170|45202|8|18|1239|83|97.18|125.36|78.97|0.00|6554.51|8065.94|10404.88|524.36|0.00|6554.51|7078.87|-1511.43| +2451965|46855|11887|59612|280291|170|45202|8|173|1239|38|47.33|52.53|8.40|0.00|319.20|1798.54|1996.14|0.00|0.00|319.20|319.20|-1479.34| +2451965|46855|17213|59612|280291|170|45202|8|180|1239|92|43.42|56.44|13.54|0.00|1245.68|3994.64|5192.48|0.00|0.00|1245.68|1245.68|-2748.96| +2451965|46855|1267|59612|280291|170|45202|8|7|1239|13|26.90|37.92|21.23|0.00|275.99|349.70|492.96|22.07|0.00|275.99|298.06|-73.71| +2451965|46855|2397|59612|280291|170|45202|8|128|1239|4|35.04|38.54|27.74|0.00|110.96|140.16|154.16|1.10|0.00|110.96|112.06|-29.20| +2451965|46855|9425|59612|280291|170|45202|8|155|1239|54|18.82|30.67|27.60|0.00|1490.40|1016.28|1656.18|44.71|0.00|1490.40|1535.11|474.12| +2451965|46855|3765|59612|280291|170|45202|8|267|1239|25|19.83|23.39|1.63|0.00|40.75|495.75|584.75|1.22|0.00|40.75|41.97|-455.00| +2451965|46855|6977|59612|280291|170|45202|8|226|1239|78|43.30|46.33|33.35|0.00|2601.30|3377.40|3613.74|52.02|0.00|2601.30|2653.32|-776.10| +2451965|46855|11711|59612|280291|170|45202|8|227|1239|6|24.53|46.36|38.01|0.00|228.06|147.18|278.16|0.00|0.00|228.06|228.06|80.88| +2451965|46855|6013|59612|280291|170|45202|8|64|1239|21|21.16|23.69|21.55|0.00|452.55|444.36|497.49|18.10|0.00|452.55|470.65|8.19| +2451965|46855|897|59612|280291|170|45202|8|194|1239|39|24.05|42.80|32.52|0.00|1268.28|937.95|1669.20|50.73|0.00|1268.28|1319.01|330.33| +2451965|46855|10185|59612|280291|170|45202|8|252|1239|24|15.04|17.14|6.17|0.00|148.08|360.96|411.36|5.92|0.00|148.08|154.00|-212.88| +2451965|46855|10283|59612|280291|170|45202|8|3|1239|69|84.08|102.57|17.43|0.00|1202.67|5801.52|7077.33|72.16|0.00|1202.67|1274.83|-4598.85| +2451965||17965|||170|45202|||1239|92|6.35|7.11|0.63||57.96|||1.73|||59.69|| +2452593|43940|12660|79407|1785226|3045|12911|2|109|1240|19|6.59|7.57|0.68|0.00|12.92|125.21|143.83|0.00|0.00|12.92|12.92|-112.29| +2452593|43940|7837|79407|1785226|3045|12911|2|168|1240|6|44.06|78.42|40.77|234.83|244.62|264.36|470.52|0.58|234.83|9.79|10.37|-254.57| +2452593|43940|17881|79407|1785226|3045|12911|2|268|1240|87|96.10|127.81|90.74|7736.49|7894.38|8360.70|11119.47|12.63|7736.49|157.89|170.52|-8202.81| +2452593|43940|3303|79407|1785226|3045|12911|2|257|1240|84|71.40|121.38|8.49|0.00|713.16|5997.60|10195.92|57.05|0.00|713.16|770.21|-5284.44| +2452593|43940|17551|79407|1785226|3045|12911|2|91|1240|46|9.23|16.61|6.31|0.00|290.26|424.58|764.06|0.00|0.00|290.26|290.26|-134.32| +2452593|43940|13903|79407|1785226|3045|12911|2|294|1240|24|13.74|14.15|2.54|0.00|60.96|329.76|339.60|4.26|0.00|60.96|65.22|-268.80| +2452593|43940|12996|79407|1785226|3045|12911|2|133|1240|74|57.45|106.28|94.58|0.00|6998.92|4251.30|7864.72|559.91|0.00|6998.92|7558.83|2747.62| +2452593|43940|13485|79407|1785226|3045|12911|2|168|1240|87|84.90|122.25|103.91|8316.95|9040.17|7386.30|10635.75|21.69|8316.95|723.22|744.91|-6663.08| +2452593|43940|13329|79407|1785226|3045|12911|2|31|1240|63|65.25|67.20|30.91|0.00|1947.33|4110.75|4233.60|136.31|0.00|1947.33|2083.64|-2163.42| +2452593|43940|16155|79407|1785226|3045|12911|2|62|1240|84|25.72|50.66|38.50|2134.44|3234.00|2160.48|4255.44|0.00|2134.44|1099.56|1099.56|-1060.92| +2452593|43940|2934|79407|1785226|3045|12911|2|193|1240|11|93.88|187.76|99.51|0.00|1094.61|1032.68|2065.36|65.67|0.00|1094.61|1160.28|61.93| +2452579|43945|11809|25105|20177|5202|20627|8|274|1241|70|83.57|99.44|63.64|0.00|4454.80|5849.90|6960.80|178.19|0.00|4454.80|4632.99|-1395.10| +2452579|43945|2065|25105|20177|5202|20627|8|282|1241|75|25.78|44.59|31.21|1053.33|2340.75|1933.50|3344.25|38.62|1053.33|1287.42|1326.04|-646.08| +2452579|43945|8493|25105|20177|5202|20627|8|84|1241|81|21.91|30.23|26.90|0.00|2178.90|1774.71|2448.63|152.52|0.00|2178.90|2331.42|404.19| +2452579|43945|3663|25105|20177|5202|20627|8|288|1241|82|59.07|83.28|32.47|0.00|2662.54|4843.74|6828.96|79.87|0.00|2662.54|2742.41|-2181.20| +2452579|43945|1453|25105|20177|5202|20627|8|22|1241|69|52.85|56.54|34.48|0.00|2379.12|3646.65|3901.26|142.74|0.00|2379.12|2521.86|-1267.53| +2452579|43945|9613|25105|20177|5202|20627|8|191|1241|88|69.09|132.65|46.42|571.89|4084.96|6079.92|11673.20|245.91|571.89|3513.07|3758.98|-2566.85| +2452579|43945|5047|25105|20177|5202|20627|8|10|1241|57|67.57|72.29|49.88|0.00|2843.16|3851.49|4120.53|56.86|0.00|2843.16|2900.02|-1008.33| +2452579|43945|11880|25105|20177|5202|20627|8|5|1241|32|80.02|118.42|88.81|0.00|2841.92|2560.64|3789.44|0.00|0.00|2841.92|2841.92|281.28| +2452579|43945|17670|25105|20177|5202|20627|8|246|1241|56|50.02|86.03|15.48|667.49|866.88|2801.12|4817.68|5.98|667.49|199.39|205.37|-2601.73| +2452579|43945|5161|25105|20177|5202|20627|8|36|1241|96|6.55|10.34|0.93|0.00|89.28|628.80|992.64|0.00|0.00|89.28|89.28|-539.52| +2452579|43945|13453|25105|20177|5202|20627|8|79|1241|83|65.55|125.85|123.33|0.00|10236.39|5440.65|10445.55|102.36|0.00|10236.39|10338.75|4795.74| +2452579|43945|13647|25105|20177|5202|20627|8|132|1241|2|3.21|6.00|3.90|0.00|7.80|6.42|12.00|0.07|0.00|7.80|7.87|1.38| +2452579|43945|5391|25105|20177|5202|20627|8|219|1241|9|48.45|85.75|81.46|102.63|733.14|436.05|771.75|12.61|102.63|630.51|643.12|194.46| +2452579|43945|2743|25105|20177|5202|20627|8|290|1241|48|14.39|14.82|4.00|0.00|192.00|690.72|711.36|15.36|0.00|192.00|207.36|-498.72| +2452106|53912|17601|35129|1889146|3610|34427|4|222|1242|62|49.44|88.49|48.66|1810.15|3016.92|3065.28|5486.38|84.47|1810.15|1206.77|1291.24|-1858.51| +2452106|53912|12725|35129|1889146|3610|34427|4|175|1242|4|95.41|115.44|108.51|0.00|434.04|381.64|461.76|34.72|0.00|434.04|468.76|52.40| +2452106|53912|2565|35129|1889146|3610|34427|4|105|1242|79|24.81|49.62|15.38|0.00|1215.02|1959.99|3919.98|36.45|0.00|1215.02|1251.47|-744.97| +2452106|53912|6763||1889146|3610|34427|4||1242|31|87.42|145.11||||2710.02|4498.41|55.77||2788.76|2844.53|| +2452106|53912|2035|35129|1889146|3610|34427|4|42|1242|30|88.70|133.05|101.11|0.00|3033.30|2661.00|3991.50|242.66|0.00|3033.30|3275.96|372.30| +2452106|53912|1387|35129|1889146|3610|34427|4|44|1242|48|53.89|101.31|24.31|0.00|1166.88|2586.72|4862.88|70.01|0.00|1166.88|1236.89|-1419.84| +2452106|53912|1821|35129|1889146|3610|34427|4|27|1242|1|36.39|48.39|41.13|0.00|41.13|36.39|48.39|2.05|0.00|41.13|43.18|4.74| +2452106|53912|12527|35129|1889146|3610|34427|4|210|1242|25|34.52|35.55|23.10|571.72|577.50|863.00|888.75|0.17|571.72|5.78|5.95|-857.22| +2452106|53912|2315|35129|1889146|3610|34427|4|150|1242|69|99.51|142.29|38.41|0.00|2650.29|6866.19|9818.01|53.00|0.00|2650.29|2703.29|-4215.90| +2452106||10017||1889146|||4|173|1242|80|||19.27|0.00|||1557.60|123.32|0.00|1541.60|1664.92|| +2452106|53912|3347|35129|1889146|3610|34427|4|2|1242|9|6.05|11.19|5.48|0.00|49.32|54.45|100.71|0.98|0.00|49.32|50.30|-5.13| +2452106|53912|977|35129|1889146|3610|34427|4|78|1242|2|73.72|126.06|81.93|119.61|163.86|147.44|252.12|2.65|119.61|44.25|46.90|-103.19| +2451861|50830|800|53099|406246|6146|36893|2|140|1243|94|6.21|7.07|4.10|111.76|385.40|583.74|664.58|24.62|111.76|273.64|298.26|-310.10| +2451861|50830|11111|53099|406246|6146|36893|2|25|1243|20|89.32|92.89|37.15|0.00|743.00|1786.40|1857.80|59.44|0.00|743.00|802.44|-1043.40| +2451861|50830|8099|53099|406246|6146|36893|2|67|1243|18|80.29|126.85|110.35|0.00|1986.30|1445.22|2283.30|19.86|0.00|1986.30|2006.16|541.08| +2451861|50830|7015|53099|406246|6146|36893|2|181|1243|26|29.00|48.43|45.52|556.25|1183.52|754.00|1259.18|31.36|556.25|627.27|658.63|-126.73| +2451861|50830|10955|53099|406246|6146|36893|2|141|1243|20|62.14|114.33|42.30|0.00|846.00|1242.80|2286.60|33.84|0.00|846.00|879.84|-396.80| +2451861|50830|16139|53099|406246|6146|36893|2|170|1243|25|97.01|157.15|133.57|0.00|3339.25|2425.25|3928.75|66.78|0.00|3339.25|3406.03|914.00| +2451861|50830|11737|53099|406246|6146|36893|2|246|1243|70|91.99|133.38|89.36|938.28|6255.20|6439.30|9336.60|212.67|938.28|5316.92|5529.59|-1122.38| +2451861|50830|11543|53099|406246|6146|36893|2|242|1243|31|15.13|23.30|20.73|0.00|642.63|469.03|722.30|25.70|0.00|642.63|668.33|173.60| +2452141|72275|1753|43026|778852|5878|14104|1|51|1244|93|73.76|115.80|32.42|2773.85|3015.06|6859.68|10769.40|12.06|2773.85|241.21|253.27|-6618.47| +2452141|72275|1323|43026|778852|5878|14104|1|285|1244|26|78.19|139.96|109.16|0.00|2838.16|2032.94|3638.96|0.00|0.00|2838.16|2838.16|805.22| +2452141|72275|16471|43026|778852|5878|14104|1|121|1244|48|73.27|80.59|27.40|0.00|1315.20|3516.96|3868.32|13.15|0.00|1315.20|1328.35|-2201.76| +||1455||778852||14104|1|137|1244|75|||||541.50||3869.25|||541.50|563.16|| +2452141|72275|8309|43026|778852|5878|14104|1|136|1244|68|20.85|26.47|25.14|632.52|1709.52|1417.80|1799.96|75.39|632.52|1077.00|1152.39|-340.80| +2452141|72275|9917|43026|778852|5878|14104|1|125|1244|64|87.31|126.59|25.31|0.00|1619.84|5587.84|8101.76|113.38|0.00|1619.84|1733.22|-3968.00| +2452141|72275|15051|43026|778852|5878|14104|1|257|1244|55|17.20|22.36|4.47|0.00|245.85|946.00|1229.80|4.91|0.00|245.85|250.76|-700.15| +2452141|72275|3223|43026|778852|5878|14104|1|33|1244|78|95.45|144.12|0.00|0.00|0.00|7445.10|11241.36|0.00|0.00|0.00|0.00|-7445.10| +2452141|72275|2537|43026|778852|5878|14104|1|104|1244|35|63.11|92.14|66.34|0.00|2321.90|2208.85|3224.90|208.97|0.00|2321.90|2530.87|113.05| +2451822|54812|4387|48967|604266|1612|16111|7|4|1245|60|55.03|107.85|91.67|0.00|5500.20|3301.80|6471.00|220.00|0.00|5500.20|5720.20|2198.40| +2451822|54812|4849|48967|604266|1612|16111|7|168|1245|76|49.43|63.76|8.92|0.00|677.92|3756.68|4845.76|33.89|0.00|677.92|711.81|-3078.76| +2451822|54812|6925|48967|604266|1612|16111|7|151|1245|23|23.69|46.66|27.52|0.00|632.96|544.87|1073.18|12.65|0.00|632.96|645.61|88.09| +2451822|54812|2219|48967|604266|1612|16111|7|69|1245|89|25.00|39.00|5.07|67.68|451.23|2225.00|3471.00|3.83|67.68|383.55|387.38|-1841.45| +2451822|54812|1273|48967|604266|1612|16111|7|44|1245|53|91.41|110.60|47.55|957.65|2520.15|4844.73|5861.80|93.75|957.65|1562.50|1656.25|-3282.23| +2451822|54812|17063|48967|604266|1612|16111|7|180|1245|6|68.92|125.43|18.81|2.25|112.86|413.52|752.58|3.31|2.25|110.61|113.92|-302.91| +2451822|54812|3199|48967|604266|1612|16111|7|113|1245|42|33.87|60.96|49.37|0.00|2073.54|1422.54|2560.32|186.61|0.00|2073.54|2260.15|651.00| +2451822||7153|48967|604266|1612|16111|7|259|1245||49.53|69.83|18.15|0.00||1287.78|||0.00|||| +2451822|54812|14816|48967|604266|1612|16111|7|2|1245|41|14.46|25.88|11.64|81.13|477.24|592.86|1061.08|31.68|81.13|396.11|427.79|-196.75| +2451822|54812|17077|48967|604266|1612|16111|7|146|1245|30|68.46|96.52|67.56|0.00|2026.80|2053.80|2895.60|162.14|0.00|2026.80|2188.94|-27.00| +2451822|54812|12302|48967|604266|1612|16111|7|40|1245|17|41.46|45.19|3.16|0.00|53.72|704.82|768.23|2.68|0.00|53.72|56.40|-651.10| +2451822|54812|14414|48967|604266|1612|16111|7|16|1245|93|35.62|50.22|5.52|385.02|513.36|3312.66|4670.46|6.41|385.02|128.34|134.75|-3184.32| +2452247|45869|2447|82843|172331|2619|45707|4|102|1246|86|4.29|8.27|0.90|0.00|77.40|368.94|711.22|3.09|0.00|77.40|80.49|-291.54| +2452247|45869|10755|82843|172331|2619|45707|4|207|1246|19|83.54|99.41|48.71|0.00|925.49|1587.26|1888.79|37.01|0.00|925.49|962.50|-661.77| +2452247|45869|6765|82843|172331|2619|45707|4|110|1246|29|5.60|7.95|4.77|0.00|138.33|162.40|230.55|9.68|0.00|138.33|148.01|-24.07| +||4223|82843|172331||45707|4|57|1246|45||8.60||||||4.32||216.45|220.77|-132.30| +2452247|45869|3909|82843|172331|2619|45707|4|185|1246|81|21.61|25.49|4.07|0.00|329.67|1750.41|2064.69|3.29|0.00|329.67|332.96|-1420.74| +||8765||172331||45707|4|47|1246|85||118.31||0.00|5027.75||10056.35|402.22|0.00|5027.75|5429.97|-3017.50| +2452247|45869|765|82843|172331|2619|45707|4|129|1246|30|87.94|165.32|21.49|0.00|644.70|2638.20|4959.60|32.23|0.00|644.70|676.93|-1993.50| +2452247|45869|3293|82843|172331|2619|45707|4|200|1246|6|21.48|36.08|31.02|0.00|186.12|128.88|216.48|13.02|0.00|186.12|199.14|57.24| +2452247|45869|11263|82843|172331|2619|45707|4|236|1246|92|39.42|72.92|25.52|0.00|2347.84|3626.64|6708.64|164.34|0.00|2347.84|2512.18|-1278.80| +2452247|45869|10979|82843|172331|2619|45707|4|215|1246|28|54.29|57.54|12.65|0.00|354.20|1520.12|1611.12|10.62|0.00|354.20|364.82|-1165.92| +2451129|39377|15202|36248|422551|6454|11557|8|153|1247|64|41.87|63.22|32.24|0.00|2063.36|2679.68|4046.08|41.26|0.00|2063.36|2104.62|-616.32| +2451129|39377|14972|36248|422551|6454|11557|8|183|1247|65|40.98|52.45|4.19|0.00|272.35|2663.70|3409.25|19.06|0.00|272.35|291.41|-2391.35| +2451129|39377|5341|36248|422551|6454|11557|8|178|1247|71|47.49|70.28|56.22|39.91|3991.62|3371.79|4989.88|237.10|39.91|3951.71|4188.81|579.92| +2451129|39377|5776|36248|422551|6454|11557|8|213|1247|26|89.86|90.75|57.17|0.00|1486.42|2336.36|2359.50|29.72|0.00|1486.42|1516.14|-849.94| +2451129|39377|3403|36248|422551|6454|11557|8|46|1247|97|58.84|97.67|31.25|0.00|3031.25|5707.48|9473.99|60.62|0.00|3031.25|3091.87|-2676.23| +2451129|39377|4276|36248|422551|6454|11557|8|9|1247|48|67.57|116.22|84.84|0.00|4072.32|3243.36|5578.56|325.78|0.00|4072.32|4398.10|828.96| +2451129|39377|6410|36248|422551|6454|11557|8|163|1247|22|32.05|36.85|26.90|0.00|591.80|705.10|810.70|47.34|0.00|591.80|639.14|-113.30| +2451129|39377|2816|36248|422551|6454|11557|8|9|1247|80|55.56|97.23|95.28|0.00|7622.40|4444.80|7778.40|533.56|0.00|7622.40|8155.96|3177.60| +||4317||262887|||4||1248||||9.42|||||0.94||18.84||| +2452611|52927|3822|43787|262887|3940|15040|4|149|1248|40|75.15|99.19|51.57|0.00|2062.80|3006.00|3967.60|185.65|0.00|2062.80|2248.45|-943.20| +2452611|52927|2793|43787|262887|3940|15040|4|33|1248|57|97.02|150.38|24.06|0.00|1371.42|5530.14|8571.66|123.42|0.00|1371.42|1494.84|-4158.72| +2452611|52927|6924|43787|262887|3940|15040|4|127|1248|9|59.80|108.23|91.99|0.00|827.91|538.20|974.07|0.00|0.00|827.91|827.91|289.71| +|52927|13857|43787||3940|15040|4|221|1248||3.57|||0.00|203.20||522.40||0.00||205.23|| +2452611|52927|2581|43787|262887|3940|15040|4|256|1248|71|27.04|47.04|46.56|330.57|3305.76|1919.84|3339.84|238.01|330.57|2975.19|3213.20|1055.35| +2452611|52927|13782|43787|262887|3940|15040|4|97|1248|98|59.74|66.90|22.07|0.00|2162.86|5854.52|6556.20|0.00|0.00|2162.86|2162.86|-3691.66| +2452611|52927|5427|43787|262887|3940|15040|4|160|1248|91|25.06|38.34|28.75|0.00|2616.25|2280.46|3488.94|156.97|0.00|2616.25|2773.22|335.79| +2452611|52927|5191|43787|262887|3940|15040|4|70|1248|73|85.81|133.00|34.58|2044.71|2524.34|6264.13|9709.00|9.59|2044.71|479.63|489.22|-5784.50| +2452611|52927|9135|43787|262887|3940|15040|4|179|1248|72|3.07|3.89|3.11|0.00|223.92|221.04|280.08|2.23|0.00|223.92|226.15|2.88| +2452611|52927|10944|43787|262887|3940|15040|4|285|1248|61|66.33|76.94|73.86|0.00|4505.46|4046.13|4693.34|315.38|0.00|4505.46|4820.84|459.33| +2452611|52927|11677|43787|262887|3940|15040|4|91|1248|10|40.08|62.92|34.60|238.74|346.00|400.80|629.20|9.65|238.74|107.26|116.91|-293.54| +2452611|52927|3705|43787|262887|3940|15040|4|293|1248|41|9.31|14.52|12.92|0.00|529.72|381.71|595.32|10.59|0.00|529.72|540.31|148.01| +2452584|32499|6273|15167|1291977|1587|25597|4|58|1249|12|53.15|62.71|34.49|0.00|413.88|637.80|752.52|4.13|0.00|413.88|418.01|-223.92| +2452584|32499|4872|15167|1291977|1587|25597|4|250|1249|44|86.62|160.24|24.03|0.00|1057.32|3811.28|7050.56|0.00|0.00|1057.32|1057.32|-2753.96| +2452584|32499|9981|15167|1291977|1587|25597|4|90|1249|57|49.22|63.49|49.52|0.00|2822.64|2805.54|3618.93|197.58|0.00|2822.64|3020.22|17.10| +2452584|32499|14388|15167|1291977|1587|25597|4|53|1249|86|28.93|29.21|5.84|281.25|502.24|2487.98|2512.06|8.83|281.25|220.99|229.82|-2266.99| +2452584|32499|1371|15167|1291977|1587|25597|4|40|1249|23|53.15|94.60|71.89|0.00|1653.47|1222.45|2175.80|33.06|0.00|1653.47|1686.53|431.02| +2452584|32499|7884|15167|1291977|1587|25597|4|112|1249|59|25.27|27.29|5.73|0.00|338.07|1490.93|1610.11|30.42|0.00|338.07|368.49|-1152.86| +2452584|32499|10275|15167|1291977|1587|25597|4|174|1249|86|33.27|47.57|29.49|0.00|2536.14|2861.22|4091.02|177.52|0.00|2536.14|2713.66|-325.08| +2452584|32499|5553|15167|1291977|1587|25597|4|57|1249|43|9.73|15.76|2.83|0.00|121.69|418.39|677.68|2.43|0.00|121.69|124.12|-296.70| +2452584|32499|1320|15167|1291977|1587|25597|4|19|1249|43|23.04|35.94|30.90|0.00|1328.70|990.72|1545.42|66.43|0.00|1328.70|1395.13|337.98| +2452584|32499|8319|15167|1291977|1587|25597|4|225|1249|11|69.91|89.48|57.26|0.00|629.86|769.01|984.28|31.49|0.00|629.86|661.35|-139.15| +2452584|32499|121|15167|1291977|1587|25597|4|73|1249|89|82.88|136.75|1.36|0.00|121.04|7376.32|12170.75|2.42|0.00|121.04|123.46|-7255.28| +2452584|32499|9774|15167|1291977|1587|25597|4|96|1249|87|95.39|156.43|61.00|0.00|5307.00|8298.93|13609.41|424.56|0.00|5307.00|5731.56|-2991.93| +2452584|32499|7632|15167|1291977|1587|25597|4|269|1249|15|28.73|34.76|26.41|0.00|396.15|430.95|521.40|15.84|0.00|396.15|411.99|-34.80| +2452584|32499|10464|15167|1291977|1587|25597|4|188|1249|98|2.65|5.27|0.36|5.29|35.28|259.70|516.46|1.49|5.29|29.99|31.48|-229.71| +2452178|75312|3655|9095|1240655|4568|5728|1|123|1250|39|43.60|79.78|32.70|0.00|1275.30|1700.40|3111.42|0.00|0.00|1275.30|1275.30|-425.10| +2452178|75312|15955|9095|1240655|4568|5728|1|218|1250|31|32.83|34.79|20.52|0.00|636.12|1017.73|1078.49|6.36|0.00|636.12|642.48|-381.61| +2452178||14809||||5728|1|225|1250|||65.23|63.27|0.00|1391.94|908.38|1435.06|41.75|0.00|1391.94||483.56| +2452178|75312|2799|||4568|||24|1250||||20.80|6.24|||||6.24|201.76||| +2452178|75312|16655|9095|1240655|4568|5728|1|134|1250|73|24.39|32.19|1.93|0.00|140.89|1780.47|2349.87|11.27|0.00|140.89|152.16|-1639.58| +2452178|75312|3267|9095|1240655|4568|5728|1|18|1250|79|73.92|81.31|34.15|0.00|2697.85|5839.68|6423.49|242.80|0.00|2697.85|2940.65|-3141.83| +2452178|75312|14825|9095|1240655|4568|5728|1|292|1250|43|61.43|103.20|19.60|0.00|842.80|2641.49|4437.60|33.71|0.00|842.80|876.51|-1798.69| +2452178|75312|4497|9095|1240655|4568|5728|1|275|1250|60|87.55|120.81|51.94|0.00|3116.40|5253.00|7248.60|155.82|0.00|3116.40|3272.22|-2136.60| +2452178|75312|9331|9095|1240655|4568|5728|1|195|1250|26|55.75|109.82|19.76|0.00|513.76|1449.50|2855.32|20.55|0.00|513.76|534.31|-935.74| +2452178|75312|9231|9095|1240655|4568|5728|1|30|1250|76|17.89|25.40|17.78|0.00|1351.28|1359.64|1930.40|27.02|0.00|1351.28|1378.30|-8.36| +2452178|75312|11541|9095|1240655|4568|5728|1|114|1250|49|54.49|64.84|1.29|0.00|63.21|2670.01|3177.16|4.42|0.00|63.21|67.63|-2606.80| +2452178|75312|7953|9095|1240655|4568|5728|1|130|1250|75|9.44|17.18|7.04|0.00|528.00|708.00|1288.50|47.52|0.00|528.00|575.52|-180.00| +2452178|75312|15853|9095|1240655|4568|5728|1|6|1250|55|95.93|183.22|142.91|0.00|7860.05|5276.15|10077.10|707.40|0.00|7860.05|8567.45|2583.90| +2452178|75312|10819|9095|1240655|4568|5728|1|12|1250|17|98.86|109.73|28.52|0.00|484.84|1680.62|1865.41|38.78|0.00|484.84|523.62|-1195.78| +2452178|75312|12955|9095|1240655|4568|5728|1|40|1250|88|40.28|65.25|22.18|0.00|1951.84|3544.64|5742.00|175.66|0.00|1951.84|2127.50|-1592.80| +2452178|75312|7213|9095|1240655|4568|5728|1|4|1250|99|47.62|82.85|48.05|0.00|4756.95|4714.38|8202.15|380.55|0.00|4756.95|5137.50|42.57| +2451357|66285|2342|83035|636163|1738|22113|1|150|1251|83|70.89|71.59|7.15|0.00|593.45|5883.87|5941.97|53.41|0.00|593.45|646.86|-5290.42| +2451357|66285|11000|83035|636163|1738|22113|1|172|1251|48|42.25|47.32|14.66|0.00|703.68|2028.00|2271.36|49.25|0.00|703.68|752.93|-1324.32| +2451357|66285|9362|83035|636163|1738|22113|1|15|1251|72|32.46|35.38|20.52|0.00|1477.44|2337.12|2547.36|0.00|0.00|1477.44|1477.44|-859.68| +2451357|66285|61|83035|636163|1738|22113|1|80|1251|31|73.01|103.67|13.47|292.29|417.57|2263.31|3213.77|5.01|292.29|125.28|130.29|-2138.03| +2451357|66285|7976|83035|636163|1738|22113|1|159|1251|21|65.78|83.54|69.33|0.00|1455.93|1381.38|1754.34|58.23|0.00|1455.93|1514.16|74.55| +||5836||636163|1738|22113|||1251|93||||507.79|2207.82||2725.83||507.79|||159.95| +2451357|66285|4879|83035|636163|1738|22113|1|117|1251|98|35.46|54.60|13.65|0.00|1337.70|3475.08|5350.80|120.39|0.00|1337.70|1458.09|-2137.38| +2451357|66285|15643|83035|636163|1738|22113|1|182|1251|77|3.08|4.71|1.50|36.96|115.50|237.16|362.67|0.00|36.96|78.54|78.54|-158.62| +2451357|66285|12880|83035|636163|1738|22113|1|89|1251|16|1.24|2.18|0.85|0.00|13.60|19.84|34.88|0.27|0.00|13.60|13.87|-6.24| +2451104|34227|14623|22184|1713169|601|48833|10|175|1252|89|54.28|80.87|54.99|0.00|4894.11|4830.92|7197.43|244.70|0.00|4894.11|5138.81|63.19| +2451104|34227|5854|22184|1713169|601|48833|10|42|1252|81|31.39|60.58|37.55|0.00|3041.55|2542.59|4906.98|182.49|0.00|3041.55|3224.04|498.96| +2451104|34227|17131|22184|1713169|601|48833|10|25|1252|72|14.06|22.77|12.52|0.00|901.44|1012.32|1639.44|9.01|0.00|901.44|910.45|-110.88| +2451104|34227|11996|22184|1713169|601|48833|10|53|1252|22|69.07|69.76|34.88|0.00|767.36|1519.54|1534.72|15.34|0.00|767.36|782.70|-752.18| +2451104|34227|8794|22184|1713169|601|48833|10|187|1252|100|16.05|26.32|2.63|0.00|263.00|1605.00|2632.00|5.26|0.00|263.00|268.26|-1342.00| +2451104||3166|22184|1713169|601|48833||216|1252|||8.44||||310.50|422.00|||113.50||-197.00| +2451104|34227|13255|22184|1713169|601|48833|10|48|1252|49|12.88|19.32|16.22|0.00|794.78|631.12|946.68|23.84|0.00|794.78|818.62|163.66| +2451104|34227|13852|22184|1713169|601|48833|10|74|1252|100|95.32|130.58|54.84|0.00|5484.00|9532.00|13058.00|383.88|0.00|5484.00|5867.88|-4048.00| +2451104||2438|22184||601||10||1252||||11.71|282.67|||1474.92|3.46|282.67|115.47|118.93|-830.07| +2451104|34227|9040|22184|1713169|601|48833|10|292|1252|25|11.99|22.42|16.59|0.00|414.75|299.75|560.50|24.88|0.00|414.75|439.63|115.00| +2451535|34078|17650|79641|554385|7082|21826|10|48|1253|20|85.65|152.45|128.05|1536.60|2561.00|1713.00|3049.00|51.22|1536.60|1024.40|1075.62|-688.60| +2451535|34078|2812|79641|554385|7082|21826|10|10|1253|69|99.35|147.03|108.80|0.00|7507.20|6855.15|10145.07|675.64|0.00|7507.20|8182.84|652.05| +2451535|34078|17378|79641|554385|7082|21826|10|214|1253|48|27.67|48.69|44.30|0.00|2126.40|1328.16|2337.12|170.11|0.00|2126.40|2296.51|798.24| +2451535|34078|10087|79641|554385|7082|21826|10|58|1253|80|75.87|81.18|48.70|0.00|3896.00|6069.60|6494.40|0.00|0.00|3896.00|3896.00|-2173.60| +2451535|34078|5888|79641|554385|7082|21826|10|258|1253|6|76.16|133.28|86.63|0.00|519.78|456.96|799.68|10.39|0.00|519.78|530.17|62.82| +2451535|34078|16627|79641|554385|7082|21826|10|213|1253|58|81.44|114.01|60.42|1682.09|3504.36|4723.52|6612.58|54.66|1682.09|1822.27|1876.93|-2901.25| +2451535|34078|15454|79641|554385|7082|21826|10|260|1253|11|89.73|179.46|12.56|0.00|138.16|987.03|1974.06|0.00|0.00|138.16|138.16|-848.87| +2451535|34078|12206|79641|554385|7082|21826|10|264|1253|54|83.44|147.68|81.22|0.00|4385.88|4505.76|7974.72|219.29|0.00|4385.88|4605.17|-119.88| +2451535|34078|16994|79641|554385|7082|21826|10|142|1253|64|11.42|19.87|9.33|0.00|597.12|730.88|1271.68|5.97|0.00|597.12|603.09|-133.76| +2451535|34078|14024|79641|554385|7082|21826|10|164|1253|79|36.17|56.06|43.16|0.00|3409.64|2857.43|4428.74|68.19|0.00|3409.64|3477.83|552.21| +2451535|34078|14353|79641|554385|7082|21826|10|256|1253|97|67.08|99.94|61.96|0.00|6010.12|6506.76|9694.18|420.70|0.00|6010.12|6430.82|-496.64| +2451535|34078|9199|79641|554385|7082|21826|10|77|1253|69|9.94|12.82|0.76|0.00|52.44|685.86|884.58|4.71|0.00|52.44|57.15|-633.42| +2451535|34078|12466|79641|554385|7082|21826|10|93|1253|6|96.97|143.51|101.89|0.00|611.34|581.82|861.06|12.22|0.00|611.34|623.56|29.52| +2451535|34078|13153|79641|554385|7082|21826|10|288|1253|70|49.54|64.40|34.77|0.00|2433.90|3467.80|4508.00|97.35|0.00|2433.90|2531.25|-1033.90| +2452150|45556|6607|80092|1841364|4236|35508|10|236|1254|24|43.29|55.41|13.85|152.90|332.40|1038.96|1329.84|5.38|152.90|179.50|184.88|-859.46| +2452150|45556|7047|80092|1841364|4236|35508|10|194|1254|38|90.13|146.91|38.19|0.00|1451.22|3424.94|5582.58|14.51|0.00|1451.22|1465.73|-1973.72| +2452150|45556|4973|80092|1841364|4236|35508|10|236|1254|2|26.73|33.67|26.59|49.45|53.18|53.46|67.34|0.29|49.45|3.73|4.02|-49.73| +2452150|45556|1473|80092|1841364|4236|35508|10|176|1254|7|21.72|39.31|36.95|0.00|258.65|152.04|275.17|18.10|0.00|258.65|276.75|106.61| +|45556|11907||1841364||35508|||1254||38.64|64.91|33.75|0.00|2531.25|2898.00|||0.00|2531.25||-366.75| +2452150|45556|17571|80092|1841364|4236|35508|10|5|1254|78|81.17|94.15|77.20|0.00|6021.60|6331.26|7343.70|120.43|0.00|6021.60|6142.03|-309.66| +2452150|45556|14677|80092|1841364|4236|35508|10|86|1254|7|73.19|109.05|27.26|0.00|190.82|512.33|763.35|3.81|0.00|190.82|194.63|-321.51| +2452150|45556|11395|80092|1841364|4236|35508|10|28|1254|87|58.65|76.83|59.92|0.00|5213.04|5102.55|6684.21|417.04|0.00|5213.04|5630.08|110.49| +2452150|45556|14511|80092|1841364|4236|35508|10|64|1254|46|99.52|115.44|15.00|0.00|690.00|4577.92|5310.24|41.40|0.00|690.00|731.40|-3887.92| +2452150|45556|13509|80092|1841364|4236|35508|10|205|1254|83|19.18|24.55|13.74|0.00|1140.42|1591.94|2037.65|79.82|0.00|1140.42|1220.24|-451.52| +2452074|33405|5329|72384|1235829|74|11966|8|207|1255|44|6.06|10.90|7.95|0.00|349.80|266.64|479.60|13.99|0.00|349.80|363.79|83.16| +2452074|33405|16551|72384|1235829|74|11966|8|54|1255|78|7.51|12.69|6.72|0.00|524.16|585.78|989.82|47.17|0.00|524.16|571.33|-61.62| +2452074|33405|51|72384|1235829|74|11966|8|255|1255|26|94.21|164.86|34.62|0.00|900.12|2449.46|4286.36|45.00|0.00|900.12|945.12|-1549.34| +2452074|33405|13799|72384|||11966|8|45|1255||43.92|||0.00|133.74||1486.08||0.00|||-656.82| +2452074|33405|12123|72384|1235829|74|11966|8|189|1255|70|29.57|53.81|27.98|0.00|1958.60|2069.90|3766.70|117.51|0.00|1958.60|2076.11|-111.30| +2452074|33405|17623|72384|1235829|74|11966|8|75|1255|13|89.18|137.33|120.85|0.00|1571.05|1159.34|1785.29|62.84|0.00|1571.05|1633.89|411.71| +2452074|33405|8151|72384|1235829|74|11966|8|142|1255|92|6.18|11.24|2.92|0.00|268.64|568.56|1034.08|0.00|0.00|268.64|268.64|-299.92| +2452074|33405|7643|72384|1235829|74|11966|8|244|1255|78|69.47|120.87|25.38|0.00|1979.64|5418.66|9427.86|39.59|0.00|1979.64|2019.23|-3439.02| +2452074|33405|14843|72384|1235829|74|11966|8|202|1255|17|48.78|57.07|18.83|0.00|320.11|829.26|970.19|28.80|0.00|320.11|348.91|-509.15| +2452156|32781|11789|26041|1194537|4186|29035|2|164|1256|100|10.14|11.86|7.35|0.00|735.00|1014.00|1186.00|58.80|0.00|735.00|793.80|-279.00| +|32781|16021|||4186|29035||119|1256|59|||45.17|0.00|2665.03|||79.95|0.00||2744.98|-722.16| +2452156|32781|543|26041|1194537|4186|29035|2|243|1256|69|96.62|187.44|13.12|0.00|905.28|6666.78|12933.36|45.26|0.00|905.28|950.54|-5761.50| +2452156|32781|17193|26041|1194537|4186|29035|2|273|1256|2|34.13|34.13|20.81|0.00|41.62|68.26|68.26|2.08|0.00|41.62|43.70|-26.64| +2452156|32781|9485|26041|1194537|4186|29035|2|168|1256|53|28.95|57.90|43.42|0.00|2301.26|1534.35|3068.70|207.11|0.00|2301.26|2508.37|766.91| +2452156|32781|961|26041|1194537|4186|29035|2|285|1256|85|5.83|11.48|3.55|0.00|301.75|495.55|975.80|27.15|0.00|301.75|328.90|-193.80| +2452156|32781|14393|26041|1194537|4186|29035|2|45|1256|96|46.67|79.80|16.75|0.00|1608.00|4480.32|7660.80|64.32|0.00|1608.00|1672.32|-2872.32| +2452156|32781|14161|26041|1194537|4186|29035|2|263|1256|23|99.08|135.73|124.87|0.00|2872.01|2278.84|3121.79|28.72|0.00|2872.01|2900.73|593.17| +2452156|32781|7049|26041|1194537|4186|29035|2|58|1256|67|61.94|89.81|44.90|0.00|3008.30|4149.98|6017.27|90.24|0.00|3008.30|3098.54|-1141.68| +2452156|32781|12181|26041|1194537|4186|29035|2|269|1256|49|69.08|81.51|23.63|0.00|1157.87|3384.92|3993.99|34.73|0.00|1157.87|1192.60|-2227.05| +2452156|32781|5345|26041|1194537|4186|29035|2|95|1256|45|71.14|71.14|36.99|632.52|1664.55|3201.30|3201.30|82.56|632.52|1032.03|1114.59|-2169.27| +2452156|32781|9845|26041|1194537|4186|29035|2|270|1256|72|26.83|38.90|32.28|0.00|2324.16|1931.76|2800.80|23.24|0.00|2324.16|2347.40|392.40| +2452156|32781|471|26041|1194537|4186|29035|2|69|1256|88|41.57|59.02|9.44|0.00|830.72|3658.16|5193.76|41.53|0.00|830.72|872.25|-2827.44| +2452156|32781|17903|26041|1194537|4186|29035|2|200|1256|57|72.64|121.30|50.94|0.00|2903.58|4140.48|6914.10|145.17|0.00|2903.58|3048.75|-1236.90| +2451684|49263|2060|38713|1558328|2743|18758|7|123|1257|14|84.30|141.62|76.47|0.00|1070.58|1180.20|1982.68|64.23|0.00|1070.58|1134.81|-109.62| +2451684|49263|14909|38713|1558328|2743|18758|7|248|1257|14|89.56|128.96|23.21|0.00|324.94|1253.84|1805.44|19.49|0.00|324.94|344.43|-928.90| +2451684|49263|5357|38713|1558328|2743|18758|7|26|1257|40|19.56|23.47|15.49|563.83|619.60|782.40|938.80|3.90|563.83|55.77|59.67|-726.63| +2451684|49263|974|38713|1558328|2743|18758|7|241|1257|19|69.64|130.92|0.00|0.00|0.00|1323.16|2487.48|0.00|0.00|0.00|0.00|-1323.16| +2451684|49263|2576|38713|1558328|2743|18758|7|253|1257|65|64.28|81.63|13.87|0.00|901.55|4178.20|5305.95|72.12|0.00|901.55|973.67|-3276.65| +2451684|49263|16975|38713|1558328|2743|18758|7|158|1257|62|10.02|15.83|5.22|200.65|323.64|621.24|981.46|8.60|200.65|122.99|131.59|-498.25| +2451684|49263|4103|38713|1558328|2743|18758|7|289|1257|32|23.71|32.24|29.98|0.00|959.36|758.72|1031.68|76.74|0.00|959.36|1036.10|200.64| +2451684|49263|17792|38713|1558328|2743|18758|7|59|1257|9|66.78|120.87|111.20|580.46|1000.80|601.02|1087.83|4.20|580.46|420.34|424.54|-180.68| +2451684|49263|5564|38713|1558328|2743|18758|7|117|1257|73|15.50|29.14|16.31|0.00|1190.63|1131.50|2127.22|107.15|0.00|1190.63|1297.78|59.13| +2451398|39209|5012|97227|799677|5774|13024|2|172|1258|27|72.89|126.09|7.56|0.00|204.12|1968.03|3404.43|14.28|0.00|204.12|218.40|-1763.91| +2451398|39209|14695|97227|799677|5774|13024|2|225|1258|34|27.25|35.15|14.76|0.00|501.84|926.50|1195.10|5.01|0.00|501.84|506.85|-424.66| +2451398|39209|15325|97227|799677|5774|13024|2|14|1258|8|99.19|124.97|1.24|0.00|9.92|793.52|999.76|0.29|0.00|9.92|10.21|-783.60| +2451398|39209|12679|97227|799677|5774|13024|2|149|1258|32|2.48|3.57|0.49|0.00|15.68|79.36|114.24|0.00|0.00|15.68|15.68|-63.68| +2451398|39209|11173|97227|799677|5774|13024|2|162|1258|16|9.08|10.89|2.17|0.00|34.72|145.28|174.24|2.43|0.00|34.72|37.15|-110.56| +2451398|39209|17401|97227|799677|5774|13024|2|240|1258|87|67.74|118.54|60.45|0.00|5259.15|5893.38|10312.98|473.32|0.00|5259.15|5732.47|-634.23| +2451398|39209|932|97227|799677|5774|13024|2|15|1258|33|26.94|44.72|7.60|0.00|250.80|889.02|1475.76|5.01|0.00|250.80|255.81|-638.22| +2451398|39209|17914|97227|799677|5774|13024|2|66|1258|50|91.80|181.76|25.44|0.00|1272.00|4590.00|9088.00|114.48|0.00|1272.00|1386.48|-3318.00| +2451398|39209|14870|97227|799677|5774|13024|2|250|1258|77|48.54|67.47|57.34|0.00|4415.18|3737.58|5195.19|309.06|0.00|4415.18|4724.24|677.60| +2451398|39209|2011|97227|799677|5774|13024|2|1|1258|73|5.55|10.82|3.24|0.00|236.52|405.15|789.86|16.55|0.00|236.52|253.07|-168.63| +2451398|39209|9376|97227|799677|5774|13024|2|245|1258|41|95.55|150.01|141.00|1098.39|5781.00|3917.55|6150.41|327.78|1098.39|4682.61|5010.39|765.06| +||17108|97227|799677|||||1258||59.61||2.86||245.96||8201.82|||||| +2452374|39212|15198|90911|1184841|1971|11501|1|182|1259|23|36.13|65.03|62.42|301.48|1435.66|830.99|1495.69|22.68|301.48|1134.18|1156.86|303.19| +2452374|39212|3174|90911|1184841|1971|11501|1|176|1259|74|91.03|93.76|75.94|0.00|5619.56|6736.22|6938.24|224.78|0.00|5619.56|5844.34|-1116.66| +2452374|39212|17553|90911|1184841|1971|11501|1|88|1259|11|16.40|21.48|3.22|0.00|35.42|180.40|236.28|1.77|0.00|35.42|37.19|-144.98| +2452374|39212|13002|90911|1184841|1971|11501|1|50|1259|88|41.46|71.31|41.35|0.00|3638.80|3648.48|6275.28|145.55|0.00|3638.80|3784.35|-9.68| +2452374|39212|7389|90911|1184841|1971|11501|1|300|1259|45|98.60|127.19|7.63|0.00|343.35|4437.00|5723.55|17.16|0.00|343.35|360.51|-4093.65| +2452374|39212|12597|90911|1184841|1971|11501|1|14|1259|86|32.43|33.07|19.18|0.00|1649.48|2788.98|2844.02|32.98|0.00|1649.48|1682.46|-1139.50| +2452374|39212|4933|90911|1184841|1971|11501|1|174|1259|1|33.95|49.56|35.18|0.00|35.18|33.95|49.56|1.05|0.00|35.18|36.23|1.23| +2452374|39212|15183|90911|1184841|1971|11501|1|66|1259|40|45.50|91.00|12.74|117.20|509.60|1820.00|3640.00|27.46|117.20|392.40|419.86|-1427.60| +2451008|63480|10370|38153|673284|4062|7193|4|275|1260|81|41.23|67.20|8.73|0.00|707.13|3339.63|5443.20|49.49|0.00|707.13|756.62|-2632.50| +2451008|63480|12158|38153|673284|4062|7193|4|86|1260|37|19.67|23.40|14.74|0.00|545.38|727.79|865.80|16.36|0.00|545.38|561.74|-182.41| +2451008|63480|1334|38153|673284|4062|7193|4|276|1260|34|46.88|53.91|11.32|0.00|384.88|1593.92|1832.94|0.00|0.00|384.88|384.88|-1209.04| +2451008|63480|16801|38153|673284|4062|7193|4|158|1260|59|60.75|108.13|88.66|2196.99|5230.94|3584.25|6379.67|0.00|2196.99|3033.95|3033.95|-550.30| +2451008|63480|8812|38153|673284|4062|7193|4|40|1260|38|60.72|88.65|76.23|0.00|2896.74|2307.36|3368.70|86.90|0.00|2896.74|2983.64|589.38| +2451008|63480|1780||673284|4062|7193|||1260||||87.94|0.00|||7383.44||0.00||6099.51|| +2451008|63480|1652|38153|673284|4062|7193|4|277|1260|24|78.85|104.08|90.54|0.00|2172.96|1892.40|2497.92|108.64|0.00|2172.96|2281.60|280.56| +2451008|63480|7112|38153|673284|4062|7193|4|194|1260|24|8.41|14.96|3.59|0.00|86.16|201.84|359.04|6.03|0.00|86.16|92.19|-115.68| +2451008|63480|8344|38153|673284|4062|7193|4|111|1260|65|85.88|87.59|84.96|0.00|5522.40|5582.20|5693.35|331.34|0.00|5522.40|5853.74|-59.80| +2451008|63480|8282|38153|673284|4062|7193|4|84|1260|64|55.94|91.74|9.17|0.00|586.88|3580.16|5871.36|35.21|0.00|586.88|622.09|-2993.28| +2451957|51819|17225|690|728665|3040|27415|2|26|1261|64|85.10|138.71|4.16|7.98|266.24|5446.40|8877.44|12.91|7.98|258.26|271.17|-5188.14| +2451957|51819|11991|690|728665|3040|27415|2|52|1261|51|20.93|23.86|2.38|0.00|121.38|1067.43|1216.86|7.28|0.00|121.38|128.66|-946.05| +2451957|51819|9901|690|728665|3040|27415|2|152|1261|10|24.63|29.06|12.20|0.00|122.00|246.30|290.60|1.22|0.00|122.00|123.22|-124.30| +2451957|51819|7607|690|728665|3040|27415|2|140|1261|12|56.49|96.59|96.59|0.00|1159.08|677.88|1159.08|57.95|0.00|1159.08|1217.03|481.20| +2451957|51819|15649|690|728665|3040|27415|2|203|1261|82|7.18|12.49|8.86|0.00|726.52|588.76|1024.18|0.00|0.00|726.52|726.52|137.76| +2451957|51819|11489|690|728665|3040|27415|2|300|1261|71|64.75|95.83|74.74|318.39|5306.54|4597.25|6803.93|249.40|318.39|4988.15|5237.55|390.90| +2451957|51819|2421|690|728665|3040|27415|2|173|1261|80|1.74|2.03|1.90|0.00|152.00|139.20|162.40|0.00|0.00|152.00|152.00|12.80| +2451957|51819|16647|690|728665|3040|27415|2|67|1261|37|6.58|8.94|0.53|18.62|19.61|243.46|330.78|0.00|18.62|0.99|0.99|-242.47| +2451957|51819|17457|690|728665|3040|27415|2|84|1261|19|84.10|147.17|55.92|0.00|1062.48|1597.90|2796.23|84.99|0.00|1062.48|1147.47|-535.42| +2451957|51819|6099|690|728665|3040|27415|2|273|1261|64|1.49|2.47|2.07|131.15|132.48|95.36|158.08|0.00|131.15|1.33|1.33|-94.03| +2451957|51819|3335|690|728665|3040|27415|2|243|1261|59|32.40|33.04|15.85|0.00|935.15|1911.60|1949.36|9.35|0.00|935.15|944.50|-976.45| +2451957|51819|17403|690|728665|3040|27415|2|74|1261|29|75.28|114.42|57.21|0.00|1659.09|2183.12|3318.18|132.72|0.00|1659.09|1791.81|-524.03| +2451957|51819|10409|690|728665|3040|27415|2|145|1261|53|29.26|45.35|14.96|0.00|792.88|1550.78|2403.55|63.43|0.00|792.88|856.31|-757.90| +2451957|51819|9793|690|728665|3040|27415|2|239|1261|98|68.63|119.41|90.75|0.00|8893.50|6725.74|11702.18|444.67|0.00|8893.50|9338.17|2167.76| +2451957|51819|9373|690|728665|3040|27415|2|121|1261|4|5.15|6.59|0.85|0.00|3.40|20.60|26.36|0.13|0.00|3.40|3.53|-17.20| +2452556|48407|5533|52467|1862922|130|34138|2|1|1262|32|73.97|144.24|70.67|0.00|2261.44|2367.04|4615.68|22.61|0.00|2261.44|2284.05|-105.60| +2452556||5667||1862922|||2|37|1262|||177.23|102.79||6578.56||||||0.00|| +2452556|48407|1065|52467|1862922|130|34138|2|291|1262|96|32.59|43.67|27.07|0.00|2598.72|3128.64|4192.32|51.97|0.00|2598.72|2650.69|-529.92| +2452556|48407|4717|52467|1862922|130|34138|2|179|1262|1|88.67|107.29|13.94|13.10|13.94|88.67|107.29|0.06|13.10|0.84|0.90|-87.83| +2452556|48407|11061|52467|1862922|130|34138|2|188|1262|54|74.32|121.14|116.29|0.00|6279.66|4013.28|6541.56|62.79|0.00|6279.66|6342.45|2266.38| +2452556|48407|4951|52467|1862922|130|34138|2|105|1262|73|24.84|27.07|0.54|0.00|39.42|1813.32|1976.11|0.39|0.00|39.42|39.81|-1773.90| +2452556|48407|1248|52467|1862922|||||1262||||||||196.46|3.31||||-98.94| +2452556|48407|318|52467|1862922|130|34138|2|69|1262|2|77.42|140.90|98.63|0.00|197.26|154.84|281.80|11.83|0.00|197.26|209.09|42.42| +2452556|48407|17863|52467|1862922|130|34138|2|127|1262|43|73.04|102.25|66.46|1171.68|2857.78|3140.72|4396.75|118.02|1171.68|1686.10|1804.12|-1454.62| +2452556|48407|1584|52467|1862922|130|34138|2|207|1262|19|8.52|10.90|6.86|129.03|130.34|161.88|207.10|0.11|129.03|1.31|1.42|-160.57| +2452168|36361|3661|66078|239688|6451|39661|1|11|1263|65|46.65|79.30|57.09|0.00|3710.85|3032.25|5154.50|259.75|0.00|3710.85|3970.60|678.60| +2452168|36361|5115|66078|239688|6451|39661|1|129|1263|40|71.08|107.33|32.19|0.00|1287.60|2843.20|4293.20|64.38|0.00|1287.60|1351.98|-1555.60| +2452168|36361|5897|66078|239688|6451|39661|1|113|1263|85|15.32|30.64|18.38|656.16|1562.30|1302.20|2604.40|54.36|656.16|906.14|960.50|-396.06| +2452168|36361|6215|66078|239688|6451|39661|1|21|1263|89|87.69|115.75|17.36|0.00|1545.04|7804.41|10301.75|77.25|0.00|1545.04|1622.29|-6259.37| +2452168|36361|13219|66078|239688|6451|39661|1|102|1263|75|16.53|17.35|12.49|281.02|936.75|1239.75|1301.25|45.90|281.02|655.73|701.63|-584.02| +2452168|36361|9229|66078|239688|6451|39661|1|21|1263|31|53.29|90.06|12.60|0.00|390.60|1651.99|2791.86|19.53|0.00|390.60|410.13|-1261.39| +2452168|36361|15615|66078|239688|6451|39661|1|233|1263|9|21.19|30.30|29.69|0.00|267.21|190.71|272.70|24.04|0.00|267.21|291.25|76.50| +2452168|36361|8157|66078|239688|6451|39661|1|185|1263|65|35.80|61.21|37.95|0.00|2466.75|2327.00|3978.65|98.67|0.00|2466.75|2565.42|139.75| +|36361|14131|66078||6451|39661|1||1263|62|90.18|153.30|||4752.30|||||4752.30||| +|36361|16111|||6451||1|100|1263|||24.63||0.00|||||0.00|||| +2452168|36361|15539|66078|239688|6451|39661|1|266|1263|11|50.04|89.07|59.67|0.00|656.37|550.44|979.77|45.94|0.00|656.37|702.31|105.93| +2452168|36361|2483|66078|239688|6451|39661|1|251|1263|50|70.25|93.43|60.72|0.00|3036.00|3512.50|4671.50|182.16|0.00|3036.00|3218.16|-476.50| +2452168|36361|759|66078|239688|6451|39661|1|57|1263|64|98.18|109.96|25.29|0.00|1618.56|6283.52|7037.44|113.29|0.00|1618.56|1731.85|-4664.96| +2452195|46158|1997|88684|824923|2409|4248|8|154|1264|97|22.89|26.78|2.14|0.00|207.58|2220.33|2597.66|6.22|0.00|207.58|213.80|-2012.75| +2452195|46158|15463|88684|824923|2409||8|37|1264|34||145.92||0.00|4117.74|3307.52|||0.00|4117.74|4200.09|| +2452195|46158|3865|88684|824923|2409|4248|8|258|1264|31|91.92|154.42|145.15|0.00|4499.65|2849.52|4787.02|224.98|0.00|4499.65|4724.63|1650.13| +2452195|46158|8933|88684|824923|2409|4248|8|266|1264|69|98.24|158.16|86.98|5341.44|6001.62|6778.56|10913.04|46.21|5341.44|660.18|706.39|-6118.38| +2452195|46158|8939|88684|824923|2409|4248|8|227|1264|77|32.16|58.53|11.12|0.00|856.24|2476.32|4506.81|17.12|0.00|856.24|873.36|-1620.08| +2452195|46158|3627|88684|824923|2409|4248|8|130|1264|16|24.44|37.14|3.34|21.91|53.44|391.04|594.24|0.31|21.91|31.53|31.84|-359.51| +2452195|46158|8445|88684|824923|2409|4248|8|173|1264|84|70.29|74.50|2.23|176.08|187.32|5904.36|6258.00|0.89|176.08|11.24|12.13|-5893.12| +2452195|46158|4827|88684|824923|2409|4248|8|105|1264|17|10.63|21.04|8.83|63.04|150.11|180.71|357.68|0.00|63.04|87.07|87.07|-93.64| +2452195|46158|7915|88684|824923|2409|4248|8|105|1264|81|69.36|96.41|0.00|0.00|0.00|5618.16|7809.21|0.00|0.00|0.00|0.00|-5618.16| +2452195|46158|6087|88684|824923|2409|4248|8|228|1264|25|79.15|109.22|56.79|0.00|1419.75|1978.75|2730.50|56.79|0.00|1419.75|1476.54|-559.00| +2452195|46158|2107|88684|824923|2409|4248|8|159|1264|35|47.57|72.30|10.84|0.00|379.40|1664.95|2530.50|30.35|0.00|379.40|409.75|-1285.55| +2452603||17016||1632236|5372||1|95|1265|64|20.91||20.44|||1338.24|1699.20|||1308.16|1347.40|-30.08| +2452603|47301|5088|11100|1632236|5372|15355|1|40|1265|85|43.50|51.76|13.45|0.00|1143.25|3697.50|4399.60|22.86|0.00|1143.25|1166.11|-2554.25| +2452603|47301|17340|11100|1632236|5372|15355|1|177|1265|77|68.26|84.64|27.08|0.00|2085.16|5256.02|6517.28|20.85|0.00|2085.16|2106.01|-3170.86| +2452603|47301|13656|11100|1632236|5372|15355|1|80|1265|95|24.42|40.78|17.12|0.00|1626.40|2319.90|3874.10|16.26|0.00|1626.40|1642.66|-693.50| +2452603|47301|15579|11100|1632236|5372|15355|1|154|1265|68|25.85|29.98|17.68|0.00|1202.24|1757.80|2038.64|84.15|0.00|1202.24|1286.39|-555.56| +2452603|47301|6588|11100|1632236|5372|15355|1|107|1265|64|10.81|21.07|17.27|0.00|1105.28|691.84|1348.48|88.42|0.00|1105.28|1193.70|413.44| +2452603||3187||1632236|5372|15355|||1265||60.82||||1901.13||5432.35|133.07||1901.13|2034.20|-2782.01| +||3009||1632236|5372|15355|||1265|7|29.14|56.24||120.89||203.98|393.68|0.40|120.89|||-198.94| +||13195||1632236||15355|1||1265|2|||||122.78|171.32||1.21||40.52|41.73|| +2451498|50559|5200|20750|1417623|5696|12216|8|161|1266|6|45.97|51.02|22.95|11.01|137.70|275.82|306.12|10.13|11.01|126.69|136.82|-149.13| +2451498|50559|7894|20750|1417623|5696|12216|8|28|1266|78|93.03|146.05|108.07|0.00|8429.46|7256.34|11391.90|505.76|0.00|8429.46|8935.22|1173.12| +2451498|50559|9872|20750|1417623|5696|12216|8|256|1266|23|60.65|111.59|64.72|0.00|1488.56|1394.95|2566.57|44.65|0.00|1488.56|1533.21|93.61| +2451498|50559|8299|20750|1417623|5696|12216|8|100|1266|57|43.26|47.15|31.59|0.00|1800.63|2465.82|2687.55|0.00|0.00|1800.63|1800.63|-665.19| +2451498|50559|17162|20750|1417623|5696|12216|8|234|1266|17|50.37|99.73|59.83|0.00|1017.11|856.29|1695.41|50.85|0.00|1017.11|1067.96|160.82| +2451498|50559|6217|20750|1417623|5696|12216|8|81|1266|91|29.77|58.34|35.58|0.00|3237.78|2709.07|5308.94|0.00|0.00|3237.78|3237.78|528.71| +2451498|50559|7462|20750|1417623|5696|12216|8|148|1266|48|72.07|115.31|102.62|0.00|4925.76|3459.36|5534.88|98.51|0.00|4925.76|5024.27|1466.40| +2451498|50559|13874|20750|1417623|5696|12216|8|35|1266|1|19.16|32.76|22.60|2.93|22.60|19.16|32.76|1.77|2.93|19.67|21.44|0.51| +2451498|50559|14810|20750|1417623|5696|12216|8|219|1266|75|42.52|42.52|0.00|0.00|0.00|3189.00|3189.00|0.00|0.00|0.00|0.00|-3189.00| +2451887|71391|9677|91273|1387255|2437|30019|1|3|1267|45|90.87|165.38|133.95|0.00|6027.75|4089.15|7442.10|180.83|0.00|6027.75|6208.58|1938.60| +2451887|71391|9271|91273|1387255|2437|30019|1|132|1267|26|82.61|108.21|32.46|531.69|843.96|2147.86|2813.46|24.98|531.69|312.27|337.25|-1835.59| +2451887|71391|16988|91273|1387255|2437|30019|1|135|1267|26|91.35|127.89|92.08|0.00|2394.08|2375.10|3325.14|143.64|0.00|2394.08|2537.72|18.98| +||1046|||2437||1|230|1267|||60.94|24.98|896.28|||3168.88|28.18|896.28|402.68|430.86|-1844.76| +2451887|71391|9164|91273|1387255|2437|30019|1|53|1267|76|6.51|10.74|6.12|427.91|465.12|494.76|816.24|1.86|427.91|37.21|39.07|-457.55| +2451887|71391|1435|91273|1387255|2437|30019|1|100|1267|15|93.33|171.72|5.15|0.00|77.25|1399.95|2575.80|3.86|0.00|77.25|81.11|-1322.70| +||4133||1387255||30019|1||1267||70.96|98.63||||6670.24||199.32||3986.54||| +2451887|71391|10472|91273|1387255|2437|30019|1|240|1267|47|57.78|93.60|57.09|0.00|2683.23|2715.66|4399.20|160.99|0.00|2683.23|2844.22|-32.43| +2452152|45626|16547|2374|1031236|4451|26513|1|161|1268|92|79.12|111.55|69.16|0.00|6362.72|7279.04|10262.60|318.13|0.00|6362.72|6680.85|-916.32| +2452152|45626|17007|2374|1031236|4451|26513|1|47|1268|75|39.10|44.57|32.09|240.67|2406.75|2932.50|3342.75|21.66|240.67|2166.08|2187.74|-766.42| +2452152|45626|9589|2374|1031236|4451|26513|1|148|1268|97|19.15|28.34|13.03|0.00|1263.91|1857.55|2748.98|25.27|0.00|1263.91|1289.18|-593.64| +2452152|45626|12145|2374|1031236|4451|26513|1|254|1268|21|42.25|76.47|74.17|0.00|1557.57|887.25|1605.87|124.60|0.00|1557.57|1682.17|670.32| +2452152|45626|5209|2374|1031236|4451|26513|1|21|1268|49|82.88|154.15|44.70|1993.17|2190.30|4061.12|7553.35|11.82|1993.17|197.13|208.95|-3863.99| +2452152|45626|7865|2374|1031236|4451|26513|1|124|1268|17|88.64|88.64|60.27|0.00|1024.59|1506.88|1506.88|51.22|0.00|1024.59|1075.81|-482.29| +2452152|45626|15697|2374|1031236|4451|26513|1|60|1268|79|97.43|118.86|117.67|0.00|9295.93|7696.97|9389.94|278.87|0.00|9295.93|9574.80|1598.96| +2452152|45626|6875|2374|1031236|4451|26513|1|217|1268|76|35.72|48.93|44.52|0.00|3383.52|2714.72|3718.68|236.84|0.00|3383.52|3620.36|668.80| +2452152|45626|10981|2374|1031236|4451|26513|1|66|1268|22|18.19|32.92|28.64|352.84|630.08|400.18|724.24|5.54|352.84|277.24|282.78|-122.94| +2452152||561|2374||||1|160|1268|31|37.43|59.13|49.66||1539.46|1160.33|1833.03|76.97||1539.46||379.13| +2452152|45626|11231|2374|1031236|4451|26513|1|254|1268|15|86.16|161.98|38.87|0.00|583.05|1292.40|2429.70|34.98|0.00|583.05|618.03|-709.35| +2452152|45626|1189|2374|1031236|4451|26513|1|48|1268|44|48.54|72.81|6.55|233.44|288.20|2135.76|3203.64|3.28|233.44|54.76|58.04|-2081.00| +2451179|61662|4015|26571|1588355|3292|17876|7|35|1269|27|59.29|117.98|79.04|0.00|2134.08|1600.83|3185.46|170.72|0.00|2134.08|2304.80|533.25| +2451179|61662|4549|26571|1588355|3292|17876|7|67|1269|85|92.63|168.58|70.80|4994.94|6018.00|7873.55|14329.30|20.46|4994.94|1023.06|1043.52|-6850.49| +2451179|61662|9248|26571|1588355|3292|17876|7|186|1269|63|51.35|78.56|22.78|703.21|1435.14|3235.05|4949.28|21.95|703.21|731.93|753.88|-2503.12| +2451179|61662|15356|26571|1588355|3292|17876|7|240|1269|21|28.98|56.80|53.39|0.00|1121.19|608.58|1192.80|78.48|0.00|1121.19|1199.67|512.61| +2451179|61662|6625|26571|1588355|3292|17876|7|110|1269|60|93.30|149.28|103.00|0.00|6180.00|5598.00|8956.80|123.60|0.00|6180.00|6303.60|582.00| +2451179|61662|13345|26571|1588355|3292|17876|7|194|1269|39|79.76|136.38|53.18|1451.81|2074.02|3110.64|5318.82|31.11|1451.81|622.21|653.32|-2488.43| +2451179|61662|16906|26571|1588355|3292|17876|7|91|1269|37|82.95|125.25|48.84|0.00|1807.08|3069.15|4634.25|0.00|0.00|1807.08|1807.08|-1262.07| +2451179|61662|14950|26571|1588355|3292|17876|7|223|1269|66|66.63|71.29|63.44|0.00|4187.04|4397.58|4705.14|334.96|0.00|4187.04|4522.00|-210.54| +2451179|61662|14143|26571|1588355|3292|17876|7|44|1269|62|12.70|24.76|24.76|0.00|1535.12|787.40|1535.12|61.40|0.00|1535.12|1596.52|747.72| +2451179|61662|14119|26571|1588355|3292|17876|7|186|1269|81|30.62|40.11|19.25|0.00|1559.25|2480.22|3248.91|62.37|0.00|1559.25|1621.62|-920.97| +2450888|35079|1178|89420|415503|2739|29975|10|152|1270|98|86.11|137.77|95.06|4471.62|9315.88|8438.78|13501.46|0.00|4471.62|4844.26|4844.26|-3594.52| +2450888|35079|10528|89420|415503|2739|29975|10|292|1270|59|42.63|66.50|39.23|208.31|2314.57|2515.17|3923.50|147.43|208.31|2106.26|2253.69|-408.91| +2450888|35079|2815|89420|415503|2739|29975|10|231|1270|52|87.53|156.67|139.43|3117.65|7250.36|4551.56|8146.84|330.61|3117.65|4132.71|4463.32|-418.85| +2450888|35079|5156|89420|415503|2739|29975|10|266|1270|19|64.84|125.14|16.26|0.00|308.94|1231.96|2377.66|21.62|0.00|308.94|330.56|-923.02| +2450888|35079|14564|89420|415503|2739|29975|10|212|1270|56|75.89|151.78|97.13|0.00|5439.28|4249.84|8499.68|217.57|0.00|5439.28|5656.85|1189.44| +2450888|35079|15248|89420|415503|2739|29975|10|141|1270|19|1.48|2.67|1.49|7.07|28.31|28.12|50.73|1.91|7.07|21.24|23.15|-6.88| +2450888|35079|5578|89420|415503|2739|29975|10|202|1270|4|7.73|11.82|4.01|0.00|16.04|30.92|47.28|0.48|0.00|16.04|16.52|-14.88| +2450888|35079|13669|89420|415503|2739|29975|10|144|1270|33|67.04|130.72|121.56|0.00|4011.48|2212.32|4313.76|160.45|0.00|4011.48|4171.93|1799.16| +2450888|35079|7441|89420|415503|2739|29975|10|81|1270|15|19.78|35.01|23.45|0.00|351.75|296.70|525.15|31.65|0.00|351.75|383.40|55.05| +2450888|35079|7598|89420|415503|2739|29975|10|47|1270|89|38.49|42.72|36.73|1242.20|3268.97|3425.61|3802.08|121.60|1242.20|2026.77|2148.37|-1398.84| +2450888|35079|806|89420|415503|2739|29975|10|81|1270|8|76.97|152.40|65.53|0.00|524.24|615.76|1219.20|0.00|0.00|524.24|524.24|-91.52| +2450888|35079|6716|89420|415503|2739|29975|10|225|1270|93|9.84|18.99|17.09|0.00|1589.37|915.12|1766.07|127.14|0.00|1589.37|1716.51|674.25| +2450888|35079|10363|89420|415503|2739|29975|10|163|1270|63|44.27|47.36|23.68|1327.73|1491.84|2789.01|2983.68|14.76|1327.73|164.11|178.87|-2624.90| +2450888|35079|916|89420|415503|2739|29975|10|96|1270|22|29.72|35.36|2.47|0.00|54.34|653.84|777.92|3.26|0.00|54.34|57.60|-599.50| +2450888|35079|11438|89420|415503|2739|29975|10|150|1270|94|92.20|175.18|94.59|0.00|8891.46|8666.80|16466.92|177.82|0.00|8891.46|9069.28|224.66| +2452572|63545|10104|27925|1622260|6995|11777|10|112|1271|65|92.53|130.46|14.35|0.00|932.75|6014.45|8479.90|65.29|0.00|932.75|998.04|-5081.70| +2452572|63545|15657|27925|1622260|6995|11777|10|79|1271|4|80.85|138.25|58.06|0.00|232.24|323.40|553.00|0.00|0.00|232.24|232.24|-91.16| +2452572|63545|4609|27925|1622260|6995|11777|10|35|1271|68|18.64|33.36|4.67|298.50|317.56|1267.52|2268.48|1.33|298.50|19.06|20.39|-1248.46| +2452572|63545|17112|27925|1622260|6995|11777|10|237|1271|59|90.24|140.77|135.13|0.00|7972.67|5324.16|8305.43|159.45|0.00|7972.67|8132.12|2648.51| +2452572|63545|17268|27925|1622260|6995|11777|10|46|1271|94|13.46|17.76|0.17|0.00|15.98|1265.24|1669.44|0.47|0.00|15.98|16.45|-1249.26| +2452572|63545|11659|27925|1622260|6995|11777|10|82|1271|89|72.51|92.81|23.20|0.00|2064.80|6453.39|8260.09|144.53|0.00|2064.80|2209.33|-4388.59| +2452572|63545|17857|27925|1622260|6995|11777|10|123|1271|64|9.54|12.68|3.80|0.00|243.20|610.56|811.52|21.88|0.00|243.20|265.08|-367.36| +2452572|63545|4777|27925|1622260|6995|11777|10|131|1271|25|33.63|53.47|28.33|354.12|708.25|840.75|1336.75|24.78|354.12|354.13|378.91|-486.62| +2452572|63545|8691|27925|1622260|6995|11777|10|42|1271|33|50.67|61.81|37.70|124.41|1244.10|1672.11|2039.73|11.19|124.41|1119.69|1130.88|-552.42| +2452572|63545|10362|27925|1622260|6995|11777|10|110|1271|26|30.22|58.62|53.34|291.23|1386.84|785.72|1524.12|54.78|291.23|1095.61|1150.39|309.89| +2452572|63545|16620|27925|1622260|6995|11777|10|272|1271|85|24.01|32.17|18.98|0.00|1613.30|2040.85|2734.45|129.06|0.00|1613.30|1742.36|-427.55| +2452572|63545|8707|27925|1622260|6995|11777|10|288|1271|2|21.79|39.43|0.00|0.00|0.00|43.58|78.86|0.00|0.00|0.00|0.00|-43.58| +2452572|63545|8353|27925|1622260|6995|11777|10|29|1271|13|69.99|96.58|72.43|273.06|941.59|909.87|1255.54|46.79|273.06|668.53|715.32|-241.34| +2452572|63545|4506|27925|1622260|6995|11777|10|22|1271|73|64.78|121.13|89.63|0.00|6542.99|4728.94|8842.49|392.57|0.00|6542.99|6935.56|1814.05| +2452572|63545|4242|27925|1622260|6995|11777|10|255|1271|12|47.94|76.22|22.10|0.00|265.20|575.28|914.64|18.56|0.00|265.20|283.76|-310.08| +2452572|63545|17331|27925|1622260|6995|11777|10|68|1271|82|75.77|90.92|40.91|0.00|3354.62|6213.14|7455.44|100.63|0.00|3354.62|3455.25|-2858.52| +2451780||10868||||29949||26|1272|68|||61.04|0.00|4150.72|4723.96|7983.20|207.53|0.00|4150.72|4358.25|| +2451780|71446|3812|58279|539267|1408|29949|8|285|1272|85|1.32|1.38|0.17|0.00|14.45|112.20|117.30|0.57|0.00|14.45|15.02|-97.75| +2451780|71446|7147|58279|539267|1408|29949|8|136|1272|39|79.58|148.01|99.16|0.00|3867.24|3103.62|5772.39|154.68|0.00|3867.24|4021.92|763.62| +2451780|71446|15595|58279|539267|1408|29949|8|259|1272|59|74.85|111.52|63.56|0.00|3750.04|4416.15|6579.68|187.50|0.00|3750.04|3937.54|-666.11| +2451780|71446|17384|58279|539267|1408|29949|8|165|1272|63|62.88|94.32|68.85|737.38|4337.55|3961.44|5942.16|36.00|737.38|3600.17|3636.17|-361.27| +2451780|71446|17150|58279|539267|1408|29949|8|73|1272|87|70.55|120.64|43.43|0.00|3778.41|6137.85|10495.68|226.70|0.00|3778.41|4005.11|-2359.44| +2451780|71446|1961||539267|1408||8|39|1272|||164.40|1.64|34.11|||13152.00|8.73|34.11||105.82|-6825.31| +2451780|71446|7040|58279|539267|1408|29949|8|174|1272|43|17.70|21.94|1.53|59.86|65.79|761.10|943.42|0.41|59.86|5.93|6.34|-755.17| +2451780|71446|16541|58279|539267|1408|29949|8|149|1272|73|46.97|83.60|74.40|0.00|5431.20|3428.81|6102.80|271.56|0.00|5431.20|5702.76|2002.39| +2451780|71446|13043|58279|539267|1408|29949|8|60|1272|63|87.23|101.18|40.47|0.00|2549.61|5495.49|6374.34|101.98|0.00|2549.61|2651.59|-2945.88| +2451780|71446|8065|58279|539267|1408|29949|8|90|1272|56|90.88|125.41|38.87|0.00|2176.72|5089.28|7022.96|108.83|0.00|2176.72|2285.55|-2912.56| +2451780|71446|3011|58279|539267|1408|29949|8|162|1272|32|19.79|38.98|4.28|0.00|136.96|633.28|1247.36|6.84|0.00|136.96|143.80|-496.32| +2451780|71446|11426|58279|539267|1408|29949|8|51|1272|10|91.60|154.80|122.29|0.00|1222.90|916.00|1548.00|12.22|0.00|1222.90|1235.12|306.90| +2451780|71446|15715|58279|539267|1408|29949|8|292|1272|89|1.64|1.86|0.03|1.04|2.67|145.96|165.54|0.13|1.04|1.63|1.76|-144.33| +2451780|71446|8101|58279|539267|1408|29949|8|185|1272|49|58.65|109.08|43.63|0.00|2137.87|2873.85|5344.92|106.89|0.00|2137.87|2244.76|-735.98| +2451397|36995|12691|84705|1319703|4748|38146|2|13|1273|75|8.47|8.63|6.47|0.00|485.25|635.25|647.25|0.00|0.00|485.25|485.25|-150.00| +2451397|36995|17558|84705|1319703|4748|38146|2|156|1273|69|14.58|26.53|1.06|0.00|73.14|1006.02|1830.57|2.92|0.00|73.14|76.06|-932.88| +2451397|36995|13628|84705|1319703|4748|38146|2|31|1273|71|43.09|72.39|9.41|0.00|668.11|3059.39|5139.69|0.00|0.00|668.11|668.11|-2391.28| +2451397|36995|13519|84705|1319703|4748|38146|2|70|1273|62|50.67|76.00|49.40|0.00|3062.80|3141.54|4712.00|183.76|0.00|3062.80|3246.56|-78.74| +2451397|36995|13982|84705|1319703|4748|38146|2|149|1273|22|91.91|124.07|55.83|171.95|1228.26|2022.02|2729.54|42.25|171.95|1056.31|1098.56|-965.71| +2451397|36995|1831|84705|1319703|4748|38146|2|269|1273|68|91.38|143.46|86.07|0.00|5852.76|6213.84|9755.28|292.63|0.00|5852.76|6145.39|-361.08| +2451397|36995|10960|84705|1319703|4748|38146|2|125|1273|88|62.73|97.23|5.83|0.00|513.04|5520.24|8556.24|15.39|0.00|513.04|528.43|-5007.20| +2451397||1852|84705||4748|38146||64|1273||17.06|32.58|25.41|0.00|||423.54|29.72|0.00||360.05|| +2451397|36995|13672|84705|1319703|4748|38146|2|7|1273|32|99.62|101.61|28.45|0.00|910.40|3187.84|3251.52|36.41|0.00|910.40|946.81|-2277.44| +2451397|36995|9928|84705|1319703|4748|38146|2|98|1273|40|86.06|154.04|60.07|0.00|2402.80|3442.40|6161.60|168.19|0.00|2402.80|2570.99|-1039.60| +2451397|36995|3368|84705|1319703|4748|38146|2|284|1273|80|11.14|20.60|12.15|0.00|972.00|891.20|1648.00|48.60|0.00|972.00|1020.60|80.80| +2451397|36995|15806|84705|1319703|4748|38146|2|3|1273|82|69.44|126.38|56.87|0.00|4663.34|5694.08|10363.16|46.63|0.00|4663.34|4709.97|-1030.74| +2451397|36995|5395|84705|1319703|4748|38146|2|259|1273|41|62.27|102.74|67.80|0.00|2779.80|2553.07|4212.34|27.79|0.00|2779.80|2807.59|226.73| +2451397|36995|7000|84705|1319703|4748|38146|2|112|1273|6|48.47|63.98|3.83|0.00|22.98|290.82|383.88|0.22|0.00|22.98|23.20|-267.84| +2451397|36995|1585|84705|1319703|4748|38146|2|110|1273|99|77.47|90.63|64.34|0.00|6369.66|7669.53|8972.37|573.26|0.00|6369.66|6942.92|-1299.87| +2451397|36995|12530|84705|1319703|4748|38146|2|232|1273|91|34.74|53.84|21.53|0.00|1959.23|3161.34|4899.44|176.33|0.00|1959.23|2135.56|-1202.11| +2452191|40384|3577||552281|6199|35764||3|1274||||5.86|||193.20|256.76|0.00|||164.08|| +2452191|40384|801|24316|552281|6199|35764|1|160|1274|62|14.11|19.61|2.54|0.00|157.48|874.82|1215.82|7.87|0.00|157.48|165.35|-717.34| +||11111|24316|||35764|1|266|1274||39.01||37.43|683.47|3106.69|3237.83|3270.20|96.92|683.47||2520.14|| +2452191|40384|8099|24316|552281|6199|35764|1|185|1274|54|68.87|133.60|38.74|0.00|2091.96|3718.98|7214.40|83.67|0.00|2091.96|2175.63|-1627.02| +2452191|40384|7015|24316|552281|6199|35764|1|141|1274|54|22.41|43.02|3.87|0.00|208.98|1210.14|2323.08|6.26|0.00|208.98|215.24|-1001.16| +2452191|40384|10955|24316|552281|6199|35764|1|240|1274|80|51.65|82.64|32.22|0.00|2577.60|4132.00|6611.20|77.32|0.00|2577.60|2654.92|-1554.40| +2452191|40384|16139|24316|552281|6199|35764|1|298|1274|33|32.32|54.29|51.03|0.00|1683.99|1066.56|1791.57|134.71|0.00|1683.99|1818.70|617.43| +2452191|40384|11737|24316|552281|6199|35764|1|7|1274|79|19.33|35.56|17.78|0.00|1404.62|1527.07|2809.24|28.09|0.00|1404.62|1432.71|-122.45| +2452191|40384|11543|24316|552281|6199|35764|1|166|1274|1|75.56|107.29|77.24|0.00|77.24|75.56|107.29|4.63|0.00|77.24|81.87|1.68| +2452191|40384|11293|24316|552281|6199|35764|1|125|1274|12|40.47|76.89|30.75|55.35|369.00|485.64|922.68|15.68|55.35|313.65|329.33|-171.99| +2452191|40384|13025|24316|552281|6199|35764|1|164|1274|82|20.54|26.08|13.30|0.00|1090.60|1684.28|2138.56|10.90|0.00|1090.60|1101.50|-593.68| +2452191|40384|12079|24316||6199||1|120|1274||4.19|||0.00||372.91||0.64|0.00||16.66|| +2452191|40384|12449|24316|552281|6199|35764|1|21|1274|52|41.45|60.10|8.41|0.00|437.32|2155.40|3125.20|13.11|0.00|437.32|450.43|-1718.08| +2450970|43058|17563|1924|251570|5216|29262|4|47|1275|30|18.36|31.57|14.52|0.00|435.60|550.80|947.10|13.06|0.00|435.60|448.66|-115.20| +2450970|43058|13939|1924|251570|5216|29262|4|34|1275|45|44.67|72.81|64.07|0.00|2883.15|2010.15|3276.45|172.98|0.00|2883.15|3056.13|873.00| +2450970||14342||||29262||213|1275|86|13.09|17.14|3.08|119.19|||||119.19|145.69||-980.05| +2450970|43058|12253|1924|251570|5216|29262|4|105|1275|1|22.81|36.49|14.96|0.00|14.96|22.81|36.49|1.04|0.00|14.96|16.00|-7.85| +2450970|43058|4741|1924|251570|5216|29262|4|264|1275|34|62.53|106.30|15.94|0.00|541.96|2126.02|3614.20|27.09|0.00|541.96|569.05|-1584.06| +2450970|43058|16520|1924|251570|5216|29262|4|230|1275|22|26.75|52.96|49.25|0.00|1083.50|588.50|1165.12|32.50|0.00|1083.50|1116.00|495.00| +2450970|43058|9235|1924|251570|5216|29262|4|55|1275|26|3.63|4.75|3.42|0.00|88.92|94.38|123.50|5.33|0.00|88.92|94.25|-5.46| +2450970|43058|11870|1924|251570|5216|29262|4|267|1275|3|47.95|71.92|45.30|0.00|135.90|143.85|215.76|5.43|0.00|135.90|141.33|-7.95| +2450970|43058|3224|1924|251570|5216|29262|4|280|1275|72|37.57|50.71|18.25|0.00|1314.00|2705.04|3651.12|13.14|0.00|1314.00|1327.14|-1391.04| +2450970|43058|283|1924|251570|5216|29262|4|104|1275|33|92.13|127.13|66.10|0.00|2181.30|3040.29|4195.29|152.69|0.00|2181.30|2333.99|-858.99| +2450970|43058|8798|1924|251570|5216|29262|4|56|1275|100|19.37|27.89|0.27|0.00|27.00|1937.00|2789.00|2.43|0.00|27.00|29.43|-1910.00| +2450970|43058|13438|1924|251570|5216|29262|4|36|1275|44|15.39|28.47|5.12|135.16|225.28|677.16|1252.68|5.40|135.16|90.12|95.52|-587.04| +2450970|43058|13681|1924|251570|5216|29262|4|174|1275|1|98.60|144.94|24.63|0.00|24.63|98.60|144.94|0.98|0.00|24.63|25.61|-73.97| +2451489|71962|6157|54565|1779342|919|42143|10|189|1276|66|39.32|73.52|66.90|309.07|4415.40|2595.12|4852.32|369.56|309.07|4106.33|4475.89|1511.21| +2451489|71962|1264|54565|1779342|919|42143|10|248|1276|97|91.97|180.26|12.61|0.00|1223.17|8921.09|17485.22|36.69|0.00|1223.17|1259.86|-7697.92| +2451489|71962|16159|54565|1779342|919|42143|10|232|1276|21|32.29|58.44|44.41|102.58|932.61|678.09|1227.24|74.70|102.58|830.03|904.73|151.94| +2451489|71962|304|54565|1779342|919|42143|10|127|1276|63|68.18|105.67|0.00|0.00|0.00|4295.34|6657.21|0.00|0.00|0.00|0.00|-4295.34| +2451489|71962|15476|54565|1779342|919|42143|10|282|1276|67|35.91|36.98|9.24|0.00|619.08|2405.97|2477.66|49.52|0.00|619.08|668.60|-1786.89| +2451489|71962|13576|54565|1779342|919|42143|10|61|1276|26|78.14|121.89|98.73|0.00|2566.98|2031.64|3169.14|0.00|0.00|2566.98|2566.98|535.34| +||5959||||42143|10||1276|16||113.15|53.18|0.00|850.88||1810.40|76.57|0.00|850.88||| +2451489|71962|1640|54565|1779342|919|42143|10|128|1276|69|49.83|50.82|31.50|0.00|2173.50|3438.27|3506.58|108.67|0.00|2173.50|2282.17|-1264.77| +2451489|71962|10603|54565|1779342|919|42143|10|189|1276|29|29.13|50.39|23.17|0.00|671.93|844.77|1461.31|6.71|0.00|671.93|678.64|-172.84| +2451489|71962|16774|54565|1779342|919|42143|10|231|1276|68|8.83|13.50|12.01|0.00|816.68|600.44|918.00|65.33|0.00|816.68|882.01|216.24| +2451489|71962|9241|54565|1779342|919|42143|10|184|1276|44|6.95|10.91|6.76|0.00|297.44|305.80|480.04|8.92|0.00|297.44|306.36|-8.36| +2450985|55066|3686|25805|221202|2552|36302|8|107|1277|46|8.08|11.79|1.41|0.00|64.86|371.68|542.34|0.64|0.00|64.86|65.50|-306.82| +2450985|55066|13375|25805|221202|2552|36302|8|49|1277|16|31.37|47.68|4.29|46.67|68.64|501.92|762.88|0.21|46.67|21.97|22.18|-479.95| +2450985|55066|2686|25805|221202|2552|36302|8|249|1277|100|71.72|93.95|49.79|0.00|4979.00|7172.00|9395.00|298.74|0.00|4979.00|5277.74|-2193.00| +2450985|55066|1564|25805|221202|2552|36302|8|68|1277|77|4.49|5.61|5.04|0.00|388.08|345.73|431.97|0.00|0.00|388.08|388.08|42.35| +2450985|55066|3307|25805|221202|2552|36302|8|95|1277|2|27.78|46.39|17.62|0.00|35.24|55.56|92.78|0.70|0.00|35.24|35.94|-20.32| +2450985|55066|8474|25805|221202|2552|36302|8|158|1277|24|74.13|147.51|42.77|0.00|1026.48|1779.12|3540.24|61.58|0.00|1026.48|1088.06|-752.64| +2450985|55066|14092|25805|221202|2552|36302|8|201|1277|52|45.32|90.64|62.54|0.00|3252.08|2356.64|4713.28|0.00|0.00|3252.08|3252.08|895.44| +2450985|55066|9739|25805||||||1277|48|58.66|||||||||662.40|695.52|| +2450985|55066|892|25805|221202|2552|36302|8|104|1277|74|1.72|2.32|0.81|8.99|59.94|127.28|171.68|3.05|8.99|50.95|54.00|-76.33| +2450985|55066|17930|25805|221202|2552|36302|8|6|1277|3|53.01|66.26|49.03|0.00|147.09|159.03|198.78|0.00|0.00|147.09|147.09|-11.94| +2450985|55066|8140|25805|221202|2552|36302|8|100|1277|84|38.89|74.27|70.55|5155.79|5926.20|3266.76|6238.68|30.81|5155.79|770.41|801.22|-2496.35| +2451778|55631|15800|8400|1869962|3171|46333|8|74|1278|65|57.79|104.59|59.61|0.00|3874.65|3756.35|6798.35|0.00|0.00|3874.65|3874.65|118.30| +2451778|55631|12131|8400|1869962|3171|46333|8|132|1278|73|5.21|5.88|2.17|0.00|158.41|380.33|429.24|4.75|0.00|158.41|163.16|-221.92| +2451778|55631|17390|8400|1869962|3171|46333|8|198|1278|79|40.23|72.81|5.82|0.00|459.78|3178.17|5751.99|32.18|0.00|459.78|491.96|-2718.39| +2451778|55631|10079|8400|1869962|3171|46333|8|21|1278|43|48.64|75.39|16.58|0.00|712.94|2091.52|3241.77|28.51|0.00|712.94|741.45|-1378.58| +2451778|55631|4927|8400|1869962|3171|46333|8|256|1278|81|52.63|105.26|23.15|0.00|1875.15|4263.03|8526.06|0.00|0.00|1875.15|1875.15|-2387.88| +2451778|55631|14875|8400|1869962|3171|46333|8|48|1278|67|91.32|140.63|9.84|0.00|659.28|6118.44|9422.21|46.14|0.00|659.28|705.42|-5459.16| +2451778|55631|14186|8400|1869962|3171|46333|8|293|1278|66|42.92|59.22|32.57|0.00|2149.62|2832.72|3908.52|85.98|0.00|2149.62|2235.60|-683.10| +2451778|55631|8726|8400|1869962|3171|46333|8|201|1278|73|60.48|85.88|8.58|0.00|626.34|4415.04|6269.24|56.37|0.00|626.34|682.71|-3788.70| +2451778|55631|607|8400|1869962|3171|46333|8|56|1278|21|36.43|53.55|30.52|0.00|640.92|765.03|1124.55|6.40|0.00|640.92|647.32|-124.11| +2451778|55631|4625|8400|1869962|3171|46333|8|216|1278|19|44.05|77.08|34.68|0.00|658.92|836.95|1464.52|46.12|0.00|658.92|705.04|-178.03| +2451778|55631|10217|8400|1869962|3171|46333|8|187|1278|53|1.17|1.33|0.93|16.26|49.29|62.01|70.49|1.65|16.26|33.03|34.68|-28.98| +2451778|55631|2948|8400|1869962|3171|46333|8|52|1278|63|25.25|49.23|24.61|186.05|1550.43|1590.75|3101.49|40.93|186.05|1364.38|1405.31|-226.37| +2451778|55631|6938|8400|1869962|3171|46333|8|27|1278|26|37.22|71.46|52.88|0.00|1374.88|967.72|1857.96|0.00|0.00|1374.88|1374.88|407.16| +2451778|55631|9905|8400|1869962|3171|46333|8|32|1278|78|33.13|63.27|50.61|0.00|3947.58|2584.14|4935.06|236.85|0.00|3947.58|4184.43|1363.44| +2451778|55631|6953|8400|1869962|3171|46333|8|293|1278|36|28.73|31.02|24.81|0.00|893.16|1034.28|1116.72|8.93|0.00|893.16|902.09|-141.12| +2451101|37950|835|63129|692680|3831|47335|10|179|1279|5|37.47|67.07|48.29|0.00|241.45|187.35|335.35|0.00|0.00|241.45|241.45|54.10| +2451101|37950|13934|63129|692680|3831|47335|10|258|1279|63|57.18|110.92|2.21|0.00|139.23|3602.34|6987.96|0.00|0.00|139.23|139.23|-3463.11| +2451101|37950|15136|63129|692680|3831|47335|10|69|1279|9|83.19|87.34|30.56|0.00|275.04|748.71|786.06|0.00|0.00|275.04|275.04|-473.67| +2451101|37950|8671|63129|692680|3831|47335|10|232|1279|28|39.72|56.40|47.37|0.00|1326.36|1112.16|1579.20|106.10|0.00|1326.36|1432.46|214.20| +2451101|37950|1369|63129|692680|3831|47335|10|296|1279|92|61.57|62.18|49.12|0.00|4519.04|5664.44|5720.56|361.52|0.00|4519.04|4880.56|-1145.40| +2451101|37950|14095|63129|692680|3831|47335|10|84|1279|33|50.28|82.45|9.89|0.00|326.37|1659.24|2720.85|16.31|0.00|326.37|342.68|-1332.87| +2451101|37950|2630|63129|692680|3831|47335|10|230|1279|74|38.27|44.39|0.88|0.00|65.12|2831.98|3284.86|3.25|0.00|65.12|68.37|-2766.86| +2451101|37950|10459|63129|692680|3831|47335|10|85|1279|60|13.73|19.77|1.97|0.00|118.20|823.80|1186.20|4.72|0.00|118.20|122.92|-705.60| +2452628|62330|595|49207|1555498|6589|14080|4|244|1280|89|91.71|129.31|95.68|0.00|8515.52|8162.19|11508.59|170.31|0.00|8515.52|8685.83|353.33| +2452628|62330|17611||1555498||14080|||1280||||25.75|0.00|||2729.50||0.00||1364.75|| +2452628|62330|14383|49207|1555498|6589|14080|4|27|1280|36|21.57|43.14|3.88|33.52|139.68|776.52|1553.04|1.06|33.52|106.16|107.22|-670.36| +2452628|62330|3144|49207|1555498|6589|14080|4|255|1280|7|36.84|46.78|10.75|11.28|75.25|257.88|327.46|3.83|11.28|63.97|67.80|-193.91| +2452628|62330|8838|49207|1555498|6589|14080|4|202|1280|68|26.01|48.89|8.31|0.00|565.08|1768.68|3324.52|39.55|0.00|565.08|604.63|-1203.60| +2452628|62330|531|49207|1555498|6589|14080|4|270|1280|88|19.02|22.06|10.36|0.00|911.68|1673.76|1941.28|72.93|0.00|911.68|984.61|-762.08| +2452628|62330|3102|49207|1555498|6589|14080|4|274|1280|100|32.77|38.01|24.70|0.00|2470.00|3277.00|3801.00|24.70|0.00|2470.00|2494.70|-807.00| +2452628|62330|9073|49207|1555498|6589|14080|4|77|1280|58|48.46|93.52|15.89|0.00|921.62|2810.68|5424.16|55.29|0.00|921.62|976.91|-1889.06| +2452628|62330|4605|49207|1555498|6589|14080|4|205|1280|73|86.91|122.54|109.06|0.00|7961.38|6344.43|8945.42|0.00|0.00|7961.38|7961.38|1616.95| +2452628|62330|15295||1555498|6589|14080|||1280|||||||4798.26|8492.52|292.97||||1061.22| +2452628|62330|6261|49207|1555498|6589|14080|4|266|1280|44|69.41|127.71|114.93|0.00|5056.92|3054.04|5619.24|0.00|0.00|5056.92|5056.92|2002.88| +2452628|62330|12102|49207|1555498|6589|14080|4|143|1280|68|54.67|65.60|16.40|33.45|1115.20|3717.56|4460.80|0.00|33.45|1081.75|1081.75|-2635.81| +2452628|62330|11695|49207|1555498|6589|14080|4|71|1280|26|20.47|30.09|3.61|0.00|93.86|532.22|782.34|8.44|0.00|93.86|102.30|-438.36| +2452628|62330|17523|49207|1555498|6589|14080|4|264|1280|19|95.96|177.52|88.76|0.00|1686.44|1823.24|3372.88|151.77|0.00|1686.44|1838.21|-136.80| +2452628|62330|4321||1555498|6589||||1280|20||37.39|35.52|0.00||530.40|||0.00|||180.00| +2452272|38983|17179|46939|1126544|4196|39401|2|1|1281|74|79.43|92.13|52.51|0.00|3885.74|5877.82|6817.62|0.00|0.00|3885.74|3885.74|-1992.08| +2452272|38983|11795|46939|1126544|4196|39401|2|269|1281|92|20.15|30.82|24.96|0.00|2296.32|1853.80|2835.44|183.70|0.00|2296.32|2480.02|442.52| +|38983|11597|46939||4196||2||1281||18.90|28.72|17.23|0.00||1417.50|2154.00||0.00|1292.25|1395.63|| +2452272|38983|587|46939|1126544|4196|39401|2|247|1281|38|67.60|68.95|53.78|0.00|2043.64|2568.80|2620.10|0.00|0.00|2043.64|2043.64|-525.16| +2452272|38983|15945|46939|1126544|4196|39401|2|48|1281|32|54.86|104.78|27.24|0.00|871.68|1755.52|3352.96|17.43|0.00|871.68|889.11|-883.84| +2452272|38983|11399|46939|1126544|4196|39401|2|216|1281|1|68.85|108.78|68.53|59.62|68.53|68.85|108.78|0.53|59.62|8.91|9.44|-59.94| +2452272|38983|9299|46939|1126544|4196|39401|2|135|1281|33|81.09|151.63|122.82|0.00|4053.06|2675.97|5003.79|40.53|0.00|4053.06|4093.59|1377.09| +2452272|38983|2089|46939|1126544|4196|39401|2|157|1281|98|56.64|63.43|48.20|0.00|4723.60|5550.72|6216.14|0.00|0.00|4723.60|4723.60|-827.12| +2452272|38983|12681|46939|1126544|4196|39401|2|260|1281|32|76.15|149.25|55.22|1095.56|1767.04|2436.80|4776.00|40.28|1095.56|671.48|711.76|-1765.32| +2452272|38983|13833|46939|1126544|4196|39401|2|140|1281|46|25.45|34.61|2.76|0.00|126.96|1170.70|1592.06|11.42|0.00|126.96|138.38|-1043.74| +2452272|38983|8397|46939|1126544|4196|39401|2|266|1281|12|42.47|81.54|54.63|0.00|655.56|509.64|978.48|0.00|0.00|655.56|655.56|145.92| +||6975||1126544||39401||95|1281|||||0.00|445.10||601.50|26.70|0.00|445.10||73.80| +2452272|38983|14567|46939|1126544|4196|39401|2|233|1281|24|33.38|37.05|8.52|0.00|204.48|801.12|889.20|18.40|0.00|204.48|222.88|-596.64| +2452272|38983|6487|46939|1126544|4196|39401|2|87|1281|9|36.02|38.90|32.28|0.00|290.52|324.18|350.10|20.33|0.00|290.52|310.85|-33.66| +2452272|38983|11295|46939|1126544|4196|39401|2|72|1281|27|51.94|55.05|14.31|0.00|386.37|1402.38|1486.35|11.59|0.00|386.37|397.96|-1016.01| +2452272|38983|12057|46939|1126544|4196|39401|2|148|1281|5|87.86|124.76|79.84|0.00|399.20|439.30|623.80|3.99|0.00|399.20|403.19|-40.10| +2452612|53564|9481|33932|131769|806|47090|4|172|1282|36|95.86|126.53|80.97|845.32|2914.92|3450.96|4555.08|0.00|845.32|2069.60|2069.60|-1381.36| +2452612|53564|8607|33932|131769|806|47090|4|37|1282|67|52.28|92.01|0.00|0.00|0.00|3502.76|6164.67|0.00|0.00|0.00|0.00|-3502.76| +2452612|53564|9907|33932|131769|806|47090|4|262|1282|94|55.10|88.71|7.98|0.00|750.12|5179.40|8338.74|60.00|0.00|750.12|810.12|-4429.28| +2452612|53564|6531|33932|131769|806|47090|4|133|1282|32|77.09|89.42|36.66|0.00|1173.12|2466.88|2861.44|70.38|0.00|1173.12|1243.50|-1293.76| +2452612|53564|997|33932|131769|806|47090|4|27|1282|79|7.22|10.18|3.76|0.00|297.04|570.38|804.22|2.97|0.00|297.04|300.01|-273.34| +2452612|53564|420|33932|131769|806|47090|4|114|1282|29|24.21|36.79|29.79|0.00|863.91|702.09|1066.91|77.75|0.00|863.91|941.66|161.82| +2452612|53564|2779|33932|131769|806|47090|4|90|1282|89|16.25|20.31|13.60|0.00|1210.40|1446.25|1807.59|108.93|0.00|1210.40|1319.33|-235.85| +2452612|53564|3709|33932|131769|806|47090|4|249|1282|22|24.53|34.34|9.27|0.00|203.94|539.66|755.48|10.19|0.00|203.94|214.13|-335.72| +2451599|69886|983|14137|885343|6596|27402|4|14|1283|39|69.77|135.35|135.35|0.00|5278.65|2721.03|5278.65|263.93|0.00|5278.65|5542.58|2557.62| +2451599|69886|7748|14137|885343|6596|27402|4|102|1283|45|30.65|32.48|25.33|0.00|1139.85|1379.25|1461.60|79.78|0.00|1139.85|1219.63|-239.40| +2451599|69886|16448|14137|885343|6596|27402|4|277|1283|22|31.85|62.74|25.72|339.50|565.84|700.70|1380.28|9.05|339.50|226.34|235.39|-474.36| +2451599|69886|1693|14137|885343|6596|27402|4|186|1283|95|13.16|16.58|6.63|0.00|629.85|1250.20|1575.10|18.89|0.00|629.85|648.74|-620.35| +2451599|69886|8531|14137|885343|6596|27402|4|81|1283|95|35.85|68.47|49.29|0.00|4682.55|3405.75|6504.65|374.60|0.00|4682.55|5057.15|1276.80| +2451599|69886|10622|14137|885343|6596|27402|4|281|1283|93|3.63|5.26|2.63|119.84|244.59|337.59|489.18|9.98|119.84|124.75|134.73|-212.84| +2451599|69886|4085|14137|885343|6596|27402|4|2|1283|32|56.77|108.99|10.89|83.63|348.48|1816.64|3487.68|7.94|83.63|264.85|272.79|-1551.79| +2451599|69886|4106|14137|885343|6596|27402|4|227|1283|31|23.36|46.25|25.43|0.00|788.33|724.16|1433.75|39.41|0.00|788.33|827.74|64.17| +2451599|69886|6263|14137|885343|6596|27402|4|97|1283|34|15.86|27.75|3.05|0.00|103.70|539.24|943.50|3.11|0.00|103.70|106.81|-435.54| +2451599|69886|2321|14137|885343|6596|27402|4|282|1283|88|33.30|49.28|38.93|0.00|3425.84|2930.40|4336.64|239.80|0.00|3425.84|3665.64|495.44| +2451599|69886|1460|14137|885343|6596|27402|4|295|1283|86|7.94|8.41|2.43|0.00|208.98|682.84|723.26|10.44|0.00|208.98|219.42|-473.86| +2452155|53170|10481|58921|95522|1276|10677|10|217|1284|42|72.06|110.97|99.87|0.00|4194.54|3026.52|4660.74|167.78|0.00|4194.54|4362.32|1168.02| +2452155|53170|10849|58921|95522|1276|10677|10|87|1284|33|33.38|52.74|44.82|0.00|1479.06|1101.54|1740.42|14.79|0.00|1479.06|1493.85|377.52| +2452155|53170|16853|58921|95522|1276|10677|10|39|1284|12|63.99|81.26|1.62|0.00|19.44|767.88|975.12|1.55|0.00|19.44|20.99|-748.44| +2452155|53170|13143|58921|95522|1276|10677|10|123|1284|73|15.98|28.28|21.49|0.00|1568.77|1166.54|2064.44|31.37|0.00|1568.77|1600.14|402.23| +2452155|53170|303|58921|95522|1276|10677|10|273|1284|53|97.00|188.18|120.43|0.00|6382.79|5141.00|9973.54|382.96|0.00|6382.79|6765.75|1241.79| +2452155|53170|4455|58921|95522|1276|10677|10|112|1284|67|24.92|33.14|19.88|1172.12|1331.96|1669.64|2220.38|4.79|1172.12|159.84|164.63|-1509.80| +2452155|53170|7045|58921|95522|1276|10677|10|163|1284|50|62.27|82.81|23.18|0.00|1159.00|3113.50|4140.50|69.54|0.00|1159.00|1228.54|-1954.50| +2452155|53170|12273|58921|95522|1276|10677|10|227|1284|1|18.17|21.07|9.69|0.00|9.69|18.17|21.07|0.19|0.00|9.69|9.88|-8.48| +2452155|53170|3263|58921|95522|1276|10677|10|276|1284|98|81.87|94.15|26.36|1239.97|2583.28|8023.26|9226.70|120.89|1239.97|1343.31|1464.20|-6679.95| +2452155|53170|2193|58921|95522|1276|10677|10|59|1284|71|3.20|6.17|0.37|0.00|26.27|227.20|438.07|1.31|0.00|26.27|27.58|-200.93| +2452155|53170|10199|58921|95522|1276|10677|10|226|1284|94|23.53|28.00|5.04|350.58|473.76|2211.82|2632.00|8.62|350.58|123.18|131.80|-2088.64| +2451876|35331|11426|77662|537127|592|40504|8|225|1285|63|71.17|103.90|25.97|0.00|1636.11|4483.71|6545.70|65.44|0.00|1636.11|1701.55|-2847.60| +2451876|35331|15715|77662|537127|592|40504|8|30|1285|7|38.81|40.36|21.39|0.00|149.73|271.67|282.52|1.49|0.00|149.73|151.22|-121.94| +2451876|35331|8101|77662|537127|592|40504|8|141|1285|19|47.76|89.78|38.60|0.00|733.40|907.44|1705.82|66.00|0.00|733.40|799.40|-174.04| +2451876|35331|8024|77662|537127|592|40504|8|79|1285|28|27.13|36.62|32.95|618.14|922.60|759.64|1025.36|21.31|618.14|304.46|325.77|-455.18| +2451876|35331|8605|77662|537127|592|40504|8|211|1285|86|21.11|37.78|9.82|413.81|844.52|1815.46|3249.08|17.22|413.81|430.71|447.93|-1384.75| +2451876|35331|2888|77662|537127|592|40504|8|98|1285|63|38.39|61.80|2.47|115.15|155.61|2418.57|3893.40|0.80|115.15|40.46|41.26|-2378.11| +2451876|35331|13922|77662|537127||||129|1285|6|89.76||||475.44||586.98||||14.55|| +2451876|35331|14360|77662|537127|592|40504|8|243|1285|61|2.71|3.06|2.47|0.00|150.67|165.31|186.66|4.52|0.00|150.67|155.19|-14.64| +2451876|35331|4675|77662|537127|592|40504|8|156|1285|38|92.55|124.94|22.48|0.00|854.24|3516.90|4747.72|34.16|0.00|854.24|888.40|-2662.66| +2451876|35331|6350|77662|537127|592|40504|8|258|1285|7|45.68|64.86|50.59|0.00|354.13|319.76|454.02|21.24|0.00|354.13|375.37|34.37| +2451876|35331|5879|77662|537127|592|40504|8|100|1285|89|39.13|69.26|27.70|1947.58|2465.30|3482.57|6164.14|31.06|1947.58|517.72|548.78|-2964.85| +2451876|35331|16897|77662|537127|592|40504|8|290|1285|74|65.63|77.44|51.88|0.00|3839.12|4856.62|5730.56|76.78|0.00|3839.12|3915.90|-1017.50| +2451876|35331|10655|77662|537127|592|40504|8|182|1285|86|4.64|8.30|0.00|0.00|0.00|399.04|713.80|0.00|0.00|0.00|0.00|-399.04| +2452215|65824|2587|84401|339283|6965|9800|7|231|1286|85|24.67|26.64|7.99|0.00|679.15|2096.95|2264.40|40.74|0.00|679.15|719.89|-1417.80| +2452215|65824|10383|84401|339283|6965|9800|7|127|1286|75|76.11|146.13|94.98|0.00|7123.50|5708.25|10959.75|0.00|0.00|7123.50|7123.50|1415.25| +2452215|65824|4203|84401|339283|6965|9800|7|199|1286|77|38.34|49.07|18.64|0.00|1435.28|2952.18|3778.39|100.46|0.00|1435.28|1535.74|-1516.90| +2452215|65824|3497|84401|339283|6965|9800|7|264|1286|62|87.12|94.08|27.28|0.00|1691.36|5401.44|5832.96|0.00|0.00|1691.36|1691.36|-3710.08| +2452215|65824|4183|84401|339283|6965|9800|7|287|1286|32|84.60|109.98|24.19|0.00|774.08|2707.20|3519.36|7.74|0.00|774.08|781.82|-1933.12| +2452215|65824|13427|84401|339283|6965|9800|7|264|1286|25|66.31|101.45|29.42|0.00|735.50|1657.75|2536.25|0.00|0.00|735.50|735.50|-922.25| +2452215|65824|665|84401|339283|6965|9800|7|55|1286|61|94.02|152.31|47.21|0.00|2879.81|5735.22|9290.91|143.99|0.00|2879.81|3023.80|-2855.41| +2452215|65824|11893|84401|339283|6965|9800|7|247|1286|57|50.33|89.08|69.48|0.00|3960.36|2868.81|5077.56|277.22|0.00|3960.36|4237.58|1091.55| +2452215|65824|11365|84401|339283|6965|9800|7|191|1286|82|99.33|116.21|90.64|2304.06|7432.48|8145.06|9529.22|461.55|2304.06|5128.42|5589.97|-3016.64| +2452215|65824|7115|84401|339283|6965|9800|7|198|1286|56|21.95|22.38|22.38|0.00|1253.28|1229.20|1253.28|75.19|0.00|1253.28|1328.47|24.08| +2452215|65824|12041|84401|339283|6965|9800|7|271|1286|5|66.08|122.24|24.44|46.43|122.20|330.40|611.20|3.78|46.43|75.77|79.55|-254.63| +2452215|65824|5929|84401|339283|6965|9800|7|140|1286|42|53.83|92.58|30.55|0.00|1283.10|2260.86|3888.36|38.49|0.00|1283.10|1321.59|-977.76| +2452215|65824|16671|84401|339283|6965|9800|7|96|1286|78|59.63|91.83|90.91|4963.68|7090.98|4651.14|7162.74|0.00|4963.68|2127.30|2127.30|-2523.84| +2452225|40164|15407|45516|66768|2773|45225|4|107|1287|39|56.82|89.77|70.91|0.00|2765.49|2215.98|3501.03|221.23|0.00|2765.49|2986.72|549.51| +2452225|40164|6161|45516|66768|2773|45225|4|288|1287|22|23.01|39.57|37.59|0.00|826.98|506.22|870.54|0.00|0.00|826.98|826.98|320.76| +2452225|40164|10351|45516|66768|2773|45225|4|196|1287|49|40.64|67.05|18.77|0.00|919.73|1991.36|3285.45|55.18|0.00|919.73|974.91|-1071.63| +|40164|14735||66768|||4||1287|2|18.97|28.45|9.38||18.76|37.94||||18.76|18.76|-19.18| +2452225|40164|5659|45516|66768|2773|45225|4|271|1287|7|55.48|57.14|29.14|0.00|203.98|388.36|399.98|10.19|0.00|203.98|214.17|-184.38| +|40164|8437|45516|66768|2773|45225|4||1287|||97.97||0.00|||||0.00|1090.32||| +2452225|40164|11105|45516|66768|2773|45225|4|191|1287|100|84.17|127.93|51.17|0.00|5117.00|8417.00|12793.00|255.85|0.00|5117.00|5372.85|-3300.00| +2452225|40164|17369|45516|66768|2773|45225|4|71|1287|24|70.35|135.07|22.96|0.00|551.04|1688.40|3241.68|44.08|0.00|551.04|595.12|-1137.36| +2452225|40164|13157|45516|66768|2773|45225|4|93|1287|22|50.38|88.66|22.16|0.00|487.52|1108.36|1950.52|34.12|0.00|487.52|521.64|-620.84| +2452225|40164|14265|45516|66768|2773|45225|4|255|1287|56|18.78|30.23|19.95|0.00|1117.20|1051.68|1692.88|11.17|0.00|1117.20|1128.37|65.52| +2452225|40164|11361|45516|66768|2773|45225|4|158|1287|69|37.22|68.48|30.13|0.00|2078.97|2568.18|4725.12|124.73|0.00|2078.97|2203.70|-489.21| +2452225|40164|7595|45516|66768|2773|45225|4|235|1287|70|57.24|90.43|69.63|0.00|4874.10|4006.80|6330.10|438.66|0.00|4874.10|5312.76|867.30| +2452207|39455|17389|88398|1072568|385|24602|4|90|1288|32|79.89|85.48|74.36|0.00|2379.52|2556.48|2735.36|190.36|0.00|2379.52|2569.88|-176.96| +2452207|39455|12609|88398|1072568|385|24602|4|67|1288|50|59.70|63.87|4.47|0.00|223.50|2985.00|3193.50|8.94|0.00|223.50|232.44|-2761.50| +2452207|39455|6423|88398|1072568|385|24602|4|145|1288|5|41.41|80.33|66.67|0.00|333.35|207.05|401.65|26.66|0.00|333.35|360.01|126.30| +2452207|39455|1179|88398|1072568|385|24602|4|255|1288|34|85.88|168.32|25.24|0.00|858.16|2919.92|5722.88|17.16|0.00|858.16|875.32|-2061.76| +2452207|39455|10529|88398|1072568|385|24602|4|95|1288|18|67.42|97.75|76.24|0.00|1372.32|1213.56|1759.50|27.44|0.00|1372.32|1399.76|158.76| +2452207|39455|2815|88398|1072568|385|24602|4|288|1288|71|55.79|81.45|74.93|0.00|5320.03|3961.09|5782.95|106.40|0.00|5320.03|5426.43|1358.94| +2452207|39455|5157|88398|1072568|385|24602|4|47|1288|22|21.18|39.60|30.49|194.52|670.78|465.96|871.20|38.10|194.52|476.26|514.36|10.30| +2452207|39455|14565|88398|1072568|385|24602|4|113|1288|24|95.92|114.14|5.70|0.00|136.80|2302.08|2739.36|10.94|0.00|136.80|147.74|-2165.28| +2452207|39455|15249|88398|1072568|385|24602|4|169|1288|75|48.62|90.91|53.63|0.00|4022.25|3646.50|6818.25|201.11|0.00|4022.25|4223.36|375.75| +2452207|39455|5579|88398|1072568|385|24602|4|4|1288|96|21.73|38.02|4.56|0.00|437.76|2086.08|3649.92|35.02|0.00|437.76|472.78|-1648.32| +2452207|39455|13669|88398|1072568|385|24602|4|64|1288|95|31.81|32.44|24.65|1873.40|2341.75|3021.95|3081.80|14.05|1873.40|468.35|482.40|-2553.60| +2452207|39455|7441|88398|1072568|385|24602|4|138|1288|71|15.42|23.59|6.13|0.00|435.23|1094.82|1674.89|30.46|0.00|435.23|465.69|-659.59| +2452092|46483|10409|28775|291223|518|37366|2|248|1289|93|79.15|123.47|88.89|0.00|8266.77|7360.95|11482.71|578.67|0.00|8266.77|8845.44|905.82| +2452092|46483|9793|28775|291223|518|37366|2|62|1289|26|37.86|70.41|10.56|0.00|274.56|984.36|1830.66|0.00|0.00|274.56|274.56|-709.80| +2452092|46483|9373|28775|291223|518|37366|2|57|1289|80|67.25|102.22|18.39|0.00|1471.20|5380.00|8177.60|88.27|0.00|1471.20|1559.47|-3908.80| +2452092|46483|2975|28775|291223|518|37366|2|194|1289|44|84.33|135.77|5.43|0.00|238.92|3710.52|5973.88|4.77|0.00|238.92|243.69|-3471.60| +2452092|46483|5203|28775|291223|518|37366|2|252|1289|27|14.71|23.24|23.00|0.00|621.00|397.17|627.48|0.00|0.00|621.00|621.00|223.83| +2452092|46483|1055|28775|291223|518|37366|2|75|1289|98|95.41|163.15|29.36|0.00|2877.28|9350.18|15988.70|172.63|0.00|2877.28|3049.91|-6472.90| +2452092|46483|11275|28775|291223|518|37366|2|79|1289|53|73.72|123.11|62.78|0.00|3327.34|3907.16|6524.83|66.54|0.00|3327.34|3393.88|-579.82| +2452092|46483|11775|28775|291223|518|37366|2|118|1289|53|57.64|73.20|42.45|0.00|2249.85|3054.92|3879.60|89.99|0.00|2249.85|2339.84|-805.07| +2452092|46483|9399|28775|291223|518|37366|2|173|1289|34|47.43|72.09|23.06|603.71|784.04|1612.62|2451.06|12.62|603.71|180.33|192.95|-1432.29| +2452092|46483|3969|28775|291223|518|37366|2|66|1289|68|29.92|56.84|2.27|0.00|154.36|2034.56|3865.12|4.63|0.00|154.36|158.99|-1880.20| +2452092|46483|6639|28775|291223|518|37366|2|204|1289|56|46.21|49.90|21.95|553.14|1229.20|2587.76|2794.40|47.32|553.14|676.06|723.38|-1911.70| +2452092|46483|6841|28775|291223|518|37366|2|58|1289|50|86.48|128.85|128.85|0.00|6442.50|4324.00|6442.50|193.27|0.00|6442.50|6635.77|2118.50| +2452092|46483|10501|28775|291223|518|37366|2|14|1289|20|70.00|130.20|108.06|0.00|2161.20|1400.00|2604.00|0.00|0.00|2161.20|2161.20|761.20| +2452069|29790|16947|847|325340|4233|21926|7|124|1290|66|17.98|28.76|12.07|0.00|796.62|1186.68|1898.16|63.72|0.00|796.62|860.34|-390.06| +2452069|29790|12155|847|325340|4233|21926|7|138|1290|94|38.13|45.37|39.92|788.02|3752.48|3584.22|4264.78|266.80|788.02|2964.46|3231.26|-619.76| +2452069|29790|4033|847|325340|4233|21926|7|172|1290|93|50.97|76.45|34.40|0.00|3199.20|4740.21|7109.85|63.98|0.00|3199.20|3263.18|-1541.01| +2452069|29790|17259|847|325340|4233|21926|7|187|1290|32|28.54|50.23|37.67|602.72|1205.44|913.28|1607.36|6.02|602.72|602.72|608.74|-310.56| +2452069|29790|2307|847|325340|4233|21926|7|113|1290|100|84.44|168.03|72.25|0.00|7225.00|8444.00|16803.00|0.00|0.00|7225.00|7225.00|-1219.00| +2452069|29790|10729|847|325340|4233|21926|7|68|1290|5|32.63|38.17|18.32|0.00|91.60|163.15|190.85|6.41|0.00|91.60|98.01|-71.55| +2452069|29790|10405|847|325340|4233|21926|7|45|1290|94|53.77|107.54|104.31|0.00|9805.14|5054.38|10108.76|98.05|0.00|9805.14|9903.19|4750.76| +2452069|29790|5743|847|325340|4233|21926|7|39|1290|10|59.09|101.63|12.19|0.00|121.90|590.90|1016.30|8.53|0.00|121.90|130.43|-469.00| +2452069|29790|13899|847|325340|4233|21926|7|3|1290|81|15.42|26.98|23.74|0.00|1922.94|1249.02|2185.38|96.14|0.00|1922.94|2019.08|673.92| +2452069|29790|8321|847|325340|4233|21926|7|100|1290|52|45.99|71.28|34.92|0.00|1815.84|2391.48|3706.56|108.95|0.00|1815.84|1924.79|-575.64| +2452069|29790|435|847|325340|4233|21926|7|39|1290|86|38.34|75.14|51.84|0.00|4458.24|3297.24|6462.04|178.32|0.00|4458.24|4636.56|1161.00| +2452610|62315|17419|83869|1434661|5453|49045|8|282|1291|19|86.20|108.61|39.09|0.00|742.71|1637.80|2063.59|37.13|0.00|742.71|779.84|-895.09| +2452610|62315|13201|83869|1434661|5453|49045|8|296|1291|9|27.60|33.12|14.24|0.00|128.16|248.40|298.08|2.56|0.00|128.16|130.72|-120.24| +2452610|62315|727|83869|1434661|5453|49045|8|44|1291|71|42.87|78.88|72.56|4378.99|5151.76|3043.77|5600.48|46.36|4378.99|772.77|819.13|-2271.00| +2452610|62315|11809|83869|1434661|5453|49045|8|178|1291|48|18.13|36.26|35.17|641.50|1688.16|870.24|1740.48|41.86|641.50|1046.66|1088.52|176.42| +2452610|62315|2065|83869|1434661|5453|49045|8|157|1291|67|88.63|107.24|80.43|0.00|5388.81|5938.21|7185.08|53.88|0.00|5388.81|5442.69|-549.40| +2452610|62315|8493|83869|1434661|5453|49045|8|248|1291|30|86.43|151.25|45.37|0.00|1361.10|2592.90|4537.50|108.88|0.00|1361.10|1469.98|-1231.80| +2452610|62315|3663|83869|1434661|5453|49045|8|254|1291|57|9.92|19.44|7.58|0.00|432.06|565.44|1108.08|0.00|0.00|432.06|432.06|-133.38| +2452610|62315|1453|83869|1434661|5453|49045|8|2|1291|10|39.01|65.14|46.24|0.00|462.40|390.10|651.40|27.74|0.00|462.40|490.14|72.30| +2452610|62315|9613|83869|1434661|5453|49045|8|76|1291|23|15.85|25.51|0.76|2.09|17.48|364.55|586.73|1.07|2.09|15.39|16.46|-349.16| +2450899|50993|9277|25617|996828|392|7956|2|94|1292|31|74.32|121.14|44.82|0.00|1389.42|2303.92|3755.34|55.57|0.00|1389.42|1444.99|-914.50| +2450899|50993|12794|25617|996828|392|7956|2|23|1292|31|77.02|128.62|14.14|0.00|438.34|2387.62|3987.22|13.15|0.00|438.34|451.49|-1949.28| +2450899|50993|578|25617|996828|392|7956|2|129|1292|41|18.67|30.24|4.23|0.00|173.43|765.47|1239.84|12.14|0.00|173.43|185.57|-592.04| +||55||||7956|2||1292|46|35.59|59.43||0.00|||||0.00||2281.34|| +2450899|50993|12943|25617|996828|392|7956|2|58|1292|49|78.25|123.63|37.08|0.00|1816.92|3834.25|6057.87|163.52|0.00|1816.92|1980.44|-2017.33| +2450899|50993|11302|25617|996828|392|7956|2|250|1292|47|18.93|32.37|17.47|0.00|821.09|889.71|1521.39|24.63|0.00|821.09|845.72|-68.62| +2450899|50993|16438|25617|996828|392|7956|2|54|1292|24|96.24|183.81|150.72|0.00|3617.28|2309.76|4411.44|325.55|0.00|3617.28|3942.83|1307.52| +2450899|50993|16532|25617|996828|392|7956|2|158|1292|39|95.46|173.73|86.86|0.00|3387.54|3722.94|6775.47|101.62|0.00|3387.54|3489.16|-335.40| +2450899|50993|13699|25617|996828|392|7956|2|60|1292|67|23.91|30.60|13.77|0.00|922.59|1601.97|2050.20|0.00|0.00|922.59|922.59|-679.38| +2450899|50993|5714|25617|996828|392|7956|2|130|1292|12|89.49|131.55|119.71|0.00|1436.52|1073.88|1578.60|114.92|0.00|1436.52|1551.44|362.64| +2450899|50993|8750|25617|996828|392|7956|2|267|1292|60|37.64|71.13|11.38|0.00|682.80|2258.40|4267.80|20.48|0.00|682.80|703.28|-1575.60| +2450899|50993|4987|25617|996828|392|7956|2|48|1292|54|38.34|72.46|23.18|0.00|1251.72|2070.36|3912.84|100.13|0.00|1251.72|1351.85|-818.64| +2450899|50993|26|25617|996828|392|7956|2|157|1292|66|19.66|23.59|11.08|0.00|731.28|1297.56|1556.94|51.18|0.00|731.28|782.46|-566.28| +2450899|50993|14194||996828||7956|2||1292|6|21.47||12.86||77.16|||6.94||77.16||-51.66| +|50993|6700||||||82|1292|63|56.53|89.88|||4416.30||5662.44|||||| +2451489|69879|15079|76510|1515488|6113|15138|4|7|1293|36|24.63|31.77|30.18|0.00|1086.48|886.68|1143.72|76.05|0.00|1086.48|1162.53|199.80| +2451489|69879|5908|76510|1515488|6113|15138|4|287|1293|89|55.83|72.02|3.60|198.64|320.40|4968.87|6409.78|8.52|198.64|121.76|130.28|-4847.11| +2451489|69879|12004|76510|1515488|6113|15138|4|260|1293|96|11.31|18.88|0.18|0.00|17.28|1085.76|1812.48|0.51|0.00|17.28|17.79|-1068.48| +2451489|69879|79|76510|1515488|6113|15138|4|116|1293|2|56.17|59.54|57.75|0.00|115.50|112.34|119.08|2.31|0.00|115.50|117.81|3.16| +2451489|69879|4636|76510|1515488|6113|15138|4|49|1293|97|21.92|39.45|24.06|0.00|2333.82|2126.24|3826.65|23.33|0.00|2333.82|2357.15|207.58| +2451489|69879|7240|76510|1515488|6113|15138|4|18|1293|89|38.89|68.05|7.48|499.29|665.72|3461.21|6056.45|1.66|499.29|166.43|168.09|-3294.78| +2451489|69879|11257|76510|1515488|6113|15138|4|258|1293|91|15.66|17.85|14.63|0.00|1331.33|1425.06|1624.35|106.50|0.00|1331.33|1437.83|-93.73| +2451489|69879|12397|76510|1515488|6113|15138|4|273|1293|92|72.09|119.66|44.27|0.00|4072.84|6632.28|11008.72|203.64|0.00|4072.84|4276.48|-2559.44| +2451489|69879|9092|76510|1515488|6113|15138|4|103|1293|93|99.53|184.13|14.73|0.00|1369.89|9256.29|17124.09|123.29|0.00|1369.89|1493.18|-7886.40| +2451489|69879|15403|76510|1515488|6113|15138|4|260|1293|51|91.27|115.00|0.00|0.00|0.00|4654.77|5865.00|0.00|0.00|0.00|0.00|-4654.77| +2451489|69879|8180|76510|1515488|6113|15138|4|282|1293|38|54.41|74.54|55.90|0.00|2124.20|2067.58|2832.52|0.00|0.00|2124.20|2124.20|56.62| +2451489|69879|17630|76510|1515488|6113|15138|4|168|1293|25|69.97|133.64|64.14|0.00|1603.50|1749.25|3341.00|128.28|0.00|1603.50|1731.78|-145.75| +2451489|69879|319|76510|1515488|6113|15138|4|35|1293|57|13.38|22.21|18.43|0.00|1050.51|762.66|1265.97|84.04|0.00|1050.51|1134.55|287.85| +2451462|63106|17464|57602|359934|3866|30998|10|110|1294|72|20.50|39.56|12.26|0.00|882.72|1476.00|2848.32|8.82|0.00|882.72|891.54|-593.28| +2451462|63106|15949|57602|359934|3866|30998|10|15|1294|73|23.48|43.67|17.03|0.00|1243.19|1714.04|3187.91|74.59|0.00|1243.19|1317.78|-470.85| +2451462|63106|11071|57602|359934|3866|30998|10|293|1294|96|16.99|16.99|7.13|0.00|684.48|1631.04|1631.04|13.68|0.00|684.48|698.16|-946.56| +2451462|63106|11434|57602|359934|3866|30998|10|161|1294|21|68.76|99.01|91.08|0.00|1912.68|1443.96|2079.21|114.76|0.00|1912.68|2027.44|468.72| +2451462|63106|5239|57602|359934|3866|30998|10|150|1294|85|38.89|60.27|55.44|0.00|4712.40|3305.65|5122.95|47.12|0.00|4712.40|4759.52|1406.75| +2451462|63106|11786|57602|359934|3866|30998|10|169|1294|83|37.15|51.26|50.74|0.00|4211.42|3083.45|4254.58|379.02|0.00|4211.42|4590.44|1127.97| +2451462|63106|11998||359934|||10||1294||5.86||||74.00||619.38|3.70||74.00||-359.64| +2451462|63106|9697|57602|359934|3866|30998|10|202|1294|39|38.88|44.71|4.47|0.00|174.33|1516.32|1743.69|15.68|0.00|174.33|190.01|-1341.99| +2451503|35360|14923|29754|1205455|2403|30504|8|210|1295|77|54.89|94.95|75.01|0.00|5775.77|4226.53|7311.15|288.78|0.00|5775.77|6064.55|1549.24| +2451503|35360|12205|29754|1205455|2403|30504|8|142|1295|79|44.73|63.51|0.63|0.00|49.77|3533.67|5017.29|3.48|0.00|49.77|53.25|-3483.90| +2451503|35360|12170|29754|1205455|2403|30504|8|187|1295|45|36.44|40.08|7.61|0.00|342.45|1639.80|1803.60|23.97|0.00|342.45|366.42|-1297.35| +2451503|35360|5137|29754|1205455|2403|30504|8|237|1295|78|79.22|141.80|123.36|0.00|9622.08|6179.16|11060.40|192.44|0.00|9622.08|9814.52|3442.92| +2451503|35360|3001|29754|1205455|2403|30504|8|211|1295|92|42.29|43.13|2.15|0.00|197.80|3890.68|3967.96|7.91|0.00|197.80|205.71|-3692.88| +2451503|35360|9703|29754|1205455|2403|30504|8|260|1295|76|46.17|72.02|36.73|0.00|2791.48|3508.92|5473.52|27.91|0.00|2791.48|2819.39|-717.44| +2451503|35360|14134|29754||2403||||1295||||51.28||2102.48|||105.12||||| +2451503|35360|17677|29754|1205455|2403|30504|8|74|1295|16|67.33|91.56|47.61|0.00|761.76|1077.28|1464.96|38.08|0.00|761.76|799.84|-315.52| +2451503|35360|5806|29754|1205455|2403|30504|8|14|1295|90|10.21|15.72|14.61|0.00|1314.90|918.90|1414.80|92.04|0.00|1314.90|1406.94|396.00| +2451503|35360|10232|29754|1205455|2403|30504|8|104|1295|64|75.93|79.72|27.90|0.00|1785.60|4859.52|5102.08|160.70|0.00|1785.60|1946.30|-3073.92| +2451503|35360|15145|29754|1205455|2403|30504|8|157|1295|85|87.25|116.91|108.72|0.00|9241.20|7416.25|9937.35|0.00|0.00|9241.20|9241.20|1824.95| +2451503|35360|12793|29754|1205455|2403|30504|8|249|1295|85|27.19|46.49|34.86|888.93|2963.10|2311.15|3951.65|20.74|888.93|2074.17|2094.91|-236.98| +2451503|35360|715|29754|1205455|2403|30504|8|240|1295|57|85.48|154.71|137.69|4395.06|7848.33|4872.36|8818.47|34.53|4395.06|3453.27|3487.80|-1419.09| +2451503|35360|16090|29754|1205455|2403|30504|8|162|1295|89|33.25|48.87|27.36|0.00|2435.04|2959.25|4349.43|48.70|0.00|2435.04|2483.74|-524.21| +2451503|35360|1600|29754|1205455|2403|30504|8|151|1295|12|26.74|51.87|29.04|0.00|348.48|320.88|622.44|13.93|0.00|348.48|362.41|27.60| +2451126|32838|9505|41472|172002|6885|18615|7|11|1296|76|58.48|110.52|100.57|0.00|7643.32|4444.48|8399.52|611.46|0.00|7643.32|8254.78|3198.84| +2451126|32838|4396|41472|172002|6885|18615|7|50|1296|80|83.80|150.84|67.87|0.00|5429.60|6704.00|12067.20|162.88|0.00|5429.60|5592.48|-1274.40| +2451126|32838|688|41472|172002|6885|18615|7|123|1296|92|57.47|79.88|39.14|0.00|3600.88|5287.24|7348.96|0.00|0.00|3600.88|3600.88|-1686.36| +2451126|32838|1195|41472|172002|6885|18615|7|84|1296|87|82.62|95.83|90.08|0.00|7836.96|7187.94|8337.21|391.84|0.00|7836.96|8228.80|649.02| +2451126|32838|7495|41472|172002|6885|18615|7|289|1296|8|88.23|95.28|70.50|0.00|564.00|705.84|762.24|16.92|0.00|564.00|580.92|-141.84| +2451126|32838|1238|41472|172002|6885|18615|7|49|1296|49|16.46|30.78|20.00|0.00|980.00|806.54|1508.22|88.20|0.00|980.00|1068.20|173.46| +2451126||10084|41472|||18615||94|1296||||30.18|0.00||6224.00|12074.24||0.00|1931.52|1950.83|| +2451126|32838|14260|41472|172002|6885|18615|7|75|1296|99|38.43|63.02|18.90|0.00|1871.10|3804.57|6238.98|37.42|0.00|1871.10|1908.52|-1933.47| +2451126|32838|4978|41472|172002|6885|18615|7|89|1296|98|99.01|153.46|41.43|121.80|4060.14|9702.98|15039.08|354.45|121.80|3938.34|4292.79|-5764.64| +2451126|32838|6698|41472|172002|6885|18615|7|68|1296|19|28.40|30.95|25.99|0.00|493.81|539.60|588.05|0.00|0.00|493.81|493.81|-45.79| +2451126|32838|640|41472|172002|6885|18615|7|138|1296|33|22.51|30.16|11.15|0.00|367.95|742.83|995.28|3.67|0.00|367.95|371.62|-374.88| +2451126|32838|15290|41472|172002|6885|18615|7|86|1296|73|66.01|87.13|43.56|0.00|3179.88|4818.73|6360.49|254.39|0.00|3179.88|3434.27|-1638.85| +2451126|32838|9488|41472|172002|6885|18615|7|84|1296|12|64.77|71.24|70.52|787.00|846.24|777.24|854.88|4.73|787.00|59.24|63.97|-718.00| +2451526|34758|8866|38925|820339|2200|46861|4|52|1297|21|25.53|33.69|10.78|0.00|226.38|536.13|707.49|4.52|0.00|226.38|230.90|-309.75| +2451526|34758|12385|38925|820339|2200|46861|4|218|1297|4|25.72|25.72|23.14|0.00|92.56|102.88|102.88|8.33|0.00|92.56|100.89|-10.32| +2451526|34758|14929|38925|820339|2200|46861|4|106|1297|24|64.44|64.44|34.79|0.00|834.96|1546.56|1546.56|50.09|0.00|834.96|885.05|-711.60| +2451526|34758|6880|38925|820339|2200|46861|4|221|1297|76|75.21|116.57|38.46|0.00|2922.96|5715.96|8859.32|29.22|0.00|2922.96|2952.18|-2793.00| +2451526|34758|17287|38925|820339|2200|46861|4|83|1297|33|5.30|8.21|1.39|0.00|45.87|174.90|270.93|1.37|0.00|45.87|47.24|-129.03| +2451526|34758|9308|38925|820339|2200|46861|4|133|1297|59|94.95|128.18|87.16|977.06|5142.44|5602.05|7562.62|83.30|977.06|4165.38|4248.68|-1436.67| +2451526|34758|1042|38925|820339|2200|46861|4|282|1297|87|52.71|100.14|86.12|0.00|7492.44|4585.77|8712.18|374.62|0.00|7492.44|7867.06|2906.67| +2451526|34758|15958|38925|820339|2200|46861|4|13|1297|28|14.49|17.53|6.66|33.56|186.48|405.72|490.84|6.11|33.56|152.92|159.03|-252.80| +2451526|34758|3442|38925|820339|2200|46861|4|156|1297|26|64.52|106.45|43.64|0.00|1134.64|1677.52|2767.70|45.38|0.00|1134.64|1180.02|-542.88| +2451526|34758|16465|38925||||4||1297|85||||0.00|1440.75||||0.00|1440.75||| +2451526|34758|5134|38925|820339|2200|46861|4|91|1297|86|49.93|70.90|12.05|0.00|1036.30|4293.98|6097.40|62.17|0.00|1036.30|1098.47|-3257.68| +2451526|34758|4702|38925|820339|2200|46861|4|291|1297|53|74.70|109.06|45.80|485.48|2427.40|3959.10|5780.18|0.00|485.48|1941.92|1941.92|-2017.18| +2451526|34758|4580|38925|820339|2200|46861|4|191|1297|4|93.29|154.86|100.65|0.00|402.60|373.16|619.44|4.02|0.00|402.60|406.62|29.44| +2451526|34758|10040|38925|820339|2200|46861|4|300|1297|65|6.76|10.07|6.74|0.00|438.10|439.40|654.55|26.28|0.00|438.10|464.38|-1.30| +2451526||12140|38925||||4|39|1297|62|5.15||||||383.16||||336.26|| +2451526|34758|3068|38925|820339|2200|46861|4|248|1297|84|50.18|64.73|33.01|0.00|2772.84|4215.12|5437.32|55.45|0.00|2772.84|2828.29|-1442.28| +2452619|32417|9054|21547|1207458|5369|32927|7|162|1298|11|64.66|77.59|68.27|0.00|750.97|711.26|853.49|0.00|0.00|750.97|750.97|39.71| +2452619|32417|14787|21547|1207458|5369|32927|7|170|1298|99|93.79|129.43|18.12|0.00|1793.88|9285.21|12813.57|35.87|0.00|1793.88|1829.75|-7491.33| +2452619|32417|5953|21547|1207458|5369|32927|7|148|1298|75|37.51|70.51|5.64|0.00|423.00|2813.25|5288.25|16.92|0.00|423.00|439.92|-2390.25| +2452619|32417|9085|21547|1207458|5369|32927|7|273|1298|50|42.03|74.81|15.71|0.00|785.50|2101.50|3740.50|31.42|0.00|785.50|816.92|-1316.00| +2452619|32417|4797|21547|1207458|5369|32927|7|4|1298|19|55.07|102.43|58.38|643.34|1109.22|1046.33|1946.17|32.61|643.34|465.88|498.49|-580.45| +2452619|32417|1249|21547|1207458|5369|32927|7|286|1298|92|95.31|188.71|181.16|0.00|16666.72|8768.52|17361.32|0.00|0.00|16666.72|16666.72|7898.20| +2452619|32417|5562|21547|1207458|5369|32927|7|142|1298|40|57.76|103.96|60.29|0.00|2411.60|2310.40|4158.40|144.69|0.00|2411.60|2556.29|101.20| +2452619|32417|846|21547|1207458|5369|32927|7|91|1298|60|81.62|136.30|32.71|1766.34|1962.60|4897.20|8178.00|7.85|1766.34|196.26|204.11|-4700.94| +2452619|32417|12883|21547|1207458|5369|32927|7|93|1298|36|65.15|65.15|43.65|0.00|1571.40|2345.40|2345.40|125.71|0.00|1571.40|1697.11|-774.00| +2452619|32417|3999|21547|1207458|5369|32927|7|185|1298|57|79.51|93.82|72.24|0.00|4117.68|4532.07|5347.74|41.17|0.00|4117.68|4158.85|-414.39| +2452619|32417|14607|21547|1207458|5369|32927|7|120|1298|10|40.01|44.01|18.92|0.00|189.20|400.10|440.10|1.89|0.00|189.20|191.09|-210.90| +2452619|32417|8175|21547|1207458|5369|32927|7|157|1298|18|4.85|5.43|0.81|0.00|14.58|87.30|97.74|0.00|0.00|14.58|14.58|-72.72| +2452619|32417|7453|21547|1207458|5369|32927|7|49|1298|79|66.58|79.89|55.12|0.00|4354.48|5259.82|6311.31|0.00|0.00|4354.48|4354.48|-905.34| +2452619|32417|15193|21547|1207458|5369|32927|7|88|1298|33|55.48|59.91|0.59|0.00|19.47|1830.84|1977.03|0.00|0.00|19.47|19.47|-1811.37| +2452619|32417|3087|21547|1207458|5369|32927|7|107|1298|78|41.40|52.57|12.61|954.07|983.58|3229.20|4100.46|2.65|954.07|29.51|32.16|-3199.69| +2452046|66252|11151|82992|1169268|1944|22973|1|35|1299|47|61.95|99.12|9.91|0.00|465.77|2911.65|4658.64|37.26|0.00|465.77|503.03|-2445.88| +|66252|17017|82992|||||276|1299||30.24||||355.14|544.32||7.95|||107.39|| +2452046|66252|15835|82992|||22973||32|1299|||||0.00|||2036.00||0.00|1282.60|1385.20|-457.60| +2452046|66252|9895|82992|1169268|1944|22973|1|37|1299|68|47.68|87.25|40.13|0.00|2728.84|3242.24|5933.00|245.59|0.00|2728.84|2974.43|-513.40| +2452046|66252|7351|82992|1169268|1944|22973|1|213|1299|79|81.04|147.49|44.24|0.00|3494.96|6402.16|11651.71|139.79|0.00|3494.96|3634.75|-2907.20| +2452046|66252|15433|82992|1169268|1944|22973|1|298|1299|55|13.64|22.64|20.82|0.00|1145.10|750.20|1245.20|0.00|0.00|1145.10|1145.10|394.90| +2452046|66252|4773|82992|1169268|1944|22973|1|221|1299|77|16.44|22.68|3.85|0.00|296.45|1265.88|1746.36|11.85|0.00|296.45|308.30|-969.43| +2452046|66252|17897|82992|1169268|1944|22973|1|168|1299|31|68.83|70.20|47.03|0.00|1457.93|2133.73|2176.20|43.73|0.00|1457.93|1501.66|-675.80| +2452046|66252|15623|82992|1169268|1944|22973|1|68|1299|40|50.63|99.23|80.37|675.10|3214.80|2025.20|3969.20|152.38|675.10|2539.70|2692.08|514.50| +2452046|66252|16587|82992|1169268|1944|22973|1|67|1299|73|8.68|9.02|1.17|49.53|85.41|633.64|658.46|2.87|49.53|35.88|38.75|-597.76| +2452046|66252|479|82992||1944|22973|||1299|45|83.06|96.34|||390.15|3737.70|4335.30|3.90||390.15||-3347.55| +2452046|66252|6187|82992|1169268|1944|22973|1|59|1299|4|98.99|178.18|28.50|0.00|114.00|395.96|712.72|7.98|0.00|114.00|121.98|-281.96| +2452046|66252|14057|82992|1169268|1944|22973|1|216|1299|29|78.91|93.11|34.45|0.00|999.05|2288.39|2700.19|19.98|0.00|999.05|1019.03|-1289.34| +2452046|66252|10009|82992|1169268|1944|22973|1|223|1299|43|86.70|108.37|42.26|0.00|1817.18|3728.10|4659.91|54.51|0.00|1817.18|1871.69|-1910.92| +2452046|66252|15213|82992|1169268|1944|22973|1|71|1299|76|21.82|37.09|36.34|0.00|2761.84|1658.32|2818.84|110.47|0.00|2761.84|2872.31|1103.52| +2452215|52216|5219|32886|1035584|4329|13874|7|122|1300|60|79.35|138.86|127.75|0.00|7665.00|4761.00|8331.60|536.55|0.00|7665.00|8201.55|2904.00| +2452215|52216|8713|32886|1035584|4329|13874|7|6|1300|48|72.67|77.75|26.43|0.00|1268.64|3488.16|3732.00|0.00|0.00|1268.64|1268.64|-2219.52| +2452215|52216|7323|32886|1035584|4329|13874|7|210|1300|80|64.07|128.14|107.63|0.00|8610.40|5125.60|10251.20|430.52|0.00|8610.40|9040.92|3484.80| +2452215|52216|7711|32886|1035584|4329|13874|7|258|1300|6|50.69|71.97|18.71|0.00|112.26|304.14|431.82|10.10|0.00|112.26|122.36|-191.88| +2452215|52216|14507|32886|1035584|4329|13874|7|212|1300|53|1.48|2.45|1.91|0.00|101.23|78.44|129.85|3.03|0.00|101.23|104.26|22.79| +2452215|52216|7305|32886||4329||||1300|27||19.16|||154.98|269.46|517.32|||||-114.48| +||11245|32886||4329|13874||7|1300||||||||12529.48|||8018.72|8259.28|| +2452215|52216|7297|32886|1035584|4329|13874|7|280|1300|50|78.24|95.45|4.77|0.00|238.50|3912.00|4772.50|7.15|0.00|238.50|245.65|-3673.50| +2452215|52216|8651|32886|1035584|4329|13874|7|24|1300|16|61.62|77.02|4.62|0.00|73.92|985.92|1232.32|5.91|0.00|73.92|79.83|-912.00| +2452215|52216|11679|32886|1035584|4329|13874|7|123|1300|74|98.28|159.21|81.19|0.00|6008.06|7272.72|11781.54|120.16|0.00|6008.06|6128.22|-1264.66| +2452215|52216|14009|32886|1035584|4329|13874|7|154|1300|85|37.91|54.96|20.88|0.00|1774.80|3222.35|4671.60|141.98|0.00|1774.80|1916.78|-1447.55| +2452215|52216|12139|32886|1035584|4329|13874|7|208|1300|74|87.24|141.32|4.23|0.00|313.02|6455.76|10457.68|28.17|0.00|313.02|341.19|-6142.74| +||6871|32886|1035584|4329|13874|7|36|1300|||40.94||0.00||1757.75|3234.26|161.06|0.00|2684.42|2845.48|926.67| +2452215|52216|10391|32886|1035584|4329|13874|7|256|1300|65|12.01|15.13|4.23|181.46|274.95|780.65|983.45|0.00|181.46|93.49|93.49|-687.16| +2452215|52216|1463|32886|1035584|4329|13874|7|223|1300|39|64.42|115.31|0.00|0.00|0.00|2512.38|4497.09|0.00|0.00|0.00|0.00|-2512.38| +2451486|30955|7576|92157|844367|3777|31987|2|70|1301|89|89.89|136.63|34.15|0.00|3039.35|8000.21|12160.07|182.36|0.00|3039.35|3221.71|-4960.86| +2451486|30955|14996|92157|844367|3777|31987|2|226|1301|44|95.76|104.37|86.62|0.00|3811.28|4213.44|4592.28|76.22|0.00|3811.28|3887.50|-402.16| +2451486|30955|901|92157|844367|3777|31987|2|54|1301|31|71.96|108.65|106.47|0.00|3300.57|2230.76|3368.15|132.02|0.00|3300.57|3432.59|1069.81| +2451486|30955|12706|92157|844367|3777|31987|2|1|1301|21|64.40|83.72|77.85|0.00|1634.85|1352.40|1758.12|130.78|0.00|1634.85|1765.63|282.45| +2451486|30955|5965|92157|844367|3777|31987|2|204|1301|94|9.68|11.22|5.61|0.00|527.34|909.92|1054.68|5.27|0.00|527.34|532.61|-382.58| +2451486|30955|8611|92157|844367|3777|31987|2|215|1301|1|35.46|69.14|10.37|0.00|10.37|35.46|69.14|0.51|0.00|10.37|10.88|-25.09| +2451486|30955|3856|92157|844367|3777|31987|2|261|1301|19|11.16|11.94|9.79|0.00|186.01|212.04|226.86|16.74|0.00|186.01|202.75|-26.03| +2451486|30955|10507|92157|844367|3777|31987|2|56|1301|59|43.77|59.52|7.14|0.00|421.26|2582.43|3511.68|8.42|0.00|421.26|429.68|-2161.17| +2451486|30955|4471|92157|844367|3777|31987|2|235|1301|12|7.72|12.73|9.42|30.52|113.04|92.64|152.76|3.30|30.52|82.52|85.82|-10.12| +2451486|30955|1972|92157|844367|3777|31987|2|105|1301|91|93.54|93.54|28.99|26.38|2638.09|8512.14|8512.14|0.00|26.38|2611.71|2611.71|-5900.43| +|30955|10939|92157||3777|31987||52|1301||73.04|80.34|77.12|0.00|||6989.58|0.00|0.00||6709.44|| +2451486|30955|817|92157|844367|3777|31987|2|17|1301|64|75.76|89.39|37.54|0.00|2402.56|4848.64|5720.96|120.12|0.00|2402.56|2522.68|-2446.08| +2451486|30955|14444|92157|844367|3777|31987|2|208|1301|54|61.53|82.45|21.43|509.17|1157.22|3322.62|4452.30|19.44|509.17|648.05|667.49|-2674.57| +2451606|44870|5161|41011|822133|6455|19577|10|174|1302|80|42.58|45.56|22.78|0.00|1822.40|3406.40|3644.80|36.44|0.00|1822.40|1858.84|-1584.00| +2451606|44870|13453|41011|822133|6455|19577|10|200|1302|38|24.01|41.29|30.96|0.00|1176.48|912.38|1569.02|70.58|0.00|1176.48|1247.06|264.10| +2451606|44870|13646|41011|822133|6455|19577|10|171|1302|6|77.14|103.36|81.65|142.07|489.90|462.84|620.16|10.43|142.07|347.83|358.26|-115.01| +2451606|44870|5390|||||||1302|88|38.09||5.19|0.00||3351.92|4156.24|22.83|0.00|||| +2451606|44870|2743|41011|822133|6455|19577|10|5|1302|59|98.31|162.21|16.22|0.00|956.98|5800.29|9570.39|19.13|0.00|956.98|976.11|-4843.31| +||1495|41011|||19577|10||1302|63|||8.24|0.00|519.12||10391.22||0.00|519.12||-4676.49| +2451606|44870|14048|41011|822133|6455|19577|10|106|1302|26|87.52|126.90|13.95|0.00|362.70|2275.52|3299.40|14.50|0.00|362.70|377.20|-1912.82| +|44870|9833||||19577|10|75|1302|14|||||634.34|||25.37||634.34||| +2451606|44870|15044|41011|822133|6455|19577|10|262|1302|91|83.46|146.88|113.09|0.00|10291.19|7594.86|13366.08|514.55|0.00|10291.19|10805.74|2696.33| +2451606|44870|8287|41011|822133|6455|19577|10|57|1302|97|65.86|71.12|68.98|0.00|6691.06|6388.42|6898.64|267.64|0.00|6691.06|6958.70|302.64| +2451606|44870|11510|41011|822133|6455|19577|10|76|1302|58|63.66|78.93|51.30|0.00|2975.40|3692.28|4577.94|29.75|0.00|2975.40|3005.15|-716.88| +2451606|44870|5389|41011|822133|6455|19577|10|145|1302|78|87.93|155.63|101.15|0.00|7889.70|6858.54|12139.14|710.07|0.00|7889.70|8599.77|1031.16| +2451606|44870|5875|41011|822133|6455|19577|10|129|1302|10|2.26|4.20|3.73|0.00|37.30|22.60|42.00|1.49|0.00|37.30|38.79|14.70| +2451606|44870|4568|41011|822133|6455|19577|10|275|1302|82|13.43|25.91|17.10|0.00|1402.20|1101.26|2124.62|14.02|0.00|1402.20|1416.22|300.94| +2451606|44870|2257|41011|822133|6455|19577|10|82|1302|43|38.97|47.15|38.66|0.00|1662.38|1675.71|2027.45|116.36|0.00|1662.38|1778.74|-13.33| +2451606|44870|10664|41011|822133|6455|19577|10|256|1302|99|26.63|33.02|21.46|0.00|2124.54|2636.37|3268.98|84.98|0.00|2124.54|2209.52|-511.83| +2451829|74901|8317|57707|890642|1177|6559|2|249|1303|42|29.07|53.77|16.13|0.00|677.46|1220.94|2258.34|33.87|0.00|677.46|711.33|-543.48| +2451829|74901|2498|57707|890642|1177|6559|2|253|1303|17|52.86|98.84|7.90|0.00|134.30|898.62|1680.28|8.05|0.00|134.30|142.35|-764.32| +2451829|74901|16343|57707|890642|1177|6559|2|112|1303|46|36.33|47.22|31.63|101.84|1454.98|1671.18|2172.12|13.53|101.84|1353.14|1366.67|-318.04| +2451829|74901|7349|57707|890642|1177|6559|2|188|1303|43|62.85|83.59|46.81|0.00|2012.83|2702.55|3594.37|161.02|0.00|2012.83|2173.85|-689.72| +2451829|74901|6128|57707|890642|1177|6559|2|17|1303|78|43.60|80.22|28.07|0.00|2189.46|3400.80|6257.16|43.78|0.00|2189.46|2233.24|-1211.34| +2451829|74901|14402|57707|890642|1177|6559|2|295|1303|28|69.70|137.30|123.57|0.00|3459.96|1951.60|3844.40|276.79|0.00|3459.96|3736.75|1508.36| +2451829|74901|12980|57707|890642|1177|6559|2|135|1303|7|69.61|107.89|21.57|0.00|150.99|487.27|755.23|9.05|0.00|150.99|160.04|-336.28| +2451829|74901|151|57707|890642|1177|6559|2|171|1303|78|71.56|132.38|9.26|0.00|722.28|5581.68|10325.64|0.00|0.00|722.28|722.28|-4859.40| +2451829|74901|1073|57707|890642|1177|6559|2|200|1303|26|66.33|131.99|18.47|0.00|480.22|1724.58|3431.74|38.41|0.00|480.22|518.63|-1244.36| +2452459|38544|13653|30370|1675336|6349|21715|7|10|1304|32|37.82|74.88|54.66|0.00|1749.12|1210.24|2396.16|157.42|0.00|1749.12|1906.54|538.88| +2452459|38544|799|30370|1675336|6349|21715|7|273|1304|16|79.31|134.82|71.45|0.00|1143.20|1268.96|2157.12|68.59|0.00|1143.20|1211.79|-125.76| +2452459|38544|16027|30370|1675336|6349|21715|7|245|1304|36|95.83|95.83|28.74|0.00|1034.64|3449.88|3449.88|72.42|0.00|1034.64|1107.06|-2415.24| +2452459|38544|4273|30370|1675336|6349|21715|7|63|1304|97|46.62|88.11|35.24|0.00|3418.28|4522.14|8546.67|102.54|0.00|3418.28|3520.82|-1103.86| +2452459|38544|14863|30370|1675336|6349|21715|7|244|1304|55|91.04|103.78|32.17|0.00|1769.35|5007.20|5707.90|106.16|0.00|1769.35|1875.51|-3237.85| +2452459|38544|9757|30370|1675336|6349|21715|7|201|1304|37|13.52|16.90|1.69|0.00|62.53|500.24|625.30|1.25|0.00|62.53|63.78|-437.71| +2452459|38544|8875|30370|1675336|6349|21715|7|270|1304|80|1.18|2.32|0.34|21.76|27.20|94.40|185.60|0.21|21.76|5.44|5.65|-88.96| +2452459|38544|10968|||6349|21715||70|1304|91|88.05|||0.00|||||0.00|||| +2452459|38544|4231|30370|1675336|6349|21715|7|34|1304|32|92.11|135.40|17.60|0.00|563.20|2947.52|4332.80|28.16|0.00|563.20|591.36|-2384.32| +2452459|38544|4959|30370|1675336|6349|21715|7|26|1304|19|37.99|64.20|14.76|0.00|280.44|721.81|1219.80|0.00|0.00|280.44|280.44|-441.37| +2452459|38544|7347|30370|1675336|6349|21715|7|103|1304|94|76.06|89.75|22.43|0.00|2108.42|7149.64|8436.50|0.00|0.00|2108.42|2108.42|-5041.22| +2451370|44584|11725|4471|1387352|5080|30361|2|237|1305|49|6.57|7.35|5.73|0.00|280.77|321.93|360.15|14.03|0.00|280.77|294.80|-41.16| +2451370|44584|16588|4471|1387352|5080|30361|2|128|1305|18|78.04|102.23|62.36|0.00|1122.48|1404.72|1840.14|56.12|0.00|1122.48|1178.60|-282.24| +2451370|44584|17960|4471|1387352|5080|30361|2|260|1305|82|90.14|137.01|72.61|1369.42|5954.02|7391.48|11234.82|275.07|1369.42|4584.60|4859.67|-2806.88| +2451370|44584|14720|4471|1387352|5080|30361|2|54|1305|86|42.12|44.64|39.72|0.00|3415.92|3622.32|3839.04|0.00|0.00|3415.92|3415.92|-206.40| +2451370|44584|17216|4471|1387352|5080|30361|2|47|1305|4|47.94|67.59|24.33|0.00|97.32|191.76|270.36|0.00|0.00|97.32|97.32|-94.44| +2451370|44584|3616|4471|1387352|5080|30361|2|117|1305|81|7.38|8.48|2.71|0.00|219.51|597.78|686.88|15.36|0.00|219.51|234.87|-378.27| +2451370|44584|13282|4471|1387352|5080|30361|2|70|1305|22|60.35|91.12|19.13|0.00|420.86|1327.70|2004.64|0.00|0.00|420.86|420.86|-906.84| +2451370|44584|13630|4471|1387352|5080|30361|2|50|1305|100|18.86|23.95|19.16|0.00|1916.00|1886.00|2395.00|172.44|0.00|1916.00|2088.44|30.00| +2451370|44584|5980|4471|1387352|5080|30361|2|185|1305|48|21.51|31.61|19.59|0.00|940.32|1032.48|1517.28|28.20|0.00|940.32|968.52|-92.16| +2451370|44584|4552|4471|1387352|5080|30361|2|236|1305|17|26.47|47.38|30.32|0.00|515.44|449.99|805.46|36.08|0.00|515.44|551.52|65.45| +|44584|10993||||30361|2||1305|11||||||571.23||20.86|||716.50|124.41| +2451370|44584|49|4471|1387352|5080|30361|2|199|1305|64|75.62|127.04|45.73|0.00|2926.72|4839.68|8130.56|204.87|0.00|2926.72|3131.59|-1912.96| +2451370|44584|4582|4471|1387352|5080|30361|2|256|1305|24|47.35|83.33|22.49|0.00|539.76|1136.40|1999.92|37.78|0.00|539.76|577.54|-596.64| +2451370|44584|13538|4471|1387352|5080|30361|2|124|1305|20|35.44|69.10|13.82|0.00|276.40|708.80|1382.00|8.29|0.00|276.40|284.69|-432.40| +2451370|44584|3248|4471|1387352|5080|30361|2|173|1305|92|6.09|10.41|4.58|0.00|421.36|560.28|957.72|21.06|0.00|421.36|442.42|-138.92| +2451370|44584|4682|4471|1387352|5080|30361|2|242|1305|87|91.17|174.13|104.47|0.00|9088.89|7931.79|15149.31|363.55|0.00|9088.89|9452.44|1157.10| +2452265|54124|8273|3102|1000974|2399|5471|4|127|1306|52|58.23|93.75|13.12|0.00|682.24|3027.96|4875.00|40.93|0.00|682.24|723.17|-2345.72| +2452265|54124|12991|3102|1000974|2399|5471|4|185|1306|19|15.13|18.45|17.52|3.32|332.88|287.47|350.55|16.47|3.32|329.56|346.03|42.09| +2452265|54124|4729|3102|1000974|2399|5471|4|182|1306|30|56.36|59.17|14.20|115.02|426.00|1690.80|1775.10|0.00|115.02|310.98|310.98|-1379.82| +2452265|54124|451|3102|1000974|2399|5471|4|2|1306|14|66.78|106.84|54.48|0.00|762.72|934.92|1495.76|30.50|0.00|762.72|793.22|-172.20| +2452265|54124|10049|3102|1000974|2399|5471|4|288|1306|42|76.94|97.71|17.58|0.00|738.36|3231.48|4103.82|29.53|0.00|738.36|767.89|-2493.12| +2452265||12455||1000974||||219|1306||1.63|1.89||0.00||154.85|||0.00|83.60|87.78|| +||8809|3102|1000974|2399|5471|4|29|1306|||39.25||0.00|2599.68||||0.00||2755.66|0.96| +2452265|54124|13305|3102|1000974|2399|5471|4|130|1306|68|98.05|106.87|9.61|0.00|653.48|6667.40|7267.16|19.60|0.00|653.48|673.08|-6013.92| +2452265|54124|10935|3102|1000974|2399|5471|4|174|1306|14|52.05|52.57|11.56|0.00|161.84|728.70|735.98|6.47|0.00|161.84|168.31|-566.86| +2452265|54124|7699|3102|1000974|2399|5471|4|275|1306|63|19.33|37.69|35.80|0.00|2255.40|1217.79|2374.47|157.87|0.00|2255.40|2413.27|1037.61| +2452265|54124|1403|3102|1000974|2399|5471|4|48|1306|47|94.08|118.54|53.34|0.00|2506.98|4421.76|5571.38|75.20|0.00|2506.98|2582.18|-1914.78| +2451529|47838|15344|15875|199411|5347|16483|4|93|1307|61|76.74|101.29|49.63|0.00|3027.43|4681.14|6178.69|151.37|0.00|3027.43|3178.80|-1653.71| +2451529|47838|11194|15875|199411|5347|16483|4|198|1307|83|43.33|73.22|41.00|0.00|3403.00|3596.39|6077.26|102.09|0.00|3403.00|3505.09|-193.39| +2451529|47838|8632|15875|199411|5347|16483|4|73|1307|43|88.86|105.74|11.63|0.00|500.09|3820.98|4546.82|40.00|0.00|500.09|540.09|-3320.89| +2451529|47838|8185|15875|199411|5347|16483|4|25|1307|38|70.72|94.76|71.07|810.19|2700.66|2687.36|3600.88|113.42|810.19|1890.47|2003.89|-796.89| +2451529|47838|10310|15875|199411|5347|16483|4|262|1307|66|53.10|82.30|14.81|0.00|977.46|3504.60|5431.80|48.87|0.00|977.46|1026.33|-2527.14| +2451529|47838|12541|15875|199411|5347|16483|4|146|1307|20|1.68|1.86|0.14|0.00|2.80|33.60|37.20|0.19|0.00|2.80|2.99|-30.80| +2451529|47838|10252||199411|||4|266|1307|87|||||1098.81|2699.61|4778.04|||||| +2451529|47838|16604|15875|199411|5347|16483|4|92|1307|67|46.90|56.28|19.13|0.00|1281.71|3142.30|3770.76|76.90|0.00|1281.71|1358.61|-1860.59| +2451529|47838|3800|15875|199411|5347|16483|4|66|1307|79|97.20|182.73|60.30|0.00|4763.70|7678.80|14435.67|47.63|0.00|4763.70|4811.33|-2915.10| +2450947||6073|61840||909||7|186|1308||||12.94||||2632.15|0.00||710.41||-1068.35| +2450947|63987|5020|61840|1154798|909|7067|7|100|1308|50|91.54|142.80|54.26|0.00|2713.00|4577.00|7140.00|217.04|0.00|2713.00|2930.04|-1864.00| +2450947|63987|14072|61840|1154798|909|7067|7|26|1308|82|24.33|48.66|13.13|0.00|1076.66|1995.06|3990.12|0.00|0.00|1076.66|1076.66|-918.40| +2450947|63987|17416|61840|1154798|909|7067|7|280|1308|7|56.91|58.61|5.27|0.00|36.89|398.37|410.27|2.21|0.00|36.89|39.10|-361.48| +2450947|63987|2768|61840|1154798|909|7067|7|48|1308|71|63.40|77.34|13.14|0.00|932.94|4501.40|5491.14|18.65|0.00|932.94|951.59|-3568.46| +2450947|63987|17984|61840|1154798|909|7067|7|62|1308|81|87.41|99.64|91.66|6830.50|7424.46|7080.21|8070.84|53.45|6830.50|593.96|647.41|-6486.25| +2450947|63987|2998|61840|1154798|909|7067|7|250|1308|26|27.00|33.48|10.37|0.00|269.62|702.00|870.48|10.78|0.00|269.62|280.40|-432.38| +2450947|63987|3712|61840|1154798|909|7067|7|76|1308|84|92.60|160.19|81.69|0.00|6861.96|7778.40|13455.96|205.85|0.00|6861.96|7067.81|-916.44| +2450947|63987|6182|61840|1154798|909|7067|7|162|1308|6|95.37|156.40|20.33|0.00|121.98|572.22|938.40|2.43|0.00|121.98|124.41|-450.24| +2450947|63987|16166|61840|1154798|909|7067|7|153|1308|32|82.17|119.14|22.63|0.00|724.16|2629.44|3812.48|36.20|0.00|724.16|760.36|-1905.28| +2450947|63987|10489|61840|1154798|909|7067|7|71|1308|51|81.86|163.72|47.47|0.00|2420.97|4174.86|8349.72|48.41|0.00|2420.97|2469.38|-1753.89| +2450947|63987|11374|61840|1154798|909|7067|7|2|1308|62|29.50|33.33|24.33|0.00|1508.46|1829.00|2066.46|60.33|0.00|1508.46|1568.79|-320.54| +2450947|63987|1180|61840|1154798|909|7067|7|261|1308|93|53.12|75.96|69.12|0.00|6428.16|4940.16|7064.28|321.40|0.00|6428.16|6749.56|1488.00| +2450947|63987|14869|61840|1154798|909|7067|7|39|1308|63|1.70|2.63|1.34|0.00|84.42|107.10|165.69|2.53|0.00|84.42|86.95|-22.68| +2450947|63987|12973|61840|1154798|909|7067|7|61|1308|24|78.30|103.35|87.84|0.00|2108.16|1879.20|2480.40|147.57|0.00|2108.16|2255.73|228.96| +2451854|66757|6161|32392|120966|5893|2020|10|152|1309|49|66.42|86.34|40.57|0.00|1987.93|3254.58|4230.66|159.03|0.00|1987.93|2146.96|-1266.65| +2451854|66757|10351|32392|120966|5893|2020|10|117|1309|52|82.20|154.53|29.36|0.00|1526.72|4274.40|8035.56|15.26|0.00|1526.72|1541.98|-2747.68| +2451854|66757|14735|32392|120966|5893|2020|10|256|1309|26|74.49|116.20|88.31|0.00|2296.06|1936.74|3021.20|183.68|0.00|2296.06|2479.74|359.32| +|66757|5659|||5893|2020||168|1309|91||58.45|43.83|||3039.40|5318.95|||||949.13| +2451854|66757|8437|32392|120966|5893|2020|10|105|1309|92|59.32|87.79|84.27|0.00|7752.84|5457.44|8076.68|387.64|0.00|7752.84|8140.48|2295.40| +2451854|66757|11105|32392|120966|5893|2020|10|207|1309|81|26.31|27.36|10.12|0.00|819.72|2131.11|2216.16|49.18|0.00|819.72|868.90|-1311.39| +2451854|66757|17369|32392|120966|5893|2020|10|212|1309|82|84.98|116.42|12.80|0.00|1049.60|6968.36|9546.44|62.97|0.00|1049.60|1112.57|-5918.76| +2451854|66757|13157|32392|120966|5893|2020|10|246|1309|5|56.94|91.67|55.00|0.00|275.00|284.70|458.35|8.25|0.00|275.00|283.25|-9.70| +2451854|66757|14264|32392|120966|5893|2020|10|246|1309|40|95.17|181.77|74.52|0.00|2980.80|3806.80|7270.80|268.27|0.00|2980.80|3249.07|-826.00| +2451854|66757|11360|32392|120966|5893|2020|10|73|1309|27|39.26|61.63|33.89|0.00|915.03|1060.02|1664.01|82.35|0.00|915.03|997.38|-144.99| +2451854|66757|7595|32392|120966|5893|2020|10|24|1309|80|92.81|123.43|32.09|0.00|2567.20|7424.80|9874.40|77.01|0.00|2567.20|2644.21|-4857.60| +2451854|66757|16417|32392|120966|5893|2020|10|5|1309|94|62.31|87.85|14.93|0.00|1403.42|5857.14|8257.90|126.30|0.00|1403.42|1529.72|-4453.72| +2451854|66757|9307|32392|120966|5893|2020|10|228|1309|20|68.61|80.95|16.99|0.00|339.80|1372.20|1619.00|23.78|0.00|339.80|363.58|-1032.40| +2451854|66757|1423|32392|120966|5893|2020|10|298|1309|65|9.34|18.11|8.33|0.00|541.45|607.10|1177.15|0.00|0.00|541.45|541.45|-65.65| +2452593|71114|4575|164|884481|2751|1821|10|23|1310|42|32.28|61.65|51.78|0.00|2174.76|1355.76|2589.30|173.98|0.00|2174.76|2348.74|819.00| +2452593|71114|6354|164|884481|2751|1821|10|173|1310|70|28.66|54.16|31.95|0.00|2236.50|2006.20|3791.20|178.92|0.00|2236.50|2415.42|230.30| +2452593|71114|6300||884481|||10||1310||||13.25|||4001.86|||||625.43|-3411.83| +2452593|71114|10903|164|884481|2751|1821|10|271|1310|76|85.26|101.45|54.78|0.00|4163.28|6479.76|7710.20|41.63|0.00|4163.28|4204.91|-2316.48| +2452593|71114|6786|164|884481|2751|1821|10|133|1310|32|81.25|107.25|92.23|0.00|2951.36|2600.00|3432.00|206.59|0.00|2951.36|3157.95|351.36| +2452593|71114|2580|164|884481|2751|1821|10|91|1310|98|63.13|85.85|16.31|0.00|1598.38|6186.74|8413.30|63.93|0.00|1598.38|1662.31|-4588.36| +2452593|71114|10785|164|884481|2751|1821|10|93|1310|38|96.95|124.09|13.64|72.56|518.32|3684.10|4715.42|40.11|72.56|445.76|485.87|-3238.34| +2452593|71114|11335|164|884481|2751|1821|10|293|1310|68|60.22|62.62|36.94|0.00|2511.92|4094.96|4258.16|200.95|0.00|2511.92|2712.87|-1583.04| +2452593|71114|6942|164|884481|2751|1821|10|163|1310|4|78.53|131.93|97.62|0.00|390.48|314.12|527.72|19.52|0.00|390.48|410.00|76.36| +2452593|71114|4461|164|884481|2751|1821|10|298|1310|41|15.87|16.66|2.66|0.00|109.06|650.67|683.06|2.18|0.00|109.06|111.24|-541.61| +2452593|71114|15465|164|884481|2751|1821|10|196|1310|39|69.28|117.08|111.22|0.00|4337.58|2701.92|4566.12|303.63|0.00|4337.58|4641.21|1635.66| +2452593|71114|783|164|884481|2751|1821|10|271|1310|32|35.90|65.33|26.78|0.00|856.96|1148.80|2090.56|42.84|0.00|856.96|899.80|-291.84| +2452593|71114|7059|164|884481|2751|1821|10|25|1310|23|16.15|26.64|23.44|0.00|539.12|371.45|612.72|37.73|0.00|539.12|576.85|167.67| +2452593|71114|13380|164|884481|2751|1821|10|31|1310|58|76.86|126.05|44.11|0.00|2558.38|4457.88|7310.90|127.91|0.00|2558.38|2686.29|-1899.50| +2452593|71114|10965|164|884481|2751|1821|10|221|1310|51|98.63|182.46|62.03|0.00|3163.53|5030.13|9305.46|126.54|0.00|3163.53|3290.07|-1866.60| +2452593|71114|6493|164|884481|2751|1821|10|282|1310|8|66.03|94.42|87.81|0.00|702.48|528.24|755.36|49.17|0.00|702.48|751.65|174.24| +2452396|52099|2371|92561|1167337|2113|18089|8|123|1311|64|22.35|37.99|23.93|0.00|1531.52|1430.40|2431.36|91.89|0.00|1531.52|1623.41|101.12| +2452396|52099|10423|92561|1167337|2113|18089|8|169|1311|74|74.71|131.48|35.49|0.00|2626.26|5528.54|9729.52|26.26|0.00|2626.26|2652.52|-2902.28| +2452396|52099|2334|92561|1167337|2113|18089|8|289|1311|6|99.59|164.32|136.38|0.00|818.28|597.54|985.92|32.73|0.00|818.28|851.01|220.74| +2452396|52099|7429|92561|1167337|2113|18089|8|78|1311|93|1.97|2.50|0.10|0.00|9.30|183.21|232.50|0.37|0.00|9.30|9.67|-173.91| +2452396|52099|8184|92561|1167337|2113|18089|8|6|1311|85|71.37|72.79|14.55|0.00|1236.75|6066.45|6187.15|74.20|0.00|1236.75|1310.95|-4829.70| +2452396|52099|2454||1167337|2113|||294|1311||||40.89|0.00|3189.42|||255.15|0.00|3189.42||| +2452396|52099|13645|92561|1167337|2113|18089|8|146|1311|7|60.90|87.08|86.20|301.70|603.40|426.30|609.56|3.01|301.70|301.70|304.71|-124.60| +2452396|52099|723|92561|1167337|2113|18089|8|153|1311|60|86.10|154.11|18.49|0.00|1109.40|5166.00|9246.60|0.00|0.00|1109.40|1109.40|-4056.60| +2452396|52099|3048|92561|1167337|2113|18089|8|126|1311|85|43.58|84.10|4.20|157.08|357.00|3704.30|7148.50|11.99|157.08|199.92|211.91|-3504.38| +2452396|52099|2251|92561|1167337|2113|18089|8|173|1311|82|30.12|51.50|17.51|0.00|1435.82|2469.84|4223.00|129.22|0.00|1435.82|1565.04|-1034.02| +2451904|34524|12157|51887|998799|6511|47023|1|241|1312|45|76.27|151.01|37.75|0.00|1698.75|3432.15|6795.45|84.93|0.00|1698.75|1783.68|-1733.40| +2451904|34524|1295|51887|998799|6511|47023|1|130|1312|14|94.14|181.69|12.71|0.00|177.94|1317.96|2543.66|16.01|0.00|177.94|193.95|-1140.02| +2451904|34524|8096|51887|998799|6511|47023|1|213|1312|76|61.70|86.99|20.00|0.00|1520.00|4689.20|6611.24|106.40|0.00|1520.00|1626.40|-3169.20| +2451904|34524|13394|51887|998799|6511|47023|1|40|1312|20|46.30|68.98|52.42|0.00|1048.40|926.00|1379.60|20.96|0.00|1048.40|1069.36|122.40| +2451904|34524|17455|51887|998799|6511|47023|1|109|1312|89|78.66|151.02|119.30|0.00|10617.70|7000.74|13440.78|424.70|0.00|10617.70|11042.40|3616.96| +2451904||2353||998799||47023|||1312|32|95.68|167.44|||3429.12|||||3429.12|3497.70|| +2451904|34524|3811|51887|998799|6511|47023|1|21|1312|44|13.83|18.11|13.40|0.00|589.60|608.52|796.84|5.89|0.00|589.60|595.49|-18.92| +2451904|34524|4538|51887|998799|6511|47023|1|192|1312|41|89.97|151.14|86.14|0.00|3531.74|3688.77|6196.74|70.63|0.00|3531.74|3602.37|-157.03| +|54432|5762||745160||24435|||1313||79.22|87.93||||7525.90|8353.35|||4927.65|5272.58|-2598.25| +2450996|54432|4348|30699|745160|5574|24435|7|222|1313|97|20.71|21.74|3.26|0.00|316.22|2008.87|2108.78|0.00|0.00|316.22|316.22|-1692.65| +2450996|54432|3061|30699|745160|5574|24435|7|232|1313|4|8.68|15.62|3.12|0.00|12.48|34.72|62.48|0.99|0.00|12.48|13.47|-22.24| +2450996|54432|6898|30699|745160|5574|24435|7|282|1313|51|41.67|44.58|40.56|1365.24|2068.56|2125.17|2273.58|63.29|1365.24|703.32|766.61|-1421.85| +2450996|54432|15416|30699|745160|5574|24435|7|93|1313|57|71.90|114.32|107.46|0.00|6125.22|4098.30|6516.24|245.00|0.00|6125.22|6370.22|2026.92| +2450996|54432|17294|30699|745160|5574|24435|7|177|1313|1|18.15|32.85|20.03|0.00|20.03|18.15|32.85|1.40|0.00|20.03|21.43|1.88| +2450996|54432|7424|30699|745160|5574|24435|7|95|1313|60|14.11|24.69|17.52|0.00|1051.20|846.60|1481.40|94.60|0.00|1051.20|1145.80|204.60| +2450996|54432|6364|30699|745160|5574|24435|7|79|1313|90|59.61|110.27|19.84|1517.76|1785.60|5364.90|9924.30|2.67|1517.76|267.84|270.51|-5097.06| +2450996|54432|14035|30699|745160|5574|24435|7|111|1313|69|78.35|105.77|8.46|0.00|583.74|5406.15|7298.13|11.67|0.00|583.74|595.41|-4822.41| +2450996|54432|4450|30699|745160|5574|24435|7|66|1313|89|74.44|81.13|12.98|0.00|1155.22|6625.16|7220.57|69.31|0.00|1155.22|1224.53|-5469.94| +2450996|54432|3427|30699|745160|5574|24435|7|17|1313|64|77.32|129.12|104.58|0.00|6693.12|4948.48|8263.68|0.00|0.00|6693.12|6693.12|1744.64| +2450996||10201||745160||24435|7||1313|34|||8.16|0.00|||||0.00|||-683.74| +2450996|54432|16033|30699|745160|5574|24435|7|111|1313|3|24.03|25.23|20.68|0.00|62.04|72.09|75.69|5.58|0.00|62.04|67.62|-10.05| +2450996|54432|6133|30699|745160|5574|24435|7|21|1313|31|8.05|10.86|3.58|0.00|110.98|249.55|336.66|9.98|0.00|110.98|120.96|-138.57| +2451313|31338|16220|22659|300606|2906|41606|4|292|1314|96|34.46|67.54|43.90|0.00|4214.40|3308.16|6483.84|126.43|0.00|4214.40|4340.83|906.24| +2451313|31338|17386|22659|300606|2906|41606|4|73|1314|64|14.65|21.97|10.76|0.00|688.64|937.60|1406.08|6.88|0.00|688.64|695.52|-248.96| +2451313|31338|17126|22659|300606|2906|41606|4|250|1314|90|80.01|120.81|100.27|0.00|9024.30|7200.90|10872.90|541.45|0.00|9024.30|9565.75|1823.40| +2451313|31338|11822|22659|300606|2906|41606|4|196|1314|63|54.06|74.60|32.82|0.00|2067.66|3405.78|4699.80|103.38|0.00|2067.66|2171.04|-1338.12| +2451313|31338|16172|22659|300606|2906|41606|4|35|1314|34|65.77|94.70|30.30|0.00|1030.20|2236.18|3219.80|10.30|0.00|1030.20|1040.50|-1205.98| +2451313|31338|4588|22659|300606|2906|41606|4|11|1314|41|93.31|136.23|91.27|0.00|3742.07|3825.71|5585.43|37.42|0.00|3742.07|3779.49|-83.64| +2451313|31338|17444|22659||2906|41606|4||1314||32.98||21.37||||949.80|||||| +2451313|31338|2119|22659|300606|2906|41606|4|282|1314|61|92.84|149.47|70.25|4070.98|4285.25|5663.24|9117.67|8.57|4070.98|214.27|222.84|-5448.97| +2451313|31338|2728|22659|300606|2906|41606|4|176|1314|22|41.46|54.31|52.13|1009.23|1146.86|912.12|1194.82|12.38|1009.23|137.63|150.01|-774.49| +2451313|31338|8491|22659|300606|2906|41606|4|50|1314|32|33.77|66.18|14.55|0.00|465.60|1080.64|2117.76|41.90|0.00|465.60|507.50|-615.04| +2451388|38404|9295|2042|1652958|5474|29190|8|256|1315|73|41.83|67.34|0.67|0.00|48.91|3053.59|4915.82|4.40|0.00|48.91|53.31|-3004.68| +2451388|38404|3874|2042|1652958|5474|29190|8|174|1315|65|14.97|16.61|12.62|0.00|820.30|973.05|1079.65|49.21|0.00|820.30|869.51|-152.75| +2451388|38404|6236|2042|1652958|5474|29190|8|74|1315|71|19.83|20.02|13.41|0.00|952.11|1407.93|1421.42|47.60|0.00|952.11|999.71|-455.82| +2451388|38404|4744|2042|1652958|5474|29190|8|207|1315|90|15.51|23.57|3.29|0.00|296.10|1395.90|2121.30|0.00|0.00|296.10|296.10|-1099.80| +2451388|38404|11776|2042|1652958|5474|29190|8|172|1315|52|68.40|79.34|30.14|0.00|1567.28|3556.80|4125.68|31.34|0.00|1567.28|1598.62|-1989.52| +2451388|38404|13420|2042|1652958|5474|29190|8|180|1315|70|91.91|151.65|15.16|0.00|1061.20|6433.70|10615.50|42.44|0.00|1061.20|1103.64|-5372.50| +2451388|38404|7963|2042|1652958|5474|29190|8|55|1315|21|16.94|33.37|6.34|0.00|133.14|355.74|700.77|1.33|0.00|133.14|134.47|-222.60| +2451388|38404|6730|2042|1652958|5474|29190|8|75|1315|81|89.16|156.03|129.50|0.00|10489.50|7221.96|12638.43|0.00|0.00|10489.50|10489.50|3267.54| +2451388|38404|11176|2042|1652958|5474|29190|8|150|1315|51|37.37|73.99|73.25|0.00|3735.75|1905.87|3773.49|74.71|0.00|3735.75|3810.46|1829.88| +2451388|38404|2875|2042|1652958|5474|29190|8|105|1315|73|40.75|50.12|2.00|0.00|146.00|2974.75|3658.76|1.46|0.00|146.00|147.46|-2828.75| +2451388|38404|16814|2042|1652958|5474|29190|8|286|1315|69|82.43|156.61|79.87|0.00|5511.03|5687.67|10806.09|385.77|0.00|5511.03|5896.80|-176.64| +2451121|38826|2119|30163|697807|2251|22248|7|211|1316|94|12.84|16.56|14.90|1358.58|1400.60|1206.96|1556.64|0.84|1358.58|42.02|42.86|-1164.94| +2451121|38826|2728|30163|697807|2251|22248|7|224|1316|67|98.65|148.96|126.61|0.00|8482.87|6609.55|9980.32|678.62|0.00|8482.87|9161.49|1873.32| +2451121|38826|8491|30163|697807|2251|22248|7|25|1316|71|14.94|14.94|9.71|0.00|689.41|1060.74|1060.74|34.47|0.00|689.41|723.88|-371.33| +|38826|3452|30163||2251|22248|7||1316|||||0.00||6475.66|||0.00|9781.55|10564.07|3305.89| +2451121|38826|6820|30163|697807|2251|22248|7|26|1316|28|74.21|89.79|21.54|554.87|603.12|2077.88|2514.12|0.96|554.87|48.25|49.21|-2029.63| +2451121|38826|12560|30163|697807|2251|22248|7|27|1316|57|33.37|48.38|24.67|449.98|1406.19|1902.09|2757.66|76.49|449.98|956.21|1032.70|-945.88| +2451121|38826|4969|30163|697807|2251|22248|7|162|1316|75|97.32|101.21|91.08|0.00|6831.00|7299.00|7590.75|0.00|0.00|6831.00|6831.00|-468.00| +2451121|38826|5444|30163|697807|2251|22248|7|208|1316|77|95.15|145.57|20.37|0.00|1568.49|7326.55|11208.89|78.42|0.00|1568.49|1646.91|-5758.06| +2451121|38826|14620|30163|697807|2251|22248|7|162|1316|71|6.11|10.81|6.26|0.00|444.46|433.81|767.51|22.22|0.00|444.46|466.68|10.65| +2451121|38826|211|30163|697807|2251|22248|7|47|1316|60|78.83|127.70|99.60|0.00|5976.00|4729.80|7662.00|59.76|0.00|5976.00|6035.76|1246.20| +2451121|38826|10906|30163|697807|2251|22248|7|121|1316|88|17.99|24.10|13.25|0.00|1166.00|1583.12|2120.80|0.00|0.00|1166.00|1166.00|-417.12| +2451121|38826|475|30163|697807|2251|22248|7|65|1316|73|19.31|20.46|14.93|435.95|1089.89|1409.63|1493.58|6.53|435.95|653.94|660.47|-755.69| +||12475|30163|697807|2251|22248|7|186|1316||60.19||31.90||1052.70||1986.27|84.21||1052.70||-933.57| +2451121|38826|16556|30163|697807|2251|22248|7|186|1316|18|14.25|15.39|1.07|0.00|19.26|256.50|277.02|0.38|0.00|19.26|19.64|-237.24| +2450895||15157|32400|1217286||20611|||1317||||63.76||5865.92|8852.24||469.27||5865.92||-2986.32| +2450895|51540|17224|32400||4572|20611|8|140|1317||8.48|11.53|0.34|0.00|||795.57|0.70|0.00|||-561.66| +2450895|51540|11990|32400|1217286|4572|20611|8|293|1317|42|48.95|75.38|19.59|0.00|822.78|2055.90|3165.96|32.91|0.00|822.78|855.69|-1233.12| +2450895|51540|9901|32400|1217286|4572|20611|8|49|1317|23|2.10|2.89|1.41|0.00|32.43|48.30|66.47|2.59|0.00|32.43|35.02|-15.87| +2450895|51540|7606|32400|1217286|4572|20611|8|219|1317|38|54.46|89.31|50.01|0.00|1900.38|2069.48|3393.78|152.03|0.00|1900.38|2052.41|-169.10| +2450895|51540|15649|32400|1217286|4572|20611|8|10|1317|14|95.26|97.16|0.00|0.00|0.00|1333.64|1360.24|0.00|0.00|0.00|0.00|-1333.64| +2450895|51540|11488|32400|1217286|4572|20611|8|43|1317|8|63.36|73.49|8.08|0.00|64.64|506.88|587.92|0.00|0.00|64.64|64.64|-442.24| +2450895|51540|2420|32400|1217286|4572|20611|8|289|1317|21|61.71|86.39|32.82|0.00|689.22|1295.91|1814.19|6.89|0.00|689.22|696.11|-606.69| +2450895|51540|16646|32400|1217286|4572|20611|8|115|1317|1|57.83|63.61|13.35|0.00|13.35|57.83|63.61|0.53|0.00|13.35|13.88|-44.48| +2450895|51540|17456|32400|1217286|4572|20611|8|54|1317|85|94.55|98.33|63.91|0.00|5432.35|8036.75|8358.05|325.94|0.00|5432.35|5758.29|-2604.40| +2450895|51540|6098|32400|1217286|4572|20611|8|92|1317|60|1.12|1.35|0.40|18.00|24.00|67.20|81.00|0.00|18.00|6.00|6.00|-61.20| +2450895|51540|3334|32400|1217286|4572|20611|8|117|1317|73|67.24|115.65|28.91|232.14|2110.43|4908.52|8442.45|75.13|232.14|1878.29|1953.42|-3030.23| +2450895|51540|17402||1217286|4572||||1317|85||63.60|40.70|1971.91|3459.50|||14.87|1971.91|||-1789.16| +2450895|51540|10408|32400|1217286|4572|20611|8|268|1317|21|59.13|104.06|80.12|0.00|1682.52|1241.73|2185.26|67.30|0.00|1682.52|1749.82|440.79| +2450895|51540|9793|32400|1217286|4572|20611|8|188|1317|38|22.57|31.14|26.46|0.00|1005.48|857.66|1183.32|20.10|0.00|1005.48|1025.58|147.82| +|63315|9306|29451||||||1318|94|||19.93||1873.42|||||1873.42|1910.88|642.02| +2452561|63315|4953|29451|1119564|4077|27390|4|204|1318|71|45.89|85.81|27.45|0.00|1948.95|3258.19|6092.51|77.95|0.00|1948.95|2026.90|-1309.24| +2452561|63315|17055|29451|1119564|4077|27390|4|5|1318|67|91.42|131.64|18.42|0.00|1234.14|6125.14|8819.88|86.38|0.00|1234.14|1320.52|-4891.00| +2452561|63315|12301|29451|1119564|4077|27390|4|54|1318|98|73.23|127.42|62.43|0.00|6118.14|7176.54|12487.16|428.26|0.00|6118.14|6546.40|-1058.40| +2452561|63315|4843|29451|1119564|4077|27390|4|224|1318|14|26.91|50.59|4.55|0.00|63.70|376.74|708.26|5.09|0.00|63.70|68.79|-313.04| +2452561|63315|1590|29451|1119564|4077|27390|4|177|1318|59|16.02|29.31|5.86|0.00|345.74|945.18|1729.29|10.37|0.00|345.74|356.11|-599.44| +2452561|63315|6519|29451|1119564|4077|27390|4|219|1318|13|39.83|78.46|66.69|0.00|866.97|517.79|1019.98|43.34|0.00|866.97|910.31|349.18| +2452561|63315|15204|29451|1119564|4077|27390|4|185|1318|99|10.55|15.71|1.25|0.00|123.75|1044.45|1555.29|4.95|0.00|123.75|128.70|-920.70| +2452561|63315|14973|29451|1119564|4077|27390|4|297|1318|80|29.53|46.65|2.33|0.00|186.40|2362.40|3732.00|0.00|0.00|186.40|186.40|-2176.00| +2451140|45182|1798|40961|1381105|6402|26146|2|96|1319|41|30.05|54.69|38.82|0.00|1591.62|1232.05|2242.29|127.32|0.00|1591.62|1718.94|359.57| +2451140|45182|10858|40961|1381105|6402|26146|2|39|1319|83|6.71|11.20|7.50|0.00|622.50|556.93|929.60|49.80|0.00|622.50|672.30|65.57| +2451140|45182|6553|40961|1381105|6402|26146|2|103|1319|10|3.58|5.44|2.06|0.00|20.60|35.80|54.40|0.00|0.00|20.60|20.60|-15.20| +2451140|45182|5756|40961|1381105|6402|26146|2|262|1319|2|24.03|33.88|3.72|0.00|7.44|48.06|67.76|0.29|0.00|7.44|7.73|-40.62| +2451140|45182|4441|40961|1381105|6402|26146|2|111|1319|77|12.68|15.21|4.71|0.00|362.67|976.36|1171.17|21.76|0.00|362.67|384.43|-613.69| +|45182|13724|40961|||26146|2|218|1319||12.97|||0.00||181.58|||0.00|117.32|126.70|-64.26| +|45182|76|||6402|||183|1319|||135.61||0.00|488.15|362.60|||0.00|488.15|517.43|| +2451140|45182|15097|40961|1381105|6402|26146|2|60|1319|44|80.89|129.42|44.00|0.00|1936.00|3559.16|5694.48|174.24|0.00|1936.00|2110.24|-1623.16| +2451140|45182|17710|40961|1381105|6402|26146|2|108|1319|100|99.93|122.91|62.68|0.00|6268.00|9993.00|12291.00|62.68|0.00|6268.00|6330.68|-3725.00| +2451140|45182|13312|40961|1381105|6402|26146|2|199|1319|17|76.91|83.06|24.91|0.00|423.47|1307.47|1412.02|0.00|0.00|423.47|423.47|-884.00| +2451140|45182|6853|40961|1381105|6402|26146|2|162|1319|72|34.12|38.55|7.32|0.00|527.04|2456.64|2775.60|47.43|0.00|527.04|574.47|-1929.60| +|45182|1018|||6402||||1319|29||46.34|18.53|0.00|537.37||1343.86||0.00||548.11|-169.94| +2451140|45182|4297|40961|1381105|6402|26146|2|127|1319|65|80.32|130.11|20.81|1285.01|1352.65|5220.80|8457.15|6.08|1285.01|67.64|73.72|-5153.16| +2451306|70946|2278|45260|306807|1601|748|2|291|1320|31|80.10|150.58|150.58|0.00|4667.98|2483.10|4667.98|373.43|0.00|4667.98|5041.41|2184.88| +2451306|70946|13012|45260|306807|1601|748|2|117|1320|53|75.17|112.00|49.28|0.00|2611.84|3984.01|5936.00|104.47|0.00|2611.84|2716.31|-1372.17| +2451306|70946|14203|45260|306807|1601|748|2|165|1320|15|81.59|115.04|33.36|0.00|500.40|1223.85|1725.60|45.03|0.00|500.40|545.43|-723.45| +2451306|70946|15350|45260|306807|1601|748|2|41|1320|15|55.26|88.41|45.97|0.00|689.55|828.90|1326.15|62.05|0.00|689.55|751.60|-139.35| +2451306|70946|1261|45260|306807|1601|748|2|90|1320|62|19.62|23.15|20.83|0.00|1291.46|1216.44|1435.30|38.74|0.00|1291.46|1330.20|75.02| +2451306|70946|4514|45260|306807|1601|748|2|1|1320|56|84.91|147.74|0.00|0.00|0.00|4754.96|8273.44|0.00|0.00|0.00|0.00|-4754.96| +2451306|70946|13322|45260|306807|1601|748|2|113|1320|15|98.42|158.45|141.02|0.00|2115.30|1476.30|2376.75|21.15|0.00|2115.30|2136.45|639.00| +2451306|70946|8251|45260|306807|1601|748|2|29|1320|99|66.36|98.21|35.35|0.00|3499.65|6569.64|9722.79|34.99|0.00|3499.65|3534.64|-3069.99| +2451306|70946|14894|45260|306807|1601|748|2|291|1320|52|87.11|102.78|59.61|0.00|3099.72|4529.72|5344.56|278.97|0.00|3099.72|3378.69|-1430.00| +||16615|45260|||748|2||1320|2||96.70|7.73|||||1.08|||16.54|| +2451306|70946|9622|45260|306807|1601|748|2|140|1320|34|33.04|40.96|23.75|0.00|807.50|1123.36|1392.64|48.45|0.00|807.50|855.95|-315.86| +||5149|||||2||1320||68.26||24.94||||||||673.38|| +2451306|70946|8708|45260|306807|1601|748|2|97|1320|21|51.25|74.82|13.46|0.00|282.66|1076.25|1571.22|5.65|0.00|282.66|288.31|-793.59| +2451306|70946|9295|45260|306807|1601|748|2|274|1320|91|81.55|150.86|7.54|0.00|686.14|7421.05|13728.26|0.00|0.00|686.14|686.14|-6734.91| +2450816|67283|538|739|1607434|5329|40024|1|72|1321|44|86.59|116.89|32.72|0.00|1439.68|3809.96|5143.16|57.58|0.00|1439.68|1497.26|-2370.28| +2450816|67283|1118|739|1607434|5329|40024|1|237|1321|22|17.42|20.38|7.54|0.00|165.88|383.24|448.36|9.95|0.00|165.88|175.83|-217.36| +2450816|67283|17608|739|1607434|5329|40024|1|147|1321|68|17.34|23.75|19.00|0.00|1292.00|1179.12|1615.00|77.52|0.00|1292.00|1369.52|112.88| +2450816|67283|4006|739|1607434|5329|40024|1|256|1321|69|24.53|36.05|12.25|0.00|845.25|1692.57|2487.45|0.00|0.00|845.25|845.25|-847.32| +2450816|67283|12103|739|1607434|5329|40024|1|298|1321|44|65.77|94.05|80.88|0.00|3558.72|2893.88|4138.20|106.76|0.00|3558.72|3665.48|664.84| +2450816|67283|11486|||5329||||1321|||||||10.39|15.37|0.19|||2.64|-7.94| +2450816|67283|14426|739|1607434|5329|40024|1|254|1321|86|31.57|38.19|8.78|0.00|755.08|2715.02|3284.34|15.10|0.00|755.08|770.18|-1959.94| +2450816|67283|9406|739|1607434|5329|40024|1|67|1321|68|19.24|34.05|2.04|83.23|138.72|1308.32|2315.40|1.66|83.23|55.49|57.15|-1252.83| +2450816|67283|16888|739|1607434|5329|40024|1|272|1321|23|21.02|35.73|32.87|0.00|756.01|483.46|821.79|0.00|0.00|756.01|756.01|272.55| +2450816|67283|14512|739|1607434|5329|40024|1|16|1321|38|81.31|113.02|108.49|0.00|4122.62|3089.78|4294.76|206.13|0.00|4122.62|4328.75|1032.84| +2450816|67283|16087|739|1607434|5329|40024|1|94|1321|47|15.31|17.14|3.25|113.03|152.75|719.57|805.58|0.39|113.03|39.72|40.11|-679.85| +2450816|67283|16807|739|1607434|5329|40024|1|70|1321|82|10.35|15.73|9.12|0.00|747.84|848.70|1289.86|37.39|0.00|747.84|785.23|-100.86| +2451326|64925|2374|4421|262032|3996|26807|10|54|1322|6|1.47|2.58|0.43|1.93|2.58|8.82|15.48|0.01|1.93|0.65|0.66|-8.17| +2451326|64925|12481|4421|262032|3996|26807|10|189|1322|9|44.24|65.03|29.91|0.00|269.19|398.16|585.27|16.15|0.00|269.19|285.34|-128.97| +2451326|64925|4558|4421|262032|3996|26807|10|1|1322|36|37.47|57.32|9.17|0.00|330.12|1348.92|2063.52|29.71|0.00|330.12|359.83|-1018.80| +2451326|64925|13582|4421|262032|3996|26807|10|148|1322|84|22.52|39.41|3.54|0.00|297.36|1891.68|3310.44|17.84|0.00|297.36|315.20|-1594.32| +2451326|64925|13784|4421|262032|3996|26807|10|104|1322|46|7.01|12.33|11.46|0.00|527.16|322.46|567.18|47.44|0.00|527.16|574.60|204.70| +2451326|64925|14828|4421|262032|3996|26807|10|75|1322|87|52.65|59.49|21.41|0.00|1862.67|4580.55|5175.63|37.25|0.00|1862.67|1899.92|-2717.88| +2451326|64925|602|4421|262032|3996|26807|10|1|1322|79|99.97|122.96|67.62|0.00|5341.98|7897.63|9713.84|213.67|0.00|5341.98|5555.65|-2555.65| +2451326|64925|12547|4421|262032|3996|26807|10|275|1322|2|48.99|80.83|41.22|55.23|82.44|97.98|161.66|0.54|55.23|27.21|27.75|-70.77| +2451326|64925|4900|4421|262032|3996|26807|10|34|1322|18|68.32|125.70|100.56|0.00|1810.08|1229.76|2262.60|126.70|0.00|1810.08|1936.78|580.32| +2451326|64925|12002|4421|262032|3996|26807|10|68|1322|8|45.56|54.21|5.96|0.00|47.68|364.48|433.68|0.95|0.00|47.68|48.63|-316.80| +2451326|64925|2905|4421|262032|3996|26807|10|27|1322|46|11.48|14.46|3.75|0.00|172.50|528.08|665.16|0.00|0.00|172.50|172.50|-355.58| +2451326|64925|1826|4421|262032|3996|26807|10|142|1322|9|20.87|32.55|32.22|37.69|289.98|187.83|292.95|10.09|37.69|252.29|262.38|64.46| +2451897|59574|16721|47815|9611|1007|6003|1|103|1323|96|42.91|62.21|6.22|0.00|597.12|4119.36|5972.16|41.79|0.00|597.12|638.91|-3522.24| +2451897|59574|860|47815|9611|1007|6003|1|251|1323|5|32.24|51.26|19.99|0.00|99.95|161.20|256.30|0.99|0.00|99.95|100.94|-61.25| +||2053|||1007||1|95|1323|||||0.00|1669.37|3670.48|4771.43||0.00|1669.37|1802.91|-2001.11| +2451897|59574|13259|47815|9611|1007|6003|1|155|1323|26|20.59|22.85|15.53|0.00|403.78|535.34|594.10|4.03|0.00|403.78|407.81|-131.56| +2451897|59574|12620|47815|9611|1007|6003|1|49|1323|80|56.31|82.21|16.44|0.00|1315.20|4504.80|6576.80|39.45|0.00|1315.20|1354.65|-3189.60| +2451897|59574|4153|47815|9611|1007|6003|1|278|1323|2|73.78|135.75|90.95|0.00|181.90|147.56|271.50|9.09|0.00|181.90|190.99|34.34| +2451897|59574|17837|47815|9611|1007|6003|1|154|1323|10|48.39|51.29|0.51|0.00|5.10|483.90|512.90|0.40|0.00|5.10|5.50|-478.80| +2451897|59574|16982|47815|9611|1007|6003|1|132|1323|71|37.15|46.06|41.45|0.00|2942.95|2637.65|3270.26|206.00|0.00|2942.95|3148.95|305.30| +2451897|59574|1613|47815|9611|1007|6003|1|226|1323|23|85.29|142.43|121.06|0.00|2784.38|1961.67|3275.89|55.68|0.00|2784.38|2840.06|822.71| +2451897|59574|2983|47815|9611|1007|6003|1|180|1323|78|74.71|102.35|100.30|0.00|7823.40|5827.38|7983.30|547.63|0.00|7823.40|8371.03|1996.02| +2451897|59574|8801|47815|9611|1007|6003|1|226|1323|10|8.60|12.04|10.23|15.34|102.30|86.00|120.40|5.21|15.34|86.96|92.17|0.96| +2451897|59574|11714|47815|9611|1007|6003|1|175|1323|76|15.35|22.71|20.21|0.00|1535.96|1166.60|1725.96|46.07|0.00|1535.96|1582.03|369.36| +2451897|59574|12517|47815|9611|1007|6003|1|57|1323|4|59.59|98.32|32.44|0.00|129.76|238.36|393.28|2.59|0.00|129.76|132.35|-108.60| +2451897|59574|223|47815|9611|1007|6003|1|235|1323|79|64.92|112.96|74.55|1649.04|5889.45|5128.68|8923.84|0.00|1649.04|4240.41|4240.41|-888.27| +2451897|59574|15500|47815|9611|1007|6003|1|147|1323|29|53.64|84.21|83.36|0.00|2417.44|1555.56|2442.09|169.22|0.00|2417.44|2586.66|861.88| +2451897|59574|2299|47815|9611|1007|6003|1|126|1323|32|71.82|135.02|29.70|589.24|950.40|2298.24|4320.64|7.22|589.24|361.16|368.38|-1937.08| +2451108|64248|8648|54705|414775|4092|3762|8|17|1324|51|9.11|9.47|3.21|0.00|163.71|464.61|482.97|14.73|0.00|163.71|178.44|-300.90| +2451108|64248|2038|54705|414775|4092|3762|8|184|1324|45|93.30|111.96|35.82|0.00|1611.90|4198.50|5038.20|48.35|0.00|1611.90|1660.25|-2586.60| +2451108|64248|11533|54705|414775|4092|3762|8|263|1324|21|1.54|2.27|0.20|0.00|4.20|32.34|47.67|0.21|0.00|4.20|4.41|-28.14| +2451108|64248|1228|54705|414775|4092|3762|8|251|1324|9|4.95|8.91|7.03|0.00|63.27|44.55|80.19|0.00|0.00|63.27|63.27|18.72| +2451108|64248|6742|54705|414775|4092|3762|8|285|1324|10|22.40|23.52|9.40|0.00|94.00|224.00|235.20|0.94|0.00|94.00|94.94|-130.00| +2451108|64248|14918|54705||4092||||1324|51|44.07||||2640.27|||79.20||2640.27||392.70| +2451108|64248|10789|54705|414775|4092|3762|8|21|1324|34|47.19|67.00|4.69|0.00|159.46|1604.46|2278.00|3.18|0.00|159.46|162.64|-1445.00| +2451108|64248|15466|54705|414775|4092|3762|8|44|1324|34|78.57|152.42|140.22|0.00|4767.48|2671.38|5182.28|0.00|0.00|4767.48|4767.48|2096.10| +2451176|59277|1576|78200|668303|2053|40603|10|197|1325|57|20.99|28.33|22.38|931.23|1275.66|1196.43|1614.81|3.44|931.23|344.43|347.87|-852.00| +2451176|59277|6988|78200|668303|2053|40603|10|286|1325|41|52.09|91.67|3.66|0.00|150.06|2135.69|3758.47|13.50|0.00|150.06|163.56|-1985.63| +2451176|59277|11948|78200|668303|2053|40603|10|62|1325|25|29.53|47.24|30.70|299.32|767.50|738.25|1181.00|23.40|299.32|468.18|491.58|-270.07| +2451176|59277|3679|78200|668303|2053|40603|10|80|1325|28|81.01|144.19|24.51|0.00|686.28|2268.28|4037.32|54.90|0.00|686.28|741.18|-1582.00| +2451176|59277|3994|78200|668303|2053|40603|10|261|1325|74|72.67|114.09|93.55|0.00|6922.70|5377.58|8442.66|138.45|0.00|6922.70|7061.15|1545.12| +2451176|59277|9796|78200|668303|2053|40603|10|110|1325|3|31.25|33.75|8.43|16.94|25.29|93.75|101.25|0.25|16.94|8.35|8.60|-85.40| +2451176|59277|8810|78200|668303|2053|40603|10|152|1325|92|60.69|100.13|68.08|4822.78|6263.36|5583.48|9211.96|57.62|4822.78|1440.58|1498.20|-4142.90| +2451176|59277|17102|78200|668303|2053|40603|10|229|1325|51|46.42|60.34|5.43|0.00|276.93|2367.42|3077.34|2.76|0.00|276.93|279.69|-2090.49| +2451176|59277|14680|78200|668303|2053|40603|10|296|1325|12|91.82|102.83|98.71|0.00|1184.52|1101.84|1233.96|11.84|0.00|1184.52|1196.36|82.68| +2451176|59277|8792|78200|668303|2053|40603|10|101|1325|17|60.50|104.06|103.01|0.00|1751.17|1028.50|1769.02|122.58|0.00|1751.17|1873.75|722.67| +2451176|59277|12728|78200|668303|2053|40603|10|66|1325|61|65.79|115.79|107.68|6371.42|6568.48|4013.19|7063.19|17.73|6371.42|197.06|214.79|-3816.13| +2451176|59277|6854|78200|668303|2053|40603|10|28|1325|16|17.04|23.00|10.58|0.00|169.28|272.64|368.00|1.69|0.00|169.28|170.97|-103.36| +2451176|59277|15577|78200|668303|2053|40603|10|186|1325|57|82.45|140.98|121.24|1243.92|6910.68|4699.65|8035.86|283.33|1243.92|5666.76|5950.09|967.11| +2451176|59277|16280|78200|668303|2053|40603|10|43|1325|36|91.46|137.19|53.50|0.00|1926.00|3292.56|4938.84|38.52|0.00|1926.00|1964.52|-1366.56| +2451176|59277|5731|78200|668303|2053|40603|10|173|1325|1|2.04|2.06|1.97|0.00|1.97|2.04|2.06|0.05|0.00|1.97|2.02|-0.07| +2451176|59277|16642|78200|668303|2053|40603|10|144|1325|95|62.15|68.98|28.97|1898.98|2752.15|5904.25|6553.10|51.19|1898.98|853.17|904.36|-5051.08| +2452180|49687|8149|15021|958557|2093|37629|1|107|1326|74|95.35|172.58|139.78|0.00|10343.72|7055.90|12770.92|310.31|0.00|10343.72|10654.03|3287.82| +2452180|49687|3259|15021|958557|2093|37629|1|198|1326|95|58.17|103.54|82.83|0.00|7868.85|5526.15|9836.30|236.06|0.00|7868.85|8104.91|2342.70| +2452180|49687|4413|15021|958557|2093|37629|1|246|1326|9|52.55|53.07|6.36|0.00|57.24|472.95|477.63|1.14|0.00|57.24|58.38|-415.71| +2452180|49687|563|15021|958557|2093|37629|1|164|1326|45|92.98|136.68|16.40|0.00|738.00|4184.10|6150.60|44.28|0.00|738.00|782.28|-3446.10| +2452180|49687|5477|15021|958557|2093|37629|1|281|1326|22|44.19|72.47|54.35|430.45|1195.70|972.18|1594.34|30.61|430.45|765.25|795.86|-206.93| +2452180|49687|17491|15021|958557|2093|37629|1|123|1326|91|8.84|13.96|0.00|0.00|0.00|804.44|1270.36|0.00|0.00|0.00|0.00|-804.44| +2452180|49687|14439|15021|958557|2093|37629|1|95|1326|91|78.17|134.45|33.61|0.00|3058.51|7113.47|12234.95|30.58|0.00|3058.51|3089.09|-4054.96| +2452180|49687|11559|15021|958557|2093|37629|1|14|1326|59|38.26|46.67|9.33|0.00|550.47|2257.34|2753.53|27.52|0.00|550.47|577.99|-1706.87| +2452180|49687|15721|15021|958557|2093||1|162|1326|84||140.28|129.05||10840.20|7185.36|11783.52|975.61||10840.20|11815.81|| +2452180|49687|4575|15021|958557|2093|37629|1|194|1326|4|32.99|44.86|18.39|0.00|73.56|131.96|179.44|2.20|0.00|73.56|75.76|-58.40| +2452180|49687|6353|15021|958557|2093|37629|1|191|1326|85|41.42|74.97|43.48|0.00|3695.80|3520.70|6372.45|184.79|0.00|3695.80|3880.59|175.10| +2452180|49687|6299|15021|958557|2093|37629|1|118|1326|71|90.80|134.38|87.34|0.00|6201.14|6446.80|9540.98|310.05|0.00|6201.14|6511.19|-245.66| +2452180|49687|10903|15021|958557|2093|37629|1|196|1326|93|33.43|57.16|14.29|0.00|1328.97|3108.99|5315.88|93.02|0.00|1328.97|1421.99|-1780.02| +2452180|49687|6785|15021|958557|2093|37629|1|113|1326|43|46.25|83.71|31.80|0.00|1367.40|1988.75|3599.53|95.71|0.00|1367.40|1463.11|-621.35| +2452180|49687|2579|15021|958557|2093|37629|1|93|1326|45|99.26|180.65|52.38|0.00|2357.10|4466.70|8129.25|23.57|0.00|2357.10|2380.67|-2109.60| +|68822|8696|48461||5307|||215|1327||28.86|||0.00||1183.26|||0.00||1139.74|-118.08| +2451291|68822|12022|48461|210084|5307|22483|2|243|1327|27|26.01|26.79|15.00|340.20|405.00|702.27|723.33|5.18|340.20|64.80|69.98|-637.47| +2451291|68822|14254|48461|210084|5307|22483|2|154|1327|38|39.06|72.26|33.96|0.00|1290.48|1484.28|2745.88|64.52|0.00|1290.48|1355.00|-193.80| +2451291|68822|8365|48461|210084|5307|22483|2|195|1327|7|74.95|116.92|17.53|71.17|122.71|524.65|818.44|1.54|71.17|51.54|53.08|-473.11| +2451291|68822|5188|48461|210084|5307|22483|2|146|1327|32|80.22|110.70|105.16|0.00|3365.12|2567.04|3542.40|269.20|0.00|3365.12|3634.32|798.08| +2451291|68822|17408|48461|210084|5307|22483|2|50|1327|28|80.80|146.24|21.93|0.00|614.04|2262.40|4094.72|49.12|0.00|614.04|663.16|-1648.36| +2451291|68822|1376|48461|210084|5307|22483|2|137|1327|35|75.91|97.92|49.93|0.00|1747.55|2656.85|3427.20|139.80|0.00|1747.55|1887.35|-909.30| +2451291|68822|328|48461|210084|5307|22483|2|92|1327|8|52.89|60.82|19.46|57.60|155.68|423.12|486.56|4.90|57.60|98.08|102.98|-325.04| +2451291|68822|5755|48461|210084|5307|22483|2|134|1327|44|79.81|154.03|60.07|2431.63|2643.08|3511.64|6777.32|12.68|2431.63|211.45|224.13|-3300.19| +2451291|68822|12496|48461|210084|5307|22483|2|186|1327|97|71.45|110.03|82.52|3361.86|8004.44|6930.65|10672.91|371.40|3361.86|4642.58|5013.98|-2288.07| +2451291|68822|4081|48461|210084|5307|22483|2|163|1327|71|29.77|47.03|5.17|0.00|367.07|2113.67|3339.13|18.35|0.00|367.07|385.42|-1746.60| +2451482|62682|2738|78767|1400126|3885|2044|4|102|1328|44|97.58|144.41|20.21|0.00|889.24|4293.52|6354.04|17.78|0.00|889.24|907.02|-3404.28| +2451482|62682|4351|78767|1400126|3885|2044|4|21|1328|65|51.39|74.51|2.98|0.00|193.70|3340.35|4843.15|3.87|0.00|193.70|197.57|-3146.65| +2451482|62682|9446|78767|1400126|3885|2044|4|205|1328|45|22.08|24.72|6.67|0.00|300.15|993.60|1112.40|0.00|0.00|300.15|300.15|-693.45| +2451482|62682|10123|78767|1400126|3885|2044|4|235|1328|13|28.24|31.34|29.14|0.00|378.82|367.12|407.42|26.51|0.00|378.82|405.33|11.70| +2451482|62682|3589|78767|1400126|3885|2044|4|114|1328|52|31.83|48.38|32.41|0.00|1685.32|1655.16|2515.76|16.85|0.00|1685.32|1702.17|30.16| +2451482|62682|2704|78767|1400126|3885|2044|4|131|1328|14|61.28|75.37|73.10|0.00|1023.40|857.92|1055.18|61.40|0.00|1023.40|1084.80|165.48| +2451482|62682|15355|78767|1400126|3885|2044|4|152|1328|89|29.64|54.24|27.66|0.00|2461.74|2637.96|4827.36|98.46|0.00|2461.74|2560.20|-176.22| +2451482|62682|3548|78767|1400126|3885|2044|4|142|1328|72|26.17|37.68|25.24|0.00|1817.28|1884.24|2712.96|127.20|0.00|1817.28|1944.48|-66.96| +2451482|62682|12952|78767|1400126|3885|2044|4|21|1328|36|47.06|64.00|55.04|0.00|1981.44|1694.16|2304.00|19.81|0.00|1981.44|2001.25|287.28| +2451482|62682|5192|78767|1400126|3885|2044|4|47|1328|78|55.98|79.49|35.77|0.00|2790.06|4366.44|6200.22|111.60|0.00|2790.06|2901.66|-1576.38| +2451482|62682|11594|78767|1400126|3885|2044|4|158|1328|9|49.79|56.76|34.62|0.00|311.58|448.11|510.84|0.00|0.00|311.58|311.58|-136.53| +2451482||17404|78767||||4|236|1328||71.26||38.52|0.00||3990.56||43.14|0.00||2200.26|| +2451482|62682|16051|78767|1400126|3885|2044|4|96|1328|77|10.13|10.33|7.74|0.00|595.98|780.01|795.41|47.67|0.00|595.98|643.65|-184.03| +2451482|62682|8840|78767|1400126|3885|2044|4|282|1328|24|28.37|54.47|50.65|0.00|1215.60|680.88|1307.28|72.93|0.00|1215.60|1288.53|534.72| +2451482|62682|15674|78767|1400126|3885|2044|4|94|1328|38|14.52|24.97|9.73|0.00|369.74|551.76|948.86|25.88|0.00|369.74|395.62|-182.02| +|62682|15565||1400126|3885|2044||293|1328|14||59.94|52.14|||650.58||51.09||729.96||79.38| +2451466|59554|17665|50005|192456|3260|39311|7|45|1329|71|25.68|37.23|32.76|465.19|2325.96|1823.28|2643.33|37.21|465.19|1860.77|1897.98|37.49| +2451466|59554|3014|50005||||7|238|1329|65|89.79|||0.00|7055.75|5836.35|11380.85|352.78|0.00|7055.75|7408.53|| +||3106|50005|192456|3260|||254|1329||93.66|||0.00||1123.92||0.00|0.00|||| +2451466|59554|6272|50005|192456|3260|39311|7|196|1329|29|45.62|82.57|66.88|0.00|1939.52|1322.98|2394.53|96.97|0.00|1939.52|2036.49|616.54| +2451466|59554|4870|50005|192456|3260|39311|7|55|1329|86|77.82|122.95|94.67|0.00|8141.62|6692.52|10573.70|162.83|0.00|8141.62|8304.45|1449.10| +2451466|59554|9980|50005|192456|3260|39311|7|297|1329|63|68.20|128.89|77.33|0.00|4871.79|4296.60|8120.07|0.00|0.00|4871.79|4871.79|575.19| +2451466|59554|14386|50005|192456|3260|39311|7|217|1329|72|6.82|12.07|8.81|0.00|634.32|491.04|869.04|57.08|0.00|634.32|691.40|143.28| +2451466|59554|1370|50005|192456|3260|39311|7|180|1329|23|85.33|162.12|84.30|0.00|1938.90|1962.59|3728.76|58.16|0.00|1938.90|1997.06|-23.69| +2451466|59554|7882|50005|192456|3260|39311|7|126|1329|85|27.65|29.86|8.65|0.00|735.25|2350.25|2538.10|36.76|0.00|735.25|772.01|-1615.00| +2451466|59554|10274|50005|192456|3260|39311|7|26|1329|82|68.15|114.49|0.00|0.00|0.00|5588.30|9388.18|0.00|0.00|0.00|0.00|-5588.30| +2451466|59554|5552|50005|192456|3260|39311|7|58|1329|54|13.68|19.42|12.04|84.52|650.16|738.72|1048.68|22.62|84.52|565.64|588.26|-173.08| +2451466|59554|1318|50005|192456|3260|39311|7|120|1329|55|44.49|80.52|78.90|0.00|4339.50|2446.95|4428.60|260.37|0.00|4339.50|4599.87|1892.55| +2451882|65451|998|56285|1176234|183|9332|1|69|1330|25|36.01|41.05|1.23|23.37|30.75|900.25|1026.25|0.14|23.37|7.38|7.52|-892.87| +2451882|65451|4040|56285|1176234|183|9332|1|292|1330|72|2.88|3.88|1.66|0.00|119.52|207.36|279.36|10.75|0.00|119.52|130.27|-87.84| +2451882|65451|17755|56285|1176234|183|9332|1|239|1330|3|29.64|57.20|52.62|0.00|157.86|88.92|171.60|0.00|0.00|157.86|157.86|68.94| +2451882|65451|4513|56285|1176234|183|9332|1|135|1330|63|51.27|81.51|21.19|0.00|1334.97|3230.01|5135.13|40.04|0.00|1334.97|1375.01|-1895.04| +2451882|65451|16886|56285|1176234|183|9332|1|224|1330|60|44.46|76.47|9.17|170.56|550.20|2667.60|4588.20|26.57|170.56|379.64|406.21|-2287.96| +2451882|65451|1519|56285|1176234|183|9332|1|297|1330|67|68.58|107.67|41.99|0.00|2813.33|4594.86|7213.89|253.19|0.00|2813.33|3066.52|-1781.53| +2451882|65451|7610|56285|1176234|183|9332|1|258|1330|68|80.00|112.80|48.50|0.00|3298.00|5440.00|7670.40|0.00|0.00|3298.00|3298.00|-2142.00| +2451882|65451|12968|56285|1176234|183|9332|1|148|1330|60|23.65|29.32|16.12|0.00|967.20|1419.00|1759.20|0.00|0.00|967.20|967.20|-451.80| +2451882|65451|4622|56285|1176234|183|9332|1|105|1330|55|93.45|112.14|5.60|0.00|308.00|5139.75|6167.70|21.56|0.00|308.00|329.56|-4831.75| +2451882|65451|17309|56285|1176234|183|9332|1|248|1330|84|91.68|173.27|79.70|0.00|6694.80|7701.12|14554.68|133.89|0.00|6694.80|6828.69|-1006.32| +2451882|65451|13717|56285|1176234|183|9332|1|4|1330|84|80.64|113.70|79.59|0.00|6685.56|6773.76|9550.80|401.13|0.00|6685.56|7086.69|-88.20| +2451882|65451|16363|56285|1176234|183|9332|1|276|1330|47|54.69|67.81|2.71|0.00|127.37|2570.43|3187.07|10.18|0.00|127.37|137.55|-2443.06| +2451882|65451|68|56285|1176234|183|9332|1|66|1330|9|27.82|53.13|36.12|299.07|325.08|250.38|478.17|0.26|299.07|26.01|26.27|-224.37| +2451882|65451|16460|56285|1176234|183|9332|1|136|1330|100|89.17|152.48|1.52|0.00|152.00|8917.00|15248.00|10.64|0.00|152.00|162.64|-8765.00| +2451882|65451|889|56285|1176234|183|9332|1|87|1330|24|88.45|153.01|67.32|0.00|1615.68|2122.80|3672.24|64.62|0.00|1615.68|1680.30|-507.12| +2451895|42862|6485|48416|1271250|2531|47012|1|127|1331|11|27.94|40.79|24.47|0.00|269.17|307.34|448.69|18.84|0.00|269.17|288.01|-38.17| +2451895|42862|10637|48416|1271250|2531|47012|1|154|1331|97|29.15|40.22|37.80|0.00|3666.60|2827.55|3901.34|293.32|0.00|3666.60|3959.92|839.05| +2451895|42862|14917|48416|1271250|2531|47012|1|2|1331|8|14.39|24.89|11.69|0.00|93.52|115.12|199.12|3.74|0.00|93.52|97.26|-21.60| +2451895|42862|3596|48416|1271250|2531|47012|1|235|1331|89|55.52|85.50|7.69|0.00|684.41|4941.28|7609.50|34.22|0.00|684.41|718.63|-4256.87| +2451895|42862|16661|48416|1271250|2531|47012|1|177|1331|57|38.23|65.37|49.68|0.00|2831.76|2179.11|3726.09|226.54|0.00|2831.76|3058.30|652.65| +2451895||2105||||47012|1||1331|78|43.50||36.36|0.00|||5157.36||0.00|||-556.92| +2451895|42862|14990|48416|1271250|2531|47012|1|50|1331|18|27.11|34.70|17.00|0.00|306.00|487.98|624.60|9.18|0.00|306.00|315.18|-181.98| +2451895|42862|4771|48416|1271250|2531|47012|1|290|1331|32|85.22|165.32|152.09|0.00|4866.88|2727.04|5290.24|97.33|0.00|4866.88|4964.21|2139.84| +2451895|42862|3991|48416|1271250|2531|47012|1|217|1331|26|89.64|147.00|107.31|0.00|2790.06|2330.64|3822.00|83.70|0.00|2790.06|2873.76|459.42| +2451895|42862|9185|48416|1271250|2531|47012|1|63|1331|81|84.26|164.30|0.00|0.00|0.00|6825.06|13308.30|0.00|0.00|0.00|0.00|-6825.06| +2451895|42862|4295|48416|1271250|2531|47012|1|169|1331|43|19.67|37.37|23.54|991.97|1012.22|845.81|1606.91|1.21|991.97|20.25|21.46|-825.56| +|42862|16525|||||||1331|||94.40|17.93||735.13|||||735.13|786.58|-2265.25| +2451895|42862|2117|48416|1271250|2531|47012|1|14|1331|17|34.64|58.54|36.88|0.00|626.96|588.88|995.18|18.80|0.00|626.96|645.76|38.08| +2451068|64337|13792|14196|703723|6403|45483|4|255|1332|22|32.72|51.37|26.71|0.00|587.62|719.84|1130.14|35.25|0.00|587.62|622.87|-132.22| +2451068|64337|13489|14196|703723|6403|45483|4|183|1332|94|31.43|33.00|12.54|0.00|1178.76|2954.42|3102.00|35.36|0.00|1178.76|1214.12|-1775.66| +2451068|64337|15448|14196|703723|6403|45483|4|173|1332|61|52.79|64.40|63.75|0.00|3888.75|3220.19|3928.40|0.00|0.00|3888.75|3888.75|668.56| +2451068|64337|6136|14196|703723|6403|45483|4|134|1332|22|51.89|75.75|62.11|0.00|1366.42|1141.58|1666.50|68.32|0.00|1366.42|1434.74|224.84| +2451068|64337|337|14196|703723|6403|45483|4|198|1332|97|67.21|117.61|2.35|75.22|227.95|6519.37|11408.17|4.58|75.22|152.73|157.31|-6366.64| +2451068|64337|8599|14196|703723|6403|45483|4|51|1332|80|61.55|83.09|65.64|0.00|5251.20|4924.00|6647.20|367.58|0.00|5251.20|5618.78|327.20| +||7264|14196||||4||1332||26.39|31.93||0.00||1820.91||4.84|0.00|||| +2451068|64337|4208|14196|703723|6403|45483|4|173|1332|26|59.38|78.38|24.29|0.00|631.54|1543.88|2037.88|37.89|0.00|631.54|669.43|-912.34| +2451068|64337|7129|14196|703723|6403|45483|4|60|1332|38|13.55|16.93|1.86|0.00|70.68|514.90|643.34|4.24|0.00|70.68|74.92|-444.22| +2451068|64337|2644|14196|703723|6403|45483|4|127|1332|37|74.10|125.97|11.33|243.14|419.21|2741.70|4660.89|8.80|243.14|176.07|184.87|-2565.63| +2451068|64337|8833|14196|703723|6403|45483|4|280|1332|82|41.25|44.96|4.94|0.00|405.08|3382.50|3686.72|12.15|0.00|405.08|417.23|-2977.42| +2451068|64337|626|14196|703723|6403|45483|4|72|1332|43|15.94|29.96|4.19|0.00|180.17|685.42|1288.28|16.21|0.00|180.17|196.38|-505.25| +2451068|64337|2570|14196|703723|6403|45483|4|207|1332|48|38.37|58.32|20.99|0.00|1007.52|1841.76|2799.36|50.37|0.00|1007.52|1057.89|-834.24| +2451068|64337|10369|14196|703723|6403|45483|4|255|1332|31|17.10|33.51|16.08|0.00|498.48|530.10|1038.81|4.98|0.00|498.48|503.46|-31.62| +2451068|64337|3772|14196|703723|6403|45483|4|144|1332|97|62.58|122.65|98.12|0.00|9517.64|6070.26|11897.05|761.41|0.00|9517.64|10279.05|3447.38| +2451068|64337|2678|14196|703723|6403|45483|4|182|1332|23|52.66|79.51|77.91|0.00|1791.93|1211.18|1828.73|35.83|0.00|1791.93|1827.76|580.75| +2451865|70567|4373|41182|829100|6171|49689|10|65|1333|77|50.55|59.64|37.57|0.00|2892.89|3892.35|4592.28|0.00|0.00|2892.89|2892.89|-999.46| +2451865|70567|16547|41182|829100|6171|49689|10|51|1333|95|74.78|84.50|38.02|0.00|3611.90|7104.10|8027.50|0.00|0.00|3611.90|3611.90|-3492.20| +2451865|70567|17006|41182|829100|6171|49689|10|184|1333|44|95.64|178.84|132.34|0.00|5822.96|4208.16|7868.96|291.14|0.00|5822.96|6114.10|1614.80| +2451865|70567|9589|41182|829100|6171|49689|10|199|1333|90|85.03|133.49|44.05|1942.60|3964.50|7652.70|12014.10|181.97|1942.60|2021.90|2203.87|-5630.80| +2451865|70567|12145|41182|829100|6171|49689|10|121|1333|80|39.21|73.32|19.06|0.00|1524.80|3136.80|5865.60|106.73|0.00|1524.80|1631.53|-1612.00| +2451865|70567|5209|41182|829100|6171|49689|10|273|1333|76|23.03|24.64|2.46|0.00|186.96|1750.28|1872.64|1.86|0.00|186.96|188.82|-1563.32| +2451865|70567|7865|41182|829100|6171|49689|10|216|1333|34|25.03|27.53|17.89|0.00|608.26|851.02|936.02|18.24|0.00|608.26|626.50|-242.76| +2451865|70567|15697|41182|829100|6171|49689|10|81|1333|63|39.47|43.41|3.47|196.74|218.61|2486.61|2734.83|0.65|196.74|21.87|22.52|-2464.74| +2451865|70567|6875|41182|829100|6171|49689|10|93|1333|64|22.00|31.24|4.68|107.82|299.52|1408.00|1999.36|9.58|107.82|191.70|201.28|-1216.30| +2451865|70567|10981|41182|829100|6171|49689|10|234|1333|75|51.56|62.90|60.38|0.00|4528.50|3867.00|4717.50|181.14|0.00|4528.50|4709.64|661.50| +2451865|70567|560|41182|829100|6171|49689|10|69|1333|23|76.50|146.88|139.53|0.00|3209.19|1759.50|3378.24|0.00|0.00|3209.19|3209.19|1449.69| +2451865|70567|11231|41182|829100|6171|49689|10|93|1333|76|15.93|23.09|3.46|0.00|262.96|1210.68|1754.84|10.51|0.00|262.96|273.47|-947.72| +||1189|41182|829100||49689|10|177|1333|86||74.03|2.96|||3979.22||||254.56|257.10|-3724.66| +||12743||829100||49689|10||1333|51|||41.13|1950.79||||11.74|1950.79|146.84||-3014.65| +2451865||1520|41182|829100|6171||10||1333|94|52.04||||5174.70|4891.76|||||5329.94|282.94| +2451865|70567|12374|41182|829100|6171|49689|10|127|1333|25|65.55|69.48|34.04|0.00|851.00|1638.75|1737.00|51.06|0.00|851.00|902.06|-787.75| +2451528|59513|10765|30511|780078|45|21259|7|186|1334|66|36.11|61.02|15.86|680.39|1046.76|2383.26|4027.32|7.32|680.39|366.37|373.69|-2016.89| +2451528|59513|8056|30511|780078|45|21259|7|53|1334|34|67.11|127.50|116.02|0.00|3944.68|2281.74|4335.00|276.12|0.00|3944.68|4220.80|1662.94| +2451528|59513|1634|30511|780078|45|21259|7|70|1334|23|40.18|72.32|5.78|0.00|132.94|924.14|1663.36|9.30|0.00|132.94|142.24|-791.20| +2451528|59513|9830|30511|780078|45|21259|7|59|1334|33|68.05|76.21|41.15|0.00|1357.95|2245.65|2514.93|13.57|0.00|1357.95|1371.52|-887.70| +2451528|59513|2716|30511|780078|45|21259|7|294|1334|78|74.48|97.56|38.04|0.00|2967.12|5809.44|7609.68|0.00|0.00|2967.12|2967.12|-2842.32| +2451528|59513|13465|30511|780078|45|21259|7|84|1334|89|16.03|28.37|15.03|0.00|1337.67|1426.67|2524.93|26.75|0.00|1337.67|1364.42|-89.00| +2451528|59513|8050|30511|780078|45|21259|7|2|1334|41|66.50|91.10|50.10|0.00|2054.10|2726.50|3735.10|0.00|0.00|2054.10|2054.10|-672.40| +2451528|59513|7222|30511|780078|45|21259|7|131|1334|93|54.36|66.86|63.51|0.00|5906.43|5055.48|6217.98|295.32|0.00|5906.43|6201.75|850.95| +2451911|43525|731|61523|1885880|4990|12947|1|135|1335|24|91.47|174.70|41.92|0.00|1006.08|2195.28|4192.80|60.36|0.00|1006.08|1066.44|-1189.20| +2451911|43525|2681|61523|1885880|4990|12947|1|170|1335|21|97.04|137.79|73.02|644.03|1533.42|2037.84|2893.59|53.36|644.03|889.39|942.75|-1148.45| +2451911|43525|10419|61523|1885880|4990|12947|1|127|1335|57|49.07|90.28|12.63|0.00|719.91|2796.99|5145.96|57.59|0.00|719.91|777.50|-2077.08| +2451911|43525|16575|61523|1885880|4990|12947|1|206|1335|59|44.22|45.10|42.84|0.00|2527.56|2608.98|2660.90|75.82|0.00|2527.56|2603.38|-81.42| +2451911|43525|7565|61523||||1||1335|69|59.47|76.12|32.73|0.00|2258.37|4103.43|||0.00|||-1845.06| +2451911|43525|17927|61523|1885880|4990|12947|1|39|1335|62|39.01|51.10|5.62|0.00|348.44|2418.62|3168.20|17.42|0.00|348.44|365.86|-2070.18| +2451911|43525|12351|61523|1885880|4990|12947|1|38|1335|73|29.89|58.88|48.87|0.00|3567.51|2181.97|4298.24|107.02|0.00|3567.51|3674.53|1385.54| +2451911|43525|1439|61523|1885880|4990|12947|1|174|1335|40|74.17|97.16|87.44|0.00|3497.60|2966.80|3886.40|69.95|0.00|3497.60|3567.55|530.80| +2451911|43525|4387|61523|1885880|4990|12947|1|131|1335|7|1.54|1.95|0.68|0.00|4.76|10.78|13.65|0.28|0.00|4.76|5.04|-6.02| +2451911|43525|4849|61523|1885880|4990|12947|1|244|1335|76|81.95|152.42|91.45|0.00|6950.20|6228.20|11583.92|278.00|0.00|6950.20|7228.20|722.00| +2451911|43525|6925|61523|1885880|4990|12947|1|41|1335|61|61.25|117.60|74.08|0.00|4518.88|3736.25|7173.60|406.69|0.00|4518.88|4925.57|782.63| +2452132|61439|5929|34435|783046|3657|10315|7|126|1336|82|82.50|132.82|50.47|1448.48|4138.54|6765.00|10891.24|242.10|1448.48|2690.06|2932.16|-4074.94| +||16671|34435|||||285|1336|36|24.85||28.65|938.57|1031.40|894.60|1413.36|0.92|938.57|92.83||-801.77| +2452132|61439|17119|34435|783046|3657|10315|7|221|1336|28|90.37|142.78|45.68|0.00|1279.04|2530.36|3997.84|63.95|0.00|1279.04|1342.99|-1251.32| +2452132|61439|17545|34435|783046|3657|10315|7|27|1336|2|65.88|67.85|60.38|0.00|120.76|131.76|135.70|6.03|0.00|120.76|126.79|-11.00| +2452132|61439|14127|34435|783046|3657|10315|7|277|1336|68|92.18|140.11|88.26|0.00|6001.68|6268.24|9527.48|60.01|0.00|6001.68|6061.69|-266.56| +2452132|61439|7629|34435|783046|3657|10315|7|72|1336|54|86.95|171.29|68.51|0.00|3699.54|4695.30|9249.66|258.96|0.00|3699.54|3958.50|-995.76| +2452132|61439|8725|34435|783046|3657|10315|7|45|1336|21|10.66|20.78|2.28|37.82|47.88|223.86|436.38|0.50|37.82|10.06|10.56|-213.80| +|61439|1673||783046||||116|1336|4||45.56||0.00|||||0.00|3.64|3.74|-104.20| +2452132||7949||783046|3657||7||1336|59||||0.00||||326.93|0.00|||-2008.95| +2452132|61439|3147|34435|783046|3657|10315|7|57|1336|10|71.70|122.60|31.87|0.00|318.70|717.00|1226.00|6.37|0.00|318.70|325.07|-398.30| +2452132|61439|10783|34435|783046|3657|10315|7|138|1336|29|92.78|141.02|50.76|279.68|1472.04|2690.62|4089.58|71.54|279.68|1192.36|1263.90|-1498.26| +2452132|61439|15703|34435|783046|3657|10315|7|115|1336|68|37.17|52.78|50.14|0.00|3409.52|2527.56|3589.04|0.00|0.00|3409.52|3409.52|881.96| +2452132|61439|6007|34435|783046|3657|10315|7|143|1336|87|6.13|11.76|9.76|0.00|849.12|533.31|1023.12|42.45|0.00|849.12|891.57|315.81| +2452132|61439|11725|34435|783046|3657|10315|7|67|1336|58|39.25|60.05|28.22|0.00|1636.76|2276.50|3482.90|114.57|0.00|1636.76|1751.33|-639.74| +2452132|61439|16589|34435|783046|3657|10315|7|161|1336|63|89.62|153.25|21.45|0.00|1351.35|5646.06|9654.75|0.00|0.00|1351.35|1351.35|-4294.71| +2451805|35090|12767|67065|503642|1819|33549|2|288|1337|70|52.29|66.40|12.61|0.00|882.70|3660.30|4648.00|17.65|0.00|882.70|900.35|-2777.60| +2451805|35090|1691|67065|503642|1819|33549|2|218|1337|81|89.37|164.44|41.11|0.00|3329.91|7238.97|13319.64|133.19|0.00|3329.91|3463.10|-3909.06| +2451805|35090|14333|67065|503642|1819|33549|2|151|1337|97|53.31|91.16|82.04|0.00|7957.88|5171.07|8842.52|397.89|0.00|7957.88|8355.77|2786.81| +2451805|35090|15373|67065|503642|1819|33549|2|75|1337|97|21.75|36.10|31.76|0.00|3080.72|2109.75|3501.70|30.80|0.00|3080.72|3111.52|970.97| +2451805|35090|8372|67065|503642|1819|33549|2|25|1337|11|89.84|109.60|71.24|0.00|783.64|988.24|1205.60|70.52|0.00|783.64|854.16|-204.60| +2451805|35090|15805|67065|503642|1819|33549|2|265|1337|33|34.68|63.81|55.51|0.00|1831.83|1144.44|2105.73|36.63|0.00|1831.83|1868.46|687.39| +2451805|35090|5507|67065|503642|1819|33549|2|62|1337|90|43.46|58.67|28.16|582.91|2534.40|3911.40|5280.30|156.11|582.91|1951.49|2107.60|-1959.91| +2451805|35090|6907|67065|503642|1819|33549|2|30|1337|18|53.08|89.17|48.15|0.00|866.70|955.44|1605.06|26.00|0.00|866.70|892.70|-88.74| +2451805|35090|6103|67065|503642|1819|33549|2|85|1337|61|20.72|24.03|9.13|0.00|556.93|1263.92|1465.83|11.13|0.00|556.93|568.06|-706.99| +2451805|35090|14075|67065|503642|1819|33549|2|35|1337|23|36.68|69.32|4.85|0.00|111.55|843.64|1594.36|8.92|0.00|111.55|120.47|-732.09| +|35090|11699|67065|503642||33549|2||1337|28|||0.68||19.04||240.80|||||-123.48| +2451272|42346|15892|48578|1060888|3951|16261|4|89|1338|95|23.53|25.64|20.76|0.00|1972.20|2235.35|2435.80|177.49|0.00|1972.20|2149.69|-263.15| +2451272|42346|13088|48578|1060888|3951|16261|4|176|1338|34|94.73|186.61|139.95|0.00|4758.30|3220.82|6344.74|0.00|0.00|4758.30|4758.30|1537.48| +2451272|42346|11446|48578|1060888|3951|16261|4|194|1338|16|3.60|4.89|4.89|0.00|78.24|57.60|78.24|6.25|0.00|78.24|84.49|20.64| +2451272|42346|15104|48578|1060888|3951|16261|4|110|1338|23|46.43|91.46|57.61|0.00|1325.03|1067.89|2103.58|39.75|0.00|1325.03|1364.78|257.14| +||1004|48578|1060888||16261||300|1338||11.73||11.48|0.00|746.20||||0.00|||-16.25| +2451272||3530|48578||||4||1338|16||8.88||9.20||||5.29|9.20||111.13|| +2451272|42346|9536|48578|1060888|3951|16261|4|132|1338|16|6.61|7.40|0.51|0.00|8.16|105.76|118.40|0.57|0.00|8.16|8.73|-97.60| +2451272|42346|2284|48578|1060888|3951|16261|4|22|1338|73|72.51|87.73|26.31|0.00|1920.63|5293.23|6404.29|115.23|0.00|1920.63|2035.86|-3372.60| +2451272|42346|16052|48578|1060888|3951|16261|4|163|1338|81|49.17|77.68|59.03|2342.90|4781.43|3982.77|6292.08|219.46|2342.90|2438.53|2657.99|-1544.24| +2451272|42346|3733|48578|1060888|3951|16261|4|122|1338|13|49.89|88.30|13.24|0.00|172.12|648.57|1147.90|13.76|0.00|172.12|185.88|-476.45| +2451272|42346|7840|48578|1060888|3951|16261|4|114|1338|48|53.79|94.13|80.01|0.00|3840.48|2581.92|4518.24|307.23|0.00|3840.48|4147.71|1258.56| +2451272|42346|5293|48578|1060888|3951|16261|4|228|1338|55|54.38|73.95|56.94|0.00|3131.70|2990.90|4067.25|281.85|0.00|3131.70|3413.55|140.80| +||4519||181925||42314|1|160|1339||67.48|71.52|68.65||3157.90|3104.08||||3157.90|3157.90|53.82| +2452392|31106|7465|55554|181925|296|42314|1|254|1339|16|20.14|37.25|15.64|0.00|250.24|322.24|596.00|7.50|0.00|250.24|257.74|-72.00| +2452392|31106|5779|55554|181925|296|42314|1|217|1339|42|66.05|119.55|94.44|0.00|3966.48|2774.10|5021.10|277.65|0.00|3966.48|4244.13|1192.38| +2452392|31106|3288|55554|181925|296|42314|1|233|1339|79|65.06|74.16|17.79|1264.86|1405.41|5139.74|5858.64|12.64|1264.86|140.55|153.19|-4999.19| +||13009|||296||1||1339|63|71.46||130.87|||4501.98|8328.60|329.79|||8574.60|3742.83| +2452392|31106|16425|55554|181925|296|42314|1|16|1339|27|1.08|1.94|1.45|29.36|39.15|29.16|52.38|0.78|29.36|9.79|10.57|-19.37| +2452392|31106|9505|55554|181925|296|42314|1|225|1339|3|97.06|178.59|75.00|54.00|225.00|291.18|535.77|10.26|54.00|171.00|181.26|-120.18| +2452392|31106|4398|55554|181925|296|42314|1|271|1339|75|77.65|79.97|32.78|639.21|2458.50|5823.75|5997.75|90.96|639.21|1819.29|1910.25|-4004.46| +2452392||690|55554|181925|296|42314||299|1339||10.36|16.26||597.77||766.64|1203.24|2.56|597.77|256.19||-510.45| +2452392||1195|55554|181925||42314|||1339|2|56.84|79.00|23.70||||158.00|2.84||47.40||| +2452392|31106|7495|55554|181925|296|42314|1|188|1339|66|78.49|146.77|49.90|0.00|3293.40|5180.34|9686.82|65.86|0.00|3293.40|3359.26|-1886.94| +2452392|31106|1239|55554|181925|296|42314|1|141|1339|83|39.11|46.14|44.75|0.00|3714.25|3246.13|3829.62|185.71|0.00|3714.25|3899.96|468.12| +2452392|31106|10086|55554|181925|296|42314|1|99|1339|83|1.74|2.33|1.28|0.00|106.24|144.42|193.39|7.43|0.00|106.24|113.67|-38.18| +2452498|59953|3387|9622|898760|5498|38683|4|97|1340|61|32.88|62.14|0.00|0.00|0.00|2005.68|3790.54|0.00|0.00|0.00|0.00|-2005.68| +2452498|59953|13801|9622|898760|5498|38683|4|291|1340|17|90.72|116.12|110.31|1593.97|1875.27|1542.24|1974.04|16.87|1593.97|281.30|298.17|-1260.94| +2452498|59953|17292|9622|898760|5498|38683|4|156|1340|18|43.27|77.88|18.69|53.82|336.42|778.86|1401.84|14.13|53.82|282.60|296.73|-496.26| +2452498|59953|402|9622|898760|5498|38683|4|201|1340|52|42.75|75.24|66.96|0.00|3481.92|2223.00|3912.48|243.73|0.00|3481.92|3725.65|1258.92| +2452498|59953|11425|9622|898760|5498|38683|4|69|1340|74|98.36|115.08|93.21|0.00|6897.54|7278.64|8515.92|551.80|0.00|6897.54|7449.34|-381.10| +2452498|59953|13533|9622|898760|5498|38683|4|262|1340|55|84.02|94.94|81.64|1347.06|4490.20|4621.10|5221.70|62.86|1347.06|3143.14|3206.00|-1477.96| +2452498|59953|16530|9622|898760|5498|38683|4|276|1340|12|95.02|181.48|30.85|118.46|370.20|1140.24|2177.76|22.65|118.46|251.74|274.39|-888.50| +2452498|59953|16224|9622|898760|5498|38683|4|72|1340|70|71.94|102.87|51.43|0.00|3600.10|5035.80|7200.90|36.00|0.00|3600.10|3636.10|-1435.70| +2452381|39434|3183|17305|134843|5981|2319|10|277|1341|58|34.26|35.97|12.58|0.00|729.64|1987.08|2086.26|51.07|0.00|729.64|780.71|-1257.44| +2452381||1269|17305|134843|5981|2319||38|1341||60.22|101.16|42.48||3186.00|4516.50||0.00|||3186.00|-1330.50| +2452381|39434|12888|17305|134843|5981|2319|10|295|1341|84|22.37|23.26|17.44|1245.21|1464.96|1879.08|1953.84|6.59|1245.21|219.75|226.34|-1659.33| +2452381|39434|13611|17305|134843|5981|2319|10|116|1341|80|13.64|19.50|15.60|0.00|1248.00|1091.20|1560.00|74.88|0.00|1248.00|1322.88|156.80| +2452381|39434|13362|17305|134843|5981|2319|10|76|1341|99|39.43|48.89|48.89|0.00|4840.11|3903.57|4840.11|193.60|0.00|4840.11|5033.71|936.54| +2452381|39434|16297|17305|134843|5981|2319|10|119|1341|76|31.84|32.79|24.26|1180.00|1843.76|2419.84|2492.04|53.10|1180.00|663.76|716.86|-1756.08| +2452381|39434|16482|17305|134843|5981|2319|10|47|1341|15|52.46|89.70|70.86|584.59|1062.90|786.90|1345.50|9.56|584.59|478.31|487.87|-308.59| +2452381|39434|15013|17305|134843|5981|2319|10|128|1341|62|26.64|39.96|23.17|0.00|1436.54|1651.68|2477.52|129.28|0.00|1436.54|1565.82|-215.14| +2452381|39434|4995|17305|134843|5981|2319|10|34|1341|84|14.63|25.01|13.75|0.00|1155.00|1228.92|2100.84|46.20|0.00|1155.00|1201.20|-73.92| +2452381|39434|16755|17305|134843|5981|2319|10|2|1341|48|43.67|64.19|50.06|0.00|2402.88|2096.16|3081.12|48.05|0.00|2402.88|2450.93|306.72| +2452381||5634|||5981|2319|10|171|1341||41.07|||||862.47||32.32||404.04||| +2452381|39434|17727|17305|134843|5981|2319|10|194|1341|39|48.53|73.28|41.03|0.00|1600.17|1892.67|2857.92|80.00|0.00|1600.17|1680.17|-292.50| +2452381|39434|7704|17305|134843|5981|2319|10|63|1341|23|4.65|4.88|4.83|0.00|111.09|106.95|112.24|1.11|0.00|111.09|112.20|4.14| +2452381|39434|10905|17305|134843|5981|2319|10|99|1341|87|96.55|98.48|82.72|0.00|7196.64|8399.85|8567.76|647.69|0.00|7196.64|7844.33|-1203.21| +2452381|39434|9691|17305|134843|5981|2319|||1341|||144.41||||1947.12||||1698.24||| +2452381|39434|8208|17305|134843|5981|2319|10|246|1341|44|83.20|91.52|63.14|2555.90|2778.16|3660.80|4026.88|11.11|2555.90|222.26|233.37|-3438.54| +2451891||14941||656860|4231|||54|1342|56||45.06|5.85||327.60|1484.56||29.48||||-1156.96| +2451891|62317|11636|21936|656860|4231|11602|8|196|1342|43|81.81|114.53|103.07|0.00|4432.01|3517.83|4924.79|310.24|0.00|4432.01|4742.25|914.18| +2451891|62317|2501|21936|656860|4231|11602|8|226|1342|67|70.03|105.74|27.49|0.00|1841.83|4692.01|7084.58|55.25|0.00|1841.83|1897.08|-2850.18| +2451891|62317|14485|21936|656860|4231|11602|8|8|1342|29|92.97|122.72|84.67|0.00|2455.43|2696.13|3558.88|171.88|0.00|2455.43|2627.31|-240.70| +2451891|62317|5899|21936|656860|4231|11602|8|209|1342|10|76.70|86.67|16.46|0.00|164.60|767.00|866.70|9.87|0.00|164.60|174.47|-602.40| +2451891|62317|8989|21936|656860|4231|11602|8|275|1342|83|86.63|104.82|100.62|0.00|8351.46|7190.29|8700.06|584.60|0.00|8351.46|8936.06|1161.17| +2451891|62317|1061|21936|656860|4231|11602|8|57|1342|26|93.35|151.22|101.31|0.00|2634.06|2427.10|3931.72|237.06|0.00|2634.06|2871.12|206.96| +2451891|62317|5639|21936|656860|4231|11602|8|246|1342|53|65.22|121.30|116.44|0.00|6171.32|3456.66|6428.90|123.42|0.00|6171.32|6294.74|2714.66| +2451891|62317|17009|21936|656860|4231|11602|8|114|1342|83|7.19|14.16|10.90|0.00|904.70|596.77|1175.28|9.04|0.00|904.70|913.74|307.93| +2451891|62317|15029|21936|656860|4231|11602|8|254|1342|84|48.82|52.72|44.28|0.00|3719.52|4100.88|4428.48|148.78|0.00|3719.52|3868.30|-381.36| +2451891|62317|12110|21936|656860|4231|11602|8|2|1342|71|51.20|98.81|87.94|1623.37|6243.74|3635.20|7015.51|369.62|1623.37|4620.37|4989.99|985.17| +2451891|62317|17911|21936|656860|4231|11602|8|292|1342|85|6.22|11.07|9.18|0.00|780.30|528.70|940.95|39.01|0.00|780.30|819.31|251.60| +2451891|62317|2341|21936|656860|4231|11602|8|229|1342|79|36.85|39.42|11.82|840.40|933.78|2911.15|3114.18|4.66|840.40|93.38|98.04|-2817.77| +|62317|835|21936|656860|4231|11602|||1342||16.04|||340.82||513.28||6.30|340.82|126.06||-387.22| +2451891|62317|13934|21936|656860|4231|11602|8|153|1342|4|83.20|119.80|113.81|0.00|455.24|332.80|479.20|22.76|0.00|455.24|478.00|122.44| +2450883|45720|9325|21019|976315|2011|14060|10|203|1343|55|90.79|107.13|101.77|0.00|5597.35|4993.45|5892.15|447.78|0.00|5597.35|6045.13|603.90| +2450883|45720|9766|21019|976315|2011|14060|10|22|1343|4|10.94|16.08|6.11|0.00|24.44|43.76|64.32|0.48|0.00|24.44|24.92|-19.32| +2450883|45720|12512|21019|976315|2011|14060|10|250|1343|100|66.17|91.31|74.87|0.00|7487.00|6617.00|9131.00|673.83|0.00|7487.00|8160.83|870.00| +2450883|45720|1717|21019|976315|2011|14060|10|232|1343|86|30.67|59.80|22.12|0.00|1902.32|2637.62|5142.80|133.16|0.00|1902.32|2035.48|-735.30| +2450883|45720|6193|21019|976315|2011|14060|10|236|1343|32|21.89|31.30|13.45|0.00|430.40|700.48|1001.60|21.52|0.00|430.40|451.92|-270.08| +2450883|45720|12290|21019|976315|2011|14060|10|251|1343|32|28.38|38.02|5.32|0.00|170.24|908.16|1216.64|6.80|0.00|170.24|177.04|-737.92| +2450883|45720|1526|21019|976315|2011|14060|10|280|1343|74|28.78|50.65|29.88|0.00|2211.12|2129.72|3748.10|66.33|0.00|2211.12|2277.45|81.40| +2450883|45720|12016|21019|976315|2011|14060|10|21|1343|36|40.88|53.96|16.72|0.00|601.92|1471.68|1942.56|24.07|0.00|601.92|625.99|-869.76| +2450883|45720|16324|21019|976315|2011|14060|10|36|1343|100|10.31|20.62|1.64|0.00|164.00|1031.00|2062.00|0.00|0.00|164.00|164.00|-867.00| +2450883|45720|4118|21019|976315|2011|14060|10|265|1343|38|46.90|74.10|27.41|0.00|1041.58|1782.20|2815.80|62.49|0.00|1041.58|1104.07|-740.62| +2450883|45720|5197|21019|976315|2011|14060|10|180|1343|21|88.24|94.41|24.54|0.00|515.34|1853.04|1982.61|46.38|0.00|515.34|561.72|-1337.70| +2450883||2312||||14060|||1343||||10.13||1002.87||1497.87|||1002.87||| +2452153|36303|499|61069|1325873|2677|33481|2|231|1344|2|32.59|38.45|15.76|0.00|31.52|65.18|76.90|0.00|0.00|31.52|31.52|-33.66| +2452153|36303|13353||||||204|1344|||29.71|||1047.48||2495.64|17.28||345.67||| +2452153|36303|13227|61069|1325873|2677|33481|2|83|1344|48|46.44|64.08|31.39|813.62|1506.72|2229.12|3075.84|34.65|813.62|693.10|727.75|-1536.02| +2452153|36303|12701|61069|1325873|2677|33481|2|67|1344|24|98.31|174.99|117.24|0.00|2813.76|2359.44|4199.76|28.13|0.00|2813.76|2841.89|454.32| +2452153|36303|1175|61069|1325873|2677|33481|2|94|1344|53|85.75|96.89|75.57|0.00|4005.21|4544.75|5135.17|40.05|0.00|4005.21|4045.26|-539.54| +2452153|36303|14379|61069|1325873|2677|33481|2|237|1344|80|76.41|98.56|96.58|3785.93|7726.40|6112.80|7884.80|197.02|3785.93|3940.47|4137.49|-2172.33| +2452153|36303|17807|61069|1325873|2677|33481|2|193|1344|70|81.22|93.40|55.10|0.00|3857.00|5685.40|6538.00|347.13|0.00|3857.00|4204.13|-1828.40| +2452153|36303|459|61069|1325873|2677|33481|2|137|1344|42|70.99|112.87|94.81|0.00|3982.02|2981.58|4740.54|119.46|0.00|3982.02|4101.48|1000.44| +2452171|61697|3067|51122|1547994|513|6585|10|99|1345|37|50.92|67.72|58.91|0.00|2179.67|1884.04|2505.64|43.59|0.00|2179.67|2223.26|295.63| +2452171|61697|8351|51122|1547994|513|6585|10|187|1345|36|80.73|117.05|69.05|0.00|2485.80|2906.28|4213.80|24.85|0.00|2485.80|2510.65|-420.48| +2452171|61697|2455|51122|1547994|513|6585|10|168|1345|13|65.16|122.50|50.22|0.00|652.86|847.08|1592.50|58.75|0.00|652.86|711.61|-194.22| +2452171|61697|13275|51122|1547994|513|6585|10|20|1345|90|29.44|52.10|7.29|0.00|656.10|2649.60|4689.00|13.12|0.00|656.10|669.22|-1993.50| +2452171|61697|15557|51122|1547994|513|6585|10|273|1345|96|16.41|31.99|31.99|0.00|3071.04|1575.36|3071.04|214.97|0.00|3071.04|3286.01|1495.68| +2452171|61697|3687|51122|1547994|513|6585|10|279|1345|65|33.17|37.81|31.38|0.00|2039.70|2156.05|2457.65|20.39|0.00|2039.70|2060.09|-116.35| +2452171|61697|13375|51122|1547994|513|6585|10|168|1345|85|6.45|7.28|4.95|0.00|420.75|548.25|618.80|33.66|0.00|420.75|454.41|-127.50| +2452171|61697|2687|||513||10||1345||||50.15|||||||1504.50|1639.90|274.50| +2452171|61697|1565|51122|1547994|513|6585|10|125|1345|69|15.60|28.39|17.03|0.00|1175.07|1076.40|1958.91|11.75|0.00|1175.07|1186.82|98.67| +2451658|46770|607|85806|1394566|1726|41286|7|59|1346|64|2.93|4.36|2.04|83.55|130.56|187.52|279.04|4.23|83.55|47.01|51.24|-140.51| +2451658|46770|4625|85806|1394566|1726|41286|7|239|1346|24|27.81|53.39|9.07|0.00|217.68|667.44|1281.36|4.35|0.00|217.68|222.03|-449.76| +2451658|46770|10217|85806|1394566|1726|41286|7|32|1346|7|10.74|12.13|11.52|55.64|80.64|75.18|84.91|2.25|55.64|25.00|27.25|-50.18| +2451658|46770|2948|85806|1394566|1726|41286|7|249|1346|48|22.51|28.81|1.72|0.00|82.56|1080.48|1382.88|0.82|0.00|82.56|83.38|-997.92| +2451658|46770|6938|85806|1394566|1726|41286|7|89|1346|84|46.93|91.51|31.11|0.00|2613.24|3942.12|7686.84|130.66|0.00|2613.24|2743.90|-1328.88| +2451658|46770|9905|85806|1394566|1726|41286|7|94|1346|3|84.67|103.29|35.11|0.00|105.33|254.01|309.87|3.15|0.00|105.33|108.48|-148.68| +2451658|46770|6953|85806|1394566|1726|41286|7|127|1346|78|95.04|114.99|77.04|0.00|6009.12|7413.12|8969.22|0.00|0.00|6009.12|6009.12|-1404.00| +2451658|46770|11039|85806|1394566|1726|41286|7|3|1346|67|36.43|60.10|29.44|0.00|1972.48|2440.81|4026.70|78.89|0.00|1972.48|2051.37|-468.33| +2451658|46770|16796|85806|1394566|1726|41286|7|27|1346|97|88.08|154.14|117.14|0.00|11362.58|8543.76|14951.58|568.12|0.00|11362.58|11930.70|2818.82| +2451658|46770|1550|85806|1394566|1726|41286|7|196|1346|74|77.42|130.83|0.00|0.00|0.00|5729.08|9681.42|0.00|0.00|0.00|0.00|-5729.08| +2451658|46770|12773|85806|1394566|1726|41286|7|294|1346|79|57.91|60.22|14.45|0.00|1141.55|4574.89|4757.38|102.73|0.00|1141.55|1244.28|-3433.34| +2451658|46770|5467|85806|1394566|1726|41286|7|10|1346|69|75.92|129.82|103.85|0.00|7165.65|5238.48|8957.58|573.25|0.00|7165.65|7738.90|1927.17| +2451658|46770|8816|85806|1394566|1726|41286|7|99|1346|90|84.11|142.14|58.27|734.20|5244.30|7569.90|12792.60|135.30|734.20|4510.10|4645.40|-3059.80| +2451658|46770|7028|85806|1394566|1726|41286|7|297|1346|17|14.31|18.88|10.76|0.00|182.92|243.27|320.96|16.46|0.00|182.92|199.38|-60.35| +2452062|36281|9587|99300|130204|4308|7612|7|60|1347|53|94.52|116.25|88.35|0.00|4682.55|5009.56|6161.25|421.42|0.00|4682.55|5103.97|-327.01| +2452062|36281|3719|99300|130204|4308|7612|7|86|1347|29|28.08|51.10|30.66|0.00|889.14|814.32|1481.90|71.13|0.00|889.14|960.27|74.82| +2452062|36281|7057|99300|130204|4308|7612|7|116|1347|55|76.91|126.90|55.83|0.00|3070.65|4230.05|6979.50|0.00|0.00|3070.65|3070.65|-1159.40| +2452062|36281|2831|99300|130204|4308|7612|7|75|1347|90|84.43|163.79|80.25|0.00|7222.50|7598.70|14741.10|0.00|0.00|7222.50|7222.50|-376.20| +2452062|36281|12401|99300|130204|4308|7612|7|195|1347|39|80.18|150.73|147.71|0.00|5760.69|3127.02|5878.47|518.46|0.00|5760.69|6279.15|2633.67| +2452062|36281|4457|99300|130204|4308|7612|7|230|1347|71|26.25|50.13|28.57|0.00|2028.47|1863.75|3559.23|81.13|0.00|2028.47|2109.60|164.72| +2452062|36281|14633|99300|130204|4308|7612|7|292|1347|60|82.11|145.33|88.65|4999.86|5319.00|4926.60|8719.80|9.57|4999.86|319.14|328.71|-4607.46| +2452062|36281|10669|99300|130204|4308|7612|7|224|1347|75|57.66|80.72|24.21|1143.92|1815.75|4324.50|6054.00|0.00|1143.92|671.83|671.83|-3652.67| +2452062|36281|3431|99300|130204|4308|7612|7|220|1347|68|29.61|55.66|6.12|0.00|416.16|2013.48|3784.88|4.16|0.00|416.16|420.32|-1597.32| +2452062|36281|12099|99300|130204|4308|7612|7|222|1347|68|71.62|130.34|126.42|0.00|8596.56|4870.16|8863.12|429.82|0.00|8596.56|9026.38|3726.40| +2452062|36281|7931|99300|130204|4308|7612|7|214|1347|54|2.25|3.37|1.44|20.21|77.76|121.50|181.98|5.17|20.21|57.55|62.72|-63.95| +2452062|36281|9755|99300|130204|4308|7612|7|84|1347|23|1.49|2.32|0.20|0.00|4.60|34.27|53.36|0.00|0.00|4.60|4.60|-29.67| +2451156|45300|3769|14056|907173|5443|1713|8|128|1348|71|41.15|53.49|20.32|0.00|1442.72|2921.65|3797.79|14.42|0.00|1442.72|1457.14|-1478.93| +2451156|45300|6404|14056|907173|5443|1713|8|293|1348|56|13.29|17.40|3.65|0.00|204.40|744.24|974.40|12.26|0.00|204.40|216.66|-539.84| +2451156|45300|13430|14056|907173|5443|1713|8|111|1348|57|64.81|99.80|93.81|0.00|5347.17|3694.17|5688.60|481.24|0.00|5347.17|5828.41|1653.00| +2451156|45300|13124|14056|907173|5443|1713|8|267|1348|73|6.00|8.88|3.99|0.00|291.27|438.00|648.24|11.65|0.00|291.27|302.92|-146.73| +2451156|45300|16414|14056|907173|5443|1713|8|298|1348|47|31.86|34.40|30.27|0.00|1422.69|1497.42|1616.80|99.58|0.00|1422.69|1522.27|-74.73| +2451156|45300|1316|14056|907173|5443|1713|8|202|1348|86|96.88|175.35|57.86|0.00|4975.96|8331.68|15080.10|398.07|0.00|4975.96|5374.03|-3355.72| +|45300|13708|14056|907173||1713|8|128|1348|||47.79|45.40||3359.60|2068.30|3536.46|134.38||3359.60|3493.98|| +2451156|45300|416|14056|907173|5443|1713|8|298|1348|58|78.69|124.33|44.75|0.00|2595.50|4564.02|7211.14|155.73|0.00|2595.50|2751.23|-1968.52| +2451156|45300|14420|14056|907173|5443|1713|8|46|1348|19|46.83|60.41|25.97|0.00|493.43|889.77|1147.79|9.86|0.00|493.43|503.29|-396.34| +2451156|45300|6949|14056|907173|5443|1713|8|244|1348|53|82.54|85.84|80.68|0.00|4276.04|4374.62|4549.52|42.76|0.00|4276.04|4318.80|-98.58| +2452165|61451|16305|68316|258402|488|24974|2|51|1349|48|1.34|2.29|1.03|0.00|49.44|64.32|109.92|3.95|0.00|49.44|53.39|-14.88| +2452165|61451|7763|68316|258402|488|24974|2|16|1349|85|86.84|86.84|18.23|852.25|1549.55|7381.40|7381.40|34.86|852.25|697.30|732.16|-6684.10| +2452165|61451|17437|68316|258402|488|24974|2|237|1349|65|95.81|114.97|25.29|0.00|1643.85|6227.65|7473.05|49.31|0.00|1643.85|1693.16|-4583.80| +2452165|61451|3319|68316|258402|488|24974|2|121|1349|87|88.77|95.87|6.71|0.00|583.77|7722.99|8340.69|0.00|0.00|583.77|583.77|-7139.22| +2452165|61451|3045|68316|258402|488|24974|2|144|1349|71|5.25|7.29|4.44|0.00|315.24|372.75|517.59|12.60|0.00|315.24|327.84|-57.51| +2452165|61451|14543|68316|258402|488|24974|2|93|1349|79|19.27|34.87|11.15|0.00|880.85|1522.33|2754.73|70.46|0.00|880.85|951.31|-641.48| +2452165|61451|4391|68316|258402|488|24974|2|226|1349|92|11.49|17.80|11.03|0.00|1014.76|1057.08|1637.60|91.32|0.00|1014.76|1106.08|-42.32| +2452165|61451|14877|68316|258402|488|24974|2|95|1349|67|5.38|6.18|2.84|49.47|190.28|360.46|414.06|0.00|49.47|140.81|140.81|-219.65| +2452165|61451|5019|68316|258402|488|24974|2|208|1349|73|24.29|47.85|28.71|1949.12|2095.83|1773.17|3493.05|2.93|1949.12|146.71|149.64|-1626.46| +2451848|41461|5165|27235|339921|2654|5740|4|235|1350|78|54.43|88.72|81.62|5220.41|6366.36|4245.54|6920.16|68.75|5220.41|1145.95|1214.70|-3099.59| +2451848|41461|9170|27235|339921|2654|5740|4|224|1350|27|35.19|36.94|1.10|0.00|29.70|950.13|997.38|2.37|0.00|29.70|32.07|-920.43| +2451848|41461|16387|27235|339921|2654|5740|4|49|1350|79|90.96|167.36|85.35|3978.16|6742.65|7185.84|13221.44|221.15|3978.16|2764.49|2985.64|-4421.35| +2451848|41461|1868|27235|339921|2654|5740|4|28|1350|54|44.95|75.96|9.87|0.00|532.98|2427.30|4101.84|47.96|0.00|532.98|580.94|-1894.32| +2451848|41461|6499|27235|339921|2654|5740|4|79|1350|74|59.54|114.91|71.24|948.91|5271.76|4405.96|8503.34|216.14|948.91|4322.85|4538.99|-83.11| +2451848|41461|12283|27235|339921|2654|5740|4|15|1350|42|53.08|95.54|52.54|0.00|2206.68|2229.36|4012.68|176.53|0.00|2206.68|2383.21|-22.68| +2451848|41461|7523|27235|339921|2654|5740|4|52|1350|87|75.13|79.63|26.27|0.00|2285.49|6536.31|6927.81|0.00|0.00|2285.49|2285.49|-4250.82| +2451848|41461|12746|27235|339921|2654|5740|4|98|1350|32|89.54|132.51|115.28|0.00|3688.96|2865.28|4240.32|221.33|0.00|3688.96|3910.29|823.68| +2451848|41461|6746|27235|339921|2654|5740|4|168|1350|22|43.32|79.70|6.37|0.00|140.14|953.04|1753.40|1.40|0.00|140.14|141.54|-812.90| +2451848|41461|4382|27235|339921|2654|5740|4|175|1350|66|81.64|146.95|42.61|0.00|2812.26|5388.24|9698.70|0.00|0.00|2812.26|2812.26|-2575.98| +2451848|41461|13022|27235|339921|2654|5740|4|16|1350|79|24.19|32.89|12.49|246.67|986.71|1911.01|2598.31|29.60|246.67|740.04|769.64|-1170.97| +2452268|48278|16173|11859|1171555|3251|10100|7|161|1351|74|5.16|6.19|1.54|23.93|113.96|381.84|458.06|0.90|23.93|90.03|90.93|-291.81| +2452268|48278|4589|11859|1171555|3251|10100|7|258|1351|89|33.66|36.68|26.40|0.00|2349.60|2995.74|3264.52|46.99|0.00|2349.60|2396.59|-646.14| +2452268|48278|17445|11859|1171555|3251|10100|7|12|1351|70|27.33|42.36|15.24|0.00|1066.80|1913.10|2965.20|0.00|0.00|1066.80|1066.80|-846.30| +2452268|48278|2119|11859|1171555|3251|10100|7|78|1351|5|54.83|60.31|37.99|0.00|189.95|274.15|301.55|11.39|0.00|189.95|201.34|-84.20| +2452268|48278|2729|11859|1171555|3251|10100|7|58|1351|3|57.56|73.10|54.82|0.00|164.46|172.68|219.30|8.22|0.00|164.46|172.68|-8.22| +2452268|48278|8491|11859|1171555|3251|10100|7|184|1351|20|98.30|167.11|23.39|0.00|467.80|1966.00|3342.20|18.71|0.00|467.80|486.51|-1498.20| +2452268|48278|3453|11859|1171555|3251|10100|7|60|1351|42|30.13|42.18|33.74|0.00|1417.08|1265.46|1771.56|42.51|0.00|1417.08|1459.59|151.62| +2452268|48278|6821|11859|1171555|3251|10100|7|180|1351|92|6.95|10.14|2.63|0.00|241.96|639.40|932.88|9.67|0.00|241.96|251.63|-397.44| +2451156|70049|17284|74638|1359165|2092|13419|10|54|1352|52|19.38|38.76|36.04|0.00|1874.08|1007.76|2015.52|93.70|0.00|1874.08|1967.78|866.32| +2451156|70049|10591|74638|1359165|2092|13419|10|52|1352|44|13.29|22.99|5.05|0.00|222.20|584.76|1011.56|13.33|0.00|222.20|235.53|-362.56| +2451156|70049|11462|74638|1359165|2092|13419|10|123|1352|65|60.03|100.85|18.15|0.00|1179.75|3901.95|6555.25|47.19|0.00|1179.75|1226.94|-2722.20| +2451156|70049|12362|74638|1359165|2092|13419|10|16|1352|88|17.93|27.97|6.99|159.93|615.12|1577.84|2461.36|9.10|159.93|455.19|464.29|-1122.65| +2451156|70049|10552|74638|1359165|2092|13419|10|44|1352|98|73.25|140.64|45.00|0.00|4410.00|7178.50|13782.72|308.70|0.00|4410.00|4718.70|-2768.50| +2451156|70049|2128|74638|1359165|2092|13419|10|26|1352|84|18.92|22.32|9.59|0.00|805.56|1589.28|1874.88|40.27|0.00|805.56|845.83|-783.72| +2451156|70049|4274|74638|1359165|2092|13419|10|69|1352|9|60.05|108.09|47.55|295.28|427.95|540.45|972.81|0.00|295.28|132.67|132.67|-407.78| +2451156|70049|7888|74638|1359165|2092|13419|10|1|1352|46|9.03|14.71|2.20|0.00|101.20|415.38|676.66|5.06|0.00|101.20|106.26|-314.18| +2451156|70049|16724|74638|1359165|2092|13419|10|202|1352|61|44.64|71.87|42.40|0.00|2586.40|2723.04|4384.07|155.18|0.00|2586.40|2741.58|-136.64| +2451156|70049|17164|74638|1359165|2092|13419|10|144|1352|7|58.15|87.80|5.26|0.00|36.82|407.05|614.60|2.57|0.00|36.82|39.39|-370.23| +2451156|70049|3338|74638|1359165|2092|13419|10|81|1352|89|30.75|31.05|14.28|0.00|1270.92|2736.75|2763.45|88.96|0.00|1270.92|1359.88|-1465.83| +2451156|70049|2206|74638|1359165|2092|13419|10|191|1352|43|24.30|44.22|15.47|0.00|665.21|1044.90|1901.46|13.30|0.00|665.21|678.51|-379.69| +2451091|62931|3290|46220|700866|6359|24605|10|105|1353|66|74.76|82.98|19.91|0.00|1314.06|4934.16|5476.68|65.70|0.00|1314.06|1379.76|-3620.10| +2451091|62931|13244|46220|700866|6359|24605|10|163|1353|73|38.85|75.36|9.04|0.00|659.92|2836.05|5501.28|32.99|0.00|659.92|692.91|-2176.13| +2451091|62931|10354|46220|700866|6359|24605|10|136|1353|82|36.39|66.59|61.26|0.00|5023.32|2983.98|5460.38|50.23|0.00|5023.32|5073.55|2039.34| +2451091|62931|6436|46220|700866|6359|24605|10|108|1353|9|81.81|112.07|77.32|0.00|695.88|736.29|1008.63|55.67|0.00|695.88|751.55|-40.41| +2451091|62931|2647|46220|700866|6359|24605|10|98|1353|18|70.02|91.02|2.73|0.00|49.14|1260.36|1638.36|4.42|0.00|49.14|53.56|-1211.22| +2451091|62931|4591|46220|700866|6359|24605|10|207|1353|5|53.77|54.84|26.87|17.46|134.35|268.85|274.20|9.35|17.46|116.89|126.24|-151.96| +2451091|62931|1936|46220|700866|6359|24605|10|183|1353|54|21.94|42.12|42.12|0.00|2274.48|1184.76|2274.48|68.23|0.00|2274.48|2342.71|1089.72| +2451091|62931|17179|46220|700866|6359|24605|10|131|1353|64|3.32|3.58|0.60|0.00|38.40|212.48|229.12|1.15|0.00|38.40|39.55|-174.08| +2451091|62931|11794|46220|700866|6359|24605|10|43|1353|1|12.21|24.05|18.27|0.00|18.27|12.21|24.05|0.91|0.00|18.27|19.18|6.06| +2451091|62931|11596|46220|700866|6359|24605|10|254|1353|92|53.27|104.40|65.77|0.00|6050.84|4900.84|9604.80|242.03|0.00|6050.84|6292.87|1150.00| +2451091|62931|586|46220|700866|6359|24605|10|286|1353|83|1.94|3.43|1.26|0.00|104.58|161.02|284.69|7.32|0.00|104.58|111.90|-56.44| +2451091|62931|15944|46220|700866|6359|24605|10|70|1353|73|29.29|31.63|2.84|0.00|207.32|2138.17|2308.99|2.07|0.00|207.32|209.39|-1930.85| +2451091||11398|46220|700866||24605|10|212|1353||26.09|51.39|26.20|0.00||||75.45|0.00|2515.20|2590.65|| +2451091|62931|9298|46220|700866|6359|24605|10|130|1353|29|90.62|146.80|135.05|0.00|3916.45|2627.98|4257.20|234.98|0.00|3916.45|4151.43|1288.47| +2452242|64449|9583|10858|1802666|4427|42188|1|246|1354|67|24.57|38.08|3.04|0.00|203.68|1646.19|2551.36|0.00|0.00|203.68|203.68|-1442.51| +2452242|64449|7287|10858|1802666|4427|42188|1|116|1354|56|66.54|109.79|96.61|0.00|5410.16|3726.24|6148.24|378.71|0.00|5410.16|5788.87|1683.92| +2452242|64449|3013|10858|1802666|4427|42188|1|141|1354|42|67.62|104.13|24.99|73.47|1049.58|2840.04|4373.46|9.76|73.47|976.11|985.87|-1863.93| +||14031|10858|1802666|4427||||1354|||61.04|53.71|0.00|3061.47|2899.59|3479.28|153.07|0.00||3214.54|| +2452242|64449|6525|10858|1802666|4427|42188|1|238|1354|4|36.58|40.96|15.56|0.00|62.24|146.32|163.84|1.24|0.00|62.24|63.48|-84.08| +2452242|64449|10025|10858|1802666|4427|42188|1|212|1354|6|5.68|9.59|0.86|0.00|5.16|34.08|57.54|0.15|0.00|5.16|5.31|-28.92| +2452242|64449|14341|10858|1802666|4427|42188|1|296|1354|6|11.27|17.13|10.62|22.93|63.72|67.62|102.78|0.40|22.93|40.79|41.19|-26.83| +2452242|64449|8511|10858|1802666|4427|42188|1|214|1354|97|75.24|94.80|79.63|0.00|7724.11|7298.28|9195.60|154.48|0.00|7724.11|7878.59|425.83| +2452242|64449|905|10858|1802666|4427|42188|1|251|1354|98|27.86|45.69|8.68|0.00|850.64|2730.28|4477.62|8.50|0.00|850.64|859.14|-1879.64| +2452242|64449|17801|10858|1802666|4427|42188|1|259|1354|63|14.75|16.22|15.08|0.00|950.04|929.25|1021.86|0.00|0.00|950.04|950.04|20.79| +2452242|64449|3327|10858|1802666|4427|42188|1|256|1354|57|39.06|49.60|21.82|0.00|1243.74|2226.42|2827.20|74.62|0.00|1243.74|1318.36|-982.68| +2452242|64449|14323|10858|1802666|4427|42188|1|164|1354|19|27.27|35.45|29.77|135.75|565.63|518.13|673.55|38.68|135.75|429.88|468.56|-88.25| +2452242|64449|10701|10858|1802666|4427|42188|1|56|1354|79|84.71|114.35|25.15|0.00|1986.85|6692.09|9033.65|139.07|0.00|1986.85|2125.92|-4705.24| +2452242|64449|17977|10858|1802666|4427|42188|1|9|1354|55|12.90|15.09|1.35|0.00|74.25|709.50|829.95|5.19|0.00|74.25|79.44|-635.25| +2452228|45578|3663|51420|182357|2894|28231|1|231|1355|18|53.55|72.82|3.64|0.00|65.52|963.90|1310.76|0.00|0.00|65.52|65.52|-898.38| +2452228|45578|1453|51420|182357|2894|28231|1|134|1355|79|15.59|27.90|27.06|0.00|2137.74|1231.61|2204.10|0.00|0.00|2137.74|2137.74|906.13| +2452228|45578|9613|51420|182357|2894|28231|1|26|1355|19|6.06|9.21|0.09|0.00|1.71|115.14|174.99|0.01|0.00|1.71|1.72|-113.43| +2452228|45578|5047|51420|182357|2894|28231|1|174|1355|45|66.38|90.27|74.02|1132.50|3330.90|2987.10|4062.15|87.93|1132.50|2198.40|2286.33|-788.70| +2452228|45578|11879|51420|182357|2894|28231|1|288|1355|16|5.99|9.64|5.10|0.00|81.60|95.84|154.24|5.71|0.00|81.60|87.31|-14.24| +2452228|45578|17669|51420|182357|2894|28231|1|157|1355|27|75.55|93.68|74.94|0.00|2023.38|2039.85|2529.36|161.87|0.00|2023.38|2185.25|-16.47| +2452228|45578|5161|51420|182357|2894|28231|1|117|1355|10|57.14|81.13|64.09|0.00|640.90|571.40|811.30|25.63|0.00|640.90|666.53|69.50| +2452228|45578|13453|51420|182357|2894|28231|1|254|1355|86|34.61|60.91|15.22|0.00|1308.92|2976.46|5238.26|52.35|0.00|1308.92|1361.27|-1667.54| +2452228|45578|13647|51420|182357|2894|28231|1|163|1355|91|23.09|40.40|5.65|0.00|514.15|2101.19|3676.40|0.00|0.00|514.15|514.15|-1587.04| +2452228|45578|5391|51420|182357|2894|28231|1|237|1355|45|73.57|92.69|83.42|0.00|3753.90|3310.65|4171.05|75.07|0.00|3753.90|3828.97|443.25| +2452228|45578|2743|51420|182357|2894|28231|1|188|1355|53|21.64|30.29|11.81|0.00|625.93|1146.92|1605.37|50.07|0.00|625.93|676.00|-520.99| +2451597|51069|11243|169|1004112|3255|21998|4|46|1356|52|8.62|11.37|0.45|0.00|23.40|448.24|591.24|0.23|0.00|23.40|23.63|-424.84| +2451597|51069|16874|169|1004112|3255|21998|4|159|1356|73|73.91|132.29|14.55|0.00|1062.15|5395.43|9657.17|53.10|0.00|1062.15|1115.25|-4333.28| +|51069|4082|169||3255|21998||270|1356|79||59.88|||850.83|3032.81||51.04|||901.87|-2181.98| +2451597|51069|4706|169|1004112|3255|21998|4|186|1356|19|8.01|15.21|7.14|0.00|135.66|152.19|288.99|12.20|0.00|135.66|147.86|-16.53| +2451597|51069|6998|169|1004112|3255|21998|4|157|1356|47|60.70|109.26|16.38|207.86|769.86|2852.90|5135.22|50.58|207.86|562.00|612.58|-2290.90| +2451597|51069|17851|169|1004112|3255|21998|4|275|1356|86|41.42|76.62|13.79|0.00|1185.94|3562.12|6589.32|59.29|0.00|1185.94|1245.23|-2376.18| +2451597|51069|9109|169|1004112|3255|21998|4|131|1356|49|15.93|18.47|15.33|0.00|751.17|780.57|905.03|15.02|0.00|751.17|766.19|-29.40| +2451597|51069|12187|169|1004112|3255|21998|4|39|1356|65|45.13|61.82|58.72|0.00|3816.80|2933.45|4018.30|152.67|0.00|3816.80|3969.47|883.35| +2451597|51069|4895|169|1004112|3255|21998|4|99|1356|30|60.93|102.97|86.49|0.00|2594.70|1827.90|3089.10|129.73|0.00|2594.70|2724.43|766.80| +2451597|51069|8858|169|1004112|3255|21998|4|130|1356|48|8.82|9.70|0.97|0.00|46.56|423.36|465.60|2.32|0.00|46.56|48.88|-376.80| +2451597|51069|6049|169|1004112|3255|21998|4|65|1356|3|33.13|44.72|15.20|0.00|45.60|99.39|134.16|4.10|0.00|45.60|49.70|-53.79| +2451597|51069|14546|169|1004112|3255|21998|4|289|1356|53|89.44|107.32|18.24|628.36|966.72|4740.32|5687.96|30.45|628.36|338.36|368.81|-4401.96| +2451597|51069|6902|169|1004112|3255|21998|4|4|1356|38|1.81|2.93|2.43|0.00|92.34|68.78|111.34|2.77|0.00|92.34|95.11|23.56| +2451597|51069|8012|169|1004112|3255|21998|4|90|1356|1|34.30|39.44|11.43|2.97|11.43|34.30|39.44|0.76|2.97|8.46|9.22|-25.84| +2451597|51069|6866|169|1004112|3255|21998|4|124|1356|61|60.78|93.60|15.91|0.00|970.51|3707.58|5709.60|0.00|0.00|970.51|970.51|-2737.07| +|51069|1706|169|||21998|||1356|||||0.00|||3781.18|219.67|0.00|||832.52| +2451359|47939|3019|70981|1016064|5500|12936|1|174|1357|30|19.68|23.22|5.80|50.46|174.00|590.40|696.60|7.41|50.46|123.54|130.95|-466.86| +2451359|47939|16444|70981|1016064|5500|12936|1|8|1357|39|82.47|103.08|17.52|375.80|683.28|3216.33|4020.12|9.22|375.80|307.48|316.70|-2908.85| +2451359|47939|15652|70981|1016064|5500|12936|1|218|1357|57|31.62|42.05|21.44|1124.31|1222.08|1802.34|2396.85|1.95|1124.31|97.77|99.72|-1704.57| +2451359|47939|3688|70981|1016064|5500|12936|1|245|1357|96|59.49|94.58|29.31|0.00|2813.76|5711.04|9079.68|168.82|0.00|2813.76|2982.58|-2897.28| +2451359|47939|8534|70981|1016064|5500|12936|1|260|1357|52|28.84|40.37|27.85|0.00|1448.20|1499.68|2099.24|72.41|0.00|1448.20|1520.61|-51.48| +2451359|47939|7369|70981|1016064|5500|12936|1|32|1357|99|51.56|66.51|3.32|0.00|328.68|5104.44|6584.49|16.43|0.00|328.68|345.11|-4775.76| +2451359|47939|3226|70981|1016064|5500|12936|1|287|1357|20|77.34|119.10|115.52|0.00|2310.40|1546.80|2382.00|46.20|0.00|2310.40|2356.60|763.60| +2451359|47939|13063|70981|1016064|5500|12936|1|35|1357|9|80.48|85.30|18.76|0.00|168.84|724.32|767.70|5.06|0.00|168.84|173.90|-555.48| +2451095|52567|13306|40309|1645178|2494|6963|10|59|1358|47|3.75|7.27|3.70|0.00|173.90|176.25|341.69|3.47|0.00|173.90|177.37|-2.35| +2451095|52567|17066|40309|1645178|2494|6963|10|63|1358|8|82.29|114.38|98.36|0.00|786.88|658.32|915.04|70.81|0.00|786.88|857.69|128.56| +2451095|52567|10322|40309|1645178|2494|6963|10|241|1358|36|30.91|52.23|25.59|211.88|921.24|1112.76|1880.28|56.74|211.88|709.36|766.10|-403.40| +2451095||12992||1645178|2494||10||1358|79|||16.84|0.00|||1603.70|66.51|0.00|1330.36|1396.87|77.42| +2451095|52567|538|40309|1645178|2494|6963|10|239|1358|85|50.88|89.54|19.69|0.00|1673.65|4324.80|7610.90|33.47|0.00|1673.65|1707.12|-2651.15| +|52567|1118|40309|||6963|||1358|11|5.49|8.34||3.98|13.75|60.39|||3.98|||-50.62| +2451095|52567|17608|40309|1645178|2494|6963|10|266|1358|89|81.43|157.15|128.86|0.00|11468.54|7247.27|13986.35|344.05|0.00|11468.54|11812.59|4221.27| +|52567|4006|40309|1645178|2494|6963|10|159|1358|82|42.47|54.36|20.11|0.00|1649.02||4457.52|49.47|0.00|1649.02|1698.49|-1833.52| +2451095|52567|12103|40309|1645178|2494|6963|10|106|1358|81|18.26|26.65|0.53|0.00|42.93|1479.06|2158.65|0.42|0.00|42.93|43.35|-1436.13| +2452392|39600|7033|64566|1913405|5091|27130|4|182|1359|18|26.36|30.31|2.72|0.00|48.96|474.48|545.58|1.46|0.00|48.96|50.42|-425.52| +2452392|39600|16795|64566|1913405|5091|27130|4|293|1359|5|44.45|84.45|44.75|0.00|223.75|222.25|422.25|11.18|0.00|223.75|234.93|1.50| +2452392|39600|16746|64566|1913405|5091|27130|4|150|1359|1|83.64|157.24|18.86|0.00|18.86|83.64|157.24|0.18|0.00|18.86|19.04|-64.78| +2452392|39600|4302|64566|1913405|5091|27130|4|120|1359|82|6.26|10.26|2.15|162.19|176.30|513.32|841.32|0.00|162.19|14.11|14.11|-499.21| +2452392|39600|16851|64566|1913405|5091|27130|4|139|1359|99|82.95|89.58|28.66|0.00|2837.34|8212.05|8868.42|198.61|0.00|2837.34|3035.95|-5374.71| +2452392|39600|4801|64566|1913405|5091|27130|4|2|1359|58|25.11|32.39|27.20|0.00|1577.60|1456.38|1878.62|47.32|0.00|1577.60|1624.92|121.22| +2452392|39600|3180|64566|1913405|5091|27130|4|87|1359|89|55.81|63.62|53.44|0.00|4756.16|4967.09|5662.18|380.49|0.00|4756.16|5136.65|-210.93| +||11805|64566|||27130|4||1359|69||73.81|36.90|1731.34||2845.56|||1731.34|814.76|855.49|| +2452392|39600|9673|64566|1913405|5091|27130|4|161|1359|62|34.15|36.88|10.69|0.00|662.78|2117.30|2286.56|26.51|0.00|662.78|689.29|-1454.52| +2452392|39600|8401|64566|1913405|5091|27130|4|164|1359|44|99.47|147.21|98.63|0.00|4339.72|4376.68|6477.24|216.98|0.00|4339.72|4556.70|-36.96| +2452392|39600|9709|64566|1913405|5091|27130|4|242|1359|55|99.52|191.07|181.51|7986.44|9983.05|5473.60|10508.85|59.89|7986.44|1996.61|2056.50|-3476.99| +2451137|49151|10915|34184|755122|3260|36598|7|279|1360|14|35.66|42.43|32.67|73.18|457.38|499.24|594.02|11.52|73.18|384.20|395.72|-115.04| +2451137|49151|5566|34184|755122|3260|36598|7|86|1360|80|6.46|6.78|0.67|0.00|53.60|516.80|542.40|1.60|0.00|53.60|55.20|-463.20| +2451137|49151|9322|34184|755122|3260|36598|7|110|1360|13|34.57|52.20|22.44|0.00|291.72|449.41|678.60|26.25|0.00|291.72|317.97|-157.69| +2451137|49151|16514|34184|755122|3260|36598|7|191|1360|96|95.83|151.41|93.87|0.00|9011.52|9199.68|14535.36|811.03|0.00|9011.52|9822.55|-188.16| +2451137|49151|15328|34184|755122|3260|36598|7|87|1360|22|93.10|164.78|92.27|0.00|2029.94|2048.20|3625.16|40.59|0.00|2029.94|2070.53|-18.26| +2451137|49151|7603|34184|755122|3260|36598|7|182|1360|28|28.42|30.12|9.63|0.00|269.64|795.76|843.36|18.87|0.00|269.64|288.51|-526.12| +2451137|49151|6310|34184|755122|3260|36598|7|233|1360|31|3.15|3.81|3.20|0.00|99.20|97.65|118.11|6.94|0.00|99.20|106.14|1.55| +2451137|49151|14788|34184|755122|3260|36598|7|145|1360|72|73.51|87.47|79.59|2578.71|5730.48|5292.72|6297.84|94.55|2578.71|3151.77|3246.32|-2140.95| +2451137|49151|64|34184|755122|3260|36598|7|189|1360|66|35.45|43.60|30.95|0.00|2042.70|2339.70|2877.60|163.41|0.00|2042.70|2206.11|-297.00| +2451121|73344|11422|49278|702540|4155|30170|8|46|1361|67|72.58|123.38|8.63|0.00|578.21|4862.86|8266.46|23.12|0.00|578.21|601.33|-4284.65| +2451121|73344|11449|49278|702540|4155|30170|8|300|1361|6|23.15|35.88|27.26|0.00|163.56|138.90|215.28|9.81|0.00|163.56|173.37|24.66| +2451121|73344|8152|49278|702540|4155|30170|8|152|1361|57|84.53|98.05|30.39|0.00|1732.23|4818.21|5588.85|0.00|0.00|1732.23|1732.23|-3085.98| +2451121|73344|517|49278|702540|4155|30170|8|31|1361|7|74.08|106.67|0.00|0.00|0.00|518.56|746.69|0.00|0.00|0.00|0.00|-518.56| +2451121|73344|3220|49278|702540|4155|30170|8|91|1361|63|9.02|13.34|4.53|0.00|285.39|568.26|840.42|8.56|0.00|285.39|293.95|-282.87| +2451121|73344|17504|49278|702540|4155|30170|8|292|1361|55|92.54|108.27|63.87|0.00|3512.85|5089.70|5954.85|105.38|0.00|3512.85|3618.23|-1576.85| +2451121|73344|12292|49278|702540|4155|30170|8|133|1361|67|84.76|91.54|35.70|0.00|2391.90|5678.92|6133.18|143.51|0.00|2391.90|2535.41|-3287.02| +2451121|73344|13705|49278|702540|4155|30170|8|258|1361|91|31.26|50.95|25.98|0.00|2364.18|2844.66|4636.45|0.00|0.00|2364.18|2364.18|-480.48| +2451121|73344|10990|49278|702540|4155|30170|8|192|1361|29|4.29|8.19|2.94|4.26|85.26|124.41|237.51|1.62|4.26|81.00|82.62|-43.41| +2451121|73344|11149|49278|702540|4155|30170|8|157|1361|4|17.47|34.06|24.52|0.00|98.08|69.88|136.24|4.90|0.00|98.08|102.98|28.20| +2451121|73344|454|49278|702540|4155|30170|8|60|1361|43|57.16|62.30|21.80|712.42|937.40|2457.88|2678.90|11.24|712.42|224.98|236.22|-2232.90| +2451121|73344|6031|49278|702540|4155|30170|8|1|1361|8|4.28|4.36|0.82|0.00|6.56|34.24|34.88|0.52|0.00|6.56|7.08|-27.68| +2452032|63756|12675|52379|568933|5134|12888|8|68|1362|90|60.71|71.63|4.29|0.00|386.10|5463.90|6446.70|19.30|0.00|386.10|405.40|-5077.80| +2452032|63756|2085|52379|568933|5134|12888|8|264|1362|44|72.24|95.35|61.02|0.00|2684.88|3178.56|4195.40|80.54|0.00|2684.88|2765.42|-493.68| +2452032|63756|1211|52379|568933|5134|12888|8|200|1362|23|36.93|72.38|44.87|0.00|1032.01|849.39|1664.74|92.88|0.00|1032.01|1124.89|182.62| +2452032|63756|17389|52379|568933|5134|12888|8|101|1362|7|59.30|67.00|60.97|0.00|426.79|415.10|469.00|4.26|0.00|426.79|431.05|11.69| +2452032|63756|12609|52379|568933|5134|12888|8|67|1362|79|81.47|103.46|45.52|0.00|3596.08|6436.13|8173.34|251.72|0.00|3596.08|3847.80|-2840.05| +2452032|63756|6423|52379|568933|5134|12888|8|271|1362|53|62.51|62.51|18.12|422.55|960.36|3313.03|3313.03|43.02|422.55|537.81|580.83|-2775.22| +2452032|63756|1179|52379|568933|5134|12888|8|131|1362|93|10.63|14.66|7.03|588.41|653.79|988.59|1363.38|1.30|588.41|65.38|66.68|-923.21| +2452032|63756|10529|52379|568933|5134|12888|8|17|1362|77|87.53|117.29|45.74|0.00|3521.98|6739.81|9031.33|246.53|0.00|3521.98|3768.51|-3217.83| +||2815||568933|5134||8||1362|65|30.39|37.98||0.00|1381.90|1975.35|2468.70|124.37|0.00|||| +2452032|63756|5157|52379|568933|5134|12888|8|53|1362|3|78.44|94.91|40.81|0.00|122.43|235.32|284.73|3.67|0.00|122.43|126.10|-112.89| +2451787|67643|5053|73014|1005581|548|14315|2|5|1363|27|8.62|14.30|11.86|0.00|320.22|232.74|386.10|19.21|0.00|320.22|339.43|87.48| +2451787|67643|7337|73014|1005581|548|14315|2|81|1363|2|92.21|110.65|101.79|0.00|203.58|184.42|221.30|18.32|0.00|203.58|221.90|19.16| +2451787|67643|1988|73014|1005581|548|14315|2|15|1363|72|53.53|97.95|35.26|0.00|2538.72|3854.16|7052.40|0.00|0.00|2538.72|2538.72|-1315.44| +2451787|67643|9679|73014|1005581|548|14315|2|209|1363|66|20.43|36.97|15.52|0.00|1024.32|1348.38|2440.02|20.48|0.00|1024.32|1044.80|-324.06| +2451787|67643|653|73014|1005581|548|14315|2|64|1363|34|94.65|121.15|47.24|0.00|1606.16|3218.10|4119.10|32.12|0.00|1606.16|1638.28|-1611.94| +2451787|67643|8906|73014|1005581|548|14315|2|104|1363|93|67.93|134.50|20.17|0.00|1875.81|6317.49|12508.50|168.82|0.00|1875.81|2044.63|-4441.68| +2451787|67643|11831|73014|1005581|548|14315|2|94|1363|83|78.16|139.90|43.36|1511.52|3598.88|6487.28|11611.70|187.86|1511.52|2087.36|2275.22|-4399.92| +2451787|67643|17156|73014|1005581|548|14315|2|270|1363|43|9.49|18.98|0.00|0.00|0.00|408.07|816.14|0.00|0.00|0.00|0.00|-408.07| +2451787|67643|8960|73014|1005581|548|14315|2|265|1363|23|34.20|35.56|16.00|0.00|368.00|786.60|817.88|14.72|0.00|368.00|382.72|-418.60| +2451787|67643|7435|73014|1005581|548|14315|2|196|1363|65|86.70|136.98|132.87|0.00|8636.55|5635.50|8903.70|172.73|0.00|8636.55|8809.28|3001.05| +2451787|67643|3847|73014|1005581|548|14315|2|72|1363|57|75.19|118.04|28.32|694.12|1614.24|4285.83|6728.28|82.81|694.12|920.12|1002.93|-3365.71| +2451787|67643|5737|73014|1005581|548|14315|2|14|1363|98|18.17|28.89|8.95|166.64|877.10|1780.66|2831.22|63.94|166.64|710.46|774.40|-1070.20| +2451787|67643|12865|73014|1005581|548|14315|2|257|1363|46|95.73|130.19|57.28|0.00|2634.88|4403.58|5988.74|237.13|0.00|2634.88|2872.01|-1768.70| +2451787|67643|3271|73014|1005581|548|14315|2|199|1363|87|26.97|29.39|11.46|0.00|997.02|2346.39|2556.93|89.73|0.00|997.02|1086.75|-1349.37| +2452592|32453|4359|11218|1501280|4142|22204|10|201|1364|40|47.08|48.02|3.84|0.00|153.60|1883.20|1920.80|1.53|0.00|153.60|155.13|-1729.60| +|32453|13921||1501280|||10|180|1364|8|||120.17|||584.08|1033.76||||999.81|377.28| +2452592|32453|17532|11218|1501280|4142|22204|10|182|1364|42|96.65|170.10|62.93|105.72|2643.06|4059.30|7144.20|0.00|105.72|2537.34|2537.34|-1521.96| +2452592|32453|3553|11218|1501280|4142|22204|10|34|1364|30|90.01|146.71|114.43|0.00|3432.90|2700.30|4401.30|171.64|0.00|3432.90|3604.54|732.60| +2452592|32453|11881|11218|1501280|4142|22204|10|216|1364|86|84.75|125.43|35.12|0.00|3020.32|7288.50|10786.98|30.20|0.00|3020.32|3050.52|-4268.18| +2452592|32453|14067|11218|1501280|4142|22204|10|154|1364|66|44.49|55.61|41.15|0.00|2715.90|2936.34|3670.26|217.27|0.00|2715.90|2933.17|-220.44| +2452592|32453|17191|11218|1501280|4142|22204|10|174|1364|68|92.91|92.91|78.97|0.00|5369.96|6317.88|6317.88|161.09|0.00|5369.96|5531.05|-947.92| +2452592|32453|7926|11218|1501280|4142|22204|10|175|1364|53|84.69|113.48|61.27|0.00|3247.31|4488.57|6014.44|32.47|0.00|3247.31|3279.78|-1241.26| +2452592|32453|1789|11218|1501280|4142|22204|10|142|1364|44|2.60|3.61|1.37|12.65|60.28|114.40|158.84|2.85|12.65|47.63|50.48|-66.77| +2451144|49833|9362|49663|551511|3759|43207|2|300|1365|71|59.21|102.43|93.21|0.00|6617.91|4203.91|7272.53|397.07|0.00|6617.91|7014.98|2414.00| +2451144|49833|61|49663|551511|3759|43207|2|277|1365|37|50.55|52.06|2.60|71.18|96.20|1870.35|1926.22|0.50|71.18|25.02|25.52|-1845.33| +2451144|49833|7976|||||2|88|1365|||100.12|24.02|0.00||3026.92|5206.24||0.00|1249.04||| +2451144|49833|5836|49663|551511|3759|43207|2|162|1365|6|59.91|77.28|13.13|0.00|78.78|359.46|463.68|0.00|0.00|78.78|78.78|-280.68| +2451144|49833|4879|49663|551511|3759|43207|2|73|1365|57|85.83|154.49|83.42|0.00|4754.94|4892.31|8805.93|47.54|0.00|4754.94|4802.48|-137.37| +2451144|49833|15643|49663|551511|3759|43207|2|233|1365|69|43.50|66.12|33.72|395.53|2326.68|3001.50|4562.28|77.24|395.53|1931.15|2008.39|-1070.35| +2451144|49833|12880|49663|551511|3759|43207|2|184|1365|83|27.50|50.05|45.54|0.00|3779.82|2282.50|4154.15|340.18|0.00|3779.82|4120.00|1497.32| +2451144|49833|14101|49663|551511|3759|43207|2|122|1365|52|93.99|173.88|151.27|0.00|7866.04|4887.48|9041.76|393.30|0.00|7866.04|8259.34|2978.56| +2451144|49833|11854|49663|551511|3759|43207|2|66|1365|39|91.64|174.11|62.67|0.00|2444.13|3573.96|6790.29|146.64|0.00|2444.13|2590.77|-1129.83| +2451326|52739|14233|58016|675056|4896|46289|4|242|1366|38|58.00|66.12|54.87|0.00|2085.06|2204.00|2512.56|20.85|0.00|2085.06|2105.91|-118.94| +2451326|52739|10759|58016|675056|4896|46289|4|175|1366|7|31.24|34.67|17.68|0.00|123.76|218.68|242.69|7.42|0.00|123.76|131.18|-94.92| +2451326|52739|10504|58016|675056|4896|46289|4|193|1366|35|21.66|41.37|35.57|0.00|1244.95|758.10|1447.95|12.44|0.00|1244.95|1257.39|486.85| +2451326|52739|5212|58016|675056|4896|46289|4|121|1366|10|43.66|44.96|39.56|0.00|395.60|436.60|449.60|7.91|0.00|395.60|403.51|-41.00| +2451326|52739|7540|58016|675056|4896|46289|4|1|1366|67|46.71|63.05|56.74|0.00|3801.58|3129.57|4224.35|0.00|0.00|3801.58|3801.58|672.01| +2451326|52739|14722|58016|675056|4896|46289|4|106|1366|67|31.69|35.80|27.92|0.00|1870.64|2123.23|2398.60|112.23|0.00|1870.64|1982.87|-252.59| +2451326|52739|16352||675056|4896|46289|4|236|1366|79|15.44||||583.81|1219.76|||||636.35|| +2451326|52739|13225|58016|675056|4896|46289|4|130|1366|87|39.35|40.92|33.96|0.00|2954.52|3423.45|3560.04|59.09|0.00|2954.52|3013.61|-468.93| +2451326||8869|58016||||4|155|1366|86||||558.95|||||558.95|2102.75|2165.83|-2310.77| +2451326|52739|3322|58016|675056|4896|46289|4|157|1366|80|3.67|4.47|2.54|87.37|203.20|293.60|357.60|4.63|87.37|115.83|120.46|-177.77| +2451326|52739|428|58016|675056|4896|46289|4|60|1366|11|27.51|31.36|0.62|3.41|6.82|302.61|344.96|0.23|3.41|3.41|3.64|-299.20| +2451326|52739|7795|58016|675056|4896|46289|4|300|1366|4|11.57|20.82|16.03|0.00|64.12|46.28|83.28|1.92|0.00|64.12|66.04|17.84| +2451326|52739|16192|58016|675056|4896|46289|4|46|1366|81|46.66|83.05|73.08|887.92|5919.48|3779.46|6727.05|352.20|887.92|5031.56|5383.76|1252.10| +2451326|52739|16606|58016|675056|4896|46289|4|283|1366|51|56.96|62.65|22.55|0.00|1150.05|2904.96|3195.15|46.00|0.00|1150.05|1196.05|-1754.91| +2451326|52739|11404|58016|675056|4896|46289|4|35|1366|41|84.45|154.54|134.44|1984.33|5512.04|3462.45|6336.14|282.21|1984.33|3527.71|3809.92|65.26| +2451326|52739|12416|58016|675056|4896|46289|4|255|1366|82|58.48|91.22|81.18|0.00|6656.76|4795.36|7480.04|599.10|0.00|6656.76|7255.86|1861.40| +2452192|71599|9637|15451|1881082|6713|12744|2|147|1367|47|70.54|124.15|117.94|0.00|5543.18|3315.38|5835.05|221.72|0.00|5543.18|5764.90|2227.80| +2452192|71599|10707|15451|1881082|6713|12744|2|149|1367|23|98.94|125.65|21.36|0.00|491.28|2275.62|2889.95|34.38|0.00|491.28|525.66|-1784.34| +2452192|71599|9643|15451|1881082|6713|12744|2|2|1367|68|15.36|27.95|24.87|0.00|1691.16|1044.48|1900.60|152.20|0.00|1691.16|1843.36|646.68| +||2543||1881082||12744|2||1367|19|90.84|108.09||1314.15||1725.96||21.57|1314.15|308.26||| +2452192|71599|3635|15451|1881082|6713|12744|2|42|1367|90|31.61|42.04|39.09|457.35|3518.10|2844.90|3783.60|244.86|457.35|3060.75|3305.61|215.85| +2452192|71599|14627|15451|1881082|6713|12744|2|115|1367|85|8.41|10.84|8.13|0.00|691.05|714.85|921.40|41.46|0.00|691.05|732.51|-23.80| +2452192|71599|14223|15451|1881082|6713|12744|2|54|1367|71|14.92|25.51|22.95|0.00|1629.45|1059.32|1811.21|130.35|0.00|1629.45|1759.80|570.13| +2452192|71599|14097|15451|1881082|6713|12744|2|158|1367|73|64.38|107.51|93.53|0.00|6827.69|4699.74|7848.23|546.21|0.00|6827.69|7373.90|2127.95| +2452192|71599|15197|15451|1881082|6713|12744|2|222|1367|53|5.14|5.44|0.21|4.34|11.13|272.42|288.32|0.13|4.34|6.79|6.92|-265.63| +2452192|71599|3173|15451|1881082|6713|12744|2|160|1367|4|12.94|13.06|12.14|0.00|48.56|51.76|52.24|4.37|0.00|48.56|52.93|-3.20| +2452192||17553|15451|||12744|2|37|1367||||13.14|||||||473.04|487.23|-469.08| +2452149|38164|5311|37296|1801904|2065|6673|7|119|1368|32|66.51|107.74|48.48|0.00|1551.36|2128.32|3447.68|62.05|0.00|1551.36|1613.41|-576.96| +2452149|38164|14743|37296|1801904|2065|6673|7|203|1368|2|45.00|71.55|32.91|0.00|65.82|90.00|143.10|5.92|0.00|65.82|71.74|-24.18| +2452149|38164|14293|37296|1801904|2065|6673|7|287|1368|53|65.57|116.05|74.27|1613.88|3936.31|3475.21|6150.65|69.67|1613.88|2322.43|2392.10|-1152.78| +2452149|38164|10869|37296|1801904|2065|6673|7|173|1368|97|1.30|2.09|1.02|0.00|98.94|126.10|202.73|5.93|0.00|98.94|104.87|-27.16| +2452149|38164|3813|37296|1801904|2065|6673|7|143|1368|68|97.04|144.58|85.30|0.00|5800.40|6598.72|9831.44|522.03|0.00|5800.40|6322.43|-798.32| +2452149|38164|7147|37296|1801904|2065|6673|7|213|1368|25|7.88|8.58|6.52|0.00|163.00|197.00|214.50|6.52|0.00|163.00|169.52|-34.00| +2452149|38164|15595|37296|1801904|2065|6673|7|300|1368|84|18.17|25.61|23.56|0.00|1979.04|1526.28|2151.24|19.79|0.00|1979.04|1998.83|452.76| +|38164|17385|37296|1801904|2065||||1368|84|71.86||58.63|0.00||6036.24|12012.00||0.00|4924.92||| +2451790|74282|8209|69808|1175561|1667|45420|1|232|1369|11|22.64|23.31|4.19|0.00|46.09|249.04|256.41|2.76|0.00|46.09|48.85|-202.95| +2451790|74282|13747|69808|1175561|1667|45420|1|127|1369|28|44.54|62.80|62.80|0.00|1758.40|1247.12|1758.40|140.67|0.00|1758.40|1899.07|511.28| +2451790|74282|11924|69808|1175561|1667|45420|1|253|1369|1|36.97|60.63|2.42|1.21|2.42|36.97|60.63|0.07|1.21|1.21|1.28|-35.76| +2451790|74282|4370|69808|1175561|1667|45420|1|179|1369|39|10.59|13.97|3.63|0.00|141.57|413.01|544.83|7.07|0.00|141.57|148.64|-271.44| +2451790|74282|6533|69808|1175561|1667|45420|1|32|1369|66|62.97|108.93|7.62|0.00|502.92|4156.02|7189.38|20.11|0.00|502.92|523.03|-3653.10| +2451790|74282|1289|69808|1175561|1667|45420|1|227|1369|97|84.54|144.56|124.32|0.00|12059.04|8200.38|14022.32|120.59|0.00|12059.04|12179.63|3858.66| +|74282|13796||||45420|1||1369|61||188.88||||6001.18|11521.68|682.07||8525.97|9208.04|| +2451790|74282|5647|69808|1175561|1667|45420|1|229|1369|45|28.14|38.83|21.35|0.00|960.75|1266.30|1747.35|86.46|0.00|960.75|1047.21|-305.55| +2451790|74282|290|69808|1175561|1667|45420|1|60|1369|34|36.75|41.16|27.16|0.00|923.44|1249.50|1399.44|46.17|0.00|923.44|969.61|-326.06| +2451790|74282|17285|69808|1175561|1667|45420|1|47|1369|95|93.75|109.68|71.29|0.00|6772.55|8906.25|10419.60|609.52|0.00|6772.55|7382.07|-2133.70| +2451790|74282|10591|69808|1175561|1667|45420|1|171|1369|74|82.63|147.08|44.12|946.81|3264.88|6114.62|10883.92|23.18|946.81|2318.07|2341.25|-3796.55| +2451790|74282|11462|69808|1175561|1667|45420|1|294|1369|92|60.33|74.80|0.74|0.00|68.08|5550.36|6881.60|5.44|0.00|68.08|73.52|-5482.28| +2451790|74282|12362|69808|1175561|1667|45420|1|118|1369|37|93.62|127.32|119.68|0.00|4428.16|3463.94|4710.84|0.00|0.00|4428.16|4428.16|964.22| +2451790|74282|10553|69808|1175561|1667|45420|1|185|1369|57|92.70|178.91|10.73|0.00|611.61|5283.90|10197.87|30.58|0.00|611.61|642.19|-4672.29| +2451790|74282|2129|69808|1175561|1667|45420|1|134|1369|92|15.78|31.40|29.20|456.68|2686.40|1451.76|2888.80|111.48|456.68|2229.72|2341.20|777.96| +|63881|11967|4961||1145|9237||19|1370||6.31|8.70|||||139.20|0.16||2.72|2.88|| +2452616|63881|10417|4961|1558253|1145|9237|2|219|1370|13|50.74|78.64|17.30|0.00|224.90|659.62|1022.32|13.49|0.00|224.90|238.39|-434.72| +2452616|63881|2823|4961|1558253|1145|9237|2|7|1370|85|27.87|39.85|24.70|0.00|2099.50|2368.95|3387.25|20.99|0.00|2099.50|2120.49|-269.45| +2452616|63881|6084|4961|1558253|1145|9237|2|209|1370|43|34.92|37.71|10.55|213.21|453.65|1501.56|1621.53|19.23|213.21|240.44|259.67|-1261.12| +2452616|63881|3751|4961|1558253|1145|9237|2|91|1370|8|20.49|38.93|5.83|0.00|46.64|163.92|311.44|4.19|0.00|46.64|50.83|-117.28| +2452616|63881|16512|4961|1558253|1145|9237|2|170|1370|28|74.39|105.63|30.63|0.00|857.64|2082.92|2957.64|68.61|0.00|857.64|926.25|-1225.28| +2452616|63881|12648|4961|1558253|1145|9237|2|79|1370|75|52.82|101.41|13.18|830.34|988.50|3961.50|7605.75|14.23|830.34|158.16|172.39|-3803.34| +2452616|63881|10885|4961|1558253|1145|9237|2|290|1370|10|39.57|67.66|12.85|0.00|128.50|395.70|676.60|2.57|0.00|128.50|131.07|-267.20| +2452616|63881|5607|4961|1558253|1145|9237|2|111|1370|86|53.07|101.89|89.66|0.00|7710.76|4564.02|8762.54|616.86|0.00|7710.76|8327.62|3146.74| +2452616|63881|9072|4961|1558253|1145|9237|2|46|1370|17|34.51|41.06|40.64|269.44|690.88|586.67|698.02|4.21|269.44|421.44|425.65|-165.23| +2452616|63881|8712|4961|1558253|1145|9237|2|273|1370|49|14.85|18.11|1.44|0.00|70.56|727.65|887.39|1.41|0.00|70.56|71.97|-657.09| +2452616|63881|15583|4961|1558253|1145|9237|2|193|1370|79|85.61|89.89|8.09|428.20|639.11|6763.19|7101.31|2.10|428.20|210.91|213.01|-6552.28| +2452616|63881|6385|4961|1558253|1145|9237|2|74|1370|17|56.47|98.82|67.19|0.00|1142.23|959.99|1679.94|79.95|0.00|1142.23|1222.18|182.24| +2452301|44255|7410|97070|73749|2781|14660|2|295|1371|9|24.41|40.27|1.61|0.14|14.49|219.69|362.43|0.43|0.14|14.35|14.78|-205.34| +2452301|44255|1017|97070|73749|2781|14660|2|187|1371|42|93.03|173.96|15.65|0.00|657.30|3907.26|7306.32|6.57|0.00|657.30|663.87|-3249.96| +2452301|44255|9289|97070|73749|2781|14660|2|91|1371|34|44.22|52.62|21.57|425.36|733.38|1503.48|1789.08|0.00|425.36|308.02|308.02|-1195.46| +2452301|44255|6463|97070|73749|2781|14660|2|248|1371|58|44.08|46.28|6.01|0.00|348.58|2556.64|2684.24|31.37|0.00|348.58|379.95|-2208.06| +2452301|44255|1705|97070|73749|2781|14660|2|204|1371|83|38.87|51.30|9.74|0.00|808.42|3226.21|4257.90|56.58|0.00|808.42|865.00|-2417.79| +2452301|44255|11364|97070|73749|2781|14660|2|231|1371|82|65.44|66.09|9.25|0.00|758.50|5366.08|5419.38|45.51|0.00|758.50|804.01|-4607.58| +2452301|44255|7080|97070|73749|2781|14660|2|65|1371|76|84.89|104.41|34.45|0.00|2618.20|6451.64|7935.16|183.27|0.00|2618.20|2801.47|-3833.44| +2452301|44255|10605|97070|73749|2781|14660|2|182|1371|82|24.66|24.90|12.45|0.00|1020.90|2022.12|2041.80|81.67|0.00|1020.90|1102.57|-1001.22| +2452301|44255|10213|97070|73749|2781|14660|2|114|1371|2|36.43|52.82|10.03|5.81|20.06|72.86|105.64|0.14|5.81|14.25|14.39|-58.61| +2452301|44255|2677|97070|73749|2781|14660|2|205|1371|30|20.79|32.22|6.44|0.00|193.20|623.70|966.60|5.79|0.00|193.20|198.99|-430.50| +2452301|44255|16809|97070|73749|2781|14660|2|126|1371|25|53.46|70.03|42.01|0.00|1050.25|1336.50|1750.75|21.00|0.00|1050.25|1071.25|-286.25| +2452301|44255|7800|97070|73749|2781|14660|2|272|1371|60|44.66|77.26|1.54|0.00|92.40|2679.60|4635.60|1.84|0.00|92.40|94.24|-2587.20| +2451439|37077|8686|32830|1771475|1320|9918|1|164|1372|37|99.62|177.32|60.28|0.00|2230.36|3685.94|6560.84|66.91|0.00|2230.36|2297.27|-1455.58| +2451439|37077|12439|32830|1771475|1320|9918|1|47|1372|61|99.50|135.32|74.42|0.00|4539.62|6069.50|8254.52|363.16|0.00|4539.62|4902.78|-1529.88| +2451439|37077|9220|32830|1771475|1320|9918|1|128|1372|56|93.33|161.46|16.14|0.00|903.84|5226.48|9041.76|36.15|0.00|903.84|939.99|-4322.64| +2451439|37077|992|32830|1771475|1320|9918|1|27|1372|95|82.04|87.78|82.51|0.00|7838.45|7793.80|8339.10|156.76|0.00|7838.45|7995.21|44.65| +2451439|37077|7486|32830|1771475|1320|9918|1|5|1372|93|44.27|58.87|31.78|0.00|2955.54|4117.11|5474.91|177.33|0.00|2955.54|3132.87|-1161.57| +2451439|37077|6289|32830|1771475|1320|9918|1|204|1372|41|30.35|49.47|36.60|0.00|1500.60|1244.35|2028.27|30.01|0.00|1500.60|1530.61|256.25| +2451439|37077|3868|32830|1771475|1320|9918|1|128|1372|17|78.02|156.04|9.36|95.47|159.12|1326.34|2652.68|2.54|95.47|63.65|66.19|-1262.69| +2451439|37077|16687|32830|1771475|1320|9918|1|14|1372|26|55.27|92.30|44.30|0.00|1151.80|1437.02|2399.80|0.00|0.00|1151.80|1151.80|-285.22| +2451439|37077|11918|32830|1771475|1320|9918|1|248|1372|75|61.76|117.34|9.38|0.00|703.50|4632.00|8800.50|28.14|0.00|703.50|731.64|-3928.50| +|47921|6149|75847|1604055||||277|1373|29|||37.93|0.00||1741.16|||0.00|1099.97|1187.96|| +2451766|47921|15440|75847|1604055|4710|38165|10|287|1373|89|5.10|7.54|1.73|0.00|153.97|453.90|671.06|13.85|0.00|153.97|167.82|-299.93| +2451766|47921|13399|75847|1604055|4710|38165|10|8|1373|12|56.26|90.01|23.40|0.00|280.80|675.12|1080.12|11.23|0.00|280.80|292.03|-394.32| +2451766|47921|12845|75847|1604055|4710|38165|10|287|1373|73|91.83|110.19|1.10|0.00|80.30|6703.59|8043.87|6.42|0.00|80.30|86.72|-6623.29| +2451766|47921|10751|75847|1604055|4710|38165|10|215|1373|38|31.63|33.84|27.41|0.00|1041.58|1201.94|1285.92|52.07|0.00|1041.58|1093.65|-160.36| +2451766|47921|10112|75847|1604055|4710|38165|10|30|1373|83|52.99|99.09|49.54|0.00|4111.82|4398.17|8224.47|246.70|0.00|4111.82|4358.52|-286.35| +2451766|47921|15524|75847|1604055|4710|38165|10|22|1373|40|70.92|71.62|35.81|974.03|1432.40|2836.80|2864.80|32.08|974.03|458.37|490.45|-2378.43| +2451766|47921|7013|75847|1604055|4710|38165|10|300|1373|9|95.38|123.04|79.97|0.00|719.73|858.42|1107.36|14.39|0.00|719.73|734.12|-138.69| +2451766|47921|12731|75847|1604055|4710|38165|10|203|1373|74|50.20|97.89|32.30|0.00|2390.20|3714.80|7243.86|167.31|0.00|2390.20|2557.51|-1324.60| +2451766|47921|4529|75847|1604055|4710|38165|10|90|1373|59|23.67|33.61|16.13|0.00|951.67|1396.53|1982.99|28.55|0.00|951.67|980.22|-444.86| +2451766|47921|16145|75847|1604055|4710|38165|10|237|1373|14|93.48|105.63|89.78|930.12|1256.92|1308.72|1478.82|13.07|930.12|326.80|339.87|-981.92| +2451766|47921|9025|75847|1604055|4710|38165|10|217|1373|61|62.91|93.73|24.36|0.00|1485.96|3837.51|5717.53|118.87|0.00|1485.96|1604.83|-2351.55| +2451896|51505|11179|16447|918563|2279|7995|10|58|1374|1|7.02|9.96|0.00|0.00|0.00|7.02|9.96|0.00|0.00|0.00|0.00|-7.02| +2451896|51505|8387|16447|918563|2279|7995|10|7|1374|53|8.85|16.19|4.20|40.06|222.60|469.05|858.07|7.30|40.06|182.54|189.84|-286.51| +2451896|51505|17735|16447|918563|2279|7995|10|41|1374|78|67.03|119.31|57.26|0.00|4466.28|5228.34|9306.18|223.31|0.00|4466.28|4689.59|-762.06| +2451896|51505|17567|16447|918563|2279|7995|10|268|1374|37|32.21|57.65|51.30|0.00|1898.10|1191.77|2133.05|94.90|0.00|1898.10|1993.00|706.33| +2451896|51505|9353|16447|918563|2279|7995|10|126|1374|25|24.74|31.66|0.63|0.00|15.75|618.50|791.50|0.15|0.00|15.75|15.90|-602.75| +2451896|51505|13292||||7995|10|129|1374||53.25||||74.12||185.30|||74.12||| +2451896|51505|17720|16447|918563|2279|7995|10|296|1374|82|69.14|137.58|49.52|0.00|4060.64|5669.48|11281.56|162.42|0.00|4060.64|4223.06|-1608.84| +2451896|51505|5911|16447|918563|2279|7995|10|250|1374|16|60.30|86.83|6.07|0.00|97.12|964.80|1389.28|6.79|0.00|97.12|103.91|-867.68| +||16475||918563|||||1374|||95.66||0.00||8851.59|9470.34||0.00||6695.33|-2222.55| +2451896|51505|17174|16447|918563|2279|7995|10|255|1374|95|21.86|43.50|20.44|0.00|1941.80|2076.70|4132.50|38.83|0.00|1941.80|1980.63|-134.90| +2451438|50207|6002|72261|1577918|1975|8032|2|35|1375|28|73.49|94.06|26.33|0.00|737.24|2057.72|2633.68|44.23|0.00|737.24|781.47|-1320.48| +2451438|50207|1618|72261||1975|8032|||1375|66|43.03|||||||||||-2798.22| +2451438|50207|4414|72261|1577918|1975|8032|2|90|1375|8|14.03|21.46|20.38|0.00|163.04|112.24|171.68|8.15|0.00|163.04|171.19|50.80| +2451438|50207|3884|72261|1577918|1975|8032|2|8|1375|14|18.23|29.89|4.48|18.81|62.72|255.22|418.46|0.00|18.81|43.91|43.91|-211.31| +2451438|50207|1616|72261|1577918|1975|8032|2|22|1375|14|63.29|91.13|91.13|178.61|1275.82|886.06|1275.82|32.91|178.61|1097.21|1130.12|211.15| +2451438|50207|5438|72261|1577918|1975|8032|2|94|1375|92|45.61|58.83|18.82|1489.03|1731.44|4196.12|5412.36|16.96|1489.03|242.41|259.37|-3953.71| +2451438|50207|14530|72261|1577918|1975|8032|2|36|1375|45|86.50|140.13|78.47|0.00|3531.15|3892.50|6305.85|70.62|0.00|3531.15|3601.77|-361.35| +2451438|50207|3644|72261||1975||2|180|1375|||59.73||0.00|720.18|659.70|1075.14|57.61|0.00|||60.48| +2451438|50207|937|72261|1577918|1975|8032|2|168|1375|54|30.63|37.67|3.01|0.00|162.54|1654.02|2034.18|0.00|0.00|162.54|162.54|-1491.48| +2451079|49683|3272|11126|1516339|3952|45965|4|244|1376|97|33.79|64.20|21.82|1333.42|2116.54|3277.63|6227.40|31.32|1333.42|783.12|814.44|-2494.51| +2451079|49683|4756|11126|1516339|3952|45965|4|234|1376|68|99.86|194.72|17.52|0.00|1191.36|6790.48|13240.96|107.22|0.00|1191.36|1298.58|-5599.12| +2451079|49683|12800|11126|1516339|3952|45965|4|230|1376|12|57.33|108.35|45.50|0.00|546.00|687.96|1300.20|0.00|0.00|546.00|546.00|-141.96| +2451079|49683|536|11126|1516339|3952|45965|4|132|1376|68|80.29|95.54|8.59|280.37|584.12|5459.72|6496.72|0.00|280.37|303.75|303.75|-5155.97| +2451079|49683|14930|11126|1516339|3952|45965|4|224|1376|50|79.75|116.43|105.95|0.00|5297.50|3987.50|5821.50|211.90|0.00|5297.50|5509.40|1310.00| +|49683|3007|11126||||4||1376|84|||15.50|0.00|1302.00|1926.12|2830.80||0.00|1302.00||-624.12| +2451079|49683|4921|11126|1516339|3952|45965|4|9|1376|72|71.28|71.28|49.89|1329.06|3592.08|5132.16|5132.16|45.26|1329.06|2263.02|2308.28|-2869.14| +2451079|49683|9712|11126|1516339|3952|45965|4|289|1376|78|69.88|129.97|55.88|0.00|4358.64|5450.64|10137.66|43.58|0.00|4358.64|4402.22|-1092.00| +2451079|49683|5402|11126||||||1376|57|||37.15|0.00|2117.55|5144.25||127.05|0.00|2117.55||| +2451079|49683|15400|11126|1516339|3952|45965|4|202|1376|3|64.43|105.02|85.06|0.00|255.18|193.29|315.06|20.41|0.00|255.18|275.59|61.89| +2451079|49683|6970|11126|1516339|3952|45965|4|62|1376|96|54.64|78.68|24.39|0.00|2341.44|5245.44|7553.28|93.65|0.00|2341.44|2435.09|-2904.00| +|49683|3128|11126||||4|169|1376|58|76.91||16.69||968.02||7448.94|87.12||968.02||| +2451079|49683|16249|11126|1516339|3952|45965|4|82|1376|74|53.56|67.48|0.67|0.00|49.58|3963.44|4993.52|0.99|0.00|49.58|50.57|-3913.86| +2451079|49683|15722|11126|1516339|3952|45965|4|263|1376|50|37.19|45.74|24.24|0.00|1212.00|1859.50|2287.00|96.96|0.00|1212.00|1308.96|-647.50| +2451079|49683|5678|11126|1516339|3952|45965|4|121|1376|7|89.91|132.16|56.82|0.00|397.74|629.37|925.12|11.93|0.00|397.74|409.67|-231.63| +2451079|49683|16390|11126|1516339|3952|45965|4|105|1376|83|98.27|188.67|188.67|10648.53|15659.61|8156.41|15659.61|350.77|10648.53|5011.08|5361.85|-3145.33| +2452537|66122|1422|47021|105967|6513|24259|1|20|1377|41|79.81|90.98|5.45|0.00|223.45|3272.21|3730.18|13.40|0.00|223.45|236.85|-3048.76| +2452537|66122|8917|47021|105967|6513|24259|1|252|1377|32|20.71|39.97|5.99|0.00|191.68|662.72|1279.04|11.50|0.00|191.68|203.18|-471.04| +2452537|66122|3624|47021|105967|6513|24259|1|130|1377|82|44.03|77.93|74.81|0.00|6134.42|3610.46|6390.26|0.00|0.00|6134.42|6134.42|2523.96| +2452537|66122|12615|47021|105967|6513|24259|1|164|1377|32|1.61|2.20|1.54|0.00|49.28|51.52|70.40|0.98|0.00|49.28|50.26|-2.24| +2452537|66122|11091|47021|105967|6513|24259|1|113|1377|97|26.93|44.70|23.69|0.00|2297.93|2612.21|4335.90|206.81|0.00|2297.93|2504.74|-314.28| +2452537|66122|14629|47021|105967|6513|24259|1|157|1377|83|37.91|65.58|22.29|0.00|1850.07|3146.53|5443.14|37.00|0.00|1850.07|1887.07|-1296.46| +2452537|66122|8958|47021|105967|6513|24259|1|3|1377|43|43.67|44.10|34.39|0.00|1478.77|1877.81|1896.30|44.36|0.00|1478.77|1523.13|-399.04| +2452537|66122|6901|47021|105967|6513|24259|1|146|1377|61|29.15|48.97|1.46|0.00|89.06|1778.15|2987.17|0.89|0.00|89.06|89.95|-1689.09| +2452537|66122|12120|47021|105967|6513|24259|1|190|1377|67|67.67|119.77|75.45|0.00|5055.15|4533.89|8024.59|202.20|0.00|5055.15|5257.35|521.26| +2452593|63751|6145|13026|1719157|4841|44972|1|51|1378|32|21.23|38.63|9.65|0.00|308.80|679.36|1236.16|27.79|0.00|308.80|336.59|-370.56| +2452593|63751|15727|13026|1719157|4841|44972|1|279|1378|14|49.09|64.30|17.36|0.00|243.04|687.26|900.20|12.15|0.00|243.04|255.19|-444.22| +2452593|63751|5226|13026|1719157|4841|44972|1|183|1378|47|76.50|97.92|47.98|1578.54|2255.06|3595.50|4602.24|40.59|1578.54|676.52|717.11|-2918.98| +2452593|63751|3853|13026|1719157|4841|44972|1|170|1378|32|98.49|172.35|120.64|2586.52|3860.48|3151.68|5515.20|0.00|2586.52|1273.96|1273.96|-1877.72| +2452593|63751|14802|13026|1719157|4841|44972|1|149|1378|39|22.41|22.85|15.99|0.00|623.61|873.99|891.15|37.41|0.00|623.61|661.02|-250.38| +2452593|63751|9655|13026|1719157|4841|44972|1|259|1378|57|83.41|145.96|0.00|0.00|0.00|4754.37|8319.72|0.00|0.00|0.00|0.00|-4754.37| +|63751|5511|13026||4841|44972|1|43|1378|63|14.83||20.70|0.00|1304.10||1672.02|52.16|0.00|1304.10|1356.26|| +2452593|63751|9393|13026|1719157|4841|44972|1|270|1378|13|47.98|55.17|20.41|0.00|265.33|623.74|717.21|5.30|0.00|265.33|270.63|-358.41| +2452593|63751|15348|13026|1719157|4841|44972|1|166|1378|54|86.54|164.42|90.43|2734.60|4883.22|4673.16|8878.68|193.37|2734.60|2148.62|2341.99|-2524.54| +2452593|63751|17413|13026|1719157|4841|44972|1|269|1378|33|2.27|2.29|1.46|0.00|48.18|74.91|75.57|1.92|0.00|48.18|50.10|-26.73| +2450916|52535|10096|97632|205944|303|39015|1|42|1379|78|61.24|103.49|37.25|0.00|2905.50|4776.72|8072.22|87.16|0.00|2905.50|2992.66|-1871.22| +2450916|52535|1898|97632|205944|303|39015|1|10|1379|31|57.32|111.20|18.90|193.34|585.90|1776.92|3447.20|0.00|193.34|392.56|392.56|-1384.36| +2450916|52535|7640|97632|205944|303|39015|1|191|1379|12|37.18|58.74|57.56|0.00|690.72|446.16|704.88|13.81|0.00|690.72|704.53|244.56| +2450916|52535|868|97632|205944|303|39015|1|189|1379|43|35.05|41.70|37.11|781.90|1595.73|1507.15|1793.10|40.69|781.90|813.83|854.52|-693.32| +2450916|52535|17240|97632|205944|303|39015|1|234|1379|59|82.82|116.77|28.02|0.00|1653.18|4886.38|6889.43|49.59|0.00|1653.18|1702.77|-3233.20| +2450916||16894|97632||303|39015|1||1379|||86.56|73.57|0.00|1986.39||2337.12|139.04|0.00||2125.43|| +2450916|52535|6140|97632|205944|303|39015|1|214|1379|16|64.74|65.38|36.61|0.00|585.76|1035.84|1046.08|5.85|0.00|585.76|591.61|-450.08| +2450916|52535|2191|97632|205944|303|39015|1|263|1379|96|93.98|142.84|89.98|0.00|8638.08|9022.08|13712.64|777.42|0.00|8638.08|9415.50|-384.00| +2450916|52535|7723|97632|205944|303|39015|1|176|1379|53|81.24|139.73|90.82|0.00|4813.46|4305.72|7405.69|0.00|0.00|4813.46|4813.46|507.74| +2450916|52535|2108|97632|205944|303|39015|1|93|1379|90|35.44|57.76|52.56|0.00|4730.40|3189.60|5198.40|47.30|0.00|4730.40|4777.70|1540.80| +2451782|48808|7159|31868|1920500|5932|39011|8|282|1380|99|26.49|45.29|22.19|0.00|2196.81|2622.51|4483.71|153.77|0.00|2196.81|2350.58|-425.70| +2451782|48808|14012|31868|1920500|5932|39011|8|11|1380|25|62.91|90.59|64.31|0.00|1607.75|1572.75|2264.75|16.07|0.00|1607.75|1623.82|35.00| +2451782|48808|13507|31868|1920500|5932|39011|8|229|1380|47|3.81|4.45|1.06|0.00|49.82|179.07|209.15|2.49|0.00|49.82|52.31|-129.25| +2451782|48808|7016|31868|1920500|5932|39011|8|76|1380|91|27.25|52.32|46.56|0.00|4236.96|2479.75|4761.12|338.95|0.00|4236.96|4575.91|1757.21| +2451782|48808|2389|31868|1920500|5932|39011|8|236|1380|33|10.53|11.37|5.57|0.00|183.81|347.49|375.21|3.67|0.00|183.81|187.48|-163.68| +2451782|48808|1477|31868|1920500|5932|39011|8|263|1380|38|79.88|90.26|84.84|0.00|3223.92|3035.44|3429.88|96.71|0.00|3223.92|3320.63|188.48| +2451782|48808|11432|31868|1920500|5932|39011|8|2|1380|24|33.61|54.44|27.76|206.53|666.24|806.64|1306.56|18.38|206.53|459.71|478.09|-346.93| +2451782|48808|10567|31868|1920500|5932|39011|8|18|1380|60|98.91|101.87|83.53|0.00|5011.80|5934.60|6112.20|150.35|0.00|5011.80|5162.15|-922.80| +2451782|48808|6740|31868|1920500|5932|39011|8|289|1380|20|23.05|32.96|28.67|538.99|573.40|461.00|659.20|1.37|538.99|34.41|35.78|-426.59| +2451816|57792|13031|16915|191258|3554|34697|2|142|1381|48|94.73|143.04|40.05|0.00|1922.40|4547.04|6865.92|0.00|0.00|1922.40|1922.40|-2624.64| +2451816|57792|12698|16915|191258|3554|34697|2|126|1381|91|11.08|17.72|7.61|0.00|692.51|1008.28|1612.52|6.92|0.00|692.51|699.43|-315.77| +2451816|57792|7477|16915|191258|3554|34697|2|3|1381|16|81.84|99.02|13.86|0.00|221.76|1309.44|1584.32|19.95|0.00|221.76|241.71|-1087.68| +2451816|57792|11053|16915|191258|3554|34697|2|5|1381|36|97.06|119.38|112.21|0.00|4039.56|3494.16|4297.68|242.37|0.00|4039.56|4281.93|545.40| +2451816|57792|10004|16915|191258|3554|34697|2|257|1381|96|57.08|79.91|79.11|2809.98|7594.56|5479.68|7671.36|334.92|2809.98|4784.58|5119.50|-695.10| +2451816|57792|11951|16915|191258|3554|34697|2|155|1381|6|93.82|93.82|61.92|0.00|371.52|562.92|562.92|3.71|0.00|371.52|375.23|-191.40| +2451816|57792|13351|16915|191258|3554|34697|2|269|1381|96|31.00|57.04|45.06|0.00|4325.76|2976.00|5475.84|302.80|0.00|4325.76|4628.56|1349.76| +2451816|57792|7397|16915|191258|3554|34697|2|27|1381|8|99.56|154.31|15.43|0.00|123.44|796.48|1234.48|0.00|0.00|123.44|123.44|-673.04| +2451816|57792|7981|16915|191258|3554|34697|2|126|1381|51|32.64|52.22|52.22|0.00|2663.22|1664.64|2663.22|53.26|0.00|2663.22|2716.48|998.58| +2451816|57792|10115|16915|191258|3554|34697|2|131|1381|10|84.17|88.37|46.83|0.00|468.30|841.70|883.70|4.68|0.00|468.30|472.98|-373.40| +2451816|57792|16349|16915||3554|34697|||1381|||||0.00|922.95|3108.70|||0.00|922.95||-2185.75| +2451816|57792|11095|16915|191258|3554|34697|2|190|1381|1|96.00|135.36|71.74|0.00|71.74|96.00|135.36|2.15|0.00|71.74|73.89|-24.26| +2451524|39699|11443|38156|1448706|4856|11053|4|85|1382|70|51.60|84.62|39.77|0.00|2783.90|3612.00|5923.40|222.71|0.00|2783.90|3006.61|-828.10| +2451524|39699|2017|38156|1448706|4856|11053|4|209|1382|34|97.38|125.62|30.14|0.00|1024.76|3310.92|4271.08|71.73|0.00|1024.76|1096.49|-2286.16| +2451524|39699|13285|38156|1448706|4856|11053|4|136|1382|19|82.28|157.97|17.37|303.62|330.03|1563.32|3001.43|0.79|303.62|26.41|27.20|-1536.91| +2451524|39699|13123|38156|1448706|4856|11053|4|154|1382|26|70.76|84.91|38.20|0.00|993.20|1839.76|2207.66|19.86|0.00|993.20|1013.06|-846.56| +2451524|39699|13663|38156|1448706|4856|11053|4|299|1382|45|57.42|67.18|3.35|0.00|150.75|2583.90|3023.10|4.52|0.00|150.75|155.27|-2433.15| +2451524|39699|13528|38156|1448706|4856|11053|4|96|1382|97|11.75|13.27|12.60|366.66|1222.20|1139.75|1287.19|17.11|366.66|855.54|872.65|-284.21| +2451524|39699|17300|38156|1448706|4856|11053|4|278|1382|16|19.07|19.83|17.45|5.58|279.20|305.12|317.28|21.88|5.58|273.62|295.50|-31.50| +2451524|39699|9592|38156|1448706|4856|11053|4|278|1382|71|57.27|92.77|54.73|0.00|3885.83|4066.17|6586.67|155.43|0.00|3885.83|4041.26|-180.34| +2451524|39699|938|38156|1448706|4856|11053|4|246|1382|69|61.93|112.09|40.35|696.03|2784.15|4273.17|7734.21|125.28|696.03|2088.12|2213.40|-2185.05| +2451524|39699|205|38156|1448706|4856|11053|4|20|1382|62|8.36|14.79|6.65|0.00|412.30|518.32|916.98|20.61|0.00|412.30|432.91|-106.02| +||12194|38156||4856|11053||40|1382|||||||85.80|147.42|0.35|||12.05|-74.10| +2451524|39699|6418|38156|1448706|4856|11053|4|137|1382|51|10.06|15.29|5.35|106.41|272.85|513.06|779.79|13.31|106.41|166.44|179.75|-346.62| +2451524|39699|12254|38156|1448706|4856|11053|4|66|1382|18|17.00|28.22|7.05|0.00|126.90|306.00|507.96|10.15|0.00|126.90|137.05|-179.10| +2451524|39699|592|38156|1448706|4856|11053|4|151|1382|52|60.68|109.22|98.29|0.00|5111.08|3155.36|5679.44|102.22|0.00|5111.08|5213.30|1955.72| +2452597|68046|7113|52635|1820441|5494|8100|4|71|1383|31|8.50|13.26|1.06|0.00|32.86|263.50|411.06|1.64|0.00|32.86|34.50|-230.64| +2452597|68046|8346|52635|1820441|5494|8100|4|133|1383|71|20.79|25.15|6.79|91.59|482.09|1476.09|1785.65|27.33|91.59|390.50|417.83|-1085.59| +2452597|68046|8283|52635|1820441|5494|8100|4|172|1383|67|89.38|160.88|8.04|0.00|538.68|5988.46|10778.96|16.16|0.00|538.68|554.84|-5449.78| +2452597|68046|8239|52635|1820441|5494|8100|4|167|1383|72|96.28|105.90|34.94|1408.78|2515.68|6932.16|7624.80|44.27|1408.78|1106.90|1151.17|-5825.26| +2452597|68046|3769|52635|1820441|5494|8100|4|35|1383|74|83.13|97.26|0.00|0.00|0.00|6151.62|7197.24|0.00|0.00|0.00|0.00|-6151.62| +2452597|68046|6405|52635|1820441|5494|8100|4|104|1383|87|69.23|118.38|46.16|2128.43|4015.92|6023.01|10299.06|169.87|2128.43|1887.49|2057.36|-4135.52| +2452597|68046|13431|52635|1820441|5494|8100|4|6|1383|73|83.14|120.55|34.95|0.00|2551.35|6069.22|8800.15|0.00|0.00|2551.35|2551.35|-3517.87| +2452597|68046|13125|52635|1820441|5494|8100|4|62|1383|67|88.67|139.21|82.13|0.00|5502.71|5940.89|9327.07|275.13|0.00|5502.71|5777.84|-438.18| +2452597|68046|16416|52635|1820441|5494|8100|4|95|1383|19|82.13|156.86|100.39|0.00|1907.41|1560.47|2980.34|114.44|0.00|1907.41|2021.85|346.94| +2451159|62889|8551|12503|597573|596|12466|8|69|1384|80|41.51|69.73|68.33|0.00|5466.40|3320.80|5578.40|54.66|0.00|5466.40|5521.06|2145.60| +2451159|62889|16334|12503|597573|596|12466|8|7|1384|42|37.36|44.45|44.00|0.00|1848.00|1569.12|1866.90|92.40|0.00|1848.00|1940.40|278.88| +2451159|62889|1394|12503|597573|596|12466|8|112|1384|61|43.40|74.64|9.70|0.00|591.70|2647.40|4553.04|23.66|0.00|591.70|615.36|-2055.70| +2451159|62889|484|12503|597573|596|12466|8|240|1384|68|66.22|66.88|39.45|2226.55|2682.60|4502.96|4547.84|31.92|2226.55|456.05|487.97|-4046.91| +2451159|62889|8908|12503|597573|596|12466|8|261|1384|50|6.20|10.85|4.12|0.00|206.00|310.00|542.50|18.54|0.00|206.00|224.54|-104.00| +2451159|62889|5890|12503|597573|596|12466|8|186|1384|62|66.45|101.00|62.62|0.00|3882.44|4119.90|6262.00|310.59|0.00|3882.44|4193.03|-237.46| +2451159|62889|17299|12503|597573|596|12466|8|81|1384|68|58.55|90.75|80.76|0.00|5491.68|3981.40|6171.00|164.75|0.00|5491.68|5656.43|1510.28| +2451159|62889|4423|12503|597573|596|12466|8|77|1384|53|83.50|166.16|79.75|0.00|4226.75|4425.50|8806.48|211.33|0.00|4226.75|4438.08|-198.75| +2451159|62889|14102|12503|597573|596|12466|8|243|1384|86|54.90|87.29|75.94|4245.04|6530.84|4721.40|7506.94|137.14|4245.04|2285.80|2422.94|-2435.60| +2451159|62889|7627|12503|597573|596|12466|8|16|1384|10|77.95|125.49|107.92|0.00|1079.20|779.50|1254.90|43.16|0.00|1079.20|1122.36|299.70| +2451159|62889|44|12503|597573|596|12466|8|22|1384|90|84.26|149.98|62.99|0.00|5669.10|7583.40|13498.20|396.83|0.00|5669.10|6065.93|-1914.30| +2451159|62889|14818|12503|597573|596|12466|8|292|1384|35|7.61|12.55|3.13|0.00|109.55|266.35|439.25|0.00|0.00|109.55|109.55|-156.80| +2451159|62889|2126|12503|597573|596|12466|8|154|1384|74|44.45|56.00|31.36|0.00|2320.64|3289.30|4144.00|162.44|0.00|2320.64|2483.08|-968.66| +2451159|62889|6146|12503|597573|596|12466|8|110|1384|59|72.54|103.00|103.00|0.00|6077.00|4279.86|6077.00|243.08|0.00|6077.00|6320.08|1797.14| +2451159|62889|9140|12503|597573|596|12466|8|10|1384|63|44.01|47.53|46.57|0.00|2933.91|2772.63|2994.39|117.35|0.00|2933.91|3051.26|161.28| +2451785|62967|10105|49037|1529709|4781|49500|7|286|1385|68|76.49|151.45|13.63|0.00|926.84|5201.32|10298.60|46.34|0.00|926.84|973.18|-4274.48| +2451785|62967|71|49037|1529709|4781|49500|7|182|1385|10|70.54|87.46|87.46|0.00|874.60|705.40|874.60|69.96|0.00|874.60|944.56|169.20| +2451785|62967|9211|49037|1529709|4781|49500|7|224|1385|68|62.26|97.12|36.90|0.00|2509.20|4233.68|6604.16|0.00|0.00|2509.20|2509.20|-1724.48| +2451785|62967|1979|49037|1529709|4781|49500|7|224|1385|65|72.66|144.59|11.56|0.00|751.40|4722.90|9398.35|7.51|0.00|751.40|758.91|-3971.50| +2451785|62967|1651|49037|1529709|4781|49500|7|215|1385|63|2.21|2.69|0.43|0.00|27.09|139.23|169.47|0.27|0.00|27.09|27.36|-112.14| +2451785|62967|10129|49037|1529709|4781|49500|7|4|1385|5|36.71|66.07|30.39|0.00|151.95|183.55|330.35|9.11|0.00|151.95|161.06|-31.60| +2451785|62967|17243|49037|1529709|4781|49500|7|75|1385|89|93.49|160.80|115.77|3606.23|10303.53|8320.61|14311.20|200.91|3606.23|6697.30|6898.21|-1623.31| +2451785|62967|3083|49037|1529709|4781|49500|7|264|1385|88|5.15|10.30|5.45|479.60|479.60|453.20|906.40|0.00|479.60|0.00|0.00|-453.20| +2451785|62967|10255|49037|1529709|4781|49500|7|162|1385|5|61.89|77.36|32.49|0.00|162.45|309.45|386.80|9.74|0.00|162.45|172.19|-147.00| +2451785|62967|4045|49037|1529709|4781|49500|7|292|1385|65|66.43|72.40|20.27|131.75|1317.55|4317.95|4706.00|71.14|131.75|1185.80|1256.94|-3132.15| +2451785|62967|14651|49037|1529709|4781|49500|7|189|1385|58|56.71|74.85|38.92|0.00|2257.36|3289.18|4341.30|135.44|0.00|2257.36|2392.80|-1031.82| +2451785|62967|13805|49037|1529709|4781|49500|7|183|1385|47|90.86|124.47|63.47|0.00|2983.09|4270.42|5850.09|178.98|0.00|2983.09|3162.07|-1287.33| +2452168|30682|4095|40417|162157|3544|47684|4|119|1386|95|33.29|50.93|0.50|29.45|47.50|3162.55|4838.35|0.72|29.45|18.05|18.77|-3144.50| +2452168|30682|4517|40417|162157|3544|47684|4|142|1386|49|53.14|74.92|26.22|0.00|1284.78|2603.86|3671.08|38.54|0.00|1284.78|1323.32|-1319.08| +2452168|30682|7181|40417|162157|3544|47684|4|288|1386|7|22.53|28.38|1.41|7.89|9.87|157.71|198.66|0.09|7.89|1.98|2.07|-155.73| +2452168|30682|9645|40417|162157|3544|47684|4|12|1386|42|49.73|82.05|71.38|0.00|2997.96|2088.66|3446.10|269.81|0.00|2997.96|3267.77|909.30| +2452168|30682|17889|40417|162157|3544|47684|4|112|1386|20|38.26|47.82|21.99|0.00|439.80|765.20|956.40|26.38|0.00|439.80|466.18|-325.40| +2452168|30682|11885|40417|162157|3544|47684|4|65|1386|37|77.19|93.39|53.23|0.00|1969.51|2856.03|3455.43|157.56|0.00|1969.51|2127.07|-886.52| +2452168|30682|421|40417|162157|3544|47684|4|145|1386|48|39.85|76.91|72.29|1908.45|3469.92|1912.80|3691.68|31.22|1908.45|1561.47|1592.69|-351.33| +2452168|30682|12803|40417|162157|3544|47684|4|196|1386|90|1.12|1.18|0.53|0.00|47.70|100.80|106.20|3.81|0.00|47.70|51.51|-53.10| +2451488|41751|484|96752|478668|2537|18806|7|247|1387|7|73.04|138.77|108.24|0.00|757.68|511.28|971.39|37.88|0.00|757.68|795.56|246.40| +2451488|41751|8908|96752|478668|2537|18806|7|268|1387|11|39.92|63.07|56.13|0.00|617.43|439.12|693.77|30.87|0.00|617.43|648.30|178.31| +2451488|41751|5890|96752|478668|2537|18806|7|150|1387|36|70.54|85.35|47.79|0.00|1720.44|2539.44|3072.60|86.02|0.00|1720.44|1806.46|-819.00| +2451488|41751|17299|96752|478668|2537|18806|7|114|1387|36|77.75|119.73|105.36|0.00|3792.96|2799.00|4310.28|341.36|0.00|3792.96|4134.32|993.96| +2451488|41751|4423|96752|478668|2537|18806|7|234|1387|57|3.87|3.94|3.11|0.00|177.27|220.59|224.58|15.95|0.00|177.27|193.22|-43.32| +2451488|41751|14102|96752|478668|2537|18806|7|237|1387|100|11.47|19.04|15.23|0.00|1523.00|1147.00|1904.00|15.23|0.00|1523.00|1538.23|376.00| +2451488|41751|7627|96752|478668|2537|18806|7|172|1387|20|20.92|22.59|18.97|299.72|379.40|418.40|451.80|3.18|299.72|79.68|82.86|-338.72| +2451488|41751|44|96752|478668|2537|18806|7|162|1387|54|3.50|6.40|0.70|0.00|37.80|189.00|345.60|1.89|0.00|37.80|39.69|-151.20| +2451488|41751|14818|96752|478668|2537|18806|7|52|1387|8|91.85|180.94|117.61|0.00|940.88|734.80|1447.52|18.81|0.00|940.88|959.69|206.08| +2451488|41751|2126|96752|478668|2537|18806|7|152|1387|19|67.11|106.70|59.75|295.16|1135.25|1275.09|2027.30|0.00|295.16|840.09|840.09|-435.00| +2451488|41751|6146|96752|478668|2537|18806|7|96|1387|36|32.98|51.11|9.19|0.00|330.84|1187.28|1839.96|13.23|0.00|330.84|344.07|-856.44| +2451488|41751|9140|96752|478668|2537|18806|7|66|1387|5|34.28|44.22|19.45|0.00|97.25|171.40|221.10|5.83|0.00|97.25|103.08|-74.15| +2451357|48654|14822|5169|679477|538|24046|4|148|1388|13|95.62|121.43|58.28|0.00|757.64|1243.06|1578.59|22.72|0.00|757.64|780.36|-485.42| +2451357|48654|7171|5169|679477|538|24046|4|90|1388|65|1.38|2.22|1.57|0.00|102.05|89.70|144.30|5.10|0.00|102.05|107.15|12.35| +2451357|48654|17104|5169|679477|538|24046|4|273|1388|64|93.78|166.92|138.54|709.32|8866.56|6001.92|10682.88|734.15|709.32|8157.24|8891.39|2155.32| +2451357|48654|12128|5169|679477|538|24046|4|291|1388|46|71.73|105.44|10.54|0.00|484.84|3299.58|4850.24|14.54|0.00|484.84|499.38|-2814.74| +2451357|48654|2036|5169|679477|538|24046|4|123|1388|3|34.66|49.21|33.46|0.00|100.38|103.98|147.63|6.02|0.00|100.38|106.40|-3.60| +2451357|48654|2524|5169|679477|538|24046|4|139|1388|57|7.82|12.59|3.90|46.68|222.30|445.74|717.63|7.02|46.68|175.62|182.64|-270.12| +2451357|48654|7280|5169|679477|538|24046|4|209|1388|44|4.17|6.67|5.20|0.00|228.80|183.48|293.48|0.00|0.00|228.80|228.80|45.32| +2451357|48654|2956|5169|679477|538|24046|4|277|1388|45|65.19|117.34|19.94|0.00|897.30|2933.55|5280.30|44.86|0.00|897.30|942.16|-2036.25| +2451357|48654|16435|5169|679477|538|24046|4|187|1388|42|62.92|118.28|52.04|0.00|2185.68|2642.64|4967.76|196.71|0.00|2185.68|2382.39|-456.96| +2451357|48654|8677|5169|679477|538|24046|4|280|1388|46|62.89|115.71|74.05|306.56|3406.30|2892.94|5322.66|154.98|306.56|3099.74|3254.72|206.80| +2451357|48654|9556|5169|679477|538|24046|4|202|1388|47|73.90|78.33|18.01|330.12|846.47|3473.30|3681.51|46.47|330.12|516.35|562.82|-2956.95| +2451357|48654|13192|5169|679477|538|24046|4|231|1388|85|24.92|27.66|0.82|0.00|69.70|2118.20|2351.10|6.27|0.00|69.70|75.97|-2048.50| +2452278|38000|11787|31286|378697|508|20719|8|35|1389|14|94.93|109.16|0.00|0.00|0.00|1329.02|1528.24|0.00|0.00|0.00|0.00|-1329.02| +|38000|12000||378697|508||8|242|1389|89|26.96||39.02|||2399.44|4510.52||||3646.41|| +2452278|38000|9697|31286|378697|508|20719|8|53|1389|67|59.84|74.80|10.47|0.00|701.49|4009.28|5011.60|63.13|0.00|701.49|764.62|-3307.79| +2452278|38000|17550|31286|378697|508|20719|8|289|1389|23|99.92|117.90|27.11|0.00|623.53|2298.16|2711.70|18.70|0.00|623.53|642.23|-1674.63| +2452278|38000|16857|31286|378697|508|20719|8|209|1389|74|40.89|54.79|29.58|0.00|2188.92|3025.86|4054.46|109.44|0.00|2188.92|2298.36|-836.94| +2452278|38000|13411|31286|378697|508|20719|8|243|1389|46|84.87|140.88|121.15|4792.69|5572.90|3904.02|6480.48|46.81|4792.69|780.21|827.02|-3123.81| +2452278|38000|3270|31286|378697|508|20719|8|205|1389|96|59.91|105.44|73.80|0.00|7084.80|5751.36|10122.24|70.84|0.00|7084.80|7155.64|1333.44| +2452278|38000|3123|31286|378697|508|20719|8|102|1389|15|54.54|70.35|14.07|0.00|211.05|818.10|1055.25|2.11|0.00|211.05|213.16|-607.05| +2452278|38000|16254|31286|378697|508|20719|8|260|1389|83|87.50|114.62|99.71|0.00|8275.93|7262.50|9513.46|662.07|0.00|8275.93|8938.00|1013.43| +2451834|70543|14624|23668|1328893|2118|2543|2|234|1390|11|84.76|166.12|111.30|0.00|1224.30|932.36|1827.32|110.18|0.00|1224.30|1334.48|291.94| +2451834|70543|11813|23668|1328893|2118|2543|2|206|1390|85|15.04|20.00|8.60|0.00|731.00|1278.40|1700.00|58.48|0.00|731.00|789.48|-547.40| +2451834|70543|1997|23668|1328893|2118|2543|2|68|1390|96|88.24|149.12|135.69|0.00|13026.24|8471.04|14315.52|1042.09|0.00|13026.24|14068.33|4555.20| +2451834|70543|15463|23668|1328893|2118|2543|2|194|1390|36|62.63|123.38|62.92|0.00|2265.12|2254.68|4441.68|203.86|0.00|2265.12|2468.98|10.44| +2451834|70543|3865|23668|1328893|2118|2543|2|295|1390|50|25.25|34.84|8.01|0.00|400.50|1262.50|1742.00|0.00|0.00|400.50|400.50|-862.00| +2451834|70543|8933|23668|1328893|2118|2543|2|196|1390|55|73.35|126.16|13.87|183.08|762.85|4034.25|6938.80|11.59|183.08|579.77|591.36|-3454.48| +2451834|70543|8939|23668|1328893|2118|2543|2|154|1390|15|51.64|70.74|19.09|211.89|286.35|774.60|1061.10|3.72|211.89|74.46|78.18|-700.14| +2451834|70543|3626|23668|1328893|2118|2543|2|83|1390|59|74.64|101.51|80.19|0.00|4731.21|4403.76|5989.09|236.56|0.00|4731.21|4967.77|327.45| +2451834|70543|8444|23668|1328893|2118|2543|2|240|1390|71|73.14|134.57|48.44|0.00|3439.24|5192.94|9554.47|137.56|0.00|3439.24|3576.80|-1753.70| +2451834|70543|4826|23668|1328893|2118|2543|2|98|1390|9|43.05|76.19|48.76|0.00|438.84|387.45|685.71|0.00|0.00|438.84|438.84|51.39| +2451834|70543|7915|23668|1328893|2118|2543|2|39|1390|27|30.55|39.10|9.38|0.00|253.26|824.85|1055.70|12.66|0.00|253.26|265.92|-571.59| +2451834|70543|6086|23668|1328893|2118|2543|2|277|1390|28|96.54|154.46|60.23|0.00|1686.44|2703.12|4324.88|0.00|0.00|1686.44|1686.44|-1016.68| +2452022|61428|1833|16838|806571|5904|17237|7|166|1391|80|69.10|89.13|26.73|1689.33|2138.40|5528.00|7130.40|0.00|1689.33|449.07|449.07|-5078.93| +2452022|61428|14835|16838|806571|5904|17237|7|44|1391|6|73.14|144.08|80.68|0.00|484.08|438.84|864.48|0.00|0.00|484.08|484.08|45.24| +2452022|61428|2735|16838|806571|5904|17237|7|70|1391|14|95.42|100.19|93.17|0.00|1304.38|1335.88|1402.66|52.17|0.00|1304.38|1356.55|-31.50| +|61428|1467|16838|||||248|1391|7|||24.92||174.44|||||174.44||-101.92| +2452022|61428|5865|16838|806571|5904|17237|7|103|1391|95|22.70|29.51|5.01|185.62|475.95|2156.50|2803.45|5.80|185.62|290.33|296.13|-1866.17| +2452022|61428|15319|16838|806571|5904|17237|7|156|1391|96|56.32|83.91|65.44|0.00|6282.24|5406.72|8055.36|251.28|0.00|6282.24|6533.52|875.52| +2452022|61428|13257|16838|806571|5904|17237|7|95|1391|44|11.41|17.68|13.08|0.00|575.52|502.04|777.92|28.77|0.00|575.52|604.29|73.48| +2452022|61428|8193|16838|806571|5904|17237|7|218|1391|64|6.41|7.62|0.91|0.00|58.24|410.24|487.68|4.65|0.00|58.24|62.89|-352.00| +2452022|61428|2261|16838|806571|5904|17237|7|11|1391|1|70.80|77.17|38.58|0.00|38.58|70.80|77.17|3.47|0.00|38.58|42.05|-32.22| +2452022|61428|6407|16838|806571|5904|17237|7|49|1391|100|64.62|119.54|38.25|0.00|3825.00|6462.00|11954.00|306.00|0.00|3825.00|4131.00|-2637.00| +2452022|61428|11291|16838|806571|5904|17237|7|148|1391|82|51.32|70.30|59.05|0.00|4842.10|4208.24|5764.60|435.78|0.00|4842.10|5277.88|633.86| +2452022|61428|13721|16838|806571|5904|17237|7|239|1391|82|16.74|19.92|6.97|0.00|571.54|1372.68|1633.44|5.71|0.00|571.54|577.25|-801.14| +2452250|49929|6191|62437|1026748|659|170|1|160|1392|42|48.07|95.65|17.21|0.00|722.82|2018.94|4017.30|43.36|0.00|722.82|766.18|-1296.12| +2452250|49929|11709|62437|1026748|659|170|1|191|1392|16|91.91|179.22|48.38|387.04|774.08|1470.56|2867.52|34.83|387.04|387.04|421.87|-1083.52| +|49929|4571||1026748||||34|1392|54|||93.04||||||||5325.60|137.70| +2452250|49929|10649|62437||659||||1392||||132.06||6074.76|3671.26||242.99||6074.76|6317.75|2403.50| +2452250|49929|4917|62437|1026748|659|170|1|196|1392|8|20.25|39.48|32.37|7.76|258.96|162.00|315.84|0.00|7.76|251.20|251.20|89.20| +2452250|49929|17489|62437|1026748|659|170|1|4|1392|12|41.26|59.82|43.07|0.00|516.84|495.12|717.84|46.51|0.00|516.84|563.35|21.72| +2452250|49929|13117|62437|1026748|659|170|1|293|1392|4|58.29|71.69|0.00|0.00|0.00|233.16|286.76|0.00|0.00|0.00|0.00|-233.16| +2452250|49929|8035|62437|1026748|659|170|1|105|1392|93|26.55|43.80|35.04|358.45|3258.72|2469.15|4073.40|0.00|358.45|2900.27|2900.27|431.12| +2452250|49929|17327|62437|1026748|659|170|1|190|1392|84|77.48|84.45|10.97|0.00|921.48|6508.32|7093.80|82.93|0.00|921.48|1004.41|-5586.84| +2452250|49929|577|62437|1026748|659|170|1|179|1392|32|68.56|119.98|17.99|0.00|575.68|2193.92|3839.36|11.51|0.00|575.68|587.19|-1618.24| +2452250|49929|14319|62437|1026748|659|170|1|92|1392|72|26.98|49.64|17.37|0.00|1250.64|1942.56|3574.08|100.05|0.00|1250.64|1350.69|-691.92| +2452250|49929|12781|62437|1026748|659|170|1|71|1392|95|10.69|19.45|15.56|0.00|1478.20|1015.55|1847.75|0.00|0.00|1478.20|1478.20|462.65| +2452250|49929|16073|62437|1026748|659|170|1|85|1392|54|62.51|80.63|63.69|0.00|3439.26|3375.54|4354.02|309.53|0.00|3439.26|3748.79|63.72| +2452249|44878|9335|75469|1409109|4349|42789|8|78|1393|28|22.45|41.53|40.69|751.95|1139.32|628.60|1162.84|15.49|751.95|387.37|402.86|-241.23| +2452249|44878|377|75469|1409109|4349|42789|8|155|1393|68|67.80|98.31|44.23|0.00|3007.64|4610.40|6685.08|120.30|0.00|3007.64|3127.94|-1602.76| +2452249|44878|16059|75469|1409109|4349|42789|8|116|1393|47|16.03|26.28|14.45|590.86|679.15|753.41|1235.16|6.18|590.86|88.29|94.47|-665.12| +2452249|44878|6517|75469|1409109|4349|42789|8|87|1393|75|33.45|56.86|27.86|0.00|2089.50|2508.75|4264.50|167.16|0.00|2089.50|2256.66|-419.25| +2452249|44878|9159||||42789|||1393|56|||||295.68||5924.24|2.24||||-5261.04| +2452249|44878|1165|75469|1409109|4349|42789|8|298|1393|69|89.62|115.60|40.46|1619.20|2791.74|6183.78|7976.40|70.35|1619.20|1172.54|1242.89|-5011.24| +2452249|44878|17479|75469|1409109|4349||8||1393|52|7.94|||0.00|409.24|412.88||4.09|0.00|409.24|413.33|-3.64| +2452249|44878|825|75469|1409109|4349|42789|8|89|1393|73|96.82|123.92|86.74|0.00|6332.02|7067.86|9046.16|443.24|0.00|6332.02|6775.26|-735.84| +2452249|44878|701|||4349|42789||24|1393|||141.32|115.88|5515.88|6489.28|||77.87|5515.88|973.40||| +2452249|44878|3861|75469|1409109|4349|42789|8|97|1393|26|57.11|70.24|0.70|8.19|18.20|1484.86|1826.24|0.20|8.19|10.01|10.21|-1474.85| +2452249|44878|14737|75469|1409109|4349|42789|8|62|1393|16|53.78|95.19|50.45|0.00|807.20|860.48|1523.04|16.14|0.00|807.20|823.34|-53.28| +2452249|44878|16973|75469|1409109|4349|42789|8|116|1393|79|89.59|169.32|93.12|1912.68|7356.48|7077.61|13376.28|435.50|1912.68|5443.80|5879.30|-1633.81| +2451163|53788|11900|81463|763617|3371|6725|8|299|1394|1|56.99|67.24|24.87|0.00|24.87|56.99|67.24|0.00|0.00|24.87|24.87|-32.12| +2451163|53788|11539|81463|763617|3371|6725|8|220|1394|23|27.87|44.59|20.06|156.86|461.38|641.01|1025.57|15.22|156.86|304.52|319.74|-336.49| +2451163|53788|12304|81463|763617|3371|6725|8|97|1394|2|50.61|75.91|72.87|0.00|145.74|101.22|151.82|1.45|0.00|145.74|147.19|44.52| +2451163|53788|2404|81463|763617|3371|6725|8|245|1394|97|22.69|26.77|17.40|0.00|1687.80|2200.93|2596.69|101.26|0.00|1687.80|1789.06|-513.13| +2451163|53788|1033|81463|763617|3371|6725|8|241|1394|95|53.44|55.04|17.61|0.00|1672.95|5076.80|5228.80|0.00|0.00|1672.95|1672.95|-3403.85| +2451163|53788|8164|81463|763617|3371|6725|8|274|1394|56|84.30|94.41|45.31|1294.05|2537.36|4720.80|5286.96|111.89|1294.05|1243.31|1355.20|-3477.49| +2451163|53788|10574|81463|763617|3371|6725|8|62|1394|66|24.69|29.87|27.18|0.00|1793.88|1629.54|1971.42|107.63|0.00|1793.88|1901.51|164.34| +|53788|8536||763617|||8|147|1394|||63.07|3.15|4.72|15.75|||0.55|4.72||11.58|-249.62| +2451163|53788|2954|81463|763617|3371|6725|8|293|1394|34|41.02|78.75|40.95|0.00|1392.30|1394.68|2677.50|69.61|0.00|1392.30|1461.91|-2.38| +2451163|53788|7132|81463|763617|3371|6725|8|28|1394|64|72.29|88.19|41.44|0.00|2652.16|4626.56|5644.16|185.65|0.00|2652.16|2837.81|-1974.40| +2451163|53788|11647|81463|763617|3371|6725|8|202|1394|25|24.59|25.32|17.97|0.00|449.25|614.75|633.00|17.97|0.00|449.25|467.22|-165.50| +2451163|53788|15637|81463|763617|3371|6725|8|255|1394|72|64.44|116.63|27.99|0.00|2015.28|4639.68|8397.36|100.76|0.00|2015.28|2116.04|-2624.40| +2451163|53788|853|81463|763617|3371|6725|8|233|1394|79|20.27|30.60|0.61|0.00|48.19|1601.33|2417.40|0.00|0.00|48.19|48.19|-1553.14| +2451163||2041|||||||1394|52|||3.43|0.00|178.36|495.56|495.56|10.70|0.00||189.06|| +2452256|72330|3431|44143|790129|2162|46548|8|89|1395|33|61.74|85.81|30.89|0.00|1019.37|2037.42|2831.73|0.00|0.00|1019.37|1019.37|-1018.05| +2452256|72330|12099|44143|790129|2162|46548|8|216|1395|79|40.67|78.49|71.42|0.00|5642.18|3212.93|6200.71|112.84|0.00|5642.18|5755.02|2429.25| +2452256|72330|7931|44143|790129|2162|46548|8|281|1395|95|9.61|18.06|13.18|0.00|1252.10|912.95|1715.70|100.16|0.00|1252.10|1352.26|339.15| +2452256|72330|9755|44143|790129|2162|46548|8|106|1395|6|58.61|87.32|20.08|0.00|120.48|351.66|523.92|2.40|0.00|120.48|122.88|-231.18| +2452256|72330|15667|44143|790129|2162|46548|8|249|1395|41|76.69|141.10|7.05|0.00|289.05|3144.29|5785.10|17.34|0.00|289.05|306.39|-2855.24| +2452256|72330|5397|44143|790129|2162|46548|8|34|1395|82|44.31|72.66|20.34|0.00|1667.88|3633.42|5958.12|133.43|0.00|1667.88|1801.31|-1965.54| +2452256|72330|1685|44143|790129|2162|46548|8|53|1395|69|60.73|113.56|79.49|1480.89|5484.81|4190.37|7835.64|280.27|1480.89|4003.92|4284.19|-186.45| +||16771||||||209|1395|||||2895.91|3967.00||||2895.91|1071.09|1124.64|| +2452256|72330|8851|44143|790129|2162|46548|8|296|1395|66|6.17|7.65|4.66|0.00|307.56|407.22|504.90|12.30|0.00|307.56|319.86|-99.66| +2452256|72330|6443|44143|790129|2162|46548|8|186|1395|13|35.61|50.56|24.77|0.00|322.01|462.93|657.28|6.44|0.00|322.01|328.45|-140.92| +2452256|72330|16433|44143|790129|2162|46548|8|266|1395|60|31.17|57.35|39.57|0.00|2374.20|1870.20|3441.00|166.19|0.00|2374.20|2540.39|504.00| +2452256|72330|12797|44143|790129|2162|46548|8|86|1395|23|28.58|42.01|23.94|0.00|550.62|657.34|966.23|0.00|0.00|550.62|550.62|-106.72| +2452256|72330|5617|44143|790129|2162|46548|8|221|1395|62|49.53|73.79|50.17|0.00|3110.54|3070.86|4574.98|124.42|0.00|3110.54|3234.96|39.68| +2452256|72330|15247|44143|790129|2162|46548|8|149|1395|75|28.12|41.33|8.26|0.00|619.50|2109.00|3099.75|6.19|0.00|619.50|625.69|-1489.50| +2451597|63917|10483|73835|375686|2337|14941|10|243|1396|98|44.17|63.16|31.58|0.00|3094.84|4328.66|6189.68|216.63|0.00|3094.84|3311.47|-1233.82| +2451597|63917|14893|73835|375686|2337|14941|10|20|1396|92|3.00|3.39|1.42|0.00|130.64|276.00|311.88|10.45|0.00|130.64|141.09|-145.36| +2451597|63917|1568|73835|||14941||89|1396||43.19|62.19|||410.30|||||410.30||-1965.15| +2451597|63917|15215|73835|375686|2337|14941|10|167|1396|17|72.69|82.13|75.55|0.00|1284.35|1235.73|1396.21|89.90|0.00|1284.35|1374.25|48.62| +2451597|63917|13070|73835|375686|2337|14941|10|177|1396|96|16.86|16.86|12.47|0.00|1197.12|1618.56|1618.56|23.94|0.00|1197.12|1221.06|-421.44| +2451597|63917|1933|73835|375686|2337|14941|10|173|1396|24|1.47|2.58|1.83|0.00|43.92|35.28|61.92|2.63|0.00|43.92|46.55|8.64| +2451597|63917|13015|73835|375686|2337|14941|10|13|1396|71|3.57|3.92|3.68|0.00|261.28|253.47|278.32|0.00|0.00|261.28|261.28|7.81| +2451597|63917|11720|73835|375686|2337|14941|10|97|1396|79|57.03|65.58|64.92|0.00|5128.68|4505.37|5180.82|51.28|0.00|5128.68|5179.96|623.31| +|63917|9494||||14941||222|1396|71|50.57|78.38|||5508.89|3590.47|5564.98|495.80||5508.89|6004.69|| +2451597|63917|11689||375686|2337|14941|||1396|||109.53|95.29||||8652.87|602.23||7527.91||| +2451597||11894|73835|||||42|1396||||81.54|||1236.78||44.03||1467.72||| +2451597|63917|15038|73835|375686|2337|14941|10|274|1396|45|35.28|57.85|17.93|330.80|806.85|1587.60|2603.25|9.52|330.80|476.05|485.57|-1111.55| +2451597|63917|5720|73835|375686|2337|14941|10|272|1396|58|30.22|41.09|34.51|0.00|2001.58|1752.76|2383.22|40.03|0.00|2001.58|2041.61|248.82| +2451597|63917|5384|73835|375686|2337|14941|10|289|1396|92|42.39|50.44|17.65|0.00|1623.80|3899.88|4640.48|81.19|0.00|1623.80|1704.99|-2276.08| +2452638|48907|15661|61176|170932|1935|41291|4|241|1397|4|31.22|59.94|43.15|139.80|172.60|124.88|239.76|1.31|139.80|32.80|34.11|-92.08| +2452638|48907|3798|61176|170932|1935|41291|4|196|1397|15|36.72|51.77|35.20|95.04|528.00|550.80|776.55|17.31|95.04|432.96|450.27|-117.84| +|48907|12153|61176||1935||4||1397|25||||1482.71||1687.75|||1482.71||79.60|| +2452638|48907|4143|61176|170932|1935|41291|4|262|1397|83|41.79|56.41|33.28|0.00|2762.24|3468.57|4682.03|220.97|0.00|2762.24|2983.21|-706.33| +2452638|48907|16573|61176|170932|1935|41291|4|74|1397|74|15.11|24.02|21.61|783.57|1599.14|1118.14|1777.48|40.77|783.57|815.57|856.34|-302.57| +2452638|48907|8514|61176|170932|1935|41291|4|68|1397|22|30.21|38.06|12.55|0.00|276.10|664.62|837.32|11.04|0.00|276.10|287.14|-388.52| +2452638|48907|4395|61176|170932|1935|41291|4|41|1397|70|20.87|23.16|20.38|0.00|1426.60|1460.90|1621.20|71.33|0.00|1426.60|1497.93|-34.30| +2452638|48907|16554|61176|170932|1935|41291|4|116|1397|15|98.98|175.19|89.34|0.00|1340.10|1484.70|2627.85|26.80|0.00|1340.10|1366.90|-144.60| +2452638|48907|2598|61176|170932|1935|41291|4|286|1397|6|42.63|50.30|34.70|0.00|208.20|255.78|301.80|10.41|0.00|208.20|218.61|-47.58| +2452638|48907|1713|61176|170932|1935|41291|4|41|1397|53|61.48|96.52|75.28|0.00|3989.84|3258.44|5115.56|119.69|0.00|3989.84|4109.53|731.40| +2452638|48907|15546|61176|170932|1935|41291|4|186|1397|77|2.16|3.71|2.30|46.04|177.10|166.32|285.67|1.31|46.04|131.06|132.37|-35.26| +2451533|39563|5972|6951|307027|350|39226|8|16|1398|92|58.10|112.13|106.52|4605.92|9799.84|5345.20|10315.96|363.57|4605.92|5193.92|5557.49|-151.28| +2451533|39563|8980|6951|307027|350|39226|8|179|1398|67|93.57|136.61|38.25|0.00|2562.75|6269.19|9152.87|76.88|0.00|2562.75|2639.63|-3706.44| +2451533|39563|6034|6951|307027|350|39226|8|167|1398|45|75.63|77.14|26.99|0.00|1214.55|3403.35|3471.30|60.72|0.00|1214.55|1275.27|-2188.80| +2451533|39563|9139|6951|307027|350|39226|8|152|1398|8|74.94|133.39|50.68|0.00|405.44|599.52|1067.12|4.05|0.00|405.44|409.49|-194.08| +2451533|39563|12676|6951|307027|350|39226|8|155|1398|10|48.56|81.09|41.35|0.00|413.50|485.60|810.90|33.08|0.00|413.50|446.58|-72.10| +2451533|39563|12974|6951|307027|350|39226|8|82|1398|64|76.63|123.37|101.16|0.00|6474.24|4904.32|7895.68|194.22|0.00|6474.24|6668.46|1569.92| +2451533|39563|5672|6951|307027|350|39226|8|265|1398|70|66.96|76.33|72.51|0.00|5075.70|4687.20|5343.10|456.81|0.00|5075.70|5532.51|388.50| +2451533|39563|12668|6951|307027|350|39226|8|83|1398|37|5.34|7.84|6.42|0.00|237.54|197.58|290.08|2.37|0.00|237.54|239.91|39.96| +2451533|39563|1486|6951|307027|350|39226|8|204|1398|59|56.69|66.32|38.46|1883.38|2269.14|3344.71|3912.88|0.00|1883.38|385.76|385.76|-2958.95| +2451533|39563|14456|6951|307027|350|39226|8|281|1398|85|6.82|9.75|3.99|301.84|339.15|579.70|828.75|2.61|301.84|37.31|39.92|-542.39| +2451533|39563|5665|6951|307027|350|39226|8|126|1398|11|82.39|84.03|17.64|0.00|194.04|906.29|924.33|1.94|0.00|194.04|195.98|-712.25| +2452232|50256|3395|81152|1522609|4419|9429|1|37|1399|1|4.11|7.60|5.92|0.00|5.92|4.11|7.60|0.23|0.00|5.92|6.15|1.81| +2452232|50256|2221|81152|1522609|4419|9429|1|185|1399|70|83.96|131.81|67.22|0.00|4705.40|5877.20|9226.70|329.37|0.00|4705.40|5034.77|-1171.80| +2452232|50256|12289|81152|1522609|4419|9429|1|219|1399|13|77.56|122.54|40.43|0.00|525.59|1008.28|1593.02|47.30|0.00|525.59|572.89|-482.69| +2452232|50256|6431|81152|1522609|4419|9429|1|88|1399|96|6.93|9.14|5.30|0.00|508.80|665.28|877.44|0.00|0.00|508.80|508.80|-156.48| +||13759||||9429||45|1399|11|80.85||||1021.90|889.35||24.83||827.74|852.57|| +2452232|50256|16523|81152|1522609|4419|9429|1|173|1399|29|29.41|49.99|25.99|437.15|753.71|852.89|1449.71|3.16|437.15|316.56|319.72|-536.33| +2452232|50256|12071|81152|1522609|4419|9429|1|11|1399|22|7.81|9.37|4.87|0.00|107.14|171.82|206.14|0.00|0.00|107.14|107.14|-64.68| +2452232|50256|12441|81152|1522609|4419|9429|1|60|1399|74|84.44|101.32|37.48|0.00|2773.52|6248.56|7497.68|138.67|0.00|2773.52|2912.19|-3475.04| +2452232|50256|12779|81152|1522609|4419|9429|1|202|1399|9|41.48|75.90|40.22|0.00|361.98|373.32|683.10|32.57|0.00|361.98|394.55|-11.34| +2451159|40016|7975|8341|83505|716|48680|7|43|1400|66|4.51|5.68|0.34|0.00|22.44|297.66|374.88|0.22|0.00|22.44|22.66|-275.22| +|40016|12674|8341|83505|716|48680||13|1400||78.50|125.60||||||||8208.70||| +2451159|40016|2084|8341|83505|716|48680|7|187|1400|37|73.12|87.01|11.31|179.94|418.47|2705.44|3219.37|14.31|179.94|238.53|252.84|-2466.91| +|40016|1210|8341|83505||48680|||1400|79||||||149.31||||21.02|22.28|-128.29| +2451159|40016|17389|8341|83505|716|48680|7|139|1400|67|26.42|28.53|21.39|0.00|1433.13|1770.14|1911.51|42.99|0.00|1433.13|1476.12|-337.01| +2451159|40016|12608|8341|83505|716|48680|7|299|1400|82|25.83|51.14|24.54|0.00|2012.28|2118.06|4193.48|80.49|0.00|2012.28|2092.77|-105.78| +2451159|40016|6422|8341|83505|716|48680|7|114|1400|52|61.46|65.14|22.14|0.00|1151.28|3195.92|3387.28|103.61|0.00|1151.28|1254.89|-2044.64| +2451159|40016|1178|8341|83505|716|48680|7|141|1400|1|91.61|149.32|19.41|0.00|19.41|91.61|149.32|1.55|0.00|19.41|20.96|-72.20| +2451886|48300|16321|79810|462020|5383|49193|2|106|1401|27|73.21|136.17|99.40|0.00|2683.80|1976.67|3676.59|214.70|0.00|2683.80|2898.50|707.13| +2451886|48300|8489|79810|462020|5383|49193|2|92|1401|89|48.27|92.67|48.18|3644.81|4288.02|4296.03|8247.63|12.86|3644.81|643.21|656.07|-3652.82| +2451886|48300|2503|79810|462020|5383|49193|2|163|1401|11|44.09|49.38|3.95|15.64|43.45|484.99|543.18|0.55|15.64|27.81|28.36|-457.18| +2451886|48300|7199|79810|462020|5383|49193|2|155|1401|26|50.76|67.51|36.45|0.00|947.70|1319.76|1755.26|75.81|0.00|947.70|1023.51|-372.06| +2451886|48300|9125|79810|462020|5383|49193|2|26|1401|81|57.42|70.62|52.96|0.00|4289.76|4651.02|5720.22|300.28|0.00|4289.76|4590.04|-361.26| +|48300|11213||462020|5383||2||1401|90|96.42||125.28|0.00|11275.20|8677.80|||0.00|11275.20||2597.40| +2451886|48300|6968|79810|462020|5383|49193|2|38|1401|7|41.94|54.10|17.85|0.00|124.95|293.58|378.70|2.49|0.00|124.95|127.44|-168.63| +2451886|48300|13985|79810|462020|5383|49193|2|42|1401|20|88.65|104.60|6.27|0.00|125.40|1773.00|2092.00|7.52|0.00|125.40|132.92|-1647.60| +2451886|48300|15209|79810|462020|5383|49193|2|9|1401|10|52.72|102.27|93.06|0.00|930.60|527.20|1022.70|74.44|0.00|930.60|1005.04|403.40| +||2204|||5383|||7|1401||3.49|||0.00|||37.44|0.86|0.00|28.80||7.86| +2451886|48300|8498|79810|462020|5383|49193|2|245|1401|85|14.13|16.39|11.80|0.00|1003.00|1201.05|1393.15|60.18|0.00|1003.00|1063.18|-198.05| +2451886|48300|8822|79810|462020|5383|49193|2|223|1401|67|60.85|120.48|30.12|0.00|2018.04|4076.95|8072.16|181.62|0.00|2018.04|2199.66|-2058.91| +|48300|95|79810|462020|||||1401|43|18.35|19.45|12.05|||789.05|836.35|||||-270.90| +2451150|36334|10406|21757|1881369|181|39440|2|75|1402|50|41.71|75.49|21.89|0.00|1094.50|2085.50|3774.50|54.72|0.00|1094.50|1149.22|-991.00| +2451150|36334|8876|21757|1881369|181|39440|2|124|1402|56|99.03|129.72|123.23|1932.24|6900.88|5545.68|7264.32|347.80|1932.24|4968.64|5316.44|-577.04| +2451150|36334|11683|21757|1881369|181|39440|2|261|1402|7|48.85|96.23|51.00|0.00|357.00|341.95|673.61|21.42|0.00|357.00|378.42|15.05| +2451150|36334|2407|21757|1881369|181|39440|2|191|1402|20|31.66|63.32|20.89|0.00|417.80|633.20|1266.40|29.24|0.00|417.80|447.04|-215.40| +2451150|36334|15520|21757|1881369|181|39440|2|228|1402|17|77.16|101.07|74.79|0.00|1271.43|1311.72|1718.19|114.42|0.00|1271.43|1385.85|-40.29| +2451150|36334|7070|21757|1881369|181|39440|2|152|1402|86|28.21|31.31|23.16|1453.98|1991.76|2426.06|2692.66|48.40|1453.98|537.78|586.18|-1888.28| +2451150|36334|6104|21757|1881369|181|39440|2|194|1402|88|43.95|56.69|56.69|0.00|4988.72|3867.60|4988.72|249.43|0.00|4988.72|5238.15|1121.12| +2451150|36334|14623|21757|1881369|181|39440|2|293|1402|35|80.18|158.75|41.27|0.00|1444.45|2806.30|5556.25|14.44|0.00|1444.45|1458.89|-1361.85| +2451150|36334|5854|21757|1881369|181|39440|2|145|1402|88|93.89|160.55|11.23|296.47|988.24|8262.32|14128.40|34.58|296.47|691.77|726.35|-7570.55| +2451150|36334|17131|21757|1881369|181|39440|2|271|1402|28|16.79|26.02|25.75|0.00|721.00|470.12|728.56|57.68|0.00|721.00|778.68|250.88| +2451150|36334|11996|21757|1881369|181|39440|2|297|1402|41|26.66|29.59|1.18|0.00|48.38|1093.06|1213.19|2.41|0.00|48.38|50.79|-1044.68| +2451150|36334|8794|21757|1881369|181|39440|2|175|1402|62|71.14|86.79|72.90|0.00|4519.80|4410.68|5380.98|135.59|0.00|4519.80|4655.39|109.12| +2451901|36092|15493|19794|1569487|1580|30073|10|13|1403|84|60.53|115.61|61.27|0.00|5146.68|5084.52|9711.24|51.46|0.00|5146.68|5198.14|62.16| +2451901|36092|5618|19794|1569487|1580|30073|10|4|1403|54|96.74|131.56|55.25|0.00|2983.50|5223.96|7104.24|149.17|0.00|2983.50|3132.67|-2240.46| +2451901|36092|8021|19794|1569487|1580|30073|10|84|1403|95|9.29|15.23|13.09|0.00|1243.55|882.55|1446.85|49.74|0.00|1243.55|1293.29|361.00| +2451901|36092|17021|19794|1569487|1580|30073|10|115|1403|45|91.91|150.73|123.59|2224.62|5561.55|4135.95|6782.85|233.58|2224.62|3336.93|3570.51|-799.02| +2451901|36092|1832|19794|1569487|1580|30073|10|277|1403|36|65.70|96.57|44.42|0.00|1599.12|2365.20|3476.52|31.98|0.00|1599.12|1631.10|-766.08| +2451901|36092|14834|19794|1569487|1580|30073|10|170|1403|32|76.35|106.89|47.03|0.00|1504.96|2443.20|3420.48|0.00|0.00|1504.96|1504.96|-938.24| +2451901|36092|2735|19794|1569487|1580|30073|10|161|1403|66|97.83|143.81|41.70|0.00|2752.20|6456.78|9491.46|220.17|0.00|2752.20|2972.37|-3704.58| +2451901|36092|1466|19794|1569487|1580|30073|10|256|1403|78|42.66|81.90|9.00|372.06|702.00|3327.48|6388.20|6.59|372.06|329.94|336.53|-2997.54| +|36523|635|95509|7288|1114|34493||163|1404|24||||0.00||1543.68|1667.04|1.66|0.00|166.56|168.22|-1377.12| +2451609|36523|557|95509|7288|1114|34493|10|191|1404|63|97.72|161.23|72.55|0.00|4570.65|6156.36|10157.49|411.35|0.00|4570.65|4982.00|-1585.71| +2451609|36523|12941|95509|7288|1114|34493|10|275|1404|69|16.33|20.08|15.66|162.08|1080.54|1126.77|1385.52|73.47|162.08|918.46|991.93|-208.31| +2451609|36523|12020|95509|7288|1114|34493|10|207|1404|33|37.01|46.63|13.52|0.00|446.16|1221.33|1538.79|35.69|0.00|446.16|481.85|-775.17| +2451609|36523|9055|95509|7288|1114|34493|10|237|1404|28|93.60|109.51|14.23|0.00|398.44|2620.80|3066.28|19.92|0.00|398.44|418.36|-2222.36| +2451609||7271|95509|7288||34493|||1404|24||50.59|||1007.52|867.36||||1007.52|1047.82|| +2451609|36523|11377|95509|7288|1114|34493|10|34|1404|51|74.95|144.65|10.12|10.32|516.12|3822.45|7377.15|5.05|10.32|505.80|510.85|-3316.65| +2451609|36523|13381|95509|7288|1114|34493|10|161|1404|47|62.46|98.06|84.33|0.00|3963.51|2935.62|4608.82|317.08|0.00|3963.51|4280.59|1027.89| +2451609|36523|12493|95509|7288|1114|34493|10|237|1404|96|58.60|73.83|31.00|0.00|2976.00|5625.60|7087.68|238.08|0.00|2976.00|3214.08|-2649.60| +2451609|36523|9749|95509|7288|1114|34493|10|193|1404|96|4.59|4.91|2.11|0.00|202.56|440.64|471.36|12.15|0.00|202.56|214.71|-238.08| +||9409|95509||1114|34493||285|1404||||9.53|0.00|905.35||2384.50|0.00|0.00||905.35|-1432.60| +2451609|36523|6605|95509|7288|1114|34493|10|72|1404|80|57.58|70.82|59.48|0.00|4758.40|4606.40|5665.60|47.58|0.00|4758.40|4805.98|152.00| +2451609|36523|7127|95509|7288|1114|34493|10|241|1404|49|6.53|10.83|7.36|0.00|360.64|319.97|530.67|7.21|0.00|360.64|367.85|40.67| +2451609|36523|17779|95509|7288|1114|34493|10|193|1404|95|57.47|108.04|93.99|3660.91|8929.05|5459.65|10263.80|474.13|3660.91|5268.14|5742.27|-191.51| +2451609|36523|2891|95509|7288|1114|34493|10|249|1404|62|96.11|155.69|88.74|0.00|5501.88|5958.82|9652.78|385.13|0.00|5501.88|5887.01|-456.94| +2451423|35905|7960|34235|626513|391|32669|1|113|1405|29|94.48|157.78|104.13|0.00|3019.77|2739.92|4575.62|211.38|0.00|3019.77|3231.15|279.85| +2451423|35905|5593|34235|626513|391|32669|1|144|1405|92|53.49|105.37|46.36|0.00|4265.12|4921.08|9694.04|42.65|0.00|4265.12|4307.77|-655.96| +2451423|35905|16828|34235|626513|391|32669|1|220|1405|38|98.34|193.72|83.29|0.00|3165.02|3736.92|7361.36|63.30|0.00|3165.02|3228.32|-571.90| +2451423|35905|16264|34235|626513|391|32669|1|273|1405|76|74.67|74.67|7.46|0.00|566.96|5674.92|5674.92|39.68|0.00|566.96|606.64|-5107.96| +2451423|35905|2330|34235|626513|391|32669|1|38|1405|60|31.89|60.27|26.51|938.45|1590.60|1913.40|3616.20|6.52|938.45|652.15|658.67|-1261.25| +2451423|35905|4096|34235|626513|391|32669|1|41|1405|75|24.94|40.40|1.21|0.00|90.75|1870.50|3030.00|6.35|0.00|90.75|97.10|-1779.75| +2451423|35905|7502|34235|626513|391|32669|1|156|1405|94|76.28|105.26|26.31|2052.70|2473.14|7170.32|9894.44|33.63|2052.70|420.44|454.07|-6749.88| +2451423|35905|5242|34235|626513|391|32669|1|283|1405|66|45.67|52.97|37.07|0.00|2446.62|3014.22|3496.02|171.26|0.00|2446.62|2617.88|-567.60| +2451423|35905|3235|34235|626513|391|32669|1|56|1405|42|43.42|52.97|16.42|0.00|689.64|1823.64|2224.74|6.89|0.00|689.64|696.53|-1134.00| +2451825|52421|9571|38652|883455|3458|21061|7|70|1406|29|82.75|95.99|32.63|0.00|946.27|2399.75|2783.71|47.31|0.00|946.27|993.58|-1453.48| +2451825|52421|10219|38652|883455|3458|21061|7|120|1406|72|8.65|8.65|2.85|0.00|205.20|622.80|622.80|2.05|0.00|205.20|207.25|-417.60| +2451825|52421|17198|38652|883455|3458|21061|7|187|1406|1|4.56|8.20|3.85|0.38|3.85|4.56|8.20|0.27|0.38|3.47|3.74|-1.09| +2451825|52421|3523|38652|883455|3458|21061|7|247|1406|81|96.24|182.85|49.36|3078.58|3998.16|7795.44|14810.85|0.00|3078.58|919.58|919.58|-6875.86| +2451825|52421|15125|38652|883455|3458|21061|7|181|1406|35|34.31|52.15|49.02|0.00|1715.70|1200.85|1825.25|51.47|0.00|1715.70|1767.17|514.85| +2451825|52421|371|38652|883455|3458|21061|7|95|1406|42|7.60|8.66|5.88|0.00|246.96|319.20|363.72|17.28|0.00|246.96|264.24|-72.24| +2451825|52421|8947|38652|883455|3458|21061|7|199|1406|7|60.87|74.26|5.94|15.80|41.58|426.09|519.82|1.03|15.80|25.78|26.81|-400.31| +2451825|52421|15386|38652|883455|3458|21061|7|79|1406|39|85.82|96.97|19.39|0.00|756.21|3346.98|3781.83|15.12|0.00|756.21|771.33|-2590.77| +2451799||8855|30665||6585|10145|||1407||||3.27|||32.70|65.40|||3.27|3.30|| +2451799|62133|15731|30665|1670225|6585|10145|8|122|1407|77|36.74|49.59|47.11|0.00|3627.47|2828.98|3818.43|36.27|0.00|3627.47|3663.74|798.49| +2451799|62133|15110|30665|1670225|6585|10145|8|135|1407|99|10.99|15.93|14.33|0.00|1418.67|1088.01|1577.07|113.49|0.00|1418.67|1532.16|330.66| +2451799|62133|5957|30665|1670225|6585|10145|8|14|1407|22|9.82|15.22|8.67|0.00|190.74|216.04|334.84|7.62|0.00|190.74|198.36|-25.30| +2451799|62133|11963|30665|1670225|6585|10145|8|60|1407|76|92.68|112.14|109.89|0.00|8351.64|7043.68|8522.64|167.03|0.00|8351.64|8518.67|1307.96| +2451799|62133|9725|30665|1670225|6585|10145|8|198|1407|56|99.21|133.93|72.32|0.00|4049.92|5555.76|7500.08|40.49|0.00|4049.92|4090.41|-1505.84| +|62133|8846|30665|1670225|||8|94|1407|94|35.47|40.79||182.83|344.98|3334.18|3834.26|9.72|182.83|162.15||| +2451799|62133|10435|30665|1670225|6585|10145|8|119|1407|10|47.12|66.43|8.63|0.00|86.30|471.20|664.30|0.00|0.00|86.30|86.30|-384.90| +2451799|62133|15650|30665|1670225|6585|10145|8|62|1407|13|71.31|87.71|7.01|0.00|91.13|927.03|1140.23|3.64|0.00|91.13|94.77|-835.90| +2451799|62133|17999|30665|1670225|6585|10145|8|12|1407|88|76.08|128.57|10.28|0.00|904.64|6695.04|11314.16|9.04|0.00|904.64|913.68|-5790.40| +2451799|62133|3614|30665|1670225|6585|10145|8|132|1407|92|98.34|172.09|34.41|0.00|3165.72|9047.28|15832.28|63.31|0.00|3165.72|3229.03|-5881.56| +2451799|62133|9266|30665|1670225|6585|10145|8|209|1407|80|73.51|80.86|34.76|0.00|2780.80|5880.80|6468.80|250.27|0.00|2780.80|3031.07|-3100.00| +2451799|62133|15943|30665|1670225|6585|10145|8|135|1407|81|87.53|175.06|42.01|0.00|3402.81|7089.93|14179.86|170.14|0.00|3402.81|3572.95|-3687.12| +2451799|62133|14189|30665|1670225|6585|10145|8|269|1407|29|42.37|68.63|28.82|0.00|835.78|1228.73|1990.27|75.22|0.00|835.78|911.00|-392.95| +2452611|59006|7326|17861|303947|5586|22072|7|299|1408|79|6.26|11.01|10.34|514.62|816.86|494.54|869.79|24.17|514.62|302.24|326.41|-192.30| +2452611||11868||||||186|1408||35.37|36.07|21.28||1851.36|||74.05||1851.36||| +2452611|59006|2893|17861|303947|5586|22072|7|273|1408|41|40.05|76.09|70.76|0.00|2901.16|1642.05|3119.69|58.02|0.00|2901.16|2959.18|1259.11| +2452611|59006|7485|17861|303947|5586|22072|7|179|1408|82|96.97|100.84|66.55|0.00|5457.10|7951.54|8268.88|436.56|0.00|5457.10|5893.66|-2494.44| +2452611|59006|7419|17861|303947|5586|22072|7|106|1408|90|97.84|153.60|52.22|2678.88|4699.80|8805.60|13824.00|40.41|2678.88|2020.92|2061.33|-6784.68| +2452611||10093||||22072|7|212|1408|||||0.00|||1086.34||0.00||453.03|-521.42| +2452611|59006|11089|17861|303947|5586|22072|7|24|1408|80|66.65|66.65|59.31|0.00|4744.80|5332.00|5332.00|0.00|0.00|4744.80|4744.80|-587.20| +2452611|59006|13911|17861|303947|5586|22072|7|80|1408|61|49.93|54.92|50.52|462.25|3081.72|3045.73|3350.12|78.58|462.25|2619.47|2698.05|-426.26| +2451778|61218|6047|8286|257902|1471|1487|10|250|1409|64|63.56|79.45|37.34|0.00|2389.76|4067.84|5084.80|167.28|0.00|2389.76|2557.04|-1678.08| +2451778|61218|4631|8286|257902|1471|1487|10|2|1409|41|58.32|82.81|50.51|414.18|2070.91|2391.12|3395.21|82.83|414.18|1656.73|1739.56|-734.39| +2451778|61218|17942|8286|257902|1471|1487|10|119|1409|17|60.43|120.25|78.16|0.00|1328.72|1027.31|2044.25|26.57|0.00|1328.72|1355.29|301.41| +2451778|61218|9842|8286|257902|1471|1487|10|223|1409|30|95.58|187.33|179.83|0.00|5394.90|2867.40|5619.90|161.84|0.00|5394.90|5556.74|2527.50| +2451778|61218|17587|8286|257902|1471|1487|10|84|1409|82|7.14|14.28|2.14|0.00|175.48|585.48|1170.96|0.00|0.00|175.48|175.48|-410.00| +2451778|61218|2900|8286|257902|1471|1487|10|155|1409|97|35.51|54.33|51.61|3354.13|5006.17|3444.47|5270.01|115.64|3354.13|1652.04|1767.68|-1792.43| +2451778|61218|5545|8286|257902|1471|1487|10|29|1409|100|20.92|31.58|15.79|773.71|1579.00|2092.00|3158.00|72.47|773.71|805.29|877.76|-1286.71| +2451778|61218|12563|8286|257902|1471|1487|10|136|1409|53|13.77|14.04|7.58|341.47|401.74|729.81|744.12|5.42|341.47|60.27|65.69|-669.54| +2451778|61218|16885|8286|257902|1471|1487|10|260|1409|76|17.55|33.87|18.96|0.00|1440.96|1333.80|2574.12|28.81|0.00|1440.96|1469.77|107.16| +2451862|34150|15637|31317|377820|6723|5226|4|220|1410|75|75.26|127.94|24.30|0.00|1822.50|5644.50|9595.50|18.22|0.00|1822.50|1840.72|-3822.00| +2451862|34150|853|31317|377820|6723|5226|4|177|1410|34|46.95|73.71|2.94|0.00|99.96|1596.30|2506.14|0.00|0.00|99.96|99.96|-1496.34| +2451862|34150|2041|31317|377820|6723|5226|4|120|1410|42|57.64|103.17|50.55|0.00|2123.10|2420.88|4333.14|84.92|0.00|2123.10|2208.02|-297.78| +2451862|34150|5143|31317|377820|6723|5226|4|141|1410|6|65.80|117.12|18.73|0.00|112.38|394.80|702.72|1.12|0.00|112.38|113.50|-282.42| +2451862|34150|6266|31317|377820|6723|5226|4|253|1410|47|20.69|41.17|17.70|590.64|831.90|972.43|1934.99|7.23|590.64|241.26|248.49|-731.17| +2451862|34150|14480|31317|377820|6723|5226|4|198|1410|13|14.70|15.14|6.20|0.00|80.60|191.10|196.82|3.22|0.00|80.60|83.82|-110.50| +2451862|34150|3275|31317|377820|6723|5226|4|19|1410|72|21.55|39.00|19.89|0.00|1432.08|1551.60|2808.00|57.28|0.00|1432.08|1489.36|-119.52| +2451862|34150|14417|31317|377820|6723|5226|4|151|1410|52|52.62|85.77|9.43|0.00|490.36|2736.24|4460.04|19.61|0.00|490.36|509.97|-2245.88| +2451862|34150|10370|31317|377820|6723|5226|4|6|1410|78|28.96|50.39|40.31|0.00|3144.18|2258.88|3930.42|94.32|0.00|3144.18|3238.50|885.30| +2451862|34150|12158|31317|377820|6723|5226|4|36|1410|60|98.59|124.22|105.58|0.00|6334.80|5915.40|7453.20|63.34|0.00|6334.80|6398.14|419.40| +2451862|34150|1334|31317|377820|6723|5226|4|247|1410|40|90.03|123.34|83.87|0.00|3354.80|3601.20|4933.60|0.00|0.00|3354.80|3354.80|-246.40| +2451862|34150|16801|31317|377820|6723|5226|4|235|1410|13|26.05|28.13|21.94|0.00|285.22|338.65|365.69|17.11|0.00|285.22|302.33|-53.43| +2451862|34150|8813|31317|377820|6723|5226|4|156|1410|24|9.02|9.11|2.09|45.14|50.16|216.48|218.64|0.15|45.14|5.02|5.17|-211.46| +2452170|62567|10275|75555|962900|4645|3190|1|88|1411|23|3.14|6.06|5.02|0.00|115.46|72.22|139.38|5.77|0.00|115.46|121.23|43.24| +2452170|62567|5553|75555|962900|4645|3190|1|29|1411|20|70.91|95.01|11.40|0.00|228.00|1418.20|1900.20|13.68|0.00|228.00|241.68|-1190.20| +2452170|62567|1319|75555|962900|4645|3190|1|38|1411|59|93.75|152.81|27.50|0.00|1622.50|5531.25|9015.79|129.80|0.00|1622.50|1752.30|-3908.75| +2452170|62567|8319|75555|962900|4645|3190|1|52|1411|70|16.67|22.00|4.40|0.00|308.00|1166.90|1540.00|18.48|0.00|308.00|326.48|-858.90| +2452170|62567|121|75555|962900|4645|3190|1|41|1411|50|78.76|113.41|52.16|0.00|2608.00|3938.00|5670.50|0.00|0.00|2608.00|2608.00|-1330.00| +2452170|62567|9773|75555|962900|4645|3190|1|50|1411|51|44.66|83.51|20.04|0.00|1022.04|2277.66|4259.01|81.76|0.00|1022.04|1103.80|-1255.62| +2452170|62567|7631|75555|962900|4645|3190|1|129|1411|11|78.84|109.58|61.36|0.00|674.96|867.24|1205.38|6.74|0.00|674.96|681.70|-192.28| +||10463|||4645||1|89|1411|80|||14.76|0.00||4496.80||94.46|0.00|||-3316.00| +2452170|62567|5057|75555|962900|4645|3190|1|236|1411|84|70.58|88.93|2.66|0.00|223.44|5928.72|7470.12|0.00|0.00|223.44|223.44|-5705.28| +2452170|62567|9153|75555|962900|4645|3190|1|93|1411|12|50.72|91.29|21.90|0.00|262.80|608.64|1095.48|2.62|0.00|262.80|265.42|-345.84| +2452170|62567|3641|75555|962900|4645|3190|1|137|1411|24|42.25|50.70|43.60|0.00|1046.40|1014.00|1216.80|62.78|0.00|1046.40|1109.18|32.40| +||13817|75555|962900|||1||1411|72|85.00|156.40|4.69||337.68|6120.00|11260.80|3.37||337.68||| +2452170|62567|16181|75555|962900|4645|3190|1|191|1411|15|93.68|175.18|47.29|0.00|709.35|1405.20|2627.70|28.37|0.00|709.35|737.72|-695.85| +2452170|62567|15199|75555|962900|4645|3190|1|168|1411|73|77.89|130.07|46.82|0.00|3417.86|5685.97|9495.11|205.07|0.00|3417.86|3622.93|-2268.11| +2452170|62567|16263|75555|962900|4645|3190|1|210|1411|77|27.32|41.25|35.88|0.00|2762.76|2103.64|3176.25|138.13|0.00|2762.76|2900.89|659.12| +2452637|67562|1963|35809|127229|696|35716|10|89|1412|64|29.27|48.58|47.60|0.00|3046.40|1873.28|3109.12|182.78|0.00|3046.40|3229.18|1173.12| +2452637|67562|6739|35809|127229|696|35716|10|9|1412|26|61.29|118.90|43.99|0.00|1143.74|1593.54|3091.40|91.49|0.00|1143.74|1235.23|-449.80| +2452637|67562|3075|35809|127229|696|35716|10|104|1412|5|5.94|6.65|6.25|0.00|31.25|29.70|33.25|2.50|0.00|31.25|33.75|1.55| +2452637|67562|14367||||35716|10||1412|||34.09|14.99|0.00||2073.60||26.98|0.00||1376.08|| +2452637|67562|11154|35809|127229|696|35716|10|181|1412|6|4.21|6.35|3.23|17.63|19.38|25.26|38.10|0.01|17.63|1.75|1.76|-23.51| +2452637|67562|7731|35809|127229|696|35716|10|62|1412|79|11.72|18.40|1.47|0.00|116.13|925.88|1453.60|1.16|0.00|116.13|117.29|-809.75| +2452637|67562|7458|35809|127229|696|35716|10|240|1412|78|79.03|104.31|54.24|0.00|4230.72|6164.34|8136.18|169.22|0.00|4230.72|4399.94|-1933.62| +2452637|67562|2841|35809|127229|696|35716|10|163|1412|55|93.42|127.98|3.83|0.00|210.65|5138.10|7038.90|12.63|0.00|210.65|223.28|-4927.45| +2452637|67562|12642|35809|127229|696|35716|10|41|1412|31|68.14|68.14|42.24|0.00|1309.44|2112.34|2112.34|91.66|0.00|1309.44|1401.10|-802.90| +2452637|67562|2595|35809|127229|696|35716|10|158|1412|74|88.54|104.47|45.96|0.00|3401.04|6551.96|7730.78|170.05|0.00|3401.04|3571.09|-3150.92| +2452637|67562|2784|35809|127229|696|35716|10|160|1412|3|20.74|27.79|20.00|0.00|60.00|62.22|83.37|0.00|0.00|60.00|60.00|-2.22| +2452637||10165|35809|||||293|1412|5|39.95||5.91||29.55|199.75|295.60|||29.55||| +2452637|67562|14773|35809|127229|696|35716|10|99|1412|52|75.49|93.60|52.41|0.00|2725.32|3925.48|4867.20|54.50|0.00|2725.32|2779.82|-1200.16| +2452637|67562|8529|35809|127229|696|35716|10|275|1412|5|78.65|125.84|123.32|0.00|616.60|393.25|629.20|36.99|0.00|616.60|653.59|223.35| +2451754|32463|13088|19526|1357030|6087|32344|4|231|1413|26|57.33|84.27|36.23|0.00|941.98|1490.58|2191.02|84.77|0.00|941.98|1026.75|-548.60| +2451754|32463|11447|19526|1357030|6087|32344|4|5|1413|19|5.93|11.44|3.88|0.00|73.72|112.67|217.36|6.63|0.00|73.72|80.35|-38.95| +2451754|32463|15104|19526|1357030|6087|32344|4|136|1413|50|47.17|63.67|25.46|0.00|1273.00|2358.50|3183.50|12.73|0.00|1273.00|1285.73|-1085.50| +2451754||1004|19526||6087|32344|4||1413|95|||133.62|||||284.34|||7392.93|| +2451754|32463|3530|19526|1357030|6087|32344|4|17|1413|90|59.42|65.36|53.59|0.00|4823.10|5347.80|5882.40|48.23|0.00|4823.10|4871.33|-524.70| +2451754|32463|9536|19526|1357030|6087|32344|4|298|1413|66|45.69|79.04|39.52|0.00|2608.32|3015.54|5216.64|130.41|0.00|2608.32|2738.73|-407.22| +2451754|32463|2285|19526|1357030|6087|32344|4|142|1413|49|61.78|101.31|14.18|521.11|694.82|3027.22|4964.19|15.63|521.11|173.71|189.34|-2853.51| +2451754|32463|16052|19526|1357030|6087|32344|4|144|1413|66|31.59|39.48|26.84|0.00|1771.44|2084.94|2605.68|141.71|0.00|1771.44|1913.15|-313.50| +2451754|32463|3733|19526|1357030|6087|32344|4|272|1413|20|22.83|43.37|9.54|0.00|190.80|456.60|867.40|3.81|0.00|190.80|194.61|-265.80| +2451754|32463|7841|19526|1357030|6087|32344|4|254|1413|74|2.92|3.38|1.41|82.42|104.34|216.08|250.12|1.75|82.42|21.92|23.67|-194.16| +2451754|32463|5293|19526|1357030|6087|32344|4|58|1413|4|94.40|103.84|66.45|0.00|265.80|377.60|415.36|2.65|0.00|265.80|268.45|-111.80| +2451754|32463|6887|19526|1357030|6087|32344|4|94|1413|67|40.14|42.14|2.52|0.00|168.84|2689.38|2823.38|13.50|0.00|168.84|182.34|-2520.54| +2451754|32463|3494|19526|1357030|6087|32344|4|191|1413|52|40.98|80.73|79.11|0.00|4113.72|2130.96|4197.96|205.68|0.00|4113.72|4319.40|1982.76| +2451754|32463|13393|19526|1357030|6087|32344|4|113|1413|17|19.70|34.86|7.66|0.00|130.22|334.90|592.62|5.20|0.00|130.22|135.42|-204.68| +2451829|50140|12467|24147|1211432|5470|46755|8|80|1414|7|24.19|44.99|11.69|0.00|81.83|169.33|314.93|1.63|0.00|81.83|83.46|-87.50| +2451829|50140|13153|24147|1211432|5470|46755|8|22|1414|42|88.83|162.55|37.38|1444.36|1569.96|3730.86|6827.10|11.30|1444.36|125.60|136.90|-3605.26| +2451829|50140|2819|24147|1211432|5470|46755|8|64|1414|48|64.56|125.89|122.11|0.00|5861.28|3098.88|6042.72|117.22|0.00|5861.28|5978.50|2762.40| +2451829|50140|11258|24147|1211432|5470|46755|8|139|1414|35|22.42|30.93|22.57|229.08|789.95|784.70|1082.55|44.86|229.08|560.87|605.73|-223.83| +2451829|50140|17798|24147|1211432|5470|46755|8|233|1414|83|66.64|113.28|22.65|0.00|1879.95|5531.12|9402.24|150.39|0.00|1879.95|2030.34|-3651.17| +2451829|50140|2828|24147|1211432|5470|46755|8|278|1414|74|81.52|125.54|91.64|0.00|6781.36|6032.48|9289.96|203.44|0.00|6781.36|6984.80|748.88| +2451829|50140|5971|24147|1211432|5470|46755|8|23|1414|73|44.20|45.08|11.72|0.00|855.56|3226.60|3290.84|8.55|0.00|855.56|864.11|-2371.04| +2451829|50140|6683|24147|1211432|5470|46755|8|33|1414|26|54.32|105.38|51.63|0.00|1342.38|1412.32|2739.88|13.42|0.00|1342.38|1355.80|-69.94| +2451829|50140|4430|24147|1211432|5470|46755||136|1414||||32.90||||365.65|6.58||164.50|171.08|| +2451829|50140|6085|24147|1211432|5470|46755|8|34|1414|81|39.19|69.36|46.47|0.00|3764.07|3174.39|5618.16|225.84|0.00|3764.07|3989.91|589.68| +2451829|50140|1760|24147|1211432|5470|46755|8|249|1414|88|84.29|115.47|8.08|0.00|711.04|7417.52|10161.36|63.99|0.00|711.04|775.03|-6706.48| +2451843|33837|12295|45519|1487783|1694|7158|8|90|1415|7|45.15|53.27|28.23|55.33|197.61|316.05|372.89|4.26|55.33|142.28|146.54|-173.77| +2451843|33837|16057|45519|1487783|1694|7158|8|106|1415|62|12.80|19.32|7.34|0.00|455.08|793.60|1197.84|9.10|0.00|455.08|464.18|-338.52| +2451843|33837|14581|45519|1487783|1694|7158|8|270|1415|38|22.75|37.08|4.44|0.00|168.72|864.50|1409.04|1.68|0.00|168.72|170.40|-695.78| +2451843|33837|12923|45519|1487783|1694|7158|8|70|1415|24|68.69|103.72|74.67|0.00|1792.08|1648.56|2489.28|161.28|0.00|1792.08|1953.36|143.52| +2451843|33837|8539|45519|1487783|1694|7158|8|297|1415|60|5.18|8.65|0.51|1.22|30.60|310.80|519.00|0.58|1.22|29.38|29.96|-281.42| +2451843|33837|15691|45519|1487783|1694|7158|8|295|1415|93|91.31|150.66|51.22|4382.38|4763.46|8491.83|14011.38|11.43|4382.38|381.08|392.51|-8110.75| +2451843|33837|7085|45519|1487783|1694|7158|8|229|1415|80|31.99|63.98|7.03|0.00|562.40|2559.20|5118.40|33.74|0.00|562.40|596.14|-1996.80| +2451843|33837|10825|45519|1487783|1694|7158|8|248|1415|67|29.68|56.39|25.37|0.00|1699.79|1988.56|3778.13|33.99|0.00|1699.79|1733.78|-288.77| +2451843|33837|15548|45519|1487783|1694|7158|8|220|1415|47|79.32|129.29|36.20|153.12|1701.40|3728.04|6076.63|61.93|153.12|1548.28|1610.21|-2179.76| +2451843|33837|9548|45519|1487783|1694|7158|8|280|1415|94|32.78|64.24|37.90|0.00|3562.60|3081.32|6038.56|71.25|0.00|3562.60|3633.85|481.28| +2451843|33837|1649|45519|1487783|1694|7158|8|194|1415|63|3.21|4.14|3.51|0.00|221.13|202.23|260.82|17.69|0.00|221.13|238.82|18.90| +2451843|33837|7043|45519|1487783|1694|7158|8|157|1415|21|46.51|61.39|53.40|0.00|1121.40|976.71|1289.19|100.92|0.00|1121.40|1222.32|144.69| +2451394|69833|3388|92100|106105|1710|31315|10|215|1416|72|58.85|65.91|3.29|146.86|236.88|4237.20|4745.52|1.80|146.86|90.02|91.82|-4147.18| +2451394|69833|12925|92100|106105|1710|31315|10|271|1416|49|22.32|39.95|3.19|0.00|156.31|1093.68|1957.55|4.68|0.00|156.31|160.99|-937.37| +2451394|69833|5857|92100|106105|1710|31315|10|11|1416|68|1.08|1.80|0.25|0.00|17.00|73.44|122.40|1.02|0.00|17.00|18.02|-56.44| +2451394|69833|16198|92100|106105|1710|31315|10|136|1416|56|99.05|192.15|149.87|0.00|8392.72|5546.80|10760.40|83.92|0.00|8392.72|8476.64|2845.92| +2451394|69833|4993|92100|106105|1710|31315|10|164|1416|72|34.53|50.75|38.57|83.31|2777.04|2486.16|3654.00|26.93|83.31|2693.73|2720.66|207.57| +2451394|69833|4766|92100|106105|1710|31315|10|271|1416|69|18.02|34.41|28.90|0.00|1994.10|1243.38|2374.29|59.82|0.00|1994.10|2053.92|750.72| +2451394|69833|16093|92100|106105|1710|31315|10|233|1416|87|85.04|164.12|95.18|0.00|8280.66|7398.48|14278.44|331.22|0.00|8280.66|8611.88|882.18| +2451394|69833|7051|92100|106105|1710|31315|10|122|1416|56|66.51|118.38|118.38|2253.95|6629.28|3724.56|6629.28|0.00|2253.95|4375.33|4375.33|650.77| +2451394|69833|12146|92100|106105|1710|31315|10|141|1416|63|13.17|15.14|11.35|357.52|715.05|829.71|953.82|7.15|357.52|357.53|364.68|-472.18| +2451394|69833|13357|92100|106105|1710|31315|10|160|1416|14|64.54|101.97|57.10|0.00|799.40|903.56|1427.58|55.95|0.00|799.40|855.35|-104.16| +2451394|69833|9280|92100|106105|1710|31315|10|73|1416|86|87.83|165.12|1.65|0.00|141.90|7553.38|14200.32|5.67|0.00|141.90|147.57|-7411.48| +2451394|69833|4162|92100|106105|1710|31315|10|284|1416|34|49.60|75.88|46.28|1227.34|1573.52|1686.40|2579.92|27.69|1227.34|346.18|373.87|-1340.22| +2451394|69833|16184|92100|106105|1710|31315|10|269|1416|80|67.92|133.12|113.15|0.00|9052.00|5433.60|10649.60|633.64|0.00|9052.00|9685.64|3618.40| +2451394|69833|13828|92100|106105|1710|31315|10|122|1416|32|98.93|107.83|2.15|0.00|68.80|3165.76|3450.56|5.50|0.00|68.80|74.30|-3096.96| +2451394|69833|11900|92100|106105|1710|31315|10|70|1416|42|82.15|161.01|6.44|0.00|270.48|3450.30|6762.42|5.40|0.00|270.48|275.88|-3179.82| +2451439|34111|3871|95582|293556|6730|17504|8|247|1417|36|13.71|15.21|2.88|0.00|103.68|493.56|547.56|2.07|0.00|103.68|105.75|-389.88| +2451439|34111|6412|95582|293556|6730|17504|8|242|1417|70|33.42|64.16|46.19|0.00|3233.30|2339.40|4491.20|290.99|0.00|3233.30|3524.29|893.90| +2451439|34111|5848|95582|293556|6730|17504|8|94|1417|11|98.61|116.35|105.87|0.00|1164.57|1084.71|1279.85|81.51|0.00|1164.57|1246.08|79.86| +2451439|34111|11468|95582|293556|6730|17504|8|232|1417|70|1.02|1.27|0.19|0.00|13.30|71.40|88.90|1.19|0.00|13.30|14.49|-58.10| +2451439|34111|1784|95582|293556|6730|17504|8|22|1417|99|1.07|1.57|0.61|0.00|60.39|105.93|155.43|5.43|0.00|60.39|65.82|-45.54| +2451439|34111|9818|95582|293556|6730|17504|8|188|1417|52|91.54|151.95|1.51|71.45|78.52|4760.08|7901.40|0.42|71.45|7.07|7.49|-4753.01| +2451439|34111|10982|95582|293556|6730|17504|8|116|1417|91|39.37|57.48|11.49|993.31|1045.59|3582.67|5230.68|1.56|993.31|52.28|53.84|-3530.39| +2451439|34111|4861|95582|293556|6730|17504|8|164|1417|56|95.01|149.16|123.80|0.00|6932.80|5320.56|8352.96|0.00|0.00|6932.80|6932.80|1612.24| +2451439|34111|8419|||6730||8|275|1417|||||3064.47|||13029.38||3064.47||636.25|-7877.46| +2452007|70907|7563|51636|1598876|6916|17403|4|157|1418|78|22.11|35.81|31.51|0.00|2457.78|1724.58|2793.18|98.31|0.00|2457.78|2556.09|733.20| +2452007|70907|6497|51636|1598876|6916|17403|4|260|1418|6|77.35|135.36|17.59|0.00|105.54|464.10|812.16|0.00|0.00|105.54|105.54|-358.56| +2452007|70907|3577|51636|1598876|6916|17403|4|37|1418|1|16.01|29.29|1.75|0.00|1.75|16.01|29.29|0.07|0.00|1.75|1.82|-14.26| +2452007|70907|801|51636|1598876|6916|17403|4|26|1418|49|13.87|23.85|5.48|233.61|268.52|679.63|1168.65|0.00|233.61|34.91|34.91|-644.72| +2452007|70907|11111|51636|1598876|6916|17403|4|194|1418|81|11.68|12.73|8.27|0.00|669.87|946.08|1031.13|6.69|0.00|669.87|676.56|-276.21| +2452007|70907|8099|51636|1598876|6916|17403|4|244|1418|4|54.23|73.21|65.88|168.65|263.52|216.92|292.84|1.89|168.65|94.87|96.76|-122.05| +2452007|70907|7015|51636|1598876|6916|17403|4|107|1418|24|29.62|45.02|44.11|0.00|1058.64|710.88|1080.48|74.10|0.00|1058.64|1132.74|347.76| +2452007|70907|10955|51636|1598876|6916|17403|4|253|1418|20|76.45|117.73|63.57|622.98|1271.40|1529.00|2354.60|19.45|622.98|648.42|667.87|-880.58| +2452007|70907|16139|51636|1598876|6916|17403|4|32|1418|65|47.33|78.56|20.42|0.00|1327.30|3076.45|5106.40|106.18|0.00|1327.30|1433.48|-1749.15| +2452264|59861|6657|3467|297555|4890|31378|4|165|1419|36|48.66|66.17|24.48|0.00|881.28|1751.76|2382.12|8.81|0.00|881.28|890.09|-870.48| +2452264|59861|721|3467|297555|4890|31378|4|260|1419|31|95.85|121.72|55.99|0.00|1735.69|2971.35|3773.32|0.00|0.00|1735.69|1735.69|-1235.66| +2452264|59861|16367|3467|297555|4890|31378|4|166|1419|85|20.53|27.51|23.65|0.00|2010.25|1745.05|2338.35|120.61|0.00|2010.25|2130.86|265.20| +2452264|59861|7583|3467|297555|4890|31378|4|19|1419|30|45.22|72.80|45.86|0.00|1375.80|1356.60|2184.00|13.75|0.00|1375.80|1389.55|19.20| +2452264|59861|3529|3467|297555|4890|31378|4|28|1419|53|62.48|94.34|67.92|0.00|3599.76|3311.44|5000.02|179.98|0.00|3599.76|3779.74|288.32| +2452264|59861|15571|3467|297555|4890|31378|4|67|1419|51|48.29|49.25|42.35|0.00|2159.85|2462.79|2511.75|194.38|0.00|2159.85|2354.23|-302.94| +2452264|59861|9033|3467|297555|4890|31378|4|143|1419|4|24.80|49.60|41.66|34.99|166.64|99.20|198.40|0.00|34.99|131.65|131.65|32.45| +2452264|59861|16761|3467|297555|4890|31378|4|14|1419|18|93.01|142.30|27.03|481.67|486.54|1674.18|2561.40|0.04|481.67|4.87|4.91|-1669.31| +2452264|59861|10147|3467|297555|4890|31378|4|255|1419|41|37.13|60.15|40.30|908.76|1652.30|1522.33|2466.15|52.04|908.76|743.54|795.58|-778.79| +2452264|59861|947|3467|297555|4890|31378|4|271|1419|60|20.14|21.54|13.78|0.00|826.80|1208.40|1292.40|66.14|0.00|826.80|892.94|-381.60| +2452628|36553|1623|69281|816460|1239|43218|8|43|1420|50|89.85|157.23|29.87|985.71|1493.50|4492.50|7861.50|45.70|985.71|507.79|553.49|-3984.71| +2452628|36553|17202|69281|816460|1239|43218|8|73|1420|37|90.32|105.67|77.13|0.00|2853.81|3341.84|3909.79|256.84|0.00|2853.81|3110.65|-488.03| +2452628|36553|17820|69281|816460|1239|43218|8|271|1420|13|95.71|121.55|66.85|165.11|869.05|1244.23|1580.15|28.15|165.11|703.94|732.09|-540.29| +2452628|36553|5145|69281||1239||||1420|||112.25|||195.30||673.50||||197.25|| +2452628||5874||816460|||8|65|1420||3.58||1.44||27.36|68.02|88.35|||27.36|29.27|| +||3955||816460|||8|259|1420|89||46.75||0.00|||||0.00|4160.75|4368.78|| +2452628|36553|14823|69281|816460|1239|43218|8|279|1420|96|32.96|44.16|24.72|0.00|2373.12|3164.16|4239.36|23.73|0.00|2373.12|2396.85|-791.04| +2452628|36553|7171|69281|816460|1239|43218|8|266|1420|66|21.00|24.57|4.66|0.00|307.56|1386.00|1621.62|12.30|0.00|307.56|319.86|-1078.44| +2452628||17106|||1239|43218|||1420|72|15.04|24.06|||1281.60|1082.88|1732.32|||1281.60||198.72| +2452628|36553|12129|69281|816460|1239|43218|8|263|1420|53|30.30|59.38|58.78|62.30|3115.34|1605.90|3147.14|152.65|62.30|3053.04|3205.69|1447.14| +2452628|36553|2037|69281|816460|1239|43218|8|10|1420|36|77.21|126.62|68.37|0.00|2461.32|2779.56|4558.32|73.83|0.00|2461.32|2535.15|-318.24| +2452628|36553|2526|69281|816460|1239|43218|8|179|1420|31|93.07|147.05|138.22|0.00|4284.82|2885.17|4558.55|171.39|0.00|4284.82|4456.21|1399.65| +2451349|46704|1369|30430|1544431|32|48880|4|210|1421|94|50.16|58.68|8.80|0.00|827.20|4715.04|5515.92|0.00|0.00|827.20|827.20|-3887.84| +2451349|46704|14095|30430|1544431|32|48880|4|73|1421|1|12.07|17.38|11.64|0.00|11.64|12.07|17.38|0.11|0.00|11.64|11.75|-0.43| +2451349|46704|2630|30430|1544431|32|48880|4|228|1421|100|15.69|25.26|16.16|0.00|1616.00|1569.00|2526.00|0.00|0.00|1616.00|1616.00|47.00| +2451349|46704|10459|30430|1544431|32|48880|4|167|1421|50|4.48|6.22|5.10|255.00|255.00|224.00|311.00|0.00|255.00|0.00|0.00|-224.00| +2451349|46704|1202|30430|1544431|32|48880|4|145|1421|65|72.58|87.82|8.78|0.00|570.70|4717.70|5708.30|5.70|0.00|570.70|576.40|-4147.00| +2451349|46704|775|30430|1544431|32|48880|4|150|1421|63|58.75|80.48|41.84|0.00|2635.92|3701.25|5070.24|105.43|0.00|2635.92|2741.35|-1065.33| +2451349|46704|16328|30430|1544431|32|48880|4|70|1421|33|75.95|131.39|40.73|0.00|1344.09|2506.35|4335.87|13.44|0.00|1344.09|1357.53|-1162.26| +2451349|46704|8408|30430|1544431|32|48880|4|211|1421|40|56.45|76.77|39.92|0.00|1596.80|2258.00|3070.80|0.00|0.00|1596.80|1596.80|-661.20| +2451349|46704|7406|30430|1544431|32|48880|4|79|1421|54|81.05|142.64|17.11|831.54|923.94|4376.70|7702.56|0.00|831.54|92.40|92.40|-4284.30| +2451349|46704|6109|30430|1544431|32|48880|4|173|1421|83|39.11|52.01|5.20|138.11|431.60|3246.13|4316.83|20.54|138.11|293.49|314.03|-2952.64| +2451349|46704|9517|30430|1544431|32|48880|4|131|1421|93|55.21|77.84|43.59|0.00|4053.87|5134.53|7239.12|243.23|0.00|4053.87|4297.10|-1080.66| +2451349|46704|3746|30430|1544431|32|48880|4|279|1421|77|48.55|81.56|13.04|572.32|1004.08|3738.35|6280.12|38.85|572.32|431.76|470.61|-3306.59| +2451349|46704|7003|30430|1544431|32|48880|4|283|1421|94|18.91|27.79|24.73|0.00|2324.62|1777.54|2612.26|23.24|0.00|2324.62|2347.86|547.08| +||9076|30430|1544431||48880|||1421|97|15.63|31.10|||1206.68|1516.11||||482.68|506.81|-1033.43| +2451234|61902|8936|71206||4229||1||1422||44.81|71.69|25.09||||6810.55|40.99||2049.86||-2207.09| +2451234|61902|12868|71206|1272741|4229|16564|1|159|1422|51|86.35|95.84|78.58|0.00|4007.58|4403.85|4887.84|80.15|0.00|4007.58|4087.73|-396.27| +2451234|61902|17852|71206|1272741|4229|16564|1|106|1422|40|8.66|13.76|5.77|180.02|230.80|346.40|550.40|4.06|180.02|50.78|54.84|-295.62| +2451234|61902|1975|71206|1272741|4229|16564|1|278|1422|57|28.97|56.20|26.97|0.00|1537.29|1651.29|3203.40|30.74|0.00|1537.29|1568.03|-114.00| +2451234|61902|4000|71206|1272741|4229|16564|1|213|1422|96|50.77|100.01|5.00|0.00|480.00|4873.92|9600.96|43.20|0.00|480.00|523.20|-4393.92| +2451234|61902|4363|71206|1272741|4229|16564|1|107|1422|24|98.88|145.35|142.44|0.00|3418.56|2373.12|3488.40|239.29|0.00|3418.56|3657.85|1045.44| +2451234|61902|3631|71206|1272741|4229|16564|1|198|1422|80|52.87|76.13|12.94|0.00|1035.20|4229.60|6090.40|82.81|0.00|1035.20|1118.01|-3194.40| +2451234|61902|4468|71206|1272741|4229|16564|1|246|1422|45|3.10|4.46|1.07|0.00|48.15|139.50|200.70|2.88|0.00|48.15|51.03|-91.35| +2451234|61902|3134||1272741|4229||1|38|1422||10.36||12.48||74.88|62.16|111.84|||59.91|64.10|| +2451234|61902|9148|71206|1272741|4229|16564|1|53|1422|55|70.73|120.94|25.39|0.00|1396.45|3890.15|6651.70|125.68|0.00|1396.45|1522.13|-2493.70| +2451234|61902|644|71206|1272741|4229|16564|1|5|1422|96|52.30|77.92|35.84|0.00|3440.64|5020.80|7480.32|34.40|0.00|3440.64|3475.04|-1580.16| +2451234|61902|10951|71206|1272741|4229|16564|1|105|1422|41|69.69|121.26|41.22|0.00|1690.02|2857.29|4971.66|152.10|0.00|1690.02|1842.12|-1167.27| +2451234|61902|11842|71206|1272741|4229|16564|1|159|1422|42|98.97|171.21|18.83|0.00|790.86|4156.74|7190.82|55.36|0.00|790.86|846.22|-3365.88| +2451234|61902|16315|71206|1272741|4229|16564|1|179|1422|25|82.95|116.13|45.29|0.00|1132.25|2073.75|2903.25|90.58|0.00|1132.25|1222.83|-941.50| +2451234|61902|8737|71206|1272741|4229|16564|1|231|1422|72|8.00|8.56|0.51|0.00|36.72|576.00|616.32|0.73|0.00|36.72|37.45|-539.28| +2451234|61902|14065|71206|1272741|4229|16564|1|80|1422|41|3.62|6.00|2.70|0.00|110.70|148.42|246.00|6.64|0.00|110.70|117.34|-37.72| +2451128|31861|6736|36651|101530|6056|18182|4|278|1423|89|3.68|6.84|0.68|0.00|60.52|327.52|608.76|5.44|0.00|60.52|65.96|-267.00| +2451128|31861|5359|36651|101530|6056|18182|4|129|1423|26|6.62|9.40|0.28|0.00|7.28|172.12|244.40|0.07|0.00|7.28|7.35|-164.84| +2451128|31861|2203|36651|101530|6056|18182|4|107|1423|65|60.91|82.83|19.87|0.00|1291.55|3959.15|5383.95|0.00|0.00|1291.55|1291.55|-2667.60| +||2168|||6056||4|208|1423||11.29|16.70||0.00||756.43|1118.90|0.00|0.00|156.11|156.11|-600.32| +2451128|31861|17030|36651|101530|6056|18182|4|3|1423|60|77.91|81.02|44.56|2566.65|2673.60|4674.60|4861.20|2.13|2566.65|106.95|109.08|-4567.65| +2451128|31861|10304|36651|101530|6056|18182|4|133|1423|10|84.11|104.29|76.13|570.97|761.30|841.10|1042.90|7.61|570.97|190.33|197.94|-650.77| +2451128|31861|433|36651|101530|6056|18182|4|140|1423|2|45.70|90.02|40.50|0.00|81.00|91.40|180.04|0.00|0.00|81.00|81.00|-10.40| +2451128|31861|9728|36651|101530|6056|18182|4|193|1423|7|57.86|65.38|56.88|0.00|398.16|405.02|457.66|0.00|0.00|398.16|398.16|-6.86| +2451128|31861|2029|36651|101530|6056|18182|4|109|1423|46|19.33|29.38|5.87|0.00|270.02|889.18|1351.48|8.10|0.00|270.02|278.12|-619.16| +2451128|31861|16732|36651|101530|6056|18182|4|129|1423|43|32.13|43.69|1.31|0.00|56.33|1381.59|1878.67|3.94|0.00|56.33|60.27|-1325.26| +2451128|31861|439|36651|101530|6056|18182|4|111|1423|97|61.66|96.80|14.52|0.00|1408.44|5981.02|9389.60|70.42|0.00|1408.44|1478.86|-4572.58| +2451128|31861|4762|36651|101530|6056|18182|4|270|1423|72|52.64|82.11|17.24|0.00|1241.28|3790.08|5911.92|62.06|0.00|1241.28|1303.34|-2548.80| +2451128|31861|15049|36651|101530|6056|18182|4|294|1423|63|12.86|15.04|11.73|0.00|738.99|810.18|947.52|51.72|0.00|738.99|790.71|-71.19| +||964|36651||6056|||225|1423|15|||||352.80||1960.80||||384.55|-767.70| +2451502|45144|15679|35366||||1|160|1424||76.73|145.01|10.15||||7395.51||||553.88|| +2451502|45144|17336|35366|28540|2514|42441|1|243|1424|47|69.94|119.59|3.58|121.14|168.26|3287.18|5620.73|1.88|121.14|47.12|49.00|-3240.06| +2451502|45144|16675|35366|28540|2514|42441|1|244|1424|44|39.44|44.56|16.48|0.00|725.12|1735.36|1960.64|50.75|0.00|725.12|775.87|-1010.24| +2451502|45144|16003|35366|28540|2514|42441|1|186|1424|65|32.18|56.31|21.96|528.13|1427.40|2091.70|3660.15|53.95|528.13|899.27|953.22|-1192.43| +2451502|45144|13640|35366|28540|2514|42441|1|65|1424|52|27.55|51.51|9.27|0.00|482.04|1432.60|2678.52|28.92|0.00|482.04|510.96|-950.56| +2451502|45144|2188|35366|28540|2514|42441|1|240|1424|75|6.62|8.34|2.00|6.00|150.00|496.50|625.50|4.32|6.00|144.00|148.32|-352.50| +2451502|45144|15064|35366|28540|2514|42441|1|255|1424|47|98.57|186.29|81.96|0.00|3852.12|4632.79|8755.63|115.56|0.00|3852.12|3967.68|-780.67| +2451502|45144|8588|35366|28540|2514|42441|1|250|1424|12|41.63|70.77|69.35|532.60|832.20|499.56|849.24|5.99|532.60|299.60|305.59|-199.96| +2451502|45144|16321|35366|28540|2514|42441|1|280|1424|35|70.00|103.60|63.19|995.24|2211.65|2450.00|3626.00|109.47|995.24|1216.41|1325.88|-1233.59| +2451502|45144|8488|35366|28540|2514|42441|1|197|1424|72|48.75|84.33|64.93|4581.46|4674.96|3510.00|6071.76|8.41|4581.46|93.50|101.91|-3416.50| +2451502|45144|2503|35366|28540|2514|42441|1|39|1424|68|63.69|75.79|53.05|0.00|3607.40|4330.92|5153.72|252.51|0.00|3607.40|3859.91|-723.52| +2451502|45144|7198|35366|28540|2514|42441|1|71|1424|36|72.36|120.11|19.21|0.00|691.56|2604.96|4323.96|20.74|0.00|691.56|712.30|-1913.40| +2451502|45144|9124|35366|28540|2514|42441|1|240|1424|80|53.80|101.68|34.57|0.00|2765.60|4304.00|8134.40|27.65|0.00|2765.60|2793.25|-1538.40| +2451502|45144|11212|35366|28540|2514|42441|1|192|1424|69|68.65|92.67|67.64|0.00|4667.16|4736.85|6394.23|373.37|0.00|4667.16|5040.53|-69.69| +2451502|45144|6968|35366|28540|2514|42441|1|27|1424|84|70.89|80.10|73.69|0.00|6189.96|5954.76|6728.40|557.09|0.00|6189.96|6747.05|235.20| +2451167|52933|13222|56510|1304597|2731|48818|7|190|1425|20|17.44|27.90|15.62|84.34|312.40|348.80|558.00|18.24|84.34|228.06|246.30|-120.74| +2451167|52933|16921|56510|1304597|2731|48818|7|232|1425|71|68.52|123.33|53.03|0.00|3765.13|4864.92|8756.43|188.25|0.00|3765.13|3953.38|-1099.79| +2451167|52933|17617|56510|1304597|2731|48818|7|220|1425|49|84.41|114.79|24.10|0.00|1180.90|4136.09|5624.71|11.80|0.00|1180.90|1192.70|-2955.19| +2451167|52933|3962|56510|1304597|2731|48818|7|58|1425|78|62.91|87.44|56.83|0.00|4432.74|4906.98|6820.32|398.94|0.00|4432.74|4831.68|-474.24| +2451167|52933|17584|56510|1304597|2731|48818|7|290|1425|29|93.15|178.84|33.97|246.28|985.13|2701.35|5186.36|7.38|246.28|738.85|746.23|-1962.50| +2451167|52933|3088|56510|1304597|2731|48818|7|52|1425|48|95.07|128.34|89.83|0.00|4311.84|4563.36|6160.32|129.35|0.00|4311.84|4441.19|-251.52| +2451167|52933|5887|56510|1304597|2731|48818|7|274|1425|16|81.49|139.34|25.08|0.00|401.28|1303.84|2229.44|20.06|0.00|401.28|421.34|-902.56| +2451167|52933|7874|56510|1304597|2731|48818|7|260|1425|65|70.03|126.05|35.29|0.00|2293.85|4551.95|8193.25|22.93|0.00|2293.85|2316.78|-2258.10| +2451167|52933|9472|56510|1304597|2731|48818|7|219|1425|8|97.73|160.27|121.80|0.00|974.40|781.84|1282.16|58.46|0.00|974.40|1032.86|192.56| +2451167|52933|4880|56510|1304597|2731|48818|7|286|1425|46|9.63|12.51|9.13|0.00|419.98|442.98|575.46|33.59|0.00|419.98|453.57|-23.00| +2451167|52933|6955|56510|1304597|2731|48818|7|13|1425|54|27.73|37.99|10.63|533.83|574.02|1497.42|2051.46|2.81|533.83|40.19|43.00|-1457.23| +2451167|52933|2150|56510|1304597|2731|48818|7|123|1425|41|40.86|66.19|27.79|0.00|1139.39|1675.26|2713.79|22.78|0.00|1139.39|1162.17|-535.87| +2451167|52933|14635|56510|1304597|2731|48818|7|216|1425|13|24.41|46.62|31.70|0.00|412.10|317.33|606.06|0.00|0.00|412.10|412.10|94.77| +2451167|52933|14488|56510|1304597|2731|48818|7|266|1425|11|19.99|36.38|24.01|0.00|264.11|219.89|400.18|2.64|0.00|264.11|266.75|44.22| +2451505|53412|15643|27093|377319|345|41324|2|8|1426|18|96.22|129.89|51.95|140.26|935.10|1731.96|2338.02|0.00|140.26|794.84|794.84|-937.12| +2451505|53412|12880|27093|377319|345|41324|2|294|1426|41|34.64|51.96|28.05|805.03|1150.05|1420.24|2130.36|13.80|805.03|345.02|358.82|-1075.22| +2451505|53412|14101|27093|377319|345|41324|2|162|1426|80|10.95|14.23|3.41|0.00|272.80|876.00|1138.40|21.82|0.00|272.80|294.62|-603.20| +2451505|53412|11854|27093|377319|345|41324|2|78|1426|66|12.36|18.78|18.59|0.00|1226.94|815.76|1239.48|49.07|0.00|1226.94|1276.01|411.18| +2451505|53412|5146|27093|377319|345|41324|2|240|1426|7|18.04|22.36|0.44|0.00|3.08|126.28|156.52|0.12|0.00|3.08|3.20|-123.20| +2451505|53412|8930|27093|377319|345|41324|2|68|1426|73|39.64|65.80|62.51|1688.39|4563.23|2893.72|4803.40|0.00|1688.39|2874.84|2874.84|-18.88| +2451505|53412|13370|27093|377319|345|41324|2|248|1426|75|63.36|116.58|15.15|465.86|1136.25|4752.00|8743.50|53.63|465.86|670.39|724.02|-4081.61| +||11047|27093|377319||41324|||1426|6|56.70|71.44||0.00||340.20|428.64|21.00|0.00|420.06|441.06|79.86| +2451505|53412|16045|27093|377319|345|41324|2|114|1426|7|92.42|141.40|80.59|0.00|564.13|646.94|989.80|50.77|0.00|564.13|614.90|-82.81| +2451505|53412|6838|27093|377319|345|41324|2|102|1426|51|8.87|17.29|15.90|0.00|810.90|452.37|881.79|40.54|0.00|810.90|851.44|358.53| +2451505|53412|17011|27093|377319|345|41324|2|62|1426|84|11.86|19.80|6.73|0.00|565.32|996.24|1663.20|0.00|0.00|565.32|565.32|-430.92| +2451505|53412|17563|27093||345||||1426||||0.00||0.00|2118.48|2774.59|||0.00|0.00|-2118.48| +2451146|52518|302|62116|1541908|3300|41092|7|201|1427|83|52.86|94.61|83.25|0.00|6909.75|4387.38|7852.63|345.48|0.00|6909.75|7255.23|2522.37| +2451146|52518|4454|62116|1541908|3300|41092|7|124|1427|7|39.34|51.14|20.45|0.00|143.15|275.38|357.98|12.88|0.00|143.15|156.03|-132.23| +2451146|52518|7045|62116|1541908|3300|41092|7|298|1427|5|79.87|89.45|9.83|0.00|49.15|399.35|447.25|1.96|0.00|49.15|51.11|-350.20| +2451146|52518|12272|62116|1541908|3300|41092|7|123|1427|37|87.74|100.02|3.00|0.00|111.00|3246.38|3700.74|2.22|0.00|111.00|113.22|-3135.38| +2451146|52518|3262|62116|1541908|3300|41092|7|59|1427|28|72.73|88.73|72.75|0.00|2037.00|2036.44|2484.44|81.48|0.00|2037.00|2118.48|0.56| +2451146||2192|62116||||7|212|1427|||||0.00||3058.55|4495.70||0.00|134.75||| +2451146|52518|10198|62116|1541908|3300|41092|7|12|1427|42|6.57|6.89|3.10|0.00|130.20|275.94|289.38|9.11|0.00|130.20|139.31|-145.74| +2451146|52518|9931|62116|1541908|3300|41092|7|50|1427|100|93.00|158.10|80.63|0.00|8063.00|9300.00|15810.00|725.67|0.00|8063.00|8788.67|-1237.00| +2451146|52518|14497|62116|1541908|3300|41092|7|261|1427|46|54.78|93.67|88.04|0.00|4049.84|2519.88|4308.82|202.49|0.00|4049.84|4252.33|1529.96| +2451146|52518|9634|62116|1541908|3300|41092|7|91|1427|93|86.88|112.07|10.08|0.00|937.44|8079.84|10422.51|9.37|0.00|937.44|946.81|-7142.40| +2451146|52518|8935|62116|1541908|3300|41092|7|142|1427|96|26.47|50.29|11.56|0.00|1109.76|2541.12|4827.84|22.19|0.00|1109.76|1131.95|-1431.36| +2451146|52518|5684|62116|1541908|3300|41092|7|1|1427|63|80.43|93.29|46.64|0.00|2938.32|5067.09|5877.27|235.06|0.00|2938.32|3173.38|-2128.77| +2451146|52518|17702|62116|1541908|3300|41092|7|72|1427|94|76.69|134.20|29.52|305.23|2774.88|7208.86|12614.80|74.08|305.23|2469.65|2543.73|-4739.21| +2451146|52518|5695|62116|1541908|3300|41092|7|51|1427|57|91.40|147.15|98.59|0.00|5619.63|5209.80|8387.55|56.19|0.00|5619.63|5675.82|409.83| +2451146|52518|9967|62116|1541908|3300|41092|7|273|1427|87|64.06|119.15|50.04|0.00|4353.48|5573.22|10366.05|0.00|0.00|4353.48|4353.48|-1219.74| +2451211|40863|10624|31444|1222913|1229|47098|8|239|1428|68|27.25|30.24|29.33|0.00|1994.44|1853.00|2056.32|19.94|0.00|1994.44|2014.38|141.44| +|40863|13996|||1229||8||1428|||7.55|1.05|0.00|||453.00||0.00||64.26|-194.40| +2451211|40863|262|31444|1222913|1229|47098|8|274|1428|5|96.87|134.64|47.12|0.00|235.60|484.35|673.20|0.00|0.00|235.60|235.60|-248.75| +2451211|40863|10099|31444|1222913|1229|47098|8|185|1428|39|91.86|99.20|68.44|0.00|2669.16|3582.54|3868.80|106.76|0.00|2669.16|2775.92|-913.38| +2451211|40863|7843|31444|1222913|1229|47098|8|292|1428|94|61.49|95.30|66.71|0.00|6270.74|5780.06|8958.20|125.41|0.00|6270.74|6396.15|490.68| +2451211|40863|16600|31444|1222913|1229|47098|8|65|1428|27|55.76|58.54|27.51|0.00|742.77|1505.52|1580.58|37.13|0.00|742.77|779.90|-762.75| +2451211|40863|14|31444|1222913|1229|47098|8|227|1428|19|99.12|175.44|50.87|0.00|966.53|1883.28|3333.36|57.99|0.00|966.53|1024.52|-916.75| +2451211|40863|13129|31444|1222913|1229|47098|8|230|1428|48|89.41|126.06|93.28|0.00|4477.44|4291.68|6050.88|223.87|0.00|4477.44|4701.31|185.76| +2451211|40863|14728|31444|1222913|1229|47098|8|216|1428|89|81.25|140.56|112.44|3702.64|10007.16|7231.25|12509.84|63.04|3702.64|6304.52|6367.56|-926.73| +2451211|40863|10228|31444|1222913|1229|47098|8|13|1428|14|78.29|117.43|100.98|0.00|1413.72|1096.06|1644.02|113.09|0.00|1413.72|1526.81|317.66| +2451211|40863|10291|31444|1222913|1229|47098|8|82|1428|98|86.40|152.92|35.17|0.00|3446.66|8467.20|14986.16|0.00|0.00|3446.66|3446.66|-5020.54| +2451211|40863|16441|31444|1222913|1229|47098|8|295|1428|20|73.90|109.37|76.55|0.00|1531.00|1478.00|2187.40|15.31|0.00|1531.00|1546.31|53.00| +2451211|40863|11408|31444|1222913|1229|47098|8|143|1428|84|62.43|117.99|100.29|0.00|8424.36|5244.12|9911.16|0.00|0.00|8424.36|8424.36|3180.24| +2451211|40863|15529|31444|1222913|1229|47098|8|73|1428|66|8.84|10.25|2.97|0.00|196.02|583.44|676.50|1.96|0.00|196.02|197.98|-387.42| +2451211|40863|6313|31444|1222913|1229|47098|8|121|1428|68|19.34|33.65|4.37|0.00|297.16|1315.12|2288.20|17.82|0.00|297.16|314.98|-1017.96| +|40863|4042|31444|1222913|1229||||1428|42|33.94|||0.00||1425.48|||0.00||216.72|-1208.76| +2451565|70452|1598|96297|232523|7149|18222|10|35|1429|28|95.81|182.99|179.33|0.00|5021.24|2682.68|5123.72|451.91|0.00|5021.24|5473.15|2338.56| +2451565|70452|8078|96297|232523|7149|18222|10|70|1429|68|41.87|68.24|42.30|0.00|2876.40|2847.16|4640.32|28.76|0.00|2876.40|2905.16|29.24| +2451565|70452|4957|96297|232523|7149|18222|10|43|1429|46|35.71|46.42|13.92|0.00|640.32|1642.66|2135.32|12.80|0.00|640.32|653.12|-1002.34| +2451565|70452|7604|96297|232523|7149|18222|10|152|1429|42|68.26|105.80|33.85|412.29|1421.70|2866.92|4443.60|40.37|412.29|1009.41|1049.78|-1857.51| +2451565|70452|15491|96297|232523|7149|18222|10|275|1429|41|6.31|11.79|9.07|0.00|371.87|258.71|483.39|26.03|0.00|371.87|397.90|113.16| +2451565|70452|3943|96297|232523|7149|18222|10|282|1429|40|69.16|127.25|2.54|0.00|101.60|2766.40|5090.00|9.14|0.00|101.60|110.74|-2664.80| +2451565|70452|5393|96297|232523|7149|18222|10|121|1429|76|51.48|96.78|70.64|0.00|5368.64|3912.48|7355.28|214.74|0.00|5368.64|5583.38|1456.16| +2451565|70452|15625|96297|232523|7149|18222|10|5|1429|78|21.91|40.09|4.40|0.00|343.20|1708.98|3127.02|27.45|0.00|343.20|370.65|-1365.78| +2451565|70452|7118|96297|232523|7149|18222|10|143|1429|32|28.49|46.72|43.91|1011.68|1405.12|911.68|1495.04|15.73|1011.68|393.44|409.17|-518.24| +2451565|70452|16088|96297|232523|7149|18222|10|81|1429|95|65.04|124.87|109.88|0.00|10438.60|6178.80|11862.65|730.70|0.00|10438.60|11169.30|4259.80| +2451565|70452|15869|96297|232523|7149|18222|10|202|1429|75|47.40|90.06|86.45|0.00|6483.75|3555.00|6754.50|129.67|0.00|6483.75|6613.42|2928.75| +2451565|70452|11024|96297|232523|7149|18222|10|14|1429|9|78.63|126.59|62.02|0.00|558.18|707.67|1139.31|0.00|0.00|558.18|558.18|-149.49| +2451565|70452|9575|96297|232523|7149|18222|10|60|1429|87|54.30|67.33|20.87|0.00|1815.69|4724.10|5857.71|90.78|0.00|1815.69|1906.47|-2908.41| +2451565|70452|3697|96297|232523|7149|18222|10|246|1429|74|68.74|136.79|75.23|0.00|5567.02|5086.76|10122.46|222.68|0.00|5567.02|5789.70|480.26| +2451565|70452|11102|96297|232523|7149|18222|10|81|1429|85|34.67|66.56|48.58|0.00|4129.30|2946.95|5657.60|289.05|0.00|4129.30|4418.35|1182.35| +2450886|44995|9004|78913|1637859|2405|17798|7|202|1430|86|43.36|50.73|31.95|0.00|2747.70|3728.96|4362.78|0.00|0.00|2747.70|2747.70|-981.26| +2450886|44995|2984|78913|1637859|2405|17798|7|17|1430|18|23.72|31.31|23.16|0.00|416.88|426.96|563.58|12.50|0.00|416.88|429.38|-10.08| +2450886|44995|11248|78913|1637859|2405|17798|7|237|1430|7|49.62|94.77|35.06|0.00|245.42|347.34|663.39|19.63|0.00|245.42|265.05|-101.92| +2450886|44995|9520|78913|1637859|2405|17798|7|44|1430|7|35.93|56.76|17.59|81.26|123.13|251.51|397.32|1.25|81.26|41.87|43.12|-209.64| +2450886|44995|6830|78913|1637859|2405|17798|7|294|1430|26|88.43|148.56|133.70|1251.43|3476.20|2299.18|3862.56|44.49|1251.43|2224.77|2269.26|-74.41| +2450886|44995|2138|78913|1637859|2405|17798|7|233|1430|67|95.62|108.05|2.16|57.88|144.72|6406.54|7239.35|5.21|57.88|86.84|92.05|-6319.70| +2450886||181||1637859|2405|17798|7||1430|5|48.73||29.15|0.00|||331.35||0.00|||| +2450886|44995|13045|78913|1637859|2405|17798|7|37|1430|82|39.98|53.17|46.25|0.00|3792.50|3278.36|4359.94|341.32|0.00|3792.50|4133.82|514.14| +2450886|44995|10154|78913|1637859|2405|17798|7|188|1430|91|20.74|35.88|21.16|0.00|1925.56|1887.34|3265.08|19.25|0.00|1925.56|1944.81|38.22| +2450886|44995|15218|78913|1637859|2405|17798|7|171|1430|60|56.24|107.41|100.96|5451.84|6057.60|3374.40|6444.60|24.23|5451.84|605.76|629.99|-2768.64| +2450886|44995|7393|78913|1637859|2405|17798|7|119|1430|21|54.86|102.03|89.78|0.00|1885.38|1152.06|2142.63|75.41|0.00|1885.38|1960.79|733.32| +2450886|44995|6445|78913|1637859|2405|17798|7|269|1430|90|30.42|51.10|28.61|0.00|2574.90|2737.80|4599.00|0.00|0.00|2574.90|2574.90|-162.90| +2450886|44995|15230|78913|1637859|2405|17798|7|12|1430|22|40.32|44.35|21.28|135.76|468.16|887.04|975.70|6.64|135.76|332.40|339.04|-554.64| +2451307|36549|7285|94544|1512848|1999|29503|10|43|1431|32|8.95|9.39|2.34|67.39|74.88|286.40|300.48|0.22|67.39|7.49|7.71|-278.91| +2451307|36549|196|94544|1512848|1999|29503|10|293|1431|55|91.06|112.91|33.87|0.00|1862.85|5008.30|6210.05|0.00|0.00|1862.85|1862.85|-3145.45| +2451307|36549|9544|94544|1512848|1999|29503|10|251|1431|24|4.13|4.29|2.10|0.00|50.40|99.12|102.96|4.03|0.00|50.40|54.43|-48.72| +2451307|36549|12625|94544|1512848|1999|29503|10|297|1431|66|52.85|56.02|43.69|0.00|2883.54|3488.10|3697.32|259.51|0.00|2883.54|3143.05|-604.56| +2451307|36549|3091|94544|1512848|1999|29503|10|246|1431|43|95.19|126.60|20.25|0.00|870.75|4093.17|5443.80|78.36|0.00|870.75|949.11|-3222.42| +2451307|36549|422|94544|1512848|1999|29503|10|69|1431|43|57.05|78.15|68.77|0.00|2957.11|2453.15|3360.45|29.57|0.00|2957.11|2986.68|503.96| +2451307|36549|5029|94544|1512848|1999|29503|10|157|1431|9|55.29|90.67|13.60|0.00|122.40|497.61|816.03|6.12|0.00|122.40|128.52|-375.21| +2451307|36549|17704|94544|1512848|1999|29503|10|200|1431|43|76.30|107.58|7.53|229.89|323.79|3280.90|4625.94|1.87|229.89|93.90|95.77|-3187.00| +2451307|36549|11234|94544|1512848|1999|29503|10|181|1431|39|45.06|81.55|43.22|0.00|1685.58|1757.34|3180.45|101.13|0.00|1685.58|1786.71|-71.76| +2451104|53073|6598|68241|1478913|3185|43904|2|250|1432|67|74.10|148.20|71.13|0.00|4765.71|4964.70|9929.40|0.00|0.00|4765.71|4765.71|-198.99| +2451104|53073|11420|68241|1478913|3185|43904|2|185|1432|1|13.14|15.76|14.65|0.00|14.65|13.14|15.76|0.00|0.00|14.65|14.65|1.51| +2451104|53073|16718|68241|1478913|3185|43904|2|145|1432|30|48.81|79.07|70.37|2068.87|2111.10|1464.30|2372.10|0.42|2068.87|42.23|42.65|-1422.07| +2451104|53073|10057|68241|1478913|3185|43904|2|4|1432|35|73.67|146.60|92.35|0.00|3232.25|2578.45|5131.00|32.32|0.00|3232.25|3264.57|653.80| +2451104|53073|17882|68241|1478913|3185|43904|2|68|1432|66|4.58|7.64|3.89|20.53|256.74|302.28|504.24|21.25|20.53|236.21|257.46|-66.07| +2451104|53073|8467|68241|1478913|3185|43904|2|101|1432|42|4.76|5.80|1.79|0.00|75.18|199.92|243.60|2.25|0.00|75.18|77.43|-124.74| +2451104|53073|6568|68241|1478913|3185|43904|2|254|1432|9|95.46|170.87|153.78|0.00|1384.02|859.14|1537.83|41.52|0.00|1384.02|1425.54|524.88| +2451104|53073|12386|68241|1478913|3185|43904|2|107|1432|42|7.96|15.44|9.26|0.00|388.92|334.32|648.48|23.33|0.00|388.92|412.25|54.60| +2451104|53073|5038|68241|1478913|3185|43904|2|37|1432|70|94.99|100.68|46.31|0.00|3241.70|6649.30|7047.60|97.25|0.00|3241.70|3338.95|-3407.60| +2451104|53073|1834|68241|1478913|3185|43904|2|232|1432|35|22.51|26.11|2.87|31.13|100.45|787.85|913.85|3.46|31.13|69.32|72.78|-718.53| +2451104|53073|14611|68241|1478913|3185|43904|2|218|1432|92|6.68|8.75|1.75|0.00|161.00|614.56|805.00|1.61|0.00|161.00|162.61|-453.56| +2451656|38592|4988|92354|703570|3139|23155|10|165|1433|10|55.51|96.03|13.44|47.04|134.40|555.10|960.30|6.11|47.04|87.36|93.47|-467.74| +2451656|38592|797|92354|703570|3139|23155|10|132|1433|59|86.98|135.68|28.49|0.00|1680.91|5131.82|8005.12|151.28|0.00|1680.91|1832.19|-3450.91| +2451656|38592|11579|92354|703570|3139|23155|10|121|1433|49|51.59|59.84|24.53|0.00|1201.97|2527.91|2932.16|24.03|0.00|1201.97|1226.00|-1325.94| +2451656|38592|14173|92354|703570|3139|23155|10|178|1433|78|49.37|80.47|57.13|0.00|4456.14|3850.86|6276.66|44.56|0.00|4456.14|4500.70|605.28| +2451656|38592|14090|92354|703570|3139|23155|10|137|1433|17|29.61|59.22|42.04|0.00|714.68|503.37|1006.74|28.58|0.00|714.68|743.26|211.31| +||2894|92354|703570||23155|||1433|||||0.00||146.68|247.76||0.00|222.68|222.68|76.00| +2451656|38592|1381|92354|703570|3139|23155|10|113|1433|79|9.41|15.43|2.62|0.00|206.98|743.39|1218.97|4.13|0.00|206.98|211.11|-536.41| +2451656|38592|9065|92354|703570|3139|23155|10|9|1433|100|99.59|174.28|38.34|0.00|3834.00|9959.00|17428.00|76.68|0.00|3834.00|3910.68|-6125.00| +||12241|||3139||||1433||79.00|||||553.00|1094.94|||||| +2451684|51909|11875|76464|1443808|752|11296|7|43|1434|94|72.26|90.32|74.06|0.00|6961.64|6792.44|8490.08|278.46|0.00|6961.64|7240.10|169.20| +2451684|51909|10214|76464|1443808|752|11296|7|279|1434|69|8.82|9.79|3.03|0.00|209.07|608.58|675.51|6.27|0.00|209.07|215.34|-399.51| +2451684|51909|12446|76464|1443808|752|11296|7|3|1434|68|12.86|16.20|15.71|0.00|1068.28|874.48|1101.60|32.04|0.00|1068.28|1100.32|193.80| +2451684|51909|245|76464|1443808|752|11296|7|23|1434|32|74.95|83.94|37.77|374.67|1208.64|2398.40|2686.08|50.03|374.67|833.97|884.00|-1564.43| +2451684|51909|1531|76464|1443808|752|11296|7|184|1434|85|8.83|14.48|7.52|0.00|639.20|750.55|1230.80|0.00|0.00|639.20|639.20|-111.35| +2451684|51909|3524|76464|1443808|752|11296|7|243|1434|55|71.60|78.76|0.00|0.00|0.00|3938.00|4331.80|0.00|0.00|0.00|0.00|-3938.00| +2451684|51909|13021|76464|1443808|752|11296|7|30|1434|36|30.29|42.70|42.27|0.00|1521.72|1090.44|1537.20|30.43|0.00|1521.72|1552.15|431.28| +2451684|51909|17263|76464|1443808|752|11296|7|32|1434|43|90.93|140.03|123.22|0.00|5298.46|3909.99|6021.29|158.95|0.00|5298.46|5457.41|1388.47| +2451684|51909|17971|76464|1443808|752|11296|7|84|1434|17|15.72|25.93|23.59|0.00|401.03|267.24|440.81|32.08|0.00|401.03|433.11|133.79| +2451684|51909|14443|76464|1443808|752|11296|7|85|1434|53|42.59|78.36|13.32|0.00|705.96|2257.27|4153.08|35.29|0.00|705.96|741.25|-1551.31| +2451684|51909|14099|76464|1443808|752|11296|7|160|1434|54|91.50|107.97|0.00|0.00|0.00|4941.00|5830.38|0.00|0.00|0.00|0.00|-4941.00| +2452193|32117|9037|9847|302678|2356|14387|2|138|1435|36|14.38|20.56|20.35|0.00|732.60|517.68|740.16|0.00|0.00|732.60|732.60|214.92| +2452193|32117|17347|9847|302678|2356|14387|2|95|1435|86|1.25|1.27|0.59|0.00|50.74|107.50|109.22|4.56|0.00|50.74|55.30|-56.76| +2452193|32117|11791|9847|302678|2356|14387|2|294|1435|70|46.16|58.62|26.96|1868.32|1887.20|3231.20|4103.40|0.18|1868.32|18.88|19.06|-3212.32| +2452193|32117|8159|9847|302678|2356|14387|2|267|1435|69|48.58|69.95|3.49|0.00|240.81|3352.02|4826.55|0.00|0.00|240.81|240.81|-3111.21| +2452193|32117|3883|9847|302678|2356|14387|2|245|1435|40|92.25|149.44|131.50|0.00|5260.00|3690.00|5977.60|157.80|0.00|5260.00|5417.80|1570.00| +2452193||1449|9847||2356|14387|2||1435|||||961.30||1349.04|1794.21|1.84|961.30|61.37|63.21|-1287.67| +2452193|32117|4061|9847|302678|2356|14387|2|3|1435|45|98.53|194.10|176.63|0.00|7948.35|4433.85|8734.50|715.35|0.00|7948.35|8663.70|3514.50| +2452193|32117|11687|9847|302678|2356|14387|2|261|1435|43|51.93|76.85|50.72|0.00|2180.96|2232.99|3304.55|196.28|0.00|2180.96|2377.24|-52.03| +2452193|32117|17499|9847|302678|2356|14387|2|173|1435|99|87.50|133.00|86.45|0.00|8558.55|8662.50|13167.00|0.00|0.00|8558.55|8558.55|-103.95| +2452193|32117|871|9847|302678|2356|14387|2|291|1435|93|2.11|3.37|2.22|0.00|206.46|196.23|313.41|8.25|0.00|206.46|214.71|10.23| +||10901||302678|2356||2|129|1435||||34.40|0.00|1823.20|3871.65|6078.04||0.00|||| +2452272|43243|8791|72155|8972|3742|39359|2|108|1436|62|95.02|166.28|43.23|0.00|2680.26|5891.24|10309.36|187.61|0.00|2680.26|2867.87|-3210.98| +2452272|43243|4985|72155|8972|3742|39359|2|45|1436|94|44.23|66.34|14.59|0.00|1371.46|4157.62|6235.96|82.28|0.00|1371.46|1453.74|-2786.16| +2452272|43243|2781|72155|8972|3742|39359|2|275|1436|54|10.26|15.90|6.83|0.00|368.82|554.04|858.60|0.00|0.00|368.82|368.82|-185.22| +2452272|43243|3359|72155|8972|3742|39359|2|103|1436|74|94.21|178.05|97.92|4202.72|7246.08|6971.54|13175.70|0.00|4202.72|3043.36|3043.36|-3928.18| +2452272|43243|2381|72155|8972|3742|39359|2|295|1436|13|7.92|13.78|12.53|0.00|162.89|102.96|179.14|14.66|0.00|162.89|177.55|59.93| +2452272|43243|4263|72155|8972|3742|39359|2|154|1436|69|77.36|136.15|122.53|0.00|8454.57|5337.84|9394.35|169.09|0.00|8454.57|8623.66|3116.73| +2452272|43243|6195|72155|8972|3742|39359|2|151|1436|50|89.13|104.28|25.02|0.00|1251.00|4456.50|5214.00|37.53|0.00|1251.00|1288.53|-3205.50| +2452272|43243|6309|72155|8972|3742|39359|2|167|1436|40|37.99|37.99|22.41|708.15|896.40|1519.60|1519.60|7.53|708.15|188.25|195.78|-1331.35| +2452272|43243|6563|72155|8972|3742|39359|2|103|1436|89|44.40|85.24|4.26|326.06|379.14|3951.60|7586.36|3.18|326.06|53.08|56.26|-3898.52| +2452272|43243|12727|72155|8972|3742|39359|2|64|1436|3|70.22|84.26|48.87|0.00|146.61|210.66|252.78|5.86|0.00|146.61|152.47|-64.05| +2452272|43243|4785|72155|8972|3742|39359|2|41|1436|83|66.46|123.61|102.59|0.00|8514.97|5516.18|10259.63|170.29|0.00|8514.97|8685.26|2998.79| +2452272|43243|13349|72155|8972|3742|39359|2|217|1436|69|2.16|2.46|1.27|0.00|87.63|149.04|169.74|0.00|0.00|87.63|87.63|-61.41| +2452272|43243|15117|72155|8972|3742|39359|2|250|1436|35|26.20|42.70|2.98|0.00|104.30|917.00|1494.50|9.38|0.00|104.30|113.68|-812.70| +2451132|67189|13897|44505|452339|4620|22101|4|118|1437|49|38.15|63.32|29.12|0.00|1426.88|1869.35|3102.68|114.15|0.00|1426.88|1541.03|-442.47| +2451132|67189|14905|44505|452339|4620|22101|4|290|1437|41|92.95|112.46|105.71|0.00|4334.11|3810.95|4610.86|260.04|0.00|4334.11|4594.15|523.16| +2451132|67189|4226|44505|452339|4620|22101|4|241|1437|98|51.28|58.45|35.65|0.00|3493.70|5025.44|5728.10|34.93|0.00|3493.70|3528.63|-1531.74| +||9625|44505|452339|4620|22101|4|4|1437||19.53|||||1581.93|2847.15|0.00|||2647.08|| +2451132|67189|17024|44505|452339|4620|22101|4|7|1437|1|74.91|86.89|39.96|0.00|39.96|74.91|86.89|1.59|0.00|39.96|41.55|-34.95| +2451132|67189|2287|44505|452339|4620|22101|4|185|1437|7|49.96|58.45|22.79|0.00|159.53|349.72|409.15|6.38|0.00|159.53|165.91|-190.19| +2451132|67189|9847|44505|452339|4620|22101|4|279|1437|93|40.84|65.34|49.65|0.00|4617.45|3798.12|6076.62|415.57|0.00|4617.45|5033.02|819.33| +2451132|67189|4279|44505|452339|4620|22101|4|33|1437|57|64.85|113.48|102.13|4307.84|5821.41|3696.45|6468.36|60.54|4307.84|1513.57|1574.11|-2182.88| +2451132|67189|17605|44505|452339|4620|22101|4|257|1437|2|100.00|147.00|52.92|0.00|105.84|200.00|294.00|3.17|0.00|105.84|109.01|-94.16| +2451132|67189|11620|44505|452339|4620|22101|4|98|1437|5|44.93|52.56|7.35|0.00|36.75|224.65|262.80|0.36|0.00|36.75|37.11|-187.90| +2451132|67189|7681|44505|452339|4620|22101|4|211|1437|42|71.67|129.00|38.70|991.49|1625.40|3010.14|5418.00|50.71|991.49|633.91|684.62|-2376.23| +2451132|67189|16285|44505|452339|4620|22101|4|57|1437|62|75.74|95.43|11.45|0.00|709.90|4695.88|5916.66|14.19|0.00|709.90|724.09|-3985.98| +2451132|67189|10813|44505|452339|4620|22101|4|3|1437|80|18.64|27.02|18.91|0.00|1512.80|1491.20|2161.60|15.12|0.00|1512.80|1527.92|21.60| +2451132|67189|3808|44505|452339|4620|22101|4|97|1437|45|80.33|93.98|26.31|0.00|1183.95|3614.85|4229.10|106.55|0.00|1183.95|1290.50|-2430.90| +2452562|53722|9559|83601|601291|5734|29424|8|42|1438|10|62.31|84.11|8.41|0.00|84.10|623.10|841.10|6.72|0.00|84.10|90.82|-539.00| +2452562|53722|16789|83601|601291|5734|29424|8|47|1438|12|80.39|151.93|144.33|0.00|1731.96|964.68|1823.16|34.63|0.00|1731.96|1766.59|767.28| +2452562|53722|13113|83601|601291|5734|29424|8|138|1438|33|24.60|47.72|26.24|0.00|865.92|811.80|1574.76|60.61|0.00|865.92|926.53|54.12| +2452562|53722|133|83601|601291|5734|29424|8|95|1438|1|5.73|6.93|1.10|0.82|1.10|5.73|6.93|0.00|0.82|0.28|0.28|-5.45| +2452562|53722|16393|83601|601291|5734|29424|8|169|1438|13|91.88|169.05|6.76|0.00|87.88|1194.44|2197.65|0.87|0.00|87.88|88.75|-1106.56| +2452562|53722|9943|83601|601291|5734|29424|8|291|1438|28|68.93|130.27|28.65|0.00|802.20|1930.04|3647.56|64.17|0.00|802.20|866.37|-1127.84| +2452562|53722|553|83601|601291|5734|29424|8|126|1438|40|2.36|3.61|1.15|29.44|46.00|94.40|144.40|0.33|29.44|16.56|16.89|-77.84| +2452562|53722|5490|83601|601291|5734|29424|8|278|1438|37|70.35|88.64|6.20|4.58|229.40|2602.95|3279.68|6.74|4.58|224.82|231.56|-2378.13| +2452562|53722|15849|83601|601291|5734|29424|8|140|1438|95|8.72|16.48|9.72|0.00|923.40|828.40|1565.60|18.46|0.00|923.40|941.86|95.00| +2452562|53722|4459|83601|601291|5734|29424|8|196|1438|64|11.02|15.53|0.77|0.00|49.28|705.28|993.92|0.00|0.00|49.28|49.28|-656.00| +2452562|53722|13824|83601|601291|5734|29424|8|205|1438|49|82.80|164.77|1.64|0.00|80.36|4057.20|8073.73|0.00|0.00|80.36|80.36|-3976.84| +2452562|53722|9138|83601|601291|5734|29424|8|146|1438|52|5.34|8.86|1.32|0.00|68.64|277.68|460.72|6.17|0.00|68.64|74.81|-209.04| +2452562|53722|5851|83601|601291|5734|29424|8|191|1438|52|67.68|110.99|104.33|0.00|5425.16|3519.36|5771.48|108.50|0.00|5425.16|5533.66|1905.80| +2452562|53722|15969|83601|601291|5734|29424|8|22|1438|30|98.81|175.88|161.80|0.00|4854.00|2964.30|5276.40|0.00|0.00|4854.00|4854.00|1889.70| +2452562|53722|4615|83601|601291|5734|29424|8|213|1438|63|16.43|23.49|21.14|0.00|1331.82|1035.09|1479.87|66.59|0.00|1331.82|1398.41|296.73| +2452562|53722|15207|83601|601291|5734|29424|8|66|1438|31|69.31|83.17|20.79|0.00|644.49|2148.61|2578.27|58.00|0.00|644.49|702.49|-1504.12| +2451568|53219|4505|88239|259675|2758|4641|7|242|1439|84|60.69|80.71|67.79|3018.01|5694.36|5097.96|6779.64|133.81|3018.01|2676.35|2810.16|-2421.61| +2451568|53219|4241|88239|259675|2758|4641|7|199|1439|61|95.45|99.26|39.70|0.00|2421.70|5822.45|6054.86|169.51|0.00|2421.70|2591.21|-3400.75| +2451568|53219|17330|88239|259675|2758|4641|7|280|1439|70|83.34|111.67|63.65|0.00|4455.50|5833.80|7816.90|89.11|0.00|4455.50|4544.61|-1378.30| +2451568|53219|7391|88239|259675|2758|4641|7|167|1439|93|71.96|117.29|23.45|0.00|2180.85|6692.28|10907.97|0.00|0.00|2180.85|2180.85|-4511.43| +2451568|53219|8336|88239|259675|2758|4641|7|17|1439|86|99.19|126.96|119.34|1744.75|10263.24|8530.34|10918.56|340.73|1744.75|8518.49|8859.22|-11.85| +|53219|17564|||2758|4641|7|104|1439|55||5.51|||14.85|210.65|303.05|0.46||6.69|7.15|| +2451568|53219|13172|88239|259675|2758|4641|7|24|1439|18|26.72|28.85|16.15|0.00|290.70|480.96|519.30|0.00|0.00|290.70|290.70|-190.26| +2451568|53219|15301|88239|259675|2758|4641|7|5|1439|69|46.03|84.23|30.32|0.00|2092.08|3176.07|5811.87|167.36|0.00|2092.08|2259.44|-1083.99| +2452578|58090|17244|89957|1825390|1176|26162|2|76|1440|34|67.52|95.87|67.10|0.00|2281.40|2295.68|3259.58|22.81|0.00|2281.40|2304.21|-14.28| +2452578|58090|3084|89957|1825390|1176|26162|2|40|1440|8|23.92|44.01|17.16|134.53|137.28|191.36|352.08|0.19|134.53|2.75|2.94|-188.61| +2452578|58090|10255|89957|1825390|1176|26162|2|293|1440|17|3.43|4.69|1.26|14.35|21.42|58.31|79.73|0.56|14.35|7.07|7.63|-51.24| +2452578|58090|4045|89957|1825390|1176|26162|2|236|1440|1|68.57|100.79|28.22|0.00|28.22|68.57|100.79|1.12|0.00|28.22|29.34|-40.35| +2452578|58090|14652|89957|1825390|1176|26162|2|274|1440|83|8.45|14.78|5.76|0.00|478.08|701.35|1226.74|9.56|0.00|478.08|487.64|-223.27| +2452578|58090|13806|89957|1825390|1176|26162|2|156|1440|17|27.66|51.44|40.12|0.00|682.04|470.22|874.48|6.82|0.00|682.04|688.86|211.82| +2452578|58090|3888|89957|1825390|1176|26162|2|270|1440|20|72.28|93.24|43.82|0.00|876.40|1445.60|1864.80|43.82|0.00|876.40|920.22|-569.20| +2452578|58090|8889|89957|1825390|1176|26162|2|115|1440|53|56.42|57.54|10.35|0.00|548.55|2990.26|3049.62|10.97|0.00|548.55|559.52|-2441.71| +2451130|44960|12644|665|1211528|3638|6101|10|26|1441|99|53.05|102.91|42.19|0.00|4176.81|5251.95|10188.09|208.84|0.00|4176.81|4385.65|-1075.14| +2451130|44960|3181|665|1211528|3638|6101|10|205|1441|6|93.82|136.03|84.33|0.00|505.98|562.92|816.18|30.35|0.00|505.98|536.33|-56.94| +2451130|44960|3397|665|1211528|3638|6101|10|173|1441|16|52.35|86.90|44.31|219.77|708.96|837.60|1390.40|24.45|219.77|489.19|513.64|-348.41| +2451130|44960|9958|665|1211528|3638|6101|10|227|1441|22|37.18|42.38|25.00|0.00|550.00|817.96|932.36|22.00|0.00|550.00|572.00|-267.96| +2451130|44960|9506|665|1211528|3638|6101|10|113|1441|17|68.61|109.77|84.52|0.00|1436.84|1166.37|1866.09|57.47|0.00|1436.84|1494.31|270.47| +2451130|44960|2359|665|1211528|3638|6101|10|61|1441|44|43.14|69.02|58.66|0.00|2581.04|1898.16|3036.88|77.43|0.00|2581.04|2658.47|682.88| +2451130|44960|17647|665|1211528|3638|6101|10|25|1441|11|65.08|95.01|54.15|0.00|595.65|715.88|1045.11|47.65|0.00|595.65|643.30|-120.23| +2451130|44960|12464|665|1211528|3638|6101|10|217|1441|65|82.69|126.51|26.56|0.00|1726.40|5374.85|8223.15|138.11|0.00|1726.40|1864.51|-3648.45| +2451130|44960|8569|665|1211528|3638|6101|10|218|1441|74|76.48|109.36|31.71|0.00|2346.54|5659.52|8092.64|164.25|0.00|2346.54|2510.79|-3312.98| +|44960|16384|||||||1441|||||0.00|44.00||||0.00||47.08|-1001.00| +2451130|44960|11371|665||3638|6101||176|1441|15||90.79||240.19||||4.27|240.19|||-364.84| +2451130|44960|15151|665|1211528|3638|6101|10|2|1441|78|12.27|12.27|10.79|0.00|841.62|957.06|957.06|50.49|0.00|841.62|892.11|-115.44| +2451130|44960|9614|665|1211528|3638|6101|10|293|1441|100|9.77|17.58|11.42|0.00|1142.00|977.00|1758.00|57.10|0.00|1142.00|1199.10|165.00| +2451130|44960|16504|665|1211528|3638|6101|10|120|1441|84|67.20|128.35|44.92|3697.81|3773.28|5644.80|10781.40|4.52|3697.81|75.47|79.99|-5569.33| +2451279|70891|16729|6768|1913939|2357|4375|10|125|1442|83|36.21|41.64|23.31|0.00|1934.73|3005.43|3456.12|154.77|0.00|1934.73|2089.50|-1070.70| +2451279|70891|14707|6768|1913939|2357|4375|10|290|1442|25|74.03|134.73|17.51|0.00|437.75|1850.75|3368.25|21.88|0.00|437.75|459.63|-1413.00| +2451279|70891|3656|6768|1913939|2357|4375|10|33|1442|68|61.55|114.48|16.02|0.00|1089.36|4185.40|7784.64|10.89|0.00|1089.36|1100.25|-3096.04| +2451279|70891|7093|6768|1913939|2357|4375|10|99|1442|78|35.10|36.15|6.50|0.00|507.00|2737.80|2819.70|40.56|0.00|507.00|547.56|-2230.80| +2451279|70891|16378|6768|1913939|2357|4375|10|283|1442|28|36.41|72.09|25.23|0.00|706.44|1019.48|2018.52|7.06|0.00|706.44|713.50|-313.04| +2451279|70891|2791|6768|1913939|2357|4375|10|50|1442|91|72.25|125.71|32.68|0.00|2973.88|6574.75|11439.61|118.95|0.00|2973.88|3092.83|-3600.87| +2451279|70891|92|6768|1913939|2357|4375|10|149|1442|98|20.37|27.49|17.31|0.00|1696.38|1996.26|2694.02|50.89|0.00|1696.38|1747.27|-299.88| +2451279|70891|529|6768|1913939|2357|4375|10|53|1442|52|43.56|52.27|32.93|0.00|1712.36|2265.12|2718.04|51.37|0.00|1712.36|1763.73|-552.76| +2451279|70891|14674|6768|1913939|2357|4375|10|279|1442|22|49.25|57.62|21.31|0.00|468.82|1083.50|1267.64|0.00|0.00|468.82|468.82|-614.68| +2451279|70891|8656|6768|1913939|2357|4375|10|23|1442|57|9.70|14.93|10.45|113.17|595.65|552.90|851.01|19.29|113.17|482.48|501.77|-70.42| +2451279|70891|11911|6768|1913939|2357|4375|10|152|1442|9|97.91|103.78|83.02|0.00|747.18|881.19|934.02|44.83|0.00|747.18|792.01|-134.01| +2451279|70891|856|6768|1913939||4375||215|1442|72|||7.93|0.00||2129.04|3810.96||0.00||588.08|-1558.08| +2451279|70891|4592|6768|1913939|2357|4375|10|82|1442|53|61.55|118.17|37.81|0.00|2003.93|3262.15|6263.01|60.11|0.00|2003.93|2064.04|-1258.22| +2451279|70891|5320|6768|1913939|2357|4375|10|95|1442|91|70.79|135.91|0.00|0.00|0.00|6441.89|12367.81|0.00|0.00|0.00|0.00|-6441.89| +2450863|40793|6110|28636|505530|6424|24918|10|46|1443|94|14.30|21.02|12.40|0.00|1165.60|1344.20|1975.88|0.00|0.00|1165.60|1165.60|-178.60| +2450863|40793|6044|28636|505530|6424|24918|10|250|1443|37|59.20|98.86|88.97|0.00|3291.89|2190.40|3657.82|197.51|0.00|3291.89|3489.40|1101.49| +2450863|40793|11209|28636|505530|6424|24918|10|238|1443|52|6.12|6.60|4.29|0.00|223.08|318.24|343.20|4.46|0.00|223.08|227.54|-95.16| +2450863|40793|6500|28636|505530|6424|24918|10|162|1443|49|67.44|116.67|53.66|0.00|2629.34|3304.56|5716.83|0.00|0.00|2629.34|2629.34|-675.22| +2450863|40793|3304|28636|505530|6424|24918|10|209|1443|60|2.72|3.64|0.65|0.00|39.00|163.20|218.40|1.95|0.00|39.00|40.95|-124.20| +2450863|40793|7576|28636|505530|6424|24918|10|287|1443|32|4.95|9.25|8.78|0.00|280.96|158.40|296.00|22.47|0.00|280.96|303.43|122.56| +2450863|40793|14996|28636|505530|6424|24918|10|220|1443|62|74.87|83.85|33.54|0.00|2079.48|4641.94|5198.70|62.38|0.00|2079.48|2141.86|-2562.46| +2450863|40793|901|28636|505530|6424|24918|10|250|1443|38|37.79|51.01|2.04|63.56|77.52|1436.02|1938.38|0.00|63.56|13.96|13.96|-1422.06| +2450863|40793|12706|28636|505530|6424|24918|10|165|1443|52|25.10|46.93|25.34|0.00|1317.68|1305.20|2440.36|105.41|0.00|1317.68|1423.09|12.48| +2450863|40793|5965|28636|505530|6424|24918|10|251|1443|58|52.95|99.54|71.66|0.00|4156.28|3071.10|5773.32|0.00|0.00|4156.28|4156.28|1085.18| +2451025|32649|961|43518|369392|2113|45634|1|69|1444|87|42.62|46.88|46.88|0.00|4078.56|3707.94|4078.56|326.28|0.00|4078.56|4404.84|370.62| +2451025|32649|14392|43518|369392|2113|45634|1|100|1444|5|84.66|115.98|105.54|0.00|527.70|423.30|579.90|15.83|0.00|527.70|543.53|104.40| +2451025|32649|14161|43518|369392|2113|45634|1|242|1444|9|74.52|96.87|76.52|0.00|688.68|670.68|871.83|55.09|0.00|688.68|743.77|18.00| +2451025|32649|7048|43518|369392|2113|45634|1|250|1444|10|3.87|5.61|4.48|34.04|44.80|38.70|56.10|0.10|34.04|10.76|10.86|-27.94| +2451025|32649|12181|43518|369392|2113|45634|1|233|1444|52|45.18|60.08|30.04|0.00|1562.08|2349.36|3124.16|0.00|0.00|1562.08|1562.08|-787.28| +2451025|32649|5344|43518|369392|2113|45634|1|270|1444|95|51.45|65.34|64.68|0.00|6144.60|4887.75|6207.30|491.56|0.00|6144.60|6636.16|1256.85| +2451025|32649|9844|43518|369392|2113|45634|1|65|1444|19|87.33|167.67|80.48|550.48|1529.12|1659.27|3185.73|9.78|550.48|978.64|988.42|-680.63| +2451025|32649|470|43518|369392|2113|45634|1|148|1444|56|63.06|69.36|36.76|0.00|2058.56|3531.36|3884.16|164.68|0.00|2058.56|2223.24|-1472.80| +2451025|32649|17902|43518|369392|2113|45634|1|139|1444|88|2.87|4.39|0.08|3.30|7.04|252.56|386.32|0.18|3.30|3.74|3.92|-248.82| +2451025|32649|518|43518|369392|2113|45634|1|267|1444|77|85.95|104.85|103.80|0.00|7992.60|6618.15|8073.45|239.77|0.00|7992.60|8232.37|1374.45| +2452225|50927|6363|55108|758272|1436|43800|1|177|1445|73|96.55|133.23|5.32|81.55|388.36|7048.15|9725.79|21.47|81.55|306.81|328.28|-6741.34| +2452225|50927|3019|55108|758272|1436|43800|1|282|1445|34|22.23|38.68|18.56|0.00|631.04|755.82|1315.12|37.86|0.00|631.04|668.90|-124.78| +2452225|50927|16445|55108|758272|1436|43800|1|154|1445|65|30.79|44.33|37.68|367.38|2449.20|2001.35|2881.45|187.36|367.38|2081.82|2269.18|80.47| +2452225|50927|15653|55108|758272|1436|43800|1|188|1445|42|10.33|20.14|5.03|0.00|211.26|433.86|845.88|14.78|0.00|211.26|226.04|-222.60| +2452225|50927|3689|55108|758272|1436|43800|1|87|1445|55|43.75|58.62|44.55|710.57|2450.25|2406.25|3224.10|104.38|710.57|1739.68|1844.06|-666.57| +|50927|8535|55108|758272|1436|43800|1||1445||24.91|43.09|31.02||1551.00|1245.50||77.55|||1628.55|| +2452225|50927|7369|55108|758272|1436|43800|1|138|1445|59|78.85|111.17|65.59|0.00|3869.81|4652.15|6559.03|232.18|0.00|3869.81|4101.99|-782.34| +2452225|50927|3227|55108|758272|1436|43800|1|25|1445|82|29.53|47.54|6.65|0.00|545.30|2421.46|3898.28|10.90|0.00|545.30|556.20|-1876.16| +2452225|50927|13063|55108|758272|1436|43800|1|231|1445|49|14.30|21.87|7.43|0.00|364.07|700.70|1071.63|0.00|0.00|364.07|364.07|-336.63| +2452225|50927|12787|55108|758272|1436|43800|1|162|1445|16|3.83|4.86|1.36|18.49|21.76|61.28|77.76|0.13|18.49|3.27|3.40|-58.01| +2452225|50927|14193|55108|758272|1436|43800|1|23|1445|67|26.17|46.84|40.75|0.00|2730.25|1753.39|3138.28|191.11|0.00|2730.25|2921.36|976.86| +2451469|39907|7564|42673|219082|3138|5947|10|266|1446|80|97.91|128.26|43.60|0.00|3488.00|7832.80|10260.80|69.76|0.00|3488.00|3557.76|-4344.80| +2451469|39907|17926|42673|219082|3138|5947|10|110|1446|15|5.37|10.36|5.80|0.00|87.00|80.55|155.40|4.35|0.00|87.00|91.35|6.45| +2451469|39907|12350|42673|219082|3138|5947|10|228|1446|71|45.87|80.73|31.48|0.00|2235.08|3256.77|5731.83|156.45|0.00|2235.08|2391.53|-1021.69| +2451469|39907|1438|42673|219082|3138|5947|10|171|1446|91|87.96|159.20|120.99|0.00|11010.09|8004.36|14487.20|770.70|0.00|11010.09|11780.79|3005.73| +2451469|39907|4387|42673|219082|3138|5947|10|109|1446|45|65.65|105.04|93.48|0.00|4206.60|2954.25|4726.80|294.46|0.00|4206.60|4501.06|1252.35| +2451469|39907|4849|42673|219082|3138|5947|10|206|1446|60|63.97|85.08|58.70|3205.02|3522.00|3838.20|5104.80|9.50|3205.02|316.98|326.48|-3521.22| +2451469|39907|6925|42673|219082|3138|5947|10|110|1446|84|17.17|24.55|24.30|0.00|2041.20|1442.28|2062.20|122.47|0.00|2041.20|2163.67|598.92| +2451469|39907|2218|42673|219082|3138|5947|10|125|1446|17|80.14|92.16|6.45|0.00|109.65|1362.38|1566.72|6.57|0.00|109.65|116.22|-1252.73| +2451469|39907|1273|42673|219082|3138|5947|10|186|1446|19|87.65|120.95|27.81|354.02|528.39|1665.35|2298.05|13.94|354.02|174.37|188.31|-1490.98| +2451469|39907|17062|42673|219082|3138|5947|10|25|1446|94|94.30|120.70|95.35|0.00|8962.90|8864.20|11345.80|0.00|0.00|8962.90|8962.90|98.70| +2451469|39907|3199|42673|219082|3138|5947|10|248|1446|27|62.68|85.87|67.83|879.07|1831.41|1692.36|2318.49|38.09|879.07|952.34|990.43|-740.02| +2452237|47053|6433|74153|49002|204|25053|10|256|1447|24|97.49|166.70|18.33|0.00|439.92|2339.76|4000.80|4.39|0.00|439.92|444.31|-1899.84| +2452237|47053|5325|74153|49002|204|25053|10|69|1447|2|71.07|126.50|104.99|0.00|209.98|142.14|253.00|10.49|0.00|209.98|220.47|67.84| +2452237|47053|3331|74153|49002|204|25053|10|150|1447|98|67.89|125.59|77.86|0.00|7630.28|6653.22|12307.82|457.81|0.00|7630.28|8088.09|977.06| +2452237|47053|7123|74153|49002|204|25053|10|225|1447|93|15.46|20.87|11.47|0.00|1066.71|1437.78|1940.91|21.33|0.00|1066.71|1088.04|-371.07| +2452237|47053|5093|74153|49002|204|25053|10|175|1447|37|39.31|53.46|1.60|0.00|59.20|1454.47|1978.02|0.00|0.00|59.20|59.20|-1395.27| +2452237|47053|13941|74153|49002|204|25053|10|173|1447|5|22.26|32.05|20.83|0.00|104.15|111.30|160.25|5.20|0.00|104.15|109.35|-7.15| +2452237|47053|7205|74153|49002|204|25053|10|222|1447|14|80.59|120.88|44.72|0.00|626.08|1128.26|1692.32|6.26|0.00|626.08|632.34|-502.18| +2452237|47053|9257|74153|49002|204|25053|10|46|1447|23|69.49|111.87|49.22|0.00|1132.06|1598.27|2573.01|67.92|0.00|1132.06|1199.98|-466.21| +2452237|47053|13269|74153|49002|204|25053|10|272|1447|34|33.73|58.01|8.70|0.00|295.80|1146.82|1972.34|11.83|0.00|295.80|307.63|-851.02| +2452237|47053|549|74153|49002|204|25053|10|121|1447|78|23.23|39.25|34.93|0.00|2724.54|1811.94|3061.50|245.20|0.00|2724.54|2969.74|912.60| +2452237|47053|17115|74153|49002|204|25053|10|150|1447|60|45.02|88.68|30.15|560.79|1809.00|2701.20|5320.80|112.33|560.79|1248.21|1360.54|-1452.99| +2452237|47053|10325|74153|49002|204|25053|10|37|1447|71|85.43|87.99|6.15|0.00|436.65|6065.53|6247.29|4.36|0.00|436.65|441.01|-5628.88| +2452237|47053|16163|74153|49002|204|25053|10|207|1447|39|79.79|112.50|99.00|0.00|3861.00|3111.81|4387.50|77.22|0.00|3861.00|3938.22|749.19| +2452237|47053|12149|74153|49002|204|25053|10|221|1447|26|86.09|157.54|122.88|0.00|3194.88|2238.34|4096.04|31.94|0.00|3194.88|3226.82|956.54| +2452237|47053|11609|74153|49002|204|25053|10|213|1447|20|28.26|29.67|25.81|0.00|516.20|565.20|593.40|0.00|0.00|516.20|516.20|-49.00| +2450923|61389|3526|33081|257585|1736|17040|8|212|1448|7|8.49|10.61|3.07|1.71|21.49|59.43|74.27|0.00|1.71|19.78|19.78|-39.65| +2450923|61389|6992|33081|257585|1736|17040|8|210|1448|1|24.59|30.24|12.39|3.96|12.39|24.59|30.24|0.25|3.96|8.43|8.68|-16.16| +2450923|61389|7030|33081|257585|1736|17040|8|159|1448|73|46.88|91.41|85.92|0.00|6272.16|3422.24|6672.93|62.72|0.00|6272.16|6334.88|2849.92| +|61389|3478||257585||17040||195|1448|96|38.60||45.34||4352.64|||0.00||43.53|43.53|| +2450923|61389|15922|33081|257585|1736|17040|8|208|1448|72|92.08|166.66|131.66|0.00|9479.52|6629.76|11999.52|853.15|0.00|9479.52|10332.67|2849.76| +2450923|61389|11077|33081|257585|1736|17040|8|288|1448|92|74.49|138.55|52.64|0.00|4842.88|6853.08|12746.60|435.85|0.00|4842.88|5278.73|-2010.20| +2450923|61389|1678|33081|257585|1736|17040|8|158|1448|38|77.68|155.36|52.82|0.00|2007.16|2951.84|5903.68|140.50|0.00|2007.16|2147.66|-944.68| +2450923|61389|8038|33081|257585|1736|17040|8|276|1448|93|31.44|36.47|32.45|0.00|3017.85|2923.92|3391.71|0.00|0.00|3017.85|3017.85|93.93| +2450923|61389|13855|33081|257585|1736|17040|8|128|1448|10|75.89|137.36|109.88|0.00|1098.80|758.90|1373.60|21.97|0.00|1098.80|1120.77|339.90| +||13324|33081||||||1448|23|||41.66|0.00|958.18|1416.11|||0.00|958.18|977.34|-457.93| +2450923|61389|2758|33081|257585|1736|17040|8|43|1448|96|71.27|133.98|26.79|0.00|2571.84|6841.92|12862.08|154.31|0.00|2571.84|2726.15|-4270.08| +2450923|61389|6658|33081|257585|1736|17040|8|197|1448|45|95.41|143.11|115.91|0.00|5215.95|4293.45|6439.95|260.79|0.00|5215.95|5476.74|922.50| +|61389|9052|33081|257585|1736|17040|8||1448|89||181.60||0.00|||16162.40||0.00|7919.22|8315.18|-587.40| +2450923|61389|14786|33081|257585|1736|17040|8|233|1448|89|67.22|77.30|34.78|1764.38|3095.42|5982.58|6879.70|106.48|1764.38|1331.04|1437.52|-4651.54| +2450923|61389|5953|33081|257585|1736|17040|8|127|1448|76|12.16|19.82|2.97|155.74|225.72|924.16|1506.32|3.49|155.74|69.98|73.47|-854.18| +2451388|69782|38|36335|663419|6503|32787|1|87|1449|33|42.51|85.02|47.61|0.00|1571.13|1402.83|2805.66|0.00|0.00|1571.13|1571.13|168.30| +2451388|69782|16123|36335|663419|6503|32787|1|197|1449|23|52.95|82.60|36.34|0.00|835.82|1217.85|1899.80|0.00|0.00|835.82|835.82|-382.03| +2451388|69782|9428|36335|663419|6503|32787|1|163|1449|87|70.53|104.38|53.23|0.00|4631.01|6136.11|9081.06|92.62|0.00|4631.01|4723.63|-1505.10| +2451388|69782|4672|36335|663419|6503|32787|1|57|1449|11|61.44|117.35|29.33|0.00|322.63|675.84|1290.85|9.67|0.00|322.63|332.30|-353.21| +2451388|69782|9970|36335|663419|6503|32787|1|79|1449|18|47.60|91.86|16.53|0.00|297.54|856.80|1653.48|2.97|0.00|297.54|300.51|-559.26| +2451388|69782|12451|36335|663419|6503|32787|1|41|1449|46|32.80|58.71|11.74|16.20|540.04|1508.80|2700.66|5.23|16.20|523.84|529.07|-984.96| +2451388|69782|3196|36335|663419|6503|32787|1|86|1449|64|51.32|55.42|24.38|1217.04|1560.32|3284.48|3546.88|20.59|1217.04|343.28|363.87|-2941.20| +2451388|69782|9854|36335|663419|6503|32787|1|92|1449|20|98.22|100.18|34.06|0.00|681.20|1964.40|2003.60|54.49|0.00|681.20|735.69|-1283.20| +2451388|69782|5648|36335|663419|6503|32787|1|101|1449|58|81.41|94.43|28.32|0.00|1642.56|4721.78|5476.94|16.42|0.00|1642.56|1658.98|-3079.22| +2451388|69782|241|36335|663419|6503|32787|1|113|1449|50|78.10|95.28|33.34|0.00|1667.00|3905.00|4764.00|100.02|0.00|1667.00|1767.02|-2238.00| +2451388|69782|9838|36335|663419|6503|32787|1|119|1449|30|63.06|92.69|0.00|0.00|0.00|1891.80|2780.70|0.00|0.00|0.00|0.00|-1891.80| +2452546|50261|5527|1031|1845209|2987|49965|1|33|1450|46|44.93|69.64|25.07|0.00|1153.22|2066.78|3203.44|80.72|0.00|1153.22|1233.94|-913.56| +2452546|50261|1554|1031|1845209|2987|49965|1|265|1450|91|75.06|142.61|85.56|0.00|7785.96|6830.46|12977.51|545.01|0.00|7785.96|8330.97|955.50| +2452546|50261|174|1031|1845209|2987|49965|1|218|1450|41|32.27|59.37|26.71|558.50|1095.11|1323.07|2434.17|10.73|558.50|536.61|547.34|-786.46| +2452546|50261|6169|1031|1845209|2987||||1450|13||27.04|17.84||231.92|311.09|351.52|||171.63|171.63|| +2452546|50261|13147|1031|1845209|2987|49965|1|110|1450|21|63.08|100.29|82.23|0.00|1726.83|1324.68|2106.09|17.26|0.00|1726.83|1744.09|402.15| +2452546|50261|7683|1031|1845209|2987|49965|1|202|1450|87|28.00|35.00|34.65|0.00|3014.55|2436.00|3045.00|211.01|0.00|3014.55|3225.56|578.55| +2452546|50261|13518|1031|1845209|2987|49965|1|197|1450|69|96.98|176.50|112.96|0.00|7794.24|6691.62|12178.50|0.00|0.00|7794.24|7794.24|1102.62| +||13839|||2987||1||1450||30.22|||568.96|||4080.70||568.96||737.12|-1510.66| +2452546|50261|13753|1031|1845209|2987|49965|1|164|1450|32|13.73|15.10|0.15|0.00|4.80|439.36|483.20|0.38|0.00|4.80|5.18|-434.56| +2452546|50261|8370|1031|1845209|2987|49965|1|157|1450|56|96.78|100.65|86.55|0.00|4846.80|5419.68|5636.40|0.00|0.00|4846.80|4846.80|-572.88| +2451987|53256|14411|25920|729990|3307|41377|2|114|1451|16|15.85|27.26|16.62|0.00|265.92|253.60|436.16|5.31|0.00|265.92|271.23|12.32| +2451987|53256|17689|25920|729990|3307|41377|2|112|1451|86|11.32|21.62|13.62|0.00|1171.32|973.52|1859.32|93.70|0.00|1171.32|1265.02|197.80| +2451987|53256|11003|25920|729990|3307|41377|2|47|1451|83|74.16|103.82|35.29|0.00|2929.07|6155.28|8617.06|205.03|0.00|2929.07|3134.10|-3226.21| +2451987|53256|4905|25920|729990|3307|41377|2|19|1451|52|29.31|50.99|40.28|0.00|2094.56|1524.12|2651.48|83.78|0.00|2094.56|2178.34|570.44| +2451987|53256|4541|25920|729990|3307|41377|2|184|1451|53|18.38|25.73|15.95|0.00|845.35|974.14|1363.69|16.90|0.00|845.35|862.25|-128.79| +|53256|9051|25920|729990||41377||210|1451||85.97|93.70|||5397.12|||485.74||5397.12|5882.86|| +2451987|53256|1561|25920|729990|3307|41377|2|265|1451|54|37.54|42.04|0.84|0.00|45.36|2027.16|2270.16|1.36|0.00|45.36|46.72|-1981.80| +2451987|53256|3041|25920|729990|3307|41377|2|160|1451|64|56.02|102.51|47.15|0.00|3017.60|3585.28|6560.64|60.35|0.00|3017.60|3077.95|-567.68| +2451987||14859|25920||3307|41377||11|1451|65|36.28|||0.00|123.50|||8.64|0.00|||-2234.70| +2451987|53256|2789|25920|729990|3307|41377|2|254|1451|68|36.70|47.71|6.67|0.00|453.56|2495.60|3244.28|27.21|0.00|453.56|480.77|-2042.04| +2451987|53256|17919|25920|729990|3307|41377|2|49|1451|71|93.21|173.37|156.03|0.00|11078.13|6617.91|12309.27|553.90|0.00|11078.13|11632.03|4460.22| +2451987|53256|3141|25920|729990|3307|41377|2|67|1451|99|31.15|31.77|17.15|0.00|1697.85|3083.85|3145.23|16.97|0.00|1697.85|1714.82|-1386.00| +2451987|53256|6957|25920|729990|3307|41377|2|183|1451|69|42.28|69.33|48.53|0.00|3348.57|2917.32|4783.77|33.48|0.00|3348.57|3382.05|431.25| +2451987|53256|3353|25920|729990|3307|41377|2|260|1451|50|39.80|58.90|19.43|19.43|971.50|1990.00|2945.00|66.64|19.43|952.07|1018.71|-1037.93| +2451987|53256|359|25920|729990|3307|41377|2|260|1451|34|82.16|160.21|1.60|0.00|54.40|2793.44|5447.14|0.00|0.00|54.40|54.40|-2739.04| +2452111|65112|8017|29909|1851125|5141|13957|8|156|1452|100|81.87|84.32|75.04|0.00|7504.00|8187.00|8432.00|225.12|0.00|7504.00|7729.12|-683.00| +2452111|65112|1011|29909|1851125|5141|13957|8|272|1452|82|10.11|20.01|19.20|0.00|1574.40|829.02|1640.82|141.69|0.00|1574.40|1716.09|745.38| +2452111|65112|7675|29909|1851125|5141|13957|8|44|1452|52|11.55|21.02|14.29|691.06|743.08|600.60|1093.04|1.04|691.06|52.02|53.06|-548.58| +2452111|65112|4479|29909|1851125|5141|13957|8|249|1452|7|73.05|143.17|124.55|0.00|871.85|511.35|1002.19|52.31|0.00|871.85|924.16|360.50| +2452111|65112|14533|29909|1851125|5141|13957|8|195|1452|17|95.64|185.54|122.45|0.00|2081.65|1625.88|3154.18|124.89|0.00|2081.65|2206.54|455.77| +2452111|65112|6401|29909|1851125|5141|13957|8|129|1452|24|29.18|38.51|10.39|0.00|249.36|700.32|924.24|17.45|0.00|249.36|266.81|-450.96| +2452111|65112|4131|29909|1851125|5141|13957|8|124|1452|10|42.75|70.53|64.88|0.00|648.80|427.50|705.30|6.48|0.00|648.80|655.28|221.30| +2452111|65112|17619|29909|1851125|5141|13957|8|118|1452|91|44.28|56.23|10.12|0.00|920.92|4029.48|5116.93|18.41|0.00|920.92|939.33|-3108.56| +2452111|65112|7903|29909|1851125|5141|13957|8|251|1452|66|10.92|19.32|13.52|0.00|892.32|720.72|1275.12|53.53|0.00|892.32|945.85|171.60| +2452111|65112|1089|29909|1851125|5141|13957|8|104|1452|41|72.38|136.07|68.03|0.00|2789.23|2967.58|5578.87|55.78|0.00|2789.23|2845.01|-178.35| +2452111|65112|7667|29909|1851125|5141|13957|8|136|1452|29|70.00|100.10|84.08|0.00|2438.32|2030.00|2902.90|0.00|0.00|2438.32|2438.32|408.32| +2452111|65112|219|29909|1851125|5141|13957|8|144|1452|56|37.44|64.77|60.88|0.00|3409.28|2096.64|3627.12|102.27|0.00|3409.28|3511.55|1312.64| +2451501|46896|8038|55686|274635|5584|4086|8|36|1453|24|13.77|14.59|8.89|0.00|213.36|330.48|350.16|2.13|0.00|213.36|215.49|-117.12| +2451501|46896|13855|55686|274635|5584|4086|8|7|1453|64|7.79|8.25|0.08|0.00|5.12|498.56|528.00|0.20|0.00|5.12|5.32|-493.44| +2451501|46896|13324|55686|274635|5584|4086|8|126|1453|29|8.64|15.55|8.86|0.00|256.94|250.56|450.95|0.00|0.00|256.94|256.94|6.38| +2451501|46896|2758|55686|274635|5584|4086|8|99|1453|100|64.23|84.14|16.82|1396.06|1682.00|6423.00|8414.00|14.29|1396.06|285.94|300.23|-6137.06| +2451501|46896|6658|55686|274635|5584|4086|8|285|1453|55|33.99|39.42|3.94|0.00|216.70|1869.45|2168.10|17.33|0.00|216.70|234.03|-1652.75| +2451501|46896|9052|55686|274635|5584|4086|8|57|1453|30|30.17|35.29|3.88|66.34|116.40|905.10|1058.70|2.00|66.34|50.06|52.06|-855.04| +2451501|46896|14786|55686|274635|5584|4086|8|282|1453|41|20.14|39.47|7.89|103.51|323.49|825.74|1618.27|0.00|103.51|219.98|219.98|-605.76| +2451501|46896|5953|55686|274635|5584|4086|8|74|1453|29|17.53|29.80|10.43|0.00|302.47|508.37|864.20|21.17|0.00|302.47|323.64|-205.90| +2451501|46896|9085|55686|274635|5584|4086|8|72|1453|20|3.60|5.86|5.80|0.00|116.00|72.00|117.20|9.28|0.00|116.00|125.28|44.00| +2451501|46896|4796|55686|274635|5584|4086|8|234|1453|49|96.50|193.00|46.32|1611.47|2269.68|4728.50|9457.00|39.49|1611.47|658.21|697.70|-4070.29| +2451501|46896|1249|55686|274635|5584|4086|8|223|1453|34|77.64|150.62|9.03|0.00|307.02|2639.76|5121.08|21.49|0.00|307.02|328.51|-2332.74| +2451087|71116|13214|54660|60903|1899|18142|7|217|1454|94|63.98|81.25|61.75|0.00|5804.50|6014.12|7637.50|348.27|0.00|5804.50|6152.77|-209.62| +2451087|71116|1346|54660|60903|1899|18142|7|250|1454|63|67.26|103.58|23.82|0.00|1500.66|4237.38|6525.54|60.02|0.00|1500.66|1560.68|-2736.72| +2451087|71116|15673|54660|60903|1899|18142|7|214|1454|24|85.97|118.63|61.68|1406.30|1480.32|2063.28|2847.12|2.96|1406.30|74.02|76.98|-1989.26| +2451087|71116|12097|54660|60903|1899|18142|7|63|1454|36|99.27|144.93|15.94|0.00|573.84|3573.72|5217.48|45.90|0.00|573.84|619.74|-2999.88| +2451087|71116|15658|54660|60903|1899|18142|7|200|1454|24|56.00|105.28|72.64|0.00|1743.36|1344.00|2526.72|0.00|0.00|1743.36|1743.36|399.36| +2451087|71116|16117|54660|60903|1899|18142|7|299|1454|51|2.66|4.73|1.89|0.00|96.39|135.66|241.23|8.67|0.00|96.39|105.06|-39.27| +2451087|71116|5584|54660|60903|1899|18142|7|80|1454|16|66.93|80.31|40.15|0.00|642.40|1070.88|1284.96|25.69|0.00|642.40|668.09|-428.48| +2451087|71116|3778|54660|60903|1899|18142|7|232|1454|50|9.14|9.50|2.66|99.75|133.00|457.00|475.00|0.99|99.75|33.25|34.24|-423.75| +2451087|71116|3835|54660|60903|1899|18142|7|10|1454|8|12.86|15.30|3.36|0.00|26.88|102.88|122.40|0.00|0.00|26.88|26.88|-76.00| +2451087|71116|925||60903|1899|18142|7|271|1454||38.73|42.60||0.00||||172.97|0.00|||62.88| +2451087|71116|3698|54660|60903|1899|18142|7|21|1454|55|80.22|124.34|78.33|0.00|4308.15|4412.10|6838.70|344.65|0.00|4308.15|4652.80|-103.95| +2451087|71116|16762|54660|60903|1899|18142|7|39|1454|20|10.47|16.12|11.60|176.32|232.00|209.40|322.40|0.00|176.32|55.68|55.68|-153.72| +2451087|71116|14198|54660|60903|1899|18142|7|11|1454|79|35.71|58.20|35.50|0.00|2804.50|2821.09|4597.80|140.22|0.00|2804.50|2944.72|-16.59| +2451087|71116|15452|54660|60903|1899|18142|7|128|1454|37|95.24|172.38|56.88|0.00|2104.56|3523.88|6378.06|0.00|0.00|2104.56|2104.56|-1419.32| +2451087|71116|14774|54660|60903|1899|18142|7|54|1454|37|72.10|137.71|19.27|0.00|712.99|2667.70|5095.27|49.90|0.00|712.99|762.89|-1954.71| +2451134|70095|14146|79119|1191967|4368|23169|2|50|1455|34|62.95|74.28|51.25|0.00|1742.50|2140.30|2525.52|104.55|0.00|1742.50|1847.05|-397.80| +2451134|70095|16594|79119|1191967|4368|23169|2|68|1455|44|76.70|138.06|84.21|0.00|3705.24|3374.80|6074.64|0.00|0.00|3705.24|3705.24|330.44| +2451134|70095|10916|79119|1191967|4368|23169|2|71|1455|54|26.43|33.56|6.71|0.00|362.34|1427.22|1812.24|21.74|0.00|362.34|384.08|-1064.88| +2451134|70095|17366|79119|1191967|4368|23169|2|2|1455|31|71.37|109.90|32.97|0.00|1022.07|2212.47|3406.90|61.32|0.00|1022.07|1083.39|-1190.40| +|70095|5245||1191967|||2|57|1455||||46.02|0.00||||129.77|0.00|2162.94|2292.71|579.04| +2451134|70095|12703|79119|1191967|4368|23169|2|100|1455|55|9.51|17.97|3.23|133.23|177.65|523.05|988.35|3.99|133.23|44.42|48.41|-478.63| +2451134|70095|208|79119|1191967|4368|23169|2|219|1455|60|74.94|128.89|56.71|0.00|3402.60|4496.40|7733.40|102.07|0.00|3402.60|3504.67|-1093.80| +2451134|70095|8317|79119|1191967|4368|23169|2|56|1455|58|15.15|15.30|6.73|0.00|390.34|878.70|887.40|27.32|0.00|390.34|417.66|-488.36| +2451134|70095|2498|79119|1191967|4368|23169|2|282|1455|61|18.70|30.66|29.74|0.00|1814.14|1140.70|1870.26|18.14|0.00|1814.14|1832.28|673.44| +2451313|49571|4063|83885|1231679|3026|820|4|98|1456|98|98.98|122.73|4.90|0.00|480.20|9700.04|12027.54|24.01|0.00|480.20|504.21|-9219.84| +2451313|49571|6328|83885|1231679|3026|820|4|158|1456|100|82.60|102.42|26.62|0.00|2662.00|8260.00|10242.00|79.86|0.00|2662.00|2741.86|-5598.00| +2451313|49571|14608|83885|1231679|3026|820|4|158|1456|77|40.07|53.69|43.48|167.39|3347.96|3085.39|4134.13|159.02|167.39|3180.57|3339.59|95.18| +2451313|49571|15352|83885|1231679|3026|820|4|265|1456|23|2.60|4.16|1.33|0.00|30.59|59.80|95.68|0.61|0.00|30.59|31.20|-29.21| +2451313|49571|3079|83885|1231679|3026|820|4|96|1456|45|60.86|111.37|12.25|0.00|551.25|2738.70|5011.65|0.00|0.00|551.25|551.25|-2187.45| +2451313|49571|14500|83885|1231679|3026|820|4|192|1456|67|41.71|59.22|26.05|0.00|1745.35|2794.57|3967.74|104.72|0.00|1745.35|1850.07|-1049.22| +|49571|5026|||||||1456|58||104.02|49.92|0.00||3549.02|||0.00|||-653.66| +2451313|49571|12626|83885|1231679|3026|820|4|137|1456|78|99.27|123.09|65.23|2442.21|5087.94|7743.06|9601.02|52.91|2442.21|2645.73|2698.64|-5097.33| +2451313|49571|10514|83885|1231679|3026|820|4|91|1456|65|95.96|163.13|14.68|47.71|954.20|6237.40|10603.45|0.00|47.71|906.49|906.49|-5330.91| +2451313|49571|16094|83885|1231679|3026|820|4|283|1456|91|65.23|87.40|55.93|0.00|5089.63|5935.93|7953.40|101.79|0.00|5089.63|5191.42|-846.30| +2451313|49571|1111|83885|1231679|3026|820|4|129|1456|90|9.21|16.94|3.72|0.00|334.80|828.90|1524.60|10.04|0.00|334.80|344.84|-494.10| +2451072|64897|6218|91966|20114|1062|8410|10|182|1457|34|69.25|85.17|73.24|0.00|2490.16|2354.50|2895.78|224.11|0.00|2490.16|2714.27|135.66| +2451072|64897|3241|91966|20114|1062|8410|10|226|1457|38|92.52|116.57|10.49|0.00|398.62|3515.76|4429.66|35.87|0.00|398.62|434.49|-3117.14| +2451072|64897|8|91966|20114|1062|8410|10|228|1457|65|32.89|39.46|3.55|0.00|230.75|2137.85|2564.90|4.61|0.00|230.75|235.36|-1907.10| +2451072|64897|2030||20114|||10|260|1457|89||61.52||||||39.37||492.17||| +2451072|64897|12212|91966|20114|1062|8410|10|12|1457|21|14.02|25.51|6.37|40.13|133.77|294.42|535.71|0.93|40.13|93.64|94.57|-200.78| +2451072|64897|7645|91966|20114|1062|8410|10|208|1457|92|43.41|58.60|28.71|0.00|2641.32|3993.72|5391.20|79.23|0.00|2641.32|2720.55|-1352.40| +2451072|64897|6428|91966|20114|1062|8410|10|179|1457|89|61.78|109.96|84.66|0.00|7534.74|5498.42|9786.44|226.04|0.00|7534.74|7760.78|2036.32| +2451072|64897|952|91966|20114|1062|8410|10|163|1457|61|63.94|74.80|21.69|0.00|1323.09|3900.34|4562.80|105.84|0.00|1323.09|1428.93|-2577.25| +2451072|64897|10897|91966|20114|1062|8410|10|156|1457|80|27.98|45.04|26.57|2040.57|2125.60|2238.40|3603.20|1.70|2040.57|85.03|86.73|-2153.37| +2451072|64897|8594||20114||8410|||1457|||92.83||||4691.10||||253.84|266.53|-4437.26| +2451392||12121||336897|||10||1458||41.82|54.36|||1402.64|3721.98||||||-2319.34| +2451392|70087|5602|66459|336897|6182|27820|10|207|1458|47|9.55|10.50|2.62|0.00|123.14|448.85|493.50|4.92|0.00|123.14|128.06|-325.71| +2451392|70087|14359|66459|336897|6182|27820|10|277|1458|78|81.58|93.00|47.43|0.00|3699.54|6363.24|7254.00|36.99|0.00|3699.54|3736.53|-2663.70| +2451392||1465|66459|336897|||10||1458|52|||12.02|0.00||||43.75|0.00|||| +2451392|70087|13096|66459|336897|6182|27820|10|120|1458|13|73.42|95.44|42.94|0.00|558.22|954.46|1240.72|27.91|0.00|558.22|586.13|-396.24| +2451392|70087|11242|66459|336897|6182|27820|10|7|1458|64|92.98|155.27|130.42|0.00|8346.88|5950.72|9937.28|333.87|0.00|8346.88|8680.75|2396.16| +2451392|70087|16874|66459|336897|6182|27820|10|298|1458|50|1.43|2.13|0.08|0.00|4.00|71.50|106.50|0.16|0.00|4.00|4.16|-67.50| +2451392|70087|4082|66459|336897|6182|27820|10|266|1458|49|48.87|83.07|53.16|0.00|2604.84|2394.63|4070.43|156.29|0.00|2604.84|2761.13|210.21| +2451392|70087|4706|66459|336897|6182|27820|10|44|1458|24|85.89|154.60|57.20|507.93|1372.80|2061.36|3710.40|0.00|507.93|864.87|864.87|-1196.49| +2451392|70087|6998|66459|336897|6182|27820|10|242|1458|77|68.32|76.51|50.49|0.00|3887.73|5260.64|5891.27|116.63|0.00|3887.73|4004.36|-1372.91| +2451392|70087|17851|66459|336897|6182|27820|10|219|1458|34|41.96|80.14|23.24|0.00|790.16|1426.64|2724.76|0.00|0.00|790.16|790.16|-636.48| +2451392|70087|9109|66459|336897|6182|27820|10|98|1458|53|20.51|26.66|23.72|0.00|1257.16|1087.03|1412.98|100.57|0.00|1257.16|1357.73|170.13| +2452134|69540|15865|93602|981895|3788|10434|1|35|1459|52|43.51|46.55|1.86|0.00|96.72|2262.52|2420.60|2.90|0.00|96.72|99.62|-2165.80| +2452134|69540|5339|93602|981895|3788|10434|1|104|1459|32|9.20|10.39|3.32|0.00|106.24|294.40|332.48|5.31|0.00|106.24|111.55|-188.16| +2452134|69540|12359|93602|981895|3788|10434|1|201|1459|88|57.24|89.86|66.49|0.00|5851.12|5037.12|7907.68|0.00|0.00|5851.12|5851.12|814.00| +||10267||981895|3788||1|242|1459|84|62.63|88.93|||4331.88||7470.12|259.91||4331.88|4591.79|-929.04| +||16115|||3788||||1459|78||159.10|||2233.14|7215.00||||||| +2452134|69540|14777|93602|981895|3788|10434|1|33|1459|77|40.52|55.51|16.65|0.00|1282.05|3120.04|4274.27|51.28|0.00|1282.05|1333.33|-1837.99| +2452134|69540|13883|93602|981895|3788|10434|1|85|1459|27|25.61|31.50|5.04|0.00|136.08|691.47|850.50|2.72|0.00|136.08|138.80|-555.39| +2452134|69540|11151|93602|981895|3788|10434|1|255|1459|33|16.89|24.49|19.83|0.00|654.39|557.37|808.17|32.71|0.00|654.39|687.10|97.02| +2452508|46359|6613|9861|112323|1985|20447|1|239|1460|73|92.24|171.56|44.60|0.00|3255.80|6733.52|12523.88|293.02|0.00|3255.80|3548.82|-3477.72| +2452508|46359|7209|9861|112323|1985|20447|1|223|1460|93|24.34|25.55|3.06|210.58|284.58|2263.62|2376.15|0.74|210.58|74.00|74.74|-2189.62| +2452508|46359|11940|9861|112323|1985|20447|1|208|1460|76|70.54|79.00|15.01|0.00|1140.76|5361.04|6004.00|91.26|0.00|1140.76|1232.02|-4220.28| +2452508|46359|4443|9861|112323|1985|20447|1|70|1460|75|21.67|41.60|4.99|37.42|374.25|1625.25|3120.00|26.94|37.42|336.83|363.77|-1288.42| +2452508|46359|10776|9861|112323|1985|20447|1|191|1460|18|8.04|13.90|4.30|0.00|77.40|144.72|250.20|0.77|0.00|77.40|78.17|-67.32| +2452508|46359|6720|9861|112323|1985|20447|1|206|1460|16|4.60|5.88|1.58|23.76|25.28|73.60|94.08|0.01|23.76|1.52|1.53|-72.08| +2452508|46359|15423|9861|112323|1985|20447|1|155|1460|92|63.89|81.14|37.32|0.00|3433.44|5877.88|7464.88|0.00|0.00|3433.44|3433.44|-2444.44| +2452508|46359|12249|9861|112323|1985|20447|1|236|1460|23|47.79|95.10|38.99|0.00|896.77|1099.17|2187.30|44.83|0.00|896.77|941.60|-202.40| +2452508|46359|9619|9861|112323|1985|20447|1|93|1460|20|36.42|48.07|4.32|77.76|86.40|728.40|961.40|0.43|77.76|8.64|9.07|-719.76| +2452508|46359|6054|9861|112323|1985|20447|1|168|1460|22|86.71|161.28|156.44|0.00|3441.68|1907.62|3548.16|275.33|0.00|3441.68|3717.01|1534.06| +2452508|46359|1935|9861|112323|1985|20447|1|140|1460|27|86.77|153.58|67.57|0.00|1824.39|2342.79|4146.66|18.24|0.00|1824.39|1842.63|-518.40| +2452508|46359|8095|9861|112323|1985|20447|1|159|1460|34|78.73|79.51|29.41|0.00|999.94|2676.82|2703.34|19.99|0.00|999.94|1019.93|-1676.88| +2452508|46359|16777|9861|112323|1985|20447|1|102|1460|13|17.08|32.79|8.19|0.00|106.47|222.04|426.27|8.51|0.00|106.47|114.98|-115.57| +2452508|46359|9403|9861|112323|1985|20447|1|30|1460|34|12.64|20.72|14.08|0.00|478.72|429.76|704.48|0.00|0.00|478.72|478.72|48.96| +2452508|46359|12715|9861|112323|1985|20447|1|278|1460|33|11.09|12.53|9.39|0.00|309.87|365.97|413.49|21.69|0.00|309.87|331.56|-56.10| +2452508|46359|1233|9861|112323|1985|20447|1|94|1460|4|26.79|45.54|38.70|0.00|154.80|107.16|182.16|7.74|0.00|154.80|162.54|47.64| +2451176|35945|15740|38369|464711|4206|16148|7|260|1461|19|97.58|101.48|73.06|0.00|1388.14|1854.02|1928.12|111.05|0.00|1388.14|1499.19|-465.88| +2451176|35945|6226|38369|464711|4206|16148|7|283|1461|17|97.13|169.97|88.38|1262.06|1502.46|1651.21|2889.49|19.23|1262.06|240.40|259.63|-1410.81| +2451176|35945|17182|38369|464711|4206|16148|7|194|1461|14|7.77|9.55|2.96|0.00|41.44|108.78|133.70|2.90|0.00|41.44|44.34|-67.34| +2451176|35945|746|38369|464711|4206|16148|7|127|1461|35|14.97|16.46|3.62|0.00|126.70|523.95|576.10|7.60|0.00|126.70|134.30|-397.25| +2451176|35945|17540|38369|464711|4206|16148|7|233|1461|53|41.08|62.03|4.96|0.00|262.88|2177.24|3287.59|0.00|0.00|262.88|262.88|-1914.36| +2451176|35945|1378|38369|464711|4206|16148|7|84|1461|10|10.23|11.25|4.16|0.00|41.60|102.30|112.50|3.74|0.00|41.60|45.34|-60.70| +2451176|35945|12631|38369|464711|4206|16148|7|97|1461|11|62.15|106.27|25.50|0.00|280.50|683.65|1168.97|16.83|0.00|280.50|297.33|-403.15| +2451176|35945|9061|38369|464711|4206|16148|7|96|1461|84|35.09|68.42|6.84|551.57|574.56|2947.56|5747.28|0.45|551.57|22.99|23.44|-2924.57| +2451176|35945|13354|38369|464711|4206|16148|7|54|1461|69|16.98|18.50|7.95|0.00|548.55|1171.62|1276.50|21.94|0.00|548.55|570.49|-623.07| +2451176|35945|5791|38369|464711|4206|16148|7|222|1461|84|45.91|80.34|65.07|0.00|5465.88|3856.44|6748.56|163.97|0.00|5465.88|5629.85|1609.44| +2451176|35945|622|38369|464711|4206|16148|7|88|1461|12|22.20|29.08|2.90|0.00|34.80|266.40|348.96|0.34|0.00|34.80|35.14|-231.60| +2451176||8347|38369||4206|16148|7||1461|||75.57|||2856.00||6347.88|||2856.00|2941.68|-2222.64| +2451176|35945|7210|38369|464711|4206|16148|7|139|1461|14|71.34|136.97|115.05|0.00|1610.70|998.76|1917.58|144.96|0.00|1610.70|1755.66|611.94| +2451176|35945|17407|38369|464711|4206|16148|7|288|1461|10|9.74|13.73|4.94|0.00|49.40|97.40|137.30|3.95|0.00|49.40|53.35|-48.00| +2451529|63056|7723|33092|1584503|1067|22831|7|110|1462|87|79.20|150.48|112.86|0.00|9818.82|6890.40|13091.76|0.00|0.00|9818.82|9818.82|2928.42| +2451529|63056|2108|33092|1584503|1067|22831|7|23|1462|78|11.95|14.45|4.76|0.00|371.28|932.10|1127.10|22.27|0.00|371.28|393.55|-560.82| +2451529|63056|11758|33092|1584503|1067|22831|7|172|1462|34|36.74|66.86|53.48|0.00|1818.32|1249.16|2273.24|145.46|0.00|1818.32|1963.78|569.16| +||4598||1584503|1067||7||1462|25|61.06||39.10|234.60|977.50|1526.50|2793.25||234.60|742.90||| +2451529|63056|14047|33092|1584503|1067|22831|7|294|1462|42|31.44|56.27|1.12|0.00|47.04|1320.48|2363.34|0.00|0.00|47.04|47.04|-1273.44| +2451529|63056|16273|33092|1584503|1067|22831|7|217|1462|1|20.01|38.21|8.02|0.00|8.02|20.01|38.21|0.48|0.00|8.02|8.50|-11.99| +2451529|63056|16696|33092|1584503|1067|22831|7|269|1462|9|50.15|66.69|54.68|127.95|492.12|451.35|600.21|29.13|127.95|364.17|393.30|-87.18| +2451529|63056|1345|33092|1584503|1067|22831|7|136|1462|38|84.50|160.55|123.62|0.00|4697.56|3211.00|6100.90|234.87|0.00|4697.56|4932.43|1486.56| +2451529|63056|12034|33092|1584503|1067|22831|7|12|1462|17|65.40|112.48|89.98|214.15|1529.66|1111.80|1912.16|105.24|214.15|1315.51|1420.75|203.71| +2451529|63056|14230|33092|1584503|1067|22831|7|243|1462|83|60.69|92.85|22.28|0.00|1849.24|5037.27|7706.55|92.46|0.00|1849.24|1941.70|-3188.03| +2451485|70907|3812|66230|211474|4718|6809|1|183|1463|55|89.96|121.44|109.29|0.00|6010.95|4947.80|6679.20|480.87|0.00|6010.95|6491.82|1063.15| +2451485|70907|7147|66230|211474|4718|6809|1|16|1463|86|59.02|73.77|70.08|0.00|6026.88|5075.72|6344.22|180.80|0.00|6026.88|6207.68|951.16| +2451485|70907|15595|66230|211474|4718|6809|1|12|1463|45|39.08|67.60|49.34|0.00|2220.30|1758.60|3042.00|155.42|0.00|2220.30|2375.72|461.70| +2451485|70907|17384|66230|211474|4718|6809|1|72|1463|90|75.67|125.61|5.02|307.22|451.80|6810.30|11304.90|4.33|307.22|144.58|148.91|-6665.72| +2451485|70907|17150|66230|211474|4718|6809|1|224|1463|90|47.56|60.40|58.58|0.00|5272.20|4280.40|5436.00|0.00|0.00|5272.20|5272.20|991.80| +2451485|70907|1960|66230|211474|4718|6809|1|235|1463|62|65.27|66.57|36.61|0.00|2269.82|4046.74|4127.34|204.28|0.00|2269.82|2474.10|-1776.92| +2451485|70907|7040|66230|211474|4718|6809|1|299|1463|95|25.05|30.81|20.33|0.00|1931.35|2379.75|2926.95|38.62|0.00|1931.35|1969.97|-448.40| +|70907|16540|||||1|108|1463|95|||52.89|0.00|5024.55||8516.75|301.47|0.00||5326.02|| +2451485|70907|13042|66230|211474|4718|6809|1|113|1463|39|8.90|16.91|9.13|245.68|356.07|347.10|659.49|2.20|245.68|110.39|112.59|-236.71| +2451485|70907|8065|66230|211474|4718|6809|1|201|1463|84|25.22|36.31|6.17|0.00|518.28|2118.48|3050.04|36.27|0.00|518.28|554.55|-1600.20| +2451485|70907|3010|66230|211474|4718|6809|1|117|1463|21|77.79|118.24|79.22|0.00|1663.62|1633.59|2483.04|149.72|0.00|1663.62|1813.34|30.03| +2451485|70907|11426|66230|211474|4718|6809|1|192|1463|87|54.35|73.37|38.15|1891.85|3319.05|4728.45|6383.19|114.17|1891.85|1427.20|1541.37|-3301.25| +2451485|70907|15715|66230|211474|4718|6809|1|108|1463|62|90.71|90.71|19.04|0.00|1180.48|5624.02|5624.02|94.43|0.00|1180.48|1274.91|-4443.54| +2451062|34779|5104|66105|426129|4200|28224|10|9|1464|5|70.44|100.72|97.69|380.99|488.45|352.20|503.60|8.59|380.99|107.46|116.05|-244.74| +2451062|34779|6064|66105|426129|4200|28224|10|252|1464|28|17.34|17.34|13.69|0.00|383.32|485.52|485.52|34.49|0.00|383.32|417.81|-102.20| +||9172|66105|426129|4200|||144|1464|1|81.37|144.02|56.16|30.32|||144.02|1.29|30.32|||| +2451062|34779|3116|66105|426129|4200|28224|10|38|1464|78|36.97|46.58|8.85|0.00|690.30|2883.66|3633.24|27.61|0.00|690.30|717.91|-2193.36| +2451062|34779|15800|66105|426129|4200|28224|10|61|1464|41|41.99|80.62|25.79|0.00|1057.39|1721.59|3305.42|10.57|0.00|1057.39|1067.96|-664.20| +2451062|34779|12130|66105|426129|4200|28224|10|38|1464|28|62.10|91.90|74.43|0.00|2084.04|1738.80|2573.20|125.04|0.00|2084.04|2209.08|345.24| +2451062|34779|17390|66105|426129|4200|28224|10|38|1464|41|70.54|92.40|32.34|0.00|1325.94|2892.14|3788.40|26.51|0.00|1325.94|1352.45|-1566.20| +2451062|34779|10078|66105|426129|4200|28224|10|76|1464|31|98.56|183.32|18.33|0.00|568.23|3055.36|5682.92|28.41|0.00|568.23|596.64|-2487.13| +2451062|34779|4927|66105|426129|4200|28224|10|244|1464|82|72.22|140.10|49.03|1206.13|4020.46|5922.04|11488.20|112.57|1206.13|2814.33|2926.90|-3107.71| +2451062|34779|14875|66105|426129|4200|28224|10|82|1464|50|72.44|144.88|11.59|0.00|579.50|3622.00|7244.00|40.56|0.00|579.50|620.06|-3042.50| +2451062|34779|14186|66105|426129|4200|28224|10|227|1464|64|78.40|96.43|1.92|0.00|122.88|5017.60|6171.52|4.91|0.00|122.88|127.79|-4894.72| +2451062|34779|8726|66105|426129|4200|28224|10|219|1464|99|67.89|115.41|55.39|0.00|5483.61|6721.11|11425.59|493.52|0.00|5483.61|5977.13|-1237.50| +2451062|34779|607|66105|426129|4200|28224|10|245|1464|65|29.95|33.84|31.47|0.00|2045.55|1946.75|2199.60|122.73|0.00|2045.55|2168.28|98.80| +2451062|34779|4624|66105|426129|4200|28224|10|33|1464|67|24.64|48.78|28.29|1895.43|1895.43|1650.88|3268.26|0.00|1895.43|0.00|0.00|-1650.88| +2452387|54773|7359|64291|140490|5936|13382|4|143|1465|20|12.90|16.25|4.87|0.00|97.40|258.00|325.00|2.92|0.00|97.40|100.32|-160.60| +2452387|54773|15585|64291|140490|5936|13382|4|118|1465|97|42.18|74.65|19.40|0.00|1881.80|4091.46|7241.05|150.54|0.00|1881.80|2032.34|-2209.66| +2452387|54773|6397|64291|140490|5936|13382|4|152|1465|18|48.27|62.75|10.04|92.16|180.72|868.86|1129.50|0.88|92.16|88.56|89.44|-780.30| +2452387|54773|17305|64291|140490|5936|13382|4|254|1465|16|12.24|21.90|18.61|0.00|297.76|195.84|350.40|17.86|0.00|297.76|315.62|101.92| +2452387|54773|7315|64291|140490|5936|13382|4|35|1465|97|39.88|66.20|35.74|0.00|3466.78|3868.36|6421.40|312.01|0.00|3466.78|3778.79|-401.58| +2452387|54773|17544|64291|140490|5936|13382|4|37|1465|34|56.98|88.88|27.55|0.00|936.70|1937.32|3021.92|74.93|0.00|936.70|1011.63|-1000.62| +2452387|54773|14641|64291|140490|5936|13382|4|136|1465|100|48.84|50.79|44.18|2915.88|4418.00|4884.00|5079.00|120.16|2915.88|1502.12|1622.28|-3381.88| +2452387|54773|5241|64291|140490|5936|13382|4|107|1465|94|23.90|28.20|1.41|18.55|132.54|2246.60|2650.80|0.00|18.55|113.99|113.99|-2132.61| +|54773|6259|64291|||13382|||1465||7.23|10.04||0.00|8.02||||0.00||8.34|| +2452387|54773|6984|64291|140490|5936|13382|4|192|1465|25|22.25|22.91|18.78|0.00|469.50|556.25|572.75|42.25|0.00|469.50|511.75|-86.75| +2452387|54773|15495|64291|140490|5936|13382|4|99|1465|50|20.26|22.28|4.01|0.00|200.50|1013.00|1114.00|14.03|0.00|200.50|214.53|-812.50| +2452387|54773|31|64291|140490|5936|13382|4|153|1465|75|93.40|186.80|184.93|0.00|13869.75|7005.00|14010.00|693.48|0.00|13869.75|14563.23|6864.75| +2452387|54773|14112|64291|140490|5936|13382|4|203|1465|72|58.07|110.33|7.72|188.98|555.84|4181.04|7943.76|29.34|188.98|366.86|396.20|-3814.18| +2452387|54773|9805|64291|140490|5936|13382|4|58|1465|61|66.94|78.98|33.96|1802.25|2071.56|4083.34|4817.78|16.15|1802.25|269.31|285.46|-3814.03| +2452387|54773|10248|64291|140490|5936|13382|4|61|1465|55|4.76|6.99|0.83|0.00|45.65|261.80|384.45|4.10|0.00|45.65|49.75|-216.15| +2451130|57949|1492|8537|703758|6979|26600|8|80|1466|28|83.89|109.89|58.24|0.00|1630.72|2348.92|3076.92|0.00|0.00|1630.72|1630.72|-718.20| +2451130|57949|5611|8537|703758|6979|26600|8|127|1466|38|70.95|127.71|125.15|0.00|4755.70|2696.10|4852.98|95.11|0.00|4755.70|4850.81|2059.60| +2451130|57949|559|8537|703758|6979|26600|8|70|1466|53|81.63|114.28|113.13|3657.49|5995.89|4326.39|6056.84|210.45|3657.49|2338.40|2548.85|-1987.99| +2451130|57949|16147|8537|703758|6979|26600|8|170|1466|30|24.85|24.85|16.40|0.00|492.00|745.50|745.50|39.36|0.00|492.00|531.36|-253.50| +2451130|57949|11875|8537|703758|6979|26600|8|295|1466|42|37.03|68.50|56.85|0.00|2387.70|1555.26|2877.00|214.89|0.00|2387.70|2602.59|832.44| +2451130|57949|10214|8537|703758|6979|26600|8|210|1466|80|69.91|84.59|20.30|0.00|1624.00|5592.80|6767.20|81.20|0.00|1624.00|1705.20|-3968.80| +2451130|57949|12446|8537|703758|6979|26600|8|25|1466|4|37.77|49.10|5.89|0.00|23.56|151.08|196.40|0.00|0.00|23.56|23.56|-127.52| +2451130|57949|244|8537|703758|6979|26600|8|146|1466|38|91.12|182.24|61.96|0.00|2354.48|3462.56|6925.12|0.00|0.00|2354.48|2354.48|-1108.08| +2451130|57949|1531|8537|703758|6979|26600|8|127|1466|57|80.83|93.76|70.32|0.00|4008.24|4607.31|5344.32|200.41|0.00|4008.24|4208.65|-599.07| +2451130|57949|3524|8537|703758|6979|26600|8|299|1466|12|14.73|17.97|1.25|0.00|15.00|176.76|215.64|0.45|0.00|15.00|15.45|-161.76| +2451907|70185|7742|3182|1724590|6445|43053|7|264|1467|28|79.90|115.85|68.35|0.00|1913.80|2237.20|3243.80|38.27|0.00|1913.80|1952.07|-323.40| +|70185|6131||1724590|6445||||1467|79|94.96|||0.00|4698.92|7501.84|13427.63|93.97|0.00|4698.92||| +2451907|70185|4400|3182|1724590|6445|43053|7|177|1467|42|97.07|110.65|4.42|0.00|185.64|4076.94|4647.30|14.85|0.00|185.64|200.49|-3891.30| +2451907|70185|13499|3182|1724590|6445|43053|7|18|1467|59|18.69|33.26|16.29|0.00|961.11|1102.71|1962.34|28.83|0.00|961.11|989.94|-141.60| +2451907|70185|15163|3182|1724590|6445|43053|7|200|1467|44|51.00|75.99|24.31|0.00|1069.64|2244.00|3343.56|42.78|0.00|1069.64|1112.42|-1174.36| +2451907|70185|11327|3182|1724590|6445|43053|7|164|1467|53|28.16|31.53|28.06|416.41|1487.18|1492.48|1671.09|85.66|416.41|1070.77|1156.43|-421.71| +2451907|70185|2653|3182|1724590|6445|43053|7|286|1467|38|9.41|10.06|9.45|0.00|359.10|357.58|382.28|7.18|0.00|359.10|366.28|1.52| +2451907||12373|3182|1724590||43053||277|1467||39.11|62.18||0.00||78.22|||0.00||19.94|-59.58| +2451907||9944|3182|1724590|6445|43053|||1467|66|43.55|51.82|41.45||||3420.12|191.49|||2927.19|| +2451907|70185|8771|3182|1724590|6445|43053|7|89|1467|19|33.33|46.32|22.69|267.28|431.11|633.27|880.08|6.55|267.28|163.83|170.38|-469.44| +2451907|70185|14534|3182|1724590|6445|43053|7|239|1467|48|99.69|148.53|43.07|0.00|2067.36|4785.12|7129.44|20.67|0.00|2067.36|2088.03|-2717.76| +2451557||5057|68902|1549633|5564||7|266|1468|15|79.64|119.46|||1540.95||1791.90|138.68||1540.95||| +2451557|65218|9152|68902|1549633|5564|9495|7|4|1468|24|12.95|19.68|15.15|0.00|363.60|310.80|472.32|0.00|0.00|363.60|363.60|52.80| +2451557|65218|3641|68902|1549633|5564|9495|7|80|1468|44|89.72|147.14|147.14|0.00|6474.16|3947.68|6474.16|582.67|0.00|6474.16|7056.83|2526.48| +|65218|13817|||5564|||108|1468||99.37|||411.24||3080.47|||411.24||860.00|-2245.51| +2451557|65218|16181|68902|1549633|5564|9495|7|147|1468|22|88.63|158.64|72.97|0.00|1605.34|1949.86|3490.08|112.37|0.00|1605.34|1717.71|-344.52| +2451557|65218|15199|68902|1549633|5564|9495|7|96|1468|86|93.81|134.14|108.65|0.00|9343.90|8067.66|11536.04|373.75|0.00|9343.90|9717.65|1276.24| +2451557|65218|16262|68902|1549633|5564|9495|7|216|1468|24|35.88|36.95|24.75|0.00|594.00|861.12|886.80|5.94|0.00|594.00|599.94|-267.12| +2451557|65218|5299|68902|1549633|5564|9495|7|198|1468|38|33.51|40.54|32.02|0.00|1216.76|1273.38|1540.52|85.17|0.00|1216.76|1301.93|-56.62| +2451557|65218|14630|68902|1549633|5564|9495|7|232|1468|58|96.05|185.37|127.90|0.00|7418.20|5570.90|10751.46|296.72|0.00|7418.20|7714.92|1847.30| +2451557|65218|16943|68902|1549633|5564|9495|7|68|1468|78|16.05|19.26|18.68|0.00|1457.04|1251.90|1502.28|116.56|0.00|1457.04|1573.60|205.14| +2451557|65218|2465|68902|1549633|5564|9495|7|60|1468|64|4.62|4.62|1.75|0.00|112.00|295.68|295.68|5.60|0.00|112.00|117.60|-183.68| +2451557|65218|8516|68902|1549633|5564|9495|7|220|1468|31|75.31|149.86|17.98|0.00|557.38|2334.61|4645.66|11.14|0.00|557.38|568.52|-1777.23| +2451557||14363||1549633|||||1468|49||||0.00|680.61|3376.59||40.83|0.00||721.44|| +2451980|41376|15993|26882|1893082|1329|6776|7|207|1469|71|3.32|5.14|4.31|0.00|306.01|235.72|364.94|9.18|0.00|306.01|315.19|70.29| +2451980|41376|17493|26882|1893082|1329|6776|7|134|1469|82|18.74|20.05|3.00|0.00|246.00|1536.68|1644.10|22.14|0.00|246.00|268.14|-1290.68| +2451980|41376|3189|26882|1893082|1329|6776|7|269|1469|81|95.75|96.70|85.09|0.00|6892.29|7755.75|7832.70|68.92|0.00|6892.29|6961.21|-863.46| +2451980|41376|5327|26882|1893082|1329|6776|7|236|1469|60|73.73|77.41|51.09|0.00|3065.40|4423.80|4644.60|30.65|0.00|3065.40|3096.05|-1358.40| +2451980|41376|15777|26882||||7|292|1469||||47.78||||15768.72|283.81|||5014.03|-4491.63| +2451980|41376|17771|26882|1893082|1329|6776|7|277|1469|10|11.90|20.82|16.65|0.00|166.50|119.00|208.20|13.32|0.00|166.50|179.82|47.50| +2451980||8145||1893082|1329|||275|1469||5.71|||0.00|286.08|365.44|||0.00|||-79.36| +2451980|41376|7009|26882|1893082|1329|6776|7|27|1469|12|66.37|114.82|80.37|0.00|964.44|796.44|1377.84|0.00|0.00|964.44|964.44|168.00| +2451980|41376|13755|26882|1893082|1329|6776|7|38|1469|48|87.70|148.21|118.56|0.00|5690.88|4209.60|7114.08|455.27|0.00|5690.88|6146.15|1481.28| +2451980|41376|2703|26882|1893082|1329|6776|7|62|1469|61|81.62|146.91|96.96|0.00|5914.56|4978.82|8961.51|0.00|0.00|5914.56|5914.56|935.74| +2451980|41376|4913|26882|1893082|1329|6776|7|101|1469|13|34.26|36.65|8.06|0.00|104.78|445.38|476.45|0.00|0.00|104.78|104.78|-340.60| +2451980|41376|14163|26882|1893082|1329|6776|7|151|1469|22|6.55|8.90|2.75|0.00|60.50|144.10|195.80|3.02|0.00|60.50|63.52|-83.60| +2451980|41376|4649|26882|1893082|1329|6776|7|282|1469|22|64.50|89.65|37.65|0.00|828.30|1419.00|1972.30|57.98|0.00|828.30|886.28|-590.70| +2451427|40709|14140|70059|117172|2685|25398|8|208|1470|31|83.17|166.34|9.98|0.00|309.38|2578.27|5156.54|9.28|0.00|309.38|318.66|-2268.89| +2451427|40709|13486|70059|117172|2685|25398|8|1|1470|37|92.64|156.56|45.40|1411.03|1679.80|3427.68|5792.72|10.75|1411.03|268.77|279.52|-3158.91| +2451427||17518||117172|||8|132|1470||33.25|62.17|11.19|0.00|828.06||4600.58||0.00||861.18|-1632.44| +2451427|40709|7111|70059|117172|2685|25398|8|279|1470|76|55.79|70.85|46.76|0.00|3553.76|4240.04|5384.60|106.61|0.00|3553.76|3660.37|-686.28| +2451427|40709|12208|70059|117172|2685|25398|8|85|1470|48|1.94|2.32|0.74|0.00|35.52|93.12|111.36|3.19|0.00|35.52|38.71|-57.60| +||14023|70059|117172|2685|25398|8|172|1470||82.76|||0.00||165.52|261.52||0.00|81.06||-84.46| +2451427|40709|2899|70059|117172|2685|25398|8|135|1470|18|29.61|34.34|17.85|289.17|321.30|532.98|618.12|2.89|289.17|32.13|35.02|-500.85| +2451427|40709|15068|70059|117172|2685|25398|8|217|1470|89|80.62|113.67|85.25|0.00|7587.25|7175.18|10116.63|455.23|0.00|7587.25|8042.48|412.07| +2450969|64614|2053|16071|441921|5371|34021|10|275|1471|47|80.84|124.49|42.32|0.00|1989.04|3799.48|5851.03|59.67|0.00|1989.04|2048.71|-1810.44| +2450969|64614|13258|16071|441921|5371|34021|10|1|1471|79|60.56|113.24|48.69|0.00|3846.51|4784.24|8945.96|346.18|0.00|3846.51|4192.69|-937.73| +2450969|64614|12620|16071|441921|5371|34021|10|15|1471|10|37.96|63.01|12.60|31.50|126.00|379.60|630.10|8.50|31.50|94.50|103.00|-285.10| +2450969|64614|4153|16071|441921|5371|34021|10|207|1471|54|87.48|121.59|7.29|0.00|393.66|4723.92|6565.86|15.74|0.00|393.66|409.40|-4330.26| +2450969|64614|17836|16071|441921|5371|34021|10|268|1471|54|10.15|11.16|2.90|0.00|156.60|548.10|602.64|3.13|0.00|156.60|159.73|-391.50| +2450969|64614|16982|16071|441921|5371|34021|10|248|1471|82|96.21|155.86|60.78|0.00|4983.96|7889.22|12780.52|49.83|0.00|4983.96|5033.79|-2905.26| +2450969|64614|1612|16071|441921|5371|34021|10|14|1471|61|3.00|3.21|1.05|0.00|64.05|183.00|195.81|3.84|0.00|64.05|67.89|-118.95| +2450969|64614|2983|16071|441921|5371|34021|10|114|1471|41|51.09|70.50|0.70|0.86|28.70|2094.69|2890.50|0.00|0.86|27.84|27.84|-2066.85| +2450969|64614|8800||441921||34021|10||1471|47|41.74||53.21|0.00||1961.78|2942.67|75.02|0.00||2575.89|| +2450969|64614|11714|16071|441921|5371|34021|10|229|1471|12|7.04|7.04|2.25|0.00|27.00|84.48|84.48|2.16|0.00|27.00|29.16|-57.48| +2451066|65009|4087|7653|948041|5676|30864|1|269|1472|52|89.12|99.81|9.98|0.00|518.96|4634.24|5190.12|5.18|0.00|518.96|524.14|-4115.28| +2451066||926||948041|5676||1|290|1472|85||20.11||||1132.20|||||1135.88|-90.10| +2451066|65009|11864|7653|||30864|||1472|89||144.81|||12629.99||12888.09|||||-6907.45| +2451066||14738|7653||||||1472||72.06|||263.81|1055.24|2666.22|2852.70|15.82|263.81|791.43|807.25|-1874.79| +2451066|65009|3164|7653|948041|5676|30864|1|145|1472|59|41.65|64.14|14.11|0.00|832.49|2457.35|3784.26|16.64|0.00|832.49|849.13|-1624.86| +2451066|65009|6721|7653|948041|5676|30864|1|35|1472|33|71.36|138.43|114.89|0.00|3791.37|2354.88|4568.19|227.48|0.00|3791.37|4018.85|1436.49| +|65009|8881|7653|||||230|1472|||99.23||0.00||1294.35|||0.00|505.95|526.18|-788.40| +2451066|65009|2467|7653|948041|5676|30864|1|103|1472|11|53.17|70.18|15.43|0.00|169.73|584.87|771.98|3.39|0.00|169.73|173.12|-415.14| +2451066|65009|1058|7653|948041|5676|30864|1|21|1472|4|27.45|42.54|27.22|0.00|108.88|109.80|170.16|8.71|0.00|108.88|117.59|-0.92| +2451066|65009|9584|7653|948041|5676|30864|1|1|1472|22|51.67|97.65|65.42|0.00|1439.24|1136.74|2148.30|86.35|0.00|1439.24|1525.59|302.50| +2451066|65009|9196|7653|948041|5676|30864|1|237|1472|93|12.10|13.91|8.20|0.00|762.60|1125.30|1293.63|38.13|0.00|762.60|800.73|-362.70| +2451066|65009|2032|7653|948041|5676|30864|1|229|1472|68|93.33|97.99|1.95|0.00|132.60|6346.44|6663.32|3.97|0.00|132.60|136.57|-6213.84| +2451890|66577|10369|28359|589989|6372|38802|4|126|1473|24|47.93|93.94|63.87|0.00|1532.88|1150.32|2254.56|107.30|0.00|1532.88|1640.18|382.56| +2451890|66577|3773|28359|589989|6372|38802|4|78|1473|16|67.70|109.67|97.60|0.00|1561.60|1083.20|1754.72|46.84|0.00|1561.60|1608.44|478.40| +2451890|66577|2678|28359|589989|6372|38802|4|288|1473|74|21.83|41.47|26.95|0.00|1994.30|1615.42|3068.78|0.00|0.00|1994.30|1994.30|378.88| +2451890|66577|9067|28359|589989|6372|38802|4|290|1473|59|91.91|104.77|40.86|0.00|2410.74|5422.69|6181.43|72.32|0.00|2410.74|2483.06|-3011.95| +|66577|1718||589989||38802|4|20|1473||66.02|126.09|46.65|||||60.27||||| +2451890|66577|3277|28359|589989|6372|38802|4|70|1473|40|62.20|99.52|61.70|493.60|2468.00|2488.00|3980.80|59.23|493.60|1974.40|2033.63|-513.60| +2451890|66577|155|28359|589989|6372|38802|4|94|1473|28|51.72|74.47|0.74|0.00|20.72|1448.16|2085.16|1.86|0.00|20.72|22.58|-1427.44| +2451890|66577|9779|28359|589989|6372|38802|4|299|1473|28|74.34|89.95|30.58|0.00|856.24|2081.52|2518.60|68.49|0.00|856.24|924.73|-1225.28| +2451890|66577|1291|28359|589989|6372|38802|4|233|1473|24|88.45|129.13|68.43|0.00|1642.32|2122.80|3099.12|131.38|0.00|1642.32|1773.70|-480.48| +2451890|66577|15086|28359|589989|6372|38802|4|186|1473|40|39.99|49.58|40.65|0.00|1626.00|1599.60|1983.20|48.78|0.00|1626.00|1674.78|26.40| +2451890|66577|7676|28359|589989|6372|38802|4|200|1473|76|68.12|76.29|41.19|0.00|3130.44|5177.12|5798.04|281.73|0.00|3130.44|3412.17|-2046.68| +2451890|66577|13373|28359|589989|6372|38802|4|139|1473|85|90.00|109.80|10.98|0.00|933.30|7650.00|9333.00|18.66|0.00|933.30|951.96|-6716.70| +|66577|5219|28359|589989||38802|4||1473||2.18|||0.00|0.00||64.24|0.00|0.00|||| +2451410|62687|1754|48199|1694666|5933|12753|2|264|1474|37|39.98|63.96|25.58|0.00|946.46|1479.26|2366.52|75.71|0.00|946.46|1022.17|-532.80| +2451410|62687|2554|48199|1694666|5933|12753|2|261|1474|79|27.05|54.10|27.59|0.00|2179.61|2136.95|4273.90|65.38|0.00|2179.61|2244.99|42.66| +2451410|62687|13066|48199|1694666|5933|12753|2|84|1474|96|55.14|79.40|19.05|0.00|1828.80|5293.44|7622.40|128.01|0.00|1828.80|1956.81|-3464.64| +2451410|62687|7186|48199|1694666|5933|12753|2|37|1474|5|3.94|5.59|3.85|0.00|19.25|19.70|27.95|0.19|0.00|19.25|19.44|-0.45| +|62687|16399||1694666|||2|288|1474|92||103.91|4.15||381.80|7709.60|9559.72|||||-7327.80| +2451410|62687|7075|48199|1694666|5933|12753|2|111|1474|72|76.92|113.07|26.00|0.00|1872.00|5538.24|8141.04|168.48|0.00|1872.00|2040.48|-3666.24| +2451410|62687|4198|48199|1694666|5933|12753|2|39|1474|96|58.64|60.39|52.53|0.00|5042.88|5629.44|5797.44|353.00|0.00|5042.88|5395.88|-586.56| +2451410|62687|12226|48199|1694666|5933|12753|2|27|1474|43|35.37|42.79|4.70|0.00|202.10|1520.91|1839.97|12.12|0.00|202.10|214.22|-1318.81| +2451410|62687|3301|48199|1694666|5933|12753|2|64|1474|25|4.77|5.77|3.46|63.14|86.50|119.25|144.25|2.10|63.14|23.36|25.46|-95.89| +2451410|62687|12658|48199|1694666|5933|12753|2|33|1474|10|93.03|106.98|10.69|0.00|106.90|930.30|1069.80|8.55|0.00|106.90|115.45|-823.40| +2451410|62687|7837|48199|1694666|5933|12753|2|275|1474|27|64.12|101.95|2.03|40.55|54.81|1731.24|2752.65|1.28|40.55|14.26|15.54|-1716.98| +2451410|62687|17881|48199|1694666|5933|12753|2|242|1474|10|95.51|99.33|47.67|0.00|476.70|955.10|993.30|0.00|0.00|476.70|476.70|-478.40| +2451410|62687|3302|48199|1694666|5933|12753|2|234|1474|21|15.25|21.80|10.68|0.00|224.28|320.25|457.80|15.69|0.00|224.28|239.97|-95.97| +2451156|32103|8725|12942|1127501|4107|36266|8|171|1475|58|74.73|139.74|121.57|0.00|7051.06|4334.34|8104.92|493.57|0.00|7051.06|7544.63|2716.72| +2451156|32103|1672|12942|1127501|4107|36266|8|284|1475|59|59.17|104.73|79.59|0.00|4695.81|3491.03|6179.07|328.70|0.00|4695.81|5024.51|1204.78| +2451156|32103|7948|12942|1127501|4107|36266|8|189|1475|97|52.57|85.16|52.79|0.00|5120.63|5099.29|8260.52|51.20|0.00|5120.63|5171.83|21.34| +2451156|32103|3146|12942|1127501|4107|36266|8|295|1475|30|11.91|20.72|14.50|0.00|435.00|357.30|621.60|13.05|0.00|435.00|448.05|77.70| +|32103|10783|||4107||8|293|1475|||21.78||0.00|921.69|1290.52||55.30|0.00|||-368.83| +2451156|32103|15703|12942|1127501|4107|36266|8|23|1475|49|87.01|159.22|89.16|0.00|4368.84|4263.49|7801.78|349.50|0.00|4368.84|4718.34|105.35| +2451156|32103|6007|12942|1127501|4107|36266|8|45|1475|47|65.37|119.62|3.58|0.00|168.26|3072.39|5622.14|15.14|0.00|168.26|183.40|-2904.13| +2451156|32103|11725|12942|1127501|4107|36266|8|256|1475|74|30.23|51.08|4.59|0.00|339.66|2237.02|3779.92|20.37|0.00|339.66|360.03|-1897.36| +2451156|32103|16588|12942|1127501|4107|36266|8|82|1475|92|87.16|157.75|123.04|0.00|11319.68|8018.72|14513.00|452.78|0.00|11319.68|11772.46|3300.96| +2452229|60020|15133|85394|647031|6715|38352|1|130|1476|8|19.73|24.46|14.92|0.00|119.36|157.84|195.68|9.54|0.00|119.36|128.90|-38.48| +2452229|60020|16659||||||278|1476|||84.04|75.63|1815.12|3025.20|3142.00|3361.60||1815.12|1210.08|1306.88|-1931.92| +2452229|60020|16507|85394|647031|6715|38352|1|191|1476|81|3.13|3.91|1.91|0.00|154.71|253.53|316.71|0.00|0.00|154.71|154.71|-98.82| +2452229|60020|10801|85394|647031|6715|38352|1|128|1476|73|65.85|81.65|56.33|0.00|4112.09|4807.05|5960.45|370.08|0.00|4112.09|4482.17|-694.96| +2452229|60020|5349|85394|647031|6715|38352|1|76|1476|28|57.10|86.79|33.84|0.00|947.52|1598.80|2430.12|47.37|0.00|947.52|994.89|-651.28| +2452229|60020|10871|85394|647031|6715|38352|1|247|1476|27|29.84|56.09|1.68|0.00|45.36|805.68|1514.43|4.08|0.00|45.36|49.44|-760.32| +2452229|60020|4181|85394|647031|6715|38352|1|235|1476|63|63.28|123.39|109.81|0.00|6918.03|3986.64|7773.57|553.44|0.00|6918.03|7471.47|2931.39| +2452229|60020|2969|85394|647031|6715|38352|1|273|1476|45|95.34|146.82|146.82|0.00|6606.90|4290.30|6606.90|330.34|0.00|6606.90|6937.24|2316.60| +2451627|46362|12644|22705|1402761|3264|37320|1|257|1477|75|34.83|67.91|15.61|0.00|1170.75|2612.25|5093.25|46.83|0.00|1170.75|1217.58|-1441.50| +||3181||||37320|1||1477|50|64.73|||0.00||||48.55|0.00|2427.50|2476.05|| +2451627|46362|3397|22705|1402761|3264|37320|1|132|1477|52|39.62|41.60|13.72|0.00|713.44|2060.24|2163.20|57.07|0.00|713.44|770.51|-1346.80| +2451627|46362|9959|22705|1402761|3264|37320|1|29|1477|73|32.51|62.09|14.28|0.00|1042.44|2373.23|4532.57|0.00|0.00|1042.44|1042.44|-1330.79| +2451627|46362|9506|22705|1402761|3264|37320|1|212|1477|28|71.41|120.68|3.62|0.00|101.36|1999.48|3379.04|6.08|0.00|101.36|107.44|-1898.12| +2451627|46362|2359|22705|1402761|3264|37320|1|216|1477|19|91.17|157.72|149.83|0.00|2846.77|1732.23|2996.68|256.20|0.00|2846.77|3102.97|1114.54| +2451627|46362|17647|22705|1402761|3264|37320|1|276|1477|16|71.34|135.54|113.85|0.00|1821.60|1141.44|2168.64|72.86|0.00|1821.60|1894.46|680.16| +2451627|46362|12464|22705|1402761|3264|37320|1|171|1477|6|71.27|128.28|65.42|282.61|392.52|427.62|769.68|7.69|282.61|109.91|117.60|-317.71| +2451627|46362|8569|22705|1402761|3264|37320|1|206|1477|2|42.08|57.22|57.22|0.00|114.44|84.16|114.44|4.57|0.00|114.44|119.01|30.28| +2451627|46362|16385|22705|1402761|3264|37320|1|263|1477|92|6.83|11.40|10.26|0.00|943.92|628.36|1048.80|9.43|0.00|943.92|953.35|315.56| +2451627|46362|11371|||||1|300|1477|||22.16|14.62|0.00|1403.52|1477.44|2127.36||0.00|1403.52|1529.83|| +2451627|46362|15151|22705|1402761|3264|37320|1|63|1477|18|31.73|49.18|13.77|0.00|247.86|571.14|885.24|4.95|0.00|247.86|252.81|-323.28| +2451627|46362|9614|22705|1402761|3264|37320|1|177|1477|19|86.38|91.56|33.87|0.00|643.53|1641.22|1739.64|45.04|0.00|643.53|688.57|-997.69| +2451627|46362|16505|22705|1402761|3264|37320|1|286|1477|31|37.80|58.21|4.07|0.00|126.17|1171.80|1804.51|7.57|0.00|126.17|133.74|-1045.63| +2451627|46362|9950|22705|1402761|3264|37320|1|177|1477|41|50.01|99.01|78.21|0.00|3206.61|2050.41|4059.41|64.13|0.00|3206.61|3270.74|1156.20| +2452318|65092|2953|65238|602690|2542|27194|2|68|1478|38|71.55|125.92|2.51|0.00|95.38|2718.90|4784.96|3.81|0.00|95.38|99.19|-2623.52| +2452318|65092|17307|65238|602690|2542|27194|2|253|1478|69|14.23|27.32|21.03|0.00|1451.07|981.87|1885.08|101.57|0.00|1451.07|1552.64|469.20| +2452318|65092|8988|65238|602690|2542|27194|2|151|1478|33|70.02|100.82|85.69|395.88|2827.77|2310.66|3327.06|145.91|395.88|2431.89|2577.80|121.23| +2452318|65092|13501|65238|602690|2542|27194|2|279|1478|3|86.40|145.15|87.09|0.00|261.27|259.20|435.45|7.83|0.00|261.27|269.10|2.07| +2452318|65092|2253|65238|602690|2542|27194|2|40|1478|93|15.01|18.91|13.61|0.00|1265.73|1395.93|1758.63|37.97|0.00|1265.73|1303.70|-130.20| +2452318|65092|2148|65238|602690|2542|27194|2|279|1478|37|85.08|158.24|136.08|2316.08|5034.96|3147.96|5854.88|108.75|2316.08|2718.88|2827.63|-429.08| +2452318|65092|90|65238|602690|2542|27194|2|170|1478|76|44.86|63.70|39.49|0.00|3001.24|3409.36|4841.20|150.06|0.00|3001.24|3151.30|-408.12| +2452318|65092|10525|65238|602690|2542|27194|2|180|1478|58|70.00|78.40|22.73|0.00|1318.34|4060.00|4547.20|79.10|0.00|1318.34|1397.44|-2741.66| +2452318|65092|1254|65238|602690|2542|27194|2|75|1478|37|38.07|61.67|19.73|0.00|730.01|1408.59|2281.79|7.30|0.00|730.01|737.31|-678.58| +2452318|65092|7428|65238|602690|2542|27194|2|200|1478|77|77.48|134.81|62.01|0.00|4774.77|5965.96|10380.37|429.72|0.00|4774.77|5204.49|-1191.19| +2452176|40144|141|17558|1812703|3401|28499|4|71|1479|23|75.91|110.82|63.16|0.00|1452.68|1745.93|2548.86|116.21|0.00|1452.68|1568.89|-293.25| +2452176|40144|16255|17558|1812703|3401|28499|4|11|1479|48|81.54|89.69|26.01|0.00|1248.48|3913.92|4305.12|49.93|0.00|1248.48|1298.41|-2665.44| +2452176|40144|3191|17558|1812703|3401|28499|4|197|1479|23|34.85|67.60|58.81|0.00|1352.63|801.55|1554.80|81.15|0.00|1352.63|1433.78|551.08| +2452176|40144|10261|17558|1812703|3401|28499|4|265|1479|6|45.91|77.58|3.87|0.00|23.22|275.46|465.48|0.00|0.00|23.22|23.22|-252.24| +2452176|40144|11181|17558|1812703|3401|28499|4|181|1479|74|6.46|12.66|4.30|120.91|318.20|478.04|936.84|9.86|120.91|197.29|207.15|-280.75| +2452176|40144|10429|17558|1812703|3401|28499|4|262|1479|65|87.57|98.95|0.00|0.00|0.00|5692.05|6431.75|0.00|0.00|0.00|0.00|-5692.05| +2452176|40144|13179|17558|1812703|3401|28499|4|208|1479|31|94.61|154.21|123.36|0.00|3824.16|2932.91|4780.51|191.20|0.00|3824.16|4015.36|891.25| +2452176|40144|10793|17558|1812703|3401|28499|4|58|1479|63|71.97|115.15|8.06|0.00|507.78|4534.11|7254.45|25.38|0.00|507.78|533.16|-4026.33| +2452176|40144|1987|17558|1812703|3401|28499|4|226|1479|79|96.91|148.27|140.85|0.00|11127.15|7655.89|11713.33|0.00|0.00|11127.15|11127.15|3471.26| +2452176|40144|1049|17558|1812703|3401|28499|4|46|1479|97|69.68|119.84|101.86|0.00|9880.42|6758.96|11624.48|98.80|0.00|9880.42|9979.22|3121.46| +2452176|40144|7179|17558|1812703|3401|28499|4|163|1479|32|27.08|39.80|19.90|0.00|636.80|866.56|1273.60|44.57|0.00|636.80|681.37|-229.76| +2452176|40144|16563|17558|1812703|3401|28499|4|11|1479|58|43.97|63.31|17.72|0.00|1027.76|2550.26|3671.98|20.55|0.00|1027.76|1048.31|-1522.50| +2452176|40144|16633|17558|1812703|3401|28499|4|160|1479|51|15.46|15.46|14.06|566.47|717.06|788.46|788.46|10.54|566.47|150.59|161.13|-637.87| +2452176|40144|11407|17558|1812703|3401|28499|4|31|1479|10|51.32|75.95|61.51|0.00|615.10|513.20|759.50|0.00|0.00|615.10|615.10|101.90| +2452176|40144|7073|17558|1812703|3401|28499|4|59|1479|22|34.89|54.77|20.26|0.00|445.72|767.58|1204.94|0.00|0.00|445.72|445.72|-321.86| +2451234|68351|14174|51808|1065652|5896|6140|2|118|1480|2|34.81|60.22|45.76|0.00|91.52|69.62|120.44|0.00|0.00|91.52|91.52|21.90| +2451234|68351|8503|51808|1065652|5896|6140|2|66|1480|75|20.51|35.07|5.96|0.00|447.00|1538.25|2630.25|22.35|0.00|447.00|469.35|-1091.25| +2451234|68351|2227|51808|1065652|5896|6140|2|162|1480|44|69.41|88.15|70.52|0.00|3102.88|3054.04|3878.60|31.02|0.00|3102.88|3133.90|48.84| +2451234|68351|1351|51808|1065652|5896|6140|2|9|1480|78|27.69|30.45|19.79|0.00|1543.62|2159.82|2375.10|92.61|0.00|1543.62|1636.23|-616.20| +2451234|68351|4255|51808|1065652|5896|6140|2|80|1480|33|94.58|100.25|96.24|0.00|3175.92|3121.14|3308.25|285.83|0.00|3175.92|3461.75|54.78| +2451234|68351|2726|51808|1065652|5896|6140|2|169|1480|16|11.98|22.40|21.72|10.42|347.52|191.68|358.40|16.85|10.42|337.10|353.95|145.42| +2451234|68351|12806|51808|1065652|5896|6140|2|124|1480|57|61.96|104.71|56.54|0.00|3222.78|3531.72|5968.47|96.68|0.00|3222.78|3319.46|-308.94| +2451234|68351|17690|51808|1065652|5896|6140|2|25|1480|75|98.48|169.38|145.66|8958.09|10924.50|7386.00|12703.50|0.00|8958.09|1966.41|1966.41|-5419.59| +2451234|68351|2612|51808|1065652|5896|6140|2|76|1480|64|28.83|38.63|15.83|0.00|1013.12|1845.12|2472.32|60.78|0.00|1013.12|1073.90|-832.00| +2451234|68351|11167|51808|1065652|5896|6140|2|162|1480|20|19.70|26.98|5.66|52.07|113.20|394.00|539.60|4.89|52.07|61.13|66.02|-332.87| +2451234|68351|17086|51808|1065652|5896|6140|2|259|1480|32|3.58|5.87|1.76|0.00|56.32|114.56|187.84|1.12|0.00|56.32|57.44|-58.24| +2451234|68351|2917|51808|1065652|5896|6140|2|263|1480|72|12.32|20.57|9.87|0.00|710.64|887.04|1481.04|56.85|0.00|710.64|767.49|-176.40| +2451234|68351|15122|51808|1065652|5896|6140|2|243|1480|18|15.94|20.40|2.65|24.80|47.70|286.92|367.20|0.45|24.80|22.90|23.35|-264.02| +2451234|68351|760|51808|1065652|5896|6140|2|193|1480|14|17.88|34.68|12.48|167.73|174.72|250.32|485.52|0.27|167.73|6.99|7.26|-243.33| +2452403|74707|5239|81736|1090045|307|33816|10|147|1481|51|27.00|30.51|12.81|0.00|653.31|1377.00|1556.01|26.13|0.00|653.31|679.44|-723.69| +2452403|74707|11787|81736|1090045|307|33816|10|283|1481|55|78.71|131.44|118.29|0.00|6505.95|4329.05|7229.20|65.05|0.00|6505.95|6571.00|2176.90| +2452403|74707|12000|81736|1090045|307|33816|10|232|1481|51|51.61|70.18|52.63|0.00|2684.13|2632.11|3579.18|214.73|0.00|2684.13|2898.86|52.02| +2452403|74707|9697|81736|1090045|307|33816|10|44|1481|40|19.45|28.20|4.51|0.00|180.40|778.00|1128.00|10.82|0.00|180.40|191.22|-597.60| +2452403|74707|17550|81736|1090045|307|33816|10|72|1481|71|73.20|84.18|65.66|0.00|4661.86|5197.20|5976.78|233.09|0.00|4661.86|4894.95|-535.34| +2452403|74707|16857|81736|1090045|307|33816|10|162|1481|60|45.29|65.67|15.76|472.80|945.60|2717.40|3940.20|9.45|472.80|472.80|482.25|-2244.60| +2452403|74707|13411|81736|1090045|307|33816|10|213|1481|35|57.84|94.27|93.32|0.00|3266.20|2024.40|3299.45|32.66|0.00|3266.20|3298.86|1241.80| +2452403|74707|3270|81736|1090045|307|33816|10|223|1481|16|56.37|92.44|32.35|0.00|517.60|901.92|1479.04|5.17|0.00|517.60|522.77|-384.32| +2452403|74707|3123|81736|1090045|307|33816|10|236|1481|55|54.26|105.80|46.55|51.20|2560.25|2984.30|5819.00|175.63|51.20|2509.05|2684.68|-475.25| +2452403|74707|16254|81736|1090045|307|33816|10|257|1481|80|63.72|89.84|58.39|0.00|4671.20|5097.60|7187.20|326.98|0.00|4671.20|4998.18|-426.40| +2452403|74707|2508|81736|1090045|307|33816|10|164|1481|99|26.66|52.78|42.75|0.00|4232.25|2639.34|5225.22|169.29|0.00|4232.25|4401.54|1592.91| +2452403|74707|14712|81736|1090045|307|33816|10|9|1481|9|35.67|51.72|40.85|0.00|367.65|321.03|465.48|3.67|0.00|367.65|371.32|46.62| +2452242|46380|17297|80393|388023|6406|38789|4|85|1482|56|2.04|2.97|2.67|70.27|149.52|114.24|166.32|0.79|70.27|79.25|80.04|-34.99| +2452242|46380|13221||388023|6406|38789||166|1482||9.63|||0.00||163.71||14.18|0.00|202.64||38.93| +2452242|46380|13795|80393|388023|6406|38789|4|61|1482|7|20.61|34.00|27.20|0.00|190.40|144.27|238.00|3.80|0.00|190.40|194.20|46.13| +2452242|46380|16649|80393|388023|6406|38789|4|298|1482|21|60.92|104.78|71.25|0.00|1496.25|1279.32|2200.38|134.66|0.00|1496.25|1630.91|216.93| +2452242|46380|15991|80393|388023|6406|38789|4|245|1482|95|72.93|102.10|69.42|0.00|6594.90|6928.35|9699.50|593.54|0.00|6594.90|7188.44|-333.45| +||2787|80393|388023|||4|161|1482||||19.48|||2460.83|4182.90|99.54||||| +2452242|46380|6677|80393|388023|6406|38789|4|8|1482|54|99.49|191.02|36.29|0.00|1959.66|5372.46|10315.08|39.19|0.00|1959.66|1998.85|-3412.80| +2452242|46380|12161|80393|388023|6406|38789|4|20|1482|52|91.85|128.59|95.15|2671.81|4947.80|4776.20|6686.68|113.79|2671.81|2275.99|2389.78|-2500.21| +2452242|46380|8433|80393|388023|6406|38789|4|239|1482|40|32.05|61.85|17.31|0.00|692.40|1282.00|2474.00|20.77|0.00|692.40|713.17|-589.60| +2452242|46380|10221|80393|388023|6406|38789|4|7|1482|41|35.90|44.87|4.03|0.00|165.23|1471.90|1839.67|1.65|0.00|165.23|166.88|-1306.67| +2452242|46380|10043|80393|388023|6406|38789|4|231|1482|85|75.58|126.21|84.56|0.00|7187.60|6424.30|10727.85|215.62|0.00|7187.60|7403.22|763.30| +2451268|58096|5635|96411|544410|4409|40408|8|170|1483|1|94.14|141.21|29.65|0.00|29.65|94.14|141.21|1.18|0.00|29.65|30.83|-64.49| +2451268|58096|5002|96411|544410|4409|40408|8|86|1483|52|3.96|6.69|1.94|0.00|100.88|205.92|347.88|0.00|0.00|100.88|100.88|-105.04| +2451268|58096|11828|96411|544410|4409|40408|8|186|1483|8|29.06|44.46|3.11|21.64|24.88|232.48|355.68|0.16|21.64|3.24|3.40|-229.24| +2451268|58096|7478|96411|544410|4409|40408|8|166|1483|14|5.88|9.81|3.23|0.00|45.22|82.32|137.34|2.71|0.00|45.22|47.93|-37.10| +2451268|58096|11312|96411|544410|4409|40408|8|279|1483|66|27.28|45.83|42.16|0.00|2782.56|1800.48|3024.78|0.00|0.00|2782.56|2782.56|982.08| +2451268|58096|13010|96411|544410|4409|40408|8|238|1483|30|43.79|53.42|22.43|87.47|672.90|1313.70|1602.60|29.27|87.47|585.43|614.70|-728.27| +2451268|58096|5744|96411|544410|4409|40408|8|168|1483|26|69.58|111.32|76.81|0.00|1997.06|1809.08|2894.32|139.79|0.00|1997.06|2136.85|187.98| +2451268|58096|184|96411|544410|4409|40408|8|22|1483|44|29.79|56.00|48.72|0.00|2143.68|1310.76|2464.00|192.93|0.00|2143.68|2336.61|832.92| +2451268|58096|2120|96411|544410|4409|40408|8|77|1483|57|28.69|43.89|19.31|0.00|1100.67|1635.33|2501.73|11.00|0.00|1100.67|1111.67|-534.66| +2451268|58096|175|96411|544410|4409|40408|8|104|1483||80.55|159.48|140.34|0.00|8420.40||9568.80|589.42|0.00|8420.40||3587.40| +2451268|58096|15838|96411|544410|4409|40408|8|82|1483|33|26.08|29.20|11.09|109.79|365.97|860.64|963.60|17.93|109.79|256.18|274.11|-604.46| +2451268|58096|16486|96411|544410|4409|40408|8|56|1483|31|39.60|68.50|26.03|0.00|806.93|1227.60|2123.50|8.06|0.00|806.93|814.99|-420.67| +2452095|64675|9163|7438|1187725|4459|15146|8|288|1484|85|9.30|13.57|7.32|55.99|622.20|790.50|1153.45|28.31|55.99|566.21|594.52|-224.29| +2452095|64675|16389|7438|1187725|4459|15146|8|104|1484|6|30.44|56.00|45.92|0.00|275.52|182.64|336.00|16.53|0.00|275.52|292.05|92.88| +2452095|64675|17969|7438|1187725|4459|15146|8|281|1484|11|88.62|142.67|62.77|0.00|690.47|974.82|1569.37|62.14|0.00|690.47|752.61|-284.35| +2452095|64675|489|7438|1187725|4459|15146|8|292|1484|57|17.38|22.42|12.33|0.00|702.81|990.66|1277.94|0.00|0.00|702.81|702.81|-287.85| +2452095|64675|2225|7438|1187725|4459|15146|8|165|1484|23|41.77|61.81|6.18|0.00|142.14|960.71|1421.63|2.84|0.00|142.14|144.98|-818.57| +2452095|64675|1013|7438|1187725|4459|15146|8|76|1484|76|71.53|128.03|64.01|0.00|4864.76|5436.28|9730.28|0.00|0.00|4864.76|4864.76|-571.52| +2452095|64675|13809|7438|1187725|4459|15146|8|96|1484|50|30.10|41.83|31.37|0.00|1568.50|1505.00|2091.50|125.48|0.00|1568.50|1693.98|63.50| +2452095|64675|2811|7438|1187725|4459|15146|8|165|1484|29|96.54|176.66|97.16|0.00|2817.64|2799.66|5123.14|84.52|0.00|2817.64|2902.16|17.98| +2452095|64675|943|7438|1187725|4459|15146|8|56|1484|26|19.60|27.83|23.65|0.00|614.90|509.60|723.58|12.29|0.00|614.90|627.19|105.30| +2452095|64675|6019|7438|1187725|4459|15146|8|124|1484|41|82.44|139.32|110.06|0.00|4512.46|3380.04|5712.12|406.12|0.00|4512.46|4918.58|1132.42| +2452095||14241|||||8||1484|55||133.13|||5198.60|3698.20|7322.15|103.97|||5302.57|| +2452095|64675|717|7438|1187725|4459|15146|8|288|1484|10|35.61|54.12|16.23|0.00|162.30|356.10|541.20|4.86|0.00|162.30|167.16|-193.80| +||11605|7438|1187725|4459|15146|||1484||89.69|||0.00|||5360.38|276.57|0.00|4609.60|4886.17|752.93| +2452095|64675|85|7438|1187725|4459|15146|8|237|1484|16|29.13|40.49|4.45|0.00|71.20|466.08|647.84|6.40|0.00|71.20|77.60|-394.88| +2452109|63016|863|55708|2171|2234|28297|2|213|1485|72|6.72|7.59|0.22|0.00|15.84|483.84|546.48|0.31|0.00|15.84|16.15|-468.00| +2452109|63016|13949|55708|2171|2234|28297|2|39|1485|66|16.62|21.93|5.04|0.00|332.64|1096.92|1447.38|26.61|0.00|332.64|359.25|-764.28| +2452109|63016|7273|55708|2171|2234|28297|2|29|1485|26|63.07|114.15|7.99|0.00|207.74|1639.82|2967.90|12.46|0.00|207.74|220.20|-1432.08| +2452109|63016|3171|55708|2171|2234|28297|2|167|1485|48|78.79|99.27|53.60|0.00|2572.80|3781.92|4764.96|180.09|0.00|2572.80|2752.89|-1209.12| +2452109|63016|3953|55708|2171|2234|28297|2|35|1485|60|62.82|85.43|57.23|0.00|3433.80|3769.20|5125.80|240.36|0.00|3433.80|3674.16|-335.40| +2452109|63016|1041|55708|2171|2234|28297|2|142|1485|44|44.69|61.22|52.03|0.00|2289.32|1966.36|2693.68|91.57|0.00|2289.32|2380.89|322.96| +2452109|63016|9105|55708|2171|2234|28297|2|47|1485|95|17.31|18.52|4.81|0.00|456.95|1644.45|1759.40|36.55|0.00|456.95|493.50|-1187.50| +2452109|63016|6255|55708|2171|2234|28297|2|119|1485|54|37.22|48.38|31.44|0.00|1697.76|2009.88|2612.52|16.97|0.00|1697.76|1714.73|-312.12| +2452109|63016|13789|55708|2171|2234|28297|2|71|1485|91|45.88|55.97|12.31|1041.79|1120.21|4175.08|5093.27|4.70|1041.79|78.42|83.12|-4096.66| +2451426|67447|9782|30837|1656191|3592|42146|1|66|1486|24|22.23|40.01|33.60|0.00|806.40|533.52|960.24|56.44|0.00|806.40|862.84|272.88| +||5125|30837|1656191|3592||1||1486|81|1.87||1.59||128.79|151.47|||||136.51|-22.68| +2451426|67447|1408|30837|1656191|3592|42146|1|13|1486|61|52.85|98.82|35.57|390.55|2169.77|3223.85|6028.02|160.12|390.55|1779.22|1939.34|-1444.63| +2451426|67447|6529|30837|1656191|3592|42146|1|185|1486|45|45.32|80.21|20.05|171.42|902.25|2039.40|3609.45|0.00|171.42|730.83|730.83|-1308.57| +2451426|67447|15632|30837|1656191|3592|42146|1|104|1486|73|89.79|125.70|124.44|0.00|9084.12|6554.67|9176.10|363.36|0.00|9084.12|9447.48|2529.45| +2451426|67447|15550|30837|1656191|3592|42146|1|177|1486|97|81.19|116.91|115.74|0.00|11226.78|7875.43|11340.27|561.33|0.00|11226.78|11788.11|3351.35| +2451426|67447|9937|30837|1656191|3592|42146|1|179|1486|64|26.33|41.86|18.41|0.00|1178.24|1685.12|2679.04|23.56|0.00|1178.24|1201.80|-506.88| +2451426|67447|4897|30837|1656191|3592|42146|1|66|1486|69|75.39|120.62|90.46|0.00|6241.74|5201.91|8322.78|561.75|0.00|6241.74|6803.49|1039.83| +2451426|67447|4135|30837|1656191|3592|42146|1|261|1486|77|76.51|87.22|12.21|0.00|940.17|5891.27|6715.94|75.21|0.00|940.17|1015.38|-4951.10| +2451426|67447|17186|30837|1656191|3592|42146|1|141|1486|43|61.33|102.42|67.59|0.00|2906.37|2637.19|4404.06|0.00|0.00|2906.37|2906.37|269.18| +2451426|67447|15553|30837|1656191|3592|42146|1|41|1486|10|50.91|71.78|40.19|0.00|401.90|509.10|717.80|28.13|0.00|401.90|430.03|-107.20| +2451714|39162|2090|35717|1753821|5078|20382|1|283|1487|71|74.48|101.29|11.14|0.00|790.94|5288.08|7191.59|31.63|0.00|790.94|822.57|-4497.14| +2451714|39162|4019|35717|1753821|5078|20382|1|145|1487|9|85.71|103.70|11.40|0.00|102.60|771.39|933.30|2.05|0.00|102.60|104.65|-668.79| +2451714||9223||1753821|||1||1487|66|4.73||5.41|0.00|357.06||368.28|21.42|0.00||378.48|| +2451714|39162|5371|35717|1753821|5078|20382|1|29|1487|86|46.90|57.21|33.75|0.00|2902.50|4033.40|4920.06|232.20|0.00|2902.50|3134.70|-1130.90| +2451714|39162|13214|35717|1753821|5078|20382|1|17|1487|92|74.55|93.18|53.11|0.00|4886.12|6858.60|8572.56|0.00|0.00|4886.12|4886.12|-1972.48| +2451714|39162|1346|35717|1753821|5078|20382|1|160|1487|62|29.75|30.34|26.39|0.00|1636.18|1844.50|1881.08|49.08|0.00|1636.18|1685.26|-208.32| +2451714|39162|15673|35717|1753821|5078|20382|1|236|1487|85|4.63|7.13|4.84|0.00|411.40|393.55|606.05|4.11|0.00|411.40|415.51|17.85| +2451714|39162|12097|35717|1753821|5078|20382|1|3|1487|46|35.76|70.80|61.59|0.00|2833.14|1644.96|3256.80|28.33|0.00|2833.14|2861.47|1188.18| +||15659|||5078|||176|1487||30.41||4.07|||1520.50||||203.50|221.81|| +2451714|39162|16117|35717|1753821|5078|20382|1|144|1487|36|87.90|130.97|47.14|0.00|1697.04|3164.40|4714.92|84.85|0.00|1697.04|1781.89|-1467.36| +2451714|39162|5585|35717|1753821|5078|20382|1|72|1487|5|43.59|66.69|21.34|3.20|106.70|217.95|333.45|6.21|3.20|103.50|109.71|-114.45| +2451714|39162|3779|35717|1753821|5078|20382|1|187|1487|60|87.77|148.33|48.94|1703.11|2936.40|5266.20|8899.80|110.99|1703.11|1233.29|1344.28|-4032.91| +2451714|39162|3835|35717|1753821|5078|20382|1|268|1487|84|33.64|42.72|41.86|0.00|3516.24|2825.76|3588.48|210.97|0.00|3516.24|3727.21|690.48| +2451714|39162|925|35717|1753821|5078|20382|1|220|1487|49|67.24|103.54|80.76|0.00|3957.24|3294.76|5073.46|0.00|0.00|3957.24|3957.24|662.48| +2451714|39162|3698|35717|1753821|5078|20382|1|272|1487|31|37.31|46.63|21.91|0.00|679.21|1156.61|1445.53|20.37|0.00|679.21|699.58|-477.40| +2451714|39162|16763|35717|1753821|5078|20382|1|127|1487|58|2.33|3.00|0.90|0.00|52.20|135.14|174.00|0.52|0.00|52.20|52.72|-82.94| +2451412|34193|14491|33822|940827|3032|29237|2|120|1488|31|12.73|13.36|2.80|0.00|86.80|394.63|414.16|5.20|0.00|86.80|92.00|-307.83| +2451412|34193|8797|33822|940827|3032|29237|2|19|1488|87|68.18|85.90|25.77|919.21|2241.99|5931.66|7473.30|105.82|919.21|1322.78|1428.60|-4608.88| +2451412|34193|1819|33822|940827|3032|29237|2|12|1488|5|8.63|9.83|7.86|0.00|39.30|43.15|49.15|2.75|0.00|39.30|42.05|-3.85| +2451412|34193|11120|33822|940827|3032|29237|2|134|1488|42|55.98|86.76|71.14|0.00|2987.88|2351.16|3643.92|89.63|0.00|2987.88|3077.51|636.72| +2451412|34193|2942|33822|940827|3032|29237|2|279|1488|93|67.51|130.29|79.47|0.00|7390.71|6278.43|12116.97|221.72|0.00|7390.71|7612.43|1112.28| +|34193|9283|33822||3032||2||1488|12|33.01|51.82||0.00|578.28||621.84||0.00|578.28|612.97|182.16| +2451412|34193|2750|33822|940827|3032|29237|2|147|1488|6|63.85|127.06|55.90|0.00|335.40|383.10|762.36|6.70|0.00|335.40|342.10|-47.70| +2451412|34193|2941|33822|940827|3032|29237|2|145|1488|17|98.01|163.67|116.20|671.63|1975.40|1666.17|2782.39|13.03|671.63|1303.77|1316.80|-362.40| +2451412|34193|12010|33822|940827|3032|29237|2|286|1488|13|43.47|70.85|9.91|0.00|128.83|565.11|921.05|1.28|0.00|128.83|130.11|-436.28| +||6050|33822|940827||29237||66|1488||16.88|29.37|9.69||784.89|1367.28|2378.97|||784.89||| +2451412|34193|1942|33822|940827|3032|29237|2|143|1488|22|32.03|62.13|41.00|0.00|902.00|704.66|1366.86|36.08|0.00|902.00|938.08|197.34| +2451412|34193|4406|33822|940827|3032|29237|2|82|1488|2|22.49|23.61|14.87|0.00|29.74|44.98|47.22|0.00|0.00|29.74|29.74|-15.24| +2451412|34193|9334|33822|940827|3032|29237|2|244|1488|16|78.46|120.82|45.91|0.00|734.56|1255.36|1933.12|22.03|0.00|734.56|756.59|-520.80| +2451412|34193|376|33822|940827|3032|29237|2|123|1488|7|91.91|125.91|99.46|529.12|696.22|643.37|881.37|5.01|529.12|167.10|172.11|-476.27| +||16058|33822||3032||||1488|17|||||||2995.57|89.86||1497.70||-130.39| +2452602|70696|801|23836|1634673|3557|27194|10|60|1489|78|80.43|112.60|56.30|2854.41|4391.40|6273.54|8782.80|46.10|2854.41|1536.99|1583.09|-4736.55| +2452602|70696|11112|23836|1634673|3557|27194|10|39|1489|63|15.48|22.13|18.14|0.00|1142.82|975.24|1394.19|79.99|0.00|1142.82|1222.81|167.58| +2452602|70696|8100|23836|1634673|3557|27194|10|216|1489|29|67.71|77.86|3.11|0.00|90.19|1963.59|2257.94|3.60|0.00|90.19|93.79|-1873.40| +2452602|70696|7015|23836|1634673|3557|27194|10|158|1489|71|57.73|111.41|73.53|0.00|5220.63|4098.83|7910.11|417.65|0.00|5220.63|5638.28|1121.80| +2452602|70696|10956|23836|1634673|3557|27194|10|53|1489|47|29.29|47.44|39.84|0.00|1872.48|1376.63|2229.68|168.52|0.00|1872.48|2041.00|495.85| +2452602|70696|16140|23836|1634673|3557|27194|10|240|1489|81|46.37|51.00|35.70|0.00|2891.70|3755.97|4131.00|115.66|0.00|2891.70|3007.36|-864.27| +|70696|11737|23836|1634673|3557|27194|||1489|63|||52.58|1424.39|||11423.79|132.17|1424.39|1888.15|2020.32|| +||11544|23836||3557|27194|10|278|1489|34|||||3662.14|2570.06|4883.08|329.59|||3991.73|| +2452602|70696|11293|23836|1634673|3557|27194|10|187|1489|70|73.03|135.10|116.18|0.00|8132.60|5112.10|9457.00|406.63|0.00|8132.60|8539.23|3020.50| +2452602|70696|13026|23836|1634673|3557|27194|10|137|1489|45|48.39|52.26|51.73|0.00|2327.85|2177.55|2351.70|116.39|0.00|2327.85|2444.24|150.30| +2452602|70696|12079|23836|1634673|3557|27194|10|240|1489|73|85.33|145.06|58.02|3388.36|4235.46|6229.09|10589.38|16.94|3388.36|847.10|864.04|-5381.99| +2451596|66134|10199|16752|105127|1949|40324|4|103|1490|87|70.83|87.82|33.37|1509.65|2903.19|6162.21|7640.34|69.67|1509.65|1393.54|1463.21|-4768.67| +2451596|66134|9931|16752|105127|1949|40324|4|71|1490|41|77.40|131.58|114.47|1220.25|4693.27|3173.40|5394.78|0.00|1220.25|3473.02|3473.02|299.62| +2451596|66134|14497|16752|105127|1949|40324|4|50|1490|12|68.67|83.09|76.44|0.00|917.28|824.04|997.08|0.00|0.00|917.28|917.28|93.24| +2451596|66134|9635|16752|105127|1949|40324|4|244|1490|80|73.49|109.50|14.23|0.00|1138.40|5879.20|8760.00|56.92|0.00|1138.40|1195.32|-4740.80| +2451596|66134|8935|16752|105127|1949|40324|4|121|1490|46|52.70|92.22|59.94|0.00|2757.24|2424.20|4242.12|82.71|0.00|2757.24|2839.95|333.04| +2451596|66134|5684|16752|105127|1949|40324|4|223|1490|11|87.11|134.14|79.14|17.41|870.54|958.21|1475.54|42.65|17.41|853.13|895.78|-105.08| +2451596|66134|17702|16752|105127|1949|40324|4|256|1490|75|92.30|182.75|42.03|0.00|3152.25|6922.50|13706.25|189.13|0.00|3152.25|3341.38|-3770.25| +2451596|66134|5695|16752|105127|1949|40324|4|244|1490|87|83.50|158.65|109.46|0.00|9523.02|7264.50|13802.55|666.61|0.00|9523.02|10189.63|2258.52| +2451596|66134|9967|16752|105127|1949|40324|4|158|1490|30|70.29|107.54|2.15|29.02|64.50|2108.70|3226.20|2.12|29.02|35.48|37.60|-2073.22| +2451474|71063|14821|6859|1054287|5362|7990|2|43|1491|32|43.18|57.86|49.75|0.00|1592.00|1381.76|1851.52|95.52|0.00|1592.00|1687.52|210.24| +2451474|71063|74|6859|1054287|5362|7990|2|3|1491|33|3.14|6.28|3.07|0.00|101.31|103.62|207.24|4.05|0.00|101.31|105.36|-2.31| +2451474|71063|616|6859|1054287|5362|7990|2|151|1491|82|43.10|78.87|31.54|0.00|2586.28|3534.20|6467.34|181.03|0.00|2586.28|2767.31|-947.92| +2451474|71063|1778|6859|1054287|5362|7990|2|235|1491|57|43.50|68.73|32.30|0.00|1841.10|2479.50|3917.61|18.41|0.00|1841.10|1859.51|-638.40| +2451474|71063|15992|6859|1054287|5362|7990|2|151|1491|40|89.98|114.27|5.71|200.99|228.40|3599.20|4570.80|1.91|200.99|27.41|29.32|-3571.79| +2451474|71063|17492|6859|1054287|5362|7990|2|13|1491|28|19.71|33.90|25.42|0.00|711.76|551.88|949.20|0.00|0.00|711.76|711.76|159.88| +2451474|71063|3188|6859|1054287|5362|7990|2|290|1491|75|14.14|16.40|12.13|454.87|909.75|1060.50|1230.00|9.09|454.87|454.88|463.97|-605.62| +2451474|71063|5326|6859|1054287|5362|7990|2|75|1491|67|32.37|59.23|37.31|0.00|2499.77|2168.79|3968.41|99.99|0.00|2499.77|2599.76|330.98| +2451726|63004|15728|16288|223611|5334|9010|8|104|1492|52|71.66|139.02|72.29|2744.12|3759.08|3726.32|7229.04|30.44|2744.12|1014.96|1045.40|-2711.36| +2451726|63004|3902|16288|223611|5334|9010|8|44|1492|5|54.64|92.34|85.87|0.00|429.35|273.20|461.70|25.76|0.00|429.35|455.11|156.15| +2451726|63004|14204|16288|223611|5334|9010|8|243|1492|98|76.99|147.05|22.05|0.00|2160.90|7545.02|14410.90|21.60|0.00|2160.90|2182.50|-5384.12| +2451726|63004|5198|16288|223611|5334|9010|8|80|1492|77|71.79|134.96|95.82|0.00|7378.14|5527.83|10391.92|590.25|0.00|7378.14|7968.39|1850.31| +2451726||5089||||9010|||1492|67|57.58|73.12||4326.80|4507.09||4899.04||4326.80|||| +2451726|63004|15139|16288|223611|5334|9010|8|153|1492|51|95.28|109.57|61.35|0.00|3128.85|4859.28|5588.07|0.00|0.00|3128.85|3128.85|-1730.43| +||4087|16288|223611||||140|1492||50.33|||||50.33|||||0.87|| +2451726|63004|926|16288|223611|5334|9010|8|2|1492|11|26.87|40.84|11.02|0.00|121.22|295.57|449.24|9.69|0.00|121.22|130.91|-174.35| +||11864|16288|223611|5334|9010|||1492|||107.32||0.00|1339.20||||0.00|||| +2452604|64392|6321|43365|374239|3866|18740|7|117|1493|2|97.20|127.33|92.95|0.00|185.90|194.40|254.66|3.71|0.00|185.90|189.61|-8.50| +2452604|64392|2295|43365|374239|3866|18740|7|170|1493|50|65.85|113.26|54.36|0.00|2718.00|3292.50|5663.00|0.00|0.00|2718.00|2718.00|-574.50| +2452604|64392|12096|43365|374239|3866|18740|7|9|1493|65|94.15|102.62|43.10|0.00|2801.50|6119.75|6670.30|140.07|0.00|2801.50|2941.57|-3318.25| +2452604|64392|8874|43365|374239|3866|18740|7|284|1493|14|14.26|21.39|21.39|0.00|299.46|199.64|299.46|11.97|0.00|299.46|311.43|99.82| +2452604|64392|16629|43365|374239|3866|18740|7|210|1493|96|43.58|75.82|0.00|0.00|0.00|4183.68|7278.72|0.00|0.00|0.00|0.00|-4183.68| +2452604|64392|16824|43365|374239|3866|18740|7|184|1493|26|80.45|123.89|18.58|0.00|483.08|2091.70|3221.14|43.47|0.00|483.08|526.55|-1608.62| +2452604|64392|13309|43365|374239|3866|18740|7|58|1493|16|74.27|82.43|27.20|0.00|435.20|1188.32|1318.88|13.05|0.00|435.20|448.25|-753.12| +2452604|64392|2061|43365|374239|3866|18740|7|53|1493|96|95.69|145.44|27.63|0.00|2652.48|9186.24|13962.24|53.04|0.00|2652.48|2705.52|-6533.76| +2452604|64392|14910|43365|374239|3866|18740|7|195|1493|43|14.08|25.06|8.01|0.00|344.43|605.44|1077.58|30.99|0.00|344.43|375.42|-261.01| +2452604|64392|5358|43365|374239|3866|18740|7|172|1493|6|67.64|75.08|58.56|108.92|351.36|405.84|450.48|2.42|108.92|242.44|244.86|-163.40| +2452604|64392|975|43365|374239|3866|18740|7|64|1493|36|38.37|44.12|38.38|0.00|1381.68|1381.32|1588.32|96.71|0.00|1381.68|1478.39|0.36| +2452604|64392|2577|43365|374239|3866|18740|7|140|1493|63|2.17|3.92|2.86|0.00|180.18|136.71|246.96|9.00|0.00|180.18|189.18|43.47| +2451067|63202|13621||356877||23732|||1494|||30.27|||948.64|2880.22||||948.64||| +2451067|63202|14314|85273|356877|6881|23732|10|190|1494|28|62.30|74.13|57.82|0.00|1618.96|1744.40|2075.64|145.70|0.00|1618.96|1764.66|-125.44| +2451067|63202|8968|85273|356877|6881|23732|10|101|1494|76|97.85|123.29|20.95|0.00|1592.20|7436.60|9370.04|127.37|0.00|1592.20|1719.57|-5844.40| +2451067|63202|8392|85273|356877|6881|23732|10|151|1494|82|44.72|51.42|9.76|0.00|800.32|3667.04|4216.44|72.02|0.00|800.32|872.34|-2866.72| +2451067|63202|13094|85273|356877|6881|23732|10|247|1494|93|44.31|53.17|0.53|0.00|49.29|4120.83|4944.81|0.98|0.00|49.29|50.27|-4071.54| +2451067|63202|6823|85273|356877|6881|23732|10|18|1494|46|98.72|118.46|34.35|0.00|1580.10|4541.12|5449.16|0.00|0.00|1580.10|1580.10|-2961.02| +2451067|63202|7088|85273|356877|6881|23732|10|264|1494|66|72.40|72.40|66.60|0.00|4395.60|4778.40|4778.40|219.78|0.00|4395.60|4615.38|-382.80| +2451067|63202|8366|85273|356877|6881|23732|10|123|1494|88|90.31|147.20|30.91|0.00|2720.08|7947.28|12953.60|0.00|0.00|2720.08|2720.08|-5227.20| +2451067|63202|9370|85273|356877|6881|23732|10|34|1494|19|78.99|107.42|60.15|0.00|1142.85|1500.81|2040.98|91.42|0.00|1142.85|1234.27|-357.96| +2451067|63202|13286|85273|356877|6881|23732|10|30|1494|11|97.56|113.16|97.31|0.00|1070.41|1073.16|1244.76|96.33|0.00|1070.41|1166.74|-2.75| +2451067|63202|15692|85273|356877|6881|23732|10|150|1494|65|87.91|98.45|62.02|0.00|4031.30|5714.15|6399.25|241.87|0.00|4031.30|4273.17|-1682.85| +2451067|63202|11674|85273|356877|6881|23732|10|168|1494|61|27.66|41.21|36.67|0.00|2236.87|1687.26|2513.81|22.36|0.00|2236.87|2259.23|549.61| +2451067|63202|3518|85273|356877|||||1494|37|81.51||61.77||2285.49|3015.87|4312.35|68.56||2285.49||-730.38| +||6677||1733212|3510|9008||153|1495|17||||0.00||||0.00|0.00||2364.70|1032.07| +2452236|35610|12161|22114|1733212|3510|9008|4|271|1495|20|85.22|139.76|113.20|0.00|2264.00|1704.40|2795.20|203.76|0.00|2264.00|2467.76|559.60| +2452236|35610|8433|22114|1733212|3510|9008|4|94|1495|55|39.89|67.41|46.51|0.00|2558.05|2193.95|3707.55|230.22|0.00|2558.05|2788.27|364.10| +2452236|35610|10221|22114|1733212|3510|9008|4|261|1495|38|15.63|15.63|7.65|0.00|290.70|593.94|593.94|5.81|0.00|290.70|296.51|-303.24| +2452236|35610|10043|22114|1733212|3510|9008|4|29|1495|81|40.96|67.58|36.49|0.00|2955.69|3317.76|5473.98|29.55|0.00|2955.69|2985.24|-362.07| +2452236|35610|10923|22114|1733212|3510|9008|4|194|1495|35|28.40|53.39|5.33|182.81|186.55|994.00|1868.65|0.26|182.81|3.74|4.00|-990.26| +2452236|35610|1137|22114|1733212|3510|9008|4|190|1495|96|91.26|133.23|74.60|0.00|7161.60|8760.96|12790.08|429.69|0.00|7161.60|7591.29|-1599.36| +2452236|35610|6695|22114|1733212|3510|9008|4|258|1495|20|6.93|7.83|0.39|0.00|7.80|138.60|156.60|0.31|0.00|7.80|8.11|-130.80| +2452236|35610|229|22114|1733212|3510|9008|4|227|1495|58|42.99|57.60|43.77|0.00|2538.66|2493.42|3340.80|76.15|0.00|2538.66|2614.81|45.24| +2452168||8623|64372||||||1496||||72.67|0.00|||6460.08|575.54|0.00|6394.96|6970.50|| +2452168|63504|17963|64372|372042|2912|32582|7|155|1496|19|19.87|22.45|0.67|0.00|12.73|377.53|426.55|0.25|0.00|12.73|12.98|-364.80| +2452168|63504|15971|64372|372042|2912|32582|7|209|1496|48|5.66|8.20|4.42|27.58|212.16|271.68|393.60|3.69|27.58|184.58|188.27|-87.10| +2452168|63504|6409|64372|372042|2912|32582|7|297|1496|52|6.04|10.14|1.92|0.00|99.84|314.08|527.28|2.99|0.00|99.84|102.83|-214.24| +2452168|63504|13661|64372|372042|2912|32582|7|142|1496|11|13.23|24.34|5.84|0.00|64.24|145.53|267.74|5.13|0.00|64.24|69.37|-81.29| +2452168|63504|10763|64372|372042|2912|32582|7|129|1496|48|86.26|99.19|42.65|655.10|2047.20|4140.48|4761.12|69.60|655.10|1392.10|1461.70|-2748.38| +2452168|63504|10197|64372|372042|2912|32582|7|183|1496|26|97.79|121.25|66.68|0.00|1733.68|2542.54|3152.50|104.02|0.00|1733.68|1837.70|-808.86| +||7301||372042|2912|||63|1496||50.50|94.43|||||7932.12|||2696.40||-1545.60| +2452168|63504|15479|64372|372042|2912|32582|7|122|1496|61|80.60|147.49|28.02|0.00|1709.22|4916.60|8996.89|136.73|0.00|1709.22|1845.95|-3207.38| +2452168|63504|9649|64372|372042|2912|32582|7|241|1496|14|87.22|153.50|53.72|0.00|752.08|1221.08|2149.00|45.12|0.00|752.08|797.20|-469.00| +2450881|59306|5236|82593|88022|5451|31280|1|224|1497|31|79.32|137.22|93.30|0.00|2892.30|2458.92|4253.82|173.53|0.00|2892.30|3065.83|433.38| +2450881|59306|2|82593|88022|5451|31280|1|226|1497|28|58.94|71.90|59.67|0.00|1670.76|1650.32|2013.20|83.53|0.00|1670.76|1754.29|20.44| +2450881|59306|14884|82593|88022|5451|31280|1|200|1497|67|7.10|10.65|2.98|0.00|199.66|475.70|713.55|7.98|0.00|199.66|207.64|-276.04| +2450881|59306|5270|82593|88022|5451|31280|1|214|1497|61|3.78|5.36|4.34|0.00|264.74|230.58|326.96|10.58|0.00|264.74|275.32|34.16| +2450881|59306|122|82593|88022|5451|31280|1|127|1497|28|91.35|158.03|9.48|0.00|265.44|2557.80|4424.84|18.58|0.00|265.44|284.02|-2292.36| +2450881|59306|5108|82593|88022|5451|||194|1497|81|||0.00|0.00|||||0.00|||-5514.48| +2450881|59306|5893|82593|88022|5451|31280|1|54|1497|84|5.74|9.41|5.64|0.00|473.76|482.16|790.44|37.90|0.00|473.76|511.66|-8.40| +2450881|59306|15901|82593|88022|5451|31280|1|8|1497|21|8.36|8.36|2.59|42.96|54.39|175.56|175.56|0.57|42.96|11.43|12.00|-164.13| +2450881|59306|4417|82593|88022|5451|31280|1|228|1497|3|54.28|77.07|70.90|0.00|212.70|162.84|231.21|8.50|0.00|212.70|221.20|49.86| +2450881|59306|15736|82593|88022|5451|31280|1|125|1497|35|2.03|3.30|3.16|59.72|110.60|71.05|115.50|4.57|59.72|50.88|55.45|-20.17| +2450881|59306|11128|82593|88022|5451|31280|1|10|1497|12|8.05|9.25|1.57|0.00|18.84|96.60|111.00|1.69|0.00|18.84|20.53|-77.76| +2450881|59306|8092|82593|88022|5451|31280|1|144|1497|2|92.76|122.44|59.99|0.00|119.98|185.52|244.88|8.39|0.00|119.98|128.37|-65.54| +2450881|59306|10444|82593|88022|5451|31280|1|214|1497|60|75.42|104.07|2.08|0.00|124.80|4525.20|6244.20|7.48|0.00|124.80|132.28|-4400.40| +2450881|59306|11750|82593|88022|5451|31280|1|2|1497|59|34.52|37.62|7.90|0.00|466.10|2036.68|2219.58|9.32|0.00|466.10|475.42|-1570.58| +2450881|59306|8684|82593|88022|5451|31280|1|287|1497|78|10.24|14.23|6.54|86.72|510.12|798.72|1109.94|33.87|86.72|423.40|457.27|-375.32| +2450881|59306|16825|82593|88022|5451|31280|1|230|1497|65|44.15|67.54|24.31|0.00|1580.15|2869.75|4390.10|79.00|0.00|1580.15|1659.15|-1289.60| +2451782|48625|12341|78800|545473|146|4372|8|17|1498|40|63.92|92.68|74.14|0.00|2965.60|2556.80|3707.20|237.24|0.00|2965.60|3202.84|408.80| +2451782|48625|14149|78800|545473|146|4372|8|193|1498|60|83.32|114.14|54.78|0.00|3286.80|4999.20|6848.40|164.34|0.00|3286.80|3451.14|-1712.40| +2451782|48625|15803|78800|545473|146|4372|8|274|1498|77|7.27|8.28|5.38|0.00|414.26|559.79|637.56|20.71|0.00|414.26|434.97|-145.53| +2451782|48625|16490|78800|545473|146|4372|8|57|1498|59|4.24|5.68|5.39|0.00|318.01|250.16|335.12|28.62|0.00|318.01|346.63|67.85| +2451782|48625|13331|78800|545473|146|4372|8|105|1498|11|91.76|144.98|40.59|343.79|446.49|1009.36|1594.78|4.10|343.79|102.70|106.80|-906.66| +2451782|48625|16909|78800|545473|146|4372|8|106|1498|9|76.09|135.44|86.68|0.00|780.12|684.81|1218.96|0.00|0.00|780.12|780.12|95.31| +|48625|11707||545473|146||8||1498|||119.08||0.00|||||0.00||1199.94|-2047.92| +2451782|48625|13448|78800||146|||12|1498|||38.79||||1999.20|3258.36|106.84||||672.00| +2451782|48625|16837|78800|545473|146|4372|8|155|1498|89|91.47|158.24|66.46|3016.61|5914.94|8140.83|14083.36|144.91|3016.61|2898.33|3043.24|-5242.50| +|48625|7303|78800|545473||4372|8|46|1498|21|63.16||||855.33|1326.36||||855.33|915.20|| +2451782|48625|8449|78800|545473|146|4372|8|80|1498|29|61.52|115.04|81.67|0.00|2368.43|1784.08|3336.16|71.05|0.00|2368.43|2439.48|584.35| +2451782|48625|16429|78800|545473|146|4372|8|196|1498|84|35.11|44.23|30.96|26.00|2600.64|2949.24|3715.32|180.22|26.00|2574.64|2754.86|-374.60| +2451782|48625|5329|78800|545473|146|4372|8|170|1498|15|56.89|95.00|60.80|0.00|912.00|853.35|1425.00|18.24|0.00|912.00|930.24|58.65| +2452292|61578|4303|10113|411240|6320|3047|7|152|1499|10|72.52|126.91|82.49|0.00|824.90|725.20|1269.10|74.24|0.00|824.90|899.14|99.70| +2452292|61578|10749|10113|411240|6320|3047|7|130|1499|98|85.77|89.20|63.33|0.00|6206.34|8405.46|8741.60|496.50|0.00|6206.34|6702.84|-2199.12| +2452292|61578|2971|10113|411240|6320|3047|7|202|1499|9|70.39|135.85|4.07|0.00|36.63|633.51|1222.65|0.00|0.00|36.63|36.63|-596.88| +2452292|61578|11397|10113|411240|6320|3047|7|207|1499|89|59.45|117.71|62.38|0.00|5551.82|5291.05|10476.19|111.03|0.00|5551.82|5662.85|260.77| +2452292|61578|16177|10113|411240|6320|3047|7|221|1499|3|44.25|70.35|14.77|0.00|44.31|132.75|211.05|1.77|0.00|44.31|46.08|-88.44| +2452292|61578|13047|10113||6320||7||1499|||||||2933.48||0.00||1390.04||| +|61578|12126|10113|||3047|7||1499|||146.18||4911.60|||7309.00|110.51|4911.60|||-2925.10| +2452292|61578|1068|10113|411240|6320|3047|7|171|1499|96|1.29|1.99|0.57|0.00|54.72|123.84|191.04|0.00|0.00|54.72|54.72|-69.12| +2452292|61578|10818|10113|411240|6320|3047|7|297|1499|75|70.45|129.62|66.10|0.00|4957.50|5283.75|9721.50|396.60|0.00|4957.50|5354.10|-326.25| +2452292|61578|2175|10113|411240|6320|3047|7|16|1499|86|50.85|88.98|65.84|0.00|5662.24|4373.10|7652.28|169.86|0.00|5662.24|5832.10|1289.14| +2451505|50533|5792|67683|547292|2901|15710|8|23|1500|12|28.13|34.88|13.95|0.00|167.40|337.56|418.56|6.69|0.00|167.40|174.09|-170.16| +2451505|50533|9074|67683|547292|2901|15710|8|37|1500|69|3.83|6.62|2.25|0.00|155.25|264.27|456.78|7.76|0.00|155.25|163.01|-109.02| +2451505|50533|11846||547292|2901||8||1500|41||47.50||0.00|467.40|||42.06|0.00|467.40||| +2451505|50533|9662|67683|547292|2901|15710|8|161|1500|84|50.68|94.77|54.01|2404.52|4536.84|4257.12|7960.68|170.58|2404.52|2132.32|2302.90|-2124.80| +2451505|50533|493|67683|547292|2901|15710|8|60|1500|51|11.42|19.87|9.33|0.00|475.83|582.42|1013.37|42.82|0.00|475.83|518.65|-106.59| +2451505|50533|11899|67683|547292|2901|15710|8|142|1500|9|60.31|83.83|43.59|0.00|392.31|542.79|754.47|3.92|0.00|392.31|396.23|-150.48| +||17275||547292||||45|1500|76||51.72|1.03||||3930.72|7.04||78.28|85.32|-2877.36| +2451505|50533|6448|67683|547292|2901|15710|8|72|1500|5|15.06|20.93|14.65|0.00|73.25|75.30|104.65|2.19|0.00|73.25|75.44|-2.05| +2451505|50533|13048|67683|547292|2901|15710|8|176|1500|19|84.60|87.98|39.59|0.00|752.21|1607.40|1671.62|30.08|0.00|752.21|782.29|-855.19| +2451505|50533|7400|67683|547292|2901|15710|8|231|1500|58|86.89|132.94|93.05|0.00|5396.90|5039.62|7710.52|0.00|0.00|5396.90|5396.90|357.28| +2451505|50533|17683|67683|547292|2901|15710|8|10|1500|56|54.16|96.94|74.64|4054.44|4179.84|3032.96|5428.64|8.77|4054.44|125.40|134.17|-2907.56| +2451505|50533|4678|67683|547292|2901|15710|8|230|1500|93|59.71|103.89|83.11|0.00|7729.23|5553.03|9661.77|154.58|0.00|7729.23|7883.81|2176.20| +2451505|50533|6811|67683|547292|2901|15710|8|102|1500|54|83.44|127.66|44.68|482.54|2412.72|4505.76|6893.64|96.50|482.54|1930.18|2026.68|-2575.58| +2451505|50533|13177|67683|547292|2901|15710|8|114|1500|5|93.12|128.50|96.37|0.00|481.85|465.60|642.50|33.72|0.00|481.85|515.57|16.25| +2452612|53614|2955|94612|969213|6661|31709|8|96|1501|87|70.80|137.35|70.04|0.00|6093.48|6159.60|11949.45|304.67|0.00|6093.48|6398.15|-66.12| +2452612|53614|7134|94612|969213|6661|31709|8|56|1501|51|75.35|123.57|59.31|0.00|3024.81|3842.85|6302.07|90.74|0.00|3024.81|3115.55|-818.04| +2452612|53614|11647|94612|969213|6661|31709|8|148|1501|51|91.49|124.42|16.17|222.66|824.67|4665.99|6345.42|18.06|222.66|602.01|620.07|-4063.98| +2452612|53614|15637|94612|969213|6661|31709|8|25|1501|92|94.43|166.19|16.61|0.00|1528.12|8687.56|15289.48|45.84|0.00|1528.12|1573.96|-7159.44| +2452612|53614|853|94612|969213|6661|31709|8|232|1501|98|13.54|14.35|1.43|0.00|140.14|1326.92|1406.30|9.80|0.00|140.14|149.94|-1186.78| +2452612|53614|2041|94612|969213|6661|31709|8|275|1501|66|30.66|50.28|13.07|0.00|862.62|2023.56|3318.48|51.75|0.00|862.62|914.37|-1160.94| +2452612|53614|5143|94612|969213|6661|31709|8|129|1501|79|72.91|97.69|9.76|0.00|771.04|5759.89|7717.51|38.55|0.00|771.04|809.59|-4988.85| +2452612|53614|6267|94612|969213|6661|31709|8|89|1501|75|95.00|156.75|53.29|0.00|3996.75|7125.00|11756.25|239.80|0.00|3996.75|4236.55|-3128.25| +2452612|53614|14481|94612|969213|6661|31709|8|211|1501|72|99.19|153.74|113.76|0.00|8190.72|7141.68|11069.28|573.35|0.00|8190.72|8764.07|1049.04| +2452612|53614|3276|94612|969213|6661|31709|8|192|1501|3|81.36|159.46|87.70|0.00|263.10|244.08|478.38|21.04|0.00|263.10|284.14|19.02| +2452612|53614|14418|94612|969213|6661|31709|8|93|1501|98|45.97|78.60|3.93|0.00|385.14|4505.06|7702.80|34.66|0.00|385.14|419.80|-4119.92| +2452612|53614|10371|94612|969213|6661|31709|8|14|1501|10|41.58|74.84|47.89|0.00|478.90|415.80|748.40|9.57|0.00|478.90|488.47|63.10| +2452612|53614|12159|94612|969213|6661|31709|8|4|1501|67|32.86|43.04|6.45|17.28|432.15|2201.62|2883.68|33.18|17.28|414.87|448.05|-1786.75| +2451056|50649|8932|28055|667109|1716|48370|1|137|1502|34|76.06|141.47|18.39|0.00|625.26|2586.04|4809.98|12.50|0.00|625.26|637.76|-1960.78| +2451056|50649|8938|28055|667109|1716|48370|1|160|1502|13|18.82|19.76|16.79|0.00|218.27|244.66|256.88|2.18|0.00|218.27|220.45|-26.39| +2451056|50649|3626|28055|667109|1716|48370|1|165|1502|14|91.42|153.58|52.21|0.00|730.94|1279.88|2150.12|65.78|0.00|730.94|796.72|-548.94| +2451056|50649|8444|28055|667109|1716|48370|1|1|1502|13|35.91|56.73|13.61|0.00|176.93|466.83|737.49|8.84|0.00|176.93|185.77|-289.90| +2451056|50649|4826|28055|667109|1716|48370|1|103|1502|61|72.02|77.06|47.77|2185.47|2913.97|4393.22|4700.66|65.56|2185.47|728.50|794.06|-3664.72| +2451056|50649|7915|28055|667109|1716|48370|1|134|1502|34|45.70|71.29|39.92|0.00|1357.28|1553.80|2423.86|108.58|0.00|1357.28|1465.86|-196.52| +||6086|28055|667109|||1||1502|67|39.22||33.79||||3652.17|||2263.93|2445.04|-363.81| +2451056|50649|2107|28055|667109|1716|48370|1|189|1502|90|96.07|128.73|86.24|5665.96|7761.60|8646.30|11585.70|62.86|5665.96|2095.64|2158.50|-6550.66| +2451056|50649|16958|28055|667109|1716|48370|1|49|1502|25|71.81|77.55|65.91|0.00|1647.75|1795.25|1938.75|16.47|0.00|1647.75|1664.22|-147.50| +2451056|50649|520|28055|667109|1716|48370|1|48|1502|75|79.59|140.07|57.42|0.00|4306.50|5969.25|10505.25|0.00|0.00|4306.50|4306.50|-1662.75| +2451056|50649|5758|28055|667109|1716|48370|1|242|1502|14|79.53|93.05|87.46|0.00|1224.44|1113.42|1302.70|85.71|0.00|1224.44|1310.15|111.02| +2451056|50649|8912|28055|667109|1716|48370|1|185|1502|61|57.38|80.33|79.52|4850.72|4850.72|3500.18|4900.13|0.00|4850.72|0.00|0.00|-3500.18| +2451397|53601|12710|93780|1192036|1492|12948|7|259|1503|99|85.78|96.07|15.37|0.00|1521.63|8492.22|9510.93|30.43|0.00|1521.63|1552.06|-6970.59| +2451397|53601|718|93780|1192036|1492|12948|7|278|1503|82|58.33|72.91|15.31|0.00|1255.42|4783.06|5978.62|0.00|0.00|1255.42|1255.42|-3527.64| +2451397|53601|14122|93780|1192036|1492|12948|7|68|1503|59|15.40|16.32|6.36|0.00|375.24|908.60|962.88|15.00|0.00|375.24|390.24|-533.36| +|53601|200|||1492||7||1503|18|82.90|||||||17.39||||| +2451397|53601|9710|93780|1192036|1492|12948|7|288|1503|74|58.72|78.68|3.93|0.00|290.82|4345.28|5822.32|5.81|0.00|290.82|296.63|-4054.46| +2451397|53601|6220|93780|1192036|1492|12948|7|182|1503|30|37.61|65.81|55.93|0.00|1677.90|1128.30|1974.30|67.11|0.00|1677.90|1745.01|549.60| +2451397|53601|7784|93780|1192036|1492|12948|7|146|1503|71|49.36|62.19|59.08|0.00|4194.68|3504.56|4415.49|251.68|0.00|4194.68|4446.36|690.12| +2451397|53601|10441|93780|1192036|1492|12948|7|124|1503|36|82.23|99.49|44.77|0.00|1611.72|2960.28|3581.64|112.82|0.00|1611.72|1724.54|-1348.56| +2451397|53601|2164|93780|1192036|1492|12948|7|272|1503|2|10.65|14.37|1.29|0.00|2.58|21.30|28.74|0.18|0.00|2.58|2.76|-18.72| +|53601|757|93780|1192036|1492||||1503|51|15.07||||149.94|||7.49||149.94|157.43|-618.63| +2451397|53601|8870|93780|1192036|1492|12948|7|204|1503|7|21.86|24.48|1.22|0.00|8.54|153.02|171.36|0.68|0.00|8.54|9.22|-144.48| +2452147|65235|2637|55450|1857854|258|29062|7|58|1504|80|6.20|8.30|4.56|0.00|364.80|496.00|664.00|0.00|0.00|364.80|364.80|-131.20| +2452147|65235|11159|55450|1857854|258|29062|7|156|1504|46|91.43|179.20|98.56|0.00|4533.76|4205.78|8243.20|408.03|0.00|4533.76|4941.79|327.98| +2452147|65235|17909|55450|1857854|258|29062|7|74|1504|39|77.68|98.65|76.94|0.00|3000.66|3029.52|3847.35|120.02|0.00|3000.66|3120.68|-28.86| +2452147|65235|15413|55450|1857854|258|29062|7|183|1504|20|41.67|60.00|11.40|0.00|228.00|833.40|1200.00|0.00|0.00|228.00|228.00|-605.40| +2452147|65235|5413|55450|1857854|258|29062|7|123|1504|67|90.10|130.64|62.70|0.00|4200.90|6036.70|8752.88|168.03|0.00|4200.90|4368.93|-1835.80| +2452147|65235|6673|55450|1857854|258|29062|7|175|1504|68|60.33|95.32|52.42|1532.76|3564.56|4102.44|6481.76|121.90|1532.76|2031.80|2153.70|-2070.64| +2452147|65235|14679|55450|1857854|258|29062|7|275|1504|64|34.71|41.99|24.77|0.00|1585.28|2221.44|2687.36|0.00|0.00|1585.28|1585.28|-636.16| +2452147|65235|13637|55450|1857854|258|29062|7|188|1504|79|25.75|43.26|9.51|0.00|751.29|2034.25|3417.54|15.02|0.00|751.29|766.31|-1282.96| +2452147|65235|17987|55450|1857854|258|29062|7|38|1504|14|82.93|86.24|48.29|0.00|676.06|1161.02|1207.36|33.80|0.00|676.06|709.86|-484.96| +2451183|62640|5131|75376|45705|2881|16010|2|45|1505|71|77.84|94.96|8.54|121.26|606.34|5526.64|6742.16|29.10|121.26|485.08|514.18|-5041.56| +2451183|62640|9914|75376|45705|2881|16010|2|163|1505|64|58.18|75.63|14.36|0.00|919.04|3723.52|4840.32|0.00|0.00|919.04|919.04|-2804.48| +2451183|62640|14936|75376|45705|2881|16010|2|57|1505|27|62.84|69.12|58.75|0.00|1586.25|1696.68|1866.24|63.45|0.00|1586.25|1649.70|-110.43| +2451183|62640|9934|75376|45705|2881|16010|2|263|1505|69|64.90|101.24|57.70|0.00|3981.30|4478.10|6985.56|199.06|0.00|3981.30|4180.36|-496.80| +2451183|62640|9835|75376|45705|2881|16010|2|73|1505|27|17.95|27.10|23.84|0.00|643.68|484.65|731.70|0.00|0.00|643.68|643.68|159.03| +2451183|62640|1558|||2881|16010|2|124|1505||50.59||18.08||1428.32|||110.55||1228.36|1338.91|| +2451183|62640|12494|75376|45705|2881|16010|2|123|1505|64|35.11|60.03|39.01|0.00|2496.64|2247.04|3841.92|99.86|0.00|2496.64|2596.50|249.60| +2451183|62640|10618|75376|45705|2881|16010|2|282|1505|58|57.19|59.47|46.38|0.00|2690.04|3317.02|3449.26|80.70|0.00|2690.04|2770.74|-626.98| +2451183|62640|12061|75376|45705|2881|16010|2|205|1505|44|12.82|16.66|8.99|0.00|395.56|564.08|733.04|31.64|0.00|395.56|427.20|-168.52| +2451183|62640|13564|75376|45705|2881|16010|2|205|1505|92|68.05|131.33|91.93|0.00|8457.56|6260.60|12082.36|338.30|0.00|8457.56|8795.86|2196.96| +2451944|43963|10549|2921|284969|2217|23865|1|300|1506|96|54.96|108.82|38.08|0.00|3655.68|5276.16|10446.72|292.45|0.00|3655.68|3948.13|-1620.48| +2451944|43963|11799|2921|284969|2217|23865|1|237|1506|42|46.72|66.80|2.67|0.00|112.14|1962.24|2805.60|5.60|0.00|112.14|117.74|-1850.10| +2451944|43963|2607|2921|284969|2217|23865|1|245|1506|59|35.30|43.41|42.54|602.36|2509.86|2082.70|2561.19|57.22|602.36|1907.50|1964.72|-175.20| +2451944|43963|3449|2921|284969|2217|23865|1|270|1506|4|34.20|49.24|5.41|11.68|21.64|136.80|196.96|0.29|11.68|9.96|10.25|-126.84| +2451944|43963|7531|2921|284969|2217|23865|1|247|1506|41|4.00|6.20|0.43|0.00|17.63|164.00|254.20|0.70|0.00|17.63|18.33|-146.37| +2451944|43963|9561|2921|284969|2217|23865|1|289|1506|98|24.01|47.77|18.63|0.00|1825.74|2352.98|4681.46|54.77|0.00|1825.74|1880.51|-527.24| +2451944|43963|15405|2921|284969|2217|23865|1|87|1506|100|9.08|10.16|3.96|0.00|396.00|908.00|1016.00|3.96|0.00|396.00|399.96|-512.00| +2451944|43963|9885|2921|284969|2217|23865|1|265|1506|18|13.13|25.34|24.83|0.00|446.94|236.34|456.12|8.93|0.00|446.94|455.87|210.60| +||5105|2921||2217|23865|||1506|24||114.11|5.70|||2321.04|2738.64|||136.80|147.74|-2184.24| +2451944||6065|||2217|23865|1|51|1506|52|55.99|61.58|48.64||2529.28|2911.48||50.58||2529.28|2579.86|| +2450958|32823|9349|49542|1332635|3728|11668|1|58|1507|82|72.61|131.42|72.28|0.00|5926.96|5954.02|10776.44|59.26|0.00|5926.96|5986.22|-27.06| +2450958|32823|13657|49542|1332635|3728|11668|1|61|1507|98|35.26|41.95|21.39|1069.07|2096.22|3455.48|4111.10|20.54|1069.07|1027.15|1047.69|-2428.33| +2450958|32823|5252|49542|1332635|3728|11668|1|178|1507|55|37.75|61.15|26.90|0.00|1479.50|2076.25|3363.25|103.56|0.00|1479.50|1583.06|-596.75| +2450958|32823|5575|49542|1332635|3728|11668|1|107|1507|14|99.63|187.30|80.53|0.00|1127.42|1394.82|2622.20|67.64|0.00|1127.42|1195.06|-267.40| +2450958|32823|4753|49542|1332635|3728|11668|1|240|1507|77|6.06|7.21|2.81|0.00|216.37|466.62|555.17|6.49|0.00|216.37|222.86|-250.25| +2450958|32823|17812|49542|1332635|3728|11668|1|43|1507|50|37.78|69.51|53.52|535.20|2676.00|1889.00|3475.50|21.40|535.20|2140.80|2162.20|251.80| +2450958|32823|16981|49542|1332635|3728|11668|1|26|1507|89|57.62|90.46|79.60|7084.40|7084.40|5128.18|8050.94|0.00|7084.40|0.00|0.00|-5128.18| +2450958|32823|5660|49542|1332635|3728|11668|1|282|1507|66|60.50|64.13|55.15|0.00|3639.90|3993.00|4232.58|109.19|0.00|3639.90|3749.09|-353.10| +2450958|32823|4732|49542|1332635|3728|11668|1|79|1507|81|28.34|47.61|33.32|0.00|2698.92|2295.54|3856.41|161.93|0.00|2698.92|2860.85|403.38| +|31886|6044||754113|2445||||1508||52.68|62.16|60.29||5004.07|4372.44|||||5204.23|| +2451862|31886|11209|91028|754113|2445|21130|7|116|1508|75|84.86|117.95|24.76|0.00|1857.00|6364.50|8846.25|148.56|0.00|1857.00|2005.56|-4507.50| +2451862|31886|6500|91028|754113|2445|21130|7|81|1508|11|50.58|54.12|10.28|0.00|113.08|556.38|595.32|5.65|0.00|113.08|118.73|-443.30| +2451862|31886|3305|91028|754113|2445|21130|7|241|1508|51|41.66|64.98|20.14|0.00|1027.14|2124.66|3313.98|20.54|0.00|1027.14|1047.68|-1097.52| +2451862|31886|7577|91028|754113|2445|21130|7|95|1508|48|72.61|108.91|96.92|0.00|4652.16|3485.28|5227.68|93.04|0.00|4652.16|4745.20|1166.88| +2451862|31886|14996|91028|754113|2445|21130|7|107|1508|51|26.58|29.76|15.47|0.00|788.97|1355.58|1517.76|47.33|0.00|788.97|836.30|-566.61| +2451862|31886|901|91028|754113|2445|21130|7|211|1508|24|11.76|18.34|14.12|0.00|338.88|282.24|440.16|10.16|0.00|338.88|349.04|56.64| +2451862|31886|12707|91028|754113|2445|21130|7|1|1508|76|39.36|77.53|19.38|530.23|1472.88|2991.36|5892.28|65.98|530.23|942.65|1008.63|-2048.71| +2451862|31886|5965|91028|754113|2445|21130|7|52|1508|90|70.20|116.53|18.64|0.00|1677.60|6318.00|10487.70|0.00|0.00|1677.60|1677.60|-4640.40| +2451862|31886|8611|91028|754113|2445|21130|7|177|1508|97|2.90|4.32|4.06|39.38|393.82|281.30|419.04|28.35|39.38|354.44|382.79|73.14| +2451862|31886|3857|91028|754113|2445|21130|7|257|1508|39|96.66|162.38|126.65|0.00|4939.35|3769.74|6332.82|49.39|0.00|4939.35|4988.74|1169.61| +2452412|50555|17823|33583|1263578|3173|17921|2|77|1509|9|80.91|147.25|141.36|0.00|1272.24|728.19|1325.25|25.44|0.00|1272.24|1297.68|544.05| +|50555|8139||1263578||17921|||1509|37|94.00|||68.61|||||68.61||44.30|| +2452412|50555|14029|33583|1263578|3173|17921|2|243|1509|29|1.73|2.64|0.87|0.00|25.23|50.17|76.56|0.75|0.00|25.23|25.98|-24.94| +2452412|50555|12081|33583|1263578|3173|17921|2|171|1509|98|22.60|32.09|17.32|0.00|1697.36|2214.80|3144.82|50.92|0.00|1697.36|1748.28|-517.44| +2452412|50555|11790|33583|1263578|3173|17921|2|120|1509|52|71.04|138.52|31.85|0.00|1656.20|3694.08|7203.04|16.56|0.00|1656.20|1672.76|-2037.88| +||16021||1263578|3173|||10|1509|98|95.17|||6301.00|8290.80|9326.66|11844.28|179.08|6301.00|1989.80|2168.88|-7336.86| +2452412|50555|543|33583|1263578|3173|17921|2|84|1509|22|74.13|134.91|35.07|532.36|771.54|1630.86|2968.02|0.00|532.36|239.18|239.18|-1391.68| +2452412|50555|17193|33583|1263578|3173|17921|2|278|1509|89|78.13|142.97|135.82|846.15|12087.98|6953.57|12724.33|786.92|846.15|11241.83|12028.75|4288.26| +2452412|50555|9486|33583|1263578|3173|17921|2|78|1509|61|7.06|13.20|11.88|0.00|724.68|430.66|805.20|36.23|0.00|724.68|760.91|294.02| +2452412|50555|961|33583|1263578|3173|17921|2|87|1509|39|72.22|106.16|50.95|1649.25|1987.05|2816.58|4140.24|0.00|1649.25|337.80|337.80|-2478.78| +2451171|53406|3433|58774|1421176|3931|48733|10|148|1510|86|93.66|128.31|119.32|0.00|10261.52|8054.76|11034.66|718.30|0.00|10261.52|10979.82|2206.76| +2451171|53406|754|58774|1421176|3931|48733|10|237|1510|31|7.02|10.17|6.81|0.00|211.11|217.62|315.27|0.00|0.00|211.11|211.11|-6.51| +2451171|53406|1424|58774|1421176|3931|48733|10|239|1510|37|82.92|138.47|88.62|0.00|3278.94|3068.04|5123.39|229.52|0.00|3278.94|3508.46|210.90| +|53406|1510||||||194|1510||39.14|||0.00||469.68||10.07|0.00|503.64||| +2451171|53406|13802|58774|1421176|3931|48733|10|92|1510|11|9.96|18.82|15.62|0.00|171.82|109.56|207.02|8.59|0.00|171.82|180.41|62.26| +||12568|58774|1421176|3931||10||1510||46.71|86.88||3034.43|3612.42|1961.82|||3034.43||595.32|| +2451171|53406|17072|58774|1421176|3931|48733|10|64|1510|69|89.18|147.14|4.41|0.00|304.29|6153.42|10152.66|12.17|0.00|304.29|316.46|-5849.13| +2451171|53406|10732|58774|1421176|3931|48733|10|294|1510|53|38.69|73.12|25.59|637.44|1356.27|2050.57|3875.36|0.00|637.44|718.83|718.83|-1331.74| +2451171|53406|13456|58774|1421176|3931|48733|10|3|1510|92|66.53|101.79|50.89|0.00|4681.88|6120.76|9364.68|187.27|0.00|4681.88|4869.15|-1438.88| +2451171|53406|7448|58774|1421176|3931|48733|10|131|1510|95|93.42|151.34|12.10|0.00|1149.50|8874.90|14377.30|11.49|0.00|1149.50|1160.99|-7725.40| +2451171|53406|2713|58774|1421176|3931|48733|10|269|1510|26|26.44|41.24|7.01|0.00|182.26|687.44|1072.24|1.82|0.00|182.26|184.08|-505.18| +2451171|53406|1399|58774|1421176|3931|48733|10|204|1510|26|18.19|32.01|19.84|0.00|515.84|472.94|832.26|36.10|0.00|515.84|551.94|42.90| +2451171|53406|7339|58774|1421176|3931|48733|10|296|1510|97|88.67|169.35|133.78|0.00|12976.66|8600.99|16426.95|648.83|0.00|12976.66|13625.49|4375.67| +2451171|53406|8686|58774|1421176|3931|48733|10|3|1510|29|86.94|173.88|36.51|0.00|1058.79|2521.26|5042.52|74.11|0.00|1058.79|1132.90|-1462.47| +2451171|53406|12439|58774|1421176|3931|48733|10|235|1510|3|47.93|57.99|10.43|0.00|31.29|143.79|173.97|2.50|0.00|31.29|33.79|-112.50| +|36943|15181||1772829||46135|1|7|1511||58.90|73.03|21.90||1839.60||6134.52|||||-3108.00| +2451504|36943|12242|67623|1772829|82|46135|1|12|1511|68|72.52|76.14|26.64|0.00|1811.52|4931.36|5177.52|144.92|0.00|1811.52|1956.44|-3119.84| +2451504|36943|9577|67623|1772829|82|46135|1|268|1511|43|18.07|26.20|14.93|0.00|641.99|777.01|1126.60|44.93|0.00|641.99|686.92|-135.02| +2451504|36943|5030|67623|1772829|82|46135|1|11|1511|85|98.76|159.99|43.19|0.00|3671.15|8394.60|13599.15|220.26|0.00|3671.15|3891.41|-4723.45| +2451504|36943|550|67623|1772829|82|46135|1|245|1511|78|33.63|44.39|22.63|0.00|1765.14|2623.14|3462.42|35.30|0.00|1765.14|1800.44|-858.00| +2451504|36943|7324|67623|1772829|82|46135|1|283|1511|96|39.01|40.18|14.46|0.00|1388.16|3744.96|3857.28|27.76|0.00|1388.16|1415.92|-2356.80| +2451504|36943|11866|67623|1772829|82|46135|1|151|1511|9|68.72|94.83|15.17|0.00|136.53|618.48|853.47|4.09|0.00|136.53|140.62|-481.95| +2451504|36943|2893|67623|1772829|82|46135|1|289|1511|44|12.72|16.40|11.97|0.00|526.68|559.68|721.60|10.53|0.00|526.68|537.21|-33.00| +2451504|36943|7484|67623|1772829|82|46135|1|109|1511|6|15.32|18.23|14.58|0.00|87.48|91.92|109.38|0.00|0.00|87.48|87.48|-4.44| +2451605|36565|15248|93974|684959|2344|18529|7|239|1512|1|85.60|134.39|10.75|10.32|10.75|85.60|134.39|0.00|10.32|0.43|0.43|-85.17| +2451605|36565|5579|93974|684959|2344|18529|7|224|1512|2|83.89|113.25|69.08|0.00|138.16|167.78|226.50|4.14|0.00|138.16|142.30|-29.62| +2451605|36565|13669|93974|684959|2344|18529|7|124|1512|23|84.83|95.85|58.46|0.00|1344.58|1951.09|2204.55|26.89|0.00|1344.58|1371.47|-606.51| +2451605|36565|7441|93974|684959|2344|18529|7|245|1512|44|46.30|72.22|64.27|0.00|2827.88|2037.20|3177.68|0.00|0.00|2827.88|2827.88|790.68| +2451605|36565|7598|93974|684959|2344|18529|7|243|1512|15|85.77|114.07|114.07|0.00|1711.05|1286.55|1711.05|51.33|0.00|1711.05|1762.38|424.50| +2451605|36565|806|93974|684959|2344|18529|7|91|1512|13|3.96|5.97|2.92|3.79|37.96|51.48|77.61|0.00|3.79|34.17|34.17|-17.31| +2451605|36565|6716|93974|684959|2344|18529|7|286|1512|89|94.66|150.50|3.01|0.00|267.89|8424.74|13394.50|0.00|0.00|267.89|267.89|-8156.85| +2451605|36565|10363|93974|684959|2344|18529|7|100|1512|91|8.47|14.82|14.67|0.00|1334.97|770.77|1348.62|0.00|0.00|1334.97|1334.97|564.20| +2451605|36565|917|93974|684959|2344|18529|7|110|1512|83|61.38|81.63|44.08|0.00|3658.64|5094.54|6775.29|329.27|0.00|3658.64|3987.91|-1435.90| +||11438|||||7||1512|82||||0.00|||4636.28||0.00||3198.82|637.14| +2451605|36565|13991|93974|684959|2344|18529|7|162|1512|99|58.50|84.82|67.00|0.00|6633.00|5791.50|8397.18|265.32|0.00|6633.00|6898.32|841.50| +2451605|36565|3551|93974|684959|2344|18529|7|183|1512|49|91.88|97.39|33.11|859.86|1622.39|4502.12|4772.11|45.75|859.86|762.53|808.28|-3739.59| +2451605|36565|301|93974|684959|2344|18529|7|244|1512|60|38.92|48.65|1.94|0.00|116.40|2335.20|2919.00|6.98|0.00|116.40|123.38|-2218.80| +2451605|36565|9361|93974|684959|2344|18529|7|53|1512|67|37.17|67.27|33.63|0.00|2253.21|2490.39|4507.09|180.25|0.00|2253.21|2433.46|-237.18| +2451241|63280|1178|85742|996100|6105|19263|8|81|1513|46|30.00|48.30|21.25|0.00|977.50|1380.00|2221.80|29.32|0.00|977.50|1006.82|-402.50| +2451241|63280|10528|85742|996100|6105|19263|8|167|1513|31|58.51|74.89|49.42|0.00|1532.02|1813.81|2321.59|61.28|0.00|1532.02|1593.30|-281.79| +2451241|63280|2815|85742|996100|6105|19263|8|161|1513|65|25.22|26.98|9.71|0.00|631.15|1639.30|1753.70|37.86|0.00|631.15|669.01|-1008.15| +2451241|63280|5156|85742|996100|6105|19263|8|76|1513|90|76.18|128.74|92.69|0.00|8342.10|6856.20|11586.60|417.10|0.00|8342.10|8759.20|1485.90| +|63280|14564|85742|||19263||87|1513|85|37.49||4.95|0.00|420.75|3186.65|6022.25|8.41|0.00||429.16|-2765.90| +2451241|63280|15248|85742|996100|6105|19263|8|114|1513|56|82.47|144.32|0.00|0.00|0.00|4618.32|8081.92|0.00|0.00|0.00|0.00|-4618.32| +2451241|63280|5578|85742|996100|6105|19263|8|227|1513|27|70.16|128.39|71.89|970.51|1941.03|1894.32|3466.53|29.11|970.51|970.52|999.63|-923.80| +2451241|63280|13669|85742|996100|6105|19263|8|205|1513|45|87.90|159.97|71.98|0.00|3239.10|3955.50|7198.65|129.56|0.00|3239.10|3368.66|-716.40| +2451241|63280|7441|85742|996100|6105|19263|8|11|1513|13|8.97|15.42|3.54|10.12|46.02|116.61|200.46|2.87|10.12|35.90|38.77|-80.71| +2451241|63280|7598|85742|996100|6105|19263|8|225|1513|13|17.06|18.08|10.48|0.00|136.24|221.78|235.04|0.00|0.00|136.24|136.24|-85.54| +2451241|63280|806|85742|996100|6105|19263|8|290|1513|73|70.24|130.64|58.78|0.00|4290.94|5127.52|9536.72|42.90|0.00|4290.94|4333.84|-836.58| +2451241|63280|6716|85742|996100|6105|19263|8|290|1513|35|17.11|30.28|26.04|0.00|911.40|598.85|1059.80|36.45|0.00|911.40|947.85|312.55| +2451241|63280|10363|85742|996100|6105|19263|8|99|1513|4|6.00|6.96|1.53|0.00|6.12|24.00|27.84|0.36|0.00|6.12|6.48|-17.88| +2451241|63280|916|85742|996100|6105|19263|8|205|1513|17|55.55|71.10|66.12|0.00|1124.04|944.35|1208.70|67.44|0.00|1124.04|1191.48|179.69| +2451241|63280|11438|85742|996100|6105|19263|8|273|1513|32|84.49|90.40|13.56|351.47|433.92|2703.68|2892.80|0.00|351.47|82.45|82.45|-2621.23| +2451177|42204|16606|62762|638288|1461|16587|10|236|1514|80|60.60|62.41|33.70|0.00|2696.00|4848.00|4992.80|134.80|0.00|2696.00|2830.80|-2152.00| +|42204|11404|62762||||10||1514||95.67|140.63||0.00|4408.25|5261.85|7734.65|0.00|0.00|4408.25||-853.60| +2451177|42204|12416|62762|638288|1461|16587|10|93|1514|25|49.38|49.87|48.87|0.00|1221.75|1234.50|1246.75|48.87|0.00|1221.75|1270.62|-12.75| +2451177|42204|8041|62762|638288|1461|16587|10|290|1514|92|71.16|73.29|27.11|2394.35|2494.12|6546.72|6742.68|2.99|2394.35|99.77|102.76|-6446.95| +2451177|42204|14764|62762|638288|1461|16587|10|115|1514|49|34.97|57.35|0.00|0.00|0.00|1713.53|2810.15|0.00|0.00|0.00|0.00|-1713.53| +2451177|42204|6937|62762|638288|1461|16587|10|67|1514|95|16.17|21.02|15.34|0.00|1457.30|1536.15|1996.90|58.29|0.00|1457.30|1515.59|-78.85| +2451177|42204|5902|62762|638288|1461|16587|10|268|1514|12|16.16|16.16|15.51|0.00|186.12|193.92|193.92|0.00|0.00|186.12|186.12|-7.80| +2451177|42204|15724|62762|638288|1461|16587|10|270|1514|58|27.59|30.62|16.84|546.96|976.72|1600.22|1775.96|21.48|546.96|429.76|451.24|-1170.46| +2451177|42204|12769|62762|638288|1461|16587|10|183|1514|3|62.16|63.40|56.42|165.87|169.26|186.48|190.20|0.27|165.87|3.39|3.66|-183.09| +2451177|42204|2000|62762|638288|1461|16587|10|99|1514|81|97.57|187.33|161.10|0.00|13049.10|7903.17|15173.73|652.45|0.00|13049.10|13701.55|5145.93| +2451177|42204|1664|62762|638288|1461|16587|10|250|1514|71|4.64|5.75|4.60|0.00|326.60|329.44|408.25|16.33|0.00|326.60|342.93|-2.84| +2451177|42204|4490|62762|638288|1461|16587|10|166|1514|50|32.66|43.43|13.46|0.00|673.00|1633.00|2171.50|20.19|0.00|673.00|693.19|-960.00| +2451177|42204|16364|62762|638288|1461|16587|10|51|1514|64|48.59|68.51|32.19|0.00|2060.16|3109.76|4384.64|144.21|0.00|2060.16|2204.37|-1049.60| +2451177|42204|12314|62762|638288|1461|16587|10|111|1514|2|58.00|64.38|23.82|0.00|47.64|116.00|128.76|4.28|0.00|47.64|51.92|-68.36| +2451177|42204|8131|62762|638288|1461|16587|10|145|1514|6|19.83|28.95|5.79|0.00|34.74|118.98|173.70|2.43|0.00|34.74|37.17|-84.24| +2451177|42204|9596|62762|638288|1461|16587|10|82|1514|18|57.04|100.96|56.53|681.75|1017.54|1026.72|1817.28|10.07|681.75|335.79|345.86|-690.93| +2451803|53076|3091|55579|551761|4390|16572|7|68|1515|22|85.75|98.61|96.63|0.00|2125.86|1886.50|2169.42|85.03|0.00|2125.86|2210.89|239.36| +2451803|53076|422|55579|551761|4390|16572|7|251|1515|79|66.56|87.19|14.82|0.00|1170.78|5258.24|6888.01|46.83|0.00|1170.78|1217.61|-4087.46| +2451803|53076|5029|55579|551761|4390|16572|7|183|1515|17|27.15|31.76|6.98|0.00|118.66|461.55|539.92|2.37|0.00|118.66|121.03|-342.89| +||17705|55579|551761|||||1515|74|86.37|95.87||||||46.79||||| +2451803|53076|11234|55579|551761|4390|16572|7|38|1515|44|40.32|40.32|15.72|0.00|691.68|1774.08|1774.08|34.58|0.00|691.68|726.26|-1082.40| +2451803|53076|17483|55579|551761|4390|16572|7|64|1515|52|2.10|2.49|0.24|0.00|12.48|109.20|129.48|0.87|0.00|12.48|13.35|-96.72| +2451803|53076|8384|55579|551761|4390|16572|7|73|1515|73|47.51|76.01|13.68|0.00|998.64|3468.23|5548.73|9.98|0.00|998.64|1008.62|-2469.59| +2451803|53076|6565|55579||||7||1515||11.19|13.31|0.26|||839.25||0.12||||| +2451803|53076|5059|55579|551761||16572|||1515||34.57|56.34|15.77||126.16|276.56|450.72|8.83||126.16||| +2451803|53076|4687|55579|551761|4390|16572|7|44|1515|5|2.62|3.85|3.11|0.00|15.55|13.10|19.25|1.39|0.00|15.55|16.94|2.45| +2451803|53076|9311|55579|551761|4390|16572|7|80|1515|85|84.19|125.44|94.08|0.00|7996.80|7156.15|10662.40|719.71|0.00|7996.80|8716.51|840.65| +2451803|53076|13199|55579|551761|4390|16572|7|67|1515|38|69.53|103.59|56.97|2078.26|2164.86|2642.14|3936.42|3.46|2078.26|86.60|90.06|-2555.54| +2451803|53076|10133|55579|551761|4390|16572|7|281|1515|48|66.30|111.38|46.77|0.00|2244.96|3182.40|5346.24|157.14|0.00|2244.96|2402.10|-937.44| +2451803|53076|4807|55579|551761|4390|16572|7|5|1515|64|29.18|49.60|43.15|0.00|2761.60|1867.52|3174.40|138.08|0.00|2761.60|2899.68|894.08| +2451803|53076|15686|55579|551761|4390|16572|7|99|1515|76|47.12|90.47|88.66|0.00|6738.16|3581.12|6875.72|404.28|0.00|6738.16|7142.44|3157.04| +2452493|73384|5925|74796|379785|1221|48210|8|47|1516|45|14.30|22.02|0.44|0.00|19.80|643.50|990.90|1.18|0.00|19.80|20.98|-623.70| +2452493|73384|16314|74796|379785|1221|48210|8|33|1516|72|50.76|55.32|11.06|0.00|796.32|3654.72|3983.04|31.85|0.00|796.32|828.17|-2858.40| +2452493|73384|16749|74796|379785|1221|48210|8|6|1516|39|84.51|149.58|101.71|0.00|3966.69|3295.89|5833.62|357.00|0.00|3966.69|4323.69|670.80| +2452493|73384|9787|74796|379785|1221|48210|8|90|1516|31|89.19|107.02|107.02|0.00|3317.62|2764.89|3317.62|33.17|0.00|3317.62|3350.79|552.73| +2452493|73384|11454|74796|379785|1221|48210|8|264|1516|84|98.66|193.37|131.49|9940.64|11045.16|8287.44|16243.08|11.04|9940.64|1104.52|1115.56|-7182.92| +2452493|73384|14154|74796|379785|1221|48210|8|180|1516|15|99.99|102.98|49.43|0.00|741.45|1499.85|1544.70|29.65|0.00|741.45|771.10|-758.40| +2452493|73384|12426|74796|379785|1221|48210|8|127|1516|58|31.62|56.28|32.64|0.00|1893.12|1833.96|3264.24|94.65|0.00|1893.12|1987.77|59.16| +2452493|73384|189|74796|379785|1221|48210|8|148|1516|21|29.30|38.67|22.81|0.00|479.01|615.30|812.07|0.00|0.00|479.01|479.01|-136.29| +2452493|73384|10687|74796|379785|1221|48210|8|15|1516|53|76.86|144.49|20.22|0.00|1071.66|4073.58|7657.97|0.00|0.00|1071.66|1071.66|-3001.92| +2452493|73384|14083|74796|379785|1221|48210|8|225|1516|19|98.77|156.05|107.67|0.00|2045.73|1876.63|2964.95|61.37|0.00|2045.73|2107.10|169.10| +2452493|73384|669|74796|379785|1221|48210|8|21|1516|20|42.06|80.75|36.33|0.00|726.60|841.20|1615.00|58.12|0.00|726.60|784.72|-114.60| +2452493|73384|213|74796|379785|1221|48210|8|253|1516|9|10.02|20.04|19.43|0.00|174.87|90.18|180.36|13.98|0.00|174.87|188.85|84.69| +2452493|73384|2709|74796|379785|1221|48210|8|154|1516|86|59.02|111.54|61.34|1688.07|5275.24|5075.72|9592.44|0.00|1688.07|3587.17|3587.17|-1488.55| +2452493|73384|10158|74796|379785|1221|48210|8|275|1516|81|47.42|54.05|15.13|0.00|1225.53|3841.02|4378.05|24.51|0.00|1225.53|1250.04|-2615.49| +2452265|47692|15627|6422|1166938|7101|42417|8|231|1517|91|42.35|82.58|28.90|604.87|2629.90|3853.85|7514.78|101.25|604.87|2025.03|2126.28|-1828.82| +2452265|47692|9459|6422|1166938|7101|42417|8|175|1517|24|56.05|100.89|51.45|0.00|1234.80|1345.20|2421.36|98.78|0.00|1234.80|1333.58|-110.40| +2452265|47692|10795|6422|1166938|7101|42417|8|148|1517|45|89.24|116.90|81.83|0.00|3682.35|4015.80|5260.50|331.41|0.00|3682.35|4013.76|-333.45| +2452265|47692|13693|6422|1166938|7101|42417|8|55|1517|35|26.47|33.08|13.89|0.00|486.15|926.45|1157.80|24.30|0.00|486.15|510.45|-440.30| +2452265|47692|12921|6422|1166938|7101|42417|8|209|1517|32|68.22|90.05|81.04|1685.63|2593.28|2183.04|2881.60|0.00|1685.63|907.65|907.65|-1275.39| +2452265|47692|4177|6422|1166938|7101|42417|8|46|1517|60|90.08|127.91|113.83|0.00|6829.80|5404.80|7674.60|546.38|0.00|6829.80|7376.18|1425.00| +2452265|47692|2361|6422|1166938|7101|42417|8|224|1517|51|52.05|64.02|42.25|732.61|2154.75|2654.55|3265.02|28.44|732.61|1422.14|1450.58|-1232.41| +2452265|47692|4323|6422|1166938|7101|42417|8|217|1517|38|62.05|88.11|43.17|0.00|1640.46|2357.90|3348.18|16.40|0.00|1640.46|1656.86|-717.44| +2452265|47692|6835|6422|1166938|7101|42417|8|204|1517|18|94.50|101.11|25.27|13.64|454.86|1701.00|1819.98|8.82|13.64|441.22|450.04|-1259.78| +2452245||4325|68912|1832080|1648|26016|||1518|54|14.62|18.71|||39.96|||0.79|||40.75|| +2452245|61563|11635|68912|1832080|1648|26016|10|297|1518|44|36.23|48.91|26.90|603.63|1183.60|1594.12|2152.04|5.79|603.63|579.97|585.76|-1014.15| +2452245|61563|14239|68912|1832080|1648|26016|10|94|1518|40|77.26|122.84|103.18|0.00|4127.20|3090.40|4913.60|123.81|0.00|4127.20|4251.01|1036.80| +2452245|61563|11661|68912|1832080|1648|26016|10|272|1518|34|88.50|146.02|36.50|0.00|1241.00|3009.00|4964.68|12.41|0.00|1241.00|1253.41|-1768.00| +2452245|61563|411|68912|1832080|1648|26016|10|35|1518|29|95.88|165.87|82.93|144.29|2404.97|2780.52|4810.23|67.82|144.29|2260.68|2328.50|-519.84| +2452245|61563|17365|68912|1832080|1648|26016|10|278|1518|97|45.41|55.85|55.85|0.00|5417.45|4404.77|5417.45|325.04|0.00|5417.45|5742.49|1012.68| +||9705||1832080||26016|||1518|||56.50||0.00|1075.76|2122.96|3842.00||0.00|||| +2452245|61563|753|68912|1832080|1648|26016|10|111|1518|79|38.13|65.58|48.52|0.00|3833.08|3012.27|5180.82|38.33|0.00|3833.08|3871.41|820.81| +2452245|61563|409|68912|1832080|1648|26016|10|172|1518|50|32.01|48.33|2.41|0.00|120.50|1600.50|2416.50|3.61|0.00|120.50|124.11|-1480.00| +2452245|61563|10167|68912|1832080|1648|26016|10|283|1518|16|34.25|55.48|0.00|0.00|0.00|548.00|887.68|0.00|0.00|0.00|0.00|-548.00| +2452245|61563|4449|68912|1832080|1648|26016|10|248|1518|68|32.11|60.68|46.11|0.00|3135.48|2183.48|4126.24|282.19|0.00|3135.48|3417.67|952.00| +2452245|61563|2447|68912|1832080|1648|26016|10|40|1518|53|29.16|34.11|22.51|0.00|1193.03|1545.48|1807.83|0.00|0.00|1193.03|1193.03|-352.45| +2452056|66980|13083|71606|170733|4707|15834|2|234|1519|40|35.04|58.51|46.80|524.16|1872.00|1401.60|2340.40|94.34|524.16|1347.84|1442.18|-53.76| +2452056|66980|5459|71606|170733|4707|15834|2|216|1519|67|26.37|40.87|11.44|0.00|766.48|1766.79|2738.29|30.65|0.00|766.48|797.13|-1000.31| +2452056|66980|5221|71606|170733|4707|15834|2|164|1519|46|42.07|50.48|29.27|0.00|1346.42|1935.22|2322.08|13.46|0.00|1346.42|1359.88|-588.80| +2452056|66980|2021|71606|170733|4707|15834|2|273|1519|82|69.09|82.21|69.05|3114.15|5662.10|5665.38|6741.22|0.00|3114.15|2547.95|2547.95|-3117.43| +2452056|66980|1557|71606|170733|4707|15834|2|164|1519|96|99.93|138.90|40.28|0.00|3866.88|9593.28|13334.40|38.66|0.00|3866.88|3905.54|-5726.40| +2452056|66980|17233|71606|170733|4707|15834|2|168|1519|64|75.68|79.46|48.47|0.00|3102.08|4843.52|5085.44|31.02|0.00|3102.08|3133.10|-1741.44| +2452056|66980|2131|71606|170733|4707|15834|2|8|1519|87|13.12|19.28|18.70|0.00|1626.90|1141.44|1677.36|0.00|0.00|1626.90|1626.90|485.46| +2452056|66980|17515|71606|170733|4707|15834|2|110|1519|80|23.26|39.07|5.46|0.00|436.80|1860.80|3125.60|8.73|0.00|436.80|445.53|-1424.00| +2452056|66980|12783|71606|170733|4707|15834|2|197|1519|85|47.71|93.51|24.31|0.00|2066.35|4055.35|7948.35|61.99|0.00|2066.35|2128.34|-1989.00| +2452056|66980|199|71606|170733|4707|15834|2|254|1519|91|16.68|24.85|1.49|0.00|135.59|1517.88|2261.35|2.71|0.00|135.59|138.30|-1382.29| +|66980|13561|71606||4707|15834|||1519||30.85||22.62||1447.68|1974.40||26.05|||460.36|| +2451152|47915|91|30178|513787|1396|31805|8|148|1520|7|40.81|56.72|2.83|0.00|19.81|285.67|397.04|0.79|0.00|19.81|20.60|-265.86| +2451152|47915|10772|30178|513787|1396|31805|8|183|1520|12|62.48|100.59|27.15|0.00|325.80|749.76|1207.08|16.29|0.00|325.80|342.09|-423.96| +2451152|47915|2386|30178|513787|1396|31805|8|283|1520|76|99.48|160.16|24.02|0.00|1825.52|7560.48|12172.16|73.02|0.00|1825.52|1898.54|-5734.96| +2451152|47915|15274|30178|513787|1396|31805|8|114|1520|32|31.23|43.09|24.99|0.00|799.68|999.36|1378.88|15.99|0.00|799.68|815.67|-199.68| +2451152|47915|10285|30178|513787|1396|31805|8|134|1520|92|73.87|120.40|44.54|0.00|4097.68|6796.04|11076.80|327.81|0.00|4097.68|4425.49|-2698.36| +2451152|47915|16723|30178|513787|1396|31805|8|143|1520|61|20.51|38.55|6.93|0.00|422.73|1251.11|2351.55|4.22|0.00|422.73|426.95|-828.38| +2451152|47915|11995|30178|513787|1396|31805|8|33|1520|3|40.75|52.16|18.25|29.01|54.75|122.25|156.48|0.77|29.01|25.74|26.51|-96.51| +2451152|47915|604|30178|513787|1396|31805|8|127|1520|21|36.85|59.69|56.10|0.00|1178.10|773.85|1253.49|23.56|0.00|1178.10|1201.66|404.25| +2451152|47915|10690|30178|513787|1396|31805|8|248|1520|19|12.40|22.44|6.05|0.00|114.95|235.60|426.36|9.19|0.00|114.95|124.14|-120.65| +2451152|47915|565|30178|513787|1396|31805|8|172|1520|84|35.44|62.37|21.82|0.00|1832.88|2976.96|5239.08|0.00|0.00|1832.88|1832.88|-1144.08| +||2431|30178|513787|||8|47|1520||47.00||||||4850.40||||4462.20|| +2451525|68628|9542|5465|1079816|214|26965|1|163|1521|14|80.21|92.24|47.96|0.00|671.44|1122.94|1291.36|40.28|0.00|671.44|711.72|-451.50| +2451525||6116||1079816|214|26965|||1521|98||65.50|||5263.58||6419.00|157.90|||5421.48|| +2451525|68628|11479|5465|1079816|214|26965|1|9|1521|52|99.01|141.58|15.57|0.00|809.64|5148.52|7362.16|64.77|0.00|809.64|874.41|-4338.88| +2451525|68628|14311|5465|1079816|214|26965|1|168|1521|56|37.15|50.15|10.03|0.00|561.68|2080.40|2808.40|0.00|0.00|561.68|561.68|-1518.72| +2451525|68628|8936|5465|1079816|214|26965|1|17|1521|58|34.56|68.08|17.70|379.84|1026.60|2004.48|3948.64|32.33|379.84|646.76|679.09|-1357.72| +2451525|68628|12868|5465|1079816|214|26965|1|275|1521|24|96.20|157.76|69.41|1549.23|1665.84|2308.80|3786.24|3.49|1549.23|116.61|120.10|-2192.19| +2451525|68628|17852|5465|1079816|214|26965|1|119|1521|11|89.11|136.33|32.71|0.00|359.81|980.21|1499.63|3.59|0.00|359.81|363.40|-620.40| +2451525|68628|1975|5465|1079816|214|26965|1|298|1521|4|42.57|60.02|36.61|0.00|146.44|170.28|240.08|1.46|0.00|146.44|147.90|-23.84| +2451525|68628|4000|5465|1079816|214|26965|1|61|1521|46|76.28|106.02|60.43|0.00|2779.78|3508.88|4876.92|83.39|0.00|2779.78|2863.17|-729.10| +||4363||1079816|214|26965|1|176|1521|86|82.63||||8494.22|7106.18||||5096.54||-2009.64| +2451525|68628|3631|5465|1079816|214|26965|1|179|1521|23|2.87|5.10|3.72|0.00|85.56|66.01|117.30|0.85|0.00|85.56|86.41|19.55| +2451525|68628|4468|5465|1079816|214|26965|1|266|1521|96|4.44|4.70|1.36|0.00|130.56|426.24|451.20|10.44|0.00|130.56|141.00|-295.68| +2451525|68628|3134|5465|1079816|214|26965|1|200|1521|20|25.77|45.87|8.71|139.36|174.20|515.40|917.40|3.13|139.36|34.84|37.97|-480.56| +2451525|68628|9148|5465|1079816|214|26965|1|151|1521|100|31.40|43.01|27.95|0.00|2795.00|3140.00|4301.00|195.65|0.00|2795.00|2990.65|-345.00| +2452019|61127|16455|81888|1761879|5070|48214|8|99|1522|91|5.68|9.37|1.68|0.00|152.88|516.88|852.67|9.17|0.00|152.88|162.05|-364.00| +2452019|61127|15121|81888|1761879|5070|48214|8|186|1522|22|40.88|62.13|44.11|0.00|970.42|899.36|1366.86|48.52|0.00|970.42|1018.94|71.06| +2452019|61127|12979|81888|1761879|5070|48214|8|287|1522|27|68.21|85.94|38.67|459.39|1044.09|1841.67|2320.38|40.92|459.39|584.70|625.62|-1256.97| +2452019|61127|9513|81888|1761879|5070|48214|8|147|1522|89|45.48|52.75|3.16|0.00|281.24|4047.72|4694.75|19.68|0.00|281.24|300.92|-3766.48| +2452019|61127|1259|81888|1761879|5070|48214|8|265|1522|20|49.77|93.06|19.54|46.89|390.80|995.40|1861.20|0.00|46.89|343.91|343.91|-651.49| +2452019|61127|7399|81888|1761879|5070|48214|8|292|1522|4|29.12|33.19|22.56|0.00|90.24|116.48|132.76|0.00|0.00|90.24|90.24|-26.24| +2452019|61127|533|81888|1761879|5070|48214|8|274|1522|79|65.17|99.05|58.43|2908.06|4615.97|5148.43|7824.95|153.71|2908.06|1707.91|1861.62|-3440.52| +|61127|15787|81888|||48214|8||1522|15|||3.66|||344.70|||||59.84|-289.80| +2452019|61127|125|81888|1761879|5070|48214|8|196|1522|13|3.30|5.94|4.33|0.00|56.29|42.90|77.22|2.81|0.00|56.29|59.10|13.39| +2452053|57707|6797|95373|423426|6679|35932|7|174|1523|56|5.19|7.05|6.76|0.00|378.56|290.64|394.80|30.28|0.00|378.56|408.84|87.92| +2452053|57707|14659|95373|423426|6679|35932|7|269|1523|78|47.80|90.82|84.46|0.00|6587.88|3728.40|7083.96|461.15|0.00|6587.88|7049.03|2859.48| +2452053|57707|12685|95373|423426|6679|35932|7|163|1523|39|7.31|14.18|10.63|0.00|414.57|285.09|553.02|20.72|0.00|414.57|435.29|129.48| +2452053|57707|3723|95373|423426|6679|35932|7|278|1523|50|38.74|54.62|27.31|0.00|1365.50|1937.00|2731.00|27.31|0.00|1365.50|1392.81|-571.50| +2452053|57707|11057|95373|423426|6679|35932|7|223|1523|100|4.81|8.41|4.20|205.80|420.00|481.00|841.00|14.99|205.80|214.20|229.19|-266.80| +2452053|57707|5387|95373|423426|6679|35932|7|287|1523|97|23.85|25.28|0.50|0.00|48.50|2313.45|2452.16|3.39|0.00|48.50|51.89|-2264.95| +2452053||14987|95373|423426|6679||7|179|1523|85|81.41||136.76||11624.60||12109.10||||12322.07|4704.75| +2452053|57707|2879|95373|423426|6679|35932|7|167|1523|82|67.02|129.34|5.17|0.00|423.94|5495.64|10605.88|38.15|0.00|423.94|462.09|-5071.70| +2452053|57707|12157|95373|423426|6679|35932|7|105|1523|76|40.71|57.80|24.27|0.00|1844.52|3093.96|4392.80|92.22|0.00|1844.52|1936.74|-1249.44| +2452053|57707|1295|95373|423426|6679|35932|7|189|1523|31|62.52|123.16|123.16|0.00|3817.96|1938.12|3817.96|152.71|0.00|3817.96|3970.67|1879.84| +2452053|57707|8097|95373|423426|6679|35932|7|78|1523|78|64.88|99.91|1.99|20.17|155.22|5060.64|7792.98|5.40|20.17|135.05|140.45|-4925.59| +2452053|57707|13395|95373|423426|6679|35932|7|287|1523|73|30.49|34.45|28.59|0.00|2087.07|2225.77|2514.85|104.35|0.00|2087.07|2191.42|-138.70| +2451491|38625|10784|18525|247397|1422|35894|2|108|1524|6|87.99|125.82|78.00|0.00|468.00|527.94|754.92|28.08|0.00|468.00|496.08|-59.94| +2451491|38625|11335|18525|247397|1422|35894|2|45|1524|84|13.15|22.74|0.00|0.00|0.00|1104.60|1910.16|0.00|0.00|0.00|0.00|-1104.60| +2451491|38625|6940|18525|247397|1422|35894|2|31|1524|45|21.69|22.34|13.18|0.00|593.10|976.05|1005.30|17.79|0.00|593.10|610.89|-382.95| +2451491|38625|4460|18525|247397|1422|35894|2|107|1524|13|64.96|129.92|36.37|0.00|472.81|844.48|1688.96|4.72|0.00|472.81|477.53|-371.67| +2451491|38625|15464|18525|247397|1422|35894|2|154|1524|64|38.76|76.74|3.06|0.00|195.84|2480.64|4911.36|11.75|0.00|195.84|207.59|-2284.80| +2451491|38625|782|18525|247397|1422|35894|2|142|1524|45|29.80|50.66|13.17|0.00|592.65|1341.00|2279.70|47.41|0.00|592.65|640.06|-748.35| +2451491|38625|7058|18525|247397|1422|35894|2|161|1524|1|53.05|106.10|88.06|0.00|88.06|53.05|106.10|1.76|0.00|88.06|89.82|35.01| +2451491|38625|13378|18525|247397|1422|35894|2|166|1524|98|53.33|60.26|40.97|0.00|4015.06|5226.34|5905.48|281.05|0.00|4015.06|4296.11|-1211.28| +2451491|38625|10964|18525|247397|1422|35894|2|180|1524|18|40.07|46.08|0.00|0.00|0.00|721.26|829.44|0.00|0.00|0.00|0.00|-721.26| +2451491|38625|6493|18525|247397|1422|35894|2|271|1524|32|95.64|97.55|1.95|0.00|62.40|3060.48|3121.60|2.49|0.00|62.40|64.89|-2998.08| +2452215|69238|8025|59798|722443|641|29686|10|163|1525|19|32.47|59.74|45.99|0.00|873.81|616.93|1135.06|52.42|0.00|873.81|926.23|256.88| +2452215||8605|59798|722443||29686|10||1525|74|||38.96||2883.04||4650.16|201.81||2883.04||473.60| +2452215|69238|2889|59798|722443|641|29686|10|62|1525|1|61.39|70.59|14.11|0.00|14.11|61.39|70.59|0.00|0.00|14.11|14.11|-47.28| +2452215|69238|13923|59798|722443|641|29686|10|243|1525|85|82.13|146.19|59.93|0.00|5094.05|6981.05|12426.15|50.94|0.00|5094.05|5144.99|-1887.00| +2452215|69238|14361|59798|722443|641|29686|10|69|1525|35|88.95|162.77|74.87|1284.02|2620.45|3113.25|5696.95|66.82|1284.02|1336.43|1403.25|-1776.82| +2452215|69238|4675|59798|722443|641|29686|10|127|1525|62|25.94|26.45|16.39|0.00|1016.18|1608.28|1639.90|91.45|0.00|1016.18|1107.63|-592.10| +2452215|69238|6351|59798|722443|641|29686|10|34|1525|96|6.49|10.70|6.52|0.00|625.92|623.04|1027.20|12.51|0.00|625.92|638.43|2.88| +2452215|69238|5879|59798|722443|641|29686|10|51|1525|47|18.33|29.69|10.98|397.36|516.06|861.51|1395.43|3.56|397.36|118.70|122.26|-742.81| +2452215|69238|16897|59798|722443|641|29686|10|184|1525|53|71.41|138.53|90.04|0.00|4772.12|3784.73|7342.09|334.04|0.00|4772.12|5106.16|987.39| +2451118|70830|2912|1583|1684298|3013|3728|4|284|1526|58|9.60|16.32|2.44|96.23|141.52|556.80|946.56|1.81|96.23|45.29|47.10|-511.51| +2451118|70830|11966|1583|1684298|3013|3728|4|222|1526|37|33.61|46.71|14.01|0.00|518.37|1243.57|1728.27|20.73|0.00|518.37|539.10|-725.20| +2451118|70830|10417|1583|1684298|3013|3728|4|127|1526|46|41.40|60.85|17.03|0.00|783.38|1904.40|2799.10|23.50|0.00|783.38|806.88|-1121.02| +2451118|70830|2822|1583|1684298|3013|3728|4|206|1526|80|26.21|30.40|2.12|64.44|169.60|2096.80|2432.00|3.15|64.44|105.16|108.31|-1991.64| +2451118|70830|6082|1583||3013|3728||85|1526|34|25.19|27.45|3.84|0.00|130.56|856.46|933.30||0.00|130.56|141.00|-725.90| +2451118|70830|3751|1583|1684298|3013|3728|4|177|1526|46|90.49|97.72|28.33|0.00|1303.18|4162.54|4495.12|13.03|0.00|1303.18|1316.21|-2859.36| +2451118|70830|16510|1583|1684298|3013|3728|4|291|1526|33|65.63|112.22|88.65|526.58|2925.45|2165.79|3703.26|119.94|526.58|2398.87|2518.81|233.08| +2451118|70830|12646|1583|1684298|3013|3728|4|135|1526|17|71.15|139.45|68.33|0.00|1161.61|1209.55|2370.65|69.69|0.00|1161.61|1231.30|-47.94| +2451118|70830|10885|1583|1684298|3013|3728|4|215|1526|38|99.54|155.28|128.88|0.00|4897.44|3782.52|5900.64|293.84|0.00|4897.44|5191.28|1114.92| +2451118|70830|5606|1583|1684298|3013|3728|4|176|1526|60|59.90|84.45|59.11|0.00|3546.60|3594.00|5067.00|106.39|0.00|3546.60|3652.99|-47.40| +2451118|70830|9070|1583|1684298|3013|3728|4|243|1526|47|58.06|59.22|39.08|0.00|1836.76|2728.82|2783.34|128.57|0.00|1836.76|1965.33|-892.06| +2451118|70830|8710|1583|1684298|3013|3728|4|163|1526|26|8.50|10.11|2.72|0.00|70.72|221.00|262.86|1.41|0.00|70.72|72.13|-150.28| +2451118|70830|15583|1583|1684298|3013|3728|4|257|1526|83|18.62|35.56|8.89|0.00|737.87|1545.46|2951.48|0.00|0.00|737.87|737.87|-807.59| +2451118|70830|6385|1583|1684298|3013|3728|4|140|1526|55|60.93|113.32|56.66|1371.17|3116.30|3351.15|6232.60|0.00|1371.17|1745.13|1745.13|-1606.02| +2451176|44326|5384|19541|1707038|5735|22432|4|246|1527|94|78.94|96.30|12.51|0.00|1175.94|7420.36|9052.20|47.03|0.00|1175.94|1222.97|-6244.42| +2451176|44326|12850|19541|1707038|5735|22432|4|270|1527|92|54.79|99.71|24.92|0.00|2292.64|5040.68|9173.32|22.92|0.00|2292.64|2315.56|-2748.04| +2451176|44326|6646|19541|1707038|5735|22432|4|259|1527|8|53.23|61.21|0.00|0.00|0.00|425.84|489.68|0.00|0.00|0.00|0.00|-425.84| +2451176|44326|5053|19541|1707038|5735|22432|4|222|1527|54|20.97|24.53|14.71|0.00|794.34|1132.38|1324.62|55.60|0.00|794.34|849.94|-338.04| +2451176|44326|7336|19541|1707038|5735|22432|4|26|1527|14|1.21|1.31|0.98|4.66|13.72|16.94|18.34|0.63|4.66|9.06|9.69|-7.88| +2451176|44326|1988|19541|1707038|5735|22432|4|271|1527|42|12.20|19.27|12.91|70.48|542.22|512.40|809.34|18.86|70.48|471.74|490.60|-40.66| +2451176|44326|9679|19541|1707038|5735|22432|4|196|1527|9|98.41|181.07|54.32|0.00|488.88|885.69|1629.63|19.55|0.00|488.88|508.43|-396.81| +2451176|44326|652|19541|1707038|5735|22432|4|279|1527|24|36.98|63.23|12.64|0.00|303.36|887.52|1517.52|0.00|0.00|303.36|303.36|-584.16| +2451176|44326|8906|19541|1707038|5735|22432|4|83|1527|80|26.94|28.82|7.78|0.00|622.40|2155.20|2305.60|0.00|0.00|622.40|622.40|-1532.80| +2451176|44326|11830|19541|1707038|5735|22432|4|165|1527|56|13.88|14.15|6.36|0.00|356.16|777.28|792.40|3.56|0.00|356.16|359.72|-421.12| +2451176|44326|17156|19541|1707038|5735|22432|4|182|1527|96|8.69|10.77|10.01|0.00|960.96|834.24|1033.92|67.26|0.00|960.96|1028.22|126.72| +2451176|44326|8960|19541|1707038|5735|22432|4|82|1527|58|83.57|89.41|69.73|0.00|4044.34|4847.06|5185.78|202.21|0.00|4044.34|4246.55|-802.72| +2451176|44326|7435|19541|1707038|5735|22432|4|55|1527|66|28.07|48.28|5.31|0.00|350.46|1852.62|3186.48|24.53|0.00|350.46|374.99|-1502.16| +2451176|44326|3847|19541|1707038|5735|22432|4|259|1527|86|16.31|28.05|15.98|0.00|1374.28|1402.66|2412.30|13.74|0.00|1374.28|1388.02|-28.38| +2451766|69959|15241|68335|1847576|2017|36058|1|184|1528|12|1.84|1.98|0.87|7.30|10.44|22.08|23.76|0.12|7.30|3.14|3.26|-18.94| +2451766|69959|3793|68335|1847576|2017|36058|1|153|1528|95|67.54|93.88|80.73|0.00|7669.35|6416.30|8918.60|460.16|0.00|7669.35|8129.51|1253.05| +2451766|69959|15689|68335|1847576|2017|36058|1|256|1528|55|84.92|168.14|139.55|307.01|7675.25|4670.60|9247.70|147.36|307.01|7368.24|7515.60|2697.64| +||12278|68335|1847576|2017|36058|||1528||||78.52|||||54.96||||| +2451766|69959|7982|68335|1847576|2017|36058|1|235|1528|9|53.66|82.09|37.76|0.00|339.84|482.94|738.81|13.59|0.00|339.84|353.43|-143.10| +2451766|69959|11717||1847576||36058||7|1528|18||121.61|||328.32||2188.98|||328.32||| +||6056|68335|||36058|1||1528||38.47||||1105.00|2615.96||0.00||1105.00||| +2451766|69959|14593|68335|1847576|2017|36058|1|158|1528|85|47.76|93.60|59.90|0.00|5091.50|4059.60|7956.00|50.91|0.00|5091.50|5142.41|1031.90| +2451985|47888|17273|65565|16048|5731|38214|4|195|1529|26|58.60|82.62|4.13|40.80|107.38|1523.60|2148.12|0.00|40.80|66.58|66.58|-1457.02| +2451985|47888|4779|65565|16048|5731|38214|4|11|1529|46|66.06|132.12|67.38|0.00|3099.48|3038.76|6077.52|0.00|0.00|3099.48|3099.48|60.72| +2451985|47888|11329|65565|16048|5731|38214|4|166|1529|80|32.80|48.21|22.17|0.00|1773.60|2624.00|3856.80|124.15|0.00|1773.60|1897.75|-850.40| +2451985|47888|7533|65565|16048|5731|38214|4|111|1529|29|26.15|42.36|20.75|0.00|601.75|758.35|1228.44|12.03|0.00|601.75|613.78|-156.60| +2451985|47888|1677|65565|16048|5731|38214|4|78|1529|55|31.47|36.50|2.19|0.00|120.45|1730.85|2007.50|4.81|0.00|120.45|125.26|-1610.40| +2451985|47888|9813|65565|16048|5731|38214|4|108|1529|70|49.33|86.32|75.96|0.00|5317.20|3453.10|6042.40|478.54|0.00|5317.20|5795.74|1864.10| +2451985|47888|11841|65565|16048|5731|38214|4|90|1529|98|42.75|45.31|3.62|0.00|354.76|4189.50|4440.38|21.28|0.00|354.76|376.04|-3834.74| +2451985|47888|7661|65565|16048|5731|38214|4|94|1529|57|29.16|29.74|18.73|0.00|1067.61|1662.12|1695.18|96.08|0.00|1067.61|1163.69|-594.51| +2451985|47888|12813|65565|16048|5731|38214|4|162|1529|19|48.57|64.11|23.72|0.00|450.68|922.83|1218.09|18.02|0.00|450.68|468.70|-472.15| +2451985|47888|7819|65565|16048|5731|38214|4|46|1529|98|54.58|78.59|55.79|2515.01|5467.42|5348.84|7701.82|206.66|2515.01|2952.41|3159.07|-2396.43| +2451985|47888|8559|65565|16048|5731|38214|4|21|1529|59|49.67|81.95|40.15|0.00|2368.85|2930.53|4835.05|213.19|0.00|2368.85|2582.04|-561.68| +2451985|47888|17497|65565|16048|5731|38214|4|63|1529|90|30.89|54.05|16.21|0.00|1458.90|2780.10|4864.50|0.00|0.00|1458.90|1458.90|-1321.20| +2451985|47888|13783|65565|16048|5731|38214|4|276|1529|76|86.96|120.87|47.13|0.00|3581.88|6608.96|9186.12|322.36|0.00|3581.88|3904.24|-3027.08| +2451985|47888|3461|65565|16048|5731|38214|4|287|1529|56|9.37|9.55|4.01|0.00|224.56|524.72|534.80|15.71|0.00|224.56|240.27|-300.16| +2451985|47888|691|65565|16048|5731|38214|4|74|1529|30|39.38|75.60|55.94|218.16|1678.20|1181.40|2268.00|14.60|218.16|1460.04|1474.64|278.64| +2451681|62173|13057|21798|822055|4359|39016|2|121|1530|88|70.61|122.86|84.77|0.00|7459.76|6213.68|10811.68|74.59|0.00|7459.76|7534.35|1246.08| +2451681|62173|14689|21798|822055|4359|39016|2|127|1530|51|93.79|135.05|124.24|0.00|6336.24|4783.29|6887.55|190.08|0.00|6336.24|6526.32|1552.95| +2451681|62173|12697|21798|822055|4359|39016|2|85|1530|77|55.89|88.86|69.31|0.00|5336.87|4303.53|6842.22|160.10|0.00|5336.87|5496.97|1033.34| +2451681|62173|6812|21798|822055|4359|39016|2|194|1530|85|3.05|4.02|3.13|0.00|266.05|259.25|341.70|7.98|0.00|266.05|274.03|6.80| +2451681|62173|3893|21798|822055|4359|39016|2|127|1530|68|64.41|65.05|58.54|0.00|3980.72|4379.88|4423.40|199.03|0.00|3980.72|4179.75|-399.16| +2451681|62173|17732|21798|822055|4359|39016|2|86|1530|11|66.70|76.70|35.28|0.00|388.08|733.70|843.70|23.28|0.00|388.08|411.36|-345.62| +2451681|62173|10433|21798|822055|4359|39016|2|217|1530|78|31.73|41.24|30.93|217.12|2412.54|2474.94|3216.72|65.86|217.12|2195.42|2261.28|-279.52| +2451681|62173|4172|21798|822055|4359|39016|2|90|1530|59|34.09|51.13|34.76|0.00|2050.84|2011.31|3016.67|143.55|0.00|2050.84|2194.39|39.53| +2451681|62173|1622|21798|822055|4359|39016|2|7|1530|61|43.70|81.71|10.62|0.00|647.82|2665.70|4984.31|45.34|0.00|647.82|693.16|-2017.88| +2451681|62173|17201|21798|822055|4359|39016|2|281|1530|93|98.19|150.23|135.20|0.00|12573.60|9131.67|13971.39|0.00|0.00|12573.60|12573.60|3441.93| +2451681|62173|17819|21798|822055|4359|39016|2|183|1530|79|90.67|146.88|123.37|0.00|9746.23|7162.93|11603.52|194.92|0.00|9746.23|9941.15|2583.30| +2451681|62173|5144|21798|822055|4359|39016|2|78|1530|9|19.23|22.88|1.60|0.00|14.40|173.07|205.92|1.29|0.00|14.40|15.69|-158.67| +2451681|62173|5873|21798|822055|4359|39016|2|156|1530|2|8.39|10.82|7.14|3.99|14.28|16.78|21.64|0.82|3.99|10.29|11.11|-6.49| +2451681|62173|3955|21798|822055|4359|39016|2|28|1530|44|62.89|84.27|30.33|0.00|1334.52|2767.16|3707.88|40.03|0.00|1334.52|1374.55|-1432.64| +2451470|35823|2602|59845|970955|5712|12869|10|112|1531|1|8.19|9.41|4.42|0.00|4.42|8.19|9.41|0.00|0.00|4.42|4.42|-3.77| +2451470|35823|7900|59845|970955|5712|12869|10|178|1531|54|77.16|141.97|58.20|0.00|3142.80|4166.64|7666.38|251.42|0.00|3142.80|3394.22|-1023.84| +2451470|35823|10063|59845|970955|5712|12869|10|85|1531|100|93.74|100.30|2.00|0.00|200.00|9374.00|10030.00|6.00|0.00|200.00|206.00|-9174.00| +2451470|35823|1840|59845|970955|5712|12869|10|287|1531|4|78.34|119.86|85.10|0.00|340.40|313.36|479.44|17.02|0.00|340.40|357.42|27.04| +2451470|35823|4178|||5712|12869|10||1531|29|||33.31|0.00|965.99||||0.00|||130.21| +2451470|35823|4012|59845|970955|5712|12869|10|212|1531|79|91.65|175.96|59.82|4205.94|4725.78|7240.35|13900.84|31.19|4205.94|519.84|551.03|-6720.51| +2451470|35823|13057|59845|970955|5712|12869|10|168|1531|72|10.12|11.73|2.34|0.00|168.48|728.64|844.56|0.00|0.00|168.48|168.48|-560.16| +2451470|35823|14689|59845|970955|5712|12869|10|46|1531|50|9.67|9.86|5.91|0.00|295.50|483.50|493.00|26.59|0.00|295.50|322.09|-188.00| +2451470|35823|12697|59845|970955|5712|12869|10|152|1531|80|82.46|164.92|3.29|0.00|263.20|6596.80|13193.60|7.89|0.00|263.20|271.09|-6333.60| +2451470|35823|6812|59845|970955|5712|12869|10|36|1531|9|7.73|14.22|9.38|0.00|84.42|69.57|127.98|0.00|0.00|84.42|84.42|14.85| +2451470|35823|3892|59845|970955|5712|12869|10|88|1531|11|17.10|28.04|24.39|0.00|268.29|188.10|308.44|2.68|0.00|268.29|270.97|80.19| +2451470|35823|17732|59845|970955|5712|12869|10|143|1531|25|65.06|109.95|9.89|0.00|247.25|1626.50|2748.75|2.47|0.00|247.25|249.72|-1379.25| +2451470|35823|10432|59845|970955|5712|12869|10|254|1531|6|48.39|92.42|4.62|0.00|27.72|290.34|554.52|1.10|0.00|27.72|28.82|-262.62| +2451470|35823|4172|59845|970955|5712|12869|10|206|1531|78|6.01|11.65|0.11|0.00|8.58|468.78|908.70|0.42|0.00|8.58|9.00|-460.20| +2451470|35823|1622|59845|970955|5712|12869|10|275|1531|57|77.46|147.17|120.67|0.00|6878.19|4415.22|8388.69|619.03|0.00|6878.19|7497.22|2462.97| +2451470|35823|17200|59845|970955|5712|12869|10|30|1531|69|7.55|13.51|9.32|0.00|643.08|520.95|932.19|19.29|0.00|643.08|662.37|122.13| +2452167|30830|14645|93032|1204604|3729|3701|10|166|1532|22|45.15|88.94|39.13|0.00|860.86|993.30|1956.68|8.60|0.00|860.86|869.46|-132.44| +2452167|30830|7529|93032|1204604|3729|3701|10|220|1532|63|9.89|17.30|1.38|0.00|86.94|623.07|1089.90|3.47|0.00|86.94|90.41|-536.13| +2452167|30830|15535|93032|1204604|3729|3701|10|215|1532|35|84.50|111.54|42.38|0.00|1483.30|2957.50|3903.90|88.99|0.00|1483.30|1572.29|-1474.20| +2452167|30830|33|93032|1204604|3729|3701|10|236|1532|57|9.46|14.56|6.11|0.00|348.27|539.22|829.92|31.34|0.00|348.27|379.61|-190.95| +2452167|30830|1923|93032|1204604|3729|3701|10|189|1532|91|12.83|15.13|1.96|0.00|178.36|1167.53|1376.83|3.56|0.00|178.36|181.92|-989.17| +2452167|30830|1709|93032|1204604|3729|3701|10|59|1532|43|92.36|166.24|43.22|1468.18|1858.46|3971.48|7148.32|31.22|1468.18|390.28|421.50|-3581.20| +2452167|30830|8899|93032|1204604|3729|3701|10|107|1532|37|19.44|34.21|13.68|0.00|506.16|719.28|1265.77|15.18|0.00|506.16|521.34|-213.12| +2452167|30830|16581|93032|1204604|3729|3701|10|217|1532|53|84.85|151.88|103.27|0.00|5473.31|4497.05|8049.64|109.46|0.00|5473.31|5582.77|976.26| +2452167|30830|8277|93032|1204604|3729|3701|10|252|1532|97|93.75|112.50|10.12|0.00|981.64|9093.75|10912.50|78.53|0.00|981.64|1060.17|-8112.11| +2452167|30830|9735|93032|1204604|3729|3701|10|248|1532|8|73.55|83.11|34.90|0.00|279.20|588.40|664.88|16.75|0.00|279.20|295.95|-309.20| +2452167|30830|17223|93032|1204604|3729|3701|10|270|1532|63|52.29|76.86|11.52|631.41|725.76|3294.27|4842.18|6.60|631.41|94.35|100.95|-3199.92| +2452167|30830|4377|93032|1204604|3729|3701|10|84|1532|28|22.64|24.45|10.26|0.00|287.28|633.92|684.60|25.85|0.00|287.28|313.13|-346.64| +2452167|30830|885|93032|1204604|3729|3701|10|34|1532|15|78.43|101.17|85.99|0.00|1289.85|1176.45|1517.55|116.08|0.00|1289.85|1405.93|113.40| +2452167|30830|4761|93032|1204604|3729|3701|10|158|1532|87|84.35|95.31|85.77|5447.25|7461.99|7338.45|8291.97|20.14|5447.25|2014.74|2034.88|-5323.71| +2451870|35459|10669|7156|1154356|5556|5956|7|234|1533|29|19.95|26.33|24.75|0.00|717.75|578.55|763.57|21.53|0.00|717.75|739.28|139.20| +2451870|35459|3431|7156|1154356|5556|5956|7|134|1533|72|66.60|77.92|24.15|0.00|1738.80|4795.20|5610.24|52.16|0.00|1738.80|1790.96|-3056.40| +2451870|35459|12098|7156|1154356|5556|5956|7|148|1533|17|1.48|2.66|1.27|19.86|21.59|25.16|45.22|0.00|19.86|1.73|1.73|-23.43| +2451870|35459|7931|7156|1154356|5556|5956|7|176|1533|17|24.46|43.53|26.11|0.00|443.87|415.82|740.01|35.50|0.00|443.87|479.37|28.05| +2451870|35459|9755|7156|1154356|5556|5956|7|18|1533|75|13.16|22.10|21.87|0.00|1640.25|987.00|1657.50|0.00|0.00|1640.25|1640.25|653.25| +2451870|35459|15667|7156|1154356|5556|5956|7|94|1533|62|91.87|174.55|85.52|0.00|5302.24|5695.94|10822.10|318.13|0.00|5302.24|5620.37|-393.70| +2451870|35459|5396|7156|1154356|5556|5956|7|274|1533|13|8.31|13.96|1.81|15.05|23.53|108.03|181.48|0.00|15.05|8.48|8.48|-99.55| +2451870|35459|1685|7156|1154356|5556|5956|7|19|1533|87|74.64|108.22|101.72|0.00|8849.64|6493.68|9415.14|530.97|0.00|8849.64|9380.61|2355.96| +2451870|35459|16771|7156|1154356|5556|5956|7|119|1533|81|68.90|118.50|5.92|0.00|479.52|5580.90|9598.50|19.18|0.00|479.52|498.70|-5101.38| +2452557|44617|8577|6397|216378|406|3634|10|276|1534|52|59.16|118.32|91.10|1705.39|4737.20|3076.32|6152.64|121.27|1705.39|3031.81|3153.08|-44.51| +2452557|44617|8442|6397|216378|406|3634|10|232|1534|37|36.62|69.57|4.17|0.00|154.29|1354.94|2574.09|0.00|0.00|154.29|154.29|-1200.65| +2452557|44617|6663|6397|216378|406|3634|10|28|1534|53|92.74|160.44|73.80|0.00|3911.40|4915.22|8503.32|352.02|0.00|3911.40|4263.42|-1003.82| +2452557|44617|16203|6397|216378|406|3634|10|120|1534|65|25.16|26.92|1.61|0.00|104.65|1635.40|1749.80|2.09|0.00|104.65|106.74|-1530.75| +2452557|44617|13146|6397|216378|406|3634|10|147|1534|92|96.91|147.30|76.59|0.00|7046.28|8915.72|13551.60|352.31|0.00|7046.28|7398.59|-1869.44| +2452557|44617|13836|6397|216378|406|3634|10|201|1534|13|11.68|15.76|15.12|0.00|196.56|151.84|204.88|17.69|0.00|196.56|214.25|44.72| +2452557|44617|3409|6397|216378|406|3634|10|105|1534|68|97.43|113.01|2.26|0.00|153.68|6625.24|7684.68|9.22|0.00|153.68|162.90|-6471.56| +2452557|44617|14431|6397|216378|406|3634|10|9|1534|60|37.63|43.65|30.99|0.00|1859.40|2257.80|2619.00|148.75|0.00|1859.40|2008.15|-398.40| +2452557|44617|8742|6397|216378|406|3634|10|12|1534|95|40.62|48.33|36.73|0.00|3489.35|3858.90|4591.35|209.36|0.00|3489.35|3698.71|-369.55| +2452557|44617|15391|6397|216378|406|3634|10|186|1534|44|17.73|31.91|14.99|0.00|659.56|780.12|1404.04|26.38|0.00|659.56|685.94|-120.56| +2452557|44617|6816|6397|216378|406|3634|10|48|1534|97|30.70|39.60|19.40|0.00|1881.80|2977.90|3841.20|150.54|0.00|1881.80|2032.34|-1096.10| +2452557|44617|3465|6397|216378|406|3634|10|6|1534|60|64.08|126.87|77.39|0.00|4643.40|3844.80|7612.20|92.86|0.00|4643.40|4736.26|798.60| +2451716|46839|15416|14241|544593|6945|15297|10|270|1535|47|95.73|135.93|6.79|0.00|319.13|4499.31|6388.71|6.38|0.00|319.13|325.51|-4180.18| +2451716|46839|17294|14241|544593|6945|15297|10|195|1535|77|16.00|20.16|15.32|0.00|1179.64|1232.00|1552.32|106.16|0.00|1179.64|1285.80|-52.36| +2451716|46839|7424|14241|544593|6945|15297|10|101|1535|97|18.43|25.98|13.24|1040.26|1284.28|1787.71|2520.06|0.00|1040.26|244.02|244.02|-1543.69| +2451716|46839|6365|14241|544593|6945|15297|10|16|1535|86|98.28|129.72|116.74|0.00|10039.64|8452.08|11155.92|702.77|0.00|10039.64|10742.41|1587.56| +2451716|46839|14035|14241|544593|6945|15297|10|81|1535|12|46.45|91.97|88.29|0.00|1059.48|557.40|1103.64|74.16|0.00|1059.48|1133.64|502.08| +2451716|46839|4451|14241|544593|6945|15297|10|206|1535|98|44.22|57.48|37.93|0.00|3717.14|4333.56|5633.04|223.02|0.00|3717.14|3940.16|-616.42| +2451716|46839|3427||544593|6945|15297||263|1535|54|80.91|131.07||||4369.14|7077.78|||||1151.28| +2451716|46839|10201|14241|544593|6945|15297|10|232|1535|28|67.52|121.53|52.25|0.00|1463.00|1890.56|3402.84|29.26|0.00|1463.00|1492.26|-427.56| +2451716|46839|16033|14241|544593|6945|15297|10|78|1535|99|37.65|49.69|7.95|0.00|787.05|3727.35|4919.31|31.48|0.00|787.05|818.53|-2940.30| +2450960|67621|17120|30622|1403607|1745|31930|7|58|1536|92|59.23|71.66|31.53|2755.72|2900.76|5449.16|6592.72|8.70|2755.72|145.04|153.74|-5304.12| +2450960|67621|10948|30622|1403607|1745|31930|7|257|1536|41|33.62|56.14|40.98|0.00|1680.18|1378.42|2301.74|84.00|0.00|1680.18|1764.18|301.76| +2450960|67621|17383|30622|1403607|1745|31930|7|290|1536|14|91.75|180.74|3.61|0.00|50.54|1284.50|2530.36|1.01|0.00|50.54|51.55|-1233.96| +2450960|67621|11380|30622|1403607|1745|31930|7|192|1536|68|6.60|12.67|1.01|0.00|68.68|448.80|861.56|1.37|0.00|68.68|70.05|-380.12| +2450960|67621|13684|30622|1403607|1745|31930|7|55|1536|83|27.68|36.53|36.53|1546.31|3031.99|2297.44|3031.99|29.71|1546.31|1485.68|1515.39|-811.76| +2450960|67621|11390|30622|1403607|1745|31930|7|221|1536|55|26.26|32.56|20.18|0.00|1109.90|1444.30|1790.80|88.79|0.00|1109.90|1198.69|-334.40| +2450960|67621|6928|30622|1403607|1745|31930|7|186|1536|92|94.82|158.34|82.33|0.00|7574.36|8723.44|14567.28|605.94|0.00|7574.36|8180.30|-1149.08| +2450960|67621|13927|30622|1403607|1745|31930|7|9|1536|2|97.31|178.07|58.76|12.92|117.52|194.62|356.14|4.18|12.92|104.60|108.78|-90.02| +2450960|67621|10768|30622|1403607|1745|31930|7|142|1536|30|8.88|10.56|6.33|0.00|189.90|266.40|316.80|3.79|0.00|189.90|193.69|-76.50| +2450960|67621|14701|30622|1403607|1745|31930|7|272|1536|81|74.11|131.91|110.80|0.00|8974.80|6002.91|10684.71|179.49|0.00|8974.80|9154.29|2971.89| +2450960|67621|17420|30622|1403607|1745|31930|7|287|1536|25|25.40|32.76|17.69|0.00|442.25|635.00|819.00|0.00|0.00|442.25|442.25|-192.75| +2450960|67621|9436|30622|1403607|1745|31930|7|182|1536|74|46.14|65.98|35.62|0.00|2635.88|3414.36|4882.52|26.35|0.00|2635.88|2662.23|-778.48| +2450960|67621|7654|30622|1403607|1745|31930|7|195|1536|43|76.66|144.88|101.41|0.00|4360.63|3296.38|6229.84|43.60|0.00|4360.63|4404.23|1064.25| +2452496|73723|9607|22297|231257|5370|9530|8|257|1537|16|61.74|93.84|26.27|0.00|420.32|987.84|1501.44|8.40|0.00|420.32|428.72|-567.52| +2452496|73723|573|22297|231257|5370|9530|8|122|1537|22|22.77|32.10|26.64|35.16|586.08|500.94|706.20|16.52|35.16|550.92|567.44|49.98| +2452496|73723|2925|22297|231257|5370|9530|8|249|1537|49|83.33|120.82|27.78|0.00|1361.22|4083.17|5920.18|27.22|0.00|1361.22|1388.44|-2721.95| +2452496|73723|12709|22297|231257|5370|9530|8|295|1537|2|31.73|39.66|16.65|0.00|33.30|63.46|79.32|1.66|0.00|33.30|34.96|-30.16| +2452496|73723|16242|22297|231257|5370|9530|8|198|1537|94|27.47|48.34|4.83|0.00|454.02|2582.18|4543.96|22.70|0.00|454.02|476.72|-2128.16| +2452496|73723|17917|22297|231257|5370|9530|8|89|1537|93|73.26|81.31|42.28|0.00|3932.04|6813.18|7561.83|39.32|0.00|3932.04|3971.36|-2881.14| +2452496|73723|6295|22297|||9530|8||1537|14|64.20|||0.00|1504.86|898.80|||0.00|||| +2452496|73723|6859|22297|231257|5370|9530|8|279|1537|55|94.71|99.44|51.70|0.00|2843.50|5209.05|5469.20|227.48|0.00|2843.50|3070.98|-2365.55| +2452496|73723|657|22297|231257|5370|9530|8|260|1537|70|15.03|16.83|12.28|0.00|859.60|1052.10|1178.10|25.78|0.00|859.60|885.38|-192.50| +2452496|73723|11316|22297|231257|5370|9530|8|200|1537|27|12.03|12.03|5.17|78.17|139.59|324.81|324.81|0.00|78.17|61.42|61.42|-263.39| +2452496|73723|17355|22297|231257|5370|9530|8|229|1537|61|77.61|121.07|46.00|0.00|2806.00|4734.21|7385.27|28.06|0.00|2806.00|2834.06|-1928.21| +2452496|73723|14115|22297|231257|5370|9530|8|36|1537|74|86.94|161.70|6.46|0.00|478.04|6433.56|11965.80|23.90|0.00|478.04|501.94|-5955.52| +|73723|15877||231257|||||1537|||114.50||0.00||1954.96|3206.00|228.25|0.00|2853.20||| +2452496|73723|4338|22297|231257|5370|9530|8|184|1537|31|20.82|21.02|4.20|0.00|130.20|645.42|651.62|11.71|0.00|130.20|141.91|-515.22| +2452132|55378|16491|64935|1291940|2945|45550|1|172|1538|54|68.88|81.27|39.82|0.00|2150.28|3719.52|4388.58|64.50|0.00|2150.28|2214.78|-1569.24| +2452132|55378|13331|64935|1291940|2945|45550|1|139|1538|37|6.73|8.21|0.65|0.00|24.05|249.01|303.77|0.48|0.00|24.05|24.53|-224.96| +2452132|55378|16909|64935|1291940|2945|45550|1|24|1538|90|33.35|33.68|13.47|0.00|1212.30|3001.50|3031.20|24.24|0.00|1212.30|1236.54|-1789.20| +2452132|55378|11707|64935|1291940|2945|45550|1|5|1538|77|48.10|60.12|41.48|0.00|3193.96|3703.70|4629.24|223.57|0.00|3193.96|3417.53|-509.74| +2452132|55378|13449|64935|1291940|2945|45550|1|60|1538|94|35.31|69.91|40.54|0.00|3810.76|3319.14|6571.54|76.21|0.00|3810.76|3886.97|491.62| +2452132|55378|16837|64935|1291940|2945|45550|1|212|1538|38|6.85|12.12|9.33|0.00|354.54|260.30|460.56|10.63|0.00|354.54|365.17|94.24| +2452132|55378|7303|64935|1291940|2945|45550|1|136|1538|62|23.98|40.52|3.24|0.00|200.88|1486.76|2512.24|10.04|0.00|200.88|210.92|-1285.88| +2452132|55378|8449|64935|1291940|2945|45550|1|267|1538|87|52.68|80.60|16.12|0.00|1402.44|4583.16|7012.20|70.12|0.00|1402.44|1472.56|-3180.72| +2452132|55378|16429|64935|1291940|2945|45550|1|179|1538|64|91.28|165.21|79.30|0.00|5075.20|5841.92|10573.44|355.26|0.00|5075.20|5430.46|-766.72| +2452132|55378|5329|64935|1291940|2945|45550|1|220|1538|28|15.35|30.70|8.28|0.00|231.84|429.80|859.60|0.00|0.00|231.84|231.84|-197.96| +2452132|55378|16551|64935|1291940|2945|45550|1|33|1538|16|62.52|122.53|37.98|0.00|607.68|1000.32|1960.48|36.46|0.00|607.68|644.14|-392.64| +2452132|55378|51|64935|1291940|2945|45550|1|148|1538|78|58.31|85.71|5.14|0.00|400.92|4548.18|6685.38|20.04|0.00|400.92|420.96|-4147.26| +2452132|55378|13799|64935|1291940|2945|45550|1|154|1538|74|31.37|58.97|10.61|0.00|785.14|2321.38|4363.78|54.95|0.00|785.14|840.09|-1536.24| +2450882|39685|1700|42121|1092545|2969|28993|7|209|1539|53|35.38|36.79|9.19|0.00|487.07|1875.14|1949.87|29.22|0.00|487.07|516.29|-1388.07| +2450882|39685|38|42121|1092545|2969|28993|7|292|1539|92|3.14|5.24|3.40|0.00|312.80|288.88|482.08|18.76|0.00|312.80|331.56|23.92| +2450882|39685|16123|42121|1092545|2969|28993|7|26|1539|67|79.06|94.87|68.30|0.00|4576.10|5297.02|6356.29|320.32|0.00|4576.10|4896.42|-720.92| +2450882|39685|9428|42121|1092545|2969|28993|7|241|1539|54|18.30|27.45|15.92|0.00|859.68|988.20|1482.30|51.58|0.00|859.68|911.26|-128.52| +2450882|39685|4672|42121|1092545|2969|28993|7|145|1539|63|15.99|30.54|18.93|0.00|1192.59|1007.37|1924.02|71.55|0.00|1192.59|1264.14|185.22| +2450882|39685|9970|42121|1092545|2969|28993|7|299|1539|55|40.02|74.43|74.43|982.47|4093.65|2201.10|4093.65|93.33|982.47|3111.18|3204.51|910.08| +2450882|39685|12451|42121|1092545|2969|28993|7|37|1539|34|76.45|92.50|63.82|0.00|2169.88|2599.30|3145.00|65.09|0.00|2169.88|2234.97|-429.42| +2450882|39685|3196|42121|1092545|2969|28993|7|68|1539|55|11.63|16.04|9.94|0.00|546.70|639.65|882.20|43.73|0.00|546.70|590.43|-92.95| +2450882|39685|9854|42121|1092545|2969|28993|7|50|1539|5|3.11|5.81|1.39|1.66|6.95|15.55|29.05|0.37|1.66|5.29|5.66|-10.26| +2450882|39685|5648|42121|1092545|2969|28993|7|277|1539|23|29.86|39.41|14.18|0.00|326.14|686.78|906.43|19.56|0.00|326.14|345.70|-360.64| +2450882|39685|241|42121|1092545|2969|28993|7|4|1539|61|15.58|20.09|6.62|314.97|403.82|950.38|1225.49|0.88|314.97|88.85|89.73|-861.53| +2450882|39685|9838|42121|1092545|2969|28993|7|78|1539|36|16.07|29.89|13.45|0.00|484.20|578.52|1076.04|29.05|0.00|484.20|513.25|-94.32| +2451449|47047|4747|55308|443226|6344|12262|7|31|1540|25|49.64|87.36|41.05|0.00|1026.25|1241.00|2184.00|61.57|0.00|1026.25|1087.82|-214.75| +2451449|47047|3064|55308|443226|6344|12262|7|14|1540|58|18.10|27.15|2.71|0.00|157.18|1049.80|1574.70|9.43|0.00|157.18|166.61|-892.62| +2451449|47047|12866|55308|443226|6344|12262|7|53|1540|82|27.01|38.62|5.79|0.00|474.78|2214.82|3166.84|4.74|0.00|474.78|479.52|-1740.04| +2451449|47047|4105|55308|443226|6344|12262|7|226|1540|98|49.83|69.76|66.27|0.00|6494.46|4883.34|6836.48|584.50|0.00|6494.46|7078.96|1611.12| +2451449|47047|3913|55308|443226|6344|12262|7|110|1540|73|72.96|120.38|81.85|0.00|5975.05|5326.08|8787.74|537.75|0.00|5975.05|6512.80|648.97| +2451449|47047|10021|55308|443226|6344|12262|7|171|1540|93|60.27|107.28|51.49|0.00|4788.57|5605.11|9977.04|383.08|0.00|4788.57|5171.65|-816.54| +2451449|47047|6374|55308|443226|6344|12262|7|122|1540|15|34.20|49.93|6.49|0.00|97.35|513.00|748.95|3.89|0.00|97.35|101.24|-415.65| +2451449|47047|5701|55308|443226|6344|12262|7|76|1540|37|58.76|115.16|57.58|0.00|2130.46|2174.12|4260.92|85.21|0.00|2130.46|2215.67|-43.66| +2451449|47047|3604|55308|443226|6344|12262|7|241|1540|45|59.05|77.35|36.35|0.00|1635.75|2657.25|3480.75|130.86|0.00|1635.75|1766.61|-1021.50| +2451449|47047|10298|55308|443226|6344|12262|7|137|1540|66|91.70|143.05|50.06|0.00|3303.96|6052.20|9441.30|99.11|0.00|3303.96|3403.07|-2748.24| +2451449|47047|6904|55308|443226|6344|12262|7|19|1540|52|10.62|15.08|2.71|15.50|140.92|552.24|784.16|8.77|15.50|125.42|134.19|-426.82| +2451449|47047|356|55308|443226|6344|12262|7|207|1540|4|90.28|115.55|90.12|0.00|360.48|361.12|462.20|14.41|0.00|360.48|374.89|-0.64| +2451449|47047|2059|55308|443226|6344|12262|7|210|1540|60|14.31|19.89|0.39|17.78|23.40|858.60|1193.40|0.33|17.78|5.62|5.95|-852.98| +2451449|47047|13676|55308|443226|6344|12262|7|149|1540|98|77.02|89.34|45.56|0.00|4464.88|7547.96|8755.32|267.89|0.00|4464.88|4732.77|-3083.08| +2451449|47047|17590|55308|443226|6344|12262|7|100|1540|42|35.35|53.02|10.07|0.00|422.94|1484.70|2226.84|38.06|0.00|422.94|461.00|-1061.76| +2451449|47047|14288|55308|443226|6344|12262|7|269|1540|32|92.91|121.71|65.72|0.00|2103.04|2973.12|3894.72|21.03|0.00|2103.04|2124.07|-870.08| +2452123|64169|6869|77263|1792476|3336|14949|4|118|1541|61|57.13|73.69|71.47|0.00|4359.67|3484.93|4495.09|392.37|0.00|4359.67|4752.04|874.74| +2452123||11075||1792476|||||1541|53|30.12|||0.00||1596.36|1612.26|6.11|0.00|305.81||-1290.55| +2452123|64169|5987|77263|1792476|3336|14949|4|239|1541|54|32.32|51.38|51.38|0.00|2774.52|1745.28|2774.52|166.47|0.00|2774.52|2940.99|1029.24| +2452123||7601||1792476||14949||37|1541|55|70.54|||0.00||3879.70|4500.10||0.00|||-504.90| +2452123|64169|9775|77263|1792476|3336|14949|4|219|1541|12|58.75|105.16|35.75|0.00|429.00|705.00|1261.92|25.74|0.00|429.00|454.74|-276.00| +||2833||1792476|||4|118|1541|||57.55|||414.20|1106.80||0.00||414.20||| +2452123|64169|4819|||3336|14949||287|1541||||19.66||1867.70|2375.00||0.00||1867.70||-507.30| +2452123|64169|2929|77263|1792476|3336|14949|4|213|1541|15|17.69|22.82|20.99|0.00|314.85|265.35|342.30|3.14|0.00|314.85|317.99|49.50| +2452123|64169|11697|77263|1792476|3336|14949|4|205|1541|90|23.87|30.79|3.69|0.00|332.10|2148.30|2771.10|6.64|0.00|332.10|338.74|-1816.20| +2451208|37048|13873|98906|1405977|248|32072|4|275|1542|18|52.25|90.39|27.11|0.00|487.98|940.50|1627.02|9.75|0.00|487.98|497.73|-452.52| +2451208|37048|10564|98906|1405977|248|32072|4|127|1542|31|47.70|49.60|38.19|0.00|1183.89|1478.70|1537.60|0.00|0.00|1183.89|1183.89|-294.81| +2451208|37048|17044|98906|1405977|248|32072|4|127|1542|23|19.67|33.24|15.62|0.00|359.26|452.41|764.52|21.55|0.00|359.26|380.81|-93.15| +2451208|37048|7592|98906|1405977|248|32072|4|103|1542|18|48.43|48.43|0.48|0.00|8.64|871.74|871.74|0.34|0.00|8.64|8.98|-863.10| +2451208|37048|6379||||32072|4||1542||94.26|122.53||0.00|||4656.14|26.53|0.00||2680.45|| +2451208|37048|15061|98906|1405977|248|32072|4|153|1542|40|54.79|101.90|50.95|0.00|2038.00|2191.60|4076.00|20.38|0.00|2038.00|2058.38|-153.60| +2451208|37048|7538|98906|1405977|248|32072|4|248|1542|39|20.53|36.13|27.09|0.00|1056.51|800.67|1409.07|73.95|0.00|1056.51|1130.46|255.84| +2451208|37048|2518|98906|1405977|248|32072|4|105|1542|95|40.44|56.21|10.67|0.00|1013.65|3841.80|5339.95|60.81|0.00|1013.65|1074.46|-2828.15| +2451208|37048|17318|98906|1405977|248|32072|4|89|1542|70|2.46|4.89|3.71|2.59|259.70|172.20|342.30|2.57|2.59|257.11|259.68|84.91| +2450881|34482|7600|99965|728724|5930|6689|7|144|1543|81|7.75|9.22|7.74|0.00|626.94|627.75|746.82|50.15|0.00|626.94|677.09|-0.81| +2450881|34482|9775||728724||6689|||1543|23|18.32|31.87||0.00||||49.82|0.00||672.66|| +2450881|34482|2833|99965|728724|5930|6689|7|277|1543|100|44.53|44.97|19.33|0.00|1933.00|4453.00|4497.00|57.99|0.00|1933.00|1990.99|-2520.00| +2450881||4819|||||7|209|1543||88.38|114.89|9.19|0.00|36.76||459.56|2.57|0.00|36.76|39.33|| +2450881|34482|2929|99965|728724|5930|6689|7|252|1543|97|91.01|137.42|87.94|0.00|8530.18|8827.97|13329.74|426.50|0.00|8530.18|8956.68|-297.79| +2450881|34482|11696|99965|728724|5930|6689|7|244|1543|36|4.20|7.81|6.48|0.00|233.28|151.20|281.16|2.33|0.00|233.28|235.61|82.08| +2450881|34482|17743|99965|728724|5930|6689|7|148|1543|73|83.55|108.61|44.53|0.00|3250.69|6099.15|7928.53|195.04|0.00|3250.69|3445.73|-2848.46| +2450881|34482|5006|99965|728724|5930|6689|7|135|1543|12|41.24|41.24|21.03|68.13|252.36|494.88|494.88|16.58|68.13|184.23|200.81|-310.65| +2450881|34482|8942|99965|728724|5930|6689|7|217|1543|30|92.73|102.00|26.52|0.00|795.60|2781.90|3060.00|0.00|0.00|795.60|795.60|-1986.30| +2450881|34482|12316|99965|728724|5930|6689|7|159|1543|44|92.68|136.23|106.25|0.00|4675.00|4077.92|5994.12|233.75|0.00|4675.00|4908.75|597.08| +2450944|61446|17642|48199|49132|3312|38292|1|184|1544|91|49.40|86.45|65.70|0.00|5978.70|4495.40|7866.95|0.00|0.00|5978.70|5978.70|1483.30| +2450944|61446|6055|48199|49132|3312|38292|1|13|1544|41|57.29|82.49|49.49|0.00|2029.09|2348.89|3382.09|40.58|0.00|2029.09|2069.67|-319.80| +2450944|61446|1105|48199|49132|3312|38292|1|110|1544|6|34.59|39.43|5.91|0.00|35.46|207.54|236.58|3.19|0.00|35.46|38.65|-172.08| +2450944|61446|15289|48199|49132|3312|38292|1|101|1544|6|68.96|75.85|48.54|0.00|291.24|413.76|455.10|23.29|0.00|291.24|314.53|-122.52| +2450944|61446|1090|48199|49132|3312|38292|1|50|1544|99|49.74|98.48|49.24|0.00|4874.76|4924.26|9749.52|194.99|0.00|4874.76|5069.75|-49.50| +2450944|61446|1838|48199|49132|3312|38292|1|253|1544|36|24.80|40.67|36.19|0.00|1302.84|892.80|1464.12|65.14|0.00|1302.84|1367.98|410.04| +2450944|61446|7292|48199|49132|3312|38292|1|258|1544|62|49.72|53.69|24.69|0.00|1530.78|3082.64|3328.78|122.46|0.00|1530.78|1653.24|-1551.86| +2450944|61446|10549|48199|49132|3312|38292|1|185|1544|60|43.55|47.90|11.01|0.00|660.60|2613.00|2874.00|0.00|0.00|660.60|660.60|-1952.40| +2450944|61446|11798|48199|49132|3312|38292|1|264|1544|25|60.82|75.41|51.27|333.25|1281.75|1520.50|1885.25|0.00|333.25|948.50|948.50|-572.00| +2450944|61446|2606|48199|49132|3312|38292|1|51|1544|10|97.54|181.42|94.33|471.65|943.30|975.40|1814.20|4.71|471.65|471.65|476.36|-503.75| +2450944|61446|3448|48199|49132|3312|38292|1|187|1544|70|89.69|116.59|66.45|0.00|4651.50|6278.30|8161.30|232.57|0.00|4651.50|4884.07|-1626.80| +2450944|61446|7531|48199|49132|3312|38292|1|25|1544|88|77.33|132.23|15.86|0.00|1395.68|6805.04|11636.24|55.82|0.00|1395.68|1451.50|-5409.36| +2450944|61446|9560|48199|49132|3312|38292|1|164|1544|66|10.02|16.93|0.50|0.00|33.00|661.32|1117.38|1.98|0.00|33.00|34.98|-628.32| +2450944|61446|15404|48199|49132|3312|38292|1|271|1544|21|95.36|104.89|57.68|0.00|1211.28|2002.56|2202.69|72.67|0.00|1211.28|1283.95|-791.28| +2450944|61446|9884|48199|49132|3312|38292|1|71|1544|5|28.79|53.83|14.53|0.00|72.65|143.95|269.15|2.90|0.00|72.65|75.55|-71.30| +2450944|61446|5104|48199|49132|3312|38292|1|272|1544|14|16.48|26.36|8.96|0.00|125.44|230.72|369.04|0.00|0.00|125.44|125.44|-105.28| +2451456|34799|8248|33078|856574|6117|34013|1|25|1545|39|85.22|102.26|88.96|3226.57|3469.44|3323.58|3988.14|7.28|3226.57|242.87|250.15|-3080.71| +2451456|34799|5716|33078|856574|6117|34013|1|64|1545|23|98.57|179.39|93.28|0.00|2145.44|2267.11|4125.97|128.72|0.00|2145.44|2274.16|-121.67| +2451456|34799|986|33078|856574|6117|34013|1|161|1545|86|27.49|40.96|18.43|0.00|1584.98|2364.14|3522.56|142.64|0.00|1584.98|1727.62|-779.16| +2451456|34799|5275|33078|856574|6117|34013|1|3|1545|68|1.30|2.48|0.71|14.00|48.28|88.40|168.64|2.74|14.00|34.28|37.02|-54.12| +2451456|34799|8176|33078|856574|6117|34013|1|275|1545|31|82.03|143.55|81.82|0.00|2536.42|2542.93|4450.05|152.18|0.00|2536.42|2688.60|-6.51| +2451456|34799|6601|33078|856574|6117|34013|1|107|1545|6|65.12|70.98|9.22|0.00|55.32|390.72|425.88|2.21|0.00|55.32|57.53|-335.40| +2451456|34799|10234|33078|856574|6117|34013|1|96|1545|74|42.60|60.06|2.40|0.00|177.60|3152.40|4444.44|5.32|0.00|177.60|182.92|-2974.80| +2451456|34799|10582|33078|856574|6117|34013|1|152|1545|52|15.07|26.67|17.33|0.00|901.16|783.64|1386.84|54.06|0.00|901.16|955.22|117.52| +2451456|34799|11017|33078|856574|6117|34013|1|268|1545|94|96.78|105.49|104.43|0.00|9816.42|9097.32|9916.06|0.00|0.00|9816.42|9816.42|719.10| +2451456|34799|15739|33078|856574|6117|34013|1|96|1545|86|24.10|47.23|25.03|0.00|2152.58|2072.60|4061.78|193.73|0.00|2152.58|2346.31|79.98| +2451456|34799|16141|33078|856574|6117|34013|1|181|1545|50|89.75|151.67|103.13|0.00|5156.50|4487.50|7583.50|360.95|0.00|5156.50|5517.45|669.00| +2451456|34799|12088|33078|856574|6117|34013|1|222|1545|19|25.14|26.14|21.43|0.00|407.17|477.66|496.66|16.28|0.00|407.17|423.45|-70.49| +2451456|34799|11413|33078|856574|6117|34013|1|116|1545|31|97.27|180.92|61.51|0.00|1906.81|3015.37|5608.52|133.47|0.00|1906.81|2040.28|-1108.56| +2451456|34799|5540|33078|856574|6117|34013|1|90|1545|48|13.36|26.72|3.20|0.00|153.60|641.28|1282.56|1.53|0.00|153.60|155.13|-487.68| +2451456|34799|13108|33078|856574|6117|34013|1|226|1545|83|66.41|130.16|14.31|0.00|1187.73|5512.03|10803.28|35.63|0.00|1187.73|1223.36|-4324.30| +2451456|34799|9862|33078|856574|6117|34013|1|215|1545|96|99.99|127.98|120.30|5889.88|11548.80|9599.04|12286.08|509.30|5889.88|5658.92|6168.22|-3940.12| +2452038|67043|2031|86172|1149523|6827|20578|1|208|1546|89|91.60|159.38|46.22|0.00|4113.58|8152.40|14184.82|82.27|0.00|4113.58|4195.85|-4038.82| +2452038|67043|12213|86172|1149523|6827|20578|1|172|1546|64|94.51|104.90|103.85|0.00|6646.40|6048.64|6713.60|66.46|0.00|6646.40|6712.86|597.76| +2452038|67043|7645|86172|1149523|6827|20578|1|261|1546|20|19.54|31.06|9.93|0.00|198.60|390.80|621.20|3.97|0.00|198.60|202.57|-192.20| +2452038|67043|6429|86172|1149523|6827|20578|1|219|1546|96|33.73|34.74|0.69|40.40|66.24|3238.08|3335.04|1.03|40.40|25.84|26.87|-3212.24| +2452038|67043|953|86172|1149523|6827|20578|1|100|1546|28|49.30|81.34|8.13|34.14|227.64|1380.40|2277.52|1.93|34.14|193.50|195.43|-1186.90| +2452038|67043|10897|86172|1149523|6827|20578|1|85|1546|81|66.23|106.63|97.03|0.00|7859.43|5364.63|8637.03|78.59|0.00|7859.43|7938.02|2494.80| +2452038|67043|8595|86172|1149523|6827|20578|1|87|1546|81|55.33|75.24|33.85|0.00|2741.85|4481.73|6094.44|164.51|0.00|2741.85|2906.36|-1739.88| +2452038|67043|5097|86172|1149523|6827|20578|1|135|1546|89|27.95|41.92|23.89|0.00|2126.21|2487.55|3730.88|148.83|0.00|2126.21|2275.04|-361.34| +2452038|67043|12593|86172|1149523|6827|20578|1|99|1546|36|61.77|94.50|89.77|0.00|3231.72|2223.72|3402.00|32.31|0.00|3231.72|3264.03|1008.00| +2452038|67043|2601|86172|1149523|6827|20578|1|236|1546|30|86.04|104.10|12.49|0.00|374.70|2581.20|3123.00|29.97|0.00|374.70|404.67|-2206.50| +2452038|67043|6951|86172|1149523|6827|20578|1|67|1546|51|91.94|93.77|69.38|2299.94|3538.38|4688.94|4782.27|86.69|2299.94|1238.44|1325.13|-3450.50| +2452038|67043|16877|86172|1149523|6827|20578|1|90|1546|21|39.71|48.04|2.40|0.00|50.40|833.91|1008.84|1.00|0.00|50.40|51.40|-783.51| +2452038|67043|12073|86172|1149523|6827|20578|1|259|1546|54|33.97|43.48|15.65|0.00|845.10|1834.38|2347.92|67.60|0.00|845.10|912.70|-989.28| +|67043|1321||1149523|6827||1||1546|61|9.16|14.38||0.00||558.76|877.18|14.03|0.00|||-208.01| +2452038|67043|15891|86172|1149523|6827|20578|1|150|1546|22|32.86|37.13|27.10|310.02|596.20|722.92|816.86|5.72|310.02|286.18|291.90|-436.74| +2452103|69541|6035|73981|1486438|4495|35752|8|208|1547|6|76.01|82.85|61.30|0.00|367.80|456.06|497.10|22.06|0.00|367.80|389.86|-88.26| +2452103||9139|73981|1486438|4495||||1547|14|||13.58|0.00||139.02|||0.00|190.12||| +|69541|12677|73981||4495||8||1547|2||85.75|||51.44||171.50|||51.44|51.44|| +2452103|69541|12975|73981|1486438|4495|35752|8|268|1547|87|16.64|32.28|5.81|0.00|505.47|1447.68|2808.36|30.32|0.00|505.47|535.79|-942.21| +2452103|69541|5673|73981|1486438|4495|35752|8|289|1547|25|46.30|46.30|42.59|0.00|1064.75|1157.50|1157.50|85.18|0.00|1064.75|1149.93|-92.75| +2452103|69541|12669|73981|1486438|4495|35752|8|270|1547|34|28.26|42.39|35.18|0.00|1196.12|960.84|1441.26|35.88|0.00|1196.12|1232.00|235.28| +2452103|69541|1487|73981|1486438|4495|35752|8|55|1547|63|88.50|128.32|28.23|0.00|1778.49|5575.50|8084.16|0.00|0.00|1778.49|1778.49|-3797.01| +2452103|69541|14457|73981|1486438|4495|35752|8|20|1547|64|96.05|97.97|97.97|6207.37|6270.08|6147.20|6270.08|0.00|6207.37|62.71|62.71|-6084.49| +2451872|53029|5015|31081|954475|269|12016|10|274|1548|55|97.27|192.59|113.62|3312.02|6249.10|5349.85|10592.45|146.85|3312.02|2937.08|3083.93|-2412.77| +2451872|53029|3979|31081|954475|269|12016|10|270|1548|82|75.05|120.83|113.58|2328.39|9313.56|6154.10|9908.06|558.81|2328.39|6985.17|7543.98|831.07| +2451872|53029|14228|31081|954475|269|12016|10|263|1548|100|91.89|129.56|44.05|0.00|4405.00|9189.00|12956.00|220.25|0.00|4405.00|4625.25|-4784.00| +2451872|53029|17533|31081|954475|269|12016|10|20|1548|91|94.91|100.60|88.52|3866.55|8055.32|8636.81|9154.60|83.77|3866.55|4188.77|4272.54|-4448.04| +2451872|53029|13543|31081|954475|269|12016|10|86|1548|42|28.31|32.27|19.68|0.00|826.56|1189.02|1355.34|16.53|0.00|826.56|843.09|-362.46| +2451872|53029|13907|31081|954475|269|12016|10|198|1548|1|40.55|68.52|63.72|38.86|63.72|40.55|68.52|1.24|38.86|24.86|26.10|-15.69| +2451872|53029|8017|31081|954475|269|12016|10|204|1548|81|76.60|124.09|34.74|0.00|2813.94|6204.60|10051.29|56.27|0.00|2813.94|2870.21|-3390.66| +2451872|53029|1010|31081|954475|269|12016|10|273|1548|15|26.17|47.10|18.36|0.00|275.40|392.55|706.50|16.52|0.00|275.40|291.92|-117.15| +2451872|53029|7675|31081|954475|269|12016|10|297|1548|70|88.43|168.90|30.40|0.00|2128.00|6190.10|11823.00|148.96|0.00|2128.00|2276.96|-4062.10| +2451872|53029|4478|31081|954475|269|12016|10|37|1548|17|61.79|98.24|81.53|0.00|1386.01|1050.43|1670.08|13.86|0.00|1386.01|1399.87|335.58| +2451872||14533|||||10|74|1548||||32.06||2596.86|4375.62|4900.50|0.00||2596.86||| +2451872|53029|6401|31081|954475|269|12016|10|115|1548|1|29.39|40.85|2.85|0.00|2.85|29.39|40.85|0.00|0.00|2.85|2.85|-26.54| +2451872|53029|4130|31081|954475|269|12016|10|231|1548|91|18.29|29.99|3.29|0.00|299.39|1664.39|2729.09|14.96|0.00|299.39|314.35|-1365.00| +2451872|53029|17618|31081|954475|269|12016|10|81|1548|91|54.71|56.89|10.80|0.00|982.80|4978.61|5176.99|68.79|0.00|982.80|1051.59|-3995.81| +2451872|53029|7903|31081|954475|269|12016|10|269|1548|83|29.51|41.90|24.30|0.00|2016.90|2449.33|3477.70|60.50|0.00|2016.90|2077.40|-432.43| +2452203|34534|3939|51839|1282251|216|3781|4|268|1549|100|44.88|52.06|4.16|232.96|416.00|4488.00|5206.00|7.32|232.96|183.04|190.36|-4304.96| +2452203|34534|14151|51839|1282251|216|3781|4|67|1549|89|18.02|35.49|8.87|0.00|789.43|1603.78|3158.61|0.00|0.00|789.43|789.43|-814.35| +2452203|34534|14167|51839|1282251|216|3781|4|246|1549|93|26.15|28.50|13.39|0.00|1245.27|2431.95|2650.50|74.71|0.00|1245.27|1319.98|-1186.68| +2452203|34534|9453|51839|1282251|216|3781|4|222|1549|1|53.30|75.15|26.30|0.00|26.30|53.30|75.15|1.57|0.00|26.30|27.87|-27.00| +2452203|34534|3715|51839|1282251|216|3781|4|78|1549|58|88.24|142.94|105.77|0.00|6134.66|5117.92|8290.52|184.03|0.00|6134.66|6318.69|1016.74| +2452203|34534|2535|51839|1282251|216|3781|4|218|1549|92|10.78|17.78|3.55|0.00|326.60|991.76|1635.76|9.79|0.00|326.60|336.39|-665.16| +2452203|34534|15965|51839|1282251|216|3781|4|13|1549|94|24.84|43.47|30.86|0.00|2900.84|2334.96|4086.18|116.03|0.00|2900.84|3016.87|565.88| +2452203|34534|15765|51839|1282251|216|3781|4|51|1549|25|10.94|13.01|2.08|0.00|52.00|273.50|325.25|2.60|0.00|52.00|54.60|-221.50| +2452203|34534|9639|51839|1282251|216|3781|4|221|1549|5|26.00|42.38|8.05|0.00|40.25|130.00|211.90|1.61|0.00|40.25|41.86|-89.75| +2452203|34534|7717|51839|1282251|216|3781|4|110|1549|9|46.13|49.82|41.35|0.00|372.15|415.17|448.38|14.88|0.00|372.15|387.03|-43.02| +2452203|34534|73|51839|1282251|216|3781|4|88|1549|1|80.26|126.00|3.78|0.00|3.78|80.26|126.00|0.07|0.00|3.78|3.85|-76.48| +2452203|34534|8271|51839|1282251|216|3781|4|15|1549|13|40.21|61.11|9.16|0.00|119.08|522.73|794.43|5.95|0.00|119.08|125.03|-403.65| +2451621|58882|15440|71960|995174|1941|31908|7|108|1550|92|90.70|148.74|40.15|0.00|3693.80|8344.40|13684.08|221.62|0.00|3693.80|3915.42|-4650.60| +2451621|58882|13399|71960|995174|1941|31908|7|201|1550|94|96.99|156.15|131.16|0.00|12329.04|9117.06|14678.10|0.00|0.00|12329.04|12329.04|3211.98| +2451621|58882|12845|71960|995174|1941|31908|7|60|1550|45|53.65|82.08|47.60|0.00|2142.00|2414.25|3693.60|0.00|0.00|2142.00|2142.00|-272.25| +2451621|58882|10751|71960|995174|1941|31908|7|23|1550|40|93.24|145.45|125.08|0.00|5003.20|3729.60|5818.00|350.22|0.00|5003.20|5353.42|1273.60| +2451621|58882|10112|71960|995174|1941|31908|7|77|1550|53|1.76|2.46|1.42|0.00|75.26|93.28|130.38|3.76|0.00|75.26|79.02|-18.02| +2451621|58882|15524|71960|995174|1941|31908|7|206|1550|6|94.75|176.23|102.21|208.50|613.26|568.50|1057.38|28.33|208.50|404.76|433.09|-163.74| +2451621|58882|7013|71960|995174|1941|31908|7|36|1550|99|75.22|85.75|70.31|0.00|6960.69|7446.78|8489.25|0.00|0.00|6960.69|6960.69|-486.09| +2451621|58882|12731|71960|995174|1941|31908|7|167|1550|57|68.90|112.30|61.76|0.00|3520.32|3927.30|6401.10|316.82|0.00|3520.32|3837.14|-406.98| +2451621|58882|4529|71960|995174|1941|31908|7|218|1550|97|6.27|12.16|9.84|0.00|954.48|608.19|1179.52|38.17|0.00|954.48|992.65|346.29| +2451621|58882|16145|71960|995174|1941|31908|7|56|1550|10|53.91|63.61|15.26|0.00|152.60|539.10|636.10|4.57|0.00|152.60|157.17|-386.50| +2451621|58882|9025|71960|995174|1941|31908|7|113|1550|16|94.18|122.43|19.58|0.00|313.28|1506.88|1958.88|25.06|0.00|313.28|338.34|-1193.60| +2452207|35027|10001|53235|1029093|4951|48409|4|14|1551|3|45.82|55.44|13.30|0.00|39.90|137.46|166.32|2.79|0.00|39.90|42.69|-97.56| +2452207|35027|12427|53235|1029093|4951|48409|4|265|1551|84|47.06|47.06|47.06|0.00|3953.04|3953.04|3953.04|79.06|0.00|3953.04|4032.10|0.00| +2452207|35027|6713|53235|1029093|4951|48409|4|252|1551|46|61.16|91.74|38.53|0.00|1772.38|2813.36|4220.04|141.79|0.00|1772.38|1914.17|-1040.98| +2452207|35027|7365|53235|1029093|4951|48409|4|69|1551|55|20.91|26.34|6.32|0.00|347.60|1150.05|1448.70|13.90|0.00|347.60|361.50|-802.45| +2452207|35027|10387|53235|1029093|4951|48409|4|84|1551|46|58.81|90.56|65.20|0.00|2999.20|2705.26|4165.76|209.94|0.00|2999.20|3209.14|293.94| +2452207|35027|1225|53235|1029093|4951|48409|4|280|1551|39|93.46|150.47|120.37|0.00|4694.43|3644.94|5868.33|422.49|0.00|4694.43|5116.92|1049.49| +2452207|35027|7725|53235|1029093|4951|48409|4|35|1551|71|60.58|69.06|49.03|0.00|3481.13|4301.18|4903.26|278.49|0.00|3481.13|3759.62|-820.05| +2452207|35027|12319|53235|1029093|4951|48409|4|222|1551|21|88.13|88.13|43.18|0.00|906.78|1850.73|1850.73|81.61|0.00|906.78|988.39|-943.95| +2452207|35027|281|53235|1029093|4951|48409|4|247|1551|54|63.27|98.70|89.81|0.00|4849.74|3416.58|5329.80|387.97|0.00|4849.74|5237.71|1433.16| +2452207|35027|15369|53235|1029093|4951|48409|4|258|1551|95|23.01|25.31|9.11|0.00|865.45|2185.95|2404.45|25.96|0.00|865.45|891.41|-1320.50| +2452207|35027|2395|53235|1029093|4951|48409|4|66|1551|26|94.84|168.81|126.60|0.00|3291.60|2465.84|4389.06|131.66|0.00|3291.60|3423.26|825.76| +2452207|35027|17143|53235|1029093|4951|48409|4|86|1551|30|18.76|33.39|22.70|238.35|681.00|562.80|1001.70|22.13|238.35|442.65|464.78|-120.15| +2452207|35027|10349|53235|1029093|4951|48409|4|151|1551|71|59.63|89.44|42.03|2566.35|2984.13|4233.73|6350.24|12.53|2566.35|417.78|430.31|-3815.95| +2452207|35027|4607|53235|1029093|4951|48409|4|89|1551|62|68.83|115.63|111.00|0.00|6882.00|4267.46|7169.06|550.56|0.00|6882.00|7432.56|2614.54| +2451202|33774|278|35475|538379|4836|6140|1|16|1552|65|97.44|148.10|146.61|6575.45|9529.65|6333.60|9626.50|0.00|6575.45|2954.20|2954.20|-3379.40| +2451202|33774|15338|35475|538379|4836|6140|1|281|1552|64|37.25|41.72|39.21|0.00|2509.44|2384.00|2670.08|25.09|0.00|2509.44|2534.53|125.44| +2451202|33774|1540|35475|538379|4836|6140|1|223|1552|14|35.47|45.75|8.23|0.00|115.22|496.58|640.50|2.30|0.00|115.22|117.52|-381.36| +2451202|33774|12577|35475|538379|4836|6140|1|11|1552|95|33.92|41.38|19.44|0.00|1846.80|3222.40|3931.10|18.46|0.00|1846.80|1865.26|-1375.60| +2451202|33774|2114|35475|538379|4836|6140|1|158|1552|47|25.29|43.49|37.40|0.00|1757.80|1188.63|2044.03|52.73|0.00|1757.80|1810.53|569.17| +2451202|33774|7219|35475|538379|4836|6140|1|144|1552|49|82.05|125.53|90.38|2922.88|4428.62|4020.45|6150.97|90.34|2922.88|1505.74|1596.08|-2514.71| +2451202|33774|12067|35475|538379|4836|6140|1|87|1552|81|48.59|54.90|52.15|0.00|4224.15|3935.79|4446.90|42.24|0.00|4224.15|4266.39|288.36| +|33774|830||538379|4836|6140|1|159|1552||||61.99||2541.59||||||2668.66|295.20| +2451202|33774|17302|35475|538379|4836|6140|1|123|1552|81|25.43|41.95|2.93|0.00|237.33|2059.83|3397.95|7.11|0.00|237.33|244.44|-1822.50| +2451202|33774|362|35475|538379|4836|6140|1|165|1552|85|96.10|192.20|69.19|2234.83|5881.15|8168.50|16337.00|72.92|2234.83|3646.32|3719.24|-4522.18| +2451202|33774|9344|35475|538379|4836|6140|1|54|1552|9|51.12|61.34|2.45|0.00|22.05|460.08|552.06|0.88|0.00|22.05|22.93|-438.03| +2451202|33774|12013|35475|538379|4836|6140|1|130|1552|18|52.15|90.74|68.05|0.00|1224.90|938.70|1633.32|24.49|0.00|1224.90|1249.39|286.20| +2451512|37628|8107|79267|6667|6867|12267|4|296|1553|84|46.26|90.20|5.41|0.00|454.44|3885.84|7576.80|0.00|0.00|454.44|454.44|-3431.40| +2451512|37628|4906|79267|6667|6867|12267|4|18|1553|30|32.75|40.93|39.29|0.00|1178.70|982.50|1227.90|58.93|0.00|1178.70|1237.63|196.20| +2451512|37628|16354|79267|6667|6867|12267|4|152|1553|14|28.42|44.90|28.73|0.00|402.22|397.88|628.60|32.17|0.00|402.22|434.39|4.34| +2451512|37628|16636|79267|6667|6867|12267|4|260|1553|78|23.03|23.03|5.52|0.00|430.56|1796.34|1796.34|30.13|0.00|430.56|460.69|-1365.78| +2451512|37628|14197|79267|6667|6867|12267|4|107|1553|83|6.65|9.44|8.49|0.00|704.67|551.95|783.52|14.09|0.00|704.67|718.76|152.72| +2451512|37628|12770|79267|6667|6867|12267|4|45|1553|88|90.77|101.66|11.18|0.00|983.84|7987.76|8946.08|9.83|0.00|983.84|993.67|-7003.92| +2451512|37628|8524|79267|6667|6867|12267|4|37|1553|31|52.55|67.78|41.34|0.00|1281.54|1629.05|2101.18|76.89|0.00|1281.54|1358.43|-347.51| +2451512|37628|10150|79267|6667|6867|12267|4|42|1553|79|7.07|13.99|13.57|0.00|1072.03|558.53|1105.21|64.32|0.00|1072.03|1136.35|513.50| +2451512|37628|8480|79267|6667|6867|12267|4|250|1553|53|31.07|33.55|15.43|0.00|817.79|1646.71|1778.15|57.24|0.00|817.79|875.03|-828.92| +2451512|37628|16210|79267|6667|6867|12267|4|112|1553|71|52.72|88.56|35.42|0.00|2514.82|3743.12|6287.76|50.29|0.00|2514.82|2565.11|-1228.30| +2451512||17180|79267||6867|12267|||1553|19|44.42||84.01|782.13||843.98|1628.87|48.84|782.13|814.06||| +2451512|37628|14290|79267|6667|6867|12267|4|109|1553|7|47.88|95.28|44.78|0.00|313.46|335.16|666.96|21.94|0.00|313.46|335.40|-21.70| +2451512|37628|9418|79267|6667|6867|12267|4|276|1553|94|12.38|14.36|3.59|0.00|337.46|1163.72|1349.84|26.99|0.00|337.46|364.45|-826.26| +2450841|67534|9020|53898|620564|2877|19990|4|129|1554|4|25.08|26.83|16.63|0.00|66.52|100.32|107.32|1.99|0.00|66.52|68.51|-33.80| +2450841|67534|15344|53898|620564|2877|19990|4|140|1554|95|2.58|2.91|1.25|0.00|118.75|245.10|276.45|5.93|0.00|118.75|124.68|-126.35| +2450841|67534|11194|53898|620564|2877|19990|4|116|1554|19|66.77|112.84|21.43|0.00|407.17|1268.63|2143.96|4.07|0.00|407.17|411.24|-861.46| +2450841|67534|8632|53898|620564|2877|19990|4|55|1554|73|80.38|92.43|73.94|0.00|5397.62|5867.74|6747.39|215.90|0.00|5397.62|5613.52|-470.12| +2450841|67534|8185|53898||2877|||236|1554|11|9.21|17.86|9.10||100.10||196.46|4.00||||| +2450841|67534|10310|53898|620564|2877|19990|4|25|1554|64|57.89|60.78|31.60|0.00|2022.40|3704.96|3889.92|20.22|0.00|2022.40|2042.62|-1682.56| +2450841|67534|12541|53898|620564|2877|19990|4|41|1554|51|70.15|115.74|1.15|0.00|58.65|3577.65|5902.74|5.27|0.00|58.65|63.92|-3519.00| +2450841|67534|10252|53898|620564|2877|19990|4|10|1554|31|1.10|1.82|0.38|0.23|11.78|34.10|56.42|0.23|0.23|11.55|11.78|-22.55| +2450841|67534|16604|53898|620564|2877|19990|4|155|1554|69|46.48|61.81|45.12|0.00|3113.28|3207.12|4264.89|280.19|0.00|3113.28|3393.47|-93.84| +2450841|67534|3800|53898|620564|2877|19990|4|273|1554|90|88.55|141.68|52.42|0.00|4717.80|7969.50|12751.20|377.42|0.00|4717.80|5095.22|-3251.70| +2452267|43433|12687|4086|958134|3382|38230|10|221|1555|58|51.67|73.88|42.85|0.00|2485.30|2996.86|4285.04|223.67|0.00|2485.30|2708.97|-511.56| +2452267|43433|15093|4086|958134|3382|38230|10|61|1555|96|33.93|35.28|18.69|0.00|1794.24|3257.28|3386.88|17.94|0.00|1794.24|1812.18|-1463.04| +2452267|43433|1857|4086|958134|3382|38230|10|106|1555|13|34.23|36.28|29.38|0.00|381.94|444.99|471.64|34.37|0.00|381.94|416.31|-63.05| +2452267|43433|3127|4086|958134|3382|38230|10|33|1555|97|57.59|112.87|4.51|0.00|437.47|5586.23|10948.39|39.37|0.00|437.47|476.84|-5148.76| +||10561|4086||3382||||1555|45|||14.97|0.00||3311.10|6125.40||0.00||714.06|-2637.45| +2452267|43433|14625|4086|958134|3382|38230|10|131|1555|22|68.94|102.72|46.22|0.00|1016.84|1516.68|2259.84|81.34|0.00|1016.84|1098.18|-499.84| +2452267|43433|11813|4086|958134|3382|38230|10|264|1555|94|67.26|90.12|70.29|0.00|6607.26|6322.44|8471.28|66.07|0.00|6607.26|6673.33|284.82| +2452267|43433|1997|4086|958134|3382|38230|10|36|1555|89|43.96|65.06|18.21|0.00|1620.69|3912.44|5790.34|97.24|0.00|1620.69|1717.93|-2291.75| +2452267|43433|15463|4086|958134|3382|38230|10|210|1555|23|70.44|107.06|54.60|0.00|1255.80|1620.12|2462.38|50.23|0.00|1255.80|1306.03|-364.32| +2452267|43433|3865|4086|958134|3382|38230|10|280|1555|51|29.37|30.54|29.01|0.00|1479.51|1497.87|1557.54|133.15|0.00|1479.51|1612.66|-18.36| +2452267|43433|8933|4086|958134|3382|38230|10|189|1555|28|80.42|86.85|19.97|0.00|559.16|2251.76|2431.80|39.14|0.00|559.16|598.30|-1692.60| +2452267|43433|8939|4086|958134|3382|38230|10|261|1555|25|84.87|110.33|57.37|0.00|1434.25|2121.75|2758.25|57.37|0.00|1434.25|1491.62|-687.50| +2451557|43920|1703|79149|1648867|985|34303|1|295|1556|3|42.20|78.49|51.01|0.00|153.03|126.60|235.47|3.06|0.00|153.03|156.09|26.43| +2451557|43920|1775|79149|1648867|985|34303|1|128|1556|97|47.64|88.13|17.62|0.00|1709.14|4621.08|8548.61|119.63|0.00|1709.14|1828.77|-2911.94| +2451557|43920|13700|79149|1648867|985|34303|1|214|1556|1|4.83|6.66|5.92|0.00|5.92|4.83|6.66|0.11|0.00|5.92|6.03|1.09| +2451557|43920|15512|79149|1648867|985|34303|1|167|1556|16|11.30|17.17|17.17|0.00|274.72|180.80|274.72|24.72|0.00|274.72|299.44|93.92| +2451557|43920|10421|79149|1648867|985|34303|1|298|1556|45|93.14|118.28|7.09|0.00|319.05|4191.30|5322.60|3.19|0.00|319.05|322.24|-3872.25| +2451557|43920|13099|79149|1648867|985|34303|1|140|1556|82|85.70|163.68|147.31|0.00|12079.42|7027.40|13421.76|0.00|0.00|12079.42|12079.42|5052.02| +2451557|43920|10985|79149|1648867|985|34303|1|214|1556|82|33.59|33.59|3.69|0.00|302.58|2754.38|2754.38|21.18|0.00|302.58|323.76|-2451.80| +2451557|43920|7712|79149|1648867|985|34303|1|175|1556|79|83.65|123.80|63.13|0.00|4987.27|6608.35|9780.20|398.98|0.00|4987.27|5386.25|-1621.08| +2452585||6714||1833814||||207|1557||1.78|||||33.82||||||-14.44| +2452585|63995|7365|15220|1833814|4761|25755|4|198|1557|54|80.18|128.28|119.30|0.00|6442.20|4329.72|6927.12|64.42|0.00|6442.20|6506.62|2112.48| +2452585|63995|10387|15220|1833814|4761|25755|4|73|1557|71|71.10|86.74|78.93|0.00|5604.03|5048.10|6158.54|336.24|0.00|5604.03|5940.27|555.93| +2452585|63995|1225|15220|1833814|4761|25755|4|63|1557|88|25.13|36.43|25.13|0.00|2211.44|2211.44|3205.84|154.80|0.00|2211.44|2366.24|0.00| +2452585|63995|7725|15220|1833814|4761|25755|4|108|1557|64|54.58|72.59|8.71|0.00|557.44|3493.12|4645.76|22.29|0.00|557.44|579.73|-2935.68| +2452585|63995|12319|15220|1833814|4761|25755|4|10|1557|98|6.85|6.85|6.02|0.00|589.96|671.30|671.30|23.59|0.00|589.96|613.55|-81.34| +2452585|63995|282|15220|1833814|4761|25755|4|187|1557|30|73.74|100.28|27.07|0.00|812.10|2212.20|3008.40|8.12|0.00|812.10|820.22|-1400.10| +2452585||15369|||4761||||1557|||23.28|8.84||450.84|||22.54|||473.38|| +2452585|63995|2395|15220|1833814|4761|25755|4|57|1557|61|60.47|64.70|52.40|0.00|3196.40|3688.67|3946.70|287.67|0.00|3196.40|3484.07|-492.27| +2452585|63995|17143|15220|1833814|4761|25755|4|274|1557|5|96.53|123.55|72.89|72.89|364.45|482.65|617.75|0.00|72.89|291.56|291.56|-191.09| +2452585|63995|10350|15220|1833814|4761|25755|4|288|1557|70|4.47|7.68|5.76|16.12|403.20|312.90|537.60|27.09|16.12|387.08|414.17|74.18| +2452585|63995|4608|15220|1833814|4761|25755|4|277|1557|1|36.58|65.47|16.36|0.00|16.36|36.58|65.47|1.30|0.00|16.36|17.66|-20.22| +2452585|63995|4069|15220|1833814|4761|25755|4|201|1557|60|1.71|2.47|0.12|0.00|7.20|102.60|148.20|0.50|0.00|7.20|7.70|-95.40| +2452585|63995|4293|15220|1833814|4761|25755|4|60|1557|20|46.41|92.82|90.03|0.00|1800.60|928.20|1856.40|18.00|0.00|1800.60|1818.60|872.40| +2452585|63995|7839|15220|1833814|4761|25755|4|170|1557|44|93.11|123.83|78.01|0.00|3432.44|4096.84|5448.52|102.97|0.00|3432.44|3535.41|-664.40| +2451407|59010|16976|51846|376715|5584|20053|7|175|1558|8|71.61|137.49|67.37|0.00|538.96|572.88|1099.92|5.38|0.00|538.96|544.34|-33.92| +2451407|59010|3490|51846|376715|5584|20053|7|137|1558|42|7.55|12.98|4.54|0.00|190.68|317.10|545.16|3.81|0.00|190.68|194.49|-126.42| +2451407|59010|10912|51846|376715|5584|20053|7|231|1558|89|20.92|22.17|2.43|0.00|216.27|1861.88|1973.13|19.46|0.00|216.27|235.73|-1645.61| +2451407|59010|9976|51846|376715|5584|20053|7|87|1558|37|41.17|42.81|12.41|0.00|459.17|1523.29|1583.97|0.00|0.00|459.17|459.17|-1064.12| +2451407|59010|17218|51846|376715|5584|20053|7|117|1558|80|40.75|72.94|64.91|0.00|5192.80|3260.00|5835.20|311.56|0.00|5192.80|5504.36|1932.80| +2451407|59010|2480|51846|376715|5584|20053|7|66|1558|77|42.74|61.54|27.69|1257.95|2132.13|3290.98|4738.58|26.22|1257.95|874.18|900.40|-2416.80| +2451407|59010|15482|51846|376715|5584|20053|7|102|1558|59|2.80|2.99|1.67|0.00|98.53|165.20|176.41|0.00|0.00|98.53|98.53|-66.67| +2451407|59010|9608|51846|376715|5584|20053|7|125|1558|94|17.96|29.45|23.56|0.00|2214.64|1688.24|2768.30|22.14|0.00|2214.64|2236.78|526.40| +2451407|59010|13546|51846|376715|5584|20053|7|61|1558|82|84.21|133.05|129.05|8359.85|10582.10|6905.22|10910.10|22.22|8359.85|2222.25|2244.47|-4682.97| +2451407|59010|17116|51846|376715|5584|20053|7|213|1558|99|18.06|30.34|1.51|49.33|149.49|1787.94|3003.66|7.01|49.33|100.16|107.17|-1687.78| +2451407|59010|10054|51846|376715|5584|20053|7|67|1558|83|44.96|55.30|42.02|0.00|3487.66|3731.68|4589.90|34.87|0.00|3487.66|3522.53|-244.02| +2451377|50651|439|1413|1064044|2322|38200|1|216|1559|9|24.73|25.71|7.97|0.00|71.73|222.57|231.39|0.00|0.00|71.73|71.73|-150.84| +2451377|50651|4762|1413|1064044|2322|38200|1|32|1559|72|98.53|154.69|108.28|0.00|7796.16|7094.16|11137.68|623.69|0.00|7796.16|8419.85|702.00| +2451377|50651|15049|1413|1064044|2322|38200|1|154|1559|77|48.75|67.27|63.90|0.00|4920.30|3753.75|5179.79|344.42|0.00|4920.30|5264.72|1166.55| +2451377|50651|964|1413|1064044|2322|38200|1|213|1559|74|49.94|90.39|50.61|1460.60|3745.14|3695.56|6688.86|91.38|1460.60|2284.54|2375.92|-1411.02| +2451377||14455|1413||2322|||27|1559|||92.37||||76.98|92.37|2.06|||27.92|| +2451377|50651|8300|1413|1064044|2322|38200|1|124|1559|95|38.71|76.64|75.87|0.00|7207.65|3677.45|7280.80|504.53|0.00|7207.65|7712.18|3530.20| +2451377|50651|13090|1413|1064044|2322|38200|1|235|1559|7|49.71|97.43|0.00|0.00|0.00|347.97|682.01|0.00|0.00|0.00|0.00|-347.97| +2451377|50651|1663|1413|1064044|2322|38200|1|41|1559|44|39.11|46.54|17.68|466.75|777.92|1720.84|2047.76|18.67|466.75|311.17|329.84|-1409.67| +2451377|50651|15962|1413|1064044|2322|38200|1|83|1559|64|37.97|41.00|8.61|0.00|551.04|2430.08|2624.00|33.06|0.00|551.04|584.10|-1879.04| +2451377|50651|9949|1413|1064044||||98|1559|17|85.19|130.34|28.67|0.00||1448.23|2215.78|9.74|0.00|||| +2451377|50651|6944|1413|1064044|2322|38200|1|255|1559|93|6.74|8.42|7.40|0.00|688.20|626.82|783.06|27.52|0.00|688.20|715.72|61.38| +2451377||14140||1064044|2322|38200|||1559|||165.97|162.65||||663.88||||101.49|-271.25| +2451377|50651|13486|1413|1064044|2322|38200|1|135|1559|100|27.14|49.66|29.29|0.00|2929.00|2714.00|4966.00|234.32|0.00|2929.00|3163.32|215.00| +2451377|50651|17518|1413|1064044|2322|38200|1|15|1559|52|62.46|91.19|54.71|0.00|2844.92|3247.92|4741.88|113.79|0.00|2844.92|2958.71|-403.00| +2451377|50651|7111|1413|1064044|2322|38200|1|250|1559|43|3.31|5.56|0.61|16.26|26.23|142.33|239.08|0.59|16.26|9.97|10.56|-132.36| +2451599|61555|5773|44470|1437837|6051|15818|8|220|1560|83|23.48|35.68|25.68|0.00|2131.44|1948.84|2961.44|63.94|0.00|2131.44|2195.38|182.60| +2451599|61555|4691|44470|1437837|6051|15818|8|104|1560|74|17.70|31.50|4.72|150.19|349.28|1309.80|2331.00|3.98|150.19|199.09|203.07|-1110.71| +2451599|61555|4364|44470|1437837|6051|15818|8|166|1560|62|42.75|56.43|46.83|0.00|2903.46|2650.50|3498.66|116.13|0.00|2903.46|3019.59|252.96| +2451599|61555|15451|44470|1437837|6051|15818|8|275|1560|57|77.29|134.48|96.82|0.00|5518.74|4405.53|7665.36|220.74|0.00|5518.74|5739.48|1113.21| +2451599|61555|13843|44470|1437837|6051|15818|8|182|1560|6|95.23|173.31|34.66|0.00|207.96|571.38|1039.86|10.39|0.00|207.96|218.35|-363.42| +2451599|61555|4739|44470|1437837|6051|15818|8|125|1560|17|41.48|77.56|55.06|0.00|936.02|705.16|1318.52|28.08|0.00|936.02|964.10|230.86| +2451599|61555|16333|44470|1437837|6051|15818|8|125|1560|91|51.13|51.64|19.10|1564.29|1738.10|4652.83|4699.24|12.16|1564.29|173.81|185.97|-4479.02| +2451599|61555|266|44470|1437837|6051|15818|8|186|1560|50|43.74|56.42|37.80|1134.00|1890.00|2187.00|2821.00|52.92|1134.00|756.00|808.92|-1431.00| +2451599|61555|2863|44470|1437837|6051|15818|8|85|1560|29|35.58|40.91|27.81|354.85|806.49|1031.82|1186.39|18.06|354.85|451.64|469.70|-580.18| +2451599|61555|14339|44470|1437837|6051|15818|8|227|1560|80|86.96|116.52|37.28|0.00|2982.40|6956.80|9321.60|59.64|0.00|2982.40|3042.04|-3974.40| +2451599|61555|3707|44470|1437837|6051|15818|8|22|1560|24|72.29|113.49|3.40|0.00|81.60|1734.96|2723.76|0.81|0.00|81.60|82.41|-1653.36| +2451599|61555|8198|44470|1437837|6051|15818|8|110|1560|13|61.20|100.98|67.65|0.00|879.45|795.60|1312.74|61.56|0.00|879.45|941.01|83.85| +2451599|61555|14450|44470|1437837|6051|15818|8|298|1560|10|10.25|16.40|4.42|0.00|44.20|102.50|164.00|0.00|0.00|44.20|44.20|-58.30| +2451599|61555|8819|44470|1437837|6051|15818|8|2|1560|37|34.76|52.83|39.09|0.00|1446.33|1286.12|1954.71|14.46|0.00|1446.33|1460.79|160.21| +2451599|61555|17270|44470|1437837|6051|15818|8|77|1560|61|20.07|29.50|23.01|0.00|1403.61|1224.27|1799.50|42.10|0.00|1403.61|1445.71|179.34| +2451172|61215|15589|93093|873071|5496|3121|7|145|1561|26|29.28|48.01|3.84|0.00|99.84|761.28|1248.26|8.98|0.00|99.84|108.82|-661.44| +2451172|61215|1144|93093|873071|5496|3121|7|293|1561|35|41.02|41.84|4.60|33.81|161.00|1435.70|1464.40|6.35|33.81|127.19|133.54|-1308.51| +2451172|61215|7969|93093|873071|5496|3121|7|80|1561|40|67.45|124.10|14.89|0.00|595.60|2698.00|4964.00|0.00|0.00|595.60|595.60|-2102.40| +2451172|61215|8134|93093|873071|5496|3121|7|81|1561|71|66.40|66.40|28.55|0.00|2027.05|4714.40|4714.40|0.00|0.00|2027.05|2027.05|-2687.35| +2451172|61215|253|93093|873071|5496|3121|7|213|1561|32|98.66|133.19|37.29|0.00|1193.28|3157.12|4262.08|23.86|0.00|1193.28|1217.14|-1963.84| +2451172|61215|3508|93093|873071|5496|3121|7|162|1561|33|36.47|67.10|30.86|1018.38|1018.38|1203.51|2214.30|0.00|1018.38|0.00|0.00|-1203.51| +2451172|61215|14390|93093|873071|5496|3121|7|107|1561|62|44.34|67.84|27.81|0.00|1724.22|2749.08|4206.08|120.69|0.00|1724.22|1844.91|-1024.86| +2451172|61215|4928|93093|873071|5496|3121|7|262|1561|42|59.75|100.38|82.31|0.00|3457.02|2509.50|4215.96|103.71|0.00|3457.02|3560.73|947.52| +2451172|61215|5440|93093|873071|5496|3121|7|216|1561|1|70.73|106.09|23.33|0.00|23.33|70.73|106.09|1.16|0.00|23.33|24.49|-47.40| +2451172|61215|15019|93093|873071|5496|3121|7|32|1561|56|64.32|66.24|38.41|0.00|2150.96|3601.92|3709.44|0.00|0.00|2150.96|2150.96|-1450.96| +2451172|61215|6559|93093|873071|5496|3121|7|179|1561|28|45.77|45.77|45.77|0.00|1281.56|1281.56|1281.56|51.26|0.00|1281.56|1332.82|0.00| +2452616|41368|474|39499|107036|4620|48157|7|119|1562|35|61.74|82.11|65.68|0.00|2298.80|2160.90|2873.85|22.98|0.00|2298.80|2321.78|137.90| +2452616|41368|13608|39499|107036|4620|48157|7|188|1562|90|33.52|57.65|17.87|128.66|1608.30|3016.80|5188.50|14.79|128.66|1479.64|1494.43|-1537.16| +2452616|41368|3625|39499|107036|4620|48157|7|76|1562|69|7.68|12.21|5.25|0.00|362.25|529.92|842.49|14.49|0.00|362.25|376.74|-167.67| +2452616|41368|3967|39499|107036|4620|48157|7|10|1562|16|94.82|146.97|69.07|0.00|1105.12|1517.12|2351.52|44.20|0.00|1105.12|1149.32|-412.00| +2452616|41368|8898|39499|107036|4620|48157|7|130|1562|56|80.08|128.92|128.92|0.00|7219.52|4484.48|7219.52|649.75|0.00|7219.52|7869.27|2735.04| +2452616|41368|9210|39499|107036|4620|48157|7|268|1562|22|60.06|97.89|32.30|0.00|710.60|1321.32|2153.58|35.53|0.00|710.60|746.13|-610.72| +2452616|41368|969|39499|107036|4620|48157|7|106|1562|44|77.62|135.83|42.10|0.00|1852.40|3415.28|5976.52|55.57|0.00|1852.40|1907.97|-1562.88| +2452616|41368|3211|39499|107036|4620|48157|7|31|1562|20|44.62|56.66|41.36|0.00|827.20|892.40|1133.20|57.90|0.00|827.20|885.10|-65.20| +2452616|41368|2553|39499|107036|4620|48157|7|64|1562|100|92.76|179.95|97.17|0.00|9717.00|9276.00|17995.00|680.19|0.00|9717.00|10397.19|441.00| +2452616|41368|14251|39499|107036|4620|48157|7|121|1562|12|42.07|55.95|13.98|0.00|167.76|504.84|671.40|13.42|0.00|167.76|181.18|-337.08| +2452616||17358|39499||4620|48157|7|185|1562|77|57.32||31.27|||4413.64||141.57|||2164.12|-2391.09| +2452616|41368|2079|39499|107036|4620|48157|7|234|1562|37|37.69|67.46|8.09|0.00|299.33|1394.53|2496.02|17.95|0.00|299.33|317.28|-1095.20| +2452616|41368|8845|39499|107036|4620|48157|7|206|1562|74|31.66|43.69|9.61|0.00|711.14|2342.84|3233.06|35.55|0.00|711.14|746.69|-1631.70| +2451883|64300|13577|22231|1157669|5907|26145|8|19|1563|82|51.28|63.07|23.33|1779.14|1913.06|4204.96|5171.74|6.69|1779.14|133.92|140.61|-4071.04| +2451883|64300|5959|22231|1157669|5907|26145|8|94|1563|69|41.99|51.22|21.00|594.09|1449.00|2897.31|3534.18|34.19|594.09|854.91|889.10|-2042.40| +2451883|64300|1640|22231|1157669|5907|26145|8|265|1563|80|15.86|19.66|5.50|0.00|440.00|1268.80|1572.80|13.20|0.00|440.00|453.20|-828.80| +|64300|10603|22231|1157669||26145|||1563|||181.32|157.74|0.00||366.32|||0.00|630.96||264.64| +2451883|64300|16775|22231|1157669|5907|26145|8|43|1563|99|21.52|41.31|15.69|745.58|1553.31|2130.48|4089.69|56.54|745.58|807.73|864.27|-1322.75| +2451883|64300|9241|22231|1157669|5907|26145|8|218|1563|78|15.38|15.53|4.96|0.00|386.88|1199.64|1211.34|11.60|0.00|386.88|398.48|-812.76| +2451883|64300|9389|22231|1157669|5907|26145|8|27|1563|62|67.78|119.97|93.57|0.00|5801.34|4202.36|7438.14|116.02|0.00|5801.34|5917.36|1598.98| +2451883|64300|625|22231|1157669|5907|26145|8|195|1563|12|80.21|83.41|79.23|0.00|950.76|962.52|1000.92|9.50|0.00|950.76|960.26|-11.76| +2451951|41090|5405|42818|572409|400|15144|4|148|1564|42|77.38|151.66|56.11|0.00|2356.62|3249.96|6369.72|141.39|0.00|2356.62|2498.01|-893.34| +2451951|41090|6581|42818|572409|400|15144|4|60|1564|57|97.15|135.03|27.00|1061.91|1539.00|5537.55|7696.71|38.16|1061.91|477.09|515.25|-5060.46| +2451951|41090|4111|42818|572409|400|15144|4|30|1564|77|52.44|87.57|1.75|0.00|134.75|4037.88|6742.89|12.12|0.00|134.75|146.87|-3903.13| +2451951|41090|17419|42818|572409|400|15144|4|177|1564|64|10.76|18.50|18.50|0.00|1184.00|688.64|1184.00|23.68|0.00|1184.00|1207.68|495.36| +2451951|41090|13201|42818|572409|400|15144|4|92|1564|52|94.21|144.14|122.51|5860.87|6370.52|4898.92|7495.28|10.19|5860.87|509.65|519.84|-4389.27| +2451951|41090|727|42818|572409|400|15144|4|244|1564|98|44.74|73.37|68.23|5349.23|6686.54|4384.52|7190.26|0.00|5349.23|1337.31|1337.31|-3047.21| +2451951|41090|11809|42818|572409|400|15144|4|232|1564|85|35.35|67.51|10.80|0.00|918.00|3004.75|5738.35|64.26|0.00|918.00|982.26|-2086.75| +2451951|41090|2065|42818|572409|400|15144|4|6|1564|17|10.90|14.60|5.25|0.00|89.25|185.30|248.20|0.89|0.00|89.25|90.14|-96.05| +2451951|41090|8493|42818|572409|400|15144|4|293|1564|86|80.04|110.45|101.61|0.00|8738.46|6883.44|9498.70|87.38|0.00|8738.46|8825.84|1855.02| +2451951|41090|3663|42818|572409|400|15144|4|50|1564|62|87.10|134.13|22.80|0.00|1413.60|5400.20|8316.06|14.13|0.00|1413.60|1427.73|-3986.60| +2452321|43964|1189|90728|396286|5491|36249|7|193|1565|96|60.65|65.50|41.26|2455.79|3960.96|5822.40|6288.00|60.20|2455.79|1505.17|1565.37|-4317.23| +2452321|43964|12744|90728|396286|5491|36249|7|70|1565|91|50.56|93.03|3.72|0.00|338.52|4600.96|8465.73|27.08|0.00|338.52|365.60|-4262.44| +2452321|43964|1521|90728|396286|5491|36249|7|91|1565|43|41.13|57.58|32.82|0.00|1411.26|1768.59|2475.94|28.22|0.00|1411.26|1439.48|-357.33| +2452321|43964|12375|90728|396286|5491|36249|7|45|1565|49|64.71|78.94|63.94|626.61|3133.06|3170.79|3868.06|75.19|626.61|2506.45|2581.64|-664.34| +2452321|43964|5235|90728|396286|5491|36249|7|285|1565|29|95.20|101.86|100.84|0.00|2924.36|2760.80|2953.94|116.97|0.00|2924.36|3041.33|163.56| +|43964|15363|||5491|36249||215|1565|99||81.87|||7699.23|||230.97||7699.23||| +||16818|90728|396286|5491|36249|7|227|1565||||||||1692.60|||||| +2452321|43964|6039|90728|396286|5491|36249|7|199|1565|81|83.71|106.31|87.17|0.00|7060.77|6780.51|8611.11|141.21|0.00|7060.77|7201.98|280.26| +2452298||11535||23869||8252|10||1566|100|63.79||3.60|0.00|360.00||7208.00||0.00||370.80|-6019.00| +2452298|36041|15975|66501|23869|4884|8252|10|126|1566|67|80.62|118.51|53.32|0.00|3572.44|5401.54|7940.17|0.00|0.00|3572.44|3572.44|-1829.10| +2452298|36041|4405|66501|23869|4884|8252|10|2|1566|71|86.07|146.31|84.85|5542.40|6024.35|6110.97|10388.01|43.37|5542.40|481.95|525.32|-5629.02| +2452298|36041|13963|66501|23869|4884|8252|10|226|1566|51|72.15|105.33|20.01|183.69|1020.51|3679.65|5371.83|0.00|183.69|836.82|836.82|-2842.83| +2452298|36041|9939|66501|23869|4884|8252|10|89|1566|38|44.82|81.57|67.70|0.00|2572.60|1703.16|3099.66|154.35|0.00|2572.60|2726.95|869.44| +2452298|36041|9891|66501|23869|4884|8252|10|238|1566|91|93.54|137.50|137.50|0.00|12512.50|8512.14|12512.50|625.62|0.00|12512.50|13138.12|4000.36| +2452298|36041|17676|66501|23869|4884|8252|10|176|1566|40|99.26|137.97|12.41|4.96|496.40|3970.40|5518.80|19.65|4.96|491.44|511.09|-3478.96| +2452298||16915||||8252|10||1566||73.71|||0.00|3170.88||5872.80||0.00|3170.88|3266.00|-367.20| +2452581|58947|7632|85279|1552989|3099|9|1|144|1567|56|29.36|43.45|13.90|0.00|778.40|1644.16|2433.20|70.05|0.00|778.40|848.45|-865.76| +2452581|58947|10464|85279|1552989|3099|9|1|31|1567|95|46.26|46.26|12.02|0.00|1141.90|4394.70|4394.70|57.09|0.00|1141.90|1198.99|-3252.80| +2452581|58947|5058|85279|1552989|3099|9|1|226|1567|92|92.75|93.67|92.73|0.00|8531.16|8533.00|8617.64|511.86|0.00|8531.16|9043.02|-1.84| +2452581|58947|9153|85279|1552989|3099|9|1|221|1567|83|89.25|139.23|15.31|0.00|1270.73|7407.75|11556.09|63.53|0.00|1270.73|1334.26|-6137.02| +2452581|58947|3642|85279|1552989|3099|9|1|20|1567|39|63.94|127.88|52.43|1206.41|2044.77|2493.66|4987.32|0.00|1206.41|838.36|838.36|-1655.30| +2452581|58947|13818|85279|1552989|3099|9|1|205|1567|9|87.45|125.05|107.54|396.82|967.86|787.05|1125.45|5.71|396.82|571.04|576.75|-216.01| +2452581|58947|16182|85279|1552989|3099|9|1|218|1567|32|88.92|129.82|11.68|213.04|373.76|2845.44|4154.24|12.85|213.04|160.72|173.57|-2684.72| +2452581|58947|15199|85279|1552989|3099|9|1|279|1567|52|32.27|45.17|43.36|0.00|2254.72|1678.04|2348.84|0.00|0.00|2254.72|2254.72|576.68| +2452581|58947|16263|85279|1552989|3099|9|1|236|1567|37|65.07|76.13|30.45|0.00|1126.65|2407.59|2816.81|56.33|0.00|1126.65|1182.98|-1280.94| +2452581|58947|5299|85279|1552989|3099|9|1|294|1567|82|82.35|90.58|9.96|0.00|816.72|6752.70|7427.56|49.00|0.00|816.72|865.72|-5935.98| +2452581|58947|14631|85279|1552989|3099|9|1|70|1567|91|82.27|141.50|103.29|0.00|9399.39|7486.57|12876.50|0.00|0.00|9399.39|9399.39|1912.82| +2452581|58947|16944|85279|1552989|3099|9|1|128|1567|22|33.98|47.23|28.81|0.00|633.82|747.56|1039.06|19.01|0.00|633.82|652.83|-113.74| +2452539|53125|1929|59449|226170|6491|17659|10|136|1568|52|85.94|91.09|91.09|0.00|4736.68|4468.88|4736.68|47.36|0.00|4736.68|4784.04|267.80| +2452539|53125|12540|59449|226170|6491|17659|10|229|1568|51|52.24|59.55|23.82|0.00|1214.82|2664.24|3037.05|97.18|0.00|1214.82|1312.00|-1449.42| +2452539|53125|5544|59449|226170|6491|17659|10|233|1568|18|74.30|121.10|88.40|0.00|1591.20|1337.40|2179.80|0.00|0.00|1591.20|1591.20|253.80| +2452539|53125|7968|59449|226170|6491|17659|10|83|1568|76|72.95|140.06|78.43|0.00|5960.68|5544.20|10644.56|0.00|0.00|5960.68|5960.68|416.48| +2452539|53125|4446|59449|226170|6491|17659|10|190|1568|56|81.90|128.58|5.14|0.00|287.84|4586.40|7200.48|5.75|0.00|287.84|293.59|-4298.56| +2452539|53125|3489|59449|226170|6491|17659|10|227|1568|57|21.73|31.94|10.54|0.00|600.78|1238.61|1820.58|54.07|0.00|600.78|654.85|-637.83| +2452539|53125|1758|59449|226170|6491|17659|10|29|1568|16|73.62|78.77|7.87|0.00|125.92|1177.92|1260.32|1.25|0.00|125.92|127.17|-1052.00| +2452539|53125|5311|59449|226170|6491|17659|10|210|1568|54|90.90|166.34|133.07|0.00|7185.78|4908.60|8982.36|431.14|0.00|7185.78|7616.92|2277.18| +2452539|53125|14743|59449|226170|6491|17659|10|220|1568|34|8.91|17.73|8.33|0.00|283.22|302.94|602.82|22.65|0.00|283.22|305.87|-19.72| +2452539|53125|14293|59449|226170|6491|17659|10|102|1568|36|31.47|37.44|1.12|0.00|40.32|1132.92|1347.84|3.62|0.00|40.32|43.94|-1092.60| +2452539|53125|10869|59449|226170|6491|17659|10|85|1568|2|9.84|10.92|7.64|0.00|15.28|19.68|21.84|0.30|0.00|15.28|15.58|-4.40| +2452539||3813|||||||1568|||81.95|22.12|||4387.11||||||| +2452539|53125|7147|59449|226170|6491|17659|10|87|1568|17|71.11|98.13|4.90|0.00|83.30|1208.87|1668.21|0.00|0.00|83.30|83.30|-1125.57| +2452351|30158|11491|21854|685434|3861|8688|10|26|1569|47|26.89|42.48|4.24|0.00|199.28|1263.83|1996.56|0.00|0.00|199.28|199.28|-1064.55| +2452351|30158|10170|21854|685434|3861|8688|10|230|1569|37|35.05|36.80|11.04|102.12|408.48|1296.85|1361.60|21.44|102.12|306.36|327.80|-990.49| +2452351|30158|10723|21854|685434|3861|8688|10|224|1569|13|77.23|122.02|0.00|0.00|0.00|1003.99|1586.26|0.00|0.00|0.00|0.00|-1003.99| +2452351|30158|4017|21854|685434|3861|8688|10|153|1569|61|26.87|52.39|22.00|201.30|1342.00|1639.07|3195.79|91.25|201.30|1140.70|1231.95|-498.37| +2452351|30158|3919|21854|685434|3861|8688|10|187|1569|54|6.48|11.53|5.18|0.00|279.72|349.92|622.62|0.00|0.00|279.72|279.72|-70.20| +2452351|30158|9483|21854|685434|3861|8688|10|179|1569|100|4.99|6.23|2.80|0.00|280.00|499.00|623.00|25.20|0.00|280.00|305.20|-219.00| +2452351|30158|9435|21854|685434|3861|8688|10|294|1569|66|10.45|19.85|6.74|0.00|444.84|689.70|1310.10|26.69|0.00|444.84|471.53|-244.86| +2452351|30158|11331|21854|685434|3861|8688|10|295|1569|27|55.56|91.67|5.50|108.40|148.50|1500.12|2475.09|3.20|108.40|40.10|43.30|-1460.02| +2452249|52766|11983|36510|1797608|4753|15851|10|241|1570|4|11.76|20.81|19.97|0.00|79.88|47.04|83.24|7.18|0.00|79.88|87.06|32.84| +2452249|52766|16299|36510|1797608|4753|15851|10|11|1570|27|68.65|131.80|81.71|0.00|2206.17|1853.55|3558.60|44.12|0.00|2206.17|2250.29|352.62| +2452249|52766|16965|36510|1797608|4753|15851|10|11|1570|49|93.55|129.09|78.74|0.00|3858.26|4583.95|6325.41|0.00|0.00|3858.26|3858.26|-725.69| +2452249|52766|5051|36510|1797608|4753|15851|10|114|1570|55|55.34|104.59|0.00|0.00|0.00|3043.70|5752.45|0.00|0.00|0.00|0.00|-3043.70| +2452249|52766|1297|36510|1797608||15851|10|118|1570|||||||2435.61||||755.79|801.13|| +2452249|52766|8953|36510|1797608|4753|15851|10|71|1570|69|33.87|45.72|21.03|0.00|1451.07|2337.03|3154.68|0.00|0.00|1451.07|1451.07|-885.96| +2452249|52766|4511|36510|1797608|4753|15851|10|267|1570|87|40.31|49.17|39.33|0.00|3421.71|3506.97|4277.79|273.73|0.00|3421.71|3695.44|-85.26| +|52766|13367|||4753|15851||112|1570|48|80.71|142.85|51.42||2468.16|3874.08|6856.80|197.45||2468.16||-1405.92| +2452249|52766|13537|36510|1797608|4753|15851|10|202|1570|30|69.71|138.72|11.09|0.00|332.70|2091.30|4161.60|16.63|0.00|332.70|349.33|-1758.60| +2451857|37342|10265|53417|1513748|6079|10810|8|18|1571|59|91.51|149.16|1.49|0.00|87.91|5399.09|8800.44|7.91|0.00|87.91|95.82|-5311.18| +2451857|37342|1795|53417|1513748|6079|10810|8|184|1571|16|41.14|56.36|10.70|0.00|171.20|658.24|901.76|8.56|0.00|171.20|179.76|-487.04| +2451857|37342|11551|53417|1513748|6079|10810|8|102|1571|87|20.28|20.68|6.61|0.00|575.07|1764.36|1799.16|40.25|0.00|575.07|615.32|-1189.29| +2451857|37342|5653|53417|1513748|6079|10810|8|282|1571|100|8.49|9.84|6.10|0.00|610.00|849.00|984.00|30.50|0.00|610.00|640.50|-239.00| +2451857|37342|407|53417|1513748|6079|10810|8|124|1571|2|41.10|61.65|2.46|0.00|4.92|82.20|123.30|0.14|0.00|4.92|5.06|-77.28| +2451857|37342|1388|53417|1513748|6079|10810|8|134|1571|48|95.86|168.71|32.05|0.00|1538.40|4601.28|8098.08|0.00|0.00|1538.40|1538.40|-3062.88| +2451857|37342|3103|53417|1513748|6079|10810|8|140|1571|21|76.59|85.78|24.87|0.00|522.27|1608.39|1801.38|20.89|0.00|522.27|543.16|-1086.12| +2451857|37342|4873|53417|1513748|6079|10810|8|111|1571|100|98.17|163.94|36.06|0.00|3606.00|9817.00|16394.00|144.24|0.00|3606.00|3750.24|-6211.00| +2451857|37342|16483|53417|1513748|6079|10810|8|263|1571|90|14.46|14.60|10.36|0.00|932.40|1301.40|1314.00|83.91|0.00|932.40|1016.31|-369.00| +2451532|60612|15676|48034|1679131|4181|27278|1|170|1572|18|86.78|164.88|13.19|0.00|237.42|1562.04|2967.84|21.36|0.00|237.42|258.78|-1324.62| +2451532|60612|11762|48034|1679131|4181|27278|1|207|1572|43|46.40|86.30|27.61|0.00|1187.23|1995.20|3710.90|11.87|0.00|1187.23|1199.10|-807.97| +2451532|60612|2014|48034|1679131|4181|27278|1|83|1572|96|52.75|64.88|62.93|5195.50|6041.28|5064.00|6228.48|67.66|5195.50|845.78|913.44|-4218.22| +2451532||8090|||4181||1|213|1572||39.16||69.00|||1566.40|3101.20|220.80||2760.00||1193.60| +2451532|60612|13189|48034|1679131|4181|27278|1|272|1572|79|30.96|34.67|2.08|0.00|164.32|2445.84|2738.93|9.85|0.00|164.32|174.17|-2281.52| +2451532|60612|14330|48034|1679131|4181|27278|1|10|1572|30|60.71|92.88|20.43|245.16|612.90|1821.30|2786.40|25.74|245.16|367.74|393.48|-1453.56| +2451532|60612|14284|48034|1679131|4181|27278|1|255|1572|7|54.58|98.78|0.98|0.00|6.86|382.06|691.46|0.41|0.00|6.86|7.27|-375.20| +2451532|60612|14452|48034|1679131|4181|27278|1|243|1572|91|2.92|2.94|0.61|0.00|55.51|265.72|267.54|3.88|0.00|55.51|59.39|-210.21| +||13346||1679131||||173|1572|70|85.78||0.00|||6004.60|||||0.00|-6004.60| +2451532|60612|4369|48034|1679131|4181|27278|1|195|1572|70|37.76|46.44|27.86|0.00|1950.20|2643.20|3250.80|58.50|0.00|1950.20|2008.70|-693.00| +2451532|60612|6608|48034|1679131|4181|27278|1|133|1572|7|82.63|157.82|52.08|0.00|364.56|578.41|1104.74|32.81|0.00|364.56|397.37|-213.85| +2451532|60612|10936|48034|1679131|4181|27278|1|143|1572|77|68.35|129.86|85.70|0.00|6598.90|5262.95|9999.22|65.98|0.00|6598.90|6664.88|1335.95| +2451532|60612|8215|48034|1679131|4181|27278|1|237|1572|5|38.05|65.82|50.02|0.00|250.10|190.25|329.10|22.50|0.00|250.10|272.60|59.85| +2451532|60612|8698|48034|1679131|4181|27278|1|211|1572|12|4.49|6.91|0.13|0.00|1.56|53.88|82.92|0.07|0.00|1.56|1.63|-52.32| +2451532|60612|4426|48034|1679131|4181|27278|1|255|1572|24|96.68|123.75|21.03|459.29|504.72|2320.32|2970.00|2.27|459.29|45.43|47.70|-2274.89| +2450825|34394|13765|18573|1060145|1717|40489|10|245|1573|87|20.82|36.64|14.28|0.00|1242.36|1811.34|3187.68|12.42|0.00|1242.36|1254.78|-568.98| +2450825|34394|11836|18573|1060145|1717|40489|10|33|1573|12|49.39|67.17|47.01|0.00|564.12|592.68|806.04|39.48|0.00|564.12|603.60|-28.56| +2450825|34394|12478|18573|1060145|1717|40489|10|10|1573|56|90.19|144.30|124.09|0.00|6949.04|5050.64|8080.80|347.45|0.00|6949.04|7296.49|1898.40| +2450825|34394|14539|18573|1060145|1717|40489|10|83|1573|76|90.09|125.22|103.93|0.00|7898.68|6846.84|9516.72|631.89|0.00|7898.68|8530.57|1051.84| +2450825||8059||1060145|1717|||143|1573||81.82|152.18|109.56|3155.32|||7304.64||3155.32|2103.56||| +2450825||10273|||||10|162|1573|89||25.22||0.00|1099.15||||0.00|1099.15||-213.60| +2450825|34394|10028|18573|1060145|1717|40489|10|151|1573|72|20.77|32.81|25.91|0.00|1865.52|1495.44|2362.32|74.62|0.00|1865.52|1940.14|370.08| +2450825|34394|6097|18573|1060145|1717|40489|10|156|1573|47|10.44|17.74|5.67|0.00|266.49|490.68|833.78|18.65|0.00|266.49|285.14|-224.19| +2450825|34394|3997|18573|1060145|1717|40489|10|68|1573|67|62.77|112.35|94.37|0.00|6322.79|4205.59|7527.45|442.59|0.00|6322.79|6765.38|2117.20| +2450825|34394|17050|18573|1060145|1717|40489|10|132|1573|45|15.35|18.88|1.51|0.00|67.95|690.75|849.60|0.67|0.00|67.95|68.62|-622.80| +2450825|34394|15236|18573|1060145|1717|40489|10|216|1573|32|10.79|17.69|1.06|0.00|33.92|345.28|566.08|2.71|0.00|33.92|36.63|-311.36| +2450825|34394|17644|18573|1060145|1717|40489|10|67|1573|80|9.25|16.55|15.39|0.00|1231.20|740.00|1324.00|73.87|0.00|1231.20|1305.07|491.20| +2450825|34394|5920|18573|1060145|1717|40489|10|66|1573|29|88.56|100.95|23.21|0.00|673.09|2568.24|2927.55|40.38|0.00|673.09|713.47|-1895.15| +2451484|46813|3211|1345|124168|4144|12167|8|77|1574|4|13.68|17.78|1.77|0.00|7.08|54.72|71.12|0.42|0.00|7.08|7.50|-47.64| +2451484|46813|2552|1345|124168|4144|12167|8|268|1574|62|7.28|9.75|0.19|8.71|11.78|451.36|604.50|0.27|8.71|3.07|3.34|-448.29| +2451484|46813|14251|1345|124168|4144|12167|8|83|1574|36|29.73|33.00|5.94|0.00|213.84|1070.28|1188.00|8.55|0.00|213.84|222.39|-856.44| +2451484|46813|17356|1345|124168|4144|12167|8|236|1574|19|2.37|3.03|0.87|0.00|16.53|45.03|57.57|1.48|0.00|16.53|18.01|-28.50| +2451484|46813|2078|1345|124168|4144|12167|8|153|1574|30|78.51|116.97|97.08|0.00|2912.40|2355.30|3509.10|0.00|0.00|2912.40|2912.40|557.10| +2451484|46813|8845|1345|124168|4144|12167|8|164|1574|50|35.41|64.44|36.73|0.00|1836.50|1770.50|3222.00|55.09|0.00|1836.50|1891.59|66.00| +2451484|46813|13892|1345|124168|4144|12167|8|7|1574|19|48.57|78.68|73.95|0.00|1405.05|922.83|1494.92|0.00|0.00|1405.05|1405.05|482.22| +2451484|46813|10694|1345|124168|4144|12167|8|100|1574|17|49.64|64.53|54.20|0.00|921.40|843.88|1097.01|18.42|0.00|921.40|939.82|77.52| +2451484|46813|4898||||12167|||1574||61.40|116.66||||||||4614.72|4845.45|| +2451484|46813|9328|1345|124168|4144|12167|8|92|1574|57|20.30|36.74|1.46|0.00|83.22|1157.10|2094.18|3.32|0.00|83.22|86.54|-1073.88| +2451484|46813|14269|1345|124168|4144|12167|8|153|1574|26|45.71|53.93|0.53|0.00|13.78|1188.46|1402.18|0.41|0.00|13.78|14.19|-1174.68| +2451484|46813|2066|1345|124168|4144|12167|8|279|1574|26|12.04|16.49|9.89|0.00|257.14|313.04|428.74|2.57|0.00|257.14|259.71|-55.90| +2451484|46813|16004|1345|124168|4144|12167|8|20|1574|28|43.89|47.84|0.47|0.00|13.16|1228.92|1339.52|1.05|0.00|13.16|14.21|-1215.76| +2452622|75502|12453|9126|1624469|1412|33155|7|194|1575|60|65.04|115.12|112.81|1556.77|6768.60|3902.40|6907.20|104.23|1556.77|5211.83|5316.06|1309.43| +2452622|75502|3669|9126|1624469|1412|33155|7|38|1575|61|70.64|107.37|107.37|0.00|6549.57|4309.04|6549.57|0.00|0.00|6549.57|6549.57|2240.53| +2452622|75502|17481|9126|1624469|1412|33155|7|8|1575|86|6.06|9.33|9.23|0.00|793.78|521.16|802.38|71.44|0.00|793.78|865.22|272.62| +2452622|75502|14061|9126|1624469|1412|33155|7|299|1575|4|95.60|174.94|13.99|0.00|55.96|382.40|699.76|0.00|0.00|55.96|55.96|-326.44| +2452622|75502|5886|9126|1624469|1412|33155|7|143|1575|49|66.03|115.55|79.72|0.00|3906.28|3235.47|5661.95|312.50|0.00|3906.28|4218.78|670.81| +||6211||1624469|1412|33155||300|1575|91|50.28||51.62|2677.52|||7457.45|181.79|2677.52|2019.90|2201.69|| +2452622|75502|7290|9126|1624469|1412|33155|7|105|1575|16|65.48|83.81|11.73|0.00|187.68|1047.68|1340.96|3.75|0.00|187.68|191.43|-860.00| +2452622|75502|8073|9126|1624469|1412|33155|7|167|1575|83|31.20|50.23|7.53|0.00|624.99|2589.60|4169.09|0.00|0.00|624.99|624.99|-1964.61| +2452622|75502|1765|9126|1624469|1412|33155|7|159|1575|88|14.05|16.71|5.68|89.97|499.84|1236.40|1470.48|12.29|89.97|409.87|422.16|-826.53| +2451234|62843|4177|13081|1366698|5115|42332|7|61|1576|47|45.50|66.43|45.83|0.00|2154.01|2138.50|3122.21|86.16|0.00|2154.01|2240.17|15.51| +2451234|62843|2360|13081|1366698|5115|42332|7|170|1576|100|25.81|37.42|22.07|0.00|2207.00|2581.00|3742.00|176.56|0.00|2207.00|2383.56|-374.00| +2451234|62843|4322|13081|1366698|5115|42332|7|154|1576|97|60.77|110.60|27.65|0.00|2682.05|5894.69|10728.20|26.82|0.00|2682.05|2708.87|-3212.64| +2451234|62843|6835|13081|1366698|5115|42332|7|262|1576|29|8.76|16.20|8.10|204.36|234.90|254.04|469.80|0.91|204.36|30.54|31.45|-223.50| +2451234|62843|7792|13081|1366698|5115|42332|7|128|1576|20|67.39|113.88|70.60|0.00|1412.00|1347.80|2277.60|98.84|0.00|1412.00|1510.84|64.20| +2451234|62843|14881|13081|1366698|5115|42332|7|264|1576|79|35.69|59.95|59.35|0.00|4688.65|2819.51|4736.05|421.97|0.00|4688.65|5110.62|1869.14| +2451234|62843|3949|13081|1366698|5115|42332|7|264|1576|60|19.45|33.25|26.93|0.00|1615.80|1167.00|1995.00|129.26|0.00|1615.80|1745.06|448.80| +2451234|62843|14146|13081|1366698|5115|42332|7|300|1576|61|4.98|7.61|0.22|0.93|13.42|303.78|464.21|0.62|0.93|12.49|13.11|-291.29| +2451234|62843|16594|13081|1366698|5115|42332|7|272|1576|54|43.39|49.46|31.15|0.00|1682.10|2343.06|2670.84|134.56|0.00|1682.10|1816.66|-660.96| +2451234|62843|10916|13081|1366698|5115|42332|7|187|1576|97|33.75|61.42|4.29|0.00|416.13|3273.75|5957.74|16.64|0.00|416.13|432.77|-2857.62| +2452641|40977|9318|97209|202713|7064|37368|1|91|1577|47|73.65|118.57|17.78|0.00|835.66|3461.55|5572.79|50.13|0.00|835.66|885.79|-2625.89| +|40977|5127||||37368|||1577||22.88|||||||7.24|||248.65|| +2452641|40977|16320|97209|202713|7064|37368|1|211|1577|32|11.03|17.42|6.96|0.00|222.72|352.96|557.44|6.68|0.00|222.72|229.40|-130.24| +2452641|40977|10710|97209|202713|7064|37368|1|104|1577|28|92.39|172.76|100.20|0.00|2805.60|2586.92|4837.28|56.11|0.00|2805.60|2861.71|218.68| +2452641|40977|4998|97209|202713|7064|37368|1|181|1577|96|41.47|69.66|44.58|0.00|4279.68|3981.12|6687.36|0.00|0.00|4279.68|4279.68|298.56| +|40977|10047||||37368|1|149|1577||76.06|135.38|43.32|0.00|3075.72|5400.26||61.51|0.00|||| +2452641|40977|15681|97209|202713|7064|37368|1|96|1577|62|77.30|117.49|104.56|1426.19|6482.72|4792.60|7284.38|0.00|1426.19|5056.53|5056.53|263.93| +2452641|40977|3081|97209|202713|7064|37368|1|38|1577|41|12.66|14.81|3.11|0.00|127.51|519.06|607.21|5.10|0.00|127.51|132.61|-391.55| +||14865|97209||||1|26|1577|||||0.00|||13029.24|117.22|0.00||2461.66|-5275.20| +2452641|40977|5923|97209|202713|7064|37368|1|30|1577|37|15.70|15.70|10.36|0.00|383.32|580.90|580.90|26.83|0.00|383.32|410.15|-197.58| +2452641|40977|12619|97209|202713|7064|37368|1|207|1577|52|2.99|5.53|0.94|6.35|48.88|155.48|287.56|0.85|6.35|42.53|43.38|-112.95| +2452641|40977|13920|97209|202713|7064|37368|1|12|1577|38|52.27|80.49|57.95|0.00|2202.10|1986.26|3058.62|154.14|0.00|2202.10|2356.24|215.84| +2452641|40977|6024|97209|202713|7064|37368|1|233|1577|75|71.88|73.31|14.66|0.00|1099.50|5391.00|5498.25|32.98|0.00|1099.50|1132.48|-4291.50| +2450900|48842|9332|56918|1593097|2076|42655|7|8|1578|87|40.70|53.72|34.38|0.00|2991.06|3540.90|4673.64|209.37|0.00|2991.06|3200.43|-549.84| +2450900||7192|56918|||42655|||1578|74|82.65|87.60|10.51||777.74||6482.40|15.55|||793.29|-5338.36| +2450900|48842|7850|56918|1593097|2076|42655|7|91|1578|13|93.63|107.67|105.51|672.09|1371.63|1217.19|1399.71|62.95|672.09|699.54|762.49|-517.65| +2450900|48842|17641|56918|1593097|2076|42655|7|55|1578|78|16.79|27.03|2.43|0.00|189.54|1309.62|2108.34|11.37|0.00|189.54|200.91|-1120.08| +2450900|48842|4747|56918|1593097|2076|42655|7|238|1578|12|13.63|21.53|21.09|0.00|253.08|163.56|258.36|17.71|0.00|253.08|270.79|89.52| +2450900|48842|3064|56918|1593097|2076|42655|7|167|1578|30|2.77|3.79|0.41|0.00|12.30|83.10|113.70|0.12|0.00|12.30|12.42|-70.80| +2450900|48842|12866|56918|1593097|2076|42655|7|288|1578|12|81.83|148.93|98.29|0.00|1179.48|981.96|1787.16|82.56|0.00|1179.48|1262.04|197.52| +2450900|48842|4105|56918|1593097|2076|42655|7|130|1578|39|76.84|132.93|38.54|210.42|1503.06|2996.76|5184.27|90.48|210.42|1292.64|1383.12|-1704.12| +2450900|48842|3913|56918|1593097|2076|42655|7|2|1578|33|46.95|80.28|61.81|0.00|2039.73|1549.35|2649.24|122.38|0.00|2039.73|2162.11|490.38| +2450900|48842|10021|56918|1593097|2076|42655|7|92|1578|88|87.23|110.78|45.41|1198.82|3996.08|7676.24|9748.64|55.94|1198.82|2797.26|2853.20|-4878.98| +2450900|48842|6374|56918|1593097|2076|42655|7|25|1578|34|57.35|100.93|70.65|0.00|2402.10|1949.90|3431.62|144.12|0.00|2402.10|2546.22|452.20| +2450900|48842|5701|56918|1593097|2076|42655|7|225|1578|12|64.23|70.65|9.18|0.00|110.16|770.76|847.80|6.60|0.00|110.16|116.76|-660.60| +2450900|48842|3604|56918|1593097|2076|42655|7|265|1578|93|49.50|82.17|7.39|0.00|687.27|4603.50|7641.81|20.61|0.00|687.27|707.88|-3916.23| +2450900|48842|10298|56918|1593097|2076|42655|7|101|1578|77|61.81|118.05|106.24|0.00|8180.48|4759.37|9089.85|409.02|0.00|8180.48|8589.50|3421.11| +2450900|48842|6904|56918|1593097|2076|42655|7|30|1578|4|3.48|6.26|0.00|0.00|0.00|13.92|25.04|0.00|0.00|0.00|0.00|-13.92| +2451498|44032|12163|37703|1052115|927|22539|7|175|1579|27|61.43|113.03|102.85|0.00|2776.95|1658.61|3051.81|166.61|0.00|2776.95|2943.56|1118.34| +2451498|44032|15398|37703|1052115|927|22539|7|278|1579|77|9.70|16.39|3.60|0.00|277.20|746.90|1262.03|5.54|0.00|277.20|282.74|-469.70| +2451498|44032|13558|37703|1052115|927|22539|7|262|1579|94|22.10|36.68|21.27|0.00|1999.38|2077.40|3447.92|0.00|0.00|1999.38|1999.38|-78.02| +2451498|44032|10148|37703|1052115|927|22539|7|65|1579|75|39.85|51.00|51.00|459.00|3825.00|2988.75|3825.00|235.62|459.00|3366.00|3601.62|377.25| +2451498|44032|16663|37703|1052115|927|22539|7|128|1579|32|60.08|73.29|61.56|0.00|1969.92|1922.56|2345.28|19.69|0.00|1969.92|1989.61|47.36| +2451498|44032|14896|37703|1052115|927|22539|7|275|1579|31|52.25|66.88|62.86|97.43|1948.66|1619.75|2073.28|37.02|97.43|1851.23|1888.25|231.48| +2451498|44032|2158|37703|1052115|927|22539|7|296|1579|71|52.25|81.51|54.61|1977.42|3877.31|3709.75|5787.21|113.99|1977.42|1899.89|2013.88|-1809.86| +2451498|44032|10015|37703|1052115|927|22539|7|82|1579|33|25.60|32.25|3.87|24.26|127.71|844.80|1064.25|3.10|24.26|103.45|106.55|-741.35| +2451498|44032|11690|37703|1052115|927|22539|7|157|1579|26|84.87|157.85|58.40|1199.53|1518.40|2206.62|4104.10|0.00|1199.53|318.87|318.87|-1887.75| +2451498|44032|14852|37703|1052115|927|22539|7|11|1579|43|98.99|126.70|32.94|0.00|1416.42|4256.57|5448.10|42.49|0.00|1416.42|1458.91|-2840.15| +2451498|44032|1262|37703|1052115|927|22539|7|48|1579|59|30.46|38.37|5.37|0.00|316.83|1797.14|2263.83|28.51|0.00|316.83|345.34|-1480.31| +2451498|44032|15850|37703|1052115|927|22539|7|227|1579|74|55.59|79.49|67.56|0.00|4999.44|4113.66|5882.26|149.98|0.00|4999.44|5149.42|885.78| +2451498|44032|7639|37703|1052115|927|22539|7|253|1579|47|90.62|173.08|138.46|2993.50|6507.62|4259.14|8134.76|210.84|2993.50|3514.12|3724.96|-745.02| +2451498|44032|6061|37703|1052115|927|22539|7|278|1579|76|82.00|146.78|64.58|0.00|4908.08|6232.00|11155.28|196.32|0.00|4908.08|5104.40|-1323.92| +2451498|44032|6790|37703|1052115|927|22539|7|149|1579|88|12.79|19.95|6.38|0.00|561.44|1125.52|1755.60|22.45|0.00|561.44|583.89|-564.08| +2451498|44032|11431|37703|1052115|927|22539|7|121|1579|63|81.94|154.04|83.18|0.00|5240.34|5162.22|9704.52|471.63|0.00|5240.34|5711.97|78.12| +2452271|68164|16841|85533|1137097|1461|3425|1|83|1580|84|99.77|132.69|27.86|0.00|2340.24|8380.68|11145.96|46.80|0.00|2340.24|2387.04|-6040.44| +2452271|68164|12689|85533|1137097|1461|3425|1|78|1580|23|85.45|111.08|83.31|1552.06|1916.13|1965.35|2554.84|3.64|1552.06|364.07|367.71|-1601.28| +2452271|68164|4825|85533|1137097|1461|3425|1|81|1580|93|9.71|15.14|5.45|0.00|506.85|903.03|1408.02|20.27|0.00|506.85|527.12|-396.18| +2452271|68164|17019|85533|1137097|1461|3425|1|108|1580|57|87.73|125.45|109.14|0.00|6220.98|5000.61|7150.65|248.83|0.00|6220.98|6469.81|1220.37| +2452271|68164|15129|85533|1137097|1461|3425|1|273|1580|48|49.11|91.83|64.28|1388.44|3085.44|2357.28|4407.84|118.79|1388.44|1697.00|1815.79|-660.28| +2452271|68164|2437|85533|1137097|1461|3425|1|101|1580|47|44.72|86.75|5.20|0.00|244.40|2101.84|4077.25|14.66|0.00|244.40|259.06|-1857.44| +2452271|68164|581|85533|1137097|1461|3425|1|42|1580|29|71.11|118.75|89.06|0.00|2582.74|2062.19|3443.75|51.65|0.00|2582.74|2634.39|520.55| +2452271|68164|9681|85533|1137097|1461|3425|1|290|1580|60|37.82|42.73|31.62|0.00|1897.20|2269.20|2563.80|170.74|0.00|1897.20|2067.94|-372.00| +2452271|68164|17625|85533|1137097|1461|3425|1|20|1580|83|72.95|75.86|22.75|1170.71|1888.25|6054.85|6296.38|14.35|1170.71|717.54|731.89|-5337.31| +2452271|68164|7719|85533|1137097|1461|3425|1|152|1580|51|28.60|45.76|23.79|752.23|1213.29|1458.60|2333.76|0.00|752.23|461.06|461.06|-997.54| +2452271|68164|11645|85533|1137097|1461|3425|1|16|1580|5|60.78|98.46|85.66|0.00|428.30|303.90|492.30|21.41|0.00|428.30|449.71|124.40| +2452271|68164|15909|85533|1137097|1461|3425|1|157|1580|50|89.71|151.60|59.12|0.00|2956.00|4485.50|7580.00|206.92|0.00|2956.00|3162.92|-1529.50| +2452271||2027|85533|||3425||2|1580|23|37.64|61.35||0.00||865.72|||0.00|649.06||| +|68164|2683|85533||1461||1|96|1580||10.48|||0.00|||659.36|0.39|0.00|39.52||| +2452553|48974|7171|68388|1919020|1299|16331|7|243|1581|71|18.53|24.08|17.09|0.00|1213.39|1315.63|1709.68|0.00|0.00|1213.39|1213.39|-102.24| +2452553|48974|17106|68388|1919020|1299|16331|7|84|1581|77|6.06|11.02|9.80|0.00|754.60|466.62|848.54|37.73|0.00|754.60|792.33|287.98| +2452553|48974|12129|68388|1919020|1299|16331|7|13|1581|80|9.72|17.69|15.56|0.00|1244.80|777.60|1415.20|37.34|0.00|1244.80|1282.14|467.20| +|48974|2037|68388||||7||1581||30.61||4.45||||1474.04|7.65|||199.00|| +2452553|48974|2526|68388|1919020|1299|16331|7|43|1581|18|32.63|60.69|40.05|0.00|720.90|587.34|1092.42|36.04|0.00|720.90|756.94|133.56| +2452553|48974|7281|68388|1919020|1299|16331|7|172|1581|12|73.22|118.61|117.42|0.00|1409.04|878.64|1423.32|112.72|0.00|1409.04|1521.76|530.40| +2452553|48974|2958|68388|1919020|1299|16331|7|253|1581|14|3.65|5.69|3.47|0.00|48.58|51.10|79.66|0.48|0.00|48.58|49.06|-2.52| +2452553|48974|16435|68388|1919020|1299|16331|7|231|1581|22|29.40|57.62|31.69|0.00|697.18|646.80|1267.64|55.77|0.00|697.18|752.95|50.38| +2452553|48974|8677|68388|1919020|1299|16331|7|165|1581|64|75.80|112.94|54.21|0.00|3469.44|4851.20|7228.16|208.16|0.00|3469.44|3677.60|-1381.76| +2452553|48974|9558|68388|1919020|1299|16331|7|99|1581|37|49.57|85.26|0.00|0.00|0.00|1834.09|3154.62|0.00|0.00|0.00|0.00|-1834.09| +2452553|48974|13194|68388|1919020|1299|16331|7|173|1581|93|47.99|92.62|37.04|0.00|3444.72|4463.07|8613.66|34.44|0.00|3444.72|3479.16|-1018.35| +2452553|48974|9955|68388|1919020|1299|16331|7|179|1581|29|99.51|198.02|17.82|0.00|516.78|2885.79|5742.58|46.51|0.00|516.78|563.29|-2369.01| +2452310|36328|17797|11095|1418298|3101|27104|4|110|1582|46|3.10|4.46|3.43|0.00|157.78|142.60|205.16|12.62|0.00|157.78|170.40|15.18| +2452310|36328|3918|11095|1418298|3101|27104|4|38|1582|72|1.24|1.51|0.49|23.63|35.28|89.28|108.72|0.23|23.63|11.65|11.88|-77.63| +2452310|36328|15241|11095|1418298|3101|27104|4|36|1582|80|37.59|57.13|21.70|0.00|1736.00|3007.20|4570.40|104.16|0.00|1736.00|1840.16|-1271.20| +2452310|36328|3793|11095|1418298|3101|27104|4|90|1582|71|84.42|145.20|84.21|3826.50|5978.91|5993.82|10309.20|150.66|3826.50|2152.41|2303.07|-3841.41| +2452310|36328|15690|11095|1418298|3101|27104|4|3|1582|99|81.27|86.95|59.12|0.00|5852.88|8045.73|8608.05|526.75|0.00|5852.88|6379.63|-2192.85| +2452310|36328|12279|11095|1418298|3101|27104|4|49|1582|32|94.57|96.46|6.75|0.00|216.00|3026.24|3086.72|10.80|0.00|216.00|226.80|-2810.24| +2452310|36328|7983|11095|1418298|3101|27104|4|53|1582|66|44.36|77.18|5.40|0.00|356.40|2927.76|5093.88|21.38|0.00|356.40|377.78|-2571.36| +2452310|36328|11718|11095|1418298|3101|27104|4|185|1582|89|16.73|23.75|15.20|906.37|1352.80|1488.97|2113.75|31.25|906.37|446.43|477.68|-1042.54| +2450874|44810|6886|10898|1708563|3828|23093|10|126|1583|93|26.76|49.23|34.46|3140.68|3204.78|2488.68|4578.39|3.84|3140.68|64.10|67.94|-2424.58| +2450874|44810|3494|10898|1708563|3828|23093|10|168|1583|79|27.26|30.80|11.08|0.00|875.32|2153.54|2433.20|8.75|0.00|875.32|884.07|-1278.22| +2450874|44810|13393|10898|1708563|3828|23093|10|56|1583|57|28.09|47.19|40.58|0.00|2313.06|1601.13|2689.83|69.39|0.00|2313.06|2382.45|711.93| +2450874|44810|14840|10898|1708563|3828|23093|10|254|1583|54|32.14|59.78|57.38|0.00|3098.52|1735.56|3228.12|123.94|0.00|3098.52|3222.46|1362.96| +2450874|44810|3386|10898|1708563|3828|23093|10|13|1583|71|58.65|74.48|59.58|423.01|4230.18|4164.15|5288.08|152.28|423.01|3807.17|3959.45|-356.98| +|44810|13801|10898|1708563|||10|114|1583|||100.81|45.36||4173.12|5455.60|9274.52|41.73||4173.12|4214.85|| +2450874|44810|17290|10898|1708563|3828|23093|10|251|1583|91|44.20|45.96|24.35|0.00|2215.85|4022.20|4182.36|66.47|0.00|2215.85|2282.32|-1806.35| +2450874||400|10898|1708563|||||1583||30.11|||0.00|||367.30|5.94|0.00|66.10|72.04|-235.00| +2450874|44810|11425|10898|1708563|3828|23093|10|24|1583|46|77.85|155.70|90.30|0.00|4153.80|3581.10|7162.20|249.22|0.00|4153.80|4403.02|572.70| +2450874||13532|||3828|||265|1583|30|8.54||4.06||121.80|||||121.80|123.01|| +2450874|44810|16528|10898|1708563|3828|23093|10|274|1583|30|65.31|74.45|62.53|0.00|1875.90|1959.30|2233.50|0.00|0.00|1875.90|1875.90|-83.40| +2450874|44810|16222|10898|1708563|3828|23093|10|212|1583|73|79.81|107.74|11.85|0.00|865.05|5826.13|7865.02|43.25|0.00|865.05|908.30|-4961.08| +2450874|44810|13327|10898|1708563|3828|23093|10|291|1583|74|41.69|63.78|61.86|0.00|4577.64|3085.06|4719.72|45.77|0.00|4577.64|4623.41|1492.58| +2451366|70807|6736|80976|1597227|3078|36129|2|186|1584|100|64.54|87.12|47.91|0.00|4791.00|6454.00|8712.00|143.73|0.00|4791.00|4934.73|-1663.00| +2451366|70807|5359|80976|1597227|3078|36129|2|83|1584|34|95.09|163.55|32.71|1012.04|1112.14|3233.06|5560.70|1.00|1012.04|100.10|101.10|-3132.96| +2451366|70807|2203|80976|1597227|3078|36129|2|157|1584|48|76.52|105.59|76.02|0.00|3648.96|3672.96|5068.32|328.40|0.00|3648.96|3977.36|-24.00| +2451366|70807|2168|80976|1597227|3078|36129|2|80|1584|100|20.95|40.85|40.03|0.00|4003.00|2095.00|4085.00|160.12|0.00|4003.00|4163.12|1908.00| +2451366|70807|17030|80976|1597227|3078|36129|2|62|1584|55|6.92|6.92|0.34|0.00|18.70|380.60|380.60|1.12|0.00|18.70|19.82|-361.90| +2451366|70807|10304|80976|1597227|3078|36129|2|141|1584|7|20.77|35.30|15.53|0.00|108.71|145.39|247.10|3.26|0.00|108.71|111.97|-36.68| +2451366|70807|433|80976|1597227|3078|36129|2|40|1584|6|41.91|45.68|10.96|30.90|65.76|251.46|274.08|1.39|30.90|34.86|36.25|-216.60| +2451366|70807|9728|80976|1597227|3078|36129|2|79|1584|86|94.25|111.21|86.74|0.00|7459.64|8105.50|9564.06|74.59|0.00|7459.64|7534.23|-645.86| +2451366|70807|2029|80976|1597227|3078|36129|2|142|1584|67|45.08|48.23|20.25|0.00|1356.75|3020.36|3231.41|94.97|0.00|1356.75|1451.72|-1663.61| +2451366|70807|16732|80976|1597227|3078|36129|2|277|1584|3|57.71|105.60|42.24|0.00|126.72|173.13|316.80|8.87|0.00|126.72|135.59|-46.41| +2451366|70807|439|80976|1597227|3078|36129|2|127|1584|36|50.55|92.00|17.48|0.00|629.28|1819.80|3312.00|12.58|0.00|629.28|641.86|-1190.52| +2451366|70807|4762|80976|1597227|3078|36129|2|287|1584|38|18.70|21.50|0.43|8.49|16.34|710.60|817.00|0.23|8.49|7.85|8.08|-702.75| +2451366|70807|15049|80976|1597227|3078|36129|2|176|1584|15|38.29|52.84|26.94|0.00|404.10|574.35|792.60|36.36|0.00|404.10|440.46|-170.25| +2451366|70807|964|80976|1597227|3078|36129|2|3|1584|29|96.39|123.37|117.20|0.00|3398.80|2795.31|3577.73|271.90|0.00|3398.80|3670.70|603.49| +2451896|39496|16652|43512|1441581|4003|47672|8|176|1585|48|65.84|119.82|14.37|0.00|689.76|3160.32|5751.36|55.18|0.00|689.76|744.94|-2470.56| +2451896|39496|6203|43512|1441581|4003|47672|8|205|1585|28|95.13|170.28|73.22|328.02|2050.16|2663.64|4767.84|86.10|328.02|1722.14|1808.24|-941.50| +2451896|39496|8549|43512|1441581|4003|47672|8|259|1585|4|91.68|95.34|93.43|0.00|373.72|366.72|381.36|29.89|0.00|373.72|403.61|7.00| +2451896|39496|3125|43512|1441581|4003|47672|8|131|1585|44|51.29|84.11|35.32|0.00|1554.08|2256.76|3700.84|46.62|0.00|1554.08|1600.70|-702.68| +2451896|39496|16295|43512|1441581|4003|47672|8|181|1585|17|70.65|94.67|57.74|0.00|981.58|1201.05|1609.39|78.52|0.00|981.58|1060.10|-219.47| +2451896|39496|11993|43512|1441581|4003|47672|8|28|1585|40|62.84|94.26|76.35|0.00|3054.00|2513.60|3770.40|61.08|0.00|3054.00|3115.08|540.40| +2451896|39496|4115|43512|1441581|4003|47672|8|202|1585|69|65.13|113.97|94.59|0.00|6526.71|4493.97|7863.93|587.40|0.00|6526.71|7114.11|2032.74| +2451896|39496|9578|43512|1441581|4003|47672|8|115|1585|95|3.49|4.71|0.14|0.00|13.30|331.55|447.45|0.00|0.00|13.30|13.30|-318.25| +2451896|39496|11342|43512|1441581|4003|47672|8|12|1585|24|81.33|156.15|146.78|0.00|3522.72|1951.92|3747.60|246.59|0.00|3522.72|3769.31|1570.80| +2451896|39496|17912|43512|1441581|4003|47672|8|185|1585|4|30.27|32.38|8.09|0.00|32.36|121.08|129.52|1.29|0.00|32.36|33.65|-88.72| +2451896|39496|6139|43512|1441581|4003|47672|8|243|1585|19|63.24|75.88|7.58|128.17|144.02|1201.56|1441.72|0.31|128.17|15.85|16.16|-1185.71| +2451896|39496|5249|43512|1441581|4003|47672|8|261|1585|78|60.05|61.85|45.15|387.38|3521.70|4683.90|4824.30|282.08|387.38|3134.32|3416.40|-1549.58| +||15167|43512|1441581||||34|1585|47||||0.00|||7520.00||0.00|||| +2451896|39496|5797|43512|1441581|4003|47672|8|111|1585|52|16.56|20.53|8.82|0.00|458.64|861.12|1067.56|18.34|0.00|458.64|476.98|-402.48| +2451896||9470||1441581|||8|65|1585|61|90.82|145.31||||5540.02|8863.91|||8154.48||2614.46| +2451896|39496|2474|43512|1441581|4003|47672|8|232|1585|33|11.38|17.07|2.21|0.00|72.93|375.54|563.31|1.45|0.00|72.93|74.38|-302.61| +2452246|47257|1125|||||8||1586|34|71.27|131.84|101.51|0.00|3451.34|||0.00|0.00|3451.34||| +2452246|47257|921|16961|1384292|3270|30248|8|55|1586|48|18.80|27.44|22.50|0.00|1080.00|902.40|1317.12|21.60|0.00|1080.00|1101.60|177.60| +2452246|47257|7357|16961|1384292|3270|30248|8|272|1586|61|7.38|11.66|4.89|178.97|298.29|450.18|711.26|7.15|178.97|119.32|126.47|-330.86| +2452246|47257|10727|16961|1384292|3270|30248|8|245|1586|66|63.34|126.68|98.81|0.00|6521.46|4180.44|8360.88|130.42|0.00|6521.46|6651.88|2341.02| +2452246|47257|8509|16961|1384292|3270|30248|8|19|1586|60|70.95|97.20|26.24|0.00|1574.40|4257.00|5832.00|31.48|0.00|1574.40|1605.88|-2682.60| +2452246|47257|11099|16961|1384292|3270|30248|8|228|1586|34|91.57|155.66|126.08|0.00|4286.72|3113.38|5292.44|0.00|0.00|4286.72|4286.72|1173.34| +2452246|47257|13831|16961|1384292|3270|30248|8|223|1586|40|3.25|6.20|3.90|0.00|156.00|130.00|248.00|14.04|0.00|156.00|170.04|26.00| +2452246|47257|233|16961|1384292|3270|30248|8|44|1586|11|88.13|141.00|19.74|0.00|217.14|969.43|1551.00|4.34|0.00|217.14|221.48|-752.29| +2452246|47257|6325|16961|1384292|3270|30248|8|31|1586|73|22.60|41.58|22.86|0.00|1668.78|1649.80|3035.34|50.06|0.00|1668.78|1718.84|18.98| +2451613|44048|17564|55811|94461|1464|22914|7|2|1587|15|26.49|29.40|16.75|0.00|251.25|397.35|441.00|22.61|0.00|251.25|273.86|-146.10| +2451613|44048|13172|55811|94461|1464|22914|7|225|1587|55|96.07|118.16|70.89|233.93|3898.95|5283.85|6498.80|256.55|233.93|3665.02|3921.57|-1618.83| +2451613|44048|15301|55811|94461|1464|22914|7|258|1587|24|33.89|55.91|17.89|0.00|429.36|813.36|1341.84|4.29|0.00|429.36|433.65|-384.00| +2451613|44048|5996|55811|94461|1464|22914|7|116|1587|36|85.61|146.39|20.49|0.00|737.64|3081.96|5270.04|22.12|0.00|737.64|759.76|-2344.32| +2451613|44048|4742|55811|94461|1464|22914|7|256|1587|12|27.86|47.08|38.60|0.00|463.20|334.32|564.96|41.68|0.00|463.20|504.88|128.88| +2451613|44048|15143|55811|94461|1464|22914|7|143|1587|63|76.57|98.00|61.74|0.00|3889.62|4823.91|6174.00|233.37|0.00|3889.62|4122.99|-934.29| +2451613|44048|8627|55811|94461|1464|22914|7|48|1587|61|20.49|36.88|18.07|0.00|1102.27|1249.89|2249.68|33.06|0.00|1102.27|1135.33|-147.62| +2451613||5119|||||7|245|1587|||71.34|0.71||10.65|||||||| +2451769|39663|12152|27705|851988|271|17754|10|79|1588|56|9.14|13.89|8.88|0.00|497.28|511.84|777.84|19.89|0.00|497.28|517.17|-14.56| +2451769|39663|4142|27705|851988|271|17754|10|194|1588|90|99.05|121.83|105.99|0.00|9539.10|8914.50|10964.70|95.39|0.00|9539.10|9634.49|624.60| +2451769|39663|16573|27705|851988|271|17754|10|32|1588|14|83.55|102.76|97.62|0.00|1366.68|1169.70|1438.64|13.66|0.00|1366.68|1380.34|196.98| +2451769|39663|8513|27705|851988|271|17754|10|191|1588|84|44.69|49.60|36.20|0.00|3040.80|3753.96|4166.40|0.00|0.00|3040.80|3040.80|-713.16| +2451769|39663|4394|27705|851988|271|17754|10|276|1588|15|2.66|5.10|1.07|14.60|16.05|39.90|76.50|0.02|14.60|1.45|1.47|-38.45| +2451769|39663|16553|27705|851988|271|17754|10|12|1588|78|37.46|70.79|3.53|0.00|275.34|2921.88|5521.62|13.76|0.00|275.34|289.10|-2646.54| +2451769|39663|2597|27705|851988|271|17754|10|116|1588|18|60.92|66.40|27.22|58.79|489.96|1096.56|1195.20|38.80|58.79|431.17|469.97|-665.39| +2451769|39663|1712|27705|851988|271|17754|10|9|1588|76|84.63|164.18|152.68|0.00|11603.68|6431.88|12477.68|928.29|0.00|11603.68|12531.97|5171.80| +2451769|39663|15545|27705|851988|271|17754|10|234|1588|65|82.46|98.12|81.43|0.00|5292.95|5359.90|6377.80|52.92|0.00|5292.95|5345.87|-66.95| +||12971|||||10||1588|26||102.68|||2536.04|1689.74||228.24||2536.04|2764.28|| +2451419|65739|9829|22649|945245|6594|5990|2|76|1589|64|33.56|57.72|43.29|0.00|2770.56|2147.84|3694.08|55.41|0.00|2770.56|2825.97|622.72| +2451419|65739|1873|22649|945245|6594|5990|2|289|1589|14|25.59|51.18|4.09|0.00|57.26|358.26|716.52|0.00|0.00|57.26|57.26|-301.00| +2451419|65739|5456|22649|945245|6594|5990|2|2|1589|20|6.58|7.30|0.14|0.00|2.80|131.60|146.00|0.00|0.00|2.80|2.80|-128.80| +2451419|65739|2774|22649|945245|6594|5990|2|16|1589|47|78.03|90.51|38.91|0.00|1828.77|3667.41|4253.97|18.28|0.00|1828.77|1847.05|-1838.64| +2451419|65739|11666|22649|945245|6594|5990|2|104|1589|36|73.62|98.65|86.81|0.00|3125.16|2650.32|3551.40|125.00|0.00|3125.16|3250.16|474.84| +2451419|65739|11536|22649|945245|6594|5990|2|95|1589|94|81.10|102.18|101.15|0.00|9508.10|7623.40|9604.92|665.56|0.00|9508.10|10173.66|1884.70| +2451419|65739|10309|22649|945245|6594|5990|2|294|1589|75|95.12|109.38|1.09|0.00|81.75|7134.00|8203.50|3.27|0.00|81.75|85.02|-7052.25| +2451419|65739|4930|22649|945245|6594|5990|2|229|1589|56|16.71|22.89|14.64|0.00|819.84|935.76|1281.84|16.39|0.00|819.84|836.23|-115.92| +2451419|65739|16246|22649|945245|6594|5990|2|26|1589|91|52.12|99.54|23.88|0.00|2173.08|4742.92|9058.14|108.65|0.00|2173.08|2281.73|-2569.84| +2451419|65739|5995|22649|945245|6594|5990|2|193|1589|38|29.29|37.49|22.11|621.73|840.18|1113.02|1424.62|17.47|621.73|218.45|235.92|-894.57| +2451419|65739|5732|22649|945245|6594|5990|2|187|1589|96|55.36|83.04|63.11|0.00|6058.56|5314.56|7971.84|242.34|0.00|6058.56|6300.90|744.00| +2451419|65739|355|22649|945245|6594|5990|2|223|1589|1|46.00|68.54|67.16|51.71|67.16|46.00|68.54|0.92|51.71|15.45|16.37|-30.55| +2451419|65739|4814|22649|945245|6594|5990|2|73|1589|98|36.72|71.23|2.13|0.00|208.74|3598.56|6980.54|8.34|0.00|208.74|217.08|-3389.82| +2451419|65739|17602|22649|945245|6594|5990|2|256|1589|87|56.11|85.84|34.33|0.00|2986.71|4881.57|7468.08|119.46|0.00|2986.71|3106.17|-1894.86| +2451419|65739|7342|22649|945245|6594|5990|2|5|1589|98|53.22|80.89|38.01|0.00|3724.98|5215.56|7927.22|0.00|0.00|3724.98|3724.98|-1490.58| +2451419|65739|13972|22649|945245|6594|5990|2|68|1589|57|72.91|125.40|66.46|0.00|3788.22|4155.87|7147.80|0.00|0.00|3788.22|3788.22|-367.65| +2452581|74524|16249|25070|524015|6160|6352|8|142|1590|12|39.77|53.29|48.49|0.00|581.88|477.24|639.48|52.36|0.00|581.88|634.24|104.64| +2452581|74524|15723|25070|524015|6160|6352|8|210|1590|85|3.06|3.21|2.34|0.00|198.90|260.10|272.85|11.93|0.00|198.90|210.83|-61.20| +2452581|74524|5679|25070|524015|6160|6352|8|289|1590|29|4.17|8.00|7.44|21.57|215.76|120.93|232.00|5.82|21.57|194.19|200.01|73.26| +2452581|74524|16392|25070|524015|6160|6352|8|17|1590|32|22.04|37.46|13.11|0.00|419.52|705.28|1198.72|12.58|0.00|419.52|432.10|-285.76| +2452581|74524|6073|25070|524015|6160|6352|8|22|1590|55|86.75|95.42|17.17|179.42|944.35|4771.25|5248.10|15.29|179.42|764.93|780.22|-4006.32| +2452581|74524|5022|25070|524015|6160|6352|8|97|1590|79|36.47|56.89|7.96|0.00|628.84|2881.13|4494.31|37.73|0.00|628.84|666.57|-2252.29| +2452581|74524|14073|25070|524015|6160|6352|8|195|1590|18|14.96|27.22|3.53|0.00|63.54|269.28|489.96|5.71|0.00|63.54|69.25|-205.74| +2452581|74524|17418|25070|524015|6160|6352|8|84|1590|44|66.55|99.15|54.53|0.00|2399.32|2928.20|4362.60|215.93|0.00|2399.32|2615.25|-528.88| +2452581|74524|2769|25070|524015|6160|6352|8|17|1590|91|72.59|100.90|1.00|89.18|91.00|6605.69|9181.90|0.00|89.18|1.82|1.82|-6603.87| +2452581|74524|17985|25070|524015|6160|6352|8|146|1590|93|76.78|92.90|28.79|0.00|2677.47|7140.54|8639.70|107.09|0.00|2677.47|2784.56|-4463.07| +2452581|74524|3000|25070|524015|6160|6352|8|266|1590|74|49.92|82.86|64.63|0.00|4782.62|3694.08|6131.64|0.00|0.00|4782.62|4782.62|1088.54| +2452581||3714||524015|6160|||207|1590|6||||||190.74|293.70|||85.14||| +2452581|74524|6183|25070|524015|6160|6352|8|279|1590|20|67.86|126.21|124.94|0.00|2498.80|1357.20|2524.20|124.94|0.00|2498.80|2623.74|1141.60| +2452624|52668|13263|35665|109812|1206|22553|8|50|1591|77|41.88|80.82|21.01|1035.37|1617.77|3224.76|6223.14|11.64|1035.37|582.40|594.04|-2642.36| +2452624|52668|8221|35665|109812|1206|22553|8|212|1591|4|96.76|156.75|39.18|0.00|156.72|387.04|627.00|0.00|0.00|156.72|156.72|-230.32| +2452624|52668|8541|35665|109812|1206|22553|8|92|1591|63|93.40|126.09|23.95|0.00|1508.85|5884.20|7943.67|90.53|0.00|1508.85|1599.38|-4375.35| +2452624|52668|4251|35665|109812|1206|22553|8|207|1591|46|36.08|68.19|12.27|0.00|564.42|1659.68|3136.74|50.79|0.00|564.42|615.21|-1095.26| +2452624|52668|16599|35665|109812|1206|22553|8|49|1591|11|36.75|69.82|46.77|0.00|514.47|404.25|768.02|46.30|0.00|514.47|560.77|110.22| +2452624|52668|12891|35665|109812|1206|22553|8|6|1591|89|77.60|79.92|75.12|0.00|6685.68|6906.40|7112.88|267.42|0.00|6685.68|6953.10|-220.72| +2452624|52668|4543|35665|109812|1206|22553|8|35|1591|37|44.20|67.18|20.15|0.00|745.55|1635.40|2485.66|59.64|0.00|745.55|805.19|-889.85| +2452624|52668|8466|35665|109812|1206|22553|8|104|1591|10|43.30|60.18|38.51|0.00|385.10|433.00|601.80|19.25|0.00|385.10|404.35|-47.90| +2452624|52668|13713|35665|109812|1206|22553|8|289|1591|31|51.32|86.73|80.65|0.00|2500.15|1590.92|2688.63|25.00|0.00|2500.15|2525.15|909.23| +2452624|52668|2497|35665|109812|1206|22553|8|118|1591|59|25.10|26.35|18.70|0.00|1103.30|1480.90|1554.65|33.09|0.00|1103.30|1136.39|-377.60| +2452624|52668|4749|35665|109812|1206|22553|8|42|1591|6|36.68|61.62|14.78|0.00|88.68|220.08|369.72|7.09|0.00|88.68|95.77|-131.40| +2452624|52668|6751|35665|109812|1206|22553|8|1|1591|4|10.53|13.16|12.23|23.48|48.92|42.12|52.64|0.00|23.48|25.44|25.44|-16.68| +2452624|52668|6043|35665|109812|1206|22553|8|179|1591|88|36.67|50.23|31.14|0.00|2740.32|3226.96|4420.24|219.22|0.00|2740.32|2959.54|-486.64| +|52668|13303|35665||||8||1591|24|49.06|68.19|||785.52|1177.44||62.84||785.52||-391.92| +2452090|63124|13573|19435|1704589|5390|47791|1|44|1592|28|21.16|27.08|24.37|641.41|682.36|592.48|758.24|3.68|641.41|40.95|44.63|-551.53| +2452090|63124|2339|19435|1704589|5390|47791|1|58|1592|89|15.05|16.70|10.02|0.00|891.78|1339.45|1486.30|62.42|0.00|891.78|954.20|-447.67| +2452090|63124|5273|19435|1704589|5390|47791|1|146|1592|23|95.18|187.50|24.37|168.15|560.51|2189.14|4312.50|15.69|168.15|392.36|408.05|-1796.78| +2452090|63124|14519|19435|1704589|5390|47791|1|117|1592|17|65.63|70.88|24.80|231.88|421.60|1115.71|1204.96|1.89|231.88|189.72|191.61|-925.99| +2452090|63124|15063|19435|1704589|5390|47791|1|230|1592|15|70.41|91.53|25.62|0.00|384.30|1056.15|1372.95|15.37|0.00|384.30|399.67|-671.85| +2452090|63124|12259|19435|1704589|5390|47791|1|116|1592|68|80.53|111.13|11.11|287.08|755.48|5476.04|7556.84|14.05|287.08|468.40|482.45|-5007.64| +2452090|63124|14647|19435|1704589|5390|47791|1|295|1592|32|63.88|110.51|99.45|0.00|3182.40|2044.16|3536.32|127.29|0.00|3182.40|3309.69|1138.24| +2452090|63124|12713|19435|1704589|5390|47791|1|120|1592|33|67.56|95.25|16.19|0.00|534.27|2229.48|3143.25|32.05|0.00|534.27|566.32|-1695.21| +2452090|63124|6327|19435|1704589|5390|47791|1|73|1592|18|57.39|88.38|4.41|46.04|79.38|1033.02|1590.84|1.00|46.04|33.34|34.34|-999.68| +2452090|63124|165|19435|1704589|5390|47791|1|162|1592|3|22.56|36.32|1.08|0.00|3.24|67.68|108.96|0.25|0.00|3.24|3.49|-64.44| +2452090|63124|16121|19435|1704589|5390|47791|1|219|1592|100|70.09|119.85|76.70|0.00|7670.00|7009.00|11985.00|536.90|0.00|7670.00|8206.90|661.00| +|63124|11903|19435||||||1592|43|42.72|83.30|31.65|0.00||||108.87|0.00|1360.95||| +2452090|63124|6025|19435|1704589|5390|47791|1|205|1592|8|56.53|57.66|39.20|0.00|313.60|452.24|461.28|18.81|0.00|313.60|332.41|-138.64| +2452090|63124|16501|19435|1704589|5390|47791|1|191|1592|80|97.12|168.01|70.56|1128.96|5644.80|7769.60|13440.80|361.26|1128.96|4515.84|4877.10|-3253.76| +2452297|52376|3471|30937|1050054|7124|49041|10|12|1593|53|3.78|6.72|6.51|0.00|345.03|200.34|356.16|27.60|0.00|345.03|372.63|144.69| +2452297|52376|3393|30937|1050054|7124|49041|10|224|1593|91|41.37|60.81|22.49|0.00|2046.59|3764.67|5533.71|61.39|0.00|2046.59|2107.98|-1718.08| +2452297|52376|9057|30937|1050054|7124|49041|10|23|1593|39|23.11|35.82|22.20|0.00|865.80|901.29|1396.98|77.92|0.00|865.80|943.72|-35.49| +2452297|52376|10995|30937|||49041|||1593|7||149.87|20.98|0.00|146.86|||1.46|0.00|146.86||| +2452297|52376|6997|30937|1050054|7124|49041|10|109|1593|46|41.76|63.89|41.52|0.00|1909.92|1920.96|2938.94|0.00|0.00|1909.92|1909.92|-11.04| +2452297|52376|1920|30937|1050054|7124|49041|10|153|1593|42|41.07|62.01|31.62|996.03|1328.04|1724.94|2604.42|0.00|996.03|332.01|332.01|-1392.93| +2452297||1539|30937|1050054||49041|10||1593||45.31|76.12|||2159.89|||||||| +2452297|52376|60|30937|1050054|7124|49041|10|175|1593|92|61.63|91.21|12.76|0.00|1173.92|5669.96|8391.32|58.69|0.00|1173.92|1232.61|-4496.04| +2452297|52376|12525|30937|1050054|7124|49041|10|295|1593|35|66.97|78.35|50.92|0.00|1782.20|2343.95|2742.25|71.28|0.00|1782.20|1853.48|-561.75| +2452297|52376|297|30937|1050054|7124|49041|10|120|1593|19|62.28|106.49|71.34|0.00|1355.46|1183.32|2023.31|121.99|0.00|1355.46|1477.45|172.14| +2452297|52376|7176|30937|1050054|7124|49041|10|144|1593|32|20.90|35.11|17.90|0.00|572.80|668.80|1123.52|0.00|0.00|572.80|572.80|-96.00| +2452297|52376|17715|30937|1050054|7124|49041|10|224|1593|54|48.47|80.94|25.90|0.00|1398.60|2617.38|4370.76|55.94|0.00|1398.60|1454.54|-1218.78| +2452297|52376|12339|30937|1050054|7124|49041|10|111|1593|96|30.18|52.51|49.88|0.00|4788.48|2897.28|5040.96|239.42|0.00|4788.48|5027.90|1891.20| +2452297|52376|10308|30937|1050054|7124|49041|10|43|1593|27|31.77|58.45|36.23|0.00|978.21|857.79|1578.15|88.03|0.00|978.21|1066.24|120.42| +2452297||4032|30937||7124|49041||89|1593|||||||1273.66|2279.00|6.60||660.48||| +2452297|52376|17463|30937|1050054|7124|49041|10|272|1593|48|50.24|52.75|26.90|0.00|1291.20|2411.52|2532.00|25.82|0.00|1291.20|1317.02|-1120.32| +2451185|46241|10666|13219|687744|3869|17994|4|104|1594|2|56.11|58.35|22.75|0.00|45.50|112.22|116.70|4.09|0.00|45.50|49.59|-66.72| +2451185|46241|1231|13219|687744|3869|17994|4|62|1594|60|14.90|27.86|13.09|0.00|785.40|894.00|1671.60|23.56|0.00|785.40|808.96|-108.60| +2451185||6692|13219|687744|3869||4||1594|42|92.99||94.07|||||237.05||3950.94||45.36| +2451185|46241|10730|13219|687744|3869|17994|4|214|1594|86|50.39|83.14|65.68|0.00|5648.48|4333.54|7150.04|56.48|0.00|5648.48|5704.96|1314.94| +2451185|46241|17839|13219|687744|3869|17994|4|164|1594|29|53.35|101.36|63.85|92.58|1851.65|1547.15|2939.44|70.36|92.58|1759.07|1829.43|211.92| +2451185|46241|10996|13219|687744|3869|17994|4|10|1594|82|28.33|31.16|17.13|0.00|1404.66|2323.06|2555.12|56.18|0.00|1404.66|1460.84|-918.40| +2451185|46241|15932|13219|687744|3869|17994|4|268|1594|1|3.00|5.64|2.19|1.09|2.19|3.00|5.64|0.08|1.09|1.10|1.18|-1.90| +2451185|46241|9631|13219|687744|3869|17994|4|251|1594|39|71.69|101.08|51.55|0.00|2010.45|2795.91|3942.12|20.10|0.00|2010.45|2030.55|-785.46| +2451185|46241|10633|13219|687744|3869|17994|4|257|1594|41|61.42|106.25|18.06|81.45|740.46|2518.22|4356.25|52.72|81.45|659.01|711.73|-1859.21| +2451185|46241|2102|13219|687744|3869|17994|4|295|1594|53|61.72|114.79|22.95|0.00|1216.35|3271.16|6083.87|36.49|0.00|1216.35|1252.84|-2054.81| +2451185|46241|2749|13219|687744|3869|17994|4|147|1594|83|50.83|66.58|41.27|239.77|3425.41|4218.89|5526.14|159.28|239.77|3185.64|3344.92|-1033.25| +2451185|46241|7255|13219|687744|3869|17994|4|42|1594|92|64.98|85.12|47.66|0.00|4384.72|5978.16|7831.04|175.38|0.00|4384.72|4560.10|-1593.44| +2451185|46241|15259|13219|687744|3869|17994|4|124|1594|22|7.62|9.90|0.19|0.00|4.18|167.64|217.80|0.37|0.00|4.18|4.55|-163.46| +2451185|46241|11018|13219|687744|3869|17994|4|101|1594|30|81.16|142.03|41.18|0.00|1235.40|2434.80|4260.90|12.35|0.00|1235.40|1247.75|-1199.40| +2451185|46241|14554|13219|687744|3869|17994|4|252|1594|74|35.25|48.99|18.12|0.00|1340.88|2608.50|3625.26|67.04|0.00|1340.88|1407.92|-1267.62| +2452178|39714|12311|15212|1108733|5901|49469|1|50|1595|53|99.11|160.55|16.05|0.00|850.65|5252.83|8509.15|51.03|0.00|850.65|901.68|-4402.18| +2452178|39714|7423|15212|1108733|5901|49469|1|124|1595|1|85.18|160.99|82.10|59.93|82.10|85.18|160.99|1.10|59.93|22.17|23.27|-63.01| +2452178|39714|1245|15212|1108733|5901|49469|1|225|1595|78|22.26|29.16|12.24|0.00|954.72|1736.28|2274.48|85.92|0.00|954.72|1040.64|-781.56| +2452178|39714|5809|15212|1108733|5901|49469|1|25|1595|75|12.30|16.72|11.36|817.92|852.00|922.50|1254.00|1.70|817.92|34.08|35.78|-888.42| +2452178|39714|1079|15212|1108733|5901|49469|1|124|1595|42|40.13|49.76|47.27|972.81|1985.34|1685.46|2089.92|91.12|972.81|1012.53|1103.65|-672.93| +2452178|39714|3085|15212|1108733|5901|49469|1|23|1595|94|72.67|140.25|28.05|2531.23|2636.70|6830.98|13183.50|6.32|2531.23|105.47|111.79|-6725.51| +2452178|39714|13477|15212|1108733|5901|49469|1|102|1595|18|12.33|21.33|17.70|0.00|318.60|221.94|383.94|12.74|0.00|318.60|331.34|96.66| +2452178|39714|5469|15212|1108733|5901|49469|1|248|1595|16|46.21|76.70|12.27|0.00|196.32|739.36|1227.20|9.81|0.00|196.32|206.13|-543.04| +2451142|53142|15562|95387|278634|5528|45622|10|118|1596|25|57.25|107.05|66.37|0.00|1659.25|1431.25|2676.25|33.18|0.00|1659.25|1692.43|228.00| +2451142|53142|11401|95387|278634|5528|45622|10|40|1596|4|59.28|85.36|80.23|131.57|320.92|237.12|341.44|13.25|131.57|189.35|202.60|-47.77| +2451142|53142|910|95387|278634|5528|45622|10|284|1596|94|38.31|72.40|34.75|0.00|3266.50|3601.14|6805.60|130.66|0.00|3266.50|3397.16|-334.64| +2451142|53142|13531|95387|278634|5528|45622|10|79|1596|76|18.01|35.11|26.68|0.00|2027.68|1368.76|2668.36|60.83|0.00|2027.68|2088.51|658.92| +2451142|53142|12364|95387|278634|5528|45622|10|282|1596|24|1.68|2.04|0.89|19.86|21.36|40.32|48.96|0.10|19.86|1.50|1.60|-38.82| +2451142|53142|4573|95387|278634|5528|45622|10|70|1596|54|85.59|101.85|4.07|0.00|219.78|4621.86|5499.90|19.78|0.00|219.78|239.56|-4402.08| +||1405|95387|||45622||31|1596||34.61|64.37||0.00|970.58|||48.52|0.00|||| +2451142|53142|6451|95387|278634|5528|45622|10|135|1596|37|69.35|74.20|34.87|0.00|1290.19|2565.95|2745.40|51.60|0.00|1290.19|1341.79|-1275.76| +2451142|53142|10844|95387|278634|5528|45622|10|151|1596|77|94.44|112.38|87.65|0.00|6749.05|7271.88|8653.26|134.98|0.00|6749.05|6884.03|-522.83| +2451142|53142|4138|95387|278634|5528|45622|10|232|1596|97|10.18|18.12|3.26|0.00|316.22|987.46|1757.64|6.32|0.00|316.22|322.54|-671.24| +2451142|53142|2023|95387|278634|5528|45622|10|178|1596|43|3.82|6.37|4.26|0.00|183.18|164.26|273.91|1.83|0.00|183.18|185.01|18.92| +2451091|48934|2149|89134|1078463|945|7074|10|113|1597|14|2.90|5.19|5.03|0.00|70.42|40.60|72.66|2.81|0.00|70.42|73.23|29.82| +2451091|48934|15254|89134|1078463|945|7074|10|218|1597|12|49.93|91.87|44.09|0.00|529.08|599.16|1102.44|0.00|0.00|529.08|529.08|-70.08| +2451091|48934|15055|89134|1078463|945|7074|10|216|1597|33|9.13|14.97|9.28|0.00|306.24|301.29|494.01|9.18|0.00|306.24|315.42|4.95| +2451091|48934|6505|89134|1078463|945|7074|10|203|1597|46|35.71|37.85|21.19|0.00|974.74|1642.66|1741.10|68.23|0.00|974.74|1042.97|-667.92| +2451091|48934|14234|89134|1078463|945|7074|10|29|1597|81|47.55|58.96|53.65|0.00|4345.65|3851.55|4775.76|304.19|0.00|4345.65|4649.84|494.10| +2451091|48934|10237|89134|1078463|945|7074|10|223|1597|96|23.16|37.28|26.46|0.00|2540.16|2223.36|3578.88|152.40|0.00|2540.16|2692.56|316.80| +2451091|48934|2522|89134|1078463|945|7074|10|86|1597|99|69.53|128.63|124.77|0.00|12352.23|6883.47|12734.37|1111.70|0.00|12352.23|13463.93|5468.76| +2451091|48934|8584|89134|1078463|945|7074|10|198|1597|79|88.16|125.18|40.05|0.00|3163.95|6964.64|9889.22|253.11|0.00|3163.95|3417.06|-3800.69| +2451806|50314|1211|5886|1574192|3298|851|8|43|1598|32|97.54|190.20|154.06|0.00|4929.92|3121.28|6086.40|147.89|0.00|4929.92|5077.81|1808.64| +2451806|50314|17389|5886|1574192|3298|851|8|34|1598|6|34.02|40.82|40.00|0.00|240.00|204.12|244.92|4.80|0.00|240.00|244.80|35.88| +2451806||12608||||851|8||1598|41|26.77|35.60||||1097.57||38.08|||461.20|| +2451806|50314|6422|5886|1574192|3298|851|8|198|1598|93|50.42|98.82|94.86|1852.61|8821.98|4689.06|9190.26|487.85|1852.61|6969.37|7457.22|2280.31| +2451806|50314|1178|5886|1574192|3298|851|8|236|1598|58|24.12|30.39|14.28|0.00|828.24|1398.96|1762.62|57.97|0.00|828.24|886.21|-570.72| +2451806|50314|10529|5886|1574192|3298|851|8|277|1598|73|68.57|76.11|60.12|0.00|4388.76|5005.61|5556.03|43.88|0.00|4388.76|4432.64|-616.85| +2451806|50314|2815|5886|1574192|3298|851|8|28|1598|38|40.09|70.95|38.31|0.00|1455.78|1523.42|2696.10|43.67|0.00|1455.78|1499.45|-67.64| +2451806|50314|5156|5886|1574192|3298|851|8|176|1598|70|16.93|26.58|11.96|0.00|837.20|1185.10|1860.60|66.97|0.00|837.20|904.17|-347.90| +2451806|50314|14564|5886|1574192|3298|851|8|253|1598|25|70.81|108.33|61.74|0.00|1543.50|1770.25|2708.25|0.00|0.00|1543.50|1543.50|-226.75| +2451806||15248||1574192|3298|851|8|92|1598|75||||0.00||||205.46|0.00|4109.25|4314.71|| +2451806|50314|5579|5886|1574192|3298|851|8|202|1598|19|81.13|151.71|24.27|0.00|461.13|1541.47|2882.49|36.89|0.00|461.13|498.02|-1080.34| +2451806|50314|13669|5886|1574192|3298|851|8|284|1598|28|1.32|1.86|1.09|18.61|30.52|36.96|52.08|0.83|18.61|11.91|12.74|-25.05| +2451806|50314|7441|5886|1574192|3298|851|8|208|1598|52|58.09|84.23|49.69|0.00|2583.88|3020.68|4379.96|129.19|0.00|2583.88|2713.07|-436.80| +2451806|50314|7598|5886|1574192|3298|851|8|100|1598|35|93.39|165.30|59.50|0.00|2082.50|3268.65|5785.50|187.42|0.00|2082.50|2269.92|-1186.15| +2451806|50314|806|5886|1574192|3298|851|8|139|1598|94|3.64|5.20|0.41|0.00|38.54|342.16|488.80|2.69|0.00|38.54|41.23|-303.62| +2451806|50314|6716|5886|1574192|3298|851|8|299|1598|58|33.27|47.24|44.40|0.00|2575.20|1929.66|2739.92|180.26|0.00|2575.20|2755.46|645.54| +2452145|29588|9837|25283|373136|1789|20042|8|53|1599|73|89.82|154.49|100.41|0.00|7329.93|6556.86|11277.77|146.59|0.00|7329.93|7476.52|773.07| +2452145|29588|11115|25283|373136|1789|20042|8|44|1599|76|32.22|36.73|10.65|0.00|809.40|2448.72|2791.48|72.84|0.00|809.40|882.24|-1639.32| +2452145|29588|7557|25283|373136|1789|20042|8|298|1599|69|41.28|63.98|62.06|0.00|4282.14|2848.32|4414.62|0.00|0.00|4282.14|4282.14|1433.82| +2452145|29588|12823|25283|373136|1789|20042|8|154|1599|85|74.44|144.41|125.63|0.00|10678.55|6327.40|12274.85|640.71|0.00|10678.55|11319.26|4351.15| +2452145|29588|7701|25283|373136|1789|20042|8|75|1599|48|7.30|12.11|11.02|105.79|528.96|350.40|581.28|29.62|105.79|423.17|452.79|72.77| +2452145|29588|443|25283|373136|1789|20042|8|16|1599|64|61.39|109.88|68.12|0.00|4359.68|3928.96|7032.32|217.98|0.00|4359.68|4577.66|430.72| +2452145|29588|2805|25283|373136|1789|20042|8|215|1599|93|59.28|84.77|12.71|0.00|1182.03|5513.04|7883.61|94.56|0.00|1182.03|1276.59|-4331.01| +2452145|29588|7371|25283|373136|1789|20042|8|28|1599|95|31.79|33.06|5.62|0.00|533.90|3020.05|3140.70|16.01|0.00|533.90|549.91|-2486.15| +|29588|14355|||||||1599|||||1278.63||2832.90||28.93|1278.63||993.52|| +2452145|29588|3027|25283|373136|1789|20042|8|141|1599|38|9.85|13.00|3.51|0.00|133.38|374.30|494.00|5.33|0.00|133.38|138.71|-240.92| +2452145|29588|5407|25283|373136|1789|20042|8|30|1599|58|47.77|95.54|0.00|0.00|0.00|2770.66|5541.32|0.00|0.00|0.00|0.00|-2770.66| +2452145|29588|1277|25283|373136|1789|20042|8|114|1599|76|57.58|77.73|55.18|0.00|4193.68|4376.08|5907.48|0.00|0.00|4193.68|4193.68|-182.40| +2452145|29588|7403|25283|373136|1789|20042|8|45|1599|37|3.30|4.38|0.26|0.00|9.62|122.10|162.06|0.00|0.00|9.62|9.62|-112.48| +2452045|42657|9315|17919|1849987|6502|13770|1|3|1600|78|14.18|17.15|7.20|0.00|561.60|1106.04|1337.70|0.00|0.00|561.60|561.60|-544.44| +2452045|42657|7689|17919|1849987|6502|13770|1|295|1600|56|98.61|103.54|41.41|0.00|2318.96|5522.16|5798.24|69.56|0.00|2318.96|2388.52|-3203.20| +2452045|42657|427|17919|1849987|6502|13770|1|47|1600|1|56.16|98.84|82.03|0.00|82.03|56.16|98.84|5.74|0.00|82.03|87.77|25.87| +2452045|42657|5925|17919|1849987|6502|13770|1|181|1600|71|85.06|170.12|42.53|0.00|3019.63|6039.26|12078.52|60.39|0.00|3019.63|3080.02|-3019.63| +2452045|42657|16313|17919|1849987|6502|13770|1|39|1600|95|50.15|59.17|59.17|0.00|5621.15|4764.25|5621.15|168.63|0.00|5621.15|5789.78|856.90| +2452045|42657|16749|17919|1849987|6502|13770|1|214|1600|31|68.44|102.66|80.07|0.00|2482.17|2121.64|3182.46|223.39|0.00|2482.17|2705.56|360.53| +2452045|42657|9787|17919|1849987|6502|13770|1|135|1600|5|20.25|38.47|16.15|51.68|80.75|101.25|192.35|0.58|51.68|29.07|29.65|-72.18| +2452045|42657|11453|17919|1849987|6502|13770|1|32|1600|19|98.02|193.09|57.92|0.00|1100.48|1862.38|3668.71|77.03|0.00|1100.48|1177.51|-761.90| +2452045|42657|14153|17919|1849987|6502|13770|1|57|1600|36|87.01|111.37|102.46|0.00|3688.56|3132.36|4009.32|110.65|0.00|3688.56|3799.21|556.20| +2452045|42657|12425|17919|1849987|6502|13770|1|128|1600|82|8.27|13.39|0.93|0.00|76.26|678.14|1097.98|4.57|0.00|76.26|80.83|-601.88| +2452045|42657|189|17919|1849987|6502|13770|1|129|1600|84|29.98|51.56|8.24|0.00|692.16|2518.32|4331.04|13.84|0.00|692.16|706.00|-1826.16| +2451454|53563|17470|3155|1829341|2727|45313|7|255|1601|12|74.84|86.81|31.25|0.00|375.00|898.08|1041.72|11.25|0.00|375.00|386.25|-523.08| +2451454|53563|5582|3155|1829341|2727|45313|7|150|1601|66|49.55|58.46|36.24|0.00|2391.84|3270.30|3858.36|215.26|0.00|2391.84|2607.10|-878.46| +2451454|53563|40|3155|1829341|2727|45313|7|52|1601|13|96.48|185.24|5.55|47.61|72.15|1254.24|2408.12|0.00|47.61|24.54|24.54|-1229.70| +2451454|53563|13030|3155|1829341|2727|45313|7|288|1601|82|35.50|65.32|15.67|0.00|1284.94|2911.00|5356.24|25.69|0.00|1284.94|1310.63|-1626.06| +2451454|53563|12698|3155|1829341|2727|45313|7|91|1601|19|20.00|36.40|6.91|0.00|131.29|380.00|691.60|3.93|0.00|131.29|135.22|-248.71| +|53563|7477||||45313|||1601|73||37.72|||||2753.56|||||| +2451454|53563|11053|3155|1829341|2727|45313|7|94|1601|67|3.36|6.14|0.12|0.00|8.04|225.12|411.38|0.32|0.00|8.04|8.36|-217.08| +2451454|53563|10004|3155|1829341|2727|45313|7|10|1601|51|25.11|28.37|5.95|84.96|303.45|1280.61|1446.87|4.36|84.96|218.49|222.85|-1062.12| +2451454|53563|11950|3155|1829341|2727|45313|7|49|1601|76|18.99|25.44|4.83|0.00|367.08|1443.24|1933.44|33.03|0.00|367.08|400.11|-1076.16| +2451454|53563|13351|3155|1829341|2727|45313|7|91|1601|68|16.70|24.71|22.98|0.00|1562.64|1135.60|1680.28|46.87|0.00|1562.64|1609.51|427.04| +2451454|53563|7396|3155|1829341|2727|45313|7|78|1601|59|27.28|43.64|0.43|0.00|25.37|1609.52|2574.76|1.26|0.00|25.37|26.63|-1584.15| +2451454|53563|7981|3155|1829341|2727|45313|7|181|1601|67|5.40|7.39|0.96|0.00|64.32|361.80|495.13|1.92|0.00|64.32|66.24|-297.48| +2451454|53563|10114|3155|1829341|2727|45313|7|295|1601|54|11.11|13.44|7.12|0.00|384.48|599.94|725.76|23.06|0.00|384.48|407.54|-215.46| +2451141|48268|175|20667|1866113|5159|37405|1|220|1602|17|38.71|51.09|27.07|349.74|460.19|658.07|868.53|3.31|349.74|110.45|113.76|-547.62| +2451141|48268|15838|20667|1866113|5159|37405|1|252|1602|96|10.72|16.61|8.96|0.00|860.16|1029.12|1594.56|17.20|0.00|860.16|877.36|-168.96| +2451141|48268|16486|20667|1866113|5159|37405|1|283|1602|33|50.27|71.38|37.11|0.00|1224.63|1658.91|2355.54|24.49|0.00|1224.63|1249.12|-434.28| +2451141|48268|2986|20667|1866113|5159|37405|1|202|1602|77|85.04|136.91|13.69|0.00|1054.13|6548.08|10542.07|0.00|0.00|1054.13|1054.13|-5493.95| +2451141|48268|14552|20667|1866113|5159|37405|1|76|1602|55|46.02|57.52|1.15|0.00|63.25|2531.10|3163.60|5.69|0.00|63.25|68.94|-2467.85| +2451141|48268|7879||1866113||37405||223|1602||28.68||5.14|0.00||1032.48|||0.00|185.04||-847.44| +2451141|48268|4159|20667|1866113|5159|37405|1|117|1602|56|38.24|42.82|36.82|0.00|2061.92|2141.44|2397.92|0.00|0.00|2061.92|2061.92|-79.52| +2451141||15925|||||1|269|1602|||53.72|||1933.60||2148.80|||||| +|50793|16694|97058|||31121|4||1603|74||94.98|79.78||5903.72|||177.11||||| +2451861|50793|326|97058|1081079|3500|31121|4|202|1603|60|83.85|148.41|90.53|0.00|5431.80|5031.00|8904.60|434.54|0.00|5431.80|5866.34|400.80| +2451861|50793|5492|97058|1081079|3500|31121|4|268|1603|30|7.76|8.07|7.50|0.00|225.00|232.80|242.10|15.75|0.00|225.00|240.75|-7.80| +2451861|50793|9620|97058|1081079|3500|31121|4|293|1603|71|37.11|60.11|31.85|0.00|2261.35|2634.81|4267.81|45.22|0.00|2261.35|2306.57|-373.46| +2451861|50793|8653|97058|1081079|3500|31121|4|205|1603|70|46.85|81.98|33.61|0.00|2352.70|3279.50|5738.60|117.63|0.00|2352.70|2470.33|-926.80| +2451861|50793|13208|97058|1081079|3500|31121|4|162|1603|66|82.41|115.37|34.61|982.23|2284.26|5439.06|7614.42|26.04|982.23|1302.03|1328.07|-4137.03| +2451861|50793|3175|97058|1081079|3500|31121|4|184|1603|88|11.05|18.89|10.38|0.00|913.44|972.40|1662.32|54.80|0.00|913.44|968.24|-58.96| +2451861|50793|10069|97058|1081079|3500|31121|4|78|1603|37|38.47|52.31|35.04|272.26|1296.48|1423.39|1935.47|20.48|272.26|1024.22|1044.70|-399.17| +2451861|50793|3755|97058|1081079|3500|31121|4|185|1603|80|11.30|20.00|11.80|0.00|944.00|904.00|1600.00|37.76|0.00|944.00|981.76|40.00| +2451861|50793|3278|97058|1081079|3500|31121|4|295|1603|34|19.35|24.96|16.47|0.00|559.98|657.90|848.64|16.79|0.00|559.98|576.77|-97.92| +2451861|50793|3542|97058|1081079|3500|31121|4|141|1603|4|91.57|162.07|124.79|0.00|499.16|366.28|648.28|4.99|0.00|499.16|504.15|132.88| +2451861|50793|7694|97058|1081079|3500|31121|4|119|1603|12|57.25|74.99|70.49|0.00|845.88|687.00|899.88|76.12|0.00|845.88|922.00|158.88| +2451715|53407|5552|33640|1703363|1439|16762|7|293|1604|76|39.44|45.75|25.62|700.96|1947.12|2997.44|3477.00|49.84|700.96|1246.16|1296.00|-1751.28| +2451715|53407|1319|33640|1703363|1439|16762|7|207|1604|35|21.70|37.10|17.06|0.00|597.10|759.50|1298.50|35.82|0.00|597.10|632.92|-162.40| +2451715|53407|8318|33640|1703363|1439|16762|7|141|1604|33|62.24|84.02|20.16|0.00|665.28|2053.92|2772.66|13.30|0.00|665.28|678.58|-1388.64| +2451715|53407|121|33640|1703363|1439|16762|7|272|1604|27|63.00|125.37|20.05|487.21|541.35|1701.00|3384.99|4.33|487.21|54.14|58.47|-1646.86| +2451715|53407|9773|33640|1703363|1439|16762|7|282|1604|68|9.66|9.94|7.05|0.00|479.40|656.88|675.92|9.58|0.00|479.40|488.98|-177.48| +2451715|53407|7631|33640|1703363|1439|16762|7|255|1604|9|61.03|90.93|18.18|0.00|163.62|549.27|818.37|6.54|0.00|163.62|170.16|-385.65| +2451715|53407|10463|33640|1703363|1439|16762|7|242|1604|79|93.75|161.25|67.72|3691.41|5349.88|7406.25|12738.75|49.75|3691.41|1658.47|1708.22|-5747.78| +2451715|53407|5057|33640|1703363|1439|16762|7|37|1604|7|87.63|175.26|108.66|0.00|760.62|613.41|1226.82|0.00|0.00|760.62|760.62|147.21| +2451715|53407|9152|33640|1703363|1439|16762|7|277|1604|31|32.83|52.52|23.63|0.00|732.53|1017.73|1628.12|43.95|0.00|732.53|776.48|-285.20| +2451715|53407|3641|33640|1703363|1439|16762|7|258|1604|33|99.34|139.07|122.38|0.00|4038.54|3278.22|4589.31|121.15|0.00|4038.54|4159.69|760.32| +2451880|34840|745|78048|1815120|1792|29711|8|178|1605|77|94.00|104.34|69.90|0.00|5382.30|7238.00|8034.18|376.76|0.00|5382.30|5759.06|-1855.70| +2451880|34840|4289|78048|1815120|1792|29711|8|179|1605|73|29.28|34.25|10.96|0.00|800.08|2137.44|2500.25|72.00|0.00|800.08|872.08|-1337.36| +2451880|34840|11267|78048|1815120|1792|29711|8|21|1605|20|49.60|74.89|49.42|0.00|988.40|992.00|1497.80|39.53|0.00|988.40|1027.93|-3.60| +2451880|34840|12260|78048|1815120|1792|29711|8|203|1605|4|66.76|111.48|109.25|0.00|437.00|267.04|445.92|39.33|0.00|437.00|476.33|169.96| +2451880|34840|1667|78048|1815120|1792|29711|8|194|1605|67|28.90|44.50|36.04|1835.15|2414.68|1936.30|2981.50|17.38|1835.15|579.53|596.91|-1356.77| +2451880|34840|13442|78048|1815120|1792|29711|8|247|1605|58|97.02|169.78|130.73|0.00|7582.34|5627.16|9847.24|303.29|0.00|7582.34|7885.63|1955.18| +2451880|34840|821|78048|1815120|1792|29711|8|240|1605|61|53.03|105.52|22.15|0.00|1351.15|3234.83|6436.72|40.53|0.00|1351.15|1391.68|-1883.68| +2451880|34840|6583|78048|1815120|1792|29711|8|253|1605|23|39.27|51.44|10.28|0.00|236.44|903.21|1183.12|0.00|0.00|236.44|236.44|-666.77| +2451880|34840|3176|78048|1815120|1792|29711|8|283|1605|12|53.08|103.50|27.94|0.00|335.28|636.96|1242.00|6.70|0.00|335.28|341.98|-301.68| +2451880|34840|2371|78048|1815120|1792|29711|8|51|1605|48|58.28|60.61|36.97|0.00|1774.56|2797.44|2909.28|141.96|0.00|1774.56|1916.52|-1022.88| +2451880|34840|10423|78048|1815120|1792|29711|8|123|1605|9|13.68|21.06|8.21|0.00|73.89|123.12|189.54|4.43|0.00|73.89|78.32|-49.23| +2451880|34840|2333|78048|1815120|1792|29711|8|296|1605|23|21.89|33.71|22.92|0.00|527.16|503.47|775.33|36.90|0.00|527.16|564.06|23.69| +2451880|34840|7429|78048|1815120|1792|29711|8|193|1605|6|91.08|171.23|35.95|0.00|215.70|546.48|1027.38|17.25|0.00|215.70|232.95|-330.78| +2451487|50564|6304|47664|1127245|2026|48854|4|63|1606|49|15.39|20.62|11.54|418.44|565.46|754.11|1010.38|7.35|418.44|147.02|154.37|-607.09| +2451487|50564|14380|47664|1127245|2026|48854|4|46|1606|73|83.06|137.04|65.77|0.00|4801.21|6063.38|10003.92|384.09|0.00|4801.21|5185.30|-1262.17| +2451487|50564|3727|47664|1127245|2026|48854|4|157|1606|60|9.09|9.90|9.00|0.00|540.00|545.40|594.00|32.40|0.00|540.00|572.40|-5.40| +||6946|47664|1127245|2026||4|107|1606||||18.39||1158.57||1544.76|||1158.57||318.78| +2451487|50564|15098|47664|1127245|2026|48854|4|262|1606|75|80.54|86.17|13.78|0.00|1033.50|6040.50|6462.75|82.68|0.00|1033.50|1116.18|-5007.00| +2451487|50564|14132|47664|1127245|2026|48854|4|55|1606|8|4.84|7.21|0.00|0.00|0.00|38.72|57.68|0.00|0.00|0.00|0.00|-38.72| +2451487|50564|8948|47664|1127245|2026|48854|4|182|1606|41|15.11|18.73|15.35|207.68|629.35|619.51|767.93|8.43|207.68|421.67|430.10|-197.84| +2451487|50564|2317|47664|1127245|2026|48854|4|178|1606|23|9.13|17.43|17.43|0.00|400.89|209.99|400.89|16.03|0.00|400.89|416.92|190.90| +2451676|51882|17861|92501|||31766|||1607|6|||22.60|134.24|135.60|92.82|||134.24|1.36|1.46|-91.46| +2451676|51882|2755|92501|1708758|4662|31766|4|208|1607|67|62.78|105.47|11.60|318.65|777.20|4206.26|7066.49|4.58|318.65|458.55|463.13|-3747.71| +2451676|51882|15287|92501|1708758|4662|31766|4|286|1607|51|37.29|59.29|32.60|0.00|1662.60|1901.79|3023.79|16.62|0.00|1662.60|1679.22|-239.19| +2451676|51882|3308|92501|1708758|4662|31766|4|105|1607|7|76.11|98.94|38.58|97.22|270.06|532.77|692.58|15.55|97.22|172.84|188.39|-359.93| +2451676|51882|7913|92501|1708758|4662|31766|4|223|1607|33|67.23|106.89|52.37|0.00|1728.21|2218.59|3527.37|17.28|0.00|1728.21|1745.49|-490.38| +2451676|51882|13733|92501|1708758|4662|31766|4|160|1607|100|6.38|11.86|7.35|0.00|735.00|638.00|1186.00|66.15|0.00|735.00|801.15|97.00| +2451676|51882|9499|92501|1708758|4662|31766|4|128|1607|35|82.01|104.15|4.16|0.00|145.60|2870.35|3645.25|13.10|0.00|145.60|158.70|-2724.75| +2451676|51882|1730|92501|1708758|4662|31766|4|123|1607|44|48.37|54.17|48.75|0.00|2145.00|2128.28|2383.48|150.15|0.00|2145.00|2295.15|16.72| +2451676|51882|13039|92501|1708758|4662|31766|4|218|1607|94|5.81|5.81|1.22|0.00|114.68|546.14|546.14|1.14|0.00|114.68|115.82|-431.46| +2451676|51882|14660|92501|1708758|4662|31766|4|150|1607|8|68.95|86.87|58.20|0.00|465.60|551.60|694.96|23.28|0.00|465.60|488.88|-86.00| +2451676|51882|6817|92501|1708758|4662|31766|4|179|1607|3|64.00|94.72|20.83|0.00|62.49|192.00|284.16|0.62|0.00|62.49|63.11|-129.51| +2451676|51882|12295|92501|1708758|4662|31766|4|254|1607|30|39.40|43.34|42.47|0.00|1274.10|1182.00|1300.20|38.22|0.00|1274.10|1312.32|92.10| +2451676|51882|16057|92501|1708758|4662|31766|4|276|1607|23|49.24|83.70|35.15|0.00|808.45|1132.52|1925.10|56.59|0.00|808.45|865.04|-324.07| +2451671|33073|10145|60586|1323239|5539|36192|8|300|1608|26|51.75|64.68|30.39|0.00|790.14|1345.50|1681.68|55.30|0.00|790.14|845.44|-555.36| +2451671|33073|5741|60586|1323239|5539|36192|8|154|1608|8|44.60|78.94|57.62|0.00|460.96|356.80|631.52|27.65|0.00|460.96|488.61|104.16| +2451671|33073|14762|60586|1323239|5539|36192|8|286|1608|1|56.10|103.78|37.36|4.10|37.36|56.10|103.78|2.99|4.10|33.26|36.25|-22.84| +2451671|33073|13736|60586|1323239|5539|36192|8|51|1608|16|72.78|132.45|121.85|0.00|1949.60|1164.48|2119.20|155.96|0.00|1949.60|2105.56|785.12| +2451671|33073|5600|60586|1323239|5539|36192|8|280|1608|88|34.01|55.43|26.05|0.00|2292.40|2992.88|4877.84|45.84|0.00|2292.40|2338.24|-700.48| +2451671|33073|13913|60586|1323239|5539|36192|8|243|1608|8|10.97|20.29|12.57|0.00|100.56|87.76|162.32|7.03|0.00|100.56|107.59|12.80| +2451671|33073|9091|60586|1323239|5539|36192|8|213|1608|93|27.61|38.10|25.52|0.00|2373.36|2567.73|3543.30|0.00|0.00|2373.36|2373.36|-194.37| +2451671|33073|703|60586|1323239|5539|36192|8|288|1608|17|28.99|35.65|17.82|0.00|302.94|492.83|606.05|27.26|0.00|302.94|330.20|-189.89| +2451619|40062|6145|49113|1384531|4749|967|4|203|1609|9|50.18|58.20|55.29|0.00|497.61|451.62|523.80|44.78|0.00|497.61|542.39|45.99| +2451619|40062|15727|49113|1384531|4749|967|4|296|1609|14|87.55|103.30|10.33|0.00|144.62|1225.70|1446.20|4.33|0.00|144.62|148.95|-1081.08| +2451619|40062|5225|49113|1384531|4749|967|4|91|1609|26|31.75|41.59|27.44|0.00|713.44|825.50|1081.34|64.20|0.00|713.44|777.64|-112.06| +2451619|40062|3853|49113|1384531|4749|967|4|168|1609|10|61.37|74.87|30.69|33.75|306.90|613.70|748.70|19.12|33.75|273.15|292.27|-340.55| +2451619|40062|14801|49113|1384531|4749|967|4|37|1609|71|80.53|103.88|92.45|0.00|6563.95|5717.63|7375.48|131.27|0.00|6563.95|6695.22|846.32| +2451619|40062|9655|49113|1384531|4749|967|4|22|1609|76|68.96|75.16|64.63|0.00|4911.88|5240.96|5712.16|442.06|0.00|4911.88|5353.94|-329.08| +2451619|40062|5510|49113|1384531|4749|967|4|80|1609|97|24.83|42.95|1.28|0.00|124.16|2408.51|4166.15|2.48|0.00|124.16|126.64|-2284.35| +2451619|40062|9392|49113|1384531|4749|967|4|66|1609|44|87.85|133.53|97.47|0.00|4288.68|3865.40|5875.32|385.98|0.00|4288.68|4674.66|423.28| +2451619|40062|15347|49113|1384531|4749|967|4|281|1609|72|53.69|87.51|28.87|0.00|2078.64|3865.68|6300.72|145.50|0.00|2078.64|2224.14|-1787.04| +2451619|40062|17413|49113|1384531|4749|967|4|162|1609|93|60.32|103.14|75.29|0.00|7001.97|5609.76|9592.02|280.07|0.00|7001.97|7282.04|1392.21| +2451619|40062|3158|49113|1384531|4749|967|4|62|1609|7|11.25|15.30|2.90|0.00|20.30|78.75|107.10|0.00|0.00|20.30|20.30|-58.45| +2451619|40062|5519|49113|1384531|4749|967|4|260|1609|22|24.26|48.03|36.02|776.59|792.44|533.72|1056.66|0.15|776.59|15.85|16.00|-517.87| +2451619|40062|6338|49113|1384531|4749|967|4|245|1609|15|22.34|29.93|26.63|0.00|399.45|335.10|448.95|35.95|0.00|399.45|435.40|64.35| +2451619||11563|||4749|967||106|1609||||110.52|0.00|4641.84|4026.96|5275.20|232.09|0.00|||| +2451619|40062|14665|49113|1384531|4749|967|4|240|1609|3|55.29|72.98|3.64|0.00|10.92|165.87|218.94|0.32|0.00|10.92|11.24|-154.95| +2451619|40062|9491|49113|1384531|4749|967|4|247|1609|31|43.13|43.99|11.43|109.84|354.33|1337.03|1363.69|7.33|109.84|244.49|251.82|-1092.54| +2451892|64155|16577|52849|1526614|5979|30144|8|137|1610|11|76.79|127.47|95.60|0.00|1051.60|844.69|1402.17|21.03|0.00|1051.60|1072.63|206.91| +2451892|64155|3721|52849|1526614|5979|30144|8|292|1610|96|52.35|53.39|16.55|0.00|1588.80|5025.60|5125.44|31.77|0.00|1588.80|1620.57|-3436.80| +2451892|64155|12872|52849|1526614|5979|30144|8|53|1610|36|49.05|82.40|63.44|0.00|2283.84|1765.80|2966.40|22.83|0.00|2283.84|2306.67|518.04| +2451892|64155|16609|52849|1526614|5979|30144|8|297|1610|51|6.43|11.89|5.23|0.00|266.73|327.93|606.39|21.33|0.00|266.73|288.06|-61.20| +2451892|64155|17197|52849|1526614|5979|30144|8|147|1610|32|37.25|37.62|29.34|0.00|938.88|1192.00|1203.84|65.72|0.00|938.88|1004.60|-253.12| +2451892|64155|866|52849|1526614|5979|30144|8|41|1610|85|97.65|195.30|97.65|0.00|8300.25|8300.25|16600.50|166.00|0.00|8300.25|8466.25|0.00| +2451892|64155|5081|52849|1526614|5979|30144|8|296|1610|7|85.42|131.54|93.39|0.00|653.73|597.94|920.78|13.07|0.00|653.73|666.80|55.79| +2451892|64155|671|52849|1526614|5979|30144|8|244|1610|17|87.87|103.68|65.31|0.00|1110.27|1493.79|1762.56|22.20|0.00|1110.27|1132.47|-383.52| +|64155|17797|52849|1526614||||157|1610|||72.21|||365.31||794.31|||241.11|260.39|| +2451892|64155|3917|52849|1526614|5979|30144|8|287|1610|88|4.70|6.58|3.09|0.00|271.92|413.60|579.04|5.43|0.00|271.92|277.35|-141.68| +2451892|64155|15241|52849|1526614|5979|30144|8|44|1610|37|38.94|68.92|22.05|0.00|815.85|1440.78|2550.04|8.15|0.00|815.85|824.00|-624.93| +2452277|34425|7704|74648|1869270|4339|45361|2|71|1611|87|27.83|53.43|20.30|0.00|1766.10|2421.21|4648.41|35.32|0.00|1766.10|1801.42|-655.11| +2452277|34425|10905|74648|1869270|4339|45361|2|56|1611|93|91.96|179.32|30.48|0.00|2834.64|8552.28|16676.76|85.03|0.00|2834.64|2919.67|-5717.64| +2452277|34425|9691|74648|1869270|4339|45361|2|67|1611|70|24.68|26.40|2.90|0.00|203.00|1727.60|1848.00|18.27|0.00|203.00|221.27|-1524.60| +2452277|34425|8208|74648|1869270|4339|45361|2|209|1611|71|33.51|63.33|57.63|0.00|4091.73|2379.21|4496.43|122.75|0.00|4091.73|4214.48|1712.52| +2452277|34425|1878|74648|1869270|4339|45361|2|24|1611|75|72.47|91.31|0.91|0.00|68.25|5435.25|6848.25|3.41|0.00|68.25|71.66|-5367.00| +2452277|34425|3030|74648|1869270|4339|45361|2|139|1611|1|28.77|51.49|5.14|1.02|5.14|28.77|51.49|0.20|1.02|4.12|4.32|-24.65| +2452277|34425|10704|74648|1869270|4339|45361|2|43|1611|94|26.71|50.48|22.71|0.00|2134.74|2510.74|4745.12|192.12|0.00|2134.74|2326.86|-376.00| +2452277|34425|3907|74648|1869270|4339|45361|2|133|1611|10|88.16|170.14|71.45|0.00|714.50|881.60|1701.40|35.72|0.00|714.50|750.22|-167.10| +2452277|34425|9423|74648|1869270|4339|45361|2|210|1611|59|17.53|26.82|3.21|124.99|189.39|1034.27|1582.38|5.15|124.99|64.40|69.55|-969.87| +2452277|34425|333|74648|1869270|4339|45361|2|110|1611|83|94.53|179.60|71.84|0.00|5962.72|7845.99|14906.80|238.50|0.00|5962.72|6201.22|-1883.27| +2452277|34425|11965|74648|1869270|4339|45361|2|40|1611|58|76.63|88.12|18.50|794.02|1073.00|4444.54|5110.96|19.52|794.02|278.98|298.50|-4165.56| +2452277|34425|1104|74648|1869270|4339|45361|2|232|1611|41|83.03|99.63|54.79|0.00|2246.39|3404.23|4084.83|157.24|0.00|2246.39|2403.63|-1157.84| +2452277|34425|2973|74648|1869270|4339|45361|2|263|1611|70|11.71|22.01|3.30|0.00|231.00|819.70|1540.70|13.86|0.00|231.00|244.86|-588.70| +2450894|36586|1300|35146|1454334|369|23957|1|81|1612|50|49.79|67.21|47.04|0.00|2352.00|2489.50|3360.50|0.00|0.00|2352.00|2352.00|-137.50| +2450894|36586|14155|35146|1454334|369|23957|1|296|1612|42|37.22|54.71|42.12|0.00|1769.04|1563.24|2297.82|88.45|0.00|1769.04|1857.49|205.80| +2450894|36586|4462|35146|1454334|369|23957|1|284|1612|10|11.67|14.23|1.56|0.00|15.60|116.70|142.30|1.09|0.00|15.60|16.69|-101.10| +2450894|36586|4201|35146|1454334|369|23957|1|186|1612|91|87.89|118.65|22.54|0.00|2051.14|7997.99|10797.15|0.00|0.00|2051.14|2051.14|-5946.85| +2450894|36586|4705|35146|1454334|369|23957|1|107|1612|74|53.46|73.77|58.27|0.00|4311.98|3956.04|5458.98|388.07|0.00|4311.98|4700.05|355.94| +2450894|36586|6908|35146|1454334|369|23957|1|298|1612|72|47.23|67.06|0.00|0.00|0.00|3400.56|4828.32|0.00|0.00|0.00|0.00|-3400.56| +2450894|36586|841|35146|1454334|369|23957|1|69|1612|76|54.92|102.15|42.90|1238.95|3260.40|4173.92|7763.40|141.50|1238.95|2021.45|2162.95|-2152.47| +2450894|36586|5140|35146|1454334|369|23957|1|147|1612|11|97.69|154.35|12.34|0.00|135.74|1074.59|1697.85|1.35|0.00|135.74|137.09|-938.85| +2450894|36586|8900|35146|1454334|369|23957|1|270|1612|23|77.53|85.28|58.84|0.00|1353.32|1783.19|1961.44|0.00|0.00|1353.32|1353.32|-429.87| +2450894|36586|10258|35146|1454334|369|23957|1|275|1612|63|33.35|38.68|35.58|0.00|2241.54|2101.05|2436.84|201.73|0.00|2241.54|2443.27|140.49| +2450894|36586|17168|35146|1454334|369|23957|1|135|1612|78|74.95|128.91|91.52|0.00|7138.56|5846.10|10054.98|0.00|0.00|7138.56|7138.56|1292.46| +2450894|36586|334|35146|1454334|369|23957|1|122|1612|30|44.74|47.87|47.87|0.00|1436.10|1342.20|1436.10|129.24|0.00|1436.10|1565.34|93.90| +2450887|47105|5600|41016|1439648|2727|22606|4|26|1613|55|1.01|1.53|1.20|0.00|66.00|55.55|84.15|1.98|0.00|66.00|67.98|10.45| +2450887|47105|13912|41016|1439648|2727|22606|4|226|1613|34|98.94|172.15|1.72|0.00|58.48|3363.96|5853.10|3.50|0.00|58.48|61.98|-3305.48| +2450887|47105|9091|41016|1439648|2727|22606|4|209|1613|79|32.71|60.51|21.17|0.00|1672.43|2584.09|4780.29|117.07|0.00|1672.43|1789.50|-911.66| +2450887|47105|703|41016|1439648|2727|22606|4|249|1613|86|83.99|162.10|157.23|5003.05|13521.78|7223.14|13940.60|596.31|5003.05|8518.73|9115.04|1295.59| +2450887|47105|12667|41016|1439648|2727|22606|4|83|1613|30|62.63|82.04|54.96|0.00|1648.80|1878.90|2461.20|115.41|0.00|1648.80|1764.21|-230.10| +2450887|47105|16870|41016|1439648|2727|22606|4|75|1613|39|31.17|48.62|47.16|128.74|1839.24|1215.63|1896.18|34.21|128.74|1710.50|1744.71|494.87| +2450887|47105|4546|41016|1439648|2727|22606|4|19|1613|45|98.57|144.89|0.00|0.00|0.00|4435.65|6520.05|0.00|0.00|0.00|0.00|-4435.65| +2450887|47105|2959|41016|1439648|2727|22606|4|290|1613|28|88.14|94.30|85.81|0.00|2402.68|2467.92|2640.40|0.00|0.00|2402.68|2402.68|-65.24| +2450887|47105|12967|41016|1439648|2727|22606|4|260|1613|49|80.46|98.96|23.75|0.00|1163.75|3942.54|4849.04|93.10|0.00|1163.75|1256.85|-2778.79| +2450887|47105|1837|41016|1439648|2727|22606|4|143|1613|12|18.61|37.22|1.11|0.00|13.32|223.32|446.64|0.39|0.00|13.32|13.71|-210.00| +2450887|47105|8666|41016|1439648|2727|22606|4|129|1613|25|32.03|51.56|14.43|0.00|360.75|800.75|1289.00|32.46|0.00|360.75|393.21|-440.00| +2450887|47105|12572|41016|1439648|2727|22606|4|138|1613|97|4.99|7.48|3.44|0.00|333.68|484.03|725.56|6.67|0.00|333.68|340.35|-150.35| +||8215||54876|6316||||1614||23.18||31.91|0.00|925.39|||0.00|0.00|||| +2452088|63939|8699|68236|54876|6316|25403|8|199|1614|8|83.79|94.68|12.30|0.00|98.40|670.32|757.44|0.00|0.00|98.40|98.40|-571.92| +2452088|63939|4427|68236|54876|6316|25403|8|71|1614|76|64.36|124.85|12.48|0.00|948.48|4891.36|9488.60|85.36|0.00|948.48|1033.84|-3942.88| +2452088|63939|5383|68236|54876|6316|25403|8|286|1614|16|74.48|76.71|5.36|14.57|85.76|1191.68|1227.36|2.84|14.57|71.19|74.03|-1120.49| +2452088|63939|7829|68236|54876|6316|25403|8|296|1614|18|24.75|34.65|4.50|0.00|81.00|445.50|623.70|2.43|0.00|81.00|83.43|-364.50| +2452088|63939|12349|68236|54876|6316|25403|8|109|1614|97|59.70|64.47|34.16|0.00|3313.52|5790.90|6253.59|198.81|0.00|3313.52|3512.33|-2477.38| +2452088|63939|3109|68236|54876|6316|25403|8|265|1614|71|50.52|94.97|69.32|1624.16|4921.72|3586.92|6742.87|230.82|1624.16|3297.56|3528.38|-289.36| +2452088|63939|2435|68236|54876|6316|25403|8|264|1614|74|65.59|101.66|60.99|2888.48|4513.26|4853.66|7522.84|81.23|2888.48|1624.78|1706.01|-3228.88| +2452088|63939|513|68236|54876|6316|25403|8|92|1614|45|54.32|72.78|59.67|1181.46|2685.15|2444.40|3275.10|0.00|1181.46|1503.69|1503.69|-940.71| +2452088|63939|5747|68236|54876|6316|25403|8|120|1614|40|7.59|8.27|0.41|3.28|16.40|303.60|330.80|0.52|3.28|13.12|13.64|-290.48| +2452088|63939|13029|68236|54876|6316|25403|8|167|1614|75|91.78|174.38|34.87|0.00|2615.25|6883.50|13078.50|26.15|0.00|2615.25|2641.40|-4268.25| +2452088|63939|15601|68236|54876|6316|25403|8|31|1614|5|19.39|38.39|26.10|0.00|130.50|96.95|191.95|1.30|0.00|130.50|131.80|33.55| +2452088|63939|14851|68236|54876|6316|25403|8|120|1614|72|19.08|22.32|16.96|0.00|1221.12|1373.76|1607.04|109.90|0.00|1221.12|1331.02|-152.64| +2452088|63939|7787|68236|54876|6316|25403|8|109|1614|84|98.75|135.28|133.92|0.00|11249.28|8295.00|11363.52|0.00|0.00|11249.28|11249.28|2954.28| +2452088|63939|9869|68236|54876|6316|25403|8|249|1614|14|19.80|29.30|1.46|0.00|20.44|277.20|410.20|0.40|0.00|20.44|20.84|-256.76| +2452088|63939|7319|68236|54876|6316|25403|8|183|1614|79|42.61|85.22|73.28|0.00|5789.12|3366.19|6732.38|289.45|0.00|5789.12|6078.57|2422.93| +2451781|53449|13574|41094|1183116|6738|26313|1|81|1615|41|84.56|124.30|118.08|0.00|4841.28|3466.96|5096.30|242.06|0.00|4841.28|5083.34|1374.32| +2451781|53449|10481|41094|1183116|6738|26313|1|201|1615|38|40.39|42.81|7.70|0.00|292.60|1534.82|1626.78|5.85|0.00|292.60|298.45|-1242.22| +2451781|53449|10849|41094|1183116|6738|26313|1|254|1615|10|53.84|75.91|36.43|0.00|364.30|538.40|759.10|25.50|0.00|364.30|389.80|-174.10| +2451781|53449|16853|41094|1183116|6738|26313|1|212|1615|7|55.48|97.64|21.48|69.16|150.36|388.36|683.48|3.24|69.16|81.20|84.44|-307.16| +2451781|53449|13142|41094|1183116|6738|26313|1|185|1615|79|62.09|84.44|67.55|0.00|5336.45|4905.11|6670.76|160.09|0.00|5336.45|5496.54|431.34| +2451781|53449|302||||26313|||1615|72||84.67|||3291.84|4064.40|6096.24|||2271.37|2384.93|| +2451781|53449|4454|41094|1183116|6738|26313|1|96|1615|79|5.05|6.00|3.90|0.00|308.10|398.95|474.00|9.24|0.00|308.10|317.34|-90.85| +2451781|53449|7045|41094|1183116|6738|26313|1|91|1615|77|77.12|85.60|52.21|0.00|4020.17|5938.24|6591.20|40.20|0.00|4020.17|4060.37|-1918.07| +2451781|53449|12272|41094|1183116|6738|26313|1|299|1615|82|57.99|105.54|83.37|0.00|6836.34|4755.18|8654.28|68.36|0.00|6836.34|6904.70|2081.16| +2451512|39339|6991|14798|1106917|1883|449|2|167|1616|45|82.48|129.49|49.20|0.00|2214.00|3711.60|5827.05|177.12|0.00|2214.00|2391.12|-1497.60| +2451512|39339|2318|14798|1106917|1883|449|2|57|1616|12|7.13|13.83|4.70|0.00|56.40|85.56|165.96|0.56|0.00|56.40|56.96|-29.16| +2451512|39339|1328|14798|1106917|1883|449|2|14|1616|76|46.91|65.67|36.11|0.00|2744.36|3565.16|4990.92|109.77|0.00|2744.36|2854.13|-820.80| +2451512|39339|7876|14798|1106917|1883|449|2|217|1616|28|55.76|59.10|45.50|0.00|1274.00|1561.28|1654.80|0.00|0.00|1274.00|1274.00|-287.28| +2451512|39339|7159|14798|1106917|1883|449|2|54|1616|70|48.13|71.71|32.26|0.00|2258.20|3369.10|5019.70|67.74|0.00|2258.20|2325.94|-1110.90| +2451512|39339|14012|14798|1106917|1883|449|2|59|1616|12|23.50|29.14|9.90|30.88|118.80|282.00|349.68|7.03|30.88|87.92|94.95|-194.08| +2451512|39339|13507||1106917|1883|||252|1616||74.50|116.22|||6557.85|4246.50|6624.54|||6557.85|6951.32|| +|39339|7016|14798||1883|449||136|1616||22.15|38.98|26.50|0.00|1139.50|||22.79|0.00||1162.29|187.05| +2451512|39339|2389|14798|1106917|1883|449|2|189|1616|57|57.34|67.08|59.70|680.58|3402.90|3268.38|3823.56|81.66|680.58|2722.32|2803.98|-546.06| +2451512|39339|1477|14798|1106917|1883|449|2|120|1616|5|69.17|129.34|16.81|0.00|84.05|345.85|646.70|3.36|0.00|84.05|87.41|-261.80| +2451512|39339|11432|14798|1106917|1883|449|2|36|1616|21|58.38|93.99|80.83|0.00|1697.43|1225.98|1973.79|101.84|0.00|1697.43|1799.27|471.45| +2451512|39339|10567|14798|1106917|1883|449|2|200|1616|48|8.24|10.87|1.84|0.00|88.32|395.52|521.76|1.76|0.00|88.32|90.08|-307.20| +2451512|39339|6740|14798|1106917|1883|449|2|293|1616|91|29.10|36.37|27.64|0.00|2515.24|2648.10|3309.67|226.37|0.00|2515.24|2741.61|-132.86| +2451666|70138|2024|90597|893583|216|402|8|187|1617|19|16.61|24.58|11.79|82.88|224.01|315.59|467.02|12.70|82.88|141.13|153.83|-174.46| +2451666|70138|16015|90597|893583|216|402|8|30|1617|30|25.68|35.69|2.49|18.67|74.70|770.40|1070.70|4.48|18.67|56.03|60.51|-714.37| +2451666|70138|17521|90597|893583|216|402|8|127|1617|96|11.98|16.77|9.55|0.00|916.80|1150.08|1609.92|73.34|0.00|916.80|990.14|-233.28| +2451666|70138|9079||893583|216||8||1617||||22.42|0.00|1950.54|4240.38|7504.62||0.00|||| +2451666|70138|6089|90597|893583|216|402|8|74|1617|55|27.78|37.22|30.52|0.00|1678.60|1527.90|2047.10|134.28|0.00|1678.60|1812.88|150.70| +2451666|70138|16841|90597|893583|216|402|8|221|1617|65|67.72|113.76|3.41|217.21|221.65|4401.80|7394.40|0.22|217.21|4.44|4.66|-4397.36| +2451666|70138|12689|90597|893583|216|402|8|252|1617|87|32.57|54.06|32.97|0.00|2868.39|2833.59|4703.22|57.36|0.00|2868.39|2925.75|34.80| +2451666|70138|4825|90597|893583|216|402|8|156|1617|100|53.48|100.00|71.00|0.00|7100.00|5348.00|10000.00|639.00|0.00|7100.00|7739.00|1752.00| +2451130|32287|14542|2832|113123|6967|25258|7|4|1618|54|40.79|62.40|23.08|0.00|1246.32|2202.66|3369.60|62.31|0.00|1246.32|1308.63|-956.34| +2451130|32287|4390|2832|113123|6967|25258|7|71|1618|63|76.58|106.44|13.83|17.42|871.29|4824.54|6705.72|25.61|17.42|853.87|879.48|-3970.67| +2451130|32287|14876|2832|113123|6967|25258|7|255|1618|83|30.68|32.82|21.66|0.00|1797.78|2546.44|2724.06|125.84|0.00|1797.78|1923.62|-748.66| +2451130|32287|5018|2832|113123|6967|25258|7|280|1618|38|20.50|40.79|6.52|175.90|247.76|779.00|1550.02|2.15|175.90|71.86|74.01|-707.14| +2451130|32287|2432|2832|113123|6967|25258|7|16|1618|30|40.78|40.78|2.03|33.49|60.90|1223.40|1223.40|1.64|33.49|27.41|29.05|-1195.99| +2451130|32287|7285|2832||6967|||279|1618|||34.21|17.44|||||69.58|||1063.66|| +2451130|32287|196|2832||6967|||110|1618|||64.59|||788.66|||63.09||788.66||-2441.18| +2451130|32287|9544|2832|113123|6967|25258|7|210|1618|63|87.41|128.49|95.08|0.00|5990.04|5506.83|8094.87|119.80|0.00|5990.04|6109.84|483.21| +2451130|32287|12625|2832|113123|6967|25258|7|42|1618|100|79.26|116.51|74.56|2013.12|7456.00|7926.00|11651.00|381.00|2013.12|5442.88|5823.88|-2483.12| +2451130|32287|3091|2832|113123|6967|25258|7|83|1618|16|18.02|21.62|7.78|17.42|124.48|288.32|345.92|7.49|17.42|107.06|114.55|-181.26| +|32287|422||113123|6967|25258|7||1618|29|||||2039.86|2747.75|4533.57|73.43||||| +2451130|32287|5029|2832|113123|6967|25258|7|168|1618|18|97.86|175.16|110.35|0.00|1986.30|1761.48|3152.88|99.31|0.00|1986.30|2085.61|224.82| +2451130|32287|17704|2832|113123|6967|25258|7|36|1618|33|69.99|116.18|37.17|0.00|1226.61|2309.67|3833.94|61.33|0.00|1226.61|1287.94|-1083.06| +2451130|32287|11234|2832|113123|6967|25258|7|125|1618|18|74.01|91.03|82.83|0.00|1490.94|1332.18|1638.54|74.54|0.00|1490.94|1565.48|158.76| +2451415|56294|320|89208|719914|2173|30879|7|151|1619|75|21.51|27.31|0.00|0.00|0.00|1613.25|2048.25|0.00|0.00|0.00|0.00|-1613.25| +2451415|56294|14900|89208|719914|2173|30879|7|205|1619|5|40.04|42.04|21.02|0.00|105.10|200.20|210.20|1.05|0.00|105.10|106.15|-95.10| +2451415|56294|17830|89208|719914|2173|30879|7|250|1619|64|56.12|82.49|37.94|0.00|2428.16|3591.68|5279.36|169.97|0.00|2428.16|2598.13|-1163.52| +2451415|56294|17144|89208|719914|2173|30879|7|231|1619|89|78.50|139.73|131.34|7598.01|11689.26|6986.50|12435.97|40.91|7598.01|4091.25|4132.16|-2895.25| +2451415|56294|16036|89208|719914|2173|30879|7|184|1619|44|55.63|63.41|21.55|0.00|948.20|2447.72|2790.04|18.96|0.00|948.20|967.16|-1499.52| +2451415|56294|12265|89208|719914|2173|30879|7|191|1619|33|30.71|56.81|12.49|0.00|412.17|1013.43|1874.73|4.12|0.00|412.17|416.29|-601.26| +2451415|56294|7309|89208|719914|2173|30879|7|99|1619|89|7.76|15.20|10.94|0.00|973.66|690.64|1352.80|68.15|0.00|973.66|1041.81|283.02| +2451415|56294|12409|89208|719914|2173|30879|7|23|1619|12|29.34|54.57|18.55|0.00|222.60|352.08|654.84|20.03|0.00|222.60|242.63|-129.48| +2451415|56294|5569|89208|719914|2173|30879|7|192|1619|96|17.52|21.02|20.38|1486.92|1956.48|1681.92|2017.92|14.08|1486.92|469.56|483.64|-1212.36| +2451415|56294|9559|89208|719914|2173|30879|7|4|1619|69|32.92|51.35|1.54|0.00|106.26|2271.48|3543.15|7.43|0.00|106.26|113.69|-2165.22| +2451415|56294|16789|89208|719914|2173|30879|7|46|1619|74|86.70|157.79|107.29|317.57|7939.46|6415.80|11676.46|0.00|317.57|7621.89|7621.89|1206.09| +2451415|56294|13112|89208|719914|2173|30879|7|159|1619|83|71.23|95.44|41.03|0.00|3405.49|5912.09|7921.52|238.38|0.00|3405.49|3643.87|-2506.60| +2451415|56294|133|89208|719914|2173|30879|7|72|1619|19|85.20|100.53|48.25|889.24|916.75|1618.80|1910.07|1.37|889.24|27.51|28.88|-1591.29| +2451415|56294|16393|89208|719914|2173|30879|7|10|1619|32|37.99|39.88|31.90|0.00|1020.80|1215.68|1276.16|0.00|0.00|1020.80|1020.80|-194.88| +2451773|33741|7358|71934|1244470|559|25895|10|10|1620|32|88.80|104.78|38.76|0.00|1240.32|2841.60|3352.96|12.40|0.00|1240.32|1252.72|-1601.28| +2451773|33741|15584|71934|1244470|559|25895|10|49|1620|56|1.67|1.90|1.65|0.00|92.40|93.52|106.40|2.77|0.00|92.40|95.17|-1.12| +2451773|33741|6397|71934|1244470|559|25895|10|27|1620|90|30.30|45.14|29.79|0.00|2681.10|2727.00|4062.60|26.81|0.00|2681.10|2707.91|-45.90| +2451773|33741|17305|71934|1244470|559|25895|10|68|1620|71|19.75|29.42|22.06|720.47|1566.26|1402.25|2088.82|25.37|720.47|845.79|871.16|-556.46| +2451773|33741|7315|71934|1244470|559|25895|10|157|1620|51|59.61|117.43|4.69|35.87|239.19|3040.11|5988.93|12.19|35.87|203.32|215.51|-2836.79| +|33741|17543|71934|||25895|10||1620|55|||35.43|0.00|1948.65|||0.00|0.00|1948.65|1948.65|| +2451773|33741|14641|71934|1244470|559|25895|10|9|1620|84|25.67|26.69|25.08|0.00|2106.72|2156.28|2241.96|189.60|0.00|2106.72|2296.32|-49.56| +2451773|33741|5240|71934|1244470|559|25895|10|206|1620|17|7.30|10.65|9.69|0.00|164.73|124.10|181.05|14.82|0.00|164.73|179.55|40.63| +2451773|33741|6259|71934|1244470|559|25895|10|95|1620|14|25.18|38.27|10.33|0.00|144.62|352.52|535.78|4.33|0.00|144.62|148.95|-207.90| +2451773|33741|6983|71934|1244470|559|25895|10|204|1620|21|28.97|34.47|0.00|0.00|0.00|608.37|723.87|0.00|0.00|0.00|0.00|-608.37| +2451773||15494||1244470||25895|||1620||78.41|||||||||6286.32|6726.36|640.80| +2451773|33741|31|71934|1244470|559|25895|10|110|1620|10|46.81|62.25|25.52|0.00|255.20|468.10|622.50|2.55|0.00|255.20|257.75|-212.90| +2451773|33741|14111|71934|1244470|559|25895|10|264|1620|35|6.50|12.93|8.92|0.00|312.20|227.50|452.55|24.97|0.00|312.20|337.17|84.70| +2451773|33741|9805|71934|1244470|559|25895|10|275|1620|72|36.03|59.80|25.71|999.60|1851.12|2594.16|4305.60|17.03|999.60|851.52|868.55|-1742.64| +2451773|33741|10247|71934|1244470|559|25895|10|278|1620|61|60.22|71.66|50.16|0.00|3059.76|3673.42|4371.26|61.19|0.00|3059.76|3120.95|-613.66| +2452627|66964|11262|82431|1609592|4400|38008|2|286|1621|59|41.08|67.37|0.00|0.00|0.00|2423.72|3974.83|0.00|0.00|0.00|0.00|-2423.72| +2452627|66964|11803|82431|1609592|4400|38008|2|240|1621|50|19.06|35.45|22.68|0.00|1134.00|953.00|1772.50|79.38|0.00|1134.00|1213.38|181.00| +2452627|66964|2922|82431|1609592|4400|38008|2|147|1621|3|74.31|89.17|81.14|0.00|243.42|222.93|267.51|4.86|0.00|243.42|248.28|20.49| +2452627|66964|4911|82431|1609592|4400|38008|2|147|1621|63|70.92|80.84|37.99|0.00|2393.37|4467.96|5092.92|95.73|0.00|2393.37|2489.10|-2074.59| +2452627|66964|11491|82431|1609592|4400|38008|2|196|1621|55|67.39|68.06|21.77|0.00|1197.35|3706.45|3743.30|83.81|0.00|1197.35|1281.16|-2509.10| +2452627|66964|10170|82431|1609592|4400|38008|2|77|1621|33|7.76|15.36|6.75|135.87|222.75|256.08|506.88|6.08|135.87|86.88|92.96|-169.20| +2452627|66964|10723|82431|1609592|4400|38008|2|78|1621|99|4.02|5.90|2.41|0.00|238.59|397.98|584.10|11.92|0.00|238.59|250.51|-159.39| +2452627|66964|4017|82431|1609592|4400|38008|2|245|1621|45|96.81|149.08|47.70|0.00|2146.50|4356.45|6708.60|150.25|0.00|2146.50|2296.75|-2209.95| +2452627|66964|3919|82431|1609592|4400|38008|2|186|1621|75|88.66|142.74|119.90|0.00|8992.50|6649.50|10705.50|269.77|0.00|8992.50|9262.27|2343.00| +2452627|66964|9483|82431|1609592|4400|38008|2|264|1621|38|66.73|70.06|39.23|0.00|1490.74|2535.74|2662.28|134.16|0.00|1490.74|1624.90|-1045.00| +2452627|66964|9435|82431|1609592|4400|38008|2|226|1621|12|86.69|124.83|29.95|0.00|359.40|1040.28|1497.96|10.78|0.00|359.40|370.18|-680.88| +2452627|66964|11331|82431|1609592|4400|38008|2|110|1621|9|26.97|48.00|10.56|0.00|95.04|242.73|432.00|0.00|0.00|95.04|95.04|-147.69| +2452170|55016|11497|98224|1324295|7166|27590|8|274|1622|36|58.09|66.22|20.52|0.00|738.72|2091.24|2383.92|29.54|0.00|738.72|768.26|-1352.52| +2452170|55016|10193|98224|1324295|7166|27590|8|110|1622|61|25.42|34.31|13.72|0.00|836.92|1550.62|2092.91|50.21|0.00|836.92|887.13|-713.70| +2452170|55016|10399|98224|1324295|7166|27590|8|227|1622|92|53.34|57.60|5.18|0.00|476.56|4907.28|5299.20|42.89|0.00|476.56|519.45|-4430.72| +2452170|55016|10135|98224|1324295|7166|27590|8|291|1622|1|51.24|75.32|15.06|0.00|15.06|51.24|75.32|0.15|0.00|15.06|15.21|-36.18| +2452170|55016|3745|98224|1324295|7166|27590|8|129|1622|35|58.20|85.55|43.63|0.00|1527.05|2037.00|2994.25|106.89|0.00|1527.05|1633.94|-509.95| +2452170|55016|7471|98224|1324295|7166|27590|8|187|1622|52|67.07|113.34|90.67|0.00|4714.84|3487.64|5893.68|188.59|0.00|4714.84|4903.43|1227.20| +2452170|55016|14129|98224|1324295|7166|27590|8|207|1622|58|47.56|84.18|69.02|0.00|4003.16|2758.48|4882.44|160.12|0.00|4003.16|4163.28|1244.68| +2452170|55016|15043|98224|1324295|7166|27590|8|201|1622|30|42.18|44.71|15.64|0.00|469.20|1265.40|1341.30|0.00|0.00|469.20|469.20|-796.20| +2452170|55016|16891|98224|1324295|7166|27590|8|31|1622|78|29.33|55.43|16.07|0.00|1253.46|2287.74|4323.54|100.27|0.00|1253.46|1353.73|-1034.28| +2452170|55016|3675|98224|1324295|7166|27590|8|170|1622|43|27.80|31.41|21.98|0.00|945.14|1195.40|1350.63|47.25|0.00|945.14|992.39|-250.26| +2452170|55016|5525|98224|1324295|7166|27590|8|122|1622|1|12.00|18.00|14.94|0.00|14.94|12.00|18.00|0.74|0.00|14.94|15.68|2.94| +2452170||6201|||||8|2|1622|38|53.65||77.96|0.00|2962.48||3445.08|207.37|0.00||3169.85|| +2452170|55016|2685|98224|1324295|7166|27590|8|228|1622|79|44.82|62.29|54.19|0.00|4281.01|3540.78|4920.91|299.67|0.00|4281.01|4580.68|740.23| +2452170|55016|2291|98224|1324295|7166|27590|8|128|1622|51|64.99|84.48|76.03|0.00|3877.53|3314.49|4308.48|155.10|0.00|3877.53|4032.63|563.04| +||7161||||27590|||1622|64|||50.82|2927.23|3252.48||3918.72||2927.23||331.75|-2093.95| +2451783|70273|15739|33223|1529255|4487|30919|4|47|1623|15|29.67|38.57|37.79|198.39|566.85|445.05|578.55|22.10|198.39|368.46|390.56|-76.59| +2451783|70273|16141|33223|1529255|4487|30919|4|22|1623|63|84.35|146.76|95.39|0.00|6009.57|5314.05|9245.88|540.86|0.00|6009.57|6550.43|695.52| +2451783|70273|12089|33223|||30919|||1623|32||156.99||1366.31||2715.52|||1366.31|642.97|655.82|-2072.55| +2451783|70273|11413|33223|1529255|4487|30919|4|225|1623|24|98.57|128.14|7.68|0.00|184.32|2365.68|3075.36|5.52|0.00|184.32|189.84|-2181.36| +2451783|70273|5540|33223|1529255|4487|30919|4|224|1623|70|9.48|9.57|1.05|0.00|73.50|663.60|669.90|5.88|0.00|73.50|79.38|-590.10| +2451783|70273|13109|33223|1529255|4487|30919|4|297|1623|89|96.95|96.95|31.02|0.00|2760.78|8628.55|8628.55|55.21|0.00|2760.78|2815.99|-5867.77| +2451783|70273|9863|33223|1529255|4487|30919|4|250|1623|100|43.27|64.90|8.43|0.00|843.00|4327.00|6490.00|75.87|0.00|843.00|918.87|-3484.00| +2451783|70273|16999|33223|1529255|4487|30919|4|226|1623|4|93.86|161.43|156.58|0.00|626.32|375.44|645.72|25.05|0.00|626.32|651.37|250.88| +|70273|10538|33223|1529255||30919|4||1623||23.41||||42.21|163.87|||||44.32|-121.66| +2451783|70273|449|33223|1529255|4487|30919|4|218|1623|74|67.96|112.81|111.68|0.00|8264.32|5029.04|8347.94|82.64|0.00|8264.32|8346.96|3235.28| +2451783|70273|15559|33223|1529255|4487|30919|4|246|1623|100|41.95|75.51|56.63|0.00|5663.00|4195.00|7551.00|396.41|0.00|5663.00|6059.41|1468.00| +2451512|52658|8335|24249|937045|1041|17884|10|134|1624|17|76.45|135.31|124.48|0.00|2116.16|1299.65|2300.27|63.48|0.00|2116.16|2179.64|816.51| +2451512|52658|9397|24249|937045|1041|17884|10|84|1624|81|64.68|75.67|61.29|0.00|4964.49|5239.08|6129.27|198.57|0.00|4964.49|5163.06|-274.59| +2451512|52658|11914|24249|937045|1041|17884|10|165|1624|62|31.59|45.80|22.90|979.66|1419.80|1958.58|2839.60|13.20|979.66|440.14|453.34|-1518.44| +2451512|52658|10651|24249|937045|1041|17884|10|139|1624|96|48.95|74.89|35.94|0.00|3450.24|4699.20|7189.44|276.01|0.00|3450.24|3726.25|-1248.96| +2451512|52658|3133|24249|937045|1041|17884|10|259|1624|30|16.57|29.99|18.59|0.00|557.70|497.10|899.70|5.57|0.00|557.70|563.27|60.60| +2451512|52658|7087|24249|937045|1041|17884|10|112|1624|73|97.49|137.46|101.72|0.00|7425.56|7116.77|10034.58|0.00|0.00|7425.56|7425.56|308.79| +2451512|52658|9346|24249|937045|1041|17884|10|199|1624|37|81.89|125.29|103.99|0.00|3847.63|3029.93|4635.73|38.47|0.00|3847.63|3886.10|817.70| +2451512|52658|6772|24249|937045|1041|17884|10|240|1624|85|62.08|89.39|24.13|0.00|2051.05|5276.80|7598.15|102.55|0.00|2051.05|2153.60|-3225.75| +2451512|52658|9094|24249|937045|1041|17884|10|10|1624|8|77.99|101.38|25.34|0.00|202.72|623.92|811.04|4.05|0.00|202.72|206.77|-421.20| +2451512|52658|1094|24249|937045|1041|17884|10|63|1624|52|29.13|38.45|28.06|0.00|1459.12|1514.76|1999.40|14.59|0.00|1459.12|1473.71|-55.64| +2451114|53188|11528|68242|345351|1307|28463|4|299|1625|56|29.65|48.03|41.78|0.00|2339.68|1660.40|2689.68|70.19|0.00|2339.68|2409.87|679.28| +2451114|53188|1604|68242|345351|1307|28463|4|276|1625|25|11.55|18.94|5.11|0.00|127.75|288.75|473.50|0.00|0.00|127.75|127.75|-161.00| +2451114|53188|12307|68242|345351|1307|28463|4|257|1625|58|81.03|123.16|27.09|0.00|1571.22|4699.74|7143.28|141.40|0.00|1571.22|1712.62|-3128.52| +2451114|53188|7238|68242|345351|1307|28463|4|126|1625|70|51.97|55.60|17.23|0.00|1206.10|3637.90|3892.00|0.00|0.00|1206.10|1206.10|-2431.80| +2451114|53188|11185|68242|345351|1307|28463|4|151|1625|31|82.34|121.03|21.78|243.06|675.18|2552.54|3751.93|25.92|243.06|432.12|458.04|-2120.42| +2451114|53188|10616|68242|345351|1307|28463|4|47|1625|34|71.41|111.39|94.68|2961.59|3219.12|2427.94|3787.26|20.60|2961.59|257.53|278.13|-2170.41| +2451114|53188|2510|68242|345351|1307|28463|4|69|1625|86|57.56|111.66|94.91|1958.94|8162.26|4950.16|9602.76|434.23|1958.94|6203.32|6637.55|1253.16| +|53188|17858|68242|345351||||262|1625||13.77|16.52|15.36|||922.59||||1029.12||106.53| +2451114|53188|8728|68242|345351|1307|28463|4|289|1625|31|15.75|23.31|4.89|27.28|151.59|488.25|722.61|3.72|27.28|124.31|128.03|-363.94| +2451114|53188|14156|68242|345351|1307|28463|4|197|1625|52|76.16|143.18|91.63|0.00|4764.76|3960.32|7445.36|142.94|0.00|4764.76|4907.70|804.44| +2451114|53188|8678|68242|345351|1307|28463|4|106|1625|85|93.00|107.88|62.57|0.00|5318.45|7905.00|9169.80|372.29|0.00|5318.45|5690.74|-2586.55| +2451114|53188|2140|68242|345351|1307|28463|4|255|1625|15|57.64|105.48|74.89|550.44|1123.35|864.60|1582.20|11.45|550.44|572.91|584.36|-291.69| +2451114|53188|17234|68242|345351|1307|28463|4|178|1625|49|49.92|66.39|52.44|642.39|2569.56|2446.08|3253.11|154.17|642.39|1927.17|2081.34|-518.91| +2451114|53188|6334|68242|345351|1307|28463|4|248|1625|56|59.45|108.19|94.12|0.00|5270.72|3329.20|6058.64|105.41|0.00|5270.72|5376.13|1941.52| +2452227|62411|9053|28277|1395412|2900|788|7|300|1626|1|73.26|111.35|6.68|0.00|6.68|73.26|111.35|0.53|0.00|6.68|7.21|-66.58| +2452227|62411|14787|28277|1395412|2900|788|7|286|1626|71|35.26|55.35|7.19|0.00|510.49|2503.46|3929.85|5.10|0.00|510.49|515.59|-1992.97| +2452227|62411|5953|28277|1395412|2900|788|7|151|1626|72|29.88|44.52|3.56|0.00|256.32|2151.36|3205.44|5.12|0.00|256.32|261.44|-1895.04| +2452227|62411|9085|28277|1395412|2900|788|7|193|1626|15|48.65|52.05|37.47|0.00|562.05|729.75|780.75|5.62|0.00|562.05|567.67|-167.70| +2452227|62411|4797|28277|1395412|2900|788|7|41|1626|91|61.14|64.19|18.61|0.00|1693.51|5563.74|5841.29|152.41|0.00|1693.51|1845.92|-3870.23| +2452227|62411|1249|28277|1395412|2900|788|7|168|1626|58|92.39|129.34|67.25|0.00|3900.50|5358.62|7501.72|156.02|0.00|3900.50|4056.52|-1458.12| +2452227|62411|5561|28277|1395412|2900|788|7|200|1626|68|51.02|96.93|91.11|0.00|6195.48|3469.36|6591.24|433.68|0.00|6195.48|6629.16|2726.12| +2452227|62411|845|28277|1395412|2900|788|7|210|1626|70|95.38|127.80|79.23|0.00|5546.10|6676.60|8946.00|0.00|0.00|5546.10|5546.10|-1130.50| +2450938|36394|17102|78473|460089|2505|3321|8|243|1627|39|33.97|34.30|18.86|0.00|735.54|1324.83|1337.70|29.42|0.00|735.54|764.96|-589.29| +2450938|36394|14680|78473|460089|2505|3321|8|124|1627|29|91.77|171.60|39.46|0.00|1144.34|2661.33|4976.40|68.66|0.00|1144.34|1213.00|-1516.99| +2450938|36394|8792|78473|460089|2505|3321|8|51|1627|91|79.33|137.24|2.74|0.00|249.34|7219.03|12488.84|12.46|0.00|249.34|261.80|-6969.69| +2450938|36394|12728|78473|460089|2505|3321|8|277|1627|40|2.56|2.94|2.73|0.00|109.20|102.40|117.60|8.73|0.00|109.20|117.93|6.80| +2450938|36394|6854|78473|460089|2505|3321|8|246|1627|29|75.18|75.93|1.51|0.00|43.79|2180.22|2201.97|0.00|0.00|43.79|43.79|-2136.43| +2450938|36394|15577|78473|460089|2505|3321|8|25|1627|67|32.65|55.17|30.34|0.00|2032.78|2187.55|3696.39|142.29|0.00|2032.78|2175.07|-154.77| +2450938|36394|16280|78473|460089|2505|3321|8|285|1627|6|40.44|71.98|58.30|0.00|349.80|242.64|431.88|0.00|0.00|349.80|349.80|107.16| +2450938|36394|5731|78473|460089|2505|3321|8|120|1627|55|50.25|53.76|32.25|0.00|1773.75|2763.75|2956.80|141.90|0.00|1773.75|1915.65|-990.00| +2450938|36394|16642|78473|460089|2505|3321|8|238|1627|42|62.24|81.53|35.87|0.00|1506.54|2614.08|3424.26|120.52|0.00|1506.54|1627.06|-1107.54| +2450938|36394|3500|78473|460089|2505|3321|8|277|1627|80|17.23|32.39|29.47|0.00|2357.60|1378.40|2591.20|212.18|0.00|2357.60|2569.78|979.20| +2450938|36394|10580|78473|460089|2505|3321|8|123|1627|40|38.47|59.24|46.79|1066.81|1871.60|1538.80|2369.60|8.04|1066.81|804.79|812.83|-734.01| +2450938|36394|17080|78473|460089|2505|3321|8|140|1627|62|61.99|107.86|88.44|0.00|5483.28|3843.38|6687.32|274.16|0.00|5483.28|5757.44|1639.90| +2451642|49504|15791|89446|953095|3944|23193|8|223|1628|10|9.80|18.91|18.72|166.60|187.20|98.00|189.10|0.00|166.60|20.60|20.60|-77.40| +||4601||||23193|8||1628|||||0.00||3334.38|6635.10||0.00|4909.77|5007.96|1575.39| +2451642|49504|14978|89446|953095|3944|23193|8|295|1628|73|13.04|13.04|1.56|34.16|113.88|951.92|951.92|5.58|34.16|79.72|85.30|-872.20| +2451642|49504|127|89446|953095|3944|23193|8|38|1628|12|58.54|80.78|33.11|294.01|397.32|702.48|969.36|6.19|294.01|103.31|109.50|-599.17| +2451642|49504|17725|89446|953095|3944|23193|8|47|1628|33|34.91|55.50|4.99|0.00|164.67|1152.03|1831.50|14.82|0.00|164.67|179.49|-987.36| +2451642|49504|4256|89446|953095|3944|23193|8|185|1628|79|77.91|119.98|101.98|0.00|8056.42|6154.89|9478.42|563.94|0.00|8056.42|8620.36|1901.53| +||8785|||3944|||299|1628|40|1.95||||77.20|78.00|88.00|||77.20|82.60|-0.80| +2451642|49504|1207|89446|953095|3944|23193|8|223|1628|61|24.95|47.90|46.46|0.00|2834.06|1521.95|2921.90|85.02|0.00|2834.06|2919.08|1312.11| +2451642|49504|9698|89446|953095|3944|23193|8|126|1628|34|8.10|11.74|11.27|0.00|383.18|275.40|399.16|22.99|0.00|383.18|406.17|107.78| +2451642|49504|7055|89446|953095|3944|23193|8|265|1628|44|4.86|6.31|3.28|0.00|144.32|213.84|277.64|2.88|0.00|144.32|147.20|-69.52| +2451642|49504|9146|89446|953095|3944|23193|8|69|1628|47|75.89|102.45|87.08|0.00|4092.76|3566.83|4815.15|245.56|0.00|4092.76|4338.32|525.93| +|49504|11279|89446||3944|23193|8|211|1628||||3.25|0.00|269.75||273.07|8.09|0.00|269.75||121.18| +2451642|49504|7247|89446|953095|3944|23193|8|293|1628|9|6.70|10.85|4.77|0.00|42.93|60.30|97.65|0.00|0.00|42.93|42.93|-17.37| +2452014|53544|7569|76549|1772864|5179|22694|4|55|1629|55|14.43|27.27|1.90|94.05|104.50|793.65|1499.85|0.94|94.05|10.45|11.39|-783.20| +2452014|53544|17245|76549|1772864|5179|22694|4|288|1629|43|77.56|141.15|32.46|0.00|1395.78|3335.08|6069.45|97.70|0.00|1395.78|1493.48|-1939.30| +2452014|53544|6305|76549|1772864|5179|22694|4|264|1629|83|4.65|9.02|6.76|0.00|561.08|385.95|748.66|28.05|0.00|561.08|589.13|175.13| +2452014|53544|14381|76549|1772864|5179|22694|4|187|1629|92|39.24|57.68|25.37|0.00|2334.04|3610.08|5306.56|186.72|0.00|2334.04|2520.76|-1276.04| +2452014|53544|3727|76549|1772864|5179|22694|4|188|1629|68|96.76|128.69|16.72|943.67|1136.96|6579.68|8750.92|13.53|943.67|193.29|206.82|-6386.39| +2452014|53544|6947|76549|1772864|5179|22694|4|246|1629|22|89.11|135.44|101.58|0.00|2234.76|1960.42|2979.68|89.39|0.00|2234.76|2324.15|274.34| +2452014|53544|15099|76549|1772864|5179|22694|4|249|1629|17|83.86|124.95|122.45|0.00|2081.65|1425.62|2124.15|83.26|0.00|2081.65|2164.91|656.03| +2452014|53544|14133|76549|1772864|5179|22694|4|291|1629|5|7.74|9.28|5.66|0.00|28.30|38.70|46.40|0.28|0.00|28.30|28.58|-10.40| +2452014|53544|8949|76549|1772864|5179|22694|4|143|1629|55|94.23|130.03|84.51|0.00|4648.05|5182.65|7151.65|325.36|0.00|4648.05|4973.41|-534.60| +2452614|45148|16959|65221|741833|398|21363|10|240|1630|80|34.26|55.15|10.47|0.00|837.60|2740.80|4412.00|67.00|0.00|837.60|904.60|-1903.20| +2452614|45148|522|65221|741833|398|21363|10|226|1630|8|87.93|169.70|59.39|0.00|475.12|703.44|1357.60|42.76|0.00|475.12|517.88|-228.32| +2452614|45148|5760|65221|741833|398|21363|10|297|1630|76|46.94|78.85|70.17|319.97|5332.92|3567.44|5992.60|350.90|319.97|5012.95|5363.85|1445.51| +2452614|45148|8913|65221|741833|398|21363|10|232|1630|38|84.84|158.65|44.42|0.00|1687.96|3223.92|6028.70|84.39|0.00|1687.96|1772.35|-1535.96| +2452614|45148|4368|65221|741833|398|21363|10|102|1630|65|87.94|148.61|66.87|0.00|4346.55|5716.10|9659.65|347.72|0.00|4346.55|4694.27|-1369.55| +2452614||9745||741833|398|21363|||1630|10|||||183.10||763.00||||190.42|| +2452614|45148|5095|65221|741833|398|21363|10|68|1630|78|5.09|7.73|5.95|0.00|464.10|397.02|602.94|9.28|0.00|464.10|473.38|67.08| +2452614||1926||||21363|10|197|1630|69||||2732.40||1854.03|3429.30|7.96|2732.40|113.85|121.81|-1740.18| +2452614|45148|17923|65221|741833|398|21363|10|232|1630|62|59.01|75.53|37.00|711.14|2294.00|3658.62|4682.86|63.31|711.14|1582.86|1646.17|-2075.76| +2452614|45148|6067|65221|741833|398|21363|10|36|1630|19|31.75|53.02|6.89|0.00|130.91|603.25|1007.38|9.16|0.00|130.91|140.07|-472.34| +2452614|45148|17196|65221|741833|398|21363|10|9|1630|57|59.87|68.25|64.83|0.00|3695.31|3412.59|3890.25|36.95|0.00|3695.31|3732.26|282.72| +2452614|45148|8257|65221|741833|398|21363|10|262|1630|29|14.31|15.74|1.88|0.00|54.52|414.99|456.46|2.72|0.00|54.52|57.24|-360.47| +2452614|45148|5934|65221|741833|398|21363|10|69|1630|92|89.04|112.19|44.87|0.00|4128.04|8191.68|10321.48|82.56|0.00|4128.04|4210.60|-4063.64| +2452614|45148|13585|65221|741833|398|21363|10|76|1630|2|75.35|123.57|79.08|0.00|158.16|150.70|247.14|12.65|0.00|158.16|170.81|7.46| +2452614|45148|9667|65221|741833|398|21363|10|70|1630|10|58.32|71.73|32.99|0.00|329.90|583.20|717.30|16.49|0.00|329.90|346.39|-253.30| +2451071|60425|400|99156|780825|1694|8516|1|215|1631|78|91.44|105.15|86.22|0.00|6725.16|7132.32|8201.70|134.50|0.00|6725.16|6859.66|-407.16| +2451071|60425|11425|99156|780825|1694|8516|1|27|1631|66|2.68|3.18|1.81|0.00|119.46|176.88|209.88|10.75|0.00|119.46|130.21|-57.42| +2451071|60425|13532|99156|780825|1694|8516|1|91|1631|27|13.15|17.22|9.29|0.00|250.83|355.05|464.94|0.00|0.00|250.83|250.83|-104.22| +2451071|60425|16528|99156|780825|1694|8516|1|68|1631|15|63.21|123.25|45.60|0.00|684.00|948.15|1848.75|0.00|0.00|684.00|684.00|-264.15| +2451071|60425|16222|99156|780825|1694|8516|1|218|1631|99|63.74|122.38|110.14|6978.47|10903.86|6310.26|12115.62|353.28|6978.47|3925.39|4278.67|-2384.87| +2451071|60425|13327|99156|780825|1694|8516|1|2|1631|73|11.64|16.99|13.59|0.00|992.07|849.72|1240.27|39.68|0.00|992.07|1031.75|142.35| +2451071|60425|13591|99156|780825|1694|8516|1|3|1631|75|8.86|8.86|7.08|531.00|531.00|664.50|664.50|0.00|531.00|0.00|0.00|-664.50| +2451071|60425|256|99156|780825|1694|8516|1|175|1631|47|78.34|87.74|81.59|0.00|3834.73|3681.98|4123.78|230.08|0.00|3834.73|4064.81|152.75| +2451500||2449|77780||5671|7840|7||1632||84.25|||||3201.50||41.68||1389.66||-1811.84| +2451500|67658|10240|77780|502749|5671|7840|7|144|1632|99|56.23|68.60|61.74|0.00|6112.26|5566.77|6791.40|61.12|0.00|6112.26|6173.38|545.49| +2451500|67658|5698|77780|502749|5671|7840|7|39|1632|44|93.68|120.84|9.66|0.00|425.04|4121.92|5316.96|34.00|0.00|425.04|459.04|-3696.88| +2451500|67658|16630|77780|502749|5671|7840|7|39|1632|69|99.31|138.04|44.17|0.00|3047.73|6852.39|9524.76|0.00|0.00|3047.73|3047.73|-3804.66| +2451500|67658|10195|77780|502749|5671|7840|7|203|1632|69|29.75|47.30|27.43|1419.50|1892.67|2052.75|3263.70|33.12|1419.50|473.17|506.29|-1579.58| +2451500|67658|11422|77780|502749|5671|7840|7|50|1632|38|93.07|107.03|94.18|1717.84|3578.84|3536.66|4067.14|18.61|1717.84|1861.00|1879.61|-1675.66| +2451500|67658|11449|77780|502749|5671|7840|7|209|1632|21|81.97|140.98|7.04|66.52|147.84|1721.37|2960.58|4.06|66.52|81.32|85.38|-1640.05| +2451500|67658|8152|77780|502749|5671|7840|7|96|1632|93|25.55|46.50|13.02|0.00|1210.86|2376.15|4324.50|108.97|0.00|1210.86|1319.83|-1165.29| +2451500|67658|517|77780|502749|5671|7840|7|24|1632|16|56.31|79.96|67.16|0.00|1074.56|900.96|1279.36|21.49|0.00|1074.56|1096.05|173.60| +2451500|67658|3220|77780|502749|5671|7840|7|45|1632|25|83.09|108.84|106.66|0.00|2666.50|2077.25|2721.00|26.66|0.00|2666.50|2693.16|589.25| +2451500|67658|17504|77780|502749|5671|7840|7|95|1632|74|97.69|132.85|61.11|0.00|4522.14|7229.06|9830.90|135.66|0.00|4522.14|4657.80|-2706.92| +2451500|67658|12292|77780|502749|5671|7840|7|227|1632|23|46.77|49.10|22.58|0.00|519.34|1075.71|1129.30|41.54|0.00|519.34|560.88|-556.37| +2451500|67658|13705|77780|502749|5671|7840||85|1632||16.35|19.94|3.98||||1435.68|||134.69|144.11|-1042.51| +2451500|67658|10990|77780|502749|5671|7840|7|105|1632|14|47.07|51.30|3.59|0.00|50.26|658.98|718.20|3.51|0.00|50.26|53.77|-608.72| +2451255|64455|2320|63571|1729849|574|14052|7|65|1633|95|43.65|82.93|6.63|459.79|629.85|4146.75|7878.35|6.80|459.79|170.06|176.86|-3976.69| +2451255|64455|1460|63571|1729849|574|14052|7|232|1633|90|15.02|15.17|8.04|0.00|723.60|1351.80|1365.30|28.94|0.00|723.60|752.54|-628.20| +2451255|64455|16216|63571|1729849|574|14052|7|51|1633|30|16.47|18.94|13.25|0.00|397.50|494.10|568.20|3.97|0.00|397.50|401.47|-96.60| +2451255|64455|10675|63571|1729849|574|14052|7|253|1633|60|46.30|86.58|59.74|0.00|3584.40|2778.00|5194.80|35.84|0.00|3584.40|3620.24|806.40| +2451255|64455|2980|63571|1729849|574|14052|7|228|1633|91|90.74|116.14|94.07|0.00|8560.37|8257.34|10568.74|85.60|0.00|8560.37|8645.97|303.03| +2451255||3469||1729849|||7||1633|35|22.02||||822.85|770.70|1371.65|24.68||||52.15| +2451255|64455|14104|63571|1729849|574|14052|7|195|1633|72|92.39|117.33|58.66|0.00|4223.52|6652.08|8447.76|0.00|0.00|4223.52|4223.52|-2428.56| +2451255|64455|4328|63571|1729849|574|14052|7|52|1633|73|39.35|66.89|48.82|0.00|3563.86|2872.55|4882.97|71.27|0.00|3563.86|3635.13|691.31| +2451255|64455|14779|63571|1729849|574|14052|7|240|1633|69|16.30|16.95|3.72|0.00|256.68|1124.70|1169.55|2.56|0.00|256.68|259.24|-868.02| +2451255|64455|5164|63571|1729849|574|14052|7|131|1633|93|71.67|123.98|79.34|0.00|7378.62|6665.31|11530.14|295.14|0.00|7378.62|7673.76|713.31| +2451255|64455|9170|63571|1729849|574|14052|7|23|1633|25|26.39|35.36|7.07|0.00|176.75|659.75|884.00|8.83|0.00|176.75|185.58|-483.00| +2452305|71050|17421|4788|1832082|529|46623|8|167|1634|100|67.47|85.68|13.70|753.50|1370.00|6747.00|8568.00|49.32|753.50|616.50|665.82|-6130.50| +2452305|71050|9438|4788|1832082|529|46623|8|203|1634|82|52.96|86.85|80.77|2649.25|6623.14|4342.72|7121.70|79.47|2649.25|3973.89|4053.36|-368.83| +2452305|71050|7656|4788|1832082|529|46623|8|73|1634|96|61.43|122.86|38.08|0.00|3655.68|5897.28|11794.56|292.45|0.00|3655.68|3948.13|-2241.60| +2452305|71050|12223|4788|1832082|529|46623|8|189|1634|60|11.51|21.86|18.36|55.08|1101.60|690.60|1311.60|73.25|55.08|1046.52|1119.77|355.92| +2452305|71050|15085|4788|1832082|529|46623|8|124|1634|62|20.74|36.91|4.79|0.00|296.98|1285.88|2288.42|20.78|0.00|296.98|317.76|-988.90| +2452305|71050|3741|4788|1832082|529|46623|8|220|1634|76|91.69|172.37|0.00|0.00|0.00|6968.44|13100.12|0.00|0.00|0.00|0.00|-6968.44| +2452305|71050|1363|4788|1832082|529|46623|8|165|1634|4|67.14|128.23|120.53|0.00|482.12|268.56|512.92|28.92|0.00|482.12|511.04|213.56| +2452305|71050|12069|4788|1832082|529|46623|8|105|1634|40|5.03|7.89|6.39|0.00|255.60|201.20|315.60|12.78|0.00|255.60|268.38|54.40| +2452305|71050|13341|4788|1832082|529|46623|8|282|1634|1|2.69|5.35|3.26|0.00|3.26|2.69|5.35|0.19|0.00|3.26|3.45|0.57| +2452305|71050|2931|4788|1832082|529|46623|8|169|1634|65|29.08|49.14|25.06|0.00|1628.90|1890.20|3194.10|48.86|0.00|1628.90|1677.76|-261.30| +2452305|71050|15783|4788|1832082|529|46623|8|298|1634|79|28.65|34.66|34.31|840.25|2710.49|2263.35|2738.14|130.91|840.25|1870.24|2001.15|-393.11| +2452305|71050|21|4788|1832082|529|46623|8|33|1634|8|65.65|120.13|100.90|686.12|807.20|525.20|961.04|1.21|686.12|121.08|122.29|-404.12| +|71050|15627|4788|||46623|8|119|1634|28|64.28|||407.13|||2393.72||407.13|239.11|255.84|-1560.73| +2452305|71050|9459|4788|1832082|529|46623|8|51|1634|95|97.92|142.96|118.65|0.00|11271.75|9302.40|13581.20|338.15|0.00|11271.75|11609.90|1969.35| +2452305|71050|10795|4788|1832082|529|46623|8|239|1634|18|32.20|63.75|3.18|0.00|57.24|579.60|1147.50|2.86|0.00|57.24|60.10|-522.36| +2450927|38302|4087|10238|1073258|5525|8145|1|142|1635|34|66.87|106.99|10.69|0.00|363.46|2273.58|3637.66|18.17|0.00|363.46|381.63|-1910.12| +2450927|38302|926|10238|1073258|5525|8145|1|74|1635|45|30.32|37.29|36.54|0.00|1644.30|1364.40|1678.05|147.98|0.00|1644.30|1792.28|279.90| +2450927|38302|11864|10238|1073258|5525|8145|1|237|1635|53|86.49|99.46|1.98|0.00|104.94|4583.97|5271.38|2.09|0.00|104.94|107.03|-4479.03| +2450927|38302|14738|10238|1073258|5525|8145|1|89|1635|70|46.47|61.34|57.04|0.00|3992.80|3252.90|4293.80|0.00|0.00|3992.80|3992.80|739.90| +||3164||1073258||8145|1|149|1635|37|||30.26|223.92||2170.42||0.00|223.92||895.70|| +2450927|38302|6721|10238|1073258|5525|8145|1|287|1635|2|77.36|97.47|21.44|7.71|42.88|154.72|194.94|3.16|7.71|35.17|38.33|-119.55| +2450927|38302|8881|10238|1073258|5525|8145|1|197|1635|73|61.68|98.68|9.86|0.00|719.78|4502.64|7203.64|14.39|0.00|719.78|734.17|-3782.86| +2450927|38302|2467|10238|1073258|5525|8145|1|75|1635|41|78.56|139.83|69.91|0.00|2866.31|3220.96|5733.03|28.66|0.00|2866.31|2894.97|-354.65| +2450927|38302|1058|10238|1073258|5525|8145|1|147|1635|6|12.71|22.49|16.86|0.00|101.16|76.26|134.94|0.00|0.00|101.16|101.16|24.90| +2450927|38302|9584|10238|1073258|5525|8145|1|186|1635|90|58.38|58.38|44.95|0.00|4045.50|5254.20|5254.20|323.64|0.00|4045.50|4369.14|-1208.70| +2450927|38302|9196|10238|1073258|5525|8145|1|17|1635|64|81.94|101.60|27.43|1386.86|1755.52|5244.16|6502.40|7.37|1386.86|368.66|376.03|-4875.50| +2450927|38302|2032|10238|1073258|5525|8145|1|93|1635|45|39.64|75.71|50.72|0.00|2282.40|1783.80|3406.95|182.59|0.00|2282.40|2464.99|498.60| +2450927|38302|10186|10238|1073258|5525|8145|1|3|1635|18|57.33|71.08|29.85|0.00|537.30|1031.94|1279.44|48.35|0.00|537.30|585.65|-494.64| +2450927|38302|1636|10238|1073258|5525|8145|1|214|1635|25|21.95|29.19|24.51|0.00|612.75|548.75|729.75|49.02|0.00|612.75|661.77|64.00| +||11917|10238|1073258|5525||1||1635||48.36|74.95|||||1948.70|47.53||||| +2450927|38302|15460|10238|1073258|5525|8145|1|240|1635|36|33.96|46.52|44.65|0.00|1607.40|1222.56|1674.72|144.66|0.00|1607.40|1752.06|384.84| +2451879|47414|2593|19122|216408|3294|40133|1|218|1636|23|33.70|48.86|45.43|0.00|1044.89|775.10|1123.78|94.04|0.00|1044.89|1138.93|269.79| +2451879|47414|16715|19122|216408|3294|40133|1|226|1636|6|10.73|18.34|12.10|0.00|72.60|64.38|110.04|5.80|0.00|72.60|78.40|8.22| +2451879|47414|313|19122|216408|3294|40133|1|157|1636|29|4.65|8.50|4.08|0.00|118.32|134.85|246.50|4.73|0.00|118.32|123.05|-16.53| +2451879|47414|17876|19122|216408|3294|40133|1|164|1636|54|74.38|78.09|18.74|0.00|1011.96|4016.52|4216.86|60.71|0.00|1011.96|1072.67|-3004.56| +2451879|47414|8744|19122|216408|3294|40133|1|213|1636|52|56.35|106.50|89.46|0.00|4651.92|2930.20|5538.00|93.03|0.00|4651.92|4744.95|1721.72| +2451879|47414|14545|19122|216408|3294|40133|1|37|1636|14|97.78|104.62|23.01|190.06|322.14|1368.92|1464.68|3.96|190.06|132.08|136.04|-1236.84| +2451879|47414|16034|19122|216408|3294|40133|1|82|1636|25|66.94|80.99|79.37|0.00|1984.25|1673.50|2024.75|119.05|0.00|1984.25|2103.30|310.75| +2451879|47414|4837|19122|216408|3294|40133|1|222|1636|30|53.41|95.60|33.46|0.00|1003.80|1602.30|2868.00|70.26|0.00|1003.80|1074.06|-598.50| +2451879|47414|9752|19122|216408|3294|40133|1|210|1636|73|31.98|48.28|3.37|0.00|246.01|2334.54|3524.44|4.92|0.00|246.01|250.93|-2088.53| +2451879|47414|4220|19122|216408|3294|40133|1|47|1636|55|10.02|11.52|6.91|0.00|380.05|551.10|633.60|30.40|0.00|380.05|410.45|-171.05| +2451879|47414|17261|19122|216408|3294|40133|1|210|1636|72|67.44|121.39|121.39|0.00|8740.08|4855.68|8740.08|262.20|0.00|8740.08|9002.28|3884.40| +2451879|47414|6427|19122|216408|3294|40133|1|139|1636|63|3.87|4.48|1.65|28.06|103.95|243.81|282.24|1.51|28.06|75.89|77.40|-167.92| +2451879|47414|4453|19122|216408|3294|40133|1|86|1636|55|61.44|121.65|7.29|0.00|400.95|3379.20|6690.75|32.07|0.00|400.95|433.02|-2978.25| +2451044|62447|9812|63973|313706|7086|42419|2|203|1637|42|66.30|116.02|63.81|2412.01|2680.02|2784.60|4872.84|10.72|2412.01|268.01|278.73|-2516.59| +2451044|62447|11840|63973|313706|7086|42419|2|100|1637|19|34.57|37.68|3.01|0.00|57.19|656.83|715.92|4.00|0.00|57.19|61.19|-599.64| +2451044|62447|7660|63973|313706|7086|42419|2|99|1637|94|47.65|69.09|6.90|0.00|648.60|4479.10|6494.46|0.00|0.00|648.60|648.60|-3830.50| +2451044|62447|12812|63973|313706|7086|42419|2|296|1637|67|72.63|87.15|12.20|0.00|817.40|4866.21|5839.05|32.69|0.00|817.40|850.09|-4048.81| +2451044|62447|7819|63973|313706|7086|42419|2|233|1637|7|24.86|29.58|15.67|0.00|109.69|174.02|207.06|0.00|0.00|109.69|109.69|-64.33| +2451044|62447|8558|63973|313706|7086|42419|2|265|1637|28|43.82|85.44|45.28|0.00|1267.84|1226.96|2392.32|114.10|0.00|1267.84|1381.94|40.88| +2451044|62447|17497|63973|313706|7086|42419|2|70|1637|66|85.97|145.28|94.43|0.00|6232.38|5674.02|9588.48|249.29|0.00|6232.38|6481.67|558.36| +2451044|62447|13783|63973|313706|7086|42419|2|249|1637|8|37.82|45.76|35.69|276.95|285.52|302.56|366.08|0.17|276.95|8.57|8.74|-293.99| +2452257|63872|1713|15522|380015|1755|28648|4|71|1638|28|4.91|9.23|3.87|73.68|108.36|137.48|258.44|1.04|73.68|34.68|35.72|-102.80| +2452257|63872|15545|15522|380015|1755|28648|4|16|1638|34|85.13|121.73|23.12|0.00|786.08|2894.42|4138.82|39.30|0.00|786.08|825.38|-2108.34| +2452257|63872|12971|15522|380015|1755|28648|4|275|1638|65|53.85|101.23|6.07|0.00|394.55|3500.25|6579.95|35.50|0.00|394.55|430.05|-3105.70| +2452257|63872|13901|15522|380015|1755|28648|4|25|1638|79|3.40|4.01|1.24|0.00|97.96|268.60|316.79|8.81|0.00|97.96|106.77|-170.64| +2452257|63872|1413|15522|380015|1755|28648|4|54|1638|22|67.36|133.37|118.69|0.00|2611.18|1481.92|2934.14|78.33|0.00|2611.18|2689.51|1129.26| +2452257|63872|8163|15522|380015|1755|28648|4|103|1638|20|87.39|90.88|14.54|229.73|290.80|1747.80|1817.60|4.27|229.73|61.07|65.34|-1686.73| +2452257|63872|13441|15522|380015|1755|28648|4|163|1638|55|31.20|38.06|29.30|0.00|1611.50|1716.00|2093.30|48.34|0.00|1611.50|1659.84|-104.50| +2452257|63872|6489|15522|380015|1755|28648|4|66|1638|77|49.47|53.92|31.27|0.00|2407.79|3809.19|4151.84|72.23|0.00|2407.79|2480.02|-1401.40| +2452257|63872|14349|15522|380015|1755|28648|4|259|1638|15|11.13|12.24|9.91|0.00|148.65|166.95|183.60|13.37|0.00|148.65|162.02|-18.30| +2452257|63872|37|15522|380015|1755|28648|4|234|1638|96|75.35|116.03|55.69|1603.87|5346.24|7233.60|11138.88|299.38|1603.87|3742.37|4041.75|-3491.23| +2452257|63872|5055|15522|380015|1755|28648|4|181|1638|89|22.21|33.09|17.20|780.70|1530.80|1976.69|2945.01|0.00|780.70|750.10|750.10|-1226.59| +2452257|63872|11015|15522|380015|1755|28648|4|103|1638|95|25.92|33.17|12.60|0.00|1197.00|2462.40|3151.15|59.85|0.00|1197.00|1256.85|-1265.40| +2452257|63872|3445|15522|380015|1755|28648|4|5|1638|90|72.60|145.20|79.86|0.00|7187.40|6534.00|13068.00|143.74|0.00|7187.40|7331.14|653.40| +2452257|63872|13933|15522|380015|1755|28648|4|236|1638|39|29.29|44.81|36.74|974.34|1432.86|1142.31|1747.59|36.68|974.34|458.52|495.20|-683.79| +2452582|60589|294|87919|1250183|6847|47511|7|35|1639|100|10.21|12.25|9.67|0.00|967.00|1021.00|1225.00|19.34|0.00|967.00|986.34|-54.00| +2452582|60589|12927|87919|1250183|6847|47511|7|81|1639|81|46.48|84.59|31.29|0.00|2534.49|3764.88|6851.79|126.72|0.00|2534.49|2661.21|-1230.39| +2452582|60589|10974|87919|1250183|6847|47511|7|112|1639|13|67.33|127.25|48.35|0.00|628.55|875.29|1654.25|37.71|0.00|628.55|666.26|-246.74| +||4123|87919|1250183|||||1639|91||||0.00||3871.14|3909.36|0.00|0.00|2932.02||| +2452582|60589|3762|87919|1250183|6847|47511|7|243|1639|94|50.60|53.13|30.81|0.00|2896.14|4756.40|4994.22|115.84|0.00|2896.14|3011.98|-1860.26| +2452582|60589|4194|87919|1250183|6847|47511|7|57|1639|67|56.27|98.47|7.87|0.00|527.29|3770.09|6597.49|47.45|0.00|527.29|574.74|-3242.80| +2452582|60589|5652|87919|1250183|6847|47511|7|115|1639|74|59.54|113.72|101.21|0.00|7489.54|4405.96|8415.28|374.47|0.00|7489.54|7864.01|3083.58| +2452582|60589|8137|87919|1250183|6847|47511|7|171|1639|93|14.65|24.90|18.17|878.70|1689.81|1362.45|2315.70|8.11|878.70|811.11|819.22|-551.34| +2452582|60589|4137|87919|1250183|6847|47511|7|220|1639|93|31.96|51.77|15.01|0.00|1395.93|2972.28|4814.61|69.79|0.00|1395.93|1465.72|-1576.35| +2452582|60589|15597|87919|1250183|6847|47511|7|236|1639|80|85.66|165.32|59.51|0.00|4760.80|6852.80|13225.60|47.60|0.00|4760.80|4808.40|-2092.00| +2452582|60589|13567|87919|1250183|6847|47511|7|260|1639|84|20.32|36.98|21.81|0.00|1832.04|1706.88|3106.32|73.28|0.00|1832.04|1905.32|125.16| +2452582|60589|9099|87919|1250183|6847|47511|7|74|1639|89|76.72|88.99|81.87|3643.21|7286.43|6828.08|7920.11|218.59|3643.21|3643.22|3861.81|-3184.86| +2452582|60589|12885|87919|1250183|6847|47511|7|17|1639|50|74.28|79.47|46.88|2156.48|2344.00|3714.00|3973.50|0.00|2156.48|187.52|187.52|-3526.48| +2452582|60589|3703|87919|1250183|6847|47511|7|267|1639|89|23.34|38.74|24.79|1809.17|2206.31|2077.26|3447.86|27.79|1809.17|397.14|424.93|-1680.12| +2452582|60589|13188|87919|1250183|6847|47511|7|121|1639|78|46.75|82.28|38.67|0.00|3016.26|3646.50|6417.84|241.30|0.00|3016.26|3257.56|-630.24| +2451845|44445|3853|98047|909243|4792|20341|10|138|1640|13|75.69|144.56|59.26|0.00|770.38|983.97|1879.28|30.81|0.00|770.38|801.19|-213.59| +2451845|44445|14801|98047|909243|4792|20341|10|122|1640|54|95.35|118.23|54.38|0.00|2936.52|5148.90|6384.42|0.00|0.00|2936.52|2936.52|-2212.38| +2451845|44445|9655|98047|909243|4792|20341|10|258|1640|69|15.89|25.58|4.09|0.00|282.21|1096.41|1765.02|0.00|0.00|282.21|282.21|-814.20| +2451845|44445|5510|98047|909243|4792|20341|10|146|1640|44|46.45|73.39|31.55|0.00|1388.20|2043.80|3229.16|111.05|0.00|1388.20|1499.25|-655.60| +||9392||||20341|||1640||20.72|40.40||0.00|||3393.60|236.14|0.00||3187.90|1211.28| +2451845|44445|15347|98047|909243|4792|20341|10|141|1640|54|28.08|54.19|20.05|0.00|1082.70|1516.32|2926.26|54.13|0.00|1082.70|1136.83|-433.62| +2451845|44445|17413|98047|909243|4792|20341|10|33|1640|34|17.96|32.32|10.34|126.56|351.56|610.64|1098.88|13.50|126.56|225.00|238.50|-385.64| +2451845|44445|3158|98047|909243|4792|20341|10|77|1640|28|71.64|98.86|48.44|0.00|1356.32|2005.92|2768.08|0.00|0.00|1356.32|1356.32|-649.60| +2451845|44445|5519|98047|909243|4792|20341|10|56|1640|95|16.87|22.09|10.60|0.00|1007.00|1602.65|2098.55|20.14|0.00|1007.00|1027.14|-595.65| +2451845|44445|6338|98047|909243|4792|20341|10|2|1640|98|97.57|187.33|43.08|0.00|4221.84|9561.86|18358.34|0.00|0.00|4221.84|4221.84|-5340.02| +2451845|44445|11563|98047|909243|4792|20341|10|20|1640|55|49.74|78.09|56.22|2102.62|3092.10|2735.70|4294.95|49.47|2102.62|989.48|1038.95|-1746.22| +2451845|44445|14665|98047|909243|4792|20341|10|221|1640|33|66.28|118.64|115.08|0.00|3797.64|2187.24|3915.12|113.92|0.00|3797.64|3911.56|1610.40| +2452534|42234|16023|38190|1892296|4449|18155|2|126|1641|99|41.35|79.80|72.61|0.00|7188.39|4093.65|7900.20|575.07|0.00|7188.39|7763.46|3094.74| +2452534|42234|13035|38190|1892296|4449|18155|2|232|1641|31|35.87|59.90|21.56|0.00|668.36|1111.97|1856.90|40.10|0.00|668.36|708.46|-443.61| +2452534|42234|3372|38190|1892296|4449|18155|2|38|1641|98|19.65|26.13|11.49|0.00|1126.02|1925.70|2560.74|33.78|0.00|1126.02|1159.80|-799.68| +2452534|42234|13633|38190|1892296|4449|18155|2|68|1641|48|65.83|79.65|11.94|464.22|573.12|3159.84|3823.20|8.71|464.22|108.90|117.61|-3050.94| +2452534|42234|14586|38190|1892296|4449|18155|2|167|1641|77|95.15|148.43|54.91|0.00|4228.07|7326.55|11429.11|211.40|0.00|4228.07|4439.47|-3098.48| +2452534|42234|9865|38190|1892296|4449|18155|2|204|1641|95|63.62|103.70|61.18|0.00|5812.10|6043.90|9851.50|464.96|0.00|5812.10|6277.06|-231.80| +2452534|42234|6879|38190|1892296|4449|18155|2|137|1641|4|11.52|20.04|19.63|0.00|78.52|46.08|80.16|0.78|0.00|78.52|79.30|32.44| +2452534|42234|17487|38190|1892296|4449|18155|2|244|1641|71|23.54|27.30|8.46|0.00|600.66|1671.34|1938.30|42.04|0.00|600.66|642.70|-1070.68| +2452534|42234|1149|38190|1892296|4449|18155|2|5|1641|99|92.55|163.81|139.23|0.00|13783.77|9162.45|16217.19|0.00|0.00|13783.77|13783.77|4621.32| +2452534|42234|12109|38190|1892296|4449|18155|2|162|1641|33|86.70|137.85|71.68|0.00|2365.44|2861.10|4549.05|47.30|0.00|2365.44|2412.74|-495.66| +2452534|42234|1860|38190|1892296|4449|18155|2|101|1641|52|49.93|54.92|42.28|0.00|2198.56|2596.36|2855.84|197.87|0.00|2198.56|2396.43|-397.80| +2452534|42234|7759|38190|1892296|4449|18155|2|127|1641|37|82.81|150.71|129.61|0.00|4795.57|3063.97|5576.27|383.64|0.00|4795.57|5179.21|1731.60| +2452534|42234|5703|38190|1892296|4449|18155|2|66|1641|2|66.60|118.54|90.09|0.00|180.18|133.20|237.08|0.00|0.00|180.18|180.18|46.98| +2452534|42234|135|38190|1892296|4449|18155|2|19|1641|48|27.62|41.15|20.98|251.76|1007.04|1325.76|1975.20|52.86|251.76|755.28|808.14|-570.48| +2451502|38217|14791|94877|1408326|6222|10871|2|206|1642|45|76.33|83.96|46.17|685.62|2077.65|3434.85|3778.20|83.52|685.62|1392.03|1475.55|-2042.82| +2451502|38217|7916|94877|1408326|6222|10871|2|5|1642|99|57.18|84.05|27.73|1070.65|2745.27|5660.82|8320.95|150.71|1070.65|1674.62|1825.33|-3986.20| +2451502|38217|11086|94877|1408326|6222|10871|2|68|1642|87|47.97|47.97|0.00|0.00|0.00|4173.39|4173.39|0.00|0.00|0.00|0.00|-4173.39| +2451502|38217|1123|94877|1408326|6222|10871|2|35|1642|75|89.45|143.12|114.49|0.00|8586.75|6708.75|10734.00|601.07|0.00|8586.75|9187.82|1878.00| +2451502|38217|15394|94877|1408326|6222|10871|2|169|1642|58|16.85|20.22|3.23|0.00|187.34|977.30|1172.76|3.74|0.00|187.34|191.08|-789.96| +2451502|38217|9394|94877|1408326|6222|10871|2|140|1642|63|4.50|7.20|0.57|35.19|35.91|283.50|453.60|0.00|35.19|0.72|0.72|-282.78| +|38217|17894|||6222||||1642|55|||26.12|0.00||3436.40|3883.00|114.92|0.00|||| +2451502|38217|728|94877|1408326|6222|10871|2|103|1642|85|93.54|124.40|24.88|0.00|2114.80|7950.90|10574.00|84.59|0.00|2114.80|2199.39|-5836.10| +2451502|38217|11564|94877|1408326|6222|10871|2|288|1642|46|16.93|23.87|5.01|0.00|230.46|778.78|1098.02|2.30|0.00|230.46|232.76|-548.32| +2451502|38217|11188|94877|1408326|6222|10871|2|176|1642|29|25.99|48.08|24.04|488.01|697.16|753.71|1394.32|6.27|488.01|209.15|215.42|-544.56| +2451502|38217|9296|94877|1408326|6222|10871|2|27|1642|2|66.71|81.38|63.47|72.35|126.94|133.42|162.76|1.09|72.35|54.59|55.68|-78.83| +2451887|71927|9176|44713|718465|5817|17969|4|6|1643|67|10.57|18.18|2.18|0.00|146.06|708.19|1218.06|10.22|0.00|146.06|156.28|-562.13| +2451887|71927|15620|44713|718465|5817|17969|4|205|1643|70|25.13|28.39|21.00|0.00|1470.00|1759.10|1987.30|0.00|0.00|1470.00|1470.00|-289.10| +2451887|71927|15743|44713|718465|5817|17969|4|194|1643|62|65.34|92.78|42.67|0.00|2645.54|4051.08|5752.36|185.18|0.00|2645.54|2830.72|-1405.54| +2451887|71927|5372|44713|718465|5817|17969|4|238|1643|74|99.47|159.15|39.78|0.00|2943.72|7360.78|11777.10|147.18|0.00|2943.72|3090.90|-4417.06| +2451887|71927|638|44713|718465|5817|17969|4|255|1643|83|16.15|19.21|6.72|0.00|557.76|1340.45|1594.43|22.31|0.00|557.76|580.07|-782.69| +2451887|71927|16219|44713|718465|5817|17969|4|19|1643|10|10.53|19.48|5.06|0.00|50.60|105.30|194.80|2.02|0.00|50.60|52.62|-54.70| +2451887|71927|8921|44713|718465|5817|17969|4|182|1643|96|12.59|16.61|13.78|0.00|1322.88|1208.64|1594.56|79.37|0.00|1322.88|1402.25|114.24| +2451887|71927|3983|44713|718465|5817|17969|4|101|1643|48|15.19|16.25|7.47|0.00|358.56|729.12|780.00|28.68|0.00|358.56|387.24|-370.56| +2451887|71927|12817|44713|718465|5817|17969|4|212|1643|15|91.76|167.92|35.26|0.00|528.90|1376.40|2518.80|5.28|0.00|528.90|534.18|-847.50| +2451887|71927|15041|44713|718465|5817|17969|4|176|1643|97|96.49|122.54|7.35|0.00|712.95|9359.53|11886.38|35.64|0.00|712.95|748.59|-8646.58| +2451887|71927|17141|44713|718465|5817|17969|4|161|1643|20|74.34|134.55|33.63|403.56|672.60|1486.80|2691.00|2.69|403.56|269.04|271.73|-1217.76| +2451887|71927|1915|44713|718465|5817|17969|4|289|1643|3|69.61|134.34|79.26|126.02|237.78|208.83|403.02|5.58|126.02|111.76|117.34|-97.07| +2451887|71927|16705|44713|718465|5817|17969|4|41|1643|93|28.51|50.17|45.65|0.00|4245.45|2651.43|4665.81|297.18|0.00|4245.45|4542.63|1594.02| +2451887|71927|5948|44713|718465|5817|17969|4|47|1643|70|94.58|164.56|148.10|0.00|10367.00|6620.60|11519.20|414.68|0.00|10367.00|10781.68|3746.40| +2451887|71927|6781|44713|718465|5817|17969|4|84|1643|87|91.96|91.96|15.63|0.00|1359.81|8000.52|8000.52|81.58|0.00|1359.81|1441.39|-6640.71| +2451887|71927|3758|44713|718465|5817|17969|4|107|1643|10|20.09|22.50|11.25|0.00|112.50|200.90|225.00|4.50|0.00|112.50|117.00|-88.40| +2452131|73369|5301|75566|746436|5306|4976|8|9|1644|6|94.46|122.79|68.76|259.91|412.56|566.76|736.74|7.63|259.91|152.65|160.28|-414.11| +2452131|73369|2861|75566|746436|5306|4976|8|190|1644|52|35.83|71.30|2.85|0.00|148.20|1863.16|3707.60|5.92|0.00|148.20|154.12|-1714.96| +2452131|73369|11109||746436||4976||82|1644|||21.05|12.84|0.00|719.04|1062.32|1178.80||0.00|||-343.28| +2452131|73369|6627|75566|746436|5306|4976|8|86|1644|58|49.17|72.77|44.38|0.00|2574.04|2851.86|4220.66|231.66|0.00|2574.04|2805.70|-277.82| +2452131|73369|7833|75566|746436|5306|4976|8|227|1644|18|86.75|95.42|69.65|639.38|1253.70|1561.50|1717.56|30.71|639.38|614.32|645.03|-947.18| +2452131|73369|11063|75566|746436|5306|4976|8|64|1644|23|98.63|108.49|69.43|0.00|1596.89|2268.49|2495.27|47.90|0.00|1596.89|1644.79|-671.60| +2452131|73369|11665|75566|746436|5306|4976|8|102|1644|65|64.04|104.38|4.17|0.00|271.05|4162.60|6784.70|13.55|0.00|271.05|284.60|-3891.55| +2452131|73369|5447|75566|746436|5306|4976|8|262|1644|18|1.32|2.36|0.02|0.34|0.36|23.76|42.48|0.00|0.34|0.02|0.02|-23.74| +2451852|44964|12095|91694|745248|2449|34050|7|63|1645|37|52.81|91.88|39.50|0.00|1461.50|1953.97|3399.56|131.53|0.00|1461.50|1593.03|-492.47| +2451852|44964|8873|91694|745248|2449|34050|7|135|1645|30|62.23|100.19|87.16|0.00|2614.80|1866.90|3005.70|156.88|0.00|2614.80|2771.68|747.90| +2451852|44964|16628|91694|745248|2449|34050|7|18|1645|60|85.22|145.72|142.80|5740.56|8568.00|5113.20|8743.20|169.64|5740.56|2827.44|2997.08|-2285.76| +2451852|44964|16823|91694|745248|2449|34050|7|49|1645|47|16.43|24.31|3.40|0.00|159.80|772.21|1142.57|4.79|0.00|159.80|164.59|-612.41| +2451852|44964|13309|91694|745248|2449|34050|7|26|1645|77|47.65|74.81|8.22|227.85|632.94|3669.05|5760.37|4.05|227.85|405.09|409.14|-3263.96| +2451852|44964|2060|91694|745248|2449|34050|7|137|1645|37|79.32|98.35|62.94|2142.47|2328.78|2934.84|3638.95|7.45|2142.47|186.31|193.76|-2748.53| +2451852|44964|14909|91694|745248|2449|34050|7|222|1645|15|10.19|19.66|12.18|0.00|182.70|152.85|294.90|1.82|0.00|182.70|184.52|29.85| +2451852|44964|5357|91694|745248|2449|34050|7|232|1645|83|4.39|7.02|2.52|0.00|209.16|364.37|582.66|2.09|0.00|209.16|211.25|-155.21| +2451852|44964|974|91694|745248|2449|34050|7|143|1645|27|71.92|129.45|64.72|332.01|1747.44|1941.84|3495.15|70.77|332.01|1415.43|1486.20|-526.41| +2451852|44964|2576|91694|745248|2449|34050|7|261|1645|27|79.07|117.02|46.80|0.00|1263.60|2134.89|3159.54|113.72|0.00|1263.60|1377.32|-871.29| +2451852|44964|16975|91694|745248|2449|34050|7|71|1645|90|34.12|41.28|31.78|972.46|2860.20|3070.80|3715.20|56.63|972.46|1887.74|1944.37|-1183.06| +2451852|44964|4103|91694|745248|2449|34050|7|17|1645|98|1.73|2.68|2.19|0.00|214.62|169.54|262.64|19.31|0.00|214.62|233.93|45.08| +2451852|44964|17792|91694|745248|2449|34050|7|60|1645|29|9.26|16.57|8.45|120.07|245.05|268.54|480.53|11.24|120.07|124.98|136.22|-143.56| +2451852|44964|5564|91694|745248|2449|34050|7|123|1645|94|87.48|137.34|134.59|0.00|12651.46|8223.12|12909.96|1138.63|0.00|12651.46|13790.09|4428.34| +2451831|46169|16940|26102|1083577|699|46574|4|143|1646|29|51.06|87.31|52.38|0.00|1519.02|1480.74|2531.99|91.14|0.00|1519.02|1610.16|38.28| +2451831|46169|9565|26102|1083577|699|46574|4|271|1646|98|8.42|13.55|6.23|0.00|610.54|825.16|1327.90|54.94|0.00|610.54|665.48|-214.62| +2451831|46169|17023|26102|1083577|699|46574|4|108|1646|97|85.20|101.38|18.24|0.00|1769.28|8264.40|9833.86|159.23|0.00|1769.28|1928.51|-6495.12| +2451831|46169|15907|26102|1083577|699|46574|4|139|1646|96|45.73|76.82|68.36|3281.28|6562.56|4390.08|7374.72|229.68|3281.28|3281.28|3510.96|-1108.80| +2451831|46169|16634|26102|1083577|699|46574|4|294|1646|64|61.44|72.49|27.54|0.00|1762.56|3932.16|4639.36|17.62|0.00|1762.56|1780.18|-2169.60| +2451831|46169|4988|26102|1083577|699|46574|4|269|1646|72|62.46|119.92|58.76|1438.44|4230.72|4497.12|8634.24|55.84|1438.44|2792.28|2848.12|-1704.84| +2451831|46169|797|26102|1083577|699|46574|4|212|1646|76|18.19|18.55|17.25|0.00|1311.00|1382.44|1409.80|117.99|0.00|1311.00|1428.99|-71.44| +2451831|46169|11579|26102|1083577|699|46574|4|78|1646|94|13.94|23.41|21.30|0.00|2002.20|1310.36|2200.54|40.04|0.00|2002.20|2042.24|691.84| +2451831|46169|14173|26102|1083577|699|46574|4|30|1646|26|50.50|53.02|1.06|0.00|27.56|1313.00|1378.52|0.55|0.00|27.56|28.11|-1285.44| +2451831||14090|26102|1083577|699||4|202|1646|98|||119.92||||15463.42|116.34||||| +2451831|46169|2894|26102|1083577|699|46574|4|296|1646|24|69.70|116.39|82.63|0.00|1983.12|1672.80|2793.36|0.00|0.00|1983.12|1983.12|310.32| +2451831|46169|1381|26102|1083577||46574|4||1646|77|22.93||1.77||||2276.89|||||| +2451831|46169|9065|26102|1083577|699|46574|4|166|1646|100|63.82|64.45|41.24|0.00|4124.00|6382.00|6445.00|82.48|0.00|4124.00|4206.48|-2258.00| +2451831|46169|12241|26102|1083577|699|46574|4|126|1646|87|46.95|64.79|22.67|0.00|1972.29|4084.65|5636.73|19.72|0.00|1972.29|1992.01|-2112.36| +2451524|42657|2566|27843|1298713|6383|6853|4|250|1647|41|16.88|26.33|10.79|0.00|442.39|692.08|1079.53|8.84|0.00|442.39|451.23|-249.69| +2451524|42657|13006|27843|1298713|6383|6853|4|94|1647|100|93.20|162.16|145.94|1167.52|14594.00|9320.00|16216.00|1074.11|1167.52|13426.48|14500.59|4106.48| +2451524|42657|9898|27843|1298713|6383|6853|4|29|1647|51|7.36|14.64|8.34|0.00|425.34|375.36|746.64|17.01|0.00|425.34|442.35|49.98| +2451524|42657|4718|27843|1298713|6383|6853|4|210|1647|95|39.40|52.79|42.23|0.00|4011.85|3743.00|5015.05|40.11|0.00|4011.85|4051.96|268.85| +2451524|42657|7807|||6383|6853|4||1647||53.93||80.03|||||5.47||91.24||-16.62| +|42657|979|27843|1298713|6383|||111|1647|||194.62|81.74||5721.80|||||5721.80||| +2451524|42657|7516|27843|1298713|6383|6853|4|200|1647|45|77.90|133.20|13.32|0.00|599.40|3505.50|5994.00|53.94|0.00|599.40|653.34|-2906.10| +2451524|42657|7094|27843|1298713|6383|6853|4|8|1647|7|67.36|105.08|85.11|0.00|595.77|471.52|735.56|5.95|0.00|595.77|601.72|124.25| +2451524|42657|7918|27843|1298713|6383|6853|4|150|1647|24|51.22|55.82|25.67|0.00|616.08|1229.28|1339.68|49.28|0.00|616.08|665.36|-613.20| +2451524|42657|11887|27843|1298713|6383|6853|4|273|1647|60|5.73|7.56|4.53|0.00|271.80|343.80|453.60|21.74|0.00|271.80|293.54|-72.00| +2451524|42657|17212|27843|1298713|6383|6853|4|50|1647|99|80.71|90.39|29.82|1239.91|2952.18|7990.29|8948.61|136.98|1239.91|1712.27|1849.25|-6278.02| +2451524|42657|1267|27843|1298713|6383|6853|4|199|1647|94|42.87|71.16|0.00|0.00|0.00|4029.78|6689.04|0.00|0.00|0.00|0.00|-4029.78| +2451524|42657|2396|27843|1298713|6383|6853|4|155|1647|84|10.09|13.62|1.08|0.00|90.72|847.56|1144.08|8.16|0.00|90.72|98.88|-756.84| +|42657|9424||1298713|6383||4|72|1647||||3.48|0.00|||5291.12||0.00|264.48|272.41|| +2452131|51270|12817|77700|536631|5162|42893|7|217|1648|69|92.22|181.67|110.81|0.00|7645.89|6363.18|12535.23|305.83|0.00|7645.89|7951.72|1282.71| +2452131|51270|15041|77700|536631|5162|42893|7|7|1648|15|87.69|164.85|135.17|0.00|2027.55|1315.35|2472.75|182.47|0.00|2027.55|2210.02|712.20| +2452131|51270|17141|77700|536631|5162|42893|7|34|1648|52|94.33|128.28|2.56|0.00|133.12|4905.16|6670.56|0.00|0.00|133.12|133.12|-4772.04| +2452131|51270|1915|77700|536631|5162|42893|7|77|1648|91|41.78|44.70|4.47|105.76|406.77|3801.98|4067.70|27.09|105.76|301.01|328.10|-3500.97| +2452131|51270|16705|77700|536631|5162|42893|7|298|1648|22|39.85|56.58|20.36|0.00|447.92|876.70|1244.76|26.87|0.00|447.92|474.79|-428.78| +2452131|51270|5949|77700|536631|5162|42893|7|46|1648|4|92.90|122.62|42.91|0.00|171.64|371.60|490.48|13.73|0.00|171.64|185.37|-199.96| +2452131|51270|6781|77700|536631|5162|42893|7|139|1648|50|11.79|13.67|13.67|0.00|683.50|589.50|683.50|6.83|0.00|683.50|690.33|94.00| +2452131|51270|3759|77700|536631|5162|42893|7|150|1648|2|92.38|99.77|45.89|0.00|91.78|184.76|199.54|3.67|0.00|91.78|95.45|-92.98| +2452131|51270|10611|77700|536631|5162|42893|7|165|1648|83|47.72|72.05|48.27|0.00|4006.41|3960.76|5980.15|0.00|0.00|4006.41|4006.41|45.65| +2452131|51270|11927|77700|536631|5162|42893|7|131|1648|36|19.87|20.06|16.24|251.39|584.64|715.32|722.16|19.99|251.39|333.25|353.24|-382.07| +2452131|51270|4361|77700|536631|5162|42893|7|184|1648|41|72.17|126.29|117.44|0.00|4815.04|2958.97|5177.89|48.15|0.00|4815.04|4863.19|1856.07| +2452131|51270|1597|77700|536631|5162|42893|7|77|1648|16|44.78|60.90|49.93|0.00|798.88|716.48|974.40|39.94|0.00|798.88|838.82|82.40| +2452131|51270|3283|77700|536631|5162|42893|7|94|1648|38|76.31|139.64|121.48|3000.55|4616.24|2899.78|5306.32|80.78|3000.55|1615.69|1696.47|-1284.09| +2452131|51270|10415|77700|536631|5162|42893|7|114|1648|82|38.09|75.03|64.52|0.00|5290.64|3123.38|6152.46|476.15|0.00|5290.64|5766.79|2167.26| +2452131|51270|13127|77700|536631|5162|42893|7|154|1648|76|37.54|69.07|8.97|0.00|681.72|2853.04|5249.32|61.35|0.00|681.72|743.07|-2171.32| +2452131|51270|9607|77700|536631|5162|42893|7|207|1648|90|76.62|109.56|26.29|0.00|2366.10|6895.80|9860.40|189.28|0.00|2366.10|2555.38|-4529.70| +2450862|68385|7526|61889|1722255|5821|41387|8|203|1649|99|70.42|78.16|28.13|0.00|2784.87|6971.58|7737.84|0.00|0.00|2784.87|2784.87|-4186.71| +2450862|68385|2350|61889|1722255|5821|41387|8|230|1649|95|28.80|43.48|40.43|0.00|3840.85|2736.00|4130.60|153.63|0.00|3840.85|3994.48|1104.85| +2450862|68385|2623|61889|1722255|5821|41387|8|118|1649|39|61.53|74.45|60.30|0.00|2351.70|2399.67|2903.55|188.13|0.00|2351.70|2539.83|-47.97| +2450862|68385|9058|61889|1722255|5821|41387|8|209|1649|6|83.02|102.94|58.67|316.81|352.02|498.12|617.64|2.46|316.81|35.21|37.67|-462.91| +2450862|68385|12340|61889|1722255|5821|41387|8|228|1649|78|94.71|156.27|3.12|0.00|243.36|7387.38|12189.06|9.73|0.00|243.36|253.09|-7144.02| +2450862|68385|14149|61889|1722255|5821|41387|8|116|1649|87|25.99|40.28|1.20|0.00|104.40|2261.13|3504.36|1.04|0.00|104.40|105.44|-2156.73| +2450862|68385|15802|61889|1722255|5821|41387|8|170|1649|83|62.40|117.31|48.09|0.00|3991.47|5179.20|9736.73|239.48|0.00|3991.47|4230.95|-1187.73| +2450862|68385|16490|61889|1722255|5821|41387|8|177|1649|8|63.59|100.47|21.09|0.00|168.72|508.72|803.76|0.00|0.00|168.72|168.72|-340.00| +2450862|68385|13330|61889|1722255|5821|41387|8|176|1649|98|9.29|17.83|2.67|0.00|261.66|910.42|1747.34|13.08|0.00|261.66|274.74|-648.76| +2450862|68385|16909|61889|1722255|5821|41387|8|74|1649|75|9.88|19.16|16.47|0.00|1235.25|741.00|1437.00|111.17|0.00|1235.25|1346.42|494.25| +2450862|68385|11707|61889|1722255|5821|41387|8|140|1649|56|9.21|10.59|0.95|15.42|53.20|515.76|593.04|1.88|15.42|37.78|39.66|-477.98| +2450862|68385|13448|61889|1722255|5821|41387|8|104|1649|6|5.58|7.81|4.21|0.00|25.26|33.48|46.86|0.00|0.00|25.26|25.26|-8.22| +|68385|16837|61889|1722255|5821||8|198|1649|90|52.37||11.78||||5891.40|||1060.20||-3653.10| +2450862|68385|7303|61889|1722255|5821|41387|8|182|1649|93|63.94|115.09|13.81|0.00|1284.33|5946.42|10703.37|0.00|0.00|1284.33|1284.33|-4662.09| +2450862|68385|8449|61889|1722255|5821|41387|8|53|1649|37|72.26|132.23|66.11|0.00|2446.07|2673.62|4892.51|73.38|0.00|2446.07|2519.45|-227.55| +2450862|68385|16429|61889|1722255|5821|41387|8|219|1649|51|74.59|146.19|38.00|0.00|1938.00|3804.09|7455.69|155.04|0.00|1938.00|2093.04|-1866.09| +2451509|69673|16081|46098|1642241|6031|30174|2|231|1650|45|97.51|141.38|82.00|0.00|3690.00|4387.95|6362.10|184.50|0.00|3690.00|3874.50|-697.95| +2451509|69673|1312|46098|1642241|6031|30174|2|67|1650|12|70.82|102.68|98.57|0.00|1182.84|849.84|1232.16|23.65|0.00|1182.84|1206.49|333.00| +|69673|7135||1642241|6031||2||1650|96|1.62|2.75|1.92|0.00|184.32|||0.00|0.00|184.32|184.32|28.80| +2451509|69673|1984|46098|1642241|6031|30174|2|185|1650|88|67.83|105.81|80.41|0.00|7076.08|5969.04|9311.28|495.32|0.00|7076.08|7571.40|1107.04| +2451509|69673|5023|46098|1642241|6031|30174|2|235|1650|64|75.75|134.83|49.88|0.00|3192.32|4848.00|8629.12|223.46|0.00|3192.32|3415.78|-1655.68| +2451509||1909|46098|1642241||30174|2|137|1650|||46.62|37.76|0.00|2492.16||3076.92|149.52|0.00||2641.68|929.94| +2451509|69673|3097|46098|1642241|6031|30174|2|36|1650|98|70.33|82.98|55.59|0.00|5447.82|6892.34|8132.04|326.86|0.00|5447.82|5774.68|-1444.52| +2451509|69673|2197|46098|1642241|6031|30174|2|166|1650|82|14.52|27.44|17.28|0.00|1416.96|1190.64|2250.08|0.00|0.00|1416.96|1416.96|226.32| +2451509|69673|5990|46098|1642241|6031|30174|2|68|1650|9|23.30|32.15|14.78|0.00|133.02|209.70|289.35|9.31|0.00|133.02|142.33|-76.68| +2451509|69673|10820|46098|1642241|6031|30174|2|138|1650|67|94.51|148.38|19.28|0.00|1291.76|6332.17|9941.46|51.67|0.00|1291.76|1343.43|-5040.41| +2451509|69673|16243|46098|1642241|6031|30174|2|236|1650|14|21.06|23.37|9.81|0.00|137.34|294.84|327.18|6.86|0.00|137.34|144.20|-157.50| +2451509|69673|2914|46098|1642241|6031|30174|2|276|1650|72|86.14|155.91|67.04|2172.09|4826.88|6202.08|11225.52|26.54|2172.09|2654.79|2681.33|-3547.29| +2451509|69673|17575|46098|1642241|6031|30174|2|234|1650|79|95.34|141.10|134.04|0.00|10589.16|7531.86|11146.90|953.02|0.00|10589.16|11542.18|3057.30| +2451509|69673|6314|46098|1642241|6031|30174|2|168|1650|47|62.68|108.43|71.56|0.00|3363.32|2945.96|5096.21|100.89|0.00|3363.32|3464.21|417.36| +2451509|69673|5152|46098|1642241|6031|30174|2|88|1650|1|40.27|64.02|51.21|0.00|51.21|40.27|64.02|4.09|0.00|51.21|55.30|10.94| +||3999||499545|||||1651||53.77||78.05|0.00|||5648.50|214.63|0.00|||| +2451989|60995|14607|69695|499545|6642|12139|1|249|1651|6|43.85|65.77|23.01|0.00|138.06|263.10|394.62|8.28|0.00|138.06|146.34|-125.04| +2451989|60995|8175|69695|499545|6642|12139|1|242|1651|70|56.43|79.56|33.41|1847.57|2338.70|3950.10|5569.20|29.46|1847.57|491.13|520.59|-3458.97| +2451989|60995|7453|69695|499545|6642|12139|1|216|1651|29|14.54|28.06|12.06|0.00|349.74|421.66|813.74|6.99|0.00|349.74|356.73|-71.92| +2451989|60995|15193|69695|499545|6642|12139|1|121|1651|42|38.79|76.02|47.89|0.00|2011.38|1629.18|3192.84|160.91|0.00|2011.38|2172.29|382.20| +2451989|60995|3087|69695|499545|6642|12139|1|141|1651|26|68.68|126.37|112.46|0.00|2923.96|1785.68|3285.62|87.71|0.00|2923.96|3011.67|1138.28| +2451989|60995|6003|69695|499545|6642|12139|1|146|1651|51|17.99|30.40|19.45|0.00|991.95|917.49|1550.40|39.67|0.00|991.95|1031.62|74.46| +2451989|60995|1619|69695|499545|6642|12139|1|222|1651|82|39.33|51.12|4.60|309.30|377.20|3225.06|4191.84|2.71|309.30|67.90|70.61|-3157.16| +2451989|60995|4415|69695|499545|6642|12139|1|278|1651|6|79.69|145.03|56.56|0.00|339.36|478.14|870.18|0.00|0.00|339.36|339.36|-138.78| +2451989|60995|3885|69695|499545|6642|12139|1|21|1651|25|78.90|95.46|75.41|603.28|1885.25|1972.50|2386.50|38.45|603.28|1281.97|1320.42|-690.53| +2451989|60995|1617|69695|499545|6642|12139|1|264|1651|10|70.11|75.01|2.25|0.00|22.50|701.10|750.10|1.57|0.00|22.50|24.07|-678.60| +2451989|60995|5439|69695|499545|6642|12139|1|219|1651|53|91.46|139.93|138.53|0.00|7342.09|4847.38|7416.29|587.36|0.00|7342.09|7929.45|2494.71| +2451989|60995|14531|69695|499545|6642|12139|1|132|1651|20|93.06|93.99|40.41|0.00|808.20|1861.20|1879.80|16.16|0.00|808.20|824.36|-1053.00| +2451989|60995|3645|69695|499545|6642|12139|1|62|1651|94|46.99|59.20|15.98|450.63|1502.12|4417.06|5564.80|73.60|450.63|1051.49|1125.09|-3365.57| +2451989|60995|937|69695|499545|6642|12139|1|53|1651|25|51.68|77.00|21.56|0.00|539.00|1292.00|1925.00|32.34|0.00|539.00|571.34|-753.00| +2451989|60995|11023|69695|499545|6642|12139|1|183|1651|4|81.54|140.24|134.63|0.00|538.52|326.16|560.96|48.46|0.00|538.52|586.98|212.36| +2451349|38996|3448|81204|1170826|448|19223|4|162|1652|88|16.67|29.17|26.54|2172.03|2335.52|1466.96|2566.96|6.53|2172.03|163.49|170.02|-1303.47| +2451349|38996|7531|81204|1170826|448|19223|4|181|1652|52|92.96|184.99|11.09|0.00|576.68|4833.92|9619.48|46.13|0.00|576.68|622.81|-4257.24| +2451349|38996|9560|81204|1170826|448|19223|4|18|1652|73|74.56|84.25|57.29|0.00|4182.17|5442.88|6150.25|250.93|0.00|4182.17|4433.10|-1260.71| +2451349|38996|15404|81204|1170826|448|19223|4|119|1652|78|58.89|83.03|16.60|828.67|1294.80|4593.42|6476.34|9.32|828.67|466.13|475.45|-4127.29| +2451349|38996|9884|81204|1170826|448|19223|4|121|1652|100|91.15|116.67|81.66|0.00|8166.00|9115.00|11667.00|163.32|0.00|8166.00|8329.32|-949.00| +2451349|38996|5104|81204|1170826|448|19223|4|26|1652|18|47.41|65.42|24.85|0.00|447.30|853.38|1177.56|4.47|0.00|447.30|451.77|-406.08| +2451349|38996|6064|81204|1170826|448|19223|4|21|1652|47|59.99|76.18|35.04|0.00|1646.88|2819.53|3580.46|131.75|0.00|1646.88|1778.63|-1172.65| +2451349|38996|9172|81204|1170826|448|19223|4|237|1652|60|10.42|18.02|14.95|0.00|897.00|625.20|1081.20|8.97|0.00|897.00|905.97|271.80| +2451349|38996|3116|81204|1170826|448|19223|4|92|1652|95|91.83|104.68|20.93|0.00|1988.35|8723.85|9944.60|19.88|0.00|1988.35|2008.23|-6735.50| +2451349|38996|15800|81204|1170826|448|19223|4|131|1652|21|16.23|32.46|3.89|0.00|81.69|340.83|681.66|3.26|0.00|81.69|84.95|-259.14| +2451349|38996|12130|81204|1170826|448|19223|4|99|1652|74|30.88|50.64|32.40|2325.67|2397.60|2285.12|3747.36|4.31|2325.67|71.93|76.24|-2213.19| +2451349|38996|17390|81204|1170826|448|19223|4|274|1652|98|58.65|64.51|43.86|0.00|4298.28|5747.70|6321.98|171.93|0.00|4298.28|4470.21|-1449.42| +2451349|38996|10078|81204|1170826|448|19223|4|155|1652|89|54.01|66.97|34.82|0.00|3098.98|4806.89|5960.33|278.90|0.00|3098.98|3377.88|-1707.91| +2451349|38996|4927|81204|1170826|448|19223|4|58|1652|87|25.06|45.10|6.76|0.00|588.12|2180.22|3923.70|23.52|0.00|588.12|611.64|-1592.10| +2451349|38996|14875|81204|1170826|448|19223|4|215|1652|53|71.52|130.16|26.03|0.00|1379.59|3790.56|6898.48|82.77|0.00|1379.59|1462.36|-2410.97| +2450988|61207|16154|76886|1804375|1464|20153|10|2|1653|9|4.16|6.15|5.84|0.00|52.56|37.44|55.35|3.15|0.00|52.56|55.71|15.12| +2450988|61207|2932|76886|1804375|1464|20153|10|137|1653|21|98.71|103.64|31.09|359.08|652.89|2072.91|2176.44|14.69|359.08|293.81|308.50|-1779.10| +2450988|61207|8522|76886|1804375|1464|20153|10|84|1653|2|77.87|130.82|20.93|0.00|41.86|155.74|261.64|3.34|0.00|41.86|45.20|-113.88| +||8749|76886|1804375||||129|1653|||104.81|19.91|0.00||3605.04|5659.74||0.00|1075.14|1075.14|-2529.90| +2450988|61207|13417|76886|1804375|1464|20153|10|61|1653|96|70.67|98.93|6.92|0.00|664.32|6784.32|9497.28|19.92|0.00|664.32|684.24|-6120.00| +2450988|61207|6775|76886|1804375|1464|20153|10|144|1653|73|19.33|31.12|18.67|0.00|1362.91|1411.09|2271.76|40.88|0.00|1362.91|1403.79|-48.18| +2450988|61207|12202|76886|1804375|1464|20153|10|135|1653|85|63.62|90.97|10.91|0.00|927.35|5407.70|7732.45|0.00|0.00|927.35|927.35|-4480.35| +2450988|61207|13376|76886|1804375|1464|20153|10|61|1653|97|49.76|66.67|58.66|682.80|5690.02|4826.72|6466.99|50.07|682.80|5007.22|5057.29|180.50| +2450988|61207|7150|76886|1804375|1464|20153|10|295|1653|27|43.35|48.55|13.10|0.00|353.70|1170.45|1310.85|17.68|0.00|353.70|371.38|-816.75| +2450988|61207|13316|76886|1804375|1464|20153|10|174|1653|34|96.27|149.21|92.51|0.00|3145.34|3273.18|5073.14|220.17|0.00|3145.34|3365.51|-127.84| +2450988|61207|11719|76886|1804375|1464|20153|10|75|1653|50|83.14|150.48|39.12|0.00|1956.00|4157.00|7524.00|58.68|0.00|1956.00|2014.68|-2201.00| +2450988|61207|7735|76886|1804375|1464|20153|10|8|1653|36|27.63|41.99|24.35|0.00|876.60|994.68|1511.64|35.06|0.00|876.60|911.66|-118.08| +2450988|61207|5548|76886|1804375|1464|20153|10|89|1653|39|9.43|13.67|6.83|0.00|266.37|367.77|533.13|15.98|0.00|266.37|282.35|-101.40| +2450988|61207|15518|76886|1804375|1464|20153|10|32|1653|38|88.40|124.64|19.94|318.24|757.72|3359.20|4736.32|21.97|318.24|439.48|461.45|-2919.72| +2450988|61207|13766|76886|1804375|1464|20153|10|211|1653|36|92.85|168.05|146.20|0.00|5263.20|3342.60|6049.80|421.05|0.00|5263.20|5684.25|1920.60| +2450988|61207|7669|76886|1804375|1464|20153|10|91|1653|16|12.07|19.07|10.67|0.00|170.72|193.12|305.12|10.24|0.00|170.72|180.96|-22.40| +2451169|45338|5834|85224|1704938|3852|5566|2|63|1654|48|29.45|40.05|3.20|0.00|153.60|1413.60|1922.40|3.07|0.00|153.60|156.67|-1260.00| +2451169|45338|3818|85224|1704938|3852|5566|2|136|1654|59|43.46|63.01|6.30|0.00|371.70|2564.14|3717.59|11.15|0.00|371.70|382.85|-2192.44| +2451169|45338|2144|85224|1704938|3852|5566|2|61|1654|80|71.21|109.66|106.37|3574.03|8509.60|5696.80|8772.80|394.84|3574.03|4935.57|5330.41|-761.23| +2451169|45338|3619|85224|1704938|3852|5566|2|132|1654|39|46.38|68.64|17.16|0.00|669.24|1808.82|2676.96|13.38|0.00|669.24|682.62|-1139.58| +2451169|45338|11702|85224|1704938|3852|5566|2|222|1654|43|10.90|12.09|7.61|0.00|327.23|468.70|519.87|16.36|0.00|327.23|343.59|-141.47| +2451169|45338|13238|85224|1704938|3852|5566|2|166|1654|40|23.20|33.17|20.56|0.00|822.40|928.00|1326.80|16.44|0.00|822.40|838.84|-105.60| +2451169|45338|9442|85224|1704938|3852|5566|2|87|1654|21|60.92|90.77|62.63|0.00|1315.23|1279.32|1906.17|78.91|0.00|1315.23|1394.14|35.91| +2451169||8848|85224||||||1654|||122.47|1.22|0.00|58.56|4387.20|||0.00|58.56|63.83|-4328.64| +2451169|45338|4052|85224|1704938|3852|5566|2|274|1654|40|83.62|155.53|88.65|567.36|3546.00|3344.80|6221.20|59.57|567.36|2978.64|3038.21|-366.16| +2451169|45338|15160|85224|1704938|3852|5566|2|72|1654|70|18.13|22.66|18.35|0.00|1284.50|1269.10|1586.20|51.38|0.00|1284.50|1335.88|15.40| +2451169|45338|13135|85224|1704938|3852|5566|2|152|1654|51|73.93|133.07|105.12|2573.33|5361.12|3770.43|6786.57|139.38|2573.33|2787.79|2927.17|-982.64| +2451169|45338|16834|85224|1704938|3852|5566|2|98|1654|63|23.77|43.26|29.41|1834.30|1852.83|1497.51|2725.38|1.66|1834.30|18.53|20.19|-1478.98| +2451169|45338|13729|85224|1704938|3852|5566|2|28|1654|70|20.37|35.03|31.17|0.00|2181.90|1425.90|2452.10|130.91|0.00|2181.90|2312.81|756.00| +|45338|12637|||3852|||108|1654||40.10||13.64||422.84||2349.18|33.82|||456.66|| +2451439|61530|17924|45421|1103085|6796|40460|10|185|1655|88|99.82|132.76|74.34|0.00|6541.92|8784.16|11682.88|327.09|0.00|6541.92|6869.01|-2242.24| +2451439|61530|7105|45421|1103085|6796|40460|10|261|1655|98|49.82|50.31|49.30|0.00|4831.40|4882.36|4930.38|144.94|0.00|4831.40|4976.34|-50.96| +2451439|61530|11437|45421|1103085|6796|40460|10|177|1655|38|38.35|52.53|24.68|356.37|937.84|1457.30|1996.14|46.51|356.37|581.47|627.98|-875.83| +2451439|61530|9595|45421|1103085|6796|40460|10|223|1655|80|77.03|86.27|18.97|0.00|1517.60|6162.40|6901.60|15.17|0.00|1517.60|1532.77|-4644.80| +2451439|61530|5317|45421|1103085|6796|40460|10|119|1655|6|82.19|140.54|42.16|0.00|252.96|493.14|843.24|15.17|0.00|252.96|268.13|-240.18| +2451439|61530|1081|45421|1103085|6796|40460|10|281|1655|25|96.87|170.49|5.11|0.00|127.75|2421.75|4262.25|11.49|0.00|127.75|139.24|-2294.00| +2451439||14588||||40460||245|1655||||||384.32||915.36|||||-378.56| +2451439|61530|17084|45421|1103085|6796|40460|10|97|1655|67|30.27|37.83|20.04|0.00|1342.68|2028.09|2534.61|13.42|0.00|1342.68|1356.10|-685.41| +2451439|61530|12178|45421|1103085|6796|40460|10|48|1655|59|11.94|14.80|0.00|0.00|0.00|704.46|873.20|0.00|0.00|0.00|0.00|-704.46| +2451439|61530|886|45421|1103085|6796|40460|10|257|1655|88|49.89|82.31|4.93|360.08|433.84|4390.32|7243.28|5.16|360.08|73.76|78.92|-4316.56| +2451439|61530|10453|45421|1103085|6796|40460|10|174|1655|84|12.81|24.33|4.13|0.00|346.92|1076.04|2043.72|0.00|0.00|346.92|346.92|-729.12| +2451439|61530|8746|45421|1103085|6796|40460|10|255|1655|88|43.51|81.36|3.25|0.00|286.00|3828.88|7159.68|17.16|0.00|286.00|303.16|-3542.88| +2451439|61530|11030|45421|1103085|6796|40460|10|75|1655|57|58.93|107.84|71.17|0.00|4056.69|3359.01|6146.88|283.96|0.00|4056.69|4340.65|697.68| +2451439|61530|17920|45421|1103085|6796|40460|10|185|1655|55|10.21|17.45|1.39|0.00|76.45|561.55|959.75|1.52|0.00|76.45|77.97|-485.10| +2451439|61530|11456|45421|1103085|6796|40460|10|281|1655|8|23.80|35.93|2.15|9.11|17.20|190.40|287.44|0.24|9.11|8.09|8.33|-182.31| +2451439|61530|10597|45421|1103085|6796|40460|10|183|1655|48|58.77|72.28|7.95|0.00|381.60|2820.96|3469.44|26.71|0.00|381.60|408.31|-2439.36| +2451762|43823|2465|87080|1340132|4384|46236|8|100|1656|47|50.17|100.34|7.02|0.00|329.94|2357.99|4715.98|19.79|0.00|329.94|349.73|-2028.05| +2451762|43823|8516|87080|1340132|4384|46236|8|71|1656|21|48.86|52.28|46.00|0.00|966.00|1026.06|1097.88|9.66|0.00|966.00|975.66|-60.06| +2451762|43823|14363|87080|1340132|4384|46236|8|159|1656|89|69.98|116.86|71.28|0.00|6343.92|6228.22|10400.54|190.31|0.00|6343.92|6534.23|115.70| +2451762|43823|2561|87080|1340132|4384|46236|8|31|1656|11|24.15|42.50|41.65|0.00|458.15|265.65|467.50|32.07|0.00|458.15|490.22|192.50| +2451762|43823|14705|87080|1340132|4384|46236|8|131|1656|11|78.35|101.85|5.09|0.00|55.99|861.85|1120.35|3.91|0.00|55.99|59.90|-805.86| +2451762|43823|3973|87080|1340132|4384|46236|8|235|1656|78|56.36|71.57|14.31|0.00|1116.18|4396.08|5582.46|66.97|0.00|1116.18|1183.15|-3279.90| +2451762|43823|6071|87080|1340132|4384|46236|8|144|1656|84|91.30|123.25|71.48|0.00|6004.32|7669.20|10353.00|540.38|0.00|6004.32|6544.70|-1664.88| +2451762||6883||||46236|8|13|1656|94|65.64||90.37|0.00||6170.16||0.00|0.00|||2324.62| +2451762|43823|380|87080|1340132|4384|46236|8|173|1656|26|58.21|68.68|23.35|0.00|607.10|1513.46|1785.68|48.56|0.00|607.10|655.66|-906.36| +2451762|43823|14324|87080|1340132|4384|46236|8|245|1656|80|41.33|59.10|15.95|0.00|1276.00|3306.40|4728.00|51.04|0.00|1276.00|1327.04|-2030.40| +2451762|43823|9661|87080|1340132|4384|46236|8|222|1656|37|19.36|29.81|19.97|0.00|738.89|716.32|1102.97|66.50|0.00|738.89|805.39|22.57| +2451762|43823|14768|87080|1340132|4384|46236|8|248|1656|33|50.49|66.64|66.64|0.00|2199.12|1666.17|2199.12|0.00|0.00|2199.12|2199.12|532.95| +2451762|43823|15685|87080|1340132|4384|46236|8|165|1656|72|71.44|123.59|123.59|6317.92|8898.48|5143.68|8898.48|77.41|6317.92|2580.56|2657.97|-2563.12| +2451762|43823|3446|87080|1340132|4384|46236|8|76|1656|97|15.64|16.73|9.03|87.59|875.91|1517.08|1622.81|0.00|87.59|788.32|788.32|-728.76| +2451762|43823|15638|87080|1340132|4384|46236|8|178|1656|30|89.15|177.40|170.30|0.00|5109.00|2674.50|5322.00|153.27|0.00|5109.00|5262.27|2434.50| +2451464|40008|7531|11333|1778836|5986|5089|7|297|1657|74|92.27|126.40|72.04|1332.74|5330.96|6827.98|9353.60|39.98|1332.74|3998.22|4038.20|-2829.76| +2451464|40008|9560|11333|1778836|5986|5089|7|299|1657|52|39.93|52.30|50.73|0.00|2637.96|2076.36|2719.60|211.03|0.00|2637.96|2848.99|561.60| +2451464|40008|15404|11333|1778836|5986|5089|7|232|1657|9|63.38|121.68|120.46|0.00|1084.14|570.42|1095.12|0.00|0.00|1084.14|1084.14|513.72| +2451464|40008|9884|11333|1778836|5986|5089|7|44|1657|3|85.96|121.20|75.14|0.00|225.42|257.88|363.60|20.28|0.00|225.42|245.70|-32.46| +2451464|40008|5104|11333|1778836|5986|5089|7|96|1657|60|19.10|28.45|22.47|0.00|1348.20|1146.00|1707.00|67.41|0.00|1348.20|1415.61|202.20| +2451464|40008|6064|11333|1778836|5986|5089|7|41|1657|27|20.14|37.46|7.49|0.00|202.23|543.78|1011.42|8.08|0.00|202.23|210.31|-341.55| +2451464|40008|9172|11333|1778836|5986|5089|7|63|1657|48|41.40|71.20|2.13|0.00|102.24|1987.20|3417.60|5.11|0.00|102.24|107.35|-1884.96| +2451464|40008|3116|11333|1778836|5986|5089|7|111|1657|40|90.84|176.22|88.11|2008.90|3524.40|3633.60|7048.80|90.93|2008.90|1515.50|1606.43|-2118.10| +2451464|40008|15800|11333|1778836|5986|5089|7|203|1657|50|6.37|12.10|2.54|0.00|127.00|318.50|605.00|11.43|0.00|127.00|138.43|-191.50| +2451464|40008|12130|11333|1778836|5986|5089|7|218|1657|76|33.58|41.97|6.71|0.00|509.96|2552.08|3189.72|40.79|0.00|509.96|550.75|-2042.12| +2451464|40008|17390|11333|1778836|5986|5089|7|281|1657|79|31.09|41.34|31.00|0.00|2449.00|2456.11|3265.86|220.41|0.00|2449.00|2669.41|-7.11| +2451464|40008|10078|11333|1778836|5986|5089|7|144|1657|69|80.86|157.67|108.79|0.00|7506.51|5579.34|10879.23|0.00|0.00|7506.51|7506.51|1927.17| +2451464|40008|4927|11333|1778836|5986|5089|7|176|1657|13|67.07|125.42|91.55|0.00|1190.15|871.91|1630.46|47.60|0.00|1190.15|1237.75|318.24| +2451464|40008|14875|11333|1778836|5986|5089|7|208|1657|59|16.94|19.31|18.34|0.00|1082.06|999.46|1139.29|0.00|0.00|1082.06|1082.06|82.60| +2451851|63919|16193|53692|742196|5100|25957|8|300|1658|34|85.87|130.52|54.81|0.00|1863.54|2919.58|4437.68|111.81|0.00|1863.54|1975.35|-1056.04| +2451851|63919|16607|53692|742196|5100|25957|8|283|1658|43|60.38|72.45|55.06|0.00|2367.58|2596.34|3115.35|71.02|0.00|2367.58|2438.60|-228.76| +2451851|63919|11405|53692|742196|5100|25957|8|145|1658|90|64.60|68.47|51.35|0.00|4621.50|5814.00|6162.30|92.43|0.00|4621.50|4713.93|-1192.50| +2451851|63919|12416|53692|742196|5100|25957|8|233|1658|55|39.75|43.32|37.68|0.00|2072.40|2186.25|2382.60|62.17|0.00|2072.40|2134.57|-113.85| +2451851|63919|8041|53692|742196|5100|25957|8|159|1658|7|5.16|7.43|4.75|5.65|33.25|36.12|52.01|1.65|5.65|27.60|29.25|-8.52| +2451851|63919|14765|53692|742196|5100|25957|8|107|1658|40|20.06|38.51|26.95|0.00|1078.00|802.40|1540.40|32.34|0.00|1078.00|1110.34|275.60| +2451851|63919|6937|53692|742196|5100|25957|8|237|1658|90|67.70|88.01|14.08|747.64|1267.20|6093.00|7920.90|25.97|747.64|519.56|545.53|-5573.44| +2451851|63919|5903|53692|742196|5100|25957|8|23|1658|91|25.73|31.64|24.67|1571.47|2244.97|2341.43|2879.24|20.20|1571.47|673.50|693.70|-1667.93| +2451851|63919|15725|53692|742196|5100|25957|8|36|1658|67|17.99|21.22|17.40|0.00|1165.80|1205.33|1421.74|58.29|0.00|1165.80|1224.09|-39.53| +2451851|63919|12769|53692|742196|5100|25957|8|192|1658|92|89.52|142.33|81.12|4925.60|7463.04|8235.84|13094.36|50.74|4925.60|2537.44|2588.18|-5698.40| +2451851|63919|2000|53692|742196|5100|25957|8|164|1658|4|71.29|138.30|38.72|0.00|154.88|285.16|553.20|6.19|0.00|154.88|161.07|-130.28| +2451544|37493|9854|27058|1346411|3364|10364|4|206|1659|31|59.00|97.35|87.61|1711.02|2715.91|1829.00|3017.85|70.34|1711.02|1004.89|1075.23|-824.11| +2451544|37493|5648|||3364|10364|||1659||10.77|14.53|||646.47|1066.23|1438.47|||||-529.65| +2451544|37493|241|27058|1346411|3364|10364|4|175|1659|4|43.91|44.78|15.22|0.00|60.88|175.64|179.12|4.87|0.00|60.88|65.75|-114.76| +2451544|37493|9838|27058|1346411|3364|10364|4|102|1659|31|68.06|100.72|35.25|0.00|1092.75|2109.86|3122.32|98.34|0.00|1092.75|1191.09|-1017.11| +||13406|27058|1346411|||4||1659|4|46.84||17.13|0.00|68.52||||0.00|68.52|74.68|| +2451544|37493|11306|27058|1346411|3364|10364|4|148|1659|99|65.12|76.19|66.28|0.00|6561.72|6446.88|7542.81|524.93|0.00|6561.72|7086.65|114.84| +2451544|37493|6106|27058|1346411|3364|10364|4|29|1659|41|50.29|72.92|59.06|0.00|2421.46|2061.89|2989.72|217.93|0.00|2421.46|2639.39|359.57| +2451544|37493|13999|27058|1346411|3364|10364|4|60|1659|20|84.29|146.66|27.86|0.00|557.20|1685.80|2933.20|11.14|0.00|557.20|568.34|-1128.60| +2451544|37493|13946|27058|1346411|3364|10364|4|161|1659|100|98.09|134.38|30.90|0.00|3090.00|9809.00|13438.00|123.60|0.00|3090.00|3213.60|-6719.00| +2451203|51217|7262|39714|1676101|5641|44428|2|107|1660|56|69.95|102.12|34.72|408.30|1944.32|3917.20|5718.72|61.44|408.30|1536.02|1597.46|-2381.18| +2451203|51217|8233|39714|1676101|5641|44428|2|196|1660|54|15.37|20.13|1.81|0.00|97.74|829.98|1087.02|6.84|0.00|97.74|104.58|-732.24| +2451203|51217|271|39714|1676101|5641|44428|2|17|1660|20|65.08|115.84|50.96|0.00|1019.20|1301.60|2316.80|71.34|0.00|1019.20|1090.54|-282.40| +2451203||16591||1676101|5641|44428|2||1660|||||0.00|1094.40|5700.00||87.55|0.00|1094.40||| +2451203|51217|17510|39714|1676101|5641|44428|2|292|1660|86|86.93|91.27|7.30|0.00|627.80|7475.98|7849.22|43.94|0.00|627.80|671.74|-6848.18| +2451203|51217|2620|39714|1676101|5641|44428|2|217|1660|36|30.84|37.62|19.18|0.00|690.48|1110.24|1354.32|27.61|0.00|690.48|718.09|-419.76| +2451203|51217|1885|39714|1676101|5641|44428|2|86|1660|8|33.67|46.12|35.97|0.00|287.76|269.36|368.96|17.26|0.00|287.76|305.02|18.40| +2451203|51217|2272|39714|1676101|5641|44428|2|167|1660|19|60.54|69.01|66.24|0.00|1258.56|1150.26|1311.19|50.34|0.00|1258.56|1308.90|108.30| +2451203|51217|14275|39714|1676101|5641|44428|2|49|1660|46|2.93|4.18|2.00|47.84|92.00|134.78|192.28|0.44|47.84|44.16|44.60|-90.62| +2451203|51217|3148|39714|1676101|5641|44428|2|141|1660|50|92.69|101.03|89.91|0.00|4495.50|4634.50|5051.50|224.77|0.00|4495.50|4720.27|-139.00| +2451203|51217|100|39714|1676101|5641|44428|2|254|1660|40|69.09|82.90|14.92|0.00|596.80|2763.60|3316.00|17.90|0.00|596.80|614.70|-2166.80| +2451203|51217|17206|39714|1676101|5641|44428|2|26|1660|54|13.85|16.75|14.40|69.98|777.60|747.90|904.50|42.45|69.98|707.62|750.07|-40.28| +2451203|51217|4303|39714|1676101|5641|44428|2|136|1660|63|75.48|92.84|67.77|0.00|4269.51|4755.24|5848.92|384.25|0.00|4269.51|4653.76|-485.73| +2451203|51217|10748|39714|1676101|5641|44428|2|284|1660|74|10.13|16.00|10.40|0.00|769.60|749.62|1184.00|15.39|0.00|769.60|784.99|19.98| +2451203|51217|2971|39714|1676101|5641|44428|2|65|1660|71|67.65|113.65|7.95|0.00|564.45|4803.15|8069.15|5.64|0.00|564.45|570.09|-4238.70| +2451203|51217|11396|39714|1676101|5641|44428|2|231|1660|92|4.31|5.94|5.40|0.00|496.80|396.52|546.48|29.80|0.00|496.80|526.60|100.28| +2452562|55159|9816|82636|1194269|5102|43582|8|106|1661|79|30.37|47.07|0.47|25.61|37.13|2399.23|3718.53|0.34|25.61|11.52|11.86|-2387.71| +2452562|55159|5269|82636|1194269|5102|43582|8|143|1661|54|33.64|44.06|35.24|742.15|1902.96|1816.56|2379.24|104.47|742.15|1160.81|1265.28|-655.75| +2452562|55159|1158|82636|1194269|5102|43582|8|266|1661|98|1.82|2.78|1.58|99.09|154.84|178.36|272.44|1.11|99.09|55.75|56.86|-122.61| +2452562|55159|3463|82636|1194269|5102|43582|8|212|1661|9|10.68|12.49|12.36|0.00|111.24|96.12|112.41|7.78|0.00|111.24|119.02|15.12| +2452562|55159|13605|82636|1194269|5102|43582|8|85|1661|89|72.48|123.21|43.12|0.00|3837.68|6450.72|10965.69|115.13|0.00|3837.68|3952.81|-2613.04| +2452562|55159|17377|82636|1194269|5102|43582|8|157|1661|62|20.17|27.63|10.22|0.00|633.64|1250.54|1713.06|44.35|0.00|633.64|677.99|-616.90| +2452562|55159|10122|82636|1194269|5102|43582|8|132|1661|85|58.86|75.92|72.12|5578.48|6130.20|5003.10|6453.20|44.13|5578.48|551.72|595.85|-4451.38| +2452562|55159|4101|82636|1194269|5102|43582|8|112|1661|58|49.95|86.41|85.54|1339.55|4961.32|2897.10|5011.78|289.74|1339.55|3621.77|3911.51|724.67| +2452562|55159|6492|82636|1194269|5102|43582|8|93|1661|69|87.22|149.14|101.41|0.00|6997.29|6018.18|10290.66|139.94|0.00|6997.29|7137.23|979.11| +|55159|12309|82636|1194269||43582|8|85|1661|64|98.57|143.91|10.07|0.00||6308.48|9210.24|0.00|0.00|||| +2452562|55159|2325|82636|1194269|5102|43582|8|43|1661|90|36.27|72.17|9.38|0.00|844.20|3264.30|6495.30|33.76|0.00|844.20|877.96|-2420.10| +2452614|64105|4383|57676|907594|359|33477|10|104|1662|15|1.13|1.61|1.49|0.00|22.35|16.95|24.15|1.11|0.00|22.35|23.46|5.40| +2452614|64105|13023|57676|907594|359|33477|10|226|1662|8|77.55|102.36|101.33|0.00|810.64|620.40|818.88|72.95|0.00|810.64|883.59|190.24| +2452614|64105|8070|57676|907594|359|33477|10|133|1662|68|66.61|86.59|86.59|0.00|5888.12|4529.48|5888.12|294.40|0.00|5888.12|6182.52|1358.64| +2452614||9469|57676|907594||33477|||1662|27|68.63|118.72|111.59|0.00||1853.01||180.77|0.00|||| +2452614|64105|10003|57676|907594|359|33477|10|28|1662|18|22.72|25.44|18.31|0.00|329.58|408.96|457.92|13.18|0.00|329.58|342.76|-79.38| +2452614|64105|12618|57676|907594|359|33477|10|218|1662|51|36.03|47.55|23.77|0.00|1212.27|1837.53|2425.05|109.10|0.00|1212.27|1321.37|-625.26| +2452614|64105|5931|57676|907594|359|33477|10|125|1662|32|60.09|62.49|48.74|0.00|1559.68|1922.88|1999.68|109.17|0.00|1559.68|1668.85|-363.20| +2452614|64105|1932|57676|907594|359|33477|10|70|1662|64|61.75|108.06|64.83|0.00|4149.12|3952.00|6915.84|331.92|0.00|4149.12|4481.04|197.12| +2452614|64105|16863|57676|907594|359|33477|10|220|1662|78|29.95|45.82|0.45|0.00|35.10|2336.10|3573.96|1.40|0.00|35.10|36.50|-2301.00| +2452614|64105|5377|57676|907594|359|33477|10|213|1662|3|39.61|68.52|54.13|0.00|162.39|118.83|205.56|9.74|0.00|162.39|172.13|43.56| +2451531|41881|16594|51854|639636|571|26753|8|181|1663|23|10.42|15.73|5.82|100.39|133.86|239.66|361.79|2.34|100.39|33.47|35.81|-206.19| +2451531|41881|10916|51854|639636|571|26753|8|195|1663|76|27.09|28.71|14.92|532.94|1133.92|2058.84|2181.96|36.05|532.94|600.98|637.03|-1457.86| +2451531|41881|17366|51854|639636|571|26753|8|99|1663|25|83.01|142.77|82.80|41.40|2070.00|2075.25|3569.25|0.00|41.40|2028.60|2028.60|-46.65| +2451531|41881|5245|51854|639636|571|26753|8|34|1663|70|13.39|19.54|2.73|101.28|191.10|937.30|1367.80|1.79|101.28|89.82|91.61|-847.48| +2451531|41881|12703|51854|639636|571|26753|8|188|1663|44|9.62|11.35|5.67|0.00|249.48|423.28|499.40|9.97|0.00|249.48|259.45|-173.80| +2451531|41881|208|51854|639636|571|26753|8|5|1663|59|32.65|53.87|10.77|0.00|635.43|1926.35|3178.33|25.41|0.00|635.43|660.84|-1290.92| +2451531|41881|8317|51854|639636|571|26753|8|300|1663|42|7.94|11.35|7.94|0.00|333.48|333.48|476.70|6.66|0.00|333.48|340.14|0.00| +2451531|41881|2498|51854|639636|571|26753|8|71|1663|5|33.27|35.59|28.47|0.00|142.35|166.35|177.95|4.27|0.00|142.35|146.62|-24.00| +2451531|41881|16342|51854|639636|571|26753|8|116|1663|71|81.92|97.48|4.87|138.30|345.77|5816.32|6921.08|0.00|138.30|207.47|207.47|-5608.85| +2451531|41881|7348|51854|639636|571|26753|8|231|1663|47|47.43|85.84|68.67|0.00|3227.49|2229.21|4034.48|225.92|0.00|3227.49|3453.41|998.28| +2451531|41881|6128|51854|639636|571|26753|8|190|1663|26|5.86|6.38|4.01|0.00|104.26|152.36|165.88|7.29|0.00|104.26|111.55|-48.10| +2451531|41881|14402|51854|639636|571|26753|8|248|1663|82|50.85|78.30|41.49|0.00|3402.18|4169.70|6420.60|0.00|0.00|3402.18|3402.18|-767.52| +2451115|65724|13369|29430|140999|5815|44584|4|81|1664|91|64.03|75.55|41.55|3705.42|3781.05|5826.73|6875.05|0.00|3705.42|75.63|75.63|-5751.10| +2451115|65724|12008|29430|140999|5815|44584|4|15|1664|8|31.89|59.63|43.52|0.00|348.16|255.12|477.04|3.48|0.00|348.16|351.64|93.04| +2451115|65724|12508|29430|140999|5815|44584|4|297|1664|23|54.08|85.98|85.12|0.00|1957.76|1243.84|1977.54|137.04|0.00|1957.76|2094.80|713.92| +2451115|65724|16678|29430|140999|5815|44584|4|225|1664|49|26.51|41.35|40.93|0.00|2005.57|1298.99|2026.15|160.44|0.00|2005.57|2166.01|706.58| +2451115|65724|499|29430|140999|5815|44584|4|219|1664|47|32.15|41.15|32.92|0.00|1547.24|1511.05|1934.05|123.77|0.00|1547.24|1671.01|36.19| +2451115|65724|13352|29430|140999|5815|44584|4|227|1664|78|20.34|31.12|25.51|0.00|1989.78|1586.52|2427.36|179.08|0.00|1989.78|2168.86|403.26| +2451115|65724|13226|29430|140999|5815|44584|4|43|1664|47|90.43|133.83|30.78|0.00|1446.66|4250.21|6290.01|14.46|0.00|1446.66|1461.12|-2803.55| +2451115|65724|12700|29430|140999|5815|44584|4|237|1664|63|68.66|124.27|86.98|0.00|5479.74|4325.58|7829.01|383.58|0.00|5479.74|5863.32|1154.16| +2451115|65724|1174|29430|140999|5815|44584|4|109|1664|51|64.98|129.31|0.00|0.00|0.00|3313.98|6594.81|0.00|0.00|0.00|0.00|-3313.98| +2451115||14378||140999|5815|44584|4|32|1664|79||116.15|19.74|0.00|1559.46||9175.85||0.00||1699.81|-4070.08| +2451454|51168|6745|75783|330907|6031|36355|1|180|1665|22|64.90|69.44|59.02|0.00|1298.44|1427.80|1527.68|12.98|0.00|1298.44|1311.42|-129.36| +2451454|51168|8203|75783|330907|6031|36355|1|5|1665|7|79.03|83.77|24.29|134.32|170.03|553.21|586.39|2.14|134.32|35.71|37.85|-517.50| +2451454|51168|4204|75783|330907|6031|36355|1|170|1665|37|16.92|20.13|13.28|0.00|491.36|626.04|744.81|39.30|0.00|491.36|530.66|-134.68| +2451454|51168|14474|75783|330907|6031|36355|1|211|1665|98|47.73|93.07|14.89|0.00|1459.22|4677.54|9120.86|87.55|0.00|1459.22|1546.77|-3218.32| +2451454|51168|9415|75783|330907|6031|36355|1|90|1665|28|24.51|38.72|37.17|0.00|1040.76|686.28|1084.16|93.66|0.00|1040.76|1134.42|354.48| +2451454|51168|5821|75783|330907|6031|36355|1|153|1665|92|27.74|55.48|32.73|0.00|3011.16|2552.08|5104.16|150.55|0.00|3011.16|3161.71|459.08| +2451454|51168|5206|75783|330907|6031|36355|1|58|1665|20|19.92|39.24|26.68|165.41|533.60|398.40|784.80|33.13|165.41|368.19|401.32|-30.21| +2451454|51168|12595|75783|330907|6031|36355|1|226|1665|79|18.65|32.82|8.86|419.96|699.94|1473.35|2592.78|5.59|419.96|279.98|285.57|-1193.37| +2451454|51168|8107|75783|330907|6031|36355|1|291|1665|55|53.87|74.87|26.95|0.00|1482.25|2962.85|4117.85|103.75|0.00|1482.25|1586.00|-1480.60| +2451454|51168|4906|75783|330907|6031|36355|1|114|1665|79|61.35|85.89|54.11|0.00|4274.69|4846.65|6785.31|299.22|0.00|4274.69|4573.91|-571.96| +2451454|51168|16354|75783|330907|6031|36355|1|226|1665|44|32.30|45.22|15.37|0.00|676.28|1421.20|1989.68|27.05|0.00|676.28|703.33|-744.92| +2451454|51168|16636|75783|330907|6031|36355|1|27|1665|94|41.24|71.34|9.98|572.25|938.12|3876.56|6705.96|3.65|572.25|365.87|369.52|-3510.69| +2451454|51168|14197|75783|330907|6031|36355|1|128|1665|80|1.16|1.64|0.95|0.00|76.00|92.80|131.20|1.52|0.00|76.00|77.52|-16.80| +2451454|51168|12770|75783|330907|6031|36355|1|118|1665|26|55.07|96.37|5.78|0.00|150.28|1431.82|2505.62|0.00|0.00|150.28|150.28|-1281.54| +2451454|51168|8524|75783|330907|6031|36355|1|117|1665|98|41.95|49.50|42.57|3796.39|4171.86|4111.10|4851.00|0.00|3796.39|375.47|375.47|-3735.63| +2451974|65242|5225|3113|1098576|5239|12255|2|13|1666|86|73.98|96.91|13.56|0.00|1166.16|6362.28|8334.26|69.96|0.00|1166.16|1236.12|-5196.12| +||3853|||5239||2||1666|22|22.27|||5.55||489.94||0.00|5.55|0.83|0.83|-489.11| +2451974|65242|14801|3113|1098576|5239|12255|2|293|1666|33|25.52|33.43|16.38|167.56|540.54|842.16|1103.19|7.45|167.56|372.98|380.43|-469.18| +2451974|65242|9655|3113|1098576|5239|12255|2|19|1666|93|17.27|23.31|13.98|0.00|1300.14|1606.11|2167.83|13.00|0.00|1300.14|1313.14|-305.97| +2451974|65242|5511|3113|1098576|5239|12255|2|137|1666|52|84.09|89.97|12.59|497.55|654.68|4372.68|4678.44|0.00|497.55|157.13|157.13|-4215.55| +2451974|65242|9393|3113|1098576|5239|12255|2|184|1666|84|98.73|146.12|43.83|883.61|3681.72|8293.32|12274.08|55.96|883.61|2798.11|2854.07|-5495.21| +2451974|65242|15347|3113|1098576|5239|12255|2|252|1666|85|90.39|109.37|79.84|0.00|6786.40|7683.15|9296.45|67.86|0.00|6786.40|6854.26|-896.75| +2451974|65242|17413|3113|1098576|5239|12255|2|178|1666|34|11.52|23.04|2.53|0.00|86.02|391.68|783.36|2.58|0.00|86.02|88.60|-305.66| +2451974|65242|3159|3113|1098576|5239|12255|2|29|1666|14|37.63|40.26|19.72|0.00|276.08|526.82|563.64|16.56|0.00|276.08|292.64|-250.74| +2451974|65242|5519|3113|1098576|5239|12255|2|171|1666|31|53.17|89.85|26.95|0.00|835.45|1648.27|2785.35|75.19|0.00|835.45|910.64|-812.82| +2451974|65242|6339|3113|1098576|5239|12255|2|298|1666|98|3.67|7.00|4.69|0.00|459.62|359.66|686.00|0.00|0.00|459.62|459.62|99.96| +2451974|65242|11563|3113|1098576|5239|12255|2|59|1666|2|6.63|9.14|2.55|2.90|5.10|13.26|18.28|0.06|2.90|2.20|2.26|-11.06| +2451974|65242|14665|3113|1098576|5239|12255|2|118|1666|59|71.40|102.10|42.88|0.00|2529.92|4212.60|6023.90|25.29|0.00|2529.92|2555.21|-1682.68| +2452153|32647|13141|6968|1900452|3183|40667|2|19|1667|24|93.20|146.32|30.72|0.00|737.28|2236.80|3511.68|58.98|0.00|737.28|796.26|-1499.52| +2452153|32647|13079|6968|1900452|3183|40667|2|173|1667|59|15.17|18.96|0.00|0.00|0.00|895.03|1118.64|0.00|0.00|0.00|0.00|-895.03| +2452153|32647|2459|6968|1900452|3183|40667|2|166|1667|51|79.60|79.60|1.59|0.00|81.09|4059.60|4059.60|5.67|0.00|81.09|86.76|-3978.51| +2452153|32647|15349|6968||3183|40667||203|1667||74.10|139.30|||2281.50||3621.80||||2304.31|| +2452153|32647|3607|6968|1900452|3183|40667|2|182|1667|30|45.00|88.65|59.39|0.00|1781.70|1350.00|2659.50|142.53|0.00|1781.70|1924.23|431.70| +2452153|32647|10153|6968|1900452|3183|40667|2|43|1667|25|67.43|105.19|104.13|2160.69|2603.25|1685.75|2629.75|4.42|2160.69|442.56|446.98|-1243.19| +2452153|32647|14117|6968|1900452|3183|40667|2|238|1667|44|34.18|66.99|15.40|0.00|677.60|1503.92|2947.56|0.00|0.00|677.60|677.60|-826.32| +2452153|32647|16381|6968|1900452|3183|40667|2|223|1667|58|15.44|27.94|18.99|0.00|1101.42|895.52|1620.52|33.04|0.00|1101.42|1134.46|205.90| +2452153|32647|11819|6968|1900452|3183|40667|2|128|1667|38|20.38|31.18|27.43|0.00|1042.34|774.44|1184.84|20.84|0.00|1042.34|1063.18|267.90| +2452153|32647|17699|6968|1900452|3183|40667|2|180|1667|42|16.83|30.29|1.81|30.40|76.02|706.86|1272.18|3.64|30.40|45.62|49.26|-661.24| +2452153|32647|631|6968|1900452|3183|40667|2|99|1667|22|76.37|138.99|123.70|2612.54|2721.40|1680.14|3057.78|4.35|2612.54|108.86|113.21|-1571.28| +2452153|32647|12705|6968|1900452|3183|40667|2|196|1667|88|13.77|15.83|4.11|0.00|361.68|1211.76|1393.04|3.61|0.00|361.68|365.29|-850.08| +2452153|32647|11319|6968|1900452|3183|40667|2|47|1667|60|76.61|80.44|61.13|1577.15|3667.80|4596.60|4826.40|41.81|1577.15|2090.65|2132.46|-2505.95| +2450913|58557|12266|91387|709136|5285|47034|8|193|1668|21|38.30|74.68|5.97|0.00|125.37|804.30|1568.28|8.77|0.00|125.37|134.14|-678.93| +2450913|58557|230|91387|709136|5285|47034|8|26|1668|33|35.02|45.17|22.58|0.00|745.14|1155.66|1490.61|59.61|0.00|745.14|804.75|-410.52| +2450913|58557|9187|91387|709136|5285|47034|8|227|1668|33|70.57|110.79|35.45|0.00|1169.85|2328.81|3656.07|58.49|0.00|1169.85|1228.34|-1158.96| +2450913|58557|6476|91387|709136|5285|47034|8|96|1668|3|36.78|46.34|0.00|0.00|0.00|110.34|139.02|0.00|0.00|0.00|0.00|-110.34| +2450913|58557|1810|91387|709136|5285|47034|8|152|1668|82|4.63|4.72|4.05|0.00|332.10|379.66|387.04|23.24|0.00|332.10|355.34|-47.56| +2450913|58557|5170|91387|709136|5285|47034|8|84|1668|92|23.88|32.47|6.49|0.00|597.08|2196.96|2987.24|53.73|0.00|597.08|650.81|-1599.88| +2450913|58557|5998|91387|709136|5285|47034|8|201|1668|13|43.19|71.69|50.18|0.00|652.34|561.47|931.97|58.71|0.00|652.34|711.05|90.87| +2450913|58557|14368|91387|709136|5285|47034|8|205|1668|39|36.38|42.92|32.19|0.00|1255.41|1418.82|1673.88|112.98|0.00|1255.41|1368.39|-163.41| +2451142|49079|15349|21620|1293553|3086|14619|10|218|1669|91|97.41|121.76|57.22|0.00|5207.02|8864.31|11080.16|156.21|0.00|5207.02|5363.23|-3657.29| +2451142|49079|3607|21620|1293553|3086|14619|10|292|1669|60|59.73|118.86|98.65|0.00|5919.00|3583.80|7131.60|236.76|0.00|5919.00|6155.76|2335.20| +2451142|49079|10153|21620|1293553|3086|14619|10|72|1669|96|25.67|50.05|40.54|0.00|3891.84|2464.32|4804.80|0.00|0.00|3891.84|3891.84|1427.52| +2451142|49079|14116|21620|1293553|3086|14619|10|259|1669|40|4.64|8.86|1.15|0.00|46.00|185.60|354.40|0.00|0.00|46.00|46.00|-139.60| +2451142|49079|16381|21620|1293553|3086|14619|10|164|1669|53|36.63|56.41|56.41|0.00|2989.73|1941.39|2989.73|269.07|0.00|2989.73|3258.80|1048.34| +2451142|49079|11818|21620|1293553|3086|14619|10|34|1669|52|4.02|4.02|1.44|0.00|74.88|209.04|209.04|3.74|0.00|74.88|78.62|-134.16| +||17698||||14619||105|1669|75||78.78||0.00||||192.00|0.00|3840.00|4032.00|| +2451142|49079|631|21620|1293553|3086|14619|10|50|1669|15|60.27|87.99|2.63|5.12|39.45|904.05|1319.85|0.68|5.12|34.33|35.01|-869.72| +2451142|49079|12704|21620|1293553|3086|14619|10|70|1669|23|63.69|82.16|61.62|0.00|1417.26|1464.87|1889.68|127.55|0.00|1417.26|1544.81|-47.61| +2451142|49079|11318|21620|1293553|3086|14619|10|211|1669|71|8.40|16.21|12.64|170.51|897.44|596.40|1150.91|7.26|170.51|726.93|734.19|130.53| +2451142|49079|13051|21620|1293553|3086|14619|10|185|1669|3|96.86|191.78|180.27|0.00|540.81|290.58|575.34|16.22|0.00|540.81|557.03|250.23| +2451142|49079|340|21620|1293553|3086|14619|10|227|1669|79|11.70|22.69|8.84|0.00|698.36|924.30|1792.51|34.91|0.00|698.36|733.27|-225.94| +2451142|49079|16810|21620|1293553|3086|14619|10|27|1669|76|38.17|44.27|31.87|0.00|2422.12|2900.92|3364.52|217.99|0.00|2422.12|2640.11|-478.80| +2451142|49079|11503|21620|1293553|3086|14619|10|157|1669|95|11.20|11.20|7.16|0.00|680.20|1064.00|1064.00|27.20|0.00|680.20|707.40|-383.80| +2451142|49079|10090|21620|1293553|3086|14619|10|109|1669|74|76.25|148.68|87.72|0.00|6491.28|5642.50|11002.32|389.47|0.00|6491.28|6880.75|848.78| +2451142|49079|17029|21620|1293553|3086|14619|10|71|1669|13|36.88|42.78|2.99|0.00|38.87|479.44|556.14|1.16|0.00|38.87|40.03|-440.57| +2452605|38559|4809|84125|49100|828|13403|4|287|1670|76|35.24|65.54|20.97|0.00|1593.72|2678.24|4981.04|0.00|0.00|1593.72|1593.72|-1084.52| +2452605|38559|5799|84125|49100|828|13403|4|126|1670|67|49.09|79.03|7.11|0.00|476.37|3289.03|5295.01|14.29|0.00|476.37|490.66|-2812.66| +2452605|38559|17659|84125|49100|828|13403|4|16|1670|48|62.52|117.53|19.98|0.00|959.04|3000.96|5641.44|38.36|0.00|959.04|997.40|-2041.92| +2452605|38559|15001|84125|49100|828|13403|4|249|1670|22|21.96|27.88|26.48|0.00|582.56|483.12|613.36|17.47|0.00|582.56|600.03|99.44| +||11005|84125|49100|||4||1670|60|22.63|||||||58.17|||785.37|| +2452605|38559|15645|84125|49100|828|13403|4|229|1670|45|90.25|154.32|115.74|0.00|5208.30|4061.25|6944.40|312.49|0.00|5208.30|5520.79|1147.05| +2452605|38559|16422|84125|49100|828|13403|4|148|1670|24|89.83|179.66|127.55|0.00|3061.20|2155.92|4311.84|153.06|0.00|3061.20|3214.26|905.28| +2452605|38559|2095|84125|49100|828|13403|4|61|1670|9|5.92|8.99|2.24|0.00|20.16|53.28|80.91|0.80|0.00|20.16|20.96|-33.12| +2452605|38559|7548|84125|49100|828|13403|4|133|1670|45|40.75|54.19|53.10|0.00|2389.50|1833.75|2438.55|95.58|0.00|2389.50|2485.08|555.75| +2452605|38559|2887|84125|49100|828||||1670|15|94.65|100.32|57.18|0.00||1419.75|1504.80||0.00|857.70|900.58|-562.05| +2452605|38559|1116|84125|49100|828|13403|4|60|1670|30|2.16|2.65|1.11|0.00|33.30|64.80|79.50|0.00|0.00|33.30|33.30|-31.50| +2452605|38559|11349|84125|49100|828|13403|4|221|1670|78|44.20|57.90|52.11|0.00|4064.58|3447.60|4516.20|365.81|0.00|4064.58|4430.39|616.98| +2452605|38559|17562|84125|49100|828|13403|4|142|1670|8|52.70|89.59|18.81|0.00|150.48|421.60|716.72|13.54|0.00|150.48|164.02|-271.12| +|38559|17298|84125|49100|828|13403|4||1670||||20.08||983.92|1006.95|1278.41||||983.92|| +2452605|38559|13221|84125|49100|828|13403|4|176|1670|91|10.89|13.83|12.58|0.00|1144.78|990.99|1258.53|0.00|0.00|1144.78|1144.78|153.79| +2452605|38559|13795|84125|49100|828|13403|4|153|1670|14|40.97|54.08|10.27|0.00|143.78|573.58|757.12|10.06|0.00|143.78|153.84|-429.80| +2452234|69993|10199|57871|593612|1353|13094|1|261|1671|29|76.61|99.59|13.94|0.00|404.26|2221.69|2888.11|28.29|0.00|404.26|432.55|-1817.43| +2452234|69993|9931|57871|593612|1353|13094|1|236|1671|95|79.42|129.45|98.38|0.00|9346.10|7544.90|12297.75|373.84|0.00|9346.10|9719.94|1801.20| +2452234|69993|14497|57871|593612|1353|13094|1|198|1671|38|43.62|79.82|0.00|0.00|0.00|1657.56|3033.16|0.00|0.00|0.00|0.00|-1657.56| +2452234|69993|9635|57871|593612|1353|13094|1|225|1671|43|75.75|136.35|128.16|0.00|5510.88|3257.25|5863.05|110.21|0.00|5510.88|5621.09|2253.63| +2452234|69993|8935|57871|593612|1353|13094|1|264|1671|89|86.71|88.44|81.36|0.00|7241.04|7717.19|7871.16|434.46|0.00|7241.04|7675.50|-476.15| +2452234|69993|5685|57871|593612|1353|13094|1|71|1671|40|93.82|103.20|62.95|0.00|2518.00|3752.80|4128.00|25.18|0.00|2518.00|2543.18|-1234.80| +2452234|69993|17703|57871|593612|1353|13094|1|22|1671|73|15.49|17.34|11.79|0.00|860.67|1130.77|1265.82|60.24|0.00|860.67|920.91|-270.10| +2452234|69993|5695|57871|593612|1353|13094|1|206|1671|15|44.66|88.42|22.98|0.00|344.70|669.90|1326.30|17.23|0.00|344.70|361.93|-325.20| +2452234|69993|9967|57871|593612|1353|13094|1|193|1671|54|19.81|32.88|11.50|0.00|621.00|1069.74|1775.52|37.26|0.00|621.00|658.26|-448.74| +2452234|69993|3901|57871|593612|1353|13094|1|232|1671|20|63.57|85.18|40.88|0.00|817.60|1271.40|1703.60|65.40|0.00|817.60|883.00|-453.80| +||8189||593612|1353|13094||273|1671|83|12.92|||0.00|268.09|1072.36||5.36|0.00|268.09|273.45|-804.27| +2452234|69993|325|57871|593612|1353|13094|1|274|1671|88|90.72|121.56|25.52|0.00|2245.76|7983.36|10697.28|202.11|0.00|2245.76|2447.87|-5737.60| +2452234|69993|13385|57871|593612|1353|13094|1|32|1671|89|17.61|28.17|17.74|0.00|1578.86|1567.29|2507.13|126.30|0.00|1578.86|1705.16|11.57| +2452234|69993|11305|57871|593612|1353|13094|1|203|1671|86|66.22|72.17|62.06|0.00|5337.16|5694.92|6206.62|373.60|0.00|5337.16|5710.76|-357.76| +2451931|43641|119|31259|19291|923|24155|1|196|1672|32|47.44|72.58|34.11|0.00|1091.52|1518.08|2322.56|43.66|0.00|1091.52|1135.18|-426.56| +2451931|43641|14313|31259|19291|923|24155|1|257|1672|17|77.65|139.77|138.37|0.00|2352.29|1320.05|2376.09|0.00|0.00|2352.29|2352.29|1032.24| +2451931|43641|6547|31259|19291|923|24155|1|259|1672|23|17.86|17.86|7.14|0.00|164.22|410.78|410.78|6.56|0.00|164.22|170.78|-246.56| +2451931|43641|17315|31259|19291|923|24155|1|135|1672|51|4.35|5.56|3.50|0.00|178.50|221.85|283.56|10.71|0.00|178.50|189.21|-43.35| +2451931|43641|10207|31259|19291|923|24155|1|117|1672|48|76.25|93.78|14.06|161.97|674.88|3660.00|4501.44|41.03|161.97|512.91|553.94|-3147.09| +2451931|43641|1327|31259|19291|923|24155|1|190|1672|24|31.22|35.59|16.37|0.00|392.88|749.28|854.16|11.78|0.00|392.88|404.66|-356.40| +2451931|43641|8227|31259|19291|923|24155|1|37|1672|16|58.44|95.84|65.17|0.00|1042.72|935.04|1533.44|83.41|0.00|1042.72|1126.13|107.68| +2451931|43641|7709|31259|19291|923|24155|1|240|1672|92|83.43|107.62|23.67|0.00|2177.64|7675.56|9901.04|130.65|0.00|2177.64|2308.29|-5497.92| +2451931|43641|11957|31259|19291|923|24155|1|14|1672|93|83.07|88.05|25.53|0.00|2374.29|7725.51|8188.65|23.74|0.00|2374.29|2398.03|-5351.22| +2451931|43641|7101|31259|19291|923|24155|1|19|1672|30|26.31|35.51|35.51|0.00|1065.30|789.30|1065.30|95.87|0.00|1065.30|1161.17|276.00| +2451931|43641|11633|31259|19291|923|24155|1|242|1672|77|14.18|14.18|2.26|0.00|174.02|1091.86|1091.86|13.92|0.00|174.02|187.94|-917.84| +2451931|43641|15485|31259|19291|923|24155|1|200|1672|96|3.06|4.49|0.98|0.00|94.08|293.76|431.04|0.00|0.00|94.08|94.08|-199.68| +2451931|43641|12419|31259||923||||1672|72|74.24||126.95||9140.40||||||9506.01|| +2451931|43641|13307|31259|19291|923|24155|1|118|1672|25|98.51|161.55|38.77|0.00|969.25|2462.75|4038.75|9.69|0.00|969.25|978.94|-1493.50| +2451931|43641|17067|31259|19291|923|24155|1|164|1672|32|95.23|112.37|39.32|754.94|1258.24|3047.36|3595.84|45.29|754.94|503.30|548.59|-2544.06| +2451931|43641|10323|31259|19291|923|24155|1|28|1672|24|2.48|2.95|0.91|0.00|21.84|59.52|70.80|0.87|0.00|21.84|22.71|-37.68| +2451077|52527|10423|67208|399677|946|38615|7|279|1673|74|51.15|58.82|47.05|974.87|3481.70|3785.10|4352.68|100.27|974.87|2506.83|2607.10|-1278.27| +2451077|52527|2332|67208|399677|946|38615|7|153|1673|44|22.17|38.57|34.71|0.00|1527.24|975.48|1697.08|61.08|0.00|1527.24|1588.32|551.76| +2451077|52527|7429|67208|399677||||173|1673|22|||34.18||751.96||2892.56|15.03|||766.99|| +2451077|52527|8182|67208|399677|946|38615|7|200|1673|72|97.50|182.32|147.67|0.00|10632.24|7020.00|13127.04|637.93|0.00|10632.24|11270.17|3612.24| +2451077|52527|2452|67208|399677|946|38615|7|126|1673|4|30.24|57.45|39.06|0.00|156.24|120.96|229.80|1.56|0.00|156.24|157.80|35.28| +2451077|52527|13645|67208|399677|946|38615|7|272|1673|56|88.39|173.24|97.01|0.00|5432.56|4949.84|9701.44|325.95|0.00|5432.56|5758.51|482.72| +2451077|52527|722|67208|399677|946|38615|7|193|1673|1|28.68|34.70|27.06|7.30|27.06|28.68|34.70|0.59|7.30|19.76|20.35|-8.92| +2451077|52527|3046|67208|399677|946|38615|7|259|1673|86|61.16|91.74|83.48|358.96|7179.28|5259.76|7889.64|68.20|358.96|6820.32|6888.52|1560.56| +2451077|52527|2251|67208|399677|946|38615|7|228|1673|59|35.76|68.65|24.02|113.37|1417.18|2109.84|4050.35|52.15|113.37|1303.81|1355.96|-806.03| +2451077|52527|17056|67208|399677|946|38615|7|110|1673|66|31.15|39.87|18.34|181.56|1210.44|2055.90|2631.42|20.57|181.56|1028.88|1049.45|-1027.02| +2451077|52527|10975|67208|399677|946|38615|7|299|1673|30|38.59|57.49|23.57|0.00|707.10|1157.70|1724.70|49.49|0.00|707.10|756.59|-450.60| +2451077|52527|1892|67208|399677|946|38615|7|85|1673|54|64.75|69.93|43.35|280.90|2340.90|3496.50|3776.22|61.80|280.90|2060.00|2121.80|-1436.50| +2451077|52527|15436|67208|399677|946|38615|7|184|1673|46|62.36|86.68|15.60|0.00|717.60|2868.56|3987.28|28.70|0.00|717.60|746.30|-2150.96| +2451077|52527|3316|67208|399677|946|38615|7|24|1673|86|62.35|109.73|73.51|0.00|6321.86|5362.10|9436.78|379.31|0.00|6321.86|6701.17|959.76| +2451437|48988|15692|69148|301417|3733|23202|1|284|1674|25|95.74|170.41|66.45|0.00|1661.25|2393.50|4260.25|149.51|0.00|1661.25|1810.76|-732.25| +2451437|48988|11674|69148|301417|3733|23202|1|260|1674|85|26.78|28.11|1.12|0.00|95.20|2276.30|2389.35|6.66|0.00|95.20|101.86|-2181.10| +2451437|48988|3518|69148|301417|3733|23202|1|127|1674|66|78.03|151.37|43.89|1680.10|2896.74|5149.98|9990.42|36.49|1680.10|1216.64|1253.13|-3933.34| +2451437|48988|5707|69148|301417|3733|23202|1|12|1674|26|55.52|107.70|22.61|0.00|587.86|1443.52|2800.20|52.90|0.00|587.86|640.76|-855.66| +2451437|48988|15100|69148|301417|3733|23202|1|90|1674|38|78.43|89.41|82.25|0.00|3125.50|2980.34|3397.58|0.00|0.00|3125.50|3125.50|145.16| +2451437|48988|16408|69148|301417|3733|23202|1|128|1674|97|52.43|76.02|27.36|0.00|2653.92|5085.71|7373.94|159.23|0.00|2653.92|2813.15|-2431.79| +2451437|48988|14185|69148|301417|3733|23202|1|186|1674|96|71.39|137.78|45.46|2007.51|4364.16|6853.44|13226.88|23.56|2007.51|2356.65|2380.21|-4496.79| +2451437|48988|10544|69148|301417|3733|23202|1|8|1674|89|16.17|16.97|15.10|671.95|1343.90|1439.13|1510.33|33.59|671.95|671.95|705.54|-767.18| +2451437|48988|6074|69148|301417|3733|23202|1|258|1674|70|76.82|108.31|30.32|0.00|2122.40|5377.40|7581.70|106.12|0.00|2122.40|2228.52|-3255.00| +2451437|48988|2714|69148|301417|3733|23202|1|118|1674|75|38.51|75.09|13.51|0.00|1013.25|2888.25|5631.75|40.53|0.00|1013.25|1053.78|-1875.00| +2451437|48988|3412|69148|301417|3733|23202|1|18|1674|36|39.62|64.58|27.76|0.00|999.36|1426.32|2324.88|69.95|0.00|999.36|1069.31|-426.96| +2451437|48988|14924|69148|301417|3733|23202|1|213|1674|46|48.86|76.71|52.16|0.00|2399.36|2247.56|3528.66|47.98|0.00|2399.36|2447.34|151.80| +2451437|48988|4063|69148|301417|3733|23202|1|139|1674|99|81.42|118.87|72.51|3302.10|7178.49|8060.58|11768.13|232.58|3302.10|3876.39|4108.97|-4184.19| +2451437|48988|6328|69148|301417|3733|23202|1|197|1674|49|50.14|96.26|63.53|0.00|3112.97|2456.86|4716.74|124.51|0.00|3112.97|3237.48|656.11| +2451976|53904|8911|45105|1059236|3767|33498|4|298|1675|95|48.10|77.44|32.52|1266.65|3089.40|4569.50|7356.80|127.59|1266.65|1822.75|1950.34|-2746.75| +2451976|53904|7189|45105|1059236|3767|33498|4|272|1675|20|59.09|72.68|37.79|0.00|755.80|1181.80|1453.60|30.23|0.00|755.80|786.03|-426.00| +2451976|53904|10725|45105|1059236|3767|33498|4|114|1675|70|79.95|118.32|114.77|0.00|8033.90|5596.50|8282.40|482.03|0.00|8033.90|8515.93|2437.40| +2451976|53904|17953|45105|1059236|3767|33498|4|280|1675|39|49.81|60.76|27.94|0.00|1089.66|1942.59|2369.64|54.48|0.00|1089.66|1144.14|-852.93| +2451976|53904|4841|45105|1059236|3767|33498|4|172|1675|99|60.83|110.10|19.81|0.00|1961.19|6022.17|10899.90|117.67|0.00|1961.19|2078.86|-4060.98| +2451976|53904|5769|45105|1059236|3767|33498|4|69|1675|70|8.55|11.88|4.51|0.00|315.70|598.50|831.60|3.15|0.00|315.70|318.85|-282.80| +||6915||1059236|3767|33498|4|268|1675|73|1.56|||0.00|53.29||206.59|4.79|0.00||58.08|| +2451976|53904|14613|45105|1059236|3767|33498|4|146|1675|34|30.66|42.31|13.96|132.89|474.64|1042.44|1438.54|3.41|132.89|341.75|345.16|-700.69| +2451976|53904|3655|45105|1059236|3767|33498|4|48|1675|67|98.65|185.46|22.25|0.00|1490.75|6609.55|12425.82|0.00|0.00|1490.75|1490.75|-5118.80| +2451976|53904|15955|45105|1059236|3767|33498|4|51|1675|67|70.13|105.89|81.53|0.00|5462.51|4698.71|7094.63|109.25|0.00|5462.51|5571.76|763.80| +2451976|53904|14809|45105|1059236|3767|33498|4|283|1675|20|78.61|88.82|20.42|0.00|408.40|1572.20|1776.40|0.00|0.00|408.40|408.40|-1163.80| +2451976|53904|2799|45105|1059236|3767|33498|4|122|1675|97|80.87|129.39|2.58|0.00|250.26|7844.39|12550.83|20.02|0.00|250.26|270.28|-7594.13| +2451976|53904|16655|45105|1059236|3767|33498|4|264|1675|42|67.66|73.07|59.18|2212.14|2485.56|2841.72|3068.94|16.40|2212.14|273.42|289.82|-2568.30| +2451976|53904|3267|45105|1059236|3767|33498|4|198|1675|2|34.20|45.48|24.10|0.00|48.20|68.40|90.96|0.48|0.00|48.20|48.68|-20.20| +2452493|66702|6672|65507|257072|3837|26155|2|226|1676|68|63.25|110.68|92.97|1959.80|6321.96|4301.00|7526.24|130.86|1959.80|4362.16|4493.02|61.16| +2452493|66702|13086|65507|257072|3837|26155|2|50|1676|56|3.57|4.56|2.23|0.00|124.88|199.92|255.36|11.23|0.00|124.88|136.11|-75.04| +2452493|66702|117|65507|257072|3837|26155|2|63|1676|70|6.62|9.06|4.98|0.00|348.60|463.40|634.20|13.94|0.00|348.60|362.54|-114.80| +2452493|66702|12822|65507|257072|3837|26155|2|237|1676|24|7.59|8.34|5.50|0.00|132.00|182.16|200.16|7.92|0.00|132.00|139.92|-50.16| +|66702|13596|65507|257072|3837||2|9|1676||24.26||28.20|1133.07|1156.20|||1.85|1133.07|||-971.53| +2452493|66702|879|65507|257072|3837|26155|2|158|1676|76|18.32|22.35|14.08|0.00|1070.08|1392.32|1698.60|32.10|0.00|1070.08|1102.18|-322.24| +2452493|66702|601|65507|257072|3837|26155|2|273|1676|18|87.98|142.52|138.24|0.00|2488.32|1583.64|2565.36|199.06|0.00|2488.32|2687.38|904.68| +2452493|66702|7755|65507|257072|3837|26155|2|252|1676|19|78.04|124.86|62.43|0.00|1186.17|1482.76|2372.34|106.75|0.00|1186.17|1292.92|-296.59| +2452493|66702|10663|65507|257072|3837|26155|2|74|1676|66|65.10|102.85|73.02|0.00|4819.32|4296.60|6788.10|192.77|0.00|4819.32|5012.09|522.72| +2452493|66702|10495|65507|257072|3837|26155|2|70|1676|17|21.64|39.16|37.20|505.92|632.40|367.88|665.72|11.38|505.92|126.48|137.86|-241.40| +2452493|66702|6594|65507|257072|3837|26155|2|8|1676|2|98.12|108.91|89.30|0.00|178.60|196.24|217.82|10.71|0.00|178.60|189.31|-17.64| +2452493|66702|17649|65507|257072|3837|26155|2|132|1676|99|96.88|133.69|40.10|1071.87|3969.90|9591.12|13235.31|231.84|1071.87|2898.03|3129.87|-6693.09| +2452493|66702|3732|65507|257072|3837|26155|2|42|1676|26|96.40|164.84|117.03|0.00|3042.78|2506.40|4285.84|0.00|0.00|3042.78|3042.78|536.38| +2452493|66702|12183|65507|257072|3837|26155|2|2|1676|80|79.96|157.52|67.73|0.00|5418.40|6396.80|12601.60|108.36|0.00|5418.40|5526.76|-978.40| +|30534|9133|73279|||||289|1677|99|||17.38|0.00|1720.62|||51.61|0.00|1720.62||| +2452618|30534|7857|73279|1027188|5897|9006|1|95|1677|90|30.17|52.19|19.31|0.00|1737.90|2715.30|4697.10|52.13|0.00|1737.90|1790.03|-977.40| +2452618|30534|5061|73279|1027188|5897|9006|1|164|1677|75|99.47|162.13|81.06|0.00|6079.50|7460.25|12159.75|60.79|0.00|6079.50|6140.29|-1380.75| +2452618|30534|4585|73279|1027188|5897|9006|1|235|1677|11|47.54|56.57|42.42|0.00|466.62|522.94|622.27|23.33|0.00|466.62|489.95|-56.32| +2452618|30534|7729|73279|1027188|5897|9006|1|129|1677|83|74.36|89.97|10.79|0.00|895.57|6171.88|7467.51|26.86|0.00|895.57|922.43|-5276.31| +2452618|30534|7411|73279|1027188|5897|9006|1|17|1677|7|14.10|27.77|16.10|0.00|112.70|98.70|194.39|4.50|0.00|112.70|117.20|14.00| +2452618|30534|13392|73279|1027188|5897|9006|1|267|1677|1|83.03|112.92|108.40|0.00|108.40|83.03|112.92|9.75|0.00|108.40|118.15|25.37| +2452618|30534|16020|73279|1027188|5897|9006|1|300|1677|46|59.21|79.34|27.76|0.00|1276.96|2723.66|3649.64|51.07|0.00|1276.96|1328.03|-1446.70| +2451522|36554|14701|16099|498258|3617|46576|7|42|1678|58|51.85|69.47|47.93|2613.14|2779.94|3007.30|4029.26|15.01|2613.14|166.80|181.81|-2840.50| +2451522|36554|17420|16099|498258|3617|46576|7|10|1678|31|41.80|48.90|27.87|0.00|863.97|1295.80|1515.90|17.27|0.00|863.97|881.24|-431.83| +2451522|36554|9436|16099|498258|3617|46576|7|248|1678|46|24.78|32.70|28.44|0.00|1308.24|1139.88|1504.20|65.41|0.00|1308.24|1373.65|168.36| +2451522|36554|7654|16099|498258|3617|46576|7|259|1678|19|10.73|15.77|8.04|0.00|152.76|203.87|299.63|4.58|0.00|152.76|157.34|-51.11| +2451522|36554|12223|16099|498258|3617|46576|7|113|1678|66|64.47|65.75|21.69|0.00|1431.54|4255.02|4339.50|42.94|0.00|1431.54|1474.48|-2823.48| +2451522|36554|15085|16099|498258|3617|46576|7|209|1678|20|46.41|83.53|48.44|0.00|968.80|928.20|1670.60|58.12|0.00|968.80|1026.92|40.60| +2451522|36554|3740|16099|498258|3617|46576|7|17|1678|11|94.64|189.28|17.03|0.00|187.33|1041.04|2082.08|7.49|0.00|187.33|194.82|-853.71| +2451522|36554|1363|16099|498258|3617|46576|7|34|1678|84|1.42|1.51|0.92|0.00|77.28|119.28|126.84|1.54|0.00|77.28|78.82|-42.00| +2451522|36554|12068|16099|498258|3617|46576|7|44|1678|3|66.43|125.55|11.29|0.00|33.87|199.29|376.65|0.00|0.00|33.87|33.87|-165.42| +2451522|36554|13340|16099|498258|3617|46576|7|7|1678|21|9.02|13.80|13.66|0.00|286.86|189.42|289.80|5.73|0.00|286.86|292.59|97.44| +2451522|36554|2930|16099|498258|3617|46576|7|108|1678|78|66.58|119.17|71.50|0.00|5577.00|5193.24|9295.26|390.39|0.00|5577.00|5967.39|383.76| +2451522|36554|15782|16099|498258|3617|46576|7|11|1678|44|7.16|12.88|10.17|0.00|447.48|315.04|566.72|31.32|0.00|447.48|478.80|132.44| +2451522|36554|20|16099|498258|3617|46576|7|32|1678|51|22.87|41.16|11.93|0.00|608.43|1166.37|2099.16|24.33|0.00|608.43|632.76|-557.94| +2451522|36554|15626|16099|498258|3617|46576|7|156|1678|8|4.28|6.29|4.90|0.00|39.20|34.24|50.32|3.13|0.00|39.20|42.33|4.96| +2451522|36554|9458|16099|498258|3617|46576|7|118|1678|93|24.17|39.88|33.10|0.00|3078.30|2247.81|3708.84|30.78|0.00|3078.30|3109.08|830.49| +2452200|53099|17113|1087|1202525|4763|12601|8|45|1679|9|22.26|28.93|4.62|0.00|41.58|200.34|260.37|3.74|0.00|41.58|45.32|-158.76| +2452200|53099|1517|1087|1202525|4763|12601|8|166|1679|65|97.52|115.07|55.23|2656.56|3589.95|6338.80|7479.55|0.00|2656.56|933.39|933.39|-5405.41| +2452200|53099|14371|1087|1202525|4763|12601|8|105|1679|62|50.11|79.67|49.39|0.00|3062.18|3106.82|4939.54|30.62|0.00|3062.18|3092.80|-44.64| +2452200|53099|2149|1087|1202525|4763|12601|8|206|1679|64|75.61|77.12|53.98|0.00|3454.72|4839.04|4935.68|103.64|0.00|3454.72|3558.36|-1384.32| +2452200|53099|15255|1087|1202525|4763|12601|8|44|1679|21|76.81|89.86|41.33|0.00|867.93|1613.01|1887.06|69.43|0.00|867.93|937.36|-745.08| +|53099|15055|1087|1202525|4763|12601|8|231|1679|||95.94|15.35|0.00|1274.05||||0.00|||| +2452200|53099|6505|1087|1202525|4763|12601|8|244|1679|58|20.36|23.41|14.51|0.00|841.58|1180.88|1357.78|16.83|0.00|841.58|858.41|-339.30| +2452200|53099|14235|1087|1202525|4763|12601|8|7|1679|4|5.75|6.26|5.32|0.00|21.28|23.00|25.04|1.48|0.00|21.28|22.76|-1.72| +2452200|53099|10237|1087|1202525|4763|12601|8|205|1679|49|61.38|109.25|24.03|0.00|1177.47|3007.62|5353.25|82.42|0.00|1177.47|1259.89|-1830.15| +2452200|53099|2523|1087|1202525|4763|12601|8|121|1679|54|62.93|82.43|16.48|0.00|889.92|3398.22|4451.22|80.09|0.00|889.92|970.01|-2508.30| +2452200|53099|8585|1087|1202525|4763|12601|8|192|1679|77|59.98|118.16|75.62|2096.18|5822.74|4618.46|9098.32|335.39|2096.18|3726.56|4061.95|-891.90| +2452200|53099|2091|1087|1202525|4763|12601|8|245|1679|29|19.43|26.81|20.10|0.00|582.90|563.47|777.49|17.48|0.00|582.90|600.38|19.43| +2452200|53099|4019|1087|1202525|4763|12601|8|133|1679|84|13.38|23.81|12.85|0.00|1079.40|1123.92|2000.04|21.58|0.00|1079.40|1100.98|-44.52| +2452200|53099|9223|1087|1202525|4763|12601|8|25|1679|94|53.07|90.21|42.39|0.00|3984.66|4988.58|8479.74|119.53|0.00|3984.66|4104.19|-1003.92| +2452200|53099|5371|1087|1202525|4763|12601|8|124|1679|55|44.92|63.78|52.29|0.00|2875.95|2470.60|3507.90|57.51|0.00|2875.95|2933.46|405.35| +2451104|66648|5197|67701|1459422|2343|38464|1|54|1680|80|50.53|66.69|62.68|0.00|5014.40|4042.40|5335.20|200.57|0.00|5014.40|5214.97|972.00| +2451104|66648|2312|67701|1459422|2343|38464|1|149|1680|37|9.63|16.37|5.23|191.57|193.51|356.31|605.69|0.01|191.57|1.94|1.95|-354.37| +2451104|66648|13814|67701|1459422|2343|38464|1|243|1680|22|93.04|107.92|84.17|0.00|1851.74|2046.88|2374.24|166.65|0.00|1851.74|2018.39|-195.14| +2451104|66648|5449|67701|1459422|2343|38464|1|287|1680|59|78.59|84.09|16.81|0.00|991.79|4636.81|4961.31|19.83|0.00|991.79|1011.62|-3645.02| +2451104||1682|67701|1459422||38464|1|208|1680|17||||184.59|||||184.59||245.92|| +2451104|66648|6631|67701|1459422|2343|38464|1|186|1680|8|95.62|129.08|78.73|0.00|629.84|764.96|1032.64|31.49|0.00|629.84|661.33|-135.12| +2451104||4666|67701||||1|100|1680|84|47.85|94.26||0.00|||||0.00||2445.92|-1644.72| +2451104|66648|10496|67701|1459422|2343|38464|1|60|1680|48|43.05|50.36|35.75|0.00|1716.00|2066.40|2417.28|34.32|0.00|1716.00|1750.32|-350.40| +2451104|66648|8398|67701|1459422|2343|38464|1|30|1680|58|11.01|11.89|4.87|0.00|282.46|638.58|689.62|0.00|0.00|282.46|282.46|-356.12| +2451104|66648|4141|67701|1459422|2343|38464|1|295|1680|88|8.37|10.62|7.85|158.88|690.80|736.56|934.56|5.31|158.88|531.92|537.23|-204.64| +2451104|66648|295|||||1|220|1680|50|97.12|178.70||0.00|1518.50|4856.00||91.11|0.00|1518.50|1609.61|| +2451104|66648|12284|67701|1459422|2343|38464|1|18|1680|11|48.20|56.87|55.73|0.00|613.03|530.20|625.57|6.13|0.00|613.03|619.16|82.83| +2451104|66648|4252|67701|1459422|2343|38464|1|70|1680|55|8.51|13.61|4.49|71.61|246.95|468.05|748.55|1.75|71.61|175.34|177.09|-292.71| +2451104|66648|4339|67701|1459422|2343|38464|1|297|1680|55|31.84|36.29|25.76|0.00|1416.80|1751.20|1995.95|56.67|0.00|1416.80|1473.47|-334.40| +2451104|66648|9682|67701|1459422|2343|38464|1|2|1680|71|29.64|44.46|28.45|1676.55|2019.95|2104.44|3156.66|6.86|1676.55|343.40|350.26|-1761.04| +2451534|67575|10418|86127|44439|1151|1063|4|144|1681|89|31.96|38.99|15.98|0.00|1422.22|2844.44|3470.11|113.77|0.00|1422.22|1535.99|-1422.22| +2451534|67575|16574|86127|44439|1151|1063|4|154|1681|90|93.85|148.28|26.69|0.00|2402.10|8446.50|13345.20|24.02|0.00|2402.10|2426.12|-6044.40| +2451534|67575|7564|86127|44439|1151|1063|4|5|1681|28|30.72|54.06|14.05|0.00|393.40|860.16|1513.68|11.80|0.00|393.40|405.20|-466.76| +2451534|67575|17926|86127|44439|1151|1063|4|238|1681|2|83.18|118.94|0.00|0.00|0.00|166.36|237.88|0.00|0.00|0.00|0.00|-166.36| +2451534|67575|12350|86127|44439|1151|1063|4|217|1681|5|9.31|12.47|1.74|0.00|8.70|46.55|62.35|0.52|0.00|8.70|9.22|-37.85| +2451534|67575|1438|86127|44439|1151|1063|4|271|1681|52|53.39|76.88|41.51|0.00|2158.52|2776.28|3997.76|64.75|0.00|2158.52|2223.27|-617.76| +2451534|67575|4387|86127|44439|1151|1063|4|217|1681|61|87.70|171.01|99.18|0.00|6049.98|5349.70|10431.61|362.99|0.00|6049.98|6412.97|700.28| +2451534|67575|4849|86127|44439|1151|1063|4|97|1681|26|18.00|20.52|1.64|41.78|42.64|468.00|533.52|0.06|41.78|0.86|0.92|-467.14| +2451534|67575|6925|86127|44439|1151|1063|4|255|1681|59|7.47|11.20|2.01|0.00|118.59|440.73|660.80|0.00|0.00|118.59|118.59|-322.14| +|67575|2218|86127||1151|1063||250|1681||||35.90|0.00||1004.53||30.51|0.00|610.30||-394.23| +2451534|67575|1273|86127|44439|1151|1063|4|70|1681|41|37.55|61.20|47.12|0.00|1931.92|1539.55|2509.20|173.87|0.00|1931.92|2105.79|392.37| +2451135|33270|3220|41697|940727|1804|834|2|285|1682|5|20.98|22.02|21.13|0.00|105.65|104.90|110.10|5.28|0.00|105.65|110.93|0.75| +2451135|33270|17504|41697|940727|1804|834|2|129|1682|29|84.17|96.79|61.94|0.00|1796.26|2440.93|2806.91|125.73|0.00|1796.26|1921.99|-644.67| +2451135|33270|12292|41697|940727|1804|834|2|259|1682|31|97.48|191.06|34.39|0.00|1066.09|3021.88|5922.86|85.28|0.00|1066.09|1151.37|-1955.79| +2451135|33270|13705|41697|940727|1804|834|2|4|1682|94|35.79|62.63|40.08|0.00|3767.52|3364.26|5887.22|188.37|0.00|3767.52|3955.89|403.26| +2451135|33270|10990|41697|940727|1804|834|2|15|1682|85|57.50|101.20|53.63|0.00|4558.55|4887.50|8602.00|319.09|0.00|4558.55|4877.64|-328.95| +2451135|33270|11149|41697|940727|1804|834|2|160|1682|95|12.21|13.67|11.75|558.12|1116.25|1159.95|1298.65|0.00|558.12|558.13|558.13|-601.82| +2451135|33270|454|41697|940727|1804|834|2|230|1682|4|92.30|124.60|52.33|188.38|209.32|369.20|498.40|0.62|188.38|20.94|21.56|-348.26| +2451135|33270|6031|41697|940727|1804|834|2|174|1682|79|42.54|68.91|37.90|0.00|2994.10|3360.66|5443.89|149.70|0.00|2994.10|3143.80|-366.56| +2451853|64255|15553|35252|799258|3264|70|1|61|1683|82|13.21|14.79|1.18|0.00|96.76|1083.22|1212.78|6.77|0.00|96.76|103.53|-986.46| +2451853|64255|12368|35252|799258|3264|70|1|204|1683|91|64.24|114.98|101.18|0.00|9207.38|5845.84|10463.18|368.29|0.00|9207.38|9575.67|3361.54| +2451853|64255|14953|35252|799258|3264|70|1|182|1683|43|30.05|54.09|25.96|0.00|1116.28|1292.15|2325.87|100.46|0.00|1116.28|1216.74|-175.87| +2451853|64255|3643|35252|799258|3264|70|1|166|1683|96|13.86|20.23|7.88|143.73|756.48|1330.56|1942.08|18.38|143.73|612.75|631.13|-717.81| +2451853|64255|3833|35252|799258|3264|70|1|165|1683|7|78.86|100.94|23.21|0.00|162.47|552.02|706.58|11.37|0.00|162.47|173.84|-389.55| +2451853|64255|6391|35252|799258|3264|70|1|60|1683|30|29.52|36.30|12.70|0.00|381.00|885.60|1089.00|15.24|0.00|381.00|396.24|-504.60| +2451853|64255|11270|35252|799258|3264|70|1|82|1683|26|40.67|65.47|25.53|0.00|663.78|1057.42|1702.22|13.27|0.00|663.78|677.05|-393.64| +2451853|64255|1001|35252|799258|3264|70|1|57|1683|91|70.57|107.26|104.04|0.00|9467.64|6421.87|9760.66|568.05|0.00|9467.64|10035.69|3045.77| +2451853|64255|11221|35252|799258|3264|70|1|144|1683|90|10.67|19.63|7.65|0.00|688.50|960.30|1766.70|55.08|0.00|688.50|743.58|-271.80| +2451853|64255|1093|35252|799258|3264|70|1|33|1683|8|77.36|111.39|56.80|0.00|454.40|618.88|891.12|9.08|0.00|454.40|463.48|-164.48| +2451853|64255|13957|35252|799258|3264|70|1|225|1683|87|21.67|21.67|13.21|666.57|1149.27|1885.29|1885.29|28.96|666.57|482.70|511.66|-1402.59| +2451853|64255|7939|35252|799258|3264|70|1|74|1683|45|90.10|153.17|143.97|0.00|6478.65|4054.50|6892.65|0.00|0.00|6478.65|6478.65|2424.15| +2451853|64255|16286|35252|799258|3264|70|1|232|1683|9|23.33|42.46|19.53|0.00|175.77|209.97|382.14|14.06|0.00|175.77|189.83|-34.20| +2451853|64255|2491|35252|799258|3264|70|1|285|1683|56|54.96|88.48|24.77|0.00|1387.12|3077.76|4954.88|97.09|0.00|1387.12|1484.21|-1690.64| +2451192|52964|13168|28046|1029222|1598|9174|2|102|1684|85|30.26|58.40|49.05|0.00|4169.25|2572.10|4964.00|291.84|0.00|4169.25|4461.09|1597.15| +2451192|52964|5332|28046|1029222|1598|9174|2|23|1684|10|29.94|47.30|38.31|0.00|383.10|299.40|473.00|19.15|0.00|383.10|402.25|83.70| +2451192|52964|1610|28046|1029222|1598|9174|2|256|1684|39|80.18|105.83|8.46|0.00|329.94|3127.02|4127.37|19.79|0.00|329.94|349.73|-2797.08| +2451192|52964|10660|28046|1029222|1598|9174|2|138|1684|89|98.44|181.12|114.10|0.00|10154.90|8761.16|16119.68|507.74|0.00|10154.90|10662.64|1393.74| +2451192|52964|5792|28046|1029222|1598|9174|2|170|1684|58|24.72|26.45|21.42|1006.31|1242.36|1433.76|1534.10|4.72|1006.31|236.05|240.77|-1197.71| +2451192|52964|9074|28046|1029222|1598|9174|2|274|1684|42|99.56|181.19|63.41|0.00|2663.22|4181.52|7609.98|159.79|0.00|2663.22|2823.01|-1518.30| +2451192|52964|11846|28046|1029222|1598|9174|2|74|1684|71|65.29|126.00|107.10|0.00|7604.10|4635.59|8946.00|380.20|0.00|7604.10|7984.30|2968.51| +2451192|52964|9662|28046|1029222|1598|9174|2|52|1684|11|12.74|20.38|7.54|0.00|82.94|140.14|224.18|2.48|0.00|82.94|85.42|-57.20| +2451192|52964|493|28046|1029222|1598|9174|2|146|1684|83|36.09|49.44|25.21|0.00|2092.43|2995.47|4103.52|188.31|0.00|2092.43|2280.74|-903.04| +2451192|52964|11899|28046|1029222|1598|9174|2|255|1684|63|34.95|56.26|32.63|0.00|2055.69|2201.85|3544.38|41.11|0.00|2055.69|2096.80|-146.16| +2451192|52964|17275|28046|1029222|1598|9174|2|162|1684|77|96.81|118.10|31.88|2307.47|2454.76|7454.37|9093.70|10.31|2307.47|147.29|157.60|-7307.08| +2451192|52964|6448|28046|1029222|1598|9174|2|204|1684|36|98.30|157.28|34.60|697.53|1245.60|3538.80|5662.08|49.32|697.53|548.07|597.39|-2990.73| +2452258||103||||6031|4|119|1685||46.85|87.14|32.24|0.00|1837.68||||0.00|1837.68||| +2452258|52447|5675|20776|1313610|844|6031|4|32|1685|87|19.28|24.67|11.34|157.85|986.58|1677.36|2146.29|74.58|157.85|828.73|903.31|-848.63| +2452258|52447|10765|20776|1313610|844|6031|4|200|1685|3|45.37|81.66|59.61|82.26|178.83|136.11|244.98|0.96|82.26|96.57|97.53|-39.54| +2452258|52447|8057|20776|1313610|844|6031|4|122|1685|55|50.07|85.61|82.18|0.00|4519.90|2753.85|4708.55|406.79|0.00|4519.90|4926.69|1766.05| +2452258|52447|1635|20776|1313610|844|6031|4|185|1685|22|5.52|7.34|4.47|0.00|98.34|121.44|161.48|5.90|0.00|98.34|104.24|-23.10| +2452258|52447|9831|20776|1313610|844|6031|4|36|1685|37|87.50|106.75|73.65|0.00|2725.05|3237.50|3949.75|136.25|0.00|2725.05|2861.30|-512.45| +2452258|52447|2717|20776|1313610|844|6031|4|243|1685|94|62.64|122.77|33.14|0.00|3115.16|5888.16|11540.38|186.90|0.00|3115.16|3302.06|-2773.00| +2452258|52447|13465|20776|1313610|844|6031|4|132|1685|95|90.02|149.43|110.57|0.00|10504.15|8551.90|14195.85|105.04|0.00|10504.15|10609.19|1952.25| +2452258|52447|8051|20776|1313610|844|6031|4|7|1685|80|4.32|6.17|4.62|195.88|369.60|345.60|493.60|0.00|195.88|173.72|173.72|-171.88| +2452258|52447|7223|20776|1313610|844|6031|4|47|1685|5|88.35|120.15|49.26|0.00|246.30|441.75|600.75|17.24|0.00|246.30|263.54|-195.45| +2452369|33726|8245|57105|984521|6690|44506|4|241|1686|30|37.57|42.82|5.99|0.00|179.70|1127.10|1284.60|12.57|0.00|179.70|192.27|-947.40| +2452369|33726|11649|57105|984521|6690|44506|4|30|1686|98|20.19|26.44|10.57|0.00|1035.86|1978.62|2591.12|31.07|0.00|1035.86|1066.93|-942.76| +2452369|33726|13155|57105|984521|6690|44506|4|61|1686|2|73.64|114.14|29.67|0.00|59.34|147.28|228.28|2.37|0.00|59.34|61.71|-87.94| +2452369|33726|15469|57105|984521|6690|44506|4|225|1686|30|89.39|113.52|10.21|76.57|306.30|2681.70|3405.60|11.48|76.57|229.73|241.21|-2451.97| +2452369|33726|13249|57105|984521|6690|44506|4|180|1686|53|89.24|178.48|44.62|2152.02|2364.86|4729.72|9459.44|10.64|2152.02|212.84|223.48|-4516.88| +2452369|33726|4059|57105|984521|6690|44506|4|98|1686|53|56.52|69.51|29.88|0.00|1583.64|2995.56|3684.03|0.00|0.00|1583.64|1583.64|-1411.92| +2452369|33726|10303|57105|984521|6690|44506|4|289|1686|76|29.45|32.68|31.37|0.00|2384.12|2238.20|2483.68|95.36|0.00|2384.12|2479.48|145.92| +2452369|33726|11367|57105|984521|6690|44506|4|221|1686|4|86.50|154.83|120.76|0.00|483.04|346.00|619.32|4.83|0.00|483.04|487.87|137.04| +2452369|33726|7513|57105|984521|6690|44506|4|259|1686|25|45.65|57.51|27.02|0.00|675.50|1141.25|1437.75|33.77|0.00|675.50|709.27|-465.75| +2452369|33726|14575|57105|984521|6690|44506|4|199|1686|57|76.56|86.51|47.58|0.00|2712.06|4363.92|4931.07|216.96|0.00|2712.06|2929.02|-1651.86| +2452369|33726|6618|57105|984521|6690|44506|4|276|1686|85|33.10|65.20|11.08|621.58|941.80|2813.50|5542.00|25.61|621.58|320.22|345.83|-2493.28| +2452369|33726|15342|57105|984521|6690|44506|4|183|1686|74|81.76|89.11|62.37|1153.84|4615.38|6050.24|6594.14|242.30|1153.84|3461.54|3703.84|-2588.70| +2452369|33726|14727|57105|984521|6690|44506|4|115|1686|87|58.62|76.79|38.39|0.00|3339.93|5099.94|6680.73|66.79|0.00|3339.93|3406.72|-1760.01| +2451527|69157|4423|13530|985526|4369|31965|10|118|1687|71|31.23|48.09|13.46|860.09|955.66|2217.33|3414.39|1.91|860.09|95.57|97.48|-2121.76| +2451527|69157|14102|13530|985526|4369|31965|10|63|1687|65|10.34|19.33|2.12|0.00|137.80|672.10|1256.45|1.37|0.00|137.80|139.17|-534.30| +2451527|69157|7627|13530|985526|4369|31965|10|139|1687|12|41.95|50.75|38.06|0.00|456.72|503.40|609.00|31.97|0.00|456.72|488.69|-46.68| +2451527|69157|44|13530|985526|4369|31965|10|72|1687|41|85.01|103.71|26.96|0.00|1105.36|3485.41|4252.11|88.42|0.00|1105.36|1193.78|-2380.05| +2451527|69157|14818|13530|985526|4369|31965|10|245|1687|32|25.24|34.32|4.80|0.00|153.60|807.68|1098.24|10.75|0.00|153.60|164.35|-654.08| +2451527|69157|2126|13530|985526|4369|31965|10|124|1687|35|69.81|131.24|129.92|0.00|4547.20|2443.35|4593.40|363.77|0.00|4547.20|4910.97|2103.85| +2451527|69157|6146|13530|985526|4369|31965|10|79|1687|54|41.97|60.43|55.59|0.00|3001.86|2266.38|3263.22|150.09|0.00|3001.86|3151.95|735.48| +2451527|69157|9140|13530|985526|4369|31965|10|24|1687|2|61.71|123.42|25.91|0.00|51.82|123.42|246.84|2.59|0.00|51.82|54.41|-71.60| +2451527|69157|2797|13530|985526|4369|31965|10|180|1687|15|74.53|138.62|13.86|0.00|207.90|1117.95|2079.30|4.15|0.00|207.90|212.05|-910.05| +2452383|34666|12717|97787|836630|3026|13022|1|250|1688|20|91.83|159.78|89.47|0.00|1789.40|1836.60|3195.60|71.57|0.00|1789.40|1860.97|-47.20| +2452383|34666|1101|97787|836630|3026|13022|1|217|1688|86|23.51|23.98|3.59|0.00|308.74|2021.86|2062.28|0.00|0.00|308.74|308.74|-1713.12| +2452383||2185||836630||13022|1|89|1688||||21.67|0.00||9188.16||145.62|0.00||2225.94|| +2452383|34666|16161|97787|836630|3026|13022|1|36|1688|44|22.25|41.16|40.33|0.00|1774.52|979.00|1811.04|17.74|0.00|1774.52|1792.26|795.52| +2452383|34666|16866|97787|836630|3026|13022|1|142|1688|89|24.74|31.91|15.31|0.00|1362.59|2201.86|2839.99|109.00|0.00|1362.59|1471.59|-839.27| +2452383|34666|5400|97787|836630|3026|13022|1|251|1688|47|75.01|149.26|10.44|0.00|490.68|3525.47|7015.22|39.25|0.00|490.68|529.93|-3034.79| +2452383|34666|17899|97787|836630|3026|13022|1|145|1688|24|36.81|68.46|67.09|0.00|1610.16|883.44|1643.04|0.00|0.00|1610.16|1610.16|726.72| +2452383|34666|6345|97787|836630|3026|13022|1|195|1688|4|19.30|34.16|34.16|0.00|136.64|77.20|136.64|5.46|0.00|136.64|142.10|59.44| +2452383|34666|1921|97787|836630|3026|13022|1|56|1688|97|43.38|43.81|39.42|0.00|3823.74|4207.86|4249.57|267.66|0.00|3823.74|4091.40|-384.12| +2452383|34666|16236|97787|836630|3026|13022|1|34|1688|91|41.94|78.42|32.15|0.00|2925.65|3816.54|7136.22|175.53|0.00|2925.65|3101.18|-890.89| +2452383|34666|11905|97787|836630|3026|13022|1|34|1688|36|77.66|143.67|109.18|0.00|3930.48|2795.76|5172.12|235.82|0.00|3930.48|4166.30|1134.72| +2452383|34666|5215|97787|836630|3026|13022|1|142|1688|23|54.71|70.57|11.99|11.03|275.77|1258.33|1623.11|15.88|11.03|264.74|280.62|-993.59| +2452383|34666|15012|97787|836630|3026|13022|1|214|1688|48|65.13|84.66|66.88|0.00|3210.24|3126.24|4063.68|288.92|0.00|3210.24|3499.16|84.00| +2452383|34666|9205|97787|836630|3026|13022|1|59|1688|30|3.30|4.55|1.68|0.00|50.40|99.00|136.50|3.52|0.00|50.40|53.92|-48.60| +2452383|34666|8280|97787|836630|3026|13022|1|70|1688|18|5.65|8.87|6.56|59.04|118.08|101.70|159.66|2.36|59.04|59.04|61.40|-42.66| +2452253|40099|11565|18241|1002693|3945|39597|10|53|1689|21|36.98|58.05|51.66|0.00|1084.86|776.58|1219.05|21.69|0.00|1084.86|1106.55|308.28| +2452253|40099|11189|18241|1002693|3945|39597|10|10|1689|83|37.80|54.05|30.26|954.40|2511.58|3137.40|4486.15|93.43|954.40|1557.18|1650.61|-1580.22| +2452253|40099|9297|18241|1002693|3945|39597|10|128|1689|85|77.46|89.85|80.86|6804.36|6873.10|6584.10|7637.25|0.00|6804.36|68.74|68.74|-6515.36| +2452253|40099|15679|18241|1002693|3945|39597|10|11|1689|99|41.96|70.07|31.53|0.00|3121.47|4154.04|6936.93|124.85|0.00|3121.47|3246.32|-1032.57| +2452253|40099|17337||||39597||169|1689|11|52.87|54.98|29.68|||581.57|||||346.06|-255.09| +2452253|40099|16675|18241|1002693|3945|39597|10|177|1689|56|26.15|47.07|6.58|0.00|368.48|1464.40|2635.92|3.68|0.00|368.48|372.16|-1095.92| +2452253|40099|16003|18241|1002693|3945|39597|10|86|1689|79|25.27|47.00|26.32|873.29|2079.28|1996.33|3713.00|84.41|873.29|1205.99|1290.40|-790.34| +2452253|40099|13641|18241|1002693|3945|39597|10|25|1689|16|72.79|132.47|71.53|0.00|1144.48|1164.64|2119.52|45.77|0.00|1144.48|1190.25|-20.16| +2452253|40099|2189|18241|1002693|3945|39597|10|149|1689|13|43.89|50.91|1.01|0.00|13.13|570.57|661.83|0.39|0.00|13.13|13.52|-557.44| +2452253|40099|15065|18241|1002693|3945|39597|10|117|1689|42|64.68|91.19|90.27|0.00|3791.34|2716.56|3829.98|151.65|0.00|3791.34|3942.99|1074.78| +2452253|40099|8589|18241|1002693|3945|39597|10|207|1689|7|57.49|95.43|6.68|0.00|46.76|402.43|668.01|2.33|0.00|46.76|49.09|-355.67| +2452253|40099|16321|18241|1002693|3945|39597|10|160|1689|62|60.05|120.10|102.08|0.00|6328.96|3723.10|7446.20|63.28|0.00|6328.96|6392.24|2605.86| +2452253|40099|8489|18241|1002693|3945|39597|10|34|1689|10|56.81|98.84|3.95|32.78|39.50|568.10|988.40|0.40|32.78|6.72|7.12|-561.38| +2452253|40099|2503|18241|1002693|3945|39597|10|214|1689|95|40.17|48.20|38.07|2929.48|3616.65|3816.15|4579.00|48.10|2929.48|687.17|735.27|-3128.98| +2451524|48464|10546|9248|372399|4350|7101|8|212|1690|45|56.75|61.85|3.09|0.00|139.05|2553.75|2783.25|12.51|0.00|139.05|151.56|-2414.70| +2451524||3193|||4350||||1690||83.87|||0.00||2096.75|2348.25|0.00|0.00||258.25|-1838.50| +2451524|48464|16010|9248|372399|4350|7101|8|126|1690|90|47.20|92.04|83.75|0.00|7537.50|4248.00|8283.60|452.25|0.00|7537.50|7989.75|3289.50| +2451524|48464|10204|9248|372399|4350|7101|8|225|1690|61|37.88|40.91|10.63|0.00|648.43|2310.68|2495.51|51.87|0.00|648.43|700.30|-1662.25| +2451524|48464|2365|9248|372399|4350|7101|8|36|1690|26|83.56|103.61|38.33|0.00|996.58|2172.56|2693.86|19.93|0.00|996.58|1016.51|-1175.98| +2451524||13988|||4350|7101|||1690|55|||12.62||694.10||1543.30|||694.10|742.68|| +2451524|48464|2414|9248|372399|4350|7101|8|129|1690|50|57.57|103.62|63.20|0.00|3160.00|2878.50|5181.00|126.40|0.00|3160.00|3286.40|281.50| +2451524|48464|3121|9248|372399|4350|7101|8|63|1690|60|65.19|113.43|63.52|0.00|3811.20|3911.40|6805.80|38.11|0.00|3811.20|3849.31|-100.20| +2451524|48464|10855|9248|372399|4350|7101|8|290|1690|50|3.34|3.67|0.33|0.00|16.50|167.00|183.50|0.66|0.00|16.50|17.16|-150.50| +2451524|48464|8518|9248|372399|4350|7101|8|193|1690|62|78.11|85.13|34.90|1601.21|2163.80|4842.82|5278.06|28.12|1601.21|562.59|590.71|-4280.23| +2451127|46831|6422|27469|433555|6841|16527|4|151|1691|89|28.34|45.06|37.85|1920.13|3368.65|2522.26|4010.34|101.39|1920.13|1448.52|1549.91|-1073.74| +2451127|46831|1178|27469|433555|6841|16527|4|75|1691|57|5.04|7.66|6.97|293.99|397.29|287.28|436.62|2.06|293.99|103.30|105.36|-183.98| +2451127|46831|10528|27469|433555|6841|16527|4|22|1691|97|74.15|113.44|26.09|0.00|2530.73|7192.55|11003.68|126.53|0.00|2530.73|2657.26|-4661.82| +2451127|46831|2815|27469|433555|6841|16527|4|57|1691|30|11.93|15.38|3.22|0.00|96.60|357.90|461.40|1.93|0.00|96.60|98.53|-261.30| +2451127|46831|5156|27469|433555|6841|16527|4|282|1691|13|12.02|18.27|2.00|0.00|26.00|156.26|237.51|1.30|0.00|26.00|27.30|-130.26| +2451127|46831|14564|27469|433555|6841|16527|4|78|1691|61|80.08|94.49|78.42|4687.94|4783.62|4884.88|5763.89|5.74|4687.94|95.68|101.42|-4789.20| +2451127|46831|15248|27469|433555|6841|16527|4|29|1691|74|20.02|32.63|30.67|0.00|2269.58|1481.48|2414.62|68.08|0.00|2269.58|2337.66|788.10| +2451127||5578|27469|||16527||288|1691|36||||0.00|||257.04|9.24|0.00|231.12|240.36|42.12| +2451127|46831|13669|27469|433555|6841|16527|4|46|1691|86|33.31|48.63|36.95|0.00|3177.70|2864.66|4182.18|127.10|0.00|3177.70|3304.80|313.04| +2451127|46831|7441|27469|433555|6841|16527|4|195|1691|37|26.45|39.67|21.81|0.00|806.97|978.65|1467.79|32.27|0.00|806.97|839.24|-171.68| +2451127|46831|7598|27469|433555|6841|16527|4|39|1691|93|5.45|9.31|1.21|0.00|112.53|506.85|865.83|7.87|0.00|112.53|120.40|-394.32| +2451127|46831|806|27469|433555|6841|16527|4|20|1691|52|12.24|13.46|4.57|0.00|237.64|636.48|699.92|2.37|0.00|237.64|240.01|-398.84| +2451521|68041|4843|21682|1682669|456|30243|1|215|1692|83|87.33|108.28|14.07|0.00|1167.81|7248.39|8987.24|0.00|0.00|1167.81|1167.81|-6080.58| +2451521|68041|1588|21682|1682669|456|30243|1|73|1692|96|29.30|51.27|28.19|2300.30|2706.24|2812.80|4921.92|28.41|2300.30|405.94|434.35|-2406.86| +2451521|68041|6518|21682|1682669|456|30243|1|216|1692|9|48.94|68.51|10.96|0.00|98.64|440.46|616.59|8.87|0.00|98.64|107.51|-341.82| +2451521|68041|15202|21682|1682669|456|30243|1|111|1692|84|33.96|46.86|23.43|0.00|1968.12|2852.64|3936.24|59.04|0.00|1968.12|2027.16|-884.52| +2451521|68041|14972|21682|1682669|456|30243|1|13|1692|63|40.97|48.34|16.91|0.00|1065.33|2581.11|3045.42|95.87|0.00|1065.33|1161.20|-1515.78| +2451521|68041|5341|21682|1682669|456|30243|1|45|1692|79|65.42|85.04|60.37|0.00|4769.23|5168.18|6718.16|95.38|0.00|4769.23|4864.61|-398.95| +2451521|68041|5776|21682|1682669|456|30243|1|29|1692|79|79.37|125.40|116.62|3040.28|9212.98|6270.23|9906.60|370.36|3040.28|6172.70|6543.06|-97.53| +2451521|68041|3403|21682|1682669|456|30243|1|240|1692|14|50.37|93.18|43.79|0.00|613.06|705.18|1304.52|6.13|0.00|613.06|619.19|-92.12| +2451521|68041|4276|21682|1682669|456|30243|1|142|1692|38|64.64|87.91|36.04|191.73|1369.52|2456.32|3340.58|35.33|191.73|1177.79|1213.12|-1278.53| +2451521|68041|6410|21682|1682669|456|30243|1|4|1692|68|9.70|14.25|1.56|102.89|106.08|659.60|969.00|0.00|102.89|3.19|3.19|-656.41| +2451854|38933|8047|86514|1317357|1800|41090|8|76|1693|83|88.61|124.05|119.08|0.00|9883.64|7354.63|10296.15|98.83|0.00|9883.64|9982.47|2529.01| +2451854|38933|12553|86514|1317357|1800|41090|8|141|1693|8|17.15|19.03|5.13|0.00|41.04|137.20|152.24|0.82|0.00|41.04|41.86|-96.16| +2451854|38933|15533|86514|1317357|1800|41090|8|224|1693|55|4.49|6.73|4.50|0.00|247.50|246.95|370.15|17.32|0.00|247.50|264.82|0.55| +2451854|38933|11936|86514|1317357|1800|41090|8|102|1693|54|78.20|81.32|39.84|839.03|2151.36|4222.80|4391.28|0.00|839.03|1312.33|1312.33|-2910.47| +2451854|38933|12299|86514|1317357|1800|41090|8|13|1693|41|39.93|51.11|33.73|0.00|1382.93|1637.13|2095.51|55.31|0.00|1382.93|1438.24|-254.20| +2451854|38933|17546|86514|1317357|1800|41090|8|261|1693|11|59.55|70.26|7.02|0.00|77.22|655.05|772.86|0.00|0.00|77.22|77.22|-577.83| +2451854|38933|3062|86514|1317357|1800|41090|8|180|1693|18|36.10|52.70|9.48|42.66|170.64|649.80|948.60|8.95|42.66|127.98|136.93|-521.82| +2451854|38933|14591|86514|1317357|1800|41090|8|175|1693|77|85.35|113.51|62.43|0.00|4807.11|6571.95|8740.27|48.07|0.00|4807.11|4855.18|-1764.84| +2451854|38933|13196|86514|1317357|1800|41090|8|238|1693|11|5.63|10.64|2.34|0.00|25.74|61.93|117.04|1.80|0.00|25.74|27.54|-36.19| +2451854|38933|7004|86514|1317357|1800|41090|8|185|1693|19|73.27|121.62|19.45|0.00|369.55|1392.13|2310.78|25.86|0.00|369.55|395.41|-1022.58| +2451854|38933|9995|86514|1317357|1800|41090|8|9|1693|85|97.53|182.38|23.70|0.00|2014.50|8290.05|15502.30|60.43|0.00|2014.50|2074.93|-6275.55| +2451854|38933|4025|86514|1317357|1800|41090|8|27|1693|16|80.34|153.44|108.94|505.48|1743.04|1285.44|2455.04|0.00|505.48|1237.56|1237.56|-47.88| +2451854|38933|15158|86514|1317357|1800|41090|8|7|1693|65|91.60|153.88|67.70|0.00|4400.50|5954.00|10002.20|220.02|0.00|4400.50|4620.52|-1553.50| +2451854|38933|6077|86514|1317357|1800|41090|8|155|1693|63|8.95|14.67|10.85|0.00|683.55|563.85|924.21|54.68|0.00|683.55|738.23|119.70| +2451854|38933|8485|86514|1317357|1800|41090|8|291|1693|37|2.43|2.43|0.70|24.08|25.90|89.91|89.91|0.07|24.08|1.82|1.89|-88.09| +2451854|38933|1441|86514|1317357|1800|41090|8|126|1693|2|82.89|102.78|63.72|66.26|127.44|165.78|205.56|4.28|66.26|61.18|65.46|-104.60| +2451657|36693|4429|73601|1271301|5801|29148|2|48|1694|43|36.70|44.04|12.77|0.00|549.11|1578.10|1893.72|27.45|0.00|549.11|576.56|-1028.99| +2451657|36693|10171|73601|1271301|5801|29148|2|133|1694|10|84.53|126.79|117.91|0.00|1179.10|845.30|1267.90|94.32|0.00|1179.10|1273.42|333.80| +2451657|36693|14081|73601|1271301|5801|29148|2|261|1694|95|65.68|66.33|24.54|0.00|2331.30|6239.60|6301.35|69.93|0.00|2331.30|2401.23|-3908.30| +2451657|36693|11654|73601|1271301|5801|29148|2|250|1694|85|81.05|125.62|71.60|0.00|6086.00|6889.25|10677.70|426.02|0.00|6086.00|6512.02|-803.25| +2451657|36693|14468|73601|1271301|5801|29148|2|25|1694|59|15.51|24.04|18.99|1109.20|1120.41|915.09|1418.36|0.78|1109.20|11.21|11.99|-903.88| +2451657|36693|13301|73601|1271301|5801|29148|2|261|1694|80|71.31|142.62|52.76|0.00|4220.80|5704.80|11409.60|379.87|0.00|4220.80|4600.67|-1484.00| +||545|73601|1271301|5801||||1694|90||||2847.81|||8951.40|0.00|2847.81|1000.59|1000.59|| +2451657|36693|955|73601|1271301|5801|29148|2|66|1694|72|94.46|188.92|75.56|0.00|5440.32|6801.12|13602.24|380.82|0.00|5440.32|5821.14|-1360.80| +2451657|36693|7034|73601|1271301|5801|29148|2|85|1694|94|66.12|109.09|66.54|0.00|6254.76|6215.28|10254.46|125.09|0.00|6254.76|6379.85|39.48| +2451657|36693|5353|73601|1271301|5801|29148|2|298|1694|70|90.61|97.85|16.63|0.00|1164.10|6342.70|6849.50|23.28|0.00|1164.10|1187.38|-5178.60| +2451657|36693|1799|73601|1271301|5801|29148|2|135|1694|4|51.15|88.48|88.48|53.08|353.92|204.60|353.92|3.00|53.08|300.84|303.84|96.24| +2451657||10859|||5801|29148|2|157|1694|91|54.58|76.95|||910.00|||7.28||||-4602.78| +2451513|39973|7552|59808|699903|5598|26493|2|140|1695|51|88.51|103.55|39.34|0.00|2006.34|4514.01|5281.05|160.50|0.00|2006.34|2166.84|-2507.67| +2451513|39973|7496|59808|699903|5598|26493|2|8|1695|48|43.43|71.65|18.62|0.00|893.76|2084.64|3439.20|26.81|0.00|893.76|920.57|-1190.88| +2451513|39973|9476|59808|699903|5598|26493|2|49|1695|72|2.44|4.07|0.00|0.00|0.00|175.68|293.04|0.00|0.00|0.00|0.00|-175.68| +2451513|39973|11614|59808|699903|5598|26493|2|279|1695|55|42.94|55.82|7.25|0.00|398.75|2361.70|3070.10|31.90|0.00|398.75|430.65|-1962.95| +2451513|39973|10454|59808|699903|5598|26493|2|288|1695|11|64.07|124.29|34.80|38.28|382.80|704.77|1367.19|13.78|38.28|344.52|358.30|-360.25| +2451513|39973|5077|59808|699903|5598|26493|2|43|1695|99|42.36|55.49|32.18|0.00|3185.82|4193.64|5493.51|159.29|0.00|3185.82|3345.11|-1007.82| +2451513|39973|12232|59808|699903|5598|26493|2|163|1695|39|57.17|73.17|37.31|0.00|1455.09|2229.63|2853.63|116.40|0.00|1455.09|1571.49|-774.54| +2451513|39973|15542|59808|699903|5598|26493|2|255|1695|16|73.07|122.75|30.68|0.00|490.88|1169.12|1964.00|44.17|0.00|490.88|535.05|-678.24| +2451513|39973|13321|59808|699903|5598|26493|2|116|1695|79|17.39|25.38|2.03|0.00|160.37|1373.81|2005.02|3.20|0.00|160.37|163.57|-1213.44| +2451513|39973|10874|59808|699903|5598|26493|2|96|1695|63|17.44|18.13|10.15|0.00|639.45|1098.72|1142.19|25.57|0.00|639.45|665.02|-459.27| +2451513|39973|15580|59808|699903|5598|26493|2|40|1695|77|3.43|3.67|1.90|0.00|146.30|264.11|282.59|8.77|0.00|146.30|155.07|-117.81| +2451513|39973|7616|59808|699903|5598|26493|2|97|1695|65|65.60|112.83|19.18|897.62|1246.70|4264.00|7333.95|0.00|897.62|349.08|349.08|-3914.92| +2451513|39973|16396|59808|699903|5598|26493|2|273|1695|75|72.25|75.86|42.48|0.00|3186.00|5418.75|5689.50|159.30|0.00|3186.00|3345.30|-2232.75| +2451513|39973|17660|59808|699903|5598|26493|2|96|1695|19|85.35|156.19|73.40|0.00|1394.60|1621.65|2967.61|83.67|0.00|1394.60|1478.27|-227.05| +2451513|39973|15310|59808|699903|5598|26493|2|61|1695|40|64.04|82.61|16.52|0.00|660.80|2561.60|3304.40|59.47|0.00|660.80|720.27|-1900.80| +2452596|34453|12913|24820|1167427|1587|36547|7|43|1696|70|38.19|74.85|19.46|0.00|1362.20|2673.30|5239.50|13.62|0.00|1362.20|1375.82|-1311.10| +|34453|4503|24820||||||1696|25|8.44||5.37|127.53||211.00||0.53|127.53|6.72||| +2452596|34453|1687|24820|1167427|1587|36547|7|5|1696|65|6.86|10.01|4.50|0.00|292.50|445.90|650.65|2.92|0.00|292.50|295.42|-153.40| +2452596|34453|8202|24820|1167427|1587|36547|7|29|1696|9|26.88|37.63|9.40|0.00|84.60|241.92|338.67|6.76|0.00|84.60|91.36|-157.32| +2452596|34453|5706|24820|1167427|1587|36547|7|34|1696|59|6.87|8.03|5.13|90.80|302.67|405.33|473.77|16.94|90.80|211.87|228.81|-193.46| +2452596|34453|2019|24820|1167427|1587|36547|7|165|1696|3|67.90|135.12|97.28|0.00|291.84|203.70|405.36|0.00|0.00|291.84|291.84|88.14| +2452596|34453|11826|24820|1167427|1587|36547|7|74|1696|37|2.86|4.26|1.27|14.09|46.99|105.82|157.62|1.97|14.09|32.90|34.87|-72.92| +2452596|34453|14595|24820|1167427|1587|36547|7|103|1696|13|61.58|120.08|7.20|0.00|93.60|800.54|1561.04|6.55|0.00|93.60|100.15|-706.94| +2452536|53731|6906|75758|1338716|6474|19032|8|1|1697|87|90.78|104.39|36.53|0.00|3178.11|7897.86|9081.93|286.02|0.00|3178.11|3464.13|-4719.75| +2452536|53731|357|75758|1338716|6474|19032|8|261|1697|64|16.41|22.97|21.36|0.00|1367.04|1050.24|1470.08|82.02|0.00|1367.04|1449.06|316.80| +2452536|53731|2059|75758|1338716|6474|19032|8|295|1697|88|35.19|47.15|24.98|967.22|2198.24|3096.72|4149.20|12.31|967.22|1231.02|1243.33|-1865.70| +2452536|53731|13677|75758|1338716|6474|19032|8|167|1697|76|26.70|32.30|7.42|0.00|563.92|2029.20|2454.80|39.47|0.00|563.92|603.39|-1465.28| +2452536|53731|17592|75758|1338716|6474|19032|8|63|1697|73|68.72|97.58|87.82|0.00|6410.86|5016.56|7123.34|320.54|0.00|6410.86|6731.40|1394.30| +2452536||14289||1338716|6474|19032|||1697|56||127.29||||||||1425.20||| +2452536|53731|7758|75758|1338716|6474|19032|8|59|1697|54|45.95|70.76|1.41|2.28|76.14|2481.30|3821.04|2.95|2.28|73.86|76.81|-2407.44| +2452536|53731|8241|75758|1338716|6474|19032|8|250|1697|97|36.27|58.75|57.57|0.00|5584.29|3518.19|5698.75|167.52|0.00|5584.29|5751.81|2066.10| +2452536|53731|10272|75758|1338716|6474|19032|8|110|1697|59|26.97|42.07|28.18|0.00|1662.62|1591.23|2482.13|49.87|0.00|1662.62|1712.49|71.39| +2452536|53731|11265|75758|1338716|6474|19032|8|71|1697|6|93.88|103.26|62.98|0.00|377.88|563.28|619.56|22.67|0.00|377.88|400.55|-185.40| +2451747||8407|61339|||||14|1698|10|85.71|113.13|47.51||475.10||1131.30|42.75||||| +2451747|60772|4483|61339|10525|3711|30897|1|166|1698|61|26.84|36.50|27.74|0.00|1692.14|1637.24|2226.50|33.84|0.00|1692.14|1725.98|54.90| +2451747|60772|7964|61339|10525|3711|30897|1|123|1698|51|79.02|135.91|47.56|0.00|2425.56|4030.02|6931.41|72.76|0.00|2425.56|2498.32|-1604.46| +2451747|60772|2966|61339|10525|3711|30897|1|220|1698|2|50.96|67.26|37.66|0.00|75.32|101.92|134.52|3.01|0.00|75.32|78.33|-26.60| +2451747|60772|1807|61339|10525|3711|30897|1|115|1698|18|30.94|60.02|31.21|106.73|561.78|556.92|1080.36|22.75|106.73|455.05|477.80|-101.87| +2451747|60772|5437|61339|10525|3711|30897|1|178|1698|31|2.40|3.60|1.33|0.00|41.23|74.40|111.60|1.23|0.00|41.23|42.46|-33.17| +2451747|60772|10295|61339|10525|3711|30897|1|27|1698|87|30.28|48.14|34.17|0.00|2972.79|2634.36|4188.18|59.45|0.00|2972.79|3032.24|338.43| +2451747|60772|8042|61339|10525|3711|30897|1|36|1698|58|87.23|117.76|10.59|0.00|614.22|5059.34|6830.08|18.42|0.00|614.22|632.64|-4445.12| +2451747|60772|14827|61339|10525|3711|30897|1|99|1698|100|89.63|129.96|32.49|0.00|3249.00|8963.00|12996.00|227.43|0.00|3249.00|3476.43|-5714.00| +2451747|60772|14833|61339|10525|3711|30897|1|119|1698|53|33.23|64.79|51.18|0.00|2712.54|1761.19|3433.87|108.50|0.00|2712.54|2821.04|951.35| +2451747|60772|11201|61339|10525|3711|30897|1|82|1698|97|77.34|91.26|33.76|0.00|3274.72|7501.98|8852.22|261.97|0.00|3274.72|3536.69|-4227.26| +2451747|60772|17095||10525||||253|1698|91||97.39|4.86||442.26|6663.93|8862.49||||477.64|| +2451615|51023|2765|94835|1065394|306|28065|2|146|1699|1|67.04|79.10|18.98|0.00|18.98|67.04|79.10|0.00|0.00|18.98|18.98|-48.06| +2451615|51023|3595|94835|1065394|306|28065|2|172|1699|3|69.12|102.29|54.21|0.00|162.63|207.36|306.87|14.63|0.00|162.63|177.26|-44.73| +2451615|51023|11411|94835|1065394|306|28065|2|267|1699|60|64.84|70.02|15.40|0.00|924.00|3890.40|4201.20|73.92|0.00|924.00|997.92|-2966.40| +2451615|51023|1141|94835|1065394|306|28065|2|229|1699|52|44.87|78.97|1.57|0.00|81.64|2333.24|4106.44|7.34|0.00|81.64|88.98|-2251.60| +2451615|51023|3638|94835|1065394|306|28065|2|137|1699|38|59.24|104.85|71.29|243.81|2709.02|2251.12|3984.30|73.95|243.81|2465.21|2539.16|214.09| +2451615|51023|16799|94835|1065394|306|28065|2|2|1699|39|67.14|122.19|19.55|0.00|762.45|2618.46|4765.41|53.37|0.00|762.45|815.82|-1856.01| +2451615|51023|4472|94835|1065394|306|28065|2|249|1699|88|30.79|40.64|14.22|0.00|1251.36|2709.52|3576.32|37.54|0.00|1251.36|1288.90|-1458.16| +2451615|51023|12115|94835|1065394|306|28065|2|156|1699|93|51.19|75.24|15.04|0.00|1398.72|4760.67|6997.32|55.94|0.00|1398.72|1454.66|-3361.95| +2451615|51023|15613|94835|1065394||28065|||1699||||1.90|0.00|36.10|279.11|||0.00|||-243.01| +2451615|51023|8696|94835|1065394|306|28065|2|220|1699|28|17.78|32.89|32.56|0.00|911.68|497.84|920.92|72.93|0.00|911.68|984.61|413.84| +2451663|47368|14051|16843|1315892|2141|27237|2|227|1700|54|59.14|114.73|64.24|0.00|3468.96|3193.56|6195.42|242.82|0.00|3468.96|3711.78|275.40| +2451663|47368|11210|16843|1315892|2141|27237|2|199|1700|90|96.21|148.16|13.33|0.00|1199.70|8658.90|13334.40|47.98|0.00|1199.70|1247.68|-7459.20| +2451663|47368|7562|16843|1315892|2141|27237|2|86|1700|59|48.10|48.58|2.91|0.00|171.69|2837.90|2866.22|5.15|0.00|171.69|176.84|-2666.21| +2451663|47368|6497|16843|1315892|2141|27237|2|217|1700|40|5.96|9.47|3.12|0.00|124.80|238.40|378.80|2.49|0.00|124.80|127.29|-113.60| +2451663|47368|3577|16843|1315892|2141|27237|2|47|1700|43|93.76|124.70|105.99|0.00|4557.57|4031.68|5362.10|0.00|0.00|4557.57|4557.57|525.89| +2451663|47368|800|16843|1315892|2141|27237|2|215|1700|3|72.88|107.13|0.00|0.00|0.00|218.64|321.39|0.00|0.00|0.00|0.00|-218.64| +2451663|47368|11111|16843|1315892|2141|27237|2|280|1700|82|65.05|122.94|55.32|0.00|4536.24|5334.10|10081.08|90.72|0.00|4536.24|4626.96|-797.86| +2451663|47368|8099|16843|1315892|2141|27237|2|24|1700|61|35.59|69.75|64.17|0.00|3914.37|2170.99|4254.75|78.28|0.00|3914.37|3992.65|1743.38| +2451663|47368|7015|16843|1315892|2141|27237|2|209|1700|47|30.97|56.98|19.94|0.00|937.18|1455.59|2678.06|37.48|0.00|937.18|974.66|-518.41| +2452348|40018|17634|49824|1869593|3409|29578|2|137|1701|31|34.35|38.47|20.77|618.11|643.87|1064.85|1192.57|0.77|618.11|25.76|26.53|-1039.09| +2452348|40018|12169|49824|1869593|3409|29578|2|72|1701|81|1.76|3.30|1.81|0.00|146.61|142.56|267.30|4.39|0.00|146.61|151.00|4.05| +2452348|40018|16209|49824|1869593|3409|29578|2|232|1701|6|89.48|177.17|1.77|0.00|10.62|536.88|1063.02|0.10|0.00|10.62|10.72|-526.26| +2452348|40018|837|49824|1869593|3409|29578|2|264|1701|42|84.16|134.65|117.14|0.00|4919.88|3534.72|5655.30|147.59|0.00|4919.88|5067.47|1385.16| +2452348|40018|11628|49824|1869593|3409|29578|2|103|1701|11|24.43|36.88|2.58|19.58|28.38|268.73|405.68|0.79|19.58|8.80|9.59|-259.93| +2452348|40018|17880||1869593|3409|29578|2|223|1701||||55.45|0.00||||170.78|0.00||4440.43|| +2452348|40018|17737|49824|1869593|3409|29578|2|253|1701|44|93.74|108.73|102.20|359.74|4496.80|4124.56|4784.12|0.00|359.74|4137.06|4137.06|12.50| +2452348|40018|6873|49824|1869593|3409|29578|2|79|1701|4|95.28|103.85|58.15|0.00|232.60|381.12|415.40|2.32|0.00|232.60|234.92|-148.52| +2452348|40018|16980|49824|1869593|3409|29578|2|71|1701|43|4.35|5.13|2.05|0.00|88.15|187.05|220.59|0.00|0.00|88.15|88.15|-98.90| +2452348|40018|7104|49824|1869593|3409|29578|2|300|1701|93|77.96|113.82|71.70|0.00|6668.10|7250.28|10585.26|266.72|0.00|6668.10|6934.82|-582.18| +2452348|40018|12078|49824|1869593|3409|29578|2|237|1701|49|79.16|116.36|30.25|770.77|1482.25|3878.84|5701.64|49.80|770.77|711.48|761.28|-3167.36| +2452348|40018|915|49824|1869593|3409|29578|2|75|1701|38|30.71|55.27|17.68|0.00|671.84|1166.98|2100.26|33.59|0.00|671.84|705.43|-495.14| +2452348|40018|1891|49824|1869593|3409|29578|2|85|1701|12|32.88|46.68|17.73|112.76|212.76|394.56|560.16|5.00|112.76|100.00|105.00|-294.56| +2452348|40018|1471|49824|1869593|3409|29578|2|177|1701|8|62.25|75.94|74.42|0.00|595.36|498.00|607.52|29.76|0.00|595.36|625.12|97.36| +2452348|40018|3475|49824|1869593|3409|29578|2|235|1701|36|12.87|13.64|9.82|0.00|353.52|463.32|491.04|21.21|0.00|353.52|374.73|-109.80| +2452348|40018|661|49824|1869593|3409|29578|2|174|1701|14|31.89|45.60|40.12|438.11|561.68|446.46|638.40|4.94|438.11|123.57|128.51|-322.89| +2451525|38336|17299|7906|1693679|5499|14472|10|180|1702|82|43.82|61.34|21.46|369.54|1759.72|3593.24|5029.88|69.50|369.54|1390.18|1459.68|-2203.06| +2451525|38336|4423|7906|1693679|5499|14472|10|249|1702|14|64.66|73.71|40.54|232.69|567.56|905.24|1031.94|0.00|232.69|334.87|334.87|-570.37| +2451525|38336|14102|7906|1693679|5499|14472|10|64|1702|91|18.49|29.21|22.49|0.00|2046.59|1682.59|2658.11|40.93|0.00|2046.59|2087.52|364.00| +2451525|38336|7627|7906|1693679|5499|14472|10|239|1702|26|30.94|45.48|23.64|0.00|614.64|804.44|1182.48|55.31|0.00|614.64|669.95|-189.80| +2451525|38336|44|7906|1693679|5499|14472|10|23|1702|11|44.95|53.49|32.09|0.00|352.99|494.45|588.39|24.70|0.00|352.99|377.69|-141.46| +2451525|38336|14818|7906|1693679|5499|14472|10|164|1702|13|84.11|86.63|19.92|0.00|258.96|1093.43|1126.19|18.12|0.00|258.96|277.08|-834.47| +2451525|38336|2126|7906|1693679|5499|14472|10|62|1702|15|77.51|106.18|52.02|0.00|780.30|1162.65|1592.70|39.01|0.00|780.30|819.31|-382.35| +2451525|38336|6146|7906|1693679|5499|14472|10|102|1702|32|95.47|187.12|172.15|0.00|5508.80|3055.04|5987.84|440.70|0.00|5508.80|5949.50|2453.76| +2451525|38336|9140|7906|1693679|5499|14472|10|134|1702|69|41.25|81.26|52.00|0.00|3588.00|2846.25|5606.94|107.64|0.00|3588.00|3695.64|741.75| +2451525|38336|2797|7906|1693679|5499|14472|10|75|1702|97|63.72|64.35|46.33|2022.30|4494.01|6180.84|6241.95|0.00|2022.30|2471.71|2471.71|-3709.13| +2451525||13702|7906|1693679||14472|10||1702|27|||48.43|||1169.91|2012.04|||1307.61|1320.68|137.70| +2451525|38336|10546|7906|1693679|5499|14472|10|72|1702|94|76.82|104.47|63.72|0.00|5989.68|7221.08|9820.18|59.89|0.00|5989.68|6049.57|-1231.40| +2451525|38336|3193|7906|1693679|5499|14472|10|242|1702|77|68.56|114.49|18.31|0.00|1409.87|5279.12|8815.73|42.29|0.00|1409.87|1452.16|-3869.25| +2451525|38336|16010|7906|1693679|5499|14472|10|265|1702|75|21.60|38.44|35.74|0.00|2680.50|1620.00|2883.00|134.02|0.00|2680.50|2814.52|1060.50| +2451525|38336|10204|7906|1693679|5499|14472|10|135|1702|10|36.53|42.00|34.44|0.00|344.40|365.30|420.00|27.55|0.00|344.40|371.95|-20.90| +2451525|38336|2365|7906|1693679|5499|14472|10|58|1702|22|38.33|54.04|38.90|51.34|855.80|843.26|1188.88|72.40|51.34|804.46|876.86|-38.80| +2451891|68417|7417|23797|1862325|4053|38181|2|192|1703|9|22.46|33.69|8.08|48.72|72.72|202.14|303.21|0.24|48.72|24.00|24.24|-178.14| +2451891|68417|7196|23797|1862325|4053|38181|2|283|1703|79|95.84|102.54|8.20|0.00|647.80|7571.36|8100.66|12.95|0.00|647.80|660.75|-6923.56| +2451891|68417|8371|23797|1862325|4053|38181|2|89|1703|47|44.04|62.97|27.70|0.00|1301.90|2069.88|2959.59|65.09|0.00|1301.90|1366.99|-767.98| +2451891|68417|5605|23797|1862325|4053|38181|2|55|1703|43|34.19|60.17|44.52|0.00|1914.36|1470.17|2587.31|19.14|0.00|1914.36|1933.50|444.19| +2451891|68417|2281|23797|1862325|4053|38181|2|182|1703|49|63.56|74.36|24.53|288.47|1201.97|3114.44|3643.64|63.94|288.47|913.50|977.44|-2200.94| +2451891|68417|12176|23797|1862325|4053|38181|2|155|1703|47|3.83|5.28|2.74|0.00|128.78|180.01|248.16|9.01|0.00|128.78|137.79|-51.23| +||2591|23797|1862325||38181|2||1703|16||6.24|4.24|0.00|67.84|||6.10|0.00|67.84||10.72| +2451891|68417|10403|23797|1862325|4053|38181|2|206|1703|55|25.70|36.75|7.71|80.56|424.05|1413.50|2021.25|10.30|80.56|343.49|353.79|-1070.01| +2451891|68417|1567|23797|1862325|4053|38181|2|52|1703|19|30.98|40.27|35.43|0.00|673.17|588.62|765.13|47.12|0.00|673.17|720.29|84.55| +2451891|68417|11473|23797|1862325|4053|38181|2|189|1703|17|5.14|7.81|4.13|32.99|70.21|87.38|132.77|2.60|32.99|37.22|39.82|-50.16| +2451891|68417|16055|23797|1862325|4053|38181|2|239|1703|18|11.51|15.65|4.38|0.00|78.84|207.18|281.70|5.51|0.00|78.84|84.35|-128.34| +2451891|68417|5005|23797|1862325|4053|38181|2|16|1703|51|8.93|15.00|15.00|0.00|765.00|455.43|765.00|38.25|0.00|765.00|803.25|309.57| +2451947|69720|2947|5751|1026165|1426|4240|7|23|1704|58|11.83|15.14|14.38|0.00|834.04|686.14|878.12|66.72|0.00|834.04|900.76|147.90| +2451947|69720|15837|5751|1026165|1426|4240|7|272|1704|95|93.01|114.40|82.36|0.00|7824.20|8835.95|10868.00|391.21|0.00|7824.20|8215.41|-1011.75| +2451947|69720|4579|5751|1026165|1426|4240|7|46|1704|64|20.22|22.84|3.19|0.00|204.16|1294.08|1461.76|16.33|0.00|204.16|220.49|-1089.92| +2451947|69720|5919|5751|1026165|1426|4240|7|98|1704|77|50.63|71.38|67.81|0.00|5221.37|3898.51|5496.26|0.00|0.00|5221.37|5221.37|1322.86| +2451947|69720|6229|5751|1026165|1426|4240|7|56|1704|34|63.30|112.67|28.16|258.50|957.44|2152.20|3830.78|62.90|258.50|698.94|761.84|-1453.26| +2451947|69720|11141|5751|1026165|1426|4240|7|257|1704|23|25.43|30.77|5.23|0.00|120.29|584.89|707.71|7.21|0.00|120.29|127.50|-464.60| +2451947|69720|3737|5751|1026165|1426|4240|7|93|1704|38|23.65|27.43|8.77|0.00|333.26|898.70|1042.34|0.00|0.00|333.26|333.26|-565.44| +2451947|69720|13471|5751|1026165|1426|4240|7|135|1704|7|81.35|128.53|20.56|0.00|143.92|569.45|899.71|7.19|0.00|143.92|151.11|-425.53| +2451947|69720|3877|5751|1026165|1426|4240|7|279|1704|1|93.97|110.88|16.63|0.00|16.63|93.97|110.88|0.49|0.00|16.63|17.12|-77.34| +2451320|36698|12115|43672|420861|897|13686|4|282|1705|20|66.35|122.08|69.58|0.00|1391.60|1327.00|2441.60|69.58|0.00|1391.60|1461.18|64.60| +2451320|36698|15613|43672|420861|897|13686|4|91|1705|98|62.74|97.24|30.14|2097.14|2953.72|6148.52|9529.52|0.00|2097.14|856.58|856.58|-5291.94| +2451320|36698|8696|43672|420861|897|13686|4|167|1705|20|92.64|166.75|153.41|0.00|3068.20|1852.80|3335.00|30.68|0.00|3068.20|3098.88|1215.40| +2451320|36698|12022|43672|420861|897|13686|4|300|1705|47|63.33|72.19|58.47|0.00|2748.09|2976.51|3392.93|137.40|0.00|2748.09|2885.49|-228.42| +2451320|36698|14254|43672|420861|897|13686|4|216|1705|48|74.99|119.23|56.03|0.00|2689.44|3599.52|5723.04|215.15|0.00|2689.44|2904.59|-910.08| +2451320|36698|8365|43672|420861|897|13686|4|266|1705|49|71.41|92.83|0.00|0.00|0.00|3499.09|4548.67|0.00|0.00|0.00|0.00|-3499.09| +2451320|36698|5188|43672|420861|897|13686|4|171|1705|84|3.25|4.55|0.27|21.77|22.68|273.00|382.20|0.00|21.77|0.91|0.91|-272.09| +2451320|36698|17408|43672|420861|897|13686|4|292|1705|30|19.05|22.66|1.58|0.00|47.40|571.50|679.80|0.00|0.00|47.40|47.40|-524.10| +2452209|37337|4133|17019|483588|4910|28279|4|126|1706|29|65.02|98.18|42.21|0.00|1224.09|1885.58|2847.22|73.44|0.00|1224.09|1297.53|-661.49| +2452209|37337|10473|17019|483588|4910|28279|4|34|1706|6|4.15|7.80|2.41|0.00|14.46|24.90|46.80|0.72|0.00|14.46|15.18|-10.44| +2452209|37337|10599|17019|483588|4910|28279|4|31|1706|4|61.88|86.63|36.38|0.00|145.52|247.52|346.52|2.91|0.00|145.52|148.43|-102.00| +2452209|37337|3975|17019|483588|4910|28279|4|248|1706|38|9.31|12.56|1.50|0.00|57.00|353.78|477.28|3.42|0.00|57.00|60.42|-296.78| +2452209|37337|15493|17019|483588|4910|28279|4|59|1706|68|92.88|147.67|20.67|0.00|1405.56|6315.84|10041.56|112.44|0.00|1405.56|1518.00|-4910.28| +2452209|37337|5619|17019|483588|4910|28279|4|285|1706|8|23.12|26.12|9.14|21.20|73.12|184.96|208.96|2.59|21.20|51.92|54.51|-133.04| +2452209|37337|8021|17019|483588|4910|28279|4|224|1706|17|22.10|34.69|16.65|0.00|283.05|375.70|589.73|11.32|0.00|283.05|294.37|-92.65| +2452209|37337|17021|17019|483588|4910|28279|4|118|1706|1|93.93|144.65|66.53|0.00|66.53|93.93|144.65|3.99|0.00|66.53|70.52|-27.40| +2452209|37337|1833|17019|483588|4910|28279|4|269|1706|15|87.89|116.01|29.00|0.00|435.00|1318.35|1740.15|30.45|0.00|435.00|465.45|-883.35| +2452209|37337|14835|17019|483588|4910|28279|4|158|1706|7|45.34|87.05|29.59|0.00|207.13|317.38|609.35|12.42|0.00|207.13|219.55|-110.25| +2452209|37337|2735|17019|483588|4910|28279|4|220|1706|22|22.89|39.37|21.65|0.00|476.30|503.58|866.14|14.28|0.00|476.30|490.58|-27.28| +2451987|69784|4289|14907|859684|6061|8438|4|91|1707|37|53.05|56.76|23.27|0.00|860.99|1962.85|2100.12|25.82|0.00|860.99|886.81|-1101.86| +2451987|69784|11267|14907|859684|6061|8438|4|163|1707|36|22.70|38.36|23.39|0.00|842.04|817.20|1380.96|16.84|0.00|842.04|858.88|24.84| +2451987|69784|12261|14907|859684|6061|8438|4|127|1707|28|80.94|148.12|93.31|0.00|2612.68|2266.32|4147.36|78.38|0.00|2612.68|2691.06|346.36| +2451987|69784|1667|14907|859684|6061|8438|4|299|1707|18|99.74|144.62|138.83|0.00|2498.94|1795.32|2603.16|0.00|0.00|2498.94|2498.94|703.62| +2451987|69784|13443|14907|859684|6061|8438|4|116|1707|93|13.08|25.63|13.84|0.00|1287.12|1216.44|2383.59|115.84|0.00|1287.12|1402.96|70.68| +2451987|69784|821|14907|859684|6061|8438|4|15|1707|23|74.42|119.07|69.06|0.00|1588.38|1711.66|2738.61|0.00|0.00|1588.38|1588.38|-123.28| +2451987|69784|6583|14907|859684|6061|8438|4|232|1707|21|9.49|15.37|9.68|93.50|203.28|199.29|322.77|8.78|93.50|109.78|118.56|-89.51| +2451987|69784|3177|14907|859684|6061|8438|4|31|1707|5|22.87|29.04|6.67|0.00|33.35|114.35|145.20|1.00|0.00|33.35|34.35|-81.00| +2451987|69784|2371|14907|859684|6061|8438|4|275|1707|46|2.92|3.59|0.43|0.00|19.78|134.32|165.14|0.79|0.00|19.78|20.57|-114.54| +2451987|69784|10423|14907|859684|6061|8438|4|46|1707|26|12.62|21.95|0.21|0.00|5.46|328.12|570.70|0.49|0.00|5.46|5.95|-322.66| +2451987|69784|2333|14907|859684|6061|8438|4|216|1707|55|69.34|70.03|19.60|0.00|1078.00|3813.70|3851.65|43.12|0.00|1078.00|1121.12|-2735.70| +2451987|69784|7429|14907|859684|6061|8438|4|128|1707|26|20.94|23.66|5.20|0.00|135.20|544.44|615.16|10.81|0.00|135.20|146.01|-409.24| +2451987|69784|8183|14907|859684|6061|8438|4|266|1707|26|62.78|102.33|6.13|0.00|159.38|1632.28|2660.58|9.56|0.00|159.38|168.94|-1472.90| +2451987|69784|2453|14907|859684|6061|8438|4|143|1707|96|97.67|192.40|111.59|535.63|10712.64|9376.32|18470.40|814.16|535.63|10177.01|10991.17|800.69| +2451987|69784|13645|14907|859684|6061|8438|4|132|1707|41|27.76|36.08|22.00|0.00|902.00|1138.16|1479.28|45.10|0.00|902.00|947.10|-236.16| +2451617|45683|10688|87957|665115|3520|49078|10|114|1708|75|40.69|45.97|15.62|0.00|1171.50|3051.75|3447.75|93.72|0.00|1171.50|1265.22|-1880.25| +2451617|45683|7235|87957|665115|3520|49078|10|71|1708|16|62.14|92.58|2.77|10.19|44.32|994.24|1481.28|3.07|10.19|34.13|37.20|-960.11| +2451617|45683|6319|87957|665115|3520|49078|10|264|1708|89|87.44|131.16|99.68|0.00|8871.52|7782.16|11673.24|709.72|0.00|8871.52|9581.24|1089.36| +2451617|45683|3511|87957|665115|3520|49078|10|36|1708|58|85.99|116.08|70.80|0.00|4106.40|4987.42|6732.64|287.44|0.00|4106.40|4393.84|-881.02| +2451617|45683|14021|87957|665115|3520|49078|10|146|1708|84|24.14|41.52|2.90|177.82|243.60|2027.76|3487.68|1.97|177.82|65.78|67.75|-1961.98| +2451617|45683|15077|87957|665115|3520|49078|10|223|1708|30|1.49|1.86|0.37|0.00|11.10|44.70|55.80|0.88|0.00|11.10|11.98|-33.60| +2451617|45683|4076|87957|665115|3520|49078|10|285|1708|96|21.95|34.02|23.81|0.00|2285.76|2107.20|3265.92|137.14|0.00|2285.76|2422.90|178.56| +2451617|45683|6776|87957|665115|3520|49078|10|257|1708|29|54.98|81.92|9.83|0.00|285.07|1594.42|2375.68|19.95|0.00|285.07|305.02|-1309.35| +2451617|45683|9391|87957|665115|3520|49078|10|275|1708|91|61.52|103.35|11.36|0.00|1033.76|5598.32|9404.85|0.00|0.00|1033.76|1033.76|-4564.56| +2451617|45683|17761|87957|665115|3520|49078|10|240|1708|58|21.67|26.00|3.38|0.00|196.04|1256.86|1508.00|1.96|0.00|196.04|198.00|-1060.82| +2451617|45683|10250|87957|665115|3520|49078|10|88|1708|36|32.82|37.74|22.26|0.00|801.36|1181.52|1358.64|24.04|0.00|801.36|825.40|-380.16| +2451617|45683|10208|87957|665115|3520|49078|10|84|1708|46|19.06|24.96|21.96|0.00|1010.16|876.76|1148.16|60.60|0.00|1010.16|1070.76|133.40| +2451617|45683|17579|87957|665115|3520|49078|10|286|1708|15|27.16|31.77|10.48|0.00|157.20|407.40|476.55|11.00|0.00|157.20|168.20|-250.20| +2451617|45683|8645|87957|665115|3520|49078|10|259|1708|9|98.70|176.67|127.20|0.00|1144.80|888.30|1590.03|45.79|0.00|1144.80|1190.59|256.50| +2451617|45683|16778|87957||3520||||1708|33||157.77|28.39||936.87|2908.62|5206.41|84.31||936.87|1021.18|| +2451824|32564|12869|90071|999317|116|962|4|40|1709|80|70.58|134.10|40.23|0.00|3218.40|5646.40|10728.00|257.47|0.00|3218.40|3475.87|-2428.00| +2451824|32564|17852|90071|999317|116|962|4|234|1709|8|13.87|20.80|20.59|0.00|164.72|110.96|166.40|0.00|0.00|164.72|164.72|53.76| +2451824|32564|1975|90071|999317|116|962|4|257|1709|78|87.93|104.63|50.22|0.00|3917.16|6858.54|8161.14|274.20|0.00|3917.16|4191.36|-2941.38| +2451824|32564|4001|90071|999317|116|962|4|287|1709|95|10.21|10.41|0.00|0.00|0.00|969.95|988.95|0.00|0.00|0.00|0.00|-969.95| +2451824|32564|4363|90071|999317|116|962|4|143|1709|58|72.44|86.92|46.06|0.00|2671.48|4201.52|5041.36|213.71|0.00|2671.48|2885.19|-1530.04| +2451824|32564|3631|90071|999317|116|962|4|162|1709|40|1.63|1.69|1.38|0.00|55.20|65.20|67.60|3.86|0.00|55.20|59.06|-10.00| +2451824|32564|4469|90071|999317|116|962|4|207|1709|59|76.10|142.30|22.76|0.00|1342.84|4489.90|8395.70|26.85|0.00|1342.84|1369.69|-3147.06| +2451824|32564|3134|90071|999317|116|962|4|32|1709|30|45.28|53.43|28.85|0.00|865.50|1358.40|1602.90|51.93|0.00|865.50|917.43|-492.90| +2451824|32564|9149|90071|999317|116|962|4|243|1709|65|52.19|66.28|42.41|0.00|2756.65|3392.35|4308.20|0.00|0.00|2756.65|2756.65|-635.70| +2451824|32564|644|90071|999317|116|962|4|52|1709|3|47.40|54.98|18.14|0.00|54.42|142.20|164.94|3.26|0.00|54.42|57.68|-87.78| +||10951|||116|||252|1709|26|83.49||3.13|0.00|81.38|||7.32|0.00||88.70|-2089.36| +2451808|33033|6803|16028|1329281|487|30893|4|70|1710|74|33.79|52.03|4.68|0.00|346.32|2500.46|3850.22|0.00|0.00|346.32|346.32|-2154.14| +2451808|33033|6926|16028|1329281|487|30893|4|255|1710|63|28.83|29.98|21.28|0.00|1340.64|1816.29|1888.74|0.00|0.00|1340.64|1340.64|-475.65| +2451808|33033|3659|16028|1329281|487|30893|4|258|1710|76|15.28|19.71|7.29|0.00|554.04|1161.28|1497.96|5.54|0.00|554.04|559.58|-607.24| +2451808|33033|13928|16028|1329281|487|30893|4|132|1710|77|52.12|96.94|1.93|0.00|148.61|4013.24|7464.38|2.97|0.00|148.61|151.58|-3864.63| +2451808|33033|7693|16028|1329281|487|30893|4|43|1710|10|60.82|65.68|57.14|142.85|571.40|608.20|656.80|0.00|142.85|428.55|428.55|-179.65| +2451808|33033|119|16028|1329281|487|30893|4|37|1710|55|11.44|16.70|3.84|0.00|211.20|629.20|918.50|14.78|0.00|211.20|225.98|-418.00| +2451808|33033|14312|16028|1329281|487|30893|4|185|1710|16|78.64|92.79|78.87|0.00|1261.92|1258.24|1484.64|100.95|0.00|1261.92|1362.87|3.68| +2451808|33033|6547|16028|1329281|487|30893|4|271|1710|16|68.15|99.49|70.63|0.00|1130.08|1090.40|1591.84|0.00|0.00|1130.08|1130.08|39.68| +2451808|33033|17315|16028|1329281|487|30893|4|293|1710|2|28.30|48.11|21.16|0.00|42.32|56.60|96.22|2.96|0.00|42.32|45.28|-14.28| +2451808|33033|10207|16028|1329281|487|30893|4|212|1710|36|4.14|4.51|1.89|0.00|68.04|149.04|162.36|0.00|0.00|68.04|68.04|-81.00| +2451808|33033|1327|16028|1329281|487|30893|4|258|1710|25|44.60|48.61|3.40|0.00|85.00|1115.00|1215.25|0.85|0.00|85.00|85.85|-1030.00| +2451808|33033|8227|16028|1329281|487|30893|4|76|1710|46|41.46|54.31|49.96|0.00|2298.16|1907.16|2498.26|137.88|0.00|2298.16|2436.04|391.00| +2451158|44751|7040|30082|1459066|6803|19787|7|40|1711|11|81.40|161.17|62.85|0.00|691.35|895.40|1772.87|41.48|0.00|691.35|732.83|-204.05| +2451158|44751|16540|30082|1459066|6803|19787|7|40|1711|95|5.47|8.80|0.00|0.00|0.00|519.65|836.00|0.00|0.00|0.00|0.00|-519.65| +2451158|44751|13042|30082|1459066|6803|19787|7|39|1711|91|3.44|6.39|5.81|0.00|528.71|313.04|581.49|0.00|0.00|528.71|528.71|215.67| +2451158|44751|8065|30082|1459066|6803|19787|7|135|1711|9|76.24|134.94|128.19|0.00|1153.71|686.16|1214.46|34.61|0.00|1153.71|1188.32|467.55| +2451158|44751|3010|30082|1459066|6803|19787|7|98|1711|54|60.44|71.92|41.71|0.00|2252.34|3263.76|3883.68|180.18|0.00|2252.34|2432.52|-1011.42| +2451158|44751|11426|30082|1459066|6803|19787|7|120|1711|15|75.68|97.62|85.90|0.00|1288.50|1135.20|1464.30|51.54|0.00|1288.50|1340.04|153.30| +2451158|44751|15715|30082|1459066|6803|19787|7|242|1711|4|85.20|164.43|139.76|0.00|559.04|340.80|657.72|39.13|0.00|559.04|598.17|218.24| +2451158|44751|8101|30082|1459066|6803|19787|7|182|1711|48|91.21|166.00|11.62|178.48|557.76|4378.08|7968.00|3.79|178.48|379.28|383.07|-3998.80| +2451158|44751|8024|30082|1459066|6803|19787|7|251|1711|99|43.00|81.27|81.27|0.00|8045.73|4257.00|8045.73|402.28|0.00|8045.73|8448.01|3788.73| +2451158|44751|8605|30082|1459066|6803|19787|7|266|1711|81|15.70|28.73|13.79|0.00|1116.99|1271.70|2327.13|78.18|0.00|1116.99|1195.17|-154.71| +2452088|30562|721|93463|1632546|2829|10712|8|34|1712|64|40.04|69.66|54.33|0.00|3477.12|2562.56|4458.24|243.39|0.00|3477.12|3720.51|914.56| +2452088|30562|16367|93463|1632546|2829|10712|8|253|1712|56|91.63|131.03|78.61|0.00|4402.16|5131.28|7337.68|308.15|0.00|4402.16|4710.31|-729.12| +2452088|30562|7583|93463|1632546|2829|10712|8|97|1712|66|41.55|45.70|4.57|153.82|301.62|2742.30|3016.20|13.30|153.82|147.80|161.10|-2594.50| +2452088|30562|3529|93463|1632546|2829|10712|8|208|1712|17|29.72|50.82|14.73|0.00|250.41|505.24|863.94|7.51|0.00|250.41|257.92|-254.83| +2452088|30562|15571|93463|1632546|2829|10712|8|131|1712|40|3.62|4.27|3.50|0.00|140.00|144.80|170.80|5.60|0.00|140.00|145.60|-4.80| +2452088|30562|9033|93463|1632546|2829|10712|8|7|1712|25|96.11|139.35|26.47|0.00|661.75|2402.75|3483.75|26.47|0.00|661.75|688.22|-1741.00| +2452088|30562|16761|93463|1632546|2829|10712|8|112|1712|51|35.85|41.58|17.46|0.00|890.46|1828.35|2120.58|26.71|0.00|890.46|917.17|-937.89| +2452088|30562|10147|93463|1632546|2829|10712|8|166|1712|37|75.73|105.26|37.89|0.00|1401.93|2802.01|3894.62|70.09|0.00|1401.93|1472.02|-1400.08| +2451844|48227|12362|27777|995240|1550|26613|1|280|1713|37|93.63|131.08|40.63|0.00|1503.31|3464.31|4849.96|0.00|0.00|1503.31|1503.31|-1961.00| +2451844|48227|10553|27777|995240|1550|26613|1|286|1713|98|96.78|157.75|58.36|0.00|5719.28|9484.44|15459.50|57.19|0.00|5719.28|5776.47|-3765.16| +2451844|48227|2129|27777|995240|1550|26613|1|199|1713|40|3.16|4.83|4.68|0.00|187.20|126.40|193.20|16.84|0.00|187.20|204.04|60.80| +|48227|4274|27777||1550||1||1713||78.68|93.62|52.42||||4212.90|||||| +2451844|48227|7889|27777|995240|1550|26613|1|60|1713|87|67.80|86.78|39.05|0.00|3397.35|5898.60|7549.86|0.00|0.00|3397.35|3397.35|-2501.25| +2451844|48227|16724|27777|995240|1550|26613|1|242|1713|40|24.37|41.42|24.43|0.00|977.20|974.80|1656.80|19.54|0.00|977.20|996.74|2.40| +2451844|48227|17165|27777|995240|1550|26613|1|75|1713|4|9.02|13.80|7.72|0.00|30.88|36.08|55.20|2.16|0.00|30.88|33.04|-5.20| +2451844|48227|3338|27777|995240|1550|26613|1|128|1713|21|55.71|96.93|0.00|0.00|0.00|1169.91|2035.53|0.00|0.00|0.00|0.00|-1169.91| +2451844|48227|2207|27777|995240|1550|26613|1|67|1713|88|98.00|102.90|96.72|0.00|8511.36|8624.00|9055.20|255.34|0.00|8511.36|8766.70|-112.64| +2451758||8948||140274|||||1714|37|63.30|||0.00|2335.81||4098.49||0.00|||-6.29| +2451758|74683|2317|64358|140274|990|10553|1|178|1714|87|14.83|29.06|23.53|0.00|2047.11|1290.21|2528.22|40.94|0.00|2047.11|2088.05|756.90| +2451758|74683|1999|64358|140274|990|10553|1|197|1714|100|80.84|98.62|9.86|0.00|986.00|8084.00|9862.00|29.58|0.00|986.00|1015.58|-7098.00| +2451758||17516|64358|140274||10553|||1714|||17.35|1.04|0.00|10.40|173.50|||0.00|10.40||-163.10| +2451758|74683|11942|64358|140274|990|10553|1|138|1714|42|83.47|132.71|19.90|710.43|835.80|3505.74|5573.82|3.76|710.43|125.37|129.13|-3380.37| +2451758|74683|1796|64358|140274|990|10553|1|274|1714|89|51.21|68.10|66.73|0.00|5938.97|4557.69|6060.90|296.94|0.00|5938.97|6235.91|1381.28| +2451758|74683|15470|64358|140274|990|10553|1|128|1714|15|97.01|131.93|36.94|0.00|554.10|1455.15|1978.95|33.24|0.00|554.10|587.34|-901.05| +2451758|74683|4982|64358|140274|990|10553|1|137|1714|19|75.06|81.81|44.99|0.00|854.81|1426.14|1554.39|68.38|0.00|854.81|923.19|-571.33| +2452533|63577|11454|61197|397230|2416|16802|2|167|1715|1|27.19|33.71|21.91|4.82|21.91|27.19|33.71|1.02|4.82|17.09|18.11|-10.10| +2452533||14154||397230||16802||249|1715||27.16|42.36|0.42||||127.08|0.01||1.26|1.27|| +2452533|63577|12426|61197|397230|2416|16802|2|57|1715|39|27.94|55.04|47.88|0.00|1867.32|1089.66|2146.56|130.71|0.00|1867.32|1998.03|777.66| +2452533|63577|189|61197|397230|2416|16802|2|223|1715|35|50.93|64.68|16.81|0.00|588.35|1782.55|2263.80|47.06|0.00|588.35|635.41|-1194.20| +2452533|63577|10687|61197|397230|2416|16802|2|246|1715|84|82.98|102.89|28.80|0.00|2419.20|6970.32|8642.76|48.38|0.00|2419.20|2467.58|-4551.12| +2452533|63577|14083|61197|397230|2416|16802|2|259|1715|5|5.05|6.71|4.56|15.73|22.80|25.25|33.55|0.63|15.73|7.07|7.70|-18.18| +2452533|63577|669|61197|||16802|2||1715||96.48|||||||169.97||8498.96|8668.93|| +2452533|63577|213|61197|397230|2416|16802|2|86|1715|43|96.85|161.73|116.44|2753.80|5006.92|4164.55|6954.39|45.06|2753.80|2253.12|2298.18|-1911.43| +2451429|68062|1886|38538|895861|3584|24860|1|250|1716|21|66.33|76.27|15.25|0.00|320.25|1392.93|1601.67|22.41|0.00|320.25|342.66|-1072.68| +2451429|68062|9031|38538|895861|3584|24860|1|65|1716|48|61.69|93.76|44.06|0.00|2114.88|2961.12|4500.48|148.04|0.00|2114.88|2262.92|-846.24| +2451429|68062|13424|38538|895861|3584|24860|1|104|1716|14|53.96|71.76|38.03|0.00|532.42|755.44|1004.64|10.64|0.00|532.42|543.06|-223.02| +2451429||6502|||3584|24860|1||1716|77||93.09||||4536.84||53.03||||766.92| +2451429|68062|13864|38538|895861|3584|24860|1|94|1716|77|16.11|19.17|7.09|152.86|545.93|1240.47|1476.09|7.86|152.86|393.07|400.93|-847.40| +2451429|68062|11575|38538|895861|3584|24860|1|122|1716|44|26.32|43.42|34.73|0.00|1528.12|1158.08|1910.48|45.84|0.00|1528.12|1573.96|370.04| +2451429|68062|6332|38538|895861|3584|24860|1|233|1716|76|73.53|76.47|63.47|0.00|4823.72|5588.28|5811.72|48.23|0.00|4823.72|4871.95|-764.56| +2451429|68062|9421|38538|895861|3584|24860|1|242|1716|3|55.81|85.38|50.37|0.00|151.11|167.43|256.14|9.06|0.00|151.11|160.17|-16.32| +2451429|68062|4378|38538|895861|3584|24860|1|292|1716|81|28.40|46.57|37.25|1267.24|3017.25|2300.40|3772.17|52.50|1267.24|1750.01|1802.51|-550.39| +2451429|68062|8047|38538|895861|3584|24860|1|293|1716|14|12.40|15.25|10.82|39.38|151.48|173.60|213.50|0.00|39.38|112.10|112.10|-61.50| +2451059|65464|830|65139|487206|5720|35834|10|212|1717|81|70.88|91.43|36.57|0.00|2962.17|5741.28|7405.83|88.86|0.00|2962.17|3051.03|-2779.11| +2451059|65464|17302|65139|487206|5720|35834|10|189|1717|4|11.16|13.50|5.26|20.19|21.04|44.64|54.00|0.02|20.19|0.85|0.87|-43.79| +2451059|65464|362|65139|487206|5720|35834|10|59|1717|55|4.73|4.96|3.72|0.00|204.60|260.15|272.80|18.41|0.00|204.60|223.01|-55.55| +2451059|65464|9344|65139|487206|5720|35834|10|68|1717|100|17.50|21.35|19.85|1746.80|1985.00|1750.00|2135.00|11.91|1746.80|238.20|250.11|-1511.80| +2451059||12013|65139||5720||||1717|67||91.74|33.94|0.00|2273.98||||0.00|||-1429.11| +2451059|65464|6550|65139|487206|5720|35834|10|186|1717|87|80.97|91.49|41.17|0.00|3581.79|7044.39|7959.63|214.90|0.00|3581.79|3796.69|-3462.60| +2451059|65464|16616|65139|487206|5720|35834|10|43|1717|61|11.70|15.09|14.93|0.00|910.73|713.70|920.49|72.85|0.00|910.73|983.58|197.03| +||11414|65139|487206|5720||10||1717|||46.49|36.26||||4230.59|197.97|||3497.63|188.37| +2451059|65464|3728|65139|487206|5720|35834|10|189|1717|1|44.63|58.91|1.76|0.00|1.76|44.63|58.91|0.12|0.00|1.76|1.88|-42.87| +2451059|65464|17446|65139|487206|5720|35834|10|28|1717|35|21.10|39.87|12.75|0.00|446.25|738.50|1395.45|13.38|0.00|446.25|459.63|-292.25| +2451059|65464|4610|65139|487206|5720|35834|10|300|1717|59|12.22|12.70|0.25|0.00|14.75|720.98|749.30|1.32|0.00|14.75|16.07|-706.23| +2451059|65464|5174|65139|487206|5720|35834|10|192|1717|20|17.19|19.76|7.50|145.50|150.00|343.80|395.20|0.04|145.50|4.50|4.54|-339.30| +2451059|65464|11467|65139|487206|5720|35834|10|227|1717|46|39.35|72.40|27.51|1189.53|1265.46|1810.10|3330.40|0.00|1189.53|75.93|75.93|-1734.17| +2451059|65464|12686|65139|487206|5720|35834|10|186|1717|80|86.98|161.78|161.78|0.00|12942.40|6958.40|12942.40|905.96|0.00|12942.40|13848.36|5984.00| +2451059|65464|15092|65139|487206|5720|35834|10|247|1717|65|47.76|65.43|10.46|0.00|679.90|3104.40|4252.95|13.59|0.00|679.90|693.49|-2424.50| +2451059|65464|1856|65139|487206|5720|35834|10|81|1717|3|17.14|19.53|8.20|0.00|24.60|51.42|58.59|1.23|0.00|24.60|25.83|-26.82| +||10246|||1017||7||1718||||47.88||1005.48||1069.74|||1005.48|1075.86|| +2451403|70523|2539|25296|600980|1017|20654|7|45|1718|19|26.12|26.38|19.25|0.00|365.75|496.28|501.22|21.94|0.00|365.75|387.69|-130.53| +2451403|70523|13496|25296|600980|1017|20654|7|96|1718|97|6.77|8.46|6.85|0.00|664.45|656.69|820.62|33.22|0.00|664.45|697.67|7.76| +2451403|70523|7184|25296|600980|1017|20654|7|290|1718|66|24.06|34.64|33.25|0.00|2194.50|1587.96|2286.24|131.67|0.00|2194.50|2326.17|606.54| +2451403|70523|7036|25296|600980|1017|20654|7|216|1718|8|65.37|120.28|75.77|72.73|606.16|522.96|962.24|32.00|72.73|533.43|565.43|10.47| +2451403|70523|13810|25296|600980|1017|20654|7|257|1718|35|61.05|81.19|22.73|0.00|795.55|2136.75|2841.65|39.77|0.00|795.55|835.32|-1341.20| +2451403|70523|13237|25296|600980|1017|20654|7|144|1718|22|34.79|37.92|18.96|0.00|417.12|765.38|834.24|8.34|0.00|417.12|425.46|-348.26| +2451403|70523|8341|25296|600980|1017|20654|7|72|1718|34|82.00|97.58|7.80|251.94|265.20|2788.00|3317.72|0.92|251.94|13.26|14.18|-2774.74| +2452546|39540|17412|7934|482273|4612|6551|7|53|1719|66|22.67|28.56|19.13|0.00|1262.58|1496.22|1884.96|50.50|0.00|1262.58|1313.08|-233.64| +2452546|39540|4893|7934|482273|4612|6551|7|190|1719|97|4.02|5.42|2.00|0.00|194.00|389.94|525.74|1.94|0.00|194.00|195.94|-195.94| +2452546|39540|16593|7934|482273|4612|6551|7|122|1719|67|2.79|3.96|3.36|0.00|225.12|186.93|265.32|15.75|0.00|225.12|240.87|38.19| +2452546|39540|793|7934|482273|4612|6551|7|18|1719|81|71.28|77.69|62.15|4279.02|5034.15|5773.68|6292.89|45.30|4279.02|755.13|800.43|-5018.55| +2452546|39540|15817|7934|482273|4612|6551|7|294|1719|9|95.85|185.94|9.29|0.00|83.61|862.65|1673.46|2.50|0.00|83.61|86.11|-779.04| +2452546|39540|17265|7934|482273|4612|6551|7|144|1719|60|49.92|83.86|47.80|0.00|2868.00|2995.20|5031.60|0.00|0.00|2868.00|2868.00|-127.20| +2452546|39540|11445|7934|482273|4612|6551|7|185|1719|73|92.85|143.91|123.76|0.00|9034.48|6778.05|10505.43|0.00|0.00|9034.48|9034.48|2256.43| +2452546|39540|9810|7934|482273|4612|6551|7|152|1719|78|23.34|37.11|33.77|131.70|2634.06|1820.52|2894.58|50.04|131.70|2502.36|2552.40|681.84| +2452546|39540|16903|7934|482273|4612|6551|7|174|1719|39|35.90|62.46|50.59|1795.43|1973.01|1400.10|2435.94|1.77|1795.43|177.58|179.35|-1222.52| +2452546|39540|3405|7934|482273|4612|6551|7|107|1719|92|96.18|144.27|8.65|493.39|795.80|8848.56|13272.84|0.00|493.39|302.41|302.41|-8546.15| +||13273||482273|4612|||181|1719|60|30.34||||2329.20||2949.00|||2329.20|2329.20|| +2452546|39540|17317|7934|482273|4612|6551|7|247|1719|37|32.36|36.56|16.08|553.31|594.96|1197.32|1352.72|1.24|553.31|41.65|42.89|-1155.67| +2452546|39540|14760|7934|482273|4612|6551|7|157|1719|99|83.50|136.94|91.74|0.00|9082.26|8266.50|13557.06|90.82|0.00|9082.26|9173.08|815.76| +2452546|39540|16428|7934|482273|4612|6551|7|12|1719|11|23.42|39.81|36.62|0.00|402.82|257.62|437.91|20.14|0.00|402.82|422.96|145.20| +2452546|39540|679|7934|482273||||192|1719|||53.60|39.66|||203.58|321.60|||||34.38| +2452597|38708|11028|90329|1889368|2360|43352|4|2|1720|33|69.99|70.68|67.14|0.00|2215.62|2309.67|2332.44|0.00|0.00|2215.62|2215.62|-94.05| +2452597|38708|8791|90329|1889368|2360|43352|4|169|1720|6|52.91|70.37|62.62|0.00|375.72|317.46|422.22|33.81|0.00|375.72|409.53|58.26| +2452597|38708|4986|90329|1889368|2360|43352|4|251|1720|78|11.11|14.88|4.31|0.00|336.18|866.58|1160.64|10.08|0.00|336.18|346.26|-530.40| +2452597|38708|2781|90329|1889368|2360|43352|4|241|1720|94|81.66|146.17|76.00|4072.08|7144.00|7676.04|13739.98|276.47|4072.08|3071.92|3348.39|-4604.12| +2452597|38708|3360|90329|1889368|2360|43352|4|132|1720|94|36.90|42.43|11.03|902.03|1036.82|3468.60|3988.42|1.34|902.03|134.79|136.13|-3333.81| +2452597|38708|2382|90329|1889368|2360|43352|4|231|1720|90|97.60|101.50|64.96|0.00|5846.40|8784.00|9135.00|409.24|0.00|5846.40|6255.64|-2937.60| +2452597|38708|4263|90329|1889368|2360|43352|4|62|1720|43|68.19|75.00|24.75|0.00|1064.25|2932.17|3225.00|95.78|0.00|1064.25|1160.03|-1867.92| +2452597|38708|6195|90329|1889368|2360|43352|4|244|1720|95|76.37|151.21|22.68|2154.60|2154.60|7255.15|14364.95|0.00|2154.60|0.00|0.00|-7255.15| +2452597|38708|6309|90329|1889368|2360|43352|4|92|1720|90|11.96|13.63|10.22|0.00|919.80|1076.40|1226.70|64.38|0.00|919.80|984.18|-156.60| +2452597|38708|6564|90329|1889368|2360|43352|4|136|1720|28|43.88|85.56|78.71|0.00|2203.88|1228.64|2395.68|176.31|0.00|2203.88|2380.19|975.24| +2452617|62312|14382|33387|639137|3734|16684|7|55|1721|85|61.19|97.90|74.40|0.00|6324.00|5201.15|8321.50|569.16|0.00|6324.00|6893.16|1122.85| +2452617|62312|3727|33387|639137|3734|16684|7|234|1721|97|12.06|21.46|1.93|0.00|187.21|1169.82|2081.62|13.10|0.00|187.21|200.31|-982.61| +2452617|62312|6948|33387|639137|3734|16684|7|274|1721|6|61.55|120.02|102.01|0.00|612.06|369.30|720.12|36.72|0.00|612.06|648.78|242.76| +2452617|62312|15099|33387|639137|3734|16684|7|53|1721|20|50.31|75.46|3.01|0.00|60.20|1006.20|1509.20|3.61|0.00|60.20|63.81|-946.00| +2452617|62312|14133|33387|639137|3734|16684|7|28|1721|72|99.08|157.53|42.53|0.00|3062.16|7133.76|11342.16|244.97|0.00|3062.16|3307.13|-4071.60| +2452617|62312|8949|33387|639137|3734|16684|7|150|1721|27|31.43|35.51|7.81|0.00|210.87|848.61|958.77|16.86|0.00|210.87|227.73|-637.74| +2452617|62312|2317|33387|639137|3734|16684|7|181|1721|30|4.18|8.06|6.12|0.00|183.60|125.40|241.80|9.18|0.00|183.60|192.78|58.20| +2452617|62312|1999|33387|639137|3734|16684|7|24|1721|60|42.84|64.68|14.22|0.00|853.20|2570.40|3880.80|76.78|0.00|853.20|929.98|-1717.20| +2452617|62312|17517|33387|639137|3734|16684|7|288|1721|95|95.29|113.39|96.38|0.00|9156.10|9052.55|10772.05|549.36|0.00|9156.10|9705.46|103.55| +||11943||||16684|||1721||44.32|67.80|36.61||||2373.00|||1281.35||| +2452617|62312|1797|33387|639137|3734|16684|7|85|1721|53|41.25|51.97|43.65|1665.68|2313.45|2186.25|2754.41|38.86|1665.68|647.77|686.63|-1538.48| +2452617|62312|15471|33387|639137|3734|16684|7|9|1721|34|76.14|148.47|93.53|0.00|3180.02|2588.76|5047.98|63.60|0.00|3180.02|3243.62|591.26| +2452617|62312|4983|33387|639137|3734|16684|7|168|1721|56|39.72|41.30|7.43|0.00|416.08|2224.32|2312.80|0.00|0.00|416.08|416.08|-1808.24| +2452129|34585|145|55069|172603|1337|19271|8|94|1722|51|31.17|39.27|21.20|1037.95|1081.20|1589.67|2002.77|3.46|1037.95|43.25|46.71|-1546.42| +2452129|34585|7243|55069|172603|1337|19271|8|219|1722|45|5.23|9.57|3.92|0.00|176.40|235.35|430.65|7.05|0.00|176.40|183.45|-58.95| +2452129|34585|15749|55069|172603|1337|19271|8|81|1722|68|91.31|168.01|117.60|0.00|7996.80|6209.08|11424.68|559.77|0.00|7996.80|8556.57|1787.72| +2452129|34585|14761|55069|172603|1337|19271|8|45|1722|49|85.24|87.79|49.16|0.00|2408.84|4176.76|4301.71|192.70|0.00|2408.84|2601.54|-1767.92| +2452129|34585|9333|55069|172603|1337|19271|8|152|1722|71|97.06|132.00|93.72|0.00|6654.12|6891.26|9372.00|266.16|0.00|6654.12|6920.28|-237.14| +2452129|34585|7193|55069|172603|1337|19271|8|167|1722|24|95.99|116.14|88.26|0.00|2118.24|2303.76|2787.36|169.45|0.00|2118.24|2287.69|-185.52| +2452129|34585|7851|55069|172603|1337|19271|8|128|1722|24|65.43|129.55|51.82|0.00|1243.68|1570.32|3109.20|12.43|0.00|1243.68|1256.11|-326.64| +2452129|34585|17641|55069|172603|1337|19271|8|25|1722|89|44.29|45.61|14.59|0.00|1298.51|3941.81|4059.29|12.98|0.00|1298.51|1311.49|-2643.30| +2452129|34585|4747|55069|172603|1337|19271|8|162|1722|78|60.32|101.33|0.00|0.00|0.00|4704.96|7903.74|0.00|0.00|0.00|0.00|-4704.96| +2452129|34585|3065|55069|172603|1337|19271|8|174|1722|67|64.87|91.46|42.98|0.00|2879.66|4346.29|6127.82|28.79|0.00|2879.66|2908.45|-1466.63| +2452129|34585|12867|55069|172603|1337|19271|8|99|1722|43|26.78|45.79|21.52|0.00|925.36|1151.54|1968.97|46.26|0.00|925.36|971.62|-226.18| +2452138|34989|17165|72829|1743585|741|25097|8|8|1723|27|53.52|90.44|83.20|2066.68|2246.40|1445.04|2441.88|8.98|2066.68|179.72|188.70|-1265.32| +2452138|34989|3339|72829|1743585|741|25097|8|46|1723|8|95.08|131.21|27.55|0.00|220.40|760.64|1049.68|15.42|0.00|220.40|235.82|-540.24| +2452138|34989|2207|72829|1743585|741|25097|8|42|1723|19|28.66|46.71|42.97|0.00|816.43|544.54|887.49|48.98|0.00|816.43|865.41|271.89| +2452138|34989|437|72829|1743585|741|25097|8|84|1723|87|65.29|114.25|3.42|0.00|297.54|5680.23|9939.75|14.87|0.00|297.54|312.41|-5382.69| +2452138|34989|11877||1743585||25097|8|113|1723|||79.73||0.00|4922.19||||0.00|4922.19||| +2452138|34989|14271|72829|1743585|741|25097|8|102|1723|56|9.16|11.54|4.61|0.00|258.16|512.96|646.24|15.48|0.00|258.16|273.64|-254.80| +2452138|34989|3725|72829|1743585|741|25097|8|7|1723|46|63.32|112.07|70.60|0.00|3247.60|2912.72|5155.22|227.33|0.00|3247.60|3474.93|334.88| +2452138|34989|6687|72829|1743585|741|25097|8|45|1723|29|28.30|44.99|6.74|0.00|195.46|820.70|1304.71|3.90|0.00|195.46|199.36|-625.24| +2452138|34989|237|72829|1743585|741|25097|8|64|1723|5|66.71|70.71|13.43|0.67|67.15|333.55|353.55|1.99|0.67|66.48|68.47|-267.07| +2452138|34989|1735|72829|1743585|741|25097|8|259|1723|100|61.21|113.85|52.37|0.00|5237.00|6121.00|11385.00|471.33|0.00|5237.00|5708.33|-884.00| +2452138|34989|9533|72829|1743585|741|25097|8|100|1723|42|34.76|43.79|23.20|0.00|974.40|1459.92|1839.18|9.74|0.00|974.40|984.14|-485.52| +2452138|34989|1223|72829|1743585|741|25097|8|78|1723|33|51.83|93.81|93.81|2507.54|3095.73|1710.39|3095.73|35.29|2507.54|588.19|623.48|-1122.20| +2452138|34989|12143|72829|1743585|741|25097|8|201|1723|60|80.84|155.21|149.00|8046.00|8940.00|4850.40|9312.60|53.64|8046.00|894.00|947.64|-3956.40| +|34989|13251||1743585||25097|||1723||43.99||2.95|0.00|262.55|||21.00|0.00|||-3652.56| +2452138|34989|12335|72829|1743585|741|25097|8|138|1723|23|93.44|152.30|71.58|1004.26|1646.34|2149.12|3502.90|38.52|1004.26|642.08|680.60|-1507.04| +2452639|67064|2373|64949|825651|3953|17055|4|260|1724|58|9.11|16.21|14.42|0.00|836.36|528.38|940.18|0.00|0.00|836.36|836.36|307.98| +2452639|67064|9123|64949|825651|3953|17055|4|165|1724|29|4.29|5.40|0.97|0.00|28.13|124.41|156.60|2.53|0.00|28.13|30.66|-96.28| +2452639|67064|7245|64949|825651|3953|17055|4|289|1724|63|12.98|15.83|5.69|0.00|358.47|817.74|997.29|25.09|0.00|358.47|383.56|-459.27| +2452639|67064|7489|64949|825651|3953|17055|4|81|1724|4|28.55|47.96|11.51|0.00|46.04|114.20|191.84|0.92|0.00|46.04|46.96|-68.16| +2452639|67064|1657|64949|825651|3953|17055|4|41|1724|75|58.47|69.57|67.48|1366.47|5061.00|4385.25|5217.75|332.50|1366.47|3694.53|4027.03|-690.72| +2452639|67064|3591|64949|825651|3953|17055|4|150|1724|61|70.98|96.53|15.44|0.00|941.84|4329.78|5888.33|75.34|0.00|941.84|1017.18|-3387.94| +2452639|67064|10537|64949|825651|3953|17055|4|219|1724|96|80.01|116.81|58.40|953.08|5606.40|7680.96|11213.76|186.13|953.08|4653.32|4839.45|-3027.64| +||153|64949|825651|||4||1724|34|||55.79|0.00||2727.14||132.78|0.00|||-830.28| +2452639|67064|15273|64949|825651|3953|17055|4|289|1724|94|90.98|157.39|9.44|638.89|887.36|8552.12|14794.66|2.48|638.89|248.47|250.95|-8303.65| +2452639|67064|8220|64949|825651|3953|17055|4|264|1724|32|94.86|110.03|86.92|0.00|2781.44|3035.52|3520.96|27.81|0.00|2781.44|2809.25|-254.08| +2452608|65523|11556|15140|1296099|1058|8817|2|248|1725|9|68.97|91.04|62.81|0.00|565.29|620.73|819.36|39.57|0.00|565.29|604.86|-55.44| +2452608|65523|10224|15140|1296099|1058|8817|2|57|1725|23|73.96|113.15|80.33|0.00|1847.59|1701.08|2602.45|18.47|0.00|1847.59|1866.06|146.51| +2452608|65523|14967|15140|1296099|1058|8817|2|157|1725|29|18.89|30.41|14.59|0.00|423.11|547.81|881.89|38.07|0.00|423.11|461.18|-124.70| +2452608|65523|8928|15140|1296099|1058|8817|2|3|1725|69|42.97|76.48|38.24|0.00|2638.56|2964.93|5277.12|105.54|0.00|2638.56|2744.10|-326.37| +2452608|65523|16039|15140|1296099|1058|8817|2|169|1725|58|67.81|71.87|64.68|0.00|3751.44|3932.98|4168.46|75.02|0.00|3751.44|3826.46|-181.54| +2452608|65523|14955|15140|1296099|1058|8817|2|124|1725|54|69.97|103.55|88.01|0.00|4752.54|3778.38|5591.70|47.52|0.00|4752.54|4800.06|974.16| +2452608|65523|6121|15140|1296099|1058|8817|2|116|1725|42|45.76|72.30|10.84|0.00|455.28|1921.92|3036.60|13.65|0.00|455.28|468.93|-1466.64| +2452608|65523|2904|15140|1296099|1058|8817|2|85|1725|89|40.80|42.43|38.18|0.00|3398.02|3631.20|3776.27|203.88|0.00|3398.02|3601.90|-233.18| +2452608|65523|11730|15140|1296099|1058|8817|2|240|1725|62|3.31|4.53|0.27|2.00|16.74|205.22|280.86|1.03|2.00|14.74|15.77|-190.48| +2452608|65523|6168|15140|1296099|1058|8817|2|73|1725|84|44.72|88.54|56.66|0.00|4759.44|3756.48|7437.36|95.18|0.00|4759.44|4854.62|1002.96| +2452608|65523|16113|15140|1296099|1058|8817|2|262|1725|32|82.37|150.73|132.64|0.00|4244.48|2635.84|4823.36|42.44|0.00|4244.48|4286.92|1608.64| +2452608|65523|12511|15140|1296099|1058|8817|2|194|1725|25|14.50|26.39|0.52|0.00|13.00|362.50|659.75|1.04|0.00|13.00|14.04|-349.50| +2451425|50840|8944|49791|1346118|4045|36019|2|291|1726|6|71.18|90.39|28.92|0.00|173.52|427.08|542.34|0.00|0.00|173.52|173.52|-253.56| +2451425|50840|10568|49791|1346118|4045|36019|2|57|1726|6|22.95|42.45|5.51|0.00|33.06|137.70|254.70|1.98|0.00|33.06|35.04|-104.64| +2451425|50840|17528|49791|1346118|4045|36019|2|53|1726|82|79.65|114.69|96.33|0.00|7899.06|6531.30|9404.58|710.91|0.00|7899.06|8609.97|1367.76| +2451425|50840|4846|49791|1346118|4045|36019|2|162|1726|41|38.45|52.67|35.28|0.00|1446.48|1576.45|2159.47|0.00|0.00|1446.48|1446.48|-129.97| +2451425|50840|490|49791|1346118|4045|36019|2|295|1726|14|15.87|16.82|15.64|190.49|218.96|222.18|235.48|1.70|190.49|28.47|30.17|-193.71| +2451425|50840|8014|49791|1346118|4045|36019|2|67|1726|39|99.21|117.06|97.15|0.00|3788.85|3869.19|4565.34|113.66|0.00|3788.85|3902.51|-80.34| +2451425|50840|13952|49791|1346118|4045|36019|2|57|1726|70|51.81|91.70|51.35|1761.30|3594.50|3626.70|6419.00|109.99|1761.30|1833.20|1943.19|-1793.50| +2451425|50840|6619|49791|1346118|4045|36019|2|41|1726|64|14.53|15.40|3.38|0.00|216.32|929.92|985.60|0.00|0.00|216.32|216.32|-713.60| +2451425|50840|10778|49791|1346118|4045|36019|2|245|1726|36|28.48|40.44|17.38|431.71|625.68|1025.28|1455.84|3.87|431.71|193.97|197.84|-831.31| +2452255|64267|12277|29910|1273079|248|23864|8|245|1727|22|58.42|85.87|51.52|453.37|1133.44|1285.24|1889.14|40.80|453.37|680.07|720.87|-605.17| +2452255|64267|14551|29910|1273079|248|23864|8|90|1727|25|59.75|68.71|54.28|0.00|1357.00|1493.75|1717.75|13.57|0.00|1357.00|1370.57|-136.75| +||16359|29910|1273079||23864||287|1727|74|72.07||33.44||2474.56||7733.00|||2474.56|2647.77|| +2452255|64267|13027|29910|1273079|248|23864|8|133|1727|45|94.08|157.11|43.99|296.93|1979.55|4233.60|7069.95|134.60|296.93|1682.62|1817.22|-2550.98| +2452255|64267|3325|29910|1273079|248|23864|8|279|1727|78|83.92|92.31|1.84|0.00|143.52|6545.76|7200.18|7.17|0.00|143.52|150.69|-6402.24| +2452255|64267|5637|29910|1273079|248|23864|8|143|1727|22|19.56|28.16|17.74|0.00|390.28|430.32|619.52|19.51|0.00|390.28|409.79|-40.04| +2452255|64267|8759|29910|1273079|248|23864|8|253|1727|19|6.59|13.18|5.79|0.00|110.01|125.21|250.42|3.30|0.00|110.01|113.31|-15.20| +2452255|64267|1367|29910|1273079|248|23864|8|7|1727|25|7.13|8.62|6.20|0.00|155.00|178.25|215.50|4.65|0.00|155.00|159.65|-23.25| +2452175|63740|12983|78951|222009|267|35772|2|267|1728|34|12.61|17.90|17.36|0.00|590.24|428.74|608.60|35.41|0.00|590.24|625.65|161.50| +2452175|63740|9115|78951|222009|267|35772|2|136|1728|8|88.25|142.96|74.33|0.00|594.64|706.00|1143.68|5.94|0.00|594.64|600.58|-111.36| +2452175|63740|855|78951|222009|267|35772|2|30|1728|54|86.38|109.70|106.40|2183.32|5745.60|4664.52|5923.80|0.00|2183.32|3562.28|3562.28|-1102.24| +2452175|63740|17113|78951|222009|267|35772|2|119|1728|18|60.15|109.47|41.59|0.00|748.62|1082.70|1970.46|22.45|0.00|748.62|771.07|-334.08| +2452175|63740|1517|78951|222009|267|35772|2|248|1728|96|17.29|20.92|14.01|0.00|1344.96|1659.84|2008.32|13.44|0.00|1344.96|1358.40|-314.88| +2452175|63740|14371|78951|222009|267|35772|2|1|1728|79|12.99|15.84|0.47|7.05|37.13|1026.21|1251.36|0.90|7.05|30.08|30.98|-996.13| +2452175|63740|2149|78951|222009|267|35772|2|275|1728|31|15.57|26.31|17.36|0.00|538.16|482.67|815.61|16.14|0.00|538.16|554.30|55.49| +2452175|63740|15255|78951|222009|267|35772|2|163|1728|4|35.68|36.39|22.56|0.00|90.24|142.72|145.56|4.51|0.00|90.24|94.75|-52.48| +2452215|62619|16261|79090|697517|3144|41971|2|83|1729|62|49.19|61.48|46.11|0.00|2858.82|3049.78|3811.76|0.00|0.00|2858.82|2858.82|-190.96| +2452215|62619|8505|79090|697517|3144|41971|2|12|1729|15|63.77|75.24|52.66|0.00|789.90|956.55|1128.60|15.79|0.00|789.90|805.69|-166.65| +2452215|62619|14473|79090|697517|3144|41971|2|264|1729|5|11.10|13.54|2.03|7.81|10.15|55.50|67.70|0.18|7.81|2.34|2.52|-53.16| +|62619|6889|79090||3144|||142|1729|39|43.07|72.35|||1269.45||2821.65|88.86||1269.45||| +2452215|62619|4171|79090|697517|3144|41971|2|276|1729|10|83.18|93.99|93.05|0.00|930.50|831.80|939.90|55.83|0.00|930.50|986.33|98.70| +2452215|62619|16791|79090|697517|3144|41971|2|189|1729|76|6.12|8.69|3.47|0.00|263.72|465.12|660.44|18.46|0.00|263.72|282.18|-201.40| +2452215|62619|2961|79090|697517|3144|41971|2|122|1729|95|3.29|3.58|0.96|54.72|91.20|312.55|340.10|1.45|54.72|36.48|37.93|-276.07| +2452215|62619|15429|79090|697517|3144|41971|2|41|1729|45|47.85|73.68|49.36|0.00|2221.20|2153.25|3315.60|155.48|0.00|2221.20|2376.68|67.95| +2452215|62619|9825|79090|697517|3144|41971|2|281|1729|90|92.96|102.25|81.80|0.00|7362.00|8366.40|9202.50|73.62|0.00|7362.00|7435.62|-1004.40| +2452215|62619|6727|79090|697517|3144|41971|2|243|1729|15|94.18|120.55|67.50|0.00|1012.50|1412.70|1808.25|20.25|0.00|1012.50|1032.75|-400.20| +2452215|62619|4925|79090|697517|3144|41971|2|168|1729|56|38.22|43.57|32.24|1751.27|1805.44|2140.32|2439.92|3.79|1751.27|54.17|57.96|-2086.15| +2451446|45861|2464|6527|684626|2372|25318|10|8|1730|51|6.86|11.66|8.27|0.00|421.77|349.86|594.66|25.30|0.00|421.77|447.07|71.91| +2451446|45861|8516|6527|684626|2372|25318|10|1|1730|71|97.55|129.74|124.55|0.00|8843.05|6926.05|9211.54|0.00|0.00|8843.05|8843.05|1917.00| +2451446|45861|14362|6527|684626|2372|25318|10|5|1730|9|50.57|100.12|46.05|0.00|414.45|455.13|901.08|20.72|0.00|414.45|435.17|-40.68| +2451446|45861|2560|6527|684626|2372|25318|10|118|1730|50|77.73|120.48|22.89|1030.05|1144.50|3886.50|6024.00|4.57|1030.05|114.45|119.02|-3772.05| +2451446|45861|14704|6527|684626|2372|25318|10|55|1730|35|47.72|74.92|57.68|847.89|2018.80|1670.20|2622.20|35.12|847.89|1170.91|1206.03|-499.29| +2451446|45861|3973|6527|684626|2372|25318|10|281|1730|47|82.63|99.98|52.98|0.00|2490.06|3883.61|4699.06|224.10|0.00|2490.06|2714.16|-1393.55| +2451446|45861|6070|6527|684626|2372|25318|10|129|1730|42|84.08|122.75|28.23|0.00|1185.66|3531.36|5155.50|71.13|0.00|1185.66|1256.79|-2345.70| +2451446|45861|6883|6527|684626|2372|25318|10|157|1730|86|88.61|172.78|48.37|0.00|4159.82|7620.46|14859.08|374.38|0.00|4159.82|4534.20|-3460.64| +2451446|45861|380|6527|684626|2372|25318|10|96|1730|29|71.46|76.46|19.87|0.00|576.23|2072.34|2217.34|17.28|0.00|576.23|593.51|-1496.11| +2451446|45861|14324|6527|684626|2372|25318|10|175|1730|3|13.44|24.32|17.26|0.00|51.78|40.32|72.96|1.55|0.00|51.78|53.33|11.46| +2451446|45861|9661||684626|2372|||253|1730||||13.83|||11.72||||13.83||| +2451446|45861|14768|6527|684626|2372|25318|10|78|1730|90|57.83|70.55|64.90|5023.26|5841.00|5204.70|6349.50|57.24|5023.26|817.74|874.98|-4386.96| +2451446|45861|15685|6527|684626|2372|25318|10|161|1730|70|74.82|148.14|78.51|0.00|5495.70|5237.40|10369.80|384.69|0.00|5495.70|5880.39|258.30| +2451446|45861|3446|6527|684626|2372|25318|10|22|1730|82|67.75|71.13|8.53|0.00|699.46|5555.50|5832.66|55.95|0.00|699.46|755.41|-4856.04| +2451446|45861|15638|6527|684626|2372|25318|10|101|1730|90|72.67|88.65|30.14|0.00|2712.60|6540.30|7978.50|189.88|0.00|2712.60|2902.48|-3827.70| +2451446|45861|10438|6527|684626|2372|25318|10|70|1730|2|86.63|123.88|37.16|0.00|74.32|173.26|247.76|5.94|0.00|74.32|80.26|-98.94| +2452208|58194|6535|12121|1769897|3320|40296|7|170|1731|22|95.07|96.97|8.72|118.94|191.84|2091.54|2133.34|2.91|118.94|72.90|75.81|-2018.64| +2452208|58194|2725|12121|1769897|3320|40296|7|134|1731|34|30.60|45.90|0.91|0.00|30.94|1040.40|1560.60|1.23|0.00|30.94|32.17|-1009.46| +2452208|58194|3055|12121|1769897|3320|40296|7|27|1731|48|79.49|155.00|12.40|0.00|595.20|3815.52|7440.00|41.66|0.00|595.20|636.86|-3220.32| +2452208|58194|1727|12121|1769897|3320|40296|7|260|1731|17|29.84|30.43|15.51|171.38|263.67|507.28|517.31|3.69|171.38|92.29|95.98|-414.99| +2452208|58194|1029|12121|1769897|3320|40296|7|225|1731|16|42.24|68.00|31.96|0.00|511.36|675.84|1088.00|46.02|0.00|511.36|557.38|-164.48| +2452208|58194|13469|12121|1769897|3320||||1731||||14.15|0.00|||||0.00|||| +2452208|58194|8783|12121|1769897|3320|40296|7|1|1731|90|8.48|12.88|1.41|0.00|126.90|763.20|1159.20|3.80|0.00|126.90|130.70|-636.30| +||17441||1769897|3320|40296|||1731|61||178.35|10.70||652.70|5849.29||||||-5196.59| +2452208|58194|11461|12121|1769897|3320|40296|7|196|1731|86|39.12|50.07|50.07|0.00|4306.02|3364.32|4306.02|258.36|0.00|4306.02|4564.38|941.70| +2452208|58194|6541|12121|1769897|3320|40296|7|147|1731|90|89.88|156.39|136.05|0.00|12244.50|8089.20|14075.10|979.56|0.00|12244.50|13224.06|4155.30| +||10308|35529|||13076||108|1732|100|81.03||45.52|4415.44|4552.00||8589.00||4415.44|136.56||| +2452320|32594|4032|35529|1310978|5627|13076|4|296|1732|95|3.95|4.30|3.69|0.00|350.55|375.25|408.50|31.54|0.00|350.55|382.09|-24.70| +2452320|32594|17463|35529|1310978|5627|13076|4|203|1732|45|20.18|26.03|22.12|776.41|995.40|908.10|1171.35|17.51|776.41|218.99|236.50|-689.11| +2452320|32594|16245|35529|1310978|5627|13076|4|207|1732|53|11.29|16.93|5.24|0.00|277.72|598.37|897.29|22.21|0.00|277.72|299.93|-320.65| +2452320|32594|10425|35529|1310978|5627|13076|4|248|1732|78|87.65|99.04|29.71|0.00|2317.38|6836.70|7725.12|0.00|0.00|2317.38|2317.38|-4519.32| +2452320|32594|15609|35529|1310978|5627|13076|4|123|1732|4|3.35|3.95|1.50|0.00|6.00|13.40|15.80|0.24|0.00|6.00|6.24|-7.40| +2452320|32594|9253|35529|1310978|5627|13076|4|76|1732|2|25.41|46.75|15.42|0.00|30.84|50.82|93.50|2.77|0.00|30.84|33.61|-19.98| +2452320|32594|17502|35529|1310978|5627|13076|4|272|1732|97|83.15|128.88|91.50|0.00|8875.50|8065.55|12501.36|0.00|0.00|8875.50|8875.50|809.95| +2452320|32594|289|35529|1310978|5627|13076|4|165|1732|44|47.83|83.22|56.58|0.00|2489.52|2104.52|3661.68|199.16|0.00|2489.52|2688.68|385.00| +2452320|32594|9993|35529|1310978|5627|13076|4|232|1732|32|84.69|136.35|84.53|0.00|2704.96|2710.08|4363.20|27.04|0.00|2704.96|2732.00|-5.12| +||16987||||13076|4|268|1732|||13.82|5.52|0.00||527.34|912.12||0.00|364.32|382.53|| +2452320|32594|9678|35529|1310978|5627|13076|4|73|1732|99|55.78|102.07|92.88|0.00|9195.12|5522.22|10104.93|275.85|0.00|9195.12|9470.97|3672.90| +2452320|32594|9271|35529|1310978|5627|13076|4|87|1732|98|80.79|107.45|69.84|0.00|6844.32|7917.42|10530.10|410.65|0.00|6844.32|7254.97|-1073.10| +2452320|32594|16989|35529|1310978|5627|13076|4|285|1732|3|16.86|25.96|7.00|1.26|21.00|50.58|77.88|0.59|1.26|19.74|20.33|-30.84| +2451861|50722|14786|88577|80564|5704|21691|7|169|1733|41|48.73|97.46|49.70|0.00|2037.70|1997.93|3995.86|40.75|0.00|2037.70|2078.45|39.77| +2451861|50722|5953|88577|80564|5704|21691|7|65|1733|90|73.18|124.40|87.08|1253.95|7837.20|6586.20|11196.00|131.66|1253.95|6583.25|6714.91|-2.95| +2451861|50722|9085|88577|80564|5704|21691|7|151|1733|20|22.20|26.86|19.07|0.00|381.40|444.00|537.20|19.07|0.00|381.40|400.47|-62.60| +2451861|50722|4796|88577|80564|5704|21691|7|248|1733|16|67.08|130.13|108.00|1365.12|1728.00|1073.28|2082.08|18.14|1365.12|362.88|381.02|-710.40| +2451861|50722|1249|88577|80564|5704|21691|7|100|1733|90|77.64|103.26|44.40|0.00|3996.00|6987.60|9293.40|239.76|0.00|3996.00|4235.76|-2991.60| +2451861|50722|5561|88577|80564|5704|21691|7|224|1733|57|4.68|8.98|6.10|0.00|347.70|266.76|511.86|17.38|0.00|347.70|365.08|80.94| +2451861|50722|845|88577|80564|5704|21691|7|164|1733|23|29.28|39.52|0.39|0.00|8.97|673.44|908.96|0.44|0.00|8.97|9.41|-664.47| +2451861|50722|12883|88577|80564|5704|21691|7|175|1733|16|42.45|78.53|26.70|0.00|427.20|679.20|1256.48|34.17|0.00|427.20|461.37|-252.00| +2451861|50722|3998|88577|80564|5704|21691|7|182|1733|67|33.37|56.06|51.01|0.00|3417.67|2235.79|3756.02|136.70|0.00|3417.67|3554.37|1181.88| +2451861|50722|14606|88577|80564|5704|21691|7|125|1733|83|69.43|116.64|1.16|0.00|96.28|5762.69|9681.12|4.81|0.00|96.28|101.09|-5666.41| +2451861|50722|8174|88577|80564|5704|21691|7|137|1733|56|79.39|142.10|9.94|0.00|556.64|4445.84|7957.60|38.96|0.00|556.64|595.60|-3889.20| +2451530|44205|5836|18104|1860390|2676|25315|1|12|1734|59|65.40|123.60|49.44|0.00|2916.96|3858.60|7292.40|29.16|0.00|2916.96|2946.12|-941.64| +2451530|44205|4879|18104|1860390|2676|25315|1|229|1734|60|98.18|125.67|36.44|0.00|2186.40|5890.80|7540.20|87.45|0.00|2186.40|2273.85|-3704.40| +2451530|44205|15643|18104|1860390|2676|25315|1|180|1734|56|16.27|23.42|18.26|828.27|1022.56|911.12|1311.52|17.48|828.27|194.29|211.77|-716.83| +2451530|44205|12880|18104|1860390|2676|25315|1|176|1734|47|5.19|7.42|4.15|0.00|195.05|243.93|348.74|7.80|0.00|195.05|202.85|-48.88| +2451530|44205|14101|18104|1860390|2676|25315|1|61|1734|72|2.12|2.94|1.99|128.95|143.28|152.64|211.68|1.28|128.95|14.33|15.61|-138.31| +2451530|44205|11854|18104|1860390|2676|25315|1|286|1734|96|9.37|13.11|3.40|0.00|326.40|899.52|1258.56|29.37|0.00|326.40|355.77|-573.12| +2451530|44205|5146|18104|1860390|2676|25315|1|152|1734|53|93.97|139.07|9.73|0.00|515.69|4980.41|7370.71|15.47|0.00|515.69|531.16|-4464.72| +2451530|44205|8930|18104|1860390|2676|25315|1|223|1734|87|47.64|68.12|15.66|790.20|1362.42|4144.68|5926.44|45.77|790.20|572.22|617.99|-3572.46| +2451530|44205|13370|18104|1860390|2676|25315|1|3|1734|73|37.68|66.31|37.79|0.00|2758.67|2750.64|4840.63|82.76|0.00|2758.67|2841.43|8.03| +2451530|44205|11047|18104|1860390|2676|25315|1|80|1734|34|12.86|16.20|1.94|0.00|65.96|437.24|550.80|2.63|0.00|65.96|68.59|-371.28| +2452202|47514|2813|56787|1083076|2965|32407|10|158|1735|77|51.66|64.05|33.30|0.00|2564.10|3977.82|4931.85|128.20|0.00|2564.10|2692.30|-1413.72| +2452202|47514|17379|56787|1083076|2965|32407|10|8|1735|69|54.04|69.17|21.44|0.00|1479.36|3728.76|4772.73|133.14|0.00|1479.36|1612.50|-2249.40| +2452202|47514|10087|56787|1083076|2965|32407|10|134|1735|81|58.93|100.18|16.02|778.57|1297.62|4773.33|8114.58|41.52|778.57|519.05|560.57|-4254.28| +2452202|47514|5889|56787|1083076|2965|32407|10|63|1735|98|93.31|183.82|97.42|0.00|9547.16|9144.38|18014.36|477.35|0.00|9547.16|10024.51|402.78| +2452202|47514|16627|56787|1083076|2965|32407|10|102|1735|63|46.60|64.77|27.20|0.00|1713.60|2935.80|4080.51|85.68|0.00|1713.60|1799.28|-1222.20| +2452202|47514|15455|56787|1083076|2965|32407|10|74|1735|14|64.72|87.37|26.21|0.00|366.94|906.08|1223.18|18.34|0.00|366.94|385.28|-539.14| +2452202|47514|12207|56787|1083076|2965|32407|10|43|1735|100|86.34|98.42|76.76|3147.16|7676.00|8634.00|9842.00|45.28|3147.16|4528.84|4574.12|-4105.16| +2452202|47514|16995|56787|1083076|2965|32407|10|165|1735|62|31.96|43.46|33.02|0.00|2047.24|1981.52|2694.52|184.25|0.00|2047.24|2231.49|65.72| +2452202|47514|14025|56787|1083076|2965|32407|10|227|1735|98|87.54|175.08|134.81|0.00|13211.38|8578.92|17157.84|660.56|0.00|13211.38|13871.94|4632.46| +||14353|56787|1083076||32407|||1735||||0.65|1.01|||||1.01|49.69|52.67|| +2451807|32396|14828|65902|902589|5676|22363|10|78|1736|66|13.67|17.90|15.39|0.00|1015.74|902.22|1181.40|50.78|0.00|1015.74|1066.52|113.52| +2451807|32396|602|65902|902589|5676|22363|10|5|1736|65|87.46|111.94|45.89|1431.76|2982.85|5684.90|7276.10|46.53|1431.76|1551.09|1597.62|-4133.81| +2451807|32396|12547|65902|902589|5676|22363|10|134|1736|76|68.63|126.96|93.95|0.00|7140.20|5215.88|9648.96|499.81|0.00|7140.20|7640.01|1924.32| +2451807|32396|4901|65902|902589|5676|22363|10|23|1736|80|86.33|113.09|106.30|0.00|8504.00|6906.40|9047.20|425.20|0.00|8504.00|8929.20|1597.60| +2451807|32396|12002|65902|902589|5676|22363|10|108|1736|63|49.28|71.45|56.44|0.00|3555.72|3104.64|4501.35|142.22|0.00|3555.72|3697.94|451.08| +2451807||2905||||22363|10||1736|||98.03|42.15|||3122.66|||||1903.07|| +2451807|32396|1826|65902|902589|5676|22363|10|50|1736|15|73.02|111.72|89.37|0.00|1340.55|1095.30|1675.80|0.00|0.00|1340.55|1340.55|245.25| +2451807|32396|16712|65902|902589|5676|22363|10|180|1736|42|14.04|21.76|2.17|0.00|91.14|589.68|913.92|7.29|0.00|91.14|98.43|-498.54| +2451807|32396|6919|65902|902589|5676|22363|10|249|1736|27|20.93|32.65|31.34|0.00|846.18|565.11|881.55|76.15|0.00|846.18|922.33|281.07| +2451807|32396|17507|65902|902589|5676|22363|10|269|1736|48|28.27|31.09|11.81|0.00|566.88|1356.96|1492.32|0.00|0.00|566.88|566.88|-790.08| +||13397|65902|||||199|1736|96|81.99|111.50|11.15||1070.40|7871.04||10.70||1070.40|1081.10|-6800.64| +2450840|55798|14204|2118|901422|726|10030|7|118|1737|77|86.88|94.69|11.36|0.00|874.72|6689.76|7291.13|69.97|0.00|874.72|944.69|-5815.04| +2450840|55798|5198|2118|901422|726|10030|7|138|1737|44|56.04|110.39|3.31|84.47|145.64|2465.76|4857.16|3.67|84.47|61.17|64.84|-2404.59| +2450840|55798|5089|2118|901422|726|10030|7|276|1737|4|4.01|6.49|4.21|0.00|16.84|16.04|25.96|0.50|0.00|16.84|17.34|0.80| +2450840|55798|15139|2118|901422|726|10030|7|126|1737|97|61.05|89.74|24.22|0.00|2349.34|5921.85|8704.78|46.98|0.00|2349.34|2396.32|-3572.51| +2450840|55798|4087|2118|901422|726|10030|7|97|1737|76|47.93|65.18|13.68|0.00|1039.68|3642.68|4953.68|10.39|0.00|1039.68|1050.07|-2603.00| +2450840|55798|926|2118|901422|726|10030|7|160|1737|69|46.29|50.91|50.40|0.00|3477.60|3194.01|3512.79|208.65|0.00|3477.60|3686.25|283.59| +2450840|55798|11864|2118|901422|726|10030|7|224|1737|2|66.40|82.33|51.86|0.00|103.72|132.80|164.66|8.29|0.00|103.72|112.01|-29.08| +2450840|55798|14738|2118|901422|726|10030|7|124|1737|32|27.89|30.40|17.93|0.00|573.76|892.48|972.80|40.16|0.00|573.76|613.92|-318.72| +2450840|55798|3164|2118|901422|726|10030|7|229|1737|81|39.44|63.49|44.44|0.00|3599.64|3194.64|5142.69|35.99|0.00|3599.64|3635.63|405.00| +2450840|55798|6721|2118|901422|726|10030|7|260|1737|11|77.45|113.85|31.87|42.06|350.57|851.95|1252.35|9.25|42.06|308.51|317.76|-543.44| +2450840||8881|2118||726|||109|1737|47||16.00|4.48||210.56||752.00|||||-248.16| +2452597|51648|14755|24995|1055182|6969|17156|4|103|1738|10|61.86|68.66|10.29|0.00|102.90|618.60|686.60|8.23|0.00|102.90|111.13|-515.70| +2452597|51648|10477|24995|1055182|6969|17156|4|220|1738|50|93.84|141.69|41.09|0.00|2054.50|4692.00|7084.50|143.81|0.00|2054.50|2198.31|-2637.50| +2452597|51648|15889|24995|1055182|6969|17156|4|223|1738|34|93.36|182.98|75.02|0.00|2550.68|3174.24|6221.32|127.53|0.00|2550.68|2678.21|-623.56| +2452597|51648|14016|24995|1055182|6969|17156|4|286|1738|14|95.63|141.53|11.32|74.48|158.48|1338.82|1981.42|1.68|74.48|84.00|85.68|-1254.82| +2452597|51648|8655|24995|1055182|6969|17156|4|136|1738|14|64.69|127.43|7.64|0.00|106.96|905.66|1784.02|3.20|0.00|106.96|110.16|-798.70| +2452597|51648|805|24995|1055182|6969|17156|4|171|1738|87|53.07|80.13|64.90|0.00|5646.30|4617.09|6971.31|395.24|0.00|5646.30|6041.54|1029.21| +2452597|51648|711|24995|1055182|6969|17156|4|67|1738|34|55.78|84.78|69.51|0.00|2363.34|1896.52|2882.52|23.63|0.00|2363.34|2386.97|466.82| +2452597|51648|5496|24995|1055182|6969|17156|4|95|1738|4|25.31|34.42|4.47|0.00|17.88|101.24|137.68|0.89|0.00|17.88|18.77|-83.36| +2452597|51648|17352|24995|1055182|6969|17156|4|156|1738|73|93.30|176.33|151.64|0.00|11069.72|6810.90|12872.09|553.48|0.00|11069.72|11623.20|4258.82| +||5631|24995||6969||4|144|1738|41|77.72|117.35|12.90|||||21.15||||-2657.62| +||17751|24995|||17156||91|1738|||||0.00|1774.96|||141.99|0.00|1774.96|1916.95|| +2452597|51648|4755||||17156|4|207|1738|||108.52|6.51|||7566.56|8247.52|||415.60||| +2452597|51648|6607|24995|1055182|6969|17156|4|141|1738|3|36.61|53.08|30.78|0.00|92.34|109.83|159.24|6.46|0.00|92.34|98.80|-17.49| +2452160|36469|17587|45844|1775120|5643|43766|4|210|1739|8|63.30|84.82|61.07|0.00|488.56|506.40|678.56|34.19|0.00|488.56|522.75|-17.84| +2452160|36469|2901|45844|1775120|5643|43766|4|288|1739|70|23.70|37.92|3.41|0.00|238.70|1659.00|2654.40|21.48|0.00|238.70|260.18|-1420.30| +2452160|36469|5545|45844|1775120|5643|43766|4|129|1739|20|60.47|104.00|40.56|0.00|811.20|1209.40|2080.00|48.67|0.00|811.20|859.87|-398.20| +2452160|36469|12563|45844|1775120|5643|43766|4|7|1739|75|56.84|72.75|67.65|0.00|5073.75|4263.00|5456.25|50.73|0.00|5073.75|5124.48|810.75| +2452160|36469|16885|45844|1775120|5643|43766|4|245|1739|36|97.47|98.44|39.37|0.00|1417.32|3508.92|3543.84|14.17|0.00|1417.32|1431.49|-2091.60| +2452160|36469|7333|45844|1775120|5643|43766|4|199|1739|15|90.54|92.35|80.34|0.00|1205.10|1358.10|1385.25|36.15|0.00|1205.10|1241.25|-153.00| +2452160|36469|9973|45844|1775120|5643|43766|4|246|1739|86|71.55|116.62|46.64|0.00|4011.04|6153.30|10029.32|120.33|0.00|4011.04|4131.37|-2142.26| +2452160|36469|12759|45844|1775120|5643|43766|4|142|1739|68|10.61|19.09|18.32|0.00|1245.76|721.48|1298.12|62.28|0.00|1245.76|1308.04|524.28| +2452160|36469|10375|45844|1775120|5643|43766|4|130|1739|71|84.43|92.87|28.78|0.00|2043.38|5994.53|6593.77|143.03|0.00|2043.38|2186.41|-3951.15| +2452160|36469|3829|45844|1775120|5643|43766|4|165|1739|26|46.08|75.57|26.44|0.00|687.44|1198.08|1964.82|6.87|0.00|687.44|694.31|-510.64| +2452160|36469|9941|45844|1775120|5643|43766|4|248|1739|37|6.18|9.20|8.28|0.00|306.36|228.66|340.40|24.50|0.00|306.36|330.86|77.70| +2452160|36469|547|45844|1775120|5643|43766|4|218|1739|27|39.68|60.71|17.60|9.50|475.20|1071.36|1639.17|27.94|9.50|465.70|493.64|-605.66| +2451863|42099|10903|3453|1906409|6965|8321|1|125|1740|77|70.64|82.64|58.67|0.00|4517.59|5439.28|6363.28|225.87|0.00|4517.59|4743.46|-921.69| +2451863|42099|6785|3453|1906409|6965|8321|1|22|1740|2|52.38|64.95|20.78|0.00|41.56|104.76|129.90|0.83|0.00|41.56|42.39|-63.20| +2451863|42099|2579|3453|1906409|6965|8321|1|244|1740|80|78.56|88.77|66.57|3834.43|5325.60|6284.80|7101.60|14.91|3834.43|1491.17|1506.08|-4793.63| +2451863|42099|10784|3453|1906409|6965|8321|1|7|1740|9|75.71|131.73|19.75|0.00|177.75|681.39|1185.57|1.77|0.00|177.75|179.52|-503.64| +2451863|42099|11335|3453|1906409|6965|8321|1|71|1740|32|60.17|80.02|29.60|28.41|947.20|1925.44|2560.64|55.12|28.41|918.79|973.91|-1006.65| +2451863|42099|6941|3453|1906409|6965|8321|1|63|1740|90|70.53|124.83|56.17|0.00|5055.30|6347.70|11234.70|151.65|0.00|5055.30|5206.95|-1292.40| +2451863|42099|4460|3453|1906409|6965|8321|1|290|1740|41|8.78|10.44|7.83|0.00|321.03|359.98|428.04|3.21|0.00|321.03|324.24|-38.95| +2451863|42099|15464|3453|1906409|6965|8321|1|33|1740|68|22.49|25.41|2.28|55.81|155.04|1529.32|1727.88|8.93|55.81|99.23|108.16|-1430.09| +2451863|42099|782|3453|1906409|6965|8321|1|96|1740|42|68.70|83.81|22.62|0.00|950.04|2885.40|3520.02|28.50|0.00|950.04|978.54|-1935.36| +2451863|42099|7058|3453|1906409|6965|8321|1|186|1740|74|19.52|35.52|1.42|0.00|105.08|1444.48|2628.48|5.25|0.00|105.08|110.33|-1339.40| +2451863|42099|13379|3453|1906409|6965|8321|1|265|1740|9|17.77|30.56|16.19|0.00|145.71|159.93|275.04|8.74|0.00|145.71|154.45|-14.22| +2451863|42099|10964|3453|1906409|6965|8321|1|10|1740|10|16.16|31.83|4.13|0.00|41.30|161.60|318.30|3.71|0.00|41.30|45.01|-120.30| +2451823||2069||1649766|||||1741|81|90.21||2.07||||8402.94|||167.67|177.73|-7139.34| +2451823|64697|2231|22568|1649766|2117|6915|7|161|1741|4|69.38|110.31|40.81|0.00|163.24|277.52|441.24|0.00|0.00|163.24|163.24|-114.28| +2451823|64697|7163|22568|1649766|2117|6915|7|248|1741|90|61.25|87.58|66.56|0.00|5990.40|5512.50|7882.20|419.32|0.00|5990.40|6409.72|477.90| +2451823|64697|10076|22568|1649766|2117|6915|7|74|1741|58|55.81|95.99|91.19|0.00|5289.02|3236.98|5567.42|52.89|0.00|5289.02|5341.91|2052.04| +2451823|64697|14648|22568|1649766|2117|6915|7|66|1741|72|63.14|73.24|71.04|0.00|5114.88|4546.08|5273.28|51.14|0.00|5114.88|5166.02|568.80| +2451823|64697|14879|22568|1649766|2117|6915|7|32|1741|52|94.98|176.66|3.53|0.00|183.56|4938.96|9186.32|16.52|0.00|183.56|200.08|-4755.40| +2451823|64697|6979|22568|1649766|2117|6915|7|20|1741|66|97.38|122.69|109.19|0.00|7206.54|6427.08|8097.54|360.32|0.00|7206.54|7566.86|779.46| +2451823||16967|22568||2117||7|2|1741|69|53.59||||||4326.30|198.99|||4178.91|| +2451823|64697|6119|22568|1649766|2117|6915|7|208|1741|98|66.36|74.32|52.02|0.00|5097.96|6503.28|7283.36|50.97|0.00|5097.96|5148.93|-1405.32| +2451823|64697|8543|22568|1649766|2117|6915|7|174|1741|16|75.82|101.59|40.63|0.00|650.08|1213.12|1625.44|39.00|0.00|650.08|689.08|-563.04| +2451823|64697|674|22568|1649766|2117|6915|7|133|1741|91|24.80|47.36|33.15|2594.31|3016.65|2256.80|4309.76|8.44|2594.31|422.34|430.78|-1834.46| +2451823|64697|8563|22568|1649766|2117|6915|7|60|1741|49|73.17|145.60|82.99|0.00|4066.51|3585.33|7134.40|203.32|0.00|4066.51|4269.83|481.18| +2451823|64697|1955|22568|1649766|2117|6915|7|213|1741|33|93.60|182.52|40.15|437.23|1324.95|3088.80|6023.16|44.38|437.23|887.72|932.10|-2201.08| +2451987|36276|13065|51245|1307625|3558|47891|1|229|1742|3|95.24|110.47|72.91|0.00|218.73|285.72|331.41|8.74|0.00|218.73|227.47|-66.99| +2451987|36276|15309|51245|1307625|3558|47891|1|170|1742|11|88.63|106.35|13.82|60.80|152.02|974.93|1169.85|5.47|60.80|91.22|96.69|-883.71| +2451987|36276|14709|51245|1307625|3558|47891|1|1|1742|61|42.23|67.99|10.87|0.00|663.07|2576.03|4147.39|26.52|0.00|663.07|689.59|-1912.96| +2451987|36276|13161|51245|1307625|3558|47891|1|34|1742|50|6.34|8.30|7.30|0.00|365.00|317.00|415.00|3.65|0.00|365.00|368.65|48.00| +2451987||7747|||3558|47891|1|227|1742||51.49|51.49||||4222.18|4222.18|||379.66|398.64|| +2451987|36276|4433|51245|1307625|3558|47891|1|299|1742|94|28.94|54.98|32.98|0.00|3100.12|2720.36|5168.12|217.00|0.00|3100.12|3317.12|379.76| +2451987|36276|14429|51245|1307625|3558|47891|1|296|1742|42|99.35|107.29|72.95|2726.87|3063.90|4172.70|4506.18|16.85|2726.87|337.03|353.88|-3835.67| +2451987|36276|9547|51245|1307625|3558|47891|1|95|1742|89|37.67|67.42|16.85|0.00|1499.65|3352.63|6000.38|14.99|0.00|1499.65|1514.64|-1852.98| +2451987|36276|1533|51245|1307625|3558|47891|1|275|1742|81|70.08|75.68|46.16|0.00|3738.96|5676.48|6130.08|74.77|0.00|3738.96|3813.73|-1937.52| +2452592|35995|10146|57320|723954|5341|42775|10|113|1743|12|70.11|86.23|72.43|0.00|869.16|841.32|1034.76|34.76|0.00|869.16|903.92|27.84| +2452592|35995|5742|57320|723954|5341|42775|10|211|1743|42|85.78|157.83|151.51|0.00|6363.42|3602.76|6628.86|0.00|0.00|6363.42|6363.42|2760.66| +2452592|35995|14763|57320|723954|5341|42775|10|299|1743|51|19.23|37.11|24.86|722.68|1267.86|980.73|1892.61|43.61|722.68|545.18|588.79|-435.55| +2452592|35995|13737|57320|723954|5341|42775|10|48|1743|44|54.29|71.66|3.58|0.00|157.52|2388.76|3153.04|1.57|0.00|157.52|159.09|-2231.24| +2452592|35995|5601|57320|723954|5341|42775|10|95|1743|83|66.92|84.31|73.34|0.00|6087.22|5554.36|6997.73|243.48|0.00|6087.22|6330.70|532.86| +2452592|35995|13914|57320|723954|5341|42775|10|139|1743|85|87.39|87.39|83.02|0.00|7056.70|7428.15|7428.15|141.13|0.00|7056.70|7197.83|-371.45| +2452592|35995|9091|57320|723954|5341|42775|10|213|1743|54|32.82|33.14|27.17|0.00|1467.18|1772.28|1789.56|132.04|0.00|1467.18|1599.22|-305.10| +2452592|35995|703|57320|723954|5341|42775|10|58|1743|6|36.35|63.24|46.16|0.00|276.96|218.10|379.44|2.76|0.00|276.96|279.72|58.86| +2452592|35995|12667|57320|723954|5341|42775|10|64|1743|63|49.02|75.00|33.00|0.00|2079.00|3088.26|4725.00|83.16|0.00|2079.00|2162.16|-1009.26| +2452592|35995|16872|57320|723954|5341|42775|10|86|1743|3|87.67|168.32|18.51|0.00|55.53|263.01|504.96|2.77|0.00|55.53|58.30|-207.48| +2452592|35995|4548|57320|723954|5341|42775|10|271|1743|17|89.60|168.44|33.68|0.00|572.56|1523.20|2863.48|11.45|0.00|572.56|584.01|-950.64| +2452592|35995|2959|57320|723954|5341|42775|10|237|1743|63|71.65|81.68|60.44|0.00|3807.72|4513.95|5145.84|190.38|0.00|3807.72|3998.10|-706.23| +2452592|35995|12967|57320|723954|5341|42775|10|6|1743|41|98.19|102.11|99.04|0.00|4060.64|4025.79|4186.51|365.45|0.00|4060.64|4426.09|34.85| +2452592|35995|1837|57320|723954|5341|42775|10|235|1743|36|82.48|103.10|103.10|0.00|3711.60|2969.28|3711.60|222.69|0.00|3711.60|3934.29|742.32| +2452592|35995|8667|57320|723954|5341|42775|10|224|1743|85|47.32|88.96|80.06|0.00|6805.10|4022.20|7561.60|272.20|0.00|6805.10|7077.30|2782.90| +||5984|51174|1069328|6329||||1744||||3.22||299.46|1699.11|||||299.46|| +2451839|38849|13520|51174|1069328|6329|24883|2|234|1744|62|83.07|104.66|9.41|52.50|583.42|5150.34|6488.92|0.00|52.50|530.92|530.92|-4619.42| +2451839|38849|13544|51174|1069328|6329|24883|2|30|1744|49|55.11|93.68|24.35|0.00|1193.15|2700.39|4590.32|71.58|0.00|1193.15|1264.73|-1507.24| +2451839|38849|686|51174|1069328|6329|24883|2|114|1744|44|15.91|29.11|3.78|0.00|166.32|700.04|1280.84|14.96|0.00|166.32|181.28|-533.72| +2451839|38849|7117|51174|1069328|6329|24883|2|253|1744|100|52.12|54.72|42.13|0.00|4213.00|5212.00|5472.00|294.91|0.00|4213.00|4507.91|-999.00| +2451839|38849|7061|51174|1069328|6329|24883|2|174|1744|23|77.05|82.44|19.78|0.00|454.94|1772.15|1896.12|18.19|0.00|454.94|473.13|-1317.21| +2451839|38849|12871|51174|1069328|6329|24883|2|55|1744|7|96.16|148.08|60.71|0.00|424.97|673.12|1036.56|38.24|0.00|424.97|463.21|-248.15| +2451839|38849|16940|51174|1069328|6329|24883|2|214|1744|11|3.17|3.83|0.53|0.00|5.83|34.87|42.13|0.29|0.00|5.83|6.12|-29.04| +2451839|38849|9565|51174|1069328|6329|24883|2|113|1744|93|16.89|25.33|10.38|839.84|965.34|1570.77|2355.69|5.02|839.84|125.50|130.52|-1445.27| +2451839|38849|17023|51174|1069328|6329|24883|2|174|1744|98|18.69|30.83|20.03|0.00|1962.94|1831.62|3021.34|137.40|0.00|1962.94|2100.34|131.32| +||15907|51174|||||169|1744|||53.18|21.27|0.00|1722.87|4307.58|||0.00||1809.01|| +2451839|38849|16634|51174|1069328|6329|24883|2|283|1744|32|41.09|47.66|6.19|0.00|198.08|1314.88|1525.12|1.98|0.00|198.08|200.06|-1116.80| +2451839|38849|4988|51174|1069328|6329|24883|2|181|1744|58|5.85|7.19|4.09|0.00|237.22|339.30|417.02|11.86|0.00|237.22|249.08|-102.08| +2451839|38849|797|51174|1069328|6329|24883|2|56|1744|84|30.89|53.74|10.21|0.00|857.64|2594.76|4514.16|17.15|0.00|857.64|874.79|-1737.12| +2451839|38849|11579|51174|1069328|6329|24883|2|262|1744|44|13.35|14.68|4.40|0.00|193.60|587.40|645.92|5.80|0.00|193.60|199.40|-393.80| +2451136|32370|736|33876|1754846|1416|18833|8|205|1745|20|17.07|33.28|33.28|0.00|665.60|341.40|665.60|33.28|0.00|665.60|698.88|324.20| +2451136|32370|16984|33876|1754846|1416|18833|8|28|1745|23|21.19|24.15|18.59|0.00|427.57|487.37|555.45|29.92|0.00|427.57|457.49|-59.80| +2451136|32370|14959|33876|||18833|8|70|1745||||69.46|||1959.36||||100.03||| +2451136|32370|14215|33876|1754846|1416|18833|8|171|1745|90|91.65|99.89|69.92|314.64|6292.80|8248.50|8990.10|59.78|314.64|5978.16|6037.94|-2270.34| +2451136|32370|7412|33876|1754846|1416|18833|8|78|1745|70|84.95|115.53|25.41|0.00|1778.70|5946.50|8087.10|0.00|0.00|1778.70|1778.70|-4167.80| +2451136|32370|4372|33876|1754846|1416|18833|8|128|1745|69|55.16|77.22|62.54|3452.20|4315.26|3806.04|5328.18|60.41|3452.20|863.06|923.47|-2942.98| +2451136|32370|16546|33876|1754846|1416|18833|8|45|1745|40|4.19|5.82|4.59|0.00|183.60|167.60|232.80|0.00|0.00|183.60|183.60|16.00| +2451136|32370|17006|33876|1754846|1416|18833|8|283|1745|35|14.71|14.71|9.70|0.00|339.50|514.85|514.85|13.58|0.00|339.50|353.08|-175.35| +2451136|32370|9589|33876|1754846|1416|18833|8|281|1745|43|43.38|48.15|32.74|0.00|1407.82|1865.34|2070.45|126.70|0.00|1407.82|1534.52|-457.52| +2451136|32370|12145|33876|1754846|1416|18833|8|147|1745|61|23.00|40.25|0.40|0.00|24.40|1403.00|2455.25|1.46|0.00|24.40|25.86|-1378.60| +2451136|32370|5209|33876|1754846|1416|18833|8|286|1745|14|29.07|50.00|17.00|230.86|238.00|406.98|700.00|0.28|230.86|7.14|7.42|-399.84| +2451136|32370|7864|33876|1754846|1416|18833|8|264|1745|25|68.56|79.52|78.72|0.00|1968.00|1714.00|1988.00|118.08|0.00|1968.00|2086.08|254.00| +2451136|32370|15697|33876|1754846|1416|18833|8|278|1745|6|41.24|77.94|28.05|166.61|168.30|247.44|467.64|0.00|166.61|1.69|1.69|-245.75| +2451136|32370|6874|33876|1754846|1416|18833|8|215|1745|66|96.66|112.12|51.57|238.25|3403.62|6379.56|7399.92|94.96|238.25|3165.37|3260.33|-3214.19| +2451136|32370|10981|33876|1754846|1416|18833|8|7|1745|78|95.87|143.80|116.47|0.00|9084.66|7477.86|11216.40|181.69|0.00|9084.66|9266.35|1606.80| +2451136||560|33876|1754846|1416||8|33|1745|9|36.95|||||332.55|598.59|7.09||118.27|125.36|| +2452636|71057|17371|28193|1018624|4608|17039|10|87|1746|40|44.13|62.66|1.87|0.00|74.80|1765.20|2506.40|0.74|0.00|74.80|75.54|-1690.40| +2452636|71057|17215|28193|1018624|4608|17039|10|145|1746|95|20.80|36.81|25.03|0.00|2377.85|1976.00|3496.95|190.22|0.00|2377.85|2568.07|401.85| +2452636|71057|4681|28193|1018624|4608|17039|10|131|1746|52|80.78|105.82|100.52|0.00|5227.04|4200.56|5502.64|313.62|0.00|5227.04|5540.66|1026.48| +2452636|71057|1579|28193|1018624|4608|17039|10|242|1746|38|58.48|89.47|50.10|533.06|1903.80|2222.24|3399.86|109.65|533.06|1370.74|1480.39|-851.50| +||17983|28193|1018624|||10|199|1746|35|||53.31||||3162.60|15.67||783.66|799.33|-1851.84| +2452636|71057|5379|28193|1018624|4608|17039|10|294|1746|76|65.96|116.08|116.08|0.00|8822.08|5012.96|8822.08|264.66|0.00|8822.08|9086.74|3809.12| +2452636|71057|9531|28193|1018624|4608|17039|10|241|1746|58|27.65|54.19|28.72|0.00|1665.76|1603.70|3143.02|16.65|0.00|1665.76|1682.41|62.06| +2452636|71057|12903|28193|1018624|4608|17039|10|72|1746|96|6.89|11.43|7.20|158.97|691.20|661.44|1097.28|31.93|158.97|532.23|564.16|-129.21| +2452636|71057|16341|28193|1018624|4608|17039|10|75|1746|59|45.73|85.51|22.23|0.00|1311.57|2698.07|5045.09|0.00|0.00|1311.57|1311.57|-1386.50| +2452636|71057|279|28193|1018624|4608|17039|10|89|1746|58|5.81|8.07|7.58|0.00|439.64|336.98|468.06|21.98|0.00|439.64|461.62|102.66| +2452636|71057|15339|28193|1018624|4608|17039|10|135|1746|24|56.86|112.58|45.03|767.31|1080.72|1364.64|2701.92|6.26|767.31|313.41|319.67|-1051.23| +2452636|71057|1542|28193|1018624|4608|17039|10|235|1746|49|37.94|41.73|23.36|0.00|1144.64|1859.06|2044.77|22.89|0.00|1144.64|1167.53|-714.42| +2452087|29564|11081|87564|1742168|6819|20223|7|204|1747|4|59.21|63.35|17.10|0.00|68.40|236.84|253.40|1.36|0.00|68.40|69.76|-168.44| +2452087|29564|15695|87564|1742168|6819|20223|7|140|1747|70|8.19|14.98|13.93|0.00|975.10|573.30|1048.60|58.50|0.00|975.10|1033.60|401.80| +2452087|29564|5555|87564|1742168|6819|20223|7|273|1747|86|93.89|174.63|97.79|0.00|8409.94|8074.54|15018.18|168.19|0.00|8409.94|8578.13|335.40| +2452087|29564|595|87564|1742168|6819|20223|7|120|1747|14|28.03|42.60|25.98|0.00|363.72|392.42|596.40|14.54|0.00|363.72|378.26|-28.70| +2452087|29564|17611|87564|1742168|6819|20223|7|63|1747|96|54.01|91.27|83.96|0.00|8060.16|5184.96|8761.92|564.21|0.00|8060.16|8624.37|2875.20| +2452087|29564|14383|87564|1742168|6819|20223|7|39|1747|90|76.79|145.13|89.98|0.00|8098.20|6911.10|13061.70|80.98|0.00|8098.20|8179.18|1187.10| +2452087|29564|3143|87564|1742168|6819|20223|7|155|1747|70|9.74|11.78|10.24|415.74|716.80|681.80|824.60|0.00|415.74|301.06|301.06|-380.74| +2452087|29564|8837|87564|1742168|6819|20223|7|170|1747|92|17.90|19.15|13.40|875.28|1232.80|1646.80|1761.80|7.15|875.28|357.52|364.67|-1289.28| +2452087|29564|531|87564|1742168|6819|20223|7|175|1747|19|87.05|98.36|25.57|0.00|485.83|1653.95|1868.84|9.71|0.00|485.83|495.54|-1168.12| +2452087|29564|3101|87564|1742168|6819|20223|7|49|1747|42|5.16|9.70|3.78|0.00|158.76|216.72|407.40|12.70|0.00|158.76|171.46|-57.96| +2452087|29564|9073|87564|1742168|6819|20223|7|141|1747|77|69.57|84.17|1.68|0.00|129.36|5356.89|6481.09|9.05|0.00|129.36|138.41|-5227.53| +||4605|||6819|20223|7||1747|57||48.32|||1183.89||2754.24|94.71||||-1275.66| +2451059|38479|8872|93606|526071|2536|47401|8|113|1748|23|53.04|93.88|4.69|0.00|107.87|1219.92|2159.24|5.39|0.00|107.87|113.26|-1112.05| +2451059|38479|16628|93606|526071|2536|47401|8|134|1748|38|70.09|100.22|99.21|0.00|3769.98|2663.42|3808.36|188.49|0.00|3769.98|3958.47|1106.56| +2451059|38479|16822|93606|526071|2536|47401|8|45|1748|96|12.56|19.46|12.45|0.00|1195.20|1205.76|1868.16|11.95|0.00|1195.20|1207.15|-10.56| +2451059|38479|13309|93606|526071|2536|47401|8|140|1748|60|54.19|66.11|46.93|0.00|2815.80|3251.40|3966.60|28.15|0.00|2815.80|2843.95|-435.60| +2451059|38479|2060|93606|526071|2536|47401|8|300|1748|57|60.19|99.91|14.98|281.77|853.86|3430.83|5694.87|17.16|281.77|572.09|589.25|-2858.74| +2451059|38479|14908|93606|526071|2536|47401|8|18|1748|58|8.83|13.95|3.76|0.00|218.08|512.14|809.10|15.26|0.00|218.08|233.34|-294.06| +2451059|38479|5356|93606|526071|2536|47401|8|79|1748|51|3.29|4.14|1.57|0.00|80.07|167.79|211.14|2.40|0.00|80.07|82.47|-87.72| +2451059|38479|974|93606|526071|2536|47401|8|276|1748|52|84.69|94.00|10.34|0.00|537.68|4403.88|4888.00|16.13|0.00|537.68|553.81|-3866.20| +2451059|38479|2576|93606|526071|2536|47401|8|291|1748|23|59.40|70.68|13.42|0.00|308.66|1366.20|1625.64|6.17|0.00|308.66|314.83|-1057.54| +2451059|38479|16975|93606|526071|2536|47401|8|31|1748|60|93.33|167.06|15.03|0.00|901.80|5599.80|10023.60|45.09|0.00|901.80|946.89|-4698.00| +2451059|38479|4102|93606|526071|2536|47401|8|62|1748|19|12.40|24.55|8.59|151.78|163.21|235.60|466.45|0.57|151.78|11.43|12.00|-224.17| +2451059|38479|17792|93606|526071|2536|47401|8|298|1748|90|57.79|84.37|73.40|0.00|6606.00|5201.10|7593.30|594.54|0.00|6606.00|7200.54|1404.90| +2451059|38479|5564|93606|526071|2536|47401|8|290|1748|26|54.54|68.17|21.13|0.00|549.38|1418.04|1772.42|0.00|0.00|549.38|549.38|-868.66| +2451059|38479|5764|93606|526071|2536|47401|8|298|1748|99|65.87|72.45|5.07|356.37|501.93|6521.13|7172.55|1.45|356.37|145.56|147.01|-6375.57| +2451059|38479|5563|93606|526071|2536|47401|8|75|1748|24|60.76|78.38|70.54|0.00|1692.96|1458.24|1881.12|135.43|0.00|1692.96|1828.39|234.72| +2452141|36226|15319|1851|69639|4980|45568|7|4|1749|72|9.02|14.07|4.36|0.00|313.92|649.44|1013.04|6.27|0.00|313.92|320.19|-335.52| +2452141|36226|13257|1851|69639|4980|45568|7|216|1749|28|99.48|164.14|91.91|0.00|2573.48|2785.44|4595.92|180.14|0.00|2573.48|2753.62|-211.96| +2452141|36226|8193|1851|69639|4980|45568|7|74|1749|49|86.34|170.95|42.73|0.00|2093.77|4230.66|8376.55|104.68|0.00|2093.77|2198.45|-2136.89| +2452141|36226|2261|1851|69639|4980|45568|7|263|1749|80|12.20|16.83|2.52|0.00|201.60|976.00|1346.40|10.08|0.00|201.60|211.68|-774.40| +2452141|36226|6407|1851|69639|4980|45568|7|242|1749|93|63.23|122.66|84.63|0.00|7870.59|5880.39|11407.38|472.23|0.00|7870.59|8342.82|1990.20| +2452141|36226|11291|1851|69639|4980|45568|7|91|1749|59|80.35|128.56|88.70|0.00|5233.30|4740.65|7585.04|156.99|0.00|5233.30|5390.29|492.65| +2452141|36226|13721|1851|69639|4980|45568|7|291|1749|68|44.97|82.74|25.64|0.00|1743.52|3057.96|5626.32|122.04|0.00|1743.52|1865.56|-1314.44| +2452141|36226|2047|1851|69639|4980|45568|7|231|1749|69|56.73|72.61|41.38|57.10|2855.22|3914.37|5010.09|83.94|57.10|2798.12|2882.06|-1116.25| +||5621|1851|69639|4980||7||1749|40||||0.00|||||0.00||1299.16|-1102.80| +2452141|36226|14985|1851|69639|4980|45568|7|152|1749|83|62.29|87.20|35.75|0.00|2967.25|5170.07|7237.60|237.38|0.00|2967.25|3204.63|-2202.82| +2452141|36226|17177|1851|69639|4980|45568|7|284|1749|31|45.57|60.60|50.29|0.00|1558.99|1412.67|1878.60|140.30|0.00|1558.99|1699.29|146.32| +2452141|36226|15059|1851|69639|4980|45568|7|272|1749|46|68.65|91.30|87.64|0.00|4031.44|3157.90|4199.80|282.20|0.00|4031.44|4313.64|873.54| +2452141|36226|1767|1851|69639|4980|45568|7|26|1749|28|11.00|18.48|2.77|0.00|77.56|308.00|517.44|3.10|0.00|77.56|80.66|-230.44| +2452141|36226|10119|1851|69639|4980|45568|7|86|1749|10|72.11|121.14|67.83|0.00|678.30|721.10|1211.40|40.69|0.00|678.30|718.99|-42.80| +2452182|35675|1649|87555|710892|1429|9566|1|82|1750|10|4.14|7.28|0.00|0.00|0.00|41.40|72.80|0.00|0.00|0.00|0.00|-41.40| +2452182|35675|7043|87555|710892|1429|9566|1|217|1750|48|77.34|88.94|73.82|0.00|3543.36|3712.32|4269.12|141.73|0.00|3543.36|3685.09|-168.96| +2452182|35675|5261|87555|710892|1429|9566|1|95|1750|41|29.46|40.94|33.57|0.00|1376.37|1207.86|1678.54|0.00|0.00|1376.37|1376.37|168.51| +2452182|35675|12121|87555|710892|1429|9566|1|161|1750|56|43.03|50.77|34.52|0.00|1933.12|2409.68|2843.12|154.64|0.00|1933.12|2087.76|-476.56| +2452182|35675|5603|87555|710892|1429|9566|1|43|1750|87|17.50|33.25|26.26|2238.92|2284.62|1522.50|2892.75|1.37|2238.92|45.70|47.07|-1476.80| +2452182|35675|14359|87555|710892|1429|9566|1|144|1750|45|57.14|74.28|32.68|0.00|1470.60|2571.30|3342.60|0.00|0.00|1470.60|1470.60|-1100.70| +2452182|35675|1465|87555|710892|1429|9566|1|125|1750|72|47.24|53.38|7.47|188.24|537.84|3401.28|3843.36|20.97|188.24|349.60|370.57|-3051.68| +2452182|35675|13097|87555|710892|1429|9566|1|123|1750|5|90.78|146.15|140.30|0.00|701.50|453.90|730.75|28.06|0.00|701.50|729.56|247.60| +2450940|34232|8923|98707|1412538|4416|22247|8|179|1751|21|46.84|93.21|46.60|0.00|978.60|983.64|1957.41|78.28|0.00|978.60|1056.88|-5.04| +2450940|34232|11020|98707|1412538|4416|22247|8|194|1751|74|65.84|109.95|64.87|0.00|4800.38|4872.16|8136.30|0.00|0.00|4800.38|4800.38|-71.78| +2450940|34232|8815|98707|1412538|4416|22247|8|141|1751|54|77.31|119.05|25.00|0.00|1350.00|4174.74|6428.70|13.50|0.00|1350.00|1363.50|-2824.74| +2450940|34232|13082|98707|1412538|4416|22247|8|23|1751|2|39.33|50.73|5.07|0.10|10.14|78.66|101.46|0.80|0.10|10.04|10.84|-68.62| +2450940|34232|5458|98707|1412538|4416|22247|8|50|1751|62|51.74|70.88|3.54|0.00|219.48|3207.88|4394.56|2.19|0.00|219.48|221.67|-2988.40| +2450940|34232|5221|98707|1412538|4416|22247|8|7|1751|77|94.51|137.03|0.00|0.00|0.00|7277.27|10551.31|0.00|0.00|0.00|0.00|-7277.27| +2450940|34232|2020|98707|1412538|4416|22247|8|82|1751|32|40.24|40.64|40.64|0.00|1300.48|1287.68|1300.48|26.00|0.00|1300.48|1326.48|12.80| +2450940|34232|1556|98707|1412538|4416|22247|8|236|1751|66|21.97|31.85|12.10|0.00|798.60|1450.02|2102.10|31.94|0.00|798.60|830.54|-651.42| +2450940|34232|17233|98707|1412538|4416|22247|8|82|1751|38|68.01|95.21|71.40|0.00|2713.20|2584.38|3617.98|135.66|0.00|2713.20|2848.86|128.82| +2450940|34232|2131|98707|1412538|4416|22247|8|76|1751|91|14.09|17.47|2.62|0.00|238.42|1282.19|1589.77|4.76|0.00|238.42|243.18|-1043.77| +2450940|34232|17515|98707|1412538|4416|22247|8|41|1751|50|94.62|159.90|20.78|0.00|1039.00|4731.00|7995.00|41.56|0.00|1039.00|1080.56|-3692.00| +2450940|34232|12782|98707|1412538|4416|22247|8|37|1751|79|20.40|27.13|14.10|89.11|1113.90|1611.60|2143.27|92.23|89.11|1024.79|1117.02|-586.81| +2450940|34232|199|98707|1412538|4416|22247|8|240|1751|17|40.04|79.27|16.64|0.00|282.88|680.68|1347.59|2.82|0.00|282.88|285.70|-397.80| +2450940|34232|13561|98707|1412538|4416|22247|8|71|1751|32|49.43|64.25|25.70|0.00|822.40|1581.76|2056.00|41.12|0.00|822.40|863.52|-759.36| +2452339|52865|6361|89671|805864|5704|15144|1|38|1752|40|91.59|119.98|3.59|0.00|143.60|3663.60|4799.20|12.92|0.00|143.60|156.52|-3520.00| +2452339|52865|5961|89671|805864|5704|15144|1|235|1752|88|15.83|15.83|12.82|0.00|1128.16|1393.04|1393.04|33.84|0.00|1128.16|1162.00|-264.88| +2452339|52865|12325|89671|805864|5704|15144|1|262|1752|93|1.01|1.52|0.48|0.00|44.64|93.93|141.36|0.00|0.00|44.64|44.64|-49.29| +2452339|52865|7555|89671|805864|5704|15144|1|176|1752|36|59.28|96.03|72.02|0.00|2592.72|2134.08|3457.08|181.49|0.00|2592.72|2774.21|458.64| +2452339|52865|1365|89671|805864|5704|15144|1|53|1752|97|22.52|40.98|27.04|0.00|2622.88|2184.44|3975.06|131.14|0.00|2622.88|2754.02|438.44| +|52865|15007|89671|805864||15144|1||1752|42||97.33||0.00|||||0.00|2043.72|2166.34|-252.84| +2452339|52865|15205|89671|805864|5704|15144|1|61|1752|75|90.74|102.53|64.59|3633.18|4844.25|6805.50|7689.75|12.11|3633.18|1211.07|1223.18|-5594.43| +2452339|52865|9384|89671|805864|5704|15144|1|194|1752|91|7.16|12.31|8.37|0.00|761.67|651.56|1120.21|60.93|0.00|761.67|822.60|110.11| +2451807|48233|5783|13691|1097068|2045|5869|2|193|1753|28|59.46|62.43|0.00|0.00|0.00|1664.88|1748.04|0.00|0.00|0.00|0.00|-1664.88| +2451807|48233|4855|13691|1097068|2045|5869|2|208|1753|82|14.10|18.75|8.06|0.00|660.92|1156.20|1537.50|19.82|0.00|660.92|680.74|-495.28| +2451807||6722|13691|1097068|||2|96|1753|5|||17.41|0.00||77.70|||0.00||94.01|9.35| +2451807|48233|12775|13691|1097068|2045|5869|2|38|1753|58|94.73|134.51|80.70|0.00|4680.60|5494.34|7801.58|46.80|0.00|4680.60|4727.40|-813.74| +2451807|48233|13190|13691|1097068|2045|5869|2|33|1753|35|91.17|181.42|128.80|0.00|4508.00|3190.95|6349.70|360.64|0.00|4508.00|4868.64|1317.05| +2451807|48233|13739|13691|1097068|2045|5869|2|17|1753|65|38.30|45.96|41.36|0.00|2688.40|2489.50|2987.40|134.42|0.00|2688.40|2822.82|198.90| +2451807|48233|6932|13691|1097068|2045|5869|2|217|1753|59|79.72|132.33|44.99|0.00|2654.41|4703.48|7807.47|0.00|0.00|2654.41|2654.41|-2049.07| +2451807|48233|13847|13691|1097068|2045|5869|2|29|1753|81|6.90|11.86|9.25|0.00|749.25|558.90|960.66|52.44|0.00|749.25|801.69|190.35| +2451807|48233|17027|13691|1097068|2045|5869|2|24|1753|81|78.09|135.09|135.09|0.00|10942.29|6325.29|10942.29|437.69|0.00|10942.29|11379.98|4617.00| +2451807|48233|10847|13691|1097068|2045|5869|2|102|1753|18|28.55|39.68|21.82|0.00|392.76|513.90|714.24|11.78|0.00|392.76|404.54|-121.14| +2451807|48233|6512|13691|1097068|2045|5869|2|93|1753|93|5.99|8.02|0.88|5.72|81.84|557.07|745.86|1.52|5.72|76.12|77.64|-480.95| +2451807|48233|4765|13691|1097068|2045|5869|2|161|1753|18|8.00|13.28|7.96|143.28|143.28|144.00|239.04|0.00|143.28|0.00|0.00|-144.00| +2451807|48233|1220|13691|1097068|2045|5869|2|86|1753|39|61.25|69.21|0.00|0.00|0.00|2388.75|2699.19|0.00|0.00|0.00|0.00|-2388.75| +2451807|48233|6542|13691|1097068|2045|5869|2|52|1753|60|69.51|116.77|0.00|0.00|0.00|4170.60|7006.20|0.00|0.00|0.00|0.00|-4170.60| +2452308|36962|270|95809|1149869|5976|45283|7|299|1754|69|27.60|47.47|40.34|0.00|2783.46|1904.40|3275.43|111.33|0.00|2783.46|2894.79|879.06| +2452308|36962|6577|95809|1149869|5976|45283|7|1|1754|47|28.38|53.92|33.43|0.00|1571.21|1333.86|2534.24|0.00|0.00|1571.21|1571.21|237.35| +2452308|36962|13555|95809|1149869|5976|45283|7|198|1754|11|36.20|55.38|21.04|162.00|231.44|398.20|609.18|6.24|162.00|69.44|75.68|-328.76| +2452308|36962|12757|95809|1149869|5976|45283|7|114|1754|18|91.43|149.03|113.26|285.41|2038.68|1645.74|2682.54|122.72|285.41|1753.27|1875.99|107.53| +2452308|36962|2328|95809|1149869|5976|45283|7|271|1754|23|61.46|81.74|17.16|0.00|394.68|1413.58|1880.02|7.89|0.00|394.68|402.57|-1018.90| +2452308|36962|12606|95809|1149869|5976|45283|7|147|1754|28|80.33|102.82|102.82|0.00|2878.96|2249.24|2878.96|172.73|0.00|2878.96|3051.69|629.72| +2452308|36962|7405|95809|1149869|5976|45283|7|245|1754|4|43.19|60.46|0.00|0.00|0.00|172.76|241.84|0.00|0.00|0.00|0.00|-172.76| +2452308|36962|5739|95809|1149869|5976|45283|7|200|1754|85|59.64|101.38|65.89|56.00|5600.65|5069.40|8617.30|388.12|56.00|5544.65|5932.77|475.25| +2452308|36962|7962|95809|1149869|5976|45283|7|116|1754|74|37.65|62.87|5.65|0.00|418.10|2786.10|4652.38|29.26|0.00|418.10|447.36|-2368.00| +2452308|36962|5593|95809|1149869|5976|45283|7|65|1754|15|96.76|100.63|49.30|0.00|739.50|1451.40|1509.45|14.79|0.00|739.50|754.29|-711.90| +2452308|36962|16830|95809|1149869|5976|45283|7|139|1754|69|47.67|66.26|33.79|349.72|2331.51|3289.23|4571.94|178.36|349.72|1981.79|2160.15|-1307.44| +2452327|67279|7189|96186|311844|655|25113|10|29|1755|58|39.26|56.14|1.68|0.00|97.44|2277.08|3256.12|0.97|0.00|97.44|98.41|-2179.64| +2452327|67279|10725|96186|311844|655|25113|10|22|1755|24|60.63|84.27|5.05|0.00|121.20|1455.12|2022.48|8.48|0.00|121.20|129.68|-1333.92| +2452327|67279|17953|96186|311844|655|25113|10|208|1755|62|32.18|56.31|45.04|670.19|2792.48|1995.16|3491.22|42.44|670.19|2122.29|2164.73|127.13| +2452327|67279|4842|96186|311844|655|25113|10|232|1755|34|96.90|174.42|62.79|0.00|2134.86|3294.60|5930.28|128.09|0.00|2134.86|2262.95|-1159.74| +2452327|67279|5769|96186|311844|655|25113|10|289|1755|34|60.70|61.91|45.81|0.00|1557.54|2063.80|2104.94|140.17|0.00|1557.54|1697.71|-506.26| +2452327|67279|6915|96186|311844|655|25113|10|138|1755|13|91.99|182.14|171.21|0.00|2225.73|1195.87|2367.82|155.80|0.00|2225.73|2381.53|1029.86| +2452327|67279|14613|96186|311844|655|25113|10|69|1755|44|27.42|30.98|13.63|0.00|599.72|1206.48|1363.12|53.97|0.00|599.72|653.69|-606.76| +2452327|67279|3655|96186|311844|655|25113|10|206|1755|96|16.72|26.41|12.14|0.00|1165.44|1605.12|2535.36|34.96|0.00|1165.44|1200.40|-439.68| +2452327|67279|15955|96186|311844|655|25113|10|202|1755|96|75.01|90.01|59.40|0.00|5702.40|7200.96|8640.96|114.04|0.00|5702.40|5816.44|-1498.56| +2452327|67279|14809|96186|311844|655|25113|10|193|1755|16|37.22|45.40|40.40|0.00|646.40|595.52|726.40|38.78|0.00|646.40|685.18|50.88| +2452327|67279|2799|96186|311844|655|25113|10|261|1755|89|66.98|101.13|27.30|2283.91|2429.70|5961.22|9000.57|7.28|2283.91|145.79|153.07|-5815.43| +2452327|67279|16656|96186||655||10||1755|6|37.69|41.83|35.13||210.78|226.14|250.98|18.97||210.78||-15.36| +2451591|29636|10219|28061|772872|3730|11341|1|268|1756|21|74.86|145.22|30.49|0.00|640.29|1572.06|3049.62|57.62|0.00|640.29|697.91|-931.77| +2451591|29636|17198|28061|772872|3730|11341|1|117|1756|65|98.65|191.38|176.06|0.00|11443.90|6412.25|12439.70|915.51|0.00|11443.90|12359.41|5031.65| +2451591|29636|3523|28061|772872|3730|11341|1|290|1756|9|37.64|38.01|7.98|0.00|71.82|338.76|342.09|3.59|0.00|71.82|75.41|-266.94| +2451591|29636|15125|28061|772872|3730|11341|1|84|1756|4|2.82|3.49|1.91|0.00|7.64|11.28|13.96|0.15|0.00|7.64|7.79|-3.64| +2451591|29636|371|28061|772872|3730|11341|1|73|1756|5|74.53|108.81|70.72|307.63|353.60|372.65|544.05|0.91|307.63|45.97|46.88|-326.68| +2451591|29636|8947|28061|772872|3730|11341|1|139|1756|80|45.61|60.66|58.84|0.00|4707.20|3648.80|4852.80|282.43|0.00|4707.20|4989.63|1058.40| +2451591|29636|15386|28061|772872|3730|11341|1|85|1756|90|52.47|88.14|60.81|0.00|5472.90|4722.30|7932.60|109.45|0.00|5472.90|5582.35|750.60| +2451591|29636|12245|28061|772872|3730|11341|1|290|1756|73|75.76|100.00|29.00|0.00|2117.00|5530.48|7300.00|105.85|0.00|2117.00|2222.85|-3413.48| +2451023|62137|4592|63717|942672|2427|7782|1|203|1757|73|29.94|33.83|17.92|0.00|1308.16|2185.62|2469.59|39.24|0.00|1308.16|1347.40|-877.46| +2451023|62137|5320|63717|942672|2427|7782|1|194|1757|15|80.74|100.11|54.05|0.00|810.75|1211.10|1501.65|16.21|0.00|810.75|826.96|-400.35| +2451023|62137|11161|63717|942672|2427|7782|1|138|1757|22|90.40|112.09|49.31|0.00|1084.82|1988.80|2465.98|65.08|0.00|1084.82|1149.90|-903.98| +2451023|62137|6860|63717|942672|2427|7782|1|259|1757|11|47.58|87.07|42.66|398.87|469.26|523.38|957.77|3.51|398.87|70.39|73.90|-452.99| +2451023|62137|5302|63717|942672|2427|7782|1|295|1757|55|11.47|20.41|20.41|875.58|1122.55|630.85|1122.55|2.46|875.58|246.97|249.43|-383.88| +2451023|62137|7687|63717|942672|2427|7782|1|13|1757|65|74.30|95.84|4.79|0.00|311.35|4829.50|6229.60|24.90|0.00|311.35|336.25|-4518.15| +2451023|62137|1096|63717|942672|2427|7782|1|260|1757|56|81.69|105.38|28.45|1147.10|1593.20|4574.64|5901.28|35.68|1147.10|446.10|481.78|-4128.54| +2451023|62137|8080|63717|942672|2427|7782|1|72|1757|44|96.15|133.64|9.35|0.00|411.40|4230.60|5880.16|16.45|0.00|411.40|427.85|-3819.20| +2451023|62137|9133|63717|942672|2427|7782|1|181|1757|94|63.22|92.93|32.52|0.00|3056.88|5942.68|8735.42|183.41|0.00|3056.88|3240.29|-2885.80| +2451023|62137|7856|63717|942672|2427|7782|1|19|1757|83|82.48|131.96|36.94|0.00|3066.02|6845.84|10952.68|245.28|0.00|3066.02|3311.30|-3779.82| +2451023|62137|5060|63717|942672|2427|7782|1|254|1757|76|73.84|101.89|62.15|0.00|4723.40|5611.84|7743.64|330.63|0.00|4723.40|5054.03|-888.44| +2451023|62137|4585|63717|942672|2427|7782|1|180|1757|25|29.57|42.87|20.57|467.96|514.25|739.25|1071.75|0.92|467.96|46.29|47.21|-692.96| +2451023|62137|7729|63717|942672|2427|7782|1|283|1757|42|3.50|3.81|2.20|92.40|92.40|147.00|160.02|0.00|92.40|0.00|0.00|-147.00| +||10771|12699||43|||191|1758||15.77||7.87||637.47||2451.87|25.49||||| +2452346|45465|14791|12699|571571|43|38515|1|286|1758|19|59.93|75.51|36.99|0.00|702.81|1138.67|1434.69|0.00|0.00|702.81|702.81|-435.86| +||7917||571571|43|38515||239|1758|54|27.98||22.96||||1722.06|61.99||1239.84||-271.08| +2452346|45465|11088|12699|571571|43|38515|1|37|1758|86|28.77|43.44|20.85|0.00|1793.10|2474.22|3735.84|89.65|0.00|1793.10|1882.75|-681.12| +2452346|45465|1123|12699|571571|43|38515|1|299|1758|69|33.99|41.12|11.51|0.00|794.19|2345.31|2837.28|39.70|0.00|794.19|833.89|-1551.12| +2452346|45465|15396|12699|571571|43|38515|1|85|1758|30|88.84|117.26|78.56|0.00|2356.80|2665.20|3517.80|117.84|0.00|2356.80|2474.64|-308.40| +2452346|45465|9396|12699|571571|43|38515|1|68|1758|50|22.86|29.71|28.22|0.00|1411.00|1143.00|1485.50|28.22|0.00|1411.00|1439.22|268.00| +2452346|45465|17895|12699|571571|43|38515|1|70|1758|90|8.21|14.20|5.53|0.00|497.70|738.90|1278.00|9.95|0.00|497.70|507.65|-241.20| +2452346|45465|729|12699|571571|43|38515|1|201|1758|75|88.46|157.45|86.59|0.00|6494.25|6634.50|11808.75|519.54|0.00|6494.25|7013.79|-140.25| +2452346|45465|11565|12699|571571|43|38515|1|173|1758|15|34.93|36.67|20.16|102.81|302.40|523.95|550.05|9.97|102.81|199.59|209.56|-324.36| +2452346|45465|11190|12699|571571|43|38515|1|65|1758|73|31.14|60.10|52.88|0.00|3860.24|2273.22|4387.30|38.60|0.00|3860.24|3898.84|1587.02| +2452346|45465|9297|12699|571571|43|38515|1|7|1758|93|35.23|59.89|31.74|0.00|2951.82|3276.39|5569.77|177.10|0.00|2951.82|3128.92|-324.57| +2452346|45465|15679|12699|571571|43|38515|1|96|1758|44|12.81|16.39|4.09|0.00|179.96|563.64|721.16|10.79|0.00|179.96|190.75|-383.68| +2452346|45465|17337|12699|571571|43|38515|1|181|1758|62|29.50|31.27|23.13|444.55|1434.06|1829.00|1938.74|89.05|444.55|989.51|1078.56|-839.49| +2452110|41111|12135|87253|1713464|122|4333|4|229|1759|72|93.25|131.48|9.20|158.97|662.40|6714.00|9466.56|20.13|158.97|503.43|523.56|-6210.57| +2452110|41111|4537|87253|1713464|122|4333|4|169|1759|97|6.53|9.99|1.89|0.00|183.33|633.41|969.03|10.99|0.00|183.33|194.32|-450.08| +2452110|41111|4809|87253|1713464|122|4333|4|204|1759|25|47.67|79.60|58.10|0.00|1452.50|1191.75|1990.00|43.57|0.00|1452.50|1496.07|260.75| +2452110|41111|5799|87253|1713464|122|4333|4|272|1759|46|82.31|135.81|118.15|2391.35|5434.90|3786.26|6247.26|30.43|2391.35|3043.55|3073.98|-742.71| +2452110|41111|17659|87253|1713464|122|4333|4|14|1759|34|58.32|107.30|13.94|0.00|473.96|1982.88|3648.20|37.91|0.00|473.96|511.87|-1508.92| +2452110|41111|15001|87253|1713464|122|4333|4|184|1759|66|11.78|18.84|13.56|0.00|894.96|777.48|1243.44|71.59|0.00|894.96|966.55|117.48| +2452110|41111|11005|87253|1713464|122|4333|4|289|1759|62|64.83|91.41|31.99|0.00|1983.38|4019.46|5667.42|0.00|0.00|1983.38|1983.38|-2036.08| +2452110|41111|15645|87253|1713464|122|4333|4|133|1759|15|48.87|84.54|8.45|0.00|126.75|733.05|1268.10|5.07|0.00|126.75|131.82|-606.30| +2452110|41111|16421|87253|1713464|122|4333|4|227|1759|42|65.07|91.09|59.20|0.00|2486.40|2732.94|3825.78|49.72|0.00|2486.40|2536.12|-246.54| +2452110|41111|2095|87253|1713464|122|4333|4|62|1759|94|39.15|53.24|36.20|0.00|3402.80|3680.10|5004.56|34.02|0.00|3402.80|3436.82|-277.30| +|41111|7547|87253|1713464|122|4333|||1759|||||0.00|1901.52|2503.44||0.00|0.00||1901.52|-601.92| +2451974|38286|4477|33927|692492|2704|37269|1|67|1760|97|90.58|154.89|27.88|0.00|2704.36|8786.26|15024.33|54.08|0.00|2704.36|2758.44|-6081.90| +2451974|38286|9351|33927|692492|2704|37269|1|201|1760|9|88.34|124.55|0.00|0.00|0.00|795.06|1120.95|0.00|0.00|0.00|0.00|-795.06| +2451974|38286|4149|33927|692492|2704|37269|1|159|1760|43|57.83|63.61|50.25|0.00|2160.75|2486.69|2735.23|43.21|0.00|2160.75|2203.96|-325.94| +2451974|38286|10231|33927|692492|2704|37269|1|118|1760|7|58.37|68.29|47.80|197.41|334.60|408.59|478.03|5.48|197.41|137.19|142.67|-271.40| +2451974|38286|7979|33927|692492|2704|37269|1|152|1760|14|11.43|18.74|8.05|0.00|112.70|160.02|262.36|0.00|0.00|112.70|112.70|-47.32| +2451974|38286|10327|33927|692492|2704|37269|1|135|1760|32|43.69|74.70|56.77|1634.97|1816.64|1398.08|2390.40|7.26|1634.97|181.67|188.93|-1216.41| +2451974|38286|8839|33927|692492|2704|37269|1|246|1760|49|2.43|3.57|0.14|0.00|6.86|119.07|174.93|0.48|0.00|6.86|7.34|-112.21| +2451974|38286|15367|33927|692492|2704|37269|1|124|1760|38|86.72|131.81|125.21|0.00|4757.98|3295.36|5008.78|95.15|0.00|4757.98|4853.13|1462.62| +||2559||692492|2704|37269|1|292|1760||||17.07||||454.02|14.33||||131.46| +2451974|38286|9167|33927|692492|2704|37269|1|202|1760|27|11.28|19.85|0.00|0.00|0.00|304.56|535.95|0.00|0.00|0.00|0.00|-304.56| +2451974|38286|16371|33927|692492|2704|37269|1|180|1760|9|90.33|176.14|15.85|138.37|142.65|812.97|1585.26|0.17|138.37|4.28|4.45|-808.69| +2450879|70697|11314|4701|933209|5717|33825|1|91|1761|88|51.45|86.43|53.58|0.00|4715.04|4527.60|7605.84|47.15|0.00|4715.04|4762.19|187.44| +2450879|70697|17354|4701|933209|5717|33825|1|7|1761|52|90.61|127.76|112.42|0.00|5845.84|4711.72|6643.52|467.66|0.00|5845.84|6313.50|1134.12| +2450879|70697|14114|4701|933209|5717|33825|1|75|1761|100|14.46|26.17|3.66|0.00|366.00|1446.00|2617.00|25.62|0.00|366.00|391.62|-1080.00| +2450879|70697|15877|4701|933209|5717|33825|1|250|1761|21|64.88|96.02|93.13|704.06|1955.73|1362.48|2016.42|87.61|704.06|1251.67|1339.28|-110.81| +2450879|70697|4336|4701|933209|5717|33825|1|210|1761|23|47.51|56.53|50.87|0.00|1170.01|1092.73|1300.19|70.20|0.00|1170.01|1240.21|77.28| +2450879||3004|||5717|33825|||1761||58.08|102.80|25.70|||2613.60||0.57|||58.40|-2555.77| +2450879|70697|6724|4701|933209|5717|33825|1|166|1761|62|63.92|90.76|35.39|0.00|2194.18|3963.04|5627.12|153.59|0.00|2194.18|2347.77|-1768.86| +2450879|70697|5726|4701|933209|5717|33825|1|262|1761|86|96.89|178.27|167.57|0.00|14411.02|8332.54|15331.22|0.00|0.00|14411.02|14411.02|6078.48| +2450879|70697|3691|4701|933209|5717|33825|1|36|1761|17|68.58|117.95|96.71|0.00|1644.07|1165.86|2005.15|65.76|0.00|1644.07|1709.83|478.21| +2452531|57165|4561|45262|1889975|1441|41707|2|144|1762|6|59.14|94.62|14.19|0.00|85.14|354.84|567.72|2.55|0.00|85.14|87.69|-269.70| +2452531|57165|780|45262|1889975|1441|41707|2|163|1762|25|38.38|72.53|71.80|0.00|1795.00|959.50|1813.25|107.70|0.00|1795.00|1902.70|835.50| +2452531|57165|11059|45262|1889975|1441|41707|2|280|1762|66|13.32|26.24|2.62|0.00|172.92|879.12|1731.84|5.18|0.00|172.92|178.10|-706.20| +2452531|57165|2082|45262|1889975|1441|41707|2|73|1762|90|3.09|5.77|0.63|0.00|56.70|278.10|519.30|0.00|0.00|56.70|56.70|-221.40| +2452531|57165|1753|45262|1889975|1441|41707|2|263|1762|4|56.63|84.94|23.78|0.00|95.12|226.52|339.76|5.70|0.00|95.12|100.82|-131.40| +2452531|57165|1323|45262|1889975|1441|41707|2|18|1762|61|20.94|23.03|5.06|0.00|308.66|1277.34|1404.83|27.77|0.00|308.66|336.43|-968.68| +2452531|57165|16471|45262|1889975|1441|41707|2|69|1762|40|98.28|128.74|56.64|0.00|2265.60|3931.20|5149.60|67.96|0.00|2265.60|2333.56|-1665.60| +2452531|57165|1455|45262|1889975|1441|41707|2|92|1762|28|42.78|80.85|8.89|0.00|248.92|1197.84|2263.80|2.48|0.00|248.92|251.40|-948.92| +2452531|57165|8310|45262|1889975|1441|41707|2|165|1762|46|39.13|74.34|67.64|2302.46|3111.44|1799.98|3419.64|32.35|2302.46|808.98|841.33|-991.00| +2452531|57165|9918|45262|1889975|1441|41707|2|128|1762|98|3.02|4.68|1.91|26.20|187.18|295.96|458.64|9.65|26.20|160.98|170.63|-134.98| +2452531|57165|15051|45262|1889975|1441|41707|2|3|1762|30|90.00|163.80|132.67|0.00|3980.10|2700.00|4914.00|238.80|0.00|3980.10|4218.90|1280.10| +2452531|57165|3223|45262|1889975|1441|41707|2|58|1762|22|18.82|29.35|11.44|67.95|251.68|414.04|645.70|3.67|67.95|183.73|187.40|-230.31| +2452531|57165|2538|45262|1889975|1441|41707|2|143|1762|82|88.78|110.97|69.91|1949.09|5732.62|7279.96|9099.54|189.17|1949.09|3783.53|3972.70|-3496.43| +2452531|57165|10335|45262|1889975|1441|41707|2|269|1762|89|90.16|169.50|118.65|0.00|10559.85|8024.24|15085.50|633.59|0.00|10559.85|11193.44|2535.61| +2452531|57165|1027|45262|1889975|1441|41707|2|227|1762|94|65.39|101.35|27.36|0.00|2571.84|6146.66|9526.90|128.59|0.00|2571.84|2700.43|-3574.82| +|67184|17210|75330|560600|2670|24740|||1763|86|69.51|109.82||||5977.86||||||| +2451547|67184|10451|75330|560600|2670|24740|7|275|1763|24|26.61|26.87|19.34|0.00|464.16|638.64|644.88|32.49|0.00|464.16|496.65|-174.48| +2451547|67184|1904|75330|560600|2670|24740|7|264|1763|93|19.70|36.64|7.69|0.00|715.17|1832.10|3407.52|50.06|0.00|715.17|765.23|-1116.93| +2451547|67184|11732|75330|560600|2670|24740|7|39|1763|6|32.60|52.48|40.40|0.00|242.40|195.60|314.88|7.27|0.00|242.40|249.67|46.80| +2451547|67184|12367|75330|560600|2670|24740|7|300|1763|50|52.60|73.11|11.69|0.00|584.50|2630.00|3655.50|23.38|0.00|584.50|607.88|-2045.50| +2451547|67184|13037|75330|560600|2670|24740|7|250|1763|56|47.89|72.31|13.01|43.71|728.56|2681.84|4049.36|61.63|43.71|684.85|746.48|-1996.99| +2451547|67184|11359|75330|560600|2670|24740|7|277|1763|53|88.68|96.66|93.76|0.00|4969.28|4700.04|5122.98|298.15|0.00|4969.28|5267.43|269.24| +2451547|67184|17137|75330|560600|2670|24740|7|129|1763|51|54.60|58.96|38.91|297.66|1984.41|2784.60|3006.96|67.47|297.66|1686.75|1754.22|-1097.85| +2451547|67184|9380|75330|560600|2670|24740|7|284|1763|20|75.86|120.61|56.68|0.00|1133.60|1517.20|2412.20|45.34|0.00|1133.60|1178.94|-383.60| +2451547|67184|5906|75330|560600|2670|24740|7|146|1763|77|80.87|109.98|51.69|0.00|3980.13|6226.99|8468.46|159.20|0.00|3980.13|4139.33|-2246.86| +2451547|67184|959|75330|560600|2670|24740|7|181|1763|75|16.48|30.32|14.25|0.00|1068.75|1236.00|2274.00|0.00|0.00|1068.75|1068.75|-167.25| +2451547|67184|3050|75330|560600|2670|24740|7|153|1763|11|46.34|78.31|34.45|7.57|378.95|509.74|861.41|11.14|7.57|371.38|382.52|-138.36| +2451547|67184|7778|75330|560600|2670|24740|7|172|1763|16|98.63|197.26|71.01|568.08|1136.16|1578.08|3156.16|28.40|568.08|568.08|596.48|-1010.00| +|67184|8905||560600|2670|24740|7|108|1763|||||||||||272.19|296.68|| +2451547|67184|5186|75330|560600|2670|24740|7|102|1763|25|77.08|111.76|110.64|0.00|2766.00|1927.00|2794.00|82.98|0.00|2766.00|2848.98|839.00| +2450847|74474|16184||1022792|4711||2|204|1764|88||21.30|11.28|||1388.64||19.85||||-396.00| +2450847|74474|13828|60073|1022792|4711|18230|2|86|1764|94|42.97|50.70|7.60|0.00|714.40|4039.18|4765.80|28.57|0.00|714.40|742.97|-3324.78| +2450847|74474|11900|60073|1022792|4711|18230|2|249|1764|20|54.85|66.36|51.76|0.00|1035.20|1097.00|1327.20|93.16|0.00|1035.20|1128.36|-61.80| +2450847|74474|11539|60073|1022792|4711|18230|2|100|1764|3|24.92|26.66|2.93|0.00|8.79|74.76|79.98|0.08|0.00|8.79|8.87|-65.97| +2450847|74474|12304|60073|1022792|4711|18230|2|262|1764|2|43.44|84.27|73.31|0.00|146.62|86.88|168.54|1.46|0.00|146.62|148.08|59.74| +2450847||2404|||4711|18230|2|238|1764||74.95|149.90|121.41|6070.50|12141.00|7495.00|||6070.50||6252.61|-1424.50| +2450847|74474|1033|60073|1022792|4711|18230|2|191|1764|14|68.43|129.33|111.22|0.00|1557.08|958.02|1810.62|31.14|0.00|1557.08|1588.22|599.06| +2450847|74474|8164|60073|1022792|4711|18230|2|221|1764|18|15.51|15.97|10.22|0.00|183.96|279.18|287.46|11.03|0.00|183.96|194.99|-95.22| +2450847|74474|10574|60073|1022792|4711|18230|2|275|1764|11|99.54|123.42|59.24|0.00|651.64|1094.94|1357.62|19.54|0.00|651.64|671.18|-443.30| +2452132|31040|7693|81696|781766|683|17553|4|292|1765|43|26.73|49.71|45.73|0.00|1966.39|1149.39|2137.53|78.65|0.00|1966.39|2045.04|817.00| +2452132|31040|119|81696|781766|683|17553|4|231|1765|78|21.22|31.40|27.00|926.64|2106.00|1655.16|2449.20|47.17|926.64|1179.36|1226.53|-475.80| +2452132|31040|14313|81696|781766|683|17553|4|75|1765|19|73.43|146.12|81.82|0.00|1554.58|1395.17|2776.28|0.00|0.00|1554.58|1554.58|159.41| +2452132||6547||781766|683|17553||138|1765||39.50|||||2528.00|3412.48|285.58|||3458.70|645.12| +2452132|31040|17315|81696|781766|683|17553|4|143|1765|31|25.68|40.31|35.47|0.00|1099.57|796.08|1249.61|87.96|0.00|1099.57|1187.53|303.49| +2452132|31040|10207|81696|781766|683|17553|4|204|1765|22|36.19|63.69|22.92|0.00|504.24|796.18|1401.18|35.29|0.00|504.24|539.53|-291.94| +2452132|31040|1327|81696|781766|683|17553|4|188|1765|5|95.19|136.12|119.78|365.32|598.90|475.95|680.60|9.34|365.32|233.58|242.92|-242.37| +2452132|31040|8227|81696|781766|683|17553|4|32|1765|54|51.49|76.20|10.66|0.00|575.64|2780.46|4114.80|34.53|0.00|575.64|610.17|-2204.82| +2452132|31040|7709|81696|781766|683|17553|4|187|1765|60|18.99|20.31|14.82|0.00|889.20|1139.40|1218.60|44.46|0.00|889.20|933.66|-250.20| +||13639||624308|2670||2|130|1766|||100.90||0.00|2209.50|2802.90|||0.00|||-593.40| +2451802|48060|1172|23546|624308|2670|49432|2|186|1766|18|36.05|71.73|30.12|449.99|542.16|648.90|1291.14|8.29|449.99|92.17|100.46|-556.73| +2451802|48060|1286|23546|624308|2670|49432|2|126|1766|19|72.97|137.18|105.62|0.00|2006.78|1386.43|2606.42|160.54|0.00|2006.78|2167.32|620.35| +2451802|48060|1397|23546|624308|2670|49432|2|264|1766|94|15.55|16.32|6.52|453.53|612.88|1461.70|1534.08|7.96|453.53|159.35|167.31|-1302.35| +2451802|48060|8648|23546|624308|2670|49432|2|183|1766|54|60.38|62.19|20.52|0.00|1108.08|3260.52|3358.26|77.56|0.00|1108.08|1185.64|-2152.44| +2451802|48060|2039|23546|624308|2670|49432|2|108|1766|33|52.87|78.77|48.04|0.00|1585.32|1744.71|2599.41|95.11|0.00|1585.32|1680.43|-159.39| +2451802|48060|11533|23546|624308|2670|49432|2|149|1766|76|92.68|115.85|40.54|1417.27|3081.04|7043.68|8804.60|49.91|1417.27|1663.77|1713.68|-5379.91| +2451802|48060|1229|23546|624308|2670|49432|2|192|1766|8|3.91|4.06|3.41|2.18|27.28|31.28|32.48|1.00|2.18|25.10|26.10|-6.18| +2451802|48060|6743|23546|624308|2670|49432|2|160|1766|39|29.17|57.75|8.08|0.00|315.12|1137.63|2252.25|9.45|0.00|315.12|324.57|-822.51| +2451263|67359|9388|86216|691280|5497|45714|1|234|1767|48|92.71|114.03|63.85|2482.48|3064.80|4450.08|5473.44|17.46|2482.48|582.32|599.78|-3867.76| +2451263|67359|625|86216|691280|5497|45714|1|85|1767|27|48.86|75.24|48.15|0.00|1300.05|1319.22|2031.48|26.00|0.00|1300.05|1326.05|-19.17| +||10318|86216|691280|||1|7|1767||56.08||||1232.00||2738.56|0.00||||| +2451263|67359|5683|86216|691280|5497|45714|1|61|1767|20|48.63|96.28|87.61|0.00|1752.20|972.60|1925.60|17.52|0.00|1752.20|1769.72|779.60| +2451263|67359|6868|86216|691280|5497|45714|1|173|1767|20|73.76|143.83|89.17|0.00|1783.40|1475.20|2876.60|160.50|0.00|1783.40|1943.90|308.20| +2451263|67359|11074|86216|691280|5497|45714|1|88|1767|52|63.32|102.57|34.87|0.00|1813.24|3292.64|5333.64|72.52|0.00|1813.24|1885.76|-1479.40| +2451263|67359|5986|86216|691280|5497|45714|1|64|1767|30|55.22|66.81|1.33|0.00|39.90|1656.60|2004.30|1.99|0.00|39.90|41.89|-1616.70| +2451263|67359|7600|86216|691280|5497|45714|1|79|1767|86|97.06|121.32|27.90|1511.62|2399.40|8347.16|10433.52|79.90|1511.62|887.78|967.68|-7459.38| +2451263|67359|9775|86216|691280|5497|45714|1|221|1767|23|11.13|17.36|8.33|1.91|191.59|255.99|399.28|3.79|1.91|189.68|193.47|-66.31| +2451263|67359|2833|86216|691280|5497|45714|1|135|1767|37|4.37|5.20|1.66|0.00|61.42|161.69|192.40|5.52|0.00|61.42|66.94|-100.27| +2451263|67359|4819|86216|691280|5497|45714|1|106|1767|99|70.12|102.37|75.75|0.00|7499.25|6941.88|10134.63|674.93|0.00|7499.25|8174.18|557.37| +2451850|40567|16430|42566|1267823|3480|43090|1|181|1768|4|87.65|137.61|61.92|0.00|247.68|350.60|550.44|0.00|0.00|247.68|247.68|-102.92| +2451850|40567|6127|42566|1267823|3480|43090|1|41|1768|98|45.97|59.76|15.53|0.00|1521.94|4505.06|5856.48|30.43|0.00|1521.94|1552.37|-2983.12| +2451850|40567|15227|42566|1267823|3480|43090|1|134|1768|81|10.98|11.30|1.24|7.03|100.44|889.38|915.30|7.47|7.03|93.41|100.88|-795.97| +2451850|40567|9817|42566|1267823|3480|43090|1|284|1768|27|46.03|67.66|3.38|79.39|91.26|1242.81|1826.82|0.23|79.39|11.87|12.10|-1230.94| +2451850|40567|15716|42566|1267823|3480|43090|1|232|1768|54|54.02|103.17|60.87|0.00|3286.98|2917.08|5571.18|164.34|0.00|3286.98|3451.32|369.90| +2451850|40567|1525|42566|1267823|3480|43090|1|28|1768|97|83.06|117.11|81.97|0.00|7951.09|8056.82|11359.67|636.08|0.00|7951.09|8587.17|-105.73| +2451850|40567|4555|42566|1267823|3480|43090|1|28|1768|47|29.88|30.47|21.32|0.00|1002.04|1404.36|1432.09|80.16|0.00|1002.04|1082.20|-402.32| +2451850|40567|15020|42566|1267823|3480|43090|1|160|1768|46|85.66|128.49|88.65|0.00|4077.90|3940.36|5910.54|244.67|0.00|4077.90|4322.57|137.54| +2451850|40567|7868|42566|1267823|3480|43090|1|44|1768|43|15.43|19.90|14.52|0.00|624.36|663.49|855.70|0.00|0.00|624.36|624.36|-39.13| +2451805|38986|3097|37841|1648168|448|9128|7|233|1769|34|10.14|13.38|8.69|0.00|295.46|344.76|454.92|8.86|0.00|295.46|304.32|-49.30| +2451805|38986|2197|37841|1648168|448|9128|7|65|1769|67|12.12|15.87|13.48|0.00|903.16|812.04|1063.29|36.12|0.00|903.16|939.28|91.12| +2451805|38986|5990|37841|1648168||9128||229|1769|94|43.04|78.33||3292.95||4045.76|7363.02|80.70|3292.95|1345.01||-2700.75| +2451805|38986|10820|37841|1648168|448|9128|7|139|1769|58|2.80|3.75|1.68|0.00|97.44|162.40|217.50|8.76|0.00|97.44|106.20|-64.96| +2451805|38986|16243|37841|1648168|448|9128|7|27|1769|20|14.76|16.97|11.03|0.00|220.60|295.20|339.40|11.03|0.00|220.60|231.63|-74.60| +2451805|38986|2915|37841|1648168|448|9128|7|264|1769|68|50.63|82.02|20.50|0.00|1394.00|3442.84|5577.36|55.76|0.00|1394.00|1449.76|-2048.84| +2451805|38986|17575|37841|1648168|448|9128|7|115|1769|78|65.86|76.39|73.33|1429.93|5719.74|5137.08|5958.42|0.00|1429.93|4289.81|4289.81|-847.27| +2451805|38986|6314|37841|1648168|448|9128|7|213|1769|42|57.25|92.74|83.46|0.00|3505.32|2404.50|3895.08|105.15|0.00|3505.32|3610.47|1100.82| +2451805|38986|5153|37841|1648168|448|9128|7|225|1769|99|45.49|64.59|52.96|5085.74|5243.04|4503.51|6394.41|11.01|5085.74|157.30|168.31|-4346.21| +2451805|38986|16610|37841|1648168|448|9128|7|269|1769|64|89.18|166.76|115.06|0.00|7363.84|5707.52|10672.64|220.91|0.00|7363.84|7584.75|1656.32| +|38986|13937|37841|1648168|448||7||1769|||105.68|||1183.60||2113.60|0.00||1183.60|1183.60|| +2451805|38986|16205|37841|1648168|448|9128|7|33|1769|69|50.55|97.56|29.26|524.92|2018.94|3487.95|6731.64|119.52|524.92|1494.02|1613.54|-1993.93| +2452224|31221|16229|73117|1474529|388|16200|8|124|1770|4|62.29|89.69|87.89|0.00|351.56|249.16|358.76|31.64|0.00|351.56|383.20|102.40| +2452224|31221|6711|73117|1474529|388|16200|8|222|1770|60|88.50|153.99|90.85|545.10|5451.00|5310.00|9239.40|392.47|545.10|4905.90|5298.37|-404.10| +2452224|31221|12837|73117|1474529|388|16200|8|158|1770|1|27.53|30.00|1.80|0.00|1.80|27.53|30.00|0.10|0.00|1.80|1.90|-25.73| +2452224|31221|8461|73117|1474529|388|16200|8|51|1770|49|47.56|58.49|40.35|0.00|1977.15|2330.44|2866.01|19.77|0.00|1977.15|1996.92|-353.29| +2452224|31221|16535|73117|1474529|388|16200|8|271|1770|45|83.78|142.42|56.96|615.16|2563.20|3770.10|6408.90|19.48|615.16|1948.04|1967.52|-1822.06| +2452224|31221|16099|73117|1474529|388|16200|8|5|1770|21|42.94|82.87|25.68|399.06|539.28|901.74|1740.27|9.81|399.06|140.22|150.03|-761.52| +|31221|5781||1474529|||8|138|1770||74.03||||||||||851.58|| +2452224|31221|3651|73117|1474529|388|16200|8|131|1770|85|11.98|21.20|8.90|627.89|756.50|1018.30|1802.00|6.43|627.89|128.61|135.04|-889.69| +2452224|31221|10571|73117|1474529|388|16200|8|226|1770|34|17.18|24.73|4.45|0.00|151.30|584.12|840.82|3.02|0.00|151.30|154.32|-432.82| +2452224|31221|5443|73117|1474529|388|16200|8|229|1770|10|88.20|112.01|31.36|0.00|313.60|882.00|1120.10|15.68|0.00|313.60|329.28|-568.40| +2452224|31221|6011|73117|1474529|388|16200|8|212|1770|71|22.74|31.83|11.14|0.00|790.94|1614.54|2259.93|63.27|0.00|790.94|854.21|-823.60| +2452224|31221|7467|73117|1474529|388|16200|8|119|1770|73|68.78|74.97|13.49|0.00|984.77|5020.94|5472.81|59.08|0.00|984.77|1043.85|-4036.17| +2452224|31221|17069||||16200|||1770||97.05|98.02|||5928.00|||18.96||||| +2452224|31221|16097|||388|16200||291|1770|||96.68|49.30|0.00|2859.40|3665.02|||0.00|2859.40|3088.15|| +2452224|31221|9933|73117|1474529|388|16200|8|109|1770|6|16.31|29.68|26.11|151.96|156.66|97.86|178.08|0.32|151.96|4.70|5.02|-93.16| +2452224|31221|11133|73117|1474529|388|16200|8|180|1770|88|81.63|138.77|30.52|0.00|2685.76|7183.44|12211.76|0.00|0.00|2685.76|2685.76|-4497.68| +2451245|51056|1988|63422|24548|4521|33294|7|66|1771|34|17.06|24.56|7.85|0.00|266.90|580.04|835.04|21.35|0.00|266.90|288.25|-313.14| +2451245|51056|9679|63422|24548|4521|33294|7|124|1771|93|66.80|132.93|107.67|0.00|10013.31|6212.40|12362.49|700.93|0.00|10013.31|10714.24|3800.91| +2451245|51056|652|63422|24548|4521|33294|7|249|1771|64|9.88|14.22|12.22|0.00|782.08|632.32|910.08|70.38|0.00|782.08|852.46|149.76| +2451245|51056|8906|63422|24548|4521|33294|7|292|1771|56|35.23|66.23|21.85|0.00|1223.60|1972.88|3708.88|0.00|0.00|1223.60|1223.60|-749.28| +2451245|51056|11830|63422|24548|4521|33294|7|15|1771|35|36.58|58.52|5.26|25.77|184.10|1280.30|2048.20|1.58|25.77|158.33|159.91|-1121.97| +2451245|51056|17156|63422|24548|4521|33294|7|212|1771|84|99.26|113.15|81.46|5474.11|6842.64|8337.84|9504.60|13.68|5474.11|1368.53|1382.21|-6969.31| +2451245|51056|8960|63422|24548|4521|33294|7|175|1771|34|58.50|63.76|40.16|0.00|1365.44|1989.00|2167.84|81.92|0.00|1365.44|1447.36|-623.56| +2451245|51056|7435|63422|24548|4521|33294|7|38|1771|60|77.77|120.54|33.75|0.00|2025.00|4666.20|7232.40|162.00|0.00|2025.00|2187.00|-2641.20| +2451245|51056|3847|63422|24548|4521|33294|7|234|1771|87|95.51|118.43|27.23|1824.13|2369.01|8309.37|10303.41|43.59|1824.13|544.88|588.47|-7764.49| +2451245|51056|5737|63422|24548|4521|33294|7|135|1771|39|1.12|2.18|1.72|42.26|67.08|43.68|85.02|0.74|42.26|24.82|25.56|-18.86| +2451245|51056|12865|63422|24548|4521|33294|7|240|1771|65|82.76|130.76|49.68|0.00|3229.20|5379.40|8499.40|32.29|0.00|3229.20|3261.49|-2150.20| +2451245|51056|3271|63422|24548|4521|33294|7|247|1771|55|95.56|137.60|44.03|72.64|2421.65|5255.80|7568.00|46.98|72.64|2349.01|2395.99|-2906.79| +2451245|51056|3541|63422|24548|4521|33294|7|172|1771|39|37.33|50.02|10.00|0.00|390.00|1455.87|1950.78|15.60|0.00|390.00|405.60|-1065.87| +2452171|37724|12153|81001|1725333|4512|11644|1|2|1772|61|76.19|108.18|47.59|0.00|2902.99|4647.59|6598.98|232.23|0.00|2902.99|3135.22|-1744.60| +2452171||4143|81001|1725333|4512|11644|||1772|78||||0.00|1255.02|||25.10|0.00||1280.12|274.56| +2452171|37724|16573|81001|1725333|4512|11644|1|185|1772|45|65.08|81.35|4.06|104.13|182.70|2928.60|3660.75|1.57|104.13|78.57|80.14|-2850.03| +2452171|37724|8513|81001|1725333|4512|11644|1|69|1772|40|67.03|132.04|87.14|0.00|3485.60|2681.20|5281.60|243.99|0.00|3485.60|3729.59|804.40| +2452171|37724|4395|81001|1725333|4512|11644|1|138|1772|16|83.84|97.25|82.66|0.00|1322.56|1341.44|1556.00|39.67|0.00|1322.56|1362.23|-18.88| +2452171|37724|16553|81001|1725333|4512|11644|1|255|1772|56|92.41|155.24|63.64|0.00|3563.84|5174.96|8693.44|285.10|0.00|3563.84|3848.94|-1611.12| +2452171|37724|2597|81001|1725333|4512|11644|1|4|1772|47|89.37|134.94|66.12|0.00|3107.64|4200.39|6342.18|186.45|0.00|3107.64|3294.09|-1092.75| +2452171|37724|1713|81001|1725333|4512|11644|1|218|1772|84|6.15|8.48|4.66|180.06|391.44|516.60|712.32|14.79|180.06|211.38|226.17|-305.22| +2452171|37724|15545|81001|1725333|4512|11644|1|117|1772|43|21.16|31.31|15.02|0.00|645.86|909.88|1346.33|12.91|0.00|645.86|658.77|-264.02| +2452171|37724|12971|81001|1725333|4512|11644|1|236|1772|71|35.01|58.11|45.90|0.00|3258.90|2485.71|4125.81|130.35|0.00|3258.90|3389.25|773.19| +2452171|37724|13901|81001|1725333|4512|11644|1|61|1772|38|96.87|166.61|131.62|4901.52|5001.56|3681.06|6331.18|1.00|4901.52|100.04|101.04|-3581.02| +2452171|37724|1413|81001|1725333|4512|11644|1|81|1772|10|3.22|3.73|0.11|0.00|1.10|32.20|37.30|0.05|0.00|1.10|1.15|-31.10| +2452171|37724|8163|81001|1725333|4512|11644|1|265|1772|80|12.24|19.46|3.11|0.00|248.80|979.20|1556.80|7.46|0.00|248.80|256.26|-730.40| +2452171|37724|13441|81001|1725333|4512|11644|1|112|1772|48|23.99|39.58|23.74|0.00|1139.52|1151.52|1899.84|34.18|0.00|1139.52|1173.70|-12.00| +2452579|34889|7993|10556|1271463|5617|38524|4|185|1773|22|70.43|123.95|12.39|21.80|272.58|1549.46|2726.90|7.52|21.80|250.78|258.30|-1298.68| +2452579|34889|15067|10556|1271463|5617|38524|4|242|1773|30|||11.86|||1716.30|2093.70|||||-1360.50| +2452579|34889|907|10556|1271463|5617|38524|4|156|1773|27|93.89|137.07|68.53|0.00|1850.31|2535.03|3700.89|0.00|0.00|1850.31|1850.31|-684.72| +2452579|34889|1407|10556|1271463|5617|38524|4|85|1773|98|56.99|68.95|13.10|0.00|1283.80|5585.02|6757.10|77.02|0.00|1283.80|1360.82|-4301.22| +2452579|34889|17785|10556|1271463|5617|38524|4|81|1773|11|60.74|83.82|41.91|0.00|461.01|668.14|922.02|9.22|0.00|461.01|470.23|-207.13| +2452579|34889|9627|10556|1271463|5617|38524|4|33|1773|87|35.45|69.48|61.83|0.00|5379.21|3084.15|6044.76|376.54|0.00|5379.21|5755.75|2295.06| +2452579|34889|17599|10556|1271463|5617|38524|4|268|1773|39|38.76|73.25|30.03|0.00|1171.17|1511.64|2856.75|81.98|0.00|1171.17|1253.15|-340.47| +2452579|34889|10008|10556|1271463|5617|38524|4|55|1773|68|96.65|169.13|50.73|0.00|3449.64|6572.20|11500.84|137.98|0.00|3449.64|3587.62|-3122.56| +2452579|34889|2178|10556|1271463|5617|38524|4|45|1773|92|14.14|26.72|8.55|0.00|786.60|1300.88|2458.24|7.86|0.00|786.60|794.46|-514.28| +2452579|34889|6522|10556|1271463|5617|38524|4|268|1773|38|53.05|78.51|14.13|0.00|536.94|2015.90|2983.38|37.58|0.00|536.94|574.52|-1478.96| +2451608|60260|17077|26412|1658794|3176|21158|2|27|1774|4|90.45|138.38|113.47|0.00|453.88|361.80|553.52|13.61|0.00|453.88|467.49|92.08| +2451608|60260|12302|26412|1658794|3176|21158|2|154|1774|51|53.01|65.73|59.15|0.00|3016.65|2703.51|3352.23|120.66|0.00|3016.65|3137.31|313.14| +2451608|60260|14414|26412|1658794|3176|21158|2|48|1774|86|16.49|18.13|3.26|0.00|280.36|1418.14|1559.18|25.23|0.00|280.36|305.59|-1137.78| +2451608|60260|13603|26412|1658794|3176|21158|2|36|1774|83|58.40|80.59|4.02|0.00|333.66|4847.20|6688.97|16.68|0.00|333.66|350.34|-4513.54| +2451608|60260|8963|26412|1658794|3176|21158|2|151|1774|92|67.26|117.03|25.74|0.00|2368.08|6187.92|10766.76|0.00|0.00|2368.08|2368.08|-3819.84| +2451608|60260|9014|26412|1658794|3176|21158|2|299|1774|68|73.68|105.36|30.55|0.00|2077.40|5010.24|7164.48|124.64|0.00|2077.40|2202.04|-2932.84| +2451608|60260|3332|26412|1658794|3176|21158|2|211|1774|4|26.88|32.79|28.19|90.20|112.76|107.52|131.16|1.80|90.20|22.56|24.36|-84.96| +2451608|60260|2699|26412|1658794|3176|21158|2|211|1774|95|90.95|95.49|95.49|0.00|9071.55|8640.25|9071.55|0.00|0.00|9071.55|9071.55|431.30| +2451608|60260|2785|26412|1658794|3176|21158|2|181|1774|22|16.30|22.65|22.19|0.00|488.18|358.60|498.30|14.64|0.00|488.18|502.82|129.58| +2451608|60260|7064|26412|1658794|3176|21158|2|183|1774|54|98.05|168.64|102.87|0.00|5554.98|5294.70|9106.56|0.00|0.00|5554.98|5554.98|260.28| +2451608|60260|2228|26412|1658794|3176|21158|2|74|1774|98|46.57|49.82|43.34|0.00|4247.32|4563.86|4882.36|169.89|0.00|4247.32|4417.21|-316.54| +2451608|60260|17885|26412|1658794|3176|21158|2|178|1774|52|72.21|129.25|94.35|1520.92|4906.20|3754.92|6721.00|0.00|1520.92|3385.28|3385.28|-369.64| +2451608|60260|7862|26412|1658794|3176|21158|2|223|1774|78|78.08|125.70|38.96|0.00|3038.88|6090.24|9804.60|121.55|0.00|3038.88|3160.43|-3051.36| +2451608|60260|14999|26412|1658794|3176|21158|2|249|1774|67|21.18|22.87|3.65|0.00|244.55|1419.06|1532.29|4.89|0.00|244.55|249.44|-1174.51| +2451608|60260|13139|26412|1658794|3176|21158|2|159|1774|49|52.67|62.15|49.09|673.51|2405.41|2580.83|3045.35|155.87|673.51|1731.90|1887.77|-848.93| +2452263|63660|15717|45341|579922|4103|49884|8|240|1775|34|95.56|169.14|84.57|632.58|2875.38|3249.04|5750.76|67.28|632.58|2242.80|2310.08|-1006.24| +2452263|63660|1525|45341|579922|4103|49884|8|231|1775|5|35.37|69.67|16.72|0.00|83.60|176.85|348.35|7.52|0.00|83.60|91.12|-93.25| +2452263|63660|4555|45341|579922|4103|49884|8|86|1775|5|64.76|110.73|40.97|0.00|204.85|323.80|553.65|6.14|0.00|204.85|210.99|-118.95| +2452263|63660|15021|45341|579922|4103|49884|8|16|1775|73|43.71|45.45|8.63|176.39|629.99|3190.83|3317.85|27.21|176.39|453.60|480.81|-2737.23| +2452263|63660|7869|45341|579922|4103|49884|8|279|1775|22|98.91|148.36|29.67|0.00|652.74|2176.02|3263.92|52.21|0.00|652.74|704.95|-1523.28| +2452263|63660|4489|45341|579922|4103|49884|8|6|1775|91|20.10|38.99|8.57|0.00|779.87|1829.10|3548.09|54.59|0.00|779.87|834.46|-1049.23| +2452263|63660|8425|45341|579922|4103|49884|8|127|1775|67|38.49|73.90|28.82|231.71|1930.94|2578.83|4951.30|16.99|231.71|1699.23|1716.22|-879.60| +2452263|63660|3695|45341|579922|4103|49884|8|213|1775|13|45.78|88.35|12.36|54.63|160.68|595.14|1148.55|2.12|54.63|106.05|108.17|-489.09| +2452263|63660|7131|45341|579922|4103|49884|8|147|1775|22|49.06|94.19|28.25|435.05|621.50|1079.32|2072.18|11.18|435.05|186.45|197.63|-892.87| +2452204|49230|2179|79964|1640274|5384|5294|2|142|1776|24|25.91|51.56|38.67|0.00|928.08|621.84|1237.44|27.84|0.00|928.08|955.92|306.24| +2452204|49230|6641|79964|1640274|5384|5294|2|136|1776|2|48.27|63.71|43.32|0.00|86.64|96.54|127.42|6.93|0.00|86.64|93.57|-9.90| +2452204|49230|8763|79964|1640274|5384|5294|2|180|1776|63|78.76|140.19|75.70|0.00|4769.10|4961.88|8831.97|190.76|0.00|4769.10|4959.86|-192.78| +2452204|49230|9727|79964|1640274|5384|5294|2|233|1776|35|96.19|187.57|168.81|0.00|5908.35|3366.65|6564.95|177.25|0.00|5908.35|6085.60|2541.70| +2452204|49230|2449|79964|1640274|5384|5294|2|296|1776|91|4.59|7.52|3.30|0.00|300.30|417.69|684.32|24.02|0.00|300.30|324.32|-117.39| +2452204|49230|10241|79964|1640274|5384|5294|2|217|1776|48|68.37|84.77|53.40|845.85|2563.20|3281.76|4068.96|103.04|845.85|1717.35|1820.39|-1564.41| +2452204|49230|5699|79964|1640274|5384|5294|2|205|1776|14|64.84|100.50|75.37|0.00|1055.18|907.76|1407.00|10.55|0.00|1055.18|1065.73|147.42| +2452204|49230|16631|79964|1640274|5384|5294|2|11|1776|81|29.13|48.64|2.91|0.00|235.71|2359.53|3939.84|0.00|0.00|235.71|235.71|-2123.82| +2452204|49230|10195|79964|1640274|5384|5294|2|119|1776|80|23.20|46.16|24.92|0.00|1993.60|1856.00|3692.80|19.93|0.00|1993.60|2013.53|137.60| +2451169|43207|8497|49984|1782475|2613|25951|10|69|1777|95|38.72|74.34|28.99|0.00|2754.05|3678.40|7062.30|27.54|0.00|2754.05|2781.59|-924.35| +2451169|43207|5728|49984|1782475|2613|25951|10|253|1777|79|15.53|19.41|10.09|0.00|797.11|1226.87|1533.39|63.76|0.00|797.11|860.87|-429.76| +2451169|43207|7987|49984|1782475|2613|25951|10|151|1777|23|32.18|35.07|1.40|0.00|32.20|740.14|806.61|0.00|0.00|32.20|32.20|-707.94| +2451169|43207|5300|49984|1782475|2613|25951|10|129|1777|93|15.04|16.54|13.89|0.00|1291.77|1398.72|1538.22|116.25|0.00|1291.77|1408.02|-106.95| +2451169|43207|2860|49984|1782475|2613|25951|10|14|1777|88|35.58|39.13|5.08|0.00|447.04|3131.04|3443.44|35.76|0.00|447.04|482.80|-2684.00| +2451169|43207|11108|49984|1782475|2613|25951|10|34|1777|52|79.41|128.64|25.72|0.00|1337.44|4129.32|6689.28|120.36|0.00|1337.44|1457.80|-2791.88| +2451169|43207|6626|49984|1782475|2613|25951|10|6|1777|90|8.16|8.97|6.72|0.00|604.80|734.40|807.30|6.04|0.00|604.80|610.84|-129.60| +2451169|43207|7832|49984||2613||10||1777|||||0.00|420.48||977.88||0.00|420.48|420.48|| +2451169|43207|11062|49984|1782475|2613|25951|10|41|1777|73|23.50|38.77|15.89|0.00|1159.97|1715.50|2830.21|81.19|0.00|1159.97|1241.16|-555.53| +2451169|43207|11665|49984|1782475|2613|25951|10|239|1777|72|89.46|118.98|9.51|0.00|684.72|6441.12|8566.56|20.54|0.00|684.72|705.26|-5756.40| +2451169|43207|5446|49984|1782475|2613|25951|10|156|1777|99|49.94|63.92|6.39|0.00|632.61|4944.06|6328.08|0.00|0.00|632.61|632.61|-4311.45| +2451169|43207|1154|49984|1782475|2613|25951|10|85|1777|96|42.14|67.84|29.17|0.00|2800.32|4045.44|6512.64|28.00|0.00|2800.32|2828.32|-1245.12| +2451169|43207|6931|49984|1782475|2613|25951|10|252|1777|38|78.78|81.14|66.53|404.50|2528.14|2993.64|3083.32|42.47|404.50|2123.64|2166.11|-870.00| +2451169|43207|6094|49984|1782475|2613|25951|10|149|1777|16|64.31|66.88|36.11|0.00|577.76|1028.96|1070.08|51.99|0.00|577.76|629.75|-451.20| +2452175|57548|17275|94355|1641583|3211|40086|2|110|1778|42|85.58|160.03|83.21|0.00|3494.82|3594.36|6721.26|244.63|0.00|3494.82|3739.45|-99.54| +2452175|57548|6449|94355|1641583|3211|40086|2|131|1778|60|83.49|86.82|85.95|0.00|5157.00|5009.40|5209.20|51.57|0.00|5157.00|5208.57|147.60| +2452175|57548|13049|94355|1641583|3211|40086|2|117|1778|65|76.28|131.20|112.83|0.00|7333.95|4958.20|8528.00|660.05|0.00|7333.95|7994.00|2375.75| +2452175|57548|7401|94355|1641583|3211|40086|2|104|1778|5|47.83|76.04|62.35|18.70|311.75|239.15|380.20|0.00|18.70|293.05|293.05|53.90| +2452175|57548|17683|94355|1641583|3211|40086|2|194|1778|30|32.67|61.09|40.93|0.00|1227.90|980.10|1832.70|61.39|0.00|1227.90|1289.29|247.80| +2452175|57548|4679|94355|1641583|3211|40086|2|35|1778|55|96.48|185.24|64.83|0.00|3565.65|5306.40|10188.20|0.00|0.00|3565.65|3565.65|-1740.75| +2452175|57548|6811|94355|1641583|3211|40086|2|238|1778|39|63.71|100.66|68.44|0.00|2669.16|2484.69|3925.74|240.22|0.00|2669.16|2909.38|184.47| +2452175|57548|13177|94355|1641583|3211|40086|2|133|1778|60|49.67|97.84|24.46|0.00|1467.60|2980.20|5870.40|58.70|0.00|1467.60|1526.30|-1512.60| +2452175|57548|2093|94355|1641583|3211|40086|2|82|1778|96|79.06|104.35|94.95|0.00|9115.20|7589.76|10017.60|729.21|0.00|9115.20|9844.41|1525.44| +2452175|57548|11203|94355|1641583|3211|40086|2|112|1778|52|57.80|71.67|43.00|1632.28|2236.00|3005.60|3726.84|6.03|1632.28|603.72|609.75|-2401.88| +2451530|32614|8650|40142||5681|19874|||1779|||||||6642.00||18.27||609.30||| +2451530|32614|11678|40142|1307935|5681|19874|4|162|1779|41|15.17|19.72|15.38|0.00|630.58|621.97|808.52|31.52|0.00|630.58|662.10|8.61| +2451530|32614|14008|40142|1307935|5681|19874|4|32|1779|82|95.23|148.55|115.86|0.00|9500.52|7808.86|12181.10|285.01|0.00|9500.52|9785.53|1691.66| +2451530|32614|12139|40142|1307935|5681|19874|4|149|1779|19|29.03|29.61|18.95|0.00|360.05|551.57|562.59|21.60|0.00|360.05|381.65|-191.52| +2451530|32614|6871|40142|1307935|5681|19874|4|98|1779|35|62.81|112.42|29.22|0.00|1022.70|2198.35|3934.70|10.22|0.00|1022.70|1032.92|-1175.65| +2451530|32614|10390|40142|1307935|5681|19874|4|152|1779|27|98.71|98.71|65.14|0.00|1758.78|2665.17|2665.17|105.52|0.00|1758.78|1864.30|-906.39| +2451530|32614|1462|40142|1307935|5681|19874|4|53|1779|12|38.72|63.50|56.51|0.00|678.12|464.64|762.00|40.68|0.00|678.12|718.80|213.48| +2451530|32614|11624|40142|1307935|5681|19874|4|252|1779|71|16.20|28.83|1.15|0.00|81.65|1150.20|2046.93|4.89|0.00|81.65|86.54|-1068.55| +2451530|32614|14578|40142|1307935|5681|19874|4|49|1779|95|83.99|104.14|7.28|0.00|691.60|7979.05|9893.30|48.41|0.00|691.60|740.01|-7287.45| +2451530|32614|8402|40142|1307935|5681|19874|4|229|1779|21|6.35|10.22|4.19|0.00|87.99|133.35|214.62|4.39|0.00|87.99|92.38|-45.36| +2451530|32614|14732|40142|1307935|5681|19874|4|10|1779|93|62.36|74.83|33.67|0.00|3131.31|5799.48|6959.19|187.87|0.00|3131.31|3319.18|-2668.17| +2451530|32614|5713|40142|1307935|5681|19874|4|129|1779|61|4.67|5.37|5.10|0.00|311.10|284.87|327.57|21.77|0.00|311.10|332.87|26.23| +2451530|32614|2329|40142|1307935|5681|19874|4|203|1779|29|57.87|85.64|5.99|0.00|173.71|1678.23|2483.56|10.42|0.00|173.71|184.13|-1504.52| +2451530|32614|10892|40142|1307935|5681|19874|4|135|1779|97|43.41|51.22|42.00|0.00|4074.00|4210.77|4968.34|366.66|0.00|4074.00|4440.66|-136.77| +2451530|32614|3418|40142|1307935|5681|19874|4|22|1779|17|53.59|71.81|4.30|69.44|73.10|911.03|1220.77|0.29|69.44|3.66|3.95|-907.37| +2451530|32614|16699|40142|1307935|5681|19874|4|167|1779|63|14.28|15.27|5.34|0.00|336.42|899.64|962.01|0.00|0.00|336.42|336.42|-563.22| +2452127|45278|12823|74196|521206|2969|4332|1|214|1780|1|14.52|20.90|14.00|0.00|14.00|14.52|20.90|0.42|0.00|14.00|14.42|-0.52| +2452127|45278|7701|74196|521206|2969|4332|1|49|1780|85|88.00|138.16|132.63|0.00|11273.55|7480.00|11743.60|338.20|0.00|11273.55|11611.75|3793.55| +2452127|45278|443|74196|521206|2969|4332|1|6|1780|31|68.06|88.47|19.46|0.00|603.26|2109.86|2742.57|12.06|0.00|603.26|615.32|-1506.60| +2452127|45278|2805|74196|521206|2969|4332|1|96|1780|4|46.21|54.98|30.23|0.00|120.92|184.84|219.92|10.88|0.00|120.92|131.80|-63.92| +2452127|45278|7371|74196|521206|2969|4332|1|203|1780|56|77.90|93.48|82.26|3869.51|4606.56|4362.40|5234.88|36.85|3869.51|737.05|773.90|-3625.35| +2452127|45278|14355|74196|521206|2969|4332|1|105|1780|92|36.60|62.58|43.18|397.25|3972.56|3367.20|5757.36|250.27|397.25|3575.31|3825.58|208.11| +2452127|45278|3027|74196|521206|2969|4332|1|280|1780|24|47.05|58.81|8.23|0.00|197.52|1129.20|1411.44|5.92|0.00|197.52|203.44|-931.68| +2452127|45278|5407|74196|521206|2969|4332|1|14|1780|80|92.50|162.80|63.49|0.00|5079.20|7400.00|13024.00|457.12|0.00|5079.20|5536.32|-2320.80| +2452127|45278|1277|74196|521206|2969|4332|1|214|1780|80|18.53|19.64|10.01|0.00|800.80|1482.40|1571.20|56.05|0.00|800.80|856.85|-681.60| +2452127|45278|7403|74196|521206|2969|4332|1|60|1780|60|1.96|3.76|1.24|0.00|74.40|117.60|225.60|6.69|0.00|74.40|81.09|-43.20| +2452127|45278|3381|74196|521206|2969|4332|1|94|1780|11|15.05|18.21|6.73|0.00|74.03|165.55|200.31|0.74|0.00|74.03|74.77|-91.52| +2451550|37634|10123|87761|473358|1355|22952|4|13|1781|67|50.37|95.70|90.91|0.00|6090.97|3374.79|6411.90|182.72|0.00|6090.97|6273.69|2716.18| +2451550|37634|3589|87761|473358|1355|22952|4|184|1781|53|9.39|16.05|6.25|298.12|331.25|497.67|850.65|2.65|298.12|33.13|35.78|-464.54| +2451550|37634|2705|87761|473358|1355|22952|4|213|1781|15|58.85|72.38|34.74|0.00|521.10|882.75|1085.70|26.05|0.00|521.10|547.15|-361.65| +2451550|37634|15355|87761|473358|1355|22952|4|210|1781|68|11.65|12.11|7.50|0.00|510.00|792.20|823.48|35.70|0.00|510.00|545.70|-282.20| +2451550|37634|3548|87761|473358|1355|22952|4|166|1781|51|94.96|108.25|69.28|3073.95|3533.28|4842.96|5520.75|0.00|3073.95|459.33|459.33|-4383.63| +2451550|37634|12953|87761|473358|1355|22952|4|209|1781|12|19.86|26.81|11.79|79.22|141.48|238.32|321.72|1.86|79.22|62.26|64.12|-176.06| +2451550|37634|5192|87761|473358|1355|22952|4|18|1781|54|23.78|47.08|47.08|0.00|2542.32|1284.12|2542.32|25.42|0.00|2542.32|2567.74|1258.20| +2451550|37634|11594|87761|473358|1355|22952|4|58|1781|58|45.80|76.48|29.82|0.00|1729.56|2656.40|4435.84|51.88|0.00|1729.56|1781.44|-926.84| +2451550|37634|17405|87761|473358|1355|22952|4|113|1781|27|55.17|93.78|36.57|0.00|987.39|1489.59|2532.06|88.86|0.00|987.39|1076.25|-502.20| +2451550|37634|16051|87761|473358|1355|22952|4|12|1781|7|67.49|77.61|69.84|0.00|488.88|472.43|543.27|24.44|0.00|488.88|513.32|16.45| +||8840||473358||22952|4||1781||72.30||||||5426.46||||5075.85|| +2451550|37634|15674|87761|473358|1355|22952|4|247|1781|68|71.65|128.97|60.61|1566.16|4121.48|4872.20|8769.96|229.97|1566.16|2555.32|2785.29|-2316.88| +||15565||||22952||88|1781|47||87.80|59.70||2805.90|2105.60||||||| +2451550|37634|3131|87761|473358|1355|22952|4|136|1781|6|45.87|89.44|11.62|0.00|69.72|275.22|536.64|1.39|0.00|69.72|71.11|-205.50| +2451731|51524|15941|14045|1362425|405|5985|10|219|1782|28|20.51|24.40|2.92|0.00|81.76|574.28|683.20|4.08|0.00|81.76|85.84|-492.52| +2451731|51524|6584|14045|1362425|405|5985|10|162|1782|46|28.02|33.34|14.66|0.00|674.36|1288.92|1533.64|33.71|0.00|674.36|708.07|-614.56| +2451731|51524|10627|14045|1362425|405|5985|10|122|1782|70|7.40|8.95|3.04|0.00|212.80|518.00|626.50|19.15|0.00|212.80|231.95|-305.20| +2451731|51524|4088|14045|1362425|405|5985|10|170|1782|7|26.91|42.51|21.68|112.30|151.76|188.37|297.57|0.78|112.30|39.46|40.24|-148.91| +2451731|51524|6062|14045|1362425|405|5985|10|53|1782|66|17.76|32.14|24.10|954.36|1590.60|1172.16|2121.24|31.81|954.36|636.24|668.05|-535.92| +2451731|51524|5623|14045|1362425|405|5985|10|175|1782|54|28.38|41.71|28.36|0.00|1531.44|1532.52|2252.34|91.88|0.00|1531.44|1623.32|-1.08| +2451731|51524|15989|14045|1362425|405|5985|10|278|1782|33|29.92|55.65|53.42|105.77|1762.86|987.36|1836.45|132.56|105.77|1657.09|1789.65|669.73| +2451731|51524|7327|14045|1362425|405|5985|10|96|1782|44|99.06|175.33|77.14|0.00|3394.16|4358.64|7714.52|67.88|0.00|3394.16|3462.04|-964.48| +2451731|51524|8360|14045|1362425|405|5985|10|49|1782|2|70.03|75.63|34.03|0.00|68.06|140.06|151.26|4.08|0.00|68.06|72.14|-72.00| +2451731|51524|11681|14045|1362425|405|5985|10|298|1782|13|68.96|80.68|10.48|0.00|136.24|896.48|1048.84|1.36|0.00|136.24|137.60|-760.24| +2451731|51524|5972|14045|1362425|405|5985|10|9|1782|12|8.63|17.08|1.19|0.00|14.28|103.56|204.96|0.14|0.00|14.28|14.42|-89.28| +2451731|51524|8981|14045|1362425|405|5985|10|256|1782|81|96.96|126.04|35.29|0.00|2858.49|7853.76|10209.24|257.26|0.00|2858.49|3115.75|-4995.27| +2451731|51524|6035|14045|1362425|405|5985|10|4|1782|96|64.15|71.84|58.90|0.00|5654.40|6158.40|6896.64|452.35|0.00|5654.40|6106.75|-504.00| +2451731|51524|9139|14045|1362425|405|5985|10|126|1782|83|7.39|10.71|8.67|0.00|719.61|613.37|888.93|21.58|0.00|719.61|741.19|106.24| +2451038|64153|9746|65200|530993|3633|15826|2|22|1783|76|51.93|78.41|11.76|0.00|893.76|3946.68|5959.16|53.62|0.00|893.76|947.38|-3052.92| +2451038|64153|7873|65200|530993|3633|15826|2|203|1783|47|55.12|91.49|32.02|0.00|1504.94|2590.64|4300.03|120.39|0.00|1504.94|1625.33|-1085.70| +2451038|64153|8084|65200|530993|3633|15826|2|140|1783|57|39.89|47.07|21.18|0.00|1207.26|2273.73|2682.99|72.43|0.00|1207.26|1279.69|-1066.47| +2451038|64153|226|65200|530993|3633|15826|2|217|1783|88|78.59|113.16|62.23|0.00|5476.24|6915.92|9958.08|383.33|0.00|5476.24|5859.57|-1439.68| +2451038|64153|17014|65200|530993|3633|15826|2|237|1783|85|74.28|86.16|23.26|0.00|1977.10|6313.80|7323.60|59.31|0.00|1977.10|2036.41|-4336.70| +2451038|64153|5086|65200|530993|3633|15826|2|111|1783|99|94.11|177.86|174.30|0.00|17255.70|9316.89|17608.14|345.11|0.00|17255.70|17600.81|7938.81| +2451038|64153|17338|65200|530993|3633|15826|2|197|1783|32|70.84|97.75|78.20|0.00|2502.40|2266.88|3128.00|50.04|0.00|2502.40|2552.44|235.52| +2451038|64153|13654|65200|530993|3633|15826|2|104|1783|69|86.40|122.68|7.36|0.00|507.84|5961.60|8464.92|0.00|0.00|507.84|507.84|-5453.76| +2451038|64153|15578|65200|530993|3633|15826|2|204|1783|83|99.79|176.62|61.81|0.00|5130.23|8282.57|14659.46|153.90|0.00|5130.23|5284.13|-3152.34| +2451038|64153|6586|65200|530993|3633|15826|2|254|1783|99|36.33|36.33|18.52|0.00|1833.48|3596.67|3596.67|146.67|0.00|1833.48|1980.15|-1763.19| +2451038|64153|3187|65200|530993|3633|15826|2|115|1783|83|30.78|32.31|15.18|0.00|1259.94|2554.74|2681.73|12.59|0.00|1259.94|1272.53|-1294.80| +2451038|64153|3008|65200|530993|3633|15826|2|171|1783|28|92.18|165.92|119.46|0.00|3344.88|2581.04|4645.76|301.03|0.00|3344.88|3645.91|763.84| +||13195|65200|530993|||||1783|77|31.65||||||||||548.03|| +2451038|64153|11611|65200|530993|3633|15826|2|50|1783|77|63.57|73.10|48.97|0.00|3770.69|4894.89|5628.70|75.41|0.00|3770.69|3846.10|-1124.20| +2451038|64153|8593|65200|530993|3633|15826|2|66|1783|35|87.50|151.37|34.81|0.00|1218.35|3062.50|5297.95|24.36|0.00|1218.35|1242.71|-1844.15| +2451038|64153|1216|65200|530993|3633|15826|2|38|1783|47|76.67|147.20|58.88|0.00|2767.36|3603.49|6918.40|83.02|0.00|2767.36|2850.38|-836.13| +2452223|69215|6525|13114|277880|4693|19906|7|258|1784|13|24.89|24.89|14.68|32.44|190.84|323.57|323.57|6.33|32.44|158.40|164.73|-165.17| +2452223|69215|10025|13114|277880|4693|19906|7|32|1784|76|20.85|39.61|36.44|0.00|2769.44|1584.60|3010.36|110.77|0.00|2769.44|2880.21|1184.84| +2452223|69215|14341|13114|277880|4693|19906|7|219|1784|70|45.09|80.71|66.18|0.00|4632.60|3156.30|5649.70|231.63|0.00|4632.60|4864.23|1476.30| +2452223|69215|8511|13114|277880|4693|19906|7|109|1784|43|59.37|108.05|61.58|0.00|2647.94|2552.91|4646.15|52.95|0.00|2647.94|2700.89|95.03| +2452223|69215|905|13114|277880|4693|19906|7|132|1784|49|2.66|4.97|3.32|42.29|162.68|130.34|243.53|4.81|42.29|120.39|125.20|-9.95| +2452223|69215|17801|13114|277880|4693|19906|7|260|1784|98|47.14|80.13|13.62|0.00|1334.76|4619.72|7852.74|40.04|0.00|1334.76|1374.80|-3284.96| +2452223|69215|3327|13114|277880|4693|19906|7|51|1784|83|87.02|130.53|39.15|0.00|3249.45|7222.66|10833.99|259.95|0.00|3249.45|3509.40|-3973.21| +2452223|69215|14323|13114|277880|4693|19906|7|170|1784|24|57.11|88.52|26.55|0.00|637.20|1370.64|2124.48|25.48|0.00|637.20|662.68|-733.44| +2451800|69137|12224|87613|830773|3886|24246|4|112|1785|26|44.86|86.13|24.11|0.00|626.86|1166.36|2239.38|25.07|0.00|626.86|651.93|-539.50| +2451800|69137|16955|87613|830773|3886|24246|4|130|1785|39|92.51|137.83|85.45|0.00|3332.55|3607.89|5375.37|133.30|0.00|3332.55|3465.85|-275.34| +2451800|69137|13748|87613|830773|3886|24246|4|50|1785|65|43.81|45.12|19.40|0.00|1261.00|2847.65|2932.80|113.49|0.00|1261.00|1374.49|-1586.65| +2451800|69137|6173|87613|830773|3886|24246|4|105|1785|48|50.72|66.44|66.44|0.00|3189.12|2434.56|3189.12|255.12|0.00|3189.12|3444.24|754.56| +2451800|69137|13909|87613|830773|3886|24246|4|189|1785|47|52.82|52.82|10.56|69.48|496.32|2482.54|2482.54|8.53|69.48|426.84|435.37|-2055.70| +|69137|16879|87613||3886|||273|1785||||1.09|0.00|28.34|50.18|97.76|1.98|0.00|28.34||-21.84| +2451800|69137|2641|87613|830773|3886|24246|4|110|1785|87|56.84|104.58|64.83|0.00|5640.21|4945.08|9098.46|112.80|0.00|5640.21|5753.01|695.13| +2451800|69137|17795|87613|830773|3886|24246|4|132|1785|82|78.10|147.60|79.70|0.00|6535.40|6404.20|12103.20|0.00|0.00|6535.40|6535.40|131.20| +2451800|69137|6943|87613|830773|3886|24246|4|124|1785|40|13.48|13.61|4.21|0.00|168.40|539.20|544.40|8.42|0.00|168.40|176.82|-370.80| +2451465|39379|1018|43636|820197|6358|10166|8|7|1786|31|90.91|177.27|42.54|0.00|1318.74|2818.21|5495.37|65.93|0.00|1318.74|1384.67|-1499.47| +2451465|39379|4297|43636|820197|6358|10166|8|282|1786|46|65.17|114.04|60.44|695.06|2780.24|2997.82|5245.84|41.70|695.06|2085.18|2126.88|-912.64| +2451465|39379|14503|43636|820197|6358|10166|8|200|1786|49|43.17|57.84|27.76|0.00|1360.24|2115.33|2834.16|27.20|0.00|1360.24|1387.44|-755.09| +2451465|39379|5116|43636|820197|6358|10166|8|110|1786|3|86.14|163.66|40.91|0.00|122.73|258.42|490.98|4.90|0.00|122.73|127.63|-135.69| +2451465|39379|15634|43636|820197|6358|10166|8|264|1786|8|62.83|90.47|34.37|79.73|274.96|502.64|723.76|17.57|79.73|195.23|212.80|-307.41| +2451465|39379|3848|43636|820197|6358|10166|8|164|1786|82|30.76|43.37|27.75|1297.03|2275.50|2522.32|3556.34|48.92|1297.03|978.47|1027.39|-1543.85| +2451465|39379|7648|43636|820197|6358|10166|8|265|1786|78|52.91|76.19|14.47|372.45|1128.66|4126.98|5942.82|15.12|372.45|756.21|771.33|-3370.77| +2451465|39379|1828|43636|820197|6358|10166|8|183|1786|5|28.90|56.93|27.89|0.00|139.45|144.50|284.65|12.55|0.00|139.45|152.00|-5.05| +2451465|39379|16472|43636|820197|6358|10166|8|4|1786|38|63.33|108.92|88.22|0.00|3352.36|2406.54|4138.96|0.00|0.00|3352.36|3352.36|945.82| +2451465|39379|9742|43636|820197|6358|10166|8|130|1786|5|84.04|152.11|15.21|0.00|76.05|420.20|760.55|4.56|0.00|76.05|80.61|-344.15| +2451465|39379|16948|43636|820197|6358|10166|8|173|1786|3|37.20|59.89|19.16|0.00|57.48|111.60|179.67|1.14|0.00|57.48|58.62|-54.12| +2451465|39379|10802|43636|820197|6358|10166|8|247|1786|3|50.63|54.68|22.41|0.00|67.23|151.89|164.04|0.00|0.00|67.23|67.23|-84.66| +2452355|30510|10494|3985|1395472|4559|37646|2|199|1787|85|42.46|73.03|24.09|0.00|2047.65|3609.10|6207.55|122.85|0.00|2047.65|2170.50|-1561.45| +2452355|30510|6897|3985|1395472|4559|37646|2|81|1787|23|3.78|4.46|2.23|0.00|51.29|86.94|102.58|3.59|0.00|51.29|54.88|-35.65| +||2586|3985||4559|37646||62|1787|6|85.33|||170.80||||0.51|170.80|||| +2452355|30510|1746|3985|1395472|4559|37646|2|187|1787|96|86.13|114.55|101.94|7339.68|9786.24|8268.48|10996.80|48.93|7339.68|2446.56|2495.49|-5821.92| +2452355|30510|3786|3985|1395472|4559|37646|2|138|1787|10|65.14|108.13|15.13|0.00|151.30|651.40|1081.30|3.02|0.00|151.30|154.32|-500.10| +2452355|30510|9013|3985|1395472|4559|37646|2|185|1787|91|75.48|109.44|42.68|0.00|3883.88|6868.68|9959.04|38.83|0.00|3883.88|3922.71|-2984.80| +2452355|30510|1993|3985|1395472|4559|37646|2|15|1787|86|14.52|21.05|12.63|0.00|1086.18|1248.72|1810.30|32.58|0.00|1086.18|1118.76|-162.54| +2452355|30510|8769|3985|1395472|4559|37646|2|72|1787|86|25.92|27.73|15.52|0.00|1334.72|2229.12|2384.78|40.04|0.00|1334.72|1374.76|-894.40| +2452355|30510|2445|3985|1395472|4559|37646|2|71|1787|37|78.10|135.89|28.53|0.00|1055.61|2889.70|5027.93|21.11|0.00|1055.61|1076.72|-1834.09| +2452355|30510|12849|3985|1395472|4559|37646|2|62|1787|21|60.61|88.49|38.05|167.80|799.05|1272.81|1858.29|0.00|167.80|631.25|631.25|-641.56| +2452355|30510|12997|3985|1395472|4559|37646|2|32|1787|9|31.47|44.37|25.73|0.00|231.57|283.23|399.33|6.94|0.00|231.57|238.51|-51.66| +|30510|5577|3985|||37646||287|1787|||||0.00|489.40||||0.00|489.40||-203.80| +2452355|30510|2874|3985|1395472|4559|37646|2|86|1787|67|78.06|152.99|12.23|0.00|819.41|5230.02|10250.33|57.35|0.00|819.41|876.76|-4410.61| +2452355|30510|9765|3985|1395472|4559|37646|2|245|1787|5|33.63|50.10|32.56|0.00|162.80|168.15|250.50|14.65|0.00|162.80|177.45|-5.35| +2452355|30510|14139|3985|1395472|4559|37646|2|85|1787|96|49.35|76.98|4.61|0.00|442.56|4737.60|7390.08|4.42|0.00|442.56|446.98|-4295.04| +2452355|30510|6465|3985|1395472|4559|37646|2|230|1787|62|81.26|101.57|31.48|0.00|1951.76|5038.12|6297.34|136.62|0.00|1951.76|2088.38|-3086.36| +2452248|39943|2635|48070|1150104|138|3772|7|176|1788|81|60.68|103.15|31.97|466.12|2589.57|4915.08|8355.15|169.87|466.12|2123.45|2293.32|-2791.63| +2452248|39943|2427|48070|1150104|138|3772|7|277|1788|71|57.08|68.49|23.28|0.00|1652.88|4052.68|4862.79|148.75|0.00|1652.88|1801.63|-2399.80| +2452248|39943|15147|48070|1150104|138|3772|7|59|1788|56|15.44|28.10|21.07|0.00|1179.92|864.64|1573.60|58.99|0.00|1179.92|1238.91|315.28| +2452248|39943|12321|48070|1150104|138|3772|7|61|1788|8|55.44|77.06|31.59|0.00|252.72|443.52|616.48|7.58|0.00|252.72|260.30|-190.80| +2452248|39943|7831|48070|1150104|138|3772|7|92|1788|16|60.60|115.74|79.86|25.55|1277.76|969.60|1851.84|62.61|25.55|1252.21|1314.82|282.61| +2452248|39943|12989|48070|1150104|138|3772|7|232|1788|95|74.21|82.37|65.89|0.00|6259.55|7049.95|7825.15|563.35|0.00|6259.55|6822.90|-790.40| +2452248|39943|12791|48070|1150104|138|3772|7|81|1788|95|48.91|93.90|49.76|4112.66|4727.20|4646.45|8920.50|49.16|4112.66|614.54|663.70|-4031.91| +2452248|39943|5515|48070|1150104|138|3772|7|40|1788|88|85.03|113.94|108.24|0.00|9525.12|7482.64|10026.72|666.75|0.00|9525.12|10191.87|2042.48| +2452248|39943|7991|48070|1150104|138|3772|7|243|1788|14|81.24|138.10|29.00|0.00|406.00|1137.36|1933.40|16.24|0.00|406.00|422.24|-731.36| +2452115|74791|3847|99109|572314|1519|16724|8|247|1789|77|55.96|62.11|42.23|2601.36|3251.71|4308.92|4782.47|6.50|2601.36|650.35|656.85|-3658.57| +2452115|74791|5737|99109|572314|1519|16724|8|66|1789|82|27.68|30.44|0.60|0.00|49.20|2269.76|2496.08|0.98|0.00|49.20|50.18|-2220.56| +2452115|74791|12865|99109|572314|1519|16724|8|105|1789|18|69.76|136.03|31.28|0.00|563.04|1255.68|2448.54|50.67|0.00|563.04|613.71|-692.64| +2452115|74791|3271|99109|572314|1519|16724|8|121|1789|2|36.43|50.63|47.08|0.00|94.16|72.86|101.26|3.76|0.00|94.16|97.92|21.30| +2452115|74791|3541|99109|572314|1519|16724|8|35|1789|91|42.13|64.03|14.08|0.00|1281.28|3833.83|5826.73|89.68|0.00|1281.28|1370.96|-2552.55| +2452115|74791|17039|99109|572314|1519|16724|8|218|1789|70|78.55|84.83|40.71|0.00|2849.70|5498.50|5938.10|227.97|0.00|2849.70|3077.67|-2648.80| +2452115|74791|11501|99109|572314|1519|16724|8|118|1789|72|46.66|69.05|4.83|0.00|347.76|3359.52|4971.60|0.00|0.00|347.76|347.76|-3011.76| +2452115||6827||||||260|1789||45.68||57.42|0.00|1492.92||1508.26|104.50|0.00|1492.92||| +2452115|74791|8023|99109|572314|1519|16724|8|86|1789|71|2.94|5.35|5.18|0.00|367.78|208.74|379.85|22.06|0.00|367.78|389.84|159.04| +2452115|74791|5429|99109|572314|1519|16724|8|81|1789|25|89.06|133.59|52.10|0.00|1302.50|2226.50|3339.75|78.15|0.00|1302.50|1380.65|-924.00| +2452115|74791|2657|99109|572314|1519|16724|8|127|1789|36|22.69|35.39|29.37|0.00|1057.32|816.84|1274.04|42.29|0.00|1057.32|1099.61|240.48| +2452115|74791|10339|99109|572314|1519|16724|8|262|1789|60|1.19|1.65|0.41|0.00|24.60|71.40|99.00|0.98|0.00|24.60|25.58|-46.80| +2452115|74791|4909|99109|572314|1519|16724|8|253|1789|93|72.01|87.85|22.84|254.89|2124.12|6696.93|8170.05|168.23|254.89|1869.23|2037.46|-4827.70| +2452115|74791|9567|99109|572314|1519|16724|8|83|1789|79|13.17|18.04|7.03|522.04|555.37|1040.43|1425.16|2.99|522.04|33.33|36.32|-1007.10| +2451303|32470|8474|98096|828779|2215|4098|7|257|1790|82|49.58|73.37|16.87|857.67|1383.34|4065.56|6016.34|36.79|857.67|525.67|562.46|-3539.89| +2451303|32470|14092|98096|828779|2215|4098|7|71|1790|50|3.26|4.82|1.59|0.00|79.50|163.00|241.00|4.77|0.00|79.50|84.27|-83.50| +2451303|32470|9739|98096|828779|2215|4098|7|46|1790|43|72.66|74.11|8.89|87.92|382.27|3124.38|3186.73|23.54|87.92|294.35|317.89|-2830.03| +2451303|32470|892|98096|828779|2215|4098|7|30|1790|23|23.51|35.97|30.57|0.00|703.11|540.73|827.31|56.24|0.00|703.11|759.35|162.38| +2451303|32470|17930|98096|828779|2215|4098|7|43|1790|57|62.13|86.98|0.86|0.00|49.02|3541.41|4957.86|3.43|0.00|49.02|52.45|-3492.39| +2451303|32470|8140|98096|828779|2215|4098|7|177|1790|3|46.68|80.75|80.75|0.00|242.25|140.04|242.25|21.80|0.00|242.25|264.05|102.21| +2451303|32470|14932|98096|828779|2215|4098|7|37|1790|25|5.81|8.88|3.10|0.00|77.50|145.25|222.00|0.77|0.00|77.50|78.27|-67.75| +|32470|11386||||4098|7||1790||46.56|88.46|59.26|0.00|2844.48|||170.66|0.00|2844.48||| +2451303|32470|1022|98096|828779|2215|4098|7|207|1790|56|22.93|24.30|8.26|0.00|462.56|1284.08|1360.80|41.63|0.00|462.56|504.19|-821.52| +2451303|32470|9142|98096|828779|2215|4098|7|89|1790|77|9.37|10.21|3.67|0.00|282.59|721.49|786.17|2.82|0.00|282.59|285.41|-438.90| +2451303|32470|1976|98096|828779|2215|4098|7|284|1790|50|42.34|78.32|38.37|0.00|1918.50|2117.00|3916.00|19.18|0.00|1918.50|1937.68|-198.50| +2451896|44717|284|45480|1787160|5287|31814|4|202|1791|12|27.83|36.17|26.40|288.28|316.80|333.96|434.04|1.71|288.28|28.52|30.23|-305.44| +2451896|44717|16946|45480|1787160|5287|31814|4|89|1791|48|31.62|45.21|14.46|0.00|694.08|1517.76|2170.08|13.88|0.00|694.08|707.96|-823.68| +2451896|44717|12155|45480|1787160|5287|31814|4|264|1791|17|85.79|114.95|29.88|0.00|507.96|1458.43|1954.15|0.00|0.00|507.96|507.96|-950.47| +2451896|44717|4033|45480|1787160|5287|31814|4|111|1791|75|30.79|46.80|30.88|0.00|2316.00|2309.25|3510.00|185.28|0.00|2316.00|2501.28|6.75| +2451896|44717|17258|45480|1787160|5287|31814|4|64|1791|92|95.57|158.64|120.56|0.00|11091.52|8792.44|14594.88|554.57|0.00|11091.52|11646.09|2299.08| +2451896|44717|2306|45480|1787160|5287|31814|4|66|1791|34|30.72|39.01|30.81|0.00|1047.54|1044.48|1326.34|62.85|0.00|1047.54|1110.39|3.06| +2451896|44717|10729|45480|1787160|5287|31814|4|146|1791|57|10.54|16.54|4.46|0.00|254.22|600.78|942.78|17.79|0.00|254.22|272.01|-346.56| +2451896|44717|10405|45480|1787160|5287|31814|4|142|1791|9|54.44|68.05|37.42|0.00|336.78|489.96|612.45|23.57|0.00|336.78|360.35|-153.18| +2451896|44717|5743|45480|1787160|5287|31814|4|153|1791|14|21.30|34.08|31.01|0.00|434.14|298.20|477.12|30.38|0.00|434.14|464.52|135.94| +2451896|44717|13898|45480|1787160|5287|31814|4|120|1791|40|82.71|147.22|105.99|0.00|4239.60|3308.40|5888.80|42.39|0.00|4239.60|4281.99|931.20| +2451896|44717|8321|45480|1787160|5287|31814|4|109|1791|57|18.63|34.27|17.13|146.46|976.41|1061.91|1953.39|8.29|146.46|829.95|838.24|-231.96| +2451172|59965|9619|13658|1361909|1713|29964|7|233|1792|89|77.75|125.17|106.39|0.00|9468.71|6919.75|11140.13|852.18|0.00|9468.71|10320.89|2548.96| +2451172|59965|6052|13658|1361909|1713|29964|7|130|1792|95|38.67|56.84|8.52|0.00|809.40|3673.65|5399.80|48.56|0.00|809.40|857.96|-2864.25| +2451172|59965|1934|13658|1361909|1713|29964|7|101|1792|50|19.16|32.76|25.55|0.00|1277.50|958.00|1638.00|25.55|0.00|1277.50|1303.05|319.50| +2451172|59965|8095|13658|1361909|1713|29964|7|152|1792|31|9.63|12.71|7.49|0.00|232.19|298.53|394.01|2.32|0.00|232.19|234.51|-66.34| +2451172|59965|16777|13658|1361909|1713|29964|7|218|1792|34|90.54|171.12|135.18|0.00|4596.12|3078.36|5818.08|0.00|0.00|4596.12|4596.12|1517.76| +2451172|59965|9403|13658|1361909|1713|29964|7|162|1792|25|79.81|120.51|54.22|0.00|1355.50|1995.25|3012.75|81.33|0.00|1355.50|1436.83|-639.75| +2451172|59965|12715|13658|1361909|1713|29964|7|104|1792|82|75.39|111.57|18.96|186.56|1554.72|6181.98|9148.74|109.45|186.56|1368.16|1477.61|-4813.82| +2451172|59965|1232|13658|1361909|1713|29964|7|147|1792|76|72.16|137.82|46.85|0.00|3560.60|5484.16|10474.32|142.42|0.00|3560.60|3703.02|-1923.56| +2451172|59965|415|13658|1361909|1713|29964|7|68|1792|74|85.50|146.20|134.50|0.00|9953.00|6327.00|10818.80|895.77|0.00|9953.00|10848.77|3626.00| +2452027|50658|10553|55699|1805177|3008|43239|10|185|1793|80|30.42|43.80|17.95|229.76|1436.00|2433.60|3504.00|84.43|229.76|1206.24|1290.67|-1227.36| +2452027|50658|2129|55699|1805177|3008|43239|10|250|1793|7|96.87|188.89|170.00|0.00|1190.00|678.09|1322.23|83.30|0.00|1190.00|1273.30|511.91| +2452027|50658|4275|55699|1805177|3008|43239|10|35|1793|63|75.98|117.00|30.42|0.00|1916.46|4786.74|7371.00|172.48|0.00|1916.46|2088.94|-2870.28| +2452027|50658|7889|55699|1805177|3008|43239|10|178|1793|88|67.79|126.76|100.14|0.00|8812.32|5965.52|11154.88|704.98|0.00|8812.32|9517.30|2846.80| +|50658|16725|55699|1805177|3008|43239|10|137|1793||37.56||19.41||||2974.32|111.80||1397.52|1509.32|-1306.80| +2452027|50658|17165|55699|1805177|3008|43239|10|226|1793|15|31.16|61.07|25.64|0.00|384.60|467.40|916.05|3.84|0.00|384.60|388.44|-82.80| +2452027|50658|3339|55699|1805177|3008|43239|10|252|1793|78|56.75|78.31|25.84|0.00|2015.52|4426.50|6108.18|20.15|0.00|2015.52|2035.67|-2410.98| +2452027|50658|2207|55699|1805177|3008|43239|10|239|1793|81|79.85|147.72|90.10|0.00|7298.10|6467.85|11965.32|145.96|0.00|7298.10|7444.06|830.25| +2452599|52397|2389|27806|412325|3691|33457|8|183|1794|82|97.51|148.21|26.67|2011.98|2186.94|7995.82|12153.22|0.00|2011.98|174.96|174.96|-7820.86| +2452599|52397|1477|27806|412325|3691|33457|8|155|1794|55|18.65|35.43|17.36|0.00|954.80|1025.75|1948.65|85.93|0.00|954.80|1040.73|-70.95| +2452599|52397|11433|27806|412325|3691|33457|8|111|1794|15|17.76|27.52|10.45|0.00|156.75|266.40|412.80|10.97|0.00|156.75|167.72|-109.65| +2452599|52397|10567|27806|412325|3691|33457|8|253|1794|24|79.35|139.65|61.44|840.49|1474.56|1904.40|3351.60|44.38|840.49|634.07|678.45|-1270.33| +2452599|52397|6741|27806|412325|3691|33457|8|255|1794|42|49.89|63.36|62.09|0.00|2607.78|2095.38|2661.12|104.31|0.00|2607.78|2712.09|512.40| +2452599|52397|13482|27806|412325|3691|33457|8|160|1794|28|10.20|12.03|0.24|0.00|6.72|285.60|336.84|0.20|0.00|6.72|6.92|-278.88| +|52397|2845|27806||3691||||1794|4|69.06|127.07||0.00|||508.28||0.00|||| +2452599|52397|3402|27806|412325|3691|33457|8|241|1794|23|25.53|43.65|15.71|0.00|361.33|587.19|1003.95|3.61|0.00|361.33|364.94|-225.86| +2452599|52397|7026|27806|412325|3691|33457|8|78|1794|53|96.17|175.99|147.83|7051.49|7834.99|5097.01|9327.47|70.51|7051.49|783.50|854.01|-4313.51| +2452599|52397|9349|27806|412325|3691|33457|8|148|1794|64|57.39|85.51|53.01|0.00|3392.64|3672.96|5472.64|237.48|0.00|3392.64|3630.12|-280.32| +2452599|52397|13657|27806|412325|3691|33457|8|126|1794|90|63.04|102.12|88.84|6316.52|7995.60|5673.60|9190.80|16.79|6316.52|1679.08|1695.87|-3994.52| +2452599|52397|5253|27806|412325|3691|33457|8|14|1794|63|65.43|83.09|35.72|0.00|2250.36|4122.09|5234.67|45.00|0.00|2250.36|2295.36|-1871.73| +2452599|52397|5575|27806|412325|3691|33457|8|111|1794|35|39.77|73.97|57.69|1736.46|2019.15|1391.95|2588.95|14.13|1736.46|282.69|296.82|-1109.26| +2452599|52397|4753|27806|412325|3691|33457|8|63|1794|51|13.63|25.76|19.57|0.00|998.07|695.13|1313.76|19.96|0.00|998.07|1018.03|302.94| +2452599|52397|17814|27806|412325|3691|33457|8|197|1794|34|92.06|150.05|142.54|0.00|4846.36|3130.04|5101.70|387.70|0.00|4846.36|5234.06|1716.32| +2452599|52397|16981|27806|412325|3691|33457|8|245|1794|31|17.31|28.04|17.66|0.00|547.46|536.61|869.24|21.89|0.00|547.46|569.35|10.85| +2451880|61720|1|52546|1166646|1546|32046|7|151|1795|42|50.33|58.88|56.52|878.32|2373.84|2113.86|2472.96|44.86|878.32|1495.52|1540.38|-618.34| +2451880|61720|589|52546|1166646|1546|32046|7|92|1795|45|21.64|37.43|31.44|0.00|1414.80|973.80|1684.35|127.33|0.00|1414.80|1542.13|441.00| +2451880|61720|7459|52546|1166646|1546|32046|7|25|1795|57|59.85|83.79|55.30|0.00|3152.10|3411.45|4776.03|0.00|0.00|3152.10|3152.10|-259.35| +2451880|61720|6421|52546|1166646|1546|32046|7|299|1795|80|82.74|124.93|6.24|0.00|499.20|6619.20|9994.40|0.00|0.00|499.20|499.20|-6120.00| +2451880|61720|17651|52546|1166646|1546|32046|7|186|1795|74|38.64|69.16|44.95|0.00|3326.30|2859.36|5117.84|299.36|0.00|3326.30|3625.66|466.94| +2451880|61720|2813|52546|1166646|1546|32046|7|199|1795|56|93.75|174.37|160.42|0.00|8983.52|5250.00|9764.72|179.67|0.00|8983.52|9163.19|3733.52| +2451880|61720|17378|52546|1166646|1546|32046|7|124|1795|66|1.23|1.89|1.60|0.00|105.60|81.18|124.74|2.11|0.00|105.60|107.71|24.42| +2451880|61720|10087||1166646||32046|7||1795|29|||20.14||584.06|||17.52||584.06||-673.96| +2451880|61720|5888|52546|1166646|1546|32046|7|152|1795|54|96.41|137.86|1.37|0.00|73.98|5206.14|7444.44|0.00|0.00|73.98|73.98|-5132.16| +2451880|61720|16627|52546|1166646|1546|32046|7|125|1795|7|84.18|85.86|70.40|0.00|492.80|589.26|601.02|39.42|0.00|492.80|532.22|-96.46| +2451880|61720|15455|52546|1166646|1546|32046|7|125|1795|76|1.19|1.41|0.07|0.00|5.32|90.44|107.16|0.47|0.00|5.32|5.79|-85.12| +2451880|61720|12206|52546|1166646|1546|32046|7|88|1795|94|98.80|140.29|8.41|0.00|790.54|9287.20|13187.26|71.14|0.00|790.54|861.68|-8496.66| +2451880|61720|16994|52546|1166646|1546|32046|7|40|1795|34|40.14|48.56|5.82|0.00|197.88|1364.76|1651.04|11.87|0.00|197.88|209.75|-1166.88| +2451880|61720|14024|52546|1166646|1546|32046|7|297|1795|67|15.03|27.35|10.66|549.94|714.22|1007.01|1832.45|3.28|549.94|164.28|167.56|-842.73| +2451852|39122|5351|4080|1365665|5297|34482|1|21|1796|17|43.82|43.82|42.06|0.00|715.02|744.94|744.94|57.20|0.00|715.02|772.22|-29.92| +2451852|39122|163|4080|1365665|5297|34482|1|205|1796|57|40.66|75.22|40.61|0.00|2314.77|2317.62|4287.54|185.18|0.00|2314.77|2499.95|-2.85| +2451852|39122|1577|4080|1365665|5297|34482|1|66|1796|19|16.68|18.51|7.58|123.85|144.02|316.92|351.69|1.41|123.85|20.17|21.58|-296.75| +2451852|39122|6989|4080|1365665|5297|34482|1|109|1796|53|1.40|2.10|0.23|0.00|12.19|74.20|111.30|0.12|0.00|12.19|12.31|-62.01| +2451852|39122|11948|4080|1365665|5297|34482|1|183|1796|22|36.68|47.68|44.81|0.00|985.82|806.96|1048.96|59.14|0.00|985.82|1044.96|178.86| +2451852|39122|3679|4080|1365665|5297|34482|1|55|1796|18|9.40|15.22|1.06|0.00|19.08|169.20|273.96|1.14|0.00|19.08|20.22|-150.12| +2451852|39122|3995|4080|1365665|5297|34482|1|212|1796|71|36.32|41.76|30.90|0.00|2193.90|2578.72|2964.96|0.00|0.00|2193.90|2193.90|-384.82| +2451852|39122|9797|4080|1365665|5297|34482|1|8|1796|88|60.63|83.66|49.35|0.00|4342.80|5335.44|7362.08|347.42|0.00|4342.80|4690.22|-992.64| +|39122|8810||||||295|1796|5||39.01||0.00||195.05||3.27|0.00||112.47|-85.85| +2451852|39122|17102|4080|1365665|5297|34482|1|188|1796|61|18.37|27.92|0.27|0.00|16.47|1120.57|1703.12|0.00|0.00|16.47|16.47|-1104.10| +2451852|39122|14681|4080|1365665|5297|34482|1|252|1796|83|98.39|117.08|48.00|0.00|3984.00|8166.37|9717.64|79.68|0.00|3984.00|4063.68|-4182.37| +2452144|62324|10395|11463|804668|7120|3925|4|296|1797|11|15.49|19.36|17.61|0.00|193.71|170.39|212.96|7.74|0.00|193.71|201.45|23.32| +2452144|62324|10075|11463|804668|7120|3925|4|171|1797|77|56.89|92.16|16.58|0.00|1276.66|4380.53|7096.32|0.00|0.00|1276.66|1276.66|-3103.87| +2452144|62324|7083|11463|804668|7120|3925|4|237|1797|67|38.53|57.79|5.20|0.00|348.40|2581.51|3871.93|0.00|0.00|348.40|348.40|-2233.11| +2452144|62324|9541|11463|804668|7120|3925|4|203|1797|20|39.62|52.29|25.09|0.00|501.80|792.40|1045.80|40.14|0.00|501.80|541.94|-290.60| +2452144|62324|10067|11463|804668|7120|3925|4|122|1797|28|25.27|41.44|31.08|0.00|870.24|707.56|1160.32|26.10|0.00|870.24|896.34|162.68| +2452144|62324|353|11463|804668|7120|3925|4|151|1797|63|83.00|106.24|38.24|505.91|2409.12|5229.00|6693.12|19.03|505.91|1903.21|1922.24|-3325.79| +2452144|62324|12657|11463|804668|7120|3925|4|92|1797|13|3.59|4.99|0.29|0.00|3.77|46.67|64.87|0.03|0.00|3.77|3.80|-42.90| +2452144|62324|10767|11463|804668|7120|3925|4|26|1797|41|84.98|157.21|154.06|0.00|6316.46|3484.18|6445.61|505.31|0.00|6316.46|6821.77|2832.28| +2452144|62324|16395|11463|804668|7120|3925|4|260|1797|76|20.57|28.38|3.40|0.00|258.40|1563.32|2156.88|12.92|0.00|258.40|271.32|-1304.92| +2451067|71013|17846|70461|253019|4824|23782|7|18|1798|54|93.67|106.78|35.23|0.00|1902.42|5058.18|5766.12|19.02|0.00|1902.42|1921.44|-3155.76| +2451067|71013|13634|70461|253019|4824|23782|7|174|1798|51|10.20|13.77|13.35|0.00|680.85|520.20|702.27|27.23|0.00|680.85|708.08|160.65| +2451067|71013|379|70461|253019|4824|23782|7|72|1798|8|95.87|122.71|3.68|0.00|29.44|766.96|981.68|1.76|0.00|29.44|31.20|-737.52| +2451067|71013|8228|70461|253019|4824|23782|7|63|1798|80|28.49|37.60|7.52|0.00|601.60|2279.20|3008.00|42.11|0.00|601.60|643.71|-1677.60| +2451067|71013|5296|70461|253019|4824|23782|7|170|1798|18|91.55|101.62|82.31|0.00|1481.58|1647.90|1829.16|14.81|0.00|1481.58|1496.39|-166.32| +|71013|2884|||4824||7||1798|83|21.64|40.68||0.00||1796.12|3376.44||0.00||674.79|-1121.33| +2451067|71013|11671|70461|253019|4824|23782|7|277|1798|60|82.88|84.53|8.45|0.00|507.00|4972.80|5071.80|35.49|0.00|507.00|542.49|-4465.80| +2451067|71013|8380|70461|253019|4824|23782|7|52|1798|48|80.77|93.69|28.10|0.00|1348.80|3876.96|4497.12|53.95|0.00|1348.80|1402.75|-2528.16| +2451067||12214|70461|253019||23782|7||1798|||66.37||1175.30||||78.69|1175.30||4013.39|-910.30| +2451067|71013|13064|70461|253019|4824|23782|7|171|1798|83|90.77|127.98|97.26|2906.12|8072.58|7533.91|10622.34|413.31|2906.12|5166.46|5579.77|-2367.45| +2451067|71013|15308|70461|253019|4824|23782|7|44|1798|87|25.73|28.04|26.35|0.00|2292.45|2238.51|2439.48|183.39|0.00|2292.45|2475.84|53.94| +2451067|71013|14708|70461|253019|4824|23782|7|62|1798|64|6.80|10.26|3.89|0.00|248.96|435.20|656.64|7.46|0.00|248.96|256.42|-186.24| +2451067|71013|13160|70461|253019|4824|23782|7|248|1798|76|26.95|41.50|30.71|0.00|2333.96|2048.20|3154.00|93.35|0.00|2333.96|2427.31|285.76| +2451067|71013|7747|70461|253019|4824|23782|7|62|1798|80|78.04|152.95|64.23|0.00|5138.40|6243.20|12236.00|359.68|0.00|5138.40|5498.08|-1104.80| +2451067|71013|4432|70461|253019|4824|23782|7|298|1798|93|94.76|137.40|13.74|0.00|1277.82|8812.68|12778.20|0.00|0.00|1277.82|1277.82|-7534.86| +2451067|71013|14428|70461|253019|4824|23782|7|194|1798|22|64.17|87.27|35.78|0.00|787.16|1411.74|1919.94|55.10|0.00|787.16|842.26|-624.58| +2451740|66332|5774|83015|767123|3231|14139|8|140|1799|8|40.39|53.71|34.37|8.24|274.96|323.12|429.68|0.00|8.24|266.72|266.72|-56.40| +2451740|66332|2995|83015|767123|3231|14139|8|135|1799|29|17.11|30.11|18.06|0.00|523.74|496.19|873.19|36.66|0.00|523.74|560.40|27.55| +2451740|66332|4633|83015|767123|3231|14139|8|26|1799|27|6.59|7.18|3.37|0.00|90.99|177.93|193.86|6.36|0.00|90.99|97.35|-86.94| +2451740|66332|12842|83015|767123|3231|14139|8|83|1799|54|74.79|103.95|16.63|422.06|898.02|4038.66|5613.30|9.51|422.06|475.96|485.47|-3562.70| +2451740|66332|5719|83015|767123|3231|14139|8|260|1799|21|56.25|76.50|35.19|0.00|738.99|1181.25|1606.50|0.00|0.00|738.99|738.99|-442.26| +2451740|66332|8834|83015|767123|3231|14139|8|263|1799|31|45.24|84.14|35.33|0.00|1095.23|1402.44|2608.34|0.00|0.00|1095.23|1095.23|-307.21| +2451740|66332|14317|83015|767123|3231|14139|8|44|1799|73|65.95|101.56|9.14|0.00|667.22|4814.35|7413.88|6.67|0.00|667.22|673.89|-4147.13| +2451740|66332|3836|83015|767123|3231|14139|8|110|1799|23|46.38|78.84|75.68|174.06|1740.64|1066.74|1813.32|46.99|174.06|1566.58|1613.57|499.84| +2452249|61314|12553|83008|624544|4480|48416|10|28|1800|69|19.73|26.83|3.75|0.00|258.75|1361.37|1851.27|7.76|0.00|258.75|266.51|-1102.62| +|61314|15533||624544|4480|48416|10|59|1800|79|85.88||3.27|0.00|||8615.74|15.49|0.00|258.33|273.82|-6526.19| +2452249|61314|11937|83008|624544|4480|48416|10|110|1800|23|53.37|64.04|16.65|0.00|382.95|1227.51|1472.92|0.00|0.00|382.95|382.95|-844.56| +2452249|61314|12299|83008|624544|4480|48416|10|204|1800|26|94.84|97.68|77.16|0.00|2006.16|2465.84|2539.68|20.06|0.00|2006.16|2026.22|-459.68| +2452249|61314|17547|83008|624544|4480|48416|10|265|1800|8|90.50|178.28|69.52|5.56|556.16|724.00|1426.24|11.01|5.56|550.60|561.61|-173.40| +2452249|61314|3063|83008|624544|4480|48416|10|261|1800|45|34.78|35.12|7.72|66.00|347.40|1565.10|1580.40|16.88|66.00|281.40|298.28|-1283.70| +2452249|61314|14591|83008|624544|4480|48416|10|94|1800|51|34.63|34.63|6.57|0.00|335.07|1766.13|1766.13|20.10|0.00|335.07|355.17|-1431.06| +2452249|61314|13197|83008|624544|4480|48416|10|54|1800|24|97.22|179.85|124.09|0.00|2978.16|2333.28|4316.40|29.78|0.00|2978.16|3007.94|644.88| +2452249|61314|7005|83008|624544|4480|48416|10|76|1800|59|97.02|155.23|99.34|0.00|5861.06|5724.18|9158.57|351.66|0.00|5861.06|6212.72|136.88| +2451851|36776|16496|34215|659175|5735|23048|2|134|1801|22|71.90|73.33|55.73|0.00|1226.06|1581.80|1613.26|85.82|0.00|1226.06|1311.88|-355.74| +2451851|36776|4751|34215|659175|5735|23048|2|184|1801|61|93.55|175.87|75.62|1337.71|4612.82|5706.55|10728.07|0.00|1337.71|3275.11|3275.11|-2431.44| +2451851|36776|17629|34215|659175|5735|23048|2|105|1801|19|9.87|15.49|8.05|48.94|152.95|187.53|294.31|4.16|48.94|104.01|108.17|-83.52| +2451851|36776|8773|34215|659175|5735|23048|2|176|1801|54|73.71|100.98|79.77|0.00|4307.58|3980.34|5452.92|215.37|0.00|4307.58|4522.95|327.24| +2451851|36776|12935|34215|659175|5735|23048|2|194|1801|31|59.85|86.78|0.00|0.00|0.00|1855.35|2690.18|0.00|0.00|0.00|0.00|-1855.35| +2451851|36776|9572|34215|659175|5735|23048|2|180|1801|14|78.30|137.80|41.34|0.00|578.76|1096.20|1929.20|46.30|0.00|578.76|625.06|-517.44| +2451851|36776|6233|34215|659175|5735|23048|2|184|1801|6|34.44|43.39|32.54|0.00|195.24|206.64|260.34|3.90|0.00|195.24|199.14|-11.40| +2451851|36776|6224|34215|659175|5735|23048|2|62|1801|44|90.04|176.47|134.11|0.00|5900.84|3961.76|7764.68|118.01|0.00|5900.84|6018.85|1939.08| +2451851|36776|1199|34215|659175|5735|23048|2|266|1801|48|95.76|122.57|87.02|0.00|4176.96|4596.48|5883.36|292.38|0.00|4176.96|4469.34|-419.52| +2451851|36776|8867|34215|659175|5735|23048|2|204|1801|17|4.01|6.57|5.71|0.00|97.07|68.17|111.69|6.79|0.00|97.07|103.86|28.90| +2452210|49513|4161|29249|1030339|2182|30072|1|61|1802|39|4.10|4.71|3.34|52.10|130.26|159.90|183.69|0.78|52.10|78.16|78.94|-81.74| +2452210|49513|17673|29249|1030339|2182|30072|1|46|1802|70|43.93|71.60|17.90|0.00|1253.00|3075.10|5012.00|112.77|0.00|1253.00|1365.77|-1822.10| +2452210|49513|7581|29249|1030339|2182|30072|1|235|1802|18|76.83|99.87|28.96|0.00|521.28|1382.94|1797.66|31.27|0.00|521.28|552.55|-861.66| +2452210|49513|9991|29249|1030339|2182|30072|1|48|1802|64|46.15|53.99|5.93|0.00|379.52|2953.60|3455.36|22.77|0.00|379.52|402.29|-2574.08| +2452210|49513|4597|29249|1030339|2182|30072|1|125|1802|73|40.56|67.73|22.35|0.00|1631.55|2960.88|4944.29|65.26|0.00|1631.55|1696.81|-1329.33| +2452210|49513|7331|29249|1030339|2182|30072|1|245|1802|59|17.70|29.55|19.79|93.40|1167.61|1044.30|1743.45|32.22|93.40|1074.21|1106.43|29.91| +2452210|49513|15973|29249|1030339|2182|30072|1|277|1802|4|57.96|64.33|33.45|0.00|133.80|231.84|257.32|6.69|0.00|133.80|140.49|-98.04| +2452210|49513|6515|29249|1030339|2182|30072|1|98|1802|97|48.75|72.15|33.18|0.00|3218.46|4728.75|6998.55|193.10|0.00|3218.46|3411.56|-1510.29| +2452210|49513|15617|29249|1030339|2182|30072|1|236|1802|67|29.24|54.67|15.85|0.00|1061.95|1959.08|3662.89|63.71|0.00|1061.95|1125.66|-897.13| +2452210|49513|10659|29249|1030339|2182|30072|1|98|1802|87|30.62|53.58|24.64|0.00|2143.68|2663.94|4661.46|21.43|0.00|2143.68|2165.11|-520.26| +2452210|49513|2605|29249|1030339|2182|30072|1|238|1802|59|47.79|52.09|15.62|0.00|921.58|2819.61|3073.31|27.64|0.00|921.58|949.22|-1898.03| +2451473|61687|6158|29148|1658717|3839|19408|8|271|1803|16|89.25|113.34|61.20|0.00|979.20|1428.00|1813.44|78.33|0.00|979.20|1057.53|-448.80| +2451473|61687|7321|29148|1658717|3839|19408|8|3|1803|13|73.37|122.52|86.98|0.00|1130.74|953.81|1592.76|90.45|0.00|1130.74|1221.19|176.93| +2451473|61687|1711|29148|1658717|3839|19408|8|89|1803|63|35.78|53.31|11.72|0.00|738.36|2254.14|3358.53|29.53|0.00|738.36|767.89|-1515.78| +2451473|61687|3716|29148|1658717|3839|19408|8|234|1803|22|59.26|62.81|29.52|194.83|649.44|1303.72|1381.82|31.82|194.83|454.61|486.43|-849.11| +2451473|61687|10831|29148|1658717|3839|19408|8|10|1803|61|47.05|70.57|55.04|0.00|3357.44|2870.05|4304.77|302.16|0.00|3357.44|3659.60|487.39| +2451473|61687|2762|29148|1658717|3839|19408|8|240|1803|59|32.82|58.74|46.40|0.00|2737.60|1936.38|3465.66|0.00|0.00|2737.60|2737.60|801.22| +2451473|61687|6100|29148|1658717|3839|19408|8|275|1803|60|59.79|80.11|37.65|1129.50|2259.00|3587.40|4806.60|0.00|1129.50|1129.50|1129.50|-2457.90| +2451473|61687|12889|29148|1658717|3839|19408|8|291|1803|66|57.38|105.00|65.10|0.00|4296.60|3787.08|6930.00|42.96|0.00|4296.60|4339.56|509.52| +2451473|61687|5947|29148|1658717|3839|19408|8|258|1803|98|33.64|55.84|29.59|1217.92|2899.82|3296.72|5472.32|117.73|1217.92|1681.90|1799.63|-1614.82| +2451473|61687|11048|29148|1658717|3839|19408|8|156|1803|28|55.93|96.75|68.69|0.00|1923.32|1566.04|2709.00|173.09|0.00|1923.32|2096.41|357.28| +2451473|61687|8126|29148|1658717|3839|19408|8|109|1803|60|47.65|58.60|13.47|323.28|808.20|2859.00|3516.00|33.94|323.28|484.92|518.86|-2374.08| +2451539|68313|2702|85553|1220662|571|8080|1|175|1804|11|88.68|100.20|100.20|0.00|1102.20|975.48|1102.20|77.15|0.00|1102.20|1179.35|126.72| +2451539|68313|4912|85553|1220662|571|8080|1|54|1804|94|60.17|80.62|56.43|0.00|5304.42|5655.98|7578.28|53.04|0.00|5304.42|5357.46|-351.56| +2451539|68313|14162|85553|1220662|571|8080|1|296|1804|96|56.75|99.88|97.88|0.00|9396.48|5448.00|9588.48|563.78|0.00|9396.48|9960.26|3948.48| +2451539|68313|4648|85553|1220662|571|8080|1|57|1804|3|52.16|69.37|43.00|0.00|129.00|156.48|208.11|1.29|0.00|129.00|130.29|-27.48| +2451539|68313|13730|85553|1220662|571|8080|1|155|1804|68|6.32|10.42|2.91|0.00|197.88|429.76|708.56|13.85|0.00|197.88|211.73|-231.88| +2451539|68313|11780|85553|1220662|571|8080|1|264|1804|83|39.05|60.13|22.84|0.00|1895.72|3241.15|4990.79|94.78|0.00|1895.72|1990.50|-1345.43| +2451539|68313|11323|85553|1220662|571|8080|1|214|1804|13|15.34|23.01|9.89|0.00|128.57|199.42|299.13|7.71|0.00|128.57|136.28|-70.85| +2451539|68313|15734|85553|1220662|571|8080|1|80|1804|82|61.60|119.50|101.57|0.00|8328.74|5051.20|9799.00|249.86|0.00|8328.74|8578.60|3277.54| +2451179|47398|6428|23756|1833335|2114|30846|4|108|1805|36|98.79|109.65|57.01|0.00|2052.36|3556.44|3947.40|123.14|0.00|2052.36|2175.50|-1504.08| +2451179|47398|952|23756|1833335|2114|30846|4|111|1805|9|58.07|95.81|66.10|0.00|594.90|522.63|862.29|53.54|0.00|594.90|648.44|72.27| +2451179|47398|10897|23756|1833335|2114|30846|4|270|1805|40|43.03|57.22|29.18|0.00|1167.20|1721.20|2288.80|0.00|0.00|1167.20|1167.20|-554.00| +2451179|47398|8594|23756|1833335|2114|30846|4|132|1805|61|16.09|23.49|19.73|96.28|1203.53|981.49|1432.89|44.29|96.28|1107.25|1151.54|125.76| +2451179|47398|5096|23756|1833335|2114|30846|4|195|1805|11|6.46|10.20|8.05|0.00|88.55|71.06|112.20|5.31|0.00|88.55|93.86|17.49| +2451179|47398|12592|23756|1833335|2114|30846|4|199|1805|17|96.45|186.14|27.92|0.00|474.64|1639.65|3164.38|4.74|0.00|474.64|479.38|-1165.01| +2451179|47398|2600|23756|1833335|2114|30846|4|27|1805|79|94.58|114.44|86.97|4534.61|6870.63|7471.82|9040.76|210.24|4534.61|2336.02|2546.26|-5135.80| +2451179|47398|6950|23756|1833335|2114|30846|4|245|1805|77|2.99|4.60|2.76|61.63|212.52|230.23|354.20|6.03|61.63|150.89|156.92|-79.34| +2451179|47398|16876|23756|1833335|2114|30846|4|16|1805|89|69.46|95.85|75.72|0.00|6739.08|6181.94|8530.65|539.12|0.00|6739.08|7278.20|557.14| +2451179|47398|12073|23756|1833335|2114|30846|4|146|1805|95|56.63|98.53|79.80|0.00|7581.00|5379.85|9360.35|75.81|0.00|7581.00|7656.81|2201.15| +2451179|47398|1321|23756|1833335|2114|30846|4|152|1805|38|30.36|42.50|30.60|0.00|1162.80|1153.68|1615.00|23.25|0.00|1162.80|1186.05|9.12| +2451179|47398|15890|23756|1833335|2114|30846|4|195|1805|89|69.16|82.30|69.13|0.00|6152.57|6155.24|7324.70|307.62|0.00|6152.57|6460.19|-2.67| +2451179||7933||1833335||||197|1805|36|50.34|||0.00|1396.44|||111.71|0.00|1396.44|1508.15|| +2451179|47398|2870|23756|1833335|2114|30846|4|184|1805|38|95.20|147.56|100.34|0.00|3812.92|3617.60|5607.28|0.00|0.00|3812.92|3812.92|195.32| +2451179|47398|12266|23756|1833335|2114|30846|4|26|1805|94|81.69|154.39|63.29|951.88|5949.26|7678.86|14512.66|199.89|951.88|4997.38|5197.27|-2681.48| +2450912|35445|613|69841|1681104|3120|37983|10|69|1806|3|44.81|86.03|73.12|0.00|219.36|134.43|258.09|2.19|0.00|219.36|221.55|84.93| +2450912|35445|8338|69841|1681104|3120|37983|10|34|1806|75|2.09|2.52|1.00|0.00|75.00|156.75|189.00|2.25|0.00|75.00|77.25|-81.75| +2450912|35445|11522|69841|1681104|3120|37983|10|109|1806|11|93.73|167.77|72.14|0.00|793.54|1031.03|1845.47|71.41|0.00|793.54|864.95|-237.49| +2450912|35445|14899|69841|1681104|3120|37983|10|195|1806|19|23.17|37.07|3.70|37.96|70.30|440.23|704.33|0.00|37.96|32.34|32.34|-407.89| +2450912||12484|||||||1806|||61.32||0.00|292.03|1815.78||17.52|0.00|||| +2450912|35445|5461|69841|1681104|3120|37983|10|14|1806|95|17.40|28.01|21.28|0.00|2021.60|1653.00|2660.95|60.64|0.00|2021.60|2082.24|368.60| +2450912|35445|734|69841|1681104|3120|37983|10|13|1806|56|32.76|59.62|22.05|0.00|1234.80|1834.56|3338.72|12.34|0.00|1234.80|1247.14|-599.76| +2450912|35445|7216|69841|1681104|3120|37983|10|120|1806|91|94.91|141.41|38.18|0.00|3474.38|8636.81|12868.31|104.23|0.00|3474.38|3578.61|-5162.43| +2450912|35445|11852|69841||3120||10|298|1806||43.54|44.41|26.20|0.00|2148.40||||0.00|||| +2450912|35445|13966|69841|1681104|3120|37983|10|227|1806|32|35.69|52.82|29.57|0.00|946.24|1142.08|1690.24|18.92|0.00|946.24|965.16|-195.84| +|53666|12679|86448|1028716|||7|31|1807|31||4.06|3.57||110.67|||3.32||||| +2451121|53666|11173|86448|1028716|1384|49252|7|273|1807|67|48.51|57.72|15.00|0.00|1005.00|3250.17|3867.24|80.40|0.00|1005.00|1085.40|-2245.17| +2451121|53666|17401|86448|1028716|||7|28|1807||||45.25|0.00|543.00|617.16|678.84||0.00||570.15|-74.16| +2451121|53666|932|86448|1028716|1384|49252|7|65|1807|82|23.09|27.01|18.36|0.00|1505.52|1893.38|2214.82|120.44|0.00|1505.52|1625.96|-387.86| +2451121|53666|17914|86448|1028716|1384|49252|7|252|1807|83|83.78|104.72|85.87|0.00|7127.21|6953.74|8691.76|0.00|0.00|7127.21|7127.21|173.47| +2451121|53666|14870|86448|1028716|1384|49252|7|243|1807|36|46.26|75.86|10.62|0.00|382.32|1665.36|2730.96|7.64|0.00|382.32|389.96|-1283.04| +2451121|53666|2011|86448|1028716|1384|49252|7|257|1807|28|89.02|144.21|60.56|525.66|1695.68|2492.56|4037.88|93.60|525.66|1170.02|1263.62|-1322.54| +2451121|53666|9376|86448|1028716|1384|49252|7|68|1807|27|33.38|46.39|31.54|136.25|851.58|901.26|1252.53|14.30|136.25|715.33|729.63|-185.93| +2452502|60923|6444|84841|1684316|4944|14019|7|12|1808|97|24.97|40.20|9.64|0.00|935.08|2422.09|3899.40|37.40|0.00|935.08|972.48|-1487.01| +2452502|60923|16434|84841|1684316|4944|14019|7|71|1808|62|87.33|146.71|110.03|4638.86|6821.86|5414.46|9096.02|65.49|4638.86|2183.00|2248.49|-3231.46| +2452502|60923|12798|84841|1684316|4944|14019|7|207|1808|13|9.26|12.50|4.50|19.89|58.50|120.38|162.50|1.54|19.89|38.61|40.15|-81.77| +2452502|60923|5617|84841|1684316|4944|14019|7|95|1808|71|89.54|96.70|62.85|0.00|4462.35|6357.34|6865.70|89.24|0.00|4462.35|4551.59|-1894.99| +2452502|60923|15247|84841|1684316|4944|14019|7|209|1808|28|76.01|82.09|39.40|0.00|1103.20|2128.28|2298.52|33.09|0.00|1103.20|1136.29|-1025.08| +2452502|60923|13137|84841|1684316|4944|14019|7|99|1808|97|5.34|7.95|0.15|3.34|14.55|517.98|771.15|0.33|3.34|11.21|11.54|-506.77| +2452502||10867|84841||||||1808|27|32.40|47.95|34.04|404.39||874.80|1294.65||404.39|514.69|514.69|| +2452502|60923|17569|84841|1684316|4944|14019|7|34|1808|41|60.78|95.42|38.16|0.00|1564.56|2491.98|3912.22|93.87|0.00|1564.56|1658.43|-927.42| +2452502|60923|7263|84841|1684316|4944|14019|7|214|1808|28|73.05|107.38|27.91|0.00|781.48|2045.40|3006.64|54.70|0.00|781.48|836.18|-1263.92| +2451421|40597|12511|65839|26086|733||8||1809|38|18.00|||0.00||684.00||36.01|0.00|514.52|550.53|| +2451421|40597|13978|65839|26086|733|46580|8|263|1809|70|37.41|37.41|6.73|179.01|471.10|2618.70|2618.70|14.60|179.01|292.09|306.69|-2326.61| +2451421|40597|10798|65839|26086|733|46580|8|56|1809|35|38.85|42.73|40.16|0.00|1405.60|1359.75|1495.55|112.44|0.00|1405.60|1518.04|45.85| +2451421|40597|202|65839|26086|733|46580|8|135|1809|7|12.53|19.17|4.02|0.00|28.14|87.71|134.19|2.25|0.00|28.14|30.39|-59.57| +2451421|40597|8581|65839|26086|733|46580|8|254|1809|53|37.17|43.86|14.47|406.46|766.91|1970.01|2324.58|0.00|406.46|360.45|360.45|-1609.56| +2451421|40597|14308|65839|26086|733|46580|8|61|1809|77|62.94|108.88|82.74|3886.29|6370.98|4846.38|8383.76|0.00|3886.29|2484.69|2484.69|-2361.69| +2451421|40597|8258|65839|26086|733|46580|8|165|1809|25|77.04|120.18|28.84|0.00|721.00|1926.00|3004.50|21.63|0.00|721.00|742.63|-1205.00| +2451421|40597|514|65839|26086|733|46580|8|291|1809|81|58.57|78.48|62.78|1779.81|5085.18|4744.17|6356.88|33.05|1779.81|3305.37|3338.42|-1438.80| +2451421|40597|14914|65839|26086|733|46580|8|248|1809|14|45.42|71.76|13.63|0.00|190.82|635.88|1004.64|0.00|0.00|190.82|190.82|-445.06| +2451421|40597|13765|65839|26086|733|46580|8|102|1809|67|75.62|105.86|31.75|0.00|2127.25|5066.54|7092.62|106.36|0.00|2127.25|2233.61|-2939.29| +2451421||11836||||46580|8|109|1809|81||125.76|51.56|0.00|4176.36|||0.00|0.00|4176.36||| +2451421|40597|12478|65839|26086|733|46580|8|78|1809|65|46.54|75.39|23.37|0.00|1519.05|3025.10|4900.35|15.19|0.00|1519.05|1534.24|-1506.05| +2451421|40597|14539|65839|26086|733|46580|8|45|1809|11|26.81|49.59|24.29|0.00|267.19|294.91|545.49|8.01|0.00|267.19|275.20|-27.72| +2451421|40597|8059|65839|26086|733|46580|8|300|1809|74|87.60|127.02|123.20|1641.02|9116.80|6482.40|9399.48|598.06|1641.02|7475.78|8073.84|993.38| +2451421|40597|10273|65839|26086|733|46580|8|201|1809|57|6.89|7.99|5.03|131.88|286.71|392.73|455.43|10.83|131.88|154.83|165.66|-237.90| +2452364|31119|16587|18611|1539744|430|38363|8|63|1810|18|10.50|13.02|0.78|1.96|14.04|189.00|234.36|0.96|1.96|12.08|13.04|-176.92| +2452364|31119|480|18611|1539744|430|38363|8|231|1810|68|29.37|57.27|9.16|591.73|622.88|1997.16|3894.36|2.49|591.73|31.15|33.64|-1966.01| +2452364|31119|6187|18611|1539744|430|38363|8|249|1810|8|82.24|83.06|68.93|242.63|551.44|657.92|664.48|12.35|242.63|308.81|321.16|-349.11| +2452364|31119|14058|18611|1539744|430|38363|8|254|1810|4|63.59|76.94|43.08|0.00|172.32|254.36|307.76|1.72|0.00|172.32|174.04|-82.04| +2452364|31119|10009|18611|1539744|430|38363|8|210|1810|6|96.96|182.28|47.39|0.00|284.34|581.76|1093.68|14.21|0.00|284.34|298.55|-297.42| +2452364|31119|15213|18611|1539744|430|38363|8|102|1810|12|16.86|24.61|19.44|67.65|233.28|202.32|295.32|0.00|67.65|165.63|165.63|-36.69| +2452364|31119|13464|18611|1539744|430|38363|8|19|1810|42|83.80|110.61|80.74|0.00|3391.08|3519.60|4645.62|0.00|0.00|3391.08|3391.08|-128.52| +2452364|31119|5815|18611|1539744|430|38363|8|231|1810|69|36.06|40.74|8.55|0.00|589.95|2488.14|2811.06|41.29|0.00|589.95|631.24|-1898.19| +2452364|31119|11535|18611|1539744|430|38363|8|178|1810|37|57.24|106.46|34.06|768.73|1260.22|2117.88|3939.02|39.31|768.73|491.49|530.80|-1626.39| +2452364|31119|15975|18611|1539744|430|38363|8|86|1810|94|28.56|35.98|34.54|0.00|3246.76|2684.64|3382.12|292.20|0.00|3246.76|3538.96|562.12| +2452364|31119|4405|18611|1539744|430|38363|8|141|1810|75|15.93|17.68|12.37|0.00|927.75|1194.75|1326.00|83.49|0.00|927.75|1011.24|-267.00| +2452364|31119|13963|18611|1539744|430|38363|8|232|1810|39|52.97|92.69|36.14|0.00|1409.46|2065.83|3614.91|84.56|0.00|1409.46|1494.02|-656.37| +2452364||9939|||430||||1810|||77.84|53.70|0.00||3351.72|4826.08||0.00|||| +2452364|31119|9891|18611|1539744|430|38363|8|289|1810|94|53.46|53.99|5.39|0.00|506.66|5025.24|5075.06|45.59|0.00|506.66|552.25|-4518.58| +||17676|||430|||138|1810||25.30||8.59|0.00||||64.16|0.00|712.97|777.13|-1386.93| +2452364|31119|16915|18611|1539744|430|38363|8|193|1810|91|68.71|69.39|6.24|68.14|567.84|6252.61|6314.49|24.98|68.14|499.70|524.68|-5752.91| +2452332|43347|1291|57261|1211032|3556|13680|2|47|1811|36|59.65|99.61|41.83|0.00|1505.88|2147.40|3585.96|45.17|0.00|1505.88|1551.05|-641.52| +2452332|43347|15087|57261|1211032|3556|13680|2|163|1811|24|45.18|70.02|4.20|0.00|100.80|1084.32|1680.48|4.03|0.00|100.80|104.83|-983.52| +2452332|43347|7677|57261|1211032|3556|13680|2|280|1811|84|12.19|22.18|8.87|707.82|745.08|1023.96|1863.12|0.74|707.82|37.26|38.00|-986.70| +2452332|43347|13374|57261|1211032|3556|13680|2|253|1811|98|89.76|175.03|71.76|0.00|7032.48|8796.48|17152.94|421.94|0.00|7032.48|7454.42|-1764.00| +2452332|43347|5220|57261|1211032|3556|13680|2|266|1811|87|68.82|92.21|50.71|0.00|4411.77|5987.34|8022.27|88.23|0.00|4411.77|4500.00|-1575.57| +2452332|43347|8713|57261|1211032|3556|13680|2|45|1811|11|83.32|152.47|100.63|0.00|1106.93|916.52|1677.17|11.06|0.00|1106.93|1117.99|190.41| +2452332|43347|7323|57261|1211032|3556|13680|2|124|1811|65|52.00|100.88|57.50|0.00|3737.50|3380.00|6557.20|261.62|0.00|3737.50|3999.12|357.50| +2452332|43347|7711|57261|1211032|3556|13680|2|254|1811|78|89.63|157.74|69.40|4222.29|5413.20|6991.14|12303.72|95.27|4222.29|1190.91|1286.18|-5800.23| +2452332|43347|14508|57261|1211032|3556|13680|2|214|1811|51|68.98|111.74|26.81|642.63|1367.31|3517.98|5698.74|36.23|642.63|724.68|760.91|-2793.30| +2452332|43347|7305|57261|1211032|3556|13680|2|214|1811|72|11.41|14.37|9.19|423.47|661.68|821.52|1034.64|2.38|423.47|238.21|240.59|-583.31| +2452332|43347|11245|57261|1211032|3556|13680|2|290|1811|60|26.87|43.26|36.33|0.00|2179.80|1612.20|2595.60|0.00|0.00|2179.80|2179.80|567.60| +2451882|46167|13898|54627|492339|2851|22963|7|31|1812|49|32.55|32.87|23.00|0.00|1127.00|1594.95|1610.63|101.43|0.00|1127.00|1228.43|-467.95| +2451882|46167|8321|54627|492339|2851|22963|7|274|1812|60|69.43|89.56|19.70|0.00|1182.00|4165.80|5373.60|94.56|0.00|1182.00|1276.56|-2983.80| +2451882|46167|434|54627|492339|2851|22963|7|13|1812|9|92.62|105.58|20.06|0.00|180.54|833.58|950.22|9.02|0.00|180.54|189.56|-653.04| +2451882|46167|14941|54627|492339|2851|22963|7|209|1812|74|15.74|20.14|3.42|0.00|253.08|1164.76|1490.36|15.18|0.00|253.08|268.26|-911.68| +2451882|46167|11636|54627|492339|2851|22963|7|6|1812|93|70.29|117.38|116.20|0.00|10806.60|6536.97|10916.34|756.46|0.00|10806.60|11563.06|4269.63| +2451882|46167|2501|54627|492339|2851|22963|7|27|1812|33|70.14|129.75|48.00|0.00|1584.00|2314.62|4281.75|31.68|0.00|1584.00|1615.68|-730.62| +2451882|46167|14485|54627|492339|2851|22963|7|99|1812|91|70.08|106.52|54.32|0.00|4943.12|6377.28|9693.32|296.58|0.00|4943.12|5239.70|-1434.16| +2451882|46167|5899|54627|492339|2851|22963|7|236|1812|86|80.22|82.62|17.35|0.00|1492.10|6898.92|7105.32|89.52|0.00|1492.10|1581.62|-5406.82| +2451882|46167|8989|54627|492339|2851|22963|7|3|1812|90|34.16|59.43|35.65|0.00|3208.50|3074.40|5348.70|288.76|0.00|3208.50|3497.26|134.10| +2452271|50691|17103|48469|694481|3693|49658|7|206|1813|10|33.90|38.30|3.83|0.00|38.30|339.00|383.00|3.44|0.00|38.30|41.74|-300.70| +2452271|50691|14681|48469|694481|3693|49658|7|158|1813|61|50.97|80.02|13.60|0.00|829.60|3109.17|4881.22|8.29|0.00|829.60|837.89|-2279.57| +2452271|50691|8793|48469|694481|3693|49658|7|220|1813|97|28.10|29.78|24.12|0.00|2339.64|2725.70|2888.66|93.58|0.00|2339.64|2433.22|-386.06| +2452271|50691|12729|48469|694481|3693|49658|7|221|1813|59|19.17|34.88|28.25|0.00|1666.75|1131.03|2057.92|133.34|0.00|1666.75|1800.09|535.72| +2452271|50691|6855|48469|694481|3693|49658|7|255|1813|45|53.80|63.48|26.02|0.00|1170.90|2421.00|2856.60|58.54|0.00|1170.90|1229.44|-1250.10| +2452271|50691|15577|48469|694481|3693|49658|7|48|1813|81|84.61|122.68|9.81|508.55|794.61|6853.41|9937.08|2.86|508.55|286.06|288.92|-6567.35| +2452271|50691|16281|48469|694481|3693|49658|7|108|1813|92|73.13|108.23|85.50|6843.42|7866.00|6727.96|9957.16|51.12|6843.42|1022.58|1073.70|-5705.38| +2452271|50691|5731|48469|694481|3693|49658|7|268|1813|31|83.15|148.83|114.59|0.00|3552.29|2577.65|4613.73|0.00|0.00|3552.29|3552.29|974.64| +2452271|50691|16643|48469|694481|3693|49658|7|269|1813|37|12.38|23.52|11.76|0.00|435.12|458.06|870.24|21.75|0.00|435.12|456.87|-22.94| +2452271|50691|3501|48469|694481|3693|49658|7|284|1813|11|59.58|67.32|56.54|0.00|621.94|655.38|740.52|55.97|0.00|621.94|677.91|-33.44| +2451126|33579|2743|60700|445707|3961|6968|4|116|1814|41|75.17|115.76|60.19|0.00|2467.79|3081.97|4746.16|172.74|0.00|2467.79|2640.53|-614.18| +2451126|33579|1495|60700|445707|3961|6968|4|16|1814|20|89.28|147.31|70.70|0.00|1414.00|1785.60|2946.20|42.42|0.00|1414.00|1456.42|-371.60| +2451126|33579|14048|60700|445707|3961|6968|4|66|1814|19|27.87|44.03|4.40|0.00|83.60|529.53|836.57|6.68|0.00|83.60|90.28|-445.93| +2451126|33579|9832|60700|445707|3961|6968|4|74|1814|96|48.75|72.15|36.07|1211.95|3462.72|4680.00|6926.40|45.01|1211.95|2250.77|2295.78|-2429.23| +2451126|33579|15044|60700|445707|3961|6968|4|137|1814|69|76.99|93.92|9.39|0.00|647.91|5312.31|6480.48|25.91|0.00|647.91|673.82|-4664.40| +2451126|33579|8287|60700|445707|3961|6968|4|172|1814|97|75.11|131.44|81.49|869.49|7904.53|7285.67|12749.68|70.35|869.49|7035.04|7105.39|-250.63| +2451126|33579|11510|60700|445707|3961|6968|4|116|1814|80|14.45|28.61|24.60|0.00|1968.00|1156.00|2288.80|0.00|0.00|1968.00|1968.00|812.00| +2451126|33579|5389|60700|445707|3961|6968|4|227|1814|67|81.95|135.21|60.84|0.00|4076.28|5490.65|9059.07|285.33|0.00|4076.28|4361.61|-1414.37| +2451126|33579|5875|60700|445707|3961|6968|4|290|1814|52|22.37|33.55|8.72|208.58|453.44|1163.24|1744.60|4.89|208.58|244.86|249.75|-918.38| +2451126|33579|4568|60700|445707|3961|6968|4|83|1814|77|58.17|87.25|41.00|0.00|3157.00|4479.09|6718.25|31.57|0.00|3157.00|3188.57|-1322.09| +2451126|33579|2257|60700|445707|3961|6968|4|116|1814|16|54.72|59.64|21.47|0.00|343.52|875.52|954.24|27.48|0.00|343.52|371.00|-532.00| +2451126||10664|60700|||6968|||1814||10.59|11.86|2.37||||901.36|7.20||||-624.72| +2451737|30742|13915|6392|1267563|2781|33599|4|195|1815|97|75.67|90.04|53.12|0.00|5152.64|7339.99|8733.88|463.73|0.00|5152.64|5616.37|-2187.35| +2451737|30742|14144|6392|1267563|2781|33599|4|191|1815|99|69.93|132.86|114.25|0.00|11310.75|6923.07|13153.14|565.53|0.00|11310.75|11876.28|4387.68| +2451737|30742|8405|6392|1267563|2781|33599|4|6|1815|77|85.29|85.29|4.26|121.36|328.02|6567.33|6567.33|10.33|121.36|206.66|216.99|-6360.67| +2451737|30742|5498|6392|1267563|2781|33599|4|232|1815|89|72.27|111.29|40.06|0.00|3565.34|6432.03|9904.81|320.88|0.00|3565.34|3886.22|-2866.69| +2451737|30742|4124|6392|1267563|2781|33599|4|52|1815|90|31.43|35.20|4.57|0.00|411.30|2828.70|3168.00|20.56|0.00|411.30|431.86|-2417.40| +2451737|30742|16910|6392|1267563|2781|33599|4|167|1815|41|31.47|53.49|23.53|0.00|964.73|1290.27|2193.09|0.00|0.00|964.73|964.73|-325.54| +2451737|30742|11773|6392|1267563|2781|33599|4|163|1815|15|5.19|9.44|3.87|0.00|58.05|77.85|141.60|4.64|0.00|58.05|62.69|-19.80| +2451737|30742|17159|6392|1267563|2781|33599|4|21|1815|74|57.49|58.06|52.83|0.00|3909.42|4254.26|4296.44|78.18|0.00|3909.42|3987.60|-344.84| +2451737|30742|9203|6392|1267563|2781|33599|4|105|1815|58|45.44|50.43|0.50|0.00|29.00|2635.52|2924.94|2.03|0.00|29.00|31.03|-2606.52| +2451737|30742|16304|6392|1267563|2781|33599|4|277|1815|1|87.67|145.53|112.05|0.00|112.05|87.67|145.53|6.72|0.00|112.05|118.77|24.38| +2451737|30742|7763|6392|1267563|2781|33599|4|184|1815|46|61.43|105.04|13.65|0.00|627.90|2825.78|4831.84|43.95|0.00|627.90|671.85|-2197.88| +2451737|30742|17437|6392|1267563|2781|33599|4|183|1815|40|8.45|15.71|0.78|0.00|31.20|338.00|628.40|1.87|0.00|31.20|33.07|-306.80| +||3319|6392||||4||1815|87|23.71|32.00||0.00|2171.52|2062.77|2784.00|86.86|0.00|2171.52|2258.38|108.75| +2451737|30742|3044|6392|1267563|2781|33599|4|244|1815|54|28.18|51.00|14.79|0.00|798.66|1521.72|2754.00|71.87|0.00|798.66|870.53|-723.06| +2451737|30742|14543|6392|1267563|2781|33599|4|249|1815|18|74.32|126.34|45.48|0.00|818.64|1337.76|2274.12|57.30|0.00|818.64|875.94|-519.12| +2450951|45020|11479||||34069|8||1816||6.19|9.90|1.58|0.00|127.98|501.39||2.55|0.00|||-373.41| +2450951|45020|14311|67669|1186069|469|34069|8|1|1816|34|25.76|43.27|7.78|0.00|264.52|875.84|1471.18|10.58|0.00|264.52|275.10|-611.32| +2450951|45020|8936|67669|1186069|469|34069|8|133|1816|88|75.78|109.12|13.09|0.00|1151.92|6668.64|9602.56|23.03|0.00|1151.92|1174.95|-5516.72| +2450951|45020|12868|67669|1186069|469|34069|8|288|1816|6|75.03|93.03|40.93|0.00|245.58|450.18|558.18|9.82|0.00|245.58|255.40|-204.60| +2450951|45020|17852|67669|1186069|469|34069|8|169|1816|85|45.07|58.59|0.58|0.00|49.30|3830.95|4980.15|2.46|0.00|49.30|51.76|-3781.65| +2450951|45020|1975|67669|1186069|469|34069|8|199|1816|40|18.60|19.90|3.38|0.00|135.20|744.00|796.00|4.05|0.00|135.20|139.25|-608.80| +2450951|45020|4000|67669|1186069|469|34069|8|113|1816|22|85.68|115.66|67.08|0.00|1475.76|1884.96|2544.52|0.00|0.00|1475.76|1475.76|-409.20| +2450951|45020|4363|67669|1186069|469|34069|8|119|1816|9|24.45|31.54|21.13|0.00|190.17|220.05|283.86|11.41|0.00|190.17|201.58|-29.88| +2450951|45020|3631|67669|1186069|469|34069|8|205|1816|22|92.82|104.88|66.07|0.00|1453.54|2042.04|2307.36|58.14|0.00|1453.54|1511.68|-588.50| +2450951|45020|4468|67669|1186069|469|34069|8|38|1816|30|71.93|88.47|22.11|0.00|663.30|2157.90|2654.10|39.79|0.00|663.30|703.09|-1494.60| +2450951|45020|3134|67669|1186069|469|34069|8|85|1816|36|69.61|96.06|2.88|0.00|103.68|2505.96|3458.16|1.03|0.00|103.68|104.71|-2402.28| +2450951|45020|9148|67669|1186069|469|34069|8|271|1816|47|76.05|86.69|4.33|0.00|203.51|3574.35|4074.43|18.31|0.00|203.51|221.82|-3370.84| +2450951|45020|644|67669|1186069|469|34069|8|235|1816|50|62.48|97.46|22.41|0.00|1120.50|3124.00|4873.00|89.64|0.00|1120.50|1210.14|-2003.50| +2452152|65237|1927|31265|1768877|6136|10548|1|176|1817|49|98.41|123.99|3.71|0.00|181.79|4822.09|6075.51|16.36|0.00|181.79|198.15|-4640.30| +2452152|65237|14541|31265|1768877|6136|10548|1|25|1817|23|16.22|23.68|22.73|460.05|522.79|373.06|544.64|1.25|460.05|62.74|63.99|-310.32| +2452152|65237|3951|31265|1768877|6136|10548|1|76|1817|47|88.44|168.03|102.49|3612.77|4817.03|4156.68|7897.41|84.29|3612.77|1204.26|1288.55|-2952.42| +2452152|65237|3647|31265|1768877|6136|10548|1|177|1817|91|70.45|91.58|6.41|0.00|583.31|6410.95|8333.78|0.00|0.00|583.31|583.31|-5827.64| +2452152|65237|14971|31265|1768877|6136|10548|1|285|1817|50|34.72|38.88|26.82|0.00|1341.00|1736.00|1944.00|13.41|0.00|1341.00|1354.41|-395.00| +2452152|65237|11973|31265|1768877|6136|10548||17|1817|||||6269.87||8246.62||176.34|6269.87|||| +2452152|65237|5635|31265|1768877|6136|10548|1|176|1817|60|6.37|7.45|7.22|368.22|433.20|382.20|447.00|0.00|368.22|64.98|64.98|-317.22| +2452152|65237|5003|31265|1768877|6136|10548|1|250|1817|21|23.02|24.86|1.98|2.07|41.58|483.42|522.06|0.39|2.07|39.51|39.90|-443.91| +2452568|53094|12735|90176|923667|4358|38602|8|201|1818|88|11.53|12.33|3.57|0.00|314.16|1014.64|1085.04|25.13|0.00|314.16|339.29|-700.48| +2452568|53094|1848|90176|923667|4358|38602|8|189|1818|22|60.05|96.08|20.17|0.00|443.74|1321.10|2113.76|13.31|0.00|443.74|457.05|-877.36| +2452568|53094|9156|90176|923667|4358|38602|8|95|1818|97|44.79|83.75|78.72|0.00|7635.84|4344.63|8123.75|76.35|0.00|7635.84|7712.19|3291.21| +2452568|53094|16405|90176|923667|4358|||184|1818|63|||||887.04|4694.76|6337.80|||745.12|789.82|-3949.64| +2452568|53094|17850|90176|923667|4358|38602|8|154|1818|20|81.27|108.90|68.60|0.00|1372.00|1625.40|2178.00|13.72|0.00|1372.00|1385.72|-253.40| +2452568|53094|2196|90176|923667|4358|38602|8|173|1818|72|3.93|4.12|0.57|0.00|41.04|282.96|296.64|3.28|0.00|41.04|44.32|-241.92| +2452568|53094|13093|90176|923667|4358|38602|8|254|1818|61|75.01|135.76|55.66|0.00|3395.26|4575.61|8281.36|271.62|0.00|3395.26|3666.88|-1180.35| +2452568|53094|17841|90176|923667|4358|38602|8|10|1818|75|3.26|4.56|2.41|0.00|180.75|244.50|342.00|7.23|0.00|180.75|187.98|-63.75| +2452568|53094|462|90176|923667|4358|38602|8|287|1818|64|14.40|23.90|8.36|0.00|535.04|921.60|1529.60|48.15|0.00|535.04|583.19|-386.56| +2452049|29167|12763|15397|792080|6716|34552|8|26|1819|38|69.61|116.94|99.39|679.82|3776.82|2645.18|4443.72|247.76|679.82|3097.00|3344.76|451.82| +2452049|29167|13743|15397|792080|6716|34552|8|75|1819|49|28.93|32.69|25.49|0.00|1249.01|1417.57|1601.81|112.41|0.00|1249.01|1361.42|-168.56| +2452049|29167|2721|15397|792080|6716|34552|8|182|1819|92|61.82|117.45|17.61|0.00|1620.12|5687.44|10805.40|145.81|0.00|1620.12|1765.93|-4067.32| +2452049|29167|2025|15397|792080|6716|34552|8|140|1819|49|88.04|170.79|73.43|0.00|3598.07|4313.96|8368.71|0.00|0.00|3598.07|3598.07|-715.89| +2452049|29167|16015|15397|792080|6716|34552|8|205|1819|84|56.98|79.20|54.64|183.59|4589.76|4786.32|6652.80|0.00|183.59|4406.17|4406.17|-380.15| +2452049|29167|17521|15397|792080|6716|34552|8|141|1819|20|39.20|50.17|21.57|0.00|431.40|784.00|1003.40|12.94|0.00|431.40|444.34|-352.60| +2452049|29167|9079|15397|792080|6716|34552|8|183|1819|49|57.99|76.54|51.28|0.00|2512.72|2841.51|3750.46|50.25|0.00|2512.72|2562.97|-328.79| +2452049|29167|6089|15397|792080|6716|34552|8|127|1819|12|83.59|112.01|95.20|708.28|1142.40|1003.08|1344.12|8.68|708.28|434.12|442.80|-568.96| +2452049|29167|16841|15397|792080|6716|34552|8|133|1819|32|2.09|3.30|0.13|0.00|4.16|66.88|105.60|0.33|0.00|4.16|4.49|-62.72| +2452049|29167|12689|15397|792080|6716|34552|8|281|1819|13|72.23|75.11|66.84|0.00|868.92|938.99|976.43|8.68|0.00|868.92|877.60|-70.07| +2452049|29167|4825|15397|792080|6716|34552|8|99|1819|15|15.34|15.34|6.44|0.00|96.60|230.10|230.10|8.69|0.00|96.60|105.29|-133.50| +2452049|29167|17019|15397|792080|6716|34552|8|145|1819|6|60.94|74.95|50.96|299.64|305.76|365.64|449.70|0.36|299.64|6.12|6.48|-359.52| +2452049|29167|15129|15397|792080|6716|34552|8|292|1819|62|42.65|75.49|15.85|0.00|982.70|2644.30|4680.38|88.44|0.00|982.70|1071.14|-1661.60| +2452049|29167|2437|15397|792080|6716|34552|8|170|1819|89|66.75|86.77|41.64|0.00|3705.96|5940.75|7722.53|74.11|0.00|3705.96|3780.07|-2234.79| +2451583|48116|2749|12276|1870895|1514|44486|1|267|1820|9|17.45|32.45|24.98|0.00|224.82|157.05|292.05|8.99|0.00|224.82|233.81|67.77| +2451583|48116|7255|12276|1870895|1514|44486|1|255|1820|9|25.95|51.12|12.78|0.00|115.02|233.55|460.08|2.30|0.00|115.02|117.32|-118.53| +2451583|48116|15259|12276|1870895|1514|44486|1|256|1820|60|8.55|8.72|5.05|0.00|303.00|513.00|523.20|21.21|0.00|303.00|324.21|-210.00| +2451583|48116|11018|12276|1870895|1514|44486|1|114|1820|57|1.18|1.34|0.99|45.14|56.43|67.26|76.38|0.33|45.14|11.29|11.62|-55.97| +2451583|48116|14555|12276|1870895|1514|44486|1|9|1820|30|14.17|14.45|8.23|0.00|246.90|425.10|433.50|14.81|0.00|246.90|261.71|-178.20| +2451583|48116|2222|12276|1870895|1514|44486|1|144|1820|7|38.55|56.66|35.12|152.42|245.84|269.85|396.62|0.93|152.42|93.42|94.35|-176.43| +2451583|48116|3557|12276|1870895|1514|44486|1|231|1820|30|85.54|133.44|69.38|0.00|2081.40|2566.20|4003.20|124.88|0.00|2081.40|2206.28|-484.80| +2451583|48116|8459|12276|1870895|1514|44486|1|90|1820|70|64.74|97.11|5.82|0.00|407.40|4531.80|6797.70|20.37|0.00|407.40|427.77|-4124.40| +2451583|48116|9701|12276|1870895|1514|44486|1|122|1820|31|41.85|48.96|37.69|0.00|1168.39|1297.35|1517.76|46.73|0.00|1168.39|1215.12|-128.96| +2451583|48116|6415|12276|1870895|1514|44486|1|5|1820|52|18.67|30.24|26.61|0.00|1383.72|970.84|1572.48|41.51|0.00|1383.72|1425.23|412.88| +2451583|48116|12500|12276|1870895|1514|44486|1|260|1820|80|49.75|80.09|73.68|0.00|5894.40|3980.00|6407.20|235.77|0.00|5894.40|6130.17|1914.40| +2451890|61644|1550|1665|1795922|3332|33989|8|130|1821|15|21.10|29.11|2.91|0.00|43.65|316.50|436.65|2.61|0.00|43.65|46.26|-272.85| +2451890|61644|12773|1665|1795922|3332|33989|8|62|1821|38|39.42|65.83|61.88|0.00|2351.44|1497.96|2501.54|188.11|0.00|2351.44|2539.55|853.48| +2451890|61644|5467|1665|1795922|3332|33989|8|146|1821|52|55.98|78.37|32.13|0.00|1670.76|2910.96|4075.24|16.70|0.00|1670.76|1687.46|-1240.20| +2451890|61644|8816|1665|1795922|3332|33989|8|107|1821|44|67.14|73.85|28.06|0.00|1234.64|2954.16|3249.40|111.11|0.00|1234.64|1345.75|-1719.52| +2451890|61644|7028|1665|1795922|3332|33989|8|60|1821|100|41.38|44.69|11.61|0.00|1161.00|4138.00|4469.00|92.88|0.00|1161.00|1253.88|-2977.00| +2451890|61644|2155|1665|1795922|3332|33989|8|269|1821|33|59.35|115.73|67.12|0.00|2214.96|1958.55|3819.09|177.19|0.00|2214.96|2392.15|256.41| +2451890|61644|15389|1665|1795922|3332|33989|8|211|1821|34|66.52|117.74|72.99|0.00|2481.66|2261.68|4003.16|173.71|0.00|2481.66|2655.37|219.98| +2451890|61644|6265|1665|1795922|3332|33989|8|174|1821|43|43.88|53.53|20.87|0.00|897.41|1886.84|2301.79|17.94|0.00|897.41|915.35|-989.43| +2451890|61644|13073|1665|1795922|3332|33989|8|70|1821|28|35.89|63.52|42.55|345.50|1191.40|1004.92|1778.56|42.29|345.50|845.90|888.19|-159.02| +2451890||4466||1795922|3332|33989||40|1821|||44.99|9.89|0.00|59.34||269.94|0.00|0.00||59.34|-163.80| +2451890|61644|7679|1665|1795922|3332|33989|8|166|1821|8|18.09|20.62|15.46|0.00|123.68|144.72|164.96|7.42|0.00|123.68|131.10|-21.04| +2451890|61644|14279|1665|1795922|3332|33989|8|28|1821|74|74.83|121.97|69.52|0.00|5144.48|5537.42|9025.78|51.44|0.00|5144.48|5195.92|-392.94| +2451890|61644|3470|1665|1795922|3332|33989|8|236|1821|99|71.04|103.71|99.56|3154.06|9856.44|7032.96|10267.29|603.21|3154.06|6702.38|7305.59|-330.58| +2451890|61644|3392|1665|1795922|3332|33989|8|78|1821|89|32.23|41.25|18.15|0.00|1615.35|2868.47|3671.25|129.22|0.00|1615.35|1744.57|-1253.12| +2451890|61644|9056|1665|1795922|3332|33989|8|128|1821|3|19.20|33.79|11.82|0.00|35.46|57.60|101.37|0.00|0.00|35.46|35.46|-22.14| +2451237|36452|98|77629|1115703|2224|22698|7|174|1822|11|6.60|11.02|8.15|12.55|89.65|72.60|121.22|3.85|12.55|77.10|80.95|4.50| +2451237|36452|5168|77629|1115703|2224|22698|7|77|1822|16|86.28|146.67|30.80|0.00|492.80|1380.48|2346.72|44.35|0.00|492.80|537.15|-887.68| +2451237|36452|698|77629|1115703|2224|22698|7|95|1822|3|24.29|42.50|17.85|0.00|53.55|72.87|127.50|4.81|0.00|53.55|58.36|-19.32| +2451237|36452|11600|77629|1115703|2224|22698|7|82|1822|83|99.05|141.64|26.91|0.00|2233.53|8221.15|11756.12|89.34|0.00|2233.53|2322.87|-5987.62| +2451237|36452|5182|77629|1115703|2224|22698|7|285|1822|96|30.95|39.30|8.64|0.00|829.44|2971.20|3772.80|0.00|0.00|829.44|829.44|-2141.76| +2451237|36452|4627|77629|1115703|2224|22698|7|39|1822|14|50.73|84.71|29.64|0.00|414.96|710.22|1185.94|12.44|0.00|414.96|427.40|-295.26| +2451237|36452|9751|77629|1115703|2224|22698|7|165|1822|71|47.54|70.83|26.91|0.00|1910.61|3375.34|5028.93|133.74|0.00|1910.61|2044.35|-1464.73| +2451237|36452|15332|77629|1115703|2224|22698|7|145|1822|20|52.47|100.74|15.11|0.00|302.20|1049.40|2014.80|15.11|0.00|302.20|317.31|-747.20| +2451237|36452|4832|77629|1115703|2224|22698|7|298|1822|75|49.59|85.29|80.17|0.00|6012.75|3719.25|6396.75|360.76|0.00|6012.75|6373.51|2293.50| +2452248|65571|1303|88270|479839|2161|18930|1|194|1823|42|25.22|39.84|39.04|0.00|1639.68|1059.24|1673.28|114.77|0.00|1639.68|1754.45|580.44| +2452248|65571|15035|88270|479839|2161|18930|1|290|1823|49|63.20|101.75|57.99|1221.84|2841.51|3096.80|4985.75|48.59|1221.84|1619.67|1668.26|-1477.13| +2452248|65571|16917|88270|479839|2161|18930|1|20|1823|95|73.80|138.00|128.34|0.00|12192.30|7011.00|13110.00|487.69|0.00|12192.30|12679.99|5181.30| +2452248|65571|15763|88270|479839|2161|18930|1|84|1823|8|80.87|150.41|94.75|0.00|758.00|646.96|1203.28|30.32|0.00|758.00|788.32|111.04| +2452248|65571|16669|88270|479839|2161|18930|1|195|1823|50|98.84|165.06|79.22|0.00|3961.00|4942.00|8253.00|198.05|0.00|3961.00|4159.05|-981.00| +2452248|65571|1633|88270|479839|2161|18930|1|51|1823|69|77.56|97.72|52.76|0.00|3640.44|5351.64|6742.68|0.00|0.00|3640.44|3640.44|-1711.20| +2452248|65571|8191|88270|479839|2161|18930|1|145|1823|12|19.14|19.71|12.22|0.00|146.64|229.68|236.52|8.79|0.00|146.64|155.43|-83.04| +2452248|65571|673|88270|479839|2161|18930|1|198|1823|36|17.77|25.58|18.92|0.00|681.12|639.72|920.88|34.05|0.00|681.12|715.17|41.40| +2452248|65571|15861|88270|479839|2161|18930|1|171|1823|66|47.91|54.13|34.10|1237.83|2250.60|3162.06|3572.58|40.51|1237.83|1012.77|1053.28|-2149.29| +2452248|65571|16485|88270|||||269|1823|39||112.69|||1933.62|3632.46||154.68|||2088.30|-1698.84| +2452248|65571|4621|88270|479839|2161|18930|1|209|1823|78|36.12|54.54|49.63|0.00|3871.14|2817.36|4254.12|193.55|0.00|3871.14|4064.69|1053.78| +2452248|65571|1861|88270|479839|2161|18930|1|4|1823|54|53.45|53.98|49.12|0.00|2652.48|2886.30|2914.92|79.57|0.00|2652.48|2732.05|-233.82| +2452248|65571|9287|88270|479839|2161|18930|1|98|1823|44|44.13|84.72|11.86|0.00|521.84|1941.72|3727.68|10.43|0.00|521.84|532.27|-1419.88| +2452248|65571|2077|88270|479839|2161|18930|1|73|1823|13|12.51|23.89|22.45|0.00|291.85|162.63|310.57|2.91|0.00|291.85|294.76|129.22| +2452248|65571|9641|88270|479839|2161|18930|1|127|1823|37|8.84|11.40|8.20|0.00|303.40|327.08|421.80|27.30|0.00|303.40|330.70|-23.68| +|65571|4853||||18930|1|94|1823||74.05||15.86||1348.10|6294.25|9629.65|||1348.10|1388.54|| +2451686|45434|6395|76675|114140|3158|48623|10|261|1824|12|61.80|103.20|60.88|0.00|730.56|741.60|1238.40|43.83|0.00|730.56|774.39|-11.04| +2451686|45434|500|76675|114140|3158|48623|10|281|1824|31|81.74|109.53|101.86|0.00|3157.66|2533.94|3395.43|189.45|0.00|3157.66|3347.11|623.72| +2451686||10790||||48623|10||1824|36||130.51|27.40|0.00|986.40|2539.80|4698.36||0.00|||| +2451686|45434|11726|76675|114140|3158|48623|10|96|1824|74|46.91|92.41|46.20|0.00|3418.80|3471.34|6838.34|34.18|0.00|3418.80|3452.98|-52.54| +2451686|45434|12251|76675|114140|3158|48623|10|16|1824|29|25.18|49.85|20.43|0.00|592.47|730.22|1445.65|11.84|0.00|592.47|604.31|-137.75| +2451686|45434|7109|76675|114140|3158|48623|10|33|1824|5|80.44|82.04|10.66|0.00|53.30|402.20|410.20|4.79|0.00|53.30|58.09|-348.90| +2451686|45434|13229|76675|114140|3158|48623|10|263|1824|87|79.05|123.31|65.35|3922.96|5685.45|6877.35|10727.97|35.24|3922.96|1762.49|1797.73|-5114.86| +2451686|45434|14855|76675|114140|3158|48623|10|15|1824|17|17.79|17.79|10.67|148.73|181.39|302.43|302.43|0.32|148.73|32.66|32.98|-269.77| +2451686|45434|13609|76675|114140|3158|48623|10|143|1824|26|21.36|42.07|10.93|0.00|284.18|555.36|1093.82|2.84|0.00|284.18|287.02|-271.18| +2451686|45434|4639|76675|114140|3158|48623|10|246|1824|85|98.27|107.11|83.54|0.00|7100.90|8352.95|9104.35|639.08|0.00|7100.90|7739.98|-1252.05| +2451686|45434|12229|76675|114140|3158|48623|10|139|1824|18|86.18|106.00|94.34|0.00|1698.12|1551.24|1908.00|152.83|0.00|1698.12|1850.95|146.88| +2451686|45434|8761|76675|114140|3158|48623|10|218|1824|31|68.88|84.03|64.70|0.00|2005.70|2135.28|2604.93|100.28|0.00|2005.70|2105.98|-129.58| +2451686|45434|12566|76675|114140|3158|48623|10|46|1824|21|53.33|69.32|10.39|72.00|218.19|1119.93|1455.72|1.46|72.00|146.19|147.65|-973.74| +2452528|36151|15381|40937|1663102|6420|26138|10|219|1825|91|45.94|59.26|38.51|0.00|3504.41|4180.54|5392.66|105.13|0.00|3504.41|3609.54|-676.13| +2452528|36151|5754|40937|1663102|6420|26138|10|124|1825|7|42.98|55.87|20.11|0.00|140.77|300.86|391.09|7.03|0.00|140.77|147.80|-160.09| +2452528|36151|12900|40937|1663102|6420|26138|10|195|1825|53|88.28|163.31|97.98|4154.35|5192.94|4678.84|8655.43|62.31|4154.35|1038.59|1100.90|-3640.25| +2452528|36151|11475|40937|1663102|6420|26138|10|105|1825|80|81.52|131.24|81.36|0.00|6508.80|6521.60|10499.20|390.52|0.00|6508.80|6899.32|-12.80| +2452528|36151|15177|40937|1663102|6420|26138|10|30|1825|17|31.08|36.36|30.54|451.68|519.18|528.36|618.12|2.02|451.68|67.50|69.52|-460.86| +2452528|36151|13897|40937|1663102|6420|26138|10|26|1825|29|51.19|89.58|13.43|0.00|389.47|1484.51|2597.82|3.89|0.00|389.47|393.36|-1095.04| +2452528|36151|14905|40937|1663102|6420|26138|10|58|1825|75|45.44|74.06|37.77|0.00|2832.75|3408.00|5554.50|254.94|0.00|2832.75|3087.69|-575.25| +2452528|36151|4227|40937|1663102|6420|26138|10|298|1825|51|36.47|40.84|1.22|0.00|62.22|1859.97|2082.84|0.62|0.00|62.22|62.84|-1797.75| +2452603|40819|10671|87622|1172371|5920|5916|4|21|1826|23|23.36|23.36|2.33|9.11|53.59|537.28|537.28|3.55|9.11|44.48|48.03|-492.80| +2452603|40819|2418|87622|1172371|5920|5916|4|145|1826|58|70.92|99.28|95.30|0.00|5527.40|4113.36|5758.24|110.54|0.00|5527.40|5637.94|1414.04| +2452603|40819|7567|87622|1172371|5920|5916|4|265|1826|88|52.77|86.54|25.09|0.00|2207.92|4643.76|7615.52|44.15|0.00|2207.92|2252.07|-2435.84| +2452603|40819|8103|87622|1172371|5920|5916|4|21|1826|41|70.37|97.81|79.22|0.00|3248.02|2885.17|4010.21|64.96|0.00|3248.02|3312.98|362.85| +2452603|40819|312|87622|1172371|5920|5916|4|206|1826|51|20.89|40.73|27.69|0.00|1412.19|1065.39|2077.23|84.73|0.00|1412.19|1496.92|346.80| +2452603|40819|3039|87622|1172371|5920|5916|4|125|1826|49|35.12|69.18|30.43|1013.92|1491.07|1720.88|3389.82|19.08|1013.92|477.15|496.23|-1243.73| +2452603|40819|162|87622|1172371|5920|5916|4|257|1826|72|86.15|168.85|111.44|0.00|8023.68|6202.80|12157.20|561.65|0.00|8023.68|8585.33|1820.88| +2452603|40819|4389|87622|1172371|5920|5916|4|82|1826|44|2.34|3.01|1.14|0.00|50.16|102.96|132.44|1.00|0.00|50.16|51.16|-52.80| +2452603|40819|16920|87622|1172371|5920|5916|4|27|1826|49|74.40|121.27|53.35|0.00|2614.15|3645.60|5942.23|156.84|0.00|2614.15|2770.99|-1031.45| +2452603|40819|14637|87622|1172371|5920|5916|4|130|1826|1|93.66|180.76|77.72|55.95|77.72|93.66|180.76|0.00|55.95|21.77|21.77|-71.89| +2452603|40819|9564|87622|1172371|5920|5916|4|194|1826|97|57.20|57.77|37.55|0.00|3642.35|5548.40|5603.69|218.54|0.00|3642.35|3860.89|-1906.05| +2452603|40819|5509|87622|1172371|5920|5916|4|291|1826|14|76.21|85.35|74.25|831.60|1039.50|1066.94|1194.90|18.71|831.60|207.90|226.61|-859.04| +2452603|40819|4677|87622|1172371|5920|5916|4|183|1826|50|75.63|133.10|46.58|0.00|2329.00|3781.50|6655.00|186.32|0.00|2329.00|2515.32|-1452.50| +|40819|2701||||5916|4|253|1826|20||||||892.60|1660.20|51.79||||402.20| +|74627|6614|55242||3271||||1827||31.28|38.16|24.04|648.35|745.24||||648.35||103.67|| +2451894|74627|4856|55242|42680|3271|39509|7|7|1827|16|61.79|71.05|70.33|0.00|1125.28|988.64|1136.80|78.76|0.00|1125.28|1204.04|136.64| +2451894|74627|4112|55242|42680|3271|39509|7|61|1827|95|8.64|13.30|7.98|0.00|758.10|820.80|1263.50|0.00|0.00|758.10|758.10|-62.70| +2451894|74627|5876|55242|42680|3271|39509|7|240|1827|45|20.79|38.66|27.06|0.00|1217.70|935.55|1739.70|48.70|0.00|1217.70|1266.40|282.15| +2451894|74627|8330|55242|42680|3271|39509|7|19|1827|72|56.87|66.53|12.64|0.00|910.08|4094.64|4790.16|81.90|0.00|910.08|991.98|-3184.56| +2451894|74627|12029|55242|42680|3271|39509|7|151|1827|85|35.30|50.83|12.70|0.00|1079.50|3000.50|4320.55|86.36|0.00|1079.50|1165.86|-1921.00| +2451894|74627|16565|55242|42680|3271|39509|7|287|1827|38|91.42|102.39|38.90|0.00|1478.20|3473.96|3890.82|44.34|0.00|1478.20|1522.54|-1995.76| +2451894|74627|9965|55242|42680|3271|39509|7|7|1827|59|52.21|93.45|63.54|0.00|3748.86|3080.39|5513.55|337.39|0.00|3748.86|4086.25|668.47| +2451894|74627|6749||42680|3271|39509|7|93|1827||16.73|22.08|17.22|0.00|||||0.00|||| +2451894|74627|15385|55242|42680|3271|39509|7|272|1827|22|71.27|138.97|125.07|0.00|2751.54|1567.94|3057.34|27.51|0.00|2751.54|2779.05|1183.60| +2451894|74627|235|55242|42680|3271|39509|7|72|1827|34|3.32|4.98|1.44|0.00|48.96|112.88|169.32|2.44|0.00|48.96|51.40|-63.92| +2451144|44632|2116|72562|696177|1392|38207|2|220|1828|87|55.90|70.43|23.24|0.00|2021.88|4863.30|6127.41|0.00|0.00|2021.88|2021.88|-2841.42| +2451144|44632|11768|72562|696177|1392|38207|2|3|1828|66|70.69|110.98|38.84|589.59|2563.44|4665.54|7324.68|59.21|589.59|1973.85|2033.06|-2691.69| +2451144|44632|3692|72562|696177|1392|38207|2|46|1828|41|45.80|90.22|28.87|0.00|1183.67|1877.80|3699.02|35.51|0.00|1183.67|1219.18|-694.13| +2451144|44632|16729|72562|696177|1392|38207|2|183|1828|1|92.98|132.96|29.25|0.00|29.25|92.98|132.96|0.87|0.00|29.25|30.12|-63.73| +||14707|72562||1392|38207|2|140|1828||89.29|100.00|59.00|0.00|||5800.00||0.00|||-1756.82| +2451144|44632|3656|72562|696177|1392|38207|2|149|1828|28|39.06|60.15|22.25|0.00|623.00|1093.68|1684.20|24.92|0.00|623.00|647.92|-470.68| +2451144|44632|7093|72562|696177|1392|38207|2|2|1828|18|46.82|75.38|51.25|0.00|922.50|842.76|1356.84|18.45|0.00|922.50|940.95|79.74| +2451144|44632|16378|72562|696177|1392|38207|2|131|1828|18|88.41|111.39|61.26|0.00|1102.68|1591.38|2005.02|66.16|0.00|1102.68|1168.84|-488.70| +2451144|44632|2791|72562|696177|1392|38207|2|58|1828|8|76.04|134.59|55.18|110.36|441.44|608.32|1076.72|16.55|110.36|331.08|347.63|-277.24| +2451144|44632|92|72562|696177|1392|38207|2|193|1828|95|89.52|141.44|26.87|0.00|2552.65|8504.40|13436.80|25.52|0.00|2552.65|2578.17|-5951.75| +2451144|44632|529|72562|696177|1392|38207|2|171|1828|13|27.20|41.61|25.79|0.00|335.27|353.60|540.93|3.35|0.00|335.27|338.62|-18.33| +2451144|44632|14674|72562|696177|1392|38207|2|25|1828|90|6.01|10.69|8.44|0.00|759.60|540.90|962.10|37.98|0.00|759.60|797.58|218.70| +2451144|44632|8656|72562|696177|1392|38207|2|14|1828|34|6.60|13.00|12.22|0.00|415.48|224.40|442.00|16.61|0.00|415.48|432.09|191.08| +2451144|44632|11911|72562|696177|1392|38207|2|297|1828|19|32.47|62.01|45.88|0.00|871.72|616.93|1178.19|52.30|0.00|871.72|924.02|254.79| +2451933|63711|11959|89003|1392905|2407|20709|2|194|1829|96|8.32|8.40|1.34|0.00|128.64|798.72|806.40|10.29|0.00|128.64|138.93|-670.08| +2451933|63711|3077|89003|1392905|2407|20709|2|280|1829|15|76.98|145.49|18.91|0.00|283.65|1154.70|2182.35|0.00|0.00|283.65|283.65|-871.05| +2451933|63711|7|89003|1392905|2407|20709|2|138|1829|26|13.90|26.41|11.09|0.00|288.34|361.40|686.66|17.30|0.00|288.34|305.64|-73.06| +2451933|63711|12633|89003|1392905|2407|20709|2|75|1829|6|14.77|22.74|20.46|72.42|122.76|88.62|136.44|0.00|72.42|50.34|50.34|-38.28| +2451933|63711|11945|89003|1392905|2407|20709|2|287|1829|87|21.02|34.68|18.38|271.84|1599.06|1828.74|3017.16|0.00|271.84|1327.22|1327.22|-501.52| +2451933|63711|16569|89003|1392905|2407|20709|2|184|1829|75|86.14|116.28|93.02|0.00|6976.50|6460.50|8721.00|0.00|0.00|6976.50|6976.50|516.00| +2451933|63711|5079|89003|1392905|2407|20709|2|210|1829|35|28.28|44.11|31.75|0.00|1111.25|989.80|1543.85|44.45|0.00|1111.25|1155.70|121.45| +2451933|63711|11369|89003|1392905|2407|20709|2|125|1829|7|16.06|16.06|5.94|0.00|41.58|112.42|112.42|3.32|0.00|41.58|44.90|-70.84| +2451933|63711|13339|89003|1392905|2407|20709|2|234|1829|67|55.08|66.09|5.28|0.00|353.76|3690.36|4428.03|7.07|0.00|353.76|360.83|-3336.60| +2451933|63711|9319|89003|1392905|2407|20709|2|98|1829|30|41.20|77.04|42.37|0.00|1271.10|1236.00|2311.20|76.26|0.00|1271.10|1347.36|35.10| +2451933|63711|5217|89003|1392905|2407|20709|2|276|1829|12|16.41|18.54|11.86|126.66|142.32|196.92|222.48|0.78|126.66|15.66|16.44|-181.26| +2451933|63711|4487|89003|1392905|2407|20709|2|111|1829|40|9.23|9.41|0.65|0.00|26.00|369.20|376.40|1.82|0.00|26.00|27.82|-343.20| +2452589|52567|10380|64462|1662063|4017|16771|4|137|1830|71|89.83|128.45|10.27|0.00|729.17|6377.93|9119.95|7.29|0.00|729.17|736.46|-5648.76| +2452589|52567|10837|64462|1662063|4017|16771|4|202|1830|8|96.57|135.19|104.09|774.42|832.72|772.56|1081.52|4.66|774.42|58.30|62.96|-714.26| +2452589|52567|13524|64462|1662063|4017|16771|4|2|1830|19|58.73|108.06|29.17|0.00|554.23|1115.87|2053.14|16.62|0.00|554.23|570.85|-561.64| +2452589|52567|8011|64462|1662063|4017|16771|4|44|1830|65|59.49|74.36|35.69|0.00|2319.85|3866.85|4833.40|23.19|0.00|2319.85|2343.04|-1547.00| +2452589|52567|8977|64462|1662063|4017|16771|4|212|1830|51|14.58|25.36|22.06|438.77|1125.06|743.58|1293.36|61.76|438.77|686.29|748.05|-57.29| +2452589|52567|15096|64462|1662063|4017|16771|4|22|1830|59|80.83|96.18|38.47|0.00|2269.73|4768.97|5674.62|158.88|0.00|2269.73|2428.61|-2499.24| +2452589|52567|16731|64462|1662063|4017|16771|4|217|1830|78|69.18|103.77|90.27|0.00|7041.06|5396.04|8094.06|492.87|0.00|7041.06|7533.93|1645.02| +2452589|52567|1164|64462|1662063|4017|16771|4|242|1830|31|81.81|94.89|5.69|0.00|176.39|2536.11|2941.59|14.11|0.00|176.39|190.50|-2359.72| +2452589|52567|7944|64462|1662063|4017|16771|4|168|1830|43|9.37|9.37|8.33|0.00|358.19|402.91|402.91|32.23|0.00|358.19|390.42|-44.72| +2452589|52567|11508|64462|1662063|4017|16771|4|190|1830|44|49.59|81.82|2.45|0.00|107.80|2181.96|3600.08|5.39|0.00|107.80|113.19|-2074.16| +2452589|52567|3096|64462|1662063|4017|16771|4|45|1830|13|48.84|74.23|60.12|0.00|781.56|634.92|964.99|7.81|0.00|781.56|789.37|146.64| +2452589|52567|225|64462|1662063|4017|16771|4|213|1830|49|51.86|102.16|34.73|1378.43|1701.77|2541.14|5005.84|25.86|1378.43|323.34|349.20|-2217.80| +2452589|52567|16477|64462|1662063|4017|16771|4|254|1830|32|62.28|92.17|76.50|2227.68|2448.00|1992.96|2949.44|15.42|2227.68|220.32|235.74|-1772.64| +2452589|52567|2898|64462|1662063|4017|16771|4|59|1830|53|40.04|62.46|4.99|0.00|264.47|2122.12|3310.38|10.57|0.00|264.47|275.04|-1857.65| +2452589|52567|13863|64462|1662063|4017|16771|4|199|1830|73|32.55|33.85|17.94|0.00|1309.62|2376.15|2471.05|65.48|0.00|1309.62|1375.10|-1066.53| +2452589|52567|3025|64462|1662063|4017|16771|4|108|1830|77|85.13|146.42|77.60|0.00|5975.20|6555.01|11274.34|418.26|0.00|5975.20|6393.46|-579.81| +2451051|54062|12052|86872|885297|6122|44194|4|26|1831|82|40.41|47.27|16.07|500.74|1317.74|3313.62|3876.14|49.02|500.74|817.00|866.02|-2496.62| +2451051|54062|8348|86872|885297|6122|44194|4|2|1831|41|95.57|102.25|53.17|0.00|2179.97|3918.37|4192.25|0.00|0.00|2179.97|2179.97|-1738.40| +2451051|54062|3763|86872|885297|6122|44194|4|167|1831|40|33.91|49.16|40.80|0.00|1632.00|1356.40|1966.40|97.92|0.00|1632.00|1729.92|275.60| +2451051|54062|10681|86872|885297|6122|44194|4|116|1831|36|27.18|52.72|14.23|266.38|512.28|978.48|1897.92|2.45|266.38|245.90|248.35|-732.58| +2451051|54062|2485|86872|885297|6122|44194|4|74|1831|38|62.15|117.46|56.38|0.00|2142.44|2361.70|4463.48|171.39|0.00|2142.44|2313.83|-219.26| +2451051|54062|16148|86872|885297|6122|44194|4|273|1831|29|71.40|107.81|25.87|0.00|750.23|2070.60|3126.49|52.51|0.00|750.23|802.74|-1320.37| +2451051|54062|15895|86872|885297|6122|44194|4|297|1831|88|71.97|139.62|127.05|0.00|11180.40|6333.36|12286.56|447.21|0.00|11180.40|11627.61|4847.04| +2451051|54062|8608|86872|885297|6122|44194|4|173|1831|62|97.09|185.44|96.42|0.00|5978.04|6019.58|11497.28|0.00|0.00|5978.04|5978.04|-41.54| +2451051|54062|2050|86872|885297|6122|44194|4|26|1831|1|89.93|105.21|53.65|0.00|53.65|89.93|105.21|3.75|0.00|53.65|57.40|-36.28| +2451051|54062|10117|86872|885297|6122|44194|4|79|1831|5|43.64|51.49|15.44|0.00|77.20|218.20|257.45|1.54|0.00|77.20|78.74|-141.00| +2451051|54062|2239|86872|885297|6122|44194|4|254|1831|79|44.19|72.91|48.12|0.00|3801.48|3491.01|5759.89|152.05|0.00|3801.48|3953.53|310.47| +2451051|54062|2989|86872|885297|6122|44194|4|294|1831|82|28.93|48.02|47.53|0.00|3897.46|2372.26|3937.64|272.82|0.00|3897.46|4170.28|1525.20| +2451051|54062|2276|86872|885297|6122|44194|4|21|1831|84|15.69|29.02|28.72|2147.10|2412.48|1317.96|2437.68|5.30|2147.10|265.38|270.68|-1052.58| +2451051|54062|5894|86872|885297|6122|44194|4|287|1831|18|88.55|143.45|30.12|444.57|542.16|1593.90|2582.10|4.87|444.57|97.59|102.46|-1496.31| +2452221|30819|16409|40020|1098709|1045|33734|8|123|1832|100|3.82|6.68|3.07|0.00|307.00|382.00|668.00|21.49|0.00|307.00|328.49|-75.00| +2452221|30819|14185|40020|1098709|1045|33734|8|196|1832|77|80.18|157.15|50.28|0.00|3871.56|6173.86|12100.55|309.72|0.00|3871.56|4181.28|-2302.30| +2452221|30819|10545|40020|1098709|1045|33734|8|36|1832|56|79.88|103.04|39.15|0.00|2192.40|4473.28|5770.24|43.84|0.00|2192.40|2236.24|-2280.88| +2452221|30819|6075|40020|1098709|1045|33734|8|276|1832|72|59.17|66.86|30.75|0.00|2214.00|4260.24|4813.92|44.28|0.00|2214.00|2258.28|-2046.24| +2452221|30819|2715|40020|1098709|1045|33734|8|141|1832|87|60.59|65.43|54.30|1842.39|4724.10|5271.33|5692.41|172.90|1842.39|2881.71|3054.61|-2389.62| +2452221|30819|3413|40020|1098709|1045|33734|8|16|1832|97|41.52|49.40|27.17|0.00|2635.49|4027.44|4791.80|184.48|0.00|2635.49|2819.97|-1391.95| +2452221|30819|14925|40020|1098709|1045|33734|8|297|1832|78|30.18|53.41|14.42|202.45|1124.76|2354.04|4165.98|0.00|202.45|922.31|922.31|-1431.73| +2452221|30819|4063|40020|1098709|1045|33734|8|176|1832|61|58.62|99.06|98.06|0.00|5981.66|3575.82|6042.66|59.81|0.00|5981.66|6041.47|2405.84| +2452221|30819|6329|40020|1098709|1045|33734|8|226|1832|2|84.76|106.79|95.04|0.00|190.08|169.52|213.58|7.60|0.00|190.08|197.68|20.56| +2452221|30819|14609|40020|1098709|1045|33734|8|86|1832|73|87.55|157.59|97.70|0.00|7132.10|6391.15|11504.07|213.96|0.00|7132.10|7346.06|740.95| +2452221|30819|15353|40020|1098709|1045|33734|8|227|1832|68|41.36|78.99|63.98|2044.80|4350.64|2812.48|5371.32|115.29|2044.80|2305.84|2421.13|-506.64| +2452221|30819|3079|40020|1098709|1045|33734|8|4|1832|10|97.50|166.72|111.70|0.00|1117.00|975.00|1667.20|89.36|0.00|1117.00|1206.36|142.00| +2452221|30819|14501|40020|1098709|1045|33734|8|185|1832|40|35.26|35.61|12.81|0.00|512.40|1410.40|1424.40|30.74|0.00|512.40|543.14|-898.00| +2452221|30819|5027|40020|1098709|1045|33734|8|271|1832|36|62.94|105.10|9.45|0.00|340.20|2265.84|3783.60|20.41|0.00|340.20|360.61|-1925.64| +2451291|39545|15460|63133|658833|2886|45386|7|4|1833|38|16.14|25.33|22.03|0.00|837.14|613.32|962.54|66.97|0.00|837.14|904.11|223.82| +2451291|39545|11779|63133|658833|2886|45386|7|256|1833|13|61.74|78.40|43.12|437.23|560.56|802.62|1019.20|2.46|437.23|123.33|125.79|-679.29| +2451291|39545|16012|63133|658833|2886|45386|7|216|1833|40|24.42|47.61|21.42|0.00|856.80|976.80|1904.40|25.70|0.00|856.80|882.50|-120.00| +2451291|39545|13726|63133|658833|2886|45386|7|209|1833|16|14.45|15.02|2.55|0.00|40.80|231.20|240.32|3.26|0.00|40.80|44.06|-190.40| +2451291|39545|16502|63133|658833|2886|45386|7|236|1833|63|38.47|39.62|28.13|0.00|1772.19|2423.61|2496.06|141.77|0.00|1772.19|1913.96|-651.42| +2451291|39545|4057|63133|658833|2886|45386|7|111|1833|95|8.32|15.05|8.72|0.00|828.40|790.40|1429.75|24.85|0.00|828.40|853.25|38.00| +2451291|39545|6454|63133|658833|2886|45386|7|195|1833|8|43.95|57.13|54.84|0.00|438.72|351.60|457.04|30.71|0.00|438.72|469.43|87.12| +2451291|39545|5401|63133|658833|2886|45386|7|24|1833|50|94.50|143.64|84.74|1949.02|4237.00|4725.00|7182.00|68.63|1949.02|2287.98|2356.61|-2437.02| +2451291|39545|566|63133|658833|2886|45386|7|84|1833|60|28.22|33.29|29.62|799.74|1777.20|1693.20|1997.40|87.97|799.74|977.46|1065.43|-715.74| +2451291|39545|13454|63133|658833|2886|45386|7|103|1833|77|66.10|109.06|97.06|0.00|7473.62|5089.70|8397.62|448.41|0.00|7473.62|7922.03|2383.92| +2451291|39545|11338|63133|658833|2886|45386|7|128|1833|68|20.18|20.98|6.92|0.00|470.56|1372.24|1426.64|37.64|0.00|470.56|508.20|-901.68| +2451291|39545|5690|63133|658833|2886|45386|7|51|1833|4|59.18|107.70|45.23|0.00|180.92|236.72|430.80|10.85|0.00|180.92|191.77|-55.80| +2451291|39545|15266|63133|658833|2886|45386|7|171|1833|85|75.71|134.76|39.08|0.00|3321.80|6435.35|11454.60|166.09|0.00|3321.80|3487.89|-3113.55| +2451291|39545|16531|63133|658833|2886|45386|7|83|1833|53|42.78|57.75|24.83|0.00|1315.99|2267.34|3060.75|52.63|0.00|1315.99|1368.62|-951.35| +2451291|39545|7483|63133|658833|2886|45386|7|12|1833|20|31.42|34.87|9.76|0.00|195.20|628.40|697.40|5.85|0.00|195.20|201.05|-433.20| +2451940|49041|14375|74|1735210|5546|29082|7|225|1834|54|46.90|66.12|61.49|3254.05|3320.46|2532.60|3570.48|1.32|3254.05|66.41|67.73|-2466.19| +2451940|49041|11341|74|1735210|5546|29082|7|109|1834|3|99.62|99.62|16.93|0.00|50.79|298.86|298.86|0.50|0.00|50.79|51.29|-248.07| +2451940|49041|2583|74|1735210|5546|29082|7|98|1834|90|72.22|120.60|43.41|2226.93|3906.90|6499.80|10854.00|134.39|2226.93|1679.97|1814.36|-4819.83| +2451940|49041|5285|74|1735210|5546|29082|7|175|1834|59|65.77|83.52|55.12|0.00|3252.08|3880.43|4927.68|65.04|0.00|3252.08|3317.12|-628.35| +2451940|49041|14655|74|1735210|5546|29082|7|99|1834|28|92.09|139.05|41.71|654.01|1167.88|2578.52|3893.40|35.97|654.01|513.87|549.84|-2064.65| +2451940|49041|273|74|1735210|5546|29082|7|78|1834|89|53.58|76.61|9.95|0.00|885.55|4768.62|6818.29|44.27|0.00|885.55|929.82|-3883.07| +2451940|49041|9431|74|1735210|5546|29082|7|177|1834|96|12.76|16.58|12.76|0.00|1224.96|1224.96|1591.68|24.49|0.00|1224.96|1249.45|0.00| +2451940|49041|15669|74|1735210|5546|29082|7|72|1834|99|87.13|142.89|107.16|0.00|10608.84|8625.87|14146.11|742.61|0.00|10608.84|11351.45|1982.97| +2451940|49041|13535|74|1735210|5546|29082|7|113|1834|69|86.34|101.01|91.91|0.00|6341.79|5957.46|6969.69|126.83|0.00|6341.79|6468.62|384.33| +2451940|49041|3099|74|1735210|5546|29082|7|221|1834|2|92.37|173.65|154.54|0.00|309.08|184.74|347.30|15.45|0.00|309.08|324.53|124.34| +2452629|34835|13443|77583|1505432|4834|23483|2|126|1835|32|98.89|102.84|24.68|0.00|789.76|3164.48|3290.88|7.89|0.00|789.76|797.65|-2374.72| +2452629|34835|822|77583|1505432|4834|23483|2|286|1835|27|23.55|28.02|17.65|252.57|476.55|635.85|756.54|11.19|252.57|223.98|235.17|-411.87| +2452629|34835|6583|77583|1505432|4834|23483|2|170|1835|6|19.70|31.71|1.58|0.00|9.48|118.20|190.26|0.09|0.00|9.48|9.57|-108.72| +2452629|34835|3177|77583|1505432|4834|23483|2|220|1835|59|72.38|92.64|26.86|0.00|1584.74|4270.42|5465.76|79.23|0.00|1584.74|1663.97|-2685.68| +2452629|34835|2371|77583|1505432|4834|23483|2|27|1835|82|36.66|52.05|19.25|757.68|1578.50|3006.12|4268.10|8.20|757.68|820.82|829.02|-2185.30| +2452629|34835|10423|77583|1505432|4834|23483|2|194|1835|1|41.32|61.15|14.67|3.81|14.67|41.32|61.15|0.00|3.81|10.86|10.86|-30.46| +2452629|34835|2334|77583|1505432|4834|23483|2|189|1835|91|71.19|101.08|66.71|0.00|6070.61|6478.29|9198.28|0.00|0.00|6070.61|6070.61|-407.68| +2452629|34835|7429|77583|1505432|4834|23483|2|292|1835|53|65.06|96.93|89.17|0.00|4726.01|3448.18|5137.29|0.00|0.00|4726.01|4726.01|1277.83| +2452629|34835|8184||||23483||264|1835|||||||438.40||||451.12|487.20|| +2452629|34835|2454|77583|1505432|4834|23483|2|56|1835|37|72.48|96.39|33.73|0.00|1248.01|2681.76|3566.43|0.00|0.00|1248.01|1248.01|-1433.75| +2452629|34835|13645|77583|1505432|4834|23483|2|270|1835|57|27.06|27.87|1.67|0.00|95.19|1542.42|1588.59|1.90|0.00|95.19|97.09|-1447.23| +2452629|34835|723|77583|1505432|4834|23483|2|72|1835|27|81.83|143.20|67.30|0.00|1817.10|2209.41|3866.40|72.68|0.00|1817.10|1889.78|-392.31| +2452629|34835|3048|77583|1505432|4834|23483|2|98|1835|42|57.60|76.03|35.73|0.00|1500.66|2419.20|3193.26|135.05|0.00|1500.66|1635.71|-918.54| +2452623|49331|1069|75087|582303|4272|19041|10|155|1836|45|59.56|102.44|0.00|0.00|0.00|2680.20|4609.80|0.00|0.00|0.00|0.00|-2680.20| +2452623|49331|1045|75087|582303|4272|19041|10|178|1836|7|16.95|28.13|5.90|0.00|41.30|118.65|196.91|2.89|0.00|41.30|44.19|-77.35| +2452623|49331|12345|75087|582303|4272|19041|10|247|1836|40|60.09|104.55|16.72|100.32|668.80|2403.60|4182.00|17.05|100.32|568.48|585.53|-1835.12| +2452623|49331|17047|75087|582303|4272|19041|10|143|1836|46|13.10|14.01|0.28|2.57|12.88|602.60|644.46|0.72|2.57|10.31|11.03|-592.29| +2452623|49331|5787|75087|582303|4272|19041|10|2|1836|51|70.15|124.86|116.11|0.00|5921.61|3577.65|6367.86|59.21|0.00|5921.61|5980.82|2343.96| +2452623|49331|816|75087|582303|4272|19041|10|69|1836|46|32.70|64.74|34.31|0.00|1578.26|1504.20|2978.04|63.13|0.00|1578.26|1641.39|74.06| +2452623|49331|630|75087|582303|4272|19041|10|31|1836|61|77.20|138.18|56.65|0.00|3455.65|4709.20|8428.98|276.45|0.00|3455.65|3732.10|-1253.55| +2452623|49331|909|75087|582303|4272|19041|10|53|1836|68|40.77|54.22|14.09|0.00|958.12|2772.36|3686.96|9.58|0.00|958.12|967.70|-1814.24| +2452623|49331|6973|75087|582303|4272|19041|10|161|1836|3|56.26|65.82|8.55|0.00|25.65|168.78|197.46|0.25|0.00|25.65|25.90|-143.13| +2452623|49331|2529|75087|582303|4272|19041|10|85|1836|1|91.46|176.51|54.71|0.00|54.71|91.46|176.51|0.54|0.00|54.71|55.25|-36.75| +2452623|49331|1350|75087|582303|4272|19041|10|133|1836|49|49.94|68.91|48.92|1054.71|2397.08|2447.06|3376.59|120.81|1054.71|1342.37|1463.18|-1104.69| +2452623|49331|13338|75087|582303|4272|19041|10|226|1836|58|82.38|93.08|52.12|1330.10|3022.96|4778.04|5398.64|118.50|1330.10|1692.86|1811.36|-3085.18| +2452623|49331|14217|75087|582303|4272|19041|10|105|1836|80|86.24|167.30|135.51|0.00|10840.80|6899.20|13384.00|867.26|0.00|10840.80|11708.06|3941.60| +2451505|43612|5258|56286|735915|6744|23173|2|294|1837|74|58.57|94.88|34.15|0.00|2527.10|4334.18|7021.12|50.54|0.00|2527.10|2577.64|-1807.08| +2451505|43612|7432|56286|735915|6744|23173|2|265|1837|83|71.82|93.36|72.82|0.00|6044.06|5961.06|7748.88|423.08|0.00|6044.06|6467.14|83.00| +2451505|43612|4078|56286|735915|6744|23173|2|57|1837|94|21.47|42.94|4.29|0.00|403.26|2018.18|4036.36|32.26|0.00|403.26|435.52|-1614.92| +2451505|43612|12044|56286|735915|6744|23173|2|157|1837|4|43.82|55.21|46.37|0.00|185.48|175.28|220.84|7.41|0.00|185.48|192.89|10.20| +2451505|43612|5110|56286|735915|6744|23173|2|288|1837|51|41.68|60.85|57.19|0.00|2916.69|2125.68|3103.35|204.16|0.00|2916.69|3120.85|791.01| +2451505|43612|16478|56286|735915|6744|23173|2|85|1837|64|38.33|58.26|6.99|0.00|447.36|2453.12|3728.64|0.00|0.00|447.36|447.36|-2005.76| +2451505|43612|11947|56286|735915|6744|23173|2|7|1837|24|37.91|61.41|23.94|120.65|574.56|909.84|1473.84|18.15|120.65|453.91|472.06|-455.93| +2451505|43612|13894|56286|735915|6744|23173|2|152|1837|33|58.58|76.15|44.92|0.00|1482.36|1933.14|2512.95|14.82|0.00|1482.36|1497.18|-450.78| +2451470||6716||267010|1094|44825|||1838|48||59.39|54.63|0.00|2622.24|||183.55|0.00||2805.79|| +2451470|35428|10363|80479|267010|1094|44825|8|233|1838|44|99.73|181.50|157.90|2014.80|6947.60|4388.12|7986.00|443.95|2014.80|4932.80|5376.75|544.68| +2451470|35428|916|80479|267010|1094|44825|8|58|1838|3|39.41|61.47|16.59|0.00|49.77|118.23|184.41|3.98|0.00|49.77|53.75|-68.46| +2451470|35428|11438|80479|267010|1094|44825|8|97|1838|48|45.62|52.46|47.73|1008.05|2291.04|2189.76|2518.08|38.48|1008.05|1282.99|1321.47|-906.77| +2451470|35428|13990|80479|267010|1094|44825|8|99|1838|39|22.47|37.97|11.77|0.00|459.03|876.33|1480.83|32.13|0.00|459.03|491.16|-417.30| +2451470|35428|3550|80479|267010|1094|44825|8|68|1838|98|2.09|3.84|0.57|13.96|55.86|204.82|376.32|0.41|13.96|41.90|42.31|-162.92| +2451470|35428|301|80479|267010|1094|44825|8|89|1838|56|90.53|96.86|48.43|0.00|2712.08|5069.68|5424.16|244.08|0.00|2712.08|2956.16|-2357.60| +|35428|9361||267010|1094|44825|||1838|24|14.25||11.34|||342.00|461.52|5.66||70.77|76.43|-271.23| +2451470|35428|12092|80479|267010|1094|44825|8|118|1838|71|84.65|110.04|78.12|4215.35|5546.52|6010.15|7812.84|13.31|4215.35|1331.17|1344.48|-4678.98| +2451470|35428|17083|80479|267010|1094|44825|8|15|1838|38|71.35|91.32|65.75|0.00|2498.50|2711.30|3470.16|24.98|0.00|2498.50|2523.48|-212.80| +2451442|67732|5047|27238|1326637|436|27788|1|41|1839|80|87.07|120.15|15.61|0.00|1248.80|6965.60|9612.00|99.90|0.00|1248.80|1348.70|-5716.80| +2451442|67732|11878|27238|1326637|436|27788|1|61|1839|17|87.04|163.63|127.63|0.00|2169.71|1479.68|2781.71|195.27|0.00|2169.71|2364.98|690.03| +2451442|67732|17668|27238|1326637|436|27788|1|52|1839|51|19.29|34.52|33.48|0.00|1707.48|983.79|1760.52|68.29|0.00|1707.48|1775.77|723.69| +2451442|67732|5161|27238|1326637|436|27788|1|220|1839|58|77.39|142.39|74.04|0.00|4294.32|4488.62|8258.62|42.94|0.00|4294.32|4337.26|-194.30| +2451442|67732|13453|27238|1326637|436|27788|1|173|1839|48|15.10|28.53|7.70|354.81|369.60|724.80|1369.44|0.00|354.81|14.79|14.79|-710.01| +2451442|67732|13646|27238|1326637|436|27788|1|181|1839|61|14.58|21.72|6.08|0.00|370.88|889.38|1324.92|11.12|0.00|370.88|382.00|-518.50| +2451442|67732|5390|27238|1326637|436|27788|1|251|1839|45|83.15|138.86|84.70|0.00|3811.50|3741.75|6248.70|38.11|0.00|3811.50|3849.61|69.75| +2451442|67732|2743|27238|1326637|436|27788|1|48|1839|97|33.48|43.85|19.29|0.00|1871.13|3247.56|4253.45|112.26|0.00|1871.13|1983.39|-1376.43| +2451442|67732|1495|27238|1326637|436|27788|1|24|1839|4|99.82|158.71|50.78|105.62|203.12|399.28|634.84|0.97|105.62|97.50|98.47|-301.78| +2451442|67732|14048|27238|1326637|436|27788|1|122|1839|100|79.30|104.67|3.14|166.42|314.00|7930.00|10467.00|10.33|166.42|147.58|157.91|-7782.42| +2451442|67732|9832|27238|1326637|436|27788|1|268|1839|91|46.80|57.56|17.26|0.00|1570.66|4258.80|5237.96|125.65|0.00|1570.66|1696.31|-2688.14| +2451442|67732|15044|27238|1326637|436|27788|1|266|1839|59|78.34|115.94|78.83|0.00|4650.97|4622.06|6840.46|186.03|0.00|4650.97|4837.00|28.91| +2451442|67732|8287|27238|1326637|436|27788|1|106|1839|95|90.93|160.94|11.26|106.97|1069.70|8638.35|15289.30|57.76|106.97|962.73|1020.49|-7675.62| +2451442|67732|11510|27238|1326637|436|27788|1|271|1839|47|73.54|125.75|116.94|0.00|5496.18|3456.38|5910.25|164.88|0.00|5496.18|5661.06|2039.80| +2451442|67732|5389|27238|1326637|436|27788|1|217|1839|91|58.43|97.57|90.74|0.00|8257.34|5317.13|8878.87|412.86|0.00|8257.34|8670.20|2940.21| +2452198|52663|10217|11855|56509|3563|47417|8|168|1840|67|97.91|180.15|169.34|0.00|11345.78|6559.97|12070.05|0.00|0.00|11345.78|11345.78|4785.81| +2452198|52663|2949|11855|56509|3563|47417|8|168|1840|7|69.39|93.67|26.22|0.00|183.54|485.73|655.69|5.50|0.00|183.54|189.04|-302.19| +2452198|52663|6939|11855|56509|3563|47417|8|108|1840|55|64.91|120.73|60.36|0.00|3319.80|3570.05|6640.15|33.19|0.00|3319.80|3352.99|-250.25| +2452198|52663|9905|11855|56509|3563|47417|8|281|1840|67|48.78|77.07|44.70|179.69|2994.90|3268.26|5163.69|225.21|179.69|2815.21|3040.42|-453.05| +2452198||6953||56509|||8||1840|57|72.30|117.84|7.07||402.99|4121.10|||||419.10|-3718.11| +2452198|52663|11039|11855|56509|3563|47417|8|182|1840|16|67.58|124.34|106.93|633.02|1710.88|1081.28|1989.44|64.67|633.02|1077.86|1142.53|-3.42| +2452198|52663|16797|11855|56509|3563|47417|8|4|1840|16|10.09|13.72|11.66|0.00|186.56|161.44|219.52|3.73|0.00|186.56|190.29|25.12| +2452198|52663|1551|11855|56509|3563|47417|8|142|1840|73|36.18|69.82|10.47|0.00|764.31|2641.14|5096.86|30.57|0.00|764.31|794.88|-1876.83| +2452505|51177|177|43580|514298|1816|48981|4|27|1841|36|39.59|54.63|10.37|0.00|373.32|1425.24|1966.68|33.59|0.00|373.32|406.91|-1051.92| +2452505|51177|16929|43580|514298|1816|48981|4|254|1841|88|84.85|101.82|70.25|0.00|6182.00|7466.80|8960.16|247.28|0.00|6182.00|6429.28|-1284.80| +2452505|51177|16741|43580|514298|1816|48981|4|276|1841|14|34.34|51.51|2.06|0.00|28.84|480.76|721.14|2.01|0.00|28.84|30.85|-451.92| +2452505|51177|11550|43580|514298|1816|48981|4|218|1841|89|37.36|63.88|61.96|0.00|5514.44|3325.04|5685.32|386.01|0.00|5514.44|5900.45|2189.40| +2452505|51177|11817|43580|514298|1816|48981|4|155|1841|82|29.16|50.44|37.32|0.00|3060.24|2391.12|4136.08|91.80|0.00|3060.24|3152.04|669.12| +2452505|51177|11653|43580|514298|1816|48981|4|10|1841|98|4.93|5.91|4.37|0.00|428.26|483.14|579.18|34.26|0.00|428.26|462.52|-54.88| +2452505|51177|12588|43580|514298|1816|48981|4|110|1841|86|75.42|78.43|54.90|0.00|4721.40|6486.12|6744.98|94.42|0.00|4721.40|4815.82|-1764.72| +2452505|51177|13069|43580|514298|1816|48981|4|5|1841|48|36.40|63.70|4.45|0.00|213.60|1747.20|3057.60|0.00|0.00|213.60|213.60|-1533.60| +2452247|39843|13137|85096|341974|3299|29377|10|41|1842|39|39.22|77.65|13.97|0.00|544.83|1529.58|3028.35|10.89|0.00|544.83|555.72|-984.75| +2452247|39843|10867|85096|341974|3299|29377|10|74|1842|57|49.17|94.89|86.34|0.00|4921.38|2802.69|5408.73|344.49|0.00|4921.38|5265.87|2118.69| +2452247|39843|17569|85096|341974|3299|29377|10|83|1842|80|63.78|87.37|62.90|0.00|5032.00|5102.40|6989.60|251.60|0.00|5032.00|5283.60|-70.40| +2452247|39843|7263|85096|341974|3299|29377|10|42|1842|45|29.60|39.07|17.97|0.00|808.65|1332.00|1758.15|8.08|0.00|808.65|816.73|-523.35| +2452247|39843|8233|85096|341974|3299|29377|10|53|1842|59|22.41|39.66|30.53|0.00|1801.27|1322.19|2339.94|144.10|0.00|1801.27|1945.37|479.08| +2452247|39843|271|85096|341974|3299|29377|10|26|1842|80|46.18|68.34|4.10|0.00|328.00|3694.40|5467.20|22.96|0.00|328.00|350.96|-3366.40| +||16591||||29377|10|23|1842|54||112.40|101.16||5462.64|5277.96|6069.60||||397.68|| +2452247|39843|17511|85096|341974|3299|29377|10|15|1842|81|81.56|123.97|111.57|0.00|9037.17|6606.36|10041.57|542.23|0.00|9037.17|9579.40|2430.81| +2451533|51223|1400|18079|1387117|602|46487|8|269|1843|66|98.54|178.35|62.42|0.00|4119.72|6503.64|11771.10|288.38|0.00|4119.72|4408.10|-2383.92| +2451533|51223|17410|18079|1387117|602|46487|8|269|1843|96|60.61|91.52|59.48|0.00|5710.08|5818.56|8785.92|171.30|0.00|5710.08|5881.38|-108.48| +2451533|51223|4892|18079|1387117|602|46487|8|142|1843|40|4.70|7.19|6.11|0.00|244.40|188.00|287.60|19.55|0.00|244.40|263.95|56.40| +2451533|51223|16592|18079|1387117|602|46487|8|209|1843|84|6.71|12.88|3.73|0.00|313.32|563.64|1081.92|12.53|0.00|313.32|325.85|-250.32| +2451533|51223|793|18079|1387117|602|46487|8|246|1843|61|92.41|127.52|21.67|0.00|1321.87|5637.01|7778.72|66.09|0.00|1321.87|1387.96|-4315.14| +2451533|51223|15817|18079|1387117|602|46487|8|234|1843|73|84.81|105.16|26.29|0.00|1919.17|6191.13|7676.68|95.95|0.00|1919.17|2015.12|-4271.96| +||17264||1387117|602||8|179|1843|12|83.69||35.40|0.00|424.80||1416.00||0.00|||-579.48| +2451533|51223|11444|18079|1387117|602|46487|8|68|1843|48|13.33|22.39|11.19|0.00|537.12|639.84|1074.72|10.74|0.00|537.12|547.86|-102.72| +2451533|51223|9808|18079|1387117|602|46487|8|30|1843|79|25.18|49.85|32.40|2329.23|2559.60|1989.22|3938.15|13.82|2329.23|230.37|244.19|-1758.85| +2451533|51223|16903|18079|1387117|602|46487|8|82|1843|76|89.75|90.64|17.22|0.00|1308.72|6821.00|6888.64|39.26|0.00|1308.72|1347.98|-5512.28| +2451062|54914|6356|84874|1254877|4880|38436|1|262|1844|45|91.49|147.29|141.39|0.00|6362.55|4117.05|6628.05|127.25|0.00|6362.55|6489.80|2245.50| +2451062|54914|7999|84874|1254877|4880|38436|1|159|1844|68|76.56|95.70|22.01|0.00|1496.68|5206.08|6507.60|119.73|0.00|1496.68|1616.41|-3709.40| +2451062|54914|12433|84874|1254877|4880|38436|1|61|1844|41|12.64|21.61|19.01|693.67|779.41|518.24|886.01|4.28|693.67|85.74|90.02|-432.50| +2451062|54914|5404|84874|1254877|4880|38436|1|222|1844|15|7.42|11.35|10.21|10.72|153.15|111.30|170.25|9.97|10.72|142.43|152.40|31.13| +2451062|54914|6580|84874|1254877|4880|38436|1|270|1844|70|65.48|80.54|10.47|0.00|732.90|4583.60|5637.80|7.32|0.00|732.90|740.22|-3850.70| +2451062|54914|4111|84874|1254877|4880|38436|1|123|1844|20|91.75|177.07|28.33|0.00|566.60|1835.00|3541.40|22.66|0.00|566.60|589.26|-1268.40| +2451062|54914|17419|84874|1254877|4880|38436|1|224|1844|59|67.53|96.56|3.86|0.00|227.74|3984.27|5697.04|0.00|0.00|227.74|227.74|-3756.53| +2451062|54914|13201|84874|1254877|4880|38436|1|294|1844|94|1.04|1.66|0.28|0.00|26.32|97.76|156.04|0.00|0.00|26.32|26.32|-71.44| +2451062|54914|727|84874|1254877|4880|38436|1|2|1844|52|70.23|139.05|133.48|3956.34|6940.96|3651.96|7230.60|59.69|3956.34|2984.62|3044.31|-667.34| +2451062|54914|11809|84874|1254877|4880|38436|1|85|1844|30|4.72|4.90|2.79|64.44|83.70|141.60|147.00|1.34|64.44|19.26|20.60|-122.34| +2451062|54914|2065|84874|1254877|4880|38436|1|271|1844|18|95.10|95.10|61.81|0.00|1112.58|1711.80|1711.80|44.50|0.00|1112.58|1157.08|-599.22| +2451062|54914|8492|84874|1254877|4880|38436|1|10|1844|7|76.16|106.62|59.70|0.00|417.90|533.12|746.34|0.00|0.00|417.90|417.90|-115.22| +2451062|54914|3662|84874|1254877|4880|38436|1|164|1844|73|76.97|121.61|65.66|2492.45|4793.18|5618.81|8877.53|92.02|2492.45|2300.73|2392.75|-3318.08| +2451062|54914|1453||||38436|||1844|||173.04|67.48||4453.68||11420.64|||||-1433.52| +2452187|43137|4437|76272|748142|5612|43882|2|296|1845|60|62.25|90.26|9.02|0.00|541.20|3735.00|5415.60|10.82|0.00|541.20|552.02|-3193.80| +2452187|43137|4197|76272|748142|5612|43882|2|126|1845|62|55.61|108.99|41.41|0.00|2567.42|3447.82|6757.38|102.69|0.00|2567.42|2670.11|-880.40| +2452187|43137|12001|76272|748142|5612|43882|2|194|1845|72|63.79|69.53|4.17|207.16|300.24|4592.88|5006.16|0.00|207.16|93.08|93.08|-4499.80| +2452187|43137|3791|76272|748142|5612|43882|2|81|1845|61|41.20|46.14|24.91|638.19|1519.51|2513.20|2814.54|17.62|638.19|881.32|898.94|-1631.88| +2452187|43137|11883|76272|748142|5612|43882|2|69|1845|30|65.76|128.88|65.72|0.00|1971.60|1972.80|3866.40|59.14|0.00|1971.60|2030.74|-1.20| +2452187|43137|15915|76272|748142|5612|43882|2|49|1845|63|86.61|108.26|47.63|0.00|3000.69|5456.43|6820.38|90.02|0.00|3000.69|3090.71|-2455.74| +2452187|43137|3299|76272|748142|5612|43882|2|82|1845|35|10.43|10.43|1.46|0.00|51.10|365.05|365.05|4.08|0.00|51.10|55.18|-313.95| +2452187|43137|4829|76272|748142|5612|43882|2|292|1845|72|59.34|99.69|85.73|0.00|6172.56|4272.48|7177.68|246.90|0.00|6172.56|6419.46|1900.08| +2452187|43137|685|76272|748142|5612|43882|2|51|1845|36|44.90|52.08|38.01|0.00|1368.36|1616.40|1874.88|82.10|0.00|1368.36|1450.46|-248.04| +2452187|43137|105|76272|748142|5612|43882|2|115|1845|94|12.33|24.53|20.35|0.00|1912.90|1159.02|2305.82|0.00|0.00|1912.90|1912.90|753.88| +2451182|49337|9088|62343|867862|4134|23653|4|107|1846|27|85.02|136.88|113.61|0.00|3067.47|2295.54|3695.76|30.67|0.00|3067.47|3098.14|771.93| +2451182|49337|4966|62343|867862|4134|23653|4|12|1846|44|6.45|12.38|0.49|0.00|21.56|283.80|544.72|0.64|0.00|21.56|22.20|-262.24| +2451182|49337|4876|62343|867862|4134|23653|4|232|1846|21|59.13|118.26|99.33|0.00|2085.93|1241.73|2483.46|62.57|0.00|2085.93|2148.50|844.20| +2451182|49337|2071|62343|867862|4134|23653|4|232|1846|68|56.49|70.04|16.10|0.00|1094.80|3841.32|4762.72|76.63|0.00|1094.80|1171.43|-2746.52| +2451182|49337|7154|62343|867862|4134|23653|4|72|1846|81|74.40|76.63|46.74|0.00|3785.94|6026.40|6207.03|265.01|0.00|3785.94|4050.95|-2240.46| +2451182|49337|12986|62343|867862|4134|23653|4|49|1846|57|89.31|118.78|13.06|0.00|744.42|5090.67|6770.46|0.00|0.00|744.42|744.42|-4346.25| +2451182|49337|5966|62343|867862|4134|23653|4|11|1846|40|27.65|36.49|11.67|196.05|466.80|1106.00|1459.60|5.41|196.05|270.75|276.16|-835.25| +2451182|49337|8575|62343|867862|4134|23653|4|158|1846|10|37.35|49.30|10.35|0.00|103.50|373.50|493.00|8.28|0.00|103.50|111.78|-270.00| +2451182|49337|6991|62343|867862|4134|23653|4|157|1846|6|40.04|51.25|9.22|0.00|55.32|240.24|307.50|4.42|0.00|55.32|59.74|-184.92| +2451182|49337|2318|62343|867862|4134|23653|4|214|1846|90|28.62|46.93|46.93|0.00|4223.70|2575.80|4223.70|211.18|0.00|4223.70|4434.88|1647.90| +2451182|49337|1328|62343|867862|4134|23653|4|257|1846|88|71.51|90.81|44.49|0.00|3915.12|6292.88|7991.28|352.36|0.00|3915.12|4267.48|-2377.76| +2451182|49337|7876|62343|867862|4134|23653|4|280|1846|39|36.74|67.23|2.01|0.00|78.39|1432.86|2621.97|4.70|0.00|78.39|83.09|-1354.47| +2451182|49337|7159|62343|867862|4134|23653|4|47|1846|51|50.10|88.17|63.48|0.00|3237.48|2555.10|4496.67|194.24|0.00|3237.48|3431.72|682.38| +2451182|49337|14012|62343|867862|4134|23653|4|157|1846|23|79.08|86.98|10.43|0.00|239.89|1818.84|2000.54|4.79|0.00|239.89|244.68|-1578.95| +2452632|48310|138|62478|1849899|6120|24341|1|191|1847|14|75.69|85.52|33.35|23.34|466.90|1059.66|1197.28|13.30|23.34|443.56|456.86|-616.10| +2452632|48310|1759|62478|1849899|6120|24341|1|45|1847|37|43.29|52.81|10.56|0.00|390.72|1601.73|1953.97|0.00|0.00|390.72|390.72|-1211.01| +2452632|48310|6681|62478|1849899|6120|24341|1|60|1847|6|14.13|15.40|4.62|0.00|27.72|84.78|92.40|0.00|0.00|27.72|27.72|-57.06| +2452632|48310|17211|62478|1849899|6120|24341|1|9|1847|43|19.45|34.62|7.61|0.00|327.23|836.35|1488.66|29.45|0.00|327.23|356.68|-509.12| +2452632|48310|10452|62478|1849899|6120|24341|1|133|1847|48|31.44|35.84|32.25|0.00|1548.00|1509.12|1720.32|139.32|0.00|1548.00|1687.32|38.88| +2452632|48310|1905|62478|1849899|6120|24341|1|243|1847|22|9.45|10.86|7.38|0.00|162.36|207.90|238.92|8.11|0.00|162.36|170.47|-45.54| +2452632|48310|11733|62478|1849899|6120|24341|1|233|1847|62|58.58|86.11|18.08|594.10|1120.96|3631.96|5338.82|0.00|594.10|526.86|526.86|-3105.10| +||12367|||6120|24341|1||1847|51||187.12|44.90||||9543.12|183.19|||2473.09|-2706.57| +2452632|48310|13038|62478|1849899|6120|24341|1|201|1847|2|13.24|21.97|9.22|0.00|18.44|26.48|43.94|0.36|0.00|18.44|18.80|-8.04| +2452632|48310|11359|62478|1849899|6120|24341|1|105|1847|73|67.86|97.03|49.48|0.00|3612.04|4953.78|7083.19|325.08|0.00|3612.04|3937.12|-1341.74| +2452586||8877||1541084|353|38469|||1848||||108.91||10564.27|5951.92|10891.16|||10564.27|10881.19|4612.35| +||11683|25015||353|38469|||1848|47|32.74|38.63|32.83|339.46|1543.01|1538.78|||339.46|1203.55||-335.23| +2452586|45594|2407|25015|1541084|353|38469|10|22|1848|54|47.41|82.96|47.28|1353.15|2553.12|2560.14|4479.84|83.99|1353.15|1199.97|1283.96|-1360.17| +2452586|45594|15522|25015|1541084|353|38469|10|220|1848|9|7.69|13.53|9.20|0.00|82.80|69.21|121.77|4.14|0.00|82.80|86.94|13.59| +2452586|45594|7071|25015|1541084|353|38469|10|187|1848|45|60.90|100.48|93.44|0.00|4204.80|2740.50|4521.60|294.33|0.00|4204.80|4499.13|1464.30| +2452586|45594|6105|25015|1541084|353|38469|10|55|1848|79|78.28|135.42|44.68|0.00|3529.72|6184.12|10698.18|141.18|0.00|3529.72|3670.90|-2654.40| +2452586|45594|14623|25015|1541084|353|38469|10|57|1848|87|52.36|78.01|31.20|2090.08|2714.40|4555.32|6786.87|49.94|2090.08|624.32|674.26|-3931.00| +2452586|45594|5856|25015|1541084|353|38469|10|12|1848|64|75.02|94.52|31.19|0.00|1996.16|4801.28|6049.28|39.92|0.00|1996.16|2036.08|-2805.12| +2452586|45594|17131|25015|1541084|353|38469|10|109|1848|98|39.47|48.94|41.59|0.00|4075.82|3868.06|4796.12|40.75|0.00|4075.82|4116.57|207.76| +2452586|45594|11997|25015|1541084|353|38469|10|54|1848|58|26.50|36.57|18.65|486.76|1081.70|1537.00|2121.06|47.59|486.76|594.94|642.53|-942.06| +2452035|62907|3853|90224|1088303|||||1849|95|||6.88|0.00||1715.70|1767.00||0.00||705.88|| +2452035|62907|14801|90224|1088303|3629|18108|10|80|1849|61|6.19|6.19|2.59|0.00|157.99|377.59|377.59|4.73|0.00|157.99|162.72|-219.60| +2452035|62907|9655|90224|1088303|3629|18108|10|111|1849|73|27.46|46.95|4.22|0.00|308.06|2004.58|3427.35|6.16|0.00|308.06|314.22|-1696.52| +2452035|62907|5511|90224|1088303|3629|18108|10|93|1849|77|73.09|90.63|42.59|0.00|3279.43|5627.93|6978.51|262.35|0.00|3279.43|3541.78|-2348.50| +2452035|62907|9393|90224|1088303|3629|18108|10|209|1849|46|59.47|62.44|41.21|0.00|1895.66|2735.62|2872.24|56.86|0.00|1895.66|1952.52|-839.96| +2452035|62907|15347|90224|1088303|3629|18108|10|204|1849|51|57.43|60.30|28.94|0.00|1475.94|2928.93|3075.30|0.00|0.00|1475.94|1475.94|-1452.99| +2452035|62907|17413|90224||||||1849|73|40.67|53.27||0.00|||||0.00||832.46|| +2452035|62907|3159|90224|1088303|3629|18108|10|241|1849|91|82.01|164.02|49.20|940.21|4477.20|7462.91|14925.82|282.95|940.21|3536.99|3819.94|-3925.92| +2452035|62907|5519|90224|1088303|3629|18108|10|38|1849|63|87.75|126.36|17.69|0.00|1114.47|5528.25|7960.68|100.30|0.00|1114.47|1214.77|-4413.78| +2452035|62907|6339|90224|1088303|3629|18108|10|190|1849|78|23.28|23.97|19.17|0.00|1495.26|1815.84|1869.66|59.81|0.00|1495.26|1555.07|-320.58| +2452035|62907|11563|90224|1088303|3629|18108|10|80|1849|91|33.35|66.36|50.43|0.00|4589.13|3034.85|6038.76|275.34|0.00|4589.13|4864.47|1554.28| +2451236|65165|5503|32297|575513|682|33098|1|120|1850|75|85.26|157.73|149.84|0.00|11238.00|6394.50|11829.75|224.76|0.00|11238.00|11462.76|4843.50| +2451236|65165|9979|32297|575513|682|33098|1|208|1850|59|53.17|91.98|55.18|0.00|3255.62|3137.03|5426.82|97.66|0.00|3255.62|3353.28|118.59| +2451236|65165|17990|32297|575513|682|33098|1|169|1850|58|44.53|69.46|24.31|0.00|1409.98|2582.74|4028.68|84.59|0.00|1409.98|1494.57|-1172.76| +2451236|65165|4736|32297|575513|682|33098|1|256|1850|79|19.46|21.60|1.72|43.48|135.88|1537.34|1706.40|7.39|43.48|92.40|99.79|-1444.94| +2451236|65165|10720|32297|575513|682|33098|1|21|1850|81|47.77|87.89|11.42|0.00|925.02|3869.37|7119.09|27.75|0.00|925.02|952.77|-2944.35| +|65165|1741|||682|33098||110|1850|||160.04|67.21|||1341.15||||1008.15|1078.72|| +|65165|4316||575513||||118|1850|||||||125.57|232.29|11.41|||201.65|64.67| +2451236|65165|3820|32297|575513|682|33098|1|92|1850|82|4.95|6.58|6.51|0.00|533.82|405.90|539.56|5.33|0.00|533.82|539.15|127.92| +2451236|65165|2792|32297|575513|682|33098|1|28|1850|76|84.41|141.80|11.34|0.00|861.84|6415.16|10776.80|68.94|0.00|861.84|930.78|-5553.32| +2451236|65165|6922|32297|575513|682|33098|1|23|1850|16|33.37|44.38|16.86|159.15|269.76|533.92|710.08|9.95|159.15|110.61|120.56|-423.31| +2451236|65165|13856|32297|575513|682|33098|1|179|1850|30|94.95|145.27|45.03|0.00|1350.90|2848.50|4358.10|40.52|0.00|1350.90|1391.42|-1497.60| +2451073|47700|3115|85745|717693|6441|6742|7|124|1851|83|44.47|84.93|43.31|0.00|3594.73|3691.01|7049.19|71.89|0.00|3594.73|3666.62|-96.28| +2451073|47700|13664|85745|717693|6441|6742|7|194|1851|41|47.00|56.40|45.68|0.00|1872.88|1927.00|2312.40|112.37|0.00|1872.88|1985.25|-54.12| +2451073|47700|10244|85745|717693|6441|6742|7|188|1851|55|85.82|113.28|100.81|0.00|5544.55|4720.10|6230.40|110.89|0.00|5544.55|5655.44|824.45| +2451073|47700|11497|85745|717693|6441|6742|7|245|1851|90|45.58|75.20|29.32|1899.93|2638.80|4102.20|6768.00|14.77|1899.93|738.87|753.64|-3363.33| +2451073|47700|10192|85745|717693|6441|6742|7|143|1851|39|20.13|37.44|19.46|0.00|758.94|785.07|1460.16|37.94|0.00|758.94|796.88|-26.13| +2451073|47700|10399|85745|717693|6441|6742|7|204|1851|26|38.77|43.03|32.27|0.00|839.02|1008.02|1118.78|58.73|0.00|839.02|897.75|-169.00| +2451073|47700|10135|85745|717693|6441|6742|7|269|1851|96|41.48|51.02|48.46|0.00|4652.16|3982.08|4897.92|418.69|0.00|4652.16|5070.85|670.08| +2451073|47700|3745|85745|717693|6441|6742|7|203|1851|17|63.31|110.79|79.76|0.00|1355.92|1076.27|1883.43|122.03|0.00|1355.92|1477.95|279.65| +2451073|47700|7471|85745|717693|6441|6742|7|285|1851|74|72.51|74.68|53.76|2307.37|3978.24|5365.74|5526.32|33.41|2307.37|1670.87|1704.28|-3694.87| +2451073|47700|14128|85745|717693|6441|6742|7|114|1851|18|20.89|29.03|11.61|0.00|208.98|376.02|522.54|18.80|0.00|208.98|227.78|-167.04| +2451073|47700|15043|85745|717693|6441|6742|7|275|1851|22|62.71|75.87|36.41|0.00|801.02|1379.62|1669.14|16.02|0.00|801.02|817.04|-578.60| +2451073|47700|16891|85745|717693|6441|6742|7|53|1851|57|52.56|79.36|73.01|0.00|4161.57|2995.92|4523.52|83.23|0.00|4161.57|4244.80|1165.65| +2451073|47700|3674|85745|717693|6441|6742|7|6|1851|45|15.60|30.57|12.53|0.00|563.85|702.00|1375.65|5.63|0.00|563.85|569.48|-138.15| +2451073|47700|5524|85745|717693|6441|6742|7|59|1851|99|8.13|11.30|0.11|0.00|10.89|804.87|1118.70|0.00|0.00|10.89|10.89|-793.98| +2451073|47700|6200|85745|717693|6441|6742|7|187|1851|87|25.28|46.76|16.83|0.00|1464.21|2199.36|4068.12|102.49|0.00|1464.21|1566.70|-735.15| +2451399|53563|1576|83125|1122753|6381|29656|2|196|1852|8|46.62|68.53|50.02|0.00|400.16|372.96|548.24|28.01|0.00|400.16|428.17|27.20| +2451399|53563|6988|83125|1122753|6381|29656|2|192|1852|80|7.12|11.67|7.35|399.84|588.00|569.60|933.60|11.28|399.84|188.16|199.44|-381.44| +2451399|53563|11948|83125|1122753|6381|29656|2|235|1852|64|43.06|85.25|76.72|2602.34|4910.08|2755.84|5456.00|46.15|2602.34|2307.74|2353.89|-448.10| +2451399|53563|3679|83125|1122753|6381|29656|2|285|1852|89|45.72|53.49|9.09|0.00|809.01|4069.08|4760.61|72.81|0.00|809.01|881.82|-3260.07| +2451399|53563|3994|83125|1122753|6381|29656|2|260|1852|2|56.15|80.85|50.12|0.00|100.24|112.30|161.70|8.01|0.00|100.24|108.25|-12.06| +2451399|53563|9796|83125|1122753|6381|29656|2|252|1852|15|8.81|11.10|0.99|12.77|14.85|132.15|166.50|0.16|12.77|2.08|2.24|-130.07| +2451399|53563|8810|83125|1122753|6381|29656|2|109|1852|94|41.43|50.95|18.34|0.00|1723.96|3894.42|4789.30|68.95|0.00|1723.96|1792.91|-2170.46| +2451399|53563|17102|83125|1122753|6381|29656|2|177|1852|97|81.29|114.61|88.24|4964.38|8559.28|7885.13|11117.17|0.00|4964.38|3594.90|3594.90|-4290.23| +2451399|53563|14680|83125|1122753|6381|29656|2|249|1852|11|11.57|12.84|6.42|0.00|70.62|127.27|141.24|6.35|0.00|70.62|76.97|-56.65| +2451399|53563|8792|83125|1122753|6381|29656|2|244|1852|91|32.08|58.06|51.09|0.00|4649.19|2919.28|5283.46|232.45|0.00|4649.19|4881.64|1729.91| +2451399|53563|12728|83125|1122753|6381|29656|2|57|1852|69|79.75|153.91|152.37|0.00|10513.53|5502.75|10619.79|841.08|0.00|10513.53|11354.61|5010.78| +2451399|53563|6854|83125|1122753|6381|29656|2|282|1852|81|69.41|123.54|101.30|0.00|8205.30|5622.21|10006.74|328.21|0.00|8205.30|8533.51|2583.09| +2451399|53563|15577|83125|1122753|6381|29656|2|292|1852|48|38.49|56.96|30.75|1136.52|1476.00|1847.52|2734.08|0.00|1136.52|339.48|339.48|-1508.04| +2451399|53563|16280|83125|1122753|6381|29656|2|264|1852|58|70.89|102.08|73.49|554.11|4262.42|4111.62|5920.64|0.00|554.11|3708.31|3708.31|-403.31| +2451399|53563|5731|83125|1122753|6381|29656|2|2|1852|73|27.54|33.04|32.37|0.00|2363.01|2010.42|2411.92|94.52|0.00|2363.01|2457.53|352.59| +2451399|53563|16642|83125|1122753|6381|29656|2|260|1852|30|93.76|125.63|43.97|0.00|1319.10|2812.80|3768.90|39.57|0.00|1319.10|1358.67|-1493.70| +2451126|34072|13880|97228|1472396|4811|45979|10|236|1853|43|49.49|62.35|56.73|0.00|2439.39|2128.07|2681.05|146.36|0.00|2439.39|2585.75|311.32| +2451126|34072|9811|97228|1472396|4811|45979|10|275|1853|63|40.61|77.56|38.00|0.00|2394.00|2558.43|4886.28|0.00|0.00|2394.00|2394.00|-164.43| +2451126|34072|3932|97228|1472396|4811|45979|10|54|1853|1|57.81|84.40|56.54|0.00|56.54|57.81|84.40|3.39|0.00|56.54|59.93|-1.27| +2451126|34072|5863|97228|1472396|4811|45979|10|294|1853|4|59.80|118.40|43.80|164.68|175.20|239.20|473.60|0.31|164.68|10.52|10.83|-228.68| +2451126|34072|4357|97228|1472396|4811|45979|10|280|1853|10|55.66|71.24|46.30|0.00|463.00|556.60|712.40|23.15|0.00|463.00|486.15|-93.60| +2451126|34072|15446|97228|1472396|4811|45979|10|179|1853|55|33.82|42.27|33.39|0.00|1836.45|1860.10|2324.85|73.45|0.00|1836.45|1909.90|-23.65| +2451126|34072|3433|97228|1472396|4811|45979|10|186|1853|24|24.38|35.35|4.59|0.00|110.16|585.12|848.40|6.60|0.00|110.16|116.76|-474.96| +|34072|754|97228|1472396|4811|45979|10||1853|17|9.46|12.10|4.59||78.03|||5.46|||83.49|| +2451126|34072|1424|97228|1472396|4811|45979|10|284|1853|83|52.45|60.31|40.40|435.91|3353.20|4353.35|5005.73|262.55|435.91|2917.29|3179.84|-1436.06| +2451126|34072|1510|97228|1472396|4811|45979|10||1853|37||14.25||226.44|226.44|366.30|||226.44||0.00|-366.30| +2451126|34072|13802|97228|1472396|4811|45979|10|296|1853|59|48.52|64.53|34.20|0.00|2017.80|2862.68|3807.27|20.17|0.00|2017.80|2037.97|-844.88| +2451126|34072|12568|97228|1472396|4811|45979|10|106|1853|72|4.60|7.68|1.76|0.00|126.72|331.20|552.96|5.06|0.00|126.72|131.78|-204.48| +2451126|34072|17072|97228|1472396|4811|45979|10|269|1853|82|91.86|151.56|151.56|7456.75|12427.92|7532.52|12427.92|49.71|7456.75|4971.17|5020.88|-2561.35| +2451665|50113|2599|70578|1659628|5925|11422|4|104|1854|100|84.66|130.37|53.45|0.00|5345.00|8466.00|13037.00|160.35|0.00|5345.00|5505.35|-3121.00| +2451665|50113|467|70578|1659628|5925|11422|4|220|1854|48|42.07|76.98|17.70|0.00|849.60|2019.36|3695.04|59.47|0.00|849.60|909.07|-1169.76| +2451665|50113|110|70578|1659628|5925|11422|4|175|1854|69|15.33|22.99|19.77|368.31|1364.13|1057.77|1586.31|49.79|368.31|995.82|1045.61|-61.95| +2451665|50113|7921|70578|1659628|5925|11422|4|216|1854|38|92.06|166.62|56.65|1872.84|2152.70|3498.28|6331.56|13.99|1872.84|279.86|293.85|-3218.42| +2451665|50113|15980|70578|1659628|5925|11422|4|41|1854|75|71.18|97.51|89.70|3027.37|6727.50|5338.50|7313.25|0.00|3027.37|3700.13|3700.13|-1638.37| +2451665|50113|8732|70578|1659628|5925|11422|4|108|1854|48|42.70|45.68|3.19|0.00|153.12|2049.60|2192.64|7.65|0.00|153.12|160.77|-1896.48| +2451665|50113|614|70578|1659628|5925|11422|4|132|1854|78|15.44|16.98|16.98|0.00|1324.44|1204.32|1324.44|79.46|0.00|1324.44|1403.90|120.12| +2451665|50113|17773|70578|1659628|5925|11422|4|148|1854|20|25.33|26.08|23.21|0.00|464.20|506.60|521.60|27.85|0.00|464.20|492.05|-42.40| +2451665|50113|15409|70578|1659628|5925|11422|4|233|1854|79|92.99|149.71|52.39|2317.73|4138.81|7346.21|11827.09|36.42|2317.73|1821.08|1857.50|-5525.13| +2451665|50113|11767|70578|1659628|5925|11422|4|97|1854|63|81.82|149.73|64.38|0.00|4055.94|5154.66|9432.99|0.00|0.00|4055.94|4055.94|-1098.72| +2451665|50113|16526|70578|1659628|5925|11422|4|231|1854|84|65.53|68.80|18.57|0.00|1559.88|5504.52|5779.20|109.19|0.00|1559.88|1669.07|-3944.64| +2451665|50113|14504|70578|1659628|5925|11422|4|222|1854|20|73.50|91.87|64.30|0.00|1286.00|1470.00|1837.40|115.74|0.00|1286.00|1401.74|-184.00| +2451665|50113|10865|70578|1659628|5925|11422|4|21|1854|100|91.68|148.52|136.63|0.00|13663.00|9168.00|14852.00|273.26|0.00|13663.00|13936.26|4495.00| +2451665|50113|11204|70578|1659628|5925|11422|4|102|1854|96|61.43|66.34|41.13|0.00|3948.48|5897.28|6368.64|39.48|0.00|3948.48|3987.96|-1948.80| +2451665|50113|14246|70578|1659628|5925|11422|4|32|1854|63|11.20|20.49|7.99|0.00|503.37|705.60|1290.87|20.13|0.00|503.37|523.50|-202.23| +2451665|50113|7475|70578|1659628|5925|11422|4|110|1854|77|60.26|78.94|67.88|0.00|5226.76|4640.02|6078.38|209.07|0.00|5226.76|5435.83|586.74| +2451805|43384|3983|69924|1134927|6683|36439|2|67|1855|16|96.17|190.41|45.69|0.00|731.04|1538.72|3046.56|58.48|0.00|731.04|789.52|-807.68| +2451805|43384|12817|69924|1134927|6683|36439|2|106|1855|100|9.48|12.32|8.25|0.00|825.00|948.00|1232.00|24.75|0.00|825.00|849.75|-123.00| +2451805|43384|15041|69924|1134927|6683|36439|2|257|1855|91|12.04|13.12|9.44|395.15|859.04|1095.64|1193.92|23.19|395.15|463.89|487.08|-631.75| +2451805|43384|17141|69924|1134927|6683|36439|2|140|1855|11|33.66|60.25|1.20|0.00|13.20|370.26|662.75|1.05|0.00|13.20|14.25|-357.06| +2451805|43384|1915|69924|1134927|6683|36439|2|222|1855|92|33.48|43.52|2.17|0.00|199.64|3080.16|4003.84|3.99|0.00|199.64|203.63|-2880.52| +2451805|43384|16705|69924|1134927|6683|36439|2|227|1855|51|37.53|55.91|48.64|0.00|2480.64|1914.03|2851.41|223.25|0.00|2480.64|2703.89|566.61| +2451805|43384|5948|69924|1134927|6683|36439|2|35|1855|71|39.67|49.98|26.98|0.00|1915.58|2816.57|3548.58|38.31|0.00|1915.58|1953.89|-900.99| +2451805||6781||||36439|2||1855||99.20|191.45|122.52|||7539.20|14550.20|78.21||3910.84||-3628.36| +2451805|43384|3758|69924|1134927|6683|36439|2|228|1855|23|73.44|141.00|43.71|0.00|1005.33|1689.12|3243.00|0.00|0.00|1005.33|1005.33|-683.79| +2451805|43384|10610|69924|1134927|6683|36439|2|202|1855|100|19.29|21.79|14.81|0.00|1481.00|1929.00|2179.00|29.62|0.00|1481.00|1510.62|-448.00| +2451805|43384|11927|69924|1134927|6683|36439|2|239|1855|20|50.28|57.31|5.15|0.00|103.00|1005.60|1146.20|6.18|0.00|103.00|109.18|-902.60| +2452594|75013|3739|98602|1805208|1061|13667|10|91|1856|71|41.27|74.69|47.80|0.00|3393.80|2930.17|5302.99|0.00|0.00|3393.80|3393.80|463.63| +2452594|75013|9273|98602|1805208|1061|13667|10|195|1856|5|90.47|167.36|8.36|0.00|41.80|452.35|836.80|0.83|0.00|41.80|42.63|-410.55| +2452594|75013|3924|98602|1805208|1061|13667|10|193|1856|11|78.73|139.35|97.54|0.00|1072.94|866.03|1532.85|53.64|0.00|1072.94|1126.58|206.91| +2452594|75013|9633|98602|1805208|1061|13667|10|12|1856|2|4.87|6.62|4.10|0.00|8.20|9.74|13.24|0.57|0.00|8.20|8.77|-1.54| +2452594|75013|12786|98602|1805208|1061|13667|10|247|1856|65|54.09|82.21|68.23|0.00|4434.95|3515.85|5343.65|354.79|0.00|4434.95|4789.74|919.10| +2452594|75013|9738|98602|1805208|1061|13667|10|76|1856|70|20.38|31.99|8.63|422.87|604.10|1426.60|2239.30|12.68|422.87|181.23|193.91|-1245.37| +2452594|75013|1015|98602|1805208|1061|13667|10|129|1856|82|87.38|170.39|81.78|0.00|6705.96|7165.16|13971.98|268.23|0.00|6705.96|6974.19|-459.20| +2452594|75013|16431|98602|1805208|1061|13667|10|226|1856|7|30.45|35.93|1.79|0.00|12.53|213.15|251.51|0.62|0.00|12.53|13.15|-200.62| +2452589|44807|14652|17485|1633395|6105|21968|10|160|1857|55|91.23|166.95|31.72|0.00|1744.60|5017.65|9182.25|69.78|0.00|1744.60|1814.38|-3273.05| +2452589|44807|13806|17485|1633395|6105|21968|10|52|1857|46|27.63|33.70|0.33|0.00|15.18|1270.98|1550.20|0.91|0.00|15.18|16.09|-1255.80| +|44807|3888|17485||6105||10|6|1857||||44.32|||2400.00|4728.00|||||-2038.34| +2452589|44807|8889|17485|1633395|6105|21968|10|152|1857|14|68.82|130.75|70.60|820.37|988.40|963.48|1830.50|0.00|820.37|168.03|168.03|-795.45| +2452589|44807|16855|17485|1633395|6105|21968|10|89|1857|86|68.60|80.94|41.27|0.00|3549.22|5899.60|6960.84|106.47|0.00|3549.22|3655.69|-2350.38| +2452589|44807|8641|17485|1633395|6105|21968|10|278|1857|63|26.41|51.23|18.44|894.52|1161.72|1663.83|3227.49|5.34|894.52|267.20|272.54|-1396.63| +2452589|44807|13053||1633395|6105|||132|1857||||83.10|||||||4819.80||-370.04| +2452589||6282|17485|||21968|||1857|||||0.00|1032.57|9480.24|10332.63|30.97|0.00|||| +2452258|37495|9117|1897|1448336|6328|27167|4|187|1858|83|98.97|131.63|78.97|0.00|6554.51|8214.51|10925.29|589.90|0.00|6554.51|7144.41|-1660.00| +2452258|37495|3941|1897|1448336|6328|27167|4|37|1858|55|83.09|103.03|46.36|0.00|2549.80|4569.95|5666.65|127.49|0.00|2549.80|2677.29|-2020.15| +2452258|37495|12337|1897|1448336|6328|27167|4|67|1858|34|67.58|79.06|27.67|0.00|940.78|2297.72|2688.04|65.85|0.00|940.78|1006.63|-1356.94| +2452258|37495|1309|1897|1448336|6328|27167|4|136|1858|24|72.13|132.71|9.28|0.00|222.72|1731.12|3185.04|20.04|0.00|222.72|242.76|-1508.40| +2452258|37495|15081|1897|1448336|6328|27167|4|213|1858|88|56.16|94.34|55.66|0.00|4898.08|4942.08|8301.92|0.00|0.00|4898.08|4898.08|-44.00| +2452258|37495|3781|1897|1448336|6328|27167|4|88|1858|63|69.26|108.73|50.01|0.00|3150.63|4363.38|6849.99|94.51|0.00|3150.63|3245.14|-1212.75| +2452258|37495|1783|1897|1448336|6328|27167|4|22|1858|16|57.82|88.46|66.34|764.23|1061.44|925.12|1415.36|2.97|764.23|297.21|300.18|-627.91| +2452258|37495|4629|1897|1448336|6328|27167|4|130|1858|26|42.87|80.16|72.94|0.00|1896.44|1114.62|2084.16|37.92|0.00|1896.44|1934.36|781.82| +2452258|37495|7773|1897|1448336|6328|27167|4|221|1858|46|56.83|82.97|67.20|463.68|3091.20|2614.18|3816.62|183.92|463.68|2627.52|2811.44|13.34| +2452600||7101||1414987||25691|2||1859|89|||||93.45|1625.14||0.00||93.45|93.45|| +2452600|50053|11634|94988|1414987|3339|25691|2|189|1859|38|72.04|111.66|56.94|0.00|2163.72|2737.52|4243.08|129.82|0.00|2163.72|2293.54|-573.80| +2452600|50053|15486|94988|1414987|3339|25691|2|235|1859|45|89.21|144.52|122.84|0.00|5527.80|4014.45|6503.40|497.50|0.00|5527.80|6025.30|1513.35| +2452600|50053|12420|94988|1414987|3339|25691|2|63|1859|41|27.90|32.92|1.97|0.00|80.77|1143.90|1349.72|0.80|0.00|80.77|81.57|-1063.13| +2452600|50053|13308|94988|1414987|3339|25691|2|107|1859|93|92.91|185.82|165.37|0.00|15379.41|8640.63|17281.26|615.17|0.00|15379.41|15994.58|6738.78| +2452600|50053|17067|94988|1414987|3339|25691|2|70|1859|65|1.24|2.03|1.11|0.00|72.15|80.60|131.95|0.72|0.00|72.15|72.87|-8.45| +2452600|50053|10323|94988|1414987|3339|25691|2|38|1859|73|47.45|50.29|29.67|0.00|2165.91|3463.85|3671.17|151.61|0.00|2165.91|2317.52|-1297.94| +2452600|50053|12993|94988|1414987|3339|25691|2|230|1859|34|85.48|111.97|43.66|133.59|1484.44|2906.32|3806.98|27.01|133.59|1350.85|1377.86|-1555.47| +2452600|50053|540|94988|1414987|3339|25691|2|166|1859|25|77.25|128.23|117.97|0.00|2949.25|1931.25|3205.75|206.44|0.00|2949.25|3155.69|1018.00| +2452600|50053|1119|94988|||25691|2||1859|23|26.28|40.73|32.99|||604.44||45.52||||| +2452600|50053|17610|94988|1414987|3339|25691|2|51|1859|41|11.34|15.42|0.92|0.00|37.72|464.94|632.22|0.37|0.00|37.72|38.09|-427.22| +2452600|50053|4008|94988|1414987|3339|25691|2|141|1859|4|55.69|86.31|73.36|0.00|293.44|222.76|345.24|14.67|0.00|293.44|308.11|70.68| +2452600|50053|12103|94988|1414987|3339|25691|2|249|1859|9|72.94|77.31|68.03|0.00|612.27|656.46|695.79|6.12|0.00|612.27|618.39|-44.19| +2451861|38911|14663|76976|1590920|864|4408|7|182|1860|100|26.06|48.47|15.51|0.00|1551.00|2606.00|4847.00|46.53|0.00|1551.00|1597.53|-1055.00| +2451861|38911|6170|76976|1590920|864|4408|7|58|1860|75|79.04|110.65|0.00|0.00|0.00|5928.00|8298.75|0.00|0.00|0.00|0.00|-5928.00| +2451861|38911|16597|76976|1590920|864|4408|7|264|1860|86|79.44|93.73|79.67|0.00|6851.62|6831.84|8060.78|137.03|0.00|6851.62|6988.65|19.78| +2451861|38911|5294|76976|1590920|864|4408|7|198|1860|2|55.18|108.15|102.74|0.00|205.48|110.36|216.30|0.00|0.00|205.48|205.48|95.12| +2451861|38911|1910|76976|1590920|864|4408|7|168|1860|33|74.79|143.59|20.10|0.00|663.30|2468.07|4738.47|46.43|0.00|663.30|709.73|-1804.77| +2451861|38911|15032|76976|1590920|864|4408|7|56|1860|29|79.94|140.69|63.31|0.00|1835.99|2318.26|4080.01|0.00|0.00|1835.99|1835.99|-482.27| +2451861|38911|1325|76976|1590920|864|4408|7|287|1860|64|89.97|151.14|69.52|0.00|4449.28|5758.08|9672.96|0.00|0.00|4449.28|4449.28|-1308.80| +2451861|38911|5497|76976|1590920|864|4408|7|60|1860|90|83.50|116.06|26.69|0.00|2402.10|7515.00|10445.40|120.10|0.00|2402.10|2522.20|-5112.90| +2451861|38911|7973|76976|1590920|864|4408|7|231|1860|51|17.93|30.66|12.87|269.11|656.37|914.43|1563.66|11.61|269.11|387.26|398.87|-527.17| +2451861|38911|3110|76976|1590920|864|4408|7|40|1860|35|98.93|148.39|129.09|0.00|4518.15|3462.55|5193.65|225.90|0.00|4518.15|4744.05|1055.60| +2451861|38911|15788|76976|1590920|864|4408|7|245|1860|43|46.19|90.99|53.68|0.00|2308.24|1986.17|3912.57|184.65|0.00|2308.24|2492.89|322.07| +2451861|38911|781|76976|1590920|864|4408|7|131|1860|30|7.07|10.11|2.32|13.92|69.60|212.10|303.30|0.55|13.92|55.68|56.23|-156.42| +2451861|38911|15791|76976|1590920|864|4408|7|230|1860|95|13.89|22.91|12.60|0.00|1197.00|1319.55|2176.45|47.88|0.00|1197.00|1244.88|-122.55| +2451861|38911|4601|76976|1590920|864|4408|7|266|1860|1|54.85|58.68|31.10|29.23|31.10|54.85|58.68|0.00|29.23|1.87|1.87|-52.98| +||14978||1590920||4408|7|107|1860||99.44||||1933.75|8452.40|14875.85|6.76||||| +2451861|38911|127|76976|1590920|864|4408|7|110|1860|56|93.36|160.57|41.74|0.00|2337.44|5228.16|8991.92|0.00|0.00|2337.44|2337.44|-2890.72| +2452596|42729|14652|92539|654150|4114|48125|1|206|1861|58|47.24|67.08|23.47|0.00|1361.26|2739.92|3890.64|108.90|0.00|1361.26|1470.16|-1378.66| +2452596|42729|13806|92539|654150|4114|48125|1|210|1861|75|85.76|152.65|102.27|0.00|7670.25|6432.00|11448.75|383.51|0.00|7670.25|8053.76|1238.25| +2452596|42729|3888|92539|654150|4114|48125|1|232|1861|62|50.16|70.72|24.04|0.00|1490.48|3109.92|4384.64|89.42|0.00|1490.48|1579.90|-1619.44| +2452596|42729|8889|92539|654150|4114|48125|1|92|1861|4|33.64|39.35|19.28|0.00|77.12|134.56|157.40|3.85|0.00|77.12|80.97|-57.44| +2452596|42729|16855|92539|654150|4114|48125|1|179|1861|15|7.18|12.63|2.14|20.22|32.10|107.70|189.45|0.59|20.22|11.88|12.47|-95.82| +2452596|42729|8641|92539|654150|4114|48125|1|3|1861|29|81.43|81.43|25.24|0.00|731.96|2361.47|2361.47|29.27|0.00|731.96|761.23|-1629.51| +2452596|42729|13053|92539|654150|4114|48125|1|104|1861|72|5.34|7.52|3.30|0.00|237.60|384.48|541.44|2.37|0.00|237.60|239.97|-146.88| +|42729|6282||||48125|1|58|1861|44|41.04||31.34|0.00|||2058.32|96.52|0.00|1378.96|1475.48|-426.80| +2452596|42729|12453|92539|654150|4114|48125|1|229|1861|62|64.07|121.73|99.81|0.00|6188.22|3972.34|7547.26|61.88|0.00|6188.22|6250.10|2215.88| +2452596|42729|3669|92539|654150|4114|48125|1|172|1861|84|3.70|7.06|5.36|0.00|450.24|310.80|593.04|4.50|0.00|450.24|454.74|139.44| +2452596||17481|||4114|48125|1|42|1861|93|||45.19||4202.67||7004.76|||4202.67|4286.72|| +2452596|42729|14061|92539|654150|4114|48125|1|3|1861|49|22.87|29.27|11.12|0.00|544.88|1120.63|1434.23|10.89|0.00|544.88|555.77|-575.75| +2452596|42729|5886|92539|654150|4114|48125|1|162|1861|19|72.57|100.14|69.09|0.00|1312.71|1378.83|1902.66|78.76|0.00|1312.71|1391.47|-66.12| +2452596|42729|6211|92539|654150|4114|48125|1|103|1861|93|41.70|72.14|61.31|0.00|5701.83|3878.10|6709.02|171.05|0.00|5701.83|5872.88|1823.73| +2451062|58024|11431|7019|996085|4351|7464|2|182|1862|21|89.73|117.54|41.13|0.00|863.73|1884.33|2468.34|17.27|0.00|863.73|881.00|-1020.60| +2451062|58024|12691|7019|996085|4351|7464|2|143|1862|31|50.83|56.42|48.52|0.00|1504.12|1575.73|1749.02|75.20|0.00|1504.12|1579.32|-71.61| +2451062|58024|17558|7019|996085|4351|7464|2|58|1862|81|78.89|125.43|109.12|2828.39|8838.72|6390.09|10159.83|540.92|2828.39|6010.33|6551.25|-379.76| +2451062|58024|13628|7019|996085|4351|7464|2|158|1862|81|80.46|114.25|86.83|4712.26|7033.23|6517.26|9254.25|162.46|4712.26|2320.97|2483.43|-4196.29| +2451062|58024|13519|7019|996085|4351|7464|2|215|1862|56|76.23|110.53|32.05|0.00|1794.80|4268.88|6189.68|161.53|0.00|1794.80|1956.33|-2474.08| +2451062|58024|13982|7019|996085|4351|7464|2|193|1862|77|64.69|124.20|98.11|0.00|7554.47|4981.13|9563.40|528.81|0.00|7554.47|8083.28|2573.34| +2451062|58024|1831|7019|996085|4351|7464|2|130|1862|3|36.24|44.21|23.87|0.00|71.61|108.72|132.63|2.14|0.00|71.61|73.75|-37.11| +2451062|58024|10960|7019|996085|4351|7464|2|224|1862|63|95.81|159.04|104.96|0.00|6612.48|6036.03|10019.52|0.00|0.00|6612.48|6612.48|576.45| +2451062||1852||||7464|||1862|40|||92.27|0.00||3805.60|||0.00|3690.80||| +2451062|58024|13672|7019|996085|4351|7464|2|44|1862|3|71.28|76.98|38.49|93.53|115.47|213.84|230.94|0.43|93.53|21.94|22.37|-191.90| +2451062|58024|9928|7019|996085|4351|7464|2|162|1862|8|74.56|110.34|101.51|0.00|812.08|596.48|882.72|24.36|0.00|812.08|836.44|215.60| +2451062|58024|3368|7019|996085|4351|7464|2|84|1862|34|17.78|31.47|14.16|0.00|481.44|604.52|1069.98|43.32|0.00|481.44|524.76|-123.08| +2451062|58024|15806|7019|996085|4351|7464|2|44|1862|88|63.30|93.68|79.62|4764.46|7006.56|5570.40|8243.84|89.68|4764.46|2242.10|2331.78|-3328.30| +2451062|58024|5395|7019|996085|4351|7464|2|74|1862|82|80.10|132.96|79.77|0.00|6541.14|6568.20|10902.72|130.82|0.00|6541.14|6671.96|-27.06| +2451062|58024|7000|7019|996085|4351|7464|2|40|1862|18|96.87|104.61|57.53|0.00|1035.54|1743.66|1882.98|93.19|0.00|1035.54|1128.73|-708.12| +2451062|58024|1585|7019|996085|4351|7464|2|175|1862|49|42.77|70.57|50.81|423.24|2489.69|2095.73|3457.93|41.32|423.24|2066.45|2107.77|-29.28| +2451648|49934|12788|27861|494541|6821|36301|1|190|1863|82|44.36|50.57|2.52|0.00|206.64|3637.52|4146.74|10.33|0.00|206.64|216.97|-3430.88| +2451648|49934|5917|27861|494541|6821|36301|1|92|1863|12|41.42|74.14|32.62|168.31|391.44|497.04|889.68|8.92|168.31|223.13|232.05|-273.91| +2451648|49934|13904|27861|494541|6821|36301|1|280|1863|90|55.30|65.25|19.57|0.00|1761.30|4977.00|5872.50|17.61|0.00|1761.30|1778.91|-3215.70| +2451648|49934|15587|27861|494541|6821|36301|1|284|1863|49|96.75|152.86|85.60|0.00|4194.40|4740.75|7490.14|41.94|0.00|4194.40|4236.34|-546.35| +2451648|49934|11741|27861|494541|6821|36301|1|96|1863|95|48.76|52.17|10.43|0.00|990.85|4632.20|4956.15|19.81|0.00|990.85|1010.66|-3641.35| +2451648|49934|1535|27861|494541|6821|36301|1|143|1863|89|27.33|28.14|13.78|0.00|1226.42|2432.37|2504.46|36.79|0.00|1226.42|1263.21|-1205.95| +2451648|49934|16765|27861|494541|6821|36301|1|147|1863|24|60.93|121.25|93.36|0.00|2240.64|1462.32|2910.00|22.40|0.00|2240.64|2263.04|778.32| +2451648|49934|12901|27861|494541|6821|36301|1|119|1863|93|11.82|19.03|2.28|120.86|212.04|1099.26|1769.79|4.55|120.86|91.18|95.73|-1008.08| +2451648|49934|6671|27861|494541|6821|36301|1|221|1863|78|27.18|48.10|34.15|0.00|2663.70|2120.04|3751.80|26.63|0.00|2663.70|2690.33|543.66| +2451648|49934|13085|27861|494541|6821|36301|1|162|1863|38|5.01|7.31|4.23|57.86|160.74|190.38|277.78|7.20|57.86|102.88|110.08|-87.50| +2451648|49934|116|27861|494541|6821|36301|1|137|1863|77|62.28|64.77|1.29|88.40|99.33|4795.56|4987.29|0.65|88.40|10.93|11.58|-4784.63| +2451648|49934|12821|27861|494541|6821|36301|1|283|1863|4|64.82|67.41|26.28|0.00|105.12|259.28|269.64|6.30|0.00|105.12|111.42|-154.16| +2451819|70315|3859|36278|1830502|4726|19750|2|151|1864|22|45.21|51.99|12.47|0.00|274.34|994.62|1143.78|19.20|0.00|274.34|293.54|-720.28| +2451819|70315|343|36278|1830502|4726|19750|2|47|1864|33|49.99|77.48|64.30|0.00|2121.90|1649.67|2556.84|169.75|0.00|2121.90|2291.65|472.23| +2451819|70315|10643|36278|1830502|4726|19750|2|86|1864|96|3.90|5.88|1.70|0.00|163.20|374.40|564.48|6.52|0.00|163.20|169.72|-211.20| +2451819|70315|9889|36278|1830502|4726|19750|2|248|1864|27|71.82|102.70|55.45|0.00|1497.15|1939.14|2772.90|14.97|0.00|1497.15|1512.12|-441.99| +|70315|3311||1830502|4726|19750||193|1864|13|43.65|69.40||||567.45|902.20|40.14||802.88||235.43| +2451819|70315|4309|36278|1830502|4726|19750|2|248|1864|40|81.63|142.03|15.62|0.00|624.80|3265.20|5681.20|31.24|0.00|624.80|656.04|-2640.40| +2451819||140|36278||||2||1864|||43.03|||129.50|159.39||7.77||129.50|137.27|| +2451819|70315|16255|36278|1830502|4726|19750|2|201|1864|86|91.67|122.83|115.46|0.00|9929.56|7883.62|10563.38|794.36|0.00|9929.56|10723.92|2045.94| +2451819|70315|3191|36278|1830502|4726|19750|2|27|1864|76|1.01|1.64|0.06|2.59|4.56|76.76|124.64|0.01|2.59|1.97|1.98|-74.79| +2451819|70315|10261|36278|1830502|4726|19750|2|165|1864|25|30.40|37.08|1.85|0.00|46.25|760.00|927.00|0.00|0.00|46.25|46.25|-713.75| +2451819|70315|11180|36278|1830502|4726|19750|2|244|1864|24|60.68|64.92|51.28|0.00|1230.72|1456.32|1558.08|98.45|0.00|1230.72|1329.17|-225.60| +2451819|70315|10429|36278|1830502|4726|19750|2|230|1864|47|56.45|110.64|67.49|0.00|3172.03|2653.15|5200.08|158.60|0.00|3172.03|3330.63|518.88| +2451819|70315|13178|36278|1830502|4726|19750|2|2|1864|38|98.17|147.25|2.94|0.00|111.72|3730.46|5595.50|7.82|0.00|111.72|119.54|-3618.74| +2451092|35739|17113|79887|854925|5963|35801|10|214|1865|70|14.67|21.56|4.31|0.00|301.70|1026.90|1509.20|18.10|0.00|301.70|319.80|-725.20| +2451092|35739|1516|79887|854925|5963|35801|10|196|1865|36|31.08|41.64|0.00|0.00|0.00|1118.88|1499.04|0.00|0.00|0.00|0.00|-1118.88| +2451092|35739|14371|79887|854925|5963|35801|10|121|1865|15|56.20|85.98|54.16|0.00|812.40|843.00|1289.70|64.99|0.00|812.40|877.39|-30.60| +2451092|35739|2149|79887|854925|5963|35801|10|119|1865|90|59.03|105.07|76.70|0.00|6903.00|5312.70|9456.30|138.06|0.00|6903.00|7041.06|1590.30| +2451092|35739|15254|79887|854925|5963|35801|10|87|1865|63|23.98|32.85|6.89|0.00|434.07|1510.74|2069.55|17.36|0.00|434.07|451.43|-1076.67| +2451092|35739|15055|79887|854925|5963|35801|10|152|1865|37|31.61|43.62|3.92|0.00|145.04|1169.57|1613.94|2.90|0.00|145.04|147.94|-1024.53| +2451092|35739|6505|79887|854925|5963|35801|10|280|1865|35|5.48|7.89|2.52|0.00|88.20|191.80|276.15|6.17|0.00|88.20|94.37|-103.60| +2451092|35739|14234|79887|854925|5963|35801|10|236|1865|26|97.28|142.02|61.06|0.00|1587.56|2529.28|3692.52|142.88|0.00|1587.56|1730.44|-941.72| +2451092|35739|10237|79887|854925|5963|35801|10|34|1865|94|93.65|117.99|58.99|0.00|5545.06|8803.10|11091.06|277.25|0.00|5545.06|5822.31|-3258.04| +2451092|35739|2522|79887|854925|5963|35801|10|118|1865|46|44.19|58.33|52.49|1448.72|2414.54|2032.74|2683.18|9.65|1448.72|965.82|975.47|-1066.92| +2451092|35739|8584|79887|854925|5963|35801|10|24|1865|51|49.73|58.68|41.07|0.00|2094.57|2536.23|2992.68|41.89|0.00|2094.57|2136.46|-441.66| +2451092|35739|2090|79887|854925|5963|35801|10|85|1865|71|88.73|165.03|103.96|0.00|7381.16|6299.83|11717.13|295.24|0.00|7381.16|7676.40|1081.33| +2451092|35739|4018|79887|854925|5963|35801|10|226|1865|49|78.23|147.07|102.94|0.00|5044.06|3833.27|7206.43|252.20|0.00|5044.06|5296.26|1210.79| +2451092|35739|9223|79887|854925|5963|35801|10|133|1865|66|37.88|60.60|34.54|0.00|2279.64|2500.08|3999.60|0.00|0.00|2279.64|2279.64|-220.44| +2451092|35739|5371|79887|854925|5963|35801|10|32|1865|53|82.95|144.33|85.15|1083.10|4512.95|4396.35|7649.49|171.49|1083.10|3429.85|3601.34|-966.50| +2452614|59869|8491|32907|178046|6385|17924|8|35|1866|28|47.51|57.01|53.01|0.00|1484.28|1330.28|1596.28|118.74|0.00|1484.28|1603.02|154.00| +2452614|59869|3453|32907|178046|6385|17924|8|140|1866|31|22.98|35.84|19.71|0.00|611.01|712.38|1111.04|0.00|0.00|611.01|611.01|-101.37| +2452614|59869|6822|32907|178046|6385|17924|8|230|1866|9|13.68|20.52|17.44|0.00|156.96|123.12|184.68|9.41|0.00|156.96|166.37|33.84| +2452614|59869|12561|32907|178046|6385|17924|8|157|1866|60|57.15|90.29|23.47|0.00|1408.20|3429.00|5417.40|112.65|0.00|1408.20|1520.85|-2020.80| +2452614|59869|4969|32907|178046|6385|17924|8|25|1866|6|96.00|103.68|41.47|0.00|248.82|576.00|622.08|2.48|0.00|248.82|251.30|-327.18| +2452614|59869|5445|32907|178046|6385|17924|8|57|1866|31|75.48|111.71|51.38|0.00|1592.78|2339.88|3463.01|143.35|0.00|1592.78|1736.13|-747.10| +2452614|59869|14622|32907|178046|6385|17924|8|106|1866|9|29.47|58.64|46.32|0.00|416.88|265.23|527.76|12.50|0.00|416.88|429.38|151.65| +2452614|59869|211|32907|178046|6385|17924|8|142|1866|20|5.08|5.18|1.81|0.00|36.20|101.60|103.60|2.53|0.00|36.20|38.73|-65.40| +2452614|59869|10908|32907|178046|6385|17924|8|6|1866|93|75.04|75.04|21.01|0.00|1953.93|6978.72|6978.72|78.15|0.00|1953.93|2032.08|-5024.79| +2452614|59869|475|32907|178046|6385|17924|8|142|1866|57|49.57|81.79|60.52|0.00|3449.64|2825.49|4662.03|137.98|0.00|3449.64|3587.62|624.15| +2452614|59869|12475|32907|178046|6385|17924|8|153|1866|29|56.26|86.07|37.01|0.00|1073.29|1631.54|2496.03|96.59|0.00|1073.29|1169.88|-558.25| +2452614|59869|16557|32907|178046|6385|17924|8|87|1866|60|8.87|15.87|7.45|0.00|447.00|532.20|952.20|40.23|0.00|447.00|487.23|-85.20| +2452614|59869|12919|32907|178046|6385|17924|8|78|1866|96|97.10|178.66|42.87|0.00|4115.52|9321.60|17151.36|0.00|0.00|4115.52|4115.52|-5206.08| +2452530|50693|17643|21508|1004197|2439|27528|8|88|1867|53|69.58|105.76|16.92|0.00|896.76|3687.74|5605.28|17.93|0.00|896.76|914.69|-2790.98| +2452530|50693|6055|21508|1004197|2439|27528|8|189|1867|85|65.08|121.04|117.40|0.00|9979.00|5531.80|10288.40|199.58|0.00|9979.00|10178.58|4447.20| +||1105||1004197|||8||1867|29|97.49|||0.00|2840.55||4438.45|198.83|0.00|2840.55||13.34| +2452530|50693|15289|21508|1004197|2439|27528|8|181|1867|41|28.32|55.79|51.32|0.00|2104.12|1161.12|2287.39|21.04|0.00|2104.12|2125.16|943.00| +2452530|50693|1092|21508|1004197|2439|27528|8|195|1867|89|35.28|64.91|47.38|0.00|4216.82|3139.92|5776.99|84.33|0.00|4216.82|4301.15|1076.90| +2452530|50693|1839|21508|1004197|2439|27528|8|39|1867|51|98.79|157.07|17.27|0.00|880.77|5038.29|8010.57|0.00|0.00|880.77|880.77|-4157.52| +2452530|50693|7293|21508|1004197|2439|27528|8|100|1867|6|6.84|12.44|7.58|0.00|45.48|41.04|74.64|1.36|0.00|45.48|46.84|4.44| +2452530|50693|10549|21508|1004197|2439|27528|8|234|1867|62|46.99|80.82|17.78|0.00|1102.36|2913.38|5010.84|11.02|0.00|1102.36|1113.38|-1811.02| +2452076|69370|17071|9515|917932|6349|27566|8|187|1868|72|73.59|133.93|17.41|0.00|1253.52|5298.48|9642.96|100.28|0.00|1253.52|1353.80|-4044.96| +2452076|69370|9255|9515|917932|6349|27566|8|154|1868|84|63.17|115.60|78.60|0.00|6602.40|5306.28|9710.40|528.19|0.00|6602.40|7130.59|1296.12| +2452076|69370|12225|9515|917932|6349|27566|8|23|1868|20|86.24|139.70|37.71|460.06|754.20|1724.80|2794.00|23.53|460.06|294.14|317.67|-1430.66| +2452076|69370|16955|9515|917932|6349|27566|8|161|1868|75|98.45|187.05|89.78|0.00|6733.50|7383.75|14028.75|606.01|0.00|6733.50|7339.51|-650.25| +2452076|69370|13749|9515|917932|6349|27566|8|165|1868|95|96.13|151.88|132.13|0.00|12552.35|9132.35|14428.60|1004.18|0.00|12552.35|13556.53|3420.00| +2452076|69370|6173|9515|917932|6349|27566|8|255|1868|96|47.68|59.12|44.34|3192.48|4256.64|4577.28|5675.52|63.84|3192.48|1064.16|1128.00|-3513.12| +2452076|69370|13909|9515|917932|6349|27566|8|150|1868|14|40.61|58.88|4.12|0.00|57.68|568.54|824.32|4.03|0.00|57.68|61.71|-510.86| +2452076|69370|16879|9515|917932|6349|27566|8|200|1868|16|79.20|157.60|52.00|149.76|832.00|1267.20|2521.60|6.82|149.76|682.24|689.06|-584.96| +2452076|69370|2641|9515|917932|6349|27566|8|300|1868|53|24.01|25.69|13.35|0.00|707.55|1272.53|1361.57|63.67|0.00|707.55|771.22|-564.98| +2452076|69370|17795|9515|917932|6349|27566|8|145|1868|17|49.24|95.03|39.91|0.00|678.47|837.08|1615.51|0.00|0.00|678.47|678.47|-158.61| +2452076|69370|6943|9515|917932|6349|27566|8|227|1868|54|96.97|121.21|7.27|0.00|392.58|5236.38|6545.34|7.85|0.00|392.58|400.43|-4843.80| +2452076|69370|11135|9515|917932|6349|27566|8|209|1868|62|1.78|1.94|0.65|0.00|40.30|110.36|120.28|0.80|0.00|40.30|41.10|-70.06| +2452076|69370|8089|9515|917932|6349|27566|8|46|1868|8|38.76|40.69|19.12|0.00|152.96|310.08|325.52|13.76|0.00|152.96|166.72|-157.12| +2452076|69370|483|9515|917932|6349|27566|8|157|1868|59|75.74|128.00|6.40|0.00|377.60|4468.66|7552.00|33.98|0.00|377.60|411.58|-4091.06| +2452076|69370|3383|9515|917932|6349|27566|8|24|1868|93|58.01|91.07|20.03|0.00|1862.79|5394.93|8469.51|37.25|0.00|1862.79|1900.04|-3532.14| +2452076|69370|16691|9515|917932|6349|27566|8|163|1868|44|2.88|4.46|2.63|0.00|115.72|126.72|196.24|6.94|0.00|115.72|122.66|-11.00| +2452187|36076|3343|16158|1675883|7161|25286|4|280|1869|79|18.30|23.79|19.50|0.00|1540.50|1445.70|1879.41|123.24|0.00|1540.50|1663.74|94.80| +2452187|36076|16667|16158|1675883|7161|25286|4|191|1869|63|62.79|70.95|21.28|1179.76|1340.64|3955.77|4469.85|1.60|1179.76|160.88|162.48|-3794.89| +2452187|36076|8991|16158|1675883|7161|25286|4|74|1869|50|44.40|70.15|39.28|0.00|1964.00|2220.00|3507.50|157.12|0.00|1964.00|2121.12|-256.00| +2452187|36076|1555|16158|1675883|7161|25286|4|197|1869|6|10.47|10.47|7.85|0.00|47.10|62.82|62.82|0.94|0.00|47.10|48.04|-15.72| +2452187|36076|1505|16158|1675883|7161|25286|4|22|1869|93|16.85|23.42|13.34|0.00|1240.62|1567.05|2178.06|12.40|0.00|1240.62|1253.02|-326.43| +2452187|36076|10073|16158|1675883|7161|25286|4|155|1869|83|13.79|25.51|22.19|0.00|1841.77|1144.57|2117.33|0.00|0.00|1841.77|1841.77|697.20| +2452187||14237|16158|1675883|||||1869|76|39.41||11.01|0.00||2995.16||50.20|0.00||886.96|| +||2293|||7161|||244|1869|1|||||45.33|78.71||||||| +2452187|36076|4287|16158|1675883|7161|25286|4|121|1869|45|46.86|70.75|14.15|89.14|636.75|2108.70|3183.75|0.00|89.14|547.61|547.61|-1561.09| +2452187|36076|12753|16158|1675883|7161|25286|4|282|1869|97|55.77|89.23|20.52|0.00|1990.44|5409.69|8655.31|19.90|0.00|1990.44|2010.34|-3419.25| +2452187|36076|839|16158|1675883|7161|25286|4|52|1869|69|55.71|80.22|53.74|0.00|3708.06|3843.99|5535.18|0.00|0.00|3708.06|3708.06|-135.93| +2452187|36076|16925|16158|1675883|7161|25286|4|57|1869|43|45.61|51.08|16.34|309.15|702.62|1961.23|2196.44|23.60|309.15|393.47|417.07|-1567.76| +2452187|36076|1009|16158|1675883|7161|25286|4|89|1869|4|57.74|106.81|41.65|0.00|166.60|230.96|427.24|3.33|0.00|166.60|169.93|-64.36| +2452207|59187|9673|15842|1511902|3962|19622|10|172|1870|48|86.49|150.49|10.53|0.00|505.44|4151.52|7223.52|45.48|0.00|505.44|550.92|-3646.08| +2452207|59187|8401|15842|1511902|3962|19622|10|294|1870|100|30.37|53.75|43.00|1204.00|4300.00|3037.00|5375.00|61.92|1204.00|3096.00|3157.92|59.00| +2452207|59187|9709|15842|1511902|3962|19622|10|239|1870|44|20.61|31.94|3.19|0.00|140.36|906.84|1405.36|9.82|0.00|140.36|150.18|-766.48| +2452207|59187|2451|15842|1511902|3962|19622|10|133|1870|81|64.92|77.25|67.20|0.00|5443.20|5258.52|6257.25|163.29|0.00|5443.20|5606.49|184.68| +2452207|59187|12599|15842|1511902|3962|19622|10|28|1870|100|81.81|115.35|103.81|0.00|10381.00|8181.00|11535.00|934.29|0.00|10381.00|11315.29|2200.00| +2452207|59187|1945|15842|1511902|3962|19622|10|194|1870|59|28.21|48.23|19.77|0.00|1166.43|1664.39|2845.57|34.99|0.00|1166.43|1201.42|-497.96| +2452207|59187|2695|15842|1511902|3962|19622|10|274|1870|80|9.75|17.16|6.86|0.00|548.80|780.00|1372.80|38.41|0.00|548.80|587.21|-231.20| +2452207|59187|3395|15842|1511902|3962|19622|10|140|1870|42|6.77|11.30|0.79|0.00|33.18|284.34|474.60|1.32|0.00|33.18|34.50|-251.16| +2452207|59187|2221|15842|1511902|3962|19622|10|72|1870|75|13.65|25.79|7.47|0.00|560.25|1023.75|1934.25|28.01|0.00|560.25|588.26|-463.50| +2452207|59187|12289|15842|1511902|3962|19622|10|246|1870|19|19.08|33.96|27.50|522.50|522.50|362.52|645.24|0.00|522.50|0.00|0.00|-362.52| +2452207|59187|6431|15842|1511902|3962|19622|10|149|1870|9|22.29|24.07|7.46|0.00|67.14|200.61|216.63|5.37|0.00|67.14|72.51|-133.47| +2452207|59187|13759|15842|1511902|3962|19622|10|28|1870|17|76.30|140.39|32.28|0.00|548.76|1297.10|2386.63|49.38|0.00|548.76|598.14|-748.34| +2452021|29432|11|150|1484455|3160|49465|10|117|1871|66|4.53|8.87|0.35|0.00|23.10|298.98|585.42|0.69|0.00|23.10|23.79|-275.88| +2452021|29432|10915|150|1484455|3160|49465|10|2|1871|89|35.11|58.98|54.85|0.00|4881.65|3124.79|5249.22|439.34|0.00|4881.65|5320.99|1756.86| +2452021|29432|5567|150|1484455|3160|49465|10|298|1871|19|13.53|16.50|6.27|0.00|119.13|257.07|313.50|10.72|0.00|119.13|129.85|-137.94| +2452021|29432|9323|150|1484455|3160|49465|10|95|1871|33|36.64|67.78|44.73|0.00|1476.09|1209.12|2236.74|44.28|0.00|1476.09|1520.37|266.97| +2452021|29432|16515|150|1484455|3160|49465|10|169|1871|76|9.48|17.25|12.93|0.00|982.68|720.48|1311.00|49.13|0.00|982.68|1031.81|262.20| +2452021|29432|15329|150|1484455|3160|49465|10|176|1871|97|10.26|17.85|14.63|0.00|1419.11|995.22|1731.45|0.00|0.00|1419.11|1419.11|423.89| +2452021|29432|7603|150|1484455|3160|49465|10|127|1871|97|21.41|22.69|15.65|0.00|1518.05|2076.77|2200.93|45.54|0.00|1518.05|1563.59|-558.72| +2452021|29432|6311|150|1484455|3160|49465|10|91|1871|53|64.28|66.85|24.06|0.00|1275.18|3406.84|3543.05|51.00|0.00|1275.18|1326.18|-2131.66| +2452021|29432|14789|150|1484455|3160|49465|10|73|1871|20|25.37|32.21|3.54|0.00|70.80|507.40|644.20|3.54|0.00|70.80|74.34|-436.60| +2452021|29432|65|150|1484455|3160|49465|10|48|1871|69|99.67|166.44|23.30|0.00|1607.70|6877.23|11484.36|80.38|0.00|1607.70|1688.08|-5269.53| +2452492|30531|16185|44312|959623|5781|33892|2|174|1872|77|96.60|177.74|72.87|0.00|5610.99|7438.20|13685.98|392.76|0.00|5610.99|6003.75|-1827.21| +2452492|30531|13830|44312|959623|5781|33892|2|231|1872|56|78.58|130.44|28.69|787.25|1606.64|4400.48|7304.64|32.77|787.25|819.39|852.16|-3581.09| +2452492|30531|11901|44312|959623|5781|33892|2|74|1872|67|42.31|80.38|79.57|0.00|5331.19|2834.77|5385.46|426.49|0.00|5331.19|5757.68|2496.42| +2452492|30531|11539|44312|959623|5781|33892|2|14|1872|47|88.57|140.82|116.88|0.00|5493.36|4162.79|6618.54|384.53|0.00|5493.36|5877.89|1330.57| +2452492|30531|12306|44312|959623|5781|33892|2|195|1872|33|2.09|3.05|1.55|0.00|51.15|68.97|100.65|4.60|0.00|51.15|55.75|-17.82| +2452492|30531|2406|44312|959623|5781|33892|2|77|1872|99|69.78|108.15|6.48|0.00|641.52|6908.22|10706.85|38.49|0.00|641.52|680.01|-6266.70| +2452492|30531|1033|||||2|109|1872||63.63||82.21||328.84|254.52|361.40||||253.21|| +2452492|30531|8166|44312|959623|5781|33892|2|259|1872|38|18.84|29.39|4.40|0.00|167.20|715.92|1116.82|1.67|0.00|167.20|168.87|-548.72| +||10575||959623|5781||2||1872|39||59.96|57.56|||1771.77||0.00||||| +2452492|30531|8538|44312|959623|5781|33892|2|274|1872|52|64.10|114.73|87.19|0.00|4533.88|3333.20|5965.96|45.33|0.00|4533.88|4579.21|1200.68| +2452492|30531|2955|44312|959623|5781|33892|2|6|1872|21|10.92|11.57|11.57|0.00|242.97|229.32|242.97|21.86|0.00|242.97|264.83|13.65| +2452492|30531|7134|44312|959623|5781|33892|2|142|1872|57|83.63|107.04|31.04|0.00|1769.28|4766.91|6101.28|159.23|0.00|1769.28|1928.51|-2997.63| +2452492|30531|11647|44312|959623|5781|33892|2|139|1872|32|57.17|98.33|19.66|0.00|629.12|1829.44|3146.56|44.03|0.00|629.12|673.15|-1200.32| +2452492|30531|15637|44312|959623|5781|33892|2|298|1872|26|33.60|41.32|0.00|0.00|0.00|873.60|1074.32|0.00|0.00|0.00|0.00|-873.60| +2451776|48967|4004|84937|56015|2646|2524|10|62|1873|98|21.17|29.21|8.47|0.00|830.06|2074.66|2862.58|0.00|0.00|830.06|830.06|-1244.60| +2451776|48967|2665|84937|56015|2646|2524|10|248|1873|9|99.34|113.24|33.97|247.64|305.73|894.06|1019.16|5.22|247.64|58.09|63.31|-835.97| +|48967|12878||56015|2646|2524|||1873|7|29.12||||130.20||289.45|3.28||46.88||-156.96| +2451776|48967|7751|84937|56015|2646|2524|10|51|1873|76|21.16|24.33|6.32|427.48|480.32|1608.16|1849.08|4.75|427.48|52.84|57.59|-1555.32| +2451776|48967|7646|84937|56015|2646|2524|10|132|1873|65|35.07|69.08|53.19|2454.71|3457.35|2279.55|4490.20|70.18|2454.71|1002.64|1072.82|-1276.91| +2451776|48967|11300|84937|56015|2646|2524|10|157|1873|57|51.53|57.19|17.15|146.63|977.55|2937.21|3259.83|58.16|146.63|830.92|889.08|-2106.29| +2451776|48967|7544|84937|56015|2646|2524|10|159|1873|39|91.23|91.23|66.59|0.00|2597.01|3557.97|3557.97|51.94|0.00|2597.01|2648.95|-960.96| +2451776|48967|14983|84937|56015|2646|2524|10|136|1873|11|85.26|169.66|120.45|0.00|1324.95|937.86|1866.26|105.99|0.00|1324.95|1430.94|387.09| +2451776|48967|1352|84937|56015||2524|||1873|35||118.01|79.06|0.00|2767.10|2208.85|4130.35||0.00||2822.44|558.25| +2451776||9367|84937||||||1873||81.05|148.32||0.00||5268.25|||0.00|7616.05||| +2451776|48967|2087|84937|56015|2646|2524|10|297|1873|27|79.26|158.52|122.06|0.00|3295.62|2140.02|4280.04|32.95|0.00|3295.62|3328.57|1155.60| +2451776|48967|8546|84937|56015|2646|2524|10|247|1873|79|16.07|30.21|6.34|0.00|500.86|1269.53|2386.59|35.06|0.00|500.86|535.92|-768.67| +2451873|49126|229|86185|1154946|6460|3621|10|245|1874|50|3.70|5.03|3.11|0.00|155.50|185.00|251.50|7.77|0.00|155.50|163.27|-29.50| +2451873|49126|2180|86185|1154946|6460|3621|10|95|1874|66|76.45|150.60|84.33|0.00|5565.78|5045.70|9939.60|389.60|0.00|5565.78|5955.38|520.08| +2451873|49126|6158|86185|1154946|6460|3621|10|146|1874|63|76.22|137.95|16.55|0.00|1042.65|4801.86|8690.85|0.00|0.00|1042.65|1042.65|-3759.21| +2451873|49126|7321|86185|1154946|6460|3621|10|258|1874|76|55.71|93.59|6.55|0.00|497.80|4233.96|7112.84|0.00|0.00|497.80|497.80|-3736.16| +2451873|49126|1711|86185|1154946|6460|3621|10|63|1874|89|33.60|59.13|44.34|0.00|3946.26|2990.40|5262.57|39.46|0.00|3946.26|3985.72|955.86| +2451873|49126|3716|86185|1154946|6460|3621|10|262|1874|43|12.42|15.77|13.40|0.00|576.20|534.06|678.11|40.33|0.00|576.20|616.53|42.14| +2451873|49126|10831|86185|1154946|6460|3621|10|73|1874|2|54.90|104.31|12.51|0.00|25.02|109.80|208.62|0.50|0.00|25.02|25.52|-84.78| +2451873|49126|2762|86185|1154946|6460|3621|10|199|1874|31|79.52|100.19|98.18|0.00|3043.58|2465.12|3105.89|152.17|0.00|3043.58|3195.75|578.46| +2451873|49126|6101|86185|1154946|6460|3621|10|108|1874|48|50.87|75.79|18.94|0.00|909.12|2441.76|3637.92|63.63|0.00|909.12|972.75|-1532.64| +2451873|49126|12889|86185|1154946|6460|3621|10|65|1874|31|99.91|131.88|117.37|0.00|3638.47|3097.21|4088.28|327.46|0.00|3638.47|3965.93|541.26| +2451873|49126|5947|86185|1154946|6460|3621|10|6|1874|94|51.15|56.77|25.54|24.00|2400.76|4808.10|5336.38|166.37|24.00|2376.76|2543.13|-2431.34| +2451873|49126|11048|86185|1154946|6460|3621|10|287|1874|66|61.53|67.06|67.06|0.00|4425.96|4060.98|4425.96|265.55|0.00|4425.96|4691.51|364.98| +2451873|49126|8126|86185|1154946|6460|3621|10|249|1874|58|97.07|141.72|80.78|0.00|4685.24|5630.06|8219.76|281.11|0.00|4685.24|4966.35|-944.82| +2451873|49126|9527|86185|1154946|6460|3621|10|107|1874|29|1.94|2.59|0.56|16.24|16.24|56.26|75.11|0.00|16.24|0.00|0.00|-56.26| +2451127|64634|2305|77208|1614135|6615|7942|7|193|1875|69|44.70|75.09|26.28|0.00|1813.32|3084.30|5181.21|108.79|0.00|1813.32|1922.11|-1270.98| +2451127|64634|745|77208|1614135|6615|7942|7|22|1875|41|26.50|38.16|4.96|0.00|203.36|1086.50|1564.56|16.26|0.00|203.36|219.62|-883.14| +2451127|64634|4288|77208|1614135|6615|7942|7|22|1875|39|55.44|108.10|31.34|965.58|1222.26|2162.16|4215.90|2.56|965.58|256.68|259.24|-1905.48| +2451127|64634|11266|77208|1614135|6615|7942|7|175|1875|71|20.08|37.75|9.81|0.00|696.51|1425.68|2680.25|27.86|0.00|696.51|724.37|-729.17| +2451127|64634|12260|77208|1614135|6615|7942|7|277|1875|40|14.79|23.07|2.07|0.00|82.80|591.60|922.80|2.48|0.00|82.80|85.28|-508.80| +2451127|64634|1666|77208|1614135|6615|7942|7|54|1875|39|76.02|148.23|148.23|404.66|5780.97|2964.78|5780.97|107.52|404.66|5376.31|5483.83|2411.53| +2451127|64634|13442|77208|1614135|6615|7942|7|78|1875|28|60.30|110.95|83.21|1141.64|2329.88|1688.40|3106.60|0.00|1141.64|1188.24|1188.24|-500.16| +2451127|64634|820|77208|1614135|6615|7942|7|106|1875|74|78.93|146.02|23.36|0.00|1728.64|5840.82|10805.48|103.71|0.00|1728.64|1832.35|-4112.18| +2451127|64634|6583|77208|1614135|6615|7942|7|185|1875|24|50.52|85.37|4.26|0.00|102.24|1212.48|2048.88|7.15|0.00|102.24|109.39|-1110.24| +2451144|62103|13652|26517|93877|437|24244|7|223|1876|62|49.77|69.67|29.26|0.00|1814.12|3085.74|4319.54|126.98|0.00|1814.12|1941.10|-1271.62| +2451144|62103|799|26517|93877|437|24244|7|225|1876|84|54.14|97.45|76.98|0.00|6466.32|4547.76|8185.80|193.98|0.00|6466.32|6660.30|1918.56| +2451144|62103|16027|26517|93877|437|24244|7|271|1876|74|96.06|131.60|122.38|0.00|9056.12|7108.44|9738.40|181.12|0.00|9056.12|9237.24|1947.68| +2451144|62103|4273|26517|93877|437|24244|7|216|1876|99|58.65|96.18|43.28|4241.87|4284.72|5806.35|9521.82|2.14|4241.87|42.85|44.99|-5763.50| +2451144|62103|14863|26517|93877|437|24244|7|275|1876|34|7.42|8.45|2.70|0.00|91.80|252.28|287.30|3.67|0.00|91.80|95.47|-160.48| +2451144|62103|9757|26517|93877|437|24244|7|59|1876|16|98.78|190.64|20.97|0.00|335.52|1580.48|3050.24|23.48|0.00|335.52|359.00|-1244.96| +2451144|62103|8875|26517|93877|437|24244|7|92|1876|94|90.83|117.17|4.68|0.00|439.92|8538.02|11013.98|21.99|0.00|439.92|461.91|-8098.10| +2451144|62103|10966|26517|93877|437|24244|7|82|1876|97|46.75|90.69|35.36|0.00|3429.92|4534.75|8796.93|68.59|0.00|3429.92|3498.51|-1104.83| +||4231|26517|93877|437|24244|7||1876||||19.29|0.00||||26.04|0.00|||| +2451858|40657|13723|88489|1243248|4854|6460|7|49|1877|98|40.34|40.34|20.17|0.00|1976.66|3953.32|3953.32|138.36|0.00|1976.66|2115.02|-1976.66| +2451858|40657|13141|88489|1243248|4854|6460|7|21|1877|15|3.18|3.18|2.54|0.00|38.10|47.70|47.70|0.38|0.00|38.10|38.48|-9.60| +||13079|88489|||||282|1877|||||35.82||5536.56|7474.12||35.82|||| +2451858|40657|2459|88489|1243248|4854|6460|7|86|1877|48|28.75|38.52|37.74|0.00|1811.52|1380.00|1848.96|36.23|0.00|1811.52|1847.75|431.52| +2451858|40657|15349|88489|1243248|4854|6460|7|181|1877|14|72.58|76.20|12.95|0.00|181.30|1016.12|1066.80|9.06|0.00|181.30|190.36|-834.82| +2451858|40657|3607|88489|1243248|4854|6460|7|62|1877|61|40.07|79.73|3.18|0.00|193.98|2444.27|4863.53|1.93|0.00|193.98|195.91|-2250.29| +2451858|40657|10153|88489|1243248|4854|6460|7|108|1877|85|9.46|14.94|9.56|593.19|812.60|804.10|1269.90|6.58|593.19|219.41|225.99|-584.69| +2451858|40657|14117|88489|1243248|4854|6460|7|39|1877|41|4.72|7.88|4.25|0.00|174.25|193.52|323.08|12.19|0.00|174.25|186.44|-19.27| +2451140|38169|8092|89040|1434105|2817|22682|1|296|1878|58|49.61|90.29|62.30|0.00|3613.40|2877.38|5236.82|0.00|0.00|3613.40|3613.40|736.02| +2451140|38169|10444|89040|1434105|2817|22682|1|277|1878|25|33.19|40.15|12.04|0.00|301.00|829.75|1003.75|27.09|0.00|301.00|328.09|-528.75| +2451140|38169|11750|89040|1434105|2817|22682|1|300|1878|6|94.95|135.77|78.74|0.00|472.44|569.70|814.62|23.62|0.00|472.44|496.06|-97.26| +2451140|38169|8684|89040|1434105|2817|22682|1|115|1878|49|17.28|25.22|1.26|0.00|61.74|846.72|1235.78|3.08|0.00|61.74|64.82|-784.98| +2451140|38169|16825|89040|1434105|2817|22682|1|19|1878|82|19.58|23.30|1.86|0.00|152.52|1605.56|1910.60|0.00|0.00|152.52|152.52|-1453.04| +2451140|38169|6307|89040|1434105|2817|22682|1|102|1878|19|62.12|83.24|17.48|0.00|332.12|1180.28|1581.56|16.60|0.00|332.12|348.72|-848.16| +2451140|38169|7328|89040|1434105|2817|22682|1|190|1878|56|93.93|141.83|9.92|0.00|555.52|5260.08|7942.48|11.11|0.00|555.52|566.63|-4704.56| +2451140|38169|10262|89040|1434105|2817|22682|1|3|1878|49|92.87|149.52|79.24|0.00|3882.76|4550.63|7326.48|155.31|0.00|3882.76|4038.07|-667.87| +2451140|38169|12949||1434105||||83|1878|62|27.25||26.60|0.00|||1773.82|65.96|0.00|||-40.30| +2451140|38169|11191|89040|1434105|2817||1||1878|||41.46|33.16|0.00|2553.32|2595.67||229.79|0.00|2553.32|2783.11|| +2451140|38169|11156|89040|1434105|2817|22682|1|170|1878|56|49.25|57.62|55.31|0.00|3097.36|2758.00|3226.72|247.78|0.00|3097.36|3345.14|339.36| +2451140|38169|4238|89040|1434105|2817|22682|1|22|1878|61|39.86|62.97|48.48|0.00|2957.28|2431.46|3841.17|236.58|0.00|2957.28|3193.86|525.82| +2451140|38169|8527|89040|1434105|2817|22682|1|119|1878|67|2.24|3.76|0.03|0.00|2.01|150.08|251.92|0.00|0.00|2.01|2.01|-148.07| +2451140|38169|15880||1434105|2817||1|113|1878||||52.67|0.00||868.32|1693.08|37.92|0.00|948.06|985.98|79.74| +2451140|38169|5264|89040|1434105|2817|22682|1|267|1878|17|81.39|104.17|45.83|264.89|779.11|1383.63|1770.89|30.85|264.89|514.22|545.07|-869.41| +2451140|38169|11035|89040|1434105|2817|22682|1|84|1878|59|19.88|34.59|22.13|1044.53|1305.67|1172.92|2040.81|0.00|1044.53|261.14|261.14|-911.78| +2452114|64940|325|67151|1729361|371|14181|1|112|1879|3|84.36|130.75|103.29|0.00|309.87|253.08|392.25|6.19|0.00|309.87|316.06|56.79| +2452114|64940|13385|67151|1729361|371|14181|1|184|1879|98|23.59|41.75|23.38|0.00|2291.24|2311.82|4091.50|206.21|0.00|2291.24|2497.45|-20.58| +2452114|64940|11305|67151|1729361|371|14181|1|232|1879|69|95.47|156.57|51.66|0.00|3564.54|6587.43|10803.33|106.93|0.00|3564.54|3671.47|-3022.89| +2452114|64940|11955|67151|1729361|371|14181|1|117|1879|18|19.28|24.29|21.86|0.00|393.48|347.04|437.22|19.67|0.00|393.48|413.15|46.44| +2452114|64940|15415|67151|1729361|371|14181|1|30|1879|69|77.58|103.18|43.33|2929.97|2989.77|5353.02|7119.42|4.78|2929.97|59.80|64.58|-5293.22| +2452114|64940|2509|67151|1729361|371|14181|1|206|1879|96|24.21|25.17|8.05|0.00|772.80|2324.16|2416.32|54.09|0.00|772.80|826.89|-1551.36| +2452114|64940|6081|67151|1729361|371|14181|1|222|1879|18|70.38|120.34|61.37|0.00|1104.66|1266.84|2166.12|55.23|0.00|1104.66|1159.89|-162.18| +2452114|64940|15481|67151|1729361|371|14181|1|80|1879|2|78.15|98.46|35.44|0.00|70.88|156.30|196.92|0.70|0.00|70.88|71.58|-85.42| +2452114|64940|12323|67151|1729361|371|14181|1|288|1879|63|35.99|42.46|41.18|0.00|2594.34|2267.37|2674.98|207.54|0.00|2594.34|2801.88|326.97| +2452114|64940|12961|67151|1729361|371|14181|1|244|1879|12|59.03|100.35|2.00|0.00|24.00|708.36|1204.20|0.48|0.00|24.00|24.48|-684.36| +2452114|64940|17429|67151|1729361|371|14181|1|235|1879|12|96.15|156.72|133.21|0.00|1598.52|1153.80|1880.64|47.95|0.00|1598.52|1646.47|444.72| +|64940|17035||||||201|1879||64.78|69.31|11.08|0.00|||||0.00||209.41|| +||285|67151|1729361|371|14181|1|263|1879||||100.69||7753.13|5313.77||387.65||||| +|64940|16947|67151||371|14181|||1879||2.81|2.95|2.59|0.00|168.35|182.65|191.75|15.15|0.00|168.35||-14.30| +2452114|64940|12155|67151|1729361|371|14181|1|62|1879|83|93.55|104.77|19.90|0.00|1651.70|7764.65|8695.91|49.55|0.00|1651.70|1701.25|-6112.95| +2452114|64940|4033|67151|1729361|371|14181|1|266|1879|56|59.06|86.22|17.24|0.00|965.44|3307.36|4828.32|38.61|0.00|965.44|1004.05|-2341.92| +2451783|55575|2006|81545|1609293|594|22695|1|206|1880|73|75.63|93.02|43.71|0.00|3190.83|5520.99|6790.46|159.54|0.00|3190.83|3350.37|-2330.16| +2451783|55575|4334|81545|1609293|594|22695|1|212|1880|44|26.93|30.43|11.86|0.00|521.84|1184.92|1338.92|46.96|0.00|521.84|568.80|-663.08| +2451783|55575|6188|81545|1609293|594|22695|1|77|1880|57|12.77|17.36|11.28|0.00|642.96|727.89|989.52|12.85|0.00|642.96|655.81|-84.93| +2451783|55575|1903|81545|1609293|594|22695|1|178|1880|61|92.12|157.52|7.87|0.00|480.07|5619.32|9608.72|43.20|0.00|480.07|523.27|-5139.25| +2451783|55575|1771|81545|1609293|594|22695|1|288|1880|13|34.52|65.58|12.46|0.00|161.98|448.76|852.54|11.33|0.00|161.98|173.31|-286.78| +2451783|55575|3782|81545|1609293|594|22695|1|224|1880|34|63.23|71.44|57.15|0.00|1943.10|2149.82|2428.96|97.15|0.00|1943.10|2040.25|-206.72| +2451783|55575|14522|81545|1609293|594|22695|1|32|1880|21|55.82|66.42|21.91|0.00|460.11|1172.22|1394.82|23.00|0.00|460.11|483.11|-712.11| +2451783|55575|2443|81545|1609293|594|22695|1|54|1880|22|73.19|74.65|52.25|0.00|1149.50|1610.18|1642.30|103.45|0.00|1149.50|1252.95|-460.68| +2451783|55575|17774|81545|1609293|594|22695|1|186|1880|15|42.67|85.34|48.64|0.00|729.60|640.05|1280.10|65.66|0.00|729.60|795.26|89.55| +|55575|3374|81545|1609293|594|22695|1||1880|38|23.91|28.45|2.56||97.28|908.58|1081.10|||51.56|54.13|| +2451783|55575|10958|81545|1609293|594|22695|1|51|1880|90|37.79|72.17|41.13|0.00|3701.70|3401.10|6495.30|111.05|0.00|3701.70|3812.75|300.60| +2451783|55575|8975|81545|1609293|594|22695|1|146|1880|82|82.92|132.67|74.29|0.00|6091.78|6799.44|10878.94|365.50|0.00|6091.78|6457.28|-707.66| +2451783|55575|12175|81545|1609293|594|22695|1|88|1880|96|32.30|55.23|25.95|0.00|2491.20|3100.80|5302.08|0.00|0.00|2491.20|2491.20|-609.60| +2451783|55575|4169|||594||1||1880|99|93.18|148.15|85.92|0.00|8506.08|||680.48|0.00|8506.08||-718.74| +2452165|48993|13457|39840|557253|7113|41326|8|298|1881|25|10.57|13.84|8.16|110.16|204.00|264.25|346.00|5.63|110.16|93.84|99.47|-170.41| +2452165|48993|7449|39840|557253|7113|41326|8|220|1881|97|60.92|65.79|26.97|366.25|2616.09|5909.24|6381.63|0.00|366.25|2249.84|2249.84|-3659.40| +2452165|48993|2713|39840|557253|7113|41326|8|69|1881|4|92.55|170.29|63.00|211.68|252.00|370.20|681.16|2.41|211.68|40.32|42.73|-329.88| +2452165|48993|1399|39840|557253|7113|41326|8|219|1881|67|95.82|144.68|125.87|0.00|8433.29|6419.94|9693.56|337.33|0.00|8433.29|8770.62|2013.35| +2452165|48993|7339|39840|557253|7113|41326|8|261|1881|35|93.50|159.88|44.76|0.00|1566.60|3272.50|5595.80|93.99|0.00|1566.60|1660.59|-1705.90| +2452165|48993|8687|39840|557253|7113|41326|8|93|1881|65|64.54|126.49|60.71|2604.45|3946.15|4195.10|8221.85|107.33|2604.45|1341.70|1449.03|-2853.40| +2452165|48993|12439|39840|557253|7113|41326|8|199|1881|74|44.36|51.01|43.35|0.00|3207.90|3282.64|3774.74|64.15|0.00|3207.90|3272.05|-74.74| +2452165|48993|9221|39840|557253|7113|41326|8|147|1881|38|37.26|74.52|65.57|0.00|2491.66|1415.88|2831.76|224.24|0.00|2491.66|2715.90|1075.78| +2452165|48993|993|39840|557253|7113|41326|8|52|1881|97|90.23|170.53|66.50|0.00|6450.50|8752.31|16541.41|451.53|0.00|6450.50|6902.03|-2301.81| +||7487|39840|557253|7113|41326|8||1881||97.04|107.71|94.78||7582.40|7763.20|8616.80|379.12||7582.40||| +2452165|48993|6289|39840|557253|7113|41326|8|128|1881|74|13.16|20.13|11.27|0.00|833.98|973.84|1489.62|33.35|0.00|833.98|867.33|-139.86| +||3869|39840||||8||1881||58.86||31.54||630.80|1177.20|1577.40|25.23|||656.03|| +2452204|63233|12231|91408|303540|3672|39804|7|190|1882|66|84.72|121.99|19.51|643.83|1287.66|5591.52|8051.34|19.31|643.83|643.83|663.14|-4947.69| +2452204|63233|16131|91408|303540|3672|39804|7|200|1882|42|22.97|34.68|10.40|0.00|436.80|964.74|1456.56|21.84|0.00|436.80|458.64|-527.94| +2452204|63233|7473|91408|303540|3672|39804|7|81|1882|91|62.82|63.44|6.97|323.47|634.27|5716.62|5773.04|15.54|323.47|310.80|326.34|-5405.82| +2452204|63233|13445|91408|303540|3672|39804|7|170|1882|56|6.46|7.49|3.89|0.00|217.84|361.76|419.44|2.17|0.00|217.84|220.01|-143.92| +2452204|63233|8551|91408|303540|3672|39804|7|296|1882|73|53.42|68.37|65.63|0.00|4790.99|3899.66|4991.01|431.18|0.00|4790.99|5222.17|891.33| +2452204|63233|16335|91408|303540|3672|39804|7|223|1882|59|56.93|112.15|17.94|0.00|1058.46|3358.87|6616.85|95.26|0.00|1058.46|1153.72|-2300.41| +2452204|63233|1395|91408|303540|3672|39804|7|45|1882|11|18.01|23.77|0.00|0.00|0.00|198.11|261.47|0.00|0.00|0.00|0.00|-198.11| +2452204|63233|485|91408|303540|3672|39804|7|259|1882|23|73.14|141.89|136.21|0.00|3132.83|1682.22|3263.47|219.29|0.00|3132.83|3352.12|1450.61| +2451048|35908|2683|28406|221630|101|18525|8|264|1883|79|31.26|43.45|43.01|0.00|3397.79|2469.54|3432.55|203.86|0.00|3397.79|3601.65|928.25| +2451048|35908|8695|28406|221630|101|18525|8|21|1883|48|51.79|89.59|20.60|0.00|988.80|2485.92|4300.32|29.66|0.00|988.80|1018.46|-1497.12| +2451048|35908|13639|28406|221630|101|18525|8|258|1883|14|65.08|121.04|91.99|0.00|1287.86|911.12|1694.56|103.02|0.00|1287.86|1390.88|376.74| +2451048|35908|1172|28406|221630|101|18525|8|72|1883|58|69.45|78.47|74.54|0.00|4323.32|4028.10|4551.26|43.23|0.00|4323.32|4366.55|295.22| +2451048|35908|1286|28406|221630|101|18525|8|183|1883|28|72.68|135.91|115.52|0.00|3234.56|2035.04|3805.48|97.03|0.00|3234.56|3331.59|1199.52| +2451048|35908|1396|28406|221630|101|18525|8|104|1883|54|13.63|15.40|10.47|254.42|565.38|736.02|831.60|6.21|254.42|310.96|317.17|-425.06| +2451048|35908|8648|28406|221630|101|18525|8|262|1883|33|18.83|21.08|12.85|0.00|424.05|621.39|695.64|12.72|0.00|424.05|436.77|-197.34| +2451048|35908|2038|28406|221630|101|18525|8|16|1883|63|34.77|68.14|41.56|0.00|2618.28|2190.51|4292.82|235.64|0.00|2618.28|2853.92|427.77| +2451048|35908|11533|||101|||182|1883||||33.84||1962.72||2230.68|58.88||1962.72||| +2451048|35908|1228|28406|221630|101|18525|8|16|1883|2|63.01|77.50|50.37|0.00|100.74|126.02|155.00|9.06|0.00|100.74|109.80|-25.28| +2451889|36825|8423|57660|247021|6782|34943|7|192|1884|2|86.81|132.81|61.09|0.00|122.18|173.62|265.62|0.00|0.00|122.18|122.18|-51.44| +2451889|36825|15251|57660|247021|6782|34943|7|262|1884|44|36.99|66.95|64.94|0.00|2857.36|1627.56|2945.80|85.72|0.00|2857.36|2943.08|1229.80| +2451889|36825|13033|57660|247021|6782|34943|7|16|1884|47|4.92|9.44|3.49|77.09|164.03|231.24|443.68|0.86|77.09|86.94|87.80|-144.30| +|36825|6704|57660|247021|6782||7||1884||20.98|33.56|19.80|0.00|1722.60|1825.26||137.80|0.00|||-102.66| +2451889|36825|14975|57660|247021|6782|34943|7|103|1884|2|37.88|49.24|42.34|0.00|84.68|75.76|98.48|3.38|0.00|84.68|88.06|8.92| +2451889|36825|6152|57660|247021|6782|34943|7|14|1884|13|56.62|90.02|21.60|252.72|280.80|736.06|1170.26|0.28|252.72|28.08|28.36|-707.98| +2451889|36825|15307|57660|247021|6782|34943|7|163|1884|71|34.91|38.75|27.51|0.00|1953.21|2478.61|2751.25|175.78|0.00|1953.21|2128.99|-525.40| +2451889|36825|7802|57660|247021|6782|34943|7|57|1884|47|37.18|69.52|12.51|540.93|587.97|1747.46|3267.44|2.35|540.93|47.04|49.39|-1700.42| +2451889|36825|16898|57660|247021|6782|34943|7|129|1884|68|22.56|27.74|19.14|0.00|1301.52|1534.08|1886.32|65.07|0.00|1301.52|1366.59|-232.56| +2451889|36825|6575|57660|247021|6782|34943|7|141|1884|94|52.68|100.09|71.06|0.00|6679.64|4951.92|9408.46|333.98|0.00|6679.64|7013.62|1727.72| +2451889|36825|7394|57660|247021|6782|34943|7|150|1884|4|8.99|14.29|0.71|0.56|2.84|35.96|57.16|0.02|0.56|2.28|2.30|-33.68| +2451889|36825|9337|57660|247021|6782|34943|7|29|1884|75|46.94|60.08|54.67|0.00|4100.25|3520.50|4506.00|41.00|0.00|4100.25|4141.25|579.75| +2451889|36825|12038|57660|247021|6782|34943|7|288|1884|69|1.66|3.30|2.60|0.00|179.40|114.54|227.70|7.17|0.00|179.40|186.57|64.86| +2451889|36825|16220|57660|247021|6782|34943|7|221|1884|72|12.45|17.43|3.66|0.00|263.52|896.40|1254.96|7.90|0.00|263.52|271.42|-632.88| +2451889|36825|17387||247021||34943|7|82|1884||||66.25|799.63|1126.25|1233.69||13.06|799.63|326.62|339.68|-907.07| +2451157|35991|6692|63799|20396|6593|19190|1|80|1885|37|98.93|174.11|128.84|0.00|4767.08|3660.41|6442.07|286.02|0.00|4767.08|5053.10|1106.67| +2451157||10730|63799|||19190||239|1885|25||22.13||0.00|298.75||553.25|5.97|0.00|298.75|304.72|| +2451157|35991|17839|63799|20396|6593|19190|1|165|1885|39|63.88|115.62|90.18|0.00|3517.02|2491.32|4509.18|35.17|0.00|3517.02|3552.19|1025.70| +2451157|35991|10996|63799|20396|6593|19190|1|50|1885|43|39.55|46.27|28.68|0.00|1233.24|1700.65|1989.61|110.99|0.00|1233.24|1344.23|-467.41| +2451157|35991|15932|63799|20396|6593|19190|1|173|1885|99|33.16|42.77|31.22|123.63|3090.78|3282.84|4234.23|89.01|123.63|2967.15|3056.16|-315.69| +2451157|35991|9631|63799|20396|6593|19190|1|82|1885|97|63.59|76.30|16.02|1383.00|1553.94|6168.23|7401.10|15.38|1383.00|170.94|186.32|-5997.29| +||10633||20396|6593|19190||114|1885||36.87||19.08|587.66||2027.85|||587.66|||-1566.11| +2451157|35991|2102|63799|20396|6593|19190|1|9|1885|16|96.62|151.69|63.70|0.00|1019.20|1545.92|2427.04|20.38|0.00|1019.20|1039.58|-526.72| +2451157|35991|2749|63799|20396|6593|19190|1|267|1885|55|66.75|94.78|92.88|0.00|5108.40|3671.25|5212.90|306.50|0.00|5108.40|5414.90|1437.15| +2451157|35991|7255|63799|20396|6593|19190|1|70|1885|27|64.60|84.62|0.84|0.00|22.68|1744.20|2284.74|1.58|0.00|22.68|24.26|-1721.52| +2451157|35991|15259|63799|20396|6593|19190|1|159|1885|97|25.89|49.70|2.98|0.00|289.06|2511.33|4820.90|17.34|0.00|289.06|306.40|-2222.27| +2451157|35991|11018|63799|20396|6593|19190|1|10|1885|89|89.83|107.79|7.54|0.00|671.06|7994.87|9593.31|6.71|0.00|671.06|677.77|-7323.81| +2451029|71485|17287|31850|518385|3767|11902|2|46|1886|29|70.18|120.70|68.79|0.00|1994.91|2035.22|3500.30|79.79|0.00|1994.91|2074.70|-40.31| +2451029|71485|9308|31850|518385|3767|11902|2|103|1886|89|77.68|79.23|10.29|0.00|915.81|6913.52|7051.47|36.63|0.00|915.81|952.44|-5997.71| +2451029|71485|1042|31850|518385|3767|11902|2|14|1886|48|50.12|77.68|15.53|74.54|745.44|2405.76|3728.64|0.00|74.54|670.90|670.90|-1734.86| +2451029|71485|15958|31850|518385|3767|11902|2|188|1886|62|98.09|179.50|53.85|0.00|3338.70|6081.58|11129.00|66.77|0.00|3338.70|3405.47|-2742.88| +2451029|71485|3442|31850|518385|3767|11902|2|92|1886|97|39.46|41.43|7.87|0.00|763.39|3827.62|4018.71|7.63|0.00|763.39|771.02|-3064.23| +2451029|71485|16465|31850|518385|3767|11902|2|122|1886|42|63.92|114.41|48.05|0.00|2018.10|2684.64|4805.22|161.44|0.00|2018.10|2179.54|-666.54| +2451029|71485|5134|31850|518385|3767|11902|2|60|1886|46|1.28|1.62|1.13|0.00|51.98|58.88|74.52|2.59|0.00|51.98|54.57|-6.90| +2451029|71485|4702|31850|518385|3767|11902|2|181|1886|29|51.48|59.71|37.02|0.00|1073.58|1492.92|1731.59|64.41|0.00|1073.58|1137.99|-419.34| +2451029|71485|4580|31850|518385|3767|11902|2|281|1886|45|5.51|11.02|5.28|0.00|237.60|247.95|495.90|14.25|0.00|237.60|251.85|-10.35| +2451029|71485|10040|31850|518385|3767|11902|2|244|1886|11|9.90|11.48|7.11|0.00|78.21|108.90|126.28|3.12|0.00|78.21|81.33|-30.69| +2451029|71485|12140|31850|518385|3767|11902|2|133|1886|44|98.80|143.26|40.11|970.66|1764.84|4347.20|6303.44|7.94|970.66|794.18|802.12|-3553.02| +2451029|71485|3068|31850|518385|3767|11902|2|94|1886|32|25.57|29.66|5.93|0.00|189.76|818.24|949.12|0.00|0.00|189.76|189.76|-628.48| +2451029|71485|12976|31850|518385|3767|11902|2|220|1886|85|46.33|66.71|14.00|0.00|1190.00|3938.05|5670.35|107.10|0.00|1190.00|1297.10|-2748.05| +2451029|71485|2599|31850|518385|3767|11902|2|176|1886|26|2.68|2.84|0.17|1.41|4.42|69.68|73.84|0.24|1.41|3.01|3.25|-66.67| +2451029|71485|466|31850|518385|3767|11902|2|103|1886|83|73.17|111.21|22.24|0.00|1845.92|6073.11|9230.43|147.67|0.00|1845.92|1993.59|-4227.19| +2452568|42790|81|33717|1132853|2036|32772|4|173|1887|74|9.24|14.41|6.48|0.00|479.52|683.76|1066.34|28.77|0.00|479.52|508.29|-204.24| +2452568|42790|8995|33717|1132853|2036|32772|4|47|1887|70|76.89|115.33|91.11|0.00|6377.70|5382.30|8073.10|510.21|0.00|6377.70|6887.91|995.40| +2452568|42790|1008|33717|1132853|2036|32772|4|206|1887|58|35.40|69.38|36.77|0.00|2132.66|2053.20|4024.04|170.61|0.00|2132.66|2303.27|79.46| +2452568|42790|7722|33717|1132853|2036|32772|4|294|1887|4|81.19|158.32|117.15|0.00|468.60|324.76|633.28|23.43|0.00|468.60|492.03|143.84| +|42790|10374|33717||||4||1887||||4.71|1.17||||3.49|1.17|||| +|42790|16681|33717||||4||1887|35||25.90|3.88||||906.50|6.79||135.80||| +2452568|42790|10333|33717|1132853|2036|32772|4|178|1887|44|26.79|52.24|13.06|0.00|574.64|1178.76|2298.56|5.74|0.00|574.64|580.38|-604.12| +2452568|42790|9841|33717|1132853|2036|32772|4|272|1887|94|23.09|37.63|13.92|0.00|1308.48|2170.46|3537.22|117.76|0.00|1308.48|1426.24|-861.98| +2451614|29505|3053||||46155|8||1888|6|||21.75||130.50||343.50|0.00||130.50||-165.66| +2451614|29505|3257|3861|899159|2457|46155|8|85|1888|77|45.92|47.29|23.64|1601.84|1820.28|3535.84|3641.33|10.92|1601.84|218.44|229.36|-3317.40| +2451614|29505|6317|3861|899159|2457|46155|8|261|1888|14|70.90|101.38|97.32|0.00|1362.48|992.60|1419.32|27.24|0.00|1362.48|1389.72|369.88| +2451614|29505|17581||899159|||8||1888|8|88.10|151.53|60.61|48.48||704.80|1212.24|26.18|48.48|436.40||-268.40| +2451614|29505|3434|3861|899159|2457|46155|8|96|1888|76|94.88|148.01|105.08|0.00|7986.08|7210.88|11248.76|479.16|0.00|7986.08|8465.24|775.20| +2451614|29505|16759|3861|899159|2457|46155|8|85|1888|13|11.57|20.59|9.05|0.00|117.65|150.41|267.67|4.70|0.00|117.65|122.35|-32.76| +2451614|29505|15655|3861|899159|2457|46155|8|48|1888|29|7.45|10.13|0.20|0.00|5.80|216.05|293.77|0.46|0.00|5.80|6.26|-210.25| +2451614|29505|11162|3861|899159|2457|46155|8|44|1888|98|49.83|92.68|9.26|0.00|907.48|4883.34|9082.64|0.00|0.00|907.48|907.48|-3975.86| +2451974|38377|7755|85235|232627|5266|31458|4|37|1889|66|16.42|21.51|19.35|0.00|1277.10|1083.72|1419.66|102.16|0.00|1277.10|1379.26|193.38| +2451974|38377|10663|85235|232627|5266|31458|4|132|1889|96|93.80|109.74|50.48|0.00|4846.08|9004.80|10535.04|0.00|0.00|4846.08|4846.08|-4158.72| +2451974|38377|10495|85235|232627|5266|31458|4|142|1889|21|37.92|65.22|48.91|0.00|1027.11|796.32|1369.62|61.62|0.00|1027.11|1088.73|230.79| +2451974|38377|6593|85235|232627|5266|31458|4|102|1889|38|46.84|60.42|18.12|0.00|688.56|1779.92|2295.96|27.54|0.00|688.56|716.10|-1091.36| +2451974|38377|17649|85235|232627|5266|31458|4|93|1889|32|39.76|71.56|32.91|0.00|1053.12|1272.32|2289.92|10.53|0.00|1053.12|1063.65|-219.20| +2451974|38377|3731|85235|232627|5266|31458|4|251|1889|64|43.96|87.92|31.65|0.00|2025.60|2813.44|5626.88|101.28|0.00|2025.60|2126.88|-787.84| +2451974|38377|12183|85235|232627|5266|31458|4|148|1889|56|81.57|119.90|111.50|4807.88|6244.00|4567.92|6714.40|0.00|4807.88|1436.12|1436.12|-3131.80| +2451974|38377|891|85235|232627|5266|31458|4|200|1889|15|56.90|112.66|111.53|0.00|1672.95|853.50|1689.90|33.45|0.00|1672.95|1706.40|819.45| +2451974|38377|16739|85235|232627|5266|31458|4|131|1889|86|2.19|3.85|0.00|0.00|0.00|188.34|331.10|0.00|0.00|0.00|0.00|-188.34| +2451122|42798|3121|76932|928560|988|36169|8|258|1890|32|45.46|59.09|47.86|0.00|1531.52|1454.72|1890.88|122.52|0.00|1531.52|1654.04|76.80| +2451122|42798|10855|76932|928560|988|36169|8|280|1890|77|95.17|141.80|4.25|0.00|327.25|7328.09|10918.60|22.90|0.00|327.25|350.15|-7000.84| +2451122|42798|8518|76932|928560|988|36169|8|281|1890|79|49.50|56.43|45.14|0.00|3566.06|3910.50|4457.97|71.32|0.00|3566.06|3637.38|-344.44| +2451122|42798|16747|76932|928560|988|36169|8|154|1890|41|28.58|38.29|3.44|0.00|141.04|1171.78|1569.89|4.23|0.00|141.04|145.27|-1030.74| +2451122|42798|15602|76932|928560|988|36169|8|83|1890|32|26.32|35.00|11.20|0.00|358.40|842.24|1120.00|10.75|0.00|358.40|369.15|-483.84| +2451122|42798|8587|76932|928560|988|36169|8|90|1890|11|16.13|24.03|9.85|0.00|108.35|177.43|264.33|7.58|0.00|108.35|115.93|-69.08| +2451122|42798|13550|76932|928560|988|36169|8|90|1890|100|49.10|80.03|70.42|0.00|7042.00|4910.00|8003.00|211.26|0.00|7042.00|7253.26|2132.00| +2451122|42798|367|76932|928560|988|36169|8|34|1890|81|48.58|90.84|36.33|2236.47|2942.73|3934.98|7358.04|35.31|2236.47|706.26|741.57|-3228.72| +2452142|55541|745|47924|1254119|4667|46862|2|226|1891|10|27.34|39.09|1.56|0.00|15.60|273.40|390.90|0.31|0.00|15.60|15.91|-257.80| +2452142|55541|4289|47924|1254119|4667|46862|2|220|1891|14|99.83|102.82|40.09|39.28|561.26|1397.62|1439.48|5.21|39.28|521.98|527.19|-875.64| +2452142|55541|11267|47924|1254119|4667|46862|2|267|1891|53|74.35|139.03|55.61|2564.17|2947.33|3940.55|7368.59|26.82|2564.17|383.16|409.98|-3557.39| +2452142|55541|12261|47924|1254119|4667|46862|2|208|1891|36|56.84|76.73|9.97|0.00|358.92|2046.24|2762.28|10.76|0.00|358.92|369.68|-1687.32| +2452142|55541|1667|47924|1254119|4667|46862|2|135|1891|58|49.36|81.44|46.42|0.00|2692.36|2862.88|4723.52|107.69|0.00|2692.36|2800.05|-170.52| +2452142|55541|13443|47924|1254119|4667|46862|2|188|1891|94|96.62|122.70|121.47|0.00|11418.18|9082.28|11533.80|685.09|0.00|11418.18|12103.27|2335.90| +2452142|55541|821|47924|1254119|4667|46862|2|44|1891|17|37.85|57.53|54.65|0.00|929.05|643.45|978.01|83.61|0.00|929.05|1012.66|285.60| +2452142|55541|6583|47924|1254119|4667|46862|2|200|1891|85|45.62|54.28|16.82|0.00|1429.70|3877.70|4613.80|0.00|0.00|1429.70|1429.70|-2448.00| +2452142|55541|3177|47924|1254119|4667|46862|2|228|1891|78|10.18|16.08|16.08|815.25|1254.24|794.04|1254.24|0.00|815.25|438.99|438.99|-355.05| +2452593|62852|16147|42897|1770828|6|20384|4|214|1892|26|43.89|47.40|20.85|0.00|542.10|1141.14|1232.40|43.36|0.00|542.10|585.46|-599.04| +2452593|62852|11875|42897|1770828|6|20384|4|258|1892|34|5.92|6.15|5.90|0.00|200.60|201.28|209.10|16.04|0.00|200.60|216.64|-0.68| +2452593|62852|10215|42897|1770828|6|20384|4|244|1892|44|61.26|77.80|17.89|0.00|787.16|2695.44|3423.20|47.22|0.00|787.16|834.38|-1908.28| +2452593|62852|12447|42897|1770828|6|20384|4|231|1892|80|3.81|4.99|0.44|0.00|35.20|304.80|399.20|0.35|0.00|35.20|35.55|-269.60| +2452593|62852|246|42897|1770828|6|20384|4|284|1892|25|13.80|15.18|4.85|0.00|121.25|345.00|379.50|8.48|0.00|121.25|129.73|-223.75| +2452593|62852|1531|42897|1770828|6|20384|4|224|1892|47|43.87|69.75|36.96|0.00|1737.12|2061.89|3278.25|86.85|0.00|1737.12|1823.97|-324.77| +2452593|62852|3525|42897|1770828|6|20384|4|86|1892|14|55.92|98.97|25.73|0.00|360.22|782.88|1385.58|3.60|0.00|360.22|363.82|-422.66| +2452593|62852|13021|42897|1770828|6|20384|4|216|1892|20|87.95|119.61|45.45|0.00|909.00|1759.00|2392.20|72.72|0.00|909.00|981.72|-850.00| +2452593|62852|17263|42897|1770828|6|20384|4|229|1892|17|8.25|14.60|14.30|0.00|243.10|140.25|248.20|19.44|0.00|243.10|262.54|102.85| +2452593|62852|17971|42897|1770828|6|20384|4|31|1892|33|47.81|86.05|31.83|0.00|1050.39|1577.73|2839.65|94.53|0.00|1050.39|1144.92|-527.34| +2452593|62852|14443|42897|1770828|6|20384|4|253|1892|86|49.62|90.30|40.63|0.00|3494.18|4267.32|7765.80|69.88|0.00|3494.18|3564.06|-773.14| +2452593|62852|14100|42897|1770828|6|20384|4|292|1892|88|38.19|75.61|56.70|2145.52|4989.60|3360.72|6653.68|85.32|2145.52|2844.08|2929.40|-516.64| +||15786|42897|1770828||20384|4|156|1892|73|68.50|82.88|63.81|0.00|4658.13||||0.00||4658.13|| +2452593|62852|10693|42897|1770828|6|20384|4|268|1892|83|45.52|81.48|48.88|1298.25|4057.04|3778.16|6762.84|137.93|1298.25|2758.79|2896.72|-1019.37| +2452626|35040|14869|85459|988823|4540|17504|8|142|1893|95|45.36|78.47|1.56|0.00|148.20|4309.20|7454.65|11.85|0.00|148.20|160.05|-4161.00| +2452626|35040|12973|85459|988823|4540|17504|8|195|1893|84|35.20|36.25|33.35|0.00|2801.40|2956.80|3045.00|196.09|0.00|2801.40|2997.49|-155.40| +2452626|35040|6363|85459|988823|4540|17504|8|255|1893|99|65.20|120.62|65.13|0.00|6447.87|6454.80|11941.38|451.35|0.00|6447.87|6899.22|-6.93| +2452626|35040|3019|85459|988823|4540|17504|8|158|1893|89|46.79|84.22|29.47|0.00|2622.83|4164.31|7495.58|52.45|0.00|2622.83|2675.28|-1541.48| +2452626|35040|16446|85459|988823|4540|17504|8|261|1893|92|52.96|90.03|72.02|0.00|6625.84|4872.32|8282.76|132.51|0.00|6625.84|6758.35|1753.52| +2452626|35040|15654|85459|988823|4540|17504|8|27|1893|76|53.47|102.12|28.59|0.00|2172.84|4063.72|7761.12|195.55|0.00|2172.84|2368.39|-1890.88| +2452626|35040|3690|85459|988823|4540|17504|8|68|1893|41|97.72|165.14|44.58|0.00|1827.78|4006.52|6770.74|109.66|0.00|1827.78|1937.44|-2178.74| +2452626|35040|8535|85459|988823|4540|17504|8|227|1893|52|17.14|21.76|20.88|0.00|1085.76|891.28|1131.52|86.86|0.00|1085.76|1172.62|194.48| +2452626|35040|7369|85459|988823|4540|17504|8|156|1893|84|99.20|122.01|54.90|0.00|4611.60|8332.80|10248.84|415.04|0.00|4611.60|5026.64|-3721.20| +2452626|35040|3228|85459|988823|4540|17504|8|197|1893|72|50.01|82.51|76.73|0.00|5524.56|3600.72|5940.72|276.22|0.00|5524.56|5800.78|1923.84| +2452626|35040|13063|85459|988823|4540|17504|8|140|1893|56|5.75|10.29|6.48|0.00|362.88|322.00|576.24|21.77|0.00|362.88|384.65|40.88| +2452626|35040|12787|85459|988823|4540|17504|8|18|1893|42|96.63|137.21|90.55|0.00|3803.10|4058.46|5762.82|228.18|0.00|3803.10|4031.28|-255.36| +|35040|14193|85459|||17504||128|1893|86|69.54|106.39|77.66|0.00|6678.76|5980.44|9149.54|467.51|0.00|6678.76||| +2452180|51979|4549|29508|438275|4195|46562|2|6|1894|36|48.11|61.09|59.86|1853.26|2154.96|1731.96|2199.24|6.03|1853.26|301.70|307.73|-1430.26| +2452180|51979|9249|29508|438275|4195|46562|2|19|1894|3|48.55|49.03|25.00|63.00|75.00|145.65|147.09|0.36|63.00|12.00|12.36|-133.65| +2452180|51979|15357|29508|438275|4195|46562|2|187|1894|76|15.47|26.76|21.67|0.00|1646.92|1175.72|2033.76|32.93|0.00|1646.92|1679.85|471.20| +2452180|51979|6625|29508|438275|4195|46562|2|298|1894|35|32.06|39.43|13.80|0.00|483.00|1122.10|1380.05|14.49|0.00|483.00|497.49|-639.10| +2452180|51979|13345|29508|438275|4195|46562|2|176|1894|61|55.82|82.61|67.74|0.00|4132.14|3405.02|5039.21|82.64|0.00|4132.14|4214.78|727.12| +2452180|51979|16907||438275|4195||2|150|1894||94.48||||1139.13||1309.41|||1139.13|1196.08|288.81| +2452180|51979|14951|29508|438275|4195|46562|2|285|1894|76|46.75|79.47|15.89|0.00|1207.64|3553.00|6039.72|48.30|0.00|1207.64|1255.94|-2345.36| +2452180|51979|14143|29508|438275|4195|46562|2|50|1894|33|67.34|97.64|87.87|0.00|2899.71|2222.22|3222.12|144.98|0.00|2899.71|3044.69|677.49| +2452180|51979|14119|29508||4195|46562||93|1894|59|17.44|31.91||0.00||||29.17|0.00|583.51||-445.45| +2452180|51979|6155|29508|438275|4195|46562|2|55|1894|76|51.22|91.17|30.08|0.00|2286.08|3892.72|6928.92|160.02|0.00|2286.08|2446.10|-1606.64| +2452180|51979|15083|29508|438275|4195|46562|2|27|1894|50|61.84|123.06|49.22|0.00|2461.00|3092.00|6153.00|221.49|0.00|2461.00|2682.49|-631.00| +2452180|51979|1385|29508|438275|4195|46562|2|243|1894|74|55.64|67.88|22.40|0.00|1657.60|4117.36|5023.12|49.72|0.00|1657.60|1707.32|-2459.76| +2452180|51979|5121|29508|438275|4195|46562|2|274|1894|83|60.32|85.65|60.81|0.00|5047.23|5006.56|7108.95|50.47|0.00|5047.23|5097.70|40.67| +2452180|51979|4617|29508|438275|4195|46562|2|67|1894|13|40.13|65.01|21.45|0.00|278.85|521.69|845.13|16.73|0.00|278.85|295.58|-242.84| +2452180|51979|8417|29508|438275|4195|46562|2|58|1894|4|10.36|17.61|10.56|0.00|42.24|41.44|70.44|2.95|0.00|42.24|45.19|0.80| +2452180|51979|12957|29508|438275|4195|46562|2|184|1894|14|31.80|42.93|32.62|0.00|456.68|445.20|601.02|9.13|0.00|456.68|465.81|11.48| +2451158|51610|2482|36331|794336|7188|19640|7|193|1895|29|25.55|45.22|17.18|0.00|498.22|740.95|1311.38|39.85|0.00|498.22|538.07|-242.73| +2451158|51610|758|36331|794336|7188|19640|7|61|1895|70|38.89|46.66|42.46|0.00|2972.20|2722.30|3266.20|148.61|0.00|2972.20|3120.81|249.90| +2451158|51610|15842|36331|794336|7188|19640|7|81|1895|32|7.76|15.36|11.36|0.00|363.52|248.32|491.52|32.71|0.00|363.52|396.23|115.20| +2451158|51610|3385|36331|794336|7188|19640|7|232|1895|4|42.01|57.97|7.53|0.00|30.12|168.04|231.88|0.90|0.00|30.12|31.02|-137.92| +2451158|51610|3104|36331|794336|7188|19640|7|85|1895|2|71.05|93.78|75.96|0.00|151.92|142.10|187.56|4.55|0.00|151.92|156.47|9.82| +2451158|51610|16694|36331|794336|7188|19640|7|254|1895|11|36.38|61.84|55.03|0.00|605.33|400.18|680.24|30.26|0.00|605.33|635.59|205.15| +2451158|51610|326|36331|794336|7188|19640|7|120|1895|73|47.13|53.25|34.08|0.00|2487.84|3440.49|3887.25|223.90|0.00|2487.84|2711.74|-952.65| +2451158|51610|5492|36331|794336|7188|19640|7|71|1895|58|71.15|85.38|59.76|0.00|3466.08|4126.70|4952.04|138.64|0.00|3466.08|3604.72|-660.62| +2451158||9620||||19640|7|292|1895||26.70||7.73|2.47|||67.28|0.90|2.47|12.99||| +2451158||8653||794336|7188|19640|7||1895|59|23.61||11.75||693.25||2312.21|||693.25||| +||13208||794336||19640|||1895||5.84|7.53|1.58||45.82||218.37|0.00||45.82|45.82|-123.54| +2451158|51610|3175|36331|794336|7188|19640|7|89|1895|9|65.91|91.61|32.97|0.00|296.73|593.19|824.49|0.00|0.00|296.73|296.73|-296.46| +2451158|51610|10069|36331|794336|7188|19640|7|244|1895|90|37.91|53.07|37.14|0.00|3342.60|3411.90|4776.30|133.70|0.00|3342.60|3476.30|-69.30| +2451158|51610|3754|36331|794336|7188|19640|7|149|1895|81|39.16|68.53|9.59|0.00|776.79|3171.96|5550.93|7.76|0.00|776.79|784.55|-2395.17| +2452191|30479|15183|88046|820457|2821|44432|4|93|1896|42|26.60|28.19|6.76|79.49|283.92|1117.20|1183.98|10.22|79.49|204.43|214.65|-912.77| +2452191|30479|3253|88046|820457|2821|44432|4|259|1896|11|55.17|68.96|61.37|0.00|675.07|606.87|758.56|33.75|0.00|675.07|708.82|68.20| +2452191|30479|12721|88046|820457|2821|44432|4|229|1896|72|68.91|103.36|44.44|127.98|3199.68|4961.52|7441.92|122.86|127.98|3071.70|3194.56|-1889.82| +2452191|30479|12741|88046|820457|2821|44432|4|208|1896|57|36.11|36.47|20.05|0.00|1142.85|2058.27|2078.79|57.14|0.00|1142.85|1199.99|-915.42| +2452191|30479|4029|88046|820457|2821|44432|4|243|1896|59|72.82|109.23|28.39|0.00|1675.01|4296.38|6444.57|0.00|0.00|1675.01|1675.01|-2621.37| +2452191|30479|17729|88046|820457|2821|44432|4|159|1896|23|25.16|41.76|38.83|0.00|893.09|578.68|960.48|0.00|0.00|893.09|893.09|314.41| +2452191|30479|17993|88046|820457|2821|44432|4|254|1896|47|13.13|15.75|3.62|0.00|170.14|617.11|740.25|15.31|0.00|170.14|185.45|-446.97| +2452191|30479|683|88046|820457|2821|44432|4|182|1896|4|48.71|61.37|2.45|0.00|9.80|194.84|245.48|0.78|0.00|9.80|10.58|-185.04| +2451849|74012|5735|81256|1006706|5756|35559|1|86|1897|2|59.96|107.32|69.75|0.00|139.50|119.92|214.64|6.97|0.00|139.50|146.47|19.58| +2451849|74012|17485|81256|1006706|5756|35559|1|228|1897|68|2.39|4.20|1.47|0.00|99.96|162.52|285.60|6.99|0.00|99.96|106.95|-62.56| +2451849|74012|3871|81256|1006706|5756|35559|1|41|1897|10|65.82|107.28|34.32|168.16|343.20|658.20|1072.80|15.75|168.16|175.04|190.79|-483.16| +2451849|74012|6413|81256|1006706|5756|35559|1|291|1897|27|96.77|165.47|153.88|0.00|4154.76|2612.79|4467.69|166.19|0.00|4154.76|4320.95|1541.97| +2451849|74012|5849|81256|1006706|5756|35559|1|85|1897|36|1.39|1.51|1.47|0.00|52.92|50.04|54.36|3.70|0.00|52.92|56.62|2.88| +2451849|74012|11468|81256|1006706|5756|35559|1|279|1897|100|26.83|51.24|46.62|0.00|4662.00|2683.00|5124.00|279.72|0.00|4662.00|4941.72|1979.00| +2451849|74012|1784|81256|1006706|5756|35559|1|218|1897|54|61.35|73.62|43.43|0.00|2345.22|3312.90|3975.48|140.71|0.00|2345.22|2485.93|-967.68| +2451849|74012|9818|81256|1006706|5756|35559|1|90|1897|37|37.85|39.36|15.74|0.00|582.38|1400.45|1456.32|46.59|0.00|582.38|628.97|-818.07| +2451410|39725|5504|1175|1590737|6252|4614|2|122|1898|32|22.84|25.58|20.46|0.00|654.72|730.88|818.56|6.54|0.00|654.72|661.26|-76.16| +2451410||12211||1590737|||2|94|1898||60.27|||||3977.82|4295.94|||2577.30|2680.39|| +2451410|39725|3481|1175|1590737|6252|4614|2|250|1898|46|13.80|19.45|14.97|0.00|688.62|634.80|894.70|34.43|0.00|688.62|723.05|53.82| +2451410|39725|3367|1175|1590737|6252|4614|2|153|1898|29|42.12|82.13|15.60|0.00|452.40|1221.48|2381.77|31.66|0.00|452.40|484.06|-769.08| +2451410|39725|5696|1175|1590737|6252|4614|2|147|1898|93|41.00|56.99|15.95|0.00|1483.35|3813.00|5300.07|74.16|0.00|1483.35|1557.51|-2329.65| +2451410|39725|6218|1175|1590737|6252|4614|2|32|1898|61|91.98|169.24|69.38|0.00|4232.18|5610.78|10323.64|338.57|0.00|4232.18|4570.75|-1378.60| +||3241|1175||6252|4614|||1898||||0.17|||2.50||0.02||0.34|0.36|| +2451410|39725|8|1175|1590737|6252|4614|2|243|1898|68|74.62|124.61|100.93|0.00|6863.24|5074.16|8473.48|205.89|0.00|6863.24|7069.13|1789.08| +2451410|39725|2030|1175|1590737|6252|4614|2|295|1898|75|8.32|12.48|11.73|0.00|879.75|624.00|936.00|52.78|0.00|879.75|932.53|255.75| +2451410|39725|12212|1175|1590737|6252|4614|2|224|1898|75|52.67|59.51|39.27|2621.27|2945.25|3950.25|4463.25|19.43|2621.27|323.98|343.41|-3626.27| +2451410|39725|7645|1175|1590737|6252|4614|2|109|1898|89|53.61|102.93|28.82|0.00|2564.98|4771.29|9160.77|128.24|0.00|2564.98|2693.22|-2206.31| +2451410|39725|6428|1175|1590737|6252|4614|2|155|1898|66|52.20|96.57|65.66|0.00|4333.56|3445.20|6373.62|260.01|0.00|4333.56|4593.57|888.36| +2451410|39725|952|1175|1590737|6252|4614|2|31|1898|4|43.52|81.81|75.26|0.00|301.04|174.08|327.24|9.03|0.00|301.04|310.07|126.96| +2451410|39725|10897|1175|1590737|6252|4614|2|229|1898|85|56.93|79.13|28.48|121.04|2420.80|4839.05|6726.05|68.99|121.04|2299.76|2368.75|-2539.29| +2451410|39725|8594|1175|1590737|6252|4614|2|263|1898|11|22.98|42.74|4.70|0.00|51.70|252.78|470.14|1.03|0.00|51.70|52.73|-201.08| +2451410||5096||1590737|||2|78|1898||57.74|89.49|||1431.80||1789.80|||1431.80|1546.34|277.00| +2451399|70791|5797|37615|698527|5486|34398|1|125|1899|14|67.47|67.47|67.47|0.00|944.58|944.58|944.58|85.01|0.00|944.58|1029.59|0.00| +2451399|70791|9470|37615|698527|5486|34398|1|261|1899|90|52.42|79.67|49.39|0.00|4445.10|4717.80|7170.30|133.35|0.00|4445.10|4578.45|-272.70| +|70791|2474|37615|698527||34398||157|1899|38||12.75|5.10|||394.06|484.50|||193.80|195.73|| +2451399|70791|16934|37615|698527|5486|34398|1|88|1899|26|98.20|105.07|63.04|1507.91|1639.04|2553.20|2731.82|2.62|1507.91|131.13|133.75|-2422.07| +2451399|70791|9145|37615|698527|5486|34398|1|29|1899|83|61.98|104.12|6.24|0.00|517.92|5144.34|8641.96|15.53|0.00|517.92|533.45|-4626.42| +2451399|70791|5222|37615|698527|5486|34398|1|271|1899|31|70.97|134.13|17.43|0.00|540.33|2200.07|4158.03|0.00|0.00|540.33|540.33|-1659.74| +2451399||17708|37615|698527|||||1899|24||||0.00|2447.76||2549.76||0.00|||| +2451399|70791|3535|37615|698527|5486|34398|1|208|1899|2|18.85|20.92|3.13|0.00|6.26|37.70|41.84|0.06|0.00|6.26|6.32|-31.44| +2451399|70791|8776|37615|698527|5486|34398|1|227|1899|100|7.73|10.12|8.80|0.00|880.00|773.00|1012.00|26.40|0.00|880.00|906.40|107.00| +2451399|70791|2248|37615|698527|5486|34398|1|186|1899|70|11.08|20.71|3.72|0.00|260.40|775.60|1449.70|23.43|0.00|260.40|283.83|-515.20| +2451399|70791|16784|37615|698527|5486|34398|1|55|1899|29|6.36|8.84|1.14|31.07|33.06|184.44|256.36|0.13|31.07|1.99|2.12|-182.45| +2451399|70791|9646|37615|698527|5486|34398|1|237|1899|96|20.04|38.87|24.48|0.00|2350.08|1923.84|3731.52|117.50|0.00|2350.08|2467.58|426.24| +2451399|70791|308|37615|698527|5486|34398|1|28|1899|47|83.33|150.82|52.78|0.00|2480.66|3916.51|7088.54|0.00|0.00|2480.66|2480.66|-1435.85| +2451940|51673|4085||1791913|||||1900|22|14.32|||0.00|87.34||324.28|0.00|0.00|87.34||-227.70| +2451940|51673|4107|13325|1791913|548|38783|8|139|1900|70|38.06|60.89|41.40|0.00|2898.00|2664.20|4262.30|86.94|0.00|2898.00|2984.94|233.80| +2451940|51673|6263|13325|1791913|548|38783|8|90|1900|19|27.03|38.65|18.93|0.00|359.67|513.57|734.35|7.19|0.00|359.67|366.86|-153.90| +2451940|51673|2321|13325|1791913|548|38783|8|3|1900|73|89.74|154.35|1.54|0.00|112.42|6551.02|11267.55|3.37|0.00|112.42|115.79|-6438.60| +2451940|51673|1461|13325|1791913|548|38783|8|54|1900|96|39.21|69.79|20.93|241.11|2009.28|3764.16|6699.84|159.13|241.11|1768.17|1927.30|-1995.99| +2451940|51673|16217|13325|1791913|548|38783|8|14|1900|55|2.10|3.48|3.23|168.76|177.65|115.50|191.40|0.44|168.76|8.89|9.33|-106.61| +2451940|51673|10675|13325|1791913|548|38783|8|208|1900|79|17.16|26.25|22.31|0.00|1762.49|1355.64|2073.75|158.62|0.00|1762.49|1921.11|406.85| +2451940|51673|2981|13325|1791913|548|38783|8|208|1900|89|17.84|34.78|2.78|0.00|247.42|1587.76|3095.42|14.84|0.00|247.42|262.26|-1340.34| +2451940|51673|3469|13325|1791913|548|38783|8|73|1900|52|37.46|60.31|32.56|0.00|1693.12|1947.92|3136.12|16.93|0.00|1693.12|1710.05|-254.80| +2451940|51673|14105|13325|1791913|548|38783|8|228|1900|83|13.81|21.81|0.00|0.00|0.00|1146.23|1810.23|0.00|0.00|0.00|0.00|-1146.23| +2451500|37203|5752|75952|898884|569|37901|8|60|1901|93|81.92|103.21|97.01|0.00|9021.93|7618.56|9598.53|541.31|0.00|9021.93|9563.24|1403.37| +2451500|37203|12898|75952|898884|569|37901|8|72|1901|61|74.31|102.54|101.51|0.00|6192.11|4532.91|6254.94|433.44|0.00|6192.11|6625.55|1659.20| +2451500|37203|11474|75952|898884|569|37901|8|22|1901|34|75.03|130.55|63.96|0.00|2174.64|2551.02|4438.70|43.49|0.00|2174.64|2218.13|-376.38| +2451500|37203|15176|75952|898884|569|37901|8|284|1901|47|50.94|85.57|62.46|0.00|2935.62|2394.18|4021.79|58.71|0.00|2935.62|2994.33|541.44| +2451500|37203|13897|75952|898884|569|37901|8|26|1901|48|30.61|57.85|37.02|0.00|1776.96|1469.28|2776.80|159.92|0.00|1776.96|1936.88|307.68| +2451500|37203|14905||898884|||8||1901|90|39.43|74.52||0.00|603.00|3548.70|||0.00|603.00|627.12|| +2451500|37203|4226|75952|898884|569|37901|8|49|1901|45|82.54|118.03|36.58|0.00|1646.10|3714.30|5311.35|16.46|0.00|1646.10|1662.56|-2068.20| +2451500|37203|9625|75952|898884|569|37901|8|194|1901|29|70.73|130.85|130.85|0.00|3794.65|2051.17|3794.65|113.83|0.00|3794.65|3908.48|1743.48| +2451500|37203|17024|75952|898884|569|37901|8|42|1901|16|62.29|117.72|63.56|0.00|1016.96|996.64|1883.52|50.84|0.00|1016.96|1067.80|20.32| +2451500|37203|2287|75952|898884|569|37901|8|243|1901|11|65.19|66.49|9.30|0.00|102.30|717.09|731.39|0.00|0.00|102.30|102.30|-614.79| +2451500|37203|9847|75952|898884|569|37901|8|43|1901|15|97.30|144.97|30.44|0.00|456.60|1459.50|2174.55|41.09|0.00|456.60|497.69|-1002.90| +2451500|37203|4279|75952|898884|569|37901|8|262|1901|22|71.88|98.47|90.59|0.00|1992.98|1581.36|2166.34|0.00|0.00|1992.98|1992.98|411.62| +2452385|36887|16407|83953|1413726|1899|7319|10|163|1902|91|45.56|66.51|14.63|0.00|1331.33|4145.96|6052.41|119.81|0.00|1331.33|1451.14|-2814.63| +2452385|36887|10314|83953|1413726|1899|7319|10|225|1902|51|99.32|163.87|44.24|0.00|2256.24|5065.32|8357.37|135.37|0.00|2256.24|2391.61|-2809.08| +2452385|36887|9114|83953|1413726|1899|7319|10|246|1902|90|27.27|31.90|25.20|0.00|2268.00|2454.30|2871.00|181.44|0.00|2268.00|2449.44|-186.30| +2452385|36887|8245|83953|1413726|1899|7319|10|216|1902|51|57.87|78.12|44.52|0.00|2270.52|2951.37|3984.12|204.34|0.00|2270.52|2474.86|-680.85| +2452385|36887|11649|83953|1413726|1899|7319|10|265|1902|69|27.44|53.23|23.42|0.00|1615.98|1893.36|3672.87|96.95|0.00|1615.98|1712.93|-277.38| +2452385|36887|13155|83953|1413726|1899|7319|10|178|1902|85|85.73|152.59|3.05|0.00|259.25|7287.05|12970.15|7.77|0.00|259.25|267.02|-7027.80| +2452385|36887|15469|83953|1413726|1899|7319|10|224|1902|55|73.71|79.60|27.06|1086.45|1488.30|4054.05|4378.00|32.14|1086.45|401.85|433.99|-3652.20| +2452385|36887|13249|83953|1413726|1899|7319|10|131|1902|83|19.21|27.08|4.87|0.00|404.21|1594.43|2247.64|0.00|0.00|404.21|404.21|-1190.22| +2452385|36887|4059|83953|1413726|1899|7319|10|135|1902|13|16.44|18.57|8.72|0.00|113.36|213.72|241.41|4.53|0.00|113.36|117.89|-100.36| +2452385|36887|10303|83953|1413726|1899|7319|10|184|1902|44|47.01|76.62|58.99|0.00|2595.56|2068.44|3371.28|129.77|0.00|2595.56|2725.33|527.12| +2452385|36887|11367|83953|1413726|1899|7319|10|141|1902|82|33.69|44.47|9.33|680.90|765.06|2762.58|3646.54|0.84|680.90|84.16|85.00|-2678.42| +2452385||7513|83953|1413726|1899|||92|1902|30|88.47|125.62|18.84||565.20|2654.10|3768.60|39.56|||604.76|-2088.90| +2452385|36887|14575|83953|1413726|1899|7319|10|160|1902|64|3.97|5.59|5.08|22.75|325.12|254.08|357.76|21.16|22.75|302.37|323.53|48.29| +2451117|62223|340|1089|763927|6765|22403|7|229|1903|34|69.44|78.46|69.82|522.25|2373.88|2360.96|2667.64|18.51|522.25|1851.63|1870.14|-509.33| +2451117|62223|16810|1089|763927|6765|22403|7|65|1903|70|5.69|10.24|9.01|0.00|630.70|398.30|716.80|56.76|0.00|630.70|687.46|232.40| +2451117|62223|11503|1089|763927|6765|22403|7|163|1903|79|20.88|38.62|25.10|0.00|1982.90|1649.52|3050.98|0.00|0.00|1982.90|1982.90|333.38| +2451117|62223|10090|1089|763927|6765|22403|7|227|1903|18|79.33|158.66|20.62|0.00|371.16|1427.94|2855.88|29.69|0.00|371.16|400.85|-1056.78| +2451117|62223|17029|1089|763927|6765|22403|7|160|1903|56|14.67|25.81|2.32|0.00|129.92|821.52|1445.36|7.79|0.00|129.92|137.71|-691.60| +2451117|62223|2731|1089|763927|6765|22403|7|131|1903|51|45.54|84.70|46.58|0.00|2375.58|2322.54|4319.70|95.02|0.00|2375.58|2470.60|53.04| +2451117|62223|10286|1089|763927|6765|22403|7|13|1903|28|24.57|42.50|7.22|0.00|202.16|687.96|1190.00|12.12|0.00|202.16|214.28|-485.80| +2451117|62223|12607|1089|763927|6765|22403|7|129|1903|24|94.29|104.66|78.49|0.00|1883.76|2262.96|2511.84|0.00|0.00|1883.76|1883.76|-379.20| +2451117|62223|3428|1089|763927|6765|22403|7|26|1903|96|38.54|51.64|34.59|697.33|3320.64|3699.84|4957.44|52.46|697.33|2623.31|2675.77|-1076.53| +2451117|62223|9022|1089|763927|6765|22403|7|74|1903|96|30.26|39.03|31.61|0.00|3034.56|2904.96|3746.88|182.07|0.00|3034.56|3216.63|129.60| +2451117|62223|6494|1089|763927|6765|22403|7|201|1903|12|90.48|117.62|105.85|0.00|1270.20|1085.76|1411.44|76.21|0.00|1270.20|1346.41|184.44| +2451117|62223|11606|1089|763927|6765|22403|7|197|1903|59|33.94|41.06|31.20|478.60|1840.80|2002.46|2422.54|68.11|478.60|1362.20|1430.31|-640.26| +2451117|62223|17821|1089|763927|6765|22403|7|172|1903|35|98.71|141.15|121.38|0.00|4248.30|3454.85|4940.25|297.38|0.00|4248.30|4545.68|793.45| +||1342||763927|6765|||247|1903||||37.43|||4516.66|8626.81|||||-2532.87| +2451117|62223|10940|1089|763927|6765|22403|7|270|1903|99|33.24|37.22|12.65|0.00|1252.35|3290.76|3684.78|62.61|0.00|1252.35|1314.96|-2038.41| +2451117|62223|16132|1089|763927|6765|22403|7|47|1903|29|71.12|100.99|58.57|1155.00|1698.53|2062.48|2928.71|43.48|1155.00|543.53|587.01|-1518.95| +2452580|62742|13452|16687|1305480|7073|41019|10|183|1904|3|26.74|43.58|15.25|0.00|45.75|80.22|130.74|3.66|0.00|45.75|49.41|-34.47| +2452580|62742|7623|16687|1305480|7073|41019|10|88|1904|93|99.94|111.93|107.45|0.00|9992.85|9294.42|10409.49|0.00|0.00|9992.85|9992.85|698.43| +2452580|62742|15090|16687|1305480|7073|41019|10|112|1904|65|2.82|3.86|0.34|0.66|22.10|183.30|250.90|1.71|0.66|21.44|23.15|-161.86| +2452580|62742|13791|16687|1305480|7073|41019|10|34|1904|28|65.41|98.76|78.02|0.00|2184.56|1831.48|2765.28|152.91|0.00|2184.56|2337.47|353.08| +2452580|62742|12384|16687|1305480|7073|41019|10|216|1904|56|47.33|79.04|68.76|0.00|3850.56|2650.48|4426.24|346.55|0.00|3850.56|4197.11|1200.08| +2452580|62742|12612|16687|1305480|7073|41019|10|188|1904|10|63.80|102.71|33.89|0.00|338.90|638.00|1027.10|16.94|0.00|338.90|355.84|-299.10| +2452580|62742|6510|16687|1305480|7073|41019|10|287|1904|85|30.29|33.62|22.18|0.00|1885.30|2574.65|2857.70|131.97|0.00|1885.30|2017.27|-689.35| +2452580|62742|8862|16687|1305480|7073|41019|10|114|1904|69|81.60|134.64|60.58|0.00|4180.02|5630.40|9290.16|125.40|0.00|4180.02|4305.42|-1450.38| +2452580|62742|5232|16687|1305480|7073|41019|10|199|1904|66|96.47|144.70|10.12|0.00|667.92|6367.02|9550.20|0.00|0.00|667.92|667.92|-5699.10| +2452580|62742|385|16687|1305480|7073|41019|10|254|1904|92|36.31|59.18|16.57|0.00|1524.44|3340.52|5444.56|106.71|0.00|1524.44|1631.15|-1816.08| +2452580|62742|6399|16687|1305480|7073|41019|10|45|1904|44|45.68|80.39|67.52|0.00|2970.88|2009.92|3537.16|237.67|0.00|2970.88|3208.55|960.96| +2452580|62742|5724|16687|1305480|7073|41019|10|14|1904|52|19.94|23.92|6.93|306.30|360.36|1036.88|1243.84|1.08|306.30|54.06|55.14|-982.82| +2451865|71147|971|64937|545932|6480|5828|4|267|1905|9|72.41|132.51|98.05|61.77|882.45|651.69|1192.59|73.86|61.77|820.68|894.54|168.99| +2451865|71147|967|64937|545932|6480|5828|4|297|1905|19|65.19|106.91|50.24|0.00|954.56|1238.61|2031.29|76.36|0.00|954.56|1030.92|-284.05| +2451865|71147|15314|64937|545932|6480|5828|4|3|1905|30|57.93|75.88|18.21|0.00|546.30|1737.90|2276.40|32.77|0.00|546.30|579.07|-1191.60| +2451865|71147|9001|64937|545932|6480|5828|4|279|1905|50|81.27|94.27|87.67|0.00|4383.50|4063.50|4713.50|175.34|0.00|4383.50|4558.84|320.00| +2451865|71147|16040|64937|545932|6480|5828|4|41|1905|99|26.52|33.68|29.63|0.00|2933.37|2625.48|3334.32|117.33|0.00|2933.37|3050.70|307.89| +2451865|71147|14039|64937|545932|6480|5828|4|128|1905|42|87.66|152.52|132.69|0.00|5572.98|3681.72|6405.84|390.10|0.00|5572.98|5963.08|1891.26| +2451865|71147|9413|64937|545932|6480|5828|4|143|1905|23|37.75|40.39|32.71|0.00|752.33|868.25|928.97|60.18|0.00|752.33|812.51|-115.92| +2451865|71147|1159|64937|545932|6480|5828|4|219|1905|69|63.02|81.29|53.65|0.00|3701.85|4348.38|5609.01|111.05|0.00|3701.85|3812.90|-646.53| +2451865|71147|17861|64937|545932|6480|5828|4|73|1905|33|14.96|16.90|4.90|0.00|161.70|493.68|557.70|12.93|0.00|161.70|174.63|-331.98| +2451865|71147|2755|64937|545932|6480|5828|4|26|1905|44|66.72|117.42|92.76|81.62|4081.44|2935.68|5166.48|159.99|81.62|3999.82|4159.81|1064.14| +2451865|71147|15287|64937|545932|6480|5828|4|75|1905|43|78.30|98.65|0.00|0.00|0.00|3366.90|4241.95|0.00|0.00|0.00|0.00|-3366.90| +2451865|71147|3308|64937|545932|6480|5828|4|237|1905|2|41.17|71.63|68.76|41.25|137.52|82.34|143.26|7.70|41.25|96.27|103.97|13.93| +2451865|71147|7913|64937|545932|6480|5828|4|166|1905|60|8.49|8.65|4.67|0.00|280.20|509.40|519.00|2.80|0.00|280.20|283.00|-229.20| +2451865|71147|13733||545932|6480|||83|1905|71|20.04|27.25|4.08||289.68||1934.75|11.58||||| +|71147|9499||545932||5828|4|238|1905||31.52|55.79|47.42|0.00|853.56|567.36|1004.22||0.00|853.56||| +||6988|47536||1259|25944||197|1906|42||32.46|16.23|0.00|681.66||||0.00|681.66|681.66|-367.08| +2451284|53663|11948|47536|1398974|1259|25944|2|137|1906|70|63.32|113.34|55.53|0.00|3887.10|4432.40|7933.80|310.96|0.00|3887.10|4198.06|-545.30| +2451284|53663|3679|47536|1398974|1259|25944|2|204|1906|37|69.26|106.66|67.19|0.00|2486.03|2562.62|3946.42|24.86|0.00|2486.03|2510.89|-76.59| +2451284|53663|3994|47536|1398974|1259|25944|2|19|1906|82|5.21|5.57|5.57|0.00|456.74|427.22|456.74|27.40|0.00|456.74|484.14|29.52| +2451284|53663|9796|47536|1398974|1259|25944|2|289|1906|77|36.37|53.10|18.58|1072.99|1430.66|2800.49|4088.70|7.15|1072.99|357.67|364.82|-2442.82| +2451284|53663|8810|47536|1398974|1259|25944|2|97|1906|31|23.48|23.94|10.53|0.00|326.43|727.88|742.14|29.37|0.00|326.43|355.80|-401.45| +2451284|53663|17102|47536|1398974|1259|25944|2|155|1906|79|46.70|49.96|17.98|0.00|1420.42|3689.30|3946.84|113.63|0.00|1420.42|1534.05|-2268.88| +2451284|53663|14680|47536|1398974|1259|25944|2|181|1906|27|92.81|120.65|107.37|0.00|2898.99|2505.87|3257.55|173.93|0.00|2898.99|3072.92|393.12| +2451284|53663|8792|47536|1398974|1259|25944|2|77|1906|45|57.03|75.27|16.55|0.00|744.75|2566.35|3387.15|44.68|0.00|744.75|789.43|-1821.60| +2451284|53663|12728|47536|1398974|1259|25944|2|119|1906|43|93.07|143.32|81.69|0.00|3512.67|4002.01|6162.76|245.88|0.00|3512.67|3758.55|-489.34| +2452441|52032|17431|59236||||10||1907|5||57.97||0.00|153.60|||10.75|0.00|153.60||| +2452441|52032|12015|59236|387750|7017|19131|10|191|1907|76|68.27|135.17|33.79|0.00|2568.04|5188.52|10272.92|102.72|0.00|2568.04|2670.76|-2620.48| +2452441|52032|8629||387750|7017|19131|||1907|||||||100.74|139.84|||43.24||| +2452441|52032|5365|59236|387750|7017|19131|10|25|1907|43|61.07|87.33|70.73|0.00|3041.39|2626.01|3755.19|121.65|0.00|3041.39|3163.04|415.38| +2452441|52032|9030|59236|387750|7017|19131|10|222|1907|29|61.47|118.63|66.43|0.00|1926.47|1782.63|3440.27|173.38|0.00|1926.47|2099.85|143.84| +2452441|52032|14299|59236|387750|7017|19131|10|129|1907|80|30.56|40.64|36.57|0.00|2925.60|2444.80|3251.20|117.02|0.00|2925.60|3042.62|480.80| +2452441|52032|17043|59236|387750|7017|19131|10|149|1907|81|16.85|31.17|22.44|0.00|1817.64|1364.85|2524.77|18.17|0.00|1817.64|1835.81|452.79| +2452441|52032|13423|59236|387750|7017|19131|10|71|1907|52|10.43|13.97|13.69|676.28|711.88|542.36|726.44|0.00|676.28|35.60|35.60|-506.76| +|52032|10428|||7017|19131|||1907||||1.19|0.00||||2.24|0.00|24.99|27.23|| +2452441|52032|6114|59236|387750|7017|19131|10|247|1907|67|75.34|93.42|70.06|0.00|4694.02|5047.78|6259.14|234.70|0.00|4694.02|4928.72|-353.76| +2451203|63900|17629|68742|1086180|2018|9503|8|291|1908|56|61.87|74.24|1.48|0.00|82.88|3464.72|4157.44|3.31|0.00|82.88|86.19|-3381.84| +2451203|63900|8773|68742|1086180|2018|9503|8|150|1908|59|27.03|31.89|25.19|297.24|1486.21|1594.77|1881.51|47.55|297.24|1188.97|1236.52|-405.80| +2451203|63900|12934|68742|1086180|2018|9503|8|202|1908|49|52.98|92.18|87.57|0.00|4290.93|2596.02|4516.82|85.81|0.00|4290.93|4376.74|1694.91| +2451203|63900|9572|68742|1086180|2018|9503|8|79|1908|11|1.42|1.74|0.05|0.00|0.55|15.62|19.14|0.01|0.00|0.55|0.56|-15.07| +2451203|63900|6232|68742|1086180|2018|9503|8|66|1908|7|55.45|81.51|57.87|0.00|405.09|388.15|570.57|12.15|0.00|405.09|417.24|16.94| +2451203|63900|6224|68742|1086180|2018|9503|8|37|1908|38|48.21|75.20|47.37|360.01|1800.06|1831.98|2857.60|72.00|360.01|1440.05|1512.05|-391.93| +2451203|63900|1198|68742|1086180|2018|9503|8|221|1908|30|84.00|112.56|97.92|0.00|2937.60|2520.00|3376.80|29.37|0.00|2937.60|2966.97|417.60| +2451203|63900|8866|68742|1086180|2018|9503|8|174|1908|51|64.19|112.33|102.22|0.00|5213.22|3273.69|5728.83|260.66|0.00|5213.22|5473.88|1939.53| +2451203|63900|12385|68742|1086180|2018|9503|8|245|1908|73|72.84|104.88|20.97|0.00|1530.81|5317.32|7656.24|137.77|0.00|1530.81|1668.58|-3786.51| +2451203|63900|14929|68742|1086180|2018|9503|8|269|1908|62|98.70|154.95|12.39|0.00|768.18|6119.40|9606.90|46.09|0.00|768.18|814.27|-5351.22| +2451203|63900|6880|68742|1086180|2018|9503|8|167|1908|45|32.57|39.40|6.69|36.12|301.05|1465.65|1773.00|18.54|36.12|264.93|283.47|-1200.72| +2451203|63900|17287|68742|1086180||9503|8||1908||81.77|86.67|||2366.00||3033.45||||1364.70|-1536.99| +2451203|63900|9308|68742|1086180|2018|9503|8|198|1908|73|58.51|99.46|77.57|0.00|5662.61|4271.23|7260.58|226.50|0.00|5662.61|5889.11|1391.38| +2451203|63900|1042|68742|1086180|2018|9503|8|137|1908|48|72.90|85.29|50.32|0.00|2415.36|3499.20|4093.92|24.15|0.00|2415.36|2439.51|-1083.84| +2451203|63900|15958|68742|1086180|2018|9503|8|34|1908|85|20.37|27.29|6.00|0.00|510.00|1731.45|2319.65|45.90|0.00|510.00|555.90|-1221.45| +2452209|62903|14731|1542|923113|3884|33902|4|43|1909|13|46.91|70.83|61.62|0.00|801.06|609.83|920.79|8.01|0.00|801.06|809.07|191.23| +2452209|62903|13061|1542|923113|3884|33902|4|112|1909|98|90.89|128.15|20.50|0.00|2009.00|8907.22|12558.70|100.45|0.00|2009.00|2109.45|-6898.22| +2452209|62903|16559|1542|923113|3884|33902|4|138|1909|44|24.24|34.66|1.03|0.00|45.32|1066.56|1525.04|1.35|0.00|45.32|46.67|-1021.24| +2452209|62903|8911|1542|923113|3884|33902|4|242|1909|93|46.84|47.30|29.79|0.00|2770.47|4356.12|4398.90|249.34|0.00|2770.47|3019.81|-1585.65| +2452209|62903|7189|1542||3884||4|194|1909||93.92||||125.19||2087.80|0.00||||-1095.77| +2452209|62903|10725|1542|923113|3884|33902|4|225|1909|13|58.05|60.95|35.35|0.00|459.55|754.65|792.35|18.38|0.00|459.55|477.93|-295.10| +2452209|62903|17953|1542|923113|3884|33902|4|150|1909|53|20.40|30.39|25.83|0.00|1368.99|1081.20|1610.67|109.51|0.00|1368.99|1478.50|287.79| +2452209|62903|4841|1542|923113|3884|33902|4|124|1909|33|6.84|12.79|8.69|0.00|286.77|225.72|422.07|5.73|0.00|286.77|292.50|61.05| +2452209|62903|5769|1542|923113|3884|33902|4|24|1909|25|40.54|47.02|15.98|0.00|399.50|1013.50|1175.50|35.95|0.00|399.50|435.45|-614.00| +2452209|62903|6915|1542|923113|3884|33902|4|251|1909|29|26.53|35.55|0.71|0.00|20.59|769.37|1030.95|0.20|0.00|20.59|20.79|-748.78| +2452209|62903|14613|1542|923113|3884|33902|4|279|1909|77|40.99|66.40|6.64|0.00|511.28|3156.23|5112.80|25.56|0.00|511.28|536.84|-2644.95| +2452209|62903|3655|1542|923113|3884|33902|4|75|1909|77|57.80|58.37|8.75|0.00|673.75|4450.60|4494.49|33.68|0.00|673.75|707.43|-3776.85| +2452209|62903|15955|1542|923113|3884|33902|4|142|1909|31|84.46|85.30|61.41|0.00|1903.71|2618.26|2644.30|19.03|0.00|1903.71|1922.74|-714.55| +2452209|62903|14809|1542|923113|3884|33902|4|43|1909|16|36.20|67.33|56.55|0.00|904.80|579.20|1077.28|18.09|0.00|904.80|922.89|325.60| +2452209|62903|2799|1542|923113|3884|33902|4|238|1909|60|57.28|57.28|18.32|0.00|1099.20|3436.80|3436.80|0.00|0.00|1099.20|1099.20|-2337.60| +2452209|62903|16655|1542|923113|3884|33902|4|25|1909|81|43.41|46.88|41.25|0.00|3341.25|3516.21|3797.28|300.71|0.00|3341.25|3641.96|-174.96| +2452314|64100|11419|69160|542995|30|40218|8|244|1910|62|61.13|119.81|40.73|1338.38|2525.26|3790.06|7428.22|35.60|1338.38|1186.88|1222.48|-2603.18| +2452314|64100|10020|69160|542995|30|40218|8|298|1910|49|19.58|35.83|5.01|0.00|245.49|959.42|1755.67|19.63|0.00|245.49|265.12|-713.93| +2452314|64100|7033|69160|542995|30|40218|8|276|1910|60|95.82|160.97|99.80|0.00|5988.00|5749.20|9658.20|299.40|0.00|5988.00|6287.40|238.80| +2452314|64100|16795|69160|542995|30|40218|8|71|1910|2|90.29|123.69|71.74|0.00|143.48|180.58|247.38|8.60|0.00|143.48|152.08|-37.10| +2452314|64100|16746|69160|542995|30|40218|8|55|1910|17|89.77|169.66|54.29|433.77|922.93|1526.09|2884.22|34.24|433.77|489.16|523.40|-1036.93| +2452314|64100|4302|69160|542995|30|40218|8|146|1910|74|55.66|74.02|4.44|0.00|328.56|4118.84|5477.48|19.71|0.00|328.56|348.27|-3790.28| +2452314|64100|16851|69160|542995|30|40218|8|199|1910|26|2.79|3.87|1.54|0.00|40.04|72.54|100.62|3.60|0.00|40.04|43.64|-32.50| +2452314|64100|4801|69160|542995|30|40218|8|198|1910|90|17.28|27.47|7.69|55.36|692.10|1555.20|2472.30|25.46|55.36|636.74|662.20|-918.46| +2452314|64100|3180|69160|542995|30|40218|8|294|1910|81|92.30|113.52|91.95|0.00|7447.95|7476.30|9195.12|521.35|0.00|7447.95|7969.30|-28.35| +2452314|64100|11805|69160|542995|30|40218|8|200|1910|30|26.79|42.32|27.93|0.00|837.90|803.70|1269.60|25.13|0.00|837.90|863.03|34.20| +2452314|64100|9673|69160|542995|30|40218|8|142|1910|54|33.52|40.22|12.06|0.00|651.24|1810.08|2171.88|32.56|0.00|651.24|683.80|-1158.84| +2452314|64100|8401|69160|542995|30|40218|8|128|1910|88|66.82|98.22|40.27|0.00|3543.76|5880.16|8643.36|35.43|0.00|3543.76|3579.19|-2336.40| +2451440|62011|6601|77559|1194484|4656|49981|8|243|1911|66|27.10|36.58|2.19|0.00|144.54|1788.60|2414.28|8.67|0.00|144.54|153.21|-1644.06| +2451440|62011|10234|77559|1194484|4656|49981|8|60|1911|57|96.98|157.10|84.83|0.00|4835.31|5527.86|8954.70|96.70|0.00|4835.31|4932.01|-692.55| +2451440|62011|10582|77559|1194484|4656|49981|8|186|1911|95|30.11|43.35|29.04|0.00|2758.80|2860.45|4118.25|220.70|0.00|2758.80|2979.50|-101.65| +2451440|62011|11017|77559|1194484|4656|49981|8|50|1911|76|55.29|75.74|38.62|0.00|2935.12|4202.04|5756.24|29.35|0.00|2935.12|2964.47|-1266.92| +2451440|62011|15739|77559|1194484|4656|49981|8|16|1911|62|57.14|74.85|48.65|0.00|3016.30|3542.68|4640.70|60.32|0.00|3016.30|3076.62|-526.38| +2451440|62011|16141|77559|1194484|4656|49981|8|70|1911|20|50.13|90.73|43.55|0.00|871.00|1002.60|1814.60|0.00|0.00|871.00|871.00|-131.60| +2451440|62011|12088|77559|1194484|4656|49981|8|67|1911|64|5.35|6.36|0.82|0.00|52.48|342.40|407.04|3.67|0.00|52.48|56.15|-289.92| +2451440|62011|11413|77559|1194484|4656|49981|8|235|1911|67|63.00|85.68|26.56|782.98|1779.52|4221.00|5740.56|69.75|782.98|996.54|1066.29|-3224.46| +2451440|62011|5540|77559|1194484|4656|49981|8|184|1911|47|78.72|157.44|36.21|0.00|1701.87|3699.84|7399.68|153.16|0.00|1701.87|1855.03|-1997.97| +2451440|62011|13108|77559|1194484|4656|49981|8|126|1911|13|86.10|99.01|22.77|44.40|296.01|1119.30|1287.13|20.12|44.40|251.61|271.73|-867.69| +2451440|62011|9862|77559|1194484|4656|49981|8|188|1911|24|7.56|14.28|9.56|0.00|229.44|181.44|342.72|2.29|0.00|229.44|231.73|48.00| +2451440|62011|16999|77559|1194484|4656|49981|8|133|1911|91|68.06|78.94|63.15|0.00|5746.65|6193.46|7183.54|287.33|0.00|5746.65|6033.98|-446.81| +2451440|62011|10538|77559|1194484|4656|49981|8|224|1911|42|74.85|145.20|91.47|0.00|3841.74|3143.70|6098.40|307.33|0.00|3841.74|4149.07|698.04| +2452494|36396|10717|73855|1727945|2035|3325|4|101|1912|69|63.25|96.77|63.86|2643.80|4406.34|4364.25|6677.13|0.00|2643.80|1762.54|1762.54|-2601.71| +2452494|36396|11766|73855|1727945|2035|3325|4|285|1912|86|67.19|119.59|98.06|0.00|8433.16|5778.34|10284.74|758.98|0.00|8433.16|9192.14|2654.82| +2452494|36396|14007|73855|1727945|2035|3325|4|195|1912|70|36.81|65.15|35.83|0.00|2508.10|2576.70|4560.50|50.16|0.00|2508.10|2558.26|-68.60| +2452494|36396|1515|73855|1727945|2035|3325|4|103|1912|27|1.10|1.41|0.21|0.00|5.67|29.70|38.07|0.00|0.00|5.67|5.67|-24.03| +2452494|36396|17431|73855|1727945|2035|3325|4|277|1912|65|9.63|16.46|7.73|0.00|502.45|625.95|1069.90|40.19|0.00|502.45|542.64|-123.50| +2452494||12015|||||4||1912|44|35.22|||||||||2463.56||| +2452494|36396|8629|73855|1727945|2035|3325|4|27|1912|73|48.92|48.92|7.33|0.00|535.09|3571.16|3571.16|0.00|0.00|535.09|535.09|-3036.07| +2452494|36396|5365|73855|1727945|2035|3325|4|20|1912|4|79.40|88.92|48.01|0.00|192.04|317.60|355.68|3.84|0.00|192.04|195.88|-125.56| +2452132|56988|6217|99625|17815|6405|6995|1|77|1913|48|18.01|26.29|5.25|0.00|252.00|864.48|1261.92|15.12|0.00|252.00|267.12|-612.48| +2452132|56988|7463|99625|17815|6405|6995|1|258|1913|64|36.86|38.70|7.35|376.32|470.40|2359.04|2476.80|5.64|376.32|94.08|99.72|-2264.96| +2452132|56988|13875|99625||6405|6995||14|1913||26.99|41.02||0.00|1476.45|1214.55|1845.90|44.29|0.00|1476.45||| +2452132|56988|14811|99625|17815|6405|6995|1|280|1913|79|9.61|12.87|9.78|0.00|772.62|759.19|1016.73|69.53|0.00|772.62|842.15|13.43| +2452132|56988|6685|99625|17815|6405|6995|1|138|1913|65|47.22|65.16|50.82|0.00|3303.30|3069.30|4235.40|198.19|0.00|3303.30|3501.49|234.00| +2452132|56988|17147|99625|17815|6405|6995|1|286|1913|24|23.91|47.82|28.69|0.00|688.56|573.84|1147.68|0.00|0.00|688.56|688.56|114.72| +2452132|56988|15863|99625|17815|6405|6995|1|197|1913|100|92.59|165.73|24.85|1068.55|2485.00|9259.00|16573.00|84.98|1068.55|1416.45|1501.43|-7842.55| +2452132|56988|13757|99625|17815|6405|6995|1|79|1913|83|70.20|127.76|106.04|0.00|8801.32|5826.60|10604.08|176.02|0.00|8801.32|8977.34|2974.72| +2452132|56988|12905|99625|17815|6405|6995|1|38|1913|95|12.60|22.93|18.80|0.00|1786.00|1197.00|2178.35|0.00|0.00|1786.00|1786.00|589.00| +2452132|56988|2643|99625|17815|6405|6995|1|281|1913|82|74.98|86.22|20.69|0.00|1696.58|6148.36|7070.04|33.93|0.00|1696.58|1730.51|-4451.78| +2452132|56988|2217|99625|17815|6405|6995|1|262|1913|22|14.39|19.28|0.77|0.00|16.94|316.58|424.16|0.33|0.00|16.94|17.27|-299.64| +2452132|56988|5655|99625|17815|6405|6995|1|42|1913|97|68.50|108.23|23.81|0.00|2309.57|6644.50|10498.31|23.09|0.00|2309.57|2332.66|-4334.93| +2452132|56988|15383|99625|17815|6405|6995|1|298|1913|20|76.90|103.81|41.52|423.50|830.40|1538.00|2076.20|12.20|423.50|406.90|419.10|-1131.10| +2452132|56988|13473|99625|17815|6405|6995|1|40|1913|72|81.56|129.68|114.11|0.00|8215.92|5872.32|9336.96|328.63|0.00|8215.92|8544.55|2343.60| +2452132|56988|10879|99625|17815|6405|6995|1|180|1913|99|50.37|75.05|66.79|0.00|6612.21|4986.63|7429.95|66.12|0.00|6612.21|6678.33|1625.58| +2451858|34681|14534|90529|442121|2066|58|2|122|1914|84|47.90|49.33|19.73|0.00|1657.32|4023.60|4143.72|149.15|0.00|1657.32|1806.47|-2366.28| +2451858|34681|2492|90529|442121|2066|58|2|192|1914|35|56.66|99.15|4.95|67.56|173.25|1983.10|3470.25|0.00|67.56|105.69|105.69|-1877.41| +2451858|34681|13549|90529|442121|2066|58|2|108|1914|89|11.26|12.38|11.26|0.00|1002.14|1002.14|1101.82|80.17|0.00|1002.14|1082.31|0.00| +2451858|34681|16826|90529|442121|2066|58|2|204|1914|70|43.21|67.40|45.83|0.00|3208.10|3024.70|4718.00|64.16|0.00|3208.10|3272.26|183.40| +2451858|34681|7313|90529|442121|2066|58|2|213|1914|18|44.00|74.80|18.70|0.00|336.60|792.00|1346.40|13.46|0.00|336.60|350.06|-455.40| +2451858|34681|6689|90529|442121|2066|58|2|199|1914|19|35.16|37.62|1.50|0.00|28.50|668.04|714.78|1.99|0.00|28.50|30.49|-639.54| +2451858|34681|6392|90529|442121|2066|58|2|10|1914|77|32.74|62.53|35.01|1078.30|2695.77|2520.98|4814.81|97.04|1078.30|1617.47|1714.51|-903.51| +2451858|34681|9913|90529|442121|2066|58|2|155|1914|71|27.34|28.70|10.33|0.00|733.43|1941.14|2037.70|22.00|0.00|733.43|755.43|-1207.71| +2452537|47420|11883|88983|549654|4278|6949|4|28|1915|92|98.08|187.33|177.96|15389.98|16372.32|9023.36|17234.36|19.64|15389.98|982.34|1001.98|-8041.02| +2452537|47420|15915|88983|549654|4278|6949|4|114|1915|29|80.23|99.48|75.60|0.00|2192.40|2326.67|2884.92|153.46|0.00|2192.40|2345.86|-134.27| +2452537|47420|3300|88983|549654|4278|6949|4|25|1915|96|2.44|4.88|0.53|0.00|50.88|234.24|468.48|1.01|0.00|50.88|51.89|-183.36| +2452537|47420|4830|88983|549654|4278|6949|4|58|1915|10|18.13|26.83|0.80|1.76|8.00|181.30|268.30|0.49|1.76|6.24|6.73|-175.06| +2452537|47420|685|88983|549654|4278|6949|4|139|1915|36|50.39|87.67|64.87|0.00|2335.32|1814.04|3156.12|0.00|0.00|2335.32|2335.32|521.28| +2452537|47420|105|88983|549654|4278|6949|4|82|1915|83|53.79|90.90|56.35|0.00|4677.05|4464.57|7544.70|0.00|0.00|4677.05|4677.05|212.48| +2452537|47420|10105|88983|549654|4278|6949|4|85|1915|91|27.05|30.02|5.70|67.43|518.70|2461.55|2731.82|27.07|67.43|451.27|478.34|-2010.28| +2452537|47420|72|88983|549654|4278|6949|4|37|1915|25|58.26|90.30|27.09|0.00|677.25|1456.50|2257.50|40.63|0.00|677.25|717.88|-779.25| +2452537|47420|9211|88983|549654|4278|6949|4|58|1915|21|25.32|39.24|20.01|0.00|420.21|531.72|824.04|12.60|0.00|420.21|432.81|-111.51| +2452537|47420|1980|88983|549654|4278|6949|4|228|1915|78|1.67|3.15|1.70|0.00|132.60|130.26|245.70|10.60|0.00|132.60|143.20|2.34| +2451992|72975|49|59735|297116|450|21384|10|244|1916|13|48.53|92.69|83.42|0.00|1084.46|630.89|1204.97|54.22|0.00|1084.46|1138.68|453.57| +2451992|72975|4583|59735|297116|450|21384|10|77|1916|100|59.18|108.29|9.74|759.72|974.00|5918.00|10829.00|17.14|759.72|214.28|231.42|-5703.72| +2451992|72975|13539|59735|297116|450|21384|10|19|1916|43|66.67|83.33|4.16|0.00|178.88|2866.81|3583.19|12.52|0.00|178.88|191.40|-2687.93| +2451992|72975|3249|59735|297116|450|21384|10|150|1916|84|68.76|96.95|48.47|0.00|4071.48|5775.84|8143.80|285.00|0.00|4071.48|4356.48|-1704.36| +2451992|72975|4683|59735|297116|450|21384|10|208|1916|86|12.71|15.88|11.91|358.49|1024.26|1093.06|1365.68|19.97|358.49|665.77|685.74|-427.29| +2451992|72975|5527|59735|297116|450|21384|10|191|1916|76|18.01|34.57|5.87|0.00|446.12|1368.76|2627.32|4.46|0.00|446.12|450.58|-922.64| +2451992|72975|1553|59735|297116|450|21384|10|125|1916|54|40.64|44.70|20.11|0.00|1085.94|2194.56|2413.80|43.43|0.00|1085.94|1129.37|-1108.62| +2451992|72975|173|59735|297116|450|21384|10|119|1916|26|54.44|83.29|70.79|0.00|1840.54|1415.44|2165.54|110.43|0.00|1840.54|1950.97|425.10| +2451992|72975|6169|59735|297116|450|21384|10|244|1916|87|37.72|66.01|58.08|0.00|5052.96|3281.64|5742.87|454.76|0.00|5052.96|5507.72|1771.32| +2451992|72975|13147|59735|297116|450|21384|10|136|1916|3|14.63|23.99|5.75|0.00|17.25|43.89|71.97|1.20|0.00|17.25|18.45|-26.64| +2451992|72975|7683|59735|297116|450|21384|10|202|1916|92|88.26|99.73|49.86|0.00|4587.12|8119.92|9175.16|137.61|0.00|4587.12|4724.73|-3532.80| +2451992|72975|13517|59735|297116|450|21384|10|217|1916|15|83.11|110.53|71.84|204.74|1077.60|1246.65|1657.95|61.10|204.74|872.86|933.96|-373.79| +2451992||13839|59735|297116|||||1916|||13.04||0.00|197.78||404.24|1.97|0.00|197.78|199.75|| +2451062|46731|8984|1550|129257|517|49024|1|98|1917|47|11.73|21.23|21.01|0.00|987.47|551.31|997.81|69.12|0.00|987.47|1056.59|436.16| +2451062|46731|14521|1550|129257|517|49024|1|87|1917|69|66.92|101.71|78.31|4809.01|5403.39|4617.48|7017.99|47.55|4809.01|594.38|641.93|-4023.10| +2451062|46731|17452|1550|129257|517|49024|1|167|1917|28|69.76|131.84|40.87|1098.58|1144.36|1953.28|3691.52|1.37|1098.58|45.78|47.15|-1907.50| +2451062|46731|1483|1550|129257|517|49024|1|69|1917|18|52.75|58.55|31.03|0.00|558.54|949.50|1053.90|22.34|0.00|558.54|580.88|-390.96| +2451062|46731|2368|1550|129257|517|49024|1|89|1917|37|87.27|127.41|100.65|2122.70|3724.05|3228.99|4714.17|16.01|2122.70|1601.35|1617.36|-1627.64| +2451062|46731|3592|1550|129257|517|49024|1|124|1917|31|78.20|147.01|145.53|0.00|4511.43|2424.20|4557.31|180.45|0.00|4511.43|4691.88|2087.23| +2451062|46731|14200|1550|129257|517|49024|1|18|1917|69|73.50|77.17|0.00|0.00|0.00|5071.50|5324.73|0.00|0.00|0.00|0.00|-5071.50| +2451062|46731|10657|1550|129257|517|49024|1|70|1917|66|45.24|64.69|39.46|0.00|2604.36|2985.84|4269.54|78.13|0.00|2604.36|2682.49|-381.48| +2451062|46731|5522|1550|129257|517|49024|1|6|1917|57|78.43|83.13|44.89|0.00|2558.73|4470.51|4738.41|127.93|0.00|2558.73|2686.66|-1911.78| +2451062|46731|5710|1550|129257|517|49024|1|226|1917|25|77.32|144.58|23.13|0.00|578.25|1933.00|3614.50|46.26|0.00|578.25|624.51|-1354.75| +2452631|51701|14733|96927|974495|1663|29631|1|132|1918|73|44.50|71.20|3.56|0.00|259.88|3248.50|5197.60|0.00|0.00|259.88|259.88|-2988.62| +2452631|51701|5713|96927|974495|1663|29631|1|175|1918|8|22.91|35.96|26.97|0.00|215.76|183.28|287.68|15.10|0.00|215.76|230.86|32.48| +2452631|51701|2329|96927|974495|1663|29631|1|99|1918|25|82.28|96.26|96.26|0.00|2406.50|2057.00|2406.50|120.32|0.00|2406.50|2526.82|349.50| +2452631|51701|10893|96927|974495|1663|29631|1|215|1918|7|76.96|153.92|55.41|0.00|387.87|538.72|1077.44|15.51|0.00|387.87|403.38|-150.85| +2452631|51701|3420|96927|974495|1663|29631|1|157|1918|55|74.85|130.23|121.11|4662.73|6661.05|4116.75|7162.65|159.86|4662.73|1998.32|2158.18|-2118.43| +2452631|51701|16699|96927|974495|1663|29631|1|257|1918|3|74.89|92.11|28.55|0.00|85.65|224.67|276.33|4.28|0.00|85.65|89.93|-139.02| +2452631|51701|8508|96927|974495|1663|29631|1|299|1918|73|66.95|128.54|8.99|0.00|656.27|4887.35|9383.42|19.68|0.00|656.27|675.95|-4231.08| +2452631|51701|4875|96927|974495|1663|29631|1|195|1918|28|68.40|95.76|72.77|0.00|2037.56|1915.20|2681.28|0.00|0.00|2037.56|2037.56|122.36| +2452631|51701|17826|96927|974495|1663|29631|1|298|1918|11|40.29|42.70|14.09|0.00|154.99|443.19|469.70|10.84|0.00|154.99|165.83|-288.20| +2452631|51701|7023|96927|974495|1663|29631|1|129|1918|75|7.10|10.43|7.61|399.52|570.75|532.50|782.25|11.98|399.52|171.23|183.21|-361.27| +2452631|51701|17065|96927|974495|1663|29631|1|102|1918|2|61.81|106.93|22.45|0.00|44.90|123.62|213.86|0.44|0.00|44.90|45.34|-78.72| +2452631|51701|17473|96927||||1|60|1918|64||41.81|||1471.36|||||1471.36|1530.21|40.32| +2452124|37187|17301|73165|1244718|6318|1071|8|98|1919|100|89.61|91.40|83.17|0.00|8317.00|8961.00|9140.00|83.17|0.00|8317.00|8400.17|-644.00| +2452124|37187|9593|73165|1244718|6318|1071|8|65|1919|46|44.24|67.24|16.13|0.00|741.98|2035.04|3093.04|37.09|0.00|741.98|779.07|-1293.06| +2452124|37187|939|73165|1244718|6318|1071|8|60|1919|29|71.30|78.43|55.68|0.00|1614.72|2067.70|2274.47|113.03|0.00|1614.72|1727.75|-452.98| +2452124|37187|205|73165|1244718|6318|1071|8|97|1919|51|16.23|30.02|12.90|0.00|657.90|827.73|1531.02|13.15|0.00|657.90|671.05|-169.83| +2452124|37187|12195|73165|1244718|6318|1071|8|262|1919|4|35.86|57.73|52.53|0.00|210.12|143.44|230.92|18.91|0.00|210.12|229.03|66.68| +2452124|37187|6419|73165|1244718|6318|1071|8|254|1919|35|91.65|154.88|105.31|589.73|3685.85|3207.75|5420.80|185.76|589.73|3096.12|3281.88|-111.63| +2452124|37187|12255|73165|1244718|6318|1071|8|110|1919|24|95.27|155.29|138.20|0.00|3316.80|2286.48|3726.96|0.00|0.00|3316.80|3316.80|1030.32| +2452124|37187|593|73165|1244718|6318|1071|8|53|1919|38|40.62|70.27|7.02|0.00|266.76|1543.56|2670.26|10.67|0.00|266.76|277.43|-1276.80| +2452124|37187|1411|73165|1244718|6318|1071|8|4|1919|89|20.76|26.36|17.92|0.00|1594.88|1847.64|2346.04|31.89|0.00|1594.88|1626.77|-252.76| +2452124|37187|13915|73165|1244718|6318|1071|8|45|1919|34|87.63|151.59|144.01|0.00|4896.34|2979.42|5154.06|48.96|0.00|4896.34|4945.30|1916.92| +2452124|37187|14145|73165|1244718|6318|1071|8|221|1919|38|19.75|36.93|15.14|460.25|575.32|750.50|1403.34|0.00|460.25|115.07|115.07|-635.43| +2450849|62199|679|54952|839648|3544|42287|4|105|1920|10|80.35|116.50|10.48|0.00|104.80|803.50|1165.00|9.43|0.00|104.80|114.23|-698.70| +2450849|62199|2374|54952|839648|3544|42287|4|248|1920|87|50.81|62.49|0.00|0.00|0.00|4420.47|5436.63|0.00|0.00|0.00|0.00|-4420.47| +2450849|62199|12481|54952|839648|3544|42287|4|243|1920|29|98.07|172.60|120.82|0.00|3503.78|2844.03|5005.40|210.22|0.00|3503.78|3714.00|659.75| +2450849|62199|4558|54952|839648|3544|42287|4|258|1920|60|78.37|152.82|19.86|0.00|1191.60|4702.20|9169.20|0.00|0.00|1191.60|1191.60|-3510.60| +2450849|62199|13582|54952|839648|3544|42287|4|220|1920|44|47.29|66.20|19.19|0.00|844.36|2080.76|2912.80|16.88|0.00|844.36|861.24|-1236.40| +2450849|62199|13784|54952|839648|3544|42287|4|288|1920|14|52.07|87.47|16.61|0.00|232.54|728.98|1224.58|13.95|0.00|232.54|246.49|-496.44| +2450849|62199|14828|54952|839648|3544|42287|4|162|1920|19|37.93|54.23|33.08|603.37|628.52|720.67|1030.37|0.75|603.37|25.15|25.90|-695.52| +2450849|62199|602|54952|839648|3544|42287|4|105|1920|86|2.51|4.54|2.58|0.00|221.88|215.86|390.44|19.96|0.00|221.88|241.84|6.02| +2450849|62199|12547|54952|839648|3544|42287|4|12|1920|17|63.02|86.96|61.74|0.00|1049.58|1071.34|1478.32|94.46|0.00|1049.58|1144.04|-21.76| +2452204|39001|3031|32681|862329|3977|17625|4|217|1921|43|74.12|104.50|5.22|0.00|224.46|3187.16|4493.50|20.20|0.00|224.46|244.66|-2962.70| +2452204|39001|16677|32681|862329|3977|17625|4|82|1921|47|30.18|31.38|18.82|0.00|884.54|1418.46|1474.86|35.38|0.00|884.54|919.92|-533.92| +2452204|39001|4525|32681|862329|3977|17625|4|40|1921|46|27.43|47.72|15.74|0.00|724.04|1261.78|2195.12|43.44|0.00|724.04|767.48|-537.74| +2452204|39001|4215|32681|862329|3977|17625|4|23|1921|61|96.81|101.65|47.77|2709.99|2913.97|5905.41|6200.65|6.11|2709.99|203.98|210.09|-5701.43| +2452204|39001|991|32681|862329|3977|17625|4|66|1921|68|27.09|34.13|13.31|18.10|905.08|1842.12|2320.84|35.47|18.10|886.98|922.45|-955.14| +2452204|39001|16279|32681|862329|3977|17625|4|273|1921|89|10.23|10.33|5.37|0.00|477.93|910.47|919.37|0.00|0.00|477.93|477.93|-432.54| +|39001|16489|||3977||||1921|42|48.14|||0.00|1315.86|2021.88||78.95|0.00|||-706.02| +2452204|39001|1817|32681|862329|3977|17625|4|10|1921|37|48.92|67.02|43.56|386.81|1611.72|1810.04|2479.74|12.24|386.81|1224.91|1237.15|-585.13| +2452204|39001|7573|32681|862329|3977|17625|4|132|1921|17|22.31|41.71|21.27|0.00|361.59|379.27|709.07|18.07|0.00|361.59|379.66|-17.68| +|39001|7521|32681|862329|3977|17625|4||1921|85|47.65|70.99|19.16|0.00||4050.25|6034.15|130.28|0.00|||-2421.65| +2452204|39001|5795|32681|862329|3977|17625|4|55|1921|98|36.09|54.85|51.01|0.00|4998.98|3536.82|5375.30|349.92|0.00|4998.98|5348.90|1462.16| +2452379|58810|15852|8355|963808|1661|38761|7|211|1922|24|97.10|142.73|34.25|0.00|822.00|2330.40|3425.52|49.32|0.00|822.00|871.32|-1508.40| +2452379|58810|7639|8355|963808|1661|38761|7|219|1922|1|37.98|72.54|67.46|52.61|67.46|37.98|72.54|1.33|52.61|14.85|16.18|-23.13| +2452379|58810|6061|8355|963808|1661|38761|7|9|1922|90|61.49|70.09|69.38|0.00|6244.20|5534.10|6308.10|0.00|0.00|6244.20|6244.20|710.10| +2452379|58810|6792|8355|963808|1661|38761|7|39|1922|29|51.53|95.33|86.75|0.00|2515.75|1494.37|2764.57|125.78|0.00|2515.75|2641.53|1021.38| +2452379|58810|11431|8355|963808|1661|38761|7|210|1922|100|81.44|114.01|20.52|0.00|2052.00|8144.00|11401.00|41.04|0.00|2052.00|2093.04|-6092.00| +2452379|58810|12691|8355|963808|1661|38761|7|92|1922|36|34.68|53.75|24.72|213.58|889.92|1248.48|1935.00|33.81|213.58|676.34|710.15|-572.14| +2452379|58810|17559|8355|963808|1661|38761|7|105|1922|25|4.16|5.11|1.22|20.74|30.50|104.00|127.75|0.19|20.74|9.76|9.95|-94.24| +2452379|58810|13629|8355|963808|1661|38761|7|260|1922|65|3.75|6.11|0.91|0.00|59.15|243.75|397.15|4.14|0.00|59.15|63.29|-184.60| +2452379|58810|13519|8355|963808|1661|38761|7|160|1922|91|90.53|92.34|34.16|0.00|3108.56|8238.23|8402.94|186.51|0.00|3108.56|3295.07|-5129.67| +2452379|58810|13983|8355|963808|1661|38761|7|147|1922|5|12.89|18.94|14.77|0.00|73.85|64.45|94.70|2.95|0.00|73.85|76.80|9.40| +2452379|58810|1831|8355|963808|1661|38761|7|177|1922|98|27.81|40.32|6.04|0.00|591.92|2725.38|3951.36|35.51|0.00|591.92|627.43|-2133.46| +2452379|58810|10962|8355|963808|1661|38761|7|50|1922|100|38.30|76.21|51.06|0.00|5106.00|3830.00|7621.00|459.54|0.00|5106.00|5565.54|1276.00| +|53442|11777|81971||3700|35541|||1923|16|||151.46|0.00||1316.16||48.46|0.00||2471.82|1107.20| +2452088|53442|13421|81971|667459|3700|35541|4|24|1923|25|38.72|47.62|24.28|0.00|607.00|968.00|1190.50|30.35|0.00|607.00|637.35|-361.00| +2452088|53442|7963|81971|667459|3700|35541|4|223|1923|20|3.88|6.55|5.76|33.40|115.20|77.60|131.00|1.63|33.40|81.80|83.43|4.20| +2452088|53442|6731|81971|667459|3700|35541|4|254|1923|8|44.33|80.68|62.12|173.93|496.96|354.64|645.44|16.15|173.93|323.03|339.18|-31.61| +2452088|53442|11177|81971|667459|3700|35541|4|240|1923|42|54.16|56.32|46.74|0.00|1963.08|2274.72|2365.44|19.63|0.00|1963.08|1982.71|-311.64| +2452088|53442|2875|81971|667459|3700|35541|4|76|1923|32|23.85|25.51|25.25|0.00|808.00|763.20|816.32|8.08|0.00|808.00|816.08|44.80| +2452088|53442|16815|81971|667459|3700|35541|4|117|1923|43|78.20|100.09|52.04|0.00|2237.72|3362.60|4303.87|156.64|0.00|2237.72|2394.36|-1124.88| +2452088|53442|11485|81971|667459|3700|35541|4|120|1923|21|13.87|19.55|1.95|0.00|40.95|291.27|410.55|2.45|0.00|40.95|43.40|-250.32| +2452511|41155|16998|39648|50150|7052|41554|1|90|1924|21|63.97|97.23|60.28|0.00|1265.88|1343.37|2041.83|37.97|0.00|1265.88|1303.85|-77.49| +2452511|41155|10719|39648|50150|7052|41554|1|259|1924|24|13.14|21.81|13.30|95.76|319.20|315.36|523.44|8.93|95.76|223.44|232.37|-91.92| +2452511|41155|6858|39648|50150|7052|41554|1|147|1924|3|8.38|12.15|1.09|0.00|3.27|25.14|36.45|0.06|0.00|3.27|3.33|-21.87| +2452511|41155|6523|39648|50150|7052|41554|1|38|1924|87|16.56|28.81|2.59|0.00|225.33|1440.72|2506.47|4.50|0.00|225.33|229.83|-1215.39| +2452511|41155|10164|39648|50150|7052|41554|1|117|1924|26|90.31|128.24|35.90|0.00|933.40|2348.06|3334.24|74.67|0.00|933.40|1008.07|-1414.66| +2452511|41155|5425|39648|50150|7052|41554|1|24|1924|96|68.49|116.43|84.99|0.00|8159.04|6575.04|11177.28|163.18|0.00|8159.04|8322.22|1584.00| +2452511|41155|6027|39648|50150|7052|41554|1|157|1924|41|92.11|159.35|86.04|0.00|3527.64|3776.51|6533.35|282.21|0.00|3527.64|3809.85|-248.87| +2452511|41155|13602|39648|50150|7052|41554|1|64|1924|48|23.54|28.71|14.35|0.00|688.80|1129.92|1378.08|41.32|0.00|688.80|730.12|-441.12| +2452511|41155|2619|39648|50150|7052|41554|1|36|1924|94|33.05|40.65|28.04|0.00|2635.76|3106.70|3821.10|184.50|0.00|2635.76|2820.26|-470.94| +2452511|41155|15132|39648|50150|7052|41554|1|82|1924|56|13.34|17.74|2.83|0.00|158.48|747.04|993.44|0.00|0.00|158.48|158.48|-588.56| +||9801||50150|7052|41554|||1924|16||120.18|62.49|||991.20|1922.88|29.99||999.84|1029.83|8.64| +2452511|41155|14935|39648|50150|7052|41554|1|52|1924|33|33.69|38.74|7.74|186.45|255.42|1111.77|1278.42|1.37|186.45|68.97|70.34|-1042.80| +2452157|62645|13325|26947|411755|6963|29771|7|21|1925|45|89.44|112.69|20.28|0.00|912.60|4024.80|5071.05|27.37|0.00|912.60|939.97|-3112.20| +2452157|62645|2759|26947|411755|6963|29771|7|90|1925|40|95.99|163.18|137.07|0.00|5482.80|3839.60|6527.20|219.31|0.00|5482.80|5702.11|1643.20| +2452157|62645|6659|26947|411755|6963|29771|7|63|1925|89|89.65|170.33|131.15|0.00|11672.35|7978.85|15159.37|0.00|0.00|11672.35|11672.35|3693.50| +2452157|62645|9053|26947|411755|6963|29771|7|42|1925|68|29.93|55.66|31.16|0.00|2118.88|2035.24|3784.88|190.69|0.00|2118.88|2309.57|83.64| +2452157|62645|14787|26947|411755|6963|29771|7|112|1925|10|21.55|42.02|30.25|0.00|302.50|215.50|420.20|9.07|0.00|302.50|311.57|87.00| +2452157|62645|5953|26947|411755|6963|29771|7|245|1925|29|73.48|78.62|8.64|0.00|250.56|2130.92|2279.98|7.51|0.00|250.56|258.07|-1880.36| +2452157|62645|9085|26947|411755|6963|29771|7|235|1925|4|32.63|57.75|25.98|0.00|103.92|130.52|231.00|8.31|0.00|103.92|112.23|-26.60| +2452157|62645|4797|26947|411755|6963|29771|7|130|1925|13|77.24|118.94|116.56|0.00|1515.28|1004.12|1546.22|90.91|0.00|1515.28|1606.19|511.16| +2452157|62645|1249|26947|411755|6963|29771|7|240|1925|51|53.44|54.50|53.41|0.00|2723.91|2725.44|2779.50|0.00|0.00|2723.91|2723.91|-1.53| +2452157|62645|5561|26947|||29771|||1925||||37.94||113.82||206.97|1.13||||| +2451346|63742|15178|19263|1325559|3855|16551|4|218|1926|36|5.46|7.48|5.68|0.00|204.48|196.56|269.28|8.17|0.00|204.48|212.65|7.92| +2451346|63742|4795|19263|1325559|3855|16551|4|134|1926|38|1.42|2.04|1.53|0.00|58.14|53.96|77.52|2.90|0.00|58.14|61.04|4.18| +2451346|63742|17254|19263|1325559|3855|16551|4|128|1926|90|28.91|36.42|11.29|0.00|1016.10|2601.90|3277.80|71.12|0.00|1016.10|1087.22|-1585.80| +||12571|19263|1325559||16551|4||1926|73||||0.00|||1399.41||0.00|||| +2451346|63742|10177|19263|1325559|3855|16551|4|280|1926|43|94.36|131.16|125.91|0.00|5414.13|4057.48|5639.88|378.98|0.00|5414.13|5793.11|1356.65| +2451346|63742|8596|19263|1325559|3855|16551|4|246|1926|54|94.92|137.63|92.21|0.00|4979.34|5125.68|7432.02|398.34|0.00|4979.34|5377.68|-146.34| +2451346|63742|3620|19263|1325559|3855|16551|4|277|1926|48|96.79|120.01|37.20|0.00|1785.60|4645.92|5760.48|89.28|0.00|1785.60|1874.88|-2860.32| +2451346|63742|12487|19263|1325559|3855|16551|4|113|1926|41|98.32|151.41|136.26|0.00|5586.66|4031.12|6207.81|391.06|0.00|5586.66|5977.72|1555.54| +2451346|63742|712|19263|1325559|3855|16551|4|244|1926|62|60.28|69.32|55.45|0.00|3437.90|3737.36|4297.84|171.89|0.00|3437.90|3609.79|-299.46| +2451346|63742|14617|19263|1325559|3855|16551|4|298|1926|42|21.85|24.69|20.49|0.00|860.58|917.70|1036.98|43.02|0.00|860.58|903.60|-57.12| +2451346|63742|5012|19263|1325559|3855|16551|4|277|1926|13|77.91|126.99|82.54|0.00|1073.02|1012.83|1650.87|85.84|0.00|1073.02|1158.86|60.19| +2451285|62634|4202|9498|76628|3758|44888|7|148|1927|17|10.83|17.00|9.69|0.00|164.73|184.11|289.00|13.17|0.00|164.73|177.90|-19.38| +2451285|62634|3496|9498|76628|3758|44888|7|57|1927|48|35.07|36.47|28.81|0.00|1382.88|1683.36|1750.56|96.80|0.00|1382.88|1479.68|-300.48| +2451285|62634|4183|9498|76628|3758|44888|7|264|1927|9|67.20|107.52|54.83|0.00|493.47|604.80|967.68|14.80|0.00|493.47|508.27|-111.33| +2451285|62634|13426|9498|76628|3758|44888|7|52|1927|78|71.26|111.87|79.42|0.00|6194.76|5558.28|8725.86|123.89|0.00|6194.76|6318.65|636.48| +2451285|62634|664|9498|76628|3758|44888|7|115|1927|79|98.98|195.98|131.30|8298.16|10372.70|7819.42|15482.42|82.98|8298.16|2074.54|2157.52|-5744.88| +2451285|62634|11893|9498|76628|3758|44888|7|221|1927|37|68.54|69.22|31.84|0.00|1178.08|2535.98|2561.14|94.24|0.00|1178.08|1272.32|-1357.90| +2451285||11365|||3758|44888|7|230|1927|38||80.98||||2262.90|||||1453.95|| +2451285|62634|7114|9498|76628|3758|44888|7|146|1927|86|88.41|155.60|76.24|0.00|6556.64|7603.26|13381.60|327.83|0.00|6556.64|6884.47|-1046.62| +2451285|62634|12040|9498|76628|3758|44888|7|241|1927|48|41.61|64.07|40.36|0.00|1937.28|1997.28|3075.36|77.49|0.00|1937.28|2014.77|-60.00| +2451285|62634|5929|9498|76628|3758|44888|7|254|1927|19|23.54|43.31|5.19|86.77|98.61|447.26|822.89|0.94|86.77|11.84|12.78|-435.42| +2452288|37208|5215|59562|37969|6776|10383|7|115|1928|68|16.94|21.85|10.92|0.00|742.56|1151.92|1485.80|14.85|0.00|742.56|757.41|-409.36| +2452288|37208|15012|59562|37969|6776|10383|7|202|1928|19|53.06|62.61|27.54|0.00|523.26|1008.14|1189.59|31.39|0.00|523.26|554.65|-484.88| +2452288|37208|9205|59562|37969|6776|10383|7|96|1928|37|55.08|84.82|49.19|0.00|1820.03|2037.96|3138.34|145.60|0.00|1820.03|1965.63|-217.93| +2452288|37208|8280|59562|37969|6776|10383|7|285|1928|11|94.90|130.96|14.40|0.00|158.40|1043.90|1440.56|0.00|0.00|158.40|158.40|-885.50| +2452288|37208|930|59562|37969|6776|10383|7|290|1928|13|56.79|69.28|1.38|0.00|17.94|738.27|900.64|1.43|0.00|17.94|19.37|-720.33| +2452288|37208|16050|59562|37969|6776|10383|7|91|1928|69|45.58|67.45|43.84|0.00|3024.96|3145.02|4654.05|120.99|0.00|3024.96|3145.95|-120.06| +2452288|37208|14755|59562|37969|6776|10383|7|35|1928|14|38.95|69.72|52.29|0.00|732.06|545.30|976.08|14.64|0.00|732.06|746.70|186.76| +2452288|37208|10477|59562|37969|6776|10383|7|204|1928|89|23.17|31.97|15.34|901.07|1365.26|2062.13|2845.33|27.85|901.07|464.19|492.04|-1597.94| +2452288|37208|15889|59562|37969|6776|10383|7|151|1928|61|46.15|50.30|45.27|0.00|2761.47|2815.15|3068.30|0.00|0.00|2761.47|2761.47|-53.68| +2452288|37208|14016|59562|37969|6776|10383|7|108|1928|2|29.99|40.48|2.42|0.00|4.84|59.98|80.96|0.14|0.00|4.84|4.98|-55.14| +2452288|37208|8655|59562|37969|6776|10383|7|14|1928|55|79.14|94.17|89.46|2017.32|4920.30|4352.70|5179.35|203.20|2017.32|2902.98|3106.18|-1449.72| +2452288|37208|805|59562|37969|6776|10383|7|61|1928|6|23.58|31.59|15.16|0.00|90.96|141.48|189.54|6.36|0.00|90.96|97.32|-50.52| +2452288|37208|711|59562|37969|6776|10383|7|288|1928|83|32.93|60.92|31.06|0.00|2577.98|2733.19|5056.36|51.55|0.00|2577.98|2629.53|-155.21| +2451069|36939|17980|90506|181453|4400|11300|8|65|1929|65|51.15|94.62|69.07|0.00|4489.55|3324.75|6150.30|0.00|0.00|4489.55|4489.55|1164.80| +||178|90506||4400|11300|8||1929||||||490.70|603.80|766.80|||490.70|500.51|| +2451069|36939|6596|90506|181453|4400|11300|8|271|1929|31|84.87|153.61|41.47|539.93|1285.57|2630.97|4761.91|29.82|539.93|745.64|775.46|-1885.33| +2451069|36939|17989|90506|181453|4400|11300|8|162|1929|64|88.10|106.60|79.95|0.00|5116.80|5638.40|6822.40|153.50|0.00|5116.80|5270.30|-521.60| +2451069|36939|13568|90506|181453|4400|11300|8|146|1929|82|58.13|79.05|52.17|0.00|4277.94|4766.66|6482.10|299.45|0.00|4277.94|4577.39|-488.72| +2451069|36939|6829|90506|181453|4400|11300|8|190|1929|88|35.24|43.34|37.27|0.00|3279.76|3101.12|3813.92|295.17|0.00|3279.76|3574.93|178.64| +2451069|36939|2182|90506|181453|4400|11300|8|163|1929|1|90.33|126.46|3.79|0.00|3.79|90.33|126.46|0.22|0.00|3.79|4.01|-86.54| +2451069|36939|1087|90506|181453|4400|11300|8|147|1929|22|12.86|18.51|17.21|0.00|378.62|282.92|407.22|7.57|0.00|378.62|386.19|95.70| +2451069||11186|90506|181453|||8|37|1929|||71.39|||1593.36|4213.44||103.24||1147.22||| +2451069|36939|14281|90506|181453|4400|11300|8|139|1929|64|79.60|99.50|24.87|0.00|1591.68|5094.40|6368.00|63.66|0.00|1591.68|1655.34|-3502.72| +2451069|36939|7081|90506|181453|4400|11300|8|121|1929|8|92.24|173.41|43.35|0.00|346.80|737.92|1387.28|13.87|0.00|346.80|360.67|-391.12| +2451069|36939|6602|90506|181453|4400|11300|8|31|1929|1|39.91|63.85|51.08|0.00|51.08|39.91|63.85|2.55|0.00|51.08|53.63|11.17| +2451196|34017|13172|85846|1424777|861|45345|10|213|1930|98|94.11|177.86|53.35|1150.22|5228.30|9222.78|17430.28|40.78|1150.22|4078.08|4118.86|-5144.70| +2451196|34017|15301|85846|1424777|861|45345|10|83|1930|66|16.09|17.37|6.60|43.56|435.60|1061.94|1146.42|0.00|43.56|392.04|392.04|-669.90| +2451196|34017|5996|85846|1424777|861|45345|10|268|1930|74|3.38|5.67|3.74|0.00|276.76|250.12|419.58|13.83|0.00|276.76|290.59|26.64| +2451196|34017|4742|85846|1424777|861|45345|10|230|1930|28|32.85|49.93|43.93|0.00|1230.04|919.80|1398.04|110.70|0.00|1230.04|1340.74|310.24| +2451196|34017|15142|85846|1424777|861|45345|10|119|1930|100|80.49|108.66|64.10|0.00|6410.00|8049.00|10866.00|256.40|0.00|6410.00|6666.40|-1639.00| +2451196|34017|8626|85846|1424777|861|45345|10|121|1930|69|62.43|91.14|88.40|0.00|6099.60|4307.67|6288.66|487.96|0.00|6099.60|6587.56|1791.93| +2451196|34017|5119|85846|1424777|861|45345|10|64|1930|43|35.56|70.05|8.40|0.00|361.20|1529.08|3012.15|25.28|0.00|361.20|386.48|-1167.88| +2451196|34017|13706|85846|1424777|861|45345|10|108|1930|68|61.48|108.20|40.03|0.00|2722.04|4180.64|7357.60|217.76|0.00|2722.04|2939.80|-1458.60| +2451196|34017|12692|85846|1424777|861|45345|10|23|1930|53|25.36|29.92|25.13|0.00|1331.89|1344.08|1585.76|0.00|0.00|1331.89|1331.89|-12.19| +2452131||4093||1197853|2811|14969|2|48|1931|12||5.97||||49.80|71.64||||17.08|| +2452131|53043|12163|63648|1197853|2811|14969|2|262|1931|56|55.41|88.65|7.97|0.00|446.32|3102.96|4964.40|13.38|0.00|446.32|459.70|-2656.64| +2452131|53043|15399|63648|1197853|2811|14969|2|35|1931|6|54.10|74.65|41.80|0.00|250.80|324.60|447.90|0.00|0.00|250.80|250.80|-73.80| +2452131|53043|13559|63648|1197853|2811|14969|2|205|1931|37|38.80|53.93|37.75|0.00|1396.75|1435.60|1995.41|125.70|0.00|1396.75|1522.45|-38.85| +2452131|53043|10149|63648|1197853|2811|14969|2|98|1931|62|94.78|173.44|15.60|0.00|967.20|5876.36|10753.28|9.67|0.00|967.20|976.87|-4909.16| +2452131|53043|16663|63648|1197853|2811|14969|2|173|1931|78|46.87|77.33|38.66|0.00|3015.48|3655.86|6031.74|211.08|0.00|3015.48|3226.56|-640.38| +2452131|53043|14897|63648|1197853|2811|14969|2|286|1931|60|77.01|137.07|121.99|0.00|7319.40|4620.60|8224.20|73.19|0.00|7319.40|7392.59|2698.80| +2452131|53043|2159|63648|1197853|2811|14969|2|134|1931|54|85.23|133.81|105.70|0.00|5707.80|4602.42|7225.74|456.62|0.00|5707.80|6164.42|1105.38| +2452131|53043|10015|63648|1197853|2811|14969|2|185|1931|34|20.97|23.06|7.60|191.21|258.40|712.98|784.04|4.03|191.21|67.19|71.22|-645.79| +2452131|53043|11691|63648|1197853|2811|14969|2|61|1931|67|84.35|156.89|3.13|119.53|209.71|5651.45|10511.63|2.70|119.53|90.18|92.88|-5561.27| +2452131|53043|14853|63648|1197853|2811|14969|2|251|1931|41|48.76|58.99|18.28|0.00|749.48|1999.16|2418.59|14.98|0.00|749.48|764.46|-1249.68| +2452131|53043|1263|63648|1197853|2811|14969|2|258|1931|85|82.82|100.21|6.01|0.00|510.85|7039.70|8517.85|5.10|0.00|510.85|515.95|-6528.85| +2452131|53043|15851|63648|1197853|2811|14969|2|44|1931|21|31.49|33.37|6.67|64.43|140.07|661.29|700.77|6.80|64.43|75.64|82.44|-585.65| +2452131|53043|7639|63648|1197853|2811|14969|2|22|1931|12|60.87|73.04|16.79|0.00|201.48|730.44|876.48|12.08|0.00|201.48|213.56|-528.96| +2452131|53043|6061|63648|1197853|2811|14969|2|250|1931|75|88.06|126.80|84.95|127.42|6371.25|6604.50|9510.00|249.75|127.42|6243.83|6493.58|-360.67| +2452131|53043|6791|63648|1197853|2811|14969|2|118|1931|41|32.70|36.29|34.47|0.00|1413.27|1340.70|1487.89|14.13|0.00|1413.27|1427.40|72.57| +|63156|9080|||976|14479||93|1932|41|8.25|12.04|3.73||152.93||||||88.21|| +2451168|63156|950|92723|1513826|976|14479|7|207|1932|48|22.46|22.68|4.76|0.00|228.48|1078.08|1088.64|11.42|0.00|228.48|239.90|-849.60| +2451168|63156|17890|92723|1513826|976|14479|7|244|1932|60|44.00|75.24|53.42|32.05|3205.20|2640.00|4514.40|190.38|32.05|3173.15|3363.53|533.15| +2451168|63156|1912|92723|1513826|976|14479|7|184|1932|87|53.77|94.09|4.70|0.00|408.90|4677.99|8185.83|24.53|0.00|408.90|433.43|-4269.09| +2451168|63156|9088|92723|1513826|976|14479|7|286|1932|56|58.68|90.95|28.19|0.00|1578.64|3286.08|5093.20|31.57|0.00|1578.64|1610.21|-1707.44| +2451168|63156|4966|92723|1513826|976|14479|7|87|1932|79|32.66|32.98|10.22|0.00|807.38|2580.14|2605.42|0.00|0.00|807.38|807.38|-1772.76| +2451168|63156|4876|92723|1513826|976|14479|7|10|1932|71|79.50|144.69|33.27|0.00|2362.17|5644.50|10272.99|212.59|0.00|2362.17|2574.76|-3282.33| +2451168|63156|2071|92723|1513826|976|14479|7|78|1932|16|27.57|30.87|7.10|102.24|113.60|441.12|493.92|0.45|102.24|11.36|11.81|-429.76| +2451168|63156|7154|92723|1513826|976|14479|7|183|1932|88|31.05|45.02|9.00|0.00|792.00|2732.40|3961.76|39.60|0.00|792.00|831.60|-1940.40| +||12986||1513826|976||7||1932|55||30.92|9.27|0.00|||1700.60||0.00||545.53|-594.55| +2451168|63156|5966|92723|1513826|976|14479|7|144|1932|88|58.51|83.66|77.80|0.00|6846.40|5148.88|7362.08|342.32|0.00|6846.40|7188.72|1697.52| +2451168|63156|8575|92723|1513826|976|14479|7|62|1932|40|21.83|34.05|17.36|229.15|694.40|873.20|1362.00|32.56|229.15|465.25|497.81|-407.95| +2451518|67576|10244|43324|661649|4327|25052|2|249|1933|74|61.21|122.42|30.60|0.00|2264.40|4529.54|9059.08|22.64|0.00|2264.40|2287.04|-2265.14| +2451518|67576|11497|43324|661649|4327|25052|2|81|1933|60|38.20|38.20|29.41|0.00|1764.60|2292.00|2292.00|0.00|0.00|1764.60|1764.60|-527.40| +2451518|67576|10192|43324|661649|4327|25052|2|214|1933|91|17.17|29.01|15.66|0.00|1425.06|1562.47|2639.91|128.25|0.00|1425.06|1553.31|-137.41| +2451518|67576|10399|43324|661649|4327|25052|2|170|1933|64|5.59|6.93|2.77|0.00|177.28|357.76|443.52|5.31|0.00|177.28|182.59|-180.48| +2451518|67576|10135|43324|661649|4327|25052|2|47|1933|24|89.09|150.56|76.78|0.00|1842.72|2138.16|3613.44|147.41|0.00|1842.72|1990.13|-295.44| +2451518|67576|3745|43324|661649|4327|25052|2|236|1933|2|88.88|115.54|110.91|133.09|221.82|177.76|231.08|3.54|133.09|88.73|92.27|-89.03| +2451518|67576|7471|43324|661649|4327|25052|2|4|1933|30|66.40|81.00|46.98|0.00|1409.40|1992.00|2430.00|42.28|0.00|1409.40|1451.68|-582.60| +2451518|67576|14128|43324|661649|4327|25052|2|33|1933|52|49.01|67.63|2.70|99.68|140.40|2548.52|3516.76|2.85|99.68|40.72|43.57|-2507.80| +2451518|67576|15043|43324|661649|4327|25052|2|88|1933|72|43.61|85.03|0.85|0.00|61.20|3139.92|6122.16|0.61|0.00|61.20|61.81|-3078.72| +2451518|67576|16891|43324|661649|4327|25052|2|11|1933|29|32.34|54.33|33.68|0.00|976.72|937.86|1575.57|78.13|0.00|976.72|1054.85|38.86| +2451518|67576|3674|43324|661649|4327|25052|2|144|1933|11|83.24|91.56|21.97|0.00|241.67|915.64|1007.16|9.66|0.00|241.67|251.33|-673.97| +2450894|63011|12709|95243|1244746|5802|41658|2|167|1934|11|68.76|94.88|49.33|0.00|542.63|756.36|1043.68|27.13|0.00|542.63|569.76|-213.73| +2450894|63011|16240|95243|1244746|5802|41658|2|243|1934|90|98.95|124.67|18.70|0.00|1683.00|8905.50|11220.30|117.81|0.00|1683.00|1800.81|-7222.50| +2450894|63011|17917|95243|1244746|5802|41658|2|70|1934|62|69.01|95.92|85.36|0.00|5292.32|4278.62|5947.04|370.46|0.00|5292.32|5662.78|1013.70| +2450894|63011|6295|95243|1244746|5802|41658|2|184|1934|76|21.05|33.25|24.93|0.00|1894.68|1599.80|2527.00|170.52|0.00|1894.68|2065.20|294.88| +2450894|63011|6859|95243|1244746|5802|41658|2|69|1934|35|88.48|145.99|116.79|0.00|4087.65|3096.80|5109.65|122.62|0.00|4087.65|4210.27|990.85| +2450894|63011|656|95243|1244746|5802|41658|2|90|1934|61|52.10|84.92|37.36|0.00|2278.96|3178.10|5180.12|113.94|0.00|2278.96|2392.90|-899.14| +2450894|63011|11314|95243|1244746|5802|41658|2|194|1934|3|47.00|51.70|22.74|0.00|68.22|141.00|155.10|2.04|0.00|68.22|70.26|-72.78| +2450894|63011|17354|95243|1244746|5802|41658|2|99|1934|7|73.85|131.45|6.57|0.00|45.99|516.95|920.15|2.75|0.00|45.99|48.74|-470.96| +2450894||14114||1244746||41658||169|1934||4.33|8.14||60.06|222.48||586.08|8.12|60.06||170.54|-149.34| +2451803|66253|11186|61949|527637|2925|25813|2|64|1935|28|93.58|166.57|38.31|0.00|1072.68|2620.24|4663.96|10.72|0.00|1072.68|1083.40|-1547.56| +2451803|66253|14281|61949|527637|2925|25813|2|113|1935|43|45.54|81.51|80.69|277.57|3469.67|1958.22|3504.93|159.60|277.57|3192.10|3351.70|1233.88| +2451803|66253|7081|61949|527637||25813|2|16|1935|38|89.54|||0.00|3603.16||||0.00||3819.34|200.64| +2451803|66253|6602|61949|527637|2925|25813|2|122|1935|24|77.97|149.70|23.95|0.00|574.80|1871.28|3592.80|0.00|0.00|574.80|574.80|-1296.48| +2451803|66253|11054|61949|527637|2925|25813|2|104|1935|26|52.07|102.57|89.23|0.00|2319.98|1353.82|2666.82|46.39|0.00|2319.98|2366.37|966.16| +2451803|66253|3872|61949|527637|2925|25813|2|188|1935|28|6.90|7.65|3.82|0.00|106.96|193.20|214.20|6.41|0.00|106.96|113.37|-86.24| +||15661|61949|||25813|||1935|94|26.47||8.86|0.00||2488.18|3085.08|49.97|0.00||882.81|-1655.34| +2451803|66253|3797|61949|527637|2925|25813|2|26|1935|40|14.68|20.69|2.06|0.00|82.40|587.20|827.60|1.64|0.00|82.40|84.04|-504.80| +2451803|66253|12152|61949|527637|2925|25813|2|266|1935|78|91.56|121.77|41.40|0.00|3229.20|7141.68|9498.06|64.58|0.00|3229.20|3293.78|-3912.48| +2451803|66253|4142|61949|527637|2925|25813|2|173|1935|53|39.98|72.76|20.37|0.00|1079.61|2118.94|3856.28|32.38|0.00|1079.61|1111.99|-1039.33| +2451452|66347|3001|91303|1780949|195|5041|2|95|1936|82|26.10|46.71|0.00|0.00|0.00|2140.20|3830.22|0.00|0.00|0.00|0.00|-2140.20| +2451452|66347|9703|91303|1780949|195|5041|2|211|1936|64|1.42|2.32|0.99|0.00|63.36|90.88|148.48|0.00|0.00|63.36|63.36|-27.52| +|66347|14134||1780949|195|5041|2|260|1936|50|85.05||64.29|0.00|3214.50|4252.50|7144.00||0.00||3310.93|-1038.00| +2451452|66347|17677|91303|1780949|195|5041|2|284|1936|34|38.07|45.68|6.85|0.00|232.90|1294.38|1553.12|20.96|0.00|232.90|253.86|-1061.48| +2451452||5806|91303|||||132|1936|30||82.14|66.53|0.00|1995.90|1665.00|||0.00|1995.90||| +2451452|66347|10232|91303|1780949|195|5041|2|32|1936|55|72.87|139.91|114.72|5426.25|6309.60|4007.85|7695.05|35.33|5426.25|883.35|918.68|-3124.50| +2451452|66347|15145|91303|1780949|195|5041|2|238|1936|11|12.66|22.15|13.95|19.94|153.45|139.26|243.65|5.34|19.94|133.51|138.85|-5.75| +2451452|66347|12793|91303|1780949|195|5041|2|192|1936|16|15.16|17.73|15.42|0.00|246.72|242.56|283.68|4.93|0.00|246.72|251.65|4.16| +2451452|66347|715|91303|1780949|195|5041|2|241|1936|24|4.94|6.57|3.94|0.00|94.56|118.56|157.68|1.89|0.00|94.56|96.45|-24.00| +2451452|66347|16090|91303|1780949|195|5041|2|282|1936|84|56.67|66.30|19.22|0.00|1614.48|4760.28|5569.20|0.00|0.00|1614.48|1614.48|-3145.80| +2451452|66347|1600|91303|1780949|195|5041|2|125|1936|52|54.71|57.44|36.18|0.00|1881.36|2844.92|2986.88|131.69|0.00|1881.36|2013.05|-963.56| +2451452|66347|15424|91303|1780949|195|5041|2|75|1936|26|2.83|2.97|0.89|0.00|23.14|73.58|77.22|0.69|0.00|23.14|23.83|-50.44| +2451452|66347|15560|91303|1780949|195|5041|2|175|1936|16|86.63|87.49|61.24|0.00|979.84|1386.08|1399.84|19.59|0.00|979.84|999.43|-406.24| +2451452|66347|11821|91303|1780949|195|5041|2|54|1936|79|78.83|152.14|132.36|0.00|10456.44|6227.57|12019.06|209.12|0.00|10456.44|10665.56|4228.87| +2451452|66347|12313|91303|||5041||202|1936|26|74.64||||2561.26||3823.04|||870.83||| +2451452|66347|3586|91303|1780949|195|5041|2|228|1936|61|15.10|25.82|19.62|0.00|1196.82|921.10|1575.02|71.80|0.00|1196.82|1268.62|275.72| +2451721|53637|11267|39232|688333|2587|18088|4|64|1937|90|81.27|117.84|16.49|0.00|1484.10|7314.30|10605.60|118.72|0.00|1484.10|1602.82|-5830.20| +2451721|53637|12260|39232|688333|2587|18088|4|145|1937|67|24.01|46.33|2.31|114.52|154.77|1608.67|3104.11|2.41|114.52|40.25|42.66|-1568.42| +2451721|53637|1667|39232|688333|2587|18088|4|199|1937|32|80.14|87.35|38.43|0.00|1229.76|2564.48|2795.20|98.38|0.00|1229.76|1328.14|-1334.72| +2451721|53637|13442|39232|688333|2587|18088|4|177|1937|3|10.11|12.43|9.94|21.76|29.82|30.33|37.29|0.64|21.76|8.06|8.70|-22.27| +2451721|53637|821|39232|688333|2587|18088|4|40|1937|83|13.82|26.81|10.99|0.00|912.17|1147.06|2225.23|0.00|0.00|912.17|912.17|-234.89| +2451721|53637|6583|39232|688333|2587|18088|4|58|1937|63|48.80|75.15|15.78|924.55|994.14|3074.40|4734.45|0.00|924.55|69.59|69.59|-3004.81| +2451721|53637|3176|39232|688333|2587|18088|4|115|1937|37|33.49|51.90|26.46|0.00|979.02|1239.13|1920.30|29.37|0.00|979.02|1008.39|-260.11| +||2371|39232|688333|2587|||155|1937||31.51||||321.64|346.61||||321.64||| +2451721|53637|10423|39232|688333|2587|18088|4|241|1937|29|29.31|47.18|42.46|0.00|1231.34|849.99|1368.22|24.62|0.00|1231.34|1255.96|381.35| +2451721|53637|2333|39232|688333|2587|18088|4|271|1937|92|86.40|128.73|3.86|0.00|355.12|7948.80|11843.16|28.40|0.00|355.12|383.52|-7593.68| +2451721|53637|7429|39232|||18088|||1937||||2.80|7.05|50.40|464.76|||7.05|43.35|46.38|| +2451721|53637|8183|39232|688333|2587|18088|4|89|1937|87|44.98|89.06|87.27|0.00|7592.49|3913.26|7748.22|303.69|0.00|7592.49|7896.18|3679.23| +2451721|53637|2453|39232|688333|2587|18088|4|48|1937|24|19.68|20.66|8.26|0.00|198.24|472.32|495.84|13.87|0.00|198.24|212.11|-274.08| +2451721|53637|13645|39232|688333|2587|18088|4|124|1937|94|80.83|150.34|34.57|0.00|3249.58|7598.02|14131.96|32.49|0.00|3249.58|3282.07|-4348.44| +2451890||140|21784|961908|||||1938|||||9.53|||51.68|0.23|9.53|||| +2451890|42592|16255|21784|961908|2032|23756|4|51|1938|73|99.86|146.79|63.11|0.00|4607.03|7289.78|10715.67|368.56|0.00|4607.03|4975.59|-2682.75| +2451890|42592|3191|21784|961908|2032|23756|4|240|1938|5|50.52|98.00|89.18|0.00|445.90|252.60|490.00|35.67|0.00|445.90|481.57|193.30| +2451890|42592|10261|21784|961908|2032|23756|4|85|1938|35|25.97|49.60|44.64|968.68|1562.40|908.95|1736.00|5.93|968.68|593.72|599.65|-315.23| +2451890|42592|11180|21784|961908|2032|23756|4|123|1938|47|82.06|94.36|13.21|248.34|620.87|3856.82|4434.92|0.00|248.34|372.53|372.53|-3484.29| +2451890|42592|10429|21784|961908|2032|23756|4|166|1938|90|52.35|78.52|29.83|751.71|2684.70|4711.50|7066.80|0.00|751.71|1932.99|1932.99|-2778.51| +2451890|42592|13178|21784|961908|2032|23756|4|14|1938|41|78.35|103.42|27.92|0.00|1144.72|3212.35|4240.22|91.57|0.00|1144.72|1236.29|-2067.63| +2451890|42592|10793|21784|961908|2032|23756|4|8|1938|33|18.66|30.41|3.34|0.00|110.22|615.78|1003.53|8.81|0.00|110.22|119.03|-505.56| +2451890|42592|1987|21784|961908|2032|23756|4|242|1938|45|84.55|125.13|60.06|810.81|2702.70|3804.75|5630.85|56.75|810.81|1891.89|1948.64|-1912.86| +2451890|42592|1049|21784|961908|2032|23756|4|234|1938|49|17.74|25.72|23.40|0.00|1146.60|869.26|1260.28|0.00|0.00|1146.60|1146.60|277.34| +2451890|42592|7178|21784|961908|2032|23756|4|10|1938|44|72.65|89.35|2.68|0.00|117.92|3196.60|3931.40|9.43|0.00|117.92|127.35|-3078.68| +|52611|5677|||2821||1||1939||36.08|37.16|33.44|0.00|1237.28||1374.92|12.37|0.00||1249.65|-97.68| +2451529|52611|16706|64092|1213280|2821|4762|1|249|1939|38|77.74|147.70|29.54|976.59|1122.52|2954.12|5612.60|1.45|976.59|145.93|147.38|-2808.19| +2451529|52611|16466|64092|1213280|2821|4762|1|41|1939|84|9.80|15.87|5.23|0.00|439.32|823.20|1333.08|35.14|0.00|439.32|474.46|-383.88| +2451529|52611|17638|64092|1213280|2821|4762|1|274|1939|93|34.35|47.40|38.86|0.00|3613.98|3194.55|4408.20|289.11|0.00|3613.98|3903.09|419.43| +2451529|52611|11144|64092|1213280|2821|4762|1|214|1939|65|89.72|92.41|47.12|183.76|3062.80|5831.80|6006.65|57.58|183.76|2879.04|2936.62|-2952.76| +2451529|52611|2152|64092|1213280|2821|4762|1|85|1939|17|35.40|44.25|32.30|0.00|549.10|601.80|752.25|38.43|0.00|549.10|587.53|-52.70| +2451529||1777|64092||||1|203|1939||29.97|37.46||0.00||||34.91|0.00||533.67|| +2451529|52611|10910|64092|1213280|2821|4762|1|39|1939|40|81.11|119.23|32.19|0.00|1287.60|3244.40|4769.20|12.87|0.00|1287.60|1300.47|-1956.80| +2451529|52611|11029|64092|1213280|2821||1|237|1939|50|||13.16|0.00|658.00|943.00|1829.00|26.32|0.00|658.00||| +2452589|37120|8031|39186|659128|3517|20306|4|164|1940|83|76.38|140.53|64.64|0.00|5365.12|6339.54|11663.99|214.60|0.00|5365.12|5579.72|-974.42| +2452589|37120|7867|39186|659128|3517|20306|4|109|1940|24|47.22|47.69|20.98|458.20|503.52|1133.28|1144.56|1.81|458.20|45.32|47.13|-1087.96| +2452589||469|39186|659128||20306|4|168|1940|76||||0.00|1654.52|3940.60|4728.72|82.72|0.00|||| +2452589|37120|8359|39186|659128|3517|20306|4|92|1940|32|28.35|46.77|45.83|0.00|1466.56|907.20|1496.64|73.32|0.00|1466.56|1539.88|559.36| +2452589|37120|849|39186|659128|3517|20306|4|299|1940|25|49.69|51.18|19.96|0.00|499.00|1242.25|1279.50|19.96|0.00|499.00|518.96|-743.25| +2452589|37120|16555|39186|659128|3517|20306|4|83|1940|38|35.41|67.98|25.15|0.00|955.70|1345.58|2583.24|66.89|0.00|955.70|1022.59|-389.88| +2452589|37120|9693|39186|659128|3517|20306|4|122|1940|36|82.40|146.67|112.93|0.00|4065.48|2966.40|5280.12|203.27|0.00|4065.48|4268.75|1099.08| +2452589|37120|10315|39186|659128|3517|20306|4|116|1940|51|25.49|33.90|7.45|0.00|379.95|1299.99|1728.90|34.19|0.00|379.95|414.14|-920.04| +2452589|37120|15666|39186|659128|3517|20306|4|293|1940|13|44.26|85.86|0.85|2.54|11.05|575.38|1116.18|0.08|2.54|8.51|8.59|-566.87| +2452589|37120|5736|39186|659128|3517|20306|4|183|1940|1|59.52|70.82|23.37|0.00|23.37|59.52|70.82|0.70|0.00|23.37|24.07|-36.15| +2452589|37120|17485|39186|659128|3517|20306|4|73|1940|95|75.25|104.59|61.70|0.00|5861.50|7148.75|9936.05|234.46|0.00|5861.50|6095.96|-1287.25| +2450911|49994|11287|53111|1615236|6514|40592|4|51|1941|45|54.98|89.61|8.06|0.00|362.70|2474.10|4032.45|7.25|0.00|362.70|369.95|-2111.40| +2450911|49994|7568|53111|1615236|6514|40592|4|189|1941|14|70.90|116.27|11.62|0.00|162.68|992.60|1627.78|14.64|0.00|162.68|177.32|-829.92| +2450911|49994|17245|53111|1615236|6514|40592|4|25|1941|10|37.17|63.93|22.37|0.00|223.70|371.70|639.30|6.71|0.00|223.70|230.41|-148.00| +2450911|49994|6304|53111|1615236|6514|40592|4|193|1941|65|65.41|68.68|39.14|0.00|2544.10|4251.65|4464.20|203.52|0.00|2544.10|2747.62|-1707.55| +2450911|49994|14380|53111|1615236|6514|40592|4|73|1941|5|23.37|43.23|25.50|0.00|127.50|116.85|216.15|11.47|0.00|127.50|138.97|10.65| +2450911|49994|3727|53111|1615236|6514|40592|4|155|1941|73|55.63|85.11|42.55|0.00|3106.15|4060.99|6213.03|31.06|0.00|3106.15|3137.21|-954.84| +2450911|49994|6946|53111|1615236|6514|40592|4|184|1941|34|63.50|66.04|60.75|0.00|2065.50|2159.00|2245.36|185.89|0.00|2065.50|2251.39|-93.50| +2450911|49994|15098|53111|1615236|6514|40592|4|77|1941|16|62.95|86.87|82.52|0.00|1320.32|1007.20|1389.92|39.60|0.00|1320.32|1359.92|313.12| +2450911|49994|14132|53111|1615236|6514|40592|4|297|1941|34|11.17|16.53|16.19|0.00|550.46|379.78|562.02|16.51|0.00|550.46|566.97|170.68| +2450911|49994|8948|53111|1615236|6514|40592|4|6|1941|95|29.53|34.55|2.07|0.00|196.65|2805.35|3282.25|9.83|0.00|196.65|206.48|-2608.70| +2450911|49994|2317|53111|1615236|6514|40592|4|203|1941|60|41.95|52.01|44.72|2495.37|2683.20|2517.00|3120.60|9.39|2495.37|187.83|197.22|-2329.17| +2450911|49994|1999|53111|1615236|6514|40592|4|187|1941|8|61.91|117.62|50.57|0.00|404.56|495.28|940.96|28.31|0.00|404.56|432.87|-90.72| +2450911|49994|17516|53111|1615236|6514|40592|4|134|1941|88|84.93|119.75|4.79|0.00|421.52|7473.84|10538.00|4.21|0.00|421.52|425.73|-7052.32| +2450911|49994|11942|53111|1615236|6514|40592|4|255|1941|6|63.13|73.23|43.20|233.28|259.20|378.78|439.38|1.03|233.28|25.92|26.95|-352.86| +2452590|59074|2301|6566|229307|848|8789|10|249|1942|100|45.78|46.23|24.96|0.00|2496.00|4578.00|4623.00|99.84|0.00|2496.00|2595.84|-2082.00| +2452590|59074|16188|6566|229307|848|8789|10|249|1942|16|16.30|30.31|11.82|11.34|189.12|260.80|484.96|1.77|11.34|177.78|179.55|-83.02| +2452590|59074|7674|6566|229307|848|8789|10|158|1942|43|27.93|48.59|28.18|0.00|1211.74|1200.99|2089.37|0.00|0.00|1211.74|1211.74|10.75| +2452590|59074|13867|6566|229307|848|8789|10|36|1942|73|81.07|88.36|77.75|3064.90|5675.75|5918.11|6450.28|26.10|3064.90|2610.85|2636.95|-3307.26| +2452590|59074|5071|6566|229307|848|8789|10|62|1942|90|58.33|93.32|43.86|0.00|3947.40|5249.70|8398.80|355.26|0.00|3947.40|4302.66|-1302.30| +2452590|59074|4695|6566|229307|848|8789|10|87|1942|55|65.39|98.08|29.42|0.00|1618.10|3596.45|5394.40|32.36|0.00|1618.10|1650.46|-1978.35| +2452590|59074|17635|6566|229307|848|8789|10|89|1942|86|58.63|103.77|95.46|0.00|8209.56|5042.18|8924.22|574.66|0.00|8209.56|8784.22|3167.38| +2452590|59074|4866|6566|229307|848|8789|10|34|1942|27|7.39|12.56|9.79|0.00|264.33|199.53|339.12|5.28|0.00|264.33|269.61|64.80| +2451177|44375|14311|2136|1392049|3562|32228|8|280|1943|94|73.85|129.23|82.70|0.00|7773.80|6941.90|12147.62|77.73|0.00|7773.80|7851.53|831.90| +2451177|44375|8936|2136|1392049|3562|32228|8|252|1943|96|28.46|53.22|31.39|2621.69|3013.44|2732.16|5109.12|0.00|2621.69|391.75|391.75|-2340.41| +2451177|44375|12868|2136|1392049|3562|32228|8|171|1943|26|79.57|97.07|76.68|1275.95|1993.68|2068.82|2523.82|7.17|1275.95|717.73|724.90|-1351.09| +2451177|44375|17852|2136|1392049|3562|32228|8|50|1943|50|24.88|25.37|17.75|0.00|887.50|1244.00|1268.50|44.37|0.00|887.50|931.87|-356.50| +2451177|44375|1975|2136|1392049|3562|32228|8|298|1943|22|61.49|99.61|55.78|0.00|1227.16|1352.78|2191.42|98.17|0.00|1227.16|1325.33|-125.62| +2451177|44375|4000|2136|1392049|3562|32228|8|282|1943|75|10.92|12.44|10.94|180.51|820.50|819.00|933.00|57.59|180.51|639.99|697.58|-179.01| +2451177|44375|4363|2136|1392049|3562|32228|8|143|1943|56|46.86|53.88|42.02|0.00|2353.12|2624.16|3017.28|70.59|0.00|2353.12|2423.71|-271.04| +2451177|44375|3631|2136|1392049|3562|32228|8|287|1943|4|58.67|85.07|66.35|63.69|265.40|234.68|340.28|4.03|63.69|201.71|205.74|-32.97| +2451177|44375|4468|2136|1392049|3562|32228|8|229|1943|99|88.63|175.48|47.37|2860.67|4689.63|8774.37|17372.52|36.57|2860.67|1828.96|1865.53|-6945.41| +2451177|44375|3134|2136|1392049|3562|32228|8|146|1943|46|91.81|175.35|166.58|0.00|7662.68|4223.26|8066.10|0.00|0.00|7662.68|7662.68|3439.42| +2451173|55022|10256|42631|894261|1091|8768|7|115|1944|20|68.30|133.18|126.52|1290.50|2530.40|1366.00|2663.60|49.59|1290.50|1239.90|1289.49|-126.10| +2451173|55022|14888|42631|894261|1091|8768|7|4|1944|23|85.13|130.24|98.98|0.00|2276.54|1957.99|2995.52|91.06|0.00|2276.54|2367.60|318.55| +2451173|55022|7165|42631|894261|1091|8768|7|209|1944|9|85.84|128.76|54.07|369.83|486.63|772.56|1158.84|9.34|369.83|116.80|126.14|-655.76| +||17810|42631|||8768|7|145|1944|93||||0.00|1830.24|5043.39|6101.73|128.11|0.00|||-3213.15| +2451173|55022|13018|42631|894261|1091|8768|7|245|1944|66|39.03|60.49|32.05|1459.55|2115.30|2575.98|3992.34|19.67|1459.55|655.75|675.42|-1920.23| +||15502||894261||||14|1944|55||||||1416.80|2252.25|24.32||1216.05|1240.37|-200.75| +2451173|55022|2042|42631|894261|1091|8768|7|192|1944|4|34.92|41.90|6.28|14.56|25.12|139.68|167.60|0.21|14.56|10.56|10.77|-129.12| +2451173|55022|1372|42631|894261|1091|8768|7|234|1944|62|43.57|83.21|76.55|1233.98|4746.10|2701.34|5159.02|280.96|1233.98|3512.12|3793.08|810.78| +2451173|55022|13987|42631|894261|1091|8768|7|148|1944|100|52.17|70.95|49.66|1241.50|4966.00|5217.00|7095.00|260.71|1241.50|3724.50|3985.21|-1492.50| +2451173|55022|16831|42631|894261|1091|8768|7|207|1944|59|14.23|17.64|12.70|0.00|749.30|839.57|1040.76|59.94|0.00|749.30|809.24|-90.27| +2451173|55022|8636|42631|894261|1091|8768|7|219|1944|70|36.40|56.05|50.44|1588.86|3530.80|2548.00|3923.50|116.51|1588.86|1941.94|2058.45|-606.06| +2451173|55022|1981|42631|894261|1091|8768|7|144|1944|98|22.09|41.08|20.95|0.00|2053.10|2164.82|4025.84|102.65|0.00|2053.10|2155.75|-111.72| +2451173|55022|14569|42631|894261|1091|8768|7|213|1944|26|54.14|76.33|4.57|0.00|118.82|1407.64|1984.58|2.37|0.00|118.82|121.19|-1288.82| +2451161|64511|8935|64799|345417|1235|15853|10|277|1945|88|17.89|21.11|5.06|0.00|445.28|1574.32|1857.68|22.26|0.00|445.28|467.54|-1129.04| +2451161|64511|5684|64799|345417|1235|15853|10|246|1945|48|68.37|77.94|49.10|0.00|2356.80|3281.76|3741.12|23.56|0.00|2356.80|2380.36|-924.96| +2451161|64511|17702|64799|345417|1235|15853|10|204|1945|78|2.17|3.27|0.39|11.55|30.42|169.26|255.06|1.13|11.55|18.87|20.00|-150.39| +2451161|64511|5695|64799|345417|1235|15853|10|96|1945|83|43.96|45.71|14.62|0.00|1213.46|3648.68|3793.93|84.94|0.00|1213.46|1298.40|-2435.22| +2451161|64511|9967|64799|345417|1235|15853|10|199|1945|15|29.38|52.59|8.41|0.00|126.15|440.70|788.85|8.83|0.00|126.15|134.98|-314.55| +2451161|64511|3901|64799|345417|1235|15853|10|147|1945|71|98.28|176.90|107.90|0.00|7660.90|6977.88|12559.90|612.87|0.00|7660.90|8273.77|683.02| +2451161|64511|8188|64799|345417|1235|15853|10|3|1945|28|78.68|84.97|73.07|0.00|2045.96|2203.04|2379.16|0.00|0.00|2045.96|2045.96|-157.08| +2451161|64511|325|64799|345417|1235|15853|10|192|1945|37|2.15|4.02|1.12|0.00|41.44|79.55|148.74|2.07|0.00|41.44|43.51|-38.11| +2451161|64511|13384|64799|345417|1235|15853|10|99|1945|89|85.06|110.57|5.52|0.00|491.28|7570.34|9840.73|19.65|0.00|491.28|510.93|-7079.06| +2451161|64511|11305|64799|345417|1235|15853|10|120|1945|26|47.65|47.65|10.48|19.07|272.48|1238.90|1238.90|15.20|19.07|253.41|268.61|-985.49| +2451161|64511|11954|64799|345417|1235|15853|10|287|1945|16|94.32|155.62|85.59|0.00|1369.44|1509.12|2489.92|109.55|0.00|1369.44|1478.99|-139.68| +2451161|64511|15415|64799|345417|1235|15853|10|83|1945|9|72.51|130.51|101.79|0.00|916.11|652.59|1174.59|27.48|0.00|916.11|943.59|263.52| +2451161|64511|2509|64799|345417|1235|15853|10|93|1945|71|79.73|118.00|62.54|3108.23|4440.34|5660.83|8378.00|26.64|3108.23|1332.11|1358.75|-4328.72| +2451442|50669|8356|30852|1420095|538|22581|4|237|1946|57|2.57|4.44|3.72|199.31|212.04|146.49|253.08|0.38|199.31|12.73|13.11|-133.76| +2451442|50669|13682|30852|1420095|538|22581|4|163|1946|92|48.53|59.69|40.58|1642.67|3733.36|4464.76|5491.48|0.00|1642.67|2090.69|2090.69|-2374.07| +2451442|50669|9806|30852|1420095|538|22581|4|286|1946|62|42.14|53.93|42.06|2138.33|2607.72|2612.68|3343.66|42.24|2138.33|469.39|511.63|-2143.29| +2451442|50669|17588|30852|1420095|538|22581|4|81|1946|39|41.43|67.94|28.53|0.00|1112.67|1615.77|2649.66|55.63|0.00|1112.67|1168.30|-503.10| +2451442|50669|2722|30852|1420095|538|22581|4|184|1946|91|28.91|50.88|25.44|0.00|2315.04|2630.81|4630.08|92.60|0.00|2315.04|2407.64|-315.77| +2451442|50669|1958|30852|1420095|538|22581|4|283|1946|57|21.56|37.08|27.81|1062.06|1585.17|1228.92|2113.56|0.00|1062.06|523.11|523.11|-705.81| +2451442|50669|9151|30852|1420095|538|22581|4|18|1946|47|85.89|86.74|22.55|0.00|1059.85|4036.83|4076.78|31.79|0.00|1059.85|1091.64|-2976.98| +2451442|50669|91|30852|1420095|538|22581|4|266|1946|45|63.82|125.72|110.63|0.00|4978.35|2871.90|5657.40|298.70|0.00|4978.35|5277.05|2106.45| +2451442|50669|10772|30852|1420095|538|22581|4|246|1946|32|14.15|16.13|13.87|0.00|443.84|452.80|516.16|17.75|0.00|443.84|461.59|-8.96| +2451442|50669|2386|30852|1420095|538|22581|4|79|1946|48|76.41|118.43|22.50|561.60|1080.00|3667.68|5684.64|10.36|561.60|518.40|528.76|-3149.28| +2451442|50669|15274|30852|1420095|538|22581|4|48|1946|44|71.56|104.47|11.49|0.00|505.56|3148.64|4596.68|25.27|0.00|505.56|530.83|-2643.08| +2451904|34464|1591|22872|1036153|4187|42268|8|21|1947|12|55.64|107.94|19.42|0.00|233.04|667.68|1295.28|16.31|0.00|233.04|249.35|-434.64| +2451904|34464|6149|22872|1036153|4187|42268|8|245|1947|95|12.24|12.60|11.34|0.00|1077.30|1162.80|1197.00|86.18|0.00|1077.30|1163.48|-85.50| +2451904|34464|15440|22872|1036153|4187|42268|8|214|1947|40|53.37|64.04|29.45|0.00|1178.00|2134.80|2561.60|106.02|0.00|1178.00|1284.02|-956.80| +2451904|34464|13399|22872|1036153|4187|42268|8|116|1947|30|46.34|82.94|43.95|791.10|1318.50|1390.20|2488.20|5.27|791.10|527.40|532.67|-862.80| +2451904|34464|12845|22872|1036153|4187|42268|8|96|1947|59|40.89|77.28|42.50|551.65|2507.50|2412.51|4559.52|97.79|551.65|1955.85|2053.64|-456.66| +2451904|34464|10751|22872|1036153|4187|42268|8|176|1947|60|84.24|135.62|52.89|0.00|3173.40|5054.40|8137.20|0.00|0.00|3173.40|3173.40|-1881.00| +2451904|34464|10112|22872|1036153|4187|42268|8|268|1947|65|1.74|3.48|0.66|0.00|42.90|113.10|226.20|3.43|0.00|42.90|46.33|-70.20| +2451904|34464|15524|22872|1036153|4187|42268|8|267|1947|9|26.97|36.67|22.73|0.00|204.57|242.73|330.03|16.36|0.00|204.57|220.93|-38.16| +2451904|34464|7013|22872|1036153|4187|42268|8|52|1947|61|13.85|25.48|18.09|0.00|1103.49|844.85|1554.28|77.24|0.00|1103.49|1180.73|258.64| +2451904|34464|12731|22872|1036153|4187|42268|8|227|1947|84|6.42|10.27|2.05|0.00|172.20|539.28|862.68|3.44|0.00|172.20|175.64|-367.08| +2451904|34464|4529|22872|1036153|4187|42268|8|30|1947|70|84.07|161.41|46.80|1015.56|3276.00|5884.90|11298.70|22.60|1015.56|2260.44|2283.04|-3624.46| +2451089|47991|12452|44008|1024233|5258|48683|10|216|1948|87|95.38|165.00|120.45|7859.36|10479.15|8298.06|14355.00|157.18|7859.36|2619.79|2776.97|-5678.27| +2451089|47991|3668|44008|1024233|5258|48683|10|255|1948|9|31.75|47.30|18.92|0.00|170.28|285.75|425.70|8.51|0.00|170.28|178.79|-115.47| +2451089|47991|17480|44008|1024233|5258|48683|10|293|1948|72|15.56|26.91|1.61|0.00|115.92|1120.32|1937.52|9.27|0.00|115.92|125.19|-1004.40| +2451089|47991|14060|44008|1024233|5258|48683|10|56|1948|30|13.40|23.04|8.98|0.00|269.40|402.00|691.20|8.08|0.00|269.40|277.48|-132.60| +2451089|47991|5884|44008|1024233|5258|48683|10|187|1948|89|70.55|130.51|117.45|0.00|10453.05|6278.95|11615.39|836.24|0.00|10453.05|11289.29|4174.10| +2451089|47991|6211|44008|1024233|5258|48683|10|168|1948|81|72.58|80.56|70.89|0.00|5742.09|5878.98|6525.36|459.36|0.00|5742.09|6201.45|-136.89| +2451089|47991|7288|44008|1024233|5258|48683|10|17|1948|40|92.63|96.33|38.53|0.00|1541.20|3705.20|3853.20|0.00|0.00|1541.20|1541.20|-2164.00| +2451089|47991|8072|44008|1024233|5258|48683|10|250|1948|37|36.95|55.05|35.23|1264.40|1303.51|1367.15|2036.85|3.12|1264.40|39.11|42.23|-1328.04| +2451089|47991|1765|44008|1024233|5258|48683|10|246|1948|63|36.93|40.62|0.40|0.00|25.20|2326.59|2559.06|0.25|0.00|25.20|25.45|-2301.39| +2451089|47991|9290|44008|1024233|5258|48683|10|245|1948|78|5.40|6.42|6.35|0.00|495.30|421.20|500.76|9.90|0.00|495.30|505.20|74.10| +2451089|47991|13807|44008|1024233|5258|48683|10|223|1948|12|65.48|120.48|20.48|0.00|245.76|785.76|1445.76|0.00|0.00|245.76|245.76|-540.00| +2451089|47991|265|44008|1024233|5258|48683|10|239|1948|93|98.13|186.44|167.79|0.00|15604.47|9126.09|17338.92|1248.35|0.00|15604.47|16852.82|6478.38| +2451089|47991|11254|44008|1024233|5258|48683|10|47|1948|90|3.80|5.47|2.95|0.00|265.50|342.00|492.30|13.27|0.00|265.50|278.77|-76.50| +2451089|47991|9859|44008|1024233|5258|48683|10|195|1948|92|16.43|22.83|15.06|0.00|1385.52|1511.56|2100.36|41.56|0.00|1385.52|1427.08|-126.04| +2451089|47991|4918|44008|1024233|5258|48683|10|133|1948|39|24.35|28.97|8.40|209.66|327.60|949.65|1129.83|2.35|209.66|117.94|120.29|-831.71| +2451089|47991|11402|44008|1024233|5258|48683|10|122|1948|80|21.73|42.80|32.52|1404.86|2601.60|1738.40|3424.00|83.77|1404.86|1196.74|1280.51|-541.66| +2452222|59115|1007|53641|1288746|3498|21403|2|213|1949|85|58.92|70.70|53.02|1712.54|4506.70|5008.20|6009.50|83.82|1712.54|2794.16|2877.98|-2214.04| +2452222|59115|7721|53641|1288746|3498|21403|2|224|1949|29|95.79|147.51|107.68|0.00|3122.72|2777.91|4277.79|187.36|0.00|3122.72|3310.08|344.81| +2452222|59115|10373|53641|1288746|3498|21403|2|155|1949|53|86.20|99.13|93.18|0.00|4938.54|4568.60|5253.89|197.54|0.00|4938.54|5136.08|369.94| +2452222|59115|16681|53641|1288746|3498|21403|2|203|1949|40|11.54|19.38|17.05|0.00|682.00|461.60|775.20|27.28|0.00|682.00|709.28|220.40| +2452222|59115|10333|53641|1288746|3498|21403|2|287|1949|50|10.55|15.19|3.94|0.00|197.00|527.50|759.50|13.79|0.00|197.00|210.79|-330.50| +2452222|59115|9841|53641|1288746|3498|21403|2|234|1949|62|97.13|144.72|17.36|0.00|1076.32|6022.06|8972.64|32.28|0.00|1076.32|1108.60|-4945.74| +2452222|59115|16119|53641|1288746|3498|21403|2|232|1949|99|36.24|50.01|50.01|0.00|4950.99|3587.76|4950.99|49.50|0.00|4950.99|5000.49|1363.23| +2452222|59115|16991|53641|1288746|3498|21403|2|37|1949|3|26.74|32.89|0.65|0.00|1.95|80.22|98.67|0.07|0.00|1.95|2.02|-78.27| +|59115|4211|53641||3498||2||1949|28|||95.01|||2742.04||0.00|||2660.28|| +2452222|59115|11989|53641|1288746|3498|21403|2|215|1949|53|28.06|48.26|41.50|0.00|2199.50|1487.18|2557.78|153.96|0.00|2199.50|2353.46|712.32| +2452222|59115|13695|53641|1288746|3498|21403|2|135|1949|75|47.27|73.74|47.19|0.00|3539.25|3545.25|5530.50|0.00|0.00|3539.25|3539.25|-6.00| +2452222|59115|14051|53641|1288746|3498|21403|2|207|1949|29|24.62|28.31|20.10|0.00|582.90|713.98|820.99|0.00|0.00|582.90|582.90|-131.08| +2452607|54105|6807|52193|1612957|1368|7779|1|217|1950|51|43.58|63.62|55.98|0.00|2854.98|2222.58|3244.62|85.64|0.00|2854.98|2940.62|632.40| +2452607|54105|6891|52193|1612957|1368|7779|1|96|1950|53|69.30|110.88|100.90|5026.83|5347.70|3672.90|5876.64|9.62|5026.83|320.87|330.49|-3352.03| +2452607|54105|17781|52193|1612957|1368|7779|1|248|1950|59|78.69|151.87|37.96|1522.95|2239.64|4642.71|8960.33|35.83|1522.95|716.69|752.52|-3926.02| +2452607|54105|4188|52193|1612957|1368|7779|1|189|1950|79|45.36|59.87|28.13|0.00|2222.27|3583.44|4729.73|155.55|0.00|2222.27|2377.82|-1361.17| +|54105|12661|52193|1612957||||182|1950||23.97|43.62||0.00||910.86|1657.56||0.00|1044.24|1075.56|| +2452607|54105|6283|52193|1612957|1368|7779|1|40|1950|59|53.39|89.16|23.18|437.63|1367.62|3150.01|5260.44|37.19|437.63|929.99|967.18|-2220.02| +2452607|54105|11253|52193|1612957|1368|7779|1|206|1950|48|75.05|105.82|92.06|0.00|4418.88|3602.40|5079.36|44.18|0.00|4418.88|4463.06|816.48| +2452607|54105|4956|52193|1612957|1368|7779|1|58|1950|100|10.41|18.52|11.48|0.00|1148.00|1041.00|1852.00|103.32|0.00|1148.00|1251.32|107.00| +2452607|54105|13849|52193|1612957|1368|7779|1|254|1950|18|99.62|178.31|99.85|1258.11|1797.30|1793.16|3209.58|32.35|1258.11|539.19|571.54|-1253.97| +2452416|45402|3517|11916|587810|4057|8351|7|174|1951|43|71.00|124.96|26.24|0.00|1128.32|3053.00|5373.28|0.00|0.00|1128.32|1128.32|-1924.68| +2452416|45402|6258|11916|587810|4057|8351|7|263|1951|31|19.80|21.18|5.71|0.00|177.01|613.80|656.58|5.31|0.00|177.01|182.32|-436.79| +2452416|45402|11484|||4057|8351||10|1951|28|28.90|||0.00|||1480.64|8.43|0.00|843.92||34.72| +2452416|45402|8731|11916|587810|4057|8351|7|211|1951|83|1.93|2.20|1.12|0.00|92.96|160.19|182.60|4.64|0.00|92.96|97.60|-67.23| +2452416|45402|2046|11916|587810|4057|8351|7|292|1951|17|40.28|62.83|49.00|0.00|833.00|684.76|1068.11|74.97|0.00|833.00|907.97|148.24| +2452416|45402|15741|11916|587810|4057|8351|7|185|1951|41|3.82|5.73|4.24|0.00|173.84|156.62|234.93|6.95|0.00|173.84|180.79|17.22| +2452416|45402|6228|11916|587810|4057|8351|7|185|1951|4|98.56|168.53|47.18|0.00|188.72|394.24|674.12|15.09|0.00|188.72|203.81|-205.52| +2452416|45402|17184|11916|587810|4057|8351|7|230|1951|5|43.81|60.45|47.75|0.00|238.75|219.05|302.25|16.71|0.00|238.75|255.46|19.70| +2452416|45402|747|11916|587810|4057|8351|7|175|1951|17|35.59|47.69|37.67|0.00|640.39|605.03|810.73|19.21|0.00|640.39|659.60|35.36| +2452416|45402|17541|11916|587810|4057|8351|7|273|1951|36|84.36|149.31|107.50|2089.80|3870.00|3036.96|5375.16|0.00|2089.80|1780.20|1780.20|-1256.76| +|45402|1380||||8351||258|1951||81.34|95.98||0.00|5908.24|6181.84|7294.48||0.00|5908.24|6203.65|| +2452416|45402|12631|11916|587810|4057|8351|7|93|1951|55|93.79|112.54|10.12|0.00|556.60|5158.45|6189.70|11.13|0.00|556.60|567.73|-4601.85| +2452416|45402|9061|11916|587810|4057|8351|7|172|1951|23|39.11|75.48|64.91|0.00|1492.93|899.53|1736.04|44.78|0.00|1492.93|1537.71|593.40| +||13356||587810|4057|8351|7|56|1951|||17.38||||767.34||1.59||159.38||| +2451066|32853|12034|25594|913605|16|11227|8|243|1952|85|33.25|63.50|6.35|0.00|539.75|2826.25|5397.50|16.19|0.00|539.75|555.94|-2286.50| +2451066|32853|14230|25594|913605|16|11227|8|98|1952|5|51.55|65.46|27.49|0.00|137.45|257.75|327.30|12.37|0.00|137.45|149.82|-120.30| +2451066|32853|14374|25594|913605|16|11227|8|268|1952|95|80.16|118.63|55.75|0.00|5296.25|7615.20|11269.85|211.85|0.00|5296.25|5508.10|-2318.95| +2451066|32853|11341|25594|913605|16|11227|8|198|1952|46|44.43|73.75|7.37|0.00|339.02|2043.78|3392.50|30.51|0.00|339.02|369.53|-1704.76| +2451066|32853|2582|25594|913605|16|11227|8|201|1952|1|25.19|26.95|15.90|8.42|15.90|25.19|26.95|0.67|8.42|7.48|8.15|-17.71| +2451066||5284||913605||11227|||1952||91.34|118.74||0.00|5186.22|7124.52||311.17|0.00|5186.22||| +2451066|32853|14654|25594|913605|16|11227|8|108|1952|17|31.88|53.23|29.80|0.00|506.60|541.96|904.91|10.13|0.00|506.60|516.73|-35.36| +2451066|32853|272|25594|913605|16|11227|8|146|1952|16|85.93|138.34|76.08|0.00|1217.28|1374.88|2213.44|109.55|0.00|1217.28|1326.83|-157.60| +2451066|32853|9430|25594|913605|16|11227|8|217|1952|31|84.66|92.27|35.06|0.00|1086.86|2624.46|2860.37|21.73|0.00|1086.86|1108.59|-1537.60| +2451066|32853|15668|25594|913605|16|11227|8|227|1952|64|7.48|12.26|7.23|393.31|462.72|478.72|784.64|3.47|393.31|69.41|72.88|-409.31| +2451066|32853|13534|25594|913605|16|11227|8|254|1952|99|10.39|12.46|8.47|0.00|838.53|1028.61|1233.54|16.77|0.00|838.53|855.30|-190.08| +2451066|32853|3098|25594|913605||11227|8|139|1952||||53.82|0.00|161.46||414.00|14.53|0.00|||| +2450894|46438|3784|78110|182518|4079|40487|8|145|1953|81|83.06|105.48|95.98|0.00|7774.38|6727.86|8543.88|77.74|0.00|7774.38|7852.12|1046.52| +2450894|46438|9013|78110|182518|4079|40487|8|45|1953|20|32.18|41.19|1.64|24.27|32.80|643.60|823.80|0.17|24.27|8.53|8.70|-635.07| +2450894|46438|1993|78110|182518|4079|40487|8|52|1953|57|28.68|45.88|3.67|0.00|209.19|1634.76|2615.16|18.82|0.00|209.19|228.01|-1425.57| +2450894|46438|8768|78110|182518|4079|40487|8|135|1953|60|75.92|100.21|47.09|0.00|2825.40|4555.20|6012.60|28.25|0.00|2825.40|2853.65|-1729.80| +||2444||182518|4079||||1953||64.42|123.04|104.58|0.00|9830.52||11565.76|196.61|0.00|9830.52|10027.13|3775.04| +2450894|46438|12848|78110|182518|4079|40487|8|102|1953|88|27.15|34.48|26.89|0.00|2366.32|2389.20|3034.24|94.65|0.00|2366.32|2460.97|-22.88| +2450894|46438|12997|78110|182518|4079|40487|8|21|1953|96|27.32|50.26|31.66|0.00|3039.36|2622.72|4824.96|0.00|0.00|3039.36|3039.36|416.64| +2450894|46438|5576|78110|182518|4079|40487|8|168|1953|65|78.23|145.50|14.55|0.00|945.75|5084.95|9457.50|28.37|0.00|945.75|974.12|-4139.20| +2450894|46438|2872|78110|182518|4079|40487|8|158|1953|87|66.83|120.29|13.23|0.00|1151.01|5814.21|10465.23|11.51|0.00|1151.01|1162.52|-4663.20| +2450894|46438|9764|78110|182518|4079|40487|8|256|1953|81|30.22|41.70|27.52|0.00|2229.12|2447.82|3377.70|133.74|0.00|2229.12|2362.86|-218.70| +2450894|46438|14138|78110|182518|4079|40487|8|189|1953|43|6.58|8.15|2.20|0.00|94.60|282.94|350.45|0.94|0.00|94.60|95.54|-188.34| +||6464||182518||40487||31|1953||97.48||||2736.72||3649.32|182.26||2025.18||-1484.10| +2450894|46438|8125|78110|182518|4079|40487|8|131|1953|15|47.24|56.68|5.10|23.71|76.50|708.60|850.20|0.00|23.71|52.79|52.79|-655.81| +2450894|46438|7156|78110|182518|4079|40487|8|189|1953|48|62.94|86.22|58.62|0.00|2813.76|3021.12|4138.56|56.27|0.00|2813.76|2870.03|-207.36| +2450894|46438|139|78110|182518|4079|40487|8|30|1953|13|51.97|85.23|62.21|0.00|808.73|675.61|1107.99|56.61|0.00|808.73|865.34|133.12| +2451425|33098|9338|16204|1862553|4102|33651|2|16|1954|17|62.41|109.84|28.55|0.00|485.35|1060.97|1867.28|38.82|0.00|485.35|524.17|-575.62| +2451425||6595|||4102||2||1954||55.54|82.75||0.00||555.40||56.26|0.00||759.56|147.90| +||14018|16204|1862553||33651|2||1954|14|40.49|||119.11|496.30||855.82||119.11|377.19||-189.67| +2451425|33098|15316|16204|1862553|4102|33651|2|258|1954|85|13.58|19.41|19.21|0.00|1632.85|1154.30|1649.85|65.31|0.00|1632.85|1698.16|478.55| +2451425|33098|17282|16204|1862553|4102|33651|2|147|1954|49|63.78|119.26|53.66|0.00|2629.34|3125.22|5843.74|157.76|0.00|2629.34|2787.10|-495.88| +2451425|33098|17434|16204|1862553|4102|33651|2|263|1954|17|9.85|13.59|4.48|0.00|76.16|167.45|231.03|4.56|0.00|76.16|80.72|-91.29| +2451425|33098|16819|16204|1862553|4102|33651|2||1954|32|33.90|||||1084.80|1637.76|||||438.08| +2451425|33098|12634|16204|1862553|4102|33651|2|211|1954|19|34.10|67.51|62.78|0.00|1192.82|647.90|1282.69|0.00|0.00|1192.82|1192.82|544.92| +2451500|45584|4075|67727|1168986|645|41151|10|1|1955|62|57.83|102.35|93.13|0.00|5774.06|3585.46|6345.70|173.22|0.00|5774.06|5947.28|2188.60| +2451500|45584|9130|67727|1168986|645|41151|10|10|1955|52|25.74|48.39|4.83|0.00|251.16|1338.48|2516.28|22.60|0.00|251.16|273.76|-1087.32| +2451500|45584|2494|67727|1168986|645|41151|10|253|1955|16|35.59|36.30|10.52|139.70|168.32|569.44|580.80|0.85|139.70|28.62|29.47|-540.82| +2451500|45584|7651|67727|1168986|645|41151|10|257|1955|64|67.88|104.53|84.66|0.00|5418.24|4344.32|6689.92|379.27|0.00|5418.24|5797.51|1073.92| +2451500|45584|3482|67727|1168986|645|41151|10|23|1955|37|73.98|86.55|6.05|0.00|223.85|2737.26|3202.35|6.71|0.00|223.85|230.56|-2513.41| +2451500|45584|4048|67727|1168986|645|41151|10|234|1955|82|3.86|4.28|3.72|149.46|305.04|316.52|350.96|4.66|149.46|155.58|160.24|-160.94| +2451500|45584|10189|67727|1168986|645|41151|10|135|1955|96|43.51|66.13|19.17|0.00|1840.32|4176.96|6348.48|73.61|0.00|1840.32|1913.93|-2336.64| +2451500|45584|146|67727|1168986|645|41151|10|69|1955|95|13.85|25.62|8.19|0.00|778.05|1315.75|2433.90|15.56|0.00|778.05|793.61|-537.70| +2451500|45584|12104|67727|1168986|645|41151|10|226|1955|38|29.33|58.66|22.29|389.62|847.02|1114.54|2229.08|41.16|389.62|457.40|498.56|-657.14| +2451500|45584|16207|67727|1168986|645|41151|10|282|1955|80|35.28|64.56|0.64|0.00|51.20|2822.40|5164.80|1.53|0.00|51.20|52.73|-2771.20| +2451500|45584|5530|67727|1168986|645|41151|10|155|1955|48|86.67|165.53|100.97|1550.89|4846.56|4160.16|7945.44|0.00|1550.89|3295.67|3295.67|-864.49| +2451500|45584|15031|67727|1168986|645|41151|10|17|1955|22|46.97|74.21|11.87|0.00|261.14|1033.34|1632.62|18.27|0.00|261.14|279.41|-772.20| +2452610|68146|7183|7880|1603272|3017|38809|8|280|1956|17|22.47|28.08|0.00|0.00|0.00|381.99|477.36|0.00|0.00|0.00|0.00|-381.99| +2452610|68146|13698|7880|1603272|3017|38809|8|210|1956|99|36.90|65.68|47.94|0.00|4746.06|3653.10|6502.32|427.14|0.00|4746.06|5173.20|1092.96| +2452610|68146|14904|7880|1603272|3017|38809|8|149|1956|50|76.90|79.97|13.59|0.00|679.50|3845.00|3998.50|61.15|0.00|679.50|740.65|-3165.50| +2452610|68146|2859|7880|1603272|3017|38809|8|131|1956|37|94.18|187.41|91.83|2344.41|3397.71|3484.66|6934.17|0.00|2344.41|1053.30|1053.30|-2431.36| +2452610|68146|12673|7880|1603272|3017|38809|8|21|1956|27|46.40|56.14|45.47|0.00|1227.69|1252.80|1515.78|0.00|0.00|1227.69|1227.69|-25.11| +2452610|68146|1|7880|1603272|3017|38809|8|31|1956|34|79.93|112.70|27.04|0.00|919.36|2717.62|3831.80|0.00|0.00|919.36|919.36|-1798.26| +2452610|68146|589|7880|1603272|3017|38809|8|61|1956|91|55.27|72.40|19.54|0.00|1778.14|5029.57|6588.40|17.78|0.00|1778.14|1795.92|-3251.43| +||7459|||3017|38809|8||1956||91.64||86.61|0.00|5456.43|5773.32|8024.31|381.95|0.00|||-316.89| +2452610|68146|6421|7880|1603272|3017|38809|8|72|1956|79|30.66|53.04|41.90|0.00|3310.10|2422.14|4190.16|99.30|0.00|3310.10|3409.40|887.96| +2452610|68146|17652|7880|1603272|3017|38809|8|250|1956|89|74.27|80.95|28.33|1386.75|2521.37|6610.03|7204.55|79.42|1386.75|1134.62|1214.04|-5475.41| +2452610|68146|2814|7880|1603272|3017|38809|8|85|1956|39|65.66|126.72|78.56|0.00|3063.84|2560.74|4942.08|30.63|0.00|3063.84|3094.47|503.10| +2452626|69532|8106|36205|614200|4195|30637|2|172|1957|41|35.10|55.80|26.22|0.00|1075.02|1439.10|2287.80|96.75|0.00|1075.02|1171.77|-364.08| +2452626|69532|16197|36205|614200|4195|30637|2|137|1957|65|7.76|12.49|8.36|0.00|543.40|504.40|811.85|48.90|0.00|543.40|592.30|39.00| +2452626|69532|5571|36205|614200|4195|30637|2|18|1957|79|36.83|61.13|31.17|2216.18|2462.43|2909.57|4829.27|12.31|2216.18|246.25|258.56|-2663.32| +2452626|69532|11923|36205|614200|4195|30637|2|100|1957|99|98.93|104.86|97.51|7336.65|9653.49|9794.07|10381.14|162.17|7336.65|2316.84|2479.01|-7477.23| +2452626|69532|9048||614200|4195|||169|1957|98||111.36||||6574.82|10913.28|||7639.10||1064.28| +2452626|69532|4693|36205|614200|4195|30637|2|33|1957|6|36.84|62.25|26.76|0.00|160.56|221.04|373.50|4.81|0.00|160.56|165.37|-60.48| +2452626|69532|16459|36205|614200|4195|30637|2|191|1957|34|69.20|104.49|18.80|0.00|639.20|2352.80|3552.66|31.96|0.00|639.20|671.16|-1713.60| +2452626|69532|8580|36205|614200|4195|30637|2|113|1957|17|18.38|29.77|2.97|0.00|50.49|312.46|506.09|3.02|0.00|50.49|53.51|-261.97| +2452626|69532|10909|36205|614200|4195|30637|2|54|1957|57|7.24|14.33|4.58|0.00|261.06|412.68|816.81|23.49|0.00|261.06|284.55|-151.62| +2452626|69532|11869|36205|614200|4195|30637|2|108|1957|9|55.14|67.82|56.96|0.00|512.64|496.26|610.38|35.88|0.00|512.64|548.52|16.38| +2452626|69532|2430|36205|614200|4195|30637|2|162|1957|8|32.60|60.96|1.21|0.00|9.68|260.80|487.68|0.09|0.00|9.68|9.77|-251.12| +2452626|69532|3351|36205|614200|4195|30637|2|173|1957|11|61.81|100.75|3.02|18.93|33.22|679.91|1108.25|0.00|18.93|14.29|14.29|-665.62| +2452626|69532|12390|36205|614200|4195|30637|2|69|1957|8|18.42|36.84|30.20|45.90|241.60|147.36|294.72|11.74|45.90|195.70|207.44|48.34| +2451356|29080|16180|26066|883357|5448|49139|10|27|1958|89|12.56|22.10|3.09|0.00|275.01|1117.84|1966.90|0.00|0.00|275.01|275.01|-842.83| +2451356|29080|15199|26066|883357|5448|49139|10|263|1958|90|8.69|9.12|3.92|0.00|352.80|782.10|820.80|21.16|0.00|352.80|373.96|-429.30| +2451356||16262||883357||||116|1958|23|19.09|||0.00|558.90||||0.00|558.90||119.83| +2451356|29080|5299|26066|883357|5448|49139|10|174|1958|44|10.29|15.02|6.30|213.44|277.20|452.76|660.88|5.73|213.44|63.76|69.49|-389.00| +2451356|29080|14630|26066|883357|5448|49139|10|97|1958|65|35.74|64.33|13.50|0.00|877.50|2323.10|4181.45|43.87|0.00|877.50|921.37|-1445.60| +2451356|29080|16942|26066|883357|5448|49139|10|279|1958|48|59.42|102.79|11.30|108.48|542.40|2852.16|4933.92|39.05|108.48|433.92|472.97|-2418.24| +2451356|29080|2464|26066|883357|5448|49139|10|41|1958|22|69.28|128.86|68.29|0.00|1502.38|1524.16|2834.92|30.04|0.00|1502.38|1532.42|-21.78| +2451356|29080|8516|26066|883357|5448|49139|10|137|1958|68|62.60|93.90|40.37|0.00|2745.16|4256.80|6385.20|192.16|0.00|2745.16|2937.32|-1511.64| +2451356|29080|14362|26066|883357|5448|49139|10|2|1958|83|6.95|10.21|2.85|0.00|236.55|576.85|847.43|14.19|0.00|236.55|250.74|-340.30| +2451356|29080|2560||883357||||203|1958||88.18|171.06|8.55|||2116.32|4105.44|||205.20||-1911.12| +2451356|29080|14704|26066|883357|5448|49139|10|143|1958|53|10.77|12.06|7.47|0.00|395.91|570.81|639.18|19.79|0.00|395.91|415.70|-174.90| +2451356|29080|3973|26066|883357|5448|49139|10|132|1958|44|81.81|143.16|14.31|0.00|629.64|3599.64|6299.04|25.18|0.00|629.64|654.82|-2970.00| +2451356|29080|6070|26066|883357|5448|49139|10|237|1958|81|3.50|4.48|4.48|0.00|362.88|283.50|362.88|29.03|0.00|362.88|391.91|79.38| +2451356||6883||||||185|1958||34.95||20.58|0.00|1584.66|||95.07|0.00||1679.73|-1106.49| +|62351|13770||1818059|1814|30369|8|168|1959|||81.10||0.00||4437.58||6.58|0.00|||| +2452628|62351|8377|85151|1818059|1814|30369|8|245|1959|68|2.57|3.49|3.49|0.00|237.32|174.76|237.32|16.61|0.00|237.32|253.93|62.56| +2452628|62351|1621|85151|1818059|1814|30369|8|93|1959|43|37.05|70.76|65.09|867.64|2798.87|1593.15|3042.68|173.81|867.64|1931.23|2105.04|338.08| +2452628||6549|85151|1818059|1814|||119|1959|48||98.61|||||4733.28|||||| +2452628|62351|10843|85151|1818059|1814|30369|8|248|1959|64|49.32|56.71|25.51|0.00|1632.64|3156.48|3629.44|65.30|0.00|1632.64|1697.94|-1523.84| +2452628|62351|8755|85151|1818059|1814|30369|8|147|1959|43|12.38|19.68|16.72|158.17|718.96|532.34|846.24|44.86|158.17|560.79|605.65|28.45| +2452628||6294||1818059|1814||||1959|10|46.08|70.96|48.25|0.00||460.80||43.42|0.00||525.92|21.70| +2452628|62351|5646|85151|1818059|1814|30369|8|204|1959|63|81.90|118.75|22.56|0.00|1421.28|5159.70|7481.25|71.06|0.00|1421.28|1492.34|-3738.42| +2452628|62351|9315|85151|1818059|1814|30369|8|252|1959|3|8.46|9.47|8.42|0.00|25.26|25.38|28.41|1.01|0.00|25.26|26.27|-0.12| +2452628|62351|7689|85151|1818059|1814|30369|8|249|1959|16|53.86|66.24|38.41|0.00|614.56|861.76|1059.84|0.00|0.00|614.56|614.56|-247.20| +2452628|62351|427|85151|1818059|1814|30369|8|79|1959|65|67.83|87.50|39.37|102.36|2559.05|4408.95|5687.50|171.96|102.36|2456.69|2628.65|-1952.26| +2452628|62351|5925|85151|1818059|1814|30369|8|129|1959|87|32.09|34.65|12.82|0.00|1115.34|2791.83|3014.55|89.22|0.00|1115.34|1204.56|-1676.49| +2451650|46757|8738|64717|1549940|316|33492|4|100|1960|98|37.26|53.65|24.67|0.00|2417.66|3651.48|5257.70|145.05|0.00|2417.66|2562.71|-1233.82| +2451650|46757|6277|64717|1549940|316|33492|4|130|1960|27|94.38|153.83|21.53|186.01|581.31|2548.26|4153.41|27.67|186.01|395.30|422.97|-2152.96| +2451650|46757|10715|64717|1549940|316|33492|4|222|1960|60|11.71|13.58|1.76|34.84|105.60|702.60|814.80|0.70|34.84|70.76|71.46|-631.84| +2451650|46757|16970|64717|1549940|316|33492|4|202|1960|58|36.24|36.60|3.29|0.00|190.82|2101.92|2122.80|11.44|0.00|190.82|202.26|-1911.10| +2451650|46757|10928|64717|1549940|316|33492|4|105|1960|47|47.40|49.77|11.94|0.00|561.18|2227.80|2339.19|39.28|0.00|561.18|600.46|-1666.62| +2451650|46757|2996|64717|1549940|316|33492|4|133|1960|52|97.56|193.16|106.23|0.00|5523.96|5073.12|10044.32|276.19|0.00|5523.96|5800.15|450.84| +2451650|46757|13171|64717|1549940|316|33492|4|64|1960|47|50.86|100.70|48.33|0.00|2271.51|2390.42|4732.90|181.72|0.00|2271.51|2453.23|-118.91| +2451650|46757|7283|64717|1549940|316|33492|4|68|1960|85|20.98|38.60|16.98|0.00|1443.30|1783.30|3281.00|101.03|0.00|1443.30|1544.33|-340.00| +2451650|46757|15337|64717|1549940|316|33492|4|126|1960|40|89.95|146.61|55.71|0.00|2228.40|3598.00|5864.40|0.00|0.00|2228.40|2228.40|-1369.60| +2451650|46757|17125|64717|1549940|316|33492|4|85|1960|60|57.09|104.47|32.38|0.00|1942.80|3425.40|6268.20|97.14|0.00|1942.80|2039.94|-1482.60| +2451650|46757|14780|64717|1549940|316|33492|4|206|1960|42|80.24|88.26|18.53|474.73|778.26|3370.08|3706.92|24.28|474.73|303.53|327.81|-3066.55| +2452208|47605|11909|75043|1224310|2824|29291|7|161|1961|3|26.88|45.15|33.86|0.00|101.58|80.64|135.45|8.12|0.00|101.58|109.70|20.94| +2452208|47605|6767|75043|1224310|2824|29291|7|254|1961|6|49.66|61.08|56.80|0.00|340.80|297.96|366.48|3.40|0.00|340.80|344.20|42.84| +2452208|47605|707|75043|1224310|2824|29291|7|183|1961|38|77.41|123.85|123.85|0.00|4706.30|2941.58|4706.30|141.18|0.00|4706.30|4847.48|1764.72| +2452208|47605|2423|75043|1224310|2824|29291|7|272|1961|98|54.31|106.44|89.40|0.00|8761.20|5322.38|10431.12|700.89|0.00|8761.20|9462.09|3438.82| +||7421|75043|1224310|2824|29291|7||1961||83.53|154.53||0.00||2004.72||97.90|0.00|1631.76||-372.96| +2452208|47605|7233|75043|1224310|2824|29291|7|59|1961|61|59.57|99.48|93.51|0.00|5704.11|3633.77|6068.28|399.28|0.00|5704.11|6103.39|2070.34| +2452208|47605|2753|75043|1224310|2824|29291|7|255|1961|56|41.61|52.42|46.12|0.00|2582.72|2330.16|2935.52|103.30|0.00|2582.72|2686.02|252.56| +2452208|47605|8567|75043|1224310|2824|29291|7|271|1961|68|74.16|119.39|76.40|0.00|5195.20|5042.88|8118.52|311.71|0.00|5195.20|5506.91|152.32| +2452208|47605|129|75043|1224310|2824|29291|7|258|1961|71|91.13|154.00|133.98|4851.41|9512.58|6470.23|10934.00|419.50|4851.41|4661.17|5080.67|-1809.06| +2452208|47605|17433|75043|1224310|2824|29291|7|168|1961|25|68.76|121.70|43.81|0.00|1095.25|1719.00|3042.50|21.90|0.00|1095.25|1117.15|-623.75| +2452208|47605|2877|75043|1224310|2824|29291|7|142|1961|56|24.20|45.73|28.35|0.00|1587.60|1355.20|2560.88|0.00|0.00|1587.60|1587.60|232.40| +2452208|47605|17323|75043|1224310|2824|29291|7|150|1961|25|26.52|49.06|29.92|0.00|748.00|663.00|1226.50|14.96|0.00|748.00|762.96|85.00| +2452208|47605|4657|75043|1224310|2824|29291|7|208|1961|74|13.73|20.18|11.90|766.12|880.60|1016.02|1493.32|1.14|766.12|114.48|115.62|-901.54| +2452208|47605|14365|75043|1224310|2824|29291|7|156|1961|94|45.07|63.09|1.26|0.00|118.44|4236.58|5930.46|8.29|0.00|118.44|126.73|-4118.14| +2452208|47605|10103|75043|1224310|2824|29291|7|246|1961|91|19.98|25.37|13.95|0.00|1269.45|1818.18|2308.67|114.25|0.00|1269.45|1383.70|-548.73| +2452262|75224|5981|2245|1335245|1845|4845|10|66|1962|20|28.93|57.86|4.05|0.00|81.00|578.60|1157.20|7.29|0.00|81.00|88.29|-497.60| +2452262|75224|4553|2245|1335245|1845|4845|10|93|1962|49|26.35|46.63|19.58|0.00|959.42|1291.15|2284.87|67.15|0.00|959.42|1026.57|-331.73| +2452262|75224|10993|2245|1335245|1845|4845|10|6|1962|50|81.14|95.74|51.69|0.00|2584.50|4057.00|4787.00|25.84|0.00|2584.50|2610.34|-1472.50| +2452262|75224|49|2245|1335245|1845|4845|10|78|1962|10|28.40|37.20|9.67|0.00|96.70|284.00|372.00|0.00|0.00|96.70|96.70|-187.30| +2452262|75224|4583|2245|1335245|1845|4845|10|50|1962|44|92.46|127.59|107.17|0.00|4715.48|4068.24|5613.96|424.39|0.00|4715.48|5139.87|647.24| +2452262|75224|13539|2245|1335245|1845|4845|10|249|1962|99|25.80|45.40|7.71|0.00|763.29|2554.20|4494.60|15.26|0.00|763.29|778.55|-1790.91| +|75224|3249||1335245||||24|1962||15.25|||0.00|367.56|||7.35|0.00||374.91|| +2452262|75224|4683|2245|1335245|1845|4845|10|67|1962|36|37.94|51.59|19.60|0.00|705.60|1365.84|1857.24|63.50|0.00|705.60|769.10|-660.24| +2452262|75224|5527|2245|1335245|1845|4845|10|183|1962|45|61.05|112.33|6.73|148.39|302.85|2747.25|5054.85|3.08|148.39|154.46|157.54|-2592.79| +2452131|71583|8663|6603|1550188|4973|36911|7|267|1963|40|88.16|161.33|9.67|0.00|386.80|3526.40|6453.20|0.00|0.00|386.80|386.80|-3139.60| +2452131|71583|17539|6603|1550188|4973|36911|7|140|1963|43|98.63|114.41|75.51|0.00|3246.93|4241.09|4919.63|162.34|0.00|3246.93|3409.27|-994.16| +2452131|71583|16331|6603|1550188|4973|36911|7|35|1963|7|83.76|103.02|99.92|0.00|699.44|586.32|721.14|41.96|0.00|699.44|741.40|113.12| +2452131|71583|7815|6603|1550188|4973|36911|7|85|1963|3|64.45|83.78|21.78|45.08|65.34|193.35|251.34|0.81|45.08|20.26|21.07|-173.09| +2452131|71583|10559|6603|1550188|4973|36911|7|285|1963|38|80.73|147.73|53.18|0.00|2020.84|3067.74|5613.74|101.04|0.00|2020.84|2121.88|-1046.90| +2452131|71583|15257|6603|1550188|4973|36911|7|234|1963|66|54.92|80.18|59.33|0.00|3915.78|3624.72|5291.88|39.15|0.00|3915.78|3954.93|291.06| +2452131|71583|8787|6603|1550188|4973|36911|7|35|1963|36|53.78|104.33|36.51|0.00|1314.36|1936.08|3755.88|52.57|0.00|1314.36|1366.93|-621.72| +2452131|71583|4941|6603|1550188|4973|36911|7||1963|54|68.87||||91.80|3718.98||1.83||91.80||-3627.18| +2451887|35197|15839|86351|418233|1315|16844|2|175|1964|56|7.08|11.68|2.80|0.00|156.80|396.48|654.08|1.56|0.00|156.80|158.36|-239.68| +2451887|35197|16487|86351|418233|1315|16844|2|160|1964|50|63.92|122.08|78.13|0.00|3906.50|3196.00|6104.00|78.13|0.00|3906.50|3984.63|710.50| +||2987|86351|418233|1315||2||1964|13|96.08|138.35||0.00|431.60|||30.21|0.00|||| +2451887|35197|14552|86351|418233|1315|16844|2|295|1964|72|51.68|87.33|6.11|0.00|439.92|3720.96|6287.76|13.19|0.00|439.92|453.11|-3281.04| +2451887|35197|7879|86351|418233|1315|16844|2|79|1964|38|96.76|169.33|37.25|0.00|1415.50|3676.88|6434.54|99.08|0.00|1415.50|1514.58|-2261.38| +2451887|35197|4159|86351|418233|1315|16844|2|83|1964|89|26.33|52.13|14.07|0.00|1252.23|2343.37|4639.57|75.13|0.00|1252.23|1327.36|-1091.14| +||15925|86351||1315|||55|1964||63.50|73.66|39.77|0.00||6286.50|||0.00||4055.34|-2349.27| +2451887|35197|8923|86351|418233|1315|16844|2|222|1964|57|84.55|108.22|5.41|0.00|308.37|4819.35|6168.54|18.50|0.00|308.37|326.87|-4510.98| +2451887|35197|11021|86351|418233|1315|16844|2|264|1964|47|67.94|67.94|43.48|0.00|2043.56|3193.18|3193.18|102.17|0.00|2043.56|2145.73|-1149.62| +2451887|35197|8815|86351|418233|1315|16844|2|161|1964|48|75.39|83.68|25.94|0.00|1245.12|3618.72|4016.64|99.60|0.00|1245.12|1344.72|-2373.60| +2451887|35197|13082|86351|418233|1315|16844|2|264|1964|47|63.41|83.70|33.48|0.00|1573.56|2980.27|3933.90|141.62|0.00|1573.56|1715.18|-1406.71| +2451887|35197|5459|86351|418233|1315|16844|2|91|1964|68|32.05|50.95|18.34|0.00|1247.12|2179.40|3464.60|62.35|0.00|1247.12|1309.47|-932.28| +2451887|35197|5221|86351|418233|1315|16844|2|120|1964|18|84.27|139.04|104.28|0.00|1877.04|1516.86|2502.72|56.31|0.00|1877.04|1933.35|360.18| +2451597|43812|4406|4860|1503892|6124|28356|10|150|1965|35|24.17|43.74|6.12|0.00|214.20|845.95|1530.90|2.14|0.00|214.20|216.34|-631.75| +2451597|43812|9335|4860|1503892|6124|28356|10|165|1965|20|62.25|124.50|41.08|0.00|821.60|1245.00|2490.00|16.43|0.00|821.60|838.03|-423.40| +2451597|43812|377|4860|1503892|6124|28356|10|126|1965|55|61.53|67.68|40.60|0.00|2233.00|3384.15|3722.40|133.98|0.00|2233.00|2366.98|-1151.15| +2451597|43812|16058|4860|1503892|6124|28356|10|255|1965|68|35.29|39.17|27.41|0.00|1863.88|2399.72|2663.56|111.83|0.00|1863.88|1975.71|-535.84| +2451597|43812|6517|4860|1503892|6124|28356|10|171|1965|69|70.76|111.09|12.21|0.00|842.49|4882.44|7665.21|0.00|0.00|842.49|842.49|-4039.95| +2451597|43812|9158|4860|1503892|6124|28356|10|36|1965|10|66.39|132.78|99.58|0.00|995.80|663.90|1327.80|69.70|0.00|995.80|1065.50|331.90| +|43812|1165||1503892||28356||8|1965||35.69|58.88|31.20|0.00||3354.86||263.95|0.00||3196.75|| +2451597|43812|17479|4860|1503892|6124|28356|10|178|1965|69|29.59|43.49|28.70|0.00|1980.30|2041.71|3000.81|59.40|0.00|1980.30|2039.70|-61.41| +2451597|43812|824|4860|1503892|6124|28356|10|24|1965|70|45.54|52.37|3.14|0.00|219.80|3187.80|3665.90|13.18|0.00|219.80|232.98|-2968.00| +2451597|43812|701|4860|1503892|6124|28356|10|234|1965|14|22.55|23.45|11.49|69.16|160.86|315.70|328.30|8.25|69.16|91.70|99.95|-224.00| +2451597|43812|3860|4860|1503892|6124|28356|10|29|1965|49|90.76|173.35|55.47|0.00|2718.03|4447.24|8494.15|217.44|0.00|2718.03|2935.47|-1729.21| +2451597|43812|14737|4860|1503892|6124|28356|10|88|1965|90|78.50|120.89|47.14|2757.69|4242.60|7065.00|10880.10|14.84|2757.69|1484.91|1499.75|-5580.09| +2451597|43812|16973|4860|1503892|6124|28356|10|295|1965|27|12.59|12.96|8.03|0.00|216.81|339.93|349.92|10.84|0.00|216.81|227.65|-123.12| +2451597|43812|12269|4860|1503892|6124|28356|10|36|1965|20|86.33|110.50|77.35|185.64|1547.00|1726.60|2210.00|54.45|185.64|1361.36|1415.81|-365.24| +2451597|43812|8225|4860|1503892|6124|28356|10|234|1965|54|99.17|160.65|146.19|0.00|7894.26|5355.18|8675.10|394.71|0.00|7894.26|8288.97|2539.08| +2451538|38350|9067|61292||412|3172|||1966|43|||||336.26|200.81||23.53||336.26|359.79|135.45| +2451538|38350|1718|61292|470052|412|3172|7|254|1966|2|27.52|37.15|0.74|0.00|1.48|55.04|74.30|0.05|0.00|1.48|1.53|-53.56| +2451538|38350|3277|61292|470052|412|3172|7|92|1966|68|31.20|43.68|15.28|0.00|1039.04|2121.60|2970.24|83.12|0.00|1039.04|1122.16|-1082.56| +2451538|38350|154|61292|470052|412|3172|7|74|1966|1|91.27|129.60|14.25|4.13|14.25|91.27|129.60|0.91|4.13|10.12|11.03|-81.15| +2451538|38350|9778|61292|470052|412|3172|7|129|1966|85|36.21|53.59|36.97|0.00|3142.45|3077.85|4555.15|0.00|0.00|3142.45|3142.45|64.60| +2451538|38350|1291|61292|470052|412|3172|7|284|1966|17|73.08|102.31|28.64|0.00|486.88|1242.36|1739.27|43.81|0.00|486.88|530.69|-755.48| +2451538|38350|15086|61292|470052|412|3172|7|278|1966|48|84.85|106.91|95.14|0.00|4566.72|4072.80|5131.68|0.00|0.00|4566.72|4566.72|493.92| +2451538|38350|7676|61292|470052|412|3172|7|289|1966|66|10.27|18.99|4.55|0.00|300.30|677.82|1253.34|21.02|0.00|300.30|321.32|-377.52| +2451538|38350|13372|61292|470052|412|3172|7|150|1966|85|56.52|90.43|9.94|346.40|844.90|4804.20|7686.55|39.88|346.40|498.50|538.38|-4305.70| +2451538|38350|5218|61292|470052|412||7|183|1966|48||155.38|127.41||6115.68||7458.24||||6543.77|| +2451538|38350|8713|61292|470052|412|3172|7|70|1966|84|12.08|13.04|0.65|0.00|54.60|1014.72|1095.36|2.18|0.00|54.60|56.78|-960.12| +2451538|38350|7322|61292|470052|412|3172|7|198|1966|98|30.78|30.78|23.39|0.00|2292.22|3016.44|3016.44|183.37|0.00|2292.22|2475.59|-724.22| +2451538|38350|7711|61292|470052|412|3172|7|47|1966|95|60.10|103.97|45.74|0.00|4345.30|5709.50|9877.15|43.45|0.00|4345.30|4388.75|-1364.20| +2451538|38350|14506|61292|470052|412|3172|7|218|1966|36|22.40|43.68|25.77|0.00|927.72|806.40|1572.48|64.94|0.00|927.72|992.66|121.32| +2451308|33426|3685|62613|610194|2222|15003|2|5|1967|55|39.75|65.98|27.05|1026.54|1487.75|2186.25|3628.90|41.50|1026.54|461.21|502.71|-1725.04| +2451308|33426|5282|62613|610194|2222|15003|2|261|1967|97|71.78|89.00|64.08|0.00|6215.76|6962.66|8633.00|124.31|0.00|6215.76|6340.07|-746.90| +2451308|33426|5434|62613|610194|2222|15003|2|120|1967|75|23.67|27.69|14.39|129.51|1079.25|1775.25|2076.75|47.48|129.51|949.74|997.22|-825.51| +2451308|33426|10366|62613|610194|2222|15003|2|165|1967|28|85.32|145.89|74.40|1270.75|2083.20|2388.96|4084.92|48.74|1270.75|812.45|861.19|-1576.51| +2451308|33426|13540|62613|610194|2222|15003|2|9|1967|1|45.44|72.24|5.77|1.84|5.77|45.44|72.24|0.19|1.84|3.93|4.12|-41.51| +2451308|33426|10586|62613|610194|2222|15003|2|89|1967|13|55.20|56.30|31.52|311.41|409.76|717.60|731.90|1.96|311.41|98.35|100.31|-619.25| +2451308|33426|2906|62613|610194|2222|15003|2|24|1967|91|88.70|159.66|59.07|0.00|5375.37|8071.70|14529.06|268.76|0.00|5375.37|5644.13|-2696.33| +2451308|33426|11576|62613|610194|2222|15003|2|203|1967|52|51.70|61.00|28.06|102.13|1459.12|2688.40|3172.00|67.84|102.13|1356.99|1424.83|-1331.41| +2451308|33426|4862|62613|610194|2222|15003|2|5|1967|100|48.41|55.18|36.97|2624.87|3697.00|4841.00|5518.00|10.72|2624.87|1072.13|1082.85|-3768.87| +2451308|33426|4129|62613|610194|2222|15003|2|217|1967|51|4.11|6.69|4.14|0.00|211.14|209.61|341.19|6.33|0.00|211.14|217.47|1.53| +2451161|68098|13130|87217|1114475|2533|31791|4|181|1968|53|98.69|154.94|139.44|0.00|7390.32|5230.57|8211.82|147.80|0.00|7390.32|7538.12|2159.75| +2451161|68098|3520|87217|1114475|2533|31791|4|8|1968|45|55.78|64.14|4.48|0.00|201.60|2510.10|2886.30|6.04|0.00|201.60|207.64|-2308.50| +2451161|68098|16846|87217|1114475|2533|31791|4|172|1968|71|66.68|108.02|52.92|3005.85|3757.32|4734.28|7669.42|0.00|3005.85|751.47|751.47|-3982.81| +2451161||3139||||31791|4||1968|47|8.53|||0.00|556.01|400.91|741.66|11.12|0.00|||| +2451161|68098|6632|87217|1114475|2533|31791|4|183|1968|70|95.83|160.99|133.62|1496.54|9353.40|6708.10|11269.30|628.54|1496.54|7856.86|8485.40|1148.76| +2451161|68098|8383|87217|1114475|2533|31791|4|125|1968|59|77.45|121.59|7.29|0.00|430.11|4569.55|7173.81|4.30|0.00|430.11|434.41|-4139.44| +2451161|68098|10717|87217|1114475|2533|31791|4|66|1968|63|37.98|72.54|19.58|0.00|1233.54|2392.74|4570.02|61.67|0.00|1233.54|1295.21|-1159.20| +2451161|68098|11764|87217|1114475|2533|31791|4|59|1968|28|2.59|2.59|1.96|0.00|54.88|72.52|72.52|4.93|0.00|54.88|59.81|-17.64| +2451161|68098|14006|87217|1114475|2533|31791|4|256|1968|31|73.47|132.24|71.40|996.03|2213.40|2277.57|4099.44|48.69|996.03|1217.37|1266.06|-1060.20| +2451161|68098|1514|87217|1114475|2533|31791|4|265|1968|94|26.18|42.41|25.02|0.00|2351.88|2460.92|3986.54|188.15|0.00|2351.88|2540.03|-109.04| +2451161|68098|17431|87217|1114475|2533|31791|4|79|1968|11|53.97|64.76|53.75|0.00|591.25|593.67|712.36|47.30|0.00|591.25|638.55|-2.42| +|68098|12014|87217|1114475||31791|4||1968|96||||||||||0.00||| +2451161|68098|8629|87217|1114475|2533|31791|4|256|1968|10|40.91|65.45|2.61|0.00|26.10|409.10|654.50|0.00|0.00|26.10|26.10|-383.00| +2451161|68098|5365|87217|1114475|2533|31791|4|80|1968|9|88.67|120.59|67.53|0.00|607.77|798.03|1085.31|24.31|0.00|607.77|632.08|-190.26| +2451161|68098|9028|87217|1114475|2533|31791|4|221|1968|41|1.27|2.10|1.59|0.00|65.19|52.07|86.10|2.60|0.00|65.19|67.79|13.12| +2452210|43718|3939|68547|1770820|293|42966|8|6|1969|29|3.09|5.74|5.28|0.00|153.12|89.61|166.46|12.24|0.00|153.12|165.36|63.51| +2452210|43718|14151|68547|1770820|293|42966|8|79|1969|60|9.72|10.59|0.63|0.00|37.80|583.20|635.40|0.37|0.00|37.80|38.17|-545.40| +2452210|43718|14167|68547|1770820|293|42966|8|49|1969|91|33.46|64.91|62.31|0.00|5670.21|3044.86|5906.81|283.51|0.00|5670.21|5953.72|2625.35| +2452210|43718|9453|68547|1770820|293|42966|8|140|1969|35|85.54|92.38|42.49|0.00|1487.15|2993.90|3233.30|104.10|0.00|1487.15|1591.25|-1506.75| +2452210|43718|3715|68547|1770820|293|42966|8|47|1969|99|37.53|61.54|35.69|0.00|3533.31|3715.47|6092.46|317.99|0.00|3533.31|3851.30|-182.16| +2452210|43718|2535|68547|1770820|293|42966|8|186|1969|85|1.35|2.38|1.88|67.11|159.80|114.75|202.30|6.48|67.11|92.69|99.17|-22.06| +2452210|43718|15965|68547|1770820|293|42966|8|206|1969|68|16.87|20.58|16.87|0.00|1147.16|1147.16|1399.44|0.00|0.00|1147.16|1147.16|0.00| +2452210|43718|15765|68547|1770820|293|42966|8|229|1969|46|74.50|133.35|93.34|3735.46|4293.64|3427.00|6134.10|44.65|3735.46|558.18|602.83|-2868.82| +2452210|43718|9639|68547|1770820|293|42966|8|128|1969|1|48.64|93.87|47.87|0.00|47.87|48.64|93.87|3.82|0.00|47.87|51.69|-0.77| +2452210|43718|7717|68547||293|42966|8|266|1969|58|60.62|||0.00|1186.10|3515.96||47.44|0.00||1233.54|| +2452210|43718|73|68547|1770820|293|42966|8|12|1969|54|50.76|86.79|12.15|0.00|656.10|2741.04|4686.66|13.12|0.00|656.10|669.22|-2084.94| +2452210|43718|8271|68547|1770820|293|42966|8|68|1969|17|79.68|100.39|39.15|0.00|665.55|1354.56|1706.63|39.93|0.00|665.55|705.48|-689.01| +2451465|36204|4298|12281|1324673|1065|29457|10|154|1970|42|83.40|102.58|76.93|0.00|3231.06|3502.80|4308.36|129.24|0.00|3231.06|3360.30|-271.74| +2451465|36204|2911|12281|1324673|1065|29457|10|262|1970|91|26.41|39.08|0.39|0.00|35.49|2403.31|3556.28|2.48|0.00|35.49|37.97|-2367.82| +2451465|36204|9580|||1065||10||1970|||||0.00|||5093.92|201.69|0.00|3361.64||450.74| +2451465|36204|15812|12281|1324673|1065|29457|10|79|1970|11|66.21|96.00|16.32|0.00|179.52|728.31|1056.00|10.77|0.00|179.52|190.29|-548.79| +2451465|36204|13525|12281|1324673|1065|29457|10|278|1970|89|44.77|68.05|23.13|0.00|2058.57|3984.53|6056.45|41.17|0.00|2058.57|2099.74|-1925.96| +||10738|12281||||10||1970||||||870.44||6222.80|||870.44|948.77|-4315.54| +2451465|36204|15554|12281|1324673|1065|29457|10|24|1970|76|18.02|18.02|5.40|369.36|410.40|1369.52|1369.52|2.46|369.36|41.04|43.50|-1328.48| +2451465|36204|17342|12281|1324673|1065|29457|10|179|1970|75|54.00|88.02|16.72|0.00|1254.00|4050.00|6601.50|100.32|0.00|1254.00|1354.32|-2796.00| +|36204|12037||||29457|||1970|||20.12|16.29|||||25.41||1270.62||| +2451465|36204|16783|12281|1324673|1065|29457|10|59|1970|66|94.18|120.55|12.05|0.00|795.30|6215.88|7956.30|47.71|0.00|795.30|843.01|-5420.58| +2451465|36204|10172|12281|1324673|1065|29457|10|193|1970|56|49.09|59.88|0.59|0.00|33.04|2749.04|3353.28|0.66|0.00|33.04|33.70|-2716.00| +2451465|36204|391|12281|1324673|1065|29457|10|199|1970|64|91.47|115.25|36.88|0.00|2360.32|5854.08|7376.00|165.22|0.00|2360.32|2525.54|-3493.76| +2451465|36204|4640|12281|1324673|1065|29457|10|22|1970|55|11.75|19.97|16.37|0.00|900.35|646.25|1098.35|72.02|0.00|900.35|972.37|254.10| +2451465|36204|17932|12281|1324673|1065|29457|10|273|1970|59|36.80|48.94|27.89|0.00|1645.51|2171.20|2887.46|16.45|0.00|1645.51|1661.96|-525.69| +2451291|39341|9511|60807|959654|4101|13748|10|192|1971|4|15.91|22.11|13.70|37.81|54.80|63.64|88.44|0.67|37.81|16.99|17.66|-46.65| +2451291|39341|16579|60807|959654|4101|13748|10|215|1971|8|5.31|8.44|7.08|0.00|56.64|42.48|67.52|1.13|0.00|56.64|57.77|14.16| +2451291|39341|6715|60807|959654|4101|13748|10|102|1971|67|34.47|64.45|3.22|0.00|215.74|2309.49|4318.15|4.31|0.00|215.74|220.05|-2093.75| +2451291|39341|6175||959654||||125|1971|2|89.12|144.37||||178.24||||||101.82| +2451291|39341|13786|60807|959654|4101|13748|10|183|1971|6|3.13|6.07|3.45|19.25|20.70|18.78|36.42|0.00|19.25|1.45|1.45|-17.33| +2451291||7946||959654||13748|10|205|1971||||118.52|0.00|2370.40|1856.60|2469.20|165.92|0.00|||513.80| +2451291|39341|3436|60807|959654|4101|13748|10|244|1971|86|30.91|45.74|22.41|0.00|1927.26|2658.26|3933.64|19.27|0.00|1927.26|1946.53|-731.00| +2451291|39341|16076|60807|959654|4101|13748|10|115|1971|3|83.38|165.92|132.73|0.00|398.19|250.14|497.76|3.98|0.00|398.19|402.17|148.05| +2451291|39341|5833|60807|959654|4101|13748|10|134|1971|8|43.15|53.50|28.35|0.00|226.80|345.20|428.00|6.80|0.00|226.80|233.60|-118.40| +2451291|39341|17188|60807|959654|4101|13748|10|278|1971|70|42.54|56.57|38.46|0.00|2692.20|2977.80|3959.90|107.68|0.00|2692.20|2799.88|-285.60| +2451291|39341|11383|60807|959654|4101|13748|10|128|1971|17|34.35|52.55|27.32|260.08|464.44|583.95|893.35|0.00|260.08|204.36|204.36|-379.59| +2451291|39341|9877|60807|959654|4101|13748|10|57|1971|93|92.57|180.51|176.89|0.00|16450.77|8609.01|16787.43|1151.55|0.00|16450.77|17602.32|7841.76| +2451291|39341|5210|60807|959654|4101|13748|10|43|1971|72|71.57|98.05|38.23|0.00|2752.56|5153.04|7059.60|27.52|0.00|2752.56|2780.08|-2400.48| +2451291|39341|14059|60807|959654|4101|13748|10|235|1971|64|29.82|41.74|22.53|0.00|1441.92|1908.48|2671.36|72.09|0.00|1441.92|1514.01|-466.56| +2452290|40298|16411|74579|120805|2584|22010|8|154|1972|49|46.53|72.12|54.09|0.00|2650.41|2279.97|3533.88|79.51|0.00|2650.41|2729.92|370.44| +2452290|40298|13626|74579|120805|2584|22010|8|23|1972|49|22.61|33.68|1.34|0.00|65.66|1107.89|1650.32|1.31|0.00|65.66|66.97|-1042.23| +2452290|40298|12810|74579|120805|2584|22010|8|58|1972|12|49.60|79.85|71.86|0.00|862.32|595.20|958.20|43.11|0.00|862.32|905.43|267.12| +2452290|40298|3415|74579|120805|2584|22010|8|141|1972|14|70.48|96.55|6.75|0.00|94.50|986.72|1351.70|4.72|0.00|94.50|99.22|-892.22| +2452290|40298|6223|74579|120805|2584|22010|8|30|1972|100|59.40|98.01|0.98|0.00|98.00|5940.00|9801.00|5.88|0.00|98.00|103.88|-5842.00| +2452290|40298|4419|74579|120805|2584|22010|8|137|1972|58|62.26|114.55|1.14|17.85|66.12|3611.08|6643.90|2.41|17.85|48.27|50.68|-3562.81| +2452290|40298|108|74579|120805|2584|22010|8|261|1972|93|93.04|117.23|97.30|0.00|9048.90|8652.72|10902.39|0.00|0.00|9048.90|9048.90|396.18| +2452290|40298|277|74579|120805|2584|22010|8|32|1972|45|43.09|45.67|38.36|0.00|1726.20|1939.05|2055.15|34.52|0.00|1726.20|1760.72|-212.85| +2452290|40298|4344|74579|120805|2584|22010|8|252|1972|93|2.49|4.38|0.00|0.00|0.00|231.57|407.34|0.00|0.00|0.00|0.00|-231.57| +2452290|40298|621|74579|120805|2584|22010|8|259|1972|6|98.10|109.87|5.49|0.00|32.94|588.60|659.22|1.64|0.00|32.94|34.58|-555.66| +2451030|50487|428|34475|657270|6112|20154|1|285|1973|2|95.11|154.07|12.32|0.00|24.64|190.22|308.14|2.21|0.00|24.64|26.85|-165.58| +2451030|50487|7795|34475|657270|6112|20154|1|76|1973|33|82.46|108.84|95.77|0.00|3160.41|2721.18|3591.72|126.41|0.00|3160.41|3286.82|439.23| +2451030|50487|16192|34475|657270|6112|20154|1|37|1973|27|93.44|180.33|86.55|514.10|2336.85|2522.88|4868.91|109.36|514.10|1822.75|1932.11|-700.13| +|50487|16606||657270||||68|1973|77|49.44|83.05|51.49|0.00||3806.88|6394.85|79.29|0.00||4044.02|| +2451030|50487|11404|34475|657270|6112|20154|1|118|1973|20|94.82|189.64|134.64|0.00|2692.80|1896.40|3792.80|80.78|0.00|2692.80|2773.58|796.40| +2451030|50487|12416|34475|657270|6112|20154|1|148|1973|22|66.09|128.21|32.05|662.79|705.10|1453.98|2820.62|1.26|662.79|42.31|43.57|-1411.67| +2451030|50487|8041|34475|657270|6112|20154|1|279|1973|49|16.68|19.84|13.68|0.00|670.32|817.32|972.16|46.92|0.00|670.32|717.24|-147.00| +2451030|50487|14764|34475|657270|6112|20154|1|277|1973|24|46.55|82.39|45.31|0.00|1087.44|1117.20|1977.36|54.37|0.00|1087.44|1141.81|-29.76| +2451030|50487|6937|34475|657270|6112|20154|1|138|1973|1|36.08|39.32|34.60|21.10|34.60|36.08|39.32|1.21|21.10|13.50|14.71|-22.58| +2451848|32035|17651|21060|1549223|1386|27149|8|203|1974|88|49.12|85.96|0.85|59.84|74.80|4322.56|7564.48|1.19|59.84|14.96|16.15|-4307.60| +2451848|32035|2813|21060|1549223|1386|27149|8|82|1974|90|50.61|65.28|13.05|0.00|1174.50|4554.90|5875.20|70.47|0.00|1174.50|1244.97|-3380.40| +2451848|32035|17378|21060||1386|27149|||1974||45.30|47.11|40.98|||2899.20|3015.04|||||| +|32035|10087||1549223|1386||8||1974||49.77|86.59||||298.62|519.54|||232.75|237.40|-65.87| +||5888||1549223||27149|8||1974|57||161.56|||8103.69|4796.55|9208.92|||8103.69||3307.14| +2451848|32035|16627|21060|1549223|1386|27149|8|181|1974|46|47.42|55.95|25.17|0.00|1157.82|2181.32|2573.70|81.04|0.00|1157.82|1238.86|-1023.50| +2451848|32035|15455|21060|1549223|1386|27149|8|188|1974|39|66.13|83.32|63.32|0.00|2469.48|2579.07|3249.48|172.86|0.00|2469.48|2642.34|-109.59| +2451848|32035|12206|21060|1549223|1386|27149|8|94|1974|14|42.64|67.37|39.74|0.00|556.36|596.96|943.18|50.07|0.00|556.36|606.43|-40.60| +2451848|32035|16994|21060|1549223|1386|27149|8|276|1974|86|60.74|118.44|72.24|0.00|6212.64|5223.64|10185.84|62.12|0.00|6212.64|6274.76|989.00| +2451848|32035|14024|21060|1549223|1386|27149|8|203|1974|87|95.41|165.05|112.23|0.00|9764.01|8300.67|14359.35|292.92|0.00|9764.01|10056.93|1463.34| +2451848|32035|14353|21060|||27149|||1974|61|||7.23||441.03|4967.23|7351.11|18.74|||393.62|-4592.35| +2451848|32035|9199|21060|1549223|1386|27149|8|173|1974|98|94.82|121.36|31.55|0.00|3091.90|9292.36|11893.28|247.35|0.00|3091.90|3339.25|-6200.46| +2451848|32035|12467|21060|1549223|1386|27149|8|207|1974|79|89.69|128.25|97.47|0.00|7700.13|7085.51|10131.75|693.01|0.00|7700.13|8393.14|614.62| +2451848|32035|13153|21060|1549223|1386|27149|8|115|1974|46|16.81|25.71|22.88|0.00|1052.48|773.26|1182.66|94.72|0.00|1052.48|1147.20|279.22| +2451848|32035|2819|21060|1549223|1386|27149|8|69|1974|46|67.11|95.29|19.05|0.00|876.30|3087.06|4383.34|78.86|0.00|876.30|955.16|-2210.76| +2451848|32035|11258|21060|1549223|1386|27149|8|88|1974|81|2.27|4.24|1.78|0.00|144.18|183.87|343.44|10.09|0.00|144.18|154.27|-39.69| +2451929|58012|12381|97710|1723835|1160|11685|10|136|1975|12|93.41|95.27|66.68|208.04|800.16|1120.92|1143.24|17.76|208.04|592.12|609.88|-528.80| +2451929|58012|10745|97710|1723835|1160|11685|10|74|1975|49|94.13|162.84|19.54|708.52|957.46|4612.37|7979.16|0.00|708.52|248.94|248.94|-4363.43| +2451929|58012|3059|97710|1723835|1160|11685|10|156|1975|2|40.78|49.34|30.09|0.00|60.18|81.56|98.68|2.40|0.00|60.18|62.58|-21.38| +|58012|13203|97710|1723835|1160||10|190|1975||25.89|||1995.41|2168.93|2304.21||13.88|1995.41|173.52|187.40|-2130.69| +2451929|58012|87|97710|1723835|1160|11685|10|32|1975|94|50.19|66.75|22.69|0.00|2132.86|4717.86|6274.50|170.62|0.00|2132.86|2303.48|-2585.00| +||9217|97710|||11685|||1975||74.40|||0.00|534.14||||0.00|534.14||-730.66| +2451929|58012|16963|97710|1723835|1160|11685|10|202|1975|73|26.28|39.68|23.01|1511.75|1679.73|1918.44|2896.64|0.00|1511.75|167.98|167.98|-1750.46| +2451929|58012|14109|97710|1723835|1160|11685|10|284|1975|59|42.36|80.06|52.83|0.00|3116.97|2499.24|4723.54|218.18|0.00|3116.97|3335.15|617.73| +2451898|69662|13073|68042|1809621|7200|21532|1|130|1976|76|40.28|73.71|6.63|0.00|503.88|3061.28|5601.96|40.31|0.00|503.88|544.19|-2557.40| +2451898|69662|4466|68042|1809621|7200|21532|1|5|1976|100|29.33|41.94|40.26|0.00|4026.00|2933.00|4194.00|362.34|0.00|4026.00|4388.34|1093.00| +2451898|69662|7679|68042|1809621|7200|21532|1|219|1976|46|26.00|31.20|3.12|93.28|143.52|1196.00|1435.20|2.00|93.28|50.24|52.24|-1145.76| +2451898|69662|14279|68042|1809621|7200|21532|1|96|1976|46|34.19|48.54|18.93|0.00|870.78|1572.74|2232.84|69.66|0.00|870.78|940.44|-701.96| +2451898||3470||||21532||97|1976||71.32||0.00||||3669.40|0.00||||-2496.20| +2451898|69662|3392|68042|1809621|7200|21532|1|248|1976|83|74.34|75.82|3.03|0.00|251.49|6170.22|6293.06|15.08|0.00|251.49|266.57|-5918.73| +2451898|69662|9056|68042|1809621|7200|21532|1|169|1976|94|44.29|66.87|9.36|0.00|879.84|4163.26|6285.78|70.38|0.00|879.84|950.22|-3283.42| +2451898|69662|10994||1809621|||1||1976|48||93.28|0.00|0.00||4347.36|4477.44|0.00|0.00|0.00||-4347.36| +2451898|69662|6997|68042|1809621|7200|21532|1|230|1976|24|44.37|82.52|52.81|0.00|1267.44|1064.88|1980.48|38.02|0.00|1267.44|1305.46|202.56| +2452375|34085|14755|48754|134900|2999|39545|8|97|1977|50|44.94|54.82|0.00|0.00|0.00|2247.00|2741.00|0.00|0.00|0.00|0.00|-2247.00| +2452375|34085|10477|48754|134900|2999|39545|8|248|1977|54|48.90|55.25|35.36|0.00|1909.44|2640.60|2983.50|57.28|0.00|1909.44|1966.72|-731.16| +2452375|34085|15889|48754|134900|2999|39545|8|203|1977|42|86.15|133.53|102.81|0.00|4318.02|3618.30|5608.26|86.36|0.00|4318.02|4404.38|699.72| +2452375|34085|14016|48754|134900|2999|39545|8|50|1977|79|77.83|118.30|16.56|0.00|1308.24|6148.57|9345.70|0.00|0.00|1308.24|1308.24|-4840.33| +2452375|34085|8655|48754|134900|2999|39545|8|262|1977|62|52.61|74.18|23.73|735.63|1471.26|3261.82|4599.16|14.71|735.63|735.63|750.34|-2526.19| +2452375|34085|805|48754|134900|2999|39545|8|116|1977|45|47.44|72.58|72.58|0.00|3266.10|2134.80|3266.10|0.00|0.00|3266.10|3266.10|1131.30| +2452375|34085|711|48754|134900|2999|39545|8|294|1977|18|75.44|90.52|14.48|125.10|260.64|1357.92|1629.36|4.06|125.10|135.54|139.60|-1222.38| +2452375|34085|5496|48754|134900|2999|39545|8|70|1977|28|1.80|3.22|0.38|0.00|10.64|50.40|90.16|0.85|0.00|10.64|11.49|-39.76| +2452375|34085|17352|48754|134900|2999|39545|8|191|1977|66|23.33|37.09|28.55|0.00|1884.30|1539.78|2447.94|113.05|0.00|1884.30|1997.35|344.52| +2451116|71776|9247|23985||||||1978||||30.41|0.00||||270.95|0.00|||| +2451116|71776|12062|23985|914065|2898|21778|8|287|1978|88|67.01|77.73|13.21|302.24|1162.48|5896.88|6840.24|68.81|302.24|860.24|929.05|-5036.64| +2451116|71776|10256|23985|914065|2898|21778|8|215|1978|93|62.69|112.21|105.47|0.00|9808.71|5830.17|10435.53|490.43|0.00|9808.71|10299.14|3978.54| +2451116|71776|14888|23985|914065|2898|21778|8|40|1978|2|60.78|113.65|46.59|0.00|93.18|121.56|227.30|7.45|0.00|93.18|100.63|-28.38| +2451116|71776|7165|23985|914065|2898|21778|8|5|1978|80|53.72|97.77|60.61|0.00|4848.80|4297.60|7821.60|48.48|0.00|4848.80|4897.28|551.20| +2451116|71776|17810|23985|914065|2898|21778|8|278|1978|30|41.85|61.10|43.99|0.00|1319.70|1255.50|1833.00|0.00|0.00|1319.70|1319.70|64.20| +2451116|71776|13018|23985|914065|2898|21778|8|37|1978|27|90.39|111.17|24.45|0.00|660.15|2440.53|3001.59|6.60|0.00|660.15|666.75|-1780.38| +|71776|15502|23985|||21778||64|1978|60|95.83|||0.00|1503.60|5749.80||30.07|0.00|||-4246.20| +2451116|71776|2042|23985|914065|2898|21778|8|31|1978|95|87.81|92.20|14.75|0.00|1401.25|8341.95|8759.00|126.11|0.00|1401.25|1527.36|-6940.70| +2451116|71776|1372|23985|914065|2898|21778|8|139|1978|98|87.03|139.24|4.17|0.00|408.66|8528.94|13645.52|4.08|0.00|408.66|412.74|-8120.28| +2451116|71776|13987|23985|914065|2898|21778|8|68|1978|86|51.53|80.38|77.96|0.00|6704.56|4431.58|6912.68|402.27|0.00|6704.56|7106.83|2272.98| +2451116|71776|16831|23985|914065|2898|21778|8|52|1978|87|10.95|21.35|5.55|0.00|482.85|952.65|1857.45|24.14|0.00|482.85|506.99|-469.80| +2451116|71776|8636|23985|914065|2898|21778|8|229|1978|1|87.98|131.97|108.21|0.00|108.21|87.98|131.97|8.65|0.00|108.21|116.86|20.23| +2451116|71776|1981|23985|914065|2898|21778|8|147|1978|40|98.64|135.13|117.56|0.00|4702.40|3945.60|5405.20|282.14|0.00|4702.40|4984.54|756.80| +2452510||8395||1335517|||1||1979|98||45.28||0.00|1862.98||4437.44|0.00|0.00|||| +2452510|75155|9969|48017|1335517|7017|45317|1|87|1979|34|24.45|33.00|4.29|0.00|145.86|831.30|1122.00|0.00|0.00|145.86|145.86|-685.44| +2452510|75155|4386|48017|1335517|7017|45317|1|100|1979|8|58.91|64.21|61.64|0.00|493.12|471.28|513.68|0.00|0.00|493.12|493.12|21.84| +2452510|75155|6579|48017|1335517|7017|45317|1|223|1979|32|84.65|161.68|121.26|0.00|3880.32|2708.80|5173.76|349.22|0.00|3880.32|4229.54|1171.52| +2452510|75155|11724|48017|1335517|7017|45317|1|220|1979|100|59.45|70.74|3.53|0.00|353.00|5945.00|7074.00|7.06|0.00|353.00|360.06|-5592.00| +2452510|75155|7789|48017|1335517||45317|1|42|1979|||||39.31|218.40||2184.39|7.16|39.31|||| +2452510|75155|1881|48017|1335517|7017|45317|1|118|1979|46|9.51|16.26|14.63|181.70|672.98|437.46|747.96|44.21|181.70|491.28|535.49|53.82| +2452510|75155|1644|48017|1335517|7017|45317|1|152|1979|65|98.84|105.75|16.92|527.90|1099.80|6424.60|6873.75|45.75|527.90|571.90|617.65|-5852.70| +2452510|75155|193|48017|1335517|7017|45317|1|91|1979|69|72.06|85.75|61.74|0.00|4260.06|4972.14|5916.75|0.00|0.00|4260.06|4260.06|-712.08| +2452510|75155|5817|48017|1335517|7017|45317|1|13|1979|34|42.54|84.22|42.95|0.00|1460.30|1446.36|2863.48|87.61|0.00|1460.30|1547.91|13.94| +2452510|75155|9243|48017|1335517|7017|45317|1|151|1979|98|5.08|9.65|2.12|0.00|207.76|497.84|945.70|14.54|0.00|207.76|222.30|-290.08| +2452510|75155|1929|48017|1335517|7017|45317|1|154|1979|53|5.72|7.03|6.32|0.00|334.96|303.16|372.59|13.39|0.00|334.96|348.35|31.80| +2452510|75155|12540|48017|1335517|7017|45317|1|63|1979|12|64.40|87.58|70.93|0.00|851.16|772.80|1050.96|25.53|0.00|851.16|876.69|78.36| +2452510|75155|5544|48017|1335517|7017|45317|1|159|1979|85|52.51|65.11|7.16|0.00|608.60|4463.35|5534.35|42.60|0.00|608.60|651.20|-3854.75| +2452510|75155|7968|48017|1335517|7017|45317|1|22|1979|58|95.73|112.00|19.04|1093.27|1104.32|5552.34|6496.00|0.77|1093.27|11.05|11.82|-5541.29| +2451231|34738|15512|22378|1150922|1066|45210|2|167|1980|59|44.98|64.77|17.48|0.00|1031.32|2653.82|3821.43|61.87|0.00|1031.32|1093.19|-1622.50| +2451231|34738|10420|22378|1150922|1066|45210|2|216|1980|76|33.07|43.32|33.78|0.00|2567.28|2513.32|3292.32|205.38|0.00|2567.28|2772.66|53.96| +2451231|34738|13099|22378|1150922|1066|45210|2|262|1980|87|36.97|69.87|1.39|0.00|120.93|3216.39|6078.69|6.04|0.00|120.93|126.97|-3095.46| +2451231|34738|10984|22378|1150922|1066|45210|2|216|1980|46|83.00|120.35|96.28|4207.43|4428.88|3818.00|5536.10|2.21|4207.43|221.45|223.66|-3596.55| +2451231|34738|7712|22378|1150922|1066|45210|2|92|1980|7|74.19|80.86|78.43|0.00|549.01|519.33|566.02|27.45|0.00|549.01|576.46|29.68| +2451231|34738|16858|22378|1150922|1066|45210|2|192|1980|87|95.42|103.05|24.73|0.00|2151.51|8301.54|8965.35|172.12|0.00|2151.51|2323.63|-6150.03| +2451231|34738|7855|22378|1150922|1066|45210|2|268|1980|36|82.08|95.21|67.59|0.00|2433.24|2954.88|3427.56|48.66|0.00|2433.24|2481.90|-521.64| +2451231|34738|9668|22378|1150922|1066|45210|2|252|1980|22|11.25|12.48|0.99|3.26|21.78|247.50|274.56|0.18|3.26|18.52|18.70|-228.98| +2451231|34738|14492|22378|1150922|1066|45210|2|85|1980|21|72.91|87.49|22.74|0.00|477.54|1531.11|1837.29|28.65|0.00|477.54|506.19|-1053.57| +2451231|34738|15994|22378|1150922|1066|45210|2|261|1980|72|1.77|3.25|0.87|0.00|62.64|127.44|234.00|3.13|0.00|62.64|65.77|-64.80| +2451231|34738|3931|22378|1150922|1066|45210|2|189|1980|32|75.03|100.54|55.29|0.00|1769.28|2400.96|3217.28|159.23|0.00|1769.28|1928.51|-631.68| +2451231|34738|7291|22378|1150922|1066|45210|2|32|1980|33|52.00|59.80|2.39|14.98|78.87|1716.00|1973.40|1.27|14.98|63.89|65.16|-1652.11| +2451231|34738|13850|22378|1150922|1066|45210|2|194|1980|65|98.62|157.79|140.43|0.00|9127.95|6410.30|10256.35|365.11|0.00|9127.95|9493.06|2717.65| +2451231|34738|109|22378|1150922|1066|45210|2|187|1980|65|64.88|92.12|88.43|0.00|5747.95|4217.20|5987.80|459.83|0.00|5747.95|6207.78|1530.75| +2451231|34738|2282|22378|1150922|1066|45210|2|263|1980|50|99.31|134.06|68.37|0.00|3418.50|4965.50|6703.00|136.74|0.00|3418.50|3555.24|-1547.00| +2451136|46397|1897|75403|140832|1014|14043|7|294|1981|70|61.81|73.55|26.47|759.68|1852.90|4326.70|5148.50|98.38|759.68|1093.22|1191.60|-3233.48| +||5408||||||199|1981|||||3.77|188.94|3688.56||1.85|3.77|185.17||| +2451136||7612||||14043|7||1981|||62.10|54.64|||||186.86||3114.48||1019.73| +2451136|46397|3526|75403|140832|1014|14043|7|156|1981|82|88.09|109.23|81.92|0.00|6717.44|7223.38|8956.86|335.87|0.00|6717.44|7053.31|-505.94| +2451136|46397|6992|75403|140832|1014|14043|7|69|1981|47|87.20|170.91|104.25|0.00|4899.75|4098.40|8032.77|391.98|0.00|4899.75|5291.73|801.35| +2451136|46397|7030|75403|140832|1014|14043|7|237|1981|7|41.65|44.14|23.83|0.00|166.81|291.55|308.98|13.34|0.00|166.81|180.15|-124.74| +2451136|46397|3478|75403|140832|1014|14043|7|267|1981|33|14.94|25.69|2.05|0.00|67.65|493.02|847.77|6.08|0.00|67.65|73.73|-425.37| +2451136|46397|15922|75403|140832|1014|14043|7|250|1981|41|62.81|74.74|14.94|581.91|612.54|2575.21|3064.34|0.91|581.91|30.63|31.54|-2544.58| +2451136|46397|11077|75403|140832|1014|14043|7|68|1981|33|55.83|96.58|47.32|765.16|1561.56|1842.39|3187.14|47.78|765.16|796.40|844.18|-1045.99| +2451136|46397|1678|75403|140832|1014|14043|7|120|1981|46|49.02|50.49|17.67|130.05|812.82|2254.92|2322.54|54.62|130.05|682.77|737.39|-1572.15| +2451136|46397|8038|75403|140832|1014|14043|7|114|1981|28|56.32|100.24|39.09|0.00|1094.52|1576.96|2806.72|0.00|0.00|1094.52|1094.52|-482.44| +2451136|46397|13855|75403|140832|1014|14043|7|238|1981|82|93.25|123.09|109.55|0.00|8983.10|7646.50|10093.38|808.47|0.00|8983.10|9791.57|1336.60| +2451136|46397|13324|75403|140832|1014|14043|7|85|1981|14|55.54|74.42|70.69|940.17|989.66|777.56|1041.88|1.48|940.17|49.49|50.97|-728.07| +2451136|46397|2758|75403|140832|1014|14043|7|144|1981|48|19.91|22.29|22.06|0.00|1058.88|955.68|1069.92|52.94|0.00|1058.88|1111.82|103.20| +2451181|51021|11545|21337|511430|6323|26063|7|116|1982|87|20.49|21.51|0.43|0.00|37.41|1782.63|1871.37|3.36|0.00|37.41|40.77|-1745.22| +2451181|51021|4184|21337|511430|6323|26063|7|209|1982|44|35.09|53.68|53.68|0.00|2361.92|1543.96|2361.92|70.85|0.00|2361.92|2432.77|817.96| +2451181|51021|7714|21337|511430|6323|26063|7|103|1982|46|23.12|29.59|21.30|0.00|979.80|1063.52|1361.14|29.39|0.00|979.80|1009.19|-83.72| +2451181|51021|12052|21337|511430|6323|26063|7|94|1982|73|45.13|68.59|41.15|0.00|3003.95|3294.49|5007.07|30.03|0.00|3003.95|3033.98|-290.54| +2451181|51021|8348|21337|511430|6323|26063|7|95|1982|69|91.96|109.43|41.58|0.00|2869.02|6345.24|7550.67|0.00|0.00|2869.02|2869.02|-3476.22| +2451181|51021|3763|21337|511430|6323|26063|7|73|1982|88|97.70|186.60|130.62|0.00|11494.56|8597.60|16420.80|0.00|0.00|11494.56|11494.56|2896.96| +2451181|51021|10681|21337|511430|6323|26063|7|218|1982|34|42.36|48.29|8.69|0.00|295.46|1440.24|1641.86|5.90|0.00|295.46|301.36|-1144.78| +2451181|51021|2485|21337|511430|6323|26063|7|170|1982|43|83.88|106.52|68.17|0.00|2931.31|3606.84|4580.36|263.81|0.00|2931.31|3195.12|-675.53| +2451181|51021|16148|21337|511430|6323|26063|7|221|1982|58|80.81|88.08|32.58|0.00|1889.64|4686.98|5108.64|18.89|0.00|1889.64|1908.53|-2797.34| +2451181|51021|15895|21337|511430|6323|26063|7|272|1982|90|9.69|19.08|5.34|0.00|480.60|872.10|1717.20|24.03|0.00|480.60|504.63|-391.50| +2452356|62493|9877|39178|1377891|251|30964|2|89|1983|8|55.55|108.87|71.85|0.00|574.80|444.40|870.96|45.98|0.00|574.80|620.78|130.40| +2452356|62493|5211|39178|1377891|251|30964|2|117|1983|97|25.51|33.41|24.05|0.00|2332.85|2474.47|3240.77|23.32|0.00|2332.85|2356.17|-141.62| +|62493|14059|39178|1377891|251||2||1983|50|||58.50|0.00||3950.00|6359.50||0.00|2925.00||-1025.00| +2452356|62493|10512|39178|1377891|251|30964|2|276|1983|28|95.25|99.06|30.70|0.00|859.60|2667.00|2773.68|42.98|0.00|859.60|902.58|-1807.40| +2452356|62493|7500|39178|1377891|251|30964|2|244|1983|28|67.85|99.73|6.98|0.00|195.44|1899.80|2792.44|9.77|0.00|195.44|205.21|-1704.36| +2452356|62493|8473|39178|1377891|251|30964|2|270|1983|45|6.12|11.26|2.13|4.79|95.85|275.40|506.70|1.82|4.79|91.06|92.88|-184.34| +2452356|62493|15018|39178|1377891|251|30964|2|289|1983|40|27.14|43.69|41.94|0.00|1677.60|1085.60|1747.60|83.88|0.00|1677.60|1761.48|592.00| +2452356|62493|16513|39178|1377891|251|30964|2|41|1983|94|31.03|32.58|11.07|0.00|1040.58|2916.82|3062.52|41.62|0.00|1040.58|1082.20|-1876.24| +2452356||12819||||30964|2|232|1983|11||60.55||||374.22|666.05|0.00||6.80||| +||6337|39178|1377891|251||||1983|30|||||1097.10|1197.60||||1097.10||-100.50| +|62493|12477||1377891|251|30964||142|1983|83|||7.40||614.20|5821.62|10245.52|||||| +2452356|62493|10107|39178|1377891|251|30964|2|54|1983|16|23.98|43.88|32.03|379.23|512.48|383.68|702.08|0.00|379.23|133.25|133.25|-250.43| +2452356|62493|10344|39178|1377891|251|30964|2|170|1983|56|8.74|15.29|11.92|0.00|667.52|489.44|856.24|46.72|0.00|667.52|714.24|178.08| +|62493|15421|39178||251|30964||36|1983||||||533.54||5340.58|37.34|||570.88|| +2452356|62493|11067|39178|1377891|251|30964|2|197|1983|41|26.62|38.06|24.73|0.00|1013.93|1091.42|1560.46|81.11|0.00|1013.93|1095.04|-77.49| +2451499|63070|8008|5123|1626394|6065|47736|7|273|1984|3|67.91|74.70|73.95|37.71|221.85|203.73|224.10|11.04|37.71|184.14|195.18|-19.59| +2451499|63070|8335|5123|1626394|6065|47736|7|149|1984|81|99.15|141.78|79.39|0.00|6430.59|8031.15|11484.18|514.44|0.00|6430.59|6945.03|-1600.56| +2451499|63070|9397|5123|1626394|6065|47736|7|26|1984|98|78.39|155.21|142.79|0.00|13993.42|7682.22|15210.58|559.73|0.00|13993.42|14553.15|6311.20| +2451499|63070|11914|5123|1626394|6065|47736|7|24|1984|27|54.54|75.26|9.03|0.00|243.81|1472.58|2032.02|4.87|0.00|243.81|248.68|-1228.77| +2451499||10651||1626394|||7|202|1984|33|50.06|50.56|33.36|0.00|1100.88||||0.00||1100.88|-551.10| +2451499|63070|3133|5123|1626394|6065|47736|7|71|1984|69|96.78|158.71|0.00|0.00|0.00|6677.82|10950.99|0.00|0.00|0.00|0.00|-6677.82| +2451499|63070|7087|5123|1626394|6065|47736|7|3|1984|56|20.68|34.32|3.08|0.00|172.48|1158.08|1921.92|1.72|0.00|172.48|174.20|-985.60| +2451499|63070|9346|5123|1626394|6065|47736|7|17|1984|77|3.79|7.35|2.05|0.00|157.85|291.83|565.95|9.47|0.00|157.85|167.32|-133.98| +2451499|63070|6772|5123|1626394|6065|47736|7|195|1984|10|92.67|98.23|61.88|0.00|618.80|926.70|982.30|24.75|0.00|618.80|643.55|-307.90| +2451499|63070|9094|5123|1626394|6065|47736|7|82|1984|82|67.78|90.82|57.21|0.00|4691.22|5557.96|7447.24|140.73|0.00|4691.22|4831.95|-866.74| +2451499|63070|1094|5123|1626394|6065|47736|7|295|1984|95|12.44|14.43|5.48|0.00|520.60|1181.80|1370.85|5.20|0.00|520.60|525.80|-661.20| +2451499|63070|3376|5123|1626394|6065|47736|7|152|1984|16|41.12|74.83|73.33|0.00|1173.28|657.92|1197.28|70.39|0.00|1173.28|1243.67|515.36| +2451499|63070|4882|5123|1626394|6065|47736|7|206|1984|40|71.02|94.45|59.50|0.00|2380.00|2840.80|3778.00|71.40|0.00|2380.00|2451.40|-460.80| +2451499|63070|8662|5123|1626394|6065|47736|7|102|1984|19|38.96|64.28|46.92|0.00|891.48|740.24|1221.32|62.40|0.00|891.48|953.88|151.24| +2451499|63070|17539|5123|1626394|6065|47736|7|273|1984|2|34.59|42.54|26.37|0.00|52.74|69.18|85.08|2.10|0.00|52.74|54.84|-16.44| +2452261|63353|10471|50012|1482700|6235|2923|2|73|1985|59|13.03|15.24|10.05|0.00|592.95|768.77|899.16|11.85|0.00|592.95|604.80|-175.82| +|63353|6059||1482700||2923||226|1985|82|||30.90|||||12.66||||-2913.46| +2452261|63353|12551|50012|1482700|6235|2923|2|39|1985|3|33.24|48.19|7.22|0.00|21.66|99.72|144.57|0.86|0.00|21.66|22.52|-78.06| +2452261|63353|2393|50012|1482700|6235|2923|2|172|1985|57|51.16|94.64|83.28|3702.62|4746.96|2916.12|5394.48|83.54|3702.62|1044.34|1127.88|-1871.78| +2452261|63353|15003|50012|1482700|6235|2923|2|252|1985|94|7.82|14.23|10.95|0.00|1029.30|735.08|1337.62|10.29|0.00|1029.30|1039.59|294.22| +2452261|63353|6135|50012|1482700|6235|2923|2|285|1985|95|56.40|62.60|33.80|0.00|3211.00|5358.00|5947.00|192.66|0.00|3211.00|3403.66|-2147.00| +2452261|63353|3613|50012|1482700|6235|2923|2|256|1985|37|92.29|178.11|178.11|0.00|6590.07|3414.73|6590.07|0.00|0.00|6590.07|6590.07|3175.34| +2452261|63353|9741|50012|1482700|6235|2923|2|6|1985|85|26.55|39.29|27.89|0.00|2370.65|2256.75|3339.65|94.82|0.00|2370.65|2465.47|113.90| +2452261|63353|619|50012|1482700|6235|2923|2|179|1985|16|36.93|62.78|52.10|0.00|833.60|590.88|1004.48|66.68|0.00|833.60|900.28|242.72| +2452261|63353|1437|50012|1482700|6235|2923|2|283|1985|45|22.24|28.02|12.04|0.00|541.80|1000.80|1260.90|5.41|0.00|541.80|547.21|-459.00| +2452261||8463|50012|1482700||2923||188|1985|||58.90|30.03|0.00|||2297.10|35.13|0.00|1171.17||-84.24| +2452261|63353|2603|50012|1482700|6235|2923|2|106|1985|49|69.15|130.00|101.40|0.00|4968.60|3388.35|6370.00|49.68|0.00|4968.60|5018.28|1580.25| +2452261|63353|7901|50012|1482700|6235|2923|2|48|1985|57|98.63|175.56|156.24|0.00|8905.68|5621.91|10006.92|267.17|0.00|8905.68|9172.85|3283.77| +2452261|63353|10063|50012|1482700|6235|2923|2|259|1985|84|10.94|12.58|0.75|0.00|63.00|918.96|1056.72|0.00|0.00|63.00|63.00|-855.96| +2452261|63353|1841|50012|1482700|6235|2923|2|26|1985|6|31.21|53.68|28.98|0.00|173.88|187.26|322.08|15.64|0.00|173.88|189.52|-13.38| +2452261|63353|4179|50012|1482700|6235|2923|2|115|1985|1|59.87|68.25|38.22|0.00|38.22|59.87|68.25|2.67|0.00|38.22|40.89|-21.65| +2452179|33922|9707|52924|276427|1083|19386|1|299|1986|72|34.10|56.94|41.56|0.00|2992.32|2455.20|4099.68|29.92|0.00|2992.32|3022.24|537.12| +2452179|33922|6935|52924|276427|1083|19386|1|164|1986|75|24.50|26.70|14.95|0.00|1121.25|1837.50|2002.50|44.85|0.00|1121.25|1166.10|-716.25| +2452179|33922|1723|52924|276427|1083|19386|1|49|1986|85|48.52|66.47|19.94|0.00|1694.90|4124.20|5649.95|50.84|0.00|1694.90|1745.74|-2429.30| +2452179|33922|12947|52924|276427|1083|19386|1|171|1986|46|93.52|175.81|158.22|800.59|7278.12|4301.92|8087.26|194.32|800.59|6477.53|6671.85|2175.61| +2452179|33922|9111|52924|276427|1083|19386|1|273|1986|59|78.60|121.83|0.00|0.00|0.00|4637.40|7187.97|0.00|0.00|0.00|0.00|-4637.40| +2452179|33922|5539|52924|276427|1083|19386|1|106|1986|48|1.29|1.54|0.92|0.00|44.16|61.92|73.92|1.32|0.00|44.16|45.48|-17.76| +2452179|33922|11981|52924|276427|1083|19386|1|123|1986|39|88.53|176.17|81.03|0.00|3160.17|3452.67|6870.63|31.60|0.00|3160.17|3191.77|-292.50| +2452179|33922|13475|52924|276427|1083|19386|1|252|1986|58|26.44|43.62|12.21|0.00|708.18|1533.52|2529.96|0.00|0.00|708.18|708.18|-825.34| +2452179|33922|8853|52924|276427|1083|19386|1|153|1986|62|44.83|68.14|18.39|0.00|1140.18|2779.46|4224.68|0.00|0.00|1140.18|1140.18|-1639.28| +2452179|33922|5423|52924|276427|1083|19386|1|189|1986|88|63.15|68.20|23.87|1932.51|2100.56|5557.20|6001.60|0.00|1932.51|168.05|168.05|-5389.15| +2452179|33922|1361|52924|276427|1083|19386|1|218|1986|59|31.91|44.35|8.42|0.00|496.78|1882.69|2616.65|14.90|0.00|496.78|511.68|-1385.91| +2452179|33922|11117|52924|276427|1083|19386|1|188|1986|98|85.01|126.66|120.32|0.00|11791.36|8330.98|12412.68|0.00|0.00|11791.36|11791.36|3460.38| +2451092|68255|15566|46457|1390149|2994|21086|8|22|1987|1|79.14|108.42|88.90|0.00|88.90|79.14|108.42|3.55|0.00|88.90|92.45|9.76| +2451092|68255|7922|46457|1390149|2994|21086|8|81|1987|93|87.76|116.72|24.51|0.00|2279.43|8161.68|10854.96|91.17|0.00|2279.43|2370.60|-5882.25| +2451092|68255|8246|46457|1390149|||||1987||85.14||46.23|0.00|||||0.00||4743.19|-3696.45| +2451092|68255|3212|46457|1390149|2994|21086|8|121|1987|41|91.62|97.11|4.85|9.94|198.85|3756.42|3981.51|7.55|9.94|188.91|196.46|-3567.51| +2451092|68255|13774|46457|1390149|2994|21086|8|213|1987|88|53.75|86.53|19.90|0.00|1751.20|4730.00|7614.64|17.51|0.00|1751.20|1768.71|-2978.80| +2451092|68255|11641|46457|1390149|2994|21086|8|287|1987|36|85.15|134.53|17.48|0.00|629.28|3065.40|4843.08|0.00|0.00|629.28|629.28|-2436.12| +2451092|68255|1801|46457|1390149|2994|21086|8|279|1987|64|36.85|56.74|55.03|2113.15|3521.92|2358.40|3631.36|42.26|2113.15|1408.77|1451.03|-949.63| +|68255|8260|||2994|21086||188|1987|||||0.00|1367.04|||13.67|0.00|||| +2451092|68255|14527|46457|1390149|2994|21086|8|293|1987|30|48.45|61.53|20.30|389.76|609.00|1453.50|1845.90|4.38|389.76|219.24|223.62|-1234.26| +2451092|68255|5414|46457|1390149|2994|21086|8|184|1987|17|55.24|89.48|85.90|0.00|1460.30|939.08|1521.16|116.82|0.00|1460.30|1577.12|521.22| +2451092|68255|13298|46457|1390149|2994|21086|8|238|1987|94|68.06|117.06|71.40|0.00|6711.60|6397.64|11003.64|201.34|0.00|6711.60|6912.94|313.96| +2451710|50951|10631|80524|8199|2274|33949|10|51|1988|91|45.04|72.96|50.34|0.00|4580.94|4098.64|6639.36|320.66|0.00|4580.94|4901.60|482.30| +2451710|50951|2654|80524|8199|2274|33949|10|274|1988|33|37.05|67.06|22.80|0.00|752.40|1222.65|2212.98|30.09|0.00|752.40|782.49|-470.25| +|50951|1051|80524|8199||33949|10||1988|47|5.12|9.16|1.28|0.00|60.16||430.52|5.41|0.00|60.16|65.57|| +2451710|50951|1075|80524|8199|2274|33949|10|191|1988|45|41.32|45.03|0.00|0.00|0.00|1859.40|2026.35|0.00|0.00|0.00|0.00|-1859.40| +2451710|50951|4160|80524|8199|2274|33949|10|154|1988|65|63.62|81.43|34.20|0.00|2223.00|4135.30|5292.95|66.69|0.00|2223.00|2289.69|-1912.30| +2451710|50951|17672|80524|8199|2274|33949|10|10|1988|32|31.97|37.40|8.97|0.00|287.04|1023.04|1196.80|0.00|0.00|287.04|287.04|-736.00| +2451710|50951|7580|80524|8199|2274|33949|10|185|1988|45|85.34|93.02|87.43|2163.89|3934.35|3840.30|4185.90|123.93|2163.89|1770.46|1894.39|-2069.84| +2451710|50951|9991|80524|8199|2274|33949|10|288|1988|92|69.98|105.66|71.84|0.00|6609.28|6438.16|9720.72|528.74|0.00|6609.28|7138.02|171.12| +2451710|50951|4597|80524|8199|2274|33949|10|283|1988|22|23.73|39.62|36.45|0.00|801.90|522.06|871.64|72.17|0.00|801.90|874.07|279.84| +2451710|50951|7331|80524|8199|2274|33949|10|88|1988|58|94.88|115.75|91.44|0.00|5303.52|5503.04|6713.50|212.14|0.00|5303.52|5515.66|-199.52| +2451710|50951|15973|80524|8199|2274|33949|10|249|1988|81|35.08|64.19|1.28|62.20|103.68|2841.48|5199.39|1.65|62.20|41.48|43.13|-2800.00| +2451710|50951|6515|80524|8199|2274|33949|10|246|1988|52|49.16|87.99|58.07|2566.69|3019.64|2556.32|4575.48|9.05|2566.69|452.95|462.00|-2103.37| +2451710|50951|15617|80524|8199|2274|33949|10|93|1988|94|57.76|72.20|12.27|0.00|1153.38|5429.44|6786.80|34.60|0.00|1153.38|1187.98|-4276.06| +||11350||1771072|4233||4|143|1989|||11.52||||228.00||17.84|||240.90|| +2451095|40431|412|8311|1771072|4233|19394|4|238|1989|59|70.16|96.82|19.36|0.00|1142.24|4139.44|5712.38|57.11|0.00|1142.24|1199.35|-2997.20| +2451095|40431|12908|8311|1771072|4233|19394|4|252|1989|87|76.89|153.78|24.60|0.00|2140.20|6689.43|13378.86|85.60|0.00|2140.20|2225.80|-4549.23| +2451095|40431|14960|8311|1771072|4233|19394|4|88|1989|65|26.77|38.54|0.77|0.00|50.05|1740.05|2505.10|0.00|0.00|50.05|50.05|-1690.00| +2451095|40431|5774|8311|1771072|4233|19394|4|288|1989|81|12.55|17.94|2.87|0.00|232.47|1016.55|1453.14|6.97|0.00|232.47|239.44|-784.08| +2451095|40431|2995|8311|1771072|4233|19394|4|230|1989|51|64.38|113.95|44.44|0.00|2266.44|3283.38|5811.45|158.65|0.00|2266.44|2425.09|-1016.94| +2451095|40431|4633|8311|1771072|4233|19394|4|207|1989|7|44.99|60.73|4.25|0.00|29.75|314.93|425.11|0.59|0.00|29.75|30.34|-285.18| +2451095|40431|12842|8311|1771072|4233|19394|4|61|1989|28|38.69|41.39|8.27|203.77|231.56|1083.32|1158.92|0.00|203.77|27.79|27.79|-1055.53| +2451095|40431|5719|8311|1771072|4233|19394|4|102|1989|72|80.81|122.02|47.58|1952.68|3425.76|5818.32|8785.44|73.65|1952.68|1473.08|1546.73|-4345.24| +2451095|40431|8834|8311|1771072|4233|19394|4|268|1989|54|63.17|109.28|46.99|0.00|2537.46|3411.18|5901.12|76.12|0.00|2537.46|2613.58|-873.72| +2451095|40431|14317|8311|1771072|4233|19394|4|26|1989|66|17.35|34.52|26.23|0.00|1731.18|1145.10|2278.32|17.31|0.00|1731.18|1748.49|586.08| +2451095|40431|3836|8311|1771072|4233|19394|4|233|1989|96|48.14|60.65|28.50|0.00|2736.00|4621.44|5822.40|164.16|0.00|2736.00|2900.16|-1885.44| +2451095|40431|5041|8311|1771072|4233|19394|4|43|1989|80|90.29|93.90|51.64|0.00|4131.20|7223.20|7512.00|371.80|0.00|4131.20|4503.00|-3092.00| +2452258|45306|2429|53882|1539991|6453|17315|1|257|1990|9|88.71|175.64|168.61|743.57|1517.49|798.39|1580.76|30.95|743.57|773.92|804.87|-24.47| +2452258|45306|3351|53882|1539991|6453|17315|1|172|1990|32|28.48|31.32|22.23|462.38|711.36|911.36|1002.24|2.48|462.38|248.98|251.46|-662.38| +2452258|45306|12389|53882|1539991|6453|17315|1|217|1990|71|94.25|150.80|3.01|0.00|213.71|6691.75|10706.80|14.95|0.00|213.71|228.66|-6478.04| +2452258|45306|10889|53882|1539991|6453|17315|1|84|1990|55|93.99|127.82|54.96|0.00|3022.80|5169.45|7030.10|90.68|0.00|3022.80|3113.48|-2146.65| +2452258|45306|8071|53882|1539991|6453|17315|1|272|1990|41|11.70|13.92|4.87|0.00|199.67|479.70|570.72|7.98|0.00|199.67|207.65|-280.03| +2452258|45306|14435|53882|1539991|6453|17315|1|102|1990|11|83.09|138.76|116.55|0.00|1282.05|913.99|1526.36|38.46|0.00|1282.05|1320.51|368.06| +2452258|45306|8775|53882|1539991|6453|17315|1|68|1990|100|88.49|166.36|14.97|0.00|1497.00|8849.00|16636.00|134.73|0.00|1497.00|1631.73|-7352.00| +2452258|45306|1537|53882|1539991|6453|17315|1|195|1990|37|21.46|39.48|9.87|0.00|365.19|794.02|1460.76|18.25|0.00|365.19|383.44|-428.83| +2452258|45306|13151|53882|1539991|6453|17315|1|144|1990|80|14.23|14.79|12.86|658.43|1028.80|1138.40|1183.20|22.22|658.43|370.37|392.59|-768.03| +2452258|45306|8171|53882|1539991|6453|17315|1|209|1990|98|63.94|86.95|74.77|0.00|7327.46|6266.12|8521.10|586.19|0.00|7327.46|7913.65|1061.34| +2452258|45306|10891|53882|1539991|6453|17315|1|31|1990|35|36.55|72.73|32.72|22.90|1145.20|1279.25|2545.55|0.00|22.90|1122.30|1122.30|-156.95| +2452258|45306|575|53882|1539991|6453|17315|1|297|1990|54|96.67|108.27|29.23|0.00|1578.42|5220.18|5846.58|63.13|0.00|1578.42|1641.55|-3641.76| +2452258|45306|17477|53882|1539991|6453|17315|1|19|1990|75|26.86|44.58|28.97|0.00|2172.75|2014.50|3343.50|86.91|0.00|2172.75|2259.66|158.25| +2452371|32920|11167|30406|1713286|6119|43799|2|252|1991|32|37.26|45.82|39.86|0.00|1275.52|1192.32|1466.24|102.04|0.00|1275.52|1377.56|83.20| +2452371|32920|17088|30406|1713286|6119|43799|2|6|1991|56|78.08|80.42|34.58|0.00|1936.48|4372.48|4503.52|154.91|0.00|1936.48|2091.39|-2436.00| +2452371|32920|2917|30406|1713286|6119|43799|2|280|1991|100|53.66|87.46|10.49|0.00|1049.00|5366.00|8746.00|0.00|0.00|1049.00|1049.00|-4317.00| +2452371|32920|15123|30406|1713286|6119|43799|2|124|1991|24|29.74|45.79|0.45|0.00|10.80|713.76|1098.96|0.43|0.00|10.80|11.23|-702.96| +2452371|32920|762|30406|1713286|6119|43799|2|76|1991|67|78.07|109.29|10.92|0.00|731.64|5230.69|7322.43|21.94|0.00|731.64|753.58|-4499.05| +||8994||||43799|2|116|1991||||9.40||310.20||862.29|||248.16|250.64|| +2452371|32920|895|30406|1713286|6119|43799|2|148|1991|23|79.69|108.37|6.50|0.00|149.50|1832.87|2492.51|2.99|0.00|149.50|152.49|-1683.37| +2452371|32920|2659|30406|1713286|6119|43799|2|152|1991|73|20.43|23.29|17.23|679.20|1257.79|1491.39|1700.17|17.35|679.20|578.59|595.94|-912.80| +2452371|32920|8053|30406|1713286|6119|43799|2|42|1991|8|40.37|44.81|14.78|0.00|118.24|322.96|358.48|3.54|0.00|118.24|121.78|-204.72| +2452371|32920|3474|30406|1713286|6119|43799|2|236|1991|92|52.31|75.32|36.15|0.00|3325.80|4812.52|6929.44|33.25|0.00|3325.80|3359.05|-1486.72| +2452525|45978|5473|72145|196867|4789|7864|10|294|1992|56|51.40|80.69|78.26|0.00|4382.56|2878.40|4518.64|350.60|0.00|4382.56|4733.16|1504.16| +2452525|45978|637|72145|196867|4789|7864|10|71|1992|69|92.34|169.90|56.06|3326.60|3868.14|6371.46|11723.10|10.83|3326.60|541.54|552.37|-5829.92| +2452525|45978|15499|72145|196867|4789|7864|10|122|1992|32|72.60|84.21|10.94|0.00|350.08|2323.20|2694.72|24.50|0.00|350.08|374.58|-1973.12| +2452525|45978|349|72145|196867|4789|7864|10|87|1992|10|62.90|74.85|1.49|0.00|14.90|629.00|748.50|0.14|0.00|14.90|15.04|-614.10| +2452525|45978|351|72145|196867|4789|7864|10|151|1992|83|37.98|62.28|51.69|0.00|4290.27|3152.34|5169.24|343.22|0.00|4290.27|4633.49|1137.93| +2452525|45978|14797|72145|196867|4789|7864|10|79|1992|98|94.99|176.68|106.00|0.00|10388.00|9309.02|17314.64|415.52|0.00|10388.00|10803.52|1078.98| +2452525|45978|3889|72145|196867|4789|7864|10|190|1992|18|59.45|115.33|93.41|924.75|1681.38|1070.10|2075.94|37.83|924.75|756.63|794.46|-313.47| +2452525|45978|9540|72145|196867|4789|7864|10|119|1992|53|72.15|142.13|103.75|0.00|5498.75|3823.95|7532.89|439.90|0.00|5498.75|5938.65|1674.80| +2452525|45978|14821|72145|196867|4789|7864|10|278|1992|64|84.05|89.09|31.18|0.00|1995.52|5379.20|5701.76|39.91|0.00|1995.52|2035.43|-3383.68| +2452525|45978|75|72145|196867|4789|7864|10|92|1992|68|6.71|10.13|4.25|0.00|289.00|456.28|688.84|0.00|0.00|289.00|289.00|-167.28| +2452525|45978|618|72145|196867|4789|7864|10|250|1992|23|91.61|174.97|157.47|0.00|3621.81|2107.03|4024.31|72.43|0.00|3621.81|3694.24|1514.78| +2452525|45978|1779|72145|196867|4789|7864|10|157|1992|83|60.82|86.97|3.47|5.76|288.01|5048.06|7218.51|11.29|5.76|282.25|293.54|-4765.81| +2452525|45978|15993|72145|196867|4789|7864|10|265|1992|54|2.28|3.96|2.25|0.00|121.50|123.12|213.84|2.43|0.00|121.50|123.93|-1.62| +2452210|38202|8463|68122|1739404|2239|14274|2|144|1993|65|39.07|58.99|53.68|0.00|3489.20|2539.55|3834.35|209.35|0.00|3489.20|3698.55|949.65| +2452210|38202|2603|68122|1739404|2239|14274|2|156|1993|17|47.63|59.53|51.19|626.56|870.23|809.71|1012.01|0.00|626.56|243.67|243.67|-566.04| +2452210|38202|7901|68122|1739404|2239|14274|2|101|1993|89|42.81|56.50|45.76|2728.66|4072.64|3810.09|5028.50|80.63|2728.66|1343.98|1424.61|-2466.11| +2452210|38202|10063|68122|1739404|2239|14274||60|1993|75|57.53|60.98|28.66|0.00|2149.50|4314.75||42.99|0.00|2149.50||-2165.25| +2452210|38202|1841|68122|1739404|2239|14274|2|26|1993|100|69.59|99.51|79.60|0.00|7960.00|6959.00|9951.00|557.20|0.00|7960.00|8517.20|1001.00| +2452210|38202|4179||1739404|2239||2|249|1993|62|55.79||28.48||1765.76|||||||| +2452210|38202|4013|68122|1739404|2239|14274|2|102|1993|60|79.32|133.25|115.92|5216.40|6955.20|4759.20|7995.00|86.94|5216.40|1738.80|1825.74|-3020.40| +2452210|38202|13057|68122|1739404|2239|14274|2|51|1993|84|54.32|103.75|6.22|0.00|522.48|4562.88|8715.00|26.12|0.00|522.48|548.60|-4040.40| +2452210|38202|14689|68122|1739404|2239|14274|2|200|1993|47|67.83|109.20|81.90|0.00|3849.30|3188.01|5132.40|76.98|0.00|3849.30|3926.28|661.29| +2452633|70373|145|9544|721753|2762|171|7|124|1994|79|88.22|89.10|50.78|0.00|4011.62|6969.38|7038.90|40.11|0.00|4011.62|4051.73|-2957.76| +2452633|70373|7243|9544|721753|2762|171|7|134|1994|83|95.86|161.04|1.61|0.00|133.63|7956.38|13366.32|10.69|0.00|133.63|144.32|-7822.75| +2452633|70373|15750|9544|721753|2762|171|7|42|1994|80|29.40|37.33|30.23|0.00|2418.40|2352.00|2986.40|169.28|0.00|2418.40|2587.68|66.40| +2452633|70373|14761|9544|721753|2762|171|7|8|1994|52|30.67|40.17|30.52|0.00|1587.04|1594.84|2088.84|111.09|0.00|1587.04|1698.13|-7.80| +2452633|70373|9333|9544|721753|2762|171|7|291|1994|68|32.45|59.38|36.81|0.00|2503.08|2206.60|4037.84|75.09|0.00|2503.08|2578.17|296.48| +2452633|70373|7194|9544|721753|2762|171|7|298|1994|36|57.44|76.96|51.56|0.00|1856.16|2067.84|2770.56|74.24|0.00|1856.16|1930.40|-211.68| +2452633|70373|7851|9544|721753|2762|171|7|54|1994|84|96.06|145.05|36.26|0.00|3045.84|8069.04|12184.20|182.75|0.00|3045.84|3228.59|-5023.20| +2452633|70373|17641|9544|721753|2762|171|7|188|1994|3|56.49|88.68|15.07|0.00|45.21|169.47|266.04|2.26|0.00|45.21|47.47|-124.26| +2452633|70373|4747|9544|721753|2762|171|7|205|1994|87|71.89|88.42|19.45|0.00|1692.15|6254.43|7692.54|0.00|0.00|1692.15|1692.15|-4562.28| +2452633|70373|3066|9544|721753|2762|171|7|118|1994|85|94.62|169.36|125.32|0.00|10652.20|8042.70|14395.60|106.52|0.00|10652.20|10758.72|2609.50| +2452559|38836|15429|37691|676712|6486|46707|10|41|1995|43|68.68|100.27|2.00|0.00|86.00|2953.24|4311.61|4.30|0.00|86.00|90.30|-2867.24| +2452559|38836|9825|37691|676712|6486|46707|10|183|1995|46|40.69|67.95|24.46|517.57|1125.16|1871.74|3125.70|24.30|517.57|607.59|631.89|-1264.15| +2452559|38836|6727|37691|676712|6486|46707|10|156|1995|56|45.91|52.79|25.33|0.00|1418.48|2570.96|2956.24|113.47|0.00|1418.48|1531.95|-1152.48| +2452559|38836|4926|37691|676712|6486|46707|10|60|1995|78|36.58|41.33|2.47|0.00|192.66|2853.24|3223.74|5.77|0.00|192.66|198.43|-2660.58| +2452559|38836|3207|37691|676712|6486|46707|10|129|1995|85|23.43|45.45|0.90|0.00|76.50|1991.55|3863.25|3.82|0.00|76.50|80.32|-1915.05| +2452559|38836|15798|37691|676712|6486|46707|10|42|1995|62|28.87|36.66|20.89|0.00|1295.18|1789.94|2272.92|38.85|0.00|1295.18|1334.03|-494.76| +2452559|38836|7686|37691|676712|6486|46707|10|232|1995|67|3.24|4.11|2.79|130.85|186.93|217.08|275.37|0.56|130.85|56.08|56.64|-161.00| +2452559|38836|789|37691|676712|6486|46707|10|189|1995|46|61.30|107.27|98.68|0.00|4539.28|2819.80|4934.42|0.00|0.00|4539.28|4539.28|1719.48| +2452559|38836|11155|37691|676712|6486|46707|10|22|1995|14|95.06|152.09|135.36|0.00|1895.04|1330.84|2129.26|0.00|0.00|1895.04|1895.04|564.20| +2452537|39317|15732|71664|1212469|5627|28785|7|24|1996|81|76.90|76.90|41.52|2825.02|3363.12|6228.90|6228.90|10.76|2825.02|538.10|548.86|-5690.80| +2452537|39317|15111|71664|1212469|5627|28785|7|44|1996|96|89.19|149.83|88.39|0.00|8485.44|8562.24|14383.68|84.85|0.00|8485.44|8570.29|-76.80| +2452537|39317|5958|71664|1212469|5627|28785|7|47|1996|53|31.32|59.19|49.12|0.00|2603.36|1659.96|3137.07|104.13|0.00|2603.36|2707.49|943.40| +2452537|39317|11964|71664|1212469|5627|28785|7|165|1996|99|36.90|56.08|12.89|1250.58|1276.11|3653.10|5551.92|1.78|1250.58|25.53|27.31|-3627.57| +2452537|39317|9726|71664|1212469|5627|28785|7|96|1996|74|45.05|66.67|10.00|0.00|740.00|3333.70|4933.58|14.80|0.00|740.00|754.80|-2593.70| +2452537|39317|8847|71664|1212469|5627|28785|7|293|1996|27|71.47|121.49|1.21|0.00|32.67|1929.69|3280.23|1.96|0.00|32.67|34.63|-1897.02| +2452537|39317|10435|71664|1212469|5627|28785|7|260|1996|72|48.55|76.22|6.09|0.00|438.48|3495.60|5487.84|26.30|0.00|438.48|464.78|-3057.12| +2452537|39317|15651|71664|1212469|5627|28785|7|212|1996|81|11.49|22.52|16.21|709.02|1313.01|930.69|1824.12|54.35|709.02|603.99|658.34|-326.70| +2452537|39317|18000|71664|1212469||28785|||1996|37|30.63|||||1133.31||0.80||||| +2452537|39317|3615|71664|1212469|5627|28785|7|289|1996|30|57.95|99.67|1.99|0.00|59.70|1738.50|2990.10|2.98|0.00|59.70|62.68|-1678.80| +2452537|39317|9267|71664|1212469|5627|28785|7|204|1996|48|90.38|178.04|140.65|0.00|6751.20|4338.24|8545.92|607.60|0.00|6751.20|7358.80|2412.96| +2452537|39317|15943|71664|1212469|5627|28785|7|91|1996|63|30.33|57.32|48.72|0.00|3069.36|1910.79|3611.16|122.77|0.00|3069.36|3192.13|1158.57| +2452537|39317|14190|71664|1212469|5627|28785|7|221|1996|9|34.50|53.13|51.00|0.00|459.00|310.50|478.17|4.59|0.00|459.00|463.59|148.50| +2452537|39317|11652|71664|1212469|5627|28785|7|300|1996|84|68.28|82.61|3.30|166.32|277.20|5735.52|6939.24|8.87|166.32|110.88|119.75|-5624.64| +2452537|39317|3441|71664|1212469|5627|28785|7|120|1996|37|37.95|72.10|21.63|0.00|800.31|1404.15|2667.70|24.00|0.00|800.31|824.31|-603.84| +2452537|39317|2839|71664|1212469|5627|28785|7|256|1996|46|82.76|153.10|3.06|0.00|140.76|3806.96|7042.60|4.22|0.00|140.76|144.98|-3666.20| +2451628|35554|10982|8752|59737|6275|22460|7|71|1997|18|47.18|65.58|8.52|0.00|153.36|849.24|1180.44|7.66|0.00|153.36|161.02|-695.88| +2451628|35554|4861|8752|59737|6275|22460|7|161|1997|99|43.37|66.35|41.13|0.00|4071.87|4293.63|6568.65|203.59|0.00|4071.87|4275.46|-221.76| +2451628|35554|8419|8752|59737|6275|22460|7|126|1997|87|50.18|65.73|9.20|0.00|800.40|4365.66|5718.51|8.00|0.00|800.40|808.40|-3565.26| +2451628|35554|8342|8752|59737|6275|22460|7|298|1997|14|94.08|130.77|94.15|0.00|1318.10|1317.12|1830.78|39.54|0.00|1318.10|1357.64|0.98| +2451628|35554|7777|8752|59737|6275|22460|7|149|1997|34|62.40|67.39|28.97|364.44|984.98|2121.60|2291.26|49.64|364.44|620.54|670.18|-1501.06| +2451628|35554|6620|8752|59737|6275|22460|7|102|1997|26|73.23|88.60|7.08|0.00|184.08|1903.98|2303.60|5.52|0.00|184.08|189.60|-1719.90| +2451628|35554|8048|8752|59737|6275|22460|7|57|1997|81|87.04|104.44|61.61|1946.25|4990.41|7050.24|8459.64|91.32|1946.25|3044.16|3135.48|-4006.08| +2451628|35554|7745|8752|59737|6275|22460|7|38|1997|96|11.68|16.00|13.44|0.00|1290.24|1121.28|1536.00|64.51|0.00|1290.24|1354.75|168.96| +2451628|35554|16883|8752|59737|6275|22460|7|187|1997|17|13.68|16.14|8.55|0.00|145.35|232.56|274.38|7.26|0.00|145.35|152.61|-87.21| +2451628|35554|10682|8752|59737|6275|22460|7|43|1997|61|2.93|5.21|1.14|0.00|69.54|178.73|317.81|1.39|0.00|69.54|70.93|-109.19| +2451628|35554|3677|8752|59737|6275|22460|7|137|1997|20|53.90|61.98|27.27|0.00|545.40|1078.00|1239.60|0.00|0.00|545.40|545.40|-532.60| +2451628|35554|7310|8752|59737|6275|22460|7|171|1997|20|69.46|120.86|84.60|0.00|1692.00|1389.20|2417.20|84.60|0.00|1692.00|1776.60|302.80| +2451628|35554|4943|8752|59737|6275|22460|7|91|1997|49|74.14|86.74|17.34|254.89|849.66|3632.86|4250.26|35.68|254.89|594.77|630.45|-3038.09| +2451628|35554|8681|8752|59737|6275|22460|7|166|1997|77|52.28|53.84|2.15|0.00|165.55|4025.56|4145.68|11.58|0.00|165.55|177.13|-3860.01| +2451628|35554|2390|8752|59737|6275|22460|7|95|1997|71|41.05|59.93|47.34|0.00|3361.14|2914.55|4255.03|268.89|0.00|3361.14|3630.03|446.59| +2451628|35554|17693|8752|59737|6275|22460|7|27|1997|51|93.88|160.53|75.44|0.00|3847.44|4787.88|8187.03|115.42|0.00|3847.44|3962.86|-940.44| +2451330|50338|650|20855|1114741|274|35007|1|235|1998|54|65.28|97.92|28.39|0.00|1533.06|3525.12|5287.68|122.64|0.00|1533.06|1655.70|-1992.06| +2451330|50338|9224|20855|1114741|274|35007|1|167|1998|4|15.39|29.54|11.52|9.21|46.08|61.56|118.16|1.84|9.21|36.87|38.71|-24.69| +2451330|50338|15919|20855|1114741|274|35007|1|80|1998|97|89.43|126.09|39.08|0.00|3790.76|8674.71|12230.73|189.53|0.00|3790.76|3980.29|-4883.95| +2451330|50338|12718|20855|1114741|274|35007|1|124|1998|67|5.17|5.84|3.09|0.00|207.03|346.39|391.28|18.63|0.00|207.03|225.66|-139.36| +2451330|50338|4663|20855|1114741|274|35007|1|124|1998|5|72.97|102.88|37.03|0.00|185.15|364.85|514.40|3.70|0.00|185.15|188.85|-179.70| +2451330|50338|9763|20855|1114741|274|35007|1|240|1998|43|13.87|25.52|18.37|0.00|789.91|596.41|1097.36|7.89|0.00|789.91|797.80|193.50| +2451330|50338|9460|20855|1114741|274|35007|1|89|1998|33|39.72|42.89|12.00|0.00|396.00|1310.76|1415.37|31.68|0.00|396.00|427.68|-914.76| +2451330|50338|12520|20855|1114741|274|35007|1|28|1998|47|29.50|55.16|14.34|390.90|673.98|1386.50|2592.52|16.98|390.90|283.08|300.06|-1103.42| +2451330|50338|3844|20855|1114741|274|35007|1|221|1998|92|86.03|100.65|77.50|0.00|7130.00|7914.76|9259.80|142.60|0.00|7130.00|7272.60|-784.76| +2451330|50338|12049|20855|1114741|274|35007|1|293|1998|92|21.07|21.49|10.74|0.00|988.08|1938.44|1977.08|29.64|0.00|988.08|1017.72|-950.36| +2451330|50338|7886|20855|1114741|274|35007|1|162|1998|9|23.38|42.08|38.29|341.16|344.61|210.42|378.72|0.03|341.16|3.45|3.48|-206.97| +2451330|50338|5101|20855|1114741|274|35007|1|148|1998|27|96.17|104.82|30.39|0.00|820.53|2596.59|2830.14|16.41|0.00|820.53|836.94|-1776.06| +2451874|37819|16471|72388|815574|5513|49899|8|60|1999|35|96.61|186.45|22.37|0.00|782.95|3381.35|6525.75|54.80|0.00|782.95|837.75|-2598.40| +2451874|37819|1454|72388|815574|5513|49899|8|83|1999|4|73.40|86.61|35.51|0.00|142.04|293.60|346.44|1.42|0.00|142.04|143.46|-151.56| +2451874|37819|8309|72388|815574|5513|49899|8|105|1999|37|7.69|15.38|14.61|0.00|540.57|284.53|569.06|43.24|0.00|540.57|583.81|256.04| +2451874|37819|9917|72388|815574|5513|49899|8|98|1999|99|74.90|107.85|91.67|0.00|9075.33|7415.10|10677.15|0.00|0.00|9075.33|9075.33|1660.23| +2451874|37819|15050|72388|815574|5513|49899|8|287|1999|82|70.05|92.46|82.28|0.00|6746.96|5744.10|7581.72|67.46|0.00|6746.96|6814.42|1002.86| +2451874|37819|3223|72388|815574|5513|49899|8|164|1999|23|41.86|63.20|13.90|0.00|319.70|962.78|1453.60|19.18|0.00|319.70|338.88|-643.08| +2451874|37819|2537|72388|815574|5513|49899|8|183|1999|21|16.22|21.73|3.91|0.00|82.11|340.62|456.33|2.46|0.00|82.11|84.57|-258.51| +2451874|37819|10334|72388|815574|5513|49899|8|72|1999|97|62.60|90.77|13.61|0.00|1320.17|6072.20|8804.69|26.40|0.00|1320.17|1346.57|-4752.03| +2451874|37819|1027|72388|815574|5513|49899|8|106|1999|87|79.07|128.88|108.25|0.00|9417.75|6879.09|11212.56|565.06|0.00|9417.75|9982.81|2538.66| +2451874|37819|6893|72388|815574|5513|49899|8|203|1999|10|7.87|9.44|7.64|0.00|76.40|78.70|94.40|0.76|0.00|76.40|77.16|-2.30| +2451874|37819|5288|72388|815574|5513|49899|8|163|1999|69|41.05|64.85|49.28|0.00|3400.32|2832.45|4474.65|306.02|0.00|3400.32|3706.34|567.87| +2452209|51320|16517|64962|1473290|1618|41146|7|199|2000|53|88.53|173.51|48.58|0.00|2574.74|4692.09|9196.03|51.49|0.00|2574.74|2626.23|-2117.35| +2452209|51320|16701|64962|1473290|1618|41146|7|261|2000|3|33.15|45.08|9.01|0.00|27.03|99.45|135.24|1.35|0.00|27.03|28.38|-72.42| +|51320|15265|64962|1473290||||271|2000|87|75.10||68.81|2693.91||6533.70|||2693.91|3292.56|3292.56|| +2452209|51320|7561|64962|1473290|1618|41146|7|76|2000|21|38.58|67.51|40.50|0.00|850.50|810.18|1417.71|0.00|0.00|850.50|850.50|40.32| +2452209|51320|3579|64962|1473290|1618|41146|7|228|2000|37|85.49|167.56|110.58|0.00|4091.46|3163.13|6199.72|327.31|0.00|4091.46|4418.77|928.33| +2452209|51320|17443|64962|1473290|1618|41146|7|215|2000|71|30.99|30.99|8.67|0.00|615.57|2200.29|2200.29|0.00|0.00|615.57|615.57|-1584.72| +2452209|51320|5155|64962|1473290|1618|41146|7|222|2000|81|82.18|160.25|153.84|8224.28|12461.04|6656.58|12980.25|254.20|8224.28|4236.76|4490.96|-2419.82| +2452209|51320|11785|64962|1473290|1618|41146|7|239|2000|4|66.17|101.90|45.85|51.35|183.40|264.68|407.60|3.96|51.35|132.05|136.01|-132.63| +|51320|1627|||1618|41146||8|2000|79||77.26|19.31|||4882.99||||564.44||| +2452209|51320|16901|64962|1473290|1618|41146|7|14|2000|98|29.72|35.66|16.40|0.00|1607.20|2912.56|3494.68|128.57|0.00|1607.20|1735.77|-1305.36| +2452209|51320|10607|64962|1473290|1618|41146|7|197|2000|41|59.05|113.96|49.00|0.00|2009.00|2421.05|4672.36|160.72|0.00|2009.00|2169.72|-412.05| +2452209|51320|399|64962|1473290|1618|41146|7|233|2000|9|91.24|108.57|108.57|0.00|977.13|821.16|977.13|9.77|0.00|977.13|986.90|155.97| +2452209|51320|7543|64962|1473290|1618|41146|7|89|2000|2|46.34|48.65|21.40|0.00|42.80|92.68|97.30|1.28|0.00|42.80|44.08|-49.88| +2452209|51320|11309|64962|1473290|1618|41146|7|213|2000|69|47.92|63.25|1.89|0.00|130.41|3306.48|4364.25|7.82|0.00|130.41|138.23|-3176.07| +2452209|51320|16537|64962|1473290|1618|41146|7|175|2000|13|3.35|3.68|1.03|5.75|13.39|43.55|47.84|0.53|5.75|7.64|8.17|-35.91| +2451743|51226|12467|89325|1211941|5757|17173|4|271|2001|41|28.67|55.90|16.77|0.00|687.57|1175.47|2291.90|61.88|0.00|687.57|749.45|-487.90| +2451743|51226|13153|89325|1211941|5757|17173|4|198|2001|67|27.36|54.17|40.08|0.00|2685.36|1833.12|3629.39|161.12|0.00|2685.36|2846.48|852.24| +2451743|51226|2819|89325|1211941|5757|17173|4|53|2001|28|68.42|78.68|20.45|0.00|572.60|1915.76|2203.04|17.17|0.00|572.60|589.77|-1343.16| +2451743|51226|11258|89325|1211941|5757|17173|4|163|2001|83|24.26|28.38|9.64|0.00|800.12|2013.58|2355.54|40.00|0.00|800.12|840.12|-1213.46| +2451743|51226|17798|89325|1211941|5757|17173|4|121|2001|61|15.07|24.86|11.93|0.00|727.73|919.27|1516.46|36.38|0.00|727.73|764.11|-191.54| +2451743|51226|2828|89325|1211941|5757|17173|4|165|2001|49|39.19|69.36|22.88|482.08|1121.12|1920.31|3398.64|25.56|482.08|639.04|664.60|-1281.27| +|51226|5971|||5757||||2001|7||||51.20|138.39||419.51|5.23|51.20|87.19|92.42|-214.65| +2451743|51226|6683|89325|1211941|5757|17173|4|231|2001|67|77.72|117.35|84.49|0.00|5660.83|5207.24|7862.45|452.86|0.00|5660.83|6113.69|453.59| +2451743|51226|4430|89325|1211941|5757|17173|4|255|2001|71|92.53|115.66|79.80|0.00|5665.80|6569.63|8211.86|0.00|0.00|5665.80|5665.80|-903.83| +2451162|71127|11476|44661|1500547|2833|7072|7|237|2002|79|35.12|49.87|44.38|0.00|3506.02|2774.48|3939.73|210.36|0.00|3506.02|3716.38|731.54| +2451162|71127|9664|44661|1500547|2833|7072|7|37|2002|37|21.47|25.97|25.19|363.49|932.03|794.39|960.89|45.48|363.49|568.54|614.02|-225.85| +2451162||10742|||2833|7072||273|2002|12|||41.09|108.47|493.08|||23.07|108.47|384.61|407.68|| +2451162|71127|6958|44661|1500547|2833|7072|7|112|2002|54|78.02|122.49|112.69|0.00|6085.26|4213.08|6614.46|547.67|0.00|6085.26|6632.93|1872.18| +2451162|71127|14956|44661|1500547|2833|7072|7|150|2002|3|44.53|55.66|40.07|0.00|120.21|133.59|166.98|6.01|0.00|120.21|126.22|-13.38| +2451162|71127|4670|44661|1500547|2833|7072|7|99|2002|10|11.45|18.66|5.97|0.00|59.70|114.50|186.60|3.58|0.00|59.70|63.28|-54.80| +2451162|71127|12694|44661|1500547|2833|7072|7|142|2002|57|45.53|63.28|0.00|0.00|0.00|2595.21|3606.96|0.00|0.00|0.00|0.00|-2595.21| +2451162|71127|16171|44661|1500547|2833|7072|7|7|2002|9|70.82|87.81|24.58|44.24|221.22|637.38|790.29|8.84|44.24|176.98|185.82|-460.40| +2451162|71127|9982||||7072|||2002|73|32.65|45.71||0.00|2235.26|2383.45|3336.83|67.05|0.00||2302.31|| +2451162|71127|6361|44661|1500547|2833|7072|7|96|2002|64|60.10|62.50|3.75|0.00|240.00|3846.40|4000.00|12.00|0.00|240.00|252.00|-3606.40| +2451162|71127|5960|44661|1500547|2833|7072|7|245|2002|23|13.79|21.92|19.72|0.00|453.56|317.17|504.16|22.67|0.00|453.56|476.23|136.39| +2452202|41612|9035|95148|1368198|3648|19098|||2003||91.80|151.47|110.57||4201.66||5755.86|||||713.26| +2452202|41612|15189|95148|1368198|3648|19098|10|253|2003|81|98.22|154.20|84.81|2541.75|6869.61|7955.82|12490.20|0.00|2541.75|4327.86|4327.86|-3627.96| +2452202|41612|4783|95148|1368198|3648|19098|10|74|2003|55|69.49|72.26|39.74|2098.27|2185.70|3821.95|3974.30|3.49|2098.27|87.43|90.92|-3734.52| +2452202|41612|1275|95148|1368198|3648|19098|10|12|2003|80|51.32|96.48|95.51|0.00|7640.80|4105.60|7718.40|229.22|0.00|7640.80|7870.02|3535.20| +2452202||13965||1368198|||||2003|31|15.69|||0.00||486.39|637.05||0.00||79.31|-410.13| +2452202|41612|11815|95148|1368198|3648|19098|10|219|2003|89|52.91|97.35|71.06|1960.54|6324.34|4708.99|8664.15|87.27|1960.54|4363.80|4451.07|-345.19| +2452202|41612|3839|95148|1368198|3648|19098|10|101|2003|55|9.06|13.77|3.30|0.00|181.50|498.30|757.35|9.07|0.00|181.50|190.57|-316.80| +2452202|41612|9247|95148|1368198|3648|19098|10|118|2003|45|89.74|169.60|64.44|0.00|2899.80|4038.30|7632.00|231.98|0.00|2899.80|3131.78|-1138.50| +||12063|95148|1368198|3648|19098||36|2003|33||140.82|5.63||||4647.06|||185.79|187.64|| +2452202|41612|10257|95148|1368198|3648|19098|10|141|2003|76|33.09|42.02|4.62|0.00|351.12|2514.84|3193.52|7.02|0.00|351.12|358.14|-2163.72| +2452202|41612|14889|95148|1368198|3648|19098|10|60|2003|90|12.77|22.85|19.42|0.00|1747.80|1149.30|2056.50|0.00|0.00|1747.80|1747.80|598.50| +2452202|41612|7165|95148|1368198|3648|19098|10|268|2003|69|40.73|51.72|9.82|0.00|677.58|2810.37|3568.68|0.00|0.00|677.58|677.58|-2132.79| +2452202|41612|17811|95148|1368198|3648|19098|10|224|2003|8|12.22|22.85|8.68|0.00|69.44|97.76|182.80|0.00|0.00|69.44|69.44|-28.32| +||13019|95148|||19098|||2003|40||69.25||0.00|||2770.00||0.00|1384.80|1412.49|-1278.80| +|41612|15503||1368198|||10||2003|31||46.97||0.00|1193.81|787.09|||0.00||1277.37|| +2451952|49134|12433|12942|1379458|1974|39643|8|230|2004|71|13.66|25.68|22.08|0.00|1567.68|969.86|1823.28|78.38|0.00|1567.68|1646.06|597.82| +2451952|49134|5405|12942|1379458|1974|39643|8|110|2004|24|32.06|52.89|10.57|0.00|253.68|769.44|1269.36|20.29|0.00|253.68|273.97|-515.76| +2451952|49134|6581|12942|1379458|1974|39643|8|300|2004|16|71.81|112.02|2.24|9.31|35.84|1148.96|1792.32|0.53|9.31|26.53|27.06|-1122.43| +2451952|49134|4111|12942|1379458|1974|39643|8|37|2004|14|12.67|17.10|9.74|0.00|136.36|177.38|239.40|12.27|0.00|136.36|148.63|-41.02| +||17419||1379458|1974||8||2004|36|29.97|36.56|34.36||||1316.16|0.00||296.88||-782.04| +2451952|49134|13201|12942|1379458|1974|39643|8|233|2004|10|41.69|49.61|10.91|0.00|109.10|416.90|496.10|9.81|0.00|109.10|118.91|-307.80| +2451952|49134|727|12942|1379458|1974|39643|8|9|2004|30|76.30|99.19|4.95|0.00|148.50|2289.00|2975.70|1.48|0.00|148.50|149.98|-2140.50| +2451952|49134|11809|12942|1379458|1974|39643|8|154|2004|41|38.58|64.42|20.61|0.00|845.01|1581.78|2641.22|50.70|0.00|845.01|895.71|-736.77| +2451952|49134|2065|12942|1379458|1974|39643|8|232|2004|1|25.28|33.36|20.34|0.00|20.34|25.28|33.36|0.40|0.00|20.34|20.74|-4.94| +|49134|8493||1379458|1974||8|35|2004||38.36|66.74||0.00||2186.52|||0.00|||-208.62| +2451952|49134|3663|12942|1379458|1974|39643|8|275|2004|48|80.97|130.36|101.68|0.00|4880.64|3886.56|6257.28|146.41|0.00|4880.64|5027.05|994.08| +2452603|68793|6559|94962|1452846|1483|20567|8|84|2005|74|24.91|41.84|21.33|0.00|1578.42|1843.34|3096.16|0.00|0.00|1578.42|1578.42|-264.92| +2452603|68793|8743|94962|1452846|1483|20567|8|221|2005|22|76.54|139.30|39.00|0.00|858.00|1683.88|3064.60|77.22|0.00|858.00|935.22|-825.88| +2452603|68793|17701|94962|1452846|1483|20567|8|121|2005|34|5.42|9.59|7.38|0.00|250.92|184.28|326.06|12.54|0.00|250.92|263.46|66.64| +2452603|68793|1995|94962|1452846|1483|20567|8|207|2005|32|87.54|175.08|168.07|0.00|5378.24|2801.28|5602.56|376.47|0.00|5378.24|5754.71|2576.96| +2452603|68793|9213|94962|1452846|1483|20567|8|157|2005|78|17.21|24.26|5.82|326.85|453.96|1342.38|1892.28|3.81|326.85|127.11|130.92|-1215.27| +2452603|68793|2593|94962|1452846|1483|20567|8|253|2005|98|47.58|72.79|61.14|0.00|5991.72|4662.84|7133.42|119.83|0.00|5991.72|6111.55|1328.88| +2452603|68793|16716|94962|1452846|1483|20567|8|189|2005|49|28.43|31.55|20.19|0.00|989.31|1393.07|1545.95|39.57|0.00|989.31|1028.88|-403.76| +2452603|68793|313|94962|1452846|1483|20567|8|110|2005|3|80.31|118.85|89.13|0.00|267.39|240.93|356.55|5.34|0.00|267.39|272.73|26.46| +2452603|68793|17877|94962|1452846|1483|20567|8|205|2005|9|46.43|84.50|63.37|347.90|570.33|417.87|760.50|8.89|347.90|222.43|231.32|-195.44| +2452641|52441|396|47999|1148166|333|24500|8|86|2006|76|81.13|133.86|65.59|0.00|4984.84|6165.88|10173.36|448.63|0.00|4984.84|5433.47|-1181.04| +2452641|52441|16068|47999|1148166|333|24500|8|127|2006|93|71.58|83.03|68.08|0.00|6331.44|6656.94|7721.79|506.51|0.00|6331.44|6837.95|-325.50| +2452641|52441|2838|47999|1148166|333|24500|8|29|2006|65|5.42|7.69|0.76|28.15|49.40|352.30|499.85|0.63|28.15|21.25|21.88|-331.05| +2452641|52441|15573|47999|1148166|333|||254|2006||||9.52|0.00|||1539.09|7.99|0.00||207.91|| +2452641|52441|11085|47999|1148166|333|24500|8|227|2006|4|53.04|66.30|0.00|0.00|0.00|212.16|265.20|0.00|0.00|0.00|0.00|-212.16| +2452641|52441|14526|47999|1148166|333|24500|8|256|2006|73|1.55|2.17|1.82|0.00|132.86|113.15|158.41|11.95|0.00|132.86|144.81|19.71| +2452641|52441|4110|47999|1148166|333|24500|8|257|2006|74|31.61|42.98|33.09|0.00|2448.66|2339.14|3180.52|0.00|0.00|2448.66|2448.66|109.52| +2452641|52441|12378|47999|1148166|333|24500|8|99|2006|9|32.81|32.81|1.96|0.00|17.64|295.29|295.29|0.70|0.00|17.64|18.34|-277.65| +2450902|63939|1214|99396|608792|4588|32737|4|194|2007|33|33.23|55.82|0.55|0.00|18.15|1096.59|1842.06|0.72|0.00|18.15|18.87|-1078.44| +2450902|63939|14948|99396|608792|4588|32737|4|250|2007|57|87.14|123.73|22.27|939.34|1269.39|4966.98|7052.61|23.10|939.34|330.05|353.15|-4636.93| +2450902|63939|17239|99396|608792|4588|32737|4|251|2007|50|86.03|99.79|50.89|0.00|2544.50|4301.50|4989.50|50.89|0.00|2544.50|2595.39|-1757.00| +2450902|63939|3748|99396|608792|4588|32737|4|75|2007|95|55.56|85.00|12.75|0.00|1211.25|5278.20|8075.00|109.01|0.00|1211.25|1320.26|-4066.95| +2450902|63939|11593|99396|608792|4588|32737|4|106|2007|40|13.47|15.49|0.92|0.00|36.80|538.80|619.60|1.47|0.00|36.80|38.27|-502.00| +2450902|63939|1339|99396|608792|4588|32737|4|65|2007|75|72.86|101.27|73.92|0.00|5544.00|5464.50|7595.25|388.08|0.00|5544.00|5932.08|79.50| +2450902|63939|14923|99396|608792|4588|32737|4|71|2007|69|45.46|67.28|46.42|0.00|3202.98|3136.74|4642.32|64.05|0.00|3202.98|3267.03|66.24| +2450902|63939|12205|99396|608792|4588|32737|4|282|2007|73|40.58|68.98|52.42|0.00|3826.66|2962.34|5035.54|38.26|0.00|3826.66|3864.92|864.32| +2450902|63939|12170|99396|608792|4588|32737|4|209|2007|20|72.00|115.20|5.76|0.00|115.20|1440.00|2304.00|0.00|0.00|115.20|115.20|-1324.80| +2450902|63939|5137|99396|608792|4588|32737|4|234|2007|70|63.24|86.63|40.71|0.00|2849.70|4426.80|6064.10|56.99|0.00|2849.70|2906.69|-1577.10| +2450902|63939|3001|99396|608792|4588|32737|4|24|2007|1|81.06|147.52|97.36|0.00|97.36|81.06|147.52|8.76|0.00|97.36|106.12|16.30| +2450902|63939|9703|99396|608792|4588|32737|4|53|2007|18|21.38|30.57|23.84|0.00|429.12|384.84|550.26|0.00|0.00|429.12|429.12|44.28| +2451267|53168|14920|50128|744063|3951||10|96|2008|||||447.93|689.13||4923.36||447.93|||-2761.02| +2451267|53168|15896|50128|744063|3951|35886|10|263|2008|45|29.21|50.53|14.14|0.00|636.30|1314.45|2273.85|38.17|0.00|636.30|674.47|-678.15| +2451267|53168|2632|50128|744063|3951|35886|10|119|2008|24|44.37|82.97|75.50|0.00|1812.00|1064.88|1991.28|144.96|0.00|1812.00|1956.96|747.12| +2451267|53168|17276|50128|744063|3951|35886|10|178|2008|37|84.97|112.16|50.47|0.00|1867.39|3143.89|4149.92|0.00|0.00|1867.39|1867.39|-1276.50| +2451267|53168|15568|50128|744063|3951|35886|10|28|2008|70|33.80|60.16|15.04|0.00|1052.80|2366.00|4211.20|94.75|0.00|1052.80|1147.55|-1313.20| +2451267|53168|15506|50128|744063|3951|35886|10|99|2008|70|64.30|104.16|70.82|0.00|4957.40|4501.00|7291.20|396.59|0.00|4957.40|5353.99|456.40| +2451267|53168|6322|50128|744063|3951|35886|10|294|2008|84|24.60|43.29|35.49|0.00|2981.16|2066.40|3636.36|59.62|0.00|2981.16|3040.78|914.76| +2451267|53168|7585|50128|744063|3951|35886|10|46|2008|97|64.14|97.49|65.31|0.00|6335.07|6221.58|9456.53|443.45|0.00|6335.07|6778.52|113.49| +2451267|53168|5671|50128|744063|3951|35886|10|205|2008|87|80.78|147.82|141.90|0.00|12345.30|7027.86|12860.34|740.71|0.00|12345.30|13086.01|5317.44| +2451267|53168|15640|50128|744063|3951|35886|10|63|2008|91|98.23|183.69|23.87|0.00|2172.17|8938.93|16715.79|43.44|0.00|2172.17|2215.61|-6766.76| +2451267|53168|9193|50128|744063|3951|35886|10|247|2008|67|24.28|25.97|22.85|0.00|1530.95|1626.76|1739.99|91.85|0.00|1530.95|1622.80|-95.81| +2451267|53168|16442|50128|744063|3951|35886|10|298|2008|13|99.17|119.99|75.59|0.00|982.67|1289.21|1559.87|29.48|0.00|982.67|1012.15|-306.54| +2451267|53168|12682|50128|744063|3951|35886|10|3|2008|75|62.03|119.71|41.89|628.35|3141.75|4652.25|8978.25|201.07|628.35|2513.40|2714.47|-2138.85| +2451398|58637|7657|96898|779467|436|31005|8|156|2009|29|44.89|47.13|5.18|0.00|150.22|1301.81|1366.77|3.00|0.00|150.22|153.22|-1151.59| +2451398|58637|7306|96898|779467|436|31005|8|196|2009|96|3.11|5.72|3.03|0.00|290.88|298.56|549.12|11.63|0.00|290.88|302.51|-7.68| +2451398|58637|3112|96898|779467|436|31005|8|295|2009|70|68.60|112.50|69.75|0.00|4882.50|4802.00|7875.00|195.30|0.00|4882.50|5077.80|80.50| +2451398|58637|12271|96898|779467|436|31005|8|25|2009|95|50.43|62.02|48.37|0.00|4595.15|4790.85|5891.90|229.75|0.00|4595.15|4824.90|-195.70| +2451398|58637|5278|96898|779467|436|31005|8|14|2009|84|78.88|121.47|21.86|0.00|1836.24|6625.92|10203.48|36.72|0.00|1836.24|1872.96|-4789.68| +2451398|58637|14882|96898|779467|436|31005|8|170|2009|97|75.97|100.28|44.12|0.00|4279.64|7369.09|9727.16|42.79|0.00|4279.64|4322.43|-3089.45| +2451398|58637|5992|96898|779467|436|31005|8|30|2009|67|21.05|30.94|7.11|0.00|476.37|1410.35|2072.98|42.87|0.00|476.37|519.24|-933.98| +2451398|58637|6848|96898|779467|436|31005|8|40|2009|74|35.62|53.43|14.42|0.00|1067.08|2635.88|3953.82|0.00|0.00|1067.08|1067.08|-1568.80| +2451398|58637|12428|96898|779467|436|31005|8|225|2009|63|99.30|170.79|8.53|0.00|537.39|6255.90|10759.77|10.74|0.00|537.39|548.13|-5718.51| +2451539|73280|7172|71187|1259814|2488|2362|7|12|2010|99|84.14|107.69|104.45|0.00|10340.55|8329.86|10661.31|413.62|0.00|10340.55|10754.17|2010.69| +2451539|73280|4054|71187|1259814|2488|2362|7|184|2010|81|96.80|170.36|64.73|0.00|5243.13|7840.80|13799.16|262.15|0.00|5243.13|5505.28|-2597.67| +2451539|73280|16046|71187|1259814|2488|2362|7|104|2010|49|80.30|86.72|11.27|0.00|552.23|3934.70|4249.28|38.65|0.00|552.23|590.88|-3382.47| +2451539|73280|10111|71187|1259814|2488|2362|7|106|2010|83|7.31|13.88|11.24|298.53|932.92|606.73|1152.04|57.09|298.53|634.39|691.48|27.66| +2451539|73280|8395|71187|1259814|2488|2362|7|252|2010|23|68.12|81.06|80.24|0.00|1845.52|1566.76|1864.38|166.09|0.00|1845.52|2011.61|278.76| +2451539|73280|9968|71187|1259814|2488|2362|7|61|2010|72|13.24|22.77|12.06|0.00|868.32|953.28|1639.44|34.73|0.00|868.32|903.05|-84.96| +2451539|73280|4384|71187|1259814|2488|2362|7|188|2010|13|28.96|55.89|0.55|0.00|7.15|376.48|726.57|0.28|0.00|7.15|7.43|-369.33| +2451539|73280|6578|71187|1259814|2488|2362|7|71|2010|54|48.23|81.02|64.81|0.00|3499.74|2604.42|4375.08|34.99|0.00|3499.74|3534.73|895.32| +2451539|73280|11722|71187|1259814|2488|2362|7|201|2010|45|20.77|36.97|1.10|0.00|49.50|934.65|1663.65|3.96|0.00|49.50|53.46|-885.15| +2451539|73280|7789|71187|1259814|2488|2362|7|134|2010|4|28.61|55.50|38.29|0.00|153.16|114.44|222.00|4.59|0.00|153.16|157.75|38.72| +2451539|73280|1880|71187|1259814|2488|2362|7|114|2010|27|52.50|67.20|24.86|349.03|671.22|1417.50|1814.40|19.33|349.03|322.19|341.52|-1095.31| +2451539|73280|1642|71187|1259814|2488|2362|7|58|2010|6|46.19|50.80|32.51|0.00|195.06|277.14|304.80|1.95|0.00|195.06|197.01|-82.08| +2451539|73280|193|71187|1259814|2488|2362|7|240|2010|56|34.07|39.52|22.13|0.00|1239.28|1907.92|2213.12|86.74|0.00|1239.28|1326.02|-668.64| +|73280|5816|71187||2488|2362|||2010|||141.56||0.00||4819.20||0.00|0.00|0.00||| +2451539|73280|9242|71187|1259814|2488|2362|7|223|2010|72|66.83|102.24|36.80|1669.24|2649.60|4811.76|7361.28|78.42|1669.24|980.36|1058.78|-3831.40| +2451539|73280|1928|71187|1259814|2488|2362|7|266|2010|39|58.73|95.14|3.80|20.74|148.20|2290.47|3710.46|11.47|20.74|127.46|138.93|-2163.01| +2452036|67213|17717|67801|1236122|5243|6317|2|46|2011|58|40.05|51.66|4.13|9.58|239.54|2322.90|2996.28|13.79|9.58|229.96|243.75|-2092.94| +2452036|67213|1037|67801|1236122|5243|6317|2|250|2011|76|5.18|6.31|0.12|4.01|9.12|393.68|479.56|0.40|4.01|5.11|5.51|-388.57| +2452036|67213|5185|67801|1236122|5243|6317|2|223|2011|68|18.28|20.47|12.89|0.00|876.52|1243.04|1391.96|61.35|0.00|876.52|937.87|-366.52| +2452036|67213|635|67801|1236122|5243|6317|2|67|2011|2|44.69|63.45|39.33|0.00|78.66|89.38|126.90|0.78|0.00|78.66|79.44|-10.72| +2452036|67213|557|67801|1236122|5243|6317|2|11|2011|47|37.26|43.96|26.81|0.00|1260.07|1751.22|2066.12|88.20|0.00|1260.07|1348.27|-491.15| +2452036||12941|67801||5243||2||2011||||7.42|0.00|155.82||1198.68||0.00|||| +2452036|67213|12021|67801|1236122|5243|6317|2|269|2011|95|91.97|103.92|100.80|0.00|9576.00|8737.15|9872.40|766.08|0.00|9576.00|10342.08|838.85| +2452036|67213|9055|67801|1236122|5243|6317|2|147|2011|67|77.49|131.73|125.14|0.00|8384.38|5191.83|8825.91|167.68|0.00|8384.38|8552.06|3192.55| +2452036|67213|7271|67801|1236122|5243|6317|2|11|2011|7|63.33|123.49|79.03|0.00|553.21|443.31|864.43|11.06|0.00|553.21|564.27|109.90| +2451296|62193|14134|72127|1188842|5910|24625|7|261|2012|60|71.57|85.88|59.25|0.00|3555.00|4294.20|5152.80|248.85|0.00|3555.00|3803.85|-739.20| +2451296|62193|17677|72127|1188842|5910|24625|7|102|2012|30|92.63|118.56|5.92|0.00|177.60|2778.90|3556.80|0.00|0.00|177.60|177.60|-2601.30| +2451296|62193|5806|72127|1188842|5910|24625|7|151|2012|17|68.70|109.23|12.01|0.00|204.17|1167.90|1856.91|8.16|0.00|204.17|212.33|-963.73| +2451296|62193|10232|72127|1188842|5910|24625|7|195|2012|80|31.56|40.71|35.82|0.00|2865.60|2524.80|3256.80|171.93|0.00|2865.60|3037.53|340.80| +2451296|62193|15145|72127|1188842|5910|24625|7|250|2012|80|57.42|87.85|2.63|0.00|210.40|4593.60|7028.00|2.10|0.00|210.40|212.50|-4383.20| +2451296|62193|12793|72127|1188842|5910|24625|7|209|2012|18|65.89|75.77|40.15|0.00|722.70|1186.02|1363.86|57.81|0.00|722.70|780.51|-463.32| +2451296|62193|715|72127|1188842|5910|24625|7|9|2012|61|66.60|129.20|54.26|0.00|3309.86|4062.60|7881.20|99.29|0.00|3309.86|3409.15|-752.74| +2451296|62193|16090|72127|1188842|5910|24625|7|58|2012|34|63.75|80.32|43.37|0.00|1474.58|2167.50|2730.88|44.23|0.00|1474.58|1518.81|-692.92| +2451296|62193|1600|72127|1188842|5910|24625|7|260|2012|17|27.58|46.61|6.99|0.00|118.83|468.86|792.37|4.75|0.00|118.83|123.58|-350.03| +2451296|62193|15424|72127|1188842|5910|24625|7|162|2012|69|95.28|175.31|61.35|0.00|4233.15|6574.32|12096.39|211.65|0.00|4233.15|4444.80|-2341.17| +2451296|62193|15560|72127|1188842|5910|24625|7|46|2012|7|65.55|86.52|0.00|0.00|0.00|458.85|605.64|0.00|0.00|0.00|0.00|-458.85| +2451789|37491|9007|5580|225365|4350|12935|7|40|2013|67|4.30|7.09|5.53|0.00|370.51|288.10|475.03|14.82|0.00|370.51|385.33|82.41| +2451789|37491|14287|5580|225365|4350|12935|7|80|2013|40|45.59|45.59|29.17|0.00|1166.80|1823.60|1823.60|35.00|0.00|1166.80|1201.80|-656.80| +2451789|37491|14741|5580|225365|4350|12935|7|1|2013|33|61.51|64.58|9.68|0.00|319.44|2029.83|2131.14|12.77|0.00|319.44|332.21|-1710.39| +2451789|37491|3421|5580|225365|4350|12935|7|58|2013|49|78.40|116.81|44.38|0.00|2174.62|3841.60|5723.69|0.00|0.00|2174.62|2174.62|-1666.98| +2451789|37491|16603|5580|225365|4350|12935|7|42|2013|98|72.59|124.12|18.61|0.00|1823.78|7113.82|12163.76|54.71|0.00|1823.78|1878.49|-5290.04| +2451789|37491|2557|5580|225365|4350|12935|7|7|2013|50|73.63|113.39|11.33|0.00|566.50|3681.50|5669.50|39.65|0.00|566.50|606.15|-3115.00| +2451789|37491|3295|5580|225365|4350|12935|7|188|2013|39|83.53|86.87|33.87|0.00|1320.93|3257.67|3387.93|66.04|0.00|1320.93|1386.97|-1936.74| +2451789|37491|12917|5580|225365|4350|12935|7|216|2013|87|39.03|71.03|18.46|0.00|1606.02|3395.61|6179.61|32.12|0.00|1606.02|1638.14|-1789.59| +2451789|37491|3805|5580|225365|4350|12935|7|146|2013|49|75.11|111.16|105.60|724.41|5174.40|3680.39|5446.84|266.99|724.41|4449.99|4716.98|769.60| +2451519|42438|14155|78631|1718103|4328|16600|8|191|2014|49|12.85|20.17|3.02|0.00|147.98|629.65|988.33|8.87|0.00|147.98|156.85|-481.67| +2451519|42438|4462|78631|1718103|4328|16600|8|202|2014|34|11.50|17.02|5.10|152.59|173.40|391.00|578.68|0.41|152.59|20.81|21.22|-370.19| +2451519|42438|4201|78631|1718103|4328|16600|8|120|2014|30|9.59|14.86|3.26|0.00|97.80|287.70|445.80|6.84|0.00|97.80|104.64|-189.90| +|42438|4705|78631||||8|237|2014|||61.87||||914.60|||||29.03|-887.72| +|42438|6908|||4328||8||2014|37||||1177.04|2140.08|3194.21|||1177.04|||| +2451519|42438|841|78631||4328|16600|8|133|2014|91|82.29||31.13|0.00||7488.39|12879.23||0.00|2832.83||| +2451519|42438|5140|78631|1718103|4328|16600|8|102|2014|100|92.01|158.25|142.42|0.00|14242.00|9201.00|15825.00|142.42|0.00|14242.00|14384.42|5041.00| +2451519|42438|8900|78631|1718103|4328|16600|8|97|2014|63|76.82|115.23|36.87|0.00|2322.81|4839.66|7259.49|139.36|0.00|2322.81|2462.17|-2516.85| +2451519|42438|10258|78631|1718103|4328|16600|8|221|2014|66|99.21|125.99|91.97|0.00|6070.02|6547.86|8315.34|485.60|0.00|6070.02|6555.62|-477.84| +2451519|42438|17168|78631|1718103|4328|16600|8|83|2014|30|82.16|132.27|126.97|1637.91|3809.10|2464.80|3968.10|151.98|1637.91|2171.19|2323.17|-293.61| +|42438|334|78631||||8||2014|73||165.79|31.50|0.00|2299.50|||91.98|0.00||2391.48|| +2451519|42438|9325|78631|1718103|4328|16600|8|97|2014|59|89.84|133.86|101.73|0.00|6002.07|5300.56|7897.74|360.12|0.00|6002.07|6362.19|701.51| +2451519|42438|9766|78631|1718103|4328|16600|8|44|2014|91|44.50|51.62|33.03|0.00|3005.73|4049.50|4697.42|150.28|0.00|3005.73|3156.01|-1043.77| +2451519|42438|12512|78631|1718103|4328|16600|8|240|2014|87|79.94|152.68|126.72|0.00|11024.64|6954.78|13283.16|440.98|0.00|11024.64|11465.62|4069.86| +2451519|42438|1717|78631|1718103|4328|16600|8|122|2014|35|53.18|84.55|10.99|0.00|384.65|1861.30|2959.25|11.53|0.00|384.65|396.18|-1476.65| +2451519|42438|6193|78631|1718103|4328|16600|8|150|2014|95|87.87|110.71|38.74|0.00|3680.30|8347.65|10517.45|331.22|0.00|3680.30|4011.52|-4667.35| +2452074|38614|15125|50883|1188960|2237|45914|1|179|2015|86|17.24|18.10|4.34|0.00|373.24|1482.64|1556.60|26.12|0.00|373.24|399.36|-1109.40| +2452074|38614|371|50883|1188960|2237|45914|1|104|2015|87|4.71|6.17|5.42|0.00|471.54|409.77|536.79|9.43|0.00|471.54|480.97|61.77| +2452074|38614|8947|50883|1188960|2237|45914|1|299|2015|31|54.76|107.87|81.98|0.00|2541.38|1697.56|3343.97|177.89|0.00|2541.38|2719.27|843.82| +2452074|38614|15387|50883|1188960|2237|45914|1|96|2015|86|61.23|118.17|116.98|0.00|10060.28|5265.78|10162.62|402.41|0.00|10060.28|10462.69|4794.50| +2452074|38614|12245|50883|1188960|2237|45914|1|91|2015|76|14.72|18.69|4.67|0.00|354.92|1118.72|1420.44|7.09|0.00|354.92|362.01|-763.80| +2452074|38614|7737|50883|1188960|2237|45914|1|282|2015|14|22.55|25.93|14.52|20.32|203.28|315.70|363.02|14.63|20.32|182.96|197.59|-132.74| +2452074|38614|16127|50883|1188960|2237|45914|1|95|2015|3|50.66|64.84|18.15|0.00|54.45|151.98|194.52|1.08|0.00|54.45|55.53|-97.53| +2452074|38614|7027|50883|1188960|2237|45914|1|210|2015|98|3.55|5.43|1.08|0.00|105.84|347.90|532.14|0.00|0.00|105.84|105.84|-242.06| +2452074|38614|7615|50883|1188960|2237|45914|1|288|2015|91|79.21|126.73|92.51|0.00|8418.41|7208.11|11532.43|589.28|0.00|8418.41|9007.69|1210.30| +2452074|38614|14913|50883|1188960|2237|45914|1|260|2015|44|67.19|80.62|20.15|0.00|886.60|2956.36|3547.28|79.79|0.00|886.60|966.39|-2069.76| +2451167|33756|6475|54570|1344736|4855|22221|7|22|2016|77|65.80|131.60|13.16|0.00|1013.32|5066.60|10133.20|20.26|0.00|1013.32|1033.58|-4053.28| +2451167|33756|13118|54570|1344736|4855|22221|7|201|2016|23|13.43|18.80|0.56|3.47|12.88|308.89|432.40|0.65|3.47|9.41|10.06|-299.48| +2451167|33756|6014|54570|1344736|4855|22221|7|245|2016|25|71.56|76.56|66.60|0.00|1665.00|1789.00|1914.00|49.95|0.00|1665.00|1714.95|-124.00| +2451167|33756|4285|54570|1344736|4855|22221|7|157|2016|40|25.06|26.56|7.17|0.00|286.80|1002.40|1062.40|22.94|0.00|286.80|309.74|-715.60| +2451167|33756|12032|54570|1344736|4855|22221|7|298|2016|23|66.34|84.91|15.28|0.00|351.44|1525.82|1952.93|0.00|0.00|351.44|351.44|-1174.38| +2451167|33756|8128|54570|1344736|4855|22221|7|283|2016|4|33.43|38.77|24.42|0.00|97.68|133.72|155.08|6.83|0.00|97.68|104.51|-36.04| +2451167|33756|3458|54570|1344736|4855|22221|7|135|2016|25|20.93|40.60|21.92|0.00|548.00|523.25|1015.00|43.84|0.00|548.00|591.84|24.75| +2451167|33756|10946|54570|1344736|4855|22221|7|25|2016|42|30.85|32.70|20.27|144.72|851.34|1295.70|1373.40|49.46|144.72|706.62|756.08|-589.08| +||403||1344736||22221||191|2016|13|||123.87||1610.31|1268.80||144.92||1610.31||341.51| +2451167|33756|2575|54570|1344736|4855|22221|7|132|2016|25|19.17|36.99|22.19|0.00|554.75|479.25|924.75|0.00|0.00|554.75|554.75|75.50| +2451167|33756|4658|54570|1344736|4855|22221|7|178|2016|51|24.04|29.80|4.76|0.00|242.76|1226.04|1519.80|16.99|0.00|242.76|259.75|-983.28| +2451167|33756|9721|54570|1344736|4855|22221|7|235|2016|100|73.52|99.98|25.99|0.00|2599.00|7352.00|9998.00|207.92|0.00|2599.00|2806.92|-4753.00| +2451167|33756|8806|54570|1344736|4855|22221|7|180|2016|70|67.82|114.61|60.74|0.00|4251.80|4747.40|8022.70|212.59|0.00|4251.80|4464.39|-495.60| +2451167|33756|13711|54570|1344736|4855|22221|7|165|2016|45|43.01|72.68|36.34|0.00|1635.30|1935.45|3270.60|98.11|0.00|1635.30|1733.41|-300.15| +2452363|47244|807|78108|513822|3936|16050|4|67|2017|75|96.20|130.83|27.47|0.00|2060.25|7215.00|9812.25|20.60|0.00|2060.25|2080.85|-5154.75| +2452363|47244|6717|78108|513822|3936|16050|4|254|2017|11|48.27|85.43|22.21|0.00|244.31|530.97|939.73|19.54|0.00|244.31|263.85|-286.66| +2452363|47244|10363|78108|513822|3936|16050|4|10|2017|39|6.73|8.47|1.35|0.00|52.65|262.47|330.33|2.10|0.00|52.65|54.75|-209.82| +2452363|47244|918|78108|513822|3936|16050|4|211|2017|8|82.60|142.89|34.29|0.00|274.32|660.80|1143.12|24.68|0.00|274.32|299.00|-386.48| +2452363|47244|11439|78108|513822|3936|16050|4|85|2017|1|88.60|127.58|2.55|0.00|2.55|88.60|127.58|0.17|0.00|2.55|2.72|-86.05| +2452363|47244|13992|78108|513822|3936|16050|4|57|2017|7|43.51|68.31|49.18|0.00|344.26|304.57|478.17|3.44|0.00|344.26|347.70|39.69| +2452363|47244|3552|78108|513822|3936|16050|4|274|2017|81|61.09|109.96|97.86|0.00|7926.66|4948.29|8906.76|396.33|0.00|7926.66|8322.99|2978.37| +2452363|47244|301|78108|513822|3936|16050|4|230|2017|34|12.29|17.82|8.73|0.00|296.82|417.86|605.88|8.90|0.00|296.82|305.72|-121.04| +2452363|47244|9361|78108|513822|3936|16050|4|72|2017|75|14.05|23.04|21.42|0.00|1606.50|1053.75|1728.00|48.19|0.00|1606.50|1654.69|552.75| +2452363|47244|12093|78108|513822|3936|16050|4|217|2017|75|60.93|84.69|53.35|0.00|4001.25|4569.75|6351.75|40.01|0.00|4001.25|4041.26|-568.50| +2452363|47244|17083|78108|513822|3936|16050|4|50|2017|33|71.16|104.60|84.72|0.00|2795.76|2348.28|3451.80|167.74|0.00|2795.76|2963.50|447.48| +2452363|47244|13932|78108|513822|3936|16050|4|143|2017|11|87.74|120.20|33.65|0.00|370.15|965.14|1322.20|3.70|0.00|370.15|373.85|-594.99| +2451974|61761|7087|71375|1360292|4409|9373|10|169|2018|79|49.50|92.56|0.00|0.00|0.00|3910.50|7312.24|0.00|0.00|0.00|0.00|-3910.50| +2451974|61761|9347|71375|1360292|4409|9373|10|42|2018|28|4.16|5.57|0.27|0.00|7.56|116.48|155.96|0.45|0.00|7.56|8.01|-108.92| +2451974|61761|6773|71375|1360292|4409|9373|10|65|2018|42|49.47|61.83|12.98|0.00|545.16|2077.74|2596.86|27.25|0.00|545.16|572.41|-1532.58| +2451974|61761|9095|71375|1360292|4409|9373|10|206|2018|51|78.20|151.70|43.99|0.00|2243.49|3988.20|7736.70|22.43|0.00|2243.49|2265.92|-1744.71| +2451974|61761|1095|71375|1360292|4409|9373|10|193|2018|52|35.10|40.71|16.28|0.00|846.56|1825.20|2116.92|59.25|0.00|846.56|905.81|-978.64| +2451974|61761|3377|71375|1360292|4409|9373|10|177|2018|12|94.27|129.14|3.87|13.00|46.44|1131.24|1549.68|2.00|13.00|33.44|35.44|-1097.80| +2451974|61761|4883|71375|1360292|4409|9373|10|165|2018|69|51.27|54.34|0.54|16.76|37.26|3537.63|3749.46|0.41|16.76|20.50|20.91|-3517.13| +2451974|61761|8663|71375|1360292|4409|9373|10|111|2018|8|28.58|30.86|25.61|0.00|204.88|228.64|246.88|12.29|0.00|204.88|217.17|-23.76| +2451974|61761|17539|71375|1360292|4409|9373|10|156|2018|32|49.22|66.93|2.00|0.00|64.00|1575.04|2141.76|5.12|0.00|64.00|69.12|-1511.04| +2451974|61761|16331|71375|1360292|4409|9373|10|297|2018|98|40.98|68.43|62.27|0.00|6102.46|4016.04|6706.14|122.04|0.00|6102.46|6224.50|2086.42| +2451974|61761|7815|71375|1360292|4409|9373|10|128|2018|67|86.08|168.71|3.37|0.00|225.79|5767.36|11303.57|15.80|0.00|225.79|241.59|-5541.57| +2451974|61761|10559|71375|1360292|4409|9373|10|286|2018|96|35.13|42.50|37.40|0.00|3590.40|3372.48|4080.00|287.23|0.00|3590.40|3877.63|217.92| +2451974|61761|15257|71375|1360292|4409|9373|10|58|2018|36|81.81|125.16|28.78|0.00|1036.08|2945.16|4505.76|82.88|0.00|1036.08|1118.96|-1909.08| +2451974|61761|8787|71375|1360292|4409|9373|10|97|2018|60|54.30|77.64|6.98|0.00|418.80|3258.00|4658.40|12.56|0.00|418.80|431.36|-2839.20| +2452223|60853|8821|80540|1065061|3182|11350|8|110|2019|46|55.25|70.16|68.75|0.00|3162.50|2541.50|3227.36|189.75|0.00|3162.50|3352.25|621.00| +2452223|60853|3243|80540|1065061|3182|11350|8|60|2019|12|99.76|193.53|87.08|0.00|1044.96|1197.12|2322.36|83.59|0.00|1044.96|1128.55|-152.16| +2452223|60853|17777|80540|1065061|3182|11350|8|176|2019|9|12.04|14.44|2.74|0.00|24.66|108.36|129.96|0.98|0.00|24.66|25.64|-83.70| +2452223|60853|8123|80540|1065061|3182|11350|8|46|2019|89|24.13|27.50|2.47|0.00|219.83|2147.57|2447.50|15.38|0.00|219.83|235.21|-1927.74| +2452223|60853|17741|80540|1065061|3182|11350|8|270|2019|87|42.24|74.76|13.45|0.00|1170.15|3674.88|6504.12|70.20|0.00|1170.15|1240.35|-2504.73| +2452223|60853|4327|80540|1065061|3182|11350|8|274|2019|84|43.23|43.66|33.61|0.00|2823.24|3631.32|3667.44|84.69|0.00|2823.24|2907.93|-808.08| +2452223|60853|4347|80540|1065061|3182|11350|8|31|2019|43|82.14|113.35|104.28|0.00|4484.04|3532.02|4874.05|358.72|0.00|4484.04|4842.76|952.02| +2452223|60853|17873|80540|1065061|3182|11350|8|55|2019|96|86.48|157.39|157.39|0.00|15109.44|8302.08|15109.44|604.37|0.00|15109.44|15713.81|6807.36| +2452223|60853|2401|80540|1065061|3182|11350|8|143|2019|29|32.34|60.79|13.37|0.00|387.73|937.86|1762.91|19.38|0.00|387.73|407.11|-550.13| +2452223|60853|3867|80540|1065061|3182|11350|8|124|2019|81|71.03|78.13|17.18|0.00|1391.58|5753.43|6328.53|55.66|0.00|1391.58|1447.24|-4361.85| +2452223|60853|14295|80540|1065061|3182|11350|8|62|2019|96|57.78|109.78|14.27|0.00|1369.92|5546.88|10538.88|109.59|0.00|1369.92|1479.51|-4176.96| +2452429|47699|13731|||5453|||182|2020|||130.93|7.85||314.00|||28.26||314.00||| +||11781|44783|1718718|5453||8|263|2020|13|7.12|13.81|2.07|0.00|||||0.00||26.91|-65.65| +2452429|47699|11323|44783|1718718|5453|4988|8|135|2020|56|84.50|150.41|33.09|0.00|1853.04|4732.00|8422.96|111.18|0.00|1853.04|1964.22|-2878.96| +2452429|47699|15735|44783|1718718|5453|4988|8|160|2020|28|49.93|68.90|25.49|0.00|713.72|1398.04|1929.20|28.54|0.00|713.72|742.26|-684.32| +2452429|47699|1356|44783|1718718|5453|4988|8|182|2020|63|29.49|44.23|10.17|0.00|640.71|1857.87|2786.49|6.40|0.00|640.71|647.11|-1217.16| +2452429|47699|9277|44783|1718718|5453|4988|8|27|2020|33|97.20|159.40|133.89|0.00|4418.37|3207.60|5260.20|309.28|0.00|4418.37|4727.65|1210.77| +2452429|47699|12795|44783|1718718|5453|4988|8|84|2020|4|2.82|4.99|1.89|0.00|7.56|11.28|19.96|0.37|0.00|7.56|7.93|-3.72| +2452429|47699|579|44783|1718718|5453|4988|8|154|2020|75|22.68|39.91|19.15|0.00|1436.25|1701.00|2993.25|100.53|0.00|1436.25|1536.78|-264.75| +2452429|47699|55|44783|1718718|5453|4988|8|285|2020|40|28.97|51.56|5.15|0.00|206.00|1158.80|2062.40|10.30|0.00|206.00|216.30|-952.80| +2452517|72684|3765|53888|1111589|5290|34991|2|121|2021|9|19.79|34.03|12.93|0.00|116.37|178.11|306.27|1.16|0.00|116.37|117.53|-61.74| +2452517|72684|6978|53888|1111589|5290|34991|2|195|2021|67|79.73|112.41|102.29|0.00|6853.43|5341.91|7531.47|548.27|0.00|6853.43|7401.70|1511.52| +|72684|11712|53888|1111589|||2||2021||92.88|178.32|135.52||||5349.60|365.90|||4431.50|| +2452517|72684|6013|53888|1111589|5290|34991|2|135|2021|27|32.40|43.41|35.59|0.00|960.93|874.80|1172.07|57.65|0.00|960.93|1018.58|86.13| +|72684|897|||5290||2||2021|9|7.52|7.74||||||1.23|||42.27|| +2452517|72684|10185|53888|1111589|5290|34991|2|92|2021|84|81.71|160.15|131.32|0.00|11030.88|6863.64|13452.60|882.47|0.00|11030.88|11913.35|4167.24| +2452517|72684|10284|53888|1111589|5290|34991|2|233|2021|70|97.27|194.54|29.18|326.81|2042.60|6808.90|13617.80|0.00|326.81|1715.79|1715.79|-5093.11| +2452517|72684|17965|53888|1111589|5290|34991|2|63|2021|25|5.18|5.64|3.60|0.00|90.00|129.50|141.00|2.70|0.00|90.00|92.70|-39.50| +2452517|72684|12799|53888|1111589|5290|34991|2|229|2021|1|45.52|64.18|1.92|0.00|1.92|45.52|64.18|0.05|0.00|1.92|1.97|-43.60| +2452517|72684|8067|53888|1111589|5290|34991|2|182|2021|23|90.95|179.17|84.20|0.00|1936.60|2091.85|4120.91|77.46|0.00|1936.60|2014.06|-155.25| +2452517|72684|16239|53888|1111589|5290|34991|2|289|2021|68|62.66|73.31|37.38|0.00|2541.84|4260.88|4985.08|50.83|0.00|2541.84|2592.67|-1719.04| +2452517|72684|17739|53888|1111589|5290|34991|2|258|2021|44|95.91|124.68|3.74|0.00|164.56|4220.04|5485.92|3.29|0.00|164.56|167.85|-4055.48| +2452517|72684|9882|53888|1111589|5290|34991|2|26|2021|68|13.02|15.75|14.01|0.00|952.68|885.36|1071.00|9.52|0.00|952.68|962.20|67.32| +2452596|45699|1609|9699|579444|1989|11734|1|139|2022|41|40.60|52.37|39.80|0.00|1631.80|1664.60|2147.17|81.59|0.00|1631.80|1713.39|-32.80| +2452596|45699|9879|9699|579444|1989|11734|1|36|2022|42|35.37|53.76|35.48|0.00|1490.16|1485.54|2257.92|119.21|0.00|1490.16|1609.37|4.62| +2452596|45699|54|9699|579444|1989|11734|1|190|2022|47|44.50|62.30|52.95|0.00|2488.65|2091.50|2928.10|223.97|0.00|2488.65|2712.62|397.15| +2452596|45699|14725|9699|579444|1989|11734|1|143|2022|8|98.48|139.84|137.04|0.00|1096.32|787.84|1118.72|21.92|0.00|1096.32|1118.24|308.48| +2452596|45699|4485||579444|1989|11734|1|136|2022|14|47.77|82.64|76.85|0.00|||1156.96|64.55|0.00|1075.90||| +2452596|45699|3073|9699|579444|1989|11734|1|273|2022|48|31.20|54.60|50.77|0.00|2436.96|1497.60|2620.80|170.58|0.00|2436.96|2607.54|939.36| +2452596|45699|12537|9699|579444|1989|11734|1|160|2022|95|35.84|54.47|4.35|0.00|413.25|3404.80|5174.65|24.79|0.00|413.25|438.04|-2991.55| +2452596|45699|14442|9699|579444|1989|11734|1|186|2022|43|52.28|88.35|59.19|0.00|2545.17|2248.04|3799.05|229.06|0.00|2545.17|2774.23|297.13| +2452596|45699|7783|9699|579444|1989|11734|1|183|2022|89|83.47|106.00|22.26|0.00|1981.14|7428.83|9434.00|0.00|0.00|1981.14|1981.14|-5447.69| +2452596|45699|4447|9699|579444|1989|11734|1|257|2022|91|39.92|54.29|49.40|0.00|4495.40|3632.72|4940.39|0.00|0.00|4495.40|4495.40|862.68| +2452596|45699|14175|9699|579444|1989|11734|1|284|2022|95|90.54|105.02|99.76|0.00|9477.20|8601.30|9976.90|758.17|0.00|9477.20|10235.37|875.90| +2452596|45699|8503|9699|579444|1989|11734|1|112|2022|65|57.85|109.91|31.87|0.00|2071.55|3760.25|7144.15|145.00|0.00|2071.55|2216.55|-1688.70| +2451128|35974|1658|28555|578822|7042|44509|7|88|2023|13|62.14|108.12|8.64|0.00|112.32|807.82|1405.56|1.12|0.00|112.32|113.44|-695.50| +2451128|35974|8692|28555|578822|7042|44509|7|39|2023|46|46.04|70.44|35.92|0.00|1652.32|2117.84|3240.24|33.04|0.00|1652.32|1685.36|-465.52| +2451128|35974|5624|28555|578822|7042|44509|7|118|2023|44|74.42|128.00|88.32|932.65|3886.08|3274.48|5632.00|118.13|932.65|2953.43|3071.56|-321.05| +|35974|3208|28555|578822|7042|44509|7|237|2023|75|32.80|37.39||||||7.82|||790.15|| +2451128|35974|10705|28555|578822|7042|44509|7|255|2023|43|38.28|62.77|49.58|511.66|2131.94|1646.04|2699.11|16.20|511.66|1620.28|1636.48|-25.76| +2451128|35974|15262|28555|578822|7042|44509|7|281|2023|72|80.23|156.44|154.87|0.00|11150.64|5776.56|11263.68|223.01|0.00|11150.64|11373.65|5374.08| +2451128|35974|8738|28555|578822|7042|44509|7|149|2023|96|15.27|18.62|10.61|0.00|1018.56|1465.92|1787.52|10.18|0.00|1018.56|1028.74|-447.36| +2451128|35974|6277|28555|578822|7042|44509|7|253|2023|54|2.68|5.33|4.90|0.00|264.60|144.72|287.82|18.52|0.00|264.60|283.12|119.88| +|35974|10714|28555|578822|7042||||2023|||102.51|87.13|77.54|87.13|||0.09|77.54|9.59|9.68|-71.13| +2451128|35974|16970|28555|578822|7042|44509|7|110|2023|48|76.98|111.62|80.36|0.00|3857.28|3695.04|5357.76|308.58|0.00|3857.28|4165.86|162.24| +2451128|35974|10928|28555|578822|7042|44509|7|218|2023|30|39.32|76.28|37.37|0.00|1121.10|1179.60|2288.40|67.26|0.00|1121.10|1188.36|-58.50| +2451128|35974|2996|28555|578822|7042|44509|7|202|2023|47|84.20|168.40|104.40|0.00|4906.80|3957.40|7914.80|441.61|0.00|4906.80|5348.41|949.40| +2451128|35974|13171|28555|578822|7042|44509|7|234|2023|60|81.34|83.78|7.54|0.00|452.40|4880.40|5026.80|9.04|0.00|452.40|461.44|-4428.00| +2451128|35974|7282|28555|578822|7042|44509|7|59|2023|63|64.71|129.42|23.29|924.38|1467.27|4076.73|8153.46|38.00|924.38|542.89|580.89|-3533.84| +2451128|35974|15337|28555|578822|7042|44509|7|186|2023|100|59.40|94.44|39.66|1388.10|3966.00|5940.00|9444.00|103.11|1388.10|2577.90|2681.01|-3362.10| +2451128|35974|17125|28555|578822|7042|44509|7|134|2023|41|1.68|1.88|1.59|0.00|65.19|68.88|77.08|5.86|0.00|65.19|71.05|-3.69| +2451632||7733||391175||3377|1||2024|69|29.91|33.20||0.00|1145.40|2063.79|||0.00|||-918.39| +2451632||16493||391175|1876||1|66|2024|||99.05|52.49|0.00|2099.60||3962.00||0.00|2099.60|2204.58|-857.20| +2451632|37641|12188|64474|391175|1876|3377|1|261|2024|91|41.08|45.59|31.00|0.00|2821.00|3738.28|4148.69|169.26|0.00|2821.00|2990.26|-917.28| +2451632|37641|13016|64474|391175|1876|3377|1|283|2024|73|90.92|148.19|17.78|0.00|1297.94|6637.16|10817.87|51.91|0.00|1297.94|1349.85|-5339.22| +2451632|37641|206|64474|391175|1876|3377|1|44|2024|21|77.05|137.14|58.97|0.00|1238.37|1618.05|2879.94|0.00|0.00|1238.37|1238.37|-379.68| +2451632|37641|5177|64474|391175|1876|3377|1|93|2024|5|7.78|12.05|8.31|0.00|41.55|38.90|60.25|2.49|0.00|41.55|44.04|2.65| +2451632|37641|9590|64474|391175|1876|3377|1|194|2024|69|40.32|75.80|68.22|0.00|4707.18|2782.08|5230.20|329.50|0.00|4707.18|5036.68|1925.10| +2451632|37641|17359|64474|391175|1876|3377|1|143|2024|30|54.15|70.39|42.93|0.00|1287.90|1624.50|2111.70|38.63|0.00|1287.90|1326.53|-336.60| +2451632|37641|17279|64474|391175|1876|3377|1|243|2024|16|44.93|75.03|43.51|97.46|696.16|718.88|1200.48|5.98|97.46|598.70|604.68|-120.18| +2451632|37641|15547|64474|391175|1876|3377|1|128|2024|65|25.16|45.53|25.49|0.00|1656.85|1635.40|2959.45|99.41|0.00|1656.85|1756.26|21.45| +2451293|44530|16100|20053|399981|5862|45390|2|52|2025|74|47.43|73.51|62.48|0.00|4623.52|3509.82|5439.74|369.88|0.00|4623.52|4993.40|1113.70| +2451293|44530|2266|20053|399981|5862|45390|2|116|2025|27|33.58|55.40|28.25|0.00|762.75|906.66|1495.80|15.25|0.00|762.75|778.00|-143.91| +2451293|44530|16078|20053|399981|5862|45390|2|227|2025|13|84.56|168.27|149.76|0.00|1946.88|1099.28|2187.51|38.93|0.00|1946.88|1985.81|847.60| +2451293|44530|15662|20053|399981|5862|45390|2|256|2025|98|80.98|106.89|43.82|0.00|4294.36|7936.04|10475.22|42.94|0.00|4294.36|4337.30|-3641.68| +2451293|44530|13405|20053|399981|5862|45390|2|299|2025|22|11.02|20.71|15.32|0.00|337.04|242.44|455.62|0.00|0.00|337.04|337.04|94.60| +2451293|44530|11524|20053|399981|5862|45390|2|166|2025|44|2.72|4.10|1.31|0.00|57.64|119.68|180.40|0.00|0.00|57.64|57.64|-62.04| +2451293|44530|17614|20053|399981|5862|45390|2|102|2025|19|97.51|185.26|66.69|0.00|1267.11|1852.69|3519.94|88.69|0.00|1267.11|1355.80|-585.58| +2451293|44530|1034|20053|399981|5862|45390|2|284|2025|65|72.88|124.62|0.00|0.00|0.00|4737.20|8100.30|0.00|0.00|0.00|0.00|-4737.20| +2451293|44530|4768|20053|399981|5862|45390|2|260|2025|42|96.66|171.08|85.54|0.00|3592.68|4059.72|7185.36|143.70|0.00|3592.68|3736.38|-467.04| +2451293|44530|502|20053|399981|5862|45390|2|119|2025|63|13.30|13.56|0.54|0.00|34.02|837.90|854.28|3.06|0.00|34.02|37.08|-803.88| +2451293|44530|1357|20053|399981|5862|45390|2|49|2025|83|31.04|52.14|39.62|0.00|3288.46|2576.32|4327.62|263.07|0.00|3288.46|3551.53|712.14| +2452267|29343|11689|26569|438145|6074|48850|1|282|2026|8|7.58|7.95|2.70|0.00|21.60|60.64|63.60|0.86|0.00|21.60|22.46|-39.04| +2452267|29343|11895|26569|438145|6074|48850|1|30|2026|66|51.85|66.36|27.87|0.00|1839.42|3422.10|4379.76|110.36|0.00|1839.42|1949.78|-1582.68| +2452267|29343|15039|26569|438145|6074|48850|1|111|2026|8|16.28|31.09|22.69|0.00|181.52|130.24|248.72|14.52|0.00|181.52|196.04|51.28| +2452267|29343|5721|26569|438145|6074|48850|1|280|2026|75|89.20|133.80|9.36|175.50|702.00|6690.00|10035.00|47.38|175.50|526.50|573.88|-6163.50| +2452267|29343|5385|26569|438145|6074|48850|1|145|2026|28|32.43|41.83|13.80|0.00|386.40|908.04|1171.24|34.77|0.00|386.40|421.17|-521.64| +2452267|29343|12851|26569|438145|6074|48850|1|111|2026|12|23.42|24.12|0.96|0.00|11.52|281.04|289.44|0.00|0.00|11.52|11.52|-269.52| +2452267|29343|6647|26569|438145|6074|48850|1|286|2026|6|27.23|38.93|15.18|0.00|91.08|163.38|233.58|4.55|0.00|91.08|95.63|-72.30| +2452267|29343|5053|26569|438145|6074|48850|1|97|2026|33|46.75|65.91|40.20|0.00|1326.60|1542.75|2175.03|106.12|0.00|1326.60|1432.72|-216.15| +2452267|29343|7337|26569|438145|6074|48850|1|263|2026|86|68.57|78.85|59.92|0.00|5153.12|5897.02|6781.10|206.12|0.00|5153.12|5359.24|-743.90| +2452267|29343|1989|26569|438145|6074|48850|1|34|2026|95|93.16|126.69|5.06|0.00|480.70|8850.20|12035.55|14.42|0.00|480.70|495.12|-8369.50| +2452267|29343|9679|26569|438145|6074|48850|1|34|2026|53|41.03|41.44|32.73|0.00|1734.69|2174.59|2196.32|52.04|0.00|1734.69|1786.73|-439.90| +2452267|29343|653|26569|438145|6074|48850|1|61|2026|45|89.12|119.42|81.20|0.00|3654.00|4010.40|5373.90|109.62|0.00|3654.00|3763.62|-356.40| +2452267|29343|8907|26569|438145|6074|48850|1|239|2026|93|94.42|101.02|61.62|0.00|5730.66|8781.06|9394.86|401.14|0.00|5730.66|6131.80|-3050.40| +2452267|29343|11831|26569|438145||48850|1|116|2026||90.94|||0.00|||8533.00||0.00|3839.16||| +2452267|29343|17157|26569|438145|6074|48850|1|102|2026|23|52.91|77.77|49.77|0.00|1144.71|1216.93|1788.71|0.00|0.00|1144.71|1144.71|-72.22| +2452267|29343|8961|26569|438145|6074|48850|1|45|2026|50|60.07|84.09|10.93|0.00|546.50|3003.50|4204.50|21.86|0.00|546.50|568.36|-2457.00| +2452245|33778|9165|48844|1404193|1142|1291|1|41|2027|73|94.15|159.11|101.83|0.00|7433.59|6872.95|11615.03|446.01|0.00|7433.59|7879.60|560.64| +2452245|33778|1435|48844|1404193|1142|1291|1|10|2027|56|46.24|88.78|25.74|0.00|1441.44|2589.44|4971.68|86.48|0.00|1441.44|1527.92|-1148.00| +2452245|33778|4133|48844|1404193|1142|1291|1|164|2027|43|10.92|14.63|4.38|0.00|188.34|469.56|629.09|3.76|0.00|188.34|192.10|-281.22| +2452245|33778|10473|48844|1404193|1142|1291|1|254|2027|89|22.53|27.93|13.12|0.00|1167.68|2005.17|2485.77|23.35|0.00|1167.68|1191.03|-837.49| +2452245|33778|10599|48844|1404193|1142|1291|1|75|2027|47|85.53|132.57|39.77|0.00|1869.19|4019.91|6230.79|93.45|0.00|1869.19|1962.64|-2150.72| +2452245|33778|3975|48844|1404193|1142|1291|1|261|2027|7|20.89|36.55|12.06|0.00|84.42|146.23|255.85|0.00|0.00|84.42|84.42|-61.81| +2452245|33778|15493|48844|1404193|1142|1291|1|39|2027|51|36.26|51.48|25.22|0.00|1286.22|1849.26|2625.48|115.75|0.00|1286.22|1401.97|-563.04| +2452245|33778|5619|48844|1404193|1142|1291|1|115|2027|25|54.75|54.75|30.11|0.00|752.75|1368.75|1368.75|52.69|0.00|752.75|805.44|-616.00| +2452245|33778|8021|48844|1404193|1142|1291|1|32|2027|26|94.17|125.24|43.83|0.00|1139.58|2448.42|3256.24|11.39|0.00|1139.58|1150.97|-1308.84| +2452245|33778|17021|48844|1404193|1142|1291|1|36|2027|2|96.32|168.56|79.22|129.92|158.44|192.64|337.12|2.28|129.92|28.52|30.80|-164.12| +2452245|33778|1833|48844|1404193|1142|1291|1|297|2027|42|49.55|90.18|73.94|0.00|3105.48|2081.10|3787.56|155.27|0.00|3105.48|3260.75|1024.38| +2452245|33778|14835|48844|1404193|1142|1291|1|266|2027|45|15.79|30.15|26.53|811.81|1193.85|710.55|1356.75|0.00|811.81|382.04|382.04|-328.51| +2451772||9659||1071821|496||||2028|||90.26|56.86|0.00|5003.68|4843.52||350.25|0.00|5003.68||| +2451772|48071|14329|82699|1071821|496|18771|1|112|2028|31|48.41|96.82|90.04|0.00|2791.24|1500.71|3001.42|195.38|0.00|2791.24|2986.62|1290.53| +2451772|48071|11471|82699|1071821|496|18771|1|36|2028|21|98.07|147.10|144.15|0.00|3027.15|2059.47|3089.10|181.62|0.00|3027.15|3208.77|967.68| +2451772|48071|215|82699|1071821|496|18771|1|19|2028|20|24.92|43.36|27.31|0.00|546.20|498.40|867.20|21.84|0.00|546.20|568.04|47.80| +2451772|48071|5977|82699|1071821|496|18771|1|89|2028|96|69.85|135.50|120.59|0.00|11576.64|6705.60|13008.00|115.76|0.00|11576.64|11692.40|4871.04| +2451772|48071|15967|82699|1071821|496|18771|1|140|2028|19|75.23|125.63|30.15|0.00|572.85|1429.37|2386.97|5.72|0.00|572.85|578.57|-856.52| +2451772|48071|17281|82699|1071821|496|18771|1|18|2028|5|99.12|175.44|45.61|0.00|228.05|495.60|877.20|2.28|0.00|228.05|230.33|-267.55| +2451772|48071|4655|82699|1071821|496|18771|1|147|2028|76|47.21|83.56|79.38|0.00|6032.88|3587.96|6350.56|301.64|0.00|6032.88|6334.52|2444.92| +2451772|48071|12929|82699|1071821|496|18771|1|40|2028|51|51.09|88.38|70.70|0.00|3605.70|2605.59|4507.38|324.51|0.00|3605.70|3930.21|1000.11| +2451772|48071|4651|82699|1071821|496|18771|1|107|2028|91|51.56|79.40|64.31|1814.18|5852.21|4691.96|7225.40|80.76|1814.18|4038.03|4118.79|-653.93| +2451772|48071|16517|82699|1071821|496|18771|1|291|2028|93|59.75|88.43|30.95|0.00|2878.35|5556.75|8223.99|57.56|0.00|2878.35|2935.91|-2678.40| +2451772|48071|16700|82699|1071821|496|18771|1|20|2028|28|13.67|15.72|2.82|0.00|78.96|382.76|440.16|2.36|0.00|78.96|81.32|-303.80| +2451772|48071|15265|82699|1071821|496|18771|1|268|2028|85|73.94|76.89|13.84|0.00|1176.40|6284.90|6535.65|94.11|0.00|1176.40|1270.51|-5108.50| +2451505|37183|9190|67874|1129952|5630|5308|4|144|2029|71|85.38|128.07|3.84|0.00|272.64|6061.98|9092.97|5.45|0.00|272.64|278.09|-5789.34| +2451505|37183|10786|67874|1129952|5630|5308|4|228|2029|79|18.70|24.12|17.12|0.00|1352.48|1477.30|1905.48|121.72|0.00|1352.48|1474.20|-124.82| +2451505|37183|9163|67874|1129952|5630|5308|4|300|2029|24|75.74|115.88|11.58|0.00|277.92|1817.76|2781.12|2.77|0.00|277.92|280.69|-1539.84| +2451505||16388|67874||5630||||2029||26.37|||||||13.80||460.32||91.14| +2451505|37183|17968|67874|1129952|5630|5308|4|239|2029|8|36.64|39.20|6.66|0.00|53.28|293.12|313.60|1.59|0.00|53.28|54.87|-239.84| +2451505|37183|488|67874|1129952|5630|5308|4|121|2029|58|1.57|2.82|2.70|21.92|156.60|91.06|163.56|12.12|21.92|134.68|146.80|43.62| +2451505|37183|2224|67874|1129952|5630|5308|4|286|2029|32|57.22|108.71|33.70|0.00|1078.40|1831.04|3478.72|64.70|0.00|1078.40|1143.10|-752.64| +2451505|37183|1012|67874|1129952|5630|5308|4|8|2029|55|44.12|55.15|14.89|0.00|818.95|2426.60|3033.25|32.75|0.00|818.95|851.70|-1607.65| +2451505|37183|13808|67874|1129952|5630|5308|4|198|2029|57|56.31|85.02|29.75|0.00|1695.75|3209.67|4846.14|101.74|0.00|1695.75|1797.49|-1513.92| +2451505|37183|2810|67874|1129952|5630|5308|4|42|2029|40|68.84|101.19|35.41|0.00|1416.40|2753.60|4047.60|84.98|0.00|1416.40|1501.38|-1337.20| +2451505|37183|943|67874|1129952|5630|5308|4|86|2029|70|95.21|137.10|21.93|0.00|1535.10|6664.70|9597.00|0.00|0.00|1535.10|1535.10|-5129.60| +2451505|37183|6019|67874|1129952|5630|5308|4|5|2029|27|79.49|108.90|83.85|0.00|2263.95|2146.23|2940.30|45.27|0.00|2263.95|2309.22|117.72| +2451505|37183|14240||1129952||||167|2029|87|38.55|||||3353.85|3789.72|||2728.32|2837.45|| +2451505|37183|716|67874|1129952|5630|5308|4|261|2029|87|3.04|4.07|2.03|0.00|176.61|264.48|354.09|15.89|0.00|176.61|192.50|-87.87| +2451505|37183|11605|67874|1129952|5630|5308|4|251|2029|8|67.92|80.14|62.50|0.00|500.00|543.36|641.12|5.00|0.00|500.00|505.00|-43.36| +2451505|37183|85|67874|1129952|5630|5308|4|190|2029|93|85.00|159.80|147.01|0.00|13671.93|7905.00|14861.40|0.00|0.00|13671.93|13671.93|5766.93| +2451529|69898|5173|90555|22617|1526|32388|1|2|2030|98|71.55|95.87|69.98|891.54|6858.04|7011.90|9395.26|178.99|891.54|5966.50|6145.49|-1045.40| +2451529|69898|973|90555|22617|1526|32388|1|252|2030|91|37.32|58.21|27.35|0.00|2488.85|3396.12|5297.11|49.77|0.00|2488.85|2538.62|-907.27| +2451529|69898|17996|90555|22617|1526|32388|1|165|2030|89|72.54|79.79|36.70|1045.21|3266.30|6456.06|7101.31|66.63|1045.21|2221.09|2287.72|-4234.97| +2451529|69898|5263|90555|22617|1526|32388|1|195|2030|4|66.14|78.70|37.77|0.00|151.08|264.56|314.80|3.02|0.00|151.08|154.10|-113.48| +2451529|69898|5941|90555|22617|1526|32388|1|264|2030|64|90.28|160.69|91.59|0.00|5861.76|5777.92|10284.16|234.47|0.00|5861.76|6096.23|83.84| +2451529|69898|8438|90555|22617|1526|32388|1|240|2030|66|84.29|104.51|10.45|0.00|689.70|5563.14|6897.66|0.00|0.00|689.70|689.70|-4873.44| +2451529|69898|15710|90555|22617|1526|32388|1|247|2030|32|40.57|45.03|10.35|264.96|331.20|1298.24|1440.96|1.32|264.96|66.24|67.56|-1232.00| +2451529|69898|1234|90555|22617|1526|32388|1|40|2030|45|35.49|44.36|31.05|0.00|1397.25|1597.05|1996.20|0.00|0.00|1397.25|1397.25|-199.80| +2451529|69898|13388|90555|22617|1526|32388|1|193|2030|1|1.16|1.29|0.19|0.00|0.19|1.16|1.29|0.01|0.00|0.19|0.20|-0.97| +||1522|||1526|||276|2030|||||784.01|||3146.22|120.96|784.01||2137.00|| +2451529|69898|6637|90555|22617|1526|32388|1|290|2030|86|94.16|126.17|30.28|0.00|2604.08|8097.76|10850.62|156.24|0.00|2604.08|2760.32|-5493.68| +2451529|69898|15832|90555|22617|1526|32388|1|152|2030|41|47.61|84.74|33.89|0.00|1389.49|1952.01|3474.34|41.68|0.00|1389.49|1431.17|-562.52| +2451529|69898|2348|90555|22617|1526|32388|1|82|2030|92|91.15|151.30|15.13|0.00|1391.96|8385.80|13919.60|27.83|0.00|1391.96|1419.79|-6993.84| +2451529|69898|4004|90555|22617|1526|32388|1|147|2030|54|68.61|90.56|67.01|0.00|3618.54|3704.94|4890.24|217.11|0.00|3618.54|3835.65|-86.40| +2451529|69898|2665|90555|22617|1526|32388|1|206|2030|25|19.47|29.01|26.97|0.00|674.25|486.75|725.25|33.71|0.00|674.25|707.96|187.50| +2452244|54443|10077|45724|264075|4026|41530|4|68|2031|69|29.63|54.22|37.95|0.00|2618.55|2044.47|3741.18|209.48|0.00|2618.55|2828.03|574.08| +2452244|54443|14649|45724|264075|4026|41530|4|66|2031|90|99.66|155.46|105.71|0.00|9513.90|8969.40|13991.40|570.83|0.00|9513.90|10084.73|544.50| +2452244|54443|14879|45724|264075|4026|41530|4|73|2031|67|24.88|46.77|2.33|0.00|156.11|1666.96|3133.59|6.24|0.00|156.11|162.35|-1510.85| +2452244|54443|6979|45724|264075|4026|41530|4|13|2031|65|73.36|89.49|8.94|0.00|581.10|4768.40|5816.85|23.24|0.00|581.10|604.34|-4187.30| +2452244|54443|16967|45724|264075|4026|41530|4|250|2031|79|8.26|16.18|13.10|0.00|1034.90|652.54|1278.22|72.44|0.00|1034.90|1107.34|382.36| +2452244|54443|6119|45724|264075|4026|41530|4|271|2031|28|22.77|29.60|28.12|0.00|787.36|637.56|828.80|23.62|0.00|787.36|810.98|149.80| +2452244|54443|8543|45724|264075|4026|41530|4|77|2031|22|13.33|23.06|12.91|0.00|284.02|293.26|507.32|2.84|0.00|284.02|286.86|-9.24| +2452244|54443|675|45724|264075|4026|41530|4|120|2031|10|46.64|89.54|78.79|0.00|787.90|466.40|895.40|7.87|0.00|787.90|795.77|321.50| +2452244|54443|8563|45724|264075|4026|41530|4|110|2031|77|2.04|3.24|0.61|0.00|46.97|157.08|249.48|0.93|0.00|46.97|47.90|-110.11| +2452244|54443|1955|45724|264075|4026|41530|4|142|2031|33|34.54|43.17|6.90|0.00|227.70|1139.82|1424.61|6.83|0.00|227.70|234.53|-912.12| +2452244|54443|4095|45724|264075|4026|41530|4|205|2031|19|95.06|163.50|73.57|0.00|1397.83|1806.14|3106.50|69.89|0.00|1397.83|1467.72|-408.31| +2452244|54443|4517|45724|264075|4026|41530|4|138|2031|4|56.92|58.05|44.11|0.00|176.44|227.68|232.20|8.82|0.00|176.44|185.26|-51.24| +2452244|54443|7181|45724|264075|4026|41530|4|42|2031|66|9.09|13.81|2.34|0.00|154.44|599.94|911.46|7.72|0.00|154.44|162.16|-445.50| +2452244|54443|9645|45724|264075|4026|41530|4|154|2031|45|59.25|69.91|44.74|0.00|2013.30|2666.25|3145.95|161.06|0.00|2013.30|2174.36|-652.95| +2452244|54443|17889|45724|264075|4026|41530|4|290|2031|99|42.19|49.78|34.84|0.00|3449.16|4176.81|4928.22|103.47|0.00|3449.16|3552.63|-727.65| +2452244|54443|11885|45724|264075|4026|41530|4|71|2031|31|33.30|51.28|15.38|438.63|476.78|1032.30|1589.68|3.43|438.63|38.15|41.58|-994.15| +2451381|69381|11552|64376|1545343|6310|28077|7|79|2032|78|19.25|21.75|21.53|0.00|1679.34|1501.50|1696.50|0.00|0.00|1679.34|1679.34|177.84| +2451381|69381|430|64376|1545343|6310|28077|7|39|2032|37|73.18|121.47|17.00|0.00|629.00|2707.66|4494.39|0.00|0.00|629.00|629.00|-2078.66| +2451381|69381|7144|64376|1545343|6310|28077|7|75|2032|50|68.28|75.10|44.30|0.00|2215.00|3414.00|3755.00|132.90|0.00|2215.00|2347.90|-1199.00| +2451381|69381|6610|64376|1545343|6310|28077|7|2|2032|5|41.04|43.91|0.87|0.00|4.35|205.20|219.55|0.00|0.00|4.35|4.35|-200.85| +2451381|69381|7801|64376|1545343|6310|28077|7|288|2032|76|10.57|18.92|14.37|0.00|1092.12|803.32|1437.92|87.36|0.00|1092.12|1179.48|288.80| +2451381|69381|6284|64376|1545343|6310|28077|7|115|2032|90|97.83|117.39|48.12|0.00|4330.80|8804.70|10565.10|389.77|0.00|4330.80|4720.57|-4473.90| +2451381|69381|5839|64376|1545343|6310|28077|7|4|2032|17|88.34|130.74|73.21|771.63|1244.57|1501.78|2222.58|28.37|771.63|472.94|501.31|-1028.84| +2451381|69381|5689|64376|1545343|6310|28077|7|296|2032|27|21.49|28.15|4.22|0.00|113.94|580.23|760.05|9.11|0.00|113.94|123.05|-466.29| +2451381|69381|9610|64376|1545343|6310|28077|7|51|2032|43|99.03|186.17|134.04|0.00|5763.72|4258.29|8005.31|57.63|0.00|5763.72|5821.35|1505.43| +2451381|69381|15997|64376|1545343|6310|28077|7|256|2032|80|99.97|190.94|169.93|0.00|13594.40|7997.60|15275.20|1223.49|0.00|13594.40|14817.89|5596.80| +2451381|69381|8821|64376|1545343|6310|28077|7|132|2032|92|64.86|80.42|46.64|0.00|4290.88|5967.12|7398.64|386.17|0.00|4290.88|4677.05|-1676.24| +2451570|71074|13646|36370|525249|4528|37007|2|81|2033|1|35.77|60.09|22.83|8.44|22.83|35.77|60.09|0.86|8.44|14.39|15.25|-21.38| +2451570|71074|5390|36370|525249|4528|37007|2|274|2033|27|84.93|103.61|51.80|0.00|1398.60|2293.11|2797.47|41.95|0.00|1398.60|1440.55|-894.51| +2451570|71074|2743|36370|525249|4528|37007|2|78|2033|72|26.27|37.56|18.40|821.37|1324.80|1891.44|2704.32|25.17|821.37|503.43|528.60|-1388.01| +2451570|71074|1495|36370|525249|4528|37007|2|176|2033|61|26.74|43.05|27.98|0.00|1706.78|1631.14|2626.05|119.47|0.00|1706.78|1826.25|75.64| +2451570|71074|14048|36370|525249|4528|37007|2|115|2033|67|22.66|25.83|3.61|137.86|241.87|1518.22|1730.61|5.20|137.86|104.01|109.21|-1414.21| +2451570|71074|9833|36370|525249|4528|37007|2|94|2033|23|76.57|143.95|128.11|1473.26|2946.53|1761.11|3310.85|117.86|1473.26|1473.27|1591.13|-287.84| +2451570|71074|15044|36370|525249|4528|37007|2|260|2033|57|16.45|21.38|20.73|0.00|1181.61|937.65|1218.66|106.34|0.00|1181.61|1287.95|243.96| +2451570|71074|8287|36370|525249|4528|37007|2|176|2033|80|79.54|82.72|3.30|0.00|264.00|6363.20|6617.60|21.12|0.00|264.00|285.12|-6099.20| +2451570|71074|11510|36370|525249|4528|37007|2|245|2033|14|64.14|116.73|36.18|0.00|506.52|897.96|1634.22|25.32|0.00|506.52|531.84|-391.44| +|71074|5389|36370|525249|4528|||12|2033||38.45|57.29|7.44||7.44|||0.44||7.44||| +2452401|44888|6633|20262||||4|76|2034||8.92||0.16||13.60||1432.25|||13.60|14.00|-744.60| +2452401|44888|8383|20262|459914|5166|43665|4|171|2034|36|6.07|11.65|8.15|0.00|293.40|218.52|419.40|14.67|0.00|293.40|308.07|74.88| +2452401|44888|10717|20262|459914|5166|43665|4|172|2034|8|35.95|67.94|38.04|0.00|304.32|287.60|543.52|27.38|0.00|304.32|331.70|16.72| +2452401|44888|11766|20262|459914|5166|43665|4|289|2034|20|49.55|52.02|23.92|0.00|478.40|991.00|1040.40|4.78|0.00|478.40|483.18|-512.60| +|44888|14007||459914||||266|2034||30.20|38.65||0.00|||3169.30||0.00|1521.10|1581.94|-955.30| +2452401|44888|1515|20262|459914|5166|43665|4|272|2034|3|42.42|55.57|41.12|0.00|123.36|127.26|166.71|8.63|0.00|123.36|131.99|-3.90| +2452401|44888|17431|20262|459914|5166|43665|4|152|2034|56|58.85|72.38|41.98|0.00|2350.88|3295.60|4053.28|211.57|0.00|2350.88|2562.45|-944.72| +2452401|44888|12015|20262|459914|5166|43665|4|282|2034|88|67.51|134.34|131.65|9847.42|11585.20|5940.88|11821.92|17.37|9847.42|1737.78|1755.15|-4203.10| +2452401|44888|8629|20262|459914|5166|43665|4|20|2034|40|25.58|39.64|15.06|0.00|602.40|1023.20|1585.60|42.16|0.00|602.40|644.56|-420.80| +2452401|44888|5365|20262|459914|5166|43665|4|119|2034|24|67.31|80.77|9.69|0.00|232.56|1615.44|1938.48|13.95|0.00|232.56|246.51|-1382.88| +||12467||||47307|2|63|2035|88|||49.89|0.00|4390.32||11553.52||0.00||4478.12|-3416.16| +2451895|45786|13153|21682|1418208|4781|47307|2|118|2035|76|53.58|100.19|69.13|4781.03|5253.88|4072.08|7614.44|14.18|4781.03|472.85|487.03|-3599.23| +2451895|45786|2819|21682|1418208|4781|47307|2|106|2035|45|17.81|33.12|20.53|452.68|923.85|801.45|1490.40|4.71|452.68|471.17|475.88|-330.28| +2451895|45786|11258|21682|1418208|4781|47307|2|115|2035|4|22.41|35.18|25.68|0.00|102.72|89.64|140.72|5.13|0.00|102.72|107.85|13.08| +||17798|||||2|247|2035||5.48||1.31|||||2.41||||| +2451895|45786|2828|21682|1418208|4781|47307|2|95|2035|24|76.68|137.25|60.39|0.00|1449.36|1840.32|3294.00|57.97|0.00|1449.36|1507.33|-390.96| +2451895|45786|5971|21682|1418208|4781|47307|2|269|2035|33|80.77|82.38|34.59|0.00|1141.47|2665.41|2718.54|68.48|0.00|1141.47|1209.95|-1523.94| +2451895|45786|6683|21682|1418208|4781|47307|2|211|2035|7|45.12|46.47|15.33|59.02|107.31|315.84|325.29|1.93|59.02|48.29|50.22|-267.55| +2451895|45786|4430|21682|1418208|4781|47307|2|274|2035|77|65.64|91.23|19.15|0.00|1474.55|5054.28|7024.71|44.23|0.00|1474.55|1518.78|-3579.73| +2450936|63450|3806|23376|1553761|4146|30131|2|217|2036|4|83.19|88.18|67.89|0.00|271.56|332.76|352.72|0.00|0.00|271.56|271.56|-61.20| +2450936|63450|10010|23376|1553761|4146|30131|2|207|2036|21|86.01|89.45|60.82|0.00|1277.22|1806.21|1878.45|0.00|0.00|1277.22|1277.22|-528.99| +2450936|63450|2761|23376|1553761|4146|30131|2|135|2036|86|82.80|135.79|58.38|0.00|5020.68|7120.80|11677.94|100.41|0.00|5020.68|5121.09|-2100.12| +2450936|63450|842|23376|1553761|4146|30131|2|91|2036|70|78.92|104.17|32.29|1356.18|2260.30|5524.40|7291.90|54.24|1356.18|904.12|958.36|-4620.28| +2450936|63450|17467|23376|1553761|4146|30131|2|191|2036|58|94.92|160.41|38.49|2031.50|2232.42|5505.36|9303.78|8.03|2031.50|200.92|208.95|-5304.44| +2450936|63450|1864|23376|1553761|4146|30131|2|58|2036|86|61.92|85.44|53.82|0.00|4628.52|5325.12|7347.84|0.00|0.00|4628.52|4628.52|-696.60| +2450936|63450|8054|23376|1553761|4146|30131|2|203|2036|67|48.45|89.14|35.65|0.00|2388.55|3246.15|5972.38|71.65|0.00|2388.55|2460.20|-857.60| +2450936|63450|5233|23376|1553761|4146|30131|2|236|2036|5|30.40|57.45|31.02|0.00|155.10|152.00|287.25|4.65|0.00|155.10|159.75|3.10| +2450936|63450|17173|23376|1553761|4146|30131|2|53|2036|49|18.16|33.41|33.41|0.00|1637.09|889.84|1637.09|65.48|0.00|1637.09|1702.57|747.25| +2450936|63450|944|23376|1553761|4146|30131|2|85|2036|90|25.61|43.79|38.97|736.53|3507.30|2304.90|3941.10|249.36|736.53|2770.77|3020.13|465.87| +2450936|63450|14683|23376|1553761|4146|30131|2|121|2036|23|2.93|3.89|3.42|0.00|78.66|67.39|89.47|6.29|0.00|78.66|84.95|11.27| +2450936|63450|11662|23376|1553761|4146|30131|2|164|2036|66|78.32|120.61|61.51|0.00|4059.66|5169.12|7960.26|121.78|0.00|4059.66|4181.44|-1109.46| +2450936|63450|2551|23376|1553761|4146|30131|2|157|2036|45|54.74|70.06|22.41|0.00|1008.45|2463.30|3152.70|0.00|0.00|1008.45|1008.45|-1454.85| +2450936|63450|15778|23376|1553761|4146|30131|2|160|2036|19|62.44|99.27|78.42|0.00|1489.98|1186.36|1886.13|134.09|0.00|1489.98|1624.07|303.62| +2450949|29665|9904|1916|1247781|1199|3064|8|207|2037|19|93.33|119.46|32.25|0.00|612.75|1773.27|2269.74|24.51|0.00|612.75|637.26|-1160.52| +2450949|29665|6952|1916|1247781|1199|3064|8|220|2037|11|17.91|29.19|27.14|0.00|298.54|197.01|321.09|26.86|0.00|298.54|325.40|101.53| +2450949|29665|11038|1916|1247781|1199|3064|8|250|2037|88|38.73|76.29|33.56|0.00|2953.28|3408.24|6713.52|59.06|0.00|2953.28|3012.34|-454.96| +2450949|29665|16796|1916|1247781|1199|3064|8|215|2037|28|43.77|57.33|9.74|0.00|272.72|1225.56|1605.24|19.09|0.00|272.72|291.81|-952.84| +2450949|29665|1550|1916|1247781|1199|3064|8|196|2037|24|50.40|50.40|11.59|0.00|278.16|1209.60|1209.60|16.68|0.00|278.16|294.84|-931.44| +2450949|29665|12772|1916|1247781|1199|3064|8|109|2037|19|10.74|14.60|5.11|0.00|97.09|204.06|277.40|6.79|0.00|97.09|103.88|-106.97| +2450949|29665|5467|1916|1247781|1199|3064|8|65|2037|62|43.12|48.72|37.02|0.00|2295.24|2673.44|3020.64|68.85|0.00|2295.24|2364.09|-378.20| +2450949|29665|8816|1916|1247781|1199|3064|8|33|2037|14|71.35|119.86|41.95|0.00|587.30|998.90|1678.04|11.74|0.00|587.30|599.04|-411.60| +2450949|29665|7028|1916|1247781|1199|3064|8|245|2037|90|81.22|116.14|34.84|0.00|3135.60|7309.80|10452.60|125.42|0.00|3135.60|3261.02|-4174.20| +2450949|29665|2155|1916|1247781|1199|3064|8|206|2037|31|6.38|10.14|9.83|0.00|304.73|197.78|314.34|15.23|0.00|304.73|319.96|106.95| +2450949|29665|15388|1916|1247781|1199|3064|8|204|2037|18|89.22|159.70|99.01|0.00|1782.18|1605.96|2874.60|53.46|0.00|1782.18|1835.64|176.22| +2451190|36489|4564|17773|458413|6253|37693|4|17|2038|4|93.85|164.23|164.23|0.00|656.92|375.40|656.92|6.56|0.00|656.92|663.48|281.52| +2451190|36489|487|17773|458413|6253|37693|4|285|2038|49|83.34|136.67|94.30|0.00|4620.70|4083.66|6696.83|323.44|0.00|4620.70|4944.14|537.04| +2451190|36489|7861|17773|458413|6253|37693|4|110|2038|21|53.13|105.19|51.54|800.93|1082.34|1115.73|2208.99|2.81|800.93|281.41|284.22|-834.32| +2451190|36489|2234|17773|458413|6253|37693|4|271|2038|10|27.90|30.41|6.69|0.00|66.90|279.00|304.10|0.00|0.00|66.90|66.90|-212.10| +2451190|36489|17474|17773|458413|6253|37693|4|100|2038|58|22.53|28.61|17.45|0.00|1012.10|1306.74|1659.38|50.60|0.00|1012.10|1062.70|-294.64| +2451190|36489|9304|17773|458413|6253|37693|4|240|2038|86|12.98|17.13|7.02|0.00|603.72|1116.28|1473.18|6.03|0.00|603.72|609.75|-512.56| +2451190|36489|4952|17773|458413|6253|37693|4|64|2038|82|82.56|104.02|9.36|0.00|767.52|6769.92|8529.64|46.05|0.00|767.52|813.57|-6002.40| +2451190|36489|17054|17773|458413|6253|37693|4|251|2038|65|51.53|101.51|13.19|0.00|857.35|3349.45|6598.15|34.29|0.00|857.35|891.64|-2492.10| +2451190|36489|12301|17773|458413|6253|37693|4|3|2038|25|91.53|97.02|35.89|0.00|897.25|2288.25|2425.50|80.75|0.00|897.25|978.00|-1391.00| +2451190|36489|4843|17773|458413|6253|37693|4|33|2038|55|8.94|16.44|8.22|275.78|452.10|491.70|904.20|3.52|275.78|176.32|179.84|-315.38| +2451190|36489|1588|17773|458413|6253|37693|4|145|2038|67|31.69|49.75|45.77|0.00|3066.59|2123.23|3333.25|153.32|0.00|3066.59|3219.91|943.36| +2451190|36489|6518|17773|458413|6253|37693|4|263|2038|88|13.88|16.10|2.89|106.81|254.32|1221.44|1416.80|0.00|106.81|147.51|147.51|-1073.93| +2451190|36489|15202|17773|458413|6253|37693|4|162|2038|89|35.19|46.80|31.82|0.00|2831.98|3131.91|4165.20|169.91|0.00|2831.98|3001.89|-299.93| +2452194|67364|2187|43791|65345|4052|47606|8|299|2039|53|65.31|110.37|89.39|0.00|4737.67|3461.43|5849.61|236.88|0.00|4737.67|4974.55|1276.24| +2452194|67364|6303|43791|65345|4052|47606|8|193|2039|34|2.97|3.62|1.77|0.00|60.18|100.98|123.08|5.41|0.00|60.18|65.59|-40.80| +2452194|67364|4689|43791|65345|4052|47606|8|214|2039|27|78.30|80.64|33.86|0.00|914.22|2114.10|2177.28|18.28|0.00|914.22|932.50|-1199.88| +2452194|67364|5869|43791|65345|4052|47606|8|81|2039|7|85.29|117.70|71.79|0.00|502.53|597.03|823.90|30.15|0.00|502.53|532.68|-94.50| +2452194|67364|12489|43791|65345|4052|47606|8|7|2039|91|9.07|10.52|4.73|0.00|430.43|825.37|957.32|30.13|0.00|430.43|460.56|-394.94| +2452194|67364|12459|43791|65345|4052|47606|8|72|2039|41|56.66|98.58|15.77|0.00|646.57|2323.06|4041.78|58.19|0.00|646.57|704.76|-1676.49| +2452194|67364|6163|43791|65345|4052|47606|8|98|2039|62|91.35|112.36|5.61|0.00|347.82|5663.70|6966.32|27.82|0.00|347.82|375.64|-5315.88| +2452194|67364|7053|43791|65345|4052|47606|8|64|2039|1|25.33|44.32|39.00|0.00|39.00|25.33|44.32|1.95|0.00|39.00|40.95|13.67| +2452194|67364|6179|43791|65345|4052|47606|8|10|2039|56|47.48|91.16|46.49|2291.02|2603.44|2658.88|5104.96|28.11|2291.02|312.42|340.53|-2346.46| +2452194|67364|9233|43791|65345|4052|47606|8|96|2039|40|52.61|63.65|31.18|0.00|1247.20|2104.40|2546.00|99.77|0.00|1247.20|1346.97|-857.20| +2452194|67364|15107|43791|65345|4052|47606|8|200|2039|88|79.16|125.86|125.86|0.00|11075.68|6966.08|11075.68|110.75|0.00|11075.68|11186.43|4109.60| +2452194|67364|3757|43791|65345|4052|47606|8|280|2039|33|33.36|36.02|30.25|0.00|998.25|1100.88|1188.66|69.87|0.00|998.25|1068.12|-102.63| +2452194|67364|14297|43791|65345|4052|47606|8|48|2039|14|72.81|95.38|90.61|0.00|1268.54|1019.34|1335.32|0.00|0.00|1268.54|1268.54|249.20| +2452194|67364|8521|43791|65345|4052|47606|8|26|2039|11|18.56|18.56|17.81|0.00|195.91|204.16|204.16|17.63|0.00|195.91|213.54|-8.25| +2452194|67364|2711|43791|65345|4052|47606|8|61|2039|88|62.55|99.45|15.91|0.00|1400.08|5504.40|8751.60|84.00|0.00|1400.08|1484.08|-4104.32| +2452194|67364|11527|43791|65345|4052|47606|8|55|2039|58|7.03|12.51|6.25|0.00|362.50|407.74|725.58|0.00|0.00|362.50|362.50|-45.24| +2452400|34376|3567|33486|109174|4415|4856|4|278|2040|22|35.04|38.19|14.51|0.00|319.22|770.88|840.18|15.96|0.00|319.22|335.18|-451.66| +2452400|34376|12984|33486|109174|4415|4856|4|53|2040|71|11.55|13.74|10.85|0.00|770.35|820.05|975.54|0.00|0.00|770.35|770.35|-49.70| +2452400|34376|9115|33486|109174|4415|4856|4|116|2040|7|54.72|90.83|53.58|0.00|375.06|383.04|635.81|26.25|0.00|375.06|401.31|-7.98| +2452400|34376|855|33486|109174|4415|4856|4|261|2040|95|98.00|134.26|89.95|0.00|8545.25|9310.00|12754.70|0.00|0.00|8545.25|8545.25|-764.75| +2452400|34376|17113|33486|109174|4415|4856|4|257|2040|21|85.17|105.61|33.79|0.00|709.59|1788.57|2217.81|56.76|0.00|709.59|766.35|-1078.98| +2452400|34376|1518|33486|109174|4415|4856|4|37|2040|32|76.38|151.99|62.31|0.00|1993.92|2444.16|4863.68|0.00|0.00|1993.92|1993.92|-450.24| +||14371||109174||4856|4|101|2040|72||6.31|3.65|197.10||235.44|454.32||197.10||70.29|| +2452400|34376|2149|33486|109174|4415|4856|4|60|2040|51|1.37|1.89|1.11|0.00|56.61|69.87|96.39|1.13|0.00|56.61|57.74|-13.26| +2452400|34376|15255|33486|109174|4415|4856|4|69|2040|53|58.64|110.24|56.22|0.00|2979.66|3107.92|5842.72|0.00|0.00|2979.66|2979.66|-128.26| +2451479|50188|571|8232|563669|5953|16190|4|175|2041|18|14.44|20.07|7.22|0.00|129.96|259.92|361.26|7.79|0.00|129.96|137.75|-129.96| +2451479|50188|16468|8232|563669|5953|16190|4|2|2041|17|9.62|11.64|11.40|0.00|193.80|163.54|197.88|15.50|0.00|193.80|209.30|30.26| +2451479|50188|15134|8232|563669|5953|16190|4|249|2041|37|20.53|38.39|18.81|480.21|695.97|759.61|1420.43|4.31|480.21|215.76|220.07|-543.85| +2451479|50188|17257|8232|563669|5953|16190|4|54|2041|94|70.33|85.80|63.49|0.00|5968.06|6611.02|8065.20|477.44|0.00|5968.06|6445.50|-642.96| +2451479|50188|16622|8232|563669|5953|16190|4|266|2041|15|55.19|97.68|88.88|0.00|1333.20|827.85|1465.20|53.32|0.00|1333.20|1386.52|505.35| +2451479|50188|16258|8232|563669|5953|16190|4|285|2041|59|7.14|11.28|6.76|0.00|398.84|421.26|665.52|11.96|0.00|398.84|410.80|-22.42| +2451479|50188|2977|8232|563669|5953|16190|4|131|2041|31|13.50|20.92|15.48|273.53|479.88|418.50|648.52|4.12|273.53|206.35|210.47|-212.15| +2451479|50188|14432|8232|563669|5953|16190|4|161|2041|24|98.26|129.70|28.53|0.00|684.72|2358.24|3112.80|6.84|0.00|684.72|691.56|-1673.52| +2451479|50188|13714|8232|563669|5953|16190|4|151|2041|56|47.41|70.64|7.06|0.00|395.36|2654.96|3955.84|23.72|0.00|395.36|419.08|-2259.60| +2450956|33917|1186|96502|1423317|2932|43700|8|121|2042|41|63.91|79.88|75.88|0.00|3111.08|2620.31|3275.08|0.00|0.00|3111.08|3111.08|490.77| +2450956|33917|511|96502|1423317|2932|43700|8|40|2042|73|20.37|28.72|0.00|0.00|0.00|1487.01|2096.56|0.00|0.00|0.00|0.00|-1487.01| +2450956||4298||||||273|2042||13.64||9.60|0.00|470.40||||0.00|470.40|512.73|-197.96| +2450956|33917|2911|96502|1423317|2932|43700|8|197|2042|1|75.94|76.69|29.14|0.00|29.14|75.94|76.69|2.62|0.00|29.14|31.76|-46.80| +2450956|33917|9580|96502|1423317|2932|43700|8|112|2042|23|13.12|14.69|10.57|235.81|243.11|301.76|337.87|0.58|235.81|7.30|7.88|-294.46| +2450956|33917|15812|96502|1423317|2932|43700|8|49|2042|93|89.01|100.58|90.52|2357.14|8418.36|8277.93|9353.94|484.89|2357.14|6061.22|6546.11|-2216.71| +||13525|96502|1423317|||8|25|2042|||11.44||0.00|1030.59|572.22|1132.56|61.83|0.00||1092.42|458.37| +2450956|33917|10738|96502|1423317|2932|43700|8|162|2042|75|1.20|1.47|0.07|0.00|5.25|90.00|110.25|0.15|0.00|5.25|5.40|-84.75| +2451995|35217|8131|75331|1684212|6246|5527|7|153|2043|82|24.27|28.63|0.85|0.00|69.70|1990.14|2347.66|5.57|0.00|69.70|75.27|-1920.44| +2451995|35217|9597|75331|1684212|6246|5527|7|112|2043|42|88.91|100.46|87.40|0.00|3670.80|3734.22|4219.32|110.12|0.00|3670.80|3780.92|-63.42| +2451995|35217|2673|75331|1684212|6246|5527|7|96|2043|12|36.61|51.62|43.36|421.45|520.32|439.32|619.44|2.96|421.45|98.87|101.83|-340.45| +2451995|35217|16673|75331|1684212|6246|5527|7|224|2043|92|86.46|130.55|120.10|0.00|11049.20|7954.32|12010.60|0.00|0.00|11049.20|11049.20|3094.88| +2451995|35217|15211|75331|1684212|6246|5527|7|12|2043|18|16.14|19.36|6.38|0.00|114.84|290.52|348.48|4.59|0.00|114.84|119.43|-175.68| +2451995|35217|8209|75331|1684212|6246|5527|7|248|2043|36|34.29|58.29|33.22|0.00|1195.92|1234.44|2098.44|11.95|0.00|1195.92|1207.87|-38.52| +2451995|35217|13747|75331|1684212|6246|5527|7|71|2043|90|72.79|85.16|73.23|659.07|6590.70|6551.10|7664.40|474.53|659.07|5931.63|6406.16|-619.47| +2451995|35217|11925|75331|1684212|6246|5527|7|65|2043|58|93.09|144.28|46.16|0.00|2677.28|5399.22|8368.24|160.63|0.00|2677.28|2837.91|-2721.94| +2451070|64112|2104|56447|122201|3955|9709|1|111|2044|48|93.03|136.75|16.41|0.00|787.68|4465.44|6564.00|0.00|0.00|787.68|787.68|-3677.76| +2451070|64112|14990|56447|122201|3955|9709|1|300|2044|85|37.58|75.16|65.38|2889.79|5557.30|3194.30|6388.60|133.37|2889.79|2667.51|2800.88|-526.79| +2451070|64112|4771|56447|122201|3955|9709|1|242|2044|80|36.36|70.90|9.21|0.00|736.80|2908.80|5672.00|66.31|0.00|736.80|803.11|-2172.00| +2451070|64112|3991|56447|122201|3955|9709|1|295|2044|10|69.85|121.53|86.28|0.00|862.80|698.50|1215.30|0.00|0.00|862.80|862.80|164.30| +2451070||9184||122201|3955|9709|1|239|2044|69||2.66|1.01|0.00||||2.78|0.00|||| +2451070|64112|4294|56447|122201|3955|9709|1|256|2044|55|63.77|116.06|74.27|0.00|4084.85|3507.35|6383.30|0.00|0.00|4084.85|4084.85|577.50| +2451070|64112|16525|56447|122201|3955|9709|1|161|2044|94|82.46|97.30|46.70|0.00|4389.80|7751.24|9146.20|131.69|0.00|4389.80|4521.49|-3361.44| +2451070|64112|2116|56447|122201|3955|9709|1|188|2044|98|39.71|63.53|51.45|0.00|5042.10|3891.58|6225.94|302.52|0.00|5042.10|5344.62|1150.52| +2451070|64112|11768|56447|122201|3955|9709|1|69|2044|17|49.59|75.87|51.59|0.00|877.03|843.03|1289.79|70.16|0.00|877.03|947.19|34.00| +2451070|64112|3692|56447|122201|3955|9709|1|220|2044|19|46.16|82.16|57.51|426.14|1092.69|877.04|1561.04|39.99|426.14|666.55|706.54|-210.49| +2451070|64112|16729|56447|122201|3955|9709|1|82|2044|82|43.66|76.40|11.46|169.14|939.72|3580.12|6264.80|69.35|169.14|770.58|839.93|-2809.54| +2451070|64112|14707|56447|122201|3955|9709|1|185|2044|9|38.08|45.31|6.34|27.95|57.06|342.72|407.79|0.00|27.95|29.11|29.11|-313.61| +||3656||122201||9709|||2044||||53.92||||4368.64|||3450.88||| +2451070|64112|7093|56447|122201|3955|9709|1|49|2044|49|93.59|109.50|70.08|0.00|3433.92|4585.91|5365.50|206.03|0.00|3433.92|3639.95|-1151.99| +2452217|67299|6349|55140|1436722|5951|6780|4|71|2045|7|67.21|89.38|15.19|58.48|106.33|470.47|625.66|2.39|58.48|47.85|50.24|-422.62| +2452217|67299|2419|55140|1436722|5951|6780|4|94|2045|52|58.47|92.38|50.80|0.00|2641.60|3040.44|4803.76|211.32|0.00|2641.60|2852.92|-398.84| +2452217|67299|11617|55140|1436722|5951|6780|4|33|2045|42|48.97|83.73|9.21|0.00|386.82|2056.74|3516.66|11.60|0.00|386.82|398.42|-1669.92| +2452217|67299|11811|55140|1436722|5951|6780|4|280|2045|16|3.62|6.26|0.25|1.32|4.00|57.92|100.16|0.10|1.32|2.68|2.78|-55.24| +2452217|67299|11839|55140|1436722|5951|6780|4|21|2045|62|26.16|50.48|24.73|352.64|1533.26|1621.92|3129.76|11.80|352.64|1180.62|1192.42|-441.30| +2452217|67299|5881|55140|1436722|5951|6780|4|149|2045|73|89.89|177.08|127.49|0.00|9306.77|6561.97|12926.84|744.54|0.00|9306.77|10051.31|2744.80| +2452217|67299|16401|55140|1436722|5951|6780|4|104|2045|32|23.62|28.34|18.70|23.93|598.40|755.84|906.88|22.97|23.93|574.47|597.44|-181.37| +2452217|67299|4475|55140|1436722|5951|6780|4|29|2045|97|28.58|48.87|3.90|0.00|378.30|2772.26|4740.39|3.78|0.00|378.30|382.08|-2393.96| +2452217|67299|3573|55140|1436722|5951|6780|4|208|2045|92|67.04|111.95|50.37|0.00|4634.04|6167.68|10299.40|231.70|0.00|4634.04|4865.74|-1533.64| +2452217|67299|591|55140|1436722|5951|6780|4|293|2045|39|20.98|22.65|18.12|466.40|706.68|818.22|883.35|2.40|466.40|240.28|242.68|-577.94| +||13760||1405557|1100|||86|2046|13||46.73|10.74||139.62|||4.18|||143.80|| +2451238|68375|2300|6814|1405557|1100|3843|10|276|2046|12|42.74|52.14|17.20|0.00|206.40|512.88|625.68|14.44|0.00|206.40|220.84|-306.48| +2451238|68375|16186|6814|1405557|1100|3843|10|74|2046|60|35.59|57.65|34.01|0.00|2040.60|2135.40|3459.00|102.03|0.00|2040.60|2142.63|-94.80| +2451238|68375|7672|6814|1405557|1100|3843|10|129|2046|76|83.58|123.69|3.71|95.86|281.96|6352.08|9400.44|1.86|95.86|186.10|187.96|-6165.98| +2451238|68375|13867|6814|1405557|1100|3843|10|104|2046|7|79.89|99.06|0.99|0.00|6.93|559.23|693.42|0.27|0.00|6.93|7.20|-552.30| +2451238|68375|5071|6814|1405557|1100|3843|10|299|2046|28|19.41|19.99|11.59|0.00|324.52|543.48|559.72|12.98|0.00|324.52|337.50|-218.96| +2451238|68375|4694|6814|1405557|1100|3843|10|288|2046|1|59.23|61.59|56.04|0.00|56.04|59.23|61.59|3.36|0.00|56.04|59.40|-3.19| +2451238|68375|17635|6814|1405557|1100|3843|10|56|2046|6|49.85|90.72|61.68|0.00|370.08|299.10|544.32|14.80|0.00|370.08|384.88|70.98| +2451238|68375|4864|6814|1405557|1100|3843|10|177|2046|73|37.57|45.83|42.16|0.00|3077.68|2742.61|3345.59|184.66|0.00|3077.68|3262.34|335.07| +2451238|68375|12938|6814|1405557|1100|3843|10|232|2046|59|83.10|152.90|85.62|0.00|5051.58|4902.90|9021.10|252.57|0.00|5051.58|5304.15|148.68| +2451238|68375|6758|6814|1405557|1100|3843|10|268|2046|70|18.00|29.52|8.85|0.00|619.50|1260.00|2066.40|24.78|0.00|619.50|644.28|-640.50| +2452206|70263|12019|36646|1825|1404|20630|1|70|2047|66|8.57|10.54|6.85|0.00|452.10|565.62|695.64|31.64|0.00|452.10|483.74|-113.52| +2452206|70263|12235|36646|1825|1404|20630|1|3|2047|30|59.28|107.88|70.12|0.00|2103.60|1778.40|3236.40|63.10|0.00|2103.60|2166.70|325.20| +2452206|70263|17945|36646|1825|1404|20630|1|219|2047|98|47.55|84.63|16.92|281.88|1658.16|4659.90|8293.74|55.05|281.88|1376.28|1431.33|-3283.62| +2452206|70263|10851|36646|1825|1404|20630|1|198|2047|72|3.50|6.65|3.65|0.00|262.80|252.00|478.80|13.14|0.00|262.80|275.94|10.80| +2452206|70263|15929|36646|1825|1404|20630|1|155|2047|4|70.90|85.08|11.06|0.00|44.24|283.60|340.32|1.32|0.00|44.24|45.56|-239.36| +2452206|70263|10647|36646|1825|1404|20630|1|263|2047|85|84.39|91.98|73.58|0.00|6254.30|7173.15|7818.30|125.08|0.00|6254.30|6379.38|-918.85| +2452206|70263|1803|36646|1825|1404|20630|1|251|2047|1|25.20|49.89|29.43|0.00|29.43|25.20|49.89|0.00|0.00|29.43|29.43|4.23| +2452206||10281||1825|1404||1|276|2047||55.07||102.01|||4295.46||44.55||1113.95|1158.50|| +2451524|51018|16528|96114|1730338|4915|4339|7|12|2048|58|45.49|76.42|71.83|0.00|4166.14|2638.42|4432.36|333.29|0.00|4166.14|4499.43|1527.72| +2451524|51018|16222|96114|1730338|4915|4339|7|120|2048|24|80.25|106.73|40.55|690.97|973.20|1926.00|2561.52|0.00|690.97|282.23|282.23|-1643.77| +||13327||1730338||||194|2048|21||173.91|149.56|||||||||1286.88| +2451524|51018|13591|96114|1730338|4915|4339|7|58|2048|92|25.49|50.72|29.41|0.00|2705.72|2345.08|4666.24|162.34|0.00|2705.72|2868.06|360.64| +2451524|51018|256|96114|1730338|4915|4339|7|206|2048|66|27.69|52.05|52.05|274.82|3435.30|1827.54|3435.30|94.81|274.82|3160.48|3255.29|1332.94| +2451524|51018|1502|96114|1730338|4915|4339|7|192|2048|57|77.51|104.63|36.62|0.00|2087.34|4418.07|5963.91|166.98|0.00|2087.34|2254.32|-2330.73| +2451524|51018|2869|96114|1730338|4915|4339|7|47|2048|67|20.47|38.48|20.39|0.00|1366.13|1371.49|2578.16|40.98|0.00|1366.13|1407.11|-5.36| +2451524|51018|464|96114|1730338|4915|4339|7|239|2048|90|53.20|73.41|33.76|0.00|3038.40|4788.00|6606.90|151.92|0.00|3038.40|3190.32|-1749.60| +2451524|51018|4237|96114|1730338|4915|4339|7|204|2048|13|18.63|36.88|23.97|0.00|311.61|242.19|479.44|18.69|0.00|311.61|330.30|69.42| +2451524|51018|1106|96114|1730338|4915|4339|7|236|2048|56|46.34|55.14|20.40|0.00|1142.40|2595.04|3087.84|102.81|0.00|1142.40|1245.21|-1452.64| +2451524|51018|10330|96114|1730338|4915|4339|7|14|2048|10|94.18|109.24|48.06|0.00|480.60|941.80|1092.40|38.44|0.00|480.60|519.04|-461.20| +2451524|51018|8924|96114|1730338|4915|4339|7|166|2048|85|64.11|98.08|44.13|0.00|3751.05|5449.35|8336.80|187.55|0.00|3751.05|3938.60|-1698.30| +2451524|51018|4567|96114|1730338|4915|4339|7|253|2048|1|28.14|36.30|14.52|10.45|14.52|28.14|36.30|0.36|10.45|4.07|4.43|-24.07| +2451524|51018|16462|96114|1730338|4915|4339|7|57|2048|73|12.53|23.18|14.37|0.00|1049.01|914.69|1692.14|31.47|0.00|1049.01|1080.48|134.32| +2450958|63194|12766|22479|1226917|4508|8499|2|105|2049|35|41.20|59.32|31.43|0.00|1100.05|1442.00|2076.20|88.00|0.00|1100.05|1188.05|-341.95| +2450958|63194|1690|22479|1226917|4508|8499|2|239|2049|6|94.00|135.36|48.72|0.00|292.32|564.00|812.16|11.69|0.00|292.32|304.01|-271.68| +2450958|63194|14332|22479|1226917|4508|8499|2|210|2049|93|16.23|25.15|3.01|0.00|279.93|1509.39|2338.95|19.59|0.00|279.93|299.52|-1229.46| +2450958|63194|15373|22479|1226917|4508|8499|2|41|2049|93|1.84|1.95|1.05|97.65|97.65|171.12|181.35|0.00|97.65|0.00|0.00|-171.12| +2450958|63194|8372|22479|1226917|4508|8499|2|290|2049|13|76.98|137.79|135.03|0.00|1755.39|1000.74|1791.27|105.32|0.00|1755.39|1860.71|754.65| +2450958|63194|15805|22479|1226917|4508|8499|2|156|2049|77|31.56|53.65|46.13|0.00|3552.01|2430.12|4131.05|248.64|0.00|3552.01|3800.65|1121.89| +2450958|63194|5506|22479|1226917|4508|8499|2|105|2049|7|32.55|50.77|31.47|0.00|220.29|227.85|355.39|2.20|0.00|220.29|222.49|-7.56| +2450958|63194|6907|22479|1226917|4508|8499|2|203|2049|95|68.35|102.52|86.11|0.00|8180.45|6493.25|9739.40|490.82|0.00|8180.45|8671.27|1687.20| +2450958|63194|6103|22479|1226917|4508|8499|2|194|2049|73|25.59|27.89|16.17|0.00|1180.41|1868.07|2035.97|35.41|0.00|1180.41|1215.82|-687.66| +2450958|63194|14074|22479|1226917|4508|8499|2|151|2049|48|44.03|59.88|49.70|0.00|2385.60|2113.44|2874.24|95.42|0.00|2385.60|2481.02|272.16| +2452232|42486|737|7352|1139198|4644|28090|1|226|2050|94|72.78|119.35|32.22|0.00|3028.68|6841.32|11218.90|30.28|0.00|3028.68|3058.96|-3812.64| +2452232|42486|16985|7352|1139198|4644|28090|1|247|2050|16|56.38|94.71|53.98|0.00|863.68|902.08|1515.36|51.82|0.00|863.68|915.50|-38.40| +2452232|42486|14959|7352|1139198|4644|28090|1|286|2050|87|8.73|9.77|3.02|0.00|262.74|759.51|849.99|2.62|0.00|262.74|265.36|-496.77| +2452232|42486|14215|7352|1139198|4644|28090|1|100|2050|76|1.05|1.41|0.94|0.00|71.44|79.80|107.16|0.00|0.00|71.44|71.44|-8.36| +2452232|42486|7413|7352|1139198|4644|28090|1|240|2050|66|98.29|147.43|116.46|0.00|7686.36|6487.14|9730.38|691.77|0.00|7686.36|8378.13|1199.22| +2452232|42486|4373|7352|1139198|4644|28090|1|96|2050|20|6.09|12.18|4.62|83.16|92.40|121.80|243.60|0.46|83.16|9.24|9.70|-112.56| +2452232|42486|16547|7352|1139198|4644|28090|1|203|2050|22|21.28|28.30|24.33|69.58|535.26|468.16|622.60|4.65|69.58|465.68|470.33|-2.48| +2452232|42486|17007|7352|1139198|4644|28090|1|153|2050|3|41.90|67.87|13.57|0.00|40.71|125.70|203.61|0.81|0.00|40.71|41.52|-84.99| +2452232|42486|9589|7352|1139198|4644|28090|1|97|2050|19|32.77|64.55|4.51|0.00|85.69|622.63|1226.45|5.99|0.00|85.69|91.68|-536.94| +2452232|42486|12145|7352|1139198|4644|28090|1|145|2050|96|12.50|21.00|17.85|0.00|1713.60|1200.00|2016.00|102.81|0.00|1713.60|1816.41|513.60| +2452232|42486|5209|7352|1139198|4644|28090|1|280|2050|4|57.88|109.97|42.88|0.00|171.52|231.52|439.88|8.57|0.00|171.52|180.09|-60.00| +2452232|42486|7865|7352|1139198|4644|28090|1|50|2050|20|98.32|115.03|48.31|0.00|966.20|1966.40|2300.60|67.63|0.00|966.20|1033.83|-1000.20| +2452232|42486|15697|7352|1139198|4644|28090|1|185|2050|62|59.48|72.56|45.71|0.00|2834.02|3687.76|4498.72|0.00|0.00|2834.02|2834.02|-853.74| +2451039|73021|16022|16583|37976|1322|14315|8|211|2051|81|57.00|111.15|73.35|0.00|5941.35|4617.00|9003.15|0.00|0.00|5941.35|5941.35|1324.35| +2451039|73021|13034|16583|37976|1322|14315|8|22|2051|89|88.98|134.35|92.70|0.00|8250.30|7919.22|11957.15|0.00|0.00|8250.30|8250.30|331.08| +||3370|16583||1322|14315|8||2051|9|68.90|||0.00|||||0.00||936.75|| +2451039|73021|13633|16583|37976|1322|14315|8|59|2051|77|41.43|50.54|35.88|2127.32|2762.76|3190.11|3891.58|0.00|2127.32|635.44|635.44|-2554.67| +2451039|73021|14584|16583|37976|1322|14315|8|121|2051|48|83.47|100.99|83.82|0.00|4023.36|4006.56|4847.52|201.16|0.00|4023.36|4224.52|16.80| +2451039|73021|9865|16583|37976|1322|14315|8|199|2051|10|62.99|69.28|16.62|0.00|166.20|629.90|692.80|14.95|0.00|166.20|181.15|-463.70| +2451039|73021|6878|16583|37976|1322|14315|8|235|2051|57|64.98|95.52|12.41|0.00|707.37|3703.86|5444.64|42.44|0.00|707.37|749.81|-2996.49| +2451039|73021|17486|16583|37976|1322|14315|8|2|2051|8|24.46|44.02|14.96|0.00|119.68|195.68|352.16|3.59|0.00|119.68|123.27|-76.00| +2451039|73021|1148|16583|37976|1322|14315|8|33|2051|13|77.70|100.23|13.02|123.55|169.26|1010.10|1302.99|0.45|123.55|45.71|46.16|-964.39| +2451039|73021|12109|16583|37976|1322|14315|8|290|2051|16|43.68|87.36|13.10|0.00|209.60|698.88|1397.76|10.48|0.00|209.60|220.08|-489.28| +|73021|1858|16583||1322|||152|2051||60.39||||4869.00|3623.40|5072.40|||||1245.60| +2451039|73021|7759|16583|37976|1322|14315|8|72|2051|97|76.54|97.20|21.38|0.00|2073.86|7424.38|9428.40|20.73|0.00|2073.86|2094.59|-5350.52| +2451039|73021|5702|||||8|178|2051|||25.50|20.65|0.00|1321.60|||52.86|0.00|1321.60|1374.46|| +2451865|39151|4529|3489|706730|5171|59|10|75|2052|86|30.00|48.30|47.81|0.00|4111.66|2580.00|4153.80|123.34|0.00|4111.66|4235.00|1531.66| +2451865|39151|16145|3489|706730|5171|59|10|74|2052|16|86.05|150.58|15.05|14.44|240.80|1376.80|2409.28|6.79|14.44|226.36|233.15|-1150.44| +2451865|39151|9025|3489|706730|5171|59|10|54|2052|8|22.45|33.00|1.32|0.00|10.56|179.60|264.00|0.42|0.00|10.56|10.98|-169.04| +2451865|39151|5411|3489|706730|5171|59|10|263|2052|66|12.76|18.75|3.56|0.00|234.96|842.16|1237.50|0.00|0.00|234.96|234.96|-607.20| +2451865|39151|14837|3489|706730|5171|59|10|274|2052|60|49.30|76.41|51.19|0.00|3071.40|2958.00|4584.60|276.42|0.00|3071.40|3347.82|113.40| +2451865||2777|3489|706730|5171|59|||2052||23.23||33.36||2335.20|1626.10||||2335.20||709.10| +2451865|39151|6920|3489|706730|5171|59|10|205|2052|51|6.47|12.09|9.55|0.00|487.05|329.97|616.59|4.87|0.00|487.05|491.92|157.08| +2451865|39151|3914|3489|706730|5171|59|10|175|2052|66|54.42|89.24|28.55|0.00|1884.30|3591.72|5889.84|37.68|0.00|1884.30|1921.98|-1707.42| +2451865|39151|10238|3489|706730|5171|59|10|29|2052|55|55.32|96.25|3.85|0.00|211.75|3042.60|5293.75|2.11|0.00|211.75|213.86|-2830.85| +2451865|39151|7201|3489|706730|5171|59|10|139|2052|51|16.16|23.10|8.08|0.00|412.08|824.16|1178.10|12.36|0.00|412.08|424.44|-412.08| +2451865|39151|445|3489|706730|5171|59|10|197|2052|8|6.89|7.85|4.71|0.00|37.68|55.12|62.80|2.63|0.00|37.68|40.31|-17.44| +2451865|39151|16025|3489|706730|5171|59|10|189|2052|91|6.37|10.44|3.13|0.00|284.83|579.67|950.04|5.69|0.00|284.83|290.52|-294.84| +2451865|39151|7652|3489|706730|5171|59|10|138|2052|65|74.54|80.50|19.32|0.00|1255.80|4845.10|5232.50|12.55|0.00|1255.80|1268.35|-3589.30| +2451865|39151|12557|3489|706730|5171|59|10|32|2052|78|41.31|55.35|46.49|0.00|3626.22|3222.18|4317.30|145.04|0.00|3626.22|3771.26|404.04| +2452431|62170|10833|15327|1314624|3521|10018|4|253|2053|79|10.55|16.45|4.93|0.00|389.47|833.45|1299.55|15.57|0.00|389.47|405.04|-443.98| +2452431|62170|4620|15327|1314624|3521|10018|4|155|2053|53|41.41|53.41|24.03|0.00|1273.59|2194.73|2830.73|12.73|0.00|1273.59|1286.32|-921.14| +2452431|62170|7927|15327|1314624|3521|10018|4|297|2053|4|33.73|64.42|33.49|0.00|133.96|134.92|257.68|12.05|0.00|133.96|146.01|-0.96| +2452431|62170|10677|15327|1314624|3521|10018|4|118|2053|57|33.89|52.52|6.30|0.00|359.10|1931.73|2993.64|28.72|0.00|359.10|387.82|-1572.63| +2452431|62170|13573|15327|1314624|3521|10018|4|281|2053|6|10.95|13.03|8.46|0.00|50.76|65.70|78.18|1.52|0.00|50.76|52.28|-14.94| +2452431|62170|2340|15327|1314624|3521|10018|4|43|2053|59|73.37|78.50|42.39|0.00|2501.01|4328.83|4631.50|150.06|0.00|2501.01|2651.07|-1827.82| +2452431|62170|5274|15327|1314624|3521|10018|4|241|2053|38|7.23|10.48|9.01|78.74|342.38|274.74|398.24|13.18|78.74|263.64|276.82|-11.10| +2452431|62170|14520|15327|1314624|3521|10018|4|293|2053|59|76.18|132.55|41.09|0.00|2424.31|4494.62|7820.45|169.70|0.00|2424.31|2594.01|-2070.31| +2452431|62170|15063|15327|1314624|3521|10018|4|193|2053|19|46.18|78.96|31.58|0.00|600.02|877.42|1500.24|0.00|0.00|600.02|600.02|-277.40| +2452431|62170|12259|15327|1314624|3521|10018|4|180|2053|49|61.48|73.77|68.60|0.00|3361.40|3012.52|3614.73|0.00|0.00|3361.40|3361.40|348.88| +2452431|62170|14647|15327|1314624|3521|10018|4|30|2053|5|28.84|29.12|0.87|0.00|4.35|144.20|145.60|0.08|0.00|4.35|4.43|-139.85| +2451909|30708|7346|88874|1105534|4241|21701|10|251|2054|88|70.71|74.24|37.12|0.00|3266.56|6222.48|6533.12|228.65|0.00|3266.56|3495.21|-2955.92| +2451909|30708|15434|88874|1105534|4241|21701|10|179|2054|77|8.41|14.80|5.92|414.81|455.84|647.57|1139.60|3.69|414.81|41.03|44.72|-606.54| +||3035|88874|1105534|4241||||2054||59.61|105.50|49.58|0.00||1728.69|3059.50|14.37|0.00|1437.82|1452.19|-290.87| +2451909|30708|16214|88874|1105534|4241|21701|10|236|2054|90|74.98|132.71|14.59|577.76|1313.10|6748.20|11943.90|14.70|577.76|735.34|750.04|-6012.86| +2451909|30708|3218|88874|1105534|4241|21701|10|124|2054|61|64.61|107.89|44.23|0.00|2698.03|3941.21|6581.29|188.86|0.00|2698.03|2886.89|-1243.18| +|30708|11618|88874|1105534|4241|21701|10||2054||||36.97|0.00|3031.54|4709.26|||0.00||3061.85|| +2451909|30708|5984|88874|1105534|4241|21701|10|54|2054|20|14.58|16.03|4.64|0.00|92.80|291.60|320.60|3.71|0.00|92.80|96.51|-198.80| +2451909|30708|13520|88874|1105534|4241|21701|10|104|2054|21|22.29|27.86|17.55|0.00|368.55|468.09|585.06|25.79|0.00|368.55|394.34|-99.54| +2451916|61472|4921|47781|438843|3505|30567|10|124|2055|8|18.96|20.47|14.73|0.00|117.84|151.68|163.76|1.17|0.00|117.84|119.01|-33.84| +2451916|61472|9713|47781|438843|3505|30567|10|165|2055|39|97.71|174.90|153.91|1860.77|6002.49|3810.69|6821.10|331.33|1860.77|4141.72|4473.05|331.03| +2451916|61472|5403|47781|438843|3505|30567|10|133|2055|40|9.31|14.80|11.84|0.00|473.60|372.40|592.00|37.88|0.00|473.60|511.48|101.20| +2451916|61472|15401|47781|438843|3505|30567|10|11|2055|89|36.41|64.08|62.79|4358.88|5588.31|3240.49|5703.12|12.29|4358.88|1229.43|1241.72|-2011.06| +2451916|61472|6971|47781|438843|3505|30567|10|258|2055|49|81.04|156.40|145.45|2423.19|7127.05|3970.96|7663.60|47.03|2423.19|4703.86|4750.89|732.90| +2451916|61472|3129|47781|438843|3505|30567|10|43|2055|20|13.23|16.66|5.33|0.00|106.60|264.60|333.20|1.06|0.00|106.60|107.66|-158.00| +2451916|61472|16249|47781|438843|3505|30567|10|189|2055|94|58.34|60.09|48.67|0.00|4574.98|5483.96|5648.46|182.99|0.00|4574.98|4757.97|-908.98| +2451916|61472|15723|47781|438843|3505|30567|10|166|2055|89|42.41|69.97|55.97|0.00|4981.33|3774.49|6227.33|398.50|0.00|4981.33|5379.83|1206.84| +2451832||7849|12644|||||270|2056|7||||0.00||230.23||19.85|0.00|||53.41| +|38987|7910||1099749||43159|||2056|10|98.18|133.52||0.00|761.00|981.80||68.49|0.00|||| +2451832|38987|12277|12644|1099749|3654|43159|10|70|2056|81|43.59|63.20|28.44|1197.89|2303.64|3530.79|5119.20|99.51|1197.89|1105.75|1205.26|-2425.04| +2451832|38987|14551|12644|1099749|3654|43159|10|151|2056|25|66.28|72.24|35.39|0.00|884.75|1657.00|1806.00|44.23|0.00|884.75|928.98|-772.25| +2451832|38987|16358|12644|1099749|3654|43159|10|83|2056|12|1.86|3.05|2.44|0.00|29.28|22.32|36.60|2.34|0.00|29.28|31.62|6.96| +2451832|38987|13027|12644|1099749|3654|43159|10|189|2056|73|10.62|14.01|12.60|0.00|919.80|775.26|1022.73|55.18|0.00|919.80|974.98|144.54| +2451832|38987|3325|12644|1099749|3654|43159|10|28|2056|29|52.27|97.22|87.49|0.00|2537.21|1515.83|2819.38|0.00|0.00|2537.21|2537.21|1021.38| +2451832|38987|5636|12644|1099749|3654|43159|10|42|2056|56|85.57|85.57|11.97|0.00|670.32|4791.92|4791.92|26.81|0.00|670.32|697.13|-4121.60| +2451832|38987|8759|12644|1099749|3654|43159|10|250|2056|39|70.06|95.28|67.64|0.00|2637.96|2732.34|3715.92|0.00|0.00|2637.96|2637.96|-94.38| +2451832|38987|1367|12644|1099749|3654|43159|10|187|2056|13|80.56|134.53|126.45|0.00|1643.85|1047.28|1748.89|0.00|0.00|1643.85|1643.85|596.57| +2451832|38987|17096|12644|1099749|3654|43159|10|179|2056|18|64.88|123.92|110.28|0.00|1985.04|1167.84|2230.56|138.95|0.00|1985.04|2123.99|817.20| +|38987|14963|||3654|||193|2056|28||||0.00|1667.40|2140.60|||0.00|||| +2450980|52533|4490|49136|1831625|3267|5756|4|178|2057|60|40.60|65.36|20.91|0.00|1254.60|2436.00|3921.60|112.91|0.00|1254.60|1367.51|-1181.40| +2450980|52533|16364|49136|1831625|3267|5756|4|268|2057|68|43.54|74.01|24.42|0.00|1660.56|2960.72|5032.68|132.84|0.00|1660.56|1793.40|-1300.16| +2450980|52533|12314|49136|1831625|3267|5756|4|65|2057|69|77.51|131.76|125.17|0.00|8636.73|5348.19|9091.44|86.36|0.00|8636.73|8723.09|3288.54| +2450980|52533|8131|49136|1831625|3267|5756|4|142|2057|49|94.40|134.99|83.69|3485.68|4100.81|4625.60|6614.51|36.90|3485.68|615.13|652.03|-4010.47| +2450980|52533|9596|49136|1831625|3267|5756|4|208|2057|88|29.65|33.50|7.70|0.00|677.60|2609.20|2948.00|54.20|0.00|677.60|731.80|-1931.60| +2450980|52533|2672|49136|1831625|3267|5756|4|206|2057|62|36.47|41.94|6.71|0.00|416.02|2261.14|2600.28|37.44|0.00|416.02|453.46|-1845.12| +2450980|52533|16672|49136|1831625|3267|5756|4|97|2057|13|83.84|106.47|28.74|0.00|373.62|1089.92|1384.11|33.62|0.00|373.62|407.24|-716.30| +2450980|52533|15211|49136|1831625|3267|5756|4|79|2057|1|6.31|10.28|3.08|0.89|3.08|6.31|10.28|0.02|0.89|2.19|2.21|-4.12| +2450980|52533|8209|49136|1831625|3267|5756|4|49|2057|5|80.95|80.95|22.66|0.00|113.30|404.75|404.75|2.26|0.00|113.30|115.56|-291.45| +|52533|13747|||3267||||2057|90||||0.00||3239.10|||0.00|3691.80|3987.14|| +2450980|52533|11924|49136|1831625|3267|5756|4|234|2057|77|23.45|44.55|30.73|0.00|2366.21|1805.65|3430.35|212.95|0.00|2366.21|2579.16|560.56| +2450980|52533|4370|49136|1831625|3267|5756|4|199|2057|100|50.66|85.10|83.39|0.00|8339.00|5066.00|8510.00|667.12|0.00|8339.00|9006.12|3273.00| +2450980|52533|6532|49136|1831625|3267|5756|4|163|2057|10|60.78|90.56|5.43|20.09|54.30|607.80|905.60|2.05|20.09|34.21|36.26|-573.59| +2450980|52533|1288|49136|1831625|3267|5756|4|251|2057|50|90.64|124.17|57.11|0.00|2855.50|4532.00|6208.50|142.77|0.00|2855.50|2998.27|-1676.50| +2450980|52533|13796|49136|1831625|3267|5756|4|4|2057|91|96.47|175.57|12.28|0.00|1117.48|8778.77|15976.87|22.34|0.00|1117.48|1139.82|-7661.29| +2452276|49927|9235|69895|234906||36128||45|2058|41|49.33||18.08||741.28||2851.55||||793.16|| +2452276|49927|11871|69895|234906|6317|36128|2|1|2058|89|46.83|90.85|10.90|0.00|970.10|4167.87|8085.65|29.10|0.00|970.10|999.20|-3197.77| +2452276|49927|3225|69895|234906|6317|36128|2|63|2058|8|68.21|72.30|67.23|0.00|537.84|545.68|578.40|21.51|0.00|537.84|559.35|-7.84| +2452276|49927|283|69895|234906|6317|36128|2|278|2058|16|71.29|132.59|2.65|0.00|42.40|1140.64|2121.44|0.42|0.00|42.40|42.82|-1098.24| +2452276|49927|8799|69895|234906|6317|36128|2|160|2058|42|42.93|50.65|38.49|0.00|1616.58|1803.06|2127.30|113.16|0.00|1616.58|1729.74|-186.48| +2452276|49927|13440|69895|234906|6317|36128|2|133|2058|74|12.41|17.74|17.03|781.33|1260.22|918.34|1312.76|38.31|781.33|478.89|517.20|-439.45| +2452276|49927|13681|69895|234906|6317|36128|2|41|2058|94|96.09|169.11|33.82|0.00|3179.08|9032.46|15896.34|222.53|0.00|3179.08|3401.61|-5853.38| +2452276|49927|8323|69895|234906|6317|36128|2|47|2058|9|74.86|76.35|64.13|0.00|577.17|673.74|687.15|11.54|0.00|577.17|588.71|-96.57| +2452276|49927|7491|69895|234906|6317|36128|2|204|2058|94|94.21|96.09|23.06|0.00|2167.64|8855.74|9032.46|21.67|0.00|2167.64|2189.31|-6688.10| +2452276|49927|5928|69895|234906|6317|36128|2|61|2058|34|36.84|48.26|13.03|0.00|443.02|1252.56|1640.84|39.87|0.00|443.02|482.89|-809.54| +2452212|61828|15881|10614|1231840|1866|2292|7|178|2059|55|27.31|51.61|2.58|0.00|141.90|1502.05|2838.55|8.51|0.00|141.90|150.41|-1360.15| +2452212|61828|5265|10614|1231840|1866|2292|7|159|2059|48|91.48|172.89|165.97|0.00|7966.56|4391.04|8298.72|477.99|0.00|7966.56|8444.55|3575.52| +2452212|61828|11035|10614|1231840|1866|2292|7|50|2059|82|91.55|107.11|80.33|0.00|6587.06|7507.10|8783.02|0.00|0.00|6587.06|6587.06|-920.04| +2452212|61828|7195|10614|1231840|1866|2292|7|250|2059|93|88.83|117.25|65.66|0.00|6106.38|8261.19|10904.25|427.44|0.00|6106.38|6533.82|-2154.81| +||14995|||1866|2292||155|2059|96|||94.13||9036.48|7368.96||||9036.48|9217.20|| +2452212|61828|1699|10614|1231840|1866|2292|7|154|2059|21|49.21|98.42|56.09|0.00|1177.89|1033.41|2066.82|94.23|0.00|1177.89|1272.12|144.48| +2452212|61828|1401|10614|1231840|1866|2292|7|215|2059|47|16.47|20.25|5.46|0.00|256.62|774.09|951.75|5.13|0.00|256.62|261.75|-517.47| +2452212|61828|17411|10614|1231840|1866|2292|7|123|2059|3|99.07|181.29|135.96|0.00|407.88|297.21|543.87|8.15|0.00|407.88|416.03|110.67| +2452212|61828|4893|10614|1231840|1866|2292|7|143|2059|33|59.46|104.64|55.45|0.00|1829.85|1962.18|3453.12|164.68|0.00|1829.85|1994.53|-132.33| +2452377|67967|6997|39897|1234923|6749||2||2060|||||||||||||| +2452377|67967|1920|39897|1234923|6749|10222|2|283|2060|32|87.92|143.30|53.02|0.00|1696.64|2813.44|4585.60|16.96|0.00|1696.64|1713.60|-1116.80| +2452377|67967|1539|39897|1234923|6749|10222|2|96|2060|52|46.58|84.30|83.45|0.00|4339.40|2422.16|4383.60|43.39|0.00|4339.40|4382.79|1917.24| +2452377|67967|60|39897|1234923|6749|10222|2|61|2060|70|49.72|84.52|48.17|2933.55|3371.90|3480.40|5916.40|26.30|2933.55|438.35|464.65|-3042.05| +2452377|67967|12525|39897|1234923|6749|10222|2|88|2060|45|20.18|33.49|14.40|0.00|648.00|908.10|1507.05|45.36|0.00|648.00|693.36|-260.10| +2452377|67967|297|39897|1234923|6749|10222|2|128|2060|52|7.97|13.78|3.99|0.00|207.48|414.44|716.56|8.29|0.00|207.48|215.77|-206.96| +2452377|67967|7176|39897|1234923|6749|10222|2|51|2060|37|56.43|64.33|57.25|0.00|2118.25|2087.91|2380.21|84.73|0.00|2118.25|2202.98|30.34| +2452377|67967|17715|39897|1234923|6749|10222|2|207|2060|80|55.76|95.90|19.18|859.26|1534.40|4460.80|7672.00|60.76|859.26|675.14|735.90|-3785.66| +2452377|67967|12339|39897|1234923|6749|10222|2|260|2060|88|32.51|59.16|55.01|0.00|4840.88|2860.88|5206.08|48.40|0.00|4840.88|4889.28|1980.00| +2452377|67967|10308|39897|1234923|6749|10222|2|201|2060|43|71.28|99.07|52.50|0.00|2257.50|3065.04|4260.01|112.87|0.00|2257.50|2370.37|-807.54| +2452377|67967|4032|39897|1234923|6749|10222|2|168|2060|8|74.96|95.94|56.60|0.00|452.80|599.68|767.52|27.16|0.00|452.80|479.96|-146.88| +2452377|67967|17463|39897|1234923|6749|10222|2|109|2060|13|11.66|20.63|16.29|0.00|211.77|151.58|268.19|8.47|0.00|211.77|220.24|60.19| +2452377|67967|16245|39897|1234923|6749|10222|2|92|2060|67|34.93|39.82|5.57|0.00|373.19|2340.31|2667.94|3.73|0.00|373.19|376.92|-1967.12| +2452377|67967|10425|39897|1234923|6749|10222|2|7|2060|53|17.84|26.22|11.01|0.00|583.53|945.52|1389.66|52.51|0.00|583.53|636.04|-361.99| +2452130|60457|475|82675|486373|4471|40969|1|254|2061|95|68.22|102.33|92.09|0.00|8748.55|6480.90|9721.35|174.97|0.00|8748.55|8923.52|2267.65| +2452130|60457|12475|82675|486373|4471|40969|1|215|2061|64|59.18|100.60|76.45|0.00|4892.80|3787.52|6438.40|97.85|0.00|4892.80|4990.65|1105.28| +2452130|60457|16557|82675|486373|4471|40969|1|28|2061|74|64.56|105.87|82.57|0.00|6110.18|4777.44|7834.38|61.10|0.00|6110.18|6171.28|1332.74| +2452130|60457|12919|82675|486373|4471|40969|1|232|2061|9|89.09|143.43|18.64|0.00|167.76|801.81|1290.87|13.42|0.00|167.76|181.18|-634.05| +2452130|60457|13735|82675|486373|4471|40969|1|60|2061|47|27.11|50.96|0.00|0.00|0.00|1274.17|2395.12|0.00|0.00|0.00|0.00|-1274.17| +2452130|60457|2801|82675|486373|4471|40969|1|65|2061|74|99.44|178.99|30.42|0.00|2251.08|7358.56|13245.26|67.53|0.00|2251.08|2318.61|-5107.48| +2452130|60457|14249|82675|486373|4471|40969|1|110|2061|29|22.44|27.37|18.88|0.00|547.52|650.76|793.73|49.27|0.00|547.52|596.79|-103.24| +2452130|60457|13505|82675|486373|4471|40969|1|165|2061|29|54.84|78.42|24.31|0.00|704.99|1590.36|2274.18|35.24|0.00|704.99|740.23|-885.37| +2452130|60457|17855|82675|486373|4471|40969|1|197|2061|53|23.39|35.55|12.44|501.08|659.32|1239.67|1884.15|7.91|501.08|158.24|166.15|-1081.43| +2452130|60457|3017|82675|486373|4471|40969|1|194|2061|44|46.79|78.60|60.52|0.00|2662.88|2058.76|3458.40|159.77|0.00|2662.88|2822.65|604.12| +2452130|60457|4015|82675|486373|4471|40969|1|179|2061|39|68.93|95.81|38.32|0.00|1494.48|2688.27|3736.59|119.55|0.00|1494.48|1614.03|-1193.79| +2452130|60457|4549|82675|486373|4471|40969|1|145|2061|88|56.42|90.27|77.63|0.00|6831.44|4964.96|7943.76|204.94|0.00|6831.44|7036.38|1866.48| +2452130|60457|9249|82675|486373|4471|40969|1|6|2061|44|91.22|159.63|118.12|0.00|5197.28|4013.68|7023.72|103.94|0.00|5197.28|5301.22|1183.60| +2452130|60457|15357|82675|486373|4471|40969|1|51|2061|31|43.45|86.03|30.97|700.85|960.07|1346.95|2666.93|23.32|700.85|259.22|282.54|-1087.73| +2451703|48927|5738|40467|1616578|1936|33247|4|44|2062|100|84.08|150.50|144.48|0.00|14448.00|8408.00|15050.00|1011.36|0.00|14448.00|15459.36|6040.00| +2451703|48927|7961|40467|1616578|1936|33247|4|234|2062|21|37.71|73.91|8.13|0.00|170.73|791.91|1552.11|5.12|0.00|170.73|175.85|-621.18| +2451703|48927|5593|40467|1616578|1936|33247|4|242|2062|60|66.38|86.95|62.60|0.00|3756.00|3982.80|5217.00|75.12|0.00|3756.00|3831.12|-226.80| +2451703|48927|16829|40467|1616578|1936|33247|4|9|2062|95|28.88|28.88|8.66|0.00|822.70|2743.60|2743.60|24.68|0.00|822.70|847.38|-1920.90| +2451703|48927|16265|40467|1616578|1936|33247|4|236|2062|83|2.68|4.55|0.36|0.00|29.88|222.44|377.65|2.68|0.00|29.88|32.56|-192.56| +2451703|48927|2330|40467|1616578|1936|33247|4|32|2062|7|39.30|74.27|47.53|66.54|332.71|275.10|519.89|2.66|66.54|266.17|268.83|-8.93| +2451703|48927|4097|40467|1616578|1936|33247|4|15|2062|43|41.73|45.48|41.38|0.00|1779.34|1794.39|1955.64|53.38|0.00|1779.34|1832.72|-15.05| +||7502|40467|1616578||33247|||2062|84|62.64||24.42|0.00|||10259.76||0.00|2051.28|2194.86|-3210.48| +2451703|48927|5243|40467|1616578|1936|33247|4|295|2062|37|11.56|17.22|0.17|2.20|6.29|427.72|637.14|0.08|2.20|4.09|4.17|-423.63| +2451703|48927|3235|40467|1616578|1936|33247|4|109|2062|26|42.48|57.34|51.60|0.00|1341.60|1104.48|1490.84|40.24|0.00|1341.60|1381.84|237.12| +2451703|48927|10316|40467|1616578|1936|33247|4|19|2062|92|4.64|9.23|7.47|0.00|687.24|426.88|849.16|6.87|0.00|687.24|694.11|260.36| +2451703|48927|11465|40467|1616578|1936|33247|4|20|2062|50|76.88|127.62|48.49|0.00|2424.50|3844.00|6381.00|121.22|0.00|2424.50|2545.72|-1419.50| +2451873|35942|11288|76674|1771834|3347|36860|1|130|2063|75|68.48|115.04|60.97|0.00|4572.75|5136.00|8628.00|91.45|0.00|4572.75|4664.20|-563.25| +2451873|35942|473|76674|1771834|3347|36860|1|127|2063|70|92.95|147.79|20.69|0.00|1448.30|6506.50|10345.30|0.00|0.00|1448.30|1448.30|-5058.20| +2451873|35942|13607|76674|1771834|3347|36860|1|32|2063|3|45.55|46.46|11.61|0.00|34.83|136.65|139.38|1.74|0.00|34.83|36.57|-101.82| +2451873|35942|3625|76674|1771834|3347|36860|1|279|2063|30|87.85|157.25|28.30|0.00|849.00|2635.50|4717.50|50.94|0.00|849.00|899.94|-1786.50| +2451873|35942|3967|76674|1771834|3347|36860|1|277|2063|36|45.42|77.21|36.28|0.00|1306.08|1635.12|2779.56|117.54|0.00|1306.08|1423.62|-329.04| +2451873|35942|8897|76674|1771834|3347|36860|1|115|2063|66|70.18|124.92|107.43|0.00|7090.38|4631.88|8244.72|425.42|0.00|7090.38|7515.80|2458.50| +2451873|35942|9209|76674|1771834|3347|36860|1|234|2063|17|83.38|159.25|54.14|414.17|920.38|1417.46|2707.25|25.31|414.17|506.21|531.52|-911.25| +2451873|35942|968|76674|1771834|3347|36860|1|239|2063|27|36.79|64.38|24.46|0.00|660.42|993.33|1738.26|6.60|0.00|660.42|667.02|-332.91| +2451873|35942|3211|76674|1771834|3347|36860|1|226|2063|41|16.20|30.61|13.16|0.00|539.56|664.20|1255.01|0.00|0.00|539.56|539.56|-124.64| +2451873|35942|2552|76674|1771834|3347|36860|1|268|2063|99|13.78|15.57|9.65|0.00|955.35|1364.22|1541.43|85.98|0.00|955.35|1041.33|-408.87| +2451873|35942|14251|76674|1771834|3347|36860|1|104|2063|83|75.63|80.92|49.36|0.00|4096.88|6277.29|6716.36|327.75|0.00|4096.88|4424.63|-2180.41| +2451873|35942|17357|76674|1771834|3347|36860|1|139|2063|3|79.25|148.19|42.97|73.47|128.91|237.75|444.57|1.66|73.47|55.44|57.10|-182.31| +2451873|35942|2078|76674|1771834|3347|36860|1|159|2063|42|22.95|41.08|23.00|125.58|966.00|963.90|1725.36|0.00|125.58|840.42|840.42|-123.48| +2452634|32958|12588|39575|237929||16237|||2064||33.34|66.01||632.01||1266.92||4.91|632.01|245.79|250.70|| +2452634|32958|13069|39575|237929|5883|16237|2|22|2064|31|54.78|61.35|23.31|0.00|722.61|1698.18|1901.85|57.80|0.00|722.61|780.41|-975.57| +2452634|32958|4507|39575|237929|5883|16237|2|189|2064|14|35.25|43.71|34.96|0.00|489.44|493.50|611.94|14.68|0.00|489.44|504.12|-4.06| +2452634|32958|13845|39575|237929|5883|16237|2|228|2064|57|10.89|19.16|16.09|0.00|917.13|620.73|1092.12|9.17|0.00|917.13|926.30|296.40| +2452634|32958|10176|39575|237929|5883|16237|2|188|2064|62|92.98|94.83|63.53|0.00|3938.86|5764.76|5879.46|157.55|0.00|3938.86|4096.41|-1825.90| +2452634|32958|769|39575|237929|5883|16237|2|182|2064|62|51.33|84.18|23.57|0.00|1461.34|3182.46|5219.16|73.06|0.00|1461.34|1534.40|-1721.12| +2452634|32958|11418|39575|237929|5883|16237|2|14|2064|50|26.29|37.85|23.08|0.00|1154.00|1314.50|1892.50|69.24|0.00|1154.00|1223.24|-160.50| +2452634|32958|7254|39575|237929|5883|16237|2|48|2064|100|12.37|18.06|0.18|0.00|18.00|1237.00|1806.00|0.90|0.00|18.00|18.90|-1219.00| +2452634|32958|6247|39575|237929|5883|16237|2|152|2064|40|45.24|60.62|32.12|0.00|1284.80|1809.60|2424.80|115.63|0.00|1284.80|1400.43|-524.80| +2452151|52079|5331|31026|964375|2171|33167|7|217|2065|41|46.53|80.03|36.81|0.00|1509.21|1907.73|3281.23|30.18|0.00|1509.21|1539.39|-398.52| +2452151|52079|1703|31026|964375|2171|33167|7|173|2065|68|86.02|121.28|99.44|4800.96|6761.92|5849.36|8247.04|58.82|4800.96|1960.96|2019.78|-3888.40| +2452151|52079|1775|31026|964375|2171|33167|7|196|2065|57|10.89|19.92|13.14|0.00|748.98|620.73|1135.44|52.42|0.00|748.98|801.40|128.25| +2452151||13701||||33167|7|238|2065||95.08|114.09||0.00||||497.62|0.00|7108.92|7606.54|| +2452151|52079|15513|31026|964375|2171|33167|7|62|2065|15|65.66|118.84|96.26|1285.07|1443.90|984.90|1782.60|12.70|1285.07|158.83|171.53|-826.07| +2452151|52079|10421|31026|964375|2171|33167|7|174|2065|39|73.59|111.12|30.00|0.00|1170.00|2870.01|4333.68|35.10|0.00|1170.00|1205.10|-1700.01| +2452151|52079|13099|31026|964375|2171|33167|7|279|2065|39|61.46|78.05|3.90|34.98|152.10|2396.94|3043.95|7.02|34.98|117.12|124.14|-2279.82| +2452151|52079|10985|31026|964375|2171|33167|7|254|2065|54|4.44|5.55|1.72|0.00|92.88|239.76|299.70|2.78|0.00|92.88|95.66|-146.88| +2452151|52079|7713|31026|964375|2171|33167|7|107|2065|31|66.19|74.13|8.89|0.00|275.59|2051.89|2298.03|11.02|0.00|275.59|286.61|-1776.30| +2452151|52079|16859|31026|964375|2171|33167|7|88|2065|20|31.68|58.92|49.49|0.00|989.80|633.60|1178.40|49.49|0.00|989.80|1039.29|356.20| +2452151|52079|7855|31026|964375|2171|33167|7|168|2065|48|76.36|84.75|4.23|119.79|203.04|3665.28|4068.00|1.66|119.79|83.25|84.91|-3582.03| +2452151|52079|9669|31026|964375|2171|33167|7|123|2065|60|2.15|2.75|1.32|0.00|79.20|129.00|165.00|6.33|0.00|79.20|85.53|-49.80| +2452151|52079|14493|31026|964375|2171|33167|7|83|2065|87|61.41|84.74|39.82|0.00|3464.34|5342.67|7372.38|69.28|0.00|3464.34|3533.62|-1878.33| +2451606|33332|1769|48048|717295|942|32189|2|53|2066|20|46.58|91.29|66.64|0.00|1332.80|931.60|1825.80|119.95|0.00|1332.80|1452.75|401.20| +2451606||260|48048|717295|942|32189|2|204|2066|27|31.19|||0.00|274.32|||10.97|0.00|||-567.81| +2451606|33332|9341|48048|717295|942|32189|2|44|2066|64|80.02|154.43|143.61|0.00|9191.04|5121.28|9883.52|827.19|0.00|9191.04|10018.23|4069.76| +2451606|33332|4247|48048|717295|942|32189|2|291|2066|20|27.36|33.10|0.66|0.00|13.20|547.20|662.00|1.05|0.00|13.20|14.25|-534.00| +2451606|33332|17525|48048|717295|942|32189|2|208|2066|53|37.90|59.50|43.43|0.00|2301.79|2008.70|3153.50|184.14|0.00|2301.79|2485.93|293.09| +2451606|33332|16969|48048|717295|942|32189|2|49|2066|58|47.23|54.31|22.81|330.74|1322.98|2739.34|3149.98|49.61|330.74|992.24|1041.85|-1747.10| +2451606|33332|7063|48048|717295|942|32189|2|292|2066|10|17.14|29.99|5.69|0.00|56.90|171.40|299.90|1.13|0.00|56.90|58.03|-114.50| +2451606|33332|1625|48048|717295|942|32189|2|241|2066|80|50.45|70.12|9.11|0.00|728.80|4036.00|5609.60|0.00|0.00|728.80|728.80|-3307.20| +2451606|33332|17371|48048|717295|942|32189|2|150|2066|50|34.47|45.50|12.74|0.00|637.00|1723.50|2275.00|12.74|0.00|637.00|649.74|-1086.50| +2451606|33332|17215|48048|717295|942|32189|2|129|2066|38|94.74|135.47|29.80|0.00|1132.40|3600.12|5147.86|33.97|0.00|1132.40|1166.37|-2467.72| +2451606|33332|4681|48048|717295|942|32189|2|24|2066|20|1.03|1.64|0.54|4.75|10.80|20.60|32.80|0.54|4.75|6.05|6.59|-14.55| +2451080|70249|14464|90926|751529|932|5935|4|106|2067|77|41.18|77.00|30.80|0.00|2371.60|3170.86|5929.00|189.72|0.00|2371.60|2561.32|-799.26| +||14845|||932|5935|||2067||30.10|59.29|48.02|||||||||842.24| +2451080|70249|554|||932|5935||100|2067|91||22.12|1.54|0.00||1585.22|2012.92||0.00||148.54|-1445.08| +2451080|70249|1117|90926|751529|932|5935|4|230|2067|33|16.60|22.07|12.35|0.00|407.55|547.80|728.31|32.60|0.00|407.55|440.15|-140.25| +2451080|70249|97|90926|751529|932|5935|4|140|2067|11|96.10|144.15|14.41|64.98|158.51|1057.10|1585.65|6.54|64.98|93.53|100.07|-963.57| +2451080|70249|10276|90926|751529|932|5935|4|156|2067|59|19.38|36.62|21.60|0.00|1274.40|1143.42|2160.58|38.23|0.00|1274.40|1312.63|130.98| +2451080|70249|632|90926|751529|932|5935|4|140|2067|61|82.49|155.08|88.39|0.00|5391.79|5031.89|9459.88|269.58|0.00|5391.79|5661.37|359.90| +2451080|70249|4234|90926|751529|932|5935|4|225|2067|5|51.19|52.72|8.96|0.00|44.80|255.95|263.60|1.79|0.00|44.80|46.59|-211.15| +2451080||4936|90926||||4||2067||||28.86|||3622.85|3767.61|0.00||1266.67||| +2451080|70249|7231|90926|751529|932|5935|4|168|2067|24|90.19|125.36|3.76|0.00|90.24|2164.56|3008.64|0.00|0.00|90.24|90.24|-2074.32| +2451080|70249|13975|90926|751529|932|5935|4|28|2067|17|24.48|33.78|14.86|0.00|252.62|416.16|574.26|17.68|0.00|252.62|270.30|-163.54| +2451080|70249|10393|90926|751529|932|5935|4|68|2067|26|61.29|83.96|8.39|0.00|218.14|1593.54|2182.96|15.26|0.00|218.14|233.40|-1375.40| +2451080|70249|1573|90926|751529|932|5935|4|298|2067|4|17.93|19.36|17.23|0.00|68.92|71.72|77.44|3.44|0.00|68.92|72.36|-2.80| +2451080|70249|998|90926|751529|932|5935|4|290|2067|95|33.25|36.57|30.71|0.00|2917.45|3158.75|3474.15|262.57|0.00|2917.45|3180.02|-241.30| +2451080|70249|4040|90926|751529|932|5935|4|79|2067|40|81.49|137.71|74.36|0.00|2974.40|3259.60|5508.40|29.74|0.00|2974.40|3004.14|-285.20| +2451411|42261|16046|70845|536175|5058|37988|1|222|2068|47|47.26|50.56|14.15|0.00|665.05|2221.22|2376.32|53.20|0.00|665.05|718.25|-1556.17| +2451411|42261|10111|70845|536175|5058|37988|1|185|2068|11|18.84|29.20|11.09|0.00|121.99|207.24|321.20|6.09|0.00|121.99|128.08|-85.25| +2451411|42261|8395|70845|536175|5058|37988|1|255|2068|88|75.51|101.18|28.33|0.00|2493.04|6644.88|8903.84|174.51|0.00|2493.04|2667.55|-4151.84| +2451411|42261|9968|70845|536175|5058|37988|1|54|2068|49|17.75|31.24|15.93|0.00|780.57|869.75|1530.76|62.44|0.00|780.57|843.01|-89.18| +2451411|42261|4384|70845|536175|5058|37988|1|13|2068|56|76.04|147.51|97.35|272.58|5451.60|4258.24|8260.56|103.58|272.58|5179.02|5282.60|920.78| +2451411|42261|6578|70845|536175|5058|37988|1|248|2068|60|77.08|94.80|30.33|1164.67|1819.80|4624.80|5688.00|52.41|1164.67|655.13|707.54|-3969.67| +2451411|42261|11722|70845|536175|5058|37988|1|103|2068|40|7.13|12.90|2.45|0.00|98.00|285.20|516.00|0.00|0.00|98.00|98.00|-187.20| +2451411|42261|7789|70845|536175|5058|37988|1|102|2068|66|2.85|4.33|3.72|0.00|245.52|188.10|285.78|7.36|0.00|245.52|252.88|57.42| +2451411|42261|1880|70845|536175|5058|37988|1|268|2068|63|93.13|148.07|121.41|7189.90|7648.83|5867.19|9328.41|22.94|7189.90|458.93|481.87|-5408.26| +2451411|42261|1642|70845|536175|5058|37988|1|81|2068|15|13.58|15.48|13.31|0.00|199.65|203.70|232.20|3.99|0.00|199.65|203.64|-4.05| +2451411|42261|193|70845|536175|5058|37988|1|189|2068|33|15.87|24.12|2.17|0.00|71.61|523.71|795.96|2.86|0.00|71.61|74.47|-452.10| +2451411|42261|5816|70845|536175|5058|37988|1|279|2068|15|16.97|26.30|25.77|0.00|386.55|254.55|394.50|0.00|0.00|386.55|386.55|132.00| +2451411|42261|9242|||5058||||2068||||75.07|0.00|450.42|440.10|506.10||0.00|450.42||10.32| +2451411|42261|1928|70845|536175|5058|37988|1|10|2068|81|43.42|79.89|78.29|0.00|6341.49|3517.02|6471.09|190.24|0.00|6341.49|6531.73|2824.47| +2451411|42261|12538|70845|536175|5058|37988|1|47|2068|86|19.22|34.59|33.89|58.29|2914.54|1652.92|2974.74|114.25|58.29|2856.25|2970.50|1203.33| +2451411|42261|5542|70845|536175|5058|37988|1|120|2068|49|72.57|77.64|46.58|296.71|2282.42|3555.93|3804.36|119.14|296.71|1985.71|2104.85|-1570.22| +2451209|65321|355|41706|1815301|4096|30891|10|227|2069|63|76.59|126.37|103.62|0.00|6528.06|4825.17|7961.31|587.52|0.00|6528.06|7115.58|1702.89| +||4814|41706||4096||10|256|2069|52|85.01|166.61||||4420.52||||||-4161.04| +2451209|65321|17602|41706|1815301|4096|30891|10|159|2069|70|9.26|15.27|10.07|0.00|704.90|648.20|1068.90|56.39|0.00|704.90|761.29|56.70| +2451209|65321|7342|41706|1815301|4096|30891|10|91|2069|42|30.80|31.72|16.81|70.60|706.02|1293.60|1332.24|6.35|70.60|635.42|641.77|-658.18| +2451209|65321|13972|41706|1815301|4096|30891|10|54|2069|41|71.00|88.75|45.26|0.00|1855.66|2911.00|3638.75|55.66|0.00|1855.66|1911.32|-1055.34| +2451209|65321|5596|41706|1815301|4096|30891|10|205|2069|41|19.54|23.64|14.89|0.00|610.49|801.14|969.24|24.41|0.00|610.49|634.90|-190.65| +2451209|65321|5000|41706|1815301|4096|30891|10|50|2069|96|51.83|94.33|35.84|0.00|3440.64|4975.68|9055.68|206.43|0.00|3440.64|3647.07|-1535.04| +2451209|65321|1069|41706|1815301|4096|30891|10|151|2069|37|71.83|79.73|21.52|414.04|796.24|2657.71|2950.01|3.82|414.04|382.20|386.02|-2275.51| +2451209|65321|1045|41706|1815301|4096|30891|10|155|2069|42|55.87|91.06|20.94|404.56|879.48|2346.54|3824.52|4.74|404.56|474.92|479.66|-1871.62| +2451209|65321|12344|41706|1815301|4096|30891|10|156|2069|49|15.88|21.75|2.17|0.00|106.33|778.12|1065.75|6.37|0.00|106.33|112.70|-671.79| +2451209|65321|17047|41706|1815301|4096|30891|10|145|2069|74|68.15|77.69|38.84|0.00|2874.16|5043.10|5749.06|86.22|0.00|2874.16|2960.38|-2168.94| +2451209|65321|5786|41706|1815301|4096|30891|10|133|2069|38|31.17|52.05|26.54|0.00|1008.52|1184.46|1977.90|60.51|0.00|1008.52|1069.03|-175.94| +2451209|65321|814|41706|1815301|4096|30891|10|198|2069|84|54.56|72.56|15.23|0.00|1279.32|4583.04|6095.04|89.55|0.00|1279.32|1368.87|-3303.72| +2451209|65321|628|41706|1815301|4096|30891|10|201|2069|72|34.17|47.15|33.00|0.00|2376.00|2460.24|3394.80|142.56|0.00|2376.00|2518.56|-84.24| +2451209|65321|908|41706|1815301|4096|30891|10|38|2069|12|86.07|139.43|20.91|0.00|250.92|1032.84|1673.16|7.52|0.00|250.92|258.44|-781.92| +2451209|65321|6973|41706|1815301|4096|30891|10|124|2069|59|21.33|37.75|15.47|0.00|912.73|1258.47|2227.25|54.76|0.00|912.73|967.49|-345.74| +2451804|58261|5204|87423|969054|6238|10622|1|288|2070|15|40.34|48.40|8.22|0.00|123.30|605.10|726.00|6.16|0.00|123.30|129.46|-481.80| +2451804|58261|12649|87423|969054|6238|10622|1|140|2070|60|94.58|119.17|60.77|0.00|3646.20|5674.80|7150.20|328.15|0.00|3646.20|3974.35|-2028.60| +2451804|58261|509|87423|969054|6238|10622|1|52|2070|22|22.33|27.68|5.81|0.00|127.82|491.26|608.96|10.22|0.00|127.82|138.04|-363.44| +2451804|58261|5417|87423|969054|6238|10622|1|36|2070|36|5.26|6.57|5.25|71.82|189.00|189.36|236.52|0.00|71.82|117.18|117.18|-72.18| +|58261|9259|87423|||10622|||2070|41||13.15|0.39|0.00|15.99||||0.00|15.99||-293.97| +2451804|58261|7535|87423|969054|6238|10622|1|272|2070|23|29.00|56.84|26.71|0.00|614.33|667.00|1307.32|30.71|0.00|614.33|645.04|-52.67| +2451804|58261|2773|87423|969054|6238|10622|1|219|2070|21|53.66|99.27|55.59|0.00|1167.39|1126.86|2084.67|23.34|0.00|1167.39|1190.73|40.53| +2451804|58261|191|87423|969054|6238|10622|1|1|2070|1|44.00|85.80|18.01|0.00|18.01|44.00|85.80|1.62|0.00|18.01|19.63|-25.99| +2450988|62037|15637|93996|274625|5654|7883|7|69|2071|76|14.32|22.05|8.59|0.00|652.84|1088.32|1675.80|52.22|0.00|652.84|705.06|-435.48| +||853|93996|274625||7883|||2071||59.73|60.92|29.85|0.00|2895.45|5793.81|5909.24|57.90|0.00|2895.45||-2898.36| +2450988|62037|2041|93996|274625|5654|7883|7|73|2071|100|11.51|17.84|1.78|0.00|178.00|1151.00|1784.00|7.12|0.00|178.00|185.12|-973.00| +2450988|62037|5143|93996|274625|5654|7883|7|237|2071|63|23.10|43.89|32.03|0.00|2017.89|1455.30|2765.07|100.89|0.00|2017.89|2118.78|562.59| +2450988|62037|6266|93996|274625|5654|7883|7|220|2071|36|45.49|50.49|44.93|0.00|1617.48|1637.64|1817.64|64.69|0.00|1617.48|1682.17|-20.16| +2450988|62037|14480|93996|274625|5654|7883|7|12|2071|30|15.89|16.68|13.17|0.00|395.10|476.70|500.40|23.70|0.00|395.10|418.80|-81.60| +2450988|62037|3274|93996|274625|5654|7883|7|290|2071|91|96.20|187.59|65.65|0.00|5974.15|8754.20|17070.69|298.70|0.00|5974.15|6272.85|-2780.05| +2450988|62037|14416|93996|274625|5654|7883|7|160|2071|64|26.43|30.39|25.52|0.00|1633.28|1691.52|1944.96|114.32|0.00|1633.28|1747.60|-58.24| +2450988|62037|10370|93996|274625|5654|7883|7|68|2071|81|77.95|129.39|51.75|0.00|4191.75|6313.95|10480.59|0.00|0.00|4191.75|4191.75|-2122.20| +2450988|62037|12158|93996|274625|5654|7883|7|187|2071|6|20.70|36.22|17.74|103.24|106.44|124.20|217.32|0.06|103.24|3.20|3.26|-121.00| +2450988|62037|1334|93996|274625|5654|7883|7|100|2071|65|82.41|140.09|43.42|1326.48|2822.30|5356.65|9105.85|29.91|1326.48|1495.82|1525.73|-3860.83| +2450988|62037|16801|93996|274625|5654|7883|7|140|2071|24|19.06|21.91|2.84|0.00|68.16|457.44|525.84|6.13|0.00|68.16|74.29|-389.28| +2450988|62037|8812|93996|274625|5654|7883|7|17|2071|38|34.91|45.38|36.30|0.00|1379.40|1326.58|1724.44|110.35|0.00|1379.40|1489.75|52.82| +2450988|62037|1780|93996|274625|5654|7883|7|294|2071|54|91.52|107.99|59.39|0.00|3207.06|4942.08|5831.46|224.49|0.00|3207.06|3431.55|-1735.02| +2450988|62037|1652|93996|274625|5654|7883|7|274|2071|62|74.13|85.24|70.74|0.00|4385.88|4596.06|5284.88|0.00|0.00|4385.88|4385.88|-210.18| +2450988|62037|7112|93996|274625|5654|7883|7|35|2071|69|15.01|23.41|11.47|0.00|791.43|1035.69|1615.29|39.57|0.00|791.43|831.00|-244.26| +2452266|35098|17393|59352|312927|4549|46971|10|188|2072|6|63.35|96.29|32.73|0.00|196.38|380.10|577.74|3.92|0.00|196.38|200.30|-183.72| +2452266|35098|16923|59352|312927|4549|46971|10|56|2072|61|32.63|58.08|52.85|0.00|3223.85|1990.43|3542.88|96.71|0.00|3223.85|3320.56|1233.42| +2452266|35098|11599|59352|312927|4549|46971|10|7|2072|59|1.72|2.13|2.08|107.99|122.72|101.48|125.67|1.17|107.99|14.73|15.90|-86.75| +2452266|35098|10475|59352|312927|4549|46971|10|202|2072|61|78.90|108.88|107.79|2038.30|6575.19|4812.90|6641.68|317.58|2038.30|4536.89|4854.47|-276.01| +2452266|35098|81|59352|312927|4549|46971|10|17|2072|96|8.39|11.66|0.81|0.00|77.76|805.44|1119.36|1.55|0.00|77.76|79.31|-727.68| +2452266|35098|8995|59352|312927|4549|46971|10|26|2072|44|66.76|105.48|51.68|0.00|2273.92|2937.44|4641.12|22.73|0.00|2273.92|2296.65|-663.52| +2452266|35098|1007|59352|312927|4549|46971|10|32|2072|32|36.35|66.88|26.75|684.80|856.00|1163.20|2140.16|1.71|684.80|171.20|172.91|-992.00| +2452266|35098|7721|59352|312927|4549|46971|10|299|2072|79|32.06|47.76|12.89|967.39|1018.31|2532.74|3773.04|2.03|967.39|50.92|52.95|-2481.82| +2452266|35098|10373|59352|312927|4549|46971|10|241|2072|63|80.03|126.44|18.96|1015.30|1194.48|5041.89|7965.72|0.00|1015.30|179.18|179.18|-4862.71| +2452266|35098|16681|59352|312927|4549|46971|10|10|2072|56|67.58|70.95|65.98|0.00|3694.88|3784.48|3973.20|0.00|0.00|3694.88|3694.88|-89.60| +2452266|35098|10333|59352|312927|4549|46971|10|59|2072|37|91.28|178.90|67.98|0.00|2515.26|3377.36|6619.30|75.45|0.00|2515.26|2590.71|-862.10| +2452266|35098|9841|59352|312927|4549|46971|10|30|2072|64|86.24|144.02|34.56|0.00|2211.84|5519.36|9217.28|199.06|0.00|2211.84|2410.90|-3307.52| +2452266|35098|16119|59352|312927|4549|46971|10|251|2072|55|36.56|60.68|33.98|1868.90|1868.90|2010.80|3337.40|0.00|1868.90|0.00|0.00|-2010.80| +2450890|63816|6031|67409|832297|4032|25720|2|140|2073|100|72.86|120.94|35.07|0.00|3507.00|7286.00|12094.00|70.14|0.00|3507.00|3577.14|-3779.00| +2450890|63816|9926|67409|832297|4032|25720|2|283|2073|84|31.34|43.56|42.25|0.00|3549.00|2632.56|3659.04|106.47|0.00|3549.00|3655.47|916.44| +2450890|63816|16345|67409|832297|4032|25720|2|268|2073|38|49.28|77.86|45.93|471.24|1745.34|1872.64|2958.68|25.48|471.24|1274.10|1299.58|-598.54| +2450890|63816|6268|67409|832297|4032|25720|2|15|2073|21|97.56|159.99|126.39|0.00|2654.19|2048.76|3359.79|106.16|0.00|2654.19|2760.35|605.43| +2450890|63816|2936|67409|832297|4032|25720|2|262|2073|94|7.30|14.60|14.45|0.00|1358.30|686.20|1372.40|67.91|0.00|1358.30|1426.21|672.10| +2450890|63816|6589|67409|832297|4032|25720|2|240|2073|12|82.31|116.05|70.79|0.00|849.48|987.72|1392.60|59.46|0.00|849.48|908.94|-138.24| +2450890|63816|13267|67409|832297|4032|25720|2|130|2073|57|9.32|10.25|0.30|0.00|17.10|531.24|584.25|1.53|0.00|17.10|18.63|-514.14| +|63816|5912||832297|4032|25720||131|2073|||117.65|82.35|||4057.00|5882.50|||||| +2450890|63816|7657|67409|832297|4032|25720|2|146|2073|39|93.50|152.40|38.10|0.00|1485.90|3646.50|5943.60|74.29|0.00|1485.90|1560.19|-2160.60| +2450890|63816|7306|67409|832297|4032|25720|2|139|2073|28|57.26|85.89|6.01|0.00|168.28|1603.28|2404.92|10.09|0.00|168.28|178.37|-1435.00| +|63816|3112|67409||||||2073|||118.90|76.09||2815.33|3055.09|||||668.93|| +2450890|63816|12271|67409|832297|4032|25720|2|91|2073|28|59.75|74.09|54.82|0.00|1534.96|1673.00|2074.52|0.00|0.00|1534.96|1534.96|-138.04| +||5278||832297|||||2073|28||89.91|||780.36|2030.28|2517.48|||||-1249.92| +2450890|63816|14882|67409|832297|4032|25720|2|96|2073|1|92.80|92.80|86.30|0.00|86.30|92.80|92.80|2.58|0.00|86.30|88.88|-6.50| +2450890|63816|5992|67409|832297|4032|25720|2|165|2073|2|40.17|44.18|41.52|0.00|83.04|80.34|88.36|2.49|0.00|83.04|85.53|2.70| +2452485|49726|5023|7321|944236|5598|4113|4|154|2074|56|9.36|17.12|13.86|0.00|776.16|524.16|958.72|46.56|0.00|776.16|822.72|252.00| +2452485|49726|1909|7321|944236|5598|4113|4|151|2074|76|46.11|87.14|30.49|0.00|2317.24|3504.36|6622.64|115.86|0.00|2317.24|2433.10|-1187.12| +2452485|49726|3097|7321|944236|5598|4113|4|273|2074|24|96.38|114.69|95.19|0.00|2284.56|2313.12|2752.56|45.69|0.00|2284.56|2330.25|-28.56| +2452485|49726|2197|7321|944236|5598|4113|4|295|2074|55|34.06|57.90|6.94|68.70|381.70|1873.30|3184.50|6.26|68.70|313.00|319.26|-1560.30| +2452485|49726|5991|7321|944236|5598|4113|4|252|2074|25|48.16|67.90|43.45|0.00|1086.25|1204.00|1697.50|43.45|0.00|1086.25|1129.70|-117.75| +2452485|49726|10821|7321|944236|5598|4113|4|273|2074|1|15.14|22.55|0.45|0.00|0.45|15.14|22.55|0.00|0.00|0.45|0.45|-14.69| +|49726|16243|7321||5598||4|162|2074||||2.73|||2880.06|4723.05|||116.79||| +2452485|49726|2916|7321|944236|5598|4113|4|299|2074|75|64.89|70.73|40.31|0.00|3023.25|4866.75|5304.75|120.93|0.00|3023.25|3144.18|-1843.50| +2452485|49726|17575|7321|944236|5598|4113|4|189|2074|19|49.46|55.88|1.67|26.97|31.73|939.74|1061.72|0.00|26.97|4.76|4.76|-934.98| +2451487|67641|14878|92360|1180945|2683|36421|1|175|2075|47|69.75|118.57|5.92|0.00|278.24|3278.25|5572.79|0.00|0.00|278.24|278.24|-3000.01| +2451487|67641|6979|92360|1180945|2683|36421|1|290|2075|20|19.76|23.11|15.02|0.00|300.40|395.20|462.20|12.01|0.00|300.40|312.41|-94.80| +2451487|67641|16966|92360|1180945|2683|36421|1|124|2075|48|95.94|136.23|123.96|0.00|5950.08|4605.12|6539.04|476.00|0.00|5950.08|6426.08|1344.96| +2451487|67641|6118|92360|1180945|2683|36421|1|242|2075|76|69.39|90.20|7.21|0.00|547.96|5273.64|6855.20|43.83|0.00|547.96|591.79|-4725.68| +2451487|67641|8542|92360|1180945|2683|36421|1|63|2075|100|3.90|4.56|2.46|0.00|246.00|390.00|456.00|17.22|0.00|246.00|263.22|-144.00| +2451487|67641|674|92360|1180945|2683|36421|1|280|2075|23|17.70|34.16|9.90|0.00|227.70|407.10|785.68|18.21|0.00|227.70|245.91|-179.40| +2451487|67641|8563|92360|1180945|2683|36421|1|74|2075|27|88.75|111.82|27.95|249.03|754.65|2396.25|3019.14|25.28|249.03|505.62|530.90|-1890.63| +2451487|67641|1954|92360|1180945|2683|36421|1|226|2075|78|74.40|113.08|22.61|0.00|1763.58|5803.20|8820.24|35.27|0.00|1763.58|1798.85|-4039.62| +2451487|67641|4094|92360|1180945|2683|36421|1|266|2075|51|15.52|28.55|27.12|0.00|1383.12|791.52|1456.05|41.49|0.00|1383.12|1424.61|591.60| +2452261|45754|13851|78907|304968|3829|8643|2|290|2076|43|85.18|122.65|40.47|348.04|1740.21|3662.74|5273.95|97.45|348.04|1392.17|1489.62|-2270.57| +2452261|45754|109|78907|304968|3829|8643|2|270|2076|67|87.88|154.66|126.82|0.00|8496.94|5887.96|10362.22|679.75|0.00|8496.94|9176.69|2608.98| +2452261|45754|2283|78907|304968|3829|8643|2|264|2076|49|83.86|119.91|116.31|0.00|5699.19|4109.14|5875.59|56.99|0.00|5699.19|5756.18|1590.05| +2452261|45754|16213|78907|304968|3829|8643|2|220|2076|44|52.63|55.26|1.65|10.16|72.60|2315.72|2431.44|0.62|10.16|62.44|63.06|-2253.28| +|45754|12959|78907|||8643||167|2076|6|2.83||||25.08|||1.25||25.08|26.33|| +2452261|45754|9733|78907|304968|3829|8643|2|205|2076|44|65.51|102.85|85.36|2065.71|3755.84|2882.44|4525.40|50.70|2065.71|1690.13|1740.83|-1192.31| +2452261|45754|9551|78907|304968|3829|8643|2|136|2076|30|23.34|35.94|24.43|0.00|732.90|700.20|1078.20|58.63|0.00|732.90|791.53|32.70| +2452261|45754|1843|78907|304968|3829|8643|2|120|2076|31|78.11|144.50|138.72|0.00|4300.32|2421.41|4479.50|387.02|0.00|4300.32|4687.34|1878.91| +2452261|45754|5471|78907|304968|3829|8643|2|157|2076|25|4.38|5.56|2.72|0.00|68.00|109.50|139.00|4.76|0.00|68.00|72.76|-41.50| +2452261|45754|4557|78907|304968|3829|8643|2|229|2076|30|91.01|152.89|77.97|0.00|2339.10|2730.30|4586.70|187.12|0.00|2339.10|2526.22|-391.20| +2452261|45754|11299|78907|304968|3829|8643|2|30|2076|25|74.69|129.21|58.14|0.00|1453.50|1867.25|3230.25|101.74|0.00|1453.50|1555.24|-413.75| +2452261|45754|9251|78907|304968|3829|8643|2|119|2076|33|1.19|1.19|1.19|0.00|39.27|39.27|39.27|3.14|0.00|39.27|42.41|0.00| +2452261|45754|16499|78907|304968|3829|8643|2|100|2076|31|77.91|127.77|111.15|0.00|3445.65|2415.21|3960.87|0.00|0.00|3445.65|3445.65|1030.44| +2452261|45754|13|78907|304968|3829|8643|2|159|2076|27|22.51|34.44|19.28|312.33|520.56|607.77|929.88|2.08|312.33|208.23|210.31|-399.54| +2451628|42511|11045|172|210849|3479|45356|2|28|2077|73|93.31|154.89|66.60|0.00|4861.80|6811.63|11306.97|194.47|0.00|4861.80|5056.27|-1949.83| +2451628|42511|9161|172|210849|3479|45356|2|2|2077|91|67.35|124.59|39.86|0.00|3627.26|6128.85|11337.69|326.45|0.00|3627.26|3953.71|-2501.59| +2451628|42511|4868|172|210849|3479|45356|2|214|2077|13|99.12|115.97|63.78|414.57|829.14|1288.56|1507.61|0.00|414.57|414.57|414.57|-873.99| +2451628|42511|3499|172|210849|3479|45356|2|80|2077|21|82.66|149.61|32.91|0.00|691.11|1735.86|3141.81|62.19|0.00|691.11|753.30|-1044.75| +2451628|42511|3185|172|210849|3479|45356|2|147|2077|55|78.60|148.55|92.10|0.00|5065.50|4323.00|8170.25|202.62|0.00|5065.50|5268.12|742.50| +2451628|42511|935|172|210849|3479|45356|2|235|2077|100|33.14|39.10|6.25|0.00|625.00|3314.00|3910.00|37.50|0.00|625.00|662.50|-2689.00| +2451628|42511|13289|172|210849|3479|45356|2|250|2077|78|1.40|2.52|0.95|0.00|74.10|109.20|196.56|1.48|0.00|74.10|75.58|-35.10| +2451628|42511|9587|172|210849|3479|45356|2|154|2077|75|81.60|115.05|60.97|0.00|4572.75|6120.00|8628.75|365.82|0.00|4572.75|4938.57|-1547.25| +2451628||3719||210849||||137|2077|4||||||||||62.72|68.36|| +2451385|73250|10970|95689|1733358|3474|13543|4|73|2078|52|61.79|106.27|52.07|0.00|2707.64|3213.08|5526.04|108.30|0.00|2707.64|2815.94|-505.44| +2451385|73250|15986|95689|1733358|3474|13543|4|23|2078|77|69.04|118.74|92.61|0.00|7130.97|5316.08|9142.98|499.16|0.00|7130.97|7630.13|1814.89| +2451385|73250|7267|95689|1733358|3474|13543|4|35|2078|54|48.84|73.26|10.25|116.23|553.50|2637.36|3956.04|30.60|116.23|437.27|467.87|-2200.09| +2451385|73250|16106|95689|1733358|3474|13543|4|204|2078|80|30.36|59.20|59.20|0.00|4736.00|2428.80|4736.00|94.72|0.00|4736.00|4830.72|2307.20| +|73250|10064||1733358|3474||4||2078||||||743.54|||||743.54||| +2451385|73250|7190|95689|1733358|3474|13543|4|181|2078|68|96.33|100.18|71.12|0.00|4836.16|6550.44|6812.24|338.53|0.00|4836.16|5174.69|-1714.28| +2451385||10933|||3474|13543|||2078|69||92.13||0.00||3294.06||16.51|0.00|825.93|842.44|-2468.13| +2451385|73250|6967|95689|1733358|3474|13543|4|299|2078|92|71.78|111.97|85.09|0.00|7828.28|6603.76|10301.24|234.84|0.00|7828.28|8063.12|1224.52| +2451385|73250|7892|95689|1733358|3474|13543|4|242|2078|28|59.77|75.31|24.85|0.00|695.80|1673.56|2108.68|0.00|0.00|695.80|695.80|-977.76| +2451165|72600|9565|69047|258666|7132|25998|7|300|2079|55|33.68|46.14|42.91|0.00|2360.05|1852.40|2537.70|23.60|0.00|2360.05|2383.65|507.65| +2451165|72600|17023|69047|258666|7132|25998|7|41|2079|37|74.21|129.12|60.68|0.00|2245.16|2745.77|4777.44|89.80|0.00|2245.16|2334.96|-500.61| +2451165|72600|15907|69047|258666|7132|25998|7|114|2079|83|63.28|70.24|50.57|0.00|4197.31|5252.24|5829.92|293.81|0.00|4197.31|4491.12|-1054.93| +2451165|72600|16634|69047|258666|7132|25998|7|85|2079|36|45.66|54.79|48.21|0.00|1735.56|1643.76|1972.44|0.00|0.00|1735.56|1735.56|91.80| +2451165|72600|4988|69047|258666|7132|25998|7|53|2079|44|25.71|35.73|7.86|0.00|345.84|1131.24|1572.12|13.83|0.00|345.84|359.67|-785.40| +2451165|72600|796|69047|258666|7132|25998|7|86|2079|22|83.03|151.11|66.48|0.00|1462.56|1826.66|3324.42|0.00|0.00|1462.56|1462.56|-364.10| +2451165|72600|11578|69047|258666|7132|25998|7|84|2079|40|35.67|64.20|30.81|0.00|1232.40|1426.80|2568.00|61.62|0.00|1232.40|1294.02|-194.40| +2451165|72600|14173|69047|258666|7132|25998|7|128|2079|75|34.06|48.70|41.88|1696.14|3141.00|2554.50|3652.50|43.34|1696.14|1444.86|1488.20|-1109.64| +2451165|72600|14090|69047|258666|7132|25998|7|66|2079|95|76.69|81.29|32.51|0.00|3088.45|7285.55|7722.55|61.76|0.00|3088.45|3150.21|-4197.10| +||2894|69047|258666||||195|2079|71||20.86||0.00|562.32|1452.66|||0.00||607.30|| +2451165|72600|1381|69047|258666|7132|25998|7|233|2079|1|89.38|115.30|95.69|0.00|95.69|89.38|115.30|7.65|0.00|95.69|103.34|6.31| +2451165|72600|9064|69047|258666|7132|25998|7|113|2079|87|51.70|101.84|85.54|1488.39|7441.98|4497.90|8860.08|238.14|1488.39|5953.59|6191.73|1455.69| +2451165|72600|12241|69047|258666|7132|25998|7|198|2079|14|81.29|84.54|76.93|0.00|1077.02|1138.06|1183.56|96.93|0.00|1077.02|1173.95|-61.04| +2451165||67|69047|258666|7132|25998||228|2079||||43.75|0.00|1181.25||2513.43||0.00|1181.25|1204.87|| +2451165|72600|14086|69047|258666|7132|25998|7|214|2079|22|7.30|13.21|8.71|0.00|191.62|160.60|290.62|15.32|0.00|191.62|206.94|31.02| +2451165|72600|4225|69047|258666|7132|25998|7|259|2079|55|27.98|50.92|45.31|274.12|2492.05|1538.90|2800.60|0.00|274.12|2217.93|2217.93|679.03| +2452238|64124|12243|58905|134192|3079|35613|7|29|2080|72|52.44|64.50|23.86|0.00|1717.92|3775.68|4644.00|0.00|0.00|1717.92|1717.92|-2057.76| +2452238|64124|9577|58905|134192|3079|35613|7|114|2080|85|70.27|122.97|23.36|0.00|1985.60|5972.95|10452.45|178.70|0.00|1985.60|2164.30|-3987.35| +2452238|64124|5031|58905|134192|3079|35613|7|44|2080|69|91.53|168.41|65.67|2945.29|4531.23|6315.57|11620.29|126.87|2945.29|1585.94|1712.81|-4729.63| +2452238|64124|551|58905|134192|3079|35613|7|234|2080|8|98.77|116.54|13.98|0.00|111.84|790.16|932.32|8.94|0.00|111.84|120.78|-678.32| +2452238|64124|7325|58905|134192|3079|35613|7|175|2080|50|63.52|88.92|77.36|2591.56|3868.00|3176.00|4446.00|114.87|2591.56|1276.44|1391.31|-1899.56| +2452238|64124|11867|58905|134192|3079|35613|7|202|2080|67|29.24|34.21|11.97|0.00|801.99|1959.08|2292.07|32.07|0.00|801.99|834.06|-1157.09| +2452238|64124|2893|58905|134192|3079|35613|7|127|2080|17|98.95|160.29|96.17|1029.98|1634.89|1682.15|2724.93|24.19|1029.98|604.91|629.10|-1077.24| +2452238|64124|7485|58905|134192|3079|35613|7|186|2080|61|48.02|94.11|66.81|0.00|4075.41|2929.22|5740.71|203.77|0.00|4075.41|4279.18|1146.19| +2451887|42015|6667|||6077|||35|2081||87.87|141.47|93.37||||||||7732.90|| +2451887|42015|5363|34586|1638908|6077|45646|10|267|2081|64|58.75|82.25|37.83|0.00|2421.12|3760.00|5264.00|24.21|0.00|2421.12|2445.33|-1338.88| +2451887|42015|13358|34586|1638908|6077|45646|10|270|2081|14|12.21|22.22|12.22|0.00|171.08|170.94|311.08|6.84|0.00|171.08|177.92|0.14| +2451887|42015|9080|34586|1638908|6077|45646|10|13|2081|4|4.96|6.74|5.86|0.00|23.44|19.84|26.96|0.70|0.00|23.44|24.14|3.60| +2451887|42015|950|34586|1638908|6077|45646|10|113|2081|79|8.78|11.85|7.70|0.00|608.30|693.62|936.15|12.16|0.00|608.30|620.46|-85.32| +2451887|42015|17891|34586|1638908|6077|45646|10|271|2081|1|32.86|44.03|7.48|3.59|7.48|32.86|44.03|0.35|3.59|3.89|4.24|-28.97| +2451887|42015|1913|34586|1638908|6077|45646|10|92|2081|41|61.34|63.18|12.00|162.36|492.00|2514.94|2590.38|29.66|162.36|329.64|359.30|-2185.30| +2451887|42015|9089|34586|1638908|6077|45646|10|29|2081|39|33.77|46.26|28.68|0.00|1118.52|1317.03|1804.14|22.37|0.00|1118.52|1140.89|-198.51| +2451887|42015|4967|34586|1638908|6077|45646|10|231|2081|27|95.99|99.82|42.92|0.00|1158.84|2591.73|2695.14|34.76|0.00|1158.84|1193.60|-1432.89| +2451887|42015|4877|34586|1638908|6077|45646|10|231|2081|57|37.42|43.03|2.58|0.00|147.06|2132.94|2452.71|4.41|0.00|147.06|151.47|-1985.88| +2451887|42015|2071|34586|1638908|6077|45646|10|11|2081|99|45.72|59.89|46.11|0.00|4564.89|4526.28|5929.11|365.19|0.00|4564.89|4930.08|38.61| +2451887|42015|7154|34586|1638908|6077|45646|10|178|2081|87|23.16|39.83|1.19|0.00|103.53|2014.92|3465.21|3.10|0.00|103.53|106.63|-1911.39| +2451887|42015|12986|34586|1638908|6077|45646|10|72|2081|58|36.76|43.74|34.99|690.00|2029.42|2132.08|2536.92|93.75|690.00|1339.42|1433.17|-792.66| +2451887|42015|5966|34586|1638908|6077|45646|10|118|2081|5|7.33|11.87|11.15|0.00|55.75|36.65|59.35|2.23|0.00|55.75|57.98|19.10| +2451887|42015|8575|34586|1638908|6077|45646|10|70|2081|81|22.23|37.34|25.01|0.00|2025.81|1800.63|3024.54|162.06|0.00|2025.81|2187.87|225.18| +2451140||1894|12835|1738810||33607||295|2082|34|||27.48|28.02|934.32|1138.66|1730.60|45.31|28.02||951.61|| +2451140|66772|1736|12835|1738810|4432|33607|2|22|2082|64|49.71|53.68|53.68|2576.64|3435.52|3181.44|3435.52|25.76|2576.64|858.88|884.64|-2322.56| +2451140|66772|9188|12835|1738810|4432|33607|2|244|2082|86|34.00|42.16|24.87|0.00|2138.82|2924.00|3625.76|128.32|0.00|2138.82|2267.14|-785.18| +2451140|66772|16261|12835|1738810|4432|33607|2|100|2082|26|88.77|136.70|13.67|0.00|355.42|2308.02|3554.20|10.66|0.00|355.42|366.08|-1952.60| +2451140||8504|||||2||2082|||50.74|48.20|||1568.05|2790.70|17.49||583.22|600.71|-984.83| +2451140|66772|14473|12835|1738810|4432|33607|2|266|2082|55|79.82|95.78|68.96|0.00|3792.80|4390.10|5267.90|0.00|0.00|3792.80|3792.80|-597.30| +2451140|66772|6889|12835|1738810|4432|33607||182|2082|||99.80|0.00|0.00|0.00|2547.02||0.00|0.00|0.00||| +2451140|66772|4171|12835|1738810|4432|33607|2|171|2082|95|2.43|4.54|2.45|32.58|232.75|230.85|431.30|18.01|32.58|200.17|218.18|-30.68| +2451140|66772|16790|12835|1738810|4432|33607|2|5|2082|16|69.30|108.10|33.51|503.99|536.16|1108.80|1729.60|1.93|503.99|32.17|34.10|-1076.63| +2451140|66772|2960|12835|1738810|4432|33607|2|255|2082|50|1.03|1.03|0.28|0.00|14.00|51.50|51.50|0.56|0.00|14.00|14.56|-37.50| +2451140|66772|15428|12835|1738810|4432|33607|2|207|2082|23|22.47|22.47|15.72|0.00|361.56|516.81|516.81|0.00|0.00|361.56|361.56|-155.25| +2451140|66772|9824|12835|1738810|4432|33607|2|42|2082|67|41.65|44.98|7.19|0.00|481.73|2790.55|3013.66|43.35|0.00|481.73|525.08|-2308.82| +2451140|66772|6727|12835|1738810|4432|33607|2|286|2082|2|27.75|40.79|2.03|0.00|4.06|55.50|81.58|0.16|0.00|4.06|4.22|-51.44| +2451140|66772|4924|12835|1738810|4432|33607|2|122|2082|55|33.51|54.95|9.34|0.00|513.70|1843.05|3022.25|41.09|0.00|513.70|554.79|-1329.35| +2451140|66772|3206|12835|1738810|4432|33607|2|179|2082|23|61.77|121.68|7.30|100.74|167.90|1420.71|2798.64|4.70|100.74|67.16|71.86|-1353.55| +2451140|66772|15796|12835|1738810|4432|33607|2|22|2082|47|67.20|104.16|46.87|2136.80|2202.89|3158.40|4895.52|0.00|2136.80|66.09|66.09|-3092.31| +2451602|52115|10280|81784|1591341|6498|20455|2|102|2083|91|38.57|42.04|29.42|0.00|2677.22|3509.87|3825.64|160.63|0.00|2677.22|2837.85|-832.65| +2451602|52115|1121|81784|1591341|6498|20455|2|160|2083|24|56.64|105.91|102.73|986.20|2465.52|1359.36|2541.84|118.34|986.20|1479.32|1597.66|119.96| +2451602|52115|16165|81784|1591341|6498|20455|2|211|2083|9|25.57|49.86|35.89|0.00|323.01|230.13|448.74|0.00|0.00|323.01|323.01|92.88| +2451602|52115|6703|81784|1591341|6498|20455|2|122|2083|27|73.77|80.40|76.38|0.00|2062.26|1991.79|2170.80|0.00|0.00|2062.26|2062.26|70.47| +2451602|52115|10483|81784|1591341|6498|20455|2|33|2083|25|16.12|24.01|23.04|0.00|576.00|403.00|600.25|46.08|0.00|576.00|622.08|173.00| +2451602|52115|14893|81784|1591341|6498|20455|2|96|2083|78|75.24|148.97|19.36|0.00|1510.08|5868.72|11619.66|30.20|0.00|1510.08|1540.28|-4358.64| +2451602|52115|1568|81784|1591341|6498|20455|2|209|2083|78|22.85|38.15|20.60|0.00|1606.80|1782.30|2975.70|0.00|0.00|1606.80|1606.80|-175.50| +2451602|52115|15215|81784|1591341|6498|20455|2|254|2083|62|52.74|71.19|17.08|222.38|1058.96|3269.88|4413.78|25.09|222.38|836.58|861.67|-2433.30| +2451602|52115|13070|81784|1591341|6498|20455|2|46|2083|21|98.44|192.94|38.58|0.00|810.18|2067.24|4051.74|72.91|0.00|810.18|883.09|-1257.06| +2451602|52115|1933|81784|1591341|6498|20455|2|201|2083|33|22.46|33.24|31.57|0.00|1041.81|741.18|1096.92|10.41|0.00|1041.81|1052.22|300.63| +2451602|52115|13015|81784|1591341|6498|20455|2|42|2083|12|54.57|105.32|81.09|0.00|973.08|654.84|1263.84|48.65|0.00|973.08|1021.73|318.24| +||8032||1256067|1346|27872||179|2084|||||0.00|2591.19||3549.63|77.73|0.00|||780.15| +2451535|36168|16957|94341|1256067|1346|27872|10|227|2084|22|55.26|59.12|7.68|0.00|168.96|1215.72|1300.64|6.75|0.00|168.96|175.71|-1046.76| +2451535|36168|6370|94341|1256067|1346|27872|10|179|2084|31|5.05|7.01|4.20|0.00|130.20|156.55|217.31|10.41|0.00|130.20|140.61|-26.35| +2451535|36168|15298|94341|1256067|1346|27872|10|44|2084|96|46.26|63.83|35.74|0.00|3431.04|4440.96|6127.68|0.00|0.00|3431.04|3431.04|-1009.92| +2451535|36168|13837|94341|1256067|1346|27872|10|100|2084|3|80.86|119.67|27.52|0.00|82.56|242.58|359.01|0.82|0.00|82.56|83.38|-160.02| +2451535|36168|14536|94341|1256067|1346|27872|10|72|2084|91|46.60|92.73|76.96|4972.38|7003.36|4240.60|8438.43|162.47|4972.38|2030.98|2193.45|-2209.62| +2451535|36168|646|94341|1256067|1346|27872|10|134|2084|51|31.47|51.29|34.87|1404.91|1778.37|1604.97|2615.79|11.20|1404.91|373.46|384.66|-1231.51| +2451535|36168|13592|94341|1256067|1346|27872|10|194|2084|16|7.75|14.33|6.30|0.00|100.80|124.00|229.28|6.04|0.00|100.80|106.84|-23.20| +2451535|36168|1039|94341|1256067|1346|27872|10|237|2084|3|90.31|140.88|28.17|0.00|84.51|270.93|422.64|0.84|0.00|84.51|85.35|-186.42| +2451535|36168|14756|94341|1256067|1346|27872|10|283|2084|89|85.51|153.06|48.97|0.00|4358.33|7610.39|13622.34|130.74|0.00|4358.33|4489.07|-3252.06| +2451535|36168|1270|94341|1256067|1346|27872|10|15|2084|63|22.46|33.24|14.95|0.00|941.85|1414.98|2094.12|84.76|0.00|941.85|1026.61|-473.13| +2451165|49702|9818|67993|1822220|3647|25427|7|93|2085|17|61.04|104.37|38.61|0.00|656.37|1037.68|1774.29|26.25|0.00|656.37|682.62|-381.31| +2451165|49702|10982|67993|1822220|3647|25427|7|141|2085|69|44.94|87.63|14.89|0.00|1027.41|3100.86|6046.47|51.37|0.00|1027.41|1078.78|-2073.45| +2451165|49702|4861|67993|1822220|3647|25427|7|203|2085|1|5.61|6.73|1.68|0.00|1.68|5.61|6.73|0.00|0.00|1.68|1.68|-3.93| +2451165|49702|8419|67993|1822220|3647|25427|7|37|2085|24|10.28|16.24|7.47|43.02|179.28|246.72|389.76|12.26|43.02|136.26|148.52|-110.46| +|49702|8342||||||13|2085|81||30.57|18.34|653.63||1997.46||58.23|653.63|831.91|890.14|-1165.55| +2451165|49702|7777|67993|1822220|3647|25427|7|62|2085|37|66.71|71.37|32.11|0.00|1188.07|2468.27|2640.69|0.00|0.00|1188.07|1188.07|-1280.20| +2451165|49702|6620|67993|1822220|3647|25427|7|276|2085|17|48.83|93.75|2.81|0.00|47.77|830.11|1593.75|3.34|0.00|47.77|51.11|-782.34| +2451165|49702|8048|67993|1822220|3647|25427|7|190|2085|94|35.41|40.36|19.37|0.00|1820.78|3328.54|3793.84|18.20|0.00|1820.78|1838.98|-1507.76| +2451165|49702|7744|67993|1822220|3647|25427|7|168|2085|19|43.80|57.81|24.28|0.00|461.32|832.20|1098.39|36.90|0.00|461.32|498.22|-370.88| +2451165|49702|16882|67993|1822220|3647|25427|7|213|2085|38|52.85|59.19|34.33|1017.54|1304.54|2008.30|2249.22|17.22|1017.54|287.00|304.22|-1721.30| +2451165||10682|67993||3647|25427|7|173|2085||||9.14||237.64|||||237.64||-450.32| +2451165|49702|3676|67993|1822220|3647|25427|7|127|2085|39|10.34|10.96|0.10|0.00|3.90|403.26|427.44|0.31|0.00|3.90|4.21|-399.36| +2451165|49702|7310|67993|1822220|3647|25427|7|223|2085|82|48.47|80.46|55.51|0.00|4551.82|3974.54|6597.72|273.10|0.00|4551.82|4824.92|577.28| +2451165|49702|4942|67993|1822220|3647|25427|7|222|2085|5|75.06|137.35|50.81|0.00|254.05|375.30|686.75|10.16|0.00|254.05|264.21|-121.25| +2451165|49702|8680|67993|1822220|3647|25427|7|281|2085|34|35.27|53.96|3.77|0.00|128.18|1199.18|1834.64|5.12|0.00|128.18|133.30|-1071.00| +2451165|49702|2390|67993|1822220|3647|25427|7|97|2085|40|24.01|39.85|33.07|0.00|1322.80|960.40|1594.00|39.68|0.00|1322.80|1362.48|362.40| +2451038|68115|1388|51486|1788528|5923|13365|10|235|2086|35|63.72|64.35|21.23|0.00|743.05|2230.20|2252.25|37.15|0.00|743.05|780.20|-1487.15| +2451038|68115|3103|51486|1788528|5923|13365|10|165|2086|61|93.42|169.09|120.05|0.00|7323.05|5698.62|10314.49|146.46|0.00|7323.05|7469.51|1624.43| +2451038|68115|4873|51486|1788528|5923|13365|10|77|2086|7|50.17|86.29|16.39|0.00|114.73|351.19|604.03|0.00|0.00|114.73|114.73|-236.46| +2451038|68115|16483|51486|1788528|5923|13365|10|166|2086|59|48.13|56.31|37.72|0.00|2225.48|2839.67|3322.29|89.01|0.00|2225.48|2314.49|-614.19| +|68115|4424|||5923|13365|10||2086|26||82.43|||1714.44|1458.08|2143.18|||1714.44||| +2451038|68115|10876|51486|1788528|5923|13365|10|196|2086|67|11.70|15.32|11.18|0.00|749.06|783.90|1026.44|67.41|0.00|749.06|816.47|-34.84| +2451038|68115|9718|51486|1788528|5923|13365|10|194|2086|54|4.60|4.60|2.71|0.00|146.34|248.40|248.40|10.24|0.00|146.34|156.58|-102.06| +2451038|68115|1900|51486|1788528|||||2086|||138.65|38.82||3338.52||||||3572.21|| +2451038|68115|15232|51486|1788528|5923|13365|10|25|2086|67|23.58|24.75|9.65|420.25|646.55|1579.86|1658.25|20.36|420.25|226.30|246.66|-1353.56| +2451038|68115|14044|51486|1788528|5923|13365|10|225|2086|26|50.30|94.06|4.70|0.00|122.20|1307.80|2445.56|2.44|0.00|122.20|124.64|-1185.60| +|39614|5911||1228052|3909||8||2087|49||50.75|||1492.05|||44.76|||1536.81|| +2451147|39614|16474|12995|1228052|3909|44166|8|286|2087|68|32.68|34.64|24.24|0.00|1648.32|2222.24|2355.52|16.48|0.00|1648.32|1664.80|-573.92| +2451147|39614|17174|12995|1228052|3909|44166|8|140|2087|92|28.28|30.54|16.18|0.00|1488.56|2601.76|2809.68|29.77|0.00|1488.56|1518.33|-1113.20| +2451147|39614|115|12995|1228052|3909|44166|8|147|2087|27|13.05|16.05|9.30|30.13|251.10|352.35|433.35|8.83|30.13|220.97|229.80|-131.38| +2451147|39614|12895|12995|1228052|3909|44166|8|290|2087|29|57.69|105.57|52.78|0.00|1530.62|1673.01|3061.53|30.61|0.00|1530.62|1561.23|-142.39| +2451147|39614|13264|12995|1228052|3909|44166|8|177|2087|4|31.82|37.54|22.89|21.05|91.56|127.28|150.16|5.64|21.05|70.51|76.15|-56.77| +2451147|39614|7225|12995|1228052|3909|44166|8|98|2087|13|39.57|75.18|63.90|0.00|830.70|514.41|977.34|41.53|0.00|830.70|872.23|316.29| +2451147|39614|16736|12995|1228052|3909|44166|8|50|2087|46|1.18|1.23|0.68|0.00|31.28|54.28|56.58|2.18|0.00|31.28|33.46|-23.00| +2451147|39614|148|12995|1228052|3909|44166|8|148|2087|88|77.06|100.17|52.08|0.00|4583.04|6781.28|8814.96|91.66|0.00|4583.04|4674.70|-2198.24| +2451147|39614|1688|12995|1228052|3909|44166|8|29|2087|77|61.64|122.66|6.13|0.00|472.01|4746.28|9444.82|0.00|0.00|472.01|472.01|-4274.27| +2451147|39614|17005|12995|1228052|3909|44166|8|196|2087|44|23.20|27.60|11.31|0.00|497.64|1020.80|1214.40|0.00|0.00|497.64|497.64|-523.16| +2451147|39614|14812|12995|1228052|3909|44166|8|183|2087|70|18.24|20.61|2.26|0.00|158.20|1276.80|1442.70|14.23|0.00|158.20|172.43|-1118.60| +2451147|39614|11350|12995|1228052|3909|44166|8|64|2087|90|25.99|35.34|4.94|0.00|444.60|2339.10|3180.60|17.78|0.00|444.60|462.38|-1894.50| +2451471|44330|13387|41573|393435|3697|48359|1|261|2088|44|46.12|46.58|16.30|0.00|717.20|2029.28|2049.52|0.00|0.00|717.20|717.20|-1312.08| +2451471|44330|14872|41573|393435|3697|48359|1|193|2088|3|35.70|38.19|35.13|0.00|105.39|107.10|114.57|0.00|0.00|105.39|105.39|-1.71| +2451471|44330|9946|41573|393435|3697|48359|1|206|2088|82|23.14|40.49|29.15|2222.97|2390.30|1897.48|3320.18|0.00|2222.97|167.33|167.33|-1730.15| +2451471|44330|7774|41573|393435|3697|48359|1|5|2088|95|61.83|117.47|0.00|0.00|0.00|5873.85|11159.65|0.00|0.00|0.00|0.00|-5873.85| +2451471|44330|3406|41573|393435|3697|48359|1|294|2088|8|11.87|11.87|7.24|0.00|57.92|94.96|94.96|4.05|0.00|57.92|61.97|-37.04| +2451471|44330|17842|41573|393435|3697|48359|1|286|2088|8|63.53|106.73|3.20|0.00|25.60|508.24|853.84|0.76|0.00|25.60|26.36|-482.64| +2451471|44330|6164|41573|393435|3697|48359|1|81|2088|23|64.49|70.29|7.73|0.00|177.79|1483.27|1616.67|1.77|0.00|177.79|179.56|-1305.48| +2451471|44330|8882|41573|393435|3697|48359|1|13|2088|16|96.78|120.97|108.87|1672.24|1741.92|1548.48|1935.52|2.78|1672.24|69.68|72.46|-1478.80| +2451471|44330|16993|41573|393435|3697|48359|1|225|2088|67|71.07|120.81|28.99|0.00|1942.33|4761.69|8094.27|135.96|0.00|1942.33|2078.29|-2819.36| +2451471|44330|11269|41573|393435|3697|48359|1|118|2088|33|23.09|45.94|33.07|0.00|1091.31|761.97|1516.02|0.00|0.00|1091.31|1091.31|329.34| +2451471|44330|3280|41573|393435|3697|48359|1|183|2088|58|33.82|57.49|31.61|1375.03|1833.38|1961.56|3334.42|18.33|1375.03|458.35|476.68|-1503.21| +||5432|||3697||1|280|2088||8.74|11.62||0.00|7.43||11.62||0.00|||-1.31| +2451471|44330|1630|41573|393435|3697|48359|1|233|2088|59|60.62|66.68|16.00|802.40|944.00|3576.58|3934.12|0.00|802.40|141.60|141.60|-3434.98| +2451471|44330|16880|41573|393435|3697|48359|1|192|2088|17|71.95|84.90|11.88|0.00|201.96|1223.15|1443.30|12.11|0.00|201.96|214.07|-1021.19| +2451471|44330|6358|41573|393435|3697|48359|1|284|2088|53|25.62|43.81|18.83|898.19|997.99|1357.86|2321.93|0.99|898.19|99.80|100.79|-1258.06| +2451703|64150|9536|71697|1860967|1437|16660|2|58|2089|69|84.59|90.51|7.24|9.99|499.56|5836.71|6245.19|44.06|9.99|489.57|533.63|-5347.14| +2451703|64150|2285|71697|1860967|1437|16660|2|132|2089|53|34.76|69.52|27.11|991.41|1436.83|1842.28|3684.56|0.00|991.41|445.42|445.42|-1396.86| +|64150|16052|||1437||2|112|2089||||76.39|0.00||5252.40|8088.48|110.00|0.00|||| +2451703|64150|3733|71697|1860967|1437|16660|2|259|2089|25|21.09|29.10|23.57|0.00|589.25|527.25|727.50|29.46|0.00|589.25|618.71|62.00| +2451703|64150|7841|71697|1860967|1437|16660|2|124|2089|49|51.81|89.63|77.08|0.00|3776.92|2538.69|4391.87|151.07|0.00|3776.92|3927.99|1238.23| +2451703|64150|5293|71697|1860967|1437|16660|2|48|2089|1|62.42|68.03|46.26|13.41|46.26|62.42|68.03|1.31|13.41|32.85|34.16|-29.57| +2451703|64150|6887|71697|1860967|1437|16660|2|83|2089|70|77.06|85.53|79.54|0.00|5567.80|5394.20|5987.10|167.03|0.00|5567.80|5734.83|173.60| +2451703|64150|3494|71697|1860967|1437|16660|2|243|2089|73|22.33|43.76|17.06|0.00|1245.38|1630.09|3194.48|87.17|0.00|1245.38|1332.55|-384.71| +2452262|31294|1705|30608|1727221|2587|10800|1|260|2090|10|9.23|14.86|5.64|0.00|56.40|92.30|148.60|2.25|0.00|56.40|58.65|-35.90| +2452262|31294|11363|30608|1727221|2587|10800|1|147|2090|12|80.41|124.63|71.03|0.00|852.36|964.92|1495.56|68.18|0.00|852.36|920.54|-112.56| +||7079|30608|1727221|||1|148|2090||||33.10|1247.87|1919.80|5388.20|10668.52|26.87|1247.87|||-4716.27| +2452262|31294|10605|30608|1727221|2587|10800|1|151|2090|80|40.27|75.70|18.16|0.00|1452.80|3221.60|6056.00|116.22|0.00|1452.80|1569.02|-1768.80| +2452262|31294|10213|30608|1727221|2587|10800|1|226|2090|23|95.43|187.04|37.40|559.13|860.20|2194.89|4301.92|15.05|559.13|301.07|316.12|-1893.82| +2452262|31294|2677|30608|1727221|2587|10800|1|104|2090|41|64.73|68.61|26.75|0.00|1096.75|2653.93|2813.01|54.83|0.00|1096.75|1151.58|-1557.18| +2452262||16809||1727221|2587|10800|1||2090||6.69||1.26|||107.04|||||20.36|-86.88| +2452262|31294|7799|30608|1727221|2587|10800|1|32|2090|26|64.44|106.97|18.18|0.00|472.68|1675.44|2781.22|9.45|0.00|472.68|482.13|-1202.76| +2452262|31294|15023|30608|1727221|2587|10800|1|44|2090|46|41.95|83.06|33.22|0.00|1528.12|1929.70|3820.76|106.96|0.00|1528.12|1635.08|-401.58| +|31294|7203|||2587||||2090|49|||||431.20|4382.56||||431.20||-3951.36| +2451154|64469|9248|38552|1200868|2031|39147|10|109|2091|3|98.71|153.00|90.27|0.00|270.81|296.13|459.00|21.66|0.00|270.81|292.47|-25.32| +2451154|64469|15356|38552|1200868|2031|39147|10|111|2091|16|29.67|42.13|14.32|0.00|229.12|474.72|674.08|13.74|0.00|229.12|242.86|-245.60| +2451154|64469|6625|38552|1200868|2031|39147|10|283|2091|75|61.10|116.70|82.85|0.00|6213.75|4582.50|8752.50|248.55|0.00|6213.75|6462.30|1631.25| +||13345|38552||||||2091||19.22||22.10|||595.82|965.03|||685.10||89.28| +2451154|64469|16906|38552|1200868|2031|39147|10|120|2091|51|2.51|2.93|0.26|0.00|13.26|128.01|149.43|0.66|0.00|13.26|13.92|-114.75| +2451154|64469|14950|38552|1200868|2031|39147|10|39|2091|23|90.41|131.09|24.90|0.00|572.70|2079.43|3015.07|40.08|0.00|572.70|612.78|-1506.73| +2451154|64469|14143|38552|1200868|2031|39147|10|95|2091|58|99.21|195.44|35.17|0.00|2039.86|5754.18|11335.52|122.39|0.00|2039.86|2162.25|-3714.32| +2451154|64469|14119|38552|1200868|2031|39147|10|217|2091|62|74.76|142.04|51.13|0.00|3170.06|4635.12|8806.48|221.90|0.00|3170.06|3391.96|-1465.06| +2451154|64469|6154|38552|1200868|2031|39147|10|233|2091|7|1.76|1.90|0.07|0.00|0.49|12.32|13.30|0.00|0.00|0.49|0.49|-11.83| +2451154|64469|15082|38552|1200868|2031|39147|10|96|2091|79|13.55|23.57|19.32|0.00|1526.28|1070.45|1862.03|30.52|0.00|1526.28|1556.80|455.83| +2451154|64469|1384|38552|1200868|2031|39147|10|86|2091|74|47.48|93.53|26.18|406.83|1937.32|3513.52|6921.22|15.30|406.83|1530.49|1545.79|-1983.03| +2451154|64469|5120|38552|1200868|2031|39147|10|28|2091|9|84.91|125.66|10.05|0.00|90.45|764.19|1130.94|7.23|0.00|90.45|97.68|-673.74| +2451154|64469|4616|38552|1200868|2031|39147|10|191|2091|76|13.72|24.55|1.71|0.00|129.96|1042.72|1865.80|1.29|0.00|129.96|131.25|-912.76| +2451154|64469|8416|38552|1200868|2031|39147|10|39|2091|48|11.81|22.43|6.05|0.00|290.40|566.88|1076.64|5.80|0.00|290.40|296.20|-276.48| +2451154|64469|12956|38552|1200868|2031|39147|10|91|2091|46|79.04|149.38|126.97|0.00|5840.62|3635.84|6871.48|0.00|0.00|5840.62|5840.62|2204.78| +2451154|64469|14336|38552|1200868|2031|39147|10|117|2091|77|1.08|1.55|1.14|0.00|87.78|83.16|119.35|6.14|0.00|87.78|93.92|4.62| +||7287|||5489||||2092|46|21.52|42.39|30.52|0.00||||112.31|0.00|1403.92|1516.23|414.00| +2452276|53898|3013|37095|849292|5489|26486|1|264|2092|83|28.11|54.81|54.26|0.00|4503.58|2333.13|4549.23|405.32|0.00|4503.58|4908.90|2170.45| +2452276|53898|14031|37095|849292|5489|26486|1|118|2092|6|1.62|3.24|2.13|0.00|12.78|9.72|19.44|0.38|0.00|12.78|13.16|3.06| +2452276|53898|6525|37095|849292|5489|26486|1|68|2092|22|12.74|13.37|2.67|0.00|58.74|280.28|294.14|4.69|0.00|58.74|63.43|-221.54| +2452276|53898|10026|37095|849292|5489|26486|1|23|2092|10|69.89|131.39|85.40|0.00|854.00|698.90|1313.90|0.00|0.00|854.00|854.00|155.10| +2452276||14341|||5489||||2092|49|||62.74|||3754.38|4880.40|61.48||3074.26|3135.74|| +2452276|53898|8511|37095|849292|5489|26486|1|189|2092|50|46.48|80.87|38.00|0.00|1900.00|2324.00|4043.50|95.00|0.00|1900.00|1995.00|-424.00| +2452276|53898|906|37095|849292|5489|26486|1|200|2092|58|64.43|97.93|6.85|0.00|397.30|3736.94|5679.94|0.00|0.00|397.30|397.30|-3339.64| +2452276|53898|17802|37095|849292|5489|26486|1|276|2092|13|94.80|117.55|19.98|0.00|259.74|1232.40|1528.15|15.58|0.00|259.74|275.32|-972.66| +2452276|53898|3327|37095|849292|5489|26486|1|269|2092|90|7.02|7.86|5.65|0.00|508.50|631.80|707.40|35.59|0.00|508.50|544.09|-123.30| +2452276|53898|14323|37095|849292|5489|26486|1|38|2092|37|32.47|37.34|16.05|0.00|593.85|1201.39|1381.58|17.81|0.00|593.85|611.66|-607.54| +2452276|53898|10701|37095|849292|5489|26486|1|103|2092|44|25.29|37.68|19.21|0.00|845.24|1112.76|1657.92|33.80|0.00|845.24|879.04|-267.52| +2452276|53898|17977|37095|849292|5489|26486|1|94|2092|83|33.76|55.02|21.45|0.00|1780.35|2802.08|4566.66|71.21|0.00|1780.35|1851.56|-1021.73| +2452276||2772|37095||5489||||2092|2||||||148.62||||7.80|8.26|-140.82| +2452276|53898|4551|37095|849292|||||2092|50|||||2894.00|||86.82||2894.00||365.00| +2452276|53898|10519|37095|849292|5489|26486|1|91|2092|73|81.16|126.60|101.28|0.00|7393.44|5924.68|9241.80|369.67|0.00|7393.44|7763.11|1468.76| +2452198|48415|12375|69298|757117|2889|43108|7|5|2093|29|84.72|152.49|15.24|0.00|441.96|2456.88|4422.21|22.09|0.00|441.96|464.05|-2014.92| +2452198|48415|5235|69298|757117|2889|43108|7|101|2093|21|4.66|8.66|8.05|0.00|169.05|97.86|181.86|1.69|0.00|169.05|170.74|71.19| +2452198|48415|15363|69298|757117|2889|43108|7|178|2093|37|45.97|73.09|72.35|0.00|2676.95|1700.89|2704.33|26.76|0.00|2676.95|2703.71|976.06| +2452198|48415|16817|69298|757117|2889|43108|7|141|2093|76|56.52|63.30|32.28|0.00|2453.28|4295.52|4810.80|98.13|0.00|2453.28|2551.41|-1842.24| +2452198|48415|6039|69298|757117|2889|43108|7|77|2093|72|61.14|114.94|88.50|0.00|6372.00|4402.08|8275.68|254.88|0.00|6372.00|6626.88|1969.92| +2452198|48415|15745|69298|757117|2889|43108|7|126|2093|57|18.03|29.74|10.70|0.00|609.90|1027.71|1695.18|6.09|0.00|609.90|615.99|-417.81| +2452198|48415|1755|69298|757117|2889|43108|7|253|2093|18|76.56|113.30|24.92|0.00|448.56|1378.08|2039.40|17.94|0.00|448.56|466.50|-929.52| +2452198|48415|2555|69298|757117|2889|43108|7|248|2093|60|40.58|48.69|21.42|0.00|1285.20|2434.80|2921.40|51.40|0.00|1285.20|1336.60|-1149.60| +2452198|48415|13067|69298|757117|2889|43108|7|137|2093|63|25.96|46.72|33.17|313.45|2089.71|1635.48|2943.36|17.76|313.45|1776.26|1794.02|140.78| +2452198|48415|7187|69298|757117|2889|43108|7|99|2093|30|19.77|25.30|19.73|0.00|591.90|593.10|759.00|11.83|0.00|591.90|603.73|-1.20| +2452198|48415|16399|69298|757117|2889|43108|7|152|2093|52|72.82|76.46|25.23|0.00|1311.96|3786.64|3975.92|39.35|0.00|1311.96|1351.31|-2474.68| +2452198|48415|7075|69298|757117|2889|43108|7|107|2093|80|48.30|69.06|49.72|0.00|3977.60|3864.00|5524.80|159.10|0.00|3977.60|4136.70|113.60| +2452502|47701|1581|39957|105833|4062|40738|7|226|2094|78|57.92|97.88|32.30|0.00|2519.40|4517.76|7634.64|151.16|0.00|2519.40|2670.56|-1998.36| +2452502|47701|6243|39957|105833|4062|40738|7|35|2094|85|18.84|21.28|8.93|0.00|759.05|1601.40|1808.80|30.36|0.00|759.05|789.41|-842.35| +2452502|47701|13467|39957|105833|4062|40738|7|153|2094|70|90.33|127.36|122.26|0.00|8558.20|6323.10|8915.20|599.07|0.00|8558.20|9157.27|2235.10| +2452502|47701|9180|39957|105833|4062|40738|7|216|2094|17|10.90|19.72|9.86|0.00|167.62|185.30|335.24|10.05|0.00|167.62|177.67|-17.68| +2452502|47701|8149|39957|105833|4062|40738|7|149|2094|75|58.48|91.22|0.00|0.00|0.00|4386.00|6841.50|0.00|0.00|0.00|0.00|-4386.00| +2452502|47701|3259|39957|105833|4062|40738|7|147|2094|16|45.23|58.34|15.75|0.00|252.00|723.68|933.44|0.00|0.00|252.00|252.00|-471.68| +2452502|47701|4413|39957|105833|4062|40738|7|57|2094|78|27.84|38.14|35.08|0.00|2736.24|2171.52|2974.92|0.00|0.00|2736.24|2736.24|564.72| +2452502|47701|564|39957|105833|4062|40738|7|41|2094|40|40.73|51.72|18.10|246.16|724.00|1629.20|2068.80|43.00|246.16|477.84|520.84|-1151.36| +2451383|60782|3586|70062|273252|6395|2385|8|239|2095|6|1.97|3.84|2.15|0.00|12.90|11.82|23.04|0.25|0.00|12.90|13.15|1.08| +2451383|60782|2258|70062|273252|6395|2385|8|192|2095|90|88.74|103.82|32.18|0.00|2896.20|7986.60|9343.80|231.69|0.00|2896.20|3127.89|-5090.40| +2451383||6787|||6395|||230|2095|||||539.72|||9638.19||539.72||7816.03|2124.37| +|60782|3484|||6395|2385|||2095|17|||7.96|0.00|135.32|176.12|255.34||0.00||144.79|| +2451383|60782|5257|70062|273252|6395|2385|8|273|2095|15|46.16|83.08|37.38|0.00|560.70|692.40|1246.20|33.64|0.00|560.70|594.34|-131.70| +2451383|60782|13597|70062|273252|6395||||2095|72|87.88|||0.00||6327.36|||0.00|3966.48|4125.13|| +2451383||6913|70062|273252|6395|2385|8||2095|88|11.11||||784.96|977.68|1603.36|7.84||784.96||-192.72| +2451383|60782|1489|70062|273252|6395|2385|8|208|2095|9|37.91|58.76|12.92|0.00|116.28|341.19|528.84|10.46|0.00|116.28|126.74|-224.91| +2451383|60782|11455|70062|273252|6395|2385|8|86|2095|46|2.29|3.77|2.14|0.00|98.44|105.34|173.42|5.90|0.00|98.44|104.34|-6.90| +2451383|60782|10526|70062|273252|6395|2385|8|224|2095|31|44.66|56.27|54.58|0.00|1691.98|1384.46|1744.37|135.35|0.00|1691.98|1827.33|307.52| +||9379|70062|273252|6395|2385||250|2095|46|71.23|99.00||||||114.76||1912.68||| +2451383|60782|8374|70062|273252|6395|2385|8|119|2095|83|57.54|87.46|76.96|0.00|6387.68|4775.82|7259.18|447.13|0.00|6387.68|6834.81|1611.86| +2451383|60782|16576|70062|273252|6395|2385|8|122|2095|42|24.49|24.97|17.22|0.00|723.24|1028.58|1048.74|14.46|0.00|723.24|737.70|-305.34| +2451383|60782|3721|70062|273252|6395|2385|8|187|2095|5|58.68|99.16|64.45|0.00|322.25|293.40|495.80|0.00|0.00|322.25|322.25|28.85| +2452017|34669|10965|88541|1304974|960|993|8|208|2096|3|37.54|43.17|19.85|42.87|59.55|112.62|129.51|0.33|42.87|16.68|17.01|-95.94| +2452017|34669|6493|88541|1304974|960|993|8|188|2096|82|75.90|92.59|46.29|0.00|3795.78|6223.80|7592.38|113.87|0.00|3795.78|3909.65|-2428.02| +2452017|34669|13055|88541|1304974|960|993|8|260|2096|6|45.40|89.89|40.45|0.00|242.70|272.40|539.34|4.85|0.00|242.70|247.55|-29.70| +2452017|34669|11857|88541|1304974|960|993|8|225|2096|59|73.73|83.31|53.31|0.00|3145.29|4350.07|4915.29|94.35|0.00|3145.29|3239.64|-1204.78| +2452017|34669|8717|88541|1304974|960|993|8|125|2096|81|97.30|188.76|56.62|0.00|4586.22|7881.30|15289.56|366.89|0.00|4586.22|4953.11|-3295.08| +2452017|34669|7355|88541|1304974|960|993|8|37|2096|44|84.70|167.70|129.12|0.00|5681.28|3726.80|7378.80|511.31|0.00|5681.28|6192.59|1954.48| +2452017|34669|12897|88541|1304974|960|993|8|118|2096|97|81.75|108.72|47.83|0.00|4639.51|7929.75|10545.84|185.58|0.00|4639.51|4825.09|-3290.24| +2452017|34669|6341|88541|1304974|960|993|8|84|2096|49|33.97|58.42|27.45|538.02|1345.05|1664.53|2862.58|8.07|538.02|807.03|815.10|-857.50| +2452017|34669|10297|88541|1304974|960|993|8|207|2096|96|85.51|112.87|80.13|1923.12|7692.48|8208.96|10835.52|173.08|1923.12|5769.36|5942.44|-2439.60| +2452017|34669|7905|88541|1304974|960|993|8|240|2096|58|85.47|86.32|23.30|0.00|1351.40|4957.26|5006.56|94.59|0.00|1351.40|1445.99|-3605.86| +2452017|34669|14447|88541|1304974|960|993|8|175|2096|96|54.62|68.82|22.71|0.00|2180.16|5243.52|6606.72|174.41|0.00|2180.16|2354.57|-3063.36| +2452017|34669|4481|88541|1304974|960|993|8|59|2096|8|11.65|17.94|6.81|0.00|54.48|93.20|143.52|4.35|0.00|54.48|58.83|-38.72| +2452017|34669|14667|88541|1304974|960|993|8|157|2096|76|17.54|27.18|19.29|1128.85|1466.04|1333.04|2065.68|6.74|1128.85|337.19|343.93|-995.85| +2452017|34669|12421|88541|1304974|960|993|8|153|2096|32|53.66|98.19|90.33|0.00|2890.56|1717.12|3142.08|28.90|0.00|2890.56|2919.46|1173.44| +2452017|34669|15859|88541|1304974|960|993|8|93|2096|33|40.08|74.94|21.73|0.00|717.09|1322.64|2473.02|14.34|0.00|717.09|731.43|-605.55| +|68715|2489|39966|265184|802||2|15|2097|42|33.34||11.78||494.76||1904.28|||494.76||| +2451777|68715|6851|39966|265184|802|23220|2|264|2097|50|27.47|41.47|29.02|0.00|1451.00|1373.50|2073.50|58.04|0.00|1451.00|1509.04|77.50| +2451777|68715|251|39966|265184|802|23220|2|96|2097|8|27.19|30.18|0.90|0.00|7.20|217.52|241.44|0.21|0.00|7.20|7.41|-210.32| +2451777|68715|16453|39966|265184|802|23220|2|270|2097|27|63.29|118.98|57.11|0.00|1541.97|1708.83|3212.46|77.09|0.00|1541.97|1619.06|-166.86| +2451777|68715|17594|39966|265184|802|23220|2|16|2097|24|23.79|39.72|25.42|0.00|610.08|570.96|953.28|48.80|0.00|610.08|658.88|39.12| +2451777|68715|13217|39966|265184|802|23220|2|231|2097|59|94.22|160.17|25.62|0.00|1511.58|5558.98|9450.03|75.57|0.00|1511.58|1587.15|-4047.40| +2451777|68715|2909|39966|265184|802|23220|2|286|2097|14|56.51|58.20|23.28|0.00|325.92|791.14|814.80|19.55|0.00|325.92|345.47|-465.22| +2451777|68715|6440|39966|265184|802|23220|2|110|2097|57|29.69|32.36|12.62|165.44|719.34|1692.33|1844.52|49.85|165.44|553.90|603.75|-1138.43| +2451777|68715|6794|39966|265184|802|23220|2|57|2097|81|27.04|43.80|22.33|0.00|1808.73|2190.24|3547.80|144.69|0.00|1808.73|1953.42|-381.51| +2451777|68715|17713|39966|265184|802|23220|2|81|2097|85|92.13|145.56|80.05|6464.03|6804.25|7831.05|12372.60|17.01|6464.03|340.22|357.23|-7490.83| +2452171|63163|10377|54955|967283|7164|35445|7|124|2098|96|77.14|103.36|103.36|0.00|9922.56|7405.44|9922.56|595.35|0.00|9922.56|10517.91|2517.12| +2452171|63163|4887|54955|967283|7164|35445|7|279|2098|95|11.36|18.40|7.54|0.00|716.30|1079.20|1748.00|7.16|0.00|716.30|723.46|-362.90| +2452171|63163|15755|54955|967283|7164|35445|7|222|2098|76|51.43|57.60|10.36|0.00|787.36|3908.68|4377.60|39.36|0.00|787.36|826.72|-3121.32| +2452171|63163|1805|54955|967283|7164|35445|7|153|2098|53|50.17|98.33|26.54|0.00|1406.62|2659.01|5211.49|112.52|0.00|1406.62|1519.14|-1252.39| +2452171|63163|16495|54955|967283|7164|35445|7|146|2098|62|24.52|39.23|0.00|0.00|0.00|1520.24|2432.26|0.00|0.00|0.00|0.00|-1520.24| +2452171|63163|2945|54955|967283|7164|35445|7|142|2098|26|69.07|70.45|13.38|0.00|347.88|1795.82|1831.70|24.35|0.00|347.88|372.23|-1447.94| +2452171|63163|6793|54955|967283|7164|35445|7|229|2098|36|32.74|57.94|3.47|0.00|124.92|1178.64|2085.84|0.00|0.00|124.92|124.92|-1053.72| +|63163|4965|||7164|35445|7|85|2098|79|17.32|33.60|||1937.08||2654.40|||1937.08||568.80| +2452171|63163|1547|54955|967283|7164|35445|7|140|2098|21|95.22|179.96|0.00|0.00|0.00|1999.62|3779.16|0.00|0.00|0.00|0.00|-1999.62| +2452171|63163|785|54955|967283||35445|7|76|2098||49.53||38.73||3524.43|4507.23|||||3806.38|-982.80| +2452171|63163|5883|54955|967283|7164|35445|7|47|2098|8|63.82|95.09|20.91|0.00|167.28|510.56|760.72|0.00|0.00|167.28|167.28|-343.28| +2452171|63163|7381|54955|967283|7164|35445|7|92|2098|77|86.03|166.89|156.87|0.00|12078.99|6624.31|12850.53|724.73|0.00|12078.99|12803.72|5454.68| +2452171|63163|13343|54955|967283|7164|35445|7|157|2098|87|43.31|55.43|19.40|0.00|1687.80|3767.97|4822.41|135.02|0.00|1687.80|1822.82|-2080.17| +2451110|44224|4048|11548|843623|4051|10126|4|208|2099|80|33.73|60.71|19.42|0.00|1553.60|2698.40|4856.80|15.53|0.00|1553.60|1569.13|-1144.80| +2451110|44224|10189|11548|843623|4051|10126|4|209|2099|17|49.49|76.70|17.64|0.00|299.88|841.33|1303.90|14.99|0.00|299.88|314.87|-541.45| +2451110|44224|146|11548|843623|4051|10126|4|223|2099|66|27.67|32.37|11.00|0.00|726.00|1826.22|2136.42|7.26|0.00|726.00|733.26|-1100.22| +2451110|44224|12104|11548|843623|4051|10126|4|139|2099|33|9.38|12.94|3.36|0.00|110.88|309.54|427.02|0.00|0.00|110.88|110.88|-198.66| +2451110|44224|16207|11548|843623|4051|10126|4|278|2099|81|74.93|127.38|113.36|0.00|9182.16|6069.33|10317.78|91.82|0.00|9182.16|9273.98|3112.83| +2451110|44224|5530|11548|843623|4051|10126|4|198|2099|22|63.60|125.92|84.36|0.00|1855.92|1399.20|2770.24|167.03|0.00|1855.92|2022.95|456.72| +2451110|44224|15031|11548|843623|4051|10126|4|43|2099|20|96.08|145.08|143.62|0.00|2872.40|1921.60|2901.60|0.00|0.00|2872.40|2872.40|950.80| +2451110|44224|15190|11548|843623|4051|10126|4|127|2099|9|87.90|90.53|83.28|0.00|749.52|791.10|814.77|0.00|0.00|749.52|749.52|-41.58| +2451110|44224|1580|11548|843623|4051|10126|4|161|2099|68|25.37|46.68|5.13|0.00|348.84|1725.16|3174.24|31.39|0.00|348.84|380.23|-1376.32| +2452123|70253|16601|27640|529547|3007|30867|1|6|2100|40|84.07|110.13|14.31|0.00|572.40|3362.80|4405.20|11.44|0.00|572.40|583.84|-2790.40| +2452123|70253|15|27640|529547|3007|30867|1|268|2100|67|19.88|35.18|22.16|890.83|1484.72|1331.96|2357.06|53.45|890.83|593.89|647.34|-738.07| +2452123|70253|13129|27640|529547|3007|30867|1|111|2100|70|61.59|74.52|8.19|332.51|573.30|4311.30|5216.40|9.63|332.51|240.79|250.42|-4070.51| +2452123|70253|14729|27640|529547|3007|30867|1|210|2100|96|30.18|44.36|7.98|0.00|766.08|2897.28|4258.56|22.98|0.00|766.08|789.06|-2131.20| +2452123|70253|10229|27640|529547|3007|30867|1|120|2100|64|62.62|95.18|0.95|0.00|60.80|4007.68|6091.52|4.86|0.00|60.80|65.66|-3946.88| +2452123|70253|10291|27640|529547|3007|30867|1|175|2100|48|48.35|64.30|44.36|0.00|2129.28|2320.80|3086.40|85.17|0.00|2129.28|2214.45|-191.52| +2452123|70253|16441|27640|529547|3007|30867|1|135|2100|64|88.99|154.84|65.03|4161.92|4161.92|5695.36|9909.76|0.00|4161.92|0.00|0.00|-5695.36| +2452123|70253|11409|27640|529547|3007|30867|1|51|2100|57|81.85|156.33|87.54|0.00|4989.78|4665.45|8910.81|299.38|0.00|4989.78|5289.16|324.33| +2452123|70253|15529|27640|529547|3007|30867|1|90|2100|63|81.79|142.31|83.96|0.00|5289.48|5152.77|8965.53|0.00|0.00|5289.48|5289.48|136.71| +2452123|70253|6313|27640|529547|3007|30867|1|268|2100|57|85.69|87.40|72.54|3142.43|4134.78|4884.33|4981.80|19.84|3142.43|992.35|1012.19|-3891.98| +2452123|70253|4043|27640|529547|3007|30867|1|152|2100|39|34.36|55.31|7.74|0.00|301.86|1340.04|2157.09|9.05|0.00|301.86|310.91|-1038.18| +2452123|70253|5599|27640|529547|3007|30867|1|163|2100|74|52.28|70.57|20.46|0.00|1514.04|3868.72|5222.18|75.70|0.00|1514.04|1589.74|-2354.68| +2452123|70253|17749|27640|529547|3007|30867|1|297|2100|11|34.20|34.54|12.43|0.00|136.73|376.20|379.94|5.46|0.00|136.73|142.19|-239.47| +2452123|70253|10829|27640|529547|3007|30867|1|136|2100|71|79.78|139.61|32.11|0.00|2279.81|5664.38|9912.31|159.58|0.00|2279.81|2439.39|-3384.57| +2452299|45359|7845|79284|432033|3982|31531|10|140|2101|36|99.23|161.74|82.48|0.00|2969.28|3572.28|5822.64|0.00|0.00|2969.28|2969.28|-603.00| +2452299|45359|12624||432033|3982|31531||257|2101|70|70.45||24.79||||7890.40|||1735.30||-3196.20| +2452299|45359|3285|79284|432033|3982|31531|10|177|2101|25|72.73|112.73|46.21|0.00|1155.25|1818.25|2818.25|57.76|0.00|1155.25|1213.01|-663.00| +2452299|45359|16257|79284|432033|3982|31531|10|11|2101|2|71.19|88.98|24.91|0.00|49.82|142.38|177.96|4.48|0.00|49.82|54.30|-92.56| +2452299|45359|10579|79284|432033|3982|31531|10|43|2101|47|38.50|77.00|74.69|0.00|3510.43|1809.50|3619.00|70.20|0.00|3510.43|3580.63|1700.93| +2452299|45359|14401|79284|432033|3982|31531|10|44|2101|65|71.49|115.09|82.86|0.00|5385.90|4646.85|7480.85|430.87|0.00|5385.90|5816.77|739.05| +2452299|45359|16621|79284|432033|3982|31531|10|70|2101|57|44.51|55.63|38.38|0.00|2187.66|2537.07|3170.91|175.01|0.00|2187.66|2362.67|-349.41| +2452299|45359|14496|79284|432033|3982|31531|10|167|2101|84|21.24|41.84|0.83|0.00|69.72|1784.16|3514.56|4.88|0.00|69.72|74.60|-1714.44| +2452299|45359|10359|79284|432033|3982|31531|10|290|2101|61|34.92|36.66|13.56|0.00|827.16|2130.12|2236.26|74.44|0.00|827.16|901.60|-1302.96| +2452299|45359|6780|79284|432033|3982|31531|10|121|2101|44|17.92|18.63|17.32|0.00|762.08|788.48|819.72|0.00|0.00|762.08|762.08|-26.40| +2452299|45359|4806|79284|432033|3982|31531|10|38|2101|63|10.80|18.36|11.75|0.00|740.25|680.40|1156.68|51.81|0.00|740.25|792.06|59.85| +2452299|45359|8424|79284|432033|3982|31531|10|29|2101|45|70.88|85.76|7.71|0.00|346.95|3189.60|3859.20|13.87|0.00|346.95|360.82|-2842.65| +|45359|15252|||3982||10|73|2101|55|21.39||5.98|0.00||1176.45|2058.65|23.02|0.00|328.90|351.92|-847.55| +2452299|45359|13033|79284|432033|3982|31531|10|274|2101|38|40.20|50.65|33.93|0.00|1289.34|1527.60|1924.70|64.46|0.00|1289.34|1353.80|-238.26| +2452299|45359|6705|79284|432033|3982|31531|10|124|2101|33|9.29|15.88|0.31|0.00|10.23|306.57|524.04|0.00|0.00|10.23|10.23|-296.34| +2452216|75176|9805|15428|1283951|3674|6597|8|274|2102|12|89.90|103.38|9.30|0.00|111.60|1078.80|1240.56|8.92|0.00|111.60|120.52|-967.20| +2452216|75176|10247||1283951|3674||8||2102||96.67|172.07|56.78||||2408.98|||794.92||| +2452216|75176|2539|15428|1283951|3674|6597|8|223|2102|76|29.78|58.36|58.36|44.35|4435.36|2263.28|4435.36|87.82|44.35|4391.01|4478.83|2127.73| +2452216|75176|13497|15428|1283951|3674|6597|8|157|2102|2|51.65|61.98|58.88|0.00|117.76|103.30|123.96|9.42|0.00|117.76|127.18|14.46| +2452216||7185|15428||3674|6597|||2102||||8.10|0.00|502.20||||0.00|502.20||| +2452216|75176|7037|15428|1283951|3674|6597|8|200|2102|77|1.78|3.48|0.10|0.00|7.70|137.06|267.96|0.23|0.00|7.70|7.93|-129.36| +2452216|75176|13811|15428|1283951|3674|6597|8|287|2102|93|46.97|80.31|69.86|0.00|6496.98|4368.21|7468.83|454.78|0.00|6496.98|6951.76|2128.77| +2452216|75176|13237|15428|1283951|3674|6597|8|106|2102|61|91.96|180.24|120.76|0.00|7366.36|5609.56|10994.64|589.30|0.00|7366.36|7955.66|1756.80| +2452216|75176|8341|15428|1283951|3674|6597|8|148|2102|53|40.31|44.74|4.92|0.00|260.76|2136.43|2371.22|18.25|0.00|260.76|279.01|-1875.67| +2452216|75176|16109|15428|1283951|3674|6597|8|276|2102|71|84.05|129.43|81.54|0.00|5789.34|5967.55|9189.53|0.00|0.00|5789.34|5789.34|-178.21| +2452216|75176|4915|15428|1283951|3674|6597|8|199|2102|29|44.23|87.57|9.63|0.00|279.27|1282.67|2539.53|11.17|0.00|279.27|290.44|-1003.40| +2452216|75176|8789|15428|1283951|3674|6597|8|163|2102|60|39.11|59.05|22.43|0.00|1345.80|2346.60|3543.00|107.66|0.00|1345.80|1453.46|-1000.80| +2452216|75176|14795|15428|1283951|3674|6597|8|163|2102|94|17.43|27.71|17.73|0.00|1666.62|1638.42|2604.74|33.33|0.00|1666.62|1699.95|28.20| +2452216|75176|4811|15428|1283951|3674|6597|8|84|2102|19|76.29|84.68|32.17|0.00|611.23|1449.51|1608.92|12.22|0.00|611.23|623.45|-838.28| +2451894|42305|11321|45943|753461|1426|26544|1|32|2103|89|43.29|56.70|13.60|0.00|1210.40|3852.81|5046.30|24.20|0.00|1210.40|1234.60|-2642.41| +2451894|42305|1417|45943|753461|1426|26544|1|191|2103|16|44.20|53.04|46.67|328.55|746.72|707.20|848.64|12.54|328.55|418.17|430.71|-289.03| +2451894|42305|1153|45943|753461|1426|26544|1|265|2103|20|67.77|88.10|74.00|281.20|1480.00|1355.40|1762.00|35.96|281.20|1198.80|1234.76|-156.60| +2451894|42305|16357|45943|753461|1426|26544|1|13|2103|19|21.01|38.44|34.21|0.00|649.99|399.19|730.36|19.49|0.00|649.99|669.48|250.80| +2451894|42305|3217|45943|753461|1426|26544|1|64|2103|13|24.07|26.47|25.41|0.00|330.33|312.91|344.11|19.81|0.00|330.33|350.14|17.42| +2451894|42305|1124|45943|753461|1426|26544|1|31|2103|8|87.30|116.98|83.05|0.00|664.40|698.40|935.84|59.79|0.00|664.40|724.19|-34.00| +2451894|42305|920|45943|753461|1426|26544|1|149|2103|80|41.88|59.88|40.11|0.00|3208.80|3350.40|4790.40|128.35|0.00|3208.80|3337.15|-141.60| +2451894|42305|7357|45943|753461|1426|26544|1|92|2103|18|44.95|87.20|21.80|0.00|392.40|809.10|1569.60|31.39|0.00|392.40|423.79|-416.70| +2451894|42305|10727|45943|753461|1426|26544|1|251|2103|49|83.23|134.00|121.94|0.00|5975.06|4078.27|6566.00|537.75|0.00|5975.06|6512.81|1896.79| +2451894|42305|8509|45943|753461|1426|26544|1|209|2103|87|26.86|30.88|2.16|39.46|187.92|2336.82|2686.56|1.48|39.46|148.46|149.94|-2188.36| +2451894|42305|11099|45943|753461|1426|26544|1|205|2103|94|78.80|82.74|56.26|0.00|5288.44|7407.20|7777.56|0.00|0.00|5288.44|5288.44|-2118.76| +2451894|42305|13831|45943|753461|1426|26544|1|27|2103|34|61.17|91.75|20.18|0.00|686.12|2079.78|3119.50|6.86|0.00|686.12|692.98|-1393.66| +2451894|42305|233|45943|753461|1426|26544|1|109|2103|63|73.10|89.91|38.66|0.00|2435.58|4605.30|5664.33|146.13|0.00|2435.58|2581.71|-2169.72| +|42305|6325||753461|1426|26544|1|95|2103|66|||20.53|1273.68|1354.98|||4.06|1273.68|81.30|85.36|-1817.52| +2452228|74525|12561|5394|925170|4593|25569|4|191|2104|74|51.35|102.70|24.64|0.00|1823.36|3799.90|7599.80|36.46|0.00|1823.36|1859.82|-1976.54| +2452228|74525|4969|5394|925170|4593|25569|4|293|2104|55|25.84|44.44|39.99|0.00|2199.45|1421.20|2444.20|21.99|0.00|2199.45|2221.44|778.25| +2452228|74525|5445|5394|925170|4593|25569|4|293|2104|26|24.21|33.16|15.58|0.00|405.08|629.46|862.16|8.10|0.00|405.08|413.18|-224.38| +2452228|74525|14621|5394|925170|4593|25569|4|217|2104|9|77.51|89.13|87.34|0.00|786.06|697.59|802.17|39.30|0.00|786.06|825.36|88.47| +2452228|74525|211|5394|925170|4593|25569|4|67|2104|86|38.17|74.81|44.13|0.00|3795.18|3282.62|6433.66|37.95|0.00|3795.18|3833.13|512.56| +2452228|74525|10907||925170|4593|25569|4||2104|88||149.19|||8664.48|||||||| +2452228|74525|475|5394|925170|4593|25569|4|293|2104|59|50.55|67.73|12.86|0.00|758.74|2982.45|3996.07|37.93|0.00|758.74|796.67|-2223.71| +2452228|74525|12475|5394|925170|4593|25569|4|219|2104|22|17.13|28.43|9.09|5.99|199.98|376.86|625.46|7.75|5.99|193.99|201.74|-182.87| +2452228|74525|16557|5394|925170|4593|25569|4|39|2104|68|23.99|41.26|37.13|0.00|2524.84|1631.32|2805.68|151.49|0.00|2524.84|2676.33|893.52| +2452228|74525|12919|5394|925170|4593|25569|4|47|2104|57|45.73|69.50|25.02|0.00|1426.14|2606.61|3961.50|114.09|0.00|1426.14|1540.23|-1180.47| +2452228|74525|13735|5394|925170|4593|25569|4|4|2104|9|75.18|77.43|35.61|0.00|320.49|676.62|696.87|22.43|0.00|320.49|342.92|-356.13| +2452228|74525|2801|5394|925170|4593|25569|4|5|2104|87|36.87|71.89|48.88|0.00|4252.56|3207.69|6254.43|170.10|0.00|4252.56|4422.66|1044.87| +2452228|74525|14249|5394|925170|4593|25569|4|192|2104|80|3.69|3.91|3.75|0.00|300.00|295.20|312.80|18.00|0.00|300.00|318.00|4.80| +2452228|74525|13505|5394|925170|4593|25569|4|129|2104|52|43.25|70.93|63.83|66.38|3319.16|2249.00|3688.36|97.58|66.38|3252.78|3350.36|1003.78| +2451104|53687|12512|15366|1382960|5307|14501|2|2|2105|14|83.52|114.42|106.41|0.00|1489.74|1169.28|1601.88|14.89|0.00|1489.74|1504.63|320.46| +2451104|53687|1717|15366|1382960|5307|14501|2|210|2105|82|3.61|4.72|4.10|0.00|336.20|296.02|387.04|13.44|0.00|336.20|349.64|40.18| +2451104|53687|6193|15366|1382960|5307|14501|2|30|2105|3|73.46|132.22|75.36|0.00|226.08|220.38|396.66|6.78|0.00|226.08|232.86|5.70| +2451104|53687|12290|15366|1382960|5307|14501|2|253|2105|52|93.15|180.71|25.29|0.00|1315.08|4843.80|9396.92|65.75|0.00|1315.08|1380.83|-3528.72| +2451104|53687|1526|15366|1382960|5307|14501|2|130|2105|84|87.03|120.10|118.89|0.00|9986.76|7310.52|10088.40|499.33|0.00|9986.76|10486.09|2676.24| +2451104|53687|12016|15366|1382960|5307|14501|2|160|2105|29|2.54|3.37|2.25|0.00|65.25|73.66|97.73|5.87|0.00|65.25|71.12|-8.41| +2451104|53687|16324|15366|1382960|5307|14501|2|274|2105|12|13.96|25.12|12.81|0.00|153.72|167.52|301.44|9.22|0.00|153.72|162.94|-13.80| +2451104|53687|4118|15366|1382960|5307|14501|2|243|2105|37|84.56|89.63|14.34|0.00|530.58|3128.72|3316.31|47.75|0.00|530.58|578.33|-2598.14| +2452174|37189|7021|58622|1275477|1932|35234|1|187|2106|59|85.53|147.96|60.66|0.00|3578.94|5046.27|8729.64|107.36|0.00|3578.94|3686.30|-1467.33| +2452174|37189|11845|58622|1275477|1932|35234|1|47|2106|51|6.41|7.11|5.83|0.00|297.33|326.91|362.61|0.00|0.00|297.33|297.33|-29.58| +2452174|37189|7637|58622|1275477|1932|35234|1|129|2106|49|71.27|135.41|83.95|0.00|4113.55|3492.23|6635.09|287.94|0.00|4113.55|4401.49|621.32| +|37189|14691||1275477|1932|35234|1|38|2106|||2.65|1.93|35.26||42.84|55.65||35.26|||| +2452174|37189|8501|58622|1275477|1932|35234|1|89|2106|45|6.81|13.27|10.48|0.00|471.60|306.45|597.15|14.14|0.00|471.60|485.74|165.15| +2452174|37189|9529|58622|1275477|1932|35234|1|4|2106|30|64.04|90.93|31.82|868.68|954.60|1921.20|2727.90|5.15|868.68|85.92|91.07|-1835.28| +2452174|37189|8455|58622|1275477|1932|35234|1|257|2106|14|69.84|99.87|80.89|0.00|1132.46|977.76|1398.18|67.94|0.00|1132.46|1200.40|154.70| +2452174|37189|4635|58622|1275477|1932|35234|1|39|2106|87|80.39|107.72|80.79|0.00|7028.73|6993.93|9371.64|492.01|0.00|7028.73|7520.74|34.80| +2451401||790|6647|506276||26734|||2107|38|44.60||||2104.82|1694.80||13.26|||1339.30|-368.76| +2451401|52551|17642|6647|506276|6705|26734|8|166|2107|70|35.96|41.35|22.32|0.00|1562.40|2517.20|2894.50|46.87|0.00|1562.40|1609.27|-954.80| +2451401|52551|6055|6647|506276|6705|26734|8|15|2107|74|7.95|10.73|9.65|0.00|714.10|588.30|794.02|57.12|0.00|714.10|771.22|125.80| +2451401|52551|1105|6647|506276|6705|26734|8|157|2107|2|52.10|76.06|55.52|0.00|111.04|104.20|152.12|5.55|0.00|111.04|116.59|6.84| +2451401|52551|15289|6647|506276|6705|26734|8|267|2107|23|32.59|33.89|16.60|0.00|381.80|749.57|779.47|22.90|0.00|381.80|404.70|-367.77| +2451401|52551|1090|6647|506276|6705|26734|8|243|2107|83|12.14|21.12|8.65|0.00|717.95|1007.62|1752.96|14.35|0.00|717.95|732.30|-289.67| +2451401|52551|1838|6647|506276|6705|26734|8|78|2107|63|45.75|64.96|59.76|828.27|3764.88|2882.25|4092.48|234.92|828.27|2936.61|3171.53|54.36| +2451401|52551|7292|6647|506276|6705|26734|8|76|2107|9|44.89|84.39|0.84|0.00|7.56|404.01|759.51|0.60|0.00|7.56|8.16|-396.45| +|30495|16201|635||7190||||2108||83.88|141.75||0.00|3329.10||11481.75|166.45|0.00|3329.10|3495.55|-3465.18| +2451126|30495|3742|635|188936|7190|33491|8|120|2108|70|76.78|135.90|6.79|0.00|475.30|5374.60|9513.00|38.02|0.00|475.30|513.32|-4899.30| +2451126|30495|8618|635|188936|7190|33491|8|213|2108|99|17.31|26.31|17.89|0.00|1771.11|1713.69|2604.69|141.68|0.00|1771.11|1912.79|57.42| +2451126|30495|8476|635|188936|7190|33491|8|39|2108|73|94.86|166.95|80.13|0.00|5849.49|6924.78|12187.35|116.98|0.00|5849.49|5966.47|-1075.29| +2451126|30495|2425|635|188936|7190|33491|8|226|2108|48|84.36|99.54|33.84|0.00|1624.32|4049.28|4777.92|113.70|0.00|1624.32|1738.02|-2424.96| +2451126|30495|8564|635|188936|7190|33491|8|114|2108|28|90.72|111.58|20.08|0.00|562.24|2540.16|3124.24|39.35|0.00|562.24|601.59|-1977.92| +2451126|30495|1444|635|188936|7190|33491|8|15|2108|93|90.66|100.63|59.37|3864.98|5521.41|8431.38|9358.59|49.69|3864.98|1656.43|1706.12|-6774.95| +2451126|30495|3565|635|188936|7190|33491|8|172|2108|42|55.17|108.68|98.89|0.00|4153.38|2317.14|4564.56|83.06|0.00|4153.38|4236.44|1836.24| +2451126|30495|14560|635|188936|7190|33491|8|124|2108|6|82.27|122.58|26.96|0.00|161.76|493.62|735.48|14.55|0.00|161.76|176.31|-331.86| +2451126|30495|6661|635|188936|7190|33491|8|278|2108|14|16.69|24.20|16.45|133.57|230.30|233.66|338.80|0.00|133.57|96.73|96.73|-136.93| +2451126|30495|17893|635|188936|7190|33491|8|218|2108|2|99.85|130.80|71.94|0.00|143.88|199.70|261.60|2.87|0.00|143.88|146.75|-55.82| +2451126|30495|847|635|188936|7190|33491|8|4|2108|71|23.11|28.42|4.83|243.48|342.93|1640.81|2017.82|0.99|243.48|99.45|100.44|-1541.36| +2452164|66689|4455|5533|1049988|5996|2404|4|226|2109|25|60.81|73.58|43.41|0.00|1085.25|1520.25|1839.50|10.85|0.00|1085.25|1096.10|-435.00| +2452164|66689|7045|5533|1049988|5996|2404|4|269|2109|2|98.42|181.09|32.59|0.00|65.18|196.84|362.18|3.25|0.00|65.18|68.43|-131.66| +2452164|66689|12273|5533|1049988|5996|2404|4|229|2109|23|43.82|56.96|5.12|11.77|117.76|1007.86|1310.08|0.00|11.77|105.99|105.99|-901.87| +2452164|66689|3263|5533|1049988|5996|2404|4|7|2109|69|15.09|18.86|1.69|0.00|116.61|1041.21|1301.34|2.33|0.00|116.61|118.94|-924.60| +2452164|66689|2193|5533|1049988|5996|2404|4|136|2109|34|57.69|69.80|46.06|438.49|1566.04|1961.46|2373.20|45.10|438.49|1127.55|1172.65|-833.91| +2452164|66689|10199|5533|1049988|5996|2404|4|104|2109|64|36.71|66.44|45.17|0.00|2890.88|2349.44|4252.16|115.63|0.00|2890.88|3006.51|541.44| +2452164|66689|9931|5533|1049988|5996|2404|4|43|2109|91|84.09|118.56|35.56|0.00|3235.96|7652.19|10788.96|64.71|0.00|3235.96|3300.67|-4416.23| +||14497|5533|||2404||158|2109|19|||||924.73|1523.99|1813.36||||980.21|-599.26| +2452164|66689|9635|5533|1049988|5996|2404|4|34|2109|45|50.93|51.43|18.00|0.00|810.00|2291.85|2314.35|24.30|0.00|810.00|834.30|-1481.85| +2452164|66689|8935|5533|1049988|5996|2404|4|3|2109|42|67.47|96.48|92.62|0.00|3890.04|2833.74|4052.16|233.40|0.00|3890.04|4123.44|1056.30| +2452164|66689|5685|5533|1049988|5996|2404|4|55|2109|93|24.50|25.48|16.30|0.00|1515.90|2278.50|2369.64|30.31|0.00|1515.90|1546.21|-762.60| +2452164|66689|17703||||2404|4|223|2109|3|44.57|76.21|16.00||||228.63|1.92||48.00||-85.71| +2452164|66689|5695|5533|1049988|5996|2404|4|187|2109|69|97.11|130.12|65.06|0.00|4489.14|6700.59|8978.28|134.67|0.00|4489.14|4623.81|-2211.45| +2451796|64609|9895|43923|485880|284|14688|10|213|2110|27|77.54|122.51|23.27|0.00|628.29|2093.58|3307.77|31.41|0.00|628.29|659.70|-1465.29| +2451796|64609|7351|43923|485880|284|14688|10|69|2110|8|5.18|8.28|2.73|0.00|21.84|41.44|66.24|1.09|0.00|21.84|22.93|-19.60| +2451796|64609|15433|43923|485880|284|14688|10|44|2110|18|10.77|17.77|11.01|0.00|198.18|193.86|319.86|15.85|0.00|198.18|214.03|4.32| +2451796|64609|4772|43923|485880|284|14688|10|255|2110|24|6.66|7.85|6.90|0.00|165.60|159.84|188.40|8.28|0.00|165.60|173.88|5.76| +2451796|64609|17897|43923|485880|284|14688|10|108|2110|32|42.11|71.16|39.13|1026.77|1252.16|1347.52|2277.12|4.50|1026.77|225.39|229.89|-1122.13| +||15623|43923|485880|284|14688|10|232|2110|||118.34|65.08||5792.12|8228.94||347.52||5792.12|6139.64|-2436.82| +2451796|64609|16586|43923|485880|284|14688|10|300|2110|65|27.85|35.09|1.75|0.00|113.75|1810.25|2280.85|10.23|0.00|113.75|123.98|-1696.50| +2451796|64609|479|43923|485880|284|14688|10|24|2110|6|50.46|100.41|31.12|0.00|186.72|302.76|602.46|16.80|0.00|186.72|203.52|-116.04| +2451687||7415||1389430|155||||2111|62|12.64|13.90||0.00|128.96|783.68||6.44|0.00|||| +2451687|61216|4891|46352|1389430|155|48880|8|292|2111|93|71.67|86.00|36.12|0.00|3359.16|6665.31|7998.00|302.32|0.00|3359.16|3661.48|-3306.15| +2451687|61216|12239|46352|1389430|155|48880|8|53|2111|14|96.72|177.96|74.74|94.17|1046.36|1354.08|2491.44|38.08|94.17|952.19|990.27|-401.89| +2451687|61216|17329|46352|1389430|155|48880|8|129|2111|26|23.35|31.05|0.31|4.03|8.06|607.10|807.30|0.12|4.03|4.03|4.15|-603.07| +2451687|61216|14282|46352|1389430|155|48880|8|72|2111|52|1.41|2.12|0.65|28.05|33.80|73.32|110.24|0.17|28.05|5.75|5.92|-67.57| +2451687|61216|2767|46352|1389430|155|48880|8|77|2111|30|42.59|57.92|9.84|206.64|295.20|1277.70|1737.60|3.54|206.64|88.56|92.10|-1189.14| +2451687|61216|17765|46352|1389430|155|48880|8|141|2111|60|27.53|40.46|30.34|0.00|1820.40|1651.80|2427.60|109.22|0.00|1820.40|1929.62|168.60| +2451687|61216|3361|46352|1389430|155|48880|8|35|2111|72|8.41|8.41|6.30|0.00|453.60|605.52|605.52|9.07|0.00|453.60|462.67|-151.92| +2451687|61216|13873|46352|1389430|155|48880|8|167|2111|80|77.00|142.45|88.31|0.00|7064.80|6160.00|11396.00|141.29|0.00|7064.80|7206.09|904.80| +2451687|61216|10565|46352|1389430|155|48880|8|249|2111|77|91.83|95.50|76.40|0.00|5882.80|7070.91|7353.50|176.48|0.00|5882.80|6059.28|-1188.11| +2451687|61216|17045|46352|1389430|155|48880|8|99|2111|59|8.97|10.85|10.63|413.93|627.17|529.23|640.15|14.92|413.93|213.24|228.16|-315.99| +2451687|61216|7592|46352|1389430|155|48880|8|151|2111|3|85.34|142.51|54.15|66.60|162.45|256.02|427.53|8.62|66.60|95.85|104.47|-160.17| +2451687|61216|6379|46352|1389430|155|48880|8|47|2111|38|34.34|46.01|34.96|0.00|1328.48|1304.92|1748.38|92.99|0.00|1328.48|1421.47|23.56| +2451687|61216|15061|46352|1389430|155|48880|8|273|2111|22|72.76|124.41|43.54|0.00|957.88|1600.72|2737.02|76.63|0.00|957.88|1034.51|-642.84| +2451687|61216|7538|46352|1389430|155|48880|8|132|2111|75|21.94|37.51|34.88|0.00|2616.00|1645.50|2813.25|235.44|0.00|2616.00|2851.44|970.50| +2452216|52928|513|53186|883568|1597|22826|10|280|2112|70|69.10|118.85|83.19|0.00|5823.30|4837.00|8319.50|465.86|0.00|5823.30|6289.16|986.30| +2452216|52928|5747|53186|883568|1597|22826|10|295|2112|19|73.99|128.74|99.12|1676.11|1883.28|1405.81|2446.06|12.43|1676.11|207.17|219.60|-1198.64| +2452216|52928|13029|53186|883568|1597|22826|10|79|2112|75|81.30|143.08|48.64|0.00|3648.00|6097.50|10731.00|145.92|0.00|3648.00|3793.92|-2449.50| +2452216|52928|15601|53186|883568|1597|22826|10|110|2112|39|26.50|35.77|13.95|0.00|544.05|1033.50|1395.03|0.00|0.00|544.05|544.05|-489.45| +2452216|52928|14851|53186|883568|1597|22826|10|120|2112|89|34.01|63.25|54.39|3630.53|4840.71|3026.89|5629.25|12.10|3630.53|1210.18|1222.28|-1816.71| +||7787||||22826|10||2112|1|44.37|||3.08|||58.56||3.08||16.27|-29.30| +2452216|52928|9869|53186|883568|1597|22826|10|80|2112|74|11.48|13.77|8.12|0.00|600.88|849.52|1018.98|6.00|0.00|600.88|606.88|-248.64| +2452216|52928|7319|53186|883568|1597|22826|10|193|2112|53|59.96|88.74|17.74|0.00|940.22|3177.88|4703.22|75.21|0.00|940.22|1015.43|-2237.66| +2452216|52928|15091|53186|883568|1597|22826|10|231|2112|51|91.24|146.89|57.28|1519.06|2921.28|4653.24|7491.39|14.02|1519.06|1402.22|1416.24|-3251.02| +2452216|52928|12585|53186|883568|1597|22826|10|265|2112|12|65.73|115.68|39.33|0.00|471.96|788.76|1388.16|0.00|0.00|471.96|471.96|-316.80| +|52928|17459||883568||22826|10||2112|||||||||||27.74||| +2452216|52928|8659|53186|883568|1597|22826|10|28|2112|7|53.53|106.52|91.60|160.30|641.20|374.71|745.64|4.80|160.30|480.90|485.70|106.19| +2451153||8848||260276|1479||2|296|2113|31|49.67||||||2247.81|115.31||||| +2451153|51162|4052|58720|260276|1479|46235|2|79|2113|97|31.83|46.15|38.76|2894.98|3759.72|3087.51|4476.55|0.00|2894.98|864.74|864.74|-2222.77| +|51162|15160|58720||1479|46235||120|2113|14||||167.61|||1656.48|14.93|167.61||228.26|-796.77| +2451153|51162|13135|58720|260276|1479|46235|2|213|2113|35|99.33|174.82|19.23|625.93|673.05|3476.55|6118.70|1.41|625.93|47.12|48.53|-3429.43| +2451153|51162|16834|58720|260276|1479|46235|2|142|2113|82|24.75|46.28|41.18|0.00|3376.76|2029.50|3794.96|270.14|0.00|3376.76|3646.90|1347.26| +2451153|51162|13729|58720|260276|1479|46235|2|218|2113|51|94.58|111.60|89.28|0.00|4553.28|4823.58|5691.60|45.53|0.00|4553.28|4598.81|-270.30| +2451153|51162|12637|58720|260276|1479|46235|2|190|2113|61|13.52|24.47|4.40|0.00|268.40|824.72|1492.67|13.42|0.00|268.40|281.82|-556.32| +2451153|51162|10130|58720|260276|1479|46235|2|114|2113|20|83.33|109.16|36.02|0.00|720.40|1666.60|2183.20|0.00|0.00|720.40|720.40|-946.20| +2451153|51162|3680|58720|260276|1479|46235|2|215|2113|88|42.17|68.31|1.36|19.14|119.68|3710.96|6011.28|5.02|19.14|100.54|105.56|-3610.42| +2451153|51162|3562|58720|260276|1479|46235|2|280|2113|38|41.03|54.56|53.46|0.00|2031.48|1559.14|2073.28|142.20|0.00|2031.48|2173.68|472.34| +2451153|51162|7742|58720|260276|1479|46235|2|253|2113|16|99.16|176.50|67.07|0.00|1073.12|1586.56|2824.00|21.46|0.00|1073.12|1094.58|-513.44| +2451153|51162|6130|58720|260276|1479|46235|2|186|2113|4|74.84|136.20|119.85|0.00|479.40|299.36|544.80|38.35|0.00|479.40|517.75|180.04| +2451153|51162|4400|58720|260276|1479|46235|2|198|2113|17|39.20|56.84|48.88|0.00|830.96|666.40|966.28|49.85|0.00|830.96|880.81|164.56| +2451153|51162|13498|58720|260276|1479|46235|2|89|2113|40|99.46|161.12|30.61|0.00|1224.40|3978.40|6444.80|73.46|0.00|1224.40|1297.86|-2754.00| +2452051||2839|||3619|21138|10|197|2114|9|87.81|92.20|39.64||356.76||829.80|||356.76|363.89|| +2452051|50786|14395|78087|1377253|3619|21138|10|61|2114|69|92.13|165.83|109.44|0.00|7551.36|6356.97|11442.27|226.54|0.00|7551.36|7777.90|1194.39| +2452051|50786|5901|78087|1377253|3619|21138|10|129|2114|19|65.94|82.42|54.39|0.00|1033.41|1252.86|1565.98|20.66|0.00|1033.41|1054.07|-219.45| +2452051|50786|8121|78087|1377253|3619|21138|10|250|2114|90|58.89|84.80|9.32|0.00|838.80|5300.10|7632.00|25.16|0.00|838.80|863.96|-4461.30| +2452051|50786|5805|78087|1377253|3619|21138|10|188|2114|67|4.47|6.88|3.99|0.00|267.33|299.49|460.96|18.71|0.00|267.33|286.04|-32.16| +2452051|50786|15375|78087|1377253|3619|21138|10|147|2114|45|11.03|13.01|11.44|0.00|514.80|496.35|585.45|0.00|0.00|514.80|514.80|18.45| +2452051|50786|11753|78087|1377253|3619|21138|10|275|2114|47|41.74|78.88|78.88|0.00|3707.36|1961.78|3707.36|148.29|0.00|3707.36|3855.65|1745.58| +2452051|50786|12505|78087|1377253|3619|21138|10|196|2114|29|64.47|108.95|37.04|0.00|1074.16|1869.63|3159.55|21.48|0.00|1074.16|1095.64|-795.47| +2452051|50786|5091|78087|1377253|3619|21138|10|294|2114|5|72.79|75.70|0.75|0.00|3.75|363.95|378.50|0.11|0.00|3.75|3.86|-360.20| +2452051|50786|6755|78087|1377253|3619|21138|10|100|2114|63|99.59|161.33|161.33|0.00|10163.79|6274.17|10163.79|0.00|0.00|10163.79|10163.79|3889.62| +2452051|50786|17667|78087|1377253|3619|21138|10|233|2114|88|95.34|133.47|122.79|1728.88|10805.52|8389.92|11745.36|726.13|1728.88|9076.64|9802.77|686.72| +2452051|50786|8863|78087|1377253|3619|21138|10|188|2114|55|45.90|85.83|40.34|820.91|2218.70|2524.50|4720.65|69.88|820.91|1397.79|1467.67|-1126.71| +2452051|50786|8429|78087|1377253|3619|21138|10|186|2114|78|32.25|37.08|35.59|1804.41|2776.02|2515.50|2892.24|77.72|1804.41|971.61|1049.33|-1543.89| +2452626|31956|1536|9530|1406427|3771|10315|4|182|2115|77|54.95|74.18|46.73|2374.81|3598.21|4231.15|5711.86|61.17|2374.81|1223.40|1284.57|-3007.75| +2452626|31956|16765|9530|1406427|3771|10315|4|297|2115|42|92.94|101.30|88.13|703.27|3701.46|3903.48|4254.60|269.83|703.27|2998.19|3268.02|-905.29| +2452626|31956|12901|9530|1406427|3771|10315|4|62|2115|20|44.88|87.96|0.87|6.61|17.40|897.60|1759.20|0.43|6.61|10.79|11.22|-886.81| +2452626|31956|6672|9530|1406427|3771|10315|4|96|2115|68|68.59|108.37|26.00|0.00|1768.00|4664.12|7369.16|123.76|0.00|1768.00|1891.76|-2896.12| +2452626|31956|13086|9530|1406427|3771|10315|4|44|2115|40|23.81|43.57|4.79|0.00|191.60|952.40|1742.80|7.66|0.00|191.60|199.26|-760.80| +2452626|31956|117|9530|1406427|3771|10315|4|76|2115|78|27.51|40.43|25.47|0.00|1986.66|2145.78|3153.54|178.79|0.00|1986.66|2165.45|-159.12| +2452626|31956|12822|9530|1406427|3771|10315|4|24|2115|98|19.06|32.40|13.28|0.00|1301.44|1867.88|3175.20|39.04|0.00|1301.44|1340.48|-566.44| +2452626|31956|13596|9530|1406427|3771|10315|4|71|2115|18|20.12|40.03|7.60|0.00|136.80|362.16|720.54|1.36|0.00|136.80|138.16|-225.36| +2452626|31956|879|9530|1406427|3771|10315|4|171|2115|55|38.98|51.06|30.12|0.00|1656.60|2143.90|2808.30|33.13|0.00|1656.60|1689.73|-487.30| +2452626|31956|601|9530|1406427|3771|10315|4|181|2115|55|76.27|132.70|94.21|0.00|5181.55|4194.85|7298.50|259.07|0.00|5181.55|5440.62|986.70| +2451468|51578|3038|23445|1123612|2979|19546|10|130|2116|61|89.02|168.24|6.72|0.00|409.92|5430.22|10262.64|20.49|0.00|409.92|430.41|-5020.30| +2451468|51578|160|23445|1123612|2979|19546|10|188|2116|95|79.83|79.83|25.54|0.00|2426.30|7583.85|7583.85|72.78|0.00|2426.30|2499.08|-5157.55| +2451468|51578|4388|23445|1123612|2979|19546|10|174|2116|38|76.35|120.63|63.93|680.21|2429.34|2901.30|4583.94|69.96|680.21|1749.13|1819.09|-1152.17| +2451468|51578|16918|23445|1123612|2979|19546|10|210|2116|9|4.72|6.46|0.83|0.00|7.47|42.48|58.14|0.52|0.00|7.47|7.99|-35.01| +2451468|51578|14636|23445|1123612|2979|19546|10|115|2116|25|74.66|137.37|87.91|0.00|2197.75|1866.50|3434.25|43.95|0.00|2197.75|2241.70|331.25| +2451468|51578|9562|23445|1123612|2979|19546|10|147|2116|39|61.99|76.24|49.55|0.00|1932.45|2417.61|2973.36|173.92|0.00|1932.45|2106.37|-485.16| +2451468|51578|5509|23445|1123612|2979|19546|10|138|2116|93|69.17|132.80|49.13|913.81|4569.09|6432.81|12350.40|0.00|913.81|3655.28|3655.28|-2777.53| +2451468|51578|4676|23445|1123612|2979|19546|10|232|2116|22|73.87|146.26|30.71|0.00|675.62|1625.14|3217.72|6.75|0.00|675.62|682.37|-949.52| +2451468|51578|2701|23445|1123612|2979|19546|10|163|2116|24|54.69|70.00|62.30|179.42|1495.20|1312.56|1680.00|105.26|179.42|1315.78|1421.04|3.22| +2451468|51578|16300|23445|1123612|2979|19546|10|168|2116|63|27.07|50.35|15.60|0.00|982.80|1705.41|3172.05|68.79|0.00|982.80|1051.59|-722.61| +2451468|51578|4207|23445|1123612|2979|19546|10|278|2116|7|24.93|27.92|8.09|6.79|56.63|174.51|195.44|3.98|6.79|49.84|53.82|-124.67| +2451468|51578|15898|23445|1123612|2979|19546|10|37|2116|58|51.61|85.67|56.54|0.00|3279.32|2993.38|4968.86|295.13|0.00|3279.32|3574.45|285.94| +2451373|48147|7315|37949|1081249|2149|28649|1|240|2117|94|17.37|32.30|21.64|0.00|2034.16|1632.78|3036.20|142.39|0.00|2034.16|2176.55|401.38| +2451373|48147|17542|37949|1081249|2149|28649|1|209|2117|61|86.66|132.58|82.19|1203.26|5013.59|5286.26|8087.38|114.30|1203.26|3810.33|3924.63|-1475.93| +2451373|48147|14641|37949|1081249|2149|28649|1|134|2117|81|77.72|136.01|42.16|0.00|3414.96|6295.32|11016.81|170.74|0.00|3414.96|3585.70|-2880.36| +2451373|48147|5240|37949|1081249|2149|28649|1|210|2117|42|39.43|62.69|38.86|0.00|1632.12|1656.06|2632.98|0.00|0.00|1632.12|1632.12|-23.94| +2451373|48147|6259|37949|1081249|2149|28649|1|297|2117|72|52.56|84.62|20.30|0.00|1461.60|3784.32|6092.64|73.08|0.00|1461.60|1534.68|-2322.72| +|48147|6982||1081249|2149|28649|1|7|2117||28.52||||368.40|427.80||0.00||368.40|368.40|-59.40| +2451373|48147|15494|37949|1081249|2149|28649|1|165|2117|94|69.74|108.09|82.14|0.00|7721.16|6555.56|10160.46|0.00|0.00|7721.16|7721.16|1165.60| +2451373|48147|31|37949|1081249|2149|28649|1|58|2117|58|3.99|6.34|5.07|0.00|294.06|231.42|367.72|2.94|0.00|294.06|297.00|62.64| +2451373|48147|14110|37949|1081249|2149|28649|1|64|2117|93|76.49|99.43|44.74|0.00|4160.82|7113.57|9246.99|41.60|0.00|4160.82|4202.42|-2952.75| +2452083|34005|15187|44440|1627964|5888|26659|4|48|2118|35|64.74|78.98|31.59|0.00|1105.65|2265.90|2764.30|77.39|0.00|1105.65|1183.04|-1160.25| +2452083|34005|14413|44440|1627964|5888|26659|4|246|2118|29|7.73|15.30|8.87|0.00|257.23|224.17|443.70|18.00|0.00|257.23|275.23|33.06| +2452083|34005|10629|44440|1627964|5888|26659|4|254|2118|85|42.87|64.73|5.17|0.00|439.45|3643.95|5502.05|26.36|0.00|439.45|465.81|-3204.50| +2452083|34005|15165|44440|1627964|5888|26659|4|55|2118|4|54.15|101.80|10.18|0.00|40.72|216.60|407.20|2.85|0.00|40.72|43.57|-175.88| +2452083|34005|17107|44440|1627964|5888|26659|4|146|2118|51|17.60|27.80|6.95|0.00|354.45|897.60|1417.80|10.63|0.00|354.45|365.08|-543.15| +2452083|34005|13107|44440|1627964|5888|26659|4|268|2118|80|57.97|67.24|26.22|0.00|2097.60|4637.60|5379.20|146.83|0.00|2097.60|2244.43|-2540.00| +2452083|34005|8303|44440|1627964|5888|26659|4|161|2118|55|98.20|191.49|158.93|8566.32|8741.15|5401.00|10531.95|1.74|8566.32|174.83|176.57|-5226.17| +2452083|34005|10013|44440|1627964|5888|26659|4|197|2118|35|20.48|29.49|20.64|0.00|722.40|716.80|1032.15|28.89|0.00|722.40|751.29|5.60| +2452083|34005|15847|44440|1627964|5888|26659|4|279|2118|7|70.21|129.18|82.67|0.00|578.69|491.47|904.26|40.50|0.00|578.69|619.19|87.22| +2452083|34005|777|44440|1627964|5888|26659|4|247|2118|90|45.30|81.08|71.35|0.00|6421.50|4077.00|7297.20|256.86|0.00|6421.50|6678.36|2344.50| +2452083|34005|1491|44440|1627964|5888|26659|4|9|2118|59|45.10|55.02|8.25|0.00|486.75|2660.90|3246.18|38.94|0.00|486.75|525.69|-2174.15| +2452211|53933|3193|64182|1647466|6303|11187|10|215|2119|14|15.49|26.64|13.58|0.00|190.12|216.86|372.96|5.70|0.00|190.12|195.82|-26.74| +2452211|53933|16011|64182|1647466|6303|11187|10|20|2119|46|37.22|70.34|51.34|0.00|2361.64|1712.12|3235.64|94.46|0.00|2361.64|2456.10|649.52| +2452211|53933|10205|64182|1647466|6303|11187|10|39|2119|53|69.25|92.79|80.72|0.00|4278.16|3670.25|4917.87|256.68|0.00|4278.16|4534.84|607.91| +2452211|53933|2365|64182|1647466|6303|11187|10|139|2119|99|66.16|129.01|112.23|0.00|11110.77|6549.84|12771.99|888.86|0.00|11110.77|11999.63|4560.93| +2452211|53933|13989|64182|1647466|6303|11187|10|55|2119|1|10.73|14.37|7.90|0.00|7.90|10.73|14.37|0.39|0.00|7.90|8.29|-2.83| +2452211|53933|2415|64182|1647466|6303|11187|10|250|2119|1|94.55|166.40|113.15|0.00|113.15|94.55|166.40|6.78|0.00|113.15|119.93|18.60| +2452211|53933|3121|64182|1647466|6303|11187|10|219|2119|97|97.07|110.65|17.70|1545.21|1716.90|9415.79|10733.05|15.45|1545.21|171.69|187.14|-9244.10| +2452211|53933|10855|64182|1647466|6303|11187|10|262|2119|53|60.78|102.71|83.19|0.00|4409.07|3221.34|5443.63|88.18|0.00|4409.07|4497.25|1187.73| +2452211|53933|8519|64182|1647466|6303|11187|10|112|2119|83|41.94|54.52|17.44|0.00|1447.52|3481.02|4525.16|115.80|0.00|1447.52|1563.32|-2033.50| +2452211|53933|16747|64182|1647466|6303|11187|10|44|2119|10|40.42|55.37|33.22|179.38|332.20|404.20|553.70|7.64|179.38|152.82|160.46|-251.38| +2451245||12949|23274|1735412|2303||4|109|2120|22||72.57|44.99|0.00|989.78|||0.00|0.00|||-201.74| +2451245|62571|11191|23274|1735412|2303|19594|4|233|2120|26|23.28|40.04|14.41|0.00|374.66|605.28|1041.04|14.98|0.00|374.66|389.64|-230.62| +2451245|62571|11156|23274|1735412|2303|19594|4|251|2120|62|27.41|50.43|10.59|0.00|656.58|1699.42|3126.66|52.52|0.00|656.58|709.10|-1042.84| +2451245|62571|4238|23274|1735412|2303|19594|4|146|2120|79|99.51|115.43|15.00|1007.25|1185.00|7861.29|9118.97|14.22|1007.25|177.75|191.97|-7683.54| +2451245|62571|8527|23274|1735412|2303|19594|4|47|2120|46|10.61|19.73|12.62|0.00|580.52|488.06|907.58|11.61|0.00|580.52|592.13|92.46| +2451245|62571|15880|23274|1735412|2303|19594|4|15|2120|52|85.09|137.84|5.51|108.87|286.52|4424.68|7167.68|10.65|108.87|177.65|188.30|-4247.03| +2451245|62571|5264|23274|1735412|2303|19594|4|7|2120|12|71.52|137.31|124.95|0.00|1499.40|858.24|1647.72|44.98|0.00|1499.40|1544.38|641.16| +2451245|62571|11035|23274|1735412|2303|19594|4|205|2120|40|20.89|38.43|26.90|946.88|1076.00|835.60|1537.20|10.32|946.88|129.12|139.44|-706.48| +2451245|62571|7195|23274|1735412|2303|19594|4|166|2120|34|66.47|116.32|95.38|616.15|3242.92|2259.98|3954.88|157.60|616.15|2626.77|2784.37|366.79| +2451245|62571|14995|23274|1735412|2303|19594|4|60|2120|17|20.98|27.69|6.09|0.00|103.53|356.66|470.73|2.07|0.00|103.53|105.60|-253.13| +2451245|62571|1699|23274|1735412|2303|19594|4|275|2120|16|36.68|43.28|13.41|0.00|214.56|586.88|692.48|0.00|0.00|214.56|214.56|-372.32| +2451245|62571|1400|23274|1735412|2303|19594|4|66|2120|47|79.64|112.29|65.12|0.00|3060.64|3743.08|5277.63|153.03|0.00|3060.64|3213.67|-682.44| +2451245|62571|17410|23274|1735412|2303|19594|4|238|2120|55|46.15|59.99|18.59|0.00|1022.45|2538.25|3299.45|61.34|0.00|1022.45|1083.79|-1515.80| +2451245|62571|4892|23274|1735412|2303|19594|4|116|2120|6|10.03|11.93|0.71|0.21|4.26|60.18|71.58|0.36|0.21|4.05|4.41|-56.13| +2451245|62571|16592|23274|1735412|2303|19594|4|205|2120|24|93.68|127.40|38.22|0.00|917.28|2248.32|3057.60|73.38|0.00|917.28|990.66|-1331.04| +2452227|29445|2499|85243|342360|5430|39347|7|67|2121|76|96.35|147.41|101.71|0.00|7729.96|7322.60|11203.16|77.29|0.00|7729.96|7807.25|407.36| +2452227|29445|16343|85243|342360|5430|39347|7|117|2121|2|10.82|19.90|1.39|0.00|2.78|21.64|39.80|0.13|0.00|2.78|2.91|-18.86| +2452227|29445|7349|85243|342360|5430|39347|7|267|2121|36|72.95|111.61|55.80|0.00|2008.80|2626.20|4017.96|120.52|0.00|2008.80|2129.32|-617.40| +2452227|29445|6129|85243|342360|5430|39347|7|271|2121|10|73.92|110.14|84.80|0.00|848.00|739.20|1101.40|33.92|0.00|848.00|881.92|108.80| +2452227|29445|14403|85243|342360|5430|39347|7|249|2121|7|69.80|137.50|122.37|0.00|856.59|488.60|962.50|42.82|0.00|856.59|899.41|367.99| +2452227|29445|12981|85243|342360|5430|39347|7|231|2121|30|48.25|54.04|44.85|0.00|1345.50|1447.50|1621.20|26.91|0.00|1345.50|1372.41|-102.00| +2452227|29445|151|85243|342360|5430|39347|7|172|2121|25|46.63|54.55|22.91|406.65|572.75|1165.75|1363.75|9.96|406.65|166.10|176.06|-999.65| +2452227|29445|1073|85243|342360|5430|39347|7|244|2121|31|78.20|138.41|94.11|0.00|2917.41|2424.20|4290.71|0.00|0.00|2917.41|2917.41|493.21| +2452227|29445|11011|85243|342360|5430|39347|7|257|2121|47|74.97|117.70|67.08|0.00|3152.76|3523.59|5531.90|31.52|0.00|3152.76|3184.28|-370.83| +2452227|29445|17947|85243|342360|5430|39347|7|242|2121|76|72.35|117.93|16.51|0.00|1254.76|5498.60|8962.68|0.00|0.00|1254.76|1254.76|-4243.84| +2452227|29445|1331|85243|342360|5430|39347|7|229|2121|50|13.03|18.11|13.76|0.00|688.00|651.50|905.50|0.00|0.00|688.00|688.00|36.50| +2452227|29445|6981|85243|342360|5430|39347|7|143|2121|68|89.17|166.74|80.03|0.00|5442.04|6063.56|11338.32|0.00|0.00|5442.04|5442.04|-621.52| +2452227|29445|7835|85243|342360|5430|39347|7|153|2121|66|3.82|7.52|7.14|0.00|471.24|252.12|496.32|9.42|0.00|471.24|480.66|219.12| +2452227|29445|3817|85243|342360|5430|39347|7|184|2121|93|30.47|52.10|23.96|0.00|2228.28|2833.71|4845.30|0.00|0.00|2228.28|2228.28|-605.43| +2451614|63486|7058|29835|1217811|2587|11763|10|125|2122|3|73.44|144.67|141.77|0.00|425.31|220.32|434.01|8.50|0.00|425.31|433.81|204.99| +2451614|63486|13379|29835|1217811|2587|11763|10|141|2122|53|62.88|88.66|46.10|0.00|2443.30|3332.64|4698.98|146.59|0.00|2443.30|2589.89|-889.34| +||10964|||2587||10|68|2122||||||71.96|384.48||||71.96||-312.52| +2451614|63486|6493|29835|1217811|2587|11763|10|122|2122|90|16.46|23.86|16.94|0.00|1524.60|1481.40|2147.40|106.72|0.00|1524.60|1631.32|43.20| +2451614|63486|13055|29835|1217811|2587|11763|10|60|2122|19|77.05|79.36|12.69|0.00|241.11|1463.95|1507.84|21.69|0.00|241.11|262.80|-1222.84| +2451614||11857||1217811||||65|2122|72|||||||9398.16|||7518.24||| +2451614|63486|8717|29835|1217811|2587|11763|10|39|2122|11|75.95|125.31|50.12|253.60|551.32|835.45|1378.41|26.79|253.60|297.72|324.51|-537.73| +2451614||7355|29835|1217811|2587|11763|10|243|2122||18.68|||||||2.23||223.44|225.67|-486.40| +2451614|63486|12896|29835|1217811|2587|11763|10|210|2122|60|52.38|68.61|56.94|3040.59|3416.40|3142.80|4116.60|11.27|3040.59|375.81|387.08|-2766.99| +2451614|63486|6341|29835|1217811|2587|11763|10|275|2122|10|96.83|101.67|1.01|0.00|10.10|968.30|1016.70|0.10|0.00|10.10|10.20|-958.20| +2452206|36056|8459|57221|856531|1088|41245|2|269|2123|100|48.47|56.22|47.78|0.00|4778.00|4847.00|5622.00|382.24|0.00|4778.00|5160.24|-69.00| +2452206|36056|9701|57221|856531|1088|41245|2|11|2123|31|95.06|132.13|52.85|0.00|1638.35|2946.86|4096.03|65.53|0.00|1638.35|1703.88|-1308.51| +2452206|36056|6415|57221|856531|1088|41245|2|79|2123|28|43.43|47.33|30.76|0.00|861.28|1216.04|1325.24|17.22|0.00|861.28|878.50|-354.76| +2452206|36056|12501|57221|856531|1088|41245|2|141|2123|86|3.50|4.79|4.55|0.00|391.30|301.00|411.94|15.65|0.00|391.30|406.95|90.30| +2452206|36056|8223|57221|856531|1088|41245|2|232|2123|4|42.00|49.14|17.69|0.00|70.76|168.00|196.56|0.00|0.00|70.76|70.76|-97.24| +2452206|36056|13621|57221|856531|1088|41245|2|291|2123|22|75.61|135.34|40.60|0.00|893.20|1663.42|2977.48|17.86|0.00|893.20|911.06|-770.22| +2452206|36056|14315|57221|856531|1088|41245|2|71|2123|35|12.14|19.90|13.13|0.00|459.55|424.90|696.50|27.57|0.00|459.55|487.12|34.65| +2452206|36056|8969|57221|856531|1088|41245|2|49|2123|67|78.43|140.38|11.23|0.00|752.41|5254.81|9405.46|45.14|0.00|752.41|797.55|-4502.40| +2452206|36056|8393|57221|856531|1088|41245|2|15|2123|16|28.40|36.06|19.83|0.00|317.28|454.40|576.96|19.03|0.00|317.28|336.31|-137.12| +2452206|36056|13095|57221|856531|1088|41245|2|171|2123|95|87.08|160.22|25.63|0.00|2434.85|8272.60|15220.90|97.39|0.00|2434.85|2532.24|-5837.75| +2452206|36056|6823|57221|856531|1088|41245|2|90|2123|7|55.04|105.67|50.72|60.35|355.04|385.28|739.69|2.94|60.35|294.69|297.63|-90.59| +2452206|36056|7089|57221|856531|1088|41245|2|72|2123|90|37.12|49.74|17.40|0.00|1566.00|3340.80|4476.60|140.94|0.00|1566.00|1706.94|-1774.80| +2452206||8367||856531|1088|41245|2||2123||48.31|96.13|5.76|0.00|||||0.00|||-638.25| +2452206|36056|9371|57221|856531|1088|41245|2|160|2123|64|51.11|84.33|61.56|0.00|3939.84|3271.04|5397.12|196.99|0.00|3939.84|4136.83|668.80| +2452206|36056|13287|57221|856531|1088|41245|2|160|2123|92|1.13|1.43|0.60|0.00|55.20|103.96|131.56|3.31|0.00|55.20|58.51|-48.76| +2451869|49943|12137|97110|682025|2140|10208|8|78|2124|80|20.94|21.35|19.21|0.00|1536.80|1675.20|1708.00|61.47|0.00|1536.80|1598.27|-138.40| +2451869|49943|13675|97110|682025|2140|10208|8|54|2124|80|65.81|80.94|68.79|0.00|5503.20|5264.80|6475.20|165.09|0.00|5503.20|5668.29|238.40| +2451869|49943|10963|97110|682025|2140|10208|8|218|2124|50|89.68|154.24|49.35|0.00|2467.50|4484.00|7712.00|123.37|0.00|2467.50|2590.87|-2016.50| +2451869|49943|10058|97110|682025|2140|10208|8|76|2124|91|54.62|63.90|46.64|0.00|4244.24|4970.42|5814.90|212.21|0.00|4244.24|4456.45|-726.18| +2451869|49943|7951|97110|682025|2140|10208|8|163|2124|12|90.40|122.94|93.43|0.00|1121.16|1084.80|1475.28|67.26|0.00|1121.16|1188.42|36.36| +2451869|49943|16411|97110|682025|2140|10208|8|177|2124|28|7.39|9.16|4.12|0.00|115.36|206.92|256.48|2.30|0.00|115.36|117.66|-91.56| +2451869|49943|13625|97110||2140|10208|||2124|96|73.71|75.92|||6996.48|7076.16|7288.32|279.85||||-79.68| +2451869|49943|12809|97110|682025|2140|10208|8|234|2124|52|19.72|35.89|27.27|14.18|1418.04|1025.44|1866.28|70.19|14.18|1403.86|1474.05|378.42| +2451869|49943|3415|97110|682025|2140|10208|8|138|2124|27|77.97|130.98|111.33|0.00|3005.91|2105.19|3536.46|0.00|0.00|3005.91|3005.91|900.72| +2451869|49943|6223|97110|682025|2140|10208|8|178|2124|70|60.93|103.58|42.46|0.00|2972.20|4265.10|7250.60|59.44|0.00|2972.20|3031.64|-1292.90| +||4418|97110|682025|2140||8|51|2124||||||7929.82|||||7929.82||| +2451869|49943|107|97110|682025|2140|10208|8|120|2124|51|4.26|4.43|0.88|0.00|44.88|217.26|225.93|2.24|0.00|44.88|47.12|-172.38| +2451869|49943|277|97110|682025|2140|10208|8|181|2124|52|39.19|60.74|1.21|0.00|62.92|2037.88|3158.48|4.40|0.00|62.92|67.32|-1974.96| +2451869|49943|4343|97110|682025|2140|10208|8|22|2124|16|53.36|85.90|5.15|0.00|82.40|853.76|1374.40|4.94|0.00|82.40|87.34|-771.36| +2451854|58955|5726|24434|1254715|2149|48030|2|230|2125|60|88.30|122.73|82.22|0.00|4933.20|5298.00|7363.80|49.33|0.00|4933.20|4982.53|-364.80| +2451854|58955|3691|24434|1254715|2149|48030|2|164|2125|17|81.69|128.25|116.70|0.00|1983.90|1388.73|2180.25|178.55|0.00|1983.90|2162.45|595.17| +2451854|58955|9848|24434|1254715|2149|48030|2|227|2125|51|75.72|135.53|29.81|0.00|1520.31|3861.72|6912.03|45.60|0.00|1520.31|1565.91|-2341.41| +2451854|58955|12931|24434|1254715|2149|48030|2|77|2125|2|17.16|24.36|5.35|0.00|10.70|34.32|48.72|0.32|0.00|10.70|11.02|-23.62| +2451854|58955|12833|24434|1254715|2149|48030|2|78|2125|25|19.98|22.77|15.48|0.00|387.00|499.50|569.25|0.00|0.00|387.00|387.00|-112.50| +2451854|58955|13586|24434|1254715|2149|48030|2|82|2125|50|53.35|94.42|92.53|0.00|4626.50|2667.50|4721.00|92.53|0.00|4626.50|4719.03|1959.00| +2451854|58955|17981|24434|1254715|2149|48030|2|178|2125|83|36.95|50.99|24.47|873.33|2031.01|3066.85|4232.17|104.19|873.33|1157.68|1261.87|-1909.17| +2451854|58955|179|24434|1254715|2149|48030|2|213|2125|10|95.96|183.28|62.31|0.00|623.10|959.60|1832.80|18.69|0.00|623.10|641.79|-336.50| +2451854|58955|6596|24434|1254715|2149|48030|2|101|2125|27|16.16|20.68|7.44|0.00|200.88|436.32|558.36|6.02|0.00|200.88|206.90|-235.44| +2451854|58955|17989|24434|1254715|2149|48030|2|213|2125|55|75.37|121.34|36.40|0.00|2002.00|4145.35|6673.70|180.18|0.00|2002.00|2182.18|-2143.35| +2451854|58955|13568|24434|1254715|2149|48030|2|203|2125|21|73.92|113.83|47.80|772.92|1003.80|1552.32|2390.43|16.16|772.92|230.88|247.04|-1321.44| +2451854|58955|6829|24434|1254715|2149|48030|2|246|2125|46|14.88|18.00|5.40|0.00|248.40|684.48|828.00|14.90|0.00|248.40|263.30|-436.08| +2451854|58955|2183|24434|1254715|2149|48030|2|264|2125|31|88.99|124.58|78.48|0.00|2432.88|2758.69|3861.98|97.31|0.00|2432.88|2530.19|-325.81| +2451854|58955|1087|24434|1254715|2149|48030|2|284|2125|38|91.76|175.26|33.29|0.00|1265.02|3486.88|6659.88|113.85|0.00|1265.02|1378.87|-2221.86| +2451854|58955|11186|24434|1254715|2149|48030|2|59|2125|11|96.32|187.82|88.27|0.00|970.97|1059.52|2066.02|67.96|0.00|970.97|1038.93|-88.55| +2451854|58955|14281|24434|1254715|2149|48030|2|263|2125|33|96.69|142.13|106.59|0.00|3517.47|3190.77|4690.29|175.87|0.00|3517.47|3693.34|326.70| +2450934|54357|16303|38970|1641793|5863|41933|1|9|2126|37|54.85|66.36|9.95|0.00|368.15|2029.45|2455.32|14.72|0.00|368.15|382.87|-1661.30| +2450934|54357|14713|||5863|41933|1||2126|||98.02|69.59|||2380.68||23.66||2366.06||-14.62| +2450934|54357|772|38970|1641793|5863|41933|1|60|2126|32|77.06|107.11|20.35|449.32|651.20|2465.92|3427.52|16.15|449.32|201.88|218.03|-2264.04| +2450934|54357|12394|38970|1641793|5863|41933|1|5|2126|78|53.16|106.32|42.52|0.00|3316.56|4146.48|8292.96|265.32|0.00|3316.56|3581.88|-829.92| +2450934|54357|9874|38970|1641793|5863|41933|1|50|2126|16|77.00|127.05|34.30|0.00|548.80|1232.00|2032.80|10.97|0.00|548.80|559.77|-683.20| +2450934|54357|3637|38970|1641793|5863|41933|1|159|2126|57|19.83|22.60|13.56|0.00|772.92|1130.31|1288.20|38.64|0.00|772.92|811.56|-357.39| +2450934|54357|12590|38970|1641793|5863|41933|1|135|2126|59|7.17|13.76|9.21|483.61|543.39|423.03|811.84|0.00|483.61|59.78|59.78|-363.25| +2450934|54357|13294|38970|1641793|5863|41933|1|243|2126|32|43.51|53.51|31.57|0.00|1010.24|1392.32|1712.32|30.30|0.00|1010.24|1040.54|-382.08| +2450934|54357|16291|38970|1641793|5863|41933|1|79|2126|21|56.00|108.64|5.43|0.00|114.03|1176.00|2281.44|10.26|0.00|114.03|124.29|-1061.97| +2450934|54357|2857|38970|1641793|5863|41933|1|185|2126|34|86.77|170.06|161.55|0.00|5492.70|2950.18|5782.04|109.85|0.00|5492.70|5602.55|2542.52| +2450934|54357|5834|38970||5863||1|113|2126|97||||0.00|5127.42|7586.37||51.27|0.00|5127.42|5178.69|| +2450934|54357|3818|38970|1641793|5863|41933|1|84|2126|60|45.42|86.75|85.88|2730.98|5152.80|2725.20|5205.00|0.00|2730.98|2421.82|2421.82|-303.38| +2450934|54357|2144|38970|1641793|5863|41933|1|26|2126|88|29.06|48.53|15.52|0.00|1365.76|2557.28|4270.64|95.60|0.00|1365.76|1461.36|-1191.52| +2452101|48677|7491|85055|578065|5475|37685|8|101|2127|94|58.50|104.13|62.47|0.00|5872.18|5499.00|9788.22|176.16|0.00|5872.18|6048.34|373.18| +2452101|48677|5927|85055|578065|5475|37685|8|276|2127|31|5.24|8.59|5.32|148.42|164.92|162.44|266.29|1.48|148.42|16.50|17.98|-145.94| +2452101|48677|15321|85055|578065|5475|37685|8|71|2127|30|2.47|3.87|0.69|15.52|20.70|74.10|116.10|0.10|15.52|5.18|5.28|-68.92| +2452101|48677|9497|85055|578065|5475|37685|8|133|2127|76|36.49|67.87|25.11|0.00|1908.36|2773.24|5158.12|0.00|0.00|1908.36|1908.36|-864.88| +2452101|48677|4127|85055|578065|5475|37685|8|124|2127|89|93.63|145.12|75.46|0.00|6715.94|8333.07|12915.68|402.95|0.00|6715.94|7118.89|-1617.13| +2452101|48677|1257|85055|578065|5475|37685|8|35|2127|48|12.44|24.63|14.77|0.00|708.96|597.12|1182.24|0.00|0.00|708.96|708.96|111.84| +||17071|||5475|37685|8|86|2127||77.29|106.66||||2009.54|2773.16|||2717.52||707.98| +2452101|48677|9255|85055|578065|5475|37685|8|286|2127|47|88.06|155.86|43.64|0.00|2051.08|4138.82|7325.42|20.51|0.00|2051.08|2071.59|-2087.74| +2452101|48677|12225|85055|578065|5475|37685|8|222|2127|40|92.85|163.41|29.41|0.00|1176.40|3714.00|6536.40|94.11|0.00|1176.40|1270.51|-2537.60| +2452101|48677|16955|85055|578065|5475|37685|8|285|2127|50|2.93|2.95|0.91|0.00|45.50|146.50|147.50|4.09|0.00|45.50|49.59|-101.00| +2452101|48677|13749|85055|578065|5475|37685|8|203|2127|50|41.58|67.35|25.59|0.00|1279.50|2079.00|3367.50|0.00|0.00|1279.50|1279.50|-799.50| +2452101|48677|6173|85055|578065|5475|37685|8|270|2127|61|5.14|6.52|5.67|0.00|345.87|313.54|397.72|27.66|0.00|345.87|373.53|32.33| +2452101|48677|13909|85055|578065|5475|37685|8|117|2127|31|20.35|32.15|29.25|0.00|906.75|630.85|996.65|27.20|0.00|906.75|933.95|275.90| +2452101|48677|16879|85055|578065|5475|37685|8|237|2127|32|34.63|67.52|5.40|0.00|172.80|1108.16|2160.64|10.36|0.00|172.80|183.16|-935.36| +2452101|48677|2641|85055|578065|5475|37685|8|78|2127|47|58.66|63.93|0.63|28.42|29.61|2757.02|3004.71|0.00|28.42|1.19|1.19|-2755.83| +2452101|48677|17795|85055|578065|5475|37685|8|299|2127|11|75.13|114.19|55.95|0.00|615.45|826.43|1256.09|0.00|0.00|615.45|615.45|-210.98| +2452275|46086|5033|42963|1833805|3847|1136|7|218|2128|29|36.68|41.81|24.24|0.00|702.96|1063.72|1212.49|56.23|0.00|702.96|759.19|-360.76| +2452275|46086|11043|42963|1833805|3847|1136|7|125|2128|76|76.69|129.60|94.60|0.00|7189.60|5828.44|9849.60|647.06|0.00|7189.60|7836.66|1361.16| +2452275|46086|8973|42963|1833805|3847|1136|7|239|2128|3|62.68|75.84|65.22|5.86|195.66|188.04|227.52|7.59|5.86|189.80|197.39|1.76| +2452275|46086|15379|42963|1833805|3847|1136|7|209|2128|88|39.18|67.38|33.01|0.00|2904.88|3447.84|5929.44|0.00|0.00|2904.88|2904.88|-542.96| +2452275|46086|2367|42963|1833805|3847|1136|7|249|2128|94|59.45|74.90|8.23|0.00|773.62|5588.30|7040.60|0.00|0.00|773.62|773.62|-4814.68| +2452275|46086|8027|42963|1833805|3847|1136|7|131|2128|59|40.37|65.80|61.85|2736.86|3649.15|2381.83|3882.20|63.86|2736.86|912.29|976.15|-1469.54| +2452275|46086|7353|42963|1833805|3847|1136|7|182|2128|24|53.20|80.86|7.27|94.21|174.48|1276.80|1940.64|1.60|94.21|80.27|81.87|-1196.53| +2452275|46086|3803|42963|1833805|3847|1136|7|37|2128|5|52.65|92.13|58.96|0.00|294.80|263.25|460.65|23.58|0.00|294.80|318.38|31.55| +|46086|12025|42963|1833805||1136|7|239|2128|||113.83||0.00|||||0.00|56.90|58.60|-853.80| +2452275|46086|1469|42963|1833805|3847|1136|7|175|2128|40|81.46|114.85|72.35|0.00|2894.00|3258.40|4594.00|57.88|0.00|2894.00|2951.88|-364.40| +2452275|46086|2073|42963|1833805|3847|1136|7|271|2128|100|87.71|96.48|47.27|0.00|4727.00|8771.00|9648.00|141.81|0.00|4727.00|4868.81|-4044.00| +2451963|67926|14537|72676|591932|2835|48491|10|121|2129|1|52.29|63.79|58.04|11.02|58.04|52.29|63.79|4.23|11.02|47.02|51.25|-5.27| +2451963||647|72676|591932|||10||2129||39.26||24.73|445.14||981.50|1030.50||445.14|||| +2451963||13593||591932|||||2129|63||||0.00||3356.64|3994.20|27.15|0.00|2715.93|2743.08|| +2451963|67926|1039|72676|591932|2835|48491|10|25|2129|62|75.13|110.44|8.83|104.01|547.46|4658.06|6847.28|26.60|104.01|443.45|470.05|-4214.61| +2451963|67926|14757|72676|591932|2835|48491|10|138|2129|81|29.55|31.02|24.50|0.00|1984.50|2393.55|2512.62|19.84|0.00|1984.50|2004.34|-409.05| +2451963|67926|1271|72676|591932|2835|48491|10|29|2129|82|16.66|30.98|7.74|0.00|634.68|1366.12|2540.36|25.38|0.00|634.68|660.06|-731.44| +2451963|67926|14699|72676|591932|2835|48491|10|74|2129|55|91.17|147.69|51.69|0.00|2842.95|5014.35|8122.95|0.00|0.00|2842.95|2842.95|-2171.40| +2451963|67926|7415|72676|591932|2835|48491|10|149|2129|89|84.21|159.15|133.68|0.00|11897.52|7494.69|14164.35|237.95|0.00|11897.52|12135.47|4402.83| +2451963|67926|4891|72676|591932|2835|48491|10|256|2129|7|69.72|123.40|28.38|0.00|198.66|488.04|863.80|3.97|0.00|198.66|202.63|-289.38| +2452621|40234|3351|34401|460628|2855|19035|7|173|2130|93|53.59|88.95|30.24|0.00|2812.32|4983.87|8272.35|253.10|0.00|2812.32|3065.42|-2171.55| +||12390|34401|460628|2855|19035||270|2130||62.73||||562.97||1251.34|33.77||562.97||| +2452621|40234|10890|34401|460628|2855|19035|7|38|2130|9|72.29|139.51|32.08|0.00|288.72|650.61|1255.59|14.43|0.00|288.72|303.15|-361.89| +2452621|40234|8071|34401|460628|2855|19035|7|286|2130|99|21.76|34.16|16.05|0.00|1588.95|2154.24|3381.84|127.11|0.00|1588.95|1716.06|-565.29| +2452621|40234|14436|34401|460628|2855|19035|7|201|2130|68|64.27|106.04|6.36|0.00|432.48|4370.36|7210.72|0.00|0.00|432.48|432.48|-3937.88| +2452621|40234|8775|34401|460628|2855|19035|7|77|2130|89|45.65|51.58|50.54|0.00|4498.06|4062.85|4590.62|44.98|0.00|4498.06|4543.04|435.21| +2452621|40234|1537|34401|460628|2855|19035|7|41|2130|83|48.30|92.25|38.74|0.00|3215.42|4008.90|7656.75|0.00|0.00|3215.42|3215.42|-793.48| +|40234|13152|||2855|19035|7||2130|72|||28.56|||1995.12|3371.04|185.06||||61.20| +2452621|40234|8172|34401|460628|2855|19035|7|287|2130|29|19.85|39.50|20.54|0.00|595.66|575.65|1145.50|53.60|0.00|595.66|649.26|20.01| +2452621|40234|10891|34401|460628|2855|19035|7|93|2130|53|82.42|84.06|79.85|0.00|4232.05|4368.26|4455.18|84.64|0.00|4232.05|4316.69|-136.21| +2452517|47300|15036|19239|1101976|4713|27450|8|22|2131|83|6.42|11.62|11.27|0.00|935.41|532.86|964.46|28.06|0.00|935.41|963.47|402.55| +2452517|47300|16917|19239|1101976|4713|27450|8|192|2131|31|6.07|7.52|4.96|89.18|153.76|188.17|233.12|3.87|89.18|64.58|68.45|-123.59| +2452517|47300|15763|19239|1101976|4713|27450|8|296|2131|35|36.64|45.43|22.71|0.00|794.85|1282.40|1590.05|39.74|0.00|794.85|834.59|-487.55| +2452517|47300|16669|19239|1101976|4713|27450|8|56|2131|74|54.58|58.94|18.27|0.00|1351.98|4038.92|4361.56|27.03|0.00|1351.98|1379.01|-2686.94| +2452517|47300|1633|19239|1101976|4713|27450|8|56|2131|92|71.94|100.71|63.44|933.83|5836.48|6618.48|9265.32|0.00|933.83|4902.65|4902.65|-1715.83| +2452517|47300|8191|19239|1101976|4713|27450|8|249|2131|18|48.26|90.72|74.39|0.00|1339.02|868.68|1632.96|53.56|0.00|1339.02|1392.58|470.34| +2452517|47300|673|19239|1101976|4713|27450|8|57|2131|29|36.44|66.32|22.54|0.00|653.66|1056.76|1923.28|13.07|0.00|653.66|666.73|-403.10| +2452517|47300|15861|19239|1101976|4713|27450|8|251|2131|86|99.43|117.32|26.98|0.00|2320.28|8550.98|10089.52|116.01|0.00|2320.28|2436.29|-6230.70| +2452517|47300|16485|19239|1101976|4713|27450|8|105|2131|6|31.17|38.65|28.98|0.00|173.88|187.02|231.90|6.95|0.00|173.88|180.83|-13.14| +2452517|47300|4621|19239|1101976|4713|27450|8|55|2131|90|36.27|54.76|36.14|3024.91|3252.60|3264.30|4928.40|6.83|3024.91|227.69|234.52|-3036.61| +2452517|47300|1861|19239|1101976|4713|27450|8|247|2131|30|19.74|28.82|6.34|123.63|190.20|592.20|864.60|0.00|123.63|66.57|66.57|-525.63| +2452595|55538|4098|11464|703766|6115|42864|7|67|2132|15|45.77|80.55|11.27|0.00|169.05|686.55|1208.25|15.21|0.00|169.05|184.26|-517.50| +2452595|55538|7503|11464|703766|6115|42864|7|29|2132|13|70.26|82.90|39.79|0.00|517.27|913.38|1077.70|5.17|0.00|517.27|522.44|-396.11| +2452595|55538|5244|11464|703766|6115|42864|7|246|2132|3|61.08|94.06|48.91|0.00|146.73|183.24|282.18|5.86|0.00|146.73|152.59|-36.51| +2452595|55538|3235|11464|703766|6115|42864|7|122|2132|44|78.83|116.66|50.16|0.00|2207.04|3468.52|5133.04|176.56|0.00|2207.04|2383.60|-1261.48| +2452595|55538|10317|11464|703766|6115|42864|7|131|2132|61|23.44|38.20|1.52|0.00|92.72|1429.84|2330.20|5.56|0.00|92.72|98.28|-1337.12| +2452595|55538|11466|11464|703766|6115|42864|7|251|2132|32|3.56|4.77|0.47|0.00|15.04|113.92|152.64|0.90|0.00|15.04|15.94|-98.88| +2452595|55538|10920|11464|703766|6115|42864|7|48|2132|79|17.83|19.79|1.58|0.00|124.82|1408.57|1563.41|3.74|0.00|124.82|128.56|-1283.75| +2452595|55538|5827|11464|703766|6115|42864|7|14|2132|37|24.87|41.78|35.51|0.00|1313.87|920.19|1545.86|52.55|0.00|1313.87|1366.42|393.68| +2452595|55538|5011|11464|703766|6115|42864|7|273|2132|73|69.81|72.60|31.21|0.00|2278.33|5096.13|5299.80|136.69|0.00|2278.33|2415.02|-2817.80| +2452595|55538|14244|11464|703766|6115|42864|7|192|2132|19|13.62|22.47|14.38|0.00|273.22|258.78|426.93|5.46|0.00|273.22|278.68|14.44| +2452595|55538|14940|11464|703766|6115|42864|7|228|2132|35|62.81|105.52|58.03|0.00|2031.05|2198.35|3693.20|182.79|0.00|2031.05|2213.84|-167.30| +2452595|55538|8803|11464|703766|6115|42864|7|92|2132|2|82.56|89.99|80.09|94.50|160.18|165.12|179.98|1.31|94.50|65.68|66.99|-99.44| +2451926|71783|16025|50009|350406|1203|8400|4|86|2133|38|68.48|71.21|42.72|0.00|1623.36|2602.24|2705.98|0.00|0.00|1623.36|1623.36|-978.88| +2451926|71783|7653|50009|350406|1203|8400|4|298|2133|32|76.58|145.50|136.77|0.00|4376.64|2450.56|4656.00|306.36|0.00|4376.64|4683.00|1926.08| +2451926|71783|12557|50009|350406|1203|8400|4|155|2133|33|75.01|89.26|72.30|0.00|2385.90|2475.33|2945.58|119.29|0.00|2385.90|2505.19|-89.43| +2451926|71783|16271|50009|350406|1203|8400|4|21|2133|48|83.58|116.17|70.86|0.00|3401.28|4011.84|5576.16|34.01|0.00|3401.28|3435.29|-610.56| +2451926|71783|1849|50009|350406|1203|8400|4|184|2133|63|81.07|81.88|58.13|3002.99|3662.19|5107.41|5158.44|13.18|3002.99|659.20|672.38|-4448.21| +2451926|71783|9043|50009|350406|1203|8400|4|155|2133|11|29.34|31.98|24.94|0.00|274.34|322.74|351.78|10.97|0.00|274.34|285.31|-48.40| +2451926|71783|7935|50009|350406|1203|8400|4|94|2133|81|36.67|52.07|8.33|0.00|674.73|2970.27|4217.67|40.48|0.00|674.73|715.21|-2295.54| +|71783|13461|50009|||||98|2133|17|87.66|||0.00|||||0.00|394.23||-1095.99| +2451926|71783|16735|50009|350406|1203|8400|4|53|2133|5|30.56|38.50|18.86|0.00|94.30|152.80|192.50|4.71|0.00|94.30|99.01|-58.50| +2451926|71783|12875|50009|350406|1203|8400|4|146|2133|27|22.67|34.91|31.41|542.76|848.07|612.09|942.57|15.26|542.76|305.31|320.57|-306.78| +2451926|71783|7955|50009|350406|1203|8400|4|4|2133|94|33.63|53.13|12.75|0.00|1198.50|3161.22|4994.22|71.91|0.00|1198.50|1270.41|-1962.72| +2451926|71783|8325|50009|350406|1203|8400|4|272|2133|78|70.11|119.88|31.16|0.00|2430.48|5468.58|9350.64|0.00|0.00|2430.48|2430.48|-3038.10| +2451926|71783|10437|50009|350406|1203|8400|4|92|2133|80|49.82|79.21|76.04|0.00|6083.20|3985.60|6336.80|0.00|0.00|6083.20|6083.20|2097.60| +2452221||14889|||4776|47434|||2134||6.92||7.61||456.60|415.20||13.69||456.60||| +||7165||1555671||||166|2134|14|||4.07|0.00|56.98|1218.14|||0.00|||| +2452221|66627|17811|11813|1555671|4776|47434|10|169|2134|80|11.40|13.68|2.05|0.00|164.00|912.00|1094.40|3.28|0.00|164.00|167.28|-748.00| +2452221|66627|13019|11813|1555671|4776|47434|10|185|2134|24|16.98|17.99|2.69|0.00|64.56|407.52|431.76|1.93|0.00|64.56|66.49|-342.96| +2452221|66627|15503|11813|1555671|4776|47434|10|125|2134|96|69.46|129.19|129.19|0.00|12402.24|6668.16|12402.24|0.00|0.00|12402.24|12402.24|5734.08| +2452221|66627|2043|11813|1555671|4776|47434|10|54|2134|52|16.55|22.50|3.15|0.00|163.80|860.60|1170.00|13.10|0.00|163.80|176.90|-696.80| +2452221|66627|1373|11813|1555671|4776|47434|10|194|2134|95|30.41|55.95|54.27|0.00|5155.65|2888.95|5315.25|206.22|0.00|5155.65|5361.87|2266.70| +2452221|66627|13987|11813|1555671|4776|47434|10|31|2134|14|1.23|1.73|0.32|0.00|4.48|17.22|24.22|0.40|0.00|4.48|4.88|-12.74| +2452221|66627|16831|11813|1555671|4776|47434|10|134|2134|8|77.87|103.56|47.63|0.00|381.04|622.96|828.48|19.05|0.00|381.04|400.09|-241.92| +2452221|66627|8637|11813|1555671|4776|47434|10|147|2134|10|57.44|82.71|63.68|0.00|636.80|574.40|827.10|50.94|0.00|636.80|687.74|62.40| +|66627|1981|||||||2134||||18.45|0.00|||||0.00|||| +2452221|66627|14569|11813|1555671|4776|47434|10|255|2134|26|14.88|18.15|8.16|0.00|212.16|386.88|471.90|14.85|0.00|212.16|227.01|-174.72| +2452221|66627|4971|11813|1555671|4776|47434|10|266|2134|80|70.11|79.22|38.81|0.00|3104.80|5608.80|6337.60|62.09|0.00|3104.80|3166.89|-2504.00| +2452221|66627|2513|11813|1555671|4776|47434|10|286|2134|77|2.86|3.26|2.77|0.00|213.29|220.22|251.02|4.26|0.00|213.29|217.55|-6.93| +2451875|65239|221|23592|784250|2508|41166|7|170|2135|71|4.76|8.37|3.01|0.00|213.71|337.96|594.27|10.68|0.00|213.71|224.39|-124.25| +2451875|65239|13955|23592|784250|2508|41166|7|74|2135|23|1.32|1.51|1.41|0.00|32.43|30.36|34.73|2.27|0.00|32.43|34.70|2.07| +2451875|65239|5366|23592|784250|2508|41166|7|17|2135|44|35.83|59.11|52.60|0.00|2314.40|1576.52|2600.84|162.00|0.00|2314.40|2476.40|737.88| +2451875|65239|15677|23592|784250|2508|41166|7|182|2135|5|93.96|97.71|79.14|35.61|395.70|469.80|488.55|25.20|35.61|360.09|385.29|-109.71| +2451875|65239|11762|23592|784250|2508|41166|7|241|2135|56|16.88|33.42|3.67|0.00|205.52|945.28|1871.52|12.33|0.00|205.52|217.85|-739.76| +2451875|65239|2015|23592|784250|2508|41166|7|266|2135|53|42.29|77.39|62.68|0.00|3322.04|2241.37|4101.67|298.98|0.00|3322.04|3621.02|1080.67| +2451875|65239|8090|23592|784250|2508|41166|7|152|2135|31|1.57|1.88|0.60|0.00|18.60|48.67|58.28|0.93|0.00|18.60|19.53|-30.07| +2451875|65239|13189|23592|784250|2508|41166|7|298|2135|24|99.06|148.59|69.83|0.00|1675.92|2377.44|3566.16|100.55|0.00|1675.92|1776.47|-701.52| +2451875|65239|14330|23592|784250|2508|41166|7|89|2135|22|13.52|17.03|10.72|0.00|235.84|297.44|374.66|21.22|0.00|235.84|257.06|-61.60| +2451473|48275|400|94638|133342|2893|8797|2|205|2136|68|15.36|25.34|7.34|0.00|499.12|1044.48|1723.12|44.92|0.00|499.12|544.04|-545.36| +2451473|48275|11425|94638|133342|2893|8797|2|111|2136|96|41.06|73.49|72.75|0.00|6984.00|3941.76|7055.04|0.00|0.00|6984.00|6984.00|3042.24| +2451473|48275|13532|94638|133342|2893|8797|2|235|2136|78|39.00|46.41|28.31|0.00|2208.18|3042.00|3619.98|154.57|0.00|2208.18|2362.75|-833.82| +2451473|48275|16528|94638|133342|2893|8797|2|127|2136|28|98.17|149.21|11.93|0.00|334.04|2748.76|4177.88|16.70|0.00|334.04|350.74|-2414.72| +2451473|48275|16222|94638|133342|2893|8797|2|37|2136|66|95.37|125.88|114.55|6426.25|7560.30|6294.42|8308.08|0.00|6426.25|1134.05|1134.05|-5160.37| +2451473|48275|13327|94638|133342|2893|8797|2|41|2136|79|55.55|93.87|81.66|0.00|6451.14|4388.45|7415.73|516.09|0.00|6451.14|6967.23|2062.69| +2451473|48275|13591|94638|133342|2893|8797|2|210|2136|37|9.28|15.31|5.05|0.00|186.85|343.36|566.47|13.07|0.00|186.85|199.92|-156.51| +|48275|256|94638|133342|2893||||2136|90||36.40|21.84||1965.60|||26.53||294.84|321.37|-1535.76| +2451473|48275|1502|94638|133342|2893|8797|2|8|2136|30|57.86|114.56|30.93|0.00|927.90|1735.80|3436.80|55.67|0.00|927.90|983.57|-807.90| +2451473|48275|2869|94638|133342|2893|8797|2|4|2136|21|30.76|31.99|30.39|0.00|638.19|645.96|671.79|38.29|0.00|638.19|676.48|-7.77| +2451523|65353|3220|77599|1212331|5463|14148|7|122|2137|73|86.47|106.35|18.07|1121.24|1319.11|6312.31|7763.55|5.93|1121.24|197.87|203.80|-6114.44| +2451523|65353|17504|77599|1212331|5463|14148|7|1|2137|11|64.51|121.27|16.97|0.00|186.67|709.61|1333.97|13.06|0.00|186.67|199.73|-522.94| +2451523|65353|12292|77599|1212331|5463|14148|7|143|2137|64|39.28|65.99|11.87|0.00|759.68|2513.92|4223.36|37.98|0.00|759.68|797.66|-1754.24| +2451523|65353|13705|77599|1212331|5463|14148|7|256|2137|9|16.49|21.76|17.84|0.00|160.56|148.41|195.84|9.63|0.00|160.56|170.19|12.15| +2451523|65353|10990|77599|1212331|5463|14148|7|11|2137|30|73.91|99.77|62.85|0.00|1885.50|2217.30|2993.10|113.13|0.00|1885.50|1998.63|-331.80| +2451523|65353|11149|77599|1212331|5463|14148|7|20|2137|11|44.41|53.29|15.98|49.21|175.78|488.51|586.19|11.39|49.21|126.57|137.96|-361.94| +||454||1212331|||7||2137|||128.85|19.32|0.00||477.24||3.47|0.00|115.92||| +2451523|65353|6031|77599|||14148|||2137|88|68.20||0.00|0.00|||10982.40||0.00|0.00||| +2451523|65353|9926|77599|1212331|5463|14148|7|296|2137|38|10.93|16.94|8.63|0.00|327.94|415.34|643.72|6.55|0.00|327.94|334.49|-87.40| +2451523|65353|16345|77599|1212331|5463|14148|7|93|2137|16|56.68|102.02|22.44|0.00|359.04|906.88|1632.32|25.13|0.00|359.04|384.17|-547.84| +2451413|41589|7768|41278|1652059|5240|44468|1|92|2138|24|35.09|36.14|35.05|0.00|841.20|842.16|867.36|67.29|0.00|841.20|908.49|-0.96| +2451413|41589|14461|41278|1652059|5240|44468|1|95|2138|62|69.01|107.65|21.53|0.00|1334.86|4278.62|6674.30|26.69|0.00|1334.86|1361.55|-2943.76| +2451413|41589|1574|41278|1652059|5240|44468|1|31|2138|63|54.74|66.78|56.76|0.00|3575.88|3448.62|4207.14|250.31|0.00|3575.88|3826.19|127.26| +2451413|41589|15536|41278|1652059|5240|44468|1|124|2138|7|4.75|7.50|6.75|0.00|47.25|33.25|52.50|2.36|0.00|47.25|49.61|14.00| +2451413|41589|9169|41278|1652059|5240|44468|1|231|2138|99|28.97|43.16|17.26|0.00|1708.74|2868.03|4272.84|85.43|0.00|1708.74|1794.17|-1159.29| +2451413|41589|1732|41278|1652059|5240|44468|1|239|2138|54|70.23|117.28|28.14|0.00|1519.56|3792.42|6333.12|15.19|0.00|1519.56|1534.75|-2272.86| +2451413|41589|1670|41278|1652059|5240|44468|1|132|2138|86|77.24|89.59|78.83|5626.88|6779.38|6642.64|7704.74|92.20|5626.88|1152.50|1244.70|-5490.14| +2451413||5227|41278|1652059|5240||1|231|2138|86||||0.00|3351.42||12415.82||0.00|3351.42||-3149.32| +2451413|41589|2516|41278|1652059|5240|44468|1|4|2138|29|43.34|45.94|28.94|0.00|839.26|1256.86|1332.26|8.39|0.00|839.26|847.65|-417.60| +2451413|41589|11480|41278|1652059|5240|44468|1|88|2138|13|13.93|22.70|21.56|280.28|280.28|181.09|295.10|0.00|280.28|0.00|0.00|-181.09| +2451413|41589|12892|41278|1652059|5240|44468|1|297|2138|31|17.83|18.36|5.50|0.00|170.50|552.73|569.16|3.41|0.00|170.50|173.91|-382.23| +2451413|41589|2533|41278|1652059|5240|44468|1|91|2138|71|44.80|48.38|4.35|0.00|308.85|3180.80|3434.98|15.44|0.00|308.85|324.29|-2871.95| +2451413|41589|15865|41278|1652059|5240|44468|1|97|2138|33|67.74|86.70|61.55|0.00|2031.15|2235.42|2861.10|20.31|0.00|2031.15|2051.46|-204.27| +2451413|41589|5338|41278|1652059|5240|44468|1|40|2138|68|63.08|82.63|8.26|0.00|561.68|4289.44|5618.84|11.23|0.00|561.68|572.91|-3727.76| +2451413|41589|12358|41278|1652059|5240|44468|1|91|2138|68|51.50|79.82|4.78|0.00|325.04|3502.00|5427.76|6.50|0.00|325.04|331.54|-3176.96| +2451158|39964|4447|72009|774655|148|35594|7|243|2139|54|15.56|25.20|5.29|0.00|285.66|840.24|1360.80|25.70|0.00|285.66|311.36|-554.58| +2451158|39964|14174|72009|774655|148|35594|7|22|2139|100|89.49|119.91|20.38|0.00|2038.00|8949.00|11991.00|40.76|0.00|2038.00|2078.76|-6911.00| +2451158|39964|8503|72009|774655|148|35594|7|233|2139|66|78.65|147.86|90.19|3035.79|5952.54|5190.90|9758.76|204.17|3035.79|2916.75|3120.92|-2274.15| +2451158|39964|2227|72009|774655|148|35594|7|249|2139|25|24.28|24.76|14.60|0.00|365.00|607.00|619.00|3.65|0.00|365.00|368.65|-242.00| +2451158|39964|1351|72009|774655|148|35594|7|91|2139|95|80.25|117.96|40.10|0.00|3809.50|7623.75|11206.20|304.76|0.00|3809.50|4114.26|-3814.25| +2451158|39964|4255|72009|774655|148|35594|7|192|2139|15|31.32|31.63|26.25|0.00|393.75|469.80|474.45|3.93|0.00|393.75|397.68|-76.05| +2451158|39964|2726|72009|774655|148|35594|7|36|2139|93|47.96|62.82|44.60|0.00|4147.80|4460.28|5842.26|165.91|0.00|4147.80|4313.71|-312.48| +2451158|39964|12806|72009|774655|148|35594|7|237|2139|8|54.55|69.82|27.92|0.00|223.36|436.40|558.56|2.23|0.00|223.36|225.59|-213.04| +2451158|39964|17690|72009|774655|148|35594|7|38|2139|63|88.38|118.42|80.52|0.00|5072.76|5567.94|7460.46|202.91|0.00|5072.76|5275.67|-495.18| +2451158|39964|2612|72009|774655|148|35594|7|200|2139|21|28.43|52.59|29.97|0.00|629.37|597.03|1104.39|44.05|0.00|629.37|673.42|32.34| +2451158|39964|11167|72009|774655|148|35594|7|271|2139|42|53.43|102.05|73.47|0.00|3085.74|2244.06|4286.10|154.28|0.00|3085.74|3240.02|841.68| +2451158|39964|17086|72009|774655|148|35594|7|15|2139|97|21.55|28.01|19.88|0.00|1928.36|2090.35|2716.97|134.98|0.00|1928.36|2063.34|-161.99| +2451158|39964|2917|72009|774655|148|35594|7|45|2139|84|30.70|47.58|34.25|834.33|2877.00|2578.80|3996.72|0.00|834.33|2042.67|2042.67|-536.13| +2451158|39964|15122|72009|774655|148|35594|7|124|2139|85|15.40|15.86|5.70|0.00|484.50|1309.00|1348.10|14.53|0.00|484.50|499.03|-824.50| +2451158|39964|760|72009|774655|148|35594|7|259|2139|5|66.70|83.37|75.03|112.54|375.15|333.50|416.85|13.13|112.54|262.61|275.74|-70.89| +|74000|5833|97068|82474|5927|6533|||2140||3.83|7.62|4.03|0.00|||594.36|25.14|0.00|314.34|339.48|15.60| +|74000|17188|97068|||6533||7|2140||43.48|66.52|||3100.45||3924.68|66.97||||-891.07| +2451147|74000|11383|97068|82474|5927|6533|2|248|2140|1|66.86|125.02|123.76|0.00|123.76|66.86|125.02|3.71|0.00|123.76|127.47|56.90| +2451147|74000|9877|97068|82474|5927|6533|2|33|2140|41|49.36|62.68|38.86|0.00|1593.26|2023.76|2569.88|127.46|0.00|1593.26|1720.72|-430.50| +2451147|74000|5210|97068|82474|5927|6533|2|1|2140|89|8.38|11.81|1.65|0.00|146.85|745.82|1051.09|5.87|0.00|146.85|152.72|-598.97| +2451147|74000|14059|97068|82474|5927|6533|2|119|2140|61|66.26|96.07|95.10|0.00|5801.10|4041.86|5860.27|116.02|0.00|5801.10|5917.12|1759.24| +2451147|74000|10510|97068|82474|5927|6533|2|95|2140|26|77.52|80.62|77.39|0.00|2012.14|2015.52|2096.12|60.36|0.00|2012.14|2072.50|-3.38| +2451147|74000|7498|97068|82474|5927|6533|2|235|2140|14|36.50|42.34|4.23|0.00|59.22|511.00|592.76|5.32|0.00|59.22|64.54|-451.78| +2451773|37025|1550|21665|103322|2132|10989|7|59|2141|25|90.54|98.68|12.82|0.00|320.50|2263.50|2467.00|19.23|0.00|320.50|339.73|-1943.00| +2451773|37025|12773|21665|103322|2132|10989|7|169|2141|44|91.18|115.79|37.05|0.00|1630.20|4011.92|5094.76|114.11|0.00|1630.20|1744.31|-2381.72| +2451773|37025|5467|21665|103322|2132|10989|7|249|2141|33|12.91|24.65|16.76|0.00|553.08|426.03|813.45|38.71|0.00|553.08|591.79|127.05| +2451773|37025|8816|21665|103322|2132|10989|7|239|2141|8|34.19|37.95|35.29|0.00|282.32|273.52|303.60|11.29|0.00|282.32|293.61|8.80| +2451773|37025|7028|21665|103322|2132|10989|7|111|2141|58|39.56|52.21|28.71|0.00|1665.18|2294.48|3028.18|16.65|0.00|1665.18|1681.83|-629.30| +2451773|37025|2155|21665|103322|2132|10989|7|24|2141|49|99.63|136.49|95.54|0.00|4681.46|4881.87|6688.01|374.51|0.00|4681.46|5055.97|-200.41| +2451773|37025|15389|21665|103322|2132|10989|7|193|2141|85|89.48|104.69|41.87|1459.16|3558.95|7605.80|8898.65|41.99|1459.16|2099.79|2141.78|-5506.01| +2451773|37025|6265|21665|103322|2132|10989|7|10|2141|80|48.43|74.58|59.66|0.00|4772.80|3874.40|5966.40|334.09|0.00|4772.80|5106.89|898.40| +2451773|37025|13073|21665|103322|2132|10989|7|83|2141|63|17.43|26.31|19.46|0.00|1225.98|1098.09|1657.53|49.03|0.00|1225.98|1275.01|127.89| +2451773|37025|4466|21665|103322|2132|10989|7|185|2141|73|18.76|23.07|2.30|0.00|167.90|1369.48|1684.11|11.75|0.00|167.90|179.65|-1201.58| +2451773|37025|7679|21665|103322|2132|10989|7|89|2141|2|78.97|151.62|40.93|0.00|81.86|157.94|303.24|2.45|0.00|81.86|84.31|-76.08| +|37025|14279||||10989||201|2141||57.35|84.30||998.75|||7839.90|534.71|998.75||7218.69|1350.43| +2451773|37025|3470|21665|103322|2132|10989|7|110|2141|84|43.25|77.41|10.06|0.00|845.04|3633.00|6502.44|67.60|0.00|845.04|912.64|-2787.96| +2451773|37025|3392|21665|103322|2132|10989|7|48|2141|22|73.85|132.93|67.79|760.60|1491.38|1624.70|2924.46|65.77|760.60|730.78|796.55|-893.92| +2451773|37025|9056|21665|103322|2132|10989|7|9|2141|83|9.58|14.08|12.53|894.39|1039.99|795.14|1168.64|8.73|894.39|145.60|154.33|-649.54| +2452238|58360|17559|53514|475320|5126|1732|1|183|2142|18|11.20|13.44|8.06|0.00|145.08|201.60|241.92|8.70|0.00|145.08|153.78|-56.52| +2452238|58360|13629|53514|475320|5126|1732|1|22|2142|62|73.91|82.04|31.17|0.00|1932.54|4582.42|5086.48|57.97|0.00|1932.54|1990.51|-2649.88| +2452238|58360|13519|53514|475320|5126|1732|1|60|2142|27|10.15|13.49|6.34|47.93|171.18|274.05|364.23|7.39|47.93|123.25|130.64|-150.80| +2452238|58360|13983|53514|475320|5126|1732|1|91|2142|15|99.65|181.36|3.62|28.77|54.30|1494.75|2720.40|1.27|28.77|25.53|26.80|-1469.22| +2452238|58360|1831|53514|475320|5126|1732|1|299|2142|77|62.32|124.01|80.60|0.00|6206.20|4798.64|9548.77|0.00|0.00|6206.20|6206.20|1407.56| +2452238|58360|10961|53514|475320|5126|1732|1|112|2142|39|44.50|57.85|9.25|0.00|360.75|1735.50|2256.15|25.25|0.00|360.75|386.00|-1374.75| +2452238|58360|1853|53514|475320|5126|1732|1|293|2142|99|71.37|104.91|97.56|0.00|9658.44|7065.63|10386.09|289.75|0.00|9658.44|9948.19|2592.81| +2452238|58360|13673|53514|475320|5126|1732|1|216|2142|33|86.04|97.22|41.80|0.00|1379.40|2839.32|3208.26|0.00|0.00|1379.40|1379.40|-1459.92| +2452613|60852|10812|58|1514212|5966|38120|1|146|2143|82|84.70|167.70|51.98|980.34|4262.36|6945.40|13751.40|0.00|980.34|3282.02|3282.02|-3663.38| +2452613|60852|3897|58|1514212|5966|38120|1|290|2143|36|25.64|44.61|7.58|0.00|272.88|923.04|1605.96|5.45|0.00|272.88|278.33|-650.16| +2452613|60852|14089|58|1514212|5966|38120|1|219|2143|95|74.27|103.97|23.91|0.00|2271.45|7055.65|9877.15|45.42|0.00|2271.45|2316.87|-4784.20| +2452613|60852|14467|58|1514212|5966|38120|1|193|2143|66|38.54|50.48|5.55|0.00|366.30|2543.64|3331.68|10.98|0.00|366.30|377.28|-2177.34| +2452613|60852|6205|58|1514212|5966|38120|1|72|2143|35|34.71|40.95|38.08|0.00|1332.80|1214.85|1433.25|79.96|0.00|1332.80|1412.76|117.95| +2452613|60852|15223|58|1514212|5966|38120|1|292|2143|77|19.64|19.83|4.56|0.00|351.12|1512.28|1526.91|31.60|0.00|351.12|382.72|-1161.16| +2452613|60852|16651|58|1514212|5966|38120|1|135|2143|68|22.92|36.21|0.00|0.00|0.00|1558.56|2462.28|0.00|0.00|0.00|0.00|-1558.56| +2452613|60852|6349|58|1514212|5966|38120|1|27|2143|66|9.20|15.73|13.84|420.18|913.44|607.20|1038.18|14.79|420.18|493.26|508.05|-113.94| +2452613|60852|2419|58|1514212|5966|38120|1|7|2143|34|86.30|159.65|150.07|0.00|5102.38|2934.20|5428.10|306.14|0.00|5102.38|5408.52|2168.18| +2452613|60852|11617|58|1514212|5966|38120|1|68|2143|23|15.26|23.50|19.74|0.00|454.02|350.98|540.50|22.70|0.00|454.02|476.72|103.04| +2452613|60852|11811|58|1514212|5966|38120|1|95|2143|45|12.57|18.98|15.18|0.00|683.10|565.65|854.10|34.15|0.00|683.10|717.25|117.45| +||4004|||4977|6785|1||2144|7||39.02|19.11|0.00|||||0.00|133.77||| +2451512|44291|2665|16561|836769|4977|6785|1|277|2144|48|45.91|78.96|50.53|0.00|2425.44|2203.68|3790.08|218.28|0.00|2425.44|2643.72|221.76| +2451512|44291|12878|16561|836769|4977|6785|1|109|2144|83|38.47|61.93|39.01|0.00|3237.83|3193.01|5140.19|64.75|0.00|3237.83|3302.58|44.82| +2451512|44291|7750|16561|836769|4977|6785|1|126|2144|35|54.64|96.71|35.78|0.00|1252.30|1912.40|3384.85|37.56|0.00|1252.30|1289.86|-660.10| +2451512|44291|7646|16561|836769|4977|6785|1|53|2144|55|1.25|2.20|1.87|0.00|102.85|68.75|121.00|1.02|0.00|102.85|103.87|34.10| +2451512|44291|11300|16561|836769|4977|6785|1|32|2144|95|46.87|80.61|42.72|3855.48|4058.40|4452.65|7657.95|16.23|3855.48|202.92|219.15|-4249.73| +2451512|44291|7544|16561|836769|4977|6785|1|296|2144|18|33.00|57.42|56.27|0.00|1012.86|594.00|1033.56|50.64|0.00|1012.86|1063.50|418.86| +2451512|44291|14983|16561|836769|4977|6785|1|237|2144|62|45.49|53.22|4.78|225.23|296.36|2820.38|3299.64|1.42|225.23|71.13|72.55|-2749.25| +2451512|44291|1352|16561|836769|4977|6785|1|227|2144|86|51.81|59.58|12.51|0.00|1075.86|4455.66|5123.88|43.03|0.00|1075.86|1118.89|-3379.80| +2451512|44291|9367|16561|836769|4977|6785|1|44|2144|68|40.38|61.78|8.64|517.01|587.52|2745.84|4201.04|4.93|517.01|70.51|75.44|-2675.33| +2451512|44291|2086|16561|836769|4977|6785|1|248|2144|63|8.74|15.99|9.11|0.00|573.93|550.62|1007.37|34.43|0.00|573.93|608.36|23.31| +2451512|44291|8546|16561|836769|4977|6785|1|236|2144|65|41.07|80.49|65.19|0.00|4237.35|2669.55|5231.85|84.74|0.00|4237.35|4322.09|1567.80| +2451614|72746|6071|36671|1229508|5192|40876|7|274|2145|93|29.64|50.98|39.25|0.00|3650.25|2756.52|4741.14|0.00|0.00|3650.25|3650.25|893.73| +2451614|72746|6883|36671|1229508|5192|40876|7|108|2145|28|87.82|100.99|54.53|0.00|1526.84|2458.96|2827.72|106.87|0.00|1526.84|1633.71|-932.12| +2451614|72746|380|36671|1229508|5192|40876|7|8|2145|54|64.67|94.41|34.93|0.00|1886.22|3492.18|5098.14|18.86|0.00|1886.22|1905.08|-1605.96| +2451614|72746|14324|36671|1229508|5192|40876|7|240|2145|62|61.93|101.56|43.67|0.00|2707.54|3839.66|6296.72|27.07|0.00|2707.54|2734.61|-1132.12| +2451614|72746|9661|36671|1229508|5192|40876|7|9|2145|25|80.06|108.08|75.65|0.00|1891.25|2001.50|2702.00|94.56|0.00|1891.25|1985.81|-110.25| +2451614|72746|14768|36671|1229508|5192|40876|7|216|2145|51|39.73|55.62|47.83|658.61|2439.33|2026.23|2836.62|124.65|658.61|1780.72|1905.37|-245.51| +2451614|72746|15685|36671|1229508|5192|40876|7|59|2145|74|65.11|85.29|69.08|5111.92|5111.92|4818.14|6311.46|0.00|5111.92|0.00|0.00|-4818.14| +2451614|72746|3446|36671|1229508|5192|40876|7|69|2145|3|47.12|69.26|52.63|0.00|157.89|141.36|207.78|12.63|0.00|157.89|170.52|16.53| +2451614|72746|15638|36671|1229508|5192|40876|7|293|2145|100|9.53|14.00|7.00|0.00|700.00|953.00|1400.00|56.00|0.00|700.00|756.00|-253.00| +2451614|72746|10439|36671|1229508|5192|40876|7|84|2145|95|64.52|119.36|83.55|0.00|7937.25|6129.40|11339.20|555.60|0.00|7937.25|8492.85|1807.85| +2451614|72746|5687|36671|1229508|5192|40876|7|209|2145|16|7.48|8.07|1.61|0.00|25.76|119.68|129.12|0.51|0.00|25.76|26.27|-93.92| +2451614|72746|6655|36671|1229508|5192|40876|7|8|2145|71|53.59|54.12|12.44|432.78|883.24|3804.89|3842.52|18.01|432.78|450.46|468.47|-3354.43| +2451614|72746|5419|36671|1229508|5192|40876|7|198|2145|58|7.65|12.24|5.63|241.63|326.54|443.70|709.92|5.94|241.63|84.91|90.85|-358.79| +2451614|72746|17887|36671|1229508|5192|40876|7|289|2145|87|12.91|18.71|2.80|0.00|243.60|1123.17|1627.77|7.30|0.00|243.60|250.90|-879.57| +2451614|72746|8669|36671|1229508|5192|40876|7|65|2145|72|84.29|167.73|140.89|0.00|10144.08|6068.88|12076.56|304.32|0.00|10144.08|10448.40|4075.20| +2451614|72746|12463|36671|1229508|5192|40876|7|252|2145|27|45.68|51.61|33.03|0.00|891.81|1233.36|1393.47|53.50|0.00|891.81|945.31|-341.55| +2451145|63602|13213|94059|1286375|5444|7713|10|212|2146|76|21.04|22.72|12.72|0.00|966.72|1599.04|1726.72|29.00|0.00|966.72|995.72|-632.32| +2451145|63602|15700|94059|1286375|5444|7713|10|34|2146|86|97.76|169.12|43.97|0.00|3781.42|8407.36|14544.32|151.25|0.00|3781.42|3932.67|-4625.94| +2451145|63602|10796|94059|1286375|5444|7713|10|186|2146|43|51.44|56.58|33.38|0.00|1435.34|2211.92|2432.94|114.82|0.00|1435.34|1550.16|-776.58| +2451145|63602|13204|94059|1286375|5444|7713|10|13|2146|55|50.47|65.10|5.20|0.00|286.00|2775.85|3580.50|0.00|0.00|286.00|286.00|-2489.85| +2451145|63602|15985|94059|1286375|5444|7713|10|230|2146|30|93.15|100.60|40.24|0.00|1207.20|2794.50|3018.00|24.14|0.00|1207.20|1231.34|-1587.30| +2451145|63602|14830|94059|1286375|5444|7713|10|114|2146|56|25.38|31.47|28.63|32.06|1603.28|1421.28|1762.32|47.13|32.06|1571.22|1618.35|149.94| +2451145|63602|8941|94059|1286375|5444|7713|10|32|2146|47|7.19|13.94|13.52|0.00|635.44|337.93|655.18|50.83|0.00|635.44|686.27|297.51| +2451145|63602|5065|94059|1286375|5444|7713|10|162|2146|39|81.79|130.04|29.90|139.93|1166.10|3189.81|5071.56|61.57|139.93|1026.17|1087.74|-2163.64| +2451145|63602|5107|94059|1286375|5444|7713|10|194|2146|20|1.97|3.86|1.27|0.00|25.40|39.40|77.20|0.76|0.00|25.40|26.16|-14.00| +2451145|63602|16267|94059|1286375|5444|7713|10|64|2146|18|46.31|92.62|68.53|1110.18|1233.54|833.58|1667.16|0.00|1110.18|123.36|123.36|-710.22| +2451145|63602|2378|94059|1286375|5444|7713|10|48|2146|63|80.46|143.21|17.18|0.00|1082.34|5068.98|9022.23|10.82|0.00|1082.34|1093.16|-3986.64| +2451145|63602|16873|94059|1286375|5444|7713|10|34|2146|18|75.40|126.67|63.33|0.00|1139.94|1357.20|2280.06|91.19|0.00|1139.94|1231.13|-217.26| +2451145|63602|12644|94059|1286375|5444|7713|10|201|2146|25|40.34|56.87|9.09|0.00|227.25|1008.50|1421.75|4.54|0.00|227.25|231.79|-781.25| +2451145|63602|3181|94059|1286375|5444|7713|10|84|2146|46|13.74|17.03|12.60|359.35|579.60|632.04|783.38|15.41|359.35|220.25|235.66|-411.79| +2451145|63602|3397|94059|1286375|5444|7713|10|106|2146|56|27.29|50.48|42.40|0.00|2374.40|1528.24|2826.88|0.00|0.00|2374.40|2374.40|846.16| +2451074|52412|11804|44680|531179|6478|44977|10|180|2147|24|10.12|17.20|1.72|0.00|41.28|242.88|412.80|3.30|0.00|41.28|44.58|-201.60| +2451074|52412|9673|44680|531179|6478|44977|10|17|2147|16|16.46|30.94|20.11|0.00|321.76|263.36|495.04|12.87|0.00|321.76|334.63|58.40| +2451074|52412|8401|44680|531179|6478|44977|10|245|2147|36|69.81|109.60|8.76|0.00|315.36|2513.16|3945.60|12.61|0.00|315.36|327.97|-2197.80| +2451074|52412|9709|44680|531179|6478|44977|10|33|2147|20|63.48|116.80|106.28|0.00|2125.60|1269.60|2336.00|127.53|0.00|2125.60|2253.13|856.00| +2451074|52412|2450|44680|531179|6478|44977|10|226|2147|49|96.91|123.07|35.69|0.00|1748.81|4748.59|6030.43|0.00|0.00|1748.81|1748.81|-2999.78| +2451074|52412|12598|44680|531179|6478|44977|10|27|2147|10|6.19|8.23|0.82|0.00|8.20|61.90|82.30|0.32|0.00|8.20|8.52|-53.70| +2451074|52412|1945|44680|531179|6478|44977|10|119|2147|99|53.99|62.08|24.21|2133.14|2396.79|5345.01|6145.92|5.27|2133.14|263.65|268.92|-5081.36| +2451074|52412|2695|44680|531179|6478|44977|10|105|2147|79|12.89|16.11|2.25|106.65|177.75|1018.31|1272.69|5.68|106.65|71.10|76.78|-947.21| +2451074|52412|3394|44680|531179|6478|44977|10|229|2147|61|50.82|89.44|69.76|0.00|4255.36|3100.02|5455.84|0.00|0.00|4255.36|4255.36|1155.34| +2451074|52412|2221|44680|531179|6478|44977|10|250|2147|30|78.71|88.15|78.45|0.00|2353.50|2361.30|2644.50|141.21|0.00|2353.50|2494.71|-7.80| +2451074|52412|12289|44680|531179|6478|44977|10|1|2147|7|64.59|101.40|80.10|257.92|560.70|452.13|709.80|6.05|257.92|302.78|308.83|-149.35| +2451074|52412|6430||531179||44977|10|14|2147||6.80||1.83|0.00|71.37|265.20|275.73|2.85|0.00|||| +|52412|13759||531179|6478|44977|10|19|2147||33.13||3.29||289.52|||11.58||||-2625.92| +2451074|52412|16522|44680|531179|6478|44977|10|18|2147|69|94.82|128.95|85.10|0.00|5871.90|6542.58|8897.55|58.71|0.00|5871.90|5930.61|-670.68| +2451074|52412|12070|44680|531179|6478|44977|10|139|2147|46|55.35|103.50|52.78|1383.89|2427.88|2546.10|4761.00|93.95|1383.89|1043.99|1137.94|-1502.11| +2451074|52412|12440|44680|531179|6478|44977|10|177|2147|47|86.24|93.13|88.47|0.00|4158.09|4053.28|4377.11|166.32|0.00|4158.09|4324.41|104.81| +2451890|61930|8246|63097|48380|4416|27699|7|202|2148|24|45.16|58.25|25.04|0.00|600.96|1083.84|1398.00|36.05|0.00|600.96|637.01|-482.88| +2451890|61930|3212|63097|48380|4416|27699|7|91|2148|21|71.97|86.36|85.49|0.00|1795.29|1511.37|1813.56|143.62|0.00|1795.29|1938.91|283.92| +2451890|61930|13775|63097|48380|4416|27699|7|242|2148|35|9.96|12.84|1.02|32.84|35.70|348.60|449.40|0.02|32.84|2.86|2.88|-345.74| +2451890|61930|11641|63097|48380|4416|27699|7|212|2148|12|49.72|98.94|44.52|0.00|534.24|596.64|1187.28|16.02|0.00|534.24|550.26|-62.40| +2451890|61930|1801|63097|48380|4416|27699|7|236|2148|44|85.57|161.72|59.83|0.00|2632.52|3765.08|7115.68|52.65|0.00|2632.52|2685.17|-1132.56| +2451890|61930|8261|63097|48380|4416|27699|7|249|2148|9|29.84|44.16|21.19|85.81|190.71|268.56|397.44|6.29|85.81|104.90|111.19|-163.66| +2451890||14527|63097|48380|4416||7||2148||79.92||12.77||||3833.70|||164.35|179.14|| +2451890|61930|5414|63097|48380|4416|27699|7|147|2148|12|32.72|36.64|6.96|0.00|83.52|392.64|439.68|3.34|0.00|83.52|86.86|-309.12| +2452605|49412|5670|81421|1226742|863|18984|1|78|2149|26|21.72|25.62|16.14|0.00|419.64|564.72|666.12|16.78|0.00|419.64|436.42|-145.08| +2452605|49412|9781|81421|1226742|863|18984|1|36|2149|14|76.90|118.42|15.39|0.00|215.46|1076.60|1657.88|17.23|0.00|215.46|232.69|-861.14| +2452605|49412|16269|81421|1226742|863|18984|1|167|2149|18|29.05|38.92|24.51|22.05|441.18|522.90|700.56|0.00|22.05|419.13|419.13|-103.77| +2452605|49412|13527|81421|1226742|863|18984|1|92|2149|83|9.29|13.65|9.82|0.00|815.06|771.07|1132.95|24.45|0.00|815.06|839.51|43.99| +2452605|49412|7812|81421|1226742|863|18984|1|17|2149|70|86.45|98.55|76.86|0.00|5380.20|6051.50|6898.50|53.80|0.00|5380.20|5434.00|-671.30| +2452605|49412|249|81421|1226742|863|18984|1|217|2149|61|16.19|26.55|23.89|0.00|1457.29|987.59|1619.55|131.15|0.00|1457.29|1588.44|469.70| +2452605|49412|6745|81421|1226742|863|18984|1|130|2149|80|77.80|86.35|8.63|158.79|690.40|6224.00|6908.00|5.31|158.79|531.61|536.92|-5692.39| +2452605|49412|8203|81421|1226742|863|18984|1|211|2149|58|55.76|81.40|13.02|0.00|755.16|3234.08|4721.20|52.86|0.00|755.16|808.02|-2478.92| +2452605|49412|4206|81421|1226742|863|18984|1|75|2149|33|93.12|97.77|21.50|0.00|709.50|3072.96|3226.41|63.85|0.00|709.50|773.35|-2363.46| +2452605|49412|14475|81421|1226742|863|18984|1|291|2149|53|55.39|91.39|84.07|0.00|4455.71|2935.67|4843.67|222.78|0.00|4455.71|4678.49|1520.04| +2452605|49412|9415|81421|1226742|863|18984|1|26|2149|39|19.98|38.76|14.72|0.00|574.08|779.22|1511.64|45.92|0.00|574.08|620.00|-205.14| +2452494|63382|16747|32437|1609367|1050|9180|10|61|2150|7|34.86|50.54|49.52|0.00|346.64|244.02|353.78|13.86|0.00|346.64|360.50|102.62| +2452494|63382|15603|32437|1609367|1050|9180|10|268|2150|34|44.17|62.27|2.49|82.96|84.66|1501.78|2117.18|0.13|82.96|1.70|1.83|-1500.08| +2452494|63382|8587|32437|1609367|1050|9180|10|168|2150|17|23.36|35.27|28.21|460.38|479.57|397.12|599.59|0.19|460.38|19.19|19.38|-377.93| +2452494|63382|13551|32437|1609367|1050|9180|10|19|2150|43|19.50|37.63|13.92|0.00|598.56|838.50|1618.09|35.91|0.00|598.56|634.47|-239.94| +2452494|63382|367|32437|1609367|1050|9180|10|126|2150|47|87.86|159.02|100.18|0.00|4708.46|4129.42|7473.94|376.67|0.00|4708.46|5085.13|579.04| +2452494|63382|10485|32437|1609367|1050|9180|10|217|2150|95|26.66|42.38|13.98|0.00|1328.10|2532.70|4026.10|92.96|0.00|1328.10|1421.06|-1204.60| +2452494|63382|2280|32437|1609367|1050|9180|10|190|2150|13|42.49|62.46|60.58|0.00|787.54|552.37|811.98|39.37|0.00|787.54|826.91|235.17| +2452494|63382|13014|32437|1609367|1050|9180|10|224|2150|42|71.97|90.68|13.60|0.00|571.20|3022.74|3808.56|39.98|0.00|571.20|611.18|-2451.54| +2452494|63382|14203|32437|1609367|1050|9180|10|71|2150|10|23.44|43.59|31.82|0.00|318.20|234.40|435.90|15.91|0.00|318.20|334.11|83.80| +2452494|63382|15351|32437|1609367|1050|9180|10|18|2150|69|29.94|32.03|14.41|0.00|994.29|2065.86|2210.07|9.94|0.00|994.29|1004.23|-1071.57| +2452494|63382|1261|32437|1609367|1050|9180|10|25|2150|83|8.95|16.37|0.16|0.00|13.28|742.85|1358.71|0.00|0.00|13.28|13.28|-729.57| +|63382|4515|32437||1050|||273|2150|87|22.40|41.66|38.32|0.00||1948.80|3624.42|233.36|0.00|3333.84|3567.20|| +2451854|51137|5708|65604|33685|519|43753|1|260|2151|74|2.24|3.02|0.51|0.00|37.74|165.76|223.48|0.00|0.00|37.74|37.74|-128.02| +2451854|51137|5381|65604|33685|519|43753|1|39|2151|85|89.64|134.46|126.39|4189.82|10743.15|7619.40|11429.10|458.73|4189.82|6553.33|7012.06|-1066.07| +2451854|51137|2563|65604|33685|519|43753|1|201|2151|2|67.30|93.54|86.05|0.00|172.10|134.60|187.08|3.44|0.00|172.10|175.54|37.50| +2451854|51137|4271|65604|33685|519|43753|1|99|2151|6|52.92|62.97|55.41|0.00|332.46|317.52|377.82|26.59|0.00|332.46|359.05|14.94| +2451854|51137|14857|65604|33685|519|43753|1|243|2151|54|78.95|105.79|58.18|0.00|3141.72|4263.30|5712.66|31.41|0.00|3141.72|3173.13|-1121.58| +2451854|51137|10625|65604|33685|519|43753|1|210|2151|86|2.17|2.34|1.17|85.52|100.62|186.62|201.24|0.15|85.52|15.10|15.25|-171.52| +2451854|51137|13997|65604|33685|519|43753|1|142|2151|62|90.22|129.01|21.93|0.00|1359.66|5593.64|7998.62|54.38|0.00|1359.66|1414.04|-4233.98| +2451854|51137|263|65604|33685|519|43753|1|132|2151|86|14.39|15.10|0.90|0.00|77.40|1237.54|1298.60|4.64|0.00|77.40|82.04|-1160.14| +2451854|51137|10099|65604|33685|519|43753|1|174|2151|100|26.86|28.20|20.86|0.00|2086.00|2686.00|2820.00|187.74|0.00|2086.00|2273.74|-600.00| +||7843||||43753|1||2151|||125.36||0.00|813.56|||65.08|0.00|||-272.25| +2452588|30312|2676|4116|608217|3071|15572|7|285|2152|8|96.65|111.14|7.77|0.00|62.16|773.20|889.12|2.48|0.00|62.16|64.64|-711.04| +2452588|30312|1788|4116|608217|3071|15572|7|275|2152|16|16.56|16.72|0.00|0.00|0.00|264.96|267.52|0.00|0.00|0.00|0.00|-264.96| +2452588|30312|1161|4116|608217|3071|15572|7|174|2152|55|14.97|22.45|11.67|0.00|641.85|823.35|1234.75|38.51|0.00|641.85|680.36|-181.50| +2452588|30312|4939|4116|608217|3071|15572|7|207|2152|2|4.76|5.18|3.72|0.00|7.44|9.52|10.36|0.29|0.00|7.44|7.73|-2.08| +2452588|30312|15831|4116|608217|3071|15572|7|235|2152|57|14.05|16.15|7.26|0.00|413.82|800.85|920.55|20.69|0.00|413.82|434.51|-387.03| +2452588|30312|5829|4116|608217|3071|15572|7|261|2152|7|73.48|91.11|47.37|318.32|331.59|514.36|637.77|0.92|318.32|13.27|14.19|-501.09| +2452588|30312|9823|4116|608217|3071|15572|7|138|2152|24|19.61|34.70|14.92|0.00|358.08|470.64|832.80|25.06|0.00|358.08|383.14|-112.56| +2452588|30312|2707|4116|608217|3071|15572|7|300|2152|93|81.93|112.24|105.50|0.00|9811.50|7619.49|10438.32|686.80|0.00|9811.50|10498.30|2192.01| +2452588|30312|15217|4116|608217|3071|15572|7|105|2152|56|89.91|97.10|61.17|0.00|3425.52|5034.96|5437.60|34.25|0.00|3425.52|3459.77|-1609.44| +2452588|30312|5487|4116|608217|3071|15572|7|230|2152|25|60.71|108.06|15.12|0.00|378.00|1517.75|2701.50|26.46|0.00|378.00|404.46|-1139.75| +2452588|30312|3609|4116|608217|3071|15572|7|268|2152|33|37.26|54.77|0.54|0.00|17.82|1229.58|1807.41|0.53|0.00|17.82|18.35|-1211.76| +2452588|30312|10365|4116||3071|15572|7|184|2152|||||||6998.47|||||1718.91|| +2452588|30312|585|4116|608217|3071|15572|7|217|2152|43|21.92|26.96|20.22|0.00|869.46|942.56|1159.28|78.25|0.00|869.46|947.71|-73.10| +2452588|30312|17364|4116|608217|3071|15572|7|73|2152|100|30.84|35.46|16.66|0.00|1666.00|3084.00|3546.00|0.00|0.00|1666.00|1666.00|-1418.00| +2452588|30312|8148|4116|608217|3071|15572|7|30|2152|55|25.58|46.55|7.44|0.00|409.20|1406.90|2560.25|32.73|0.00|409.20|441.93|-997.70| +2452588|30312|8274|4116|608217|3071|15572|7|207|2152|96|28.13|46.97|3.28|0.00|314.88|2700.48|4509.12|6.29|0.00|314.88|321.17|-2385.60| +2451179|58841|8155|40019|315945|318|17044|7|285|2153|82|55.13|93.16|70.80|0.00|5805.60|4520.66|7639.12|464.44|0.00|5805.60|6270.04|1284.94| +2451179|58841|2372|40019|315945|318|17044|7|119|2153|23|6.43|11.25|2.47|0.00|56.81|147.89|258.75|3.97|0.00|56.81|60.78|-91.08| +2451179|58841|9122|40019|315945|318|17044|7|95|2153|44|76.42|106.98|86.65|0.00|3812.60|3362.48|4707.12|152.50|0.00|3812.60|3965.10|450.12| +2451179|58841|7244|40019|315945|318|17044|7|42|2153|85|94.14|131.79|13.17|0.00|1119.45|8001.90|11202.15|22.38|0.00|1119.45|1141.83|-6882.45| +2451179|58841|7489|40019|315945|318|17044|7|113|2153|86|50.47|51.98|6.23|0.00|535.78|4340.42|4470.28|32.14|0.00|535.78|567.92|-3804.64| +2451179|58841|1657|40019|315945|318|17044|7|212|2153|39|32.98|43.20|4.32|0.00|168.48|1286.22|1684.80|11.79|0.00|168.48|180.27|-1117.74| +2451179|58841|3590|40019|315945|318|17044|7|51|2153|10|50.42|87.73|74.57|246.08|745.70|504.20|877.30|9.99|246.08|499.62|509.61|-4.58| +2451179|58841|10537|40019|315945|318|17044|7|218|2153|43|68.76|137.52|68.76|0.00|2956.68|2956.68|5913.36|177.40|0.00|2956.68|3134.08|0.00| +2451179|58841|152|40019|315945|318|17044|7|288|2153|6|5.95|8.38|1.92|0.00|11.52|35.70|50.28|1.03|0.00|11.52|12.55|-24.18| +2451179|58841|15272|40019|315945|318|17044|7|55|2153|62|18.56|32.29|12.27|0.00|760.74|1150.72|2001.98|53.25|0.00|760.74|813.99|-389.98| +||8218|40019|315945|318||||2153|||75.65|49.17|||2644.93||||||-530.62| +2451179|58841|5512|40019|315945|318|17044|7|49|2153|26|16.42|19.53|7.81|0.00|203.06|426.92|507.78|8.12|0.00|203.06|211.18|-223.86| +2451179|58841|10226|40019|315945|318|17044|7|224|2153|89|1.87|3.57|1.60|0.00|142.40|166.43|317.73|11.39|0.00|142.40|153.79|-24.03| +2452173|70971|12459|44140|679991|642|13693|10|235|2154|23|59.53|91.67|25.66|0.00|590.18|1369.19|2108.41|47.21|0.00|590.18|637.39|-779.01| +2452173|70971|6163|44140|679991|642|13693|10|167|2154|16|24.60|25.83|8.78|0.00|140.48|393.60|413.28|4.21|0.00|140.48|144.69|-253.12| +2452173|70971|7053|44140|679991|642|13693|10|190|2154|16|45.51|74.18|71.95|1093.64|1151.20|728.16|1186.88|1.15|1093.64|57.56|58.71|-670.60| +2452173|70971|6179|44140|679991|642|13693|10|170|2154|57|88.28|118.29|101.72|0.00|5798.04|5031.96|6742.53|347.88|0.00|5798.04|6145.92|766.08| +2452173|70971|9233|44140|679991|642|13693|10|45|2154|49|61.55|94.17|19.77|0.00|968.73|3015.95|4614.33|87.18|0.00|968.73|1055.91|-2047.22| +2452173|70971|15107|44140|679991|642|13693|10|213|2154|86|99.63|153.43|49.09|0.00|4221.74|8568.18|13194.98|337.73|0.00|4221.74|4559.47|-4346.44| +2452173|70971|3757|44140|679991|642|13693|10|123|2154|30|72.69|108.30|36.82|0.00|1104.60|2180.70|3249.00|33.13|0.00|1104.60|1137.73|-1076.10| +2452173|70971|14297|44140|679991|642|13693|10|219|2154|21|72.16|115.45|31.17|0.00|654.57|1515.36|2424.45|32.72|0.00|654.57|687.29|-860.79| +2452173|70971|8521|44140||||10||2154|73||22.75|14.33|0.00|||1660.75||0.00|||| +2452173|70971|2711|44140|679991|642|13693|10|62|2154|69|64.59|79.44|12.71|0.00|876.99|4456.71|5481.36|61.38|0.00|876.99|938.37|-3579.72| +2451224|41214|5779|72392|30646|5700|13601|1|107|2155|90|54.22|71.57|67.27|0.00|6054.30|4879.80|6441.30|302.71|0.00|6054.30|6357.01|1174.50| +2451224|41214|3286|72392|30646|5700|13601|1|49|2155|22|81.03|82.65|7.43|0.00|163.46|1782.66|1818.30|9.80|0.00|163.46|173.26|-1619.20| +2451224|41214|13009|72392|30646|5700|13601|1|51|2155|71|61.26|94.95|18.99|0.00|1348.29|4349.46|6741.45|13.48|0.00|1348.29|1361.77|-3001.17| +2451224|41214|16424|72392|30646|5700|13601|1|55|2155|33|34.61|65.75|33.53|0.00|1106.49|1142.13|2169.75|99.58|0.00|1106.49|1206.07|-35.64| +2451224|41214|9505|72392|30646|5700|13601|1|225|2155|58|53.99|75.58|55.92|0.00|3243.36|3131.42|4383.64|0.00|0.00|3243.36|3243.36|111.94| +2451224|41214|4396|72392|30646|5700|13601|1|242|2155|1|45.83|80.20|27.26|0.00|27.26|45.83|80.20|0.00|0.00|27.26|27.26|-18.57| +2451224|41214|688|72392|30646|5700|13601|1|5|2155|76|86.64|126.49|80.95|3629.79|6152.20|6584.64|9613.24|50.44|3629.79|2522.41|2572.85|-4062.23| +2451224|41214|1195|72392|30646|5700|13601|1|227|2155|66|92.52|115.65|10.40|0.00|686.40|6106.32|7632.90|54.91|0.00|686.40|741.31|-5419.92| +2451224|41214|7495|72392|30646|5700|13601|1|84|2155|30|55.75|94.21|66.88|0.00|2006.40|1672.50|2826.30|120.38|0.00|2006.40|2126.78|333.90| +2451224|41214|1238|72392|30646|5700|13601|1|255|2155|18|4.59|6.51|5.27|0.00|94.86|82.62|117.18|4.74|0.00|94.86|99.60|12.24| +2451224|41214|10084|72392|30646|5700|13601|1|39|2155|82|6.45|11.86|9.84|0.00|806.88|528.90|972.52|72.61|0.00|806.88|879.49|277.98| +2452568|62583|6918|62487|1504621|708|30042|10|259|2156|20|14.87|26.17|11.25|0.00|225.00|297.40|523.40|9.00|0.00|225.00|234.00|-72.40| +2452568|62583|11739|62487|1504621|708|30042|10|17|2156|84|90.83|153.50|44.51|0.00|3738.84|7629.72|12894.00|186.94|0.00|3738.84|3925.78|-3890.88| +2452568|62583|14226|62487|1504621|708|30042|10|198|2156|79|16.76|31.50|2.20|0.00|173.80|1324.04|2488.50|15.64|0.00|173.80|189.44|-1150.24| +2452568|62583|8115|62487|1504621|708|30042|10|67|2156|70|52.88|62.92|12.58|0.00|880.60|3701.60|4404.40|26.41|0.00|880.60|907.01|-2821.00| +2452568|62583|2527|62487|1504621|708|30042|10|159|2156|32|93.26|93.26|6.52|0.00|208.64|2984.32|2984.32|14.60|0.00|208.64|223.24|-2775.68| +2452568|62583|15841|62487|1504621|708|30042|10|289|2156|9|7.63|8.46|5.66|0.00|50.94|68.67|76.14|3.56|0.00|50.94|54.50|-17.73| +2452568|62583|11443|62487|1504621|708|30042|10|248|2156|6|21.65|33.55|12.07|0.00|72.42|129.90|201.30|4.34|0.00|72.42|76.76|-57.48| +2452568|62583|2017|62487|1504621|708|30042|10|21|2156|54|48.78|77.56|44.20|1933.30|2386.80|2634.12|4188.24|4.53|1933.30|453.50|458.03|-2180.62| +2452568|62583|13285|62487|1504621|708|30042|10|30|2156|62|43.89|48.27|20.75|0.00|1286.50|2721.18|2992.74|115.78|0.00|1286.50|1402.28|-1434.68| +2452568|62583|13123|62487|1504621|708|30042|10|63|2156|15|81.56|104.39|103.34|0.00|1550.10|1223.40|1565.85|15.50|0.00|1550.10|1565.60|326.70| +2452568|62583|13663|62487|1504621|708|30042|10|23|2156|10|15.21|26.61|2.39|0.00|23.90|152.10|266.10|1.43|0.00|23.90|25.33|-128.20| +2452568|62583|13530|62487|1504621|708|30042|10|60|2156|62|18.31|27.28|21.27|870.36|1318.74|1135.22|1691.36|22.41|870.36|448.38|470.79|-686.84| +2452620|51680|11184|42044|1731063|2035|22607|10|231|2157|91|62.83|112.46|105.71|0.00|9619.61|5717.53|10233.86|865.76|0.00|9619.61|10485.37|3902.08| +2452620|51680|7494|42044|1731063|2035|22607|10|109|2157|13|68.43|100.59|57.33|0.00|745.29|889.59|1307.67|37.26|0.00|745.29|782.55|-144.30| +2452620|51680|3939|42044|1731063|2035|22607|10|170|2157|43|5.21|5.62|4.04|0.00|173.72|224.03|241.66|6.94|0.00|173.72|180.66|-50.31| +2452620|51680|14151|42044|1731063|2035|22607|10|25|2157|14|14.09|22.54|19.83|0.00|277.62|197.26|315.56|8.32|0.00|277.62|285.94|80.36| +2452620|51680|14167|42044|1731063|2035|22607|10|33|2157|51|17.60|28.86|16.73|0.00|853.23|897.60|1471.86|68.25|0.00|853.23|921.48|-44.37| +2452620|51680|9453|42044|1731063|2035|22607|10|197|2157|9|81.54|119.04|110.70|0.00|996.30|733.86|1071.36|39.85|0.00|996.30|1036.15|262.44| +2452620|51680|3715|42044|1731063|2035|22607|10|34|2157|62|60.11|78.14|35.94|1827.18|2228.28|3726.82|4844.68|16.04|1827.18|401.10|417.14|-3325.72| +2452620|51680|2535|||||10||2157|3|3.53|5.25||0.00|15.75||15.75|0.47|0.00||16.22|| +2452148||13773||1119970||||31|2158|||69.95|||372.02||1329.05|||372.02||| +2452148|46650|2953|62301|1119970|4039|22155|2|193|2158|79|85.17|154.15|32.37|0.00|2557.23|6728.43|12177.85|51.14|0.00|2557.23|2608.37|-4171.20| +|46650|17307|62301|1119970|||||2158||||12.18|||||47.25||1181.46||| +|46650|8987|62301|1119970||22155|2|295|2158|36|49.67||||1320.12|1788.12|1859.40|||||| +2452148|46650|13501|62301|1119970|4039|22155|2|176|2158|35|26.66|50.65|12.66|62.03|443.10|933.10|1772.75|0.00|62.03|381.07|381.07|-552.03| +2452148|46650|2253|62301|1119970|4039|22155|2|41|2158|98|23.99|34.30|18.52|0.00|1814.96|2351.02|3361.40|90.74|0.00|1814.96|1905.70|-536.06| +2452148|46650|2147|62301|1119970|4039|22155|2|99|2158|96|97.08|105.81|93.11|0.00|8938.56|9319.68|10157.76|357.54|0.00|8938.56|9296.10|-381.12| +2452148|46650|89|62301|1119970|4039|22155|2|75|2158|76|30.00|47.40|18.96|0.00|1440.96|2280.00|3602.40|115.27|0.00|1440.96|1556.23|-839.04| +2452148||10525||1119970|||2||2158|||17.66|6.00|0.00||766.92|1165.56||0.00|396.00||| +2452148|46650|1253|62301|1119970|4039|22155|2|111|2158|10|81.05|84.29|23.60|73.16|236.00|810.50|842.90|6.51|73.16|162.84|169.35|-647.66| +2452148|46650|7427|62301|1119970|4039|22155|2|71|2158|8|21.02|24.59|16.72|9.36|133.76|168.16|196.72|9.95|9.36|124.40|134.35|-43.76| +2452148|46650|10823|62301|1119970|4039|22155|2|92|2158|80|42.82|52.24|9.40|0.00|752.00|3425.60|4179.20|22.56|0.00|752.00|774.56|-2673.60| +2451502|36023|500|78887|909516|6192|20468|4|5|2159|40|38.42|48.79|36.59|0.00|1463.60|1536.80|1951.60|102.45|0.00|1463.60|1566.05|-73.20| +2451502|36023|10790|78887|909516|6192|20468|4|156|2159|73|76.34|125.96|55.42|2548.76|4045.66|5572.82|9195.08|74.84|2548.76|1496.90|1571.74|-4075.92| +2451502|36023|11726|78887|909516|6192|20468|4|257|2159|22|2.90|5.71|5.48|0.00|120.56|63.80|125.62|1.20|0.00|120.56|121.76|56.76| +2451502|36023|12250|78887|909516|6192|20468|4|129|2159|94|94.52|96.41|59.77|0.00|5618.38|8884.88|9062.54|280.91|0.00|5618.38|5899.29|-3266.50| +2451502|36023|7108|78887|909516|6192|20468|4|186|2159|55|6.65|9.77|8.49|0.00|466.95|365.75|537.35|18.67|0.00|466.95|485.62|101.20| +2451502|36023|13228|78887|909516|6192|20468|4|212|2159|63|9.62|16.83|12.79|0.00|805.77|606.06|1060.29|16.11|0.00|805.77|821.88|199.71| +2451502|36023|14854|78887|909516|6192|20468|4|54|2159|11|76.77|128.97|82.54|0.00|907.94|844.47|1418.67|27.23|0.00|907.94|935.17|63.47| +2451502|36023|13609|78887|909516|6192|20468|4|282|2159|74|74.56|139.42|64.13|0.00|4745.62|5517.44|10317.08|237.28|0.00|4745.62|4982.90|-771.82| +2451857|65862|4274|81216|1681962|2261|11007|4|290|2160|66|72.59|119.04|7.14|0.00|471.24|4790.94|7856.64|42.41|0.00|471.24|513.65|-4319.70| +2451857|65862|7889|81216|1681962|2261|11007|4|145|2160|35|14.09|25.78|2.57|0.00|89.95|493.15|902.30|0.89|0.00|89.95|90.84|-403.20| +2451857|65862|16724|81216|1681962|2261|11007|4|5|2160|10|48.50|78.57|32.21|0.00|322.10|485.00|785.70|0.00|0.00|322.10|322.10|-162.90| +2451857|65862|17165|81216|1681962|2261|11007|4|69|2160|91|84.77|144.10|100.87|0.00|9179.17|7714.07|13113.10|91.79|0.00|9179.17|9270.96|1465.10| +2451857|65862|3338|81216|1681962|2261|11007|4|118|2160|71|78.31|148.78|35.70|0.00|2534.70|5560.01|10563.38|101.38|0.00|2534.70|2636.08|-3025.31| +2451857|65862|2207|81216|1681962|2261|11007|4|278|2160|72|8.60|9.37|5.80|58.46|417.60|619.20|674.64|17.95|58.46|359.14|377.09|-260.06| +2451857|65862|437|81216|1681962|2261|11007|4|142|2160|60|89.88|93.47|27.10|0.00|1626.00|5392.80|5608.20|65.04|0.00|1626.00|1691.04|-3766.80| +2451857|65862|11876|81216|1681962|2261|11007|4|226|2160|78|93.99|174.82|73.42|0.00|5726.76|7331.22|13635.96|229.07|0.00|5726.76|5955.83|-1604.46| +|65862|14270|81216|1681962||11007|4|229|2160||66.80|68.80|36.46||||1238.40|13.12||656.28|669.40|| +2451857|65862|3725|81216|1681962|2261|11007|4|23|2160|41|60.60|115.14|23.02|0.00|943.82|2484.60|4720.74|28.31|0.00|943.82|972.13|-1540.78| +2451661|40329|11330|40541|177001|5908|4768|7|14|2161|32|30.59|46.49|19.06|0.00|609.92|978.88|1487.68|30.49|0.00|609.92|640.41|-368.96| +2451661|40329|17273|40541|177001|5908|4768|7|124|2161|73|36.65|71.83|2.15|117.71|156.95|2675.45|5243.59|2.74|117.71|39.24|41.98|-2636.21| +2451661|40329|4778|40541|177001|5908|4768|7|51|2161|52|96.91|139.55|0.00|0.00|0.00|5039.32|7256.60|0.00|0.00|0.00|0.00|-5039.32| +|40329|11329|40541|177001||4768|||2161|||77.06||||603.09|693.54|||194.13|194.13|-408.96| +2451661|40329|7532|40541|177001|5908|4768|7|179|2161|38|48.65|78.81|59.10|0.00|2245.80|1848.70|2994.78|22.45|0.00|2245.80|2268.25|397.10| +2451661|40329|1676|40541|177001|5908|4768|7|270|2161|64|5.93|8.42|7.32|0.00|468.48|379.52|538.88|23.42|0.00|468.48|491.90|88.96| +2451661|40329|9812|40541|177001|5908|4768|7|21|2161|3|44.09|65.25|52.20|10.96|156.60|132.27|195.75|0.00|10.96|145.64|145.64|13.37| +2451661|40329|11840|40541|177001|5908|4768|7|134|2161|8|95.48|152.76|139.01|0.00|1112.08|763.84|1222.08|100.08|0.00|1112.08|1212.16|348.24| +2451661|40329|7661|40541|177001|5908|4768|7|295|2161|19|3.60|5.54|0.66|0.00|12.54|68.40|105.26|1.12|0.00|12.54|13.66|-55.86| +2451661|40329|12812|40541|177001|5908|4768|7|31|2161|51|63.17|75.17|63.14|0.00|3220.14|3221.67|3833.67|225.40|0.00|3220.14|3445.54|-1.53| +2451661|40329|7819|40541|177001|5908|4768|7|102|2161|46|29.51|32.75|1.96|0.00|90.16|1357.46|1506.50|3.60|0.00|90.16|93.76|-1267.30| +2451661|40329|8558|40541|177001|5908|4768|7|102|2161|36|49.83|72.75|24.73|0.00|890.28|1793.88|2619.00|8.90|0.00|890.28|899.18|-903.60| +2451773|61583|11204|36156|1120018|2727|24989|4|265|2162|30|21.10|24.26|23.53|0.00|705.90|633.00|727.80|42.35|0.00|705.90|748.25|72.90| +2451773|61583|14246|36156|1120018|2727|24989|4|41|2162|65|98.78|182.74|20.10|1293.43|1306.50|6420.70|11878.10|1.17|1293.43|13.07|14.24|-6407.63| +2451773|61583|7475|36156|1120018|2727|24989|4|233|2162|23|74.16|88.25|0.88|0.00|20.24|1705.68|2029.75|1.61|0.00|20.24|21.85|-1685.44| +2451773|61583|17060|36156|1120018|2727|24989|4|196|2162|71|50.89|65.64|20.34|0.00|1444.14|3613.19|4660.44|28.88|0.00|1444.14|1473.02|-2169.05| +2451773|61583|14078|36156|1120018|2727|24989|4|31|2162|100|22.41|26.44|17.45|0.00|1745.00|2241.00|2644.00|0.00|0.00|1745.00|1745.00|-496.00| +|61583|1793|36156||2727||4||2162||43.13||||2149.58|1983.98|3412.28||||2149.58|| +2451773|61583|16303|36156|1120018|2727|24989|4|155|2162|39|59.02|104.46|34.47|0.00|1344.33|2301.78|4073.94|53.77|0.00|1344.33|1398.10|-957.45| +2451773|61583|14713|36156|1120018|2727|24989|4|160|2162|57|87.56|131.34|87.99|0.00|5015.43|4990.92|7486.38|0.00|0.00|5015.43|5015.43|24.51| +2451773|61583|773|36156|1120018|2727|24989|4|173|2162|60|93.17|113.66|92.06|0.00|5523.60|5590.20|6819.60|441.88|0.00|5523.60|5965.48|-66.60| +2451773|61583|12395|36156|1120018|2727|24989|4|69|2162|81|61.40|85.96|69.62|0.00|5639.22|4973.40|6962.76|112.78|0.00|5639.22|5752.00|665.82| +2451773|61583|9875|36156|1120018|2727|24989|4|150|2162|99|87.82|129.97|120.87|0.00|11966.13|8694.18|12867.03|239.32|0.00|11966.13|12205.45|3271.95| +2451773|61583|3637|36156|1120018|2727|24989|4|178|2162|74|5.38|5.97|4.35|0.00|321.90|398.12|441.78|12.87|0.00|321.90|334.77|-76.22| +2451773|61583|12590|36156|1120018|2727|24989|4|240|2162|91|38.61|52.12|5.73|380.64|521.43|3513.51|4742.92|5.63|380.64|140.79|146.42|-3372.72| +2451773|61583|13295|36156|1120018|2727|24989|4|167|2162|6|22.80|37.39|17.57|0.00|105.42|136.80|224.34|1.05|0.00|105.42|106.47|-31.38| +2451357|74963|16886|42099|1256058|6817|31576|4|183|2163|89|76.88|96.10|68.23|0.00|6072.47|6842.32|8552.90|60.72|0.00|6072.47|6133.19|-769.85| +2451357|74963|1519|42099|1256058|6817|31576|4|120|2163|49|28.06|31.70|29.79|0.00|1459.71|1374.94|1553.30|102.17|0.00|1459.71|1561.88|84.77| +2451357|74963|7610|42099|1256058|6817|31576|4|244|2163|74|61.28|97.43|95.48|0.00|7065.52|4534.72|7209.82|0.00|0.00|7065.52|7065.52|2530.80| +2451357|74963|12968|42099|1256058|6817|31576|4|28|2163|97|10.30|14.93|0.59|0.00|57.23|999.10|1448.21|3.43|0.00|57.23|60.66|-941.87| +2451357|74963|4622|42099|1256058|6817|31576|4|17|2163|13|87.74|112.30|15.72|0.00|204.36|1140.62|1459.90|18.39|0.00|204.36|222.75|-936.26| +2451357|74963|17308|42099|1256058|6817|31576|4|33|2163|65|57.11|94.23|32.03|0.00|2081.95|3712.15|6124.95|20.81|0.00|2081.95|2102.76|-1630.20| +2451357|74963|13717|42099|1256058|6817|31576|4|122|2163|48|73.33|107.06|61.02|0.00|2928.96|3519.84|5138.88|146.44|0.00|2928.96|3075.40|-590.88| +2451357|74963|16363|42099|1256058|6817|31576|4|54|2163|10|57.43|83.84|47.78|0.00|477.80|574.30|838.40|28.66|0.00|477.80|506.46|-96.50| +2451357|74963|68|42099|1256058|6817|31576|4|19|2163|39|30.49|39.63|16.24|0.00|633.36|1189.11|1545.57|19.00|0.00|633.36|652.36|-555.75| +2451257|43085|15902|38812|358285|1405|11278|10|49|2164|54|88.76|163.31|39.19|1142.78|2116.26|4793.04|8818.74|58.40|1142.78|973.48|1031.88|-3819.56| +2451257|43085|17864|38812|358285|1405|11278|10|10|2164|33|25.12|35.92|24.06|0.00|793.98|828.96|1185.36|71.45|0.00|793.98|865.43|-34.98| +2451257|43085|9244|38812|358285|1405|11278|10|91|2164|46|94.58|163.62|60.53|0.00|2784.38|4350.68|7526.52|250.59|0.00|2784.38|3034.97|-1566.30| +2451257|43085|12811|38812|358285|1405|11278|10|240|2164|23|3.38|5.57|5.40|0.00|124.20|77.74|128.11|1.24|0.00|124.20|125.44|46.46| +||5128||358285|||||2164||45.77||||1958.57||2720.76||||2115.25|| +2451257|43085|11672|38812|358285|1405|11278|10|287|2164|44|44.72|88.99|53.39|0.00|2349.16|1967.68|3915.56|117.45|0.00|2349.16|2466.61|381.48| +2451257|43085|6386|38812|358285|1405|11278|10|113|2164|95|98.42|147.63|29.52|0.00|2804.40|9349.90|14024.85|224.35|0.00|2804.40|3028.75|-6545.50| +2451257|43085|1177|38812|358285|1405|11278|10|290|2164|65|69.14|76.05|17.49|0.00|1136.85|4494.10|4943.25|0.00|0.00|1136.85|1136.85|-3357.25| +2451257|43085|12841|38812|358285|1405|11278|10|143|2164|94|50.04|51.04|25.52|0.00|2398.88|4703.76|4797.76|23.98|0.00|2398.88|2422.86|-2304.88| +2451474|49688|3331|68513|905873|1633|23840|7|174|2165|99|9.50|10.64|0.21|0.00|20.79|940.50|1053.36|1.66|0.00|20.79|22.45|-919.71| +2451474|49688|7123|68513|905873|1633|23840|7|201|2165|80|38.33|54.81|7.12|96.83|569.60|3066.40|4384.80|0.00|96.83|472.77|472.77|-2593.63| +2451474|49688|5092|68513|905873|1633|23840|7|276|2165|83|18.60|35.34|1.76|0.00|146.08|1543.80|2933.22|7.30|0.00|146.08|153.38|-1397.72| +2451474|49688|13940|68513|905873|1633|23840|7|241|2165|96|96.34|134.87|101.15|0.00|9710.40|9248.64|12947.52|388.41|0.00|9710.40|10098.81|461.76| +2451474|49688|7204|68513|905873|1633|23840|7|149|2165|7|58.93|76.60|59.74|0.00|418.18|412.51|536.20|16.72|0.00|418.18|434.90|5.67| +2451474|49688|9256|68513|905873|1633|23840|7|172|2165|39|4.30|7.18|6.31|0.00|246.09|167.70|280.02|19.68|0.00|246.09|265.77|78.39| +2451474|49688|13268|68513|905873|1633|23840|7|189|2165|64|4.32|6.04|2.59|0.00|165.76|276.48|386.56|11.60|0.00|165.76|177.36|-110.72| +2451474||548|68513|905873|1633||7||2165|||161.13|3.22|0.00||325.52|644.52|0.00|0.00|12.88||| +2451873|63870|1841|8818|1542545|1072|30399|7|39|2166|94|75.58|108.07|36.74|2382.95|3453.56|7104.52|10158.58|96.35|2382.95|1070.61|1166.96|-6033.91| +2451873|63870|4178|8818|1542545|1072|30399|7|15|2166|3|25.57|34.51|9.66|0.00|28.98|76.71|103.53|0.86|0.00|28.98|29.84|-47.73| +2451873|63870|4013|8818|1542545|1072|30399|7|161|2166|65|89.38|92.95|91.09|0.00|5920.85|5809.70|6041.75|0.00|0.00|5920.85|5920.85|111.15| +2451873|63870|13057|8818|1542545|1072|30399|7|18|2166|26|32.55|32.55|25.38|0.00|659.88|846.30|846.30|13.19|0.00|659.88|673.07|-186.42| +2451873|63870|14689|8818|1542545|1072|30399|7|283|2166|92|47.81|58.80|17.05|407.83|1568.60|4398.52|5409.60|58.03|407.83|1160.77|1218.80|-3237.75| +|63870|12697||1542545|1072||||2166|84|||52.65||||5025.72||||4537.58|1393.35| +2451873|63870|6812|8818|1542545|1072|30399|7|6|2166|72|76.58|89.59|55.54|0.00|3998.88|5513.76|6450.48|39.98|0.00|3998.88|4038.86|-1514.88| +|63870|3893||1542545|||7||2166|||35.99|12.59|0.00|||863.76||0.00|302.16||-241.20| +2451873|63870|17732|8818|1542545|1072|30399|7|269|2166|54|81.45|156.38|28.14|0.00|1519.56|4398.30|8444.52|0.00|0.00|1519.56|1519.56|-2878.74| +2451873|63870|10433|8818|1542545|1072|30399|7|68|2166|98|12.50|15.00|13.80|0.00|1352.40|1225.00|1470.00|13.52|0.00|1352.40|1365.92|127.40| +2451873|63870|4172|8818|1542545|1072|30399|7|103|2166|81|60.47|119.12|55.98|1133.59|4534.38|4898.07|9648.72|238.05|1133.59|3400.79|3638.84|-1497.28| +2451873|63870|1622|8818|1542545|1072|30399|7|115|2166|85|73.75|87.76|21.06|0.00|1790.10|6268.75|7459.60|53.70|0.00|1790.10|1843.80|-4478.65| +2451873|63870|17201|8818|1542545|1072|30399|7|299|2166|98|47.61|60.94|26.81|0.00|2627.38|4665.78|5972.12|210.19|0.00|2627.38|2837.57|-2038.40| +2452515|52046|8221|4127|1391029|5484|32339|10|187|2167|6|72.75|90.21|79.38|290.53|476.28|436.50|541.26|11.14|290.53|185.75|196.89|-250.75| +2452515|52046|8541|4127|1391029|5484|32339|10|213|2167|74|72.23|72.23|26.72|0.00|1977.28|5345.02|5345.02|118.63|0.00|1977.28|2095.91|-3367.74| +|52046|4251||||32339|10||2167|63|69.67|125.40||||4389.21|7900.20||||3160.08|| +2452515|52046|16599|4127|1391029|5484|32339|10|284|2167|70|28.02|34.74|19.10|0.00|1337.00|1961.40|2431.80|120.33|0.00|1337.00|1457.33|-624.40| +2452515|52046|12891|4127|1391029|5484|32339|10|242|2167|33|92.66|136.21|72.19|0.00|2382.27|3057.78|4494.93|71.46|0.00|2382.27|2453.73|-675.51| +2452515|52046|4543|4127|1391029|5484|32339|10|4|2167|82|69.02|86.96|17.39|0.00|1425.98|5659.64|7130.72|28.51|0.00|1425.98|1454.49|-4233.66| +2452515|52046|8466|4127|1391029|5484|32339|10|255|2167|51|20.90|41.59|22.04|0.00|1124.04|1065.90|2121.09|78.68|0.00|1124.04|1202.72|58.14| +2452515|52046|13713|4127|1391029|5484|32339|10|54|2167|97|35.41|57.01|6.27|0.00|608.19|3434.77|5529.97|24.32|0.00|608.19|632.51|-2826.58| +2452515|52046|2497|4127|1391029|5484|32339|10|179|2167|70|66.89|92.30|13.84|0.00|968.80|4682.30|6461.00|19.37|0.00|968.80|988.17|-3713.50| +2452515|52046|4749|4127|1391029|5484|32339|10|20|2167|19|62.80|90.43|11.75|0.00|223.25|1193.20|1718.17|2.23|0.00|223.25|225.48|-969.95| +2452515|52046|6751|4127|1391029|5484|32339|10|293|2167|14|74.83|82.31|4.11|0.00|57.54|1047.62|1152.34|2.30|0.00|57.54|59.84|-990.08| +2452515|52046|6043|4127|1391029|5484|32339|10|96|2167|78|59.56|61.34|8.58|0.00|669.24|4645.68|4784.52|60.23|0.00|669.24|729.47|-3976.44| +2452515|52046|13303|4127|1391029|5484|32339|10|89|2167|60|8.20|9.02|3.33|0.00|199.80|492.00|541.20|13.98|0.00|199.80|213.78|-292.20| +2452515|52046|7375|4127|1391029|5484|32339|10|268|2167|64|13.18|20.95|11.31|0.00|723.84|843.52|1340.80|0.00|0.00|723.84|723.84|-119.68| +2452207|38971|9567|20782|501894|7006|||281|2168|||105.91||0.00|952.01|2312.29|||0.00|952.01|990.09|-1360.28| +2452207|38971|1443|20782|501894|7006|27318|7|164|2168|41|2.05|3.95|1.14|0.00|46.74|84.05|161.95|3.73|0.00|46.74|50.47|-37.31| +2452207|38971|6287|20782|501894|7006|27318|7|272|2168|22|38.89|59.11|29.55|0.00|650.10|855.58|1300.42|0.00|0.00|650.10|650.10|-205.48| +2452207|38971|5255|20782|501894|7006|27318|7|289|2168|66|39.85|69.33|33.27|483.08|2195.82|2630.10|4575.78|102.76|483.08|1712.74|1815.50|-917.36| +2452207|38971|13261|20782|501894|7006|27318|7|126|2168|15|40.14|42.14|24.86|167.80|372.90|602.10|632.10|4.10|167.80|205.10|209.20|-397.00| +2452207|38971|1421|20782|501894|7006|27318|7|214|2168|27|13.72|25.79|17.27|0.00|466.29|370.44|696.33|4.66|0.00|466.29|470.95|95.85| +2452207|38971|8917|20782|501894|7006|27318|7|197|2168|8|37.45|40.07|5.20|0.00|41.60|299.60|320.56|1.66|0.00|41.60|43.26|-258.00| +|38971|3623|20782|501894|7006|27318|7||2168|95|47.12|59.37|44.52|0.00|4229.40||5640.15|169.17|0.00|||-247.00| +2452207|38971|12615|20782|501894|7006|27318|7|238|2168|63|79.02|100.35|85.29|0.00|5373.27|4978.26|6322.05|322.39|0.00|5373.27|5695.66|395.01| +2452207|38971|11091|20782|501894|7006|27318|7|64|2168|80|47.20|85.90|78.16|0.00|6252.80|3776.00|6872.00|437.69|0.00|6252.80|6690.49|2476.80| +2452207|38971|14629|20782|501894|7006|27318|7|296|2168|56|71.95|100.73|98.71|829.16|5527.76|4029.20|5640.88|187.94|829.16|4698.60|4886.54|669.40| +2452207|38971|8957|20782|501894|7006|27318|7|27|2168|40|13.43|16.38|1.14|0.00|45.60|537.20|655.20|1.82|0.00|45.60|47.42|-491.60| +2452207|38971|6901|20782|501894|7006|27318|7|206|2168|17|83.67|118.81|87.91|0.00|1494.47|1422.39|2019.77|104.61|0.00|1494.47|1599.08|72.08| +2452207|38971|12119|20782|501894|7006|27318|7|55|2168|40|2.85|4.44|0.26|4.16|10.40|114.00|177.60|0.43|4.16|6.24|6.67|-107.76| +2451155|68441|13336|89425|842121|2017|6713|4|115|2169|62|32.26|40.64|1.62|0.00|100.44|2000.12|2519.68|7.03|0.00|100.44|107.47|-1899.68| +2451155|68441|14216|89425|842121|2017|6713|4|20|2169|52|40.29|72.11|62.73|0.00|3261.96|2095.08|3749.72|228.33|0.00|3261.96|3490.29|1166.88| +2451155|68441|2834|89425|842121|2017|6713|4|216|2169|27|74.78|111.42|92.47|0.00|2496.69|2019.06|3008.34|49.93|0.00|2496.69|2546.62|477.63| +2451155|68441|4117|89425|842121|2017|6713|4|23|2169|32|5.36|5.36|2.94|0.00|94.08|171.52|171.52|2.82|0.00|94.08|96.90|-77.44| +|68441|5035|89425|842121||6713|4|212|2169||||||||||||3699.40|-352.00| +||662||842121|2017||||2169|31|44.17|76.85||||1369.27||0.00||2143.96||| +2451155|68441|10804|89425|842121|2017|6713|4|299|2169|29|73.56|73.56|48.54|0.00|1407.66|2133.24|2133.24|84.45|0.00|1407.66|1492.11|-725.58| +2451155|68441|7345|89425|842121|2017|6713|4|204|2169|8|94.59|139.04|133.47|0.00|1067.76|756.72|1112.32|74.74|0.00|1067.76|1142.50|311.04| +2451155|68441|11378|||||4||2169|48|81.90||66.82|||3931.20|4009.44|||3207.36|3239.43|-723.84| +2451155|68441|11164|89425|842121|2017|6713|4|282|2169|85|87.36|134.53|123.76|0.00|10519.60|7425.60|11435.05|946.76|0.00|10519.60|11466.36|3094.00| +||13778|89425|842121||6713|4|293|2169|54||39.28|0.39|0.00|||2121.12||0.00||21.48|| +2451155|68441|9850|89425|842121|2017|6713|4|153|2169|82|99.07|169.40|99.94|0.00|8195.08|8123.74|13890.80|163.90|0.00|8195.08|8358.98|71.34| +2451155|68441|13666|89425|842121|2017|6713|4|141|2169|37|32.84|53.85|37.15|701.02|1374.55|1215.08|1992.45|13.47|701.02|673.53|687.00|-541.55| +2451155|68441|15526|89425|842121|2017|6713|4|242|2169|48|19.35|27.67|3.87|0.00|185.76|928.80|1328.16|3.71|0.00|185.76|189.47|-743.04| +2451155|68441|9292|89425|842121|2017|6713|4|79|2169|67|19.75|24.49|20.08|0.00|1345.36|1323.25|1640.83|40.36|0.00|1345.36|1385.72|22.11| +2451814|36023|4538|77445|114261|6724|41587|4|170|2170|77|26.15|33.47|7.02|0.00|540.54|2013.55|2577.19|48.64|0.00|540.54|589.18|-1473.01| +2451814|36023|5375|77445|114261|6724|41587|4|110|2170|65|78.87|84.39|13.50|0.00|877.50|5126.55|5485.35|70.20|0.00|877.50|947.70|-4249.05| +2451814|36023|10883|77445|114261|6724|41587|4|11|2170|13|70.26|100.47|42.19|0.00|548.47|913.38|1306.11|38.39|0.00|548.47|586.86|-364.91| +2451814|36023|395|77445|114261|6724|41587|4|264|2170|65|56.28|60.21|52.98|2686.08|3443.70|3658.20|3913.65|45.45|2686.08|757.62|803.07|-2900.58| +2451814|36023|16067|77445|114261|6724|41587|4|169|2170|16|97.54|102.41|89.09|1040.57|1425.44|1560.64|1638.56|11.54|1040.57|384.87|396.41|-1175.77| +2451814|36023|2837|77445|114261|6724|41587|4|1|2170|93|93.43|175.64|89.57|0.00|8330.01|8688.99|16334.52|0.00|0.00|8330.01|8330.01|-358.98| +2451814|36023|15572|77445|114261|6724|41587|4|297|2170|66|9.24|9.42|4.61|0.00|304.26|609.84|621.72|24.34|0.00|304.26|328.60|-305.58| +2451814|36023|11084|77445|114261|6724|41587|4|24|2170|69|66.61|117.89|75.44|0.00|5205.36|4596.09|8134.41|208.21|0.00|5205.36|5413.57|609.27| +2451814|36023|14525|77445|114261|6724|41587|4|12|2170|78|73.13|145.52|136.78|0.00|10668.84|5704.14|11350.56|213.37|0.00|10668.84|10882.21|4964.70| +2451814|36023|4109|77445|114261|6724|41587|4|152|2170|92|75.19|120.30|42.10|0.00|3873.20|6917.48|11067.60|77.46|0.00|3873.20|3950.66|-3044.28| +2451814|36023|12377|77445|114261|6724|41587|4|190|2170|95|19.21|21.13|19.22|0.00|1825.90|1824.95|2007.35|127.81|0.00|1825.90|1953.71|0.95| +2451814|36023|13235|77445|114261|6724|41587|4|65|2170|34|28.22|44.58|3.12|63.64|106.08|959.48|1515.72|2.12|63.64|42.44|44.56|-917.04| +2451814|36023|8573|77445|114261|6724|41587|4|42|2170|94|87.93|101.11|63.69|2334.87|5986.86|8265.42|9504.34|328.67|2334.87|3651.99|3980.66|-4613.43| +2451814|36023|7621|77445|114261|6724|41587|4|259|2170|74|33.43|46.46|19.51|0.00|1443.74|2473.82|3438.04|115.49|0.00|1443.74|1559.23|-1030.08| +2451814|36023|10999|77445|114261|6724|41587|4|231|2170|89|68.91|88.20|0.88|23.49|78.32|6132.99|7849.80|1.09|23.49|54.83|55.92|-6078.16| +2452264|35308|1715|61675|475219|6810|32354|7|190|2171|70|42.01|81.49|9.77|629.18|683.90|2940.70|5704.30|1.64|629.18|54.72|56.36|-2885.98| +2452264|35308|35|61675|475219|6810|32354|7|248|2171|87|56.89|104.10|5.20|0.00|452.40|4949.43|9056.70|22.62|0.00|452.40|475.02|-4497.03| +2452264|35308|4345|61675|475219|6810|32354|7|248|2171|66|22.38|41.40|32.70|0.00|2158.20|1477.08|2732.40|151.07|0.00|2158.20|2309.27|681.12| +2452264|35308|4067|61675|475219|6810|32354|7|289|2171|39|42.08|71.53|49.35|0.00|1924.65|1641.12|2789.67|96.23|0.00|1924.65|2020.88|283.53| +2452264|35308|4563|61675|475219|6810|32354|7|280|2171|67|72.52|120.38|0.00|0.00|0.00|4858.84|8065.46|0.00|0.00|0.00|0.00|-4858.84| +2452264|35308|10835|61675|475219|6810|32354|7|189|2171|99|25.09|38.13|34.69|858.57|3434.31|2483.91|3774.87|231.81|858.57|2575.74|2807.55|91.83| +2452264|35308|17747|61675|475219|6810|32354|7|233|2171|7|81.08|132.97|77.12|5.39|539.84|567.56|930.79|48.10|5.39|534.45|582.55|-33.11| +2452264|35308|11629|61675|475219|6810|32354|7|85|2171|14|53.22|71.84|52.44|0.00|734.16|745.08|1005.76|66.07|0.00|734.16|800.23|-10.92| +2452264|35308|11079|61675|475219|6810|32354|7|7|2171|52|77.35|93.59|61.76|0.00|3211.52|4022.20|4866.68|32.11|0.00|3211.52|3243.63|-810.68| +2452264|35308|2843|61675|475219|6810|32354|7|249|2171|58|77.43|143.24|107.43|2305.44|6230.94|4490.94|8307.92|39.25|2305.44|3925.50|3964.75|-565.44| +2452262|74352|10431|66480|1225486|6539|46792|8|124|2172|38|9.32|13.79|0.68|0.00|25.84|354.16|524.02|0.00|0.00|25.84|25.84|-328.32| +2452262|74352|12281|66480|1225486|6539|46792|8|46|2172|65|24.40|39.28|27.49|0.00|1786.85|1586.00|2553.20|142.94|0.00|1786.85|1929.79|200.85| +2452262|74352|7929|66480|1225486|6539|46792|8|212|2172|20|18.51|24.43|23.69|0.00|473.80|370.20|488.60|23.69|0.00|473.80|497.49|103.60| +2452262|74352|10577|66480|1225486|6539|46792|8|30|2172|89|28.74|53.16|0.53|0.00|47.17|2557.86|4731.24|4.24|0.00|47.17|51.41|-2510.69| +2452262|74352|13253|66480|1225486|6539|46792|8|208|2172|92|67.53|84.41|71.74|0.00|6600.08|6212.76|7765.72|528.00|0.00|6600.08|7128.08|387.32| +2452262|74352|16289|66480|1225486|6539|46792|8|8|2172|85|78.71|87.36|20.09|0.00|1707.65|6690.35|7425.60|17.07|0.00|1707.65|1724.72|-4982.70| +2452262|74352|16845|66480|1225486|6539|46792|8|122|2172|63|49.54|74.80|45.62|0.00|2874.06|3121.02|4712.40|258.66|0.00|2874.06|3132.72|-246.96| +2452262|74352|16757|66480|1225486|6539|46792|8|112|2172|39|30.51|35.39|5.30|0.00|206.70|1189.89|1380.21|4.13|0.00|206.70|210.83|-983.19| +2452262|74352|17627|66480|1225486|6539|46792|8|9|2172|5|84.26|137.34|96.13|240.32|480.65|421.30|686.70|4.80|240.32|240.33|245.13|-180.97| +2452262|74352|2635|66480|1225486|6539|46792|8|53|2172|11|63.96|115.12|1.15|0.00|12.65|703.56|1266.32|0.37|0.00|12.65|13.02|-690.91| +2452262|74352|2427|66480|1225486|6539|46792|8|48|2172|85|36.18|60.05|51.04|0.00|4338.40|3075.30|5104.25|390.45|0.00|4338.40|4728.85|1263.10| +2452262|74352|15147|66480|1225486|6539|46792|8|253|2172|94|79.05|80.63|78.21|0.00|7351.74|7430.70|7579.22|294.06|0.00|7351.74|7645.80|-78.96| +2452262|74352|12321|66480|1225486|6539|46792|8|9|2172|38|58.64|102.62|54.38|0.00|2066.44|2228.32|3899.56|61.99|0.00|2066.44|2128.43|-161.88| +2452262|74352|7831|66480|1225486|6539|46792|8|263|2172|83|66.34|128.03|76.81|1083.78|6375.23|5506.22|10626.49|158.74|1083.78|5291.45|5450.19|-214.77| +|37486|1972|85097|1546238||||244|2173|44||106.96|41.71|0.00|1835.24|2801.48|4706.24||0.00||1963.70|| +2451222|37486|10939|85097|1546238|3826|5367|7|212|2173|98|16.33|30.37|12.45|0.00|1220.10|1600.34|2976.26|109.80|0.00|1220.10|1329.90|-380.24| +2451222|37486|817|85097||3826|5367||21|2173|76|30.22|37.77|27.94|1465.17|||||1465.17|658.27||-1638.45| +2451222|37486|14444|85097|1546238|3826|5367|7|120|2173|23|57.82|68.22|24.55|400.90|564.65|1329.86|1569.06|8.18|400.90|163.75|171.93|-1166.11| +2451222|37486|7468|85097|1546238|3826|5367|7|73|2173|19|45.54|56.46|40.08|0.00|761.52|865.26|1072.74|38.07|0.00|761.52|799.59|-103.74| +2451222|37486|374|85097|1546238|3826|5367|7|71|2173|39|21.43|22.93|5.96|0.00|232.44|835.77|894.27|9.29|0.00|232.44|241.73|-603.33| +2451222|37486|14557|85097|1546238|3826|5367|7|50|2173|8|62.06|104.88|8.39|0.00|67.12|496.48|839.04|2.01|0.00|67.12|69.13|-429.36| +||3320|85097||||7||2173|||130.24|52.09|||||118.76||||| +2451222|37486|10384|85097|1546238|3826|5367|7|61|2173|9|16.86|32.37|22.01|184.22|198.09|151.74|291.33|1.24|184.22|13.87|15.11|-137.87| +2451222|37486|5500|85097|1546238|3826|5367|7|72|2173|98|49.98|81.46|46.43|0.00|4550.14|4898.04|7983.08|409.51|0.00|4550.14|4959.65|-347.90| +2451222|37486|10562|85097|1546238|3826|5367|7|39|2173|69|26.83|39.17|9.40|0.00|648.60|1851.27|2702.73|6.48|0.00|648.60|655.08|-1202.67| +2451162|63928|14446|57474|948677|1480|38283|1|108|2174|41|56.00|94.08|1.88|0.00|77.08|2296.00|3857.28|0.00|0.00|77.08|77.08|-2218.92| +2451162|63928|4480|57474|948677|1480|38283|1|87|2174|33|36.08|43.65|21.38|0.00|705.54|1190.64|1440.45|35.27|0.00|705.54|740.81|-485.10| +2451162|63928|14666|57474|948677|1480|38283|1|274|2174|79|87.31|102.15|3.06|0.00|241.74|6897.49|8069.85|16.92|0.00|241.74|258.66|-6655.75| +2451162||12421||||||214|2174|54||8.14|||413.10|248.40|439.56|37.17|||450.27|164.70| +2451162|63928|15859|57474|948677|1480|38283|1|220|2174|64|4.46|7.89|5.36|0.00|343.04|285.44|504.96|13.72|0.00|343.04|356.76|57.60| +2451162|63928|6994|57474|948677|1480|38283|1|194|2174|14|67.81|109.17|98.25|0.00|1375.50|949.34|1528.38|13.75|0.00|1375.50|1389.25|426.16| +2451162|63928|505|57474|948677|1480|38283|1|211|2174|37|93.98|180.44|68.56|0.00|2536.72|3477.26|6676.28|101.46|0.00|2536.72|2638.18|-940.54| +2451162|63928|10070|57474|948677|1480|38283|1|236|2174|60|53.31|90.09|39.63|0.00|2377.80|3198.60|5405.40|118.89|0.00|2377.80|2496.69|-820.80| +2451162|63928|12166|57474|948677|1480|38283|1|283|2174|84|30.44|34.39|17.88|0.00|1501.92|2556.96|2888.76|30.03|0.00|1501.92|1531.95|-1055.04| +2451162|63928|7816|57474|948677|1480|38283|1|203|2174|49|21.93|27.63|6.35|0.00|311.15|1074.57|1353.87|24.89|0.00|311.15|336.04|-763.42| +2451162|63928|6985|57474|948677|1480|38283|1|183|2174|8|78.52|130.34|78.20|0.00|625.60|628.16|1042.72|12.51|0.00|625.60|638.11|-2.56| +2451162|63928|12847|57474|948677|1480|38283|1|101|2174|34|94.09|145.83|106.45|2243.96|3619.30|3199.06|4958.22|68.76|2243.96|1375.34|1444.10|-1823.72| +2451380|46825|3644|365|867919|3473|9427|4|65|2175|64|48.60|52.00|45.24|1824.07|2895.36|3110.40|3328.00|0.00|1824.07|1071.29|1071.29|-2039.11| +2451380|46825|937|365|867919|3473|9427|4|145|2175|6|95.01|138.71|66.58|0.00|399.48|570.06|832.26|35.95|0.00|399.48|435.43|-170.58| +2451380|46825|11023|365|867919|3473|9427|4|62|2175|31|47.12|81.51|12.22|0.00|378.82|1460.72|2526.81|26.51|0.00|378.82|405.33|-1081.90| +2451380|46825|13280|365|867919|3473|9427|4|3|2175|100|29.22|35.64|19.60|0.00|1960.00|2922.00|3564.00|117.60|0.00|1960.00|2077.60|-962.00| +2451380|46825|16102|365|867919|3473|9427|4|216|2175|30|96.97|112.48|62.98|0.00|1889.40|2909.10|3374.40|37.78|0.00|1889.40|1927.18|-1019.70| +2451380|46825|4498|365|867919|3473|9427|4|218|2175|43|83.49|95.17|74.23|0.00|3191.89|3590.07|4092.31|95.75|0.00|3191.89|3287.64|-398.18| +2451380|46825|14491|365|867919|3473|9427|4|199|2175|83|31.42|45.87|4.12|324.86|341.96|2607.86|3807.21|0.00|324.86|17.10|17.10|-2590.76| +2451380|46825|8797|365|867919|3473|9427|4|102|2175|14|38.95|49.46|25.71|0.00|359.94|545.30|692.44|7.19|0.00|359.94|367.13|-185.36| +2452496|70013|463|89919|733060|1448|38189|7|61|2176|19|18.88|27.56|9.37|0.00|178.03|358.72|523.64|3.56|0.00|178.03|181.59|-180.69| +2452496|70013|583|89919|733060|1448|38189|7|51|2176|41|34.70|52.05|32.27|0.00|1323.07|1422.70|2134.05|119.07|0.00|1323.07|1442.14|-99.63| +2452496|70013|10039|89919|733060|1448|38189|7|223|2176|41|63.42|71.66|71.66|0.00|2938.06|2600.22|2938.06|117.52|0.00|2938.06|3055.58|337.84| +2452496|70013|11247|89919|733060|1448|38189|7|230|2176|63|98.73|145.13|71.11|0.00|4479.93|6219.99|9143.19|134.39|0.00|4479.93|4614.32|-1740.06| +2452496|70013|12985|89919|733060|1448|38189|7|250|2176|30|26.73|39.29|18.07|0.00|542.10|801.90|1178.70|21.68|0.00|542.10|563.78|-259.80| +2452496|70013|3037|89919|733060|1448|38189|7|173|2176|35|9.95|12.33|4.93|8.62|172.55|348.25|431.55|0.00|8.62|163.93|163.93|-184.32| +2452496|70013|10269|89919|733060|1448|38189|7|32|2176|100|55.15|110.30|39.70|0.00|3970.00|5515.00|11030.00|238.20|0.00|3970.00|4208.20|-1545.00| +2452496|70013|12381|89919|733060|1448|38189|7|120|2176|70|25.86|34.39|12.03|530.52|842.10|1810.20|2407.30|12.46|530.52|311.58|324.04|-1498.62| +2452496|70013|10746|89919|733060|1448|38189|7|175|2176|82|71.72|127.66|70.21|0.00|5757.22|5881.04|10468.12|115.14|0.00|5757.22|5872.36|-123.82| +2452496|70013|3060|89919|733060|1448|38189|7|296|2176|30|43.61|84.16|73.21|0.00|2196.30|1308.30|2524.80|153.74|0.00|2196.30|2350.04|888.00| +2452496|70013|13203|89919|733060|1448|38189|7|150|2176|34|96.35|98.27|83.52|624.72|2839.68|3275.90|3341.18|22.14|624.72|2214.96|2237.10|-1060.94| +2452496|70013|87|89919|733060|1448|38189|7|178|2176|37|5.96|10.01|9.40|0.00|347.80|220.52|370.37|27.82|0.00|347.80|375.62|127.28| +2452496|70013|9217|89919|733060|1448|38189|7|200|2176|5|84.28|150.86|12.06|0.00|60.30|421.40|754.30|4.22|0.00|60.30|64.52|-361.10| +2452496|70013|16963|89919|733060|1448|38189|7|99|2176|31|30.70|53.11|15.40|0.00|477.40|951.70|1646.41|14.32|0.00|477.40|491.72|-474.30| +2452496|70013|14109|89919|733060|1448|38189|7|142|2176|5|96.22|123.16|82.51|0.00|412.55|481.10|615.80|4.12|0.00|412.55|416.67|-68.55| +2452496|70013|13620|89919|733060|1448|38189|7|290|2176|94|31.26|60.64|27.89|0.00|2621.66|2938.44|5700.16|131.08|0.00|2621.66|2752.74|-316.78| +2451124|38483|2227|34985|597804|2223|5842|8|181|2177|17|46.70|75.18|7.51|0.00|127.67|793.90|1278.06|3.83|0.00|127.67|131.50|-666.23| +2451124|38483|1351|34985|597804|2223|5842|8|13|2177|92|11.49|11.60|5.56|0.00|511.52|1057.08|1067.20|10.23|0.00|511.52|521.75|-545.56| +2451124|38483|4255|34985|597804|2223|5842|8|249|2177|20|66.74|104.78|54.48|0.00|1089.60|1334.80|2095.60|43.58|0.00|1089.60|1133.18|-245.20| +2451124|38483|2726|34985|597804|2223|5842|8|125|2177|34|67.61|124.40|18.66|0.00|634.44|2298.74|4229.60|57.09|0.00|634.44|691.53|-1664.30| +2451124|38483|12806|34985|597804|2223|5842|8|272|2177|5|31.06|32.92|9.87|46.38|49.35|155.30|164.60|0.14|46.38|2.97|3.11|-152.33| +2451124|38483|17690|34985|597804|2223|5842|8|70|2177|22|19.41|38.43|36.12|0.00|794.64|427.02|845.46|0.00|0.00|794.64|794.64|367.62| +2451124|38483|2612|34985|597804|2223|5842|8|183|2177|75|67.24|91.44|84.12|0.00|6309.00|5043.00|6858.00|189.27|0.00|6309.00|6498.27|1266.00| +2451124|38483|11167|34985|597804|2223|5842|8|3|2177|37|48.69|70.11|59.59|0.00|2204.83|1801.53|2594.07|132.28|0.00|2204.83|2337.11|403.30| +2451124|38483|17086|34985|597804|2223|5842|8|90|2177|75|46.60|92.73|51.92|0.00|3894.00|3495.00|6954.75|38.94|0.00|3894.00|3932.94|399.00| +2451124|38483|2917|34985|597804|2223|5842|8|281|2177|25|33.84|63.95|40.92|439.89|1023.00|846.00|1598.75|34.98|439.89|583.11|618.09|-262.89| +2451124|38483|15122|34985|597804|2223|5842|8|144|2177|73|84.53|107.35|8.58|0.00|626.34|6170.69|7836.55|43.84|0.00|626.34|670.18|-5544.35| +2451940|53138|479|52173|1890020|3420|17667|2|150|2178|69|16.88|26.83|5.36|0.00|369.84|1164.72|1851.27|3.69|0.00|369.84|373.53|-794.88| +2451940|53138|6187|52173|1890020|3420|17667|2|58|2178|5|22.63|34.62|6.57|0.00|32.85|113.15|173.10|1.31|0.00|32.85|34.16|-80.30| +2451940|53138|14057|52173|1890020|3420|17667|2|45|2178|65|28.60|30.88|18.21|0.00|1183.65|1859.00|2007.20|106.52|0.00|1183.65|1290.17|-675.35| +2451940|53138|10009|52173|1890020|3420|17667|2|68|2178|92|17.10|33.68|15.82|0.00|1455.44|1573.20|3098.56|14.55|0.00|1455.44|1469.99|-117.76| +2451940|53138|15213|52173|1890020|3420|17667|2|73|2178|15|45.96|66.18|22.50|70.87|337.50|689.40|992.70|2.66|70.87|266.63|269.29|-422.77| +2451940|53138|13463|52173|1890020|3420|17667|2|170|2178|33|10.26|20.21|11.92|0.00|393.36|338.58|666.93|3.93|0.00|393.36|397.29|54.78| +2451940|53138|5815|52173|1890020|3420|17667|2|269|2178|43|91.94|162.73|22.78|0.00|979.54|3953.42|6997.39|48.97|0.00|979.54|1028.51|-2973.88| +2451940|53138|11535|52173|1890020|3420|17667|2|207|2178|57|32.17|58.87|52.39|0.00|2986.23|1833.69|3355.59|238.89|0.00|2986.23|3225.12|1152.54| +2451940|53138|15975|52173|1890020|3420|17667|2|296|2178|49|40.50|73.71|2.94|0.00|144.06|1984.50|3611.79|11.52|0.00|144.06|155.58|-1840.44| +2451940||4405|52173||3420||||2178|10|86.80|||0.00||868.00|1414.80||0.00||1046.90|178.90| +2451940|53138|13963|52173|1890020|3420|17667|2|223|2178|21|87.92|91.43|55.77|0.00|1171.17|1846.32|1920.03|35.13|0.00|1171.17|1206.30|-675.15| +2451940|53138|9939|52173|1890020|3420|17667|2|113|2178|88|2.28|3.87|0.77|0.00|67.76|200.64|340.56|0.00|0.00|67.76|67.76|-132.88| +2451940|53138|9891|52173|1890020|3420|17667|2|221|2178|51|55.32|64.17|34.65|0.00|1767.15|2821.32|3272.67|53.01|0.00|1767.15|1820.16|-1054.17| +2451940|53138|17675|52173|1890020|3420|17667|2|3|2178|37|93.86|101.36|35.47|419.96|1312.39|3472.82|3750.32|0.00|419.96|892.43|892.43|-2580.39| +2451398|34119|68|76792|381883|2174|45265|10|286|2179|37|16.84|24.41|16.11|0.00|596.07|623.08|903.17|47.68|0.00|596.07|643.75|-27.01| +2451398|34119|16460|76792|381883|2174|45265|10|253|2179|85|85.71|124.27|18.64|0.00|1584.40|7285.35|10562.95|47.53|0.00|1584.40|1631.93|-5700.95| +2451398|34119|889|76792|381883|2174|45265|10|202|2179|44|81.08|159.72|126.17|0.00|5551.48|3567.52|7027.68|444.11|0.00|5551.48|5995.59|1983.96| +2451398|34119|12026|76792|381883|2174|45265|10|161|2179|16|17.73|32.62|12.72|0.00|203.52|283.68|521.92|6.10|0.00|203.52|209.62|-80.16| +2451398|34119|1184|76792|381883|2174|45265|10|20|2179|60|56.20|75.30|48.19|0.00|2891.40|3372.00|4518.00|57.82|0.00|2891.40|2949.22|-480.60| +2451398|34119|16082|76792|381883|2174|45265|10|276|2179|27|33.10|53.95|39.38|0.00|1063.26|893.70|1456.65|42.53|0.00|1063.26|1105.79|169.56| +2451398|34119|7945|76792|381883|2174|45265|10|77|2179|10|45.12|64.97|22.73|0.00|227.30|451.20|649.70|15.91|0.00|227.30|243.21|-223.90| +2451398|34119|13483|76792|381883|2174|45265|10|28|2179|58|86.79|171.84|104.82|0.00|6079.56|5033.82|9966.72|547.16|0.00|6079.56|6626.72|1045.74| +2451398|34119|8332|76792|381883|2174|45265|10|176|2179|21|42.09|42.51|9.35|0.00|196.35|883.89|892.71|13.74|0.00|196.35|210.09|-687.54| +2451398|34119|15133|76792|381883|2174|45265|10|168|2179|90|52.35|59.67|45.34|0.00|4080.60|4711.50|5370.30|81.61|0.00|4080.60|4162.21|-630.90| +2451398|34119|16658|76792|381883|2174|45265|10|168|2179|53|99.45|121.32|35.18|0.00|1864.54|5270.85|6429.96|74.58|0.00|1864.54|1939.12|-3406.31| +2451398|34119|16507|76792|381883|2174|45265|10|209|2179|82|63.30|108.87|14.15|754.19|1160.30|5190.60|8927.34|12.18|754.19|406.11|418.29|-4784.49| +2451398|34119|10801|76792|381883|2174|45265|10|60|2179|81|74.89|117.57|49.37|2959.23|3998.97|6066.09|9523.17|41.58|2959.23|1039.74|1081.32|-5026.35| +2451398|34119|5348|76792|381883|2174|45265|10|122|2179|30|38.84|73.79|47.96|0.00|1438.80|1165.20|2213.70|115.10|0.00|1438.80|1553.90|273.60| +||10870|76792|381883|||10|48|2179|6||||0.00|100.98|90.54|112.26||0.00|||10.44| +2451398|34119|4180|76792|381883|2174|45265|10|25|2179|82|18.25|22.99|0.45|0.00|36.90|1496.50|1885.18|2.58|0.00|36.90|39.48|-1459.60| +2451654|53726|8003|64078|1755902|973|5597|10|151|2180|31|41.41|59.21|44.40|110.11|1376.40|1283.71|1835.51|50.65|110.11|1266.29|1316.94|-17.42| +2451654|53726|2048|64078|1755902|973|5597|10|54|2180|65|53.31|105.02|11.55|0.00|750.75|3465.15|6826.30|37.53|0.00|750.75|788.28|-2714.40| +2451654|53726|15872|64078|1755902|973|5597|10|12|2180|20|25.12|26.37|12.92|0.00|258.40|502.40|527.40|2.58|0.00|258.40|260.98|-244.00| +2451654|53726|2666|64078|1755902|973|5597|10|260|2180|21|36.03|49.00|46.06|309.52|967.26|756.63|1029.00|46.04|309.52|657.74|703.78|-98.89| +2451654|53726|15998|64078|1755902|973|5597|10|140|2180|75|35.56|54.76|20.26|0.00|1519.50|2667.00|4107.00|0.00|0.00|1519.50|1519.50|-1147.50| +2451654|53726|15728|64078|1755902|973|5597|10|269|2180|3|21.02|22.91|6.64|0.00|19.92|63.06|68.73|0.00|0.00|19.92|19.92|-43.14| +2451654|53726|3902|64078|1755902|973|5597|10|134|2180|60|38.47|47.31|8.04|0.00|482.40|2308.20|2838.60|19.29|0.00|482.40|501.69|-1825.80| +2451654|53726|14204|64078|1755902|973|5597|10|125|2180|32|56.74|59.00|25.37|300.38|811.84|1815.68|1888.00|10.22|300.38|511.46|521.68|-1304.22| +2451654|53726|5198|64078|1755902|973|5597|10|300|2180|31|94.96|143.38|57.35|0.00|1777.85|2943.76|4444.78|142.22|0.00|1777.85|1920.07|-1165.91| +2451654|53726|5089|64078|1755902|973|5597|10|217|2180|51|59.41|109.31|49.18|0.00|2508.18|3029.91|5574.81|0.00|0.00|2508.18|2508.18|-521.73| +2451654|53726|15139|64078|1755902|973|5597|10|120|2180|56|19.41|29.69|26.42|488.24|1479.52|1086.96|1662.64|29.73|488.24|991.28|1021.01|-95.68| +2451654|53726|4087|64078|1755902|973|5597|10|290|2180|75|63.23|71.44|16.43|468.25|1232.25|4742.25|5358.00|30.56|468.25|764.00|794.56|-3978.25| +2451654|53726|926|64078|1755902|973|5597|10|222|2180|74|42.50|81.17|72.24|0.00|5345.76|3145.00|6006.58|320.74|0.00|5345.76|5666.50|2200.76| +2451654|53726|11864|64078|1755902|973|5597|10|123|2180|77|43.36|60.27|3.61|0.00|277.97|3338.72|4640.79|11.11|0.00|277.97|289.08|-3060.75| +2451654|53726|14738|64078|1755902|973|5597|10|262|2180|27|50.63|86.57|2.59|0.00|69.93|1367.01|2337.39|6.29|0.00|69.93|76.22|-1297.08| +2451906|40755|8888|81700|69222|3144|13819|1|185|2181|44|11.84|17.52|16.64|600.37|732.16|520.96|770.88|2.63|600.37|131.79|134.42|-389.17| +2451906|40755|16855|81700|69222|3144|13819|1|131|2181|54|99.71|133.61|8.01|328.73|432.54|5384.34|7214.94|1.03|328.73|103.81|104.84|-5280.53| +2451906|40755|8641|81700|69222|3144|13819|1|210|2181|85|2.51|3.18|2.98|0.00|253.30|213.35|270.30|5.06|0.00|253.30|258.36|39.95| +2451906|40755|13052|81700|69222|3144|13819|1|248|2181|77|95.39|95.39|56.28|0.00|4333.56|7345.03|7345.03|173.34|0.00|4333.56|4506.90|-3011.47| +2451906|40755|6281|81700|69222|3144|13819|1|158|2181|66|7.54|7.69|5.69|217.81|375.54|497.64|507.54|1.57|217.81|157.73|159.30|-339.91| +2451906|40755|12452|81700|69222|3144|13819|1|174|2181|82|95.20|141.84|75.17|0.00|6163.94|7806.40|11630.88|431.47|0.00|6163.94|6595.41|-1642.46| +2451906|40755|3668|81700|69222|3144|13819|1|83|2181|69|12.21|14.52|4.93|0.00|340.17|842.49|1001.88|6.80|0.00|340.17|346.97|-502.32| +2451906|40755|17480|81700|69222|3144|13819|1|35|2181|79|36.14|69.02|13.11|0.00|1035.69|2855.06|5452.58|31.07|0.00|1035.69|1066.76|-1819.37| +2451906|40755|14060|81700|69222|3144|13819|1|81|2181|66|54.14|103.40|15.51|0.00|1023.66|3573.24|6824.40|81.89|0.00|1023.66|1105.55|-2549.58| +2451906|40755|5885|81700|69222|3144|13819|1|126|2181|22|73.26|120.14|62.47|0.00|1374.34|1611.72|2643.08|0.00|0.00|1374.34|1374.34|-237.38| +2451906|40755|6211|81700|69222|3144|13819|1|52|2181|63|14.19|22.84|18.04|0.00|1136.52|893.97|1438.92|34.09|0.00|1136.52|1170.61|242.55| +2451906|40755|7289|81700|69222|3144|13819|1|64|2181|91|30.89|42.01|37.80|0.00|3439.80|2810.99|3822.91|137.59|0.00|3439.80|3577.39|628.81| +2451906|40755|8072|81700|69222|3144|13819|1|153|2181|67|91.47|154.58|17.00|0.00|1139.00|6128.49|10356.86|34.17|0.00|1139.00|1173.17|-4989.49| +2451906|40755|1765|81700|69222|3144|13819|1|258|2181|42|3.54|5.20|3.27|0.00|137.34|148.68|218.40|2.74|0.00|137.34|140.08|-11.34| +2451906|40755|9290|81700|69222|3144|13819|1|246|2181|21|97.33|100.24|47.11|662.83|989.31|2043.93|2105.04|9.79|662.83|326.48|336.27|-1717.45| +2451486|53742|5908|66258|632430|202|18421|4|65|2182|70|70.43|134.52|99.54|0.00|6967.80|4930.10|9416.40|348.39|0.00|6967.80|7316.19|2037.70| +2451486|53742|12004|66258|632430|202|18421|4|14|2182|86|40.66|57.33|39.55|0.00|3401.30|3496.76|4930.38|306.11|0.00|3401.30|3707.41|-95.46| +2451486|53742|79|66258|632430|202|18421|4|194|2182|28|95.67|145.41|116.32|0.00|3256.96|2678.76|4071.48|293.12|0.00|3256.96|3550.08|578.20| +2451486|53742|4636|66258|632430|202|18421|4|29|2182|56|40.51|40.51|34.02|0.00|1905.12|2268.56|2268.56|19.05|0.00|1905.12|1924.17|-363.44| +2451486|53742|7240|66258|632430|202|18421|4|143|2182|96|85.73|116.59|9.32|0.00|894.72|8230.08|11192.64|53.68|0.00|894.72|948.40|-7335.36| +2451486|53742|11257|66258|632430|202|18421|4|126|2182|52|20.62|32.16|31.19|0.00|1621.88|1072.24|1672.32|48.65|0.00|1621.88|1670.53|549.64| +2451486|53742|12397|66258|632430|202|18421|4|75|2182|73|22.75|29.34|14.67|428.36|1070.91|1660.75|2141.82|0.00|428.36|642.55|642.55|-1018.20| +2451486|53742|9092|66258|632430|202|18421|4|259|2182|80|54.43|62.05|13.03|0.00|1042.40|4354.40|4964.00|31.27|0.00|1042.40|1073.67|-3312.00| +2451486|53742|15403|66258|632430|202|18421|4|64|2182|75|25.02|41.03|26.66|0.00|1999.50|1876.50|3077.25|99.97|0.00|1999.50|2099.47|123.00| +2451486|53742|8180|66258|632430|202|18421|4|113|2182|71|51.04|84.72|54.22|0.00|3849.62|3623.84|6015.12|307.96|0.00|3849.62|4157.58|225.78| +|53742|17630|66258||||4|32|2182|24|7.43|9.21|||152.40|178.32|221.04|||||| +2451486|53742|319|66258|632430|202|18421|4|194|2182|85|96.30|161.78|37.20|0.00|3162.00|8185.50|13751.30|189.72|0.00|3162.00|3351.72|-5023.50| +2451486|53742|11218|66258|632430|202|18421|4|236|2182|49|40.80|45.28|23.54|0.00|1153.46|1999.20|2218.72|103.81|0.00|1153.46|1257.27|-845.74| +2451486|53742|8413|66258|632430|202|18421|4|234|2182|21|49.59|61.49|24.59|0.00|516.39|1041.39|1291.29|36.14|0.00|516.39|552.53|-525.00| +2452237|63764|11157|85872|1213959|4446|41333|7|269|2183|7|71.38|92.08|8.28|0.00|57.96|499.66|644.56|4.05|0.00|57.96|62.01|-441.70| +2452237|63764|4239|85872|1213959|4446|41333|7|261|2183|83|18.13|28.46|6.54|0.00|542.82|1504.79|2362.18|27.14|0.00|542.82|569.96|-961.97| +2452237|63764|8527|85872|1213959|4446|41333|7|162|2183|54|82.24|95.39|50.55|0.00|2729.70|4440.96|5151.06|245.67|0.00|2729.70|2975.37|-1711.26| +2452237|63764|15881|85872|1213959|4446|41333|7|122|2183|49|1.75|2.85|0.19|0.00|9.31|85.75|139.65|0.65|0.00|9.31|9.96|-76.44| +2452237|63764|5265|85872|1213959|4446|41333|7|265|2183|50|36.13|45.88|41.29|0.00|2064.50|1806.50|2294.00|103.22|0.00|2064.50|2167.72|258.00| +2452237|63764|11035|85872|1213959|4446|41333|7|147|2183|81|63.80|73.37|53.56|0.00|4338.36|5167.80|5942.97|260.30|0.00|4338.36|4598.66|-829.44| +||7195|85872|1213959||||80|2183||||||6806.49|||||||-396.73| +2452237|63764|14995|85872|1213959|4446|41333|7|108|2183|51|36.12|59.23|33.16|0.00|1691.16|1842.12|3020.73|0.00|0.00|1691.16|1691.16|-150.96| +2452237|63764|1699|85872|1213959|4446|41333|7|266|2183|79|97.54|160.94|28.96|0.00|2287.84|7705.66|12714.26|160.14|0.00|2287.84|2447.98|-5417.82| +2451961|40944|3265|35079|1196122|6412|42655|1|202|2184|86|56.25|82.12|22.99|1720.11|1977.14|4837.50|7062.32|0.00|1720.11|257.03|257.03|-4580.47| +2451961|40944|1197|35079|1196122|6412|42655|1|201|2184|73|93.76|160.32|36.87|0.00|2691.51|6844.48|11703.36|161.49|0.00|2691.51|2853.00|-4152.97| +2451961|40944|9487|35079|1196122|6412|42655|1|6|2184|10|98.35|168.17|104.26|0.00|1042.60|983.50|1681.70|72.98|0.00|1042.60|1115.58|59.10| +2451961|40944|16105|35079|1196122|6412|42655|1|243|2184|24|16.99|28.03|9.81|0.00|235.44|407.76|672.72|7.06|0.00|235.44|242.50|-172.32| +2451961|40944|4027|35079|1196122|6412|42655|1|228|2184|88|36.28|68.93|44.11|2057.29|3881.68|3192.64|6065.84|36.48|2057.29|1824.39|1860.87|-1368.25| +2451961|40944|143|35079|1196122|6412|42655|1|16|2184|18|9.73|15.66|10.96|88.77|197.28|175.14|281.88|0.00|88.77|108.51|108.51|-66.63| +2451961|40944|15921|35079|1196122|6412|42655|1|242|2184|34|18.70|29.17|13.41|0.00|455.94|635.80|991.78|4.55|0.00|455.94|460.49|-179.86| +2451961|40944|3255|35079|1196122|6412|42655|1|184|2184|5|41.07|79.67|0.79|0.00|3.95|205.35|398.35|0.35|0.00|3.95|4.30|-201.40| +2451267|75047|7756|65722|61285|848|46811|2|245|2185|78|76.23|112.82|56.41|0.00|4399.98|5945.94|8799.96|351.99|0.00|4399.98|4751.97|-1545.96| +2451267|75047|8240|65722|61285|848|46811|2|165|2185|20|7.22|13.42|3.35|0.00|67.00|144.40|268.40|4.69|0.00|67.00|71.69|-77.40| +2451267|75047|10270|65722|61285|848|46811|2|91|2185|58|45.36|71.66|13.61|0.00|789.38|2630.88|4156.28|23.68|0.00|789.38|813.06|-1841.50| +2451267|75047|11264|65722|61285|848|46811|2|244|2185|10|73.10|107.45|93.48|0.00|934.80|731.00|1074.50|84.13|0.00|934.80|1018.93|203.80| +2451267|75047|4090|65722|61285|848|46811|2|85|2185|33|16.87|26.48|5.03|0.00|165.99|556.71|873.84|1.65|0.00|165.99|167.64|-390.72| +2451267|75047|7387|65722||848||2|193|2185|91||||0.00|1859.13||5469.10||0.00|||| +2451267|75047|14980|65722|61285|848|46811|2|156|2185|71|85.65|121.62|121.62|4317.51|8635.02|6081.15|8635.02|302.22|4317.51|4317.51|4619.73|-1763.64| +2451267|75047|2587|65722|61285|848|46811|2|28|2185|89|39.59|46.32|41.68|0.00|3709.52|3523.51|4122.48|333.85|0.00|3709.52|4043.37|186.01| +2451267|75047|10382|65722|61285|848|46811|2|29|2185|6|26.04|44.52|43.62|0.00|261.72|156.24|267.12|0.00|0.00|261.72|261.72|105.48| +2451267|75047|4202|65722|61285|848|46811|2|1|2185|77|71.82|114.19|90.21|0.00|6946.17|5530.14|8792.63|625.15|0.00|6946.17|7571.32|1416.03| +2451267|75047|3496|65722|61285|848|46811|2|243|2185|65|47.72|58.21|29.68|1755.57|1929.20|3101.80|3783.65|8.68|1755.57|173.63|182.31|-2928.17| +2451267|75047|4183|65722|61285|848|46811|2|120|2185|26|64.25|68.74|43.30|0.00|1125.80|1670.50|1787.24|90.06|0.00|1125.80|1215.86|-544.70| +2451267|75047|13426|65722|61285|848|46811|2|231|2185|43|46.31|90.30|13.54|0.00|582.22|1991.33|3882.90|34.93|0.00|582.22|617.15|-1409.11| +2451267|75047|664|65722|61285|848|46811|2|261|2185|48|90.53|101.39|66.91|0.00|3211.68|4345.44|4866.72|160.58|0.00|3211.68|3372.26|-1133.76| +2451267|75047|11893|65722||||2||2185|56|47.75||||1200.08|2674.00|3529.68|||252.02||| +2451552||416||1442737||38434||5|2186||55.09||9.00|||||||288.00|302.40|| +2451552|59084|14420|29065|1442737|5562|38434|1|40|2186|32|37.69|45.60|26.90|0.00|860.80|1206.08|1459.20|43.04|0.00|860.80|903.84|-345.28| +2451552|59084|6949|29065|1442737|5562|38434|1|282|2186|27|92.72|165.04|107.27|0.00|2896.29|2503.44|4456.08|28.96|0.00|2896.29|2925.25|392.85| +2451552|59084|7825|29065|1442737|5562|38434|1|141|2186|46|77.12|138.04|63.49|0.00|2920.54|3547.52|6349.84|116.82|0.00|2920.54|3037.36|-626.98| +2451552|59084|14351|29065|1442737|5562|38434|1|63|2186|100|13.20|22.70|16.34|0.00|1634.00|1320.00|2270.00|147.06|0.00|1634.00|1781.06|314.00| +2451552|59084|8719|29065|1442737|5562|38434|1|63|2186|72|18.41|24.11|6.26|0.00|450.72|1325.52|1735.92|9.01|0.00|450.72|459.73|-874.80| +2451552|59084|10555|29065|1442737|5562|38434|1|52|2186|71|45.03|49.98|24.99|0.00|1774.29|3197.13|3548.58|88.71|0.00|1774.29|1863.00|-1422.84| +2451552|59084|15119|29065|1442737|5562|38434|1|144|2186|56|67.58|97.99|33.31|0.00|1865.36|3784.48|5487.44|130.57|0.00|1865.36|1995.93|-1919.12| +2451552|59084|11216|29065|1442737|5562|38434|1|185|2186|85|17.30|33.21|25.57|0.00|2173.45|1470.50|2822.85|43.46|0.00|2173.45|2216.91|702.95| +2451552|59084|16657|29065|1442737|5562|38434|1|283|2186|98|89.78|149.03|119.22|0.00|11683.56|8798.44|14604.94|817.84|0.00|11683.56|12501.40|2885.12| +2451552|59084|6446|29065|1442737|5562|38434|1|237|2186|59|41.89|44.82|42.13|994.26|2485.67|2471.51|2644.38|14.91|994.26|1491.41|1506.32|-980.10| +2451552|59084|14867|29065|1442737|5562|38434|1|56|2186|85|3.01|4.54|0.95|0.00|80.75|255.85|385.90|3.23|0.00|80.75|83.98|-175.10| +2451552|59084|16007|29065|1442737|5562|38434|1|9|2186|81|28.05|53.29|28.24|0.00|2287.44|2272.05|4316.49|68.62|0.00|2287.44|2356.06|15.39| +2451582|43887|6985|10200|1624319|2791|10510|2|2|2187|61|22.24|29.80|15.79|173.37|963.19|1356.64|1817.80|55.28|173.37|789.82|845.10|-566.82| +2451582|43887|12847|10200|1624319|2791|10510|2|1|2187|40|83.31|139.12|114.07|0.00|4562.80|3332.40|5564.80|182.51|0.00|4562.80|4745.31|1230.40| +2451582|43887|6206|10200|1624319|2791|10510|2|22|2187|6|33.40|42.75|0.00|0.00|0.00|200.40|256.50|0.00|0.00|0.00|0.00|-200.40| +2451582|43887|1544|10200|1624319|2791|10510|2|14|2187|85|38.32|70.89|63.09|0.00|5362.65|3257.20|6025.65|107.25|0.00|5362.65|5469.90|2105.45| +2451582|43887|14054|10200|1624319|2791|10510|2|242|2187|91|15.15|22.72|9.54|0.00|868.14|1378.65|2067.52|43.40|0.00|868.14|911.54|-510.51| +2451582|43887|7010|10200|1624319|2791|10510|2|148|2187|43|86.49|162.60|100.81|0.00|4334.83|3719.07|6991.80|86.69|0.00|4334.83|4421.52|615.76| +2451582|43887|4232|10200|1624319|2791|10510|2|180|2187|65|88.20|125.24|7.51|0.00|488.15|5733.00|8140.60|0.00|0.00|488.15|488.15|-5244.85| +2451582|43887|17252|10200|1624319|2791|10510|2|80|2187|33|99.68|172.44|96.56|0.00|3186.48|3289.44|5690.52|159.32|0.00|3186.48|3345.80|-102.96| +2451582|43887|11587|10200|1624319|2791|10510|2|65|2187|56|42.62|80.97|43.72|0.00|2448.32|2386.72|4534.32|48.96|0.00|2448.32|2497.28|61.60| +2451582|43887|15619|10200|1624319|2791|10510|2|132|2187|71|25.26|47.48|3.79|0.00|269.09|1793.46|3371.08|24.21|0.00|269.09|293.30|-1524.37| +2451582|43887|17450|10200|1624319|2791|10510|2|50|2187|61|77.53|90.71|18.14|0.00|1106.54|4729.33|5533.31|11.06|0.00|1106.54|1117.60|-3622.79| +2451582|43887|2377|10200|1624319|2791|10510|2|269|2187|38|35.50|68.51|17.12|0.00|650.56|1349.00|2603.38|0.00|0.00|650.56|650.56|-698.44| +2451582|43887|5179|10200|1624319|2791|10510|2|140|2187|41|48.26|92.17|85.71|0.00|3514.11|1978.66|3778.97|105.42|0.00|3514.11|3619.53|1535.45| +2451582|43887|15757|10200|1624319|2791|10510|2|157|2187|91|30.57|41.26|2.88|0.00|262.08|2781.87|3754.66|13.10|0.00|262.08|275.18|-2519.79| +2451582|43887|3667|10200|1624319|2791|10510|2|131|2187|41|14.84|16.32|5.05|169.78|207.05|608.44|669.12|0.37|169.78|37.27|37.64|-571.17| +|43887|17341||1624319||||130|2187|||7.63||0.00||456.98|556.99|0.00|0.00||540.20|| +2450820|39273|16444|57218|577890|6209|36881|8|217|2188|35|82.74|152.24|143.10|0.00|5008.50|2895.90|5328.40|400.68|0.00|5008.50|5409.18|2112.60| +2450820|39273|15652|57218|577890|6209|36881|8|293|2188|60|10.93|21.75|2.17|0.00|130.20|655.80|1305.00|11.71|0.00|130.20|141.91|-525.60| +2450820|39273|3688|57218|577890|6209|36881|8|261|2188|87|13.06|23.63|11.34|0.00|986.58|1136.22|2055.81|19.73|0.00|986.58|1006.31|-149.64| +2450820|39273|8534|57218|577890|6209|36881|8|202|2188|75|56.80|65.32|62.05|0.00|4653.75|4260.00|4899.00|46.53|0.00|4653.75|4700.28|393.75| +2450820|39273|7369|57218|577890|6209|36881|8|75|2188|62|31.20|50.54|26.28|0.00|1629.36|1934.40|3133.48|97.76|0.00|1629.36|1727.12|-305.04| +2450820|39273|3226|57218|577890|6209|36881|8|108|2188|19|11.48|13.54|11.77|0.00|223.63|218.12|257.26|4.47|0.00|223.63|228.10|5.51| +2450820|39273|13063|57218|577890|6209|36881|8|36|2188|66|83.81|145.82|0.00|0.00|0.00|5531.46|9624.12|0.00|0.00|0.00|0.00|-5531.46| +2450820|39273|12787|57218|577890|6209|36881|8|126|2188|63|42.47|77.29|65.69|0.00|4138.47|2675.61|4869.27|206.92|0.00|4138.47|4345.39|1462.86| +2450820|39273|14192|57218|577890|6209|36881|8|98|2188|37|31.15|34.88|18.13|670.81|670.81|1152.55|1290.56|0.00|670.81|0.00|0.00|-1152.55| +2450820|39273|4036|57218|577890|6209|36881|8|149|2188|29|10.21|16.84|9.76|0.00|283.04|296.09|488.36|11.32|0.00|283.04|294.36|-13.05| +2450820|39273|8168|57218|577890|6209|36881|8|251|2188|65|17.43|28.75|5.17|0.00|336.05|1132.95|1868.75|23.52|0.00|336.05|359.57|-796.90| +2450820|39273|1681|57218|577890|6209|36881|8|179|2188|73|30.23|50.78|40.62|2698.38|2965.26|2206.79|3706.94|10.67|2698.38|266.88|277.55|-1939.91| +2450820|39273|6679|57218|577890|6209|36881|8|181|2188|41|75.63|106.63|30.92|0.00|1267.72|3100.83|4371.83|88.74|0.00|1267.72|1356.46|-1833.11| +2450820|39273|4586|57218|577890|6209|36881|8|272|2188|84|68.54|95.27|19.05|0.00|1600.20|5757.36|8002.68|96.01|0.00|1600.20|1696.21|-4157.16| +2450820|39273|56|57218|577890|6209|36881|8|38|2188|45|80.81|119.59|75.34|0.00|3390.30|3636.45|5381.55|203.41|0.00|3390.30|3593.71|-246.15| +2451495|63794|1406|1483|881114|3945|27576|8|39|2189|22|97.09|101.94|47.91|368.90|1054.02|2135.98|2242.68|0.00|368.90|685.12|685.12|-1450.86| +2451495|63794|17785|1483|881114|3945|27576|8|262|2189|11|29.31|40.15|32.52|0.00|357.72|322.41|441.65|21.46|0.00|357.72|379.18|35.31| +2451495|63794|9626|1483|881114|3945|27576|8|148|2189|85|13.53|17.72|8.32|0.00|707.20|1150.05|1506.20|49.50|0.00|707.20|756.70|-442.85| +2451495|63794|17599|1483|881114|3945|27576|8|256|2189|37|51.32|80.05|54.43|1530.57|2013.91|1898.84|2961.85|4.83|1530.57|483.34|488.17|-1415.50| +2451495|63794|10006|1483|881114|3945|27576|8|269|2189|85|9.97|17.24|11.37|0.00|966.45|847.45|1465.40|48.32|0.00|966.45|1014.77|119.00| +2451495|63794|2176|1483|881114|3945|27576|8|32|2189|54|14.50|20.01|3.40|0.00|183.60|783.00|1080.54|16.52|0.00|183.60|200.12|-599.40| +2451495|63794|6520|1483|881114|3945|27576|8|280|2189|49|83.00|102.09|38.79|0.00|1900.71|4067.00|5002.41|19.00|0.00|1900.71|1919.71|-2166.29| +2451495|63794|8450|1483|881114|3945|27576|8|286|2189|46|5.91|11.28|2.25|80.73|103.50|271.86|518.88|0.45|80.73|22.77|23.22|-249.09| +2451181|67800|13040|56895|545070|445|47420|2|292|2190|96|53.44|105.27|73.68|3890.30|7073.28|5130.24|10105.92|127.31|3890.30|3182.98|3310.29|-1947.26| +2451181|67800|9260|56895|545070|445|47420|2|39|2190|84|5.24|10.37|5.49|0.00|461.16|440.16|871.08|32.28|0.00|461.16|493.44|21.00| +2451181|67800|13627|56895|545070|445|47420|2|140|2190|58|37.50|71.62|23.63|0.00|1370.54|2175.00|4153.96|41.11|0.00|1370.54|1411.65|-804.46| +2451181|67800|17360|56895|545070|445|47420|2|18|2190|96|93.87|114.52|89.32|0.00|8574.72|9011.52|10993.92|171.49|0.00|8574.72|8746.21|-436.80| +2451181|67800|802|56895|545070|445|47420|2|268|2190|100|8.83|13.33|5.86|0.00|586.00|883.00|1333.00|41.02|0.00|586.00|627.02|-297.00| +2451181|67800|4072|56895|545070|445|47420|2|101|2190|61|52.68|102.72|87.31|1118.44|5325.91|3213.48|6265.92|252.44|1118.44|4207.47|4459.91|993.99| +2451181|67800|8062|56895|545070|445|47420|2|70|2190|47|72.93|120.33|1.20|0.00|56.40|3427.71|5655.51|4.51|0.00|56.40|60.91|-3371.31| +2451181|67800|9194|56895|545070|445|47420|2|219|2190|88|94.44|149.21|128.32|0.00|11292.16|8310.72|13130.48|338.76|0.00|11292.16|11630.92|2981.44| +2451181|67800|2236|56895|545070|445|47420|2|54|2190|34|80.24|85.05|60.38|0.00|2052.92|2728.16|2891.70|184.76|0.00|2052.92|2237.68|-675.24| +2451252|62530|9520|8872|1754868|6017|33213|7|265|2191|40|25.49|36.96|22.54|838.48|901.60|1019.60|1478.40|2.52|838.48|63.12|65.64|-956.48| +2451252|62530|6830|8872|1754868|6017|33213|7|182|2191|5|50.20|50.70|48.16|0.00|240.80|251.00|253.50|4.81|0.00|240.80|245.61|-10.20| +2451252|62530|2138|8872|1754868|6017|33213|7|146|2191|34|94.59|135.26|110.91|0.00|3770.94|3216.06|4598.84|339.38|0.00|3770.94|4110.32|554.88| +||181||1754868|||||2191||||||303.16||1685.20|18.18||||| +2451252|62530|13045|8872|1754868|6017|33213|7|68|2191|84|86.17|163.72|80.22|0.00|6738.48|7238.28|13752.48|539.07|0.00|6738.48|7277.55|-499.80| +2451252||10154||1754868|6017||||2191||67.35|||0.00|194.95||649.90||0.00|194.95|200.79|-141.80| +2451252|62530|15218|8872|1754868|6017|33213|7|246|2191|38|23.22|24.61|19.44|0.00|738.72|882.36|935.18|7.38|0.00|738.72|746.10|-143.64| +2451252|62530|7393|8872|1754868|6017|33213|7|16|2191|75|6.45|6.70|5.15|0.00|386.25|483.75|502.50|11.58|0.00|386.25|397.83|-97.50| +2451252|62530|6445|8872|1754868|6017|33213|7|274|2191|95|34.12|58.34|11.66|0.00|1107.70|3241.40|5542.30|99.69|0.00|1107.70|1207.39|-2133.70| +2451252|62530|15230|8872|1754868|6017|33213|7|105|2191|51|97.89|183.05|23.79|0.00|1213.29|4992.39|9335.55|60.66|0.00|1213.29|1273.95|-3779.10| +2451252|62530|9853|8872|1754868|6017|33213|7|45|2191|55|47.44|60.72|39.46|0.00|2170.30|2609.20|3339.60|0.00|0.00|2170.30|2170.30|-438.90| +2451252|62530|8266|8872|1754868|6017|33213|7|165|2191|65|48.55|67.48|15.52|0.00|1008.80|3155.75|4386.20|60.52|0.00|1008.80|1069.32|-2146.95| +2451252|62530|7844|8872|1754868|6017|33213|7|245|2191|55|37.73|67.53|54.69|0.00|3007.95|2075.15|3714.15|120.31|0.00|3007.95|3128.26|932.80| +2451538|41352|2203|56215|740150|438|48726|7|136|2192|22|75.47|86.79|7.81|0.00|171.82|1660.34|1909.38|15.46|0.00|171.82|187.28|-1488.52| +2451538|41352|2168|56215|740150|438|48726|7|266|2192|29|58.09|59.25|9.48|0.00|274.92|1684.61|1718.25|24.74|0.00|274.92|299.66|-1409.69| +2451538|41352|17030|56215|740150|438|48726|7|110|2192|24|5.77|9.92|5.65|0.00|135.60|138.48|238.08|0.00|0.00|135.60|135.60|-2.88| +2451538|41352|10304|56215|740150|438|48726|7|130|2192|4|96.31|104.97|52.48|2.09|209.92|385.24|419.88|2.07|2.09|207.83|209.90|-177.41| +2451538|41352|433|56215|740150|438|48726|7|9|2192|91|31.04|32.59|18.25|0.00|1660.75|2824.64|2965.69|49.82|0.00|1660.75|1710.57|-1163.89| +2451538|41352|9728|56215|740150|438|48726|7|270|2192|96|2.28|3.39|3.11|0.00|298.56|218.88|325.44|26.87|0.00|298.56|325.43|79.68| +2451538|41352|2029|56215|740150|438|48726|7|259|2192|49|65.28|107.71|93.70|4040.34|4591.30|3198.72|5277.79|38.56|4040.34|550.96|589.52|-2647.76| +2451538|41352|16732|56215|740150|438|48726|7|164|2192|10|84.11|91.67|67.83|0.00|678.30|841.10|916.70|33.91|0.00|678.30|712.21|-162.80| +2451538|41352|439|56215|740150|438|48726|7|103|2192|72|65.48|91.01|48.23|0.00|3472.56|4714.56|6552.72|277.80|0.00|3472.56|3750.36|-1242.00| +2451538|41352|4762|56215|740150|438|48726|7|56|2192|97|24.07|39.71|30.57|0.00|2965.29|2334.79|3851.87|148.26|0.00|2965.29|3113.55|630.50| +2451538|41352|15049|56215|740150|438|48726|7|106|2192|7|32.40|36.61|23.79|0.00|166.53|226.80|256.27|4.99|0.00|166.53|171.52|-60.27| +2451538|41352|964|56215|740150|438|48726|7|77|2192|81|56.98|75.78|48.49|0.00|3927.69|4615.38|6138.18|314.21|0.00|3927.69|4241.90|-687.69| +2451538|41352|14455|56215|740150|438|48726|7|260|2192|37|33.75|34.42|4.13|106.96|152.81|1248.75|1273.54|3.20|106.96|45.85|49.05|-1202.90| +2451538|41352|8300|56215|740150|438|48726|7|63|2192|84|90.97|161.92|35.62|209.44|2992.08|7641.48|13601.28|194.78|209.44|2782.64|2977.42|-4858.84| +2451538|41352|13090|56215|740150|438|48726|7|283|2192|18|33.88|50.48|1.00|0.00|18.00|609.84|908.64|0.72|0.00|18.00|18.72|-591.84| +2452544|69650|4057|61562|38039|2963|15087|4|280|2193|76|95.97|172.74|145.10|0.00|11027.60|7293.72|13128.24|551.38|0.00|11027.60|11578.98|3733.88| +2452544|69650|6456|61562|38039|2963|15087|4|144|2193|98|48.53|65.51|9.82|0.00|962.36|4755.94|6419.98|38.49|0.00|962.36|1000.85|-3793.58| +2452544|69650|5401|61562|38039|2963|15087|4|275|2193|35|82.59|141.22|86.14|0.00|3014.90|2890.65|4942.70|211.04|0.00|3014.90|3225.94|124.25| +||567|61562|38039|2963|15087|||2193|71|||12.74||904.54|6328.23|6960.84||||940.72|| +2452544|69650|13455|61562|38039|2963|15087|4|24|2193|19|73.58|88.29|37.08|0.00|704.52|1398.02|1677.51|63.40|0.00|704.52|767.92|-693.50| +2452544|69650|11340|61562|38039|2963|15087|4|115|2193|16|25.68|39.29|23.18|211.40|370.88|410.88|628.64|4.78|211.40|159.48|164.26|-251.40| +2452544|69650|5691|61562|38039|2963|15087|4|131|2193|46|64.70|117.10|28.10|0.00|1292.60|2976.20|5386.60|77.55|0.00|1292.60|1370.15|-1683.60| +2452544|69650|15267|61562|38039|2963|15087|4|192|2193|34|81.23|152.71|134.38|0.00|4568.92|2761.82|5192.14|91.37|0.00|4568.92|4660.29|1807.10| +2452544|69650|16531|61562|38039|2963|15087|4|45|2193|23|6.08|9.66|3.47|0.00|79.81|139.84|222.18|0.79|0.00|79.81|80.60|-60.03| +2452544|69650|7483|61562|38039|2963|15087|4|98|2193|91|42.29|69.35|2.08|0.00|189.28|3848.39|6310.85|3.78|0.00|189.28|193.06|-3659.11| +2452544|69650|14220|61562|38039|2963|15087|4|83|2193|71|79.78|118.07|40.14|0.00|2849.94|5664.38|8382.97|56.99|0.00|2849.94|2906.93|-2814.44| +2452544|69650|11358|61562|38039|2963|15087|4|202|2193|3|67.91|131.74|36.88|0.00|110.64|203.73|395.22|9.95|0.00|110.64|120.59|-93.09| +2452544|69650|1791|61562|38039|2963|15087|4|282|2193|35|69.86|137.62|35.78|25.04|1252.30|2445.10|4816.70|98.18|25.04|1227.26|1325.44|-1217.84| +2452544|69650|3571|61562|38039|2963|15087|4|83|2193|87|49.01|72.04|72.04|4261.88|6267.48|4263.87|6267.48|0.00|4261.88|2005.60|2005.60|-2258.27| +|69650|13879|61562|38039|2963|15087||237|2193|40|||||||5718.80|144.51||||| +2452544|69650|14113|61562|38039|2963|15087|4|156|2193|80|99.04|143.60|1.43|0.00|114.40|7923.20|11488.00|8.00|0.00|114.40|122.40|-7808.80| +2451884|49032|8918|70895|332075|4455|18882|7|267|2194|80|87.43|158.24|53.80|0.00|4304.00|6994.40|12659.20|344.32|0.00|4304.00|4648.32|-2690.40| +2451884|49032|7166|70895|332075|4455|18882|7|126|2194|45|73.25|141.37|127.23|0.00|5725.35|3296.25|6361.65|171.76|0.00|5725.35|5897.11|2429.10| +2451884|49032|11107|70895|332075|4455|18882|7|291|2194|18|63.51|97.17|96.19|0.00|1731.42|1143.18|1749.06|34.62|0.00|1731.42|1766.04|588.24| +2451884|49032|7663|70895|332075|4455|18882|7|12|2194|89|9.88|16.59|15.09|0.00|1343.01|879.32|1476.51|67.15|0.00|1343.01|1410.16|463.69| +||13574||332075|4455||||2194|87|61.07||||3293.82|5313.09|8234.55|||3293.82||| +2451884|49032|10481|70895|332075|4455|18882|7|151|2194|16|47.59|80.42|24.12|0.00|385.92|761.44|1286.72|3.85|0.00|385.92|389.77|-375.52| +2451884|49032|10849|70895|332075|4455|18882|7|12|2194|57|98.35|99.33|0.99|0.00|56.43|5605.95|5661.81|1.69|0.00|56.43|58.12|-5549.52| +2451884|49032|16853|70895|332075|4455|18882|7|248|2194|60|68.85|128.06|28.17|0.00|1690.20|4131.00|7683.60|84.51|0.00|1690.20|1774.71|-2440.80| +2451884|49032|13142|70895|332075|4455|18882|7|255|2194|50|27.97|35.52|15.27|0.00|763.50|1398.50|1776.00|30.54|0.00|763.50|794.04|-635.00| +2451884|49032|302|70895|332075|4455|18882|7|179|2194|31|58.33|71.16|20.63|613.94|639.53|1808.23|2205.96|0.51|613.94|25.59|26.10|-1782.64| +2451884|49032|4454|70895|332075|4455|18882|7|272|2194|81|13.33|15.72|6.44|292.11|521.64|1079.73|1273.32|4.59|292.11|229.53|234.12|-850.20| +2452234|40260|3793|10059|91623|3234|35598|10|54|2195|13|26.79|46.61|7.92|0.00|102.96|348.27|605.93|8.23|0.00|102.96|111.19|-245.31| +2452234|40260|15689|10059|91623|3234|35598|10|135|2195|12|32.63|37.52|13.88|0.00|166.56|391.56|450.24|11.65|0.00|166.56|178.21|-225.00| +2452234|40260|12279|10059|91623|3234|35598|10|234|2195|11|42.52|55.27|29.29|0.00|322.19|467.72|607.97|0.00|0.00|322.19|322.19|-145.53| +2452234|40260|7983|10059|91623|3234|35598|10|36|2195|95|43.86|53.50|14.44|0.00|1371.80|4166.70|5082.50|0.00|0.00|1371.80|1371.80|-2794.90| +2452234|40260|11717|10059|91623|3234|35598|10|255|2195|21|33.14|36.45|9.84|0.00|206.64|695.94|765.45|12.39|0.00|206.64|219.03|-489.30| +2452234|40260|6057|10059|91623|3234|35598|10|80|2195|26|13.80|19.87|14.70|0.00|382.20|358.80|516.62|7.64|0.00|382.20|389.84|23.40| +2452234|40260|14593|10059|91623|3234|35598|10|284|2195|79|96.20|140.45|134.83|0.00|10651.57|7599.80|11095.55|0.00|0.00|10651.57|10651.57|3051.77| +2452234|40260|6439|10059|91623|3234|35598|10|233|2195|98|97.87|147.78|63.54|0.00|6226.92|9591.26|14482.44|373.61|0.00|6226.92|6600.53|-3364.34| +2451177|36825|17900|19752|541968|5784|27600|8|94|2196|23|18.65|36.18|13.74|0.00|316.02|428.95|832.14|3.16|0.00|316.02|319.18|-112.93| +||1594||541968||27600||26|2196|||92.72||0.00||5682.90||47.55|0.00||4803.25|| +2451177|36825|10465|19752|541968|5784|27600|8|74|2196|36|15.84|26.45|5.02|0.00|180.72|570.24|952.20|0.00|0.00|180.72|180.72|-389.52| +2451177|36825|3664|19752|541968|5784|27600|8|222|2196|63|35.93|50.30|20.62|0.00|1299.06|2263.59|3168.90|25.98|0.00|1299.06|1325.04|-964.53| +2451177|36825|12445|19752|541968|5784|27600|8|300|2196|25|15.83|23.90|7.88|0.00|197.00|395.75|597.50|7.88|0.00|197.00|204.88|-198.75| +2451177|36825|5642|19752|541968|5784|27600|8|46|2196|50|11.72|17.93|0.00|0.00|0.00|586.00|896.50|0.00|0.00|0.00|0.00|-586.00| +2451177|36825|10136|19752|541968|5784|27600|8|16|2196|20|58.54|67.32|60.58|0.00|1211.60|1170.80|1346.40|109.04|0.00|1211.60|1320.64|40.80| +2451177|36825|17128|19752|541968|5784|27600|8|104|2196|18|8.92|9.18|1.37|0.00|24.66|160.56|165.24|0.00|0.00|24.66|24.66|-135.90| +2451177|36825|2402|19752|541968|5784|27600|8|161|2196|3|60.10|87.74|8.77|0.00|26.31|180.30|263.22|1.05|0.00|26.31|27.36|-153.99| +2451177|36825|16174|19752|541968|5784|27600|8|86|2196|23|73.32|142.24|22.75|0.00|523.25|1686.36|3271.52|20.93|0.00|523.25|544.18|-1163.11| +2451177|36825|8582|19752|541968|5784|27600|8|294|2196|30|9.42|17.80|7.47|192.72|224.10|282.60|534.00|0.94|192.72|31.38|32.32|-251.22| +2451177|36825|10639|19752|541968|5784|27600|8|258|2196|17|58.20|103.59|29.00|310.59|493.00|989.40|1761.03|5.47|310.59|182.41|187.88|-806.99| +2451177|36825|1303|19752|541968|5784|27600|8|240|2196|77|2.24|4.25|0.25|2.31|19.25|172.48|327.25|1.01|2.31|16.94|17.95|-155.54| +2451177|36825|15034|19752|541968|5784|27600|8|70|2196|74|21.05|26.10|19.05|0.00|1409.70|1557.70|1931.40|14.09|0.00|1409.70|1423.79|-148.00| +2451177||16916|19752|541968|5784|27600||28|2196||87.60|161.18||0.00|3836.05||13700.30||0.00||3912.77|-3609.95| +2451177|36825|15763|19752|541968|5784|27600|8|248|2196|49|84.23|97.70|0.97|0.00|47.53|4127.27|4787.30|0.00|0.00|47.53|47.53|-4079.74| +||2985||||18904|8||2197||5.13||||352.75|||14.11||352.75|366.86|-73.04| +2452294|53674|11250|29102|1830636|4170|18904|8|49|2197|14|15.54|15.69|14.27|83.90|199.78|217.56|219.66|3.47|83.90|115.88|119.35|-101.68| +2452294|53674|9522|29102|1830636|4170|18904|8|132|2197|5|86.07|154.92|37.18|0.00|185.90|430.35|774.60|3.71|0.00|185.90|189.61|-244.45| +2452294|53674|6831|29102|1830636|4170|18904|8|124|2197|85|51.37|77.56|52.74|0.00|4482.90|4366.45|6592.60|179.31|0.00|4482.90|4662.21|116.45| +2452294|53674|2139|29102|1830636|4170|18904|8|195|2197|41|44.15|56.95|44.42|0.00|1821.22|1810.15|2334.95|127.48|0.00|1821.22|1948.70|11.07| +2452294|53674|181|29102|1830636|4170|18904|8|65|2197|43|29.30|33.69|15.83|0.00|680.69|1259.90|1448.67|13.61|0.00|680.69|694.30|-579.21| +2452294|53674|13045|29102|1830636|4170|18904|8|257|2197|14|62.93|125.23|2.50|0.00|35.00|881.02|1753.22|1.75|0.00|35.00|36.75|-846.02| +2452294|53674|10155|29102|1830636|4170|18904|8|132|2197|67|41.25|63.93|23.01|0.00|1541.67|2763.75|4283.31|0.00|0.00|1541.67|1541.67|-1222.08| +2452294|53674|15219|29102|1830636|4170|18904|8|134|2197|18|63.40|106.51|90.53|0.00|1629.54|1141.20|1917.18|0.00|0.00|1629.54|1629.54|488.34| +2452294|53674|7393|29102|1830636|4170|18904|8|260|2197|79|79.51|155.04|23.25|0.00|1836.75|6281.29|12248.16|128.57|0.00|1836.75|1965.32|-4444.54| +2452294|53674|6445|29102|1830636|4170|18904|8|49|2197|93|23.98|24.69|24.44|0.00|2272.92|2230.14|2296.17|68.18|0.00|2272.92|2341.10|42.78| +2452294|53674|15231|29102|1830636|4170|18904|8|47|2197|66|32.01|61.77|18.53|0.00|1222.98|2112.66|4076.82|85.60|0.00|1222.98|1308.58|-889.68| +2452294|53674|9853|29102|1830636|4170|18904|8|135|2197|88|77.11|94.07|39.50|0.00|3476.00|6785.68|8278.16|173.80|0.00|3476.00|3649.80|-3309.68| +2451066|65775|5734|89327|1750848|1169|27490|1|48|2198|54|46.60|79.68|21.51|0.00|1161.54|2516.40|4302.72|11.61|0.00|1161.54|1173.15|-1354.86| +2451066|65775|17485|89327|1750848|1169|27490|1|293|2198|79|28.71|41.62|11.23|0.00|887.17|2268.09|3287.98|70.97|0.00|887.17|958.14|-1380.92| +2451066|65775|3871|89327|1750848|1169|27490|1|20|2198|51|75.30|78.31|51.68|0.00|2635.68|3840.30|3993.81|52.71|0.00|2635.68|2688.39|-1204.62| +2451066|65775|6412|89327|1750848|1169|27490|1|64|2198|57|41.40|75.76|38.63|0.00|2201.91|2359.80|4318.32|132.11|0.00|2201.91|2334.02|-157.89| +2451066|65775|5848|89327|1750848|1169|27490|1|127|2198|73|35.88|61.35|52.76|0.00|3851.48|2619.24|4478.55|346.63|0.00|3851.48|4198.11|1232.24| +2451066|65775|11468|89327|1750848|1169|27490|1|147|2198|63|53.47|95.71|46.89|0.00|2954.07|3368.61|6029.73|206.78|0.00|2954.07|3160.85|-414.54| +2451066|65775|1784|89327|1750848|1169|27490|1|113|2198|14|15.96|27.77|16.66|0.00|233.24|223.44|388.78|18.65|0.00|233.24|251.89|9.80| +2451066|65775|9818|89327|1750848|1169|27490|1|139|2198|76|59.59|72.10|3.60|0.00|273.60|4528.84|5479.60|5.47|0.00|273.60|279.07|-4255.24| +2451066|65775|10982|89327|1750848|1169|27490|1|195|2198|92|14.16|20.53|9.85|0.00|906.20|1302.72|1888.76|27.18|0.00|906.20|933.38|-396.52| +2451066|65775|4861|89327|1750848|1169|27490|1|91|2198|74|15.16|15.31|3.06|0.00|226.44|1121.84|1132.94|11.32|0.00|226.44|237.76|-895.40| +2451066|65775|8419|89327|1750848|1169|27490|1|182|2198|89|79.31|112.62|61.94|0.00|5512.66|7058.59|10023.18|385.88|0.00|5512.66|5898.54|-1545.93| +2451066|65775|8342|89327|1750848|1169|27490|1|280|2198|67|57.50|102.92|15.43|0.00|1033.81|3852.50|6895.64|51.69|0.00|1033.81|1085.50|-2818.69| +2451066|65775|7777|89327|1750848|1169|27490|1|62|2198|87|23.59|24.06|6.01|0.00|522.87|2052.33|2093.22|47.05|0.00|522.87|569.92|-1529.46| +2451066||6620|||||||2198|1||106.90|91.93|0.00|91.93||106.90||0.00||100.20|-6.15| +2451066|65775|8048|89327|1750848|1169|27490|1|55|2198|80|63.68|80.87|28.30|0.00|2264.00|5094.40|6469.60|0.00|0.00|2264.00|2264.00|-2830.40| +2451066|65775|7744|89327|1750848|1169|27490|1|300|2198|90|44.11|68.37|47.85|0.00|4306.50|3969.90|6153.30|387.58|0.00|4306.50|4694.08|336.60| +2452025|47112|2811|2833|1271308|2630|8705|10|83|2199|19|11.21|11.77|2.00|0.00|38.00|212.99|223.63|2.66|0.00|38.00|40.66|-174.99| +2452025|47112|943|2833|1271308|2630|8705|10|240|2199|11|96.32|106.91|41.69|0.00|458.59|1059.52|1176.01|27.51|0.00|458.59|486.10|-600.93| +2452025|47112|6019|2833|1271308|2630|8705|10|143|2199|16|16.29|31.76|5.08|0.00|81.28|260.64|508.16|4.87|0.00|81.28|86.15|-179.36| +2452025|47112|14241|2833|1271308|2630|8705|10|177|2199|29|23.67|46.86|32.33|0.00|937.57|686.43|1358.94|46.87|0.00|937.57|984.44|251.14| +2452025|47112|717|2833|1271308|2630|8705|10|288|2199|39|22.35|29.50|12.68|0.00|494.52|871.65|1150.50|34.61|0.00|494.52|529.13|-377.13| +||11605||1271308|2630||10||2199||90.30||52.08||4687.20|||||4687.20||| +2452025|47112|85|2833|1271308|2630|8705|10|53|2199|98|18.99|37.41|19.82|1553.88|1942.36|1861.02|3666.18|11.65|1553.88|388.48|400.13|-1472.54| +2452025|47112|7767|2833|1271308|2630|8705|10|87|2199|32|46.51|57.20|30.31|0.00|969.92|1488.32|1830.40|48.49|0.00|969.92|1018.41|-518.40| +2452025|47112|9605|2833|1271308|2630|8705|10|81|2199|4|3.66|5.12|0.00|0.00|0.00|14.64|20.48|0.00|0.00|0.00|0.00|-14.64| +2452025|47112|4003|2833|1271308|2630|8705|10|126|2199|61|61.38|115.39|110.77|0.00|6756.97|3744.18|7038.79|405.41|0.00|6756.97|7162.38|3012.79| +2452025|47112|14483|2833|1271308|2630|8705|10|300|2199|59|46.19|52.65|1.05|0.00|61.95|2725.21|3106.35|0.61|0.00|61.95|62.56|-2663.26| +2452025|47112|17471|2833|1271308|2630|8705|10|276|2199|84|14.14|26.72|23.51|1895.84|1974.84|1187.76|2244.48|3.95|1895.84|79.00|82.95|-1108.76| +2452025|47112|5583|2833|1271308|2630|8705|10|180|2199|66|28.05|33.94|23.07|0.00|1522.62|1851.30|2240.04|121.80|0.00|1522.62|1644.42|-328.68| +2452025|47112|41|2833|1271308|2630|8705|10|247|2199|92|74.71|135.22|63.55|0.00|5846.60|6873.32|12440.24|58.46|0.00|5846.60|5905.06|-1026.72| +2452025|47112|13031|2833|1271308|2630|8705|10|196|2199|91|85.73|100.30|54.16|0.00|4928.56|7801.43|9127.30|344.99|0.00|4928.56|5273.55|-2872.87| +2451087|46297|9511|38055|1544150|998|13093|1|80|2200|4|23.55|27.31|17.75|0.00|71.00|94.20|109.24|6.39|0.00|71.00|77.39|-23.20| +2451087|46297|16579|38055||998||1||2200|82|53.34|56.00|38.64|||||126.73||||-1205.40| +2451087|46297|6715|38055|1544150|998|13093|1|212|2200|60|92.82|142.01|5.68|0.00|340.80|5569.20|8520.60|20.44|0.00|340.80|361.24|-5228.40| +2451087|46297|6175|38055|1544150|998|13093|1|32|2200|80|51.53|82.96|77.15|0.00|6172.00|4122.40|6636.80|555.48|0.00|6172.00|6727.48|2049.60| +2451087|46297|13786|38055|1544150|998|13093|1|264|2200|41|75.00|135.00|99.90|0.00|4095.90|3075.00|5535.00|122.87|0.00|4095.90|4218.77|1020.90| +2451087|46297|7946||1544150|998|||161|2200|||69.09||0.00|180.27|522.54|621.81||0.00||185.67|| +2451087|46297|3436|38055|1544150|998|13093|1|235|2200|31|57.89|80.46|53.10|0.00|1646.10|1794.59|2494.26|49.38|0.00|1646.10|1695.48|-148.49| +2451087|46297|16076|38055|1544150|998|13093|1|99|2200|75|64.98|117.61|34.10|0.00|2557.50|4873.50|8820.75|127.87|0.00|2557.50|2685.37|-2316.00| +2451087|46297|5833|38055|1544150|998|13093|1|88|2200|99|68.18|115.90|61.42|0.00|6080.58|6749.82|11474.10|486.44|0.00|6080.58|6567.02|-669.24| +2451087|46297|17188|38055|1544150|998|13093|1|127|2200|12|32.94|49.73|20.38|0.00|244.56|395.28|596.76|12.22|0.00|244.56|256.78|-150.72| +2451087|46297|11383|38055|1544150|998|13093|1|138|2200|95|99.05|126.78|120.44|0.00|11441.80|9409.75|12044.10|686.50|0.00|11441.80|12128.30|2032.05| +2451087|46297|9877|38055|1544150|998|13093|1|130|2200|51|4.88|7.07|6.43|0.00|327.93|248.88|360.57|29.51|0.00|327.93|357.44|79.05| +2451087|46297|5210|38055|1544150|998|13093|1|50|2200|37|5.26|5.47|2.62|0.00|96.94|194.62|202.39|4.84|0.00|96.94|101.78|-97.68| +||2659||||35909||175|2201||71.81||49.24|476.64||6319.28||77.12|476.64|||| +||8053|||3565||||2201|62||177.08|104.47|0.00|6477.14|6032.60|10978.96||0.00|6477.14||| +2450994|36046|3472|36924|1652421|3565|35909|10|136|2201|55|6.48|7.51|6.45|0.00|354.75|356.40|413.05|7.09|0.00|354.75|361.84|-1.65| +||1729|36924||3565||10|106|2201|||112.73|||682.00|4627.15|6200.15|20.46||682.00||-3945.15| +2450994|36046|15746|36924|1652421|3565|35909|10|269|2201|30|72.22|132.88|50.49|348.38|1514.70|2166.60|3986.40|11.66|348.38|1166.32|1177.98|-1000.28| +2450994|36046|5246|36924|1652421|3565|35909|10|173|2201|36|17.65|35.30|14.12|0.00|508.32|635.40|1270.80|5.08|0.00|508.32|513.40|-127.08| +2450994|36046|17596|36924|1652421|3565|35909|10|289|2201|34|31.31|41.01|39.36|0.00|1338.24|1064.54|1394.34|93.67|0.00|1338.24|1431.91|273.70| +2450994|36046|3824|36924|1652421|3565|35909|10|178|2201|56|43.34|78.44|25.88|0.00|1449.28|2427.04|4392.64|28.98|0.00|1449.28|1478.26|-977.76| +2450994|36046|2965|36924|1652421|3565|35909|10|109|2201|54|6.42|10.91|0.43|0.00|23.22|346.68|589.14|2.08|0.00|23.22|25.30|-323.46| +2450994|36046|12286|36924|1652421|3565|35909|10|63|2201|91|91.64|174.11|48.75|0.00|4436.25|8339.24|15844.01|133.08|0.00|4436.25|4569.33|-3902.99| +2450994|36046|5810|36924|1652421|3565|35909|10|22|2201|40|28.80|47.52|21.85|0.00|874.00|1152.00|1900.80|34.96|0.00|874.00|908.96|-278.00| +2450994|36046|3610|36924|1652421|3565|35909|10|224|2201|69|2.90|5.22|0.57|0.00|39.33|200.10|360.18|2.35|0.00|39.33|41.68|-160.77| +2451738|63844|6203|26856|430115|2527|986|8|17|2202|76|37.02|69.59|62.63|0.00|4759.88|2813.52|5288.84|333.19|0.00|4759.88|5093.07|1946.36| +2451738|63844|8549|26856|430115|2527|986|8|220|2202|9|13.21|19.81|2.17|17.96|19.53|118.89|178.29|0.10|17.96|1.57|1.67|-117.32| +2451738|63844|3125|26856|430115|2527|986|8|103|2202|79|85.47|129.05|107.11|0.00|8461.69|6752.13|10194.95|169.23|0.00|8461.69|8630.92|1709.56| +2451738|63844|16295|26856|430115|2527|986|8|254|2202|86|93.30|125.02|71.26|0.00|6128.36|8023.80|10751.72|0.00|0.00|6128.36|6128.36|-1895.44| +2451738|63844|11993|26856|430115|2527|986|8|154|2202|63|29.18|49.60|36.70|0.00|2312.10|1838.34|3124.80|208.08|0.00|2312.10|2520.18|473.76| +2451738|63844|4115|26856|430115|2527|986|8|129|2202|18|94.78|117.52|29.38|0.00|528.84|1706.04|2115.36|5.28|0.00|528.84|534.12|-1177.20| +2451738|63844|9578|26856|430115|2527|986|8|242|2202|79|40.28|66.46|11.29|0.00|891.91|3182.12|5250.34|80.27|0.00|891.91|972.18|-2290.21| +2451738|63844|11342|26856|430115|2527|986|8|213|2202|77|14.32|18.61|1.11|0.00|85.47|1102.64|1432.97|2.56|0.00|85.47|88.03|-1017.17| +2451738|63844|17912|26856|430115|2527|986|8|106|2202|35|56.58|87.13|68.83|0.00|2409.05|1980.30|3049.55|48.18|0.00|2409.05|2457.23|428.75| +2451738|63844|6139|26856|430115|2527|986|8|275|2202|51|56.06|67.83|58.33|0.00|2974.83|2859.06|3459.33|89.24|0.00|2974.83|3064.07|115.77| +2451738|63844|5249|26856|430115|2527|986|8|49|2202|62|82.46|102.25|43.96|0.00|2725.52|5112.52|6339.50|27.25|0.00|2725.52|2752.77|-2387.00| +2451738|63844|15167|26856|430115|2527|986|8|120|2202|87|63.00|68.67|27.46|0.00|2389.02|5481.00|5974.29|23.89|0.00|2389.02|2412.91|-3091.98| +||6027||394633||||163|2203||66.83||109.82||4722.26|2873.69|4769.99|330.55||||1848.57| +2452002|68597|13601|96869|394633|397|41714|7|132|2203|89|12.90|13.80|10.35|0.00|921.15|1148.10|1228.20|73.69|0.00|921.15|994.84|-226.95| +2452002|68597|2619|96869|394633|397|41714|7|110|2203|98|98.34|192.74|36.62|0.00|3588.76|9637.32|18888.52|143.55|0.00|3588.76|3732.31|-6048.56| +2452002|68597|15131|96869|394633|397|41714|7|168|2203|56|7.61|12.70|6.35|0.00|355.60|426.16|711.20|10.66|0.00|355.60|366.26|-70.56| +2452002|68597|9801|96869|394633|397|41714|7|249|2203|42|14.33|21.63|4.54|0.00|190.68|601.86|908.46|1.90|0.00|190.68|192.58|-411.18| +2452002|68597|14935|96869|394633|397|41714|7|159|2203|54|7.61|8.59|5.75|167.67|310.50|410.94|463.86|12.85|167.67|142.83|155.68|-268.11| +2452002|68597|9087|96869|394633|397|41714|7|26|2203|96|24.65|35.74|1.42|0.00|136.32|2366.40|3431.04|6.81|0.00|136.32|143.13|-2230.08| +2452002|68597|5313|96869|394633|397|41714|7|160|2203|27|47.79|79.33|11.10|0.00|299.70|1290.33|2141.91|23.97|0.00|299.70|323.67|-990.63| +2452002|68597|1451|96869|394633|397|41714|7|293|2203|20|66.39|111.53|53.53|0.00|1070.60|1327.80|2230.60|32.11|0.00|1070.60|1102.71|-257.20| +2452002|68597|5483|96869|394633|397|41714|7|18|2203|94|52.86|56.56|45.81|0.00|4306.14|4968.84|5316.64|258.36|0.00|4306.14|4564.50|-662.70| +2452002|68597|8715|96869|394633|397|41714|7|238|2203|55|45.38|45.38|39.93|0.00|2196.15|2495.90|2495.90|87.84|0.00|2196.15|2283.99|-299.75| +2452002|68597|7207|96869|394633|397|41714|7|242|2203|4|25.24|28.77|16.97|0.00|67.88|100.96|115.08|3.39|0.00|67.88|71.27|-33.08| +2452002|68597|17037|96869|394633|397|41714|7|178|2203|46|81.42|141.67|28.33|0.00|1303.18|3745.32|6516.82|0.00|0.00|1303.18|1303.18|-2442.14| +2452002|68597|14171|96869|394633|397|41714|7|174|2203|38|21.46|37.34|1.12|0.00|42.56|815.48|1418.92|1.27|0.00|42.56|43.83|-772.92| +2452002|68597|5133|96869|394633|397|41714|7|231|2203|87|53.66|76.73|6.90|0.00|600.30|4668.42|6675.51|0.00|0.00|600.30|600.30|-4068.12| +2452459|51207|6633|53730|101344|958|18125|4|64|2204|3|55.97|92.35|11.08|0.00|33.24|167.91|277.05|2.99|0.00|33.24|36.23|-134.67| +2452459|51207|8383|53730|101344|958|18125|4|270|2204|1|89.09|121.16|4.84|0.00|4.84|89.09|121.16|0.04|0.00|4.84|4.88|-84.25| +2452459|51207|10717|53730|101344|958|18125|4|81|2204|91|31.99|32.62|7.17|0.00|652.47|2911.09|2968.42|39.14|0.00|652.47|691.61|-2258.62| +2452459|51207|11766|53730|101344|958|18125|4|286|2204|32|35.04|69.02|66.94|0.00|2142.08|1121.28|2208.64|64.26|0.00|2142.08|2206.34|1020.80| +2452459|51207|14007|53730|101344|958|18125|4|40|2204|56|62.26|108.33|42.24|0.00|2365.44|3486.56|6066.48|165.58|0.00|2365.44|2531.02|-1121.12| +2452459|51207|1515|53730|101344|958|18125|4|122|2204|100|85.53|122.30|119.85|359.55|11985.00|8553.00|12230.00|116.25|359.55|11625.45|11741.70|3072.45| +2452459|51207|17431|53730|101344|958|18125|4|128|2204|20|50.35|89.11|74.85|0.00|1497.00|1007.00|1782.20|134.73|0.00|1497.00|1631.73|490.00| +2452459|51207|12015|53730|101344|958|18125|4|159|2204|26|62.92|62.92|13.21|319.41|343.46|1635.92|1635.92|1.20|319.41|24.05|25.25|-1611.87| +2452459|51207|8629|53730|101344|958|18125|4|47|2204|11|80.25|153.27|88.89|0.00|977.79|882.75|1685.97|19.55|0.00|977.79|997.34|95.04| +2452459|51207|5365|53730|101344|958|18125|4|119|2204|39|66.36|84.27|54.77|0.00|2136.03|2588.04|3286.53|64.08|0.00|2136.03|2200.11|-452.01| +2452459|51207|9030|53730|101344|958|18125|4|87|2204|30|64.65|77.58|11.63|0.00|348.90|1939.50|2327.40|13.95|0.00|348.90|362.85|-1590.60| +2452459|51207|14299|53730|101344|958|18125|4|73|2204|93|21.00|35.91|10.41|0.00|968.13|1953.00|3339.63|19.36|0.00|968.13|987.49|-984.87| +2452459|51207|17043|53730|101344|958|18125|4|188|2204|4|55.67|101.31|36.47|59.81|145.88|222.68|405.24|1.72|59.81|86.07|87.79|-136.61| +|51207|13423|53730||958|18125|4||2204|18|24.90|34.86|||81.54||627.48|5.70||81.54||-366.66| +2452459|51207|10428|53730|101344|958|18125|4|160|2204|52|60.25|115.68|12.72|0.00|661.44|3133.00|6015.36|6.61|0.00|661.44|668.05|-2471.56| +||12200|10203|1134144|6885|35626|4|163|2205|13|38.60||||377.52||662.35||||399.15|| +2450909||12422|||6885|35626|4||2205|12|16.15|30.03|||205.32||360.36|||205.32|213.53|11.52| +2450909|34362|2101|10203|1134144|6885|35626|4|115|2205|44|76.30|89.27|25.88|0.00|1138.72|3357.20|3927.88|11.38|0.00|1138.72|1150.10|-2218.48| +2450909|34362|15940|10203|1134144|6885|35626|4|94|2205|76|39.12|44.59|43.69|99.61|3320.44|2973.12|3388.84|32.20|99.61|3220.83|3253.03|247.71| +2450909|34362|6584||1134144||35626|4|274|2205|||49.33||0.00||1903.65|||0.00|652.19|678.27|| +2450909|34362|10627|10203|1134144|6885|35626|4|169|2205|68|32.45|36.99|35.14|0.00|2389.52|2206.60|2515.32|191.16|0.00|2389.52|2580.68|182.92| +2450909|34362|4088|10203|1134144|6885|35626|4|9|2205|75|13.76|25.45|17.56|0.00|1317.00|1032.00|1908.75|26.34|0.00|1317.00|1343.34|285.00| +2450909|34362|6062|10203|1134144|6885|35626|4|139|2205|7|4.62|6.05|4.53|0.00|31.71|32.34|42.35|1.26|0.00|31.71|32.97|-0.63| +2450909|34362|5623|10203|1134144|6885|35626|4|44|2205|37|8.23|9.38|6.37|0.00|235.69|304.51|347.06|2.35|0.00|235.69|238.04|-68.82| +2450909|34362|15988|10203|1134144|6885|35626|4|103|2205|56|28.07|49.40|13.83|0.00|774.48|1571.92|2766.40|23.23|0.00|774.48|797.71|-797.44| +2451437|41281|9982|28179|362128|3277|14826|2|82|2206|69|93.81|137.90|81.36|0.00|5613.84|6472.89|9515.10|168.41|0.00|5613.84|5782.25|-859.05| +2451437|41281|6361|28179|362128|3277|14826|2|184|2206|68|10.49|18.98|10.24|0.00|696.32|713.32|1290.64|27.85|0.00|696.32|724.17|-17.00| +2451437|41281|5960|28179|362128|3277|14826|2|174|2206|52|92.88|117.95|51.89|0.00|2698.28|4829.76|6133.40|188.87|0.00|2698.28|2887.15|-2131.48| +2451437|41281|12325|28179|362128|3277|14826|2|54|2206|29|22.87|29.27|9.95|0.00|288.55|663.23|848.83|25.96|0.00|288.55|314.51|-374.68| +2451437|41281|7555|28179|362128|3277|14826|2|161|2206|22|89.63|116.51|110.68|0.00|2434.96|1971.86|2563.22|0.00|0.00|2434.96|2434.96|463.10| +2451437|41281|1364|28179|362128|3277|14826|2|155|2206|8|66.59|112.53|97.90|657.88|783.20|532.72|900.24|1.25|657.88|125.32|126.57|-407.40| +2451437|41281|15007|28179|362128|3277|14826|2|288|2206|53|35.62|70.17|61.74|1177.99|3272.22|1887.86|3719.01|104.71|1177.99|2094.23|2198.94|206.37| +2451437|41281|15205|28179|362128|3277|14826|2|144|2206|62|65.81|111.87|100.68|0.00|6242.16|4080.22|6935.94|187.26|0.00|6242.16|6429.42|2161.94| +2451437|41281|9382|28179|362128|3277|14826|2|41|2206|67|21.98|26.81|6.16|0.00|412.72|1472.66|1796.27|12.38|0.00|412.72|425.10|-1059.94| +2451437|41281|3424|28179|362128|3277|14826|2|284|2206|35|52.93|90.51|89.60|2477.44|3136.00|1852.55|3167.85|26.34|2477.44|658.56|684.90|-1193.99| +2451437|41281|16570|28179|362128|3277|14826|2|291|2206|18|15.97|22.19|12.87|0.00|231.66|287.46|399.42|4.63|0.00|231.66|236.29|-55.80| +2451437|41281|12950|28179|362128|3277|14826|2|102|2206|43|49.14|62.89|45.28|0.00|1947.04|2113.02|2704.27|97.35|0.00|1947.04|2044.39|-165.98| +2452148|38017|3445|63808|1060328|4495|5815|2|222|2207|31|40.42|66.28|8.61|0.00|266.91|1253.02|2054.68|0.00|0.00|266.91|266.91|-986.11| +2452148|38017|13933|63808|1060328|4495|5815|2|166|2207|65|60.72|80.75|66.21|0.00|4303.65|3946.80|5248.75|258.21|0.00|4303.65|4561.86|356.85| +2452148|38017|11589|63808|1060328|4495|5815|2|3|2207|37|57.46|95.38|0.00|0.00|0.00|2126.02|3529.06|0.00|0.00|0.00|0.00|-2126.02| +2452148|38017|15171|63808|1060328|4495|5815|2|187|2207|100|7.78|9.49|2.18|0.00|218.00|778.00|949.00|13.08|0.00|218.00|231.08|-560.00| +2452148|38017|6833|63808|1060328|4495|5815|2|43|2207|80|74.47|111.70|36.86|2329.55|2948.80|5957.60|8936.00|6.19|2329.55|619.25|625.44|-5338.35| +2452148|38017|13159|63808|1060328|4495|5815|2|77|2207|72|35.71|55.35|52.58|0.00|3785.76|2571.12|3985.20|113.57|0.00|3785.76|3899.33|1214.64| +2452148|38017|1215|63808|1060328|4495|5815|2|268|2207|82|64.13|128.26|116.71|0.00|9570.22|5258.66|10517.32|382.80|0.00|9570.22|9953.02|4311.56| +2452148|38017|14949|63808|1060328|4495|5815|2|256|2207|20|85.26|117.65|49.41|0.00|988.20|1705.20|2353.00|59.29|0.00|988.20|1047.49|-717.00| +2452148|38017|17239|63808|1060328|4495|5815|2|251|2207|57|45.33|65.72|34.83|0.00|1985.31|2583.81|3746.04|59.55|0.00|1985.31|2044.86|-598.50| +2452148|38017|3749|63808|1060328|4495|5815|2|140|2207|94|83.86|138.36|26.28|2247.99|2470.32|7882.84|13005.84|13.33|2247.99|222.33|235.66|-7660.51| +2452148|38017|11593|63808|1060328|4495|5815|2|219|2207|97|25.22|30.51|28.37|0.00|2751.89|2446.34|2959.47|82.55|0.00|2751.89|2834.44|305.55| +2452148|38017|1339|63808|1060328|4495|5815|2|194|2207|70|66.10|115.67|108.72|0.00|7610.40|4627.00|8096.90|608.83|0.00|7610.40|8219.23|2983.40| +2452148|38017|14923|63808|1060328|4495|5815|2|208|2207|96|1.91|3.62|1.91|0.00|183.36|183.36|347.52|3.66|0.00|183.36|187.02|0.00| +2452148|38017|12205|63808|1060328|4495|5815|2|84|2207|32|53.20|80.33|5.62|10.79|179.84|1702.40|2570.56|10.14|10.79|169.05|179.19|-1533.35| +2452148|38017|12171|63808|1060328|4495|5815|2|29|2207|79|59.96|74.35|26.02|0.00|2055.58|4736.84|5873.65|164.44|0.00|2055.58|2220.02|-2681.26| +2452148|38017|5137|63808|1060328|4495|5815|2|233|2207|98|71.34|99.87|26.96|0.00|2642.08|6991.32|9787.26|237.78|0.00|2642.08|2879.86|-4349.24| +2450892|33911|12814|84352|1053310|4668|16026|2|57|2208|91|14.81|24.14|2.89|186.72|262.99|1347.71|2196.74|1.52|186.72|76.27|77.79|-1271.44| +2450892||697|||4668|16026|||2208|68||21.67|7.80||530.40||||||535.70|-620.84| +2450892|33911|17048|84352|1053310|4668|16026|2|203|2208|70|59.72|68.67|6.18|0.00|432.60|4180.40|4806.90|34.60|0.00|432.60|467.20|-3747.80| +2450892|33911|2254|84352|1053310|4668|16026|2|134|2208|63|26.94|29.63|10.96|0.00|690.48|1697.22|1866.69|48.33|0.00|690.48|738.81|-1006.74| +2450892|33911|3364|84352|1053310|4668|16026|2|128|2208|19|49.32|53.26|18.64|0.00|354.16|937.08|1011.94|3.54|0.00|354.16|357.70|-582.92| +2450892|33911|1882|84352|1053310|4668|16026|2|17|2208|95|68.46|73.93|64.31|0.00|6109.45|6503.70|7023.35|0.00|0.00|6109.45|6109.45|-394.25| +2450892|33911|6368|84352|1053310|4668|16026|2|5|2208|36|62.54|86.93|32.16|0.00|1157.76|2251.44|3129.48|57.88|0.00|1157.76|1215.64|-1093.68| +2450892|33911|13087|84352|1053310|4668|16026|2|74|2208|50|42.75|65.83|62.53|0.00|3126.50|2137.50|3291.50|187.59|0.00|3126.50|3314.09|989.00| +2450892|33911|808|84352|1053310|4668|16026|2|292|2208|84|17.65|32.12|17.66|0.00|1483.44|1482.60|2698.08|89.00|0.00|1483.44|1572.44|0.84| +2450892|33911|5158|84352|1053310|4668|16026|2|138|2208|25|19.78|35.99|11.87|127.60|296.75|494.50|899.75|15.22|127.60|169.15|184.37|-325.35| +2452256|51068|10835|37764|83639|3467|7342|7|183|2209|9|91.28|164.30|151.15|0.00|1360.35|821.52|1478.70|108.82|0.00|1360.35|1469.17|538.83| +2452256||17747||83639||||14|2209|5|96.00|121.92|||||609.60|8.96|||307.66|-181.30| +2452256|51068|11629|37764|83639|3467|7342|7|91|2209|4|56.23|72.53|44.96|0.00|179.84|224.92|290.12|10.79|0.00|179.84|190.63|-45.08| +2452256|51068|11079|37764|83639|3467|7342|7|144|2209|2|20.40|35.08|3.15|0.00|6.30|40.80|70.16|0.50|0.00|6.30|6.80|-34.50| +2452256|51068|2843|37764|83639|3467|7342|7|184|2209|9|23.41|46.82|3.27|27.66|29.43|210.69|421.38|0.10|27.66|1.77|1.87|-208.92| +2452256|51068|3671|37764|83639|3467|7342|7|38|2209|100|92.50|184.07|108.60|0.00|10860.00|9250.00|18407.00|977.40|0.00|10860.00|11837.40|1610.00| +2452256|51068|3389|37764|83639|3467|7342|7|150|2209|97|14.44|28.30|20.94|0.00|2031.18|1400.68|2745.10|0.00|0.00|2031.18|2031.18|630.50| +2452256|51068|12925|37764|83639|3467|7342|7|180|2209|99|78.44|98.83|58.30|0.00|5771.70|7765.56|9784.17|519.45|0.00|5771.70|6291.15|-1993.86| +2452256|51068|5857|37764|83639|3467|7342|7|246|2209|56|79.98|151.96|80.53|0.00|4509.68|4478.88|8509.76|135.29|0.00|4509.68|4644.97|30.80| +2452256|51068|16199|37764|83639|3467|7342|7|92|2209|87|14.12|27.81|16.68|0.00|1451.16|1228.44|2419.47|14.51|0.00|1451.16|1465.67|222.72| +2450890|46833|16660|16768|479444|3720|45996|10|51|2210|11|33.15|40.77|40.36|0.00|443.96|364.65|448.47|13.31|0.00|443.96|457.27|79.31| +2450890|46833|2104|16768|479444|3720|45996|10|183|2210|50|95.26|114.31|4.57|0.00|228.50|4763.00|5715.50|2.28|0.00|228.50|230.78|-4534.50| +2450890|46833|14990|16768|479444|3720|45996|10|12|2210|27|92.00|166.52|163.18|0.00|4405.86|2484.00|4496.04|132.17|0.00|4405.86|4538.03|1921.86| +2450890|46833|4771|16768|479444|3720|45996|10|97|2210|65|98.82|179.85|84.52|0.00|5493.80|6423.30|11690.25|54.93|0.00|5493.80|5548.73|-929.50| +2450890|46833|3991|16768|479444|3720|45996|10|217|2210|65|87.80|120.28|42.09|0.00|2735.85|5707.00|7818.20|82.07|0.00|2735.85|2817.92|-2971.15| +2450890|46833|9184||479444|||10||2210|15||||||519.30|893.10|24.99||312.45|337.44|-206.85| +2450890|46833|4294|16768|479444|3720|45996|10|248|2210|43|14.96|17.35|15.26|0.00|656.18|643.28|746.05|19.68|0.00|656.18|675.86|12.90| +2450890|46833|16525|16768|479444|3720|45996|10|203|2210|94|5.65|11.24|6.18|429.88|580.92|531.10|1056.56|4.53|429.88|151.04|155.57|-380.06| +2450890|46833|2116|16768|479444|3720|45996|10|214|2210|41|19.70|33.29|25.63|0.00|1050.83|807.70|1364.89|63.04|0.00|1050.83|1113.87|243.13| +2450890|46833|11768|16768|479444|3720|45996|10|108|2210|7|69.40|71.48|33.59|0.00|235.13|485.80|500.36|0.00|0.00|235.13|235.13|-250.67| +2450890|46833|3692|16768|479444|3720|45996|10|85|2210|87|14.97|20.20|7.07|0.00|615.09|1302.39|1757.40|0.00|0.00|615.09|615.09|-687.30| +2450890|46833|16729|16768|479444|3720|45996|10|88|2210|29|32.96|46.14|45.67|0.00|1324.43|955.84|1338.06|39.73|0.00|1324.43|1364.16|368.59| +2450890|46833|14707|16768|479444|3720|45996|10|219|2210|50|83.90|133.40|45.35|0.00|2267.50|4195.00|6670.00|181.40|0.00|2267.50|2448.90|-1927.50| +2450890|46833|3656|16768|479444|3720|45996|10|144|2210|88|7.38|10.55|1.05|0.00|92.40|649.44|928.40|7.39|0.00|92.40|99.79|-557.04| +2450890|46833|7093|16768|479444|3720|45996|10|78|2210|89|27.07|27.88|2.50|0.00|222.50|2409.23|2481.32|8.90|0.00|222.50|231.40|-2186.73| +2452234|36195|5217|39209|1373154|5052|14150|8|227|2211|72|98.82|101.78|80.40|0.00|5788.80|7115.04|7328.16|173.66|0.00|5788.80|5962.46|-1326.24| +2452234|36195|4487|39209|1373154|5052|14150|8|126|2211|4|8.68|9.28|3.15|0.00|12.60|34.72|37.12|0.88|0.00|12.60|13.48|-22.12| +2452234|36195|15793|39209|1373154|5052|14150|8|228|2211|95|16.39|24.91|24.41|0.00|2318.95|1557.05|2366.45|185.51|0.00|2318.95|2504.46|761.90| +2452234|36195|5657|39209|1373154|5052|14150|8|7|2211|54|58.14|70.93|29.08|62.81|1570.32|3139.56|3830.22|135.67|62.81|1507.51|1643.18|-1632.05| +2452234|36195|8919|39209|1373154|5052|14150|8|227|2211|92|77.52|138.76|73.54|0.00|6765.68|7131.84|12765.92|338.28|0.00|6765.68|7103.96|-366.16| +2452234|36195|7167|39209|1373154|5052|14150|8|142|2211|36|9.67|18.27|4.75|0.00|171.00|348.12|657.72|5.13|0.00|171.00|176.13|-177.12| +2452234|36195|11107|39209|1373154|5052|14150|8|3|2211|97|92.03|92.95|5.57|0.00|540.29|8926.91|9016.15|27.01|0.00|540.29|567.30|-8386.62| +2452234|36195|7663|39209|1373154|5052|14150|8|260|2211|53|17.76|19.35|13.54|0.00|717.62|941.28|1025.55|28.70|0.00|717.62|746.32|-223.66| +2452234|36195|13575|39209|1373154|5052|14150|8|16|2211|94|56.70|100.92|16.14|0.00|1517.16|5329.80|9486.48|91.02|0.00|1517.16|1608.18|-3812.64| +2452234|36195|10481|39209|1373154|5052|14150|8|167|2211|33|58.92|80.72|4.03|0.00|132.99|1944.36|2663.76|11.96|0.00|132.99|144.95|-1811.37| +2452234|36195|10849|39209|1373154|5052|14150|8|212|2211|82|64.87|112.22|53.86|0.00|4416.52|5319.34|9202.04|264.99|0.00|4416.52|4681.51|-902.82| +2452234|36195|16853|39209|1373154|5052|14150|8|129|2211|36|30.74|52.56|35.21|342.24|1267.56|1106.64|1892.16|27.75|342.24|925.32|953.07|-181.32| +2452234|36195|13143|39209|1373154|5052|14150|8|113|2211|96|78.41|141.13|122.78|0.00|11786.88|7527.36|13548.48|471.47|0.00|11786.88|12258.35|4259.52| +|36195|303|||||||2211|71|62.91|65.42|57.56|0.00|4086.76|4466.61|4644.82||0.00|4086.76||-379.85| +2452234|36195|4455|39209|1373154|5052|14150|8|193|2211|20|38.26|74.60|64.90|0.00|1298.00|765.20|1492.00|64.90|0.00|1298.00|1362.90|532.80| +2452234|36195|7045|39209|1373154|5052|14150|8|253|2211|59|93.65|144.22|15.86|0.00|935.74|5525.35|8508.98|46.78|0.00|935.74|982.52|-4589.61| +2451543|34883|14648|30782|1297943|4304|9818|4|81|2212|93|19.13|36.53|8.76|814.68|814.68|1779.09|3397.29|0.00|814.68|0.00|0.00|-1779.09| +2451543|34883|14878|30782|1297943|4304|9818|4|257|2212|52|41.43|77.47|65.84|0.00|3423.68|2154.36|4028.44|0.00|0.00|3423.68|3423.68|1269.32| +2451543|34883|6979|30782|1297943|4304|9818|4|271|2212|47|15.51|19.54|10.74|0.00|504.78|728.97|918.38|15.14|0.00|504.78|519.92|-224.19| +2451543|34883|16966|30782|1297943|4304|9818|4|108|2212|3|27.22|32.66|22.20|0.00|66.60|81.66|97.98|3.99|0.00|66.60|70.59|-15.06| +2451543|34883|6118|30782|1297943|4304|9818|4|95|2212|94|80.40|130.24|9.11|0.00|856.34|7557.60|12242.56|77.07|0.00|856.34|933.41|-6701.26| +2451543|34883|8542|30782|1297943|4304|9818|4|119|2212|53|45.01|83.71|27.62|0.00|1463.86|2385.53|4436.63|58.55|0.00|1463.86|1522.41|-921.67| +2451543|34883|674|30782|1297943|4304|9818|4|59|2212|31|6.50|7.80|3.51|0.00|108.81|201.50|241.80|2.17|0.00|108.81|110.98|-92.69| +2451543|34883|8563|30782|1297943|4304|9818|4|52|2212|26|16.95|27.79|17.22|0.00|447.72|440.70|722.54|8.95|0.00|447.72|456.67|7.02| +2451543|34883|1954|30782|1297943|4304|9818|4|4|2212|89|42.63|63.94|30.69|0.00|2731.41|3794.07|5690.66|54.62|0.00|2731.41|2786.03|-1062.66| +2451205|42596|7850|37888|370035|3782|43529|2|146|2213|22|24.55|34.12|27.97|0.00|615.34|540.10|750.64|36.92|0.00|615.34|652.26|75.24| +2451205|42596|17641|37888|370035|3782|43529|2|137|2213|83|42.88|84.04|77.31|0.00|6416.73|3559.04|6975.32|385.00|0.00|6416.73|6801.73|2857.69| +2451205|42596|4747|37888|370035|3782|43529|2|145|2213|8|35.01|36.76|15.43|0.00|123.44|280.08|294.08|3.70|0.00|123.44|127.14|-156.64| +2451205|42596|3064|37888|370035|3782|43529|2|168|2213|61|11.91|22.27|2.00|0.00|122.00|726.51|1358.47|1.22|0.00|122.00|123.22|-604.51| +2451205|42596|12866|37888|370035|3782|43529|2|260|2213|91|85.44|138.41|135.64|0.00|12343.24|7775.04|12595.31|617.16|0.00|12343.24|12960.40|4568.20| +2451205|42596|4105|37888|370035|3782|43529|2|68|2213|5|53.19|79.78|19.14|0.00|95.70|265.95|398.90|8.61|0.00|95.70|104.31|-170.25| +2451205||3913|37888|370035|3782|43529|2||2213||32.81|||3.27|12.60||||3.27|||| +2451205|42596|10021|37888|370035|3782|43529|2|269|2213|82|93.80|170.71|143.39|0.00|11757.98|7691.60|13998.22|235.15|0.00|11757.98|11993.13|4066.38| +2451205|42596|6374|37888|370035|3782|43529|2|265|2213|81|77.72|125.12|16.26|0.00|1317.06|6295.32|10134.72|13.17|0.00|1317.06|1330.23|-4978.26| +2451205|42596|5701|37888|370035|3782|43529|2|135|2213|78|88.67|91.33|32.87|0.00|2563.86|6916.26|7123.74|0.00|0.00|2563.86|2563.86|-4352.40| +2451205|42596|3604|37888|370035|3782|43529|2|171|2213|33|89.72|104.07|37.46|927.13|1236.18|2960.76|3434.31|12.36|927.13|309.05|321.41|-2651.71| +2451205|42596|10298|37888||3782|43529|2||2213||||37.34|0.00|3248.58||||0.00|3248.58|3313.55|| +2452604|71524|17832|13679|1879360|5488|42816|2|136|2214|89|68.53|93.88|53.51|0.00|4762.39|6099.17|8355.32|428.61|0.00|4762.39|5191.00|-1336.78| +2452604|71524|17145|13679|1879360|5488|42816|2|40|2214|38|78.19|122.75|8.59|0.00|326.42|2971.22|4664.50|16.32|0.00|326.42|342.74|-2644.80| +2452604|71524|16038|13679|1879360|5488|42816|2|131|2214|100|53.91|98.65|51.29|4154.49|5129.00|5391.00|9865.00|38.98|4154.49|974.51|1013.49|-4416.49| +2452604|71524|12265|13679|1879360|5488|42816|2|92|2214|30|71.51|77.94|41.30|0.00|1239.00|2145.30|2338.20|99.12|0.00|1239.00|1338.12|-906.30| +2452604|71524|7309|13679|1879360|5488|42816|2|254|2214|89|68.11|111.70|37.97|0.00|3379.33|6061.79|9941.30|304.13|0.00|3379.33|3683.46|-2682.46| +2452604|71524|12409|13679|1879360|5488|42816|2|173|2214|99|99.19|104.14|34.36|0.00|3401.64|9819.81|10309.86|204.09|0.00|3401.64|3605.73|-6418.17| +2452604|71524|5569|13679|1879360|5488|42816|2|42|2214|75|84.88|131.56|60.51|0.00|4538.25|6366.00|9867.00|272.29|0.00|4538.25|4810.54|-1827.75| +2452604|71524|9559|13679|1879360|5488|42816|2|123|2214|79|31.54|36.27|28.65|0.00|2263.35|2491.66|2865.33|45.26|0.00|2263.35|2308.61|-228.31| +|71524|16789||1879360||42816||288|2214|77|23.18|||0.00||1784.86|2391.62|35.86|0.00|1793.33|1829.19|8.47| +2452604|71524|13113|13679|1879360|5488|42816|2|12|2214|3|36.26|71.06|51.16|0.00|153.48|108.78|213.18|7.67|0.00|153.48|161.15|44.70| +|71524|133|13679|1879360|5488|42816|2|189|2214||80.94||77.36|0.00|1237.76||1566.88||0.00||1299.64|| +2452604|71524|16393|13679|1879360|5488|42816|2|128|2214|98|8.21|15.02|9.31|0.00|912.38|804.58|1471.96|9.12|0.00|912.38|921.50|107.80| +2452604|71524|9943|13679|1879360|5488|42816|2|271|2214|30|86.75|134.46|9.41|197.61|282.30|2602.50|4033.80|7.62|197.61|84.69|92.31|-2517.81| +2452604|71524|553|13679|1879360|5488|42816|2|262|2214|37|86.00|148.78|10.41|0.00|385.17|3182.00|5504.86|3.85|0.00|385.17|389.02|-2796.83| +2452604|71524|5490|13679|1879360|5488|42816|2|88|2214|51|58.87|104.19|23.96|0.00|1221.96|3002.37|5313.69|12.21|0.00|1221.96|1234.17|-1780.41| +2451040|44609|16441|16039|403645|3698|30504|4|243|2215|29|33.63|48.76|27.79|0.00|805.91|975.27|1414.04|40.29|0.00|805.91|846.20|-169.36| +2451040|44609|11408|16039|403645|3698|30504|4|12|2215|56|42.94|72.13|0.72|0.00|40.32|2404.64|4039.28|0.80|0.00|40.32|41.12|-2364.32| +2451040|44609|15529|16039|403645|3698|30504|4|151|2215|61|9.76|18.44|13.83|0.00|843.63|595.36|1124.84|0.00|0.00|843.63|843.63|248.27| +2451040||6313|16039||3698||4|264|2215|||50.87|39.16|715.06|861.52|||7.32|715.06|146.46|153.78|-826.82| +2451040|44609|4042|16039|403645|3698|30504|4|205|2215|30|3.82|6.30|3.27|0.00|98.10|114.60|189.00|7.84|0.00|98.10|105.94|-16.50| +2451040|44609|5599|16039|403645|3698|30504|4|9|2215|31|86.55|137.61|92.19|0.00|2857.89|2683.05|4265.91|228.63|0.00|2857.89|3086.52|174.84| +2451040|44609|17749|16039|403645|3698|30504|4|237|2215|31|75.47|104.14|58.31|0.00|1807.61|2339.57|3228.34|54.22|0.00|1807.61|1861.83|-531.96| +2451040|44609|10828|16039|403645|3698|30504|4|195|2215|12|39.84|65.73|38.12|0.00|457.44|478.08|788.76|36.59|0.00|457.44|494.03|-20.64| +2452268|71742|3537|12761|404411|2461|23711|2|74|2216|39|90.96|173.73|1.73|0.00|67.47|3547.44|6775.47|3.37|0.00|67.47|70.84|-3479.97| +2452268|71742|15427|12761|404411|2461|23711|2|294|2216|83|73.89|97.53|10.72|0.00|889.76|6132.87|8094.99|62.28|0.00|889.76|952.04|-5243.11| +2452268|71742|12575|12761|404411|2461|23711|2|95|2216|49|15.53|28.41|21.30|0.00|1043.70|760.97|1392.09|73.05|0.00|1043.70|1116.75|282.73| +2452268|71742|3899|12761|404411|2461|23711|2|225|2216|29|95.00|164.35|147.91|0.00|4289.39|2755.00|4766.15|300.25|0.00|4289.39|4589.64|1534.39| +2452268|71742|523|12761|404411|2461|23711|2|236|2216|11|57.10|58.24|16.30|0.00|179.30|628.10|640.64|7.17|0.00|179.30|186.47|-448.80| +2452268|71742|361|12761|404411|2461|23711|2|37|2216|48|7.64|13.29|3.05|0.00|146.40|366.72|637.92|13.17|0.00|146.40|159.57|-220.32| +2452268|71742|4465|12761|404411|2461|23711|2|103|2216|23|58.91|76.58|19.91|430.45|457.93|1354.93|1761.34|0.82|430.45|27.48|28.30|-1327.45| +|71742|9427|12761|404411|2461|||150|2216|55|82.89|144.22||0.00|||7932.10|477.47|0.00||7298.57|| +2452268|71742|16009|12761|404411|2461|23711|2|108|2216|45|4.33|7.40|2.22|0.00|99.90|194.85|333.00|1.99|0.00|99.90|101.89|-94.95| +2451929|69058|6941|31646|92143|1887|22784|10|85|2217|33|7.15|10.43|3.12|0.00|102.96|235.95|344.19|5.14|0.00|102.96|108.10|-132.99| +2451929|69058|4461|31646|92143|1887|22784|10|203|2217|74|73.49|97.00|0.00|0.00|0.00|5438.26|7178.00|0.00|0.00|0.00|0.00|-5438.26| +2451929|69058|15465|31646|92143|1887|22784|10|74|2217|5|94.88|175.52|103.55|0.00|517.75|474.40|877.60|36.24|0.00|517.75|553.99|43.35| +2451929|69058|783|31646|92143|1887|22784|10|55|2217|2|69.93|86.71|65.03|0.00|130.06|139.86|173.42|6.50|0.00|130.06|136.56|-9.80| +2451929|69058|7059|31646|92143|1887|22784|10|70|2217|92|76.35|145.82|74.36|0.00|6841.12|7024.20|13415.44|615.70|0.00|6841.12|7456.82|-183.08| +|69058|13379||||22784|10|22|2217||84.04||||2664.00|3109.48||||||| +2451929||10965|31646|92143|1887|22784|10||2217||20.18|28.65||0.00|77.10|||0.77|0.00|77.10|77.87|| +|69058|6493|31646||1887|22784|||2217|||157.96|102.67|5183.80||4331.43|8055.96|4.18|5183.80|52.37||-4279.06| +2451929|69058|13055|31646||1887|22784|10||2217||||||1483.90|6667.10||48.82||697.44||-5969.66| +2452546|70053|5761|30140|586619|6298|18377|4|293|2218|33|73.76|124.65|64.81|0.00|2138.73|2434.08|4113.45|106.93|0.00|2138.73|2245.66|-295.35| +2452546|70053|8292|30140|586619|6298|18377|4|154|2218|3|22.41|23.53|21.88|29.53|65.64|67.23|70.59|1.80|29.53|36.11|37.91|-31.12| +2452546|70053|14389|30140|586619|6298|18377|4|241|2218|21|8.15|10.18|0.30|0.00|6.30|171.15|213.78|0.18|0.00|6.30|6.48|-164.85| +2452546|70053|3673|30140|586619|6298|18377|4|200|2218|75|92.60|134.27|14.76|0.00|1107.00|6945.00|10070.25|55.35|0.00|1107.00|1162.35|-5838.00| +2452546|70053|8479|30140|586619|6298|18377|4|261|2218|50|1.54|2.24|2.19|0.00|109.50|77.00|112.00|9.85|0.00|109.50|119.35|32.50| +2452546|70053|16722|30140|586619|6298|18377|4|198|2218|7|78.95|112.10|67.26|0.00|470.82|552.65|784.70|28.24|0.00|470.82|499.06|-81.83| +2452546|70053|861|30140|586619|6298|18377|4|167|2218|56|90.23|157.00|81.64|2240.20|4571.84|5052.88|8792.00|23.31|2240.20|2331.64|2354.95|-2721.24| +2452546|70053|2053|30140|586619|6298|18377|4|242|2218|60|81.45|152.31|65.49|0.00|3929.40|4887.00|9138.60|117.88|0.00|3929.40|4047.28|-957.60| +2452546|70053|13260|30140|586619|6298|18377|4|53|2218|76|11.98|23.36|2.56|0.00|194.56|910.48|1775.36|3.89|0.00|194.56|198.45|-715.92| +2452546|70053|12621|30140|586619|6298|18377|4|24|2218|86|57.75|75.65|49.17|0.00|4228.62|4966.50|6505.90|0.00|0.00|4228.62|4228.62|-737.88| +2452546|70053|4153|30140|586619|6298|18377|4|111|2218|80|85.85|156.24|143.74|0.00|11499.20|6868.00|12499.20|804.94|0.00|11499.20|12304.14|4631.20| +2452546|70053|17838|30140|586619|6298|18377|4|190|2218|26|41.92|43.17|2.15|0.00|55.90|1089.92|1122.42|0.00|0.00|55.90|55.90|-1034.02| +2452546|70053|16983|30140|586619|6298|18377|4|136|2218|83|51.82|83.43|16.68|0.00|1384.44|4301.06|6924.69|13.84|0.00|1384.44|1398.28|-2916.62| +2452546|70053|1614|30140|586619|6298|18377|4|22|2218|69|54.46|93.12|79.15|0.00|5461.35|3757.74|6425.28|436.90|0.00|5461.35|5898.25|1703.61| +|72462|3278||1128723|1638||4|87|2219||50.48|||97.94||||7.52|97.94|94.11|101.63|-1066.93| +2451490|72462|3542|81408|1128723|1638|36200|4|78|2219|20|84.21|101.05|39.40|0.00|788.00|1684.20|2021.00|47.28|0.00|788.00|835.28|-896.20| +||7694|81408|1128723|1638|36200||103|2219||50.91|||0.00|||5049.90|51.48|0.00|858.08||| +2451490|72462|1762|81408|1128723|1638|36200|4|197|2219|39|59.35|63.50|38.73|0.00|1510.47|2314.65|2476.50|45.31|0.00|1510.47|1555.78|-804.18| +2451490|72462|4822|81408|1128723|1638|36200|4|179|2219|59|62.75|88.47|36.27|0.00|2139.93|3702.25|5219.73|192.59|0.00|2139.93|2332.52|-1562.32| +2451490|72462|5228|81408|1128723|1638|36200|4|14|2219|100|44.69|70.61|51.54|0.00|5154.00|4469.00|7061.00|360.78|0.00|5154.00|5514.78|685.00| +2451490|72462|25|81408|1128723|1638|36200|4|36|2219|58|10.42|12.29|4.42|0.00|256.36|604.36|712.82|10.25|0.00|256.36|266.61|-348.00| +2451490|72462|17161|81408|1128723|1638|36200|4|136|2219|65|43.82|46.01|14.26|0.00|926.90|2848.30|2990.65|27.80|0.00|926.90|954.70|-1921.40| +2451490|72462|2110|81408|1128723|1638|36200|4|264|2219|88|44.36|54.56|21.82|0.00|1920.16|3903.68|4801.28|38.40|0.00|1920.16|1958.56|-1983.52| +2451490|72462|16804|81408|1128723|1638|36200|4|12|2219|29|92.45|154.39|115.79|0.00|3357.91|2681.05|4477.31|33.57|0.00|3357.91|3391.48|676.86| +2451490|72462|15458|81408|1128723|1638|36200|4|186|2219|27|94.22|175.24|119.16|0.00|3217.32|2543.94|4731.48|128.69|0.00|3217.32|3346.01|673.38| +2451490|72462|10810|81408|1128723|1638|36200|4|36|2219|100|50.74|64.94|31.17|0.00|3117.00|5074.00|6494.00|155.85|0.00|3117.00|3272.85|-1957.00| +2452594|69085|157|24611|1118910|409|23461|7|87|2220|94|69.32|101.20|22.26|0.00|2092.44|6516.08|9512.80|104.62|0.00|2092.44|2197.06|-4423.64| +2452594|69085|10699|24611|1118910|409|23461|7|240|2220|76|35.66|55.62|3.33|55.67|253.08|2710.16|4227.12|1.97|55.67|197.41|199.38|-2512.75| +2452594|69085|15157|24611|1118910|409|23461|7|76|2220|14|2.75|4.81|3.22|0.00|45.08|38.50|67.34|2.70|0.00|45.08|47.78|6.58| +2452594|69085|17226|24611|1118910|409|23461|7|231|2220|99|57.10|91.93|35.85|0.00|3549.15|5652.90|9101.07|248.44|0.00|3549.15|3797.59|-2103.75| +2452594|69085|11991|24611|1118910|||7|237|2220|6|47.23|55.73|45.14|0.00|270.84|283.38||2.70|0.00|||-12.54| +2452594|69085|9901|24611|1118910|409|23461|7|242|2220|82|47.67|50.05|44.04|1083.38|3611.28|3908.94|4104.10|151.67|1083.38|2527.90|2679.57|-1381.04| +2452594|69085|7608|24611|1118910|409|23461|7|195|2220|26|25.79|45.13|32.49|0.00|844.74|670.54|1173.38|50.68|0.00|844.74|895.42|174.20| +2452594|69085|15649|24611|1118910|409|23461|7|91|2220|36|22.88|35.92|29.09|607.39|1047.24|823.68|1293.12|21.99|607.39|439.85|461.84|-383.83| +2452594|69085|11490|24611|1118910|409|23461|7|107|2220|84|20.87|35.06|34.35|0.00|2885.40|1753.08|2945.04|0.00|0.00|2885.40|2885.40|1132.32| +2452594|69085|2421|24611|1118910|409|23461|7|22|2220|57|8.25|15.51|11.63|0.00|662.91|470.25|884.07|46.40|0.00|662.91|709.31|192.66| +2452594|69085|16647|24611|1118910|409|23461|7|131|2220|55|70.34|136.45|21.83|0.00|1200.65|3868.70|7504.75|48.02|0.00|1200.65|1248.67|-2668.05| +2451882|50171|7709|42162|1421243|1924|29805|4|296|2221|30|67.51|75.61|24.19|0.00|725.70|2025.30|2268.30|29.02|0.00|725.70|754.72|-1299.60| +|50171|11957||1421243|1924|||193|2221||70.86|111.95||0.00|2297.16||6045.30|0.00|0.00|2297.16|2297.16|| +2451882|50171|7100|42162|1421243|1924|29805|4|190|2221|54|3.94|4.88|4.58|0.00|247.32|212.76|263.52|17.31|0.00|247.32|264.63|34.56| +2451882|50171|11633|42162|1421243|1924|29805|4|8|2221|82|85.85|85.85|4.29|0.00|351.78|7039.70|7039.70|14.07|0.00|351.78|365.85|-6687.92| +2451882|50171|15485|42162|1421243|1924|29805|4|162|2221|78|86.47|128.84|109.51|0.00|8541.78|6744.66|10049.52|512.50|0.00|8541.78|9054.28|1797.12| +2451882|50171|12419|42162|1421243|1924|29805|4|244|2221|97|37.07|41.51|8.30|499.16|805.10|3595.79|4026.47|12.23|499.16|305.94|318.17|-3289.85| +2451882|50171|13307|42162|1421243|1924|29805|4|97|2221|1|11.61|14.86|11.59|8.57|11.59|11.61|14.86|0.12|8.57|3.02|3.14|-8.59| +2451882|50171|17066|42162|1421243|1924|29805|4|44|2221|25|25.68|30.81|18.17|186.24|454.25|642.00|770.25|24.12|186.24|268.01|292.13|-373.99| +2451882|50171|10322|42162|1421243|1924|29805|4|264|2221|10|46.45|75.24|63.20|0.00|632.00|464.50|752.40|56.88|0.00|632.00|688.88|167.50| +2451882|50171|12992|42162|1421243|1924|29805|4|16|2221|91|12.75|17.72|2.48|0.00|225.68|1160.25|1612.52|2.25|0.00|225.68|227.93|-934.57| +2451882|50171|539|42162|1421243|1924|29805|4|171|2221|85|53.83|82.35|30.46|0.00|2589.10|4575.55|6999.75|207.12|0.00|2589.10|2796.22|-1986.45| +2452180|56481|15145|19548|1077433|5718|32003|8|77|2222|46|4.00|7.56|5.74|139.94|264.04|184.00|347.76|0.00|139.94|124.10|124.10|-59.90| +2452180|56481|12793|19548|1077433|5718|32003|8|214|2222|61|98.23|152.25|111.14|0.00|6779.54|5992.03|9287.25|0.00|0.00|6779.54|6779.54|787.51| +2452180|56481|715|19548|1077433|5718|32003|8|112|2222|14|17.45|32.28|15.17|0.00|212.38|244.30|451.92|10.61|0.00|212.38|222.99|-31.92| +|56481|16091||1077433|||8|171|2222|||||0.00|368.76||6158.04||0.00||372.44|| +2452180|56481|1601|19548|1077433|5718|32003|8|285|2222|6|56.45|76.77|21.49|0.00|128.94|338.70|460.62|6.44|0.00|128.94|135.38|-209.76| +2452180|56481|15425|19548|1077433|5718|32003|8|247|2222|34|1.19|1.77|0.72|0.00|24.48|40.46|60.18|1.95|0.00|24.48|26.43|-15.98| +2452180|56481|15561|19548|1077433|5718|32003|8|276|2222|79|8.67|11.01|2.75|0.00|217.25|684.93|869.79|0.00|0.00|217.25|217.25|-467.68| +2452180|56481|11821|19548|1077433|5718|32003|8|185|2222|78|58.97|58.97|35.97|0.00|2805.66|4599.66|4599.66|112.22|0.00|2805.66|2917.88|-1794.00| +2452180|56481|12313|19548|1077433|5718|32003|8|140|2222|41|10.27|13.45|6.05|0.00|248.05|421.07|551.45|9.92|0.00|248.05|257.97|-173.02| +2452180|56481|3587|19548|1077433|5718|32003|8|143|2222|64|85.41|140.92|108.50|0.00|6944.00|5466.24|9018.88|347.20|0.00|6944.00|7291.20|1477.76| +2452180|56481|2259|19548|1077433|5718|32003|8|126|2222|68|41.43|50.95|30.06|1185.56|2044.08|2817.24|3464.60|25.75|1185.56|858.52|884.27|-1958.72| +2452180|56481|6787|19548|1077433|5718|32003|8|135|2222|2|30.31|35.46|14.18|0.00|28.36|60.62|70.92|1.70|0.00|28.36|30.06|-32.26| +2452180|56481|3485|19548|1077433|5718|32003|8|133|2222|53|80.47|140.82|47.87|2207.28|2537.11|4264.91|7463.46|19.78|2207.28|329.83|349.61|-3935.08| +2451687|40867|5209|53859|623649|5792|4382|1|61|2223|90|33.08|54.91|49.41|0.00|4446.90|2977.20|4941.90|222.34|0.00|4446.90|4669.24|1469.70| +2451687|40867|7865|53859|623649|5792|4382|1|120|2223|46|3.26|5.41|1.24|53.04|57.04|149.96|248.86|0.20|53.04|4.00|4.20|-145.96| +2451687|40867|15697|53859|623649|5792|4382|1|130|2223|80|91.38|118.79|104.53|0.00|8362.40|7310.40|9503.20|752.61|0.00|8362.40|9115.01|1052.00| +2451687|40867|6875|53859|623649|5792|4382|1|154|2223|88|10.29|16.36|13.41|0.00|1180.08|905.52|1439.68|82.60|0.00|1180.08|1262.68|274.56| +2451687|40867|10981|53859|623649|5792|4382|1|203|2223|40|35.90|39.84|7.17|0.00|286.80|1436.00|1593.60|11.47|0.00|286.80|298.27|-1149.20| +2451687|40867|560|53859|623649|5792|4382|1|46|2223|6|65.42|69.34|67.95|0.00|407.70|392.52|416.04|8.15|0.00|407.70|415.85|15.18| +2451687|40867|11231|53859|623649|5792|4382|1|64|2223|38|33.60|48.04|2.88|0.00|109.44|1276.80|1825.52|0.00|0.00|109.44|109.44|-1167.36| +2451687|40867|1189|53859|623649|5792|4382|1|121|2223|91|15.77|26.33|23.43|0.00|2132.13|1435.07|2396.03|191.89|0.00|2132.13|2324.02|697.06| +2451687|40867|12743|53859|623649|5792|4382|1|274|2223|54|94.28|97.10|11.65|434.07|629.10|5091.12|5243.40|13.65|434.07|195.03|208.68|-4896.09| +2451687|40867|1520|53859|623649|5792|4382|1|234|2223|73|42.04|42.46|14.86|357.97|1084.78|3068.92|3099.58|50.87|357.97|726.81|777.68|-2342.11| +2451866|38753|17549|60379|642327|7028|41376|7|194|2224|12|24.25|47.04|2.82|0.00|33.84|291.00|564.48|1.35|0.00|33.84|35.19|-257.16| +2451866|38753|16856|60379|642327|7028|41376|7|215|2224|91|62.39|70.50|66.97|0.00|6094.27|5677.49|6415.50|121.88|0.00|6094.27|6216.15|416.78| +2451866|38753|13411|60379|642327|7028|41376|7|58|2224|2|32.04|36.52|13.14|0.00|26.28|64.08|73.04|2.36|0.00|26.28|28.64|-37.80| +2451866|38753|3269|60379|642327|7028|41376|7|74|2224|24|20.68|34.74|18.75|0.00|450.00|496.32|833.76|22.50|0.00|450.00|472.50|-46.32| +2451866|38753|3122|60379|642327|7028|41376|7|160|2224|7|57.07|89.02|64.98|423.01|454.86|399.49|623.14|0.95|423.01|31.85|32.80|-367.64| +2451866|38753|16253|60379|642327|7028|41376|7|290|2224|92|8.68|11.28|10.94|0.00|1006.48|798.56|1037.76|20.12|0.00|1006.48|1026.60|207.92| +2451866|38753|2507|60379|642327|7028|41376|7|81|2224|70|36.83|39.03|9.75|0.00|682.50|2578.10|2732.10|61.42|0.00|682.50|743.92|-1895.60| +2451866|38753|14711|60379|642327|7028|41376|7|221|2224|56|34.56|66.00|28.38|15.89|1589.28|1935.36|3696.00|94.40|15.89|1573.39|1667.79|-361.97| +2451866||14000|60379|||41376|7||2224||||164.21|144.50||4326.52|7225.24|212.42|144.50|7080.74|7293.16|2754.22| +2451866|38753|643|60379|642327|7028|41376|7|248|2224|22|82.03|134.52|65.91|754.01|1450.02|1804.66|2959.44|55.68|754.01|696.01|751.69|-1108.65| +2451866|38753|3379|60379|642327|7028|41376|7|123|2224|30|36.15|41.93|33.96|0.00|1018.80|1084.50|1257.90|50.94|0.00|1018.80|1069.74|-65.70| +2451866|38753|9278|60379|642327|7028|41376|7|222|2224|41|78.45|106.69|35.20|0.00|1443.20|3216.45|4374.29|86.59|0.00|1443.20|1529.79|-1773.25| +2451177|35874|17528|10229|1235392|555|16118|4|248|2225|51|82.77|151.46|93.90|0.00|4788.90|4221.27|7724.46|431.00|0.00|4788.90|5219.90|567.63| +2451177|35874|4846|10229|1235392|555|16118|4|298|2225|61|70.90|103.51|17.59|0.00|1072.99|4324.90|6314.11|0.00|0.00|1072.99|1072.99|-3251.91| +2451177|35874|490|10229|1235392|555|16118|4|31|2225|25|20.93|29.92|0.89|0.00|22.25|523.25|748.00|2.00|0.00|22.25|24.25|-501.00| +2451177|35874|8014|10229|1235392|555|16118|4|126|2225|9|95.62|122.39|112.59|557.32|1013.31|860.58|1101.51|22.79|557.32|455.99|478.78|-404.59| +2451177|35874|13952|10229|1235392|555|16118|4|87|2225|64|97.10|141.76|2.83|0.00|181.12|6214.40|9072.64|9.05|0.00|181.12|190.17|-6033.28| +2451177|35874|6619|10229|1235392|555|16118|4|4|2225|26|60.29|71.74|71.02|0.00|1846.52|1567.54|1865.24|18.46|0.00|1846.52|1864.98|278.98| +2451177|35874|10778|10229|1235392|555|16118|4|214|2225|37|26.21|41.41|29.40|0.00|1087.80|969.77|1532.17|65.26|0.00|1087.80|1153.06|118.03| +2451177|35874|16016|10229|1235392|555|16118|4|40|2225|86|52.47|52.99|23.31|0.00|2004.66|4512.42|4557.14|180.41|0.00|2004.66|2185.07|-2507.76| +2451177|35874|12655|10229|1235392|555|16118|4|150|2225|24|30.87|56.49|30.50|0.00|732.00|740.88|1355.76|58.56|0.00|732.00|790.56|-8.88| +2451177|35874|6733|10229|1235392|555|16118|4|211|2225|8|79.02|97.98|69.56|0.00|556.48|632.16|783.84|44.51|0.00|556.48|600.99|-75.68| +2451177|35874|7760|10229|1235392|555|16118|4|284|2225|84|20.18|31.68|12.67|0.00|1064.28|1695.12|2661.12|10.64|0.00|1064.28|1074.92|-630.84| +||14107||1235392|555||||2225||61.21|||||1469.04|2556.00|20.44|||276.04|| +2451252|51914|7798|1558|1594667|6976|43285|2|100|2226|83|15.83|19.47|18.69|0.00|1551.27|1313.89|1616.01|124.10|0.00|1551.27|1675.37|237.38| +2451252|51914|15022|1558|1594667|6976|43285|2|15|2226|8|34.30|42.87|23.14|0.00|185.12|274.40|342.96|5.55|0.00|185.12|190.67|-89.28| +2451252|51914|7202|1558|1594667|6976|43285|2|236|2226|65|40.42|63.86|42.78|0.00|2780.70|2627.30|4150.90|83.42|0.00|2780.70|2864.12|153.40| +2451252|51914|5122|1558|1594667|6976|43285|2|268|2226|67|39.02|65.55|22.94|0.00|1536.98|2614.34|4391.85|76.84|0.00|1536.98|1613.82|-1077.36| +2451252|51914|3787|1558|1594667|6976|43285|2|70|2226|15|25.33|41.28|2.47|0.00|37.05|379.95|619.20|2.22|0.00|37.05|39.27|-342.90| +2451252|51914|3547|1558|1594667|6976|43285|2|172|2226|76|41.29|75.14|68.37|0.00|5196.12|3138.04|5710.64|311.76|0.00|5196.12|5507.88|2058.08| +2451252|51914|8623|1558|1594667|6976|43285|2|11|2226|4|16.26|25.69|8.99|25.17|35.96|65.04|102.76|0.86|25.17|10.79|11.65|-54.25| +2451252|51914|17962|1558|1594667|6976|43285|2|179|2226|65|36.05|68.85|8.95|0.00|581.75|2343.25|4475.25|5.81|0.00|581.75|587.56|-1761.50| +2451252|51914|15970|1558|1594667|6976|43285|2|232|2226|90|89.67|173.06|169.59|0.00|15263.10|8070.30|15575.40|915.78|0.00|15263.10|16178.88|7192.80| +2451252|51914|6409|1558|1594667|6976|43285|2|91|2226|27|63.86|83.01|46.48|0.00|1254.96|1724.22|2241.27|62.74|0.00|1254.96|1317.70|-469.26| +2450983|51234|2474|16000|910942|5437|47579|4|239|2227|17|81.36|97.63|90.79|0.00|1543.43|1383.12|1659.71|77.17|0.00|1543.43|1620.60|160.31| +2450983|51234|16934|16000|910942|5437|47579|4|71|2227|13|83.90|140.11|102.28|0.00|1329.64|1090.70|1821.43|106.37|0.00|1329.64|1436.01|238.94| +2450983|51234|9145|16000|910942|5437|47579|4|30|2227|53|26.56|26.56|13.81|0.00|731.93|1407.68|1407.68|14.63|0.00|731.93|746.56|-675.75| +2450983|51234|5222|16000|910942|5437|47579|4|78|2227|87|17.62|21.49|11.81|0.00|1027.47|1532.94|1869.63|61.64|0.00|1027.47|1089.11|-505.47| +2450983|51234|17708|16000|910942|5437|47579|4|222|2227|15|38.90|73.52|32.34|0.00|485.10|583.50|1102.80|14.55|0.00|485.10|499.65|-98.40| +2450983|51234|3535|16000|910942|5437|47579|4|186|2227|31|44.90|55.67|36.74|0.00|1138.94|1391.90|1725.77|34.16|0.00|1138.94|1173.10|-252.96| +2450983|51234|8776|16000|910942|5437|47579|4|296|2227|40|29.16|31.20|20.90|0.00|836.00|1166.40|1248.00|25.08|0.00|836.00|861.08|-330.40| +2450983|51234|2248|16000|910942|5437|47579|4|9|2227|4|56.95|109.34|18.58|0.00|74.32|227.80|437.36|2.97|0.00|74.32|77.29|-153.48| +2450983|51234|16784|16000|910942|5437|47579|4|110|2227|3|32.22|36.08|17.67|0.00|53.01|96.66|108.24|3.71|0.00|53.01|56.72|-43.65| +|51234|9646|16000|910942|5437|47579|4|98|2227|3||22.82|13.00|0.00||64.59|68.46|1.56|0.00|||-25.59| +2450983|51234|308|16000|910942|5437|47579|4|118|2227|71|28.81|32.26|26.13|0.00|1855.23|2045.51|2290.46|111.31|0.00|1855.23|1966.54|-190.28| +2450983|51234|15187|16000|910942|5437|47579|4|159|2227|78|41.24|73.81|15.50|0.00|1209.00|3216.72|5757.18|24.18|0.00|1209.00|1233.18|-2007.72| +2450983|51234|14413|16000|910942|5437|47579|4|171|2227|35|88.39|122.86|104.43|0.00|3655.05|3093.65|4300.10|255.85|0.00|3655.05|3910.90|561.40| +2450983|51234|10628|16000|910942|5437|47579|4|215|2227|58|39.81|69.66|68.26|0.00|3959.08|2308.98|4040.28|237.54|0.00|3959.08|4196.62|1650.10| +||15164|16000||5437|47579||212|2227||63.03|77.52||||||||||-1047.06| +2451768|57137|3785|87754|923959|3011|40416|10|296|2228|100|37.33|49.64|37.72|3394.80|3772.00|3733.00|4964.00|22.63|3394.80|377.20|399.83|-3355.80| +2451768|57137|9013|87754|923959|3011|40416|10|299|2228|52|48.57|65.56|18.35|0.00|954.20|2525.64|3409.12|66.79|0.00|954.20|1020.99|-1571.44| +2451768|57137|1993|87754|923959|3011|40416|10|139|2228|16|9.99|19.08|4.19|0.00|67.04|159.84|305.28|3.35|0.00|67.04|70.39|-92.80| +||8768||923959||40416|10||2228|74|93.81||41.06|972.30||6941.94||61.98|972.30||2128.12|-4875.80| +2451768|57137|2444|87754|923959|3011|40416|10|107|2228|20|91.80|152.38|144.76|0.00|2895.20|1836.00|3047.60|202.66|0.00|2895.20|3097.86|1059.20| +2451768|57137|12848|87754|923959|3011|40416|10|4|2228|33|79.06|147.05|22.05|0.00|727.65|2608.98|4852.65|43.65|0.00|727.65|771.30|-1881.33| +2451768|57137|12997|87754|923959|3011|40416|10|289|2228|24|82.99|152.70|105.36|0.00|2528.64|1991.76|3664.80|202.29|0.00|2528.64|2730.93|536.88| +2451768|57137|5576|87754|923959|3011|40416|10|232|2228|50|4.06|5.68|0.51|0.00|25.50|203.00|284.00|2.29|0.00|25.50|27.79|-177.50| +2451119|74474|3286|16563|1695587|2611|37635|8|292|2229|86|12.71|23.25|18.36|1389.48|1578.96|1093.06|1999.50|13.26|1389.48|189.48|202.74|-903.58| +2451119|74474|13009|16563|1695587|2611|37635|8|108|2229|15|26.67|33.33|23.99|241.09|359.85|400.05|499.95|3.56|241.09|118.76|122.32|-281.29| +2451119|74474|16424|16563|1695587|2611|37635|8|220|2229|88|96.67|145.00|71.05|0.00|6252.40|8506.96|12760.00|187.57|0.00|6252.40|6439.97|-2254.56| +2451119|74474|9505|16563|1695587|2611|37635|8|18|2229|32|37.30|47.74|42.48|0.00|1359.36|1193.60|1527.68|67.96|0.00|1359.36|1427.32|165.76| +2451119|74474|4396|16563|1695587|2611|37635|8|176|2229|61|30.87|47.84|6.21|0.00|378.81|1883.07|2918.24|30.30|0.00|378.81|409.11|-1504.26| +2451119|74474|688|16563|1695587|2611|37635|8|148|2229|56|21.74|29.13|4.95|0.00|277.20|1217.44|1631.28|2.77|0.00|277.20|279.97|-940.24| +2451119||1195||1695587||37635|||2229||97.69|||582.00||||471.42|582.00|5238.00||| +2451119|74474|7495|16563|1695587|2611|37635|8|235|2229|78|62.96|107.03|44.95|0.00|3506.10|4910.88|8348.34|35.06|0.00|3506.10|3541.16|-1404.78| +2451119|74474|1238|16563|1695587|2611|37635|8|145|2229|91|3.13|5.00|3.30|0.00|300.30|284.83|455.00|3.00|0.00|300.30|303.30|15.47| +2451119|74474|10084|16563|1695587|2611|37635|8|259|2229|23|71.27|126.86|107.83|0.00|2480.09|1639.21|2917.78|173.60|0.00|2480.09|2653.69|840.88| +2451119|74474|14260|16563|1695587|2611|37635|8|218|2229|48|24.90|39.09|22.28|0.00|1069.44|1195.20|1876.32|85.55|0.00|1069.44|1154.99|-125.76| +2451119|74474|4978|16563|1695587|2611|37635|8|64|2229|28|54.25|72.15|55.55|0.00|1555.40|1519.00|2020.20|139.98|0.00|1555.40|1695.38|36.40| +2451119|74474|6698|16563|1695587|2611|37635|8|233|2229|57|63.47|64.10|48.71|0.00|2776.47|3617.79|3653.70|27.76|0.00|2776.47|2804.23|-841.32| +2451119|74474|640|16563|1695587|2611|37635|8|122|2229|4|9.50|16.91|14.54|0.00|58.16|38.00|67.64|3.48|0.00|58.16|61.64|20.16| +2451119|74474|15290|16563|1695587|2611|37635|8|161|2229|20|88.36|167.00|35.07|0.00|701.40|1767.20|3340.00|7.01|0.00|701.40|708.41|-1065.80| +2451119|74474|9488|16563|1695587|2611|37635|8|114|2229|77|99.81|185.64|22.27|977.43|1714.79|7685.37|14294.28|36.86|977.43|737.36|774.22|-6948.01| +2451810|50343|5585|38507|1823544|5381|9218|1|250|2230|49|86.91|92.12|51.58|0.00|2527.42|4258.59|4513.88|202.19|0.00|2527.42|2729.61|-1731.17| +2451810|50343|3779|38507|1823544|5381|9218|1|98|2230|27|20.39|25.48|10.95|0.00|295.65|550.53|687.96|26.60|0.00|295.65|322.25|-254.88| +||3835||1823544||||1|2230|56|39.36|50.38||||||4.72||||-2136.62| +2451810|50343|925|38507|1823544|5381|9218|1|285|2230|21|20.03|22.83|22.83|0.00|479.43|420.63|479.43|43.14|0.00|479.43|522.57|58.80| +2451810|50343|3698|38507|1823544|5381|9218|1|29|2230|80|96.50|123.52|44.46|0.00|3556.80|7720.00|9881.60|142.27|0.00|3556.80|3699.07|-4163.20| +2451810|50343|16763|38507|1823544|5381|9218|1|298|2230|86|36.80|61.08|19.54|0.00|1680.44|3164.80|5252.88|100.82|0.00|1680.44|1781.26|-1484.36| +2451810|50343|14198|38507|1823544|5381|9218|1|37|2230|100|22.10|30.71|13.20|0.00|1320.00|2210.00|3071.00|52.80|0.00|1320.00|1372.80|-890.00| +2451810|50343|15452|38507|1823544|5381|9218|1|246|2230|62|44.80|71.68|29.38|746.83|1821.56|2777.60|4444.16|64.48|746.83|1074.73|1139.21|-1702.87| +2451810|50343|14774|38507|1823544|5381|9218|1|167|2230|72|78.12|130.46|43.05|898.88|3099.60|5624.64|9393.12|176.05|898.88|2200.72|2376.77|-3423.92| +2452131|71242|1079|35008|423368|3728|13641|8|46|2231|42|34.48|61.02|12.81|145.26|538.02|1448.16|2562.84|15.71|145.26|392.76|408.47|-1055.40| +2452131|71242|3085|35008|423368|3728|13641|8|186|2231|93|15.36|24.26|9.94|0.00|924.42|1428.48|2256.18|55.46|0.00|924.42|979.88|-504.06| +2452131|71242|13477|35008|423368|3728|13641|8|32|2231|35|14.18|17.15|13.72|441.78|480.20|496.30|600.25|1.15|441.78|38.42|39.57|-457.88| +2452131|71242|5469|35008|423368|3728|13641|8|221|2231|70|68.84|106.70|80.02|0.00|5601.40|4818.80|7469.00|112.02|0.00|5601.40|5713.42|782.60| +2452131|71242|5305||423368|||8|20|2231|||102.98||||||346.43|||6120.38|-405.09| +2452131|71242|1133|35008|423368|3728|13641|8|19|2231|38|31.90|34.77|28.16|0.00|1070.08|1212.20|1321.26|10.70|0.00|1070.08|1080.78|-142.12| +2452131|71242|4775|35008|423368|3728|13641|8|61|2231|57|9.86|14.88|7.14|288.95|406.98|562.02|848.16|9.44|288.95|118.03|127.47|-443.99| +2452131|71242|5009|35008|423368|3728|13641|8|23|2231|5|11.10|16.76|15.08|0.00|75.40|55.50|83.80|0.75|0.00|75.40|76.15|19.90| +2452131|71242|255|35008|423368|3728|13641|8|272|2231|8|74.84|79.33|64.25|0.00|514.00|598.72|634.64|25.70|0.00|514.00|539.70|-84.72| +2452131|71242|3203|35008|423368|3728|13641|8|78|2231|5|59.60|75.69|21.19|0.00|105.95|298.00|378.45|4.23|0.00|105.95|110.18|-192.05| +2452131|71242|1917|35008|423368|3728|13641|8|120|2231|18|96.47|182.32|36.46|551.27|656.28|1736.46|3281.76|7.35|551.27|105.01|112.36|-1631.45| +2452131|71242|481|35008|423368|3728|13641|8|59|2231|30|66.28|123.28|12.32|0.00|369.60|1988.40|3698.40|0.00|0.00|369.60|369.60|-1618.80| +2452131|71242|9771|35008|423368|3728|13641|8|246|2231|54|24.66|26.38|10.55|193.69|569.70|1331.64|1424.52|11.28|193.69|376.01|387.29|-955.63| +2451902|48140|3035|91730|203091|4052|49413|2|123|2232|81|96.46|103.21|3.09|0.00|250.29|7813.26|8360.01|15.01|0.00|250.29|265.30|-7562.97| +2451902|48140|16214|91730|203091|4052|49413|2|22|2232|33|43.37|76.33|73.27|0.00|2417.91|1431.21|2518.89|193.43|0.00|2417.91|2611.34|986.70| +2451902|48140|3218|91730|203091|4052|49413|2|133|2232|39|36.45|43.01|1.72|0.00|67.08|1421.55|1677.39|0.00|0.00|67.08|67.08|-1354.47| +2451902|48140|11618|91730|203091|4052|49413|2|146|2232|16|26.77|30.78|0.30|0.00|4.80|428.32|492.48|0.04|0.00|4.80|4.84|-423.52| +2451902||5984||||||176|2232|37|78.70||7.36||||3028.08|||||| +2451902|48140|13520|91730|203091|4052|49413|2|235|2232|30|23.13||24.16||724.80||763.20|57.98||||30.90| +2451902|48140|13544|91730|203091|4052|49413|2|174|2232|72|93.24|123.07|0.00|0.00|0.00|6713.28|8861.04|0.00|0.00|0.00|0.00|-6713.28| +2451902|48140|686|91730|203091|4052|49413|2|66|2232|64|37.21|45.02|18.00|0.00|1152.00|2381.44|2881.28|57.60|0.00|1152.00|1209.60|-1229.44| +2451902|48140|7117|91730|203091|4052|49413|2|199|2232|29|11.42|19.75|18.36|0.00|532.44|331.18|572.75|37.27|0.00|532.44|569.71|201.26| +2451902|48140|7061|91730|203091|4052|49413|2|69|2232|100|95.88|174.50|155.30|0.00|15530.00|9588.00|17450.00|931.80|0.00|15530.00|16461.80|5942.00| +2451902|48140|12871|91730|203091|4052|49413|2|124|2232|79|88.70|109.10|81.82|0.00|6463.78|7007.30|8618.90|517.10|0.00|6463.78|6980.88|-543.52| +2451902|48140|16940|91730|203091|4052|49413|2|202|2232|56|31.11|31.42|26.70|0.00|1495.20|1742.16|1759.52|104.66|0.00|1495.20|1599.86|-246.96| +2451902|48140|9565|91730|203091|4052|49413|2|49|2232|14|10.58|11.10|5.77|0.00|80.78|148.12|155.40|2.42|0.00|80.78|83.20|-67.34| +2451902|48140|17023|91730|203091|4052|49413|2|82|2232|66|39.01|62.02|27.90|0.00|1841.40|2574.66|4093.32|55.24|0.00|1841.40|1896.64|-733.26| +2451902|48140|15907|91730|203091|4052|49413|2|276|2232|13|24.63|36.69|15.40|0.00|200.20|320.19|476.97|16.01|0.00|200.20|216.21|-119.99| +2451843|50557|11858|48458|382154|1897|15193|4|266|2233|70|50.97|53.00|47.17|0.00|3301.90|3567.90|3710.00|264.15|0.00|3301.90|3566.05|-266.00| +2451843|50557|10640|48458|382154|1897|15193|4|143|2233|98|19.52|31.62|13.91|0.00|1363.18|1912.96|3098.76|54.52|0.00|1363.18|1417.70|-549.78| +2451843|50557|17948|48458|382154|1897|15193|4|172|2233|24|25.66|29.76|18.74|0.00|449.76|615.84|714.24|13.49|0.00|449.76|463.25|-166.08| +2451843|50557|7451|48458|382154|1897|15193|4|87|2233|81|3.64|6.47|3.49|0.00|282.69|294.84|524.07|11.30|0.00|282.69|293.99|-12.15| +2451843|50557|1430|48458|382154|1897|15193|4|73|2233|83|3.15|4.12|3.74|0.00|310.42|261.45|341.96|24.83|0.00|310.42|335.25|48.97| +2451843|50557|13003|48458|382154|1897|15193|4|36|2233|70|15.46|28.29|25.17|0.00|1761.90|1082.20|1980.30|140.95|0.00|1761.90|1902.85|679.70| +2451843|50557|10448|48458|382154|1897|15193|4|100|2233|29|79.76|150.74|114.56|0.00|3322.24|2313.04|4371.46|232.55|0.00|3322.24|3554.79|1009.20| +2451843|50557|2468|48458|382154|1897|15193|4|84|2233|56|94.62|122.05|108.62|0.00|6082.72|5298.72|6834.80|547.44|0.00|6082.72|6630.16|784.00| +2451843|50557|14702|48458|382154|1897|15193|4|23|2233|34|65.33|129.35|0.00|0.00|0.00|2221.22|4397.90|0.00|0.00|0.00|0.00|-2221.22| +2451843|50557|4151|48458|382154|1897|15193|4|28|2233|19|78.78|81.14|63.28|1058.04|1202.32|1496.82|1541.66|7.21|1058.04|144.28|151.49|-1352.54| +2451921|32686|11793|58400|1162996|3145|11741|8|228|2234|31|8.83|13.68|5.19|24.13|160.89|273.73|424.08|6.83|24.13|136.76|143.59|-136.97| +2451921|32686|15293|58400|1162996|3145|11741|8|101|2234|65|88.97|94.30|33.00|0.00|2145.00|5783.05|6129.50|150.15|0.00|2145.00|2295.15|-3638.05| +2451921|32686|6271|58400|1162996|3145|11741|8|87|2234|38|6.23|11.89|6.18|190.22|234.84|236.74|451.82|0.44|190.22|44.62|45.06|-192.12| +2451921|32686|5465|58400|1162996|3145|11741|8|12|2234|46|79.50|112.09|20.17|0.00|927.82|3657.00|5156.14|83.50|0.00|927.82|1011.32|-2729.18| +2451921|32686|2927|58400|1162996|3145|11741|8|211|2234|99|64.51|78.70|6.29|0.00|622.71|6386.49|7791.30|43.58|0.00|622.71|666.29|-5763.78| +2451921|32686|7995|58400|1162996|3145|11741|8|267|2234|98|5.31|8.28|7.45|0.00|730.10|520.38|811.44|0.00|0.00|730.10|730.10|209.72| +2451921|32686|2055|58400|1162996|3145|11741|8|239|2234|35|52.78|80.22|8.02|0.00|280.70|1847.30|2807.70|16.84|0.00|280.70|297.54|-1566.60| +2451921|32686|3237|58400|1162996|3145|11741|8|93|2234|56|43.45|45.18|4.06|0.00|227.36|2433.20|2530.08|15.91|0.00|227.36|243.27|-2205.84| +2451921|32686|159|58400|1162996|3145|11741|8|66|2234|39|88.57|175.36|101.70|0.00|3966.30|3454.23|6839.04|277.64|0.00|3966.30|4243.94|512.07| +2451921|32686|10033|58400|1162996|3145|11741|8|157|2234|21|22.59|39.30|5.50|0.00|115.50|474.39|825.30|2.31|0.00|115.50|117.81|-358.89| +2451921|32686|5761|58400|1162996|3145|11741|8|213|2234|88|69.81|102.62|29.75|0.00|2618.00|6143.28|9030.56|235.62|0.00|2618.00|2853.62|-3525.28| +2451921|32686|8291|58400|1162996|3145|11741|8|212|2234|49|87.35|100.45|91.40|0.00|4478.60|4280.15|4922.05|179.14|0.00|4478.60|4657.74|198.45| +2452535|41356|11274|26861|112502|2496|47337|1|201|2235|16|23.45|43.14|37.96|0.00|607.36|375.20|690.24|24.29|0.00|607.36|631.65|232.16| +2452535|41356|4075|26861|112502|2496|47337|1|210|2235|7|87.30|147.53|50.16|0.00|351.12|611.10|1032.71|3.51|0.00|351.12|354.63|-259.98| +2452535|41356|9132|26861|112502|2496|47337|1|237|2235|82|94.13|159.07|112.93|6852.59|9260.26|7718.66|13043.74|168.53|6852.59|2407.67|2576.20|-5310.99| +2452535|41356|2496|26861|112502|2496|47337|1|206|2235|46|85.30|88.71|85.16|0.00|3917.36|3923.80|4080.66|156.69|0.00|3917.36|4074.05|-6.44| +2452535|41356|7651|26861|112502|2496|47337|1|279|2235|26|80.87|101.89|64.19|0.00|1668.94|2102.62|2649.14|83.44|0.00|1668.94|1752.38|-433.68| +2452535|41356|3483|26861|112502|2496|47337|1|180|2235|50|58.47|109.33|76.53|0.00|3826.50|2923.50|5466.50|76.53|0.00|3826.50|3903.03|903.00| +2452535|41356|4050|26861|112502|2496|47337|1|240|2235|75|82.55|99.06|70.33|0.00|5274.75|6191.25|7429.50|210.99|0.00|5274.75|5485.74|-916.50| +2452535|41356|10189|26861|112502|2496|47337|1|183|2235|58|8.74|15.90|13.99|0.00|811.42|506.92|922.20|24.34|0.00|811.42|835.76|304.50| +2452535|41356|147|26861|112502|2496|47337|1|48|2235|94|55.43|109.75|50.48|0.00|4745.12|5210.42|10316.50|189.80|0.00|4745.12|4934.92|-465.30| +2452535|41356|12105|26861|112502|2496|47337|1|200|2235|11|13.94|25.37|21.81|0.00|239.91|153.34|279.07|16.79|0.00|239.91|256.70|86.57| +2452535|41356|16207|26861|112502|2496|47337|1|42|2235|31|97.93|107.72|24.77|598.93|767.87|3035.83|3339.32|3.37|598.93|168.94|172.31|-2866.89| +2452535|41356|5532|26861|112502|2496|47337|1|246|2235|47|34.08|62.02|32.87|0.00|1544.89|1601.76|2914.94|77.24|0.00|1544.89|1622.13|-56.87| +2452618|42553|14511|56392|1739309|1873|36390|4|295|2236|12|8.38|15.83|14.08|59.13|168.96|100.56|189.96|8.78|59.13|109.83|118.61|9.27| +2452618|42553|13509|56392|1739309|1873|36390|4|29|2236|72|55.50|64.38|24.46|0.00|1761.12|3996.00|4635.36|140.88|0.00|1761.12|1902.00|-2234.88| +2452618|42553|9999|56392|1739309|1873|36390|4|36|2236|63|26.79|32.68|0.98|46.30|61.74|1687.77|2058.84|0.61|46.30|15.44|16.05|-1672.33| +2452618|42553|6291|56392|1739309|1873|36390|4|99|2236|5|41.10|53.01|24.91|0.00|124.55|205.50|265.05|6.22|0.00|124.55|130.77|-80.95| +2452618|42553|11515|56392|1739309|1873|36390|4|128|2236|58|11.04|19.20|17.28|0.00|1002.24|640.32|1113.60|10.02|0.00|1002.24|1012.26|361.92| +2452618|42553|8265|56392|1739309|1873|36390|4|49|2236|29|68.29|130.43|101.73|2330.63|2950.17|1980.41|3782.47|30.97|2330.63|619.54|650.51|-1360.87| +2452618|42553|9183|56392|1739309|1873|36390|4|136|2236|81|41.69|65.03|52.67|0.00|4266.27|3376.89|5267.43|42.66|0.00|4266.27|4308.93|889.38| +2452618|42553|16782|56392|1739309|1873|36390|4|300|2236|43|54.58|92.24|17.52|0.00|753.36|2346.94|3966.32|7.53|0.00|753.36|760.89|-1593.58| +2452618|42553|300|56392|1739309|1873|36390|4|102|2236|90|6.43|11.70|9.00|0.00|810.00|578.70|1053.00|32.40|0.00|810.00|842.40|231.30| +2452618|42553|2413|56392|1739309|1873|36390|4|69|2236|9|57.81|83.24|82.40|519.12|741.60|520.29|749.16|17.79|519.12|222.48|240.27|-297.81| +2451126|33562|2402|33845|804240|6188|4183|1|114|2237|31|55.89|111.22|53.38|893.58|1654.78|1732.59|3447.82|0.00|893.58|761.20|761.20|-971.39| +2451126|33562|16174|33845|804240|6188|4183|1|218|2237|36|84.22|117.90|96.67|0.00|3480.12|3031.92|4244.40|34.80|0.00|3480.12|3514.92|448.20| +||8582|||6188|4183|1|260|2237||||17.75|0.00||||0.00|0.00|1011.75|1011.75|| +2451126|33562|10639|33845|804240|6188|4183|1|222|2237|7|3.62|5.75|2.41|0.00|16.87|25.34|40.25|0.84|0.00|16.87|17.71|-8.47| +2451126|33562|1303|33845|804240|6188|4183|1|46|2237|51|50.28|100.56|61.34|0.00|3128.34|2564.28|5128.56|93.85|0.00|3128.34|3222.19|564.06| +2451126|33562|15034|33845|804240|6188|4183|1|47|2237|24|75.51|92.12|44.21|339.53|1061.04|1812.24|2210.88|0.00|339.53|721.51|721.51|-1090.73| +2451126|33562|16916|33845|804240|6188|4183|1|3|2237|54|25.14|49.77|45.29|0.00|2445.66|1357.56|2687.58|171.19|0.00|2445.66|2616.85|1088.10| +2451126|33562|15763|33845|804240|6188|4183|1|251|2237|39|31.76|49.86|39.88|0.00|1555.32|1238.64|1944.54|15.55|0.00|1555.32|1570.87|316.68| +2451126|33562|16669|33845|804240|6188|4183|1|54|2237|41|13.48|26.42|22.72|0.00|931.52|552.68|1083.22|27.94|0.00|931.52|959.46|378.84| +2451126|33562|1633|33845|804240|6188|4183|1|142|2237|64|19.77|30.44|6.08|0.00|389.12|1265.28|1948.16|7.78|0.00|389.12|396.90|-876.16| +2452135|32590|10249|5933|1625813|5340|12420|7|97|2238|60|68.86|121.88|54.84|98.71|3290.40|4131.60|7312.80|63.83|98.71|3191.69|3255.52|-939.91| +2452135|32590|13571|5933|1625813|5340|12420|7|101|2238|24|1.06|1.16|0.68|0.00|16.32|25.44|27.84|0.48|0.00|16.32|16.80|-9.12| +2452135|32590|10081|5933|1625813|5340|12420|7|246|2238|92|93.26|151.08|145.03|0.00|13342.76|8579.92|13899.36|400.28|0.00|13342.76|13743.04|4762.84| +2452135|32590|16375|5933|1625813|5340|12420|7|287|2238|29|23.45|44.55|16.92|0.00|490.68|680.05|1291.95|0.00|0.00|490.68|490.68|-189.37| +2452135|32590|5259|5933|1625813|5340|12420|7|192|2238|80|18.80|26.13|25.60|0.00|2048.00|1504.00|2090.40|184.32|0.00|2048.00|2232.32|544.00| +2452135|32590|7433|5933|1625813|5340|12420|7|92|2238|36|31.96|45.06|19.37|0.00|697.32|1150.56|1622.16|13.94|0.00|697.32|711.26|-453.24| +2452135|32590|4079|5933|1625813|5340|12420|7|219|2238|88|28.47|49.25|1.97|0.00|173.36|2505.36|4334.00|6.93|0.00|173.36|180.29|-2332.00| +2452135|32590|12045|5933|1625813|5340|12420|7|166|2238|76|48.82|73.71|27.27|0.00|2072.52|3710.32|5601.96|103.62|0.00|2072.52|2176.14|-1637.80| +2452135|32590|5111|5933|1625813|5340|12420|7|199|2238|29|84.99|108.78|17.40|0.00|504.60|2464.71|3154.62|25.23|0.00|504.60|529.83|-1960.11| +2452135|32590|16479|5933|1625813|5340|12420|7|163|2238|90|90.07|154.01|135.52|0.00|12196.80|8106.30|13860.90|121.96|0.00|12196.80|12318.76|4090.50| +2452135|32590|11947|5933|1625813|5340|12420|7|54|2238|9|94.99|151.98|127.66|0.00|1148.94|854.91|1367.82|34.46|0.00|1148.94|1183.40|294.03| +2452135|32590|13895|5933|1625813|5340|12420|7|69|2238|38|86.14|96.47|92.61|0.00|3519.18|3273.32|3665.86|211.15|0.00|3519.18|3730.33|245.86| +2452135|32590|9795|5933|1625813|5340|12420|7|186|2238|77|94.70|152.46|18.29|0.00|1408.33|7291.90|11739.42|28.16|0.00|1408.33|1436.49|-5883.57| +2452252|63138|14445|13838|844117|4349|41028|8|263|2239|97|68.67|128.41|86.03|0.00|8344.91|6660.99|12455.77|500.69|0.00|8344.91|8845.60|1683.92| +2452252|63138|7469|13838|844117|4349|41028|8|175|2239|48|95.28|173.40|52.02|0.00|2496.96|4573.44|8323.20|74.90|0.00|2496.96|2571.86|-2076.48| +2452252|63138|375|13838|844117|4349|41028|8|195|2239|70|81.11|160.59|154.16|0.00|10791.20|5677.70|11241.30|971.20|0.00|10791.20|11762.40|5113.50| +2452252|63138|14557|13838|844117|4349|41028|8|219|2239|69|27.15|40.72|32.16|0.00|2219.04|1873.35|2809.68|88.76|0.00|2219.04|2307.80|345.69| +2452252|63138|3321|13838|844117|4349|41028|8|12|2239|77|98.52|139.89|134.29|0.00|10340.33|7586.04|10771.53|0.00|0.00|10340.33|10340.33|2754.29| +|63138|10385||844117||41028|8|72|2239||60.44||||3188.64|4714.32||||3188.64|3475.61|-1525.68| +2452252|63138|5501|13838|844117|4349|41028|8|299|2239|21|91.53|161.09|130.48|0.00|2740.08|1922.13|3382.89|27.40|0.00|2740.08|2767.48|817.95| +2452252|63138|10563|13838|844117|4349|41028|8|295|2239|28|18.16|22.70|16.79|0.00|470.12|508.48|635.60|4.70|0.00|470.12|474.82|-38.36| +2451117|42347|835|29687|354149|2258|29567|1|226|2240|77|25.32|46.84|41.21|0.00|3173.17|1949.64|3606.68|158.65|0.00|3173.17|3331.82|1223.53| +2451117|42347|13934|29687|354149|2258|29567|1|255|2240|71|88.07|161.16|116.03|0.00|8238.13|6252.97|11442.36|82.38|0.00|8238.13|8320.51|1985.16| +2451117|42347|15136|29687|354149|2258|29567|1|232|2240|9|32.03|32.03|23.06|0.00|207.54|288.27|288.27|16.60|0.00|207.54|224.14|-80.73| +2451117|42347|8671|29687|354149|2258|29567|1|152|2240|71|82.04|91.06|83.77|0.00|5947.67|5824.84|6465.26|59.47|0.00|5947.67|6007.14|122.83| +2451117|42347|1369|29687|354149|2258|29567|1|36|2240|56|37.22|64.76|4.53|0.00|253.68|2084.32|3626.56|5.07|0.00|253.68|258.75|-1830.64| +2451117|42347|14095|29687|354149|2258|29567|1|12|2240|80|64.53|75.50|52.85|0.00|4228.00|5162.40|6040.00|84.56|0.00|4228.00|4312.56|-934.40| +2451117|42347|2630|29687|354149|2258|29567|1|257|2240|2|82.58|141.21|4.23|0.00|8.46|165.16|282.42|0.16|0.00|8.46|8.62|-156.70| +2451117|42347|10459|29687|354149|2258|29567|1|137|2240|80|58.54|64.39|48.93|0.00|3914.40|4683.20|5151.20|39.14|0.00|3914.40|3953.54|-768.80| +2451117|42347|1202|29687|354149|2258|29567|1|69|2240|37|32.32|56.56|44.68|0.00|1653.16|1195.84|2092.72|132.25|0.00|1653.16|1785.41|457.32| +2451117|42347|775|29687|354149|2258|29567|1|48|2240|40|67.39|93.67|45.89|0.00|1835.60|2695.60|3746.80|91.78|0.00|1835.60|1927.38|-860.00| +2452508|51836|3883|72802|407631|2813|39195|10|56|2241|9|78.21|155.63|52.91|0.00|476.19|703.89|1400.67|33.33|0.00|476.19|509.52|-227.70| +2452508||1449|||||10|75|2241|16|29.70|||313.20|666.40|475.20|784.00|3.53|313.20|353.20|356.73|| +2452508|51836|4062|72802|407631|2813|39195|10|129|2241|42|94.45|131.28|66.95|0.00|2811.90|3966.90|5513.76|196.83|0.00|2811.90|3008.73|-1155.00| +2452508|51836|11688|72802|407631|2813|39195|10|12|2241|25|59.92|68.90|22.73|494.37|568.25|1498.00|1722.50|5.17|494.37|73.88|79.05|-1424.12| +2452508|51836|17499|72802|407631|2813|39195|10|17|2241|80|96.42|96.42|50.13|0.00|4010.40|7713.60|7713.60|200.52|0.00|4010.40|4210.92|-3703.20| +2452508|51836|871|72802|407631|2813|39195|10|152|2241|14|98.37|146.57|137.77|0.00|1928.78|1377.18|2051.98|77.15|0.00|1928.78|2005.93|551.60| +2452508|51836|10902|72802|407631|2813|39195|10|245|2241|22|37.60|44.74|4.02|0.00|88.44|827.20|984.28|5.30|0.00|88.44|93.74|-738.76| +2452508|51836|4578|72802|407631|2813|39195|10|243|2241|53|99.63|169.37|6.77|0.00|358.81|5280.39|8976.61|10.76|0.00|358.81|369.57|-4921.58| +2451138|59140|16291|95142|846023|6857|14891|2|147|2242|60|30.46|31.67|21.53|0.00|1291.80|1827.60|1900.20|77.50|0.00|1291.80|1369.30|-535.80| +2451138|59140|2857|95142|846023|6857|14891|2|116|2242|72|23.69|37.43|27.32|1258.90|1967.04|1705.68|2694.96|21.24|1258.90|708.14|729.38|-997.54| +2451138|59140|5834|95142|846023|6857|14891|2|1|2242|52|86.79|118.90|82.04|3583.50|4266.08|4513.08|6182.80|13.65|3583.50|682.58|696.23|-3830.50| +2451138|59140|3818|95142|846023|6857|14891|2|83|2242|55|72.62|103.84|30.11|0.00|1656.05|3994.10|5711.20|149.04|0.00|1656.05|1805.09|-2338.05| +||2144|95142||||2||2242||48.72||5.40||||4650.02|37.15|||501.55|| +2451138|59140|3619|95142|846023|6857|14891|2|265|2242|30|69.69|71.08|30.56|0.00|916.80|2090.70|2132.40|73.34|0.00|916.80|990.14|-1173.90| +2451138|59140|11702|95142|846023|6857|14891|2|246|2242|62|21.11|42.00|32.34|0.00|2005.08|1308.82|2604.00|40.10|0.00|2005.08|2045.18|696.26| +2451138|59140|13238|95142|846023|6857|14891|2|90|2242|33|61.42|71.86|9.34|0.00|308.22|2026.86|2371.38|18.49|0.00|308.22|326.71|-1718.64| +2451091|64018|4810|13882|1588846|5857|38021|4|252|2243|16|22.78|31.89|2.55|0.00|40.80|364.48|510.24|2.04|0.00|40.80|42.84|-323.68| +2451091|64018|11182|13882|1588846|5857|38021|4|55|2243|43|26.79|36.97|0.73|0.00|31.39|1151.97|1589.71|0.94|0.00|31.39|32.33|-1120.58| +2451091|64018|7492|13882|1588846|5857|38021|4|77|2243|24|6.23|10.34|5.79|38.90|138.96|149.52|248.16|4.00|38.90|100.06|104.06|-49.46| +2451091|64018|3938|13882|1588846|5857|38021|4|34|2243|10|61.58|97.29|18.48|0.00|184.80|615.80|972.90|14.78|0.00|184.80|199.58|-431.00| +2451091|64018|14150|13882|1588846|5857|38021|4|239|2243|30|84.49|104.76|40.85|1004.91|1225.50|2534.70|3142.80|4.41|1004.91|220.59|225.00|-2314.11| +2451091|64018|14167|13882|1588846|5857|38021|4|128|2243|21|7.00|11.13|8.12|10.23|170.52|147.00|233.73|3.20|10.23|160.29|163.49|13.29| +2451091|64018|9452|13882|1588846|5857|38021|4|116|2243|34|40.13|45.74|3.65|0.00|124.10|1364.42|1555.16|8.68|0.00|124.10|132.78|-1240.32| +2451091|64018|3715|13882|1588846|5857|38021|4|182|2243|23|65.05|88.46|17.69|191.22|406.87|1496.15|2034.58|12.93|191.22|215.65|228.58|-1280.50| +2451091|64018|2534|13882|1588846|5857|38021|4|65|2243|94|16.86|26.47|3.97|0.00|373.18|1584.84|2488.18|11.19|0.00|373.18|384.37|-1211.66| +2451091|64018|15964|13882|1588846|5857|38021|4|120|2243|30|92.17|109.68|77.87|0.00|2336.10|2765.10|3290.40|186.88|0.00|2336.10|2522.98|-429.00| +2451091|64018|15764|13882|1588846|5857|38021|4|266|2243|53|15.39|22.77|7.51|0.00|398.03|815.67|1206.81|19.90|0.00|398.03|417.93|-417.64| +2451091|64018|9638|13882|1588846|5857|38021|4|97|2243|46|2.13|4.11|0.65|0.00|29.90|97.98|189.06|0.89|0.00|29.90|30.79|-68.08| +2451091|64018|7717|13882|1588846|5857|38021|4|249|2243|45|26.57|38.26|34.81|0.00|1566.45|1195.65|1721.70|125.31|0.00|1566.45|1691.76|370.80| +2451091|64018|73|13882|1588846|5857|38021|4|254|2243|97|70.65|91.84|13.77|160.28|1335.69|6853.05|8908.48|94.03|160.28|1175.41|1269.44|-5677.64| +2451091|64018|8270|13882|1588846|5857|38021|4|111|2243|16|67.38|90.96|54.57|0.00|873.12|1078.08|1455.36|52.38|0.00|873.12|925.50|-204.96| +2451091|64018|22|13882|||38021|4||2243|||103.07|71.11||3839.94|2976.48||307.19||3839.94|4147.13|| +2451878|74531|3644|67463|912315|6767|30819|2|190|2244|28|14.66|15.83|10.92|0.00|305.76|410.48|443.24|18.34|0.00|305.76|324.10|-104.72| +2451878|74531|937|67463|912315|6767|30819|2|245|2244|8|75.41|130.45|122.62|0.00|980.96|603.28|1043.60|58.85|0.00|980.96|1039.81|377.68| +2451878|74531|11023|67463|912315|6767|30819|2|267|2244|43|41.88|67.00|1.34|0.00|57.62|1800.84|2881.00|4.03|0.00|57.62|61.65|-1743.22| +2451878|74531|13280|67463|912315|6767|30819|2|117|2244|31|16.90|17.23|9.30|0.00|288.30|523.90|534.13|23.06|0.00|288.30|311.36|-235.60| +||16103|67463|||30819||168|2244||||27.61||1628.99|||||||| +2451878|74531|4499|67463|912315|6767|30819|2|242|2244|31|39.84|58.96|13.56|290.04|420.36|1235.04|1827.76|9.12|290.04|130.32|139.44|-1104.72| +2451878|74531|14491|67463|912315|6767|30819|2|222|2244|75|97.48|149.14|143.17|0.00|10737.75|7311.00|11185.50|429.51|0.00|10737.75|11167.26|3426.75| +2451878|74531|8797|67463|912315|6767|30819|2|72|2244|6|45.09|54.55|7.63|0.00|45.78|270.54|327.30|3.66|0.00|45.78|49.44|-224.76| +2451878|74531|1819|67463|912315|6767|30819|2|247|2244|8|70.00|123.90|101.59|0.00|812.72|560.00|991.20|48.76|0.00|812.72|861.48|252.72| +2451878||11120|67463|912315|6767||2||2244|19|||49.68|||||||943.92|1000.55|347.70| +2451878|74531|2942|67463|912315|6767|30819|2|216|2244|25|34.03|55.80|50.77|0.00|1269.25|850.75|1395.00|101.54|0.00|1269.25|1370.79|418.50| +2451878|74531|9283|67463|912315|6767|30819|2|140|2244|28|76.85|117.58|94.06|0.00|2633.68|2151.80|3292.24|0.00|0.00|2633.68|2633.68|481.88| +2451878|74531|2750|67463|912315|6767|30819|2|117|2244|35|98.83|113.65|70.46|0.00|2466.10|3459.05|3977.75|73.98|0.00|2466.10|2540.08|-992.95| +2451878|74531|2941|67463|912315|6767|30819|2|193|2244|14|10.53|12.53|9.39|55.21|131.46|147.42|175.42|5.33|55.21|76.25|81.58|-71.17| +2451878|74531|12011|67463|912315|6767|30819|2|289|2244|21|40.78|57.90|20.84|0.00|437.64|856.38|1215.90|35.01|0.00|437.64|472.65|-418.74| +2451878|74531|6050|67463|912315|6767|30819|2|237|2244|21|64.07|69.19|45.66|949.27|958.86|1345.47|1452.99|0.09|949.27|9.59|9.68|-1335.88| +2452130|57191|5917|2949|118116|5283|23650|8|80|2245|68|32.56|59.25|27.84|0.00|1893.12|2214.08|4029.00|94.65|0.00|1893.12|1987.77|-320.96| +2452130|57191|13905|2949|118116|5283|23650|8|223|2245|58|15.31|16.84|5.38|0.00|312.04|887.98|976.72|28.08|0.00|312.04|340.12|-575.94| +2452130|57191|15587|||5283|23650|8|94|2245|70|||88.83||6218.10|3777.20|7403.20|435.26|||6653.36|| +2452130|57191|11741|2949|118116|5283|23650|8|274|2245|89|70.11|96.75|56.11|0.00|4993.79|6239.79|8610.75|249.68|0.00|4993.79|5243.47|-1246.00| +2452130|57191|1535|2949|118116|5283|23650|8|271|2245|6|58.05|98.10|97.11|0.00|582.66|348.30|588.60|29.13|0.00|582.66|611.79|234.36| +2452130|57191|16765|2949|118116|5283|23650|8|134|2245|12|29.10|36.66|3.66|0.00|43.92|349.20|439.92|3.95|0.00|43.92|47.87|-305.28| +2452130|57191|12901|2949|118116|5283|23650|8|276|2245|28|95.00|152.00|112.48|0.00|3149.44|2660.00|4256.00|62.98|0.00|3149.44|3212.42|489.44| +2452130|57191|6671|2949|118116|5283|23650|8|100|2245|46|86.12|143.82|96.35|4387.77|4432.10|3961.52|6615.72|1.32|4387.77|44.33|45.65|-3917.19| +2450921|51761|16975|92397|546691|1785|46479|1|10|2246|62|86.13|171.39|149.10|0.00|9244.20|5340.06|10626.18|0.00|0.00|9244.20|9244.20|3904.14| +2450921|51761|4102|92397|546691|1785|46479|1|268|2246|17|20.14|33.43|23.40|0.00|397.80|342.38|568.31|19.89|0.00|397.80|417.69|55.42| +2450921|51761|17792|92397|546691|1785|46479|1|278|2246|6|39.48|47.37|39.31|0.00|235.86|236.88|284.22|2.35|0.00|235.86|238.21|-1.02| +2450921|51761|5564|92397|546691|1785|46479|1|8|2246|37|30.72|59.28|27.26|0.00|1008.62|1136.64|2193.36|20.17|0.00|1008.62|1028.79|-128.02| +2450921|51761|5764|92397|546691|1785|46479|1|230|2246|66|27.81|43.93|36.90|0.00|2435.40|1835.46|2899.38|24.35|0.00|2435.40|2459.75|599.94| +2450921|51761|5563|92397|546691|1785|46479|1|29|2246|19|89.11|107.82|9.70|7.37|184.30|1693.09|2048.58|3.53|7.37|176.93|180.46|-1516.16| +2450921|51761|12907|92397|546691|1785|46479|1|24|2246|18|24.93|40.88|4.08|24.23|73.44|448.74|735.84|3.93|24.23|49.21|53.14|-399.53| +2450921|51761|12523|92397|546691|1785|46479|1|235|2246|1|73.75|146.02|32.12|0.00|32.12|73.75|146.02|1.28|0.00|32.12|33.40|-41.63| +2450921|51761|4420|92397|546691|1785|46479|1|105|2246|58|69.28|122.62|34.33|1792.02|1991.14|4018.24|7111.96|5.97|1792.02|199.12|205.09|-3819.12| +2452601|47179|7929|71843|1709210|2577|23591|10|41|2247|36|29.12|57.07|4.56|0.00|164.16|1048.32|2054.52|11.49|0.00|164.16|175.65|-884.16| +2452601|47179|10578|71843|1709210|2577|23591|10|18|2247|10|47.29|52.96|16.41|77.12|164.10|472.90|529.60|3.47|77.12|86.98|90.45|-385.92| +2452601|47179|13254|71843|1709210|2577|23591|10|184|2247|54|75.09|111.13|48.89|0.00|2640.06|4054.86|6001.02|52.80|0.00|2640.06|2692.86|-1414.80| +2452601|47179|16290|71843|1709210|2577|23591|10|125|2247|51|67.71|106.98|90.93|0.00|4637.43|3453.21|5455.98|231.87|0.00|4637.43|4869.30|1184.22| +2452601|47179|16845|71843|1709210|2577|23591|10|242|2247|60|70.45|72.56|44.26|26.55|2655.60|4227.00|4353.60|78.87|26.55|2629.05|2707.92|-1597.95| +2452601|47179|16758|71843|1709210|2577|23591|10|71|2247|51|39.17|57.57|11.51|170.23|587.01|1997.67|2936.07|12.50|170.23|416.78|429.28|-1580.89| +2452601|47179|17628|71843|1709210|2577|23591|10|174|2247|17|11.86|12.57|0.00|0.00|0.00|201.62|213.69|0.00|0.00|0.00|0.00|-201.62| +2452601|47179|2635|71843|1709210|2577|23591|10|21|2247|10|90.04|131.45|31.54|0.00|315.40|900.40|1314.50|3.15|0.00|315.40|318.55|-585.00| +2452601|47179|2427|71843|1709210|2577|23591|10|65|2247|16|53.54|58.35|3.50|0.00|56.00|856.64|933.60|2.24|0.00|56.00|58.24|-800.64| +2452601|47179|15147|71843|1709210|2577|23591|10|291|2247|79|62.01|83.09|1.66|0.00|131.14|4898.79|6564.11|0.00|0.00|131.14|131.14|-4767.65| +2452601|47179|12321|71843|1709210|2577|23591|10|195|2247|90|92.03|178.53|112.47|9008.84|10122.30|8282.70|16067.70|0.00|9008.84|1113.46|1113.46|-7169.24| +2452601|47179|7831|71843|1709210|2577|23591|10|208|2247|83|17.57|34.61|27.34|90.76|2269.22|1458.31|2872.63|196.06|90.76|2178.46|2374.52|720.15| +2452601|47179|12990|71843|1709210|2577|23591|10|216|2247|28|83.43|104.28|26.07|0.00|729.96|2336.04|2919.84|7.29|0.00|729.96|737.25|-1606.08| +2452055||5167||||23376|8|48|2248|91||128.96||0.00|3989.44|7244.51|11735.36||0.00|3989.44||| +2452055|69732|1501|44842|838722|249|23376|8|129|2248|67|21.64|25.31|19.99|0.00|1339.33|1449.88|1695.77|107.14|0.00|1339.33|1446.47|-110.55| +2452055|69732|923|44842|838722|249|23376|8|89|2248|56|90.90|131.80|27.67|0.00|1549.52|5090.40|7380.80|15.49|0.00|1549.52|1565.01|-3540.88| +2452055|69732|9731|44842|838722|249|23376|8|271|2248|69|91.26|96.73|95.76|0.00|6607.44|6296.94|6674.37|330.37|0.00|6607.44|6937.81|310.50| +2452055|69732|14587|44842|838722|249|23376|8|39|2248|36|65.38|125.52|90.37|0.00|3253.32|2353.68|4518.72|292.79|0.00|3253.32|3546.11|899.64| +2452055||1171|44842|||23376|8|210|2248|86||101.43|53.75||4622.50|||416.02||4622.50||-2527.54| +2452055|69732|5033|44842|838722|249|23376|8|214|2248|91|88.94|108.50|54.25|0.00|4936.75|8093.54|9873.50|296.20|0.00|4936.75|5232.95|-3156.79| +2452055|69732|11043|44842|838722|249|23376|8|284|2248|55|13.11|26.22|10.75|0.00|591.25|721.05|1442.10|23.65|0.00|591.25|614.90|-129.80| +2452055|69732|8973|44842|838722|249|23376|8|220|2248|49|90.84|124.45|114.49|729.30|5610.01|4451.16|6098.05|195.22|729.30|4880.71|5075.93|429.55| +2452055|69732|15379|44842|838722|249|23376|8|219|2248|31|96.63|151.70|47.02|0.00|1457.62|2995.53|4702.70|0.00|0.00|1457.62|1457.62|-1537.91| +2452055|69732|2367|44842|838722|249|23376|8|29|2248|77|6.30|11.65|5.94|0.00|457.38|485.10|897.05|4.57|0.00|457.38|461.95|-27.72| +2452055|69732|8027|44842|838722|249|23376|8|108|2248|69|92.94|117.10|81.97|0.00|5655.93|6412.86|8079.90|282.79|0.00|5655.93|5938.72|-756.93| +2452055|69732|7353|44842|838722|249|23376|8|9|2248|64|77.24|83.41|71.73|0.00|4590.72|4943.36|5338.24|367.25|0.00|4590.72|4957.97|-352.64| +2452213|36810|735|43157|1206930|1550|12523|1|165|2249|58|18.65|30.39|17.62|0.00|1021.96|1081.70|1762.62|51.09|0.00|1021.96|1073.05|-59.74| +2452213|36810|7217|43157|1206930|1550|12523|1|223|2249|82|76.40|77.16|43.20|0.00|3542.40|6264.80|6327.12|141.69|0.00|3542.40|3684.09|-2722.40| +2452213|36810|11853|43157|1206930|1550|12523|1|203|2249|87|39.45|58.78|24.09|0.00|2095.83|3432.15|5113.86|62.87|0.00|2095.83|2158.70|-1336.32| +2452213||13967|||||1|136|2249|28|11.76|||||329.28|378.56|0.07|||7.63|| +2452213|36810|17823|43157|1206930|1550|12523|1|76|2249|83|58.60|59.18|33.14|0.00|2750.62|4863.80|4911.94|55.01|0.00|2750.62|2805.63|-2113.18| +2452213|36810|8139|43157|1206930|1550|12523|1|110|2249|96|99.59|118.51|109.02|0.00|10465.92|9560.64|11376.96|418.63|0.00|10465.92|10884.55|905.28| +2452213|36810|14029|43157|1206930|1550|12523|1|129|2249|36|10.62|10.93|10.16|0.00|365.76|382.32|393.48|18.28|0.00|365.76|384.04|-16.56| +2452213|36810|12081|43157|1206930|1550|12523|1|64|2249|57|79.51|158.22|33.22|0.00|1893.54|4532.07|9018.54|75.74|0.00|1893.54|1969.28|-2638.53| +2452213|36810|11789|43157|1206930|1550|12523|1|144|2249|66|80.51|120.76|54.34|0.00|3586.44|5313.66|7970.16|215.18|0.00|3586.44|3801.62|-1727.22| +2452213|36810|16021|43157|1206930|1550|12523|1|7|2249|36|17.33|19.23|4.42|0.00|159.12|623.88|692.28|4.77|0.00|159.12|163.89|-464.76| +2452213|36810|543|43157|1206930|1550|12523|1|253|2249|64|24.64|40.90|35.17|0.00|2250.88|1576.96|2617.60|67.52|0.00|2250.88|2318.40|673.92| +2452213|36810|17193|43157|1206930|1550|12523|1|78|2249|72|96.14|167.28|46.83|0.00|3371.76|6922.08|12044.16|101.15|0.00|3371.76|3472.91|-3550.32| +2451112|47784|9442|53555|116194|3633|21872|8|6|2250|26|20.94|21.35|17.29|0.00|449.54|544.44|555.10|26.97|0.00|449.54|476.51|-94.90| +2451112|47784|8848|53555|116194|3633|21872|8|258|2250|88|53.57|96.96|6.78|0.00|596.64|4714.16|8532.48|17.89|0.00|596.64|614.53|-4117.52| +2451112|47784|4052|53555|116194|3633|21872|8|192|2250|34|1.70|2.97|2.16|0.00|73.44|57.80|100.98|1.46|0.00|73.44|74.90|15.64| +2451112|47784|15160|53555|116194|3633|21872|8|73|2250|99|37.72|75.06|52.54|0.00|5201.46|3734.28|7430.94|52.01|0.00|5201.46|5253.47|1467.18| +2451112|47784|13135|53555|116194|3633|21872|8|152|2250|64|5.24|6.18|1.29|42.10|82.56|335.36|395.52|0.00|42.10|40.46|40.46|-294.90| +2451112|47784|16834|53555|116194|3633|21872|8|29|2250|100|49.79|78.17|32.83|0.00|3283.00|4979.00|7817.00|131.32|0.00|3283.00|3414.32|-1696.00| +|47784|13729|53555|||21872|||2250|59||87.65||0.00||4925.32|5171.35||0.00|3722.90|3722.90|-1202.42| +2451112|47784|12637|53555|116194|3633|21872|8|299|2250|64|61.60|72.07|52.61|0.00|3367.04|3942.40|4612.48|67.34|0.00|3367.04|3434.38|-575.36| +2451112|47784|10130|53555|116194|3633|21872|8|152|2250|48|80.78|131.67|43.45|0.00|2085.60|3877.44|6320.16|125.13|0.00|2085.60|2210.73|-1791.84| +2451112|47784|3680|53555|116194|3633|21872|8|48|2250|95|5.94|8.49|6.45|386.03|612.75|564.30|806.55|2.26|386.03|226.72|228.98|-337.58| +2451112|47784|3562|53555|116194|3633|21872|8|19|2250|61|92.12|144.62|2.89|0.00|176.29|5619.32|8821.82|7.05|0.00|176.29|183.34|-5443.03| +2451112|47784|7742|53555|116194|3633|21872|8|41|2250|47|72.08|104.51|62.70|854.60|2946.90|3387.76|4911.97|146.46|854.60|2092.30|2238.76|-1295.46| +2451112|47784|6130|53555|116194|3633|21872|8|155|2250|79|42.86|45.86|28.89|0.00|2282.31|3385.94|3622.94|205.40|0.00|2282.31|2487.71|-1103.63| +2451112|47784|4400|53555|116194|3633|21872|8|268|2250|94|49.12|54.03|29.17|274.19|2741.98|4617.28|5078.82|24.67|274.19|2467.79|2492.46|-2149.49| +2451112|47784|13498|53555|116194|3633|21872|8|170|2250|80|53.26|58.05|17.99|0.00|1439.20|4260.80|4644.00|43.17|0.00|1439.20|1482.37|-2821.60| +2451112|47784|15163|53555|116194|3633|21872|8|147|2250|63|80.83|105.07|23.11|0.00|1455.93|5092.29|6619.41|0.00|0.00|1455.93|1455.93|-3636.36| +2451677|68410|7157|50666|1143417|||1|212|2251||||7.54|||3591.28||5.17||||| +2451677|68410|139|50666|1143417|1768|5672|1|219|2251|67|50.43|93.79|8.44|0.00|565.48|3378.81|6283.93|16.96|0.00|565.48|582.44|-2813.33| +2451677|68410|10460|50666|1143417|1768|5672|1|91|2251|45|39.00|42.12|8.42|261.44|378.90|1755.00|1895.40|7.04|261.44|117.46|124.50|-1637.54| +2451677|68410|8767|50666|1143417|1768|5672|1|88|2251|20|11.00|12.54|4.38|0.00|87.60|220.00|250.80|7.88|0.00|87.60|95.48|-132.40| +2451677|68410|7574|50666|1143417|1768|5672|1|135|2251|7|82.66|95.88|64.23|0.00|449.61|578.62|671.16|0.00|0.00|449.61|449.61|-129.01| +2451677|68410|11960|50666|1143417|1768|5672|1|180|2251|20|26.98|42.08|5.89|0.00|117.80|539.60|841.60|1.17|0.00|117.80|118.97|-421.80| +2451677|68410|14803|50666|1143417|1768|5672|1|169|2251|29|52.36|89.53|38.49|0.00|1116.21|1518.44|2596.37|44.64|0.00|1116.21|1160.85|-402.23| +2451677|68410|13502|50666|1143417|1768|5672|1|251|2251|65|77.41|130.04|14.30|0.00|929.50|5031.65|8452.60|74.36|0.00|929.50|1003.86|-4102.15| +2451677|68410|677|50666|1143417|1768|5672|1|15|2251|20|56.15|65.69|53.86|0.00|1077.20|1123.00|1313.80|0.00|0.00|1077.20|1077.20|-45.80| +2451677|68410|12533|50666|1143417|1768|5672|1|222|2251|30|51.84|60.13|45.69|0.00|1370.70|1555.20|1803.90|13.70|0.00|1370.70|1384.40|-184.50| +2451677||10346|50666|1143417||5672|||2251||63.85|87.47|||1819.20|2554.00|||||1837.39|| +2451677|68410|4034|50666|1143417|1768|5672|1|117|2251|51|59.30|91.91|53.30|0.00|2718.30|3024.30|4687.41|135.91|0.00|2718.30|2854.21|-306.00| +2451931|54399|9835|6762|1707073|7031|28189|7|79|2252|10|72.85|136.95|13.69|0.00|136.90|728.50|1369.50|6.84|0.00|136.90|143.74|-591.60| +2451931|54399|1559|6762|1707073|7031|28189|7|196|2252|3|1.82|3.07|0.61|0.00|1.83|5.46|9.21|0.16|0.00|1.83|1.99|-3.63| +2451931|54399|12495|6762|1707073|7031|28189|7|234|2252|75|65.40|120.99|26.61|0.00|1995.75|4905.00|9074.25|19.95|0.00|1995.75|2015.70|-2909.25| +2451931|54399|10619|6762|1707073|7031|28189|7|135|2252|30|40.53|47.01|0.00|0.00|0.00|1215.90|1410.30|0.00|0.00|0.00|0.00|-1215.90| +2451931|54399|12061|6762|1707073|7031|28189|7|84|2252|21|60.86|62.07|32.27|454.03|677.67|1278.06|1303.47|15.65|454.03|223.64|239.29|-1054.42| +2451931|54399|13565|6762|1707073|7031|28189|7|2|2252|19|71.84|94.11|30.11|0.00|572.09|1364.96|1788.09|0.00|0.00|572.09|572.09|-792.87| +2451931|54399|14943|6762|1707073|7031|28189|7|187|2252|64|43.28|74.44|31.26|0.00|2000.64|2769.92|4764.16|160.05|0.00|2000.64|2160.69|-769.28| +2451931|54399|15365|6762|1707073|7031|28189|7|65|2252|70|56.09|57.77|5.19|225.24|363.30|3926.30|4043.90|11.04|225.24|138.06|149.10|-3788.24| +2451071|67329|9919|29926|1507675|7024|3973|10|60|2253|4|54.54|98.17|19.63|47.89|78.52|218.16|392.68|1.22|47.89|30.63|31.85|-187.53| +2451071|67329|6769|29926|1507675|7024|3973|10|178|2253|83|77.03|131.72|13.17|0.00|1093.11|6393.49|10932.76|65.58|0.00|1093.11|1158.69|-5300.38| +2451071|67329|7549|||7024||||2253||72.28||15.42|0.00||4264.52|8272.98|63.68|0.00|909.78||| +2451071|67329|10760|29926|1507675|7024|3973|10|104|2253|50|47.25|67.09|41.59|0.00|2079.50|2362.50|3354.50|62.38|0.00|2079.50|2141.88|-283.00| +2451071|67329|11554|29926|1507675|7024|3973|10|162|2253|98|46.27|78.19|28.93|2665.03|2835.14|4534.46|7662.62|1.70|2665.03|170.11|171.81|-4364.35| +2451071|67329|10222|29926|1507675|7024|3973|10|159|2253|2|41.30|81.77|77.68|0.00|155.36|82.60|163.54|13.98|0.00|155.36|169.34|72.76| +2451071|67329|14966|29926|1507675|7024|3973|10|298|2253|13|94.57|117.26|28.14|153.64|365.82|1229.41|1524.38|16.97|153.64|212.18|229.15|-1017.23| +2451071|67329|8926|29926|1507675|7024|3973|10|76|2253|14|87.72|117.54|68.17|286.31|954.38|1228.08|1645.56|6.68|286.31|668.07|674.75|-560.01| +2451071|67329|16039|29926|1507675|7024|3973|10|97|2253|49|36.14|54.93|37.90|1299.97|1857.10|1770.86|2691.57|33.42|1299.97|557.13|590.55|-1213.73| +2452608|37272|823|92550|892102|2501|19653|4|263|2254|90|45.21|47.01|33.37|0.00|3003.30|4068.90|4230.90|90.09|0.00|3003.30|3093.39|-1065.60| +2452608|37272|14658|92550|892102|2501|19653|4|54|2254|44|20.10|20.30|18.87|0.00|830.28|884.40|893.20|49.81|0.00|830.28|880.09|-54.12| +2452608|37272|10593|92550|892102|2501|19653|4|271|2254|6|40.75|45.64|17.34|0.00|104.04|244.50|273.84|9.36|0.00|104.04|113.40|-140.46| +2452608|37272|12555|92550|892102|2501|19653|4|125|2254|57|27.62|40.87|1.22|0.00|69.54|1574.34|2329.59|2.78|0.00|69.54|72.32|-1504.80| +2452608|37272|10632|92550|892102|2501|19653|4|174|2254|81|24.22|41.41|9.11|339.43|737.91|1961.82|3354.21|11.95|339.43|398.48|410.43|-1563.34| +2452608|37272|2655|92550|892102|2501|19653|4|76|2254|16|73.97|76.18|6.09|0.00|97.44|1183.52|1218.88|4.87|0.00|97.44|102.31|-1086.08| +2452608|37272|1051|92550|892102|2501|19653|4|142|2254|1|22.14|40.51|22.68|0.00|22.68|22.14|40.51|2.04|0.00|22.68|24.72|0.54| +2452608|37272|1075|92550|892102|2501|19653|4|180|2254|100|83.87|117.41|66.92|0.00|6692.00|8387.00|11741.00|334.60|0.00|6692.00|7026.60|-1695.00| +2452608|37272|4161|92550|892102|2501|19653|4|300|2254|83|36.61|73.22|16.84|0.00|1397.72|3038.63|6077.26|125.79|0.00|1397.72|1523.51|-1640.91| +2452608|37272|17673|92550|892102|2501|19653|4|19|2254|73|15.89|16.52|15.19|0.00|1108.87|1159.97|1205.96|99.79|0.00|1108.87|1208.66|-51.10| +2452608|37272|7581|92550|892102|2501|19653|4|45|2254|57|70.84|138.84|40.26|0.00|2294.82|4037.88|7913.88|91.79|0.00|2294.82|2386.61|-1743.06| +2452608|37272|9991|92550|892102|2501|19653|4|130|2254|67|80.62|141.08|126.97|0.00|8506.99|5401.54|9452.36|510.41|0.00|8506.99|9017.40|3105.45| +2452608|37272|4597|92550|892102|2501|19653|4|257|2254|71|75.26|91.06|46.44|0.00|3297.24|5343.46|6465.26|0.00|0.00|3297.24|3297.24|-2046.22| +2452608|37272|7332|92550|892102|2501|19653|4|231|2254|70|26.58|52.09|48.44|0.00|3390.80|1860.60|3646.30|0.00|0.00|3390.80|3390.80|1530.20| +2452608|37272|15973|92550|892102|2501|19653|4|189|2254|89|87.42|111.02|87.70|0.00|7805.30|7780.38|9880.78|234.15|0.00|7805.30|8039.45|24.92| +2451808|31492|7355|7656|1275846|5989|17732|2|283|2255|66|21.40|23.75|19.95|0.00|1316.70|1412.40|1567.50|65.83|0.00|1316.70|1382.53|-95.70| +2451808|31492|12896|7656|1275846|5989|17732|2|150|2255|89|57.68|70.94|46.82|0.00|4166.98|5133.52|6313.66|41.66|0.00|4166.98|4208.64|-966.54| +2451808|31492|6341|7656|1275846|5989|17732|2|62|2255|58|33.90|34.57|29.03|0.00|1683.74|1966.20|2005.06|84.18|0.00|1683.74|1767.92|-282.46| +2451808|31492|10297|7656|1275846|5989|17732|2|101|2255|49|92.72|121.46|117.81|1962.71|5772.69|4543.28|5951.54|266.69|1962.71|3809.98|4076.67|-733.30| +2451808|31492|7904|7656|1275846|5989|17732|2|294|2255|27|8.51|9.70|4.75|0.00|128.25|229.77|261.90|5.13|0.00|128.25|133.38|-101.52| +2451808|31492|14447|7656|1275846|5989|17732|2|263|2255|41|73.97|142.02|112.19|0.00|4599.79|3032.77|5822.82|367.98|0.00|4599.79|4967.77|1567.02| +2451808|31492|4481|7656|1275846|5989|17732|2|150|2255|33|85.17|126.90|86.29|0.00|2847.57|2810.61|4187.70|0.00|0.00|2847.57|2847.57|36.96| +2451808|31492|14666|7656|1275846|5989|17732|2|9|2255|70|17.84|31.93|0.31|0.00|21.70|1248.80|2235.10|1.95|0.00|21.70|23.65|-1227.10| +2451808|31492|12421|7656|1275846|5989|17732|2|247|2255|11|21.93|41.00|6.97|0.00|76.67|241.23|451.00|2.30|0.00|76.67|78.97|-164.56| +2451808|31492|15859|7656|1275846|5989|17732|2|92|2255|87|6.18|6.61|0.72|27.56|62.64|537.66|575.07|2.10|27.56|35.08|37.18|-502.58| +2451831|49023|4115|70017|1089585|5153|24321|4|255|2256|58|89.33|96.47|82.96|3849.34|4811.68|5181.14|5595.26|19.24|3849.34|962.34|981.58|-4218.80| +2451831|49023|9578|70017|1089585|5153|24321|4|165|2256|93|5.99|11.02|2.42|0.00|225.06|557.07|1024.86|18.00|0.00|225.06|243.06|-332.01| +2451831|49023|11342|70017|1089585|5153|24321|4|67|2256|26|77.17|106.49|74.54|0.00|1938.04|2006.42|2768.74|155.04|0.00|1938.04|2093.08|-68.38| +2451831|49023|17912|70017|1089585|5153|24321|4|235|2256|51|46.65|62.97|28.33|0.00|1444.83|2379.15|3211.47|72.24|0.00|1444.83|1517.07|-934.32| +2451831|49023|6139|70017|1089585|5153|24321|4|37|2256|12|12.51|19.64|9.82|117.84|117.84|150.12|235.68|0.00|117.84|0.00|0.00|-150.12| +|49023|5249|70017||||4||2256|43||||||835.06||34.86|||1197.15|| +2451831|49023|15167|70017|1089585|5153|24321|4|266|2256|40|49.81|67.24|60.51|0.00|2420.40|1992.40|2689.60|96.81|0.00|2420.40|2517.21|428.00| +2451831|49023|5797|70017|1089585|5153|24321|4|249|2256|73|15.47|22.89|12.58|0.00|918.34|1129.31|1670.97|82.65|0.00|918.34|1000.99|-210.97| +2451831|49023|9470|70017|1089585|5153|24321|4|55|2256|60|66.41|68.40|23.25|0.00|1395.00|3984.60|4104.00|0.00|0.00|1395.00|1395.00|-2589.60| +2451831|49023|2474|70017|1089585|5153|24321|4|240|2256|78|79.31|155.44|32.64|0.00|2545.92|6186.18|12124.32|229.13|0.00|2545.92|2775.05|-3640.26| +2451831|49023|16934|70017|1089585|5153|24321|4|257|2256|92|79.12|143.99|82.07|2944.67|7550.44|7279.04|13247.08|368.46|2944.67|4605.77|4974.23|-2673.27| +2451831|49023|9145|70017|1089585|5153|24321|4|34|2256|18|29.06|38.35|11.88|59.87|213.84|523.08|690.30|13.85|59.87|153.97|167.82|-369.11| +2451831|49023|5222|70017|1089585|5153|24321|4|158|2256|98|30.30|38.48|26.16|0.00|2563.68|2969.40|3771.04|102.54|0.00|2563.68|2666.22|-405.72| +2451831|49023|17708|70017|1089585|5153|24321|4|104|2256|86|43.62|66.30|51.05|0.00|4390.30|3751.32|5701.80|307.32|0.00|4390.30|4697.62|638.98| +2451831|49023|3535|70017|1089585|5153|24321|4|218|2256|6|98.93|170.15|51.04|0.00|306.24|593.58|1020.90|15.31|0.00|306.24|321.55|-287.34| +2451831|49023|8777|70017|1089585|5153|24321|4|265|2256|30|98.97|163.30|104.51|0.00|3135.30|2969.10|4899.00|250.82|0.00|3135.30|3386.12|166.20| +2451513|28873|17416|84083|1163234|3185|24736|2|232|2257|44|28.82|31.99|14.39|0.00|633.16|1268.08|1407.56|0.00|0.00|633.16|633.16|-634.92| +2451513|28873|2768|84083|1163234|3185|24736|2|50|2257|26|54.16|94.78|71.08|0.00|1848.08|1408.16|2464.28|18.48|0.00|1848.08|1866.56|439.92| +2451513|28873|17984|84083|1163234|3185|24736|2|186|2257|59|29.38|35.54|22.74|670.83|1341.66|1733.42|2096.86|40.24|670.83|670.83|711.07|-1062.59| +2451513|28873|2998|84083|1163234|3185|24736|2|84|2257|21|64.45|80.56|46.72|0.00|981.12|1353.45|1691.76|0.00|0.00|981.12|981.12|-372.33| +2451513|28873|3712|84083|1163234|3185|24736|2|148|2257|55|3.19|5.48|4.27|0.00|234.85|175.45|301.40|9.39|0.00|234.85|244.24|59.40| +2451513|28873|6182||1163234|3185|24736|2||2257|31||128.38|3.85|0.00||2456.75|3979.78|2.38|0.00|119.35||| +2451513|28873|16166|84083|1163234|3185|24736|2|139|2257|4|14.76|28.19|27.06|0.00|108.24|59.04|112.76|9.74|0.00|108.24|117.98|49.20| +2451513|28873|10489|84083|1163234|3185|24736|2|139|2257|44|48.74|66.77|9.34|0.00|410.96|2144.56|2937.88|24.65|0.00|410.96|435.61|-1733.60| +2451513|28873|11374||1163234|3185|||121|2257|9|9.99|16.18||0.00|59.67||145.62||0.00|59.67|59.67|-30.24| +2451513|28873|1180|84083|1163234|3185|24736|2|175|2257|10|9.30|16.55|3.80|0.00|38.00|93.00|165.50|1.14|0.00|38.00|39.14|-55.00| +2452557|61170|2947|77894|812451|7008|34812|4|95|2258|45|13.75|19.38|8.91|0.00|400.95|618.75|872.10|20.04|0.00|400.95|420.99|-217.80| +2452557|61170|15837|77894|812451|7008|34812|4|118|2258|71|63.28|67.07|42.25|0.00|2999.75|4492.88|4761.97|179.98|0.00|2999.75|3179.73|-1493.13| +2452557|61170|4579|77894|812451|7008|34812|4|189|2258|92|53.11|72.76|66.21|0.00|6091.32|4886.12|6693.92|365.47|0.00|6091.32|6456.79|1205.20| +2452557|61170|5919|77894|812451|7008|34812|4|248|2258|3|72.59|113.24|50.95|0.00|152.85|217.77|339.72|1.52|0.00|152.85|154.37|-64.92| +2452557|61170|6229|77894|812451|7008|34812|4|58|2258|8|89.18|131.09|106.18|543.64|849.44|713.44|1048.72|24.46|543.64|305.80|330.26|-407.64| +2452557|61170|11142|77894|812451|7008|34812|4|177|2258|94|48.29|86.43|16.42|663.69|1543.48|4539.26|8124.42|61.58|663.69|879.79|941.37|-3659.47| +2452557|61170|3738|77894|812451|7008|34812|4|191|2258|65|22.57|31.82|11.13|0.00|723.45|1467.05|2068.30|21.70|0.00|723.45|745.15|-743.60| +2452557|61170|13471|77894|812451|7008|34812|4|176|2258|34|64.61|124.69|6.23|0.00|211.82|2196.74|4239.46|19.06|0.00|211.82|230.88|-1984.92| +2452557|61170|3877|77894|812451|7008|34812|4|190|2258|25|51.73|73.45|29.38|95.48|734.50|1293.25|1836.25|25.56|95.48|639.02|664.58|-654.23| +2452557|61170|16413|77894|812451|7008|34812|4|166|2258|67|12.68|22.82|7.98|0.00|534.66|849.56|1528.94|0.00|0.00|534.66|534.66|-314.90| +2452557|61170|13752|77894|812451|7008|34812|4|10|2258|83|97.21|139.98|54.59|0.00|4530.97|8068.43|11618.34|0.00|0.00|4530.97|4530.97|-3537.46| +2452557|61170|12330|77894|812451|7008|34812|4|188|2258|31|1.77|2.24|0.42|0.00|13.02|54.87|69.44|0.13|0.00|13.02|13.15|-41.85| +2452557|61170|8472|77894|812451|7008|34812|4|46|2258|83|22.35|29.27|28.68|452.28|2380.44|1855.05|2429.41|96.40|452.28|1928.16|2024.56|73.11| +2451242|69568|13219|31206|825140|7074|35808|7|15|2259|35|14.13|25.29|23.01|0.00|805.35|494.55|885.15|64.42|0.00|805.35|869.77|310.80| +2451242|69568|9229|31206|825140|7074|35808|7|297|2259|22|8.40|9.40|2.53|0.00|55.66|184.80|206.80|0.00|0.00|55.66|55.66|-129.14| +2451242|69568|15614|31206|825140|7074|35808|7|52|2259|60|90.20|149.73|86.84|0.00|5210.40|5412.00|8983.80|208.41|0.00|5210.40|5418.81|-201.60| +2451242|69568|8156|31206|825140|7074|35808|7|149|2259|8|15.16|24.71|5.68|0.00|45.44|121.28|197.68|0.90|0.00|45.44|46.34|-75.84| +2451242|69568|14131|31206|825140|7074|35808|7|252|2259|11|26.53|49.61|11.41|0.00|125.51|291.83|545.71|11.29|0.00|125.51|136.80|-166.32| +2451242|69568|16111||825140|||7||2259|||120.34|86.64||5718.24|5124.24|7942.44||||5775.42|| +2451242|69568|15538|31206|825140|7074|35808|7|206|2259|100|91.86|173.61|86.80|0.00|8680.00|9186.00|17361.00|86.80|0.00|8680.00|8766.80|-506.00| +2451242|69568|2482|31206|825140|7074|35808|7|49|2259|37|97.42|136.38|62.73|0.00|2321.01|3604.54|5046.06|46.42|0.00|2321.01|2367.43|-1283.53| +2451242|69568|758|31206|825140|7074|35808|7|95|2259|66|51.97|53.52|9.63|0.00|635.58|3430.02|3532.32|57.20|0.00|635.58|692.78|-2794.44| +2451242|69568|15842|31206|825140|7074|35808|7|16|2259|5|16.33|16.98|5.43|7.60|27.15|81.65|84.90|0.97|7.60|19.55|20.52|-62.10| +2451242|69568|3385|31206|825140|7074|35808|7|11|2259|72|21.14|36.78|21.70|0.00|1562.40|1522.08|2648.16|46.87|0.00|1562.40|1609.27|40.32| +2451242|69568|3104|31206|825140|7074|35808|7|192|2259|7|29.15|54.21|15.17|0.00|106.19|204.05|379.47|6.37|0.00|106.19|112.56|-97.86| +||16694|31206|825140|||||2259|54|20.58||||2079.54|1111.32|||||2204.31|968.22| +2451242|69568|326|31206|825140|7074|35808|7|164|2259|44|17.52|33.11|25.16|0.00|1107.04|770.88|1456.84|0.00|0.00|1107.04|1107.04|336.16| +2451242|69568|5492|31206|825140|7074|35808|7|225|2259|25|21.41|30.40|10.03|152.95|250.75|535.25|760.00|0.00|152.95|97.80|97.80|-437.45| +2450957|48785|6854|68959|329857|5155|19253|4|146|2260|64|29.46|42.42|24.17|0.00|1546.88|1885.44|2714.88|139.21|0.00|1546.88|1686.09|-338.56| +2450957|48785|15577|68959|329857|5155|19253|4|44|2260|39|79.04|126.46|16.43|0.00|640.77|3082.56|4931.94|0.00|0.00|640.77|640.77|-2441.79| +2450957|48785|16280|68959|329857|5155|19253|4|169|2260|29|68.43|124.54|108.34|0.00|3141.86|1984.47|3611.66|94.25|0.00|3141.86|3236.11|1157.39| +2450957|48785|5731|68959|329857|5155|19253|4|293|2260|1|93.75|151.87|27.33|0.00|27.33|93.75|151.87|1.91|0.00|27.33|29.24|-66.42| +2450957|48785|16642|68959|329857|5155|19253|4|173|2260|87|83.35|95.01|42.75|0.00|3719.25|7251.45|8265.87|37.19|0.00|3719.25|3756.44|-3532.20| +2450957|48785|3500|68959|329857|5155|19253|4|147|2260|88|89.75|90.64|11.78|0.00|1036.64|7898.00|7976.32|62.19|0.00|1036.64|1098.83|-6861.36| +2450957|48785|10580|68959|329857|5155|19253|4|122|2260|100|76.48|123.13|65.25|326.25|6525.00|7648.00|12313.00|61.98|326.25|6198.75|6260.73|-1449.25| +2450957|48785|17080|68959|329857|5155|19253|4|85|2260|10|77.81|139.27|111.41|0.00|1114.10|778.10|1392.70|100.26|0.00|1114.10|1214.36|336.00| +2450957|48785|3517|68959|329857|5155|19253|4|96|2260|99|60.18|82.44|49.46|0.00|4896.54|5957.82|8161.56|146.89|0.00|4896.54|5043.43|-1061.28| +2450938|52363|13963|50643|1899197|4104|11458|1|114|2261|74|61.62|107.83|86.26|0.00|6383.24|4559.88|7979.42|0.00|0.00|6383.24|6383.24|1823.36| +2450938|52363|9938|50643|1899197|4104|11458|1|203|2261|93|40.57|76.27|44.23|0.00|4113.39|3773.01|7093.11|82.26|0.00|4113.39|4195.65|340.38| +2450938|52363|9890|50643|1899197|4104|11458|1|38|2261|66|43.43|61.23|21.43|0.00|1414.38|2866.38|4041.18|56.57|0.00|1414.38|1470.95|-1452.00| +2450938|52363|17674|50643|1899197|4104|11458|1|34|2261|44|33.80|51.71|28.44|0.00|1251.36|1487.20|2275.24|37.54|0.00|1251.36|1288.90|-235.84| +2450938|52363|16915|50643|1899197|4104|11458|1|144|2261|58|37.60|57.15|6.28|0.00|364.24|2180.80|3314.70|0.00|0.00|364.24|364.24|-1816.56| +2450938|52363|5462|50643|1899197|4104|11458|1|255|2261|95|8.14|8.79|5.62|0.00|533.90|773.30|835.05|32.03|0.00|533.90|565.93|-239.40| +2450938|52363|10945|50643|1899197|4104|11458|1|235|2261|69|58.03|63.83|7.02|0.00|484.38|4004.07|4404.27|43.59|0.00|484.38|527.97|-3519.69| +2450938|52363|17684|50643|1899197|4104|11458|1|188|2261|49|2.00|2.80|0.05|0.00|2.45|98.00|137.20|0.09|0.00|2.45|2.54|-95.55| +2450938|52363|2638|50643|1899197|4104|11458|1|196|2261|19|20.30|25.57|21.99|0.00|417.81|385.70|485.83|12.53|0.00|417.81|430.34|32.11| +2452233|58694|13879|70711|1357770|2076|23935|4|205|2262|23|20.37|24.03|3.36|76.50|77.28|468.51|552.69|0.00|76.50|0.78|0.78|-467.73| +2452233|58694|14113|70711|1357770|2076|23935|4|237|2262|59|31.61|55.63|9.45|0.00|557.55|1864.99|3282.17|27.87|0.00|557.55|585.42|-1307.44| +2452233|58694|16727|70711|1357770|2076|23935|4|168|2262|96|41.37|44.26|14.60|0.00|1401.60|3971.52|4248.96|42.04|0.00|1401.60|1443.64|-2569.92| +2452233|58694|16549|70711|1357770|2076|23935|4|182|2262|7|76.82|139.81|50.33|0.00|352.31|537.74|978.67|17.61|0.00|352.31|369.92|-185.43| +2452233|58694|16905|70711|1357770|2076|23935|4|216|2262|74|28.83|55.06|52.85|0.00|3910.90|2133.42|4074.44|195.54|0.00|3910.90|4106.44|1777.48| +2452233|58694|1969|70711|1357770|2076|23935|4|291|2262|90|1.69|2.72|0.16|0.00|14.40|152.10|244.80|0.00|0.00|14.40|14.40|-137.70| +2452233|58694|3771|70711|1357770|2076|23935|4|187|2262|37|16.45|32.73|0.32|0.00|11.84|608.65|1211.01|0.11|0.00|11.84|11.95|-596.81| +2452233|58694|14619|70711|1357770|2076|23935|4|112|2262|23|97.77|155.45|150.78|0.00|3467.94|2248.71|3575.35|69.35|0.00|3467.94|3537.29|1219.23| +2452233|58694|7275|70711|1357770|2076|23935|4|161|2262|27|86.42|103.70|2.07|33.53|55.89|2333.34|2799.90|0.67|33.53|22.36|23.03|-2310.98| +2451082|70950|8006|83468|1818962|2009|9903|10|56|2263|81|23.07|42.91|33.89|0.00|2745.09|1868.67|3475.71|247.05|0.00|2745.09|2992.14|876.42| +2451082|70950|16376|83468|1818962|2009|9903|10|299|2263|36|30.77|58.15|6.97|0.00|250.92|1107.72|2093.40|12.54|0.00|250.92|263.46|-856.80| +2451082|70950|10400|83468|1818962|2009|9903|10|75|2263|91|80.30|131.69|10.53|0.00|958.23|7307.30|11983.79|19.16|0.00|958.23|977.39|-6349.07| +2451082|70950|15334|83468|1818962|2009|9903|10|206|2263|8|25.15|48.03|23.05|47.94|184.40|201.20|384.24|9.55|47.94|136.46|146.01|-64.74| +2451082|70950|12664|83468|1818962|2009|9903|10|27|2263|67|99.86|120.83|30.20|0.00|2023.40|6690.62|8095.61|60.70|0.00|2023.40|2084.10|-4667.22| +2451082|70950|15769|83468|1818962|2009|9903|10|49|2263|66|23.30|33.08|32.08|0.00|2117.28|1537.80|2183.28|190.55|0.00|2117.28|2307.83|579.48| +2451082|70950|10771|83468|1818962|2009|9903|10|62|2263|44|10.19|10.90|1.96|0.00|86.24|448.36|479.60|2.58|0.00|86.24|88.82|-362.12| +2451082|70950|14791|83468|1818962|2009|9903|10|262|2263|65|35.09|51.58|39.20|0.00|2548.00|2280.85|3352.70|76.44|0.00|2548.00|2624.44|267.15| +||7916|83468||2009|||5|2263|23||165.12|21.46|||2020.09|3797.76||||503.45|-1526.51| +2451082|70950|11086|83468|1818962|2009|9903|10|26|2263|14|11.08|14.18|1.27|0.00|17.78|155.12|198.52|0.53|0.00|17.78|18.31|-137.34| +2451082|70950|1123|83468|1818962|2009|9903|10|119|2263|29|59.79|117.18|36.32|926.88|1053.28|1733.91|3398.22|0.00|926.88|126.40|126.40|-1607.51| +2451082|70950|15394|83468|1818962|2009|9903|10|275|2263|57|94.58|102.14|18.38|0.00|1047.66|5391.06|5821.98|10.47|0.00|1047.66|1058.13|-4343.40| +2451082|70950|9394|83468|1818962|2009|9903|10|238|2263|8|7.12|9.61|6.15|1.96|49.20|56.96|76.88|2.36|1.96|47.24|49.60|-9.72| +2451082|70950|17894|83468|1818962|2009|9903|10|275|2263|85|19.57|23.67|20.59|0.00|1750.15|1663.45|2011.95|70.00|0.00|1750.15|1820.15|86.70| +2452600|62117|16419|7405|812633|2308|38336|8|289|2264|74|68.84|120.47|71.07|0.00|5259.18|5094.16|8914.78|262.95|0.00|5259.18|5522.13|165.02| +2452600|62117|13885|7405|812633|2308|38336|8|89|2264|42|33.83|34.50|18.28|0.00|767.76|1420.86|1449.00|61.42|0.00|767.76|829.18|-653.10| +|62117|8167|7405|812633||38336||171|2264|8|92.87|||0.00|234.64||869.20||0.00|||| +2452600|62117|5832|7405|812633|2308|38336|8|111|2264|49|85.17|131.16|40.65|1354.45|1991.85|4173.33|6426.84|25.49|1354.45|637.40|662.89|-3535.93| +2452600|62117|16767|7405|812633|2308|38336|8|90|2264|32|35.34|70.32|59.06|0.00|1889.92|1130.88|2250.24|75.59|0.00|1889.92|1965.51|759.04| +2452600|62117|8250|7405|812633|2308|38336|8|63|2264|89|42.14|56.46|31.05|0.00|2763.45|3750.46|5024.94|221.07|0.00|2763.45|2984.52|-987.01| +2452600|62117|5718|7405|812633|2308|38336|8|209|2264|36|40.55|62.44|42.45|0.00|1528.20|1459.80|2247.84|76.41|0.00|1528.20|1604.61|68.40| +2452600|62117|987|7405|812633||38336||245|2264||17.64||||474.25||||||497.96|| +2452600|62117|5275|7405|812633|2308|38336|8|187|2264|2|12.67|24.07|16.60|0.00|33.20|25.34|48.14|0.33|0.00|33.20|33.53|7.86| +2452600|62117|8178|7405|812633|2308|38336|8|162|2264|10|69.34|90.14|39.66|0.00|396.60|693.40|901.40|7.93|0.00|396.60|404.53|-296.80| +2451088|55989|15610|77373|1791654|5135|24216|1|47|2265|13|29.97|57.24|25.18|0.00|327.34|389.61|744.12|16.36|0.00|327.34|343.70|-62.27| +2451088|55989|3632|77373|1791654|5135|24216|1|220|2265|8|58.55|62.06|40.33|0.00|322.64|468.40|496.48|25.81|0.00|322.64|348.45|-145.76| +2451088|55989|13100|77373|1791654|5135|24216|1|77|2265|28|28.20|39.19|30.96|0.00|866.88|789.60|1097.32|78.01|0.00|866.88|944.89|77.28| +2451088|55989|1226||1791654|||||2265||93.98|177.62|79.92|0.00|3276.72|||131.06|0.00|3276.72|3407.78|| +2451088||9760||||24216|1||2265||42.10|77.04||2837.01|||6702.48|0.00|2837.01|2725.77||-936.93| +2451088|55989|14804|77373|1791654|5135|24216|1|258|2265|58|55.03|108.95|29.41|0.00|1705.78|3191.74|6319.10|0.00|0.00|1705.78|1705.78|-1485.96| +2451088|55989|10492|77373|1791654|5135|24216|1|223|2265|96|40.75|47.27|37.34|1541.39|3584.64|3912.00|4537.92|183.89|1541.39|2043.25|2227.14|-1868.75| +2451088|55989|6896|77373|1791654|5135|24216|1|213|2265|96|15.33|21.00|15.12|0.00|1451.52|1471.68|2016.00|101.60|0.00|1451.52|1553.12|-20.16| +||2584|77373||5135|24216|||2265||98.53|139.91||0.00||3744.14|5316.58|239.24|0.00|||| +2451088|55989|1744|77373|1791654|5135|24216|1|163|2265|10|67.28|133.21|45.29|0.00|452.90|672.80|1332.10|40.76|0.00|452.90|493.66|-219.90| +2451877|39685|215|10248|134622|3443|8664|4|127|2266|55|85.66|146.47|42.47|2032.18|2335.85|4711.30|8055.85|9.11|2032.18|303.67|312.78|-4407.63| +2451877|39685|5977|10248|134622|3443|8664|4|142|2266|88|6.53|12.60|0.50|15.84|44.00|574.64|1108.80|1.68|15.84|28.16|29.84|-546.48| +2451877|39685|15967|10248|134622|3443|8664|4|234|2266|76|88.70|129.50|10.36|165.34|787.36|6741.20|9842.00|12.44|165.34|622.02|634.46|-6119.18| +2451877|39685|17281|10248|134622|3443|8664|4|263|2266|13|14.67|19.80|14.05|0.00|182.65|190.71|257.40|3.65|0.00|182.65|186.30|-8.06| +2451877|39685|4655|10248|134622|3443|8664|4|42|2266|96|77.24|115.86|68.35|0.00|6561.60|7415.04|11122.56|65.61|0.00|6561.60|6627.21|-853.44| +2451877|39685|12929|10248|134622|3443|8664|4|32|2266|52|57.82|109.27|104.89|0.00|5454.28|3006.64|5682.04|436.34|0.00|5454.28|5890.62|2447.64| +2451877|39685|4651|10248|134622|3443|8664|4|208|2266|73|44.46|68.91|25.49|0.00|1860.77|3245.58|5030.43|37.21|0.00|1860.77|1897.98|-1384.81| +2451877|39685|16517|10248|134622|3443|8664|4|20|2266|18|85.87|102.18|68.46|0.00|1232.28|1545.66|1839.24|61.61|0.00|1232.28|1293.89|-313.38| +2451877|39685|16700|10248|134622|3443|8664|4|41|2266|99|5.25|7.61|4.41|0.00|436.59|519.75|753.39|0.00|0.00|436.59|436.59|-83.16| +2451877|39685|15265|10248|134622|3443|8664|4|217|2266|29|40.44|42.86|12.85|104.34|372.65|1172.76|1242.94|24.14|104.34|268.31|292.45|-904.45| +2451877|39685|7561|10248|134622|3443|8664|4|277|2266|24|89.88|164.48|50.98|0.00|1223.52|2157.12|3947.52|85.64|0.00|1223.52|1309.16|-933.60| +2451877|39685|3578|10248|134622|3443|8664|4|107|2266|39|85.53|120.59|55.47|0.00|2163.33|3335.67|4703.01|108.16|0.00|2163.33|2271.49|-1172.34| +2451877|39685|17443|10248|134622|3443|8664|4|89|2266|94|24.64|40.90|2.45|172.72|230.30|2316.16|3844.60|0.57|172.72|57.58|58.15|-2258.58| +2451877|39685|5155|10248|134622|3443|8664|4|4|2266|62|93.55|119.74|39.51|0.00|2449.62|5800.10|7423.88|122.48|0.00|2449.62|2572.10|-3350.48| +2451799|43619|14359|10176|1201329|4482|24035|7|32|2267|96|75.53|137.46|9.62|0.00|923.52|7250.88|13196.16|36.94|0.00|923.52|960.46|-6327.36| +2451799|43619|1465||1201329|4482||7|148|2267|||69.30|||||4643.10|32.03||3203.27|3235.30|406.02| +2451799|43619|13097|10176|1201329|4482|24035|7|98|2267|79|18.17|21.80|6.54|201.49|516.66|1435.43|1722.20|22.06|201.49|315.17|337.23|-1120.26| +2451799|43619|11243|10176|1201329|4482|24035|7|135|2267|85|11.01|20.58|18.31|1245.08|1556.35|935.85|1749.30|3.11|1245.08|311.27|314.38|-624.58| +2451799|43619|16874|10176|1201329|4482|24035|7|244|2267|27|73.41|96.16|82.69|0.00|2232.63|1982.07|2596.32|178.61|0.00|2232.63|2411.24|250.56| +2451799|43619|4082|10176|1201329|4482|24035|7|71|2267|79|38.80|64.79|64.79|0.00|5118.41|3065.20|5118.41|153.55|0.00|5118.41|5271.96|2053.21| +2451799|43619|4706|10176|1201329|4482|24035|7|237|2267|11|82.02|87.76|6.14|0.00|67.54|902.22|965.36|5.40|0.00|67.54|72.94|-834.68| +2451799|43619|6998|10176|1201329|4482|24035|7|210|2267|78|37.06|72.63|11.62|0.00|906.36|2890.68|5665.14|27.19|0.00|906.36|933.55|-1984.32| +2451799|43619|17851|10176|1201329|4482|24035|7|227|2267|15|57.46|79.29|19.02|0.00|285.30|861.90|1189.35|0.00|0.00|285.30|285.30|-576.60| +2451066|52047|3601|36877|103354|1478|31955|10|10|2268|70|34.37|43.99|7.03|0.00|492.10|2405.90|3079.30|39.36|0.00|492.10|531.46|-1913.80| +2451066|52047|2275|36877|103354|1478|31955|10|190|2268|57|93.49|141.16|103.04|0.00|5873.28|5328.93|8046.12|58.73|0.00|5873.28|5932.01|544.35| +2451066|52047|17834|36877|103354|1478|31955|10|167|2268|25|82.84|119.28|76.33|362.56|1908.25|2071.00|2982.00|61.82|362.56|1545.69|1607.51|-525.31| +2451066|52047|610|36877|103354|1478|31955|10|203|2268|55|16.19|28.33|7.36|0.00|404.80|890.45|1558.15|4.04|0.00|404.80|408.84|-485.65| +2451066|52047|11792|36877|103354|1478|31955|10|237|2268|65|74.94|108.66|32.59|0.00|2118.35|4871.10|7062.90|21.18|0.00|2118.35|2139.53|-2752.75| +2451066|52047|15292|36877|103354|1478|31955|10|47|2268|88|66.57|129.81|75.28|0.00|6624.64|5858.16|11423.28|66.24|0.00|6624.64|6690.88|766.48| +2451066|52047|6271|36877|103354|1478|31955|10|172|2268|56|59.36|96.75|55.14|0.00|3087.84|3324.16|5418.00|123.51|0.00|3087.84|3211.35|-236.32| +2451066|52047|5464|36877|103354|1478|31955|10|168|2268|45|52.73|71.18|59.07|1010.09|2658.15|2372.85|3203.10|32.96|1010.09|1648.06|1681.02|-724.79| +2452584|64724|5787|12011|377947|3436|33725|2|226|2269|46|69.65|79.40|26.20|807.48|1205.20|3203.90|3652.40|15.90|807.48|397.72|413.62|-2806.18| +2452584|64724|816|12011|377947|3436|33725|2|82|2269|47|60.76|69.87|60.08|0.00|2823.76|2855.72|3283.89|254.13|0.00|2823.76|3077.89|-31.96| +2452584|64724|630|12011|377947|3436|33725|2|115|2269|88|72.79|129.56|119.19|6817.66|10488.72|6405.52|11401.28|293.68|6817.66|3671.06|3964.74|-2734.46| +2452584|64724|909|12011|377947|3436|33725|2|278|2269|44|57.66|58.81|37.63|0.00|1655.72|2537.04|2587.64|99.34|0.00|1655.72|1755.06|-881.32| +2452584|64724|6973|12011|377947|3436|33725|2|274|2269|52|99.97|117.96|51.90|0.00|2698.80|5198.44|6133.92|80.96|0.00|2698.80|2779.76|-2499.64| +2452584|64724|2529|12011|377947|3436|33725|2|160|2269|11|46.63|85.33|46.93|0.00|516.23|512.93|938.63|36.13|0.00|516.23|552.36|3.30| +2452584|64724|1350|12011|377947|3436|33725|2|43|2269|58|94.66|149.56|20.93|60.69|1213.94|5490.28|8674.48|103.79|60.69|1153.25|1257.04|-4337.03| +2452584|64724|13338|12011|377947|3436|33725|2|146|2269|99|14.36|24.84|14.65|0.00|1450.35|1421.64|2459.16|43.51|0.00|1450.35|1493.86|28.71| +2452584|64724|14217|12011|377947|3436|33725|2|298|2269|34|66.61|87.92|3.51|0.00|119.34|2264.74|2989.28|5.96|0.00|119.34|125.30|-2145.40| +2452584|64724|2835|12011|377947|3436|33725|2|299|2269|16|91.89|177.34|31.92|0.00|510.72|1470.24|2837.44|0.00|0.00|510.72|510.72|-959.52| +2452584|64724|4117|12011|377947|3436|33725|2|271|2269|46|69.44|88.88|72.88|0.00|3352.48|3194.24|4088.48|234.67|0.00|3352.48|3587.15|158.24| +2451690|49896|17299|30118|1830397|||||2270||48.09||59.37||653.07|528.99|666.49|||653.07||124.08| +2451690|49896|4423|30118|1830397|5429|26908|7|241|2270|50|13.47|20.60|14.83|593.20|741.50|673.50|1030.00|2.96|593.20|148.30|151.26|-525.20| +2451690|49896|14102|30118|1830397|5429|26908|7|220|2270|79|12.50|22.37|17.00|0.00|1343.00|987.50|1767.23|26.86|0.00|1343.00|1369.86|355.50| +2451690|49896|7627|30118|1830397|5429|26908|7|43|2270|82|75.09|117.89|53.05|0.00|4350.10|6157.38|9666.98|130.50|0.00|4350.10|4480.60|-1807.28| +2451690|49896|44|30118|1830397|5429|26908|7|47|2270|62|77.69|150.71|27.12|0.00|1681.44|4816.78|9344.02|33.62|0.00|1681.44|1715.06|-3135.34| +2451690|49896|14819|30118|1830397|5429|26908|7|47|2270|29|16.83|30.29|9.99|0.00|289.71|488.07|878.41|2.89|0.00|289.71|292.60|-198.36| +2451690|49896|2126|30118|1830397|5429|26908|7|289|2270|33|69.68|69.68|45.98|1229.04|1517.34|2299.44|2299.44|23.06|1229.04|288.30|311.36|-2011.14| +2451690|49896|6146|30118|1830397|5429|26908|7|112|2270|57|84.56|155.59|21.78|124.14|1241.46|4819.92|8868.63|33.51|124.14|1117.32|1150.83|-3702.60| +2451690|49896|9140|30118|1830397|5429|26908|7|267|2270|38|51.79|69.91|23.76|0.00|902.88|1968.02|2656.58|18.05|0.00|902.88|920.93|-1065.14| +||2797|||5429|26908||253|2270||19.15|20.49||0.00|||1004.01|8.73|0.00|873.18|881.91|-65.17| +2451690|49896|13703|30118|1830397|5429|26908|7|82|2270|90|4.68|7.16|5.37|28.99|483.30|421.20|644.40|13.62|28.99|454.31|467.93|33.11| +2451690|49896|10547|30118|1830397|5429|26908|7|296|2270|91|16.36|32.72|26.50|0.00|2411.50|1488.76|2977.52|96.46|0.00|2411.50|2507.96|922.74| +2451690|49896|3193|30118|1830397|5429|26908|7|98|2270|12|56.19|62.37|40.54|0.00|486.48|674.28|748.44|38.91|0.00|486.48|525.39|-187.80| +2451552|48691|2150|30994|1920001|208|32015|7|127|2271|39|37.53|58.54|57.36|0.00|2237.04|1463.67|2283.06|22.37|0.00|2237.04|2259.41|773.37| +2451552|48691|14635|30994|1920001|208|32015|7|153|2271|46|93.89|169.94|20.39|0.00|937.94|4318.94|7817.24|37.51|0.00|937.94|975.45|-3381.00| +2451552||14489|30994|1920001||||14|2271||67.95|||0.00||||23.88|0.00|341.25|365.13|| +2451552|48691|3455|30994|1920001|208|32015|7|3|2271|49|37.76|67.21|62.50|0.00|3062.50|1850.24|3293.29|153.12|0.00|3062.50|3215.62|1212.26| +2451552|48691|10159|30994|1920001|208|32015|7|89|2271|96|78.74|85.82|78.95|0.00|7579.20|7559.04|8238.72|0.00|0.00|7579.20|7579.20|20.16| +2451552|48691|17753|30994|1920001|208|32015|7|121|2271|99|28.74|31.32|5.32|94.80|526.68|2845.26|3100.68|4.31|94.80|431.88|436.19|-2413.38| +2451552|48691|1333|30994|1920001|208|32015|7|9|2271|51|62.37|82.95|30.69|0.00|1565.19|3180.87|4230.45|109.56|0.00|1565.19|1674.75|-1615.68| +2451552|48691|17000|30994|1920001|208|32015|7|204|2271|57|47.49|71.23|17.80|0.00|1014.60|2706.93|4060.11|20.29|0.00|1014.60|1034.89|-1692.33| +2451552|48691|5954|30994|1920001|208|32015|7|166|2271|69|90.79|95.32|20.01|0.00|1380.69|6264.51|6577.08|0.00|0.00|1380.69|1380.69|-4883.82| +2451552|48691|2609|30994|1920001|208|32015|7|124|2271|6|72.38|114.36|21.72|0.00|130.32|434.28|686.16|0.00|0.00|130.32|130.32|-303.96| +2451552|48691|10328|30994|1920001|208|32015|7|297|2271|77|70.60|92.48|78.60|0.00|6052.20|5436.20|7120.96|423.65|0.00|6052.20|6475.85|616.00| +2451552|48691|7376|30994|1920001|208|32015|7|181|2271|38|12.10|23.23|3.01|0.00|114.38|459.80|882.74|4.57|0.00|114.38|118.95|-345.42| +2451552|48691|13279|30994|1920001|208|32015|7|46|2271|6|93.69|111.49|50.17|0.00|301.02|562.14|668.94|3.01|0.00|301.02|304.03|-261.12| +2451552|48691|3944|30994|1920001|208|32015|7|35|2271|4|62.65|72.04|20.89|14.20|83.56|250.60|288.16|3.46|14.20|69.36|72.82|-181.24| +2451552|48691|14180|30994|1920001|208|32015|7|217|2271|48|29.49|52.49|46.19|0.00|2217.12|1415.52|2519.52|110.85|0.00|2217.12|2327.97|801.60| +2451552|48691|1415|30994|1920001|208|32015|7|71|2271|41|79.98|88.77|71.01|2503.81|2911.41|3279.18|3639.57|28.53|2503.81|407.60|436.13|-2871.58| +2451122|38217|10192|33805|1363409|2166|6970|10|52|2272|15|87.17|121.16|101.77|0.00|1526.55|1307.55|1817.40|91.59|0.00|1526.55|1618.14|219.00| +2451122|38217|10399|33805|1363409|2166|6970|10|252|2272|18|94.89|168.90|121.60|0.00|2188.80|1708.02|3040.20|109.44|0.00|2188.80|2298.24|480.78| +2451122|38217|10135|33805|1363409|2166|6970|10|162|2272|19|27.54|39.38|34.65|289.67|658.35|523.26|748.22|14.74|289.67|368.68|383.42|-154.58| +2451122|38217|3745|33805|1363409|2166|6970|10|104|2272|94|8.42|12.29|11.67|0.00|1096.98|791.48|1155.26|43.87|0.00|1096.98|1140.85|305.50| +2451122|38217|7471|33805|1363409|2166|6970|10|238|2272|73|97.99|159.72|132.56|2419.22|9676.88|7153.27|11659.56|72.57|2419.22|7257.66|7330.23|104.39| +2451122||14128|||2166|6970||272|2272|||||674.90|1730.52||2622.23|0.00|674.90|1055.62||-716.30| +2451122|38217|15043|33805|1363409|2166|6970|10|242|2272|26|14.33|28.08|8.14|124.86|211.64|372.58|730.08|6.94|124.86|86.78|93.72|-285.80| +2451122|38217|16891|33805|1363409|2166|6970|10|151|2272|36|75.37|131.89|129.25|0.00|4653.00|2713.32|4748.04|418.77|0.00|4653.00|5071.77|1939.68| +2451122||3674|||2166||10||2272||3.66||||0.00|||||0.00|0.00|| +2451460||782||1865218|||||2273|||||||4740.32|6826.04|||136.24|147.13|| +2451460|30985|7058|5038|1865218|1398|17931|8|186|2273|48|50.49|90.88|25.44|0.00|1221.12|2423.52|4362.24|85.47|0.00|1221.12|1306.59|-1202.40| +||13378||1865218|1398|17931|8||2273||||46.23||3051.18|||152.55|||3203.73|| +2451460|30985|10964|5038|1865218|1398|17931|8|26|2273|44|14.92|23.87|23.87|0.00|1050.28|656.48|1050.28|63.01|0.00|1050.28|1113.29|393.80| +2451460|30985|6493|5038|1865218|1398|17931|8|80|2273|10|84.89|97.62|22.45|0.00|224.50|848.90|976.20|17.96|0.00|224.50|242.46|-624.40| +2451460|30985|13054|5038|1865218|1398|17931|8|58|2273|9|26.14|36.85|35.74|106.14|321.66|235.26|331.65|10.77|106.14|215.52|226.29|-19.74| +2451460|30985|11857|5038|1865218|1398|17931|8|145|2273|30|3.73|7.27|1.81|0.00|54.30|111.90|218.10|0.00|0.00|54.30|54.30|-57.60| +2451460|30985|8716|5038|1865218|1398|17931|8|235|2273|26|10.15|11.06|6.85|0.00|178.10|263.90|287.56|1.78|0.00|178.10|179.88|-85.80| +2451112|44706|2347|78905|1471859|2860|16214|4|66|2274|40|39.65|50.35|16.11|399.52|644.40|1586.00|2014.00|2.44|399.52|244.88|247.32|-1341.12| +2451112|44706|1528|78905|1471859|2860|16214|4|25|2274|79|36.43|66.66|64.66|0.00|5108.14|2877.97|5266.14|51.08|0.00|5108.14|5159.22|2230.17| +2451112|44706|11072|78905|1471859|2860|16214|4|265|2274|16|45.92|84.49|62.52|0.00|1000.32|734.72|1351.84|50.01|0.00|1000.32|1050.33|265.60| +2451112|44706|1609|78905|1471859|2860|16214|4|271|2274|57|18.13|21.39|19.67|459.68|1121.19|1033.41|1219.23|0.00|459.68|661.51|661.51|-371.90| +2451112|44706|9878|78905|1471859|2860|16214|4|66|2274|46|80.89|104.34|18.78|0.00|863.88|3720.94|4799.64|34.55|0.00|863.88|898.43|-2857.06| +2451112|44706|52|78905|1471859|2860|16214|4|18|2274|9|33.80|44.61|29.88|0.00|268.92|304.20|401.49|18.82|0.00|268.92|287.74|-35.28| +2451112|44706|14725|78905|1471859|2860|16214|4|272|2274|37|78.69|142.42|88.30|0.00|3267.10|2911.53|5269.54|294.03|0.00|3267.10|3561.13|355.57| +2451112|44706|4484|78905|1471859|2860|16214|4|235|2274|59|31.17|42.70|5.12|0.00|302.08|1839.03|2519.30|18.12|0.00|302.08|320.20|-1536.95| +2451877|64248|8744|54627|1003967|6107|49847|1|189|2275|23|68.62|83.03|59.78|0.00|1374.94|1578.26|1909.69|27.49|0.00|1374.94|1402.43|-203.32| +2451877|64248|14545|54627|1003967|6107|49847|1|295|2275|89|26.44|39.13|37.56|0.00|3342.84|2353.16|3482.57|200.57|0.00|3342.84|3543.41|989.68| +2451877|64248|16034|54627|1003967|6107|49847|1|232|2275|43|96.57|150.64|75.32|0.00|3238.76|4152.51|6477.52|0.00|0.00|3238.76|3238.76|-913.75| +2451877|64248|4837|54627|1003967|6107|49847|1|173|2275|50|92.29|143.97|93.58|0.00|4679.00|4614.50|7198.50|421.11|0.00|4679.00|5100.11|64.50| +2451877|64248|9752|54627|1003967|6107|49847|1|274|2275|64|13.01|14.57|12.23|0.00|782.72|832.64|932.48|39.13|0.00|782.72|821.85|-49.92| +2451877|64248|4220|54627|1003967|6107|49847|1|57|2275|33|68.96|117.92|31.83|0.00|1050.39|2275.68|3891.36|94.53|0.00|1050.39|1144.92|-1225.29| +2451877|64248|17261|54627|1003967|6107|49847|1|147|2275|28|56.90|71.69|42.29|0.00|1184.12|1593.20|2007.32|0.00|0.00|1184.12|1184.12|-409.08| +|64248|6427|54627||6107||||2275|73||42.79|39.79|0.00|2904.67|1754.92|3123.67||0.00||3166.09|| +2451877|64248|4453|54627|1003967|6107|49847|1|216|2275|64|84.77|166.14|107.99|3179.22|6911.36|5425.28|10632.96|0.00|3179.22|3732.14|3732.14|-1693.14| +2451877|64248|17353|54627|1003967|6107|49847|1|176|2275|17|50.59|67.79|24.40|0.00|414.80|860.03|1152.43|37.33|0.00|414.80|452.13|-445.23| +2451877|64248|16454|54627|1003967|6107|49847|1|295|2275|41|73.90|97.54|25.36|0.00|1039.76|3029.90|3999.14|31.19|0.00|1039.76|1070.95|-1990.14| +2451877|64248|15121|54627|1003967|6107|49847|1|136|2275|29|12.53|21.42|1.92|37.30|55.68|363.37|621.18|0.55|37.30|18.38|18.93|-344.99| +2451877|64248|12979|54627|1003967|6107|49847|1|190|2275|24|76.71|129.63|7.77|0.00|186.48|1841.04|3111.12|1.86|0.00|186.48|188.34|-1654.56| +2452549|69612|10128|29460|974122|2784|38646|8|215|2276|40|84.87|133.24|55.96|0.00|2238.40|3394.80|5329.60|156.68|0.00|2238.40|2395.08|-1156.40| +2452549|69612|9387|29460|974122|2784|38646|8|224|2276|78|34.30|49.04|10.78|0.00|840.84|2675.40|3825.12|75.67|0.00|840.84|916.51|-1834.56| +2452549|69612|5131|29460|974122|2784|38646|8|239|2276|67|36.36|47.63|24.76|0.00|1658.92|2436.12|3191.21|33.17|0.00|1658.92|1692.09|-777.20| +|69612|9915|29460|974122|2784||8||2276||92.20|105.10|||||6200.90|||||-789.42| +2452549|69612|14937|29460|974122|2784|38646|8|13|2276|84|10.50|17.64|12.70|170.68|1066.80|882.00|1481.76|44.80|170.68|896.12|940.92|14.12| +2452549|69612|9936|29460|974122|2784|38646|8|294|2276|90|5.79|10.24|5.83|0.00|524.70|521.10|921.60|20.98|0.00|524.70|545.68|3.60| +2452549|69612|9835|29460|974122|2784|38646|8|95|2276|34|73.33|114.39|57.19|0.00|1944.46|2493.22|3889.26|19.44|0.00|1944.46|1963.90|-548.76| +2452549|69612|1560|29460|974122|2784|38646|8|285|2276|8|14.46|28.48|7.40|0.00|59.20|115.68|227.84|4.73|0.00|59.20|63.93|-56.48| +2452549|69612|12495|29460|974122|2784|38646|8|223|2276|80|31.68|44.35|4.87|0.00|389.60|2534.40|3548.00|19.48|0.00|389.60|409.08|-2144.80| +2452549|69612|10620|29460|974122|2784|38646|8|202|2276|67|48.05|95.61|25.81|0.00|1729.27|3219.35|6405.87|155.63|0.00|1729.27|1884.90|-1490.08| +2452549|69612|12061|29460|974122|2784|38646|8|60|2276|82|10.72|17.04|5.28|0.00|432.96|879.04|1397.28|34.63|0.00|432.96|467.59|-446.08| +2452549|69612|13566|29460|974122|2784|38646|8|233|2276|50|30.56|40.95|19.65|0.00|982.50|1528.00|2047.50|49.12|0.00|982.50|1031.62|-545.50| +2452549|69612|14943|29460|974122|2784|38646|8|235|2276|73|98.60|118.32|35.49|0.00|2590.77|7197.80|8637.36|155.44|0.00|2590.77|2746.21|-4607.03| +2452549|69612|15366|29460|974122|2784|38646|8|227|2276|73|48.05|56.69|19.27|0.00|1406.71|3507.65|4138.37|126.60|0.00|1406.71|1533.31|-2100.94| +2452549|69612|13081|29460|974122|2784|38646|8|128|2276|22|34.86|48.80|26.84|0.00|590.48|766.92|1073.60|23.61|0.00|590.48|614.09|-176.44| +2452549|69612|8316|29460|||38646||149|2276|35|47.66|||0.00||||106.64|0.00||1630.19|-144.55| +2451807|51063|8921|59722|867430|4385|37340|10|212|2277|57|60.29|82.59|72.67|0.00|4142.19|3436.53|4707.63|248.53|0.00|4142.19|4390.72|705.66| +2451807|51063|3983|59722|867430|4385|37340|10|39|2277|23|66.98|128.60|111.88|0.00|2573.24|1540.54|2957.80|128.66|0.00|2573.24|2701.90|1032.70| +2451807|51063|12817|59722|867430|4385|37340|10|226|2277|94|75.82|141.78|17.01|0.00|1598.94|7127.08|13327.32|111.92|0.00|1598.94|1710.86|-5528.14| +||15041||867430||37340|||2277|14|||15.55|143.68||374.92||6.66|143.68|||-300.90| +2451807|51063|17141|59722|867430|4385|37340|10|169|2277|47|78.03|143.57|34.45|1198.17|1619.15|3667.41|6747.79|16.83|1198.17|420.98|437.81|-3246.43| +2451807|51063|1915|59722|867430|4385|37340|10|10|2277|11|85.58|93.28|17.72|0.00|194.92|941.38|1026.08|11.69|0.00|194.92|206.61|-746.46| +2451807|51063|16705|59722|867430|4385|37340|10|103|2277|74|64.52|94.19|6.59|0.00|487.66|4774.48|6970.06|4.87|0.00|487.66|492.53|-4286.82| +2451807|51063|5948|59722|867430|4385|37340|10|193|2277|28|5.44|6.41|1.53|0.00|42.84|152.32|179.48|0.42|0.00|42.84|43.26|-109.48| +2451807|51063|6781|59722|867430|4385|37340|10|197|2277|77|18.09|18.09|5.42|0.00|417.34|1392.93|1392.93|12.52|0.00|417.34|429.86|-975.59| +2451807|51063|3758|59722|867430|4385|37340|10|205|2277|88|8.47|11.34|0.34|0.00|29.92|745.36|997.92|2.39|0.00|29.92|32.31|-715.44| +2451807|51063|10610|59722|867430|4385|37340|10|84|2277|50|73.45|80.79|11.31|0.00|565.50|3672.50|4039.50|33.93|0.00|565.50|599.43|-3107.00| +2451807|51063|11927|59722|867430|4385|37340|10|205|2277|30|37.65|60.99|26.83|0.00|804.90|1129.50|1829.70|32.19|0.00|804.90|837.09|-324.60| +2451807|51063|4361|59722|867430|4385|37340|10|100|2277|89|93.59|95.46|41.04|3214.25|3652.56|8329.51|8495.94|21.91|3214.25|438.31|460.22|-7891.20| +2451807||1597|59722||||10|289|2277|||129.04||||3964.74||347.45|||4690.61|378.42| +2452244|72737|1203|87273|436407|294|1828|8|187|2278|70|32.49|35.73|25.72|0.00|1800.40|2274.30|2501.10|18.00|0.00|1800.40|1818.40|-473.90| +2452244|72737|775|87273|436407|294|1828|8|178|2278|22|4.30|6.83|0.88|0.00|19.36|94.60|150.26|1.54|0.00|19.36|20.90|-75.24| +2452244|72737|16329|87273|436407|294|1828|8|23|2278|57|89.53|134.29|33.57|0.00|1913.49|5103.21|7654.53|0.00|0.00|1913.49|1913.49|-3189.72| +2452244|72737|8409|87273|436407|294|1828|8|98|2278|60|12.27|18.77|3.94|0.00|236.40|736.20|1126.20|21.27|0.00|236.40|257.67|-499.80| +2452244|72737|7407|87273|436407|294|1828|8|106|2278|18|92.33|112.64|67.58|462.24|1216.44|1661.94|2027.52|22.62|462.24|754.20|776.82|-907.74| +||6109|||||||2278||85.23||||6768.66||14714.90|||609.18|645.73|-7658.13| +2452244|72737|9517|87273|436407|294|1828|8|50|2278|58|90.68|108.81|39.17|0.00|2271.86|5259.44|6310.98|181.74|0.00|2271.86|2453.60|-2987.58| +2452244|72737|3747|87273|436407|294|1828|8|20|2278|59|50.86|76.79|33.78|1255.60|1993.02|3000.74|4530.61|44.24|1255.60|737.42|781.66|-2263.32| +2451954|52490|13347|41968|834591|4461|36284|10|6|2279|67|97.09|183.50|143.13|0.00|9589.71|6505.03|12294.50|0.00|0.00|9589.71|9589.71|3084.68| +2451954|52490|4369|41968|834591|4461|36284|10|173|2279|100|40.33|45.97|39.53|0.00|3953.00|4033.00|4597.00|79.06|0.00|3953.00|4032.06|-80.00| +2451954|52490|6609|41968|834591|4461|36284|10|57|2279|78|13.67|27.34|8.47|33.03|660.66|1066.26|2132.52|43.93|33.03|627.63|671.56|-438.63| +2451954|52490|10937|41968|834591|4461|36284|10|150|2279|37|18.55|27.63|1.93|24.27|71.41|686.35|1022.31|1.41|24.27|47.14|48.55|-639.21| +2451954|52490|8215|41968|834591|4461|36284|10|233|2279|32|15.90|21.94|8.77|89.80|280.64|508.80|702.08|3.81|89.80|190.84|194.65|-317.96| +2451954|52490|8699|41968|834591|4461|36284|10|142|2279|69|88.18|141.96|90.85|0.00|6268.65|6084.42|9795.24|438.80|0.00|6268.65|6707.45|184.23| +2451954|52490|4427|41968|834591|4461|36284|10|32|2279|58|1.13|1.76|0.79|0.00|45.82|65.54|102.08|3.66|0.00|45.82|49.48|-19.72| +2451954|52490|5383|41968|834591|4461|36284|10|278|2279|86|37.52|54.02|50.77|0.00|4366.22|3226.72|4645.72|43.66|0.00|4366.22|4409.88|1139.50| +2451954|52490|7829|41968||4461||||2279|94||||||3180.96||282.74|||4321.92|858.22| +2451954|52490|12349|41968|834591|4461|36284|10|41|2279|37|40.92|81.02|17.82|0.00|659.34|1514.04|2997.74|39.56|0.00|659.34|698.90|-854.70| +2451954|52490|3109|41968|834591|4461|36284|10|150|2279|8|19.24|24.62|10.58|0.00|84.64|153.92|196.96|4.23|0.00|84.64|88.87|-69.28| +2451954||2435|||4461|36284|||2279||||||1355.48|1237.04||94.88||||| +2451954|52490|513|41968|834591|4461|36284|10|195|2279|13|73.93|112.37|73.04|0.00|949.52|961.09|1460.81|85.45|0.00|949.52|1034.97|-11.57| +2451954|52490|5747|41968|834591|4461|36284|10|51|2279|44|17.44|18.66|11.19|0.00|492.36|767.36|821.04|14.77|0.00|492.36|507.13|-275.00| +2451954|52490|13029|41968|834591|4461|36284|10|257|2279|4|33.65|37.68|25.24|0.00|100.96|134.60|150.72|5.04|0.00|100.96|106.00|-33.64| +2451954|52490|15601|41968|834591|4461|36284|10|175|2279|25|30.14|30.74|15.67|266.39|391.75|753.50|768.50|5.01|266.39|125.36|130.37|-628.14| +2450974|64179|4358|20288|1548220|6999|37778|8|129|2280|100|5.24|10.11|0.80|0.00|80.00|524.00|1011.00|5.60|0.00|80.00|85.60|-444.00| +2450974|64179|13921|20288|1548220|6999|37778|8|173|2280|88|42.55|43.82|5.69|215.30|500.72|3744.40|3856.16|8.56|215.30|285.42|293.98|-3458.98| +2450974|64179|17530|20288|1548220|6999|37778|8|227|2280|33|59.21|77.56|4.65|127.36|153.45|1953.93|2559.48|1.82|127.36|26.09|27.91|-1927.84| +2450974|64179|3553|20288|1548220|6999|37778|8|221|2280|43|13.07|13.59|11.41|0.00|490.63|562.01|584.37|19.62|0.00|490.63|510.25|-71.38| +2450974|64179|11881|20288|1548220|6999|37778|8|125|2280|14|52.52|66.17|18.52|233.35|259.28|735.28|926.38|0.77|233.35|25.93|26.70|-709.35| +2450974|64179|14066|20288|1548220|6999|37778|8|151|2280|25|49.65|71.49|39.31|0.00|982.75|1241.25|1787.25|78.62|0.00|982.75|1061.37|-258.50| +2450974|64179|17191|20288|1548220|6999|37778|8|154|2280|58|6.79|7.46|3.20|0.00|185.60|393.82|432.68|14.84|0.00|185.60|200.44|-208.22| +2450974|64179|7924|20288|1548220|6999|37778|8|188|2280|26|59.03|82.05|17.23|0.00|447.98|1534.78|2133.30|31.35|0.00|447.98|479.33|-1086.80| +2450974|64179|1789|20288|1548220|6999|37778|8|126|2280|40|23.72|34.86|7.66|0.00|306.40|948.80|1394.40|18.38|0.00|306.40|324.78|-642.40| +2450974|64179|11143|20288|1548220|6999|37778|8|69|2280|25|56.54|74.63|29.85|0.00|746.25|1413.50|1865.75|29.85|0.00|746.25|776.10|-667.25| +2450974|64179|14041|20288|1548220|6999|37778|8|157|2280|35|75.35|94.94|67.40|0.00|2359.00|2637.25|3322.90|188.72|0.00|2359.00|2547.72|-278.25| +2450974|64179|13291|20288|1548220|6999|37778|8|195|2280|55|3.28|6.33|1.26|0.00|69.30|180.40|348.15|0.69|0.00|69.30|69.99|-111.10| +2450974|64179|12331|20288|1548220|6999|37778|8|118|2280|56|81.68|142.12|96.64|0.00|5411.84|4574.08|7958.72|270.59|0.00|5411.84|5682.43|837.76| +2450974|64179|650|20288|1548220|6999|37778|8|227|2280|73|81.22|119.39|101.48|0.00|7408.04|5929.06|8715.47|518.56|0.00|7408.04|7926.60|1478.98| +2450974|64179|9224||||37778|8|253|2280||||35.99|0.00||||45.34|0.00||549.20|-514.22| +2450974|64179|15919|20288|1548220|6999|37778|8|259|2280|77|14.44|23.97|12.22|0.00|940.94|1111.88|1845.69|56.45|0.00|940.94|997.39|-170.94| +||11699|||373|||207|2281||11.58|13.89|10.41|473.65|676.65|752.70|||473.65|||-549.70| +2451657|65528|6788|35672|117046|373|2947|7|181|2281|24|82.09|114.10|62.75|0.00|1506.00|1970.16|2738.40|120.48|0.00|1506.00|1626.48|-464.16| +|65528|2383|35672|||2947|||2281||23.95||19.72|795.11||||18.74|795.11||643.47|-1099.67| +2451657|65528|7727|35672|117046|373|2947|7|8|2281|93|57.13|99.40|7.95|0.00|739.35|5313.09|9244.20|22.18|0.00|739.35|761.53|-4573.74| +2451657|65528|10601|35672|117046|373|2947|7|129|2281|43|36.58|64.74|41.43|1211.41|1781.49|1572.94|2783.82|28.50|1211.41|570.08|598.58|-1002.86| +2451657|65528|10388|35672|117046|373|2947|7|73|2281|15|14.99|28.18|16.34|0.00|245.10|224.85|422.70|19.60|0.00|245.10|264.70|20.25| +2451657|65528|2795|35672|117046|373|2947|7|46|2281|46|66.37|87.60|5.25|0.00|241.50|3053.02|4029.60|0.00|0.00|241.50|241.50|-2811.52| +2451657|65528|15671|35672|117046|373|2947|7|245|2281|28|96.21|115.45|66.96|0.00|1874.88|2693.88|3232.60|37.49|0.00|1874.88|1912.37|-819.00| +2451657|65528|1337|35672|117046|373|2947|7|285|2281|28|11.32|20.14|17.32|0.00|484.96|316.96|563.92|9.69|0.00|484.96|494.65|168.00| +2451657|65528|8269|35672|117046|373|2947|7|254|2281|5|45.01|59.41|32.67|0.00|163.35|225.05|297.05|14.70|0.00|163.35|178.05|-61.70| +2451657|65528|9439|35672|117046|373|2947|7|163|2281|49|20.25|37.46|33.33|538.94|1633.17|992.25|1835.54|98.48|538.94|1094.23|1192.71|101.98| +2451657|65528|8231|35672|117046|373|2947|7|286|2281|83|41.39|45.94|10.56|52.58|876.48|3435.37|3813.02|74.15|52.58|823.90|898.05|-2611.47| +2452403|34546|3037|78211|374886|2314|20989|8|272|2282|75|21.79|26.36|3.69|0.00|276.75|1634.25|1977.00|2.76|0.00|276.75|279.51|-1357.50| +2452403|34546|10269|78211|374886|2314|20989|8|174|2282|13|75.59|119.43|53.74|0.00|698.62|982.67|1552.59|13.97|0.00|698.62|712.59|-284.05| +2452403|34546|12381|78211|374886|2314|20989|8|71|2282|9|80.89|127.80|49.84|0.00|448.56|728.01|1150.20|22.42|0.00|448.56|470.98|-279.45| +2452403|34546|10746|78211|374886|2314|20989|8|107|2282|29|2.05|3.89|0.11|0.00|3.19|59.45|112.81|0.03|0.00|3.19|3.22|-56.26| +2452403|34546|3060|78211|374886|2314|20989|8|248|2282|24|39.77|59.25|15.40|0.00|369.60|954.48|1422.00|11.08|0.00|369.60|380.68|-584.88| +2452403|34546|13203|78211|374886|2314|20989|8|120|2282|85|4.32|4.53|4.30|0.00|365.50|367.20|385.05|0.00|0.00|365.50|365.50|-1.70| +2452403|34546|87|78211|374886|2314|20989|8|262|2282|84|35.96|36.31|9.07|0.00|761.88|3020.64|3050.04|30.47|0.00|761.88|792.35|-2258.76| +2452403|34546|9217|78211|374886|2314|20989|8|67|2282|30|77.12|148.07|111.05|0.00|3331.50|2313.60|4442.10|166.57|0.00|3331.50|3498.07|1017.90| +2452403|34546|16963|78211|374886|2314|20989|8|167|2282|60|40.90|47.85|6.22|0.00|373.20|2454.00|2871.00|14.92|0.00|373.20|388.12|-2080.80| +2452403|34546|14109|78211|374886|2314|20989|8|201|2282|26|21.64|22.50|18.22|341.07|473.72|562.64|585.00|7.95|341.07|132.65|140.60|-429.99| +2450859|54722|12178|74070|43222|3927|19032|2|69|2283|83|39.95|71.51|36.47|877.83|3027.01|3315.85|5935.33|193.42|877.83|2149.18|2342.60|-1166.67| +2450859|54722|886|74070|43222|3927|19032|2|188|2283|18|50.86|51.87|42.53|0.00|765.54|915.48|933.66|22.96|0.00|765.54|788.50|-149.94| +2450859|54722|10453|74070|43222|3927|19032|2|169|2283|15|60.56|69.03|62.12|0.00|931.80|908.40|1035.45|27.95|0.00|931.80|959.75|23.40| +2450859|54722|8746|74070|43222|3927|19032|2|58|2283|83|96.14|97.10|87.39|0.00|7253.37|7979.62|8059.30|507.73|0.00|7253.37|7761.10|-726.25| +2450859|54722|11030|74070|43222|3927|19032|2|268|2283|23|54.75|87.60|2.62|0.00|60.26|1259.25|2014.80|4.82|0.00|60.26|65.08|-1198.99| +2450859|54722|17920|74070|43222|3927|19032|2|69|2283|60|36.56|54.84|47.71|0.00|2862.60|2193.60|3290.40|229.00|0.00|2862.60|3091.60|669.00| +2450859|54722|11456|74070|43222|3927|19032|2|75|2283|3|29.51|57.24|53.80|0.00|161.40|88.53|171.72|14.52|0.00|161.40|175.92|72.87| +2450859|54722|10597|74070|43222|3927|19032|2|208|2283|7|93.88|103.26|13.42|0.00|93.94|657.16|722.82|4.69|0.00|93.94|98.63|-563.22| +2450859|54722|10970|74070|43222|3927|19032|2|166|2283|85|67.65|94.71|54.93|0.00|4669.05|5750.25|8050.35|373.52|0.00|4669.05|5042.57|-1081.20| +2450859|54722|15986|74070|43222|3927|19032|2|189|2283|71|30.65|35.24|9.86|147.01|700.06|2176.15|2502.04|5.53|147.01|553.05|558.58|-1623.10| +2450859|54722|7267|74070|43222|3927|19032|2|140|2283|43|41.68|74.60|29.09|0.00|1250.87|1792.24|3207.80|62.54|0.00|1250.87|1313.41|-541.37| +||16106||43222|3927|||274|2283|36|28.32|30.01|13.20|0.00||1019.52|1080.36|23.76|0.00|475.20|498.96|-544.32| +2450859|54722|10064|74070|43222|3927|19032|2|35|2283|14|90.68|109.72|50.47|98.92|706.58|1269.52|1536.08|18.22|98.92|607.66|625.88|-661.86| +2450859|54722|7190|74070||||2|79|2283||||56.07||168.21|133.50|186.90|3.36||||| +2451481|53733|5260|90419|1281618|4017|4016|8|148|2284|43|89.46|168.18|126.13|0.00|5423.59|3846.78|7231.74|488.12|0.00|5423.59|5911.71|1576.81| +||12121|90419|1281618|||8||2284|39|63.08|122.37|||3245.19||4772.43|162.25||3245.19|3407.44|785.07| +2451481|53733|5602|90419|1281618|4017|4016|8|64|2284|68|85.18|122.65|82.17|0.00|5587.56|5792.24|8340.20|447.00|0.00|5587.56|6034.56|-204.68| +2451481|53733|14359|90419|1281618|4017|4016|8|286|2284|40|86.40|119.23|23.84|0.00|953.60|3456.00|4769.20|47.68|0.00|953.60|1001.28|-2502.40| +2451481|53733|1465|90419|1281618|4017|4016|8|273|2284|80|82.70|128.18|96.13|0.00|7690.40|6616.00|10254.40|384.52|0.00|7690.40|8074.92|1074.40| +||13096|||4017|4016|8||2284|||6.23|5.66|||85.28|161.98|11.77||147.16|158.93|61.88| +2451481|53733|11242|90419|1281618|4017|4016|8|166|2284|89|89.70|106.74|85.39|0.00|7599.71|7983.30|9499.86|683.97|0.00|7599.71|8283.68|-383.59| +2451481|53733|16874|90419|1281618|4017|4016|8|153|2284|39|2.13|3.51|1.01|16.93|39.39|83.07|136.89|0.00|16.93|22.46|22.46|-60.61| +2451481|53733|4082|90419|1281618|4017|4016|8|122|2284|30|22.75|33.44|17.72|0.00|531.60|682.50|1003.20|47.84|0.00|531.60|579.44|-150.90| +2451481|53733|4706|90419|1281618|4017|4016|8|129|2284|5|69.19|91.33|80.37|0.00|401.85|345.95|456.65|16.07|0.00|401.85|417.92|55.90| +2451481|53733|6998|90419|1281618|4017|4016|8|49|2284|72|11.28|22.56|5.18|0.00|372.96|812.16|1624.32|29.83|0.00|372.96|402.79|-439.20| +2451481|53733|17851|90419|1281618|4017|4016|8|143|2284|28|98.25|163.09|141.88|0.00|3972.64|2751.00|4566.52|357.53|0.00|3972.64|4330.17|1221.64| +2451481|53733|9109|90419|1281618|4017|4016|8|103|2284|51|63.89|116.91|84.17|0.00|4292.67|3258.39|5962.41|171.70|0.00|4292.67|4464.37|1034.28| +2451481|53733|12187|90419|1281618|4017|4016|8|243|2284|77|35.70|67.83|27.81|0.00|2141.37|2748.90|5222.91|0.00|0.00|2141.37|2141.37|-607.53| +2451481|53733|4894|90419|1281618|4017|4016|8|71|2284|100|30.99|36.25|35.52|0.00|3552.00|3099.00|3625.00|142.08|0.00|3552.00|3694.08|453.00| +2451481|53733|8858|90419|1281618|4017|4016|8|263|2284|87|6.35|12.06|10.13|0.00|881.31|552.45|1049.22|17.62|0.00|881.31|898.93|328.86| +2452284|50683|14461|18591|860331|6424|3717|7|47|2285|86|95.66|181.75|78.15|0.00|6720.90|8226.76|15630.50|134.41|0.00|6720.90|6855.31|-1505.86| +2452284|50683|1575|18591|860331|6424|3717|7|251|2285|35|40.24|47.88|39.74|0.00|1390.90|1408.40|1675.80|111.27|0.00|1390.90|1502.17|-17.50| +2452284|50683|15537|18591|860331|6424|3717|7|40|2285|14|99.61|198.22|148.66|0.00|2081.24|1394.54|2775.08|20.81|0.00|2081.24|2102.05|686.70| +2452284|50683|9169|18591|860331|6424|3717|7|84|2285|48|76.65|147.93|8.87|0.00|425.76|3679.20|7100.64|38.31|0.00|425.76|464.07|-3253.44| +2452284||1734||||3717|||2285|5|||||||61.50||||0.93|-32.59| +2452284|50683|1671|18591|860331|6424|3717|7|224|2285|62|73.21|85.65|35.97|0.00|2230.14|4539.02|5310.30|22.30|0.00|2230.14|2252.44|-2308.88| +2452284|50683|5227|18591|860331|6424|3717|7|153|2285|59|36.79|62.17|60.92|0.00|3594.28|2170.61|3668.03|0.00|0.00|3594.28|3594.28|1423.67| +2452284|50683|2517|18591|860331|6424|3717|7|202|2285|16|77.18|115.77|64.83|456.40|1037.28|1234.88|1852.32|34.85|456.40|580.88|615.73|-654.00| +2452284|50683|11481|18591|860331|6424|3717|7|231|2285|43|97.73|183.73|176.38|0.00|7584.34|4202.39|7900.39|682.59|0.00|7584.34|8266.93|3381.95| +2450893|36340|16105|11557|597075|5440|34205|10|61|2286|42|88.37|169.67|154.39|0.00|6484.38|3711.54|7126.14|129.68|0.00|6484.38|6614.06|2772.84| +2450893|36340|4027|11557|597075|5440|34205|10|142|2286|5|87.00|140.07|12.60|0.00|63.00|435.00|700.35|5.67|0.00|63.00|68.67|-372.00| +2450893|36340|142|11557|597075|5440|34205|10|27|2286|32|28.05|31.41|20.73|378.11|663.36|897.60|1005.12|5.70|378.11|285.25|290.95|-612.35| +2450893|36340|15920|11557|597075|5440|34205|10|256|2286|40|11.77|17.18|9.27|0.00|370.80|470.80|687.20|25.95|0.00|370.80|396.75|-100.00| +2450893|36340|3254|11557|597075|5440|34205|10|188|2286|82|31.80|39.75|7.95|0.00|651.90|2607.60|3259.50|13.03|0.00|651.90|664.93|-1955.70| +2450893|36340|15884|11557|597075|5440|34205|10|158|2286|17|85.16|101.34|26.34|40.30|447.78|1447.72|1722.78|28.52|40.30|407.48|436.00|-1040.24| +2450893|36340|11512|11557|597075|5440|34205|10|93|2286|6|49.06|73.59|72.11|0.00|432.66|294.36|441.54|17.30|0.00|432.66|449.96|138.30| +|36340|14084||597075|||10|149|2286||37.14||53.94||1941.84|1337.04|2339.64||||1941.84|| +2450893|36340|2617|11557|597075|5440|34205|10|55|2286|57|59.92|94.67|9.46|512.25|539.22|3415.44|5396.19|2.42|512.25|26.97|29.39|-3388.47| +2450893|36340|2744|11557|597075|5440|34205|10|132|2286|82|26.13|32.40|2.59|0.00|212.38|2142.66|2656.80|0.00|0.00|212.38|212.38|-1930.28| +2450893|36340|17768|11557|597075|5440|34205|10|113|2286|59|27.64|40.07|6.01|0.00|354.59|1630.76|2364.13|31.91|0.00|354.59|386.50|-1276.17| +2450893|36340|7316|11557|597075|5440|34205|10|49|2286|83|83.58|148.77|81.82|5976.13|6791.06|6937.14|12347.91|48.89|5976.13|814.93|863.82|-6122.21| +2450893|36340|14563|11557|597075|5440|34205|10|75|2286|5|8.44|13.33|6.26|0.00|31.30|42.20|66.65|2.81|0.00|31.30|34.11|-10.90| +2450893|36340|6844|11557|597075|5440|34205|10|62|2286|73|71.78|133.51|124.16|0.00|9063.68|5239.94|9746.23|453.18|0.00|9063.68|9516.86|3823.74| +2450893|36340|3794|11557|597075|5440|34205|10|114|2286|55|13.32|24.50|6.86|0.00|377.30|732.60|1347.50|3.77|0.00|377.30|381.07|-355.30| +2451160|42630|3715|64151|1115224|3544|31017|2|56|2287|39|23.53|24.00|20.88|0.00|814.32|917.67|936.00|40.71|0.00|814.32|855.03|-103.35| +2451160|42630|2534|64151|1115224|3544|31017|2|294|2287|20|13.75|17.87|7.68|0.00|153.60|275.00|357.40|1.53|0.00|153.60|155.13|-121.40| +2451160||15964|64151||3544||2|10|2287|9||13.65|12.14|||99.09||3.27|||112.53|| +2451160|42630|15764|64151|1115224|3544|31017|2|24|2287|5|35.77|60.09|51.07|97.03|255.35|178.85|300.45|7.91|97.03|158.32|166.23|-20.53| +2451160|42630|9638|64151|1115224|3544|31017|2|115|2287|42|92.26|107.02|1.07|38.64|44.94|3874.92|4494.84|0.44|38.64|6.30|6.74|-3868.62| +2451160|42630|7717|64151|1115224|3544|31017|2|42|2287|92|47.45|48.87|25.90|0.00|2382.80|4365.40|4496.04|71.48|0.00|2382.80|2454.28|-1982.60| +2451160|42630|73|64151|1115224|3544|31017|2|244|2287|1|87.87|102.80|41.12|0.00|41.12|87.87|102.80|1.64|0.00|41.12|42.76|-46.75| +2451160|42630|8270|64151|1115224|3544|31017|2|178|2287|21|92.53|159.15|66.84|0.00|1403.64|1943.13|3342.15|126.32|0.00|1403.64|1529.96|-539.49| +2451160|42630|22|64151|1115224|3544|31017|2|59|2287|90|5.54|7.75|1.39|0.00|125.10|498.60|697.50|8.75|0.00|125.10|133.85|-373.50| +2451160|42630|17293||1115224||||87|2287|1||41.78|34.67|24.26||29.22|||24.26||11.03|| +2451160|42630|17396|64151|1115224|3544|31017|2|294|2287|41|21.84|38.22|31.34|0.00|1284.94|895.44|1567.02|89.94|0.00|1284.94|1374.88|389.50| +2451160|42630|346|64151|1115224|3544|31017|2|141|2287|77|73.43|118.22|28.37|764.57|2184.49|5654.11|9102.94|113.59|764.57|1419.92|1533.51|-4234.19| +2451160|42630|11311|64151|1115224|3544|31017|2|192|2287|17|76.83|114.47|13.73|0.00|233.41|1306.11|1945.99|16.33|0.00|233.41|249.74|-1072.70| +2450844|35744|7856|87994|1736370|5993|41216|7|264|2288|60|1.40|2.63|2.18|45.78|130.80|84.00|157.80|7.65|45.78|85.02|92.67|1.02| +2450844|35744|5060|87994|1736370|5993|41216|7|85|2288|77|58.88|109.51|12.04|0.00|927.08|4533.76|8432.27|18.54|0.00|927.08|945.62|-3606.68| +2450844|35744|4585|87994|1736370|5993|41216|7|115|2288|36|83.72|160.74|102.87|1999.79|3703.32|3013.92|5786.64|102.21|1999.79|1703.53|1805.74|-1310.39| +2450844|35744|7729|87994|1736370|5993|41216|7|56|2288|95|80.91|139.97|74.18|0.00|7047.10|7686.45|13297.15|211.41|0.00|7047.10|7258.51|-639.35| +2450844|35744|7411|87994|1736370|5993|41216|7|278|2288|1|9.69|16.18|9.38|4.69|9.38|9.69|16.18|0.37|4.69|4.69|5.06|-5.00| +2450844|35744|13390|87994|1736370|5993|41216|7|127|2288|38|13.07|18.42|7.55|0.00|286.90|496.66|699.96|22.95|0.00|286.90|309.85|-209.76| +2450844|35744|16018|87994|1736370|5993|41216|7|195|2288|61|69.52|98.71|98.71|3853.63|6021.31|4240.72|6021.31|108.38|3853.63|2167.68|2276.06|-2073.04| +2450844|35744|4813|87994|1736370|5993|41216|7|54|2288|59|96.22|137.59|107.32|759.82|6331.88|5676.98|8117.81|0.00|759.82|5572.06|5572.06|-104.92| +2450844|35744|6656|87994|1736370|5993|41216|7|172|2288|36|84.62|132.85|41.18|0.00|1482.48|3046.32|4782.60|44.47|0.00|1482.48|1526.95|-1563.84| +2450844|35744|721|87994|1736370|5993|41216|7|169|2288|25|60.32|103.75|87.15|0.00|2178.75|1508.00|2593.75|87.15|0.00|2178.75|2265.90|670.75| +2450844|35744|16366|87994|1736370|5993|41216|7|104|2288|64|65.18|96.46|6.75|0.00|432.00|4171.52|6173.44|17.28|0.00|432.00|449.28|-3739.52| +2452457|47018|12651|89207|695459|2511|26763|4|90|2289|72|33.11|50.98|34.15|0.00|2458.80|2383.92|3670.56|221.29|0.00|2458.80|2680.09|74.88| +||5844|||2511|26763|4||2289|17|69.72|135.25||0.00|91.97||||0.00|91.97|91.97|-1093.27| +2452457|47018|14688|89207|695459|2511|26763|4|20|2289|12|15.99|17.42|7.66|0.00|91.92|191.88|209.04|6.43|0.00|91.92|98.35|-99.96| +2452457|47018|6069|89207|695459|2511|26763|4|50|2289|28|67.71|127.97|116.45|2119.39|3260.60|1895.88|3583.16|102.70|2119.39|1141.21|1243.91|-754.67| +2452457|47018|13563|89207|695459|2511|26763|4|60|2289|75|35.45|59.55|28.58|0.00|2143.50|2658.75|4466.25|128.61|0.00|2143.50|2272.11|-515.25| +2452457|47018|13959|89207|695459|2511|26763|4|31|2289|79|51.47|74.11|9.63|22.82|760.77|4066.13|5854.69|59.03|22.82|737.95|796.98|-3328.18| +2452457|47018|1809|89207|695459|2511|26763|4|138|2289|1|67.33|109.74|87.79|0.00|87.79|67.33|109.74|7.90|0.00|87.79|95.69|20.46| +2452457|47018|16195|89207|695459|2511|26763|4|222|2289|72|60.37|98.40|42.31|0.00|3046.32|4346.64|7084.80|182.77|0.00|3046.32|3229.09|-1300.32| +|47018|15939|89207||2511|26763|4||2289||||19.18||364.42||1041.20|10.05||251.45||| +2452457|47018|5238|89207|695459|2511|26763|4|155|2289|51|50.68|54.73|4.37|0.00|222.87|2584.68|2791.23|17.82|0.00|222.87|240.69|-2361.81| +2451203|65884|1219|3820|599043|3513|43936|10|258|2290|9|88.41|160.02|126.41|0.00|1137.69|795.69|1440.18|91.01|0.00|1137.69|1228.70|342.00| +2451203|65884|331|3820|599043|3513|43936|10|174|2290|93|49.85|98.70|3.94|0.00|366.42|4636.05|9179.10|10.99|0.00|366.42|377.41|-4269.63| +2451203|65884|4375|3820|599043|3513|43936|10|207|2290|5|3.12|5.02|3.51|0.00|17.55|15.60|25.10|0.17|0.00|17.55|17.72|1.95| +2451203|65884|13870|3820|599043|3513|43936|10|280|2290|94|57.59|106.54|92.68|0.00|8711.92|5413.46|10014.76|87.11|0.00|8711.92|8799.03|3298.46| +2451203|65884|1768|3820|599043|3513|43936|10|138|2290|31|73.68|74.41|13.39|294.71|415.09|2284.08|2306.71|8.42|294.71|120.38|128.80|-2163.70| +2451203|65884|260|3820|599043|3513|43936|10|272|2290|64|25.67|27.21|0.00|0.00|0.00|1642.88|1741.44|0.00|0.00|0.00|0.00|-1642.88| +2451203|65884|9340|3820|599043|3513|43936|10|229|2290|55|33.88|66.06|61.43|405.43|3378.65|1863.40|3633.30|59.46|405.43|2973.22|3032.68|1109.82| +2451203|65884|4246|3820|599043|3513|43936|10|3|2290|76|7.51|8.71|6.53|0.00|496.28|570.76|661.96|0.00|0.00|496.28|496.28|-74.48| +2451203|65884|17524|3820|599043|3513|43936|10|184|2290|99|3.20|5.31|0.79|0.00|78.21|316.80|525.69|2.34|0.00|78.21|80.55|-238.59| +2451203|65884|16969|3820|599043|3513|43936|10|13|2290|37|16.50|29.04|10.74|0.00|397.38|610.50|1074.48|35.76|0.00|397.38|433.14|-213.12| +2451203|65884|7063|3820|599043|3513|43936|10|231|2290|80|35.93|45.99|45.07|0.00|3605.60|2874.40|3679.20|0.00|0.00|3605.60|3605.60|731.20| +2451203|65884|1624|3820|599043|3513|43936|10|54|2290|86|18.81|22.76|16.38|0.00|1408.68|1617.66|1957.36|56.34|0.00|1408.68|1465.02|-208.98| +2451150|62555|1705|17766|620290|2814|13785|1|256|2291|72|78.57|96.64|85.04|0.00|6122.88|5657.04|6958.08|551.05|0.00|6122.88|6673.93|465.84| +2451150|62555|11362|17766|620290|2814|13785|1|22|2291|25|94.97|125.36|104.04|0.00|2601.00|2374.25|3134.00|156.06|0.00|2601.00|2757.06|226.75| +2451150|62555|7078|17766|620290|2814|13785|1|48|2291|42|71.37|83.50|5.84|0.00|245.28|2997.54|3507.00|22.07|0.00|245.28|267.35|-2752.26| +2451150|62555|10604|17766|620290|2814|13785|1|84|2291|40|72.20|96.02|70.09|0.00|2803.60|2888.00|3840.80|224.28|0.00|2803.60|3027.88|-84.40| +2451150|62555|10213|17766|620290|2814|13785|1|65|2291|45|69.94|87.42|49.82|0.00|2241.90|3147.30|3933.90|67.25|0.00|2241.90|2309.15|-905.40| +2451150|62555|2677|17766|620290|2814|13785|1|255|2291|61|13.14|20.76|17.64|0.00|1076.04|801.54|1266.36|43.04|0.00|1076.04|1119.08|274.50| +2451150|62555|16808|17766|620290|2814|13785|1|268|2291|25|95.15|124.64|16.20|0.00|405.00|2378.75|3116.00|36.45|0.00|405.00|441.45|-1973.75| +2451150|62555|7798|17766|620290|2814|13785|1|219|2291|87|11.36|21.01|3.36|277.70|292.32|988.32|1827.87|0.14|277.70|14.62|14.76|-973.70| +2451150|62555|15022|17766|620290|2814|13785|1|30|2291|99|84.70|89.78|29.62|1583.48|2932.38|8385.30|8888.22|80.93|1583.48|1348.90|1429.83|-7036.40| +2451150|62555|7202|17766|620290|2814|13785|1|93|2291|90|77.09|94.04|42.31|0.00|3807.90|6938.10|8463.60|190.39|0.00|3807.90|3998.29|-3130.20| +2451150|62555|5122|17766|620290|2814|13785|1|292|2291|27|8.85|15.57|10.27|0.00|277.29|238.95|420.39|2.77|0.00|277.29|280.06|38.34| +2451616|52037|2450|46594|432297|1200|26856|4|130|2292|65|30.70|47.89|7.18|0.00|466.70|1995.50|3112.85|28.00|0.00|466.70|494.70|-1528.80| +2451616||12599|46594|432297|1200|26856|4||2292|57|||71.34|0.00|||||0.00|||237.12| +2451616|52037|1945|46594|432297|1200|26856|4|193|2292|97|21.53|23.25|6.27|291.93|608.19|2088.41|2255.25|22.13|291.93|316.26|338.39|-1772.15| +2451616|52037|2695|46594|432297|1200|26856|4|37|2292|62|38.13|73.59|16.92|0.00|1049.04|2364.06|4562.58|83.92|0.00|1049.04|1132.96|-1315.02| +2451616|52037|3395|46594|432297|1200|26856|4|49|2292|85|55.39|60.37|22.94|0.00|1949.90|4708.15|5131.45|77.99|0.00|1949.90|2027.89|-2758.25| +2451616|52037|2221|46594|432297|1200|26856|4|284|2292|35|92.84|165.25|165.25|0.00|5783.75|3249.40|5783.75|173.51|0.00|5783.75|5957.26|2534.35| +2451616|52037|12289|46594|432297|1200|26856|4|92|2292|10|33.32|34.65|21.13|0.00|211.30|333.20|346.50|2.11|0.00|211.30|213.41|-121.90| +2451616|52037|6431|46594|432297|1200|26856|4|188|2292|41|10.92|15.61|8.27|0.00|339.07|447.72|640.01|20.34|0.00|339.07|359.41|-108.65| +2451616|52037|13759|46594|432297|1200|26856|4|6|2292|34|12.07|18.46|4.98|0.00|169.32|410.38|627.64|0.00|0.00|169.32|169.32|-241.06| +2451616|52037|16523|46594|432297|1200|26856|4|66|2292|82|8.80|13.11|6.16|0.00|505.12|721.60|1075.02|10.10|0.00|505.12|515.22|-216.48| +2452605|33296|13251|39620|102001|6335|39516|2|124|2293|20|31.94|49.82|26.40|0.00|528.00|638.80|996.40|31.68|0.00|528.00|559.68|-110.80| +|33296|12336|39620||||||2293|76|60.76|117.26|31.66|0.00||4617.76|8911.76|120.30|0.00|2406.16||-2211.60| +2452605|33296|16927|39620|102001|6335|39516|2|26|2293|20|73.81|93.00|53.94|0.00|1078.80|1476.20|1860.00|86.30|0.00|1078.80|1165.10|-397.40| +2452605|33296|8364|39620|102001|6335|39516|2|252|2293|76|73.91|96.82|55.18|0.00|4193.68|5617.16|7358.32|293.55|0.00|4193.68|4487.23|-1423.48| +2452605|33296|14347|39620|102001|6335|39516|2|300|2293|79|90.46|121.21|32.72|516.97|2584.88|7146.34|9575.59|144.75|516.97|2067.91|2212.66|-5078.43| +2452605|33296|11517|39620|102001|6335|39516|2|156|2293|75|58.17|59.91|29.35|0.00|2201.25|4362.75|4493.25|66.03|0.00|2201.25|2267.28|-2161.50| +2452605|33296|1099|39620|102001|6335|39516|2|163|2293|87|64.36|111.34|25.60|0.00|2227.20|5599.32|9686.58|22.27|0.00|2227.20|2249.47|-3372.12| +2452605||15541||102001||39516||145|2293||25.82|36.40||0.00||309.84||2.70|0.00|135.36|138.06|| +2452605|33296|2244|39620|102001|6335|39516|2|257|2293|27|51.92|55.03|23.11|0.00|623.97|1401.84|1485.81|49.91|0.00|623.97|673.88|-777.87| +2452213|42713|3865|33348|1121849|2350|1251|2|286|2294|29|54.49|75.19|48.87|0.00|1417.23|1580.21|2180.51|127.55|0.00|1417.23|1544.78|-162.98| +2452213|42713|8933|33348|1121849|2350|1251|2|249|2294|99|32.61|59.67|1.79|0.00|177.21|3228.39|5907.33|14.17|0.00|177.21|191.38|-3051.18| +2452213|42713|8939|33348|1121849|2350|1251|2|134|2294|78|51.76|77.64|3.10|0.00|241.80|4037.28|6055.92|21.76|0.00|241.80|263.56|-3795.48| +2452213|42713|3627|33348|1121849|2350|1251|2|33|2294|60|71.17|140.91|23.95|0.00|1437.00|4270.20|8454.60|43.11|0.00|1437.00|1480.11|-2833.20| +2452213|42713|8445|33348|1121849|2350|1251|2|179|2294|59|26.94|34.48|14.13|0.00|833.67|1589.46|2034.32|25.01|0.00|833.67|858.68|-755.79| +2452213|42713|4827|33348|1121849|2350|1251|2|273|2294|8|53.82|101.71|46.78|0.00|374.24|430.56|813.68|7.48|0.00|374.24|381.72|-56.32| +2452213|42713|7915|33348|1121849|2350|1251|2|208|2294|7|97.27|193.56|106.45|0.00|745.15|680.89|1354.92|59.61|0.00|745.15|804.76|64.26| +2452213|42713|6087|33348|1121849|2350|1251|2|162|2294|11|24.87|38.79|34.13|0.00|375.43|273.57|426.69|26.28|0.00|375.43|401.71|101.86| +||2107|33348|1121849||||275|2294|||155.43|141.44|0.00|9476.48||10413.81||0.00||9760.77|3165.08| +2452213|42713|16959|33348|1121849|2350|1251|2|55|2294|32|93.15|174.19|168.96|0.00|5406.72|2980.80|5574.08|324.40|0.00|5406.72|5731.12|2425.92| +2452213|42713|521|33348|1121849|2350|1251|2|278|2294|38|45.74|87.82|37.76|0.00|1434.88|1738.12|3337.16|129.13|0.00|1434.88|1564.01|-303.24| +|42713|5759|33348||2350||2||2294||18.36|23.31|22.14|0.00||1578.96|2004.66||0.00|1904.04|2018.28|| +2452213||8913||||1251|2||2294|61|76.17||||7322.44|4646.37||73.22||7322.44||2676.07| +2451512|68104|5366|95393|1664403|6213|16423|4|265|2295|3|77.95|128.61|42.44|0.00|127.32|233.85|385.83|11.45|0.00|127.32|138.77|-106.53| +2451512|68104|15676|95393|1664403|6213|16423|4|201|2295|58|58.78|93.46|18.69|476.96|1084.02|3409.24|5420.68|24.28|476.96|607.06|631.34|-2802.18| +2451512|68104|11762|95393|1664403|6213|16423|4|2|2295|44|9.04|14.55|12.94|0.00|569.36|397.76|640.20|51.24|0.00|569.36|620.60|171.60| +2451512|68104|2014|95393|1664403|6213|16423|4|204|2295|4|55.87|74.30|73.55|0.00|294.20|223.48|297.20|17.65|0.00|294.20|311.85|70.72| +2451512|68104|8090|95393|1664403|6213|16423|4|240|2295|48|99.04|104.98|61.93|0.00|2972.64|4753.92|5039.04|29.72|0.00|2972.64|3002.36|-1781.28| +2451512|68104|13189|95393|1664403|6213|16423|4|182|2295|61|87.50|163.62|135.80|0.00|8283.80|5337.50|9980.82|248.51|0.00|8283.80|8532.31|2946.30| +2451512|68104|14330|95393|1664403|6213|16423|4|39|2295|30|72.14|114.70|96.34|28.90|2890.20|2164.20|3441.00|0.00|28.90|2861.30|2861.30|697.10| +2451512|68104|14284|95393|1664403|6213|16423|4|276|2295|62|12.58|25.16|21.38|0.00|1325.56|779.96|1559.92|119.30|0.00|1325.56|1444.86|545.60| +2451512|68104|14452|95393|1664403|6213|16423|4|54|2295|69|46.18|67.88|40.72|0.00|2809.68|3186.42|4683.72|196.67|0.00|2809.68|3006.35|-376.74| +2451512|68104|13346|95393|1664403|6213|16423|4|233|2295|10|35.25|69.44|29.85|0.00|298.50|352.50|694.40|23.88|0.00|298.50|322.38|-54.00| +2451512|68104|4369|95393|1664403|6213|16423|4|71|2295|6|98.64|106.53|11.71|0.00|70.26|591.84|639.18|4.91|0.00|70.26|75.17|-521.58| +2451512|68104|6608|95393|1664403|6213|16423|4|154|2295|53|94.67|156.20|92.15|0.00|4883.95|5017.51|8278.60|390.71|0.00|4883.95|5274.66|-133.56| +2451512|68104|10936|95393|1664403|6213|16423|4|112|2295|79|85.54|147.98|32.55|0.00|2571.45|6757.66|11690.42|25.71|0.00|2571.45|2597.16|-4186.21| +2451512|68104|8215|95393|1664403|6213|16423|4|182|2295|6|72.48|88.42|46.86|0.00|281.16|434.88|530.52|2.81|0.00|281.16|283.97|-153.72| +2450930|66274|8575|92166|158101|483|1218|4|151|2296|93|62.62|65.12|38.42|0.00|3573.06|5823.66|6056.16|142.92|0.00|3573.06|3715.98|-2250.60| +2450930|66274|6991|92166|158101|483|1218|4|5|2296|74|2.69|3.95|1.97|75.80|145.78|199.06|292.30|5.59|75.80|69.98|75.57|-129.08| +2450930|66274|2318|92166|158101|483|1218|4|117|2296|85|79.64|143.35|139.04|0.00|11818.40|6769.40|12184.75|354.55|0.00|11818.40|12172.95|5049.00| +2450930|66274|1328|92166|158101|483|1218|4|225|2296|55|69.49|138.98|122.30|0.00|6726.50|3821.95|7643.90|403.59|0.00|6726.50|7130.09|2904.55| +2450930|66274|7876|92166|158101|483|1218|4|56|2296|59|5.03|6.58|3.88|0.00|228.92|296.77|388.22|6.86|0.00|228.92|235.78|-67.85| +2450930|66274|7159|92166|158101|483|1218|4|220|2296|10|99.01|183.16|153.85|0.00|1538.50|990.10|1831.60|30.77|0.00|1538.50|1569.27|548.40| +2450930|66274|14012|92166|158101|483|1218|4|244|2296|73|30.37|58.91|8.83|0.00|644.59|2217.01|4300.43|19.33|0.00|644.59|663.92|-1572.42| +2450930|66274|13507|92166|158101|483|1218|4|147|2296|46|51.32|80.05|69.64|0.00|3203.44|2360.72|3682.30|224.24|0.00|3203.44|3427.68|842.72| +2450930|66274|7016|92166|158101|483|1218|4|222|2296|27|42.28|61.72|58.01|0.00|1566.27|1141.56|1666.44|93.97|0.00|1566.27|1660.24|424.71| +2450930|66274|2389|92166|158101|483|1218|4|127|2296|31|61.61|83.17|56.55|0.00|1753.05|1909.91|2578.27|17.53|0.00|1753.05|1770.58|-156.86| +2450930|66274|1477|92166|158101|483|1218|4|144|2296|11|58.90|86.58|13.85|0.00|152.35|647.90|952.38|1.52|0.00|152.35|153.87|-495.55| +2450930|66274|11432|92166|158101|483|1218|4|176|2296|61|27.82|49.79|27.88|0.00|1700.68|1697.02|3037.19|85.03|0.00|1700.68|1785.71|3.66| +2450930|66274|10567|92166|158101|483|1218|4|166|2296|10|15.19|22.32|2.90|0.00|29.00|151.90|223.20|1.45|0.00|29.00|30.45|-122.90| +2450930|66274|6740|92166|158101|483|1218|4|174|2296|71|34.87|64.16|28.87|0.00|2049.77|2475.77|4555.36|143.48|0.00|2049.77|2193.25|-426.00| +2451886|35495|17069|24014|177205|796|32340|8|154|2297|5|12.60|13.73|1.51|0.00|7.55|63.00|68.65|0.52|0.00|7.55|8.07|-55.45| +2451886|35495|16097|24014|177205|796|32340|8|66|2297|11|53.74|88.67|24.82|0.00|273.02|591.14|975.37|24.57|0.00|273.02|297.59|-318.12| +2451886|35495|9932|24014|177205|796|32340|8|210|2297|27|35.61|43.08|41.35|0.00|1116.45|961.47|1163.16|44.65|0.00|1116.45|1161.10|154.98| +2451886|35495|11132|24014|177205|796|32340|8|89|2297|60|1.01|1.49|1.07|23.75|64.20|60.60|89.40|1.61|23.75|40.45|42.06|-20.15| +2451886|35495|440|24014|177205|796|32340|8|252|2297|74|81.85|99.85|76.88|0.00|5689.12|6056.90|7388.90|56.89|0.00|5689.12|5746.01|-367.78| +2451886|35495|10873|24014|177205|796|32340|8|42|2297|4|45.58|61.07|31.14|0.00|124.56|182.32|244.28|11.21|0.00|124.56|135.77|-57.76| +2451886|35495|833|24014|177205|796|32340|8|44|2297|72|76.01|76.77|69.09|0.00|4974.48|5472.72|5527.44|248.72|0.00|4974.48|5223.20|-498.24| +2451886|35495|1241|24014|177205|796|32340|8|40|2297|19|34.96|35.30|29.65|0.00|563.35|664.24|670.70|45.06|0.00|563.35|608.41|-100.89| +||1543|24014|177205||32340|8||2297||21.28||14.42|0.00|605.64|893.76|1009.68|6.05|0.00|||-288.12| +2451886|35495|15515|24014|177205|796|32340|8|143|2297|86|77.55|84.52|70.15|0.00|6032.90|6669.30|7268.72|422.30|0.00|6032.90|6455.20|-636.40| +2452380|44805|6819|43914|1514368|4822|48476|7|32|2298|61|76.10|144.59|120.00|0.00|7320.00|4642.10|8819.99|219.60|0.00|7320.00|7539.60|2677.90| +2452380|44805|942|43914|1514368|4822|48476|7|267|2298|66|39.60|45.93|16.07|0.00|1060.62|2613.60|3031.38|84.84|0.00|1060.62|1145.46|-1552.98| +2452380|44805|12201|43914|1514368|4822|48476|7|151|2298|100|33.08|57.22|2.86|0.00|286.00|3308.00|5722.00|14.30|0.00|286.00|300.30|-3022.00| +2452380|44805|12423|43914|1514368|4822|48476|7|40|2298|81|43.38|68.10|44.94|0.00|3640.14|3513.78|5516.10|218.40|0.00|3640.14|3858.54|126.36| +2452380|44805|2101|43914|1514368|4822|48476|7|215|2298|46|56.91|97.88|65.57|0.00|3016.22|2617.86|4502.48|241.29|0.00|3016.22|3257.51|398.36| +2452380|44805|15942|43914|1514368|4822|48476|7|130|2298|42|55.84|101.07|59.63|0.00|2504.46|2345.28|4244.94|0.00|0.00|2504.46|2504.46|159.18| +2452380|44805|6585|43914|1514368|4822|48476|7|5|2298|77|96.92|157.01|114.61|8383.72|8824.97|7462.84|12089.77|26.47|8383.72|441.25|467.72|-7021.59| +2452380|44805|10627|43914|1514368|4822|48476|7|220|2298|23|59.34|64.68|7.11|0.00|163.53|1364.82|1487.64|0.00|0.00|163.53|163.53|-1201.29| +2452380|44805|4089|43914|1514368|4822|48476|7|196|2298|1|95.34|167.79|115.77|5.78|115.77|95.34|167.79|0.00|5.78|109.99|109.99|14.65| +2452380|44805|6063|43914|1514368|4822|48476|7|222|2298|11|87.99|168.06|60.50|0.00|665.50|967.89|1848.66|19.96|0.00|665.50|685.46|-302.39| +2452380|44805|5623|43914|1514368|4822|48476|7|264|2298|81|20.79|26.61|6.38|0.00|516.78|1683.99|2155.41|10.33|0.00|516.78|527.11|-1167.21| +2452380|44805|15990|43914|1514368|4822|48476|7|258|2298|92|70.90|124.07|16.12|0.00|1483.04|6522.80|11414.44|44.49|0.00|1483.04|1527.53|-5039.76| +2451849|59853|6905|69380|1756407|5834|8461|10|161|2299|27|23.92|39.70|1.19|0.00|32.13|645.84|1071.90|0.64|0.00|32.13|32.77|-613.71| +2451849|59853|356|69380|1756407|5834|8461|10|212|2299|60|55.51|97.14|19.42|0.00|1165.20|3330.60|5828.40|81.56|0.00|1165.20|1246.76|-2165.40| +2451849|59853|2059|69380|1756407|5834|8461|10|66|2299|78|70.24|70.24|59.70|0.00|4656.60|5478.72|5478.72|232.83|0.00|4656.60|4889.43|-822.12| +2451849|59853|13676|69380|1756407|5834|8461|10|111|2299|72|52.04|60.36|38.63|0.00|2781.36|3746.88|4345.92|83.44|0.00|2781.36|2864.80|-965.52| +2451849|59853|17591|69380|1756407|5834|8461|10|207|2299|33|32.79|42.29|30.44|0.00|1004.52|1082.07|1395.57|80.36|0.00|1004.52|1084.88|-77.55| +2451849|59853|14288|69380|1756407|5834|8461|10|185|2299|67|85.69|148.24|85.97|1209.59|5759.99|5741.23|9932.08|91.00|1209.59|4550.40|4641.40|-1190.83| +2451849|59853|7757|69380|1756407|5834|8461|10|286|2299|2|72.39|135.36|14.88|0.00|29.76|144.78|270.72|1.48|0.00|29.76|31.24|-115.02| +|59853|8240|69380||5834|8461|10||2299||||42.89||557.57|432.25|||||579.87|125.32| +2451849|59853|10271|69380|1756407|5834|8461|10|5|2299|52|91.59|130.97|51.07|0.00|2655.64|4762.68|6810.44|79.66|0.00|2655.64|2735.30|-2107.04| +2451849|59853|11264|69380|1756407|5834|8461|10|199|2299|10|13.63|14.17|13.17|0.00|131.70|136.30|141.70|6.58|0.00|131.70|138.28|-4.60| +2451849|59853|4091|69380|1756407|5834|8461|10|25|2299|29|9.69|16.37|4.09|0.00|118.61|281.01|474.73|9.48|0.00|118.61|128.09|-162.40| +2451849|59853|7387|69380|1756407|5834|8461|10|24|2299|21|34.07|60.98|52.44|0.00|1101.24|715.47|1280.58|77.08|0.00|1101.24|1178.32|385.77| +2451849|59853|14981|69380|1756407|5834|8461|10|218|2299|29|15.13|29.50|6.19|0.00|179.51|438.77|855.50|7.18|0.00|179.51|186.69|-259.26| +2451849|59853|2587|69380|1756407|5834|8461|10|271|2299|85|27.84|28.39|7.09|247.08|602.65|2366.40|2413.15|7.11|247.08|355.57|362.68|-2010.83| +2451849|59853|10382|69380|1756407|5834|8461|10|146|2299|97|63.80|120.58|14.46|0.00|1402.62|6188.60|11696.26|84.15|0.00|1402.62|1486.77|-4785.98| +2450991|53843|4030|53069|1043504|1139|39174|7|178|2300|91|66.75|126.82|123.01|5820.83|11193.91|6074.25|11540.62|161.19|5820.83|5373.08|5534.27|-701.17| +2450991|53843|17462|53069|1043504|1139|39174|7|271|2300|15|3.03|3.48|1.74|2.34|26.10|45.45|52.20|0.71|2.34|23.76|24.47|-21.69| +2450991|53843|16244|53069|1043504|1139|39174|7|104|2300|100|37.94|62.98|10.70|0.00|1070.00|3794.00|6298.00|85.60|0.00|1070.00|1155.60|-2724.00| +2450991|53843|10424|53069|1043504|1139|39174|7|260|2300|48|37.34|42.94|15.45|0.00|741.60|1792.32|2061.12|0.00|0.00|741.60|741.60|-1050.72| +2450991|53843|15608|53069|1043504|1139|39174|7|89|2300|64|28.69|53.07|37.67|1808.16|2410.88|1836.16|3396.48|48.21|1808.16|602.72|650.93|-1233.44| +2450991|53843|9253|53069|1043504|1139|39174|7|226|2300|58|74.87|80.85|46.89|0.00|2719.62|4342.46|4689.30|27.19|0.00|2719.62|2746.81|-1622.84| +2450991|53843|17500|53069|1043504|1139|39174|7|292|2300|77|93.15|137.86|9.65|0.00|743.05|7172.55|10615.22|14.86|0.00|743.05|757.91|-6429.50| +2450991|53843|289|53069|1043504|1139|39174|7|33|2300|78|92.92|99.42|60.64|0.00|4729.92|7247.76|7754.76|189.19|0.00|4729.92|4919.11|-2517.84| +2452185|61521|5087|60084|683156|628|33700|10|48|2301|43|32.79|61.31|26.97|927.76|1159.71|1409.97|2636.33|0.00|927.76|231.95|231.95|-1178.02| +2452185|61521|17339|60084|683156|628|33700|10|21|2301|32|74.78|106.93|94.09|0.00|3010.88|2392.96|3421.76|0.00|0.00|3010.88|3010.88|617.92| +2452185|61521|13655|60084|683156|628|33700|10|11|2301|43|38.12|72.42|52.14|0.00|2242.02|1639.16|3114.06|156.94|0.00|2242.02|2398.96|602.86| +2452185||15579|||628|33700|10|263|2301|94|||21.23|0.00||3646.26|4339.04|99.78|0.00|1995.62|2095.40|-1650.64| +2452185|61521|6587|60084|683156|628|33700|10|80|2301|33|17.98|33.44|23.07|0.00|761.31|593.34|1103.52|68.51|0.00|761.31|829.82|167.97| +2452185||3187|60084|683156|628||10|261|2301|68|69.67||||2103.24|4737.56|5684.80|||2103.24||| +2452185|61521|3009|60084|683156|628|33700|10|150|2301|15|35.82|60.53|6.65|20.94|99.75|537.30|907.95|2.36|20.94|78.81|81.17|-458.49| +2452185|61521|13195|60084|683156|628|33700|10|210|2301|53|65.60|106.92|7.48|0.00|396.44|3476.80|5666.76|15.85|0.00|396.44|412.29|-3080.36| +2452185|61521|11611|60084|683156|628|33700|10|116|2301|7|48.31|60.38|27.77|0.00|194.39|338.17|422.66|15.55|0.00|194.39|209.94|-143.78| +2452225|71776|4647|93552|1364814|76|44138|10|193|2302|44|9.18|12.30|2.33|0.00|102.52|403.92|541.20|6.15|0.00|102.52|108.67|-301.40| +2452225|71776|15829|93552|1364814|76|44138|10|141|2302|55|94.09|174.06|48.73|0.00|2680.15|5174.95|9573.30|187.61|0.00|2680.15|2867.76|-2494.80| +2452225|71776|7039|93552|1364814|76|44138|10|300|2302|50|50.97|56.06|31.95|0.00|1597.50|2548.50|2803.00|127.80|0.00|1597.50|1725.30|-951.00| +2452225|71776|11975|93552|1364814|76|44138|10|206|2302|94|80.55|140.96|28.19|0.00|2649.86|7571.70|13250.24|52.99|0.00|2649.86|2702.85|-4921.84| +2452225|71776|13451|93552|1364814|76|44138|10|136|2302|100|97.65|124.01|7.44|0.00|744.00|9765.00|12401.00|37.20|0.00|744.00|781.20|-9021.00| +2452225|71776|7623|93552|1364814|76|44138|10|45|2302|53|78.48|91.03|25.48|0.00|1350.44|4159.44|4824.59|94.53|0.00|1350.44|1444.97|-2809.00| +2452225|71776|15089|93552|1364814|76|44138|10|215|2302|60|74.46|125.09|36.27|0.00|2176.20|4467.60|7505.40|174.09|0.00|2176.20|2350.29|-2291.40| +2452225|71776|13791|93552|1364814|76|44138|10|91|2302|61|67.07|125.42|66.47|0.00|4054.67|4091.27|7650.62|202.73|0.00|4054.67|4257.40|-36.60| +2452225|71776|12383|93552|1364814|76|44138|10|208|2302|8|86.43|114.95|73.56|494.32|588.48|691.44|919.60|3.76|494.32|94.16|97.92|-597.28| +2451501|51868|625|91248|141649|50|15251|10|170|2303|93|84.23|116.23|12.78|0.00|1188.54|7833.39|10809.39|11.88|0.00|1188.54|1200.42|-6644.85| +2451501|51868|10318|91248|141649|50|15251|10|150|2303|87|69.76|69.76|56.50|0.00|4915.50|6069.12|6069.12|245.77|0.00|4915.50|5161.27|-1153.62| +2451501|51868|5683|91248|141649|50|15251|10|31|2303|13|35.61|53.41|19.22|0.00|249.86|462.93|694.33|0.00|0.00|249.86|249.86|-213.07| +2451501|51868|6868|91248|141649|50|15251|10|96|2303|63|56.98|113.39|90.71|0.00|5714.73|3589.74|7143.57|228.58|0.00|5714.73|5943.31|2124.99| +||11074||||15251|10|187|2303||11.80|15.57||||||11.81||590.64|602.45|-223.56| +2451501|51868|5986|91248|141649|50|15251|10|156|2303|84|61.45|110.61|3.31|0.00|278.04|5161.80|9291.24|19.46|0.00|278.04|297.50|-4883.76| +2451501|51868|7600|91248|141649|50|15251|10|182|2303|20|74.75|141.27|105.95|0.00|2119.00|1495.00|2825.40|84.76|0.00|2119.00|2203.76|624.00| +2451501|51868|9775|91248|141649|50|15251|10|56|2303|53|52.74|52.74|11.60|141.40|614.80|2795.22|2795.22|14.20|141.40|473.40|487.60|-2321.82| +|51868|2833|91248|141649||15251|10||2303|33|||20.51|0.00|676.83|||6.76|0.00|676.83||| +2451501|51868|4819|91248|141649|50|15251|10|147|2303|49|94.25|128.18|25.63|0.00|1255.87|4618.25|6280.82|113.02|0.00|1255.87|1368.89|-3362.38| +2451501|51868|2929|91248|141649|50|15251|10|183|2303|67|26.48|36.54|25.21|1672.17|1689.07|1774.16|2448.18|0.67|1672.17|16.90|17.57|-1757.26| +2451501|51868|11696|91248|141649|50|15251|10|121|2303|63|4.42|5.83|3.14|0.00|197.82|278.46|367.29|11.86|0.00|197.82|209.68|-80.64| +2451469|41124|5564|84704|1347655|5806|26404|8|263|2304|3|75.87|105.45|100.17|0.00|300.51|227.61|316.35|3.00|0.00|300.51|303.51|72.90| +2451469|41124|5764|84704|1347655|5806|26404|8|285|2304|48|26.60|42.82|36.82|0.00|1767.36|1276.80|2055.36|106.04|0.00|1767.36|1873.40|490.56| +2451469|41124|5563|84704|1347655|5806|26404|8|216|2304|63|37.17|65.41|43.82|0.00|2760.66|2341.71|4120.83|193.24|0.00|2760.66|2953.90|418.95| +2451469||12907|84704|1347655|5806|||72|2304||92.92||||||5385.60|416.82||||1286.28| +2451469|41124|12523|84704|1347655|5806|26404|8|139|2304|74|28.67|28.95|20.26|419.78|1499.24|2121.58|2142.30|32.38|419.78|1079.46|1111.84|-1042.12| +2451469|41124|4420|84704|1347655|5806|26404|8|248|2304|60|2.48|4.63|1.71|0.00|102.60|148.80|277.80|0.00|0.00|102.60|102.60|-46.20| +2451469|41124|17950|84704|1347655|5806|26404|8|113|2304|40|44.02|44.02|43.13|0.00|1725.20|1760.80|1760.80|69.00|0.00|1725.20|1794.20|-35.60| +2451469|41124|5536|84704|1347655|5806|26404|8|127|2304|15|89.21|99.02|22.77|0.00|341.55|1338.15|1485.30|6.83|0.00|341.55|348.38|-996.60| +2451469|41124|11581|84704|1347655|5806|26404|8|168|2304|17|89.32|176.85|171.54|0.00|2916.18|1518.44|3006.45|204.13|0.00|2916.18|3120.31|1397.74| +2451469|41124|3904|84704|1347655|5806||8||2304|47|43.60|59.73|2.38|0.00|||2807.31||0.00|||| +2451469|41124|10672|84704|1347655|5806|26404|8|4|2304|44|11.24|13.48|12.94|0.00|569.36|494.56|593.12|39.85|0.00|569.36|609.21|74.80| +||15770||||26404|||2304|40|56.81||18.25||730.00||3476.40|||||-1615.40| +2451469|41124|12116|84704|1347655|5806|26404|8|83|2304|62|13.39|21.95|8.99|0.00|557.38|830.18|1360.90|50.16|0.00|557.38|607.54|-272.80| +2451469|41124|15430|84704|1347655|5806|26404|8|141|2304|62|65.30|107.74|24.78|0.00|1536.36|4048.60|6679.88|138.27|0.00|1536.36|1674.63|-2512.24| +2451469|41124|14191|84704|1347655|5806|26404|8|146|2304|29|72.09|77.85|20.24|410.87|586.96|2090.61|2257.65|3.52|410.87|176.09|179.61|-1914.52| +2451469|41124|14302|84704|1347655|5806|26404|8|27|2304|17|24.92|38.87|15.93|0.00|270.81|423.64|660.79|2.70|0.00|270.81|273.51|-152.83| +2450953|56148|2866|16988|193273|1911|36596|8|226|2305|67|78.53|152.34|77.69|0.00|5205.23|5261.51|10206.78|260.26|0.00|5205.23|5465.49|-56.28| +2450953|56148|4165|16988|193273|1911|36596|8|154|2305|33|41.70|62.13|41.00|0.00|1353.00|1376.10|2050.29|54.12|0.00|1353.00|1407.12|-23.10| +2450953|56148|9116|16988|193273|1911|36596|8|105|2305|59|78.58|153.23|30.64|0.00|1807.76|4636.22|9040.57|126.54|0.00|1807.76|1934.30|-2828.46| +2450953|56148|3940|16988|193273|1911|36596|8|105|2305|72|60.98|95.12|61.82|0.00|4451.04|4390.56|6848.64|133.53|0.00|4451.04|4584.57|60.48| +||12337|16988|193273||36596|||2305|||124.06||0.00|8646.90||10172.92||0.00|8646.90|9252.18|3056.96| +2450953|56148|1309|16988|193273|1911|36596|8|263|2305|100|77.19|90.31|60.50|0.00|6050.00|7719.00|9031.00|242.00|0.00|6050.00|6292.00|-1669.00| +2450953|56148|15080|16988|193273|1911|36596|8|126|2305|36|83.22|133.98|104.50|0.00|3762.00|2995.92|4823.28|338.58|0.00|3762.00|4100.58|766.08| +2450953|56148|3781|16988|193273|1911|36596|8|58|2305|12|10.04|18.97|7.96|0.00|95.52|120.48|227.64|8.59|0.00|95.52|104.11|-24.96| +2450953|56148|1783|16988|193273|1911|36596|8|92|2305|60|49.17|88.50|19.47|0.00|1168.20|2950.20|5310.00|93.45|0.00|1168.20|1261.65|-1782.00| +2450953|56148|4628|16988|193273|1911|36596|8|19|2305|46|8.65|13.75|0.27|6.33|12.42|397.90|632.50|0.12|6.33|6.09|6.21|-391.81| +2450953|56148|7772|16988|193273|1911|36596|8|149|2305|90|87.22|109.89|83.51|0.00|7515.90|7849.80|9890.10|375.79|0.00|7515.90|7891.69|-333.90| +2450953|56148|751|16988|193273|1911|36596|8|48|2305|23|29.11|45.70|21.93|0.00|504.39|669.53|1051.10|25.21|0.00|504.39|529.60|-165.14| +2450953|56148|17959|16988|193273|1911|36596|8|231|2305|2|22.35|36.20|24.25|0.00|48.50|44.70|72.40|0.00|0.00|48.50|48.50|3.80| +2451412|51161|12536|2942|1622877|412|20747|4|129|2306|91|13.90|14.31|1.71|0.00|155.61|1264.90|1302.21|14.00|0.00|155.61|169.61|-1109.29| +2451412|51161|14440|2942|1622877|412|20747|4|18|2306|69|52.83|57.58|12.66|0.00|873.54|3645.27|3973.02|0.00|0.00|873.54|873.54|-2771.73| +2451412|51161|7783|2942|1622877|412|20747|4|69|2306|92|20.58|23.66|11.83|0.00|1088.36|1893.36|2176.72|21.76|0.00|1088.36|1110.12|-805.00| +2451412|51161|4447|2942|1622877|412|20747|4|66|2306|37|10.78|15.30|11.78|0.00|435.86|398.86|566.10|8.71|0.00|435.86|444.57|37.00| +2451412|51161|14174|2942|1622877|412|20747|4|76|2306|84|87.02|165.33|13.22|0.00|1110.48|7309.68|13887.72|0.00|0.00|1110.48|1110.48|-6199.20| +2451412|51161|8503|2942|1622877|412|20747|4|247|2306|60|42.81|75.77|7.57|0.00|454.20|2568.60|4546.20|13.62|0.00|454.20|467.82|-2114.40| +2451412|51161|2227|2942|1622877|412|20747|4|157|2306|42|53.82|79.11|43.51|1352.29|1827.42|2260.44|3322.62|4.75|1352.29|475.13|479.88|-1785.31| +2451412|51161|1351|2942|1622877|412|20747|4|150|2306|23|9.34|10.46|4.60|0.00|105.80|214.82|240.58|3.17|0.00|105.80|108.97|-109.02| +2451412|51161|4255|2942|1622877|412|20747|4|144|2306|7|94.91|116.73|28.01|0.00|196.07|664.37|817.11|1.96|0.00|196.07|198.03|-468.30| +2451412|51161|2726|2942|1622877|412|20747|4|54|2306|78|75.63|95.29|22.86|0.00|1783.08|5899.14|7432.62|0.00|0.00|1783.08|1783.08|-4116.06| +2451412|51161|12806|2942|1622877|412|20747|4|253|2306|68|78.34|148.06|45.89|0.00|3120.52|5327.12|10068.08|187.23|0.00|3120.52|3307.75|-2206.60| +2451412|51161|17690|2942|1622877|412|20747|4|108|2306|26|23.66|39.27|32.20|0.00|837.20|615.16|1021.02|41.86|0.00|837.20|879.06|222.04| +2451412|51161|2612|2942|1622877|412|20747|4|15|2306|97|83.69|95.40|40.06|0.00|3885.82|8117.93|9253.80|38.85|0.00|3885.82|3924.67|-4232.11| +2451412|51161|11167|2942|1622877|412|20747|4|95|2306|12|44.37|80.30|78.69|0.00|944.28|532.44|963.60|66.09|0.00|944.28|1010.37|411.84| +2452555|36215|5743|1373|1193385|2197|11362|10|135|2307|74|21.92|21.92|2.41|0.00|178.34|1622.08|1622.08|16.05|0.00|178.34|194.39|-1443.74| +2452555|36215|13899|1373|1193385|2197|11362|10|262|2307|14|95.11|170.24|149.81|0.00|2097.34|1331.54|2383.36|146.81|0.00|2097.34|2244.15|765.80| +2452555|36215|8322|1373|1193385|2197|11362|10|129|2307|24|38.61|52.89|29.61|0.00|710.64|926.64|1269.36|56.85|0.00|710.64|767.49|-216.00| +2452555|36215|435|1373|1193385|2197|11362|10|255|2307|42|34.55|41.46|28.19|0.00|1183.98|1451.10|1741.32|23.67|0.00|1183.98|1207.65|-267.12| +2452555|36215|14941|1373|1193385|2197|11362|10|233|2307|51|25.56|39.36|32.66|0.00|1665.66|1303.56|2007.36|149.90|0.00|1665.66|1815.56|362.10| +2452555|36215|11637||1193385|||||2307|100|56.32||32.20|0.00|3220.00||||0.00||3509.80|-2412.00| +2452555|36215|2502|1373|1193385|2197|11362|10|114|2307|76|36.53|65.75|63.12|0.00|4797.12|2776.28|4997.00|383.76|0.00|4797.12|5180.88|2020.84| +|36215|14485||||||205|2307|14||135.15||0.00|||||0.00|||-1062.32| +2452555|36215|5899|1373|1193385|2197|11362|10|236|2307|54|73.15|111.91|45.88|0.00|2477.52|3950.10|6043.14|222.97|0.00|2477.52|2700.49|-1472.58| +2452555|36215|8989|1373|1193385|2197|11362|10|12|2307|100|89.94|158.29|118.71|0.00|11871.00|8994.00|15829.00|118.71|0.00|11871.00|11989.71|2877.00| +2452555|36215|1062|1373|1193385|2197|11362|10|141|2307|92|89.38|134.96|63.43|0.00|5835.56|8222.96|12416.32|466.84|0.00|5835.56|6302.40|-2387.40| +2452555|36215|5640|1373|1193385|||||2307||6.64|8.43|6.82|0.00|||632.25|46.03|0.00|511.50||13.50| +2451911|44393|7161|3322|696677|6832|46587|4|67|2308|1|83.48|95.16|67.56|0.00|67.56|83.48|95.16|2.70|0.00|67.56|70.26|-15.92| +2451911|44393|2063|3322|696677|6832|46587|4|252|2308|12|4.62|5.95|4.93|0.00|59.16|55.44|71.40|0.00|0.00|59.16|59.16|3.72| +2451911|44393|7897|3322|696677|6832|46587|4|122|2308|16|94.40|120.83|95.45|0.00|1527.20|1510.40|1933.28|91.63|0.00|1527.20|1618.83|16.80| +2451911|44393|17091|3322|696677|6832|46587|4|214|2308|24|89.10|177.30|1.77|32.70|42.48|2138.40|4255.20|0.39|32.70|9.78|10.17|-2128.62| +2451911|44393|3313|3322|696677|6832|46587|4|191|2308|69|66.30|130.61|18.28|0.00|1261.32|4574.70|9012.09|25.22|0.00|1261.32|1286.54|-3313.38| +2451911|44393|919|3322|696677|6832|46587|4|127|2308|63|70.71|112.42|39.34|0.00|2478.42|4454.73|7082.46|74.35|0.00|2478.42|2552.77|-1976.31| +2451911|44393|12019|3322|696677|6832|46587|4|64|2308|67|13.64|20.18|9.48|0.00|635.16|913.88|1352.06|38.10|0.00|635.16|673.26|-278.72| +2451911|44393|12235|3322|696677|6832|46587|4|78|2308|50|10.85|17.90|14.67|0.00|733.50|542.50|895.00|22.00|0.00|733.50|755.50|191.00| +2451911||17945|3322||||4||2308||55.83|93.23|40.08|511.42|1162.32||2703.67|13.01|511.42|||-968.17| +2451911|44393|10851|3322|696677|6832|46587|4|228|2308|67|80.12|137.00|137.00|0.00|9179.00|5368.04|9179.00|458.95|0.00|9179.00|9637.95|3810.96| +2451911|44393|15929|3322|696677|6832|46587|4|227|2308|84|71.02|72.44|50.70|0.00|4258.80|5965.68|6084.96|383.29|0.00|4258.80|4642.09|-1706.88| +2451911|44393|10647|3322|696677|6832|46587|4|260|2308|37|99.44|102.42|15.36|0.00|568.32|3679.28|3789.54|28.41|0.00|568.32|596.73|-3110.96| +2451911||1803||696677|6832|46587|4|273|2308|58|24.26|27.17|0.27|11.90|15.66|||0.18|11.90|||| +2451911|44393|10281|3322|696677|6832|46587|4|258|2308|48|83.76|92.13|39.61|0.00|1901.28|4020.48|4422.24|95.06|0.00|1901.28|1996.34|-2119.20| +2451911|44393|1121|3322|696677|6832|46587|4|152|2308|51|37.14|40.11|38.10|0.00|1943.10|1894.14|2045.61|38.86|0.00|1943.10|1981.96|48.96| +2451911|44393|16165|3322|696677|6832|46587|4|202|2308|2|33.18|53.41|28.30|0.00|56.60|66.36|106.82|5.09|0.00|56.60|61.69|-9.76| +2451122|35025|848|11391|1912181|2571|1761|1|138|2309|20|89.12|116.74|33.85|0.00|677.00|1782.40|2334.80|6.77|0.00|677.00|683.77|-1105.40| +2451122|35025|16555|11391|1912181|2571|1761|1|217|2309|89|68.21|90.03|76.52|0.00|6810.28|6070.69|8012.67|204.30|0.00|6810.28|7014.58|739.59| +2451122|35025|9692|11391|1912181|2571|1761|1|101|2309|20|67.04|104.58|35.55|0.00|711.00|1340.80|2091.60|28.44|0.00|711.00|739.44|-629.80| +2451122|35025|10315|11391|1912181|2571|1761|1|159|2309|36|56.29|92.31|24.00|0.00|864.00|2026.44|3323.16|25.92|0.00|864.00|889.92|-1162.44| +2451122|35025|15664|11391|1912181|2571|1761|1|119|2309|100|60.03|90.04|31.51|0.00|3151.00|6003.00|9004.00|126.04|0.00|3151.00|3277.04|-2852.00| +2451122|35025|5734|11391|1912181|2571|1761|1|134|2309|22|3.25|4.51|3.78|0.00|83.16|71.50|99.22|7.48|0.00|83.16|90.64|11.66| +2451122|35025|17485|11391|1912181|2571|1761|1|55|2309|84|41.30|53.69|34.36|0.00|2886.24|3469.20|4509.96|173.17|0.00|2886.24|3059.41|-582.96| +2451122|35025|3871|11391|1912181|2571|1761|1|198|2309|33|19.00|30.78|1.23|0.00|40.59|627.00|1015.74|2.43|0.00|40.59|43.02|-586.41| +2451122|35025|6412|11391|1912181|2571|1761|1|109|2309|40|39.96|75.92|34.92|0.00|1396.80|1598.40|3036.80|27.93|0.00|1396.80|1424.73|-201.60| +2451122|35025|5848|11391|1912181|2571|1761|1|119|2309|11|82.85|135.87|93.75|0.00|1031.25|911.35|1494.57|61.87|0.00|1031.25|1093.12|119.90| +2451122|35025|11468|11391|1912181|2571|1761|1|198|2309|80|82.35|153.17|102.62|0.00|8209.60|6588.00|12253.60|656.76|0.00|8209.60|8866.36|1621.60| +2451122|35025|1784|11391|1912181|2571|1761|1|76|2309|32|19.08|23.85|17.41|490.26|557.12|610.56|763.20|3.34|490.26|66.86|70.20|-543.70| +2451122|35025|9818|11391|1912181|2571|1761|1|250|2309|42|8.83|13.95|3.76|0.00|157.92|370.86|585.90|12.63|0.00|157.92|170.55|-212.94| +2451122|35025|10982|11391|1912181|2571|1761|1|31|2309|53|73.84|138.08|4.14|57.04|219.42|3913.52|7318.24|1.62|57.04|162.38|164.00|-3751.14| +2451122|35025|4861|11391|1912181|2571|1761|1|62|2309|46|25.94|35.79|8.23|0.00|378.58|1193.24|1646.34|15.14|0.00|378.58|393.72|-814.66| +2452455|68579|15445|85154|1585403|852|21101|7|297|2310|4|80.60|151.52|7.57|16.35|30.28|322.40|606.08|0.41|16.35|13.93|14.34|-308.47| +2452455|68579|4981|85154|1585403|852|21101|7|13|2310|98|58.73|64.01|58.24|0.00|5707.52|5755.54|6272.98|57.07|0.00|5707.52|5764.59|-48.02| +2452455|68579|17376|85154|1585403|852|21101|7|91|2310|29|66.20|70.17|42.80|0.00|1241.20|1919.80|2034.93|62.06|0.00|1241.20|1303.26|-678.60| +2452455|68579|16509|85154|1585403|852|21101|7|146|2310|87|22.59|31.17|8.41|329.25|731.67|1965.33|2711.79|0.00|329.25|402.42|402.42|-1562.91| +|68579|11827|||||7||2310|||83.25|51.61|0.00||1530.00|2830.50||0.00|||| +2452455|68579|8895|85154|1585403|852|21101|7|281|2310|57|39.58|51.84|3.62|0.00|206.34|2256.06|2954.88|6.19|0.00|206.34|212.53|-2049.72| +2452455|68579|10353|85154|1585403|852|21101|7|79|2310|36||16.58|12.76|||574.20||||459.36|486.92|| +2452455|68579|6231|85154|1585403|852|21101|7|288|2310|68|83.31|108.30|42.23|0.00|2871.64|5665.08|7364.40|57.43|0.00|2871.64|2929.07|-2793.44| +|68579|1867|85154|1585403||21101|||2310||75.83|||325.59|664.48|1213.28|2013.92|30.50|325.59|||| +2452455|68579|12258|85154|1585403|852|21101|7|28|2310|86|22.56|37.44|10.10|0.00|868.60|1940.16|3219.84|52.11|0.00|868.60|920.71|-1071.56| +2452455|68579|6276|85154|1585403|852|21101|7|23|2310|18|40.45|79.68|10.35|0.00|186.30|728.10|1434.24|16.76|0.00|186.30|203.06|-541.80| +2452455|68579|12589|85154|1585403|852|21101|7|133|2310|71|87.28|157.10|128.82|0.00|9146.22|6196.88|11154.10|274.38|0.00|9146.22|9420.60|2949.34| +2452455|68579|5307|85154|1585403|852|21101|7|202|2310|82|70.92|79.43|22.24|1823.68|1823.68|5815.44|6513.26|0.00|1823.68|0.00|0.00|-5815.44| +2452455|68579|4836|85154|1585403|852|21101|7|84|2310|67|98.90|192.85|113.78|3201.76|7623.26|6626.30|12920.95|132.64|3201.76|4421.50|4554.14|-2204.80| +|68579|14640|85154||852||7|83|2310|43||14.34|||357.33|331.53|616.62|0.00||357.33|357.33|25.80| +2452455|68579|14184|85154|1585403|852|21101|7|229|2310|83|19.15|22.02|8.36|0.00|693.88|1589.45|1827.66|48.57|0.00|693.88|742.45|-895.57| +2451199|53867|9772|37876|718937|5598|9106|4|9|2311|23|34.28|60.67|51.56|0.00|1185.88|788.44|1395.41|71.15|0.00|1185.88|1257.03|397.44| +|53867|7630|37876||5598|||253|2311||||18.91||1607.35||2088.45|||1607.35|1607.35|277.10| +2451199|53867|10462|37876|718937|5598|9106|4|163|2311|39|23.31|35.66|6.77|0.00|264.03|909.09|1390.74|2.64|0.00|264.03|266.67|-645.06| +2451199|53867|5056|37876|718937|5598|9106|4|239|2311|9|68.38|115.56|8.08|0.00|72.72|615.42|1040.04|2.18|0.00|72.72|74.90|-542.70| +2451199|53867|9152|37876|718937|5598|9106|4|62|2311|63|16.04|26.14|5.48|0.00|345.24|1010.52|1646.82|24.16|0.00|345.24|369.40|-665.28| +2451199|53867|3640|37876|718937|5598|9106|4|69|2311|5|68.09|121.88|78.00|0.00|390.00|340.45|609.40|19.50|0.00|390.00|409.50|49.55| +2451199|53867|13816|37876|718937|5598|9106|4|242|2311|24|48.47|60.58|41.80|0.00|1003.20|1163.28|1453.92|0.00|0.00|1003.20|1003.20|-160.08| +2451199|53867|16180|37876|718937|5598|9106|4|39|2311|75|28.77|42.00|16.80|0.00|1260.00|2157.75|3150.00|100.80|0.00|1260.00|1360.80|-897.75| +2451199|53867|15199|37876|718937|5598|9106|4|192|2311|96|21.98|31.21|15.60|0.00|1497.60|2110.08|2996.16|74.88|0.00|1497.60|1572.48|-612.48| +2451199|53867|16262|37876|718937|5598|9106|4|271|2311|5|29.29|42.47|39.92|165.66|199.60|146.45|212.35|3.05|165.66|33.94|36.99|-112.51| +2451032|46849|10172|50639|1393132|4872|30212|7|72|2312|54|68.50|123.30|2.46|0.00|132.84|3699.00|6658.20|2.65|0.00|132.84|135.49|-3566.16| +2451032|46849|391|||4872|30212|7||2312|92||4.36|3.00|0.00|276.00||||0.00|||-50.60| +2451032|46849|4640|50639|1393132|4872|30212|7|106|2312|28|57.09|82.20|54.25|0.00|1519.00|1598.52|2301.60|75.95|0.00|1519.00|1594.95|-79.52| +2451032|46849|17932|50639|1393132|4872|30212|7|209|2312|40|25.93|40.71|10.99|0.00|439.60|1037.20|1628.40|39.56|0.00|439.60|479.16|-597.60| +2451032|46849|17846|50639|1393132|4872|30212|7|153|2312|61|65.89|79.72|76.53|0.00|4668.33|4019.29|4862.92|373.46|0.00|4668.33|5041.79|649.04| +2451032|46849|13634|50639|1393132|4872|30212|7|59|2312|34|89.25|174.03|121.82|0.00|4141.88|3034.50|5917.02|289.93|0.00|4141.88|4431.81|1107.38| +2451032|46849|379|50639|1393132|4872|30212|7|111|2312|33|53.72|72.52|15.22|0.00|502.26|1772.76|2393.16|30.13|0.00|502.26|532.39|-1270.50| +2451032|46849|8228|50639|1393132|4872|30212|7|10|2312|40|90.18|94.68|84.26|0.00|3370.40|3607.20|3787.20|67.40|0.00|3370.40|3437.80|-236.80| +2451032|46849|5296|50639|1393132|4872|30212|7|119|2312|85|71.14|78.25|29.73|0.00|2527.05|6046.90|6651.25|176.89|0.00|2527.05|2703.94|-3519.85| +2451032|46849|2884|50639|1393132|4872|30212|7|23|2312|4|6.44|11.39|1.02|0.00|4.08|25.76|45.56|0.04|0.00|4.08|4.12|-21.68| +2451032|46849|11671|50639|1393132|4872|30212|7|122|2312|60|68.53|69.21|40.83|0.00|2449.80|4111.80|4152.60|0.00|0.00|2449.80|2449.80|-1662.00| +||8380|50639|||30212|7||2312||38.13||4.20||147.00||2455.25||||154.35|-1187.55| +2451032|46849|12214|50639|1393132|4872|30212|7|298|2312|44|92.36|176.40|171.10|0.00|7528.40|4063.84|7761.60|75.28|0.00|7528.40|7603.68|3464.56| +2451032|46849|13064|50639|1393132|4872|30212|7|269|2312|36|56.63|62.29|34.25|0.00|1233.00|2038.68|2242.44|73.98|0.00|1233.00|1306.98|-805.68| +2451032|46849|15308|50639|1393132|4872|30212|7|248|2312|98|64.42|93.40|30.82|0.00|3020.36|6313.16|9153.20|30.20|0.00|3020.36|3050.56|-3292.80| +2451814|37631|181|41646|1079566|3886|17006|2|276|2313|57|84.73|110.99|93.23|0.00|5314.11|4829.61|6326.43|425.12|0.00|5314.11|5739.23|484.50| +2451814|37631|13045|41646|1079566|3886|17006|2|265|2313|68|25.10|46.43|25.07|0.00|1704.76|1706.80|3157.24|51.14|0.00|1704.76|1755.90|-2.04| +2451814|37631|10154|41646|1079566|3886|17006|2|189|2313|67|19.90|29.65|26.68|0.00|1787.56|1333.30|1986.55|0.00|0.00|1787.56|1787.56|454.26| +2451814|37631|15218|41646|1079566|3886|17006|2|226|2313|85|57.23|77.83|71.60|0.00|6086.00|4864.55|6615.55|304.30|0.00|6086.00|6390.30|1221.45| +2451814|37631|7393|41646|1079566|3886|17006|2|293|2313|72|65.91|103.47|81.74|0.00|5885.28|4745.52|7449.84|353.11|0.00|5885.28|6238.39|1139.76| +2451814|37631|6445|41646|1079566|3886|17006|2|81|2313|81|73.94|114.60|29.79|0.00|2412.99|5989.14|9282.60|96.51|0.00|2412.99|2509.50|-3576.15| +|37631|15230||||17006|2|269|2313|||59.48|51.74|0.00|||||0.00|1448.72|1550.13|| +2451814|37631|9853|41646|1079566|3886|17006|2|27|2313|94|59.95|97.71|36.15|0.00|3398.10|5635.30|9184.74|169.90|0.00|3398.10|3568.00|-2237.20| +2451814|37631|8267|41646|1079566|3886|17006|2|30|2313|75|87.37|140.66|85.80|0.00|6435.00|6552.75|10549.50|514.80|0.00|6435.00|6949.80|-117.75| +2451814|37631|7844|41646|1079566|3886|17006|2|97|2313|5|22.09|35.12|34.41|0.00|172.05|110.45|175.60|10.32|0.00|172.05|182.37|61.60| +2451814|37631|12623|41646|1079566|3886|17006|2|293|2313|98|2.14|4.04|2.26|0.00|221.48|209.72|395.92|15.50|0.00|221.48|236.98|11.76| +2451814|37631|3284|41646|1079566|3886|17006|2|204|2313|96|96.49|132.19|62.12|0.00|5963.52|9263.04|12690.24|0.00|0.00|5963.52|5963.52|-3299.52| +2451814|37631|16256|41646|1079566|3886|17006|2|162|2313|95|76.42|132.97|74.46|0.00|7073.70|7259.90|12632.15|565.89|0.00|7073.70|7639.59|-186.20| +2451814|37631|10579|41646|1079566|3886|17006|2|208|2313|62|55.86|97.19|0.97|0.00|60.14|3463.32|6025.78|1.80|0.00|60.14|61.94|-3403.18| +2451814|37631|14401|41646|1079566|3886|17006|2|101|2313|71|62.68|96.52|71.42|2890.36|5070.82|4450.28|6852.92|21.80|2890.36|2180.46|2202.26|-2269.82| +2451814|37631|16621|41646|1079566|3886|17006|2|102|2313|44|86.34|92.38|17.55|0.00|772.20|3798.96|4064.72|7.72|0.00|772.20|779.92|-3026.76| +2452220|47511|1675|10793|1397197|3601|24005|2|277|2314|93|6.95|7.22|1.94|135.31|180.42|646.35|671.46|3.60|135.31|45.11|48.71|-601.24| +2452220|47511|5629|10793|1397197|3601|24005|2|165|2314|32|28.30|32.26|12.90|78.43|412.80|905.60|1032.32|30.09|78.43|334.37|364.46|-571.23| +2452220|47511|16951|10793|1397197|3601|24005|2|154|2314|97|53.39|79.55|61.25|831.77|5941.25|5178.83|7716.35|0.00|831.77|5109.48|5109.48|-69.35| +2452220|47511|8557|10793|1397197|3601|24005|2|239|2314|35|91.90|147.04|48.52|1562.34|1698.20|3216.50|5146.40|5.43|1562.34|135.86|141.29|-3080.64| +2452220|47511|527|10793|1397197|3601|24005|2|275|2314|61|42.75|76.95|28.47|0.00|1736.67|2607.75|4693.95|52.10|0.00|1736.67|1788.77|-871.08| +2452220|47511|11863|10793|1397197|3601|24005|2|12|2314|65|17.75|32.48|1.29|0.00|83.85|1153.75|2111.20|4.19|0.00|83.85|88.04|-1069.90| +2452220|47511|15825|10793|1397197|3601|24005|2|97|2314|60|16.40|30.99|26.34|0.00|1580.40|984.00|1859.40|63.21|0.00|1580.40|1643.61|596.40| +2452220|47511|15819|10793|1397197|3601|24005|2|130|2314|51|92.35|106.20|54.16|0.00|2762.16|4709.85|5416.20|248.59|0.00|2762.16|3010.75|-1947.69| +2452220|47511|9987|10793|1397197|3601|24005|2|17|2314|35|54.05|54.59|39.85|0.00|1394.75|1891.75|1910.65|0.00|0.00|1394.75|1394.75|-497.00| +2451544|46710|9512|18619|760538|4504|8423|2|201|2315|23|64.74|126.24|54.28|0.00|1248.44|1489.02|2903.52|99.87|0.00|1248.44|1348.31|-240.58| +2451544|46710|1258|18619|760538|4504|8423|2|194|2315|1|44.33|62.94|52.24|0.00|52.24|44.33|62.94|1.04|0.00|52.24|53.28|7.91| +2451544|46710|7399|18619|760538|4504|8423|2|275|2315|83|64.72|80.90|75.23|0.00|6244.09|5371.76|6714.70|249.76|0.00|6244.09|6493.85|872.33| +2451544|46710|532|18619|760538|4504|8423|2|184|2315|88|2.59|4.76|2.57|0.00|226.16|227.92|418.88|20.35|0.00|226.16|246.51|-1.76| +2451544|46710|15787|18619|760538|4504|8423|2|16|2315|99|85.58|103.55|64.20|0.00|6355.80|8472.42|10251.45|444.90|0.00|6355.80|6800.70|-2116.62| +2451544|46710|124|18619|760538|4504|8423|2|214|2315|68|19.20|36.48|29.91|0.00|2033.88|1305.60|2480.64|61.01|0.00|2033.88|2094.89|728.28| +2451544|46710|9404|18619|760538|4504|8423|2|183|2315|71|48.51|68.88|14.46|0.00|1026.66|3444.21|4890.48|71.86|0.00|1026.66|1098.52|-2417.55| +2451544|46710|2918|18619|760538|4504|8423|2|180|2315|94|9.21|14.09|1.69|0.00|158.86|865.74|1324.46|14.29|0.00|158.86|173.15|-706.88| +2451544|46710|16309|18619|760538|4504|8423|2|45|2315|56|10.14|10.95|3.50|0.00|196.00|567.84|613.20|7.84|0.00|196.00|203.84|-371.84| +2451544|46710|13429|18619|760538|4504|8423|2|267|2315|91|99.50|174.12|34.82|0.00|3168.62|9054.50|15844.92|221.80|0.00|3168.62|3390.42|-5885.88| +2451544|46710|9103|18619|760538|4504|8423|2|227|2315|50|56.10|58.34|36.75|1120.87|1837.50|2805.00|2917.00|7.16|1120.87|716.63|723.79|-2088.37| +2451544|46710|17092|18619|760538|4504|8423|2|27|2315|65|50.71|84.17|24.40|63.44|1586.00|3296.15|5471.05|30.45|63.44|1522.56|1553.01|-1773.59| +2451544|46710|14596|18619|760538|4504|8423|2|139|2315|49|8.36|13.54|12.99|0.00|636.51|409.64|663.46|50.92|0.00|636.51|687.43|226.87| +2451800|33251|14327|84516|1441016|716|38369|4|204|2316|15|94.22|178.07|147.79|886.74|2216.85|1413.30|2671.05|0.00|886.74|1330.11|1330.11|-83.19| +2451800|33251|15947|84516|1441016|716|38369|4|273|2316|81|92.81|153.13|120.97|7838.85|9798.57|7517.61|12403.53|97.98|7838.85|1959.72|2057.70|-5557.89| +2451800|33251|17311|84516|1441016|716|38369|4|130|2316|39|40.68|51.66|47.52|0.00|1853.28|1586.52|2014.74|148.26|0.00|1853.28|2001.54|266.76| +2451800|33251|8216|84516|1441016|716|38369|4|97|2316|41|66.83|123.63|122.39|0.00|5017.99|2740.03|5068.83|100.35|0.00|5017.99|5118.34|2277.96| +2451800|33251|9871|84516|1441016|716|38369|4|215|2316|26|50.12|63.15|22.10|379.23|574.60|1303.12|1641.90|13.67|379.23|195.37|209.04|-1107.75| +2451800|33251|3661|84516||716|||201|2316|45||184.94|||1248.30|4450.50||24.96||1248.30|1273.26|| +2451800|33251|5114|84516|1441016|716|38369|4|246|2316|48|2.64|2.69|1.82|0.00|87.36|126.72|129.12|1.74|0.00|87.36|89.10|-39.36| +2451800|33251|5897|84516|1441016|716|38369|4|48|2316|39|82.85|104.39|21.92|76.93|854.88|3231.15|4071.21|62.23|76.93|777.95|840.18|-2453.20| +2451800|33251|6215|84516|1441016|716|38369|4|9|2316|78|33.41|55.46|9.42|0.00|734.76|2605.98|4325.88|51.43|0.00|734.76|786.19|-1871.22| +2451800|33251|13219|84516|1441016|716|38369|4|203|2316|20|73.16|121.44|24.28|0.00|485.60|1463.20|2428.80|19.42|0.00|485.60|505.02|-977.60| +2451800|33251|9229|84516|1441016|716|38369|4|283|2316|84|49.08|62.82|21.98|0.00|1846.32|4122.72|5276.88|147.70|0.00|1846.32|1994.02|-2276.40| +2452272|51457|6991|10619|1659406|1346||||2317|64|31.17|||0.00|474.24|1994.88|3390.72||0.00|474.24||-1520.64| +2452272|51457|2319|10619|1659406|1346|33733|2|256|2317|85|46.18|51.72|13.96|0.00|1186.60|3925.30|4396.20|0.00|0.00|1186.60|1186.60|-2738.70| +2452272|51457|1329|10619|1659406|1346|33733|2|149|2317|51|72.76|117.87|70.72|0.00|3606.72|3710.76|6011.37|36.06|0.00|3606.72|3642.78|-104.04| +2452272|51457|7877|10619|1659406|1346|33733|2|217|2317|50|76.89|105.33|105.33|895.30|5266.50|3844.50|5266.50|262.27|895.30|4371.20|4633.47|526.70| +2452272|51457|7159|10619|1659406|1346|33733|2|289|2317|42|28.66|28.94|28.07|0.00|1178.94|1203.72|1215.48|11.78|0.00|1178.94|1190.72|-24.78| +2452272|51457|14013|10619|1659406|1346|33733|2|211|2317|46|13.70|15.07|7.98|297.33|367.08|630.20|693.22|0.69|297.33|69.75|70.44|-560.45| +2452272|51457|13507|10619|1659406|1346|33733|2|241|2317|8|87.16|109.82|56.00|0.00|448.00|697.28|878.56|17.92|0.00|448.00|465.92|-249.28| +2452272|51457|7017|10619|1659406|1346|33733|2|1|2317|4|73.09|77.47|3.09|2.22|12.36|292.36|309.88|0.40|2.22|10.14|10.54|-282.22| +2452272|51457|2389|10619|1659406|1346|33733|2|51|2317|19|63.69|83.43|51.72|0.00|982.68|1210.11|1585.17|29.48|0.00|982.68|1012.16|-227.43| +2452272|51457|1477|10619|1659406|1346|33733|2|220|2317|74|64.97|95.50|11.46|0.00|848.04|4807.78|7067.00|33.92|0.00|848.04|881.96|-3959.74| +2452272|51457|11433|10619|1659406|1346|33733|2|78|2317|95|8.50|15.81|10.11|0.00|960.45|807.50|1501.95|38.41|0.00|960.45|998.86|152.95| +2452272|51457|10567|10619|1659406|1346|33733|2|212|2317|15|65.35|124.81|99.84|0.00|1497.60|980.25|1872.15|44.92|0.00|1497.60|1542.52|517.35| +2452272|51457|6741|10619|1659406|1346|33733|2|126|2317|91|21.62|41.72|20.86|0.00|1898.26|1967.42|3796.52|75.93|0.00|1898.26|1974.19|-69.16| +2452272|51457|13481|10619|1659406|1346|33733|2|182|2317|15|2.27|2.56|1.15|0.00|17.25|34.05|38.40|1.38|0.00|17.25|18.63|-16.80| +2451909|43566|5450|92119|1728669|419|47016|10|16|2318|16|20.10|32.16|14.47|41.67|231.52|321.60|514.56|11.39|41.67|189.85|201.24|-131.75| +2451909|43566|5453|92119|1728669|419|47016|10|294|2318|36|76.16|89.10|66.82|0.00|2405.52|2741.76|3207.60|120.27|0.00|2405.52|2525.79|-336.24| +2451909|43566|9671|92119|1728669|419|47016|10|205|2318|61|63.00|78.75|29.92|0.00|1825.12|3843.00|4803.75|0.00|0.00|1825.12|1825.12|-2017.88| +2451909|43566|16913|92119|1728669|419|47016|10|105|2318|6|98.56|154.73|57.25|0.00|343.50|591.36|928.38|17.17|0.00|343.50|360.67|-247.86| +2451909|43566|9689|92119|1728669|419|47016|10|93|2318|15|46.86|73.10|68.71|0.00|1030.65|702.90|1096.50|0.00|0.00|1030.65|1030.65|327.75| +2451909|43566|17717|92119|1728669|419|47016|10|59|2318|65|55.68|62.36|39.28|0.00|2553.20|3619.20|4053.40|51.06|0.00|2553.20|2604.26|-1066.00| +2451909|43566|1037|92119|1728669|419|47016|10|271|2318|50|25.34|48.90|16.62|0.00|831.00|1267.00|2445.00|24.93|0.00|831.00|855.93|-436.00| +2451909|43566|5185|92119|1728669|419|47016|10|160|2318|37|37.36|59.40|48.11|1281.65|1780.07|1382.32|2197.80|24.92|1281.65|498.42|523.34|-883.90| +2451909|43566|635|92119|1728669|419|47016|10|212|2318|63|5.69|6.71|3.75|0.00|236.25|358.47|422.73|2.36|0.00|236.25|238.61|-122.22| +2451909|43566|557|92119|1728669|419|47016|10|63|2318|53|18.38|24.81|9.42|0.00|499.26|974.14|1314.93|29.95|0.00|499.26|529.21|-474.88| +2451909|43566|12941|92119|1728669|419|47016|10|70|2318|89|13.00|18.98|2.84|0.00|252.76|1157.00|1689.22|7.58|0.00|252.76|260.34|-904.24| +2451909|43566|12020|92119|1728669|419|47016|10|48|2318|62|23.53|28.70|22.09|0.00|1369.58|1458.86|1779.40|68.47|0.00|1369.58|1438.05|-89.28| +2451909||9055|||419||10|63|2318|36||78.44|18.82||||2823.84|0.00||677.52|677.52|-874.08| +2451909|43566|7271|92119|1728669|419|47016|10|22|2318|13|10.28|12.85|1.02|0.00|13.26|133.64|167.05|0.79|0.00|13.26|14.05|-120.38| +2451909|43566|11377|92119|1728669|419|47016|10|9|2318|33|7.72|8.95|7.42|0.00|244.86|254.76|295.35|14.69|0.00|244.86|259.55|-9.90| +2451909|43566|13381|92119||||10|265|2318||||5.15||||4074.03|4.06||406.85||| +2452032|32744|16487|75966|1005709|6652|47595|7|169|2319|77|90.31|111.98|42.55|0.00|3276.35|6953.87|8622.46|65.52|0.00|3276.35|3341.87|-3677.52| +2452032|32744|2987|75966|1005709|6652|47595|7|40|2319|59|49.77|70.67|69.25|2492.30|4085.75|2936.43|4169.53|15.93|2492.30|1593.45|1609.38|-1342.98| +2452032|32744|14553|75966|1005709|6652|47595|7|247|2319|76|6.32|10.55|9.49|425.53|721.24|480.32|801.80|26.61|425.53|295.71|322.32|-184.61| +2452032|32744|7879|75966|1005709|6652|47595|7|51|2319|65|49.77|91.57|30.21|0.00|1963.65|3235.05|5952.05|176.72|0.00|1963.65|2140.37|-1271.40| +2452032|32744|4159|75966|1005709|6652|47595|7|273|2319|38|14.31|22.46|0.67|0.00|25.46|543.78|853.48|0.25|0.00|25.46|25.71|-518.32| +||15925||1005709|||7||2319|||24.24||0.00|||||0.00||845.20|| +2452032|32744|8923|75966|1005709|6652|47595|7|9|2319|73|92.74|170.64|167.22|8300.80|12207.06|6770.02|12456.72|0.00|8300.80|3906.26|3906.26|-2863.76| +|32744|11021|75966||6652|||203|2319|88||92.63|69.47|||||||6113.36||| +2452032|32744|8815|75966|1005709|6652|47595|7|70|2319|95|70.08|96.71|81.23|0.00|7716.85|6657.60|9187.45|0.00|0.00|7716.85|7716.85|1059.25| +2452032|32744|13083|75966|1005709|6652|47595|7|297|2319|12|46.59|68.48|43.82|0.00|525.84|559.08|821.76|0.00|0.00|525.84|525.84|-33.24| +2452032|32744|5459|75966|1005709|6652|47595|7|120|2319|56|97.21|186.64|162.37|0.00|9092.72|5443.76|10451.84|90.92|0.00|9092.72|9183.64|3648.96| +2452032|32744|5221|75966|1005709||47595||278|2319|4|82.33|111.96||0.00|||447.84||0.00|165.68|177.27|| +2452032|32744|2021|75966|1005709|6652|47595|7|69|2319|34|27.07|46.56|13.03|0.00|443.02|920.38|1583.04|31.01|0.00|443.02|474.03|-477.36| +2452032|32744|1557|75966|1005709|6652|47595|7|143|2319|64|88.47|116.78|65.39|0.00|4184.96|5662.08|7473.92|251.09|0.00|4184.96|4436.05|-1477.12| +2452032|32744|17233|||6652|||36|2319||44.71|||||2637.89||0.00||3879.25||| +2452032|32744|2131|75966|1005709|6652|47595|7|68|2319|68|33.51|51.94|11.42|0.00|776.56|2278.68|3531.92|62.12|0.00|776.56|838.68|-1502.12| +2451284|51464|7576|||1963|8533|2|95|2320||89.79|140.97|12.68|467.25||6015.93|||467.25||393.77|| +2451284|51464|14996|12470|630240|1963|8533|2|6|2320|44|28.22|55.02|37.96|0.00|1670.24|1241.68|2420.88|66.80|0.00|1670.24|1737.04|428.56| +2451284|51464|901|12470|630240|1963|8533|2|209|2320|2|42.74|68.81|39.22|0.00|78.44|85.48|137.62|0.78|0.00|78.44|79.22|-7.04| +2451284|51464|12706|12470|630240|1963|8533|2|268|2320|98|16.73|27.43|0.00|0.00|0.00|1639.54|2688.14|0.00|0.00|0.00|0.00|-1639.54| +2451284|51464|5965|12470|630240|1963|8533|2|225|2320|93|75.48|150.96|120.76|0.00|11230.68|7019.64|14039.28|449.22|0.00|11230.68|11679.90|4211.04| +2451284|51464|8611|12470|630240|1963|8533|2|296|2320|37|26.38|39.30|22.40|0.00|828.80|976.06|1454.10|74.59|0.00|828.80|903.39|-147.26| +2451284|51464|3856|12470|630240|1963|8533|2|144|2320|29|47.58|66.61|57.95|0.00|1680.55|1379.82|1931.69|16.80|0.00|1680.55|1697.35|300.73| +2451284|51464|10507|12470|630240|1963|8533|2|272|2320|76|80.00|86.40|27.64|0.00|2100.64|6080.00|6566.40|168.05|0.00|2100.64|2268.69|-3979.36| +2451284|51464|4471|12470|630240|1963|8533|2|122|2320|19|20.45|23.31|6.06|0.00|115.14|388.55|442.89|8.05|0.00|115.14|123.19|-273.41| +2451284|51464|1972|12470|630240|1963|8533|2|217|2320|72|76.10|88.27|30.89|0.00|2224.08|5479.20|6355.44|177.92|0.00|2224.08|2402.00|-3255.12| +2451284|51464|10939|12470|630240|1963|8533|2|141|2320|45|65.67|79.46|22.24|0.00|1000.80|2955.15|3575.70|30.02|0.00|1000.80|1030.82|-1954.35| +2451284|51464|817|12470|630240|1963|8533|2|88|2320|54|35.66|63.11|56.79|0.00|3066.66|1925.64|3407.94|275.99|0.00|3066.66|3342.65|1141.02| +2451284|51464|14444|12470|630240|1963|8533|2|150|2320|28|95.76|98.63|74.95|0.00|2098.60|2681.28|2761.64|41.97|0.00|2098.60|2140.57|-582.68| +2452261|69492|6567|79346|1591205|3511|8170|2|17|2321|77|62.13|86.98|81.76|0.00|6295.52|4784.01|6697.46|377.73|0.00|6295.52|6673.25|1511.51| +2452261|69492|12613|79346|1591205|3511|8170|2|26|2321|79|93.15|161.14|66.06|0.00|5218.74|7358.85|12730.06|52.18|0.00|5218.74|5270.92|-2140.11| +2452261|69492|5049|79346|1591205|3511|8170|2|124|2321|70|47.29|60.53|59.92|0.00|4194.40|3310.30|4237.10|41.94|0.00|4194.40|4236.34|884.10| +2452261|69492|2939|79346|1591205|3511|8170|2|114|2321|11|91.76|118.37|78.12|0.00|859.32|1009.36|1302.07|17.18|0.00|859.32|876.50|-150.04| +2452261|69492|6199|79346|1591205|3511|8170|2|35|2321|61|17.28|25.40|23.87|0.00|1456.07|1054.08|1549.40|0.00|0.00|1456.07|1456.07|401.99| +2452261|69492|1875|79346|1591205|3511|8170|2|73|2321|87|8.51|11.57|3.35|0.00|291.45|740.37|1006.59|20.40|0.00|291.45|311.85|-448.92| +2452261|69492|10615|79346|1591205|3511|8170|2|114|2321|3|61.11|72.72|39.99|0.00|119.97|183.33|218.16|0.00|0.00|119.97|119.97|-63.36| +2452261|69492|1429|79346|1591205|3511|8170|2|288|2321|87|79.63|138.55|67.88|531.50|5905.56|6927.81|12053.85|107.48|531.50|5374.06|5481.54|-1553.75| +2452261|69492|3261|79346|1591205|3511|8170|2|126|2321|9|55.95|106.86|102.58|0.00|923.22|503.55|961.74|64.62|0.00|923.22|987.84|419.67| +2452261|69492|11345|79346|1591205|3511|8170|2|137|2321|51|76.90|117.65|57.64|2880.84|2939.64|3921.90|6000.15|4.11|2880.84|58.80|62.91|-3863.10| +2452261|69492|13643|79346|1591205|3511|8170|2|10|2321|29|39.93|57.09|39.96|0.00|1158.84|1157.97|1655.61|46.35|0.00|1158.84|1205.19|0.87| +2452261|69492|17033|79346|1591205|3511|8170|2|60|2321|17|53.26|53.79|43.56|0.00|740.52|905.42|914.43|59.24|0.00|740.52|799.76|-164.90| +2452261|69492|14927|79346|1591205|3511|8170|2|51|2321|8|12.96|14.12|2.54|0.00|20.32|103.68|112.96|1.42|0.00|20.32|21.74|-83.36| +2452261|69492|2611|79346|1591205|3511|8170|2|259|2321|57|89.53|154.88|58.85|0.00|3354.45|5103.21|8828.16|301.90|0.00|3354.45|3656.35|-1748.76| +2452261|69492|3043|79346|1591205|3511|8170|2|273|2321|50|9.88|16.20|11.17|284.83|558.50|494.00|810.00|19.15|284.83|273.67|292.82|-220.33| +2452261|69492|1393|79346|1591205|3511|8170|2|119|2321|75|23.33|25.66|21.55|0.00|1616.25|1749.75|1924.50|129.30|0.00|1616.25|1745.55|-133.50| +2451901|47775|14573|37795|733634|226|8865|7|77|2322|78|99.03|165.38|3.30|0.00|257.40|7724.34|12899.64|12.87|0.00|257.40|270.27|-7466.94| +2451901|47775|1844|37795|733634|226|8865|7|5|2322|35|5.09|6.61|0.33|0.00|11.55|178.15|231.35|0.46|0.00|11.55|12.01|-166.60| +2451901|47775|14792|37795|733634|226|8865|7|272|2322|59|88.32|136.89|121.83|0.00|7187.97|5210.88|8076.51|575.03|0.00|7187.97|7763.00|1977.09| +2451901|47775|8173|37795|733634|226|8865|7|12|2322|43|64.15|78.90|71.01|0.00|3053.43|2758.45|3392.70|213.74|0.00|3053.43|3267.17|294.98| +2451901|47775|3986|37795|733634|226|8865|7|275|2322|34|67.97|114.18|15.98|0.00|543.32|2310.98|3882.12|48.89|0.00|543.32|592.21|-1767.66| +2451901|47775|12763|37795|733634|226|8865|7|146|2322|93|64.03|105.00|45.15|0.00|4198.95|5954.79|9765.00|41.98|0.00|4198.95|4240.93|-1755.84| +2451901|47775|13742|37795|733634|226|8865|7|33|2322|72|54.74|89.77|62.83|3980.90|4523.76|3941.28|6463.44|32.57|3980.90|542.86|575.43|-3398.42| +2451901|47775|2720|37795|733634|226|8865|7|227|2322|70|34.26|50.36|14.10|0.00|987.00|2398.20|3525.20|9.87|0.00|987.00|996.87|-1411.20| +2451901||2024||733634|||||2322||47.58||16.51|0.00||1427.40|1598.40|19.81|0.00|495.30|515.11|| +2451901|47775|16015|37795|733634|226|8865|7|287|2322|44|86.44|128.79|64.39|0.00|2833.16|3803.36|5666.76|254.98|0.00|2833.16|3088.14|-970.20| +2452129|34223|17707|13569|1322988|7160|28132|10|80|2323|55|90.55|131.29|103.71|0.00|5704.05|4980.25|7220.95|114.08|0.00|5704.05|5818.13|723.80| +2452129|34223|2663|13569|1322988|7160|28132|10|129|2323|31|36.78|37.51|19.13|0.00|593.03|1140.18|1162.81|53.37|0.00|593.03|646.40|-547.15| +2452129|34223|12391|13569|1322988|7160|28132|10|79|2323|4|38.16|50.37|32.23|0.00|128.92|152.64|201.48|3.86|0.00|128.92|132.78|-23.72| +2452129|34223|16231|13569|1322988|7160|28132|10|85|2323|16|34.16|60.46|0.60|2.59|9.60|546.56|967.36|0.49|2.59|7.01|7.50|-539.55| +2452129|34223|14993|13569|1322988|7160|28132|10|110|2323|58|11.45|11.79|3.06|0.00|177.48|664.10|683.82|0.00|0.00|177.48|177.48|-486.62| +2452129|34223|7537|13569|1322988|7160|28132|10|228|2323|19|69.89|77.57|19.39|0.00|368.41|1327.91|1473.83|0.00|0.00|368.41|368.41|-959.50| +2452129|34223|15607|13569|1322988|7160|28132|10|87|2323|74|11.23|22.34|2.90|0.00|214.60|831.02|1653.16|8.58|0.00|214.60|223.18|-616.42| +2452129|34223|7069|13569|1322988|7160|28132|10|300|2323|9|87.53|91.03|75.55|231.18|679.95|787.77|819.27|8.97|231.18|448.77|457.74|-339.00| +2451790|39195|107|49481|1366603|6125|19247|8|122|2324|75|67.69|95.44|20.04|0.00|1503.00|5076.75|7158.00|135.27|0.00|1503.00|1638.27|-3573.75| +2451790|39195|277|49481|1366603|6125|19247|8|103|2324|29|48.61|84.09|24.38|289.87|707.02|1409.69|2438.61|16.68|289.87|417.15|433.83|-992.54| +2451790|39195|4343|49481|1366603|6125|19247|8|261|2324|7|79.26|152.97|133.08|0.00|931.56|554.82|1070.79|27.94|0.00|931.56|959.50|376.74| +2451790|39195|620|49481|1366603|6125|19247|8|270|2324|76|58.71|72.80|45.86|0.00|3485.36|4461.96|5532.80|34.85|0.00|3485.36|3520.21|-976.60| +2451790|39195|13820|49481|1366603|6125|19247|8|63|2324|95|73.21|89.31|66.98|4199.64|6363.10|6954.95|8484.45|108.17|4199.64|2163.46|2271.63|-4791.49| +2451790|39195|10394|49481|1366603|6125|19247|8|113|2324|6|34.33|67.28|27.58|0.00|165.48|205.98|403.68|13.23|0.00|165.48|178.71|-40.50| +2451790|39195|10075|49481|1366603|6125|19247|8|29|2324|87|32.55|50.77|29.95|1094.37|2605.65|2831.85|4416.99|120.90|1094.37|1511.28|1632.18|-1320.57| +2451790|39195|7082|49481|1366603|6125|19247|8|258|2324|14|56.45|59.27|29.63|0.00|414.82|790.30|829.78|16.59|0.00|414.82|431.41|-375.48| +2451790|39195|9541|49481|1366603|6125|19247|8|273|2324|12|95.84|155.26|0.00|0.00|0.00|1150.08|1863.12|0.00|0.00|0.00|0.00|-1150.08| +2451790|39195|10067|49481|1366603|6125|19247|8|52|2324|24|60.19|90.28|26.18|0.00|628.32|1444.56|2166.72|25.13|0.00|628.32|653.45|-816.24| +2451790|39195|353|49481|1366603|6125|19247|8|58|2324|6|57.39|70.58|69.16|0.00|414.96|344.34|423.48|33.19|0.00|414.96|448.15|70.62| +2451790|39195|12656|49481|1366603|6125|19247|8|224|2324|72|73.62|93.49|56.09|3150.01|4038.48|5300.64|6731.28|71.07|3150.01|888.47|959.54|-4412.17| +2451790|39195|10766|49481|1366603|6125|19247|8|75|2324|47|30.36|49.18|30.98|0.00|1456.06|1426.92|2311.46|0.00|0.00|1456.06|1456.06|29.14| +2451790|39195|16394|49481|1366603|6125|19247|8|164|2324|11|50.75|90.84|23.61|0.00|259.71|558.25|999.24|15.58|0.00|259.71|275.29|-298.54| +||5276||1366603||19247|8|2|2324|69||||0.00||4549.17|6368.70||0.00|3184.35|3311.72|-1364.82| +2452373|58678|13254|22796|1728808|3173|14338|1|226|2325|89|94.48|184.23|75.53|0.00|6722.17|8408.72|16396.47|403.33|0.00|6722.17|7125.50|-1686.55| +2452373|58678|16290|22796|1728808|3173|14338|1|143|2325|100|81.45|87.15|77.56|0.00|7756.00|8145.00|8715.00|698.04|0.00|7756.00|8454.04|-389.00| +2452373|58678|16845|22796|1728808|3173|14338|1|84|2325|64|62.18|73.37|25.67|476.43|1642.88|3979.52|4695.68|69.98|476.43|1166.45|1236.43|-2813.07| +2452373|58678|16758|22796|1728808|3173|14338|1|168|2325|29|67.29|101.60|50.80|0.00|1473.20|1951.41|2946.40|88.39|0.00|1473.20|1561.59|-478.21| +2452373|58678|17628|22796|1728808|3173|14338|1|143|2325|68|13.02|14.71|10.88|0.00|739.84|885.36|1000.28|36.99|0.00|739.84|776.83|-145.52| +2452373|58678|2635|22796|1728808|3173|14338|1|208|2325|37|76.57|110.26|101.43|3640.32|3752.91|2833.09|4079.62|2.25|3640.32|112.59|114.84|-2720.50| +2452373|58678|2427|22796|1728808|3173|14338|1|157|2325|20|91.57|176.73|63.62|0.00|1272.40|1831.40|3534.60|0.00|0.00|1272.40|1272.40|-559.00| +2452373|58678|15147|22796|1728808|3173|14338|1|267|2325|12|86.49|105.51|16.88|0.00|202.56|1037.88|1266.12|4.05|0.00|202.56|206.61|-835.32| +2452373|58678|12321|22796|1728808|3173|14338|1|208|2325|89|35.13|60.77|21.26|0.00|1892.14|3126.57|5408.53|94.60|0.00|1892.14|1986.74|-1234.43| +2452373|58678|7831|22796|1728808|3173|14338|1|191|2325|39|97.84|194.70|138.23|2479.84|5390.97|3815.76|7593.30|262.00|2479.84|2911.13|3173.13|-904.63| +2452373|58678|12990|22796|1728808|3173|14338|1|102|2325|47|35.57|50.50|23.23|655.08|1091.81|1671.79|2373.50|26.20|655.08|436.73|462.93|-1235.06| +2452373|58678|12792|22796|1728808|3173|14338|1|291|2325|99|24.13|38.12|16.39|0.00|1622.61|2388.87|3773.88|97.35|0.00|1622.61|1719.96|-766.26| +2452373|58678|5515|22796|1728808|3173|14338|1|190|2325|9|53.53|70.65|38.85|0.00|349.65|481.77|635.85|17.48|0.00|349.65|367.13|-132.12| +2452373|58678|7992|22796|1728808|3173|14338|1|30|2325|26|16.24|25.33|21.27|0.00|553.02|422.24|658.58|33.18|0.00|553.02|586.20|130.78| +2452373|58678|9685||||14338|||2325||67.21|99.47||4385.08|4818.78||5669.79|17.34|4385.08|||| +2452373|58678|11037|22796|1728808|3173|14338|1|42|2325|99|66.39|126.80|5.07|296.13|501.93|6572.61|12553.20|10.29|296.13|205.80|216.09|-6366.81| +2451841|63939|15845||1739487|4327|49967|4||2326|12|32.27||6.52|0.00|78.24|387.24|460.80||0.00|78.24||-309.00| +2451841|63939|17633|21847|1739487|4327|49967|4|292|2326|89|46.04|70.90|15.59|0.00|1387.51|4097.56|6310.10|0.00|0.00|1387.51|1387.51|-2710.05| +2451841|63939|12169|21847|1739487|4327|49967|4|236|2326|59|60.11|117.81|43.58|0.00|2571.22|3546.49|6950.79|0.00|0.00|2571.22|2571.22|-975.27| +2451841|63939|16208|21847|1739487|4327|49967|4|107|2326|55|14.00|17.50|2.97|0.00|163.35|770.00|962.50|11.43|0.00|163.35|174.78|-606.65| +2451841|63939|836|21847|1739487|4327|49967|4|64|2326|11|67.17|116.20|4.64|0.00|51.04|738.87|1278.20|0.51|0.00|51.04|51.55|-687.83| +2451841|63939|11627|21847|1739487|4327|49967|4|268|2326|80|65.84|77.69|51.27|0.00|4101.60|5267.20|6215.20|0.00|0.00|4101.60|4101.60|-1165.60| +2451841|63939|17879|21847|1739487|4327|49967|4|253|2326|20|88.94|129.85|25.97|0.00|519.40|1778.80|2597.00|20.77|0.00|519.40|540.17|-1259.40| +2451841|63939|17737|21847|1739487|4327|49967|4|104|2326|36|87.79|109.73|94.36|0.00|3396.96|3160.44|3950.28|271.75|0.00|3396.96|3668.71|236.52| +2451841|63939|6872|21847|1739487|4327|49967|4|182|2326|32|22.09|43.51|17.40|0.00|556.80|706.88|1392.32|11.13|0.00|556.80|567.93|-150.08| +2451841|63939|16979|21847|1739487|4327|49967|4|290|2326|43|38.45|55.75|30.10|0.00|1294.30|1653.35|2397.25|25.88|0.00|1294.30|1320.18|-359.05| +2451841|63939|7103|21847|1739487|4327|49967|4|36|2326|72|70.70|76.35|34.35|0.00|2473.20|5090.40|5497.20|49.46|0.00|2473.20|2522.66|-2617.20| +2451841|63939|12077|21847|1739487|4327|49967|4|168|2326|48|53.76|103.21|14.44|0.00|693.12|2580.48|4954.08|6.93|0.00|693.12|700.05|-1887.36| +2451841|63939|914|21847|1739487|4327|49967|4|29|2326|66|19.25|31.18|24.94|0.00|1646.04|1270.50|2057.88|0.00|0.00|1646.04|1646.04|375.54| +2451841|63939|1891|21847|1739487|4327|49967|4|192|2326|32|11.99|16.42|4.10|0.00|131.20|383.68|525.44|0.00|0.00|131.20|131.20|-252.48| +2451841|63939|1471|21847|1739487|4327|49967|4|13|2326|40|99.22|120.05|60.02|2040.68|2400.80|3968.80|4802.00|32.41|2040.68|360.12|392.53|-3608.68| +|33937|5041|||||||2327||53.52|70.64|61.45|||749.28|988.96|0.00||860.30||111.02| +2451174|33937|4534|40056|613150|4997|39691|4|295|2327|5|15.34|16.56|9.10|0.00|45.50|76.70|82.80|1.36|0.00|45.50|46.86|-31.20| +2451174|33937|4594|40056|613150|4997|39691|4|183|2327|81|39.15|62.24|42.94|0.00|3478.14|3171.15|5041.44|139.12|0.00|3478.14|3617.26|306.99| +|33937|11546||613150||39691|4|57|2327|24||24.82||71.42||||1.60|71.42|17.86||-487.10| +2451174|33937|12236|40056|613150|4997|39691|4|49|2327|33|20.79|25.57|15.34|0.00|506.22|686.07|843.81|5.06|0.00|506.22|511.28|-179.85| +||2671||613150|4997|||249|2327||93.10||||999.90||1075.25|29.99||999.90|1029.89|-24.20| +2451174|33937|15025|40056|613150|4997|39691|4|159|2327|31|29.17|31.50|10.39|0.00|322.09|904.27|976.50|25.76|0.00|322.09|347.85|-582.18| +2451174|33937|2137|40056|613150|4997|39691|4|63|2327|32|94.45|176.62|141.29|0.00|4521.28|3022.40|5651.84|180.85|0.00|4521.28|4702.13|1498.88| +2451174|33937|16028|40056|613150|4997|39691|4|104|2327|5|78.24|155.69|12.45|0.00|62.25|391.20|778.45|4.98|0.00|62.25|67.23|-328.95| +2451174|33937|7138|40056|613150|4997|39691|4|217|2327|9|88.32|110.40|64.03|0.00|576.27|794.88|993.60|23.05|0.00|576.27|599.32|-218.61| +2451174|33937|11440|40056|613150|4997|39691|4|141|2327|95|82.28|119.30|79.93|0.00|7593.35|7816.60|11333.50|379.66|0.00|7593.35|7973.01|-223.25| +2451174|33937|10756|40056|613150|4997|39691|4|12|2327|2|92.63|109.30|9.83|0.00|19.66|185.26|218.60|1.37|0.00|19.66|21.03|-165.60| +2451174|33937|1304|40056|613150|4997|39691|4|127|2327|43|20.02|35.63|30.28|0.00|1302.04|860.86|1532.09|117.18|0.00|1302.04|1419.22|441.18| +2451471|60344|8521|63732|1745567|1802|33684|4|31|2328|20|62.83|63.45|9.51|0.00|190.20|1256.60|1269.00|13.31|0.00|190.20|203.51|-1066.40| +2451471|60344|2710|63732|1745567|1802|33684|4|145|2328|90|59.02|90.89|34.53|0.00|3107.70|5311.80|8180.10|124.30|0.00|3107.70|3232.00|-2204.10| +2451471|60344|11527|63732|1745567|1802|33684|4|105|2328|21|77.93|88.06|64.28|0.00|1349.88|1636.53|1849.26|53.99|0.00|1349.88|1403.87|-286.65| +2451471|60344|9829|63732|1745567|1802|33684|4|238|2328|36|26.56|27.09|11.10|0.00|399.60|956.16|975.24|35.96|0.00|399.60|435.56|-556.56| +2451471|60344|1873|63732|1745567|1802|33684|4|282|2328|73|65.72|111.72|91.61|0.00|6687.53|4797.56|8155.56|200.62|0.00|6687.53|6888.15|1889.97| +||5456||1745567|||||2328||5.05||||466.46|419.15||||466.46|485.11|| +2451471|60344|2774|63732|1745567|1802|33684|4|173|2328|70|55.92|88.91|71.12|1045.46|4978.40|3914.40|6223.70|78.65|1045.46|3932.94|4011.59|18.54| +2451471|60344|11666|63732|1745567|1802|33684|4|112|2328|33|25.16|28.17|21.69|0.00|715.77|830.28|929.61|7.15|0.00|715.77|722.92|-114.51| +2451471|60344|11536|63732|1745567|1802|33684|4|79|2328|76|61.53|84.91|26.32|0.00|2000.32|4676.28|6453.16|80.01|0.00|2000.32|2080.33|-2675.96| +2451471|60344|10309|63732|1745567|1802|33684|4|177|2328|30|30.75|45.51|8.19|0.00|245.70|922.50|1365.30|17.19|0.00|245.70|262.89|-676.80| +2451471|60344|4930|63732|1745567|1802|33684|4|215|2328|79|49.31|70.02|33.60|1937.71|2654.40|3895.49|5531.58|0.00|1937.71|716.69|716.69|-3178.80| +2451471|60344|16246|63732|1745567|1802|33684|4|249|2328|83|59.54|79.78|31.11|0.00|2582.13|4941.82|6621.74|25.82|0.00|2582.13|2607.95|-2359.69| +2451471|60344|5995|63732|1745567|1802|33684|4|281|2328|93|20.59|22.64|12.90|0.00|1199.70|1914.87|2105.52|35.99|0.00|1199.70|1235.69|-715.17| +2451351|38958|4772|99900|432725|4303|41881|8|110|2329|16|92.32|120.01|46.80|0.00|748.80|1477.12|1920.16|37.44|0.00|748.80|786.24|-728.32| +2451351|38958|17896|99900|432725|4303|41881|8|106|2329|13|43.32|58.04|4.06|0.00|52.78|563.16|754.52|1.05|0.00|52.78|53.83|-510.38| +2451351|38958|15622|99900|432725|4303|41881|8|276|2329|1|39.19|53.29|36.77|0.00|36.77|39.19|53.29|0.00|0.00|36.77|36.77|-2.42| +2451351|38958|16586|99900|432725|4303|41881|8|155|2329|85|25.70|36.75|31.97|0.00|2717.45|2184.50|3123.75|0.00|0.00|2717.45|2717.45|532.95| +2451351|38958|478|99900|432725|4303|41881|8|12|2329|47|23.65|25.06|18.04|0.00|847.88|1111.55|1177.82|0.00|0.00|847.88|847.88|-263.67| +2451351|38958|6187|99900|432725|4303|41881|8|190|2329|42|92.40|140.44|35.11|0.00|1474.62|3880.80|5898.48|14.74|0.00|1474.62|1489.36|-2406.18| +2451351|38958|14056|99900|432725|4303|41881|8|169|2329|58|81.50|142.62|18.54|0.00|1075.32|4727.00|8271.96|0.00|0.00|1075.32|1075.32|-3651.68| +2451351|38958|10009|99900|432725|4303|41881|8|35|2329|75|63.96|104.25|32.31|1138.92|2423.25|4797.00|7818.75|102.74|1138.92|1284.33|1387.07|-3512.67| +2451351|38958|15212|99900|432725|4303|41881|8|98|2329|10|95.42|185.11|172.15|0.00|1721.50|954.20|1851.10|0.00|0.00|1721.50|1721.50|767.30| +2451351|38958|13462|99900|432725|4303|41881|8|136|2329|58|68.64|133.84|120.45|0.00|6986.10|3981.12|7762.72|279.44|0.00|6986.10|7265.54|3004.98| +2451905|62397|13640|78994|973992|2943|28294|8|41|2330|22|56.65|100.83|62.51|0.00|1375.22|1246.30|2218.26|68.76|0.00|1375.22|1443.98|128.92| +2451905|62397|2189|78994|973992|2943|28294|8|252|2330|9|73.06|130.04|65.02|0.00|585.18|657.54|1170.36|35.11|0.00|585.18|620.29|-72.36| +2451905|62397|15065|78994|973992|2943|28294|8|36|2330|2|57.17|107.47|94.57|0.00|189.14|114.34|214.94|3.78|0.00|189.14|192.92|74.80| +2451905|62397|8588|78994|973992|2943|28294|8|162|2330|72|10.02|16.43|0.32|0.00|23.04|721.44|1182.96|0.69|0.00|23.04|23.73|-698.40| +2451905|62397|16321|78994|973992|2943|28294|8|211|2330|94|65.54|121.90|91.42|0.00|8593.48|6160.76|11458.60|257.80|0.00|8593.48|8851.28|2432.72| +2451905|62397|8489|78994|973992|2943|28294|8|263|2330|24|21.64|33.75|20.25|0.00|486.00|519.36|810.00|9.72|0.00|486.00|495.72|-33.36| +2451905|62397|2503|78994|973992|2943|28294|8|167|2330|68|18.63|19.74|8.68|584.33|590.24|1266.84|1342.32|0.47|584.33|5.91|6.38|-1260.93| +2451905|62397|7199|78994|973992|2943|28294|8|162|2330|88|65.76|119.02|2.38|0.00|209.44|5786.88|10473.76|12.56|0.00|209.44|222.00|-5577.44| +2451905|62397|9125|78994|973992|2943|28294|8|105|2330|76|6.55|11.85|9.95|0.00|756.20|497.80|900.60|30.24|0.00|756.20|786.44|258.40| +2451905|62397|11213|78994|973992|2943|28294|8|158|2330|54|63.13|82.06|18.05|955.20|974.70|3409.02|4431.24|1.17|955.20|19.50|20.67|-3389.52| +2451905|62397|6968|78994|973992|2943|28294|8|27|2330|37|47.32|77.13|60.93|0.00|2254.41|1750.84|2853.81|135.26|0.00|2254.41|2389.67|503.57| +2451905|62397|13985|78994|973992|2943|28294|8|111|2330|83|22.09|37.77|0.37|21.18|30.71|1833.47|3134.91|0.38|21.18|9.53|9.91|-1823.94| +||3867|||3509|43336|4||2331|||||0.00|4622.10|||92.44|0.00|4622.10||| +2452585|50482|14295|69455|998482|3509|43336|4|21|2331|91|55.57|85.02|62.06|0.00|5647.46|5056.87|7736.82|338.84|0.00|5647.46|5986.30|590.59| +2452585|50482|5202|69455|998482|3509|43336|4|223|2331|82|72.52|136.33|107.70|0.00|8831.40|5946.64|11179.06|794.82|0.00|8831.40|9626.22|2884.76| +2452585|50482|7896|69455|998482|3509|43336|4|246|2331|10|73.86|116.69|28.00|0.00|280.00|738.60|1166.90|11.20|0.00|280.00|291.20|-458.60| +2452585|50482|9873|69455|998482|3509|43336|4|130|2331|33|63.18|71.39|40.69|0.00|1342.77|2084.94|2355.87|40.28|0.00|1342.77|1383.05|-742.17| +2452585|50482|8299|69455|998482|3509|43336|4|134|2331|13|94.60|102.16|79.68|0.00|1035.84|1229.80|1328.08|10.35|0.00|1035.84|1046.19|-193.96| +2452585|50482|17163|69455|998482|3509|43336|4|15|2331|91|94.72|162.91|34.21|124.52|3113.11|8619.52|14824.81|0.00|124.52|2988.59|2988.59|-5630.93| +2452585|50482|6217|69455|998482|3509|43336|4|171|2331|99|75.65|147.51|67.85|0.00|6717.15|7489.35|14603.49|537.37|0.00|6717.15|7254.52|-772.20| +2451138||10238|25987|300718|6296||||2332||31.69||33.76||810.24|760.56||||372.72||-387.84| +|61292|7201|25987|300718||29027|||2332||21.30|||||1341.90|||||169.47|| +2451138|61292|445|25987|300718|6296|29027|2|293|2332|54|51.65|68.17|34.76|0.00|1877.04|2789.10|3681.18|93.85|0.00|1877.04|1970.89|-912.06| +2451138|61292|16024|25987|300718|6296|29027|2|217|2332|14|67.52|110.05|72.63|0.00|1016.82|945.28|1540.70|10.16|0.00|1016.82|1026.98|71.54| +2451138|61292|7652|25987|300718|6296|29027|2|64|2332|6|92.85|138.34|125.88|0.00|755.28|557.10|830.04|52.86|0.00|755.28|808.14|198.18| +2451138|61292|12556|25987|300718|6296|29027|2|24|2332|71|68.01|86.37|60.45|0.00|4291.95|4828.71|6132.27|0.00|0.00|4291.95|4291.95|-536.76| +2451138|61292|16270|25987|300718|6296|29027|2|106|2332|41|91.14|147.64|23.62|0.00|968.42|3736.74|6053.24|67.78|0.00|968.42|1036.20|-2768.32| +|61292|1849|||6296|29027|2||2332|95|9.67|14.40|10.65|809.40|||1368.00||809.40|||-716.30| +2451138|61292|9043|25987|300718|6296|29027|2|196|2332|96|17.80|29.72|3.56|0.00|341.76|1708.80|2853.12|27.34|0.00|341.76|369.10|-1367.04| +2451138|61292|7934|25987|300718|6296|29027|2|132|2332|64|44.05|62.11|48.44|0.00|3100.16|2819.20|3975.04|31.00|0.00|3100.16|3131.16|280.96| +2451138|61292|13460|25987|300718|6296|29027|2|193|2332|78|79.90|158.20|158.20|0.00|12339.60|6232.20|12339.60|123.39|0.00|12339.60|12462.99|6107.40| +2451138|61292|16735|25987|300718|6296|29027|2|274|2332|35|68.17|128.84|24.47|0.00|856.45|2385.95|4509.40|17.12|0.00|856.45|873.57|-1529.50| +2451138|61292|12874|25987|300718|6296|29027|2|279|2332|89|60.67|103.13|56.72|4088.94|5048.08|5399.63|9178.57|86.32|4088.94|959.14|1045.46|-4440.49| +2451138|61292|7954|25987|300718|6296|29027|2|228|2332|75|54.67|102.23|50.09|0.00|3756.75|4100.25|7667.25|112.70|0.00|3756.75|3869.45|-343.50| +2451138|61292|8324|25987|300718|6296|29027|2|20|2332|95|8.52|15.25|9.76|0.00|927.20|809.40|1448.75|27.81|0.00|927.20|955.01|117.80| +2451138|61292|10436|25987|300718|6296|29027|2|265|2332|15|58.76|61.69|25.29|0.00|379.35|881.40|925.35|30.34|0.00|379.35|409.69|-502.05| +2452465|61274|10872|59584|803253|2122|23391|10|194|2333|5|58.01|63.23|1.26|0.00|6.30|290.05|316.15|0.00|0.00|6.30|6.30|-283.75| +2452465|61274|4182|59584|803253|2122|23391|10|92|2333|82|95.06|113.12|50.90|2504.28|4173.80|7794.92|9275.84|100.17|2504.28|1669.52|1769.69|-6125.40| +2452465|61274|2970|59584|803253|2122|23391|10|209|2333|90|90.40|95.82|45.03|0.00|4052.70|8136.00|8623.80|40.52|0.00|4052.70|4093.22|-4083.30| +2452465|61274|9519|59584|803253|2122|23391|10|264|2333|50|11.19|14.32|11.31|0.00|565.50|559.50|716.00|11.31|0.00|565.50|576.81|6.00| +2452465|61274|9012|59584|803253|2122|23391|10|252|2333|51|11.60|20.88|7.93|56.62|404.43|591.60|1064.88|27.82|56.62|347.81|375.63|-243.79| +2452465|61274|8625|59584|803253|2122|23391|10|290|2333|8|44.43|87.97|8.79|0.00|70.32|355.44|703.76|1.40|0.00|70.32|71.72|-285.12| +2452465|61274|10536|59584|803253|2122|23391|10|116|2333|87|3.48|6.71|0.53|0.00|46.11|302.76|583.77|3.22|0.00|46.11|49.33|-256.65| +2452465|61274|6001|59584|803253|2122|23391|10|83|2333|69|36.24|44.21|19.01|0.00|1311.69|2500.56|3050.49|78.70|0.00|1311.69|1390.39|-1188.87| +2452465|61274|11569|59584|803253|2122|23391|10|283|2333|19|98.35|185.88|52.04|0.00|988.76|1868.65|3531.72|0.00|0.00|988.76|988.76|-879.89| +2452465|61274|14731|59584|803253|2122|23391|10|247|2333|79|85.16|158.39|134.63|0.00|10635.77|6727.64|12512.81|0.00|0.00|10635.77|10635.77|3908.13| +2452465|61274|13062|59584|803253|2122|23391|10|203|2333|32|93.03|132.10|67.37|0.00|2155.84|2976.96|4227.20|172.46|0.00|2155.84|2328.30|-821.12| +2452329|69650|5455|1587|725028|1716|45604|1|51|2334|50|47.72|91.14|85.67|0.00|4283.50|2386.00|4557.00|214.17|0.00|4283.50|4497.67|1897.50| +2452329|69650|1500|1587|725028|1716|45604|1|40|2334|65|84.18|165.83|159.19|0.00|10347.35|5471.70|10778.95|413.89|0.00|10347.35|10761.24|4875.65| +2452329|69650|9402|1587|725028|1716|45604|1|266|2334|6|85.13|97.04|62.10|0.00|372.60|510.78|582.24|3.72|0.00|372.60|376.32|-138.18| +2452329|69650|12084|1587|725028|1716|45604|1|23|2334|71|68.77|114.15|9.13|0.00|648.23|4882.67|8104.65|51.85|0.00|648.23|700.08|-4234.44| +2452329|69650|11010|1587|725028|1716|45604|1|197|2334|31|20.83|30.20|12.38|0.00|383.78|645.73|936.20|11.51|0.00|383.78|395.29|-261.95| +2452329|69650|15306|1587|725028|1716|45604|1|200|2334|55|20.74|26.54|18.04|0.00|992.20|1140.70|1459.70|59.53|0.00|992.20|1051.73|-148.50| +2452329|69650|9318|1587|725028|1716|45604|1|141|2334|46|97.08|167.94|159.54|0.00|7338.84|4465.68|7725.24|220.16|0.00|7338.84|7559.00|2873.16| +2452329|69650|5127|1587|725028|1716|45604|1|149|2334|74|62.64|92.08|77.34|4406.83|5723.16|4635.36|6813.92|13.16|4406.83|1316.33|1329.49|-3319.03| +2452329|69650|16320|1587|725028|1716|45604|1|190|2334|38|37.98|58.10|20.33|0.00|772.54|1443.24|2207.80|61.80|0.00|772.54|834.34|-670.70| +2451511|41595|7466|54086|1819856|2426|25396|7|129|2335|52|19.02|20.92|10.25|0.00|533.00|989.04|1087.84|5.33|0.00|533.00|538.33|-456.04| +2451511|41595|17068|54086|1819856|2426|25396|7|8|2335|26|48.03|83.09|70.62|0.00|1836.12|1248.78|2160.34|128.52|0.00|1836.12|1964.64|587.34| +2451511|41595|16096|54086|1819856|2426|25396|7|258|2335|86|78.42|147.42|112.03|0.00|9634.58|6744.12|12678.12|96.34|0.00|9634.58|9730.92|2890.46| +2451511|41595|9932|54086|1819856|2426|25396|7|289|2335|74|96.73|132.52|47.70|0.00|3529.80|7158.02|9806.48|176.49|0.00|3529.80|3706.29|-3628.22| +2451511|41595|11132|54086|1819856|2426|25396|7|201|2335|77|1.22|2.31|0.83|0.00|63.91|93.94|177.87|3.19|0.00|63.91|67.10|-30.03| +2451511|41595|440|54086|1819856|2426|25396|7|290|2335|36|37.24|46.55|10.24|0.00|368.64|1340.64|1675.80|22.11|0.00|368.64|390.75|-972.00| +2451511|41595|10873|54086|1819856|2426|25396|7|227|2335|11|52.99|68.88|55.10|0.00|606.10|582.89|757.68|36.36|0.00|606.10|642.46|23.21| +2451511|41595|832|54086|1819856|2426|25396|7|172|2335|40|94.39|137.80|63.38|1242.24|2535.20|3775.60|5512.00|0.00|1242.24|1292.96|1292.96|-2482.64| +2451511|41595|1240|54086|1819856|2426|25396|7|280|2335|35|93.76|134.07|64.35|1283.78|2252.25|3281.60|4692.45|19.36|1283.78|968.47|987.83|-2313.13| +2451511|41595|1543|54086|1819856|2426|25396|7|187|2335|32|73.54|110.31|17.64|400.78|564.48|2353.28|3529.92|11.45|400.78|163.70|175.15|-2189.58| +2451511|41595|15514|54086|1819856|2426|25396|7|107|2335|14|93.89|155.85|45.19|0.00|632.66|1314.46|2181.90|12.65|0.00|632.66|645.31|-681.80| +2451910|32813|6289|23595|1067353|3111|15669|1|288|2336|48|82.82|100.21|46.09|619.44|2212.32|3975.36|4810.08|95.57|619.44|1592.88|1688.45|-2382.48| +2451910|32813|3869|23595|1067353|3111|15669|1|193|2336|76|12.51|20.39|11.82|0.00|898.32|950.76|1549.64|71.86|0.00|898.32|970.18|-52.44| +2451910|32813|16687|23595|1067353|3111|15669|1|14|2336|45|67.07|73.77|0.00|0.00|0.00|3018.15|3319.65|0.00|0.00|0.00|0.00|-3018.15| +2451910|32813|11918|23595|1067353|3111|15669|1|185|2336|43|10.50|19.95|0.19|0.00|8.17|451.50|857.85|0.08|0.00|8.17|8.25|-443.33| +2451910|32813|14815|23595|1067353|3111|15669|1|50|2336|90|29.39|47.61|19.99|0.00|1799.10|2645.10|4284.90|53.97|0.00|1799.10|1853.07|-846.00| +2451910|32813|12755|23595|1067353|3111|15669|1|213|2336|74|35.97|38.84|17.47|0.00|1292.78|2661.78|2874.16|12.92|0.00|1292.78|1305.70|-1369.00| +2451910|32813|1151|23595|1067353|3111|15669|1|38|2336|7|16.79|25.68|2.56|0.00|17.92|117.53|179.76|0.53|0.00|17.92|18.45|-99.61| +2451910|32813|6962|23595|1067353|3111|15669|1|8|2336|8|34.42|45.09|22.54|138.84|180.32|275.36|360.72|3.31|138.84|41.48|44.79|-233.88| +2451910|32813|12863|23595|1067353|3111|15669|1|222|2336|26|54.47|62.64|52.61|0.00|1367.86|1416.22|1628.64|95.75|0.00|1367.86|1463.61|-48.36| +2451910|32813|4219|23595|1067353|3111|15669|1|80|2336|77|62.74|125.48|84.07|0.00|6473.39|4830.98|9661.96|64.73|0.00|6473.39|6538.12|1642.41| +|32813|4268|23595|1067353|||1|121|2336||||||||11050.00|||10829.00||| +2451910|32813|1109|23595|1067353|3111|15669|1|39|2336|80|5.62|7.41|6.22|0.00|497.60|449.60|592.80|24.88|0.00|497.60|522.48|48.00| +2451910|32813|11227|23595|1067353|3111|15669|1|285|2336|18|98.43|103.35|10.33|0.00|185.94|1771.74|1860.30|9.29|0.00|185.94|195.23|-1585.80| +||1310|23595|1067353|3111||||2336|88|97.00||24.24|0.00|2133.12|8536.00|10157.84|63.99|0.00|2133.12|2197.11|| +2451910|32813|12113|23595|1067353|3111|15669|1|255|2336|40|16.91|29.25|15.21|0.00|608.40|676.40|1170.00|18.25|0.00|608.40|626.65|-68.00| +2452589|33063|10590|60581|171231|7128|32225|1|202|2337|36|16.95|27.62|10.21|25.72|367.56|610.20|994.32|3.41|25.72|341.84|345.25|-268.36| +2452589|33063|7335|60581|171231|7128|32225|1|72|2337|26|42.72|62.37|29.93|155.63|778.18|1110.72|1621.62|56.02|155.63|622.55|678.57|-488.17| +2452589|33063|3960|60581|171231|7128|32225|1|220|2337|79|88.25|90.01|20.70|0.00|1635.30|6971.75|7110.79|147.17|0.00|1635.30|1782.47|-5336.45| +2452589|33063|13435|60581|171231|7128|32225|1|158|2337|22|85.99|124.68|92.26|101.48|2029.72|1891.78|2742.96|0.00|101.48|1928.24|1928.24|36.46| +2452589|33063|17289|60581|171231|7128|32225|1|117|2337|29|67.61|123.72|54.43|757.66|1578.47|1960.69|3587.88|41.04|757.66|820.81|861.85|-1139.88| +2452589|33063|829|60581|171231|7128|32225|1|81|2337|65|40.49|45.34|43.52|2743.93|2828.80|2631.85|2947.10|7.63|2743.93|84.87|92.50|-2546.98| +||2263|||7128||||2337|||||0.00||425.59|778.80||0.00||657.70|212.96| +2452589|33063|7431|60581|171231|7128|32225|1|68|2337|69|87.95|130.16|10.41|0.00|718.29|6068.55|8981.04|43.09|0.00|718.29|761.38|-5350.26| +2452589|33063|3342|60581|171231|7128|32225|1|204|2337|19|63.39|96.98|9.69|25.77|184.11|1204.41|1842.62|0.00|25.77|158.34|158.34|-1046.07| +2452589|33063|5173|60581|171231|7128|32225|1|284|2337|23|7.13|12.83|5.90|0.00|135.70|163.99|295.09|10.85|0.00|135.70|146.55|-28.29| +2452589|33063|973|60581|171231|7128|32225|1|300|2337|14|86.42|105.43|95.94|0.00|1343.16|1209.88|1476.02|13.43|0.00|1343.16|1356.59|133.28| +2452589|33063|17997|60581|171231|7128|32225|1|205|2337|92|98.64|184.45|123.58|1819.09|11369.36|9074.88|16969.40|764.02|1819.09|9550.27|10314.29|475.39| +2452589|33063|5263|60581|171231|7128|32225|1|292|2337|39|30.60|38.25|9.94|379.90|387.66|1193.40|1491.75|0.31|379.90|7.76|8.07|-1185.64| +2452589|33063|5941|60581|171231|7128|32225|1|288|2337|71|5.60|9.74|8.86|0.00|629.06|397.60|691.54|50.32|0.00|629.06|679.38|231.46| +2452589|33063|8439|60581|171231|7128|32225|1|209|2337|9|8.03|14.85|9.50|50.44|85.50|72.27|133.65|2.80|50.44|35.06|37.86|-37.21| +2452589|33063|15711|60581|171231|7128|32225|1|59|2337|28|48.91|54.77|19.16|337.98|536.48|1369.48|1533.56|11.91|337.98|198.50|210.41|-1170.98| +2451855|63096|12763|26541|1312274|54|8187|10|85|2338|20|38.72|40.26|27.77|83.31|555.40|774.40|805.20|33.04|83.31|472.09|505.13|-302.31| +2451855|63096|13742|26541|1312274|54|8187|10|281|2338|58|61.99|74.38|8.92|0.00|517.36|3595.42|4314.04|25.86|0.00|517.36|543.22|-3078.06| +2451855|63096|2720|26541|1312274|54|8187|10|298|2338|8|85.78|107.22|75.05|0.00|600.40|686.24|857.76|12.00|0.00|600.40|612.40|-85.84| +2451855|63096|2024|26541|1312274|54|8187|10|112|2338|71|83.61|127.08|0.00|0.00|0.00|5936.31|9022.68|0.00|0.00|0.00|0.00|-5936.31| +2451855|63096|16015|26541|1312274|54|8187|10|284|2338|46|81.88|110.53|102.79|3167.98|4728.34|3766.48|5084.38|62.41|3167.98|1560.36|1622.77|-2206.12| +2451855|63096|17521|26541|1312274|54|8187|10|31|2338|9|48.73|72.12|6.49|0.00|58.41|438.57|649.08|0.58|0.00|58.41|58.99|-380.16| +||9079|26541|||8187|10|297|2338||5.89||4.94|0.00|||79.47||0.00|||-8.55| +2451855|63096|6089|26541|1312274|54|8187|10|17|2338|56|94.36|177.39|24.83|361.52|1390.48|5284.16|9933.84|92.60|361.52|1028.96|1121.56|-4255.20| +2451855|63096|16841|26541|1312274|54|8187|10|241|2338|14|46.05|70.91|53.89|0.00|754.46|644.70|992.74|67.90|0.00|754.46|822.36|109.76| +2451855|63096|12689|26541|1312274|54|8187|10|214|2338|41|52.15|61.53|31.99|0.00|1311.59|2138.15|2522.73|65.57|0.00|1311.59|1377.16|-826.56| +2451855|63096|4825|26541|1312274|54|8187|10|175|2338|32|97.23|127.37|48.40|0.00|1548.80|3111.36|4075.84|92.92|0.00|1548.80|1641.72|-1562.56| +2451855|63096|17018|26541|1312274|54|8187|10|298|2338|3|100.00|152.00|47.12|62.19|141.36|300.00|456.00|0.79|62.19|79.17|79.96|-220.83| +2451855|63096|15128|26541|1312274|54|8187|10|120|2338|42|37.20|46.87|25.30|0.00|1062.60|1562.40|1968.54|31.87|0.00|1062.60|1094.47|-499.80| +2451855|63096|2437|26541|1312274|54|8187|10|217|2338|58|1.12|1.24|0.80|0.00|46.40|64.96|71.92|1.85|0.00|46.40|48.25|-18.56| +2452312||13830|41085||4145||8|48|2339|44|41.61||15.70|0.00|690.80|1830.84|||0.00|||| +2452312|36690|11901|41085|1838615|4145|13275|8|32|2339|86|2.76|5.38|4.35|0.00|374.10|237.36|462.68|18.70|0.00|374.10|392.80|136.74| +2452312|36690|11539|41085|1838615|4145|13275|8|233|2339|51|36.93|55.76|28.99|0.00|1478.49|1883.43|2843.76|73.92|0.00|1478.49|1552.41|-404.94| +2452312|36690|12306|41085|1838615|4145|13275|8|66|2339|38|56.04|86.30|39.69|0.00|1508.22|2129.52|3279.40|15.08|0.00|1508.22|1523.30|-621.30| +2452312|36690|2406|41085|1838615|4145|13275|8|158|2339|91|62.03|76.29|6.10|0.00|555.10|5644.73|6942.39|27.75|0.00|555.10|582.85|-5089.63| +2452312|36690|1033|41085|1838615|4145|13275|8|86|2339|14|57.90|82.21|29.59|0.00|414.26|810.60|1150.94|4.14|0.00|414.26|418.40|-396.34| +2452312|36690|8166|41085|1838615|4145|13275|8|174|2339|63|43.90|57.50|52.32|0.00|3296.16|2765.70|3622.50|230.73|0.00|3296.16|3526.89|530.46| +2452312|36690|10575|41085|1838615|4145|13275|8|3|2339|88|62.47|71.21|48.42|0.00|4260.96|5497.36|6266.48|298.26|0.00|4260.96|4559.22|-1236.40| +2452312|36690|8538|41085|1838615|4145|13275|8|197|2339|90|53.05|61.00|18.91|0.00|1701.90|4774.50|5490.00|51.05|0.00|1701.90|1752.95|-3072.60| +2452312|36690|2955|41085|1838615|4145|13275|8|250|2339|6|83.38|122.56|24.51|8.82|147.06|500.28|735.36|2.76|8.82|138.24|141.00|-362.04| +2452312|36690|7134|41085|1838615|4145|13275|8|37|2339|56|55.02|55.02|17.60|0.00|985.60|3081.12|3081.12|9.85|0.00|985.60|995.45|-2095.52| +2452312|36690|11647|41085|1838615|4145|13275|8|92|2339|81|99.00|163.35|106.17|0.00|8599.77|8019.00|13231.35|85.99|0.00|8599.77|8685.76|580.77| +2452312|36690|15637|41085|1838615|4145|13275|8|212|2339|38|99.71|189.44|164.81|0.00|6262.78|3788.98|7198.72|375.76|0.00|6262.78|6638.54|2473.80| +2452312|36690|853|41085|1838615|4145|13275|8|205|2339|68|26.29|39.43|14.19|0.00|964.92|1787.72|2681.24|38.59|0.00|964.92|1003.51|-822.80| +2452312|36690|2041|41085|1838615|4145|13275|8|16|2339|68|15.76|26.16|5.23|224.05|355.64|1071.68|1778.88|0.00|224.05|131.59|131.59|-940.09| +2452312|36690|5143|41085|1838615|4145|13275|8|88|2339|38|26.43|43.08|16.37|0.00|622.06|1004.34|1637.04|24.88|0.00|622.06|646.94|-382.28| +2451862|66807|10307|23193|246153|5113|10026|7|139|2340|31|35.64|57.38|55.65|0.00|1725.15|1104.84|1778.78|34.50|0.00|1725.15|1759.65|620.31| +2451862|66807|4031|23193|246153|5113|10026|7|16|2340|12|32.10|32.42|32.09|0.00|385.08|385.20|389.04|11.55|0.00|385.08|396.63|-0.12| +2451862|66807|17462|23193|246153|5113|10026|7|198|2340|95|16.04|26.94|19.66|0.00|1867.70|1523.80|2559.30|149.41|0.00|1867.70|2017.11|343.90| +2451862|66807|16244|23193|246153|5113|10026|7|216|2340|12|70.53|112.14|54.94|382.38|659.28|846.36|1345.68|22.15|382.38|276.90|299.05|-569.46| +2451862|66807|10424|23193|246153|5113|10026|7|246|2340|69|66.02|108.93|52.28|541.09|3607.32|4555.38|7516.17|30.66|541.09|3066.23|3096.89|-1489.15| +2451862|66807|15608|23193|246153|5113|10026|7|139|2340|16|2.17|3.05|2.56|13.92|40.96|34.72|48.80|2.43|13.92|27.04|29.47|-7.68| +2451862|66807|9253|23193|246153|5113|10026|7|293|2340|88|39.85|74.51|12.66|133.68|1114.08|3506.80|6556.88|49.02|133.68|980.40|1029.42|-2526.40| +2451862|66807|17501|23193|246153|5113|10026|7|132|2340|52|71.49|117.95|23.59|0.00|1226.68|3717.48|6133.40|110.40|0.00|1226.68|1337.08|-2490.80| +2451862|66807|289|23193|246153|5113|10026|7|249|2340|71|96.25|96.25|90.47|0.00|6423.37|6833.75|6833.75|449.63|0.00|6423.37|6873.00|-410.38| +2451862|66807|9992|23193|246153|5113|10026|7|81|2340|26|11.89|21.63|3.24|0.00|84.24|309.14|562.38|7.58|0.00|84.24|91.82|-224.90| +||16987|23193||5113||||2340||||||528.20|841.60|1354.80|1.26||||| +2451862|66807|9677|23193|246153|5113|10026|7|4|2340|75|48.94|75.36|45.96|0.00|3447.00|3670.50|5652.00|310.23|0.00|3447.00|3757.23|-223.50| +2451862|66807|9271|23193|246153|5113|10026|7|47|2340|64|42.84|59.97|35.98|0.00|2302.72|2741.76|3838.08|115.13|0.00|2302.72|2417.85|-439.04| +2451862|66807|16988|23193|246153|5113|10026|7|285|2340|43|8.48|13.56|8.40|332.30|361.20|364.64|583.08|0.86|332.30|28.90|29.76|-335.74| +2451668|71824|5690|93845|51846|1047|2201|2|110|2341|93|83.03|161.90|11.33|316.10|1053.69|7721.79|15056.70|22.12|316.10|737.59|759.71|-6984.20| +2451668|71824|15266|93845|51846|1047|2201|2|258|2341|96|60.22|115.02|11.50|0.00|1104.00|5781.12|11041.92|88.32|0.00|1104.00|1192.32|-4677.12| +2451668|71824|16531|93845|51846|1047|2201|2|175|2341|79|68.01|130.57|112.29|0.00|8870.91|5372.79|10315.03|354.83|0.00|8870.91|9225.74|3498.12| +2451668|71824|7483|93845|51846|1047|2201|2|261|2341|32|76.06|148.31|130.51|0.00|4176.32|2433.92|4745.92|334.10|0.00|4176.32|4510.42|1742.40| +2451668|71824|14219|93845|51846|1047|2201|2|251|2341|100|55.19|95.47|11.45|0.00|1145.00|5519.00|9547.00|0.00|0.00|1145.00|1145.00|-4374.00| +2451668|71824|11357|93845|51846|1047|2201|2|291|2341|11|51.49|75.69|73.41|0.00|807.51|566.39|832.59|48.45|0.00|807.51|855.96|241.12| +2451668|71824|1790|93845|51846|1047|2201|2|131|2341|31|65.66|120.15|79.29|2310.51|2457.99|2035.46|3724.65|11.79|2310.51|147.48|159.27|-1887.98| +2451668|71824|3571|93845|51846|1047|2201|2|17|2341|7|14.43|20.34|8.94|0.00|62.58|101.01|142.38|0.00|0.00|62.58|62.58|-38.43| +2451668|71824|13879|93845|51846|1047|2201|2|25|2341|79|34.49|48.63|15.56|0.00|1229.24|2724.71|3841.77|110.63|0.00|1229.24|1339.87|-1495.47| +2451668|71824|14113|93845|51846|1047|2201|2|40|2341|8|33.88|45.73|21.03|111.03|168.24|271.04|365.84|2.28|111.03|57.21|59.49|-213.83| +2451668|71824|16727|93845|51846|1047|2201|2|180|2341|11|54.06|81.09|18.65|0.00|205.15|594.66|891.99|12.30|0.00|205.15|217.45|-389.51| +2451668|71824|16549|93845|51846|1047|2201|2|9|2341|15|58.88|70.06|35.03|0.00|525.45|883.20|1050.90|10.50|0.00|525.45|535.95|-357.75| +2451668|71824|16904|93845|51846|1047|2201|2|10|2341|81|33.48|51.22|47.12|0.00|3816.72|2711.88|4148.82|76.33|0.00|3816.72|3893.05|1104.84| +2451668|71824|1969|93845|51846|1047|2201|2|289|2341|98|82.64|135.52|111.12|10236.37|10889.76|8098.72|13280.96|19.60|10236.37|653.39|672.99|-7445.33| +2451668|71824|3770|93845|51846|1047|2201|2|160|2341|33|85.83|115.87|4.63|0.00|152.79|2832.39|3823.71|3.05|0.00|152.79|155.84|-2679.60| +2452630|47094|14118|88462|646434|4406|7978|4|248|2342|56|71.72|129.09|2.58|0.00|144.48|4016.32|7229.04|5.77|0.00|144.48|150.25|-3871.84| +2452630|47094|16381|88462|646434|4406|7978|4|99|2342|51|72.86|83.06|30.73|0.00|1567.23|3715.86|4236.06|47.01|0.00|1567.23|1614.24|-2148.63| +2452630|47094|11820|88462|646434|4406|7978|4|10|2342|34|52.71|99.62|53.79|0.00|1828.86|1792.14|3387.08|73.15|0.00|1828.86|1902.01|36.72| +2452630|47094|17700|88462|646434|4406|7978|4|290|2342|35|52.96|68.31|59.42|0.00|2079.70|1853.60|2390.85|41.59|0.00|2079.70|2121.29|226.10| +2452630|47094|631|88462|646434|4406|7978|4|126|2342|23|9.35|10.00|7.00|0.00|161.00|215.05|230.00|14.49|0.00|161.00|175.49|-54.05| +2452630|47094|12705|88462|646434|4406|7978|4|290|2342|58|91.11|131.19|116.75|0.00|6771.50|5284.38|7609.02|406.29|0.00|6771.50|7177.79|1487.12| +2452630|47094|11319|88462|646434|4406|7978|4|64|2342|100|93.47|130.85|109.91|0.00|10991.00|9347.00|13085.00|769.37|0.00|10991.00|11760.37|1644.00| +2452630|47094|13051|88462|646434|4406|7978|4|168|2342|5|96.94|158.98|6.35|29.52|31.75|484.70|794.90|0.11|29.52|2.23|2.34|-482.47| +2452630|47094|342|88462|646434|4406|7978|4|28|2342|44|72.06|125.38|67.70|0.00|2978.80|3170.64|5516.72|119.15|0.00|2978.80|3097.95|-191.84| +2452630|47094|16812|88462|646434|4406|7978|4|262|2342|33|22.51|26.33|10.79|0.00|356.07|742.83|868.89|10.68|0.00|356.07|366.75|-386.76| +2451819|70336|13787|48121|1355678|5988|34698|8|74|2343|99|76.05|104.18|6.25|0.00|618.75|7528.95|10313.82|0.00|0.00|618.75|618.75|-6910.20| +2451819|70336|7946|48121|1355678|5988|34698|8|244|2343|46|25.36|30.68|16.56|0.00|761.76|1166.56|1411.28|53.32|0.00|761.76|815.08|-404.80| +2451819|70336|3437|48121|1355678|5988|34698|8|246|2343|85|37.18|72.50|3.62|0.00|307.70|3160.30|6162.50|21.53|0.00|307.70|329.23|-2852.60| +2451819|70336|16076|48121|1355678|5988|34698|8|88|2343|93|79.98|107.97|14.03|0.00|1304.79|7438.14|10041.21|117.43|0.00|1304.79|1422.22|-6133.35| +2451819|70336|5833|48121|1355678|5988|34698|8|266|2343|9|36.38|50.20|5.02|8.13|45.18|327.42|451.80|2.22|8.13|37.05|39.27|-290.37| +2451819|70336|17189|48121|1355678|5988|34698|8|221|2343|41|70.51|77.56|65.15|0.00|2671.15|2890.91|3179.96|0.00|0.00|2671.15|2671.15|-219.76| +2451819|70336|11383|48121|1355678|5988|34698|8|190|2343|16|57.39|97.56|69.26|443.26|1108.16|918.24|1560.96|39.89|443.26|664.90|704.79|-253.34| +2451819|70336|9877|48121|1355678|5988|34698|8|107|2343|29|66.96|87.04|36.55|0.00|1059.95|1941.84|2524.16|63.59|0.00|1059.95|1123.54|-881.89| +||5210|||5988|34698|8|182|2343|88||191.01|||16639.92|||499.19||||7930.56| +2451819|70336|14059|48121|1355678|5988|34698|8|182|2343|71|41.64|78.28|30.52|0.00|2166.92|2956.44|5557.88|173.35|0.00|2166.92|2340.27|-789.52| +2451819|70336|10511|48121|1355678|5988|34698|8|235|2343|32|17.37|23.44|16.64|0.00|532.48|555.84|750.08|37.27|0.00|532.48|569.75|-23.36| +2451815|50482|3025||1349410|5730|43297|||2344||58.84|68.25|||0.00|4824.88|5596.50|||||-4824.88| +2451815|50482|877|97443|1349410|5730|43297|8|127|2344|36|91.52|95.18|92.32|1329.40|3323.52|3294.72|3426.48|139.58|1329.40|1994.12|2133.70|-1300.60| +|50482|17089||1349410||43297||214|2344|19||32.08|||225.34||609.52|||225.34||| +2451815|50482|812|97443|1349410|5730|43297|8|149|2344|57|91.80|162.48|125.10|784.37|7130.70|5232.60|9261.36|0.00|784.37|6346.33|6346.33|1113.73| +2451815|50482|6020|97443|1349410|5730|43297|8|118|2344|30|46.26|82.34|32.93|0.00|987.90|1387.80|2470.20|9.87|0.00|987.90|997.77|-399.90| +2451815|50482|2531|97443|1349410|5730|43297|8|108|2344|77|43.32|56.74|43.68|0.00|3363.36|3335.64|4368.98|67.26|0.00|3363.36|3430.62|27.72| +2451815|50482|16142||1349410|5730|||125|2344||91.65|144.80|18.82|0.00||||70.76|0.00|884.54|955.30|| +2451815|50482|9535|97443|1349410|5730|43297|8|211|2344|38|78.23|148.63|120.39|2241.66|4574.82|2972.74|5647.94|69.99|2241.66|2333.16|2403.15|-639.58| +2451815|50482|1939|97443|1349410|5730|43297|8|122|2344|86|64.07|76.88|13.06|0.00|1123.16|5510.02|6611.68|0.00|0.00|1123.16|1123.16|-4386.86| +2451815|50482|10082|97443|1349410|5730|43297|8|205|2344|95|20.02|29.42|3.23|0.00|306.85|1901.90|2794.90|18.41|0.00|306.85|325.26|-1595.05| +2451815|50482|14615|97443|1349410|5730|43297|8|43|2344|85|86.20|151.71|121.36|0.00|10315.60|7327.00|12895.35|412.62|0.00|10315.60|10728.22|2988.60| +2451815|50482|6185|97443|1349410|5730|43297|8|152|2344|14|68.57|127.54|90.55|849.35|1267.70|959.98|1785.56|37.65|849.35|418.35|456.00|-541.63| +2451815|50482|2462|97443|1349410|5730|43297|8|148|2344|6|21.34|28.80|19.87|0.00|119.22|128.04|172.80|3.57|0.00|119.22|122.79|-8.82| +2451815|50482|6425|97443|1349410|5730|43297|8|6|2344|1|18.59|28.62|26.90|0.26|26.90|18.59|28.62|0.53|0.26|26.64|27.17|8.05| +2452268|48542|14941|4362|1749912|3462|39012|1|15|2345|90|35.85|53.05|13.26|0.00|1193.40|3226.50|4774.50|35.80|0.00|1193.40|1229.20|-2033.10| +2452268|48542|11637|4362|1749912|3462|39012|1|151|2345|60|27.20|44.06|13.21|301.18|792.60|1632.00|2643.60|29.48|301.18|491.42|520.90|-1140.58| +2452268|48542|2501|4362|1749912|3462|39012|1|114|2345|26|75.80|138.71|38.83|0.00|1009.58|1970.80|3606.46|40.38|0.00|1009.58|1049.96|-961.22| +2452268|48542|14485|4362|1749912|3462|39012|1|264|2345|2|59.40|62.96|4.40|0.00|8.80|118.80|125.92|0.17|0.00|8.80|8.97|-110.00| +2452268|48542|5899|4362|1749912|3462|39012|1|112|2345|9|18.63|31.11|4.35|0.00|39.15|167.67|279.99|1.95|0.00|39.15|41.10|-128.52| +2452268|48542|8989|4362|1749912|3462|39012|1|34|2345|26|69.30|135.13|105.40|0.00|2740.40|1801.80|3513.38|82.21|0.00|2740.40|2822.61|938.60| +2452268|48542|1061|4362|1749912|3462|39012|1|90|2345|2|91.55|103.45|41.38|0.00|82.76|183.10|206.90|4.13|0.00|82.76|86.89|-100.34| +2452268|48542|5639|4362|1749912|3462|39012|1|82|2345|71|37.07|43.00|29.67|1643.12|2106.57|2631.97|3053.00|13.90|1643.12|463.45|477.35|-2168.52| +2452268|48542|17009|4362|1749912|3462|39012|1|60|2345|98|87.49|164.48|74.01|6817.80|7252.98|8574.02|16119.04|8.70|6817.80|435.18|443.88|-8138.84| +2452268|48542|15029|4362|1749912|3462|39012|1|284|2345|99|87.15|165.58|19.86|0.00|1966.14|8627.85|16392.42|58.98|0.00|1966.14|2025.12|-6661.71| +2452268|48542|12111|4362|1749912|3462|39012|1|175|2345|11|58.73|74.58|11.93|98.42|131.23|646.03|820.38|0.98|98.42|32.81|33.79|-613.22| +2451519|34446|10171|8055|1704655|2855|12608|7|150|2346|52|65.45|79.19|52.26|1956.61|2717.52|3403.40|4117.88|60.87|1956.61|760.91|821.78|-2642.49| +2451519|34446|14080|8055|1704655|2855|12608|7|251|2346|54|90.56|149.42|92.64|0.00|5002.56|4890.24|8068.68|450.23|0.00|5002.56|5452.79|112.32| +2451519||11654|||||7|276|2346|||||||9100.80|9282.24|||6125.76|6554.56|| +2451519|34446|14468|8055|1704655|2855|12608|7|83|2346|38|10.77|17.77|4.62|0.00|175.56|409.26|675.26|8.77|0.00|175.56|184.33|-233.70| +|34446|13300|8055|1704655||12608|7|3|2346|64||80.14||0.00|||||0.00||1346.45|-3372.80| +2451519|34446|544|8055|1704655|2855|12608|7|28|2346|90|69.77|108.84|105.57|0.00|9501.30|6279.30|9795.60|0.00|0.00|9501.30|9501.30|3222.00| +2451519|34446|955|8055|1704655|2855|12608|7|280|2346|80|6.27|11.09|1.77|0.00|141.60|501.60|887.20|5.66|0.00|141.60|147.26|-360.00| +2451519|34446|7034|8055|1704655|2855|12608|7|178|2346|63|26.87|40.84|18.78|295.78|1183.14|1692.81|2572.92|26.62|295.78|887.36|913.98|-805.45| +2451519|34446|5353|8055|1704655|2855|12608|7|238|2346|48|7.67|12.73|5.21|0.00|250.08|368.16|611.04|10.00|0.00|250.08|260.08|-118.08| +2451519|34446|1798|8055|1704655|2855|12608|7|185|2346|97|32.35|49.49|7.42|0.00|719.74|3137.95|4800.53|28.78|0.00|719.74|748.52|-2418.21| +2451519|34446|10858|8055|1704655|2855|12608|7|140|2346|46|47.84|89.46|23.25|866.29|1069.50|2200.64|4115.16|6.09|866.29|203.21|209.30|-1997.43| +2451729||5840|48661|1864537||3042|||2347|||63.35|44.97||2518.32||3547.60|226.64||2518.32|2744.96|568.96| +2451729|37802|6241|48661|1864537|2366|3042|8|197|2347|30|97.13|170.94|165.81|0.00|4974.30|2913.90|5128.20|49.74|0.00|4974.30|5024.04|2060.40| +2451729|37802|15709|48661|1864537|2366|3042|8|273|2347|60|95.40|114.48|40.06|0.00|2403.60|5724.00|6868.80|144.21|0.00|2403.60|2547.81|-3320.40| +|37802|9206||1864537||3042|8|178|2347||22.02||20.09|0.00||638.58|1142.89||0.00|||| +2451729|37802|13334|48661|1864537|2366|3042|8|259|2347|83|61.61|97.95|69.54|0.00|5771.82|5113.63|8129.85|519.46|0.00|5771.82|6291.28|658.19| +2451729|37802|12482|48661|1864537|2366|3042|8|260|2347|49|30.13|31.93|18.20|0.00|891.80|1476.37|1564.57|80.26|0.00|891.80|972.06|-584.57| +2451729|37802|12443|48661|1864537|2366|3042|8|43|2347|62|35.25|44.06|3.96|0.00|245.52|2185.50|2731.72|19.64|0.00|245.52|265.16|-1939.98| +2451729|37802|15599|48661|1864537|2366|3042|8|23|2347|47|98.30|123.85|74.31|1327.17|3492.57|4620.10|5820.95|21.65|1327.17|2165.40|2187.05|-2454.70| +2451729|37802|4526|48661|1864537|2366|3042|8|35|2347|52|94.57|156.98|43.95|0.00|2285.40|4917.64|8162.96|114.27|0.00|2285.40|2399.67|-2632.24| +2451729|37802|10957|48661|1864537|2366|3042|8|230|2347|7|42.92|69.53|63.27|115.15|442.89|300.44|486.71|9.83|115.15|327.74|337.57|27.30| +2451729|37802|314|48661|1864537|2366|3042|8|295|2347|58|92.76|175.31|124.47|0.00|7219.26|5380.08|10167.98|144.38|0.00|7219.26|7363.64|1839.18| +2451729|37802|4652|48661|1864537|2366|3042|8|148|2347|30|74.38|110.08|26.41|0.00|792.30|2231.40|3302.40|15.84|0.00|792.30|808.14|-1439.10| +2451729|37802|2545|48661|1864537|2366|3042|8|136|2347|40|85.80|109.82|98.83|0.00|3953.20|3432.00|4392.80|39.53|0.00|3953.20|3992.73|521.20| +2452571|66594|16623|51174|1587749|4324|39299|7|268|2348|93|91.53|123.56|51.89|2412.88|4825.77|8512.29|11491.08|48.25|2412.88|2412.89|2461.14|-6099.40| +2452571|66594|16260|51174|1587749|4324|39299|7|3|2348|5|23.57|31.11|22.08|0.00|110.40|117.85|155.55|5.52|0.00|110.40|115.92|-7.45| +2452571||2977|51174|1587749|4324|39299|||2348|15|74.83|||||1122.45||6.45||71.70||| +||14433|||||7|4|2348|75|37.86|71.93|||3236.25||5394.75|194.17||3236.25|3430.42|396.75| +2452571|66594|13716|51174|1587749|4324|39299|7|141|2348|35|34.19|60.51|13.31|149.07|465.85|1196.65|2117.85|12.67|149.07|316.78|329.45|-879.87| +2452571|66594|7515|51174|1587749|4324|39299|7|2|2348|19|81.42|162.02|147.43|0.00|2801.17|1546.98|3078.38|0.00|0.00|2801.17|2801.17|1254.19| +2452571|66594|11334|51174|1587749|4324|39299|7|160|2348|32|34.08|52.82|18.48|0.00|591.36|1090.56|1690.24|47.30|0.00|591.36|638.66|-499.20| +2452571|66594|13993|51174|1587749|4324|39299|7|19|2348|40|20.63|37.34|35.84|0.00|1433.60|825.20|1493.60|100.35|0.00|1433.60|1533.95|608.40| +2452571|66594|5581|51174|1587749|4324|39299|7|38|2348|80|53.78|82.82|49.69|0.00|3975.20|4302.40|6625.60|318.01|0.00|3975.20|4293.21|-327.20| +2452571|66594|3852|51174|1587749|4324|39299|7|150|2348|84|73.78|77.46|46.47|0.00|3903.48|6197.52|6506.64|312.27|0.00|3903.48|4215.75|-2294.04| +2452571|66594|15282|51174|1587749|4324|39299|7|18|2348|25|84.59|147.18|14.71|261.10|367.75|2114.75|3679.50|0.00|261.10|106.65|106.65|-2008.10| +2452571|66594|1341|51174|1587749|4324|39299||216|2348|84|56.32||63.93|0.00||||0.00|0.00|||| +2452571|66594|2748|51174|1587749|4324|39299|7|22|2348|80|69.04|131.17|94.44|0.00|7555.20|5523.20|10493.60|75.55|0.00|7555.20|7630.75|2032.00| +2452571|66594|17335|51174|1587749|4324|39299|7|164|2348|18|38.15|64.85|3.89|0.00|70.02|686.70|1167.30|0.00|0.00|70.02|70.02|-616.68| +2452043|67127|6095|26025|1826438|3204|22017|8|48|2349|75|12.70|14.22|8.53|0.00|639.75|952.50|1066.50|12.79|0.00|639.75|652.54|-312.75| +2452043|67127|9525|26025|1826438|3204|22017|8|111|2349|18|45.13|66.34|53.07|0.00|955.26|812.34|1194.12|57.31|0.00|955.26|1012.57|142.92| +2452043|67127|8281|26025|1826438|3204|22017|8|178|2349|86|86.84|165.86|6.63|0.00|570.18|7468.24|14263.96|34.21|0.00|570.18|604.39|-6898.06| +2452043|67127|7125|26025|1826438|3204|22017|8|82|2349|98|2.00|2.62|1.17|0.00|114.66|196.00|256.76|5.73|0.00|114.66|120.39|-81.34| +2452043|67127|17967|26025|1826438|3204|22017|8|156|2349|36|75.14|109.70|8.77|0.00|315.72|2705.04|3949.20|28.41|0.00|315.72|344.13|-2389.32| +2452043|67127|17845|26025|1826438|3204|22017|8|255|2349|41|63.27|99.33|71.51|0.00|2931.91|2594.07|4072.53|146.59|0.00|2931.91|3078.50|337.84| +2452043||15327||1826438|3204||8||2349||76.61||||2580.58|||103.22||2580.58|2683.80|-330.60| +2452043|67127|7937|26025|1826438|3204|22017|8|269|2349|29|75.08|124.63|4.98|0.00|144.42|2177.32|3614.27|10.10|0.00|144.42|154.52|-2032.90| +2452043|67127|9523|26025|1826438|3204|22017|8|118|2349|77|24.21|26.14|3.65|151.76|281.05|1864.17|2012.78|1.29|151.76|129.29|130.58|-1734.88| +2452043|67127|16787|26025|1826438|3204|22017|8|43|2349|82|24.78|32.46|4.21|0.00|345.22|2031.96|2661.72|20.71|0.00|345.22|365.93|-1686.74| +2452043|67127|4851|26025|1826438|3204|22017|8|152|2349|38|9.37|10.58|8.56|42.28|325.28|356.06|402.04|0.00|42.28|283.00|283.00|-73.06| +2451153|38537|8266|92181|1186228|4628|21475|4|11|2350|31|93.30|170.73|13.65|253.89|423.15|2892.30|5292.63|8.46|253.89|169.26|177.72|-2723.04| +2451153|38537|7844|92181|1186228|4628|21475|4|240|2350|81|19.09|26.53|23.87|0.00|1933.47|1546.29|2148.93|96.67|0.00|1933.47|2030.14|387.18| +2451153|38537|12622|92181|1186228|4628|21475|4|286|2350|36|20.51|23.38|14.96|0.00|538.56|738.36|841.68|48.47|0.00|538.56|587.03|-199.80| +2451153|38537|3284|92181|1186228|4628|21475|4|62|2350|14|93.53|114.10|78.72|495.93|1102.08|1309.42|1597.40|42.43|495.93|606.15|648.58|-703.27| +2451153|38537|16256|92181|1186228|4628|21475|4|219|2350|2|77.23|94.99|83.59|0.00|167.18|154.46|189.98|3.34|0.00|167.18|170.52|12.72| +2451153|38537|10579|92181|1186228|4628|21475|4|210|2350|13|43.17|76.41|5.34|0.00|69.42|561.21|993.33|5.55|0.00|69.42|74.97|-491.79| +2451153|38537|14401|92181|1186228|4628|21475|4|125|2350|91|83.17|143.88|109.34|0.00|9949.94|7568.47|13093.08|397.99|0.00|9949.94|10347.93|2381.47| +2451153|38537|16621|92181|1186228|4628|21475|4|169|2350|20|46.25|62.43|16.85|0.00|337.00|925.00|1248.60|10.11|0.00|337.00|347.11|-588.00| +2451153|38537|14494|92181|1186228|4628|21475|4|103|2350|47|42.46|52.65|49.49|0.00|2326.03|1995.62|2474.55|69.78|0.00|2326.03|2395.81|330.41| +2451153|38537|10358|92181|1186228|4628|21475|4|276|2350|47|81.38|130.20|117.18|0.00|5507.46|3824.86|6119.40|220.29|0.00|5507.46|5727.75|1682.60| +2451153|38537|6778|92181|1186228|4628|21475|4|167|2350|56|96.44|168.77|165.39|0.00|9261.84|5400.64|9451.12|648.32|0.00|9261.84|9910.16|3861.20| +2451153|38537|4804|92181|1186228|4628|21475|4|280|2350|82|88.98|104.99|10.49|0.00|860.18|7296.36|8609.18|68.81|0.00|860.18|928.99|-6436.18| +2451907|68294|4435|47150|224087|2271|44408|2|8|2351|54|75.00|126.00|3.78|97.97|204.12|4050.00|6804.00|1.06|97.97|106.15|107.21|-3943.85| +2451907|68294|3182|47150|224087|2271|44408|||2351|79|70.32|127.27||||5555.28|10054.33|||||2789.49| +2451907|68294|1268|47150|224087|2271|44408|2|144|2351|25|61.60|88.08|73.10|1571.65|1827.50|1540.00|2202.00|20.46|1571.65|255.85|276.31|-1284.15| +||12887||224087|2271|44408|2||2351||15.07|||0.00||482.24|||0.00||309.16|-182.08| +2451907|68294|13610|47150|224087|2271|44408|2|137|2351|52|7.42|10.98|3.84|0.00|199.68|385.84|570.96|15.97|0.00|199.68|215.65|-186.16| +2451907|68294|13361|47150|224087|2271|44408|2|299|2351|27|36.64|39.93|1.99|0.00|53.73|989.28|1078.11|2.14|0.00|53.73|55.87|-935.55| +2451907|68294|16297|47150|224087|2271|44408|2|228|2351|36|1.28|2.21|1.67|0.00|60.12|46.08|79.56|4.20|0.00|60.12|64.32|14.04| +||16481|47150|224087||44408|||2351|5|40.04|43.24||0.00||200.20|216.20||0.00||148.93|-59.70| +2452246|61362|11179|11483|269988|3693|49845|7|184|2352|1|25.29|36.16|26.03|0.00|26.03|25.29|36.16|1.56|0.00|26.03|27.59|0.74| +2452246|61362|8387||269988|3693|49845||217|2352||||||968.25|||||958.57||-1551.68| +2452246|61362|17735|11483|269988|3693|49845|7|260|2352|91|4.41|4.45|1.60|0.00|145.60|401.31|404.95|1.45|0.00|145.60|147.05|-255.71| +2452246|61362|17567|11483|269988|3693|49845|7|256|2352|55|5.70|9.80|0.78|23.59|42.90|313.50|539.00|1.35|23.59|19.31|20.66|-294.19| +||9353||||||169|2352|83||111.79|107.31|||||801.60||8906.73|9708.33|| +2452246|61362|13293|11483|269988|3693|49845|7|228|2352|81|60.96|117.04|104.16|0.00|8436.96|4937.76|9480.24|759.32|0.00|8436.96|9196.28|3499.20| +2452246|61362|17721|11483|269988|3693|49845|7|6|2352|71|34.90|63.86|13.41|0.00|952.11|2477.90|4534.06|66.64|0.00|952.11|1018.75|-1525.79| +2452246|61362|5911|11483|269988|3693|49845|7|78|2352|17|43.31|74.92|74.92|853.33|1273.64|736.27|1273.64|16.81|853.33|420.31|437.12|-315.96| +2452246|61362|16475|11483|269988|3693|49845|7|44|2352|69|32.25|40.31|1.61|55.54|111.09|2225.25|2781.39|2.77|55.54|55.55|58.32|-2169.70| +2452246|61362|17175|11483|269988|3693|49845|7|45|2352|46|70.39|88.69|42.57|0.00|1958.22|3237.94|4079.74|137.07|0.00|1958.22|2095.29|-1279.72| +2452246|61362|115|11483|269988|3693|49845|7|297|2352|58|61.89|118.20|112.29|0.00|6512.82|3589.62|6855.60|455.89|0.00|6512.82|6968.71|2923.20| +2452246|61362|12895|11483|269988|3693|49845|7|128|2352|57|12.48|23.58|6.60|0.00|376.20|711.36|1344.06|30.09|0.00|376.20|406.29|-335.16| +2452246||13265||269988|||7|147|2352|69|||36.66||2529.54|5551.74||||||-3022.20| +2452246|61362|7225|11483|269988|3693|49845|7|38|2352|21|17.61|28.70|24.39|0.00|512.19|369.81|602.70|0.00|0.00|512.19|512.19|142.38| +2452246|61362|16737|11483|269988|3693|49845|7|69|2352|78|93.18|165.86|28.19|0.00|2198.82|7268.04|12937.08|153.91|0.00|2198.82|2352.73|-5069.22| +2450992|46880|16423|98838|577195|2752|30662|1|241|2353|70|26.83|46.14|17.99|176.30|1259.30|1878.10|3229.80|0.00|176.30|1083.00|1083.00|-795.10| +2450992|46880|10807|98838|577195|2752|30662|1|291|2353|25|64.10|108.32|74.74|0.00|1868.50|1602.50|2708.00|168.16|0.00|1868.50|2036.66|266.00| +2450992|46880|15808|98838|577195|2752|30662|1|11|2353|75|51.74|81.23|63.35|0.00|4751.25|3880.50|6092.25|142.53|0.00|4751.25|4893.78|870.75| +2450992|46880|1696|98838|577195|2752|30662|1|7|2353|42|84.59|148.87|8.93|266.29|375.06|3552.78|6252.54|8.70|266.29|108.77|117.47|-3444.01| +||16069||577195|2752|30662|||2353||44.63||20.34|0.00|1159.38|2543.91||34.78|0.00|1159.38|1194.16|-1384.53| +2450992|46880|9508|98838|577195|2752|30662|1|25|2353|60|89.95|172.70|165.79|6366.33|9947.40|5397.00|10362.00|179.05|6366.33|3581.07|3760.12|-1815.93| +2450992|46880|6895|98838|577195|2752|30662|1|223|2353|26|78.83|85.13|5.95|0.00|154.70|2049.58|2213.38|0.00|0.00|154.70|154.70|-1894.88| +2450992|46880|6484|98838|577195|2752|30662|1|183|2353|61|93.18|101.56|97.49|0.00|5946.89|5683.98|6195.16|297.34|0.00|5946.89|6244.23|262.91| +2451219|63834|9646|64675|563036|2644|6252|4|105|2354|100|29.02|40.33|7.25|464.00|725.00|2902.00|4033.00|7.83|464.00|261.00|268.83|-2641.00| +2451219|63834|308|64675|563036|2644|6252|4|201|2354|4|47.55|50.40|10.58|0.00|42.32|190.20|201.60|3.38|0.00|42.32|45.70|-147.88| +2451219|63834|15187|64675|563036|2644|6252|4|225|2354|87|18.52|29.44|25.02|0.00|2176.74|1611.24|2561.28|21.76|0.00|2176.74|2198.50|565.50| +2451219|63834|14413|64675|563036|2644|6252|4|73|2354|72|23.80|27.37|6.02|0.00|433.44|1713.60|1970.64|0.00|0.00|433.44|433.44|-1280.16| +2451219|63834|10628|64675|563036|2644|6252|4|99|2354|38|73.88|79.79|67.82|0.00|2577.16|2807.44|3032.02|206.17|0.00|2577.16|2783.33|-230.28| +2451219|63834|15164|64675|563036|2644|6252|4|137|2354|73|76.73|136.57|101.06|6492.09|7377.38|5601.29|9969.61|53.11|6492.09|885.29|938.40|-4716.00| +2451219|63834|17107|64675|563036|2644|6252|4|53|2354|82|76.18|113.50|48.80|0.00|4001.60|6246.76|9307.00|240.09|0.00|4001.60|4241.69|-2245.16| +2451219|63834|13106|64675|563036|2644|6252|4|161|2354|35|17.42|27.52|20.08|0.00|702.80|609.70|963.20|28.11|0.00|702.80|730.91|93.10| +2451219|63834|8302|64675|||6252|||2354|44||82.34|||3333.00|3019.28||199.98||||313.72| +2451219|63834|10012|64675|563036|2644|6252|4|81|2354|35|85.99|92.86|87.28|0.00|3054.80|3009.65|3250.10|122.19|0.00|3054.80|3176.99|45.15| +2451219|63834|15847|64675|563036|2644|6252|4|138|2354|63|57.47|62.64|16.28|0.00|1025.64|3620.61|3946.32|30.76|0.00|1025.64|1056.40|-2594.97| +2451219||776|64675|563036||||160|2354|46||42.18|14.76||678.96||1940.28|40.73||||-469.20| +2451219||1490|64675|||6252|||2354|34||75.99||278.48||||29.62|278.48|||| +2451165|39806|17222|11103|1694489|6322|17308|1|293|2355|83|47.40|47.40|44.08|0.00|3658.64|3934.20|3934.20|329.27|0.00|3658.64|3987.91|-275.56| +2451165|39806|4376|11103|1694489|6322|17308|1|217|2355|80|23.64|38.53|10.01|0.00|800.80|1891.20|3082.40|24.02|0.00|800.80|824.82|-1090.40| +2451165|39806|884|11103|1694489|6322|17308|1|166|2355|80|7.06|9.95|0.49|28.22|39.20|564.80|796.00|0.10|28.22|10.98|11.08|-553.82| +2451165|39806|4760|11103|1694489|6322|17308|1|43|2355|25|15.34|29.45|16.49|0.00|412.25|383.50|736.25|16.49|0.00|412.25|428.74|28.75| +2451165|39806|17372|11103|1694489|6322|17308|1|285|2355|23|47.43|64.97|6.49|100.01|149.27|1090.89|1494.31|2.95|100.01|49.26|52.21|-1041.63| +2451165|39806|17251|11103|1694489|6322|17308|1|164|2355|32|51.38|97.62|45.88|924.94|1468.16|1644.16|3123.84|5.43|924.94|543.22|548.65|-1100.94| +2451165|39806|13687|11103|1694489|6322|17308|1|259|2355|49|84.17|168.34|77.43|0.00|3794.07|4124.33|8248.66|303.52|0.00|3794.07|4097.59|-330.26| +2451165|39806|13840|11103|1694489|6322|17308|1|105|2355|77|90.84|115.36|26.53|0.00|2042.81|6994.68|8882.72|163.42|0.00|2042.81|2206.23|-4951.87| +2451165|39806|14002|11103|1694489|6322|17308|1|262|2355|60|86.19|114.63|11.46|0.00|687.60|5171.40|6877.80|34.38|0.00|687.60|721.98|-4483.80| +2451165|39806|17132|11103|1694489|6322|17308|1|21|2355|38|24.20|31.46|12.89|0.00|489.82|919.60|1195.48|34.28|0.00|489.82|524.10|-429.78| +2452147|42956|14249|70093|763403|6313|49623|1|38|2356|54|66.28|92.79|90.00|0.00|4860.00|3579.12|5010.66|145.80|0.00|4860.00|5005.80|1280.88| +2452147|42956|13505|70093|763403|6313|49623|1|255|2356|27|98.79|137.31|91.99|0.00|2483.73|2667.33|3707.37|173.86|0.00|2483.73|2657.59|-183.60| +2452147|42956|17855|70093|763403|6313|49623|1|176|2356|53|67.66|101.49|73.07|2594.71|3872.71|3585.98|5378.97|12.78|2594.71|1278.00|1290.78|-2307.98| +2452147|42956|3017|70093|763403|6313|49623|1|171|2356|53|11.19|22.04|15.64|0.00|828.92|593.07|1168.12|8.28|0.00|828.92|837.20|235.85| +2452147||4015|70093||||1||2356|68||81.21|80.39|0.00||||218.66|0.00|||490.96| +2452147|42956|4549|70093|763403|6313|49623|1|24|2356|16|83.70|107.13|22.49|0.00|359.84|1339.20|1714.08|3.59|0.00|359.84|363.43|-979.36| +2452147|42956|9249|70093|763403|6313|49623|1|298|2356|52|35.09|66.32|11.27|175.81|586.04|1824.68|3448.64|0.00|175.81|410.23|410.23|-1414.45| +2452147|42956|15357|70093|763403|6313|49623|1|68|2356|31|82.08|147.74|116.71|0.00|3618.01|2544.48|4579.94|325.62|0.00|3618.01|3943.63|1073.53| +2452215|47892|4569|2238|1788174|2910|48446|8|24|2357|64|23.57|25.21|15.37|0.00|983.68|1508.48|1613.44|68.85|0.00|983.68|1052.53|-524.80| +2452215|47892|2257|2238|1788174|2910|48446|8|42|2357|74|28.17|36.90|5.16|103.09|381.84|2084.58|2730.60|2.78|103.09|278.75|281.53|-1805.83| +2452215|47892|10665|2238|1788174|2910|48446|8|215|2357|75|74.43|99.73|26.92|0.00|2019.00|5582.25|7479.75|100.95|0.00|2019.00|2119.95|-3563.25| +2452215|47892|15445|2238|1788174|2910|48446|8|285|2357|36|53.14|104.68|90.02|0.00|3240.72|1913.04|3768.48|162.03|0.00|3240.72|3402.75|1327.68| +2452215|47892|4981|2238|1788174|2910|48446|8|278|2357|78|55.48|109.29|51.36|2684.07|4006.08|4327.44|8524.62|0.00|2684.07|1322.01|1322.01|-3005.43| +2452215|47892|17375|2238|1788174|2910|48446|8|110|2357|20|15.31|29.24|7.01|0.00|140.20|306.20|584.80|12.61|0.00|140.20|152.81|-166.00| +2452215|47892|16509|2238|1788174|2910|48446|8|127|2357|9|92.98|114.36|13.72|0.00|123.48|836.82|1029.24|11.11|0.00|123.48|134.59|-713.34| +2452215|47892|11827|2238|1788174|2910|48446|8|151|2357|2|63.31|110.79|57.61|0.00|115.22|126.62|221.58|1.15|0.00|115.22|116.37|-11.40| +2452215|47892|8895|2238|1788174|2910|48446|8|150|2357|25|51.02|93.36|5.60|0.00|140.00|1275.50|2334.00|5.60|0.00|140.00|145.60|-1135.50| +2452215|47892|10353|2238|1788174|2910|48446|8|234|2357|30|41.83|70.27|63.94|0.00|1918.20|1254.90|2108.10|172.63|0.00|1918.20|2090.83|663.30| +2451440|65024|11440|80133|1755647|408|7969|7|33|2358|47|89.61|175.63|56.20|0.00|2641.40|4211.67|8254.61|0.00|0.00|2641.40|2641.40|-1570.27| +2451440|65024|10756|80133|1755647|408|7969|7|44|2358|49|89.97|144.85|72.42|0.00|3548.58|4408.53|7097.65|35.48|0.00|3548.58|3584.06|-859.95| +2451440|65024|1304|80133|1755647|408|7969|7|124|2358|82|90.74|150.62|84.34|0.00|6915.88|7440.68|12350.84|622.42|0.00|6915.88|7538.30|-524.80| +2451440|65024|12748|80133|1755647|408|7969|7|35|2358|20|61.06|121.50|72.90|0.00|1458.00|1221.20|2430.00|102.06|0.00|1458.00|1560.06|236.80| +||11509|80133|||7969|7||2358|||25.40|7.36|0.00|507.84|942.54||45.70|0.00|||| +2451440|65024|16418|80133|1755647|408|7969|7|299|2358|33|63.71|65.62|54.46|0.00|1797.18|2102.43|2165.46|89.85|0.00|1797.18|1887.03|-305.25| +2451440|65024|13885|80133|1755647|408|7969|7|89|2358|100|60.39|105.68|64.46|6381.54|6446.00|6039.00|10568.00|1.93|6381.54|64.46|66.39|-5974.54| +2451440|65024|8167|80133|1755647|408|7969|7|79|2358|24|82.43|84.07|3.36|0.00|80.64|1978.32|2017.68|2.41|0.00|80.64|83.05|-1897.68| +2451440|65024|5830|80133|1755647|408|7969|7|117|2358|87|82.52|105.62|88.72|0.00|7718.64|7179.24|9188.94|385.93|0.00|7718.64|8104.57|539.40| +2451440|65024|16766|80133|1755647|408|7969|7|265|2358|84|64.57|93.62|5.61|414.69|471.24|5423.88|7864.08|5.08|414.69|56.55|61.63|-5367.33| +2451440|65024|8248|80133|1755647|408|7969|7|155|2358|24|90.15|105.47|64.33|0.00|1543.92|2163.60|2531.28|92.63|0.00|1543.92|1636.55|-619.68| +2451440|65024|5716|80133|1755647|408|7969|7|170|2358|53|49.34|89.79|80.81|0.00|4282.93|2615.02|4758.87|342.63|0.00|4282.93|4625.56|1667.91| +2451440|65024|986|80133|1755647|408|7969|7|61|2358|4|49.18|55.08|54.52|0.00|218.08|196.72|220.32|19.62|0.00|218.08|237.70|21.36| +2451440|65024|5275|80133|1755647|408|7969|7|69|2358|68|65.24|106.99|22.46|687.27|1527.28|4436.32|7275.32|42.00|687.27|840.01|882.01|-3596.31| +2450837|46915|8698|19990|133830|5717|32357|1|162|2359|10|63.55|80.07|69.66|160.21|696.60|635.50|800.70|0.00|160.21|536.39|536.39|-99.11| +2450837|46915|4426|19990|133830|5717|32357|1|212|2359|17|4.76|5.56|2.55|0.00|43.35|80.92|94.52|0.00|0.00|43.35|43.35|-37.57| +2450837|46915|5383|19990|133830|5717|32357|1|31|2359|56|97.84|147.73|31.02|0.00|1737.12|5479.04|8272.88|0.00|0.00|1737.12|1737.12|-3741.92| +2450837|46915|7828|19990|133830|5717|32357|1|114|2359|41|35.76|64.72|43.36|0.00|1777.76|1466.16|2653.52|35.55|0.00|1777.76|1813.31|311.60| +2450837|46915|12349|19990|133830|5717|32357|1|169|2359|87|62.95|112.05|7.84|0.00|682.08|5476.65|9748.35|40.92|0.00|682.08|723.00|-4794.57| +2450837|46915|3109|19990|133830|5717|32357|1|41|2359|82|71.86|86.95|86.95|0.00|7129.90|5892.52|7129.90|142.59|0.00|7129.90|7272.49|1237.38| +2450837|46915|2434|19990|133830|5717|32357|1|99|2359|13|11.12|11.23|7.97|0.00|103.61|144.56|145.99|7.25|0.00|103.61|110.86|-40.95| +2450837|46915|512|19990|133830|5717|32357|1|273|2359|88|52.60|54.70|6.01|0.00|528.88|4628.80|4813.60|10.57|0.00|528.88|539.45|-4099.92| +2450837|46915|5746|19990|133830|5717|32357|1|281|2359|48|23.77|24.00|0.24|0.00|11.52|1140.96|1152.00|0.57|0.00|11.52|12.09|-1129.44| +2450837|46915|13028|19990|133830|5717|32357|1|149|2359|34|34.11|56.96|10.25|0.00|348.50|1159.74|1936.64|3.48|0.00|348.50|351.98|-811.24| +|46915|15601|19990|133830||32357|1|125|2359||75.66||||4568.85|||38.83|||815.54|| +2450837|46915|14851|19990|133830|5717|32357|1|198|2359|9|88.21|99.67|63.78|0.00|574.02|793.89|897.03|28.70|0.00|574.02|602.72|-219.87| +2450837|46915|7786|19990|133830|5717|32357|1|188|2359|56|44.64|51.33|18.47|0.00|1034.32|2499.84|2874.48|82.74|0.00|1034.32|1117.06|-1465.52| +2450837|46915|9868|19990|133830|5717|32357|1|25|2359|2|46.48|62.28|13.07|0.00|26.14|92.96|124.56|1.30|0.00|26.14|27.44|-66.82| +2450837|46915|7318|19990|133830|5717|32357|1|131|2359|33|60.11|79.34|11.90|0.00|392.70|1983.63|2618.22|0.00|0.00|392.70|392.70|-1590.93| +2450837|46915|15091|19990|133830|5717|32357|1|129|2359|29|6.86|10.56|4.75|0.00|137.75|198.94|306.24|9.64|0.00|137.75|147.39|-61.19| +2452046|34230|10481|36581|582642|4703|45964|8|76|2360|68|33.49|65.30|28.07|0.00|1908.76|2277.32|4440.40|38.17|0.00|1908.76|1946.93|-368.56| +2452046|34230|10849|36581|582642|4703|45964|8|1|2360|89|43.92|55.77|16.73|0.00|1488.97|3908.88|4963.53|14.88|0.00|1488.97|1503.85|-2419.91| +2452046|34230|16853|36581|582642|4703|45964|8|182|2360|81|40.22|55.10|45.18|0.00|3659.58|3257.82|4463.10|329.36|0.00|3659.58|3988.94|401.76| +2452046|34230|13143|36581|582642|4703|45964|8|12|2360|75|56.40|77.26|23.17|1650.86|1737.75|4230.00|5794.50|6.08|1650.86|86.89|92.97|-4143.11| +2452046|34230|303|36581|582642|4703|45964|8|247|2360|10|74.80|98.73|42.45|0.00|424.50|748.00|987.30|21.22|0.00|424.50|445.72|-323.50| +2452046|34230|4455|36581|582642|4703|45964|8|64|2360|66|17.48|30.93|22.26|0.00|1469.16|1153.68|2041.38|117.53|0.00|1469.16|1586.69|315.48| +2452046|34230|7045|36581|582642|4703|45964|8|18|2360|33|71.20|116.76|108.58|0.00|3583.14|2349.60|3853.08|107.49|0.00|3583.14|3690.63|1233.54| +2452046|34230|12273|36581|582642|4703|45964|8|223|2360|89|56.75|64.69|13.58|0.00|1208.62|5050.75|5757.41|84.60|0.00|1208.62|1293.22|-3842.13| +2452046|34230|3263|36581|582642|4703|45964|8|300|2360|22|85.90|164.92|79.16|0.00|1741.52|1889.80|3628.24|87.07|0.00|1741.52|1828.59|-148.28| +2452046|34230|2193|36581|582642|4703|45964|8|204|2360|70|28.78|50.07|0.00|0.00|0.00|2014.60|3504.90|0.00|0.00|0.00|0.00|-2014.60| +2452046|34230|10199|36581|582642|4703|45964|8|149|2360|84|13.60|14.55|5.09|0.00|427.56|1142.40|1222.20|0.00|0.00|427.56|427.56|-714.84| +2452046|34230|9931|36581|582642|4703|45964|8|144|2360|30|43.87|73.26|62.27|0.00|1868.10|1316.10|2197.80|130.76|0.00|1868.10|1998.86|552.00| +2452046|34230|14497|36581|582642|4703|45964|8|194|2360|13|10.87|11.73|2.22|0.00|28.86|141.31|152.49|0.57|0.00|28.86|29.43|-112.45| +2452046|34230|9635|36581|582642|4703|45964|8|6|2360|84|29.54|34.56|3.45|0.00|289.80|2481.36|2903.04|26.08|0.00|289.80|315.88|-2191.56| +2452046|34230|8935|36581|582642|4703|45964|8|178|2360|40|56.29|102.44|73.75|0.00|2950.00|2251.60|4097.60|88.50|0.00|2950.00|3038.50|698.40| +2451827|62868|10871|86656|1427150|1597|34091|1|75|2361|54|57.84|60.15|21.65|0.00|1169.10|3123.36|3248.10|0.00|0.00|1169.10|1169.10|-1954.26| +2451827|62868|4181|86656|1427150|1597|34091|1|190|2361|45|62.24|101.45|89.27|923.94|4017.15|2800.80|4565.25|61.86|923.94|3093.21|3155.07|292.41| +2451827|62868|2969|86656|1427150|1597|34091|1|115|2361|79|50.04|92.07|82.86|3993.02|6545.94|3953.16|7273.53|127.64|3993.02|2552.92|2680.56|-1400.24| +2451827|62868|9518|86656|1427150|1597|34091|1|194|2361|66|28.76|31.63|7.59|0.00|500.94|1898.16|2087.58|15.02|0.00|500.94|515.96|-1397.22| +2451827|62868|9011|86656|1427150|1597|34091|1|245|2361|24|79.50|144.69|56.42|0.00|1354.08|1908.00|3472.56|108.32|0.00|1354.08|1462.40|-553.92| +2451827|62868|8624|86656|1427150|1597|34091|1|90|2361|83|85.94|132.34|39.70|0.00|3295.10|7133.02|10984.22|263.60|0.00|3295.10|3558.70|-3837.92| +2451827|62868|10535|86656|1427150|1597|34091|1|35|2361|49|55.37|77.51|24.80|12.15|1215.20|2713.13|3797.99|12.03|12.15|1203.05|1215.08|-1510.08| +2451827|62868|6001|86656|1427150|1597|34091|1|139|2361|62|17.01|31.46|6.92|347.52|429.04|1054.62|1950.52|4.89|347.52|81.52|86.41|-973.10| +2451902|53107|16255|3096|1628884|1519|21354|8|6|2362|26|1.64|3.18|2.92|4.55|75.92|42.64|82.68|2.85|4.55|71.37|74.22|28.73| +2451902|53107|3191|3096|1628884|1519|21354|8|143|2362|24|53.44|80.69|48.41|0.00|1161.84|1282.56|1936.56|69.71|0.00|1161.84|1231.55|-120.72| +2451902|53107|10261|3096|1628884|1519|21354|8|31|2362|94|25.96|38.68|6.18|0.00|580.92|2440.24|3635.92|5.80|0.00|580.92|586.72|-1859.32| +||11180|3096||1519||8|120|2362||||||244.91|||||244.91||-0.76| +2451902|53107|10429|3096|1628884|1519|21354|8|299|2362|43|97.92|103.79|30.09|0.00|1293.87|4210.56|4462.97|77.63|0.00|1293.87|1371.50|-2916.69| +2451902|53107|13178|3096|1628884|1519|21354|8|127|2362|86|3.60|4.96|4.36|0.00|374.96|309.60|426.56|18.74|0.00|374.96|393.70|65.36| +2451902|53107|10793|3096|1628884|1519|21354|8|185|2362|93|53.61|72.37|0.72|0.00|66.96|4985.73|6730.41|6.02|0.00|66.96|72.98|-4918.77| +2451902|53107|1987|3096|1628884|1519|21354|8|267|2362|75|85.74|104.60|48.11|1912.37|3608.25|6430.50|7845.00|0.00|1912.37|1695.88|1695.88|-4734.62| +2451902||1049|3096||||||2362|82|95.40||97.07|7243.36|7959.74||||7243.36|716.38|780.85|| +2451902|53107|7178|3096|1628884|1519|21354|8|167|2362|14|8.76|14.71|11.91|0.00|166.74|122.64|205.94|3.33|0.00|166.74|170.07|44.10| +2451902|53107|16562|3096|1628884|1519|21354|8|169|2362|95|35.76|39.33|12.19|173.70|1158.05|3397.20|3736.35|68.90|173.70|984.35|1053.25|-2412.85| +2451902|53107|16633|3096|1628884|1519|21354|8|171|2362|21|6.24|6.24|2.99|44.58|62.79|131.04|131.04|0.00|44.58|18.21|18.21|-112.83| +2451902|53107|11407|3096|1628884|1519|21354|8|181|2362|97|47.92|62.77|20.08|155.82|1947.76|4648.24|6088.69|143.35|155.82|1791.94|1935.29|-2856.30| +2451902|53107|7073|3096|1628884|1519|21354|8|39|2362|59|61.78|63.01|47.88|2598.92|2824.92|3645.02|3717.59|11.30|2598.92|226.00|237.30|-3419.02| +2451902|53107|15194|3096|1628884|1519|21354|8|195|2362|16|11.42|13.24|10.45|138.77|167.20|182.72|211.84|0.56|138.77|28.43|28.99|-154.29| +2451989|70569|4741|16915|635301|1334|44920|4|151|2363|6|54.69|55.78|17.29|0.00|103.74|328.14|334.68|8.29|0.00|103.74|112.03|-224.40| +2451989|70569|16521|16915|635301|1334|44920|4|258|2363|70|32.49|33.46|4.01|0.00|280.70|2274.30|2342.20|5.61|0.00|280.70|286.31|-1993.60| +2451989|70569|9235|16915|635301|1334|44920|4|279|2363|37|92.74|121.48|4.85|0.00|179.45|3431.38|4494.76|10.76|0.00|179.45|190.21|-3251.93| +2451989|70569|11871|16915|635301|1334|44920|4|183|2363|82|57.76|69.31|13.86|0.00|1136.52|4736.32|5683.42|90.92|0.00|1136.52|1227.44|-3599.80| +2451989|70569|3225|16915|635301|1334|44920|4|99|2363|41|88.25|137.67|93.61|0.00|3838.01|3618.25|5644.47|115.14|0.00|3838.01|3953.15|219.76| +2451989|70569|283|16915|635301|1334|44920|4|282|2363|32|92.42|170.97|47.87|168.50|1531.84|2957.44|5471.04|68.16|168.50|1363.34|1431.50|-1594.10| +2451989|70569|8799|16915|635301|1334|44920|4|37|2363|85|5.03|9.50|3.89|0.00|330.65|427.55|807.50|16.53|0.00|330.65|347.18|-96.90| +2451989|70569|13439|16915|635301|1334|44920|4|210|2363|50|31.01|44.03|29.50|0.00|1475.00|1550.50|2201.50|73.75|0.00|1475.00|1548.75|-75.50| +|70569|13681||635301|1334|44920||278|2363||55.68||48.23|3401.17||4565.76|||3401.17|||-4012.07| +2451989|70569|8323|16915|635301|1334|44920|4|195|2363|89|29.16|33.53|0.33|0.00|29.37|2595.24|2984.17|0.88|0.00|29.37|30.25|-2565.87| +2451989|70569|7491|16915|635301|1334|44920|4|288|2363|51|31.39|60.26|8.43|0.00|429.93|1600.89|3073.26|4.29|0.00|429.93|434.22|-1170.96| +2451989|70569|5927|16915|635301|1334|44920|4|150|2363|73|29.65|52.48|14.16|0.00|1033.68|2164.45|3831.04|10.33|0.00|1033.68|1044.01|-1130.77| +2451989||15321||635301|1334|44920|||2363||||1.72|||||||2.07||-4.43| +2452218|52364|2517|75165|913872|5732|11639|8|117|2364|69|66.60|109.89|52.74|3602.66|3639.06|4595.40|7582.41|1.09|3602.66|36.40|37.49|-4559.00| +2452218|52364|11481|75165|913872|5732|11639|8|86|2364|66|99.74|125.67|25.13|978.56|1658.58|6582.84|8294.22|27.20|978.56|680.02|707.22|-5902.82| +2452218|52364|12893|75165|913872|5732|11639|8|161|2364|72|10.54|18.55|15.76|0.00|1134.72|758.88|1335.60|11.34|0.00|1134.72|1146.06|375.84| +2452218|52364|2533|75165|913872|5732|11639|8|162|2364|63|36.22|63.74|49.07|0.00|3091.41|2281.86|4015.62|247.31|0.00|3091.41|3338.72|809.55| +2452218|52364|15865|75165|913872|5732|11639|8|99|2364|14|85.91|125.42|21.32|0.00|298.48|1202.74|1755.88|20.89|0.00|298.48|319.37|-904.26| +2452218|52364|5339|75165|913872|5732|11639|8|91|2364|40|72.45|82.59|28.08|0.00|1123.20|2898.00|3303.60|0.00|0.00|1123.20|1123.20|-1774.80| +2452218|52364|12359|75165|913872|5732|11639|8|139|2364|62|63.09|73.81|16.97|0.00|1052.14|3911.58|4576.22|0.00|0.00|1052.14|1052.14|-2859.44| +2452218|52364|10267|75165|913872|5732|11639|8|98|2364|39|85.00|130.90|111.26|0.00|4339.14|3315.00|5105.10|173.56|0.00|4339.14|4512.70|1024.14| +2452218|52364|16115|75165|913872|5732|11639|8|248|2364|39|23.28|23.74|0.47|0.00|18.33|907.92|925.86|0.18|0.00|18.33|18.51|-889.59| +2452508|43051|2503|56017|1445538|2259|39427|2|49|2365|49|99.97|134.95|2.69|0.00|131.81|4898.53|6612.55|0.00|0.00|131.81|131.81|-4766.72| +2452508|43051|7200|56017|1445538|2259|39427|2|189|2365|99|42.47|49.26|19.21|0.00|1901.79|4204.53|4876.74|76.07|0.00|1901.79|1977.86|-2302.74| +2452508|43051|9126|56017|1445538|2259|39427|2|106|2365|78|72.47|125.37|1.25|0.00|97.50|5652.66|9778.86|0.00|0.00|97.50|97.50|-5555.16| +2452508|43051|11214|56017|1445538|2259|39427|2|210|2365|27|60.67|74.01|50.32|0.00|1358.64|1638.09|1998.27|81.51|0.00|1358.64|1440.15|-279.45| +2452508|43051|6969|56017|1445538|2259|39427|2|159|2365|60|41.37|78.60|8.64|0.00|518.40|2482.20|4716.00|31.10|0.00|518.40|549.50|-1963.80| +2452508|43051|13986|56017|1445538|2259|39427|2|164|2365|69|76.72|114.31|42.29|0.00|2918.01|5293.68|7887.39|58.36|0.00|2918.01|2976.37|-2375.67| +2452508|43051|15210|56017|1445538|2259|39427|2|112|2365|25|12.98|14.92|10.29|0.00|257.25|324.50|373.00|7.71|0.00|257.25|264.96|-67.25| +2452508|43051|2205|56017|1445538|2259|39427|2|91|2365|33|81.12|145.20|59.53|687.57|1964.49|2676.96|4791.60|38.30|687.57|1276.92|1315.22|-1400.04| +2452508||8499|56017|||39427||3|2365||11.32|11.54|10.96|||452.80|461.60|6.70||74.53|81.23|-378.27| +2452508|43051|8823|56017|1445538|2259|39427|2|180|2365|15|79.82|87.80|40.38|0.00|605.70|1197.30|1317.00|18.17|0.00|605.70|623.87|-591.60| +2452508|43051|96|56017|1445538|2259|39427|2|235|2365|98|12.91|25.17|0.00|0.00|0.00|1265.18|2466.66|0.00|0.00|0.00|0.00|-1265.18| +2451227|52528|4442|76742|806082|3775|38315|1|86|2366|39|61.48|105.13|31.53|0.00|1229.67|2397.72|4100.07|61.48|0.00|1229.67|1291.15|-1168.05| +2451227|52528|10774|76742|806082|3775|38315|1|154|2366|82|2.79|5.21|1.56|38.37|127.92|228.78|427.22|2.68|38.37|89.55|92.23|-139.23| +2451227|52528|6718|76742|806082|3775|38315|1|96|2366|8|98.65|115.42|23.08|0.00|184.64|789.20|923.36|3.69|0.00|184.64|188.33|-604.56| +2451227|52528|15422|76742|806082|3775|38315|1|230|2366|45|47.37|71.52|62.93|792.91|2831.85|2131.65|3218.40|61.16|792.91|2038.94|2100.10|-92.71| +2451227|52528|12248|76742|806082|3775|38315|1|143|2366|43|64.65|127.36|127.36|0.00|5476.48|2779.95|5476.48|109.52|0.00|5476.48|5586.00|2696.53| +2451227|52528|9619|76742|806082|3775|38315|1|77|2366|14|73.29|93.07|68.87|0.00|964.18|1026.06|1302.98|0.00|0.00|964.18|964.18|-61.88| +2451227|52528|6052|76742|806082|3775|38315|1|296|2366|93|44.86|89.72|87.02|0.00|8092.86|4171.98|8343.96|647.42|0.00|8092.86|8740.28|3920.88| +2451227|52528|1934|76742|806082|3775|38315|1|62|2366|77|9.67|18.85|1.31|0.00|100.87|744.59|1451.45|4.03|0.00|100.87|104.90|-643.72| +2451227|52528|8095|76742|806082|3775|38315|1|151|2366|74|81.26|105.63|12.67|0.00|937.58|6013.24|7816.62|84.38|0.00|937.58|1021.96|-5075.66| +2451227|52528|16777|76742|806082|3775|38315|1|25|2366|81|14.01|21.71|18.67|0.00|1512.27|1134.81|1758.51|90.73|0.00|1512.27|1603.00|377.46| +2451227|52528|9403|76742|806082|3775|38315|1|63|2366|45|63.88|125.20|113.93|0.00|5126.85|2874.60|5634.00|410.14|0.00|5126.85|5536.99|2252.25| +2451227|52528|12715|76742|806082|3775|38315|1|116|2366|81|74.26|79.45|34.95|0.00|2830.95|6015.06|6435.45|169.85|0.00|2830.95|3000.80|-3184.11| +2451227|52528|1232|76742|806082|3775|38315|1|105|2366|26|8.57|9.34|7.37|0.00|191.62|222.82|242.84|5.74|0.00|191.62|197.36|-31.20| +2451227|52528|415|76742|806082|3775|38315|1|206|2366|29|95.20|120.90|71.33|0.00|2068.57|2760.80|3506.10|20.68|0.00|2068.57|2089.25|-692.23| +2451227|52528|15439|76742|806082|3775|38315|1|44|2366|16|45.45|64.53|60.01|384.06|960.16|727.20|1032.48|34.56|384.06|576.10|610.66|-151.10| +2451227|52528|8312|76742|806082|3775|38315|1|184|2366|76|91.76|167.92|1.67|0.00|126.92|6973.76|12761.92|5.07|0.00|126.92|131.99|-6846.84| +2451535|75058|10544|51161|1360236|864|1939|7|224|2367|98|65.24|108.29|82.30|0.00|8065.40|6393.52|10612.42|322.61|0.00|8065.40|8388.01|1671.88| +2451535|75058|6074|51161|1360236|864|1939|7|288|2367|70|33.72|34.05|8.17|0.00|571.90|2360.40|2383.50|34.31|0.00|571.90|606.21|-1788.50| +2451535|75058|2714|51161|1360236|864|1939|7|215|2367|39|38.38|62.94|50.98|0.00|1988.22|1496.82|2454.66|159.05|0.00|1988.22|2147.27|491.40| +2451535|75058|3412|51161|1360236|864|1939|7|23|2367|35|85.83|142.47|32.76|0.00|1146.60|3004.05|4986.45|22.93|0.00|1146.60|1169.53|-1857.45| +2451535|75058|14924|51161|1360236|864|1939|7|300|2367|92|74.85|112.27|111.14|0.00|10224.88|6886.20|10328.84|0.00|0.00|10224.88|10224.88|3338.68| +2451535|75058|4063|51161|1360236|864|1939|7|205|2367|11|27.25|50.95|9.17|0.00|100.87|299.75|560.45|0.00|0.00|100.87|100.87|-198.88| +2451535|75058|6328|51161|1360236|864|1939|7|27|2367|52|93.63|164.78|133.47|0.00|6940.44|4868.76|8568.56|69.40|0.00|6940.44|7009.84|2071.68| +2451535|75058|14608|51161|1360236|864|1939|7|36|2367|75|29.33|45.16|2.70|0.00|202.50|2199.75|3387.00|18.22|0.00|202.50|220.72|-1997.25| +2451535|75058|15352|51161|1360236|864|1939|7|5|2367|52|60.38|92.38|20.32|0.00|1056.64|3139.76|4803.76|0.00|0.00|1056.64|1056.64|-2083.12| +2451136|61479|1882|10886|921079|1319|27559|2|193|2368|19|86.97|94.79|8.53|0.00|162.07|1652.43|1801.01|0.00|0.00|162.07|162.07|-1490.36| +2451136|61479|6368|10886|921079|1319|27559|2|247|2368|54|94.33|121.68|41.37|0.00|2233.98|5093.82|6570.72|0.00|0.00|2233.98|2233.98|-2859.84| +2451136|61479|13087|10886|921079|1319|27559|2|66|2368|13|85.65|134.47|79.33|0.00|1031.29|1113.45|1748.11|41.25|0.00|1031.29|1072.54|-82.16| +2451136|61479|808|10886|921079|1319|27559|2|154|2368|99|99.16|101.14|101.14|0.00|10012.86|9816.84|10012.86|100.12|0.00|10012.86|10112.98|196.02| +2451136|61479|5158|10886|921079|1319|27559|2|171|2368|86|10.11|17.99|8.99|0.00|773.14|869.46|1547.14|38.65|0.00|773.14|811.79|-96.32| +2451136|61479|6554|10886|921079|1319|27559|2|146|2368|95|87.06|95.76|3.83|243.77|363.85|8270.70|9097.20|4.80|243.77|120.08|124.88|-8150.62| +2451136|61479|826|10886|921079|1319|27559|2|193|2368|13|65.51|100.23|88.20|57.33|1146.60|851.63|1302.99|54.46|57.33|1089.27|1143.73|237.64| +2451136|61479|16327|10886|921079|1319|27559|2|101|2368|71|39.14|65.75|32.21|0.00|2286.91|2778.94|4668.25|137.21|0.00|2286.91|2424.12|-492.03| +2451136|61479|14890|10886|921079|1319|27559|2|297|2368|90|94.47|146.42|103.95|0.00|9355.50|8502.30|13177.80|654.88|0.00|9355.50|10010.38|853.20| +2451136|61479|14749|10886|921079|1319|27559|2|67|2368|84|29.98|51.86|21.26|0.00|1785.84|2518.32|4356.24|160.72|0.00|1785.84|1946.56|-732.48| +2451136|61479|6092|10886|921079|1319|27559|2|7|2368|85|2.49|4.28|2.05|33.10|174.25|211.65|363.80|0.00|33.10|141.15|141.15|-70.50| +2451136|61479|4999|10886|921079|1319|27559|2|165|2368|22|39.84|63.74|1.91|0.00|42.02|876.48|1402.28|1.68|0.00|42.02|43.70|-834.46| +2451136|61479|11138|10886|921079|1319|27559|2|67|2368|83|2.87|3.78|2.79|0.00|231.57|238.21|313.74|13.89|0.00|231.57|245.46|-6.64| +2451136|61479|10411|10886|921079|1319|27559|2|128|2368|87|69.48|110.47|54.13|2025.00|4709.31|6044.76|9610.89|0.00|2025.00|2684.31|2684.31|-3360.45| +2451136|61479|2342|10886|921079|1319|27559|2|172|2368|52|8.34|10.34|5.68|0.00|295.36|433.68|537.68|26.58|0.00|295.36|321.94|-138.32| +2451136|61479|11000|10886|921079|1319|27559|2|254|2368|74|93.01|105.10|74.62|3423.56|5521.88|6882.74|7777.40|83.93|3423.56|2098.32|2182.25|-4784.42| +2451412|63150|14216|44792|1007101|3010|4119|2|174|2369|16|4.04|7.27|6.39|0.00|102.24|64.64|116.32|3.06|0.00|102.24|105.30|37.60| +|63150|2834|44792|1007101||4119|2||2369||70.10|107.95||249.27|830.90|||0.00|249.27||581.63|| +2451412|63150|4117|44792|1007101|3010|4119|2|169|2369|62|37.47|46.46|15.79|0.00|978.98|2323.14|2880.52|88.10|0.00|978.98|1067.08|-1344.16| +2451412|63150|5035|44792|1007101|3010|4119|2|82|2369|8|30.05|31.55|2.83|0.00|22.64|240.40|252.40|0.45|0.00|22.64|23.09|-217.76| +2451412|63150|662|44792|1007101|3010|4119|2|110|2369|93|3.17|4.34|2.17|0.00|201.81|294.81|403.62|4.03|0.00|201.81|205.84|-93.00| +2451412|63150|10804|44792|1007101|3010|4119|2|239|2369|73|65.28|74.41|44.64|0.00|3258.72|4765.44|5431.93|162.93|0.00|3258.72|3421.65|-1506.72| +2451412|63150|7345|44792|1007101|3010|4119|2|255|2369|8|70.97|92.97|55.78|0.00|446.24|567.76|743.76|35.69|0.00|446.24|481.93|-121.52| +2451412|63150|11378|44792|1007101|3010|4119|2|67|2369|44|97.79|194.60|66.16|0.00|2911.04|4302.76|8562.40|232.88|0.00|2911.04|3143.92|-1391.72| +2451412||11164|||||2|184|2369|26|58.12|||||||149.80||||| +2452523|49339|14934|8261|418809|3404|36799|2|205|2370|3|98.61|136.08|8.16|0.00|24.48|295.83|408.24|1.95|0.00|24.48|26.43|-271.35| +2452523|49339|11388|8261|418809|3404|36799|2|151|2370|15|62.82|125.01|78.75|0.00|1181.25|942.30|1875.15|0.00|0.00|1181.25|1181.25|238.95| +2452523|49339|1023|8261|418809|3404|36799|2|179|2370|5|45.51|46.87|29.52|0.00|147.60|227.55|234.35|10.33|0.00|147.60|157.93|-79.95| +2452523|49339|9144|8261|418809|3404|36799|2|124|2370|95|89.64|179.28|37.64|1752.14|3575.80|8515.80|17031.60|109.41|1752.14|1823.66|1933.07|-6692.14| +2452523|49339|1977|8261|418809|3404|36799|2|83|2370|94|44.04|48.00|13.92|0.00|1308.48|4139.76|4512.00|117.76|0.00|1308.48|1426.24|-2831.28| +2452523|49339|5025|8261|418809|3404|36799|2|292|2370|63|72.75|86.57|65.79|1657.90|4144.77|4583.25|5453.91|124.34|1657.90|2486.87|2611.21|-2096.38| +2452523|49339|6666|8261|418809|3404|36799|2|47|2370|87|84.45|93.73|21.55|0.00|1874.85|7347.15|8154.51|18.74|0.00|1874.85|1893.59|-5472.30| +2452523|49339|10609|8261|418809|3404|36799|2|141|2370|9|67.42|81.57|9.78|0.00|88.02|606.78|734.13|3.52|0.00|88.02|91.54|-518.76| +2452523|49339|1285|8261|418809|3404|36799|2|204|2370|42|73.49|74.95|16.48|0.00|692.16|3086.58|3147.90|13.84|0.00|692.16|706.00|-2394.42| +2452523|49339|1167|8261|418809|3404|36799|2|188|2370|74|16.37|22.09|0.00|0.00|0.00|1211.38|1634.66|0.00|0.00|0.00|0.00|-1211.38| +2452523|49339|1957|8261|418809|3404|36799|2|239|2370|19|19.27|27.94|21.79|0.00|414.01|366.13|530.86|24.84|0.00|414.01|438.85|47.88| +||3139|3121||686|12045|10||2371|50|57.56|91.52||0.00|1281.00||4576.00||0.00|1281.00|1281.00|| +2451622|64798|6632|3121|865327|686|12045|10|120|2371|20|5.06|9.31|2.42|0.00|48.40|101.20|186.20|0.48|0.00|48.40|48.88|-52.80| +2451622|64798|8383|3121|865327|686|12045|10|280|2371|42|80.61|102.37|98.27|0.00|4127.34|3385.62|4299.54|0.00|0.00|4127.34|4127.34|741.72| +2451622|64798|10717|3121|865327|686|12045|10|91|2371|43|99.97|167.94|157.86|0.00|6787.98|4298.71|7221.42|271.51|0.00|6787.98|7059.49|2489.27| +||11765||||12045|10|28|2371|71|||15.13|0.00|1074.23|||10.74|0.00||1084.97|| +2451622|64798|14006|3121|865327|686|12045|10|146|2371|14|45.12|89.33|12.50|0.00|175.00|631.68|1250.62|5.25|0.00|175.00|180.25|-456.68| +2451622|64798|1514|3121|865327|686|12045|10|23|2371|84|96.68|147.92|2.95|0.00|247.80|8121.12|12425.28|9.91|0.00|247.80|257.71|-7873.32| +2451622|64798|17431|3121|865327|686|12045|10|160|2371|26|31.05|52.16|15.64|0.00|406.64|807.30|1356.16|36.59|0.00|406.64|443.23|-400.66| +2451622|64798|12014|3121|865327|686|12045|10|136|2371|69|88.46|104.38|11.48|0.00|792.12|6103.74|7202.22|39.60|0.00|792.12|831.72|-5311.62| +2452428|63069|14316|83186|812902|2171|5087|2|137|2372|80|78.92|117.59|88.19|0.00|7055.20|6313.60|9407.20|423.31|0.00|7055.20|7478.51|741.60| +2452428|63069|8970|83186|812902|2171|5087|2|173|2372|65|15.78|16.88|5.40|203.58|351.00|1025.70|1097.20|8.84|203.58|147.42|156.26|-878.28| +2452428|63069|8394|83186|812902|2171|5087|2|54|2372|43|88.96|115.64|82.10|2541.81|3530.30|3825.28|4972.52|9.88|2541.81|988.49|998.37|-2836.79| +2452428|63069|13095|83186|812902|2171|5087|2|146|2372|5|21.19|31.57|7.89|0.00|39.45|105.95|157.85|3.15|0.00|39.45|42.60|-66.50| +2452428|63069|6823|83186|812902|2171|5087|2|98|2372|49|39.83|42.21|18.99|0.00|930.51|1951.67|2068.29|0.00|0.00|930.51|930.51|-1021.16| +2452428|63069|7089|83186|812902|2171|5087|2|231|2372|100|37.27|70.06|2.10|0.00|210.00|3727.00|7006.00|14.70|0.00|210.00|224.70|-3517.00| +2452428|63069|8367|83186|812902|2171|5087|2|10|2372|43|35.58|59.06|49.61|831.95|2133.23|1529.94|2539.58|52.05|831.95|1301.28|1353.33|-228.66| +2452428|63069|9372|83186|812902|2171|5087|2|297|2372|43|85.41|155.44|80.82|0.00|3475.26|3672.63|6683.92|278.02|0.00|3475.26|3753.28|-197.37| +2452428|63069|13287|83186|812902|2171|5087|2|50|2372|45|2.29|2.38|0.66|0.00|29.70|103.05|107.10|0.89|0.00|29.70|30.59|-73.35| +2452428|63069|15693|83186|812902|2171|5087|2|90|2372|64|67.13|83.91|1.67|0.00|106.88|4296.32|5370.24|2.13|0.00|106.88|109.01|-4189.44| +2451864|39989|11903|17498|393904|4357|30010|10|300|2373|25|63.30|72.79|48.04|0.00|1201.00|1582.50|1819.75|12.01|0.00|1201.00|1213.01|-381.50| +2451864|39989|6025|17498|393904|4357|30010|10|54|2373|77|77.68|100.98|14.13|0.00|1088.01|5981.36|7775.46|10.88|0.00|1088.01|1098.89|-4893.35| +2451864|39989|16501|17498|393904|4357|30010|10|25|2373|28|71.03|84.52|63.39|0.00|1774.92|1988.84|2366.56|159.74|0.00|1774.92|1934.66|-213.92| +||17653|17498||4357|30010|10|177|2373|36||||0.00|||4466.88|52.26|0.00|580.68||| +2451864|39989|7993|17498|393904|4357|30010|10|7|2373|4|93.98|113.71|14.78|0.00|59.12|375.92|454.84|4.13|0.00|59.12|63.25|-316.80| +2451864|39989|15067|17498|393904|4357|30010|10|259|2373|36|27.52|43.20|0.00|0.00|0.00|990.72|1555.20|0.00|0.00|0.00|0.00|-990.72| +2451864|39989|907|17498|393904|4357|30010|10|35|2373|52|47.13|61.26|34.91|0.00|1815.32|2450.76|3185.52|90.76|0.00|1815.32|1906.08|-635.44| +2451864|39989|1406|17498|393904|4357|30010|10|270|2373|62|63.09|121.13|111.43|0.00|6908.66|3911.58|7510.06|276.34|0.00|6908.66|7185.00|2997.08| +2451864|39989|17785|17498|393904|4357|30010|10|17|2373|43|47.74|89.75|31.41|0.00|1350.63|2052.82|3859.25|94.54|0.00|1350.63|1445.17|-702.19| +2451864|39989|9626|17498|393904|4357|30010|10|123|2373|61|21.28|27.02|9.99|0.00|609.39|1298.08|1648.22|6.09|0.00|609.39|615.48|-688.69| +2451864|39989|17599|17498|393904|4357|30010|10|71|2373|19|86.43|95.93|89.21|0.00|1694.99|1642.17|1822.67|152.54|0.00|1694.99|1847.53|52.82| +||10007||||30010|10||2373||||13.74|||||0.54|||55.50|-17.04| +2451864|39989|2177|17498|393904|4357|30010|10|210|2373|60|41.65|60.39|28.98|0.00|1738.80|2499.00|3623.40|139.10|0.00|1738.80|1877.90|-760.20| +2451864|39989|6521|17498|393904|4357|30010|10|206|2373|11|3.52|4.96|2.43|0.00|26.73|38.72|54.56|1.87|0.00|26.73|28.60|-11.99| +2451864|39989|8450|17498|393904|4357|30010|10|7|2373|30|78.62|91.98|62.54|131.33|1876.20|2358.60|2759.40|104.69|131.33|1744.87|1849.56|-613.73| +2451864|39989|16157|17498|393904|4357|30010|10|209|2373|69|77.92|97.40|12.66|0.00|873.54|5376.48|6720.60|0.00|0.00|873.54|873.54|-4502.94| +2451043|53642|7130|47779|7990|1448|7259|2|182|2374|40|11.75|18.33|5.49|0.00|219.60|470.00|733.20|2.19|0.00|219.60|221.79|-250.40| +2451043|53642|2546|47779|7990|1448|7259|2|117|2374|54|11.11|15.99|14.71|0.00|794.34|599.94|863.46|55.60|0.00|794.34|849.94|194.40| +2451043|53642|10747|47779|7990|1448|7259|2|183|2374|47|38.44|49.97|10.99|0.00|516.53|1806.68|2348.59|46.48|0.00|516.53|563.01|-1290.15| +2451043|53642|10498|47779|7990|1448|7259|2|3|2374|77|72.27|81.66|5.71|0.00|439.67|5564.79|6287.82|35.17|0.00|439.67|474.84|-5125.12| +2451043|53642|10678|47779|7990|1448|7259|2|99|2374|82|73.44|122.64|85.84|5701.49|7038.88|6022.08|10056.48|0.00|5701.49|1337.39|1337.39|-4684.69| +2451043|53642|12914|47779|7990|1448|7259|2|204|2374|11|61.30|109.11|67.64|0.00|744.04|674.30|1200.21|44.64|0.00|744.04|788.68|69.74| +2451043|53642|9601|47779|7990|1448|7259|2|114|2374|58|13.56|14.37|13.93|0.00|807.94|786.48|833.46|40.39|0.00|807.94|848.33|21.46| +2451043|53642|10880|47779|7990|1448|7259|2|211|2374|49|42.19|66.23|15.23|0.00|746.27|2067.31|3245.27|67.16|0.00|746.27|813.43|-1321.04| +2451043|53642|16772|47779|7990|1448|7259|2|48|2374|79|96.25|142.45|106.83|0.00|8439.57|7603.75|11253.55|84.39|0.00|8439.57|8523.96|835.82| +2451799|33507|11171|4616|1777637|5858|2079|10|207|2375|4|40.31|47.96|7.19|0.00|28.76|161.24|191.84|1.72|0.00|28.76|30.48|-132.48| +2451799|33507|3890|4616|1777637|||10|255|2375||||65.15|0.00||2350.80||19.54|0.00|||-396.30| +2451799|33507|4508|4616|1777637|5858|2079|10|1|2375|67|21.01|40.33|35.08|0.00|2350.36|1407.67|2702.11|141.02|0.00|2350.36|2491.38|942.69| +2451799|33507|17503|4616|1777637|5858|2079|10|114|2375|71|69.75|123.45|35.80|0.00|2541.80|4952.25|8764.95|152.50|0.00|2541.80|2694.30|-2410.45| +2451799|33507|10688|4616|1777637|5858|2079|10|35|2375|9|37.55|72.47|53.62|0.00|482.58|337.95|652.23|9.65|0.00|482.58|492.23|144.63| +2451799|33507|7235|4616|1777637|5858|2079|10|53|2375|1|28.29|45.26|21.27|0.00|21.27|28.29|45.26|0.42|0.00|21.27|21.69|-7.02| +2451799|33507|6319|4616|1777637|5858|2079|10|269|2375|56|38.37|46.81|23.40|0.00|1310.40|2148.72|2621.36|117.93|0.00|1310.40|1428.33|-838.32| +2451799|33507|3511|4616|1777637|5858|2079|10|72|2375|45|58.20|67.51|15.52|0.00|698.40|2619.00|3037.95|20.95|0.00|698.40|719.35|-1920.60| +2450970|63960|10724|28055|421334|6613|6292|10|276|2376|96|25.42|28.97|0.00|0.00|0.00|2440.32|2781.12|0.00|0.00|0.00|0.00|-2440.32| +2450970|63960|17953|28055|421334|6613|6292|10|208|2376|80|95.58|176.82|17.68|1329.53|1414.40|7646.40|14145.60|0.84|1329.53|84.87|85.71|-7561.53| +2450970|63960|4840|28055|421334|6613|6292|10|117|2376|33|83.84|93.90|44.13|0.00|1456.29|2766.72|3098.70|101.94|0.00|1456.29|1558.23|-1310.43| +2450970|63960|5768|28055|421334|6613|6292|10|15|2376|28|41.73|44.65|21.87|0.00|612.36|1168.44|1250.20|36.74|0.00|612.36|649.10|-556.08| +2450970|63960|6914|28055|421334|6613|6292|10|49|2376|30|15.63|22.50|13.95|0.00|418.50|468.90|675.00|0.00|0.00|418.50|418.50|-50.40| +2450970|63960|14612|28055|421334|6613|6292|10|195|2376|3|38.29|66.24|30.47|0.00|91.41|114.87|198.72|2.74|0.00|91.41|94.15|-23.46| +2450970|63960|3655|28055|421334|6613|6292|10|63|2376|100|95.26|122.88|51.60|0.00|5160.00|9526.00|12288.00|51.60|0.00|5160.00|5211.60|-4366.00| +2450970|63960|15955|28055|421334|6613|6292|10|13|2376|97|46.28|54.61|4.36|0.00|422.92|4489.16|5297.17|8.45|0.00|422.92|431.37|-4066.24| +2450970|63960|14809|28055|421334|6613|6292|10|160|2376|62|66.86|96.94|70.76|0.00|4387.12|4145.32|6010.28|43.87|0.00|4387.12|4430.99|241.80| +2450970|63960|2798|28055|421334|6613|6292|10|123|2376|93|90.76|124.34|33.57|0.00|3122.01|8440.68|11563.62|187.32|0.00|3122.01|3309.33|-5318.67| +2450970|63960|16654|28055|421334|6613|6292|10|246|2376|16|61.84|92.14|78.31|0.00|1252.96|989.44|1474.24|112.76|0.00|1252.96|1365.72|263.52| +2450970|63960|3266|28055|421334|6613|6292|10|60|2376|14|4.61|9.17|4.76|0.00|66.64|64.54|128.38|0.00|0.00|66.64|66.64|2.10| +2450970|63960|14824|28055|421334|6613|6292|10|16|2376|7|51.09|56.19|42.70|0.00|298.90|357.63|393.33|5.97|0.00|298.90|304.87|-58.73| +2450970|63960|4496|28055|421334|6613|6292|10|216|2376|60|30.53|31.14|2.80|0.00|168.00|1831.80|1868.40|6.72|0.00|168.00|174.72|-1663.80| +2451903|34146|15571|74498|972612|3526|38840|10|270|2377|39|42.61|84.79|0.00|0.00|0.00|1661.79|3306.81|0.00|0.00|0.00|0.00|-1661.79| +2451903|34146|9032|74498|972612|3526|38840|10|290|2377|23|58.64|72.12|49.76|755.35|1144.48|1348.72|1658.76|15.56|755.35|389.13|404.69|-959.59| +2451903||16760|74498||3526||||2377|50|55.44|||0.00|||3215.50||0.00|321.50||| +2451903|34146|10147|74498|972612|3526|38840|10|261|2377|92|9.19|10.56|9.82|0.00|903.44|845.48|971.52|9.03|0.00|903.44|912.47|57.96| +2451903|34146|947|74498|972612|3526|38840|10|159|2377|22|45.09|87.92|10.55|0.00|232.10|991.98|1934.24|16.24|0.00|232.10|248.34|-759.88| +2451903|34146|14165|74498|972612|3526|38840|10|35|2377|31|32.17|46.96|13.61|0.00|421.91|997.27|1455.76|12.65|0.00|421.91|434.56|-575.36| +2451903|34146|13213|74498|972612|3526|38840|10|108|2377|74|44.02|62.06|6.82|0.00|504.68|3257.48|4592.44|30.28|0.00|504.68|534.96|-2752.80| +2451903|34146|15701|74498|972612|3526|38840|10|45|2377|87|17.65|21.18|20.54|393.13|1786.98|1535.55|1842.66|13.93|393.13|1393.85|1407.78|-141.70| +2451903|34146|10796|74498|972612|3526|38840|10|33|2377|60|44.35|69.62|59.17|0.00|3550.20|2661.00|4177.20|0.00|0.00|3550.20|3550.20|889.20| +2452235|67012|8081|91474|1091579|484|25789|7|205|2378|76|97.46|185.17|68.51|0.00|5206.76|7406.96|14072.92|312.40|0.00|5206.76|5519.16|-2200.20| +2452235|67012|9133|91474|1091579|484|25789|7|109|2378|62|51.70|101.84|91.65|0.00|5682.30|3205.40|6314.08|511.40|0.00|5682.30|6193.70|2476.90| +2452235|67012|7857|91474|1091579|484|25789|7|289|2378|88|98.18|160.03|62.41|0.00|5492.08|8639.84|14082.64|109.84|0.00|5492.08|5601.92|-3147.76| +2452235|67012|5061|91474|1091579|484|25789|7|203|2378|16|50.73|65.44|48.42|0.00|774.72|811.68|1047.04|7.74|0.00|774.72|782.46|-36.96| +2452235|67012|4585|91474|1091579|484|25789|7|91|2378|30|24.61|35.68|25.33|0.00|759.90|738.30|1070.40|53.19|0.00|759.90|813.09|21.60| +2452235|67012|7729|91474|1091579|484|25789|7|81|2378|72|44.63|66.49|48.53|1467.54|3494.16|3213.36|4787.28|121.59|1467.54|2026.62|2148.21|-1186.74| +2452235|67012|7411|91474|1091579|484|25789|7|249|2378|97|75.03|138.05|85.59|0.00|8302.23|7277.91|13390.85|747.20|0.00|8302.23|9049.43|1024.32| +2452235|67012|13391|91474|1091579|484|25789|7|19|2378|19|70.83|77.20|4.63|0.00|87.97|1345.77|1466.80|4.39|0.00|87.97|92.36|-1257.80| +2452235|67012|16019|91474|1091579|484|25789|7|32|2378|8|10.38|18.06|13.90|0.00|111.20|83.04|144.48|5.56|0.00|111.20|116.76|28.16| +2452235|67012|4813|91474|1091579|484|25789|7|105|2378|55|46.15|81.68|28.58|0.00|1571.90|2538.25|4492.40|78.59|0.00|1571.90|1650.49|-966.35| +2452235|67012|6657|91474|1091579|484|25789|7|290|2378|39|65.98|99.62|97.62|0.00|3807.18|2573.22|3885.18|228.43|0.00|3807.18|4035.61|1233.96| +2452235|67012|721|91474|1091579|484|25789|7|25|2378|17|22.21|31.98|21.74|0.00|369.58|377.57|543.66|33.26|0.00|369.58|402.84|-7.99| +2452235|67012|16367|91474|1091579|484|25789|7|101|2378|49|53.97|79.87|31.94|281.71|1565.06|2644.53|3913.63|115.50|281.71|1283.35|1398.85|-1361.18| +2452235|67012|7583|91474|1091579|484|25789|7|85|2378|100|32.35|40.76|24.86|0.00|2486.00|3235.00|4076.00|149.16|0.00|2486.00|2635.16|-749.00| +2452235|67012|3529|91474|1091579|484|25789|7|109|2378|88|16.28|22.46|13.70|0.00|1205.60|1432.64|1976.48|0.00|0.00|1205.60|1205.60|-227.04| +2452235|67012|15571|91474|1091579|484|25789|7|179|2378|19|99.85|143.78|33.06|0.00|628.14|1897.15|2731.82|31.40|0.00|628.14|659.54|-1269.01| +2450825|47003|12032|67276|1381450|1671|49092|7|166|2379|74|52.37|54.46|23.96|0.00|1773.04|3875.38|4030.04|0.00|0.00|1773.04|1773.04|-2102.34| +2450825|47003|8128|67276|1381450|1671|49092|7|87|2379|12|80.75|84.78|11.02|0.00|132.24|969.00|1017.36|7.93|0.00|132.24|140.17|-836.76| +2450825|47003|3458|67276|1381450|1671|49092|7|202|2379|5|79.50|139.92|39.17|95.96|195.85|397.50|699.60|1.99|95.96|99.89|101.88|-297.61| +2450825|47003|10946|67276|1381450|1671|49092|7|298|2379|35|33.30|49.95|2.49|0.00|87.15|1165.50|1748.25|0.87|0.00|87.15|88.02|-1078.35| +2450825|47003|403|67276|1381450|1671|49092|7|130|2379|65|46.51|62.78|11.30|0.00|734.50|3023.15|4080.70|22.03|0.00|734.50|756.53|-2288.65| +2450825|47003|2575|67276|1381450|1671|49092|7|235|2379|8|42.60|82.21|41.92|0.00|335.36|340.80|657.68|0.00|0.00|335.36|335.36|-5.44| +2450825|47003|4658|67276|1381450|1671|49092|7|293|2379|70|22.52|31.30|7.19|0.00|503.30|1576.40|2191.00|35.23|0.00|503.30|538.53|-1073.10| +2450825|47003|9721|67276|1381450|1671|49092|7|232|2379|7|21.39|23.31|11.18|0.00|78.26|149.73|163.17|4.69|0.00|78.26|82.95|-71.47| +2450825|47003|8806|67276|1381450|1671|49092|7|201|2379|85|73.74|111.34|81.27|0.00|6907.95|6267.90|9463.90|276.31|0.00|6907.95|7184.26|640.05| +2450825|47003|13711|67276|1381450|1671|49092|7|6|2379|6|29.71|41.29|40.46|0.00|242.76|178.26|247.74|12.13|0.00|242.76|254.89|64.50| +2450825|47003|13648|67276|1381450|1671|49092|7|106|2379|17|94.59|118.23|31.92|0.00|542.64|1608.03|2009.91|43.41|0.00|542.64|586.05|-1065.39| +2450825|47003|19|67276|1381450|1671|49092|7|268|2379|32|19.14|31.96|21.09|0.00|674.88|612.48|1022.72|40.49|0.00|674.88|715.37|62.40| +2452139|62654|8459|38014|93764|714|16064|2|39|2380|96|90.75|127.95|72.93|0.00|7001.28|8712.00|12283.20|140.02|0.00|7001.28|7141.30|-1710.72| +2452139|62654|9701|38014|93764|714|16064|2|34|2380|35|20.55|41.10|3.28|0.00|114.80|719.25|1438.50|3.44|0.00|114.80|118.24|-604.45| +2452139|62654|6415|38014|93764|714|16064|2|148|2380|40|51.47|61.24|11.02|0.00|440.80|2058.80|2449.60|0.00|0.00|440.80|440.80|-1618.00| +2452139|62654|12501|38014|93764|714|16064|2|55|2380|97|62.99|112.75|54.12|2257.34|5249.64|6110.03|10936.75|119.69|2257.34|2992.30|3111.99|-3117.73| +2452139|62654|8223|38014|93764|714|16064|2|119|2380|69|72.42|139.04|47.27|0.00|3261.63|4996.98|9593.76|0.00|0.00|3261.63|3261.63|-1735.35| +2452139|62654|13621|38014|93764|714|16064|2|45|2380|99|73.03|94.93|55.05|0.00|5449.95|7229.97|9398.07|54.49|0.00|5449.95|5504.44|-1780.02| +2452139|62654|14315|38014|93764|714|16064|2|252|2380|64|93.37|180.20|169.38|0.00|10840.32|5975.68|11532.80|975.62|0.00|10840.32|11815.94|4864.64| +2452139|62654|8969|38014|93764|714|16064|2|128|2380|93|19.90|28.65|18.33|1483.08|1704.69|1850.70|2664.45|15.51|1483.08|221.61|237.12|-1629.09| +2452139|62654|8393|38014|93764|714|16064|2|162|2380|70|50.33|93.11|7.44|0.00|520.80|3523.10|6517.70|41.66|0.00|520.80|562.46|-3002.30| +2452139|62654|13095|38014|93764|714|16064|2|229|2380|34|98.86|135.43|134.07|1276.34|4558.38|3361.24|4604.62|32.82|1276.34|3282.04|3314.86|-79.20| +2452139|62654|6823|38014|93764|714|16064|2|259|2380|35|84.53|132.71|78.29|0.00|2740.15|2958.55|4644.85|54.80|0.00|2740.15|2794.95|-218.40| +2452139|62654|7089|38014|93764|714|16064|2|46|2380|42|11.58|18.52|4.63|175.01|194.46|486.36|777.84|0.19|175.01|19.45|19.64|-466.91| +2452139|62654|8367|38014|93764|714|16064|2|137|2380|94|69.05|102.88|33.95|0.00|3191.30|6490.70|9670.72|255.30|0.00|3191.30|3446.60|-3299.40| +2452237|74420|7667|74431|1207759|3934|37662|2|119|2381|4|45.48|90.96|73.67|250.47|294.68|181.92|363.84|0.88|250.47|44.21|45.09|-137.71| +2452237||219|74431|1207759||37662|2||2381|||81.81|34.36|0.00|3367.28|4287.50|8017.38||0.00||3501.97|-920.22| +2452237|74420|13995|74431|1207759|3934|37662|2|51|2381|27|57.60|89.85|21.56|477.33|582.12|1555.20|2425.95|1.04|477.33|104.79|105.83|-1450.41| +2452237|74420|13277|74431|1207759|3934|37662|2|120|2381|8|75.90|119.92|65.95|0.00|527.60|607.20|959.36|47.48|0.00|527.60|575.08|-79.60| +2452237|74420|11097|74431|1207759|3934|37662|2|291|2381|68|79.44|127.89|121.49|0.00|8261.32|5401.92|8696.52|578.29|0.00|8261.32|8839.61|2859.40| +2452237|74420|16075|74431|1207759|3934|37662|2|201|2381|23|67.79|110.49|81.76|150.43|1880.48|1559.17|2541.27|69.20|150.43|1730.05|1799.25|170.88| +2452237|74420|10161|74431|1207759|3934|37662|2|258|2381|40|72.51|94.26|37.70|0.00|1508.00|2900.40|3770.40|60.32|0.00|1508.00|1568.32|-1392.40| +2452237|74420|12413|||3934|37662|||2381|||169.29|138.81|5863.33|9161.46|6458.76||0.00|5863.33||3298.13|| +2451364|36840|2750||||37494|7||2382||7.36|14.05|8.85|0.00|407.10|338.56|||0.00||443.73|68.54| +2451364|36840|2941|2479|88135|191|37494|7|154|2382|5|15.90|30.52|4.27|0.00|21.35|79.50|152.60|0.85|0.00|21.35|22.20|-58.15| +2451364|36840|12010|2479|88135|191|37494|7|43|2382|28|84.88|89.12|71.29|0.00|1996.12|2376.64|2495.36|179.65|0.00|1996.12|2175.77|-380.52| +2451364|36840|6050|2479|88135|191|37494|7|58|2382|17|79.77|149.16|29.83|0.00|507.11|1356.09|2535.72|20.28|0.00|507.11|527.39|-848.98| +2451364|36840|1942|2479|88135|191|37494|7|138|2382|68|96.12|127.83|109.93|0.00|7475.24|6536.16|8692.44|672.77|0.00|7475.24|8148.01|939.08| +2451364|36840|4406|2479|88135|191|37494|7|86|2382|35|30.56|37.89|4.16|0.00|145.60|1069.60|1326.15|0.00|0.00|145.60|145.60|-924.00| +2451364|36840|9334|2479|88135|191|37494|7|42|2382|75|76.87|148.35|72.69|0.00|5451.75|5765.25|11126.25|381.62|0.00|5451.75|5833.37|-313.50| +2451364|36840|376|2479|88135|191|37494|7|230|2382|93|44.25|58.85|48.25|0.00|4487.25|4115.25|5473.05|314.10|0.00|4487.25|4801.35|372.00| +2451364|36840|16058|2479|88135|191|37494|7|132|2382|80|16.95|23.39|16.37|0.00|1309.60|1356.00|1871.20|0.00|0.00|1309.60|1309.60|-46.40| +2451284|39961|11998|32864|1423852|5240|25381|8|58|2383|93|91.94|166.41|36.61|0.00|3404.73|8550.42|15476.13|306.42|0.00|3404.73|3711.15|-5145.69| +2451284|39961|9697|32864|1423852|5240|25381|8|36|2383|39|36.29|51.89|28.02|0.00|1092.78|1415.31|2023.71|21.85|0.00|1092.78|1114.63|-322.53| +2451284|39961|17548|32864|1423852|5240|25381|8|98|2383|78|42.62|51.99|22.35|0.00|1743.30|3324.36|4055.22|69.73|0.00|1743.30|1813.03|-1581.06| +2451284|39961|16856|32864|1423852|5240|25381|8|9|2383|25|48.90|84.10|48.77|0.00|1219.25|1222.50|2102.50|48.77|0.00|1219.25|1268.02|-3.25| +2451284|39961|13411|32864|1423852|5240|25381|8|138|2383|78|18.98|32.45|14.92|0.00|1163.76|1480.44|2531.10|11.63|0.00|1163.76|1175.39|-316.68| +2451284|39961|3268|32864|1423852|5240|25381|8|218|2383|35|58.94|107.86|40.98|0.00|1434.30|2062.90|3775.10|129.08|0.00|1434.30|1563.38|-628.60| +2451284|39961|3122|32864|1423852|5240|25381|8|35|2383|80|66.89|76.25|37.36|2570.36|2988.80|5351.20|6100.00|4.18|2570.36|418.44|422.62|-4932.76| +2451284|39961|16252|32864|1423852|5240|25381|8|66|2383|41|37.60|62.79|31.39|0.00|1286.99|1541.60|2574.39|90.08|0.00|1286.99|1377.07|-254.61| +2451284|39961|2506|32864|1423852|5240|25381|8|238|2383|56|11.35|18.72|8.42|315.91|471.52|635.60|1048.32|10.89|315.91|155.61|166.50|-479.99| +2451284|39961|14710|32864|1423852|5240|25381|8|21|2383|45|44.63|65.15|48.86|0.00|2198.70|2008.35|2931.75|109.93|0.00|2198.70|2308.63|190.35| +2451284|39961|14000|32864|1423852|5240|25381|8|8|2383|10|87.12|103.67|43.54|0.00|435.40|871.20|1036.70|34.83|0.00|435.40|470.23|-435.80| +2451284|39961|643|32864|1423852|5240|25381|8|162|2383|47|13.11|20.71|3.31|0.00|155.57|616.17|973.37|10.88|0.00|155.57|166.45|-460.60| +2451432|73638|16640|56768|363970|3172|43574|2|291|2384|4|62.16|116.86|84.13|0.00|336.52|248.64|467.44|3.36|0.00|336.52|339.88|87.88| +2451432|73638|16945|56768|363970|3172|43574|2|235|2384|6|55.22|89.45|42.04|0.00|252.24|331.32|536.70|15.13|0.00|252.24|267.37|-79.08| +2451432|73638|3250|56768|363970|3172|43574|2|273|2384|70|96.75|136.41|77.75|0.00|5442.50|6772.50|9548.70|380.97|0.00|5442.50|5823.47|-1330.00| +2451432|73638|7586|56768|363970|3172|43574|2|97|2384|19|62.48|106.84|72.65|0.00|1380.35|1187.12|2029.96|96.62|0.00|1380.35|1476.97|193.23| +2451432|73638|9493|56768|363970|3172|43574|2|44|2384|85|68.78|86.66|73.66|0.00|6261.10|5846.30|7366.10|375.66|0.00|6261.10|6636.76|414.80| +2451432|73638|4244|||||||2384|11|||107.40|||767.91||||||-637.95| +2451432|73638|12580|56768|363970|3172|43574|2|111|2384|83|20.33|36.79|22.80|0.00|1892.40|1687.39|3053.57|56.77|0.00|1892.40|1949.17|205.01| +2451432|73638|11977|56768|363970|3172|43574|2|232|2384|83|36.96|59.50|47.60|0.00|3950.80|3067.68|4938.50|237.04|0.00|3950.80|4187.84|883.12| +2451432|73638|3070|56768|363970|3172|43574|2|125|2384|2|85.75|125.19|90.13|79.31|180.26|171.50|250.38|0.00|79.31|100.95|100.95|-70.55| +2451432|73638|8780|56768|363970|3172|43574|2|162|2384|76|10.82|14.49|2.75|0.00|209.00|822.32|1101.24|16.72|0.00|209.00|225.72|-613.32| +2451432|73638|5767|56768|363970|3172|43574|2|48|2384|94|38.45|68.82|6.88|0.00|646.72|3614.30|6469.08|19.40|0.00|646.72|666.12|-2967.58| +2451432|73638|3457|56768|363970|3172|43574|2|118|2384|14|27.40|30.14|25.01|0.00|350.14|383.60|421.96|31.51|0.00|350.14|381.65|-33.46| +2451432|73638|16402|56768|363970|3172|43574|2|273|2384|51|31.41|37.37|19.43|673.83|990.93|1601.91|1905.87|9.51|673.83|317.10|326.61|-1284.81| +2451432|73638|6544|56768|363970|3172|43574|2|155|2384|72|55.51|107.68|76.45|0.00|5504.40|3996.72|7752.96|440.35|0.00|5504.40|5944.75|1507.68| +2451432|73638|5725|56768|363970|3172|43574|2|22|2384|47|83.35|91.68|86.17|0.00|4049.99|3917.45|4308.96|323.99|0.00|4049.99|4373.98|132.54| +2451432|73638|11282|56768|363970|3172|43574|2|106|2384|54|39.63|77.67|56.69|0.00|3061.26|2140.02|4194.18|30.61|0.00|3061.26|3091.87|921.24| +2452233|53198|6729|14440|1135734|581|37075|10|169|2385|76|32.45|35.69|12.84|0.00|975.84|2466.20|2712.44|58.55|0.00|975.84|1034.39|-1490.36| +2452233|53198|15705|14440|1135734|581|37075|10|136|2385|10|83.31|139.12|107.12|0.00|1071.20|833.10|1391.20|10.71|0.00|1071.20|1081.91|238.10| +2452233|53198|10513|14440|1135734|581|37075|10|275|2385|21|25.65|34.62|28.04|0.00|588.84|538.65|727.02|41.21|0.00|588.84|630.05|50.19| +2452233|53198|16961|14440|1135734|581|37075|10|57|2385|27|3.61|6.06|3.63|0.00|98.01|97.47|163.62|6.86|0.00|98.01|104.87|0.54| +2452233|53198|1219|14440|1135734|581|37075|10|247|2385|18|82.15|151.97|0.00|0.00|0.00|1478.70|2735.46|0.00|0.00|0.00|0.00|-1478.70| +2452233|53198|331|14440|1135734|581|37075|10|214|2385|26|83.99|83.99|45.35|0.00|1179.10|2183.74|2183.74|35.37|0.00|1179.10|1214.47|-1004.64| +2452233|53198|4375|14440|1135734|581|37075|10|73|2385|23|43.12|66.40|49.80|0.00|1145.40|991.76|1527.20|22.90|0.00|1145.40|1168.30|153.64| +2452233|53198|13871|14440|1135734|581|37075|10|53|2385|19|99.86|195.72|33.27|0.00|632.13|1897.34|3718.68|31.60|0.00|632.13|663.73|-1265.21| +2452233|53198|1769|14440|1135734|581|37075|10|212|2385|49|86.89|112.08|30.26|0.00|1482.74|4257.61|5491.92|118.61|0.00|1482.74|1601.35|-2774.87| +2452605|54696|4167|84560|882910|4107|15704|7|233|2386|20|55.85|92.15|6.45|0.00|129.00|1117.00|1843.00|7.74|0.00|129.00|136.74|-988.00| +2452605|54696|6373|84560|882910|4107|15704|7|225|2386|29|60.08|79.90|8.78|0.00|254.62|1742.32|2317.10|2.54|0.00|254.62|257.16|-1487.70| +2452605|54696|2247|84560|882910|4107|15704|7|194|2386|96|90.92|128.19|52.55|0.00|5044.80|8728.32|12306.24|403.58|0.00|5044.80|5448.38|-3683.52| +2452605|54696|4435|84560|882910|4107|15704|7|242|2386|28|51.84|85.01|5.10|0.00|142.80|1451.52|2380.28|12.85|0.00|142.80|155.65|-1308.72| +2452605|54696|3183|84560|882910|4107|15704|7|284|2386|7|41.11|81.80|56.44|0.00|395.08|287.77|572.60|27.65|0.00|395.08|422.73|107.31| +2452605|54696|1269|84560|882910|4107|15704|7|99|2386|48|71.42|127.84|34.51|0.00|1656.48|3428.16|6136.32|82.82|0.00|1656.48|1739.30|-1771.68| +2452605|54696|12888|84560|882910|4107|15704|7|64|2386|53|70.68|134.99|47.24|0.00|2503.72|3746.04|7154.47|200.29|0.00|2503.72|2704.01|-1242.32| +||13611|84560||4107||7||2386|24|48.01|62.41|||||1497.84|||1497.84|1632.64|345.60| +2452605|54696|13362|84560|882910|4107|15704|7|173|2386|35|21.44|39.66|9.51|0.00|332.85|750.40|1388.10|19.97|0.00|332.85|352.82|-417.55| +2452605|54696|16297|84560|882910|4107|15704|7|39|2386|98|47.76|81.66|65.32|0.00|6401.36|4680.48|8002.68|64.01|0.00|6401.36|6465.37|1720.88| +2452605|54696|16482|84560|882910|4107|15704|7|17|2386|26|25.46|43.02|12.47|0.00|324.22|661.96|1118.52|12.96|0.00|324.22|337.18|-337.74| +2452605|54696|15013|84560|882910|4107|15704|7|35|2386|48|5.79|8.22|7.64|0.00|366.72|277.92|394.56|11.00|0.00|366.72|377.72|88.80| +2452605|54696|4995|84560|882910|4107|15704|7|206|2386|58|9.79|15.56|2.80|0.00|162.40|567.82|902.48|6.49|0.00|162.40|168.89|-405.42| +2452605|54696|16755|84560|882910|4107|15704|7|104|2386|39|51.42|79.70|10.36|0.00|404.04|2005.38|3108.30|16.16|0.00|404.04|420.20|-1601.34| +2452605|54696|5634|84560|882910|4107|15704|7|138|2386|37|9.86|14.49|10.14|0.00|375.18|364.82|536.13|22.51|0.00|375.18|397.69|10.36| +2451864|66569|16268|45743|329078|766|34854|10||2387|13|88.62||85.02|187.89||1152.06|1417.00|64.21|187.89|||| +2451864|66569|13526|45743|329078|766|34854|10|145|2387|90|91.20|169.63|149.27|0.00|13434.30|8208.00|15266.70|671.71|0.00|13434.30|14106.01|5226.30| +2451864|66569|7811|45743|329078|766|34854|10|130|2387|2|95.87|138.05|20.70|0.00|41.40|191.74|276.10|3.72|0.00|41.40|45.12|-150.34| +2451864|66569|248|45743|329078|766|34854|10|135|2387|96|58.37|81.13|60.84|2161.03|5840.64|5603.52|7788.48|294.36|2161.03|3679.61|3973.97|-1923.91| +2451864|66569|6745|45743|329078|766|34854|10|244|2387|19|4.67|4.90|4.75|78.51|90.25|88.73|93.10|0.11|78.51|11.74|11.85|-76.99| +2451864|66569|8203|45743|329078|766|34854|10|77|2387|59|68.39|118.31|42.59|0.00|2512.81|4035.01|6980.29|50.25|0.00|2512.81|2563.06|-1522.20| +|66569|4205|||||10|267|2387|62|86.63|||0.00|6634.00||10365.78||0.00||6833.02|| +2451864|66569|14474|45743|329078|766|34854|10|239|2387|30|32.95|49.42|9.38|0.00|281.40|988.50|1482.60|14.07|0.00|281.40|295.47|-707.10| +2451864|66569|9415|45743|329078|766|34854|10|61|2387|79|45.60|45.60|15.96|0.00|1260.84|3602.40|3602.40|75.65|0.00|1260.84|1336.49|-2341.56| +2451864|66569|5821|45743|329078|766|34854|10|125|2387|85|84.07|116.01|88.16|0.00|7493.60|7145.95|9860.85|149.87|0.00|7493.60|7643.47|347.65| +2451864|66569|5207|45743|329078|766|34854|10|103|2387|1|61.28|118.88|35.66|0.00|35.66|61.28|118.88|2.13|0.00|35.66|37.79|-25.62| +2451864|66569|12595|45743|329078|766|34854|10|108|2387|67|75.56|115.60|8.09|0.00|542.03|5062.52|7745.20|16.26|0.00|542.03|558.29|-4520.49| +2451743|53374|3716|19136|1020196|3656|40763|4|21|2388|81|74.45|140.71|99.90|0.00|8091.90|6030.45|11397.51|0.00|0.00|8091.90|8091.90|2061.45| +2451743|53374|10831|19136|1020196|3656|40763|4|189|2388|10|24.46|32.28|0.96|0.00|9.60|244.60|322.80|0.28|0.00|9.60|9.88|-235.00| +||2762|||||4||2388||||||1726.14|||138.09||||454.74| +2451743|53374|6101|19136|1020196|3656|40763|4|226|2388|92|55.35|100.18|57.10|0.00|5253.20|5092.20|9216.56|420.25|0.00|5253.20|5673.45|161.00| +2451743|53374|12889|19136|1020196|3656|40763|4|14|2388|77|38.75|56.96|17.08|0.00|1315.16|2983.75|4385.92|0.00|0.00|1315.16|1315.16|-1668.59| +2451743|53374|5947|19136|1020196|3656|40763|4|270|2388|85|36.75|61.74|36.42|0.00|3095.70|3123.75|5247.90|154.78|0.00|3095.70|3250.48|-28.05| +2451743|53374|11048|19136|1020196|3656|40763|4|221|2388|62|15.43|26.38|15.56|77.17|964.72|956.66|1635.56|17.75|77.17|887.55|905.30|-69.11| +2451743|53374|8126|19136|1020196|3656|40763|4|229|2388|34|24.87|39.29|8.25|0.00|280.50|845.58|1335.86|8.41|0.00|280.50|288.91|-565.08| +2451743|53374|9527|19136|1020196|3656|40763|4|169|2388|44|31.19|58.94|8.25|0.00|363.00|1372.36|2593.36|14.52|0.00|363.00|377.52|-1009.36| +2451743|53374|17582|19136|1020196|3656|40763|4|130|2388|48|77.82|84.82|51.74|1142.41|2483.52|3735.36|4071.36|40.23|1142.41|1341.11|1381.34|-2394.25| +2451743|53374|12326|19136|1020196|3656|40763|4|62|2388|65|59.70|117.60|107.01|0.00|6955.65|3880.50|7644.00|417.33|0.00|6955.65|7372.98|3075.15| +2450870||7501|||||||2389|14|50.73||35.15|0.00||710.22|1406.16|39.36|0.00|492.10|531.46|| +2450870|51443|13825|41860|1844656|2652|39848|7|237|2389|55|63.07|79.46|20.65|0.00|1135.75|3468.85|4370.30|90.86|0.00|1135.75|1226.61|-2333.10| +2450870|51443|6469|41860|1844656|2652|39848|7|90|2389|10|77.33|88.92|56.01|548.89|560.10|773.30|889.20|1.00|548.89|11.21|12.21|-762.09| +2450870|51443|11602|41860|1844656|2652|39848|7|224|2389|57|40.82|46.53|1.86|0.00|106.02|2326.74|2652.21|5.30|0.00|106.02|111.32|-2220.72| +2450870|51443|2912|41860|1844656|2652|39848|7|169|2389|41|93.66|119.88|21.57|813.62|884.37|3840.06|4915.08|4.95|813.62|70.75|75.70|-3769.31| +2450870|51443|11966|41860|1844656|2652|39848|7|212|2389|62|21.22|22.49|21.81|0.00|1352.22|1315.64|1394.38|0.00|0.00|1352.22|1352.22|36.58| +2450870|51443|10417|41860|1844656|2652|39848|7|141|2389|76|73.33|135.66|44.76|0.00|3401.76|5573.08|10310.16|68.03|0.00|3401.76|3469.79|-2171.32| +2450870|51443|2822|41860|1844656|2652|39848|7|213|2389|63|37.48|56.59|35.65|0.00|2245.95|2361.24|3565.17|202.13|0.00|2245.95|2448.08|-115.29| +2450870|51443|6082|41860|1844656|2652|39848|7|221|2389|75|29.51|56.06|44.28|0.00|3321.00|2213.25|4204.50|132.84|0.00|3321.00|3453.84|1107.75| +2450870|51443|3751|41860|1844656|2652|39848|7|64|2389|45|41.34|50.84|5.59|0.00|251.55|1860.30|2287.80|5.03|0.00|251.55|256.58|-1608.75| +2450870|51443|16510|41860|1844656|2652|39848|7|194|2389|39|96.30|101.11|49.54|0.00|1932.06|3755.70|3943.29|154.56|0.00|1932.06|2086.62|-1823.64| +2450870|51443|12646|41860|1844656|2652|39848|7|282|2389|50|34.18|62.20|10.57|406.94|528.50|1709.00|3110.00|3.64|406.94|121.56|125.20|-1587.44| +2450870|51443|10885|41860|1844656|2652|39848|7|279|2389|80|15.23|24.67|7.89|0.00|631.20|1218.40|1973.60|44.18|0.00|631.20|675.38|-587.20| +2450999|69308|14935|12152|927394|3883|45125|4|121|2390|61|40.09|48.50|16.00|0.00|976.00|2445.49|2958.50|19.52|0.00|976.00|995.52|-1469.49| +2450999|69308|9086|12152|927394|3883|45125|4|239|2390|46|27.27|32.17|17.37|455.44|799.02|1254.42|1479.82|0.00|455.44|343.58|343.58|-910.84| +2450999|69308|5312|12152|927394|3883|45125|4|136|2390|11|64.72|116.49|116.49|0.00|1281.39|711.92|1281.39|38.44|0.00|1281.39|1319.83|569.47| +2450999|69308|1450|12152|927394|3883|45125|4|263|2390|2|78.03|148.25|100.81|0.00|201.62|156.06|296.50|10.08|0.00|201.62|211.70|45.56| +2450999|69308|5482|12152|927394|3883|45125|4|117|2390|35|23.66|24.60|18.20|0.00|637.00|828.10|861.00|12.74|0.00|637.00|649.74|-191.10| +2450999|69308|8714|12152|927394|3883|45125|4|227|2390|77|17.92|31.53|0.94|0.00|72.38|1379.84|2427.81|5.79|0.00|72.38|78.17|-1307.46| +2450999|69308|7207|12152|927394|3883|45125|4|51|2390|44|9.41|14.30|4.43|58.47|194.92|414.04|629.20|6.82|58.47|136.45|143.27|-277.59| +|69308|17036||927394|3883|||64|2390||50.17|||296.35|302.40||||296.35|6.05||-495.65| +2450999|69308|14170|12152|927394|3883|45125|4|153|2390|71|10.76|16.03|0.64|0.00|45.44|763.96|1138.13|0.90|0.00|45.44|46.34|-718.52| +2450999|69308|5132|12152|927394|3883|45125|4|217|2390|23|75.38|92.71|60.26|0.00|1385.98|1733.74|2132.33|41.57|0.00|1385.98|1427.55|-347.76| +2450999|69308|15457|12152|927394|3883|45125|4|290|2390|16|27.17|32.06|15.06|195.17|240.96|434.72|512.96|2.74|195.17|45.79|48.53|-388.93| +2450999|69308|3652|12152|927394|3883|45125|4|139|2390|93|60.99|101.85|45.83|0.00|4262.19|5672.07|9472.05|85.24|0.00|4262.19|4347.43|-1409.88| +2451831||800|||||||2391|47|||50.47||2372.09|1743.70|2929.04|71.16||2372.09|2443.25|| +2451831|41220|11111|24321|401737|3953|23369|1|153|2391|64|42.31|71.08|22.74|0.00|1455.36|2707.84|4549.12|87.32|0.00|1455.36|1542.68|-1252.48| +2451831|41220|8099|24321|401737|3953|23369|1|55|2391|49|73.14|121.41|104.41|0.00|5116.09|3583.86|5949.09|204.64|0.00|5116.09|5320.73|1532.23| +2451831|41220|7015|24321|401737|3953|23369|1|72|2391|42|87.69|117.50|49.35|1264.34|2072.70|3682.98|4935.00|32.33|1264.34|808.36|840.69|-2874.62| +2451831|41220|10955|24321|401737|3953|23369|1|203|2391|53|70.34|75.26|73.75|0.00|3908.75|3728.02|3988.78|39.08|0.00|3908.75|3947.83|180.73| +2451831|41220|16139|24321|401737|3953|23369|1|299|2391|3|22.05|29.76|21.72|0.00|65.16|66.15|89.28|5.21|0.00|65.16|70.37|-0.99| +2451831|41220|11737|24321|401737|3953|23369|1|101|2391|49|12.08|23.67|14.20|0.00|695.80|591.92|1159.83|0.00|0.00|695.80|695.80|103.88| +2451831|41220|11543|24321|401737|3953|23369|1|225|2391|81|79.29|79.29|14.27|0.00|1155.87|6422.49|6422.49|34.67|0.00|1155.87|1190.54|-5266.62| +2451831|41220|11293|24321|401737|3953|23369|1|74|2391|2|8.01|13.29|0.26|0.07|0.52|16.02|26.58|0.02|0.07|0.45|0.47|-15.57| +2452471|53653|14989|4964|1488195|4579|30963|2|88|2392|72|64.64|90.49|63.34|1276.93|4560.48|4654.08|6515.28|131.34|1276.93|3283.55|3414.89|-1370.53| +2452471|53653|6708|4964|1488195|4579|30963|2|146|2392|31|48.89|62.09|26.07|0.00|808.17|1515.59|1924.79|32.32|0.00|808.17|840.49|-707.42| +2452471|53653|17976|4964|1488195|4579|30963|2|47|2392|75|15.29|27.36|25.99|0.00|1949.25|1146.75|2052.00|97.46|0.00|1949.25|2046.71|802.50| +2452471|53653|2737|4964|1488195|4579|30963|2|16|2392|85|91.90|103.84|46.72|0.00|3971.20|7811.50|8826.40|39.71|0.00|3971.20|4010.91|-3840.30| +2452471|53653|2355|4964|1488195|4579|30963|2|61|2392|96|77.50|106.17|19.11|0.00|1834.56|7440.00|10192.32|110.07|0.00|1834.56|1944.63|-5605.44| +2452471|53653|14664|4964|1488195|4579|30963|2|93|2392|12|33.49|61.28|51.47|43.23|617.64|401.88|735.36|0.00|43.23|574.41|574.41|172.53| +2452471|53653|6171|4964|1488195|4579|30963|2|93|2392|67|14.40|17.28|6.39|329.66|428.13|964.80|1157.76|7.87|329.66|98.47|106.34|-866.33| +2452471|53653|16597|4964|1488195|4579||2||2392|71|54.36|56.53||0.00||||0.00|0.00|1885.76|1885.76|-1973.80| +2452471|53653|5295|4964|1488195|4579|30963|2|22|2392|56|17.54|22.45|2.24|0.00|125.44|982.24|1257.20|2.50|0.00|125.44|127.94|-856.80| +2451447|51965|14665|50293|1887477|4661|2645|4|203|2393|27|69.22|113.52|103.30|2426.51|2789.10|1868.94|3065.04|3.62|2426.51|362.59|366.21|-1506.35| +2451447|51965|9490|50293|1887477|4661|2645|4|32|2393|12|81.17|91.72|4.58|0.00|54.96|974.04|1100.64|0.54|0.00|54.96|55.50|-919.08| +2451447|51965|8036|50293|1887477|4661|2645|4|273|2393|47|11.52|11.86|1.18|0.00|55.46|541.44|557.42|2.77|0.00|55.46|58.23|-485.98| +2451447|51965|14582|50293|1887477|4661|2645|4|172|2393|86|27.79|34.18|12.98|0.00|1116.28|2389.94|2939.48|0.00|0.00|1116.28|1116.28|-1273.66| +2451447|51965|10976|50293|1887477|4661|2645|4|154|2393|28|23.07|36.21|13.39|0.00|374.92|645.96|1013.88|29.99|0.00|374.92|404.91|-271.04| +2451447|51965|7570|50293|1887477|4661|2645|4|136|2393|27|66.84|104.93|74.50|0.00|2011.50|1804.68|2833.11|80.46|0.00|2011.50|2091.96|206.82| +2451447|51965|1250|50293|1887477|4661|2645|4|269|2393|76|80.09|116.13|78.96|0.00|6000.96|6086.84|8825.88|240.03|0.00|6000.96|6240.99|-85.88| +2451447|51965|5785|50293|1887477|4661|2645|4|130|2393|61|75.90|120.68|89.30|3649.69|5447.30|4629.90|7361.48|35.95|3649.69|1797.61|1833.56|-2832.29| +2451447|51965|2719|50293|1887477|4661|2645|4|251|2393|32|94.07|160.85|128.68|3417.74|4117.76|3010.24|5147.20|28.00|3417.74|700.02|728.02|-2310.22| +2451447|51965|10898|50293|1887477|4661|2645|4|9|2393|83|3.49|6.56|2.36|0.00|195.88|289.67|544.48|15.67|0.00|195.88|211.55|-93.79| +2451447|51965|4306|50293|1887477|4661|2645|4|224|2393|70|82.61|96.65|67.65|0.00|4735.50|5782.70|6765.50|142.06|0.00|4735.50|4877.56|-1047.20| +2451729|68715|10742|72943|145462|6781|900|1|45|2394|99|61.53|92.29|49.83|0.00|4933.17|6091.47|9136.71|49.33|0.00|4933.17|4982.50|-1158.30| +2451729|68715|6959|72943|145462|6781|900|1|115|2394|38|64.98|85.12|68.09|0.00|2587.42|2469.24|3234.56|155.24|0.00|2587.42|2742.66|118.18| +2451729|68715|14957|72943|145462|6781|900|1|43|2394|30|31.87|49.39|12.34|0.00|370.20|956.10|1481.70|33.31|0.00|370.20|403.51|-585.90| +2451729|68715|4670|72943|145462|6781|900|1|207|2394|14|92.19|164.09|114.86|498.49|1608.04|1290.66|2297.26|88.76|498.49|1109.55|1198.31|-181.11| +|68715|12695|72943|145462||900|||2394|29|||7.48|||532.15|638.58|||||-315.23| +2451729|68715|16171|72943|145462|6781|900|1|67|2394|68|15.44|15.74|6.29|115.48|427.72|1049.92|1070.32|6.24|115.48|312.24|318.48|-737.68| +2451729|68715|9983|72943|145462|6781|900|1|137|2394|25|78.99|100.31|7.02|22.81|175.50|1974.75|2507.75|12.21|22.81|152.69|164.90|-1822.06| +2451729|68715|6361|72943|145462|6781|900|1|254|2394|64|94.95|181.35|108.81|0.00|6963.84|6076.80|11606.40|69.63|0.00|6963.84|7033.47|887.04| +2451729|68715|5960|72943|145462|6781|900|1|253|2394|43|1.62|3.24|1.26|0.00|54.18|69.66|139.32|3.25|0.00|54.18|57.43|-15.48| +2451729|68715|12325|72943|145462|6781|900|1|275|2394|96|15.11|19.79|3.16|0.00|303.36|1450.56|1899.84|3.03|0.00|303.36|306.39|-1147.20| +2451729|68715|7555|72943|145462|6781|900|1|200|2394|67|84.15|109.39|31.72|0.00|2125.24|5638.05|7329.13|21.25|0.00|2125.24|2146.49|-3512.81| +2451973|32200|1725|34028|240790|2752|44761|4|41|2395|55|52.96|93.73|0.00|0.00|0.00|2912.80|5155.15|0.00|0.00|0.00|0.00|-2912.80| +2451973|32200|17827|34028|240790|2752|44761|4|21|2395|2|73.44|110.16|30.84|0.00|61.68|146.88|220.32|0.61|0.00|61.68|62.29|-85.20| +2451973|32200|957|34028|240790|2752|44761|4|210|2395|69|87.29|118.71|68.85|0.00|4750.65|6023.01|8190.99|47.50|0.00|4750.65|4798.15|-1272.36| +2451973|32200|3003|34028|240790|2752|44761|4|187|2395|77|30.37|40.39|5.25|0.00|404.25|2338.49|3110.03|12.12|0.00|404.25|416.37|-1934.24| +2451973|32200|13169|34028|240790|2752|44761|4|18|2395|43|51.60|86.68|11.26|0.00|484.18|2218.80|3727.24|29.05|0.00|484.18|513.23|-1734.62| +2451973|32200|5333|34028|240790|2752|44761|4|161|2395|5|85.81|113.26|14.72|0.00|73.60|429.05|566.30|4.41|0.00|73.60|78.01|-355.45| +2451973|32200|1611|34028|240790|2752|44761|4|33|2395|48|30.88|51.26|40.49|0.00|1943.52|1482.24|2460.48|19.43|0.00|1943.52|1962.95|461.28| +2451973|32200|10661|34028|240790|2752|44761|4|75|2395|50|74.69|107.55|95.71|0.00|4785.50|3734.50|5377.50|95.71|0.00|4785.50|4881.21|1051.00| +2451973|32200|5793|34028|240790|2752|44761|4|262|2395|49|82.28|136.58|34.14|1187.73|1672.86|4031.72|6692.42|43.66|1187.73|485.13|528.79|-3546.59| +||9075|34028||2752||||2395|69||173.86|36.51|||6519.81|11996.34|176.34|||2695.53|-4000.62| +2451973|32200|11847|34028|240790|2752|44761|4|112|2395|99|77.16|137.34|123.60|0.00|12236.40|7638.84|13596.66|1101.27|0.00|12236.40|13337.67|4597.56| +2451542|52783|2782|5595|439685|5328|5771|7|71|2396|31|77.12|104.11|98.90|0.00|3065.90|2390.72|3227.41|122.63|0.00|3065.90|3188.53|675.18| +2451542|52783|10165|5595|439685|5328|5771|7|234|2396|31|98.74|165.88|132.70|0.00|4113.70|3060.94|5142.28|246.82|0.00|4113.70|4360.52|1052.76| +2451542|52783|14773|5595|439685|5328|5771|7|272|2396|100|34.10|40.92|8.59|0.00|859.00|3410.00|4092.00|60.13|0.00|859.00|919.13|-2551.00| +2451542|52783|8528|5595|439685|5328|5771|7|117|2396|39|84.31|126.46|93.58|0.00|3649.62|3288.09|4931.94|328.46|0.00|3649.62|3978.08|361.53| +2451542|52783|12164|5595|439685|5328|5771|7|289|2396|64|17.44|30.69|6.75|0.00|432.00|1116.16|1964.16|8.64|0.00|432.00|440.64|-684.16| +2451542|52783|12007|5595|439685|5328|5771|7|70|2396|14|85.69|107.11|23.56|0.00|329.84|1199.66|1499.54|6.59|0.00|329.84|336.43|-869.82| +2451542|52783|13942|5595|439685|5328|5771|7|156|2396|29|54.91|107.62|74.25|0.00|2153.25|1592.39|3120.98|193.79|0.00|2153.25|2347.04|560.86| +2451542|52783|9019|5595|439685|5328|5771|7|115|2396|17|49.55|81.26|73.94|0.00|1256.98|842.35|1381.42|100.55|0.00|1256.98|1357.53|414.63| +2451542|52783|3512|5595|439685|5328|5771|7|280|2396|24|21.71|31.26|3.75|0.00|90.00|521.04|750.24|4.50|0.00|90.00|94.50|-431.04| +2451542|52783|9445|5595|439685|5328|5771|7|210|2396|53|58.65|114.36|76.62|0.00|4060.86|3108.45|6061.08|162.43|0.00|4060.86|4223.29|952.41| +2451542|52783|11858|5595|439685|5328|5771|7|251|2396|42|26.47|34.94|11.87|59.82|498.54|1111.74|1467.48|35.09|59.82|438.72|473.81|-673.02| +2451542|52783|10640|5595|439685|5328|5771|7|19|2396|78|84.95|166.50|114.88|0.00|8960.64|6626.10|12987.00|89.60|0.00|8960.64|9050.24|2334.54| +2451542|52783|17948|5595|439685|5328|5771|7|58|2396|86|52.43|56.10|15.70|0.00|1350.20|4508.98|4824.60|27.00|0.00|1350.20|1377.20|-3158.78| +2451064|63646|14504|94959|906179|1399|266|10|20|2397|55|32.92|54.31|10.31|0.00|567.05|1810.60|2987.05|0.00|0.00|567.05|567.05|-1243.55| +2451064|63646|10864|94959|906179|1399|266|10|138|2397|87|23.08|31.15|27.41|166.92|2384.67|2007.96|2710.05|0.00|166.92|2217.75|2217.75|209.79| +2451064|63646|11204|94959|906179|1399|266|10|16|2397|85|6.18|8.77|6.13|422.05|521.05|525.30|745.45|0.00|422.05|99.00|99.00|-426.30| +2451064|63646|14246|94959|906179|1399|266|10|109|2397|71|49.96|98.42|12.79|0.00|908.09|3547.16|6987.82|63.56|0.00|908.09|971.65|-2639.07| +|63646|7474|94959||1399||10|49|2397||||0.61|0.00|||15.35|0.00|0.00|||-10.05| +2451064|63646|17060|94959|906179|1399|266|10|169|2397|63|28.40|51.40|1.02|0.00|64.26|1789.20|3238.20|5.78|0.00|64.26|70.04|-1724.94| +2451064|63646|14078|94959|906179|1399|266|10|160|2397|39|27.39|53.41|8.01|0.00|312.39|1068.21|2082.99|6.24|0.00|312.39|318.63|-755.82| +2451064|63646|1792|94959|906179|1399|266|10|206|2397|40|51.30|71.30|17.11|0.00|684.40|2052.00|2852.00|54.75|0.00|684.40|739.15|-1367.60| +2451064|63646|16303|94959|906179|1399|266|10|291|2397|8|26.43|32.50|8.45|0.00|67.60|211.44|260.00|6.08|0.00|67.60|73.68|-143.84| +2451392|49066|7009|92740|1249991|6889|6540|2|107|2398|39|46.91|75.05|10.50|0.00|409.50|1829.49|2926.95|32.76|0.00|409.50|442.26|-1419.99| +2451392|49066|13754|92740|1249991|6889|6540|2|284|2398|36|46.34|60.70|43.70|0.00|1573.20|1668.24|2185.20|31.46|0.00|1573.20|1604.66|-95.04| +2451392|49066|2702|92740|1249991|6889|6540|2|171|2398|20|1.50|2.67|2.10|0.00|42.00|30.00|53.40|2.10|0.00|42.00|44.10|12.00| +2451392|49066|4912|92740|1249991|6889|6540|2|216|2398|38|8.94|15.01|13.35|350.03|507.30|339.72|570.38|6.29|350.03|157.27|163.56|-182.45| +2451392|49066|14162|92740|1249991|6889|6540|2|129|2398|10|91.09|108.39|68.28|0.00|682.80|910.90|1083.90|6.82|0.00|682.80|689.62|-228.10| +2451392|49066|4648|92740|1249991|6889|6540|2|128|2398|45|97.05|181.48|7.25|0.00|326.25|4367.25|8166.60|0.00|0.00|326.25|326.25|-4041.00| +2451392|49066|13730|92740|1249991|6889|6540|2|36|2398|30|24.41|27.33|21.86|0.00|655.80|732.30|819.90|19.67|0.00|655.80|675.47|-76.50| +2451392|49066|11780|92740|1249991|6889|6540|2|39|2398|53|1.16|1.68|1.27|39.71|67.31|61.48|89.04|1.65|39.71|27.60|29.25|-33.88| +2451392|49066|11323|92740|1249991|6889|6540|2|144|2398|64|36.01|43.21|24.62|0.00|1575.68|2304.64|2765.44|110.29|0.00|1575.68|1685.97|-728.96| +2451392|49066|15734|92740|1249991|6889|6540|2|288|2398|5|19.98|33.56|1.34|0.00|6.70|99.90|167.80|0.13|0.00|6.70|6.83|-93.20| +2451392|49066|1354|92740|1249991|6889|6540|2|135|2398|28|77.23|99.62|18.92|0.00|529.76|2162.44|2789.36|37.08|0.00|529.76|566.84|-1632.68| +2451392|49066|9277|92740|1249991|6889|6540|2|172|2398|21|44.52|73.01|30.66|0.00|643.86|934.92|1533.21|19.31|0.00|643.86|663.17|-291.06| +2451392|49066|12794|92740|1249991|6889|6540|2|249|2398|84|14.63|15.06|12.65|0.00|1062.60|1228.92|1265.04|74.38|0.00|1062.60|1136.98|-166.32| +2452151|37574|5657|92742|94370|3998|45626|7|188|2399|49|53.10|105.66|98.26|962.94|4814.74|2601.90|5177.34|154.07|962.94|3851.80|4005.87|1249.90| +2452151|37574|8919|92742|94370|3998|45626|7|72|2399|41|26.69|33.09|21.17|720.41|867.97|1094.29|1356.69|8.85|720.41|147.56|156.41|-946.73| +2452151|37574|7167|92742|94370|3998|45626|7|283|2399|1|84.24|155.00|18.60|0.00|18.60|84.24|155.00|0.00|0.00|18.60|18.60|-65.64| +2452151|37574|11107|92742|94370|3998|45626|7|208|2399|19|9.58|16.86|11.29|0.00|214.51|182.02|320.34|6.43|0.00|214.51|220.94|32.49| +2452151|37574|7663|92742|94370|3998|45626|7|257|2399|76|91.05|111.99|110.87|0.00|8426.12|6919.80|8511.24|674.08|0.00|8426.12|9100.20|1506.32| +2452151|37574|13575|92742|94370|3998|45626|7|75|2399|17|72.37|116.51|19.80|0.00|336.60|1230.29|1980.67|6.73|0.00|336.60|343.33|-893.69| +2452151|37574|10481|92742|94370|3998|45626|7|282|2399|90|35.00|44.45|28.00|0.00|2520.00|3150.00|4000.50|100.80|0.00|2520.00|2620.80|-630.00| +2452151|37574|10849|92742|94370|3998|45626|7|127|2399|34|92.61|178.73|8.93|0.00|303.62|3148.74|6076.82|9.10|0.00|303.62|312.72|-2845.12| +2452151|37574|16853|92742|94370|3998|45626|7|20|2399|82|88.80|147.40|0.00|0.00|0.00|7281.60|12086.80|0.00|0.00|0.00|0.00|-7281.60| +2452151|37574|13143|92742|94370|3998|45626|7|190|2399|75|37.27|40.62|7.71|514.64|578.25|2795.25|3046.50|1.90|514.64|63.61|65.51|-2731.64| +2452151|37574|303|92742|94370|3998|45626|7|89|2399|75|8.89|17.42|16.37|0.00|1227.75|666.75|1306.50|61.38|0.00|1227.75|1289.13|561.00| +2452151||4455|||3998||||2399|26|22.97||26.04|0.00||||40.62|0.00|677.04||79.82| +2452151|37574|7045|92742|94370|3998|45626|7|38|2399|27|48.32|96.64|15.46|0.00|417.42|1304.64|2609.28|4.17|0.00|417.42|421.59|-887.22| +2451850|63835|7801|3499|1351185|6071|2557|7|211|2400|19|62.21|92.07|28.54|86.76|542.26|1181.99|1749.33|27.33|86.76|455.50|482.83|-726.49| +2451850|63835|6284|3499|1351185|6071|2557|7|261|2400|54|23.94|47.88|26.33|0.00|1421.82|1292.76|2585.52|56.87|0.00|1421.82|1478.69|129.06| +2451850|63835|5839|3499||6071|2557|7||2400|93|||55.88|5196.84||6617.88||0.00|5196.84|0.00|0.00|-6617.88| +2451850|63835|5689|3499|1351185|6071|2557|7|111|2400|58|97.61|124.94|99.95|0.00|5797.10|5661.38|7246.52|231.88|0.00|5797.10|6028.98|135.72| +2451850|63835|9611|3499|1351185|6071|2557|7|120|2400|46|54.21|54.75|28.47|641.71|1309.62|2493.66|2518.50|26.71|641.71|667.91|694.62|-1825.75| diff --git a/spark-connector/spark-common/src/test/resources/data/tpcds/time_dim/time_dim.dat b/spark-connector/spark-common/src/test/resources/data/tpcds/time_dim/time_dim.dat new file mode 100644 index 00000000000..a5cac387af7 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/data/tpcds/time_dim/time_dim.dat @@ -0,0 +1,86400 @@ +0|AAAAAAAABAAAAAAA|0|0|0|0|AM|third|night|| +1|AAAAAAAACAAAAAAA|1|0|0|1|AM|third|night|| +2|AAAAAAAADAAAAAAA|2|0|0|2|AM|third|night|| +3|AAAAAAAAEAAAAAAA|3|0|0|3|AM|third|night|| +4|AAAAAAAAFAAAAAAA|4|0|0|4|AM|third|night|| +5|AAAAAAAAGAAAAAAA|5|0|0|5|AM|third|night|| +6|AAAAAAAAHAAAAAAA|6|0|0|6|AM|third|night|| +7|AAAAAAAAIAAAAAAA|7|0|0|7|AM|third|night|| +8|AAAAAAAAJAAAAAAA|8|0|0|8|AM|third|night|| +9|AAAAAAAAKAAAAAAA|9|0|0|9|AM|third|night|| +10|AAAAAAAALAAAAAAA|10|0|0|10|AM|third|night|| +11|AAAAAAAAMAAAAAAA|11|0|0|11|AM|third|night|| +12|AAAAAAAANAAAAAAA|12|0|0|12|AM|third|night|| +13|AAAAAAAAOAAAAAAA|13|0|0|13|AM|third|night|| +14|AAAAAAAAPAAAAAAA|14|0|0|14|AM|third|night|| +15|AAAAAAAAABAAAAAA|15|0|0|15|AM|third|night|| +16|AAAAAAAABBAAAAAA|16|0|0|16|AM|third|night|| +17|AAAAAAAACBAAAAAA|17|0|0|17|AM|third|night|| +18|AAAAAAAADBAAAAAA|18|0|0|18|AM|third|night|| +19|AAAAAAAAEBAAAAAA|19|0|0|19|AM|third|night|| +20|AAAAAAAAFBAAAAAA|20|0|0|20|AM|third|night|| +21|AAAAAAAAGBAAAAAA|21|0|0|21|AM|third|night|| +22|AAAAAAAAHBAAAAAA|22|0|0|22|AM|third|night|| +23|AAAAAAAAIBAAAAAA|23|0|0|23|AM|third|night|| +24|AAAAAAAAJBAAAAAA|24|0|0|24|AM|third|night|| +25|AAAAAAAAKBAAAAAA|25|0|0|25|AM|third|night|| +26|AAAAAAAALBAAAAAA|26|0|0|26|AM|third|night|| +27|AAAAAAAAMBAAAAAA|27|0|0|27|AM|third|night|| +28|AAAAAAAANBAAAAAA|28|0|0|28|AM|third|night|| +29|AAAAAAAAOBAAAAAA|29|0|0|29|AM|third|night|| +30|AAAAAAAAPBAAAAAA|30|0|0|30|AM|third|night|| +31|AAAAAAAAACAAAAAA|31|0|0|31|AM|third|night|| +32|AAAAAAAABCAAAAAA|32|0|0|32|AM|third|night|| +33|AAAAAAAACCAAAAAA|33|0|0|33|AM|third|night|| +34|AAAAAAAADCAAAAAA|34|0|0|34|AM|third|night|| +35|AAAAAAAAECAAAAAA|35|0|0|35|AM|third|night|| +36|AAAAAAAAFCAAAAAA|36|0|0|36|AM|third|night|| +37|AAAAAAAAGCAAAAAA|37|0|0|37|AM|third|night|| +38|AAAAAAAAHCAAAAAA|38|0|0|38|AM|third|night|| +39|AAAAAAAAICAAAAAA|39|0|0|39|AM|third|night|| +40|AAAAAAAAJCAAAAAA|40|0|0|40|AM|third|night|| +41|AAAAAAAAKCAAAAAA|41|0|0|41|AM|third|night|| +42|AAAAAAAALCAAAAAA|42|0|0|42|AM|third|night|| +43|AAAAAAAAMCAAAAAA|43|0|0|43|AM|third|night|| +44|AAAAAAAANCAAAAAA|44|0|0|44|AM|third|night|| +45|AAAAAAAAOCAAAAAA|45|0|0|45|AM|third|night|| +46|AAAAAAAAPCAAAAAA|46|0|0|46|AM|third|night|| +47|AAAAAAAAADAAAAAA|47|0|0|47|AM|third|night|| +48|AAAAAAAABDAAAAAA|48|0|0|48|AM|third|night|| +49|AAAAAAAACDAAAAAA|49|0|0|49|AM|third|night|| +50|AAAAAAAADDAAAAAA|50|0|0|50|AM|third|night|| +51|AAAAAAAAEDAAAAAA|51|0|0|51|AM|third|night|| +52|AAAAAAAAFDAAAAAA|52|0|0|52|AM|third|night|| +53|AAAAAAAAGDAAAAAA|53|0|0|53|AM|third|night|| +54|AAAAAAAAHDAAAAAA|54|0|0|54|AM|third|night|| +55|AAAAAAAAIDAAAAAA|55|0|0|55|AM|third|night|| +56|AAAAAAAAJDAAAAAA|56|0|0|56|AM|third|night|| +57|AAAAAAAAKDAAAAAA|57|0|0|57|AM|third|night|| +58|AAAAAAAALDAAAAAA|58|0|0|58|AM|third|night|| +59|AAAAAAAAMDAAAAAA|59|0|0|59|AM|third|night|| +60|AAAAAAAANDAAAAAA|60|0|1|0|AM|third|night|| +61|AAAAAAAAODAAAAAA|61|0|1|1|AM|third|night|| +62|AAAAAAAAPDAAAAAA|62|0|1|2|AM|third|night|| +63|AAAAAAAAAEAAAAAA|63|0|1|3|AM|third|night|| +64|AAAAAAAABEAAAAAA|64|0|1|4|AM|third|night|| +65|AAAAAAAACEAAAAAA|65|0|1|5|AM|third|night|| +66|AAAAAAAADEAAAAAA|66|0|1|6|AM|third|night|| +67|AAAAAAAAEEAAAAAA|67|0|1|7|AM|third|night|| +68|AAAAAAAAFEAAAAAA|68|0|1|8|AM|third|night|| +69|AAAAAAAAGEAAAAAA|69|0|1|9|AM|third|night|| +70|AAAAAAAAHEAAAAAA|70|0|1|10|AM|third|night|| +71|AAAAAAAAIEAAAAAA|71|0|1|11|AM|third|night|| +72|AAAAAAAAJEAAAAAA|72|0|1|12|AM|third|night|| +73|AAAAAAAAKEAAAAAA|73|0|1|13|AM|third|night|| +74|AAAAAAAALEAAAAAA|74|0|1|14|AM|third|night|| +75|AAAAAAAAMEAAAAAA|75|0|1|15|AM|third|night|| +76|AAAAAAAANEAAAAAA|76|0|1|16|AM|third|night|| +77|AAAAAAAAOEAAAAAA|77|0|1|17|AM|third|night|| +78|AAAAAAAAPEAAAAAA|78|0|1|18|AM|third|night|| +79|AAAAAAAAAFAAAAAA|79|0|1|19|AM|third|night|| +80|AAAAAAAABFAAAAAA|80|0|1|20|AM|third|night|| +81|AAAAAAAACFAAAAAA|81|0|1|21|AM|third|night|| +82|AAAAAAAADFAAAAAA|82|0|1|22|AM|third|night|| +83|AAAAAAAAEFAAAAAA|83|0|1|23|AM|third|night|| +84|AAAAAAAAFFAAAAAA|84|0|1|24|AM|third|night|| +85|AAAAAAAAGFAAAAAA|85|0|1|25|AM|third|night|| +86|AAAAAAAAHFAAAAAA|86|0|1|26|AM|third|night|| +87|AAAAAAAAIFAAAAAA|87|0|1|27|AM|third|night|| +88|AAAAAAAAJFAAAAAA|88|0|1|28|AM|third|night|| +89|AAAAAAAAKFAAAAAA|89|0|1|29|AM|third|night|| +90|AAAAAAAALFAAAAAA|90|0|1|30|AM|third|night|| +91|AAAAAAAAMFAAAAAA|91|0|1|31|AM|third|night|| +92|AAAAAAAANFAAAAAA|92|0|1|32|AM|third|night|| +93|AAAAAAAAOFAAAAAA|93|0|1|33|AM|third|night|| +94|AAAAAAAAPFAAAAAA|94|0|1|34|AM|third|night|| +95|AAAAAAAAAGAAAAAA|95|0|1|35|AM|third|night|| +96|AAAAAAAABGAAAAAA|96|0|1|36|AM|third|night|| +97|AAAAAAAACGAAAAAA|97|0|1|37|AM|third|night|| +98|AAAAAAAADGAAAAAA|98|0|1|38|AM|third|night|| +99|AAAAAAAAEGAAAAAA|99|0|1|39|AM|third|night|| +100|AAAAAAAAFGAAAAAA|100|0|1|40|AM|third|night|| +101|AAAAAAAAGGAAAAAA|101|0|1|41|AM|third|night|| +102|AAAAAAAAHGAAAAAA|102|0|1|42|AM|third|night|| +103|AAAAAAAAIGAAAAAA|103|0|1|43|AM|third|night|| +104|AAAAAAAAJGAAAAAA|104|0|1|44|AM|third|night|| +105|AAAAAAAAKGAAAAAA|105|0|1|45|AM|third|night|| +106|AAAAAAAALGAAAAAA|106|0|1|46|AM|third|night|| +107|AAAAAAAAMGAAAAAA|107|0|1|47|AM|third|night|| +108|AAAAAAAANGAAAAAA|108|0|1|48|AM|third|night|| +109|AAAAAAAAOGAAAAAA|109|0|1|49|AM|third|night|| +110|AAAAAAAAPGAAAAAA|110|0|1|50|AM|third|night|| +111|AAAAAAAAAHAAAAAA|111|0|1|51|AM|third|night|| +112|AAAAAAAABHAAAAAA|112|0|1|52|AM|third|night|| +113|AAAAAAAACHAAAAAA|113|0|1|53|AM|third|night|| +114|AAAAAAAADHAAAAAA|114|0|1|54|AM|third|night|| +115|AAAAAAAAEHAAAAAA|115|0|1|55|AM|third|night|| +116|AAAAAAAAFHAAAAAA|116|0|1|56|AM|third|night|| +117|AAAAAAAAGHAAAAAA|117|0|1|57|AM|third|night|| +118|AAAAAAAAHHAAAAAA|118|0|1|58|AM|third|night|| +119|AAAAAAAAIHAAAAAA|119|0|1|59|AM|third|night|| +120|AAAAAAAAJHAAAAAA|120|0|2|0|AM|third|night|| +121|AAAAAAAAKHAAAAAA|121|0|2|1|AM|third|night|| +122|AAAAAAAALHAAAAAA|122|0|2|2|AM|third|night|| +123|AAAAAAAAMHAAAAAA|123|0|2|3|AM|third|night|| +124|AAAAAAAANHAAAAAA|124|0|2|4|AM|third|night|| +125|AAAAAAAAOHAAAAAA|125|0|2|5|AM|third|night|| +126|AAAAAAAAPHAAAAAA|126|0|2|6|AM|third|night|| +127|AAAAAAAAAIAAAAAA|127|0|2|7|AM|third|night|| +128|AAAAAAAABIAAAAAA|128|0|2|8|AM|third|night|| +129|AAAAAAAACIAAAAAA|129|0|2|9|AM|third|night|| +130|AAAAAAAADIAAAAAA|130|0|2|10|AM|third|night|| +131|AAAAAAAAEIAAAAAA|131|0|2|11|AM|third|night|| +132|AAAAAAAAFIAAAAAA|132|0|2|12|AM|third|night|| +133|AAAAAAAAGIAAAAAA|133|0|2|13|AM|third|night|| +134|AAAAAAAAHIAAAAAA|134|0|2|14|AM|third|night|| +135|AAAAAAAAIIAAAAAA|135|0|2|15|AM|third|night|| +136|AAAAAAAAJIAAAAAA|136|0|2|16|AM|third|night|| +137|AAAAAAAAKIAAAAAA|137|0|2|17|AM|third|night|| +138|AAAAAAAALIAAAAAA|138|0|2|18|AM|third|night|| +139|AAAAAAAAMIAAAAAA|139|0|2|19|AM|third|night|| +140|AAAAAAAANIAAAAAA|140|0|2|20|AM|third|night|| +141|AAAAAAAAOIAAAAAA|141|0|2|21|AM|third|night|| +142|AAAAAAAAPIAAAAAA|142|0|2|22|AM|third|night|| +143|AAAAAAAAAJAAAAAA|143|0|2|23|AM|third|night|| +144|AAAAAAAABJAAAAAA|144|0|2|24|AM|third|night|| +145|AAAAAAAACJAAAAAA|145|0|2|25|AM|third|night|| +146|AAAAAAAADJAAAAAA|146|0|2|26|AM|third|night|| +147|AAAAAAAAEJAAAAAA|147|0|2|27|AM|third|night|| +148|AAAAAAAAFJAAAAAA|148|0|2|28|AM|third|night|| +149|AAAAAAAAGJAAAAAA|149|0|2|29|AM|third|night|| +150|AAAAAAAAHJAAAAAA|150|0|2|30|AM|third|night|| +151|AAAAAAAAIJAAAAAA|151|0|2|31|AM|third|night|| +152|AAAAAAAAJJAAAAAA|152|0|2|32|AM|third|night|| +153|AAAAAAAAKJAAAAAA|153|0|2|33|AM|third|night|| +154|AAAAAAAALJAAAAAA|154|0|2|34|AM|third|night|| +155|AAAAAAAAMJAAAAAA|155|0|2|35|AM|third|night|| +156|AAAAAAAANJAAAAAA|156|0|2|36|AM|third|night|| +157|AAAAAAAAOJAAAAAA|157|0|2|37|AM|third|night|| +158|AAAAAAAAPJAAAAAA|158|0|2|38|AM|third|night|| +159|AAAAAAAAAKAAAAAA|159|0|2|39|AM|third|night|| +160|AAAAAAAABKAAAAAA|160|0|2|40|AM|third|night|| +161|AAAAAAAACKAAAAAA|161|0|2|41|AM|third|night|| +162|AAAAAAAADKAAAAAA|162|0|2|42|AM|third|night|| +163|AAAAAAAAEKAAAAAA|163|0|2|43|AM|third|night|| +164|AAAAAAAAFKAAAAAA|164|0|2|44|AM|third|night|| +165|AAAAAAAAGKAAAAAA|165|0|2|45|AM|third|night|| +166|AAAAAAAAHKAAAAAA|166|0|2|46|AM|third|night|| +167|AAAAAAAAIKAAAAAA|167|0|2|47|AM|third|night|| +168|AAAAAAAAJKAAAAAA|168|0|2|48|AM|third|night|| +169|AAAAAAAAKKAAAAAA|169|0|2|49|AM|third|night|| +170|AAAAAAAALKAAAAAA|170|0|2|50|AM|third|night|| +171|AAAAAAAAMKAAAAAA|171|0|2|51|AM|third|night|| +172|AAAAAAAANKAAAAAA|172|0|2|52|AM|third|night|| +173|AAAAAAAAOKAAAAAA|173|0|2|53|AM|third|night|| +174|AAAAAAAAPKAAAAAA|174|0|2|54|AM|third|night|| +175|AAAAAAAAALAAAAAA|175|0|2|55|AM|third|night|| +176|AAAAAAAABLAAAAAA|176|0|2|56|AM|third|night|| +177|AAAAAAAACLAAAAAA|177|0|2|57|AM|third|night|| +178|AAAAAAAADLAAAAAA|178|0|2|58|AM|third|night|| +179|AAAAAAAAELAAAAAA|179|0|2|59|AM|third|night|| +180|AAAAAAAAFLAAAAAA|180|0|3|0|AM|third|night|| +181|AAAAAAAAGLAAAAAA|181|0|3|1|AM|third|night|| +182|AAAAAAAAHLAAAAAA|182|0|3|2|AM|third|night|| +183|AAAAAAAAILAAAAAA|183|0|3|3|AM|third|night|| +184|AAAAAAAAJLAAAAAA|184|0|3|4|AM|third|night|| +185|AAAAAAAAKLAAAAAA|185|0|3|5|AM|third|night|| +186|AAAAAAAALLAAAAAA|186|0|3|6|AM|third|night|| +187|AAAAAAAAMLAAAAAA|187|0|3|7|AM|third|night|| +188|AAAAAAAANLAAAAAA|188|0|3|8|AM|third|night|| +189|AAAAAAAAOLAAAAAA|189|0|3|9|AM|third|night|| +190|AAAAAAAAPLAAAAAA|190|0|3|10|AM|third|night|| +191|AAAAAAAAAMAAAAAA|191|0|3|11|AM|third|night|| +192|AAAAAAAABMAAAAAA|192|0|3|12|AM|third|night|| +193|AAAAAAAACMAAAAAA|193|0|3|13|AM|third|night|| +194|AAAAAAAADMAAAAAA|194|0|3|14|AM|third|night|| +195|AAAAAAAAEMAAAAAA|195|0|3|15|AM|third|night|| +196|AAAAAAAAFMAAAAAA|196|0|3|16|AM|third|night|| +197|AAAAAAAAGMAAAAAA|197|0|3|17|AM|third|night|| +198|AAAAAAAAHMAAAAAA|198|0|3|18|AM|third|night|| +199|AAAAAAAAIMAAAAAA|199|0|3|19|AM|third|night|| +200|AAAAAAAAJMAAAAAA|200|0|3|20|AM|third|night|| +201|AAAAAAAAKMAAAAAA|201|0|3|21|AM|third|night|| +202|AAAAAAAALMAAAAAA|202|0|3|22|AM|third|night|| +203|AAAAAAAAMMAAAAAA|203|0|3|23|AM|third|night|| +204|AAAAAAAANMAAAAAA|204|0|3|24|AM|third|night|| +205|AAAAAAAAOMAAAAAA|205|0|3|25|AM|third|night|| +206|AAAAAAAAPMAAAAAA|206|0|3|26|AM|third|night|| +207|AAAAAAAAANAAAAAA|207|0|3|27|AM|third|night|| +208|AAAAAAAABNAAAAAA|208|0|3|28|AM|third|night|| +209|AAAAAAAACNAAAAAA|209|0|3|29|AM|third|night|| +210|AAAAAAAADNAAAAAA|210|0|3|30|AM|third|night|| +211|AAAAAAAAENAAAAAA|211|0|3|31|AM|third|night|| +212|AAAAAAAAFNAAAAAA|212|0|3|32|AM|third|night|| +213|AAAAAAAAGNAAAAAA|213|0|3|33|AM|third|night|| +214|AAAAAAAAHNAAAAAA|214|0|3|34|AM|third|night|| +215|AAAAAAAAINAAAAAA|215|0|3|35|AM|third|night|| +216|AAAAAAAAJNAAAAAA|216|0|3|36|AM|third|night|| +217|AAAAAAAAKNAAAAAA|217|0|3|37|AM|third|night|| +218|AAAAAAAALNAAAAAA|218|0|3|38|AM|third|night|| +219|AAAAAAAAMNAAAAAA|219|0|3|39|AM|third|night|| +220|AAAAAAAANNAAAAAA|220|0|3|40|AM|third|night|| +221|AAAAAAAAONAAAAAA|221|0|3|41|AM|third|night|| +222|AAAAAAAAPNAAAAAA|222|0|3|42|AM|third|night|| +223|AAAAAAAAAOAAAAAA|223|0|3|43|AM|third|night|| +224|AAAAAAAABOAAAAAA|224|0|3|44|AM|third|night|| +225|AAAAAAAACOAAAAAA|225|0|3|45|AM|third|night|| +226|AAAAAAAADOAAAAAA|226|0|3|46|AM|third|night|| +227|AAAAAAAAEOAAAAAA|227|0|3|47|AM|third|night|| +228|AAAAAAAAFOAAAAAA|228|0|3|48|AM|third|night|| +229|AAAAAAAAGOAAAAAA|229|0|3|49|AM|third|night|| +230|AAAAAAAAHOAAAAAA|230|0|3|50|AM|third|night|| +231|AAAAAAAAIOAAAAAA|231|0|3|51|AM|third|night|| +232|AAAAAAAAJOAAAAAA|232|0|3|52|AM|third|night|| +233|AAAAAAAAKOAAAAAA|233|0|3|53|AM|third|night|| +234|AAAAAAAALOAAAAAA|234|0|3|54|AM|third|night|| +235|AAAAAAAAMOAAAAAA|235|0|3|55|AM|third|night|| +236|AAAAAAAANOAAAAAA|236|0|3|56|AM|third|night|| +237|AAAAAAAAOOAAAAAA|237|0|3|57|AM|third|night|| +238|AAAAAAAAPOAAAAAA|238|0|3|58|AM|third|night|| +239|AAAAAAAAAPAAAAAA|239|0|3|59|AM|third|night|| +240|AAAAAAAABPAAAAAA|240|0|4|0|AM|third|night|| +241|AAAAAAAACPAAAAAA|241|0|4|1|AM|third|night|| +242|AAAAAAAADPAAAAAA|242|0|4|2|AM|third|night|| +243|AAAAAAAAEPAAAAAA|243|0|4|3|AM|third|night|| +244|AAAAAAAAFPAAAAAA|244|0|4|4|AM|third|night|| +245|AAAAAAAAGPAAAAAA|245|0|4|5|AM|third|night|| +246|AAAAAAAAHPAAAAAA|246|0|4|6|AM|third|night|| +247|AAAAAAAAIPAAAAAA|247|0|4|7|AM|third|night|| +248|AAAAAAAAJPAAAAAA|248|0|4|8|AM|third|night|| +249|AAAAAAAAKPAAAAAA|249|0|4|9|AM|third|night|| +250|AAAAAAAALPAAAAAA|250|0|4|10|AM|third|night|| +251|AAAAAAAAMPAAAAAA|251|0|4|11|AM|third|night|| +252|AAAAAAAANPAAAAAA|252|0|4|12|AM|third|night|| +253|AAAAAAAAOPAAAAAA|253|0|4|13|AM|third|night|| +254|AAAAAAAAPPAAAAAA|254|0|4|14|AM|third|night|| +255|AAAAAAAAAABAAAAA|255|0|4|15|AM|third|night|| +256|AAAAAAAABABAAAAA|256|0|4|16|AM|third|night|| +257|AAAAAAAACABAAAAA|257|0|4|17|AM|third|night|| +258|AAAAAAAADABAAAAA|258|0|4|18|AM|third|night|| +259|AAAAAAAAEABAAAAA|259|0|4|19|AM|third|night|| +260|AAAAAAAAFABAAAAA|260|0|4|20|AM|third|night|| +261|AAAAAAAAGABAAAAA|261|0|4|21|AM|third|night|| +262|AAAAAAAAHABAAAAA|262|0|4|22|AM|third|night|| +263|AAAAAAAAIABAAAAA|263|0|4|23|AM|third|night|| +264|AAAAAAAAJABAAAAA|264|0|4|24|AM|third|night|| +265|AAAAAAAAKABAAAAA|265|0|4|25|AM|third|night|| +266|AAAAAAAALABAAAAA|266|0|4|26|AM|third|night|| +267|AAAAAAAAMABAAAAA|267|0|4|27|AM|third|night|| +268|AAAAAAAANABAAAAA|268|0|4|28|AM|third|night|| +269|AAAAAAAAOABAAAAA|269|0|4|29|AM|third|night|| +270|AAAAAAAAPABAAAAA|270|0|4|30|AM|third|night|| +271|AAAAAAAAABBAAAAA|271|0|4|31|AM|third|night|| +272|AAAAAAAABBBAAAAA|272|0|4|32|AM|third|night|| +273|AAAAAAAACBBAAAAA|273|0|4|33|AM|third|night|| +274|AAAAAAAADBBAAAAA|274|0|4|34|AM|third|night|| +275|AAAAAAAAEBBAAAAA|275|0|4|35|AM|third|night|| +276|AAAAAAAAFBBAAAAA|276|0|4|36|AM|third|night|| +277|AAAAAAAAGBBAAAAA|277|0|4|37|AM|third|night|| +278|AAAAAAAAHBBAAAAA|278|0|4|38|AM|third|night|| +279|AAAAAAAAIBBAAAAA|279|0|4|39|AM|third|night|| +280|AAAAAAAAJBBAAAAA|280|0|4|40|AM|third|night|| +281|AAAAAAAAKBBAAAAA|281|0|4|41|AM|third|night|| +282|AAAAAAAALBBAAAAA|282|0|4|42|AM|third|night|| +283|AAAAAAAAMBBAAAAA|283|0|4|43|AM|third|night|| +284|AAAAAAAANBBAAAAA|284|0|4|44|AM|third|night|| +285|AAAAAAAAOBBAAAAA|285|0|4|45|AM|third|night|| +286|AAAAAAAAPBBAAAAA|286|0|4|46|AM|third|night|| +287|AAAAAAAAACBAAAAA|287|0|4|47|AM|third|night|| +288|AAAAAAAABCBAAAAA|288|0|4|48|AM|third|night|| +289|AAAAAAAACCBAAAAA|289|0|4|49|AM|third|night|| +290|AAAAAAAADCBAAAAA|290|0|4|50|AM|third|night|| +291|AAAAAAAAECBAAAAA|291|0|4|51|AM|third|night|| +292|AAAAAAAAFCBAAAAA|292|0|4|52|AM|third|night|| +293|AAAAAAAAGCBAAAAA|293|0|4|53|AM|third|night|| +294|AAAAAAAAHCBAAAAA|294|0|4|54|AM|third|night|| +295|AAAAAAAAICBAAAAA|295|0|4|55|AM|third|night|| +296|AAAAAAAAJCBAAAAA|296|0|4|56|AM|third|night|| +297|AAAAAAAAKCBAAAAA|297|0|4|57|AM|third|night|| +298|AAAAAAAALCBAAAAA|298|0|4|58|AM|third|night|| +299|AAAAAAAAMCBAAAAA|299|0|4|59|AM|third|night|| +300|AAAAAAAANCBAAAAA|300|0|5|0|AM|third|night|| +301|AAAAAAAAOCBAAAAA|301|0|5|1|AM|third|night|| +302|AAAAAAAAPCBAAAAA|302|0|5|2|AM|third|night|| +303|AAAAAAAAADBAAAAA|303|0|5|3|AM|third|night|| +304|AAAAAAAABDBAAAAA|304|0|5|4|AM|third|night|| +305|AAAAAAAACDBAAAAA|305|0|5|5|AM|third|night|| +306|AAAAAAAADDBAAAAA|306|0|5|6|AM|third|night|| +307|AAAAAAAAEDBAAAAA|307|0|5|7|AM|third|night|| +308|AAAAAAAAFDBAAAAA|308|0|5|8|AM|third|night|| +309|AAAAAAAAGDBAAAAA|309|0|5|9|AM|third|night|| +310|AAAAAAAAHDBAAAAA|310|0|5|10|AM|third|night|| +311|AAAAAAAAIDBAAAAA|311|0|5|11|AM|third|night|| +312|AAAAAAAAJDBAAAAA|312|0|5|12|AM|third|night|| +313|AAAAAAAAKDBAAAAA|313|0|5|13|AM|third|night|| +314|AAAAAAAALDBAAAAA|314|0|5|14|AM|third|night|| +315|AAAAAAAAMDBAAAAA|315|0|5|15|AM|third|night|| +316|AAAAAAAANDBAAAAA|316|0|5|16|AM|third|night|| +317|AAAAAAAAODBAAAAA|317|0|5|17|AM|third|night|| +318|AAAAAAAAPDBAAAAA|318|0|5|18|AM|third|night|| +319|AAAAAAAAAEBAAAAA|319|0|5|19|AM|third|night|| +320|AAAAAAAABEBAAAAA|320|0|5|20|AM|third|night|| +321|AAAAAAAACEBAAAAA|321|0|5|21|AM|third|night|| +322|AAAAAAAADEBAAAAA|322|0|5|22|AM|third|night|| +323|AAAAAAAAEEBAAAAA|323|0|5|23|AM|third|night|| +324|AAAAAAAAFEBAAAAA|324|0|5|24|AM|third|night|| +325|AAAAAAAAGEBAAAAA|325|0|5|25|AM|third|night|| +326|AAAAAAAAHEBAAAAA|326|0|5|26|AM|third|night|| +327|AAAAAAAAIEBAAAAA|327|0|5|27|AM|third|night|| +328|AAAAAAAAJEBAAAAA|328|0|5|28|AM|third|night|| +329|AAAAAAAAKEBAAAAA|329|0|5|29|AM|third|night|| +330|AAAAAAAALEBAAAAA|330|0|5|30|AM|third|night|| +331|AAAAAAAAMEBAAAAA|331|0|5|31|AM|third|night|| +332|AAAAAAAANEBAAAAA|332|0|5|32|AM|third|night|| +333|AAAAAAAAOEBAAAAA|333|0|5|33|AM|third|night|| +334|AAAAAAAAPEBAAAAA|334|0|5|34|AM|third|night|| +335|AAAAAAAAAFBAAAAA|335|0|5|35|AM|third|night|| +336|AAAAAAAABFBAAAAA|336|0|5|36|AM|third|night|| +337|AAAAAAAACFBAAAAA|337|0|5|37|AM|third|night|| +338|AAAAAAAADFBAAAAA|338|0|5|38|AM|third|night|| +339|AAAAAAAAEFBAAAAA|339|0|5|39|AM|third|night|| +340|AAAAAAAAFFBAAAAA|340|0|5|40|AM|third|night|| +341|AAAAAAAAGFBAAAAA|341|0|5|41|AM|third|night|| +342|AAAAAAAAHFBAAAAA|342|0|5|42|AM|third|night|| +343|AAAAAAAAIFBAAAAA|343|0|5|43|AM|third|night|| +344|AAAAAAAAJFBAAAAA|344|0|5|44|AM|third|night|| +345|AAAAAAAAKFBAAAAA|345|0|5|45|AM|third|night|| +346|AAAAAAAALFBAAAAA|346|0|5|46|AM|third|night|| +347|AAAAAAAAMFBAAAAA|347|0|5|47|AM|third|night|| +348|AAAAAAAANFBAAAAA|348|0|5|48|AM|third|night|| +349|AAAAAAAAOFBAAAAA|349|0|5|49|AM|third|night|| +350|AAAAAAAAPFBAAAAA|350|0|5|50|AM|third|night|| +351|AAAAAAAAAGBAAAAA|351|0|5|51|AM|third|night|| +352|AAAAAAAABGBAAAAA|352|0|5|52|AM|third|night|| +353|AAAAAAAACGBAAAAA|353|0|5|53|AM|third|night|| +354|AAAAAAAADGBAAAAA|354|0|5|54|AM|third|night|| +355|AAAAAAAAEGBAAAAA|355|0|5|55|AM|third|night|| +356|AAAAAAAAFGBAAAAA|356|0|5|56|AM|third|night|| +357|AAAAAAAAGGBAAAAA|357|0|5|57|AM|third|night|| +358|AAAAAAAAHGBAAAAA|358|0|5|58|AM|third|night|| +359|AAAAAAAAIGBAAAAA|359|0|5|59|AM|third|night|| +360|AAAAAAAAJGBAAAAA|360|0|6|0|AM|third|night|| +361|AAAAAAAAKGBAAAAA|361|0|6|1|AM|third|night|| +362|AAAAAAAALGBAAAAA|362|0|6|2|AM|third|night|| +363|AAAAAAAAMGBAAAAA|363|0|6|3|AM|third|night|| +364|AAAAAAAANGBAAAAA|364|0|6|4|AM|third|night|| +365|AAAAAAAAOGBAAAAA|365|0|6|5|AM|third|night|| +366|AAAAAAAAPGBAAAAA|366|0|6|6|AM|third|night|| +367|AAAAAAAAAHBAAAAA|367|0|6|7|AM|third|night|| +368|AAAAAAAABHBAAAAA|368|0|6|8|AM|third|night|| +369|AAAAAAAACHBAAAAA|369|0|6|9|AM|third|night|| +370|AAAAAAAADHBAAAAA|370|0|6|10|AM|third|night|| +371|AAAAAAAAEHBAAAAA|371|0|6|11|AM|third|night|| +372|AAAAAAAAFHBAAAAA|372|0|6|12|AM|third|night|| +373|AAAAAAAAGHBAAAAA|373|0|6|13|AM|third|night|| +374|AAAAAAAAHHBAAAAA|374|0|6|14|AM|third|night|| +375|AAAAAAAAIHBAAAAA|375|0|6|15|AM|third|night|| +376|AAAAAAAAJHBAAAAA|376|0|6|16|AM|third|night|| +377|AAAAAAAAKHBAAAAA|377|0|6|17|AM|third|night|| +378|AAAAAAAALHBAAAAA|378|0|6|18|AM|third|night|| +379|AAAAAAAAMHBAAAAA|379|0|6|19|AM|third|night|| +380|AAAAAAAANHBAAAAA|380|0|6|20|AM|third|night|| +381|AAAAAAAAOHBAAAAA|381|0|6|21|AM|third|night|| +382|AAAAAAAAPHBAAAAA|382|0|6|22|AM|third|night|| +383|AAAAAAAAAIBAAAAA|383|0|6|23|AM|third|night|| +384|AAAAAAAABIBAAAAA|384|0|6|24|AM|third|night|| +385|AAAAAAAACIBAAAAA|385|0|6|25|AM|third|night|| +386|AAAAAAAADIBAAAAA|386|0|6|26|AM|third|night|| +387|AAAAAAAAEIBAAAAA|387|0|6|27|AM|third|night|| +388|AAAAAAAAFIBAAAAA|388|0|6|28|AM|third|night|| +389|AAAAAAAAGIBAAAAA|389|0|6|29|AM|third|night|| +390|AAAAAAAAHIBAAAAA|390|0|6|30|AM|third|night|| +391|AAAAAAAAIIBAAAAA|391|0|6|31|AM|third|night|| +392|AAAAAAAAJIBAAAAA|392|0|6|32|AM|third|night|| +393|AAAAAAAAKIBAAAAA|393|0|6|33|AM|third|night|| +394|AAAAAAAALIBAAAAA|394|0|6|34|AM|third|night|| +395|AAAAAAAAMIBAAAAA|395|0|6|35|AM|third|night|| +396|AAAAAAAANIBAAAAA|396|0|6|36|AM|third|night|| +397|AAAAAAAAOIBAAAAA|397|0|6|37|AM|third|night|| +398|AAAAAAAAPIBAAAAA|398|0|6|38|AM|third|night|| +399|AAAAAAAAAJBAAAAA|399|0|6|39|AM|third|night|| +400|AAAAAAAABJBAAAAA|400|0|6|40|AM|third|night|| +401|AAAAAAAACJBAAAAA|401|0|6|41|AM|third|night|| +402|AAAAAAAADJBAAAAA|402|0|6|42|AM|third|night|| +403|AAAAAAAAEJBAAAAA|403|0|6|43|AM|third|night|| +404|AAAAAAAAFJBAAAAA|404|0|6|44|AM|third|night|| +405|AAAAAAAAGJBAAAAA|405|0|6|45|AM|third|night|| +406|AAAAAAAAHJBAAAAA|406|0|6|46|AM|third|night|| +407|AAAAAAAAIJBAAAAA|407|0|6|47|AM|third|night|| +408|AAAAAAAAJJBAAAAA|408|0|6|48|AM|third|night|| +409|AAAAAAAAKJBAAAAA|409|0|6|49|AM|third|night|| +410|AAAAAAAALJBAAAAA|410|0|6|50|AM|third|night|| +411|AAAAAAAAMJBAAAAA|411|0|6|51|AM|third|night|| +412|AAAAAAAANJBAAAAA|412|0|6|52|AM|third|night|| +413|AAAAAAAAOJBAAAAA|413|0|6|53|AM|third|night|| +414|AAAAAAAAPJBAAAAA|414|0|6|54|AM|third|night|| +415|AAAAAAAAAKBAAAAA|415|0|6|55|AM|third|night|| +416|AAAAAAAABKBAAAAA|416|0|6|56|AM|third|night|| +417|AAAAAAAACKBAAAAA|417|0|6|57|AM|third|night|| +418|AAAAAAAADKBAAAAA|418|0|6|58|AM|third|night|| +419|AAAAAAAAEKBAAAAA|419|0|6|59|AM|third|night|| +420|AAAAAAAAFKBAAAAA|420|0|7|0|AM|third|night|| +421|AAAAAAAAGKBAAAAA|421|0|7|1|AM|third|night|| +422|AAAAAAAAHKBAAAAA|422|0|7|2|AM|third|night|| +423|AAAAAAAAIKBAAAAA|423|0|7|3|AM|third|night|| +424|AAAAAAAAJKBAAAAA|424|0|7|4|AM|third|night|| +425|AAAAAAAAKKBAAAAA|425|0|7|5|AM|third|night|| +426|AAAAAAAALKBAAAAA|426|0|7|6|AM|third|night|| +427|AAAAAAAAMKBAAAAA|427|0|7|7|AM|third|night|| +428|AAAAAAAANKBAAAAA|428|0|7|8|AM|third|night|| +429|AAAAAAAAOKBAAAAA|429|0|7|9|AM|third|night|| +430|AAAAAAAAPKBAAAAA|430|0|7|10|AM|third|night|| +431|AAAAAAAAALBAAAAA|431|0|7|11|AM|third|night|| +432|AAAAAAAABLBAAAAA|432|0|7|12|AM|third|night|| +433|AAAAAAAACLBAAAAA|433|0|7|13|AM|third|night|| +434|AAAAAAAADLBAAAAA|434|0|7|14|AM|third|night|| +435|AAAAAAAAELBAAAAA|435|0|7|15|AM|third|night|| +436|AAAAAAAAFLBAAAAA|436|0|7|16|AM|third|night|| +437|AAAAAAAAGLBAAAAA|437|0|7|17|AM|third|night|| +438|AAAAAAAAHLBAAAAA|438|0|7|18|AM|third|night|| +439|AAAAAAAAILBAAAAA|439|0|7|19|AM|third|night|| +440|AAAAAAAAJLBAAAAA|440|0|7|20|AM|third|night|| +441|AAAAAAAAKLBAAAAA|441|0|7|21|AM|third|night|| +442|AAAAAAAALLBAAAAA|442|0|7|22|AM|third|night|| +443|AAAAAAAAMLBAAAAA|443|0|7|23|AM|third|night|| +444|AAAAAAAANLBAAAAA|444|0|7|24|AM|third|night|| +445|AAAAAAAAOLBAAAAA|445|0|7|25|AM|third|night|| +446|AAAAAAAAPLBAAAAA|446|0|7|26|AM|third|night|| +447|AAAAAAAAAMBAAAAA|447|0|7|27|AM|third|night|| +448|AAAAAAAABMBAAAAA|448|0|7|28|AM|third|night|| +449|AAAAAAAACMBAAAAA|449|0|7|29|AM|third|night|| +450|AAAAAAAADMBAAAAA|450|0|7|30|AM|third|night|| +451|AAAAAAAAEMBAAAAA|451|0|7|31|AM|third|night|| +452|AAAAAAAAFMBAAAAA|452|0|7|32|AM|third|night|| +453|AAAAAAAAGMBAAAAA|453|0|7|33|AM|third|night|| +454|AAAAAAAAHMBAAAAA|454|0|7|34|AM|third|night|| +455|AAAAAAAAIMBAAAAA|455|0|7|35|AM|third|night|| +456|AAAAAAAAJMBAAAAA|456|0|7|36|AM|third|night|| +457|AAAAAAAAKMBAAAAA|457|0|7|37|AM|third|night|| +458|AAAAAAAALMBAAAAA|458|0|7|38|AM|third|night|| +459|AAAAAAAAMMBAAAAA|459|0|7|39|AM|third|night|| +460|AAAAAAAANMBAAAAA|460|0|7|40|AM|third|night|| +461|AAAAAAAAOMBAAAAA|461|0|7|41|AM|third|night|| +462|AAAAAAAAPMBAAAAA|462|0|7|42|AM|third|night|| +463|AAAAAAAAANBAAAAA|463|0|7|43|AM|third|night|| +464|AAAAAAAABNBAAAAA|464|0|7|44|AM|third|night|| +465|AAAAAAAACNBAAAAA|465|0|7|45|AM|third|night|| +466|AAAAAAAADNBAAAAA|466|0|7|46|AM|third|night|| +467|AAAAAAAAENBAAAAA|467|0|7|47|AM|third|night|| +468|AAAAAAAAFNBAAAAA|468|0|7|48|AM|third|night|| +469|AAAAAAAAGNBAAAAA|469|0|7|49|AM|third|night|| +470|AAAAAAAAHNBAAAAA|470|0|7|50|AM|third|night|| +471|AAAAAAAAINBAAAAA|471|0|7|51|AM|third|night|| +472|AAAAAAAAJNBAAAAA|472|0|7|52|AM|third|night|| +473|AAAAAAAAKNBAAAAA|473|0|7|53|AM|third|night|| +474|AAAAAAAALNBAAAAA|474|0|7|54|AM|third|night|| +475|AAAAAAAAMNBAAAAA|475|0|7|55|AM|third|night|| +476|AAAAAAAANNBAAAAA|476|0|7|56|AM|third|night|| +477|AAAAAAAAONBAAAAA|477|0|7|57|AM|third|night|| +478|AAAAAAAAPNBAAAAA|478|0|7|58|AM|third|night|| +479|AAAAAAAAAOBAAAAA|479|0|7|59|AM|third|night|| +480|AAAAAAAABOBAAAAA|480|0|8|0|AM|third|night|| +481|AAAAAAAACOBAAAAA|481|0|8|1|AM|third|night|| +482|AAAAAAAADOBAAAAA|482|0|8|2|AM|third|night|| +483|AAAAAAAAEOBAAAAA|483|0|8|3|AM|third|night|| +484|AAAAAAAAFOBAAAAA|484|0|8|4|AM|third|night|| +485|AAAAAAAAGOBAAAAA|485|0|8|5|AM|third|night|| +486|AAAAAAAAHOBAAAAA|486|0|8|6|AM|third|night|| +487|AAAAAAAAIOBAAAAA|487|0|8|7|AM|third|night|| +488|AAAAAAAAJOBAAAAA|488|0|8|8|AM|third|night|| +489|AAAAAAAAKOBAAAAA|489|0|8|9|AM|third|night|| +490|AAAAAAAALOBAAAAA|490|0|8|10|AM|third|night|| +491|AAAAAAAAMOBAAAAA|491|0|8|11|AM|third|night|| +492|AAAAAAAANOBAAAAA|492|0|8|12|AM|third|night|| +493|AAAAAAAAOOBAAAAA|493|0|8|13|AM|third|night|| +494|AAAAAAAAPOBAAAAA|494|0|8|14|AM|third|night|| +495|AAAAAAAAAPBAAAAA|495|0|8|15|AM|third|night|| +496|AAAAAAAABPBAAAAA|496|0|8|16|AM|third|night|| +497|AAAAAAAACPBAAAAA|497|0|8|17|AM|third|night|| +498|AAAAAAAADPBAAAAA|498|0|8|18|AM|third|night|| +499|AAAAAAAAEPBAAAAA|499|0|8|19|AM|third|night|| +500|AAAAAAAAFPBAAAAA|500|0|8|20|AM|third|night|| +501|AAAAAAAAGPBAAAAA|501|0|8|21|AM|third|night|| +502|AAAAAAAAHPBAAAAA|502|0|8|22|AM|third|night|| +503|AAAAAAAAIPBAAAAA|503|0|8|23|AM|third|night|| +504|AAAAAAAAJPBAAAAA|504|0|8|24|AM|third|night|| +505|AAAAAAAAKPBAAAAA|505|0|8|25|AM|third|night|| +506|AAAAAAAALPBAAAAA|506|0|8|26|AM|third|night|| +507|AAAAAAAAMPBAAAAA|507|0|8|27|AM|third|night|| +508|AAAAAAAANPBAAAAA|508|0|8|28|AM|third|night|| +509|AAAAAAAAOPBAAAAA|509|0|8|29|AM|third|night|| +510|AAAAAAAAPPBAAAAA|510|0|8|30|AM|third|night|| +511|AAAAAAAAAACAAAAA|511|0|8|31|AM|third|night|| +512|AAAAAAAABACAAAAA|512|0|8|32|AM|third|night|| +513|AAAAAAAACACAAAAA|513|0|8|33|AM|third|night|| +514|AAAAAAAADACAAAAA|514|0|8|34|AM|third|night|| +515|AAAAAAAAEACAAAAA|515|0|8|35|AM|third|night|| +516|AAAAAAAAFACAAAAA|516|0|8|36|AM|third|night|| +517|AAAAAAAAGACAAAAA|517|0|8|37|AM|third|night|| +518|AAAAAAAAHACAAAAA|518|0|8|38|AM|third|night|| +519|AAAAAAAAIACAAAAA|519|0|8|39|AM|third|night|| +520|AAAAAAAAJACAAAAA|520|0|8|40|AM|third|night|| +521|AAAAAAAAKACAAAAA|521|0|8|41|AM|third|night|| +522|AAAAAAAALACAAAAA|522|0|8|42|AM|third|night|| +523|AAAAAAAAMACAAAAA|523|0|8|43|AM|third|night|| +524|AAAAAAAANACAAAAA|524|0|8|44|AM|third|night|| +525|AAAAAAAAOACAAAAA|525|0|8|45|AM|third|night|| +526|AAAAAAAAPACAAAAA|526|0|8|46|AM|third|night|| +527|AAAAAAAAABCAAAAA|527|0|8|47|AM|third|night|| +528|AAAAAAAABBCAAAAA|528|0|8|48|AM|third|night|| +529|AAAAAAAACBCAAAAA|529|0|8|49|AM|third|night|| +530|AAAAAAAADBCAAAAA|530|0|8|50|AM|third|night|| +531|AAAAAAAAEBCAAAAA|531|0|8|51|AM|third|night|| +532|AAAAAAAAFBCAAAAA|532|0|8|52|AM|third|night|| +533|AAAAAAAAGBCAAAAA|533|0|8|53|AM|third|night|| +534|AAAAAAAAHBCAAAAA|534|0|8|54|AM|third|night|| +535|AAAAAAAAIBCAAAAA|535|0|8|55|AM|third|night|| +536|AAAAAAAAJBCAAAAA|536|0|8|56|AM|third|night|| +537|AAAAAAAAKBCAAAAA|537|0|8|57|AM|third|night|| +538|AAAAAAAALBCAAAAA|538|0|8|58|AM|third|night|| +539|AAAAAAAAMBCAAAAA|539|0|8|59|AM|third|night|| +540|AAAAAAAANBCAAAAA|540|0|9|0|AM|third|night|| +541|AAAAAAAAOBCAAAAA|541|0|9|1|AM|third|night|| +542|AAAAAAAAPBCAAAAA|542|0|9|2|AM|third|night|| +543|AAAAAAAAACCAAAAA|543|0|9|3|AM|third|night|| +544|AAAAAAAABCCAAAAA|544|0|9|4|AM|third|night|| +545|AAAAAAAACCCAAAAA|545|0|9|5|AM|third|night|| +546|AAAAAAAADCCAAAAA|546|0|9|6|AM|third|night|| +547|AAAAAAAAECCAAAAA|547|0|9|7|AM|third|night|| +548|AAAAAAAAFCCAAAAA|548|0|9|8|AM|third|night|| +549|AAAAAAAAGCCAAAAA|549|0|9|9|AM|third|night|| +550|AAAAAAAAHCCAAAAA|550|0|9|10|AM|third|night|| +551|AAAAAAAAICCAAAAA|551|0|9|11|AM|third|night|| +552|AAAAAAAAJCCAAAAA|552|0|9|12|AM|third|night|| +553|AAAAAAAAKCCAAAAA|553|0|9|13|AM|third|night|| +554|AAAAAAAALCCAAAAA|554|0|9|14|AM|third|night|| +555|AAAAAAAAMCCAAAAA|555|0|9|15|AM|third|night|| +556|AAAAAAAANCCAAAAA|556|0|9|16|AM|third|night|| +557|AAAAAAAAOCCAAAAA|557|0|9|17|AM|third|night|| +558|AAAAAAAAPCCAAAAA|558|0|9|18|AM|third|night|| +559|AAAAAAAAADCAAAAA|559|0|9|19|AM|third|night|| +560|AAAAAAAABDCAAAAA|560|0|9|20|AM|third|night|| +561|AAAAAAAACDCAAAAA|561|0|9|21|AM|third|night|| +562|AAAAAAAADDCAAAAA|562|0|9|22|AM|third|night|| +563|AAAAAAAAEDCAAAAA|563|0|9|23|AM|third|night|| +564|AAAAAAAAFDCAAAAA|564|0|9|24|AM|third|night|| +565|AAAAAAAAGDCAAAAA|565|0|9|25|AM|third|night|| +566|AAAAAAAAHDCAAAAA|566|0|9|26|AM|third|night|| +567|AAAAAAAAIDCAAAAA|567|0|9|27|AM|third|night|| +568|AAAAAAAAJDCAAAAA|568|0|9|28|AM|third|night|| +569|AAAAAAAAKDCAAAAA|569|0|9|29|AM|third|night|| +570|AAAAAAAALDCAAAAA|570|0|9|30|AM|third|night|| +571|AAAAAAAAMDCAAAAA|571|0|9|31|AM|third|night|| +572|AAAAAAAANDCAAAAA|572|0|9|32|AM|third|night|| +573|AAAAAAAAODCAAAAA|573|0|9|33|AM|third|night|| +574|AAAAAAAAPDCAAAAA|574|0|9|34|AM|third|night|| +575|AAAAAAAAAECAAAAA|575|0|9|35|AM|third|night|| +576|AAAAAAAABECAAAAA|576|0|9|36|AM|third|night|| +577|AAAAAAAACECAAAAA|577|0|9|37|AM|third|night|| +578|AAAAAAAADECAAAAA|578|0|9|38|AM|third|night|| +579|AAAAAAAAEECAAAAA|579|0|9|39|AM|third|night|| +580|AAAAAAAAFECAAAAA|580|0|9|40|AM|third|night|| +581|AAAAAAAAGECAAAAA|581|0|9|41|AM|third|night|| +582|AAAAAAAAHECAAAAA|582|0|9|42|AM|third|night|| +583|AAAAAAAAIECAAAAA|583|0|9|43|AM|third|night|| +584|AAAAAAAAJECAAAAA|584|0|9|44|AM|third|night|| +585|AAAAAAAAKECAAAAA|585|0|9|45|AM|third|night|| +586|AAAAAAAALECAAAAA|586|0|9|46|AM|third|night|| +587|AAAAAAAAMECAAAAA|587|0|9|47|AM|third|night|| +588|AAAAAAAANECAAAAA|588|0|9|48|AM|third|night|| +589|AAAAAAAAOECAAAAA|589|0|9|49|AM|third|night|| +590|AAAAAAAAPECAAAAA|590|0|9|50|AM|third|night|| +591|AAAAAAAAAFCAAAAA|591|0|9|51|AM|third|night|| +592|AAAAAAAABFCAAAAA|592|0|9|52|AM|third|night|| +593|AAAAAAAACFCAAAAA|593|0|9|53|AM|third|night|| +594|AAAAAAAADFCAAAAA|594|0|9|54|AM|third|night|| +595|AAAAAAAAEFCAAAAA|595|0|9|55|AM|third|night|| +596|AAAAAAAAFFCAAAAA|596|0|9|56|AM|third|night|| +597|AAAAAAAAGFCAAAAA|597|0|9|57|AM|third|night|| +598|AAAAAAAAHFCAAAAA|598|0|9|58|AM|third|night|| +599|AAAAAAAAIFCAAAAA|599|0|9|59|AM|third|night|| +600|AAAAAAAAJFCAAAAA|600|0|10|0|AM|third|night|| +601|AAAAAAAAKFCAAAAA|601|0|10|1|AM|third|night|| +602|AAAAAAAALFCAAAAA|602|0|10|2|AM|third|night|| +603|AAAAAAAAMFCAAAAA|603|0|10|3|AM|third|night|| +604|AAAAAAAANFCAAAAA|604|0|10|4|AM|third|night|| +605|AAAAAAAAOFCAAAAA|605|0|10|5|AM|third|night|| +606|AAAAAAAAPFCAAAAA|606|0|10|6|AM|third|night|| +607|AAAAAAAAAGCAAAAA|607|0|10|7|AM|third|night|| +608|AAAAAAAABGCAAAAA|608|0|10|8|AM|third|night|| +609|AAAAAAAACGCAAAAA|609|0|10|9|AM|third|night|| +610|AAAAAAAADGCAAAAA|610|0|10|10|AM|third|night|| +611|AAAAAAAAEGCAAAAA|611|0|10|11|AM|third|night|| +612|AAAAAAAAFGCAAAAA|612|0|10|12|AM|third|night|| +613|AAAAAAAAGGCAAAAA|613|0|10|13|AM|third|night|| +614|AAAAAAAAHGCAAAAA|614|0|10|14|AM|third|night|| +615|AAAAAAAAIGCAAAAA|615|0|10|15|AM|third|night|| +616|AAAAAAAAJGCAAAAA|616|0|10|16|AM|third|night|| +617|AAAAAAAAKGCAAAAA|617|0|10|17|AM|third|night|| +618|AAAAAAAALGCAAAAA|618|0|10|18|AM|third|night|| +619|AAAAAAAAMGCAAAAA|619|0|10|19|AM|third|night|| +620|AAAAAAAANGCAAAAA|620|0|10|20|AM|third|night|| +621|AAAAAAAAOGCAAAAA|621|0|10|21|AM|third|night|| +622|AAAAAAAAPGCAAAAA|622|0|10|22|AM|third|night|| +623|AAAAAAAAAHCAAAAA|623|0|10|23|AM|third|night|| +624|AAAAAAAABHCAAAAA|624|0|10|24|AM|third|night|| +625|AAAAAAAACHCAAAAA|625|0|10|25|AM|third|night|| +626|AAAAAAAADHCAAAAA|626|0|10|26|AM|third|night|| +627|AAAAAAAAEHCAAAAA|627|0|10|27|AM|third|night|| +628|AAAAAAAAFHCAAAAA|628|0|10|28|AM|third|night|| +629|AAAAAAAAGHCAAAAA|629|0|10|29|AM|third|night|| +630|AAAAAAAAHHCAAAAA|630|0|10|30|AM|third|night|| +631|AAAAAAAAIHCAAAAA|631|0|10|31|AM|third|night|| +632|AAAAAAAAJHCAAAAA|632|0|10|32|AM|third|night|| +633|AAAAAAAAKHCAAAAA|633|0|10|33|AM|third|night|| +634|AAAAAAAALHCAAAAA|634|0|10|34|AM|third|night|| +635|AAAAAAAAMHCAAAAA|635|0|10|35|AM|third|night|| +636|AAAAAAAANHCAAAAA|636|0|10|36|AM|third|night|| +637|AAAAAAAAOHCAAAAA|637|0|10|37|AM|third|night|| +638|AAAAAAAAPHCAAAAA|638|0|10|38|AM|third|night|| +639|AAAAAAAAAICAAAAA|639|0|10|39|AM|third|night|| +640|AAAAAAAABICAAAAA|640|0|10|40|AM|third|night|| +641|AAAAAAAACICAAAAA|641|0|10|41|AM|third|night|| +642|AAAAAAAADICAAAAA|642|0|10|42|AM|third|night|| +643|AAAAAAAAEICAAAAA|643|0|10|43|AM|third|night|| +644|AAAAAAAAFICAAAAA|644|0|10|44|AM|third|night|| +645|AAAAAAAAGICAAAAA|645|0|10|45|AM|third|night|| +646|AAAAAAAAHICAAAAA|646|0|10|46|AM|third|night|| +647|AAAAAAAAIICAAAAA|647|0|10|47|AM|third|night|| +648|AAAAAAAAJICAAAAA|648|0|10|48|AM|third|night|| +649|AAAAAAAAKICAAAAA|649|0|10|49|AM|third|night|| +650|AAAAAAAALICAAAAA|650|0|10|50|AM|third|night|| +651|AAAAAAAAMICAAAAA|651|0|10|51|AM|third|night|| +652|AAAAAAAANICAAAAA|652|0|10|52|AM|third|night|| +653|AAAAAAAAOICAAAAA|653|0|10|53|AM|third|night|| +654|AAAAAAAAPICAAAAA|654|0|10|54|AM|third|night|| +655|AAAAAAAAAJCAAAAA|655|0|10|55|AM|third|night|| +656|AAAAAAAABJCAAAAA|656|0|10|56|AM|third|night|| +657|AAAAAAAACJCAAAAA|657|0|10|57|AM|third|night|| +658|AAAAAAAADJCAAAAA|658|0|10|58|AM|third|night|| +659|AAAAAAAAEJCAAAAA|659|0|10|59|AM|third|night|| +660|AAAAAAAAFJCAAAAA|660|0|11|0|AM|third|night|| +661|AAAAAAAAGJCAAAAA|661|0|11|1|AM|third|night|| +662|AAAAAAAAHJCAAAAA|662|0|11|2|AM|third|night|| +663|AAAAAAAAIJCAAAAA|663|0|11|3|AM|third|night|| +664|AAAAAAAAJJCAAAAA|664|0|11|4|AM|third|night|| +665|AAAAAAAAKJCAAAAA|665|0|11|5|AM|third|night|| +666|AAAAAAAALJCAAAAA|666|0|11|6|AM|third|night|| +667|AAAAAAAAMJCAAAAA|667|0|11|7|AM|third|night|| +668|AAAAAAAANJCAAAAA|668|0|11|8|AM|third|night|| +669|AAAAAAAAOJCAAAAA|669|0|11|9|AM|third|night|| +670|AAAAAAAAPJCAAAAA|670|0|11|10|AM|third|night|| +671|AAAAAAAAAKCAAAAA|671|0|11|11|AM|third|night|| +672|AAAAAAAABKCAAAAA|672|0|11|12|AM|third|night|| +673|AAAAAAAACKCAAAAA|673|0|11|13|AM|third|night|| +674|AAAAAAAADKCAAAAA|674|0|11|14|AM|third|night|| +675|AAAAAAAAEKCAAAAA|675|0|11|15|AM|third|night|| +676|AAAAAAAAFKCAAAAA|676|0|11|16|AM|third|night|| +677|AAAAAAAAGKCAAAAA|677|0|11|17|AM|third|night|| +678|AAAAAAAAHKCAAAAA|678|0|11|18|AM|third|night|| +679|AAAAAAAAIKCAAAAA|679|0|11|19|AM|third|night|| +680|AAAAAAAAJKCAAAAA|680|0|11|20|AM|third|night|| +681|AAAAAAAAKKCAAAAA|681|0|11|21|AM|third|night|| +682|AAAAAAAALKCAAAAA|682|0|11|22|AM|third|night|| +683|AAAAAAAAMKCAAAAA|683|0|11|23|AM|third|night|| +684|AAAAAAAANKCAAAAA|684|0|11|24|AM|third|night|| +685|AAAAAAAAOKCAAAAA|685|0|11|25|AM|third|night|| +686|AAAAAAAAPKCAAAAA|686|0|11|26|AM|third|night|| +687|AAAAAAAAALCAAAAA|687|0|11|27|AM|third|night|| +688|AAAAAAAABLCAAAAA|688|0|11|28|AM|third|night|| +689|AAAAAAAACLCAAAAA|689|0|11|29|AM|third|night|| +690|AAAAAAAADLCAAAAA|690|0|11|30|AM|third|night|| +691|AAAAAAAAELCAAAAA|691|0|11|31|AM|third|night|| +692|AAAAAAAAFLCAAAAA|692|0|11|32|AM|third|night|| +693|AAAAAAAAGLCAAAAA|693|0|11|33|AM|third|night|| +694|AAAAAAAAHLCAAAAA|694|0|11|34|AM|third|night|| +695|AAAAAAAAILCAAAAA|695|0|11|35|AM|third|night|| +696|AAAAAAAAJLCAAAAA|696|0|11|36|AM|third|night|| +697|AAAAAAAAKLCAAAAA|697|0|11|37|AM|third|night|| +698|AAAAAAAALLCAAAAA|698|0|11|38|AM|third|night|| +699|AAAAAAAAMLCAAAAA|699|0|11|39|AM|third|night|| +700|AAAAAAAANLCAAAAA|700|0|11|40|AM|third|night|| +701|AAAAAAAAOLCAAAAA|701|0|11|41|AM|third|night|| +702|AAAAAAAAPLCAAAAA|702|0|11|42|AM|third|night|| +703|AAAAAAAAAMCAAAAA|703|0|11|43|AM|third|night|| +704|AAAAAAAABMCAAAAA|704|0|11|44|AM|third|night|| +705|AAAAAAAACMCAAAAA|705|0|11|45|AM|third|night|| +706|AAAAAAAADMCAAAAA|706|0|11|46|AM|third|night|| +707|AAAAAAAAEMCAAAAA|707|0|11|47|AM|third|night|| +708|AAAAAAAAFMCAAAAA|708|0|11|48|AM|third|night|| +709|AAAAAAAAGMCAAAAA|709|0|11|49|AM|third|night|| +710|AAAAAAAAHMCAAAAA|710|0|11|50|AM|third|night|| +711|AAAAAAAAIMCAAAAA|711|0|11|51|AM|third|night|| +712|AAAAAAAAJMCAAAAA|712|0|11|52|AM|third|night|| +713|AAAAAAAAKMCAAAAA|713|0|11|53|AM|third|night|| +714|AAAAAAAALMCAAAAA|714|0|11|54|AM|third|night|| +715|AAAAAAAAMMCAAAAA|715|0|11|55|AM|third|night|| +716|AAAAAAAANMCAAAAA|716|0|11|56|AM|third|night|| +717|AAAAAAAAOMCAAAAA|717|0|11|57|AM|third|night|| +718|AAAAAAAAPMCAAAAA|718|0|11|58|AM|third|night|| +719|AAAAAAAAANCAAAAA|719|0|11|59|AM|third|night|| +720|AAAAAAAABNCAAAAA|720|0|12|0|AM|third|night|| +721|AAAAAAAACNCAAAAA|721|0|12|1|AM|third|night|| +722|AAAAAAAADNCAAAAA|722|0|12|2|AM|third|night|| +723|AAAAAAAAENCAAAAA|723|0|12|3|AM|third|night|| +724|AAAAAAAAFNCAAAAA|724|0|12|4|AM|third|night|| +725|AAAAAAAAGNCAAAAA|725|0|12|5|AM|third|night|| +726|AAAAAAAAHNCAAAAA|726|0|12|6|AM|third|night|| +727|AAAAAAAAINCAAAAA|727|0|12|7|AM|third|night|| +728|AAAAAAAAJNCAAAAA|728|0|12|8|AM|third|night|| +729|AAAAAAAAKNCAAAAA|729|0|12|9|AM|third|night|| +730|AAAAAAAALNCAAAAA|730|0|12|10|AM|third|night|| +731|AAAAAAAAMNCAAAAA|731|0|12|11|AM|third|night|| +732|AAAAAAAANNCAAAAA|732|0|12|12|AM|third|night|| +733|AAAAAAAAONCAAAAA|733|0|12|13|AM|third|night|| +734|AAAAAAAAPNCAAAAA|734|0|12|14|AM|third|night|| +735|AAAAAAAAAOCAAAAA|735|0|12|15|AM|third|night|| +736|AAAAAAAABOCAAAAA|736|0|12|16|AM|third|night|| +737|AAAAAAAACOCAAAAA|737|0|12|17|AM|third|night|| +738|AAAAAAAADOCAAAAA|738|0|12|18|AM|third|night|| +739|AAAAAAAAEOCAAAAA|739|0|12|19|AM|third|night|| +740|AAAAAAAAFOCAAAAA|740|0|12|20|AM|third|night|| +741|AAAAAAAAGOCAAAAA|741|0|12|21|AM|third|night|| +742|AAAAAAAAHOCAAAAA|742|0|12|22|AM|third|night|| +743|AAAAAAAAIOCAAAAA|743|0|12|23|AM|third|night|| +744|AAAAAAAAJOCAAAAA|744|0|12|24|AM|third|night|| +745|AAAAAAAAKOCAAAAA|745|0|12|25|AM|third|night|| +746|AAAAAAAALOCAAAAA|746|0|12|26|AM|third|night|| +747|AAAAAAAAMOCAAAAA|747|0|12|27|AM|third|night|| +748|AAAAAAAANOCAAAAA|748|0|12|28|AM|third|night|| +749|AAAAAAAAOOCAAAAA|749|0|12|29|AM|third|night|| +750|AAAAAAAAPOCAAAAA|750|0|12|30|AM|third|night|| +751|AAAAAAAAAPCAAAAA|751|0|12|31|AM|third|night|| +752|AAAAAAAABPCAAAAA|752|0|12|32|AM|third|night|| +753|AAAAAAAACPCAAAAA|753|0|12|33|AM|third|night|| +754|AAAAAAAADPCAAAAA|754|0|12|34|AM|third|night|| +755|AAAAAAAAEPCAAAAA|755|0|12|35|AM|third|night|| +756|AAAAAAAAFPCAAAAA|756|0|12|36|AM|third|night|| +757|AAAAAAAAGPCAAAAA|757|0|12|37|AM|third|night|| +758|AAAAAAAAHPCAAAAA|758|0|12|38|AM|third|night|| +759|AAAAAAAAIPCAAAAA|759|0|12|39|AM|third|night|| +760|AAAAAAAAJPCAAAAA|760|0|12|40|AM|third|night|| +761|AAAAAAAAKPCAAAAA|761|0|12|41|AM|third|night|| +762|AAAAAAAALPCAAAAA|762|0|12|42|AM|third|night|| +763|AAAAAAAAMPCAAAAA|763|0|12|43|AM|third|night|| +764|AAAAAAAANPCAAAAA|764|0|12|44|AM|third|night|| +765|AAAAAAAAOPCAAAAA|765|0|12|45|AM|third|night|| +766|AAAAAAAAPPCAAAAA|766|0|12|46|AM|third|night|| +767|AAAAAAAAAADAAAAA|767|0|12|47|AM|third|night|| +768|AAAAAAAABADAAAAA|768|0|12|48|AM|third|night|| +769|AAAAAAAACADAAAAA|769|0|12|49|AM|third|night|| +770|AAAAAAAADADAAAAA|770|0|12|50|AM|third|night|| +771|AAAAAAAAEADAAAAA|771|0|12|51|AM|third|night|| +772|AAAAAAAAFADAAAAA|772|0|12|52|AM|third|night|| +773|AAAAAAAAGADAAAAA|773|0|12|53|AM|third|night|| +774|AAAAAAAAHADAAAAA|774|0|12|54|AM|third|night|| +775|AAAAAAAAIADAAAAA|775|0|12|55|AM|third|night|| +776|AAAAAAAAJADAAAAA|776|0|12|56|AM|third|night|| +777|AAAAAAAAKADAAAAA|777|0|12|57|AM|third|night|| +778|AAAAAAAALADAAAAA|778|0|12|58|AM|third|night|| +779|AAAAAAAAMADAAAAA|779|0|12|59|AM|third|night|| +780|AAAAAAAANADAAAAA|780|0|13|0|AM|third|night|| +781|AAAAAAAAOADAAAAA|781|0|13|1|AM|third|night|| +782|AAAAAAAAPADAAAAA|782|0|13|2|AM|third|night|| +783|AAAAAAAAABDAAAAA|783|0|13|3|AM|third|night|| +784|AAAAAAAABBDAAAAA|784|0|13|4|AM|third|night|| +785|AAAAAAAACBDAAAAA|785|0|13|5|AM|third|night|| +786|AAAAAAAADBDAAAAA|786|0|13|6|AM|third|night|| +787|AAAAAAAAEBDAAAAA|787|0|13|7|AM|third|night|| +788|AAAAAAAAFBDAAAAA|788|0|13|8|AM|third|night|| +789|AAAAAAAAGBDAAAAA|789|0|13|9|AM|third|night|| +790|AAAAAAAAHBDAAAAA|790|0|13|10|AM|third|night|| +791|AAAAAAAAIBDAAAAA|791|0|13|11|AM|third|night|| +792|AAAAAAAAJBDAAAAA|792|0|13|12|AM|third|night|| +793|AAAAAAAAKBDAAAAA|793|0|13|13|AM|third|night|| +794|AAAAAAAALBDAAAAA|794|0|13|14|AM|third|night|| +795|AAAAAAAAMBDAAAAA|795|0|13|15|AM|third|night|| +796|AAAAAAAANBDAAAAA|796|0|13|16|AM|third|night|| +797|AAAAAAAAOBDAAAAA|797|0|13|17|AM|third|night|| +798|AAAAAAAAPBDAAAAA|798|0|13|18|AM|third|night|| +799|AAAAAAAAACDAAAAA|799|0|13|19|AM|third|night|| +800|AAAAAAAABCDAAAAA|800|0|13|20|AM|third|night|| +801|AAAAAAAACCDAAAAA|801|0|13|21|AM|third|night|| +802|AAAAAAAADCDAAAAA|802|0|13|22|AM|third|night|| +803|AAAAAAAAECDAAAAA|803|0|13|23|AM|third|night|| +804|AAAAAAAAFCDAAAAA|804|0|13|24|AM|third|night|| +805|AAAAAAAAGCDAAAAA|805|0|13|25|AM|third|night|| +806|AAAAAAAAHCDAAAAA|806|0|13|26|AM|third|night|| +807|AAAAAAAAICDAAAAA|807|0|13|27|AM|third|night|| +808|AAAAAAAAJCDAAAAA|808|0|13|28|AM|third|night|| +809|AAAAAAAAKCDAAAAA|809|0|13|29|AM|third|night|| +810|AAAAAAAALCDAAAAA|810|0|13|30|AM|third|night|| +811|AAAAAAAAMCDAAAAA|811|0|13|31|AM|third|night|| +812|AAAAAAAANCDAAAAA|812|0|13|32|AM|third|night|| +813|AAAAAAAAOCDAAAAA|813|0|13|33|AM|third|night|| +814|AAAAAAAAPCDAAAAA|814|0|13|34|AM|third|night|| +815|AAAAAAAAADDAAAAA|815|0|13|35|AM|third|night|| +816|AAAAAAAABDDAAAAA|816|0|13|36|AM|third|night|| +817|AAAAAAAACDDAAAAA|817|0|13|37|AM|third|night|| +818|AAAAAAAADDDAAAAA|818|0|13|38|AM|third|night|| +819|AAAAAAAAEDDAAAAA|819|0|13|39|AM|third|night|| +820|AAAAAAAAFDDAAAAA|820|0|13|40|AM|third|night|| +821|AAAAAAAAGDDAAAAA|821|0|13|41|AM|third|night|| +822|AAAAAAAAHDDAAAAA|822|0|13|42|AM|third|night|| +823|AAAAAAAAIDDAAAAA|823|0|13|43|AM|third|night|| +824|AAAAAAAAJDDAAAAA|824|0|13|44|AM|third|night|| +825|AAAAAAAAKDDAAAAA|825|0|13|45|AM|third|night|| +826|AAAAAAAALDDAAAAA|826|0|13|46|AM|third|night|| +827|AAAAAAAAMDDAAAAA|827|0|13|47|AM|third|night|| +828|AAAAAAAANDDAAAAA|828|0|13|48|AM|third|night|| +829|AAAAAAAAODDAAAAA|829|0|13|49|AM|third|night|| +830|AAAAAAAAPDDAAAAA|830|0|13|50|AM|third|night|| +831|AAAAAAAAAEDAAAAA|831|0|13|51|AM|third|night|| +832|AAAAAAAABEDAAAAA|832|0|13|52|AM|third|night|| +833|AAAAAAAACEDAAAAA|833|0|13|53|AM|third|night|| +834|AAAAAAAADEDAAAAA|834|0|13|54|AM|third|night|| +835|AAAAAAAAEEDAAAAA|835|0|13|55|AM|third|night|| +836|AAAAAAAAFEDAAAAA|836|0|13|56|AM|third|night|| +837|AAAAAAAAGEDAAAAA|837|0|13|57|AM|third|night|| +838|AAAAAAAAHEDAAAAA|838|0|13|58|AM|third|night|| +839|AAAAAAAAIEDAAAAA|839|0|13|59|AM|third|night|| +840|AAAAAAAAJEDAAAAA|840|0|14|0|AM|third|night|| +841|AAAAAAAAKEDAAAAA|841|0|14|1|AM|third|night|| +842|AAAAAAAALEDAAAAA|842|0|14|2|AM|third|night|| +843|AAAAAAAAMEDAAAAA|843|0|14|3|AM|third|night|| +844|AAAAAAAANEDAAAAA|844|0|14|4|AM|third|night|| +845|AAAAAAAAOEDAAAAA|845|0|14|5|AM|third|night|| +846|AAAAAAAAPEDAAAAA|846|0|14|6|AM|third|night|| +847|AAAAAAAAAFDAAAAA|847|0|14|7|AM|third|night|| +848|AAAAAAAABFDAAAAA|848|0|14|8|AM|third|night|| +849|AAAAAAAACFDAAAAA|849|0|14|9|AM|third|night|| +850|AAAAAAAADFDAAAAA|850|0|14|10|AM|third|night|| +851|AAAAAAAAEFDAAAAA|851|0|14|11|AM|third|night|| +852|AAAAAAAAFFDAAAAA|852|0|14|12|AM|third|night|| +853|AAAAAAAAGFDAAAAA|853|0|14|13|AM|third|night|| +854|AAAAAAAAHFDAAAAA|854|0|14|14|AM|third|night|| +855|AAAAAAAAIFDAAAAA|855|0|14|15|AM|third|night|| +856|AAAAAAAAJFDAAAAA|856|0|14|16|AM|third|night|| +857|AAAAAAAAKFDAAAAA|857|0|14|17|AM|third|night|| +858|AAAAAAAALFDAAAAA|858|0|14|18|AM|third|night|| +859|AAAAAAAAMFDAAAAA|859|0|14|19|AM|third|night|| +860|AAAAAAAANFDAAAAA|860|0|14|20|AM|third|night|| +861|AAAAAAAAOFDAAAAA|861|0|14|21|AM|third|night|| +862|AAAAAAAAPFDAAAAA|862|0|14|22|AM|third|night|| +863|AAAAAAAAAGDAAAAA|863|0|14|23|AM|third|night|| +864|AAAAAAAABGDAAAAA|864|0|14|24|AM|third|night|| +865|AAAAAAAACGDAAAAA|865|0|14|25|AM|third|night|| +866|AAAAAAAADGDAAAAA|866|0|14|26|AM|third|night|| +867|AAAAAAAAEGDAAAAA|867|0|14|27|AM|third|night|| +868|AAAAAAAAFGDAAAAA|868|0|14|28|AM|third|night|| +869|AAAAAAAAGGDAAAAA|869|0|14|29|AM|third|night|| +870|AAAAAAAAHGDAAAAA|870|0|14|30|AM|third|night|| +871|AAAAAAAAIGDAAAAA|871|0|14|31|AM|third|night|| +872|AAAAAAAAJGDAAAAA|872|0|14|32|AM|third|night|| +873|AAAAAAAAKGDAAAAA|873|0|14|33|AM|third|night|| +874|AAAAAAAALGDAAAAA|874|0|14|34|AM|third|night|| +875|AAAAAAAAMGDAAAAA|875|0|14|35|AM|third|night|| +876|AAAAAAAANGDAAAAA|876|0|14|36|AM|third|night|| +877|AAAAAAAAOGDAAAAA|877|0|14|37|AM|third|night|| +878|AAAAAAAAPGDAAAAA|878|0|14|38|AM|third|night|| +879|AAAAAAAAAHDAAAAA|879|0|14|39|AM|third|night|| +880|AAAAAAAABHDAAAAA|880|0|14|40|AM|third|night|| +881|AAAAAAAACHDAAAAA|881|0|14|41|AM|third|night|| +882|AAAAAAAADHDAAAAA|882|0|14|42|AM|third|night|| +883|AAAAAAAAEHDAAAAA|883|0|14|43|AM|third|night|| +884|AAAAAAAAFHDAAAAA|884|0|14|44|AM|third|night|| +885|AAAAAAAAGHDAAAAA|885|0|14|45|AM|third|night|| +886|AAAAAAAAHHDAAAAA|886|0|14|46|AM|third|night|| +887|AAAAAAAAIHDAAAAA|887|0|14|47|AM|third|night|| +888|AAAAAAAAJHDAAAAA|888|0|14|48|AM|third|night|| +889|AAAAAAAAKHDAAAAA|889|0|14|49|AM|third|night|| +890|AAAAAAAALHDAAAAA|890|0|14|50|AM|third|night|| +891|AAAAAAAAMHDAAAAA|891|0|14|51|AM|third|night|| +892|AAAAAAAANHDAAAAA|892|0|14|52|AM|third|night|| +893|AAAAAAAAOHDAAAAA|893|0|14|53|AM|third|night|| +894|AAAAAAAAPHDAAAAA|894|0|14|54|AM|third|night|| +895|AAAAAAAAAIDAAAAA|895|0|14|55|AM|third|night|| +896|AAAAAAAABIDAAAAA|896|0|14|56|AM|third|night|| +897|AAAAAAAACIDAAAAA|897|0|14|57|AM|third|night|| +898|AAAAAAAADIDAAAAA|898|0|14|58|AM|third|night|| +899|AAAAAAAAEIDAAAAA|899|0|14|59|AM|third|night|| +900|AAAAAAAAFIDAAAAA|900|0|15|0|AM|third|night|| +901|AAAAAAAAGIDAAAAA|901|0|15|1|AM|third|night|| +902|AAAAAAAAHIDAAAAA|902|0|15|2|AM|third|night|| +903|AAAAAAAAIIDAAAAA|903|0|15|3|AM|third|night|| +904|AAAAAAAAJIDAAAAA|904|0|15|4|AM|third|night|| +905|AAAAAAAAKIDAAAAA|905|0|15|5|AM|third|night|| +906|AAAAAAAALIDAAAAA|906|0|15|6|AM|third|night|| +907|AAAAAAAAMIDAAAAA|907|0|15|7|AM|third|night|| +908|AAAAAAAANIDAAAAA|908|0|15|8|AM|third|night|| +909|AAAAAAAAOIDAAAAA|909|0|15|9|AM|third|night|| +910|AAAAAAAAPIDAAAAA|910|0|15|10|AM|third|night|| +911|AAAAAAAAAJDAAAAA|911|0|15|11|AM|third|night|| +912|AAAAAAAABJDAAAAA|912|0|15|12|AM|third|night|| +913|AAAAAAAACJDAAAAA|913|0|15|13|AM|third|night|| +914|AAAAAAAADJDAAAAA|914|0|15|14|AM|third|night|| +915|AAAAAAAAEJDAAAAA|915|0|15|15|AM|third|night|| +916|AAAAAAAAFJDAAAAA|916|0|15|16|AM|third|night|| +917|AAAAAAAAGJDAAAAA|917|0|15|17|AM|third|night|| +918|AAAAAAAAHJDAAAAA|918|0|15|18|AM|third|night|| +919|AAAAAAAAIJDAAAAA|919|0|15|19|AM|third|night|| +920|AAAAAAAAJJDAAAAA|920|0|15|20|AM|third|night|| +921|AAAAAAAAKJDAAAAA|921|0|15|21|AM|third|night|| +922|AAAAAAAALJDAAAAA|922|0|15|22|AM|third|night|| +923|AAAAAAAAMJDAAAAA|923|0|15|23|AM|third|night|| +924|AAAAAAAANJDAAAAA|924|0|15|24|AM|third|night|| +925|AAAAAAAAOJDAAAAA|925|0|15|25|AM|third|night|| +926|AAAAAAAAPJDAAAAA|926|0|15|26|AM|third|night|| +927|AAAAAAAAAKDAAAAA|927|0|15|27|AM|third|night|| +928|AAAAAAAABKDAAAAA|928|0|15|28|AM|third|night|| +929|AAAAAAAACKDAAAAA|929|0|15|29|AM|third|night|| +930|AAAAAAAADKDAAAAA|930|0|15|30|AM|third|night|| +931|AAAAAAAAEKDAAAAA|931|0|15|31|AM|third|night|| +932|AAAAAAAAFKDAAAAA|932|0|15|32|AM|third|night|| +933|AAAAAAAAGKDAAAAA|933|0|15|33|AM|third|night|| +934|AAAAAAAAHKDAAAAA|934|0|15|34|AM|third|night|| +935|AAAAAAAAIKDAAAAA|935|0|15|35|AM|third|night|| +936|AAAAAAAAJKDAAAAA|936|0|15|36|AM|third|night|| +937|AAAAAAAAKKDAAAAA|937|0|15|37|AM|third|night|| +938|AAAAAAAALKDAAAAA|938|0|15|38|AM|third|night|| +939|AAAAAAAAMKDAAAAA|939|0|15|39|AM|third|night|| +940|AAAAAAAANKDAAAAA|940|0|15|40|AM|third|night|| +941|AAAAAAAAOKDAAAAA|941|0|15|41|AM|third|night|| +942|AAAAAAAAPKDAAAAA|942|0|15|42|AM|third|night|| +943|AAAAAAAAALDAAAAA|943|0|15|43|AM|third|night|| +944|AAAAAAAABLDAAAAA|944|0|15|44|AM|third|night|| +945|AAAAAAAACLDAAAAA|945|0|15|45|AM|third|night|| +946|AAAAAAAADLDAAAAA|946|0|15|46|AM|third|night|| +947|AAAAAAAAELDAAAAA|947|0|15|47|AM|third|night|| +948|AAAAAAAAFLDAAAAA|948|0|15|48|AM|third|night|| +949|AAAAAAAAGLDAAAAA|949|0|15|49|AM|third|night|| +950|AAAAAAAAHLDAAAAA|950|0|15|50|AM|third|night|| +951|AAAAAAAAILDAAAAA|951|0|15|51|AM|third|night|| +952|AAAAAAAAJLDAAAAA|952|0|15|52|AM|third|night|| +953|AAAAAAAAKLDAAAAA|953|0|15|53|AM|third|night|| +954|AAAAAAAALLDAAAAA|954|0|15|54|AM|third|night|| +955|AAAAAAAAMLDAAAAA|955|0|15|55|AM|third|night|| +956|AAAAAAAANLDAAAAA|956|0|15|56|AM|third|night|| +957|AAAAAAAAOLDAAAAA|957|0|15|57|AM|third|night|| +958|AAAAAAAAPLDAAAAA|958|0|15|58|AM|third|night|| +959|AAAAAAAAAMDAAAAA|959|0|15|59|AM|third|night|| +960|AAAAAAAABMDAAAAA|960|0|16|0|AM|third|night|| +961|AAAAAAAACMDAAAAA|961|0|16|1|AM|third|night|| +962|AAAAAAAADMDAAAAA|962|0|16|2|AM|third|night|| +963|AAAAAAAAEMDAAAAA|963|0|16|3|AM|third|night|| +964|AAAAAAAAFMDAAAAA|964|0|16|4|AM|third|night|| +965|AAAAAAAAGMDAAAAA|965|0|16|5|AM|third|night|| +966|AAAAAAAAHMDAAAAA|966|0|16|6|AM|third|night|| +967|AAAAAAAAIMDAAAAA|967|0|16|7|AM|third|night|| +968|AAAAAAAAJMDAAAAA|968|0|16|8|AM|third|night|| +969|AAAAAAAAKMDAAAAA|969|0|16|9|AM|third|night|| +970|AAAAAAAALMDAAAAA|970|0|16|10|AM|third|night|| +971|AAAAAAAAMMDAAAAA|971|0|16|11|AM|third|night|| +972|AAAAAAAANMDAAAAA|972|0|16|12|AM|third|night|| +973|AAAAAAAAOMDAAAAA|973|0|16|13|AM|third|night|| +974|AAAAAAAAPMDAAAAA|974|0|16|14|AM|third|night|| +975|AAAAAAAAANDAAAAA|975|0|16|15|AM|third|night|| +976|AAAAAAAABNDAAAAA|976|0|16|16|AM|third|night|| +977|AAAAAAAACNDAAAAA|977|0|16|17|AM|third|night|| +978|AAAAAAAADNDAAAAA|978|0|16|18|AM|third|night|| +979|AAAAAAAAENDAAAAA|979|0|16|19|AM|third|night|| +980|AAAAAAAAFNDAAAAA|980|0|16|20|AM|third|night|| +981|AAAAAAAAGNDAAAAA|981|0|16|21|AM|third|night|| +982|AAAAAAAAHNDAAAAA|982|0|16|22|AM|third|night|| +983|AAAAAAAAINDAAAAA|983|0|16|23|AM|third|night|| +984|AAAAAAAAJNDAAAAA|984|0|16|24|AM|third|night|| +985|AAAAAAAAKNDAAAAA|985|0|16|25|AM|third|night|| +986|AAAAAAAALNDAAAAA|986|0|16|26|AM|third|night|| +987|AAAAAAAAMNDAAAAA|987|0|16|27|AM|third|night|| +988|AAAAAAAANNDAAAAA|988|0|16|28|AM|third|night|| +989|AAAAAAAAONDAAAAA|989|0|16|29|AM|third|night|| +990|AAAAAAAAPNDAAAAA|990|0|16|30|AM|third|night|| +991|AAAAAAAAAODAAAAA|991|0|16|31|AM|third|night|| +992|AAAAAAAABODAAAAA|992|0|16|32|AM|third|night|| +993|AAAAAAAACODAAAAA|993|0|16|33|AM|third|night|| +994|AAAAAAAADODAAAAA|994|0|16|34|AM|third|night|| +995|AAAAAAAAEODAAAAA|995|0|16|35|AM|third|night|| +996|AAAAAAAAFODAAAAA|996|0|16|36|AM|third|night|| +997|AAAAAAAAGODAAAAA|997|0|16|37|AM|third|night|| +998|AAAAAAAAHODAAAAA|998|0|16|38|AM|third|night|| +999|AAAAAAAAIODAAAAA|999|0|16|39|AM|third|night|| +1000|AAAAAAAAJODAAAAA|1000|0|16|40|AM|third|night|| +1001|AAAAAAAAKODAAAAA|1001|0|16|41|AM|third|night|| +1002|AAAAAAAALODAAAAA|1002|0|16|42|AM|third|night|| +1003|AAAAAAAAMODAAAAA|1003|0|16|43|AM|third|night|| +1004|AAAAAAAANODAAAAA|1004|0|16|44|AM|third|night|| +1005|AAAAAAAAOODAAAAA|1005|0|16|45|AM|third|night|| +1006|AAAAAAAAPODAAAAA|1006|0|16|46|AM|third|night|| +1007|AAAAAAAAAPDAAAAA|1007|0|16|47|AM|third|night|| +1008|AAAAAAAABPDAAAAA|1008|0|16|48|AM|third|night|| +1009|AAAAAAAACPDAAAAA|1009|0|16|49|AM|third|night|| +1010|AAAAAAAADPDAAAAA|1010|0|16|50|AM|third|night|| +1011|AAAAAAAAEPDAAAAA|1011|0|16|51|AM|third|night|| +1012|AAAAAAAAFPDAAAAA|1012|0|16|52|AM|third|night|| +1013|AAAAAAAAGPDAAAAA|1013|0|16|53|AM|third|night|| +1014|AAAAAAAAHPDAAAAA|1014|0|16|54|AM|third|night|| +1015|AAAAAAAAIPDAAAAA|1015|0|16|55|AM|third|night|| +1016|AAAAAAAAJPDAAAAA|1016|0|16|56|AM|third|night|| +1017|AAAAAAAAKPDAAAAA|1017|0|16|57|AM|third|night|| +1018|AAAAAAAALPDAAAAA|1018|0|16|58|AM|third|night|| +1019|AAAAAAAAMPDAAAAA|1019|0|16|59|AM|third|night|| +1020|AAAAAAAANPDAAAAA|1020|0|17|0|AM|third|night|| +1021|AAAAAAAAOPDAAAAA|1021|0|17|1|AM|third|night|| +1022|AAAAAAAAPPDAAAAA|1022|0|17|2|AM|third|night|| +1023|AAAAAAAAAAEAAAAA|1023|0|17|3|AM|third|night|| +1024|AAAAAAAABAEAAAAA|1024|0|17|4|AM|third|night|| +1025|AAAAAAAACAEAAAAA|1025|0|17|5|AM|third|night|| +1026|AAAAAAAADAEAAAAA|1026|0|17|6|AM|third|night|| +1027|AAAAAAAAEAEAAAAA|1027|0|17|7|AM|third|night|| +1028|AAAAAAAAFAEAAAAA|1028|0|17|8|AM|third|night|| +1029|AAAAAAAAGAEAAAAA|1029|0|17|9|AM|third|night|| +1030|AAAAAAAAHAEAAAAA|1030|0|17|10|AM|third|night|| +1031|AAAAAAAAIAEAAAAA|1031|0|17|11|AM|third|night|| +1032|AAAAAAAAJAEAAAAA|1032|0|17|12|AM|third|night|| +1033|AAAAAAAAKAEAAAAA|1033|0|17|13|AM|third|night|| +1034|AAAAAAAALAEAAAAA|1034|0|17|14|AM|third|night|| +1035|AAAAAAAAMAEAAAAA|1035|0|17|15|AM|third|night|| +1036|AAAAAAAANAEAAAAA|1036|0|17|16|AM|third|night|| +1037|AAAAAAAAOAEAAAAA|1037|0|17|17|AM|third|night|| +1038|AAAAAAAAPAEAAAAA|1038|0|17|18|AM|third|night|| +1039|AAAAAAAAABEAAAAA|1039|0|17|19|AM|third|night|| +1040|AAAAAAAABBEAAAAA|1040|0|17|20|AM|third|night|| +1041|AAAAAAAACBEAAAAA|1041|0|17|21|AM|third|night|| +1042|AAAAAAAADBEAAAAA|1042|0|17|22|AM|third|night|| +1043|AAAAAAAAEBEAAAAA|1043|0|17|23|AM|third|night|| +1044|AAAAAAAAFBEAAAAA|1044|0|17|24|AM|third|night|| +1045|AAAAAAAAGBEAAAAA|1045|0|17|25|AM|third|night|| +1046|AAAAAAAAHBEAAAAA|1046|0|17|26|AM|third|night|| +1047|AAAAAAAAIBEAAAAA|1047|0|17|27|AM|third|night|| +1048|AAAAAAAAJBEAAAAA|1048|0|17|28|AM|third|night|| +1049|AAAAAAAAKBEAAAAA|1049|0|17|29|AM|third|night|| +1050|AAAAAAAALBEAAAAA|1050|0|17|30|AM|third|night|| +1051|AAAAAAAAMBEAAAAA|1051|0|17|31|AM|third|night|| +1052|AAAAAAAANBEAAAAA|1052|0|17|32|AM|third|night|| +1053|AAAAAAAAOBEAAAAA|1053|0|17|33|AM|third|night|| +1054|AAAAAAAAPBEAAAAA|1054|0|17|34|AM|third|night|| +1055|AAAAAAAAACEAAAAA|1055|0|17|35|AM|third|night|| +1056|AAAAAAAABCEAAAAA|1056|0|17|36|AM|third|night|| +1057|AAAAAAAACCEAAAAA|1057|0|17|37|AM|third|night|| +1058|AAAAAAAADCEAAAAA|1058|0|17|38|AM|third|night|| +1059|AAAAAAAAECEAAAAA|1059|0|17|39|AM|third|night|| +1060|AAAAAAAAFCEAAAAA|1060|0|17|40|AM|third|night|| +1061|AAAAAAAAGCEAAAAA|1061|0|17|41|AM|third|night|| +1062|AAAAAAAAHCEAAAAA|1062|0|17|42|AM|third|night|| +1063|AAAAAAAAICEAAAAA|1063|0|17|43|AM|third|night|| +1064|AAAAAAAAJCEAAAAA|1064|0|17|44|AM|third|night|| +1065|AAAAAAAAKCEAAAAA|1065|0|17|45|AM|third|night|| +1066|AAAAAAAALCEAAAAA|1066|0|17|46|AM|third|night|| +1067|AAAAAAAAMCEAAAAA|1067|0|17|47|AM|third|night|| +1068|AAAAAAAANCEAAAAA|1068|0|17|48|AM|third|night|| +1069|AAAAAAAAOCEAAAAA|1069|0|17|49|AM|third|night|| +1070|AAAAAAAAPCEAAAAA|1070|0|17|50|AM|third|night|| +1071|AAAAAAAAADEAAAAA|1071|0|17|51|AM|third|night|| +1072|AAAAAAAABDEAAAAA|1072|0|17|52|AM|third|night|| +1073|AAAAAAAACDEAAAAA|1073|0|17|53|AM|third|night|| +1074|AAAAAAAADDEAAAAA|1074|0|17|54|AM|third|night|| +1075|AAAAAAAAEDEAAAAA|1075|0|17|55|AM|third|night|| +1076|AAAAAAAAFDEAAAAA|1076|0|17|56|AM|third|night|| +1077|AAAAAAAAGDEAAAAA|1077|0|17|57|AM|third|night|| +1078|AAAAAAAAHDEAAAAA|1078|0|17|58|AM|third|night|| +1079|AAAAAAAAIDEAAAAA|1079|0|17|59|AM|third|night|| +1080|AAAAAAAAJDEAAAAA|1080|0|18|0|AM|third|night|| +1081|AAAAAAAAKDEAAAAA|1081|0|18|1|AM|third|night|| +1082|AAAAAAAALDEAAAAA|1082|0|18|2|AM|third|night|| +1083|AAAAAAAAMDEAAAAA|1083|0|18|3|AM|third|night|| +1084|AAAAAAAANDEAAAAA|1084|0|18|4|AM|third|night|| +1085|AAAAAAAAODEAAAAA|1085|0|18|5|AM|third|night|| +1086|AAAAAAAAPDEAAAAA|1086|0|18|6|AM|third|night|| +1087|AAAAAAAAAEEAAAAA|1087|0|18|7|AM|third|night|| +1088|AAAAAAAABEEAAAAA|1088|0|18|8|AM|third|night|| +1089|AAAAAAAACEEAAAAA|1089|0|18|9|AM|third|night|| +1090|AAAAAAAADEEAAAAA|1090|0|18|10|AM|third|night|| +1091|AAAAAAAAEEEAAAAA|1091|0|18|11|AM|third|night|| +1092|AAAAAAAAFEEAAAAA|1092|0|18|12|AM|third|night|| +1093|AAAAAAAAGEEAAAAA|1093|0|18|13|AM|third|night|| +1094|AAAAAAAAHEEAAAAA|1094|0|18|14|AM|third|night|| +1095|AAAAAAAAIEEAAAAA|1095|0|18|15|AM|third|night|| +1096|AAAAAAAAJEEAAAAA|1096|0|18|16|AM|third|night|| +1097|AAAAAAAAKEEAAAAA|1097|0|18|17|AM|third|night|| +1098|AAAAAAAALEEAAAAA|1098|0|18|18|AM|third|night|| +1099|AAAAAAAAMEEAAAAA|1099|0|18|19|AM|third|night|| +1100|AAAAAAAANEEAAAAA|1100|0|18|20|AM|third|night|| +1101|AAAAAAAAOEEAAAAA|1101|0|18|21|AM|third|night|| +1102|AAAAAAAAPEEAAAAA|1102|0|18|22|AM|third|night|| +1103|AAAAAAAAAFEAAAAA|1103|0|18|23|AM|third|night|| +1104|AAAAAAAABFEAAAAA|1104|0|18|24|AM|third|night|| +1105|AAAAAAAACFEAAAAA|1105|0|18|25|AM|third|night|| +1106|AAAAAAAADFEAAAAA|1106|0|18|26|AM|third|night|| +1107|AAAAAAAAEFEAAAAA|1107|0|18|27|AM|third|night|| +1108|AAAAAAAAFFEAAAAA|1108|0|18|28|AM|third|night|| +1109|AAAAAAAAGFEAAAAA|1109|0|18|29|AM|third|night|| +1110|AAAAAAAAHFEAAAAA|1110|0|18|30|AM|third|night|| +1111|AAAAAAAAIFEAAAAA|1111|0|18|31|AM|third|night|| +1112|AAAAAAAAJFEAAAAA|1112|0|18|32|AM|third|night|| +1113|AAAAAAAAKFEAAAAA|1113|0|18|33|AM|third|night|| +1114|AAAAAAAALFEAAAAA|1114|0|18|34|AM|third|night|| +1115|AAAAAAAAMFEAAAAA|1115|0|18|35|AM|third|night|| +1116|AAAAAAAANFEAAAAA|1116|0|18|36|AM|third|night|| +1117|AAAAAAAAOFEAAAAA|1117|0|18|37|AM|third|night|| +1118|AAAAAAAAPFEAAAAA|1118|0|18|38|AM|third|night|| +1119|AAAAAAAAAGEAAAAA|1119|0|18|39|AM|third|night|| +1120|AAAAAAAABGEAAAAA|1120|0|18|40|AM|third|night|| +1121|AAAAAAAACGEAAAAA|1121|0|18|41|AM|third|night|| +1122|AAAAAAAADGEAAAAA|1122|0|18|42|AM|third|night|| +1123|AAAAAAAAEGEAAAAA|1123|0|18|43|AM|third|night|| +1124|AAAAAAAAFGEAAAAA|1124|0|18|44|AM|third|night|| +1125|AAAAAAAAGGEAAAAA|1125|0|18|45|AM|third|night|| +1126|AAAAAAAAHGEAAAAA|1126|0|18|46|AM|third|night|| +1127|AAAAAAAAIGEAAAAA|1127|0|18|47|AM|third|night|| +1128|AAAAAAAAJGEAAAAA|1128|0|18|48|AM|third|night|| +1129|AAAAAAAAKGEAAAAA|1129|0|18|49|AM|third|night|| +1130|AAAAAAAALGEAAAAA|1130|0|18|50|AM|third|night|| +1131|AAAAAAAAMGEAAAAA|1131|0|18|51|AM|third|night|| +1132|AAAAAAAANGEAAAAA|1132|0|18|52|AM|third|night|| +1133|AAAAAAAAOGEAAAAA|1133|0|18|53|AM|third|night|| +1134|AAAAAAAAPGEAAAAA|1134|0|18|54|AM|third|night|| +1135|AAAAAAAAAHEAAAAA|1135|0|18|55|AM|third|night|| +1136|AAAAAAAABHEAAAAA|1136|0|18|56|AM|third|night|| +1137|AAAAAAAACHEAAAAA|1137|0|18|57|AM|third|night|| +1138|AAAAAAAADHEAAAAA|1138|0|18|58|AM|third|night|| +1139|AAAAAAAAEHEAAAAA|1139|0|18|59|AM|third|night|| +1140|AAAAAAAAFHEAAAAA|1140|0|19|0|AM|third|night|| +1141|AAAAAAAAGHEAAAAA|1141|0|19|1|AM|third|night|| +1142|AAAAAAAAHHEAAAAA|1142|0|19|2|AM|third|night|| +1143|AAAAAAAAIHEAAAAA|1143|0|19|3|AM|third|night|| +1144|AAAAAAAAJHEAAAAA|1144|0|19|4|AM|third|night|| +1145|AAAAAAAAKHEAAAAA|1145|0|19|5|AM|third|night|| +1146|AAAAAAAALHEAAAAA|1146|0|19|6|AM|third|night|| +1147|AAAAAAAAMHEAAAAA|1147|0|19|7|AM|third|night|| +1148|AAAAAAAANHEAAAAA|1148|0|19|8|AM|third|night|| +1149|AAAAAAAAOHEAAAAA|1149|0|19|9|AM|third|night|| +1150|AAAAAAAAPHEAAAAA|1150|0|19|10|AM|third|night|| +1151|AAAAAAAAAIEAAAAA|1151|0|19|11|AM|third|night|| +1152|AAAAAAAABIEAAAAA|1152|0|19|12|AM|third|night|| +1153|AAAAAAAACIEAAAAA|1153|0|19|13|AM|third|night|| +1154|AAAAAAAADIEAAAAA|1154|0|19|14|AM|third|night|| +1155|AAAAAAAAEIEAAAAA|1155|0|19|15|AM|third|night|| +1156|AAAAAAAAFIEAAAAA|1156|0|19|16|AM|third|night|| +1157|AAAAAAAAGIEAAAAA|1157|0|19|17|AM|third|night|| +1158|AAAAAAAAHIEAAAAA|1158|0|19|18|AM|third|night|| +1159|AAAAAAAAIIEAAAAA|1159|0|19|19|AM|third|night|| +1160|AAAAAAAAJIEAAAAA|1160|0|19|20|AM|third|night|| +1161|AAAAAAAAKIEAAAAA|1161|0|19|21|AM|third|night|| +1162|AAAAAAAALIEAAAAA|1162|0|19|22|AM|third|night|| +1163|AAAAAAAAMIEAAAAA|1163|0|19|23|AM|third|night|| +1164|AAAAAAAANIEAAAAA|1164|0|19|24|AM|third|night|| +1165|AAAAAAAAOIEAAAAA|1165|0|19|25|AM|third|night|| +1166|AAAAAAAAPIEAAAAA|1166|0|19|26|AM|third|night|| +1167|AAAAAAAAAJEAAAAA|1167|0|19|27|AM|third|night|| +1168|AAAAAAAABJEAAAAA|1168|0|19|28|AM|third|night|| +1169|AAAAAAAACJEAAAAA|1169|0|19|29|AM|third|night|| +1170|AAAAAAAADJEAAAAA|1170|0|19|30|AM|third|night|| +1171|AAAAAAAAEJEAAAAA|1171|0|19|31|AM|third|night|| +1172|AAAAAAAAFJEAAAAA|1172|0|19|32|AM|third|night|| +1173|AAAAAAAAGJEAAAAA|1173|0|19|33|AM|third|night|| +1174|AAAAAAAAHJEAAAAA|1174|0|19|34|AM|third|night|| +1175|AAAAAAAAIJEAAAAA|1175|0|19|35|AM|third|night|| +1176|AAAAAAAAJJEAAAAA|1176|0|19|36|AM|third|night|| +1177|AAAAAAAAKJEAAAAA|1177|0|19|37|AM|third|night|| +1178|AAAAAAAALJEAAAAA|1178|0|19|38|AM|third|night|| +1179|AAAAAAAAMJEAAAAA|1179|0|19|39|AM|third|night|| +1180|AAAAAAAANJEAAAAA|1180|0|19|40|AM|third|night|| +1181|AAAAAAAAOJEAAAAA|1181|0|19|41|AM|third|night|| +1182|AAAAAAAAPJEAAAAA|1182|0|19|42|AM|third|night|| +1183|AAAAAAAAAKEAAAAA|1183|0|19|43|AM|third|night|| +1184|AAAAAAAABKEAAAAA|1184|0|19|44|AM|third|night|| +1185|AAAAAAAACKEAAAAA|1185|0|19|45|AM|third|night|| +1186|AAAAAAAADKEAAAAA|1186|0|19|46|AM|third|night|| +1187|AAAAAAAAEKEAAAAA|1187|0|19|47|AM|third|night|| +1188|AAAAAAAAFKEAAAAA|1188|0|19|48|AM|third|night|| +1189|AAAAAAAAGKEAAAAA|1189|0|19|49|AM|third|night|| +1190|AAAAAAAAHKEAAAAA|1190|0|19|50|AM|third|night|| +1191|AAAAAAAAIKEAAAAA|1191|0|19|51|AM|third|night|| +1192|AAAAAAAAJKEAAAAA|1192|0|19|52|AM|third|night|| +1193|AAAAAAAAKKEAAAAA|1193|0|19|53|AM|third|night|| +1194|AAAAAAAALKEAAAAA|1194|0|19|54|AM|third|night|| +1195|AAAAAAAAMKEAAAAA|1195|0|19|55|AM|third|night|| +1196|AAAAAAAANKEAAAAA|1196|0|19|56|AM|third|night|| +1197|AAAAAAAAOKEAAAAA|1197|0|19|57|AM|third|night|| +1198|AAAAAAAAPKEAAAAA|1198|0|19|58|AM|third|night|| +1199|AAAAAAAAALEAAAAA|1199|0|19|59|AM|third|night|| +1200|AAAAAAAABLEAAAAA|1200|0|20|0|AM|third|night|| +1201|AAAAAAAACLEAAAAA|1201|0|20|1|AM|third|night|| +1202|AAAAAAAADLEAAAAA|1202|0|20|2|AM|third|night|| +1203|AAAAAAAAELEAAAAA|1203|0|20|3|AM|third|night|| +1204|AAAAAAAAFLEAAAAA|1204|0|20|4|AM|third|night|| +1205|AAAAAAAAGLEAAAAA|1205|0|20|5|AM|third|night|| +1206|AAAAAAAAHLEAAAAA|1206|0|20|6|AM|third|night|| +1207|AAAAAAAAILEAAAAA|1207|0|20|7|AM|third|night|| +1208|AAAAAAAAJLEAAAAA|1208|0|20|8|AM|third|night|| +1209|AAAAAAAAKLEAAAAA|1209|0|20|9|AM|third|night|| +1210|AAAAAAAALLEAAAAA|1210|0|20|10|AM|third|night|| +1211|AAAAAAAAMLEAAAAA|1211|0|20|11|AM|third|night|| +1212|AAAAAAAANLEAAAAA|1212|0|20|12|AM|third|night|| +1213|AAAAAAAAOLEAAAAA|1213|0|20|13|AM|third|night|| +1214|AAAAAAAAPLEAAAAA|1214|0|20|14|AM|third|night|| +1215|AAAAAAAAAMEAAAAA|1215|0|20|15|AM|third|night|| +1216|AAAAAAAABMEAAAAA|1216|0|20|16|AM|third|night|| +1217|AAAAAAAACMEAAAAA|1217|0|20|17|AM|third|night|| +1218|AAAAAAAADMEAAAAA|1218|0|20|18|AM|third|night|| +1219|AAAAAAAAEMEAAAAA|1219|0|20|19|AM|third|night|| +1220|AAAAAAAAFMEAAAAA|1220|0|20|20|AM|third|night|| +1221|AAAAAAAAGMEAAAAA|1221|0|20|21|AM|third|night|| +1222|AAAAAAAAHMEAAAAA|1222|0|20|22|AM|third|night|| +1223|AAAAAAAAIMEAAAAA|1223|0|20|23|AM|third|night|| +1224|AAAAAAAAJMEAAAAA|1224|0|20|24|AM|third|night|| +1225|AAAAAAAAKMEAAAAA|1225|0|20|25|AM|third|night|| +1226|AAAAAAAALMEAAAAA|1226|0|20|26|AM|third|night|| +1227|AAAAAAAAMMEAAAAA|1227|0|20|27|AM|third|night|| +1228|AAAAAAAANMEAAAAA|1228|0|20|28|AM|third|night|| +1229|AAAAAAAAOMEAAAAA|1229|0|20|29|AM|third|night|| +1230|AAAAAAAAPMEAAAAA|1230|0|20|30|AM|third|night|| +1231|AAAAAAAAANEAAAAA|1231|0|20|31|AM|third|night|| +1232|AAAAAAAABNEAAAAA|1232|0|20|32|AM|third|night|| +1233|AAAAAAAACNEAAAAA|1233|0|20|33|AM|third|night|| +1234|AAAAAAAADNEAAAAA|1234|0|20|34|AM|third|night|| +1235|AAAAAAAAENEAAAAA|1235|0|20|35|AM|third|night|| +1236|AAAAAAAAFNEAAAAA|1236|0|20|36|AM|third|night|| +1237|AAAAAAAAGNEAAAAA|1237|0|20|37|AM|third|night|| +1238|AAAAAAAAHNEAAAAA|1238|0|20|38|AM|third|night|| +1239|AAAAAAAAINEAAAAA|1239|0|20|39|AM|third|night|| +1240|AAAAAAAAJNEAAAAA|1240|0|20|40|AM|third|night|| +1241|AAAAAAAAKNEAAAAA|1241|0|20|41|AM|third|night|| +1242|AAAAAAAALNEAAAAA|1242|0|20|42|AM|third|night|| +1243|AAAAAAAAMNEAAAAA|1243|0|20|43|AM|third|night|| +1244|AAAAAAAANNEAAAAA|1244|0|20|44|AM|third|night|| +1245|AAAAAAAAONEAAAAA|1245|0|20|45|AM|third|night|| +1246|AAAAAAAAPNEAAAAA|1246|0|20|46|AM|third|night|| +1247|AAAAAAAAAOEAAAAA|1247|0|20|47|AM|third|night|| +1248|AAAAAAAABOEAAAAA|1248|0|20|48|AM|third|night|| +1249|AAAAAAAACOEAAAAA|1249|0|20|49|AM|third|night|| +1250|AAAAAAAADOEAAAAA|1250|0|20|50|AM|third|night|| +1251|AAAAAAAAEOEAAAAA|1251|0|20|51|AM|third|night|| +1252|AAAAAAAAFOEAAAAA|1252|0|20|52|AM|third|night|| +1253|AAAAAAAAGOEAAAAA|1253|0|20|53|AM|third|night|| +1254|AAAAAAAAHOEAAAAA|1254|0|20|54|AM|third|night|| +1255|AAAAAAAAIOEAAAAA|1255|0|20|55|AM|third|night|| +1256|AAAAAAAAJOEAAAAA|1256|0|20|56|AM|third|night|| +1257|AAAAAAAAKOEAAAAA|1257|0|20|57|AM|third|night|| +1258|AAAAAAAALOEAAAAA|1258|0|20|58|AM|third|night|| +1259|AAAAAAAAMOEAAAAA|1259|0|20|59|AM|third|night|| +1260|AAAAAAAANOEAAAAA|1260|0|21|0|AM|third|night|| +1261|AAAAAAAAOOEAAAAA|1261|0|21|1|AM|third|night|| +1262|AAAAAAAAPOEAAAAA|1262|0|21|2|AM|third|night|| +1263|AAAAAAAAAPEAAAAA|1263|0|21|3|AM|third|night|| +1264|AAAAAAAABPEAAAAA|1264|0|21|4|AM|third|night|| +1265|AAAAAAAACPEAAAAA|1265|0|21|5|AM|third|night|| +1266|AAAAAAAADPEAAAAA|1266|0|21|6|AM|third|night|| +1267|AAAAAAAAEPEAAAAA|1267|0|21|7|AM|third|night|| +1268|AAAAAAAAFPEAAAAA|1268|0|21|8|AM|third|night|| +1269|AAAAAAAAGPEAAAAA|1269|0|21|9|AM|third|night|| +1270|AAAAAAAAHPEAAAAA|1270|0|21|10|AM|third|night|| +1271|AAAAAAAAIPEAAAAA|1271|0|21|11|AM|third|night|| +1272|AAAAAAAAJPEAAAAA|1272|0|21|12|AM|third|night|| +1273|AAAAAAAAKPEAAAAA|1273|0|21|13|AM|third|night|| +1274|AAAAAAAALPEAAAAA|1274|0|21|14|AM|third|night|| +1275|AAAAAAAAMPEAAAAA|1275|0|21|15|AM|third|night|| +1276|AAAAAAAANPEAAAAA|1276|0|21|16|AM|third|night|| +1277|AAAAAAAAOPEAAAAA|1277|0|21|17|AM|third|night|| +1278|AAAAAAAAPPEAAAAA|1278|0|21|18|AM|third|night|| +1279|AAAAAAAAAAFAAAAA|1279|0|21|19|AM|third|night|| +1280|AAAAAAAABAFAAAAA|1280|0|21|20|AM|third|night|| +1281|AAAAAAAACAFAAAAA|1281|0|21|21|AM|third|night|| +1282|AAAAAAAADAFAAAAA|1282|0|21|22|AM|third|night|| +1283|AAAAAAAAEAFAAAAA|1283|0|21|23|AM|third|night|| +1284|AAAAAAAAFAFAAAAA|1284|0|21|24|AM|third|night|| +1285|AAAAAAAAGAFAAAAA|1285|0|21|25|AM|third|night|| +1286|AAAAAAAAHAFAAAAA|1286|0|21|26|AM|third|night|| +1287|AAAAAAAAIAFAAAAA|1287|0|21|27|AM|third|night|| +1288|AAAAAAAAJAFAAAAA|1288|0|21|28|AM|third|night|| +1289|AAAAAAAAKAFAAAAA|1289|0|21|29|AM|third|night|| +1290|AAAAAAAALAFAAAAA|1290|0|21|30|AM|third|night|| +1291|AAAAAAAAMAFAAAAA|1291|0|21|31|AM|third|night|| +1292|AAAAAAAANAFAAAAA|1292|0|21|32|AM|third|night|| +1293|AAAAAAAAOAFAAAAA|1293|0|21|33|AM|third|night|| +1294|AAAAAAAAPAFAAAAA|1294|0|21|34|AM|third|night|| +1295|AAAAAAAAABFAAAAA|1295|0|21|35|AM|third|night|| +1296|AAAAAAAABBFAAAAA|1296|0|21|36|AM|third|night|| +1297|AAAAAAAACBFAAAAA|1297|0|21|37|AM|third|night|| +1298|AAAAAAAADBFAAAAA|1298|0|21|38|AM|third|night|| +1299|AAAAAAAAEBFAAAAA|1299|0|21|39|AM|third|night|| +1300|AAAAAAAAFBFAAAAA|1300|0|21|40|AM|third|night|| +1301|AAAAAAAAGBFAAAAA|1301|0|21|41|AM|third|night|| +1302|AAAAAAAAHBFAAAAA|1302|0|21|42|AM|third|night|| +1303|AAAAAAAAIBFAAAAA|1303|0|21|43|AM|third|night|| +1304|AAAAAAAAJBFAAAAA|1304|0|21|44|AM|third|night|| +1305|AAAAAAAAKBFAAAAA|1305|0|21|45|AM|third|night|| +1306|AAAAAAAALBFAAAAA|1306|0|21|46|AM|third|night|| +1307|AAAAAAAAMBFAAAAA|1307|0|21|47|AM|third|night|| +1308|AAAAAAAANBFAAAAA|1308|0|21|48|AM|third|night|| +1309|AAAAAAAAOBFAAAAA|1309|0|21|49|AM|third|night|| +1310|AAAAAAAAPBFAAAAA|1310|0|21|50|AM|third|night|| +1311|AAAAAAAAACFAAAAA|1311|0|21|51|AM|third|night|| +1312|AAAAAAAABCFAAAAA|1312|0|21|52|AM|third|night|| +1313|AAAAAAAACCFAAAAA|1313|0|21|53|AM|third|night|| +1314|AAAAAAAADCFAAAAA|1314|0|21|54|AM|third|night|| +1315|AAAAAAAAECFAAAAA|1315|0|21|55|AM|third|night|| +1316|AAAAAAAAFCFAAAAA|1316|0|21|56|AM|third|night|| +1317|AAAAAAAAGCFAAAAA|1317|0|21|57|AM|third|night|| +1318|AAAAAAAAHCFAAAAA|1318|0|21|58|AM|third|night|| +1319|AAAAAAAAICFAAAAA|1319|0|21|59|AM|third|night|| +1320|AAAAAAAAJCFAAAAA|1320|0|22|0|AM|third|night|| +1321|AAAAAAAAKCFAAAAA|1321|0|22|1|AM|third|night|| +1322|AAAAAAAALCFAAAAA|1322|0|22|2|AM|third|night|| +1323|AAAAAAAAMCFAAAAA|1323|0|22|3|AM|third|night|| +1324|AAAAAAAANCFAAAAA|1324|0|22|4|AM|third|night|| +1325|AAAAAAAAOCFAAAAA|1325|0|22|5|AM|third|night|| +1326|AAAAAAAAPCFAAAAA|1326|0|22|6|AM|third|night|| +1327|AAAAAAAAADFAAAAA|1327|0|22|7|AM|third|night|| +1328|AAAAAAAABDFAAAAA|1328|0|22|8|AM|third|night|| +1329|AAAAAAAACDFAAAAA|1329|0|22|9|AM|third|night|| +1330|AAAAAAAADDFAAAAA|1330|0|22|10|AM|third|night|| +1331|AAAAAAAAEDFAAAAA|1331|0|22|11|AM|third|night|| +1332|AAAAAAAAFDFAAAAA|1332|0|22|12|AM|third|night|| +1333|AAAAAAAAGDFAAAAA|1333|0|22|13|AM|third|night|| +1334|AAAAAAAAHDFAAAAA|1334|0|22|14|AM|third|night|| +1335|AAAAAAAAIDFAAAAA|1335|0|22|15|AM|third|night|| +1336|AAAAAAAAJDFAAAAA|1336|0|22|16|AM|third|night|| +1337|AAAAAAAAKDFAAAAA|1337|0|22|17|AM|third|night|| +1338|AAAAAAAALDFAAAAA|1338|0|22|18|AM|third|night|| +1339|AAAAAAAAMDFAAAAA|1339|0|22|19|AM|third|night|| +1340|AAAAAAAANDFAAAAA|1340|0|22|20|AM|third|night|| +1341|AAAAAAAAODFAAAAA|1341|0|22|21|AM|third|night|| +1342|AAAAAAAAPDFAAAAA|1342|0|22|22|AM|third|night|| +1343|AAAAAAAAAEFAAAAA|1343|0|22|23|AM|third|night|| +1344|AAAAAAAABEFAAAAA|1344|0|22|24|AM|third|night|| +1345|AAAAAAAACEFAAAAA|1345|0|22|25|AM|third|night|| +1346|AAAAAAAADEFAAAAA|1346|0|22|26|AM|third|night|| +1347|AAAAAAAAEEFAAAAA|1347|0|22|27|AM|third|night|| +1348|AAAAAAAAFEFAAAAA|1348|0|22|28|AM|third|night|| +1349|AAAAAAAAGEFAAAAA|1349|0|22|29|AM|third|night|| +1350|AAAAAAAAHEFAAAAA|1350|0|22|30|AM|third|night|| +1351|AAAAAAAAIEFAAAAA|1351|0|22|31|AM|third|night|| +1352|AAAAAAAAJEFAAAAA|1352|0|22|32|AM|third|night|| +1353|AAAAAAAAKEFAAAAA|1353|0|22|33|AM|third|night|| +1354|AAAAAAAALEFAAAAA|1354|0|22|34|AM|third|night|| +1355|AAAAAAAAMEFAAAAA|1355|0|22|35|AM|third|night|| +1356|AAAAAAAANEFAAAAA|1356|0|22|36|AM|third|night|| +1357|AAAAAAAAOEFAAAAA|1357|0|22|37|AM|third|night|| +1358|AAAAAAAAPEFAAAAA|1358|0|22|38|AM|third|night|| +1359|AAAAAAAAAFFAAAAA|1359|0|22|39|AM|third|night|| +1360|AAAAAAAABFFAAAAA|1360|0|22|40|AM|third|night|| +1361|AAAAAAAACFFAAAAA|1361|0|22|41|AM|third|night|| +1362|AAAAAAAADFFAAAAA|1362|0|22|42|AM|third|night|| +1363|AAAAAAAAEFFAAAAA|1363|0|22|43|AM|third|night|| +1364|AAAAAAAAFFFAAAAA|1364|0|22|44|AM|third|night|| +1365|AAAAAAAAGFFAAAAA|1365|0|22|45|AM|third|night|| +1366|AAAAAAAAHFFAAAAA|1366|0|22|46|AM|third|night|| +1367|AAAAAAAAIFFAAAAA|1367|0|22|47|AM|third|night|| +1368|AAAAAAAAJFFAAAAA|1368|0|22|48|AM|third|night|| +1369|AAAAAAAAKFFAAAAA|1369|0|22|49|AM|third|night|| +1370|AAAAAAAALFFAAAAA|1370|0|22|50|AM|third|night|| +1371|AAAAAAAAMFFAAAAA|1371|0|22|51|AM|third|night|| +1372|AAAAAAAANFFAAAAA|1372|0|22|52|AM|third|night|| +1373|AAAAAAAAOFFAAAAA|1373|0|22|53|AM|third|night|| +1374|AAAAAAAAPFFAAAAA|1374|0|22|54|AM|third|night|| +1375|AAAAAAAAAGFAAAAA|1375|0|22|55|AM|third|night|| +1376|AAAAAAAABGFAAAAA|1376|0|22|56|AM|third|night|| +1377|AAAAAAAACGFAAAAA|1377|0|22|57|AM|third|night|| +1378|AAAAAAAADGFAAAAA|1378|0|22|58|AM|third|night|| +1379|AAAAAAAAEGFAAAAA|1379|0|22|59|AM|third|night|| +1380|AAAAAAAAFGFAAAAA|1380|0|23|0|AM|third|night|| +1381|AAAAAAAAGGFAAAAA|1381|0|23|1|AM|third|night|| +1382|AAAAAAAAHGFAAAAA|1382|0|23|2|AM|third|night|| +1383|AAAAAAAAIGFAAAAA|1383|0|23|3|AM|third|night|| +1384|AAAAAAAAJGFAAAAA|1384|0|23|4|AM|third|night|| +1385|AAAAAAAAKGFAAAAA|1385|0|23|5|AM|third|night|| +1386|AAAAAAAALGFAAAAA|1386|0|23|6|AM|third|night|| +1387|AAAAAAAAMGFAAAAA|1387|0|23|7|AM|third|night|| +1388|AAAAAAAANGFAAAAA|1388|0|23|8|AM|third|night|| +1389|AAAAAAAAOGFAAAAA|1389|0|23|9|AM|third|night|| +1390|AAAAAAAAPGFAAAAA|1390|0|23|10|AM|third|night|| +1391|AAAAAAAAAHFAAAAA|1391|0|23|11|AM|third|night|| +1392|AAAAAAAABHFAAAAA|1392|0|23|12|AM|third|night|| +1393|AAAAAAAACHFAAAAA|1393|0|23|13|AM|third|night|| +1394|AAAAAAAADHFAAAAA|1394|0|23|14|AM|third|night|| +1395|AAAAAAAAEHFAAAAA|1395|0|23|15|AM|third|night|| +1396|AAAAAAAAFHFAAAAA|1396|0|23|16|AM|third|night|| +1397|AAAAAAAAGHFAAAAA|1397|0|23|17|AM|third|night|| +1398|AAAAAAAAHHFAAAAA|1398|0|23|18|AM|third|night|| +1399|AAAAAAAAIHFAAAAA|1399|0|23|19|AM|third|night|| +1400|AAAAAAAAJHFAAAAA|1400|0|23|20|AM|third|night|| +1401|AAAAAAAAKHFAAAAA|1401|0|23|21|AM|third|night|| +1402|AAAAAAAALHFAAAAA|1402|0|23|22|AM|third|night|| +1403|AAAAAAAAMHFAAAAA|1403|0|23|23|AM|third|night|| +1404|AAAAAAAANHFAAAAA|1404|0|23|24|AM|third|night|| +1405|AAAAAAAAOHFAAAAA|1405|0|23|25|AM|third|night|| +1406|AAAAAAAAPHFAAAAA|1406|0|23|26|AM|third|night|| +1407|AAAAAAAAAIFAAAAA|1407|0|23|27|AM|third|night|| +1408|AAAAAAAABIFAAAAA|1408|0|23|28|AM|third|night|| +1409|AAAAAAAACIFAAAAA|1409|0|23|29|AM|third|night|| +1410|AAAAAAAADIFAAAAA|1410|0|23|30|AM|third|night|| +1411|AAAAAAAAEIFAAAAA|1411|0|23|31|AM|third|night|| +1412|AAAAAAAAFIFAAAAA|1412|0|23|32|AM|third|night|| +1413|AAAAAAAAGIFAAAAA|1413|0|23|33|AM|third|night|| +1414|AAAAAAAAHIFAAAAA|1414|0|23|34|AM|third|night|| +1415|AAAAAAAAIIFAAAAA|1415|0|23|35|AM|third|night|| +1416|AAAAAAAAJIFAAAAA|1416|0|23|36|AM|third|night|| +1417|AAAAAAAAKIFAAAAA|1417|0|23|37|AM|third|night|| +1418|AAAAAAAALIFAAAAA|1418|0|23|38|AM|third|night|| +1419|AAAAAAAAMIFAAAAA|1419|0|23|39|AM|third|night|| +1420|AAAAAAAANIFAAAAA|1420|0|23|40|AM|third|night|| +1421|AAAAAAAAOIFAAAAA|1421|0|23|41|AM|third|night|| +1422|AAAAAAAAPIFAAAAA|1422|0|23|42|AM|third|night|| +1423|AAAAAAAAAJFAAAAA|1423|0|23|43|AM|third|night|| +1424|AAAAAAAABJFAAAAA|1424|0|23|44|AM|third|night|| +1425|AAAAAAAACJFAAAAA|1425|0|23|45|AM|third|night|| +1426|AAAAAAAADJFAAAAA|1426|0|23|46|AM|third|night|| +1427|AAAAAAAAEJFAAAAA|1427|0|23|47|AM|third|night|| +1428|AAAAAAAAFJFAAAAA|1428|0|23|48|AM|third|night|| +1429|AAAAAAAAGJFAAAAA|1429|0|23|49|AM|third|night|| +1430|AAAAAAAAHJFAAAAA|1430|0|23|50|AM|third|night|| +1431|AAAAAAAAIJFAAAAA|1431|0|23|51|AM|third|night|| +1432|AAAAAAAAJJFAAAAA|1432|0|23|52|AM|third|night|| +1433|AAAAAAAAKJFAAAAA|1433|0|23|53|AM|third|night|| +1434|AAAAAAAALJFAAAAA|1434|0|23|54|AM|third|night|| +1435|AAAAAAAAMJFAAAAA|1435|0|23|55|AM|third|night|| +1436|AAAAAAAANJFAAAAA|1436|0|23|56|AM|third|night|| +1437|AAAAAAAAOJFAAAAA|1437|0|23|57|AM|third|night|| +1438|AAAAAAAAPJFAAAAA|1438|0|23|58|AM|third|night|| +1439|AAAAAAAAAKFAAAAA|1439|0|23|59|AM|third|night|| +1440|AAAAAAAABKFAAAAA|1440|0|24|0|AM|third|night|| +1441|AAAAAAAACKFAAAAA|1441|0|24|1|AM|third|night|| +1442|AAAAAAAADKFAAAAA|1442|0|24|2|AM|third|night|| +1443|AAAAAAAAEKFAAAAA|1443|0|24|3|AM|third|night|| +1444|AAAAAAAAFKFAAAAA|1444|0|24|4|AM|third|night|| +1445|AAAAAAAAGKFAAAAA|1445|0|24|5|AM|third|night|| +1446|AAAAAAAAHKFAAAAA|1446|0|24|6|AM|third|night|| +1447|AAAAAAAAIKFAAAAA|1447|0|24|7|AM|third|night|| +1448|AAAAAAAAJKFAAAAA|1448|0|24|8|AM|third|night|| +1449|AAAAAAAAKKFAAAAA|1449|0|24|9|AM|third|night|| +1450|AAAAAAAALKFAAAAA|1450|0|24|10|AM|third|night|| +1451|AAAAAAAAMKFAAAAA|1451|0|24|11|AM|third|night|| +1452|AAAAAAAANKFAAAAA|1452|0|24|12|AM|third|night|| +1453|AAAAAAAAOKFAAAAA|1453|0|24|13|AM|third|night|| +1454|AAAAAAAAPKFAAAAA|1454|0|24|14|AM|third|night|| +1455|AAAAAAAAALFAAAAA|1455|0|24|15|AM|third|night|| +1456|AAAAAAAABLFAAAAA|1456|0|24|16|AM|third|night|| +1457|AAAAAAAACLFAAAAA|1457|0|24|17|AM|third|night|| +1458|AAAAAAAADLFAAAAA|1458|0|24|18|AM|third|night|| +1459|AAAAAAAAELFAAAAA|1459|0|24|19|AM|third|night|| +1460|AAAAAAAAFLFAAAAA|1460|0|24|20|AM|third|night|| +1461|AAAAAAAAGLFAAAAA|1461|0|24|21|AM|third|night|| +1462|AAAAAAAAHLFAAAAA|1462|0|24|22|AM|third|night|| +1463|AAAAAAAAILFAAAAA|1463|0|24|23|AM|third|night|| +1464|AAAAAAAAJLFAAAAA|1464|0|24|24|AM|third|night|| +1465|AAAAAAAAKLFAAAAA|1465|0|24|25|AM|third|night|| +1466|AAAAAAAALLFAAAAA|1466|0|24|26|AM|third|night|| +1467|AAAAAAAAMLFAAAAA|1467|0|24|27|AM|third|night|| +1468|AAAAAAAANLFAAAAA|1468|0|24|28|AM|third|night|| +1469|AAAAAAAAOLFAAAAA|1469|0|24|29|AM|third|night|| +1470|AAAAAAAAPLFAAAAA|1470|0|24|30|AM|third|night|| +1471|AAAAAAAAAMFAAAAA|1471|0|24|31|AM|third|night|| +1472|AAAAAAAABMFAAAAA|1472|0|24|32|AM|third|night|| +1473|AAAAAAAACMFAAAAA|1473|0|24|33|AM|third|night|| +1474|AAAAAAAADMFAAAAA|1474|0|24|34|AM|third|night|| +1475|AAAAAAAAEMFAAAAA|1475|0|24|35|AM|third|night|| +1476|AAAAAAAAFMFAAAAA|1476|0|24|36|AM|third|night|| +1477|AAAAAAAAGMFAAAAA|1477|0|24|37|AM|third|night|| +1478|AAAAAAAAHMFAAAAA|1478|0|24|38|AM|third|night|| +1479|AAAAAAAAIMFAAAAA|1479|0|24|39|AM|third|night|| +1480|AAAAAAAAJMFAAAAA|1480|0|24|40|AM|third|night|| +1481|AAAAAAAAKMFAAAAA|1481|0|24|41|AM|third|night|| +1482|AAAAAAAALMFAAAAA|1482|0|24|42|AM|third|night|| +1483|AAAAAAAAMMFAAAAA|1483|0|24|43|AM|third|night|| +1484|AAAAAAAANMFAAAAA|1484|0|24|44|AM|third|night|| +1485|AAAAAAAAOMFAAAAA|1485|0|24|45|AM|third|night|| +1486|AAAAAAAAPMFAAAAA|1486|0|24|46|AM|third|night|| +1487|AAAAAAAAANFAAAAA|1487|0|24|47|AM|third|night|| +1488|AAAAAAAABNFAAAAA|1488|0|24|48|AM|third|night|| +1489|AAAAAAAACNFAAAAA|1489|0|24|49|AM|third|night|| +1490|AAAAAAAADNFAAAAA|1490|0|24|50|AM|third|night|| +1491|AAAAAAAAENFAAAAA|1491|0|24|51|AM|third|night|| +1492|AAAAAAAAFNFAAAAA|1492|0|24|52|AM|third|night|| +1493|AAAAAAAAGNFAAAAA|1493|0|24|53|AM|third|night|| +1494|AAAAAAAAHNFAAAAA|1494|0|24|54|AM|third|night|| +1495|AAAAAAAAINFAAAAA|1495|0|24|55|AM|third|night|| +1496|AAAAAAAAJNFAAAAA|1496|0|24|56|AM|third|night|| +1497|AAAAAAAAKNFAAAAA|1497|0|24|57|AM|third|night|| +1498|AAAAAAAALNFAAAAA|1498|0|24|58|AM|third|night|| +1499|AAAAAAAAMNFAAAAA|1499|0|24|59|AM|third|night|| +1500|AAAAAAAANNFAAAAA|1500|0|25|0|AM|third|night|| +1501|AAAAAAAAONFAAAAA|1501|0|25|1|AM|third|night|| +1502|AAAAAAAAPNFAAAAA|1502|0|25|2|AM|third|night|| +1503|AAAAAAAAAOFAAAAA|1503|0|25|3|AM|third|night|| +1504|AAAAAAAABOFAAAAA|1504|0|25|4|AM|third|night|| +1505|AAAAAAAACOFAAAAA|1505|0|25|5|AM|third|night|| +1506|AAAAAAAADOFAAAAA|1506|0|25|6|AM|third|night|| +1507|AAAAAAAAEOFAAAAA|1507|0|25|7|AM|third|night|| +1508|AAAAAAAAFOFAAAAA|1508|0|25|8|AM|third|night|| +1509|AAAAAAAAGOFAAAAA|1509|0|25|9|AM|third|night|| +1510|AAAAAAAAHOFAAAAA|1510|0|25|10|AM|third|night|| +1511|AAAAAAAAIOFAAAAA|1511|0|25|11|AM|third|night|| +1512|AAAAAAAAJOFAAAAA|1512|0|25|12|AM|third|night|| +1513|AAAAAAAAKOFAAAAA|1513|0|25|13|AM|third|night|| +1514|AAAAAAAALOFAAAAA|1514|0|25|14|AM|third|night|| +1515|AAAAAAAAMOFAAAAA|1515|0|25|15|AM|third|night|| +1516|AAAAAAAANOFAAAAA|1516|0|25|16|AM|third|night|| +1517|AAAAAAAAOOFAAAAA|1517|0|25|17|AM|third|night|| +1518|AAAAAAAAPOFAAAAA|1518|0|25|18|AM|third|night|| +1519|AAAAAAAAAPFAAAAA|1519|0|25|19|AM|third|night|| +1520|AAAAAAAABPFAAAAA|1520|0|25|20|AM|third|night|| +1521|AAAAAAAACPFAAAAA|1521|0|25|21|AM|third|night|| +1522|AAAAAAAADPFAAAAA|1522|0|25|22|AM|third|night|| +1523|AAAAAAAAEPFAAAAA|1523|0|25|23|AM|third|night|| +1524|AAAAAAAAFPFAAAAA|1524|0|25|24|AM|third|night|| +1525|AAAAAAAAGPFAAAAA|1525|0|25|25|AM|third|night|| +1526|AAAAAAAAHPFAAAAA|1526|0|25|26|AM|third|night|| +1527|AAAAAAAAIPFAAAAA|1527|0|25|27|AM|third|night|| +1528|AAAAAAAAJPFAAAAA|1528|0|25|28|AM|third|night|| +1529|AAAAAAAAKPFAAAAA|1529|0|25|29|AM|third|night|| +1530|AAAAAAAALPFAAAAA|1530|0|25|30|AM|third|night|| +1531|AAAAAAAAMPFAAAAA|1531|0|25|31|AM|third|night|| +1532|AAAAAAAANPFAAAAA|1532|0|25|32|AM|third|night|| +1533|AAAAAAAAOPFAAAAA|1533|0|25|33|AM|third|night|| +1534|AAAAAAAAPPFAAAAA|1534|0|25|34|AM|third|night|| +1535|AAAAAAAAAAGAAAAA|1535|0|25|35|AM|third|night|| +1536|AAAAAAAABAGAAAAA|1536|0|25|36|AM|third|night|| +1537|AAAAAAAACAGAAAAA|1537|0|25|37|AM|third|night|| +1538|AAAAAAAADAGAAAAA|1538|0|25|38|AM|third|night|| +1539|AAAAAAAAEAGAAAAA|1539|0|25|39|AM|third|night|| +1540|AAAAAAAAFAGAAAAA|1540|0|25|40|AM|third|night|| +1541|AAAAAAAAGAGAAAAA|1541|0|25|41|AM|third|night|| +1542|AAAAAAAAHAGAAAAA|1542|0|25|42|AM|third|night|| +1543|AAAAAAAAIAGAAAAA|1543|0|25|43|AM|third|night|| +1544|AAAAAAAAJAGAAAAA|1544|0|25|44|AM|third|night|| +1545|AAAAAAAAKAGAAAAA|1545|0|25|45|AM|third|night|| +1546|AAAAAAAALAGAAAAA|1546|0|25|46|AM|third|night|| +1547|AAAAAAAAMAGAAAAA|1547|0|25|47|AM|third|night|| +1548|AAAAAAAANAGAAAAA|1548|0|25|48|AM|third|night|| +1549|AAAAAAAAOAGAAAAA|1549|0|25|49|AM|third|night|| +1550|AAAAAAAAPAGAAAAA|1550|0|25|50|AM|third|night|| +1551|AAAAAAAAABGAAAAA|1551|0|25|51|AM|third|night|| +1552|AAAAAAAABBGAAAAA|1552|0|25|52|AM|third|night|| +1553|AAAAAAAACBGAAAAA|1553|0|25|53|AM|third|night|| +1554|AAAAAAAADBGAAAAA|1554|0|25|54|AM|third|night|| +1555|AAAAAAAAEBGAAAAA|1555|0|25|55|AM|third|night|| +1556|AAAAAAAAFBGAAAAA|1556|0|25|56|AM|third|night|| +1557|AAAAAAAAGBGAAAAA|1557|0|25|57|AM|third|night|| +1558|AAAAAAAAHBGAAAAA|1558|0|25|58|AM|third|night|| +1559|AAAAAAAAIBGAAAAA|1559|0|25|59|AM|third|night|| +1560|AAAAAAAAJBGAAAAA|1560|0|26|0|AM|third|night|| +1561|AAAAAAAAKBGAAAAA|1561|0|26|1|AM|third|night|| +1562|AAAAAAAALBGAAAAA|1562|0|26|2|AM|third|night|| +1563|AAAAAAAAMBGAAAAA|1563|0|26|3|AM|third|night|| +1564|AAAAAAAANBGAAAAA|1564|0|26|4|AM|third|night|| +1565|AAAAAAAAOBGAAAAA|1565|0|26|5|AM|third|night|| +1566|AAAAAAAAPBGAAAAA|1566|0|26|6|AM|third|night|| +1567|AAAAAAAAACGAAAAA|1567|0|26|7|AM|third|night|| +1568|AAAAAAAABCGAAAAA|1568|0|26|8|AM|third|night|| +1569|AAAAAAAACCGAAAAA|1569|0|26|9|AM|third|night|| +1570|AAAAAAAADCGAAAAA|1570|0|26|10|AM|third|night|| +1571|AAAAAAAAECGAAAAA|1571|0|26|11|AM|third|night|| +1572|AAAAAAAAFCGAAAAA|1572|0|26|12|AM|third|night|| +1573|AAAAAAAAGCGAAAAA|1573|0|26|13|AM|third|night|| +1574|AAAAAAAAHCGAAAAA|1574|0|26|14|AM|third|night|| +1575|AAAAAAAAICGAAAAA|1575|0|26|15|AM|third|night|| +1576|AAAAAAAAJCGAAAAA|1576|0|26|16|AM|third|night|| +1577|AAAAAAAAKCGAAAAA|1577|0|26|17|AM|third|night|| +1578|AAAAAAAALCGAAAAA|1578|0|26|18|AM|third|night|| +1579|AAAAAAAAMCGAAAAA|1579|0|26|19|AM|third|night|| +1580|AAAAAAAANCGAAAAA|1580|0|26|20|AM|third|night|| +1581|AAAAAAAAOCGAAAAA|1581|0|26|21|AM|third|night|| +1582|AAAAAAAAPCGAAAAA|1582|0|26|22|AM|third|night|| +1583|AAAAAAAAADGAAAAA|1583|0|26|23|AM|third|night|| +1584|AAAAAAAABDGAAAAA|1584|0|26|24|AM|third|night|| +1585|AAAAAAAACDGAAAAA|1585|0|26|25|AM|third|night|| +1586|AAAAAAAADDGAAAAA|1586|0|26|26|AM|third|night|| +1587|AAAAAAAAEDGAAAAA|1587|0|26|27|AM|third|night|| +1588|AAAAAAAAFDGAAAAA|1588|0|26|28|AM|third|night|| +1589|AAAAAAAAGDGAAAAA|1589|0|26|29|AM|third|night|| +1590|AAAAAAAAHDGAAAAA|1590|0|26|30|AM|third|night|| +1591|AAAAAAAAIDGAAAAA|1591|0|26|31|AM|third|night|| +1592|AAAAAAAAJDGAAAAA|1592|0|26|32|AM|third|night|| +1593|AAAAAAAAKDGAAAAA|1593|0|26|33|AM|third|night|| +1594|AAAAAAAALDGAAAAA|1594|0|26|34|AM|third|night|| +1595|AAAAAAAAMDGAAAAA|1595|0|26|35|AM|third|night|| +1596|AAAAAAAANDGAAAAA|1596|0|26|36|AM|third|night|| +1597|AAAAAAAAODGAAAAA|1597|0|26|37|AM|third|night|| +1598|AAAAAAAAPDGAAAAA|1598|0|26|38|AM|third|night|| +1599|AAAAAAAAAEGAAAAA|1599|0|26|39|AM|third|night|| +1600|AAAAAAAABEGAAAAA|1600|0|26|40|AM|third|night|| +1601|AAAAAAAACEGAAAAA|1601|0|26|41|AM|third|night|| +1602|AAAAAAAADEGAAAAA|1602|0|26|42|AM|third|night|| +1603|AAAAAAAAEEGAAAAA|1603|0|26|43|AM|third|night|| +1604|AAAAAAAAFEGAAAAA|1604|0|26|44|AM|third|night|| +1605|AAAAAAAAGEGAAAAA|1605|0|26|45|AM|third|night|| +1606|AAAAAAAAHEGAAAAA|1606|0|26|46|AM|third|night|| +1607|AAAAAAAAIEGAAAAA|1607|0|26|47|AM|third|night|| +1608|AAAAAAAAJEGAAAAA|1608|0|26|48|AM|third|night|| +1609|AAAAAAAAKEGAAAAA|1609|0|26|49|AM|third|night|| +1610|AAAAAAAALEGAAAAA|1610|0|26|50|AM|third|night|| +1611|AAAAAAAAMEGAAAAA|1611|0|26|51|AM|third|night|| +1612|AAAAAAAANEGAAAAA|1612|0|26|52|AM|third|night|| +1613|AAAAAAAAOEGAAAAA|1613|0|26|53|AM|third|night|| +1614|AAAAAAAAPEGAAAAA|1614|0|26|54|AM|third|night|| +1615|AAAAAAAAAFGAAAAA|1615|0|26|55|AM|third|night|| +1616|AAAAAAAABFGAAAAA|1616|0|26|56|AM|third|night|| +1617|AAAAAAAACFGAAAAA|1617|0|26|57|AM|third|night|| +1618|AAAAAAAADFGAAAAA|1618|0|26|58|AM|third|night|| +1619|AAAAAAAAEFGAAAAA|1619|0|26|59|AM|third|night|| +1620|AAAAAAAAFFGAAAAA|1620|0|27|0|AM|third|night|| +1621|AAAAAAAAGFGAAAAA|1621|0|27|1|AM|third|night|| +1622|AAAAAAAAHFGAAAAA|1622|0|27|2|AM|third|night|| +1623|AAAAAAAAIFGAAAAA|1623|0|27|3|AM|third|night|| +1624|AAAAAAAAJFGAAAAA|1624|0|27|4|AM|third|night|| +1625|AAAAAAAAKFGAAAAA|1625|0|27|5|AM|third|night|| +1626|AAAAAAAALFGAAAAA|1626|0|27|6|AM|third|night|| +1627|AAAAAAAAMFGAAAAA|1627|0|27|7|AM|third|night|| +1628|AAAAAAAANFGAAAAA|1628|0|27|8|AM|third|night|| +1629|AAAAAAAAOFGAAAAA|1629|0|27|9|AM|third|night|| +1630|AAAAAAAAPFGAAAAA|1630|0|27|10|AM|third|night|| +1631|AAAAAAAAAGGAAAAA|1631|0|27|11|AM|third|night|| +1632|AAAAAAAABGGAAAAA|1632|0|27|12|AM|third|night|| +1633|AAAAAAAACGGAAAAA|1633|0|27|13|AM|third|night|| +1634|AAAAAAAADGGAAAAA|1634|0|27|14|AM|third|night|| +1635|AAAAAAAAEGGAAAAA|1635|0|27|15|AM|third|night|| +1636|AAAAAAAAFGGAAAAA|1636|0|27|16|AM|third|night|| +1637|AAAAAAAAGGGAAAAA|1637|0|27|17|AM|third|night|| +1638|AAAAAAAAHGGAAAAA|1638|0|27|18|AM|third|night|| +1639|AAAAAAAAIGGAAAAA|1639|0|27|19|AM|third|night|| +1640|AAAAAAAAJGGAAAAA|1640|0|27|20|AM|third|night|| +1641|AAAAAAAAKGGAAAAA|1641|0|27|21|AM|third|night|| +1642|AAAAAAAALGGAAAAA|1642|0|27|22|AM|third|night|| +1643|AAAAAAAAMGGAAAAA|1643|0|27|23|AM|third|night|| +1644|AAAAAAAANGGAAAAA|1644|0|27|24|AM|third|night|| +1645|AAAAAAAAOGGAAAAA|1645|0|27|25|AM|third|night|| +1646|AAAAAAAAPGGAAAAA|1646|0|27|26|AM|third|night|| +1647|AAAAAAAAAHGAAAAA|1647|0|27|27|AM|third|night|| +1648|AAAAAAAABHGAAAAA|1648|0|27|28|AM|third|night|| +1649|AAAAAAAACHGAAAAA|1649|0|27|29|AM|third|night|| +1650|AAAAAAAADHGAAAAA|1650|0|27|30|AM|third|night|| +1651|AAAAAAAAEHGAAAAA|1651|0|27|31|AM|third|night|| +1652|AAAAAAAAFHGAAAAA|1652|0|27|32|AM|third|night|| +1653|AAAAAAAAGHGAAAAA|1653|0|27|33|AM|third|night|| +1654|AAAAAAAAHHGAAAAA|1654|0|27|34|AM|third|night|| +1655|AAAAAAAAIHGAAAAA|1655|0|27|35|AM|third|night|| +1656|AAAAAAAAJHGAAAAA|1656|0|27|36|AM|third|night|| +1657|AAAAAAAAKHGAAAAA|1657|0|27|37|AM|third|night|| +1658|AAAAAAAALHGAAAAA|1658|0|27|38|AM|third|night|| +1659|AAAAAAAAMHGAAAAA|1659|0|27|39|AM|third|night|| +1660|AAAAAAAANHGAAAAA|1660|0|27|40|AM|third|night|| +1661|AAAAAAAAOHGAAAAA|1661|0|27|41|AM|third|night|| +1662|AAAAAAAAPHGAAAAA|1662|0|27|42|AM|third|night|| +1663|AAAAAAAAAIGAAAAA|1663|0|27|43|AM|third|night|| +1664|AAAAAAAABIGAAAAA|1664|0|27|44|AM|third|night|| +1665|AAAAAAAACIGAAAAA|1665|0|27|45|AM|third|night|| +1666|AAAAAAAADIGAAAAA|1666|0|27|46|AM|third|night|| +1667|AAAAAAAAEIGAAAAA|1667|0|27|47|AM|third|night|| +1668|AAAAAAAAFIGAAAAA|1668|0|27|48|AM|third|night|| +1669|AAAAAAAAGIGAAAAA|1669|0|27|49|AM|third|night|| +1670|AAAAAAAAHIGAAAAA|1670|0|27|50|AM|third|night|| +1671|AAAAAAAAIIGAAAAA|1671|0|27|51|AM|third|night|| +1672|AAAAAAAAJIGAAAAA|1672|0|27|52|AM|third|night|| +1673|AAAAAAAAKIGAAAAA|1673|0|27|53|AM|third|night|| +1674|AAAAAAAALIGAAAAA|1674|0|27|54|AM|third|night|| +1675|AAAAAAAAMIGAAAAA|1675|0|27|55|AM|third|night|| +1676|AAAAAAAANIGAAAAA|1676|0|27|56|AM|third|night|| +1677|AAAAAAAAOIGAAAAA|1677|0|27|57|AM|third|night|| +1678|AAAAAAAAPIGAAAAA|1678|0|27|58|AM|third|night|| +1679|AAAAAAAAAJGAAAAA|1679|0|27|59|AM|third|night|| +1680|AAAAAAAABJGAAAAA|1680|0|28|0|AM|third|night|| +1681|AAAAAAAACJGAAAAA|1681|0|28|1|AM|third|night|| +1682|AAAAAAAADJGAAAAA|1682|0|28|2|AM|third|night|| +1683|AAAAAAAAEJGAAAAA|1683|0|28|3|AM|third|night|| +1684|AAAAAAAAFJGAAAAA|1684|0|28|4|AM|third|night|| +1685|AAAAAAAAGJGAAAAA|1685|0|28|5|AM|third|night|| +1686|AAAAAAAAHJGAAAAA|1686|0|28|6|AM|third|night|| +1687|AAAAAAAAIJGAAAAA|1687|0|28|7|AM|third|night|| +1688|AAAAAAAAJJGAAAAA|1688|0|28|8|AM|third|night|| +1689|AAAAAAAAKJGAAAAA|1689|0|28|9|AM|third|night|| +1690|AAAAAAAALJGAAAAA|1690|0|28|10|AM|third|night|| +1691|AAAAAAAAMJGAAAAA|1691|0|28|11|AM|third|night|| +1692|AAAAAAAANJGAAAAA|1692|0|28|12|AM|third|night|| +1693|AAAAAAAAOJGAAAAA|1693|0|28|13|AM|third|night|| +1694|AAAAAAAAPJGAAAAA|1694|0|28|14|AM|third|night|| +1695|AAAAAAAAAKGAAAAA|1695|0|28|15|AM|third|night|| +1696|AAAAAAAABKGAAAAA|1696|0|28|16|AM|third|night|| +1697|AAAAAAAACKGAAAAA|1697|0|28|17|AM|third|night|| +1698|AAAAAAAADKGAAAAA|1698|0|28|18|AM|third|night|| +1699|AAAAAAAAEKGAAAAA|1699|0|28|19|AM|third|night|| +1700|AAAAAAAAFKGAAAAA|1700|0|28|20|AM|third|night|| +1701|AAAAAAAAGKGAAAAA|1701|0|28|21|AM|third|night|| +1702|AAAAAAAAHKGAAAAA|1702|0|28|22|AM|third|night|| +1703|AAAAAAAAIKGAAAAA|1703|0|28|23|AM|third|night|| +1704|AAAAAAAAJKGAAAAA|1704|0|28|24|AM|third|night|| +1705|AAAAAAAAKKGAAAAA|1705|0|28|25|AM|third|night|| +1706|AAAAAAAALKGAAAAA|1706|0|28|26|AM|third|night|| +1707|AAAAAAAAMKGAAAAA|1707|0|28|27|AM|third|night|| +1708|AAAAAAAANKGAAAAA|1708|0|28|28|AM|third|night|| +1709|AAAAAAAAOKGAAAAA|1709|0|28|29|AM|third|night|| +1710|AAAAAAAAPKGAAAAA|1710|0|28|30|AM|third|night|| +1711|AAAAAAAAALGAAAAA|1711|0|28|31|AM|third|night|| +1712|AAAAAAAABLGAAAAA|1712|0|28|32|AM|third|night|| +1713|AAAAAAAACLGAAAAA|1713|0|28|33|AM|third|night|| +1714|AAAAAAAADLGAAAAA|1714|0|28|34|AM|third|night|| +1715|AAAAAAAAELGAAAAA|1715|0|28|35|AM|third|night|| +1716|AAAAAAAAFLGAAAAA|1716|0|28|36|AM|third|night|| +1717|AAAAAAAAGLGAAAAA|1717|0|28|37|AM|third|night|| +1718|AAAAAAAAHLGAAAAA|1718|0|28|38|AM|third|night|| +1719|AAAAAAAAILGAAAAA|1719|0|28|39|AM|third|night|| +1720|AAAAAAAAJLGAAAAA|1720|0|28|40|AM|third|night|| +1721|AAAAAAAAKLGAAAAA|1721|0|28|41|AM|third|night|| +1722|AAAAAAAALLGAAAAA|1722|0|28|42|AM|third|night|| +1723|AAAAAAAAMLGAAAAA|1723|0|28|43|AM|third|night|| +1724|AAAAAAAANLGAAAAA|1724|0|28|44|AM|third|night|| +1725|AAAAAAAAOLGAAAAA|1725|0|28|45|AM|third|night|| +1726|AAAAAAAAPLGAAAAA|1726|0|28|46|AM|third|night|| +1727|AAAAAAAAAMGAAAAA|1727|0|28|47|AM|third|night|| +1728|AAAAAAAABMGAAAAA|1728|0|28|48|AM|third|night|| +1729|AAAAAAAACMGAAAAA|1729|0|28|49|AM|third|night|| +1730|AAAAAAAADMGAAAAA|1730|0|28|50|AM|third|night|| +1731|AAAAAAAAEMGAAAAA|1731|0|28|51|AM|third|night|| +1732|AAAAAAAAFMGAAAAA|1732|0|28|52|AM|third|night|| +1733|AAAAAAAAGMGAAAAA|1733|0|28|53|AM|third|night|| +1734|AAAAAAAAHMGAAAAA|1734|0|28|54|AM|third|night|| +1735|AAAAAAAAIMGAAAAA|1735|0|28|55|AM|third|night|| +1736|AAAAAAAAJMGAAAAA|1736|0|28|56|AM|third|night|| +1737|AAAAAAAAKMGAAAAA|1737|0|28|57|AM|third|night|| +1738|AAAAAAAALMGAAAAA|1738|0|28|58|AM|third|night|| +1739|AAAAAAAAMMGAAAAA|1739|0|28|59|AM|third|night|| +1740|AAAAAAAANMGAAAAA|1740|0|29|0|AM|third|night|| +1741|AAAAAAAAOMGAAAAA|1741|0|29|1|AM|third|night|| +1742|AAAAAAAAPMGAAAAA|1742|0|29|2|AM|third|night|| +1743|AAAAAAAAANGAAAAA|1743|0|29|3|AM|third|night|| +1744|AAAAAAAABNGAAAAA|1744|0|29|4|AM|third|night|| +1745|AAAAAAAACNGAAAAA|1745|0|29|5|AM|third|night|| +1746|AAAAAAAADNGAAAAA|1746|0|29|6|AM|third|night|| +1747|AAAAAAAAENGAAAAA|1747|0|29|7|AM|third|night|| +1748|AAAAAAAAFNGAAAAA|1748|0|29|8|AM|third|night|| +1749|AAAAAAAAGNGAAAAA|1749|0|29|9|AM|third|night|| +1750|AAAAAAAAHNGAAAAA|1750|0|29|10|AM|third|night|| +1751|AAAAAAAAINGAAAAA|1751|0|29|11|AM|third|night|| +1752|AAAAAAAAJNGAAAAA|1752|0|29|12|AM|third|night|| +1753|AAAAAAAAKNGAAAAA|1753|0|29|13|AM|third|night|| +1754|AAAAAAAALNGAAAAA|1754|0|29|14|AM|third|night|| +1755|AAAAAAAAMNGAAAAA|1755|0|29|15|AM|third|night|| +1756|AAAAAAAANNGAAAAA|1756|0|29|16|AM|third|night|| +1757|AAAAAAAAONGAAAAA|1757|0|29|17|AM|third|night|| +1758|AAAAAAAAPNGAAAAA|1758|0|29|18|AM|third|night|| +1759|AAAAAAAAAOGAAAAA|1759|0|29|19|AM|third|night|| +1760|AAAAAAAABOGAAAAA|1760|0|29|20|AM|third|night|| +1761|AAAAAAAACOGAAAAA|1761|0|29|21|AM|third|night|| +1762|AAAAAAAADOGAAAAA|1762|0|29|22|AM|third|night|| +1763|AAAAAAAAEOGAAAAA|1763|0|29|23|AM|third|night|| +1764|AAAAAAAAFOGAAAAA|1764|0|29|24|AM|third|night|| +1765|AAAAAAAAGOGAAAAA|1765|0|29|25|AM|third|night|| +1766|AAAAAAAAHOGAAAAA|1766|0|29|26|AM|third|night|| +1767|AAAAAAAAIOGAAAAA|1767|0|29|27|AM|third|night|| +1768|AAAAAAAAJOGAAAAA|1768|0|29|28|AM|third|night|| +1769|AAAAAAAAKOGAAAAA|1769|0|29|29|AM|third|night|| +1770|AAAAAAAALOGAAAAA|1770|0|29|30|AM|third|night|| +1771|AAAAAAAAMOGAAAAA|1771|0|29|31|AM|third|night|| +1772|AAAAAAAANOGAAAAA|1772|0|29|32|AM|third|night|| +1773|AAAAAAAAOOGAAAAA|1773|0|29|33|AM|third|night|| +1774|AAAAAAAAPOGAAAAA|1774|0|29|34|AM|third|night|| +1775|AAAAAAAAAPGAAAAA|1775|0|29|35|AM|third|night|| +1776|AAAAAAAABPGAAAAA|1776|0|29|36|AM|third|night|| +1777|AAAAAAAACPGAAAAA|1777|0|29|37|AM|third|night|| +1778|AAAAAAAADPGAAAAA|1778|0|29|38|AM|third|night|| +1779|AAAAAAAAEPGAAAAA|1779|0|29|39|AM|third|night|| +1780|AAAAAAAAFPGAAAAA|1780|0|29|40|AM|third|night|| +1781|AAAAAAAAGPGAAAAA|1781|0|29|41|AM|third|night|| +1782|AAAAAAAAHPGAAAAA|1782|0|29|42|AM|third|night|| +1783|AAAAAAAAIPGAAAAA|1783|0|29|43|AM|third|night|| +1784|AAAAAAAAJPGAAAAA|1784|0|29|44|AM|third|night|| +1785|AAAAAAAAKPGAAAAA|1785|0|29|45|AM|third|night|| +1786|AAAAAAAALPGAAAAA|1786|0|29|46|AM|third|night|| +1787|AAAAAAAAMPGAAAAA|1787|0|29|47|AM|third|night|| +1788|AAAAAAAANPGAAAAA|1788|0|29|48|AM|third|night|| +1789|AAAAAAAAOPGAAAAA|1789|0|29|49|AM|third|night|| +1790|AAAAAAAAPPGAAAAA|1790|0|29|50|AM|third|night|| +1791|AAAAAAAAAAHAAAAA|1791|0|29|51|AM|third|night|| +1792|AAAAAAAABAHAAAAA|1792|0|29|52|AM|third|night|| +1793|AAAAAAAACAHAAAAA|1793|0|29|53|AM|third|night|| +1794|AAAAAAAADAHAAAAA|1794|0|29|54|AM|third|night|| +1795|AAAAAAAAEAHAAAAA|1795|0|29|55|AM|third|night|| +1796|AAAAAAAAFAHAAAAA|1796|0|29|56|AM|third|night|| +1797|AAAAAAAAGAHAAAAA|1797|0|29|57|AM|third|night|| +1798|AAAAAAAAHAHAAAAA|1798|0|29|58|AM|third|night|| +1799|AAAAAAAAIAHAAAAA|1799|0|29|59|AM|third|night|| +1800|AAAAAAAAJAHAAAAA|1800|0|30|0|AM|third|night|| +1801|AAAAAAAAKAHAAAAA|1801|0|30|1|AM|third|night|| +1802|AAAAAAAALAHAAAAA|1802|0|30|2|AM|third|night|| +1803|AAAAAAAAMAHAAAAA|1803|0|30|3|AM|third|night|| +1804|AAAAAAAANAHAAAAA|1804|0|30|4|AM|third|night|| +1805|AAAAAAAAOAHAAAAA|1805|0|30|5|AM|third|night|| +1806|AAAAAAAAPAHAAAAA|1806|0|30|6|AM|third|night|| +1807|AAAAAAAAABHAAAAA|1807|0|30|7|AM|third|night|| +1808|AAAAAAAABBHAAAAA|1808|0|30|8|AM|third|night|| +1809|AAAAAAAACBHAAAAA|1809|0|30|9|AM|third|night|| +1810|AAAAAAAADBHAAAAA|1810|0|30|10|AM|third|night|| +1811|AAAAAAAAEBHAAAAA|1811|0|30|11|AM|third|night|| +1812|AAAAAAAAFBHAAAAA|1812|0|30|12|AM|third|night|| +1813|AAAAAAAAGBHAAAAA|1813|0|30|13|AM|third|night|| +1814|AAAAAAAAHBHAAAAA|1814|0|30|14|AM|third|night|| +1815|AAAAAAAAIBHAAAAA|1815|0|30|15|AM|third|night|| +1816|AAAAAAAAJBHAAAAA|1816|0|30|16|AM|third|night|| +1817|AAAAAAAAKBHAAAAA|1817|0|30|17|AM|third|night|| +1818|AAAAAAAALBHAAAAA|1818|0|30|18|AM|third|night|| +1819|AAAAAAAAMBHAAAAA|1819|0|30|19|AM|third|night|| +1820|AAAAAAAANBHAAAAA|1820|0|30|20|AM|third|night|| +1821|AAAAAAAAOBHAAAAA|1821|0|30|21|AM|third|night|| +1822|AAAAAAAAPBHAAAAA|1822|0|30|22|AM|third|night|| +1823|AAAAAAAAACHAAAAA|1823|0|30|23|AM|third|night|| +1824|AAAAAAAABCHAAAAA|1824|0|30|24|AM|third|night|| +1825|AAAAAAAACCHAAAAA|1825|0|30|25|AM|third|night|| +1826|AAAAAAAADCHAAAAA|1826|0|30|26|AM|third|night|| +1827|AAAAAAAAECHAAAAA|1827|0|30|27|AM|third|night|| +1828|AAAAAAAAFCHAAAAA|1828|0|30|28|AM|third|night|| +1829|AAAAAAAAGCHAAAAA|1829|0|30|29|AM|third|night|| +1830|AAAAAAAAHCHAAAAA|1830|0|30|30|AM|third|night|| +1831|AAAAAAAAICHAAAAA|1831|0|30|31|AM|third|night|| +1832|AAAAAAAAJCHAAAAA|1832|0|30|32|AM|third|night|| +1833|AAAAAAAAKCHAAAAA|1833|0|30|33|AM|third|night|| +1834|AAAAAAAALCHAAAAA|1834|0|30|34|AM|third|night|| +1835|AAAAAAAAMCHAAAAA|1835|0|30|35|AM|third|night|| +1836|AAAAAAAANCHAAAAA|1836|0|30|36|AM|third|night|| +1837|AAAAAAAAOCHAAAAA|1837|0|30|37|AM|third|night|| +1838|AAAAAAAAPCHAAAAA|1838|0|30|38|AM|third|night|| +1839|AAAAAAAAADHAAAAA|1839|0|30|39|AM|third|night|| +1840|AAAAAAAABDHAAAAA|1840|0|30|40|AM|third|night|| +1841|AAAAAAAACDHAAAAA|1841|0|30|41|AM|third|night|| +1842|AAAAAAAADDHAAAAA|1842|0|30|42|AM|third|night|| +1843|AAAAAAAAEDHAAAAA|1843|0|30|43|AM|third|night|| +1844|AAAAAAAAFDHAAAAA|1844|0|30|44|AM|third|night|| +1845|AAAAAAAAGDHAAAAA|1845|0|30|45|AM|third|night|| +1846|AAAAAAAAHDHAAAAA|1846|0|30|46|AM|third|night|| +1847|AAAAAAAAIDHAAAAA|1847|0|30|47|AM|third|night|| +1848|AAAAAAAAJDHAAAAA|1848|0|30|48|AM|third|night|| +1849|AAAAAAAAKDHAAAAA|1849|0|30|49|AM|third|night|| +1850|AAAAAAAALDHAAAAA|1850|0|30|50|AM|third|night|| +1851|AAAAAAAAMDHAAAAA|1851|0|30|51|AM|third|night|| +1852|AAAAAAAANDHAAAAA|1852|0|30|52|AM|third|night|| +1853|AAAAAAAAODHAAAAA|1853|0|30|53|AM|third|night|| +1854|AAAAAAAAPDHAAAAA|1854|0|30|54|AM|third|night|| +1855|AAAAAAAAAEHAAAAA|1855|0|30|55|AM|third|night|| +1856|AAAAAAAABEHAAAAA|1856|0|30|56|AM|third|night|| +1857|AAAAAAAACEHAAAAA|1857|0|30|57|AM|third|night|| +1858|AAAAAAAADEHAAAAA|1858|0|30|58|AM|third|night|| +1859|AAAAAAAAEEHAAAAA|1859|0|30|59|AM|third|night|| +1860|AAAAAAAAFEHAAAAA|1860|0|31|0|AM|third|night|| +1861|AAAAAAAAGEHAAAAA|1861|0|31|1|AM|third|night|| +1862|AAAAAAAAHEHAAAAA|1862|0|31|2|AM|third|night|| +1863|AAAAAAAAIEHAAAAA|1863|0|31|3|AM|third|night|| +1864|AAAAAAAAJEHAAAAA|1864|0|31|4|AM|third|night|| +1865|AAAAAAAAKEHAAAAA|1865|0|31|5|AM|third|night|| +1866|AAAAAAAALEHAAAAA|1866|0|31|6|AM|third|night|| +1867|AAAAAAAAMEHAAAAA|1867|0|31|7|AM|third|night|| +1868|AAAAAAAANEHAAAAA|1868|0|31|8|AM|third|night|| +1869|AAAAAAAAOEHAAAAA|1869|0|31|9|AM|third|night|| +1870|AAAAAAAAPEHAAAAA|1870|0|31|10|AM|third|night|| +1871|AAAAAAAAAFHAAAAA|1871|0|31|11|AM|third|night|| +1872|AAAAAAAABFHAAAAA|1872|0|31|12|AM|third|night|| +1873|AAAAAAAACFHAAAAA|1873|0|31|13|AM|third|night|| +1874|AAAAAAAADFHAAAAA|1874|0|31|14|AM|third|night|| +1875|AAAAAAAAEFHAAAAA|1875|0|31|15|AM|third|night|| +1876|AAAAAAAAFFHAAAAA|1876|0|31|16|AM|third|night|| +1877|AAAAAAAAGFHAAAAA|1877|0|31|17|AM|third|night|| +1878|AAAAAAAAHFHAAAAA|1878|0|31|18|AM|third|night|| +1879|AAAAAAAAIFHAAAAA|1879|0|31|19|AM|third|night|| +1880|AAAAAAAAJFHAAAAA|1880|0|31|20|AM|third|night|| +1881|AAAAAAAAKFHAAAAA|1881|0|31|21|AM|third|night|| +1882|AAAAAAAALFHAAAAA|1882|0|31|22|AM|third|night|| +1883|AAAAAAAAMFHAAAAA|1883|0|31|23|AM|third|night|| +1884|AAAAAAAANFHAAAAA|1884|0|31|24|AM|third|night|| +1885|AAAAAAAAOFHAAAAA|1885|0|31|25|AM|third|night|| +1886|AAAAAAAAPFHAAAAA|1886|0|31|26|AM|third|night|| +1887|AAAAAAAAAGHAAAAA|1887|0|31|27|AM|third|night|| +1888|AAAAAAAABGHAAAAA|1888|0|31|28|AM|third|night|| +1889|AAAAAAAACGHAAAAA|1889|0|31|29|AM|third|night|| +1890|AAAAAAAADGHAAAAA|1890|0|31|30|AM|third|night|| +1891|AAAAAAAAEGHAAAAA|1891|0|31|31|AM|third|night|| +1892|AAAAAAAAFGHAAAAA|1892|0|31|32|AM|third|night|| +1893|AAAAAAAAGGHAAAAA|1893|0|31|33|AM|third|night|| +1894|AAAAAAAAHGHAAAAA|1894|0|31|34|AM|third|night|| +1895|AAAAAAAAIGHAAAAA|1895|0|31|35|AM|third|night|| +1896|AAAAAAAAJGHAAAAA|1896|0|31|36|AM|third|night|| +1897|AAAAAAAAKGHAAAAA|1897|0|31|37|AM|third|night|| +1898|AAAAAAAALGHAAAAA|1898|0|31|38|AM|third|night|| +1899|AAAAAAAAMGHAAAAA|1899|0|31|39|AM|third|night|| +1900|AAAAAAAANGHAAAAA|1900|0|31|40|AM|third|night|| +1901|AAAAAAAAOGHAAAAA|1901|0|31|41|AM|third|night|| +1902|AAAAAAAAPGHAAAAA|1902|0|31|42|AM|third|night|| +1903|AAAAAAAAAHHAAAAA|1903|0|31|43|AM|third|night|| +1904|AAAAAAAABHHAAAAA|1904|0|31|44|AM|third|night|| +1905|AAAAAAAACHHAAAAA|1905|0|31|45|AM|third|night|| +1906|AAAAAAAADHHAAAAA|1906|0|31|46|AM|third|night|| +1907|AAAAAAAAEHHAAAAA|1907|0|31|47|AM|third|night|| +1908|AAAAAAAAFHHAAAAA|1908|0|31|48|AM|third|night|| +1909|AAAAAAAAGHHAAAAA|1909|0|31|49|AM|third|night|| +1910|AAAAAAAAHHHAAAAA|1910|0|31|50|AM|third|night|| +1911|AAAAAAAAIHHAAAAA|1911|0|31|51|AM|third|night|| +1912|AAAAAAAAJHHAAAAA|1912|0|31|52|AM|third|night|| +1913|AAAAAAAAKHHAAAAA|1913|0|31|53|AM|third|night|| +1914|AAAAAAAALHHAAAAA|1914|0|31|54|AM|third|night|| +1915|AAAAAAAAMHHAAAAA|1915|0|31|55|AM|third|night|| +1916|AAAAAAAANHHAAAAA|1916|0|31|56|AM|third|night|| +1917|AAAAAAAAOHHAAAAA|1917|0|31|57|AM|third|night|| +1918|AAAAAAAAPHHAAAAA|1918|0|31|58|AM|third|night|| +1919|AAAAAAAAAIHAAAAA|1919|0|31|59|AM|third|night|| +1920|AAAAAAAABIHAAAAA|1920|0|32|0|AM|third|night|| +1921|AAAAAAAACIHAAAAA|1921|0|32|1|AM|third|night|| +1922|AAAAAAAADIHAAAAA|1922|0|32|2|AM|third|night|| +1923|AAAAAAAAEIHAAAAA|1923|0|32|3|AM|third|night|| +1924|AAAAAAAAFIHAAAAA|1924|0|32|4|AM|third|night|| +1925|AAAAAAAAGIHAAAAA|1925|0|32|5|AM|third|night|| +1926|AAAAAAAAHIHAAAAA|1926|0|32|6|AM|third|night|| +1927|AAAAAAAAIIHAAAAA|1927|0|32|7|AM|third|night|| +1928|AAAAAAAAJIHAAAAA|1928|0|32|8|AM|third|night|| +1929|AAAAAAAAKIHAAAAA|1929|0|32|9|AM|third|night|| +1930|AAAAAAAALIHAAAAA|1930|0|32|10|AM|third|night|| +1931|AAAAAAAAMIHAAAAA|1931|0|32|11|AM|third|night|| +1932|AAAAAAAANIHAAAAA|1932|0|32|12|AM|third|night|| +1933|AAAAAAAAOIHAAAAA|1933|0|32|13|AM|third|night|| +1934|AAAAAAAAPIHAAAAA|1934|0|32|14|AM|third|night|| +1935|AAAAAAAAAJHAAAAA|1935|0|32|15|AM|third|night|| +1936|AAAAAAAABJHAAAAA|1936|0|32|16|AM|third|night|| +1937|AAAAAAAACJHAAAAA|1937|0|32|17|AM|third|night|| +1938|AAAAAAAADJHAAAAA|1938|0|32|18|AM|third|night|| +1939|AAAAAAAAEJHAAAAA|1939|0|32|19|AM|third|night|| +1940|AAAAAAAAFJHAAAAA|1940|0|32|20|AM|third|night|| +1941|AAAAAAAAGJHAAAAA|1941|0|32|21|AM|third|night|| +1942|AAAAAAAAHJHAAAAA|1942|0|32|22|AM|third|night|| +1943|AAAAAAAAIJHAAAAA|1943|0|32|23|AM|third|night|| +1944|AAAAAAAAJJHAAAAA|1944|0|32|24|AM|third|night|| +1945|AAAAAAAAKJHAAAAA|1945|0|32|25|AM|third|night|| +1946|AAAAAAAALJHAAAAA|1946|0|32|26|AM|third|night|| +1947|AAAAAAAAMJHAAAAA|1947|0|32|27|AM|third|night|| +1948|AAAAAAAANJHAAAAA|1948|0|32|28|AM|third|night|| +1949|AAAAAAAAOJHAAAAA|1949|0|32|29|AM|third|night|| +1950|AAAAAAAAPJHAAAAA|1950|0|32|30|AM|third|night|| +1951|AAAAAAAAAKHAAAAA|1951|0|32|31|AM|third|night|| +1952|AAAAAAAABKHAAAAA|1952|0|32|32|AM|third|night|| +1953|AAAAAAAACKHAAAAA|1953|0|32|33|AM|third|night|| +1954|AAAAAAAADKHAAAAA|1954|0|32|34|AM|third|night|| +1955|AAAAAAAAEKHAAAAA|1955|0|32|35|AM|third|night|| +1956|AAAAAAAAFKHAAAAA|1956|0|32|36|AM|third|night|| +1957|AAAAAAAAGKHAAAAA|1957|0|32|37|AM|third|night|| +1958|AAAAAAAAHKHAAAAA|1958|0|32|38|AM|third|night|| +1959|AAAAAAAAIKHAAAAA|1959|0|32|39|AM|third|night|| +1960|AAAAAAAAJKHAAAAA|1960|0|32|40|AM|third|night|| +1961|AAAAAAAAKKHAAAAA|1961|0|32|41|AM|third|night|| +1962|AAAAAAAALKHAAAAA|1962|0|32|42|AM|third|night|| +1963|AAAAAAAAMKHAAAAA|1963|0|32|43|AM|third|night|| +1964|AAAAAAAANKHAAAAA|1964|0|32|44|AM|third|night|| +1965|AAAAAAAAOKHAAAAA|1965|0|32|45|AM|third|night|| +1966|AAAAAAAAPKHAAAAA|1966|0|32|46|AM|third|night|| +1967|AAAAAAAAALHAAAAA|1967|0|32|47|AM|third|night|| +1968|AAAAAAAABLHAAAAA|1968|0|32|48|AM|third|night|| +1969|AAAAAAAACLHAAAAA|1969|0|32|49|AM|third|night|| +1970|AAAAAAAADLHAAAAA|1970|0|32|50|AM|third|night|| +1971|AAAAAAAAELHAAAAA|1971|0|32|51|AM|third|night|| +1972|AAAAAAAAFLHAAAAA|1972|0|32|52|AM|third|night|| +1973|AAAAAAAAGLHAAAAA|1973|0|32|53|AM|third|night|| +1974|AAAAAAAAHLHAAAAA|1974|0|32|54|AM|third|night|| +1975|AAAAAAAAILHAAAAA|1975|0|32|55|AM|third|night|| +1976|AAAAAAAAJLHAAAAA|1976|0|32|56|AM|third|night|| +1977|AAAAAAAAKLHAAAAA|1977|0|32|57|AM|third|night|| +1978|AAAAAAAALLHAAAAA|1978|0|32|58|AM|third|night|| +1979|AAAAAAAAMLHAAAAA|1979|0|32|59|AM|third|night|| +1980|AAAAAAAANLHAAAAA|1980|0|33|0|AM|third|night|| +1981|AAAAAAAAOLHAAAAA|1981|0|33|1|AM|third|night|| +1982|AAAAAAAAPLHAAAAA|1982|0|33|2|AM|third|night|| +1983|AAAAAAAAAMHAAAAA|1983|0|33|3|AM|third|night|| +1984|AAAAAAAABMHAAAAA|1984|0|33|4|AM|third|night|| +1985|AAAAAAAACMHAAAAA|1985|0|33|5|AM|third|night|| +1986|AAAAAAAADMHAAAAA|1986|0|33|6|AM|third|night|| +1987|AAAAAAAAEMHAAAAA|1987|0|33|7|AM|third|night|| +1988|AAAAAAAAFMHAAAAA|1988|0|33|8|AM|third|night|| +1989|AAAAAAAAGMHAAAAA|1989|0|33|9|AM|third|night|| +1990|AAAAAAAAHMHAAAAA|1990|0|33|10|AM|third|night|| +1991|AAAAAAAAIMHAAAAA|1991|0|33|11|AM|third|night|| +1992|AAAAAAAAJMHAAAAA|1992|0|33|12|AM|third|night|| +1993|AAAAAAAAKMHAAAAA|1993|0|33|13|AM|third|night|| +1994|AAAAAAAALMHAAAAA|1994|0|33|14|AM|third|night|| +1995|AAAAAAAAMMHAAAAA|1995|0|33|15|AM|third|night|| +1996|AAAAAAAANMHAAAAA|1996|0|33|16|AM|third|night|| +1997|AAAAAAAAOMHAAAAA|1997|0|33|17|AM|third|night|| +1998|AAAAAAAAPMHAAAAA|1998|0|33|18|AM|third|night|| +1999|AAAAAAAAANHAAAAA|1999|0|33|19|AM|third|night|| +2000|AAAAAAAABNHAAAAA|2000|0|33|20|AM|third|night|| +2001|AAAAAAAACNHAAAAA|2001|0|33|21|AM|third|night|| +2002|AAAAAAAADNHAAAAA|2002|0|33|22|AM|third|night|| +2003|AAAAAAAAENHAAAAA|2003|0|33|23|AM|third|night|| +2004|AAAAAAAAFNHAAAAA|2004|0|33|24|AM|third|night|| +2005|AAAAAAAAGNHAAAAA|2005|0|33|25|AM|third|night|| +2006|AAAAAAAAHNHAAAAA|2006|0|33|26|AM|third|night|| +2007|AAAAAAAAINHAAAAA|2007|0|33|27|AM|third|night|| +2008|AAAAAAAAJNHAAAAA|2008|0|33|28|AM|third|night|| +2009|AAAAAAAAKNHAAAAA|2009|0|33|29|AM|third|night|| +2010|AAAAAAAALNHAAAAA|2010|0|33|30|AM|third|night|| +2011|AAAAAAAAMNHAAAAA|2011|0|33|31|AM|third|night|| +2012|AAAAAAAANNHAAAAA|2012|0|33|32|AM|third|night|| +2013|AAAAAAAAONHAAAAA|2013|0|33|33|AM|third|night|| +2014|AAAAAAAAPNHAAAAA|2014|0|33|34|AM|third|night|| +2015|AAAAAAAAAOHAAAAA|2015|0|33|35|AM|third|night|| +2016|AAAAAAAABOHAAAAA|2016|0|33|36|AM|third|night|| +2017|AAAAAAAACOHAAAAA|2017|0|33|37|AM|third|night|| +2018|AAAAAAAADOHAAAAA|2018|0|33|38|AM|third|night|| +2019|AAAAAAAAEOHAAAAA|2019|0|33|39|AM|third|night|| +2020|AAAAAAAAFOHAAAAA|2020|0|33|40|AM|third|night|| +2021|AAAAAAAAGOHAAAAA|2021|0|33|41|AM|third|night|| +2022|AAAAAAAAHOHAAAAA|2022|0|33|42|AM|third|night|| +2023|AAAAAAAAIOHAAAAA|2023|0|33|43|AM|third|night|| +2024|AAAAAAAAJOHAAAAA|2024|0|33|44|AM|third|night|| +2025|AAAAAAAAKOHAAAAA|2025|0|33|45|AM|third|night|| +2026|AAAAAAAALOHAAAAA|2026|0|33|46|AM|third|night|| +2027|AAAAAAAAMOHAAAAA|2027|0|33|47|AM|third|night|| +2028|AAAAAAAANOHAAAAA|2028|0|33|48|AM|third|night|| +2029|AAAAAAAAOOHAAAAA|2029|0|33|49|AM|third|night|| +2030|AAAAAAAAPOHAAAAA|2030|0|33|50|AM|third|night|| +2031|AAAAAAAAAPHAAAAA|2031|0|33|51|AM|third|night|| +2032|AAAAAAAABPHAAAAA|2032|0|33|52|AM|third|night|| +2033|AAAAAAAACPHAAAAA|2033|0|33|53|AM|third|night|| +2034|AAAAAAAADPHAAAAA|2034|0|33|54|AM|third|night|| +2035|AAAAAAAAEPHAAAAA|2035|0|33|55|AM|third|night|| +2036|AAAAAAAAFPHAAAAA|2036|0|33|56|AM|third|night|| +2037|AAAAAAAAGPHAAAAA|2037|0|33|57|AM|third|night|| +2038|AAAAAAAAHPHAAAAA|2038|0|33|58|AM|third|night|| +2039|AAAAAAAAIPHAAAAA|2039|0|33|59|AM|third|night|| +2040|AAAAAAAAJPHAAAAA|2040|0|34|0|AM|third|night|| +2041|AAAAAAAAKPHAAAAA|2041|0|34|1|AM|third|night|| +2042|AAAAAAAALPHAAAAA|2042|0|34|2|AM|third|night|| +2043|AAAAAAAAMPHAAAAA|2043|0|34|3|AM|third|night|| +2044|AAAAAAAANPHAAAAA|2044|0|34|4|AM|third|night|| +2045|AAAAAAAAOPHAAAAA|2045|0|34|5|AM|third|night|| +2046|AAAAAAAAPPHAAAAA|2046|0|34|6|AM|third|night|| +2047|AAAAAAAAAAIAAAAA|2047|0|34|7|AM|third|night|| +2048|AAAAAAAABAIAAAAA|2048|0|34|8|AM|third|night|| +2049|AAAAAAAACAIAAAAA|2049|0|34|9|AM|third|night|| +2050|AAAAAAAADAIAAAAA|2050|0|34|10|AM|third|night|| +2051|AAAAAAAAEAIAAAAA|2051|0|34|11|AM|third|night|| +2052|AAAAAAAAFAIAAAAA|2052|0|34|12|AM|third|night|| +2053|AAAAAAAAGAIAAAAA|2053|0|34|13|AM|third|night|| +2054|AAAAAAAAHAIAAAAA|2054|0|34|14|AM|third|night|| +2055|AAAAAAAAIAIAAAAA|2055|0|34|15|AM|third|night|| +2056|AAAAAAAAJAIAAAAA|2056|0|34|16|AM|third|night|| +2057|AAAAAAAAKAIAAAAA|2057|0|34|17|AM|third|night|| +2058|AAAAAAAALAIAAAAA|2058|0|34|18|AM|third|night|| +2059|AAAAAAAAMAIAAAAA|2059|0|34|19|AM|third|night|| +2060|AAAAAAAANAIAAAAA|2060|0|34|20|AM|third|night|| +2061|AAAAAAAAOAIAAAAA|2061|0|34|21|AM|third|night|| +2062|AAAAAAAAPAIAAAAA|2062|0|34|22|AM|third|night|| +2063|AAAAAAAAABIAAAAA|2063|0|34|23|AM|third|night|| +2064|AAAAAAAABBIAAAAA|2064|0|34|24|AM|third|night|| +2065|AAAAAAAACBIAAAAA|2065|0|34|25|AM|third|night|| +2066|AAAAAAAADBIAAAAA|2066|0|34|26|AM|third|night|| +2067|AAAAAAAAEBIAAAAA|2067|0|34|27|AM|third|night|| +2068|AAAAAAAAFBIAAAAA|2068|0|34|28|AM|third|night|| +2069|AAAAAAAAGBIAAAAA|2069|0|34|29|AM|third|night|| +2070|AAAAAAAAHBIAAAAA|2070|0|34|30|AM|third|night|| +2071|AAAAAAAAIBIAAAAA|2071|0|34|31|AM|third|night|| +2072|AAAAAAAAJBIAAAAA|2072|0|34|32|AM|third|night|| +2073|AAAAAAAAKBIAAAAA|2073|0|34|33|AM|third|night|| +2074|AAAAAAAALBIAAAAA|2074|0|34|34|AM|third|night|| +2075|AAAAAAAAMBIAAAAA|2075|0|34|35|AM|third|night|| +2076|AAAAAAAANBIAAAAA|2076|0|34|36|AM|third|night|| +2077|AAAAAAAAOBIAAAAA|2077|0|34|37|AM|third|night|| +2078|AAAAAAAAPBIAAAAA|2078|0|34|38|AM|third|night|| +2079|AAAAAAAAACIAAAAA|2079|0|34|39|AM|third|night|| +2080|AAAAAAAABCIAAAAA|2080|0|34|40|AM|third|night|| +2081|AAAAAAAACCIAAAAA|2081|0|34|41|AM|third|night|| +2082|AAAAAAAADCIAAAAA|2082|0|34|42|AM|third|night|| +2083|AAAAAAAAECIAAAAA|2083|0|34|43|AM|third|night|| +2084|AAAAAAAAFCIAAAAA|2084|0|34|44|AM|third|night|| +2085|AAAAAAAAGCIAAAAA|2085|0|34|45|AM|third|night|| +2086|AAAAAAAAHCIAAAAA|2086|0|34|46|AM|third|night|| +2087|AAAAAAAAICIAAAAA|2087|0|34|47|AM|third|night|| +2088|AAAAAAAAJCIAAAAA|2088|0|34|48|AM|third|night|| +2089|AAAAAAAAKCIAAAAA|2089|0|34|49|AM|third|night|| +2090|AAAAAAAALCIAAAAA|2090|0|34|50|AM|third|night|| +2091|AAAAAAAAMCIAAAAA|2091|0|34|51|AM|third|night|| +2092|AAAAAAAANCIAAAAA|2092|0|34|52|AM|third|night|| +2093|AAAAAAAAOCIAAAAA|2093|0|34|53|AM|third|night|| +2094|AAAAAAAAPCIAAAAA|2094|0|34|54|AM|third|night|| +2095|AAAAAAAAADIAAAAA|2095|0|34|55|AM|third|night|| +2096|AAAAAAAABDIAAAAA|2096|0|34|56|AM|third|night|| +2097|AAAAAAAACDIAAAAA|2097|0|34|57|AM|third|night|| +2098|AAAAAAAADDIAAAAA|2098|0|34|58|AM|third|night|| +2099|AAAAAAAAEDIAAAAA|2099|0|34|59|AM|third|night|| +2100|AAAAAAAAFDIAAAAA|2100|0|35|0|AM|third|night|| +2101|AAAAAAAAGDIAAAAA|2101|0|35|1|AM|third|night|| +2102|AAAAAAAAHDIAAAAA|2102|0|35|2|AM|third|night|| +2103|AAAAAAAAIDIAAAAA|2103|0|35|3|AM|third|night|| +2104|AAAAAAAAJDIAAAAA|2104|0|35|4|AM|third|night|| +2105|AAAAAAAAKDIAAAAA|2105|0|35|5|AM|third|night|| +2106|AAAAAAAALDIAAAAA|2106|0|35|6|AM|third|night|| +2107|AAAAAAAAMDIAAAAA|2107|0|35|7|AM|third|night|| +2108|AAAAAAAANDIAAAAA|2108|0|35|8|AM|third|night|| +2109|AAAAAAAAODIAAAAA|2109|0|35|9|AM|third|night|| +2110|AAAAAAAAPDIAAAAA|2110|0|35|10|AM|third|night|| +2111|AAAAAAAAAEIAAAAA|2111|0|35|11|AM|third|night|| +2112|AAAAAAAABEIAAAAA|2112|0|35|12|AM|third|night|| +2113|AAAAAAAACEIAAAAA|2113|0|35|13|AM|third|night|| +2114|AAAAAAAADEIAAAAA|2114|0|35|14|AM|third|night|| +2115|AAAAAAAAEEIAAAAA|2115|0|35|15|AM|third|night|| +2116|AAAAAAAAFEIAAAAA|2116|0|35|16|AM|third|night|| +2117|AAAAAAAAGEIAAAAA|2117|0|35|17|AM|third|night|| +2118|AAAAAAAAHEIAAAAA|2118|0|35|18|AM|third|night|| +2119|AAAAAAAAIEIAAAAA|2119|0|35|19|AM|third|night|| +2120|AAAAAAAAJEIAAAAA|2120|0|35|20|AM|third|night|| +2121|AAAAAAAAKEIAAAAA|2121|0|35|21|AM|third|night|| +2122|AAAAAAAALEIAAAAA|2122|0|35|22|AM|third|night|| +2123|AAAAAAAAMEIAAAAA|2123|0|35|23|AM|third|night|| +2124|AAAAAAAANEIAAAAA|2124|0|35|24|AM|third|night|| +2125|AAAAAAAAOEIAAAAA|2125|0|35|25|AM|third|night|| +2126|AAAAAAAAPEIAAAAA|2126|0|35|26|AM|third|night|| +2127|AAAAAAAAAFIAAAAA|2127|0|35|27|AM|third|night|| +2128|AAAAAAAABFIAAAAA|2128|0|35|28|AM|third|night|| +2129|AAAAAAAACFIAAAAA|2129|0|35|29|AM|third|night|| +2130|AAAAAAAADFIAAAAA|2130|0|35|30|AM|third|night|| +2131|AAAAAAAAEFIAAAAA|2131|0|35|31|AM|third|night|| +2132|AAAAAAAAFFIAAAAA|2132|0|35|32|AM|third|night|| +2133|AAAAAAAAGFIAAAAA|2133|0|35|33|AM|third|night|| +2134|AAAAAAAAHFIAAAAA|2134|0|35|34|AM|third|night|| +2135|AAAAAAAAIFIAAAAA|2135|0|35|35|AM|third|night|| +2136|AAAAAAAAJFIAAAAA|2136|0|35|36|AM|third|night|| +2137|AAAAAAAAKFIAAAAA|2137|0|35|37|AM|third|night|| +2138|AAAAAAAALFIAAAAA|2138|0|35|38|AM|third|night|| +2139|AAAAAAAAMFIAAAAA|2139|0|35|39|AM|third|night|| +2140|AAAAAAAANFIAAAAA|2140|0|35|40|AM|third|night|| +2141|AAAAAAAAOFIAAAAA|2141|0|35|41|AM|third|night|| +2142|AAAAAAAAPFIAAAAA|2142|0|35|42|AM|third|night|| +2143|AAAAAAAAAGIAAAAA|2143|0|35|43|AM|third|night|| +2144|AAAAAAAABGIAAAAA|2144|0|35|44|AM|third|night|| +2145|AAAAAAAACGIAAAAA|2145|0|35|45|AM|third|night|| +2146|AAAAAAAADGIAAAAA|2146|0|35|46|AM|third|night|| +2147|AAAAAAAAEGIAAAAA|2147|0|35|47|AM|third|night|| +2148|AAAAAAAAFGIAAAAA|2148|0|35|48|AM|third|night|| +2149|AAAAAAAAGGIAAAAA|2149|0|35|49|AM|third|night|| +2150|AAAAAAAAHGIAAAAA|2150|0|35|50|AM|third|night|| +2151|AAAAAAAAIGIAAAAA|2151|0|35|51|AM|third|night|| +2152|AAAAAAAAJGIAAAAA|2152|0|35|52|AM|third|night|| +2153|AAAAAAAAKGIAAAAA|2153|0|35|53|AM|third|night|| +2154|AAAAAAAALGIAAAAA|2154|0|35|54|AM|third|night|| +2155|AAAAAAAAMGIAAAAA|2155|0|35|55|AM|third|night|| +2156|AAAAAAAANGIAAAAA|2156|0|35|56|AM|third|night|| +2157|AAAAAAAAOGIAAAAA|2157|0|35|57|AM|third|night|| +2158|AAAAAAAAPGIAAAAA|2158|0|35|58|AM|third|night|| +2159|AAAAAAAAAHIAAAAA|2159|0|35|59|AM|third|night|| +2160|AAAAAAAABHIAAAAA|2160|0|36|0|AM|third|night|| +2161|AAAAAAAACHIAAAAA|2161|0|36|1|AM|third|night|| +2162|AAAAAAAADHIAAAAA|2162|0|36|2|AM|third|night|| +2163|AAAAAAAAEHIAAAAA|2163|0|36|3|AM|third|night|| +2164|AAAAAAAAFHIAAAAA|2164|0|36|4|AM|third|night|| +2165|AAAAAAAAGHIAAAAA|2165|0|36|5|AM|third|night|| +2166|AAAAAAAAHHIAAAAA|2166|0|36|6|AM|third|night|| +2167|AAAAAAAAIHIAAAAA|2167|0|36|7|AM|third|night|| +2168|AAAAAAAAJHIAAAAA|2168|0|36|8|AM|third|night|| +2169|AAAAAAAAKHIAAAAA|2169|0|36|9|AM|third|night|| +2170|AAAAAAAALHIAAAAA|2170|0|36|10|AM|third|night|| +2171|AAAAAAAAMHIAAAAA|2171|0|36|11|AM|third|night|| +2172|AAAAAAAANHIAAAAA|2172|0|36|12|AM|third|night|| +2173|AAAAAAAAOHIAAAAA|2173|0|36|13|AM|third|night|| +2174|AAAAAAAAPHIAAAAA|2174|0|36|14|AM|third|night|| +2175|AAAAAAAAAIIAAAAA|2175|0|36|15|AM|third|night|| +2176|AAAAAAAABIIAAAAA|2176|0|36|16|AM|third|night|| +2177|AAAAAAAACIIAAAAA|2177|0|36|17|AM|third|night|| +2178|AAAAAAAADIIAAAAA|2178|0|36|18|AM|third|night|| +2179|AAAAAAAAEIIAAAAA|2179|0|36|19|AM|third|night|| +2180|AAAAAAAAFIIAAAAA|2180|0|36|20|AM|third|night|| +2181|AAAAAAAAGIIAAAAA|2181|0|36|21|AM|third|night|| +2182|AAAAAAAAHIIAAAAA|2182|0|36|22|AM|third|night|| +2183|AAAAAAAAIIIAAAAA|2183|0|36|23|AM|third|night|| +2184|AAAAAAAAJIIAAAAA|2184|0|36|24|AM|third|night|| +2185|AAAAAAAAKIIAAAAA|2185|0|36|25|AM|third|night|| +2186|AAAAAAAALIIAAAAA|2186|0|36|26|AM|third|night|| +2187|AAAAAAAAMIIAAAAA|2187|0|36|27|AM|third|night|| +2188|AAAAAAAANIIAAAAA|2188|0|36|28|AM|third|night|| +2189|AAAAAAAAOIIAAAAA|2189|0|36|29|AM|third|night|| +2190|AAAAAAAAPIIAAAAA|2190|0|36|30|AM|third|night|| +2191|AAAAAAAAAJIAAAAA|2191|0|36|31|AM|third|night|| +2192|AAAAAAAABJIAAAAA|2192|0|36|32|AM|third|night|| +2193|AAAAAAAACJIAAAAA|2193|0|36|33|AM|third|night|| +2194|AAAAAAAADJIAAAAA|2194|0|36|34|AM|third|night|| +2195|AAAAAAAAEJIAAAAA|2195|0|36|35|AM|third|night|| +2196|AAAAAAAAFJIAAAAA|2196|0|36|36|AM|third|night|| +2197|AAAAAAAAGJIAAAAA|2197|0|36|37|AM|third|night|| +2198|AAAAAAAAHJIAAAAA|2198|0|36|38|AM|third|night|| +2199|AAAAAAAAIJIAAAAA|2199|0|36|39|AM|third|night|| +2200|AAAAAAAAJJIAAAAA|2200|0|36|40|AM|third|night|| +2201|AAAAAAAAKJIAAAAA|2201|0|36|41|AM|third|night|| +2202|AAAAAAAALJIAAAAA|2202|0|36|42|AM|third|night|| +2203|AAAAAAAAMJIAAAAA|2203|0|36|43|AM|third|night|| +2204|AAAAAAAANJIAAAAA|2204|0|36|44|AM|third|night|| +2205|AAAAAAAAOJIAAAAA|2205|0|36|45|AM|third|night|| +2206|AAAAAAAAPJIAAAAA|2206|0|36|46|AM|third|night|| +2207|AAAAAAAAAKIAAAAA|2207|0|36|47|AM|third|night|| +2208|AAAAAAAABKIAAAAA|2208|0|36|48|AM|third|night|| +2209|AAAAAAAACKIAAAAA|2209|0|36|49|AM|third|night|| +2210|AAAAAAAADKIAAAAA|2210|0|36|50|AM|third|night|| +2211|AAAAAAAAEKIAAAAA|2211|0|36|51|AM|third|night|| +2212|AAAAAAAAFKIAAAAA|2212|0|36|52|AM|third|night|| +2213|AAAAAAAAGKIAAAAA|2213|0|36|53|AM|third|night|| +2214|AAAAAAAAHKIAAAAA|2214|0|36|54|AM|third|night|| +2215|AAAAAAAAIKIAAAAA|2215|0|36|55|AM|third|night|| +2216|AAAAAAAAJKIAAAAA|2216|0|36|56|AM|third|night|| +2217|AAAAAAAAKKIAAAAA|2217|0|36|57|AM|third|night|| +2218|AAAAAAAALKIAAAAA|2218|0|36|58|AM|third|night|| +2219|AAAAAAAAMKIAAAAA|2219|0|36|59|AM|third|night|| +2220|AAAAAAAANKIAAAAA|2220|0|37|0|AM|third|night|| +2221|AAAAAAAAOKIAAAAA|2221|0|37|1|AM|third|night|| +2222|AAAAAAAAPKIAAAAA|2222|0|37|2|AM|third|night|| +2223|AAAAAAAAALIAAAAA|2223|0|37|3|AM|third|night|| +2224|AAAAAAAABLIAAAAA|2224|0|37|4|AM|third|night|| +2225|AAAAAAAACLIAAAAA|2225|0|37|5|AM|third|night|| +2226|AAAAAAAADLIAAAAA|2226|0|37|6|AM|third|night|| +2227|AAAAAAAAELIAAAAA|2227|0|37|7|AM|third|night|| +2228|AAAAAAAAFLIAAAAA|2228|0|37|8|AM|third|night|| +2229|AAAAAAAAGLIAAAAA|2229|0|37|9|AM|third|night|| +2230|AAAAAAAAHLIAAAAA|2230|0|37|10|AM|third|night|| +2231|AAAAAAAAILIAAAAA|2231|0|37|11|AM|third|night|| +2232|AAAAAAAAJLIAAAAA|2232|0|37|12|AM|third|night|| +2233|AAAAAAAAKLIAAAAA|2233|0|37|13|AM|third|night|| +2234|AAAAAAAALLIAAAAA|2234|0|37|14|AM|third|night|| +2235|AAAAAAAAMLIAAAAA|2235|0|37|15|AM|third|night|| +2236|AAAAAAAANLIAAAAA|2236|0|37|16|AM|third|night|| +2237|AAAAAAAAOLIAAAAA|2237|0|37|17|AM|third|night|| +2238|AAAAAAAAPLIAAAAA|2238|0|37|18|AM|third|night|| +2239|AAAAAAAAAMIAAAAA|2239|0|37|19|AM|third|night|| +2240|AAAAAAAABMIAAAAA|2240|0|37|20|AM|third|night|| +2241|AAAAAAAACMIAAAAA|2241|0|37|21|AM|third|night|| +2242|AAAAAAAADMIAAAAA|2242|0|37|22|AM|third|night|| +2243|AAAAAAAAEMIAAAAA|2243|0|37|23|AM|third|night|| +2244|AAAAAAAAFMIAAAAA|2244|0|37|24|AM|third|night|| +2245|AAAAAAAAGMIAAAAA|2245|0|37|25|AM|third|night|| +2246|AAAAAAAAHMIAAAAA|2246|0|37|26|AM|third|night|| +2247|AAAAAAAAIMIAAAAA|2247|0|37|27|AM|third|night|| +2248|AAAAAAAAJMIAAAAA|2248|0|37|28|AM|third|night|| +2249|AAAAAAAAKMIAAAAA|2249|0|37|29|AM|third|night|| +2250|AAAAAAAALMIAAAAA|2250|0|37|30|AM|third|night|| +2251|AAAAAAAAMMIAAAAA|2251|0|37|31|AM|third|night|| +2252|AAAAAAAANMIAAAAA|2252|0|37|32|AM|third|night|| +2253|AAAAAAAAOMIAAAAA|2253|0|37|33|AM|third|night|| +2254|AAAAAAAAPMIAAAAA|2254|0|37|34|AM|third|night|| +2255|AAAAAAAAANIAAAAA|2255|0|37|35|AM|third|night|| +2256|AAAAAAAABNIAAAAA|2256|0|37|36|AM|third|night|| +2257|AAAAAAAACNIAAAAA|2257|0|37|37|AM|third|night|| +2258|AAAAAAAADNIAAAAA|2258|0|37|38|AM|third|night|| +2259|AAAAAAAAENIAAAAA|2259|0|37|39|AM|third|night|| +2260|AAAAAAAAFNIAAAAA|2260|0|37|40|AM|third|night|| +2261|AAAAAAAAGNIAAAAA|2261|0|37|41|AM|third|night|| +2262|AAAAAAAAHNIAAAAA|2262|0|37|42|AM|third|night|| +2263|AAAAAAAAINIAAAAA|2263|0|37|43|AM|third|night|| +2264|AAAAAAAAJNIAAAAA|2264|0|37|44|AM|third|night|| +2265|AAAAAAAAKNIAAAAA|2265|0|37|45|AM|third|night|| +2266|AAAAAAAALNIAAAAA|2266|0|37|46|AM|third|night|| +2267|AAAAAAAAMNIAAAAA|2267|0|37|47|AM|third|night|| +2268|AAAAAAAANNIAAAAA|2268|0|37|48|AM|third|night|| +2269|AAAAAAAAONIAAAAA|2269|0|37|49|AM|third|night|| +2270|AAAAAAAAPNIAAAAA|2270|0|37|50|AM|third|night|| +2271|AAAAAAAAAOIAAAAA|2271|0|37|51|AM|third|night|| +2272|AAAAAAAABOIAAAAA|2272|0|37|52|AM|third|night|| +2273|AAAAAAAACOIAAAAA|2273|0|37|53|AM|third|night|| +2274|AAAAAAAADOIAAAAA|2274|0|37|54|AM|third|night|| +2275|AAAAAAAAEOIAAAAA|2275|0|37|55|AM|third|night|| +2276|AAAAAAAAFOIAAAAA|2276|0|37|56|AM|third|night|| +2277|AAAAAAAAGOIAAAAA|2277|0|37|57|AM|third|night|| +2278|AAAAAAAAHOIAAAAA|2278|0|37|58|AM|third|night|| +2279|AAAAAAAAIOIAAAAA|2279|0|37|59|AM|third|night|| +2280|AAAAAAAAJOIAAAAA|2280|0|38|0|AM|third|night|| +2281|AAAAAAAAKOIAAAAA|2281|0|38|1|AM|third|night|| +2282|AAAAAAAALOIAAAAA|2282|0|38|2|AM|third|night|| +2283|AAAAAAAAMOIAAAAA|2283|0|38|3|AM|third|night|| +2284|AAAAAAAANOIAAAAA|2284|0|38|4|AM|third|night|| +2285|AAAAAAAAOOIAAAAA|2285|0|38|5|AM|third|night|| +2286|AAAAAAAAPOIAAAAA|2286|0|38|6|AM|third|night|| +2287|AAAAAAAAAPIAAAAA|2287|0|38|7|AM|third|night|| +2288|AAAAAAAABPIAAAAA|2288|0|38|8|AM|third|night|| +2289|AAAAAAAACPIAAAAA|2289|0|38|9|AM|third|night|| +2290|AAAAAAAADPIAAAAA|2290|0|38|10|AM|third|night|| +2291|AAAAAAAAEPIAAAAA|2291|0|38|11|AM|third|night|| +2292|AAAAAAAAFPIAAAAA|2292|0|38|12|AM|third|night|| +2293|AAAAAAAAGPIAAAAA|2293|0|38|13|AM|third|night|| +2294|AAAAAAAAHPIAAAAA|2294|0|38|14|AM|third|night|| +2295|AAAAAAAAIPIAAAAA|2295|0|38|15|AM|third|night|| +2296|AAAAAAAAJPIAAAAA|2296|0|38|16|AM|third|night|| +2297|AAAAAAAAKPIAAAAA|2297|0|38|17|AM|third|night|| +2298|AAAAAAAALPIAAAAA|2298|0|38|18|AM|third|night|| +2299|AAAAAAAAMPIAAAAA|2299|0|38|19|AM|third|night|| +2300|AAAAAAAANPIAAAAA|2300|0|38|20|AM|third|night|| +2301|AAAAAAAAOPIAAAAA|2301|0|38|21|AM|third|night|| +2302|AAAAAAAAPPIAAAAA|2302|0|38|22|AM|third|night|| +2303|AAAAAAAAAAJAAAAA|2303|0|38|23|AM|third|night|| +2304|AAAAAAAABAJAAAAA|2304|0|38|24|AM|third|night|| +2305|AAAAAAAACAJAAAAA|2305|0|38|25|AM|third|night|| +2306|AAAAAAAADAJAAAAA|2306|0|38|26|AM|third|night|| +2307|AAAAAAAAEAJAAAAA|2307|0|38|27|AM|third|night|| +2308|AAAAAAAAFAJAAAAA|2308|0|38|28|AM|third|night|| +2309|AAAAAAAAGAJAAAAA|2309|0|38|29|AM|third|night|| +2310|AAAAAAAAHAJAAAAA|2310|0|38|30|AM|third|night|| +2311|AAAAAAAAIAJAAAAA|2311|0|38|31|AM|third|night|| +2312|AAAAAAAAJAJAAAAA|2312|0|38|32|AM|third|night|| +2313|AAAAAAAAKAJAAAAA|2313|0|38|33|AM|third|night|| +2314|AAAAAAAALAJAAAAA|2314|0|38|34|AM|third|night|| +2315|AAAAAAAAMAJAAAAA|2315|0|38|35|AM|third|night|| +2316|AAAAAAAANAJAAAAA|2316|0|38|36|AM|third|night|| +2317|AAAAAAAAOAJAAAAA|2317|0|38|37|AM|third|night|| +2318|AAAAAAAAPAJAAAAA|2318|0|38|38|AM|third|night|| +2319|AAAAAAAAABJAAAAA|2319|0|38|39|AM|third|night|| +2320|AAAAAAAABBJAAAAA|2320|0|38|40|AM|third|night|| +2321|AAAAAAAACBJAAAAA|2321|0|38|41|AM|third|night|| +2322|AAAAAAAADBJAAAAA|2322|0|38|42|AM|third|night|| +2323|AAAAAAAAEBJAAAAA|2323|0|38|43|AM|third|night|| +2324|AAAAAAAAFBJAAAAA|2324|0|38|44|AM|third|night|| +2325|AAAAAAAAGBJAAAAA|2325|0|38|45|AM|third|night|| +2326|AAAAAAAAHBJAAAAA|2326|0|38|46|AM|third|night|| +2327|AAAAAAAAIBJAAAAA|2327|0|38|47|AM|third|night|| +2328|AAAAAAAAJBJAAAAA|2328|0|38|48|AM|third|night|| +2329|AAAAAAAAKBJAAAAA|2329|0|38|49|AM|third|night|| +2330|AAAAAAAALBJAAAAA|2330|0|38|50|AM|third|night|| +2331|AAAAAAAAMBJAAAAA|2331|0|38|51|AM|third|night|| +2332|AAAAAAAANBJAAAAA|2332|0|38|52|AM|third|night|| +2333|AAAAAAAAOBJAAAAA|2333|0|38|53|AM|third|night|| +2334|AAAAAAAAPBJAAAAA|2334|0|38|54|AM|third|night|| +2335|AAAAAAAAACJAAAAA|2335|0|38|55|AM|third|night|| +2336|AAAAAAAABCJAAAAA|2336|0|38|56|AM|third|night|| +2337|AAAAAAAACCJAAAAA|2337|0|38|57|AM|third|night|| +2338|AAAAAAAADCJAAAAA|2338|0|38|58|AM|third|night|| +2339|AAAAAAAAECJAAAAA|2339|0|38|59|AM|third|night|| +2340|AAAAAAAAFCJAAAAA|2340|0|39|0|AM|third|night|| +2341|AAAAAAAAGCJAAAAA|2341|0|39|1|AM|third|night|| +2342|AAAAAAAAHCJAAAAA|2342|0|39|2|AM|third|night|| +2343|AAAAAAAAICJAAAAA|2343|0|39|3|AM|third|night|| +2344|AAAAAAAAJCJAAAAA|2344|0|39|4|AM|third|night|| +2345|AAAAAAAAKCJAAAAA|2345|0|39|5|AM|third|night|| +2346|AAAAAAAALCJAAAAA|2346|0|39|6|AM|third|night|| +2347|AAAAAAAAMCJAAAAA|2347|0|39|7|AM|third|night|| +2348|AAAAAAAANCJAAAAA|2348|0|39|8|AM|third|night|| +2349|AAAAAAAAOCJAAAAA|2349|0|39|9|AM|third|night|| +2350|AAAAAAAAPCJAAAAA|2350|0|39|10|AM|third|night|| +2351|AAAAAAAAADJAAAAA|2351|0|39|11|AM|third|night|| +2352|AAAAAAAABDJAAAAA|2352|0|39|12|AM|third|night|| +2353|AAAAAAAACDJAAAAA|2353|0|39|13|AM|third|night|| +2354|AAAAAAAADDJAAAAA|2354|0|39|14|AM|third|night|| +2355|AAAAAAAAEDJAAAAA|2355|0|39|15|AM|third|night|| +2356|AAAAAAAAFDJAAAAA|2356|0|39|16|AM|third|night|| +2357|AAAAAAAAGDJAAAAA|2357|0|39|17|AM|third|night|| +2358|AAAAAAAAHDJAAAAA|2358|0|39|18|AM|third|night|| +2359|AAAAAAAAIDJAAAAA|2359|0|39|19|AM|third|night|| +2360|AAAAAAAAJDJAAAAA|2360|0|39|20|AM|third|night|| +2361|AAAAAAAAKDJAAAAA|2361|0|39|21|AM|third|night|| +2362|AAAAAAAALDJAAAAA|2362|0|39|22|AM|third|night|| +2363|AAAAAAAAMDJAAAAA|2363|0|39|23|AM|third|night|| +2364|AAAAAAAANDJAAAAA|2364|0|39|24|AM|third|night|| +2365|AAAAAAAAODJAAAAA|2365|0|39|25|AM|third|night|| +2366|AAAAAAAAPDJAAAAA|2366|0|39|26|AM|third|night|| +2367|AAAAAAAAAEJAAAAA|2367|0|39|27|AM|third|night|| +2368|AAAAAAAABEJAAAAA|2368|0|39|28|AM|third|night|| +2369|AAAAAAAACEJAAAAA|2369|0|39|29|AM|third|night|| +2370|AAAAAAAADEJAAAAA|2370|0|39|30|AM|third|night|| +2371|AAAAAAAAEEJAAAAA|2371|0|39|31|AM|third|night|| +2372|AAAAAAAAFEJAAAAA|2372|0|39|32|AM|third|night|| +2373|AAAAAAAAGEJAAAAA|2373|0|39|33|AM|third|night|| +2374|AAAAAAAAHEJAAAAA|2374|0|39|34|AM|third|night|| +2375|AAAAAAAAIEJAAAAA|2375|0|39|35|AM|third|night|| +2376|AAAAAAAAJEJAAAAA|2376|0|39|36|AM|third|night|| +2377|AAAAAAAAKEJAAAAA|2377|0|39|37|AM|third|night|| +2378|AAAAAAAALEJAAAAA|2378|0|39|38|AM|third|night|| +2379|AAAAAAAAMEJAAAAA|2379|0|39|39|AM|third|night|| +2380|AAAAAAAANEJAAAAA|2380|0|39|40|AM|third|night|| +2381|AAAAAAAAOEJAAAAA|2381|0|39|41|AM|third|night|| +2382|AAAAAAAAPEJAAAAA|2382|0|39|42|AM|third|night|| +2383|AAAAAAAAAFJAAAAA|2383|0|39|43|AM|third|night|| +2384|AAAAAAAABFJAAAAA|2384|0|39|44|AM|third|night|| +2385|AAAAAAAACFJAAAAA|2385|0|39|45|AM|third|night|| +2386|AAAAAAAADFJAAAAA|2386|0|39|46|AM|third|night|| +2387|AAAAAAAAEFJAAAAA|2387|0|39|47|AM|third|night|| +2388|AAAAAAAAFFJAAAAA|2388|0|39|48|AM|third|night|| +2389|AAAAAAAAGFJAAAAA|2389|0|39|49|AM|third|night|| +2390|AAAAAAAAHFJAAAAA|2390|0|39|50|AM|third|night|| +2391|AAAAAAAAIFJAAAAA|2391|0|39|51|AM|third|night|| +2392|AAAAAAAAJFJAAAAA|2392|0|39|52|AM|third|night|| +2393|AAAAAAAAKFJAAAAA|2393|0|39|53|AM|third|night|| +2394|AAAAAAAALFJAAAAA|2394|0|39|54|AM|third|night|| +2395|AAAAAAAAMFJAAAAA|2395|0|39|55|AM|third|night|| +2396|AAAAAAAANFJAAAAA|2396|0|39|56|AM|third|night|| +2397|AAAAAAAAOFJAAAAA|2397|0|39|57|AM|third|night|| +2398|AAAAAAAAPFJAAAAA|2398|0|39|58|AM|third|night|| +2399|AAAAAAAAAGJAAAAA|2399|0|39|59|AM|third|night|| +2400|AAAAAAAABGJAAAAA|2400|0|40|0|AM|third|night|| +2401|AAAAAAAACGJAAAAA|2401|0|40|1|AM|third|night|| +2402|AAAAAAAADGJAAAAA|2402|0|40|2|AM|third|night|| +2403|AAAAAAAAEGJAAAAA|2403|0|40|3|AM|third|night|| +2404|AAAAAAAAFGJAAAAA|2404|0|40|4|AM|third|night|| +2405|AAAAAAAAGGJAAAAA|2405|0|40|5|AM|third|night|| +2406|AAAAAAAAHGJAAAAA|2406|0|40|6|AM|third|night|| +2407|AAAAAAAAIGJAAAAA|2407|0|40|7|AM|third|night|| +2408|AAAAAAAAJGJAAAAA|2408|0|40|8|AM|third|night|| +2409|AAAAAAAAKGJAAAAA|2409|0|40|9|AM|third|night|| +2410|AAAAAAAALGJAAAAA|2410|0|40|10|AM|third|night|| +2411|AAAAAAAAMGJAAAAA|2411|0|40|11|AM|third|night|| +2412|AAAAAAAANGJAAAAA|2412|0|40|12|AM|third|night|| +2413|AAAAAAAAOGJAAAAA|2413|0|40|13|AM|third|night|| +2414|AAAAAAAAPGJAAAAA|2414|0|40|14|AM|third|night|| +2415|AAAAAAAAAHJAAAAA|2415|0|40|15|AM|third|night|| +2416|AAAAAAAABHJAAAAA|2416|0|40|16|AM|third|night|| +2417|AAAAAAAACHJAAAAA|2417|0|40|17|AM|third|night|| +2418|AAAAAAAADHJAAAAA|2418|0|40|18|AM|third|night|| +2419|AAAAAAAAEHJAAAAA|2419|0|40|19|AM|third|night|| +2420|AAAAAAAAFHJAAAAA|2420|0|40|20|AM|third|night|| +2421|AAAAAAAAGHJAAAAA|2421|0|40|21|AM|third|night|| +2422|AAAAAAAAHHJAAAAA|2422|0|40|22|AM|third|night|| +2423|AAAAAAAAIHJAAAAA|2423|0|40|23|AM|third|night|| +2424|AAAAAAAAJHJAAAAA|2424|0|40|24|AM|third|night|| +2425|AAAAAAAAKHJAAAAA|2425|0|40|25|AM|third|night|| +2426|AAAAAAAALHJAAAAA|2426|0|40|26|AM|third|night|| +2427|AAAAAAAAMHJAAAAA|2427|0|40|27|AM|third|night|| +2428|AAAAAAAANHJAAAAA|2428|0|40|28|AM|third|night|| +2429|AAAAAAAAOHJAAAAA|2429|0|40|29|AM|third|night|| +2430|AAAAAAAAPHJAAAAA|2430|0|40|30|AM|third|night|| +2431|AAAAAAAAAIJAAAAA|2431|0|40|31|AM|third|night|| +2432|AAAAAAAABIJAAAAA|2432|0|40|32|AM|third|night|| +2433|AAAAAAAACIJAAAAA|2433|0|40|33|AM|third|night|| +2434|AAAAAAAADIJAAAAA|2434|0|40|34|AM|third|night|| +2435|AAAAAAAAEIJAAAAA|2435|0|40|35|AM|third|night|| +2436|AAAAAAAAFIJAAAAA|2436|0|40|36|AM|third|night|| +2437|AAAAAAAAGIJAAAAA|2437|0|40|37|AM|third|night|| +2438|AAAAAAAAHIJAAAAA|2438|0|40|38|AM|third|night|| +2439|AAAAAAAAIIJAAAAA|2439|0|40|39|AM|third|night|| +2440|AAAAAAAAJIJAAAAA|2440|0|40|40|AM|third|night|| +2441|AAAAAAAAKIJAAAAA|2441|0|40|41|AM|third|night|| +2442|AAAAAAAALIJAAAAA|2442|0|40|42|AM|third|night|| +2443|AAAAAAAAMIJAAAAA|2443|0|40|43|AM|third|night|| +2444|AAAAAAAANIJAAAAA|2444|0|40|44|AM|third|night|| +2445|AAAAAAAAOIJAAAAA|2445|0|40|45|AM|third|night|| +2446|AAAAAAAAPIJAAAAA|2446|0|40|46|AM|third|night|| +2447|AAAAAAAAAJJAAAAA|2447|0|40|47|AM|third|night|| +2448|AAAAAAAABJJAAAAA|2448|0|40|48|AM|third|night|| +2449|AAAAAAAACJJAAAAA|2449|0|40|49|AM|third|night|| +2450|AAAAAAAADJJAAAAA|2450|0|40|50|AM|third|night|| +2451|AAAAAAAAEJJAAAAA|2451|0|40|51|AM|third|night|| +2452|AAAAAAAAFJJAAAAA|2452|0|40|52|AM|third|night|| +2453|AAAAAAAAGJJAAAAA|2453|0|40|53|AM|third|night|| +2454|AAAAAAAAHJJAAAAA|2454|0|40|54|AM|third|night|| +2455|AAAAAAAAIJJAAAAA|2455|0|40|55|AM|third|night|| +2456|AAAAAAAAJJJAAAAA|2456|0|40|56|AM|third|night|| +2457|AAAAAAAAKJJAAAAA|2457|0|40|57|AM|third|night|| +2458|AAAAAAAALJJAAAAA|2458|0|40|58|AM|third|night|| +2459|AAAAAAAAMJJAAAAA|2459|0|40|59|AM|third|night|| +2460|AAAAAAAANJJAAAAA|2460|0|41|0|AM|third|night|| +2461|AAAAAAAAOJJAAAAA|2461|0|41|1|AM|third|night|| +2462|AAAAAAAAPJJAAAAA|2462|0|41|2|AM|third|night|| +2463|AAAAAAAAAKJAAAAA|2463|0|41|3|AM|third|night|| +2464|AAAAAAAABKJAAAAA|2464|0|41|4|AM|third|night|| +2465|AAAAAAAACKJAAAAA|2465|0|41|5|AM|third|night|| +2466|AAAAAAAADKJAAAAA|2466|0|41|6|AM|third|night|| +2467|AAAAAAAAEKJAAAAA|2467|0|41|7|AM|third|night|| +2468|AAAAAAAAFKJAAAAA|2468|0|41|8|AM|third|night|| +2469|AAAAAAAAGKJAAAAA|2469|0|41|9|AM|third|night|| +2470|AAAAAAAAHKJAAAAA|2470|0|41|10|AM|third|night|| +2471|AAAAAAAAIKJAAAAA|2471|0|41|11|AM|third|night|| +2472|AAAAAAAAJKJAAAAA|2472|0|41|12|AM|third|night|| +2473|AAAAAAAAKKJAAAAA|2473|0|41|13|AM|third|night|| +2474|AAAAAAAALKJAAAAA|2474|0|41|14|AM|third|night|| +2475|AAAAAAAAMKJAAAAA|2475|0|41|15|AM|third|night|| +2476|AAAAAAAANKJAAAAA|2476|0|41|16|AM|third|night|| +2477|AAAAAAAAOKJAAAAA|2477|0|41|17|AM|third|night|| +2478|AAAAAAAAPKJAAAAA|2478|0|41|18|AM|third|night|| +2479|AAAAAAAAALJAAAAA|2479|0|41|19|AM|third|night|| +2480|AAAAAAAABLJAAAAA|2480|0|41|20|AM|third|night|| +2481|AAAAAAAACLJAAAAA|2481|0|41|21|AM|third|night|| +2482|AAAAAAAADLJAAAAA|2482|0|41|22|AM|third|night|| +2483|AAAAAAAAELJAAAAA|2483|0|41|23|AM|third|night|| +2484|AAAAAAAAFLJAAAAA|2484|0|41|24|AM|third|night|| +2485|AAAAAAAAGLJAAAAA|2485|0|41|25|AM|third|night|| +2486|AAAAAAAAHLJAAAAA|2486|0|41|26|AM|third|night|| +2487|AAAAAAAAILJAAAAA|2487|0|41|27|AM|third|night|| +2488|AAAAAAAAJLJAAAAA|2488|0|41|28|AM|third|night|| +2489|AAAAAAAAKLJAAAAA|2489|0|41|29|AM|third|night|| +2490|AAAAAAAALLJAAAAA|2490|0|41|30|AM|third|night|| +2491|AAAAAAAAMLJAAAAA|2491|0|41|31|AM|third|night|| +2492|AAAAAAAANLJAAAAA|2492|0|41|32|AM|third|night|| +2493|AAAAAAAAOLJAAAAA|2493|0|41|33|AM|third|night|| +2494|AAAAAAAAPLJAAAAA|2494|0|41|34|AM|third|night|| +2495|AAAAAAAAAMJAAAAA|2495|0|41|35|AM|third|night|| +2496|AAAAAAAABMJAAAAA|2496|0|41|36|AM|third|night|| +2497|AAAAAAAACMJAAAAA|2497|0|41|37|AM|third|night|| +2498|AAAAAAAADMJAAAAA|2498|0|41|38|AM|third|night|| +2499|AAAAAAAAEMJAAAAA|2499|0|41|39|AM|third|night|| +2500|AAAAAAAAFMJAAAAA|2500|0|41|40|AM|third|night|| +2501|AAAAAAAAGMJAAAAA|2501|0|41|41|AM|third|night|| +2502|AAAAAAAAHMJAAAAA|2502|0|41|42|AM|third|night|| +2503|AAAAAAAAIMJAAAAA|2503|0|41|43|AM|third|night|| +2504|AAAAAAAAJMJAAAAA|2504|0|41|44|AM|third|night|| +2505|AAAAAAAAKMJAAAAA|2505|0|41|45|AM|third|night|| +2506|AAAAAAAALMJAAAAA|2506|0|41|46|AM|third|night|| +2507|AAAAAAAAMMJAAAAA|2507|0|41|47|AM|third|night|| +2508|AAAAAAAANMJAAAAA|2508|0|41|48|AM|third|night|| +2509|AAAAAAAAOMJAAAAA|2509|0|41|49|AM|third|night|| +2510|AAAAAAAAPMJAAAAA|2510|0|41|50|AM|third|night|| +2511|AAAAAAAAANJAAAAA|2511|0|41|51|AM|third|night|| +2512|AAAAAAAABNJAAAAA|2512|0|41|52|AM|third|night|| +2513|AAAAAAAACNJAAAAA|2513|0|41|53|AM|third|night|| +2514|AAAAAAAADNJAAAAA|2514|0|41|54|AM|third|night|| +2515|AAAAAAAAENJAAAAA|2515|0|41|55|AM|third|night|| +2516|AAAAAAAAFNJAAAAA|2516|0|41|56|AM|third|night|| +2517|AAAAAAAAGNJAAAAA|2517|0|41|57|AM|third|night|| +2518|AAAAAAAAHNJAAAAA|2518|0|41|58|AM|third|night|| +2519|AAAAAAAAINJAAAAA|2519|0|41|59|AM|third|night|| +2520|AAAAAAAAJNJAAAAA|2520|0|42|0|AM|third|night|| +2521|AAAAAAAAKNJAAAAA|2521|0|42|1|AM|third|night|| +2522|AAAAAAAALNJAAAAA|2522|0|42|2|AM|third|night|| +2523|AAAAAAAAMNJAAAAA|2523|0|42|3|AM|third|night|| +2524|AAAAAAAANNJAAAAA|2524|0|42|4|AM|third|night|| +2525|AAAAAAAAONJAAAAA|2525|0|42|5|AM|third|night|| +2526|AAAAAAAAPNJAAAAA|2526|0|42|6|AM|third|night|| +2527|AAAAAAAAAOJAAAAA|2527|0|42|7|AM|third|night|| +2528|AAAAAAAABOJAAAAA|2528|0|42|8|AM|third|night|| +2529|AAAAAAAACOJAAAAA|2529|0|42|9|AM|third|night|| +2530|AAAAAAAADOJAAAAA|2530|0|42|10|AM|third|night|| +2531|AAAAAAAAEOJAAAAA|2531|0|42|11|AM|third|night|| +2532|AAAAAAAAFOJAAAAA|2532|0|42|12|AM|third|night|| +2533|AAAAAAAAGOJAAAAA|2533|0|42|13|AM|third|night|| +2534|AAAAAAAAHOJAAAAA|2534|0|42|14|AM|third|night|| +2535|AAAAAAAAIOJAAAAA|2535|0|42|15|AM|third|night|| +2536|AAAAAAAAJOJAAAAA|2536|0|42|16|AM|third|night|| +2537|AAAAAAAAKOJAAAAA|2537|0|42|17|AM|third|night|| +2538|AAAAAAAALOJAAAAA|2538|0|42|18|AM|third|night|| +2539|AAAAAAAAMOJAAAAA|2539|0|42|19|AM|third|night|| +2540|AAAAAAAANOJAAAAA|2540|0|42|20|AM|third|night|| +2541|AAAAAAAAOOJAAAAA|2541|0|42|21|AM|third|night|| +2542|AAAAAAAAPOJAAAAA|2542|0|42|22|AM|third|night|| +2543|AAAAAAAAAPJAAAAA|2543|0|42|23|AM|third|night|| +2544|AAAAAAAABPJAAAAA|2544|0|42|24|AM|third|night|| +2545|AAAAAAAACPJAAAAA|2545|0|42|25|AM|third|night|| +2546|AAAAAAAADPJAAAAA|2546|0|42|26|AM|third|night|| +2547|AAAAAAAAEPJAAAAA|2547|0|42|27|AM|third|night|| +2548|AAAAAAAAFPJAAAAA|2548|0|42|28|AM|third|night|| +2549|AAAAAAAAGPJAAAAA|2549|0|42|29|AM|third|night|| +2550|AAAAAAAAHPJAAAAA|2550|0|42|30|AM|third|night|| +2551|AAAAAAAAIPJAAAAA|2551|0|42|31|AM|third|night|| +2552|AAAAAAAAJPJAAAAA|2552|0|42|32|AM|third|night|| +2553|AAAAAAAAKPJAAAAA|2553|0|42|33|AM|third|night|| +2554|AAAAAAAALPJAAAAA|2554|0|42|34|AM|third|night|| +2555|AAAAAAAAMPJAAAAA|2555|0|42|35|AM|third|night|| +2556|AAAAAAAANPJAAAAA|2556|0|42|36|AM|third|night|| +2557|AAAAAAAAOPJAAAAA|2557|0|42|37|AM|third|night|| +2558|AAAAAAAAPPJAAAAA|2558|0|42|38|AM|third|night|| +2559|AAAAAAAAAAKAAAAA|2559|0|42|39|AM|third|night|| +2560|AAAAAAAABAKAAAAA|2560|0|42|40|AM|third|night|| +2561|AAAAAAAACAKAAAAA|2561|0|42|41|AM|third|night|| +2562|AAAAAAAADAKAAAAA|2562|0|42|42|AM|third|night|| +2563|AAAAAAAAEAKAAAAA|2563|0|42|43|AM|third|night|| +2564|AAAAAAAAFAKAAAAA|2564|0|42|44|AM|third|night|| +2565|AAAAAAAAGAKAAAAA|2565|0|42|45|AM|third|night|| +2566|AAAAAAAAHAKAAAAA|2566|0|42|46|AM|third|night|| +2567|AAAAAAAAIAKAAAAA|2567|0|42|47|AM|third|night|| +2568|AAAAAAAAJAKAAAAA|2568|0|42|48|AM|third|night|| +2569|AAAAAAAAKAKAAAAA|2569|0|42|49|AM|third|night|| +2570|AAAAAAAALAKAAAAA|2570|0|42|50|AM|third|night|| +2571|AAAAAAAAMAKAAAAA|2571|0|42|51|AM|third|night|| +2572|AAAAAAAANAKAAAAA|2572|0|42|52|AM|third|night|| +2573|AAAAAAAAOAKAAAAA|2573|0|42|53|AM|third|night|| +2574|AAAAAAAAPAKAAAAA|2574|0|42|54|AM|third|night|| +2575|AAAAAAAAABKAAAAA|2575|0|42|55|AM|third|night|| +2576|AAAAAAAABBKAAAAA|2576|0|42|56|AM|third|night|| +2577|AAAAAAAACBKAAAAA|2577|0|42|57|AM|third|night|| +2578|AAAAAAAADBKAAAAA|2578|0|42|58|AM|third|night|| +2579|AAAAAAAAEBKAAAAA|2579|0|42|59|AM|third|night|| +2580|AAAAAAAAFBKAAAAA|2580|0|43|0|AM|third|night|| +2581|AAAAAAAAGBKAAAAA|2581|0|43|1|AM|third|night|| +2582|AAAAAAAAHBKAAAAA|2582|0|43|2|AM|third|night|| +2583|AAAAAAAAIBKAAAAA|2583|0|43|3|AM|third|night|| +2584|AAAAAAAAJBKAAAAA|2584|0|43|4|AM|third|night|| +2585|AAAAAAAAKBKAAAAA|2585|0|43|5|AM|third|night|| +2586|AAAAAAAALBKAAAAA|2586|0|43|6|AM|third|night|| +2587|AAAAAAAAMBKAAAAA|2587|0|43|7|AM|third|night|| +2588|AAAAAAAANBKAAAAA|2588|0|43|8|AM|third|night|| +2589|AAAAAAAAOBKAAAAA|2589|0|43|9|AM|third|night|| +2590|AAAAAAAAPBKAAAAA|2590|0|43|10|AM|third|night|| +2591|AAAAAAAAACKAAAAA|2591|0|43|11|AM|third|night|| +2592|AAAAAAAABCKAAAAA|2592|0|43|12|AM|third|night|| +2593|AAAAAAAACCKAAAAA|2593|0|43|13|AM|third|night|| +2594|AAAAAAAADCKAAAAA|2594|0|43|14|AM|third|night|| +2595|AAAAAAAAECKAAAAA|2595|0|43|15|AM|third|night|| +2596|AAAAAAAAFCKAAAAA|2596|0|43|16|AM|third|night|| +2597|AAAAAAAAGCKAAAAA|2597|0|43|17|AM|third|night|| +2598|AAAAAAAAHCKAAAAA|2598|0|43|18|AM|third|night|| +2599|AAAAAAAAICKAAAAA|2599|0|43|19|AM|third|night|| +2600|AAAAAAAAJCKAAAAA|2600|0|43|20|AM|third|night|| +2601|AAAAAAAAKCKAAAAA|2601|0|43|21|AM|third|night|| +2602|AAAAAAAALCKAAAAA|2602|0|43|22|AM|third|night|| +2603|AAAAAAAAMCKAAAAA|2603|0|43|23|AM|third|night|| +2604|AAAAAAAANCKAAAAA|2604|0|43|24|AM|third|night|| +2605|AAAAAAAAOCKAAAAA|2605|0|43|25|AM|third|night|| +2606|AAAAAAAAPCKAAAAA|2606|0|43|26|AM|third|night|| +2607|AAAAAAAAADKAAAAA|2607|0|43|27|AM|third|night|| +2608|AAAAAAAABDKAAAAA|2608|0|43|28|AM|third|night|| +2609|AAAAAAAACDKAAAAA|2609|0|43|29|AM|third|night|| +2610|AAAAAAAADDKAAAAA|2610|0|43|30|AM|third|night|| +2611|AAAAAAAAEDKAAAAA|2611|0|43|31|AM|third|night|| +2612|AAAAAAAAFDKAAAAA|2612|0|43|32|AM|third|night|| +2613|AAAAAAAAGDKAAAAA|2613|0|43|33|AM|third|night|| +2614|AAAAAAAAHDKAAAAA|2614|0|43|34|AM|third|night|| +2615|AAAAAAAAIDKAAAAA|2615|0|43|35|AM|third|night|| +2616|AAAAAAAAJDKAAAAA|2616|0|43|36|AM|third|night|| +2617|AAAAAAAAKDKAAAAA|2617|0|43|37|AM|third|night|| +2618|AAAAAAAALDKAAAAA|2618|0|43|38|AM|third|night|| +2619|AAAAAAAAMDKAAAAA|2619|0|43|39|AM|third|night|| +2620|AAAAAAAANDKAAAAA|2620|0|43|40|AM|third|night|| +2621|AAAAAAAAODKAAAAA|2621|0|43|41|AM|third|night|| +2622|AAAAAAAAPDKAAAAA|2622|0|43|42|AM|third|night|| +2623|AAAAAAAAAEKAAAAA|2623|0|43|43|AM|third|night|| +2624|AAAAAAAABEKAAAAA|2624|0|43|44|AM|third|night|| +2625|AAAAAAAACEKAAAAA|2625|0|43|45|AM|third|night|| +2626|AAAAAAAADEKAAAAA|2626|0|43|46|AM|third|night|| +2627|AAAAAAAAEEKAAAAA|2627|0|43|47|AM|third|night|| +2628|AAAAAAAAFEKAAAAA|2628|0|43|48|AM|third|night|| +2629|AAAAAAAAGEKAAAAA|2629|0|43|49|AM|third|night|| +2630|AAAAAAAAHEKAAAAA|2630|0|43|50|AM|third|night|| +2631|AAAAAAAAIEKAAAAA|2631|0|43|51|AM|third|night|| +2632|AAAAAAAAJEKAAAAA|2632|0|43|52|AM|third|night|| +2633|AAAAAAAAKEKAAAAA|2633|0|43|53|AM|third|night|| +2634|AAAAAAAALEKAAAAA|2634|0|43|54|AM|third|night|| +2635|AAAAAAAAMEKAAAAA|2635|0|43|55|AM|third|night|| +2636|AAAAAAAANEKAAAAA|2636|0|43|56|AM|third|night|| +2637|AAAAAAAAOEKAAAAA|2637|0|43|57|AM|third|night|| +2638|AAAAAAAAPEKAAAAA|2638|0|43|58|AM|third|night|| +2639|AAAAAAAAAFKAAAAA|2639|0|43|59|AM|third|night|| +2640|AAAAAAAABFKAAAAA|2640|0|44|0|AM|third|night|| +2641|AAAAAAAACFKAAAAA|2641|0|44|1|AM|third|night|| +2642|AAAAAAAADFKAAAAA|2642|0|44|2|AM|third|night|| +2643|AAAAAAAAEFKAAAAA|2643|0|44|3|AM|third|night|| +2644|AAAAAAAAFFKAAAAA|2644|0|44|4|AM|third|night|| +2645|AAAAAAAAGFKAAAAA|2645|0|44|5|AM|third|night|| +2646|AAAAAAAAHFKAAAAA|2646|0|44|6|AM|third|night|| +2647|AAAAAAAAIFKAAAAA|2647|0|44|7|AM|third|night|| +2648|AAAAAAAAJFKAAAAA|2648|0|44|8|AM|third|night|| +2649|AAAAAAAAKFKAAAAA|2649|0|44|9|AM|third|night|| +2650|AAAAAAAALFKAAAAA|2650|0|44|10|AM|third|night|| +2651|AAAAAAAAMFKAAAAA|2651|0|44|11|AM|third|night|| +2652|AAAAAAAANFKAAAAA|2652|0|44|12|AM|third|night|| +2653|AAAAAAAAOFKAAAAA|2653|0|44|13|AM|third|night|| +2654|AAAAAAAAPFKAAAAA|2654|0|44|14|AM|third|night|| +2655|AAAAAAAAAGKAAAAA|2655|0|44|15|AM|third|night|| +2656|AAAAAAAABGKAAAAA|2656|0|44|16|AM|third|night|| +2657|AAAAAAAACGKAAAAA|2657|0|44|17|AM|third|night|| +2658|AAAAAAAADGKAAAAA|2658|0|44|18|AM|third|night|| +2659|AAAAAAAAEGKAAAAA|2659|0|44|19|AM|third|night|| +2660|AAAAAAAAFGKAAAAA|2660|0|44|20|AM|third|night|| +2661|AAAAAAAAGGKAAAAA|2661|0|44|21|AM|third|night|| +2662|AAAAAAAAHGKAAAAA|2662|0|44|22|AM|third|night|| +2663|AAAAAAAAIGKAAAAA|2663|0|44|23|AM|third|night|| +2664|AAAAAAAAJGKAAAAA|2664|0|44|24|AM|third|night|| +2665|AAAAAAAAKGKAAAAA|2665|0|44|25|AM|third|night|| +2666|AAAAAAAALGKAAAAA|2666|0|44|26|AM|third|night|| +2667|AAAAAAAAMGKAAAAA|2667|0|44|27|AM|third|night|| +2668|AAAAAAAANGKAAAAA|2668|0|44|28|AM|third|night|| +2669|AAAAAAAAOGKAAAAA|2669|0|44|29|AM|third|night|| +2670|AAAAAAAAPGKAAAAA|2670|0|44|30|AM|third|night|| +2671|AAAAAAAAAHKAAAAA|2671|0|44|31|AM|third|night|| +2672|AAAAAAAABHKAAAAA|2672|0|44|32|AM|third|night|| +2673|AAAAAAAACHKAAAAA|2673|0|44|33|AM|third|night|| +2674|AAAAAAAADHKAAAAA|2674|0|44|34|AM|third|night|| +2675|AAAAAAAAEHKAAAAA|2675|0|44|35|AM|third|night|| +2676|AAAAAAAAFHKAAAAA|2676|0|44|36|AM|third|night|| +2677|AAAAAAAAGHKAAAAA|2677|0|44|37|AM|third|night|| +2678|AAAAAAAAHHKAAAAA|2678|0|44|38|AM|third|night|| +2679|AAAAAAAAIHKAAAAA|2679|0|44|39|AM|third|night|| +2680|AAAAAAAAJHKAAAAA|2680|0|44|40|AM|third|night|| +2681|AAAAAAAAKHKAAAAA|2681|0|44|41|AM|third|night|| +2682|AAAAAAAALHKAAAAA|2682|0|44|42|AM|third|night|| +2683|AAAAAAAAMHKAAAAA|2683|0|44|43|AM|third|night|| +2684|AAAAAAAANHKAAAAA|2684|0|44|44|AM|third|night|| +2685|AAAAAAAAOHKAAAAA|2685|0|44|45|AM|third|night|| +2686|AAAAAAAAPHKAAAAA|2686|0|44|46|AM|third|night|| +2687|AAAAAAAAAIKAAAAA|2687|0|44|47|AM|third|night|| +2688|AAAAAAAABIKAAAAA|2688|0|44|48|AM|third|night|| +2689|AAAAAAAACIKAAAAA|2689|0|44|49|AM|third|night|| +2690|AAAAAAAADIKAAAAA|2690|0|44|50|AM|third|night|| +2691|AAAAAAAAEIKAAAAA|2691|0|44|51|AM|third|night|| +2692|AAAAAAAAFIKAAAAA|2692|0|44|52|AM|third|night|| +2693|AAAAAAAAGIKAAAAA|2693|0|44|53|AM|third|night|| +2694|AAAAAAAAHIKAAAAA|2694|0|44|54|AM|third|night|| +2695|AAAAAAAAIIKAAAAA|2695|0|44|55|AM|third|night|| +2696|AAAAAAAAJIKAAAAA|2696|0|44|56|AM|third|night|| +2697|AAAAAAAAKIKAAAAA|2697|0|44|57|AM|third|night|| +2698|AAAAAAAALIKAAAAA|2698|0|44|58|AM|third|night|| +2699|AAAAAAAAMIKAAAAA|2699|0|44|59|AM|third|night|| +2700|AAAAAAAANIKAAAAA|2700|0|45|0|AM|third|night|| +2701|AAAAAAAAOIKAAAAA|2701|0|45|1|AM|third|night|| +2702|AAAAAAAAPIKAAAAA|2702|0|45|2|AM|third|night|| +2703|AAAAAAAAAJKAAAAA|2703|0|45|3|AM|third|night|| +2704|AAAAAAAABJKAAAAA|2704|0|45|4|AM|third|night|| +2705|AAAAAAAACJKAAAAA|2705|0|45|5|AM|third|night|| +2706|AAAAAAAADJKAAAAA|2706|0|45|6|AM|third|night|| +2707|AAAAAAAAEJKAAAAA|2707|0|45|7|AM|third|night|| +2708|AAAAAAAAFJKAAAAA|2708|0|45|8|AM|third|night|| +2709|AAAAAAAAGJKAAAAA|2709|0|45|9|AM|third|night|| +2710|AAAAAAAAHJKAAAAA|2710|0|45|10|AM|third|night|| +2711|AAAAAAAAIJKAAAAA|2711|0|45|11|AM|third|night|| +2712|AAAAAAAAJJKAAAAA|2712|0|45|12|AM|third|night|| +2713|AAAAAAAAKJKAAAAA|2713|0|45|13|AM|third|night|| +2714|AAAAAAAALJKAAAAA|2714|0|45|14|AM|third|night|| +2715|AAAAAAAAMJKAAAAA|2715|0|45|15|AM|third|night|| +2716|AAAAAAAANJKAAAAA|2716|0|45|16|AM|third|night|| +2717|AAAAAAAAOJKAAAAA|2717|0|45|17|AM|third|night|| +2718|AAAAAAAAPJKAAAAA|2718|0|45|18|AM|third|night|| +2719|AAAAAAAAAKKAAAAA|2719|0|45|19|AM|third|night|| +2720|AAAAAAAABKKAAAAA|2720|0|45|20|AM|third|night|| +2721|AAAAAAAACKKAAAAA|2721|0|45|21|AM|third|night|| +2722|AAAAAAAADKKAAAAA|2722|0|45|22|AM|third|night|| +2723|AAAAAAAAEKKAAAAA|2723|0|45|23|AM|third|night|| +2724|AAAAAAAAFKKAAAAA|2724|0|45|24|AM|third|night|| +2725|AAAAAAAAGKKAAAAA|2725|0|45|25|AM|third|night|| +2726|AAAAAAAAHKKAAAAA|2726|0|45|26|AM|third|night|| +2727|AAAAAAAAIKKAAAAA|2727|0|45|27|AM|third|night|| +2728|AAAAAAAAJKKAAAAA|2728|0|45|28|AM|third|night|| +2729|AAAAAAAAKKKAAAAA|2729|0|45|29|AM|third|night|| +2730|AAAAAAAALKKAAAAA|2730|0|45|30|AM|third|night|| +2731|AAAAAAAAMKKAAAAA|2731|0|45|31|AM|third|night|| +2732|AAAAAAAANKKAAAAA|2732|0|45|32|AM|third|night|| +2733|AAAAAAAAOKKAAAAA|2733|0|45|33|AM|third|night|| +2734|AAAAAAAAPKKAAAAA|2734|0|45|34|AM|third|night|| +2735|AAAAAAAAALKAAAAA|2735|0|45|35|AM|third|night|| +2736|AAAAAAAABLKAAAAA|2736|0|45|36|AM|third|night|| +2737|AAAAAAAACLKAAAAA|2737|0|45|37|AM|third|night|| +2738|AAAAAAAADLKAAAAA|2738|0|45|38|AM|third|night|| +2739|AAAAAAAAELKAAAAA|2739|0|45|39|AM|third|night|| +2740|AAAAAAAAFLKAAAAA|2740|0|45|40|AM|third|night|| +2741|AAAAAAAAGLKAAAAA|2741|0|45|41|AM|third|night|| +2742|AAAAAAAAHLKAAAAA|2742|0|45|42|AM|third|night|| +2743|AAAAAAAAILKAAAAA|2743|0|45|43|AM|third|night|| +2744|AAAAAAAAJLKAAAAA|2744|0|45|44|AM|third|night|| +2745|AAAAAAAAKLKAAAAA|2745|0|45|45|AM|third|night|| +2746|AAAAAAAALLKAAAAA|2746|0|45|46|AM|third|night|| +2747|AAAAAAAAMLKAAAAA|2747|0|45|47|AM|third|night|| +2748|AAAAAAAANLKAAAAA|2748|0|45|48|AM|third|night|| +2749|AAAAAAAAOLKAAAAA|2749|0|45|49|AM|third|night|| +2750|AAAAAAAAPLKAAAAA|2750|0|45|50|AM|third|night|| +2751|AAAAAAAAAMKAAAAA|2751|0|45|51|AM|third|night|| +2752|AAAAAAAABMKAAAAA|2752|0|45|52|AM|third|night|| +2753|AAAAAAAACMKAAAAA|2753|0|45|53|AM|third|night|| +2754|AAAAAAAADMKAAAAA|2754|0|45|54|AM|third|night|| +2755|AAAAAAAAEMKAAAAA|2755|0|45|55|AM|third|night|| +2756|AAAAAAAAFMKAAAAA|2756|0|45|56|AM|third|night|| +2757|AAAAAAAAGMKAAAAA|2757|0|45|57|AM|third|night|| +2758|AAAAAAAAHMKAAAAA|2758|0|45|58|AM|third|night|| +2759|AAAAAAAAIMKAAAAA|2759|0|45|59|AM|third|night|| +2760|AAAAAAAAJMKAAAAA|2760|0|46|0|AM|third|night|| +2761|AAAAAAAAKMKAAAAA|2761|0|46|1|AM|third|night|| +2762|AAAAAAAALMKAAAAA|2762|0|46|2|AM|third|night|| +2763|AAAAAAAAMMKAAAAA|2763|0|46|3|AM|third|night|| +2764|AAAAAAAANMKAAAAA|2764|0|46|4|AM|third|night|| +2765|AAAAAAAAOMKAAAAA|2765|0|46|5|AM|third|night|| +2766|AAAAAAAAPMKAAAAA|2766|0|46|6|AM|third|night|| +2767|AAAAAAAAANKAAAAA|2767|0|46|7|AM|third|night|| +2768|AAAAAAAABNKAAAAA|2768|0|46|8|AM|third|night|| +2769|AAAAAAAACNKAAAAA|2769|0|46|9|AM|third|night|| +2770|AAAAAAAADNKAAAAA|2770|0|46|10|AM|third|night|| +2771|AAAAAAAAENKAAAAA|2771|0|46|11|AM|third|night|| +2772|AAAAAAAAFNKAAAAA|2772|0|46|12|AM|third|night|| +2773|AAAAAAAAGNKAAAAA|2773|0|46|13|AM|third|night|| +2774|AAAAAAAAHNKAAAAA|2774|0|46|14|AM|third|night|| +2775|AAAAAAAAINKAAAAA|2775|0|46|15|AM|third|night|| +2776|AAAAAAAAJNKAAAAA|2776|0|46|16|AM|third|night|| +2777|AAAAAAAAKNKAAAAA|2777|0|46|17|AM|third|night|| +2778|AAAAAAAALNKAAAAA|2778|0|46|18|AM|third|night|| +2779|AAAAAAAAMNKAAAAA|2779|0|46|19|AM|third|night|| +2780|AAAAAAAANNKAAAAA|2780|0|46|20|AM|third|night|| +2781|AAAAAAAAONKAAAAA|2781|0|46|21|AM|third|night|| +2782|AAAAAAAAPNKAAAAA|2782|0|46|22|AM|third|night|| +2783|AAAAAAAAAOKAAAAA|2783|0|46|23|AM|third|night|| +2784|AAAAAAAABOKAAAAA|2784|0|46|24|AM|third|night|| +2785|AAAAAAAACOKAAAAA|2785|0|46|25|AM|third|night|| +2786|AAAAAAAADOKAAAAA|2786|0|46|26|AM|third|night|| +2787|AAAAAAAAEOKAAAAA|2787|0|46|27|AM|third|night|| +2788|AAAAAAAAFOKAAAAA|2788|0|46|28|AM|third|night|| +2789|AAAAAAAAGOKAAAAA|2789|0|46|29|AM|third|night|| +2790|AAAAAAAAHOKAAAAA|2790|0|46|30|AM|third|night|| +2791|AAAAAAAAIOKAAAAA|2791|0|46|31|AM|third|night|| +2792|AAAAAAAAJOKAAAAA|2792|0|46|32|AM|third|night|| +2793|AAAAAAAAKOKAAAAA|2793|0|46|33|AM|third|night|| +2794|AAAAAAAALOKAAAAA|2794|0|46|34|AM|third|night|| +2795|AAAAAAAAMOKAAAAA|2795|0|46|35|AM|third|night|| +2796|AAAAAAAANOKAAAAA|2796|0|46|36|AM|third|night|| +2797|AAAAAAAAOOKAAAAA|2797|0|46|37|AM|third|night|| +2798|AAAAAAAAPOKAAAAA|2798|0|46|38|AM|third|night|| +2799|AAAAAAAAAPKAAAAA|2799|0|46|39|AM|third|night|| +2800|AAAAAAAABPKAAAAA|2800|0|46|40|AM|third|night|| +2801|AAAAAAAACPKAAAAA|2801|0|46|41|AM|third|night|| +2802|AAAAAAAADPKAAAAA|2802|0|46|42|AM|third|night|| +2803|AAAAAAAAEPKAAAAA|2803|0|46|43|AM|third|night|| +2804|AAAAAAAAFPKAAAAA|2804|0|46|44|AM|third|night|| +2805|AAAAAAAAGPKAAAAA|2805|0|46|45|AM|third|night|| +2806|AAAAAAAAHPKAAAAA|2806|0|46|46|AM|third|night|| +2807|AAAAAAAAIPKAAAAA|2807|0|46|47|AM|third|night|| +2808|AAAAAAAAJPKAAAAA|2808|0|46|48|AM|third|night|| +2809|AAAAAAAAKPKAAAAA|2809|0|46|49|AM|third|night|| +2810|AAAAAAAALPKAAAAA|2810|0|46|50|AM|third|night|| +2811|AAAAAAAAMPKAAAAA|2811|0|46|51|AM|third|night|| +2812|AAAAAAAANPKAAAAA|2812|0|46|52|AM|third|night|| +2813|AAAAAAAAOPKAAAAA|2813|0|46|53|AM|third|night|| +2814|AAAAAAAAPPKAAAAA|2814|0|46|54|AM|third|night|| +2815|AAAAAAAAAALAAAAA|2815|0|46|55|AM|third|night|| +2816|AAAAAAAABALAAAAA|2816|0|46|56|AM|third|night|| +2817|AAAAAAAACALAAAAA|2817|0|46|57|AM|third|night|| +2818|AAAAAAAADALAAAAA|2818|0|46|58|AM|third|night|| +2819|AAAAAAAAEALAAAAA|2819|0|46|59|AM|third|night|| +2820|AAAAAAAAFALAAAAA|2820|0|47|0|AM|third|night|| +2821|AAAAAAAAGALAAAAA|2821|0|47|1|AM|third|night|| +2822|AAAAAAAAHALAAAAA|2822|0|47|2|AM|third|night|| +2823|AAAAAAAAIALAAAAA|2823|0|47|3|AM|third|night|| +2824|AAAAAAAAJALAAAAA|2824|0|47|4|AM|third|night|| +2825|AAAAAAAAKALAAAAA|2825|0|47|5|AM|third|night|| +2826|AAAAAAAALALAAAAA|2826|0|47|6|AM|third|night|| +2827|AAAAAAAAMALAAAAA|2827|0|47|7|AM|third|night|| +2828|AAAAAAAANALAAAAA|2828|0|47|8|AM|third|night|| +2829|AAAAAAAAOALAAAAA|2829|0|47|9|AM|third|night|| +2830|AAAAAAAAPALAAAAA|2830|0|47|10|AM|third|night|| +2831|AAAAAAAAABLAAAAA|2831|0|47|11|AM|third|night|| +2832|AAAAAAAABBLAAAAA|2832|0|47|12|AM|third|night|| +2833|AAAAAAAACBLAAAAA|2833|0|47|13|AM|third|night|| +2834|AAAAAAAADBLAAAAA|2834|0|47|14|AM|third|night|| +2835|AAAAAAAAEBLAAAAA|2835|0|47|15|AM|third|night|| +2836|AAAAAAAAFBLAAAAA|2836|0|47|16|AM|third|night|| +2837|AAAAAAAAGBLAAAAA|2837|0|47|17|AM|third|night|| +2838|AAAAAAAAHBLAAAAA|2838|0|47|18|AM|third|night|| +2839|AAAAAAAAIBLAAAAA|2839|0|47|19|AM|third|night|| +2840|AAAAAAAAJBLAAAAA|2840|0|47|20|AM|third|night|| +2841|AAAAAAAAKBLAAAAA|2841|0|47|21|AM|third|night|| +2842|AAAAAAAALBLAAAAA|2842|0|47|22|AM|third|night|| +2843|AAAAAAAAMBLAAAAA|2843|0|47|23|AM|third|night|| +2844|AAAAAAAANBLAAAAA|2844|0|47|24|AM|third|night|| +2845|AAAAAAAAOBLAAAAA|2845|0|47|25|AM|third|night|| +2846|AAAAAAAAPBLAAAAA|2846|0|47|26|AM|third|night|| +2847|AAAAAAAAACLAAAAA|2847|0|47|27|AM|third|night|| +2848|AAAAAAAABCLAAAAA|2848|0|47|28|AM|third|night|| +2849|AAAAAAAACCLAAAAA|2849|0|47|29|AM|third|night|| +2850|AAAAAAAADCLAAAAA|2850|0|47|30|AM|third|night|| +2851|AAAAAAAAECLAAAAA|2851|0|47|31|AM|third|night|| +2852|AAAAAAAAFCLAAAAA|2852|0|47|32|AM|third|night|| +2853|AAAAAAAAGCLAAAAA|2853|0|47|33|AM|third|night|| +2854|AAAAAAAAHCLAAAAA|2854|0|47|34|AM|third|night|| +2855|AAAAAAAAICLAAAAA|2855|0|47|35|AM|third|night|| +2856|AAAAAAAAJCLAAAAA|2856|0|47|36|AM|third|night|| +2857|AAAAAAAAKCLAAAAA|2857|0|47|37|AM|third|night|| +2858|AAAAAAAALCLAAAAA|2858|0|47|38|AM|third|night|| +2859|AAAAAAAAMCLAAAAA|2859|0|47|39|AM|third|night|| +2860|AAAAAAAANCLAAAAA|2860|0|47|40|AM|third|night|| +2861|AAAAAAAAOCLAAAAA|2861|0|47|41|AM|third|night|| +2862|AAAAAAAAPCLAAAAA|2862|0|47|42|AM|third|night|| +2863|AAAAAAAAADLAAAAA|2863|0|47|43|AM|third|night|| +2864|AAAAAAAABDLAAAAA|2864|0|47|44|AM|third|night|| +2865|AAAAAAAACDLAAAAA|2865|0|47|45|AM|third|night|| +2866|AAAAAAAADDLAAAAA|2866|0|47|46|AM|third|night|| +2867|AAAAAAAAEDLAAAAA|2867|0|47|47|AM|third|night|| +2868|AAAAAAAAFDLAAAAA|2868|0|47|48|AM|third|night|| +2869|AAAAAAAAGDLAAAAA|2869|0|47|49|AM|third|night|| +2870|AAAAAAAAHDLAAAAA|2870|0|47|50|AM|third|night|| +2871|AAAAAAAAIDLAAAAA|2871|0|47|51|AM|third|night|| +2872|AAAAAAAAJDLAAAAA|2872|0|47|52|AM|third|night|| +2873|AAAAAAAAKDLAAAAA|2873|0|47|53|AM|third|night|| +2874|AAAAAAAALDLAAAAA|2874|0|47|54|AM|third|night|| +2875|AAAAAAAAMDLAAAAA|2875|0|47|55|AM|third|night|| +2876|AAAAAAAANDLAAAAA|2876|0|47|56|AM|third|night|| +2877|AAAAAAAAODLAAAAA|2877|0|47|57|AM|third|night|| +2878|AAAAAAAAPDLAAAAA|2878|0|47|58|AM|third|night|| +2879|AAAAAAAAAELAAAAA|2879|0|47|59|AM|third|night|| +2880|AAAAAAAABELAAAAA|2880|0|48|0|AM|third|night|| +2881|AAAAAAAACELAAAAA|2881|0|48|1|AM|third|night|| +2882|AAAAAAAADELAAAAA|2882|0|48|2|AM|third|night|| +2883|AAAAAAAAEELAAAAA|2883|0|48|3|AM|third|night|| +2884|AAAAAAAAFELAAAAA|2884|0|48|4|AM|third|night|| +2885|AAAAAAAAGELAAAAA|2885|0|48|5|AM|third|night|| +2886|AAAAAAAAHELAAAAA|2886|0|48|6|AM|third|night|| +2887|AAAAAAAAIELAAAAA|2887|0|48|7|AM|third|night|| +2888|AAAAAAAAJELAAAAA|2888|0|48|8|AM|third|night|| +2889|AAAAAAAAKELAAAAA|2889|0|48|9|AM|third|night|| +2890|AAAAAAAALELAAAAA|2890|0|48|10|AM|third|night|| +2891|AAAAAAAAMELAAAAA|2891|0|48|11|AM|third|night|| +2892|AAAAAAAANELAAAAA|2892|0|48|12|AM|third|night|| +2893|AAAAAAAAOELAAAAA|2893|0|48|13|AM|third|night|| +2894|AAAAAAAAPELAAAAA|2894|0|48|14|AM|third|night|| +2895|AAAAAAAAAFLAAAAA|2895|0|48|15|AM|third|night|| +2896|AAAAAAAABFLAAAAA|2896|0|48|16|AM|third|night|| +2897|AAAAAAAACFLAAAAA|2897|0|48|17|AM|third|night|| +2898|AAAAAAAADFLAAAAA|2898|0|48|18|AM|third|night|| +2899|AAAAAAAAEFLAAAAA|2899|0|48|19|AM|third|night|| +2900|AAAAAAAAFFLAAAAA|2900|0|48|20|AM|third|night|| +2901|AAAAAAAAGFLAAAAA|2901|0|48|21|AM|third|night|| +2902|AAAAAAAAHFLAAAAA|2902|0|48|22|AM|third|night|| +2903|AAAAAAAAIFLAAAAA|2903|0|48|23|AM|third|night|| +2904|AAAAAAAAJFLAAAAA|2904|0|48|24|AM|third|night|| +2905|AAAAAAAAKFLAAAAA|2905|0|48|25|AM|third|night|| +2906|AAAAAAAALFLAAAAA|2906|0|48|26|AM|third|night|| +2907|AAAAAAAAMFLAAAAA|2907|0|48|27|AM|third|night|| +2908|AAAAAAAANFLAAAAA|2908|0|48|28|AM|third|night|| +2909|AAAAAAAAOFLAAAAA|2909|0|48|29|AM|third|night|| +2910|AAAAAAAAPFLAAAAA|2910|0|48|30|AM|third|night|| +2911|AAAAAAAAAGLAAAAA|2911|0|48|31|AM|third|night|| +2912|AAAAAAAABGLAAAAA|2912|0|48|32|AM|third|night|| +2913|AAAAAAAACGLAAAAA|2913|0|48|33|AM|third|night|| +2914|AAAAAAAADGLAAAAA|2914|0|48|34|AM|third|night|| +2915|AAAAAAAAEGLAAAAA|2915|0|48|35|AM|third|night|| +2916|AAAAAAAAFGLAAAAA|2916|0|48|36|AM|third|night|| +2917|AAAAAAAAGGLAAAAA|2917|0|48|37|AM|third|night|| +2918|AAAAAAAAHGLAAAAA|2918|0|48|38|AM|third|night|| +2919|AAAAAAAAIGLAAAAA|2919|0|48|39|AM|third|night|| +2920|AAAAAAAAJGLAAAAA|2920|0|48|40|AM|third|night|| +2921|AAAAAAAAKGLAAAAA|2921|0|48|41|AM|third|night|| +2922|AAAAAAAALGLAAAAA|2922|0|48|42|AM|third|night|| +2923|AAAAAAAAMGLAAAAA|2923|0|48|43|AM|third|night|| +2924|AAAAAAAANGLAAAAA|2924|0|48|44|AM|third|night|| +2925|AAAAAAAAOGLAAAAA|2925|0|48|45|AM|third|night|| +2926|AAAAAAAAPGLAAAAA|2926|0|48|46|AM|third|night|| +2927|AAAAAAAAAHLAAAAA|2927|0|48|47|AM|third|night|| +2928|AAAAAAAABHLAAAAA|2928|0|48|48|AM|third|night|| +2929|AAAAAAAACHLAAAAA|2929|0|48|49|AM|third|night|| +2930|AAAAAAAADHLAAAAA|2930|0|48|50|AM|third|night|| +2931|AAAAAAAAEHLAAAAA|2931|0|48|51|AM|third|night|| +2932|AAAAAAAAFHLAAAAA|2932|0|48|52|AM|third|night|| +2933|AAAAAAAAGHLAAAAA|2933|0|48|53|AM|third|night|| +2934|AAAAAAAAHHLAAAAA|2934|0|48|54|AM|third|night|| +2935|AAAAAAAAIHLAAAAA|2935|0|48|55|AM|third|night|| +2936|AAAAAAAAJHLAAAAA|2936|0|48|56|AM|third|night|| +2937|AAAAAAAAKHLAAAAA|2937|0|48|57|AM|third|night|| +2938|AAAAAAAALHLAAAAA|2938|0|48|58|AM|third|night|| +2939|AAAAAAAAMHLAAAAA|2939|0|48|59|AM|third|night|| +2940|AAAAAAAANHLAAAAA|2940|0|49|0|AM|third|night|| +2941|AAAAAAAAOHLAAAAA|2941|0|49|1|AM|third|night|| +2942|AAAAAAAAPHLAAAAA|2942|0|49|2|AM|third|night|| +2943|AAAAAAAAAILAAAAA|2943|0|49|3|AM|third|night|| +2944|AAAAAAAABILAAAAA|2944|0|49|4|AM|third|night|| +2945|AAAAAAAACILAAAAA|2945|0|49|5|AM|third|night|| +2946|AAAAAAAADILAAAAA|2946|0|49|6|AM|third|night|| +2947|AAAAAAAAEILAAAAA|2947|0|49|7|AM|third|night|| +2948|AAAAAAAAFILAAAAA|2948|0|49|8|AM|third|night|| +2949|AAAAAAAAGILAAAAA|2949|0|49|9|AM|third|night|| +2950|AAAAAAAAHILAAAAA|2950|0|49|10|AM|third|night|| +2951|AAAAAAAAIILAAAAA|2951|0|49|11|AM|third|night|| +2952|AAAAAAAAJILAAAAA|2952|0|49|12|AM|third|night|| +2953|AAAAAAAAKILAAAAA|2953|0|49|13|AM|third|night|| +2954|AAAAAAAALILAAAAA|2954|0|49|14|AM|third|night|| +2955|AAAAAAAAMILAAAAA|2955|0|49|15|AM|third|night|| +2956|AAAAAAAANILAAAAA|2956|0|49|16|AM|third|night|| +2957|AAAAAAAAOILAAAAA|2957|0|49|17|AM|third|night|| +2958|AAAAAAAAPILAAAAA|2958|0|49|18|AM|third|night|| +2959|AAAAAAAAAJLAAAAA|2959|0|49|19|AM|third|night|| +2960|AAAAAAAABJLAAAAA|2960|0|49|20|AM|third|night|| +2961|AAAAAAAACJLAAAAA|2961|0|49|21|AM|third|night|| +2962|AAAAAAAADJLAAAAA|2962|0|49|22|AM|third|night|| +2963|AAAAAAAAEJLAAAAA|2963|0|49|23|AM|third|night|| +2964|AAAAAAAAFJLAAAAA|2964|0|49|24|AM|third|night|| +2965|AAAAAAAAGJLAAAAA|2965|0|49|25|AM|third|night|| +2966|AAAAAAAAHJLAAAAA|2966|0|49|26|AM|third|night|| +2967|AAAAAAAAIJLAAAAA|2967|0|49|27|AM|third|night|| +2968|AAAAAAAAJJLAAAAA|2968|0|49|28|AM|third|night|| +2969|AAAAAAAAKJLAAAAA|2969|0|49|29|AM|third|night|| +2970|AAAAAAAALJLAAAAA|2970|0|49|30|AM|third|night|| +2971|AAAAAAAAMJLAAAAA|2971|0|49|31|AM|third|night|| +2972|AAAAAAAANJLAAAAA|2972|0|49|32|AM|third|night|| +2973|AAAAAAAAOJLAAAAA|2973|0|49|33|AM|third|night|| +2974|AAAAAAAAPJLAAAAA|2974|0|49|34|AM|third|night|| +2975|AAAAAAAAAKLAAAAA|2975|0|49|35|AM|third|night|| +2976|AAAAAAAABKLAAAAA|2976|0|49|36|AM|third|night|| +2977|AAAAAAAACKLAAAAA|2977|0|49|37|AM|third|night|| +2978|AAAAAAAADKLAAAAA|2978|0|49|38|AM|third|night|| +2979|AAAAAAAAEKLAAAAA|2979|0|49|39|AM|third|night|| +2980|AAAAAAAAFKLAAAAA|2980|0|49|40|AM|third|night|| +2981|AAAAAAAAGKLAAAAA|2981|0|49|41|AM|third|night|| +2982|AAAAAAAAHKLAAAAA|2982|0|49|42|AM|third|night|| +2983|AAAAAAAAIKLAAAAA|2983|0|49|43|AM|third|night|| +2984|AAAAAAAAJKLAAAAA|2984|0|49|44|AM|third|night|| +2985|AAAAAAAAKKLAAAAA|2985|0|49|45|AM|third|night|| +2986|AAAAAAAALKLAAAAA|2986|0|49|46|AM|third|night|| +2987|AAAAAAAAMKLAAAAA|2987|0|49|47|AM|third|night|| +2988|AAAAAAAANKLAAAAA|2988|0|49|48|AM|third|night|| +2989|AAAAAAAAOKLAAAAA|2989|0|49|49|AM|third|night|| +2990|AAAAAAAAPKLAAAAA|2990|0|49|50|AM|third|night|| +2991|AAAAAAAAALLAAAAA|2991|0|49|51|AM|third|night|| +2992|AAAAAAAABLLAAAAA|2992|0|49|52|AM|third|night|| +2993|AAAAAAAACLLAAAAA|2993|0|49|53|AM|third|night|| +2994|AAAAAAAADLLAAAAA|2994|0|49|54|AM|third|night|| +2995|AAAAAAAAELLAAAAA|2995|0|49|55|AM|third|night|| +2996|AAAAAAAAFLLAAAAA|2996|0|49|56|AM|third|night|| +2997|AAAAAAAAGLLAAAAA|2997|0|49|57|AM|third|night|| +2998|AAAAAAAAHLLAAAAA|2998|0|49|58|AM|third|night|| +2999|AAAAAAAAILLAAAAA|2999|0|49|59|AM|third|night|| +3000|AAAAAAAAJLLAAAAA|3000|0|50|0|AM|third|night|| +3001|AAAAAAAAKLLAAAAA|3001|0|50|1|AM|third|night|| +3002|AAAAAAAALLLAAAAA|3002|0|50|2|AM|third|night|| +3003|AAAAAAAAMLLAAAAA|3003|0|50|3|AM|third|night|| +3004|AAAAAAAANLLAAAAA|3004|0|50|4|AM|third|night|| +3005|AAAAAAAAOLLAAAAA|3005|0|50|5|AM|third|night|| +3006|AAAAAAAAPLLAAAAA|3006|0|50|6|AM|third|night|| +3007|AAAAAAAAAMLAAAAA|3007|0|50|7|AM|third|night|| +3008|AAAAAAAABMLAAAAA|3008|0|50|8|AM|third|night|| +3009|AAAAAAAACMLAAAAA|3009|0|50|9|AM|third|night|| +3010|AAAAAAAADMLAAAAA|3010|0|50|10|AM|third|night|| +3011|AAAAAAAAEMLAAAAA|3011|0|50|11|AM|third|night|| +3012|AAAAAAAAFMLAAAAA|3012|0|50|12|AM|third|night|| +3013|AAAAAAAAGMLAAAAA|3013|0|50|13|AM|third|night|| +3014|AAAAAAAAHMLAAAAA|3014|0|50|14|AM|third|night|| +3015|AAAAAAAAIMLAAAAA|3015|0|50|15|AM|third|night|| +3016|AAAAAAAAJMLAAAAA|3016|0|50|16|AM|third|night|| +3017|AAAAAAAAKMLAAAAA|3017|0|50|17|AM|third|night|| +3018|AAAAAAAALMLAAAAA|3018|0|50|18|AM|third|night|| +3019|AAAAAAAAMMLAAAAA|3019|0|50|19|AM|third|night|| +3020|AAAAAAAANMLAAAAA|3020|0|50|20|AM|third|night|| +3021|AAAAAAAAOMLAAAAA|3021|0|50|21|AM|third|night|| +3022|AAAAAAAAPMLAAAAA|3022|0|50|22|AM|third|night|| +3023|AAAAAAAAANLAAAAA|3023|0|50|23|AM|third|night|| +3024|AAAAAAAABNLAAAAA|3024|0|50|24|AM|third|night|| +3025|AAAAAAAACNLAAAAA|3025|0|50|25|AM|third|night|| +3026|AAAAAAAADNLAAAAA|3026|0|50|26|AM|third|night|| +3027|AAAAAAAAENLAAAAA|3027|0|50|27|AM|third|night|| +3028|AAAAAAAAFNLAAAAA|3028|0|50|28|AM|third|night|| +3029|AAAAAAAAGNLAAAAA|3029|0|50|29|AM|third|night|| +3030|AAAAAAAAHNLAAAAA|3030|0|50|30|AM|third|night|| +3031|AAAAAAAAINLAAAAA|3031|0|50|31|AM|third|night|| +3032|AAAAAAAAJNLAAAAA|3032|0|50|32|AM|third|night|| +3033|AAAAAAAAKNLAAAAA|3033|0|50|33|AM|third|night|| +3034|AAAAAAAALNLAAAAA|3034|0|50|34|AM|third|night|| +3035|AAAAAAAAMNLAAAAA|3035|0|50|35|AM|third|night|| +3036|AAAAAAAANNLAAAAA|3036|0|50|36|AM|third|night|| +3037|AAAAAAAAONLAAAAA|3037|0|50|37|AM|third|night|| +3038|AAAAAAAAPNLAAAAA|3038|0|50|38|AM|third|night|| +3039|AAAAAAAAAOLAAAAA|3039|0|50|39|AM|third|night|| +3040|AAAAAAAABOLAAAAA|3040|0|50|40|AM|third|night|| +3041|AAAAAAAACOLAAAAA|3041|0|50|41|AM|third|night|| +3042|AAAAAAAADOLAAAAA|3042|0|50|42|AM|third|night|| +3043|AAAAAAAAEOLAAAAA|3043|0|50|43|AM|third|night|| +3044|AAAAAAAAFOLAAAAA|3044|0|50|44|AM|third|night|| +3045|AAAAAAAAGOLAAAAA|3045|0|50|45|AM|third|night|| +3046|AAAAAAAAHOLAAAAA|3046|0|50|46|AM|third|night|| +3047|AAAAAAAAIOLAAAAA|3047|0|50|47|AM|third|night|| +3048|AAAAAAAAJOLAAAAA|3048|0|50|48|AM|third|night|| +3049|AAAAAAAAKOLAAAAA|3049|0|50|49|AM|third|night|| +3050|AAAAAAAALOLAAAAA|3050|0|50|50|AM|third|night|| +3051|AAAAAAAAMOLAAAAA|3051|0|50|51|AM|third|night|| +3052|AAAAAAAANOLAAAAA|3052|0|50|52|AM|third|night|| +3053|AAAAAAAAOOLAAAAA|3053|0|50|53|AM|third|night|| +3054|AAAAAAAAPOLAAAAA|3054|0|50|54|AM|third|night|| +3055|AAAAAAAAAPLAAAAA|3055|0|50|55|AM|third|night|| +3056|AAAAAAAABPLAAAAA|3056|0|50|56|AM|third|night|| +3057|AAAAAAAACPLAAAAA|3057|0|50|57|AM|third|night|| +3058|AAAAAAAADPLAAAAA|3058|0|50|58|AM|third|night|| +3059|AAAAAAAAEPLAAAAA|3059|0|50|59|AM|third|night|| +3060|AAAAAAAAFPLAAAAA|3060|0|51|0|AM|third|night|| +3061|AAAAAAAAGPLAAAAA|3061|0|51|1|AM|third|night|| +3062|AAAAAAAAHPLAAAAA|3062|0|51|2|AM|third|night|| +3063|AAAAAAAAIPLAAAAA|3063|0|51|3|AM|third|night|| +3064|AAAAAAAAJPLAAAAA|3064|0|51|4|AM|third|night|| +3065|AAAAAAAAKPLAAAAA|3065|0|51|5|AM|third|night|| +3066|AAAAAAAALPLAAAAA|3066|0|51|6|AM|third|night|| +3067|AAAAAAAAMPLAAAAA|3067|0|51|7|AM|third|night|| +3068|AAAAAAAANPLAAAAA|3068|0|51|8|AM|third|night|| +3069|AAAAAAAAOPLAAAAA|3069|0|51|9|AM|third|night|| +3070|AAAAAAAAPPLAAAAA|3070|0|51|10|AM|third|night|| +3071|AAAAAAAAAAMAAAAA|3071|0|51|11|AM|third|night|| +3072|AAAAAAAABAMAAAAA|3072|0|51|12|AM|third|night|| +3073|AAAAAAAACAMAAAAA|3073|0|51|13|AM|third|night|| +3074|AAAAAAAADAMAAAAA|3074|0|51|14|AM|third|night|| +3075|AAAAAAAAEAMAAAAA|3075|0|51|15|AM|third|night|| +3076|AAAAAAAAFAMAAAAA|3076|0|51|16|AM|third|night|| +3077|AAAAAAAAGAMAAAAA|3077|0|51|17|AM|third|night|| +3078|AAAAAAAAHAMAAAAA|3078|0|51|18|AM|third|night|| +3079|AAAAAAAAIAMAAAAA|3079|0|51|19|AM|third|night|| +3080|AAAAAAAAJAMAAAAA|3080|0|51|20|AM|third|night|| +3081|AAAAAAAAKAMAAAAA|3081|0|51|21|AM|third|night|| +3082|AAAAAAAALAMAAAAA|3082|0|51|22|AM|third|night|| +3083|AAAAAAAAMAMAAAAA|3083|0|51|23|AM|third|night|| +3084|AAAAAAAANAMAAAAA|3084|0|51|24|AM|third|night|| +3085|AAAAAAAAOAMAAAAA|3085|0|51|25|AM|third|night|| +3086|AAAAAAAAPAMAAAAA|3086|0|51|26|AM|third|night|| +3087|AAAAAAAAABMAAAAA|3087|0|51|27|AM|third|night|| +3088|AAAAAAAABBMAAAAA|3088|0|51|28|AM|third|night|| +3089|AAAAAAAACBMAAAAA|3089|0|51|29|AM|third|night|| +3090|AAAAAAAADBMAAAAA|3090|0|51|30|AM|third|night|| +3091|AAAAAAAAEBMAAAAA|3091|0|51|31|AM|third|night|| +3092|AAAAAAAAFBMAAAAA|3092|0|51|32|AM|third|night|| +3093|AAAAAAAAGBMAAAAA|3093|0|51|33|AM|third|night|| +3094|AAAAAAAAHBMAAAAA|3094|0|51|34|AM|third|night|| +3095|AAAAAAAAIBMAAAAA|3095|0|51|35|AM|third|night|| +3096|AAAAAAAAJBMAAAAA|3096|0|51|36|AM|third|night|| +3097|AAAAAAAAKBMAAAAA|3097|0|51|37|AM|third|night|| +3098|AAAAAAAALBMAAAAA|3098|0|51|38|AM|third|night|| +3099|AAAAAAAAMBMAAAAA|3099|0|51|39|AM|third|night|| +3100|AAAAAAAANBMAAAAA|3100|0|51|40|AM|third|night|| +3101|AAAAAAAAOBMAAAAA|3101|0|51|41|AM|third|night|| +3102|AAAAAAAAPBMAAAAA|3102|0|51|42|AM|third|night|| +3103|AAAAAAAAACMAAAAA|3103|0|51|43|AM|third|night|| +3104|AAAAAAAABCMAAAAA|3104|0|51|44|AM|third|night|| +3105|AAAAAAAACCMAAAAA|3105|0|51|45|AM|third|night|| +3106|AAAAAAAADCMAAAAA|3106|0|51|46|AM|third|night|| +3107|AAAAAAAAECMAAAAA|3107|0|51|47|AM|third|night|| +3108|AAAAAAAAFCMAAAAA|3108|0|51|48|AM|third|night|| +3109|AAAAAAAAGCMAAAAA|3109|0|51|49|AM|third|night|| +3110|AAAAAAAAHCMAAAAA|3110|0|51|50|AM|third|night|| +3111|AAAAAAAAICMAAAAA|3111|0|51|51|AM|third|night|| +3112|AAAAAAAAJCMAAAAA|3112|0|51|52|AM|third|night|| +3113|AAAAAAAAKCMAAAAA|3113|0|51|53|AM|third|night|| +3114|AAAAAAAALCMAAAAA|3114|0|51|54|AM|third|night|| +3115|AAAAAAAAMCMAAAAA|3115|0|51|55|AM|third|night|| +3116|AAAAAAAANCMAAAAA|3116|0|51|56|AM|third|night|| +3117|AAAAAAAAOCMAAAAA|3117|0|51|57|AM|third|night|| +3118|AAAAAAAAPCMAAAAA|3118|0|51|58|AM|third|night|| +3119|AAAAAAAAADMAAAAA|3119|0|51|59|AM|third|night|| +3120|AAAAAAAABDMAAAAA|3120|0|52|0|AM|third|night|| +3121|AAAAAAAACDMAAAAA|3121|0|52|1|AM|third|night|| +3122|AAAAAAAADDMAAAAA|3122|0|52|2|AM|third|night|| +3123|AAAAAAAAEDMAAAAA|3123|0|52|3|AM|third|night|| +3124|AAAAAAAAFDMAAAAA|3124|0|52|4|AM|third|night|| +3125|AAAAAAAAGDMAAAAA|3125|0|52|5|AM|third|night|| +3126|AAAAAAAAHDMAAAAA|3126|0|52|6|AM|third|night|| +3127|AAAAAAAAIDMAAAAA|3127|0|52|7|AM|third|night|| +3128|AAAAAAAAJDMAAAAA|3128|0|52|8|AM|third|night|| +3129|AAAAAAAAKDMAAAAA|3129|0|52|9|AM|third|night|| +3130|AAAAAAAALDMAAAAA|3130|0|52|10|AM|third|night|| +3131|AAAAAAAAMDMAAAAA|3131|0|52|11|AM|third|night|| +3132|AAAAAAAANDMAAAAA|3132|0|52|12|AM|third|night|| +3133|AAAAAAAAODMAAAAA|3133|0|52|13|AM|third|night|| +3134|AAAAAAAAPDMAAAAA|3134|0|52|14|AM|third|night|| +3135|AAAAAAAAAEMAAAAA|3135|0|52|15|AM|third|night|| +3136|AAAAAAAABEMAAAAA|3136|0|52|16|AM|third|night|| +3137|AAAAAAAACEMAAAAA|3137|0|52|17|AM|third|night|| +3138|AAAAAAAADEMAAAAA|3138|0|52|18|AM|third|night|| +3139|AAAAAAAAEEMAAAAA|3139|0|52|19|AM|third|night|| +3140|AAAAAAAAFEMAAAAA|3140|0|52|20|AM|third|night|| +3141|AAAAAAAAGEMAAAAA|3141|0|52|21|AM|third|night|| +3142|AAAAAAAAHEMAAAAA|3142|0|52|22|AM|third|night|| +3143|AAAAAAAAIEMAAAAA|3143|0|52|23|AM|third|night|| +3144|AAAAAAAAJEMAAAAA|3144|0|52|24|AM|third|night|| +3145|AAAAAAAAKEMAAAAA|3145|0|52|25|AM|third|night|| +3146|AAAAAAAALEMAAAAA|3146|0|52|26|AM|third|night|| +3147|AAAAAAAAMEMAAAAA|3147|0|52|27|AM|third|night|| +3148|AAAAAAAANEMAAAAA|3148|0|52|28|AM|third|night|| +3149|AAAAAAAAOEMAAAAA|3149|0|52|29|AM|third|night|| +3150|AAAAAAAAPEMAAAAA|3150|0|52|30|AM|third|night|| +3151|AAAAAAAAAFMAAAAA|3151|0|52|31|AM|third|night|| +3152|AAAAAAAABFMAAAAA|3152|0|52|32|AM|third|night|| +3153|AAAAAAAACFMAAAAA|3153|0|52|33|AM|third|night|| +3154|AAAAAAAADFMAAAAA|3154|0|52|34|AM|third|night|| +3155|AAAAAAAAEFMAAAAA|3155|0|52|35|AM|third|night|| +3156|AAAAAAAAFFMAAAAA|3156|0|52|36|AM|third|night|| +3157|AAAAAAAAGFMAAAAA|3157|0|52|37|AM|third|night|| +3158|AAAAAAAAHFMAAAAA|3158|0|52|38|AM|third|night|| +3159|AAAAAAAAIFMAAAAA|3159|0|52|39|AM|third|night|| +3160|AAAAAAAAJFMAAAAA|3160|0|52|40|AM|third|night|| +3161|AAAAAAAAKFMAAAAA|3161|0|52|41|AM|third|night|| +3162|AAAAAAAALFMAAAAA|3162|0|52|42|AM|third|night|| +3163|AAAAAAAAMFMAAAAA|3163|0|52|43|AM|third|night|| +3164|AAAAAAAANFMAAAAA|3164|0|52|44|AM|third|night|| +3165|AAAAAAAAOFMAAAAA|3165|0|52|45|AM|third|night|| +3166|AAAAAAAAPFMAAAAA|3166|0|52|46|AM|third|night|| +3167|AAAAAAAAAGMAAAAA|3167|0|52|47|AM|third|night|| +3168|AAAAAAAABGMAAAAA|3168|0|52|48|AM|third|night|| +3169|AAAAAAAACGMAAAAA|3169|0|52|49|AM|third|night|| +3170|AAAAAAAADGMAAAAA|3170|0|52|50|AM|third|night|| +3171|AAAAAAAAEGMAAAAA|3171|0|52|51|AM|third|night|| +3172|AAAAAAAAFGMAAAAA|3172|0|52|52|AM|third|night|| +3173|AAAAAAAAGGMAAAAA|3173|0|52|53|AM|third|night|| +3174|AAAAAAAAHGMAAAAA|3174|0|52|54|AM|third|night|| +3175|AAAAAAAAIGMAAAAA|3175|0|52|55|AM|third|night|| +3176|AAAAAAAAJGMAAAAA|3176|0|52|56|AM|third|night|| +3177|AAAAAAAAKGMAAAAA|3177|0|52|57|AM|third|night|| +3178|AAAAAAAALGMAAAAA|3178|0|52|58|AM|third|night|| +3179|AAAAAAAAMGMAAAAA|3179|0|52|59|AM|third|night|| +3180|AAAAAAAANGMAAAAA|3180|0|53|0|AM|third|night|| +3181|AAAAAAAAOGMAAAAA|3181|0|53|1|AM|third|night|| +3182|AAAAAAAAPGMAAAAA|3182|0|53|2|AM|third|night|| +3183|AAAAAAAAAHMAAAAA|3183|0|53|3|AM|third|night|| +3184|AAAAAAAABHMAAAAA|3184|0|53|4|AM|third|night|| +3185|AAAAAAAACHMAAAAA|3185|0|53|5|AM|third|night|| +3186|AAAAAAAADHMAAAAA|3186|0|53|6|AM|third|night|| +3187|AAAAAAAAEHMAAAAA|3187|0|53|7|AM|third|night|| +3188|AAAAAAAAFHMAAAAA|3188|0|53|8|AM|third|night|| +3189|AAAAAAAAGHMAAAAA|3189|0|53|9|AM|third|night|| +3190|AAAAAAAAHHMAAAAA|3190|0|53|10|AM|third|night|| +3191|AAAAAAAAIHMAAAAA|3191|0|53|11|AM|third|night|| +3192|AAAAAAAAJHMAAAAA|3192|0|53|12|AM|third|night|| +3193|AAAAAAAAKHMAAAAA|3193|0|53|13|AM|third|night|| +3194|AAAAAAAALHMAAAAA|3194|0|53|14|AM|third|night|| +3195|AAAAAAAAMHMAAAAA|3195|0|53|15|AM|third|night|| +3196|AAAAAAAANHMAAAAA|3196|0|53|16|AM|third|night|| +3197|AAAAAAAAOHMAAAAA|3197|0|53|17|AM|third|night|| +3198|AAAAAAAAPHMAAAAA|3198|0|53|18|AM|third|night|| +3199|AAAAAAAAAIMAAAAA|3199|0|53|19|AM|third|night|| +3200|AAAAAAAABIMAAAAA|3200|0|53|20|AM|third|night|| +3201|AAAAAAAACIMAAAAA|3201|0|53|21|AM|third|night|| +3202|AAAAAAAADIMAAAAA|3202|0|53|22|AM|third|night|| +3203|AAAAAAAAEIMAAAAA|3203|0|53|23|AM|third|night|| +3204|AAAAAAAAFIMAAAAA|3204|0|53|24|AM|third|night|| +3205|AAAAAAAAGIMAAAAA|3205|0|53|25|AM|third|night|| +3206|AAAAAAAAHIMAAAAA|3206|0|53|26|AM|third|night|| +3207|AAAAAAAAIIMAAAAA|3207|0|53|27|AM|third|night|| +3208|AAAAAAAAJIMAAAAA|3208|0|53|28|AM|third|night|| +3209|AAAAAAAAKIMAAAAA|3209|0|53|29|AM|third|night|| +3210|AAAAAAAALIMAAAAA|3210|0|53|30|AM|third|night|| +3211|AAAAAAAAMIMAAAAA|3211|0|53|31|AM|third|night|| +3212|AAAAAAAANIMAAAAA|3212|0|53|32|AM|third|night|| +3213|AAAAAAAAOIMAAAAA|3213|0|53|33|AM|third|night|| +3214|AAAAAAAAPIMAAAAA|3214|0|53|34|AM|third|night|| +3215|AAAAAAAAAJMAAAAA|3215|0|53|35|AM|third|night|| +3216|AAAAAAAABJMAAAAA|3216|0|53|36|AM|third|night|| +3217|AAAAAAAACJMAAAAA|3217|0|53|37|AM|third|night|| +3218|AAAAAAAADJMAAAAA|3218|0|53|38|AM|third|night|| +3219|AAAAAAAAEJMAAAAA|3219|0|53|39|AM|third|night|| +3220|AAAAAAAAFJMAAAAA|3220|0|53|40|AM|third|night|| +3221|AAAAAAAAGJMAAAAA|3221|0|53|41|AM|third|night|| +3222|AAAAAAAAHJMAAAAA|3222|0|53|42|AM|third|night|| +3223|AAAAAAAAIJMAAAAA|3223|0|53|43|AM|third|night|| +3224|AAAAAAAAJJMAAAAA|3224|0|53|44|AM|third|night|| +3225|AAAAAAAAKJMAAAAA|3225|0|53|45|AM|third|night|| +3226|AAAAAAAALJMAAAAA|3226|0|53|46|AM|third|night|| +3227|AAAAAAAAMJMAAAAA|3227|0|53|47|AM|third|night|| +3228|AAAAAAAANJMAAAAA|3228|0|53|48|AM|third|night|| +3229|AAAAAAAAOJMAAAAA|3229|0|53|49|AM|third|night|| +3230|AAAAAAAAPJMAAAAA|3230|0|53|50|AM|third|night|| +3231|AAAAAAAAAKMAAAAA|3231|0|53|51|AM|third|night|| +3232|AAAAAAAABKMAAAAA|3232|0|53|52|AM|third|night|| +3233|AAAAAAAACKMAAAAA|3233|0|53|53|AM|third|night|| +3234|AAAAAAAADKMAAAAA|3234|0|53|54|AM|third|night|| +3235|AAAAAAAAEKMAAAAA|3235|0|53|55|AM|third|night|| +3236|AAAAAAAAFKMAAAAA|3236|0|53|56|AM|third|night|| +3237|AAAAAAAAGKMAAAAA|3237|0|53|57|AM|third|night|| +3238|AAAAAAAAHKMAAAAA|3238|0|53|58|AM|third|night|| +3239|AAAAAAAAIKMAAAAA|3239|0|53|59|AM|third|night|| +3240|AAAAAAAAJKMAAAAA|3240|0|54|0|AM|third|night|| +3241|AAAAAAAAKKMAAAAA|3241|0|54|1|AM|third|night|| +3242|AAAAAAAALKMAAAAA|3242|0|54|2|AM|third|night|| +3243|AAAAAAAAMKMAAAAA|3243|0|54|3|AM|third|night|| +3244|AAAAAAAANKMAAAAA|3244|0|54|4|AM|third|night|| +3245|AAAAAAAAOKMAAAAA|3245|0|54|5|AM|third|night|| +3246|AAAAAAAAPKMAAAAA|3246|0|54|6|AM|third|night|| +3247|AAAAAAAAALMAAAAA|3247|0|54|7|AM|third|night|| +3248|AAAAAAAABLMAAAAA|3248|0|54|8|AM|third|night|| +3249|AAAAAAAACLMAAAAA|3249|0|54|9|AM|third|night|| +3250|AAAAAAAADLMAAAAA|3250|0|54|10|AM|third|night|| +3251|AAAAAAAAELMAAAAA|3251|0|54|11|AM|third|night|| +3252|AAAAAAAAFLMAAAAA|3252|0|54|12|AM|third|night|| +3253|AAAAAAAAGLMAAAAA|3253|0|54|13|AM|third|night|| +3254|AAAAAAAAHLMAAAAA|3254|0|54|14|AM|third|night|| +3255|AAAAAAAAILMAAAAA|3255|0|54|15|AM|third|night|| +3256|AAAAAAAAJLMAAAAA|3256|0|54|16|AM|third|night|| +3257|AAAAAAAAKLMAAAAA|3257|0|54|17|AM|third|night|| +3258|AAAAAAAALLMAAAAA|3258|0|54|18|AM|third|night|| +3259|AAAAAAAAMLMAAAAA|3259|0|54|19|AM|third|night|| +3260|AAAAAAAANLMAAAAA|3260|0|54|20|AM|third|night|| +3261|AAAAAAAAOLMAAAAA|3261|0|54|21|AM|third|night|| +3262|AAAAAAAAPLMAAAAA|3262|0|54|22|AM|third|night|| +3263|AAAAAAAAAMMAAAAA|3263|0|54|23|AM|third|night|| +3264|AAAAAAAABMMAAAAA|3264|0|54|24|AM|third|night|| +3265|AAAAAAAACMMAAAAA|3265|0|54|25|AM|third|night|| +3266|AAAAAAAADMMAAAAA|3266|0|54|26|AM|third|night|| +3267|AAAAAAAAEMMAAAAA|3267|0|54|27|AM|third|night|| +3268|AAAAAAAAFMMAAAAA|3268|0|54|28|AM|third|night|| +3269|AAAAAAAAGMMAAAAA|3269|0|54|29|AM|third|night|| +3270|AAAAAAAAHMMAAAAA|3270|0|54|30|AM|third|night|| +3271|AAAAAAAAIMMAAAAA|3271|0|54|31|AM|third|night|| +3272|AAAAAAAAJMMAAAAA|3272|0|54|32|AM|third|night|| +3273|AAAAAAAAKMMAAAAA|3273|0|54|33|AM|third|night|| +3274|AAAAAAAALMMAAAAA|3274|0|54|34|AM|third|night|| +3275|AAAAAAAAMMMAAAAA|3275|0|54|35|AM|third|night|| +3276|AAAAAAAANMMAAAAA|3276|0|54|36|AM|third|night|| +3277|AAAAAAAAOMMAAAAA|3277|0|54|37|AM|third|night|| +3278|AAAAAAAAPMMAAAAA|3278|0|54|38|AM|third|night|| +3279|AAAAAAAAANMAAAAA|3279|0|54|39|AM|third|night|| +3280|AAAAAAAABNMAAAAA|3280|0|54|40|AM|third|night|| +3281|AAAAAAAACNMAAAAA|3281|0|54|41|AM|third|night|| +3282|AAAAAAAADNMAAAAA|3282|0|54|42|AM|third|night|| +3283|AAAAAAAAENMAAAAA|3283|0|54|43|AM|third|night|| +3284|AAAAAAAAFNMAAAAA|3284|0|54|44|AM|third|night|| +3285|AAAAAAAAGNMAAAAA|3285|0|54|45|AM|third|night|| +3286|AAAAAAAAHNMAAAAA|3286|0|54|46|AM|third|night|| +3287|AAAAAAAAINMAAAAA|3287|0|54|47|AM|third|night|| +3288|AAAAAAAAJNMAAAAA|3288|0|54|48|AM|third|night|| +3289|AAAAAAAAKNMAAAAA|3289|0|54|49|AM|third|night|| +3290|AAAAAAAALNMAAAAA|3290|0|54|50|AM|third|night|| +3291|AAAAAAAAMNMAAAAA|3291|0|54|51|AM|third|night|| +3292|AAAAAAAANNMAAAAA|3292|0|54|52|AM|third|night|| +3293|AAAAAAAAONMAAAAA|3293|0|54|53|AM|third|night|| +3294|AAAAAAAAPNMAAAAA|3294|0|54|54|AM|third|night|| +3295|AAAAAAAAAOMAAAAA|3295|0|54|55|AM|third|night|| +3296|AAAAAAAABOMAAAAA|3296|0|54|56|AM|third|night|| +3297|AAAAAAAACOMAAAAA|3297|0|54|57|AM|third|night|| +3298|AAAAAAAADOMAAAAA|3298|0|54|58|AM|third|night|| +3299|AAAAAAAAEOMAAAAA|3299|0|54|59|AM|third|night|| +3300|AAAAAAAAFOMAAAAA|3300|0|55|0|AM|third|night|| +3301|AAAAAAAAGOMAAAAA|3301|0|55|1|AM|third|night|| +3302|AAAAAAAAHOMAAAAA|3302|0|55|2|AM|third|night|| +3303|AAAAAAAAIOMAAAAA|3303|0|55|3|AM|third|night|| +3304|AAAAAAAAJOMAAAAA|3304|0|55|4|AM|third|night|| +3305|AAAAAAAAKOMAAAAA|3305|0|55|5|AM|third|night|| +3306|AAAAAAAALOMAAAAA|3306|0|55|6|AM|third|night|| +3307|AAAAAAAAMOMAAAAA|3307|0|55|7|AM|third|night|| +3308|AAAAAAAANOMAAAAA|3308|0|55|8|AM|third|night|| +3309|AAAAAAAAOOMAAAAA|3309|0|55|9|AM|third|night|| +3310|AAAAAAAAPOMAAAAA|3310|0|55|10|AM|third|night|| +3311|AAAAAAAAAPMAAAAA|3311|0|55|11|AM|third|night|| +3312|AAAAAAAABPMAAAAA|3312|0|55|12|AM|third|night|| +3313|AAAAAAAACPMAAAAA|3313|0|55|13|AM|third|night|| +3314|AAAAAAAADPMAAAAA|3314|0|55|14|AM|third|night|| +3315|AAAAAAAAEPMAAAAA|3315|0|55|15|AM|third|night|| +3316|AAAAAAAAFPMAAAAA|3316|0|55|16|AM|third|night|| +3317|AAAAAAAAGPMAAAAA|3317|0|55|17|AM|third|night|| +3318|AAAAAAAAHPMAAAAA|3318|0|55|18|AM|third|night|| +3319|AAAAAAAAIPMAAAAA|3319|0|55|19|AM|third|night|| +3320|AAAAAAAAJPMAAAAA|3320|0|55|20|AM|third|night|| +3321|AAAAAAAAKPMAAAAA|3321|0|55|21|AM|third|night|| +3322|AAAAAAAALPMAAAAA|3322|0|55|22|AM|third|night|| +3323|AAAAAAAAMPMAAAAA|3323|0|55|23|AM|third|night|| +3324|AAAAAAAANPMAAAAA|3324|0|55|24|AM|third|night|| +3325|AAAAAAAAOPMAAAAA|3325|0|55|25|AM|third|night|| +3326|AAAAAAAAPPMAAAAA|3326|0|55|26|AM|third|night|| +3327|AAAAAAAAAANAAAAA|3327|0|55|27|AM|third|night|| +3328|AAAAAAAABANAAAAA|3328|0|55|28|AM|third|night|| +3329|AAAAAAAACANAAAAA|3329|0|55|29|AM|third|night|| +3330|AAAAAAAADANAAAAA|3330|0|55|30|AM|third|night|| +3331|AAAAAAAAEANAAAAA|3331|0|55|31|AM|third|night|| +3332|AAAAAAAAFANAAAAA|3332|0|55|32|AM|third|night|| +3333|AAAAAAAAGANAAAAA|3333|0|55|33|AM|third|night|| +3334|AAAAAAAAHANAAAAA|3334|0|55|34|AM|third|night|| +3335|AAAAAAAAIANAAAAA|3335|0|55|35|AM|third|night|| +3336|AAAAAAAAJANAAAAA|3336|0|55|36|AM|third|night|| +3337|AAAAAAAAKANAAAAA|3337|0|55|37|AM|third|night|| +3338|AAAAAAAALANAAAAA|3338|0|55|38|AM|third|night|| +3339|AAAAAAAAMANAAAAA|3339|0|55|39|AM|third|night|| +3340|AAAAAAAANANAAAAA|3340|0|55|40|AM|third|night|| +3341|AAAAAAAAOANAAAAA|3341|0|55|41|AM|third|night|| +3342|AAAAAAAAPANAAAAA|3342|0|55|42|AM|third|night|| +3343|AAAAAAAAABNAAAAA|3343|0|55|43|AM|third|night|| +3344|AAAAAAAABBNAAAAA|3344|0|55|44|AM|third|night|| +3345|AAAAAAAACBNAAAAA|3345|0|55|45|AM|third|night|| +3346|AAAAAAAADBNAAAAA|3346|0|55|46|AM|third|night|| +3347|AAAAAAAAEBNAAAAA|3347|0|55|47|AM|third|night|| +3348|AAAAAAAAFBNAAAAA|3348|0|55|48|AM|third|night|| +3349|AAAAAAAAGBNAAAAA|3349|0|55|49|AM|third|night|| +3350|AAAAAAAAHBNAAAAA|3350|0|55|50|AM|third|night|| +3351|AAAAAAAAIBNAAAAA|3351|0|55|51|AM|third|night|| +3352|AAAAAAAAJBNAAAAA|3352|0|55|52|AM|third|night|| +3353|AAAAAAAAKBNAAAAA|3353|0|55|53|AM|third|night|| +3354|AAAAAAAALBNAAAAA|3354|0|55|54|AM|third|night|| +3355|AAAAAAAAMBNAAAAA|3355|0|55|55|AM|third|night|| +3356|AAAAAAAANBNAAAAA|3356|0|55|56|AM|third|night|| +3357|AAAAAAAAOBNAAAAA|3357|0|55|57|AM|third|night|| +3358|AAAAAAAAPBNAAAAA|3358|0|55|58|AM|third|night|| +3359|AAAAAAAAACNAAAAA|3359|0|55|59|AM|third|night|| +3360|AAAAAAAABCNAAAAA|3360|0|56|0|AM|third|night|| +3361|AAAAAAAACCNAAAAA|3361|0|56|1|AM|third|night|| +3362|AAAAAAAADCNAAAAA|3362|0|56|2|AM|third|night|| +3363|AAAAAAAAECNAAAAA|3363|0|56|3|AM|third|night|| +3364|AAAAAAAAFCNAAAAA|3364|0|56|4|AM|third|night|| +3365|AAAAAAAAGCNAAAAA|3365|0|56|5|AM|third|night|| +3366|AAAAAAAAHCNAAAAA|3366|0|56|6|AM|third|night|| +3367|AAAAAAAAICNAAAAA|3367|0|56|7|AM|third|night|| +3368|AAAAAAAAJCNAAAAA|3368|0|56|8|AM|third|night|| +3369|AAAAAAAAKCNAAAAA|3369|0|56|9|AM|third|night|| +3370|AAAAAAAALCNAAAAA|3370|0|56|10|AM|third|night|| +3371|AAAAAAAAMCNAAAAA|3371|0|56|11|AM|third|night|| +3372|AAAAAAAANCNAAAAA|3372|0|56|12|AM|third|night|| +3373|AAAAAAAAOCNAAAAA|3373|0|56|13|AM|third|night|| +3374|AAAAAAAAPCNAAAAA|3374|0|56|14|AM|third|night|| +3375|AAAAAAAAADNAAAAA|3375|0|56|15|AM|third|night|| +3376|AAAAAAAABDNAAAAA|3376|0|56|16|AM|third|night|| +3377|AAAAAAAACDNAAAAA|3377|0|56|17|AM|third|night|| +3378|AAAAAAAADDNAAAAA|3378|0|56|18|AM|third|night|| +3379|AAAAAAAAEDNAAAAA|3379|0|56|19|AM|third|night|| +3380|AAAAAAAAFDNAAAAA|3380|0|56|20|AM|third|night|| +3381|AAAAAAAAGDNAAAAA|3381|0|56|21|AM|third|night|| +3382|AAAAAAAAHDNAAAAA|3382|0|56|22|AM|third|night|| +3383|AAAAAAAAIDNAAAAA|3383|0|56|23|AM|third|night|| +3384|AAAAAAAAJDNAAAAA|3384|0|56|24|AM|third|night|| +3385|AAAAAAAAKDNAAAAA|3385|0|56|25|AM|third|night|| +3386|AAAAAAAALDNAAAAA|3386|0|56|26|AM|third|night|| +3387|AAAAAAAAMDNAAAAA|3387|0|56|27|AM|third|night|| +3388|AAAAAAAANDNAAAAA|3388|0|56|28|AM|third|night|| +3389|AAAAAAAAODNAAAAA|3389|0|56|29|AM|third|night|| +3390|AAAAAAAAPDNAAAAA|3390|0|56|30|AM|third|night|| +3391|AAAAAAAAAENAAAAA|3391|0|56|31|AM|third|night|| +3392|AAAAAAAABENAAAAA|3392|0|56|32|AM|third|night|| +3393|AAAAAAAACENAAAAA|3393|0|56|33|AM|third|night|| +3394|AAAAAAAADENAAAAA|3394|0|56|34|AM|third|night|| +3395|AAAAAAAAEENAAAAA|3395|0|56|35|AM|third|night|| +3396|AAAAAAAAFENAAAAA|3396|0|56|36|AM|third|night|| +3397|AAAAAAAAGENAAAAA|3397|0|56|37|AM|third|night|| +3398|AAAAAAAAHENAAAAA|3398|0|56|38|AM|third|night|| +3399|AAAAAAAAIENAAAAA|3399|0|56|39|AM|third|night|| +3400|AAAAAAAAJENAAAAA|3400|0|56|40|AM|third|night|| +3401|AAAAAAAAKENAAAAA|3401|0|56|41|AM|third|night|| +3402|AAAAAAAALENAAAAA|3402|0|56|42|AM|third|night|| +3403|AAAAAAAAMENAAAAA|3403|0|56|43|AM|third|night|| +3404|AAAAAAAANENAAAAA|3404|0|56|44|AM|third|night|| +3405|AAAAAAAAOENAAAAA|3405|0|56|45|AM|third|night|| +3406|AAAAAAAAPENAAAAA|3406|0|56|46|AM|third|night|| +3407|AAAAAAAAAFNAAAAA|3407|0|56|47|AM|third|night|| +3408|AAAAAAAABFNAAAAA|3408|0|56|48|AM|third|night|| +3409|AAAAAAAACFNAAAAA|3409|0|56|49|AM|third|night|| +3410|AAAAAAAADFNAAAAA|3410|0|56|50|AM|third|night|| +3411|AAAAAAAAEFNAAAAA|3411|0|56|51|AM|third|night|| +3412|AAAAAAAAFFNAAAAA|3412|0|56|52|AM|third|night|| +3413|AAAAAAAAGFNAAAAA|3413|0|56|53|AM|third|night|| +3414|AAAAAAAAHFNAAAAA|3414|0|56|54|AM|third|night|| +3415|AAAAAAAAIFNAAAAA|3415|0|56|55|AM|third|night|| +3416|AAAAAAAAJFNAAAAA|3416|0|56|56|AM|third|night|| +3417|AAAAAAAAKFNAAAAA|3417|0|56|57|AM|third|night|| +3418|AAAAAAAALFNAAAAA|3418|0|56|58|AM|third|night|| +3419|AAAAAAAAMFNAAAAA|3419|0|56|59|AM|third|night|| +3420|AAAAAAAANFNAAAAA|3420|0|57|0|AM|third|night|| +3421|AAAAAAAAOFNAAAAA|3421|0|57|1|AM|third|night|| +3422|AAAAAAAAPFNAAAAA|3422|0|57|2|AM|third|night|| +3423|AAAAAAAAAGNAAAAA|3423|0|57|3|AM|third|night|| +3424|AAAAAAAABGNAAAAA|3424|0|57|4|AM|third|night|| +3425|AAAAAAAACGNAAAAA|3425|0|57|5|AM|third|night|| +3426|AAAAAAAADGNAAAAA|3426|0|57|6|AM|third|night|| +3427|AAAAAAAAEGNAAAAA|3427|0|57|7|AM|third|night|| +3428|AAAAAAAAFGNAAAAA|3428|0|57|8|AM|third|night|| +3429|AAAAAAAAGGNAAAAA|3429|0|57|9|AM|third|night|| +3430|AAAAAAAAHGNAAAAA|3430|0|57|10|AM|third|night|| +3431|AAAAAAAAIGNAAAAA|3431|0|57|11|AM|third|night|| +3432|AAAAAAAAJGNAAAAA|3432|0|57|12|AM|third|night|| +3433|AAAAAAAAKGNAAAAA|3433|0|57|13|AM|third|night|| +3434|AAAAAAAALGNAAAAA|3434|0|57|14|AM|third|night|| +3435|AAAAAAAAMGNAAAAA|3435|0|57|15|AM|third|night|| +3436|AAAAAAAANGNAAAAA|3436|0|57|16|AM|third|night|| +3437|AAAAAAAAOGNAAAAA|3437|0|57|17|AM|third|night|| +3438|AAAAAAAAPGNAAAAA|3438|0|57|18|AM|third|night|| +3439|AAAAAAAAAHNAAAAA|3439|0|57|19|AM|third|night|| +3440|AAAAAAAABHNAAAAA|3440|0|57|20|AM|third|night|| +3441|AAAAAAAACHNAAAAA|3441|0|57|21|AM|third|night|| +3442|AAAAAAAADHNAAAAA|3442|0|57|22|AM|third|night|| +3443|AAAAAAAAEHNAAAAA|3443|0|57|23|AM|third|night|| +3444|AAAAAAAAFHNAAAAA|3444|0|57|24|AM|third|night|| +3445|AAAAAAAAGHNAAAAA|3445|0|57|25|AM|third|night|| +3446|AAAAAAAAHHNAAAAA|3446|0|57|26|AM|third|night|| +3447|AAAAAAAAIHNAAAAA|3447|0|57|27|AM|third|night|| +3448|AAAAAAAAJHNAAAAA|3448|0|57|28|AM|third|night|| +3449|AAAAAAAAKHNAAAAA|3449|0|57|29|AM|third|night|| +3450|AAAAAAAALHNAAAAA|3450|0|57|30|AM|third|night|| +3451|AAAAAAAAMHNAAAAA|3451|0|57|31|AM|third|night|| +3452|AAAAAAAANHNAAAAA|3452|0|57|32|AM|third|night|| +3453|AAAAAAAAOHNAAAAA|3453|0|57|33|AM|third|night|| +3454|AAAAAAAAPHNAAAAA|3454|0|57|34|AM|third|night|| +3455|AAAAAAAAAINAAAAA|3455|0|57|35|AM|third|night|| +3456|AAAAAAAABINAAAAA|3456|0|57|36|AM|third|night|| +3457|AAAAAAAACINAAAAA|3457|0|57|37|AM|third|night|| +3458|AAAAAAAADINAAAAA|3458|0|57|38|AM|third|night|| +3459|AAAAAAAAEINAAAAA|3459|0|57|39|AM|third|night|| +3460|AAAAAAAAFINAAAAA|3460|0|57|40|AM|third|night|| +3461|AAAAAAAAGINAAAAA|3461|0|57|41|AM|third|night|| +3462|AAAAAAAAHINAAAAA|3462|0|57|42|AM|third|night|| +3463|AAAAAAAAIINAAAAA|3463|0|57|43|AM|third|night|| +3464|AAAAAAAAJINAAAAA|3464|0|57|44|AM|third|night|| +3465|AAAAAAAAKINAAAAA|3465|0|57|45|AM|third|night|| +3466|AAAAAAAALINAAAAA|3466|0|57|46|AM|third|night|| +3467|AAAAAAAAMINAAAAA|3467|0|57|47|AM|third|night|| +3468|AAAAAAAANINAAAAA|3468|0|57|48|AM|third|night|| +3469|AAAAAAAAOINAAAAA|3469|0|57|49|AM|third|night|| +3470|AAAAAAAAPINAAAAA|3470|0|57|50|AM|third|night|| +3471|AAAAAAAAAJNAAAAA|3471|0|57|51|AM|third|night|| +3472|AAAAAAAABJNAAAAA|3472|0|57|52|AM|third|night|| +3473|AAAAAAAACJNAAAAA|3473|0|57|53|AM|third|night|| +3474|AAAAAAAADJNAAAAA|3474|0|57|54|AM|third|night|| +3475|AAAAAAAAEJNAAAAA|3475|0|57|55|AM|third|night|| +3476|AAAAAAAAFJNAAAAA|3476|0|57|56|AM|third|night|| +3477|AAAAAAAAGJNAAAAA|3477|0|57|57|AM|third|night|| +3478|AAAAAAAAHJNAAAAA|3478|0|57|58|AM|third|night|| +3479|AAAAAAAAIJNAAAAA|3479|0|57|59|AM|third|night|| +3480|AAAAAAAAJJNAAAAA|3480|0|58|0|AM|third|night|| +3481|AAAAAAAAKJNAAAAA|3481|0|58|1|AM|third|night|| +3482|AAAAAAAALJNAAAAA|3482|0|58|2|AM|third|night|| +3483|AAAAAAAAMJNAAAAA|3483|0|58|3|AM|third|night|| +3484|AAAAAAAANJNAAAAA|3484|0|58|4|AM|third|night|| +3485|AAAAAAAAOJNAAAAA|3485|0|58|5|AM|third|night|| +3486|AAAAAAAAPJNAAAAA|3486|0|58|6|AM|third|night|| +3487|AAAAAAAAAKNAAAAA|3487|0|58|7|AM|third|night|| +3488|AAAAAAAABKNAAAAA|3488|0|58|8|AM|third|night|| +3489|AAAAAAAACKNAAAAA|3489|0|58|9|AM|third|night|| +3490|AAAAAAAADKNAAAAA|3490|0|58|10|AM|third|night|| +3491|AAAAAAAAEKNAAAAA|3491|0|58|11|AM|third|night|| +3492|AAAAAAAAFKNAAAAA|3492|0|58|12|AM|third|night|| +3493|AAAAAAAAGKNAAAAA|3493|0|58|13|AM|third|night|| +3494|AAAAAAAAHKNAAAAA|3494|0|58|14|AM|third|night|| +3495|AAAAAAAAIKNAAAAA|3495|0|58|15|AM|third|night|| +3496|AAAAAAAAJKNAAAAA|3496|0|58|16|AM|third|night|| +3497|AAAAAAAAKKNAAAAA|3497|0|58|17|AM|third|night|| +3498|AAAAAAAALKNAAAAA|3498|0|58|18|AM|third|night|| +3499|AAAAAAAAMKNAAAAA|3499|0|58|19|AM|third|night|| +3500|AAAAAAAANKNAAAAA|3500|0|58|20|AM|third|night|| +3501|AAAAAAAAOKNAAAAA|3501|0|58|21|AM|third|night|| +3502|AAAAAAAAPKNAAAAA|3502|0|58|22|AM|third|night|| +3503|AAAAAAAAALNAAAAA|3503|0|58|23|AM|third|night|| +3504|AAAAAAAABLNAAAAA|3504|0|58|24|AM|third|night|| +3505|AAAAAAAACLNAAAAA|3505|0|58|25|AM|third|night|| +3506|AAAAAAAADLNAAAAA|3506|0|58|26|AM|third|night|| +3507|AAAAAAAAELNAAAAA|3507|0|58|27|AM|third|night|| +3508|AAAAAAAAFLNAAAAA|3508|0|58|28|AM|third|night|| +3509|AAAAAAAAGLNAAAAA|3509|0|58|29|AM|third|night|| +3510|AAAAAAAAHLNAAAAA|3510|0|58|30|AM|third|night|| +3511|AAAAAAAAILNAAAAA|3511|0|58|31|AM|third|night|| +3512|AAAAAAAAJLNAAAAA|3512|0|58|32|AM|third|night|| +3513|AAAAAAAAKLNAAAAA|3513|0|58|33|AM|third|night|| +3514|AAAAAAAALLNAAAAA|3514|0|58|34|AM|third|night|| +3515|AAAAAAAAMLNAAAAA|3515|0|58|35|AM|third|night|| +3516|AAAAAAAANLNAAAAA|3516|0|58|36|AM|third|night|| +3517|AAAAAAAAOLNAAAAA|3517|0|58|37|AM|third|night|| +3518|AAAAAAAAPLNAAAAA|3518|0|58|38|AM|third|night|| +3519|AAAAAAAAAMNAAAAA|3519|0|58|39|AM|third|night|| +3520|AAAAAAAABMNAAAAA|3520|0|58|40|AM|third|night|| +3521|AAAAAAAACMNAAAAA|3521|0|58|41|AM|third|night|| +3522|AAAAAAAADMNAAAAA|3522|0|58|42|AM|third|night|| +3523|AAAAAAAAEMNAAAAA|3523|0|58|43|AM|third|night|| +3524|AAAAAAAAFMNAAAAA|3524|0|58|44|AM|third|night|| +3525|AAAAAAAAGMNAAAAA|3525|0|58|45|AM|third|night|| +3526|AAAAAAAAHMNAAAAA|3526|0|58|46|AM|third|night|| +3527|AAAAAAAAIMNAAAAA|3527|0|58|47|AM|third|night|| +3528|AAAAAAAAJMNAAAAA|3528|0|58|48|AM|third|night|| +3529|AAAAAAAAKMNAAAAA|3529|0|58|49|AM|third|night|| +3530|AAAAAAAALMNAAAAA|3530|0|58|50|AM|third|night|| +3531|AAAAAAAAMMNAAAAA|3531|0|58|51|AM|third|night|| +3532|AAAAAAAANMNAAAAA|3532|0|58|52|AM|third|night|| +3533|AAAAAAAAOMNAAAAA|3533|0|58|53|AM|third|night|| +3534|AAAAAAAAPMNAAAAA|3534|0|58|54|AM|third|night|| +3535|AAAAAAAAANNAAAAA|3535|0|58|55|AM|third|night|| +3536|AAAAAAAABNNAAAAA|3536|0|58|56|AM|third|night|| +3537|AAAAAAAACNNAAAAA|3537|0|58|57|AM|third|night|| +3538|AAAAAAAADNNAAAAA|3538|0|58|58|AM|third|night|| +3539|AAAAAAAAENNAAAAA|3539|0|58|59|AM|third|night|| +3540|AAAAAAAAFNNAAAAA|3540|0|59|0|AM|third|night|| +3541|AAAAAAAAGNNAAAAA|3541|0|59|1|AM|third|night|| +3542|AAAAAAAAHNNAAAAA|3542|0|59|2|AM|third|night|| +3543|AAAAAAAAINNAAAAA|3543|0|59|3|AM|third|night|| +3544|AAAAAAAAJNNAAAAA|3544|0|59|4|AM|third|night|| +3545|AAAAAAAAKNNAAAAA|3545|0|59|5|AM|third|night|| +3546|AAAAAAAALNNAAAAA|3546|0|59|6|AM|third|night|| +3547|AAAAAAAAMNNAAAAA|3547|0|59|7|AM|third|night|| +3548|AAAAAAAANNNAAAAA|3548|0|59|8|AM|third|night|| +3549|AAAAAAAAONNAAAAA|3549|0|59|9|AM|third|night|| +3550|AAAAAAAAPNNAAAAA|3550|0|59|10|AM|third|night|| +3551|AAAAAAAAAONAAAAA|3551|0|59|11|AM|third|night|| +3552|AAAAAAAABONAAAAA|3552|0|59|12|AM|third|night|| +3553|AAAAAAAACONAAAAA|3553|0|59|13|AM|third|night|| +3554|AAAAAAAADONAAAAA|3554|0|59|14|AM|third|night|| +3555|AAAAAAAAEONAAAAA|3555|0|59|15|AM|third|night|| +3556|AAAAAAAAFONAAAAA|3556|0|59|16|AM|third|night|| +3557|AAAAAAAAGONAAAAA|3557|0|59|17|AM|third|night|| +3558|AAAAAAAAHONAAAAA|3558|0|59|18|AM|third|night|| +3559|AAAAAAAAIONAAAAA|3559|0|59|19|AM|third|night|| +3560|AAAAAAAAJONAAAAA|3560|0|59|20|AM|third|night|| +3561|AAAAAAAAKONAAAAA|3561|0|59|21|AM|third|night|| +3562|AAAAAAAALONAAAAA|3562|0|59|22|AM|third|night|| +3563|AAAAAAAAMONAAAAA|3563|0|59|23|AM|third|night|| +3564|AAAAAAAANONAAAAA|3564|0|59|24|AM|third|night|| +3565|AAAAAAAAOONAAAAA|3565|0|59|25|AM|third|night|| +3566|AAAAAAAAPONAAAAA|3566|0|59|26|AM|third|night|| +3567|AAAAAAAAAPNAAAAA|3567|0|59|27|AM|third|night|| +3568|AAAAAAAABPNAAAAA|3568|0|59|28|AM|third|night|| +3569|AAAAAAAACPNAAAAA|3569|0|59|29|AM|third|night|| +3570|AAAAAAAADPNAAAAA|3570|0|59|30|AM|third|night|| +3571|AAAAAAAAEPNAAAAA|3571|0|59|31|AM|third|night|| +3572|AAAAAAAAFPNAAAAA|3572|0|59|32|AM|third|night|| +3573|AAAAAAAAGPNAAAAA|3573|0|59|33|AM|third|night|| +3574|AAAAAAAAHPNAAAAA|3574|0|59|34|AM|third|night|| +3575|AAAAAAAAIPNAAAAA|3575|0|59|35|AM|third|night|| +3576|AAAAAAAAJPNAAAAA|3576|0|59|36|AM|third|night|| +3577|AAAAAAAAKPNAAAAA|3577|0|59|37|AM|third|night|| +3578|AAAAAAAALPNAAAAA|3578|0|59|38|AM|third|night|| +3579|AAAAAAAAMPNAAAAA|3579|0|59|39|AM|third|night|| +3580|AAAAAAAANPNAAAAA|3580|0|59|40|AM|third|night|| +3581|AAAAAAAAOPNAAAAA|3581|0|59|41|AM|third|night|| +3582|AAAAAAAAPPNAAAAA|3582|0|59|42|AM|third|night|| +3583|AAAAAAAAAAOAAAAA|3583|0|59|43|AM|third|night|| +3584|AAAAAAAABAOAAAAA|3584|0|59|44|AM|third|night|| +3585|AAAAAAAACAOAAAAA|3585|0|59|45|AM|third|night|| +3586|AAAAAAAADAOAAAAA|3586|0|59|46|AM|third|night|| +3587|AAAAAAAAEAOAAAAA|3587|0|59|47|AM|third|night|| +3588|AAAAAAAAFAOAAAAA|3588|0|59|48|AM|third|night|| +3589|AAAAAAAAGAOAAAAA|3589|0|59|49|AM|third|night|| +3590|AAAAAAAAHAOAAAAA|3590|0|59|50|AM|third|night|| +3591|AAAAAAAAIAOAAAAA|3591|0|59|51|AM|third|night|| +3592|AAAAAAAAJAOAAAAA|3592|0|59|52|AM|third|night|| +3593|AAAAAAAAKAOAAAAA|3593|0|59|53|AM|third|night|| +3594|AAAAAAAALAOAAAAA|3594|0|59|54|AM|third|night|| +3595|AAAAAAAAMAOAAAAA|3595|0|59|55|AM|third|night|| +3596|AAAAAAAANAOAAAAA|3596|0|59|56|AM|third|night|| +3597|AAAAAAAAOAOAAAAA|3597|0|59|57|AM|third|night|| +3598|AAAAAAAAPAOAAAAA|3598|0|59|58|AM|third|night|| +3599|AAAAAAAAABOAAAAA|3599|0|59|59|AM|third|night|| +3600|AAAAAAAABBOAAAAA|3600|1|0|0|AM|third|night|| +3601|AAAAAAAACBOAAAAA|3601|1|0|1|AM|third|night|| +3602|AAAAAAAADBOAAAAA|3602|1|0|2|AM|third|night|| +3603|AAAAAAAAEBOAAAAA|3603|1|0|3|AM|third|night|| +3604|AAAAAAAAFBOAAAAA|3604|1|0|4|AM|third|night|| +3605|AAAAAAAAGBOAAAAA|3605|1|0|5|AM|third|night|| +3606|AAAAAAAAHBOAAAAA|3606|1|0|6|AM|third|night|| +3607|AAAAAAAAIBOAAAAA|3607|1|0|7|AM|third|night|| +3608|AAAAAAAAJBOAAAAA|3608|1|0|8|AM|third|night|| +3609|AAAAAAAAKBOAAAAA|3609|1|0|9|AM|third|night|| +3610|AAAAAAAALBOAAAAA|3610|1|0|10|AM|third|night|| +3611|AAAAAAAAMBOAAAAA|3611|1|0|11|AM|third|night|| +3612|AAAAAAAANBOAAAAA|3612|1|0|12|AM|third|night|| +3613|AAAAAAAAOBOAAAAA|3613|1|0|13|AM|third|night|| +3614|AAAAAAAAPBOAAAAA|3614|1|0|14|AM|third|night|| +3615|AAAAAAAAACOAAAAA|3615|1|0|15|AM|third|night|| +3616|AAAAAAAABCOAAAAA|3616|1|0|16|AM|third|night|| +3617|AAAAAAAACCOAAAAA|3617|1|0|17|AM|third|night|| +3618|AAAAAAAADCOAAAAA|3618|1|0|18|AM|third|night|| +3619|AAAAAAAAECOAAAAA|3619|1|0|19|AM|third|night|| +3620|AAAAAAAAFCOAAAAA|3620|1|0|20|AM|third|night|| +3621|AAAAAAAAGCOAAAAA|3621|1|0|21|AM|third|night|| +3622|AAAAAAAAHCOAAAAA|3622|1|0|22|AM|third|night|| +3623|AAAAAAAAICOAAAAA|3623|1|0|23|AM|third|night|| +3624|AAAAAAAAJCOAAAAA|3624|1|0|24|AM|third|night|| +3625|AAAAAAAAKCOAAAAA|3625|1|0|25|AM|third|night|| +3626|AAAAAAAALCOAAAAA|3626|1|0|26|AM|third|night|| +3627|AAAAAAAAMCOAAAAA|3627|1|0|27|AM|third|night|| +3628|AAAAAAAANCOAAAAA|3628|1|0|28|AM|third|night|| +3629|AAAAAAAAOCOAAAAA|3629|1|0|29|AM|third|night|| +3630|AAAAAAAAPCOAAAAA|3630|1|0|30|AM|third|night|| +3631|AAAAAAAAADOAAAAA|3631|1|0|31|AM|third|night|| +3632|AAAAAAAABDOAAAAA|3632|1|0|32|AM|third|night|| +3633|AAAAAAAACDOAAAAA|3633|1|0|33|AM|third|night|| +3634|AAAAAAAADDOAAAAA|3634|1|0|34|AM|third|night|| +3635|AAAAAAAAEDOAAAAA|3635|1|0|35|AM|third|night|| +3636|AAAAAAAAFDOAAAAA|3636|1|0|36|AM|third|night|| +3637|AAAAAAAAGDOAAAAA|3637|1|0|37|AM|third|night|| +3638|AAAAAAAAHDOAAAAA|3638|1|0|38|AM|third|night|| +3639|AAAAAAAAIDOAAAAA|3639|1|0|39|AM|third|night|| +3640|AAAAAAAAJDOAAAAA|3640|1|0|40|AM|third|night|| +3641|AAAAAAAAKDOAAAAA|3641|1|0|41|AM|third|night|| +3642|AAAAAAAALDOAAAAA|3642|1|0|42|AM|third|night|| +3643|AAAAAAAAMDOAAAAA|3643|1|0|43|AM|third|night|| +3644|AAAAAAAANDOAAAAA|3644|1|0|44|AM|third|night|| +3645|AAAAAAAAODOAAAAA|3645|1|0|45|AM|third|night|| +3646|AAAAAAAAPDOAAAAA|3646|1|0|46|AM|third|night|| +3647|AAAAAAAAAEOAAAAA|3647|1|0|47|AM|third|night|| +3648|AAAAAAAABEOAAAAA|3648|1|0|48|AM|third|night|| +3649|AAAAAAAACEOAAAAA|3649|1|0|49|AM|third|night|| +3650|AAAAAAAADEOAAAAA|3650|1|0|50|AM|third|night|| +3651|AAAAAAAAEEOAAAAA|3651|1|0|51|AM|third|night|| +3652|AAAAAAAAFEOAAAAA|3652|1|0|52|AM|third|night|| +3653|AAAAAAAAGEOAAAAA|3653|1|0|53|AM|third|night|| +3654|AAAAAAAAHEOAAAAA|3654|1|0|54|AM|third|night|| +3655|AAAAAAAAIEOAAAAA|3655|1|0|55|AM|third|night|| +3656|AAAAAAAAJEOAAAAA|3656|1|0|56|AM|third|night|| +3657|AAAAAAAAKEOAAAAA|3657|1|0|57|AM|third|night|| +3658|AAAAAAAALEOAAAAA|3658|1|0|58|AM|third|night|| +3659|AAAAAAAAMEOAAAAA|3659|1|0|59|AM|third|night|| +3660|AAAAAAAANEOAAAAA|3660|1|1|0|AM|third|night|| +3661|AAAAAAAAOEOAAAAA|3661|1|1|1|AM|third|night|| +3662|AAAAAAAAPEOAAAAA|3662|1|1|2|AM|third|night|| +3663|AAAAAAAAAFOAAAAA|3663|1|1|3|AM|third|night|| +3664|AAAAAAAABFOAAAAA|3664|1|1|4|AM|third|night|| +3665|AAAAAAAACFOAAAAA|3665|1|1|5|AM|third|night|| +3666|AAAAAAAADFOAAAAA|3666|1|1|6|AM|third|night|| +3667|AAAAAAAAEFOAAAAA|3667|1|1|7|AM|third|night|| +3668|AAAAAAAAFFOAAAAA|3668|1|1|8|AM|third|night|| +3669|AAAAAAAAGFOAAAAA|3669|1|1|9|AM|third|night|| +3670|AAAAAAAAHFOAAAAA|3670|1|1|10|AM|third|night|| +3671|AAAAAAAAIFOAAAAA|3671|1|1|11|AM|third|night|| +3672|AAAAAAAAJFOAAAAA|3672|1|1|12|AM|third|night|| +3673|AAAAAAAAKFOAAAAA|3673|1|1|13|AM|third|night|| +3674|AAAAAAAALFOAAAAA|3674|1|1|14|AM|third|night|| +3675|AAAAAAAAMFOAAAAA|3675|1|1|15|AM|third|night|| +3676|AAAAAAAANFOAAAAA|3676|1|1|16|AM|third|night|| +3677|AAAAAAAAOFOAAAAA|3677|1|1|17|AM|third|night|| +3678|AAAAAAAAPFOAAAAA|3678|1|1|18|AM|third|night|| +3679|AAAAAAAAAGOAAAAA|3679|1|1|19|AM|third|night|| +3680|AAAAAAAABGOAAAAA|3680|1|1|20|AM|third|night|| +3681|AAAAAAAACGOAAAAA|3681|1|1|21|AM|third|night|| +3682|AAAAAAAADGOAAAAA|3682|1|1|22|AM|third|night|| +3683|AAAAAAAAEGOAAAAA|3683|1|1|23|AM|third|night|| +3684|AAAAAAAAFGOAAAAA|3684|1|1|24|AM|third|night|| +3685|AAAAAAAAGGOAAAAA|3685|1|1|25|AM|third|night|| +3686|AAAAAAAAHGOAAAAA|3686|1|1|26|AM|third|night|| +3687|AAAAAAAAIGOAAAAA|3687|1|1|27|AM|third|night|| +3688|AAAAAAAAJGOAAAAA|3688|1|1|28|AM|third|night|| +3689|AAAAAAAAKGOAAAAA|3689|1|1|29|AM|third|night|| +3690|AAAAAAAALGOAAAAA|3690|1|1|30|AM|third|night|| +3691|AAAAAAAAMGOAAAAA|3691|1|1|31|AM|third|night|| +3692|AAAAAAAANGOAAAAA|3692|1|1|32|AM|third|night|| +3693|AAAAAAAAOGOAAAAA|3693|1|1|33|AM|third|night|| +3694|AAAAAAAAPGOAAAAA|3694|1|1|34|AM|third|night|| +3695|AAAAAAAAAHOAAAAA|3695|1|1|35|AM|third|night|| +3696|AAAAAAAABHOAAAAA|3696|1|1|36|AM|third|night|| +3697|AAAAAAAACHOAAAAA|3697|1|1|37|AM|third|night|| +3698|AAAAAAAADHOAAAAA|3698|1|1|38|AM|third|night|| +3699|AAAAAAAAEHOAAAAA|3699|1|1|39|AM|third|night|| +3700|AAAAAAAAFHOAAAAA|3700|1|1|40|AM|third|night|| +3701|AAAAAAAAGHOAAAAA|3701|1|1|41|AM|third|night|| +3702|AAAAAAAAHHOAAAAA|3702|1|1|42|AM|third|night|| +3703|AAAAAAAAIHOAAAAA|3703|1|1|43|AM|third|night|| +3704|AAAAAAAAJHOAAAAA|3704|1|1|44|AM|third|night|| +3705|AAAAAAAAKHOAAAAA|3705|1|1|45|AM|third|night|| +3706|AAAAAAAALHOAAAAA|3706|1|1|46|AM|third|night|| +3707|AAAAAAAAMHOAAAAA|3707|1|1|47|AM|third|night|| +3708|AAAAAAAANHOAAAAA|3708|1|1|48|AM|third|night|| +3709|AAAAAAAAOHOAAAAA|3709|1|1|49|AM|third|night|| +3710|AAAAAAAAPHOAAAAA|3710|1|1|50|AM|third|night|| +3711|AAAAAAAAAIOAAAAA|3711|1|1|51|AM|third|night|| +3712|AAAAAAAABIOAAAAA|3712|1|1|52|AM|third|night|| +3713|AAAAAAAACIOAAAAA|3713|1|1|53|AM|third|night|| +3714|AAAAAAAADIOAAAAA|3714|1|1|54|AM|third|night|| +3715|AAAAAAAAEIOAAAAA|3715|1|1|55|AM|third|night|| +3716|AAAAAAAAFIOAAAAA|3716|1|1|56|AM|third|night|| +3717|AAAAAAAAGIOAAAAA|3717|1|1|57|AM|third|night|| +3718|AAAAAAAAHIOAAAAA|3718|1|1|58|AM|third|night|| +3719|AAAAAAAAIIOAAAAA|3719|1|1|59|AM|third|night|| +3720|AAAAAAAAJIOAAAAA|3720|1|2|0|AM|third|night|| +3721|AAAAAAAAKIOAAAAA|3721|1|2|1|AM|third|night|| +3722|AAAAAAAALIOAAAAA|3722|1|2|2|AM|third|night|| +3723|AAAAAAAAMIOAAAAA|3723|1|2|3|AM|third|night|| +3724|AAAAAAAANIOAAAAA|3724|1|2|4|AM|third|night|| +3725|AAAAAAAAOIOAAAAA|3725|1|2|5|AM|third|night|| +3726|AAAAAAAAPIOAAAAA|3726|1|2|6|AM|third|night|| +3727|AAAAAAAAAJOAAAAA|3727|1|2|7|AM|third|night|| +3728|AAAAAAAABJOAAAAA|3728|1|2|8|AM|third|night|| +3729|AAAAAAAACJOAAAAA|3729|1|2|9|AM|third|night|| +3730|AAAAAAAADJOAAAAA|3730|1|2|10|AM|third|night|| +3731|AAAAAAAAEJOAAAAA|3731|1|2|11|AM|third|night|| +3732|AAAAAAAAFJOAAAAA|3732|1|2|12|AM|third|night|| +3733|AAAAAAAAGJOAAAAA|3733|1|2|13|AM|third|night|| +3734|AAAAAAAAHJOAAAAA|3734|1|2|14|AM|third|night|| +3735|AAAAAAAAIJOAAAAA|3735|1|2|15|AM|third|night|| +3736|AAAAAAAAJJOAAAAA|3736|1|2|16|AM|third|night|| +3737|AAAAAAAAKJOAAAAA|3737|1|2|17|AM|third|night|| +3738|AAAAAAAALJOAAAAA|3738|1|2|18|AM|third|night|| +3739|AAAAAAAAMJOAAAAA|3739|1|2|19|AM|third|night|| +3740|AAAAAAAANJOAAAAA|3740|1|2|20|AM|third|night|| +3741|AAAAAAAAOJOAAAAA|3741|1|2|21|AM|third|night|| +3742|AAAAAAAAPJOAAAAA|3742|1|2|22|AM|third|night|| +3743|AAAAAAAAAKOAAAAA|3743|1|2|23|AM|third|night|| +3744|AAAAAAAABKOAAAAA|3744|1|2|24|AM|third|night|| +3745|AAAAAAAACKOAAAAA|3745|1|2|25|AM|third|night|| +3746|AAAAAAAADKOAAAAA|3746|1|2|26|AM|third|night|| +3747|AAAAAAAAEKOAAAAA|3747|1|2|27|AM|third|night|| +3748|AAAAAAAAFKOAAAAA|3748|1|2|28|AM|third|night|| +3749|AAAAAAAAGKOAAAAA|3749|1|2|29|AM|third|night|| +3750|AAAAAAAAHKOAAAAA|3750|1|2|30|AM|third|night|| +3751|AAAAAAAAIKOAAAAA|3751|1|2|31|AM|third|night|| +3752|AAAAAAAAJKOAAAAA|3752|1|2|32|AM|third|night|| +3753|AAAAAAAAKKOAAAAA|3753|1|2|33|AM|third|night|| +3754|AAAAAAAALKOAAAAA|3754|1|2|34|AM|third|night|| +3755|AAAAAAAAMKOAAAAA|3755|1|2|35|AM|third|night|| +3756|AAAAAAAANKOAAAAA|3756|1|2|36|AM|third|night|| +3757|AAAAAAAAOKOAAAAA|3757|1|2|37|AM|third|night|| +3758|AAAAAAAAPKOAAAAA|3758|1|2|38|AM|third|night|| +3759|AAAAAAAAALOAAAAA|3759|1|2|39|AM|third|night|| +3760|AAAAAAAABLOAAAAA|3760|1|2|40|AM|third|night|| +3761|AAAAAAAACLOAAAAA|3761|1|2|41|AM|third|night|| +3762|AAAAAAAADLOAAAAA|3762|1|2|42|AM|third|night|| +3763|AAAAAAAAELOAAAAA|3763|1|2|43|AM|third|night|| +3764|AAAAAAAAFLOAAAAA|3764|1|2|44|AM|third|night|| +3765|AAAAAAAAGLOAAAAA|3765|1|2|45|AM|third|night|| +3766|AAAAAAAAHLOAAAAA|3766|1|2|46|AM|third|night|| +3767|AAAAAAAAILOAAAAA|3767|1|2|47|AM|third|night|| +3768|AAAAAAAAJLOAAAAA|3768|1|2|48|AM|third|night|| +3769|AAAAAAAAKLOAAAAA|3769|1|2|49|AM|third|night|| +3770|AAAAAAAALLOAAAAA|3770|1|2|50|AM|third|night|| +3771|AAAAAAAAMLOAAAAA|3771|1|2|51|AM|third|night|| +3772|AAAAAAAANLOAAAAA|3772|1|2|52|AM|third|night|| +3773|AAAAAAAAOLOAAAAA|3773|1|2|53|AM|third|night|| +3774|AAAAAAAAPLOAAAAA|3774|1|2|54|AM|third|night|| +3775|AAAAAAAAAMOAAAAA|3775|1|2|55|AM|third|night|| +3776|AAAAAAAABMOAAAAA|3776|1|2|56|AM|third|night|| +3777|AAAAAAAACMOAAAAA|3777|1|2|57|AM|third|night|| +3778|AAAAAAAADMOAAAAA|3778|1|2|58|AM|third|night|| +3779|AAAAAAAAEMOAAAAA|3779|1|2|59|AM|third|night|| +3780|AAAAAAAAFMOAAAAA|3780|1|3|0|AM|third|night|| +3781|AAAAAAAAGMOAAAAA|3781|1|3|1|AM|third|night|| +3782|AAAAAAAAHMOAAAAA|3782|1|3|2|AM|third|night|| +3783|AAAAAAAAIMOAAAAA|3783|1|3|3|AM|third|night|| +3784|AAAAAAAAJMOAAAAA|3784|1|3|4|AM|third|night|| +3785|AAAAAAAAKMOAAAAA|3785|1|3|5|AM|third|night|| +3786|AAAAAAAALMOAAAAA|3786|1|3|6|AM|third|night|| +3787|AAAAAAAAMMOAAAAA|3787|1|3|7|AM|third|night|| +3788|AAAAAAAANMOAAAAA|3788|1|3|8|AM|third|night|| +3789|AAAAAAAAOMOAAAAA|3789|1|3|9|AM|third|night|| +3790|AAAAAAAAPMOAAAAA|3790|1|3|10|AM|third|night|| +3791|AAAAAAAAANOAAAAA|3791|1|3|11|AM|third|night|| +3792|AAAAAAAABNOAAAAA|3792|1|3|12|AM|third|night|| +3793|AAAAAAAACNOAAAAA|3793|1|3|13|AM|third|night|| +3794|AAAAAAAADNOAAAAA|3794|1|3|14|AM|third|night|| +3795|AAAAAAAAENOAAAAA|3795|1|3|15|AM|third|night|| +3796|AAAAAAAAFNOAAAAA|3796|1|3|16|AM|third|night|| +3797|AAAAAAAAGNOAAAAA|3797|1|3|17|AM|third|night|| +3798|AAAAAAAAHNOAAAAA|3798|1|3|18|AM|third|night|| +3799|AAAAAAAAINOAAAAA|3799|1|3|19|AM|third|night|| +3800|AAAAAAAAJNOAAAAA|3800|1|3|20|AM|third|night|| +3801|AAAAAAAAKNOAAAAA|3801|1|3|21|AM|third|night|| +3802|AAAAAAAALNOAAAAA|3802|1|3|22|AM|third|night|| +3803|AAAAAAAAMNOAAAAA|3803|1|3|23|AM|third|night|| +3804|AAAAAAAANNOAAAAA|3804|1|3|24|AM|third|night|| +3805|AAAAAAAAONOAAAAA|3805|1|3|25|AM|third|night|| +3806|AAAAAAAAPNOAAAAA|3806|1|3|26|AM|third|night|| +3807|AAAAAAAAAOOAAAAA|3807|1|3|27|AM|third|night|| +3808|AAAAAAAABOOAAAAA|3808|1|3|28|AM|third|night|| +3809|AAAAAAAACOOAAAAA|3809|1|3|29|AM|third|night|| +3810|AAAAAAAADOOAAAAA|3810|1|3|30|AM|third|night|| +3811|AAAAAAAAEOOAAAAA|3811|1|3|31|AM|third|night|| +3812|AAAAAAAAFOOAAAAA|3812|1|3|32|AM|third|night|| +3813|AAAAAAAAGOOAAAAA|3813|1|3|33|AM|third|night|| +3814|AAAAAAAAHOOAAAAA|3814|1|3|34|AM|third|night|| +3815|AAAAAAAAIOOAAAAA|3815|1|3|35|AM|third|night|| +3816|AAAAAAAAJOOAAAAA|3816|1|3|36|AM|third|night|| +3817|AAAAAAAAKOOAAAAA|3817|1|3|37|AM|third|night|| +3818|AAAAAAAALOOAAAAA|3818|1|3|38|AM|third|night|| +3819|AAAAAAAAMOOAAAAA|3819|1|3|39|AM|third|night|| +3820|AAAAAAAANOOAAAAA|3820|1|3|40|AM|third|night|| +3821|AAAAAAAAOOOAAAAA|3821|1|3|41|AM|third|night|| +3822|AAAAAAAAPOOAAAAA|3822|1|3|42|AM|third|night|| +3823|AAAAAAAAAPOAAAAA|3823|1|3|43|AM|third|night|| +3824|AAAAAAAABPOAAAAA|3824|1|3|44|AM|third|night|| +3825|AAAAAAAACPOAAAAA|3825|1|3|45|AM|third|night|| +3826|AAAAAAAADPOAAAAA|3826|1|3|46|AM|third|night|| +3827|AAAAAAAAEPOAAAAA|3827|1|3|47|AM|third|night|| +3828|AAAAAAAAFPOAAAAA|3828|1|3|48|AM|third|night|| +3829|AAAAAAAAGPOAAAAA|3829|1|3|49|AM|third|night|| +3830|AAAAAAAAHPOAAAAA|3830|1|3|50|AM|third|night|| +3831|AAAAAAAAIPOAAAAA|3831|1|3|51|AM|third|night|| +3832|AAAAAAAAJPOAAAAA|3832|1|3|52|AM|third|night|| +3833|AAAAAAAAKPOAAAAA|3833|1|3|53|AM|third|night|| +3834|AAAAAAAALPOAAAAA|3834|1|3|54|AM|third|night|| +3835|AAAAAAAAMPOAAAAA|3835|1|3|55|AM|third|night|| +3836|AAAAAAAANPOAAAAA|3836|1|3|56|AM|third|night|| +3837|AAAAAAAAOPOAAAAA|3837|1|3|57|AM|third|night|| +3838|AAAAAAAAPPOAAAAA|3838|1|3|58|AM|third|night|| +3839|AAAAAAAAAAPAAAAA|3839|1|3|59|AM|third|night|| +3840|AAAAAAAABAPAAAAA|3840|1|4|0|AM|third|night|| +3841|AAAAAAAACAPAAAAA|3841|1|4|1|AM|third|night|| +3842|AAAAAAAADAPAAAAA|3842|1|4|2|AM|third|night|| +3843|AAAAAAAAEAPAAAAA|3843|1|4|3|AM|third|night|| +3844|AAAAAAAAFAPAAAAA|3844|1|4|4|AM|third|night|| +3845|AAAAAAAAGAPAAAAA|3845|1|4|5|AM|third|night|| +3846|AAAAAAAAHAPAAAAA|3846|1|4|6|AM|third|night|| +3847|AAAAAAAAIAPAAAAA|3847|1|4|7|AM|third|night|| +3848|AAAAAAAAJAPAAAAA|3848|1|4|8|AM|third|night|| +3849|AAAAAAAAKAPAAAAA|3849|1|4|9|AM|third|night|| +3850|AAAAAAAALAPAAAAA|3850|1|4|10|AM|third|night|| +3851|AAAAAAAAMAPAAAAA|3851|1|4|11|AM|third|night|| +3852|AAAAAAAANAPAAAAA|3852|1|4|12|AM|third|night|| +3853|AAAAAAAAOAPAAAAA|3853|1|4|13|AM|third|night|| +3854|AAAAAAAAPAPAAAAA|3854|1|4|14|AM|third|night|| +3855|AAAAAAAAABPAAAAA|3855|1|4|15|AM|third|night|| +3856|AAAAAAAABBPAAAAA|3856|1|4|16|AM|third|night|| +3857|AAAAAAAACBPAAAAA|3857|1|4|17|AM|third|night|| +3858|AAAAAAAADBPAAAAA|3858|1|4|18|AM|third|night|| +3859|AAAAAAAAEBPAAAAA|3859|1|4|19|AM|third|night|| +3860|AAAAAAAAFBPAAAAA|3860|1|4|20|AM|third|night|| +3861|AAAAAAAAGBPAAAAA|3861|1|4|21|AM|third|night|| +3862|AAAAAAAAHBPAAAAA|3862|1|4|22|AM|third|night|| +3863|AAAAAAAAIBPAAAAA|3863|1|4|23|AM|third|night|| +3864|AAAAAAAAJBPAAAAA|3864|1|4|24|AM|third|night|| +3865|AAAAAAAAKBPAAAAA|3865|1|4|25|AM|third|night|| +3866|AAAAAAAALBPAAAAA|3866|1|4|26|AM|third|night|| +3867|AAAAAAAAMBPAAAAA|3867|1|4|27|AM|third|night|| +3868|AAAAAAAANBPAAAAA|3868|1|4|28|AM|third|night|| +3869|AAAAAAAAOBPAAAAA|3869|1|4|29|AM|third|night|| +3870|AAAAAAAAPBPAAAAA|3870|1|4|30|AM|third|night|| +3871|AAAAAAAAACPAAAAA|3871|1|4|31|AM|third|night|| +3872|AAAAAAAABCPAAAAA|3872|1|4|32|AM|third|night|| +3873|AAAAAAAACCPAAAAA|3873|1|4|33|AM|third|night|| +3874|AAAAAAAADCPAAAAA|3874|1|4|34|AM|third|night|| +3875|AAAAAAAAECPAAAAA|3875|1|4|35|AM|third|night|| +3876|AAAAAAAAFCPAAAAA|3876|1|4|36|AM|third|night|| +3877|AAAAAAAAGCPAAAAA|3877|1|4|37|AM|third|night|| +3878|AAAAAAAAHCPAAAAA|3878|1|4|38|AM|third|night|| +3879|AAAAAAAAICPAAAAA|3879|1|4|39|AM|third|night|| +3880|AAAAAAAAJCPAAAAA|3880|1|4|40|AM|third|night|| +3881|AAAAAAAAKCPAAAAA|3881|1|4|41|AM|third|night|| +3882|AAAAAAAALCPAAAAA|3882|1|4|42|AM|third|night|| +3883|AAAAAAAAMCPAAAAA|3883|1|4|43|AM|third|night|| +3884|AAAAAAAANCPAAAAA|3884|1|4|44|AM|third|night|| +3885|AAAAAAAAOCPAAAAA|3885|1|4|45|AM|third|night|| +3886|AAAAAAAAPCPAAAAA|3886|1|4|46|AM|third|night|| +3887|AAAAAAAAADPAAAAA|3887|1|4|47|AM|third|night|| +3888|AAAAAAAABDPAAAAA|3888|1|4|48|AM|third|night|| +3889|AAAAAAAACDPAAAAA|3889|1|4|49|AM|third|night|| +3890|AAAAAAAADDPAAAAA|3890|1|4|50|AM|third|night|| +3891|AAAAAAAAEDPAAAAA|3891|1|4|51|AM|third|night|| +3892|AAAAAAAAFDPAAAAA|3892|1|4|52|AM|third|night|| +3893|AAAAAAAAGDPAAAAA|3893|1|4|53|AM|third|night|| +3894|AAAAAAAAHDPAAAAA|3894|1|4|54|AM|third|night|| +3895|AAAAAAAAIDPAAAAA|3895|1|4|55|AM|third|night|| +3896|AAAAAAAAJDPAAAAA|3896|1|4|56|AM|third|night|| +3897|AAAAAAAAKDPAAAAA|3897|1|4|57|AM|third|night|| +3898|AAAAAAAALDPAAAAA|3898|1|4|58|AM|third|night|| +3899|AAAAAAAAMDPAAAAA|3899|1|4|59|AM|third|night|| +3900|AAAAAAAANDPAAAAA|3900|1|5|0|AM|third|night|| +3901|AAAAAAAAODPAAAAA|3901|1|5|1|AM|third|night|| +3902|AAAAAAAAPDPAAAAA|3902|1|5|2|AM|third|night|| +3903|AAAAAAAAAEPAAAAA|3903|1|5|3|AM|third|night|| +3904|AAAAAAAABEPAAAAA|3904|1|5|4|AM|third|night|| +3905|AAAAAAAACEPAAAAA|3905|1|5|5|AM|third|night|| +3906|AAAAAAAADEPAAAAA|3906|1|5|6|AM|third|night|| +3907|AAAAAAAAEEPAAAAA|3907|1|5|7|AM|third|night|| +3908|AAAAAAAAFEPAAAAA|3908|1|5|8|AM|third|night|| +3909|AAAAAAAAGEPAAAAA|3909|1|5|9|AM|third|night|| +3910|AAAAAAAAHEPAAAAA|3910|1|5|10|AM|third|night|| +3911|AAAAAAAAIEPAAAAA|3911|1|5|11|AM|third|night|| +3912|AAAAAAAAJEPAAAAA|3912|1|5|12|AM|third|night|| +3913|AAAAAAAAKEPAAAAA|3913|1|5|13|AM|third|night|| +3914|AAAAAAAALEPAAAAA|3914|1|5|14|AM|third|night|| +3915|AAAAAAAAMEPAAAAA|3915|1|5|15|AM|third|night|| +3916|AAAAAAAANEPAAAAA|3916|1|5|16|AM|third|night|| +3917|AAAAAAAAOEPAAAAA|3917|1|5|17|AM|third|night|| +3918|AAAAAAAAPEPAAAAA|3918|1|5|18|AM|third|night|| +3919|AAAAAAAAAFPAAAAA|3919|1|5|19|AM|third|night|| +3920|AAAAAAAABFPAAAAA|3920|1|5|20|AM|third|night|| +3921|AAAAAAAACFPAAAAA|3921|1|5|21|AM|third|night|| +3922|AAAAAAAADFPAAAAA|3922|1|5|22|AM|third|night|| +3923|AAAAAAAAEFPAAAAA|3923|1|5|23|AM|third|night|| +3924|AAAAAAAAFFPAAAAA|3924|1|5|24|AM|third|night|| +3925|AAAAAAAAGFPAAAAA|3925|1|5|25|AM|third|night|| +3926|AAAAAAAAHFPAAAAA|3926|1|5|26|AM|third|night|| +3927|AAAAAAAAIFPAAAAA|3927|1|5|27|AM|third|night|| +3928|AAAAAAAAJFPAAAAA|3928|1|5|28|AM|third|night|| +3929|AAAAAAAAKFPAAAAA|3929|1|5|29|AM|third|night|| +3930|AAAAAAAALFPAAAAA|3930|1|5|30|AM|third|night|| +3931|AAAAAAAAMFPAAAAA|3931|1|5|31|AM|third|night|| +3932|AAAAAAAANFPAAAAA|3932|1|5|32|AM|third|night|| +3933|AAAAAAAAOFPAAAAA|3933|1|5|33|AM|third|night|| +3934|AAAAAAAAPFPAAAAA|3934|1|5|34|AM|third|night|| +3935|AAAAAAAAAGPAAAAA|3935|1|5|35|AM|third|night|| +3936|AAAAAAAABGPAAAAA|3936|1|5|36|AM|third|night|| +3937|AAAAAAAACGPAAAAA|3937|1|5|37|AM|third|night|| +3938|AAAAAAAADGPAAAAA|3938|1|5|38|AM|third|night|| +3939|AAAAAAAAEGPAAAAA|3939|1|5|39|AM|third|night|| +3940|AAAAAAAAFGPAAAAA|3940|1|5|40|AM|third|night|| +3941|AAAAAAAAGGPAAAAA|3941|1|5|41|AM|third|night|| +3942|AAAAAAAAHGPAAAAA|3942|1|5|42|AM|third|night|| +3943|AAAAAAAAIGPAAAAA|3943|1|5|43|AM|third|night|| +3944|AAAAAAAAJGPAAAAA|3944|1|5|44|AM|third|night|| +3945|AAAAAAAAKGPAAAAA|3945|1|5|45|AM|third|night|| +3946|AAAAAAAALGPAAAAA|3946|1|5|46|AM|third|night|| +3947|AAAAAAAAMGPAAAAA|3947|1|5|47|AM|third|night|| +3948|AAAAAAAANGPAAAAA|3948|1|5|48|AM|third|night|| +3949|AAAAAAAAOGPAAAAA|3949|1|5|49|AM|third|night|| +3950|AAAAAAAAPGPAAAAA|3950|1|5|50|AM|third|night|| +3951|AAAAAAAAAHPAAAAA|3951|1|5|51|AM|third|night|| +3952|AAAAAAAABHPAAAAA|3952|1|5|52|AM|third|night|| +3953|AAAAAAAACHPAAAAA|3953|1|5|53|AM|third|night|| +3954|AAAAAAAADHPAAAAA|3954|1|5|54|AM|third|night|| +3955|AAAAAAAAEHPAAAAA|3955|1|5|55|AM|third|night|| +3956|AAAAAAAAFHPAAAAA|3956|1|5|56|AM|third|night|| +3957|AAAAAAAAGHPAAAAA|3957|1|5|57|AM|third|night|| +3958|AAAAAAAAHHPAAAAA|3958|1|5|58|AM|third|night|| +3959|AAAAAAAAIHPAAAAA|3959|1|5|59|AM|third|night|| +3960|AAAAAAAAJHPAAAAA|3960|1|6|0|AM|third|night|| +3961|AAAAAAAAKHPAAAAA|3961|1|6|1|AM|third|night|| +3962|AAAAAAAALHPAAAAA|3962|1|6|2|AM|third|night|| +3963|AAAAAAAAMHPAAAAA|3963|1|6|3|AM|third|night|| +3964|AAAAAAAANHPAAAAA|3964|1|6|4|AM|third|night|| +3965|AAAAAAAAOHPAAAAA|3965|1|6|5|AM|third|night|| +3966|AAAAAAAAPHPAAAAA|3966|1|6|6|AM|third|night|| +3967|AAAAAAAAAIPAAAAA|3967|1|6|7|AM|third|night|| +3968|AAAAAAAABIPAAAAA|3968|1|6|8|AM|third|night|| +3969|AAAAAAAACIPAAAAA|3969|1|6|9|AM|third|night|| +3970|AAAAAAAADIPAAAAA|3970|1|6|10|AM|third|night|| +3971|AAAAAAAAEIPAAAAA|3971|1|6|11|AM|third|night|| +3972|AAAAAAAAFIPAAAAA|3972|1|6|12|AM|third|night|| +3973|AAAAAAAAGIPAAAAA|3973|1|6|13|AM|third|night|| +3974|AAAAAAAAHIPAAAAA|3974|1|6|14|AM|third|night|| +3975|AAAAAAAAIIPAAAAA|3975|1|6|15|AM|third|night|| +3976|AAAAAAAAJIPAAAAA|3976|1|6|16|AM|third|night|| +3977|AAAAAAAAKIPAAAAA|3977|1|6|17|AM|third|night|| +3978|AAAAAAAALIPAAAAA|3978|1|6|18|AM|third|night|| +3979|AAAAAAAAMIPAAAAA|3979|1|6|19|AM|third|night|| +3980|AAAAAAAANIPAAAAA|3980|1|6|20|AM|third|night|| +3981|AAAAAAAAOIPAAAAA|3981|1|6|21|AM|third|night|| +3982|AAAAAAAAPIPAAAAA|3982|1|6|22|AM|third|night|| +3983|AAAAAAAAAJPAAAAA|3983|1|6|23|AM|third|night|| +3984|AAAAAAAABJPAAAAA|3984|1|6|24|AM|third|night|| +3985|AAAAAAAACJPAAAAA|3985|1|6|25|AM|third|night|| +3986|AAAAAAAADJPAAAAA|3986|1|6|26|AM|third|night|| +3987|AAAAAAAAEJPAAAAA|3987|1|6|27|AM|third|night|| +3988|AAAAAAAAFJPAAAAA|3988|1|6|28|AM|third|night|| +3989|AAAAAAAAGJPAAAAA|3989|1|6|29|AM|third|night|| +3990|AAAAAAAAHJPAAAAA|3990|1|6|30|AM|third|night|| +3991|AAAAAAAAIJPAAAAA|3991|1|6|31|AM|third|night|| +3992|AAAAAAAAJJPAAAAA|3992|1|6|32|AM|third|night|| +3993|AAAAAAAAKJPAAAAA|3993|1|6|33|AM|third|night|| +3994|AAAAAAAALJPAAAAA|3994|1|6|34|AM|third|night|| +3995|AAAAAAAAMJPAAAAA|3995|1|6|35|AM|third|night|| +3996|AAAAAAAANJPAAAAA|3996|1|6|36|AM|third|night|| +3997|AAAAAAAAOJPAAAAA|3997|1|6|37|AM|third|night|| +3998|AAAAAAAAPJPAAAAA|3998|1|6|38|AM|third|night|| +3999|AAAAAAAAAKPAAAAA|3999|1|6|39|AM|third|night|| +4000|AAAAAAAABKPAAAAA|4000|1|6|40|AM|third|night|| +4001|AAAAAAAACKPAAAAA|4001|1|6|41|AM|third|night|| +4002|AAAAAAAADKPAAAAA|4002|1|6|42|AM|third|night|| +4003|AAAAAAAAEKPAAAAA|4003|1|6|43|AM|third|night|| +4004|AAAAAAAAFKPAAAAA|4004|1|6|44|AM|third|night|| +4005|AAAAAAAAGKPAAAAA|4005|1|6|45|AM|third|night|| +4006|AAAAAAAAHKPAAAAA|4006|1|6|46|AM|third|night|| +4007|AAAAAAAAIKPAAAAA|4007|1|6|47|AM|third|night|| +4008|AAAAAAAAJKPAAAAA|4008|1|6|48|AM|third|night|| +4009|AAAAAAAAKKPAAAAA|4009|1|6|49|AM|third|night|| +4010|AAAAAAAALKPAAAAA|4010|1|6|50|AM|third|night|| +4011|AAAAAAAAMKPAAAAA|4011|1|6|51|AM|third|night|| +4012|AAAAAAAANKPAAAAA|4012|1|6|52|AM|third|night|| +4013|AAAAAAAAOKPAAAAA|4013|1|6|53|AM|third|night|| +4014|AAAAAAAAPKPAAAAA|4014|1|6|54|AM|third|night|| +4015|AAAAAAAAALPAAAAA|4015|1|6|55|AM|third|night|| +4016|AAAAAAAABLPAAAAA|4016|1|6|56|AM|third|night|| +4017|AAAAAAAACLPAAAAA|4017|1|6|57|AM|third|night|| +4018|AAAAAAAADLPAAAAA|4018|1|6|58|AM|third|night|| +4019|AAAAAAAAELPAAAAA|4019|1|6|59|AM|third|night|| +4020|AAAAAAAAFLPAAAAA|4020|1|7|0|AM|third|night|| +4021|AAAAAAAAGLPAAAAA|4021|1|7|1|AM|third|night|| +4022|AAAAAAAAHLPAAAAA|4022|1|7|2|AM|third|night|| +4023|AAAAAAAAILPAAAAA|4023|1|7|3|AM|third|night|| +4024|AAAAAAAAJLPAAAAA|4024|1|7|4|AM|third|night|| +4025|AAAAAAAAKLPAAAAA|4025|1|7|5|AM|third|night|| +4026|AAAAAAAALLPAAAAA|4026|1|7|6|AM|third|night|| +4027|AAAAAAAAMLPAAAAA|4027|1|7|7|AM|third|night|| +4028|AAAAAAAANLPAAAAA|4028|1|7|8|AM|third|night|| +4029|AAAAAAAAOLPAAAAA|4029|1|7|9|AM|third|night|| +4030|AAAAAAAAPLPAAAAA|4030|1|7|10|AM|third|night|| +4031|AAAAAAAAAMPAAAAA|4031|1|7|11|AM|third|night|| +4032|AAAAAAAABMPAAAAA|4032|1|7|12|AM|third|night|| +4033|AAAAAAAACMPAAAAA|4033|1|7|13|AM|third|night|| +4034|AAAAAAAADMPAAAAA|4034|1|7|14|AM|third|night|| +4035|AAAAAAAAEMPAAAAA|4035|1|7|15|AM|third|night|| +4036|AAAAAAAAFMPAAAAA|4036|1|7|16|AM|third|night|| +4037|AAAAAAAAGMPAAAAA|4037|1|7|17|AM|third|night|| +4038|AAAAAAAAHMPAAAAA|4038|1|7|18|AM|third|night|| +4039|AAAAAAAAIMPAAAAA|4039|1|7|19|AM|third|night|| +4040|AAAAAAAAJMPAAAAA|4040|1|7|20|AM|third|night|| +4041|AAAAAAAAKMPAAAAA|4041|1|7|21|AM|third|night|| +4042|AAAAAAAALMPAAAAA|4042|1|7|22|AM|third|night|| +4043|AAAAAAAAMMPAAAAA|4043|1|7|23|AM|third|night|| +4044|AAAAAAAANMPAAAAA|4044|1|7|24|AM|third|night|| +4045|AAAAAAAAOMPAAAAA|4045|1|7|25|AM|third|night|| +4046|AAAAAAAAPMPAAAAA|4046|1|7|26|AM|third|night|| +4047|AAAAAAAAANPAAAAA|4047|1|7|27|AM|third|night|| +4048|AAAAAAAABNPAAAAA|4048|1|7|28|AM|third|night|| +4049|AAAAAAAACNPAAAAA|4049|1|7|29|AM|third|night|| +4050|AAAAAAAADNPAAAAA|4050|1|7|30|AM|third|night|| +4051|AAAAAAAAENPAAAAA|4051|1|7|31|AM|third|night|| +4052|AAAAAAAAFNPAAAAA|4052|1|7|32|AM|third|night|| +4053|AAAAAAAAGNPAAAAA|4053|1|7|33|AM|third|night|| +4054|AAAAAAAAHNPAAAAA|4054|1|7|34|AM|third|night|| +4055|AAAAAAAAINPAAAAA|4055|1|7|35|AM|third|night|| +4056|AAAAAAAAJNPAAAAA|4056|1|7|36|AM|third|night|| +4057|AAAAAAAAKNPAAAAA|4057|1|7|37|AM|third|night|| +4058|AAAAAAAALNPAAAAA|4058|1|7|38|AM|third|night|| +4059|AAAAAAAAMNPAAAAA|4059|1|7|39|AM|third|night|| +4060|AAAAAAAANNPAAAAA|4060|1|7|40|AM|third|night|| +4061|AAAAAAAAONPAAAAA|4061|1|7|41|AM|third|night|| +4062|AAAAAAAAPNPAAAAA|4062|1|7|42|AM|third|night|| +4063|AAAAAAAAAOPAAAAA|4063|1|7|43|AM|third|night|| +4064|AAAAAAAABOPAAAAA|4064|1|7|44|AM|third|night|| +4065|AAAAAAAACOPAAAAA|4065|1|7|45|AM|third|night|| +4066|AAAAAAAADOPAAAAA|4066|1|7|46|AM|third|night|| +4067|AAAAAAAAEOPAAAAA|4067|1|7|47|AM|third|night|| +4068|AAAAAAAAFOPAAAAA|4068|1|7|48|AM|third|night|| +4069|AAAAAAAAGOPAAAAA|4069|1|7|49|AM|third|night|| +4070|AAAAAAAAHOPAAAAA|4070|1|7|50|AM|third|night|| +4071|AAAAAAAAIOPAAAAA|4071|1|7|51|AM|third|night|| +4072|AAAAAAAAJOPAAAAA|4072|1|7|52|AM|third|night|| +4073|AAAAAAAAKOPAAAAA|4073|1|7|53|AM|third|night|| +4074|AAAAAAAALOPAAAAA|4074|1|7|54|AM|third|night|| +4075|AAAAAAAAMOPAAAAA|4075|1|7|55|AM|third|night|| +4076|AAAAAAAANOPAAAAA|4076|1|7|56|AM|third|night|| +4077|AAAAAAAAOOPAAAAA|4077|1|7|57|AM|third|night|| +4078|AAAAAAAAPOPAAAAA|4078|1|7|58|AM|third|night|| +4079|AAAAAAAAAPPAAAAA|4079|1|7|59|AM|third|night|| +4080|AAAAAAAABPPAAAAA|4080|1|8|0|AM|third|night|| +4081|AAAAAAAACPPAAAAA|4081|1|8|1|AM|third|night|| +4082|AAAAAAAADPPAAAAA|4082|1|8|2|AM|third|night|| +4083|AAAAAAAAEPPAAAAA|4083|1|8|3|AM|third|night|| +4084|AAAAAAAAFPPAAAAA|4084|1|8|4|AM|third|night|| +4085|AAAAAAAAGPPAAAAA|4085|1|8|5|AM|third|night|| +4086|AAAAAAAAHPPAAAAA|4086|1|8|6|AM|third|night|| +4087|AAAAAAAAIPPAAAAA|4087|1|8|7|AM|third|night|| +4088|AAAAAAAAJPPAAAAA|4088|1|8|8|AM|third|night|| +4089|AAAAAAAAKPPAAAAA|4089|1|8|9|AM|third|night|| +4090|AAAAAAAALPPAAAAA|4090|1|8|10|AM|third|night|| +4091|AAAAAAAAMPPAAAAA|4091|1|8|11|AM|third|night|| +4092|AAAAAAAANPPAAAAA|4092|1|8|12|AM|third|night|| +4093|AAAAAAAAOPPAAAAA|4093|1|8|13|AM|third|night|| +4094|AAAAAAAAPPPAAAAA|4094|1|8|14|AM|third|night|| +4095|AAAAAAAAAAABAAAA|4095|1|8|15|AM|third|night|| +4096|AAAAAAAABAABAAAA|4096|1|8|16|AM|third|night|| +4097|AAAAAAAACAABAAAA|4097|1|8|17|AM|third|night|| +4098|AAAAAAAADAABAAAA|4098|1|8|18|AM|third|night|| +4099|AAAAAAAAEAABAAAA|4099|1|8|19|AM|third|night|| +4100|AAAAAAAAFAABAAAA|4100|1|8|20|AM|third|night|| +4101|AAAAAAAAGAABAAAA|4101|1|8|21|AM|third|night|| +4102|AAAAAAAAHAABAAAA|4102|1|8|22|AM|third|night|| +4103|AAAAAAAAIAABAAAA|4103|1|8|23|AM|third|night|| +4104|AAAAAAAAJAABAAAA|4104|1|8|24|AM|third|night|| +4105|AAAAAAAAKAABAAAA|4105|1|8|25|AM|third|night|| +4106|AAAAAAAALAABAAAA|4106|1|8|26|AM|third|night|| +4107|AAAAAAAAMAABAAAA|4107|1|8|27|AM|third|night|| +4108|AAAAAAAANAABAAAA|4108|1|8|28|AM|third|night|| +4109|AAAAAAAAOAABAAAA|4109|1|8|29|AM|third|night|| +4110|AAAAAAAAPAABAAAA|4110|1|8|30|AM|third|night|| +4111|AAAAAAAAABABAAAA|4111|1|8|31|AM|third|night|| +4112|AAAAAAAABBABAAAA|4112|1|8|32|AM|third|night|| +4113|AAAAAAAACBABAAAA|4113|1|8|33|AM|third|night|| +4114|AAAAAAAADBABAAAA|4114|1|8|34|AM|third|night|| +4115|AAAAAAAAEBABAAAA|4115|1|8|35|AM|third|night|| +4116|AAAAAAAAFBABAAAA|4116|1|8|36|AM|third|night|| +4117|AAAAAAAAGBABAAAA|4117|1|8|37|AM|third|night|| +4118|AAAAAAAAHBABAAAA|4118|1|8|38|AM|third|night|| +4119|AAAAAAAAIBABAAAA|4119|1|8|39|AM|third|night|| +4120|AAAAAAAAJBABAAAA|4120|1|8|40|AM|third|night|| +4121|AAAAAAAAKBABAAAA|4121|1|8|41|AM|third|night|| +4122|AAAAAAAALBABAAAA|4122|1|8|42|AM|third|night|| +4123|AAAAAAAAMBABAAAA|4123|1|8|43|AM|third|night|| +4124|AAAAAAAANBABAAAA|4124|1|8|44|AM|third|night|| +4125|AAAAAAAAOBABAAAA|4125|1|8|45|AM|third|night|| +4126|AAAAAAAAPBABAAAA|4126|1|8|46|AM|third|night|| +4127|AAAAAAAAACABAAAA|4127|1|8|47|AM|third|night|| +4128|AAAAAAAABCABAAAA|4128|1|8|48|AM|third|night|| +4129|AAAAAAAACCABAAAA|4129|1|8|49|AM|third|night|| +4130|AAAAAAAADCABAAAA|4130|1|8|50|AM|third|night|| +4131|AAAAAAAAECABAAAA|4131|1|8|51|AM|third|night|| +4132|AAAAAAAAFCABAAAA|4132|1|8|52|AM|third|night|| +4133|AAAAAAAAGCABAAAA|4133|1|8|53|AM|third|night|| +4134|AAAAAAAAHCABAAAA|4134|1|8|54|AM|third|night|| +4135|AAAAAAAAICABAAAA|4135|1|8|55|AM|third|night|| +4136|AAAAAAAAJCABAAAA|4136|1|8|56|AM|third|night|| +4137|AAAAAAAAKCABAAAA|4137|1|8|57|AM|third|night|| +4138|AAAAAAAALCABAAAA|4138|1|8|58|AM|third|night|| +4139|AAAAAAAAMCABAAAA|4139|1|8|59|AM|third|night|| +4140|AAAAAAAANCABAAAA|4140|1|9|0|AM|third|night|| +4141|AAAAAAAAOCABAAAA|4141|1|9|1|AM|third|night|| +4142|AAAAAAAAPCABAAAA|4142|1|9|2|AM|third|night|| +4143|AAAAAAAAADABAAAA|4143|1|9|3|AM|third|night|| +4144|AAAAAAAABDABAAAA|4144|1|9|4|AM|third|night|| +4145|AAAAAAAACDABAAAA|4145|1|9|5|AM|third|night|| +4146|AAAAAAAADDABAAAA|4146|1|9|6|AM|third|night|| +4147|AAAAAAAAEDABAAAA|4147|1|9|7|AM|third|night|| +4148|AAAAAAAAFDABAAAA|4148|1|9|8|AM|third|night|| +4149|AAAAAAAAGDABAAAA|4149|1|9|9|AM|third|night|| +4150|AAAAAAAAHDABAAAA|4150|1|9|10|AM|third|night|| +4151|AAAAAAAAIDABAAAA|4151|1|9|11|AM|third|night|| +4152|AAAAAAAAJDABAAAA|4152|1|9|12|AM|third|night|| +4153|AAAAAAAAKDABAAAA|4153|1|9|13|AM|third|night|| +4154|AAAAAAAALDABAAAA|4154|1|9|14|AM|third|night|| +4155|AAAAAAAAMDABAAAA|4155|1|9|15|AM|third|night|| +4156|AAAAAAAANDABAAAA|4156|1|9|16|AM|third|night|| +4157|AAAAAAAAODABAAAA|4157|1|9|17|AM|third|night|| +4158|AAAAAAAAPDABAAAA|4158|1|9|18|AM|third|night|| +4159|AAAAAAAAAEABAAAA|4159|1|9|19|AM|third|night|| +4160|AAAAAAAABEABAAAA|4160|1|9|20|AM|third|night|| +4161|AAAAAAAACEABAAAA|4161|1|9|21|AM|third|night|| +4162|AAAAAAAADEABAAAA|4162|1|9|22|AM|third|night|| +4163|AAAAAAAAEEABAAAA|4163|1|9|23|AM|third|night|| +4164|AAAAAAAAFEABAAAA|4164|1|9|24|AM|third|night|| +4165|AAAAAAAAGEABAAAA|4165|1|9|25|AM|third|night|| +4166|AAAAAAAAHEABAAAA|4166|1|9|26|AM|third|night|| +4167|AAAAAAAAIEABAAAA|4167|1|9|27|AM|third|night|| +4168|AAAAAAAAJEABAAAA|4168|1|9|28|AM|third|night|| +4169|AAAAAAAAKEABAAAA|4169|1|9|29|AM|third|night|| +4170|AAAAAAAALEABAAAA|4170|1|9|30|AM|third|night|| +4171|AAAAAAAAMEABAAAA|4171|1|9|31|AM|third|night|| +4172|AAAAAAAANEABAAAA|4172|1|9|32|AM|third|night|| +4173|AAAAAAAAOEABAAAA|4173|1|9|33|AM|third|night|| +4174|AAAAAAAAPEABAAAA|4174|1|9|34|AM|third|night|| +4175|AAAAAAAAAFABAAAA|4175|1|9|35|AM|third|night|| +4176|AAAAAAAABFABAAAA|4176|1|9|36|AM|third|night|| +4177|AAAAAAAACFABAAAA|4177|1|9|37|AM|third|night|| +4178|AAAAAAAADFABAAAA|4178|1|9|38|AM|third|night|| +4179|AAAAAAAAEFABAAAA|4179|1|9|39|AM|third|night|| +4180|AAAAAAAAFFABAAAA|4180|1|9|40|AM|third|night|| +4181|AAAAAAAAGFABAAAA|4181|1|9|41|AM|third|night|| +4182|AAAAAAAAHFABAAAA|4182|1|9|42|AM|third|night|| +4183|AAAAAAAAIFABAAAA|4183|1|9|43|AM|third|night|| +4184|AAAAAAAAJFABAAAA|4184|1|9|44|AM|third|night|| +4185|AAAAAAAAKFABAAAA|4185|1|9|45|AM|third|night|| +4186|AAAAAAAALFABAAAA|4186|1|9|46|AM|third|night|| +4187|AAAAAAAAMFABAAAA|4187|1|9|47|AM|third|night|| +4188|AAAAAAAANFABAAAA|4188|1|9|48|AM|third|night|| +4189|AAAAAAAAOFABAAAA|4189|1|9|49|AM|third|night|| +4190|AAAAAAAAPFABAAAA|4190|1|9|50|AM|third|night|| +4191|AAAAAAAAAGABAAAA|4191|1|9|51|AM|third|night|| +4192|AAAAAAAABGABAAAA|4192|1|9|52|AM|third|night|| +4193|AAAAAAAACGABAAAA|4193|1|9|53|AM|third|night|| +4194|AAAAAAAADGABAAAA|4194|1|9|54|AM|third|night|| +4195|AAAAAAAAEGABAAAA|4195|1|9|55|AM|third|night|| +4196|AAAAAAAAFGABAAAA|4196|1|9|56|AM|third|night|| +4197|AAAAAAAAGGABAAAA|4197|1|9|57|AM|third|night|| +4198|AAAAAAAAHGABAAAA|4198|1|9|58|AM|third|night|| +4199|AAAAAAAAIGABAAAA|4199|1|9|59|AM|third|night|| +4200|AAAAAAAAJGABAAAA|4200|1|10|0|AM|third|night|| +4201|AAAAAAAAKGABAAAA|4201|1|10|1|AM|third|night|| +4202|AAAAAAAALGABAAAA|4202|1|10|2|AM|third|night|| +4203|AAAAAAAAMGABAAAA|4203|1|10|3|AM|third|night|| +4204|AAAAAAAANGABAAAA|4204|1|10|4|AM|third|night|| +4205|AAAAAAAAOGABAAAA|4205|1|10|5|AM|third|night|| +4206|AAAAAAAAPGABAAAA|4206|1|10|6|AM|third|night|| +4207|AAAAAAAAAHABAAAA|4207|1|10|7|AM|third|night|| +4208|AAAAAAAABHABAAAA|4208|1|10|8|AM|third|night|| +4209|AAAAAAAACHABAAAA|4209|1|10|9|AM|third|night|| +4210|AAAAAAAADHABAAAA|4210|1|10|10|AM|third|night|| +4211|AAAAAAAAEHABAAAA|4211|1|10|11|AM|third|night|| +4212|AAAAAAAAFHABAAAA|4212|1|10|12|AM|third|night|| +4213|AAAAAAAAGHABAAAA|4213|1|10|13|AM|third|night|| +4214|AAAAAAAAHHABAAAA|4214|1|10|14|AM|third|night|| +4215|AAAAAAAAIHABAAAA|4215|1|10|15|AM|third|night|| +4216|AAAAAAAAJHABAAAA|4216|1|10|16|AM|third|night|| +4217|AAAAAAAAKHABAAAA|4217|1|10|17|AM|third|night|| +4218|AAAAAAAALHABAAAA|4218|1|10|18|AM|third|night|| +4219|AAAAAAAAMHABAAAA|4219|1|10|19|AM|third|night|| +4220|AAAAAAAANHABAAAA|4220|1|10|20|AM|third|night|| +4221|AAAAAAAAOHABAAAA|4221|1|10|21|AM|third|night|| +4222|AAAAAAAAPHABAAAA|4222|1|10|22|AM|third|night|| +4223|AAAAAAAAAIABAAAA|4223|1|10|23|AM|third|night|| +4224|AAAAAAAABIABAAAA|4224|1|10|24|AM|third|night|| +4225|AAAAAAAACIABAAAA|4225|1|10|25|AM|third|night|| +4226|AAAAAAAADIABAAAA|4226|1|10|26|AM|third|night|| +4227|AAAAAAAAEIABAAAA|4227|1|10|27|AM|third|night|| +4228|AAAAAAAAFIABAAAA|4228|1|10|28|AM|third|night|| +4229|AAAAAAAAGIABAAAA|4229|1|10|29|AM|third|night|| +4230|AAAAAAAAHIABAAAA|4230|1|10|30|AM|third|night|| +4231|AAAAAAAAIIABAAAA|4231|1|10|31|AM|third|night|| +4232|AAAAAAAAJIABAAAA|4232|1|10|32|AM|third|night|| +4233|AAAAAAAAKIABAAAA|4233|1|10|33|AM|third|night|| +4234|AAAAAAAALIABAAAA|4234|1|10|34|AM|third|night|| +4235|AAAAAAAAMIABAAAA|4235|1|10|35|AM|third|night|| +4236|AAAAAAAANIABAAAA|4236|1|10|36|AM|third|night|| +4237|AAAAAAAAOIABAAAA|4237|1|10|37|AM|third|night|| +4238|AAAAAAAAPIABAAAA|4238|1|10|38|AM|third|night|| +4239|AAAAAAAAAJABAAAA|4239|1|10|39|AM|third|night|| +4240|AAAAAAAABJABAAAA|4240|1|10|40|AM|third|night|| +4241|AAAAAAAACJABAAAA|4241|1|10|41|AM|third|night|| +4242|AAAAAAAADJABAAAA|4242|1|10|42|AM|third|night|| +4243|AAAAAAAAEJABAAAA|4243|1|10|43|AM|third|night|| +4244|AAAAAAAAFJABAAAA|4244|1|10|44|AM|third|night|| +4245|AAAAAAAAGJABAAAA|4245|1|10|45|AM|third|night|| +4246|AAAAAAAAHJABAAAA|4246|1|10|46|AM|third|night|| +4247|AAAAAAAAIJABAAAA|4247|1|10|47|AM|third|night|| +4248|AAAAAAAAJJABAAAA|4248|1|10|48|AM|third|night|| +4249|AAAAAAAAKJABAAAA|4249|1|10|49|AM|third|night|| +4250|AAAAAAAALJABAAAA|4250|1|10|50|AM|third|night|| +4251|AAAAAAAAMJABAAAA|4251|1|10|51|AM|third|night|| +4252|AAAAAAAANJABAAAA|4252|1|10|52|AM|third|night|| +4253|AAAAAAAAOJABAAAA|4253|1|10|53|AM|third|night|| +4254|AAAAAAAAPJABAAAA|4254|1|10|54|AM|third|night|| +4255|AAAAAAAAAKABAAAA|4255|1|10|55|AM|third|night|| +4256|AAAAAAAABKABAAAA|4256|1|10|56|AM|third|night|| +4257|AAAAAAAACKABAAAA|4257|1|10|57|AM|third|night|| +4258|AAAAAAAADKABAAAA|4258|1|10|58|AM|third|night|| +4259|AAAAAAAAEKABAAAA|4259|1|10|59|AM|third|night|| +4260|AAAAAAAAFKABAAAA|4260|1|11|0|AM|third|night|| +4261|AAAAAAAAGKABAAAA|4261|1|11|1|AM|third|night|| +4262|AAAAAAAAHKABAAAA|4262|1|11|2|AM|third|night|| +4263|AAAAAAAAIKABAAAA|4263|1|11|3|AM|third|night|| +4264|AAAAAAAAJKABAAAA|4264|1|11|4|AM|third|night|| +4265|AAAAAAAAKKABAAAA|4265|1|11|5|AM|third|night|| +4266|AAAAAAAALKABAAAA|4266|1|11|6|AM|third|night|| +4267|AAAAAAAAMKABAAAA|4267|1|11|7|AM|third|night|| +4268|AAAAAAAANKABAAAA|4268|1|11|8|AM|third|night|| +4269|AAAAAAAAOKABAAAA|4269|1|11|9|AM|third|night|| +4270|AAAAAAAAPKABAAAA|4270|1|11|10|AM|third|night|| +4271|AAAAAAAAALABAAAA|4271|1|11|11|AM|third|night|| +4272|AAAAAAAABLABAAAA|4272|1|11|12|AM|third|night|| +4273|AAAAAAAACLABAAAA|4273|1|11|13|AM|third|night|| +4274|AAAAAAAADLABAAAA|4274|1|11|14|AM|third|night|| +4275|AAAAAAAAELABAAAA|4275|1|11|15|AM|third|night|| +4276|AAAAAAAAFLABAAAA|4276|1|11|16|AM|third|night|| +4277|AAAAAAAAGLABAAAA|4277|1|11|17|AM|third|night|| +4278|AAAAAAAAHLABAAAA|4278|1|11|18|AM|third|night|| +4279|AAAAAAAAILABAAAA|4279|1|11|19|AM|third|night|| +4280|AAAAAAAAJLABAAAA|4280|1|11|20|AM|third|night|| +4281|AAAAAAAAKLABAAAA|4281|1|11|21|AM|third|night|| +4282|AAAAAAAALLABAAAA|4282|1|11|22|AM|third|night|| +4283|AAAAAAAAMLABAAAA|4283|1|11|23|AM|third|night|| +4284|AAAAAAAANLABAAAA|4284|1|11|24|AM|third|night|| +4285|AAAAAAAAOLABAAAA|4285|1|11|25|AM|third|night|| +4286|AAAAAAAAPLABAAAA|4286|1|11|26|AM|third|night|| +4287|AAAAAAAAAMABAAAA|4287|1|11|27|AM|third|night|| +4288|AAAAAAAABMABAAAA|4288|1|11|28|AM|third|night|| +4289|AAAAAAAACMABAAAA|4289|1|11|29|AM|third|night|| +4290|AAAAAAAADMABAAAA|4290|1|11|30|AM|third|night|| +4291|AAAAAAAAEMABAAAA|4291|1|11|31|AM|third|night|| +4292|AAAAAAAAFMABAAAA|4292|1|11|32|AM|third|night|| +4293|AAAAAAAAGMABAAAA|4293|1|11|33|AM|third|night|| +4294|AAAAAAAAHMABAAAA|4294|1|11|34|AM|third|night|| +4295|AAAAAAAAIMABAAAA|4295|1|11|35|AM|third|night|| +4296|AAAAAAAAJMABAAAA|4296|1|11|36|AM|third|night|| +4297|AAAAAAAAKMABAAAA|4297|1|11|37|AM|third|night|| +4298|AAAAAAAALMABAAAA|4298|1|11|38|AM|third|night|| +4299|AAAAAAAAMMABAAAA|4299|1|11|39|AM|third|night|| +4300|AAAAAAAANMABAAAA|4300|1|11|40|AM|third|night|| +4301|AAAAAAAAOMABAAAA|4301|1|11|41|AM|third|night|| +4302|AAAAAAAAPMABAAAA|4302|1|11|42|AM|third|night|| +4303|AAAAAAAAANABAAAA|4303|1|11|43|AM|third|night|| +4304|AAAAAAAABNABAAAA|4304|1|11|44|AM|third|night|| +4305|AAAAAAAACNABAAAA|4305|1|11|45|AM|third|night|| +4306|AAAAAAAADNABAAAA|4306|1|11|46|AM|third|night|| +4307|AAAAAAAAENABAAAA|4307|1|11|47|AM|third|night|| +4308|AAAAAAAAFNABAAAA|4308|1|11|48|AM|third|night|| +4309|AAAAAAAAGNABAAAA|4309|1|11|49|AM|third|night|| +4310|AAAAAAAAHNABAAAA|4310|1|11|50|AM|third|night|| +4311|AAAAAAAAINABAAAA|4311|1|11|51|AM|third|night|| +4312|AAAAAAAAJNABAAAA|4312|1|11|52|AM|third|night|| +4313|AAAAAAAAKNABAAAA|4313|1|11|53|AM|third|night|| +4314|AAAAAAAALNABAAAA|4314|1|11|54|AM|third|night|| +4315|AAAAAAAAMNABAAAA|4315|1|11|55|AM|third|night|| +4316|AAAAAAAANNABAAAA|4316|1|11|56|AM|third|night|| +4317|AAAAAAAAONABAAAA|4317|1|11|57|AM|third|night|| +4318|AAAAAAAAPNABAAAA|4318|1|11|58|AM|third|night|| +4319|AAAAAAAAAOABAAAA|4319|1|11|59|AM|third|night|| +4320|AAAAAAAABOABAAAA|4320|1|12|0|AM|third|night|| +4321|AAAAAAAACOABAAAA|4321|1|12|1|AM|third|night|| +4322|AAAAAAAADOABAAAA|4322|1|12|2|AM|third|night|| +4323|AAAAAAAAEOABAAAA|4323|1|12|3|AM|third|night|| +4324|AAAAAAAAFOABAAAA|4324|1|12|4|AM|third|night|| +4325|AAAAAAAAGOABAAAA|4325|1|12|5|AM|third|night|| +4326|AAAAAAAAHOABAAAA|4326|1|12|6|AM|third|night|| +4327|AAAAAAAAIOABAAAA|4327|1|12|7|AM|third|night|| +4328|AAAAAAAAJOABAAAA|4328|1|12|8|AM|third|night|| +4329|AAAAAAAAKOABAAAA|4329|1|12|9|AM|third|night|| +4330|AAAAAAAALOABAAAA|4330|1|12|10|AM|third|night|| +4331|AAAAAAAAMOABAAAA|4331|1|12|11|AM|third|night|| +4332|AAAAAAAANOABAAAA|4332|1|12|12|AM|third|night|| +4333|AAAAAAAAOOABAAAA|4333|1|12|13|AM|third|night|| +4334|AAAAAAAAPOABAAAA|4334|1|12|14|AM|third|night|| +4335|AAAAAAAAAPABAAAA|4335|1|12|15|AM|third|night|| +4336|AAAAAAAABPABAAAA|4336|1|12|16|AM|third|night|| +4337|AAAAAAAACPABAAAA|4337|1|12|17|AM|third|night|| +4338|AAAAAAAADPABAAAA|4338|1|12|18|AM|third|night|| +4339|AAAAAAAAEPABAAAA|4339|1|12|19|AM|third|night|| +4340|AAAAAAAAFPABAAAA|4340|1|12|20|AM|third|night|| +4341|AAAAAAAAGPABAAAA|4341|1|12|21|AM|third|night|| +4342|AAAAAAAAHPABAAAA|4342|1|12|22|AM|third|night|| +4343|AAAAAAAAIPABAAAA|4343|1|12|23|AM|third|night|| +4344|AAAAAAAAJPABAAAA|4344|1|12|24|AM|third|night|| +4345|AAAAAAAAKPABAAAA|4345|1|12|25|AM|third|night|| +4346|AAAAAAAALPABAAAA|4346|1|12|26|AM|third|night|| +4347|AAAAAAAAMPABAAAA|4347|1|12|27|AM|third|night|| +4348|AAAAAAAANPABAAAA|4348|1|12|28|AM|third|night|| +4349|AAAAAAAAOPABAAAA|4349|1|12|29|AM|third|night|| +4350|AAAAAAAAPPABAAAA|4350|1|12|30|AM|third|night|| +4351|AAAAAAAAAABBAAAA|4351|1|12|31|AM|third|night|| +4352|AAAAAAAABABBAAAA|4352|1|12|32|AM|third|night|| +4353|AAAAAAAACABBAAAA|4353|1|12|33|AM|third|night|| +4354|AAAAAAAADABBAAAA|4354|1|12|34|AM|third|night|| +4355|AAAAAAAAEABBAAAA|4355|1|12|35|AM|third|night|| +4356|AAAAAAAAFABBAAAA|4356|1|12|36|AM|third|night|| +4357|AAAAAAAAGABBAAAA|4357|1|12|37|AM|third|night|| +4358|AAAAAAAAHABBAAAA|4358|1|12|38|AM|third|night|| +4359|AAAAAAAAIABBAAAA|4359|1|12|39|AM|third|night|| +4360|AAAAAAAAJABBAAAA|4360|1|12|40|AM|third|night|| +4361|AAAAAAAAKABBAAAA|4361|1|12|41|AM|third|night|| +4362|AAAAAAAALABBAAAA|4362|1|12|42|AM|third|night|| +4363|AAAAAAAAMABBAAAA|4363|1|12|43|AM|third|night|| +4364|AAAAAAAANABBAAAA|4364|1|12|44|AM|third|night|| +4365|AAAAAAAAOABBAAAA|4365|1|12|45|AM|third|night|| +4366|AAAAAAAAPABBAAAA|4366|1|12|46|AM|third|night|| +4367|AAAAAAAAABBBAAAA|4367|1|12|47|AM|third|night|| +4368|AAAAAAAABBBBAAAA|4368|1|12|48|AM|third|night|| +4369|AAAAAAAACBBBAAAA|4369|1|12|49|AM|third|night|| +4370|AAAAAAAADBBBAAAA|4370|1|12|50|AM|third|night|| +4371|AAAAAAAAEBBBAAAA|4371|1|12|51|AM|third|night|| +4372|AAAAAAAAFBBBAAAA|4372|1|12|52|AM|third|night|| +4373|AAAAAAAAGBBBAAAA|4373|1|12|53|AM|third|night|| +4374|AAAAAAAAHBBBAAAA|4374|1|12|54|AM|third|night|| +4375|AAAAAAAAIBBBAAAA|4375|1|12|55|AM|third|night|| +4376|AAAAAAAAJBBBAAAA|4376|1|12|56|AM|third|night|| +4377|AAAAAAAAKBBBAAAA|4377|1|12|57|AM|third|night|| +4378|AAAAAAAALBBBAAAA|4378|1|12|58|AM|third|night|| +4379|AAAAAAAAMBBBAAAA|4379|1|12|59|AM|third|night|| +4380|AAAAAAAANBBBAAAA|4380|1|13|0|AM|third|night|| +4381|AAAAAAAAOBBBAAAA|4381|1|13|1|AM|third|night|| +4382|AAAAAAAAPBBBAAAA|4382|1|13|2|AM|third|night|| +4383|AAAAAAAAACBBAAAA|4383|1|13|3|AM|third|night|| +4384|AAAAAAAABCBBAAAA|4384|1|13|4|AM|third|night|| +4385|AAAAAAAACCBBAAAA|4385|1|13|5|AM|third|night|| +4386|AAAAAAAADCBBAAAA|4386|1|13|6|AM|third|night|| +4387|AAAAAAAAECBBAAAA|4387|1|13|7|AM|third|night|| +4388|AAAAAAAAFCBBAAAA|4388|1|13|8|AM|third|night|| +4389|AAAAAAAAGCBBAAAA|4389|1|13|9|AM|third|night|| +4390|AAAAAAAAHCBBAAAA|4390|1|13|10|AM|third|night|| +4391|AAAAAAAAICBBAAAA|4391|1|13|11|AM|third|night|| +4392|AAAAAAAAJCBBAAAA|4392|1|13|12|AM|third|night|| +4393|AAAAAAAAKCBBAAAA|4393|1|13|13|AM|third|night|| +4394|AAAAAAAALCBBAAAA|4394|1|13|14|AM|third|night|| +4395|AAAAAAAAMCBBAAAA|4395|1|13|15|AM|third|night|| +4396|AAAAAAAANCBBAAAA|4396|1|13|16|AM|third|night|| +4397|AAAAAAAAOCBBAAAA|4397|1|13|17|AM|third|night|| +4398|AAAAAAAAPCBBAAAA|4398|1|13|18|AM|third|night|| +4399|AAAAAAAAADBBAAAA|4399|1|13|19|AM|third|night|| +4400|AAAAAAAABDBBAAAA|4400|1|13|20|AM|third|night|| +4401|AAAAAAAACDBBAAAA|4401|1|13|21|AM|third|night|| +4402|AAAAAAAADDBBAAAA|4402|1|13|22|AM|third|night|| +4403|AAAAAAAAEDBBAAAA|4403|1|13|23|AM|third|night|| +4404|AAAAAAAAFDBBAAAA|4404|1|13|24|AM|third|night|| +4405|AAAAAAAAGDBBAAAA|4405|1|13|25|AM|third|night|| +4406|AAAAAAAAHDBBAAAA|4406|1|13|26|AM|third|night|| +4407|AAAAAAAAIDBBAAAA|4407|1|13|27|AM|third|night|| +4408|AAAAAAAAJDBBAAAA|4408|1|13|28|AM|third|night|| +4409|AAAAAAAAKDBBAAAA|4409|1|13|29|AM|third|night|| +4410|AAAAAAAALDBBAAAA|4410|1|13|30|AM|third|night|| +4411|AAAAAAAAMDBBAAAA|4411|1|13|31|AM|third|night|| +4412|AAAAAAAANDBBAAAA|4412|1|13|32|AM|third|night|| +4413|AAAAAAAAODBBAAAA|4413|1|13|33|AM|third|night|| +4414|AAAAAAAAPDBBAAAA|4414|1|13|34|AM|third|night|| +4415|AAAAAAAAAEBBAAAA|4415|1|13|35|AM|third|night|| +4416|AAAAAAAABEBBAAAA|4416|1|13|36|AM|third|night|| +4417|AAAAAAAACEBBAAAA|4417|1|13|37|AM|third|night|| +4418|AAAAAAAADEBBAAAA|4418|1|13|38|AM|third|night|| +4419|AAAAAAAAEEBBAAAA|4419|1|13|39|AM|third|night|| +4420|AAAAAAAAFEBBAAAA|4420|1|13|40|AM|third|night|| +4421|AAAAAAAAGEBBAAAA|4421|1|13|41|AM|third|night|| +4422|AAAAAAAAHEBBAAAA|4422|1|13|42|AM|third|night|| +4423|AAAAAAAAIEBBAAAA|4423|1|13|43|AM|third|night|| +4424|AAAAAAAAJEBBAAAA|4424|1|13|44|AM|third|night|| +4425|AAAAAAAAKEBBAAAA|4425|1|13|45|AM|third|night|| +4426|AAAAAAAALEBBAAAA|4426|1|13|46|AM|third|night|| +4427|AAAAAAAAMEBBAAAA|4427|1|13|47|AM|third|night|| +4428|AAAAAAAANEBBAAAA|4428|1|13|48|AM|third|night|| +4429|AAAAAAAAOEBBAAAA|4429|1|13|49|AM|third|night|| +4430|AAAAAAAAPEBBAAAA|4430|1|13|50|AM|third|night|| +4431|AAAAAAAAAFBBAAAA|4431|1|13|51|AM|third|night|| +4432|AAAAAAAABFBBAAAA|4432|1|13|52|AM|third|night|| +4433|AAAAAAAACFBBAAAA|4433|1|13|53|AM|third|night|| +4434|AAAAAAAADFBBAAAA|4434|1|13|54|AM|third|night|| +4435|AAAAAAAAEFBBAAAA|4435|1|13|55|AM|third|night|| +4436|AAAAAAAAFFBBAAAA|4436|1|13|56|AM|third|night|| +4437|AAAAAAAAGFBBAAAA|4437|1|13|57|AM|third|night|| +4438|AAAAAAAAHFBBAAAA|4438|1|13|58|AM|third|night|| +4439|AAAAAAAAIFBBAAAA|4439|1|13|59|AM|third|night|| +4440|AAAAAAAAJFBBAAAA|4440|1|14|0|AM|third|night|| +4441|AAAAAAAAKFBBAAAA|4441|1|14|1|AM|third|night|| +4442|AAAAAAAALFBBAAAA|4442|1|14|2|AM|third|night|| +4443|AAAAAAAAMFBBAAAA|4443|1|14|3|AM|third|night|| +4444|AAAAAAAANFBBAAAA|4444|1|14|4|AM|third|night|| +4445|AAAAAAAAOFBBAAAA|4445|1|14|5|AM|third|night|| +4446|AAAAAAAAPFBBAAAA|4446|1|14|6|AM|third|night|| +4447|AAAAAAAAAGBBAAAA|4447|1|14|7|AM|third|night|| +4448|AAAAAAAABGBBAAAA|4448|1|14|8|AM|third|night|| +4449|AAAAAAAACGBBAAAA|4449|1|14|9|AM|third|night|| +4450|AAAAAAAADGBBAAAA|4450|1|14|10|AM|third|night|| +4451|AAAAAAAAEGBBAAAA|4451|1|14|11|AM|third|night|| +4452|AAAAAAAAFGBBAAAA|4452|1|14|12|AM|third|night|| +4453|AAAAAAAAGGBBAAAA|4453|1|14|13|AM|third|night|| +4454|AAAAAAAAHGBBAAAA|4454|1|14|14|AM|third|night|| +4455|AAAAAAAAIGBBAAAA|4455|1|14|15|AM|third|night|| +4456|AAAAAAAAJGBBAAAA|4456|1|14|16|AM|third|night|| +4457|AAAAAAAAKGBBAAAA|4457|1|14|17|AM|third|night|| +4458|AAAAAAAALGBBAAAA|4458|1|14|18|AM|third|night|| +4459|AAAAAAAAMGBBAAAA|4459|1|14|19|AM|third|night|| +4460|AAAAAAAANGBBAAAA|4460|1|14|20|AM|third|night|| +4461|AAAAAAAAOGBBAAAA|4461|1|14|21|AM|third|night|| +4462|AAAAAAAAPGBBAAAA|4462|1|14|22|AM|third|night|| +4463|AAAAAAAAAHBBAAAA|4463|1|14|23|AM|third|night|| +4464|AAAAAAAABHBBAAAA|4464|1|14|24|AM|third|night|| +4465|AAAAAAAACHBBAAAA|4465|1|14|25|AM|third|night|| +4466|AAAAAAAADHBBAAAA|4466|1|14|26|AM|third|night|| +4467|AAAAAAAAEHBBAAAA|4467|1|14|27|AM|third|night|| +4468|AAAAAAAAFHBBAAAA|4468|1|14|28|AM|third|night|| +4469|AAAAAAAAGHBBAAAA|4469|1|14|29|AM|third|night|| +4470|AAAAAAAAHHBBAAAA|4470|1|14|30|AM|third|night|| +4471|AAAAAAAAIHBBAAAA|4471|1|14|31|AM|third|night|| +4472|AAAAAAAAJHBBAAAA|4472|1|14|32|AM|third|night|| +4473|AAAAAAAAKHBBAAAA|4473|1|14|33|AM|third|night|| +4474|AAAAAAAALHBBAAAA|4474|1|14|34|AM|third|night|| +4475|AAAAAAAAMHBBAAAA|4475|1|14|35|AM|third|night|| +4476|AAAAAAAANHBBAAAA|4476|1|14|36|AM|third|night|| +4477|AAAAAAAAOHBBAAAA|4477|1|14|37|AM|third|night|| +4478|AAAAAAAAPHBBAAAA|4478|1|14|38|AM|third|night|| +4479|AAAAAAAAAIBBAAAA|4479|1|14|39|AM|third|night|| +4480|AAAAAAAABIBBAAAA|4480|1|14|40|AM|third|night|| +4481|AAAAAAAACIBBAAAA|4481|1|14|41|AM|third|night|| +4482|AAAAAAAADIBBAAAA|4482|1|14|42|AM|third|night|| +4483|AAAAAAAAEIBBAAAA|4483|1|14|43|AM|third|night|| +4484|AAAAAAAAFIBBAAAA|4484|1|14|44|AM|third|night|| +4485|AAAAAAAAGIBBAAAA|4485|1|14|45|AM|third|night|| +4486|AAAAAAAAHIBBAAAA|4486|1|14|46|AM|third|night|| +4487|AAAAAAAAIIBBAAAA|4487|1|14|47|AM|third|night|| +4488|AAAAAAAAJIBBAAAA|4488|1|14|48|AM|third|night|| +4489|AAAAAAAAKIBBAAAA|4489|1|14|49|AM|third|night|| +4490|AAAAAAAALIBBAAAA|4490|1|14|50|AM|third|night|| +4491|AAAAAAAAMIBBAAAA|4491|1|14|51|AM|third|night|| +4492|AAAAAAAANIBBAAAA|4492|1|14|52|AM|third|night|| +4493|AAAAAAAAOIBBAAAA|4493|1|14|53|AM|third|night|| +4494|AAAAAAAAPIBBAAAA|4494|1|14|54|AM|third|night|| +4495|AAAAAAAAAJBBAAAA|4495|1|14|55|AM|third|night|| +4496|AAAAAAAABJBBAAAA|4496|1|14|56|AM|third|night|| +4497|AAAAAAAACJBBAAAA|4497|1|14|57|AM|third|night|| +4498|AAAAAAAADJBBAAAA|4498|1|14|58|AM|third|night|| +4499|AAAAAAAAEJBBAAAA|4499|1|14|59|AM|third|night|| +4500|AAAAAAAAFJBBAAAA|4500|1|15|0|AM|third|night|| +4501|AAAAAAAAGJBBAAAA|4501|1|15|1|AM|third|night|| +4502|AAAAAAAAHJBBAAAA|4502|1|15|2|AM|third|night|| +4503|AAAAAAAAIJBBAAAA|4503|1|15|3|AM|third|night|| +4504|AAAAAAAAJJBBAAAA|4504|1|15|4|AM|third|night|| +4505|AAAAAAAAKJBBAAAA|4505|1|15|5|AM|third|night|| +4506|AAAAAAAALJBBAAAA|4506|1|15|6|AM|third|night|| +4507|AAAAAAAAMJBBAAAA|4507|1|15|7|AM|third|night|| +4508|AAAAAAAANJBBAAAA|4508|1|15|8|AM|third|night|| +4509|AAAAAAAAOJBBAAAA|4509|1|15|9|AM|third|night|| +4510|AAAAAAAAPJBBAAAA|4510|1|15|10|AM|third|night|| +4511|AAAAAAAAAKBBAAAA|4511|1|15|11|AM|third|night|| +4512|AAAAAAAABKBBAAAA|4512|1|15|12|AM|third|night|| +4513|AAAAAAAACKBBAAAA|4513|1|15|13|AM|third|night|| +4514|AAAAAAAADKBBAAAA|4514|1|15|14|AM|third|night|| +4515|AAAAAAAAEKBBAAAA|4515|1|15|15|AM|third|night|| +4516|AAAAAAAAFKBBAAAA|4516|1|15|16|AM|third|night|| +4517|AAAAAAAAGKBBAAAA|4517|1|15|17|AM|third|night|| +4518|AAAAAAAAHKBBAAAA|4518|1|15|18|AM|third|night|| +4519|AAAAAAAAIKBBAAAA|4519|1|15|19|AM|third|night|| +4520|AAAAAAAAJKBBAAAA|4520|1|15|20|AM|third|night|| +4521|AAAAAAAAKKBBAAAA|4521|1|15|21|AM|third|night|| +4522|AAAAAAAALKBBAAAA|4522|1|15|22|AM|third|night|| +4523|AAAAAAAAMKBBAAAA|4523|1|15|23|AM|third|night|| +4524|AAAAAAAANKBBAAAA|4524|1|15|24|AM|third|night|| +4525|AAAAAAAAOKBBAAAA|4525|1|15|25|AM|third|night|| +4526|AAAAAAAAPKBBAAAA|4526|1|15|26|AM|third|night|| +4527|AAAAAAAAALBBAAAA|4527|1|15|27|AM|third|night|| +4528|AAAAAAAABLBBAAAA|4528|1|15|28|AM|third|night|| +4529|AAAAAAAACLBBAAAA|4529|1|15|29|AM|third|night|| +4530|AAAAAAAADLBBAAAA|4530|1|15|30|AM|third|night|| +4531|AAAAAAAAELBBAAAA|4531|1|15|31|AM|third|night|| +4532|AAAAAAAAFLBBAAAA|4532|1|15|32|AM|third|night|| +4533|AAAAAAAAGLBBAAAA|4533|1|15|33|AM|third|night|| +4534|AAAAAAAAHLBBAAAA|4534|1|15|34|AM|third|night|| +4535|AAAAAAAAILBBAAAA|4535|1|15|35|AM|third|night|| +4536|AAAAAAAAJLBBAAAA|4536|1|15|36|AM|third|night|| +4537|AAAAAAAAKLBBAAAA|4537|1|15|37|AM|third|night|| +4538|AAAAAAAALLBBAAAA|4538|1|15|38|AM|third|night|| +4539|AAAAAAAAMLBBAAAA|4539|1|15|39|AM|third|night|| +4540|AAAAAAAANLBBAAAA|4540|1|15|40|AM|third|night|| +4541|AAAAAAAAOLBBAAAA|4541|1|15|41|AM|third|night|| +4542|AAAAAAAAPLBBAAAA|4542|1|15|42|AM|third|night|| +4543|AAAAAAAAAMBBAAAA|4543|1|15|43|AM|third|night|| +4544|AAAAAAAABMBBAAAA|4544|1|15|44|AM|third|night|| +4545|AAAAAAAACMBBAAAA|4545|1|15|45|AM|third|night|| +4546|AAAAAAAADMBBAAAA|4546|1|15|46|AM|third|night|| +4547|AAAAAAAAEMBBAAAA|4547|1|15|47|AM|third|night|| +4548|AAAAAAAAFMBBAAAA|4548|1|15|48|AM|third|night|| +4549|AAAAAAAAGMBBAAAA|4549|1|15|49|AM|third|night|| +4550|AAAAAAAAHMBBAAAA|4550|1|15|50|AM|third|night|| +4551|AAAAAAAAIMBBAAAA|4551|1|15|51|AM|third|night|| +4552|AAAAAAAAJMBBAAAA|4552|1|15|52|AM|third|night|| +4553|AAAAAAAAKMBBAAAA|4553|1|15|53|AM|third|night|| +4554|AAAAAAAALMBBAAAA|4554|1|15|54|AM|third|night|| +4555|AAAAAAAAMMBBAAAA|4555|1|15|55|AM|third|night|| +4556|AAAAAAAANMBBAAAA|4556|1|15|56|AM|third|night|| +4557|AAAAAAAAOMBBAAAA|4557|1|15|57|AM|third|night|| +4558|AAAAAAAAPMBBAAAA|4558|1|15|58|AM|third|night|| +4559|AAAAAAAAANBBAAAA|4559|1|15|59|AM|third|night|| +4560|AAAAAAAABNBBAAAA|4560|1|16|0|AM|third|night|| +4561|AAAAAAAACNBBAAAA|4561|1|16|1|AM|third|night|| +4562|AAAAAAAADNBBAAAA|4562|1|16|2|AM|third|night|| +4563|AAAAAAAAENBBAAAA|4563|1|16|3|AM|third|night|| +4564|AAAAAAAAFNBBAAAA|4564|1|16|4|AM|third|night|| +4565|AAAAAAAAGNBBAAAA|4565|1|16|5|AM|third|night|| +4566|AAAAAAAAHNBBAAAA|4566|1|16|6|AM|third|night|| +4567|AAAAAAAAINBBAAAA|4567|1|16|7|AM|third|night|| +4568|AAAAAAAAJNBBAAAA|4568|1|16|8|AM|third|night|| +4569|AAAAAAAAKNBBAAAA|4569|1|16|9|AM|third|night|| +4570|AAAAAAAALNBBAAAA|4570|1|16|10|AM|third|night|| +4571|AAAAAAAAMNBBAAAA|4571|1|16|11|AM|third|night|| +4572|AAAAAAAANNBBAAAA|4572|1|16|12|AM|third|night|| +4573|AAAAAAAAONBBAAAA|4573|1|16|13|AM|third|night|| +4574|AAAAAAAAPNBBAAAA|4574|1|16|14|AM|third|night|| +4575|AAAAAAAAAOBBAAAA|4575|1|16|15|AM|third|night|| +4576|AAAAAAAABOBBAAAA|4576|1|16|16|AM|third|night|| +4577|AAAAAAAACOBBAAAA|4577|1|16|17|AM|third|night|| +4578|AAAAAAAADOBBAAAA|4578|1|16|18|AM|third|night|| +4579|AAAAAAAAEOBBAAAA|4579|1|16|19|AM|third|night|| +4580|AAAAAAAAFOBBAAAA|4580|1|16|20|AM|third|night|| +4581|AAAAAAAAGOBBAAAA|4581|1|16|21|AM|third|night|| +4582|AAAAAAAAHOBBAAAA|4582|1|16|22|AM|third|night|| +4583|AAAAAAAAIOBBAAAA|4583|1|16|23|AM|third|night|| +4584|AAAAAAAAJOBBAAAA|4584|1|16|24|AM|third|night|| +4585|AAAAAAAAKOBBAAAA|4585|1|16|25|AM|third|night|| +4586|AAAAAAAALOBBAAAA|4586|1|16|26|AM|third|night|| +4587|AAAAAAAAMOBBAAAA|4587|1|16|27|AM|third|night|| +4588|AAAAAAAANOBBAAAA|4588|1|16|28|AM|third|night|| +4589|AAAAAAAAOOBBAAAA|4589|1|16|29|AM|third|night|| +4590|AAAAAAAAPOBBAAAA|4590|1|16|30|AM|third|night|| +4591|AAAAAAAAAPBBAAAA|4591|1|16|31|AM|third|night|| +4592|AAAAAAAABPBBAAAA|4592|1|16|32|AM|third|night|| +4593|AAAAAAAACPBBAAAA|4593|1|16|33|AM|third|night|| +4594|AAAAAAAADPBBAAAA|4594|1|16|34|AM|third|night|| +4595|AAAAAAAAEPBBAAAA|4595|1|16|35|AM|third|night|| +4596|AAAAAAAAFPBBAAAA|4596|1|16|36|AM|third|night|| +4597|AAAAAAAAGPBBAAAA|4597|1|16|37|AM|third|night|| +4598|AAAAAAAAHPBBAAAA|4598|1|16|38|AM|third|night|| +4599|AAAAAAAAIPBBAAAA|4599|1|16|39|AM|third|night|| +4600|AAAAAAAAJPBBAAAA|4600|1|16|40|AM|third|night|| +4601|AAAAAAAAKPBBAAAA|4601|1|16|41|AM|third|night|| +4602|AAAAAAAALPBBAAAA|4602|1|16|42|AM|third|night|| +4603|AAAAAAAAMPBBAAAA|4603|1|16|43|AM|third|night|| +4604|AAAAAAAANPBBAAAA|4604|1|16|44|AM|third|night|| +4605|AAAAAAAAOPBBAAAA|4605|1|16|45|AM|third|night|| +4606|AAAAAAAAPPBBAAAA|4606|1|16|46|AM|third|night|| +4607|AAAAAAAAAACBAAAA|4607|1|16|47|AM|third|night|| +4608|AAAAAAAABACBAAAA|4608|1|16|48|AM|third|night|| +4609|AAAAAAAACACBAAAA|4609|1|16|49|AM|third|night|| +4610|AAAAAAAADACBAAAA|4610|1|16|50|AM|third|night|| +4611|AAAAAAAAEACBAAAA|4611|1|16|51|AM|third|night|| +4612|AAAAAAAAFACBAAAA|4612|1|16|52|AM|third|night|| +4613|AAAAAAAAGACBAAAA|4613|1|16|53|AM|third|night|| +4614|AAAAAAAAHACBAAAA|4614|1|16|54|AM|third|night|| +4615|AAAAAAAAIACBAAAA|4615|1|16|55|AM|third|night|| +4616|AAAAAAAAJACBAAAA|4616|1|16|56|AM|third|night|| +4617|AAAAAAAAKACBAAAA|4617|1|16|57|AM|third|night|| +4618|AAAAAAAALACBAAAA|4618|1|16|58|AM|third|night|| +4619|AAAAAAAAMACBAAAA|4619|1|16|59|AM|third|night|| +4620|AAAAAAAANACBAAAA|4620|1|17|0|AM|third|night|| +4621|AAAAAAAAOACBAAAA|4621|1|17|1|AM|third|night|| +4622|AAAAAAAAPACBAAAA|4622|1|17|2|AM|third|night|| +4623|AAAAAAAAABCBAAAA|4623|1|17|3|AM|third|night|| +4624|AAAAAAAABBCBAAAA|4624|1|17|4|AM|third|night|| +4625|AAAAAAAACBCBAAAA|4625|1|17|5|AM|third|night|| +4626|AAAAAAAADBCBAAAA|4626|1|17|6|AM|third|night|| +4627|AAAAAAAAEBCBAAAA|4627|1|17|7|AM|third|night|| +4628|AAAAAAAAFBCBAAAA|4628|1|17|8|AM|third|night|| +4629|AAAAAAAAGBCBAAAA|4629|1|17|9|AM|third|night|| +4630|AAAAAAAAHBCBAAAA|4630|1|17|10|AM|third|night|| +4631|AAAAAAAAIBCBAAAA|4631|1|17|11|AM|third|night|| +4632|AAAAAAAAJBCBAAAA|4632|1|17|12|AM|third|night|| +4633|AAAAAAAAKBCBAAAA|4633|1|17|13|AM|third|night|| +4634|AAAAAAAALBCBAAAA|4634|1|17|14|AM|third|night|| +4635|AAAAAAAAMBCBAAAA|4635|1|17|15|AM|third|night|| +4636|AAAAAAAANBCBAAAA|4636|1|17|16|AM|third|night|| +4637|AAAAAAAAOBCBAAAA|4637|1|17|17|AM|third|night|| +4638|AAAAAAAAPBCBAAAA|4638|1|17|18|AM|third|night|| +4639|AAAAAAAAACCBAAAA|4639|1|17|19|AM|third|night|| +4640|AAAAAAAABCCBAAAA|4640|1|17|20|AM|third|night|| +4641|AAAAAAAACCCBAAAA|4641|1|17|21|AM|third|night|| +4642|AAAAAAAADCCBAAAA|4642|1|17|22|AM|third|night|| +4643|AAAAAAAAECCBAAAA|4643|1|17|23|AM|third|night|| +4644|AAAAAAAAFCCBAAAA|4644|1|17|24|AM|third|night|| +4645|AAAAAAAAGCCBAAAA|4645|1|17|25|AM|third|night|| +4646|AAAAAAAAHCCBAAAA|4646|1|17|26|AM|third|night|| +4647|AAAAAAAAICCBAAAA|4647|1|17|27|AM|third|night|| +4648|AAAAAAAAJCCBAAAA|4648|1|17|28|AM|third|night|| +4649|AAAAAAAAKCCBAAAA|4649|1|17|29|AM|third|night|| +4650|AAAAAAAALCCBAAAA|4650|1|17|30|AM|third|night|| +4651|AAAAAAAAMCCBAAAA|4651|1|17|31|AM|third|night|| +4652|AAAAAAAANCCBAAAA|4652|1|17|32|AM|third|night|| +4653|AAAAAAAAOCCBAAAA|4653|1|17|33|AM|third|night|| +4654|AAAAAAAAPCCBAAAA|4654|1|17|34|AM|third|night|| +4655|AAAAAAAAADCBAAAA|4655|1|17|35|AM|third|night|| +4656|AAAAAAAABDCBAAAA|4656|1|17|36|AM|third|night|| +4657|AAAAAAAACDCBAAAA|4657|1|17|37|AM|third|night|| +4658|AAAAAAAADDCBAAAA|4658|1|17|38|AM|third|night|| +4659|AAAAAAAAEDCBAAAA|4659|1|17|39|AM|third|night|| +4660|AAAAAAAAFDCBAAAA|4660|1|17|40|AM|third|night|| +4661|AAAAAAAAGDCBAAAA|4661|1|17|41|AM|third|night|| +4662|AAAAAAAAHDCBAAAA|4662|1|17|42|AM|third|night|| +4663|AAAAAAAAIDCBAAAA|4663|1|17|43|AM|third|night|| +4664|AAAAAAAAJDCBAAAA|4664|1|17|44|AM|third|night|| +4665|AAAAAAAAKDCBAAAA|4665|1|17|45|AM|third|night|| +4666|AAAAAAAALDCBAAAA|4666|1|17|46|AM|third|night|| +4667|AAAAAAAAMDCBAAAA|4667|1|17|47|AM|third|night|| +4668|AAAAAAAANDCBAAAA|4668|1|17|48|AM|third|night|| +4669|AAAAAAAAODCBAAAA|4669|1|17|49|AM|third|night|| +4670|AAAAAAAAPDCBAAAA|4670|1|17|50|AM|third|night|| +4671|AAAAAAAAAECBAAAA|4671|1|17|51|AM|third|night|| +4672|AAAAAAAABECBAAAA|4672|1|17|52|AM|third|night|| +4673|AAAAAAAACECBAAAA|4673|1|17|53|AM|third|night|| +4674|AAAAAAAADECBAAAA|4674|1|17|54|AM|third|night|| +4675|AAAAAAAAEECBAAAA|4675|1|17|55|AM|third|night|| +4676|AAAAAAAAFECBAAAA|4676|1|17|56|AM|third|night|| +4677|AAAAAAAAGECBAAAA|4677|1|17|57|AM|third|night|| +4678|AAAAAAAAHECBAAAA|4678|1|17|58|AM|third|night|| +4679|AAAAAAAAIECBAAAA|4679|1|17|59|AM|third|night|| +4680|AAAAAAAAJECBAAAA|4680|1|18|0|AM|third|night|| +4681|AAAAAAAAKECBAAAA|4681|1|18|1|AM|third|night|| +4682|AAAAAAAALECBAAAA|4682|1|18|2|AM|third|night|| +4683|AAAAAAAAMECBAAAA|4683|1|18|3|AM|third|night|| +4684|AAAAAAAANECBAAAA|4684|1|18|4|AM|third|night|| +4685|AAAAAAAAOECBAAAA|4685|1|18|5|AM|third|night|| +4686|AAAAAAAAPECBAAAA|4686|1|18|6|AM|third|night|| +4687|AAAAAAAAAFCBAAAA|4687|1|18|7|AM|third|night|| +4688|AAAAAAAABFCBAAAA|4688|1|18|8|AM|third|night|| +4689|AAAAAAAACFCBAAAA|4689|1|18|9|AM|third|night|| +4690|AAAAAAAADFCBAAAA|4690|1|18|10|AM|third|night|| +4691|AAAAAAAAEFCBAAAA|4691|1|18|11|AM|third|night|| +4692|AAAAAAAAFFCBAAAA|4692|1|18|12|AM|third|night|| +4693|AAAAAAAAGFCBAAAA|4693|1|18|13|AM|third|night|| +4694|AAAAAAAAHFCBAAAA|4694|1|18|14|AM|third|night|| +4695|AAAAAAAAIFCBAAAA|4695|1|18|15|AM|third|night|| +4696|AAAAAAAAJFCBAAAA|4696|1|18|16|AM|third|night|| +4697|AAAAAAAAKFCBAAAA|4697|1|18|17|AM|third|night|| +4698|AAAAAAAALFCBAAAA|4698|1|18|18|AM|third|night|| +4699|AAAAAAAAMFCBAAAA|4699|1|18|19|AM|third|night|| +4700|AAAAAAAANFCBAAAA|4700|1|18|20|AM|third|night|| +4701|AAAAAAAAOFCBAAAA|4701|1|18|21|AM|third|night|| +4702|AAAAAAAAPFCBAAAA|4702|1|18|22|AM|third|night|| +4703|AAAAAAAAAGCBAAAA|4703|1|18|23|AM|third|night|| +4704|AAAAAAAABGCBAAAA|4704|1|18|24|AM|third|night|| +4705|AAAAAAAACGCBAAAA|4705|1|18|25|AM|third|night|| +4706|AAAAAAAADGCBAAAA|4706|1|18|26|AM|third|night|| +4707|AAAAAAAAEGCBAAAA|4707|1|18|27|AM|third|night|| +4708|AAAAAAAAFGCBAAAA|4708|1|18|28|AM|third|night|| +4709|AAAAAAAAGGCBAAAA|4709|1|18|29|AM|third|night|| +4710|AAAAAAAAHGCBAAAA|4710|1|18|30|AM|third|night|| +4711|AAAAAAAAIGCBAAAA|4711|1|18|31|AM|third|night|| +4712|AAAAAAAAJGCBAAAA|4712|1|18|32|AM|third|night|| +4713|AAAAAAAAKGCBAAAA|4713|1|18|33|AM|third|night|| +4714|AAAAAAAALGCBAAAA|4714|1|18|34|AM|third|night|| +4715|AAAAAAAAMGCBAAAA|4715|1|18|35|AM|third|night|| +4716|AAAAAAAANGCBAAAA|4716|1|18|36|AM|third|night|| +4717|AAAAAAAAOGCBAAAA|4717|1|18|37|AM|third|night|| +4718|AAAAAAAAPGCBAAAA|4718|1|18|38|AM|third|night|| +4719|AAAAAAAAAHCBAAAA|4719|1|18|39|AM|third|night|| +4720|AAAAAAAABHCBAAAA|4720|1|18|40|AM|third|night|| +4721|AAAAAAAACHCBAAAA|4721|1|18|41|AM|third|night|| +4722|AAAAAAAADHCBAAAA|4722|1|18|42|AM|third|night|| +4723|AAAAAAAAEHCBAAAA|4723|1|18|43|AM|third|night|| +4724|AAAAAAAAFHCBAAAA|4724|1|18|44|AM|third|night|| +4725|AAAAAAAAGHCBAAAA|4725|1|18|45|AM|third|night|| +4726|AAAAAAAAHHCBAAAA|4726|1|18|46|AM|third|night|| +4727|AAAAAAAAIHCBAAAA|4727|1|18|47|AM|third|night|| +4728|AAAAAAAAJHCBAAAA|4728|1|18|48|AM|third|night|| +4729|AAAAAAAAKHCBAAAA|4729|1|18|49|AM|third|night|| +4730|AAAAAAAALHCBAAAA|4730|1|18|50|AM|third|night|| +4731|AAAAAAAAMHCBAAAA|4731|1|18|51|AM|third|night|| +4732|AAAAAAAANHCBAAAA|4732|1|18|52|AM|third|night|| +4733|AAAAAAAAOHCBAAAA|4733|1|18|53|AM|third|night|| +4734|AAAAAAAAPHCBAAAA|4734|1|18|54|AM|third|night|| +4735|AAAAAAAAAICBAAAA|4735|1|18|55|AM|third|night|| +4736|AAAAAAAABICBAAAA|4736|1|18|56|AM|third|night|| +4737|AAAAAAAACICBAAAA|4737|1|18|57|AM|third|night|| +4738|AAAAAAAADICBAAAA|4738|1|18|58|AM|third|night|| +4739|AAAAAAAAEICBAAAA|4739|1|18|59|AM|third|night|| +4740|AAAAAAAAFICBAAAA|4740|1|19|0|AM|third|night|| +4741|AAAAAAAAGICBAAAA|4741|1|19|1|AM|third|night|| +4742|AAAAAAAAHICBAAAA|4742|1|19|2|AM|third|night|| +4743|AAAAAAAAIICBAAAA|4743|1|19|3|AM|third|night|| +4744|AAAAAAAAJICBAAAA|4744|1|19|4|AM|third|night|| +4745|AAAAAAAAKICBAAAA|4745|1|19|5|AM|third|night|| +4746|AAAAAAAALICBAAAA|4746|1|19|6|AM|third|night|| +4747|AAAAAAAAMICBAAAA|4747|1|19|7|AM|third|night|| +4748|AAAAAAAANICBAAAA|4748|1|19|8|AM|third|night|| +4749|AAAAAAAAOICBAAAA|4749|1|19|9|AM|third|night|| +4750|AAAAAAAAPICBAAAA|4750|1|19|10|AM|third|night|| +4751|AAAAAAAAAJCBAAAA|4751|1|19|11|AM|third|night|| +4752|AAAAAAAABJCBAAAA|4752|1|19|12|AM|third|night|| +4753|AAAAAAAACJCBAAAA|4753|1|19|13|AM|third|night|| +4754|AAAAAAAADJCBAAAA|4754|1|19|14|AM|third|night|| +4755|AAAAAAAAEJCBAAAA|4755|1|19|15|AM|third|night|| +4756|AAAAAAAAFJCBAAAA|4756|1|19|16|AM|third|night|| +4757|AAAAAAAAGJCBAAAA|4757|1|19|17|AM|third|night|| +4758|AAAAAAAAHJCBAAAA|4758|1|19|18|AM|third|night|| +4759|AAAAAAAAIJCBAAAA|4759|1|19|19|AM|third|night|| +4760|AAAAAAAAJJCBAAAA|4760|1|19|20|AM|third|night|| +4761|AAAAAAAAKJCBAAAA|4761|1|19|21|AM|third|night|| +4762|AAAAAAAALJCBAAAA|4762|1|19|22|AM|third|night|| +4763|AAAAAAAAMJCBAAAA|4763|1|19|23|AM|third|night|| +4764|AAAAAAAANJCBAAAA|4764|1|19|24|AM|third|night|| +4765|AAAAAAAAOJCBAAAA|4765|1|19|25|AM|third|night|| +4766|AAAAAAAAPJCBAAAA|4766|1|19|26|AM|third|night|| +4767|AAAAAAAAAKCBAAAA|4767|1|19|27|AM|third|night|| +4768|AAAAAAAABKCBAAAA|4768|1|19|28|AM|third|night|| +4769|AAAAAAAACKCBAAAA|4769|1|19|29|AM|third|night|| +4770|AAAAAAAADKCBAAAA|4770|1|19|30|AM|third|night|| +4771|AAAAAAAAEKCBAAAA|4771|1|19|31|AM|third|night|| +4772|AAAAAAAAFKCBAAAA|4772|1|19|32|AM|third|night|| +4773|AAAAAAAAGKCBAAAA|4773|1|19|33|AM|third|night|| +4774|AAAAAAAAHKCBAAAA|4774|1|19|34|AM|third|night|| +4775|AAAAAAAAIKCBAAAA|4775|1|19|35|AM|third|night|| +4776|AAAAAAAAJKCBAAAA|4776|1|19|36|AM|third|night|| +4777|AAAAAAAAKKCBAAAA|4777|1|19|37|AM|third|night|| +4778|AAAAAAAALKCBAAAA|4778|1|19|38|AM|third|night|| +4779|AAAAAAAAMKCBAAAA|4779|1|19|39|AM|third|night|| +4780|AAAAAAAANKCBAAAA|4780|1|19|40|AM|third|night|| +4781|AAAAAAAAOKCBAAAA|4781|1|19|41|AM|third|night|| +4782|AAAAAAAAPKCBAAAA|4782|1|19|42|AM|third|night|| +4783|AAAAAAAAALCBAAAA|4783|1|19|43|AM|third|night|| +4784|AAAAAAAABLCBAAAA|4784|1|19|44|AM|third|night|| +4785|AAAAAAAACLCBAAAA|4785|1|19|45|AM|third|night|| +4786|AAAAAAAADLCBAAAA|4786|1|19|46|AM|third|night|| +4787|AAAAAAAAELCBAAAA|4787|1|19|47|AM|third|night|| +4788|AAAAAAAAFLCBAAAA|4788|1|19|48|AM|third|night|| +4789|AAAAAAAAGLCBAAAA|4789|1|19|49|AM|third|night|| +4790|AAAAAAAAHLCBAAAA|4790|1|19|50|AM|third|night|| +4791|AAAAAAAAILCBAAAA|4791|1|19|51|AM|third|night|| +4792|AAAAAAAAJLCBAAAA|4792|1|19|52|AM|third|night|| +4793|AAAAAAAAKLCBAAAA|4793|1|19|53|AM|third|night|| +4794|AAAAAAAALLCBAAAA|4794|1|19|54|AM|third|night|| +4795|AAAAAAAAMLCBAAAA|4795|1|19|55|AM|third|night|| +4796|AAAAAAAANLCBAAAA|4796|1|19|56|AM|third|night|| +4797|AAAAAAAAOLCBAAAA|4797|1|19|57|AM|third|night|| +4798|AAAAAAAAPLCBAAAA|4798|1|19|58|AM|third|night|| +4799|AAAAAAAAAMCBAAAA|4799|1|19|59|AM|third|night|| +4800|AAAAAAAABMCBAAAA|4800|1|20|0|AM|third|night|| +4801|AAAAAAAACMCBAAAA|4801|1|20|1|AM|third|night|| +4802|AAAAAAAADMCBAAAA|4802|1|20|2|AM|third|night|| +4803|AAAAAAAAEMCBAAAA|4803|1|20|3|AM|third|night|| +4804|AAAAAAAAFMCBAAAA|4804|1|20|4|AM|third|night|| +4805|AAAAAAAAGMCBAAAA|4805|1|20|5|AM|third|night|| +4806|AAAAAAAAHMCBAAAA|4806|1|20|6|AM|third|night|| +4807|AAAAAAAAIMCBAAAA|4807|1|20|7|AM|third|night|| +4808|AAAAAAAAJMCBAAAA|4808|1|20|8|AM|third|night|| +4809|AAAAAAAAKMCBAAAA|4809|1|20|9|AM|third|night|| +4810|AAAAAAAALMCBAAAA|4810|1|20|10|AM|third|night|| +4811|AAAAAAAAMMCBAAAA|4811|1|20|11|AM|third|night|| +4812|AAAAAAAANMCBAAAA|4812|1|20|12|AM|third|night|| +4813|AAAAAAAAOMCBAAAA|4813|1|20|13|AM|third|night|| +4814|AAAAAAAAPMCBAAAA|4814|1|20|14|AM|third|night|| +4815|AAAAAAAAANCBAAAA|4815|1|20|15|AM|third|night|| +4816|AAAAAAAABNCBAAAA|4816|1|20|16|AM|third|night|| +4817|AAAAAAAACNCBAAAA|4817|1|20|17|AM|third|night|| +4818|AAAAAAAADNCBAAAA|4818|1|20|18|AM|third|night|| +4819|AAAAAAAAENCBAAAA|4819|1|20|19|AM|third|night|| +4820|AAAAAAAAFNCBAAAA|4820|1|20|20|AM|third|night|| +4821|AAAAAAAAGNCBAAAA|4821|1|20|21|AM|third|night|| +4822|AAAAAAAAHNCBAAAA|4822|1|20|22|AM|third|night|| +4823|AAAAAAAAINCBAAAA|4823|1|20|23|AM|third|night|| +4824|AAAAAAAAJNCBAAAA|4824|1|20|24|AM|third|night|| +4825|AAAAAAAAKNCBAAAA|4825|1|20|25|AM|third|night|| +4826|AAAAAAAALNCBAAAA|4826|1|20|26|AM|third|night|| +4827|AAAAAAAAMNCBAAAA|4827|1|20|27|AM|third|night|| +4828|AAAAAAAANNCBAAAA|4828|1|20|28|AM|third|night|| +4829|AAAAAAAAONCBAAAA|4829|1|20|29|AM|third|night|| +4830|AAAAAAAAPNCBAAAA|4830|1|20|30|AM|third|night|| +4831|AAAAAAAAAOCBAAAA|4831|1|20|31|AM|third|night|| +4832|AAAAAAAABOCBAAAA|4832|1|20|32|AM|third|night|| +4833|AAAAAAAACOCBAAAA|4833|1|20|33|AM|third|night|| +4834|AAAAAAAADOCBAAAA|4834|1|20|34|AM|third|night|| +4835|AAAAAAAAEOCBAAAA|4835|1|20|35|AM|third|night|| +4836|AAAAAAAAFOCBAAAA|4836|1|20|36|AM|third|night|| +4837|AAAAAAAAGOCBAAAA|4837|1|20|37|AM|third|night|| +4838|AAAAAAAAHOCBAAAA|4838|1|20|38|AM|third|night|| +4839|AAAAAAAAIOCBAAAA|4839|1|20|39|AM|third|night|| +4840|AAAAAAAAJOCBAAAA|4840|1|20|40|AM|third|night|| +4841|AAAAAAAAKOCBAAAA|4841|1|20|41|AM|third|night|| +4842|AAAAAAAALOCBAAAA|4842|1|20|42|AM|third|night|| +4843|AAAAAAAAMOCBAAAA|4843|1|20|43|AM|third|night|| +4844|AAAAAAAANOCBAAAA|4844|1|20|44|AM|third|night|| +4845|AAAAAAAAOOCBAAAA|4845|1|20|45|AM|third|night|| +4846|AAAAAAAAPOCBAAAA|4846|1|20|46|AM|third|night|| +4847|AAAAAAAAAPCBAAAA|4847|1|20|47|AM|third|night|| +4848|AAAAAAAABPCBAAAA|4848|1|20|48|AM|third|night|| +4849|AAAAAAAACPCBAAAA|4849|1|20|49|AM|third|night|| +4850|AAAAAAAADPCBAAAA|4850|1|20|50|AM|third|night|| +4851|AAAAAAAAEPCBAAAA|4851|1|20|51|AM|third|night|| +4852|AAAAAAAAFPCBAAAA|4852|1|20|52|AM|third|night|| +4853|AAAAAAAAGPCBAAAA|4853|1|20|53|AM|third|night|| +4854|AAAAAAAAHPCBAAAA|4854|1|20|54|AM|third|night|| +4855|AAAAAAAAIPCBAAAA|4855|1|20|55|AM|third|night|| +4856|AAAAAAAAJPCBAAAA|4856|1|20|56|AM|third|night|| +4857|AAAAAAAAKPCBAAAA|4857|1|20|57|AM|third|night|| +4858|AAAAAAAALPCBAAAA|4858|1|20|58|AM|third|night|| +4859|AAAAAAAAMPCBAAAA|4859|1|20|59|AM|third|night|| +4860|AAAAAAAANPCBAAAA|4860|1|21|0|AM|third|night|| +4861|AAAAAAAAOPCBAAAA|4861|1|21|1|AM|third|night|| +4862|AAAAAAAAPPCBAAAA|4862|1|21|2|AM|third|night|| +4863|AAAAAAAAAADBAAAA|4863|1|21|3|AM|third|night|| +4864|AAAAAAAABADBAAAA|4864|1|21|4|AM|third|night|| +4865|AAAAAAAACADBAAAA|4865|1|21|5|AM|third|night|| +4866|AAAAAAAADADBAAAA|4866|1|21|6|AM|third|night|| +4867|AAAAAAAAEADBAAAA|4867|1|21|7|AM|third|night|| +4868|AAAAAAAAFADBAAAA|4868|1|21|8|AM|third|night|| +4869|AAAAAAAAGADBAAAA|4869|1|21|9|AM|third|night|| +4870|AAAAAAAAHADBAAAA|4870|1|21|10|AM|third|night|| +4871|AAAAAAAAIADBAAAA|4871|1|21|11|AM|third|night|| +4872|AAAAAAAAJADBAAAA|4872|1|21|12|AM|third|night|| +4873|AAAAAAAAKADBAAAA|4873|1|21|13|AM|third|night|| +4874|AAAAAAAALADBAAAA|4874|1|21|14|AM|third|night|| +4875|AAAAAAAAMADBAAAA|4875|1|21|15|AM|third|night|| +4876|AAAAAAAANADBAAAA|4876|1|21|16|AM|third|night|| +4877|AAAAAAAAOADBAAAA|4877|1|21|17|AM|third|night|| +4878|AAAAAAAAPADBAAAA|4878|1|21|18|AM|third|night|| +4879|AAAAAAAAABDBAAAA|4879|1|21|19|AM|third|night|| +4880|AAAAAAAABBDBAAAA|4880|1|21|20|AM|third|night|| +4881|AAAAAAAACBDBAAAA|4881|1|21|21|AM|third|night|| +4882|AAAAAAAADBDBAAAA|4882|1|21|22|AM|third|night|| +4883|AAAAAAAAEBDBAAAA|4883|1|21|23|AM|third|night|| +4884|AAAAAAAAFBDBAAAA|4884|1|21|24|AM|third|night|| +4885|AAAAAAAAGBDBAAAA|4885|1|21|25|AM|third|night|| +4886|AAAAAAAAHBDBAAAA|4886|1|21|26|AM|third|night|| +4887|AAAAAAAAIBDBAAAA|4887|1|21|27|AM|third|night|| +4888|AAAAAAAAJBDBAAAA|4888|1|21|28|AM|third|night|| +4889|AAAAAAAAKBDBAAAA|4889|1|21|29|AM|third|night|| +4890|AAAAAAAALBDBAAAA|4890|1|21|30|AM|third|night|| +4891|AAAAAAAAMBDBAAAA|4891|1|21|31|AM|third|night|| +4892|AAAAAAAANBDBAAAA|4892|1|21|32|AM|third|night|| +4893|AAAAAAAAOBDBAAAA|4893|1|21|33|AM|third|night|| +4894|AAAAAAAAPBDBAAAA|4894|1|21|34|AM|third|night|| +4895|AAAAAAAAACDBAAAA|4895|1|21|35|AM|third|night|| +4896|AAAAAAAABCDBAAAA|4896|1|21|36|AM|third|night|| +4897|AAAAAAAACCDBAAAA|4897|1|21|37|AM|third|night|| +4898|AAAAAAAADCDBAAAA|4898|1|21|38|AM|third|night|| +4899|AAAAAAAAECDBAAAA|4899|1|21|39|AM|third|night|| +4900|AAAAAAAAFCDBAAAA|4900|1|21|40|AM|third|night|| +4901|AAAAAAAAGCDBAAAA|4901|1|21|41|AM|third|night|| +4902|AAAAAAAAHCDBAAAA|4902|1|21|42|AM|third|night|| +4903|AAAAAAAAICDBAAAA|4903|1|21|43|AM|third|night|| +4904|AAAAAAAAJCDBAAAA|4904|1|21|44|AM|third|night|| +4905|AAAAAAAAKCDBAAAA|4905|1|21|45|AM|third|night|| +4906|AAAAAAAALCDBAAAA|4906|1|21|46|AM|third|night|| +4907|AAAAAAAAMCDBAAAA|4907|1|21|47|AM|third|night|| +4908|AAAAAAAANCDBAAAA|4908|1|21|48|AM|third|night|| +4909|AAAAAAAAOCDBAAAA|4909|1|21|49|AM|third|night|| +4910|AAAAAAAAPCDBAAAA|4910|1|21|50|AM|third|night|| +4911|AAAAAAAAADDBAAAA|4911|1|21|51|AM|third|night|| +4912|AAAAAAAABDDBAAAA|4912|1|21|52|AM|third|night|| +4913|AAAAAAAACDDBAAAA|4913|1|21|53|AM|third|night|| +4914|AAAAAAAADDDBAAAA|4914|1|21|54|AM|third|night|| +4915|AAAAAAAAEDDBAAAA|4915|1|21|55|AM|third|night|| +4916|AAAAAAAAFDDBAAAA|4916|1|21|56|AM|third|night|| +4917|AAAAAAAAGDDBAAAA|4917|1|21|57|AM|third|night|| +4918|AAAAAAAAHDDBAAAA|4918|1|21|58|AM|third|night|| +4919|AAAAAAAAIDDBAAAA|4919|1|21|59|AM|third|night|| +4920|AAAAAAAAJDDBAAAA|4920|1|22|0|AM|third|night|| +4921|AAAAAAAAKDDBAAAA|4921|1|22|1|AM|third|night|| +4922|AAAAAAAALDDBAAAA|4922|1|22|2|AM|third|night|| +4923|AAAAAAAAMDDBAAAA|4923|1|22|3|AM|third|night|| +4924|AAAAAAAANDDBAAAA|4924|1|22|4|AM|third|night|| +4925|AAAAAAAAODDBAAAA|4925|1|22|5|AM|third|night|| +4926|AAAAAAAAPDDBAAAA|4926|1|22|6|AM|third|night|| +4927|AAAAAAAAAEDBAAAA|4927|1|22|7|AM|third|night|| +4928|AAAAAAAABEDBAAAA|4928|1|22|8|AM|third|night|| +4929|AAAAAAAACEDBAAAA|4929|1|22|9|AM|third|night|| +4930|AAAAAAAADEDBAAAA|4930|1|22|10|AM|third|night|| +4931|AAAAAAAAEEDBAAAA|4931|1|22|11|AM|third|night|| +4932|AAAAAAAAFEDBAAAA|4932|1|22|12|AM|third|night|| +4933|AAAAAAAAGEDBAAAA|4933|1|22|13|AM|third|night|| +4934|AAAAAAAAHEDBAAAA|4934|1|22|14|AM|third|night|| +4935|AAAAAAAAIEDBAAAA|4935|1|22|15|AM|third|night|| +4936|AAAAAAAAJEDBAAAA|4936|1|22|16|AM|third|night|| +4937|AAAAAAAAKEDBAAAA|4937|1|22|17|AM|third|night|| +4938|AAAAAAAALEDBAAAA|4938|1|22|18|AM|third|night|| +4939|AAAAAAAAMEDBAAAA|4939|1|22|19|AM|third|night|| +4940|AAAAAAAANEDBAAAA|4940|1|22|20|AM|third|night|| +4941|AAAAAAAAOEDBAAAA|4941|1|22|21|AM|third|night|| +4942|AAAAAAAAPEDBAAAA|4942|1|22|22|AM|third|night|| +4943|AAAAAAAAAFDBAAAA|4943|1|22|23|AM|third|night|| +4944|AAAAAAAABFDBAAAA|4944|1|22|24|AM|third|night|| +4945|AAAAAAAACFDBAAAA|4945|1|22|25|AM|third|night|| +4946|AAAAAAAADFDBAAAA|4946|1|22|26|AM|third|night|| +4947|AAAAAAAAEFDBAAAA|4947|1|22|27|AM|third|night|| +4948|AAAAAAAAFFDBAAAA|4948|1|22|28|AM|third|night|| +4949|AAAAAAAAGFDBAAAA|4949|1|22|29|AM|third|night|| +4950|AAAAAAAAHFDBAAAA|4950|1|22|30|AM|third|night|| +4951|AAAAAAAAIFDBAAAA|4951|1|22|31|AM|third|night|| +4952|AAAAAAAAJFDBAAAA|4952|1|22|32|AM|third|night|| +4953|AAAAAAAAKFDBAAAA|4953|1|22|33|AM|third|night|| +4954|AAAAAAAALFDBAAAA|4954|1|22|34|AM|third|night|| +4955|AAAAAAAAMFDBAAAA|4955|1|22|35|AM|third|night|| +4956|AAAAAAAANFDBAAAA|4956|1|22|36|AM|third|night|| +4957|AAAAAAAAOFDBAAAA|4957|1|22|37|AM|third|night|| +4958|AAAAAAAAPFDBAAAA|4958|1|22|38|AM|third|night|| +4959|AAAAAAAAAGDBAAAA|4959|1|22|39|AM|third|night|| +4960|AAAAAAAABGDBAAAA|4960|1|22|40|AM|third|night|| +4961|AAAAAAAACGDBAAAA|4961|1|22|41|AM|third|night|| +4962|AAAAAAAADGDBAAAA|4962|1|22|42|AM|third|night|| +4963|AAAAAAAAEGDBAAAA|4963|1|22|43|AM|third|night|| +4964|AAAAAAAAFGDBAAAA|4964|1|22|44|AM|third|night|| +4965|AAAAAAAAGGDBAAAA|4965|1|22|45|AM|third|night|| +4966|AAAAAAAAHGDBAAAA|4966|1|22|46|AM|third|night|| +4967|AAAAAAAAIGDBAAAA|4967|1|22|47|AM|third|night|| +4968|AAAAAAAAJGDBAAAA|4968|1|22|48|AM|third|night|| +4969|AAAAAAAAKGDBAAAA|4969|1|22|49|AM|third|night|| +4970|AAAAAAAALGDBAAAA|4970|1|22|50|AM|third|night|| +4971|AAAAAAAAMGDBAAAA|4971|1|22|51|AM|third|night|| +4972|AAAAAAAANGDBAAAA|4972|1|22|52|AM|third|night|| +4973|AAAAAAAAOGDBAAAA|4973|1|22|53|AM|third|night|| +4974|AAAAAAAAPGDBAAAA|4974|1|22|54|AM|third|night|| +4975|AAAAAAAAAHDBAAAA|4975|1|22|55|AM|third|night|| +4976|AAAAAAAABHDBAAAA|4976|1|22|56|AM|third|night|| +4977|AAAAAAAACHDBAAAA|4977|1|22|57|AM|third|night|| +4978|AAAAAAAADHDBAAAA|4978|1|22|58|AM|third|night|| +4979|AAAAAAAAEHDBAAAA|4979|1|22|59|AM|third|night|| +4980|AAAAAAAAFHDBAAAA|4980|1|23|0|AM|third|night|| +4981|AAAAAAAAGHDBAAAA|4981|1|23|1|AM|third|night|| +4982|AAAAAAAAHHDBAAAA|4982|1|23|2|AM|third|night|| +4983|AAAAAAAAIHDBAAAA|4983|1|23|3|AM|third|night|| +4984|AAAAAAAAJHDBAAAA|4984|1|23|4|AM|third|night|| +4985|AAAAAAAAKHDBAAAA|4985|1|23|5|AM|third|night|| +4986|AAAAAAAALHDBAAAA|4986|1|23|6|AM|third|night|| +4987|AAAAAAAAMHDBAAAA|4987|1|23|7|AM|third|night|| +4988|AAAAAAAANHDBAAAA|4988|1|23|8|AM|third|night|| +4989|AAAAAAAAOHDBAAAA|4989|1|23|9|AM|third|night|| +4990|AAAAAAAAPHDBAAAA|4990|1|23|10|AM|third|night|| +4991|AAAAAAAAAIDBAAAA|4991|1|23|11|AM|third|night|| +4992|AAAAAAAABIDBAAAA|4992|1|23|12|AM|third|night|| +4993|AAAAAAAACIDBAAAA|4993|1|23|13|AM|third|night|| +4994|AAAAAAAADIDBAAAA|4994|1|23|14|AM|third|night|| +4995|AAAAAAAAEIDBAAAA|4995|1|23|15|AM|third|night|| +4996|AAAAAAAAFIDBAAAA|4996|1|23|16|AM|third|night|| +4997|AAAAAAAAGIDBAAAA|4997|1|23|17|AM|third|night|| +4998|AAAAAAAAHIDBAAAA|4998|1|23|18|AM|third|night|| +4999|AAAAAAAAIIDBAAAA|4999|1|23|19|AM|third|night|| +5000|AAAAAAAAJIDBAAAA|5000|1|23|20|AM|third|night|| +5001|AAAAAAAAKIDBAAAA|5001|1|23|21|AM|third|night|| +5002|AAAAAAAALIDBAAAA|5002|1|23|22|AM|third|night|| +5003|AAAAAAAAMIDBAAAA|5003|1|23|23|AM|third|night|| +5004|AAAAAAAANIDBAAAA|5004|1|23|24|AM|third|night|| +5005|AAAAAAAAOIDBAAAA|5005|1|23|25|AM|third|night|| +5006|AAAAAAAAPIDBAAAA|5006|1|23|26|AM|third|night|| +5007|AAAAAAAAAJDBAAAA|5007|1|23|27|AM|third|night|| +5008|AAAAAAAABJDBAAAA|5008|1|23|28|AM|third|night|| +5009|AAAAAAAACJDBAAAA|5009|1|23|29|AM|third|night|| +5010|AAAAAAAADJDBAAAA|5010|1|23|30|AM|third|night|| +5011|AAAAAAAAEJDBAAAA|5011|1|23|31|AM|third|night|| +5012|AAAAAAAAFJDBAAAA|5012|1|23|32|AM|third|night|| +5013|AAAAAAAAGJDBAAAA|5013|1|23|33|AM|third|night|| +5014|AAAAAAAAHJDBAAAA|5014|1|23|34|AM|third|night|| +5015|AAAAAAAAIJDBAAAA|5015|1|23|35|AM|third|night|| +5016|AAAAAAAAJJDBAAAA|5016|1|23|36|AM|third|night|| +5017|AAAAAAAAKJDBAAAA|5017|1|23|37|AM|third|night|| +5018|AAAAAAAALJDBAAAA|5018|1|23|38|AM|third|night|| +5019|AAAAAAAAMJDBAAAA|5019|1|23|39|AM|third|night|| +5020|AAAAAAAANJDBAAAA|5020|1|23|40|AM|third|night|| +5021|AAAAAAAAOJDBAAAA|5021|1|23|41|AM|third|night|| +5022|AAAAAAAAPJDBAAAA|5022|1|23|42|AM|third|night|| +5023|AAAAAAAAAKDBAAAA|5023|1|23|43|AM|third|night|| +5024|AAAAAAAABKDBAAAA|5024|1|23|44|AM|third|night|| +5025|AAAAAAAACKDBAAAA|5025|1|23|45|AM|third|night|| +5026|AAAAAAAADKDBAAAA|5026|1|23|46|AM|third|night|| +5027|AAAAAAAAEKDBAAAA|5027|1|23|47|AM|third|night|| +5028|AAAAAAAAFKDBAAAA|5028|1|23|48|AM|third|night|| +5029|AAAAAAAAGKDBAAAA|5029|1|23|49|AM|third|night|| +5030|AAAAAAAAHKDBAAAA|5030|1|23|50|AM|third|night|| +5031|AAAAAAAAIKDBAAAA|5031|1|23|51|AM|third|night|| +5032|AAAAAAAAJKDBAAAA|5032|1|23|52|AM|third|night|| +5033|AAAAAAAAKKDBAAAA|5033|1|23|53|AM|third|night|| +5034|AAAAAAAALKDBAAAA|5034|1|23|54|AM|third|night|| +5035|AAAAAAAAMKDBAAAA|5035|1|23|55|AM|third|night|| +5036|AAAAAAAANKDBAAAA|5036|1|23|56|AM|third|night|| +5037|AAAAAAAAOKDBAAAA|5037|1|23|57|AM|third|night|| +5038|AAAAAAAAPKDBAAAA|5038|1|23|58|AM|third|night|| +5039|AAAAAAAAALDBAAAA|5039|1|23|59|AM|third|night|| +5040|AAAAAAAABLDBAAAA|5040|1|24|0|AM|third|night|| +5041|AAAAAAAACLDBAAAA|5041|1|24|1|AM|third|night|| +5042|AAAAAAAADLDBAAAA|5042|1|24|2|AM|third|night|| +5043|AAAAAAAAELDBAAAA|5043|1|24|3|AM|third|night|| +5044|AAAAAAAAFLDBAAAA|5044|1|24|4|AM|third|night|| +5045|AAAAAAAAGLDBAAAA|5045|1|24|5|AM|third|night|| +5046|AAAAAAAAHLDBAAAA|5046|1|24|6|AM|third|night|| +5047|AAAAAAAAILDBAAAA|5047|1|24|7|AM|third|night|| +5048|AAAAAAAAJLDBAAAA|5048|1|24|8|AM|third|night|| +5049|AAAAAAAAKLDBAAAA|5049|1|24|9|AM|third|night|| +5050|AAAAAAAALLDBAAAA|5050|1|24|10|AM|third|night|| +5051|AAAAAAAAMLDBAAAA|5051|1|24|11|AM|third|night|| +5052|AAAAAAAANLDBAAAA|5052|1|24|12|AM|third|night|| +5053|AAAAAAAAOLDBAAAA|5053|1|24|13|AM|third|night|| +5054|AAAAAAAAPLDBAAAA|5054|1|24|14|AM|third|night|| +5055|AAAAAAAAAMDBAAAA|5055|1|24|15|AM|third|night|| +5056|AAAAAAAABMDBAAAA|5056|1|24|16|AM|third|night|| +5057|AAAAAAAACMDBAAAA|5057|1|24|17|AM|third|night|| +5058|AAAAAAAADMDBAAAA|5058|1|24|18|AM|third|night|| +5059|AAAAAAAAEMDBAAAA|5059|1|24|19|AM|third|night|| +5060|AAAAAAAAFMDBAAAA|5060|1|24|20|AM|third|night|| +5061|AAAAAAAAGMDBAAAA|5061|1|24|21|AM|third|night|| +5062|AAAAAAAAHMDBAAAA|5062|1|24|22|AM|third|night|| +5063|AAAAAAAAIMDBAAAA|5063|1|24|23|AM|third|night|| +5064|AAAAAAAAJMDBAAAA|5064|1|24|24|AM|third|night|| +5065|AAAAAAAAKMDBAAAA|5065|1|24|25|AM|third|night|| +5066|AAAAAAAALMDBAAAA|5066|1|24|26|AM|third|night|| +5067|AAAAAAAAMMDBAAAA|5067|1|24|27|AM|third|night|| +5068|AAAAAAAANMDBAAAA|5068|1|24|28|AM|third|night|| +5069|AAAAAAAAOMDBAAAA|5069|1|24|29|AM|third|night|| +5070|AAAAAAAAPMDBAAAA|5070|1|24|30|AM|third|night|| +5071|AAAAAAAAANDBAAAA|5071|1|24|31|AM|third|night|| +5072|AAAAAAAABNDBAAAA|5072|1|24|32|AM|third|night|| +5073|AAAAAAAACNDBAAAA|5073|1|24|33|AM|third|night|| +5074|AAAAAAAADNDBAAAA|5074|1|24|34|AM|third|night|| +5075|AAAAAAAAENDBAAAA|5075|1|24|35|AM|third|night|| +5076|AAAAAAAAFNDBAAAA|5076|1|24|36|AM|third|night|| +5077|AAAAAAAAGNDBAAAA|5077|1|24|37|AM|third|night|| +5078|AAAAAAAAHNDBAAAA|5078|1|24|38|AM|third|night|| +5079|AAAAAAAAINDBAAAA|5079|1|24|39|AM|third|night|| +5080|AAAAAAAAJNDBAAAA|5080|1|24|40|AM|third|night|| +5081|AAAAAAAAKNDBAAAA|5081|1|24|41|AM|third|night|| +5082|AAAAAAAALNDBAAAA|5082|1|24|42|AM|third|night|| +5083|AAAAAAAAMNDBAAAA|5083|1|24|43|AM|third|night|| +5084|AAAAAAAANNDBAAAA|5084|1|24|44|AM|third|night|| +5085|AAAAAAAAONDBAAAA|5085|1|24|45|AM|third|night|| +5086|AAAAAAAAPNDBAAAA|5086|1|24|46|AM|third|night|| +5087|AAAAAAAAAODBAAAA|5087|1|24|47|AM|third|night|| +5088|AAAAAAAABODBAAAA|5088|1|24|48|AM|third|night|| +5089|AAAAAAAACODBAAAA|5089|1|24|49|AM|third|night|| +5090|AAAAAAAADODBAAAA|5090|1|24|50|AM|third|night|| +5091|AAAAAAAAEODBAAAA|5091|1|24|51|AM|third|night|| +5092|AAAAAAAAFODBAAAA|5092|1|24|52|AM|third|night|| +5093|AAAAAAAAGODBAAAA|5093|1|24|53|AM|third|night|| +5094|AAAAAAAAHODBAAAA|5094|1|24|54|AM|third|night|| +5095|AAAAAAAAIODBAAAA|5095|1|24|55|AM|third|night|| +5096|AAAAAAAAJODBAAAA|5096|1|24|56|AM|third|night|| +5097|AAAAAAAAKODBAAAA|5097|1|24|57|AM|third|night|| +5098|AAAAAAAALODBAAAA|5098|1|24|58|AM|third|night|| +5099|AAAAAAAAMODBAAAA|5099|1|24|59|AM|third|night|| +5100|AAAAAAAANODBAAAA|5100|1|25|0|AM|third|night|| +5101|AAAAAAAAOODBAAAA|5101|1|25|1|AM|third|night|| +5102|AAAAAAAAPODBAAAA|5102|1|25|2|AM|third|night|| +5103|AAAAAAAAAPDBAAAA|5103|1|25|3|AM|third|night|| +5104|AAAAAAAABPDBAAAA|5104|1|25|4|AM|third|night|| +5105|AAAAAAAACPDBAAAA|5105|1|25|5|AM|third|night|| +5106|AAAAAAAADPDBAAAA|5106|1|25|6|AM|third|night|| +5107|AAAAAAAAEPDBAAAA|5107|1|25|7|AM|third|night|| +5108|AAAAAAAAFPDBAAAA|5108|1|25|8|AM|third|night|| +5109|AAAAAAAAGPDBAAAA|5109|1|25|9|AM|third|night|| +5110|AAAAAAAAHPDBAAAA|5110|1|25|10|AM|third|night|| +5111|AAAAAAAAIPDBAAAA|5111|1|25|11|AM|third|night|| +5112|AAAAAAAAJPDBAAAA|5112|1|25|12|AM|third|night|| +5113|AAAAAAAAKPDBAAAA|5113|1|25|13|AM|third|night|| +5114|AAAAAAAALPDBAAAA|5114|1|25|14|AM|third|night|| +5115|AAAAAAAAMPDBAAAA|5115|1|25|15|AM|third|night|| +5116|AAAAAAAANPDBAAAA|5116|1|25|16|AM|third|night|| +5117|AAAAAAAAOPDBAAAA|5117|1|25|17|AM|third|night|| +5118|AAAAAAAAPPDBAAAA|5118|1|25|18|AM|third|night|| +5119|AAAAAAAAAAEBAAAA|5119|1|25|19|AM|third|night|| +5120|AAAAAAAABAEBAAAA|5120|1|25|20|AM|third|night|| +5121|AAAAAAAACAEBAAAA|5121|1|25|21|AM|third|night|| +5122|AAAAAAAADAEBAAAA|5122|1|25|22|AM|third|night|| +5123|AAAAAAAAEAEBAAAA|5123|1|25|23|AM|third|night|| +5124|AAAAAAAAFAEBAAAA|5124|1|25|24|AM|third|night|| +5125|AAAAAAAAGAEBAAAA|5125|1|25|25|AM|third|night|| +5126|AAAAAAAAHAEBAAAA|5126|1|25|26|AM|third|night|| +5127|AAAAAAAAIAEBAAAA|5127|1|25|27|AM|third|night|| +5128|AAAAAAAAJAEBAAAA|5128|1|25|28|AM|third|night|| +5129|AAAAAAAAKAEBAAAA|5129|1|25|29|AM|third|night|| +5130|AAAAAAAALAEBAAAA|5130|1|25|30|AM|third|night|| +5131|AAAAAAAAMAEBAAAA|5131|1|25|31|AM|third|night|| +5132|AAAAAAAANAEBAAAA|5132|1|25|32|AM|third|night|| +5133|AAAAAAAAOAEBAAAA|5133|1|25|33|AM|third|night|| +5134|AAAAAAAAPAEBAAAA|5134|1|25|34|AM|third|night|| +5135|AAAAAAAAABEBAAAA|5135|1|25|35|AM|third|night|| +5136|AAAAAAAABBEBAAAA|5136|1|25|36|AM|third|night|| +5137|AAAAAAAACBEBAAAA|5137|1|25|37|AM|third|night|| +5138|AAAAAAAADBEBAAAA|5138|1|25|38|AM|third|night|| +5139|AAAAAAAAEBEBAAAA|5139|1|25|39|AM|third|night|| +5140|AAAAAAAAFBEBAAAA|5140|1|25|40|AM|third|night|| +5141|AAAAAAAAGBEBAAAA|5141|1|25|41|AM|third|night|| +5142|AAAAAAAAHBEBAAAA|5142|1|25|42|AM|third|night|| +5143|AAAAAAAAIBEBAAAA|5143|1|25|43|AM|third|night|| +5144|AAAAAAAAJBEBAAAA|5144|1|25|44|AM|third|night|| +5145|AAAAAAAAKBEBAAAA|5145|1|25|45|AM|third|night|| +5146|AAAAAAAALBEBAAAA|5146|1|25|46|AM|third|night|| +5147|AAAAAAAAMBEBAAAA|5147|1|25|47|AM|third|night|| +5148|AAAAAAAANBEBAAAA|5148|1|25|48|AM|third|night|| +5149|AAAAAAAAOBEBAAAA|5149|1|25|49|AM|third|night|| +5150|AAAAAAAAPBEBAAAA|5150|1|25|50|AM|third|night|| +5151|AAAAAAAAACEBAAAA|5151|1|25|51|AM|third|night|| +5152|AAAAAAAABCEBAAAA|5152|1|25|52|AM|third|night|| +5153|AAAAAAAACCEBAAAA|5153|1|25|53|AM|third|night|| +5154|AAAAAAAADCEBAAAA|5154|1|25|54|AM|third|night|| +5155|AAAAAAAAECEBAAAA|5155|1|25|55|AM|third|night|| +5156|AAAAAAAAFCEBAAAA|5156|1|25|56|AM|third|night|| +5157|AAAAAAAAGCEBAAAA|5157|1|25|57|AM|third|night|| +5158|AAAAAAAAHCEBAAAA|5158|1|25|58|AM|third|night|| +5159|AAAAAAAAICEBAAAA|5159|1|25|59|AM|third|night|| +5160|AAAAAAAAJCEBAAAA|5160|1|26|0|AM|third|night|| +5161|AAAAAAAAKCEBAAAA|5161|1|26|1|AM|third|night|| +5162|AAAAAAAALCEBAAAA|5162|1|26|2|AM|third|night|| +5163|AAAAAAAAMCEBAAAA|5163|1|26|3|AM|third|night|| +5164|AAAAAAAANCEBAAAA|5164|1|26|4|AM|third|night|| +5165|AAAAAAAAOCEBAAAA|5165|1|26|5|AM|third|night|| +5166|AAAAAAAAPCEBAAAA|5166|1|26|6|AM|third|night|| +5167|AAAAAAAAADEBAAAA|5167|1|26|7|AM|third|night|| +5168|AAAAAAAABDEBAAAA|5168|1|26|8|AM|third|night|| +5169|AAAAAAAACDEBAAAA|5169|1|26|9|AM|third|night|| +5170|AAAAAAAADDEBAAAA|5170|1|26|10|AM|third|night|| +5171|AAAAAAAAEDEBAAAA|5171|1|26|11|AM|third|night|| +5172|AAAAAAAAFDEBAAAA|5172|1|26|12|AM|third|night|| +5173|AAAAAAAAGDEBAAAA|5173|1|26|13|AM|third|night|| +5174|AAAAAAAAHDEBAAAA|5174|1|26|14|AM|third|night|| +5175|AAAAAAAAIDEBAAAA|5175|1|26|15|AM|third|night|| +5176|AAAAAAAAJDEBAAAA|5176|1|26|16|AM|third|night|| +5177|AAAAAAAAKDEBAAAA|5177|1|26|17|AM|third|night|| +5178|AAAAAAAALDEBAAAA|5178|1|26|18|AM|third|night|| +5179|AAAAAAAAMDEBAAAA|5179|1|26|19|AM|third|night|| +5180|AAAAAAAANDEBAAAA|5180|1|26|20|AM|third|night|| +5181|AAAAAAAAODEBAAAA|5181|1|26|21|AM|third|night|| +5182|AAAAAAAAPDEBAAAA|5182|1|26|22|AM|third|night|| +5183|AAAAAAAAAEEBAAAA|5183|1|26|23|AM|third|night|| +5184|AAAAAAAABEEBAAAA|5184|1|26|24|AM|third|night|| +5185|AAAAAAAACEEBAAAA|5185|1|26|25|AM|third|night|| +5186|AAAAAAAADEEBAAAA|5186|1|26|26|AM|third|night|| +5187|AAAAAAAAEEEBAAAA|5187|1|26|27|AM|third|night|| +5188|AAAAAAAAFEEBAAAA|5188|1|26|28|AM|third|night|| +5189|AAAAAAAAGEEBAAAA|5189|1|26|29|AM|third|night|| +5190|AAAAAAAAHEEBAAAA|5190|1|26|30|AM|third|night|| +5191|AAAAAAAAIEEBAAAA|5191|1|26|31|AM|third|night|| +5192|AAAAAAAAJEEBAAAA|5192|1|26|32|AM|third|night|| +5193|AAAAAAAAKEEBAAAA|5193|1|26|33|AM|third|night|| +5194|AAAAAAAALEEBAAAA|5194|1|26|34|AM|third|night|| +5195|AAAAAAAAMEEBAAAA|5195|1|26|35|AM|third|night|| +5196|AAAAAAAANEEBAAAA|5196|1|26|36|AM|third|night|| +5197|AAAAAAAAOEEBAAAA|5197|1|26|37|AM|third|night|| +5198|AAAAAAAAPEEBAAAA|5198|1|26|38|AM|third|night|| +5199|AAAAAAAAAFEBAAAA|5199|1|26|39|AM|third|night|| +5200|AAAAAAAABFEBAAAA|5200|1|26|40|AM|third|night|| +5201|AAAAAAAACFEBAAAA|5201|1|26|41|AM|third|night|| +5202|AAAAAAAADFEBAAAA|5202|1|26|42|AM|third|night|| +5203|AAAAAAAAEFEBAAAA|5203|1|26|43|AM|third|night|| +5204|AAAAAAAAFFEBAAAA|5204|1|26|44|AM|third|night|| +5205|AAAAAAAAGFEBAAAA|5205|1|26|45|AM|third|night|| +5206|AAAAAAAAHFEBAAAA|5206|1|26|46|AM|third|night|| +5207|AAAAAAAAIFEBAAAA|5207|1|26|47|AM|third|night|| +5208|AAAAAAAAJFEBAAAA|5208|1|26|48|AM|third|night|| +5209|AAAAAAAAKFEBAAAA|5209|1|26|49|AM|third|night|| +5210|AAAAAAAALFEBAAAA|5210|1|26|50|AM|third|night|| +5211|AAAAAAAAMFEBAAAA|5211|1|26|51|AM|third|night|| +5212|AAAAAAAANFEBAAAA|5212|1|26|52|AM|third|night|| +5213|AAAAAAAAOFEBAAAA|5213|1|26|53|AM|third|night|| +5214|AAAAAAAAPFEBAAAA|5214|1|26|54|AM|third|night|| +5215|AAAAAAAAAGEBAAAA|5215|1|26|55|AM|third|night|| +5216|AAAAAAAABGEBAAAA|5216|1|26|56|AM|third|night|| +5217|AAAAAAAACGEBAAAA|5217|1|26|57|AM|third|night|| +5218|AAAAAAAADGEBAAAA|5218|1|26|58|AM|third|night|| +5219|AAAAAAAAEGEBAAAA|5219|1|26|59|AM|third|night|| +5220|AAAAAAAAFGEBAAAA|5220|1|27|0|AM|third|night|| +5221|AAAAAAAAGGEBAAAA|5221|1|27|1|AM|third|night|| +5222|AAAAAAAAHGEBAAAA|5222|1|27|2|AM|third|night|| +5223|AAAAAAAAIGEBAAAA|5223|1|27|3|AM|third|night|| +5224|AAAAAAAAJGEBAAAA|5224|1|27|4|AM|third|night|| +5225|AAAAAAAAKGEBAAAA|5225|1|27|5|AM|third|night|| +5226|AAAAAAAALGEBAAAA|5226|1|27|6|AM|third|night|| +5227|AAAAAAAAMGEBAAAA|5227|1|27|7|AM|third|night|| +5228|AAAAAAAANGEBAAAA|5228|1|27|8|AM|third|night|| +5229|AAAAAAAAOGEBAAAA|5229|1|27|9|AM|third|night|| +5230|AAAAAAAAPGEBAAAA|5230|1|27|10|AM|third|night|| +5231|AAAAAAAAAHEBAAAA|5231|1|27|11|AM|third|night|| +5232|AAAAAAAABHEBAAAA|5232|1|27|12|AM|third|night|| +5233|AAAAAAAACHEBAAAA|5233|1|27|13|AM|third|night|| +5234|AAAAAAAADHEBAAAA|5234|1|27|14|AM|third|night|| +5235|AAAAAAAAEHEBAAAA|5235|1|27|15|AM|third|night|| +5236|AAAAAAAAFHEBAAAA|5236|1|27|16|AM|third|night|| +5237|AAAAAAAAGHEBAAAA|5237|1|27|17|AM|third|night|| +5238|AAAAAAAAHHEBAAAA|5238|1|27|18|AM|third|night|| +5239|AAAAAAAAIHEBAAAA|5239|1|27|19|AM|third|night|| +5240|AAAAAAAAJHEBAAAA|5240|1|27|20|AM|third|night|| +5241|AAAAAAAAKHEBAAAA|5241|1|27|21|AM|third|night|| +5242|AAAAAAAALHEBAAAA|5242|1|27|22|AM|third|night|| +5243|AAAAAAAAMHEBAAAA|5243|1|27|23|AM|third|night|| +5244|AAAAAAAANHEBAAAA|5244|1|27|24|AM|third|night|| +5245|AAAAAAAAOHEBAAAA|5245|1|27|25|AM|third|night|| +5246|AAAAAAAAPHEBAAAA|5246|1|27|26|AM|third|night|| +5247|AAAAAAAAAIEBAAAA|5247|1|27|27|AM|third|night|| +5248|AAAAAAAABIEBAAAA|5248|1|27|28|AM|third|night|| +5249|AAAAAAAACIEBAAAA|5249|1|27|29|AM|third|night|| +5250|AAAAAAAADIEBAAAA|5250|1|27|30|AM|third|night|| +5251|AAAAAAAAEIEBAAAA|5251|1|27|31|AM|third|night|| +5252|AAAAAAAAFIEBAAAA|5252|1|27|32|AM|third|night|| +5253|AAAAAAAAGIEBAAAA|5253|1|27|33|AM|third|night|| +5254|AAAAAAAAHIEBAAAA|5254|1|27|34|AM|third|night|| +5255|AAAAAAAAIIEBAAAA|5255|1|27|35|AM|third|night|| +5256|AAAAAAAAJIEBAAAA|5256|1|27|36|AM|third|night|| +5257|AAAAAAAAKIEBAAAA|5257|1|27|37|AM|third|night|| +5258|AAAAAAAALIEBAAAA|5258|1|27|38|AM|third|night|| +5259|AAAAAAAAMIEBAAAA|5259|1|27|39|AM|third|night|| +5260|AAAAAAAANIEBAAAA|5260|1|27|40|AM|third|night|| +5261|AAAAAAAAOIEBAAAA|5261|1|27|41|AM|third|night|| +5262|AAAAAAAAPIEBAAAA|5262|1|27|42|AM|third|night|| +5263|AAAAAAAAAJEBAAAA|5263|1|27|43|AM|third|night|| +5264|AAAAAAAABJEBAAAA|5264|1|27|44|AM|third|night|| +5265|AAAAAAAACJEBAAAA|5265|1|27|45|AM|third|night|| +5266|AAAAAAAADJEBAAAA|5266|1|27|46|AM|third|night|| +5267|AAAAAAAAEJEBAAAA|5267|1|27|47|AM|third|night|| +5268|AAAAAAAAFJEBAAAA|5268|1|27|48|AM|third|night|| +5269|AAAAAAAAGJEBAAAA|5269|1|27|49|AM|third|night|| +5270|AAAAAAAAHJEBAAAA|5270|1|27|50|AM|third|night|| +5271|AAAAAAAAIJEBAAAA|5271|1|27|51|AM|third|night|| +5272|AAAAAAAAJJEBAAAA|5272|1|27|52|AM|third|night|| +5273|AAAAAAAAKJEBAAAA|5273|1|27|53|AM|third|night|| +5274|AAAAAAAALJEBAAAA|5274|1|27|54|AM|third|night|| +5275|AAAAAAAAMJEBAAAA|5275|1|27|55|AM|third|night|| +5276|AAAAAAAANJEBAAAA|5276|1|27|56|AM|third|night|| +5277|AAAAAAAAOJEBAAAA|5277|1|27|57|AM|third|night|| +5278|AAAAAAAAPJEBAAAA|5278|1|27|58|AM|third|night|| +5279|AAAAAAAAAKEBAAAA|5279|1|27|59|AM|third|night|| +5280|AAAAAAAABKEBAAAA|5280|1|28|0|AM|third|night|| +5281|AAAAAAAACKEBAAAA|5281|1|28|1|AM|third|night|| +5282|AAAAAAAADKEBAAAA|5282|1|28|2|AM|third|night|| +5283|AAAAAAAAEKEBAAAA|5283|1|28|3|AM|third|night|| +5284|AAAAAAAAFKEBAAAA|5284|1|28|4|AM|third|night|| +5285|AAAAAAAAGKEBAAAA|5285|1|28|5|AM|third|night|| +5286|AAAAAAAAHKEBAAAA|5286|1|28|6|AM|third|night|| +5287|AAAAAAAAIKEBAAAA|5287|1|28|7|AM|third|night|| +5288|AAAAAAAAJKEBAAAA|5288|1|28|8|AM|third|night|| +5289|AAAAAAAAKKEBAAAA|5289|1|28|9|AM|third|night|| +5290|AAAAAAAALKEBAAAA|5290|1|28|10|AM|third|night|| +5291|AAAAAAAAMKEBAAAA|5291|1|28|11|AM|third|night|| +5292|AAAAAAAANKEBAAAA|5292|1|28|12|AM|third|night|| +5293|AAAAAAAAOKEBAAAA|5293|1|28|13|AM|third|night|| +5294|AAAAAAAAPKEBAAAA|5294|1|28|14|AM|third|night|| +5295|AAAAAAAAALEBAAAA|5295|1|28|15|AM|third|night|| +5296|AAAAAAAABLEBAAAA|5296|1|28|16|AM|third|night|| +5297|AAAAAAAACLEBAAAA|5297|1|28|17|AM|third|night|| +5298|AAAAAAAADLEBAAAA|5298|1|28|18|AM|third|night|| +5299|AAAAAAAAELEBAAAA|5299|1|28|19|AM|third|night|| +5300|AAAAAAAAFLEBAAAA|5300|1|28|20|AM|third|night|| +5301|AAAAAAAAGLEBAAAA|5301|1|28|21|AM|third|night|| +5302|AAAAAAAAHLEBAAAA|5302|1|28|22|AM|third|night|| +5303|AAAAAAAAILEBAAAA|5303|1|28|23|AM|third|night|| +5304|AAAAAAAAJLEBAAAA|5304|1|28|24|AM|third|night|| +5305|AAAAAAAAKLEBAAAA|5305|1|28|25|AM|third|night|| +5306|AAAAAAAALLEBAAAA|5306|1|28|26|AM|third|night|| +5307|AAAAAAAAMLEBAAAA|5307|1|28|27|AM|third|night|| +5308|AAAAAAAANLEBAAAA|5308|1|28|28|AM|third|night|| +5309|AAAAAAAAOLEBAAAA|5309|1|28|29|AM|third|night|| +5310|AAAAAAAAPLEBAAAA|5310|1|28|30|AM|third|night|| +5311|AAAAAAAAAMEBAAAA|5311|1|28|31|AM|third|night|| +5312|AAAAAAAABMEBAAAA|5312|1|28|32|AM|third|night|| +5313|AAAAAAAACMEBAAAA|5313|1|28|33|AM|third|night|| +5314|AAAAAAAADMEBAAAA|5314|1|28|34|AM|third|night|| +5315|AAAAAAAAEMEBAAAA|5315|1|28|35|AM|third|night|| +5316|AAAAAAAAFMEBAAAA|5316|1|28|36|AM|third|night|| +5317|AAAAAAAAGMEBAAAA|5317|1|28|37|AM|third|night|| +5318|AAAAAAAAHMEBAAAA|5318|1|28|38|AM|third|night|| +5319|AAAAAAAAIMEBAAAA|5319|1|28|39|AM|third|night|| +5320|AAAAAAAAJMEBAAAA|5320|1|28|40|AM|third|night|| +5321|AAAAAAAAKMEBAAAA|5321|1|28|41|AM|third|night|| +5322|AAAAAAAALMEBAAAA|5322|1|28|42|AM|third|night|| +5323|AAAAAAAAMMEBAAAA|5323|1|28|43|AM|third|night|| +5324|AAAAAAAANMEBAAAA|5324|1|28|44|AM|third|night|| +5325|AAAAAAAAOMEBAAAA|5325|1|28|45|AM|third|night|| +5326|AAAAAAAAPMEBAAAA|5326|1|28|46|AM|third|night|| +5327|AAAAAAAAANEBAAAA|5327|1|28|47|AM|third|night|| +5328|AAAAAAAABNEBAAAA|5328|1|28|48|AM|third|night|| +5329|AAAAAAAACNEBAAAA|5329|1|28|49|AM|third|night|| +5330|AAAAAAAADNEBAAAA|5330|1|28|50|AM|third|night|| +5331|AAAAAAAAENEBAAAA|5331|1|28|51|AM|third|night|| +5332|AAAAAAAAFNEBAAAA|5332|1|28|52|AM|third|night|| +5333|AAAAAAAAGNEBAAAA|5333|1|28|53|AM|third|night|| +5334|AAAAAAAAHNEBAAAA|5334|1|28|54|AM|third|night|| +5335|AAAAAAAAINEBAAAA|5335|1|28|55|AM|third|night|| +5336|AAAAAAAAJNEBAAAA|5336|1|28|56|AM|third|night|| +5337|AAAAAAAAKNEBAAAA|5337|1|28|57|AM|third|night|| +5338|AAAAAAAALNEBAAAA|5338|1|28|58|AM|third|night|| +5339|AAAAAAAAMNEBAAAA|5339|1|28|59|AM|third|night|| +5340|AAAAAAAANNEBAAAA|5340|1|29|0|AM|third|night|| +5341|AAAAAAAAONEBAAAA|5341|1|29|1|AM|third|night|| +5342|AAAAAAAAPNEBAAAA|5342|1|29|2|AM|third|night|| +5343|AAAAAAAAAOEBAAAA|5343|1|29|3|AM|third|night|| +5344|AAAAAAAABOEBAAAA|5344|1|29|4|AM|third|night|| +5345|AAAAAAAACOEBAAAA|5345|1|29|5|AM|third|night|| +5346|AAAAAAAADOEBAAAA|5346|1|29|6|AM|third|night|| +5347|AAAAAAAAEOEBAAAA|5347|1|29|7|AM|third|night|| +5348|AAAAAAAAFOEBAAAA|5348|1|29|8|AM|third|night|| +5349|AAAAAAAAGOEBAAAA|5349|1|29|9|AM|third|night|| +5350|AAAAAAAAHOEBAAAA|5350|1|29|10|AM|third|night|| +5351|AAAAAAAAIOEBAAAA|5351|1|29|11|AM|third|night|| +5352|AAAAAAAAJOEBAAAA|5352|1|29|12|AM|third|night|| +5353|AAAAAAAAKOEBAAAA|5353|1|29|13|AM|third|night|| +5354|AAAAAAAALOEBAAAA|5354|1|29|14|AM|third|night|| +5355|AAAAAAAAMOEBAAAA|5355|1|29|15|AM|third|night|| +5356|AAAAAAAANOEBAAAA|5356|1|29|16|AM|third|night|| +5357|AAAAAAAAOOEBAAAA|5357|1|29|17|AM|third|night|| +5358|AAAAAAAAPOEBAAAA|5358|1|29|18|AM|third|night|| +5359|AAAAAAAAAPEBAAAA|5359|1|29|19|AM|third|night|| +5360|AAAAAAAABPEBAAAA|5360|1|29|20|AM|third|night|| +5361|AAAAAAAACPEBAAAA|5361|1|29|21|AM|third|night|| +5362|AAAAAAAADPEBAAAA|5362|1|29|22|AM|third|night|| +5363|AAAAAAAAEPEBAAAA|5363|1|29|23|AM|third|night|| +5364|AAAAAAAAFPEBAAAA|5364|1|29|24|AM|third|night|| +5365|AAAAAAAAGPEBAAAA|5365|1|29|25|AM|third|night|| +5366|AAAAAAAAHPEBAAAA|5366|1|29|26|AM|third|night|| +5367|AAAAAAAAIPEBAAAA|5367|1|29|27|AM|third|night|| +5368|AAAAAAAAJPEBAAAA|5368|1|29|28|AM|third|night|| +5369|AAAAAAAAKPEBAAAA|5369|1|29|29|AM|third|night|| +5370|AAAAAAAALPEBAAAA|5370|1|29|30|AM|third|night|| +5371|AAAAAAAAMPEBAAAA|5371|1|29|31|AM|third|night|| +5372|AAAAAAAANPEBAAAA|5372|1|29|32|AM|third|night|| +5373|AAAAAAAAOPEBAAAA|5373|1|29|33|AM|third|night|| +5374|AAAAAAAAPPEBAAAA|5374|1|29|34|AM|third|night|| +5375|AAAAAAAAAAFBAAAA|5375|1|29|35|AM|third|night|| +5376|AAAAAAAABAFBAAAA|5376|1|29|36|AM|third|night|| +5377|AAAAAAAACAFBAAAA|5377|1|29|37|AM|third|night|| +5378|AAAAAAAADAFBAAAA|5378|1|29|38|AM|third|night|| +5379|AAAAAAAAEAFBAAAA|5379|1|29|39|AM|third|night|| +5380|AAAAAAAAFAFBAAAA|5380|1|29|40|AM|third|night|| +5381|AAAAAAAAGAFBAAAA|5381|1|29|41|AM|third|night|| +5382|AAAAAAAAHAFBAAAA|5382|1|29|42|AM|third|night|| +5383|AAAAAAAAIAFBAAAA|5383|1|29|43|AM|third|night|| +5384|AAAAAAAAJAFBAAAA|5384|1|29|44|AM|third|night|| +5385|AAAAAAAAKAFBAAAA|5385|1|29|45|AM|third|night|| +5386|AAAAAAAALAFBAAAA|5386|1|29|46|AM|third|night|| +5387|AAAAAAAAMAFBAAAA|5387|1|29|47|AM|third|night|| +5388|AAAAAAAANAFBAAAA|5388|1|29|48|AM|third|night|| +5389|AAAAAAAAOAFBAAAA|5389|1|29|49|AM|third|night|| +5390|AAAAAAAAPAFBAAAA|5390|1|29|50|AM|third|night|| +5391|AAAAAAAAABFBAAAA|5391|1|29|51|AM|third|night|| +5392|AAAAAAAABBFBAAAA|5392|1|29|52|AM|third|night|| +5393|AAAAAAAACBFBAAAA|5393|1|29|53|AM|third|night|| +5394|AAAAAAAADBFBAAAA|5394|1|29|54|AM|third|night|| +5395|AAAAAAAAEBFBAAAA|5395|1|29|55|AM|third|night|| +5396|AAAAAAAAFBFBAAAA|5396|1|29|56|AM|third|night|| +5397|AAAAAAAAGBFBAAAA|5397|1|29|57|AM|third|night|| +5398|AAAAAAAAHBFBAAAA|5398|1|29|58|AM|third|night|| +5399|AAAAAAAAIBFBAAAA|5399|1|29|59|AM|third|night|| +5400|AAAAAAAAJBFBAAAA|5400|1|30|0|AM|third|night|| +5401|AAAAAAAAKBFBAAAA|5401|1|30|1|AM|third|night|| +5402|AAAAAAAALBFBAAAA|5402|1|30|2|AM|third|night|| +5403|AAAAAAAAMBFBAAAA|5403|1|30|3|AM|third|night|| +5404|AAAAAAAANBFBAAAA|5404|1|30|4|AM|third|night|| +5405|AAAAAAAAOBFBAAAA|5405|1|30|5|AM|third|night|| +5406|AAAAAAAAPBFBAAAA|5406|1|30|6|AM|third|night|| +5407|AAAAAAAAACFBAAAA|5407|1|30|7|AM|third|night|| +5408|AAAAAAAABCFBAAAA|5408|1|30|8|AM|third|night|| +5409|AAAAAAAACCFBAAAA|5409|1|30|9|AM|third|night|| +5410|AAAAAAAADCFBAAAA|5410|1|30|10|AM|third|night|| +5411|AAAAAAAAECFBAAAA|5411|1|30|11|AM|third|night|| +5412|AAAAAAAAFCFBAAAA|5412|1|30|12|AM|third|night|| +5413|AAAAAAAAGCFBAAAA|5413|1|30|13|AM|third|night|| +5414|AAAAAAAAHCFBAAAA|5414|1|30|14|AM|third|night|| +5415|AAAAAAAAICFBAAAA|5415|1|30|15|AM|third|night|| +5416|AAAAAAAAJCFBAAAA|5416|1|30|16|AM|third|night|| +5417|AAAAAAAAKCFBAAAA|5417|1|30|17|AM|third|night|| +5418|AAAAAAAALCFBAAAA|5418|1|30|18|AM|third|night|| +5419|AAAAAAAAMCFBAAAA|5419|1|30|19|AM|third|night|| +5420|AAAAAAAANCFBAAAA|5420|1|30|20|AM|third|night|| +5421|AAAAAAAAOCFBAAAA|5421|1|30|21|AM|third|night|| +5422|AAAAAAAAPCFBAAAA|5422|1|30|22|AM|third|night|| +5423|AAAAAAAAADFBAAAA|5423|1|30|23|AM|third|night|| +5424|AAAAAAAABDFBAAAA|5424|1|30|24|AM|third|night|| +5425|AAAAAAAACDFBAAAA|5425|1|30|25|AM|third|night|| +5426|AAAAAAAADDFBAAAA|5426|1|30|26|AM|third|night|| +5427|AAAAAAAAEDFBAAAA|5427|1|30|27|AM|third|night|| +5428|AAAAAAAAFDFBAAAA|5428|1|30|28|AM|third|night|| +5429|AAAAAAAAGDFBAAAA|5429|1|30|29|AM|third|night|| +5430|AAAAAAAAHDFBAAAA|5430|1|30|30|AM|third|night|| +5431|AAAAAAAAIDFBAAAA|5431|1|30|31|AM|third|night|| +5432|AAAAAAAAJDFBAAAA|5432|1|30|32|AM|third|night|| +5433|AAAAAAAAKDFBAAAA|5433|1|30|33|AM|third|night|| +5434|AAAAAAAALDFBAAAA|5434|1|30|34|AM|third|night|| +5435|AAAAAAAAMDFBAAAA|5435|1|30|35|AM|third|night|| +5436|AAAAAAAANDFBAAAA|5436|1|30|36|AM|third|night|| +5437|AAAAAAAAODFBAAAA|5437|1|30|37|AM|third|night|| +5438|AAAAAAAAPDFBAAAA|5438|1|30|38|AM|third|night|| +5439|AAAAAAAAAEFBAAAA|5439|1|30|39|AM|third|night|| +5440|AAAAAAAABEFBAAAA|5440|1|30|40|AM|third|night|| +5441|AAAAAAAACEFBAAAA|5441|1|30|41|AM|third|night|| +5442|AAAAAAAADEFBAAAA|5442|1|30|42|AM|third|night|| +5443|AAAAAAAAEEFBAAAA|5443|1|30|43|AM|third|night|| +5444|AAAAAAAAFEFBAAAA|5444|1|30|44|AM|third|night|| +5445|AAAAAAAAGEFBAAAA|5445|1|30|45|AM|third|night|| +5446|AAAAAAAAHEFBAAAA|5446|1|30|46|AM|third|night|| +5447|AAAAAAAAIEFBAAAA|5447|1|30|47|AM|third|night|| +5448|AAAAAAAAJEFBAAAA|5448|1|30|48|AM|third|night|| +5449|AAAAAAAAKEFBAAAA|5449|1|30|49|AM|third|night|| +5450|AAAAAAAALEFBAAAA|5450|1|30|50|AM|third|night|| +5451|AAAAAAAAMEFBAAAA|5451|1|30|51|AM|third|night|| +5452|AAAAAAAANEFBAAAA|5452|1|30|52|AM|third|night|| +5453|AAAAAAAAOEFBAAAA|5453|1|30|53|AM|third|night|| +5454|AAAAAAAAPEFBAAAA|5454|1|30|54|AM|third|night|| +5455|AAAAAAAAAFFBAAAA|5455|1|30|55|AM|third|night|| +5456|AAAAAAAABFFBAAAA|5456|1|30|56|AM|third|night|| +5457|AAAAAAAACFFBAAAA|5457|1|30|57|AM|third|night|| +5458|AAAAAAAADFFBAAAA|5458|1|30|58|AM|third|night|| +5459|AAAAAAAAEFFBAAAA|5459|1|30|59|AM|third|night|| +5460|AAAAAAAAFFFBAAAA|5460|1|31|0|AM|third|night|| +5461|AAAAAAAAGFFBAAAA|5461|1|31|1|AM|third|night|| +5462|AAAAAAAAHFFBAAAA|5462|1|31|2|AM|third|night|| +5463|AAAAAAAAIFFBAAAA|5463|1|31|3|AM|third|night|| +5464|AAAAAAAAJFFBAAAA|5464|1|31|4|AM|third|night|| +5465|AAAAAAAAKFFBAAAA|5465|1|31|5|AM|third|night|| +5466|AAAAAAAALFFBAAAA|5466|1|31|6|AM|third|night|| +5467|AAAAAAAAMFFBAAAA|5467|1|31|7|AM|third|night|| +5468|AAAAAAAANFFBAAAA|5468|1|31|8|AM|third|night|| +5469|AAAAAAAAOFFBAAAA|5469|1|31|9|AM|third|night|| +5470|AAAAAAAAPFFBAAAA|5470|1|31|10|AM|third|night|| +5471|AAAAAAAAAGFBAAAA|5471|1|31|11|AM|third|night|| +5472|AAAAAAAABGFBAAAA|5472|1|31|12|AM|third|night|| +5473|AAAAAAAACGFBAAAA|5473|1|31|13|AM|third|night|| +5474|AAAAAAAADGFBAAAA|5474|1|31|14|AM|third|night|| +5475|AAAAAAAAEGFBAAAA|5475|1|31|15|AM|third|night|| +5476|AAAAAAAAFGFBAAAA|5476|1|31|16|AM|third|night|| +5477|AAAAAAAAGGFBAAAA|5477|1|31|17|AM|third|night|| +5478|AAAAAAAAHGFBAAAA|5478|1|31|18|AM|third|night|| +5479|AAAAAAAAIGFBAAAA|5479|1|31|19|AM|third|night|| +5480|AAAAAAAAJGFBAAAA|5480|1|31|20|AM|third|night|| +5481|AAAAAAAAKGFBAAAA|5481|1|31|21|AM|third|night|| +5482|AAAAAAAALGFBAAAA|5482|1|31|22|AM|third|night|| +5483|AAAAAAAAMGFBAAAA|5483|1|31|23|AM|third|night|| +5484|AAAAAAAANGFBAAAA|5484|1|31|24|AM|third|night|| +5485|AAAAAAAAOGFBAAAA|5485|1|31|25|AM|third|night|| +5486|AAAAAAAAPGFBAAAA|5486|1|31|26|AM|third|night|| +5487|AAAAAAAAAHFBAAAA|5487|1|31|27|AM|third|night|| +5488|AAAAAAAABHFBAAAA|5488|1|31|28|AM|third|night|| +5489|AAAAAAAACHFBAAAA|5489|1|31|29|AM|third|night|| +5490|AAAAAAAADHFBAAAA|5490|1|31|30|AM|third|night|| +5491|AAAAAAAAEHFBAAAA|5491|1|31|31|AM|third|night|| +5492|AAAAAAAAFHFBAAAA|5492|1|31|32|AM|third|night|| +5493|AAAAAAAAGHFBAAAA|5493|1|31|33|AM|third|night|| +5494|AAAAAAAAHHFBAAAA|5494|1|31|34|AM|third|night|| +5495|AAAAAAAAIHFBAAAA|5495|1|31|35|AM|third|night|| +5496|AAAAAAAAJHFBAAAA|5496|1|31|36|AM|third|night|| +5497|AAAAAAAAKHFBAAAA|5497|1|31|37|AM|third|night|| +5498|AAAAAAAALHFBAAAA|5498|1|31|38|AM|third|night|| +5499|AAAAAAAAMHFBAAAA|5499|1|31|39|AM|third|night|| +5500|AAAAAAAANHFBAAAA|5500|1|31|40|AM|third|night|| +5501|AAAAAAAAOHFBAAAA|5501|1|31|41|AM|third|night|| +5502|AAAAAAAAPHFBAAAA|5502|1|31|42|AM|third|night|| +5503|AAAAAAAAAIFBAAAA|5503|1|31|43|AM|third|night|| +5504|AAAAAAAABIFBAAAA|5504|1|31|44|AM|third|night|| +5505|AAAAAAAACIFBAAAA|5505|1|31|45|AM|third|night|| +5506|AAAAAAAADIFBAAAA|5506|1|31|46|AM|third|night|| +5507|AAAAAAAAEIFBAAAA|5507|1|31|47|AM|third|night|| +5508|AAAAAAAAFIFBAAAA|5508|1|31|48|AM|third|night|| +5509|AAAAAAAAGIFBAAAA|5509|1|31|49|AM|third|night|| +5510|AAAAAAAAHIFBAAAA|5510|1|31|50|AM|third|night|| +5511|AAAAAAAAIIFBAAAA|5511|1|31|51|AM|third|night|| +5512|AAAAAAAAJIFBAAAA|5512|1|31|52|AM|third|night|| +5513|AAAAAAAAKIFBAAAA|5513|1|31|53|AM|third|night|| +5514|AAAAAAAALIFBAAAA|5514|1|31|54|AM|third|night|| +5515|AAAAAAAAMIFBAAAA|5515|1|31|55|AM|third|night|| +5516|AAAAAAAANIFBAAAA|5516|1|31|56|AM|third|night|| +5517|AAAAAAAAOIFBAAAA|5517|1|31|57|AM|third|night|| +5518|AAAAAAAAPIFBAAAA|5518|1|31|58|AM|third|night|| +5519|AAAAAAAAAJFBAAAA|5519|1|31|59|AM|third|night|| +5520|AAAAAAAABJFBAAAA|5520|1|32|0|AM|third|night|| +5521|AAAAAAAACJFBAAAA|5521|1|32|1|AM|third|night|| +5522|AAAAAAAADJFBAAAA|5522|1|32|2|AM|third|night|| +5523|AAAAAAAAEJFBAAAA|5523|1|32|3|AM|third|night|| +5524|AAAAAAAAFJFBAAAA|5524|1|32|4|AM|third|night|| +5525|AAAAAAAAGJFBAAAA|5525|1|32|5|AM|third|night|| +5526|AAAAAAAAHJFBAAAA|5526|1|32|6|AM|third|night|| +5527|AAAAAAAAIJFBAAAA|5527|1|32|7|AM|third|night|| +5528|AAAAAAAAJJFBAAAA|5528|1|32|8|AM|third|night|| +5529|AAAAAAAAKJFBAAAA|5529|1|32|9|AM|third|night|| +5530|AAAAAAAALJFBAAAA|5530|1|32|10|AM|third|night|| +5531|AAAAAAAAMJFBAAAA|5531|1|32|11|AM|third|night|| +5532|AAAAAAAANJFBAAAA|5532|1|32|12|AM|third|night|| +5533|AAAAAAAAOJFBAAAA|5533|1|32|13|AM|third|night|| +5534|AAAAAAAAPJFBAAAA|5534|1|32|14|AM|third|night|| +5535|AAAAAAAAAKFBAAAA|5535|1|32|15|AM|third|night|| +5536|AAAAAAAABKFBAAAA|5536|1|32|16|AM|third|night|| +5537|AAAAAAAACKFBAAAA|5537|1|32|17|AM|third|night|| +5538|AAAAAAAADKFBAAAA|5538|1|32|18|AM|third|night|| +5539|AAAAAAAAEKFBAAAA|5539|1|32|19|AM|third|night|| +5540|AAAAAAAAFKFBAAAA|5540|1|32|20|AM|third|night|| +5541|AAAAAAAAGKFBAAAA|5541|1|32|21|AM|third|night|| +5542|AAAAAAAAHKFBAAAA|5542|1|32|22|AM|third|night|| +5543|AAAAAAAAIKFBAAAA|5543|1|32|23|AM|third|night|| +5544|AAAAAAAAJKFBAAAA|5544|1|32|24|AM|third|night|| +5545|AAAAAAAAKKFBAAAA|5545|1|32|25|AM|third|night|| +5546|AAAAAAAALKFBAAAA|5546|1|32|26|AM|third|night|| +5547|AAAAAAAAMKFBAAAA|5547|1|32|27|AM|third|night|| +5548|AAAAAAAANKFBAAAA|5548|1|32|28|AM|third|night|| +5549|AAAAAAAAOKFBAAAA|5549|1|32|29|AM|third|night|| +5550|AAAAAAAAPKFBAAAA|5550|1|32|30|AM|third|night|| +5551|AAAAAAAAALFBAAAA|5551|1|32|31|AM|third|night|| +5552|AAAAAAAABLFBAAAA|5552|1|32|32|AM|third|night|| +5553|AAAAAAAACLFBAAAA|5553|1|32|33|AM|third|night|| +5554|AAAAAAAADLFBAAAA|5554|1|32|34|AM|third|night|| +5555|AAAAAAAAELFBAAAA|5555|1|32|35|AM|third|night|| +5556|AAAAAAAAFLFBAAAA|5556|1|32|36|AM|third|night|| +5557|AAAAAAAAGLFBAAAA|5557|1|32|37|AM|third|night|| +5558|AAAAAAAAHLFBAAAA|5558|1|32|38|AM|third|night|| +5559|AAAAAAAAILFBAAAA|5559|1|32|39|AM|third|night|| +5560|AAAAAAAAJLFBAAAA|5560|1|32|40|AM|third|night|| +5561|AAAAAAAAKLFBAAAA|5561|1|32|41|AM|third|night|| +5562|AAAAAAAALLFBAAAA|5562|1|32|42|AM|third|night|| +5563|AAAAAAAAMLFBAAAA|5563|1|32|43|AM|third|night|| +5564|AAAAAAAANLFBAAAA|5564|1|32|44|AM|third|night|| +5565|AAAAAAAAOLFBAAAA|5565|1|32|45|AM|third|night|| +5566|AAAAAAAAPLFBAAAA|5566|1|32|46|AM|third|night|| +5567|AAAAAAAAAMFBAAAA|5567|1|32|47|AM|third|night|| +5568|AAAAAAAABMFBAAAA|5568|1|32|48|AM|third|night|| +5569|AAAAAAAACMFBAAAA|5569|1|32|49|AM|third|night|| +5570|AAAAAAAADMFBAAAA|5570|1|32|50|AM|third|night|| +5571|AAAAAAAAEMFBAAAA|5571|1|32|51|AM|third|night|| +5572|AAAAAAAAFMFBAAAA|5572|1|32|52|AM|third|night|| +5573|AAAAAAAAGMFBAAAA|5573|1|32|53|AM|third|night|| +5574|AAAAAAAAHMFBAAAA|5574|1|32|54|AM|third|night|| +5575|AAAAAAAAIMFBAAAA|5575|1|32|55|AM|third|night|| +5576|AAAAAAAAJMFBAAAA|5576|1|32|56|AM|third|night|| +5577|AAAAAAAAKMFBAAAA|5577|1|32|57|AM|third|night|| +5578|AAAAAAAALMFBAAAA|5578|1|32|58|AM|third|night|| +5579|AAAAAAAAMMFBAAAA|5579|1|32|59|AM|third|night|| +5580|AAAAAAAANMFBAAAA|5580|1|33|0|AM|third|night|| +5581|AAAAAAAAOMFBAAAA|5581|1|33|1|AM|third|night|| +5582|AAAAAAAAPMFBAAAA|5582|1|33|2|AM|third|night|| +5583|AAAAAAAAANFBAAAA|5583|1|33|3|AM|third|night|| +5584|AAAAAAAABNFBAAAA|5584|1|33|4|AM|third|night|| +5585|AAAAAAAACNFBAAAA|5585|1|33|5|AM|third|night|| +5586|AAAAAAAADNFBAAAA|5586|1|33|6|AM|third|night|| +5587|AAAAAAAAENFBAAAA|5587|1|33|7|AM|third|night|| +5588|AAAAAAAAFNFBAAAA|5588|1|33|8|AM|third|night|| +5589|AAAAAAAAGNFBAAAA|5589|1|33|9|AM|third|night|| +5590|AAAAAAAAHNFBAAAA|5590|1|33|10|AM|third|night|| +5591|AAAAAAAAINFBAAAA|5591|1|33|11|AM|third|night|| +5592|AAAAAAAAJNFBAAAA|5592|1|33|12|AM|third|night|| +5593|AAAAAAAAKNFBAAAA|5593|1|33|13|AM|third|night|| +5594|AAAAAAAALNFBAAAA|5594|1|33|14|AM|third|night|| +5595|AAAAAAAAMNFBAAAA|5595|1|33|15|AM|third|night|| +5596|AAAAAAAANNFBAAAA|5596|1|33|16|AM|third|night|| +5597|AAAAAAAAONFBAAAA|5597|1|33|17|AM|third|night|| +5598|AAAAAAAAPNFBAAAA|5598|1|33|18|AM|third|night|| +5599|AAAAAAAAAOFBAAAA|5599|1|33|19|AM|third|night|| +5600|AAAAAAAABOFBAAAA|5600|1|33|20|AM|third|night|| +5601|AAAAAAAACOFBAAAA|5601|1|33|21|AM|third|night|| +5602|AAAAAAAADOFBAAAA|5602|1|33|22|AM|third|night|| +5603|AAAAAAAAEOFBAAAA|5603|1|33|23|AM|third|night|| +5604|AAAAAAAAFOFBAAAA|5604|1|33|24|AM|third|night|| +5605|AAAAAAAAGOFBAAAA|5605|1|33|25|AM|third|night|| +5606|AAAAAAAAHOFBAAAA|5606|1|33|26|AM|third|night|| +5607|AAAAAAAAIOFBAAAA|5607|1|33|27|AM|third|night|| +5608|AAAAAAAAJOFBAAAA|5608|1|33|28|AM|third|night|| +5609|AAAAAAAAKOFBAAAA|5609|1|33|29|AM|third|night|| +5610|AAAAAAAALOFBAAAA|5610|1|33|30|AM|third|night|| +5611|AAAAAAAAMOFBAAAA|5611|1|33|31|AM|third|night|| +5612|AAAAAAAANOFBAAAA|5612|1|33|32|AM|third|night|| +5613|AAAAAAAAOOFBAAAA|5613|1|33|33|AM|third|night|| +5614|AAAAAAAAPOFBAAAA|5614|1|33|34|AM|third|night|| +5615|AAAAAAAAAPFBAAAA|5615|1|33|35|AM|third|night|| +5616|AAAAAAAABPFBAAAA|5616|1|33|36|AM|third|night|| +5617|AAAAAAAACPFBAAAA|5617|1|33|37|AM|third|night|| +5618|AAAAAAAADPFBAAAA|5618|1|33|38|AM|third|night|| +5619|AAAAAAAAEPFBAAAA|5619|1|33|39|AM|third|night|| +5620|AAAAAAAAFPFBAAAA|5620|1|33|40|AM|third|night|| +5621|AAAAAAAAGPFBAAAA|5621|1|33|41|AM|third|night|| +5622|AAAAAAAAHPFBAAAA|5622|1|33|42|AM|third|night|| +5623|AAAAAAAAIPFBAAAA|5623|1|33|43|AM|third|night|| +5624|AAAAAAAAJPFBAAAA|5624|1|33|44|AM|third|night|| +5625|AAAAAAAAKPFBAAAA|5625|1|33|45|AM|third|night|| +5626|AAAAAAAALPFBAAAA|5626|1|33|46|AM|third|night|| +5627|AAAAAAAAMPFBAAAA|5627|1|33|47|AM|third|night|| +5628|AAAAAAAANPFBAAAA|5628|1|33|48|AM|third|night|| +5629|AAAAAAAAOPFBAAAA|5629|1|33|49|AM|third|night|| +5630|AAAAAAAAPPFBAAAA|5630|1|33|50|AM|third|night|| +5631|AAAAAAAAAAGBAAAA|5631|1|33|51|AM|third|night|| +5632|AAAAAAAABAGBAAAA|5632|1|33|52|AM|third|night|| +5633|AAAAAAAACAGBAAAA|5633|1|33|53|AM|third|night|| +5634|AAAAAAAADAGBAAAA|5634|1|33|54|AM|third|night|| +5635|AAAAAAAAEAGBAAAA|5635|1|33|55|AM|third|night|| +5636|AAAAAAAAFAGBAAAA|5636|1|33|56|AM|third|night|| +5637|AAAAAAAAGAGBAAAA|5637|1|33|57|AM|third|night|| +5638|AAAAAAAAHAGBAAAA|5638|1|33|58|AM|third|night|| +5639|AAAAAAAAIAGBAAAA|5639|1|33|59|AM|third|night|| +5640|AAAAAAAAJAGBAAAA|5640|1|34|0|AM|third|night|| +5641|AAAAAAAAKAGBAAAA|5641|1|34|1|AM|third|night|| +5642|AAAAAAAALAGBAAAA|5642|1|34|2|AM|third|night|| +5643|AAAAAAAAMAGBAAAA|5643|1|34|3|AM|third|night|| +5644|AAAAAAAANAGBAAAA|5644|1|34|4|AM|third|night|| +5645|AAAAAAAAOAGBAAAA|5645|1|34|5|AM|third|night|| +5646|AAAAAAAAPAGBAAAA|5646|1|34|6|AM|third|night|| +5647|AAAAAAAAABGBAAAA|5647|1|34|7|AM|third|night|| +5648|AAAAAAAABBGBAAAA|5648|1|34|8|AM|third|night|| +5649|AAAAAAAACBGBAAAA|5649|1|34|9|AM|third|night|| +5650|AAAAAAAADBGBAAAA|5650|1|34|10|AM|third|night|| +5651|AAAAAAAAEBGBAAAA|5651|1|34|11|AM|third|night|| +5652|AAAAAAAAFBGBAAAA|5652|1|34|12|AM|third|night|| +5653|AAAAAAAAGBGBAAAA|5653|1|34|13|AM|third|night|| +5654|AAAAAAAAHBGBAAAA|5654|1|34|14|AM|third|night|| +5655|AAAAAAAAIBGBAAAA|5655|1|34|15|AM|third|night|| +5656|AAAAAAAAJBGBAAAA|5656|1|34|16|AM|third|night|| +5657|AAAAAAAAKBGBAAAA|5657|1|34|17|AM|third|night|| +5658|AAAAAAAALBGBAAAA|5658|1|34|18|AM|third|night|| +5659|AAAAAAAAMBGBAAAA|5659|1|34|19|AM|third|night|| +5660|AAAAAAAANBGBAAAA|5660|1|34|20|AM|third|night|| +5661|AAAAAAAAOBGBAAAA|5661|1|34|21|AM|third|night|| +5662|AAAAAAAAPBGBAAAA|5662|1|34|22|AM|third|night|| +5663|AAAAAAAAACGBAAAA|5663|1|34|23|AM|third|night|| +5664|AAAAAAAABCGBAAAA|5664|1|34|24|AM|third|night|| +5665|AAAAAAAACCGBAAAA|5665|1|34|25|AM|third|night|| +5666|AAAAAAAADCGBAAAA|5666|1|34|26|AM|third|night|| +5667|AAAAAAAAECGBAAAA|5667|1|34|27|AM|third|night|| +5668|AAAAAAAAFCGBAAAA|5668|1|34|28|AM|third|night|| +5669|AAAAAAAAGCGBAAAA|5669|1|34|29|AM|third|night|| +5670|AAAAAAAAHCGBAAAA|5670|1|34|30|AM|third|night|| +5671|AAAAAAAAICGBAAAA|5671|1|34|31|AM|third|night|| +5672|AAAAAAAAJCGBAAAA|5672|1|34|32|AM|third|night|| +5673|AAAAAAAAKCGBAAAA|5673|1|34|33|AM|third|night|| +5674|AAAAAAAALCGBAAAA|5674|1|34|34|AM|third|night|| +5675|AAAAAAAAMCGBAAAA|5675|1|34|35|AM|third|night|| +5676|AAAAAAAANCGBAAAA|5676|1|34|36|AM|third|night|| +5677|AAAAAAAAOCGBAAAA|5677|1|34|37|AM|third|night|| +5678|AAAAAAAAPCGBAAAA|5678|1|34|38|AM|third|night|| +5679|AAAAAAAAADGBAAAA|5679|1|34|39|AM|third|night|| +5680|AAAAAAAABDGBAAAA|5680|1|34|40|AM|third|night|| +5681|AAAAAAAACDGBAAAA|5681|1|34|41|AM|third|night|| +5682|AAAAAAAADDGBAAAA|5682|1|34|42|AM|third|night|| +5683|AAAAAAAAEDGBAAAA|5683|1|34|43|AM|third|night|| +5684|AAAAAAAAFDGBAAAA|5684|1|34|44|AM|third|night|| +5685|AAAAAAAAGDGBAAAA|5685|1|34|45|AM|third|night|| +5686|AAAAAAAAHDGBAAAA|5686|1|34|46|AM|third|night|| +5687|AAAAAAAAIDGBAAAA|5687|1|34|47|AM|third|night|| +5688|AAAAAAAAJDGBAAAA|5688|1|34|48|AM|third|night|| +5689|AAAAAAAAKDGBAAAA|5689|1|34|49|AM|third|night|| +5690|AAAAAAAALDGBAAAA|5690|1|34|50|AM|third|night|| +5691|AAAAAAAAMDGBAAAA|5691|1|34|51|AM|third|night|| +5692|AAAAAAAANDGBAAAA|5692|1|34|52|AM|third|night|| +5693|AAAAAAAAODGBAAAA|5693|1|34|53|AM|third|night|| +5694|AAAAAAAAPDGBAAAA|5694|1|34|54|AM|third|night|| +5695|AAAAAAAAAEGBAAAA|5695|1|34|55|AM|third|night|| +5696|AAAAAAAABEGBAAAA|5696|1|34|56|AM|third|night|| +5697|AAAAAAAACEGBAAAA|5697|1|34|57|AM|third|night|| +5698|AAAAAAAADEGBAAAA|5698|1|34|58|AM|third|night|| +5699|AAAAAAAAEEGBAAAA|5699|1|34|59|AM|third|night|| +5700|AAAAAAAAFEGBAAAA|5700|1|35|0|AM|third|night|| +5701|AAAAAAAAGEGBAAAA|5701|1|35|1|AM|third|night|| +5702|AAAAAAAAHEGBAAAA|5702|1|35|2|AM|third|night|| +5703|AAAAAAAAIEGBAAAA|5703|1|35|3|AM|third|night|| +5704|AAAAAAAAJEGBAAAA|5704|1|35|4|AM|third|night|| +5705|AAAAAAAAKEGBAAAA|5705|1|35|5|AM|third|night|| +5706|AAAAAAAALEGBAAAA|5706|1|35|6|AM|third|night|| +5707|AAAAAAAAMEGBAAAA|5707|1|35|7|AM|third|night|| +5708|AAAAAAAANEGBAAAA|5708|1|35|8|AM|third|night|| +5709|AAAAAAAAOEGBAAAA|5709|1|35|9|AM|third|night|| +5710|AAAAAAAAPEGBAAAA|5710|1|35|10|AM|third|night|| +5711|AAAAAAAAAFGBAAAA|5711|1|35|11|AM|third|night|| +5712|AAAAAAAABFGBAAAA|5712|1|35|12|AM|third|night|| +5713|AAAAAAAACFGBAAAA|5713|1|35|13|AM|third|night|| +5714|AAAAAAAADFGBAAAA|5714|1|35|14|AM|third|night|| +5715|AAAAAAAAEFGBAAAA|5715|1|35|15|AM|third|night|| +5716|AAAAAAAAFFGBAAAA|5716|1|35|16|AM|third|night|| +5717|AAAAAAAAGFGBAAAA|5717|1|35|17|AM|third|night|| +5718|AAAAAAAAHFGBAAAA|5718|1|35|18|AM|third|night|| +5719|AAAAAAAAIFGBAAAA|5719|1|35|19|AM|third|night|| +5720|AAAAAAAAJFGBAAAA|5720|1|35|20|AM|third|night|| +5721|AAAAAAAAKFGBAAAA|5721|1|35|21|AM|third|night|| +5722|AAAAAAAALFGBAAAA|5722|1|35|22|AM|third|night|| +5723|AAAAAAAAMFGBAAAA|5723|1|35|23|AM|third|night|| +5724|AAAAAAAANFGBAAAA|5724|1|35|24|AM|third|night|| +5725|AAAAAAAAOFGBAAAA|5725|1|35|25|AM|third|night|| +5726|AAAAAAAAPFGBAAAA|5726|1|35|26|AM|third|night|| +5727|AAAAAAAAAGGBAAAA|5727|1|35|27|AM|third|night|| +5728|AAAAAAAABGGBAAAA|5728|1|35|28|AM|third|night|| +5729|AAAAAAAACGGBAAAA|5729|1|35|29|AM|third|night|| +5730|AAAAAAAADGGBAAAA|5730|1|35|30|AM|third|night|| +5731|AAAAAAAAEGGBAAAA|5731|1|35|31|AM|third|night|| +5732|AAAAAAAAFGGBAAAA|5732|1|35|32|AM|third|night|| +5733|AAAAAAAAGGGBAAAA|5733|1|35|33|AM|third|night|| +5734|AAAAAAAAHGGBAAAA|5734|1|35|34|AM|third|night|| +5735|AAAAAAAAIGGBAAAA|5735|1|35|35|AM|third|night|| +5736|AAAAAAAAJGGBAAAA|5736|1|35|36|AM|third|night|| +5737|AAAAAAAAKGGBAAAA|5737|1|35|37|AM|third|night|| +5738|AAAAAAAALGGBAAAA|5738|1|35|38|AM|third|night|| +5739|AAAAAAAAMGGBAAAA|5739|1|35|39|AM|third|night|| +5740|AAAAAAAANGGBAAAA|5740|1|35|40|AM|third|night|| +5741|AAAAAAAAOGGBAAAA|5741|1|35|41|AM|third|night|| +5742|AAAAAAAAPGGBAAAA|5742|1|35|42|AM|third|night|| +5743|AAAAAAAAAHGBAAAA|5743|1|35|43|AM|third|night|| +5744|AAAAAAAABHGBAAAA|5744|1|35|44|AM|third|night|| +5745|AAAAAAAACHGBAAAA|5745|1|35|45|AM|third|night|| +5746|AAAAAAAADHGBAAAA|5746|1|35|46|AM|third|night|| +5747|AAAAAAAAEHGBAAAA|5747|1|35|47|AM|third|night|| +5748|AAAAAAAAFHGBAAAA|5748|1|35|48|AM|third|night|| +5749|AAAAAAAAGHGBAAAA|5749|1|35|49|AM|third|night|| +5750|AAAAAAAAHHGBAAAA|5750|1|35|50|AM|third|night|| +5751|AAAAAAAAIHGBAAAA|5751|1|35|51|AM|third|night|| +5752|AAAAAAAAJHGBAAAA|5752|1|35|52|AM|third|night|| +5753|AAAAAAAAKHGBAAAA|5753|1|35|53|AM|third|night|| +5754|AAAAAAAALHGBAAAA|5754|1|35|54|AM|third|night|| +5755|AAAAAAAAMHGBAAAA|5755|1|35|55|AM|third|night|| +5756|AAAAAAAANHGBAAAA|5756|1|35|56|AM|third|night|| +5757|AAAAAAAAOHGBAAAA|5757|1|35|57|AM|third|night|| +5758|AAAAAAAAPHGBAAAA|5758|1|35|58|AM|third|night|| +5759|AAAAAAAAAIGBAAAA|5759|1|35|59|AM|third|night|| +5760|AAAAAAAABIGBAAAA|5760|1|36|0|AM|third|night|| +5761|AAAAAAAACIGBAAAA|5761|1|36|1|AM|third|night|| +5762|AAAAAAAADIGBAAAA|5762|1|36|2|AM|third|night|| +5763|AAAAAAAAEIGBAAAA|5763|1|36|3|AM|third|night|| +5764|AAAAAAAAFIGBAAAA|5764|1|36|4|AM|third|night|| +5765|AAAAAAAAGIGBAAAA|5765|1|36|5|AM|third|night|| +5766|AAAAAAAAHIGBAAAA|5766|1|36|6|AM|third|night|| +5767|AAAAAAAAIIGBAAAA|5767|1|36|7|AM|third|night|| +5768|AAAAAAAAJIGBAAAA|5768|1|36|8|AM|third|night|| +5769|AAAAAAAAKIGBAAAA|5769|1|36|9|AM|third|night|| +5770|AAAAAAAALIGBAAAA|5770|1|36|10|AM|third|night|| +5771|AAAAAAAAMIGBAAAA|5771|1|36|11|AM|third|night|| +5772|AAAAAAAANIGBAAAA|5772|1|36|12|AM|third|night|| +5773|AAAAAAAAOIGBAAAA|5773|1|36|13|AM|third|night|| +5774|AAAAAAAAPIGBAAAA|5774|1|36|14|AM|third|night|| +5775|AAAAAAAAAJGBAAAA|5775|1|36|15|AM|third|night|| +5776|AAAAAAAABJGBAAAA|5776|1|36|16|AM|third|night|| +5777|AAAAAAAACJGBAAAA|5777|1|36|17|AM|third|night|| +5778|AAAAAAAADJGBAAAA|5778|1|36|18|AM|third|night|| +5779|AAAAAAAAEJGBAAAA|5779|1|36|19|AM|third|night|| +5780|AAAAAAAAFJGBAAAA|5780|1|36|20|AM|third|night|| +5781|AAAAAAAAGJGBAAAA|5781|1|36|21|AM|third|night|| +5782|AAAAAAAAHJGBAAAA|5782|1|36|22|AM|third|night|| +5783|AAAAAAAAIJGBAAAA|5783|1|36|23|AM|third|night|| +5784|AAAAAAAAJJGBAAAA|5784|1|36|24|AM|third|night|| +5785|AAAAAAAAKJGBAAAA|5785|1|36|25|AM|third|night|| +5786|AAAAAAAALJGBAAAA|5786|1|36|26|AM|third|night|| +5787|AAAAAAAAMJGBAAAA|5787|1|36|27|AM|third|night|| +5788|AAAAAAAANJGBAAAA|5788|1|36|28|AM|third|night|| +5789|AAAAAAAAOJGBAAAA|5789|1|36|29|AM|third|night|| +5790|AAAAAAAAPJGBAAAA|5790|1|36|30|AM|third|night|| +5791|AAAAAAAAAKGBAAAA|5791|1|36|31|AM|third|night|| +5792|AAAAAAAABKGBAAAA|5792|1|36|32|AM|third|night|| +5793|AAAAAAAACKGBAAAA|5793|1|36|33|AM|third|night|| +5794|AAAAAAAADKGBAAAA|5794|1|36|34|AM|third|night|| +5795|AAAAAAAAEKGBAAAA|5795|1|36|35|AM|third|night|| +5796|AAAAAAAAFKGBAAAA|5796|1|36|36|AM|third|night|| +5797|AAAAAAAAGKGBAAAA|5797|1|36|37|AM|third|night|| +5798|AAAAAAAAHKGBAAAA|5798|1|36|38|AM|third|night|| +5799|AAAAAAAAIKGBAAAA|5799|1|36|39|AM|third|night|| +5800|AAAAAAAAJKGBAAAA|5800|1|36|40|AM|third|night|| +5801|AAAAAAAAKKGBAAAA|5801|1|36|41|AM|third|night|| +5802|AAAAAAAALKGBAAAA|5802|1|36|42|AM|third|night|| +5803|AAAAAAAAMKGBAAAA|5803|1|36|43|AM|third|night|| +5804|AAAAAAAANKGBAAAA|5804|1|36|44|AM|third|night|| +5805|AAAAAAAAOKGBAAAA|5805|1|36|45|AM|third|night|| +5806|AAAAAAAAPKGBAAAA|5806|1|36|46|AM|third|night|| +5807|AAAAAAAAALGBAAAA|5807|1|36|47|AM|third|night|| +5808|AAAAAAAABLGBAAAA|5808|1|36|48|AM|third|night|| +5809|AAAAAAAACLGBAAAA|5809|1|36|49|AM|third|night|| +5810|AAAAAAAADLGBAAAA|5810|1|36|50|AM|third|night|| +5811|AAAAAAAAELGBAAAA|5811|1|36|51|AM|third|night|| +5812|AAAAAAAAFLGBAAAA|5812|1|36|52|AM|third|night|| +5813|AAAAAAAAGLGBAAAA|5813|1|36|53|AM|third|night|| +5814|AAAAAAAAHLGBAAAA|5814|1|36|54|AM|third|night|| +5815|AAAAAAAAILGBAAAA|5815|1|36|55|AM|third|night|| +5816|AAAAAAAAJLGBAAAA|5816|1|36|56|AM|third|night|| +5817|AAAAAAAAKLGBAAAA|5817|1|36|57|AM|third|night|| +5818|AAAAAAAALLGBAAAA|5818|1|36|58|AM|third|night|| +5819|AAAAAAAAMLGBAAAA|5819|1|36|59|AM|third|night|| +5820|AAAAAAAANLGBAAAA|5820|1|37|0|AM|third|night|| +5821|AAAAAAAAOLGBAAAA|5821|1|37|1|AM|third|night|| +5822|AAAAAAAAPLGBAAAA|5822|1|37|2|AM|third|night|| +5823|AAAAAAAAAMGBAAAA|5823|1|37|3|AM|third|night|| +5824|AAAAAAAABMGBAAAA|5824|1|37|4|AM|third|night|| +5825|AAAAAAAACMGBAAAA|5825|1|37|5|AM|third|night|| +5826|AAAAAAAADMGBAAAA|5826|1|37|6|AM|third|night|| +5827|AAAAAAAAEMGBAAAA|5827|1|37|7|AM|third|night|| +5828|AAAAAAAAFMGBAAAA|5828|1|37|8|AM|third|night|| +5829|AAAAAAAAGMGBAAAA|5829|1|37|9|AM|third|night|| +5830|AAAAAAAAHMGBAAAA|5830|1|37|10|AM|third|night|| +5831|AAAAAAAAIMGBAAAA|5831|1|37|11|AM|third|night|| +5832|AAAAAAAAJMGBAAAA|5832|1|37|12|AM|third|night|| +5833|AAAAAAAAKMGBAAAA|5833|1|37|13|AM|third|night|| +5834|AAAAAAAALMGBAAAA|5834|1|37|14|AM|third|night|| +5835|AAAAAAAAMMGBAAAA|5835|1|37|15|AM|third|night|| +5836|AAAAAAAANMGBAAAA|5836|1|37|16|AM|third|night|| +5837|AAAAAAAAOMGBAAAA|5837|1|37|17|AM|third|night|| +5838|AAAAAAAAPMGBAAAA|5838|1|37|18|AM|third|night|| +5839|AAAAAAAAANGBAAAA|5839|1|37|19|AM|third|night|| +5840|AAAAAAAABNGBAAAA|5840|1|37|20|AM|third|night|| +5841|AAAAAAAACNGBAAAA|5841|1|37|21|AM|third|night|| +5842|AAAAAAAADNGBAAAA|5842|1|37|22|AM|third|night|| +5843|AAAAAAAAENGBAAAA|5843|1|37|23|AM|third|night|| +5844|AAAAAAAAFNGBAAAA|5844|1|37|24|AM|third|night|| +5845|AAAAAAAAGNGBAAAA|5845|1|37|25|AM|third|night|| +5846|AAAAAAAAHNGBAAAA|5846|1|37|26|AM|third|night|| +5847|AAAAAAAAINGBAAAA|5847|1|37|27|AM|third|night|| +5848|AAAAAAAAJNGBAAAA|5848|1|37|28|AM|third|night|| +5849|AAAAAAAAKNGBAAAA|5849|1|37|29|AM|third|night|| +5850|AAAAAAAALNGBAAAA|5850|1|37|30|AM|third|night|| +5851|AAAAAAAAMNGBAAAA|5851|1|37|31|AM|third|night|| +5852|AAAAAAAANNGBAAAA|5852|1|37|32|AM|third|night|| +5853|AAAAAAAAONGBAAAA|5853|1|37|33|AM|third|night|| +5854|AAAAAAAAPNGBAAAA|5854|1|37|34|AM|third|night|| +5855|AAAAAAAAAOGBAAAA|5855|1|37|35|AM|third|night|| +5856|AAAAAAAABOGBAAAA|5856|1|37|36|AM|third|night|| +5857|AAAAAAAACOGBAAAA|5857|1|37|37|AM|third|night|| +5858|AAAAAAAADOGBAAAA|5858|1|37|38|AM|third|night|| +5859|AAAAAAAAEOGBAAAA|5859|1|37|39|AM|third|night|| +5860|AAAAAAAAFOGBAAAA|5860|1|37|40|AM|third|night|| +5861|AAAAAAAAGOGBAAAA|5861|1|37|41|AM|third|night|| +5862|AAAAAAAAHOGBAAAA|5862|1|37|42|AM|third|night|| +5863|AAAAAAAAIOGBAAAA|5863|1|37|43|AM|third|night|| +5864|AAAAAAAAJOGBAAAA|5864|1|37|44|AM|third|night|| +5865|AAAAAAAAKOGBAAAA|5865|1|37|45|AM|third|night|| +5866|AAAAAAAALOGBAAAA|5866|1|37|46|AM|third|night|| +5867|AAAAAAAAMOGBAAAA|5867|1|37|47|AM|third|night|| +5868|AAAAAAAANOGBAAAA|5868|1|37|48|AM|third|night|| +5869|AAAAAAAAOOGBAAAA|5869|1|37|49|AM|third|night|| +5870|AAAAAAAAPOGBAAAA|5870|1|37|50|AM|third|night|| +5871|AAAAAAAAAPGBAAAA|5871|1|37|51|AM|third|night|| +5872|AAAAAAAABPGBAAAA|5872|1|37|52|AM|third|night|| +5873|AAAAAAAACPGBAAAA|5873|1|37|53|AM|third|night|| +5874|AAAAAAAADPGBAAAA|5874|1|37|54|AM|third|night|| +5875|AAAAAAAAEPGBAAAA|5875|1|37|55|AM|third|night|| +5876|AAAAAAAAFPGBAAAA|5876|1|37|56|AM|third|night|| +5877|AAAAAAAAGPGBAAAA|5877|1|37|57|AM|third|night|| +5878|AAAAAAAAHPGBAAAA|5878|1|37|58|AM|third|night|| +5879|AAAAAAAAIPGBAAAA|5879|1|37|59|AM|third|night|| +5880|AAAAAAAAJPGBAAAA|5880|1|38|0|AM|third|night|| +5881|AAAAAAAAKPGBAAAA|5881|1|38|1|AM|third|night|| +5882|AAAAAAAALPGBAAAA|5882|1|38|2|AM|third|night|| +5883|AAAAAAAAMPGBAAAA|5883|1|38|3|AM|third|night|| +5884|AAAAAAAANPGBAAAA|5884|1|38|4|AM|third|night|| +5885|AAAAAAAAOPGBAAAA|5885|1|38|5|AM|third|night|| +5886|AAAAAAAAPPGBAAAA|5886|1|38|6|AM|third|night|| +5887|AAAAAAAAAAHBAAAA|5887|1|38|7|AM|third|night|| +5888|AAAAAAAABAHBAAAA|5888|1|38|8|AM|third|night|| +5889|AAAAAAAACAHBAAAA|5889|1|38|9|AM|third|night|| +5890|AAAAAAAADAHBAAAA|5890|1|38|10|AM|third|night|| +5891|AAAAAAAAEAHBAAAA|5891|1|38|11|AM|third|night|| +5892|AAAAAAAAFAHBAAAA|5892|1|38|12|AM|third|night|| +5893|AAAAAAAAGAHBAAAA|5893|1|38|13|AM|third|night|| +5894|AAAAAAAAHAHBAAAA|5894|1|38|14|AM|third|night|| +5895|AAAAAAAAIAHBAAAA|5895|1|38|15|AM|third|night|| +5896|AAAAAAAAJAHBAAAA|5896|1|38|16|AM|third|night|| +5897|AAAAAAAAKAHBAAAA|5897|1|38|17|AM|third|night|| +5898|AAAAAAAALAHBAAAA|5898|1|38|18|AM|third|night|| +5899|AAAAAAAAMAHBAAAA|5899|1|38|19|AM|third|night|| +5900|AAAAAAAANAHBAAAA|5900|1|38|20|AM|third|night|| +5901|AAAAAAAAOAHBAAAA|5901|1|38|21|AM|third|night|| +5902|AAAAAAAAPAHBAAAA|5902|1|38|22|AM|third|night|| +5903|AAAAAAAAABHBAAAA|5903|1|38|23|AM|third|night|| +5904|AAAAAAAABBHBAAAA|5904|1|38|24|AM|third|night|| +5905|AAAAAAAACBHBAAAA|5905|1|38|25|AM|third|night|| +5906|AAAAAAAADBHBAAAA|5906|1|38|26|AM|third|night|| +5907|AAAAAAAAEBHBAAAA|5907|1|38|27|AM|third|night|| +5908|AAAAAAAAFBHBAAAA|5908|1|38|28|AM|third|night|| +5909|AAAAAAAAGBHBAAAA|5909|1|38|29|AM|third|night|| +5910|AAAAAAAAHBHBAAAA|5910|1|38|30|AM|third|night|| +5911|AAAAAAAAIBHBAAAA|5911|1|38|31|AM|third|night|| +5912|AAAAAAAAJBHBAAAA|5912|1|38|32|AM|third|night|| +5913|AAAAAAAAKBHBAAAA|5913|1|38|33|AM|third|night|| +5914|AAAAAAAALBHBAAAA|5914|1|38|34|AM|third|night|| +5915|AAAAAAAAMBHBAAAA|5915|1|38|35|AM|third|night|| +5916|AAAAAAAANBHBAAAA|5916|1|38|36|AM|third|night|| +5917|AAAAAAAAOBHBAAAA|5917|1|38|37|AM|third|night|| +5918|AAAAAAAAPBHBAAAA|5918|1|38|38|AM|third|night|| +5919|AAAAAAAAACHBAAAA|5919|1|38|39|AM|third|night|| +5920|AAAAAAAABCHBAAAA|5920|1|38|40|AM|third|night|| +5921|AAAAAAAACCHBAAAA|5921|1|38|41|AM|third|night|| +5922|AAAAAAAADCHBAAAA|5922|1|38|42|AM|third|night|| +5923|AAAAAAAAECHBAAAA|5923|1|38|43|AM|third|night|| +5924|AAAAAAAAFCHBAAAA|5924|1|38|44|AM|third|night|| +5925|AAAAAAAAGCHBAAAA|5925|1|38|45|AM|third|night|| +5926|AAAAAAAAHCHBAAAA|5926|1|38|46|AM|third|night|| +5927|AAAAAAAAICHBAAAA|5927|1|38|47|AM|third|night|| +5928|AAAAAAAAJCHBAAAA|5928|1|38|48|AM|third|night|| +5929|AAAAAAAAKCHBAAAA|5929|1|38|49|AM|third|night|| +5930|AAAAAAAALCHBAAAA|5930|1|38|50|AM|third|night|| +5931|AAAAAAAAMCHBAAAA|5931|1|38|51|AM|third|night|| +5932|AAAAAAAANCHBAAAA|5932|1|38|52|AM|third|night|| +5933|AAAAAAAAOCHBAAAA|5933|1|38|53|AM|third|night|| +5934|AAAAAAAAPCHBAAAA|5934|1|38|54|AM|third|night|| +5935|AAAAAAAAADHBAAAA|5935|1|38|55|AM|third|night|| +5936|AAAAAAAABDHBAAAA|5936|1|38|56|AM|third|night|| +5937|AAAAAAAACDHBAAAA|5937|1|38|57|AM|third|night|| +5938|AAAAAAAADDHBAAAA|5938|1|38|58|AM|third|night|| +5939|AAAAAAAAEDHBAAAA|5939|1|38|59|AM|third|night|| +5940|AAAAAAAAFDHBAAAA|5940|1|39|0|AM|third|night|| +5941|AAAAAAAAGDHBAAAA|5941|1|39|1|AM|third|night|| +5942|AAAAAAAAHDHBAAAA|5942|1|39|2|AM|third|night|| +5943|AAAAAAAAIDHBAAAA|5943|1|39|3|AM|third|night|| +5944|AAAAAAAAJDHBAAAA|5944|1|39|4|AM|third|night|| +5945|AAAAAAAAKDHBAAAA|5945|1|39|5|AM|third|night|| +5946|AAAAAAAALDHBAAAA|5946|1|39|6|AM|third|night|| +5947|AAAAAAAAMDHBAAAA|5947|1|39|7|AM|third|night|| +5948|AAAAAAAANDHBAAAA|5948|1|39|8|AM|third|night|| +5949|AAAAAAAAODHBAAAA|5949|1|39|9|AM|third|night|| +5950|AAAAAAAAPDHBAAAA|5950|1|39|10|AM|third|night|| +5951|AAAAAAAAAEHBAAAA|5951|1|39|11|AM|third|night|| +5952|AAAAAAAABEHBAAAA|5952|1|39|12|AM|third|night|| +5953|AAAAAAAACEHBAAAA|5953|1|39|13|AM|third|night|| +5954|AAAAAAAADEHBAAAA|5954|1|39|14|AM|third|night|| +5955|AAAAAAAAEEHBAAAA|5955|1|39|15|AM|third|night|| +5956|AAAAAAAAFEHBAAAA|5956|1|39|16|AM|third|night|| +5957|AAAAAAAAGEHBAAAA|5957|1|39|17|AM|third|night|| +5958|AAAAAAAAHEHBAAAA|5958|1|39|18|AM|third|night|| +5959|AAAAAAAAIEHBAAAA|5959|1|39|19|AM|third|night|| +5960|AAAAAAAAJEHBAAAA|5960|1|39|20|AM|third|night|| +5961|AAAAAAAAKEHBAAAA|5961|1|39|21|AM|third|night|| +5962|AAAAAAAALEHBAAAA|5962|1|39|22|AM|third|night|| +5963|AAAAAAAAMEHBAAAA|5963|1|39|23|AM|third|night|| +5964|AAAAAAAANEHBAAAA|5964|1|39|24|AM|third|night|| +5965|AAAAAAAAOEHBAAAA|5965|1|39|25|AM|third|night|| +5966|AAAAAAAAPEHBAAAA|5966|1|39|26|AM|third|night|| +5967|AAAAAAAAAFHBAAAA|5967|1|39|27|AM|third|night|| +5968|AAAAAAAABFHBAAAA|5968|1|39|28|AM|third|night|| +5969|AAAAAAAACFHBAAAA|5969|1|39|29|AM|third|night|| +5970|AAAAAAAADFHBAAAA|5970|1|39|30|AM|third|night|| +5971|AAAAAAAAEFHBAAAA|5971|1|39|31|AM|third|night|| +5972|AAAAAAAAFFHBAAAA|5972|1|39|32|AM|third|night|| +5973|AAAAAAAAGFHBAAAA|5973|1|39|33|AM|third|night|| +5974|AAAAAAAAHFHBAAAA|5974|1|39|34|AM|third|night|| +5975|AAAAAAAAIFHBAAAA|5975|1|39|35|AM|third|night|| +5976|AAAAAAAAJFHBAAAA|5976|1|39|36|AM|third|night|| +5977|AAAAAAAAKFHBAAAA|5977|1|39|37|AM|third|night|| +5978|AAAAAAAALFHBAAAA|5978|1|39|38|AM|third|night|| +5979|AAAAAAAAMFHBAAAA|5979|1|39|39|AM|third|night|| +5980|AAAAAAAANFHBAAAA|5980|1|39|40|AM|third|night|| +5981|AAAAAAAAOFHBAAAA|5981|1|39|41|AM|third|night|| +5982|AAAAAAAAPFHBAAAA|5982|1|39|42|AM|third|night|| +5983|AAAAAAAAAGHBAAAA|5983|1|39|43|AM|third|night|| +5984|AAAAAAAABGHBAAAA|5984|1|39|44|AM|third|night|| +5985|AAAAAAAACGHBAAAA|5985|1|39|45|AM|third|night|| +5986|AAAAAAAADGHBAAAA|5986|1|39|46|AM|third|night|| +5987|AAAAAAAAEGHBAAAA|5987|1|39|47|AM|third|night|| +5988|AAAAAAAAFGHBAAAA|5988|1|39|48|AM|third|night|| +5989|AAAAAAAAGGHBAAAA|5989|1|39|49|AM|third|night|| +5990|AAAAAAAAHGHBAAAA|5990|1|39|50|AM|third|night|| +5991|AAAAAAAAIGHBAAAA|5991|1|39|51|AM|third|night|| +5992|AAAAAAAAJGHBAAAA|5992|1|39|52|AM|third|night|| +5993|AAAAAAAAKGHBAAAA|5993|1|39|53|AM|third|night|| +5994|AAAAAAAALGHBAAAA|5994|1|39|54|AM|third|night|| +5995|AAAAAAAAMGHBAAAA|5995|1|39|55|AM|third|night|| +5996|AAAAAAAANGHBAAAA|5996|1|39|56|AM|third|night|| +5997|AAAAAAAAOGHBAAAA|5997|1|39|57|AM|third|night|| +5998|AAAAAAAAPGHBAAAA|5998|1|39|58|AM|third|night|| +5999|AAAAAAAAAHHBAAAA|5999|1|39|59|AM|third|night|| +6000|AAAAAAAABHHBAAAA|6000|1|40|0|AM|third|night|| +6001|AAAAAAAACHHBAAAA|6001|1|40|1|AM|third|night|| +6002|AAAAAAAADHHBAAAA|6002|1|40|2|AM|third|night|| +6003|AAAAAAAAEHHBAAAA|6003|1|40|3|AM|third|night|| +6004|AAAAAAAAFHHBAAAA|6004|1|40|4|AM|third|night|| +6005|AAAAAAAAGHHBAAAA|6005|1|40|5|AM|third|night|| +6006|AAAAAAAAHHHBAAAA|6006|1|40|6|AM|third|night|| +6007|AAAAAAAAIHHBAAAA|6007|1|40|7|AM|third|night|| +6008|AAAAAAAAJHHBAAAA|6008|1|40|8|AM|third|night|| +6009|AAAAAAAAKHHBAAAA|6009|1|40|9|AM|third|night|| +6010|AAAAAAAALHHBAAAA|6010|1|40|10|AM|third|night|| +6011|AAAAAAAAMHHBAAAA|6011|1|40|11|AM|third|night|| +6012|AAAAAAAANHHBAAAA|6012|1|40|12|AM|third|night|| +6013|AAAAAAAAOHHBAAAA|6013|1|40|13|AM|third|night|| +6014|AAAAAAAAPHHBAAAA|6014|1|40|14|AM|third|night|| +6015|AAAAAAAAAIHBAAAA|6015|1|40|15|AM|third|night|| +6016|AAAAAAAABIHBAAAA|6016|1|40|16|AM|third|night|| +6017|AAAAAAAACIHBAAAA|6017|1|40|17|AM|third|night|| +6018|AAAAAAAADIHBAAAA|6018|1|40|18|AM|third|night|| +6019|AAAAAAAAEIHBAAAA|6019|1|40|19|AM|third|night|| +6020|AAAAAAAAFIHBAAAA|6020|1|40|20|AM|third|night|| +6021|AAAAAAAAGIHBAAAA|6021|1|40|21|AM|third|night|| +6022|AAAAAAAAHIHBAAAA|6022|1|40|22|AM|third|night|| +6023|AAAAAAAAIIHBAAAA|6023|1|40|23|AM|third|night|| +6024|AAAAAAAAJIHBAAAA|6024|1|40|24|AM|third|night|| +6025|AAAAAAAAKIHBAAAA|6025|1|40|25|AM|third|night|| +6026|AAAAAAAALIHBAAAA|6026|1|40|26|AM|third|night|| +6027|AAAAAAAAMIHBAAAA|6027|1|40|27|AM|third|night|| +6028|AAAAAAAANIHBAAAA|6028|1|40|28|AM|third|night|| +6029|AAAAAAAAOIHBAAAA|6029|1|40|29|AM|third|night|| +6030|AAAAAAAAPIHBAAAA|6030|1|40|30|AM|third|night|| +6031|AAAAAAAAAJHBAAAA|6031|1|40|31|AM|third|night|| +6032|AAAAAAAABJHBAAAA|6032|1|40|32|AM|third|night|| +6033|AAAAAAAACJHBAAAA|6033|1|40|33|AM|third|night|| +6034|AAAAAAAADJHBAAAA|6034|1|40|34|AM|third|night|| +6035|AAAAAAAAEJHBAAAA|6035|1|40|35|AM|third|night|| +6036|AAAAAAAAFJHBAAAA|6036|1|40|36|AM|third|night|| +6037|AAAAAAAAGJHBAAAA|6037|1|40|37|AM|third|night|| +6038|AAAAAAAAHJHBAAAA|6038|1|40|38|AM|third|night|| +6039|AAAAAAAAIJHBAAAA|6039|1|40|39|AM|third|night|| +6040|AAAAAAAAJJHBAAAA|6040|1|40|40|AM|third|night|| +6041|AAAAAAAAKJHBAAAA|6041|1|40|41|AM|third|night|| +6042|AAAAAAAALJHBAAAA|6042|1|40|42|AM|third|night|| +6043|AAAAAAAAMJHBAAAA|6043|1|40|43|AM|third|night|| +6044|AAAAAAAANJHBAAAA|6044|1|40|44|AM|third|night|| +6045|AAAAAAAAOJHBAAAA|6045|1|40|45|AM|third|night|| +6046|AAAAAAAAPJHBAAAA|6046|1|40|46|AM|third|night|| +6047|AAAAAAAAAKHBAAAA|6047|1|40|47|AM|third|night|| +6048|AAAAAAAABKHBAAAA|6048|1|40|48|AM|third|night|| +6049|AAAAAAAACKHBAAAA|6049|1|40|49|AM|third|night|| +6050|AAAAAAAADKHBAAAA|6050|1|40|50|AM|third|night|| +6051|AAAAAAAAEKHBAAAA|6051|1|40|51|AM|third|night|| +6052|AAAAAAAAFKHBAAAA|6052|1|40|52|AM|third|night|| +6053|AAAAAAAAGKHBAAAA|6053|1|40|53|AM|third|night|| +6054|AAAAAAAAHKHBAAAA|6054|1|40|54|AM|third|night|| +6055|AAAAAAAAIKHBAAAA|6055|1|40|55|AM|third|night|| +6056|AAAAAAAAJKHBAAAA|6056|1|40|56|AM|third|night|| +6057|AAAAAAAAKKHBAAAA|6057|1|40|57|AM|third|night|| +6058|AAAAAAAALKHBAAAA|6058|1|40|58|AM|third|night|| +6059|AAAAAAAAMKHBAAAA|6059|1|40|59|AM|third|night|| +6060|AAAAAAAANKHBAAAA|6060|1|41|0|AM|third|night|| +6061|AAAAAAAAOKHBAAAA|6061|1|41|1|AM|third|night|| +6062|AAAAAAAAPKHBAAAA|6062|1|41|2|AM|third|night|| +6063|AAAAAAAAALHBAAAA|6063|1|41|3|AM|third|night|| +6064|AAAAAAAABLHBAAAA|6064|1|41|4|AM|third|night|| +6065|AAAAAAAACLHBAAAA|6065|1|41|5|AM|third|night|| +6066|AAAAAAAADLHBAAAA|6066|1|41|6|AM|third|night|| +6067|AAAAAAAAELHBAAAA|6067|1|41|7|AM|third|night|| +6068|AAAAAAAAFLHBAAAA|6068|1|41|8|AM|third|night|| +6069|AAAAAAAAGLHBAAAA|6069|1|41|9|AM|third|night|| +6070|AAAAAAAAHLHBAAAA|6070|1|41|10|AM|third|night|| +6071|AAAAAAAAILHBAAAA|6071|1|41|11|AM|third|night|| +6072|AAAAAAAAJLHBAAAA|6072|1|41|12|AM|third|night|| +6073|AAAAAAAAKLHBAAAA|6073|1|41|13|AM|third|night|| +6074|AAAAAAAALLHBAAAA|6074|1|41|14|AM|third|night|| +6075|AAAAAAAAMLHBAAAA|6075|1|41|15|AM|third|night|| +6076|AAAAAAAANLHBAAAA|6076|1|41|16|AM|third|night|| +6077|AAAAAAAAOLHBAAAA|6077|1|41|17|AM|third|night|| +6078|AAAAAAAAPLHBAAAA|6078|1|41|18|AM|third|night|| +6079|AAAAAAAAAMHBAAAA|6079|1|41|19|AM|third|night|| +6080|AAAAAAAABMHBAAAA|6080|1|41|20|AM|third|night|| +6081|AAAAAAAACMHBAAAA|6081|1|41|21|AM|third|night|| +6082|AAAAAAAADMHBAAAA|6082|1|41|22|AM|third|night|| +6083|AAAAAAAAEMHBAAAA|6083|1|41|23|AM|third|night|| +6084|AAAAAAAAFMHBAAAA|6084|1|41|24|AM|third|night|| +6085|AAAAAAAAGMHBAAAA|6085|1|41|25|AM|third|night|| +6086|AAAAAAAAHMHBAAAA|6086|1|41|26|AM|third|night|| +6087|AAAAAAAAIMHBAAAA|6087|1|41|27|AM|third|night|| +6088|AAAAAAAAJMHBAAAA|6088|1|41|28|AM|third|night|| +6089|AAAAAAAAKMHBAAAA|6089|1|41|29|AM|third|night|| +6090|AAAAAAAALMHBAAAA|6090|1|41|30|AM|third|night|| +6091|AAAAAAAAMMHBAAAA|6091|1|41|31|AM|third|night|| +6092|AAAAAAAANMHBAAAA|6092|1|41|32|AM|third|night|| +6093|AAAAAAAAOMHBAAAA|6093|1|41|33|AM|third|night|| +6094|AAAAAAAAPMHBAAAA|6094|1|41|34|AM|third|night|| +6095|AAAAAAAAANHBAAAA|6095|1|41|35|AM|third|night|| +6096|AAAAAAAABNHBAAAA|6096|1|41|36|AM|third|night|| +6097|AAAAAAAACNHBAAAA|6097|1|41|37|AM|third|night|| +6098|AAAAAAAADNHBAAAA|6098|1|41|38|AM|third|night|| +6099|AAAAAAAAENHBAAAA|6099|1|41|39|AM|third|night|| +6100|AAAAAAAAFNHBAAAA|6100|1|41|40|AM|third|night|| +6101|AAAAAAAAGNHBAAAA|6101|1|41|41|AM|third|night|| +6102|AAAAAAAAHNHBAAAA|6102|1|41|42|AM|third|night|| +6103|AAAAAAAAINHBAAAA|6103|1|41|43|AM|third|night|| +6104|AAAAAAAAJNHBAAAA|6104|1|41|44|AM|third|night|| +6105|AAAAAAAAKNHBAAAA|6105|1|41|45|AM|third|night|| +6106|AAAAAAAALNHBAAAA|6106|1|41|46|AM|third|night|| +6107|AAAAAAAAMNHBAAAA|6107|1|41|47|AM|third|night|| +6108|AAAAAAAANNHBAAAA|6108|1|41|48|AM|third|night|| +6109|AAAAAAAAONHBAAAA|6109|1|41|49|AM|third|night|| +6110|AAAAAAAAPNHBAAAA|6110|1|41|50|AM|third|night|| +6111|AAAAAAAAAOHBAAAA|6111|1|41|51|AM|third|night|| +6112|AAAAAAAABOHBAAAA|6112|1|41|52|AM|third|night|| +6113|AAAAAAAACOHBAAAA|6113|1|41|53|AM|third|night|| +6114|AAAAAAAADOHBAAAA|6114|1|41|54|AM|third|night|| +6115|AAAAAAAAEOHBAAAA|6115|1|41|55|AM|third|night|| +6116|AAAAAAAAFOHBAAAA|6116|1|41|56|AM|third|night|| +6117|AAAAAAAAGOHBAAAA|6117|1|41|57|AM|third|night|| +6118|AAAAAAAAHOHBAAAA|6118|1|41|58|AM|third|night|| +6119|AAAAAAAAIOHBAAAA|6119|1|41|59|AM|third|night|| +6120|AAAAAAAAJOHBAAAA|6120|1|42|0|AM|third|night|| +6121|AAAAAAAAKOHBAAAA|6121|1|42|1|AM|third|night|| +6122|AAAAAAAALOHBAAAA|6122|1|42|2|AM|third|night|| +6123|AAAAAAAAMOHBAAAA|6123|1|42|3|AM|third|night|| +6124|AAAAAAAANOHBAAAA|6124|1|42|4|AM|third|night|| +6125|AAAAAAAAOOHBAAAA|6125|1|42|5|AM|third|night|| +6126|AAAAAAAAPOHBAAAA|6126|1|42|6|AM|third|night|| +6127|AAAAAAAAAPHBAAAA|6127|1|42|7|AM|third|night|| +6128|AAAAAAAABPHBAAAA|6128|1|42|8|AM|third|night|| +6129|AAAAAAAACPHBAAAA|6129|1|42|9|AM|third|night|| +6130|AAAAAAAADPHBAAAA|6130|1|42|10|AM|third|night|| +6131|AAAAAAAAEPHBAAAA|6131|1|42|11|AM|third|night|| +6132|AAAAAAAAFPHBAAAA|6132|1|42|12|AM|third|night|| +6133|AAAAAAAAGPHBAAAA|6133|1|42|13|AM|third|night|| +6134|AAAAAAAAHPHBAAAA|6134|1|42|14|AM|third|night|| +6135|AAAAAAAAIPHBAAAA|6135|1|42|15|AM|third|night|| +6136|AAAAAAAAJPHBAAAA|6136|1|42|16|AM|third|night|| +6137|AAAAAAAAKPHBAAAA|6137|1|42|17|AM|third|night|| +6138|AAAAAAAALPHBAAAA|6138|1|42|18|AM|third|night|| +6139|AAAAAAAAMPHBAAAA|6139|1|42|19|AM|third|night|| +6140|AAAAAAAANPHBAAAA|6140|1|42|20|AM|third|night|| +6141|AAAAAAAAOPHBAAAA|6141|1|42|21|AM|third|night|| +6142|AAAAAAAAPPHBAAAA|6142|1|42|22|AM|third|night|| +6143|AAAAAAAAAAIBAAAA|6143|1|42|23|AM|third|night|| +6144|AAAAAAAABAIBAAAA|6144|1|42|24|AM|third|night|| +6145|AAAAAAAACAIBAAAA|6145|1|42|25|AM|third|night|| +6146|AAAAAAAADAIBAAAA|6146|1|42|26|AM|third|night|| +6147|AAAAAAAAEAIBAAAA|6147|1|42|27|AM|third|night|| +6148|AAAAAAAAFAIBAAAA|6148|1|42|28|AM|third|night|| +6149|AAAAAAAAGAIBAAAA|6149|1|42|29|AM|third|night|| +6150|AAAAAAAAHAIBAAAA|6150|1|42|30|AM|third|night|| +6151|AAAAAAAAIAIBAAAA|6151|1|42|31|AM|third|night|| +6152|AAAAAAAAJAIBAAAA|6152|1|42|32|AM|third|night|| +6153|AAAAAAAAKAIBAAAA|6153|1|42|33|AM|third|night|| +6154|AAAAAAAALAIBAAAA|6154|1|42|34|AM|third|night|| +6155|AAAAAAAAMAIBAAAA|6155|1|42|35|AM|third|night|| +6156|AAAAAAAANAIBAAAA|6156|1|42|36|AM|third|night|| +6157|AAAAAAAAOAIBAAAA|6157|1|42|37|AM|third|night|| +6158|AAAAAAAAPAIBAAAA|6158|1|42|38|AM|third|night|| +6159|AAAAAAAAABIBAAAA|6159|1|42|39|AM|third|night|| +6160|AAAAAAAABBIBAAAA|6160|1|42|40|AM|third|night|| +6161|AAAAAAAACBIBAAAA|6161|1|42|41|AM|third|night|| +6162|AAAAAAAADBIBAAAA|6162|1|42|42|AM|third|night|| +6163|AAAAAAAAEBIBAAAA|6163|1|42|43|AM|third|night|| +6164|AAAAAAAAFBIBAAAA|6164|1|42|44|AM|third|night|| +6165|AAAAAAAAGBIBAAAA|6165|1|42|45|AM|third|night|| +6166|AAAAAAAAHBIBAAAA|6166|1|42|46|AM|third|night|| +6167|AAAAAAAAIBIBAAAA|6167|1|42|47|AM|third|night|| +6168|AAAAAAAAJBIBAAAA|6168|1|42|48|AM|third|night|| +6169|AAAAAAAAKBIBAAAA|6169|1|42|49|AM|third|night|| +6170|AAAAAAAALBIBAAAA|6170|1|42|50|AM|third|night|| +6171|AAAAAAAAMBIBAAAA|6171|1|42|51|AM|third|night|| +6172|AAAAAAAANBIBAAAA|6172|1|42|52|AM|third|night|| +6173|AAAAAAAAOBIBAAAA|6173|1|42|53|AM|third|night|| +6174|AAAAAAAAPBIBAAAA|6174|1|42|54|AM|third|night|| +6175|AAAAAAAAACIBAAAA|6175|1|42|55|AM|third|night|| +6176|AAAAAAAABCIBAAAA|6176|1|42|56|AM|third|night|| +6177|AAAAAAAACCIBAAAA|6177|1|42|57|AM|third|night|| +6178|AAAAAAAADCIBAAAA|6178|1|42|58|AM|third|night|| +6179|AAAAAAAAECIBAAAA|6179|1|42|59|AM|third|night|| +6180|AAAAAAAAFCIBAAAA|6180|1|43|0|AM|third|night|| +6181|AAAAAAAAGCIBAAAA|6181|1|43|1|AM|third|night|| +6182|AAAAAAAAHCIBAAAA|6182|1|43|2|AM|third|night|| +6183|AAAAAAAAICIBAAAA|6183|1|43|3|AM|third|night|| +6184|AAAAAAAAJCIBAAAA|6184|1|43|4|AM|third|night|| +6185|AAAAAAAAKCIBAAAA|6185|1|43|5|AM|third|night|| +6186|AAAAAAAALCIBAAAA|6186|1|43|6|AM|third|night|| +6187|AAAAAAAAMCIBAAAA|6187|1|43|7|AM|third|night|| +6188|AAAAAAAANCIBAAAA|6188|1|43|8|AM|third|night|| +6189|AAAAAAAAOCIBAAAA|6189|1|43|9|AM|third|night|| +6190|AAAAAAAAPCIBAAAA|6190|1|43|10|AM|third|night|| +6191|AAAAAAAAADIBAAAA|6191|1|43|11|AM|third|night|| +6192|AAAAAAAABDIBAAAA|6192|1|43|12|AM|third|night|| +6193|AAAAAAAACDIBAAAA|6193|1|43|13|AM|third|night|| +6194|AAAAAAAADDIBAAAA|6194|1|43|14|AM|third|night|| +6195|AAAAAAAAEDIBAAAA|6195|1|43|15|AM|third|night|| +6196|AAAAAAAAFDIBAAAA|6196|1|43|16|AM|third|night|| +6197|AAAAAAAAGDIBAAAA|6197|1|43|17|AM|third|night|| +6198|AAAAAAAAHDIBAAAA|6198|1|43|18|AM|third|night|| +6199|AAAAAAAAIDIBAAAA|6199|1|43|19|AM|third|night|| +6200|AAAAAAAAJDIBAAAA|6200|1|43|20|AM|third|night|| +6201|AAAAAAAAKDIBAAAA|6201|1|43|21|AM|third|night|| +6202|AAAAAAAALDIBAAAA|6202|1|43|22|AM|third|night|| +6203|AAAAAAAAMDIBAAAA|6203|1|43|23|AM|third|night|| +6204|AAAAAAAANDIBAAAA|6204|1|43|24|AM|third|night|| +6205|AAAAAAAAODIBAAAA|6205|1|43|25|AM|third|night|| +6206|AAAAAAAAPDIBAAAA|6206|1|43|26|AM|third|night|| +6207|AAAAAAAAAEIBAAAA|6207|1|43|27|AM|third|night|| +6208|AAAAAAAABEIBAAAA|6208|1|43|28|AM|third|night|| +6209|AAAAAAAACEIBAAAA|6209|1|43|29|AM|third|night|| +6210|AAAAAAAADEIBAAAA|6210|1|43|30|AM|third|night|| +6211|AAAAAAAAEEIBAAAA|6211|1|43|31|AM|third|night|| +6212|AAAAAAAAFEIBAAAA|6212|1|43|32|AM|third|night|| +6213|AAAAAAAAGEIBAAAA|6213|1|43|33|AM|third|night|| +6214|AAAAAAAAHEIBAAAA|6214|1|43|34|AM|third|night|| +6215|AAAAAAAAIEIBAAAA|6215|1|43|35|AM|third|night|| +6216|AAAAAAAAJEIBAAAA|6216|1|43|36|AM|third|night|| +6217|AAAAAAAAKEIBAAAA|6217|1|43|37|AM|third|night|| +6218|AAAAAAAALEIBAAAA|6218|1|43|38|AM|third|night|| +6219|AAAAAAAAMEIBAAAA|6219|1|43|39|AM|third|night|| +6220|AAAAAAAANEIBAAAA|6220|1|43|40|AM|third|night|| +6221|AAAAAAAAOEIBAAAA|6221|1|43|41|AM|third|night|| +6222|AAAAAAAAPEIBAAAA|6222|1|43|42|AM|third|night|| +6223|AAAAAAAAAFIBAAAA|6223|1|43|43|AM|third|night|| +6224|AAAAAAAABFIBAAAA|6224|1|43|44|AM|third|night|| +6225|AAAAAAAACFIBAAAA|6225|1|43|45|AM|third|night|| +6226|AAAAAAAADFIBAAAA|6226|1|43|46|AM|third|night|| +6227|AAAAAAAAEFIBAAAA|6227|1|43|47|AM|third|night|| +6228|AAAAAAAAFFIBAAAA|6228|1|43|48|AM|third|night|| +6229|AAAAAAAAGFIBAAAA|6229|1|43|49|AM|third|night|| +6230|AAAAAAAAHFIBAAAA|6230|1|43|50|AM|third|night|| +6231|AAAAAAAAIFIBAAAA|6231|1|43|51|AM|third|night|| +6232|AAAAAAAAJFIBAAAA|6232|1|43|52|AM|third|night|| +6233|AAAAAAAAKFIBAAAA|6233|1|43|53|AM|third|night|| +6234|AAAAAAAALFIBAAAA|6234|1|43|54|AM|third|night|| +6235|AAAAAAAAMFIBAAAA|6235|1|43|55|AM|third|night|| +6236|AAAAAAAANFIBAAAA|6236|1|43|56|AM|third|night|| +6237|AAAAAAAAOFIBAAAA|6237|1|43|57|AM|third|night|| +6238|AAAAAAAAPFIBAAAA|6238|1|43|58|AM|third|night|| +6239|AAAAAAAAAGIBAAAA|6239|1|43|59|AM|third|night|| +6240|AAAAAAAABGIBAAAA|6240|1|44|0|AM|third|night|| +6241|AAAAAAAACGIBAAAA|6241|1|44|1|AM|third|night|| +6242|AAAAAAAADGIBAAAA|6242|1|44|2|AM|third|night|| +6243|AAAAAAAAEGIBAAAA|6243|1|44|3|AM|third|night|| +6244|AAAAAAAAFGIBAAAA|6244|1|44|4|AM|third|night|| +6245|AAAAAAAAGGIBAAAA|6245|1|44|5|AM|third|night|| +6246|AAAAAAAAHGIBAAAA|6246|1|44|6|AM|third|night|| +6247|AAAAAAAAIGIBAAAA|6247|1|44|7|AM|third|night|| +6248|AAAAAAAAJGIBAAAA|6248|1|44|8|AM|third|night|| +6249|AAAAAAAAKGIBAAAA|6249|1|44|9|AM|third|night|| +6250|AAAAAAAALGIBAAAA|6250|1|44|10|AM|third|night|| +6251|AAAAAAAAMGIBAAAA|6251|1|44|11|AM|third|night|| +6252|AAAAAAAANGIBAAAA|6252|1|44|12|AM|third|night|| +6253|AAAAAAAAOGIBAAAA|6253|1|44|13|AM|third|night|| +6254|AAAAAAAAPGIBAAAA|6254|1|44|14|AM|third|night|| +6255|AAAAAAAAAHIBAAAA|6255|1|44|15|AM|third|night|| +6256|AAAAAAAABHIBAAAA|6256|1|44|16|AM|third|night|| +6257|AAAAAAAACHIBAAAA|6257|1|44|17|AM|third|night|| +6258|AAAAAAAADHIBAAAA|6258|1|44|18|AM|third|night|| +6259|AAAAAAAAEHIBAAAA|6259|1|44|19|AM|third|night|| +6260|AAAAAAAAFHIBAAAA|6260|1|44|20|AM|third|night|| +6261|AAAAAAAAGHIBAAAA|6261|1|44|21|AM|third|night|| +6262|AAAAAAAAHHIBAAAA|6262|1|44|22|AM|third|night|| +6263|AAAAAAAAIHIBAAAA|6263|1|44|23|AM|third|night|| +6264|AAAAAAAAJHIBAAAA|6264|1|44|24|AM|third|night|| +6265|AAAAAAAAKHIBAAAA|6265|1|44|25|AM|third|night|| +6266|AAAAAAAALHIBAAAA|6266|1|44|26|AM|third|night|| +6267|AAAAAAAAMHIBAAAA|6267|1|44|27|AM|third|night|| +6268|AAAAAAAANHIBAAAA|6268|1|44|28|AM|third|night|| +6269|AAAAAAAAOHIBAAAA|6269|1|44|29|AM|third|night|| +6270|AAAAAAAAPHIBAAAA|6270|1|44|30|AM|third|night|| +6271|AAAAAAAAAIIBAAAA|6271|1|44|31|AM|third|night|| +6272|AAAAAAAABIIBAAAA|6272|1|44|32|AM|third|night|| +6273|AAAAAAAACIIBAAAA|6273|1|44|33|AM|third|night|| +6274|AAAAAAAADIIBAAAA|6274|1|44|34|AM|third|night|| +6275|AAAAAAAAEIIBAAAA|6275|1|44|35|AM|third|night|| +6276|AAAAAAAAFIIBAAAA|6276|1|44|36|AM|third|night|| +6277|AAAAAAAAGIIBAAAA|6277|1|44|37|AM|third|night|| +6278|AAAAAAAAHIIBAAAA|6278|1|44|38|AM|third|night|| +6279|AAAAAAAAIIIBAAAA|6279|1|44|39|AM|third|night|| +6280|AAAAAAAAJIIBAAAA|6280|1|44|40|AM|third|night|| +6281|AAAAAAAAKIIBAAAA|6281|1|44|41|AM|third|night|| +6282|AAAAAAAALIIBAAAA|6282|1|44|42|AM|third|night|| +6283|AAAAAAAAMIIBAAAA|6283|1|44|43|AM|third|night|| +6284|AAAAAAAANIIBAAAA|6284|1|44|44|AM|third|night|| +6285|AAAAAAAAOIIBAAAA|6285|1|44|45|AM|third|night|| +6286|AAAAAAAAPIIBAAAA|6286|1|44|46|AM|third|night|| +6287|AAAAAAAAAJIBAAAA|6287|1|44|47|AM|third|night|| +6288|AAAAAAAABJIBAAAA|6288|1|44|48|AM|third|night|| +6289|AAAAAAAACJIBAAAA|6289|1|44|49|AM|third|night|| +6290|AAAAAAAADJIBAAAA|6290|1|44|50|AM|third|night|| +6291|AAAAAAAAEJIBAAAA|6291|1|44|51|AM|third|night|| +6292|AAAAAAAAFJIBAAAA|6292|1|44|52|AM|third|night|| +6293|AAAAAAAAGJIBAAAA|6293|1|44|53|AM|third|night|| +6294|AAAAAAAAHJIBAAAA|6294|1|44|54|AM|third|night|| +6295|AAAAAAAAIJIBAAAA|6295|1|44|55|AM|third|night|| +6296|AAAAAAAAJJIBAAAA|6296|1|44|56|AM|third|night|| +6297|AAAAAAAAKJIBAAAA|6297|1|44|57|AM|third|night|| +6298|AAAAAAAALJIBAAAA|6298|1|44|58|AM|third|night|| +6299|AAAAAAAAMJIBAAAA|6299|1|44|59|AM|third|night|| +6300|AAAAAAAANJIBAAAA|6300|1|45|0|AM|third|night|| +6301|AAAAAAAAOJIBAAAA|6301|1|45|1|AM|third|night|| +6302|AAAAAAAAPJIBAAAA|6302|1|45|2|AM|third|night|| +6303|AAAAAAAAAKIBAAAA|6303|1|45|3|AM|third|night|| +6304|AAAAAAAABKIBAAAA|6304|1|45|4|AM|third|night|| +6305|AAAAAAAACKIBAAAA|6305|1|45|5|AM|third|night|| +6306|AAAAAAAADKIBAAAA|6306|1|45|6|AM|third|night|| +6307|AAAAAAAAEKIBAAAA|6307|1|45|7|AM|third|night|| +6308|AAAAAAAAFKIBAAAA|6308|1|45|8|AM|third|night|| +6309|AAAAAAAAGKIBAAAA|6309|1|45|9|AM|third|night|| +6310|AAAAAAAAHKIBAAAA|6310|1|45|10|AM|third|night|| +6311|AAAAAAAAIKIBAAAA|6311|1|45|11|AM|third|night|| +6312|AAAAAAAAJKIBAAAA|6312|1|45|12|AM|third|night|| +6313|AAAAAAAAKKIBAAAA|6313|1|45|13|AM|third|night|| +6314|AAAAAAAALKIBAAAA|6314|1|45|14|AM|third|night|| +6315|AAAAAAAAMKIBAAAA|6315|1|45|15|AM|third|night|| +6316|AAAAAAAANKIBAAAA|6316|1|45|16|AM|third|night|| +6317|AAAAAAAAOKIBAAAA|6317|1|45|17|AM|third|night|| +6318|AAAAAAAAPKIBAAAA|6318|1|45|18|AM|third|night|| +6319|AAAAAAAAALIBAAAA|6319|1|45|19|AM|third|night|| +6320|AAAAAAAABLIBAAAA|6320|1|45|20|AM|third|night|| +6321|AAAAAAAACLIBAAAA|6321|1|45|21|AM|third|night|| +6322|AAAAAAAADLIBAAAA|6322|1|45|22|AM|third|night|| +6323|AAAAAAAAELIBAAAA|6323|1|45|23|AM|third|night|| +6324|AAAAAAAAFLIBAAAA|6324|1|45|24|AM|third|night|| +6325|AAAAAAAAGLIBAAAA|6325|1|45|25|AM|third|night|| +6326|AAAAAAAAHLIBAAAA|6326|1|45|26|AM|third|night|| +6327|AAAAAAAAILIBAAAA|6327|1|45|27|AM|third|night|| +6328|AAAAAAAAJLIBAAAA|6328|1|45|28|AM|third|night|| +6329|AAAAAAAAKLIBAAAA|6329|1|45|29|AM|third|night|| +6330|AAAAAAAALLIBAAAA|6330|1|45|30|AM|third|night|| +6331|AAAAAAAAMLIBAAAA|6331|1|45|31|AM|third|night|| +6332|AAAAAAAANLIBAAAA|6332|1|45|32|AM|third|night|| +6333|AAAAAAAAOLIBAAAA|6333|1|45|33|AM|third|night|| +6334|AAAAAAAAPLIBAAAA|6334|1|45|34|AM|third|night|| +6335|AAAAAAAAAMIBAAAA|6335|1|45|35|AM|third|night|| +6336|AAAAAAAABMIBAAAA|6336|1|45|36|AM|third|night|| +6337|AAAAAAAACMIBAAAA|6337|1|45|37|AM|third|night|| +6338|AAAAAAAADMIBAAAA|6338|1|45|38|AM|third|night|| +6339|AAAAAAAAEMIBAAAA|6339|1|45|39|AM|third|night|| +6340|AAAAAAAAFMIBAAAA|6340|1|45|40|AM|third|night|| +6341|AAAAAAAAGMIBAAAA|6341|1|45|41|AM|third|night|| +6342|AAAAAAAAHMIBAAAA|6342|1|45|42|AM|third|night|| +6343|AAAAAAAAIMIBAAAA|6343|1|45|43|AM|third|night|| +6344|AAAAAAAAJMIBAAAA|6344|1|45|44|AM|third|night|| +6345|AAAAAAAAKMIBAAAA|6345|1|45|45|AM|third|night|| +6346|AAAAAAAALMIBAAAA|6346|1|45|46|AM|third|night|| +6347|AAAAAAAAMMIBAAAA|6347|1|45|47|AM|third|night|| +6348|AAAAAAAANMIBAAAA|6348|1|45|48|AM|third|night|| +6349|AAAAAAAAOMIBAAAA|6349|1|45|49|AM|third|night|| +6350|AAAAAAAAPMIBAAAA|6350|1|45|50|AM|third|night|| +6351|AAAAAAAAANIBAAAA|6351|1|45|51|AM|third|night|| +6352|AAAAAAAABNIBAAAA|6352|1|45|52|AM|third|night|| +6353|AAAAAAAACNIBAAAA|6353|1|45|53|AM|third|night|| +6354|AAAAAAAADNIBAAAA|6354|1|45|54|AM|third|night|| +6355|AAAAAAAAENIBAAAA|6355|1|45|55|AM|third|night|| +6356|AAAAAAAAFNIBAAAA|6356|1|45|56|AM|third|night|| +6357|AAAAAAAAGNIBAAAA|6357|1|45|57|AM|third|night|| +6358|AAAAAAAAHNIBAAAA|6358|1|45|58|AM|third|night|| +6359|AAAAAAAAINIBAAAA|6359|1|45|59|AM|third|night|| +6360|AAAAAAAAJNIBAAAA|6360|1|46|0|AM|third|night|| +6361|AAAAAAAAKNIBAAAA|6361|1|46|1|AM|third|night|| +6362|AAAAAAAALNIBAAAA|6362|1|46|2|AM|third|night|| +6363|AAAAAAAAMNIBAAAA|6363|1|46|3|AM|third|night|| +6364|AAAAAAAANNIBAAAA|6364|1|46|4|AM|third|night|| +6365|AAAAAAAAONIBAAAA|6365|1|46|5|AM|third|night|| +6366|AAAAAAAAPNIBAAAA|6366|1|46|6|AM|third|night|| +6367|AAAAAAAAAOIBAAAA|6367|1|46|7|AM|third|night|| +6368|AAAAAAAABOIBAAAA|6368|1|46|8|AM|third|night|| +6369|AAAAAAAACOIBAAAA|6369|1|46|9|AM|third|night|| +6370|AAAAAAAADOIBAAAA|6370|1|46|10|AM|third|night|| +6371|AAAAAAAAEOIBAAAA|6371|1|46|11|AM|third|night|| +6372|AAAAAAAAFOIBAAAA|6372|1|46|12|AM|third|night|| +6373|AAAAAAAAGOIBAAAA|6373|1|46|13|AM|third|night|| +6374|AAAAAAAAHOIBAAAA|6374|1|46|14|AM|third|night|| +6375|AAAAAAAAIOIBAAAA|6375|1|46|15|AM|third|night|| +6376|AAAAAAAAJOIBAAAA|6376|1|46|16|AM|third|night|| +6377|AAAAAAAAKOIBAAAA|6377|1|46|17|AM|third|night|| +6378|AAAAAAAALOIBAAAA|6378|1|46|18|AM|third|night|| +6379|AAAAAAAAMOIBAAAA|6379|1|46|19|AM|third|night|| +6380|AAAAAAAANOIBAAAA|6380|1|46|20|AM|third|night|| +6381|AAAAAAAAOOIBAAAA|6381|1|46|21|AM|third|night|| +6382|AAAAAAAAPOIBAAAA|6382|1|46|22|AM|third|night|| +6383|AAAAAAAAAPIBAAAA|6383|1|46|23|AM|third|night|| +6384|AAAAAAAABPIBAAAA|6384|1|46|24|AM|third|night|| +6385|AAAAAAAACPIBAAAA|6385|1|46|25|AM|third|night|| +6386|AAAAAAAADPIBAAAA|6386|1|46|26|AM|third|night|| +6387|AAAAAAAAEPIBAAAA|6387|1|46|27|AM|third|night|| +6388|AAAAAAAAFPIBAAAA|6388|1|46|28|AM|third|night|| +6389|AAAAAAAAGPIBAAAA|6389|1|46|29|AM|third|night|| +6390|AAAAAAAAHPIBAAAA|6390|1|46|30|AM|third|night|| +6391|AAAAAAAAIPIBAAAA|6391|1|46|31|AM|third|night|| +6392|AAAAAAAAJPIBAAAA|6392|1|46|32|AM|third|night|| +6393|AAAAAAAAKPIBAAAA|6393|1|46|33|AM|third|night|| +6394|AAAAAAAALPIBAAAA|6394|1|46|34|AM|third|night|| +6395|AAAAAAAAMPIBAAAA|6395|1|46|35|AM|third|night|| +6396|AAAAAAAANPIBAAAA|6396|1|46|36|AM|third|night|| +6397|AAAAAAAAOPIBAAAA|6397|1|46|37|AM|third|night|| +6398|AAAAAAAAPPIBAAAA|6398|1|46|38|AM|third|night|| +6399|AAAAAAAAAAJBAAAA|6399|1|46|39|AM|third|night|| +6400|AAAAAAAABAJBAAAA|6400|1|46|40|AM|third|night|| +6401|AAAAAAAACAJBAAAA|6401|1|46|41|AM|third|night|| +6402|AAAAAAAADAJBAAAA|6402|1|46|42|AM|third|night|| +6403|AAAAAAAAEAJBAAAA|6403|1|46|43|AM|third|night|| +6404|AAAAAAAAFAJBAAAA|6404|1|46|44|AM|third|night|| +6405|AAAAAAAAGAJBAAAA|6405|1|46|45|AM|third|night|| +6406|AAAAAAAAHAJBAAAA|6406|1|46|46|AM|third|night|| +6407|AAAAAAAAIAJBAAAA|6407|1|46|47|AM|third|night|| +6408|AAAAAAAAJAJBAAAA|6408|1|46|48|AM|third|night|| +6409|AAAAAAAAKAJBAAAA|6409|1|46|49|AM|third|night|| +6410|AAAAAAAALAJBAAAA|6410|1|46|50|AM|third|night|| +6411|AAAAAAAAMAJBAAAA|6411|1|46|51|AM|third|night|| +6412|AAAAAAAANAJBAAAA|6412|1|46|52|AM|third|night|| +6413|AAAAAAAAOAJBAAAA|6413|1|46|53|AM|third|night|| +6414|AAAAAAAAPAJBAAAA|6414|1|46|54|AM|third|night|| +6415|AAAAAAAAABJBAAAA|6415|1|46|55|AM|third|night|| +6416|AAAAAAAABBJBAAAA|6416|1|46|56|AM|third|night|| +6417|AAAAAAAACBJBAAAA|6417|1|46|57|AM|third|night|| +6418|AAAAAAAADBJBAAAA|6418|1|46|58|AM|third|night|| +6419|AAAAAAAAEBJBAAAA|6419|1|46|59|AM|third|night|| +6420|AAAAAAAAFBJBAAAA|6420|1|47|0|AM|third|night|| +6421|AAAAAAAAGBJBAAAA|6421|1|47|1|AM|third|night|| +6422|AAAAAAAAHBJBAAAA|6422|1|47|2|AM|third|night|| +6423|AAAAAAAAIBJBAAAA|6423|1|47|3|AM|third|night|| +6424|AAAAAAAAJBJBAAAA|6424|1|47|4|AM|third|night|| +6425|AAAAAAAAKBJBAAAA|6425|1|47|5|AM|third|night|| +6426|AAAAAAAALBJBAAAA|6426|1|47|6|AM|third|night|| +6427|AAAAAAAAMBJBAAAA|6427|1|47|7|AM|third|night|| +6428|AAAAAAAANBJBAAAA|6428|1|47|8|AM|third|night|| +6429|AAAAAAAAOBJBAAAA|6429|1|47|9|AM|third|night|| +6430|AAAAAAAAPBJBAAAA|6430|1|47|10|AM|third|night|| +6431|AAAAAAAAACJBAAAA|6431|1|47|11|AM|third|night|| +6432|AAAAAAAABCJBAAAA|6432|1|47|12|AM|third|night|| +6433|AAAAAAAACCJBAAAA|6433|1|47|13|AM|third|night|| +6434|AAAAAAAADCJBAAAA|6434|1|47|14|AM|third|night|| +6435|AAAAAAAAECJBAAAA|6435|1|47|15|AM|third|night|| +6436|AAAAAAAAFCJBAAAA|6436|1|47|16|AM|third|night|| +6437|AAAAAAAAGCJBAAAA|6437|1|47|17|AM|third|night|| +6438|AAAAAAAAHCJBAAAA|6438|1|47|18|AM|third|night|| +6439|AAAAAAAAICJBAAAA|6439|1|47|19|AM|third|night|| +6440|AAAAAAAAJCJBAAAA|6440|1|47|20|AM|third|night|| +6441|AAAAAAAAKCJBAAAA|6441|1|47|21|AM|third|night|| +6442|AAAAAAAALCJBAAAA|6442|1|47|22|AM|third|night|| +6443|AAAAAAAAMCJBAAAA|6443|1|47|23|AM|third|night|| +6444|AAAAAAAANCJBAAAA|6444|1|47|24|AM|third|night|| +6445|AAAAAAAAOCJBAAAA|6445|1|47|25|AM|third|night|| +6446|AAAAAAAAPCJBAAAA|6446|1|47|26|AM|third|night|| +6447|AAAAAAAAADJBAAAA|6447|1|47|27|AM|third|night|| +6448|AAAAAAAABDJBAAAA|6448|1|47|28|AM|third|night|| +6449|AAAAAAAACDJBAAAA|6449|1|47|29|AM|third|night|| +6450|AAAAAAAADDJBAAAA|6450|1|47|30|AM|third|night|| +6451|AAAAAAAAEDJBAAAA|6451|1|47|31|AM|third|night|| +6452|AAAAAAAAFDJBAAAA|6452|1|47|32|AM|third|night|| +6453|AAAAAAAAGDJBAAAA|6453|1|47|33|AM|third|night|| +6454|AAAAAAAAHDJBAAAA|6454|1|47|34|AM|third|night|| +6455|AAAAAAAAIDJBAAAA|6455|1|47|35|AM|third|night|| +6456|AAAAAAAAJDJBAAAA|6456|1|47|36|AM|third|night|| +6457|AAAAAAAAKDJBAAAA|6457|1|47|37|AM|third|night|| +6458|AAAAAAAALDJBAAAA|6458|1|47|38|AM|third|night|| +6459|AAAAAAAAMDJBAAAA|6459|1|47|39|AM|third|night|| +6460|AAAAAAAANDJBAAAA|6460|1|47|40|AM|third|night|| +6461|AAAAAAAAODJBAAAA|6461|1|47|41|AM|third|night|| +6462|AAAAAAAAPDJBAAAA|6462|1|47|42|AM|third|night|| +6463|AAAAAAAAAEJBAAAA|6463|1|47|43|AM|third|night|| +6464|AAAAAAAABEJBAAAA|6464|1|47|44|AM|third|night|| +6465|AAAAAAAACEJBAAAA|6465|1|47|45|AM|third|night|| +6466|AAAAAAAADEJBAAAA|6466|1|47|46|AM|third|night|| +6467|AAAAAAAAEEJBAAAA|6467|1|47|47|AM|third|night|| +6468|AAAAAAAAFEJBAAAA|6468|1|47|48|AM|third|night|| +6469|AAAAAAAAGEJBAAAA|6469|1|47|49|AM|third|night|| +6470|AAAAAAAAHEJBAAAA|6470|1|47|50|AM|third|night|| +6471|AAAAAAAAIEJBAAAA|6471|1|47|51|AM|third|night|| +6472|AAAAAAAAJEJBAAAA|6472|1|47|52|AM|third|night|| +6473|AAAAAAAAKEJBAAAA|6473|1|47|53|AM|third|night|| +6474|AAAAAAAALEJBAAAA|6474|1|47|54|AM|third|night|| +6475|AAAAAAAAMEJBAAAA|6475|1|47|55|AM|third|night|| +6476|AAAAAAAANEJBAAAA|6476|1|47|56|AM|third|night|| +6477|AAAAAAAAOEJBAAAA|6477|1|47|57|AM|third|night|| +6478|AAAAAAAAPEJBAAAA|6478|1|47|58|AM|third|night|| +6479|AAAAAAAAAFJBAAAA|6479|1|47|59|AM|third|night|| +6480|AAAAAAAABFJBAAAA|6480|1|48|0|AM|third|night|| +6481|AAAAAAAACFJBAAAA|6481|1|48|1|AM|third|night|| +6482|AAAAAAAADFJBAAAA|6482|1|48|2|AM|third|night|| +6483|AAAAAAAAEFJBAAAA|6483|1|48|3|AM|third|night|| +6484|AAAAAAAAFFJBAAAA|6484|1|48|4|AM|third|night|| +6485|AAAAAAAAGFJBAAAA|6485|1|48|5|AM|third|night|| +6486|AAAAAAAAHFJBAAAA|6486|1|48|6|AM|third|night|| +6487|AAAAAAAAIFJBAAAA|6487|1|48|7|AM|third|night|| +6488|AAAAAAAAJFJBAAAA|6488|1|48|8|AM|third|night|| +6489|AAAAAAAAKFJBAAAA|6489|1|48|9|AM|third|night|| +6490|AAAAAAAALFJBAAAA|6490|1|48|10|AM|third|night|| +6491|AAAAAAAAMFJBAAAA|6491|1|48|11|AM|third|night|| +6492|AAAAAAAANFJBAAAA|6492|1|48|12|AM|third|night|| +6493|AAAAAAAAOFJBAAAA|6493|1|48|13|AM|third|night|| +6494|AAAAAAAAPFJBAAAA|6494|1|48|14|AM|third|night|| +6495|AAAAAAAAAGJBAAAA|6495|1|48|15|AM|third|night|| +6496|AAAAAAAABGJBAAAA|6496|1|48|16|AM|third|night|| +6497|AAAAAAAACGJBAAAA|6497|1|48|17|AM|third|night|| +6498|AAAAAAAADGJBAAAA|6498|1|48|18|AM|third|night|| +6499|AAAAAAAAEGJBAAAA|6499|1|48|19|AM|third|night|| +6500|AAAAAAAAFGJBAAAA|6500|1|48|20|AM|third|night|| +6501|AAAAAAAAGGJBAAAA|6501|1|48|21|AM|third|night|| +6502|AAAAAAAAHGJBAAAA|6502|1|48|22|AM|third|night|| +6503|AAAAAAAAIGJBAAAA|6503|1|48|23|AM|third|night|| +6504|AAAAAAAAJGJBAAAA|6504|1|48|24|AM|third|night|| +6505|AAAAAAAAKGJBAAAA|6505|1|48|25|AM|third|night|| +6506|AAAAAAAALGJBAAAA|6506|1|48|26|AM|third|night|| +6507|AAAAAAAAMGJBAAAA|6507|1|48|27|AM|third|night|| +6508|AAAAAAAANGJBAAAA|6508|1|48|28|AM|third|night|| +6509|AAAAAAAAOGJBAAAA|6509|1|48|29|AM|third|night|| +6510|AAAAAAAAPGJBAAAA|6510|1|48|30|AM|third|night|| +6511|AAAAAAAAAHJBAAAA|6511|1|48|31|AM|third|night|| +6512|AAAAAAAABHJBAAAA|6512|1|48|32|AM|third|night|| +6513|AAAAAAAACHJBAAAA|6513|1|48|33|AM|third|night|| +6514|AAAAAAAADHJBAAAA|6514|1|48|34|AM|third|night|| +6515|AAAAAAAAEHJBAAAA|6515|1|48|35|AM|third|night|| +6516|AAAAAAAAFHJBAAAA|6516|1|48|36|AM|third|night|| +6517|AAAAAAAAGHJBAAAA|6517|1|48|37|AM|third|night|| +6518|AAAAAAAAHHJBAAAA|6518|1|48|38|AM|third|night|| +6519|AAAAAAAAIHJBAAAA|6519|1|48|39|AM|third|night|| +6520|AAAAAAAAJHJBAAAA|6520|1|48|40|AM|third|night|| +6521|AAAAAAAAKHJBAAAA|6521|1|48|41|AM|third|night|| +6522|AAAAAAAALHJBAAAA|6522|1|48|42|AM|third|night|| +6523|AAAAAAAAMHJBAAAA|6523|1|48|43|AM|third|night|| +6524|AAAAAAAANHJBAAAA|6524|1|48|44|AM|third|night|| +6525|AAAAAAAAOHJBAAAA|6525|1|48|45|AM|third|night|| +6526|AAAAAAAAPHJBAAAA|6526|1|48|46|AM|third|night|| +6527|AAAAAAAAAIJBAAAA|6527|1|48|47|AM|third|night|| +6528|AAAAAAAABIJBAAAA|6528|1|48|48|AM|third|night|| +6529|AAAAAAAACIJBAAAA|6529|1|48|49|AM|third|night|| +6530|AAAAAAAADIJBAAAA|6530|1|48|50|AM|third|night|| +6531|AAAAAAAAEIJBAAAA|6531|1|48|51|AM|third|night|| +6532|AAAAAAAAFIJBAAAA|6532|1|48|52|AM|third|night|| +6533|AAAAAAAAGIJBAAAA|6533|1|48|53|AM|third|night|| +6534|AAAAAAAAHIJBAAAA|6534|1|48|54|AM|third|night|| +6535|AAAAAAAAIIJBAAAA|6535|1|48|55|AM|third|night|| +6536|AAAAAAAAJIJBAAAA|6536|1|48|56|AM|third|night|| +6537|AAAAAAAAKIJBAAAA|6537|1|48|57|AM|third|night|| +6538|AAAAAAAALIJBAAAA|6538|1|48|58|AM|third|night|| +6539|AAAAAAAAMIJBAAAA|6539|1|48|59|AM|third|night|| +6540|AAAAAAAANIJBAAAA|6540|1|49|0|AM|third|night|| +6541|AAAAAAAAOIJBAAAA|6541|1|49|1|AM|third|night|| +6542|AAAAAAAAPIJBAAAA|6542|1|49|2|AM|third|night|| +6543|AAAAAAAAAJJBAAAA|6543|1|49|3|AM|third|night|| +6544|AAAAAAAABJJBAAAA|6544|1|49|4|AM|third|night|| +6545|AAAAAAAACJJBAAAA|6545|1|49|5|AM|third|night|| +6546|AAAAAAAADJJBAAAA|6546|1|49|6|AM|third|night|| +6547|AAAAAAAAEJJBAAAA|6547|1|49|7|AM|third|night|| +6548|AAAAAAAAFJJBAAAA|6548|1|49|8|AM|third|night|| +6549|AAAAAAAAGJJBAAAA|6549|1|49|9|AM|third|night|| +6550|AAAAAAAAHJJBAAAA|6550|1|49|10|AM|third|night|| +6551|AAAAAAAAIJJBAAAA|6551|1|49|11|AM|third|night|| +6552|AAAAAAAAJJJBAAAA|6552|1|49|12|AM|third|night|| +6553|AAAAAAAAKJJBAAAA|6553|1|49|13|AM|third|night|| +6554|AAAAAAAALJJBAAAA|6554|1|49|14|AM|third|night|| +6555|AAAAAAAAMJJBAAAA|6555|1|49|15|AM|third|night|| +6556|AAAAAAAANJJBAAAA|6556|1|49|16|AM|third|night|| +6557|AAAAAAAAOJJBAAAA|6557|1|49|17|AM|third|night|| +6558|AAAAAAAAPJJBAAAA|6558|1|49|18|AM|third|night|| +6559|AAAAAAAAAKJBAAAA|6559|1|49|19|AM|third|night|| +6560|AAAAAAAABKJBAAAA|6560|1|49|20|AM|third|night|| +6561|AAAAAAAACKJBAAAA|6561|1|49|21|AM|third|night|| +6562|AAAAAAAADKJBAAAA|6562|1|49|22|AM|third|night|| +6563|AAAAAAAAEKJBAAAA|6563|1|49|23|AM|third|night|| +6564|AAAAAAAAFKJBAAAA|6564|1|49|24|AM|third|night|| +6565|AAAAAAAAGKJBAAAA|6565|1|49|25|AM|third|night|| +6566|AAAAAAAAHKJBAAAA|6566|1|49|26|AM|third|night|| +6567|AAAAAAAAIKJBAAAA|6567|1|49|27|AM|third|night|| +6568|AAAAAAAAJKJBAAAA|6568|1|49|28|AM|third|night|| +6569|AAAAAAAAKKJBAAAA|6569|1|49|29|AM|third|night|| +6570|AAAAAAAALKJBAAAA|6570|1|49|30|AM|third|night|| +6571|AAAAAAAAMKJBAAAA|6571|1|49|31|AM|third|night|| +6572|AAAAAAAANKJBAAAA|6572|1|49|32|AM|third|night|| +6573|AAAAAAAAOKJBAAAA|6573|1|49|33|AM|third|night|| +6574|AAAAAAAAPKJBAAAA|6574|1|49|34|AM|third|night|| +6575|AAAAAAAAALJBAAAA|6575|1|49|35|AM|third|night|| +6576|AAAAAAAABLJBAAAA|6576|1|49|36|AM|third|night|| +6577|AAAAAAAACLJBAAAA|6577|1|49|37|AM|third|night|| +6578|AAAAAAAADLJBAAAA|6578|1|49|38|AM|third|night|| +6579|AAAAAAAAELJBAAAA|6579|1|49|39|AM|third|night|| +6580|AAAAAAAAFLJBAAAA|6580|1|49|40|AM|third|night|| +6581|AAAAAAAAGLJBAAAA|6581|1|49|41|AM|third|night|| +6582|AAAAAAAAHLJBAAAA|6582|1|49|42|AM|third|night|| +6583|AAAAAAAAILJBAAAA|6583|1|49|43|AM|third|night|| +6584|AAAAAAAAJLJBAAAA|6584|1|49|44|AM|third|night|| +6585|AAAAAAAAKLJBAAAA|6585|1|49|45|AM|third|night|| +6586|AAAAAAAALLJBAAAA|6586|1|49|46|AM|third|night|| +6587|AAAAAAAAMLJBAAAA|6587|1|49|47|AM|third|night|| +6588|AAAAAAAANLJBAAAA|6588|1|49|48|AM|third|night|| +6589|AAAAAAAAOLJBAAAA|6589|1|49|49|AM|third|night|| +6590|AAAAAAAAPLJBAAAA|6590|1|49|50|AM|third|night|| +6591|AAAAAAAAAMJBAAAA|6591|1|49|51|AM|third|night|| +6592|AAAAAAAABMJBAAAA|6592|1|49|52|AM|third|night|| +6593|AAAAAAAACMJBAAAA|6593|1|49|53|AM|third|night|| +6594|AAAAAAAADMJBAAAA|6594|1|49|54|AM|third|night|| +6595|AAAAAAAAEMJBAAAA|6595|1|49|55|AM|third|night|| +6596|AAAAAAAAFMJBAAAA|6596|1|49|56|AM|third|night|| +6597|AAAAAAAAGMJBAAAA|6597|1|49|57|AM|third|night|| +6598|AAAAAAAAHMJBAAAA|6598|1|49|58|AM|third|night|| +6599|AAAAAAAAIMJBAAAA|6599|1|49|59|AM|third|night|| +6600|AAAAAAAAJMJBAAAA|6600|1|50|0|AM|third|night|| +6601|AAAAAAAAKMJBAAAA|6601|1|50|1|AM|third|night|| +6602|AAAAAAAALMJBAAAA|6602|1|50|2|AM|third|night|| +6603|AAAAAAAAMMJBAAAA|6603|1|50|3|AM|third|night|| +6604|AAAAAAAANMJBAAAA|6604|1|50|4|AM|third|night|| +6605|AAAAAAAAOMJBAAAA|6605|1|50|5|AM|third|night|| +6606|AAAAAAAAPMJBAAAA|6606|1|50|6|AM|third|night|| +6607|AAAAAAAAANJBAAAA|6607|1|50|7|AM|third|night|| +6608|AAAAAAAABNJBAAAA|6608|1|50|8|AM|third|night|| +6609|AAAAAAAACNJBAAAA|6609|1|50|9|AM|third|night|| +6610|AAAAAAAADNJBAAAA|6610|1|50|10|AM|third|night|| +6611|AAAAAAAAENJBAAAA|6611|1|50|11|AM|third|night|| +6612|AAAAAAAAFNJBAAAA|6612|1|50|12|AM|third|night|| +6613|AAAAAAAAGNJBAAAA|6613|1|50|13|AM|third|night|| +6614|AAAAAAAAHNJBAAAA|6614|1|50|14|AM|third|night|| +6615|AAAAAAAAINJBAAAA|6615|1|50|15|AM|third|night|| +6616|AAAAAAAAJNJBAAAA|6616|1|50|16|AM|third|night|| +6617|AAAAAAAAKNJBAAAA|6617|1|50|17|AM|third|night|| +6618|AAAAAAAALNJBAAAA|6618|1|50|18|AM|third|night|| +6619|AAAAAAAAMNJBAAAA|6619|1|50|19|AM|third|night|| +6620|AAAAAAAANNJBAAAA|6620|1|50|20|AM|third|night|| +6621|AAAAAAAAONJBAAAA|6621|1|50|21|AM|third|night|| +6622|AAAAAAAAPNJBAAAA|6622|1|50|22|AM|third|night|| +6623|AAAAAAAAAOJBAAAA|6623|1|50|23|AM|third|night|| +6624|AAAAAAAABOJBAAAA|6624|1|50|24|AM|third|night|| +6625|AAAAAAAACOJBAAAA|6625|1|50|25|AM|third|night|| +6626|AAAAAAAADOJBAAAA|6626|1|50|26|AM|third|night|| +6627|AAAAAAAAEOJBAAAA|6627|1|50|27|AM|third|night|| +6628|AAAAAAAAFOJBAAAA|6628|1|50|28|AM|third|night|| +6629|AAAAAAAAGOJBAAAA|6629|1|50|29|AM|third|night|| +6630|AAAAAAAAHOJBAAAA|6630|1|50|30|AM|third|night|| +6631|AAAAAAAAIOJBAAAA|6631|1|50|31|AM|third|night|| +6632|AAAAAAAAJOJBAAAA|6632|1|50|32|AM|third|night|| +6633|AAAAAAAAKOJBAAAA|6633|1|50|33|AM|third|night|| +6634|AAAAAAAALOJBAAAA|6634|1|50|34|AM|third|night|| +6635|AAAAAAAAMOJBAAAA|6635|1|50|35|AM|third|night|| +6636|AAAAAAAANOJBAAAA|6636|1|50|36|AM|third|night|| +6637|AAAAAAAAOOJBAAAA|6637|1|50|37|AM|third|night|| +6638|AAAAAAAAPOJBAAAA|6638|1|50|38|AM|third|night|| +6639|AAAAAAAAAPJBAAAA|6639|1|50|39|AM|third|night|| +6640|AAAAAAAABPJBAAAA|6640|1|50|40|AM|third|night|| +6641|AAAAAAAACPJBAAAA|6641|1|50|41|AM|third|night|| +6642|AAAAAAAADPJBAAAA|6642|1|50|42|AM|third|night|| +6643|AAAAAAAAEPJBAAAA|6643|1|50|43|AM|third|night|| +6644|AAAAAAAAFPJBAAAA|6644|1|50|44|AM|third|night|| +6645|AAAAAAAAGPJBAAAA|6645|1|50|45|AM|third|night|| +6646|AAAAAAAAHPJBAAAA|6646|1|50|46|AM|third|night|| +6647|AAAAAAAAIPJBAAAA|6647|1|50|47|AM|third|night|| +6648|AAAAAAAAJPJBAAAA|6648|1|50|48|AM|third|night|| +6649|AAAAAAAAKPJBAAAA|6649|1|50|49|AM|third|night|| +6650|AAAAAAAALPJBAAAA|6650|1|50|50|AM|third|night|| +6651|AAAAAAAAMPJBAAAA|6651|1|50|51|AM|third|night|| +6652|AAAAAAAANPJBAAAA|6652|1|50|52|AM|third|night|| +6653|AAAAAAAAOPJBAAAA|6653|1|50|53|AM|third|night|| +6654|AAAAAAAAPPJBAAAA|6654|1|50|54|AM|third|night|| +6655|AAAAAAAAAAKBAAAA|6655|1|50|55|AM|third|night|| +6656|AAAAAAAABAKBAAAA|6656|1|50|56|AM|third|night|| +6657|AAAAAAAACAKBAAAA|6657|1|50|57|AM|third|night|| +6658|AAAAAAAADAKBAAAA|6658|1|50|58|AM|third|night|| +6659|AAAAAAAAEAKBAAAA|6659|1|50|59|AM|third|night|| +6660|AAAAAAAAFAKBAAAA|6660|1|51|0|AM|third|night|| +6661|AAAAAAAAGAKBAAAA|6661|1|51|1|AM|third|night|| +6662|AAAAAAAAHAKBAAAA|6662|1|51|2|AM|third|night|| +6663|AAAAAAAAIAKBAAAA|6663|1|51|3|AM|third|night|| +6664|AAAAAAAAJAKBAAAA|6664|1|51|4|AM|third|night|| +6665|AAAAAAAAKAKBAAAA|6665|1|51|5|AM|third|night|| +6666|AAAAAAAALAKBAAAA|6666|1|51|6|AM|third|night|| +6667|AAAAAAAAMAKBAAAA|6667|1|51|7|AM|third|night|| +6668|AAAAAAAANAKBAAAA|6668|1|51|8|AM|third|night|| +6669|AAAAAAAAOAKBAAAA|6669|1|51|9|AM|third|night|| +6670|AAAAAAAAPAKBAAAA|6670|1|51|10|AM|third|night|| +6671|AAAAAAAAABKBAAAA|6671|1|51|11|AM|third|night|| +6672|AAAAAAAABBKBAAAA|6672|1|51|12|AM|third|night|| +6673|AAAAAAAACBKBAAAA|6673|1|51|13|AM|third|night|| +6674|AAAAAAAADBKBAAAA|6674|1|51|14|AM|third|night|| +6675|AAAAAAAAEBKBAAAA|6675|1|51|15|AM|third|night|| +6676|AAAAAAAAFBKBAAAA|6676|1|51|16|AM|third|night|| +6677|AAAAAAAAGBKBAAAA|6677|1|51|17|AM|third|night|| +6678|AAAAAAAAHBKBAAAA|6678|1|51|18|AM|third|night|| +6679|AAAAAAAAIBKBAAAA|6679|1|51|19|AM|third|night|| +6680|AAAAAAAAJBKBAAAA|6680|1|51|20|AM|third|night|| +6681|AAAAAAAAKBKBAAAA|6681|1|51|21|AM|third|night|| +6682|AAAAAAAALBKBAAAA|6682|1|51|22|AM|third|night|| +6683|AAAAAAAAMBKBAAAA|6683|1|51|23|AM|third|night|| +6684|AAAAAAAANBKBAAAA|6684|1|51|24|AM|third|night|| +6685|AAAAAAAAOBKBAAAA|6685|1|51|25|AM|third|night|| +6686|AAAAAAAAPBKBAAAA|6686|1|51|26|AM|third|night|| +6687|AAAAAAAAACKBAAAA|6687|1|51|27|AM|third|night|| +6688|AAAAAAAABCKBAAAA|6688|1|51|28|AM|third|night|| +6689|AAAAAAAACCKBAAAA|6689|1|51|29|AM|third|night|| +6690|AAAAAAAADCKBAAAA|6690|1|51|30|AM|third|night|| +6691|AAAAAAAAECKBAAAA|6691|1|51|31|AM|third|night|| +6692|AAAAAAAAFCKBAAAA|6692|1|51|32|AM|third|night|| +6693|AAAAAAAAGCKBAAAA|6693|1|51|33|AM|third|night|| +6694|AAAAAAAAHCKBAAAA|6694|1|51|34|AM|third|night|| +6695|AAAAAAAAICKBAAAA|6695|1|51|35|AM|third|night|| +6696|AAAAAAAAJCKBAAAA|6696|1|51|36|AM|third|night|| +6697|AAAAAAAAKCKBAAAA|6697|1|51|37|AM|third|night|| +6698|AAAAAAAALCKBAAAA|6698|1|51|38|AM|third|night|| +6699|AAAAAAAAMCKBAAAA|6699|1|51|39|AM|third|night|| +6700|AAAAAAAANCKBAAAA|6700|1|51|40|AM|third|night|| +6701|AAAAAAAAOCKBAAAA|6701|1|51|41|AM|third|night|| +6702|AAAAAAAAPCKBAAAA|6702|1|51|42|AM|third|night|| +6703|AAAAAAAAADKBAAAA|6703|1|51|43|AM|third|night|| +6704|AAAAAAAABDKBAAAA|6704|1|51|44|AM|third|night|| +6705|AAAAAAAACDKBAAAA|6705|1|51|45|AM|third|night|| +6706|AAAAAAAADDKBAAAA|6706|1|51|46|AM|third|night|| +6707|AAAAAAAAEDKBAAAA|6707|1|51|47|AM|third|night|| +6708|AAAAAAAAFDKBAAAA|6708|1|51|48|AM|third|night|| +6709|AAAAAAAAGDKBAAAA|6709|1|51|49|AM|third|night|| +6710|AAAAAAAAHDKBAAAA|6710|1|51|50|AM|third|night|| +6711|AAAAAAAAIDKBAAAA|6711|1|51|51|AM|third|night|| +6712|AAAAAAAAJDKBAAAA|6712|1|51|52|AM|third|night|| +6713|AAAAAAAAKDKBAAAA|6713|1|51|53|AM|third|night|| +6714|AAAAAAAALDKBAAAA|6714|1|51|54|AM|third|night|| +6715|AAAAAAAAMDKBAAAA|6715|1|51|55|AM|third|night|| +6716|AAAAAAAANDKBAAAA|6716|1|51|56|AM|third|night|| +6717|AAAAAAAAODKBAAAA|6717|1|51|57|AM|third|night|| +6718|AAAAAAAAPDKBAAAA|6718|1|51|58|AM|third|night|| +6719|AAAAAAAAAEKBAAAA|6719|1|51|59|AM|third|night|| +6720|AAAAAAAABEKBAAAA|6720|1|52|0|AM|third|night|| +6721|AAAAAAAACEKBAAAA|6721|1|52|1|AM|third|night|| +6722|AAAAAAAADEKBAAAA|6722|1|52|2|AM|third|night|| +6723|AAAAAAAAEEKBAAAA|6723|1|52|3|AM|third|night|| +6724|AAAAAAAAFEKBAAAA|6724|1|52|4|AM|third|night|| +6725|AAAAAAAAGEKBAAAA|6725|1|52|5|AM|third|night|| +6726|AAAAAAAAHEKBAAAA|6726|1|52|6|AM|third|night|| +6727|AAAAAAAAIEKBAAAA|6727|1|52|7|AM|third|night|| +6728|AAAAAAAAJEKBAAAA|6728|1|52|8|AM|third|night|| +6729|AAAAAAAAKEKBAAAA|6729|1|52|9|AM|third|night|| +6730|AAAAAAAALEKBAAAA|6730|1|52|10|AM|third|night|| +6731|AAAAAAAAMEKBAAAA|6731|1|52|11|AM|third|night|| +6732|AAAAAAAANEKBAAAA|6732|1|52|12|AM|third|night|| +6733|AAAAAAAAOEKBAAAA|6733|1|52|13|AM|third|night|| +6734|AAAAAAAAPEKBAAAA|6734|1|52|14|AM|third|night|| +6735|AAAAAAAAAFKBAAAA|6735|1|52|15|AM|third|night|| +6736|AAAAAAAABFKBAAAA|6736|1|52|16|AM|third|night|| +6737|AAAAAAAACFKBAAAA|6737|1|52|17|AM|third|night|| +6738|AAAAAAAADFKBAAAA|6738|1|52|18|AM|third|night|| +6739|AAAAAAAAEFKBAAAA|6739|1|52|19|AM|third|night|| +6740|AAAAAAAAFFKBAAAA|6740|1|52|20|AM|third|night|| +6741|AAAAAAAAGFKBAAAA|6741|1|52|21|AM|third|night|| +6742|AAAAAAAAHFKBAAAA|6742|1|52|22|AM|third|night|| +6743|AAAAAAAAIFKBAAAA|6743|1|52|23|AM|third|night|| +6744|AAAAAAAAJFKBAAAA|6744|1|52|24|AM|third|night|| +6745|AAAAAAAAKFKBAAAA|6745|1|52|25|AM|third|night|| +6746|AAAAAAAALFKBAAAA|6746|1|52|26|AM|third|night|| +6747|AAAAAAAAMFKBAAAA|6747|1|52|27|AM|third|night|| +6748|AAAAAAAANFKBAAAA|6748|1|52|28|AM|third|night|| +6749|AAAAAAAAOFKBAAAA|6749|1|52|29|AM|third|night|| +6750|AAAAAAAAPFKBAAAA|6750|1|52|30|AM|third|night|| +6751|AAAAAAAAAGKBAAAA|6751|1|52|31|AM|third|night|| +6752|AAAAAAAABGKBAAAA|6752|1|52|32|AM|third|night|| +6753|AAAAAAAACGKBAAAA|6753|1|52|33|AM|third|night|| +6754|AAAAAAAADGKBAAAA|6754|1|52|34|AM|third|night|| +6755|AAAAAAAAEGKBAAAA|6755|1|52|35|AM|third|night|| +6756|AAAAAAAAFGKBAAAA|6756|1|52|36|AM|third|night|| +6757|AAAAAAAAGGKBAAAA|6757|1|52|37|AM|third|night|| +6758|AAAAAAAAHGKBAAAA|6758|1|52|38|AM|third|night|| +6759|AAAAAAAAIGKBAAAA|6759|1|52|39|AM|third|night|| +6760|AAAAAAAAJGKBAAAA|6760|1|52|40|AM|third|night|| +6761|AAAAAAAAKGKBAAAA|6761|1|52|41|AM|third|night|| +6762|AAAAAAAALGKBAAAA|6762|1|52|42|AM|third|night|| +6763|AAAAAAAAMGKBAAAA|6763|1|52|43|AM|third|night|| +6764|AAAAAAAANGKBAAAA|6764|1|52|44|AM|third|night|| +6765|AAAAAAAAOGKBAAAA|6765|1|52|45|AM|third|night|| +6766|AAAAAAAAPGKBAAAA|6766|1|52|46|AM|third|night|| +6767|AAAAAAAAAHKBAAAA|6767|1|52|47|AM|third|night|| +6768|AAAAAAAABHKBAAAA|6768|1|52|48|AM|third|night|| +6769|AAAAAAAACHKBAAAA|6769|1|52|49|AM|third|night|| +6770|AAAAAAAADHKBAAAA|6770|1|52|50|AM|third|night|| +6771|AAAAAAAAEHKBAAAA|6771|1|52|51|AM|third|night|| +6772|AAAAAAAAFHKBAAAA|6772|1|52|52|AM|third|night|| +6773|AAAAAAAAGHKBAAAA|6773|1|52|53|AM|third|night|| +6774|AAAAAAAAHHKBAAAA|6774|1|52|54|AM|third|night|| +6775|AAAAAAAAIHKBAAAA|6775|1|52|55|AM|third|night|| +6776|AAAAAAAAJHKBAAAA|6776|1|52|56|AM|third|night|| +6777|AAAAAAAAKHKBAAAA|6777|1|52|57|AM|third|night|| +6778|AAAAAAAALHKBAAAA|6778|1|52|58|AM|third|night|| +6779|AAAAAAAAMHKBAAAA|6779|1|52|59|AM|third|night|| +6780|AAAAAAAANHKBAAAA|6780|1|53|0|AM|third|night|| +6781|AAAAAAAAOHKBAAAA|6781|1|53|1|AM|third|night|| +6782|AAAAAAAAPHKBAAAA|6782|1|53|2|AM|third|night|| +6783|AAAAAAAAAIKBAAAA|6783|1|53|3|AM|third|night|| +6784|AAAAAAAABIKBAAAA|6784|1|53|4|AM|third|night|| +6785|AAAAAAAACIKBAAAA|6785|1|53|5|AM|third|night|| +6786|AAAAAAAADIKBAAAA|6786|1|53|6|AM|third|night|| +6787|AAAAAAAAEIKBAAAA|6787|1|53|7|AM|third|night|| +6788|AAAAAAAAFIKBAAAA|6788|1|53|8|AM|third|night|| +6789|AAAAAAAAGIKBAAAA|6789|1|53|9|AM|third|night|| +6790|AAAAAAAAHIKBAAAA|6790|1|53|10|AM|third|night|| +6791|AAAAAAAAIIKBAAAA|6791|1|53|11|AM|third|night|| +6792|AAAAAAAAJIKBAAAA|6792|1|53|12|AM|third|night|| +6793|AAAAAAAAKIKBAAAA|6793|1|53|13|AM|third|night|| +6794|AAAAAAAALIKBAAAA|6794|1|53|14|AM|third|night|| +6795|AAAAAAAAMIKBAAAA|6795|1|53|15|AM|third|night|| +6796|AAAAAAAANIKBAAAA|6796|1|53|16|AM|third|night|| +6797|AAAAAAAAOIKBAAAA|6797|1|53|17|AM|third|night|| +6798|AAAAAAAAPIKBAAAA|6798|1|53|18|AM|third|night|| +6799|AAAAAAAAAJKBAAAA|6799|1|53|19|AM|third|night|| +6800|AAAAAAAABJKBAAAA|6800|1|53|20|AM|third|night|| +6801|AAAAAAAACJKBAAAA|6801|1|53|21|AM|third|night|| +6802|AAAAAAAADJKBAAAA|6802|1|53|22|AM|third|night|| +6803|AAAAAAAAEJKBAAAA|6803|1|53|23|AM|third|night|| +6804|AAAAAAAAFJKBAAAA|6804|1|53|24|AM|third|night|| +6805|AAAAAAAAGJKBAAAA|6805|1|53|25|AM|third|night|| +6806|AAAAAAAAHJKBAAAA|6806|1|53|26|AM|third|night|| +6807|AAAAAAAAIJKBAAAA|6807|1|53|27|AM|third|night|| +6808|AAAAAAAAJJKBAAAA|6808|1|53|28|AM|third|night|| +6809|AAAAAAAAKJKBAAAA|6809|1|53|29|AM|third|night|| +6810|AAAAAAAALJKBAAAA|6810|1|53|30|AM|third|night|| +6811|AAAAAAAAMJKBAAAA|6811|1|53|31|AM|third|night|| +6812|AAAAAAAANJKBAAAA|6812|1|53|32|AM|third|night|| +6813|AAAAAAAAOJKBAAAA|6813|1|53|33|AM|third|night|| +6814|AAAAAAAAPJKBAAAA|6814|1|53|34|AM|third|night|| +6815|AAAAAAAAAKKBAAAA|6815|1|53|35|AM|third|night|| +6816|AAAAAAAABKKBAAAA|6816|1|53|36|AM|third|night|| +6817|AAAAAAAACKKBAAAA|6817|1|53|37|AM|third|night|| +6818|AAAAAAAADKKBAAAA|6818|1|53|38|AM|third|night|| +6819|AAAAAAAAEKKBAAAA|6819|1|53|39|AM|third|night|| +6820|AAAAAAAAFKKBAAAA|6820|1|53|40|AM|third|night|| +6821|AAAAAAAAGKKBAAAA|6821|1|53|41|AM|third|night|| +6822|AAAAAAAAHKKBAAAA|6822|1|53|42|AM|third|night|| +6823|AAAAAAAAIKKBAAAA|6823|1|53|43|AM|third|night|| +6824|AAAAAAAAJKKBAAAA|6824|1|53|44|AM|third|night|| +6825|AAAAAAAAKKKBAAAA|6825|1|53|45|AM|third|night|| +6826|AAAAAAAALKKBAAAA|6826|1|53|46|AM|third|night|| +6827|AAAAAAAAMKKBAAAA|6827|1|53|47|AM|third|night|| +6828|AAAAAAAANKKBAAAA|6828|1|53|48|AM|third|night|| +6829|AAAAAAAAOKKBAAAA|6829|1|53|49|AM|third|night|| +6830|AAAAAAAAPKKBAAAA|6830|1|53|50|AM|third|night|| +6831|AAAAAAAAALKBAAAA|6831|1|53|51|AM|third|night|| +6832|AAAAAAAABLKBAAAA|6832|1|53|52|AM|third|night|| +6833|AAAAAAAACLKBAAAA|6833|1|53|53|AM|third|night|| +6834|AAAAAAAADLKBAAAA|6834|1|53|54|AM|third|night|| +6835|AAAAAAAAELKBAAAA|6835|1|53|55|AM|third|night|| +6836|AAAAAAAAFLKBAAAA|6836|1|53|56|AM|third|night|| +6837|AAAAAAAAGLKBAAAA|6837|1|53|57|AM|third|night|| +6838|AAAAAAAAHLKBAAAA|6838|1|53|58|AM|third|night|| +6839|AAAAAAAAILKBAAAA|6839|1|53|59|AM|third|night|| +6840|AAAAAAAAJLKBAAAA|6840|1|54|0|AM|third|night|| +6841|AAAAAAAAKLKBAAAA|6841|1|54|1|AM|third|night|| +6842|AAAAAAAALLKBAAAA|6842|1|54|2|AM|third|night|| +6843|AAAAAAAAMLKBAAAA|6843|1|54|3|AM|third|night|| +6844|AAAAAAAANLKBAAAA|6844|1|54|4|AM|third|night|| +6845|AAAAAAAAOLKBAAAA|6845|1|54|5|AM|third|night|| +6846|AAAAAAAAPLKBAAAA|6846|1|54|6|AM|third|night|| +6847|AAAAAAAAAMKBAAAA|6847|1|54|7|AM|third|night|| +6848|AAAAAAAABMKBAAAA|6848|1|54|8|AM|third|night|| +6849|AAAAAAAACMKBAAAA|6849|1|54|9|AM|third|night|| +6850|AAAAAAAADMKBAAAA|6850|1|54|10|AM|third|night|| +6851|AAAAAAAAEMKBAAAA|6851|1|54|11|AM|third|night|| +6852|AAAAAAAAFMKBAAAA|6852|1|54|12|AM|third|night|| +6853|AAAAAAAAGMKBAAAA|6853|1|54|13|AM|third|night|| +6854|AAAAAAAAHMKBAAAA|6854|1|54|14|AM|third|night|| +6855|AAAAAAAAIMKBAAAA|6855|1|54|15|AM|third|night|| +6856|AAAAAAAAJMKBAAAA|6856|1|54|16|AM|third|night|| +6857|AAAAAAAAKMKBAAAA|6857|1|54|17|AM|third|night|| +6858|AAAAAAAALMKBAAAA|6858|1|54|18|AM|third|night|| +6859|AAAAAAAAMMKBAAAA|6859|1|54|19|AM|third|night|| +6860|AAAAAAAANMKBAAAA|6860|1|54|20|AM|third|night|| +6861|AAAAAAAAOMKBAAAA|6861|1|54|21|AM|third|night|| +6862|AAAAAAAAPMKBAAAA|6862|1|54|22|AM|third|night|| +6863|AAAAAAAAANKBAAAA|6863|1|54|23|AM|third|night|| +6864|AAAAAAAABNKBAAAA|6864|1|54|24|AM|third|night|| +6865|AAAAAAAACNKBAAAA|6865|1|54|25|AM|third|night|| +6866|AAAAAAAADNKBAAAA|6866|1|54|26|AM|third|night|| +6867|AAAAAAAAENKBAAAA|6867|1|54|27|AM|third|night|| +6868|AAAAAAAAFNKBAAAA|6868|1|54|28|AM|third|night|| +6869|AAAAAAAAGNKBAAAA|6869|1|54|29|AM|third|night|| +6870|AAAAAAAAHNKBAAAA|6870|1|54|30|AM|third|night|| +6871|AAAAAAAAINKBAAAA|6871|1|54|31|AM|third|night|| +6872|AAAAAAAAJNKBAAAA|6872|1|54|32|AM|third|night|| +6873|AAAAAAAAKNKBAAAA|6873|1|54|33|AM|third|night|| +6874|AAAAAAAALNKBAAAA|6874|1|54|34|AM|third|night|| +6875|AAAAAAAAMNKBAAAA|6875|1|54|35|AM|third|night|| +6876|AAAAAAAANNKBAAAA|6876|1|54|36|AM|third|night|| +6877|AAAAAAAAONKBAAAA|6877|1|54|37|AM|third|night|| +6878|AAAAAAAAPNKBAAAA|6878|1|54|38|AM|third|night|| +6879|AAAAAAAAAOKBAAAA|6879|1|54|39|AM|third|night|| +6880|AAAAAAAABOKBAAAA|6880|1|54|40|AM|third|night|| +6881|AAAAAAAACOKBAAAA|6881|1|54|41|AM|third|night|| +6882|AAAAAAAADOKBAAAA|6882|1|54|42|AM|third|night|| +6883|AAAAAAAAEOKBAAAA|6883|1|54|43|AM|third|night|| +6884|AAAAAAAAFOKBAAAA|6884|1|54|44|AM|third|night|| +6885|AAAAAAAAGOKBAAAA|6885|1|54|45|AM|third|night|| +6886|AAAAAAAAHOKBAAAA|6886|1|54|46|AM|third|night|| +6887|AAAAAAAAIOKBAAAA|6887|1|54|47|AM|third|night|| +6888|AAAAAAAAJOKBAAAA|6888|1|54|48|AM|third|night|| +6889|AAAAAAAAKOKBAAAA|6889|1|54|49|AM|third|night|| +6890|AAAAAAAALOKBAAAA|6890|1|54|50|AM|third|night|| +6891|AAAAAAAAMOKBAAAA|6891|1|54|51|AM|third|night|| +6892|AAAAAAAANOKBAAAA|6892|1|54|52|AM|third|night|| +6893|AAAAAAAAOOKBAAAA|6893|1|54|53|AM|third|night|| +6894|AAAAAAAAPOKBAAAA|6894|1|54|54|AM|third|night|| +6895|AAAAAAAAAPKBAAAA|6895|1|54|55|AM|third|night|| +6896|AAAAAAAABPKBAAAA|6896|1|54|56|AM|third|night|| +6897|AAAAAAAACPKBAAAA|6897|1|54|57|AM|third|night|| +6898|AAAAAAAADPKBAAAA|6898|1|54|58|AM|third|night|| +6899|AAAAAAAAEPKBAAAA|6899|1|54|59|AM|third|night|| +6900|AAAAAAAAFPKBAAAA|6900|1|55|0|AM|third|night|| +6901|AAAAAAAAGPKBAAAA|6901|1|55|1|AM|third|night|| +6902|AAAAAAAAHPKBAAAA|6902|1|55|2|AM|third|night|| +6903|AAAAAAAAIPKBAAAA|6903|1|55|3|AM|third|night|| +6904|AAAAAAAAJPKBAAAA|6904|1|55|4|AM|third|night|| +6905|AAAAAAAAKPKBAAAA|6905|1|55|5|AM|third|night|| +6906|AAAAAAAALPKBAAAA|6906|1|55|6|AM|third|night|| +6907|AAAAAAAAMPKBAAAA|6907|1|55|7|AM|third|night|| +6908|AAAAAAAANPKBAAAA|6908|1|55|8|AM|third|night|| +6909|AAAAAAAAOPKBAAAA|6909|1|55|9|AM|third|night|| +6910|AAAAAAAAPPKBAAAA|6910|1|55|10|AM|third|night|| +6911|AAAAAAAAAALBAAAA|6911|1|55|11|AM|third|night|| +6912|AAAAAAAABALBAAAA|6912|1|55|12|AM|third|night|| +6913|AAAAAAAACALBAAAA|6913|1|55|13|AM|third|night|| +6914|AAAAAAAADALBAAAA|6914|1|55|14|AM|third|night|| +6915|AAAAAAAAEALBAAAA|6915|1|55|15|AM|third|night|| +6916|AAAAAAAAFALBAAAA|6916|1|55|16|AM|third|night|| +6917|AAAAAAAAGALBAAAA|6917|1|55|17|AM|third|night|| +6918|AAAAAAAAHALBAAAA|6918|1|55|18|AM|third|night|| +6919|AAAAAAAAIALBAAAA|6919|1|55|19|AM|third|night|| +6920|AAAAAAAAJALBAAAA|6920|1|55|20|AM|third|night|| +6921|AAAAAAAAKALBAAAA|6921|1|55|21|AM|third|night|| +6922|AAAAAAAALALBAAAA|6922|1|55|22|AM|third|night|| +6923|AAAAAAAAMALBAAAA|6923|1|55|23|AM|third|night|| +6924|AAAAAAAANALBAAAA|6924|1|55|24|AM|third|night|| +6925|AAAAAAAAOALBAAAA|6925|1|55|25|AM|third|night|| +6926|AAAAAAAAPALBAAAA|6926|1|55|26|AM|third|night|| +6927|AAAAAAAAABLBAAAA|6927|1|55|27|AM|third|night|| +6928|AAAAAAAABBLBAAAA|6928|1|55|28|AM|third|night|| +6929|AAAAAAAACBLBAAAA|6929|1|55|29|AM|third|night|| +6930|AAAAAAAADBLBAAAA|6930|1|55|30|AM|third|night|| +6931|AAAAAAAAEBLBAAAA|6931|1|55|31|AM|third|night|| +6932|AAAAAAAAFBLBAAAA|6932|1|55|32|AM|third|night|| +6933|AAAAAAAAGBLBAAAA|6933|1|55|33|AM|third|night|| +6934|AAAAAAAAHBLBAAAA|6934|1|55|34|AM|third|night|| +6935|AAAAAAAAIBLBAAAA|6935|1|55|35|AM|third|night|| +6936|AAAAAAAAJBLBAAAA|6936|1|55|36|AM|third|night|| +6937|AAAAAAAAKBLBAAAA|6937|1|55|37|AM|third|night|| +6938|AAAAAAAALBLBAAAA|6938|1|55|38|AM|third|night|| +6939|AAAAAAAAMBLBAAAA|6939|1|55|39|AM|third|night|| +6940|AAAAAAAANBLBAAAA|6940|1|55|40|AM|third|night|| +6941|AAAAAAAAOBLBAAAA|6941|1|55|41|AM|third|night|| +6942|AAAAAAAAPBLBAAAA|6942|1|55|42|AM|third|night|| +6943|AAAAAAAAACLBAAAA|6943|1|55|43|AM|third|night|| +6944|AAAAAAAABCLBAAAA|6944|1|55|44|AM|third|night|| +6945|AAAAAAAACCLBAAAA|6945|1|55|45|AM|third|night|| +6946|AAAAAAAADCLBAAAA|6946|1|55|46|AM|third|night|| +6947|AAAAAAAAECLBAAAA|6947|1|55|47|AM|third|night|| +6948|AAAAAAAAFCLBAAAA|6948|1|55|48|AM|third|night|| +6949|AAAAAAAAGCLBAAAA|6949|1|55|49|AM|third|night|| +6950|AAAAAAAAHCLBAAAA|6950|1|55|50|AM|third|night|| +6951|AAAAAAAAICLBAAAA|6951|1|55|51|AM|third|night|| +6952|AAAAAAAAJCLBAAAA|6952|1|55|52|AM|third|night|| +6953|AAAAAAAAKCLBAAAA|6953|1|55|53|AM|third|night|| +6954|AAAAAAAALCLBAAAA|6954|1|55|54|AM|third|night|| +6955|AAAAAAAAMCLBAAAA|6955|1|55|55|AM|third|night|| +6956|AAAAAAAANCLBAAAA|6956|1|55|56|AM|third|night|| +6957|AAAAAAAAOCLBAAAA|6957|1|55|57|AM|third|night|| +6958|AAAAAAAAPCLBAAAA|6958|1|55|58|AM|third|night|| +6959|AAAAAAAAADLBAAAA|6959|1|55|59|AM|third|night|| +6960|AAAAAAAABDLBAAAA|6960|1|56|0|AM|third|night|| +6961|AAAAAAAACDLBAAAA|6961|1|56|1|AM|third|night|| +6962|AAAAAAAADDLBAAAA|6962|1|56|2|AM|third|night|| +6963|AAAAAAAAEDLBAAAA|6963|1|56|3|AM|third|night|| +6964|AAAAAAAAFDLBAAAA|6964|1|56|4|AM|third|night|| +6965|AAAAAAAAGDLBAAAA|6965|1|56|5|AM|third|night|| +6966|AAAAAAAAHDLBAAAA|6966|1|56|6|AM|third|night|| +6967|AAAAAAAAIDLBAAAA|6967|1|56|7|AM|third|night|| +6968|AAAAAAAAJDLBAAAA|6968|1|56|8|AM|third|night|| +6969|AAAAAAAAKDLBAAAA|6969|1|56|9|AM|third|night|| +6970|AAAAAAAALDLBAAAA|6970|1|56|10|AM|third|night|| +6971|AAAAAAAAMDLBAAAA|6971|1|56|11|AM|third|night|| +6972|AAAAAAAANDLBAAAA|6972|1|56|12|AM|third|night|| +6973|AAAAAAAAODLBAAAA|6973|1|56|13|AM|third|night|| +6974|AAAAAAAAPDLBAAAA|6974|1|56|14|AM|third|night|| +6975|AAAAAAAAAELBAAAA|6975|1|56|15|AM|third|night|| +6976|AAAAAAAABELBAAAA|6976|1|56|16|AM|third|night|| +6977|AAAAAAAACELBAAAA|6977|1|56|17|AM|third|night|| +6978|AAAAAAAADELBAAAA|6978|1|56|18|AM|third|night|| +6979|AAAAAAAAEELBAAAA|6979|1|56|19|AM|third|night|| +6980|AAAAAAAAFELBAAAA|6980|1|56|20|AM|third|night|| +6981|AAAAAAAAGELBAAAA|6981|1|56|21|AM|third|night|| +6982|AAAAAAAAHELBAAAA|6982|1|56|22|AM|third|night|| +6983|AAAAAAAAIELBAAAA|6983|1|56|23|AM|third|night|| +6984|AAAAAAAAJELBAAAA|6984|1|56|24|AM|third|night|| +6985|AAAAAAAAKELBAAAA|6985|1|56|25|AM|third|night|| +6986|AAAAAAAALELBAAAA|6986|1|56|26|AM|third|night|| +6987|AAAAAAAAMELBAAAA|6987|1|56|27|AM|third|night|| +6988|AAAAAAAANELBAAAA|6988|1|56|28|AM|third|night|| +6989|AAAAAAAAOELBAAAA|6989|1|56|29|AM|third|night|| +6990|AAAAAAAAPELBAAAA|6990|1|56|30|AM|third|night|| +6991|AAAAAAAAAFLBAAAA|6991|1|56|31|AM|third|night|| +6992|AAAAAAAABFLBAAAA|6992|1|56|32|AM|third|night|| +6993|AAAAAAAACFLBAAAA|6993|1|56|33|AM|third|night|| +6994|AAAAAAAADFLBAAAA|6994|1|56|34|AM|third|night|| +6995|AAAAAAAAEFLBAAAA|6995|1|56|35|AM|third|night|| +6996|AAAAAAAAFFLBAAAA|6996|1|56|36|AM|third|night|| +6997|AAAAAAAAGFLBAAAA|6997|1|56|37|AM|third|night|| +6998|AAAAAAAAHFLBAAAA|6998|1|56|38|AM|third|night|| +6999|AAAAAAAAIFLBAAAA|6999|1|56|39|AM|third|night|| +7000|AAAAAAAAJFLBAAAA|7000|1|56|40|AM|third|night|| +7001|AAAAAAAAKFLBAAAA|7001|1|56|41|AM|third|night|| +7002|AAAAAAAALFLBAAAA|7002|1|56|42|AM|third|night|| +7003|AAAAAAAAMFLBAAAA|7003|1|56|43|AM|third|night|| +7004|AAAAAAAANFLBAAAA|7004|1|56|44|AM|third|night|| +7005|AAAAAAAAOFLBAAAA|7005|1|56|45|AM|third|night|| +7006|AAAAAAAAPFLBAAAA|7006|1|56|46|AM|third|night|| +7007|AAAAAAAAAGLBAAAA|7007|1|56|47|AM|third|night|| +7008|AAAAAAAABGLBAAAA|7008|1|56|48|AM|third|night|| +7009|AAAAAAAACGLBAAAA|7009|1|56|49|AM|third|night|| +7010|AAAAAAAADGLBAAAA|7010|1|56|50|AM|third|night|| +7011|AAAAAAAAEGLBAAAA|7011|1|56|51|AM|third|night|| +7012|AAAAAAAAFGLBAAAA|7012|1|56|52|AM|third|night|| +7013|AAAAAAAAGGLBAAAA|7013|1|56|53|AM|third|night|| +7014|AAAAAAAAHGLBAAAA|7014|1|56|54|AM|third|night|| +7015|AAAAAAAAIGLBAAAA|7015|1|56|55|AM|third|night|| +7016|AAAAAAAAJGLBAAAA|7016|1|56|56|AM|third|night|| +7017|AAAAAAAAKGLBAAAA|7017|1|56|57|AM|third|night|| +7018|AAAAAAAALGLBAAAA|7018|1|56|58|AM|third|night|| +7019|AAAAAAAAMGLBAAAA|7019|1|56|59|AM|third|night|| +7020|AAAAAAAANGLBAAAA|7020|1|57|0|AM|third|night|| +7021|AAAAAAAAOGLBAAAA|7021|1|57|1|AM|third|night|| +7022|AAAAAAAAPGLBAAAA|7022|1|57|2|AM|third|night|| +7023|AAAAAAAAAHLBAAAA|7023|1|57|3|AM|third|night|| +7024|AAAAAAAABHLBAAAA|7024|1|57|4|AM|third|night|| +7025|AAAAAAAACHLBAAAA|7025|1|57|5|AM|third|night|| +7026|AAAAAAAADHLBAAAA|7026|1|57|6|AM|third|night|| +7027|AAAAAAAAEHLBAAAA|7027|1|57|7|AM|third|night|| +7028|AAAAAAAAFHLBAAAA|7028|1|57|8|AM|third|night|| +7029|AAAAAAAAGHLBAAAA|7029|1|57|9|AM|third|night|| +7030|AAAAAAAAHHLBAAAA|7030|1|57|10|AM|third|night|| +7031|AAAAAAAAIHLBAAAA|7031|1|57|11|AM|third|night|| +7032|AAAAAAAAJHLBAAAA|7032|1|57|12|AM|third|night|| +7033|AAAAAAAAKHLBAAAA|7033|1|57|13|AM|third|night|| +7034|AAAAAAAALHLBAAAA|7034|1|57|14|AM|third|night|| +7035|AAAAAAAAMHLBAAAA|7035|1|57|15|AM|third|night|| +7036|AAAAAAAANHLBAAAA|7036|1|57|16|AM|third|night|| +7037|AAAAAAAAOHLBAAAA|7037|1|57|17|AM|third|night|| +7038|AAAAAAAAPHLBAAAA|7038|1|57|18|AM|third|night|| +7039|AAAAAAAAAILBAAAA|7039|1|57|19|AM|third|night|| +7040|AAAAAAAABILBAAAA|7040|1|57|20|AM|third|night|| +7041|AAAAAAAACILBAAAA|7041|1|57|21|AM|third|night|| +7042|AAAAAAAADILBAAAA|7042|1|57|22|AM|third|night|| +7043|AAAAAAAAEILBAAAA|7043|1|57|23|AM|third|night|| +7044|AAAAAAAAFILBAAAA|7044|1|57|24|AM|third|night|| +7045|AAAAAAAAGILBAAAA|7045|1|57|25|AM|third|night|| +7046|AAAAAAAAHILBAAAA|7046|1|57|26|AM|third|night|| +7047|AAAAAAAAIILBAAAA|7047|1|57|27|AM|third|night|| +7048|AAAAAAAAJILBAAAA|7048|1|57|28|AM|third|night|| +7049|AAAAAAAAKILBAAAA|7049|1|57|29|AM|third|night|| +7050|AAAAAAAALILBAAAA|7050|1|57|30|AM|third|night|| +7051|AAAAAAAAMILBAAAA|7051|1|57|31|AM|third|night|| +7052|AAAAAAAANILBAAAA|7052|1|57|32|AM|third|night|| +7053|AAAAAAAAOILBAAAA|7053|1|57|33|AM|third|night|| +7054|AAAAAAAAPILBAAAA|7054|1|57|34|AM|third|night|| +7055|AAAAAAAAAJLBAAAA|7055|1|57|35|AM|third|night|| +7056|AAAAAAAABJLBAAAA|7056|1|57|36|AM|third|night|| +7057|AAAAAAAACJLBAAAA|7057|1|57|37|AM|third|night|| +7058|AAAAAAAADJLBAAAA|7058|1|57|38|AM|third|night|| +7059|AAAAAAAAEJLBAAAA|7059|1|57|39|AM|third|night|| +7060|AAAAAAAAFJLBAAAA|7060|1|57|40|AM|third|night|| +7061|AAAAAAAAGJLBAAAA|7061|1|57|41|AM|third|night|| +7062|AAAAAAAAHJLBAAAA|7062|1|57|42|AM|third|night|| +7063|AAAAAAAAIJLBAAAA|7063|1|57|43|AM|third|night|| +7064|AAAAAAAAJJLBAAAA|7064|1|57|44|AM|third|night|| +7065|AAAAAAAAKJLBAAAA|7065|1|57|45|AM|third|night|| +7066|AAAAAAAALJLBAAAA|7066|1|57|46|AM|third|night|| +7067|AAAAAAAAMJLBAAAA|7067|1|57|47|AM|third|night|| +7068|AAAAAAAANJLBAAAA|7068|1|57|48|AM|third|night|| +7069|AAAAAAAAOJLBAAAA|7069|1|57|49|AM|third|night|| +7070|AAAAAAAAPJLBAAAA|7070|1|57|50|AM|third|night|| +7071|AAAAAAAAAKLBAAAA|7071|1|57|51|AM|third|night|| +7072|AAAAAAAABKLBAAAA|7072|1|57|52|AM|third|night|| +7073|AAAAAAAACKLBAAAA|7073|1|57|53|AM|third|night|| +7074|AAAAAAAADKLBAAAA|7074|1|57|54|AM|third|night|| +7075|AAAAAAAAEKLBAAAA|7075|1|57|55|AM|third|night|| +7076|AAAAAAAAFKLBAAAA|7076|1|57|56|AM|third|night|| +7077|AAAAAAAAGKLBAAAA|7077|1|57|57|AM|third|night|| +7078|AAAAAAAAHKLBAAAA|7078|1|57|58|AM|third|night|| +7079|AAAAAAAAIKLBAAAA|7079|1|57|59|AM|third|night|| +7080|AAAAAAAAJKLBAAAA|7080|1|58|0|AM|third|night|| +7081|AAAAAAAAKKLBAAAA|7081|1|58|1|AM|third|night|| +7082|AAAAAAAALKLBAAAA|7082|1|58|2|AM|third|night|| +7083|AAAAAAAAMKLBAAAA|7083|1|58|3|AM|third|night|| +7084|AAAAAAAANKLBAAAA|7084|1|58|4|AM|third|night|| +7085|AAAAAAAAOKLBAAAA|7085|1|58|5|AM|third|night|| +7086|AAAAAAAAPKLBAAAA|7086|1|58|6|AM|third|night|| +7087|AAAAAAAAALLBAAAA|7087|1|58|7|AM|third|night|| +7088|AAAAAAAABLLBAAAA|7088|1|58|8|AM|third|night|| +7089|AAAAAAAACLLBAAAA|7089|1|58|9|AM|third|night|| +7090|AAAAAAAADLLBAAAA|7090|1|58|10|AM|third|night|| +7091|AAAAAAAAELLBAAAA|7091|1|58|11|AM|third|night|| +7092|AAAAAAAAFLLBAAAA|7092|1|58|12|AM|third|night|| +7093|AAAAAAAAGLLBAAAA|7093|1|58|13|AM|third|night|| +7094|AAAAAAAAHLLBAAAA|7094|1|58|14|AM|third|night|| +7095|AAAAAAAAILLBAAAA|7095|1|58|15|AM|third|night|| +7096|AAAAAAAAJLLBAAAA|7096|1|58|16|AM|third|night|| +7097|AAAAAAAAKLLBAAAA|7097|1|58|17|AM|third|night|| +7098|AAAAAAAALLLBAAAA|7098|1|58|18|AM|third|night|| +7099|AAAAAAAAMLLBAAAA|7099|1|58|19|AM|third|night|| +7100|AAAAAAAANLLBAAAA|7100|1|58|20|AM|third|night|| +7101|AAAAAAAAOLLBAAAA|7101|1|58|21|AM|third|night|| +7102|AAAAAAAAPLLBAAAA|7102|1|58|22|AM|third|night|| +7103|AAAAAAAAAMLBAAAA|7103|1|58|23|AM|third|night|| +7104|AAAAAAAABMLBAAAA|7104|1|58|24|AM|third|night|| +7105|AAAAAAAACMLBAAAA|7105|1|58|25|AM|third|night|| +7106|AAAAAAAADMLBAAAA|7106|1|58|26|AM|third|night|| +7107|AAAAAAAAEMLBAAAA|7107|1|58|27|AM|third|night|| +7108|AAAAAAAAFMLBAAAA|7108|1|58|28|AM|third|night|| +7109|AAAAAAAAGMLBAAAA|7109|1|58|29|AM|third|night|| +7110|AAAAAAAAHMLBAAAA|7110|1|58|30|AM|third|night|| +7111|AAAAAAAAIMLBAAAA|7111|1|58|31|AM|third|night|| +7112|AAAAAAAAJMLBAAAA|7112|1|58|32|AM|third|night|| +7113|AAAAAAAAKMLBAAAA|7113|1|58|33|AM|third|night|| +7114|AAAAAAAALMLBAAAA|7114|1|58|34|AM|third|night|| +7115|AAAAAAAAMMLBAAAA|7115|1|58|35|AM|third|night|| +7116|AAAAAAAANMLBAAAA|7116|1|58|36|AM|third|night|| +7117|AAAAAAAAOMLBAAAA|7117|1|58|37|AM|third|night|| +7118|AAAAAAAAPMLBAAAA|7118|1|58|38|AM|third|night|| +7119|AAAAAAAAANLBAAAA|7119|1|58|39|AM|third|night|| +7120|AAAAAAAABNLBAAAA|7120|1|58|40|AM|third|night|| +7121|AAAAAAAACNLBAAAA|7121|1|58|41|AM|third|night|| +7122|AAAAAAAADNLBAAAA|7122|1|58|42|AM|third|night|| +7123|AAAAAAAAENLBAAAA|7123|1|58|43|AM|third|night|| +7124|AAAAAAAAFNLBAAAA|7124|1|58|44|AM|third|night|| +7125|AAAAAAAAGNLBAAAA|7125|1|58|45|AM|third|night|| +7126|AAAAAAAAHNLBAAAA|7126|1|58|46|AM|third|night|| +7127|AAAAAAAAINLBAAAA|7127|1|58|47|AM|third|night|| +7128|AAAAAAAAJNLBAAAA|7128|1|58|48|AM|third|night|| +7129|AAAAAAAAKNLBAAAA|7129|1|58|49|AM|third|night|| +7130|AAAAAAAALNLBAAAA|7130|1|58|50|AM|third|night|| +7131|AAAAAAAAMNLBAAAA|7131|1|58|51|AM|third|night|| +7132|AAAAAAAANNLBAAAA|7132|1|58|52|AM|third|night|| +7133|AAAAAAAAONLBAAAA|7133|1|58|53|AM|third|night|| +7134|AAAAAAAAPNLBAAAA|7134|1|58|54|AM|third|night|| +7135|AAAAAAAAAOLBAAAA|7135|1|58|55|AM|third|night|| +7136|AAAAAAAABOLBAAAA|7136|1|58|56|AM|third|night|| +7137|AAAAAAAACOLBAAAA|7137|1|58|57|AM|third|night|| +7138|AAAAAAAADOLBAAAA|7138|1|58|58|AM|third|night|| +7139|AAAAAAAAEOLBAAAA|7139|1|58|59|AM|third|night|| +7140|AAAAAAAAFOLBAAAA|7140|1|59|0|AM|third|night|| +7141|AAAAAAAAGOLBAAAA|7141|1|59|1|AM|third|night|| +7142|AAAAAAAAHOLBAAAA|7142|1|59|2|AM|third|night|| +7143|AAAAAAAAIOLBAAAA|7143|1|59|3|AM|third|night|| +7144|AAAAAAAAJOLBAAAA|7144|1|59|4|AM|third|night|| +7145|AAAAAAAAKOLBAAAA|7145|1|59|5|AM|third|night|| +7146|AAAAAAAALOLBAAAA|7146|1|59|6|AM|third|night|| +7147|AAAAAAAAMOLBAAAA|7147|1|59|7|AM|third|night|| +7148|AAAAAAAANOLBAAAA|7148|1|59|8|AM|third|night|| +7149|AAAAAAAAOOLBAAAA|7149|1|59|9|AM|third|night|| +7150|AAAAAAAAPOLBAAAA|7150|1|59|10|AM|third|night|| +7151|AAAAAAAAAPLBAAAA|7151|1|59|11|AM|third|night|| +7152|AAAAAAAABPLBAAAA|7152|1|59|12|AM|third|night|| +7153|AAAAAAAACPLBAAAA|7153|1|59|13|AM|third|night|| +7154|AAAAAAAADPLBAAAA|7154|1|59|14|AM|third|night|| +7155|AAAAAAAAEPLBAAAA|7155|1|59|15|AM|third|night|| +7156|AAAAAAAAFPLBAAAA|7156|1|59|16|AM|third|night|| +7157|AAAAAAAAGPLBAAAA|7157|1|59|17|AM|third|night|| +7158|AAAAAAAAHPLBAAAA|7158|1|59|18|AM|third|night|| +7159|AAAAAAAAIPLBAAAA|7159|1|59|19|AM|third|night|| +7160|AAAAAAAAJPLBAAAA|7160|1|59|20|AM|third|night|| +7161|AAAAAAAAKPLBAAAA|7161|1|59|21|AM|third|night|| +7162|AAAAAAAALPLBAAAA|7162|1|59|22|AM|third|night|| +7163|AAAAAAAAMPLBAAAA|7163|1|59|23|AM|third|night|| +7164|AAAAAAAANPLBAAAA|7164|1|59|24|AM|third|night|| +7165|AAAAAAAAOPLBAAAA|7165|1|59|25|AM|third|night|| +7166|AAAAAAAAPPLBAAAA|7166|1|59|26|AM|third|night|| +7167|AAAAAAAAAAMBAAAA|7167|1|59|27|AM|third|night|| +7168|AAAAAAAABAMBAAAA|7168|1|59|28|AM|third|night|| +7169|AAAAAAAACAMBAAAA|7169|1|59|29|AM|third|night|| +7170|AAAAAAAADAMBAAAA|7170|1|59|30|AM|third|night|| +7171|AAAAAAAAEAMBAAAA|7171|1|59|31|AM|third|night|| +7172|AAAAAAAAFAMBAAAA|7172|1|59|32|AM|third|night|| +7173|AAAAAAAAGAMBAAAA|7173|1|59|33|AM|third|night|| +7174|AAAAAAAAHAMBAAAA|7174|1|59|34|AM|third|night|| +7175|AAAAAAAAIAMBAAAA|7175|1|59|35|AM|third|night|| +7176|AAAAAAAAJAMBAAAA|7176|1|59|36|AM|third|night|| +7177|AAAAAAAAKAMBAAAA|7177|1|59|37|AM|third|night|| +7178|AAAAAAAALAMBAAAA|7178|1|59|38|AM|third|night|| +7179|AAAAAAAAMAMBAAAA|7179|1|59|39|AM|third|night|| +7180|AAAAAAAANAMBAAAA|7180|1|59|40|AM|third|night|| +7181|AAAAAAAAOAMBAAAA|7181|1|59|41|AM|third|night|| +7182|AAAAAAAAPAMBAAAA|7182|1|59|42|AM|third|night|| +7183|AAAAAAAAABMBAAAA|7183|1|59|43|AM|third|night|| +7184|AAAAAAAABBMBAAAA|7184|1|59|44|AM|third|night|| +7185|AAAAAAAACBMBAAAA|7185|1|59|45|AM|third|night|| +7186|AAAAAAAADBMBAAAA|7186|1|59|46|AM|third|night|| +7187|AAAAAAAAEBMBAAAA|7187|1|59|47|AM|third|night|| +7188|AAAAAAAAFBMBAAAA|7188|1|59|48|AM|third|night|| +7189|AAAAAAAAGBMBAAAA|7189|1|59|49|AM|third|night|| +7190|AAAAAAAAHBMBAAAA|7190|1|59|50|AM|third|night|| +7191|AAAAAAAAIBMBAAAA|7191|1|59|51|AM|third|night|| +7192|AAAAAAAAJBMBAAAA|7192|1|59|52|AM|third|night|| +7193|AAAAAAAAKBMBAAAA|7193|1|59|53|AM|third|night|| +7194|AAAAAAAALBMBAAAA|7194|1|59|54|AM|third|night|| +7195|AAAAAAAAMBMBAAAA|7195|1|59|55|AM|third|night|| +7196|AAAAAAAANBMBAAAA|7196|1|59|56|AM|third|night|| +7197|AAAAAAAAOBMBAAAA|7197|1|59|57|AM|third|night|| +7198|AAAAAAAAPBMBAAAA|7198|1|59|58|AM|third|night|| +7199|AAAAAAAAACMBAAAA|7199|1|59|59|AM|third|night|| +7200|AAAAAAAABCMBAAAA|7200|2|0|0|AM|third|night|| +7201|AAAAAAAACCMBAAAA|7201|2|0|1|AM|third|night|| +7202|AAAAAAAADCMBAAAA|7202|2|0|2|AM|third|night|| +7203|AAAAAAAAECMBAAAA|7203|2|0|3|AM|third|night|| +7204|AAAAAAAAFCMBAAAA|7204|2|0|4|AM|third|night|| +7205|AAAAAAAAGCMBAAAA|7205|2|0|5|AM|third|night|| +7206|AAAAAAAAHCMBAAAA|7206|2|0|6|AM|third|night|| +7207|AAAAAAAAICMBAAAA|7207|2|0|7|AM|third|night|| +7208|AAAAAAAAJCMBAAAA|7208|2|0|8|AM|third|night|| +7209|AAAAAAAAKCMBAAAA|7209|2|0|9|AM|third|night|| +7210|AAAAAAAALCMBAAAA|7210|2|0|10|AM|third|night|| +7211|AAAAAAAAMCMBAAAA|7211|2|0|11|AM|third|night|| +7212|AAAAAAAANCMBAAAA|7212|2|0|12|AM|third|night|| +7213|AAAAAAAAOCMBAAAA|7213|2|0|13|AM|third|night|| +7214|AAAAAAAAPCMBAAAA|7214|2|0|14|AM|third|night|| +7215|AAAAAAAAADMBAAAA|7215|2|0|15|AM|third|night|| +7216|AAAAAAAABDMBAAAA|7216|2|0|16|AM|third|night|| +7217|AAAAAAAACDMBAAAA|7217|2|0|17|AM|third|night|| +7218|AAAAAAAADDMBAAAA|7218|2|0|18|AM|third|night|| +7219|AAAAAAAAEDMBAAAA|7219|2|0|19|AM|third|night|| +7220|AAAAAAAAFDMBAAAA|7220|2|0|20|AM|third|night|| +7221|AAAAAAAAGDMBAAAA|7221|2|0|21|AM|third|night|| +7222|AAAAAAAAHDMBAAAA|7222|2|0|22|AM|third|night|| +7223|AAAAAAAAIDMBAAAA|7223|2|0|23|AM|third|night|| +7224|AAAAAAAAJDMBAAAA|7224|2|0|24|AM|third|night|| +7225|AAAAAAAAKDMBAAAA|7225|2|0|25|AM|third|night|| +7226|AAAAAAAALDMBAAAA|7226|2|0|26|AM|third|night|| +7227|AAAAAAAAMDMBAAAA|7227|2|0|27|AM|third|night|| +7228|AAAAAAAANDMBAAAA|7228|2|0|28|AM|third|night|| +7229|AAAAAAAAODMBAAAA|7229|2|0|29|AM|third|night|| +7230|AAAAAAAAPDMBAAAA|7230|2|0|30|AM|third|night|| +7231|AAAAAAAAAEMBAAAA|7231|2|0|31|AM|third|night|| +7232|AAAAAAAABEMBAAAA|7232|2|0|32|AM|third|night|| +7233|AAAAAAAACEMBAAAA|7233|2|0|33|AM|third|night|| +7234|AAAAAAAADEMBAAAA|7234|2|0|34|AM|third|night|| +7235|AAAAAAAAEEMBAAAA|7235|2|0|35|AM|third|night|| +7236|AAAAAAAAFEMBAAAA|7236|2|0|36|AM|third|night|| +7237|AAAAAAAAGEMBAAAA|7237|2|0|37|AM|third|night|| +7238|AAAAAAAAHEMBAAAA|7238|2|0|38|AM|third|night|| +7239|AAAAAAAAIEMBAAAA|7239|2|0|39|AM|third|night|| +7240|AAAAAAAAJEMBAAAA|7240|2|0|40|AM|third|night|| +7241|AAAAAAAAKEMBAAAA|7241|2|0|41|AM|third|night|| +7242|AAAAAAAALEMBAAAA|7242|2|0|42|AM|third|night|| +7243|AAAAAAAAMEMBAAAA|7243|2|0|43|AM|third|night|| +7244|AAAAAAAANEMBAAAA|7244|2|0|44|AM|third|night|| +7245|AAAAAAAAOEMBAAAA|7245|2|0|45|AM|third|night|| +7246|AAAAAAAAPEMBAAAA|7246|2|0|46|AM|third|night|| +7247|AAAAAAAAAFMBAAAA|7247|2|0|47|AM|third|night|| +7248|AAAAAAAABFMBAAAA|7248|2|0|48|AM|third|night|| +7249|AAAAAAAACFMBAAAA|7249|2|0|49|AM|third|night|| +7250|AAAAAAAADFMBAAAA|7250|2|0|50|AM|third|night|| +7251|AAAAAAAAEFMBAAAA|7251|2|0|51|AM|third|night|| +7252|AAAAAAAAFFMBAAAA|7252|2|0|52|AM|third|night|| +7253|AAAAAAAAGFMBAAAA|7253|2|0|53|AM|third|night|| +7254|AAAAAAAAHFMBAAAA|7254|2|0|54|AM|third|night|| +7255|AAAAAAAAIFMBAAAA|7255|2|0|55|AM|third|night|| +7256|AAAAAAAAJFMBAAAA|7256|2|0|56|AM|third|night|| +7257|AAAAAAAAKFMBAAAA|7257|2|0|57|AM|third|night|| +7258|AAAAAAAALFMBAAAA|7258|2|0|58|AM|third|night|| +7259|AAAAAAAAMFMBAAAA|7259|2|0|59|AM|third|night|| +7260|AAAAAAAANFMBAAAA|7260|2|1|0|AM|third|night|| +7261|AAAAAAAAOFMBAAAA|7261|2|1|1|AM|third|night|| +7262|AAAAAAAAPFMBAAAA|7262|2|1|2|AM|third|night|| +7263|AAAAAAAAAGMBAAAA|7263|2|1|3|AM|third|night|| +7264|AAAAAAAABGMBAAAA|7264|2|1|4|AM|third|night|| +7265|AAAAAAAACGMBAAAA|7265|2|1|5|AM|third|night|| +7266|AAAAAAAADGMBAAAA|7266|2|1|6|AM|third|night|| +7267|AAAAAAAAEGMBAAAA|7267|2|1|7|AM|third|night|| +7268|AAAAAAAAFGMBAAAA|7268|2|1|8|AM|third|night|| +7269|AAAAAAAAGGMBAAAA|7269|2|1|9|AM|third|night|| +7270|AAAAAAAAHGMBAAAA|7270|2|1|10|AM|third|night|| +7271|AAAAAAAAIGMBAAAA|7271|2|1|11|AM|third|night|| +7272|AAAAAAAAJGMBAAAA|7272|2|1|12|AM|third|night|| +7273|AAAAAAAAKGMBAAAA|7273|2|1|13|AM|third|night|| +7274|AAAAAAAALGMBAAAA|7274|2|1|14|AM|third|night|| +7275|AAAAAAAAMGMBAAAA|7275|2|1|15|AM|third|night|| +7276|AAAAAAAANGMBAAAA|7276|2|1|16|AM|third|night|| +7277|AAAAAAAAOGMBAAAA|7277|2|1|17|AM|third|night|| +7278|AAAAAAAAPGMBAAAA|7278|2|1|18|AM|third|night|| +7279|AAAAAAAAAHMBAAAA|7279|2|1|19|AM|third|night|| +7280|AAAAAAAABHMBAAAA|7280|2|1|20|AM|third|night|| +7281|AAAAAAAACHMBAAAA|7281|2|1|21|AM|third|night|| +7282|AAAAAAAADHMBAAAA|7282|2|1|22|AM|third|night|| +7283|AAAAAAAAEHMBAAAA|7283|2|1|23|AM|third|night|| +7284|AAAAAAAAFHMBAAAA|7284|2|1|24|AM|third|night|| +7285|AAAAAAAAGHMBAAAA|7285|2|1|25|AM|third|night|| +7286|AAAAAAAAHHMBAAAA|7286|2|1|26|AM|third|night|| +7287|AAAAAAAAIHMBAAAA|7287|2|1|27|AM|third|night|| +7288|AAAAAAAAJHMBAAAA|7288|2|1|28|AM|third|night|| +7289|AAAAAAAAKHMBAAAA|7289|2|1|29|AM|third|night|| +7290|AAAAAAAALHMBAAAA|7290|2|1|30|AM|third|night|| +7291|AAAAAAAAMHMBAAAA|7291|2|1|31|AM|third|night|| +7292|AAAAAAAANHMBAAAA|7292|2|1|32|AM|third|night|| +7293|AAAAAAAAOHMBAAAA|7293|2|1|33|AM|third|night|| +7294|AAAAAAAAPHMBAAAA|7294|2|1|34|AM|third|night|| +7295|AAAAAAAAAIMBAAAA|7295|2|1|35|AM|third|night|| +7296|AAAAAAAABIMBAAAA|7296|2|1|36|AM|third|night|| +7297|AAAAAAAACIMBAAAA|7297|2|1|37|AM|third|night|| +7298|AAAAAAAADIMBAAAA|7298|2|1|38|AM|third|night|| +7299|AAAAAAAAEIMBAAAA|7299|2|1|39|AM|third|night|| +7300|AAAAAAAAFIMBAAAA|7300|2|1|40|AM|third|night|| +7301|AAAAAAAAGIMBAAAA|7301|2|1|41|AM|third|night|| +7302|AAAAAAAAHIMBAAAA|7302|2|1|42|AM|third|night|| +7303|AAAAAAAAIIMBAAAA|7303|2|1|43|AM|third|night|| +7304|AAAAAAAAJIMBAAAA|7304|2|1|44|AM|third|night|| +7305|AAAAAAAAKIMBAAAA|7305|2|1|45|AM|third|night|| +7306|AAAAAAAALIMBAAAA|7306|2|1|46|AM|third|night|| +7307|AAAAAAAAMIMBAAAA|7307|2|1|47|AM|third|night|| +7308|AAAAAAAANIMBAAAA|7308|2|1|48|AM|third|night|| +7309|AAAAAAAAOIMBAAAA|7309|2|1|49|AM|third|night|| +7310|AAAAAAAAPIMBAAAA|7310|2|1|50|AM|third|night|| +7311|AAAAAAAAAJMBAAAA|7311|2|1|51|AM|third|night|| +7312|AAAAAAAABJMBAAAA|7312|2|1|52|AM|third|night|| +7313|AAAAAAAACJMBAAAA|7313|2|1|53|AM|third|night|| +7314|AAAAAAAADJMBAAAA|7314|2|1|54|AM|third|night|| +7315|AAAAAAAAEJMBAAAA|7315|2|1|55|AM|third|night|| +7316|AAAAAAAAFJMBAAAA|7316|2|1|56|AM|third|night|| +7317|AAAAAAAAGJMBAAAA|7317|2|1|57|AM|third|night|| +7318|AAAAAAAAHJMBAAAA|7318|2|1|58|AM|third|night|| +7319|AAAAAAAAIJMBAAAA|7319|2|1|59|AM|third|night|| +7320|AAAAAAAAJJMBAAAA|7320|2|2|0|AM|third|night|| +7321|AAAAAAAAKJMBAAAA|7321|2|2|1|AM|third|night|| +7322|AAAAAAAALJMBAAAA|7322|2|2|2|AM|third|night|| +7323|AAAAAAAAMJMBAAAA|7323|2|2|3|AM|third|night|| +7324|AAAAAAAANJMBAAAA|7324|2|2|4|AM|third|night|| +7325|AAAAAAAAOJMBAAAA|7325|2|2|5|AM|third|night|| +7326|AAAAAAAAPJMBAAAA|7326|2|2|6|AM|third|night|| +7327|AAAAAAAAAKMBAAAA|7327|2|2|7|AM|third|night|| +7328|AAAAAAAABKMBAAAA|7328|2|2|8|AM|third|night|| +7329|AAAAAAAACKMBAAAA|7329|2|2|9|AM|third|night|| +7330|AAAAAAAADKMBAAAA|7330|2|2|10|AM|third|night|| +7331|AAAAAAAAEKMBAAAA|7331|2|2|11|AM|third|night|| +7332|AAAAAAAAFKMBAAAA|7332|2|2|12|AM|third|night|| +7333|AAAAAAAAGKMBAAAA|7333|2|2|13|AM|third|night|| +7334|AAAAAAAAHKMBAAAA|7334|2|2|14|AM|third|night|| +7335|AAAAAAAAIKMBAAAA|7335|2|2|15|AM|third|night|| +7336|AAAAAAAAJKMBAAAA|7336|2|2|16|AM|third|night|| +7337|AAAAAAAAKKMBAAAA|7337|2|2|17|AM|third|night|| +7338|AAAAAAAALKMBAAAA|7338|2|2|18|AM|third|night|| +7339|AAAAAAAAMKMBAAAA|7339|2|2|19|AM|third|night|| +7340|AAAAAAAANKMBAAAA|7340|2|2|20|AM|third|night|| +7341|AAAAAAAAOKMBAAAA|7341|2|2|21|AM|third|night|| +7342|AAAAAAAAPKMBAAAA|7342|2|2|22|AM|third|night|| +7343|AAAAAAAAALMBAAAA|7343|2|2|23|AM|third|night|| +7344|AAAAAAAABLMBAAAA|7344|2|2|24|AM|third|night|| +7345|AAAAAAAACLMBAAAA|7345|2|2|25|AM|third|night|| +7346|AAAAAAAADLMBAAAA|7346|2|2|26|AM|third|night|| +7347|AAAAAAAAELMBAAAA|7347|2|2|27|AM|third|night|| +7348|AAAAAAAAFLMBAAAA|7348|2|2|28|AM|third|night|| +7349|AAAAAAAAGLMBAAAA|7349|2|2|29|AM|third|night|| +7350|AAAAAAAAHLMBAAAA|7350|2|2|30|AM|third|night|| +7351|AAAAAAAAILMBAAAA|7351|2|2|31|AM|third|night|| +7352|AAAAAAAAJLMBAAAA|7352|2|2|32|AM|third|night|| +7353|AAAAAAAAKLMBAAAA|7353|2|2|33|AM|third|night|| +7354|AAAAAAAALLMBAAAA|7354|2|2|34|AM|third|night|| +7355|AAAAAAAAMLMBAAAA|7355|2|2|35|AM|third|night|| +7356|AAAAAAAANLMBAAAA|7356|2|2|36|AM|third|night|| +7357|AAAAAAAAOLMBAAAA|7357|2|2|37|AM|third|night|| +7358|AAAAAAAAPLMBAAAA|7358|2|2|38|AM|third|night|| +7359|AAAAAAAAAMMBAAAA|7359|2|2|39|AM|third|night|| +7360|AAAAAAAABMMBAAAA|7360|2|2|40|AM|third|night|| +7361|AAAAAAAACMMBAAAA|7361|2|2|41|AM|third|night|| +7362|AAAAAAAADMMBAAAA|7362|2|2|42|AM|third|night|| +7363|AAAAAAAAEMMBAAAA|7363|2|2|43|AM|third|night|| +7364|AAAAAAAAFMMBAAAA|7364|2|2|44|AM|third|night|| +7365|AAAAAAAAGMMBAAAA|7365|2|2|45|AM|third|night|| +7366|AAAAAAAAHMMBAAAA|7366|2|2|46|AM|third|night|| +7367|AAAAAAAAIMMBAAAA|7367|2|2|47|AM|third|night|| +7368|AAAAAAAAJMMBAAAA|7368|2|2|48|AM|third|night|| +7369|AAAAAAAAKMMBAAAA|7369|2|2|49|AM|third|night|| +7370|AAAAAAAALMMBAAAA|7370|2|2|50|AM|third|night|| +7371|AAAAAAAAMMMBAAAA|7371|2|2|51|AM|third|night|| +7372|AAAAAAAANMMBAAAA|7372|2|2|52|AM|third|night|| +7373|AAAAAAAAOMMBAAAA|7373|2|2|53|AM|third|night|| +7374|AAAAAAAAPMMBAAAA|7374|2|2|54|AM|third|night|| +7375|AAAAAAAAANMBAAAA|7375|2|2|55|AM|third|night|| +7376|AAAAAAAABNMBAAAA|7376|2|2|56|AM|third|night|| +7377|AAAAAAAACNMBAAAA|7377|2|2|57|AM|third|night|| +7378|AAAAAAAADNMBAAAA|7378|2|2|58|AM|third|night|| +7379|AAAAAAAAENMBAAAA|7379|2|2|59|AM|third|night|| +7380|AAAAAAAAFNMBAAAA|7380|2|3|0|AM|third|night|| +7381|AAAAAAAAGNMBAAAA|7381|2|3|1|AM|third|night|| +7382|AAAAAAAAHNMBAAAA|7382|2|3|2|AM|third|night|| +7383|AAAAAAAAINMBAAAA|7383|2|3|3|AM|third|night|| +7384|AAAAAAAAJNMBAAAA|7384|2|3|4|AM|third|night|| +7385|AAAAAAAAKNMBAAAA|7385|2|3|5|AM|third|night|| +7386|AAAAAAAALNMBAAAA|7386|2|3|6|AM|third|night|| +7387|AAAAAAAAMNMBAAAA|7387|2|3|7|AM|third|night|| +7388|AAAAAAAANNMBAAAA|7388|2|3|8|AM|third|night|| +7389|AAAAAAAAONMBAAAA|7389|2|3|9|AM|third|night|| +7390|AAAAAAAAPNMBAAAA|7390|2|3|10|AM|third|night|| +7391|AAAAAAAAAOMBAAAA|7391|2|3|11|AM|third|night|| +7392|AAAAAAAABOMBAAAA|7392|2|3|12|AM|third|night|| +7393|AAAAAAAACOMBAAAA|7393|2|3|13|AM|third|night|| +7394|AAAAAAAADOMBAAAA|7394|2|3|14|AM|third|night|| +7395|AAAAAAAAEOMBAAAA|7395|2|3|15|AM|third|night|| +7396|AAAAAAAAFOMBAAAA|7396|2|3|16|AM|third|night|| +7397|AAAAAAAAGOMBAAAA|7397|2|3|17|AM|third|night|| +7398|AAAAAAAAHOMBAAAA|7398|2|3|18|AM|third|night|| +7399|AAAAAAAAIOMBAAAA|7399|2|3|19|AM|third|night|| +7400|AAAAAAAAJOMBAAAA|7400|2|3|20|AM|third|night|| +7401|AAAAAAAAKOMBAAAA|7401|2|3|21|AM|third|night|| +7402|AAAAAAAALOMBAAAA|7402|2|3|22|AM|third|night|| +7403|AAAAAAAAMOMBAAAA|7403|2|3|23|AM|third|night|| +7404|AAAAAAAANOMBAAAA|7404|2|3|24|AM|third|night|| +7405|AAAAAAAAOOMBAAAA|7405|2|3|25|AM|third|night|| +7406|AAAAAAAAPOMBAAAA|7406|2|3|26|AM|third|night|| +7407|AAAAAAAAAPMBAAAA|7407|2|3|27|AM|third|night|| +7408|AAAAAAAABPMBAAAA|7408|2|3|28|AM|third|night|| +7409|AAAAAAAACPMBAAAA|7409|2|3|29|AM|third|night|| +7410|AAAAAAAADPMBAAAA|7410|2|3|30|AM|third|night|| +7411|AAAAAAAAEPMBAAAA|7411|2|3|31|AM|third|night|| +7412|AAAAAAAAFPMBAAAA|7412|2|3|32|AM|third|night|| +7413|AAAAAAAAGPMBAAAA|7413|2|3|33|AM|third|night|| +7414|AAAAAAAAHPMBAAAA|7414|2|3|34|AM|third|night|| +7415|AAAAAAAAIPMBAAAA|7415|2|3|35|AM|third|night|| +7416|AAAAAAAAJPMBAAAA|7416|2|3|36|AM|third|night|| +7417|AAAAAAAAKPMBAAAA|7417|2|3|37|AM|third|night|| +7418|AAAAAAAALPMBAAAA|7418|2|3|38|AM|third|night|| +7419|AAAAAAAAMPMBAAAA|7419|2|3|39|AM|third|night|| +7420|AAAAAAAANPMBAAAA|7420|2|3|40|AM|third|night|| +7421|AAAAAAAAOPMBAAAA|7421|2|3|41|AM|third|night|| +7422|AAAAAAAAPPMBAAAA|7422|2|3|42|AM|third|night|| +7423|AAAAAAAAAANBAAAA|7423|2|3|43|AM|third|night|| +7424|AAAAAAAABANBAAAA|7424|2|3|44|AM|third|night|| +7425|AAAAAAAACANBAAAA|7425|2|3|45|AM|third|night|| +7426|AAAAAAAADANBAAAA|7426|2|3|46|AM|third|night|| +7427|AAAAAAAAEANBAAAA|7427|2|3|47|AM|third|night|| +7428|AAAAAAAAFANBAAAA|7428|2|3|48|AM|third|night|| +7429|AAAAAAAAGANBAAAA|7429|2|3|49|AM|third|night|| +7430|AAAAAAAAHANBAAAA|7430|2|3|50|AM|third|night|| +7431|AAAAAAAAIANBAAAA|7431|2|3|51|AM|third|night|| +7432|AAAAAAAAJANBAAAA|7432|2|3|52|AM|third|night|| +7433|AAAAAAAAKANBAAAA|7433|2|3|53|AM|third|night|| +7434|AAAAAAAALANBAAAA|7434|2|3|54|AM|third|night|| +7435|AAAAAAAAMANBAAAA|7435|2|3|55|AM|third|night|| +7436|AAAAAAAANANBAAAA|7436|2|3|56|AM|third|night|| +7437|AAAAAAAAOANBAAAA|7437|2|3|57|AM|third|night|| +7438|AAAAAAAAPANBAAAA|7438|2|3|58|AM|third|night|| +7439|AAAAAAAAABNBAAAA|7439|2|3|59|AM|third|night|| +7440|AAAAAAAABBNBAAAA|7440|2|4|0|AM|third|night|| +7441|AAAAAAAACBNBAAAA|7441|2|4|1|AM|third|night|| +7442|AAAAAAAADBNBAAAA|7442|2|4|2|AM|third|night|| +7443|AAAAAAAAEBNBAAAA|7443|2|4|3|AM|third|night|| +7444|AAAAAAAAFBNBAAAA|7444|2|4|4|AM|third|night|| +7445|AAAAAAAAGBNBAAAA|7445|2|4|5|AM|third|night|| +7446|AAAAAAAAHBNBAAAA|7446|2|4|6|AM|third|night|| +7447|AAAAAAAAIBNBAAAA|7447|2|4|7|AM|third|night|| +7448|AAAAAAAAJBNBAAAA|7448|2|4|8|AM|third|night|| +7449|AAAAAAAAKBNBAAAA|7449|2|4|9|AM|third|night|| +7450|AAAAAAAALBNBAAAA|7450|2|4|10|AM|third|night|| +7451|AAAAAAAAMBNBAAAA|7451|2|4|11|AM|third|night|| +7452|AAAAAAAANBNBAAAA|7452|2|4|12|AM|third|night|| +7453|AAAAAAAAOBNBAAAA|7453|2|4|13|AM|third|night|| +7454|AAAAAAAAPBNBAAAA|7454|2|4|14|AM|third|night|| +7455|AAAAAAAAACNBAAAA|7455|2|4|15|AM|third|night|| +7456|AAAAAAAABCNBAAAA|7456|2|4|16|AM|third|night|| +7457|AAAAAAAACCNBAAAA|7457|2|4|17|AM|third|night|| +7458|AAAAAAAADCNBAAAA|7458|2|4|18|AM|third|night|| +7459|AAAAAAAAECNBAAAA|7459|2|4|19|AM|third|night|| +7460|AAAAAAAAFCNBAAAA|7460|2|4|20|AM|third|night|| +7461|AAAAAAAAGCNBAAAA|7461|2|4|21|AM|third|night|| +7462|AAAAAAAAHCNBAAAA|7462|2|4|22|AM|third|night|| +7463|AAAAAAAAICNBAAAA|7463|2|4|23|AM|third|night|| +7464|AAAAAAAAJCNBAAAA|7464|2|4|24|AM|third|night|| +7465|AAAAAAAAKCNBAAAA|7465|2|4|25|AM|third|night|| +7466|AAAAAAAALCNBAAAA|7466|2|4|26|AM|third|night|| +7467|AAAAAAAAMCNBAAAA|7467|2|4|27|AM|third|night|| +7468|AAAAAAAANCNBAAAA|7468|2|4|28|AM|third|night|| +7469|AAAAAAAAOCNBAAAA|7469|2|4|29|AM|third|night|| +7470|AAAAAAAAPCNBAAAA|7470|2|4|30|AM|third|night|| +7471|AAAAAAAAADNBAAAA|7471|2|4|31|AM|third|night|| +7472|AAAAAAAABDNBAAAA|7472|2|4|32|AM|third|night|| +7473|AAAAAAAACDNBAAAA|7473|2|4|33|AM|third|night|| +7474|AAAAAAAADDNBAAAA|7474|2|4|34|AM|third|night|| +7475|AAAAAAAAEDNBAAAA|7475|2|4|35|AM|third|night|| +7476|AAAAAAAAFDNBAAAA|7476|2|4|36|AM|third|night|| +7477|AAAAAAAAGDNBAAAA|7477|2|4|37|AM|third|night|| +7478|AAAAAAAAHDNBAAAA|7478|2|4|38|AM|third|night|| +7479|AAAAAAAAIDNBAAAA|7479|2|4|39|AM|third|night|| +7480|AAAAAAAAJDNBAAAA|7480|2|4|40|AM|third|night|| +7481|AAAAAAAAKDNBAAAA|7481|2|4|41|AM|third|night|| +7482|AAAAAAAALDNBAAAA|7482|2|4|42|AM|third|night|| +7483|AAAAAAAAMDNBAAAA|7483|2|4|43|AM|third|night|| +7484|AAAAAAAANDNBAAAA|7484|2|4|44|AM|third|night|| +7485|AAAAAAAAODNBAAAA|7485|2|4|45|AM|third|night|| +7486|AAAAAAAAPDNBAAAA|7486|2|4|46|AM|third|night|| +7487|AAAAAAAAAENBAAAA|7487|2|4|47|AM|third|night|| +7488|AAAAAAAABENBAAAA|7488|2|4|48|AM|third|night|| +7489|AAAAAAAACENBAAAA|7489|2|4|49|AM|third|night|| +7490|AAAAAAAADENBAAAA|7490|2|4|50|AM|third|night|| +7491|AAAAAAAAEENBAAAA|7491|2|4|51|AM|third|night|| +7492|AAAAAAAAFENBAAAA|7492|2|4|52|AM|third|night|| +7493|AAAAAAAAGENBAAAA|7493|2|4|53|AM|third|night|| +7494|AAAAAAAAHENBAAAA|7494|2|4|54|AM|third|night|| +7495|AAAAAAAAIENBAAAA|7495|2|4|55|AM|third|night|| +7496|AAAAAAAAJENBAAAA|7496|2|4|56|AM|third|night|| +7497|AAAAAAAAKENBAAAA|7497|2|4|57|AM|third|night|| +7498|AAAAAAAALENBAAAA|7498|2|4|58|AM|third|night|| +7499|AAAAAAAAMENBAAAA|7499|2|4|59|AM|third|night|| +7500|AAAAAAAANENBAAAA|7500|2|5|0|AM|third|night|| +7501|AAAAAAAAOENBAAAA|7501|2|5|1|AM|third|night|| +7502|AAAAAAAAPENBAAAA|7502|2|5|2|AM|third|night|| +7503|AAAAAAAAAFNBAAAA|7503|2|5|3|AM|third|night|| +7504|AAAAAAAABFNBAAAA|7504|2|5|4|AM|third|night|| +7505|AAAAAAAACFNBAAAA|7505|2|5|5|AM|third|night|| +7506|AAAAAAAADFNBAAAA|7506|2|5|6|AM|third|night|| +7507|AAAAAAAAEFNBAAAA|7507|2|5|7|AM|third|night|| +7508|AAAAAAAAFFNBAAAA|7508|2|5|8|AM|third|night|| +7509|AAAAAAAAGFNBAAAA|7509|2|5|9|AM|third|night|| +7510|AAAAAAAAHFNBAAAA|7510|2|5|10|AM|third|night|| +7511|AAAAAAAAIFNBAAAA|7511|2|5|11|AM|third|night|| +7512|AAAAAAAAJFNBAAAA|7512|2|5|12|AM|third|night|| +7513|AAAAAAAAKFNBAAAA|7513|2|5|13|AM|third|night|| +7514|AAAAAAAALFNBAAAA|7514|2|5|14|AM|third|night|| +7515|AAAAAAAAMFNBAAAA|7515|2|5|15|AM|third|night|| +7516|AAAAAAAANFNBAAAA|7516|2|5|16|AM|third|night|| +7517|AAAAAAAAOFNBAAAA|7517|2|5|17|AM|third|night|| +7518|AAAAAAAAPFNBAAAA|7518|2|5|18|AM|third|night|| +7519|AAAAAAAAAGNBAAAA|7519|2|5|19|AM|third|night|| +7520|AAAAAAAABGNBAAAA|7520|2|5|20|AM|third|night|| +7521|AAAAAAAACGNBAAAA|7521|2|5|21|AM|third|night|| +7522|AAAAAAAADGNBAAAA|7522|2|5|22|AM|third|night|| +7523|AAAAAAAAEGNBAAAA|7523|2|5|23|AM|third|night|| +7524|AAAAAAAAFGNBAAAA|7524|2|5|24|AM|third|night|| +7525|AAAAAAAAGGNBAAAA|7525|2|5|25|AM|third|night|| +7526|AAAAAAAAHGNBAAAA|7526|2|5|26|AM|third|night|| +7527|AAAAAAAAIGNBAAAA|7527|2|5|27|AM|third|night|| +7528|AAAAAAAAJGNBAAAA|7528|2|5|28|AM|third|night|| +7529|AAAAAAAAKGNBAAAA|7529|2|5|29|AM|third|night|| +7530|AAAAAAAALGNBAAAA|7530|2|5|30|AM|third|night|| +7531|AAAAAAAAMGNBAAAA|7531|2|5|31|AM|third|night|| +7532|AAAAAAAANGNBAAAA|7532|2|5|32|AM|third|night|| +7533|AAAAAAAAOGNBAAAA|7533|2|5|33|AM|third|night|| +7534|AAAAAAAAPGNBAAAA|7534|2|5|34|AM|third|night|| +7535|AAAAAAAAAHNBAAAA|7535|2|5|35|AM|third|night|| +7536|AAAAAAAABHNBAAAA|7536|2|5|36|AM|third|night|| +7537|AAAAAAAACHNBAAAA|7537|2|5|37|AM|third|night|| +7538|AAAAAAAADHNBAAAA|7538|2|5|38|AM|third|night|| +7539|AAAAAAAAEHNBAAAA|7539|2|5|39|AM|third|night|| +7540|AAAAAAAAFHNBAAAA|7540|2|5|40|AM|third|night|| +7541|AAAAAAAAGHNBAAAA|7541|2|5|41|AM|third|night|| +7542|AAAAAAAAHHNBAAAA|7542|2|5|42|AM|third|night|| +7543|AAAAAAAAIHNBAAAA|7543|2|5|43|AM|third|night|| +7544|AAAAAAAAJHNBAAAA|7544|2|5|44|AM|third|night|| +7545|AAAAAAAAKHNBAAAA|7545|2|5|45|AM|third|night|| +7546|AAAAAAAALHNBAAAA|7546|2|5|46|AM|third|night|| +7547|AAAAAAAAMHNBAAAA|7547|2|5|47|AM|third|night|| +7548|AAAAAAAANHNBAAAA|7548|2|5|48|AM|third|night|| +7549|AAAAAAAAOHNBAAAA|7549|2|5|49|AM|third|night|| +7550|AAAAAAAAPHNBAAAA|7550|2|5|50|AM|third|night|| +7551|AAAAAAAAAINBAAAA|7551|2|5|51|AM|third|night|| +7552|AAAAAAAABINBAAAA|7552|2|5|52|AM|third|night|| +7553|AAAAAAAACINBAAAA|7553|2|5|53|AM|third|night|| +7554|AAAAAAAADINBAAAA|7554|2|5|54|AM|third|night|| +7555|AAAAAAAAEINBAAAA|7555|2|5|55|AM|third|night|| +7556|AAAAAAAAFINBAAAA|7556|2|5|56|AM|third|night|| +7557|AAAAAAAAGINBAAAA|7557|2|5|57|AM|third|night|| +7558|AAAAAAAAHINBAAAA|7558|2|5|58|AM|third|night|| +7559|AAAAAAAAIINBAAAA|7559|2|5|59|AM|third|night|| +7560|AAAAAAAAJINBAAAA|7560|2|6|0|AM|third|night|| +7561|AAAAAAAAKINBAAAA|7561|2|6|1|AM|third|night|| +7562|AAAAAAAALINBAAAA|7562|2|6|2|AM|third|night|| +7563|AAAAAAAAMINBAAAA|7563|2|6|3|AM|third|night|| +7564|AAAAAAAANINBAAAA|7564|2|6|4|AM|third|night|| +7565|AAAAAAAAOINBAAAA|7565|2|6|5|AM|third|night|| +7566|AAAAAAAAPINBAAAA|7566|2|6|6|AM|third|night|| +7567|AAAAAAAAAJNBAAAA|7567|2|6|7|AM|third|night|| +7568|AAAAAAAABJNBAAAA|7568|2|6|8|AM|third|night|| +7569|AAAAAAAACJNBAAAA|7569|2|6|9|AM|third|night|| +7570|AAAAAAAADJNBAAAA|7570|2|6|10|AM|third|night|| +7571|AAAAAAAAEJNBAAAA|7571|2|6|11|AM|third|night|| +7572|AAAAAAAAFJNBAAAA|7572|2|6|12|AM|third|night|| +7573|AAAAAAAAGJNBAAAA|7573|2|6|13|AM|third|night|| +7574|AAAAAAAAHJNBAAAA|7574|2|6|14|AM|third|night|| +7575|AAAAAAAAIJNBAAAA|7575|2|6|15|AM|third|night|| +7576|AAAAAAAAJJNBAAAA|7576|2|6|16|AM|third|night|| +7577|AAAAAAAAKJNBAAAA|7577|2|6|17|AM|third|night|| +7578|AAAAAAAALJNBAAAA|7578|2|6|18|AM|third|night|| +7579|AAAAAAAAMJNBAAAA|7579|2|6|19|AM|third|night|| +7580|AAAAAAAANJNBAAAA|7580|2|6|20|AM|third|night|| +7581|AAAAAAAAOJNBAAAA|7581|2|6|21|AM|third|night|| +7582|AAAAAAAAPJNBAAAA|7582|2|6|22|AM|third|night|| +7583|AAAAAAAAAKNBAAAA|7583|2|6|23|AM|third|night|| +7584|AAAAAAAABKNBAAAA|7584|2|6|24|AM|third|night|| +7585|AAAAAAAACKNBAAAA|7585|2|6|25|AM|third|night|| +7586|AAAAAAAADKNBAAAA|7586|2|6|26|AM|third|night|| +7587|AAAAAAAAEKNBAAAA|7587|2|6|27|AM|third|night|| +7588|AAAAAAAAFKNBAAAA|7588|2|6|28|AM|third|night|| +7589|AAAAAAAAGKNBAAAA|7589|2|6|29|AM|third|night|| +7590|AAAAAAAAHKNBAAAA|7590|2|6|30|AM|third|night|| +7591|AAAAAAAAIKNBAAAA|7591|2|6|31|AM|third|night|| +7592|AAAAAAAAJKNBAAAA|7592|2|6|32|AM|third|night|| +7593|AAAAAAAAKKNBAAAA|7593|2|6|33|AM|third|night|| +7594|AAAAAAAALKNBAAAA|7594|2|6|34|AM|third|night|| +7595|AAAAAAAAMKNBAAAA|7595|2|6|35|AM|third|night|| +7596|AAAAAAAANKNBAAAA|7596|2|6|36|AM|third|night|| +7597|AAAAAAAAOKNBAAAA|7597|2|6|37|AM|third|night|| +7598|AAAAAAAAPKNBAAAA|7598|2|6|38|AM|third|night|| +7599|AAAAAAAAALNBAAAA|7599|2|6|39|AM|third|night|| +7600|AAAAAAAABLNBAAAA|7600|2|6|40|AM|third|night|| +7601|AAAAAAAACLNBAAAA|7601|2|6|41|AM|third|night|| +7602|AAAAAAAADLNBAAAA|7602|2|6|42|AM|third|night|| +7603|AAAAAAAAELNBAAAA|7603|2|6|43|AM|third|night|| +7604|AAAAAAAAFLNBAAAA|7604|2|6|44|AM|third|night|| +7605|AAAAAAAAGLNBAAAA|7605|2|6|45|AM|third|night|| +7606|AAAAAAAAHLNBAAAA|7606|2|6|46|AM|third|night|| +7607|AAAAAAAAILNBAAAA|7607|2|6|47|AM|third|night|| +7608|AAAAAAAAJLNBAAAA|7608|2|6|48|AM|third|night|| +7609|AAAAAAAAKLNBAAAA|7609|2|6|49|AM|third|night|| +7610|AAAAAAAALLNBAAAA|7610|2|6|50|AM|third|night|| +7611|AAAAAAAAMLNBAAAA|7611|2|6|51|AM|third|night|| +7612|AAAAAAAANLNBAAAA|7612|2|6|52|AM|third|night|| +7613|AAAAAAAAOLNBAAAA|7613|2|6|53|AM|third|night|| +7614|AAAAAAAAPLNBAAAA|7614|2|6|54|AM|third|night|| +7615|AAAAAAAAAMNBAAAA|7615|2|6|55|AM|third|night|| +7616|AAAAAAAABMNBAAAA|7616|2|6|56|AM|third|night|| +7617|AAAAAAAACMNBAAAA|7617|2|6|57|AM|third|night|| +7618|AAAAAAAADMNBAAAA|7618|2|6|58|AM|third|night|| +7619|AAAAAAAAEMNBAAAA|7619|2|6|59|AM|third|night|| +7620|AAAAAAAAFMNBAAAA|7620|2|7|0|AM|third|night|| +7621|AAAAAAAAGMNBAAAA|7621|2|7|1|AM|third|night|| +7622|AAAAAAAAHMNBAAAA|7622|2|7|2|AM|third|night|| +7623|AAAAAAAAIMNBAAAA|7623|2|7|3|AM|third|night|| +7624|AAAAAAAAJMNBAAAA|7624|2|7|4|AM|third|night|| +7625|AAAAAAAAKMNBAAAA|7625|2|7|5|AM|third|night|| +7626|AAAAAAAALMNBAAAA|7626|2|7|6|AM|third|night|| +7627|AAAAAAAAMMNBAAAA|7627|2|7|7|AM|third|night|| +7628|AAAAAAAANMNBAAAA|7628|2|7|8|AM|third|night|| +7629|AAAAAAAAOMNBAAAA|7629|2|7|9|AM|third|night|| +7630|AAAAAAAAPMNBAAAA|7630|2|7|10|AM|third|night|| +7631|AAAAAAAAANNBAAAA|7631|2|7|11|AM|third|night|| +7632|AAAAAAAABNNBAAAA|7632|2|7|12|AM|third|night|| +7633|AAAAAAAACNNBAAAA|7633|2|7|13|AM|third|night|| +7634|AAAAAAAADNNBAAAA|7634|2|7|14|AM|third|night|| +7635|AAAAAAAAENNBAAAA|7635|2|7|15|AM|third|night|| +7636|AAAAAAAAFNNBAAAA|7636|2|7|16|AM|third|night|| +7637|AAAAAAAAGNNBAAAA|7637|2|7|17|AM|third|night|| +7638|AAAAAAAAHNNBAAAA|7638|2|7|18|AM|third|night|| +7639|AAAAAAAAINNBAAAA|7639|2|7|19|AM|third|night|| +7640|AAAAAAAAJNNBAAAA|7640|2|7|20|AM|third|night|| +7641|AAAAAAAAKNNBAAAA|7641|2|7|21|AM|third|night|| +7642|AAAAAAAALNNBAAAA|7642|2|7|22|AM|third|night|| +7643|AAAAAAAAMNNBAAAA|7643|2|7|23|AM|third|night|| +7644|AAAAAAAANNNBAAAA|7644|2|7|24|AM|third|night|| +7645|AAAAAAAAONNBAAAA|7645|2|7|25|AM|third|night|| +7646|AAAAAAAAPNNBAAAA|7646|2|7|26|AM|third|night|| +7647|AAAAAAAAAONBAAAA|7647|2|7|27|AM|third|night|| +7648|AAAAAAAABONBAAAA|7648|2|7|28|AM|third|night|| +7649|AAAAAAAACONBAAAA|7649|2|7|29|AM|third|night|| +7650|AAAAAAAADONBAAAA|7650|2|7|30|AM|third|night|| +7651|AAAAAAAAEONBAAAA|7651|2|7|31|AM|third|night|| +7652|AAAAAAAAFONBAAAA|7652|2|7|32|AM|third|night|| +7653|AAAAAAAAGONBAAAA|7653|2|7|33|AM|third|night|| +7654|AAAAAAAAHONBAAAA|7654|2|7|34|AM|third|night|| +7655|AAAAAAAAIONBAAAA|7655|2|7|35|AM|third|night|| +7656|AAAAAAAAJONBAAAA|7656|2|7|36|AM|third|night|| +7657|AAAAAAAAKONBAAAA|7657|2|7|37|AM|third|night|| +7658|AAAAAAAALONBAAAA|7658|2|7|38|AM|third|night|| +7659|AAAAAAAAMONBAAAA|7659|2|7|39|AM|third|night|| +7660|AAAAAAAANONBAAAA|7660|2|7|40|AM|third|night|| +7661|AAAAAAAAOONBAAAA|7661|2|7|41|AM|third|night|| +7662|AAAAAAAAPONBAAAA|7662|2|7|42|AM|third|night|| +7663|AAAAAAAAAPNBAAAA|7663|2|7|43|AM|third|night|| +7664|AAAAAAAABPNBAAAA|7664|2|7|44|AM|third|night|| +7665|AAAAAAAACPNBAAAA|7665|2|7|45|AM|third|night|| +7666|AAAAAAAADPNBAAAA|7666|2|7|46|AM|third|night|| +7667|AAAAAAAAEPNBAAAA|7667|2|7|47|AM|third|night|| +7668|AAAAAAAAFPNBAAAA|7668|2|7|48|AM|third|night|| +7669|AAAAAAAAGPNBAAAA|7669|2|7|49|AM|third|night|| +7670|AAAAAAAAHPNBAAAA|7670|2|7|50|AM|third|night|| +7671|AAAAAAAAIPNBAAAA|7671|2|7|51|AM|third|night|| +7672|AAAAAAAAJPNBAAAA|7672|2|7|52|AM|third|night|| +7673|AAAAAAAAKPNBAAAA|7673|2|7|53|AM|third|night|| +7674|AAAAAAAALPNBAAAA|7674|2|7|54|AM|third|night|| +7675|AAAAAAAAMPNBAAAA|7675|2|7|55|AM|third|night|| +7676|AAAAAAAANPNBAAAA|7676|2|7|56|AM|third|night|| +7677|AAAAAAAAOPNBAAAA|7677|2|7|57|AM|third|night|| +7678|AAAAAAAAPPNBAAAA|7678|2|7|58|AM|third|night|| +7679|AAAAAAAAAAOBAAAA|7679|2|7|59|AM|third|night|| +7680|AAAAAAAABAOBAAAA|7680|2|8|0|AM|third|night|| +7681|AAAAAAAACAOBAAAA|7681|2|8|1|AM|third|night|| +7682|AAAAAAAADAOBAAAA|7682|2|8|2|AM|third|night|| +7683|AAAAAAAAEAOBAAAA|7683|2|8|3|AM|third|night|| +7684|AAAAAAAAFAOBAAAA|7684|2|8|4|AM|third|night|| +7685|AAAAAAAAGAOBAAAA|7685|2|8|5|AM|third|night|| +7686|AAAAAAAAHAOBAAAA|7686|2|8|6|AM|third|night|| +7687|AAAAAAAAIAOBAAAA|7687|2|8|7|AM|third|night|| +7688|AAAAAAAAJAOBAAAA|7688|2|8|8|AM|third|night|| +7689|AAAAAAAAKAOBAAAA|7689|2|8|9|AM|third|night|| +7690|AAAAAAAALAOBAAAA|7690|2|8|10|AM|third|night|| +7691|AAAAAAAAMAOBAAAA|7691|2|8|11|AM|third|night|| +7692|AAAAAAAANAOBAAAA|7692|2|8|12|AM|third|night|| +7693|AAAAAAAAOAOBAAAA|7693|2|8|13|AM|third|night|| +7694|AAAAAAAAPAOBAAAA|7694|2|8|14|AM|third|night|| +7695|AAAAAAAAABOBAAAA|7695|2|8|15|AM|third|night|| +7696|AAAAAAAABBOBAAAA|7696|2|8|16|AM|third|night|| +7697|AAAAAAAACBOBAAAA|7697|2|8|17|AM|third|night|| +7698|AAAAAAAADBOBAAAA|7698|2|8|18|AM|third|night|| +7699|AAAAAAAAEBOBAAAA|7699|2|8|19|AM|third|night|| +7700|AAAAAAAAFBOBAAAA|7700|2|8|20|AM|third|night|| +7701|AAAAAAAAGBOBAAAA|7701|2|8|21|AM|third|night|| +7702|AAAAAAAAHBOBAAAA|7702|2|8|22|AM|third|night|| +7703|AAAAAAAAIBOBAAAA|7703|2|8|23|AM|third|night|| +7704|AAAAAAAAJBOBAAAA|7704|2|8|24|AM|third|night|| +7705|AAAAAAAAKBOBAAAA|7705|2|8|25|AM|third|night|| +7706|AAAAAAAALBOBAAAA|7706|2|8|26|AM|third|night|| +7707|AAAAAAAAMBOBAAAA|7707|2|8|27|AM|third|night|| +7708|AAAAAAAANBOBAAAA|7708|2|8|28|AM|third|night|| +7709|AAAAAAAAOBOBAAAA|7709|2|8|29|AM|third|night|| +7710|AAAAAAAAPBOBAAAA|7710|2|8|30|AM|third|night|| +7711|AAAAAAAAACOBAAAA|7711|2|8|31|AM|third|night|| +7712|AAAAAAAABCOBAAAA|7712|2|8|32|AM|third|night|| +7713|AAAAAAAACCOBAAAA|7713|2|8|33|AM|third|night|| +7714|AAAAAAAADCOBAAAA|7714|2|8|34|AM|third|night|| +7715|AAAAAAAAECOBAAAA|7715|2|8|35|AM|third|night|| +7716|AAAAAAAAFCOBAAAA|7716|2|8|36|AM|third|night|| +7717|AAAAAAAAGCOBAAAA|7717|2|8|37|AM|third|night|| +7718|AAAAAAAAHCOBAAAA|7718|2|8|38|AM|third|night|| +7719|AAAAAAAAICOBAAAA|7719|2|8|39|AM|third|night|| +7720|AAAAAAAAJCOBAAAA|7720|2|8|40|AM|third|night|| +7721|AAAAAAAAKCOBAAAA|7721|2|8|41|AM|third|night|| +7722|AAAAAAAALCOBAAAA|7722|2|8|42|AM|third|night|| +7723|AAAAAAAAMCOBAAAA|7723|2|8|43|AM|third|night|| +7724|AAAAAAAANCOBAAAA|7724|2|8|44|AM|third|night|| +7725|AAAAAAAAOCOBAAAA|7725|2|8|45|AM|third|night|| +7726|AAAAAAAAPCOBAAAA|7726|2|8|46|AM|third|night|| +7727|AAAAAAAAADOBAAAA|7727|2|8|47|AM|third|night|| +7728|AAAAAAAABDOBAAAA|7728|2|8|48|AM|third|night|| +7729|AAAAAAAACDOBAAAA|7729|2|8|49|AM|third|night|| +7730|AAAAAAAADDOBAAAA|7730|2|8|50|AM|third|night|| +7731|AAAAAAAAEDOBAAAA|7731|2|8|51|AM|third|night|| +7732|AAAAAAAAFDOBAAAA|7732|2|8|52|AM|third|night|| +7733|AAAAAAAAGDOBAAAA|7733|2|8|53|AM|third|night|| +7734|AAAAAAAAHDOBAAAA|7734|2|8|54|AM|third|night|| +7735|AAAAAAAAIDOBAAAA|7735|2|8|55|AM|third|night|| +7736|AAAAAAAAJDOBAAAA|7736|2|8|56|AM|third|night|| +7737|AAAAAAAAKDOBAAAA|7737|2|8|57|AM|third|night|| +7738|AAAAAAAALDOBAAAA|7738|2|8|58|AM|third|night|| +7739|AAAAAAAAMDOBAAAA|7739|2|8|59|AM|third|night|| +7740|AAAAAAAANDOBAAAA|7740|2|9|0|AM|third|night|| +7741|AAAAAAAAODOBAAAA|7741|2|9|1|AM|third|night|| +7742|AAAAAAAAPDOBAAAA|7742|2|9|2|AM|third|night|| +7743|AAAAAAAAAEOBAAAA|7743|2|9|3|AM|third|night|| +7744|AAAAAAAABEOBAAAA|7744|2|9|4|AM|third|night|| +7745|AAAAAAAACEOBAAAA|7745|2|9|5|AM|third|night|| +7746|AAAAAAAADEOBAAAA|7746|2|9|6|AM|third|night|| +7747|AAAAAAAAEEOBAAAA|7747|2|9|7|AM|third|night|| +7748|AAAAAAAAFEOBAAAA|7748|2|9|8|AM|third|night|| +7749|AAAAAAAAGEOBAAAA|7749|2|9|9|AM|third|night|| +7750|AAAAAAAAHEOBAAAA|7750|2|9|10|AM|third|night|| +7751|AAAAAAAAIEOBAAAA|7751|2|9|11|AM|third|night|| +7752|AAAAAAAAJEOBAAAA|7752|2|9|12|AM|third|night|| +7753|AAAAAAAAKEOBAAAA|7753|2|9|13|AM|third|night|| +7754|AAAAAAAALEOBAAAA|7754|2|9|14|AM|third|night|| +7755|AAAAAAAAMEOBAAAA|7755|2|9|15|AM|third|night|| +7756|AAAAAAAANEOBAAAA|7756|2|9|16|AM|third|night|| +7757|AAAAAAAAOEOBAAAA|7757|2|9|17|AM|third|night|| +7758|AAAAAAAAPEOBAAAA|7758|2|9|18|AM|third|night|| +7759|AAAAAAAAAFOBAAAA|7759|2|9|19|AM|third|night|| +7760|AAAAAAAABFOBAAAA|7760|2|9|20|AM|third|night|| +7761|AAAAAAAACFOBAAAA|7761|2|9|21|AM|third|night|| +7762|AAAAAAAADFOBAAAA|7762|2|9|22|AM|third|night|| +7763|AAAAAAAAEFOBAAAA|7763|2|9|23|AM|third|night|| +7764|AAAAAAAAFFOBAAAA|7764|2|9|24|AM|third|night|| +7765|AAAAAAAAGFOBAAAA|7765|2|9|25|AM|third|night|| +7766|AAAAAAAAHFOBAAAA|7766|2|9|26|AM|third|night|| +7767|AAAAAAAAIFOBAAAA|7767|2|9|27|AM|third|night|| +7768|AAAAAAAAJFOBAAAA|7768|2|9|28|AM|third|night|| +7769|AAAAAAAAKFOBAAAA|7769|2|9|29|AM|third|night|| +7770|AAAAAAAALFOBAAAA|7770|2|9|30|AM|third|night|| +7771|AAAAAAAAMFOBAAAA|7771|2|9|31|AM|third|night|| +7772|AAAAAAAANFOBAAAA|7772|2|9|32|AM|third|night|| +7773|AAAAAAAAOFOBAAAA|7773|2|9|33|AM|third|night|| +7774|AAAAAAAAPFOBAAAA|7774|2|9|34|AM|third|night|| +7775|AAAAAAAAAGOBAAAA|7775|2|9|35|AM|third|night|| +7776|AAAAAAAABGOBAAAA|7776|2|9|36|AM|third|night|| +7777|AAAAAAAACGOBAAAA|7777|2|9|37|AM|third|night|| +7778|AAAAAAAADGOBAAAA|7778|2|9|38|AM|third|night|| +7779|AAAAAAAAEGOBAAAA|7779|2|9|39|AM|third|night|| +7780|AAAAAAAAFGOBAAAA|7780|2|9|40|AM|third|night|| +7781|AAAAAAAAGGOBAAAA|7781|2|9|41|AM|third|night|| +7782|AAAAAAAAHGOBAAAA|7782|2|9|42|AM|third|night|| +7783|AAAAAAAAIGOBAAAA|7783|2|9|43|AM|third|night|| +7784|AAAAAAAAJGOBAAAA|7784|2|9|44|AM|third|night|| +7785|AAAAAAAAKGOBAAAA|7785|2|9|45|AM|third|night|| +7786|AAAAAAAALGOBAAAA|7786|2|9|46|AM|third|night|| +7787|AAAAAAAAMGOBAAAA|7787|2|9|47|AM|third|night|| +7788|AAAAAAAANGOBAAAA|7788|2|9|48|AM|third|night|| +7789|AAAAAAAAOGOBAAAA|7789|2|9|49|AM|third|night|| +7790|AAAAAAAAPGOBAAAA|7790|2|9|50|AM|third|night|| +7791|AAAAAAAAAHOBAAAA|7791|2|9|51|AM|third|night|| +7792|AAAAAAAABHOBAAAA|7792|2|9|52|AM|third|night|| +7793|AAAAAAAACHOBAAAA|7793|2|9|53|AM|third|night|| +7794|AAAAAAAADHOBAAAA|7794|2|9|54|AM|third|night|| +7795|AAAAAAAAEHOBAAAA|7795|2|9|55|AM|third|night|| +7796|AAAAAAAAFHOBAAAA|7796|2|9|56|AM|third|night|| +7797|AAAAAAAAGHOBAAAA|7797|2|9|57|AM|third|night|| +7798|AAAAAAAAHHOBAAAA|7798|2|9|58|AM|third|night|| +7799|AAAAAAAAIHOBAAAA|7799|2|9|59|AM|third|night|| +7800|AAAAAAAAJHOBAAAA|7800|2|10|0|AM|third|night|| +7801|AAAAAAAAKHOBAAAA|7801|2|10|1|AM|third|night|| +7802|AAAAAAAALHOBAAAA|7802|2|10|2|AM|third|night|| +7803|AAAAAAAAMHOBAAAA|7803|2|10|3|AM|third|night|| +7804|AAAAAAAANHOBAAAA|7804|2|10|4|AM|third|night|| +7805|AAAAAAAAOHOBAAAA|7805|2|10|5|AM|third|night|| +7806|AAAAAAAAPHOBAAAA|7806|2|10|6|AM|third|night|| +7807|AAAAAAAAAIOBAAAA|7807|2|10|7|AM|third|night|| +7808|AAAAAAAABIOBAAAA|7808|2|10|8|AM|third|night|| +7809|AAAAAAAACIOBAAAA|7809|2|10|9|AM|third|night|| +7810|AAAAAAAADIOBAAAA|7810|2|10|10|AM|third|night|| +7811|AAAAAAAAEIOBAAAA|7811|2|10|11|AM|third|night|| +7812|AAAAAAAAFIOBAAAA|7812|2|10|12|AM|third|night|| +7813|AAAAAAAAGIOBAAAA|7813|2|10|13|AM|third|night|| +7814|AAAAAAAAHIOBAAAA|7814|2|10|14|AM|third|night|| +7815|AAAAAAAAIIOBAAAA|7815|2|10|15|AM|third|night|| +7816|AAAAAAAAJIOBAAAA|7816|2|10|16|AM|third|night|| +7817|AAAAAAAAKIOBAAAA|7817|2|10|17|AM|third|night|| +7818|AAAAAAAALIOBAAAA|7818|2|10|18|AM|third|night|| +7819|AAAAAAAAMIOBAAAA|7819|2|10|19|AM|third|night|| +7820|AAAAAAAANIOBAAAA|7820|2|10|20|AM|third|night|| +7821|AAAAAAAAOIOBAAAA|7821|2|10|21|AM|third|night|| +7822|AAAAAAAAPIOBAAAA|7822|2|10|22|AM|third|night|| +7823|AAAAAAAAAJOBAAAA|7823|2|10|23|AM|third|night|| +7824|AAAAAAAABJOBAAAA|7824|2|10|24|AM|third|night|| +7825|AAAAAAAACJOBAAAA|7825|2|10|25|AM|third|night|| +7826|AAAAAAAADJOBAAAA|7826|2|10|26|AM|third|night|| +7827|AAAAAAAAEJOBAAAA|7827|2|10|27|AM|third|night|| +7828|AAAAAAAAFJOBAAAA|7828|2|10|28|AM|third|night|| +7829|AAAAAAAAGJOBAAAA|7829|2|10|29|AM|third|night|| +7830|AAAAAAAAHJOBAAAA|7830|2|10|30|AM|third|night|| +7831|AAAAAAAAIJOBAAAA|7831|2|10|31|AM|third|night|| +7832|AAAAAAAAJJOBAAAA|7832|2|10|32|AM|third|night|| +7833|AAAAAAAAKJOBAAAA|7833|2|10|33|AM|third|night|| +7834|AAAAAAAALJOBAAAA|7834|2|10|34|AM|third|night|| +7835|AAAAAAAAMJOBAAAA|7835|2|10|35|AM|third|night|| +7836|AAAAAAAANJOBAAAA|7836|2|10|36|AM|third|night|| +7837|AAAAAAAAOJOBAAAA|7837|2|10|37|AM|third|night|| +7838|AAAAAAAAPJOBAAAA|7838|2|10|38|AM|third|night|| +7839|AAAAAAAAAKOBAAAA|7839|2|10|39|AM|third|night|| +7840|AAAAAAAABKOBAAAA|7840|2|10|40|AM|third|night|| +7841|AAAAAAAACKOBAAAA|7841|2|10|41|AM|third|night|| +7842|AAAAAAAADKOBAAAA|7842|2|10|42|AM|third|night|| +7843|AAAAAAAAEKOBAAAA|7843|2|10|43|AM|third|night|| +7844|AAAAAAAAFKOBAAAA|7844|2|10|44|AM|third|night|| +7845|AAAAAAAAGKOBAAAA|7845|2|10|45|AM|third|night|| +7846|AAAAAAAAHKOBAAAA|7846|2|10|46|AM|third|night|| +7847|AAAAAAAAIKOBAAAA|7847|2|10|47|AM|third|night|| +7848|AAAAAAAAJKOBAAAA|7848|2|10|48|AM|third|night|| +7849|AAAAAAAAKKOBAAAA|7849|2|10|49|AM|third|night|| +7850|AAAAAAAALKOBAAAA|7850|2|10|50|AM|third|night|| +7851|AAAAAAAAMKOBAAAA|7851|2|10|51|AM|third|night|| +7852|AAAAAAAANKOBAAAA|7852|2|10|52|AM|third|night|| +7853|AAAAAAAAOKOBAAAA|7853|2|10|53|AM|third|night|| +7854|AAAAAAAAPKOBAAAA|7854|2|10|54|AM|third|night|| +7855|AAAAAAAAALOBAAAA|7855|2|10|55|AM|third|night|| +7856|AAAAAAAABLOBAAAA|7856|2|10|56|AM|third|night|| +7857|AAAAAAAACLOBAAAA|7857|2|10|57|AM|third|night|| +7858|AAAAAAAADLOBAAAA|7858|2|10|58|AM|third|night|| +7859|AAAAAAAAELOBAAAA|7859|2|10|59|AM|third|night|| +7860|AAAAAAAAFLOBAAAA|7860|2|11|0|AM|third|night|| +7861|AAAAAAAAGLOBAAAA|7861|2|11|1|AM|third|night|| +7862|AAAAAAAAHLOBAAAA|7862|2|11|2|AM|third|night|| +7863|AAAAAAAAILOBAAAA|7863|2|11|3|AM|third|night|| +7864|AAAAAAAAJLOBAAAA|7864|2|11|4|AM|third|night|| +7865|AAAAAAAAKLOBAAAA|7865|2|11|5|AM|third|night|| +7866|AAAAAAAALLOBAAAA|7866|2|11|6|AM|third|night|| +7867|AAAAAAAAMLOBAAAA|7867|2|11|7|AM|third|night|| +7868|AAAAAAAANLOBAAAA|7868|2|11|8|AM|third|night|| +7869|AAAAAAAAOLOBAAAA|7869|2|11|9|AM|third|night|| +7870|AAAAAAAAPLOBAAAA|7870|2|11|10|AM|third|night|| +7871|AAAAAAAAAMOBAAAA|7871|2|11|11|AM|third|night|| +7872|AAAAAAAABMOBAAAA|7872|2|11|12|AM|third|night|| +7873|AAAAAAAACMOBAAAA|7873|2|11|13|AM|third|night|| +7874|AAAAAAAADMOBAAAA|7874|2|11|14|AM|third|night|| +7875|AAAAAAAAEMOBAAAA|7875|2|11|15|AM|third|night|| +7876|AAAAAAAAFMOBAAAA|7876|2|11|16|AM|third|night|| +7877|AAAAAAAAGMOBAAAA|7877|2|11|17|AM|third|night|| +7878|AAAAAAAAHMOBAAAA|7878|2|11|18|AM|third|night|| +7879|AAAAAAAAIMOBAAAA|7879|2|11|19|AM|third|night|| +7880|AAAAAAAAJMOBAAAA|7880|2|11|20|AM|third|night|| +7881|AAAAAAAAKMOBAAAA|7881|2|11|21|AM|third|night|| +7882|AAAAAAAALMOBAAAA|7882|2|11|22|AM|third|night|| +7883|AAAAAAAAMMOBAAAA|7883|2|11|23|AM|third|night|| +7884|AAAAAAAANMOBAAAA|7884|2|11|24|AM|third|night|| +7885|AAAAAAAAOMOBAAAA|7885|2|11|25|AM|third|night|| +7886|AAAAAAAAPMOBAAAA|7886|2|11|26|AM|third|night|| +7887|AAAAAAAAANOBAAAA|7887|2|11|27|AM|third|night|| +7888|AAAAAAAABNOBAAAA|7888|2|11|28|AM|third|night|| +7889|AAAAAAAACNOBAAAA|7889|2|11|29|AM|third|night|| +7890|AAAAAAAADNOBAAAA|7890|2|11|30|AM|third|night|| +7891|AAAAAAAAENOBAAAA|7891|2|11|31|AM|third|night|| +7892|AAAAAAAAFNOBAAAA|7892|2|11|32|AM|third|night|| +7893|AAAAAAAAGNOBAAAA|7893|2|11|33|AM|third|night|| +7894|AAAAAAAAHNOBAAAA|7894|2|11|34|AM|third|night|| +7895|AAAAAAAAINOBAAAA|7895|2|11|35|AM|third|night|| +7896|AAAAAAAAJNOBAAAA|7896|2|11|36|AM|third|night|| +7897|AAAAAAAAKNOBAAAA|7897|2|11|37|AM|third|night|| +7898|AAAAAAAALNOBAAAA|7898|2|11|38|AM|third|night|| +7899|AAAAAAAAMNOBAAAA|7899|2|11|39|AM|third|night|| +7900|AAAAAAAANNOBAAAA|7900|2|11|40|AM|third|night|| +7901|AAAAAAAAONOBAAAA|7901|2|11|41|AM|third|night|| +7902|AAAAAAAAPNOBAAAA|7902|2|11|42|AM|third|night|| +7903|AAAAAAAAAOOBAAAA|7903|2|11|43|AM|third|night|| +7904|AAAAAAAABOOBAAAA|7904|2|11|44|AM|third|night|| +7905|AAAAAAAACOOBAAAA|7905|2|11|45|AM|third|night|| +7906|AAAAAAAADOOBAAAA|7906|2|11|46|AM|third|night|| +7907|AAAAAAAAEOOBAAAA|7907|2|11|47|AM|third|night|| +7908|AAAAAAAAFOOBAAAA|7908|2|11|48|AM|third|night|| +7909|AAAAAAAAGOOBAAAA|7909|2|11|49|AM|third|night|| +7910|AAAAAAAAHOOBAAAA|7910|2|11|50|AM|third|night|| +7911|AAAAAAAAIOOBAAAA|7911|2|11|51|AM|third|night|| +7912|AAAAAAAAJOOBAAAA|7912|2|11|52|AM|third|night|| +7913|AAAAAAAAKOOBAAAA|7913|2|11|53|AM|third|night|| +7914|AAAAAAAALOOBAAAA|7914|2|11|54|AM|third|night|| +7915|AAAAAAAAMOOBAAAA|7915|2|11|55|AM|third|night|| +7916|AAAAAAAANOOBAAAA|7916|2|11|56|AM|third|night|| +7917|AAAAAAAAOOOBAAAA|7917|2|11|57|AM|third|night|| +7918|AAAAAAAAPOOBAAAA|7918|2|11|58|AM|third|night|| +7919|AAAAAAAAAPOBAAAA|7919|2|11|59|AM|third|night|| +7920|AAAAAAAABPOBAAAA|7920|2|12|0|AM|third|night|| +7921|AAAAAAAACPOBAAAA|7921|2|12|1|AM|third|night|| +7922|AAAAAAAADPOBAAAA|7922|2|12|2|AM|third|night|| +7923|AAAAAAAAEPOBAAAA|7923|2|12|3|AM|third|night|| +7924|AAAAAAAAFPOBAAAA|7924|2|12|4|AM|third|night|| +7925|AAAAAAAAGPOBAAAA|7925|2|12|5|AM|third|night|| +7926|AAAAAAAAHPOBAAAA|7926|2|12|6|AM|third|night|| +7927|AAAAAAAAIPOBAAAA|7927|2|12|7|AM|third|night|| +7928|AAAAAAAAJPOBAAAA|7928|2|12|8|AM|third|night|| +7929|AAAAAAAAKPOBAAAA|7929|2|12|9|AM|third|night|| +7930|AAAAAAAALPOBAAAA|7930|2|12|10|AM|third|night|| +7931|AAAAAAAAMPOBAAAA|7931|2|12|11|AM|third|night|| +7932|AAAAAAAANPOBAAAA|7932|2|12|12|AM|third|night|| +7933|AAAAAAAAOPOBAAAA|7933|2|12|13|AM|third|night|| +7934|AAAAAAAAPPOBAAAA|7934|2|12|14|AM|third|night|| +7935|AAAAAAAAAAPBAAAA|7935|2|12|15|AM|third|night|| +7936|AAAAAAAABAPBAAAA|7936|2|12|16|AM|third|night|| +7937|AAAAAAAACAPBAAAA|7937|2|12|17|AM|third|night|| +7938|AAAAAAAADAPBAAAA|7938|2|12|18|AM|third|night|| +7939|AAAAAAAAEAPBAAAA|7939|2|12|19|AM|third|night|| +7940|AAAAAAAAFAPBAAAA|7940|2|12|20|AM|third|night|| +7941|AAAAAAAAGAPBAAAA|7941|2|12|21|AM|third|night|| +7942|AAAAAAAAHAPBAAAA|7942|2|12|22|AM|third|night|| +7943|AAAAAAAAIAPBAAAA|7943|2|12|23|AM|third|night|| +7944|AAAAAAAAJAPBAAAA|7944|2|12|24|AM|third|night|| +7945|AAAAAAAAKAPBAAAA|7945|2|12|25|AM|third|night|| +7946|AAAAAAAALAPBAAAA|7946|2|12|26|AM|third|night|| +7947|AAAAAAAAMAPBAAAA|7947|2|12|27|AM|third|night|| +7948|AAAAAAAANAPBAAAA|7948|2|12|28|AM|third|night|| +7949|AAAAAAAAOAPBAAAA|7949|2|12|29|AM|third|night|| +7950|AAAAAAAAPAPBAAAA|7950|2|12|30|AM|third|night|| +7951|AAAAAAAAABPBAAAA|7951|2|12|31|AM|third|night|| +7952|AAAAAAAABBPBAAAA|7952|2|12|32|AM|third|night|| +7953|AAAAAAAACBPBAAAA|7953|2|12|33|AM|third|night|| +7954|AAAAAAAADBPBAAAA|7954|2|12|34|AM|third|night|| +7955|AAAAAAAAEBPBAAAA|7955|2|12|35|AM|third|night|| +7956|AAAAAAAAFBPBAAAA|7956|2|12|36|AM|third|night|| +7957|AAAAAAAAGBPBAAAA|7957|2|12|37|AM|third|night|| +7958|AAAAAAAAHBPBAAAA|7958|2|12|38|AM|third|night|| +7959|AAAAAAAAIBPBAAAA|7959|2|12|39|AM|third|night|| +7960|AAAAAAAAJBPBAAAA|7960|2|12|40|AM|third|night|| +7961|AAAAAAAAKBPBAAAA|7961|2|12|41|AM|third|night|| +7962|AAAAAAAALBPBAAAA|7962|2|12|42|AM|third|night|| +7963|AAAAAAAAMBPBAAAA|7963|2|12|43|AM|third|night|| +7964|AAAAAAAANBPBAAAA|7964|2|12|44|AM|third|night|| +7965|AAAAAAAAOBPBAAAA|7965|2|12|45|AM|third|night|| +7966|AAAAAAAAPBPBAAAA|7966|2|12|46|AM|third|night|| +7967|AAAAAAAAACPBAAAA|7967|2|12|47|AM|third|night|| +7968|AAAAAAAABCPBAAAA|7968|2|12|48|AM|third|night|| +7969|AAAAAAAACCPBAAAA|7969|2|12|49|AM|third|night|| +7970|AAAAAAAADCPBAAAA|7970|2|12|50|AM|third|night|| +7971|AAAAAAAAECPBAAAA|7971|2|12|51|AM|third|night|| +7972|AAAAAAAAFCPBAAAA|7972|2|12|52|AM|third|night|| +7973|AAAAAAAAGCPBAAAA|7973|2|12|53|AM|third|night|| +7974|AAAAAAAAHCPBAAAA|7974|2|12|54|AM|third|night|| +7975|AAAAAAAAICPBAAAA|7975|2|12|55|AM|third|night|| +7976|AAAAAAAAJCPBAAAA|7976|2|12|56|AM|third|night|| +7977|AAAAAAAAKCPBAAAA|7977|2|12|57|AM|third|night|| +7978|AAAAAAAALCPBAAAA|7978|2|12|58|AM|third|night|| +7979|AAAAAAAAMCPBAAAA|7979|2|12|59|AM|third|night|| +7980|AAAAAAAANCPBAAAA|7980|2|13|0|AM|third|night|| +7981|AAAAAAAAOCPBAAAA|7981|2|13|1|AM|third|night|| +7982|AAAAAAAAPCPBAAAA|7982|2|13|2|AM|third|night|| +7983|AAAAAAAAADPBAAAA|7983|2|13|3|AM|third|night|| +7984|AAAAAAAABDPBAAAA|7984|2|13|4|AM|third|night|| +7985|AAAAAAAACDPBAAAA|7985|2|13|5|AM|third|night|| +7986|AAAAAAAADDPBAAAA|7986|2|13|6|AM|third|night|| +7987|AAAAAAAAEDPBAAAA|7987|2|13|7|AM|third|night|| +7988|AAAAAAAAFDPBAAAA|7988|2|13|8|AM|third|night|| +7989|AAAAAAAAGDPBAAAA|7989|2|13|9|AM|third|night|| +7990|AAAAAAAAHDPBAAAA|7990|2|13|10|AM|third|night|| +7991|AAAAAAAAIDPBAAAA|7991|2|13|11|AM|third|night|| +7992|AAAAAAAAJDPBAAAA|7992|2|13|12|AM|third|night|| +7993|AAAAAAAAKDPBAAAA|7993|2|13|13|AM|third|night|| +7994|AAAAAAAALDPBAAAA|7994|2|13|14|AM|third|night|| +7995|AAAAAAAAMDPBAAAA|7995|2|13|15|AM|third|night|| +7996|AAAAAAAANDPBAAAA|7996|2|13|16|AM|third|night|| +7997|AAAAAAAAODPBAAAA|7997|2|13|17|AM|third|night|| +7998|AAAAAAAAPDPBAAAA|7998|2|13|18|AM|third|night|| +7999|AAAAAAAAAEPBAAAA|7999|2|13|19|AM|third|night|| +8000|AAAAAAAABEPBAAAA|8000|2|13|20|AM|third|night|| +8001|AAAAAAAACEPBAAAA|8001|2|13|21|AM|third|night|| +8002|AAAAAAAADEPBAAAA|8002|2|13|22|AM|third|night|| +8003|AAAAAAAAEEPBAAAA|8003|2|13|23|AM|third|night|| +8004|AAAAAAAAFEPBAAAA|8004|2|13|24|AM|third|night|| +8005|AAAAAAAAGEPBAAAA|8005|2|13|25|AM|third|night|| +8006|AAAAAAAAHEPBAAAA|8006|2|13|26|AM|third|night|| +8007|AAAAAAAAIEPBAAAA|8007|2|13|27|AM|third|night|| +8008|AAAAAAAAJEPBAAAA|8008|2|13|28|AM|third|night|| +8009|AAAAAAAAKEPBAAAA|8009|2|13|29|AM|third|night|| +8010|AAAAAAAALEPBAAAA|8010|2|13|30|AM|third|night|| +8011|AAAAAAAAMEPBAAAA|8011|2|13|31|AM|third|night|| +8012|AAAAAAAANEPBAAAA|8012|2|13|32|AM|third|night|| +8013|AAAAAAAAOEPBAAAA|8013|2|13|33|AM|third|night|| +8014|AAAAAAAAPEPBAAAA|8014|2|13|34|AM|third|night|| +8015|AAAAAAAAAFPBAAAA|8015|2|13|35|AM|third|night|| +8016|AAAAAAAABFPBAAAA|8016|2|13|36|AM|third|night|| +8017|AAAAAAAACFPBAAAA|8017|2|13|37|AM|third|night|| +8018|AAAAAAAADFPBAAAA|8018|2|13|38|AM|third|night|| +8019|AAAAAAAAEFPBAAAA|8019|2|13|39|AM|third|night|| +8020|AAAAAAAAFFPBAAAA|8020|2|13|40|AM|third|night|| +8021|AAAAAAAAGFPBAAAA|8021|2|13|41|AM|third|night|| +8022|AAAAAAAAHFPBAAAA|8022|2|13|42|AM|third|night|| +8023|AAAAAAAAIFPBAAAA|8023|2|13|43|AM|third|night|| +8024|AAAAAAAAJFPBAAAA|8024|2|13|44|AM|third|night|| +8025|AAAAAAAAKFPBAAAA|8025|2|13|45|AM|third|night|| +8026|AAAAAAAALFPBAAAA|8026|2|13|46|AM|third|night|| +8027|AAAAAAAAMFPBAAAA|8027|2|13|47|AM|third|night|| +8028|AAAAAAAANFPBAAAA|8028|2|13|48|AM|third|night|| +8029|AAAAAAAAOFPBAAAA|8029|2|13|49|AM|third|night|| +8030|AAAAAAAAPFPBAAAA|8030|2|13|50|AM|third|night|| +8031|AAAAAAAAAGPBAAAA|8031|2|13|51|AM|third|night|| +8032|AAAAAAAABGPBAAAA|8032|2|13|52|AM|third|night|| +8033|AAAAAAAACGPBAAAA|8033|2|13|53|AM|third|night|| +8034|AAAAAAAADGPBAAAA|8034|2|13|54|AM|third|night|| +8035|AAAAAAAAEGPBAAAA|8035|2|13|55|AM|third|night|| +8036|AAAAAAAAFGPBAAAA|8036|2|13|56|AM|third|night|| +8037|AAAAAAAAGGPBAAAA|8037|2|13|57|AM|third|night|| +8038|AAAAAAAAHGPBAAAA|8038|2|13|58|AM|third|night|| +8039|AAAAAAAAIGPBAAAA|8039|2|13|59|AM|third|night|| +8040|AAAAAAAAJGPBAAAA|8040|2|14|0|AM|third|night|| +8041|AAAAAAAAKGPBAAAA|8041|2|14|1|AM|third|night|| +8042|AAAAAAAALGPBAAAA|8042|2|14|2|AM|third|night|| +8043|AAAAAAAAMGPBAAAA|8043|2|14|3|AM|third|night|| +8044|AAAAAAAANGPBAAAA|8044|2|14|4|AM|third|night|| +8045|AAAAAAAAOGPBAAAA|8045|2|14|5|AM|third|night|| +8046|AAAAAAAAPGPBAAAA|8046|2|14|6|AM|third|night|| +8047|AAAAAAAAAHPBAAAA|8047|2|14|7|AM|third|night|| +8048|AAAAAAAABHPBAAAA|8048|2|14|8|AM|third|night|| +8049|AAAAAAAACHPBAAAA|8049|2|14|9|AM|third|night|| +8050|AAAAAAAADHPBAAAA|8050|2|14|10|AM|third|night|| +8051|AAAAAAAAEHPBAAAA|8051|2|14|11|AM|third|night|| +8052|AAAAAAAAFHPBAAAA|8052|2|14|12|AM|third|night|| +8053|AAAAAAAAGHPBAAAA|8053|2|14|13|AM|third|night|| +8054|AAAAAAAAHHPBAAAA|8054|2|14|14|AM|third|night|| +8055|AAAAAAAAIHPBAAAA|8055|2|14|15|AM|third|night|| +8056|AAAAAAAAJHPBAAAA|8056|2|14|16|AM|third|night|| +8057|AAAAAAAAKHPBAAAA|8057|2|14|17|AM|third|night|| +8058|AAAAAAAALHPBAAAA|8058|2|14|18|AM|third|night|| +8059|AAAAAAAAMHPBAAAA|8059|2|14|19|AM|third|night|| +8060|AAAAAAAANHPBAAAA|8060|2|14|20|AM|third|night|| +8061|AAAAAAAAOHPBAAAA|8061|2|14|21|AM|third|night|| +8062|AAAAAAAAPHPBAAAA|8062|2|14|22|AM|third|night|| +8063|AAAAAAAAAIPBAAAA|8063|2|14|23|AM|third|night|| +8064|AAAAAAAABIPBAAAA|8064|2|14|24|AM|third|night|| +8065|AAAAAAAACIPBAAAA|8065|2|14|25|AM|third|night|| +8066|AAAAAAAADIPBAAAA|8066|2|14|26|AM|third|night|| +8067|AAAAAAAAEIPBAAAA|8067|2|14|27|AM|third|night|| +8068|AAAAAAAAFIPBAAAA|8068|2|14|28|AM|third|night|| +8069|AAAAAAAAGIPBAAAA|8069|2|14|29|AM|third|night|| +8070|AAAAAAAAHIPBAAAA|8070|2|14|30|AM|third|night|| +8071|AAAAAAAAIIPBAAAA|8071|2|14|31|AM|third|night|| +8072|AAAAAAAAJIPBAAAA|8072|2|14|32|AM|third|night|| +8073|AAAAAAAAKIPBAAAA|8073|2|14|33|AM|third|night|| +8074|AAAAAAAALIPBAAAA|8074|2|14|34|AM|third|night|| +8075|AAAAAAAAMIPBAAAA|8075|2|14|35|AM|third|night|| +8076|AAAAAAAANIPBAAAA|8076|2|14|36|AM|third|night|| +8077|AAAAAAAAOIPBAAAA|8077|2|14|37|AM|third|night|| +8078|AAAAAAAAPIPBAAAA|8078|2|14|38|AM|third|night|| +8079|AAAAAAAAAJPBAAAA|8079|2|14|39|AM|third|night|| +8080|AAAAAAAABJPBAAAA|8080|2|14|40|AM|third|night|| +8081|AAAAAAAACJPBAAAA|8081|2|14|41|AM|third|night|| +8082|AAAAAAAADJPBAAAA|8082|2|14|42|AM|third|night|| +8083|AAAAAAAAEJPBAAAA|8083|2|14|43|AM|third|night|| +8084|AAAAAAAAFJPBAAAA|8084|2|14|44|AM|third|night|| +8085|AAAAAAAAGJPBAAAA|8085|2|14|45|AM|third|night|| +8086|AAAAAAAAHJPBAAAA|8086|2|14|46|AM|third|night|| +8087|AAAAAAAAIJPBAAAA|8087|2|14|47|AM|third|night|| +8088|AAAAAAAAJJPBAAAA|8088|2|14|48|AM|third|night|| +8089|AAAAAAAAKJPBAAAA|8089|2|14|49|AM|third|night|| +8090|AAAAAAAALJPBAAAA|8090|2|14|50|AM|third|night|| +8091|AAAAAAAAMJPBAAAA|8091|2|14|51|AM|third|night|| +8092|AAAAAAAANJPBAAAA|8092|2|14|52|AM|third|night|| +8093|AAAAAAAAOJPBAAAA|8093|2|14|53|AM|third|night|| +8094|AAAAAAAAPJPBAAAA|8094|2|14|54|AM|third|night|| +8095|AAAAAAAAAKPBAAAA|8095|2|14|55|AM|third|night|| +8096|AAAAAAAABKPBAAAA|8096|2|14|56|AM|third|night|| +8097|AAAAAAAACKPBAAAA|8097|2|14|57|AM|third|night|| +8098|AAAAAAAADKPBAAAA|8098|2|14|58|AM|third|night|| +8099|AAAAAAAAEKPBAAAA|8099|2|14|59|AM|third|night|| +8100|AAAAAAAAFKPBAAAA|8100|2|15|0|AM|third|night|| +8101|AAAAAAAAGKPBAAAA|8101|2|15|1|AM|third|night|| +8102|AAAAAAAAHKPBAAAA|8102|2|15|2|AM|third|night|| +8103|AAAAAAAAIKPBAAAA|8103|2|15|3|AM|third|night|| +8104|AAAAAAAAJKPBAAAA|8104|2|15|4|AM|third|night|| +8105|AAAAAAAAKKPBAAAA|8105|2|15|5|AM|third|night|| +8106|AAAAAAAALKPBAAAA|8106|2|15|6|AM|third|night|| +8107|AAAAAAAAMKPBAAAA|8107|2|15|7|AM|third|night|| +8108|AAAAAAAANKPBAAAA|8108|2|15|8|AM|third|night|| +8109|AAAAAAAAOKPBAAAA|8109|2|15|9|AM|third|night|| +8110|AAAAAAAAPKPBAAAA|8110|2|15|10|AM|third|night|| +8111|AAAAAAAAALPBAAAA|8111|2|15|11|AM|third|night|| +8112|AAAAAAAABLPBAAAA|8112|2|15|12|AM|third|night|| +8113|AAAAAAAACLPBAAAA|8113|2|15|13|AM|third|night|| +8114|AAAAAAAADLPBAAAA|8114|2|15|14|AM|third|night|| +8115|AAAAAAAAELPBAAAA|8115|2|15|15|AM|third|night|| +8116|AAAAAAAAFLPBAAAA|8116|2|15|16|AM|third|night|| +8117|AAAAAAAAGLPBAAAA|8117|2|15|17|AM|third|night|| +8118|AAAAAAAAHLPBAAAA|8118|2|15|18|AM|third|night|| +8119|AAAAAAAAILPBAAAA|8119|2|15|19|AM|third|night|| +8120|AAAAAAAAJLPBAAAA|8120|2|15|20|AM|third|night|| +8121|AAAAAAAAKLPBAAAA|8121|2|15|21|AM|third|night|| +8122|AAAAAAAALLPBAAAA|8122|2|15|22|AM|third|night|| +8123|AAAAAAAAMLPBAAAA|8123|2|15|23|AM|third|night|| +8124|AAAAAAAANLPBAAAA|8124|2|15|24|AM|third|night|| +8125|AAAAAAAAOLPBAAAA|8125|2|15|25|AM|third|night|| +8126|AAAAAAAAPLPBAAAA|8126|2|15|26|AM|third|night|| +8127|AAAAAAAAAMPBAAAA|8127|2|15|27|AM|third|night|| +8128|AAAAAAAABMPBAAAA|8128|2|15|28|AM|third|night|| +8129|AAAAAAAACMPBAAAA|8129|2|15|29|AM|third|night|| +8130|AAAAAAAADMPBAAAA|8130|2|15|30|AM|third|night|| +8131|AAAAAAAAEMPBAAAA|8131|2|15|31|AM|third|night|| +8132|AAAAAAAAFMPBAAAA|8132|2|15|32|AM|third|night|| +8133|AAAAAAAAGMPBAAAA|8133|2|15|33|AM|third|night|| +8134|AAAAAAAAHMPBAAAA|8134|2|15|34|AM|third|night|| +8135|AAAAAAAAIMPBAAAA|8135|2|15|35|AM|third|night|| +8136|AAAAAAAAJMPBAAAA|8136|2|15|36|AM|third|night|| +8137|AAAAAAAAKMPBAAAA|8137|2|15|37|AM|third|night|| +8138|AAAAAAAALMPBAAAA|8138|2|15|38|AM|third|night|| +8139|AAAAAAAAMMPBAAAA|8139|2|15|39|AM|third|night|| +8140|AAAAAAAANMPBAAAA|8140|2|15|40|AM|third|night|| +8141|AAAAAAAAOMPBAAAA|8141|2|15|41|AM|third|night|| +8142|AAAAAAAAPMPBAAAA|8142|2|15|42|AM|third|night|| +8143|AAAAAAAAANPBAAAA|8143|2|15|43|AM|third|night|| +8144|AAAAAAAABNPBAAAA|8144|2|15|44|AM|third|night|| +8145|AAAAAAAACNPBAAAA|8145|2|15|45|AM|third|night|| +8146|AAAAAAAADNPBAAAA|8146|2|15|46|AM|third|night|| +8147|AAAAAAAAENPBAAAA|8147|2|15|47|AM|third|night|| +8148|AAAAAAAAFNPBAAAA|8148|2|15|48|AM|third|night|| +8149|AAAAAAAAGNPBAAAA|8149|2|15|49|AM|third|night|| +8150|AAAAAAAAHNPBAAAA|8150|2|15|50|AM|third|night|| +8151|AAAAAAAAINPBAAAA|8151|2|15|51|AM|third|night|| +8152|AAAAAAAAJNPBAAAA|8152|2|15|52|AM|third|night|| +8153|AAAAAAAAKNPBAAAA|8153|2|15|53|AM|third|night|| +8154|AAAAAAAALNPBAAAA|8154|2|15|54|AM|third|night|| +8155|AAAAAAAAMNPBAAAA|8155|2|15|55|AM|third|night|| +8156|AAAAAAAANNPBAAAA|8156|2|15|56|AM|third|night|| +8157|AAAAAAAAONPBAAAA|8157|2|15|57|AM|third|night|| +8158|AAAAAAAAPNPBAAAA|8158|2|15|58|AM|third|night|| +8159|AAAAAAAAAOPBAAAA|8159|2|15|59|AM|third|night|| +8160|AAAAAAAABOPBAAAA|8160|2|16|0|AM|third|night|| +8161|AAAAAAAACOPBAAAA|8161|2|16|1|AM|third|night|| +8162|AAAAAAAADOPBAAAA|8162|2|16|2|AM|third|night|| +8163|AAAAAAAAEOPBAAAA|8163|2|16|3|AM|third|night|| +8164|AAAAAAAAFOPBAAAA|8164|2|16|4|AM|third|night|| +8165|AAAAAAAAGOPBAAAA|8165|2|16|5|AM|third|night|| +8166|AAAAAAAAHOPBAAAA|8166|2|16|6|AM|third|night|| +8167|AAAAAAAAIOPBAAAA|8167|2|16|7|AM|third|night|| +8168|AAAAAAAAJOPBAAAA|8168|2|16|8|AM|third|night|| +8169|AAAAAAAAKOPBAAAA|8169|2|16|9|AM|third|night|| +8170|AAAAAAAALOPBAAAA|8170|2|16|10|AM|third|night|| +8171|AAAAAAAAMOPBAAAA|8171|2|16|11|AM|third|night|| +8172|AAAAAAAANOPBAAAA|8172|2|16|12|AM|third|night|| +8173|AAAAAAAAOOPBAAAA|8173|2|16|13|AM|third|night|| +8174|AAAAAAAAPOPBAAAA|8174|2|16|14|AM|third|night|| +8175|AAAAAAAAAPPBAAAA|8175|2|16|15|AM|third|night|| +8176|AAAAAAAABPPBAAAA|8176|2|16|16|AM|third|night|| +8177|AAAAAAAACPPBAAAA|8177|2|16|17|AM|third|night|| +8178|AAAAAAAADPPBAAAA|8178|2|16|18|AM|third|night|| +8179|AAAAAAAAEPPBAAAA|8179|2|16|19|AM|third|night|| +8180|AAAAAAAAFPPBAAAA|8180|2|16|20|AM|third|night|| +8181|AAAAAAAAGPPBAAAA|8181|2|16|21|AM|third|night|| +8182|AAAAAAAAHPPBAAAA|8182|2|16|22|AM|third|night|| +8183|AAAAAAAAIPPBAAAA|8183|2|16|23|AM|third|night|| +8184|AAAAAAAAJPPBAAAA|8184|2|16|24|AM|third|night|| +8185|AAAAAAAAKPPBAAAA|8185|2|16|25|AM|third|night|| +8186|AAAAAAAALPPBAAAA|8186|2|16|26|AM|third|night|| +8187|AAAAAAAAMPPBAAAA|8187|2|16|27|AM|third|night|| +8188|AAAAAAAANPPBAAAA|8188|2|16|28|AM|third|night|| +8189|AAAAAAAAOPPBAAAA|8189|2|16|29|AM|third|night|| +8190|AAAAAAAAPPPBAAAA|8190|2|16|30|AM|third|night|| +8191|AAAAAAAAAAACAAAA|8191|2|16|31|AM|third|night|| +8192|AAAAAAAABAACAAAA|8192|2|16|32|AM|third|night|| +8193|AAAAAAAACAACAAAA|8193|2|16|33|AM|third|night|| +8194|AAAAAAAADAACAAAA|8194|2|16|34|AM|third|night|| +8195|AAAAAAAAEAACAAAA|8195|2|16|35|AM|third|night|| +8196|AAAAAAAAFAACAAAA|8196|2|16|36|AM|third|night|| +8197|AAAAAAAAGAACAAAA|8197|2|16|37|AM|third|night|| +8198|AAAAAAAAHAACAAAA|8198|2|16|38|AM|third|night|| +8199|AAAAAAAAIAACAAAA|8199|2|16|39|AM|third|night|| +8200|AAAAAAAAJAACAAAA|8200|2|16|40|AM|third|night|| +8201|AAAAAAAAKAACAAAA|8201|2|16|41|AM|third|night|| +8202|AAAAAAAALAACAAAA|8202|2|16|42|AM|third|night|| +8203|AAAAAAAAMAACAAAA|8203|2|16|43|AM|third|night|| +8204|AAAAAAAANAACAAAA|8204|2|16|44|AM|third|night|| +8205|AAAAAAAAOAACAAAA|8205|2|16|45|AM|third|night|| +8206|AAAAAAAAPAACAAAA|8206|2|16|46|AM|third|night|| +8207|AAAAAAAAABACAAAA|8207|2|16|47|AM|third|night|| +8208|AAAAAAAABBACAAAA|8208|2|16|48|AM|third|night|| +8209|AAAAAAAACBACAAAA|8209|2|16|49|AM|third|night|| +8210|AAAAAAAADBACAAAA|8210|2|16|50|AM|third|night|| +8211|AAAAAAAAEBACAAAA|8211|2|16|51|AM|third|night|| +8212|AAAAAAAAFBACAAAA|8212|2|16|52|AM|third|night|| +8213|AAAAAAAAGBACAAAA|8213|2|16|53|AM|third|night|| +8214|AAAAAAAAHBACAAAA|8214|2|16|54|AM|third|night|| +8215|AAAAAAAAIBACAAAA|8215|2|16|55|AM|third|night|| +8216|AAAAAAAAJBACAAAA|8216|2|16|56|AM|third|night|| +8217|AAAAAAAAKBACAAAA|8217|2|16|57|AM|third|night|| +8218|AAAAAAAALBACAAAA|8218|2|16|58|AM|third|night|| +8219|AAAAAAAAMBACAAAA|8219|2|16|59|AM|third|night|| +8220|AAAAAAAANBACAAAA|8220|2|17|0|AM|third|night|| +8221|AAAAAAAAOBACAAAA|8221|2|17|1|AM|third|night|| +8222|AAAAAAAAPBACAAAA|8222|2|17|2|AM|third|night|| +8223|AAAAAAAAACACAAAA|8223|2|17|3|AM|third|night|| +8224|AAAAAAAABCACAAAA|8224|2|17|4|AM|third|night|| +8225|AAAAAAAACCACAAAA|8225|2|17|5|AM|third|night|| +8226|AAAAAAAADCACAAAA|8226|2|17|6|AM|third|night|| +8227|AAAAAAAAECACAAAA|8227|2|17|7|AM|third|night|| +8228|AAAAAAAAFCACAAAA|8228|2|17|8|AM|third|night|| +8229|AAAAAAAAGCACAAAA|8229|2|17|9|AM|third|night|| +8230|AAAAAAAAHCACAAAA|8230|2|17|10|AM|third|night|| +8231|AAAAAAAAICACAAAA|8231|2|17|11|AM|third|night|| +8232|AAAAAAAAJCACAAAA|8232|2|17|12|AM|third|night|| +8233|AAAAAAAAKCACAAAA|8233|2|17|13|AM|third|night|| +8234|AAAAAAAALCACAAAA|8234|2|17|14|AM|third|night|| +8235|AAAAAAAAMCACAAAA|8235|2|17|15|AM|third|night|| +8236|AAAAAAAANCACAAAA|8236|2|17|16|AM|third|night|| +8237|AAAAAAAAOCACAAAA|8237|2|17|17|AM|third|night|| +8238|AAAAAAAAPCACAAAA|8238|2|17|18|AM|third|night|| +8239|AAAAAAAAADACAAAA|8239|2|17|19|AM|third|night|| +8240|AAAAAAAABDACAAAA|8240|2|17|20|AM|third|night|| +8241|AAAAAAAACDACAAAA|8241|2|17|21|AM|third|night|| +8242|AAAAAAAADDACAAAA|8242|2|17|22|AM|third|night|| +8243|AAAAAAAAEDACAAAA|8243|2|17|23|AM|third|night|| +8244|AAAAAAAAFDACAAAA|8244|2|17|24|AM|third|night|| +8245|AAAAAAAAGDACAAAA|8245|2|17|25|AM|third|night|| +8246|AAAAAAAAHDACAAAA|8246|2|17|26|AM|third|night|| +8247|AAAAAAAAIDACAAAA|8247|2|17|27|AM|third|night|| +8248|AAAAAAAAJDACAAAA|8248|2|17|28|AM|third|night|| +8249|AAAAAAAAKDACAAAA|8249|2|17|29|AM|third|night|| +8250|AAAAAAAALDACAAAA|8250|2|17|30|AM|third|night|| +8251|AAAAAAAAMDACAAAA|8251|2|17|31|AM|third|night|| +8252|AAAAAAAANDACAAAA|8252|2|17|32|AM|third|night|| +8253|AAAAAAAAODACAAAA|8253|2|17|33|AM|third|night|| +8254|AAAAAAAAPDACAAAA|8254|2|17|34|AM|third|night|| +8255|AAAAAAAAAEACAAAA|8255|2|17|35|AM|third|night|| +8256|AAAAAAAABEACAAAA|8256|2|17|36|AM|third|night|| +8257|AAAAAAAACEACAAAA|8257|2|17|37|AM|third|night|| +8258|AAAAAAAADEACAAAA|8258|2|17|38|AM|third|night|| +8259|AAAAAAAAEEACAAAA|8259|2|17|39|AM|third|night|| +8260|AAAAAAAAFEACAAAA|8260|2|17|40|AM|third|night|| +8261|AAAAAAAAGEACAAAA|8261|2|17|41|AM|third|night|| +8262|AAAAAAAAHEACAAAA|8262|2|17|42|AM|third|night|| +8263|AAAAAAAAIEACAAAA|8263|2|17|43|AM|third|night|| +8264|AAAAAAAAJEACAAAA|8264|2|17|44|AM|third|night|| +8265|AAAAAAAAKEACAAAA|8265|2|17|45|AM|third|night|| +8266|AAAAAAAALEACAAAA|8266|2|17|46|AM|third|night|| +8267|AAAAAAAAMEACAAAA|8267|2|17|47|AM|third|night|| +8268|AAAAAAAANEACAAAA|8268|2|17|48|AM|third|night|| +8269|AAAAAAAAOEACAAAA|8269|2|17|49|AM|third|night|| +8270|AAAAAAAAPEACAAAA|8270|2|17|50|AM|third|night|| +8271|AAAAAAAAAFACAAAA|8271|2|17|51|AM|third|night|| +8272|AAAAAAAABFACAAAA|8272|2|17|52|AM|third|night|| +8273|AAAAAAAACFACAAAA|8273|2|17|53|AM|third|night|| +8274|AAAAAAAADFACAAAA|8274|2|17|54|AM|third|night|| +8275|AAAAAAAAEFACAAAA|8275|2|17|55|AM|third|night|| +8276|AAAAAAAAFFACAAAA|8276|2|17|56|AM|third|night|| +8277|AAAAAAAAGFACAAAA|8277|2|17|57|AM|third|night|| +8278|AAAAAAAAHFACAAAA|8278|2|17|58|AM|third|night|| +8279|AAAAAAAAIFACAAAA|8279|2|17|59|AM|third|night|| +8280|AAAAAAAAJFACAAAA|8280|2|18|0|AM|third|night|| +8281|AAAAAAAAKFACAAAA|8281|2|18|1|AM|third|night|| +8282|AAAAAAAALFACAAAA|8282|2|18|2|AM|third|night|| +8283|AAAAAAAAMFACAAAA|8283|2|18|3|AM|third|night|| +8284|AAAAAAAANFACAAAA|8284|2|18|4|AM|third|night|| +8285|AAAAAAAAOFACAAAA|8285|2|18|5|AM|third|night|| +8286|AAAAAAAAPFACAAAA|8286|2|18|6|AM|third|night|| +8287|AAAAAAAAAGACAAAA|8287|2|18|7|AM|third|night|| +8288|AAAAAAAABGACAAAA|8288|2|18|8|AM|third|night|| +8289|AAAAAAAACGACAAAA|8289|2|18|9|AM|third|night|| +8290|AAAAAAAADGACAAAA|8290|2|18|10|AM|third|night|| +8291|AAAAAAAAEGACAAAA|8291|2|18|11|AM|third|night|| +8292|AAAAAAAAFGACAAAA|8292|2|18|12|AM|third|night|| +8293|AAAAAAAAGGACAAAA|8293|2|18|13|AM|third|night|| +8294|AAAAAAAAHGACAAAA|8294|2|18|14|AM|third|night|| +8295|AAAAAAAAIGACAAAA|8295|2|18|15|AM|third|night|| +8296|AAAAAAAAJGACAAAA|8296|2|18|16|AM|third|night|| +8297|AAAAAAAAKGACAAAA|8297|2|18|17|AM|third|night|| +8298|AAAAAAAALGACAAAA|8298|2|18|18|AM|third|night|| +8299|AAAAAAAAMGACAAAA|8299|2|18|19|AM|third|night|| +8300|AAAAAAAANGACAAAA|8300|2|18|20|AM|third|night|| +8301|AAAAAAAAOGACAAAA|8301|2|18|21|AM|third|night|| +8302|AAAAAAAAPGACAAAA|8302|2|18|22|AM|third|night|| +8303|AAAAAAAAAHACAAAA|8303|2|18|23|AM|third|night|| +8304|AAAAAAAABHACAAAA|8304|2|18|24|AM|third|night|| +8305|AAAAAAAACHACAAAA|8305|2|18|25|AM|third|night|| +8306|AAAAAAAADHACAAAA|8306|2|18|26|AM|third|night|| +8307|AAAAAAAAEHACAAAA|8307|2|18|27|AM|third|night|| +8308|AAAAAAAAFHACAAAA|8308|2|18|28|AM|third|night|| +8309|AAAAAAAAGHACAAAA|8309|2|18|29|AM|third|night|| +8310|AAAAAAAAHHACAAAA|8310|2|18|30|AM|third|night|| +8311|AAAAAAAAIHACAAAA|8311|2|18|31|AM|third|night|| +8312|AAAAAAAAJHACAAAA|8312|2|18|32|AM|third|night|| +8313|AAAAAAAAKHACAAAA|8313|2|18|33|AM|third|night|| +8314|AAAAAAAALHACAAAA|8314|2|18|34|AM|third|night|| +8315|AAAAAAAAMHACAAAA|8315|2|18|35|AM|third|night|| +8316|AAAAAAAANHACAAAA|8316|2|18|36|AM|third|night|| +8317|AAAAAAAAOHACAAAA|8317|2|18|37|AM|third|night|| +8318|AAAAAAAAPHACAAAA|8318|2|18|38|AM|third|night|| +8319|AAAAAAAAAIACAAAA|8319|2|18|39|AM|third|night|| +8320|AAAAAAAABIACAAAA|8320|2|18|40|AM|third|night|| +8321|AAAAAAAACIACAAAA|8321|2|18|41|AM|third|night|| +8322|AAAAAAAADIACAAAA|8322|2|18|42|AM|third|night|| +8323|AAAAAAAAEIACAAAA|8323|2|18|43|AM|third|night|| +8324|AAAAAAAAFIACAAAA|8324|2|18|44|AM|third|night|| +8325|AAAAAAAAGIACAAAA|8325|2|18|45|AM|third|night|| +8326|AAAAAAAAHIACAAAA|8326|2|18|46|AM|third|night|| +8327|AAAAAAAAIIACAAAA|8327|2|18|47|AM|third|night|| +8328|AAAAAAAAJIACAAAA|8328|2|18|48|AM|third|night|| +8329|AAAAAAAAKIACAAAA|8329|2|18|49|AM|third|night|| +8330|AAAAAAAALIACAAAA|8330|2|18|50|AM|third|night|| +8331|AAAAAAAAMIACAAAA|8331|2|18|51|AM|third|night|| +8332|AAAAAAAANIACAAAA|8332|2|18|52|AM|third|night|| +8333|AAAAAAAAOIACAAAA|8333|2|18|53|AM|third|night|| +8334|AAAAAAAAPIACAAAA|8334|2|18|54|AM|third|night|| +8335|AAAAAAAAAJACAAAA|8335|2|18|55|AM|third|night|| +8336|AAAAAAAABJACAAAA|8336|2|18|56|AM|third|night|| +8337|AAAAAAAACJACAAAA|8337|2|18|57|AM|third|night|| +8338|AAAAAAAADJACAAAA|8338|2|18|58|AM|third|night|| +8339|AAAAAAAAEJACAAAA|8339|2|18|59|AM|third|night|| +8340|AAAAAAAAFJACAAAA|8340|2|19|0|AM|third|night|| +8341|AAAAAAAAGJACAAAA|8341|2|19|1|AM|third|night|| +8342|AAAAAAAAHJACAAAA|8342|2|19|2|AM|third|night|| +8343|AAAAAAAAIJACAAAA|8343|2|19|3|AM|third|night|| +8344|AAAAAAAAJJACAAAA|8344|2|19|4|AM|third|night|| +8345|AAAAAAAAKJACAAAA|8345|2|19|5|AM|third|night|| +8346|AAAAAAAALJACAAAA|8346|2|19|6|AM|third|night|| +8347|AAAAAAAAMJACAAAA|8347|2|19|7|AM|third|night|| +8348|AAAAAAAANJACAAAA|8348|2|19|8|AM|third|night|| +8349|AAAAAAAAOJACAAAA|8349|2|19|9|AM|third|night|| +8350|AAAAAAAAPJACAAAA|8350|2|19|10|AM|third|night|| +8351|AAAAAAAAAKACAAAA|8351|2|19|11|AM|third|night|| +8352|AAAAAAAABKACAAAA|8352|2|19|12|AM|third|night|| +8353|AAAAAAAACKACAAAA|8353|2|19|13|AM|third|night|| +8354|AAAAAAAADKACAAAA|8354|2|19|14|AM|third|night|| +8355|AAAAAAAAEKACAAAA|8355|2|19|15|AM|third|night|| +8356|AAAAAAAAFKACAAAA|8356|2|19|16|AM|third|night|| +8357|AAAAAAAAGKACAAAA|8357|2|19|17|AM|third|night|| +8358|AAAAAAAAHKACAAAA|8358|2|19|18|AM|third|night|| +8359|AAAAAAAAIKACAAAA|8359|2|19|19|AM|third|night|| +8360|AAAAAAAAJKACAAAA|8360|2|19|20|AM|third|night|| +8361|AAAAAAAAKKACAAAA|8361|2|19|21|AM|third|night|| +8362|AAAAAAAALKACAAAA|8362|2|19|22|AM|third|night|| +8363|AAAAAAAAMKACAAAA|8363|2|19|23|AM|third|night|| +8364|AAAAAAAANKACAAAA|8364|2|19|24|AM|third|night|| +8365|AAAAAAAAOKACAAAA|8365|2|19|25|AM|third|night|| +8366|AAAAAAAAPKACAAAA|8366|2|19|26|AM|third|night|| +8367|AAAAAAAAALACAAAA|8367|2|19|27|AM|third|night|| +8368|AAAAAAAABLACAAAA|8368|2|19|28|AM|third|night|| +8369|AAAAAAAACLACAAAA|8369|2|19|29|AM|third|night|| +8370|AAAAAAAADLACAAAA|8370|2|19|30|AM|third|night|| +8371|AAAAAAAAELACAAAA|8371|2|19|31|AM|third|night|| +8372|AAAAAAAAFLACAAAA|8372|2|19|32|AM|third|night|| +8373|AAAAAAAAGLACAAAA|8373|2|19|33|AM|third|night|| +8374|AAAAAAAAHLACAAAA|8374|2|19|34|AM|third|night|| +8375|AAAAAAAAILACAAAA|8375|2|19|35|AM|third|night|| +8376|AAAAAAAAJLACAAAA|8376|2|19|36|AM|third|night|| +8377|AAAAAAAAKLACAAAA|8377|2|19|37|AM|third|night|| +8378|AAAAAAAALLACAAAA|8378|2|19|38|AM|third|night|| +8379|AAAAAAAAMLACAAAA|8379|2|19|39|AM|third|night|| +8380|AAAAAAAANLACAAAA|8380|2|19|40|AM|third|night|| +8381|AAAAAAAAOLACAAAA|8381|2|19|41|AM|third|night|| +8382|AAAAAAAAPLACAAAA|8382|2|19|42|AM|third|night|| +8383|AAAAAAAAAMACAAAA|8383|2|19|43|AM|third|night|| +8384|AAAAAAAABMACAAAA|8384|2|19|44|AM|third|night|| +8385|AAAAAAAACMACAAAA|8385|2|19|45|AM|third|night|| +8386|AAAAAAAADMACAAAA|8386|2|19|46|AM|third|night|| +8387|AAAAAAAAEMACAAAA|8387|2|19|47|AM|third|night|| +8388|AAAAAAAAFMACAAAA|8388|2|19|48|AM|third|night|| +8389|AAAAAAAAGMACAAAA|8389|2|19|49|AM|third|night|| +8390|AAAAAAAAHMACAAAA|8390|2|19|50|AM|third|night|| +8391|AAAAAAAAIMACAAAA|8391|2|19|51|AM|third|night|| +8392|AAAAAAAAJMACAAAA|8392|2|19|52|AM|third|night|| +8393|AAAAAAAAKMACAAAA|8393|2|19|53|AM|third|night|| +8394|AAAAAAAALMACAAAA|8394|2|19|54|AM|third|night|| +8395|AAAAAAAAMMACAAAA|8395|2|19|55|AM|third|night|| +8396|AAAAAAAANMACAAAA|8396|2|19|56|AM|third|night|| +8397|AAAAAAAAOMACAAAA|8397|2|19|57|AM|third|night|| +8398|AAAAAAAAPMACAAAA|8398|2|19|58|AM|third|night|| +8399|AAAAAAAAANACAAAA|8399|2|19|59|AM|third|night|| +8400|AAAAAAAABNACAAAA|8400|2|20|0|AM|third|night|| +8401|AAAAAAAACNACAAAA|8401|2|20|1|AM|third|night|| +8402|AAAAAAAADNACAAAA|8402|2|20|2|AM|third|night|| +8403|AAAAAAAAENACAAAA|8403|2|20|3|AM|third|night|| +8404|AAAAAAAAFNACAAAA|8404|2|20|4|AM|third|night|| +8405|AAAAAAAAGNACAAAA|8405|2|20|5|AM|third|night|| +8406|AAAAAAAAHNACAAAA|8406|2|20|6|AM|third|night|| +8407|AAAAAAAAINACAAAA|8407|2|20|7|AM|third|night|| +8408|AAAAAAAAJNACAAAA|8408|2|20|8|AM|third|night|| +8409|AAAAAAAAKNACAAAA|8409|2|20|9|AM|third|night|| +8410|AAAAAAAALNACAAAA|8410|2|20|10|AM|third|night|| +8411|AAAAAAAAMNACAAAA|8411|2|20|11|AM|third|night|| +8412|AAAAAAAANNACAAAA|8412|2|20|12|AM|third|night|| +8413|AAAAAAAAONACAAAA|8413|2|20|13|AM|third|night|| +8414|AAAAAAAAPNACAAAA|8414|2|20|14|AM|third|night|| +8415|AAAAAAAAAOACAAAA|8415|2|20|15|AM|third|night|| +8416|AAAAAAAABOACAAAA|8416|2|20|16|AM|third|night|| +8417|AAAAAAAACOACAAAA|8417|2|20|17|AM|third|night|| +8418|AAAAAAAADOACAAAA|8418|2|20|18|AM|third|night|| +8419|AAAAAAAAEOACAAAA|8419|2|20|19|AM|third|night|| +8420|AAAAAAAAFOACAAAA|8420|2|20|20|AM|third|night|| +8421|AAAAAAAAGOACAAAA|8421|2|20|21|AM|third|night|| +8422|AAAAAAAAHOACAAAA|8422|2|20|22|AM|third|night|| +8423|AAAAAAAAIOACAAAA|8423|2|20|23|AM|third|night|| +8424|AAAAAAAAJOACAAAA|8424|2|20|24|AM|third|night|| +8425|AAAAAAAAKOACAAAA|8425|2|20|25|AM|third|night|| +8426|AAAAAAAALOACAAAA|8426|2|20|26|AM|third|night|| +8427|AAAAAAAAMOACAAAA|8427|2|20|27|AM|third|night|| +8428|AAAAAAAANOACAAAA|8428|2|20|28|AM|third|night|| +8429|AAAAAAAAOOACAAAA|8429|2|20|29|AM|third|night|| +8430|AAAAAAAAPOACAAAA|8430|2|20|30|AM|third|night|| +8431|AAAAAAAAAPACAAAA|8431|2|20|31|AM|third|night|| +8432|AAAAAAAABPACAAAA|8432|2|20|32|AM|third|night|| +8433|AAAAAAAACPACAAAA|8433|2|20|33|AM|third|night|| +8434|AAAAAAAADPACAAAA|8434|2|20|34|AM|third|night|| +8435|AAAAAAAAEPACAAAA|8435|2|20|35|AM|third|night|| +8436|AAAAAAAAFPACAAAA|8436|2|20|36|AM|third|night|| +8437|AAAAAAAAGPACAAAA|8437|2|20|37|AM|third|night|| +8438|AAAAAAAAHPACAAAA|8438|2|20|38|AM|third|night|| +8439|AAAAAAAAIPACAAAA|8439|2|20|39|AM|third|night|| +8440|AAAAAAAAJPACAAAA|8440|2|20|40|AM|third|night|| +8441|AAAAAAAAKPACAAAA|8441|2|20|41|AM|third|night|| +8442|AAAAAAAALPACAAAA|8442|2|20|42|AM|third|night|| +8443|AAAAAAAAMPACAAAA|8443|2|20|43|AM|third|night|| +8444|AAAAAAAANPACAAAA|8444|2|20|44|AM|third|night|| +8445|AAAAAAAAOPACAAAA|8445|2|20|45|AM|third|night|| +8446|AAAAAAAAPPACAAAA|8446|2|20|46|AM|third|night|| +8447|AAAAAAAAAABCAAAA|8447|2|20|47|AM|third|night|| +8448|AAAAAAAABABCAAAA|8448|2|20|48|AM|third|night|| +8449|AAAAAAAACABCAAAA|8449|2|20|49|AM|third|night|| +8450|AAAAAAAADABCAAAA|8450|2|20|50|AM|third|night|| +8451|AAAAAAAAEABCAAAA|8451|2|20|51|AM|third|night|| +8452|AAAAAAAAFABCAAAA|8452|2|20|52|AM|third|night|| +8453|AAAAAAAAGABCAAAA|8453|2|20|53|AM|third|night|| +8454|AAAAAAAAHABCAAAA|8454|2|20|54|AM|third|night|| +8455|AAAAAAAAIABCAAAA|8455|2|20|55|AM|third|night|| +8456|AAAAAAAAJABCAAAA|8456|2|20|56|AM|third|night|| +8457|AAAAAAAAKABCAAAA|8457|2|20|57|AM|third|night|| +8458|AAAAAAAALABCAAAA|8458|2|20|58|AM|third|night|| +8459|AAAAAAAAMABCAAAA|8459|2|20|59|AM|third|night|| +8460|AAAAAAAANABCAAAA|8460|2|21|0|AM|third|night|| +8461|AAAAAAAAOABCAAAA|8461|2|21|1|AM|third|night|| +8462|AAAAAAAAPABCAAAA|8462|2|21|2|AM|third|night|| +8463|AAAAAAAAABBCAAAA|8463|2|21|3|AM|third|night|| +8464|AAAAAAAABBBCAAAA|8464|2|21|4|AM|third|night|| +8465|AAAAAAAACBBCAAAA|8465|2|21|5|AM|third|night|| +8466|AAAAAAAADBBCAAAA|8466|2|21|6|AM|third|night|| +8467|AAAAAAAAEBBCAAAA|8467|2|21|7|AM|third|night|| +8468|AAAAAAAAFBBCAAAA|8468|2|21|8|AM|third|night|| +8469|AAAAAAAAGBBCAAAA|8469|2|21|9|AM|third|night|| +8470|AAAAAAAAHBBCAAAA|8470|2|21|10|AM|third|night|| +8471|AAAAAAAAIBBCAAAA|8471|2|21|11|AM|third|night|| +8472|AAAAAAAAJBBCAAAA|8472|2|21|12|AM|third|night|| +8473|AAAAAAAAKBBCAAAA|8473|2|21|13|AM|third|night|| +8474|AAAAAAAALBBCAAAA|8474|2|21|14|AM|third|night|| +8475|AAAAAAAAMBBCAAAA|8475|2|21|15|AM|third|night|| +8476|AAAAAAAANBBCAAAA|8476|2|21|16|AM|third|night|| +8477|AAAAAAAAOBBCAAAA|8477|2|21|17|AM|third|night|| +8478|AAAAAAAAPBBCAAAA|8478|2|21|18|AM|third|night|| +8479|AAAAAAAAACBCAAAA|8479|2|21|19|AM|third|night|| +8480|AAAAAAAABCBCAAAA|8480|2|21|20|AM|third|night|| +8481|AAAAAAAACCBCAAAA|8481|2|21|21|AM|third|night|| +8482|AAAAAAAADCBCAAAA|8482|2|21|22|AM|third|night|| +8483|AAAAAAAAECBCAAAA|8483|2|21|23|AM|third|night|| +8484|AAAAAAAAFCBCAAAA|8484|2|21|24|AM|third|night|| +8485|AAAAAAAAGCBCAAAA|8485|2|21|25|AM|third|night|| +8486|AAAAAAAAHCBCAAAA|8486|2|21|26|AM|third|night|| +8487|AAAAAAAAICBCAAAA|8487|2|21|27|AM|third|night|| +8488|AAAAAAAAJCBCAAAA|8488|2|21|28|AM|third|night|| +8489|AAAAAAAAKCBCAAAA|8489|2|21|29|AM|third|night|| +8490|AAAAAAAALCBCAAAA|8490|2|21|30|AM|third|night|| +8491|AAAAAAAAMCBCAAAA|8491|2|21|31|AM|third|night|| +8492|AAAAAAAANCBCAAAA|8492|2|21|32|AM|third|night|| +8493|AAAAAAAAOCBCAAAA|8493|2|21|33|AM|third|night|| +8494|AAAAAAAAPCBCAAAA|8494|2|21|34|AM|third|night|| +8495|AAAAAAAAADBCAAAA|8495|2|21|35|AM|third|night|| +8496|AAAAAAAABDBCAAAA|8496|2|21|36|AM|third|night|| +8497|AAAAAAAACDBCAAAA|8497|2|21|37|AM|third|night|| +8498|AAAAAAAADDBCAAAA|8498|2|21|38|AM|third|night|| +8499|AAAAAAAAEDBCAAAA|8499|2|21|39|AM|third|night|| +8500|AAAAAAAAFDBCAAAA|8500|2|21|40|AM|third|night|| +8501|AAAAAAAAGDBCAAAA|8501|2|21|41|AM|third|night|| +8502|AAAAAAAAHDBCAAAA|8502|2|21|42|AM|third|night|| +8503|AAAAAAAAIDBCAAAA|8503|2|21|43|AM|third|night|| +8504|AAAAAAAAJDBCAAAA|8504|2|21|44|AM|third|night|| +8505|AAAAAAAAKDBCAAAA|8505|2|21|45|AM|third|night|| +8506|AAAAAAAALDBCAAAA|8506|2|21|46|AM|third|night|| +8507|AAAAAAAAMDBCAAAA|8507|2|21|47|AM|third|night|| +8508|AAAAAAAANDBCAAAA|8508|2|21|48|AM|third|night|| +8509|AAAAAAAAODBCAAAA|8509|2|21|49|AM|third|night|| +8510|AAAAAAAAPDBCAAAA|8510|2|21|50|AM|third|night|| +8511|AAAAAAAAAEBCAAAA|8511|2|21|51|AM|third|night|| +8512|AAAAAAAABEBCAAAA|8512|2|21|52|AM|third|night|| +8513|AAAAAAAACEBCAAAA|8513|2|21|53|AM|third|night|| +8514|AAAAAAAADEBCAAAA|8514|2|21|54|AM|third|night|| +8515|AAAAAAAAEEBCAAAA|8515|2|21|55|AM|third|night|| +8516|AAAAAAAAFEBCAAAA|8516|2|21|56|AM|third|night|| +8517|AAAAAAAAGEBCAAAA|8517|2|21|57|AM|third|night|| +8518|AAAAAAAAHEBCAAAA|8518|2|21|58|AM|third|night|| +8519|AAAAAAAAIEBCAAAA|8519|2|21|59|AM|third|night|| +8520|AAAAAAAAJEBCAAAA|8520|2|22|0|AM|third|night|| +8521|AAAAAAAAKEBCAAAA|8521|2|22|1|AM|third|night|| +8522|AAAAAAAALEBCAAAA|8522|2|22|2|AM|third|night|| +8523|AAAAAAAAMEBCAAAA|8523|2|22|3|AM|third|night|| +8524|AAAAAAAANEBCAAAA|8524|2|22|4|AM|third|night|| +8525|AAAAAAAAOEBCAAAA|8525|2|22|5|AM|third|night|| +8526|AAAAAAAAPEBCAAAA|8526|2|22|6|AM|third|night|| +8527|AAAAAAAAAFBCAAAA|8527|2|22|7|AM|third|night|| +8528|AAAAAAAABFBCAAAA|8528|2|22|8|AM|third|night|| +8529|AAAAAAAACFBCAAAA|8529|2|22|9|AM|third|night|| +8530|AAAAAAAADFBCAAAA|8530|2|22|10|AM|third|night|| +8531|AAAAAAAAEFBCAAAA|8531|2|22|11|AM|third|night|| +8532|AAAAAAAAFFBCAAAA|8532|2|22|12|AM|third|night|| +8533|AAAAAAAAGFBCAAAA|8533|2|22|13|AM|third|night|| +8534|AAAAAAAAHFBCAAAA|8534|2|22|14|AM|third|night|| +8535|AAAAAAAAIFBCAAAA|8535|2|22|15|AM|third|night|| +8536|AAAAAAAAJFBCAAAA|8536|2|22|16|AM|third|night|| +8537|AAAAAAAAKFBCAAAA|8537|2|22|17|AM|third|night|| +8538|AAAAAAAALFBCAAAA|8538|2|22|18|AM|third|night|| +8539|AAAAAAAAMFBCAAAA|8539|2|22|19|AM|third|night|| +8540|AAAAAAAANFBCAAAA|8540|2|22|20|AM|third|night|| +8541|AAAAAAAAOFBCAAAA|8541|2|22|21|AM|third|night|| +8542|AAAAAAAAPFBCAAAA|8542|2|22|22|AM|third|night|| +8543|AAAAAAAAAGBCAAAA|8543|2|22|23|AM|third|night|| +8544|AAAAAAAABGBCAAAA|8544|2|22|24|AM|third|night|| +8545|AAAAAAAACGBCAAAA|8545|2|22|25|AM|third|night|| +8546|AAAAAAAADGBCAAAA|8546|2|22|26|AM|third|night|| +8547|AAAAAAAAEGBCAAAA|8547|2|22|27|AM|third|night|| +8548|AAAAAAAAFGBCAAAA|8548|2|22|28|AM|third|night|| +8549|AAAAAAAAGGBCAAAA|8549|2|22|29|AM|third|night|| +8550|AAAAAAAAHGBCAAAA|8550|2|22|30|AM|third|night|| +8551|AAAAAAAAIGBCAAAA|8551|2|22|31|AM|third|night|| +8552|AAAAAAAAJGBCAAAA|8552|2|22|32|AM|third|night|| +8553|AAAAAAAAKGBCAAAA|8553|2|22|33|AM|third|night|| +8554|AAAAAAAALGBCAAAA|8554|2|22|34|AM|third|night|| +8555|AAAAAAAAMGBCAAAA|8555|2|22|35|AM|third|night|| +8556|AAAAAAAANGBCAAAA|8556|2|22|36|AM|third|night|| +8557|AAAAAAAAOGBCAAAA|8557|2|22|37|AM|third|night|| +8558|AAAAAAAAPGBCAAAA|8558|2|22|38|AM|third|night|| +8559|AAAAAAAAAHBCAAAA|8559|2|22|39|AM|third|night|| +8560|AAAAAAAABHBCAAAA|8560|2|22|40|AM|third|night|| +8561|AAAAAAAACHBCAAAA|8561|2|22|41|AM|third|night|| +8562|AAAAAAAADHBCAAAA|8562|2|22|42|AM|third|night|| +8563|AAAAAAAAEHBCAAAA|8563|2|22|43|AM|third|night|| +8564|AAAAAAAAFHBCAAAA|8564|2|22|44|AM|third|night|| +8565|AAAAAAAAGHBCAAAA|8565|2|22|45|AM|third|night|| +8566|AAAAAAAAHHBCAAAA|8566|2|22|46|AM|third|night|| +8567|AAAAAAAAIHBCAAAA|8567|2|22|47|AM|third|night|| +8568|AAAAAAAAJHBCAAAA|8568|2|22|48|AM|third|night|| +8569|AAAAAAAAKHBCAAAA|8569|2|22|49|AM|third|night|| +8570|AAAAAAAALHBCAAAA|8570|2|22|50|AM|third|night|| +8571|AAAAAAAAMHBCAAAA|8571|2|22|51|AM|third|night|| +8572|AAAAAAAANHBCAAAA|8572|2|22|52|AM|third|night|| +8573|AAAAAAAAOHBCAAAA|8573|2|22|53|AM|third|night|| +8574|AAAAAAAAPHBCAAAA|8574|2|22|54|AM|third|night|| +8575|AAAAAAAAAIBCAAAA|8575|2|22|55|AM|third|night|| +8576|AAAAAAAABIBCAAAA|8576|2|22|56|AM|third|night|| +8577|AAAAAAAACIBCAAAA|8577|2|22|57|AM|third|night|| +8578|AAAAAAAADIBCAAAA|8578|2|22|58|AM|third|night|| +8579|AAAAAAAAEIBCAAAA|8579|2|22|59|AM|third|night|| +8580|AAAAAAAAFIBCAAAA|8580|2|23|0|AM|third|night|| +8581|AAAAAAAAGIBCAAAA|8581|2|23|1|AM|third|night|| +8582|AAAAAAAAHIBCAAAA|8582|2|23|2|AM|third|night|| +8583|AAAAAAAAIIBCAAAA|8583|2|23|3|AM|third|night|| +8584|AAAAAAAAJIBCAAAA|8584|2|23|4|AM|third|night|| +8585|AAAAAAAAKIBCAAAA|8585|2|23|5|AM|third|night|| +8586|AAAAAAAALIBCAAAA|8586|2|23|6|AM|third|night|| +8587|AAAAAAAAMIBCAAAA|8587|2|23|7|AM|third|night|| +8588|AAAAAAAANIBCAAAA|8588|2|23|8|AM|third|night|| +8589|AAAAAAAAOIBCAAAA|8589|2|23|9|AM|third|night|| +8590|AAAAAAAAPIBCAAAA|8590|2|23|10|AM|third|night|| +8591|AAAAAAAAAJBCAAAA|8591|2|23|11|AM|third|night|| +8592|AAAAAAAABJBCAAAA|8592|2|23|12|AM|third|night|| +8593|AAAAAAAACJBCAAAA|8593|2|23|13|AM|third|night|| +8594|AAAAAAAADJBCAAAA|8594|2|23|14|AM|third|night|| +8595|AAAAAAAAEJBCAAAA|8595|2|23|15|AM|third|night|| +8596|AAAAAAAAFJBCAAAA|8596|2|23|16|AM|third|night|| +8597|AAAAAAAAGJBCAAAA|8597|2|23|17|AM|third|night|| +8598|AAAAAAAAHJBCAAAA|8598|2|23|18|AM|third|night|| +8599|AAAAAAAAIJBCAAAA|8599|2|23|19|AM|third|night|| +8600|AAAAAAAAJJBCAAAA|8600|2|23|20|AM|third|night|| +8601|AAAAAAAAKJBCAAAA|8601|2|23|21|AM|third|night|| +8602|AAAAAAAALJBCAAAA|8602|2|23|22|AM|third|night|| +8603|AAAAAAAAMJBCAAAA|8603|2|23|23|AM|third|night|| +8604|AAAAAAAANJBCAAAA|8604|2|23|24|AM|third|night|| +8605|AAAAAAAAOJBCAAAA|8605|2|23|25|AM|third|night|| +8606|AAAAAAAAPJBCAAAA|8606|2|23|26|AM|third|night|| +8607|AAAAAAAAAKBCAAAA|8607|2|23|27|AM|third|night|| +8608|AAAAAAAABKBCAAAA|8608|2|23|28|AM|third|night|| +8609|AAAAAAAACKBCAAAA|8609|2|23|29|AM|third|night|| +8610|AAAAAAAADKBCAAAA|8610|2|23|30|AM|third|night|| +8611|AAAAAAAAEKBCAAAA|8611|2|23|31|AM|third|night|| +8612|AAAAAAAAFKBCAAAA|8612|2|23|32|AM|third|night|| +8613|AAAAAAAAGKBCAAAA|8613|2|23|33|AM|third|night|| +8614|AAAAAAAAHKBCAAAA|8614|2|23|34|AM|third|night|| +8615|AAAAAAAAIKBCAAAA|8615|2|23|35|AM|third|night|| +8616|AAAAAAAAJKBCAAAA|8616|2|23|36|AM|third|night|| +8617|AAAAAAAAKKBCAAAA|8617|2|23|37|AM|third|night|| +8618|AAAAAAAALKBCAAAA|8618|2|23|38|AM|third|night|| +8619|AAAAAAAAMKBCAAAA|8619|2|23|39|AM|third|night|| +8620|AAAAAAAANKBCAAAA|8620|2|23|40|AM|third|night|| +8621|AAAAAAAAOKBCAAAA|8621|2|23|41|AM|third|night|| +8622|AAAAAAAAPKBCAAAA|8622|2|23|42|AM|third|night|| +8623|AAAAAAAAALBCAAAA|8623|2|23|43|AM|third|night|| +8624|AAAAAAAABLBCAAAA|8624|2|23|44|AM|third|night|| +8625|AAAAAAAACLBCAAAA|8625|2|23|45|AM|third|night|| +8626|AAAAAAAADLBCAAAA|8626|2|23|46|AM|third|night|| +8627|AAAAAAAAELBCAAAA|8627|2|23|47|AM|third|night|| +8628|AAAAAAAAFLBCAAAA|8628|2|23|48|AM|third|night|| +8629|AAAAAAAAGLBCAAAA|8629|2|23|49|AM|third|night|| +8630|AAAAAAAAHLBCAAAA|8630|2|23|50|AM|third|night|| +8631|AAAAAAAAILBCAAAA|8631|2|23|51|AM|third|night|| +8632|AAAAAAAAJLBCAAAA|8632|2|23|52|AM|third|night|| +8633|AAAAAAAAKLBCAAAA|8633|2|23|53|AM|third|night|| +8634|AAAAAAAALLBCAAAA|8634|2|23|54|AM|third|night|| +8635|AAAAAAAAMLBCAAAA|8635|2|23|55|AM|third|night|| +8636|AAAAAAAANLBCAAAA|8636|2|23|56|AM|third|night|| +8637|AAAAAAAAOLBCAAAA|8637|2|23|57|AM|third|night|| +8638|AAAAAAAAPLBCAAAA|8638|2|23|58|AM|third|night|| +8639|AAAAAAAAAMBCAAAA|8639|2|23|59|AM|third|night|| +8640|AAAAAAAABMBCAAAA|8640|2|24|0|AM|third|night|| +8641|AAAAAAAACMBCAAAA|8641|2|24|1|AM|third|night|| +8642|AAAAAAAADMBCAAAA|8642|2|24|2|AM|third|night|| +8643|AAAAAAAAEMBCAAAA|8643|2|24|3|AM|third|night|| +8644|AAAAAAAAFMBCAAAA|8644|2|24|4|AM|third|night|| +8645|AAAAAAAAGMBCAAAA|8645|2|24|5|AM|third|night|| +8646|AAAAAAAAHMBCAAAA|8646|2|24|6|AM|third|night|| +8647|AAAAAAAAIMBCAAAA|8647|2|24|7|AM|third|night|| +8648|AAAAAAAAJMBCAAAA|8648|2|24|8|AM|third|night|| +8649|AAAAAAAAKMBCAAAA|8649|2|24|9|AM|third|night|| +8650|AAAAAAAALMBCAAAA|8650|2|24|10|AM|third|night|| +8651|AAAAAAAAMMBCAAAA|8651|2|24|11|AM|third|night|| +8652|AAAAAAAANMBCAAAA|8652|2|24|12|AM|third|night|| +8653|AAAAAAAAOMBCAAAA|8653|2|24|13|AM|third|night|| +8654|AAAAAAAAPMBCAAAA|8654|2|24|14|AM|third|night|| +8655|AAAAAAAAANBCAAAA|8655|2|24|15|AM|third|night|| +8656|AAAAAAAABNBCAAAA|8656|2|24|16|AM|third|night|| +8657|AAAAAAAACNBCAAAA|8657|2|24|17|AM|third|night|| +8658|AAAAAAAADNBCAAAA|8658|2|24|18|AM|third|night|| +8659|AAAAAAAAENBCAAAA|8659|2|24|19|AM|third|night|| +8660|AAAAAAAAFNBCAAAA|8660|2|24|20|AM|third|night|| +8661|AAAAAAAAGNBCAAAA|8661|2|24|21|AM|third|night|| +8662|AAAAAAAAHNBCAAAA|8662|2|24|22|AM|third|night|| +8663|AAAAAAAAINBCAAAA|8663|2|24|23|AM|third|night|| +8664|AAAAAAAAJNBCAAAA|8664|2|24|24|AM|third|night|| +8665|AAAAAAAAKNBCAAAA|8665|2|24|25|AM|third|night|| +8666|AAAAAAAALNBCAAAA|8666|2|24|26|AM|third|night|| +8667|AAAAAAAAMNBCAAAA|8667|2|24|27|AM|third|night|| +8668|AAAAAAAANNBCAAAA|8668|2|24|28|AM|third|night|| +8669|AAAAAAAAONBCAAAA|8669|2|24|29|AM|third|night|| +8670|AAAAAAAAPNBCAAAA|8670|2|24|30|AM|third|night|| +8671|AAAAAAAAAOBCAAAA|8671|2|24|31|AM|third|night|| +8672|AAAAAAAABOBCAAAA|8672|2|24|32|AM|third|night|| +8673|AAAAAAAACOBCAAAA|8673|2|24|33|AM|third|night|| +8674|AAAAAAAADOBCAAAA|8674|2|24|34|AM|third|night|| +8675|AAAAAAAAEOBCAAAA|8675|2|24|35|AM|third|night|| +8676|AAAAAAAAFOBCAAAA|8676|2|24|36|AM|third|night|| +8677|AAAAAAAAGOBCAAAA|8677|2|24|37|AM|third|night|| +8678|AAAAAAAAHOBCAAAA|8678|2|24|38|AM|third|night|| +8679|AAAAAAAAIOBCAAAA|8679|2|24|39|AM|third|night|| +8680|AAAAAAAAJOBCAAAA|8680|2|24|40|AM|third|night|| +8681|AAAAAAAAKOBCAAAA|8681|2|24|41|AM|third|night|| +8682|AAAAAAAALOBCAAAA|8682|2|24|42|AM|third|night|| +8683|AAAAAAAAMOBCAAAA|8683|2|24|43|AM|third|night|| +8684|AAAAAAAANOBCAAAA|8684|2|24|44|AM|third|night|| +8685|AAAAAAAAOOBCAAAA|8685|2|24|45|AM|third|night|| +8686|AAAAAAAAPOBCAAAA|8686|2|24|46|AM|third|night|| +8687|AAAAAAAAAPBCAAAA|8687|2|24|47|AM|third|night|| +8688|AAAAAAAABPBCAAAA|8688|2|24|48|AM|third|night|| +8689|AAAAAAAACPBCAAAA|8689|2|24|49|AM|third|night|| +8690|AAAAAAAADPBCAAAA|8690|2|24|50|AM|third|night|| +8691|AAAAAAAAEPBCAAAA|8691|2|24|51|AM|third|night|| +8692|AAAAAAAAFPBCAAAA|8692|2|24|52|AM|third|night|| +8693|AAAAAAAAGPBCAAAA|8693|2|24|53|AM|third|night|| +8694|AAAAAAAAHPBCAAAA|8694|2|24|54|AM|third|night|| +8695|AAAAAAAAIPBCAAAA|8695|2|24|55|AM|third|night|| +8696|AAAAAAAAJPBCAAAA|8696|2|24|56|AM|third|night|| +8697|AAAAAAAAKPBCAAAA|8697|2|24|57|AM|third|night|| +8698|AAAAAAAALPBCAAAA|8698|2|24|58|AM|third|night|| +8699|AAAAAAAAMPBCAAAA|8699|2|24|59|AM|third|night|| +8700|AAAAAAAANPBCAAAA|8700|2|25|0|AM|third|night|| +8701|AAAAAAAAOPBCAAAA|8701|2|25|1|AM|third|night|| +8702|AAAAAAAAPPBCAAAA|8702|2|25|2|AM|third|night|| +8703|AAAAAAAAAACCAAAA|8703|2|25|3|AM|third|night|| +8704|AAAAAAAABACCAAAA|8704|2|25|4|AM|third|night|| +8705|AAAAAAAACACCAAAA|8705|2|25|5|AM|third|night|| +8706|AAAAAAAADACCAAAA|8706|2|25|6|AM|third|night|| +8707|AAAAAAAAEACCAAAA|8707|2|25|7|AM|third|night|| +8708|AAAAAAAAFACCAAAA|8708|2|25|8|AM|third|night|| +8709|AAAAAAAAGACCAAAA|8709|2|25|9|AM|third|night|| +8710|AAAAAAAAHACCAAAA|8710|2|25|10|AM|third|night|| +8711|AAAAAAAAIACCAAAA|8711|2|25|11|AM|third|night|| +8712|AAAAAAAAJACCAAAA|8712|2|25|12|AM|third|night|| +8713|AAAAAAAAKACCAAAA|8713|2|25|13|AM|third|night|| +8714|AAAAAAAALACCAAAA|8714|2|25|14|AM|third|night|| +8715|AAAAAAAAMACCAAAA|8715|2|25|15|AM|third|night|| +8716|AAAAAAAANACCAAAA|8716|2|25|16|AM|third|night|| +8717|AAAAAAAAOACCAAAA|8717|2|25|17|AM|third|night|| +8718|AAAAAAAAPACCAAAA|8718|2|25|18|AM|third|night|| +8719|AAAAAAAAABCCAAAA|8719|2|25|19|AM|third|night|| +8720|AAAAAAAABBCCAAAA|8720|2|25|20|AM|third|night|| +8721|AAAAAAAACBCCAAAA|8721|2|25|21|AM|third|night|| +8722|AAAAAAAADBCCAAAA|8722|2|25|22|AM|third|night|| +8723|AAAAAAAAEBCCAAAA|8723|2|25|23|AM|third|night|| +8724|AAAAAAAAFBCCAAAA|8724|2|25|24|AM|third|night|| +8725|AAAAAAAAGBCCAAAA|8725|2|25|25|AM|third|night|| +8726|AAAAAAAAHBCCAAAA|8726|2|25|26|AM|third|night|| +8727|AAAAAAAAIBCCAAAA|8727|2|25|27|AM|third|night|| +8728|AAAAAAAAJBCCAAAA|8728|2|25|28|AM|third|night|| +8729|AAAAAAAAKBCCAAAA|8729|2|25|29|AM|third|night|| +8730|AAAAAAAALBCCAAAA|8730|2|25|30|AM|third|night|| +8731|AAAAAAAAMBCCAAAA|8731|2|25|31|AM|third|night|| +8732|AAAAAAAANBCCAAAA|8732|2|25|32|AM|third|night|| +8733|AAAAAAAAOBCCAAAA|8733|2|25|33|AM|third|night|| +8734|AAAAAAAAPBCCAAAA|8734|2|25|34|AM|third|night|| +8735|AAAAAAAAACCCAAAA|8735|2|25|35|AM|third|night|| +8736|AAAAAAAABCCCAAAA|8736|2|25|36|AM|third|night|| +8737|AAAAAAAACCCCAAAA|8737|2|25|37|AM|third|night|| +8738|AAAAAAAADCCCAAAA|8738|2|25|38|AM|third|night|| +8739|AAAAAAAAECCCAAAA|8739|2|25|39|AM|third|night|| +8740|AAAAAAAAFCCCAAAA|8740|2|25|40|AM|third|night|| +8741|AAAAAAAAGCCCAAAA|8741|2|25|41|AM|third|night|| +8742|AAAAAAAAHCCCAAAA|8742|2|25|42|AM|third|night|| +8743|AAAAAAAAICCCAAAA|8743|2|25|43|AM|third|night|| +8744|AAAAAAAAJCCCAAAA|8744|2|25|44|AM|third|night|| +8745|AAAAAAAAKCCCAAAA|8745|2|25|45|AM|third|night|| +8746|AAAAAAAALCCCAAAA|8746|2|25|46|AM|third|night|| +8747|AAAAAAAAMCCCAAAA|8747|2|25|47|AM|third|night|| +8748|AAAAAAAANCCCAAAA|8748|2|25|48|AM|third|night|| +8749|AAAAAAAAOCCCAAAA|8749|2|25|49|AM|third|night|| +8750|AAAAAAAAPCCCAAAA|8750|2|25|50|AM|third|night|| +8751|AAAAAAAAADCCAAAA|8751|2|25|51|AM|third|night|| +8752|AAAAAAAABDCCAAAA|8752|2|25|52|AM|third|night|| +8753|AAAAAAAACDCCAAAA|8753|2|25|53|AM|third|night|| +8754|AAAAAAAADDCCAAAA|8754|2|25|54|AM|third|night|| +8755|AAAAAAAAEDCCAAAA|8755|2|25|55|AM|third|night|| +8756|AAAAAAAAFDCCAAAA|8756|2|25|56|AM|third|night|| +8757|AAAAAAAAGDCCAAAA|8757|2|25|57|AM|third|night|| +8758|AAAAAAAAHDCCAAAA|8758|2|25|58|AM|third|night|| +8759|AAAAAAAAIDCCAAAA|8759|2|25|59|AM|third|night|| +8760|AAAAAAAAJDCCAAAA|8760|2|26|0|AM|third|night|| +8761|AAAAAAAAKDCCAAAA|8761|2|26|1|AM|third|night|| +8762|AAAAAAAALDCCAAAA|8762|2|26|2|AM|third|night|| +8763|AAAAAAAAMDCCAAAA|8763|2|26|3|AM|third|night|| +8764|AAAAAAAANDCCAAAA|8764|2|26|4|AM|third|night|| +8765|AAAAAAAAODCCAAAA|8765|2|26|5|AM|third|night|| +8766|AAAAAAAAPDCCAAAA|8766|2|26|6|AM|third|night|| +8767|AAAAAAAAAECCAAAA|8767|2|26|7|AM|third|night|| +8768|AAAAAAAABECCAAAA|8768|2|26|8|AM|third|night|| +8769|AAAAAAAACECCAAAA|8769|2|26|9|AM|third|night|| +8770|AAAAAAAADECCAAAA|8770|2|26|10|AM|third|night|| +8771|AAAAAAAAEECCAAAA|8771|2|26|11|AM|third|night|| +8772|AAAAAAAAFECCAAAA|8772|2|26|12|AM|third|night|| +8773|AAAAAAAAGECCAAAA|8773|2|26|13|AM|third|night|| +8774|AAAAAAAAHECCAAAA|8774|2|26|14|AM|third|night|| +8775|AAAAAAAAIECCAAAA|8775|2|26|15|AM|third|night|| +8776|AAAAAAAAJECCAAAA|8776|2|26|16|AM|third|night|| +8777|AAAAAAAAKECCAAAA|8777|2|26|17|AM|third|night|| +8778|AAAAAAAALECCAAAA|8778|2|26|18|AM|third|night|| +8779|AAAAAAAAMECCAAAA|8779|2|26|19|AM|third|night|| +8780|AAAAAAAANECCAAAA|8780|2|26|20|AM|third|night|| +8781|AAAAAAAAOECCAAAA|8781|2|26|21|AM|third|night|| +8782|AAAAAAAAPECCAAAA|8782|2|26|22|AM|third|night|| +8783|AAAAAAAAAFCCAAAA|8783|2|26|23|AM|third|night|| +8784|AAAAAAAABFCCAAAA|8784|2|26|24|AM|third|night|| +8785|AAAAAAAACFCCAAAA|8785|2|26|25|AM|third|night|| +8786|AAAAAAAADFCCAAAA|8786|2|26|26|AM|third|night|| +8787|AAAAAAAAEFCCAAAA|8787|2|26|27|AM|third|night|| +8788|AAAAAAAAFFCCAAAA|8788|2|26|28|AM|third|night|| +8789|AAAAAAAAGFCCAAAA|8789|2|26|29|AM|third|night|| +8790|AAAAAAAAHFCCAAAA|8790|2|26|30|AM|third|night|| +8791|AAAAAAAAIFCCAAAA|8791|2|26|31|AM|third|night|| +8792|AAAAAAAAJFCCAAAA|8792|2|26|32|AM|third|night|| +8793|AAAAAAAAKFCCAAAA|8793|2|26|33|AM|third|night|| +8794|AAAAAAAALFCCAAAA|8794|2|26|34|AM|third|night|| +8795|AAAAAAAAMFCCAAAA|8795|2|26|35|AM|third|night|| +8796|AAAAAAAANFCCAAAA|8796|2|26|36|AM|third|night|| +8797|AAAAAAAAOFCCAAAA|8797|2|26|37|AM|third|night|| +8798|AAAAAAAAPFCCAAAA|8798|2|26|38|AM|third|night|| +8799|AAAAAAAAAGCCAAAA|8799|2|26|39|AM|third|night|| +8800|AAAAAAAABGCCAAAA|8800|2|26|40|AM|third|night|| +8801|AAAAAAAACGCCAAAA|8801|2|26|41|AM|third|night|| +8802|AAAAAAAADGCCAAAA|8802|2|26|42|AM|third|night|| +8803|AAAAAAAAEGCCAAAA|8803|2|26|43|AM|third|night|| +8804|AAAAAAAAFGCCAAAA|8804|2|26|44|AM|third|night|| +8805|AAAAAAAAGGCCAAAA|8805|2|26|45|AM|third|night|| +8806|AAAAAAAAHGCCAAAA|8806|2|26|46|AM|third|night|| +8807|AAAAAAAAIGCCAAAA|8807|2|26|47|AM|third|night|| +8808|AAAAAAAAJGCCAAAA|8808|2|26|48|AM|third|night|| +8809|AAAAAAAAKGCCAAAA|8809|2|26|49|AM|third|night|| +8810|AAAAAAAALGCCAAAA|8810|2|26|50|AM|third|night|| +8811|AAAAAAAAMGCCAAAA|8811|2|26|51|AM|third|night|| +8812|AAAAAAAANGCCAAAA|8812|2|26|52|AM|third|night|| +8813|AAAAAAAAOGCCAAAA|8813|2|26|53|AM|third|night|| +8814|AAAAAAAAPGCCAAAA|8814|2|26|54|AM|third|night|| +8815|AAAAAAAAAHCCAAAA|8815|2|26|55|AM|third|night|| +8816|AAAAAAAABHCCAAAA|8816|2|26|56|AM|third|night|| +8817|AAAAAAAACHCCAAAA|8817|2|26|57|AM|third|night|| +8818|AAAAAAAADHCCAAAA|8818|2|26|58|AM|third|night|| +8819|AAAAAAAAEHCCAAAA|8819|2|26|59|AM|third|night|| +8820|AAAAAAAAFHCCAAAA|8820|2|27|0|AM|third|night|| +8821|AAAAAAAAGHCCAAAA|8821|2|27|1|AM|third|night|| +8822|AAAAAAAAHHCCAAAA|8822|2|27|2|AM|third|night|| +8823|AAAAAAAAIHCCAAAA|8823|2|27|3|AM|third|night|| +8824|AAAAAAAAJHCCAAAA|8824|2|27|4|AM|third|night|| +8825|AAAAAAAAKHCCAAAA|8825|2|27|5|AM|third|night|| +8826|AAAAAAAALHCCAAAA|8826|2|27|6|AM|third|night|| +8827|AAAAAAAAMHCCAAAA|8827|2|27|7|AM|third|night|| +8828|AAAAAAAANHCCAAAA|8828|2|27|8|AM|third|night|| +8829|AAAAAAAAOHCCAAAA|8829|2|27|9|AM|third|night|| +8830|AAAAAAAAPHCCAAAA|8830|2|27|10|AM|third|night|| +8831|AAAAAAAAAICCAAAA|8831|2|27|11|AM|third|night|| +8832|AAAAAAAABICCAAAA|8832|2|27|12|AM|third|night|| +8833|AAAAAAAACICCAAAA|8833|2|27|13|AM|third|night|| +8834|AAAAAAAADICCAAAA|8834|2|27|14|AM|third|night|| +8835|AAAAAAAAEICCAAAA|8835|2|27|15|AM|third|night|| +8836|AAAAAAAAFICCAAAA|8836|2|27|16|AM|third|night|| +8837|AAAAAAAAGICCAAAA|8837|2|27|17|AM|third|night|| +8838|AAAAAAAAHICCAAAA|8838|2|27|18|AM|third|night|| +8839|AAAAAAAAIICCAAAA|8839|2|27|19|AM|third|night|| +8840|AAAAAAAAJICCAAAA|8840|2|27|20|AM|third|night|| +8841|AAAAAAAAKICCAAAA|8841|2|27|21|AM|third|night|| +8842|AAAAAAAALICCAAAA|8842|2|27|22|AM|third|night|| +8843|AAAAAAAAMICCAAAA|8843|2|27|23|AM|third|night|| +8844|AAAAAAAANICCAAAA|8844|2|27|24|AM|third|night|| +8845|AAAAAAAAOICCAAAA|8845|2|27|25|AM|third|night|| +8846|AAAAAAAAPICCAAAA|8846|2|27|26|AM|third|night|| +8847|AAAAAAAAAJCCAAAA|8847|2|27|27|AM|third|night|| +8848|AAAAAAAABJCCAAAA|8848|2|27|28|AM|third|night|| +8849|AAAAAAAACJCCAAAA|8849|2|27|29|AM|third|night|| +8850|AAAAAAAADJCCAAAA|8850|2|27|30|AM|third|night|| +8851|AAAAAAAAEJCCAAAA|8851|2|27|31|AM|third|night|| +8852|AAAAAAAAFJCCAAAA|8852|2|27|32|AM|third|night|| +8853|AAAAAAAAGJCCAAAA|8853|2|27|33|AM|third|night|| +8854|AAAAAAAAHJCCAAAA|8854|2|27|34|AM|third|night|| +8855|AAAAAAAAIJCCAAAA|8855|2|27|35|AM|third|night|| +8856|AAAAAAAAJJCCAAAA|8856|2|27|36|AM|third|night|| +8857|AAAAAAAAKJCCAAAA|8857|2|27|37|AM|third|night|| +8858|AAAAAAAALJCCAAAA|8858|2|27|38|AM|third|night|| +8859|AAAAAAAAMJCCAAAA|8859|2|27|39|AM|third|night|| +8860|AAAAAAAANJCCAAAA|8860|2|27|40|AM|third|night|| +8861|AAAAAAAAOJCCAAAA|8861|2|27|41|AM|third|night|| +8862|AAAAAAAAPJCCAAAA|8862|2|27|42|AM|third|night|| +8863|AAAAAAAAAKCCAAAA|8863|2|27|43|AM|third|night|| +8864|AAAAAAAABKCCAAAA|8864|2|27|44|AM|third|night|| +8865|AAAAAAAACKCCAAAA|8865|2|27|45|AM|third|night|| +8866|AAAAAAAADKCCAAAA|8866|2|27|46|AM|third|night|| +8867|AAAAAAAAEKCCAAAA|8867|2|27|47|AM|third|night|| +8868|AAAAAAAAFKCCAAAA|8868|2|27|48|AM|third|night|| +8869|AAAAAAAAGKCCAAAA|8869|2|27|49|AM|third|night|| +8870|AAAAAAAAHKCCAAAA|8870|2|27|50|AM|third|night|| +8871|AAAAAAAAIKCCAAAA|8871|2|27|51|AM|third|night|| +8872|AAAAAAAAJKCCAAAA|8872|2|27|52|AM|third|night|| +8873|AAAAAAAAKKCCAAAA|8873|2|27|53|AM|third|night|| +8874|AAAAAAAALKCCAAAA|8874|2|27|54|AM|third|night|| +8875|AAAAAAAAMKCCAAAA|8875|2|27|55|AM|third|night|| +8876|AAAAAAAANKCCAAAA|8876|2|27|56|AM|third|night|| +8877|AAAAAAAAOKCCAAAA|8877|2|27|57|AM|third|night|| +8878|AAAAAAAAPKCCAAAA|8878|2|27|58|AM|third|night|| +8879|AAAAAAAAALCCAAAA|8879|2|27|59|AM|third|night|| +8880|AAAAAAAABLCCAAAA|8880|2|28|0|AM|third|night|| +8881|AAAAAAAACLCCAAAA|8881|2|28|1|AM|third|night|| +8882|AAAAAAAADLCCAAAA|8882|2|28|2|AM|third|night|| +8883|AAAAAAAAELCCAAAA|8883|2|28|3|AM|third|night|| +8884|AAAAAAAAFLCCAAAA|8884|2|28|4|AM|third|night|| +8885|AAAAAAAAGLCCAAAA|8885|2|28|5|AM|third|night|| +8886|AAAAAAAAHLCCAAAA|8886|2|28|6|AM|third|night|| +8887|AAAAAAAAILCCAAAA|8887|2|28|7|AM|third|night|| +8888|AAAAAAAAJLCCAAAA|8888|2|28|8|AM|third|night|| +8889|AAAAAAAAKLCCAAAA|8889|2|28|9|AM|third|night|| +8890|AAAAAAAALLCCAAAA|8890|2|28|10|AM|third|night|| +8891|AAAAAAAAMLCCAAAA|8891|2|28|11|AM|third|night|| +8892|AAAAAAAANLCCAAAA|8892|2|28|12|AM|third|night|| +8893|AAAAAAAAOLCCAAAA|8893|2|28|13|AM|third|night|| +8894|AAAAAAAAPLCCAAAA|8894|2|28|14|AM|third|night|| +8895|AAAAAAAAAMCCAAAA|8895|2|28|15|AM|third|night|| +8896|AAAAAAAABMCCAAAA|8896|2|28|16|AM|third|night|| +8897|AAAAAAAACMCCAAAA|8897|2|28|17|AM|third|night|| +8898|AAAAAAAADMCCAAAA|8898|2|28|18|AM|third|night|| +8899|AAAAAAAAEMCCAAAA|8899|2|28|19|AM|third|night|| +8900|AAAAAAAAFMCCAAAA|8900|2|28|20|AM|third|night|| +8901|AAAAAAAAGMCCAAAA|8901|2|28|21|AM|third|night|| +8902|AAAAAAAAHMCCAAAA|8902|2|28|22|AM|third|night|| +8903|AAAAAAAAIMCCAAAA|8903|2|28|23|AM|third|night|| +8904|AAAAAAAAJMCCAAAA|8904|2|28|24|AM|third|night|| +8905|AAAAAAAAKMCCAAAA|8905|2|28|25|AM|third|night|| +8906|AAAAAAAALMCCAAAA|8906|2|28|26|AM|third|night|| +8907|AAAAAAAAMMCCAAAA|8907|2|28|27|AM|third|night|| +8908|AAAAAAAANMCCAAAA|8908|2|28|28|AM|third|night|| +8909|AAAAAAAAOMCCAAAA|8909|2|28|29|AM|third|night|| +8910|AAAAAAAAPMCCAAAA|8910|2|28|30|AM|third|night|| +8911|AAAAAAAAANCCAAAA|8911|2|28|31|AM|third|night|| +8912|AAAAAAAABNCCAAAA|8912|2|28|32|AM|third|night|| +8913|AAAAAAAACNCCAAAA|8913|2|28|33|AM|third|night|| +8914|AAAAAAAADNCCAAAA|8914|2|28|34|AM|third|night|| +8915|AAAAAAAAENCCAAAA|8915|2|28|35|AM|third|night|| +8916|AAAAAAAAFNCCAAAA|8916|2|28|36|AM|third|night|| +8917|AAAAAAAAGNCCAAAA|8917|2|28|37|AM|third|night|| +8918|AAAAAAAAHNCCAAAA|8918|2|28|38|AM|third|night|| +8919|AAAAAAAAINCCAAAA|8919|2|28|39|AM|third|night|| +8920|AAAAAAAAJNCCAAAA|8920|2|28|40|AM|third|night|| +8921|AAAAAAAAKNCCAAAA|8921|2|28|41|AM|third|night|| +8922|AAAAAAAALNCCAAAA|8922|2|28|42|AM|third|night|| +8923|AAAAAAAAMNCCAAAA|8923|2|28|43|AM|third|night|| +8924|AAAAAAAANNCCAAAA|8924|2|28|44|AM|third|night|| +8925|AAAAAAAAONCCAAAA|8925|2|28|45|AM|third|night|| +8926|AAAAAAAAPNCCAAAA|8926|2|28|46|AM|third|night|| +8927|AAAAAAAAAOCCAAAA|8927|2|28|47|AM|third|night|| +8928|AAAAAAAABOCCAAAA|8928|2|28|48|AM|third|night|| +8929|AAAAAAAACOCCAAAA|8929|2|28|49|AM|third|night|| +8930|AAAAAAAADOCCAAAA|8930|2|28|50|AM|third|night|| +8931|AAAAAAAAEOCCAAAA|8931|2|28|51|AM|third|night|| +8932|AAAAAAAAFOCCAAAA|8932|2|28|52|AM|third|night|| +8933|AAAAAAAAGOCCAAAA|8933|2|28|53|AM|third|night|| +8934|AAAAAAAAHOCCAAAA|8934|2|28|54|AM|third|night|| +8935|AAAAAAAAIOCCAAAA|8935|2|28|55|AM|third|night|| +8936|AAAAAAAAJOCCAAAA|8936|2|28|56|AM|third|night|| +8937|AAAAAAAAKOCCAAAA|8937|2|28|57|AM|third|night|| +8938|AAAAAAAALOCCAAAA|8938|2|28|58|AM|third|night|| +8939|AAAAAAAAMOCCAAAA|8939|2|28|59|AM|third|night|| +8940|AAAAAAAANOCCAAAA|8940|2|29|0|AM|third|night|| +8941|AAAAAAAAOOCCAAAA|8941|2|29|1|AM|third|night|| +8942|AAAAAAAAPOCCAAAA|8942|2|29|2|AM|third|night|| +8943|AAAAAAAAAPCCAAAA|8943|2|29|3|AM|third|night|| +8944|AAAAAAAABPCCAAAA|8944|2|29|4|AM|third|night|| +8945|AAAAAAAACPCCAAAA|8945|2|29|5|AM|third|night|| +8946|AAAAAAAADPCCAAAA|8946|2|29|6|AM|third|night|| +8947|AAAAAAAAEPCCAAAA|8947|2|29|7|AM|third|night|| +8948|AAAAAAAAFPCCAAAA|8948|2|29|8|AM|third|night|| +8949|AAAAAAAAGPCCAAAA|8949|2|29|9|AM|third|night|| +8950|AAAAAAAAHPCCAAAA|8950|2|29|10|AM|third|night|| +8951|AAAAAAAAIPCCAAAA|8951|2|29|11|AM|third|night|| +8952|AAAAAAAAJPCCAAAA|8952|2|29|12|AM|third|night|| +8953|AAAAAAAAKPCCAAAA|8953|2|29|13|AM|third|night|| +8954|AAAAAAAALPCCAAAA|8954|2|29|14|AM|third|night|| +8955|AAAAAAAAMPCCAAAA|8955|2|29|15|AM|third|night|| +8956|AAAAAAAANPCCAAAA|8956|2|29|16|AM|third|night|| +8957|AAAAAAAAOPCCAAAA|8957|2|29|17|AM|third|night|| +8958|AAAAAAAAPPCCAAAA|8958|2|29|18|AM|third|night|| +8959|AAAAAAAAAADCAAAA|8959|2|29|19|AM|third|night|| +8960|AAAAAAAABADCAAAA|8960|2|29|20|AM|third|night|| +8961|AAAAAAAACADCAAAA|8961|2|29|21|AM|third|night|| +8962|AAAAAAAADADCAAAA|8962|2|29|22|AM|third|night|| +8963|AAAAAAAAEADCAAAA|8963|2|29|23|AM|third|night|| +8964|AAAAAAAAFADCAAAA|8964|2|29|24|AM|third|night|| +8965|AAAAAAAAGADCAAAA|8965|2|29|25|AM|third|night|| +8966|AAAAAAAAHADCAAAA|8966|2|29|26|AM|third|night|| +8967|AAAAAAAAIADCAAAA|8967|2|29|27|AM|third|night|| +8968|AAAAAAAAJADCAAAA|8968|2|29|28|AM|third|night|| +8969|AAAAAAAAKADCAAAA|8969|2|29|29|AM|third|night|| +8970|AAAAAAAALADCAAAA|8970|2|29|30|AM|third|night|| +8971|AAAAAAAAMADCAAAA|8971|2|29|31|AM|third|night|| +8972|AAAAAAAANADCAAAA|8972|2|29|32|AM|third|night|| +8973|AAAAAAAAOADCAAAA|8973|2|29|33|AM|third|night|| +8974|AAAAAAAAPADCAAAA|8974|2|29|34|AM|third|night|| +8975|AAAAAAAAABDCAAAA|8975|2|29|35|AM|third|night|| +8976|AAAAAAAABBDCAAAA|8976|2|29|36|AM|third|night|| +8977|AAAAAAAACBDCAAAA|8977|2|29|37|AM|third|night|| +8978|AAAAAAAADBDCAAAA|8978|2|29|38|AM|third|night|| +8979|AAAAAAAAEBDCAAAA|8979|2|29|39|AM|third|night|| +8980|AAAAAAAAFBDCAAAA|8980|2|29|40|AM|third|night|| +8981|AAAAAAAAGBDCAAAA|8981|2|29|41|AM|third|night|| +8982|AAAAAAAAHBDCAAAA|8982|2|29|42|AM|third|night|| +8983|AAAAAAAAIBDCAAAA|8983|2|29|43|AM|third|night|| +8984|AAAAAAAAJBDCAAAA|8984|2|29|44|AM|third|night|| +8985|AAAAAAAAKBDCAAAA|8985|2|29|45|AM|third|night|| +8986|AAAAAAAALBDCAAAA|8986|2|29|46|AM|third|night|| +8987|AAAAAAAAMBDCAAAA|8987|2|29|47|AM|third|night|| +8988|AAAAAAAANBDCAAAA|8988|2|29|48|AM|third|night|| +8989|AAAAAAAAOBDCAAAA|8989|2|29|49|AM|third|night|| +8990|AAAAAAAAPBDCAAAA|8990|2|29|50|AM|third|night|| +8991|AAAAAAAAACDCAAAA|8991|2|29|51|AM|third|night|| +8992|AAAAAAAABCDCAAAA|8992|2|29|52|AM|third|night|| +8993|AAAAAAAACCDCAAAA|8993|2|29|53|AM|third|night|| +8994|AAAAAAAADCDCAAAA|8994|2|29|54|AM|third|night|| +8995|AAAAAAAAECDCAAAA|8995|2|29|55|AM|third|night|| +8996|AAAAAAAAFCDCAAAA|8996|2|29|56|AM|third|night|| +8997|AAAAAAAAGCDCAAAA|8997|2|29|57|AM|third|night|| +8998|AAAAAAAAHCDCAAAA|8998|2|29|58|AM|third|night|| +8999|AAAAAAAAICDCAAAA|8999|2|29|59|AM|third|night|| +9000|AAAAAAAAJCDCAAAA|9000|2|30|0|AM|third|night|| +9001|AAAAAAAAKCDCAAAA|9001|2|30|1|AM|third|night|| +9002|AAAAAAAALCDCAAAA|9002|2|30|2|AM|third|night|| +9003|AAAAAAAAMCDCAAAA|9003|2|30|3|AM|third|night|| +9004|AAAAAAAANCDCAAAA|9004|2|30|4|AM|third|night|| +9005|AAAAAAAAOCDCAAAA|9005|2|30|5|AM|third|night|| +9006|AAAAAAAAPCDCAAAA|9006|2|30|6|AM|third|night|| +9007|AAAAAAAAADDCAAAA|9007|2|30|7|AM|third|night|| +9008|AAAAAAAABDDCAAAA|9008|2|30|8|AM|third|night|| +9009|AAAAAAAACDDCAAAA|9009|2|30|9|AM|third|night|| +9010|AAAAAAAADDDCAAAA|9010|2|30|10|AM|third|night|| +9011|AAAAAAAAEDDCAAAA|9011|2|30|11|AM|third|night|| +9012|AAAAAAAAFDDCAAAA|9012|2|30|12|AM|third|night|| +9013|AAAAAAAAGDDCAAAA|9013|2|30|13|AM|third|night|| +9014|AAAAAAAAHDDCAAAA|9014|2|30|14|AM|third|night|| +9015|AAAAAAAAIDDCAAAA|9015|2|30|15|AM|third|night|| +9016|AAAAAAAAJDDCAAAA|9016|2|30|16|AM|third|night|| +9017|AAAAAAAAKDDCAAAA|9017|2|30|17|AM|third|night|| +9018|AAAAAAAALDDCAAAA|9018|2|30|18|AM|third|night|| +9019|AAAAAAAAMDDCAAAA|9019|2|30|19|AM|third|night|| +9020|AAAAAAAANDDCAAAA|9020|2|30|20|AM|third|night|| +9021|AAAAAAAAODDCAAAA|9021|2|30|21|AM|third|night|| +9022|AAAAAAAAPDDCAAAA|9022|2|30|22|AM|third|night|| +9023|AAAAAAAAAEDCAAAA|9023|2|30|23|AM|third|night|| +9024|AAAAAAAABEDCAAAA|9024|2|30|24|AM|third|night|| +9025|AAAAAAAACEDCAAAA|9025|2|30|25|AM|third|night|| +9026|AAAAAAAADEDCAAAA|9026|2|30|26|AM|third|night|| +9027|AAAAAAAAEEDCAAAA|9027|2|30|27|AM|third|night|| +9028|AAAAAAAAFEDCAAAA|9028|2|30|28|AM|third|night|| +9029|AAAAAAAAGEDCAAAA|9029|2|30|29|AM|third|night|| +9030|AAAAAAAAHEDCAAAA|9030|2|30|30|AM|third|night|| +9031|AAAAAAAAIEDCAAAA|9031|2|30|31|AM|third|night|| +9032|AAAAAAAAJEDCAAAA|9032|2|30|32|AM|third|night|| +9033|AAAAAAAAKEDCAAAA|9033|2|30|33|AM|third|night|| +9034|AAAAAAAALEDCAAAA|9034|2|30|34|AM|third|night|| +9035|AAAAAAAAMEDCAAAA|9035|2|30|35|AM|third|night|| +9036|AAAAAAAANEDCAAAA|9036|2|30|36|AM|third|night|| +9037|AAAAAAAAOEDCAAAA|9037|2|30|37|AM|third|night|| +9038|AAAAAAAAPEDCAAAA|9038|2|30|38|AM|third|night|| +9039|AAAAAAAAAFDCAAAA|9039|2|30|39|AM|third|night|| +9040|AAAAAAAABFDCAAAA|9040|2|30|40|AM|third|night|| +9041|AAAAAAAACFDCAAAA|9041|2|30|41|AM|third|night|| +9042|AAAAAAAADFDCAAAA|9042|2|30|42|AM|third|night|| +9043|AAAAAAAAEFDCAAAA|9043|2|30|43|AM|third|night|| +9044|AAAAAAAAFFDCAAAA|9044|2|30|44|AM|third|night|| +9045|AAAAAAAAGFDCAAAA|9045|2|30|45|AM|third|night|| +9046|AAAAAAAAHFDCAAAA|9046|2|30|46|AM|third|night|| +9047|AAAAAAAAIFDCAAAA|9047|2|30|47|AM|third|night|| +9048|AAAAAAAAJFDCAAAA|9048|2|30|48|AM|third|night|| +9049|AAAAAAAAKFDCAAAA|9049|2|30|49|AM|third|night|| +9050|AAAAAAAALFDCAAAA|9050|2|30|50|AM|third|night|| +9051|AAAAAAAAMFDCAAAA|9051|2|30|51|AM|third|night|| +9052|AAAAAAAANFDCAAAA|9052|2|30|52|AM|third|night|| +9053|AAAAAAAAOFDCAAAA|9053|2|30|53|AM|third|night|| +9054|AAAAAAAAPFDCAAAA|9054|2|30|54|AM|third|night|| +9055|AAAAAAAAAGDCAAAA|9055|2|30|55|AM|third|night|| +9056|AAAAAAAABGDCAAAA|9056|2|30|56|AM|third|night|| +9057|AAAAAAAACGDCAAAA|9057|2|30|57|AM|third|night|| +9058|AAAAAAAADGDCAAAA|9058|2|30|58|AM|third|night|| +9059|AAAAAAAAEGDCAAAA|9059|2|30|59|AM|third|night|| +9060|AAAAAAAAFGDCAAAA|9060|2|31|0|AM|third|night|| +9061|AAAAAAAAGGDCAAAA|9061|2|31|1|AM|third|night|| +9062|AAAAAAAAHGDCAAAA|9062|2|31|2|AM|third|night|| +9063|AAAAAAAAIGDCAAAA|9063|2|31|3|AM|third|night|| +9064|AAAAAAAAJGDCAAAA|9064|2|31|4|AM|third|night|| +9065|AAAAAAAAKGDCAAAA|9065|2|31|5|AM|third|night|| +9066|AAAAAAAALGDCAAAA|9066|2|31|6|AM|third|night|| +9067|AAAAAAAAMGDCAAAA|9067|2|31|7|AM|third|night|| +9068|AAAAAAAANGDCAAAA|9068|2|31|8|AM|third|night|| +9069|AAAAAAAAOGDCAAAA|9069|2|31|9|AM|third|night|| +9070|AAAAAAAAPGDCAAAA|9070|2|31|10|AM|third|night|| +9071|AAAAAAAAAHDCAAAA|9071|2|31|11|AM|third|night|| +9072|AAAAAAAABHDCAAAA|9072|2|31|12|AM|third|night|| +9073|AAAAAAAACHDCAAAA|9073|2|31|13|AM|third|night|| +9074|AAAAAAAADHDCAAAA|9074|2|31|14|AM|third|night|| +9075|AAAAAAAAEHDCAAAA|9075|2|31|15|AM|third|night|| +9076|AAAAAAAAFHDCAAAA|9076|2|31|16|AM|third|night|| +9077|AAAAAAAAGHDCAAAA|9077|2|31|17|AM|third|night|| +9078|AAAAAAAAHHDCAAAA|9078|2|31|18|AM|third|night|| +9079|AAAAAAAAIHDCAAAA|9079|2|31|19|AM|third|night|| +9080|AAAAAAAAJHDCAAAA|9080|2|31|20|AM|third|night|| +9081|AAAAAAAAKHDCAAAA|9081|2|31|21|AM|third|night|| +9082|AAAAAAAALHDCAAAA|9082|2|31|22|AM|third|night|| +9083|AAAAAAAAMHDCAAAA|9083|2|31|23|AM|third|night|| +9084|AAAAAAAANHDCAAAA|9084|2|31|24|AM|third|night|| +9085|AAAAAAAAOHDCAAAA|9085|2|31|25|AM|third|night|| +9086|AAAAAAAAPHDCAAAA|9086|2|31|26|AM|third|night|| +9087|AAAAAAAAAIDCAAAA|9087|2|31|27|AM|third|night|| +9088|AAAAAAAABIDCAAAA|9088|2|31|28|AM|third|night|| +9089|AAAAAAAACIDCAAAA|9089|2|31|29|AM|third|night|| +9090|AAAAAAAADIDCAAAA|9090|2|31|30|AM|third|night|| +9091|AAAAAAAAEIDCAAAA|9091|2|31|31|AM|third|night|| +9092|AAAAAAAAFIDCAAAA|9092|2|31|32|AM|third|night|| +9093|AAAAAAAAGIDCAAAA|9093|2|31|33|AM|third|night|| +9094|AAAAAAAAHIDCAAAA|9094|2|31|34|AM|third|night|| +9095|AAAAAAAAIIDCAAAA|9095|2|31|35|AM|third|night|| +9096|AAAAAAAAJIDCAAAA|9096|2|31|36|AM|third|night|| +9097|AAAAAAAAKIDCAAAA|9097|2|31|37|AM|third|night|| +9098|AAAAAAAALIDCAAAA|9098|2|31|38|AM|third|night|| +9099|AAAAAAAAMIDCAAAA|9099|2|31|39|AM|third|night|| +9100|AAAAAAAANIDCAAAA|9100|2|31|40|AM|third|night|| +9101|AAAAAAAAOIDCAAAA|9101|2|31|41|AM|third|night|| +9102|AAAAAAAAPIDCAAAA|9102|2|31|42|AM|third|night|| +9103|AAAAAAAAAJDCAAAA|9103|2|31|43|AM|third|night|| +9104|AAAAAAAABJDCAAAA|9104|2|31|44|AM|third|night|| +9105|AAAAAAAACJDCAAAA|9105|2|31|45|AM|third|night|| +9106|AAAAAAAADJDCAAAA|9106|2|31|46|AM|third|night|| +9107|AAAAAAAAEJDCAAAA|9107|2|31|47|AM|third|night|| +9108|AAAAAAAAFJDCAAAA|9108|2|31|48|AM|third|night|| +9109|AAAAAAAAGJDCAAAA|9109|2|31|49|AM|third|night|| +9110|AAAAAAAAHJDCAAAA|9110|2|31|50|AM|third|night|| +9111|AAAAAAAAIJDCAAAA|9111|2|31|51|AM|third|night|| +9112|AAAAAAAAJJDCAAAA|9112|2|31|52|AM|third|night|| +9113|AAAAAAAAKJDCAAAA|9113|2|31|53|AM|third|night|| +9114|AAAAAAAALJDCAAAA|9114|2|31|54|AM|third|night|| +9115|AAAAAAAAMJDCAAAA|9115|2|31|55|AM|third|night|| +9116|AAAAAAAANJDCAAAA|9116|2|31|56|AM|third|night|| +9117|AAAAAAAAOJDCAAAA|9117|2|31|57|AM|third|night|| +9118|AAAAAAAAPJDCAAAA|9118|2|31|58|AM|third|night|| +9119|AAAAAAAAAKDCAAAA|9119|2|31|59|AM|third|night|| +9120|AAAAAAAABKDCAAAA|9120|2|32|0|AM|third|night|| +9121|AAAAAAAACKDCAAAA|9121|2|32|1|AM|third|night|| +9122|AAAAAAAADKDCAAAA|9122|2|32|2|AM|third|night|| +9123|AAAAAAAAEKDCAAAA|9123|2|32|3|AM|third|night|| +9124|AAAAAAAAFKDCAAAA|9124|2|32|4|AM|third|night|| +9125|AAAAAAAAGKDCAAAA|9125|2|32|5|AM|third|night|| +9126|AAAAAAAAHKDCAAAA|9126|2|32|6|AM|third|night|| +9127|AAAAAAAAIKDCAAAA|9127|2|32|7|AM|third|night|| +9128|AAAAAAAAJKDCAAAA|9128|2|32|8|AM|third|night|| +9129|AAAAAAAAKKDCAAAA|9129|2|32|9|AM|third|night|| +9130|AAAAAAAALKDCAAAA|9130|2|32|10|AM|third|night|| +9131|AAAAAAAAMKDCAAAA|9131|2|32|11|AM|third|night|| +9132|AAAAAAAANKDCAAAA|9132|2|32|12|AM|third|night|| +9133|AAAAAAAAOKDCAAAA|9133|2|32|13|AM|third|night|| +9134|AAAAAAAAPKDCAAAA|9134|2|32|14|AM|third|night|| +9135|AAAAAAAAALDCAAAA|9135|2|32|15|AM|third|night|| +9136|AAAAAAAABLDCAAAA|9136|2|32|16|AM|third|night|| +9137|AAAAAAAACLDCAAAA|9137|2|32|17|AM|third|night|| +9138|AAAAAAAADLDCAAAA|9138|2|32|18|AM|third|night|| +9139|AAAAAAAAELDCAAAA|9139|2|32|19|AM|third|night|| +9140|AAAAAAAAFLDCAAAA|9140|2|32|20|AM|third|night|| +9141|AAAAAAAAGLDCAAAA|9141|2|32|21|AM|third|night|| +9142|AAAAAAAAHLDCAAAA|9142|2|32|22|AM|third|night|| +9143|AAAAAAAAILDCAAAA|9143|2|32|23|AM|third|night|| +9144|AAAAAAAAJLDCAAAA|9144|2|32|24|AM|third|night|| +9145|AAAAAAAAKLDCAAAA|9145|2|32|25|AM|third|night|| +9146|AAAAAAAALLDCAAAA|9146|2|32|26|AM|third|night|| +9147|AAAAAAAAMLDCAAAA|9147|2|32|27|AM|third|night|| +9148|AAAAAAAANLDCAAAA|9148|2|32|28|AM|third|night|| +9149|AAAAAAAAOLDCAAAA|9149|2|32|29|AM|third|night|| +9150|AAAAAAAAPLDCAAAA|9150|2|32|30|AM|third|night|| +9151|AAAAAAAAAMDCAAAA|9151|2|32|31|AM|third|night|| +9152|AAAAAAAABMDCAAAA|9152|2|32|32|AM|third|night|| +9153|AAAAAAAACMDCAAAA|9153|2|32|33|AM|third|night|| +9154|AAAAAAAADMDCAAAA|9154|2|32|34|AM|third|night|| +9155|AAAAAAAAEMDCAAAA|9155|2|32|35|AM|third|night|| +9156|AAAAAAAAFMDCAAAA|9156|2|32|36|AM|third|night|| +9157|AAAAAAAAGMDCAAAA|9157|2|32|37|AM|third|night|| +9158|AAAAAAAAHMDCAAAA|9158|2|32|38|AM|third|night|| +9159|AAAAAAAAIMDCAAAA|9159|2|32|39|AM|third|night|| +9160|AAAAAAAAJMDCAAAA|9160|2|32|40|AM|third|night|| +9161|AAAAAAAAKMDCAAAA|9161|2|32|41|AM|third|night|| +9162|AAAAAAAALMDCAAAA|9162|2|32|42|AM|third|night|| +9163|AAAAAAAAMMDCAAAA|9163|2|32|43|AM|third|night|| +9164|AAAAAAAANMDCAAAA|9164|2|32|44|AM|third|night|| +9165|AAAAAAAAOMDCAAAA|9165|2|32|45|AM|third|night|| +9166|AAAAAAAAPMDCAAAA|9166|2|32|46|AM|third|night|| +9167|AAAAAAAAANDCAAAA|9167|2|32|47|AM|third|night|| +9168|AAAAAAAABNDCAAAA|9168|2|32|48|AM|third|night|| +9169|AAAAAAAACNDCAAAA|9169|2|32|49|AM|third|night|| +9170|AAAAAAAADNDCAAAA|9170|2|32|50|AM|third|night|| +9171|AAAAAAAAENDCAAAA|9171|2|32|51|AM|third|night|| +9172|AAAAAAAAFNDCAAAA|9172|2|32|52|AM|third|night|| +9173|AAAAAAAAGNDCAAAA|9173|2|32|53|AM|third|night|| +9174|AAAAAAAAHNDCAAAA|9174|2|32|54|AM|third|night|| +9175|AAAAAAAAINDCAAAA|9175|2|32|55|AM|third|night|| +9176|AAAAAAAAJNDCAAAA|9176|2|32|56|AM|third|night|| +9177|AAAAAAAAKNDCAAAA|9177|2|32|57|AM|third|night|| +9178|AAAAAAAALNDCAAAA|9178|2|32|58|AM|third|night|| +9179|AAAAAAAAMNDCAAAA|9179|2|32|59|AM|third|night|| +9180|AAAAAAAANNDCAAAA|9180|2|33|0|AM|third|night|| +9181|AAAAAAAAONDCAAAA|9181|2|33|1|AM|third|night|| +9182|AAAAAAAAPNDCAAAA|9182|2|33|2|AM|third|night|| +9183|AAAAAAAAAODCAAAA|9183|2|33|3|AM|third|night|| +9184|AAAAAAAABODCAAAA|9184|2|33|4|AM|third|night|| +9185|AAAAAAAACODCAAAA|9185|2|33|5|AM|third|night|| +9186|AAAAAAAADODCAAAA|9186|2|33|6|AM|third|night|| +9187|AAAAAAAAEODCAAAA|9187|2|33|7|AM|third|night|| +9188|AAAAAAAAFODCAAAA|9188|2|33|8|AM|third|night|| +9189|AAAAAAAAGODCAAAA|9189|2|33|9|AM|third|night|| +9190|AAAAAAAAHODCAAAA|9190|2|33|10|AM|third|night|| +9191|AAAAAAAAIODCAAAA|9191|2|33|11|AM|third|night|| +9192|AAAAAAAAJODCAAAA|9192|2|33|12|AM|third|night|| +9193|AAAAAAAAKODCAAAA|9193|2|33|13|AM|third|night|| +9194|AAAAAAAALODCAAAA|9194|2|33|14|AM|third|night|| +9195|AAAAAAAAMODCAAAA|9195|2|33|15|AM|third|night|| +9196|AAAAAAAANODCAAAA|9196|2|33|16|AM|third|night|| +9197|AAAAAAAAOODCAAAA|9197|2|33|17|AM|third|night|| +9198|AAAAAAAAPODCAAAA|9198|2|33|18|AM|third|night|| +9199|AAAAAAAAAPDCAAAA|9199|2|33|19|AM|third|night|| +9200|AAAAAAAABPDCAAAA|9200|2|33|20|AM|third|night|| +9201|AAAAAAAACPDCAAAA|9201|2|33|21|AM|third|night|| +9202|AAAAAAAADPDCAAAA|9202|2|33|22|AM|third|night|| +9203|AAAAAAAAEPDCAAAA|9203|2|33|23|AM|third|night|| +9204|AAAAAAAAFPDCAAAA|9204|2|33|24|AM|third|night|| +9205|AAAAAAAAGPDCAAAA|9205|2|33|25|AM|third|night|| +9206|AAAAAAAAHPDCAAAA|9206|2|33|26|AM|third|night|| +9207|AAAAAAAAIPDCAAAA|9207|2|33|27|AM|third|night|| +9208|AAAAAAAAJPDCAAAA|9208|2|33|28|AM|third|night|| +9209|AAAAAAAAKPDCAAAA|9209|2|33|29|AM|third|night|| +9210|AAAAAAAALPDCAAAA|9210|2|33|30|AM|third|night|| +9211|AAAAAAAAMPDCAAAA|9211|2|33|31|AM|third|night|| +9212|AAAAAAAANPDCAAAA|9212|2|33|32|AM|third|night|| +9213|AAAAAAAAOPDCAAAA|9213|2|33|33|AM|third|night|| +9214|AAAAAAAAPPDCAAAA|9214|2|33|34|AM|third|night|| +9215|AAAAAAAAAAECAAAA|9215|2|33|35|AM|third|night|| +9216|AAAAAAAABAECAAAA|9216|2|33|36|AM|third|night|| +9217|AAAAAAAACAECAAAA|9217|2|33|37|AM|third|night|| +9218|AAAAAAAADAECAAAA|9218|2|33|38|AM|third|night|| +9219|AAAAAAAAEAECAAAA|9219|2|33|39|AM|third|night|| +9220|AAAAAAAAFAECAAAA|9220|2|33|40|AM|third|night|| +9221|AAAAAAAAGAECAAAA|9221|2|33|41|AM|third|night|| +9222|AAAAAAAAHAECAAAA|9222|2|33|42|AM|third|night|| +9223|AAAAAAAAIAECAAAA|9223|2|33|43|AM|third|night|| +9224|AAAAAAAAJAECAAAA|9224|2|33|44|AM|third|night|| +9225|AAAAAAAAKAECAAAA|9225|2|33|45|AM|third|night|| +9226|AAAAAAAALAECAAAA|9226|2|33|46|AM|third|night|| +9227|AAAAAAAAMAECAAAA|9227|2|33|47|AM|third|night|| +9228|AAAAAAAANAECAAAA|9228|2|33|48|AM|third|night|| +9229|AAAAAAAAOAECAAAA|9229|2|33|49|AM|third|night|| +9230|AAAAAAAAPAECAAAA|9230|2|33|50|AM|third|night|| +9231|AAAAAAAAABECAAAA|9231|2|33|51|AM|third|night|| +9232|AAAAAAAABBECAAAA|9232|2|33|52|AM|third|night|| +9233|AAAAAAAACBECAAAA|9233|2|33|53|AM|third|night|| +9234|AAAAAAAADBECAAAA|9234|2|33|54|AM|third|night|| +9235|AAAAAAAAEBECAAAA|9235|2|33|55|AM|third|night|| +9236|AAAAAAAAFBECAAAA|9236|2|33|56|AM|third|night|| +9237|AAAAAAAAGBECAAAA|9237|2|33|57|AM|third|night|| +9238|AAAAAAAAHBECAAAA|9238|2|33|58|AM|third|night|| +9239|AAAAAAAAIBECAAAA|9239|2|33|59|AM|third|night|| +9240|AAAAAAAAJBECAAAA|9240|2|34|0|AM|third|night|| +9241|AAAAAAAAKBECAAAA|9241|2|34|1|AM|third|night|| +9242|AAAAAAAALBECAAAA|9242|2|34|2|AM|third|night|| +9243|AAAAAAAAMBECAAAA|9243|2|34|3|AM|third|night|| +9244|AAAAAAAANBECAAAA|9244|2|34|4|AM|third|night|| +9245|AAAAAAAAOBECAAAA|9245|2|34|5|AM|third|night|| +9246|AAAAAAAAPBECAAAA|9246|2|34|6|AM|third|night|| +9247|AAAAAAAAACECAAAA|9247|2|34|7|AM|third|night|| +9248|AAAAAAAABCECAAAA|9248|2|34|8|AM|third|night|| +9249|AAAAAAAACCECAAAA|9249|2|34|9|AM|third|night|| +9250|AAAAAAAADCECAAAA|9250|2|34|10|AM|third|night|| +9251|AAAAAAAAECECAAAA|9251|2|34|11|AM|third|night|| +9252|AAAAAAAAFCECAAAA|9252|2|34|12|AM|third|night|| +9253|AAAAAAAAGCECAAAA|9253|2|34|13|AM|third|night|| +9254|AAAAAAAAHCECAAAA|9254|2|34|14|AM|third|night|| +9255|AAAAAAAAICECAAAA|9255|2|34|15|AM|third|night|| +9256|AAAAAAAAJCECAAAA|9256|2|34|16|AM|third|night|| +9257|AAAAAAAAKCECAAAA|9257|2|34|17|AM|third|night|| +9258|AAAAAAAALCECAAAA|9258|2|34|18|AM|third|night|| +9259|AAAAAAAAMCECAAAA|9259|2|34|19|AM|third|night|| +9260|AAAAAAAANCECAAAA|9260|2|34|20|AM|third|night|| +9261|AAAAAAAAOCECAAAA|9261|2|34|21|AM|third|night|| +9262|AAAAAAAAPCECAAAA|9262|2|34|22|AM|third|night|| +9263|AAAAAAAAADECAAAA|9263|2|34|23|AM|third|night|| +9264|AAAAAAAABDECAAAA|9264|2|34|24|AM|third|night|| +9265|AAAAAAAACDECAAAA|9265|2|34|25|AM|third|night|| +9266|AAAAAAAADDECAAAA|9266|2|34|26|AM|third|night|| +9267|AAAAAAAAEDECAAAA|9267|2|34|27|AM|third|night|| +9268|AAAAAAAAFDECAAAA|9268|2|34|28|AM|third|night|| +9269|AAAAAAAAGDECAAAA|9269|2|34|29|AM|third|night|| +9270|AAAAAAAAHDECAAAA|9270|2|34|30|AM|third|night|| +9271|AAAAAAAAIDECAAAA|9271|2|34|31|AM|third|night|| +9272|AAAAAAAAJDECAAAA|9272|2|34|32|AM|third|night|| +9273|AAAAAAAAKDECAAAA|9273|2|34|33|AM|third|night|| +9274|AAAAAAAALDECAAAA|9274|2|34|34|AM|third|night|| +9275|AAAAAAAAMDECAAAA|9275|2|34|35|AM|third|night|| +9276|AAAAAAAANDECAAAA|9276|2|34|36|AM|third|night|| +9277|AAAAAAAAODECAAAA|9277|2|34|37|AM|third|night|| +9278|AAAAAAAAPDECAAAA|9278|2|34|38|AM|third|night|| +9279|AAAAAAAAAEECAAAA|9279|2|34|39|AM|third|night|| +9280|AAAAAAAABEECAAAA|9280|2|34|40|AM|third|night|| +9281|AAAAAAAACEECAAAA|9281|2|34|41|AM|third|night|| +9282|AAAAAAAADEECAAAA|9282|2|34|42|AM|third|night|| +9283|AAAAAAAAEEECAAAA|9283|2|34|43|AM|third|night|| +9284|AAAAAAAAFEECAAAA|9284|2|34|44|AM|third|night|| +9285|AAAAAAAAGEECAAAA|9285|2|34|45|AM|third|night|| +9286|AAAAAAAAHEECAAAA|9286|2|34|46|AM|third|night|| +9287|AAAAAAAAIEECAAAA|9287|2|34|47|AM|third|night|| +9288|AAAAAAAAJEECAAAA|9288|2|34|48|AM|third|night|| +9289|AAAAAAAAKEECAAAA|9289|2|34|49|AM|third|night|| +9290|AAAAAAAALEECAAAA|9290|2|34|50|AM|third|night|| +9291|AAAAAAAAMEECAAAA|9291|2|34|51|AM|third|night|| +9292|AAAAAAAANEECAAAA|9292|2|34|52|AM|third|night|| +9293|AAAAAAAAOEECAAAA|9293|2|34|53|AM|third|night|| +9294|AAAAAAAAPEECAAAA|9294|2|34|54|AM|third|night|| +9295|AAAAAAAAAFECAAAA|9295|2|34|55|AM|third|night|| +9296|AAAAAAAABFECAAAA|9296|2|34|56|AM|third|night|| +9297|AAAAAAAACFECAAAA|9297|2|34|57|AM|third|night|| +9298|AAAAAAAADFECAAAA|9298|2|34|58|AM|third|night|| +9299|AAAAAAAAEFECAAAA|9299|2|34|59|AM|third|night|| +9300|AAAAAAAAFFECAAAA|9300|2|35|0|AM|third|night|| +9301|AAAAAAAAGFECAAAA|9301|2|35|1|AM|third|night|| +9302|AAAAAAAAHFECAAAA|9302|2|35|2|AM|third|night|| +9303|AAAAAAAAIFECAAAA|9303|2|35|3|AM|third|night|| +9304|AAAAAAAAJFECAAAA|9304|2|35|4|AM|third|night|| +9305|AAAAAAAAKFECAAAA|9305|2|35|5|AM|third|night|| +9306|AAAAAAAALFECAAAA|9306|2|35|6|AM|third|night|| +9307|AAAAAAAAMFECAAAA|9307|2|35|7|AM|third|night|| +9308|AAAAAAAANFECAAAA|9308|2|35|8|AM|third|night|| +9309|AAAAAAAAOFECAAAA|9309|2|35|9|AM|third|night|| +9310|AAAAAAAAPFECAAAA|9310|2|35|10|AM|third|night|| +9311|AAAAAAAAAGECAAAA|9311|2|35|11|AM|third|night|| +9312|AAAAAAAABGECAAAA|9312|2|35|12|AM|third|night|| +9313|AAAAAAAACGECAAAA|9313|2|35|13|AM|third|night|| +9314|AAAAAAAADGECAAAA|9314|2|35|14|AM|third|night|| +9315|AAAAAAAAEGECAAAA|9315|2|35|15|AM|third|night|| +9316|AAAAAAAAFGECAAAA|9316|2|35|16|AM|third|night|| +9317|AAAAAAAAGGECAAAA|9317|2|35|17|AM|third|night|| +9318|AAAAAAAAHGECAAAA|9318|2|35|18|AM|third|night|| +9319|AAAAAAAAIGECAAAA|9319|2|35|19|AM|third|night|| +9320|AAAAAAAAJGECAAAA|9320|2|35|20|AM|third|night|| +9321|AAAAAAAAKGECAAAA|9321|2|35|21|AM|third|night|| +9322|AAAAAAAALGECAAAA|9322|2|35|22|AM|third|night|| +9323|AAAAAAAAMGECAAAA|9323|2|35|23|AM|third|night|| +9324|AAAAAAAANGECAAAA|9324|2|35|24|AM|third|night|| +9325|AAAAAAAAOGECAAAA|9325|2|35|25|AM|third|night|| +9326|AAAAAAAAPGECAAAA|9326|2|35|26|AM|third|night|| +9327|AAAAAAAAAHECAAAA|9327|2|35|27|AM|third|night|| +9328|AAAAAAAABHECAAAA|9328|2|35|28|AM|third|night|| +9329|AAAAAAAACHECAAAA|9329|2|35|29|AM|third|night|| +9330|AAAAAAAADHECAAAA|9330|2|35|30|AM|third|night|| +9331|AAAAAAAAEHECAAAA|9331|2|35|31|AM|third|night|| +9332|AAAAAAAAFHECAAAA|9332|2|35|32|AM|third|night|| +9333|AAAAAAAAGHECAAAA|9333|2|35|33|AM|third|night|| +9334|AAAAAAAAHHECAAAA|9334|2|35|34|AM|third|night|| +9335|AAAAAAAAIHECAAAA|9335|2|35|35|AM|third|night|| +9336|AAAAAAAAJHECAAAA|9336|2|35|36|AM|third|night|| +9337|AAAAAAAAKHECAAAA|9337|2|35|37|AM|third|night|| +9338|AAAAAAAALHECAAAA|9338|2|35|38|AM|third|night|| +9339|AAAAAAAAMHECAAAA|9339|2|35|39|AM|third|night|| +9340|AAAAAAAANHECAAAA|9340|2|35|40|AM|third|night|| +9341|AAAAAAAAOHECAAAA|9341|2|35|41|AM|third|night|| +9342|AAAAAAAAPHECAAAA|9342|2|35|42|AM|third|night|| +9343|AAAAAAAAAIECAAAA|9343|2|35|43|AM|third|night|| +9344|AAAAAAAABIECAAAA|9344|2|35|44|AM|third|night|| +9345|AAAAAAAACIECAAAA|9345|2|35|45|AM|third|night|| +9346|AAAAAAAADIECAAAA|9346|2|35|46|AM|third|night|| +9347|AAAAAAAAEIECAAAA|9347|2|35|47|AM|third|night|| +9348|AAAAAAAAFIECAAAA|9348|2|35|48|AM|third|night|| +9349|AAAAAAAAGIECAAAA|9349|2|35|49|AM|third|night|| +9350|AAAAAAAAHIECAAAA|9350|2|35|50|AM|third|night|| +9351|AAAAAAAAIIECAAAA|9351|2|35|51|AM|third|night|| +9352|AAAAAAAAJIECAAAA|9352|2|35|52|AM|third|night|| +9353|AAAAAAAAKIECAAAA|9353|2|35|53|AM|third|night|| +9354|AAAAAAAALIECAAAA|9354|2|35|54|AM|third|night|| +9355|AAAAAAAAMIECAAAA|9355|2|35|55|AM|third|night|| +9356|AAAAAAAANIECAAAA|9356|2|35|56|AM|third|night|| +9357|AAAAAAAAOIECAAAA|9357|2|35|57|AM|third|night|| +9358|AAAAAAAAPIECAAAA|9358|2|35|58|AM|third|night|| +9359|AAAAAAAAAJECAAAA|9359|2|35|59|AM|third|night|| +9360|AAAAAAAABJECAAAA|9360|2|36|0|AM|third|night|| +9361|AAAAAAAACJECAAAA|9361|2|36|1|AM|third|night|| +9362|AAAAAAAADJECAAAA|9362|2|36|2|AM|third|night|| +9363|AAAAAAAAEJECAAAA|9363|2|36|3|AM|third|night|| +9364|AAAAAAAAFJECAAAA|9364|2|36|4|AM|third|night|| +9365|AAAAAAAAGJECAAAA|9365|2|36|5|AM|third|night|| +9366|AAAAAAAAHJECAAAA|9366|2|36|6|AM|third|night|| +9367|AAAAAAAAIJECAAAA|9367|2|36|7|AM|third|night|| +9368|AAAAAAAAJJECAAAA|9368|2|36|8|AM|third|night|| +9369|AAAAAAAAKJECAAAA|9369|2|36|9|AM|third|night|| +9370|AAAAAAAALJECAAAA|9370|2|36|10|AM|third|night|| +9371|AAAAAAAAMJECAAAA|9371|2|36|11|AM|third|night|| +9372|AAAAAAAANJECAAAA|9372|2|36|12|AM|third|night|| +9373|AAAAAAAAOJECAAAA|9373|2|36|13|AM|third|night|| +9374|AAAAAAAAPJECAAAA|9374|2|36|14|AM|third|night|| +9375|AAAAAAAAAKECAAAA|9375|2|36|15|AM|third|night|| +9376|AAAAAAAABKECAAAA|9376|2|36|16|AM|third|night|| +9377|AAAAAAAACKECAAAA|9377|2|36|17|AM|third|night|| +9378|AAAAAAAADKECAAAA|9378|2|36|18|AM|third|night|| +9379|AAAAAAAAEKECAAAA|9379|2|36|19|AM|third|night|| +9380|AAAAAAAAFKECAAAA|9380|2|36|20|AM|third|night|| +9381|AAAAAAAAGKECAAAA|9381|2|36|21|AM|third|night|| +9382|AAAAAAAAHKECAAAA|9382|2|36|22|AM|third|night|| +9383|AAAAAAAAIKECAAAA|9383|2|36|23|AM|third|night|| +9384|AAAAAAAAJKECAAAA|9384|2|36|24|AM|third|night|| +9385|AAAAAAAAKKECAAAA|9385|2|36|25|AM|third|night|| +9386|AAAAAAAALKECAAAA|9386|2|36|26|AM|third|night|| +9387|AAAAAAAAMKECAAAA|9387|2|36|27|AM|third|night|| +9388|AAAAAAAANKECAAAA|9388|2|36|28|AM|third|night|| +9389|AAAAAAAAOKECAAAA|9389|2|36|29|AM|third|night|| +9390|AAAAAAAAPKECAAAA|9390|2|36|30|AM|third|night|| +9391|AAAAAAAAALECAAAA|9391|2|36|31|AM|third|night|| +9392|AAAAAAAABLECAAAA|9392|2|36|32|AM|third|night|| +9393|AAAAAAAACLECAAAA|9393|2|36|33|AM|third|night|| +9394|AAAAAAAADLECAAAA|9394|2|36|34|AM|third|night|| +9395|AAAAAAAAELECAAAA|9395|2|36|35|AM|third|night|| +9396|AAAAAAAAFLECAAAA|9396|2|36|36|AM|third|night|| +9397|AAAAAAAAGLECAAAA|9397|2|36|37|AM|third|night|| +9398|AAAAAAAAHLECAAAA|9398|2|36|38|AM|third|night|| +9399|AAAAAAAAILECAAAA|9399|2|36|39|AM|third|night|| +9400|AAAAAAAAJLECAAAA|9400|2|36|40|AM|third|night|| +9401|AAAAAAAAKLECAAAA|9401|2|36|41|AM|third|night|| +9402|AAAAAAAALLECAAAA|9402|2|36|42|AM|third|night|| +9403|AAAAAAAAMLECAAAA|9403|2|36|43|AM|third|night|| +9404|AAAAAAAANLECAAAA|9404|2|36|44|AM|third|night|| +9405|AAAAAAAAOLECAAAA|9405|2|36|45|AM|third|night|| +9406|AAAAAAAAPLECAAAA|9406|2|36|46|AM|third|night|| +9407|AAAAAAAAAMECAAAA|9407|2|36|47|AM|third|night|| +9408|AAAAAAAABMECAAAA|9408|2|36|48|AM|third|night|| +9409|AAAAAAAACMECAAAA|9409|2|36|49|AM|third|night|| +9410|AAAAAAAADMECAAAA|9410|2|36|50|AM|third|night|| +9411|AAAAAAAAEMECAAAA|9411|2|36|51|AM|third|night|| +9412|AAAAAAAAFMECAAAA|9412|2|36|52|AM|third|night|| +9413|AAAAAAAAGMECAAAA|9413|2|36|53|AM|third|night|| +9414|AAAAAAAAHMECAAAA|9414|2|36|54|AM|third|night|| +9415|AAAAAAAAIMECAAAA|9415|2|36|55|AM|third|night|| +9416|AAAAAAAAJMECAAAA|9416|2|36|56|AM|third|night|| +9417|AAAAAAAAKMECAAAA|9417|2|36|57|AM|third|night|| +9418|AAAAAAAALMECAAAA|9418|2|36|58|AM|third|night|| +9419|AAAAAAAAMMECAAAA|9419|2|36|59|AM|third|night|| +9420|AAAAAAAANMECAAAA|9420|2|37|0|AM|third|night|| +9421|AAAAAAAAOMECAAAA|9421|2|37|1|AM|third|night|| +9422|AAAAAAAAPMECAAAA|9422|2|37|2|AM|third|night|| +9423|AAAAAAAAANECAAAA|9423|2|37|3|AM|third|night|| +9424|AAAAAAAABNECAAAA|9424|2|37|4|AM|third|night|| +9425|AAAAAAAACNECAAAA|9425|2|37|5|AM|third|night|| +9426|AAAAAAAADNECAAAA|9426|2|37|6|AM|third|night|| +9427|AAAAAAAAENECAAAA|9427|2|37|7|AM|third|night|| +9428|AAAAAAAAFNECAAAA|9428|2|37|8|AM|third|night|| +9429|AAAAAAAAGNECAAAA|9429|2|37|9|AM|third|night|| +9430|AAAAAAAAHNECAAAA|9430|2|37|10|AM|third|night|| +9431|AAAAAAAAINECAAAA|9431|2|37|11|AM|third|night|| +9432|AAAAAAAAJNECAAAA|9432|2|37|12|AM|third|night|| +9433|AAAAAAAAKNECAAAA|9433|2|37|13|AM|third|night|| +9434|AAAAAAAALNECAAAA|9434|2|37|14|AM|third|night|| +9435|AAAAAAAAMNECAAAA|9435|2|37|15|AM|third|night|| +9436|AAAAAAAANNECAAAA|9436|2|37|16|AM|third|night|| +9437|AAAAAAAAONECAAAA|9437|2|37|17|AM|third|night|| +9438|AAAAAAAAPNECAAAA|9438|2|37|18|AM|third|night|| +9439|AAAAAAAAAOECAAAA|9439|2|37|19|AM|third|night|| +9440|AAAAAAAABOECAAAA|9440|2|37|20|AM|third|night|| +9441|AAAAAAAACOECAAAA|9441|2|37|21|AM|third|night|| +9442|AAAAAAAADOECAAAA|9442|2|37|22|AM|third|night|| +9443|AAAAAAAAEOECAAAA|9443|2|37|23|AM|third|night|| +9444|AAAAAAAAFOECAAAA|9444|2|37|24|AM|third|night|| +9445|AAAAAAAAGOECAAAA|9445|2|37|25|AM|third|night|| +9446|AAAAAAAAHOECAAAA|9446|2|37|26|AM|third|night|| +9447|AAAAAAAAIOECAAAA|9447|2|37|27|AM|third|night|| +9448|AAAAAAAAJOECAAAA|9448|2|37|28|AM|third|night|| +9449|AAAAAAAAKOECAAAA|9449|2|37|29|AM|third|night|| +9450|AAAAAAAALOECAAAA|9450|2|37|30|AM|third|night|| +9451|AAAAAAAAMOECAAAA|9451|2|37|31|AM|third|night|| +9452|AAAAAAAANOECAAAA|9452|2|37|32|AM|third|night|| +9453|AAAAAAAAOOECAAAA|9453|2|37|33|AM|third|night|| +9454|AAAAAAAAPOECAAAA|9454|2|37|34|AM|third|night|| +9455|AAAAAAAAAPECAAAA|9455|2|37|35|AM|third|night|| +9456|AAAAAAAABPECAAAA|9456|2|37|36|AM|third|night|| +9457|AAAAAAAACPECAAAA|9457|2|37|37|AM|third|night|| +9458|AAAAAAAADPECAAAA|9458|2|37|38|AM|third|night|| +9459|AAAAAAAAEPECAAAA|9459|2|37|39|AM|third|night|| +9460|AAAAAAAAFPECAAAA|9460|2|37|40|AM|third|night|| +9461|AAAAAAAAGPECAAAA|9461|2|37|41|AM|third|night|| +9462|AAAAAAAAHPECAAAA|9462|2|37|42|AM|third|night|| +9463|AAAAAAAAIPECAAAA|9463|2|37|43|AM|third|night|| +9464|AAAAAAAAJPECAAAA|9464|2|37|44|AM|third|night|| +9465|AAAAAAAAKPECAAAA|9465|2|37|45|AM|third|night|| +9466|AAAAAAAALPECAAAA|9466|2|37|46|AM|third|night|| +9467|AAAAAAAAMPECAAAA|9467|2|37|47|AM|third|night|| +9468|AAAAAAAANPECAAAA|9468|2|37|48|AM|third|night|| +9469|AAAAAAAAOPECAAAA|9469|2|37|49|AM|third|night|| +9470|AAAAAAAAPPECAAAA|9470|2|37|50|AM|third|night|| +9471|AAAAAAAAAAFCAAAA|9471|2|37|51|AM|third|night|| +9472|AAAAAAAABAFCAAAA|9472|2|37|52|AM|third|night|| +9473|AAAAAAAACAFCAAAA|9473|2|37|53|AM|third|night|| +9474|AAAAAAAADAFCAAAA|9474|2|37|54|AM|third|night|| +9475|AAAAAAAAEAFCAAAA|9475|2|37|55|AM|third|night|| +9476|AAAAAAAAFAFCAAAA|9476|2|37|56|AM|third|night|| +9477|AAAAAAAAGAFCAAAA|9477|2|37|57|AM|third|night|| +9478|AAAAAAAAHAFCAAAA|9478|2|37|58|AM|third|night|| +9479|AAAAAAAAIAFCAAAA|9479|2|37|59|AM|third|night|| +9480|AAAAAAAAJAFCAAAA|9480|2|38|0|AM|third|night|| +9481|AAAAAAAAKAFCAAAA|9481|2|38|1|AM|third|night|| +9482|AAAAAAAALAFCAAAA|9482|2|38|2|AM|third|night|| +9483|AAAAAAAAMAFCAAAA|9483|2|38|3|AM|third|night|| +9484|AAAAAAAANAFCAAAA|9484|2|38|4|AM|third|night|| +9485|AAAAAAAAOAFCAAAA|9485|2|38|5|AM|third|night|| +9486|AAAAAAAAPAFCAAAA|9486|2|38|6|AM|third|night|| +9487|AAAAAAAAABFCAAAA|9487|2|38|7|AM|third|night|| +9488|AAAAAAAABBFCAAAA|9488|2|38|8|AM|third|night|| +9489|AAAAAAAACBFCAAAA|9489|2|38|9|AM|third|night|| +9490|AAAAAAAADBFCAAAA|9490|2|38|10|AM|third|night|| +9491|AAAAAAAAEBFCAAAA|9491|2|38|11|AM|third|night|| +9492|AAAAAAAAFBFCAAAA|9492|2|38|12|AM|third|night|| +9493|AAAAAAAAGBFCAAAA|9493|2|38|13|AM|third|night|| +9494|AAAAAAAAHBFCAAAA|9494|2|38|14|AM|third|night|| +9495|AAAAAAAAIBFCAAAA|9495|2|38|15|AM|third|night|| +9496|AAAAAAAAJBFCAAAA|9496|2|38|16|AM|third|night|| +9497|AAAAAAAAKBFCAAAA|9497|2|38|17|AM|third|night|| +9498|AAAAAAAALBFCAAAA|9498|2|38|18|AM|third|night|| +9499|AAAAAAAAMBFCAAAA|9499|2|38|19|AM|third|night|| +9500|AAAAAAAANBFCAAAA|9500|2|38|20|AM|third|night|| +9501|AAAAAAAAOBFCAAAA|9501|2|38|21|AM|third|night|| +9502|AAAAAAAAPBFCAAAA|9502|2|38|22|AM|third|night|| +9503|AAAAAAAAACFCAAAA|9503|2|38|23|AM|third|night|| +9504|AAAAAAAABCFCAAAA|9504|2|38|24|AM|third|night|| +9505|AAAAAAAACCFCAAAA|9505|2|38|25|AM|third|night|| +9506|AAAAAAAADCFCAAAA|9506|2|38|26|AM|third|night|| +9507|AAAAAAAAECFCAAAA|9507|2|38|27|AM|third|night|| +9508|AAAAAAAAFCFCAAAA|9508|2|38|28|AM|third|night|| +9509|AAAAAAAAGCFCAAAA|9509|2|38|29|AM|third|night|| +9510|AAAAAAAAHCFCAAAA|9510|2|38|30|AM|third|night|| +9511|AAAAAAAAICFCAAAA|9511|2|38|31|AM|third|night|| +9512|AAAAAAAAJCFCAAAA|9512|2|38|32|AM|third|night|| +9513|AAAAAAAAKCFCAAAA|9513|2|38|33|AM|third|night|| +9514|AAAAAAAALCFCAAAA|9514|2|38|34|AM|third|night|| +9515|AAAAAAAAMCFCAAAA|9515|2|38|35|AM|third|night|| +9516|AAAAAAAANCFCAAAA|9516|2|38|36|AM|third|night|| +9517|AAAAAAAAOCFCAAAA|9517|2|38|37|AM|third|night|| +9518|AAAAAAAAPCFCAAAA|9518|2|38|38|AM|third|night|| +9519|AAAAAAAAADFCAAAA|9519|2|38|39|AM|third|night|| +9520|AAAAAAAABDFCAAAA|9520|2|38|40|AM|third|night|| +9521|AAAAAAAACDFCAAAA|9521|2|38|41|AM|third|night|| +9522|AAAAAAAADDFCAAAA|9522|2|38|42|AM|third|night|| +9523|AAAAAAAAEDFCAAAA|9523|2|38|43|AM|third|night|| +9524|AAAAAAAAFDFCAAAA|9524|2|38|44|AM|third|night|| +9525|AAAAAAAAGDFCAAAA|9525|2|38|45|AM|third|night|| +9526|AAAAAAAAHDFCAAAA|9526|2|38|46|AM|third|night|| +9527|AAAAAAAAIDFCAAAA|9527|2|38|47|AM|third|night|| +9528|AAAAAAAAJDFCAAAA|9528|2|38|48|AM|third|night|| +9529|AAAAAAAAKDFCAAAA|9529|2|38|49|AM|third|night|| +9530|AAAAAAAALDFCAAAA|9530|2|38|50|AM|third|night|| +9531|AAAAAAAAMDFCAAAA|9531|2|38|51|AM|third|night|| +9532|AAAAAAAANDFCAAAA|9532|2|38|52|AM|third|night|| +9533|AAAAAAAAODFCAAAA|9533|2|38|53|AM|third|night|| +9534|AAAAAAAAPDFCAAAA|9534|2|38|54|AM|third|night|| +9535|AAAAAAAAAEFCAAAA|9535|2|38|55|AM|third|night|| +9536|AAAAAAAABEFCAAAA|9536|2|38|56|AM|third|night|| +9537|AAAAAAAACEFCAAAA|9537|2|38|57|AM|third|night|| +9538|AAAAAAAADEFCAAAA|9538|2|38|58|AM|third|night|| +9539|AAAAAAAAEEFCAAAA|9539|2|38|59|AM|third|night|| +9540|AAAAAAAAFEFCAAAA|9540|2|39|0|AM|third|night|| +9541|AAAAAAAAGEFCAAAA|9541|2|39|1|AM|third|night|| +9542|AAAAAAAAHEFCAAAA|9542|2|39|2|AM|third|night|| +9543|AAAAAAAAIEFCAAAA|9543|2|39|3|AM|third|night|| +9544|AAAAAAAAJEFCAAAA|9544|2|39|4|AM|third|night|| +9545|AAAAAAAAKEFCAAAA|9545|2|39|5|AM|third|night|| +9546|AAAAAAAALEFCAAAA|9546|2|39|6|AM|third|night|| +9547|AAAAAAAAMEFCAAAA|9547|2|39|7|AM|third|night|| +9548|AAAAAAAANEFCAAAA|9548|2|39|8|AM|third|night|| +9549|AAAAAAAAOEFCAAAA|9549|2|39|9|AM|third|night|| +9550|AAAAAAAAPEFCAAAA|9550|2|39|10|AM|third|night|| +9551|AAAAAAAAAFFCAAAA|9551|2|39|11|AM|third|night|| +9552|AAAAAAAABFFCAAAA|9552|2|39|12|AM|third|night|| +9553|AAAAAAAACFFCAAAA|9553|2|39|13|AM|third|night|| +9554|AAAAAAAADFFCAAAA|9554|2|39|14|AM|third|night|| +9555|AAAAAAAAEFFCAAAA|9555|2|39|15|AM|third|night|| +9556|AAAAAAAAFFFCAAAA|9556|2|39|16|AM|third|night|| +9557|AAAAAAAAGFFCAAAA|9557|2|39|17|AM|third|night|| +9558|AAAAAAAAHFFCAAAA|9558|2|39|18|AM|third|night|| +9559|AAAAAAAAIFFCAAAA|9559|2|39|19|AM|third|night|| +9560|AAAAAAAAJFFCAAAA|9560|2|39|20|AM|third|night|| +9561|AAAAAAAAKFFCAAAA|9561|2|39|21|AM|third|night|| +9562|AAAAAAAALFFCAAAA|9562|2|39|22|AM|third|night|| +9563|AAAAAAAAMFFCAAAA|9563|2|39|23|AM|third|night|| +9564|AAAAAAAANFFCAAAA|9564|2|39|24|AM|third|night|| +9565|AAAAAAAAOFFCAAAA|9565|2|39|25|AM|third|night|| +9566|AAAAAAAAPFFCAAAA|9566|2|39|26|AM|third|night|| +9567|AAAAAAAAAGFCAAAA|9567|2|39|27|AM|third|night|| +9568|AAAAAAAABGFCAAAA|9568|2|39|28|AM|third|night|| +9569|AAAAAAAACGFCAAAA|9569|2|39|29|AM|third|night|| +9570|AAAAAAAADGFCAAAA|9570|2|39|30|AM|third|night|| +9571|AAAAAAAAEGFCAAAA|9571|2|39|31|AM|third|night|| +9572|AAAAAAAAFGFCAAAA|9572|2|39|32|AM|third|night|| +9573|AAAAAAAAGGFCAAAA|9573|2|39|33|AM|third|night|| +9574|AAAAAAAAHGFCAAAA|9574|2|39|34|AM|third|night|| +9575|AAAAAAAAIGFCAAAA|9575|2|39|35|AM|third|night|| +9576|AAAAAAAAJGFCAAAA|9576|2|39|36|AM|third|night|| +9577|AAAAAAAAKGFCAAAA|9577|2|39|37|AM|third|night|| +9578|AAAAAAAALGFCAAAA|9578|2|39|38|AM|third|night|| +9579|AAAAAAAAMGFCAAAA|9579|2|39|39|AM|third|night|| +9580|AAAAAAAANGFCAAAA|9580|2|39|40|AM|third|night|| +9581|AAAAAAAAOGFCAAAA|9581|2|39|41|AM|third|night|| +9582|AAAAAAAAPGFCAAAA|9582|2|39|42|AM|third|night|| +9583|AAAAAAAAAHFCAAAA|9583|2|39|43|AM|third|night|| +9584|AAAAAAAABHFCAAAA|9584|2|39|44|AM|third|night|| +9585|AAAAAAAACHFCAAAA|9585|2|39|45|AM|third|night|| +9586|AAAAAAAADHFCAAAA|9586|2|39|46|AM|third|night|| +9587|AAAAAAAAEHFCAAAA|9587|2|39|47|AM|third|night|| +9588|AAAAAAAAFHFCAAAA|9588|2|39|48|AM|third|night|| +9589|AAAAAAAAGHFCAAAA|9589|2|39|49|AM|third|night|| +9590|AAAAAAAAHHFCAAAA|9590|2|39|50|AM|third|night|| +9591|AAAAAAAAIHFCAAAA|9591|2|39|51|AM|third|night|| +9592|AAAAAAAAJHFCAAAA|9592|2|39|52|AM|third|night|| +9593|AAAAAAAAKHFCAAAA|9593|2|39|53|AM|third|night|| +9594|AAAAAAAALHFCAAAA|9594|2|39|54|AM|third|night|| +9595|AAAAAAAAMHFCAAAA|9595|2|39|55|AM|third|night|| +9596|AAAAAAAANHFCAAAA|9596|2|39|56|AM|third|night|| +9597|AAAAAAAAOHFCAAAA|9597|2|39|57|AM|third|night|| +9598|AAAAAAAAPHFCAAAA|9598|2|39|58|AM|third|night|| +9599|AAAAAAAAAIFCAAAA|9599|2|39|59|AM|third|night|| +9600|AAAAAAAABIFCAAAA|9600|2|40|0|AM|third|night|| +9601|AAAAAAAACIFCAAAA|9601|2|40|1|AM|third|night|| +9602|AAAAAAAADIFCAAAA|9602|2|40|2|AM|third|night|| +9603|AAAAAAAAEIFCAAAA|9603|2|40|3|AM|third|night|| +9604|AAAAAAAAFIFCAAAA|9604|2|40|4|AM|third|night|| +9605|AAAAAAAAGIFCAAAA|9605|2|40|5|AM|third|night|| +9606|AAAAAAAAHIFCAAAA|9606|2|40|6|AM|third|night|| +9607|AAAAAAAAIIFCAAAA|9607|2|40|7|AM|third|night|| +9608|AAAAAAAAJIFCAAAA|9608|2|40|8|AM|third|night|| +9609|AAAAAAAAKIFCAAAA|9609|2|40|9|AM|third|night|| +9610|AAAAAAAALIFCAAAA|9610|2|40|10|AM|third|night|| +9611|AAAAAAAAMIFCAAAA|9611|2|40|11|AM|third|night|| +9612|AAAAAAAANIFCAAAA|9612|2|40|12|AM|third|night|| +9613|AAAAAAAAOIFCAAAA|9613|2|40|13|AM|third|night|| +9614|AAAAAAAAPIFCAAAA|9614|2|40|14|AM|third|night|| +9615|AAAAAAAAAJFCAAAA|9615|2|40|15|AM|third|night|| +9616|AAAAAAAABJFCAAAA|9616|2|40|16|AM|third|night|| +9617|AAAAAAAACJFCAAAA|9617|2|40|17|AM|third|night|| +9618|AAAAAAAADJFCAAAA|9618|2|40|18|AM|third|night|| +9619|AAAAAAAAEJFCAAAA|9619|2|40|19|AM|third|night|| +9620|AAAAAAAAFJFCAAAA|9620|2|40|20|AM|third|night|| +9621|AAAAAAAAGJFCAAAA|9621|2|40|21|AM|third|night|| +9622|AAAAAAAAHJFCAAAA|9622|2|40|22|AM|third|night|| +9623|AAAAAAAAIJFCAAAA|9623|2|40|23|AM|third|night|| +9624|AAAAAAAAJJFCAAAA|9624|2|40|24|AM|third|night|| +9625|AAAAAAAAKJFCAAAA|9625|2|40|25|AM|third|night|| +9626|AAAAAAAALJFCAAAA|9626|2|40|26|AM|third|night|| +9627|AAAAAAAAMJFCAAAA|9627|2|40|27|AM|third|night|| +9628|AAAAAAAANJFCAAAA|9628|2|40|28|AM|third|night|| +9629|AAAAAAAAOJFCAAAA|9629|2|40|29|AM|third|night|| +9630|AAAAAAAAPJFCAAAA|9630|2|40|30|AM|third|night|| +9631|AAAAAAAAAKFCAAAA|9631|2|40|31|AM|third|night|| +9632|AAAAAAAABKFCAAAA|9632|2|40|32|AM|third|night|| +9633|AAAAAAAACKFCAAAA|9633|2|40|33|AM|third|night|| +9634|AAAAAAAADKFCAAAA|9634|2|40|34|AM|third|night|| +9635|AAAAAAAAEKFCAAAA|9635|2|40|35|AM|third|night|| +9636|AAAAAAAAFKFCAAAA|9636|2|40|36|AM|third|night|| +9637|AAAAAAAAGKFCAAAA|9637|2|40|37|AM|third|night|| +9638|AAAAAAAAHKFCAAAA|9638|2|40|38|AM|third|night|| +9639|AAAAAAAAIKFCAAAA|9639|2|40|39|AM|third|night|| +9640|AAAAAAAAJKFCAAAA|9640|2|40|40|AM|third|night|| +9641|AAAAAAAAKKFCAAAA|9641|2|40|41|AM|third|night|| +9642|AAAAAAAALKFCAAAA|9642|2|40|42|AM|third|night|| +9643|AAAAAAAAMKFCAAAA|9643|2|40|43|AM|third|night|| +9644|AAAAAAAANKFCAAAA|9644|2|40|44|AM|third|night|| +9645|AAAAAAAAOKFCAAAA|9645|2|40|45|AM|third|night|| +9646|AAAAAAAAPKFCAAAA|9646|2|40|46|AM|third|night|| +9647|AAAAAAAAALFCAAAA|9647|2|40|47|AM|third|night|| +9648|AAAAAAAABLFCAAAA|9648|2|40|48|AM|third|night|| +9649|AAAAAAAACLFCAAAA|9649|2|40|49|AM|third|night|| +9650|AAAAAAAADLFCAAAA|9650|2|40|50|AM|third|night|| +9651|AAAAAAAAELFCAAAA|9651|2|40|51|AM|third|night|| +9652|AAAAAAAAFLFCAAAA|9652|2|40|52|AM|third|night|| +9653|AAAAAAAAGLFCAAAA|9653|2|40|53|AM|third|night|| +9654|AAAAAAAAHLFCAAAA|9654|2|40|54|AM|third|night|| +9655|AAAAAAAAILFCAAAA|9655|2|40|55|AM|third|night|| +9656|AAAAAAAAJLFCAAAA|9656|2|40|56|AM|third|night|| +9657|AAAAAAAAKLFCAAAA|9657|2|40|57|AM|third|night|| +9658|AAAAAAAALLFCAAAA|9658|2|40|58|AM|third|night|| +9659|AAAAAAAAMLFCAAAA|9659|2|40|59|AM|third|night|| +9660|AAAAAAAANLFCAAAA|9660|2|41|0|AM|third|night|| +9661|AAAAAAAAOLFCAAAA|9661|2|41|1|AM|third|night|| +9662|AAAAAAAAPLFCAAAA|9662|2|41|2|AM|third|night|| +9663|AAAAAAAAAMFCAAAA|9663|2|41|3|AM|third|night|| +9664|AAAAAAAABMFCAAAA|9664|2|41|4|AM|third|night|| +9665|AAAAAAAACMFCAAAA|9665|2|41|5|AM|third|night|| +9666|AAAAAAAADMFCAAAA|9666|2|41|6|AM|third|night|| +9667|AAAAAAAAEMFCAAAA|9667|2|41|7|AM|third|night|| +9668|AAAAAAAAFMFCAAAA|9668|2|41|8|AM|third|night|| +9669|AAAAAAAAGMFCAAAA|9669|2|41|9|AM|third|night|| +9670|AAAAAAAAHMFCAAAA|9670|2|41|10|AM|third|night|| +9671|AAAAAAAAIMFCAAAA|9671|2|41|11|AM|third|night|| +9672|AAAAAAAAJMFCAAAA|9672|2|41|12|AM|third|night|| +9673|AAAAAAAAKMFCAAAA|9673|2|41|13|AM|third|night|| +9674|AAAAAAAALMFCAAAA|9674|2|41|14|AM|third|night|| +9675|AAAAAAAAMMFCAAAA|9675|2|41|15|AM|third|night|| +9676|AAAAAAAANMFCAAAA|9676|2|41|16|AM|third|night|| +9677|AAAAAAAAOMFCAAAA|9677|2|41|17|AM|third|night|| +9678|AAAAAAAAPMFCAAAA|9678|2|41|18|AM|third|night|| +9679|AAAAAAAAANFCAAAA|9679|2|41|19|AM|third|night|| +9680|AAAAAAAABNFCAAAA|9680|2|41|20|AM|third|night|| +9681|AAAAAAAACNFCAAAA|9681|2|41|21|AM|third|night|| +9682|AAAAAAAADNFCAAAA|9682|2|41|22|AM|third|night|| +9683|AAAAAAAAENFCAAAA|9683|2|41|23|AM|third|night|| +9684|AAAAAAAAFNFCAAAA|9684|2|41|24|AM|third|night|| +9685|AAAAAAAAGNFCAAAA|9685|2|41|25|AM|third|night|| +9686|AAAAAAAAHNFCAAAA|9686|2|41|26|AM|third|night|| +9687|AAAAAAAAINFCAAAA|9687|2|41|27|AM|third|night|| +9688|AAAAAAAAJNFCAAAA|9688|2|41|28|AM|third|night|| +9689|AAAAAAAAKNFCAAAA|9689|2|41|29|AM|third|night|| +9690|AAAAAAAALNFCAAAA|9690|2|41|30|AM|third|night|| +9691|AAAAAAAAMNFCAAAA|9691|2|41|31|AM|third|night|| +9692|AAAAAAAANNFCAAAA|9692|2|41|32|AM|third|night|| +9693|AAAAAAAAONFCAAAA|9693|2|41|33|AM|third|night|| +9694|AAAAAAAAPNFCAAAA|9694|2|41|34|AM|third|night|| +9695|AAAAAAAAAOFCAAAA|9695|2|41|35|AM|third|night|| +9696|AAAAAAAABOFCAAAA|9696|2|41|36|AM|third|night|| +9697|AAAAAAAACOFCAAAA|9697|2|41|37|AM|third|night|| +9698|AAAAAAAADOFCAAAA|9698|2|41|38|AM|third|night|| +9699|AAAAAAAAEOFCAAAA|9699|2|41|39|AM|third|night|| +9700|AAAAAAAAFOFCAAAA|9700|2|41|40|AM|third|night|| +9701|AAAAAAAAGOFCAAAA|9701|2|41|41|AM|third|night|| +9702|AAAAAAAAHOFCAAAA|9702|2|41|42|AM|third|night|| +9703|AAAAAAAAIOFCAAAA|9703|2|41|43|AM|third|night|| +9704|AAAAAAAAJOFCAAAA|9704|2|41|44|AM|third|night|| +9705|AAAAAAAAKOFCAAAA|9705|2|41|45|AM|third|night|| +9706|AAAAAAAALOFCAAAA|9706|2|41|46|AM|third|night|| +9707|AAAAAAAAMOFCAAAA|9707|2|41|47|AM|third|night|| +9708|AAAAAAAANOFCAAAA|9708|2|41|48|AM|third|night|| +9709|AAAAAAAAOOFCAAAA|9709|2|41|49|AM|third|night|| +9710|AAAAAAAAPOFCAAAA|9710|2|41|50|AM|third|night|| +9711|AAAAAAAAAPFCAAAA|9711|2|41|51|AM|third|night|| +9712|AAAAAAAABPFCAAAA|9712|2|41|52|AM|third|night|| +9713|AAAAAAAACPFCAAAA|9713|2|41|53|AM|third|night|| +9714|AAAAAAAADPFCAAAA|9714|2|41|54|AM|third|night|| +9715|AAAAAAAAEPFCAAAA|9715|2|41|55|AM|third|night|| +9716|AAAAAAAAFPFCAAAA|9716|2|41|56|AM|third|night|| +9717|AAAAAAAAGPFCAAAA|9717|2|41|57|AM|third|night|| +9718|AAAAAAAAHPFCAAAA|9718|2|41|58|AM|third|night|| +9719|AAAAAAAAIPFCAAAA|9719|2|41|59|AM|third|night|| +9720|AAAAAAAAJPFCAAAA|9720|2|42|0|AM|third|night|| +9721|AAAAAAAAKPFCAAAA|9721|2|42|1|AM|third|night|| +9722|AAAAAAAALPFCAAAA|9722|2|42|2|AM|third|night|| +9723|AAAAAAAAMPFCAAAA|9723|2|42|3|AM|third|night|| +9724|AAAAAAAANPFCAAAA|9724|2|42|4|AM|third|night|| +9725|AAAAAAAAOPFCAAAA|9725|2|42|5|AM|third|night|| +9726|AAAAAAAAPPFCAAAA|9726|2|42|6|AM|third|night|| +9727|AAAAAAAAAAGCAAAA|9727|2|42|7|AM|third|night|| +9728|AAAAAAAABAGCAAAA|9728|2|42|8|AM|third|night|| +9729|AAAAAAAACAGCAAAA|9729|2|42|9|AM|third|night|| +9730|AAAAAAAADAGCAAAA|9730|2|42|10|AM|third|night|| +9731|AAAAAAAAEAGCAAAA|9731|2|42|11|AM|third|night|| +9732|AAAAAAAAFAGCAAAA|9732|2|42|12|AM|third|night|| +9733|AAAAAAAAGAGCAAAA|9733|2|42|13|AM|third|night|| +9734|AAAAAAAAHAGCAAAA|9734|2|42|14|AM|third|night|| +9735|AAAAAAAAIAGCAAAA|9735|2|42|15|AM|third|night|| +9736|AAAAAAAAJAGCAAAA|9736|2|42|16|AM|third|night|| +9737|AAAAAAAAKAGCAAAA|9737|2|42|17|AM|third|night|| +9738|AAAAAAAALAGCAAAA|9738|2|42|18|AM|third|night|| +9739|AAAAAAAAMAGCAAAA|9739|2|42|19|AM|third|night|| +9740|AAAAAAAANAGCAAAA|9740|2|42|20|AM|third|night|| +9741|AAAAAAAAOAGCAAAA|9741|2|42|21|AM|third|night|| +9742|AAAAAAAAPAGCAAAA|9742|2|42|22|AM|third|night|| +9743|AAAAAAAAABGCAAAA|9743|2|42|23|AM|third|night|| +9744|AAAAAAAABBGCAAAA|9744|2|42|24|AM|third|night|| +9745|AAAAAAAACBGCAAAA|9745|2|42|25|AM|third|night|| +9746|AAAAAAAADBGCAAAA|9746|2|42|26|AM|third|night|| +9747|AAAAAAAAEBGCAAAA|9747|2|42|27|AM|third|night|| +9748|AAAAAAAAFBGCAAAA|9748|2|42|28|AM|third|night|| +9749|AAAAAAAAGBGCAAAA|9749|2|42|29|AM|third|night|| +9750|AAAAAAAAHBGCAAAA|9750|2|42|30|AM|third|night|| +9751|AAAAAAAAIBGCAAAA|9751|2|42|31|AM|third|night|| +9752|AAAAAAAAJBGCAAAA|9752|2|42|32|AM|third|night|| +9753|AAAAAAAAKBGCAAAA|9753|2|42|33|AM|third|night|| +9754|AAAAAAAALBGCAAAA|9754|2|42|34|AM|third|night|| +9755|AAAAAAAAMBGCAAAA|9755|2|42|35|AM|third|night|| +9756|AAAAAAAANBGCAAAA|9756|2|42|36|AM|third|night|| +9757|AAAAAAAAOBGCAAAA|9757|2|42|37|AM|third|night|| +9758|AAAAAAAAPBGCAAAA|9758|2|42|38|AM|third|night|| +9759|AAAAAAAAACGCAAAA|9759|2|42|39|AM|third|night|| +9760|AAAAAAAABCGCAAAA|9760|2|42|40|AM|third|night|| +9761|AAAAAAAACCGCAAAA|9761|2|42|41|AM|third|night|| +9762|AAAAAAAADCGCAAAA|9762|2|42|42|AM|third|night|| +9763|AAAAAAAAECGCAAAA|9763|2|42|43|AM|third|night|| +9764|AAAAAAAAFCGCAAAA|9764|2|42|44|AM|third|night|| +9765|AAAAAAAAGCGCAAAA|9765|2|42|45|AM|third|night|| +9766|AAAAAAAAHCGCAAAA|9766|2|42|46|AM|third|night|| +9767|AAAAAAAAICGCAAAA|9767|2|42|47|AM|third|night|| +9768|AAAAAAAAJCGCAAAA|9768|2|42|48|AM|third|night|| +9769|AAAAAAAAKCGCAAAA|9769|2|42|49|AM|third|night|| +9770|AAAAAAAALCGCAAAA|9770|2|42|50|AM|third|night|| +9771|AAAAAAAAMCGCAAAA|9771|2|42|51|AM|third|night|| +9772|AAAAAAAANCGCAAAA|9772|2|42|52|AM|third|night|| +9773|AAAAAAAAOCGCAAAA|9773|2|42|53|AM|third|night|| +9774|AAAAAAAAPCGCAAAA|9774|2|42|54|AM|third|night|| +9775|AAAAAAAAADGCAAAA|9775|2|42|55|AM|third|night|| +9776|AAAAAAAABDGCAAAA|9776|2|42|56|AM|third|night|| +9777|AAAAAAAACDGCAAAA|9777|2|42|57|AM|third|night|| +9778|AAAAAAAADDGCAAAA|9778|2|42|58|AM|third|night|| +9779|AAAAAAAAEDGCAAAA|9779|2|42|59|AM|third|night|| +9780|AAAAAAAAFDGCAAAA|9780|2|43|0|AM|third|night|| +9781|AAAAAAAAGDGCAAAA|9781|2|43|1|AM|third|night|| +9782|AAAAAAAAHDGCAAAA|9782|2|43|2|AM|third|night|| +9783|AAAAAAAAIDGCAAAA|9783|2|43|3|AM|third|night|| +9784|AAAAAAAAJDGCAAAA|9784|2|43|4|AM|third|night|| +9785|AAAAAAAAKDGCAAAA|9785|2|43|5|AM|third|night|| +9786|AAAAAAAALDGCAAAA|9786|2|43|6|AM|third|night|| +9787|AAAAAAAAMDGCAAAA|9787|2|43|7|AM|third|night|| +9788|AAAAAAAANDGCAAAA|9788|2|43|8|AM|third|night|| +9789|AAAAAAAAODGCAAAA|9789|2|43|9|AM|third|night|| +9790|AAAAAAAAPDGCAAAA|9790|2|43|10|AM|third|night|| +9791|AAAAAAAAAEGCAAAA|9791|2|43|11|AM|third|night|| +9792|AAAAAAAABEGCAAAA|9792|2|43|12|AM|third|night|| +9793|AAAAAAAACEGCAAAA|9793|2|43|13|AM|third|night|| +9794|AAAAAAAADEGCAAAA|9794|2|43|14|AM|third|night|| +9795|AAAAAAAAEEGCAAAA|9795|2|43|15|AM|third|night|| +9796|AAAAAAAAFEGCAAAA|9796|2|43|16|AM|third|night|| +9797|AAAAAAAAGEGCAAAA|9797|2|43|17|AM|third|night|| +9798|AAAAAAAAHEGCAAAA|9798|2|43|18|AM|third|night|| +9799|AAAAAAAAIEGCAAAA|9799|2|43|19|AM|third|night|| +9800|AAAAAAAAJEGCAAAA|9800|2|43|20|AM|third|night|| +9801|AAAAAAAAKEGCAAAA|9801|2|43|21|AM|third|night|| +9802|AAAAAAAALEGCAAAA|9802|2|43|22|AM|third|night|| +9803|AAAAAAAAMEGCAAAA|9803|2|43|23|AM|third|night|| +9804|AAAAAAAANEGCAAAA|9804|2|43|24|AM|third|night|| +9805|AAAAAAAAOEGCAAAA|9805|2|43|25|AM|third|night|| +9806|AAAAAAAAPEGCAAAA|9806|2|43|26|AM|third|night|| +9807|AAAAAAAAAFGCAAAA|9807|2|43|27|AM|third|night|| +9808|AAAAAAAABFGCAAAA|9808|2|43|28|AM|third|night|| +9809|AAAAAAAACFGCAAAA|9809|2|43|29|AM|third|night|| +9810|AAAAAAAADFGCAAAA|9810|2|43|30|AM|third|night|| +9811|AAAAAAAAEFGCAAAA|9811|2|43|31|AM|third|night|| +9812|AAAAAAAAFFGCAAAA|9812|2|43|32|AM|third|night|| +9813|AAAAAAAAGFGCAAAA|9813|2|43|33|AM|third|night|| +9814|AAAAAAAAHFGCAAAA|9814|2|43|34|AM|third|night|| +9815|AAAAAAAAIFGCAAAA|9815|2|43|35|AM|third|night|| +9816|AAAAAAAAJFGCAAAA|9816|2|43|36|AM|third|night|| +9817|AAAAAAAAKFGCAAAA|9817|2|43|37|AM|third|night|| +9818|AAAAAAAALFGCAAAA|9818|2|43|38|AM|third|night|| +9819|AAAAAAAAMFGCAAAA|9819|2|43|39|AM|third|night|| +9820|AAAAAAAANFGCAAAA|9820|2|43|40|AM|third|night|| +9821|AAAAAAAAOFGCAAAA|9821|2|43|41|AM|third|night|| +9822|AAAAAAAAPFGCAAAA|9822|2|43|42|AM|third|night|| +9823|AAAAAAAAAGGCAAAA|9823|2|43|43|AM|third|night|| +9824|AAAAAAAABGGCAAAA|9824|2|43|44|AM|third|night|| +9825|AAAAAAAACGGCAAAA|9825|2|43|45|AM|third|night|| +9826|AAAAAAAADGGCAAAA|9826|2|43|46|AM|third|night|| +9827|AAAAAAAAEGGCAAAA|9827|2|43|47|AM|third|night|| +9828|AAAAAAAAFGGCAAAA|9828|2|43|48|AM|third|night|| +9829|AAAAAAAAGGGCAAAA|9829|2|43|49|AM|third|night|| +9830|AAAAAAAAHGGCAAAA|9830|2|43|50|AM|third|night|| +9831|AAAAAAAAIGGCAAAA|9831|2|43|51|AM|third|night|| +9832|AAAAAAAAJGGCAAAA|9832|2|43|52|AM|third|night|| +9833|AAAAAAAAKGGCAAAA|9833|2|43|53|AM|third|night|| +9834|AAAAAAAALGGCAAAA|9834|2|43|54|AM|third|night|| +9835|AAAAAAAAMGGCAAAA|9835|2|43|55|AM|third|night|| +9836|AAAAAAAANGGCAAAA|9836|2|43|56|AM|third|night|| +9837|AAAAAAAAOGGCAAAA|9837|2|43|57|AM|third|night|| +9838|AAAAAAAAPGGCAAAA|9838|2|43|58|AM|third|night|| +9839|AAAAAAAAAHGCAAAA|9839|2|43|59|AM|third|night|| +9840|AAAAAAAABHGCAAAA|9840|2|44|0|AM|third|night|| +9841|AAAAAAAACHGCAAAA|9841|2|44|1|AM|third|night|| +9842|AAAAAAAADHGCAAAA|9842|2|44|2|AM|third|night|| +9843|AAAAAAAAEHGCAAAA|9843|2|44|3|AM|third|night|| +9844|AAAAAAAAFHGCAAAA|9844|2|44|4|AM|third|night|| +9845|AAAAAAAAGHGCAAAA|9845|2|44|5|AM|third|night|| +9846|AAAAAAAAHHGCAAAA|9846|2|44|6|AM|third|night|| +9847|AAAAAAAAIHGCAAAA|9847|2|44|7|AM|third|night|| +9848|AAAAAAAAJHGCAAAA|9848|2|44|8|AM|third|night|| +9849|AAAAAAAAKHGCAAAA|9849|2|44|9|AM|third|night|| +9850|AAAAAAAALHGCAAAA|9850|2|44|10|AM|third|night|| +9851|AAAAAAAAMHGCAAAA|9851|2|44|11|AM|third|night|| +9852|AAAAAAAANHGCAAAA|9852|2|44|12|AM|third|night|| +9853|AAAAAAAAOHGCAAAA|9853|2|44|13|AM|third|night|| +9854|AAAAAAAAPHGCAAAA|9854|2|44|14|AM|third|night|| +9855|AAAAAAAAAIGCAAAA|9855|2|44|15|AM|third|night|| +9856|AAAAAAAABIGCAAAA|9856|2|44|16|AM|third|night|| +9857|AAAAAAAACIGCAAAA|9857|2|44|17|AM|third|night|| +9858|AAAAAAAADIGCAAAA|9858|2|44|18|AM|third|night|| +9859|AAAAAAAAEIGCAAAA|9859|2|44|19|AM|third|night|| +9860|AAAAAAAAFIGCAAAA|9860|2|44|20|AM|third|night|| +9861|AAAAAAAAGIGCAAAA|9861|2|44|21|AM|third|night|| +9862|AAAAAAAAHIGCAAAA|9862|2|44|22|AM|third|night|| +9863|AAAAAAAAIIGCAAAA|9863|2|44|23|AM|third|night|| +9864|AAAAAAAAJIGCAAAA|9864|2|44|24|AM|third|night|| +9865|AAAAAAAAKIGCAAAA|9865|2|44|25|AM|third|night|| +9866|AAAAAAAALIGCAAAA|9866|2|44|26|AM|third|night|| +9867|AAAAAAAAMIGCAAAA|9867|2|44|27|AM|third|night|| +9868|AAAAAAAANIGCAAAA|9868|2|44|28|AM|third|night|| +9869|AAAAAAAAOIGCAAAA|9869|2|44|29|AM|third|night|| +9870|AAAAAAAAPIGCAAAA|9870|2|44|30|AM|third|night|| +9871|AAAAAAAAAJGCAAAA|9871|2|44|31|AM|third|night|| +9872|AAAAAAAABJGCAAAA|9872|2|44|32|AM|third|night|| +9873|AAAAAAAACJGCAAAA|9873|2|44|33|AM|third|night|| +9874|AAAAAAAADJGCAAAA|9874|2|44|34|AM|third|night|| +9875|AAAAAAAAEJGCAAAA|9875|2|44|35|AM|third|night|| +9876|AAAAAAAAFJGCAAAA|9876|2|44|36|AM|third|night|| +9877|AAAAAAAAGJGCAAAA|9877|2|44|37|AM|third|night|| +9878|AAAAAAAAHJGCAAAA|9878|2|44|38|AM|third|night|| +9879|AAAAAAAAIJGCAAAA|9879|2|44|39|AM|third|night|| +9880|AAAAAAAAJJGCAAAA|9880|2|44|40|AM|third|night|| +9881|AAAAAAAAKJGCAAAA|9881|2|44|41|AM|third|night|| +9882|AAAAAAAALJGCAAAA|9882|2|44|42|AM|third|night|| +9883|AAAAAAAAMJGCAAAA|9883|2|44|43|AM|third|night|| +9884|AAAAAAAANJGCAAAA|9884|2|44|44|AM|third|night|| +9885|AAAAAAAAOJGCAAAA|9885|2|44|45|AM|third|night|| +9886|AAAAAAAAPJGCAAAA|9886|2|44|46|AM|third|night|| +9887|AAAAAAAAAKGCAAAA|9887|2|44|47|AM|third|night|| +9888|AAAAAAAABKGCAAAA|9888|2|44|48|AM|third|night|| +9889|AAAAAAAACKGCAAAA|9889|2|44|49|AM|third|night|| +9890|AAAAAAAADKGCAAAA|9890|2|44|50|AM|third|night|| +9891|AAAAAAAAEKGCAAAA|9891|2|44|51|AM|third|night|| +9892|AAAAAAAAFKGCAAAA|9892|2|44|52|AM|third|night|| +9893|AAAAAAAAGKGCAAAA|9893|2|44|53|AM|third|night|| +9894|AAAAAAAAHKGCAAAA|9894|2|44|54|AM|third|night|| +9895|AAAAAAAAIKGCAAAA|9895|2|44|55|AM|third|night|| +9896|AAAAAAAAJKGCAAAA|9896|2|44|56|AM|third|night|| +9897|AAAAAAAAKKGCAAAA|9897|2|44|57|AM|third|night|| +9898|AAAAAAAALKGCAAAA|9898|2|44|58|AM|third|night|| +9899|AAAAAAAAMKGCAAAA|9899|2|44|59|AM|third|night|| +9900|AAAAAAAANKGCAAAA|9900|2|45|0|AM|third|night|| +9901|AAAAAAAAOKGCAAAA|9901|2|45|1|AM|third|night|| +9902|AAAAAAAAPKGCAAAA|9902|2|45|2|AM|third|night|| +9903|AAAAAAAAALGCAAAA|9903|2|45|3|AM|third|night|| +9904|AAAAAAAABLGCAAAA|9904|2|45|4|AM|third|night|| +9905|AAAAAAAACLGCAAAA|9905|2|45|5|AM|third|night|| +9906|AAAAAAAADLGCAAAA|9906|2|45|6|AM|third|night|| +9907|AAAAAAAAELGCAAAA|9907|2|45|7|AM|third|night|| +9908|AAAAAAAAFLGCAAAA|9908|2|45|8|AM|third|night|| +9909|AAAAAAAAGLGCAAAA|9909|2|45|9|AM|third|night|| +9910|AAAAAAAAHLGCAAAA|9910|2|45|10|AM|third|night|| +9911|AAAAAAAAILGCAAAA|9911|2|45|11|AM|third|night|| +9912|AAAAAAAAJLGCAAAA|9912|2|45|12|AM|third|night|| +9913|AAAAAAAAKLGCAAAA|9913|2|45|13|AM|third|night|| +9914|AAAAAAAALLGCAAAA|9914|2|45|14|AM|third|night|| +9915|AAAAAAAAMLGCAAAA|9915|2|45|15|AM|third|night|| +9916|AAAAAAAANLGCAAAA|9916|2|45|16|AM|third|night|| +9917|AAAAAAAAOLGCAAAA|9917|2|45|17|AM|third|night|| +9918|AAAAAAAAPLGCAAAA|9918|2|45|18|AM|third|night|| +9919|AAAAAAAAAMGCAAAA|9919|2|45|19|AM|third|night|| +9920|AAAAAAAABMGCAAAA|9920|2|45|20|AM|third|night|| +9921|AAAAAAAACMGCAAAA|9921|2|45|21|AM|third|night|| +9922|AAAAAAAADMGCAAAA|9922|2|45|22|AM|third|night|| +9923|AAAAAAAAEMGCAAAA|9923|2|45|23|AM|third|night|| +9924|AAAAAAAAFMGCAAAA|9924|2|45|24|AM|third|night|| +9925|AAAAAAAAGMGCAAAA|9925|2|45|25|AM|third|night|| +9926|AAAAAAAAHMGCAAAA|9926|2|45|26|AM|third|night|| +9927|AAAAAAAAIMGCAAAA|9927|2|45|27|AM|third|night|| +9928|AAAAAAAAJMGCAAAA|9928|2|45|28|AM|third|night|| +9929|AAAAAAAAKMGCAAAA|9929|2|45|29|AM|third|night|| +9930|AAAAAAAALMGCAAAA|9930|2|45|30|AM|third|night|| +9931|AAAAAAAAMMGCAAAA|9931|2|45|31|AM|third|night|| +9932|AAAAAAAANMGCAAAA|9932|2|45|32|AM|third|night|| +9933|AAAAAAAAOMGCAAAA|9933|2|45|33|AM|third|night|| +9934|AAAAAAAAPMGCAAAA|9934|2|45|34|AM|third|night|| +9935|AAAAAAAAANGCAAAA|9935|2|45|35|AM|third|night|| +9936|AAAAAAAABNGCAAAA|9936|2|45|36|AM|third|night|| +9937|AAAAAAAACNGCAAAA|9937|2|45|37|AM|third|night|| +9938|AAAAAAAADNGCAAAA|9938|2|45|38|AM|third|night|| +9939|AAAAAAAAENGCAAAA|9939|2|45|39|AM|third|night|| +9940|AAAAAAAAFNGCAAAA|9940|2|45|40|AM|third|night|| +9941|AAAAAAAAGNGCAAAA|9941|2|45|41|AM|third|night|| +9942|AAAAAAAAHNGCAAAA|9942|2|45|42|AM|third|night|| +9943|AAAAAAAAINGCAAAA|9943|2|45|43|AM|third|night|| +9944|AAAAAAAAJNGCAAAA|9944|2|45|44|AM|third|night|| +9945|AAAAAAAAKNGCAAAA|9945|2|45|45|AM|third|night|| +9946|AAAAAAAALNGCAAAA|9946|2|45|46|AM|third|night|| +9947|AAAAAAAAMNGCAAAA|9947|2|45|47|AM|third|night|| +9948|AAAAAAAANNGCAAAA|9948|2|45|48|AM|third|night|| +9949|AAAAAAAAONGCAAAA|9949|2|45|49|AM|third|night|| +9950|AAAAAAAAPNGCAAAA|9950|2|45|50|AM|third|night|| +9951|AAAAAAAAAOGCAAAA|9951|2|45|51|AM|third|night|| +9952|AAAAAAAABOGCAAAA|9952|2|45|52|AM|third|night|| +9953|AAAAAAAACOGCAAAA|9953|2|45|53|AM|third|night|| +9954|AAAAAAAADOGCAAAA|9954|2|45|54|AM|third|night|| +9955|AAAAAAAAEOGCAAAA|9955|2|45|55|AM|third|night|| +9956|AAAAAAAAFOGCAAAA|9956|2|45|56|AM|third|night|| +9957|AAAAAAAAGOGCAAAA|9957|2|45|57|AM|third|night|| +9958|AAAAAAAAHOGCAAAA|9958|2|45|58|AM|third|night|| +9959|AAAAAAAAIOGCAAAA|9959|2|45|59|AM|third|night|| +9960|AAAAAAAAJOGCAAAA|9960|2|46|0|AM|third|night|| +9961|AAAAAAAAKOGCAAAA|9961|2|46|1|AM|third|night|| +9962|AAAAAAAALOGCAAAA|9962|2|46|2|AM|third|night|| +9963|AAAAAAAAMOGCAAAA|9963|2|46|3|AM|third|night|| +9964|AAAAAAAANOGCAAAA|9964|2|46|4|AM|third|night|| +9965|AAAAAAAAOOGCAAAA|9965|2|46|5|AM|third|night|| +9966|AAAAAAAAPOGCAAAA|9966|2|46|6|AM|third|night|| +9967|AAAAAAAAAPGCAAAA|9967|2|46|7|AM|third|night|| +9968|AAAAAAAABPGCAAAA|9968|2|46|8|AM|third|night|| +9969|AAAAAAAACPGCAAAA|9969|2|46|9|AM|third|night|| +9970|AAAAAAAADPGCAAAA|9970|2|46|10|AM|third|night|| +9971|AAAAAAAAEPGCAAAA|9971|2|46|11|AM|third|night|| +9972|AAAAAAAAFPGCAAAA|9972|2|46|12|AM|third|night|| +9973|AAAAAAAAGPGCAAAA|9973|2|46|13|AM|third|night|| +9974|AAAAAAAAHPGCAAAA|9974|2|46|14|AM|third|night|| +9975|AAAAAAAAIPGCAAAA|9975|2|46|15|AM|third|night|| +9976|AAAAAAAAJPGCAAAA|9976|2|46|16|AM|third|night|| +9977|AAAAAAAAKPGCAAAA|9977|2|46|17|AM|third|night|| +9978|AAAAAAAALPGCAAAA|9978|2|46|18|AM|third|night|| +9979|AAAAAAAAMPGCAAAA|9979|2|46|19|AM|third|night|| +9980|AAAAAAAANPGCAAAA|9980|2|46|20|AM|third|night|| +9981|AAAAAAAAOPGCAAAA|9981|2|46|21|AM|third|night|| +9982|AAAAAAAAPPGCAAAA|9982|2|46|22|AM|third|night|| +9983|AAAAAAAAAAHCAAAA|9983|2|46|23|AM|third|night|| +9984|AAAAAAAABAHCAAAA|9984|2|46|24|AM|third|night|| +9985|AAAAAAAACAHCAAAA|9985|2|46|25|AM|third|night|| +9986|AAAAAAAADAHCAAAA|9986|2|46|26|AM|third|night|| +9987|AAAAAAAAEAHCAAAA|9987|2|46|27|AM|third|night|| +9988|AAAAAAAAFAHCAAAA|9988|2|46|28|AM|third|night|| +9989|AAAAAAAAGAHCAAAA|9989|2|46|29|AM|third|night|| +9990|AAAAAAAAHAHCAAAA|9990|2|46|30|AM|third|night|| +9991|AAAAAAAAIAHCAAAA|9991|2|46|31|AM|third|night|| +9992|AAAAAAAAJAHCAAAA|9992|2|46|32|AM|third|night|| +9993|AAAAAAAAKAHCAAAA|9993|2|46|33|AM|third|night|| +9994|AAAAAAAALAHCAAAA|9994|2|46|34|AM|third|night|| +9995|AAAAAAAAMAHCAAAA|9995|2|46|35|AM|third|night|| +9996|AAAAAAAANAHCAAAA|9996|2|46|36|AM|third|night|| +9997|AAAAAAAAOAHCAAAA|9997|2|46|37|AM|third|night|| +9998|AAAAAAAAPAHCAAAA|9998|2|46|38|AM|third|night|| +9999|AAAAAAAAABHCAAAA|9999|2|46|39|AM|third|night|| +10000|AAAAAAAABBHCAAAA|10000|2|46|40|AM|third|night|| +10001|AAAAAAAACBHCAAAA|10001|2|46|41|AM|third|night|| +10002|AAAAAAAADBHCAAAA|10002|2|46|42|AM|third|night|| +10003|AAAAAAAAEBHCAAAA|10003|2|46|43|AM|third|night|| +10004|AAAAAAAAFBHCAAAA|10004|2|46|44|AM|third|night|| +10005|AAAAAAAAGBHCAAAA|10005|2|46|45|AM|third|night|| +10006|AAAAAAAAHBHCAAAA|10006|2|46|46|AM|third|night|| +10007|AAAAAAAAIBHCAAAA|10007|2|46|47|AM|third|night|| +10008|AAAAAAAAJBHCAAAA|10008|2|46|48|AM|third|night|| +10009|AAAAAAAAKBHCAAAA|10009|2|46|49|AM|third|night|| +10010|AAAAAAAALBHCAAAA|10010|2|46|50|AM|third|night|| +10011|AAAAAAAAMBHCAAAA|10011|2|46|51|AM|third|night|| +10012|AAAAAAAANBHCAAAA|10012|2|46|52|AM|third|night|| +10013|AAAAAAAAOBHCAAAA|10013|2|46|53|AM|third|night|| +10014|AAAAAAAAPBHCAAAA|10014|2|46|54|AM|third|night|| +10015|AAAAAAAAACHCAAAA|10015|2|46|55|AM|third|night|| +10016|AAAAAAAABCHCAAAA|10016|2|46|56|AM|third|night|| +10017|AAAAAAAACCHCAAAA|10017|2|46|57|AM|third|night|| +10018|AAAAAAAADCHCAAAA|10018|2|46|58|AM|third|night|| +10019|AAAAAAAAECHCAAAA|10019|2|46|59|AM|third|night|| +10020|AAAAAAAAFCHCAAAA|10020|2|47|0|AM|third|night|| +10021|AAAAAAAAGCHCAAAA|10021|2|47|1|AM|third|night|| +10022|AAAAAAAAHCHCAAAA|10022|2|47|2|AM|third|night|| +10023|AAAAAAAAICHCAAAA|10023|2|47|3|AM|third|night|| +10024|AAAAAAAAJCHCAAAA|10024|2|47|4|AM|third|night|| +10025|AAAAAAAAKCHCAAAA|10025|2|47|5|AM|third|night|| +10026|AAAAAAAALCHCAAAA|10026|2|47|6|AM|third|night|| +10027|AAAAAAAAMCHCAAAA|10027|2|47|7|AM|third|night|| +10028|AAAAAAAANCHCAAAA|10028|2|47|8|AM|third|night|| +10029|AAAAAAAAOCHCAAAA|10029|2|47|9|AM|third|night|| +10030|AAAAAAAAPCHCAAAA|10030|2|47|10|AM|third|night|| +10031|AAAAAAAAADHCAAAA|10031|2|47|11|AM|third|night|| +10032|AAAAAAAABDHCAAAA|10032|2|47|12|AM|third|night|| +10033|AAAAAAAACDHCAAAA|10033|2|47|13|AM|third|night|| +10034|AAAAAAAADDHCAAAA|10034|2|47|14|AM|third|night|| +10035|AAAAAAAAEDHCAAAA|10035|2|47|15|AM|third|night|| +10036|AAAAAAAAFDHCAAAA|10036|2|47|16|AM|third|night|| +10037|AAAAAAAAGDHCAAAA|10037|2|47|17|AM|third|night|| +10038|AAAAAAAAHDHCAAAA|10038|2|47|18|AM|third|night|| +10039|AAAAAAAAIDHCAAAA|10039|2|47|19|AM|third|night|| +10040|AAAAAAAAJDHCAAAA|10040|2|47|20|AM|third|night|| +10041|AAAAAAAAKDHCAAAA|10041|2|47|21|AM|third|night|| +10042|AAAAAAAALDHCAAAA|10042|2|47|22|AM|third|night|| +10043|AAAAAAAAMDHCAAAA|10043|2|47|23|AM|third|night|| +10044|AAAAAAAANDHCAAAA|10044|2|47|24|AM|third|night|| +10045|AAAAAAAAODHCAAAA|10045|2|47|25|AM|third|night|| +10046|AAAAAAAAPDHCAAAA|10046|2|47|26|AM|third|night|| +10047|AAAAAAAAAEHCAAAA|10047|2|47|27|AM|third|night|| +10048|AAAAAAAABEHCAAAA|10048|2|47|28|AM|third|night|| +10049|AAAAAAAACEHCAAAA|10049|2|47|29|AM|third|night|| +10050|AAAAAAAADEHCAAAA|10050|2|47|30|AM|third|night|| +10051|AAAAAAAAEEHCAAAA|10051|2|47|31|AM|third|night|| +10052|AAAAAAAAFEHCAAAA|10052|2|47|32|AM|third|night|| +10053|AAAAAAAAGEHCAAAA|10053|2|47|33|AM|third|night|| +10054|AAAAAAAAHEHCAAAA|10054|2|47|34|AM|third|night|| +10055|AAAAAAAAIEHCAAAA|10055|2|47|35|AM|third|night|| +10056|AAAAAAAAJEHCAAAA|10056|2|47|36|AM|third|night|| +10057|AAAAAAAAKEHCAAAA|10057|2|47|37|AM|third|night|| +10058|AAAAAAAALEHCAAAA|10058|2|47|38|AM|third|night|| +10059|AAAAAAAAMEHCAAAA|10059|2|47|39|AM|third|night|| +10060|AAAAAAAANEHCAAAA|10060|2|47|40|AM|third|night|| +10061|AAAAAAAAOEHCAAAA|10061|2|47|41|AM|third|night|| +10062|AAAAAAAAPEHCAAAA|10062|2|47|42|AM|third|night|| +10063|AAAAAAAAAFHCAAAA|10063|2|47|43|AM|third|night|| +10064|AAAAAAAABFHCAAAA|10064|2|47|44|AM|third|night|| +10065|AAAAAAAACFHCAAAA|10065|2|47|45|AM|third|night|| +10066|AAAAAAAADFHCAAAA|10066|2|47|46|AM|third|night|| +10067|AAAAAAAAEFHCAAAA|10067|2|47|47|AM|third|night|| +10068|AAAAAAAAFFHCAAAA|10068|2|47|48|AM|third|night|| +10069|AAAAAAAAGFHCAAAA|10069|2|47|49|AM|third|night|| +10070|AAAAAAAAHFHCAAAA|10070|2|47|50|AM|third|night|| +10071|AAAAAAAAIFHCAAAA|10071|2|47|51|AM|third|night|| +10072|AAAAAAAAJFHCAAAA|10072|2|47|52|AM|third|night|| +10073|AAAAAAAAKFHCAAAA|10073|2|47|53|AM|third|night|| +10074|AAAAAAAALFHCAAAA|10074|2|47|54|AM|third|night|| +10075|AAAAAAAAMFHCAAAA|10075|2|47|55|AM|third|night|| +10076|AAAAAAAANFHCAAAA|10076|2|47|56|AM|third|night|| +10077|AAAAAAAAOFHCAAAA|10077|2|47|57|AM|third|night|| +10078|AAAAAAAAPFHCAAAA|10078|2|47|58|AM|third|night|| +10079|AAAAAAAAAGHCAAAA|10079|2|47|59|AM|third|night|| +10080|AAAAAAAABGHCAAAA|10080|2|48|0|AM|third|night|| +10081|AAAAAAAACGHCAAAA|10081|2|48|1|AM|third|night|| +10082|AAAAAAAADGHCAAAA|10082|2|48|2|AM|third|night|| +10083|AAAAAAAAEGHCAAAA|10083|2|48|3|AM|third|night|| +10084|AAAAAAAAFGHCAAAA|10084|2|48|4|AM|third|night|| +10085|AAAAAAAAGGHCAAAA|10085|2|48|5|AM|third|night|| +10086|AAAAAAAAHGHCAAAA|10086|2|48|6|AM|third|night|| +10087|AAAAAAAAIGHCAAAA|10087|2|48|7|AM|third|night|| +10088|AAAAAAAAJGHCAAAA|10088|2|48|8|AM|third|night|| +10089|AAAAAAAAKGHCAAAA|10089|2|48|9|AM|third|night|| +10090|AAAAAAAALGHCAAAA|10090|2|48|10|AM|third|night|| +10091|AAAAAAAAMGHCAAAA|10091|2|48|11|AM|third|night|| +10092|AAAAAAAANGHCAAAA|10092|2|48|12|AM|third|night|| +10093|AAAAAAAAOGHCAAAA|10093|2|48|13|AM|third|night|| +10094|AAAAAAAAPGHCAAAA|10094|2|48|14|AM|third|night|| +10095|AAAAAAAAAHHCAAAA|10095|2|48|15|AM|third|night|| +10096|AAAAAAAABHHCAAAA|10096|2|48|16|AM|third|night|| +10097|AAAAAAAACHHCAAAA|10097|2|48|17|AM|third|night|| +10098|AAAAAAAADHHCAAAA|10098|2|48|18|AM|third|night|| +10099|AAAAAAAAEHHCAAAA|10099|2|48|19|AM|third|night|| +10100|AAAAAAAAFHHCAAAA|10100|2|48|20|AM|third|night|| +10101|AAAAAAAAGHHCAAAA|10101|2|48|21|AM|third|night|| +10102|AAAAAAAAHHHCAAAA|10102|2|48|22|AM|third|night|| +10103|AAAAAAAAIHHCAAAA|10103|2|48|23|AM|third|night|| +10104|AAAAAAAAJHHCAAAA|10104|2|48|24|AM|third|night|| +10105|AAAAAAAAKHHCAAAA|10105|2|48|25|AM|third|night|| +10106|AAAAAAAALHHCAAAA|10106|2|48|26|AM|third|night|| +10107|AAAAAAAAMHHCAAAA|10107|2|48|27|AM|third|night|| +10108|AAAAAAAANHHCAAAA|10108|2|48|28|AM|third|night|| +10109|AAAAAAAAOHHCAAAA|10109|2|48|29|AM|third|night|| +10110|AAAAAAAAPHHCAAAA|10110|2|48|30|AM|third|night|| +10111|AAAAAAAAAIHCAAAA|10111|2|48|31|AM|third|night|| +10112|AAAAAAAABIHCAAAA|10112|2|48|32|AM|third|night|| +10113|AAAAAAAACIHCAAAA|10113|2|48|33|AM|third|night|| +10114|AAAAAAAADIHCAAAA|10114|2|48|34|AM|third|night|| +10115|AAAAAAAAEIHCAAAA|10115|2|48|35|AM|third|night|| +10116|AAAAAAAAFIHCAAAA|10116|2|48|36|AM|third|night|| +10117|AAAAAAAAGIHCAAAA|10117|2|48|37|AM|third|night|| +10118|AAAAAAAAHIHCAAAA|10118|2|48|38|AM|third|night|| +10119|AAAAAAAAIIHCAAAA|10119|2|48|39|AM|third|night|| +10120|AAAAAAAAJIHCAAAA|10120|2|48|40|AM|third|night|| +10121|AAAAAAAAKIHCAAAA|10121|2|48|41|AM|third|night|| +10122|AAAAAAAALIHCAAAA|10122|2|48|42|AM|third|night|| +10123|AAAAAAAAMIHCAAAA|10123|2|48|43|AM|third|night|| +10124|AAAAAAAANIHCAAAA|10124|2|48|44|AM|third|night|| +10125|AAAAAAAAOIHCAAAA|10125|2|48|45|AM|third|night|| +10126|AAAAAAAAPIHCAAAA|10126|2|48|46|AM|third|night|| +10127|AAAAAAAAAJHCAAAA|10127|2|48|47|AM|third|night|| +10128|AAAAAAAABJHCAAAA|10128|2|48|48|AM|third|night|| +10129|AAAAAAAACJHCAAAA|10129|2|48|49|AM|third|night|| +10130|AAAAAAAADJHCAAAA|10130|2|48|50|AM|third|night|| +10131|AAAAAAAAEJHCAAAA|10131|2|48|51|AM|third|night|| +10132|AAAAAAAAFJHCAAAA|10132|2|48|52|AM|third|night|| +10133|AAAAAAAAGJHCAAAA|10133|2|48|53|AM|third|night|| +10134|AAAAAAAAHJHCAAAA|10134|2|48|54|AM|third|night|| +10135|AAAAAAAAIJHCAAAA|10135|2|48|55|AM|third|night|| +10136|AAAAAAAAJJHCAAAA|10136|2|48|56|AM|third|night|| +10137|AAAAAAAAKJHCAAAA|10137|2|48|57|AM|third|night|| +10138|AAAAAAAALJHCAAAA|10138|2|48|58|AM|third|night|| +10139|AAAAAAAAMJHCAAAA|10139|2|48|59|AM|third|night|| +10140|AAAAAAAANJHCAAAA|10140|2|49|0|AM|third|night|| +10141|AAAAAAAAOJHCAAAA|10141|2|49|1|AM|third|night|| +10142|AAAAAAAAPJHCAAAA|10142|2|49|2|AM|third|night|| +10143|AAAAAAAAAKHCAAAA|10143|2|49|3|AM|third|night|| +10144|AAAAAAAABKHCAAAA|10144|2|49|4|AM|third|night|| +10145|AAAAAAAACKHCAAAA|10145|2|49|5|AM|third|night|| +10146|AAAAAAAADKHCAAAA|10146|2|49|6|AM|third|night|| +10147|AAAAAAAAEKHCAAAA|10147|2|49|7|AM|third|night|| +10148|AAAAAAAAFKHCAAAA|10148|2|49|8|AM|third|night|| +10149|AAAAAAAAGKHCAAAA|10149|2|49|9|AM|third|night|| +10150|AAAAAAAAHKHCAAAA|10150|2|49|10|AM|third|night|| +10151|AAAAAAAAIKHCAAAA|10151|2|49|11|AM|third|night|| +10152|AAAAAAAAJKHCAAAA|10152|2|49|12|AM|third|night|| +10153|AAAAAAAAKKHCAAAA|10153|2|49|13|AM|third|night|| +10154|AAAAAAAALKHCAAAA|10154|2|49|14|AM|third|night|| +10155|AAAAAAAAMKHCAAAA|10155|2|49|15|AM|third|night|| +10156|AAAAAAAANKHCAAAA|10156|2|49|16|AM|third|night|| +10157|AAAAAAAAOKHCAAAA|10157|2|49|17|AM|third|night|| +10158|AAAAAAAAPKHCAAAA|10158|2|49|18|AM|third|night|| +10159|AAAAAAAAALHCAAAA|10159|2|49|19|AM|third|night|| +10160|AAAAAAAABLHCAAAA|10160|2|49|20|AM|third|night|| +10161|AAAAAAAACLHCAAAA|10161|2|49|21|AM|third|night|| +10162|AAAAAAAADLHCAAAA|10162|2|49|22|AM|third|night|| +10163|AAAAAAAAELHCAAAA|10163|2|49|23|AM|third|night|| +10164|AAAAAAAAFLHCAAAA|10164|2|49|24|AM|third|night|| +10165|AAAAAAAAGLHCAAAA|10165|2|49|25|AM|third|night|| +10166|AAAAAAAAHLHCAAAA|10166|2|49|26|AM|third|night|| +10167|AAAAAAAAILHCAAAA|10167|2|49|27|AM|third|night|| +10168|AAAAAAAAJLHCAAAA|10168|2|49|28|AM|third|night|| +10169|AAAAAAAAKLHCAAAA|10169|2|49|29|AM|third|night|| +10170|AAAAAAAALLHCAAAA|10170|2|49|30|AM|third|night|| +10171|AAAAAAAAMLHCAAAA|10171|2|49|31|AM|third|night|| +10172|AAAAAAAANLHCAAAA|10172|2|49|32|AM|third|night|| +10173|AAAAAAAAOLHCAAAA|10173|2|49|33|AM|third|night|| +10174|AAAAAAAAPLHCAAAA|10174|2|49|34|AM|third|night|| +10175|AAAAAAAAAMHCAAAA|10175|2|49|35|AM|third|night|| +10176|AAAAAAAABMHCAAAA|10176|2|49|36|AM|third|night|| +10177|AAAAAAAACMHCAAAA|10177|2|49|37|AM|third|night|| +10178|AAAAAAAADMHCAAAA|10178|2|49|38|AM|third|night|| +10179|AAAAAAAAEMHCAAAA|10179|2|49|39|AM|third|night|| +10180|AAAAAAAAFMHCAAAA|10180|2|49|40|AM|third|night|| +10181|AAAAAAAAGMHCAAAA|10181|2|49|41|AM|third|night|| +10182|AAAAAAAAHMHCAAAA|10182|2|49|42|AM|third|night|| +10183|AAAAAAAAIMHCAAAA|10183|2|49|43|AM|third|night|| +10184|AAAAAAAAJMHCAAAA|10184|2|49|44|AM|third|night|| +10185|AAAAAAAAKMHCAAAA|10185|2|49|45|AM|third|night|| +10186|AAAAAAAALMHCAAAA|10186|2|49|46|AM|third|night|| +10187|AAAAAAAAMMHCAAAA|10187|2|49|47|AM|third|night|| +10188|AAAAAAAANMHCAAAA|10188|2|49|48|AM|third|night|| +10189|AAAAAAAAOMHCAAAA|10189|2|49|49|AM|third|night|| +10190|AAAAAAAAPMHCAAAA|10190|2|49|50|AM|third|night|| +10191|AAAAAAAAANHCAAAA|10191|2|49|51|AM|third|night|| +10192|AAAAAAAABNHCAAAA|10192|2|49|52|AM|third|night|| +10193|AAAAAAAACNHCAAAA|10193|2|49|53|AM|third|night|| +10194|AAAAAAAADNHCAAAA|10194|2|49|54|AM|third|night|| +10195|AAAAAAAAENHCAAAA|10195|2|49|55|AM|third|night|| +10196|AAAAAAAAFNHCAAAA|10196|2|49|56|AM|third|night|| +10197|AAAAAAAAGNHCAAAA|10197|2|49|57|AM|third|night|| +10198|AAAAAAAAHNHCAAAA|10198|2|49|58|AM|third|night|| +10199|AAAAAAAAINHCAAAA|10199|2|49|59|AM|third|night|| +10200|AAAAAAAAJNHCAAAA|10200|2|50|0|AM|third|night|| +10201|AAAAAAAAKNHCAAAA|10201|2|50|1|AM|third|night|| +10202|AAAAAAAALNHCAAAA|10202|2|50|2|AM|third|night|| +10203|AAAAAAAAMNHCAAAA|10203|2|50|3|AM|third|night|| +10204|AAAAAAAANNHCAAAA|10204|2|50|4|AM|third|night|| +10205|AAAAAAAAONHCAAAA|10205|2|50|5|AM|third|night|| +10206|AAAAAAAAPNHCAAAA|10206|2|50|6|AM|third|night|| +10207|AAAAAAAAAOHCAAAA|10207|2|50|7|AM|third|night|| +10208|AAAAAAAABOHCAAAA|10208|2|50|8|AM|third|night|| +10209|AAAAAAAACOHCAAAA|10209|2|50|9|AM|third|night|| +10210|AAAAAAAADOHCAAAA|10210|2|50|10|AM|third|night|| +10211|AAAAAAAAEOHCAAAA|10211|2|50|11|AM|third|night|| +10212|AAAAAAAAFOHCAAAA|10212|2|50|12|AM|third|night|| +10213|AAAAAAAAGOHCAAAA|10213|2|50|13|AM|third|night|| +10214|AAAAAAAAHOHCAAAA|10214|2|50|14|AM|third|night|| +10215|AAAAAAAAIOHCAAAA|10215|2|50|15|AM|third|night|| +10216|AAAAAAAAJOHCAAAA|10216|2|50|16|AM|third|night|| +10217|AAAAAAAAKOHCAAAA|10217|2|50|17|AM|third|night|| +10218|AAAAAAAALOHCAAAA|10218|2|50|18|AM|third|night|| +10219|AAAAAAAAMOHCAAAA|10219|2|50|19|AM|third|night|| +10220|AAAAAAAANOHCAAAA|10220|2|50|20|AM|third|night|| +10221|AAAAAAAAOOHCAAAA|10221|2|50|21|AM|third|night|| +10222|AAAAAAAAPOHCAAAA|10222|2|50|22|AM|third|night|| +10223|AAAAAAAAAPHCAAAA|10223|2|50|23|AM|third|night|| +10224|AAAAAAAABPHCAAAA|10224|2|50|24|AM|third|night|| +10225|AAAAAAAACPHCAAAA|10225|2|50|25|AM|third|night|| +10226|AAAAAAAADPHCAAAA|10226|2|50|26|AM|third|night|| +10227|AAAAAAAAEPHCAAAA|10227|2|50|27|AM|third|night|| +10228|AAAAAAAAFPHCAAAA|10228|2|50|28|AM|third|night|| +10229|AAAAAAAAGPHCAAAA|10229|2|50|29|AM|third|night|| +10230|AAAAAAAAHPHCAAAA|10230|2|50|30|AM|third|night|| +10231|AAAAAAAAIPHCAAAA|10231|2|50|31|AM|third|night|| +10232|AAAAAAAAJPHCAAAA|10232|2|50|32|AM|third|night|| +10233|AAAAAAAAKPHCAAAA|10233|2|50|33|AM|third|night|| +10234|AAAAAAAALPHCAAAA|10234|2|50|34|AM|third|night|| +10235|AAAAAAAAMPHCAAAA|10235|2|50|35|AM|third|night|| +10236|AAAAAAAANPHCAAAA|10236|2|50|36|AM|third|night|| +10237|AAAAAAAAOPHCAAAA|10237|2|50|37|AM|third|night|| +10238|AAAAAAAAPPHCAAAA|10238|2|50|38|AM|third|night|| +10239|AAAAAAAAAAICAAAA|10239|2|50|39|AM|third|night|| +10240|AAAAAAAABAICAAAA|10240|2|50|40|AM|third|night|| +10241|AAAAAAAACAICAAAA|10241|2|50|41|AM|third|night|| +10242|AAAAAAAADAICAAAA|10242|2|50|42|AM|third|night|| +10243|AAAAAAAAEAICAAAA|10243|2|50|43|AM|third|night|| +10244|AAAAAAAAFAICAAAA|10244|2|50|44|AM|third|night|| +10245|AAAAAAAAGAICAAAA|10245|2|50|45|AM|third|night|| +10246|AAAAAAAAHAICAAAA|10246|2|50|46|AM|third|night|| +10247|AAAAAAAAIAICAAAA|10247|2|50|47|AM|third|night|| +10248|AAAAAAAAJAICAAAA|10248|2|50|48|AM|third|night|| +10249|AAAAAAAAKAICAAAA|10249|2|50|49|AM|third|night|| +10250|AAAAAAAALAICAAAA|10250|2|50|50|AM|third|night|| +10251|AAAAAAAAMAICAAAA|10251|2|50|51|AM|third|night|| +10252|AAAAAAAANAICAAAA|10252|2|50|52|AM|third|night|| +10253|AAAAAAAAOAICAAAA|10253|2|50|53|AM|third|night|| +10254|AAAAAAAAPAICAAAA|10254|2|50|54|AM|third|night|| +10255|AAAAAAAAABICAAAA|10255|2|50|55|AM|third|night|| +10256|AAAAAAAABBICAAAA|10256|2|50|56|AM|third|night|| +10257|AAAAAAAACBICAAAA|10257|2|50|57|AM|third|night|| +10258|AAAAAAAADBICAAAA|10258|2|50|58|AM|third|night|| +10259|AAAAAAAAEBICAAAA|10259|2|50|59|AM|third|night|| +10260|AAAAAAAAFBICAAAA|10260|2|51|0|AM|third|night|| +10261|AAAAAAAAGBICAAAA|10261|2|51|1|AM|third|night|| +10262|AAAAAAAAHBICAAAA|10262|2|51|2|AM|third|night|| +10263|AAAAAAAAIBICAAAA|10263|2|51|3|AM|third|night|| +10264|AAAAAAAAJBICAAAA|10264|2|51|4|AM|third|night|| +10265|AAAAAAAAKBICAAAA|10265|2|51|5|AM|third|night|| +10266|AAAAAAAALBICAAAA|10266|2|51|6|AM|third|night|| +10267|AAAAAAAAMBICAAAA|10267|2|51|7|AM|third|night|| +10268|AAAAAAAANBICAAAA|10268|2|51|8|AM|third|night|| +10269|AAAAAAAAOBICAAAA|10269|2|51|9|AM|third|night|| +10270|AAAAAAAAPBICAAAA|10270|2|51|10|AM|third|night|| +10271|AAAAAAAAACICAAAA|10271|2|51|11|AM|third|night|| +10272|AAAAAAAABCICAAAA|10272|2|51|12|AM|third|night|| +10273|AAAAAAAACCICAAAA|10273|2|51|13|AM|third|night|| +10274|AAAAAAAADCICAAAA|10274|2|51|14|AM|third|night|| +10275|AAAAAAAAECICAAAA|10275|2|51|15|AM|third|night|| +10276|AAAAAAAAFCICAAAA|10276|2|51|16|AM|third|night|| +10277|AAAAAAAAGCICAAAA|10277|2|51|17|AM|third|night|| +10278|AAAAAAAAHCICAAAA|10278|2|51|18|AM|third|night|| +10279|AAAAAAAAICICAAAA|10279|2|51|19|AM|third|night|| +10280|AAAAAAAAJCICAAAA|10280|2|51|20|AM|third|night|| +10281|AAAAAAAAKCICAAAA|10281|2|51|21|AM|third|night|| +10282|AAAAAAAALCICAAAA|10282|2|51|22|AM|third|night|| +10283|AAAAAAAAMCICAAAA|10283|2|51|23|AM|third|night|| +10284|AAAAAAAANCICAAAA|10284|2|51|24|AM|third|night|| +10285|AAAAAAAAOCICAAAA|10285|2|51|25|AM|third|night|| +10286|AAAAAAAAPCICAAAA|10286|2|51|26|AM|third|night|| +10287|AAAAAAAAADICAAAA|10287|2|51|27|AM|third|night|| +10288|AAAAAAAABDICAAAA|10288|2|51|28|AM|third|night|| +10289|AAAAAAAACDICAAAA|10289|2|51|29|AM|third|night|| +10290|AAAAAAAADDICAAAA|10290|2|51|30|AM|third|night|| +10291|AAAAAAAAEDICAAAA|10291|2|51|31|AM|third|night|| +10292|AAAAAAAAFDICAAAA|10292|2|51|32|AM|third|night|| +10293|AAAAAAAAGDICAAAA|10293|2|51|33|AM|third|night|| +10294|AAAAAAAAHDICAAAA|10294|2|51|34|AM|third|night|| +10295|AAAAAAAAIDICAAAA|10295|2|51|35|AM|third|night|| +10296|AAAAAAAAJDICAAAA|10296|2|51|36|AM|third|night|| +10297|AAAAAAAAKDICAAAA|10297|2|51|37|AM|third|night|| +10298|AAAAAAAALDICAAAA|10298|2|51|38|AM|third|night|| +10299|AAAAAAAAMDICAAAA|10299|2|51|39|AM|third|night|| +10300|AAAAAAAANDICAAAA|10300|2|51|40|AM|third|night|| +10301|AAAAAAAAODICAAAA|10301|2|51|41|AM|third|night|| +10302|AAAAAAAAPDICAAAA|10302|2|51|42|AM|third|night|| +10303|AAAAAAAAAEICAAAA|10303|2|51|43|AM|third|night|| +10304|AAAAAAAABEICAAAA|10304|2|51|44|AM|third|night|| +10305|AAAAAAAACEICAAAA|10305|2|51|45|AM|third|night|| +10306|AAAAAAAADEICAAAA|10306|2|51|46|AM|third|night|| +10307|AAAAAAAAEEICAAAA|10307|2|51|47|AM|third|night|| +10308|AAAAAAAAFEICAAAA|10308|2|51|48|AM|third|night|| +10309|AAAAAAAAGEICAAAA|10309|2|51|49|AM|third|night|| +10310|AAAAAAAAHEICAAAA|10310|2|51|50|AM|third|night|| +10311|AAAAAAAAIEICAAAA|10311|2|51|51|AM|third|night|| +10312|AAAAAAAAJEICAAAA|10312|2|51|52|AM|third|night|| +10313|AAAAAAAAKEICAAAA|10313|2|51|53|AM|third|night|| +10314|AAAAAAAALEICAAAA|10314|2|51|54|AM|third|night|| +10315|AAAAAAAAMEICAAAA|10315|2|51|55|AM|third|night|| +10316|AAAAAAAANEICAAAA|10316|2|51|56|AM|third|night|| +10317|AAAAAAAAOEICAAAA|10317|2|51|57|AM|third|night|| +10318|AAAAAAAAPEICAAAA|10318|2|51|58|AM|third|night|| +10319|AAAAAAAAAFICAAAA|10319|2|51|59|AM|third|night|| +10320|AAAAAAAABFICAAAA|10320|2|52|0|AM|third|night|| +10321|AAAAAAAACFICAAAA|10321|2|52|1|AM|third|night|| +10322|AAAAAAAADFICAAAA|10322|2|52|2|AM|third|night|| +10323|AAAAAAAAEFICAAAA|10323|2|52|3|AM|third|night|| +10324|AAAAAAAAFFICAAAA|10324|2|52|4|AM|third|night|| +10325|AAAAAAAAGFICAAAA|10325|2|52|5|AM|third|night|| +10326|AAAAAAAAHFICAAAA|10326|2|52|6|AM|third|night|| +10327|AAAAAAAAIFICAAAA|10327|2|52|7|AM|third|night|| +10328|AAAAAAAAJFICAAAA|10328|2|52|8|AM|third|night|| +10329|AAAAAAAAKFICAAAA|10329|2|52|9|AM|third|night|| +10330|AAAAAAAALFICAAAA|10330|2|52|10|AM|third|night|| +10331|AAAAAAAAMFICAAAA|10331|2|52|11|AM|third|night|| +10332|AAAAAAAANFICAAAA|10332|2|52|12|AM|third|night|| +10333|AAAAAAAAOFICAAAA|10333|2|52|13|AM|third|night|| +10334|AAAAAAAAPFICAAAA|10334|2|52|14|AM|third|night|| +10335|AAAAAAAAAGICAAAA|10335|2|52|15|AM|third|night|| +10336|AAAAAAAABGICAAAA|10336|2|52|16|AM|third|night|| +10337|AAAAAAAACGICAAAA|10337|2|52|17|AM|third|night|| +10338|AAAAAAAADGICAAAA|10338|2|52|18|AM|third|night|| +10339|AAAAAAAAEGICAAAA|10339|2|52|19|AM|third|night|| +10340|AAAAAAAAFGICAAAA|10340|2|52|20|AM|third|night|| +10341|AAAAAAAAGGICAAAA|10341|2|52|21|AM|third|night|| +10342|AAAAAAAAHGICAAAA|10342|2|52|22|AM|third|night|| +10343|AAAAAAAAIGICAAAA|10343|2|52|23|AM|third|night|| +10344|AAAAAAAAJGICAAAA|10344|2|52|24|AM|third|night|| +10345|AAAAAAAAKGICAAAA|10345|2|52|25|AM|third|night|| +10346|AAAAAAAALGICAAAA|10346|2|52|26|AM|third|night|| +10347|AAAAAAAAMGICAAAA|10347|2|52|27|AM|third|night|| +10348|AAAAAAAANGICAAAA|10348|2|52|28|AM|third|night|| +10349|AAAAAAAAOGICAAAA|10349|2|52|29|AM|third|night|| +10350|AAAAAAAAPGICAAAA|10350|2|52|30|AM|third|night|| +10351|AAAAAAAAAHICAAAA|10351|2|52|31|AM|third|night|| +10352|AAAAAAAABHICAAAA|10352|2|52|32|AM|third|night|| +10353|AAAAAAAACHICAAAA|10353|2|52|33|AM|third|night|| +10354|AAAAAAAADHICAAAA|10354|2|52|34|AM|third|night|| +10355|AAAAAAAAEHICAAAA|10355|2|52|35|AM|third|night|| +10356|AAAAAAAAFHICAAAA|10356|2|52|36|AM|third|night|| +10357|AAAAAAAAGHICAAAA|10357|2|52|37|AM|third|night|| +10358|AAAAAAAAHHICAAAA|10358|2|52|38|AM|third|night|| +10359|AAAAAAAAIHICAAAA|10359|2|52|39|AM|third|night|| +10360|AAAAAAAAJHICAAAA|10360|2|52|40|AM|third|night|| +10361|AAAAAAAAKHICAAAA|10361|2|52|41|AM|third|night|| +10362|AAAAAAAALHICAAAA|10362|2|52|42|AM|third|night|| +10363|AAAAAAAAMHICAAAA|10363|2|52|43|AM|third|night|| +10364|AAAAAAAANHICAAAA|10364|2|52|44|AM|third|night|| +10365|AAAAAAAAOHICAAAA|10365|2|52|45|AM|third|night|| +10366|AAAAAAAAPHICAAAA|10366|2|52|46|AM|third|night|| +10367|AAAAAAAAAIICAAAA|10367|2|52|47|AM|third|night|| +10368|AAAAAAAABIICAAAA|10368|2|52|48|AM|third|night|| +10369|AAAAAAAACIICAAAA|10369|2|52|49|AM|third|night|| +10370|AAAAAAAADIICAAAA|10370|2|52|50|AM|third|night|| +10371|AAAAAAAAEIICAAAA|10371|2|52|51|AM|third|night|| +10372|AAAAAAAAFIICAAAA|10372|2|52|52|AM|third|night|| +10373|AAAAAAAAGIICAAAA|10373|2|52|53|AM|third|night|| +10374|AAAAAAAAHIICAAAA|10374|2|52|54|AM|third|night|| +10375|AAAAAAAAIIICAAAA|10375|2|52|55|AM|third|night|| +10376|AAAAAAAAJIICAAAA|10376|2|52|56|AM|third|night|| +10377|AAAAAAAAKIICAAAA|10377|2|52|57|AM|third|night|| +10378|AAAAAAAALIICAAAA|10378|2|52|58|AM|third|night|| +10379|AAAAAAAAMIICAAAA|10379|2|52|59|AM|third|night|| +10380|AAAAAAAANIICAAAA|10380|2|53|0|AM|third|night|| +10381|AAAAAAAAOIICAAAA|10381|2|53|1|AM|third|night|| +10382|AAAAAAAAPIICAAAA|10382|2|53|2|AM|third|night|| +10383|AAAAAAAAAJICAAAA|10383|2|53|3|AM|third|night|| +10384|AAAAAAAABJICAAAA|10384|2|53|4|AM|third|night|| +10385|AAAAAAAACJICAAAA|10385|2|53|5|AM|third|night|| +10386|AAAAAAAADJICAAAA|10386|2|53|6|AM|third|night|| +10387|AAAAAAAAEJICAAAA|10387|2|53|7|AM|third|night|| +10388|AAAAAAAAFJICAAAA|10388|2|53|8|AM|third|night|| +10389|AAAAAAAAGJICAAAA|10389|2|53|9|AM|third|night|| +10390|AAAAAAAAHJICAAAA|10390|2|53|10|AM|third|night|| +10391|AAAAAAAAIJICAAAA|10391|2|53|11|AM|third|night|| +10392|AAAAAAAAJJICAAAA|10392|2|53|12|AM|third|night|| +10393|AAAAAAAAKJICAAAA|10393|2|53|13|AM|third|night|| +10394|AAAAAAAALJICAAAA|10394|2|53|14|AM|third|night|| +10395|AAAAAAAAMJICAAAA|10395|2|53|15|AM|third|night|| +10396|AAAAAAAANJICAAAA|10396|2|53|16|AM|third|night|| +10397|AAAAAAAAOJICAAAA|10397|2|53|17|AM|third|night|| +10398|AAAAAAAAPJICAAAA|10398|2|53|18|AM|third|night|| +10399|AAAAAAAAAKICAAAA|10399|2|53|19|AM|third|night|| +10400|AAAAAAAABKICAAAA|10400|2|53|20|AM|third|night|| +10401|AAAAAAAACKICAAAA|10401|2|53|21|AM|third|night|| +10402|AAAAAAAADKICAAAA|10402|2|53|22|AM|third|night|| +10403|AAAAAAAAEKICAAAA|10403|2|53|23|AM|third|night|| +10404|AAAAAAAAFKICAAAA|10404|2|53|24|AM|third|night|| +10405|AAAAAAAAGKICAAAA|10405|2|53|25|AM|third|night|| +10406|AAAAAAAAHKICAAAA|10406|2|53|26|AM|third|night|| +10407|AAAAAAAAIKICAAAA|10407|2|53|27|AM|third|night|| +10408|AAAAAAAAJKICAAAA|10408|2|53|28|AM|third|night|| +10409|AAAAAAAAKKICAAAA|10409|2|53|29|AM|third|night|| +10410|AAAAAAAALKICAAAA|10410|2|53|30|AM|third|night|| +10411|AAAAAAAAMKICAAAA|10411|2|53|31|AM|third|night|| +10412|AAAAAAAANKICAAAA|10412|2|53|32|AM|third|night|| +10413|AAAAAAAAOKICAAAA|10413|2|53|33|AM|third|night|| +10414|AAAAAAAAPKICAAAA|10414|2|53|34|AM|third|night|| +10415|AAAAAAAAALICAAAA|10415|2|53|35|AM|third|night|| +10416|AAAAAAAABLICAAAA|10416|2|53|36|AM|third|night|| +10417|AAAAAAAACLICAAAA|10417|2|53|37|AM|third|night|| +10418|AAAAAAAADLICAAAA|10418|2|53|38|AM|third|night|| +10419|AAAAAAAAELICAAAA|10419|2|53|39|AM|third|night|| +10420|AAAAAAAAFLICAAAA|10420|2|53|40|AM|third|night|| +10421|AAAAAAAAGLICAAAA|10421|2|53|41|AM|third|night|| +10422|AAAAAAAAHLICAAAA|10422|2|53|42|AM|third|night|| +10423|AAAAAAAAILICAAAA|10423|2|53|43|AM|third|night|| +10424|AAAAAAAAJLICAAAA|10424|2|53|44|AM|third|night|| +10425|AAAAAAAAKLICAAAA|10425|2|53|45|AM|third|night|| +10426|AAAAAAAALLICAAAA|10426|2|53|46|AM|third|night|| +10427|AAAAAAAAMLICAAAA|10427|2|53|47|AM|third|night|| +10428|AAAAAAAANLICAAAA|10428|2|53|48|AM|third|night|| +10429|AAAAAAAAOLICAAAA|10429|2|53|49|AM|third|night|| +10430|AAAAAAAAPLICAAAA|10430|2|53|50|AM|third|night|| +10431|AAAAAAAAAMICAAAA|10431|2|53|51|AM|third|night|| +10432|AAAAAAAABMICAAAA|10432|2|53|52|AM|third|night|| +10433|AAAAAAAACMICAAAA|10433|2|53|53|AM|third|night|| +10434|AAAAAAAADMICAAAA|10434|2|53|54|AM|third|night|| +10435|AAAAAAAAEMICAAAA|10435|2|53|55|AM|third|night|| +10436|AAAAAAAAFMICAAAA|10436|2|53|56|AM|third|night|| +10437|AAAAAAAAGMICAAAA|10437|2|53|57|AM|third|night|| +10438|AAAAAAAAHMICAAAA|10438|2|53|58|AM|third|night|| +10439|AAAAAAAAIMICAAAA|10439|2|53|59|AM|third|night|| +10440|AAAAAAAAJMICAAAA|10440|2|54|0|AM|third|night|| +10441|AAAAAAAAKMICAAAA|10441|2|54|1|AM|third|night|| +10442|AAAAAAAALMICAAAA|10442|2|54|2|AM|third|night|| +10443|AAAAAAAAMMICAAAA|10443|2|54|3|AM|third|night|| +10444|AAAAAAAANMICAAAA|10444|2|54|4|AM|third|night|| +10445|AAAAAAAAOMICAAAA|10445|2|54|5|AM|third|night|| +10446|AAAAAAAAPMICAAAA|10446|2|54|6|AM|third|night|| +10447|AAAAAAAAANICAAAA|10447|2|54|7|AM|third|night|| +10448|AAAAAAAABNICAAAA|10448|2|54|8|AM|third|night|| +10449|AAAAAAAACNICAAAA|10449|2|54|9|AM|third|night|| +10450|AAAAAAAADNICAAAA|10450|2|54|10|AM|third|night|| +10451|AAAAAAAAENICAAAA|10451|2|54|11|AM|third|night|| +10452|AAAAAAAAFNICAAAA|10452|2|54|12|AM|third|night|| +10453|AAAAAAAAGNICAAAA|10453|2|54|13|AM|third|night|| +10454|AAAAAAAAHNICAAAA|10454|2|54|14|AM|third|night|| +10455|AAAAAAAAINICAAAA|10455|2|54|15|AM|third|night|| +10456|AAAAAAAAJNICAAAA|10456|2|54|16|AM|third|night|| +10457|AAAAAAAAKNICAAAA|10457|2|54|17|AM|third|night|| +10458|AAAAAAAALNICAAAA|10458|2|54|18|AM|third|night|| +10459|AAAAAAAAMNICAAAA|10459|2|54|19|AM|third|night|| +10460|AAAAAAAANNICAAAA|10460|2|54|20|AM|third|night|| +10461|AAAAAAAAONICAAAA|10461|2|54|21|AM|third|night|| +10462|AAAAAAAAPNICAAAA|10462|2|54|22|AM|third|night|| +10463|AAAAAAAAAOICAAAA|10463|2|54|23|AM|third|night|| +10464|AAAAAAAABOICAAAA|10464|2|54|24|AM|third|night|| +10465|AAAAAAAACOICAAAA|10465|2|54|25|AM|third|night|| +10466|AAAAAAAADOICAAAA|10466|2|54|26|AM|third|night|| +10467|AAAAAAAAEOICAAAA|10467|2|54|27|AM|third|night|| +10468|AAAAAAAAFOICAAAA|10468|2|54|28|AM|third|night|| +10469|AAAAAAAAGOICAAAA|10469|2|54|29|AM|third|night|| +10470|AAAAAAAAHOICAAAA|10470|2|54|30|AM|third|night|| +10471|AAAAAAAAIOICAAAA|10471|2|54|31|AM|third|night|| +10472|AAAAAAAAJOICAAAA|10472|2|54|32|AM|third|night|| +10473|AAAAAAAAKOICAAAA|10473|2|54|33|AM|third|night|| +10474|AAAAAAAALOICAAAA|10474|2|54|34|AM|third|night|| +10475|AAAAAAAAMOICAAAA|10475|2|54|35|AM|third|night|| +10476|AAAAAAAANOICAAAA|10476|2|54|36|AM|third|night|| +10477|AAAAAAAAOOICAAAA|10477|2|54|37|AM|third|night|| +10478|AAAAAAAAPOICAAAA|10478|2|54|38|AM|third|night|| +10479|AAAAAAAAAPICAAAA|10479|2|54|39|AM|third|night|| +10480|AAAAAAAABPICAAAA|10480|2|54|40|AM|third|night|| +10481|AAAAAAAACPICAAAA|10481|2|54|41|AM|third|night|| +10482|AAAAAAAADPICAAAA|10482|2|54|42|AM|third|night|| +10483|AAAAAAAAEPICAAAA|10483|2|54|43|AM|third|night|| +10484|AAAAAAAAFPICAAAA|10484|2|54|44|AM|third|night|| +10485|AAAAAAAAGPICAAAA|10485|2|54|45|AM|third|night|| +10486|AAAAAAAAHPICAAAA|10486|2|54|46|AM|third|night|| +10487|AAAAAAAAIPICAAAA|10487|2|54|47|AM|third|night|| +10488|AAAAAAAAJPICAAAA|10488|2|54|48|AM|third|night|| +10489|AAAAAAAAKPICAAAA|10489|2|54|49|AM|third|night|| +10490|AAAAAAAALPICAAAA|10490|2|54|50|AM|third|night|| +10491|AAAAAAAAMPICAAAA|10491|2|54|51|AM|third|night|| +10492|AAAAAAAANPICAAAA|10492|2|54|52|AM|third|night|| +10493|AAAAAAAAOPICAAAA|10493|2|54|53|AM|third|night|| +10494|AAAAAAAAPPICAAAA|10494|2|54|54|AM|third|night|| +10495|AAAAAAAAAAJCAAAA|10495|2|54|55|AM|third|night|| +10496|AAAAAAAABAJCAAAA|10496|2|54|56|AM|third|night|| +10497|AAAAAAAACAJCAAAA|10497|2|54|57|AM|third|night|| +10498|AAAAAAAADAJCAAAA|10498|2|54|58|AM|third|night|| +10499|AAAAAAAAEAJCAAAA|10499|2|54|59|AM|third|night|| +10500|AAAAAAAAFAJCAAAA|10500|2|55|0|AM|third|night|| +10501|AAAAAAAAGAJCAAAA|10501|2|55|1|AM|third|night|| +10502|AAAAAAAAHAJCAAAA|10502|2|55|2|AM|third|night|| +10503|AAAAAAAAIAJCAAAA|10503|2|55|3|AM|third|night|| +10504|AAAAAAAAJAJCAAAA|10504|2|55|4|AM|third|night|| +10505|AAAAAAAAKAJCAAAA|10505|2|55|5|AM|third|night|| +10506|AAAAAAAALAJCAAAA|10506|2|55|6|AM|third|night|| +10507|AAAAAAAAMAJCAAAA|10507|2|55|7|AM|third|night|| +10508|AAAAAAAANAJCAAAA|10508|2|55|8|AM|third|night|| +10509|AAAAAAAAOAJCAAAA|10509|2|55|9|AM|third|night|| +10510|AAAAAAAAPAJCAAAA|10510|2|55|10|AM|third|night|| +10511|AAAAAAAAABJCAAAA|10511|2|55|11|AM|third|night|| +10512|AAAAAAAABBJCAAAA|10512|2|55|12|AM|third|night|| +10513|AAAAAAAACBJCAAAA|10513|2|55|13|AM|third|night|| +10514|AAAAAAAADBJCAAAA|10514|2|55|14|AM|third|night|| +10515|AAAAAAAAEBJCAAAA|10515|2|55|15|AM|third|night|| +10516|AAAAAAAAFBJCAAAA|10516|2|55|16|AM|third|night|| +10517|AAAAAAAAGBJCAAAA|10517|2|55|17|AM|third|night|| +10518|AAAAAAAAHBJCAAAA|10518|2|55|18|AM|third|night|| +10519|AAAAAAAAIBJCAAAA|10519|2|55|19|AM|third|night|| +10520|AAAAAAAAJBJCAAAA|10520|2|55|20|AM|third|night|| +10521|AAAAAAAAKBJCAAAA|10521|2|55|21|AM|third|night|| +10522|AAAAAAAALBJCAAAA|10522|2|55|22|AM|third|night|| +10523|AAAAAAAAMBJCAAAA|10523|2|55|23|AM|third|night|| +10524|AAAAAAAANBJCAAAA|10524|2|55|24|AM|third|night|| +10525|AAAAAAAAOBJCAAAA|10525|2|55|25|AM|third|night|| +10526|AAAAAAAAPBJCAAAA|10526|2|55|26|AM|third|night|| +10527|AAAAAAAAACJCAAAA|10527|2|55|27|AM|third|night|| +10528|AAAAAAAABCJCAAAA|10528|2|55|28|AM|third|night|| +10529|AAAAAAAACCJCAAAA|10529|2|55|29|AM|third|night|| +10530|AAAAAAAADCJCAAAA|10530|2|55|30|AM|third|night|| +10531|AAAAAAAAECJCAAAA|10531|2|55|31|AM|third|night|| +10532|AAAAAAAAFCJCAAAA|10532|2|55|32|AM|third|night|| +10533|AAAAAAAAGCJCAAAA|10533|2|55|33|AM|third|night|| +10534|AAAAAAAAHCJCAAAA|10534|2|55|34|AM|third|night|| +10535|AAAAAAAAICJCAAAA|10535|2|55|35|AM|third|night|| +10536|AAAAAAAAJCJCAAAA|10536|2|55|36|AM|third|night|| +10537|AAAAAAAAKCJCAAAA|10537|2|55|37|AM|third|night|| +10538|AAAAAAAALCJCAAAA|10538|2|55|38|AM|third|night|| +10539|AAAAAAAAMCJCAAAA|10539|2|55|39|AM|third|night|| +10540|AAAAAAAANCJCAAAA|10540|2|55|40|AM|third|night|| +10541|AAAAAAAAOCJCAAAA|10541|2|55|41|AM|third|night|| +10542|AAAAAAAAPCJCAAAA|10542|2|55|42|AM|third|night|| +10543|AAAAAAAAADJCAAAA|10543|2|55|43|AM|third|night|| +10544|AAAAAAAABDJCAAAA|10544|2|55|44|AM|third|night|| +10545|AAAAAAAACDJCAAAA|10545|2|55|45|AM|third|night|| +10546|AAAAAAAADDJCAAAA|10546|2|55|46|AM|third|night|| +10547|AAAAAAAAEDJCAAAA|10547|2|55|47|AM|third|night|| +10548|AAAAAAAAFDJCAAAA|10548|2|55|48|AM|third|night|| +10549|AAAAAAAAGDJCAAAA|10549|2|55|49|AM|third|night|| +10550|AAAAAAAAHDJCAAAA|10550|2|55|50|AM|third|night|| +10551|AAAAAAAAIDJCAAAA|10551|2|55|51|AM|third|night|| +10552|AAAAAAAAJDJCAAAA|10552|2|55|52|AM|third|night|| +10553|AAAAAAAAKDJCAAAA|10553|2|55|53|AM|third|night|| +10554|AAAAAAAALDJCAAAA|10554|2|55|54|AM|third|night|| +10555|AAAAAAAAMDJCAAAA|10555|2|55|55|AM|third|night|| +10556|AAAAAAAANDJCAAAA|10556|2|55|56|AM|third|night|| +10557|AAAAAAAAODJCAAAA|10557|2|55|57|AM|third|night|| +10558|AAAAAAAAPDJCAAAA|10558|2|55|58|AM|third|night|| +10559|AAAAAAAAAEJCAAAA|10559|2|55|59|AM|third|night|| +10560|AAAAAAAABEJCAAAA|10560|2|56|0|AM|third|night|| +10561|AAAAAAAACEJCAAAA|10561|2|56|1|AM|third|night|| +10562|AAAAAAAADEJCAAAA|10562|2|56|2|AM|third|night|| +10563|AAAAAAAAEEJCAAAA|10563|2|56|3|AM|third|night|| +10564|AAAAAAAAFEJCAAAA|10564|2|56|4|AM|third|night|| +10565|AAAAAAAAGEJCAAAA|10565|2|56|5|AM|third|night|| +10566|AAAAAAAAHEJCAAAA|10566|2|56|6|AM|third|night|| +10567|AAAAAAAAIEJCAAAA|10567|2|56|7|AM|third|night|| +10568|AAAAAAAAJEJCAAAA|10568|2|56|8|AM|third|night|| +10569|AAAAAAAAKEJCAAAA|10569|2|56|9|AM|third|night|| +10570|AAAAAAAALEJCAAAA|10570|2|56|10|AM|third|night|| +10571|AAAAAAAAMEJCAAAA|10571|2|56|11|AM|third|night|| +10572|AAAAAAAANEJCAAAA|10572|2|56|12|AM|third|night|| +10573|AAAAAAAAOEJCAAAA|10573|2|56|13|AM|third|night|| +10574|AAAAAAAAPEJCAAAA|10574|2|56|14|AM|third|night|| +10575|AAAAAAAAAFJCAAAA|10575|2|56|15|AM|third|night|| +10576|AAAAAAAABFJCAAAA|10576|2|56|16|AM|third|night|| +10577|AAAAAAAACFJCAAAA|10577|2|56|17|AM|third|night|| +10578|AAAAAAAADFJCAAAA|10578|2|56|18|AM|third|night|| +10579|AAAAAAAAEFJCAAAA|10579|2|56|19|AM|third|night|| +10580|AAAAAAAAFFJCAAAA|10580|2|56|20|AM|third|night|| +10581|AAAAAAAAGFJCAAAA|10581|2|56|21|AM|third|night|| +10582|AAAAAAAAHFJCAAAA|10582|2|56|22|AM|third|night|| +10583|AAAAAAAAIFJCAAAA|10583|2|56|23|AM|third|night|| +10584|AAAAAAAAJFJCAAAA|10584|2|56|24|AM|third|night|| +10585|AAAAAAAAKFJCAAAA|10585|2|56|25|AM|third|night|| +10586|AAAAAAAALFJCAAAA|10586|2|56|26|AM|third|night|| +10587|AAAAAAAAMFJCAAAA|10587|2|56|27|AM|third|night|| +10588|AAAAAAAANFJCAAAA|10588|2|56|28|AM|third|night|| +10589|AAAAAAAAOFJCAAAA|10589|2|56|29|AM|third|night|| +10590|AAAAAAAAPFJCAAAA|10590|2|56|30|AM|third|night|| +10591|AAAAAAAAAGJCAAAA|10591|2|56|31|AM|third|night|| +10592|AAAAAAAABGJCAAAA|10592|2|56|32|AM|third|night|| +10593|AAAAAAAACGJCAAAA|10593|2|56|33|AM|third|night|| +10594|AAAAAAAADGJCAAAA|10594|2|56|34|AM|third|night|| +10595|AAAAAAAAEGJCAAAA|10595|2|56|35|AM|third|night|| +10596|AAAAAAAAFGJCAAAA|10596|2|56|36|AM|third|night|| +10597|AAAAAAAAGGJCAAAA|10597|2|56|37|AM|third|night|| +10598|AAAAAAAAHGJCAAAA|10598|2|56|38|AM|third|night|| +10599|AAAAAAAAIGJCAAAA|10599|2|56|39|AM|third|night|| +10600|AAAAAAAAJGJCAAAA|10600|2|56|40|AM|third|night|| +10601|AAAAAAAAKGJCAAAA|10601|2|56|41|AM|third|night|| +10602|AAAAAAAALGJCAAAA|10602|2|56|42|AM|third|night|| +10603|AAAAAAAAMGJCAAAA|10603|2|56|43|AM|third|night|| +10604|AAAAAAAANGJCAAAA|10604|2|56|44|AM|third|night|| +10605|AAAAAAAAOGJCAAAA|10605|2|56|45|AM|third|night|| +10606|AAAAAAAAPGJCAAAA|10606|2|56|46|AM|third|night|| +10607|AAAAAAAAAHJCAAAA|10607|2|56|47|AM|third|night|| +10608|AAAAAAAABHJCAAAA|10608|2|56|48|AM|third|night|| +10609|AAAAAAAACHJCAAAA|10609|2|56|49|AM|third|night|| +10610|AAAAAAAADHJCAAAA|10610|2|56|50|AM|third|night|| +10611|AAAAAAAAEHJCAAAA|10611|2|56|51|AM|third|night|| +10612|AAAAAAAAFHJCAAAA|10612|2|56|52|AM|third|night|| +10613|AAAAAAAAGHJCAAAA|10613|2|56|53|AM|third|night|| +10614|AAAAAAAAHHJCAAAA|10614|2|56|54|AM|third|night|| +10615|AAAAAAAAIHJCAAAA|10615|2|56|55|AM|third|night|| +10616|AAAAAAAAJHJCAAAA|10616|2|56|56|AM|third|night|| +10617|AAAAAAAAKHJCAAAA|10617|2|56|57|AM|third|night|| +10618|AAAAAAAALHJCAAAA|10618|2|56|58|AM|third|night|| +10619|AAAAAAAAMHJCAAAA|10619|2|56|59|AM|third|night|| +10620|AAAAAAAANHJCAAAA|10620|2|57|0|AM|third|night|| +10621|AAAAAAAAOHJCAAAA|10621|2|57|1|AM|third|night|| +10622|AAAAAAAAPHJCAAAA|10622|2|57|2|AM|third|night|| +10623|AAAAAAAAAIJCAAAA|10623|2|57|3|AM|third|night|| +10624|AAAAAAAABIJCAAAA|10624|2|57|4|AM|third|night|| +10625|AAAAAAAACIJCAAAA|10625|2|57|5|AM|third|night|| +10626|AAAAAAAADIJCAAAA|10626|2|57|6|AM|third|night|| +10627|AAAAAAAAEIJCAAAA|10627|2|57|7|AM|third|night|| +10628|AAAAAAAAFIJCAAAA|10628|2|57|8|AM|third|night|| +10629|AAAAAAAAGIJCAAAA|10629|2|57|9|AM|third|night|| +10630|AAAAAAAAHIJCAAAA|10630|2|57|10|AM|third|night|| +10631|AAAAAAAAIIJCAAAA|10631|2|57|11|AM|third|night|| +10632|AAAAAAAAJIJCAAAA|10632|2|57|12|AM|third|night|| +10633|AAAAAAAAKIJCAAAA|10633|2|57|13|AM|third|night|| +10634|AAAAAAAALIJCAAAA|10634|2|57|14|AM|third|night|| +10635|AAAAAAAAMIJCAAAA|10635|2|57|15|AM|third|night|| +10636|AAAAAAAANIJCAAAA|10636|2|57|16|AM|third|night|| +10637|AAAAAAAAOIJCAAAA|10637|2|57|17|AM|third|night|| +10638|AAAAAAAAPIJCAAAA|10638|2|57|18|AM|third|night|| +10639|AAAAAAAAAJJCAAAA|10639|2|57|19|AM|third|night|| +10640|AAAAAAAABJJCAAAA|10640|2|57|20|AM|third|night|| +10641|AAAAAAAACJJCAAAA|10641|2|57|21|AM|third|night|| +10642|AAAAAAAADJJCAAAA|10642|2|57|22|AM|third|night|| +10643|AAAAAAAAEJJCAAAA|10643|2|57|23|AM|third|night|| +10644|AAAAAAAAFJJCAAAA|10644|2|57|24|AM|third|night|| +10645|AAAAAAAAGJJCAAAA|10645|2|57|25|AM|third|night|| +10646|AAAAAAAAHJJCAAAA|10646|2|57|26|AM|third|night|| +10647|AAAAAAAAIJJCAAAA|10647|2|57|27|AM|third|night|| +10648|AAAAAAAAJJJCAAAA|10648|2|57|28|AM|third|night|| +10649|AAAAAAAAKJJCAAAA|10649|2|57|29|AM|third|night|| +10650|AAAAAAAALJJCAAAA|10650|2|57|30|AM|third|night|| +10651|AAAAAAAAMJJCAAAA|10651|2|57|31|AM|third|night|| +10652|AAAAAAAANJJCAAAA|10652|2|57|32|AM|third|night|| +10653|AAAAAAAAOJJCAAAA|10653|2|57|33|AM|third|night|| +10654|AAAAAAAAPJJCAAAA|10654|2|57|34|AM|third|night|| +10655|AAAAAAAAAKJCAAAA|10655|2|57|35|AM|third|night|| +10656|AAAAAAAABKJCAAAA|10656|2|57|36|AM|third|night|| +10657|AAAAAAAACKJCAAAA|10657|2|57|37|AM|third|night|| +10658|AAAAAAAADKJCAAAA|10658|2|57|38|AM|third|night|| +10659|AAAAAAAAEKJCAAAA|10659|2|57|39|AM|third|night|| +10660|AAAAAAAAFKJCAAAA|10660|2|57|40|AM|third|night|| +10661|AAAAAAAAGKJCAAAA|10661|2|57|41|AM|third|night|| +10662|AAAAAAAAHKJCAAAA|10662|2|57|42|AM|third|night|| +10663|AAAAAAAAIKJCAAAA|10663|2|57|43|AM|third|night|| +10664|AAAAAAAAJKJCAAAA|10664|2|57|44|AM|third|night|| +10665|AAAAAAAAKKJCAAAA|10665|2|57|45|AM|third|night|| +10666|AAAAAAAALKJCAAAA|10666|2|57|46|AM|third|night|| +10667|AAAAAAAAMKJCAAAA|10667|2|57|47|AM|third|night|| +10668|AAAAAAAANKJCAAAA|10668|2|57|48|AM|third|night|| +10669|AAAAAAAAOKJCAAAA|10669|2|57|49|AM|third|night|| +10670|AAAAAAAAPKJCAAAA|10670|2|57|50|AM|third|night|| +10671|AAAAAAAAALJCAAAA|10671|2|57|51|AM|third|night|| +10672|AAAAAAAABLJCAAAA|10672|2|57|52|AM|third|night|| +10673|AAAAAAAACLJCAAAA|10673|2|57|53|AM|third|night|| +10674|AAAAAAAADLJCAAAA|10674|2|57|54|AM|third|night|| +10675|AAAAAAAAELJCAAAA|10675|2|57|55|AM|third|night|| +10676|AAAAAAAAFLJCAAAA|10676|2|57|56|AM|third|night|| +10677|AAAAAAAAGLJCAAAA|10677|2|57|57|AM|third|night|| +10678|AAAAAAAAHLJCAAAA|10678|2|57|58|AM|third|night|| +10679|AAAAAAAAILJCAAAA|10679|2|57|59|AM|third|night|| +10680|AAAAAAAAJLJCAAAA|10680|2|58|0|AM|third|night|| +10681|AAAAAAAAKLJCAAAA|10681|2|58|1|AM|third|night|| +10682|AAAAAAAALLJCAAAA|10682|2|58|2|AM|third|night|| +10683|AAAAAAAAMLJCAAAA|10683|2|58|3|AM|third|night|| +10684|AAAAAAAANLJCAAAA|10684|2|58|4|AM|third|night|| +10685|AAAAAAAAOLJCAAAA|10685|2|58|5|AM|third|night|| +10686|AAAAAAAAPLJCAAAA|10686|2|58|6|AM|third|night|| +10687|AAAAAAAAAMJCAAAA|10687|2|58|7|AM|third|night|| +10688|AAAAAAAABMJCAAAA|10688|2|58|8|AM|third|night|| +10689|AAAAAAAACMJCAAAA|10689|2|58|9|AM|third|night|| +10690|AAAAAAAADMJCAAAA|10690|2|58|10|AM|third|night|| +10691|AAAAAAAAEMJCAAAA|10691|2|58|11|AM|third|night|| +10692|AAAAAAAAFMJCAAAA|10692|2|58|12|AM|third|night|| +10693|AAAAAAAAGMJCAAAA|10693|2|58|13|AM|third|night|| +10694|AAAAAAAAHMJCAAAA|10694|2|58|14|AM|third|night|| +10695|AAAAAAAAIMJCAAAA|10695|2|58|15|AM|third|night|| +10696|AAAAAAAAJMJCAAAA|10696|2|58|16|AM|third|night|| +10697|AAAAAAAAKMJCAAAA|10697|2|58|17|AM|third|night|| +10698|AAAAAAAALMJCAAAA|10698|2|58|18|AM|third|night|| +10699|AAAAAAAAMMJCAAAA|10699|2|58|19|AM|third|night|| +10700|AAAAAAAANMJCAAAA|10700|2|58|20|AM|third|night|| +10701|AAAAAAAAOMJCAAAA|10701|2|58|21|AM|third|night|| +10702|AAAAAAAAPMJCAAAA|10702|2|58|22|AM|third|night|| +10703|AAAAAAAAANJCAAAA|10703|2|58|23|AM|third|night|| +10704|AAAAAAAABNJCAAAA|10704|2|58|24|AM|third|night|| +10705|AAAAAAAACNJCAAAA|10705|2|58|25|AM|third|night|| +10706|AAAAAAAADNJCAAAA|10706|2|58|26|AM|third|night|| +10707|AAAAAAAAENJCAAAA|10707|2|58|27|AM|third|night|| +10708|AAAAAAAAFNJCAAAA|10708|2|58|28|AM|third|night|| +10709|AAAAAAAAGNJCAAAA|10709|2|58|29|AM|third|night|| +10710|AAAAAAAAHNJCAAAA|10710|2|58|30|AM|third|night|| +10711|AAAAAAAAINJCAAAA|10711|2|58|31|AM|third|night|| +10712|AAAAAAAAJNJCAAAA|10712|2|58|32|AM|third|night|| +10713|AAAAAAAAKNJCAAAA|10713|2|58|33|AM|third|night|| +10714|AAAAAAAALNJCAAAA|10714|2|58|34|AM|third|night|| +10715|AAAAAAAAMNJCAAAA|10715|2|58|35|AM|third|night|| +10716|AAAAAAAANNJCAAAA|10716|2|58|36|AM|third|night|| +10717|AAAAAAAAONJCAAAA|10717|2|58|37|AM|third|night|| +10718|AAAAAAAAPNJCAAAA|10718|2|58|38|AM|third|night|| +10719|AAAAAAAAAOJCAAAA|10719|2|58|39|AM|third|night|| +10720|AAAAAAAABOJCAAAA|10720|2|58|40|AM|third|night|| +10721|AAAAAAAACOJCAAAA|10721|2|58|41|AM|third|night|| +10722|AAAAAAAADOJCAAAA|10722|2|58|42|AM|third|night|| +10723|AAAAAAAAEOJCAAAA|10723|2|58|43|AM|third|night|| +10724|AAAAAAAAFOJCAAAA|10724|2|58|44|AM|third|night|| +10725|AAAAAAAAGOJCAAAA|10725|2|58|45|AM|third|night|| +10726|AAAAAAAAHOJCAAAA|10726|2|58|46|AM|third|night|| +10727|AAAAAAAAIOJCAAAA|10727|2|58|47|AM|third|night|| +10728|AAAAAAAAJOJCAAAA|10728|2|58|48|AM|third|night|| +10729|AAAAAAAAKOJCAAAA|10729|2|58|49|AM|third|night|| +10730|AAAAAAAALOJCAAAA|10730|2|58|50|AM|third|night|| +10731|AAAAAAAAMOJCAAAA|10731|2|58|51|AM|third|night|| +10732|AAAAAAAANOJCAAAA|10732|2|58|52|AM|third|night|| +10733|AAAAAAAAOOJCAAAA|10733|2|58|53|AM|third|night|| +10734|AAAAAAAAPOJCAAAA|10734|2|58|54|AM|third|night|| +10735|AAAAAAAAAPJCAAAA|10735|2|58|55|AM|third|night|| +10736|AAAAAAAABPJCAAAA|10736|2|58|56|AM|third|night|| +10737|AAAAAAAACPJCAAAA|10737|2|58|57|AM|third|night|| +10738|AAAAAAAADPJCAAAA|10738|2|58|58|AM|third|night|| +10739|AAAAAAAAEPJCAAAA|10739|2|58|59|AM|third|night|| +10740|AAAAAAAAFPJCAAAA|10740|2|59|0|AM|third|night|| +10741|AAAAAAAAGPJCAAAA|10741|2|59|1|AM|third|night|| +10742|AAAAAAAAHPJCAAAA|10742|2|59|2|AM|third|night|| +10743|AAAAAAAAIPJCAAAA|10743|2|59|3|AM|third|night|| +10744|AAAAAAAAJPJCAAAA|10744|2|59|4|AM|third|night|| +10745|AAAAAAAAKPJCAAAA|10745|2|59|5|AM|third|night|| +10746|AAAAAAAALPJCAAAA|10746|2|59|6|AM|third|night|| +10747|AAAAAAAAMPJCAAAA|10747|2|59|7|AM|third|night|| +10748|AAAAAAAANPJCAAAA|10748|2|59|8|AM|third|night|| +10749|AAAAAAAAOPJCAAAA|10749|2|59|9|AM|third|night|| +10750|AAAAAAAAPPJCAAAA|10750|2|59|10|AM|third|night|| +10751|AAAAAAAAAAKCAAAA|10751|2|59|11|AM|third|night|| +10752|AAAAAAAABAKCAAAA|10752|2|59|12|AM|third|night|| +10753|AAAAAAAACAKCAAAA|10753|2|59|13|AM|third|night|| +10754|AAAAAAAADAKCAAAA|10754|2|59|14|AM|third|night|| +10755|AAAAAAAAEAKCAAAA|10755|2|59|15|AM|third|night|| +10756|AAAAAAAAFAKCAAAA|10756|2|59|16|AM|third|night|| +10757|AAAAAAAAGAKCAAAA|10757|2|59|17|AM|third|night|| +10758|AAAAAAAAHAKCAAAA|10758|2|59|18|AM|third|night|| +10759|AAAAAAAAIAKCAAAA|10759|2|59|19|AM|third|night|| +10760|AAAAAAAAJAKCAAAA|10760|2|59|20|AM|third|night|| +10761|AAAAAAAAKAKCAAAA|10761|2|59|21|AM|third|night|| +10762|AAAAAAAALAKCAAAA|10762|2|59|22|AM|third|night|| +10763|AAAAAAAAMAKCAAAA|10763|2|59|23|AM|third|night|| +10764|AAAAAAAANAKCAAAA|10764|2|59|24|AM|third|night|| +10765|AAAAAAAAOAKCAAAA|10765|2|59|25|AM|third|night|| +10766|AAAAAAAAPAKCAAAA|10766|2|59|26|AM|third|night|| +10767|AAAAAAAAABKCAAAA|10767|2|59|27|AM|third|night|| +10768|AAAAAAAABBKCAAAA|10768|2|59|28|AM|third|night|| +10769|AAAAAAAACBKCAAAA|10769|2|59|29|AM|third|night|| +10770|AAAAAAAADBKCAAAA|10770|2|59|30|AM|third|night|| +10771|AAAAAAAAEBKCAAAA|10771|2|59|31|AM|third|night|| +10772|AAAAAAAAFBKCAAAA|10772|2|59|32|AM|third|night|| +10773|AAAAAAAAGBKCAAAA|10773|2|59|33|AM|third|night|| +10774|AAAAAAAAHBKCAAAA|10774|2|59|34|AM|third|night|| +10775|AAAAAAAAIBKCAAAA|10775|2|59|35|AM|third|night|| +10776|AAAAAAAAJBKCAAAA|10776|2|59|36|AM|third|night|| +10777|AAAAAAAAKBKCAAAA|10777|2|59|37|AM|third|night|| +10778|AAAAAAAALBKCAAAA|10778|2|59|38|AM|third|night|| +10779|AAAAAAAAMBKCAAAA|10779|2|59|39|AM|third|night|| +10780|AAAAAAAANBKCAAAA|10780|2|59|40|AM|third|night|| +10781|AAAAAAAAOBKCAAAA|10781|2|59|41|AM|third|night|| +10782|AAAAAAAAPBKCAAAA|10782|2|59|42|AM|third|night|| +10783|AAAAAAAAACKCAAAA|10783|2|59|43|AM|third|night|| +10784|AAAAAAAABCKCAAAA|10784|2|59|44|AM|third|night|| +10785|AAAAAAAACCKCAAAA|10785|2|59|45|AM|third|night|| +10786|AAAAAAAADCKCAAAA|10786|2|59|46|AM|third|night|| +10787|AAAAAAAAECKCAAAA|10787|2|59|47|AM|third|night|| +10788|AAAAAAAAFCKCAAAA|10788|2|59|48|AM|third|night|| +10789|AAAAAAAAGCKCAAAA|10789|2|59|49|AM|third|night|| +10790|AAAAAAAAHCKCAAAA|10790|2|59|50|AM|third|night|| +10791|AAAAAAAAICKCAAAA|10791|2|59|51|AM|third|night|| +10792|AAAAAAAAJCKCAAAA|10792|2|59|52|AM|third|night|| +10793|AAAAAAAAKCKCAAAA|10793|2|59|53|AM|third|night|| +10794|AAAAAAAALCKCAAAA|10794|2|59|54|AM|third|night|| +10795|AAAAAAAAMCKCAAAA|10795|2|59|55|AM|third|night|| +10796|AAAAAAAANCKCAAAA|10796|2|59|56|AM|third|night|| +10797|AAAAAAAAOCKCAAAA|10797|2|59|57|AM|third|night|| +10798|AAAAAAAAPCKCAAAA|10798|2|59|58|AM|third|night|| +10799|AAAAAAAAADKCAAAA|10799|2|59|59|AM|third|night|| +10800|AAAAAAAABDKCAAAA|10800|3|0|0|AM|third|night|| +10801|AAAAAAAACDKCAAAA|10801|3|0|1|AM|third|night|| +10802|AAAAAAAADDKCAAAA|10802|3|0|2|AM|third|night|| +10803|AAAAAAAAEDKCAAAA|10803|3|0|3|AM|third|night|| +10804|AAAAAAAAFDKCAAAA|10804|3|0|4|AM|third|night|| +10805|AAAAAAAAGDKCAAAA|10805|3|0|5|AM|third|night|| +10806|AAAAAAAAHDKCAAAA|10806|3|0|6|AM|third|night|| +10807|AAAAAAAAIDKCAAAA|10807|3|0|7|AM|third|night|| +10808|AAAAAAAAJDKCAAAA|10808|3|0|8|AM|third|night|| +10809|AAAAAAAAKDKCAAAA|10809|3|0|9|AM|third|night|| +10810|AAAAAAAALDKCAAAA|10810|3|0|10|AM|third|night|| +10811|AAAAAAAAMDKCAAAA|10811|3|0|11|AM|third|night|| +10812|AAAAAAAANDKCAAAA|10812|3|0|12|AM|third|night|| +10813|AAAAAAAAODKCAAAA|10813|3|0|13|AM|third|night|| +10814|AAAAAAAAPDKCAAAA|10814|3|0|14|AM|third|night|| +10815|AAAAAAAAAEKCAAAA|10815|3|0|15|AM|third|night|| +10816|AAAAAAAABEKCAAAA|10816|3|0|16|AM|third|night|| +10817|AAAAAAAACEKCAAAA|10817|3|0|17|AM|third|night|| +10818|AAAAAAAADEKCAAAA|10818|3|0|18|AM|third|night|| +10819|AAAAAAAAEEKCAAAA|10819|3|0|19|AM|third|night|| +10820|AAAAAAAAFEKCAAAA|10820|3|0|20|AM|third|night|| +10821|AAAAAAAAGEKCAAAA|10821|3|0|21|AM|third|night|| +10822|AAAAAAAAHEKCAAAA|10822|3|0|22|AM|third|night|| +10823|AAAAAAAAIEKCAAAA|10823|3|0|23|AM|third|night|| +10824|AAAAAAAAJEKCAAAA|10824|3|0|24|AM|third|night|| +10825|AAAAAAAAKEKCAAAA|10825|3|0|25|AM|third|night|| +10826|AAAAAAAALEKCAAAA|10826|3|0|26|AM|third|night|| +10827|AAAAAAAAMEKCAAAA|10827|3|0|27|AM|third|night|| +10828|AAAAAAAANEKCAAAA|10828|3|0|28|AM|third|night|| +10829|AAAAAAAAOEKCAAAA|10829|3|0|29|AM|third|night|| +10830|AAAAAAAAPEKCAAAA|10830|3|0|30|AM|third|night|| +10831|AAAAAAAAAFKCAAAA|10831|3|0|31|AM|third|night|| +10832|AAAAAAAABFKCAAAA|10832|3|0|32|AM|third|night|| +10833|AAAAAAAACFKCAAAA|10833|3|0|33|AM|third|night|| +10834|AAAAAAAADFKCAAAA|10834|3|0|34|AM|third|night|| +10835|AAAAAAAAEFKCAAAA|10835|3|0|35|AM|third|night|| +10836|AAAAAAAAFFKCAAAA|10836|3|0|36|AM|third|night|| +10837|AAAAAAAAGFKCAAAA|10837|3|0|37|AM|third|night|| +10838|AAAAAAAAHFKCAAAA|10838|3|0|38|AM|third|night|| +10839|AAAAAAAAIFKCAAAA|10839|3|0|39|AM|third|night|| +10840|AAAAAAAAJFKCAAAA|10840|3|0|40|AM|third|night|| +10841|AAAAAAAAKFKCAAAA|10841|3|0|41|AM|third|night|| +10842|AAAAAAAALFKCAAAA|10842|3|0|42|AM|third|night|| +10843|AAAAAAAAMFKCAAAA|10843|3|0|43|AM|third|night|| +10844|AAAAAAAANFKCAAAA|10844|3|0|44|AM|third|night|| +10845|AAAAAAAAOFKCAAAA|10845|3|0|45|AM|third|night|| +10846|AAAAAAAAPFKCAAAA|10846|3|0|46|AM|third|night|| +10847|AAAAAAAAAGKCAAAA|10847|3|0|47|AM|third|night|| +10848|AAAAAAAABGKCAAAA|10848|3|0|48|AM|third|night|| +10849|AAAAAAAACGKCAAAA|10849|3|0|49|AM|third|night|| +10850|AAAAAAAADGKCAAAA|10850|3|0|50|AM|third|night|| +10851|AAAAAAAAEGKCAAAA|10851|3|0|51|AM|third|night|| +10852|AAAAAAAAFGKCAAAA|10852|3|0|52|AM|third|night|| +10853|AAAAAAAAGGKCAAAA|10853|3|0|53|AM|third|night|| +10854|AAAAAAAAHGKCAAAA|10854|3|0|54|AM|third|night|| +10855|AAAAAAAAIGKCAAAA|10855|3|0|55|AM|third|night|| +10856|AAAAAAAAJGKCAAAA|10856|3|0|56|AM|third|night|| +10857|AAAAAAAAKGKCAAAA|10857|3|0|57|AM|third|night|| +10858|AAAAAAAALGKCAAAA|10858|3|0|58|AM|third|night|| +10859|AAAAAAAAMGKCAAAA|10859|3|0|59|AM|third|night|| +10860|AAAAAAAANGKCAAAA|10860|3|1|0|AM|third|night|| +10861|AAAAAAAAOGKCAAAA|10861|3|1|1|AM|third|night|| +10862|AAAAAAAAPGKCAAAA|10862|3|1|2|AM|third|night|| +10863|AAAAAAAAAHKCAAAA|10863|3|1|3|AM|third|night|| +10864|AAAAAAAABHKCAAAA|10864|3|1|4|AM|third|night|| +10865|AAAAAAAACHKCAAAA|10865|3|1|5|AM|third|night|| +10866|AAAAAAAADHKCAAAA|10866|3|1|6|AM|third|night|| +10867|AAAAAAAAEHKCAAAA|10867|3|1|7|AM|third|night|| +10868|AAAAAAAAFHKCAAAA|10868|3|1|8|AM|third|night|| +10869|AAAAAAAAGHKCAAAA|10869|3|1|9|AM|third|night|| +10870|AAAAAAAAHHKCAAAA|10870|3|1|10|AM|third|night|| +10871|AAAAAAAAIHKCAAAA|10871|3|1|11|AM|third|night|| +10872|AAAAAAAAJHKCAAAA|10872|3|1|12|AM|third|night|| +10873|AAAAAAAAKHKCAAAA|10873|3|1|13|AM|third|night|| +10874|AAAAAAAALHKCAAAA|10874|3|1|14|AM|third|night|| +10875|AAAAAAAAMHKCAAAA|10875|3|1|15|AM|third|night|| +10876|AAAAAAAANHKCAAAA|10876|3|1|16|AM|third|night|| +10877|AAAAAAAAOHKCAAAA|10877|3|1|17|AM|third|night|| +10878|AAAAAAAAPHKCAAAA|10878|3|1|18|AM|third|night|| +10879|AAAAAAAAAIKCAAAA|10879|3|1|19|AM|third|night|| +10880|AAAAAAAABIKCAAAA|10880|3|1|20|AM|third|night|| +10881|AAAAAAAACIKCAAAA|10881|3|1|21|AM|third|night|| +10882|AAAAAAAADIKCAAAA|10882|3|1|22|AM|third|night|| +10883|AAAAAAAAEIKCAAAA|10883|3|1|23|AM|third|night|| +10884|AAAAAAAAFIKCAAAA|10884|3|1|24|AM|third|night|| +10885|AAAAAAAAGIKCAAAA|10885|3|1|25|AM|third|night|| +10886|AAAAAAAAHIKCAAAA|10886|3|1|26|AM|third|night|| +10887|AAAAAAAAIIKCAAAA|10887|3|1|27|AM|third|night|| +10888|AAAAAAAAJIKCAAAA|10888|3|1|28|AM|third|night|| +10889|AAAAAAAAKIKCAAAA|10889|3|1|29|AM|third|night|| +10890|AAAAAAAALIKCAAAA|10890|3|1|30|AM|third|night|| +10891|AAAAAAAAMIKCAAAA|10891|3|1|31|AM|third|night|| +10892|AAAAAAAANIKCAAAA|10892|3|1|32|AM|third|night|| +10893|AAAAAAAAOIKCAAAA|10893|3|1|33|AM|third|night|| +10894|AAAAAAAAPIKCAAAA|10894|3|1|34|AM|third|night|| +10895|AAAAAAAAAJKCAAAA|10895|3|1|35|AM|third|night|| +10896|AAAAAAAABJKCAAAA|10896|3|1|36|AM|third|night|| +10897|AAAAAAAACJKCAAAA|10897|3|1|37|AM|third|night|| +10898|AAAAAAAADJKCAAAA|10898|3|1|38|AM|third|night|| +10899|AAAAAAAAEJKCAAAA|10899|3|1|39|AM|third|night|| +10900|AAAAAAAAFJKCAAAA|10900|3|1|40|AM|third|night|| +10901|AAAAAAAAGJKCAAAA|10901|3|1|41|AM|third|night|| +10902|AAAAAAAAHJKCAAAA|10902|3|1|42|AM|third|night|| +10903|AAAAAAAAIJKCAAAA|10903|3|1|43|AM|third|night|| +10904|AAAAAAAAJJKCAAAA|10904|3|1|44|AM|third|night|| +10905|AAAAAAAAKJKCAAAA|10905|3|1|45|AM|third|night|| +10906|AAAAAAAALJKCAAAA|10906|3|1|46|AM|third|night|| +10907|AAAAAAAAMJKCAAAA|10907|3|1|47|AM|third|night|| +10908|AAAAAAAANJKCAAAA|10908|3|1|48|AM|third|night|| +10909|AAAAAAAAOJKCAAAA|10909|3|1|49|AM|third|night|| +10910|AAAAAAAAPJKCAAAA|10910|3|1|50|AM|third|night|| +10911|AAAAAAAAAKKCAAAA|10911|3|1|51|AM|third|night|| +10912|AAAAAAAABKKCAAAA|10912|3|1|52|AM|third|night|| +10913|AAAAAAAACKKCAAAA|10913|3|1|53|AM|third|night|| +10914|AAAAAAAADKKCAAAA|10914|3|1|54|AM|third|night|| +10915|AAAAAAAAEKKCAAAA|10915|3|1|55|AM|third|night|| +10916|AAAAAAAAFKKCAAAA|10916|3|1|56|AM|third|night|| +10917|AAAAAAAAGKKCAAAA|10917|3|1|57|AM|third|night|| +10918|AAAAAAAAHKKCAAAA|10918|3|1|58|AM|third|night|| +10919|AAAAAAAAIKKCAAAA|10919|3|1|59|AM|third|night|| +10920|AAAAAAAAJKKCAAAA|10920|3|2|0|AM|third|night|| +10921|AAAAAAAAKKKCAAAA|10921|3|2|1|AM|third|night|| +10922|AAAAAAAALKKCAAAA|10922|3|2|2|AM|third|night|| +10923|AAAAAAAAMKKCAAAA|10923|3|2|3|AM|third|night|| +10924|AAAAAAAANKKCAAAA|10924|3|2|4|AM|third|night|| +10925|AAAAAAAAOKKCAAAA|10925|3|2|5|AM|third|night|| +10926|AAAAAAAAPKKCAAAA|10926|3|2|6|AM|third|night|| +10927|AAAAAAAAALKCAAAA|10927|3|2|7|AM|third|night|| +10928|AAAAAAAABLKCAAAA|10928|3|2|8|AM|third|night|| +10929|AAAAAAAACLKCAAAA|10929|3|2|9|AM|third|night|| +10930|AAAAAAAADLKCAAAA|10930|3|2|10|AM|third|night|| +10931|AAAAAAAAELKCAAAA|10931|3|2|11|AM|third|night|| +10932|AAAAAAAAFLKCAAAA|10932|3|2|12|AM|third|night|| +10933|AAAAAAAAGLKCAAAA|10933|3|2|13|AM|third|night|| +10934|AAAAAAAAHLKCAAAA|10934|3|2|14|AM|third|night|| +10935|AAAAAAAAILKCAAAA|10935|3|2|15|AM|third|night|| +10936|AAAAAAAAJLKCAAAA|10936|3|2|16|AM|third|night|| +10937|AAAAAAAAKLKCAAAA|10937|3|2|17|AM|third|night|| +10938|AAAAAAAALLKCAAAA|10938|3|2|18|AM|third|night|| +10939|AAAAAAAAMLKCAAAA|10939|3|2|19|AM|third|night|| +10940|AAAAAAAANLKCAAAA|10940|3|2|20|AM|third|night|| +10941|AAAAAAAAOLKCAAAA|10941|3|2|21|AM|third|night|| +10942|AAAAAAAAPLKCAAAA|10942|3|2|22|AM|third|night|| +10943|AAAAAAAAAMKCAAAA|10943|3|2|23|AM|third|night|| +10944|AAAAAAAABMKCAAAA|10944|3|2|24|AM|third|night|| +10945|AAAAAAAACMKCAAAA|10945|3|2|25|AM|third|night|| +10946|AAAAAAAADMKCAAAA|10946|3|2|26|AM|third|night|| +10947|AAAAAAAAEMKCAAAA|10947|3|2|27|AM|third|night|| +10948|AAAAAAAAFMKCAAAA|10948|3|2|28|AM|third|night|| +10949|AAAAAAAAGMKCAAAA|10949|3|2|29|AM|third|night|| +10950|AAAAAAAAHMKCAAAA|10950|3|2|30|AM|third|night|| +10951|AAAAAAAAIMKCAAAA|10951|3|2|31|AM|third|night|| +10952|AAAAAAAAJMKCAAAA|10952|3|2|32|AM|third|night|| +10953|AAAAAAAAKMKCAAAA|10953|3|2|33|AM|third|night|| +10954|AAAAAAAALMKCAAAA|10954|3|2|34|AM|third|night|| +10955|AAAAAAAAMMKCAAAA|10955|3|2|35|AM|third|night|| +10956|AAAAAAAANMKCAAAA|10956|3|2|36|AM|third|night|| +10957|AAAAAAAAOMKCAAAA|10957|3|2|37|AM|third|night|| +10958|AAAAAAAAPMKCAAAA|10958|3|2|38|AM|third|night|| +10959|AAAAAAAAANKCAAAA|10959|3|2|39|AM|third|night|| +10960|AAAAAAAABNKCAAAA|10960|3|2|40|AM|third|night|| +10961|AAAAAAAACNKCAAAA|10961|3|2|41|AM|third|night|| +10962|AAAAAAAADNKCAAAA|10962|3|2|42|AM|third|night|| +10963|AAAAAAAAENKCAAAA|10963|3|2|43|AM|third|night|| +10964|AAAAAAAAFNKCAAAA|10964|3|2|44|AM|third|night|| +10965|AAAAAAAAGNKCAAAA|10965|3|2|45|AM|third|night|| +10966|AAAAAAAAHNKCAAAA|10966|3|2|46|AM|third|night|| +10967|AAAAAAAAINKCAAAA|10967|3|2|47|AM|third|night|| +10968|AAAAAAAAJNKCAAAA|10968|3|2|48|AM|third|night|| +10969|AAAAAAAAKNKCAAAA|10969|3|2|49|AM|third|night|| +10970|AAAAAAAALNKCAAAA|10970|3|2|50|AM|third|night|| +10971|AAAAAAAAMNKCAAAA|10971|3|2|51|AM|third|night|| +10972|AAAAAAAANNKCAAAA|10972|3|2|52|AM|third|night|| +10973|AAAAAAAAONKCAAAA|10973|3|2|53|AM|third|night|| +10974|AAAAAAAAPNKCAAAA|10974|3|2|54|AM|third|night|| +10975|AAAAAAAAAOKCAAAA|10975|3|2|55|AM|third|night|| +10976|AAAAAAAABOKCAAAA|10976|3|2|56|AM|third|night|| +10977|AAAAAAAACOKCAAAA|10977|3|2|57|AM|third|night|| +10978|AAAAAAAADOKCAAAA|10978|3|2|58|AM|third|night|| +10979|AAAAAAAAEOKCAAAA|10979|3|2|59|AM|third|night|| +10980|AAAAAAAAFOKCAAAA|10980|3|3|0|AM|third|night|| +10981|AAAAAAAAGOKCAAAA|10981|3|3|1|AM|third|night|| +10982|AAAAAAAAHOKCAAAA|10982|3|3|2|AM|third|night|| +10983|AAAAAAAAIOKCAAAA|10983|3|3|3|AM|third|night|| +10984|AAAAAAAAJOKCAAAA|10984|3|3|4|AM|third|night|| +10985|AAAAAAAAKOKCAAAA|10985|3|3|5|AM|third|night|| +10986|AAAAAAAALOKCAAAA|10986|3|3|6|AM|third|night|| +10987|AAAAAAAAMOKCAAAA|10987|3|3|7|AM|third|night|| +10988|AAAAAAAANOKCAAAA|10988|3|3|8|AM|third|night|| +10989|AAAAAAAAOOKCAAAA|10989|3|3|9|AM|third|night|| +10990|AAAAAAAAPOKCAAAA|10990|3|3|10|AM|third|night|| +10991|AAAAAAAAAPKCAAAA|10991|3|3|11|AM|third|night|| +10992|AAAAAAAABPKCAAAA|10992|3|3|12|AM|third|night|| +10993|AAAAAAAACPKCAAAA|10993|3|3|13|AM|third|night|| +10994|AAAAAAAADPKCAAAA|10994|3|3|14|AM|third|night|| +10995|AAAAAAAAEPKCAAAA|10995|3|3|15|AM|third|night|| +10996|AAAAAAAAFPKCAAAA|10996|3|3|16|AM|third|night|| +10997|AAAAAAAAGPKCAAAA|10997|3|3|17|AM|third|night|| +10998|AAAAAAAAHPKCAAAA|10998|3|3|18|AM|third|night|| +10999|AAAAAAAAIPKCAAAA|10999|3|3|19|AM|third|night|| +11000|AAAAAAAAJPKCAAAA|11000|3|3|20|AM|third|night|| +11001|AAAAAAAAKPKCAAAA|11001|3|3|21|AM|third|night|| +11002|AAAAAAAALPKCAAAA|11002|3|3|22|AM|third|night|| +11003|AAAAAAAAMPKCAAAA|11003|3|3|23|AM|third|night|| +11004|AAAAAAAANPKCAAAA|11004|3|3|24|AM|third|night|| +11005|AAAAAAAAOPKCAAAA|11005|3|3|25|AM|third|night|| +11006|AAAAAAAAPPKCAAAA|11006|3|3|26|AM|third|night|| +11007|AAAAAAAAAALCAAAA|11007|3|3|27|AM|third|night|| +11008|AAAAAAAABALCAAAA|11008|3|3|28|AM|third|night|| +11009|AAAAAAAACALCAAAA|11009|3|3|29|AM|third|night|| +11010|AAAAAAAADALCAAAA|11010|3|3|30|AM|third|night|| +11011|AAAAAAAAEALCAAAA|11011|3|3|31|AM|third|night|| +11012|AAAAAAAAFALCAAAA|11012|3|3|32|AM|third|night|| +11013|AAAAAAAAGALCAAAA|11013|3|3|33|AM|third|night|| +11014|AAAAAAAAHALCAAAA|11014|3|3|34|AM|third|night|| +11015|AAAAAAAAIALCAAAA|11015|3|3|35|AM|third|night|| +11016|AAAAAAAAJALCAAAA|11016|3|3|36|AM|third|night|| +11017|AAAAAAAAKALCAAAA|11017|3|3|37|AM|third|night|| +11018|AAAAAAAALALCAAAA|11018|3|3|38|AM|third|night|| +11019|AAAAAAAAMALCAAAA|11019|3|3|39|AM|third|night|| +11020|AAAAAAAANALCAAAA|11020|3|3|40|AM|third|night|| +11021|AAAAAAAAOALCAAAA|11021|3|3|41|AM|third|night|| +11022|AAAAAAAAPALCAAAA|11022|3|3|42|AM|third|night|| +11023|AAAAAAAAABLCAAAA|11023|3|3|43|AM|third|night|| +11024|AAAAAAAABBLCAAAA|11024|3|3|44|AM|third|night|| +11025|AAAAAAAACBLCAAAA|11025|3|3|45|AM|third|night|| +11026|AAAAAAAADBLCAAAA|11026|3|3|46|AM|third|night|| +11027|AAAAAAAAEBLCAAAA|11027|3|3|47|AM|third|night|| +11028|AAAAAAAAFBLCAAAA|11028|3|3|48|AM|third|night|| +11029|AAAAAAAAGBLCAAAA|11029|3|3|49|AM|third|night|| +11030|AAAAAAAAHBLCAAAA|11030|3|3|50|AM|third|night|| +11031|AAAAAAAAIBLCAAAA|11031|3|3|51|AM|third|night|| +11032|AAAAAAAAJBLCAAAA|11032|3|3|52|AM|third|night|| +11033|AAAAAAAAKBLCAAAA|11033|3|3|53|AM|third|night|| +11034|AAAAAAAALBLCAAAA|11034|3|3|54|AM|third|night|| +11035|AAAAAAAAMBLCAAAA|11035|3|3|55|AM|third|night|| +11036|AAAAAAAANBLCAAAA|11036|3|3|56|AM|third|night|| +11037|AAAAAAAAOBLCAAAA|11037|3|3|57|AM|third|night|| +11038|AAAAAAAAPBLCAAAA|11038|3|3|58|AM|third|night|| +11039|AAAAAAAAACLCAAAA|11039|3|3|59|AM|third|night|| +11040|AAAAAAAABCLCAAAA|11040|3|4|0|AM|third|night|| +11041|AAAAAAAACCLCAAAA|11041|3|4|1|AM|third|night|| +11042|AAAAAAAADCLCAAAA|11042|3|4|2|AM|third|night|| +11043|AAAAAAAAECLCAAAA|11043|3|4|3|AM|third|night|| +11044|AAAAAAAAFCLCAAAA|11044|3|4|4|AM|third|night|| +11045|AAAAAAAAGCLCAAAA|11045|3|4|5|AM|third|night|| +11046|AAAAAAAAHCLCAAAA|11046|3|4|6|AM|third|night|| +11047|AAAAAAAAICLCAAAA|11047|3|4|7|AM|third|night|| +11048|AAAAAAAAJCLCAAAA|11048|3|4|8|AM|third|night|| +11049|AAAAAAAAKCLCAAAA|11049|3|4|9|AM|third|night|| +11050|AAAAAAAALCLCAAAA|11050|3|4|10|AM|third|night|| +11051|AAAAAAAAMCLCAAAA|11051|3|4|11|AM|third|night|| +11052|AAAAAAAANCLCAAAA|11052|3|4|12|AM|third|night|| +11053|AAAAAAAAOCLCAAAA|11053|3|4|13|AM|third|night|| +11054|AAAAAAAAPCLCAAAA|11054|3|4|14|AM|third|night|| +11055|AAAAAAAAADLCAAAA|11055|3|4|15|AM|third|night|| +11056|AAAAAAAABDLCAAAA|11056|3|4|16|AM|third|night|| +11057|AAAAAAAACDLCAAAA|11057|3|4|17|AM|third|night|| +11058|AAAAAAAADDLCAAAA|11058|3|4|18|AM|third|night|| +11059|AAAAAAAAEDLCAAAA|11059|3|4|19|AM|third|night|| +11060|AAAAAAAAFDLCAAAA|11060|3|4|20|AM|third|night|| +11061|AAAAAAAAGDLCAAAA|11061|3|4|21|AM|third|night|| +11062|AAAAAAAAHDLCAAAA|11062|3|4|22|AM|third|night|| +11063|AAAAAAAAIDLCAAAA|11063|3|4|23|AM|third|night|| +11064|AAAAAAAAJDLCAAAA|11064|3|4|24|AM|third|night|| +11065|AAAAAAAAKDLCAAAA|11065|3|4|25|AM|third|night|| +11066|AAAAAAAALDLCAAAA|11066|3|4|26|AM|third|night|| +11067|AAAAAAAAMDLCAAAA|11067|3|4|27|AM|third|night|| +11068|AAAAAAAANDLCAAAA|11068|3|4|28|AM|third|night|| +11069|AAAAAAAAODLCAAAA|11069|3|4|29|AM|third|night|| +11070|AAAAAAAAPDLCAAAA|11070|3|4|30|AM|third|night|| +11071|AAAAAAAAAELCAAAA|11071|3|4|31|AM|third|night|| +11072|AAAAAAAABELCAAAA|11072|3|4|32|AM|third|night|| +11073|AAAAAAAACELCAAAA|11073|3|4|33|AM|third|night|| +11074|AAAAAAAADELCAAAA|11074|3|4|34|AM|third|night|| +11075|AAAAAAAAEELCAAAA|11075|3|4|35|AM|third|night|| +11076|AAAAAAAAFELCAAAA|11076|3|4|36|AM|third|night|| +11077|AAAAAAAAGELCAAAA|11077|3|4|37|AM|third|night|| +11078|AAAAAAAAHELCAAAA|11078|3|4|38|AM|third|night|| +11079|AAAAAAAAIELCAAAA|11079|3|4|39|AM|third|night|| +11080|AAAAAAAAJELCAAAA|11080|3|4|40|AM|third|night|| +11081|AAAAAAAAKELCAAAA|11081|3|4|41|AM|third|night|| +11082|AAAAAAAALELCAAAA|11082|3|4|42|AM|third|night|| +11083|AAAAAAAAMELCAAAA|11083|3|4|43|AM|third|night|| +11084|AAAAAAAANELCAAAA|11084|3|4|44|AM|third|night|| +11085|AAAAAAAAOELCAAAA|11085|3|4|45|AM|third|night|| +11086|AAAAAAAAPELCAAAA|11086|3|4|46|AM|third|night|| +11087|AAAAAAAAAFLCAAAA|11087|3|4|47|AM|third|night|| +11088|AAAAAAAABFLCAAAA|11088|3|4|48|AM|third|night|| +11089|AAAAAAAACFLCAAAA|11089|3|4|49|AM|third|night|| +11090|AAAAAAAADFLCAAAA|11090|3|4|50|AM|third|night|| +11091|AAAAAAAAEFLCAAAA|11091|3|4|51|AM|third|night|| +11092|AAAAAAAAFFLCAAAA|11092|3|4|52|AM|third|night|| +11093|AAAAAAAAGFLCAAAA|11093|3|4|53|AM|third|night|| +11094|AAAAAAAAHFLCAAAA|11094|3|4|54|AM|third|night|| +11095|AAAAAAAAIFLCAAAA|11095|3|4|55|AM|third|night|| +11096|AAAAAAAAJFLCAAAA|11096|3|4|56|AM|third|night|| +11097|AAAAAAAAKFLCAAAA|11097|3|4|57|AM|third|night|| +11098|AAAAAAAALFLCAAAA|11098|3|4|58|AM|third|night|| +11099|AAAAAAAAMFLCAAAA|11099|3|4|59|AM|third|night|| +11100|AAAAAAAANFLCAAAA|11100|3|5|0|AM|third|night|| +11101|AAAAAAAAOFLCAAAA|11101|3|5|1|AM|third|night|| +11102|AAAAAAAAPFLCAAAA|11102|3|5|2|AM|third|night|| +11103|AAAAAAAAAGLCAAAA|11103|3|5|3|AM|third|night|| +11104|AAAAAAAABGLCAAAA|11104|3|5|4|AM|third|night|| +11105|AAAAAAAACGLCAAAA|11105|3|5|5|AM|third|night|| +11106|AAAAAAAADGLCAAAA|11106|3|5|6|AM|third|night|| +11107|AAAAAAAAEGLCAAAA|11107|3|5|7|AM|third|night|| +11108|AAAAAAAAFGLCAAAA|11108|3|5|8|AM|third|night|| +11109|AAAAAAAAGGLCAAAA|11109|3|5|9|AM|third|night|| +11110|AAAAAAAAHGLCAAAA|11110|3|5|10|AM|third|night|| +11111|AAAAAAAAIGLCAAAA|11111|3|5|11|AM|third|night|| +11112|AAAAAAAAJGLCAAAA|11112|3|5|12|AM|third|night|| +11113|AAAAAAAAKGLCAAAA|11113|3|5|13|AM|third|night|| +11114|AAAAAAAALGLCAAAA|11114|3|5|14|AM|third|night|| +11115|AAAAAAAAMGLCAAAA|11115|3|5|15|AM|third|night|| +11116|AAAAAAAANGLCAAAA|11116|3|5|16|AM|third|night|| +11117|AAAAAAAAOGLCAAAA|11117|3|5|17|AM|third|night|| +11118|AAAAAAAAPGLCAAAA|11118|3|5|18|AM|third|night|| +11119|AAAAAAAAAHLCAAAA|11119|3|5|19|AM|third|night|| +11120|AAAAAAAABHLCAAAA|11120|3|5|20|AM|third|night|| +11121|AAAAAAAACHLCAAAA|11121|3|5|21|AM|third|night|| +11122|AAAAAAAADHLCAAAA|11122|3|5|22|AM|third|night|| +11123|AAAAAAAAEHLCAAAA|11123|3|5|23|AM|third|night|| +11124|AAAAAAAAFHLCAAAA|11124|3|5|24|AM|third|night|| +11125|AAAAAAAAGHLCAAAA|11125|3|5|25|AM|third|night|| +11126|AAAAAAAAHHLCAAAA|11126|3|5|26|AM|third|night|| +11127|AAAAAAAAIHLCAAAA|11127|3|5|27|AM|third|night|| +11128|AAAAAAAAJHLCAAAA|11128|3|5|28|AM|third|night|| +11129|AAAAAAAAKHLCAAAA|11129|3|5|29|AM|third|night|| +11130|AAAAAAAALHLCAAAA|11130|3|5|30|AM|third|night|| +11131|AAAAAAAAMHLCAAAA|11131|3|5|31|AM|third|night|| +11132|AAAAAAAANHLCAAAA|11132|3|5|32|AM|third|night|| +11133|AAAAAAAAOHLCAAAA|11133|3|5|33|AM|third|night|| +11134|AAAAAAAAPHLCAAAA|11134|3|5|34|AM|third|night|| +11135|AAAAAAAAAILCAAAA|11135|3|5|35|AM|third|night|| +11136|AAAAAAAABILCAAAA|11136|3|5|36|AM|third|night|| +11137|AAAAAAAACILCAAAA|11137|3|5|37|AM|third|night|| +11138|AAAAAAAADILCAAAA|11138|3|5|38|AM|third|night|| +11139|AAAAAAAAEILCAAAA|11139|3|5|39|AM|third|night|| +11140|AAAAAAAAFILCAAAA|11140|3|5|40|AM|third|night|| +11141|AAAAAAAAGILCAAAA|11141|3|5|41|AM|third|night|| +11142|AAAAAAAAHILCAAAA|11142|3|5|42|AM|third|night|| +11143|AAAAAAAAIILCAAAA|11143|3|5|43|AM|third|night|| +11144|AAAAAAAAJILCAAAA|11144|3|5|44|AM|third|night|| +11145|AAAAAAAAKILCAAAA|11145|3|5|45|AM|third|night|| +11146|AAAAAAAALILCAAAA|11146|3|5|46|AM|third|night|| +11147|AAAAAAAAMILCAAAA|11147|3|5|47|AM|third|night|| +11148|AAAAAAAANILCAAAA|11148|3|5|48|AM|third|night|| +11149|AAAAAAAAOILCAAAA|11149|3|5|49|AM|third|night|| +11150|AAAAAAAAPILCAAAA|11150|3|5|50|AM|third|night|| +11151|AAAAAAAAAJLCAAAA|11151|3|5|51|AM|third|night|| +11152|AAAAAAAABJLCAAAA|11152|3|5|52|AM|third|night|| +11153|AAAAAAAACJLCAAAA|11153|3|5|53|AM|third|night|| +11154|AAAAAAAADJLCAAAA|11154|3|5|54|AM|third|night|| +11155|AAAAAAAAEJLCAAAA|11155|3|5|55|AM|third|night|| +11156|AAAAAAAAFJLCAAAA|11156|3|5|56|AM|third|night|| +11157|AAAAAAAAGJLCAAAA|11157|3|5|57|AM|third|night|| +11158|AAAAAAAAHJLCAAAA|11158|3|5|58|AM|third|night|| +11159|AAAAAAAAIJLCAAAA|11159|3|5|59|AM|third|night|| +11160|AAAAAAAAJJLCAAAA|11160|3|6|0|AM|third|night|| +11161|AAAAAAAAKJLCAAAA|11161|3|6|1|AM|third|night|| +11162|AAAAAAAALJLCAAAA|11162|3|6|2|AM|third|night|| +11163|AAAAAAAAMJLCAAAA|11163|3|6|3|AM|third|night|| +11164|AAAAAAAANJLCAAAA|11164|3|6|4|AM|third|night|| +11165|AAAAAAAAOJLCAAAA|11165|3|6|5|AM|third|night|| +11166|AAAAAAAAPJLCAAAA|11166|3|6|6|AM|third|night|| +11167|AAAAAAAAAKLCAAAA|11167|3|6|7|AM|third|night|| +11168|AAAAAAAABKLCAAAA|11168|3|6|8|AM|third|night|| +11169|AAAAAAAACKLCAAAA|11169|3|6|9|AM|third|night|| +11170|AAAAAAAADKLCAAAA|11170|3|6|10|AM|third|night|| +11171|AAAAAAAAEKLCAAAA|11171|3|6|11|AM|third|night|| +11172|AAAAAAAAFKLCAAAA|11172|3|6|12|AM|third|night|| +11173|AAAAAAAAGKLCAAAA|11173|3|6|13|AM|third|night|| +11174|AAAAAAAAHKLCAAAA|11174|3|6|14|AM|third|night|| +11175|AAAAAAAAIKLCAAAA|11175|3|6|15|AM|third|night|| +11176|AAAAAAAAJKLCAAAA|11176|3|6|16|AM|third|night|| +11177|AAAAAAAAKKLCAAAA|11177|3|6|17|AM|third|night|| +11178|AAAAAAAALKLCAAAA|11178|3|6|18|AM|third|night|| +11179|AAAAAAAAMKLCAAAA|11179|3|6|19|AM|third|night|| +11180|AAAAAAAANKLCAAAA|11180|3|6|20|AM|third|night|| +11181|AAAAAAAAOKLCAAAA|11181|3|6|21|AM|third|night|| +11182|AAAAAAAAPKLCAAAA|11182|3|6|22|AM|third|night|| +11183|AAAAAAAAALLCAAAA|11183|3|6|23|AM|third|night|| +11184|AAAAAAAABLLCAAAA|11184|3|6|24|AM|third|night|| +11185|AAAAAAAACLLCAAAA|11185|3|6|25|AM|third|night|| +11186|AAAAAAAADLLCAAAA|11186|3|6|26|AM|third|night|| +11187|AAAAAAAAELLCAAAA|11187|3|6|27|AM|third|night|| +11188|AAAAAAAAFLLCAAAA|11188|3|6|28|AM|third|night|| +11189|AAAAAAAAGLLCAAAA|11189|3|6|29|AM|third|night|| +11190|AAAAAAAAHLLCAAAA|11190|3|6|30|AM|third|night|| +11191|AAAAAAAAILLCAAAA|11191|3|6|31|AM|third|night|| +11192|AAAAAAAAJLLCAAAA|11192|3|6|32|AM|third|night|| +11193|AAAAAAAAKLLCAAAA|11193|3|6|33|AM|third|night|| +11194|AAAAAAAALLLCAAAA|11194|3|6|34|AM|third|night|| +11195|AAAAAAAAMLLCAAAA|11195|3|6|35|AM|third|night|| +11196|AAAAAAAANLLCAAAA|11196|3|6|36|AM|third|night|| +11197|AAAAAAAAOLLCAAAA|11197|3|6|37|AM|third|night|| +11198|AAAAAAAAPLLCAAAA|11198|3|6|38|AM|third|night|| +11199|AAAAAAAAAMLCAAAA|11199|3|6|39|AM|third|night|| +11200|AAAAAAAABMLCAAAA|11200|3|6|40|AM|third|night|| +11201|AAAAAAAACMLCAAAA|11201|3|6|41|AM|third|night|| +11202|AAAAAAAADMLCAAAA|11202|3|6|42|AM|third|night|| +11203|AAAAAAAAEMLCAAAA|11203|3|6|43|AM|third|night|| +11204|AAAAAAAAFMLCAAAA|11204|3|6|44|AM|third|night|| +11205|AAAAAAAAGMLCAAAA|11205|3|6|45|AM|third|night|| +11206|AAAAAAAAHMLCAAAA|11206|3|6|46|AM|third|night|| +11207|AAAAAAAAIMLCAAAA|11207|3|6|47|AM|third|night|| +11208|AAAAAAAAJMLCAAAA|11208|3|6|48|AM|third|night|| +11209|AAAAAAAAKMLCAAAA|11209|3|6|49|AM|third|night|| +11210|AAAAAAAALMLCAAAA|11210|3|6|50|AM|third|night|| +11211|AAAAAAAAMMLCAAAA|11211|3|6|51|AM|third|night|| +11212|AAAAAAAANMLCAAAA|11212|3|6|52|AM|third|night|| +11213|AAAAAAAAOMLCAAAA|11213|3|6|53|AM|third|night|| +11214|AAAAAAAAPMLCAAAA|11214|3|6|54|AM|third|night|| +11215|AAAAAAAAANLCAAAA|11215|3|6|55|AM|third|night|| +11216|AAAAAAAABNLCAAAA|11216|3|6|56|AM|third|night|| +11217|AAAAAAAACNLCAAAA|11217|3|6|57|AM|third|night|| +11218|AAAAAAAADNLCAAAA|11218|3|6|58|AM|third|night|| +11219|AAAAAAAAENLCAAAA|11219|3|6|59|AM|third|night|| +11220|AAAAAAAAFNLCAAAA|11220|3|7|0|AM|third|night|| +11221|AAAAAAAAGNLCAAAA|11221|3|7|1|AM|third|night|| +11222|AAAAAAAAHNLCAAAA|11222|3|7|2|AM|third|night|| +11223|AAAAAAAAINLCAAAA|11223|3|7|3|AM|third|night|| +11224|AAAAAAAAJNLCAAAA|11224|3|7|4|AM|third|night|| +11225|AAAAAAAAKNLCAAAA|11225|3|7|5|AM|third|night|| +11226|AAAAAAAALNLCAAAA|11226|3|7|6|AM|third|night|| +11227|AAAAAAAAMNLCAAAA|11227|3|7|7|AM|third|night|| +11228|AAAAAAAANNLCAAAA|11228|3|7|8|AM|third|night|| +11229|AAAAAAAAONLCAAAA|11229|3|7|9|AM|third|night|| +11230|AAAAAAAAPNLCAAAA|11230|3|7|10|AM|third|night|| +11231|AAAAAAAAAOLCAAAA|11231|3|7|11|AM|third|night|| +11232|AAAAAAAABOLCAAAA|11232|3|7|12|AM|third|night|| +11233|AAAAAAAACOLCAAAA|11233|3|7|13|AM|third|night|| +11234|AAAAAAAADOLCAAAA|11234|3|7|14|AM|third|night|| +11235|AAAAAAAAEOLCAAAA|11235|3|7|15|AM|third|night|| +11236|AAAAAAAAFOLCAAAA|11236|3|7|16|AM|third|night|| +11237|AAAAAAAAGOLCAAAA|11237|3|7|17|AM|third|night|| +11238|AAAAAAAAHOLCAAAA|11238|3|7|18|AM|third|night|| +11239|AAAAAAAAIOLCAAAA|11239|3|7|19|AM|third|night|| +11240|AAAAAAAAJOLCAAAA|11240|3|7|20|AM|third|night|| +11241|AAAAAAAAKOLCAAAA|11241|3|7|21|AM|third|night|| +11242|AAAAAAAALOLCAAAA|11242|3|7|22|AM|third|night|| +11243|AAAAAAAAMOLCAAAA|11243|3|7|23|AM|third|night|| +11244|AAAAAAAANOLCAAAA|11244|3|7|24|AM|third|night|| +11245|AAAAAAAAOOLCAAAA|11245|3|7|25|AM|third|night|| +11246|AAAAAAAAPOLCAAAA|11246|3|7|26|AM|third|night|| +11247|AAAAAAAAAPLCAAAA|11247|3|7|27|AM|third|night|| +11248|AAAAAAAABPLCAAAA|11248|3|7|28|AM|third|night|| +11249|AAAAAAAACPLCAAAA|11249|3|7|29|AM|third|night|| +11250|AAAAAAAADPLCAAAA|11250|3|7|30|AM|third|night|| +11251|AAAAAAAAEPLCAAAA|11251|3|7|31|AM|third|night|| +11252|AAAAAAAAFPLCAAAA|11252|3|7|32|AM|third|night|| +11253|AAAAAAAAGPLCAAAA|11253|3|7|33|AM|third|night|| +11254|AAAAAAAAHPLCAAAA|11254|3|7|34|AM|third|night|| +11255|AAAAAAAAIPLCAAAA|11255|3|7|35|AM|third|night|| +11256|AAAAAAAAJPLCAAAA|11256|3|7|36|AM|third|night|| +11257|AAAAAAAAKPLCAAAA|11257|3|7|37|AM|third|night|| +11258|AAAAAAAALPLCAAAA|11258|3|7|38|AM|third|night|| +11259|AAAAAAAAMPLCAAAA|11259|3|7|39|AM|third|night|| +11260|AAAAAAAANPLCAAAA|11260|3|7|40|AM|third|night|| +11261|AAAAAAAAOPLCAAAA|11261|3|7|41|AM|third|night|| +11262|AAAAAAAAPPLCAAAA|11262|3|7|42|AM|third|night|| +11263|AAAAAAAAAAMCAAAA|11263|3|7|43|AM|third|night|| +11264|AAAAAAAABAMCAAAA|11264|3|7|44|AM|third|night|| +11265|AAAAAAAACAMCAAAA|11265|3|7|45|AM|third|night|| +11266|AAAAAAAADAMCAAAA|11266|3|7|46|AM|third|night|| +11267|AAAAAAAAEAMCAAAA|11267|3|7|47|AM|third|night|| +11268|AAAAAAAAFAMCAAAA|11268|3|7|48|AM|third|night|| +11269|AAAAAAAAGAMCAAAA|11269|3|7|49|AM|third|night|| +11270|AAAAAAAAHAMCAAAA|11270|3|7|50|AM|third|night|| +11271|AAAAAAAAIAMCAAAA|11271|3|7|51|AM|third|night|| +11272|AAAAAAAAJAMCAAAA|11272|3|7|52|AM|third|night|| +11273|AAAAAAAAKAMCAAAA|11273|3|7|53|AM|third|night|| +11274|AAAAAAAALAMCAAAA|11274|3|7|54|AM|third|night|| +11275|AAAAAAAAMAMCAAAA|11275|3|7|55|AM|third|night|| +11276|AAAAAAAANAMCAAAA|11276|3|7|56|AM|third|night|| +11277|AAAAAAAAOAMCAAAA|11277|3|7|57|AM|third|night|| +11278|AAAAAAAAPAMCAAAA|11278|3|7|58|AM|third|night|| +11279|AAAAAAAAABMCAAAA|11279|3|7|59|AM|third|night|| +11280|AAAAAAAABBMCAAAA|11280|3|8|0|AM|third|night|| +11281|AAAAAAAACBMCAAAA|11281|3|8|1|AM|third|night|| +11282|AAAAAAAADBMCAAAA|11282|3|8|2|AM|third|night|| +11283|AAAAAAAAEBMCAAAA|11283|3|8|3|AM|third|night|| +11284|AAAAAAAAFBMCAAAA|11284|3|8|4|AM|third|night|| +11285|AAAAAAAAGBMCAAAA|11285|3|8|5|AM|third|night|| +11286|AAAAAAAAHBMCAAAA|11286|3|8|6|AM|third|night|| +11287|AAAAAAAAIBMCAAAA|11287|3|8|7|AM|third|night|| +11288|AAAAAAAAJBMCAAAA|11288|3|8|8|AM|third|night|| +11289|AAAAAAAAKBMCAAAA|11289|3|8|9|AM|third|night|| +11290|AAAAAAAALBMCAAAA|11290|3|8|10|AM|third|night|| +11291|AAAAAAAAMBMCAAAA|11291|3|8|11|AM|third|night|| +11292|AAAAAAAANBMCAAAA|11292|3|8|12|AM|third|night|| +11293|AAAAAAAAOBMCAAAA|11293|3|8|13|AM|third|night|| +11294|AAAAAAAAPBMCAAAA|11294|3|8|14|AM|third|night|| +11295|AAAAAAAAACMCAAAA|11295|3|8|15|AM|third|night|| +11296|AAAAAAAABCMCAAAA|11296|3|8|16|AM|third|night|| +11297|AAAAAAAACCMCAAAA|11297|3|8|17|AM|third|night|| +11298|AAAAAAAADCMCAAAA|11298|3|8|18|AM|third|night|| +11299|AAAAAAAAECMCAAAA|11299|3|8|19|AM|third|night|| +11300|AAAAAAAAFCMCAAAA|11300|3|8|20|AM|third|night|| +11301|AAAAAAAAGCMCAAAA|11301|3|8|21|AM|third|night|| +11302|AAAAAAAAHCMCAAAA|11302|3|8|22|AM|third|night|| +11303|AAAAAAAAICMCAAAA|11303|3|8|23|AM|third|night|| +11304|AAAAAAAAJCMCAAAA|11304|3|8|24|AM|third|night|| +11305|AAAAAAAAKCMCAAAA|11305|3|8|25|AM|third|night|| +11306|AAAAAAAALCMCAAAA|11306|3|8|26|AM|third|night|| +11307|AAAAAAAAMCMCAAAA|11307|3|8|27|AM|third|night|| +11308|AAAAAAAANCMCAAAA|11308|3|8|28|AM|third|night|| +11309|AAAAAAAAOCMCAAAA|11309|3|8|29|AM|third|night|| +11310|AAAAAAAAPCMCAAAA|11310|3|8|30|AM|third|night|| +11311|AAAAAAAAADMCAAAA|11311|3|8|31|AM|third|night|| +11312|AAAAAAAABDMCAAAA|11312|3|8|32|AM|third|night|| +11313|AAAAAAAACDMCAAAA|11313|3|8|33|AM|third|night|| +11314|AAAAAAAADDMCAAAA|11314|3|8|34|AM|third|night|| +11315|AAAAAAAAEDMCAAAA|11315|3|8|35|AM|third|night|| +11316|AAAAAAAAFDMCAAAA|11316|3|8|36|AM|third|night|| +11317|AAAAAAAAGDMCAAAA|11317|3|8|37|AM|third|night|| +11318|AAAAAAAAHDMCAAAA|11318|3|8|38|AM|third|night|| +11319|AAAAAAAAIDMCAAAA|11319|3|8|39|AM|third|night|| +11320|AAAAAAAAJDMCAAAA|11320|3|8|40|AM|third|night|| +11321|AAAAAAAAKDMCAAAA|11321|3|8|41|AM|third|night|| +11322|AAAAAAAALDMCAAAA|11322|3|8|42|AM|third|night|| +11323|AAAAAAAAMDMCAAAA|11323|3|8|43|AM|third|night|| +11324|AAAAAAAANDMCAAAA|11324|3|8|44|AM|third|night|| +11325|AAAAAAAAODMCAAAA|11325|3|8|45|AM|third|night|| +11326|AAAAAAAAPDMCAAAA|11326|3|8|46|AM|third|night|| +11327|AAAAAAAAAEMCAAAA|11327|3|8|47|AM|third|night|| +11328|AAAAAAAABEMCAAAA|11328|3|8|48|AM|third|night|| +11329|AAAAAAAACEMCAAAA|11329|3|8|49|AM|third|night|| +11330|AAAAAAAADEMCAAAA|11330|3|8|50|AM|third|night|| +11331|AAAAAAAAEEMCAAAA|11331|3|8|51|AM|third|night|| +11332|AAAAAAAAFEMCAAAA|11332|3|8|52|AM|third|night|| +11333|AAAAAAAAGEMCAAAA|11333|3|8|53|AM|third|night|| +11334|AAAAAAAAHEMCAAAA|11334|3|8|54|AM|third|night|| +11335|AAAAAAAAIEMCAAAA|11335|3|8|55|AM|third|night|| +11336|AAAAAAAAJEMCAAAA|11336|3|8|56|AM|third|night|| +11337|AAAAAAAAKEMCAAAA|11337|3|8|57|AM|third|night|| +11338|AAAAAAAALEMCAAAA|11338|3|8|58|AM|third|night|| +11339|AAAAAAAAMEMCAAAA|11339|3|8|59|AM|third|night|| +11340|AAAAAAAANEMCAAAA|11340|3|9|0|AM|third|night|| +11341|AAAAAAAAOEMCAAAA|11341|3|9|1|AM|third|night|| +11342|AAAAAAAAPEMCAAAA|11342|3|9|2|AM|third|night|| +11343|AAAAAAAAAFMCAAAA|11343|3|9|3|AM|third|night|| +11344|AAAAAAAABFMCAAAA|11344|3|9|4|AM|third|night|| +11345|AAAAAAAACFMCAAAA|11345|3|9|5|AM|third|night|| +11346|AAAAAAAADFMCAAAA|11346|3|9|6|AM|third|night|| +11347|AAAAAAAAEFMCAAAA|11347|3|9|7|AM|third|night|| +11348|AAAAAAAAFFMCAAAA|11348|3|9|8|AM|third|night|| +11349|AAAAAAAAGFMCAAAA|11349|3|9|9|AM|third|night|| +11350|AAAAAAAAHFMCAAAA|11350|3|9|10|AM|third|night|| +11351|AAAAAAAAIFMCAAAA|11351|3|9|11|AM|third|night|| +11352|AAAAAAAAJFMCAAAA|11352|3|9|12|AM|third|night|| +11353|AAAAAAAAKFMCAAAA|11353|3|9|13|AM|third|night|| +11354|AAAAAAAALFMCAAAA|11354|3|9|14|AM|third|night|| +11355|AAAAAAAAMFMCAAAA|11355|3|9|15|AM|third|night|| +11356|AAAAAAAANFMCAAAA|11356|3|9|16|AM|third|night|| +11357|AAAAAAAAOFMCAAAA|11357|3|9|17|AM|third|night|| +11358|AAAAAAAAPFMCAAAA|11358|3|9|18|AM|third|night|| +11359|AAAAAAAAAGMCAAAA|11359|3|9|19|AM|third|night|| +11360|AAAAAAAABGMCAAAA|11360|3|9|20|AM|third|night|| +11361|AAAAAAAACGMCAAAA|11361|3|9|21|AM|third|night|| +11362|AAAAAAAADGMCAAAA|11362|3|9|22|AM|third|night|| +11363|AAAAAAAAEGMCAAAA|11363|3|9|23|AM|third|night|| +11364|AAAAAAAAFGMCAAAA|11364|3|9|24|AM|third|night|| +11365|AAAAAAAAGGMCAAAA|11365|3|9|25|AM|third|night|| +11366|AAAAAAAAHGMCAAAA|11366|3|9|26|AM|third|night|| +11367|AAAAAAAAIGMCAAAA|11367|3|9|27|AM|third|night|| +11368|AAAAAAAAJGMCAAAA|11368|3|9|28|AM|third|night|| +11369|AAAAAAAAKGMCAAAA|11369|3|9|29|AM|third|night|| +11370|AAAAAAAALGMCAAAA|11370|3|9|30|AM|third|night|| +11371|AAAAAAAAMGMCAAAA|11371|3|9|31|AM|third|night|| +11372|AAAAAAAANGMCAAAA|11372|3|9|32|AM|third|night|| +11373|AAAAAAAAOGMCAAAA|11373|3|9|33|AM|third|night|| +11374|AAAAAAAAPGMCAAAA|11374|3|9|34|AM|third|night|| +11375|AAAAAAAAAHMCAAAA|11375|3|9|35|AM|third|night|| +11376|AAAAAAAABHMCAAAA|11376|3|9|36|AM|third|night|| +11377|AAAAAAAACHMCAAAA|11377|3|9|37|AM|third|night|| +11378|AAAAAAAADHMCAAAA|11378|3|9|38|AM|third|night|| +11379|AAAAAAAAEHMCAAAA|11379|3|9|39|AM|third|night|| +11380|AAAAAAAAFHMCAAAA|11380|3|9|40|AM|third|night|| +11381|AAAAAAAAGHMCAAAA|11381|3|9|41|AM|third|night|| +11382|AAAAAAAAHHMCAAAA|11382|3|9|42|AM|third|night|| +11383|AAAAAAAAIHMCAAAA|11383|3|9|43|AM|third|night|| +11384|AAAAAAAAJHMCAAAA|11384|3|9|44|AM|third|night|| +11385|AAAAAAAAKHMCAAAA|11385|3|9|45|AM|third|night|| +11386|AAAAAAAALHMCAAAA|11386|3|9|46|AM|third|night|| +11387|AAAAAAAAMHMCAAAA|11387|3|9|47|AM|third|night|| +11388|AAAAAAAANHMCAAAA|11388|3|9|48|AM|third|night|| +11389|AAAAAAAAOHMCAAAA|11389|3|9|49|AM|third|night|| +11390|AAAAAAAAPHMCAAAA|11390|3|9|50|AM|third|night|| +11391|AAAAAAAAAIMCAAAA|11391|3|9|51|AM|third|night|| +11392|AAAAAAAABIMCAAAA|11392|3|9|52|AM|third|night|| +11393|AAAAAAAACIMCAAAA|11393|3|9|53|AM|third|night|| +11394|AAAAAAAADIMCAAAA|11394|3|9|54|AM|third|night|| +11395|AAAAAAAAEIMCAAAA|11395|3|9|55|AM|third|night|| +11396|AAAAAAAAFIMCAAAA|11396|3|9|56|AM|third|night|| +11397|AAAAAAAAGIMCAAAA|11397|3|9|57|AM|third|night|| +11398|AAAAAAAAHIMCAAAA|11398|3|9|58|AM|third|night|| +11399|AAAAAAAAIIMCAAAA|11399|3|9|59|AM|third|night|| +11400|AAAAAAAAJIMCAAAA|11400|3|10|0|AM|third|night|| +11401|AAAAAAAAKIMCAAAA|11401|3|10|1|AM|third|night|| +11402|AAAAAAAALIMCAAAA|11402|3|10|2|AM|third|night|| +11403|AAAAAAAAMIMCAAAA|11403|3|10|3|AM|third|night|| +11404|AAAAAAAANIMCAAAA|11404|3|10|4|AM|third|night|| +11405|AAAAAAAAOIMCAAAA|11405|3|10|5|AM|third|night|| +11406|AAAAAAAAPIMCAAAA|11406|3|10|6|AM|third|night|| +11407|AAAAAAAAAJMCAAAA|11407|3|10|7|AM|third|night|| +11408|AAAAAAAABJMCAAAA|11408|3|10|8|AM|third|night|| +11409|AAAAAAAACJMCAAAA|11409|3|10|9|AM|third|night|| +11410|AAAAAAAADJMCAAAA|11410|3|10|10|AM|third|night|| +11411|AAAAAAAAEJMCAAAA|11411|3|10|11|AM|third|night|| +11412|AAAAAAAAFJMCAAAA|11412|3|10|12|AM|third|night|| +11413|AAAAAAAAGJMCAAAA|11413|3|10|13|AM|third|night|| +11414|AAAAAAAAHJMCAAAA|11414|3|10|14|AM|third|night|| +11415|AAAAAAAAIJMCAAAA|11415|3|10|15|AM|third|night|| +11416|AAAAAAAAJJMCAAAA|11416|3|10|16|AM|third|night|| +11417|AAAAAAAAKJMCAAAA|11417|3|10|17|AM|third|night|| +11418|AAAAAAAALJMCAAAA|11418|3|10|18|AM|third|night|| +11419|AAAAAAAAMJMCAAAA|11419|3|10|19|AM|third|night|| +11420|AAAAAAAANJMCAAAA|11420|3|10|20|AM|third|night|| +11421|AAAAAAAAOJMCAAAA|11421|3|10|21|AM|third|night|| +11422|AAAAAAAAPJMCAAAA|11422|3|10|22|AM|third|night|| +11423|AAAAAAAAAKMCAAAA|11423|3|10|23|AM|third|night|| +11424|AAAAAAAABKMCAAAA|11424|3|10|24|AM|third|night|| +11425|AAAAAAAACKMCAAAA|11425|3|10|25|AM|third|night|| +11426|AAAAAAAADKMCAAAA|11426|3|10|26|AM|third|night|| +11427|AAAAAAAAEKMCAAAA|11427|3|10|27|AM|third|night|| +11428|AAAAAAAAFKMCAAAA|11428|3|10|28|AM|third|night|| +11429|AAAAAAAAGKMCAAAA|11429|3|10|29|AM|third|night|| +11430|AAAAAAAAHKMCAAAA|11430|3|10|30|AM|third|night|| +11431|AAAAAAAAIKMCAAAA|11431|3|10|31|AM|third|night|| +11432|AAAAAAAAJKMCAAAA|11432|3|10|32|AM|third|night|| +11433|AAAAAAAAKKMCAAAA|11433|3|10|33|AM|third|night|| +11434|AAAAAAAALKMCAAAA|11434|3|10|34|AM|third|night|| +11435|AAAAAAAAMKMCAAAA|11435|3|10|35|AM|third|night|| +11436|AAAAAAAANKMCAAAA|11436|3|10|36|AM|third|night|| +11437|AAAAAAAAOKMCAAAA|11437|3|10|37|AM|third|night|| +11438|AAAAAAAAPKMCAAAA|11438|3|10|38|AM|third|night|| +11439|AAAAAAAAALMCAAAA|11439|3|10|39|AM|third|night|| +11440|AAAAAAAABLMCAAAA|11440|3|10|40|AM|third|night|| +11441|AAAAAAAACLMCAAAA|11441|3|10|41|AM|third|night|| +11442|AAAAAAAADLMCAAAA|11442|3|10|42|AM|third|night|| +11443|AAAAAAAAELMCAAAA|11443|3|10|43|AM|third|night|| +11444|AAAAAAAAFLMCAAAA|11444|3|10|44|AM|third|night|| +11445|AAAAAAAAGLMCAAAA|11445|3|10|45|AM|third|night|| +11446|AAAAAAAAHLMCAAAA|11446|3|10|46|AM|third|night|| +11447|AAAAAAAAILMCAAAA|11447|3|10|47|AM|third|night|| +11448|AAAAAAAAJLMCAAAA|11448|3|10|48|AM|third|night|| +11449|AAAAAAAAKLMCAAAA|11449|3|10|49|AM|third|night|| +11450|AAAAAAAALLMCAAAA|11450|3|10|50|AM|third|night|| +11451|AAAAAAAAMLMCAAAA|11451|3|10|51|AM|third|night|| +11452|AAAAAAAANLMCAAAA|11452|3|10|52|AM|third|night|| +11453|AAAAAAAAOLMCAAAA|11453|3|10|53|AM|third|night|| +11454|AAAAAAAAPLMCAAAA|11454|3|10|54|AM|third|night|| +11455|AAAAAAAAAMMCAAAA|11455|3|10|55|AM|third|night|| +11456|AAAAAAAABMMCAAAA|11456|3|10|56|AM|third|night|| +11457|AAAAAAAACMMCAAAA|11457|3|10|57|AM|third|night|| +11458|AAAAAAAADMMCAAAA|11458|3|10|58|AM|third|night|| +11459|AAAAAAAAEMMCAAAA|11459|3|10|59|AM|third|night|| +11460|AAAAAAAAFMMCAAAA|11460|3|11|0|AM|third|night|| +11461|AAAAAAAAGMMCAAAA|11461|3|11|1|AM|third|night|| +11462|AAAAAAAAHMMCAAAA|11462|3|11|2|AM|third|night|| +11463|AAAAAAAAIMMCAAAA|11463|3|11|3|AM|third|night|| +11464|AAAAAAAAJMMCAAAA|11464|3|11|4|AM|third|night|| +11465|AAAAAAAAKMMCAAAA|11465|3|11|5|AM|third|night|| +11466|AAAAAAAALMMCAAAA|11466|3|11|6|AM|third|night|| +11467|AAAAAAAAMMMCAAAA|11467|3|11|7|AM|third|night|| +11468|AAAAAAAANMMCAAAA|11468|3|11|8|AM|third|night|| +11469|AAAAAAAAOMMCAAAA|11469|3|11|9|AM|third|night|| +11470|AAAAAAAAPMMCAAAA|11470|3|11|10|AM|third|night|| +11471|AAAAAAAAANMCAAAA|11471|3|11|11|AM|third|night|| +11472|AAAAAAAABNMCAAAA|11472|3|11|12|AM|third|night|| +11473|AAAAAAAACNMCAAAA|11473|3|11|13|AM|third|night|| +11474|AAAAAAAADNMCAAAA|11474|3|11|14|AM|third|night|| +11475|AAAAAAAAENMCAAAA|11475|3|11|15|AM|third|night|| +11476|AAAAAAAAFNMCAAAA|11476|3|11|16|AM|third|night|| +11477|AAAAAAAAGNMCAAAA|11477|3|11|17|AM|third|night|| +11478|AAAAAAAAHNMCAAAA|11478|3|11|18|AM|third|night|| +11479|AAAAAAAAINMCAAAA|11479|3|11|19|AM|third|night|| +11480|AAAAAAAAJNMCAAAA|11480|3|11|20|AM|third|night|| +11481|AAAAAAAAKNMCAAAA|11481|3|11|21|AM|third|night|| +11482|AAAAAAAALNMCAAAA|11482|3|11|22|AM|third|night|| +11483|AAAAAAAAMNMCAAAA|11483|3|11|23|AM|third|night|| +11484|AAAAAAAANNMCAAAA|11484|3|11|24|AM|third|night|| +11485|AAAAAAAAONMCAAAA|11485|3|11|25|AM|third|night|| +11486|AAAAAAAAPNMCAAAA|11486|3|11|26|AM|third|night|| +11487|AAAAAAAAAOMCAAAA|11487|3|11|27|AM|third|night|| +11488|AAAAAAAABOMCAAAA|11488|3|11|28|AM|third|night|| +11489|AAAAAAAACOMCAAAA|11489|3|11|29|AM|third|night|| +11490|AAAAAAAADOMCAAAA|11490|3|11|30|AM|third|night|| +11491|AAAAAAAAEOMCAAAA|11491|3|11|31|AM|third|night|| +11492|AAAAAAAAFOMCAAAA|11492|3|11|32|AM|third|night|| +11493|AAAAAAAAGOMCAAAA|11493|3|11|33|AM|third|night|| +11494|AAAAAAAAHOMCAAAA|11494|3|11|34|AM|third|night|| +11495|AAAAAAAAIOMCAAAA|11495|3|11|35|AM|third|night|| +11496|AAAAAAAAJOMCAAAA|11496|3|11|36|AM|third|night|| +11497|AAAAAAAAKOMCAAAA|11497|3|11|37|AM|third|night|| +11498|AAAAAAAALOMCAAAA|11498|3|11|38|AM|third|night|| +11499|AAAAAAAAMOMCAAAA|11499|3|11|39|AM|third|night|| +11500|AAAAAAAANOMCAAAA|11500|3|11|40|AM|third|night|| +11501|AAAAAAAAOOMCAAAA|11501|3|11|41|AM|third|night|| +11502|AAAAAAAAPOMCAAAA|11502|3|11|42|AM|third|night|| +11503|AAAAAAAAAPMCAAAA|11503|3|11|43|AM|third|night|| +11504|AAAAAAAABPMCAAAA|11504|3|11|44|AM|third|night|| +11505|AAAAAAAACPMCAAAA|11505|3|11|45|AM|third|night|| +11506|AAAAAAAADPMCAAAA|11506|3|11|46|AM|third|night|| +11507|AAAAAAAAEPMCAAAA|11507|3|11|47|AM|third|night|| +11508|AAAAAAAAFPMCAAAA|11508|3|11|48|AM|third|night|| +11509|AAAAAAAAGPMCAAAA|11509|3|11|49|AM|third|night|| +11510|AAAAAAAAHPMCAAAA|11510|3|11|50|AM|third|night|| +11511|AAAAAAAAIPMCAAAA|11511|3|11|51|AM|third|night|| +11512|AAAAAAAAJPMCAAAA|11512|3|11|52|AM|third|night|| +11513|AAAAAAAAKPMCAAAA|11513|3|11|53|AM|third|night|| +11514|AAAAAAAALPMCAAAA|11514|3|11|54|AM|third|night|| +11515|AAAAAAAAMPMCAAAA|11515|3|11|55|AM|third|night|| +11516|AAAAAAAANPMCAAAA|11516|3|11|56|AM|third|night|| +11517|AAAAAAAAOPMCAAAA|11517|3|11|57|AM|third|night|| +11518|AAAAAAAAPPMCAAAA|11518|3|11|58|AM|third|night|| +11519|AAAAAAAAAANCAAAA|11519|3|11|59|AM|third|night|| +11520|AAAAAAAABANCAAAA|11520|3|12|0|AM|third|night|| +11521|AAAAAAAACANCAAAA|11521|3|12|1|AM|third|night|| +11522|AAAAAAAADANCAAAA|11522|3|12|2|AM|third|night|| +11523|AAAAAAAAEANCAAAA|11523|3|12|3|AM|third|night|| +11524|AAAAAAAAFANCAAAA|11524|3|12|4|AM|third|night|| +11525|AAAAAAAAGANCAAAA|11525|3|12|5|AM|third|night|| +11526|AAAAAAAAHANCAAAA|11526|3|12|6|AM|third|night|| +11527|AAAAAAAAIANCAAAA|11527|3|12|7|AM|third|night|| +11528|AAAAAAAAJANCAAAA|11528|3|12|8|AM|third|night|| +11529|AAAAAAAAKANCAAAA|11529|3|12|9|AM|third|night|| +11530|AAAAAAAALANCAAAA|11530|3|12|10|AM|third|night|| +11531|AAAAAAAAMANCAAAA|11531|3|12|11|AM|third|night|| +11532|AAAAAAAANANCAAAA|11532|3|12|12|AM|third|night|| +11533|AAAAAAAAOANCAAAA|11533|3|12|13|AM|third|night|| +11534|AAAAAAAAPANCAAAA|11534|3|12|14|AM|third|night|| +11535|AAAAAAAAABNCAAAA|11535|3|12|15|AM|third|night|| +11536|AAAAAAAABBNCAAAA|11536|3|12|16|AM|third|night|| +11537|AAAAAAAACBNCAAAA|11537|3|12|17|AM|third|night|| +11538|AAAAAAAADBNCAAAA|11538|3|12|18|AM|third|night|| +11539|AAAAAAAAEBNCAAAA|11539|3|12|19|AM|third|night|| +11540|AAAAAAAAFBNCAAAA|11540|3|12|20|AM|third|night|| +11541|AAAAAAAAGBNCAAAA|11541|3|12|21|AM|third|night|| +11542|AAAAAAAAHBNCAAAA|11542|3|12|22|AM|third|night|| +11543|AAAAAAAAIBNCAAAA|11543|3|12|23|AM|third|night|| +11544|AAAAAAAAJBNCAAAA|11544|3|12|24|AM|third|night|| +11545|AAAAAAAAKBNCAAAA|11545|3|12|25|AM|third|night|| +11546|AAAAAAAALBNCAAAA|11546|3|12|26|AM|third|night|| +11547|AAAAAAAAMBNCAAAA|11547|3|12|27|AM|third|night|| +11548|AAAAAAAANBNCAAAA|11548|3|12|28|AM|third|night|| +11549|AAAAAAAAOBNCAAAA|11549|3|12|29|AM|third|night|| +11550|AAAAAAAAPBNCAAAA|11550|3|12|30|AM|third|night|| +11551|AAAAAAAAACNCAAAA|11551|3|12|31|AM|third|night|| +11552|AAAAAAAABCNCAAAA|11552|3|12|32|AM|third|night|| +11553|AAAAAAAACCNCAAAA|11553|3|12|33|AM|third|night|| +11554|AAAAAAAADCNCAAAA|11554|3|12|34|AM|third|night|| +11555|AAAAAAAAECNCAAAA|11555|3|12|35|AM|third|night|| +11556|AAAAAAAAFCNCAAAA|11556|3|12|36|AM|third|night|| +11557|AAAAAAAAGCNCAAAA|11557|3|12|37|AM|third|night|| +11558|AAAAAAAAHCNCAAAA|11558|3|12|38|AM|third|night|| +11559|AAAAAAAAICNCAAAA|11559|3|12|39|AM|third|night|| +11560|AAAAAAAAJCNCAAAA|11560|3|12|40|AM|third|night|| +11561|AAAAAAAAKCNCAAAA|11561|3|12|41|AM|third|night|| +11562|AAAAAAAALCNCAAAA|11562|3|12|42|AM|third|night|| +11563|AAAAAAAAMCNCAAAA|11563|3|12|43|AM|third|night|| +11564|AAAAAAAANCNCAAAA|11564|3|12|44|AM|third|night|| +11565|AAAAAAAAOCNCAAAA|11565|3|12|45|AM|third|night|| +11566|AAAAAAAAPCNCAAAA|11566|3|12|46|AM|third|night|| +11567|AAAAAAAAADNCAAAA|11567|3|12|47|AM|third|night|| +11568|AAAAAAAABDNCAAAA|11568|3|12|48|AM|third|night|| +11569|AAAAAAAACDNCAAAA|11569|3|12|49|AM|third|night|| +11570|AAAAAAAADDNCAAAA|11570|3|12|50|AM|third|night|| +11571|AAAAAAAAEDNCAAAA|11571|3|12|51|AM|third|night|| +11572|AAAAAAAAFDNCAAAA|11572|3|12|52|AM|third|night|| +11573|AAAAAAAAGDNCAAAA|11573|3|12|53|AM|third|night|| +11574|AAAAAAAAHDNCAAAA|11574|3|12|54|AM|third|night|| +11575|AAAAAAAAIDNCAAAA|11575|3|12|55|AM|third|night|| +11576|AAAAAAAAJDNCAAAA|11576|3|12|56|AM|third|night|| +11577|AAAAAAAAKDNCAAAA|11577|3|12|57|AM|third|night|| +11578|AAAAAAAALDNCAAAA|11578|3|12|58|AM|third|night|| +11579|AAAAAAAAMDNCAAAA|11579|3|12|59|AM|third|night|| +11580|AAAAAAAANDNCAAAA|11580|3|13|0|AM|third|night|| +11581|AAAAAAAAODNCAAAA|11581|3|13|1|AM|third|night|| +11582|AAAAAAAAPDNCAAAA|11582|3|13|2|AM|third|night|| +11583|AAAAAAAAAENCAAAA|11583|3|13|3|AM|third|night|| +11584|AAAAAAAABENCAAAA|11584|3|13|4|AM|third|night|| +11585|AAAAAAAACENCAAAA|11585|3|13|5|AM|third|night|| +11586|AAAAAAAADENCAAAA|11586|3|13|6|AM|third|night|| +11587|AAAAAAAAEENCAAAA|11587|3|13|7|AM|third|night|| +11588|AAAAAAAAFENCAAAA|11588|3|13|8|AM|third|night|| +11589|AAAAAAAAGENCAAAA|11589|3|13|9|AM|third|night|| +11590|AAAAAAAAHENCAAAA|11590|3|13|10|AM|third|night|| +11591|AAAAAAAAIENCAAAA|11591|3|13|11|AM|third|night|| +11592|AAAAAAAAJENCAAAA|11592|3|13|12|AM|third|night|| +11593|AAAAAAAAKENCAAAA|11593|3|13|13|AM|third|night|| +11594|AAAAAAAALENCAAAA|11594|3|13|14|AM|third|night|| +11595|AAAAAAAAMENCAAAA|11595|3|13|15|AM|third|night|| +11596|AAAAAAAANENCAAAA|11596|3|13|16|AM|third|night|| +11597|AAAAAAAAOENCAAAA|11597|3|13|17|AM|third|night|| +11598|AAAAAAAAPENCAAAA|11598|3|13|18|AM|third|night|| +11599|AAAAAAAAAFNCAAAA|11599|3|13|19|AM|third|night|| +11600|AAAAAAAABFNCAAAA|11600|3|13|20|AM|third|night|| +11601|AAAAAAAACFNCAAAA|11601|3|13|21|AM|third|night|| +11602|AAAAAAAADFNCAAAA|11602|3|13|22|AM|third|night|| +11603|AAAAAAAAEFNCAAAA|11603|3|13|23|AM|third|night|| +11604|AAAAAAAAFFNCAAAA|11604|3|13|24|AM|third|night|| +11605|AAAAAAAAGFNCAAAA|11605|3|13|25|AM|third|night|| +11606|AAAAAAAAHFNCAAAA|11606|3|13|26|AM|third|night|| +11607|AAAAAAAAIFNCAAAA|11607|3|13|27|AM|third|night|| +11608|AAAAAAAAJFNCAAAA|11608|3|13|28|AM|third|night|| +11609|AAAAAAAAKFNCAAAA|11609|3|13|29|AM|third|night|| +11610|AAAAAAAALFNCAAAA|11610|3|13|30|AM|third|night|| +11611|AAAAAAAAMFNCAAAA|11611|3|13|31|AM|third|night|| +11612|AAAAAAAANFNCAAAA|11612|3|13|32|AM|third|night|| +11613|AAAAAAAAOFNCAAAA|11613|3|13|33|AM|third|night|| +11614|AAAAAAAAPFNCAAAA|11614|3|13|34|AM|third|night|| +11615|AAAAAAAAAGNCAAAA|11615|3|13|35|AM|third|night|| +11616|AAAAAAAABGNCAAAA|11616|3|13|36|AM|third|night|| +11617|AAAAAAAACGNCAAAA|11617|3|13|37|AM|third|night|| +11618|AAAAAAAADGNCAAAA|11618|3|13|38|AM|third|night|| +11619|AAAAAAAAEGNCAAAA|11619|3|13|39|AM|third|night|| +11620|AAAAAAAAFGNCAAAA|11620|3|13|40|AM|third|night|| +11621|AAAAAAAAGGNCAAAA|11621|3|13|41|AM|third|night|| +11622|AAAAAAAAHGNCAAAA|11622|3|13|42|AM|third|night|| +11623|AAAAAAAAIGNCAAAA|11623|3|13|43|AM|third|night|| +11624|AAAAAAAAJGNCAAAA|11624|3|13|44|AM|third|night|| +11625|AAAAAAAAKGNCAAAA|11625|3|13|45|AM|third|night|| +11626|AAAAAAAALGNCAAAA|11626|3|13|46|AM|third|night|| +11627|AAAAAAAAMGNCAAAA|11627|3|13|47|AM|third|night|| +11628|AAAAAAAANGNCAAAA|11628|3|13|48|AM|third|night|| +11629|AAAAAAAAOGNCAAAA|11629|3|13|49|AM|third|night|| +11630|AAAAAAAAPGNCAAAA|11630|3|13|50|AM|third|night|| +11631|AAAAAAAAAHNCAAAA|11631|3|13|51|AM|third|night|| +11632|AAAAAAAABHNCAAAA|11632|3|13|52|AM|third|night|| +11633|AAAAAAAACHNCAAAA|11633|3|13|53|AM|third|night|| +11634|AAAAAAAADHNCAAAA|11634|3|13|54|AM|third|night|| +11635|AAAAAAAAEHNCAAAA|11635|3|13|55|AM|third|night|| +11636|AAAAAAAAFHNCAAAA|11636|3|13|56|AM|third|night|| +11637|AAAAAAAAGHNCAAAA|11637|3|13|57|AM|third|night|| +11638|AAAAAAAAHHNCAAAA|11638|3|13|58|AM|third|night|| +11639|AAAAAAAAIHNCAAAA|11639|3|13|59|AM|third|night|| +11640|AAAAAAAAJHNCAAAA|11640|3|14|0|AM|third|night|| +11641|AAAAAAAAKHNCAAAA|11641|3|14|1|AM|third|night|| +11642|AAAAAAAALHNCAAAA|11642|3|14|2|AM|third|night|| +11643|AAAAAAAAMHNCAAAA|11643|3|14|3|AM|third|night|| +11644|AAAAAAAANHNCAAAA|11644|3|14|4|AM|third|night|| +11645|AAAAAAAAOHNCAAAA|11645|3|14|5|AM|third|night|| +11646|AAAAAAAAPHNCAAAA|11646|3|14|6|AM|third|night|| +11647|AAAAAAAAAINCAAAA|11647|3|14|7|AM|third|night|| +11648|AAAAAAAABINCAAAA|11648|3|14|8|AM|third|night|| +11649|AAAAAAAACINCAAAA|11649|3|14|9|AM|third|night|| +11650|AAAAAAAADINCAAAA|11650|3|14|10|AM|third|night|| +11651|AAAAAAAAEINCAAAA|11651|3|14|11|AM|third|night|| +11652|AAAAAAAAFINCAAAA|11652|3|14|12|AM|third|night|| +11653|AAAAAAAAGINCAAAA|11653|3|14|13|AM|third|night|| +11654|AAAAAAAAHINCAAAA|11654|3|14|14|AM|third|night|| +11655|AAAAAAAAIINCAAAA|11655|3|14|15|AM|third|night|| +11656|AAAAAAAAJINCAAAA|11656|3|14|16|AM|third|night|| +11657|AAAAAAAAKINCAAAA|11657|3|14|17|AM|third|night|| +11658|AAAAAAAALINCAAAA|11658|3|14|18|AM|third|night|| +11659|AAAAAAAAMINCAAAA|11659|3|14|19|AM|third|night|| +11660|AAAAAAAANINCAAAA|11660|3|14|20|AM|third|night|| +11661|AAAAAAAAOINCAAAA|11661|3|14|21|AM|third|night|| +11662|AAAAAAAAPINCAAAA|11662|3|14|22|AM|third|night|| +11663|AAAAAAAAAJNCAAAA|11663|3|14|23|AM|third|night|| +11664|AAAAAAAABJNCAAAA|11664|3|14|24|AM|third|night|| +11665|AAAAAAAACJNCAAAA|11665|3|14|25|AM|third|night|| +11666|AAAAAAAADJNCAAAA|11666|3|14|26|AM|third|night|| +11667|AAAAAAAAEJNCAAAA|11667|3|14|27|AM|third|night|| +11668|AAAAAAAAFJNCAAAA|11668|3|14|28|AM|third|night|| +11669|AAAAAAAAGJNCAAAA|11669|3|14|29|AM|third|night|| +11670|AAAAAAAAHJNCAAAA|11670|3|14|30|AM|third|night|| +11671|AAAAAAAAIJNCAAAA|11671|3|14|31|AM|third|night|| +11672|AAAAAAAAJJNCAAAA|11672|3|14|32|AM|third|night|| +11673|AAAAAAAAKJNCAAAA|11673|3|14|33|AM|third|night|| +11674|AAAAAAAALJNCAAAA|11674|3|14|34|AM|third|night|| +11675|AAAAAAAAMJNCAAAA|11675|3|14|35|AM|third|night|| +11676|AAAAAAAANJNCAAAA|11676|3|14|36|AM|third|night|| +11677|AAAAAAAAOJNCAAAA|11677|3|14|37|AM|third|night|| +11678|AAAAAAAAPJNCAAAA|11678|3|14|38|AM|third|night|| +11679|AAAAAAAAAKNCAAAA|11679|3|14|39|AM|third|night|| +11680|AAAAAAAABKNCAAAA|11680|3|14|40|AM|third|night|| +11681|AAAAAAAACKNCAAAA|11681|3|14|41|AM|third|night|| +11682|AAAAAAAADKNCAAAA|11682|3|14|42|AM|third|night|| +11683|AAAAAAAAEKNCAAAA|11683|3|14|43|AM|third|night|| +11684|AAAAAAAAFKNCAAAA|11684|3|14|44|AM|third|night|| +11685|AAAAAAAAGKNCAAAA|11685|3|14|45|AM|third|night|| +11686|AAAAAAAAHKNCAAAA|11686|3|14|46|AM|third|night|| +11687|AAAAAAAAIKNCAAAA|11687|3|14|47|AM|third|night|| +11688|AAAAAAAAJKNCAAAA|11688|3|14|48|AM|third|night|| +11689|AAAAAAAAKKNCAAAA|11689|3|14|49|AM|third|night|| +11690|AAAAAAAALKNCAAAA|11690|3|14|50|AM|third|night|| +11691|AAAAAAAAMKNCAAAA|11691|3|14|51|AM|third|night|| +11692|AAAAAAAANKNCAAAA|11692|3|14|52|AM|third|night|| +11693|AAAAAAAAOKNCAAAA|11693|3|14|53|AM|third|night|| +11694|AAAAAAAAPKNCAAAA|11694|3|14|54|AM|third|night|| +11695|AAAAAAAAALNCAAAA|11695|3|14|55|AM|third|night|| +11696|AAAAAAAABLNCAAAA|11696|3|14|56|AM|third|night|| +11697|AAAAAAAACLNCAAAA|11697|3|14|57|AM|third|night|| +11698|AAAAAAAADLNCAAAA|11698|3|14|58|AM|third|night|| +11699|AAAAAAAAELNCAAAA|11699|3|14|59|AM|third|night|| +11700|AAAAAAAAFLNCAAAA|11700|3|15|0|AM|third|night|| +11701|AAAAAAAAGLNCAAAA|11701|3|15|1|AM|third|night|| +11702|AAAAAAAAHLNCAAAA|11702|3|15|2|AM|third|night|| +11703|AAAAAAAAILNCAAAA|11703|3|15|3|AM|third|night|| +11704|AAAAAAAAJLNCAAAA|11704|3|15|4|AM|third|night|| +11705|AAAAAAAAKLNCAAAA|11705|3|15|5|AM|third|night|| +11706|AAAAAAAALLNCAAAA|11706|3|15|6|AM|third|night|| +11707|AAAAAAAAMLNCAAAA|11707|3|15|7|AM|third|night|| +11708|AAAAAAAANLNCAAAA|11708|3|15|8|AM|third|night|| +11709|AAAAAAAAOLNCAAAA|11709|3|15|9|AM|third|night|| +11710|AAAAAAAAPLNCAAAA|11710|3|15|10|AM|third|night|| +11711|AAAAAAAAAMNCAAAA|11711|3|15|11|AM|third|night|| +11712|AAAAAAAABMNCAAAA|11712|3|15|12|AM|third|night|| +11713|AAAAAAAACMNCAAAA|11713|3|15|13|AM|third|night|| +11714|AAAAAAAADMNCAAAA|11714|3|15|14|AM|third|night|| +11715|AAAAAAAAEMNCAAAA|11715|3|15|15|AM|third|night|| +11716|AAAAAAAAFMNCAAAA|11716|3|15|16|AM|third|night|| +11717|AAAAAAAAGMNCAAAA|11717|3|15|17|AM|third|night|| +11718|AAAAAAAAHMNCAAAA|11718|3|15|18|AM|third|night|| +11719|AAAAAAAAIMNCAAAA|11719|3|15|19|AM|third|night|| +11720|AAAAAAAAJMNCAAAA|11720|3|15|20|AM|third|night|| +11721|AAAAAAAAKMNCAAAA|11721|3|15|21|AM|third|night|| +11722|AAAAAAAALMNCAAAA|11722|3|15|22|AM|third|night|| +11723|AAAAAAAAMMNCAAAA|11723|3|15|23|AM|third|night|| +11724|AAAAAAAANMNCAAAA|11724|3|15|24|AM|third|night|| +11725|AAAAAAAAOMNCAAAA|11725|3|15|25|AM|third|night|| +11726|AAAAAAAAPMNCAAAA|11726|3|15|26|AM|third|night|| +11727|AAAAAAAAANNCAAAA|11727|3|15|27|AM|third|night|| +11728|AAAAAAAABNNCAAAA|11728|3|15|28|AM|third|night|| +11729|AAAAAAAACNNCAAAA|11729|3|15|29|AM|third|night|| +11730|AAAAAAAADNNCAAAA|11730|3|15|30|AM|third|night|| +11731|AAAAAAAAENNCAAAA|11731|3|15|31|AM|third|night|| +11732|AAAAAAAAFNNCAAAA|11732|3|15|32|AM|third|night|| +11733|AAAAAAAAGNNCAAAA|11733|3|15|33|AM|third|night|| +11734|AAAAAAAAHNNCAAAA|11734|3|15|34|AM|third|night|| +11735|AAAAAAAAINNCAAAA|11735|3|15|35|AM|third|night|| +11736|AAAAAAAAJNNCAAAA|11736|3|15|36|AM|third|night|| +11737|AAAAAAAAKNNCAAAA|11737|3|15|37|AM|third|night|| +11738|AAAAAAAALNNCAAAA|11738|3|15|38|AM|third|night|| +11739|AAAAAAAAMNNCAAAA|11739|3|15|39|AM|third|night|| +11740|AAAAAAAANNNCAAAA|11740|3|15|40|AM|third|night|| +11741|AAAAAAAAONNCAAAA|11741|3|15|41|AM|third|night|| +11742|AAAAAAAAPNNCAAAA|11742|3|15|42|AM|third|night|| +11743|AAAAAAAAAONCAAAA|11743|3|15|43|AM|third|night|| +11744|AAAAAAAABONCAAAA|11744|3|15|44|AM|third|night|| +11745|AAAAAAAACONCAAAA|11745|3|15|45|AM|third|night|| +11746|AAAAAAAADONCAAAA|11746|3|15|46|AM|third|night|| +11747|AAAAAAAAEONCAAAA|11747|3|15|47|AM|third|night|| +11748|AAAAAAAAFONCAAAA|11748|3|15|48|AM|third|night|| +11749|AAAAAAAAGONCAAAA|11749|3|15|49|AM|third|night|| +11750|AAAAAAAAHONCAAAA|11750|3|15|50|AM|third|night|| +11751|AAAAAAAAIONCAAAA|11751|3|15|51|AM|third|night|| +11752|AAAAAAAAJONCAAAA|11752|3|15|52|AM|third|night|| +11753|AAAAAAAAKONCAAAA|11753|3|15|53|AM|third|night|| +11754|AAAAAAAALONCAAAA|11754|3|15|54|AM|third|night|| +11755|AAAAAAAAMONCAAAA|11755|3|15|55|AM|third|night|| +11756|AAAAAAAANONCAAAA|11756|3|15|56|AM|third|night|| +11757|AAAAAAAAOONCAAAA|11757|3|15|57|AM|third|night|| +11758|AAAAAAAAPONCAAAA|11758|3|15|58|AM|third|night|| +11759|AAAAAAAAAPNCAAAA|11759|3|15|59|AM|third|night|| +11760|AAAAAAAABPNCAAAA|11760|3|16|0|AM|third|night|| +11761|AAAAAAAACPNCAAAA|11761|3|16|1|AM|third|night|| +11762|AAAAAAAADPNCAAAA|11762|3|16|2|AM|third|night|| +11763|AAAAAAAAEPNCAAAA|11763|3|16|3|AM|third|night|| +11764|AAAAAAAAFPNCAAAA|11764|3|16|4|AM|third|night|| +11765|AAAAAAAAGPNCAAAA|11765|3|16|5|AM|third|night|| +11766|AAAAAAAAHPNCAAAA|11766|3|16|6|AM|third|night|| +11767|AAAAAAAAIPNCAAAA|11767|3|16|7|AM|third|night|| +11768|AAAAAAAAJPNCAAAA|11768|3|16|8|AM|third|night|| +11769|AAAAAAAAKPNCAAAA|11769|3|16|9|AM|third|night|| +11770|AAAAAAAALPNCAAAA|11770|3|16|10|AM|third|night|| +11771|AAAAAAAAMPNCAAAA|11771|3|16|11|AM|third|night|| +11772|AAAAAAAANPNCAAAA|11772|3|16|12|AM|third|night|| +11773|AAAAAAAAOPNCAAAA|11773|3|16|13|AM|third|night|| +11774|AAAAAAAAPPNCAAAA|11774|3|16|14|AM|third|night|| +11775|AAAAAAAAAAOCAAAA|11775|3|16|15|AM|third|night|| +11776|AAAAAAAABAOCAAAA|11776|3|16|16|AM|third|night|| +11777|AAAAAAAACAOCAAAA|11777|3|16|17|AM|third|night|| +11778|AAAAAAAADAOCAAAA|11778|3|16|18|AM|third|night|| +11779|AAAAAAAAEAOCAAAA|11779|3|16|19|AM|third|night|| +11780|AAAAAAAAFAOCAAAA|11780|3|16|20|AM|third|night|| +11781|AAAAAAAAGAOCAAAA|11781|3|16|21|AM|third|night|| +11782|AAAAAAAAHAOCAAAA|11782|3|16|22|AM|third|night|| +11783|AAAAAAAAIAOCAAAA|11783|3|16|23|AM|third|night|| +11784|AAAAAAAAJAOCAAAA|11784|3|16|24|AM|third|night|| +11785|AAAAAAAAKAOCAAAA|11785|3|16|25|AM|third|night|| +11786|AAAAAAAALAOCAAAA|11786|3|16|26|AM|third|night|| +11787|AAAAAAAAMAOCAAAA|11787|3|16|27|AM|third|night|| +11788|AAAAAAAANAOCAAAA|11788|3|16|28|AM|third|night|| +11789|AAAAAAAAOAOCAAAA|11789|3|16|29|AM|third|night|| +11790|AAAAAAAAPAOCAAAA|11790|3|16|30|AM|third|night|| +11791|AAAAAAAAABOCAAAA|11791|3|16|31|AM|third|night|| +11792|AAAAAAAABBOCAAAA|11792|3|16|32|AM|third|night|| +11793|AAAAAAAACBOCAAAA|11793|3|16|33|AM|third|night|| +11794|AAAAAAAADBOCAAAA|11794|3|16|34|AM|third|night|| +11795|AAAAAAAAEBOCAAAA|11795|3|16|35|AM|third|night|| +11796|AAAAAAAAFBOCAAAA|11796|3|16|36|AM|third|night|| +11797|AAAAAAAAGBOCAAAA|11797|3|16|37|AM|third|night|| +11798|AAAAAAAAHBOCAAAA|11798|3|16|38|AM|third|night|| +11799|AAAAAAAAIBOCAAAA|11799|3|16|39|AM|third|night|| +11800|AAAAAAAAJBOCAAAA|11800|3|16|40|AM|third|night|| +11801|AAAAAAAAKBOCAAAA|11801|3|16|41|AM|third|night|| +11802|AAAAAAAALBOCAAAA|11802|3|16|42|AM|third|night|| +11803|AAAAAAAAMBOCAAAA|11803|3|16|43|AM|third|night|| +11804|AAAAAAAANBOCAAAA|11804|3|16|44|AM|third|night|| +11805|AAAAAAAAOBOCAAAA|11805|3|16|45|AM|third|night|| +11806|AAAAAAAAPBOCAAAA|11806|3|16|46|AM|third|night|| +11807|AAAAAAAAACOCAAAA|11807|3|16|47|AM|third|night|| +11808|AAAAAAAABCOCAAAA|11808|3|16|48|AM|third|night|| +11809|AAAAAAAACCOCAAAA|11809|3|16|49|AM|third|night|| +11810|AAAAAAAADCOCAAAA|11810|3|16|50|AM|third|night|| +11811|AAAAAAAAECOCAAAA|11811|3|16|51|AM|third|night|| +11812|AAAAAAAAFCOCAAAA|11812|3|16|52|AM|third|night|| +11813|AAAAAAAAGCOCAAAA|11813|3|16|53|AM|third|night|| +11814|AAAAAAAAHCOCAAAA|11814|3|16|54|AM|third|night|| +11815|AAAAAAAAICOCAAAA|11815|3|16|55|AM|third|night|| +11816|AAAAAAAAJCOCAAAA|11816|3|16|56|AM|third|night|| +11817|AAAAAAAAKCOCAAAA|11817|3|16|57|AM|third|night|| +11818|AAAAAAAALCOCAAAA|11818|3|16|58|AM|third|night|| +11819|AAAAAAAAMCOCAAAA|11819|3|16|59|AM|third|night|| +11820|AAAAAAAANCOCAAAA|11820|3|17|0|AM|third|night|| +11821|AAAAAAAAOCOCAAAA|11821|3|17|1|AM|third|night|| +11822|AAAAAAAAPCOCAAAA|11822|3|17|2|AM|third|night|| +11823|AAAAAAAAADOCAAAA|11823|3|17|3|AM|third|night|| +11824|AAAAAAAABDOCAAAA|11824|3|17|4|AM|third|night|| +11825|AAAAAAAACDOCAAAA|11825|3|17|5|AM|third|night|| +11826|AAAAAAAADDOCAAAA|11826|3|17|6|AM|third|night|| +11827|AAAAAAAAEDOCAAAA|11827|3|17|7|AM|third|night|| +11828|AAAAAAAAFDOCAAAA|11828|3|17|8|AM|third|night|| +11829|AAAAAAAAGDOCAAAA|11829|3|17|9|AM|third|night|| +11830|AAAAAAAAHDOCAAAA|11830|3|17|10|AM|third|night|| +11831|AAAAAAAAIDOCAAAA|11831|3|17|11|AM|third|night|| +11832|AAAAAAAAJDOCAAAA|11832|3|17|12|AM|third|night|| +11833|AAAAAAAAKDOCAAAA|11833|3|17|13|AM|third|night|| +11834|AAAAAAAALDOCAAAA|11834|3|17|14|AM|third|night|| +11835|AAAAAAAAMDOCAAAA|11835|3|17|15|AM|third|night|| +11836|AAAAAAAANDOCAAAA|11836|3|17|16|AM|third|night|| +11837|AAAAAAAAODOCAAAA|11837|3|17|17|AM|third|night|| +11838|AAAAAAAAPDOCAAAA|11838|3|17|18|AM|third|night|| +11839|AAAAAAAAAEOCAAAA|11839|3|17|19|AM|third|night|| +11840|AAAAAAAABEOCAAAA|11840|3|17|20|AM|third|night|| +11841|AAAAAAAACEOCAAAA|11841|3|17|21|AM|third|night|| +11842|AAAAAAAADEOCAAAA|11842|3|17|22|AM|third|night|| +11843|AAAAAAAAEEOCAAAA|11843|3|17|23|AM|third|night|| +11844|AAAAAAAAFEOCAAAA|11844|3|17|24|AM|third|night|| +11845|AAAAAAAAGEOCAAAA|11845|3|17|25|AM|third|night|| +11846|AAAAAAAAHEOCAAAA|11846|3|17|26|AM|third|night|| +11847|AAAAAAAAIEOCAAAA|11847|3|17|27|AM|third|night|| +11848|AAAAAAAAJEOCAAAA|11848|3|17|28|AM|third|night|| +11849|AAAAAAAAKEOCAAAA|11849|3|17|29|AM|third|night|| +11850|AAAAAAAALEOCAAAA|11850|3|17|30|AM|third|night|| +11851|AAAAAAAAMEOCAAAA|11851|3|17|31|AM|third|night|| +11852|AAAAAAAANEOCAAAA|11852|3|17|32|AM|third|night|| +11853|AAAAAAAAOEOCAAAA|11853|3|17|33|AM|third|night|| +11854|AAAAAAAAPEOCAAAA|11854|3|17|34|AM|third|night|| +11855|AAAAAAAAAFOCAAAA|11855|3|17|35|AM|third|night|| +11856|AAAAAAAABFOCAAAA|11856|3|17|36|AM|third|night|| +11857|AAAAAAAACFOCAAAA|11857|3|17|37|AM|third|night|| +11858|AAAAAAAADFOCAAAA|11858|3|17|38|AM|third|night|| +11859|AAAAAAAAEFOCAAAA|11859|3|17|39|AM|third|night|| +11860|AAAAAAAAFFOCAAAA|11860|3|17|40|AM|third|night|| +11861|AAAAAAAAGFOCAAAA|11861|3|17|41|AM|third|night|| +11862|AAAAAAAAHFOCAAAA|11862|3|17|42|AM|third|night|| +11863|AAAAAAAAIFOCAAAA|11863|3|17|43|AM|third|night|| +11864|AAAAAAAAJFOCAAAA|11864|3|17|44|AM|third|night|| +11865|AAAAAAAAKFOCAAAA|11865|3|17|45|AM|third|night|| +11866|AAAAAAAALFOCAAAA|11866|3|17|46|AM|third|night|| +11867|AAAAAAAAMFOCAAAA|11867|3|17|47|AM|third|night|| +11868|AAAAAAAANFOCAAAA|11868|3|17|48|AM|third|night|| +11869|AAAAAAAAOFOCAAAA|11869|3|17|49|AM|third|night|| +11870|AAAAAAAAPFOCAAAA|11870|3|17|50|AM|third|night|| +11871|AAAAAAAAAGOCAAAA|11871|3|17|51|AM|third|night|| +11872|AAAAAAAABGOCAAAA|11872|3|17|52|AM|third|night|| +11873|AAAAAAAACGOCAAAA|11873|3|17|53|AM|third|night|| +11874|AAAAAAAADGOCAAAA|11874|3|17|54|AM|third|night|| +11875|AAAAAAAAEGOCAAAA|11875|3|17|55|AM|third|night|| +11876|AAAAAAAAFGOCAAAA|11876|3|17|56|AM|third|night|| +11877|AAAAAAAAGGOCAAAA|11877|3|17|57|AM|third|night|| +11878|AAAAAAAAHGOCAAAA|11878|3|17|58|AM|third|night|| +11879|AAAAAAAAIGOCAAAA|11879|3|17|59|AM|third|night|| +11880|AAAAAAAAJGOCAAAA|11880|3|18|0|AM|third|night|| +11881|AAAAAAAAKGOCAAAA|11881|3|18|1|AM|third|night|| +11882|AAAAAAAALGOCAAAA|11882|3|18|2|AM|third|night|| +11883|AAAAAAAAMGOCAAAA|11883|3|18|3|AM|third|night|| +11884|AAAAAAAANGOCAAAA|11884|3|18|4|AM|third|night|| +11885|AAAAAAAAOGOCAAAA|11885|3|18|5|AM|third|night|| +11886|AAAAAAAAPGOCAAAA|11886|3|18|6|AM|third|night|| +11887|AAAAAAAAAHOCAAAA|11887|3|18|7|AM|third|night|| +11888|AAAAAAAABHOCAAAA|11888|3|18|8|AM|third|night|| +11889|AAAAAAAACHOCAAAA|11889|3|18|9|AM|third|night|| +11890|AAAAAAAADHOCAAAA|11890|3|18|10|AM|third|night|| +11891|AAAAAAAAEHOCAAAA|11891|3|18|11|AM|third|night|| +11892|AAAAAAAAFHOCAAAA|11892|3|18|12|AM|third|night|| +11893|AAAAAAAAGHOCAAAA|11893|3|18|13|AM|third|night|| +11894|AAAAAAAAHHOCAAAA|11894|3|18|14|AM|third|night|| +11895|AAAAAAAAIHOCAAAA|11895|3|18|15|AM|third|night|| +11896|AAAAAAAAJHOCAAAA|11896|3|18|16|AM|third|night|| +11897|AAAAAAAAKHOCAAAA|11897|3|18|17|AM|third|night|| +11898|AAAAAAAALHOCAAAA|11898|3|18|18|AM|third|night|| +11899|AAAAAAAAMHOCAAAA|11899|3|18|19|AM|third|night|| +11900|AAAAAAAANHOCAAAA|11900|3|18|20|AM|third|night|| +11901|AAAAAAAAOHOCAAAA|11901|3|18|21|AM|third|night|| +11902|AAAAAAAAPHOCAAAA|11902|3|18|22|AM|third|night|| +11903|AAAAAAAAAIOCAAAA|11903|3|18|23|AM|third|night|| +11904|AAAAAAAABIOCAAAA|11904|3|18|24|AM|third|night|| +11905|AAAAAAAACIOCAAAA|11905|3|18|25|AM|third|night|| +11906|AAAAAAAADIOCAAAA|11906|3|18|26|AM|third|night|| +11907|AAAAAAAAEIOCAAAA|11907|3|18|27|AM|third|night|| +11908|AAAAAAAAFIOCAAAA|11908|3|18|28|AM|third|night|| +11909|AAAAAAAAGIOCAAAA|11909|3|18|29|AM|third|night|| +11910|AAAAAAAAHIOCAAAA|11910|3|18|30|AM|third|night|| +11911|AAAAAAAAIIOCAAAA|11911|3|18|31|AM|third|night|| +11912|AAAAAAAAJIOCAAAA|11912|3|18|32|AM|third|night|| +11913|AAAAAAAAKIOCAAAA|11913|3|18|33|AM|third|night|| +11914|AAAAAAAALIOCAAAA|11914|3|18|34|AM|third|night|| +11915|AAAAAAAAMIOCAAAA|11915|3|18|35|AM|third|night|| +11916|AAAAAAAANIOCAAAA|11916|3|18|36|AM|third|night|| +11917|AAAAAAAAOIOCAAAA|11917|3|18|37|AM|third|night|| +11918|AAAAAAAAPIOCAAAA|11918|3|18|38|AM|third|night|| +11919|AAAAAAAAAJOCAAAA|11919|3|18|39|AM|third|night|| +11920|AAAAAAAABJOCAAAA|11920|3|18|40|AM|third|night|| +11921|AAAAAAAACJOCAAAA|11921|3|18|41|AM|third|night|| +11922|AAAAAAAADJOCAAAA|11922|3|18|42|AM|third|night|| +11923|AAAAAAAAEJOCAAAA|11923|3|18|43|AM|third|night|| +11924|AAAAAAAAFJOCAAAA|11924|3|18|44|AM|third|night|| +11925|AAAAAAAAGJOCAAAA|11925|3|18|45|AM|third|night|| +11926|AAAAAAAAHJOCAAAA|11926|3|18|46|AM|third|night|| +11927|AAAAAAAAIJOCAAAA|11927|3|18|47|AM|third|night|| +11928|AAAAAAAAJJOCAAAA|11928|3|18|48|AM|third|night|| +11929|AAAAAAAAKJOCAAAA|11929|3|18|49|AM|third|night|| +11930|AAAAAAAALJOCAAAA|11930|3|18|50|AM|third|night|| +11931|AAAAAAAAMJOCAAAA|11931|3|18|51|AM|third|night|| +11932|AAAAAAAANJOCAAAA|11932|3|18|52|AM|third|night|| +11933|AAAAAAAAOJOCAAAA|11933|3|18|53|AM|third|night|| +11934|AAAAAAAAPJOCAAAA|11934|3|18|54|AM|third|night|| +11935|AAAAAAAAAKOCAAAA|11935|3|18|55|AM|third|night|| +11936|AAAAAAAABKOCAAAA|11936|3|18|56|AM|third|night|| +11937|AAAAAAAACKOCAAAA|11937|3|18|57|AM|third|night|| +11938|AAAAAAAADKOCAAAA|11938|3|18|58|AM|third|night|| +11939|AAAAAAAAEKOCAAAA|11939|3|18|59|AM|third|night|| +11940|AAAAAAAAFKOCAAAA|11940|3|19|0|AM|third|night|| +11941|AAAAAAAAGKOCAAAA|11941|3|19|1|AM|third|night|| +11942|AAAAAAAAHKOCAAAA|11942|3|19|2|AM|third|night|| +11943|AAAAAAAAIKOCAAAA|11943|3|19|3|AM|third|night|| +11944|AAAAAAAAJKOCAAAA|11944|3|19|4|AM|third|night|| +11945|AAAAAAAAKKOCAAAA|11945|3|19|5|AM|third|night|| +11946|AAAAAAAALKOCAAAA|11946|3|19|6|AM|third|night|| +11947|AAAAAAAAMKOCAAAA|11947|3|19|7|AM|third|night|| +11948|AAAAAAAANKOCAAAA|11948|3|19|8|AM|third|night|| +11949|AAAAAAAAOKOCAAAA|11949|3|19|9|AM|third|night|| +11950|AAAAAAAAPKOCAAAA|11950|3|19|10|AM|third|night|| +11951|AAAAAAAAALOCAAAA|11951|3|19|11|AM|third|night|| +11952|AAAAAAAABLOCAAAA|11952|3|19|12|AM|third|night|| +11953|AAAAAAAACLOCAAAA|11953|3|19|13|AM|third|night|| +11954|AAAAAAAADLOCAAAA|11954|3|19|14|AM|third|night|| +11955|AAAAAAAAELOCAAAA|11955|3|19|15|AM|third|night|| +11956|AAAAAAAAFLOCAAAA|11956|3|19|16|AM|third|night|| +11957|AAAAAAAAGLOCAAAA|11957|3|19|17|AM|third|night|| +11958|AAAAAAAAHLOCAAAA|11958|3|19|18|AM|third|night|| +11959|AAAAAAAAILOCAAAA|11959|3|19|19|AM|third|night|| +11960|AAAAAAAAJLOCAAAA|11960|3|19|20|AM|third|night|| +11961|AAAAAAAAKLOCAAAA|11961|3|19|21|AM|third|night|| +11962|AAAAAAAALLOCAAAA|11962|3|19|22|AM|third|night|| +11963|AAAAAAAAMLOCAAAA|11963|3|19|23|AM|third|night|| +11964|AAAAAAAANLOCAAAA|11964|3|19|24|AM|third|night|| +11965|AAAAAAAAOLOCAAAA|11965|3|19|25|AM|third|night|| +11966|AAAAAAAAPLOCAAAA|11966|3|19|26|AM|third|night|| +11967|AAAAAAAAAMOCAAAA|11967|3|19|27|AM|third|night|| +11968|AAAAAAAABMOCAAAA|11968|3|19|28|AM|third|night|| +11969|AAAAAAAACMOCAAAA|11969|3|19|29|AM|third|night|| +11970|AAAAAAAADMOCAAAA|11970|3|19|30|AM|third|night|| +11971|AAAAAAAAEMOCAAAA|11971|3|19|31|AM|third|night|| +11972|AAAAAAAAFMOCAAAA|11972|3|19|32|AM|third|night|| +11973|AAAAAAAAGMOCAAAA|11973|3|19|33|AM|third|night|| +11974|AAAAAAAAHMOCAAAA|11974|3|19|34|AM|third|night|| +11975|AAAAAAAAIMOCAAAA|11975|3|19|35|AM|third|night|| +11976|AAAAAAAAJMOCAAAA|11976|3|19|36|AM|third|night|| +11977|AAAAAAAAKMOCAAAA|11977|3|19|37|AM|third|night|| +11978|AAAAAAAALMOCAAAA|11978|3|19|38|AM|third|night|| +11979|AAAAAAAAMMOCAAAA|11979|3|19|39|AM|third|night|| +11980|AAAAAAAANMOCAAAA|11980|3|19|40|AM|third|night|| +11981|AAAAAAAAOMOCAAAA|11981|3|19|41|AM|third|night|| +11982|AAAAAAAAPMOCAAAA|11982|3|19|42|AM|third|night|| +11983|AAAAAAAAANOCAAAA|11983|3|19|43|AM|third|night|| +11984|AAAAAAAABNOCAAAA|11984|3|19|44|AM|third|night|| +11985|AAAAAAAACNOCAAAA|11985|3|19|45|AM|third|night|| +11986|AAAAAAAADNOCAAAA|11986|3|19|46|AM|third|night|| +11987|AAAAAAAAENOCAAAA|11987|3|19|47|AM|third|night|| +11988|AAAAAAAAFNOCAAAA|11988|3|19|48|AM|third|night|| +11989|AAAAAAAAGNOCAAAA|11989|3|19|49|AM|third|night|| +11990|AAAAAAAAHNOCAAAA|11990|3|19|50|AM|third|night|| +11991|AAAAAAAAINOCAAAA|11991|3|19|51|AM|third|night|| +11992|AAAAAAAAJNOCAAAA|11992|3|19|52|AM|third|night|| +11993|AAAAAAAAKNOCAAAA|11993|3|19|53|AM|third|night|| +11994|AAAAAAAALNOCAAAA|11994|3|19|54|AM|third|night|| +11995|AAAAAAAAMNOCAAAA|11995|3|19|55|AM|third|night|| +11996|AAAAAAAANNOCAAAA|11996|3|19|56|AM|third|night|| +11997|AAAAAAAAONOCAAAA|11997|3|19|57|AM|third|night|| +11998|AAAAAAAAPNOCAAAA|11998|3|19|58|AM|third|night|| +11999|AAAAAAAAAOOCAAAA|11999|3|19|59|AM|third|night|| +12000|AAAAAAAABOOCAAAA|12000|3|20|0|AM|third|night|| +12001|AAAAAAAACOOCAAAA|12001|3|20|1|AM|third|night|| +12002|AAAAAAAADOOCAAAA|12002|3|20|2|AM|third|night|| +12003|AAAAAAAAEOOCAAAA|12003|3|20|3|AM|third|night|| +12004|AAAAAAAAFOOCAAAA|12004|3|20|4|AM|third|night|| +12005|AAAAAAAAGOOCAAAA|12005|3|20|5|AM|third|night|| +12006|AAAAAAAAHOOCAAAA|12006|3|20|6|AM|third|night|| +12007|AAAAAAAAIOOCAAAA|12007|3|20|7|AM|third|night|| +12008|AAAAAAAAJOOCAAAA|12008|3|20|8|AM|third|night|| +12009|AAAAAAAAKOOCAAAA|12009|3|20|9|AM|third|night|| +12010|AAAAAAAALOOCAAAA|12010|3|20|10|AM|third|night|| +12011|AAAAAAAAMOOCAAAA|12011|3|20|11|AM|third|night|| +12012|AAAAAAAANOOCAAAA|12012|3|20|12|AM|third|night|| +12013|AAAAAAAAOOOCAAAA|12013|3|20|13|AM|third|night|| +12014|AAAAAAAAPOOCAAAA|12014|3|20|14|AM|third|night|| +12015|AAAAAAAAAPOCAAAA|12015|3|20|15|AM|third|night|| +12016|AAAAAAAABPOCAAAA|12016|3|20|16|AM|third|night|| +12017|AAAAAAAACPOCAAAA|12017|3|20|17|AM|third|night|| +12018|AAAAAAAADPOCAAAA|12018|3|20|18|AM|third|night|| +12019|AAAAAAAAEPOCAAAA|12019|3|20|19|AM|third|night|| +12020|AAAAAAAAFPOCAAAA|12020|3|20|20|AM|third|night|| +12021|AAAAAAAAGPOCAAAA|12021|3|20|21|AM|third|night|| +12022|AAAAAAAAHPOCAAAA|12022|3|20|22|AM|third|night|| +12023|AAAAAAAAIPOCAAAA|12023|3|20|23|AM|third|night|| +12024|AAAAAAAAJPOCAAAA|12024|3|20|24|AM|third|night|| +12025|AAAAAAAAKPOCAAAA|12025|3|20|25|AM|third|night|| +12026|AAAAAAAALPOCAAAA|12026|3|20|26|AM|third|night|| +12027|AAAAAAAAMPOCAAAA|12027|3|20|27|AM|third|night|| +12028|AAAAAAAANPOCAAAA|12028|3|20|28|AM|third|night|| +12029|AAAAAAAAOPOCAAAA|12029|3|20|29|AM|third|night|| +12030|AAAAAAAAPPOCAAAA|12030|3|20|30|AM|third|night|| +12031|AAAAAAAAAAPCAAAA|12031|3|20|31|AM|third|night|| +12032|AAAAAAAABAPCAAAA|12032|3|20|32|AM|third|night|| +12033|AAAAAAAACAPCAAAA|12033|3|20|33|AM|third|night|| +12034|AAAAAAAADAPCAAAA|12034|3|20|34|AM|third|night|| +12035|AAAAAAAAEAPCAAAA|12035|3|20|35|AM|third|night|| +12036|AAAAAAAAFAPCAAAA|12036|3|20|36|AM|third|night|| +12037|AAAAAAAAGAPCAAAA|12037|3|20|37|AM|third|night|| +12038|AAAAAAAAHAPCAAAA|12038|3|20|38|AM|third|night|| +12039|AAAAAAAAIAPCAAAA|12039|3|20|39|AM|third|night|| +12040|AAAAAAAAJAPCAAAA|12040|3|20|40|AM|third|night|| +12041|AAAAAAAAKAPCAAAA|12041|3|20|41|AM|third|night|| +12042|AAAAAAAALAPCAAAA|12042|3|20|42|AM|third|night|| +12043|AAAAAAAAMAPCAAAA|12043|3|20|43|AM|third|night|| +12044|AAAAAAAANAPCAAAA|12044|3|20|44|AM|third|night|| +12045|AAAAAAAAOAPCAAAA|12045|3|20|45|AM|third|night|| +12046|AAAAAAAAPAPCAAAA|12046|3|20|46|AM|third|night|| +12047|AAAAAAAAABPCAAAA|12047|3|20|47|AM|third|night|| +12048|AAAAAAAABBPCAAAA|12048|3|20|48|AM|third|night|| +12049|AAAAAAAACBPCAAAA|12049|3|20|49|AM|third|night|| +12050|AAAAAAAADBPCAAAA|12050|3|20|50|AM|third|night|| +12051|AAAAAAAAEBPCAAAA|12051|3|20|51|AM|third|night|| +12052|AAAAAAAAFBPCAAAA|12052|3|20|52|AM|third|night|| +12053|AAAAAAAAGBPCAAAA|12053|3|20|53|AM|third|night|| +12054|AAAAAAAAHBPCAAAA|12054|3|20|54|AM|third|night|| +12055|AAAAAAAAIBPCAAAA|12055|3|20|55|AM|third|night|| +12056|AAAAAAAAJBPCAAAA|12056|3|20|56|AM|third|night|| +12057|AAAAAAAAKBPCAAAA|12057|3|20|57|AM|third|night|| +12058|AAAAAAAALBPCAAAA|12058|3|20|58|AM|third|night|| +12059|AAAAAAAAMBPCAAAA|12059|3|20|59|AM|third|night|| +12060|AAAAAAAANBPCAAAA|12060|3|21|0|AM|third|night|| +12061|AAAAAAAAOBPCAAAA|12061|3|21|1|AM|third|night|| +12062|AAAAAAAAPBPCAAAA|12062|3|21|2|AM|third|night|| +12063|AAAAAAAAACPCAAAA|12063|3|21|3|AM|third|night|| +12064|AAAAAAAABCPCAAAA|12064|3|21|4|AM|third|night|| +12065|AAAAAAAACCPCAAAA|12065|3|21|5|AM|third|night|| +12066|AAAAAAAADCPCAAAA|12066|3|21|6|AM|third|night|| +12067|AAAAAAAAECPCAAAA|12067|3|21|7|AM|third|night|| +12068|AAAAAAAAFCPCAAAA|12068|3|21|8|AM|third|night|| +12069|AAAAAAAAGCPCAAAA|12069|3|21|9|AM|third|night|| +12070|AAAAAAAAHCPCAAAA|12070|3|21|10|AM|third|night|| +12071|AAAAAAAAICPCAAAA|12071|3|21|11|AM|third|night|| +12072|AAAAAAAAJCPCAAAA|12072|3|21|12|AM|third|night|| +12073|AAAAAAAAKCPCAAAA|12073|3|21|13|AM|third|night|| +12074|AAAAAAAALCPCAAAA|12074|3|21|14|AM|third|night|| +12075|AAAAAAAAMCPCAAAA|12075|3|21|15|AM|third|night|| +12076|AAAAAAAANCPCAAAA|12076|3|21|16|AM|third|night|| +12077|AAAAAAAAOCPCAAAA|12077|3|21|17|AM|third|night|| +12078|AAAAAAAAPCPCAAAA|12078|3|21|18|AM|third|night|| +12079|AAAAAAAAADPCAAAA|12079|3|21|19|AM|third|night|| +12080|AAAAAAAABDPCAAAA|12080|3|21|20|AM|third|night|| +12081|AAAAAAAACDPCAAAA|12081|3|21|21|AM|third|night|| +12082|AAAAAAAADDPCAAAA|12082|3|21|22|AM|third|night|| +12083|AAAAAAAAEDPCAAAA|12083|3|21|23|AM|third|night|| +12084|AAAAAAAAFDPCAAAA|12084|3|21|24|AM|third|night|| +12085|AAAAAAAAGDPCAAAA|12085|3|21|25|AM|third|night|| +12086|AAAAAAAAHDPCAAAA|12086|3|21|26|AM|third|night|| +12087|AAAAAAAAIDPCAAAA|12087|3|21|27|AM|third|night|| +12088|AAAAAAAAJDPCAAAA|12088|3|21|28|AM|third|night|| +12089|AAAAAAAAKDPCAAAA|12089|3|21|29|AM|third|night|| +12090|AAAAAAAALDPCAAAA|12090|3|21|30|AM|third|night|| +12091|AAAAAAAAMDPCAAAA|12091|3|21|31|AM|third|night|| +12092|AAAAAAAANDPCAAAA|12092|3|21|32|AM|third|night|| +12093|AAAAAAAAODPCAAAA|12093|3|21|33|AM|third|night|| +12094|AAAAAAAAPDPCAAAA|12094|3|21|34|AM|third|night|| +12095|AAAAAAAAAEPCAAAA|12095|3|21|35|AM|third|night|| +12096|AAAAAAAABEPCAAAA|12096|3|21|36|AM|third|night|| +12097|AAAAAAAACEPCAAAA|12097|3|21|37|AM|third|night|| +12098|AAAAAAAADEPCAAAA|12098|3|21|38|AM|third|night|| +12099|AAAAAAAAEEPCAAAA|12099|3|21|39|AM|third|night|| +12100|AAAAAAAAFEPCAAAA|12100|3|21|40|AM|third|night|| +12101|AAAAAAAAGEPCAAAA|12101|3|21|41|AM|third|night|| +12102|AAAAAAAAHEPCAAAA|12102|3|21|42|AM|third|night|| +12103|AAAAAAAAIEPCAAAA|12103|3|21|43|AM|third|night|| +12104|AAAAAAAAJEPCAAAA|12104|3|21|44|AM|third|night|| +12105|AAAAAAAAKEPCAAAA|12105|3|21|45|AM|third|night|| +12106|AAAAAAAALEPCAAAA|12106|3|21|46|AM|third|night|| +12107|AAAAAAAAMEPCAAAA|12107|3|21|47|AM|third|night|| +12108|AAAAAAAANEPCAAAA|12108|3|21|48|AM|third|night|| +12109|AAAAAAAAOEPCAAAA|12109|3|21|49|AM|third|night|| +12110|AAAAAAAAPEPCAAAA|12110|3|21|50|AM|third|night|| +12111|AAAAAAAAAFPCAAAA|12111|3|21|51|AM|third|night|| +12112|AAAAAAAABFPCAAAA|12112|3|21|52|AM|third|night|| +12113|AAAAAAAACFPCAAAA|12113|3|21|53|AM|third|night|| +12114|AAAAAAAADFPCAAAA|12114|3|21|54|AM|third|night|| +12115|AAAAAAAAEFPCAAAA|12115|3|21|55|AM|third|night|| +12116|AAAAAAAAFFPCAAAA|12116|3|21|56|AM|third|night|| +12117|AAAAAAAAGFPCAAAA|12117|3|21|57|AM|third|night|| +12118|AAAAAAAAHFPCAAAA|12118|3|21|58|AM|third|night|| +12119|AAAAAAAAIFPCAAAA|12119|3|21|59|AM|third|night|| +12120|AAAAAAAAJFPCAAAA|12120|3|22|0|AM|third|night|| +12121|AAAAAAAAKFPCAAAA|12121|3|22|1|AM|third|night|| +12122|AAAAAAAALFPCAAAA|12122|3|22|2|AM|third|night|| +12123|AAAAAAAAMFPCAAAA|12123|3|22|3|AM|third|night|| +12124|AAAAAAAANFPCAAAA|12124|3|22|4|AM|third|night|| +12125|AAAAAAAAOFPCAAAA|12125|3|22|5|AM|third|night|| +12126|AAAAAAAAPFPCAAAA|12126|3|22|6|AM|third|night|| +12127|AAAAAAAAAGPCAAAA|12127|3|22|7|AM|third|night|| +12128|AAAAAAAABGPCAAAA|12128|3|22|8|AM|third|night|| +12129|AAAAAAAACGPCAAAA|12129|3|22|9|AM|third|night|| +12130|AAAAAAAADGPCAAAA|12130|3|22|10|AM|third|night|| +12131|AAAAAAAAEGPCAAAA|12131|3|22|11|AM|third|night|| +12132|AAAAAAAAFGPCAAAA|12132|3|22|12|AM|third|night|| +12133|AAAAAAAAGGPCAAAA|12133|3|22|13|AM|third|night|| +12134|AAAAAAAAHGPCAAAA|12134|3|22|14|AM|third|night|| +12135|AAAAAAAAIGPCAAAA|12135|3|22|15|AM|third|night|| +12136|AAAAAAAAJGPCAAAA|12136|3|22|16|AM|third|night|| +12137|AAAAAAAAKGPCAAAA|12137|3|22|17|AM|third|night|| +12138|AAAAAAAALGPCAAAA|12138|3|22|18|AM|third|night|| +12139|AAAAAAAAMGPCAAAA|12139|3|22|19|AM|third|night|| +12140|AAAAAAAANGPCAAAA|12140|3|22|20|AM|third|night|| +12141|AAAAAAAAOGPCAAAA|12141|3|22|21|AM|third|night|| +12142|AAAAAAAAPGPCAAAA|12142|3|22|22|AM|third|night|| +12143|AAAAAAAAAHPCAAAA|12143|3|22|23|AM|third|night|| +12144|AAAAAAAABHPCAAAA|12144|3|22|24|AM|third|night|| +12145|AAAAAAAACHPCAAAA|12145|3|22|25|AM|third|night|| +12146|AAAAAAAADHPCAAAA|12146|3|22|26|AM|third|night|| +12147|AAAAAAAAEHPCAAAA|12147|3|22|27|AM|third|night|| +12148|AAAAAAAAFHPCAAAA|12148|3|22|28|AM|third|night|| +12149|AAAAAAAAGHPCAAAA|12149|3|22|29|AM|third|night|| +12150|AAAAAAAAHHPCAAAA|12150|3|22|30|AM|third|night|| +12151|AAAAAAAAIHPCAAAA|12151|3|22|31|AM|third|night|| +12152|AAAAAAAAJHPCAAAA|12152|3|22|32|AM|third|night|| +12153|AAAAAAAAKHPCAAAA|12153|3|22|33|AM|third|night|| +12154|AAAAAAAALHPCAAAA|12154|3|22|34|AM|third|night|| +12155|AAAAAAAAMHPCAAAA|12155|3|22|35|AM|third|night|| +12156|AAAAAAAANHPCAAAA|12156|3|22|36|AM|third|night|| +12157|AAAAAAAAOHPCAAAA|12157|3|22|37|AM|third|night|| +12158|AAAAAAAAPHPCAAAA|12158|3|22|38|AM|third|night|| +12159|AAAAAAAAAIPCAAAA|12159|3|22|39|AM|third|night|| +12160|AAAAAAAABIPCAAAA|12160|3|22|40|AM|third|night|| +12161|AAAAAAAACIPCAAAA|12161|3|22|41|AM|third|night|| +12162|AAAAAAAADIPCAAAA|12162|3|22|42|AM|third|night|| +12163|AAAAAAAAEIPCAAAA|12163|3|22|43|AM|third|night|| +12164|AAAAAAAAFIPCAAAA|12164|3|22|44|AM|third|night|| +12165|AAAAAAAAGIPCAAAA|12165|3|22|45|AM|third|night|| +12166|AAAAAAAAHIPCAAAA|12166|3|22|46|AM|third|night|| +12167|AAAAAAAAIIPCAAAA|12167|3|22|47|AM|third|night|| +12168|AAAAAAAAJIPCAAAA|12168|3|22|48|AM|third|night|| +12169|AAAAAAAAKIPCAAAA|12169|3|22|49|AM|third|night|| +12170|AAAAAAAALIPCAAAA|12170|3|22|50|AM|third|night|| +12171|AAAAAAAAMIPCAAAA|12171|3|22|51|AM|third|night|| +12172|AAAAAAAANIPCAAAA|12172|3|22|52|AM|third|night|| +12173|AAAAAAAAOIPCAAAA|12173|3|22|53|AM|third|night|| +12174|AAAAAAAAPIPCAAAA|12174|3|22|54|AM|third|night|| +12175|AAAAAAAAAJPCAAAA|12175|3|22|55|AM|third|night|| +12176|AAAAAAAABJPCAAAA|12176|3|22|56|AM|third|night|| +12177|AAAAAAAACJPCAAAA|12177|3|22|57|AM|third|night|| +12178|AAAAAAAADJPCAAAA|12178|3|22|58|AM|third|night|| +12179|AAAAAAAAEJPCAAAA|12179|3|22|59|AM|third|night|| +12180|AAAAAAAAFJPCAAAA|12180|3|23|0|AM|third|night|| +12181|AAAAAAAAGJPCAAAA|12181|3|23|1|AM|third|night|| +12182|AAAAAAAAHJPCAAAA|12182|3|23|2|AM|third|night|| +12183|AAAAAAAAIJPCAAAA|12183|3|23|3|AM|third|night|| +12184|AAAAAAAAJJPCAAAA|12184|3|23|4|AM|third|night|| +12185|AAAAAAAAKJPCAAAA|12185|3|23|5|AM|third|night|| +12186|AAAAAAAALJPCAAAA|12186|3|23|6|AM|third|night|| +12187|AAAAAAAAMJPCAAAA|12187|3|23|7|AM|third|night|| +12188|AAAAAAAANJPCAAAA|12188|3|23|8|AM|third|night|| +12189|AAAAAAAAOJPCAAAA|12189|3|23|9|AM|third|night|| +12190|AAAAAAAAPJPCAAAA|12190|3|23|10|AM|third|night|| +12191|AAAAAAAAAKPCAAAA|12191|3|23|11|AM|third|night|| +12192|AAAAAAAABKPCAAAA|12192|3|23|12|AM|third|night|| +12193|AAAAAAAACKPCAAAA|12193|3|23|13|AM|third|night|| +12194|AAAAAAAADKPCAAAA|12194|3|23|14|AM|third|night|| +12195|AAAAAAAAEKPCAAAA|12195|3|23|15|AM|third|night|| +12196|AAAAAAAAFKPCAAAA|12196|3|23|16|AM|third|night|| +12197|AAAAAAAAGKPCAAAA|12197|3|23|17|AM|third|night|| +12198|AAAAAAAAHKPCAAAA|12198|3|23|18|AM|third|night|| +12199|AAAAAAAAIKPCAAAA|12199|3|23|19|AM|third|night|| +12200|AAAAAAAAJKPCAAAA|12200|3|23|20|AM|third|night|| +12201|AAAAAAAAKKPCAAAA|12201|3|23|21|AM|third|night|| +12202|AAAAAAAALKPCAAAA|12202|3|23|22|AM|third|night|| +12203|AAAAAAAAMKPCAAAA|12203|3|23|23|AM|third|night|| +12204|AAAAAAAANKPCAAAA|12204|3|23|24|AM|third|night|| +12205|AAAAAAAAOKPCAAAA|12205|3|23|25|AM|third|night|| +12206|AAAAAAAAPKPCAAAA|12206|3|23|26|AM|third|night|| +12207|AAAAAAAAALPCAAAA|12207|3|23|27|AM|third|night|| +12208|AAAAAAAABLPCAAAA|12208|3|23|28|AM|third|night|| +12209|AAAAAAAACLPCAAAA|12209|3|23|29|AM|third|night|| +12210|AAAAAAAADLPCAAAA|12210|3|23|30|AM|third|night|| +12211|AAAAAAAAELPCAAAA|12211|3|23|31|AM|third|night|| +12212|AAAAAAAAFLPCAAAA|12212|3|23|32|AM|third|night|| +12213|AAAAAAAAGLPCAAAA|12213|3|23|33|AM|third|night|| +12214|AAAAAAAAHLPCAAAA|12214|3|23|34|AM|third|night|| +12215|AAAAAAAAILPCAAAA|12215|3|23|35|AM|third|night|| +12216|AAAAAAAAJLPCAAAA|12216|3|23|36|AM|third|night|| +12217|AAAAAAAAKLPCAAAA|12217|3|23|37|AM|third|night|| +12218|AAAAAAAALLPCAAAA|12218|3|23|38|AM|third|night|| +12219|AAAAAAAAMLPCAAAA|12219|3|23|39|AM|third|night|| +12220|AAAAAAAANLPCAAAA|12220|3|23|40|AM|third|night|| +12221|AAAAAAAAOLPCAAAA|12221|3|23|41|AM|third|night|| +12222|AAAAAAAAPLPCAAAA|12222|3|23|42|AM|third|night|| +12223|AAAAAAAAAMPCAAAA|12223|3|23|43|AM|third|night|| +12224|AAAAAAAABMPCAAAA|12224|3|23|44|AM|third|night|| +12225|AAAAAAAACMPCAAAA|12225|3|23|45|AM|third|night|| +12226|AAAAAAAADMPCAAAA|12226|3|23|46|AM|third|night|| +12227|AAAAAAAAEMPCAAAA|12227|3|23|47|AM|third|night|| +12228|AAAAAAAAFMPCAAAA|12228|3|23|48|AM|third|night|| +12229|AAAAAAAAGMPCAAAA|12229|3|23|49|AM|third|night|| +12230|AAAAAAAAHMPCAAAA|12230|3|23|50|AM|third|night|| +12231|AAAAAAAAIMPCAAAA|12231|3|23|51|AM|third|night|| +12232|AAAAAAAAJMPCAAAA|12232|3|23|52|AM|third|night|| +12233|AAAAAAAAKMPCAAAA|12233|3|23|53|AM|third|night|| +12234|AAAAAAAALMPCAAAA|12234|3|23|54|AM|third|night|| +12235|AAAAAAAAMMPCAAAA|12235|3|23|55|AM|third|night|| +12236|AAAAAAAANMPCAAAA|12236|3|23|56|AM|third|night|| +12237|AAAAAAAAOMPCAAAA|12237|3|23|57|AM|third|night|| +12238|AAAAAAAAPMPCAAAA|12238|3|23|58|AM|third|night|| +12239|AAAAAAAAANPCAAAA|12239|3|23|59|AM|third|night|| +12240|AAAAAAAABNPCAAAA|12240|3|24|0|AM|third|night|| +12241|AAAAAAAACNPCAAAA|12241|3|24|1|AM|third|night|| +12242|AAAAAAAADNPCAAAA|12242|3|24|2|AM|third|night|| +12243|AAAAAAAAENPCAAAA|12243|3|24|3|AM|third|night|| +12244|AAAAAAAAFNPCAAAA|12244|3|24|4|AM|third|night|| +12245|AAAAAAAAGNPCAAAA|12245|3|24|5|AM|third|night|| +12246|AAAAAAAAHNPCAAAA|12246|3|24|6|AM|third|night|| +12247|AAAAAAAAINPCAAAA|12247|3|24|7|AM|third|night|| +12248|AAAAAAAAJNPCAAAA|12248|3|24|8|AM|third|night|| +12249|AAAAAAAAKNPCAAAA|12249|3|24|9|AM|third|night|| +12250|AAAAAAAALNPCAAAA|12250|3|24|10|AM|third|night|| +12251|AAAAAAAAMNPCAAAA|12251|3|24|11|AM|third|night|| +12252|AAAAAAAANNPCAAAA|12252|3|24|12|AM|third|night|| +12253|AAAAAAAAONPCAAAA|12253|3|24|13|AM|third|night|| +12254|AAAAAAAAPNPCAAAA|12254|3|24|14|AM|third|night|| +12255|AAAAAAAAAOPCAAAA|12255|3|24|15|AM|third|night|| +12256|AAAAAAAABOPCAAAA|12256|3|24|16|AM|third|night|| +12257|AAAAAAAACOPCAAAA|12257|3|24|17|AM|third|night|| +12258|AAAAAAAADOPCAAAA|12258|3|24|18|AM|third|night|| +12259|AAAAAAAAEOPCAAAA|12259|3|24|19|AM|third|night|| +12260|AAAAAAAAFOPCAAAA|12260|3|24|20|AM|third|night|| +12261|AAAAAAAAGOPCAAAA|12261|3|24|21|AM|third|night|| +12262|AAAAAAAAHOPCAAAA|12262|3|24|22|AM|third|night|| +12263|AAAAAAAAIOPCAAAA|12263|3|24|23|AM|third|night|| +12264|AAAAAAAAJOPCAAAA|12264|3|24|24|AM|third|night|| +12265|AAAAAAAAKOPCAAAA|12265|3|24|25|AM|third|night|| +12266|AAAAAAAALOPCAAAA|12266|3|24|26|AM|third|night|| +12267|AAAAAAAAMOPCAAAA|12267|3|24|27|AM|third|night|| +12268|AAAAAAAANOPCAAAA|12268|3|24|28|AM|third|night|| +12269|AAAAAAAAOOPCAAAA|12269|3|24|29|AM|third|night|| +12270|AAAAAAAAPOPCAAAA|12270|3|24|30|AM|third|night|| +12271|AAAAAAAAAPPCAAAA|12271|3|24|31|AM|third|night|| +12272|AAAAAAAABPPCAAAA|12272|3|24|32|AM|third|night|| +12273|AAAAAAAACPPCAAAA|12273|3|24|33|AM|third|night|| +12274|AAAAAAAADPPCAAAA|12274|3|24|34|AM|third|night|| +12275|AAAAAAAAEPPCAAAA|12275|3|24|35|AM|third|night|| +12276|AAAAAAAAFPPCAAAA|12276|3|24|36|AM|third|night|| +12277|AAAAAAAAGPPCAAAA|12277|3|24|37|AM|third|night|| +12278|AAAAAAAAHPPCAAAA|12278|3|24|38|AM|third|night|| +12279|AAAAAAAAIPPCAAAA|12279|3|24|39|AM|third|night|| +12280|AAAAAAAAJPPCAAAA|12280|3|24|40|AM|third|night|| +12281|AAAAAAAAKPPCAAAA|12281|3|24|41|AM|third|night|| +12282|AAAAAAAALPPCAAAA|12282|3|24|42|AM|third|night|| +12283|AAAAAAAAMPPCAAAA|12283|3|24|43|AM|third|night|| +12284|AAAAAAAANPPCAAAA|12284|3|24|44|AM|third|night|| +12285|AAAAAAAAOPPCAAAA|12285|3|24|45|AM|third|night|| +12286|AAAAAAAAPPPCAAAA|12286|3|24|46|AM|third|night|| +12287|AAAAAAAAAAADAAAA|12287|3|24|47|AM|third|night|| +12288|AAAAAAAABAADAAAA|12288|3|24|48|AM|third|night|| +12289|AAAAAAAACAADAAAA|12289|3|24|49|AM|third|night|| +12290|AAAAAAAADAADAAAA|12290|3|24|50|AM|third|night|| +12291|AAAAAAAAEAADAAAA|12291|3|24|51|AM|third|night|| +12292|AAAAAAAAFAADAAAA|12292|3|24|52|AM|third|night|| +12293|AAAAAAAAGAADAAAA|12293|3|24|53|AM|third|night|| +12294|AAAAAAAAHAADAAAA|12294|3|24|54|AM|third|night|| +12295|AAAAAAAAIAADAAAA|12295|3|24|55|AM|third|night|| +12296|AAAAAAAAJAADAAAA|12296|3|24|56|AM|third|night|| +12297|AAAAAAAAKAADAAAA|12297|3|24|57|AM|third|night|| +12298|AAAAAAAALAADAAAA|12298|3|24|58|AM|third|night|| +12299|AAAAAAAAMAADAAAA|12299|3|24|59|AM|third|night|| +12300|AAAAAAAANAADAAAA|12300|3|25|0|AM|third|night|| +12301|AAAAAAAAOAADAAAA|12301|3|25|1|AM|third|night|| +12302|AAAAAAAAPAADAAAA|12302|3|25|2|AM|third|night|| +12303|AAAAAAAAABADAAAA|12303|3|25|3|AM|third|night|| +12304|AAAAAAAABBADAAAA|12304|3|25|4|AM|third|night|| +12305|AAAAAAAACBADAAAA|12305|3|25|5|AM|third|night|| +12306|AAAAAAAADBADAAAA|12306|3|25|6|AM|third|night|| +12307|AAAAAAAAEBADAAAA|12307|3|25|7|AM|third|night|| +12308|AAAAAAAAFBADAAAA|12308|3|25|8|AM|third|night|| +12309|AAAAAAAAGBADAAAA|12309|3|25|9|AM|third|night|| +12310|AAAAAAAAHBADAAAA|12310|3|25|10|AM|third|night|| +12311|AAAAAAAAIBADAAAA|12311|3|25|11|AM|third|night|| +12312|AAAAAAAAJBADAAAA|12312|3|25|12|AM|third|night|| +12313|AAAAAAAAKBADAAAA|12313|3|25|13|AM|third|night|| +12314|AAAAAAAALBADAAAA|12314|3|25|14|AM|third|night|| +12315|AAAAAAAAMBADAAAA|12315|3|25|15|AM|third|night|| +12316|AAAAAAAANBADAAAA|12316|3|25|16|AM|third|night|| +12317|AAAAAAAAOBADAAAA|12317|3|25|17|AM|third|night|| +12318|AAAAAAAAPBADAAAA|12318|3|25|18|AM|third|night|| +12319|AAAAAAAAACADAAAA|12319|3|25|19|AM|third|night|| +12320|AAAAAAAABCADAAAA|12320|3|25|20|AM|third|night|| +12321|AAAAAAAACCADAAAA|12321|3|25|21|AM|third|night|| +12322|AAAAAAAADCADAAAA|12322|3|25|22|AM|third|night|| +12323|AAAAAAAAECADAAAA|12323|3|25|23|AM|third|night|| +12324|AAAAAAAAFCADAAAA|12324|3|25|24|AM|third|night|| +12325|AAAAAAAAGCADAAAA|12325|3|25|25|AM|third|night|| +12326|AAAAAAAAHCADAAAA|12326|3|25|26|AM|third|night|| +12327|AAAAAAAAICADAAAA|12327|3|25|27|AM|third|night|| +12328|AAAAAAAAJCADAAAA|12328|3|25|28|AM|third|night|| +12329|AAAAAAAAKCADAAAA|12329|3|25|29|AM|third|night|| +12330|AAAAAAAALCADAAAA|12330|3|25|30|AM|third|night|| +12331|AAAAAAAAMCADAAAA|12331|3|25|31|AM|third|night|| +12332|AAAAAAAANCADAAAA|12332|3|25|32|AM|third|night|| +12333|AAAAAAAAOCADAAAA|12333|3|25|33|AM|third|night|| +12334|AAAAAAAAPCADAAAA|12334|3|25|34|AM|third|night|| +12335|AAAAAAAAADADAAAA|12335|3|25|35|AM|third|night|| +12336|AAAAAAAABDADAAAA|12336|3|25|36|AM|third|night|| +12337|AAAAAAAACDADAAAA|12337|3|25|37|AM|third|night|| +12338|AAAAAAAADDADAAAA|12338|3|25|38|AM|third|night|| +12339|AAAAAAAAEDADAAAA|12339|3|25|39|AM|third|night|| +12340|AAAAAAAAFDADAAAA|12340|3|25|40|AM|third|night|| +12341|AAAAAAAAGDADAAAA|12341|3|25|41|AM|third|night|| +12342|AAAAAAAAHDADAAAA|12342|3|25|42|AM|third|night|| +12343|AAAAAAAAIDADAAAA|12343|3|25|43|AM|third|night|| +12344|AAAAAAAAJDADAAAA|12344|3|25|44|AM|third|night|| +12345|AAAAAAAAKDADAAAA|12345|3|25|45|AM|third|night|| +12346|AAAAAAAALDADAAAA|12346|3|25|46|AM|third|night|| +12347|AAAAAAAAMDADAAAA|12347|3|25|47|AM|third|night|| +12348|AAAAAAAANDADAAAA|12348|3|25|48|AM|third|night|| +12349|AAAAAAAAODADAAAA|12349|3|25|49|AM|third|night|| +12350|AAAAAAAAPDADAAAA|12350|3|25|50|AM|third|night|| +12351|AAAAAAAAAEADAAAA|12351|3|25|51|AM|third|night|| +12352|AAAAAAAABEADAAAA|12352|3|25|52|AM|third|night|| +12353|AAAAAAAACEADAAAA|12353|3|25|53|AM|third|night|| +12354|AAAAAAAADEADAAAA|12354|3|25|54|AM|third|night|| +12355|AAAAAAAAEEADAAAA|12355|3|25|55|AM|third|night|| +12356|AAAAAAAAFEADAAAA|12356|3|25|56|AM|third|night|| +12357|AAAAAAAAGEADAAAA|12357|3|25|57|AM|third|night|| +12358|AAAAAAAAHEADAAAA|12358|3|25|58|AM|third|night|| +12359|AAAAAAAAIEADAAAA|12359|3|25|59|AM|third|night|| +12360|AAAAAAAAJEADAAAA|12360|3|26|0|AM|third|night|| +12361|AAAAAAAAKEADAAAA|12361|3|26|1|AM|third|night|| +12362|AAAAAAAALEADAAAA|12362|3|26|2|AM|third|night|| +12363|AAAAAAAAMEADAAAA|12363|3|26|3|AM|third|night|| +12364|AAAAAAAANEADAAAA|12364|3|26|4|AM|third|night|| +12365|AAAAAAAAOEADAAAA|12365|3|26|5|AM|third|night|| +12366|AAAAAAAAPEADAAAA|12366|3|26|6|AM|third|night|| +12367|AAAAAAAAAFADAAAA|12367|3|26|7|AM|third|night|| +12368|AAAAAAAABFADAAAA|12368|3|26|8|AM|third|night|| +12369|AAAAAAAACFADAAAA|12369|3|26|9|AM|third|night|| +12370|AAAAAAAADFADAAAA|12370|3|26|10|AM|third|night|| +12371|AAAAAAAAEFADAAAA|12371|3|26|11|AM|third|night|| +12372|AAAAAAAAFFADAAAA|12372|3|26|12|AM|third|night|| +12373|AAAAAAAAGFADAAAA|12373|3|26|13|AM|third|night|| +12374|AAAAAAAAHFADAAAA|12374|3|26|14|AM|third|night|| +12375|AAAAAAAAIFADAAAA|12375|3|26|15|AM|third|night|| +12376|AAAAAAAAJFADAAAA|12376|3|26|16|AM|third|night|| +12377|AAAAAAAAKFADAAAA|12377|3|26|17|AM|third|night|| +12378|AAAAAAAALFADAAAA|12378|3|26|18|AM|third|night|| +12379|AAAAAAAAMFADAAAA|12379|3|26|19|AM|third|night|| +12380|AAAAAAAANFADAAAA|12380|3|26|20|AM|third|night|| +12381|AAAAAAAAOFADAAAA|12381|3|26|21|AM|third|night|| +12382|AAAAAAAAPFADAAAA|12382|3|26|22|AM|third|night|| +12383|AAAAAAAAAGADAAAA|12383|3|26|23|AM|third|night|| +12384|AAAAAAAABGADAAAA|12384|3|26|24|AM|third|night|| +12385|AAAAAAAACGADAAAA|12385|3|26|25|AM|third|night|| +12386|AAAAAAAADGADAAAA|12386|3|26|26|AM|third|night|| +12387|AAAAAAAAEGADAAAA|12387|3|26|27|AM|third|night|| +12388|AAAAAAAAFGADAAAA|12388|3|26|28|AM|third|night|| +12389|AAAAAAAAGGADAAAA|12389|3|26|29|AM|third|night|| +12390|AAAAAAAAHGADAAAA|12390|3|26|30|AM|third|night|| +12391|AAAAAAAAIGADAAAA|12391|3|26|31|AM|third|night|| +12392|AAAAAAAAJGADAAAA|12392|3|26|32|AM|third|night|| +12393|AAAAAAAAKGADAAAA|12393|3|26|33|AM|third|night|| +12394|AAAAAAAALGADAAAA|12394|3|26|34|AM|third|night|| +12395|AAAAAAAAMGADAAAA|12395|3|26|35|AM|third|night|| +12396|AAAAAAAANGADAAAA|12396|3|26|36|AM|third|night|| +12397|AAAAAAAAOGADAAAA|12397|3|26|37|AM|third|night|| +12398|AAAAAAAAPGADAAAA|12398|3|26|38|AM|third|night|| +12399|AAAAAAAAAHADAAAA|12399|3|26|39|AM|third|night|| +12400|AAAAAAAABHADAAAA|12400|3|26|40|AM|third|night|| +12401|AAAAAAAACHADAAAA|12401|3|26|41|AM|third|night|| +12402|AAAAAAAADHADAAAA|12402|3|26|42|AM|third|night|| +12403|AAAAAAAAEHADAAAA|12403|3|26|43|AM|third|night|| +12404|AAAAAAAAFHADAAAA|12404|3|26|44|AM|third|night|| +12405|AAAAAAAAGHADAAAA|12405|3|26|45|AM|third|night|| +12406|AAAAAAAAHHADAAAA|12406|3|26|46|AM|third|night|| +12407|AAAAAAAAIHADAAAA|12407|3|26|47|AM|third|night|| +12408|AAAAAAAAJHADAAAA|12408|3|26|48|AM|third|night|| +12409|AAAAAAAAKHADAAAA|12409|3|26|49|AM|third|night|| +12410|AAAAAAAALHADAAAA|12410|3|26|50|AM|third|night|| +12411|AAAAAAAAMHADAAAA|12411|3|26|51|AM|third|night|| +12412|AAAAAAAANHADAAAA|12412|3|26|52|AM|third|night|| +12413|AAAAAAAAOHADAAAA|12413|3|26|53|AM|third|night|| +12414|AAAAAAAAPHADAAAA|12414|3|26|54|AM|third|night|| +12415|AAAAAAAAAIADAAAA|12415|3|26|55|AM|third|night|| +12416|AAAAAAAABIADAAAA|12416|3|26|56|AM|third|night|| +12417|AAAAAAAACIADAAAA|12417|3|26|57|AM|third|night|| +12418|AAAAAAAADIADAAAA|12418|3|26|58|AM|third|night|| +12419|AAAAAAAAEIADAAAA|12419|3|26|59|AM|third|night|| +12420|AAAAAAAAFIADAAAA|12420|3|27|0|AM|third|night|| +12421|AAAAAAAAGIADAAAA|12421|3|27|1|AM|third|night|| +12422|AAAAAAAAHIADAAAA|12422|3|27|2|AM|third|night|| +12423|AAAAAAAAIIADAAAA|12423|3|27|3|AM|third|night|| +12424|AAAAAAAAJIADAAAA|12424|3|27|4|AM|third|night|| +12425|AAAAAAAAKIADAAAA|12425|3|27|5|AM|third|night|| +12426|AAAAAAAALIADAAAA|12426|3|27|6|AM|third|night|| +12427|AAAAAAAAMIADAAAA|12427|3|27|7|AM|third|night|| +12428|AAAAAAAANIADAAAA|12428|3|27|8|AM|third|night|| +12429|AAAAAAAAOIADAAAA|12429|3|27|9|AM|third|night|| +12430|AAAAAAAAPIADAAAA|12430|3|27|10|AM|third|night|| +12431|AAAAAAAAAJADAAAA|12431|3|27|11|AM|third|night|| +12432|AAAAAAAABJADAAAA|12432|3|27|12|AM|third|night|| +12433|AAAAAAAACJADAAAA|12433|3|27|13|AM|third|night|| +12434|AAAAAAAADJADAAAA|12434|3|27|14|AM|third|night|| +12435|AAAAAAAAEJADAAAA|12435|3|27|15|AM|third|night|| +12436|AAAAAAAAFJADAAAA|12436|3|27|16|AM|third|night|| +12437|AAAAAAAAGJADAAAA|12437|3|27|17|AM|third|night|| +12438|AAAAAAAAHJADAAAA|12438|3|27|18|AM|third|night|| +12439|AAAAAAAAIJADAAAA|12439|3|27|19|AM|third|night|| +12440|AAAAAAAAJJADAAAA|12440|3|27|20|AM|third|night|| +12441|AAAAAAAAKJADAAAA|12441|3|27|21|AM|third|night|| +12442|AAAAAAAALJADAAAA|12442|3|27|22|AM|third|night|| +12443|AAAAAAAAMJADAAAA|12443|3|27|23|AM|third|night|| +12444|AAAAAAAANJADAAAA|12444|3|27|24|AM|third|night|| +12445|AAAAAAAAOJADAAAA|12445|3|27|25|AM|third|night|| +12446|AAAAAAAAPJADAAAA|12446|3|27|26|AM|third|night|| +12447|AAAAAAAAAKADAAAA|12447|3|27|27|AM|third|night|| +12448|AAAAAAAABKADAAAA|12448|3|27|28|AM|third|night|| +12449|AAAAAAAACKADAAAA|12449|3|27|29|AM|third|night|| +12450|AAAAAAAADKADAAAA|12450|3|27|30|AM|third|night|| +12451|AAAAAAAAEKADAAAA|12451|3|27|31|AM|third|night|| +12452|AAAAAAAAFKADAAAA|12452|3|27|32|AM|third|night|| +12453|AAAAAAAAGKADAAAA|12453|3|27|33|AM|third|night|| +12454|AAAAAAAAHKADAAAA|12454|3|27|34|AM|third|night|| +12455|AAAAAAAAIKADAAAA|12455|3|27|35|AM|third|night|| +12456|AAAAAAAAJKADAAAA|12456|3|27|36|AM|third|night|| +12457|AAAAAAAAKKADAAAA|12457|3|27|37|AM|third|night|| +12458|AAAAAAAALKADAAAA|12458|3|27|38|AM|third|night|| +12459|AAAAAAAAMKADAAAA|12459|3|27|39|AM|third|night|| +12460|AAAAAAAANKADAAAA|12460|3|27|40|AM|third|night|| +12461|AAAAAAAAOKADAAAA|12461|3|27|41|AM|third|night|| +12462|AAAAAAAAPKADAAAA|12462|3|27|42|AM|third|night|| +12463|AAAAAAAAALADAAAA|12463|3|27|43|AM|third|night|| +12464|AAAAAAAABLADAAAA|12464|3|27|44|AM|third|night|| +12465|AAAAAAAACLADAAAA|12465|3|27|45|AM|third|night|| +12466|AAAAAAAADLADAAAA|12466|3|27|46|AM|third|night|| +12467|AAAAAAAAELADAAAA|12467|3|27|47|AM|third|night|| +12468|AAAAAAAAFLADAAAA|12468|3|27|48|AM|third|night|| +12469|AAAAAAAAGLADAAAA|12469|3|27|49|AM|third|night|| +12470|AAAAAAAAHLADAAAA|12470|3|27|50|AM|third|night|| +12471|AAAAAAAAILADAAAA|12471|3|27|51|AM|third|night|| +12472|AAAAAAAAJLADAAAA|12472|3|27|52|AM|third|night|| +12473|AAAAAAAAKLADAAAA|12473|3|27|53|AM|third|night|| +12474|AAAAAAAALLADAAAA|12474|3|27|54|AM|third|night|| +12475|AAAAAAAAMLADAAAA|12475|3|27|55|AM|third|night|| +12476|AAAAAAAANLADAAAA|12476|3|27|56|AM|third|night|| +12477|AAAAAAAAOLADAAAA|12477|3|27|57|AM|third|night|| +12478|AAAAAAAAPLADAAAA|12478|3|27|58|AM|third|night|| +12479|AAAAAAAAAMADAAAA|12479|3|27|59|AM|third|night|| +12480|AAAAAAAABMADAAAA|12480|3|28|0|AM|third|night|| +12481|AAAAAAAACMADAAAA|12481|3|28|1|AM|third|night|| +12482|AAAAAAAADMADAAAA|12482|3|28|2|AM|third|night|| +12483|AAAAAAAAEMADAAAA|12483|3|28|3|AM|third|night|| +12484|AAAAAAAAFMADAAAA|12484|3|28|4|AM|third|night|| +12485|AAAAAAAAGMADAAAA|12485|3|28|5|AM|third|night|| +12486|AAAAAAAAHMADAAAA|12486|3|28|6|AM|third|night|| +12487|AAAAAAAAIMADAAAA|12487|3|28|7|AM|third|night|| +12488|AAAAAAAAJMADAAAA|12488|3|28|8|AM|third|night|| +12489|AAAAAAAAKMADAAAA|12489|3|28|9|AM|third|night|| +12490|AAAAAAAALMADAAAA|12490|3|28|10|AM|third|night|| +12491|AAAAAAAAMMADAAAA|12491|3|28|11|AM|third|night|| +12492|AAAAAAAANMADAAAA|12492|3|28|12|AM|third|night|| +12493|AAAAAAAAOMADAAAA|12493|3|28|13|AM|third|night|| +12494|AAAAAAAAPMADAAAA|12494|3|28|14|AM|third|night|| +12495|AAAAAAAAANADAAAA|12495|3|28|15|AM|third|night|| +12496|AAAAAAAABNADAAAA|12496|3|28|16|AM|third|night|| +12497|AAAAAAAACNADAAAA|12497|3|28|17|AM|third|night|| +12498|AAAAAAAADNADAAAA|12498|3|28|18|AM|third|night|| +12499|AAAAAAAAENADAAAA|12499|3|28|19|AM|third|night|| +12500|AAAAAAAAFNADAAAA|12500|3|28|20|AM|third|night|| +12501|AAAAAAAAGNADAAAA|12501|3|28|21|AM|third|night|| +12502|AAAAAAAAHNADAAAA|12502|3|28|22|AM|third|night|| +12503|AAAAAAAAINADAAAA|12503|3|28|23|AM|third|night|| +12504|AAAAAAAAJNADAAAA|12504|3|28|24|AM|third|night|| +12505|AAAAAAAAKNADAAAA|12505|3|28|25|AM|third|night|| +12506|AAAAAAAALNADAAAA|12506|3|28|26|AM|third|night|| +12507|AAAAAAAAMNADAAAA|12507|3|28|27|AM|third|night|| +12508|AAAAAAAANNADAAAA|12508|3|28|28|AM|third|night|| +12509|AAAAAAAAONADAAAA|12509|3|28|29|AM|third|night|| +12510|AAAAAAAAPNADAAAA|12510|3|28|30|AM|third|night|| +12511|AAAAAAAAAOADAAAA|12511|3|28|31|AM|third|night|| +12512|AAAAAAAABOADAAAA|12512|3|28|32|AM|third|night|| +12513|AAAAAAAACOADAAAA|12513|3|28|33|AM|third|night|| +12514|AAAAAAAADOADAAAA|12514|3|28|34|AM|third|night|| +12515|AAAAAAAAEOADAAAA|12515|3|28|35|AM|third|night|| +12516|AAAAAAAAFOADAAAA|12516|3|28|36|AM|third|night|| +12517|AAAAAAAAGOADAAAA|12517|3|28|37|AM|third|night|| +12518|AAAAAAAAHOADAAAA|12518|3|28|38|AM|third|night|| +12519|AAAAAAAAIOADAAAA|12519|3|28|39|AM|third|night|| +12520|AAAAAAAAJOADAAAA|12520|3|28|40|AM|third|night|| +12521|AAAAAAAAKOADAAAA|12521|3|28|41|AM|third|night|| +12522|AAAAAAAALOADAAAA|12522|3|28|42|AM|third|night|| +12523|AAAAAAAAMOADAAAA|12523|3|28|43|AM|third|night|| +12524|AAAAAAAANOADAAAA|12524|3|28|44|AM|third|night|| +12525|AAAAAAAAOOADAAAA|12525|3|28|45|AM|third|night|| +12526|AAAAAAAAPOADAAAA|12526|3|28|46|AM|third|night|| +12527|AAAAAAAAAPADAAAA|12527|3|28|47|AM|third|night|| +12528|AAAAAAAABPADAAAA|12528|3|28|48|AM|third|night|| +12529|AAAAAAAACPADAAAA|12529|3|28|49|AM|third|night|| +12530|AAAAAAAADPADAAAA|12530|3|28|50|AM|third|night|| +12531|AAAAAAAAEPADAAAA|12531|3|28|51|AM|third|night|| +12532|AAAAAAAAFPADAAAA|12532|3|28|52|AM|third|night|| +12533|AAAAAAAAGPADAAAA|12533|3|28|53|AM|third|night|| +12534|AAAAAAAAHPADAAAA|12534|3|28|54|AM|third|night|| +12535|AAAAAAAAIPADAAAA|12535|3|28|55|AM|third|night|| +12536|AAAAAAAAJPADAAAA|12536|3|28|56|AM|third|night|| +12537|AAAAAAAAKPADAAAA|12537|3|28|57|AM|third|night|| +12538|AAAAAAAALPADAAAA|12538|3|28|58|AM|third|night|| +12539|AAAAAAAAMPADAAAA|12539|3|28|59|AM|third|night|| +12540|AAAAAAAANPADAAAA|12540|3|29|0|AM|third|night|| +12541|AAAAAAAAOPADAAAA|12541|3|29|1|AM|third|night|| +12542|AAAAAAAAPPADAAAA|12542|3|29|2|AM|third|night|| +12543|AAAAAAAAAABDAAAA|12543|3|29|3|AM|third|night|| +12544|AAAAAAAABABDAAAA|12544|3|29|4|AM|third|night|| +12545|AAAAAAAACABDAAAA|12545|3|29|5|AM|third|night|| +12546|AAAAAAAADABDAAAA|12546|3|29|6|AM|third|night|| +12547|AAAAAAAAEABDAAAA|12547|3|29|7|AM|third|night|| +12548|AAAAAAAAFABDAAAA|12548|3|29|8|AM|third|night|| +12549|AAAAAAAAGABDAAAA|12549|3|29|9|AM|third|night|| +12550|AAAAAAAAHABDAAAA|12550|3|29|10|AM|third|night|| +12551|AAAAAAAAIABDAAAA|12551|3|29|11|AM|third|night|| +12552|AAAAAAAAJABDAAAA|12552|3|29|12|AM|third|night|| +12553|AAAAAAAAKABDAAAA|12553|3|29|13|AM|third|night|| +12554|AAAAAAAALABDAAAA|12554|3|29|14|AM|third|night|| +12555|AAAAAAAAMABDAAAA|12555|3|29|15|AM|third|night|| +12556|AAAAAAAANABDAAAA|12556|3|29|16|AM|third|night|| +12557|AAAAAAAAOABDAAAA|12557|3|29|17|AM|third|night|| +12558|AAAAAAAAPABDAAAA|12558|3|29|18|AM|third|night|| +12559|AAAAAAAAABBDAAAA|12559|3|29|19|AM|third|night|| +12560|AAAAAAAABBBDAAAA|12560|3|29|20|AM|third|night|| +12561|AAAAAAAACBBDAAAA|12561|3|29|21|AM|third|night|| +12562|AAAAAAAADBBDAAAA|12562|3|29|22|AM|third|night|| +12563|AAAAAAAAEBBDAAAA|12563|3|29|23|AM|third|night|| +12564|AAAAAAAAFBBDAAAA|12564|3|29|24|AM|third|night|| +12565|AAAAAAAAGBBDAAAA|12565|3|29|25|AM|third|night|| +12566|AAAAAAAAHBBDAAAA|12566|3|29|26|AM|third|night|| +12567|AAAAAAAAIBBDAAAA|12567|3|29|27|AM|third|night|| +12568|AAAAAAAAJBBDAAAA|12568|3|29|28|AM|third|night|| +12569|AAAAAAAAKBBDAAAA|12569|3|29|29|AM|third|night|| +12570|AAAAAAAALBBDAAAA|12570|3|29|30|AM|third|night|| +12571|AAAAAAAAMBBDAAAA|12571|3|29|31|AM|third|night|| +12572|AAAAAAAANBBDAAAA|12572|3|29|32|AM|third|night|| +12573|AAAAAAAAOBBDAAAA|12573|3|29|33|AM|third|night|| +12574|AAAAAAAAPBBDAAAA|12574|3|29|34|AM|third|night|| +12575|AAAAAAAAACBDAAAA|12575|3|29|35|AM|third|night|| +12576|AAAAAAAABCBDAAAA|12576|3|29|36|AM|third|night|| +12577|AAAAAAAACCBDAAAA|12577|3|29|37|AM|third|night|| +12578|AAAAAAAADCBDAAAA|12578|3|29|38|AM|third|night|| +12579|AAAAAAAAECBDAAAA|12579|3|29|39|AM|third|night|| +12580|AAAAAAAAFCBDAAAA|12580|3|29|40|AM|third|night|| +12581|AAAAAAAAGCBDAAAA|12581|3|29|41|AM|third|night|| +12582|AAAAAAAAHCBDAAAA|12582|3|29|42|AM|third|night|| +12583|AAAAAAAAICBDAAAA|12583|3|29|43|AM|third|night|| +12584|AAAAAAAAJCBDAAAA|12584|3|29|44|AM|third|night|| +12585|AAAAAAAAKCBDAAAA|12585|3|29|45|AM|third|night|| +12586|AAAAAAAALCBDAAAA|12586|3|29|46|AM|third|night|| +12587|AAAAAAAAMCBDAAAA|12587|3|29|47|AM|third|night|| +12588|AAAAAAAANCBDAAAA|12588|3|29|48|AM|third|night|| +12589|AAAAAAAAOCBDAAAA|12589|3|29|49|AM|third|night|| +12590|AAAAAAAAPCBDAAAA|12590|3|29|50|AM|third|night|| +12591|AAAAAAAAADBDAAAA|12591|3|29|51|AM|third|night|| +12592|AAAAAAAABDBDAAAA|12592|3|29|52|AM|third|night|| +12593|AAAAAAAACDBDAAAA|12593|3|29|53|AM|third|night|| +12594|AAAAAAAADDBDAAAA|12594|3|29|54|AM|third|night|| +12595|AAAAAAAAEDBDAAAA|12595|3|29|55|AM|third|night|| +12596|AAAAAAAAFDBDAAAA|12596|3|29|56|AM|third|night|| +12597|AAAAAAAAGDBDAAAA|12597|3|29|57|AM|third|night|| +12598|AAAAAAAAHDBDAAAA|12598|3|29|58|AM|third|night|| +12599|AAAAAAAAIDBDAAAA|12599|3|29|59|AM|third|night|| +12600|AAAAAAAAJDBDAAAA|12600|3|30|0|AM|third|night|| +12601|AAAAAAAAKDBDAAAA|12601|3|30|1|AM|third|night|| +12602|AAAAAAAALDBDAAAA|12602|3|30|2|AM|third|night|| +12603|AAAAAAAAMDBDAAAA|12603|3|30|3|AM|third|night|| +12604|AAAAAAAANDBDAAAA|12604|3|30|4|AM|third|night|| +12605|AAAAAAAAODBDAAAA|12605|3|30|5|AM|third|night|| +12606|AAAAAAAAPDBDAAAA|12606|3|30|6|AM|third|night|| +12607|AAAAAAAAAEBDAAAA|12607|3|30|7|AM|third|night|| +12608|AAAAAAAABEBDAAAA|12608|3|30|8|AM|third|night|| +12609|AAAAAAAACEBDAAAA|12609|3|30|9|AM|third|night|| +12610|AAAAAAAADEBDAAAA|12610|3|30|10|AM|third|night|| +12611|AAAAAAAAEEBDAAAA|12611|3|30|11|AM|third|night|| +12612|AAAAAAAAFEBDAAAA|12612|3|30|12|AM|third|night|| +12613|AAAAAAAAGEBDAAAA|12613|3|30|13|AM|third|night|| +12614|AAAAAAAAHEBDAAAA|12614|3|30|14|AM|third|night|| +12615|AAAAAAAAIEBDAAAA|12615|3|30|15|AM|third|night|| +12616|AAAAAAAAJEBDAAAA|12616|3|30|16|AM|third|night|| +12617|AAAAAAAAKEBDAAAA|12617|3|30|17|AM|third|night|| +12618|AAAAAAAALEBDAAAA|12618|3|30|18|AM|third|night|| +12619|AAAAAAAAMEBDAAAA|12619|3|30|19|AM|third|night|| +12620|AAAAAAAANEBDAAAA|12620|3|30|20|AM|third|night|| +12621|AAAAAAAAOEBDAAAA|12621|3|30|21|AM|third|night|| +12622|AAAAAAAAPEBDAAAA|12622|3|30|22|AM|third|night|| +12623|AAAAAAAAAFBDAAAA|12623|3|30|23|AM|third|night|| +12624|AAAAAAAABFBDAAAA|12624|3|30|24|AM|third|night|| +12625|AAAAAAAACFBDAAAA|12625|3|30|25|AM|third|night|| +12626|AAAAAAAADFBDAAAA|12626|3|30|26|AM|third|night|| +12627|AAAAAAAAEFBDAAAA|12627|3|30|27|AM|third|night|| +12628|AAAAAAAAFFBDAAAA|12628|3|30|28|AM|third|night|| +12629|AAAAAAAAGFBDAAAA|12629|3|30|29|AM|third|night|| +12630|AAAAAAAAHFBDAAAA|12630|3|30|30|AM|third|night|| +12631|AAAAAAAAIFBDAAAA|12631|3|30|31|AM|third|night|| +12632|AAAAAAAAJFBDAAAA|12632|3|30|32|AM|third|night|| +12633|AAAAAAAAKFBDAAAA|12633|3|30|33|AM|third|night|| +12634|AAAAAAAALFBDAAAA|12634|3|30|34|AM|third|night|| +12635|AAAAAAAAMFBDAAAA|12635|3|30|35|AM|third|night|| +12636|AAAAAAAANFBDAAAA|12636|3|30|36|AM|third|night|| +12637|AAAAAAAAOFBDAAAA|12637|3|30|37|AM|third|night|| +12638|AAAAAAAAPFBDAAAA|12638|3|30|38|AM|third|night|| +12639|AAAAAAAAAGBDAAAA|12639|3|30|39|AM|third|night|| +12640|AAAAAAAABGBDAAAA|12640|3|30|40|AM|third|night|| +12641|AAAAAAAACGBDAAAA|12641|3|30|41|AM|third|night|| +12642|AAAAAAAADGBDAAAA|12642|3|30|42|AM|third|night|| +12643|AAAAAAAAEGBDAAAA|12643|3|30|43|AM|third|night|| +12644|AAAAAAAAFGBDAAAA|12644|3|30|44|AM|third|night|| +12645|AAAAAAAAGGBDAAAA|12645|3|30|45|AM|third|night|| +12646|AAAAAAAAHGBDAAAA|12646|3|30|46|AM|third|night|| +12647|AAAAAAAAIGBDAAAA|12647|3|30|47|AM|third|night|| +12648|AAAAAAAAJGBDAAAA|12648|3|30|48|AM|third|night|| +12649|AAAAAAAAKGBDAAAA|12649|3|30|49|AM|third|night|| +12650|AAAAAAAALGBDAAAA|12650|3|30|50|AM|third|night|| +12651|AAAAAAAAMGBDAAAA|12651|3|30|51|AM|third|night|| +12652|AAAAAAAANGBDAAAA|12652|3|30|52|AM|third|night|| +12653|AAAAAAAAOGBDAAAA|12653|3|30|53|AM|third|night|| +12654|AAAAAAAAPGBDAAAA|12654|3|30|54|AM|third|night|| +12655|AAAAAAAAAHBDAAAA|12655|3|30|55|AM|third|night|| +12656|AAAAAAAABHBDAAAA|12656|3|30|56|AM|third|night|| +12657|AAAAAAAACHBDAAAA|12657|3|30|57|AM|third|night|| +12658|AAAAAAAADHBDAAAA|12658|3|30|58|AM|third|night|| +12659|AAAAAAAAEHBDAAAA|12659|3|30|59|AM|third|night|| +12660|AAAAAAAAFHBDAAAA|12660|3|31|0|AM|third|night|| +12661|AAAAAAAAGHBDAAAA|12661|3|31|1|AM|third|night|| +12662|AAAAAAAAHHBDAAAA|12662|3|31|2|AM|third|night|| +12663|AAAAAAAAIHBDAAAA|12663|3|31|3|AM|third|night|| +12664|AAAAAAAAJHBDAAAA|12664|3|31|4|AM|third|night|| +12665|AAAAAAAAKHBDAAAA|12665|3|31|5|AM|third|night|| +12666|AAAAAAAALHBDAAAA|12666|3|31|6|AM|third|night|| +12667|AAAAAAAAMHBDAAAA|12667|3|31|7|AM|third|night|| +12668|AAAAAAAANHBDAAAA|12668|3|31|8|AM|third|night|| +12669|AAAAAAAAOHBDAAAA|12669|3|31|9|AM|third|night|| +12670|AAAAAAAAPHBDAAAA|12670|3|31|10|AM|third|night|| +12671|AAAAAAAAAIBDAAAA|12671|3|31|11|AM|third|night|| +12672|AAAAAAAABIBDAAAA|12672|3|31|12|AM|third|night|| +12673|AAAAAAAACIBDAAAA|12673|3|31|13|AM|third|night|| +12674|AAAAAAAADIBDAAAA|12674|3|31|14|AM|third|night|| +12675|AAAAAAAAEIBDAAAA|12675|3|31|15|AM|third|night|| +12676|AAAAAAAAFIBDAAAA|12676|3|31|16|AM|third|night|| +12677|AAAAAAAAGIBDAAAA|12677|3|31|17|AM|third|night|| +12678|AAAAAAAAHIBDAAAA|12678|3|31|18|AM|third|night|| +12679|AAAAAAAAIIBDAAAA|12679|3|31|19|AM|third|night|| +12680|AAAAAAAAJIBDAAAA|12680|3|31|20|AM|third|night|| +12681|AAAAAAAAKIBDAAAA|12681|3|31|21|AM|third|night|| +12682|AAAAAAAALIBDAAAA|12682|3|31|22|AM|third|night|| +12683|AAAAAAAAMIBDAAAA|12683|3|31|23|AM|third|night|| +12684|AAAAAAAANIBDAAAA|12684|3|31|24|AM|third|night|| +12685|AAAAAAAAOIBDAAAA|12685|3|31|25|AM|third|night|| +12686|AAAAAAAAPIBDAAAA|12686|3|31|26|AM|third|night|| +12687|AAAAAAAAAJBDAAAA|12687|3|31|27|AM|third|night|| +12688|AAAAAAAABJBDAAAA|12688|3|31|28|AM|third|night|| +12689|AAAAAAAACJBDAAAA|12689|3|31|29|AM|third|night|| +12690|AAAAAAAADJBDAAAA|12690|3|31|30|AM|third|night|| +12691|AAAAAAAAEJBDAAAA|12691|3|31|31|AM|third|night|| +12692|AAAAAAAAFJBDAAAA|12692|3|31|32|AM|third|night|| +12693|AAAAAAAAGJBDAAAA|12693|3|31|33|AM|third|night|| +12694|AAAAAAAAHJBDAAAA|12694|3|31|34|AM|third|night|| +12695|AAAAAAAAIJBDAAAA|12695|3|31|35|AM|third|night|| +12696|AAAAAAAAJJBDAAAA|12696|3|31|36|AM|third|night|| +12697|AAAAAAAAKJBDAAAA|12697|3|31|37|AM|third|night|| +12698|AAAAAAAALJBDAAAA|12698|3|31|38|AM|third|night|| +12699|AAAAAAAAMJBDAAAA|12699|3|31|39|AM|third|night|| +12700|AAAAAAAANJBDAAAA|12700|3|31|40|AM|third|night|| +12701|AAAAAAAAOJBDAAAA|12701|3|31|41|AM|third|night|| +12702|AAAAAAAAPJBDAAAA|12702|3|31|42|AM|third|night|| +12703|AAAAAAAAAKBDAAAA|12703|3|31|43|AM|third|night|| +12704|AAAAAAAABKBDAAAA|12704|3|31|44|AM|third|night|| +12705|AAAAAAAACKBDAAAA|12705|3|31|45|AM|third|night|| +12706|AAAAAAAADKBDAAAA|12706|3|31|46|AM|third|night|| +12707|AAAAAAAAEKBDAAAA|12707|3|31|47|AM|third|night|| +12708|AAAAAAAAFKBDAAAA|12708|3|31|48|AM|third|night|| +12709|AAAAAAAAGKBDAAAA|12709|3|31|49|AM|third|night|| +12710|AAAAAAAAHKBDAAAA|12710|3|31|50|AM|third|night|| +12711|AAAAAAAAIKBDAAAA|12711|3|31|51|AM|third|night|| +12712|AAAAAAAAJKBDAAAA|12712|3|31|52|AM|third|night|| +12713|AAAAAAAAKKBDAAAA|12713|3|31|53|AM|third|night|| +12714|AAAAAAAALKBDAAAA|12714|3|31|54|AM|third|night|| +12715|AAAAAAAAMKBDAAAA|12715|3|31|55|AM|third|night|| +12716|AAAAAAAANKBDAAAA|12716|3|31|56|AM|third|night|| +12717|AAAAAAAAOKBDAAAA|12717|3|31|57|AM|third|night|| +12718|AAAAAAAAPKBDAAAA|12718|3|31|58|AM|third|night|| +12719|AAAAAAAAALBDAAAA|12719|3|31|59|AM|third|night|| +12720|AAAAAAAABLBDAAAA|12720|3|32|0|AM|third|night|| +12721|AAAAAAAACLBDAAAA|12721|3|32|1|AM|third|night|| +12722|AAAAAAAADLBDAAAA|12722|3|32|2|AM|third|night|| +12723|AAAAAAAAELBDAAAA|12723|3|32|3|AM|third|night|| +12724|AAAAAAAAFLBDAAAA|12724|3|32|4|AM|third|night|| +12725|AAAAAAAAGLBDAAAA|12725|3|32|5|AM|third|night|| +12726|AAAAAAAAHLBDAAAA|12726|3|32|6|AM|third|night|| +12727|AAAAAAAAILBDAAAA|12727|3|32|7|AM|third|night|| +12728|AAAAAAAAJLBDAAAA|12728|3|32|8|AM|third|night|| +12729|AAAAAAAAKLBDAAAA|12729|3|32|9|AM|third|night|| +12730|AAAAAAAALLBDAAAA|12730|3|32|10|AM|third|night|| +12731|AAAAAAAAMLBDAAAA|12731|3|32|11|AM|third|night|| +12732|AAAAAAAANLBDAAAA|12732|3|32|12|AM|third|night|| +12733|AAAAAAAAOLBDAAAA|12733|3|32|13|AM|third|night|| +12734|AAAAAAAAPLBDAAAA|12734|3|32|14|AM|third|night|| +12735|AAAAAAAAAMBDAAAA|12735|3|32|15|AM|third|night|| +12736|AAAAAAAABMBDAAAA|12736|3|32|16|AM|third|night|| +12737|AAAAAAAACMBDAAAA|12737|3|32|17|AM|third|night|| +12738|AAAAAAAADMBDAAAA|12738|3|32|18|AM|third|night|| +12739|AAAAAAAAEMBDAAAA|12739|3|32|19|AM|third|night|| +12740|AAAAAAAAFMBDAAAA|12740|3|32|20|AM|third|night|| +12741|AAAAAAAAGMBDAAAA|12741|3|32|21|AM|third|night|| +12742|AAAAAAAAHMBDAAAA|12742|3|32|22|AM|third|night|| +12743|AAAAAAAAIMBDAAAA|12743|3|32|23|AM|third|night|| +12744|AAAAAAAAJMBDAAAA|12744|3|32|24|AM|third|night|| +12745|AAAAAAAAKMBDAAAA|12745|3|32|25|AM|third|night|| +12746|AAAAAAAALMBDAAAA|12746|3|32|26|AM|third|night|| +12747|AAAAAAAAMMBDAAAA|12747|3|32|27|AM|third|night|| +12748|AAAAAAAANMBDAAAA|12748|3|32|28|AM|third|night|| +12749|AAAAAAAAOMBDAAAA|12749|3|32|29|AM|third|night|| +12750|AAAAAAAAPMBDAAAA|12750|3|32|30|AM|third|night|| +12751|AAAAAAAAANBDAAAA|12751|3|32|31|AM|third|night|| +12752|AAAAAAAABNBDAAAA|12752|3|32|32|AM|third|night|| +12753|AAAAAAAACNBDAAAA|12753|3|32|33|AM|third|night|| +12754|AAAAAAAADNBDAAAA|12754|3|32|34|AM|third|night|| +12755|AAAAAAAAENBDAAAA|12755|3|32|35|AM|third|night|| +12756|AAAAAAAAFNBDAAAA|12756|3|32|36|AM|third|night|| +12757|AAAAAAAAGNBDAAAA|12757|3|32|37|AM|third|night|| +12758|AAAAAAAAHNBDAAAA|12758|3|32|38|AM|third|night|| +12759|AAAAAAAAINBDAAAA|12759|3|32|39|AM|third|night|| +12760|AAAAAAAAJNBDAAAA|12760|3|32|40|AM|third|night|| +12761|AAAAAAAAKNBDAAAA|12761|3|32|41|AM|third|night|| +12762|AAAAAAAALNBDAAAA|12762|3|32|42|AM|third|night|| +12763|AAAAAAAAMNBDAAAA|12763|3|32|43|AM|third|night|| +12764|AAAAAAAANNBDAAAA|12764|3|32|44|AM|third|night|| +12765|AAAAAAAAONBDAAAA|12765|3|32|45|AM|third|night|| +12766|AAAAAAAAPNBDAAAA|12766|3|32|46|AM|third|night|| +12767|AAAAAAAAAOBDAAAA|12767|3|32|47|AM|third|night|| +12768|AAAAAAAABOBDAAAA|12768|3|32|48|AM|third|night|| +12769|AAAAAAAACOBDAAAA|12769|3|32|49|AM|third|night|| +12770|AAAAAAAADOBDAAAA|12770|3|32|50|AM|third|night|| +12771|AAAAAAAAEOBDAAAA|12771|3|32|51|AM|third|night|| +12772|AAAAAAAAFOBDAAAA|12772|3|32|52|AM|third|night|| +12773|AAAAAAAAGOBDAAAA|12773|3|32|53|AM|third|night|| +12774|AAAAAAAAHOBDAAAA|12774|3|32|54|AM|third|night|| +12775|AAAAAAAAIOBDAAAA|12775|3|32|55|AM|third|night|| +12776|AAAAAAAAJOBDAAAA|12776|3|32|56|AM|third|night|| +12777|AAAAAAAAKOBDAAAA|12777|3|32|57|AM|third|night|| +12778|AAAAAAAALOBDAAAA|12778|3|32|58|AM|third|night|| +12779|AAAAAAAAMOBDAAAA|12779|3|32|59|AM|third|night|| +12780|AAAAAAAANOBDAAAA|12780|3|33|0|AM|third|night|| +12781|AAAAAAAAOOBDAAAA|12781|3|33|1|AM|third|night|| +12782|AAAAAAAAPOBDAAAA|12782|3|33|2|AM|third|night|| +12783|AAAAAAAAAPBDAAAA|12783|3|33|3|AM|third|night|| +12784|AAAAAAAABPBDAAAA|12784|3|33|4|AM|third|night|| +12785|AAAAAAAACPBDAAAA|12785|3|33|5|AM|third|night|| +12786|AAAAAAAADPBDAAAA|12786|3|33|6|AM|third|night|| +12787|AAAAAAAAEPBDAAAA|12787|3|33|7|AM|third|night|| +12788|AAAAAAAAFPBDAAAA|12788|3|33|8|AM|third|night|| +12789|AAAAAAAAGPBDAAAA|12789|3|33|9|AM|third|night|| +12790|AAAAAAAAHPBDAAAA|12790|3|33|10|AM|third|night|| +12791|AAAAAAAAIPBDAAAA|12791|3|33|11|AM|third|night|| +12792|AAAAAAAAJPBDAAAA|12792|3|33|12|AM|third|night|| +12793|AAAAAAAAKPBDAAAA|12793|3|33|13|AM|third|night|| +12794|AAAAAAAALPBDAAAA|12794|3|33|14|AM|third|night|| +12795|AAAAAAAAMPBDAAAA|12795|3|33|15|AM|third|night|| +12796|AAAAAAAANPBDAAAA|12796|3|33|16|AM|third|night|| +12797|AAAAAAAAOPBDAAAA|12797|3|33|17|AM|third|night|| +12798|AAAAAAAAPPBDAAAA|12798|3|33|18|AM|third|night|| +12799|AAAAAAAAAACDAAAA|12799|3|33|19|AM|third|night|| +12800|AAAAAAAABACDAAAA|12800|3|33|20|AM|third|night|| +12801|AAAAAAAACACDAAAA|12801|3|33|21|AM|third|night|| +12802|AAAAAAAADACDAAAA|12802|3|33|22|AM|third|night|| +12803|AAAAAAAAEACDAAAA|12803|3|33|23|AM|third|night|| +12804|AAAAAAAAFACDAAAA|12804|3|33|24|AM|third|night|| +12805|AAAAAAAAGACDAAAA|12805|3|33|25|AM|third|night|| +12806|AAAAAAAAHACDAAAA|12806|3|33|26|AM|third|night|| +12807|AAAAAAAAIACDAAAA|12807|3|33|27|AM|third|night|| +12808|AAAAAAAAJACDAAAA|12808|3|33|28|AM|third|night|| +12809|AAAAAAAAKACDAAAA|12809|3|33|29|AM|third|night|| +12810|AAAAAAAALACDAAAA|12810|3|33|30|AM|third|night|| +12811|AAAAAAAAMACDAAAA|12811|3|33|31|AM|third|night|| +12812|AAAAAAAANACDAAAA|12812|3|33|32|AM|third|night|| +12813|AAAAAAAAOACDAAAA|12813|3|33|33|AM|third|night|| +12814|AAAAAAAAPACDAAAA|12814|3|33|34|AM|third|night|| +12815|AAAAAAAAABCDAAAA|12815|3|33|35|AM|third|night|| +12816|AAAAAAAABBCDAAAA|12816|3|33|36|AM|third|night|| +12817|AAAAAAAACBCDAAAA|12817|3|33|37|AM|third|night|| +12818|AAAAAAAADBCDAAAA|12818|3|33|38|AM|third|night|| +12819|AAAAAAAAEBCDAAAA|12819|3|33|39|AM|third|night|| +12820|AAAAAAAAFBCDAAAA|12820|3|33|40|AM|third|night|| +12821|AAAAAAAAGBCDAAAA|12821|3|33|41|AM|third|night|| +12822|AAAAAAAAHBCDAAAA|12822|3|33|42|AM|third|night|| +12823|AAAAAAAAIBCDAAAA|12823|3|33|43|AM|third|night|| +12824|AAAAAAAAJBCDAAAA|12824|3|33|44|AM|third|night|| +12825|AAAAAAAAKBCDAAAA|12825|3|33|45|AM|third|night|| +12826|AAAAAAAALBCDAAAA|12826|3|33|46|AM|third|night|| +12827|AAAAAAAAMBCDAAAA|12827|3|33|47|AM|third|night|| +12828|AAAAAAAANBCDAAAA|12828|3|33|48|AM|third|night|| +12829|AAAAAAAAOBCDAAAA|12829|3|33|49|AM|third|night|| +12830|AAAAAAAAPBCDAAAA|12830|3|33|50|AM|third|night|| +12831|AAAAAAAAACCDAAAA|12831|3|33|51|AM|third|night|| +12832|AAAAAAAABCCDAAAA|12832|3|33|52|AM|third|night|| +12833|AAAAAAAACCCDAAAA|12833|3|33|53|AM|third|night|| +12834|AAAAAAAADCCDAAAA|12834|3|33|54|AM|third|night|| +12835|AAAAAAAAECCDAAAA|12835|3|33|55|AM|third|night|| +12836|AAAAAAAAFCCDAAAA|12836|3|33|56|AM|third|night|| +12837|AAAAAAAAGCCDAAAA|12837|3|33|57|AM|third|night|| +12838|AAAAAAAAHCCDAAAA|12838|3|33|58|AM|third|night|| +12839|AAAAAAAAICCDAAAA|12839|3|33|59|AM|third|night|| +12840|AAAAAAAAJCCDAAAA|12840|3|34|0|AM|third|night|| +12841|AAAAAAAAKCCDAAAA|12841|3|34|1|AM|third|night|| +12842|AAAAAAAALCCDAAAA|12842|3|34|2|AM|third|night|| +12843|AAAAAAAAMCCDAAAA|12843|3|34|3|AM|third|night|| +12844|AAAAAAAANCCDAAAA|12844|3|34|4|AM|third|night|| +12845|AAAAAAAAOCCDAAAA|12845|3|34|5|AM|third|night|| +12846|AAAAAAAAPCCDAAAA|12846|3|34|6|AM|third|night|| +12847|AAAAAAAAADCDAAAA|12847|3|34|7|AM|third|night|| +12848|AAAAAAAABDCDAAAA|12848|3|34|8|AM|third|night|| +12849|AAAAAAAACDCDAAAA|12849|3|34|9|AM|third|night|| +12850|AAAAAAAADDCDAAAA|12850|3|34|10|AM|third|night|| +12851|AAAAAAAAEDCDAAAA|12851|3|34|11|AM|third|night|| +12852|AAAAAAAAFDCDAAAA|12852|3|34|12|AM|third|night|| +12853|AAAAAAAAGDCDAAAA|12853|3|34|13|AM|third|night|| +12854|AAAAAAAAHDCDAAAA|12854|3|34|14|AM|third|night|| +12855|AAAAAAAAIDCDAAAA|12855|3|34|15|AM|third|night|| +12856|AAAAAAAAJDCDAAAA|12856|3|34|16|AM|third|night|| +12857|AAAAAAAAKDCDAAAA|12857|3|34|17|AM|third|night|| +12858|AAAAAAAALDCDAAAA|12858|3|34|18|AM|third|night|| +12859|AAAAAAAAMDCDAAAA|12859|3|34|19|AM|third|night|| +12860|AAAAAAAANDCDAAAA|12860|3|34|20|AM|third|night|| +12861|AAAAAAAAODCDAAAA|12861|3|34|21|AM|third|night|| +12862|AAAAAAAAPDCDAAAA|12862|3|34|22|AM|third|night|| +12863|AAAAAAAAAECDAAAA|12863|3|34|23|AM|third|night|| +12864|AAAAAAAABECDAAAA|12864|3|34|24|AM|third|night|| +12865|AAAAAAAACECDAAAA|12865|3|34|25|AM|third|night|| +12866|AAAAAAAADECDAAAA|12866|3|34|26|AM|third|night|| +12867|AAAAAAAAEECDAAAA|12867|3|34|27|AM|third|night|| +12868|AAAAAAAAFECDAAAA|12868|3|34|28|AM|third|night|| +12869|AAAAAAAAGECDAAAA|12869|3|34|29|AM|third|night|| +12870|AAAAAAAAHECDAAAA|12870|3|34|30|AM|third|night|| +12871|AAAAAAAAIECDAAAA|12871|3|34|31|AM|third|night|| +12872|AAAAAAAAJECDAAAA|12872|3|34|32|AM|third|night|| +12873|AAAAAAAAKECDAAAA|12873|3|34|33|AM|third|night|| +12874|AAAAAAAALECDAAAA|12874|3|34|34|AM|third|night|| +12875|AAAAAAAAMECDAAAA|12875|3|34|35|AM|third|night|| +12876|AAAAAAAANECDAAAA|12876|3|34|36|AM|third|night|| +12877|AAAAAAAAOECDAAAA|12877|3|34|37|AM|third|night|| +12878|AAAAAAAAPECDAAAA|12878|3|34|38|AM|third|night|| +12879|AAAAAAAAAFCDAAAA|12879|3|34|39|AM|third|night|| +12880|AAAAAAAABFCDAAAA|12880|3|34|40|AM|third|night|| +12881|AAAAAAAACFCDAAAA|12881|3|34|41|AM|third|night|| +12882|AAAAAAAADFCDAAAA|12882|3|34|42|AM|third|night|| +12883|AAAAAAAAEFCDAAAA|12883|3|34|43|AM|third|night|| +12884|AAAAAAAAFFCDAAAA|12884|3|34|44|AM|third|night|| +12885|AAAAAAAAGFCDAAAA|12885|3|34|45|AM|third|night|| +12886|AAAAAAAAHFCDAAAA|12886|3|34|46|AM|third|night|| +12887|AAAAAAAAIFCDAAAA|12887|3|34|47|AM|third|night|| +12888|AAAAAAAAJFCDAAAA|12888|3|34|48|AM|third|night|| +12889|AAAAAAAAKFCDAAAA|12889|3|34|49|AM|third|night|| +12890|AAAAAAAALFCDAAAA|12890|3|34|50|AM|third|night|| +12891|AAAAAAAAMFCDAAAA|12891|3|34|51|AM|third|night|| +12892|AAAAAAAANFCDAAAA|12892|3|34|52|AM|third|night|| +12893|AAAAAAAAOFCDAAAA|12893|3|34|53|AM|third|night|| +12894|AAAAAAAAPFCDAAAA|12894|3|34|54|AM|third|night|| +12895|AAAAAAAAAGCDAAAA|12895|3|34|55|AM|third|night|| +12896|AAAAAAAABGCDAAAA|12896|3|34|56|AM|third|night|| +12897|AAAAAAAACGCDAAAA|12897|3|34|57|AM|third|night|| +12898|AAAAAAAADGCDAAAA|12898|3|34|58|AM|third|night|| +12899|AAAAAAAAEGCDAAAA|12899|3|34|59|AM|third|night|| +12900|AAAAAAAAFGCDAAAA|12900|3|35|0|AM|third|night|| +12901|AAAAAAAAGGCDAAAA|12901|3|35|1|AM|third|night|| +12902|AAAAAAAAHGCDAAAA|12902|3|35|2|AM|third|night|| +12903|AAAAAAAAIGCDAAAA|12903|3|35|3|AM|third|night|| +12904|AAAAAAAAJGCDAAAA|12904|3|35|4|AM|third|night|| +12905|AAAAAAAAKGCDAAAA|12905|3|35|5|AM|third|night|| +12906|AAAAAAAALGCDAAAA|12906|3|35|6|AM|third|night|| +12907|AAAAAAAAMGCDAAAA|12907|3|35|7|AM|third|night|| +12908|AAAAAAAANGCDAAAA|12908|3|35|8|AM|third|night|| +12909|AAAAAAAAOGCDAAAA|12909|3|35|9|AM|third|night|| +12910|AAAAAAAAPGCDAAAA|12910|3|35|10|AM|third|night|| +12911|AAAAAAAAAHCDAAAA|12911|3|35|11|AM|third|night|| +12912|AAAAAAAABHCDAAAA|12912|3|35|12|AM|third|night|| +12913|AAAAAAAACHCDAAAA|12913|3|35|13|AM|third|night|| +12914|AAAAAAAADHCDAAAA|12914|3|35|14|AM|third|night|| +12915|AAAAAAAAEHCDAAAA|12915|3|35|15|AM|third|night|| +12916|AAAAAAAAFHCDAAAA|12916|3|35|16|AM|third|night|| +12917|AAAAAAAAGHCDAAAA|12917|3|35|17|AM|third|night|| +12918|AAAAAAAAHHCDAAAA|12918|3|35|18|AM|third|night|| +12919|AAAAAAAAIHCDAAAA|12919|3|35|19|AM|third|night|| +12920|AAAAAAAAJHCDAAAA|12920|3|35|20|AM|third|night|| +12921|AAAAAAAAKHCDAAAA|12921|3|35|21|AM|third|night|| +12922|AAAAAAAALHCDAAAA|12922|3|35|22|AM|third|night|| +12923|AAAAAAAAMHCDAAAA|12923|3|35|23|AM|third|night|| +12924|AAAAAAAANHCDAAAA|12924|3|35|24|AM|third|night|| +12925|AAAAAAAAOHCDAAAA|12925|3|35|25|AM|third|night|| +12926|AAAAAAAAPHCDAAAA|12926|3|35|26|AM|third|night|| +12927|AAAAAAAAAICDAAAA|12927|3|35|27|AM|third|night|| +12928|AAAAAAAABICDAAAA|12928|3|35|28|AM|third|night|| +12929|AAAAAAAACICDAAAA|12929|3|35|29|AM|third|night|| +12930|AAAAAAAADICDAAAA|12930|3|35|30|AM|third|night|| +12931|AAAAAAAAEICDAAAA|12931|3|35|31|AM|third|night|| +12932|AAAAAAAAFICDAAAA|12932|3|35|32|AM|third|night|| +12933|AAAAAAAAGICDAAAA|12933|3|35|33|AM|third|night|| +12934|AAAAAAAAHICDAAAA|12934|3|35|34|AM|third|night|| +12935|AAAAAAAAIICDAAAA|12935|3|35|35|AM|third|night|| +12936|AAAAAAAAJICDAAAA|12936|3|35|36|AM|third|night|| +12937|AAAAAAAAKICDAAAA|12937|3|35|37|AM|third|night|| +12938|AAAAAAAALICDAAAA|12938|3|35|38|AM|third|night|| +12939|AAAAAAAAMICDAAAA|12939|3|35|39|AM|third|night|| +12940|AAAAAAAANICDAAAA|12940|3|35|40|AM|third|night|| +12941|AAAAAAAAOICDAAAA|12941|3|35|41|AM|third|night|| +12942|AAAAAAAAPICDAAAA|12942|3|35|42|AM|third|night|| +12943|AAAAAAAAAJCDAAAA|12943|3|35|43|AM|third|night|| +12944|AAAAAAAABJCDAAAA|12944|3|35|44|AM|third|night|| +12945|AAAAAAAACJCDAAAA|12945|3|35|45|AM|third|night|| +12946|AAAAAAAADJCDAAAA|12946|3|35|46|AM|third|night|| +12947|AAAAAAAAEJCDAAAA|12947|3|35|47|AM|third|night|| +12948|AAAAAAAAFJCDAAAA|12948|3|35|48|AM|third|night|| +12949|AAAAAAAAGJCDAAAA|12949|3|35|49|AM|third|night|| +12950|AAAAAAAAHJCDAAAA|12950|3|35|50|AM|third|night|| +12951|AAAAAAAAIJCDAAAA|12951|3|35|51|AM|third|night|| +12952|AAAAAAAAJJCDAAAA|12952|3|35|52|AM|third|night|| +12953|AAAAAAAAKJCDAAAA|12953|3|35|53|AM|third|night|| +12954|AAAAAAAALJCDAAAA|12954|3|35|54|AM|third|night|| +12955|AAAAAAAAMJCDAAAA|12955|3|35|55|AM|third|night|| +12956|AAAAAAAANJCDAAAA|12956|3|35|56|AM|third|night|| +12957|AAAAAAAAOJCDAAAA|12957|3|35|57|AM|third|night|| +12958|AAAAAAAAPJCDAAAA|12958|3|35|58|AM|third|night|| +12959|AAAAAAAAAKCDAAAA|12959|3|35|59|AM|third|night|| +12960|AAAAAAAABKCDAAAA|12960|3|36|0|AM|third|night|| +12961|AAAAAAAACKCDAAAA|12961|3|36|1|AM|third|night|| +12962|AAAAAAAADKCDAAAA|12962|3|36|2|AM|third|night|| +12963|AAAAAAAAEKCDAAAA|12963|3|36|3|AM|third|night|| +12964|AAAAAAAAFKCDAAAA|12964|3|36|4|AM|third|night|| +12965|AAAAAAAAGKCDAAAA|12965|3|36|5|AM|third|night|| +12966|AAAAAAAAHKCDAAAA|12966|3|36|6|AM|third|night|| +12967|AAAAAAAAIKCDAAAA|12967|3|36|7|AM|third|night|| +12968|AAAAAAAAJKCDAAAA|12968|3|36|8|AM|third|night|| +12969|AAAAAAAAKKCDAAAA|12969|3|36|9|AM|third|night|| +12970|AAAAAAAALKCDAAAA|12970|3|36|10|AM|third|night|| +12971|AAAAAAAAMKCDAAAA|12971|3|36|11|AM|third|night|| +12972|AAAAAAAANKCDAAAA|12972|3|36|12|AM|third|night|| +12973|AAAAAAAAOKCDAAAA|12973|3|36|13|AM|third|night|| +12974|AAAAAAAAPKCDAAAA|12974|3|36|14|AM|third|night|| +12975|AAAAAAAAALCDAAAA|12975|3|36|15|AM|third|night|| +12976|AAAAAAAABLCDAAAA|12976|3|36|16|AM|third|night|| +12977|AAAAAAAACLCDAAAA|12977|3|36|17|AM|third|night|| +12978|AAAAAAAADLCDAAAA|12978|3|36|18|AM|third|night|| +12979|AAAAAAAAELCDAAAA|12979|3|36|19|AM|third|night|| +12980|AAAAAAAAFLCDAAAA|12980|3|36|20|AM|third|night|| +12981|AAAAAAAAGLCDAAAA|12981|3|36|21|AM|third|night|| +12982|AAAAAAAAHLCDAAAA|12982|3|36|22|AM|third|night|| +12983|AAAAAAAAILCDAAAA|12983|3|36|23|AM|third|night|| +12984|AAAAAAAAJLCDAAAA|12984|3|36|24|AM|third|night|| +12985|AAAAAAAAKLCDAAAA|12985|3|36|25|AM|third|night|| +12986|AAAAAAAALLCDAAAA|12986|3|36|26|AM|third|night|| +12987|AAAAAAAAMLCDAAAA|12987|3|36|27|AM|third|night|| +12988|AAAAAAAANLCDAAAA|12988|3|36|28|AM|third|night|| +12989|AAAAAAAAOLCDAAAA|12989|3|36|29|AM|third|night|| +12990|AAAAAAAAPLCDAAAA|12990|3|36|30|AM|third|night|| +12991|AAAAAAAAAMCDAAAA|12991|3|36|31|AM|third|night|| +12992|AAAAAAAABMCDAAAA|12992|3|36|32|AM|third|night|| +12993|AAAAAAAACMCDAAAA|12993|3|36|33|AM|third|night|| +12994|AAAAAAAADMCDAAAA|12994|3|36|34|AM|third|night|| +12995|AAAAAAAAEMCDAAAA|12995|3|36|35|AM|third|night|| +12996|AAAAAAAAFMCDAAAA|12996|3|36|36|AM|third|night|| +12997|AAAAAAAAGMCDAAAA|12997|3|36|37|AM|third|night|| +12998|AAAAAAAAHMCDAAAA|12998|3|36|38|AM|third|night|| +12999|AAAAAAAAIMCDAAAA|12999|3|36|39|AM|third|night|| +13000|AAAAAAAAJMCDAAAA|13000|3|36|40|AM|third|night|| +13001|AAAAAAAAKMCDAAAA|13001|3|36|41|AM|third|night|| +13002|AAAAAAAALMCDAAAA|13002|3|36|42|AM|third|night|| +13003|AAAAAAAAMMCDAAAA|13003|3|36|43|AM|third|night|| +13004|AAAAAAAANMCDAAAA|13004|3|36|44|AM|third|night|| +13005|AAAAAAAAOMCDAAAA|13005|3|36|45|AM|third|night|| +13006|AAAAAAAAPMCDAAAA|13006|3|36|46|AM|third|night|| +13007|AAAAAAAAANCDAAAA|13007|3|36|47|AM|third|night|| +13008|AAAAAAAABNCDAAAA|13008|3|36|48|AM|third|night|| +13009|AAAAAAAACNCDAAAA|13009|3|36|49|AM|third|night|| +13010|AAAAAAAADNCDAAAA|13010|3|36|50|AM|third|night|| +13011|AAAAAAAAENCDAAAA|13011|3|36|51|AM|third|night|| +13012|AAAAAAAAFNCDAAAA|13012|3|36|52|AM|third|night|| +13013|AAAAAAAAGNCDAAAA|13013|3|36|53|AM|third|night|| +13014|AAAAAAAAHNCDAAAA|13014|3|36|54|AM|third|night|| +13015|AAAAAAAAINCDAAAA|13015|3|36|55|AM|third|night|| +13016|AAAAAAAAJNCDAAAA|13016|3|36|56|AM|third|night|| +13017|AAAAAAAAKNCDAAAA|13017|3|36|57|AM|third|night|| +13018|AAAAAAAALNCDAAAA|13018|3|36|58|AM|third|night|| +13019|AAAAAAAAMNCDAAAA|13019|3|36|59|AM|third|night|| +13020|AAAAAAAANNCDAAAA|13020|3|37|0|AM|third|night|| +13021|AAAAAAAAONCDAAAA|13021|3|37|1|AM|third|night|| +13022|AAAAAAAAPNCDAAAA|13022|3|37|2|AM|third|night|| +13023|AAAAAAAAAOCDAAAA|13023|3|37|3|AM|third|night|| +13024|AAAAAAAABOCDAAAA|13024|3|37|4|AM|third|night|| +13025|AAAAAAAACOCDAAAA|13025|3|37|5|AM|third|night|| +13026|AAAAAAAADOCDAAAA|13026|3|37|6|AM|third|night|| +13027|AAAAAAAAEOCDAAAA|13027|3|37|7|AM|third|night|| +13028|AAAAAAAAFOCDAAAA|13028|3|37|8|AM|third|night|| +13029|AAAAAAAAGOCDAAAA|13029|3|37|9|AM|third|night|| +13030|AAAAAAAAHOCDAAAA|13030|3|37|10|AM|third|night|| +13031|AAAAAAAAIOCDAAAA|13031|3|37|11|AM|third|night|| +13032|AAAAAAAAJOCDAAAA|13032|3|37|12|AM|third|night|| +13033|AAAAAAAAKOCDAAAA|13033|3|37|13|AM|third|night|| +13034|AAAAAAAALOCDAAAA|13034|3|37|14|AM|third|night|| +13035|AAAAAAAAMOCDAAAA|13035|3|37|15|AM|third|night|| +13036|AAAAAAAANOCDAAAA|13036|3|37|16|AM|third|night|| +13037|AAAAAAAAOOCDAAAA|13037|3|37|17|AM|third|night|| +13038|AAAAAAAAPOCDAAAA|13038|3|37|18|AM|third|night|| +13039|AAAAAAAAAPCDAAAA|13039|3|37|19|AM|third|night|| +13040|AAAAAAAABPCDAAAA|13040|3|37|20|AM|third|night|| +13041|AAAAAAAACPCDAAAA|13041|3|37|21|AM|third|night|| +13042|AAAAAAAADPCDAAAA|13042|3|37|22|AM|third|night|| +13043|AAAAAAAAEPCDAAAA|13043|3|37|23|AM|third|night|| +13044|AAAAAAAAFPCDAAAA|13044|3|37|24|AM|third|night|| +13045|AAAAAAAAGPCDAAAA|13045|3|37|25|AM|third|night|| +13046|AAAAAAAAHPCDAAAA|13046|3|37|26|AM|third|night|| +13047|AAAAAAAAIPCDAAAA|13047|3|37|27|AM|third|night|| +13048|AAAAAAAAJPCDAAAA|13048|3|37|28|AM|third|night|| +13049|AAAAAAAAKPCDAAAA|13049|3|37|29|AM|third|night|| +13050|AAAAAAAALPCDAAAA|13050|3|37|30|AM|third|night|| +13051|AAAAAAAAMPCDAAAA|13051|3|37|31|AM|third|night|| +13052|AAAAAAAANPCDAAAA|13052|3|37|32|AM|third|night|| +13053|AAAAAAAAOPCDAAAA|13053|3|37|33|AM|third|night|| +13054|AAAAAAAAPPCDAAAA|13054|3|37|34|AM|third|night|| +13055|AAAAAAAAAADDAAAA|13055|3|37|35|AM|third|night|| +13056|AAAAAAAABADDAAAA|13056|3|37|36|AM|third|night|| +13057|AAAAAAAACADDAAAA|13057|3|37|37|AM|third|night|| +13058|AAAAAAAADADDAAAA|13058|3|37|38|AM|third|night|| +13059|AAAAAAAAEADDAAAA|13059|3|37|39|AM|third|night|| +13060|AAAAAAAAFADDAAAA|13060|3|37|40|AM|third|night|| +13061|AAAAAAAAGADDAAAA|13061|3|37|41|AM|third|night|| +13062|AAAAAAAAHADDAAAA|13062|3|37|42|AM|third|night|| +13063|AAAAAAAAIADDAAAA|13063|3|37|43|AM|third|night|| +13064|AAAAAAAAJADDAAAA|13064|3|37|44|AM|third|night|| +13065|AAAAAAAAKADDAAAA|13065|3|37|45|AM|third|night|| +13066|AAAAAAAALADDAAAA|13066|3|37|46|AM|third|night|| +13067|AAAAAAAAMADDAAAA|13067|3|37|47|AM|third|night|| +13068|AAAAAAAANADDAAAA|13068|3|37|48|AM|third|night|| +13069|AAAAAAAAOADDAAAA|13069|3|37|49|AM|third|night|| +13070|AAAAAAAAPADDAAAA|13070|3|37|50|AM|third|night|| +13071|AAAAAAAAABDDAAAA|13071|3|37|51|AM|third|night|| +13072|AAAAAAAABBDDAAAA|13072|3|37|52|AM|third|night|| +13073|AAAAAAAACBDDAAAA|13073|3|37|53|AM|third|night|| +13074|AAAAAAAADBDDAAAA|13074|3|37|54|AM|third|night|| +13075|AAAAAAAAEBDDAAAA|13075|3|37|55|AM|third|night|| +13076|AAAAAAAAFBDDAAAA|13076|3|37|56|AM|third|night|| +13077|AAAAAAAAGBDDAAAA|13077|3|37|57|AM|third|night|| +13078|AAAAAAAAHBDDAAAA|13078|3|37|58|AM|third|night|| +13079|AAAAAAAAIBDDAAAA|13079|3|37|59|AM|third|night|| +13080|AAAAAAAAJBDDAAAA|13080|3|38|0|AM|third|night|| +13081|AAAAAAAAKBDDAAAA|13081|3|38|1|AM|third|night|| +13082|AAAAAAAALBDDAAAA|13082|3|38|2|AM|third|night|| +13083|AAAAAAAAMBDDAAAA|13083|3|38|3|AM|third|night|| +13084|AAAAAAAANBDDAAAA|13084|3|38|4|AM|third|night|| +13085|AAAAAAAAOBDDAAAA|13085|3|38|5|AM|third|night|| +13086|AAAAAAAAPBDDAAAA|13086|3|38|6|AM|third|night|| +13087|AAAAAAAAACDDAAAA|13087|3|38|7|AM|third|night|| +13088|AAAAAAAABCDDAAAA|13088|3|38|8|AM|third|night|| +13089|AAAAAAAACCDDAAAA|13089|3|38|9|AM|third|night|| +13090|AAAAAAAADCDDAAAA|13090|3|38|10|AM|third|night|| +13091|AAAAAAAAECDDAAAA|13091|3|38|11|AM|third|night|| +13092|AAAAAAAAFCDDAAAA|13092|3|38|12|AM|third|night|| +13093|AAAAAAAAGCDDAAAA|13093|3|38|13|AM|third|night|| +13094|AAAAAAAAHCDDAAAA|13094|3|38|14|AM|third|night|| +13095|AAAAAAAAICDDAAAA|13095|3|38|15|AM|third|night|| +13096|AAAAAAAAJCDDAAAA|13096|3|38|16|AM|third|night|| +13097|AAAAAAAAKCDDAAAA|13097|3|38|17|AM|third|night|| +13098|AAAAAAAALCDDAAAA|13098|3|38|18|AM|third|night|| +13099|AAAAAAAAMCDDAAAA|13099|3|38|19|AM|third|night|| +13100|AAAAAAAANCDDAAAA|13100|3|38|20|AM|third|night|| +13101|AAAAAAAAOCDDAAAA|13101|3|38|21|AM|third|night|| +13102|AAAAAAAAPCDDAAAA|13102|3|38|22|AM|third|night|| +13103|AAAAAAAAADDDAAAA|13103|3|38|23|AM|third|night|| +13104|AAAAAAAABDDDAAAA|13104|3|38|24|AM|third|night|| +13105|AAAAAAAACDDDAAAA|13105|3|38|25|AM|third|night|| +13106|AAAAAAAADDDDAAAA|13106|3|38|26|AM|third|night|| +13107|AAAAAAAAEDDDAAAA|13107|3|38|27|AM|third|night|| +13108|AAAAAAAAFDDDAAAA|13108|3|38|28|AM|third|night|| +13109|AAAAAAAAGDDDAAAA|13109|3|38|29|AM|third|night|| +13110|AAAAAAAAHDDDAAAA|13110|3|38|30|AM|third|night|| +13111|AAAAAAAAIDDDAAAA|13111|3|38|31|AM|third|night|| +13112|AAAAAAAAJDDDAAAA|13112|3|38|32|AM|third|night|| +13113|AAAAAAAAKDDDAAAA|13113|3|38|33|AM|third|night|| +13114|AAAAAAAALDDDAAAA|13114|3|38|34|AM|third|night|| +13115|AAAAAAAAMDDDAAAA|13115|3|38|35|AM|third|night|| +13116|AAAAAAAANDDDAAAA|13116|3|38|36|AM|third|night|| +13117|AAAAAAAAODDDAAAA|13117|3|38|37|AM|third|night|| +13118|AAAAAAAAPDDDAAAA|13118|3|38|38|AM|third|night|| +13119|AAAAAAAAAEDDAAAA|13119|3|38|39|AM|third|night|| +13120|AAAAAAAABEDDAAAA|13120|3|38|40|AM|third|night|| +13121|AAAAAAAACEDDAAAA|13121|3|38|41|AM|third|night|| +13122|AAAAAAAADEDDAAAA|13122|3|38|42|AM|third|night|| +13123|AAAAAAAAEEDDAAAA|13123|3|38|43|AM|third|night|| +13124|AAAAAAAAFEDDAAAA|13124|3|38|44|AM|third|night|| +13125|AAAAAAAAGEDDAAAA|13125|3|38|45|AM|third|night|| +13126|AAAAAAAAHEDDAAAA|13126|3|38|46|AM|third|night|| +13127|AAAAAAAAIEDDAAAA|13127|3|38|47|AM|third|night|| +13128|AAAAAAAAJEDDAAAA|13128|3|38|48|AM|third|night|| +13129|AAAAAAAAKEDDAAAA|13129|3|38|49|AM|third|night|| +13130|AAAAAAAALEDDAAAA|13130|3|38|50|AM|third|night|| +13131|AAAAAAAAMEDDAAAA|13131|3|38|51|AM|third|night|| +13132|AAAAAAAANEDDAAAA|13132|3|38|52|AM|third|night|| +13133|AAAAAAAAOEDDAAAA|13133|3|38|53|AM|third|night|| +13134|AAAAAAAAPEDDAAAA|13134|3|38|54|AM|third|night|| +13135|AAAAAAAAAFDDAAAA|13135|3|38|55|AM|third|night|| +13136|AAAAAAAABFDDAAAA|13136|3|38|56|AM|third|night|| +13137|AAAAAAAACFDDAAAA|13137|3|38|57|AM|third|night|| +13138|AAAAAAAADFDDAAAA|13138|3|38|58|AM|third|night|| +13139|AAAAAAAAEFDDAAAA|13139|3|38|59|AM|third|night|| +13140|AAAAAAAAFFDDAAAA|13140|3|39|0|AM|third|night|| +13141|AAAAAAAAGFDDAAAA|13141|3|39|1|AM|third|night|| +13142|AAAAAAAAHFDDAAAA|13142|3|39|2|AM|third|night|| +13143|AAAAAAAAIFDDAAAA|13143|3|39|3|AM|third|night|| +13144|AAAAAAAAJFDDAAAA|13144|3|39|4|AM|third|night|| +13145|AAAAAAAAKFDDAAAA|13145|3|39|5|AM|third|night|| +13146|AAAAAAAALFDDAAAA|13146|3|39|6|AM|third|night|| +13147|AAAAAAAAMFDDAAAA|13147|3|39|7|AM|third|night|| +13148|AAAAAAAANFDDAAAA|13148|3|39|8|AM|third|night|| +13149|AAAAAAAAOFDDAAAA|13149|3|39|9|AM|third|night|| +13150|AAAAAAAAPFDDAAAA|13150|3|39|10|AM|third|night|| +13151|AAAAAAAAAGDDAAAA|13151|3|39|11|AM|third|night|| +13152|AAAAAAAABGDDAAAA|13152|3|39|12|AM|third|night|| +13153|AAAAAAAACGDDAAAA|13153|3|39|13|AM|third|night|| +13154|AAAAAAAADGDDAAAA|13154|3|39|14|AM|third|night|| +13155|AAAAAAAAEGDDAAAA|13155|3|39|15|AM|third|night|| +13156|AAAAAAAAFGDDAAAA|13156|3|39|16|AM|third|night|| +13157|AAAAAAAAGGDDAAAA|13157|3|39|17|AM|third|night|| +13158|AAAAAAAAHGDDAAAA|13158|3|39|18|AM|third|night|| +13159|AAAAAAAAIGDDAAAA|13159|3|39|19|AM|third|night|| +13160|AAAAAAAAJGDDAAAA|13160|3|39|20|AM|third|night|| +13161|AAAAAAAAKGDDAAAA|13161|3|39|21|AM|third|night|| +13162|AAAAAAAALGDDAAAA|13162|3|39|22|AM|third|night|| +13163|AAAAAAAAMGDDAAAA|13163|3|39|23|AM|third|night|| +13164|AAAAAAAANGDDAAAA|13164|3|39|24|AM|third|night|| +13165|AAAAAAAAOGDDAAAA|13165|3|39|25|AM|third|night|| +13166|AAAAAAAAPGDDAAAA|13166|3|39|26|AM|third|night|| +13167|AAAAAAAAAHDDAAAA|13167|3|39|27|AM|third|night|| +13168|AAAAAAAABHDDAAAA|13168|3|39|28|AM|third|night|| +13169|AAAAAAAACHDDAAAA|13169|3|39|29|AM|third|night|| +13170|AAAAAAAADHDDAAAA|13170|3|39|30|AM|third|night|| +13171|AAAAAAAAEHDDAAAA|13171|3|39|31|AM|third|night|| +13172|AAAAAAAAFHDDAAAA|13172|3|39|32|AM|third|night|| +13173|AAAAAAAAGHDDAAAA|13173|3|39|33|AM|third|night|| +13174|AAAAAAAAHHDDAAAA|13174|3|39|34|AM|third|night|| +13175|AAAAAAAAIHDDAAAA|13175|3|39|35|AM|third|night|| +13176|AAAAAAAAJHDDAAAA|13176|3|39|36|AM|third|night|| +13177|AAAAAAAAKHDDAAAA|13177|3|39|37|AM|third|night|| +13178|AAAAAAAALHDDAAAA|13178|3|39|38|AM|third|night|| +13179|AAAAAAAAMHDDAAAA|13179|3|39|39|AM|third|night|| +13180|AAAAAAAANHDDAAAA|13180|3|39|40|AM|third|night|| +13181|AAAAAAAAOHDDAAAA|13181|3|39|41|AM|third|night|| +13182|AAAAAAAAPHDDAAAA|13182|3|39|42|AM|third|night|| +13183|AAAAAAAAAIDDAAAA|13183|3|39|43|AM|third|night|| +13184|AAAAAAAABIDDAAAA|13184|3|39|44|AM|third|night|| +13185|AAAAAAAACIDDAAAA|13185|3|39|45|AM|third|night|| +13186|AAAAAAAADIDDAAAA|13186|3|39|46|AM|third|night|| +13187|AAAAAAAAEIDDAAAA|13187|3|39|47|AM|third|night|| +13188|AAAAAAAAFIDDAAAA|13188|3|39|48|AM|third|night|| +13189|AAAAAAAAGIDDAAAA|13189|3|39|49|AM|third|night|| +13190|AAAAAAAAHIDDAAAA|13190|3|39|50|AM|third|night|| +13191|AAAAAAAAIIDDAAAA|13191|3|39|51|AM|third|night|| +13192|AAAAAAAAJIDDAAAA|13192|3|39|52|AM|third|night|| +13193|AAAAAAAAKIDDAAAA|13193|3|39|53|AM|third|night|| +13194|AAAAAAAALIDDAAAA|13194|3|39|54|AM|third|night|| +13195|AAAAAAAAMIDDAAAA|13195|3|39|55|AM|third|night|| +13196|AAAAAAAANIDDAAAA|13196|3|39|56|AM|third|night|| +13197|AAAAAAAAOIDDAAAA|13197|3|39|57|AM|third|night|| +13198|AAAAAAAAPIDDAAAA|13198|3|39|58|AM|third|night|| +13199|AAAAAAAAAJDDAAAA|13199|3|39|59|AM|third|night|| +13200|AAAAAAAABJDDAAAA|13200|3|40|0|AM|third|night|| +13201|AAAAAAAACJDDAAAA|13201|3|40|1|AM|third|night|| +13202|AAAAAAAADJDDAAAA|13202|3|40|2|AM|third|night|| +13203|AAAAAAAAEJDDAAAA|13203|3|40|3|AM|third|night|| +13204|AAAAAAAAFJDDAAAA|13204|3|40|4|AM|third|night|| +13205|AAAAAAAAGJDDAAAA|13205|3|40|5|AM|third|night|| +13206|AAAAAAAAHJDDAAAA|13206|3|40|6|AM|third|night|| +13207|AAAAAAAAIJDDAAAA|13207|3|40|7|AM|third|night|| +13208|AAAAAAAAJJDDAAAA|13208|3|40|8|AM|third|night|| +13209|AAAAAAAAKJDDAAAA|13209|3|40|9|AM|third|night|| +13210|AAAAAAAALJDDAAAA|13210|3|40|10|AM|third|night|| +13211|AAAAAAAAMJDDAAAA|13211|3|40|11|AM|third|night|| +13212|AAAAAAAANJDDAAAA|13212|3|40|12|AM|third|night|| +13213|AAAAAAAAOJDDAAAA|13213|3|40|13|AM|third|night|| +13214|AAAAAAAAPJDDAAAA|13214|3|40|14|AM|third|night|| +13215|AAAAAAAAAKDDAAAA|13215|3|40|15|AM|third|night|| +13216|AAAAAAAABKDDAAAA|13216|3|40|16|AM|third|night|| +13217|AAAAAAAACKDDAAAA|13217|3|40|17|AM|third|night|| +13218|AAAAAAAADKDDAAAA|13218|3|40|18|AM|third|night|| +13219|AAAAAAAAEKDDAAAA|13219|3|40|19|AM|third|night|| +13220|AAAAAAAAFKDDAAAA|13220|3|40|20|AM|third|night|| +13221|AAAAAAAAGKDDAAAA|13221|3|40|21|AM|third|night|| +13222|AAAAAAAAHKDDAAAA|13222|3|40|22|AM|third|night|| +13223|AAAAAAAAIKDDAAAA|13223|3|40|23|AM|third|night|| +13224|AAAAAAAAJKDDAAAA|13224|3|40|24|AM|third|night|| +13225|AAAAAAAAKKDDAAAA|13225|3|40|25|AM|third|night|| +13226|AAAAAAAALKDDAAAA|13226|3|40|26|AM|third|night|| +13227|AAAAAAAAMKDDAAAA|13227|3|40|27|AM|third|night|| +13228|AAAAAAAANKDDAAAA|13228|3|40|28|AM|third|night|| +13229|AAAAAAAAOKDDAAAA|13229|3|40|29|AM|third|night|| +13230|AAAAAAAAPKDDAAAA|13230|3|40|30|AM|third|night|| +13231|AAAAAAAAALDDAAAA|13231|3|40|31|AM|third|night|| +13232|AAAAAAAABLDDAAAA|13232|3|40|32|AM|third|night|| +13233|AAAAAAAACLDDAAAA|13233|3|40|33|AM|third|night|| +13234|AAAAAAAADLDDAAAA|13234|3|40|34|AM|third|night|| +13235|AAAAAAAAELDDAAAA|13235|3|40|35|AM|third|night|| +13236|AAAAAAAAFLDDAAAA|13236|3|40|36|AM|third|night|| +13237|AAAAAAAAGLDDAAAA|13237|3|40|37|AM|third|night|| +13238|AAAAAAAAHLDDAAAA|13238|3|40|38|AM|third|night|| +13239|AAAAAAAAILDDAAAA|13239|3|40|39|AM|third|night|| +13240|AAAAAAAAJLDDAAAA|13240|3|40|40|AM|third|night|| +13241|AAAAAAAAKLDDAAAA|13241|3|40|41|AM|third|night|| +13242|AAAAAAAALLDDAAAA|13242|3|40|42|AM|third|night|| +13243|AAAAAAAAMLDDAAAA|13243|3|40|43|AM|third|night|| +13244|AAAAAAAANLDDAAAA|13244|3|40|44|AM|third|night|| +13245|AAAAAAAAOLDDAAAA|13245|3|40|45|AM|third|night|| +13246|AAAAAAAAPLDDAAAA|13246|3|40|46|AM|third|night|| +13247|AAAAAAAAAMDDAAAA|13247|3|40|47|AM|third|night|| +13248|AAAAAAAABMDDAAAA|13248|3|40|48|AM|third|night|| +13249|AAAAAAAACMDDAAAA|13249|3|40|49|AM|third|night|| +13250|AAAAAAAADMDDAAAA|13250|3|40|50|AM|third|night|| +13251|AAAAAAAAEMDDAAAA|13251|3|40|51|AM|third|night|| +13252|AAAAAAAAFMDDAAAA|13252|3|40|52|AM|third|night|| +13253|AAAAAAAAGMDDAAAA|13253|3|40|53|AM|third|night|| +13254|AAAAAAAAHMDDAAAA|13254|3|40|54|AM|third|night|| +13255|AAAAAAAAIMDDAAAA|13255|3|40|55|AM|third|night|| +13256|AAAAAAAAJMDDAAAA|13256|3|40|56|AM|third|night|| +13257|AAAAAAAAKMDDAAAA|13257|3|40|57|AM|third|night|| +13258|AAAAAAAALMDDAAAA|13258|3|40|58|AM|third|night|| +13259|AAAAAAAAMMDDAAAA|13259|3|40|59|AM|third|night|| +13260|AAAAAAAANMDDAAAA|13260|3|41|0|AM|third|night|| +13261|AAAAAAAAOMDDAAAA|13261|3|41|1|AM|third|night|| +13262|AAAAAAAAPMDDAAAA|13262|3|41|2|AM|third|night|| +13263|AAAAAAAAANDDAAAA|13263|3|41|3|AM|third|night|| +13264|AAAAAAAABNDDAAAA|13264|3|41|4|AM|third|night|| +13265|AAAAAAAACNDDAAAA|13265|3|41|5|AM|third|night|| +13266|AAAAAAAADNDDAAAA|13266|3|41|6|AM|third|night|| +13267|AAAAAAAAENDDAAAA|13267|3|41|7|AM|third|night|| +13268|AAAAAAAAFNDDAAAA|13268|3|41|8|AM|third|night|| +13269|AAAAAAAAGNDDAAAA|13269|3|41|9|AM|third|night|| +13270|AAAAAAAAHNDDAAAA|13270|3|41|10|AM|third|night|| +13271|AAAAAAAAINDDAAAA|13271|3|41|11|AM|third|night|| +13272|AAAAAAAAJNDDAAAA|13272|3|41|12|AM|third|night|| +13273|AAAAAAAAKNDDAAAA|13273|3|41|13|AM|third|night|| +13274|AAAAAAAALNDDAAAA|13274|3|41|14|AM|third|night|| +13275|AAAAAAAAMNDDAAAA|13275|3|41|15|AM|third|night|| +13276|AAAAAAAANNDDAAAA|13276|3|41|16|AM|third|night|| +13277|AAAAAAAAONDDAAAA|13277|3|41|17|AM|third|night|| +13278|AAAAAAAAPNDDAAAA|13278|3|41|18|AM|third|night|| +13279|AAAAAAAAAODDAAAA|13279|3|41|19|AM|third|night|| +13280|AAAAAAAABODDAAAA|13280|3|41|20|AM|third|night|| +13281|AAAAAAAACODDAAAA|13281|3|41|21|AM|third|night|| +13282|AAAAAAAADODDAAAA|13282|3|41|22|AM|third|night|| +13283|AAAAAAAAEODDAAAA|13283|3|41|23|AM|third|night|| +13284|AAAAAAAAFODDAAAA|13284|3|41|24|AM|third|night|| +13285|AAAAAAAAGODDAAAA|13285|3|41|25|AM|third|night|| +13286|AAAAAAAAHODDAAAA|13286|3|41|26|AM|third|night|| +13287|AAAAAAAAIODDAAAA|13287|3|41|27|AM|third|night|| +13288|AAAAAAAAJODDAAAA|13288|3|41|28|AM|third|night|| +13289|AAAAAAAAKODDAAAA|13289|3|41|29|AM|third|night|| +13290|AAAAAAAALODDAAAA|13290|3|41|30|AM|third|night|| +13291|AAAAAAAAMODDAAAA|13291|3|41|31|AM|third|night|| +13292|AAAAAAAANODDAAAA|13292|3|41|32|AM|third|night|| +13293|AAAAAAAAOODDAAAA|13293|3|41|33|AM|third|night|| +13294|AAAAAAAAPODDAAAA|13294|3|41|34|AM|third|night|| +13295|AAAAAAAAAPDDAAAA|13295|3|41|35|AM|third|night|| +13296|AAAAAAAABPDDAAAA|13296|3|41|36|AM|third|night|| +13297|AAAAAAAACPDDAAAA|13297|3|41|37|AM|third|night|| +13298|AAAAAAAADPDDAAAA|13298|3|41|38|AM|third|night|| +13299|AAAAAAAAEPDDAAAA|13299|3|41|39|AM|third|night|| +13300|AAAAAAAAFPDDAAAA|13300|3|41|40|AM|third|night|| +13301|AAAAAAAAGPDDAAAA|13301|3|41|41|AM|third|night|| +13302|AAAAAAAAHPDDAAAA|13302|3|41|42|AM|third|night|| +13303|AAAAAAAAIPDDAAAA|13303|3|41|43|AM|third|night|| +13304|AAAAAAAAJPDDAAAA|13304|3|41|44|AM|third|night|| +13305|AAAAAAAAKPDDAAAA|13305|3|41|45|AM|third|night|| +13306|AAAAAAAALPDDAAAA|13306|3|41|46|AM|third|night|| +13307|AAAAAAAAMPDDAAAA|13307|3|41|47|AM|third|night|| +13308|AAAAAAAANPDDAAAA|13308|3|41|48|AM|third|night|| +13309|AAAAAAAAOPDDAAAA|13309|3|41|49|AM|third|night|| +13310|AAAAAAAAPPDDAAAA|13310|3|41|50|AM|third|night|| +13311|AAAAAAAAAAEDAAAA|13311|3|41|51|AM|third|night|| +13312|AAAAAAAABAEDAAAA|13312|3|41|52|AM|third|night|| +13313|AAAAAAAACAEDAAAA|13313|3|41|53|AM|third|night|| +13314|AAAAAAAADAEDAAAA|13314|3|41|54|AM|third|night|| +13315|AAAAAAAAEAEDAAAA|13315|3|41|55|AM|third|night|| +13316|AAAAAAAAFAEDAAAA|13316|3|41|56|AM|third|night|| +13317|AAAAAAAAGAEDAAAA|13317|3|41|57|AM|third|night|| +13318|AAAAAAAAHAEDAAAA|13318|3|41|58|AM|third|night|| +13319|AAAAAAAAIAEDAAAA|13319|3|41|59|AM|third|night|| +13320|AAAAAAAAJAEDAAAA|13320|3|42|0|AM|third|night|| +13321|AAAAAAAAKAEDAAAA|13321|3|42|1|AM|third|night|| +13322|AAAAAAAALAEDAAAA|13322|3|42|2|AM|third|night|| +13323|AAAAAAAAMAEDAAAA|13323|3|42|3|AM|third|night|| +13324|AAAAAAAANAEDAAAA|13324|3|42|4|AM|third|night|| +13325|AAAAAAAAOAEDAAAA|13325|3|42|5|AM|third|night|| +13326|AAAAAAAAPAEDAAAA|13326|3|42|6|AM|third|night|| +13327|AAAAAAAAABEDAAAA|13327|3|42|7|AM|third|night|| +13328|AAAAAAAABBEDAAAA|13328|3|42|8|AM|third|night|| +13329|AAAAAAAACBEDAAAA|13329|3|42|9|AM|third|night|| +13330|AAAAAAAADBEDAAAA|13330|3|42|10|AM|third|night|| +13331|AAAAAAAAEBEDAAAA|13331|3|42|11|AM|third|night|| +13332|AAAAAAAAFBEDAAAA|13332|3|42|12|AM|third|night|| +13333|AAAAAAAAGBEDAAAA|13333|3|42|13|AM|third|night|| +13334|AAAAAAAAHBEDAAAA|13334|3|42|14|AM|third|night|| +13335|AAAAAAAAIBEDAAAA|13335|3|42|15|AM|third|night|| +13336|AAAAAAAAJBEDAAAA|13336|3|42|16|AM|third|night|| +13337|AAAAAAAAKBEDAAAA|13337|3|42|17|AM|third|night|| +13338|AAAAAAAALBEDAAAA|13338|3|42|18|AM|third|night|| +13339|AAAAAAAAMBEDAAAA|13339|3|42|19|AM|third|night|| +13340|AAAAAAAANBEDAAAA|13340|3|42|20|AM|third|night|| +13341|AAAAAAAAOBEDAAAA|13341|3|42|21|AM|third|night|| +13342|AAAAAAAAPBEDAAAA|13342|3|42|22|AM|third|night|| +13343|AAAAAAAAACEDAAAA|13343|3|42|23|AM|third|night|| +13344|AAAAAAAABCEDAAAA|13344|3|42|24|AM|third|night|| +13345|AAAAAAAACCEDAAAA|13345|3|42|25|AM|third|night|| +13346|AAAAAAAADCEDAAAA|13346|3|42|26|AM|third|night|| +13347|AAAAAAAAECEDAAAA|13347|3|42|27|AM|third|night|| +13348|AAAAAAAAFCEDAAAA|13348|3|42|28|AM|third|night|| +13349|AAAAAAAAGCEDAAAA|13349|3|42|29|AM|third|night|| +13350|AAAAAAAAHCEDAAAA|13350|3|42|30|AM|third|night|| +13351|AAAAAAAAICEDAAAA|13351|3|42|31|AM|third|night|| +13352|AAAAAAAAJCEDAAAA|13352|3|42|32|AM|third|night|| +13353|AAAAAAAAKCEDAAAA|13353|3|42|33|AM|third|night|| +13354|AAAAAAAALCEDAAAA|13354|3|42|34|AM|third|night|| +13355|AAAAAAAAMCEDAAAA|13355|3|42|35|AM|third|night|| +13356|AAAAAAAANCEDAAAA|13356|3|42|36|AM|third|night|| +13357|AAAAAAAAOCEDAAAA|13357|3|42|37|AM|third|night|| +13358|AAAAAAAAPCEDAAAA|13358|3|42|38|AM|third|night|| +13359|AAAAAAAAADEDAAAA|13359|3|42|39|AM|third|night|| +13360|AAAAAAAABDEDAAAA|13360|3|42|40|AM|third|night|| +13361|AAAAAAAACDEDAAAA|13361|3|42|41|AM|third|night|| +13362|AAAAAAAADDEDAAAA|13362|3|42|42|AM|third|night|| +13363|AAAAAAAAEDEDAAAA|13363|3|42|43|AM|third|night|| +13364|AAAAAAAAFDEDAAAA|13364|3|42|44|AM|third|night|| +13365|AAAAAAAAGDEDAAAA|13365|3|42|45|AM|third|night|| +13366|AAAAAAAAHDEDAAAA|13366|3|42|46|AM|third|night|| +13367|AAAAAAAAIDEDAAAA|13367|3|42|47|AM|third|night|| +13368|AAAAAAAAJDEDAAAA|13368|3|42|48|AM|third|night|| +13369|AAAAAAAAKDEDAAAA|13369|3|42|49|AM|third|night|| +13370|AAAAAAAALDEDAAAA|13370|3|42|50|AM|third|night|| +13371|AAAAAAAAMDEDAAAA|13371|3|42|51|AM|third|night|| +13372|AAAAAAAANDEDAAAA|13372|3|42|52|AM|third|night|| +13373|AAAAAAAAODEDAAAA|13373|3|42|53|AM|third|night|| +13374|AAAAAAAAPDEDAAAA|13374|3|42|54|AM|third|night|| +13375|AAAAAAAAAEEDAAAA|13375|3|42|55|AM|third|night|| +13376|AAAAAAAABEEDAAAA|13376|3|42|56|AM|third|night|| +13377|AAAAAAAACEEDAAAA|13377|3|42|57|AM|third|night|| +13378|AAAAAAAADEEDAAAA|13378|3|42|58|AM|third|night|| +13379|AAAAAAAAEEEDAAAA|13379|3|42|59|AM|third|night|| +13380|AAAAAAAAFEEDAAAA|13380|3|43|0|AM|third|night|| +13381|AAAAAAAAGEEDAAAA|13381|3|43|1|AM|third|night|| +13382|AAAAAAAAHEEDAAAA|13382|3|43|2|AM|third|night|| +13383|AAAAAAAAIEEDAAAA|13383|3|43|3|AM|third|night|| +13384|AAAAAAAAJEEDAAAA|13384|3|43|4|AM|third|night|| +13385|AAAAAAAAKEEDAAAA|13385|3|43|5|AM|third|night|| +13386|AAAAAAAALEEDAAAA|13386|3|43|6|AM|third|night|| +13387|AAAAAAAAMEEDAAAA|13387|3|43|7|AM|third|night|| +13388|AAAAAAAANEEDAAAA|13388|3|43|8|AM|third|night|| +13389|AAAAAAAAOEEDAAAA|13389|3|43|9|AM|third|night|| +13390|AAAAAAAAPEEDAAAA|13390|3|43|10|AM|third|night|| +13391|AAAAAAAAAFEDAAAA|13391|3|43|11|AM|third|night|| +13392|AAAAAAAABFEDAAAA|13392|3|43|12|AM|third|night|| +13393|AAAAAAAACFEDAAAA|13393|3|43|13|AM|third|night|| +13394|AAAAAAAADFEDAAAA|13394|3|43|14|AM|third|night|| +13395|AAAAAAAAEFEDAAAA|13395|3|43|15|AM|third|night|| +13396|AAAAAAAAFFEDAAAA|13396|3|43|16|AM|third|night|| +13397|AAAAAAAAGFEDAAAA|13397|3|43|17|AM|third|night|| +13398|AAAAAAAAHFEDAAAA|13398|3|43|18|AM|third|night|| +13399|AAAAAAAAIFEDAAAA|13399|3|43|19|AM|third|night|| +13400|AAAAAAAAJFEDAAAA|13400|3|43|20|AM|third|night|| +13401|AAAAAAAAKFEDAAAA|13401|3|43|21|AM|third|night|| +13402|AAAAAAAALFEDAAAA|13402|3|43|22|AM|third|night|| +13403|AAAAAAAAMFEDAAAA|13403|3|43|23|AM|third|night|| +13404|AAAAAAAANFEDAAAA|13404|3|43|24|AM|third|night|| +13405|AAAAAAAAOFEDAAAA|13405|3|43|25|AM|third|night|| +13406|AAAAAAAAPFEDAAAA|13406|3|43|26|AM|third|night|| +13407|AAAAAAAAAGEDAAAA|13407|3|43|27|AM|third|night|| +13408|AAAAAAAABGEDAAAA|13408|3|43|28|AM|third|night|| +13409|AAAAAAAACGEDAAAA|13409|3|43|29|AM|third|night|| +13410|AAAAAAAADGEDAAAA|13410|3|43|30|AM|third|night|| +13411|AAAAAAAAEGEDAAAA|13411|3|43|31|AM|third|night|| +13412|AAAAAAAAFGEDAAAA|13412|3|43|32|AM|third|night|| +13413|AAAAAAAAGGEDAAAA|13413|3|43|33|AM|third|night|| +13414|AAAAAAAAHGEDAAAA|13414|3|43|34|AM|third|night|| +13415|AAAAAAAAIGEDAAAA|13415|3|43|35|AM|third|night|| +13416|AAAAAAAAJGEDAAAA|13416|3|43|36|AM|third|night|| +13417|AAAAAAAAKGEDAAAA|13417|3|43|37|AM|third|night|| +13418|AAAAAAAALGEDAAAA|13418|3|43|38|AM|third|night|| +13419|AAAAAAAAMGEDAAAA|13419|3|43|39|AM|third|night|| +13420|AAAAAAAANGEDAAAA|13420|3|43|40|AM|third|night|| +13421|AAAAAAAAOGEDAAAA|13421|3|43|41|AM|third|night|| +13422|AAAAAAAAPGEDAAAA|13422|3|43|42|AM|third|night|| +13423|AAAAAAAAAHEDAAAA|13423|3|43|43|AM|third|night|| +13424|AAAAAAAABHEDAAAA|13424|3|43|44|AM|third|night|| +13425|AAAAAAAACHEDAAAA|13425|3|43|45|AM|third|night|| +13426|AAAAAAAADHEDAAAA|13426|3|43|46|AM|third|night|| +13427|AAAAAAAAEHEDAAAA|13427|3|43|47|AM|third|night|| +13428|AAAAAAAAFHEDAAAA|13428|3|43|48|AM|third|night|| +13429|AAAAAAAAGHEDAAAA|13429|3|43|49|AM|third|night|| +13430|AAAAAAAAHHEDAAAA|13430|3|43|50|AM|third|night|| +13431|AAAAAAAAIHEDAAAA|13431|3|43|51|AM|third|night|| +13432|AAAAAAAAJHEDAAAA|13432|3|43|52|AM|third|night|| +13433|AAAAAAAAKHEDAAAA|13433|3|43|53|AM|third|night|| +13434|AAAAAAAALHEDAAAA|13434|3|43|54|AM|third|night|| +13435|AAAAAAAAMHEDAAAA|13435|3|43|55|AM|third|night|| +13436|AAAAAAAANHEDAAAA|13436|3|43|56|AM|third|night|| +13437|AAAAAAAAOHEDAAAA|13437|3|43|57|AM|third|night|| +13438|AAAAAAAAPHEDAAAA|13438|3|43|58|AM|third|night|| +13439|AAAAAAAAAIEDAAAA|13439|3|43|59|AM|third|night|| +13440|AAAAAAAABIEDAAAA|13440|3|44|0|AM|third|night|| +13441|AAAAAAAACIEDAAAA|13441|3|44|1|AM|third|night|| +13442|AAAAAAAADIEDAAAA|13442|3|44|2|AM|third|night|| +13443|AAAAAAAAEIEDAAAA|13443|3|44|3|AM|third|night|| +13444|AAAAAAAAFIEDAAAA|13444|3|44|4|AM|third|night|| +13445|AAAAAAAAGIEDAAAA|13445|3|44|5|AM|third|night|| +13446|AAAAAAAAHIEDAAAA|13446|3|44|6|AM|third|night|| +13447|AAAAAAAAIIEDAAAA|13447|3|44|7|AM|third|night|| +13448|AAAAAAAAJIEDAAAA|13448|3|44|8|AM|third|night|| +13449|AAAAAAAAKIEDAAAA|13449|3|44|9|AM|third|night|| +13450|AAAAAAAALIEDAAAA|13450|3|44|10|AM|third|night|| +13451|AAAAAAAAMIEDAAAA|13451|3|44|11|AM|third|night|| +13452|AAAAAAAANIEDAAAA|13452|3|44|12|AM|third|night|| +13453|AAAAAAAAOIEDAAAA|13453|3|44|13|AM|third|night|| +13454|AAAAAAAAPIEDAAAA|13454|3|44|14|AM|third|night|| +13455|AAAAAAAAAJEDAAAA|13455|3|44|15|AM|third|night|| +13456|AAAAAAAABJEDAAAA|13456|3|44|16|AM|third|night|| +13457|AAAAAAAACJEDAAAA|13457|3|44|17|AM|third|night|| +13458|AAAAAAAADJEDAAAA|13458|3|44|18|AM|third|night|| +13459|AAAAAAAAEJEDAAAA|13459|3|44|19|AM|third|night|| +13460|AAAAAAAAFJEDAAAA|13460|3|44|20|AM|third|night|| +13461|AAAAAAAAGJEDAAAA|13461|3|44|21|AM|third|night|| +13462|AAAAAAAAHJEDAAAA|13462|3|44|22|AM|third|night|| +13463|AAAAAAAAIJEDAAAA|13463|3|44|23|AM|third|night|| +13464|AAAAAAAAJJEDAAAA|13464|3|44|24|AM|third|night|| +13465|AAAAAAAAKJEDAAAA|13465|3|44|25|AM|third|night|| +13466|AAAAAAAALJEDAAAA|13466|3|44|26|AM|third|night|| +13467|AAAAAAAAMJEDAAAA|13467|3|44|27|AM|third|night|| +13468|AAAAAAAANJEDAAAA|13468|3|44|28|AM|third|night|| +13469|AAAAAAAAOJEDAAAA|13469|3|44|29|AM|third|night|| +13470|AAAAAAAAPJEDAAAA|13470|3|44|30|AM|third|night|| +13471|AAAAAAAAAKEDAAAA|13471|3|44|31|AM|third|night|| +13472|AAAAAAAABKEDAAAA|13472|3|44|32|AM|third|night|| +13473|AAAAAAAACKEDAAAA|13473|3|44|33|AM|third|night|| +13474|AAAAAAAADKEDAAAA|13474|3|44|34|AM|third|night|| +13475|AAAAAAAAEKEDAAAA|13475|3|44|35|AM|third|night|| +13476|AAAAAAAAFKEDAAAA|13476|3|44|36|AM|third|night|| +13477|AAAAAAAAGKEDAAAA|13477|3|44|37|AM|third|night|| +13478|AAAAAAAAHKEDAAAA|13478|3|44|38|AM|third|night|| +13479|AAAAAAAAIKEDAAAA|13479|3|44|39|AM|third|night|| +13480|AAAAAAAAJKEDAAAA|13480|3|44|40|AM|third|night|| +13481|AAAAAAAAKKEDAAAA|13481|3|44|41|AM|third|night|| +13482|AAAAAAAALKEDAAAA|13482|3|44|42|AM|third|night|| +13483|AAAAAAAAMKEDAAAA|13483|3|44|43|AM|third|night|| +13484|AAAAAAAANKEDAAAA|13484|3|44|44|AM|third|night|| +13485|AAAAAAAAOKEDAAAA|13485|3|44|45|AM|third|night|| +13486|AAAAAAAAPKEDAAAA|13486|3|44|46|AM|third|night|| +13487|AAAAAAAAALEDAAAA|13487|3|44|47|AM|third|night|| +13488|AAAAAAAABLEDAAAA|13488|3|44|48|AM|third|night|| +13489|AAAAAAAACLEDAAAA|13489|3|44|49|AM|third|night|| +13490|AAAAAAAADLEDAAAA|13490|3|44|50|AM|third|night|| +13491|AAAAAAAAELEDAAAA|13491|3|44|51|AM|third|night|| +13492|AAAAAAAAFLEDAAAA|13492|3|44|52|AM|third|night|| +13493|AAAAAAAAGLEDAAAA|13493|3|44|53|AM|third|night|| +13494|AAAAAAAAHLEDAAAA|13494|3|44|54|AM|third|night|| +13495|AAAAAAAAILEDAAAA|13495|3|44|55|AM|third|night|| +13496|AAAAAAAAJLEDAAAA|13496|3|44|56|AM|third|night|| +13497|AAAAAAAAKLEDAAAA|13497|3|44|57|AM|third|night|| +13498|AAAAAAAALLEDAAAA|13498|3|44|58|AM|third|night|| +13499|AAAAAAAAMLEDAAAA|13499|3|44|59|AM|third|night|| +13500|AAAAAAAANLEDAAAA|13500|3|45|0|AM|third|night|| +13501|AAAAAAAAOLEDAAAA|13501|3|45|1|AM|third|night|| +13502|AAAAAAAAPLEDAAAA|13502|3|45|2|AM|third|night|| +13503|AAAAAAAAAMEDAAAA|13503|3|45|3|AM|third|night|| +13504|AAAAAAAABMEDAAAA|13504|3|45|4|AM|third|night|| +13505|AAAAAAAACMEDAAAA|13505|3|45|5|AM|third|night|| +13506|AAAAAAAADMEDAAAA|13506|3|45|6|AM|third|night|| +13507|AAAAAAAAEMEDAAAA|13507|3|45|7|AM|third|night|| +13508|AAAAAAAAFMEDAAAA|13508|3|45|8|AM|third|night|| +13509|AAAAAAAAGMEDAAAA|13509|3|45|9|AM|third|night|| +13510|AAAAAAAAHMEDAAAA|13510|3|45|10|AM|third|night|| +13511|AAAAAAAAIMEDAAAA|13511|3|45|11|AM|third|night|| +13512|AAAAAAAAJMEDAAAA|13512|3|45|12|AM|third|night|| +13513|AAAAAAAAKMEDAAAA|13513|3|45|13|AM|third|night|| +13514|AAAAAAAALMEDAAAA|13514|3|45|14|AM|third|night|| +13515|AAAAAAAAMMEDAAAA|13515|3|45|15|AM|third|night|| +13516|AAAAAAAANMEDAAAA|13516|3|45|16|AM|third|night|| +13517|AAAAAAAAOMEDAAAA|13517|3|45|17|AM|third|night|| +13518|AAAAAAAAPMEDAAAA|13518|3|45|18|AM|third|night|| +13519|AAAAAAAAANEDAAAA|13519|3|45|19|AM|third|night|| +13520|AAAAAAAABNEDAAAA|13520|3|45|20|AM|third|night|| +13521|AAAAAAAACNEDAAAA|13521|3|45|21|AM|third|night|| +13522|AAAAAAAADNEDAAAA|13522|3|45|22|AM|third|night|| +13523|AAAAAAAAENEDAAAA|13523|3|45|23|AM|third|night|| +13524|AAAAAAAAFNEDAAAA|13524|3|45|24|AM|third|night|| +13525|AAAAAAAAGNEDAAAA|13525|3|45|25|AM|third|night|| +13526|AAAAAAAAHNEDAAAA|13526|3|45|26|AM|third|night|| +13527|AAAAAAAAINEDAAAA|13527|3|45|27|AM|third|night|| +13528|AAAAAAAAJNEDAAAA|13528|3|45|28|AM|third|night|| +13529|AAAAAAAAKNEDAAAA|13529|3|45|29|AM|third|night|| +13530|AAAAAAAALNEDAAAA|13530|3|45|30|AM|third|night|| +13531|AAAAAAAAMNEDAAAA|13531|3|45|31|AM|third|night|| +13532|AAAAAAAANNEDAAAA|13532|3|45|32|AM|third|night|| +13533|AAAAAAAAONEDAAAA|13533|3|45|33|AM|third|night|| +13534|AAAAAAAAPNEDAAAA|13534|3|45|34|AM|third|night|| +13535|AAAAAAAAAOEDAAAA|13535|3|45|35|AM|third|night|| +13536|AAAAAAAABOEDAAAA|13536|3|45|36|AM|third|night|| +13537|AAAAAAAACOEDAAAA|13537|3|45|37|AM|third|night|| +13538|AAAAAAAADOEDAAAA|13538|3|45|38|AM|third|night|| +13539|AAAAAAAAEOEDAAAA|13539|3|45|39|AM|third|night|| +13540|AAAAAAAAFOEDAAAA|13540|3|45|40|AM|third|night|| +13541|AAAAAAAAGOEDAAAA|13541|3|45|41|AM|third|night|| +13542|AAAAAAAAHOEDAAAA|13542|3|45|42|AM|third|night|| +13543|AAAAAAAAIOEDAAAA|13543|3|45|43|AM|third|night|| +13544|AAAAAAAAJOEDAAAA|13544|3|45|44|AM|third|night|| +13545|AAAAAAAAKOEDAAAA|13545|3|45|45|AM|third|night|| +13546|AAAAAAAALOEDAAAA|13546|3|45|46|AM|third|night|| +13547|AAAAAAAAMOEDAAAA|13547|3|45|47|AM|third|night|| +13548|AAAAAAAANOEDAAAA|13548|3|45|48|AM|third|night|| +13549|AAAAAAAAOOEDAAAA|13549|3|45|49|AM|third|night|| +13550|AAAAAAAAPOEDAAAA|13550|3|45|50|AM|third|night|| +13551|AAAAAAAAAPEDAAAA|13551|3|45|51|AM|third|night|| +13552|AAAAAAAABPEDAAAA|13552|3|45|52|AM|third|night|| +13553|AAAAAAAACPEDAAAA|13553|3|45|53|AM|third|night|| +13554|AAAAAAAADPEDAAAA|13554|3|45|54|AM|third|night|| +13555|AAAAAAAAEPEDAAAA|13555|3|45|55|AM|third|night|| +13556|AAAAAAAAFPEDAAAA|13556|3|45|56|AM|third|night|| +13557|AAAAAAAAGPEDAAAA|13557|3|45|57|AM|third|night|| +13558|AAAAAAAAHPEDAAAA|13558|3|45|58|AM|third|night|| +13559|AAAAAAAAIPEDAAAA|13559|3|45|59|AM|third|night|| +13560|AAAAAAAAJPEDAAAA|13560|3|46|0|AM|third|night|| +13561|AAAAAAAAKPEDAAAA|13561|3|46|1|AM|third|night|| +13562|AAAAAAAALPEDAAAA|13562|3|46|2|AM|third|night|| +13563|AAAAAAAAMPEDAAAA|13563|3|46|3|AM|third|night|| +13564|AAAAAAAANPEDAAAA|13564|3|46|4|AM|third|night|| +13565|AAAAAAAAOPEDAAAA|13565|3|46|5|AM|third|night|| +13566|AAAAAAAAPPEDAAAA|13566|3|46|6|AM|third|night|| +13567|AAAAAAAAAAFDAAAA|13567|3|46|7|AM|third|night|| +13568|AAAAAAAABAFDAAAA|13568|3|46|8|AM|third|night|| +13569|AAAAAAAACAFDAAAA|13569|3|46|9|AM|third|night|| +13570|AAAAAAAADAFDAAAA|13570|3|46|10|AM|third|night|| +13571|AAAAAAAAEAFDAAAA|13571|3|46|11|AM|third|night|| +13572|AAAAAAAAFAFDAAAA|13572|3|46|12|AM|third|night|| +13573|AAAAAAAAGAFDAAAA|13573|3|46|13|AM|third|night|| +13574|AAAAAAAAHAFDAAAA|13574|3|46|14|AM|third|night|| +13575|AAAAAAAAIAFDAAAA|13575|3|46|15|AM|third|night|| +13576|AAAAAAAAJAFDAAAA|13576|3|46|16|AM|third|night|| +13577|AAAAAAAAKAFDAAAA|13577|3|46|17|AM|third|night|| +13578|AAAAAAAALAFDAAAA|13578|3|46|18|AM|third|night|| +13579|AAAAAAAAMAFDAAAA|13579|3|46|19|AM|third|night|| +13580|AAAAAAAANAFDAAAA|13580|3|46|20|AM|third|night|| +13581|AAAAAAAAOAFDAAAA|13581|3|46|21|AM|third|night|| +13582|AAAAAAAAPAFDAAAA|13582|3|46|22|AM|third|night|| +13583|AAAAAAAAABFDAAAA|13583|3|46|23|AM|third|night|| +13584|AAAAAAAABBFDAAAA|13584|3|46|24|AM|third|night|| +13585|AAAAAAAACBFDAAAA|13585|3|46|25|AM|third|night|| +13586|AAAAAAAADBFDAAAA|13586|3|46|26|AM|third|night|| +13587|AAAAAAAAEBFDAAAA|13587|3|46|27|AM|third|night|| +13588|AAAAAAAAFBFDAAAA|13588|3|46|28|AM|third|night|| +13589|AAAAAAAAGBFDAAAA|13589|3|46|29|AM|third|night|| +13590|AAAAAAAAHBFDAAAA|13590|3|46|30|AM|third|night|| +13591|AAAAAAAAIBFDAAAA|13591|3|46|31|AM|third|night|| +13592|AAAAAAAAJBFDAAAA|13592|3|46|32|AM|third|night|| +13593|AAAAAAAAKBFDAAAA|13593|3|46|33|AM|third|night|| +13594|AAAAAAAALBFDAAAA|13594|3|46|34|AM|third|night|| +13595|AAAAAAAAMBFDAAAA|13595|3|46|35|AM|third|night|| +13596|AAAAAAAANBFDAAAA|13596|3|46|36|AM|third|night|| +13597|AAAAAAAAOBFDAAAA|13597|3|46|37|AM|third|night|| +13598|AAAAAAAAPBFDAAAA|13598|3|46|38|AM|third|night|| +13599|AAAAAAAAACFDAAAA|13599|3|46|39|AM|third|night|| +13600|AAAAAAAABCFDAAAA|13600|3|46|40|AM|third|night|| +13601|AAAAAAAACCFDAAAA|13601|3|46|41|AM|third|night|| +13602|AAAAAAAADCFDAAAA|13602|3|46|42|AM|third|night|| +13603|AAAAAAAAECFDAAAA|13603|3|46|43|AM|third|night|| +13604|AAAAAAAAFCFDAAAA|13604|3|46|44|AM|third|night|| +13605|AAAAAAAAGCFDAAAA|13605|3|46|45|AM|third|night|| +13606|AAAAAAAAHCFDAAAA|13606|3|46|46|AM|third|night|| +13607|AAAAAAAAICFDAAAA|13607|3|46|47|AM|third|night|| +13608|AAAAAAAAJCFDAAAA|13608|3|46|48|AM|third|night|| +13609|AAAAAAAAKCFDAAAA|13609|3|46|49|AM|third|night|| +13610|AAAAAAAALCFDAAAA|13610|3|46|50|AM|third|night|| +13611|AAAAAAAAMCFDAAAA|13611|3|46|51|AM|third|night|| +13612|AAAAAAAANCFDAAAA|13612|3|46|52|AM|third|night|| +13613|AAAAAAAAOCFDAAAA|13613|3|46|53|AM|third|night|| +13614|AAAAAAAAPCFDAAAA|13614|3|46|54|AM|third|night|| +13615|AAAAAAAAADFDAAAA|13615|3|46|55|AM|third|night|| +13616|AAAAAAAABDFDAAAA|13616|3|46|56|AM|third|night|| +13617|AAAAAAAACDFDAAAA|13617|3|46|57|AM|third|night|| +13618|AAAAAAAADDFDAAAA|13618|3|46|58|AM|third|night|| +13619|AAAAAAAAEDFDAAAA|13619|3|46|59|AM|third|night|| +13620|AAAAAAAAFDFDAAAA|13620|3|47|0|AM|third|night|| +13621|AAAAAAAAGDFDAAAA|13621|3|47|1|AM|third|night|| +13622|AAAAAAAAHDFDAAAA|13622|3|47|2|AM|third|night|| +13623|AAAAAAAAIDFDAAAA|13623|3|47|3|AM|third|night|| +13624|AAAAAAAAJDFDAAAA|13624|3|47|4|AM|third|night|| +13625|AAAAAAAAKDFDAAAA|13625|3|47|5|AM|third|night|| +13626|AAAAAAAALDFDAAAA|13626|3|47|6|AM|third|night|| +13627|AAAAAAAAMDFDAAAA|13627|3|47|7|AM|third|night|| +13628|AAAAAAAANDFDAAAA|13628|3|47|8|AM|third|night|| +13629|AAAAAAAAODFDAAAA|13629|3|47|9|AM|third|night|| +13630|AAAAAAAAPDFDAAAA|13630|3|47|10|AM|third|night|| +13631|AAAAAAAAAEFDAAAA|13631|3|47|11|AM|third|night|| +13632|AAAAAAAABEFDAAAA|13632|3|47|12|AM|third|night|| +13633|AAAAAAAACEFDAAAA|13633|3|47|13|AM|third|night|| +13634|AAAAAAAADEFDAAAA|13634|3|47|14|AM|third|night|| +13635|AAAAAAAAEEFDAAAA|13635|3|47|15|AM|third|night|| +13636|AAAAAAAAFEFDAAAA|13636|3|47|16|AM|third|night|| +13637|AAAAAAAAGEFDAAAA|13637|3|47|17|AM|third|night|| +13638|AAAAAAAAHEFDAAAA|13638|3|47|18|AM|third|night|| +13639|AAAAAAAAIEFDAAAA|13639|3|47|19|AM|third|night|| +13640|AAAAAAAAJEFDAAAA|13640|3|47|20|AM|third|night|| +13641|AAAAAAAAKEFDAAAA|13641|3|47|21|AM|third|night|| +13642|AAAAAAAALEFDAAAA|13642|3|47|22|AM|third|night|| +13643|AAAAAAAAMEFDAAAA|13643|3|47|23|AM|third|night|| +13644|AAAAAAAANEFDAAAA|13644|3|47|24|AM|third|night|| +13645|AAAAAAAAOEFDAAAA|13645|3|47|25|AM|third|night|| +13646|AAAAAAAAPEFDAAAA|13646|3|47|26|AM|third|night|| +13647|AAAAAAAAAFFDAAAA|13647|3|47|27|AM|third|night|| +13648|AAAAAAAABFFDAAAA|13648|3|47|28|AM|third|night|| +13649|AAAAAAAACFFDAAAA|13649|3|47|29|AM|third|night|| +13650|AAAAAAAADFFDAAAA|13650|3|47|30|AM|third|night|| +13651|AAAAAAAAEFFDAAAA|13651|3|47|31|AM|third|night|| +13652|AAAAAAAAFFFDAAAA|13652|3|47|32|AM|third|night|| +13653|AAAAAAAAGFFDAAAA|13653|3|47|33|AM|third|night|| +13654|AAAAAAAAHFFDAAAA|13654|3|47|34|AM|third|night|| +13655|AAAAAAAAIFFDAAAA|13655|3|47|35|AM|third|night|| +13656|AAAAAAAAJFFDAAAA|13656|3|47|36|AM|third|night|| +13657|AAAAAAAAKFFDAAAA|13657|3|47|37|AM|third|night|| +13658|AAAAAAAALFFDAAAA|13658|3|47|38|AM|third|night|| +13659|AAAAAAAAMFFDAAAA|13659|3|47|39|AM|third|night|| +13660|AAAAAAAANFFDAAAA|13660|3|47|40|AM|third|night|| +13661|AAAAAAAAOFFDAAAA|13661|3|47|41|AM|third|night|| +13662|AAAAAAAAPFFDAAAA|13662|3|47|42|AM|third|night|| +13663|AAAAAAAAAGFDAAAA|13663|3|47|43|AM|third|night|| +13664|AAAAAAAABGFDAAAA|13664|3|47|44|AM|third|night|| +13665|AAAAAAAACGFDAAAA|13665|3|47|45|AM|third|night|| +13666|AAAAAAAADGFDAAAA|13666|3|47|46|AM|third|night|| +13667|AAAAAAAAEGFDAAAA|13667|3|47|47|AM|third|night|| +13668|AAAAAAAAFGFDAAAA|13668|3|47|48|AM|third|night|| +13669|AAAAAAAAGGFDAAAA|13669|3|47|49|AM|third|night|| +13670|AAAAAAAAHGFDAAAA|13670|3|47|50|AM|third|night|| +13671|AAAAAAAAIGFDAAAA|13671|3|47|51|AM|third|night|| +13672|AAAAAAAAJGFDAAAA|13672|3|47|52|AM|third|night|| +13673|AAAAAAAAKGFDAAAA|13673|3|47|53|AM|third|night|| +13674|AAAAAAAALGFDAAAA|13674|3|47|54|AM|third|night|| +13675|AAAAAAAAMGFDAAAA|13675|3|47|55|AM|third|night|| +13676|AAAAAAAANGFDAAAA|13676|3|47|56|AM|third|night|| +13677|AAAAAAAAOGFDAAAA|13677|3|47|57|AM|third|night|| +13678|AAAAAAAAPGFDAAAA|13678|3|47|58|AM|third|night|| +13679|AAAAAAAAAHFDAAAA|13679|3|47|59|AM|third|night|| +13680|AAAAAAAABHFDAAAA|13680|3|48|0|AM|third|night|| +13681|AAAAAAAACHFDAAAA|13681|3|48|1|AM|third|night|| +13682|AAAAAAAADHFDAAAA|13682|3|48|2|AM|third|night|| +13683|AAAAAAAAEHFDAAAA|13683|3|48|3|AM|third|night|| +13684|AAAAAAAAFHFDAAAA|13684|3|48|4|AM|third|night|| +13685|AAAAAAAAGHFDAAAA|13685|3|48|5|AM|third|night|| +13686|AAAAAAAAHHFDAAAA|13686|3|48|6|AM|third|night|| +13687|AAAAAAAAIHFDAAAA|13687|3|48|7|AM|third|night|| +13688|AAAAAAAAJHFDAAAA|13688|3|48|8|AM|third|night|| +13689|AAAAAAAAKHFDAAAA|13689|3|48|9|AM|third|night|| +13690|AAAAAAAALHFDAAAA|13690|3|48|10|AM|third|night|| +13691|AAAAAAAAMHFDAAAA|13691|3|48|11|AM|third|night|| +13692|AAAAAAAANHFDAAAA|13692|3|48|12|AM|third|night|| +13693|AAAAAAAAOHFDAAAA|13693|3|48|13|AM|third|night|| +13694|AAAAAAAAPHFDAAAA|13694|3|48|14|AM|third|night|| +13695|AAAAAAAAAIFDAAAA|13695|3|48|15|AM|third|night|| +13696|AAAAAAAABIFDAAAA|13696|3|48|16|AM|third|night|| +13697|AAAAAAAACIFDAAAA|13697|3|48|17|AM|third|night|| +13698|AAAAAAAADIFDAAAA|13698|3|48|18|AM|third|night|| +13699|AAAAAAAAEIFDAAAA|13699|3|48|19|AM|third|night|| +13700|AAAAAAAAFIFDAAAA|13700|3|48|20|AM|third|night|| +13701|AAAAAAAAGIFDAAAA|13701|3|48|21|AM|third|night|| +13702|AAAAAAAAHIFDAAAA|13702|3|48|22|AM|third|night|| +13703|AAAAAAAAIIFDAAAA|13703|3|48|23|AM|third|night|| +13704|AAAAAAAAJIFDAAAA|13704|3|48|24|AM|third|night|| +13705|AAAAAAAAKIFDAAAA|13705|3|48|25|AM|third|night|| +13706|AAAAAAAALIFDAAAA|13706|3|48|26|AM|third|night|| +13707|AAAAAAAAMIFDAAAA|13707|3|48|27|AM|third|night|| +13708|AAAAAAAANIFDAAAA|13708|3|48|28|AM|third|night|| +13709|AAAAAAAAOIFDAAAA|13709|3|48|29|AM|third|night|| +13710|AAAAAAAAPIFDAAAA|13710|3|48|30|AM|third|night|| +13711|AAAAAAAAAJFDAAAA|13711|3|48|31|AM|third|night|| +13712|AAAAAAAABJFDAAAA|13712|3|48|32|AM|third|night|| +13713|AAAAAAAACJFDAAAA|13713|3|48|33|AM|third|night|| +13714|AAAAAAAADJFDAAAA|13714|3|48|34|AM|third|night|| +13715|AAAAAAAAEJFDAAAA|13715|3|48|35|AM|third|night|| +13716|AAAAAAAAFJFDAAAA|13716|3|48|36|AM|third|night|| +13717|AAAAAAAAGJFDAAAA|13717|3|48|37|AM|third|night|| +13718|AAAAAAAAHJFDAAAA|13718|3|48|38|AM|third|night|| +13719|AAAAAAAAIJFDAAAA|13719|3|48|39|AM|third|night|| +13720|AAAAAAAAJJFDAAAA|13720|3|48|40|AM|third|night|| +13721|AAAAAAAAKJFDAAAA|13721|3|48|41|AM|third|night|| +13722|AAAAAAAALJFDAAAA|13722|3|48|42|AM|third|night|| +13723|AAAAAAAAMJFDAAAA|13723|3|48|43|AM|third|night|| +13724|AAAAAAAANJFDAAAA|13724|3|48|44|AM|third|night|| +13725|AAAAAAAAOJFDAAAA|13725|3|48|45|AM|third|night|| +13726|AAAAAAAAPJFDAAAA|13726|3|48|46|AM|third|night|| +13727|AAAAAAAAAKFDAAAA|13727|3|48|47|AM|third|night|| +13728|AAAAAAAABKFDAAAA|13728|3|48|48|AM|third|night|| +13729|AAAAAAAACKFDAAAA|13729|3|48|49|AM|third|night|| +13730|AAAAAAAADKFDAAAA|13730|3|48|50|AM|third|night|| +13731|AAAAAAAAEKFDAAAA|13731|3|48|51|AM|third|night|| +13732|AAAAAAAAFKFDAAAA|13732|3|48|52|AM|third|night|| +13733|AAAAAAAAGKFDAAAA|13733|3|48|53|AM|third|night|| +13734|AAAAAAAAHKFDAAAA|13734|3|48|54|AM|third|night|| +13735|AAAAAAAAIKFDAAAA|13735|3|48|55|AM|third|night|| +13736|AAAAAAAAJKFDAAAA|13736|3|48|56|AM|third|night|| +13737|AAAAAAAAKKFDAAAA|13737|3|48|57|AM|third|night|| +13738|AAAAAAAALKFDAAAA|13738|3|48|58|AM|third|night|| +13739|AAAAAAAAMKFDAAAA|13739|3|48|59|AM|third|night|| +13740|AAAAAAAANKFDAAAA|13740|3|49|0|AM|third|night|| +13741|AAAAAAAAOKFDAAAA|13741|3|49|1|AM|third|night|| +13742|AAAAAAAAPKFDAAAA|13742|3|49|2|AM|third|night|| +13743|AAAAAAAAALFDAAAA|13743|3|49|3|AM|third|night|| +13744|AAAAAAAABLFDAAAA|13744|3|49|4|AM|third|night|| +13745|AAAAAAAACLFDAAAA|13745|3|49|5|AM|third|night|| +13746|AAAAAAAADLFDAAAA|13746|3|49|6|AM|third|night|| +13747|AAAAAAAAELFDAAAA|13747|3|49|7|AM|third|night|| +13748|AAAAAAAAFLFDAAAA|13748|3|49|8|AM|third|night|| +13749|AAAAAAAAGLFDAAAA|13749|3|49|9|AM|third|night|| +13750|AAAAAAAAHLFDAAAA|13750|3|49|10|AM|third|night|| +13751|AAAAAAAAILFDAAAA|13751|3|49|11|AM|third|night|| +13752|AAAAAAAAJLFDAAAA|13752|3|49|12|AM|third|night|| +13753|AAAAAAAAKLFDAAAA|13753|3|49|13|AM|third|night|| +13754|AAAAAAAALLFDAAAA|13754|3|49|14|AM|third|night|| +13755|AAAAAAAAMLFDAAAA|13755|3|49|15|AM|third|night|| +13756|AAAAAAAANLFDAAAA|13756|3|49|16|AM|third|night|| +13757|AAAAAAAAOLFDAAAA|13757|3|49|17|AM|third|night|| +13758|AAAAAAAAPLFDAAAA|13758|3|49|18|AM|third|night|| +13759|AAAAAAAAAMFDAAAA|13759|3|49|19|AM|third|night|| +13760|AAAAAAAABMFDAAAA|13760|3|49|20|AM|third|night|| +13761|AAAAAAAACMFDAAAA|13761|3|49|21|AM|third|night|| +13762|AAAAAAAADMFDAAAA|13762|3|49|22|AM|third|night|| +13763|AAAAAAAAEMFDAAAA|13763|3|49|23|AM|third|night|| +13764|AAAAAAAAFMFDAAAA|13764|3|49|24|AM|third|night|| +13765|AAAAAAAAGMFDAAAA|13765|3|49|25|AM|third|night|| +13766|AAAAAAAAHMFDAAAA|13766|3|49|26|AM|third|night|| +13767|AAAAAAAAIMFDAAAA|13767|3|49|27|AM|third|night|| +13768|AAAAAAAAJMFDAAAA|13768|3|49|28|AM|third|night|| +13769|AAAAAAAAKMFDAAAA|13769|3|49|29|AM|third|night|| +13770|AAAAAAAALMFDAAAA|13770|3|49|30|AM|third|night|| +13771|AAAAAAAAMMFDAAAA|13771|3|49|31|AM|third|night|| +13772|AAAAAAAANMFDAAAA|13772|3|49|32|AM|third|night|| +13773|AAAAAAAAOMFDAAAA|13773|3|49|33|AM|third|night|| +13774|AAAAAAAAPMFDAAAA|13774|3|49|34|AM|third|night|| +13775|AAAAAAAAANFDAAAA|13775|3|49|35|AM|third|night|| +13776|AAAAAAAABNFDAAAA|13776|3|49|36|AM|third|night|| +13777|AAAAAAAACNFDAAAA|13777|3|49|37|AM|third|night|| +13778|AAAAAAAADNFDAAAA|13778|3|49|38|AM|third|night|| +13779|AAAAAAAAENFDAAAA|13779|3|49|39|AM|third|night|| +13780|AAAAAAAAFNFDAAAA|13780|3|49|40|AM|third|night|| +13781|AAAAAAAAGNFDAAAA|13781|3|49|41|AM|third|night|| +13782|AAAAAAAAHNFDAAAA|13782|3|49|42|AM|third|night|| +13783|AAAAAAAAINFDAAAA|13783|3|49|43|AM|third|night|| +13784|AAAAAAAAJNFDAAAA|13784|3|49|44|AM|third|night|| +13785|AAAAAAAAKNFDAAAA|13785|3|49|45|AM|third|night|| +13786|AAAAAAAALNFDAAAA|13786|3|49|46|AM|third|night|| +13787|AAAAAAAAMNFDAAAA|13787|3|49|47|AM|third|night|| +13788|AAAAAAAANNFDAAAA|13788|3|49|48|AM|third|night|| +13789|AAAAAAAAONFDAAAA|13789|3|49|49|AM|third|night|| +13790|AAAAAAAAPNFDAAAA|13790|3|49|50|AM|third|night|| +13791|AAAAAAAAAOFDAAAA|13791|3|49|51|AM|third|night|| +13792|AAAAAAAABOFDAAAA|13792|3|49|52|AM|third|night|| +13793|AAAAAAAACOFDAAAA|13793|3|49|53|AM|third|night|| +13794|AAAAAAAADOFDAAAA|13794|3|49|54|AM|third|night|| +13795|AAAAAAAAEOFDAAAA|13795|3|49|55|AM|third|night|| +13796|AAAAAAAAFOFDAAAA|13796|3|49|56|AM|third|night|| +13797|AAAAAAAAGOFDAAAA|13797|3|49|57|AM|third|night|| +13798|AAAAAAAAHOFDAAAA|13798|3|49|58|AM|third|night|| +13799|AAAAAAAAIOFDAAAA|13799|3|49|59|AM|third|night|| +13800|AAAAAAAAJOFDAAAA|13800|3|50|0|AM|third|night|| +13801|AAAAAAAAKOFDAAAA|13801|3|50|1|AM|third|night|| +13802|AAAAAAAALOFDAAAA|13802|3|50|2|AM|third|night|| +13803|AAAAAAAAMOFDAAAA|13803|3|50|3|AM|third|night|| +13804|AAAAAAAANOFDAAAA|13804|3|50|4|AM|third|night|| +13805|AAAAAAAAOOFDAAAA|13805|3|50|5|AM|third|night|| +13806|AAAAAAAAPOFDAAAA|13806|3|50|6|AM|third|night|| +13807|AAAAAAAAAPFDAAAA|13807|3|50|7|AM|third|night|| +13808|AAAAAAAABPFDAAAA|13808|3|50|8|AM|third|night|| +13809|AAAAAAAACPFDAAAA|13809|3|50|9|AM|third|night|| +13810|AAAAAAAADPFDAAAA|13810|3|50|10|AM|third|night|| +13811|AAAAAAAAEPFDAAAA|13811|3|50|11|AM|third|night|| +13812|AAAAAAAAFPFDAAAA|13812|3|50|12|AM|third|night|| +13813|AAAAAAAAGPFDAAAA|13813|3|50|13|AM|third|night|| +13814|AAAAAAAAHPFDAAAA|13814|3|50|14|AM|third|night|| +13815|AAAAAAAAIPFDAAAA|13815|3|50|15|AM|third|night|| +13816|AAAAAAAAJPFDAAAA|13816|3|50|16|AM|third|night|| +13817|AAAAAAAAKPFDAAAA|13817|3|50|17|AM|third|night|| +13818|AAAAAAAALPFDAAAA|13818|3|50|18|AM|third|night|| +13819|AAAAAAAAMPFDAAAA|13819|3|50|19|AM|third|night|| +13820|AAAAAAAANPFDAAAA|13820|3|50|20|AM|third|night|| +13821|AAAAAAAAOPFDAAAA|13821|3|50|21|AM|third|night|| +13822|AAAAAAAAPPFDAAAA|13822|3|50|22|AM|third|night|| +13823|AAAAAAAAAAGDAAAA|13823|3|50|23|AM|third|night|| +13824|AAAAAAAABAGDAAAA|13824|3|50|24|AM|third|night|| +13825|AAAAAAAACAGDAAAA|13825|3|50|25|AM|third|night|| +13826|AAAAAAAADAGDAAAA|13826|3|50|26|AM|third|night|| +13827|AAAAAAAAEAGDAAAA|13827|3|50|27|AM|third|night|| +13828|AAAAAAAAFAGDAAAA|13828|3|50|28|AM|third|night|| +13829|AAAAAAAAGAGDAAAA|13829|3|50|29|AM|third|night|| +13830|AAAAAAAAHAGDAAAA|13830|3|50|30|AM|third|night|| +13831|AAAAAAAAIAGDAAAA|13831|3|50|31|AM|third|night|| +13832|AAAAAAAAJAGDAAAA|13832|3|50|32|AM|third|night|| +13833|AAAAAAAAKAGDAAAA|13833|3|50|33|AM|third|night|| +13834|AAAAAAAALAGDAAAA|13834|3|50|34|AM|third|night|| +13835|AAAAAAAAMAGDAAAA|13835|3|50|35|AM|third|night|| +13836|AAAAAAAANAGDAAAA|13836|3|50|36|AM|third|night|| +13837|AAAAAAAAOAGDAAAA|13837|3|50|37|AM|third|night|| +13838|AAAAAAAAPAGDAAAA|13838|3|50|38|AM|third|night|| +13839|AAAAAAAAABGDAAAA|13839|3|50|39|AM|third|night|| +13840|AAAAAAAABBGDAAAA|13840|3|50|40|AM|third|night|| +13841|AAAAAAAACBGDAAAA|13841|3|50|41|AM|third|night|| +13842|AAAAAAAADBGDAAAA|13842|3|50|42|AM|third|night|| +13843|AAAAAAAAEBGDAAAA|13843|3|50|43|AM|third|night|| +13844|AAAAAAAAFBGDAAAA|13844|3|50|44|AM|third|night|| +13845|AAAAAAAAGBGDAAAA|13845|3|50|45|AM|third|night|| +13846|AAAAAAAAHBGDAAAA|13846|3|50|46|AM|third|night|| +13847|AAAAAAAAIBGDAAAA|13847|3|50|47|AM|third|night|| +13848|AAAAAAAAJBGDAAAA|13848|3|50|48|AM|third|night|| +13849|AAAAAAAAKBGDAAAA|13849|3|50|49|AM|third|night|| +13850|AAAAAAAALBGDAAAA|13850|3|50|50|AM|third|night|| +13851|AAAAAAAAMBGDAAAA|13851|3|50|51|AM|third|night|| +13852|AAAAAAAANBGDAAAA|13852|3|50|52|AM|third|night|| +13853|AAAAAAAAOBGDAAAA|13853|3|50|53|AM|third|night|| +13854|AAAAAAAAPBGDAAAA|13854|3|50|54|AM|third|night|| +13855|AAAAAAAAACGDAAAA|13855|3|50|55|AM|third|night|| +13856|AAAAAAAABCGDAAAA|13856|3|50|56|AM|third|night|| +13857|AAAAAAAACCGDAAAA|13857|3|50|57|AM|third|night|| +13858|AAAAAAAADCGDAAAA|13858|3|50|58|AM|third|night|| +13859|AAAAAAAAECGDAAAA|13859|3|50|59|AM|third|night|| +13860|AAAAAAAAFCGDAAAA|13860|3|51|0|AM|third|night|| +13861|AAAAAAAAGCGDAAAA|13861|3|51|1|AM|third|night|| +13862|AAAAAAAAHCGDAAAA|13862|3|51|2|AM|third|night|| +13863|AAAAAAAAICGDAAAA|13863|3|51|3|AM|third|night|| +13864|AAAAAAAAJCGDAAAA|13864|3|51|4|AM|third|night|| +13865|AAAAAAAAKCGDAAAA|13865|3|51|5|AM|third|night|| +13866|AAAAAAAALCGDAAAA|13866|3|51|6|AM|third|night|| +13867|AAAAAAAAMCGDAAAA|13867|3|51|7|AM|third|night|| +13868|AAAAAAAANCGDAAAA|13868|3|51|8|AM|third|night|| +13869|AAAAAAAAOCGDAAAA|13869|3|51|9|AM|third|night|| +13870|AAAAAAAAPCGDAAAA|13870|3|51|10|AM|third|night|| +13871|AAAAAAAAADGDAAAA|13871|3|51|11|AM|third|night|| +13872|AAAAAAAABDGDAAAA|13872|3|51|12|AM|third|night|| +13873|AAAAAAAACDGDAAAA|13873|3|51|13|AM|third|night|| +13874|AAAAAAAADDGDAAAA|13874|3|51|14|AM|third|night|| +13875|AAAAAAAAEDGDAAAA|13875|3|51|15|AM|third|night|| +13876|AAAAAAAAFDGDAAAA|13876|3|51|16|AM|third|night|| +13877|AAAAAAAAGDGDAAAA|13877|3|51|17|AM|third|night|| +13878|AAAAAAAAHDGDAAAA|13878|3|51|18|AM|third|night|| +13879|AAAAAAAAIDGDAAAA|13879|3|51|19|AM|third|night|| +13880|AAAAAAAAJDGDAAAA|13880|3|51|20|AM|third|night|| +13881|AAAAAAAAKDGDAAAA|13881|3|51|21|AM|third|night|| +13882|AAAAAAAALDGDAAAA|13882|3|51|22|AM|third|night|| +13883|AAAAAAAAMDGDAAAA|13883|3|51|23|AM|third|night|| +13884|AAAAAAAANDGDAAAA|13884|3|51|24|AM|third|night|| +13885|AAAAAAAAODGDAAAA|13885|3|51|25|AM|third|night|| +13886|AAAAAAAAPDGDAAAA|13886|3|51|26|AM|third|night|| +13887|AAAAAAAAAEGDAAAA|13887|3|51|27|AM|third|night|| +13888|AAAAAAAABEGDAAAA|13888|3|51|28|AM|third|night|| +13889|AAAAAAAACEGDAAAA|13889|3|51|29|AM|third|night|| +13890|AAAAAAAADEGDAAAA|13890|3|51|30|AM|third|night|| +13891|AAAAAAAAEEGDAAAA|13891|3|51|31|AM|third|night|| +13892|AAAAAAAAFEGDAAAA|13892|3|51|32|AM|third|night|| +13893|AAAAAAAAGEGDAAAA|13893|3|51|33|AM|third|night|| +13894|AAAAAAAAHEGDAAAA|13894|3|51|34|AM|third|night|| +13895|AAAAAAAAIEGDAAAA|13895|3|51|35|AM|third|night|| +13896|AAAAAAAAJEGDAAAA|13896|3|51|36|AM|third|night|| +13897|AAAAAAAAKEGDAAAA|13897|3|51|37|AM|third|night|| +13898|AAAAAAAALEGDAAAA|13898|3|51|38|AM|third|night|| +13899|AAAAAAAAMEGDAAAA|13899|3|51|39|AM|third|night|| +13900|AAAAAAAANEGDAAAA|13900|3|51|40|AM|third|night|| +13901|AAAAAAAAOEGDAAAA|13901|3|51|41|AM|third|night|| +13902|AAAAAAAAPEGDAAAA|13902|3|51|42|AM|third|night|| +13903|AAAAAAAAAFGDAAAA|13903|3|51|43|AM|third|night|| +13904|AAAAAAAABFGDAAAA|13904|3|51|44|AM|third|night|| +13905|AAAAAAAACFGDAAAA|13905|3|51|45|AM|third|night|| +13906|AAAAAAAADFGDAAAA|13906|3|51|46|AM|third|night|| +13907|AAAAAAAAEFGDAAAA|13907|3|51|47|AM|third|night|| +13908|AAAAAAAAFFGDAAAA|13908|3|51|48|AM|third|night|| +13909|AAAAAAAAGFGDAAAA|13909|3|51|49|AM|third|night|| +13910|AAAAAAAAHFGDAAAA|13910|3|51|50|AM|third|night|| +13911|AAAAAAAAIFGDAAAA|13911|3|51|51|AM|third|night|| +13912|AAAAAAAAJFGDAAAA|13912|3|51|52|AM|third|night|| +13913|AAAAAAAAKFGDAAAA|13913|3|51|53|AM|third|night|| +13914|AAAAAAAALFGDAAAA|13914|3|51|54|AM|third|night|| +13915|AAAAAAAAMFGDAAAA|13915|3|51|55|AM|third|night|| +13916|AAAAAAAANFGDAAAA|13916|3|51|56|AM|third|night|| +13917|AAAAAAAAOFGDAAAA|13917|3|51|57|AM|third|night|| +13918|AAAAAAAAPFGDAAAA|13918|3|51|58|AM|third|night|| +13919|AAAAAAAAAGGDAAAA|13919|3|51|59|AM|third|night|| +13920|AAAAAAAABGGDAAAA|13920|3|52|0|AM|third|night|| +13921|AAAAAAAACGGDAAAA|13921|3|52|1|AM|third|night|| +13922|AAAAAAAADGGDAAAA|13922|3|52|2|AM|third|night|| +13923|AAAAAAAAEGGDAAAA|13923|3|52|3|AM|third|night|| +13924|AAAAAAAAFGGDAAAA|13924|3|52|4|AM|third|night|| +13925|AAAAAAAAGGGDAAAA|13925|3|52|5|AM|third|night|| +13926|AAAAAAAAHGGDAAAA|13926|3|52|6|AM|third|night|| +13927|AAAAAAAAIGGDAAAA|13927|3|52|7|AM|third|night|| +13928|AAAAAAAAJGGDAAAA|13928|3|52|8|AM|third|night|| +13929|AAAAAAAAKGGDAAAA|13929|3|52|9|AM|third|night|| +13930|AAAAAAAALGGDAAAA|13930|3|52|10|AM|third|night|| +13931|AAAAAAAAMGGDAAAA|13931|3|52|11|AM|third|night|| +13932|AAAAAAAANGGDAAAA|13932|3|52|12|AM|third|night|| +13933|AAAAAAAAOGGDAAAA|13933|3|52|13|AM|third|night|| +13934|AAAAAAAAPGGDAAAA|13934|3|52|14|AM|third|night|| +13935|AAAAAAAAAHGDAAAA|13935|3|52|15|AM|third|night|| +13936|AAAAAAAABHGDAAAA|13936|3|52|16|AM|third|night|| +13937|AAAAAAAACHGDAAAA|13937|3|52|17|AM|third|night|| +13938|AAAAAAAADHGDAAAA|13938|3|52|18|AM|third|night|| +13939|AAAAAAAAEHGDAAAA|13939|3|52|19|AM|third|night|| +13940|AAAAAAAAFHGDAAAA|13940|3|52|20|AM|third|night|| +13941|AAAAAAAAGHGDAAAA|13941|3|52|21|AM|third|night|| +13942|AAAAAAAAHHGDAAAA|13942|3|52|22|AM|third|night|| +13943|AAAAAAAAIHGDAAAA|13943|3|52|23|AM|third|night|| +13944|AAAAAAAAJHGDAAAA|13944|3|52|24|AM|third|night|| +13945|AAAAAAAAKHGDAAAA|13945|3|52|25|AM|third|night|| +13946|AAAAAAAALHGDAAAA|13946|3|52|26|AM|third|night|| +13947|AAAAAAAAMHGDAAAA|13947|3|52|27|AM|third|night|| +13948|AAAAAAAANHGDAAAA|13948|3|52|28|AM|third|night|| +13949|AAAAAAAAOHGDAAAA|13949|3|52|29|AM|third|night|| +13950|AAAAAAAAPHGDAAAA|13950|3|52|30|AM|third|night|| +13951|AAAAAAAAAIGDAAAA|13951|3|52|31|AM|third|night|| +13952|AAAAAAAABIGDAAAA|13952|3|52|32|AM|third|night|| +13953|AAAAAAAACIGDAAAA|13953|3|52|33|AM|third|night|| +13954|AAAAAAAADIGDAAAA|13954|3|52|34|AM|third|night|| +13955|AAAAAAAAEIGDAAAA|13955|3|52|35|AM|third|night|| +13956|AAAAAAAAFIGDAAAA|13956|3|52|36|AM|third|night|| +13957|AAAAAAAAGIGDAAAA|13957|3|52|37|AM|third|night|| +13958|AAAAAAAAHIGDAAAA|13958|3|52|38|AM|third|night|| +13959|AAAAAAAAIIGDAAAA|13959|3|52|39|AM|third|night|| +13960|AAAAAAAAJIGDAAAA|13960|3|52|40|AM|third|night|| +13961|AAAAAAAAKIGDAAAA|13961|3|52|41|AM|third|night|| +13962|AAAAAAAALIGDAAAA|13962|3|52|42|AM|third|night|| +13963|AAAAAAAAMIGDAAAA|13963|3|52|43|AM|third|night|| +13964|AAAAAAAANIGDAAAA|13964|3|52|44|AM|third|night|| +13965|AAAAAAAAOIGDAAAA|13965|3|52|45|AM|third|night|| +13966|AAAAAAAAPIGDAAAA|13966|3|52|46|AM|third|night|| +13967|AAAAAAAAAJGDAAAA|13967|3|52|47|AM|third|night|| +13968|AAAAAAAABJGDAAAA|13968|3|52|48|AM|third|night|| +13969|AAAAAAAACJGDAAAA|13969|3|52|49|AM|third|night|| +13970|AAAAAAAADJGDAAAA|13970|3|52|50|AM|third|night|| +13971|AAAAAAAAEJGDAAAA|13971|3|52|51|AM|third|night|| +13972|AAAAAAAAFJGDAAAA|13972|3|52|52|AM|third|night|| +13973|AAAAAAAAGJGDAAAA|13973|3|52|53|AM|third|night|| +13974|AAAAAAAAHJGDAAAA|13974|3|52|54|AM|third|night|| +13975|AAAAAAAAIJGDAAAA|13975|3|52|55|AM|third|night|| +13976|AAAAAAAAJJGDAAAA|13976|3|52|56|AM|third|night|| +13977|AAAAAAAAKJGDAAAA|13977|3|52|57|AM|third|night|| +13978|AAAAAAAALJGDAAAA|13978|3|52|58|AM|third|night|| +13979|AAAAAAAAMJGDAAAA|13979|3|52|59|AM|third|night|| +13980|AAAAAAAANJGDAAAA|13980|3|53|0|AM|third|night|| +13981|AAAAAAAAOJGDAAAA|13981|3|53|1|AM|third|night|| +13982|AAAAAAAAPJGDAAAA|13982|3|53|2|AM|third|night|| +13983|AAAAAAAAAKGDAAAA|13983|3|53|3|AM|third|night|| +13984|AAAAAAAABKGDAAAA|13984|3|53|4|AM|third|night|| +13985|AAAAAAAACKGDAAAA|13985|3|53|5|AM|third|night|| +13986|AAAAAAAADKGDAAAA|13986|3|53|6|AM|third|night|| +13987|AAAAAAAAEKGDAAAA|13987|3|53|7|AM|third|night|| +13988|AAAAAAAAFKGDAAAA|13988|3|53|8|AM|third|night|| +13989|AAAAAAAAGKGDAAAA|13989|3|53|9|AM|third|night|| +13990|AAAAAAAAHKGDAAAA|13990|3|53|10|AM|third|night|| +13991|AAAAAAAAIKGDAAAA|13991|3|53|11|AM|third|night|| +13992|AAAAAAAAJKGDAAAA|13992|3|53|12|AM|third|night|| +13993|AAAAAAAAKKGDAAAA|13993|3|53|13|AM|third|night|| +13994|AAAAAAAALKGDAAAA|13994|3|53|14|AM|third|night|| +13995|AAAAAAAAMKGDAAAA|13995|3|53|15|AM|third|night|| +13996|AAAAAAAANKGDAAAA|13996|3|53|16|AM|third|night|| +13997|AAAAAAAAOKGDAAAA|13997|3|53|17|AM|third|night|| +13998|AAAAAAAAPKGDAAAA|13998|3|53|18|AM|third|night|| +13999|AAAAAAAAALGDAAAA|13999|3|53|19|AM|third|night|| +14000|AAAAAAAABLGDAAAA|14000|3|53|20|AM|third|night|| +14001|AAAAAAAACLGDAAAA|14001|3|53|21|AM|third|night|| +14002|AAAAAAAADLGDAAAA|14002|3|53|22|AM|third|night|| +14003|AAAAAAAAELGDAAAA|14003|3|53|23|AM|third|night|| +14004|AAAAAAAAFLGDAAAA|14004|3|53|24|AM|third|night|| +14005|AAAAAAAAGLGDAAAA|14005|3|53|25|AM|third|night|| +14006|AAAAAAAAHLGDAAAA|14006|3|53|26|AM|third|night|| +14007|AAAAAAAAILGDAAAA|14007|3|53|27|AM|third|night|| +14008|AAAAAAAAJLGDAAAA|14008|3|53|28|AM|third|night|| +14009|AAAAAAAAKLGDAAAA|14009|3|53|29|AM|third|night|| +14010|AAAAAAAALLGDAAAA|14010|3|53|30|AM|third|night|| +14011|AAAAAAAAMLGDAAAA|14011|3|53|31|AM|third|night|| +14012|AAAAAAAANLGDAAAA|14012|3|53|32|AM|third|night|| +14013|AAAAAAAAOLGDAAAA|14013|3|53|33|AM|third|night|| +14014|AAAAAAAAPLGDAAAA|14014|3|53|34|AM|third|night|| +14015|AAAAAAAAAMGDAAAA|14015|3|53|35|AM|third|night|| +14016|AAAAAAAABMGDAAAA|14016|3|53|36|AM|third|night|| +14017|AAAAAAAACMGDAAAA|14017|3|53|37|AM|third|night|| +14018|AAAAAAAADMGDAAAA|14018|3|53|38|AM|third|night|| +14019|AAAAAAAAEMGDAAAA|14019|3|53|39|AM|third|night|| +14020|AAAAAAAAFMGDAAAA|14020|3|53|40|AM|third|night|| +14021|AAAAAAAAGMGDAAAA|14021|3|53|41|AM|third|night|| +14022|AAAAAAAAHMGDAAAA|14022|3|53|42|AM|third|night|| +14023|AAAAAAAAIMGDAAAA|14023|3|53|43|AM|third|night|| +14024|AAAAAAAAJMGDAAAA|14024|3|53|44|AM|third|night|| +14025|AAAAAAAAKMGDAAAA|14025|3|53|45|AM|third|night|| +14026|AAAAAAAALMGDAAAA|14026|3|53|46|AM|third|night|| +14027|AAAAAAAAMMGDAAAA|14027|3|53|47|AM|third|night|| +14028|AAAAAAAANMGDAAAA|14028|3|53|48|AM|third|night|| +14029|AAAAAAAAOMGDAAAA|14029|3|53|49|AM|third|night|| +14030|AAAAAAAAPMGDAAAA|14030|3|53|50|AM|third|night|| +14031|AAAAAAAAANGDAAAA|14031|3|53|51|AM|third|night|| +14032|AAAAAAAABNGDAAAA|14032|3|53|52|AM|third|night|| +14033|AAAAAAAACNGDAAAA|14033|3|53|53|AM|third|night|| +14034|AAAAAAAADNGDAAAA|14034|3|53|54|AM|third|night|| +14035|AAAAAAAAENGDAAAA|14035|3|53|55|AM|third|night|| +14036|AAAAAAAAFNGDAAAA|14036|3|53|56|AM|third|night|| +14037|AAAAAAAAGNGDAAAA|14037|3|53|57|AM|third|night|| +14038|AAAAAAAAHNGDAAAA|14038|3|53|58|AM|third|night|| +14039|AAAAAAAAINGDAAAA|14039|3|53|59|AM|third|night|| +14040|AAAAAAAAJNGDAAAA|14040|3|54|0|AM|third|night|| +14041|AAAAAAAAKNGDAAAA|14041|3|54|1|AM|third|night|| +14042|AAAAAAAALNGDAAAA|14042|3|54|2|AM|third|night|| +14043|AAAAAAAAMNGDAAAA|14043|3|54|3|AM|third|night|| +14044|AAAAAAAANNGDAAAA|14044|3|54|4|AM|third|night|| +14045|AAAAAAAAONGDAAAA|14045|3|54|5|AM|third|night|| +14046|AAAAAAAAPNGDAAAA|14046|3|54|6|AM|third|night|| +14047|AAAAAAAAAOGDAAAA|14047|3|54|7|AM|third|night|| +14048|AAAAAAAABOGDAAAA|14048|3|54|8|AM|third|night|| +14049|AAAAAAAACOGDAAAA|14049|3|54|9|AM|third|night|| +14050|AAAAAAAADOGDAAAA|14050|3|54|10|AM|third|night|| +14051|AAAAAAAAEOGDAAAA|14051|3|54|11|AM|third|night|| +14052|AAAAAAAAFOGDAAAA|14052|3|54|12|AM|third|night|| +14053|AAAAAAAAGOGDAAAA|14053|3|54|13|AM|third|night|| +14054|AAAAAAAAHOGDAAAA|14054|3|54|14|AM|third|night|| +14055|AAAAAAAAIOGDAAAA|14055|3|54|15|AM|third|night|| +14056|AAAAAAAAJOGDAAAA|14056|3|54|16|AM|third|night|| +14057|AAAAAAAAKOGDAAAA|14057|3|54|17|AM|third|night|| +14058|AAAAAAAALOGDAAAA|14058|3|54|18|AM|third|night|| +14059|AAAAAAAAMOGDAAAA|14059|3|54|19|AM|third|night|| +14060|AAAAAAAANOGDAAAA|14060|3|54|20|AM|third|night|| +14061|AAAAAAAAOOGDAAAA|14061|3|54|21|AM|third|night|| +14062|AAAAAAAAPOGDAAAA|14062|3|54|22|AM|third|night|| +14063|AAAAAAAAAPGDAAAA|14063|3|54|23|AM|third|night|| +14064|AAAAAAAABPGDAAAA|14064|3|54|24|AM|third|night|| +14065|AAAAAAAACPGDAAAA|14065|3|54|25|AM|third|night|| +14066|AAAAAAAADPGDAAAA|14066|3|54|26|AM|third|night|| +14067|AAAAAAAAEPGDAAAA|14067|3|54|27|AM|third|night|| +14068|AAAAAAAAFPGDAAAA|14068|3|54|28|AM|third|night|| +14069|AAAAAAAAGPGDAAAA|14069|3|54|29|AM|third|night|| +14070|AAAAAAAAHPGDAAAA|14070|3|54|30|AM|third|night|| +14071|AAAAAAAAIPGDAAAA|14071|3|54|31|AM|third|night|| +14072|AAAAAAAAJPGDAAAA|14072|3|54|32|AM|third|night|| +14073|AAAAAAAAKPGDAAAA|14073|3|54|33|AM|third|night|| +14074|AAAAAAAALPGDAAAA|14074|3|54|34|AM|third|night|| +14075|AAAAAAAAMPGDAAAA|14075|3|54|35|AM|third|night|| +14076|AAAAAAAANPGDAAAA|14076|3|54|36|AM|third|night|| +14077|AAAAAAAAOPGDAAAA|14077|3|54|37|AM|third|night|| +14078|AAAAAAAAPPGDAAAA|14078|3|54|38|AM|third|night|| +14079|AAAAAAAAAAHDAAAA|14079|3|54|39|AM|third|night|| +14080|AAAAAAAABAHDAAAA|14080|3|54|40|AM|third|night|| +14081|AAAAAAAACAHDAAAA|14081|3|54|41|AM|third|night|| +14082|AAAAAAAADAHDAAAA|14082|3|54|42|AM|third|night|| +14083|AAAAAAAAEAHDAAAA|14083|3|54|43|AM|third|night|| +14084|AAAAAAAAFAHDAAAA|14084|3|54|44|AM|third|night|| +14085|AAAAAAAAGAHDAAAA|14085|3|54|45|AM|third|night|| +14086|AAAAAAAAHAHDAAAA|14086|3|54|46|AM|third|night|| +14087|AAAAAAAAIAHDAAAA|14087|3|54|47|AM|third|night|| +14088|AAAAAAAAJAHDAAAA|14088|3|54|48|AM|third|night|| +14089|AAAAAAAAKAHDAAAA|14089|3|54|49|AM|third|night|| +14090|AAAAAAAALAHDAAAA|14090|3|54|50|AM|third|night|| +14091|AAAAAAAAMAHDAAAA|14091|3|54|51|AM|third|night|| +14092|AAAAAAAANAHDAAAA|14092|3|54|52|AM|third|night|| +14093|AAAAAAAAOAHDAAAA|14093|3|54|53|AM|third|night|| +14094|AAAAAAAAPAHDAAAA|14094|3|54|54|AM|third|night|| +14095|AAAAAAAAABHDAAAA|14095|3|54|55|AM|third|night|| +14096|AAAAAAAABBHDAAAA|14096|3|54|56|AM|third|night|| +14097|AAAAAAAACBHDAAAA|14097|3|54|57|AM|third|night|| +14098|AAAAAAAADBHDAAAA|14098|3|54|58|AM|third|night|| +14099|AAAAAAAAEBHDAAAA|14099|3|54|59|AM|third|night|| +14100|AAAAAAAAFBHDAAAA|14100|3|55|0|AM|third|night|| +14101|AAAAAAAAGBHDAAAA|14101|3|55|1|AM|third|night|| +14102|AAAAAAAAHBHDAAAA|14102|3|55|2|AM|third|night|| +14103|AAAAAAAAIBHDAAAA|14103|3|55|3|AM|third|night|| +14104|AAAAAAAAJBHDAAAA|14104|3|55|4|AM|third|night|| +14105|AAAAAAAAKBHDAAAA|14105|3|55|5|AM|third|night|| +14106|AAAAAAAALBHDAAAA|14106|3|55|6|AM|third|night|| +14107|AAAAAAAAMBHDAAAA|14107|3|55|7|AM|third|night|| +14108|AAAAAAAANBHDAAAA|14108|3|55|8|AM|third|night|| +14109|AAAAAAAAOBHDAAAA|14109|3|55|9|AM|third|night|| +14110|AAAAAAAAPBHDAAAA|14110|3|55|10|AM|third|night|| +14111|AAAAAAAAACHDAAAA|14111|3|55|11|AM|third|night|| +14112|AAAAAAAABCHDAAAA|14112|3|55|12|AM|third|night|| +14113|AAAAAAAACCHDAAAA|14113|3|55|13|AM|third|night|| +14114|AAAAAAAADCHDAAAA|14114|3|55|14|AM|third|night|| +14115|AAAAAAAAECHDAAAA|14115|3|55|15|AM|third|night|| +14116|AAAAAAAAFCHDAAAA|14116|3|55|16|AM|third|night|| +14117|AAAAAAAAGCHDAAAA|14117|3|55|17|AM|third|night|| +14118|AAAAAAAAHCHDAAAA|14118|3|55|18|AM|third|night|| +14119|AAAAAAAAICHDAAAA|14119|3|55|19|AM|third|night|| +14120|AAAAAAAAJCHDAAAA|14120|3|55|20|AM|third|night|| +14121|AAAAAAAAKCHDAAAA|14121|3|55|21|AM|third|night|| +14122|AAAAAAAALCHDAAAA|14122|3|55|22|AM|third|night|| +14123|AAAAAAAAMCHDAAAA|14123|3|55|23|AM|third|night|| +14124|AAAAAAAANCHDAAAA|14124|3|55|24|AM|third|night|| +14125|AAAAAAAAOCHDAAAA|14125|3|55|25|AM|third|night|| +14126|AAAAAAAAPCHDAAAA|14126|3|55|26|AM|third|night|| +14127|AAAAAAAAADHDAAAA|14127|3|55|27|AM|third|night|| +14128|AAAAAAAABDHDAAAA|14128|3|55|28|AM|third|night|| +14129|AAAAAAAACDHDAAAA|14129|3|55|29|AM|third|night|| +14130|AAAAAAAADDHDAAAA|14130|3|55|30|AM|third|night|| +14131|AAAAAAAAEDHDAAAA|14131|3|55|31|AM|third|night|| +14132|AAAAAAAAFDHDAAAA|14132|3|55|32|AM|third|night|| +14133|AAAAAAAAGDHDAAAA|14133|3|55|33|AM|third|night|| +14134|AAAAAAAAHDHDAAAA|14134|3|55|34|AM|third|night|| +14135|AAAAAAAAIDHDAAAA|14135|3|55|35|AM|third|night|| +14136|AAAAAAAAJDHDAAAA|14136|3|55|36|AM|third|night|| +14137|AAAAAAAAKDHDAAAA|14137|3|55|37|AM|third|night|| +14138|AAAAAAAALDHDAAAA|14138|3|55|38|AM|third|night|| +14139|AAAAAAAAMDHDAAAA|14139|3|55|39|AM|third|night|| +14140|AAAAAAAANDHDAAAA|14140|3|55|40|AM|third|night|| +14141|AAAAAAAAODHDAAAA|14141|3|55|41|AM|third|night|| +14142|AAAAAAAAPDHDAAAA|14142|3|55|42|AM|third|night|| +14143|AAAAAAAAAEHDAAAA|14143|3|55|43|AM|third|night|| +14144|AAAAAAAABEHDAAAA|14144|3|55|44|AM|third|night|| +14145|AAAAAAAACEHDAAAA|14145|3|55|45|AM|third|night|| +14146|AAAAAAAADEHDAAAA|14146|3|55|46|AM|third|night|| +14147|AAAAAAAAEEHDAAAA|14147|3|55|47|AM|third|night|| +14148|AAAAAAAAFEHDAAAA|14148|3|55|48|AM|third|night|| +14149|AAAAAAAAGEHDAAAA|14149|3|55|49|AM|third|night|| +14150|AAAAAAAAHEHDAAAA|14150|3|55|50|AM|third|night|| +14151|AAAAAAAAIEHDAAAA|14151|3|55|51|AM|third|night|| +14152|AAAAAAAAJEHDAAAA|14152|3|55|52|AM|third|night|| +14153|AAAAAAAAKEHDAAAA|14153|3|55|53|AM|third|night|| +14154|AAAAAAAALEHDAAAA|14154|3|55|54|AM|third|night|| +14155|AAAAAAAAMEHDAAAA|14155|3|55|55|AM|third|night|| +14156|AAAAAAAANEHDAAAA|14156|3|55|56|AM|third|night|| +14157|AAAAAAAAOEHDAAAA|14157|3|55|57|AM|third|night|| +14158|AAAAAAAAPEHDAAAA|14158|3|55|58|AM|third|night|| +14159|AAAAAAAAAFHDAAAA|14159|3|55|59|AM|third|night|| +14160|AAAAAAAABFHDAAAA|14160|3|56|0|AM|third|night|| +14161|AAAAAAAACFHDAAAA|14161|3|56|1|AM|third|night|| +14162|AAAAAAAADFHDAAAA|14162|3|56|2|AM|third|night|| +14163|AAAAAAAAEFHDAAAA|14163|3|56|3|AM|third|night|| +14164|AAAAAAAAFFHDAAAA|14164|3|56|4|AM|third|night|| +14165|AAAAAAAAGFHDAAAA|14165|3|56|5|AM|third|night|| +14166|AAAAAAAAHFHDAAAA|14166|3|56|6|AM|third|night|| +14167|AAAAAAAAIFHDAAAA|14167|3|56|7|AM|third|night|| +14168|AAAAAAAAJFHDAAAA|14168|3|56|8|AM|third|night|| +14169|AAAAAAAAKFHDAAAA|14169|3|56|9|AM|third|night|| +14170|AAAAAAAALFHDAAAA|14170|3|56|10|AM|third|night|| +14171|AAAAAAAAMFHDAAAA|14171|3|56|11|AM|third|night|| +14172|AAAAAAAANFHDAAAA|14172|3|56|12|AM|third|night|| +14173|AAAAAAAAOFHDAAAA|14173|3|56|13|AM|third|night|| +14174|AAAAAAAAPFHDAAAA|14174|3|56|14|AM|third|night|| +14175|AAAAAAAAAGHDAAAA|14175|3|56|15|AM|third|night|| +14176|AAAAAAAABGHDAAAA|14176|3|56|16|AM|third|night|| +14177|AAAAAAAACGHDAAAA|14177|3|56|17|AM|third|night|| +14178|AAAAAAAADGHDAAAA|14178|3|56|18|AM|third|night|| +14179|AAAAAAAAEGHDAAAA|14179|3|56|19|AM|third|night|| +14180|AAAAAAAAFGHDAAAA|14180|3|56|20|AM|third|night|| +14181|AAAAAAAAGGHDAAAA|14181|3|56|21|AM|third|night|| +14182|AAAAAAAAHGHDAAAA|14182|3|56|22|AM|third|night|| +14183|AAAAAAAAIGHDAAAA|14183|3|56|23|AM|third|night|| +14184|AAAAAAAAJGHDAAAA|14184|3|56|24|AM|third|night|| +14185|AAAAAAAAKGHDAAAA|14185|3|56|25|AM|third|night|| +14186|AAAAAAAALGHDAAAA|14186|3|56|26|AM|third|night|| +14187|AAAAAAAAMGHDAAAA|14187|3|56|27|AM|third|night|| +14188|AAAAAAAANGHDAAAA|14188|3|56|28|AM|third|night|| +14189|AAAAAAAAOGHDAAAA|14189|3|56|29|AM|third|night|| +14190|AAAAAAAAPGHDAAAA|14190|3|56|30|AM|third|night|| +14191|AAAAAAAAAHHDAAAA|14191|3|56|31|AM|third|night|| +14192|AAAAAAAABHHDAAAA|14192|3|56|32|AM|third|night|| +14193|AAAAAAAACHHDAAAA|14193|3|56|33|AM|third|night|| +14194|AAAAAAAADHHDAAAA|14194|3|56|34|AM|third|night|| +14195|AAAAAAAAEHHDAAAA|14195|3|56|35|AM|third|night|| +14196|AAAAAAAAFHHDAAAA|14196|3|56|36|AM|third|night|| +14197|AAAAAAAAGHHDAAAA|14197|3|56|37|AM|third|night|| +14198|AAAAAAAAHHHDAAAA|14198|3|56|38|AM|third|night|| +14199|AAAAAAAAIHHDAAAA|14199|3|56|39|AM|third|night|| +14200|AAAAAAAAJHHDAAAA|14200|3|56|40|AM|third|night|| +14201|AAAAAAAAKHHDAAAA|14201|3|56|41|AM|third|night|| +14202|AAAAAAAALHHDAAAA|14202|3|56|42|AM|third|night|| +14203|AAAAAAAAMHHDAAAA|14203|3|56|43|AM|third|night|| +14204|AAAAAAAANHHDAAAA|14204|3|56|44|AM|third|night|| +14205|AAAAAAAAOHHDAAAA|14205|3|56|45|AM|third|night|| +14206|AAAAAAAAPHHDAAAA|14206|3|56|46|AM|third|night|| +14207|AAAAAAAAAIHDAAAA|14207|3|56|47|AM|third|night|| +14208|AAAAAAAABIHDAAAA|14208|3|56|48|AM|third|night|| +14209|AAAAAAAACIHDAAAA|14209|3|56|49|AM|third|night|| +14210|AAAAAAAADIHDAAAA|14210|3|56|50|AM|third|night|| +14211|AAAAAAAAEIHDAAAA|14211|3|56|51|AM|third|night|| +14212|AAAAAAAAFIHDAAAA|14212|3|56|52|AM|third|night|| +14213|AAAAAAAAGIHDAAAA|14213|3|56|53|AM|third|night|| +14214|AAAAAAAAHIHDAAAA|14214|3|56|54|AM|third|night|| +14215|AAAAAAAAIIHDAAAA|14215|3|56|55|AM|third|night|| +14216|AAAAAAAAJIHDAAAA|14216|3|56|56|AM|third|night|| +14217|AAAAAAAAKIHDAAAA|14217|3|56|57|AM|third|night|| +14218|AAAAAAAALIHDAAAA|14218|3|56|58|AM|third|night|| +14219|AAAAAAAAMIHDAAAA|14219|3|56|59|AM|third|night|| +14220|AAAAAAAANIHDAAAA|14220|3|57|0|AM|third|night|| +14221|AAAAAAAAOIHDAAAA|14221|3|57|1|AM|third|night|| +14222|AAAAAAAAPIHDAAAA|14222|3|57|2|AM|third|night|| +14223|AAAAAAAAAJHDAAAA|14223|3|57|3|AM|third|night|| +14224|AAAAAAAABJHDAAAA|14224|3|57|4|AM|third|night|| +14225|AAAAAAAACJHDAAAA|14225|3|57|5|AM|third|night|| +14226|AAAAAAAADJHDAAAA|14226|3|57|6|AM|third|night|| +14227|AAAAAAAAEJHDAAAA|14227|3|57|7|AM|third|night|| +14228|AAAAAAAAFJHDAAAA|14228|3|57|8|AM|third|night|| +14229|AAAAAAAAGJHDAAAA|14229|3|57|9|AM|third|night|| +14230|AAAAAAAAHJHDAAAA|14230|3|57|10|AM|third|night|| +14231|AAAAAAAAIJHDAAAA|14231|3|57|11|AM|third|night|| +14232|AAAAAAAAJJHDAAAA|14232|3|57|12|AM|third|night|| +14233|AAAAAAAAKJHDAAAA|14233|3|57|13|AM|third|night|| +14234|AAAAAAAALJHDAAAA|14234|3|57|14|AM|third|night|| +14235|AAAAAAAAMJHDAAAA|14235|3|57|15|AM|third|night|| +14236|AAAAAAAANJHDAAAA|14236|3|57|16|AM|third|night|| +14237|AAAAAAAAOJHDAAAA|14237|3|57|17|AM|third|night|| +14238|AAAAAAAAPJHDAAAA|14238|3|57|18|AM|third|night|| +14239|AAAAAAAAAKHDAAAA|14239|3|57|19|AM|third|night|| +14240|AAAAAAAABKHDAAAA|14240|3|57|20|AM|third|night|| +14241|AAAAAAAACKHDAAAA|14241|3|57|21|AM|third|night|| +14242|AAAAAAAADKHDAAAA|14242|3|57|22|AM|third|night|| +14243|AAAAAAAAEKHDAAAA|14243|3|57|23|AM|third|night|| +14244|AAAAAAAAFKHDAAAA|14244|3|57|24|AM|third|night|| +14245|AAAAAAAAGKHDAAAA|14245|3|57|25|AM|third|night|| +14246|AAAAAAAAHKHDAAAA|14246|3|57|26|AM|third|night|| +14247|AAAAAAAAIKHDAAAA|14247|3|57|27|AM|third|night|| +14248|AAAAAAAAJKHDAAAA|14248|3|57|28|AM|third|night|| +14249|AAAAAAAAKKHDAAAA|14249|3|57|29|AM|third|night|| +14250|AAAAAAAALKHDAAAA|14250|3|57|30|AM|third|night|| +14251|AAAAAAAAMKHDAAAA|14251|3|57|31|AM|third|night|| +14252|AAAAAAAANKHDAAAA|14252|3|57|32|AM|third|night|| +14253|AAAAAAAAOKHDAAAA|14253|3|57|33|AM|third|night|| +14254|AAAAAAAAPKHDAAAA|14254|3|57|34|AM|third|night|| +14255|AAAAAAAAALHDAAAA|14255|3|57|35|AM|third|night|| +14256|AAAAAAAABLHDAAAA|14256|3|57|36|AM|third|night|| +14257|AAAAAAAACLHDAAAA|14257|3|57|37|AM|third|night|| +14258|AAAAAAAADLHDAAAA|14258|3|57|38|AM|third|night|| +14259|AAAAAAAAELHDAAAA|14259|3|57|39|AM|third|night|| +14260|AAAAAAAAFLHDAAAA|14260|3|57|40|AM|third|night|| +14261|AAAAAAAAGLHDAAAA|14261|3|57|41|AM|third|night|| +14262|AAAAAAAAHLHDAAAA|14262|3|57|42|AM|third|night|| +14263|AAAAAAAAILHDAAAA|14263|3|57|43|AM|third|night|| +14264|AAAAAAAAJLHDAAAA|14264|3|57|44|AM|third|night|| +14265|AAAAAAAAKLHDAAAA|14265|3|57|45|AM|third|night|| +14266|AAAAAAAALLHDAAAA|14266|3|57|46|AM|third|night|| +14267|AAAAAAAAMLHDAAAA|14267|3|57|47|AM|third|night|| +14268|AAAAAAAANLHDAAAA|14268|3|57|48|AM|third|night|| +14269|AAAAAAAAOLHDAAAA|14269|3|57|49|AM|third|night|| +14270|AAAAAAAAPLHDAAAA|14270|3|57|50|AM|third|night|| +14271|AAAAAAAAAMHDAAAA|14271|3|57|51|AM|third|night|| +14272|AAAAAAAABMHDAAAA|14272|3|57|52|AM|third|night|| +14273|AAAAAAAACMHDAAAA|14273|3|57|53|AM|third|night|| +14274|AAAAAAAADMHDAAAA|14274|3|57|54|AM|third|night|| +14275|AAAAAAAAEMHDAAAA|14275|3|57|55|AM|third|night|| +14276|AAAAAAAAFMHDAAAA|14276|3|57|56|AM|third|night|| +14277|AAAAAAAAGMHDAAAA|14277|3|57|57|AM|third|night|| +14278|AAAAAAAAHMHDAAAA|14278|3|57|58|AM|third|night|| +14279|AAAAAAAAIMHDAAAA|14279|3|57|59|AM|third|night|| +14280|AAAAAAAAJMHDAAAA|14280|3|58|0|AM|third|night|| +14281|AAAAAAAAKMHDAAAA|14281|3|58|1|AM|third|night|| +14282|AAAAAAAALMHDAAAA|14282|3|58|2|AM|third|night|| +14283|AAAAAAAAMMHDAAAA|14283|3|58|3|AM|third|night|| +14284|AAAAAAAANMHDAAAA|14284|3|58|4|AM|third|night|| +14285|AAAAAAAAOMHDAAAA|14285|3|58|5|AM|third|night|| +14286|AAAAAAAAPMHDAAAA|14286|3|58|6|AM|third|night|| +14287|AAAAAAAAANHDAAAA|14287|3|58|7|AM|third|night|| +14288|AAAAAAAABNHDAAAA|14288|3|58|8|AM|third|night|| +14289|AAAAAAAACNHDAAAA|14289|3|58|9|AM|third|night|| +14290|AAAAAAAADNHDAAAA|14290|3|58|10|AM|third|night|| +14291|AAAAAAAAENHDAAAA|14291|3|58|11|AM|third|night|| +14292|AAAAAAAAFNHDAAAA|14292|3|58|12|AM|third|night|| +14293|AAAAAAAAGNHDAAAA|14293|3|58|13|AM|third|night|| +14294|AAAAAAAAHNHDAAAA|14294|3|58|14|AM|third|night|| +14295|AAAAAAAAINHDAAAA|14295|3|58|15|AM|third|night|| +14296|AAAAAAAAJNHDAAAA|14296|3|58|16|AM|third|night|| +14297|AAAAAAAAKNHDAAAA|14297|3|58|17|AM|third|night|| +14298|AAAAAAAALNHDAAAA|14298|3|58|18|AM|third|night|| +14299|AAAAAAAAMNHDAAAA|14299|3|58|19|AM|third|night|| +14300|AAAAAAAANNHDAAAA|14300|3|58|20|AM|third|night|| +14301|AAAAAAAAONHDAAAA|14301|3|58|21|AM|third|night|| +14302|AAAAAAAAPNHDAAAA|14302|3|58|22|AM|third|night|| +14303|AAAAAAAAAOHDAAAA|14303|3|58|23|AM|third|night|| +14304|AAAAAAAABOHDAAAA|14304|3|58|24|AM|third|night|| +14305|AAAAAAAACOHDAAAA|14305|3|58|25|AM|third|night|| +14306|AAAAAAAADOHDAAAA|14306|3|58|26|AM|third|night|| +14307|AAAAAAAAEOHDAAAA|14307|3|58|27|AM|third|night|| +14308|AAAAAAAAFOHDAAAA|14308|3|58|28|AM|third|night|| +14309|AAAAAAAAGOHDAAAA|14309|3|58|29|AM|third|night|| +14310|AAAAAAAAHOHDAAAA|14310|3|58|30|AM|third|night|| +14311|AAAAAAAAIOHDAAAA|14311|3|58|31|AM|third|night|| +14312|AAAAAAAAJOHDAAAA|14312|3|58|32|AM|third|night|| +14313|AAAAAAAAKOHDAAAA|14313|3|58|33|AM|third|night|| +14314|AAAAAAAALOHDAAAA|14314|3|58|34|AM|third|night|| +14315|AAAAAAAAMOHDAAAA|14315|3|58|35|AM|third|night|| +14316|AAAAAAAANOHDAAAA|14316|3|58|36|AM|third|night|| +14317|AAAAAAAAOOHDAAAA|14317|3|58|37|AM|third|night|| +14318|AAAAAAAAPOHDAAAA|14318|3|58|38|AM|third|night|| +14319|AAAAAAAAAPHDAAAA|14319|3|58|39|AM|third|night|| +14320|AAAAAAAABPHDAAAA|14320|3|58|40|AM|third|night|| +14321|AAAAAAAACPHDAAAA|14321|3|58|41|AM|third|night|| +14322|AAAAAAAADPHDAAAA|14322|3|58|42|AM|third|night|| +14323|AAAAAAAAEPHDAAAA|14323|3|58|43|AM|third|night|| +14324|AAAAAAAAFPHDAAAA|14324|3|58|44|AM|third|night|| +14325|AAAAAAAAGPHDAAAA|14325|3|58|45|AM|third|night|| +14326|AAAAAAAAHPHDAAAA|14326|3|58|46|AM|third|night|| +14327|AAAAAAAAIPHDAAAA|14327|3|58|47|AM|third|night|| +14328|AAAAAAAAJPHDAAAA|14328|3|58|48|AM|third|night|| +14329|AAAAAAAAKPHDAAAA|14329|3|58|49|AM|third|night|| +14330|AAAAAAAALPHDAAAA|14330|3|58|50|AM|third|night|| +14331|AAAAAAAAMPHDAAAA|14331|3|58|51|AM|third|night|| +14332|AAAAAAAANPHDAAAA|14332|3|58|52|AM|third|night|| +14333|AAAAAAAAOPHDAAAA|14333|3|58|53|AM|third|night|| +14334|AAAAAAAAPPHDAAAA|14334|3|58|54|AM|third|night|| +14335|AAAAAAAAAAIDAAAA|14335|3|58|55|AM|third|night|| +14336|AAAAAAAABAIDAAAA|14336|3|58|56|AM|third|night|| +14337|AAAAAAAACAIDAAAA|14337|3|58|57|AM|third|night|| +14338|AAAAAAAADAIDAAAA|14338|3|58|58|AM|third|night|| +14339|AAAAAAAAEAIDAAAA|14339|3|58|59|AM|third|night|| +14340|AAAAAAAAFAIDAAAA|14340|3|59|0|AM|third|night|| +14341|AAAAAAAAGAIDAAAA|14341|3|59|1|AM|third|night|| +14342|AAAAAAAAHAIDAAAA|14342|3|59|2|AM|third|night|| +14343|AAAAAAAAIAIDAAAA|14343|3|59|3|AM|third|night|| +14344|AAAAAAAAJAIDAAAA|14344|3|59|4|AM|third|night|| +14345|AAAAAAAAKAIDAAAA|14345|3|59|5|AM|third|night|| +14346|AAAAAAAALAIDAAAA|14346|3|59|6|AM|third|night|| +14347|AAAAAAAAMAIDAAAA|14347|3|59|7|AM|third|night|| +14348|AAAAAAAANAIDAAAA|14348|3|59|8|AM|third|night|| +14349|AAAAAAAAOAIDAAAA|14349|3|59|9|AM|third|night|| +14350|AAAAAAAAPAIDAAAA|14350|3|59|10|AM|third|night|| +14351|AAAAAAAAABIDAAAA|14351|3|59|11|AM|third|night|| +14352|AAAAAAAABBIDAAAA|14352|3|59|12|AM|third|night|| +14353|AAAAAAAACBIDAAAA|14353|3|59|13|AM|third|night|| +14354|AAAAAAAADBIDAAAA|14354|3|59|14|AM|third|night|| +14355|AAAAAAAAEBIDAAAA|14355|3|59|15|AM|third|night|| +14356|AAAAAAAAFBIDAAAA|14356|3|59|16|AM|third|night|| +14357|AAAAAAAAGBIDAAAA|14357|3|59|17|AM|third|night|| +14358|AAAAAAAAHBIDAAAA|14358|3|59|18|AM|third|night|| +14359|AAAAAAAAIBIDAAAA|14359|3|59|19|AM|third|night|| +14360|AAAAAAAAJBIDAAAA|14360|3|59|20|AM|third|night|| +14361|AAAAAAAAKBIDAAAA|14361|3|59|21|AM|third|night|| +14362|AAAAAAAALBIDAAAA|14362|3|59|22|AM|third|night|| +14363|AAAAAAAAMBIDAAAA|14363|3|59|23|AM|third|night|| +14364|AAAAAAAANBIDAAAA|14364|3|59|24|AM|third|night|| +14365|AAAAAAAAOBIDAAAA|14365|3|59|25|AM|third|night|| +14366|AAAAAAAAPBIDAAAA|14366|3|59|26|AM|third|night|| +14367|AAAAAAAAACIDAAAA|14367|3|59|27|AM|third|night|| +14368|AAAAAAAABCIDAAAA|14368|3|59|28|AM|third|night|| +14369|AAAAAAAACCIDAAAA|14369|3|59|29|AM|third|night|| +14370|AAAAAAAADCIDAAAA|14370|3|59|30|AM|third|night|| +14371|AAAAAAAAECIDAAAA|14371|3|59|31|AM|third|night|| +14372|AAAAAAAAFCIDAAAA|14372|3|59|32|AM|third|night|| +14373|AAAAAAAAGCIDAAAA|14373|3|59|33|AM|third|night|| +14374|AAAAAAAAHCIDAAAA|14374|3|59|34|AM|third|night|| +14375|AAAAAAAAICIDAAAA|14375|3|59|35|AM|third|night|| +14376|AAAAAAAAJCIDAAAA|14376|3|59|36|AM|third|night|| +14377|AAAAAAAAKCIDAAAA|14377|3|59|37|AM|third|night|| +14378|AAAAAAAALCIDAAAA|14378|3|59|38|AM|third|night|| +14379|AAAAAAAAMCIDAAAA|14379|3|59|39|AM|third|night|| +14380|AAAAAAAANCIDAAAA|14380|3|59|40|AM|third|night|| +14381|AAAAAAAAOCIDAAAA|14381|3|59|41|AM|third|night|| +14382|AAAAAAAAPCIDAAAA|14382|3|59|42|AM|third|night|| +14383|AAAAAAAAADIDAAAA|14383|3|59|43|AM|third|night|| +14384|AAAAAAAABDIDAAAA|14384|3|59|44|AM|third|night|| +14385|AAAAAAAACDIDAAAA|14385|3|59|45|AM|third|night|| +14386|AAAAAAAADDIDAAAA|14386|3|59|46|AM|third|night|| +14387|AAAAAAAAEDIDAAAA|14387|3|59|47|AM|third|night|| +14388|AAAAAAAAFDIDAAAA|14388|3|59|48|AM|third|night|| +14389|AAAAAAAAGDIDAAAA|14389|3|59|49|AM|third|night|| +14390|AAAAAAAAHDIDAAAA|14390|3|59|50|AM|third|night|| +14391|AAAAAAAAIDIDAAAA|14391|3|59|51|AM|third|night|| +14392|AAAAAAAAJDIDAAAA|14392|3|59|52|AM|third|night|| +14393|AAAAAAAAKDIDAAAA|14393|3|59|53|AM|third|night|| +14394|AAAAAAAALDIDAAAA|14394|3|59|54|AM|third|night|| +14395|AAAAAAAAMDIDAAAA|14395|3|59|55|AM|third|night|| +14396|AAAAAAAANDIDAAAA|14396|3|59|56|AM|third|night|| +14397|AAAAAAAAODIDAAAA|14397|3|59|57|AM|third|night|| +14398|AAAAAAAAPDIDAAAA|14398|3|59|58|AM|third|night|| +14399|AAAAAAAAAEIDAAAA|14399|3|59|59|AM|third|night|| +14400|AAAAAAAABEIDAAAA|14400|4|0|0|AM|third|night|| +14401|AAAAAAAACEIDAAAA|14401|4|0|1|AM|third|night|| +14402|AAAAAAAADEIDAAAA|14402|4|0|2|AM|third|night|| +14403|AAAAAAAAEEIDAAAA|14403|4|0|3|AM|third|night|| +14404|AAAAAAAAFEIDAAAA|14404|4|0|4|AM|third|night|| +14405|AAAAAAAAGEIDAAAA|14405|4|0|5|AM|third|night|| +14406|AAAAAAAAHEIDAAAA|14406|4|0|6|AM|third|night|| +14407|AAAAAAAAIEIDAAAA|14407|4|0|7|AM|third|night|| +14408|AAAAAAAAJEIDAAAA|14408|4|0|8|AM|third|night|| +14409|AAAAAAAAKEIDAAAA|14409|4|0|9|AM|third|night|| +14410|AAAAAAAALEIDAAAA|14410|4|0|10|AM|third|night|| +14411|AAAAAAAAMEIDAAAA|14411|4|0|11|AM|third|night|| +14412|AAAAAAAANEIDAAAA|14412|4|0|12|AM|third|night|| +14413|AAAAAAAAOEIDAAAA|14413|4|0|13|AM|third|night|| +14414|AAAAAAAAPEIDAAAA|14414|4|0|14|AM|third|night|| +14415|AAAAAAAAAFIDAAAA|14415|4|0|15|AM|third|night|| +14416|AAAAAAAABFIDAAAA|14416|4|0|16|AM|third|night|| +14417|AAAAAAAACFIDAAAA|14417|4|0|17|AM|third|night|| +14418|AAAAAAAADFIDAAAA|14418|4|0|18|AM|third|night|| +14419|AAAAAAAAEFIDAAAA|14419|4|0|19|AM|third|night|| +14420|AAAAAAAAFFIDAAAA|14420|4|0|20|AM|third|night|| +14421|AAAAAAAAGFIDAAAA|14421|4|0|21|AM|third|night|| +14422|AAAAAAAAHFIDAAAA|14422|4|0|22|AM|third|night|| +14423|AAAAAAAAIFIDAAAA|14423|4|0|23|AM|third|night|| +14424|AAAAAAAAJFIDAAAA|14424|4|0|24|AM|third|night|| +14425|AAAAAAAAKFIDAAAA|14425|4|0|25|AM|third|night|| +14426|AAAAAAAALFIDAAAA|14426|4|0|26|AM|third|night|| +14427|AAAAAAAAMFIDAAAA|14427|4|0|27|AM|third|night|| +14428|AAAAAAAANFIDAAAA|14428|4|0|28|AM|third|night|| +14429|AAAAAAAAOFIDAAAA|14429|4|0|29|AM|third|night|| +14430|AAAAAAAAPFIDAAAA|14430|4|0|30|AM|third|night|| +14431|AAAAAAAAAGIDAAAA|14431|4|0|31|AM|third|night|| +14432|AAAAAAAABGIDAAAA|14432|4|0|32|AM|third|night|| +14433|AAAAAAAACGIDAAAA|14433|4|0|33|AM|third|night|| +14434|AAAAAAAADGIDAAAA|14434|4|0|34|AM|third|night|| +14435|AAAAAAAAEGIDAAAA|14435|4|0|35|AM|third|night|| +14436|AAAAAAAAFGIDAAAA|14436|4|0|36|AM|third|night|| +14437|AAAAAAAAGGIDAAAA|14437|4|0|37|AM|third|night|| +14438|AAAAAAAAHGIDAAAA|14438|4|0|38|AM|third|night|| +14439|AAAAAAAAIGIDAAAA|14439|4|0|39|AM|third|night|| +14440|AAAAAAAAJGIDAAAA|14440|4|0|40|AM|third|night|| +14441|AAAAAAAAKGIDAAAA|14441|4|0|41|AM|third|night|| +14442|AAAAAAAALGIDAAAA|14442|4|0|42|AM|third|night|| +14443|AAAAAAAAMGIDAAAA|14443|4|0|43|AM|third|night|| +14444|AAAAAAAANGIDAAAA|14444|4|0|44|AM|third|night|| +14445|AAAAAAAAOGIDAAAA|14445|4|0|45|AM|third|night|| +14446|AAAAAAAAPGIDAAAA|14446|4|0|46|AM|third|night|| +14447|AAAAAAAAAHIDAAAA|14447|4|0|47|AM|third|night|| +14448|AAAAAAAABHIDAAAA|14448|4|0|48|AM|third|night|| +14449|AAAAAAAACHIDAAAA|14449|4|0|49|AM|third|night|| +14450|AAAAAAAADHIDAAAA|14450|4|0|50|AM|third|night|| +14451|AAAAAAAAEHIDAAAA|14451|4|0|51|AM|third|night|| +14452|AAAAAAAAFHIDAAAA|14452|4|0|52|AM|third|night|| +14453|AAAAAAAAGHIDAAAA|14453|4|0|53|AM|third|night|| +14454|AAAAAAAAHHIDAAAA|14454|4|0|54|AM|third|night|| +14455|AAAAAAAAIHIDAAAA|14455|4|0|55|AM|third|night|| +14456|AAAAAAAAJHIDAAAA|14456|4|0|56|AM|third|night|| +14457|AAAAAAAAKHIDAAAA|14457|4|0|57|AM|third|night|| +14458|AAAAAAAALHIDAAAA|14458|4|0|58|AM|third|night|| +14459|AAAAAAAAMHIDAAAA|14459|4|0|59|AM|third|night|| +14460|AAAAAAAANHIDAAAA|14460|4|1|0|AM|third|night|| +14461|AAAAAAAAOHIDAAAA|14461|4|1|1|AM|third|night|| +14462|AAAAAAAAPHIDAAAA|14462|4|1|2|AM|third|night|| +14463|AAAAAAAAAIIDAAAA|14463|4|1|3|AM|third|night|| +14464|AAAAAAAABIIDAAAA|14464|4|1|4|AM|third|night|| +14465|AAAAAAAACIIDAAAA|14465|4|1|5|AM|third|night|| +14466|AAAAAAAADIIDAAAA|14466|4|1|6|AM|third|night|| +14467|AAAAAAAAEIIDAAAA|14467|4|1|7|AM|third|night|| +14468|AAAAAAAAFIIDAAAA|14468|4|1|8|AM|third|night|| +14469|AAAAAAAAGIIDAAAA|14469|4|1|9|AM|third|night|| +14470|AAAAAAAAHIIDAAAA|14470|4|1|10|AM|third|night|| +14471|AAAAAAAAIIIDAAAA|14471|4|1|11|AM|third|night|| +14472|AAAAAAAAJIIDAAAA|14472|4|1|12|AM|third|night|| +14473|AAAAAAAAKIIDAAAA|14473|4|1|13|AM|third|night|| +14474|AAAAAAAALIIDAAAA|14474|4|1|14|AM|third|night|| +14475|AAAAAAAAMIIDAAAA|14475|4|1|15|AM|third|night|| +14476|AAAAAAAANIIDAAAA|14476|4|1|16|AM|third|night|| +14477|AAAAAAAAOIIDAAAA|14477|4|1|17|AM|third|night|| +14478|AAAAAAAAPIIDAAAA|14478|4|1|18|AM|third|night|| +14479|AAAAAAAAAJIDAAAA|14479|4|1|19|AM|third|night|| +14480|AAAAAAAABJIDAAAA|14480|4|1|20|AM|third|night|| +14481|AAAAAAAACJIDAAAA|14481|4|1|21|AM|third|night|| +14482|AAAAAAAADJIDAAAA|14482|4|1|22|AM|third|night|| +14483|AAAAAAAAEJIDAAAA|14483|4|1|23|AM|third|night|| +14484|AAAAAAAAFJIDAAAA|14484|4|1|24|AM|third|night|| +14485|AAAAAAAAGJIDAAAA|14485|4|1|25|AM|third|night|| +14486|AAAAAAAAHJIDAAAA|14486|4|1|26|AM|third|night|| +14487|AAAAAAAAIJIDAAAA|14487|4|1|27|AM|third|night|| +14488|AAAAAAAAJJIDAAAA|14488|4|1|28|AM|third|night|| +14489|AAAAAAAAKJIDAAAA|14489|4|1|29|AM|third|night|| +14490|AAAAAAAALJIDAAAA|14490|4|1|30|AM|third|night|| +14491|AAAAAAAAMJIDAAAA|14491|4|1|31|AM|third|night|| +14492|AAAAAAAANJIDAAAA|14492|4|1|32|AM|third|night|| +14493|AAAAAAAAOJIDAAAA|14493|4|1|33|AM|third|night|| +14494|AAAAAAAAPJIDAAAA|14494|4|1|34|AM|third|night|| +14495|AAAAAAAAAKIDAAAA|14495|4|1|35|AM|third|night|| +14496|AAAAAAAABKIDAAAA|14496|4|1|36|AM|third|night|| +14497|AAAAAAAACKIDAAAA|14497|4|1|37|AM|third|night|| +14498|AAAAAAAADKIDAAAA|14498|4|1|38|AM|third|night|| +14499|AAAAAAAAEKIDAAAA|14499|4|1|39|AM|third|night|| +14500|AAAAAAAAFKIDAAAA|14500|4|1|40|AM|third|night|| +14501|AAAAAAAAGKIDAAAA|14501|4|1|41|AM|third|night|| +14502|AAAAAAAAHKIDAAAA|14502|4|1|42|AM|third|night|| +14503|AAAAAAAAIKIDAAAA|14503|4|1|43|AM|third|night|| +14504|AAAAAAAAJKIDAAAA|14504|4|1|44|AM|third|night|| +14505|AAAAAAAAKKIDAAAA|14505|4|1|45|AM|third|night|| +14506|AAAAAAAALKIDAAAA|14506|4|1|46|AM|third|night|| +14507|AAAAAAAAMKIDAAAA|14507|4|1|47|AM|third|night|| +14508|AAAAAAAANKIDAAAA|14508|4|1|48|AM|third|night|| +14509|AAAAAAAAOKIDAAAA|14509|4|1|49|AM|third|night|| +14510|AAAAAAAAPKIDAAAA|14510|4|1|50|AM|third|night|| +14511|AAAAAAAAALIDAAAA|14511|4|1|51|AM|third|night|| +14512|AAAAAAAABLIDAAAA|14512|4|1|52|AM|third|night|| +14513|AAAAAAAACLIDAAAA|14513|4|1|53|AM|third|night|| +14514|AAAAAAAADLIDAAAA|14514|4|1|54|AM|third|night|| +14515|AAAAAAAAELIDAAAA|14515|4|1|55|AM|third|night|| +14516|AAAAAAAAFLIDAAAA|14516|4|1|56|AM|third|night|| +14517|AAAAAAAAGLIDAAAA|14517|4|1|57|AM|third|night|| +14518|AAAAAAAAHLIDAAAA|14518|4|1|58|AM|third|night|| +14519|AAAAAAAAILIDAAAA|14519|4|1|59|AM|third|night|| +14520|AAAAAAAAJLIDAAAA|14520|4|2|0|AM|third|night|| +14521|AAAAAAAAKLIDAAAA|14521|4|2|1|AM|third|night|| +14522|AAAAAAAALLIDAAAA|14522|4|2|2|AM|third|night|| +14523|AAAAAAAAMLIDAAAA|14523|4|2|3|AM|third|night|| +14524|AAAAAAAANLIDAAAA|14524|4|2|4|AM|third|night|| +14525|AAAAAAAAOLIDAAAA|14525|4|2|5|AM|third|night|| +14526|AAAAAAAAPLIDAAAA|14526|4|2|6|AM|third|night|| +14527|AAAAAAAAAMIDAAAA|14527|4|2|7|AM|third|night|| +14528|AAAAAAAABMIDAAAA|14528|4|2|8|AM|third|night|| +14529|AAAAAAAACMIDAAAA|14529|4|2|9|AM|third|night|| +14530|AAAAAAAADMIDAAAA|14530|4|2|10|AM|third|night|| +14531|AAAAAAAAEMIDAAAA|14531|4|2|11|AM|third|night|| +14532|AAAAAAAAFMIDAAAA|14532|4|2|12|AM|third|night|| +14533|AAAAAAAAGMIDAAAA|14533|4|2|13|AM|third|night|| +14534|AAAAAAAAHMIDAAAA|14534|4|2|14|AM|third|night|| +14535|AAAAAAAAIMIDAAAA|14535|4|2|15|AM|third|night|| +14536|AAAAAAAAJMIDAAAA|14536|4|2|16|AM|third|night|| +14537|AAAAAAAAKMIDAAAA|14537|4|2|17|AM|third|night|| +14538|AAAAAAAALMIDAAAA|14538|4|2|18|AM|third|night|| +14539|AAAAAAAAMMIDAAAA|14539|4|2|19|AM|third|night|| +14540|AAAAAAAANMIDAAAA|14540|4|2|20|AM|third|night|| +14541|AAAAAAAAOMIDAAAA|14541|4|2|21|AM|third|night|| +14542|AAAAAAAAPMIDAAAA|14542|4|2|22|AM|third|night|| +14543|AAAAAAAAANIDAAAA|14543|4|2|23|AM|third|night|| +14544|AAAAAAAABNIDAAAA|14544|4|2|24|AM|third|night|| +14545|AAAAAAAACNIDAAAA|14545|4|2|25|AM|third|night|| +14546|AAAAAAAADNIDAAAA|14546|4|2|26|AM|third|night|| +14547|AAAAAAAAENIDAAAA|14547|4|2|27|AM|third|night|| +14548|AAAAAAAAFNIDAAAA|14548|4|2|28|AM|third|night|| +14549|AAAAAAAAGNIDAAAA|14549|4|2|29|AM|third|night|| +14550|AAAAAAAAHNIDAAAA|14550|4|2|30|AM|third|night|| +14551|AAAAAAAAINIDAAAA|14551|4|2|31|AM|third|night|| +14552|AAAAAAAAJNIDAAAA|14552|4|2|32|AM|third|night|| +14553|AAAAAAAAKNIDAAAA|14553|4|2|33|AM|third|night|| +14554|AAAAAAAALNIDAAAA|14554|4|2|34|AM|third|night|| +14555|AAAAAAAAMNIDAAAA|14555|4|2|35|AM|third|night|| +14556|AAAAAAAANNIDAAAA|14556|4|2|36|AM|third|night|| +14557|AAAAAAAAONIDAAAA|14557|4|2|37|AM|third|night|| +14558|AAAAAAAAPNIDAAAA|14558|4|2|38|AM|third|night|| +14559|AAAAAAAAAOIDAAAA|14559|4|2|39|AM|third|night|| +14560|AAAAAAAABOIDAAAA|14560|4|2|40|AM|third|night|| +14561|AAAAAAAACOIDAAAA|14561|4|2|41|AM|third|night|| +14562|AAAAAAAADOIDAAAA|14562|4|2|42|AM|third|night|| +14563|AAAAAAAAEOIDAAAA|14563|4|2|43|AM|third|night|| +14564|AAAAAAAAFOIDAAAA|14564|4|2|44|AM|third|night|| +14565|AAAAAAAAGOIDAAAA|14565|4|2|45|AM|third|night|| +14566|AAAAAAAAHOIDAAAA|14566|4|2|46|AM|third|night|| +14567|AAAAAAAAIOIDAAAA|14567|4|2|47|AM|third|night|| +14568|AAAAAAAAJOIDAAAA|14568|4|2|48|AM|third|night|| +14569|AAAAAAAAKOIDAAAA|14569|4|2|49|AM|third|night|| +14570|AAAAAAAALOIDAAAA|14570|4|2|50|AM|third|night|| +14571|AAAAAAAAMOIDAAAA|14571|4|2|51|AM|third|night|| +14572|AAAAAAAANOIDAAAA|14572|4|2|52|AM|third|night|| +14573|AAAAAAAAOOIDAAAA|14573|4|2|53|AM|third|night|| +14574|AAAAAAAAPOIDAAAA|14574|4|2|54|AM|third|night|| +14575|AAAAAAAAAPIDAAAA|14575|4|2|55|AM|third|night|| +14576|AAAAAAAABPIDAAAA|14576|4|2|56|AM|third|night|| +14577|AAAAAAAACPIDAAAA|14577|4|2|57|AM|third|night|| +14578|AAAAAAAADPIDAAAA|14578|4|2|58|AM|third|night|| +14579|AAAAAAAAEPIDAAAA|14579|4|2|59|AM|third|night|| +14580|AAAAAAAAFPIDAAAA|14580|4|3|0|AM|third|night|| +14581|AAAAAAAAGPIDAAAA|14581|4|3|1|AM|third|night|| +14582|AAAAAAAAHPIDAAAA|14582|4|3|2|AM|third|night|| +14583|AAAAAAAAIPIDAAAA|14583|4|3|3|AM|third|night|| +14584|AAAAAAAAJPIDAAAA|14584|4|3|4|AM|third|night|| +14585|AAAAAAAAKPIDAAAA|14585|4|3|5|AM|third|night|| +14586|AAAAAAAALPIDAAAA|14586|4|3|6|AM|third|night|| +14587|AAAAAAAAMPIDAAAA|14587|4|3|7|AM|third|night|| +14588|AAAAAAAANPIDAAAA|14588|4|3|8|AM|third|night|| +14589|AAAAAAAAOPIDAAAA|14589|4|3|9|AM|third|night|| +14590|AAAAAAAAPPIDAAAA|14590|4|3|10|AM|third|night|| +14591|AAAAAAAAAAJDAAAA|14591|4|3|11|AM|third|night|| +14592|AAAAAAAABAJDAAAA|14592|4|3|12|AM|third|night|| +14593|AAAAAAAACAJDAAAA|14593|4|3|13|AM|third|night|| +14594|AAAAAAAADAJDAAAA|14594|4|3|14|AM|third|night|| +14595|AAAAAAAAEAJDAAAA|14595|4|3|15|AM|third|night|| +14596|AAAAAAAAFAJDAAAA|14596|4|3|16|AM|third|night|| +14597|AAAAAAAAGAJDAAAA|14597|4|3|17|AM|third|night|| +14598|AAAAAAAAHAJDAAAA|14598|4|3|18|AM|third|night|| +14599|AAAAAAAAIAJDAAAA|14599|4|3|19|AM|third|night|| +14600|AAAAAAAAJAJDAAAA|14600|4|3|20|AM|third|night|| +14601|AAAAAAAAKAJDAAAA|14601|4|3|21|AM|third|night|| +14602|AAAAAAAALAJDAAAA|14602|4|3|22|AM|third|night|| +14603|AAAAAAAAMAJDAAAA|14603|4|3|23|AM|third|night|| +14604|AAAAAAAANAJDAAAA|14604|4|3|24|AM|third|night|| +14605|AAAAAAAAOAJDAAAA|14605|4|3|25|AM|third|night|| +14606|AAAAAAAAPAJDAAAA|14606|4|3|26|AM|third|night|| +14607|AAAAAAAAABJDAAAA|14607|4|3|27|AM|third|night|| +14608|AAAAAAAABBJDAAAA|14608|4|3|28|AM|third|night|| +14609|AAAAAAAACBJDAAAA|14609|4|3|29|AM|third|night|| +14610|AAAAAAAADBJDAAAA|14610|4|3|30|AM|third|night|| +14611|AAAAAAAAEBJDAAAA|14611|4|3|31|AM|third|night|| +14612|AAAAAAAAFBJDAAAA|14612|4|3|32|AM|third|night|| +14613|AAAAAAAAGBJDAAAA|14613|4|3|33|AM|third|night|| +14614|AAAAAAAAHBJDAAAA|14614|4|3|34|AM|third|night|| +14615|AAAAAAAAIBJDAAAA|14615|4|3|35|AM|third|night|| +14616|AAAAAAAAJBJDAAAA|14616|4|3|36|AM|third|night|| +14617|AAAAAAAAKBJDAAAA|14617|4|3|37|AM|third|night|| +14618|AAAAAAAALBJDAAAA|14618|4|3|38|AM|third|night|| +14619|AAAAAAAAMBJDAAAA|14619|4|3|39|AM|third|night|| +14620|AAAAAAAANBJDAAAA|14620|4|3|40|AM|third|night|| +14621|AAAAAAAAOBJDAAAA|14621|4|3|41|AM|third|night|| +14622|AAAAAAAAPBJDAAAA|14622|4|3|42|AM|third|night|| +14623|AAAAAAAAACJDAAAA|14623|4|3|43|AM|third|night|| +14624|AAAAAAAABCJDAAAA|14624|4|3|44|AM|third|night|| +14625|AAAAAAAACCJDAAAA|14625|4|3|45|AM|third|night|| +14626|AAAAAAAADCJDAAAA|14626|4|3|46|AM|third|night|| +14627|AAAAAAAAECJDAAAA|14627|4|3|47|AM|third|night|| +14628|AAAAAAAAFCJDAAAA|14628|4|3|48|AM|third|night|| +14629|AAAAAAAAGCJDAAAA|14629|4|3|49|AM|third|night|| +14630|AAAAAAAAHCJDAAAA|14630|4|3|50|AM|third|night|| +14631|AAAAAAAAICJDAAAA|14631|4|3|51|AM|third|night|| +14632|AAAAAAAAJCJDAAAA|14632|4|3|52|AM|third|night|| +14633|AAAAAAAAKCJDAAAA|14633|4|3|53|AM|third|night|| +14634|AAAAAAAALCJDAAAA|14634|4|3|54|AM|third|night|| +14635|AAAAAAAAMCJDAAAA|14635|4|3|55|AM|third|night|| +14636|AAAAAAAANCJDAAAA|14636|4|3|56|AM|third|night|| +14637|AAAAAAAAOCJDAAAA|14637|4|3|57|AM|third|night|| +14638|AAAAAAAAPCJDAAAA|14638|4|3|58|AM|third|night|| +14639|AAAAAAAAADJDAAAA|14639|4|3|59|AM|third|night|| +14640|AAAAAAAABDJDAAAA|14640|4|4|0|AM|third|night|| +14641|AAAAAAAACDJDAAAA|14641|4|4|1|AM|third|night|| +14642|AAAAAAAADDJDAAAA|14642|4|4|2|AM|third|night|| +14643|AAAAAAAAEDJDAAAA|14643|4|4|3|AM|third|night|| +14644|AAAAAAAAFDJDAAAA|14644|4|4|4|AM|third|night|| +14645|AAAAAAAAGDJDAAAA|14645|4|4|5|AM|third|night|| +14646|AAAAAAAAHDJDAAAA|14646|4|4|6|AM|third|night|| +14647|AAAAAAAAIDJDAAAA|14647|4|4|7|AM|third|night|| +14648|AAAAAAAAJDJDAAAA|14648|4|4|8|AM|third|night|| +14649|AAAAAAAAKDJDAAAA|14649|4|4|9|AM|third|night|| +14650|AAAAAAAALDJDAAAA|14650|4|4|10|AM|third|night|| +14651|AAAAAAAAMDJDAAAA|14651|4|4|11|AM|third|night|| +14652|AAAAAAAANDJDAAAA|14652|4|4|12|AM|third|night|| +14653|AAAAAAAAODJDAAAA|14653|4|4|13|AM|third|night|| +14654|AAAAAAAAPDJDAAAA|14654|4|4|14|AM|third|night|| +14655|AAAAAAAAAEJDAAAA|14655|4|4|15|AM|third|night|| +14656|AAAAAAAABEJDAAAA|14656|4|4|16|AM|third|night|| +14657|AAAAAAAACEJDAAAA|14657|4|4|17|AM|third|night|| +14658|AAAAAAAADEJDAAAA|14658|4|4|18|AM|third|night|| +14659|AAAAAAAAEEJDAAAA|14659|4|4|19|AM|third|night|| +14660|AAAAAAAAFEJDAAAA|14660|4|4|20|AM|third|night|| +14661|AAAAAAAAGEJDAAAA|14661|4|4|21|AM|third|night|| +14662|AAAAAAAAHEJDAAAA|14662|4|4|22|AM|third|night|| +14663|AAAAAAAAIEJDAAAA|14663|4|4|23|AM|third|night|| +14664|AAAAAAAAJEJDAAAA|14664|4|4|24|AM|third|night|| +14665|AAAAAAAAKEJDAAAA|14665|4|4|25|AM|third|night|| +14666|AAAAAAAALEJDAAAA|14666|4|4|26|AM|third|night|| +14667|AAAAAAAAMEJDAAAA|14667|4|4|27|AM|third|night|| +14668|AAAAAAAANEJDAAAA|14668|4|4|28|AM|third|night|| +14669|AAAAAAAAOEJDAAAA|14669|4|4|29|AM|third|night|| +14670|AAAAAAAAPEJDAAAA|14670|4|4|30|AM|third|night|| +14671|AAAAAAAAAFJDAAAA|14671|4|4|31|AM|third|night|| +14672|AAAAAAAABFJDAAAA|14672|4|4|32|AM|third|night|| +14673|AAAAAAAACFJDAAAA|14673|4|4|33|AM|third|night|| +14674|AAAAAAAADFJDAAAA|14674|4|4|34|AM|third|night|| +14675|AAAAAAAAEFJDAAAA|14675|4|4|35|AM|third|night|| +14676|AAAAAAAAFFJDAAAA|14676|4|4|36|AM|third|night|| +14677|AAAAAAAAGFJDAAAA|14677|4|4|37|AM|third|night|| +14678|AAAAAAAAHFJDAAAA|14678|4|4|38|AM|third|night|| +14679|AAAAAAAAIFJDAAAA|14679|4|4|39|AM|third|night|| +14680|AAAAAAAAJFJDAAAA|14680|4|4|40|AM|third|night|| +14681|AAAAAAAAKFJDAAAA|14681|4|4|41|AM|third|night|| +14682|AAAAAAAALFJDAAAA|14682|4|4|42|AM|third|night|| +14683|AAAAAAAAMFJDAAAA|14683|4|4|43|AM|third|night|| +14684|AAAAAAAANFJDAAAA|14684|4|4|44|AM|third|night|| +14685|AAAAAAAAOFJDAAAA|14685|4|4|45|AM|third|night|| +14686|AAAAAAAAPFJDAAAA|14686|4|4|46|AM|third|night|| +14687|AAAAAAAAAGJDAAAA|14687|4|4|47|AM|third|night|| +14688|AAAAAAAABGJDAAAA|14688|4|4|48|AM|third|night|| +14689|AAAAAAAACGJDAAAA|14689|4|4|49|AM|third|night|| +14690|AAAAAAAADGJDAAAA|14690|4|4|50|AM|third|night|| +14691|AAAAAAAAEGJDAAAA|14691|4|4|51|AM|third|night|| +14692|AAAAAAAAFGJDAAAA|14692|4|4|52|AM|third|night|| +14693|AAAAAAAAGGJDAAAA|14693|4|4|53|AM|third|night|| +14694|AAAAAAAAHGJDAAAA|14694|4|4|54|AM|third|night|| +14695|AAAAAAAAIGJDAAAA|14695|4|4|55|AM|third|night|| +14696|AAAAAAAAJGJDAAAA|14696|4|4|56|AM|third|night|| +14697|AAAAAAAAKGJDAAAA|14697|4|4|57|AM|third|night|| +14698|AAAAAAAALGJDAAAA|14698|4|4|58|AM|third|night|| +14699|AAAAAAAAMGJDAAAA|14699|4|4|59|AM|third|night|| +14700|AAAAAAAANGJDAAAA|14700|4|5|0|AM|third|night|| +14701|AAAAAAAAOGJDAAAA|14701|4|5|1|AM|third|night|| +14702|AAAAAAAAPGJDAAAA|14702|4|5|2|AM|third|night|| +14703|AAAAAAAAAHJDAAAA|14703|4|5|3|AM|third|night|| +14704|AAAAAAAABHJDAAAA|14704|4|5|4|AM|third|night|| +14705|AAAAAAAACHJDAAAA|14705|4|5|5|AM|third|night|| +14706|AAAAAAAADHJDAAAA|14706|4|5|6|AM|third|night|| +14707|AAAAAAAAEHJDAAAA|14707|4|5|7|AM|third|night|| +14708|AAAAAAAAFHJDAAAA|14708|4|5|8|AM|third|night|| +14709|AAAAAAAAGHJDAAAA|14709|4|5|9|AM|third|night|| +14710|AAAAAAAAHHJDAAAA|14710|4|5|10|AM|third|night|| +14711|AAAAAAAAIHJDAAAA|14711|4|5|11|AM|third|night|| +14712|AAAAAAAAJHJDAAAA|14712|4|5|12|AM|third|night|| +14713|AAAAAAAAKHJDAAAA|14713|4|5|13|AM|third|night|| +14714|AAAAAAAALHJDAAAA|14714|4|5|14|AM|third|night|| +14715|AAAAAAAAMHJDAAAA|14715|4|5|15|AM|third|night|| +14716|AAAAAAAANHJDAAAA|14716|4|5|16|AM|third|night|| +14717|AAAAAAAAOHJDAAAA|14717|4|5|17|AM|third|night|| +14718|AAAAAAAAPHJDAAAA|14718|4|5|18|AM|third|night|| +14719|AAAAAAAAAIJDAAAA|14719|4|5|19|AM|third|night|| +14720|AAAAAAAABIJDAAAA|14720|4|5|20|AM|third|night|| +14721|AAAAAAAACIJDAAAA|14721|4|5|21|AM|third|night|| +14722|AAAAAAAADIJDAAAA|14722|4|5|22|AM|third|night|| +14723|AAAAAAAAEIJDAAAA|14723|4|5|23|AM|third|night|| +14724|AAAAAAAAFIJDAAAA|14724|4|5|24|AM|third|night|| +14725|AAAAAAAAGIJDAAAA|14725|4|5|25|AM|third|night|| +14726|AAAAAAAAHIJDAAAA|14726|4|5|26|AM|third|night|| +14727|AAAAAAAAIIJDAAAA|14727|4|5|27|AM|third|night|| +14728|AAAAAAAAJIJDAAAA|14728|4|5|28|AM|third|night|| +14729|AAAAAAAAKIJDAAAA|14729|4|5|29|AM|third|night|| +14730|AAAAAAAALIJDAAAA|14730|4|5|30|AM|third|night|| +14731|AAAAAAAAMIJDAAAA|14731|4|5|31|AM|third|night|| +14732|AAAAAAAANIJDAAAA|14732|4|5|32|AM|third|night|| +14733|AAAAAAAAOIJDAAAA|14733|4|5|33|AM|third|night|| +14734|AAAAAAAAPIJDAAAA|14734|4|5|34|AM|third|night|| +14735|AAAAAAAAAJJDAAAA|14735|4|5|35|AM|third|night|| +14736|AAAAAAAABJJDAAAA|14736|4|5|36|AM|third|night|| +14737|AAAAAAAACJJDAAAA|14737|4|5|37|AM|third|night|| +14738|AAAAAAAADJJDAAAA|14738|4|5|38|AM|third|night|| +14739|AAAAAAAAEJJDAAAA|14739|4|5|39|AM|third|night|| +14740|AAAAAAAAFJJDAAAA|14740|4|5|40|AM|third|night|| +14741|AAAAAAAAGJJDAAAA|14741|4|5|41|AM|third|night|| +14742|AAAAAAAAHJJDAAAA|14742|4|5|42|AM|third|night|| +14743|AAAAAAAAIJJDAAAA|14743|4|5|43|AM|third|night|| +14744|AAAAAAAAJJJDAAAA|14744|4|5|44|AM|third|night|| +14745|AAAAAAAAKJJDAAAA|14745|4|5|45|AM|third|night|| +14746|AAAAAAAALJJDAAAA|14746|4|5|46|AM|third|night|| +14747|AAAAAAAAMJJDAAAA|14747|4|5|47|AM|third|night|| +14748|AAAAAAAANJJDAAAA|14748|4|5|48|AM|third|night|| +14749|AAAAAAAAOJJDAAAA|14749|4|5|49|AM|third|night|| +14750|AAAAAAAAPJJDAAAA|14750|4|5|50|AM|third|night|| +14751|AAAAAAAAAKJDAAAA|14751|4|5|51|AM|third|night|| +14752|AAAAAAAABKJDAAAA|14752|4|5|52|AM|third|night|| +14753|AAAAAAAACKJDAAAA|14753|4|5|53|AM|third|night|| +14754|AAAAAAAADKJDAAAA|14754|4|5|54|AM|third|night|| +14755|AAAAAAAAEKJDAAAA|14755|4|5|55|AM|third|night|| +14756|AAAAAAAAFKJDAAAA|14756|4|5|56|AM|third|night|| +14757|AAAAAAAAGKJDAAAA|14757|4|5|57|AM|third|night|| +14758|AAAAAAAAHKJDAAAA|14758|4|5|58|AM|third|night|| +14759|AAAAAAAAIKJDAAAA|14759|4|5|59|AM|third|night|| +14760|AAAAAAAAJKJDAAAA|14760|4|6|0|AM|third|night|| +14761|AAAAAAAAKKJDAAAA|14761|4|6|1|AM|third|night|| +14762|AAAAAAAALKJDAAAA|14762|4|6|2|AM|third|night|| +14763|AAAAAAAAMKJDAAAA|14763|4|6|3|AM|third|night|| +14764|AAAAAAAANKJDAAAA|14764|4|6|4|AM|third|night|| +14765|AAAAAAAAOKJDAAAA|14765|4|6|5|AM|third|night|| +14766|AAAAAAAAPKJDAAAA|14766|4|6|6|AM|third|night|| +14767|AAAAAAAAALJDAAAA|14767|4|6|7|AM|third|night|| +14768|AAAAAAAABLJDAAAA|14768|4|6|8|AM|third|night|| +14769|AAAAAAAACLJDAAAA|14769|4|6|9|AM|third|night|| +14770|AAAAAAAADLJDAAAA|14770|4|6|10|AM|third|night|| +14771|AAAAAAAAELJDAAAA|14771|4|6|11|AM|third|night|| +14772|AAAAAAAAFLJDAAAA|14772|4|6|12|AM|third|night|| +14773|AAAAAAAAGLJDAAAA|14773|4|6|13|AM|third|night|| +14774|AAAAAAAAHLJDAAAA|14774|4|6|14|AM|third|night|| +14775|AAAAAAAAILJDAAAA|14775|4|6|15|AM|third|night|| +14776|AAAAAAAAJLJDAAAA|14776|4|6|16|AM|third|night|| +14777|AAAAAAAAKLJDAAAA|14777|4|6|17|AM|third|night|| +14778|AAAAAAAALLJDAAAA|14778|4|6|18|AM|third|night|| +14779|AAAAAAAAMLJDAAAA|14779|4|6|19|AM|third|night|| +14780|AAAAAAAANLJDAAAA|14780|4|6|20|AM|third|night|| +14781|AAAAAAAAOLJDAAAA|14781|4|6|21|AM|third|night|| +14782|AAAAAAAAPLJDAAAA|14782|4|6|22|AM|third|night|| +14783|AAAAAAAAAMJDAAAA|14783|4|6|23|AM|third|night|| +14784|AAAAAAAABMJDAAAA|14784|4|6|24|AM|third|night|| +14785|AAAAAAAACMJDAAAA|14785|4|6|25|AM|third|night|| +14786|AAAAAAAADMJDAAAA|14786|4|6|26|AM|third|night|| +14787|AAAAAAAAEMJDAAAA|14787|4|6|27|AM|third|night|| +14788|AAAAAAAAFMJDAAAA|14788|4|6|28|AM|third|night|| +14789|AAAAAAAAGMJDAAAA|14789|4|6|29|AM|third|night|| +14790|AAAAAAAAHMJDAAAA|14790|4|6|30|AM|third|night|| +14791|AAAAAAAAIMJDAAAA|14791|4|6|31|AM|third|night|| +14792|AAAAAAAAJMJDAAAA|14792|4|6|32|AM|third|night|| +14793|AAAAAAAAKMJDAAAA|14793|4|6|33|AM|third|night|| +14794|AAAAAAAALMJDAAAA|14794|4|6|34|AM|third|night|| +14795|AAAAAAAAMMJDAAAA|14795|4|6|35|AM|third|night|| +14796|AAAAAAAANMJDAAAA|14796|4|6|36|AM|third|night|| +14797|AAAAAAAAOMJDAAAA|14797|4|6|37|AM|third|night|| +14798|AAAAAAAAPMJDAAAA|14798|4|6|38|AM|third|night|| +14799|AAAAAAAAANJDAAAA|14799|4|6|39|AM|third|night|| +14800|AAAAAAAABNJDAAAA|14800|4|6|40|AM|third|night|| +14801|AAAAAAAACNJDAAAA|14801|4|6|41|AM|third|night|| +14802|AAAAAAAADNJDAAAA|14802|4|6|42|AM|third|night|| +14803|AAAAAAAAENJDAAAA|14803|4|6|43|AM|third|night|| +14804|AAAAAAAAFNJDAAAA|14804|4|6|44|AM|third|night|| +14805|AAAAAAAAGNJDAAAA|14805|4|6|45|AM|third|night|| +14806|AAAAAAAAHNJDAAAA|14806|4|6|46|AM|third|night|| +14807|AAAAAAAAINJDAAAA|14807|4|6|47|AM|third|night|| +14808|AAAAAAAAJNJDAAAA|14808|4|6|48|AM|third|night|| +14809|AAAAAAAAKNJDAAAA|14809|4|6|49|AM|third|night|| +14810|AAAAAAAALNJDAAAA|14810|4|6|50|AM|third|night|| +14811|AAAAAAAAMNJDAAAA|14811|4|6|51|AM|third|night|| +14812|AAAAAAAANNJDAAAA|14812|4|6|52|AM|third|night|| +14813|AAAAAAAAONJDAAAA|14813|4|6|53|AM|third|night|| +14814|AAAAAAAAPNJDAAAA|14814|4|6|54|AM|third|night|| +14815|AAAAAAAAAOJDAAAA|14815|4|6|55|AM|third|night|| +14816|AAAAAAAABOJDAAAA|14816|4|6|56|AM|third|night|| +14817|AAAAAAAACOJDAAAA|14817|4|6|57|AM|third|night|| +14818|AAAAAAAADOJDAAAA|14818|4|6|58|AM|third|night|| +14819|AAAAAAAAEOJDAAAA|14819|4|6|59|AM|third|night|| +14820|AAAAAAAAFOJDAAAA|14820|4|7|0|AM|third|night|| +14821|AAAAAAAAGOJDAAAA|14821|4|7|1|AM|third|night|| +14822|AAAAAAAAHOJDAAAA|14822|4|7|2|AM|third|night|| +14823|AAAAAAAAIOJDAAAA|14823|4|7|3|AM|third|night|| +14824|AAAAAAAAJOJDAAAA|14824|4|7|4|AM|third|night|| +14825|AAAAAAAAKOJDAAAA|14825|4|7|5|AM|third|night|| +14826|AAAAAAAALOJDAAAA|14826|4|7|6|AM|third|night|| +14827|AAAAAAAAMOJDAAAA|14827|4|7|7|AM|third|night|| +14828|AAAAAAAANOJDAAAA|14828|4|7|8|AM|third|night|| +14829|AAAAAAAAOOJDAAAA|14829|4|7|9|AM|third|night|| +14830|AAAAAAAAPOJDAAAA|14830|4|7|10|AM|third|night|| +14831|AAAAAAAAAPJDAAAA|14831|4|7|11|AM|third|night|| +14832|AAAAAAAABPJDAAAA|14832|4|7|12|AM|third|night|| +14833|AAAAAAAACPJDAAAA|14833|4|7|13|AM|third|night|| +14834|AAAAAAAADPJDAAAA|14834|4|7|14|AM|third|night|| +14835|AAAAAAAAEPJDAAAA|14835|4|7|15|AM|third|night|| +14836|AAAAAAAAFPJDAAAA|14836|4|7|16|AM|third|night|| +14837|AAAAAAAAGPJDAAAA|14837|4|7|17|AM|third|night|| +14838|AAAAAAAAHPJDAAAA|14838|4|7|18|AM|third|night|| +14839|AAAAAAAAIPJDAAAA|14839|4|7|19|AM|third|night|| +14840|AAAAAAAAJPJDAAAA|14840|4|7|20|AM|third|night|| +14841|AAAAAAAAKPJDAAAA|14841|4|7|21|AM|third|night|| +14842|AAAAAAAALPJDAAAA|14842|4|7|22|AM|third|night|| +14843|AAAAAAAAMPJDAAAA|14843|4|7|23|AM|third|night|| +14844|AAAAAAAANPJDAAAA|14844|4|7|24|AM|third|night|| +14845|AAAAAAAAOPJDAAAA|14845|4|7|25|AM|third|night|| +14846|AAAAAAAAPPJDAAAA|14846|4|7|26|AM|third|night|| +14847|AAAAAAAAAAKDAAAA|14847|4|7|27|AM|third|night|| +14848|AAAAAAAABAKDAAAA|14848|4|7|28|AM|third|night|| +14849|AAAAAAAACAKDAAAA|14849|4|7|29|AM|third|night|| +14850|AAAAAAAADAKDAAAA|14850|4|7|30|AM|third|night|| +14851|AAAAAAAAEAKDAAAA|14851|4|7|31|AM|third|night|| +14852|AAAAAAAAFAKDAAAA|14852|4|7|32|AM|third|night|| +14853|AAAAAAAAGAKDAAAA|14853|4|7|33|AM|third|night|| +14854|AAAAAAAAHAKDAAAA|14854|4|7|34|AM|third|night|| +14855|AAAAAAAAIAKDAAAA|14855|4|7|35|AM|third|night|| +14856|AAAAAAAAJAKDAAAA|14856|4|7|36|AM|third|night|| +14857|AAAAAAAAKAKDAAAA|14857|4|7|37|AM|third|night|| +14858|AAAAAAAALAKDAAAA|14858|4|7|38|AM|third|night|| +14859|AAAAAAAAMAKDAAAA|14859|4|7|39|AM|third|night|| +14860|AAAAAAAANAKDAAAA|14860|4|7|40|AM|third|night|| +14861|AAAAAAAAOAKDAAAA|14861|4|7|41|AM|third|night|| +14862|AAAAAAAAPAKDAAAA|14862|4|7|42|AM|third|night|| +14863|AAAAAAAAABKDAAAA|14863|4|7|43|AM|third|night|| +14864|AAAAAAAABBKDAAAA|14864|4|7|44|AM|third|night|| +14865|AAAAAAAACBKDAAAA|14865|4|7|45|AM|third|night|| +14866|AAAAAAAADBKDAAAA|14866|4|7|46|AM|third|night|| +14867|AAAAAAAAEBKDAAAA|14867|4|7|47|AM|third|night|| +14868|AAAAAAAAFBKDAAAA|14868|4|7|48|AM|third|night|| +14869|AAAAAAAAGBKDAAAA|14869|4|7|49|AM|third|night|| +14870|AAAAAAAAHBKDAAAA|14870|4|7|50|AM|third|night|| +14871|AAAAAAAAIBKDAAAA|14871|4|7|51|AM|third|night|| +14872|AAAAAAAAJBKDAAAA|14872|4|7|52|AM|third|night|| +14873|AAAAAAAAKBKDAAAA|14873|4|7|53|AM|third|night|| +14874|AAAAAAAALBKDAAAA|14874|4|7|54|AM|third|night|| +14875|AAAAAAAAMBKDAAAA|14875|4|7|55|AM|third|night|| +14876|AAAAAAAANBKDAAAA|14876|4|7|56|AM|third|night|| +14877|AAAAAAAAOBKDAAAA|14877|4|7|57|AM|third|night|| +14878|AAAAAAAAPBKDAAAA|14878|4|7|58|AM|third|night|| +14879|AAAAAAAAACKDAAAA|14879|4|7|59|AM|third|night|| +14880|AAAAAAAABCKDAAAA|14880|4|8|0|AM|third|night|| +14881|AAAAAAAACCKDAAAA|14881|4|8|1|AM|third|night|| +14882|AAAAAAAADCKDAAAA|14882|4|8|2|AM|third|night|| +14883|AAAAAAAAECKDAAAA|14883|4|8|3|AM|third|night|| +14884|AAAAAAAAFCKDAAAA|14884|4|8|4|AM|third|night|| +14885|AAAAAAAAGCKDAAAA|14885|4|8|5|AM|third|night|| +14886|AAAAAAAAHCKDAAAA|14886|4|8|6|AM|third|night|| +14887|AAAAAAAAICKDAAAA|14887|4|8|7|AM|third|night|| +14888|AAAAAAAAJCKDAAAA|14888|4|8|8|AM|third|night|| +14889|AAAAAAAAKCKDAAAA|14889|4|8|9|AM|third|night|| +14890|AAAAAAAALCKDAAAA|14890|4|8|10|AM|third|night|| +14891|AAAAAAAAMCKDAAAA|14891|4|8|11|AM|third|night|| +14892|AAAAAAAANCKDAAAA|14892|4|8|12|AM|third|night|| +14893|AAAAAAAAOCKDAAAA|14893|4|8|13|AM|third|night|| +14894|AAAAAAAAPCKDAAAA|14894|4|8|14|AM|third|night|| +14895|AAAAAAAAADKDAAAA|14895|4|8|15|AM|third|night|| +14896|AAAAAAAABDKDAAAA|14896|4|8|16|AM|third|night|| +14897|AAAAAAAACDKDAAAA|14897|4|8|17|AM|third|night|| +14898|AAAAAAAADDKDAAAA|14898|4|8|18|AM|third|night|| +14899|AAAAAAAAEDKDAAAA|14899|4|8|19|AM|third|night|| +14900|AAAAAAAAFDKDAAAA|14900|4|8|20|AM|third|night|| +14901|AAAAAAAAGDKDAAAA|14901|4|8|21|AM|third|night|| +14902|AAAAAAAAHDKDAAAA|14902|4|8|22|AM|third|night|| +14903|AAAAAAAAIDKDAAAA|14903|4|8|23|AM|third|night|| +14904|AAAAAAAAJDKDAAAA|14904|4|8|24|AM|third|night|| +14905|AAAAAAAAKDKDAAAA|14905|4|8|25|AM|third|night|| +14906|AAAAAAAALDKDAAAA|14906|4|8|26|AM|third|night|| +14907|AAAAAAAAMDKDAAAA|14907|4|8|27|AM|third|night|| +14908|AAAAAAAANDKDAAAA|14908|4|8|28|AM|third|night|| +14909|AAAAAAAAODKDAAAA|14909|4|8|29|AM|third|night|| +14910|AAAAAAAAPDKDAAAA|14910|4|8|30|AM|third|night|| +14911|AAAAAAAAAEKDAAAA|14911|4|8|31|AM|third|night|| +14912|AAAAAAAABEKDAAAA|14912|4|8|32|AM|third|night|| +14913|AAAAAAAACEKDAAAA|14913|4|8|33|AM|third|night|| +14914|AAAAAAAADEKDAAAA|14914|4|8|34|AM|third|night|| +14915|AAAAAAAAEEKDAAAA|14915|4|8|35|AM|third|night|| +14916|AAAAAAAAFEKDAAAA|14916|4|8|36|AM|third|night|| +14917|AAAAAAAAGEKDAAAA|14917|4|8|37|AM|third|night|| +14918|AAAAAAAAHEKDAAAA|14918|4|8|38|AM|third|night|| +14919|AAAAAAAAIEKDAAAA|14919|4|8|39|AM|third|night|| +14920|AAAAAAAAJEKDAAAA|14920|4|8|40|AM|third|night|| +14921|AAAAAAAAKEKDAAAA|14921|4|8|41|AM|third|night|| +14922|AAAAAAAALEKDAAAA|14922|4|8|42|AM|third|night|| +14923|AAAAAAAAMEKDAAAA|14923|4|8|43|AM|third|night|| +14924|AAAAAAAANEKDAAAA|14924|4|8|44|AM|third|night|| +14925|AAAAAAAAOEKDAAAA|14925|4|8|45|AM|third|night|| +14926|AAAAAAAAPEKDAAAA|14926|4|8|46|AM|third|night|| +14927|AAAAAAAAAFKDAAAA|14927|4|8|47|AM|third|night|| +14928|AAAAAAAABFKDAAAA|14928|4|8|48|AM|third|night|| +14929|AAAAAAAACFKDAAAA|14929|4|8|49|AM|third|night|| +14930|AAAAAAAADFKDAAAA|14930|4|8|50|AM|third|night|| +14931|AAAAAAAAEFKDAAAA|14931|4|8|51|AM|third|night|| +14932|AAAAAAAAFFKDAAAA|14932|4|8|52|AM|third|night|| +14933|AAAAAAAAGFKDAAAA|14933|4|8|53|AM|third|night|| +14934|AAAAAAAAHFKDAAAA|14934|4|8|54|AM|third|night|| +14935|AAAAAAAAIFKDAAAA|14935|4|8|55|AM|third|night|| +14936|AAAAAAAAJFKDAAAA|14936|4|8|56|AM|third|night|| +14937|AAAAAAAAKFKDAAAA|14937|4|8|57|AM|third|night|| +14938|AAAAAAAALFKDAAAA|14938|4|8|58|AM|third|night|| +14939|AAAAAAAAMFKDAAAA|14939|4|8|59|AM|third|night|| +14940|AAAAAAAANFKDAAAA|14940|4|9|0|AM|third|night|| +14941|AAAAAAAAOFKDAAAA|14941|4|9|1|AM|third|night|| +14942|AAAAAAAAPFKDAAAA|14942|4|9|2|AM|third|night|| +14943|AAAAAAAAAGKDAAAA|14943|4|9|3|AM|third|night|| +14944|AAAAAAAABGKDAAAA|14944|4|9|4|AM|third|night|| +14945|AAAAAAAACGKDAAAA|14945|4|9|5|AM|third|night|| +14946|AAAAAAAADGKDAAAA|14946|4|9|6|AM|third|night|| +14947|AAAAAAAAEGKDAAAA|14947|4|9|7|AM|third|night|| +14948|AAAAAAAAFGKDAAAA|14948|4|9|8|AM|third|night|| +14949|AAAAAAAAGGKDAAAA|14949|4|9|9|AM|third|night|| +14950|AAAAAAAAHGKDAAAA|14950|4|9|10|AM|third|night|| +14951|AAAAAAAAIGKDAAAA|14951|4|9|11|AM|third|night|| +14952|AAAAAAAAJGKDAAAA|14952|4|9|12|AM|third|night|| +14953|AAAAAAAAKGKDAAAA|14953|4|9|13|AM|third|night|| +14954|AAAAAAAALGKDAAAA|14954|4|9|14|AM|third|night|| +14955|AAAAAAAAMGKDAAAA|14955|4|9|15|AM|third|night|| +14956|AAAAAAAANGKDAAAA|14956|4|9|16|AM|third|night|| +14957|AAAAAAAAOGKDAAAA|14957|4|9|17|AM|third|night|| +14958|AAAAAAAAPGKDAAAA|14958|4|9|18|AM|third|night|| +14959|AAAAAAAAAHKDAAAA|14959|4|9|19|AM|third|night|| +14960|AAAAAAAABHKDAAAA|14960|4|9|20|AM|third|night|| +14961|AAAAAAAACHKDAAAA|14961|4|9|21|AM|third|night|| +14962|AAAAAAAADHKDAAAA|14962|4|9|22|AM|third|night|| +14963|AAAAAAAAEHKDAAAA|14963|4|9|23|AM|third|night|| +14964|AAAAAAAAFHKDAAAA|14964|4|9|24|AM|third|night|| +14965|AAAAAAAAGHKDAAAA|14965|4|9|25|AM|third|night|| +14966|AAAAAAAAHHKDAAAA|14966|4|9|26|AM|third|night|| +14967|AAAAAAAAIHKDAAAA|14967|4|9|27|AM|third|night|| +14968|AAAAAAAAJHKDAAAA|14968|4|9|28|AM|third|night|| +14969|AAAAAAAAKHKDAAAA|14969|4|9|29|AM|third|night|| +14970|AAAAAAAALHKDAAAA|14970|4|9|30|AM|third|night|| +14971|AAAAAAAAMHKDAAAA|14971|4|9|31|AM|third|night|| +14972|AAAAAAAANHKDAAAA|14972|4|9|32|AM|third|night|| +14973|AAAAAAAAOHKDAAAA|14973|4|9|33|AM|third|night|| +14974|AAAAAAAAPHKDAAAA|14974|4|9|34|AM|third|night|| +14975|AAAAAAAAAIKDAAAA|14975|4|9|35|AM|third|night|| +14976|AAAAAAAABIKDAAAA|14976|4|9|36|AM|third|night|| +14977|AAAAAAAACIKDAAAA|14977|4|9|37|AM|third|night|| +14978|AAAAAAAADIKDAAAA|14978|4|9|38|AM|third|night|| +14979|AAAAAAAAEIKDAAAA|14979|4|9|39|AM|third|night|| +14980|AAAAAAAAFIKDAAAA|14980|4|9|40|AM|third|night|| +14981|AAAAAAAAGIKDAAAA|14981|4|9|41|AM|third|night|| +14982|AAAAAAAAHIKDAAAA|14982|4|9|42|AM|third|night|| +14983|AAAAAAAAIIKDAAAA|14983|4|9|43|AM|third|night|| +14984|AAAAAAAAJIKDAAAA|14984|4|9|44|AM|third|night|| +14985|AAAAAAAAKIKDAAAA|14985|4|9|45|AM|third|night|| +14986|AAAAAAAALIKDAAAA|14986|4|9|46|AM|third|night|| +14987|AAAAAAAAMIKDAAAA|14987|4|9|47|AM|third|night|| +14988|AAAAAAAANIKDAAAA|14988|4|9|48|AM|third|night|| +14989|AAAAAAAAOIKDAAAA|14989|4|9|49|AM|third|night|| +14990|AAAAAAAAPIKDAAAA|14990|4|9|50|AM|third|night|| +14991|AAAAAAAAAJKDAAAA|14991|4|9|51|AM|third|night|| +14992|AAAAAAAABJKDAAAA|14992|4|9|52|AM|third|night|| +14993|AAAAAAAACJKDAAAA|14993|4|9|53|AM|third|night|| +14994|AAAAAAAADJKDAAAA|14994|4|9|54|AM|third|night|| +14995|AAAAAAAAEJKDAAAA|14995|4|9|55|AM|third|night|| +14996|AAAAAAAAFJKDAAAA|14996|4|9|56|AM|third|night|| +14997|AAAAAAAAGJKDAAAA|14997|4|9|57|AM|third|night|| +14998|AAAAAAAAHJKDAAAA|14998|4|9|58|AM|third|night|| +14999|AAAAAAAAIJKDAAAA|14999|4|9|59|AM|third|night|| +15000|AAAAAAAAJJKDAAAA|15000|4|10|0|AM|third|night|| +15001|AAAAAAAAKJKDAAAA|15001|4|10|1|AM|third|night|| +15002|AAAAAAAALJKDAAAA|15002|4|10|2|AM|third|night|| +15003|AAAAAAAAMJKDAAAA|15003|4|10|3|AM|third|night|| +15004|AAAAAAAANJKDAAAA|15004|4|10|4|AM|third|night|| +15005|AAAAAAAAOJKDAAAA|15005|4|10|5|AM|third|night|| +15006|AAAAAAAAPJKDAAAA|15006|4|10|6|AM|third|night|| +15007|AAAAAAAAAKKDAAAA|15007|4|10|7|AM|third|night|| +15008|AAAAAAAABKKDAAAA|15008|4|10|8|AM|third|night|| +15009|AAAAAAAACKKDAAAA|15009|4|10|9|AM|third|night|| +15010|AAAAAAAADKKDAAAA|15010|4|10|10|AM|third|night|| +15011|AAAAAAAAEKKDAAAA|15011|4|10|11|AM|third|night|| +15012|AAAAAAAAFKKDAAAA|15012|4|10|12|AM|third|night|| +15013|AAAAAAAAGKKDAAAA|15013|4|10|13|AM|third|night|| +15014|AAAAAAAAHKKDAAAA|15014|4|10|14|AM|third|night|| +15015|AAAAAAAAIKKDAAAA|15015|4|10|15|AM|third|night|| +15016|AAAAAAAAJKKDAAAA|15016|4|10|16|AM|third|night|| +15017|AAAAAAAAKKKDAAAA|15017|4|10|17|AM|third|night|| +15018|AAAAAAAALKKDAAAA|15018|4|10|18|AM|third|night|| +15019|AAAAAAAAMKKDAAAA|15019|4|10|19|AM|third|night|| +15020|AAAAAAAANKKDAAAA|15020|4|10|20|AM|third|night|| +15021|AAAAAAAAOKKDAAAA|15021|4|10|21|AM|third|night|| +15022|AAAAAAAAPKKDAAAA|15022|4|10|22|AM|third|night|| +15023|AAAAAAAAALKDAAAA|15023|4|10|23|AM|third|night|| +15024|AAAAAAAABLKDAAAA|15024|4|10|24|AM|third|night|| +15025|AAAAAAAACLKDAAAA|15025|4|10|25|AM|third|night|| +15026|AAAAAAAADLKDAAAA|15026|4|10|26|AM|third|night|| +15027|AAAAAAAAELKDAAAA|15027|4|10|27|AM|third|night|| +15028|AAAAAAAAFLKDAAAA|15028|4|10|28|AM|third|night|| +15029|AAAAAAAAGLKDAAAA|15029|4|10|29|AM|third|night|| +15030|AAAAAAAAHLKDAAAA|15030|4|10|30|AM|third|night|| +15031|AAAAAAAAILKDAAAA|15031|4|10|31|AM|third|night|| +15032|AAAAAAAAJLKDAAAA|15032|4|10|32|AM|third|night|| +15033|AAAAAAAAKLKDAAAA|15033|4|10|33|AM|third|night|| +15034|AAAAAAAALLKDAAAA|15034|4|10|34|AM|third|night|| +15035|AAAAAAAAMLKDAAAA|15035|4|10|35|AM|third|night|| +15036|AAAAAAAANLKDAAAA|15036|4|10|36|AM|third|night|| +15037|AAAAAAAAOLKDAAAA|15037|4|10|37|AM|third|night|| +15038|AAAAAAAAPLKDAAAA|15038|4|10|38|AM|third|night|| +15039|AAAAAAAAAMKDAAAA|15039|4|10|39|AM|third|night|| +15040|AAAAAAAABMKDAAAA|15040|4|10|40|AM|third|night|| +15041|AAAAAAAACMKDAAAA|15041|4|10|41|AM|third|night|| +15042|AAAAAAAADMKDAAAA|15042|4|10|42|AM|third|night|| +15043|AAAAAAAAEMKDAAAA|15043|4|10|43|AM|third|night|| +15044|AAAAAAAAFMKDAAAA|15044|4|10|44|AM|third|night|| +15045|AAAAAAAAGMKDAAAA|15045|4|10|45|AM|third|night|| +15046|AAAAAAAAHMKDAAAA|15046|4|10|46|AM|third|night|| +15047|AAAAAAAAIMKDAAAA|15047|4|10|47|AM|third|night|| +15048|AAAAAAAAJMKDAAAA|15048|4|10|48|AM|third|night|| +15049|AAAAAAAAKMKDAAAA|15049|4|10|49|AM|third|night|| +15050|AAAAAAAALMKDAAAA|15050|4|10|50|AM|third|night|| +15051|AAAAAAAAMMKDAAAA|15051|4|10|51|AM|third|night|| +15052|AAAAAAAANMKDAAAA|15052|4|10|52|AM|third|night|| +15053|AAAAAAAAOMKDAAAA|15053|4|10|53|AM|third|night|| +15054|AAAAAAAAPMKDAAAA|15054|4|10|54|AM|third|night|| +15055|AAAAAAAAANKDAAAA|15055|4|10|55|AM|third|night|| +15056|AAAAAAAABNKDAAAA|15056|4|10|56|AM|third|night|| +15057|AAAAAAAACNKDAAAA|15057|4|10|57|AM|third|night|| +15058|AAAAAAAADNKDAAAA|15058|4|10|58|AM|third|night|| +15059|AAAAAAAAENKDAAAA|15059|4|10|59|AM|third|night|| +15060|AAAAAAAAFNKDAAAA|15060|4|11|0|AM|third|night|| +15061|AAAAAAAAGNKDAAAA|15061|4|11|1|AM|third|night|| +15062|AAAAAAAAHNKDAAAA|15062|4|11|2|AM|third|night|| +15063|AAAAAAAAINKDAAAA|15063|4|11|3|AM|third|night|| +15064|AAAAAAAAJNKDAAAA|15064|4|11|4|AM|third|night|| +15065|AAAAAAAAKNKDAAAA|15065|4|11|5|AM|third|night|| +15066|AAAAAAAALNKDAAAA|15066|4|11|6|AM|third|night|| +15067|AAAAAAAAMNKDAAAA|15067|4|11|7|AM|third|night|| +15068|AAAAAAAANNKDAAAA|15068|4|11|8|AM|third|night|| +15069|AAAAAAAAONKDAAAA|15069|4|11|9|AM|third|night|| +15070|AAAAAAAAPNKDAAAA|15070|4|11|10|AM|third|night|| +15071|AAAAAAAAAOKDAAAA|15071|4|11|11|AM|third|night|| +15072|AAAAAAAABOKDAAAA|15072|4|11|12|AM|third|night|| +15073|AAAAAAAACOKDAAAA|15073|4|11|13|AM|third|night|| +15074|AAAAAAAADOKDAAAA|15074|4|11|14|AM|third|night|| +15075|AAAAAAAAEOKDAAAA|15075|4|11|15|AM|third|night|| +15076|AAAAAAAAFOKDAAAA|15076|4|11|16|AM|third|night|| +15077|AAAAAAAAGOKDAAAA|15077|4|11|17|AM|third|night|| +15078|AAAAAAAAHOKDAAAA|15078|4|11|18|AM|third|night|| +15079|AAAAAAAAIOKDAAAA|15079|4|11|19|AM|third|night|| +15080|AAAAAAAAJOKDAAAA|15080|4|11|20|AM|third|night|| +15081|AAAAAAAAKOKDAAAA|15081|4|11|21|AM|third|night|| +15082|AAAAAAAALOKDAAAA|15082|4|11|22|AM|third|night|| +15083|AAAAAAAAMOKDAAAA|15083|4|11|23|AM|third|night|| +15084|AAAAAAAANOKDAAAA|15084|4|11|24|AM|third|night|| +15085|AAAAAAAAOOKDAAAA|15085|4|11|25|AM|third|night|| +15086|AAAAAAAAPOKDAAAA|15086|4|11|26|AM|third|night|| +15087|AAAAAAAAAPKDAAAA|15087|4|11|27|AM|third|night|| +15088|AAAAAAAABPKDAAAA|15088|4|11|28|AM|third|night|| +15089|AAAAAAAACPKDAAAA|15089|4|11|29|AM|third|night|| +15090|AAAAAAAADPKDAAAA|15090|4|11|30|AM|third|night|| +15091|AAAAAAAAEPKDAAAA|15091|4|11|31|AM|third|night|| +15092|AAAAAAAAFPKDAAAA|15092|4|11|32|AM|third|night|| +15093|AAAAAAAAGPKDAAAA|15093|4|11|33|AM|third|night|| +15094|AAAAAAAAHPKDAAAA|15094|4|11|34|AM|third|night|| +15095|AAAAAAAAIPKDAAAA|15095|4|11|35|AM|third|night|| +15096|AAAAAAAAJPKDAAAA|15096|4|11|36|AM|third|night|| +15097|AAAAAAAAKPKDAAAA|15097|4|11|37|AM|third|night|| +15098|AAAAAAAALPKDAAAA|15098|4|11|38|AM|third|night|| +15099|AAAAAAAAMPKDAAAA|15099|4|11|39|AM|third|night|| +15100|AAAAAAAANPKDAAAA|15100|4|11|40|AM|third|night|| +15101|AAAAAAAAOPKDAAAA|15101|4|11|41|AM|third|night|| +15102|AAAAAAAAPPKDAAAA|15102|4|11|42|AM|third|night|| +15103|AAAAAAAAAALDAAAA|15103|4|11|43|AM|third|night|| +15104|AAAAAAAABALDAAAA|15104|4|11|44|AM|third|night|| +15105|AAAAAAAACALDAAAA|15105|4|11|45|AM|third|night|| +15106|AAAAAAAADALDAAAA|15106|4|11|46|AM|third|night|| +15107|AAAAAAAAEALDAAAA|15107|4|11|47|AM|third|night|| +15108|AAAAAAAAFALDAAAA|15108|4|11|48|AM|third|night|| +15109|AAAAAAAAGALDAAAA|15109|4|11|49|AM|third|night|| +15110|AAAAAAAAHALDAAAA|15110|4|11|50|AM|third|night|| +15111|AAAAAAAAIALDAAAA|15111|4|11|51|AM|third|night|| +15112|AAAAAAAAJALDAAAA|15112|4|11|52|AM|third|night|| +15113|AAAAAAAAKALDAAAA|15113|4|11|53|AM|third|night|| +15114|AAAAAAAALALDAAAA|15114|4|11|54|AM|third|night|| +15115|AAAAAAAAMALDAAAA|15115|4|11|55|AM|third|night|| +15116|AAAAAAAANALDAAAA|15116|4|11|56|AM|third|night|| +15117|AAAAAAAAOALDAAAA|15117|4|11|57|AM|third|night|| +15118|AAAAAAAAPALDAAAA|15118|4|11|58|AM|third|night|| +15119|AAAAAAAAABLDAAAA|15119|4|11|59|AM|third|night|| +15120|AAAAAAAABBLDAAAA|15120|4|12|0|AM|third|night|| +15121|AAAAAAAACBLDAAAA|15121|4|12|1|AM|third|night|| +15122|AAAAAAAADBLDAAAA|15122|4|12|2|AM|third|night|| +15123|AAAAAAAAEBLDAAAA|15123|4|12|3|AM|third|night|| +15124|AAAAAAAAFBLDAAAA|15124|4|12|4|AM|third|night|| +15125|AAAAAAAAGBLDAAAA|15125|4|12|5|AM|third|night|| +15126|AAAAAAAAHBLDAAAA|15126|4|12|6|AM|third|night|| +15127|AAAAAAAAIBLDAAAA|15127|4|12|7|AM|third|night|| +15128|AAAAAAAAJBLDAAAA|15128|4|12|8|AM|third|night|| +15129|AAAAAAAAKBLDAAAA|15129|4|12|9|AM|third|night|| +15130|AAAAAAAALBLDAAAA|15130|4|12|10|AM|third|night|| +15131|AAAAAAAAMBLDAAAA|15131|4|12|11|AM|third|night|| +15132|AAAAAAAANBLDAAAA|15132|4|12|12|AM|third|night|| +15133|AAAAAAAAOBLDAAAA|15133|4|12|13|AM|third|night|| +15134|AAAAAAAAPBLDAAAA|15134|4|12|14|AM|third|night|| +15135|AAAAAAAAACLDAAAA|15135|4|12|15|AM|third|night|| +15136|AAAAAAAABCLDAAAA|15136|4|12|16|AM|third|night|| +15137|AAAAAAAACCLDAAAA|15137|4|12|17|AM|third|night|| +15138|AAAAAAAADCLDAAAA|15138|4|12|18|AM|third|night|| +15139|AAAAAAAAECLDAAAA|15139|4|12|19|AM|third|night|| +15140|AAAAAAAAFCLDAAAA|15140|4|12|20|AM|third|night|| +15141|AAAAAAAAGCLDAAAA|15141|4|12|21|AM|third|night|| +15142|AAAAAAAAHCLDAAAA|15142|4|12|22|AM|third|night|| +15143|AAAAAAAAICLDAAAA|15143|4|12|23|AM|third|night|| +15144|AAAAAAAAJCLDAAAA|15144|4|12|24|AM|third|night|| +15145|AAAAAAAAKCLDAAAA|15145|4|12|25|AM|third|night|| +15146|AAAAAAAALCLDAAAA|15146|4|12|26|AM|third|night|| +15147|AAAAAAAAMCLDAAAA|15147|4|12|27|AM|third|night|| +15148|AAAAAAAANCLDAAAA|15148|4|12|28|AM|third|night|| +15149|AAAAAAAAOCLDAAAA|15149|4|12|29|AM|third|night|| +15150|AAAAAAAAPCLDAAAA|15150|4|12|30|AM|third|night|| +15151|AAAAAAAAADLDAAAA|15151|4|12|31|AM|third|night|| +15152|AAAAAAAABDLDAAAA|15152|4|12|32|AM|third|night|| +15153|AAAAAAAACDLDAAAA|15153|4|12|33|AM|third|night|| +15154|AAAAAAAADDLDAAAA|15154|4|12|34|AM|third|night|| +15155|AAAAAAAAEDLDAAAA|15155|4|12|35|AM|third|night|| +15156|AAAAAAAAFDLDAAAA|15156|4|12|36|AM|third|night|| +15157|AAAAAAAAGDLDAAAA|15157|4|12|37|AM|third|night|| +15158|AAAAAAAAHDLDAAAA|15158|4|12|38|AM|third|night|| +15159|AAAAAAAAIDLDAAAA|15159|4|12|39|AM|third|night|| +15160|AAAAAAAAJDLDAAAA|15160|4|12|40|AM|third|night|| +15161|AAAAAAAAKDLDAAAA|15161|4|12|41|AM|third|night|| +15162|AAAAAAAALDLDAAAA|15162|4|12|42|AM|third|night|| +15163|AAAAAAAAMDLDAAAA|15163|4|12|43|AM|third|night|| +15164|AAAAAAAANDLDAAAA|15164|4|12|44|AM|third|night|| +15165|AAAAAAAAODLDAAAA|15165|4|12|45|AM|third|night|| +15166|AAAAAAAAPDLDAAAA|15166|4|12|46|AM|third|night|| +15167|AAAAAAAAAELDAAAA|15167|4|12|47|AM|third|night|| +15168|AAAAAAAABELDAAAA|15168|4|12|48|AM|third|night|| +15169|AAAAAAAACELDAAAA|15169|4|12|49|AM|third|night|| +15170|AAAAAAAADELDAAAA|15170|4|12|50|AM|third|night|| +15171|AAAAAAAAEELDAAAA|15171|4|12|51|AM|third|night|| +15172|AAAAAAAAFELDAAAA|15172|4|12|52|AM|third|night|| +15173|AAAAAAAAGELDAAAA|15173|4|12|53|AM|third|night|| +15174|AAAAAAAAHELDAAAA|15174|4|12|54|AM|third|night|| +15175|AAAAAAAAIELDAAAA|15175|4|12|55|AM|third|night|| +15176|AAAAAAAAJELDAAAA|15176|4|12|56|AM|third|night|| +15177|AAAAAAAAKELDAAAA|15177|4|12|57|AM|third|night|| +15178|AAAAAAAALELDAAAA|15178|4|12|58|AM|third|night|| +15179|AAAAAAAAMELDAAAA|15179|4|12|59|AM|third|night|| +15180|AAAAAAAANELDAAAA|15180|4|13|0|AM|third|night|| +15181|AAAAAAAAOELDAAAA|15181|4|13|1|AM|third|night|| +15182|AAAAAAAAPELDAAAA|15182|4|13|2|AM|third|night|| +15183|AAAAAAAAAFLDAAAA|15183|4|13|3|AM|third|night|| +15184|AAAAAAAABFLDAAAA|15184|4|13|4|AM|third|night|| +15185|AAAAAAAACFLDAAAA|15185|4|13|5|AM|third|night|| +15186|AAAAAAAADFLDAAAA|15186|4|13|6|AM|third|night|| +15187|AAAAAAAAEFLDAAAA|15187|4|13|7|AM|third|night|| +15188|AAAAAAAAFFLDAAAA|15188|4|13|8|AM|third|night|| +15189|AAAAAAAAGFLDAAAA|15189|4|13|9|AM|third|night|| +15190|AAAAAAAAHFLDAAAA|15190|4|13|10|AM|third|night|| +15191|AAAAAAAAIFLDAAAA|15191|4|13|11|AM|third|night|| +15192|AAAAAAAAJFLDAAAA|15192|4|13|12|AM|third|night|| +15193|AAAAAAAAKFLDAAAA|15193|4|13|13|AM|third|night|| +15194|AAAAAAAALFLDAAAA|15194|4|13|14|AM|third|night|| +15195|AAAAAAAAMFLDAAAA|15195|4|13|15|AM|third|night|| +15196|AAAAAAAANFLDAAAA|15196|4|13|16|AM|third|night|| +15197|AAAAAAAAOFLDAAAA|15197|4|13|17|AM|third|night|| +15198|AAAAAAAAPFLDAAAA|15198|4|13|18|AM|third|night|| +15199|AAAAAAAAAGLDAAAA|15199|4|13|19|AM|third|night|| +15200|AAAAAAAABGLDAAAA|15200|4|13|20|AM|third|night|| +15201|AAAAAAAACGLDAAAA|15201|4|13|21|AM|third|night|| +15202|AAAAAAAADGLDAAAA|15202|4|13|22|AM|third|night|| +15203|AAAAAAAAEGLDAAAA|15203|4|13|23|AM|third|night|| +15204|AAAAAAAAFGLDAAAA|15204|4|13|24|AM|third|night|| +15205|AAAAAAAAGGLDAAAA|15205|4|13|25|AM|third|night|| +15206|AAAAAAAAHGLDAAAA|15206|4|13|26|AM|third|night|| +15207|AAAAAAAAIGLDAAAA|15207|4|13|27|AM|third|night|| +15208|AAAAAAAAJGLDAAAA|15208|4|13|28|AM|third|night|| +15209|AAAAAAAAKGLDAAAA|15209|4|13|29|AM|third|night|| +15210|AAAAAAAALGLDAAAA|15210|4|13|30|AM|third|night|| +15211|AAAAAAAAMGLDAAAA|15211|4|13|31|AM|third|night|| +15212|AAAAAAAANGLDAAAA|15212|4|13|32|AM|third|night|| +15213|AAAAAAAAOGLDAAAA|15213|4|13|33|AM|third|night|| +15214|AAAAAAAAPGLDAAAA|15214|4|13|34|AM|third|night|| +15215|AAAAAAAAAHLDAAAA|15215|4|13|35|AM|third|night|| +15216|AAAAAAAABHLDAAAA|15216|4|13|36|AM|third|night|| +15217|AAAAAAAACHLDAAAA|15217|4|13|37|AM|third|night|| +15218|AAAAAAAADHLDAAAA|15218|4|13|38|AM|third|night|| +15219|AAAAAAAAEHLDAAAA|15219|4|13|39|AM|third|night|| +15220|AAAAAAAAFHLDAAAA|15220|4|13|40|AM|third|night|| +15221|AAAAAAAAGHLDAAAA|15221|4|13|41|AM|third|night|| +15222|AAAAAAAAHHLDAAAA|15222|4|13|42|AM|third|night|| +15223|AAAAAAAAIHLDAAAA|15223|4|13|43|AM|third|night|| +15224|AAAAAAAAJHLDAAAA|15224|4|13|44|AM|third|night|| +15225|AAAAAAAAKHLDAAAA|15225|4|13|45|AM|third|night|| +15226|AAAAAAAALHLDAAAA|15226|4|13|46|AM|third|night|| +15227|AAAAAAAAMHLDAAAA|15227|4|13|47|AM|third|night|| +15228|AAAAAAAANHLDAAAA|15228|4|13|48|AM|third|night|| +15229|AAAAAAAAOHLDAAAA|15229|4|13|49|AM|third|night|| +15230|AAAAAAAAPHLDAAAA|15230|4|13|50|AM|third|night|| +15231|AAAAAAAAAILDAAAA|15231|4|13|51|AM|third|night|| +15232|AAAAAAAABILDAAAA|15232|4|13|52|AM|third|night|| +15233|AAAAAAAACILDAAAA|15233|4|13|53|AM|third|night|| +15234|AAAAAAAADILDAAAA|15234|4|13|54|AM|third|night|| +15235|AAAAAAAAEILDAAAA|15235|4|13|55|AM|third|night|| +15236|AAAAAAAAFILDAAAA|15236|4|13|56|AM|third|night|| +15237|AAAAAAAAGILDAAAA|15237|4|13|57|AM|third|night|| +15238|AAAAAAAAHILDAAAA|15238|4|13|58|AM|third|night|| +15239|AAAAAAAAIILDAAAA|15239|4|13|59|AM|third|night|| +15240|AAAAAAAAJILDAAAA|15240|4|14|0|AM|third|night|| +15241|AAAAAAAAKILDAAAA|15241|4|14|1|AM|third|night|| +15242|AAAAAAAALILDAAAA|15242|4|14|2|AM|third|night|| +15243|AAAAAAAAMILDAAAA|15243|4|14|3|AM|third|night|| +15244|AAAAAAAANILDAAAA|15244|4|14|4|AM|third|night|| +15245|AAAAAAAAOILDAAAA|15245|4|14|5|AM|third|night|| +15246|AAAAAAAAPILDAAAA|15246|4|14|6|AM|third|night|| +15247|AAAAAAAAAJLDAAAA|15247|4|14|7|AM|third|night|| +15248|AAAAAAAABJLDAAAA|15248|4|14|8|AM|third|night|| +15249|AAAAAAAACJLDAAAA|15249|4|14|9|AM|third|night|| +15250|AAAAAAAADJLDAAAA|15250|4|14|10|AM|third|night|| +15251|AAAAAAAAEJLDAAAA|15251|4|14|11|AM|third|night|| +15252|AAAAAAAAFJLDAAAA|15252|4|14|12|AM|third|night|| +15253|AAAAAAAAGJLDAAAA|15253|4|14|13|AM|third|night|| +15254|AAAAAAAAHJLDAAAA|15254|4|14|14|AM|third|night|| +15255|AAAAAAAAIJLDAAAA|15255|4|14|15|AM|third|night|| +15256|AAAAAAAAJJLDAAAA|15256|4|14|16|AM|third|night|| +15257|AAAAAAAAKJLDAAAA|15257|4|14|17|AM|third|night|| +15258|AAAAAAAALJLDAAAA|15258|4|14|18|AM|third|night|| +15259|AAAAAAAAMJLDAAAA|15259|4|14|19|AM|third|night|| +15260|AAAAAAAANJLDAAAA|15260|4|14|20|AM|third|night|| +15261|AAAAAAAAOJLDAAAA|15261|4|14|21|AM|third|night|| +15262|AAAAAAAAPJLDAAAA|15262|4|14|22|AM|third|night|| +15263|AAAAAAAAAKLDAAAA|15263|4|14|23|AM|third|night|| +15264|AAAAAAAABKLDAAAA|15264|4|14|24|AM|third|night|| +15265|AAAAAAAACKLDAAAA|15265|4|14|25|AM|third|night|| +15266|AAAAAAAADKLDAAAA|15266|4|14|26|AM|third|night|| +15267|AAAAAAAAEKLDAAAA|15267|4|14|27|AM|third|night|| +15268|AAAAAAAAFKLDAAAA|15268|4|14|28|AM|third|night|| +15269|AAAAAAAAGKLDAAAA|15269|4|14|29|AM|third|night|| +15270|AAAAAAAAHKLDAAAA|15270|4|14|30|AM|third|night|| +15271|AAAAAAAAIKLDAAAA|15271|4|14|31|AM|third|night|| +15272|AAAAAAAAJKLDAAAA|15272|4|14|32|AM|third|night|| +15273|AAAAAAAAKKLDAAAA|15273|4|14|33|AM|third|night|| +15274|AAAAAAAALKLDAAAA|15274|4|14|34|AM|third|night|| +15275|AAAAAAAAMKLDAAAA|15275|4|14|35|AM|third|night|| +15276|AAAAAAAANKLDAAAA|15276|4|14|36|AM|third|night|| +15277|AAAAAAAAOKLDAAAA|15277|4|14|37|AM|third|night|| +15278|AAAAAAAAPKLDAAAA|15278|4|14|38|AM|third|night|| +15279|AAAAAAAAALLDAAAA|15279|4|14|39|AM|third|night|| +15280|AAAAAAAABLLDAAAA|15280|4|14|40|AM|third|night|| +15281|AAAAAAAACLLDAAAA|15281|4|14|41|AM|third|night|| +15282|AAAAAAAADLLDAAAA|15282|4|14|42|AM|third|night|| +15283|AAAAAAAAELLDAAAA|15283|4|14|43|AM|third|night|| +15284|AAAAAAAAFLLDAAAA|15284|4|14|44|AM|third|night|| +15285|AAAAAAAAGLLDAAAA|15285|4|14|45|AM|third|night|| +15286|AAAAAAAAHLLDAAAA|15286|4|14|46|AM|third|night|| +15287|AAAAAAAAILLDAAAA|15287|4|14|47|AM|third|night|| +15288|AAAAAAAAJLLDAAAA|15288|4|14|48|AM|third|night|| +15289|AAAAAAAAKLLDAAAA|15289|4|14|49|AM|third|night|| +15290|AAAAAAAALLLDAAAA|15290|4|14|50|AM|third|night|| +15291|AAAAAAAAMLLDAAAA|15291|4|14|51|AM|third|night|| +15292|AAAAAAAANLLDAAAA|15292|4|14|52|AM|third|night|| +15293|AAAAAAAAOLLDAAAA|15293|4|14|53|AM|third|night|| +15294|AAAAAAAAPLLDAAAA|15294|4|14|54|AM|third|night|| +15295|AAAAAAAAAMLDAAAA|15295|4|14|55|AM|third|night|| +15296|AAAAAAAABMLDAAAA|15296|4|14|56|AM|third|night|| +15297|AAAAAAAACMLDAAAA|15297|4|14|57|AM|third|night|| +15298|AAAAAAAADMLDAAAA|15298|4|14|58|AM|third|night|| +15299|AAAAAAAAEMLDAAAA|15299|4|14|59|AM|third|night|| +15300|AAAAAAAAFMLDAAAA|15300|4|15|0|AM|third|night|| +15301|AAAAAAAAGMLDAAAA|15301|4|15|1|AM|third|night|| +15302|AAAAAAAAHMLDAAAA|15302|4|15|2|AM|third|night|| +15303|AAAAAAAAIMLDAAAA|15303|4|15|3|AM|third|night|| +15304|AAAAAAAAJMLDAAAA|15304|4|15|4|AM|third|night|| +15305|AAAAAAAAKMLDAAAA|15305|4|15|5|AM|third|night|| +15306|AAAAAAAALMLDAAAA|15306|4|15|6|AM|third|night|| +15307|AAAAAAAAMMLDAAAA|15307|4|15|7|AM|third|night|| +15308|AAAAAAAANMLDAAAA|15308|4|15|8|AM|third|night|| +15309|AAAAAAAAOMLDAAAA|15309|4|15|9|AM|third|night|| +15310|AAAAAAAAPMLDAAAA|15310|4|15|10|AM|third|night|| +15311|AAAAAAAAANLDAAAA|15311|4|15|11|AM|third|night|| +15312|AAAAAAAABNLDAAAA|15312|4|15|12|AM|third|night|| +15313|AAAAAAAACNLDAAAA|15313|4|15|13|AM|third|night|| +15314|AAAAAAAADNLDAAAA|15314|4|15|14|AM|third|night|| +15315|AAAAAAAAENLDAAAA|15315|4|15|15|AM|third|night|| +15316|AAAAAAAAFNLDAAAA|15316|4|15|16|AM|third|night|| +15317|AAAAAAAAGNLDAAAA|15317|4|15|17|AM|third|night|| +15318|AAAAAAAAHNLDAAAA|15318|4|15|18|AM|third|night|| +15319|AAAAAAAAINLDAAAA|15319|4|15|19|AM|third|night|| +15320|AAAAAAAAJNLDAAAA|15320|4|15|20|AM|third|night|| +15321|AAAAAAAAKNLDAAAA|15321|4|15|21|AM|third|night|| +15322|AAAAAAAALNLDAAAA|15322|4|15|22|AM|third|night|| +15323|AAAAAAAAMNLDAAAA|15323|4|15|23|AM|third|night|| +15324|AAAAAAAANNLDAAAA|15324|4|15|24|AM|third|night|| +15325|AAAAAAAAONLDAAAA|15325|4|15|25|AM|third|night|| +15326|AAAAAAAAPNLDAAAA|15326|4|15|26|AM|third|night|| +15327|AAAAAAAAAOLDAAAA|15327|4|15|27|AM|third|night|| +15328|AAAAAAAABOLDAAAA|15328|4|15|28|AM|third|night|| +15329|AAAAAAAACOLDAAAA|15329|4|15|29|AM|third|night|| +15330|AAAAAAAADOLDAAAA|15330|4|15|30|AM|third|night|| +15331|AAAAAAAAEOLDAAAA|15331|4|15|31|AM|third|night|| +15332|AAAAAAAAFOLDAAAA|15332|4|15|32|AM|third|night|| +15333|AAAAAAAAGOLDAAAA|15333|4|15|33|AM|third|night|| +15334|AAAAAAAAHOLDAAAA|15334|4|15|34|AM|third|night|| +15335|AAAAAAAAIOLDAAAA|15335|4|15|35|AM|third|night|| +15336|AAAAAAAAJOLDAAAA|15336|4|15|36|AM|third|night|| +15337|AAAAAAAAKOLDAAAA|15337|4|15|37|AM|third|night|| +15338|AAAAAAAALOLDAAAA|15338|4|15|38|AM|third|night|| +15339|AAAAAAAAMOLDAAAA|15339|4|15|39|AM|third|night|| +15340|AAAAAAAANOLDAAAA|15340|4|15|40|AM|third|night|| +15341|AAAAAAAAOOLDAAAA|15341|4|15|41|AM|third|night|| +15342|AAAAAAAAPOLDAAAA|15342|4|15|42|AM|third|night|| +15343|AAAAAAAAAPLDAAAA|15343|4|15|43|AM|third|night|| +15344|AAAAAAAABPLDAAAA|15344|4|15|44|AM|third|night|| +15345|AAAAAAAACPLDAAAA|15345|4|15|45|AM|third|night|| +15346|AAAAAAAADPLDAAAA|15346|4|15|46|AM|third|night|| +15347|AAAAAAAAEPLDAAAA|15347|4|15|47|AM|third|night|| +15348|AAAAAAAAFPLDAAAA|15348|4|15|48|AM|third|night|| +15349|AAAAAAAAGPLDAAAA|15349|4|15|49|AM|third|night|| +15350|AAAAAAAAHPLDAAAA|15350|4|15|50|AM|third|night|| +15351|AAAAAAAAIPLDAAAA|15351|4|15|51|AM|third|night|| +15352|AAAAAAAAJPLDAAAA|15352|4|15|52|AM|third|night|| +15353|AAAAAAAAKPLDAAAA|15353|4|15|53|AM|third|night|| +15354|AAAAAAAALPLDAAAA|15354|4|15|54|AM|third|night|| +15355|AAAAAAAAMPLDAAAA|15355|4|15|55|AM|third|night|| +15356|AAAAAAAANPLDAAAA|15356|4|15|56|AM|third|night|| +15357|AAAAAAAAOPLDAAAA|15357|4|15|57|AM|third|night|| +15358|AAAAAAAAPPLDAAAA|15358|4|15|58|AM|third|night|| +15359|AAAAAAAAAAMDAAAA|15359|4|15|59|AM|third|night|| +15360|AAAAAAAABAMDAAAA|15360|4|16|0|AM|third|night|| +15361|AAAAAAAACAMDAAAA|15361|4|16|1|AM|third|night|| +15362|AAAAAAAADAMDAAAA|15362|4|16|2|AM|third|night|| +15363|AAAAAAAAEAMDAAAA|15363|4|16|3|AM|third|night|| +15364|AAAAAAAAFAMDAAAA|15364|4|16|4|AM|third|night|| +15365|AAAAAAAAGAMDAAAA|15365|4|16|5|AM|third|night|| +15366|AAAAAAAAHAMDAAAA|15366|4|16|6|AM|third|night|| +15367|AAAAAAAAIAMDAAAA|15367|4|16|7|AM|third|night|| +15368|AAAAAAAAJAMDAAAA|15368|4|16|8|AM|third|night|| +15369|AAAAAAAAKAMDAAAA|15369|4|16|9|AM|third|night|| +15370|AAAAAAAALAMDAAAA|15370|4|16|10|AM|third|night|| +15371|AAAAAAAAMAMDAAAA|15371|4|16|11|AM|third|night|| +15372|AAAAAAAANAMDAAAA|15372|4|16|12|AM|third|night|| +15373|AAAAAAAAOAMDAAAA|15373|4|16|13|AM|third|night|| +15374|AAAAAAAAPAMDAAAA|15374|4|16|14|AM|third|night|| +15375|AAAAAAAAABMDAAAA|15375|4|16|15|AM|third|night|| +15376|AAAAAAAABBMDAAAA|15376|4|16|16|AM|third|night|| +15377|AAAAAAAACBMDAAAA|15377|4|16|17|AM|third|night|| +15378|AAAAAAAADBMDAAAA|15378|4|16|18|AM|third|night|| +15379|AAAAAAAAEBMDAAAA|15379|4|16|19|AM|third|night|| +15380|AAAAAAAAFBMDAAAA|15380|4|16|20|AM|third|night|| +15381|AAAAAAAAGBMDAAAA|15381|4|16|21|AM|third|night|| +15382|AAAAAAAAHBMDAAAA|15382|4|16|22|AM|third|night|| +15383|AAAAAAAAIBMDAAAA|15383|4|16|23|AM|third|night|| +15384|AAAAAAAAJBMDAAAA|15384|4|16|24|AM|third|night|| +15385|AAAAAAAAKBMDAAAA|15385|4|16|25|AM|third|night|| +15386|AAAAAAAALBMDAAAA|15386|4|16|26|AM|third|night|| +15387|AAAAAAAAMBMDAAAA|15387|4|16|27|AM|third|night|| +15388|AAAAAAAANBMDAAAA|15388|4|16|28|AM|third|night|| +15389|AAAAAAAAOBMDAAAA|15389|4|16|29|AM|third|night|| +15390|AAAAAAAAPBMDAAAA|15390|4|16|30|AM|third|night|| +15391|AAAAAAAAACMDAAAA|15391|4|16|31|AM|third|night|| +15392|AAAAAAAABCMDAAAA|15392|4|16|32|AM|third|night|| +15393|AAAAAAAACCMDAAAA|15393|4|16|33|AM|third|night|| +15394|AAAAAAAADCMDAAAA|15394|4|16|34|AM|third|night|| +15395|AAAAAAAAECMDAAAA|15395|4|16|35|AM|third|night|| +15396|AAAAAAAAFCMDAAAA|15396|4|16|36|AM|third|night|| +15397|AAAAAAAAGCMDAAAA|15397|4|16|37|AM|third|night|| +15398|AAAAAAAAHCMDAAAA|15398|4|16|38|AM|third|night|| +15399|AAAAAAAAICMDAAAA|15399|4|16|39|AM|third|night|| +15400|AAAAAAAAJCMDAAAA|15400|4|16|40|AM|third|night|| +15401|AAAAAAAAKCMDAAAA|15401|4|16|41|AM|third|night|| +15402|AAAAAAAALCMDAAAA|15402|4|16|42|AM|third|night|| +15403|AAAAAAAAMCMDAAAA|15403|4|16|43|AM|third|night|| +15404|AAAAAAAANCMDAAAA|15404|4|16|44|AM|third|night|| +15405|AAAAAAAAOCMDAAAA|15405|4|16|45|AM|third|night|| +15406|AAAAAAAAPCMDAAAA|15406|4|16|46|AM|third|night|| +15407|AAAAAAAAADMDAAAA|15407|4|16|47|AM|third|night|| +15408|AAAAAAAABDMDAAAA|15408|4|16|48|AM|third|night|| +15409|AAAAAAAACDMDAAAA|15409|4|16|49|AM|third|night|| +15410|AAAAAAAADDMDAAAA|15410|4|16|50|AM|third|night|| +15411|AAAAAAAAEDMDAAAA|15411|4|16|51|AM|third|night|| +15412|AAAAAAAAFDMDAAAA|15412|4|16|52|AM|third|night|| +15413|AAAAAAAAGDMDAAAA|15413|4|16|53|AM|third|night|| +15414|AAAAAAAAHDMDAAAA|15414|4|16|54|AM|third|night|| +15415|AAAAAAAAIDMDAAAA|15415|4|16|55|AM|third|night|| +15416|AAAAAAAAJDMDAAAA|15416|4|16|56|AM|third|night|| +15417|AAAAAAAAKDMDAAAA|15417|4|16|57|AM|third|night|| +15418|AAAAAAAALDMDAAAA|15418|4|16|58|AM|third|night|| +15419|AAAAAAAAMDMDAAAA|15419|4|16|59|AM|third|night|| +15420|AAAAAAAANDMDAAAA|15420|4|17|0|AM|third|night|| +15421|AAAAAAAAODMDAAAA|15421|4|17|1|AM|third|night|| +15422|AAAAAAAAPDMDAAAA|15422|4|17|2|AM|third|night|| +15423|AAAAAAAAAEMDAAAA|15423|4|17|3|AM|third|night|| +15424|AAAAAAAABEMDAAAA|15424|4|17|4|AM|third|night|| +15425|AAAAAAAACEMDAAAA|15425|4|17|5|AM|third|night|| +15426|AAAAAAAADEMDAAAA|15426|4|17|6|AM|third|night|| +15427|AAAAAAAAEEMDAAAA|15427|4|17|7|AM|third|night|| +15428|AAAAAAAAFEMDAAAA|15428|4|17|8|AM|third|night|| +15429|AAAAAAAAGEMDAAAA|15429|4|17|9|AM|third|night|| +15430|AAAAAAAAHEMDAAAA|15430|4|17|10|AM|third|night|| +15431|AAAAAAAAIEMDAAAA|15431|4|17|11|AM|third|night|| +15432|AAAAAAAAJEMDAAAA|15432|4|17|12|AM|third|night|| +15433|AAAAAAAAKEMDAAAA|15433|4|17|13|AM|third|night|| +15434|AAAAAAAALEMDAAAA|15434|4|17|14|AM|third|night|| +15435|AAAAAAAAMEMDAAAA|15435|4|17|15|AM|third|night|| +15436|AAAAAAAANEMDAAAA|15436|4|17|16|AM|third|night|| +15437|AAAAAAAAOEMDAAAA|15437|4|17|17|AM|third|night|| +15438|AAAAAAAAPEMDAAAA|15438|4|17|18|AM|third|night|| +15439|AAAAAAAAAFMDAAAA|15439|4|17|19|AM|third|night|| +15440|AAAAAAAABFMDAAAA|15440|4|17|20|AM|third|night|| +15441|AAAAAAAACFMDAAAA|15441|4|17|21|AM|third|night|| +15442|AAAAAAAADFMDAAAA|15442|4|17|22|AM|third|night|| +15443|AAAAAAAAEFMDAAAA|15443|4|17|23|AM|third|night|| +15444|AAAAAAAAFFMDAAAA|15444|4|17|24|AM|third|night|| +15445|AAAAAAAAGFMDAAAA|15445|4|17|25|AM|third|night|| +15446|AAAAAAAAHFMDAAAA|15446|4|17|26|AM|third|night|| +15447|AAAAAAAAIFMDAAAA|15447|4|17|27|AM|third|night|| +15448|AAAAAAAAJFMDAAAA|15448|4|17|28|AM|third|night|| +15449|AAAAAAAAKFMDAAAA|15449|4|17|29|AM|third|night|| +15450|AAAAAAAALFMDAAAA|15450|4|17|30|AM|third|night|| +15451|AAAAAAAAMFMDAAAA|15451|4|17|31|AM|third|night|| +15452|AAAAAAAANFMDAAAA|15452|4|17|32|AM|third|night|| +15453|AAAAAAAAOFMDAAAA|15453|4|17|33|AM|third|night|| +15454|AAAAAAAAPFMDAAAA|15454|4|17|34|AM|third|night|| +15455|AAAAAAAAAGMDAAAA|15455|4|17|35|AM|third|night|| +15456|AAAAAAAABGMDAAAA|15456|4|17|36|AM|third|night|| +15457|AAAAAAAACGMDAAAA|15457|4|17|37|AM|third|night|| +15458|AAAAAAAADGMDAAAA|15458|4|17|38|AM|third|night|| +15459|AAAAAAAAEGMDAAAA|15459|4|17|39|AM|third|night|| +15460|AAAAAAAAFGMDAAAA|15460|4|17|40|AM|third|night|| +15461|AAAAAAAAGGMDAAAA|15461|4|17|41|AM|third|night|| +15462|AAAAAAAAHGMDAAAA|15462|4|17|42|AM|third|night|| +15463|AAAAAAAAIGMDAAAA|15463|4|17|43|AM|third|night|| +15464|AAAAAAAAJGMDAAAA|15464|4|17|44|AM|third|night|| +15465|AAAAAAAAKGMDAAAA|15465|4|17|45|AM|third|night|| +15466|AAAAAAAALGMDAAAA|15466|4|17|46|AM|third|night|| +15467|AAAAAAAAMGMDAAAA|15467|4|17|47|AM|third|night|| +15468|AAAAAAAANGMDAAAA|15468|4|17|48|AM|third|night|| +15469|AAAAAAAAOGMDAAAA|15469|4|17|49|AM|third|night|| +15470|AAAAAAAAPGMDAAAA|15470|4|17|50|AM|third|night|| +15471|AAAAAAAAAHMDAAAA|15471|4|17|51|AM|third|night|| +15472|AAAAAAAABHMDAAAA|15472|4|17|52|AM|third|night|| +15473|AAAAAAAACHMDAAAA|15473|4|17|53|AM|third|night|| +15474|AAAAAAAADHMDAAAA|15474|4|17|54|AM|third|night|| +15475|AAAAAAAAEHMDAAAA|15475|4|17|55|AM|third|night|| +15476|AAAAAAAAFHMDAAAA|15476|4|17|56|AM|third|night|| +15477|AAAAAAAAGHMDAAAA|15477|4|17|57|AM|third|night|| +15478|AAAAAAAAHHMDAAAA|15478|4|17|58|AM|third|night|| +15479|AAAAAAAAIHMDAAAA|15479|4|17|59|AM|third|night|| +15480|AAAAAAAAJHMDAAAA|15480|4|18|0|AM|third|night|| +15481|AAAAAAAAKHMDAAAA|15481|4|18|1|AM|third|night|| +15482|AAAAAAAALHMDAAAA|15482|4|18|2|AM|third|night|| +15483|AAAAAAAAMHMDAAAA|15483|4|18|3|AM|third|night|| +15484|AAAAAAAANHMDAAAA|15484|4|18|4|AM|third|night|| +15485|AAAAAAAAOHMDAAAA|15485|4|18|5|AM|third|night|| +15486|AAAAAAAAPHMDAAAA|15486|4|18|6|AM|third|night|| +15487|AAAAAAAAAIMDAAAA|15487|4|18|7|AM|third|night|| +15488|AAAAAAAABIMDAAAA|15488|4|18|8|AM|third|night|| +15489|AAAAAAAACIMDAAAA|15489|4|18|9|AM|third|night|| +15490|AAAAAAAADIMDAAAA|15490|4|18|10|AM|third|night|| +15491|AAAAAAAAEIMDAAAA|15491|4|18|11|AM|third|night|| +15492|AAAAAAAAFIMDAAAA|15492|4|18|12|AM|third|night|| +15493|AAAAAAAAGIMDAAAA|15493|4|18|13|AM|third|night|| +15494|AAAAAAAAHIMDAAAA|15494|4|18|14|AM|third|night|| +15495|AAAAAAAAIIMDAAAA|15495|4|18|15|AM|third|night|| +15496|AAAAAAAAJIMDAAAA|15496|4|18|16|AM|third|night|| +15497|AAAAAAAAKIMDAAAA|15497|4|18|17|AM|third|night|| +15498|AAAAAAAALIMDAAAA|15498|4|18|18|AM|third|night|| +15499|AAAAAAAAMIMDAAAA|15499|4|18|19|AM|third|night|| +15500|AAAAAAAANIMDAAAA|15500|4|18|20|AM|third|night|| +15501|AAAAAAAAOIMDAAAA|15501|4|18|21|AM|third|night|| +15502|AAAAAAAAPIMDAAAA|15502|4|18|22|AM|third|night|| +15503|AAAAAAAAAJMDAAAA|15503|4|18|23|AM|third|night|| +15504|AAAAAAAABJMDAAAA|15504|4|18|24|AM|third|night|| +15505|AAAAAAAACJMDAAAA|15505|4|18|25|AM|third|night|| +15506|AAAAAAAADJMDAAAA|15506|4|18|26|AM|third|night|| +15507|AAAAAAAAEJMDAAAA|15507|4|18|27|AM|third|night|| +15508|AAAAAAAAFJMDAAAA|15508|4|18|28|AM|third|night|| +15509|AAAAAAAAGJMDAAAA|15509|4|18|29|AM|third|night|| +15510|AAAAAAAAHJMDAAAA|15510|4|18|30|AM|third|night|| +15511|AAAAAAAAIJMDAAAA|15511|4|18|31|AM|third|night|| +15512|AAAAAAAAJJMDAAAA|15512|4|18|32|AM|third|night|| +15513|AAAAAAAAKJMDAAAA|15513|4|18|33|AM|third|night|| +15514|AAAAAAAALJMDAAAA|15514|4|18|34|AM|third|night|| +15515|AAAAAAAAMJMDAAAA|15515|4|18|35|AM|third|night|| +15516|AAAAAAAANJMDAAAA|15516|4|18|36|AM|third|night|| +15517|AAAAAAAAOJMDAAAA|15517|4|18|37|AM|third|night|| +15518|AAAAAAAAPJMDAAAA|15518|4|18|38|AM|third|night|| +15519|AAAAAAAAAKMDAAAA|15519|4|18|39|AM|third|night|| +15520|AAAAAAAABKMDAAAA|15520|4|18|40|AM|third|night|| +15521|AAAAAAAACKMDAAAA|15521|4|18|41|AM|third|night|| +15522|AAAAAAAADKMDAAAA|15522|4|18|42|AM|third|night|| +15523|AAAAAAAAEKMDAAAA|15523|4|18|43|AM|third|night|| +15524|AAAAAAAAFKMDAAAA|15524|4|18|44|AM|third|night|| +15525|AAAAAAAAGKMDAAAA|15525|4|18|45|AM|third|night|| +15526|AAAAAAAAHKMDAAAA|15526|4|18|46|AM|third|night|| +15527|AAAAAAAAIKMDAAAA|15527|4|18|47|AM|third|night|| +15528|AAAAAAAAJKMDAAAA|15528|4|18|48|AM|third|night|| +15529|AAAAAAAAKKMDAAAA|15529|4|18|49|AM|third|night|| +15530|AAAAAAAALKMDAAAA|15530|4|18|50|AM|third|night|| +15531|AAAAAAAAMKMDAAAA|15531|4|18|51|AM|third|night|| +15532|AAAAAAAANKMDAAAA|15532|4|18|52|AM|third|night|| +15533|AAAAAAAAOKMDAAAA|15533|4|18|53|AM|third|night|| +15534|AAAAAAAAPKMDAAAA|15534|4|18|54|AM|third|night|| +15535|AAAAAAAAALMDAAAA|15535|4|18|55|AM|third|night|| +15536|AAAAAAAABLMDAAAA|15536|4|18|56|AM|third|night|| +15537|AAAAAAAACLMDAAAA|15537|4|18|57|AM|third|night|| +15538|AAAAAAAADLMDAAAA|15538|4|18|58|AM|third|night|| +15539|AAAAAAAAELMDAAAA|15539|4|18|59|AM|third|night|| +15540|AAAAAAAAFLMDAAAA|15540|4|19|0|AM|third|night|| +15541|AAAAAAAAGLMDAAAA|15541|4|19|1|AM|third|night|| +15542|AAAAAAAAHLMDAAAA|15542|4|19|2|AM|third|night|| +15543|AAAAAAAAILMDAAAA|15543|4|19|3|AM|third|night|| +15544|AAAAAAAAJLMDAAAA|15544|4|19|4|AM|third|night|| +15545|AAAAAAAAKLMDAAAA|15545|4|19|5|AM|third|night|| +15546|AAAAAAAALLMDAAAA|15546|4|19|6|AM|third|night|| +15547|AAAAAAAAMLMDAAAA|15547|4|19|7|AM|third|night|| +15548|AAAAAAAANLMDAAAA|15548|4|19|8|AM|third|night|| +15549|AAAAAAAAOLMDAAAA|15549|4|19|9|AM|third|night|| +15550|AAAAAAAAPLMDAAAA|15550|4|19|10|AM|third|night|| +15551|AAAAAAAAAMMDAAAA|15551|4|19|11|AM|third|night|| +15552|AAAAAAAABMMDAAAA|15552|4|19|12|AM|third|night|| +15553|AAAAAAAACMMDAAAA|15553|4|19|13|AM|third|night|| +15554|AAAAAAAADMMDAAAA|15554|4|19|14|AM|third|night|| +15555|AAAAAAAAEMMDAAAA|15555|4|19|15|AM|third|night|| +15556|AAAAAAAAFMMDAAAA|15556|4|19|16|AM|third|night|| +15557|AAAAAAAAGMMDAAAA|15557|4|19|17|AM|third|night|| +15558|AAAAAAAAHMMDAAAA|15558|4|19|18|AM|third|night|| +15559|AAAAAAAAIMMDAAAA|15559|4|19|19|AM|third|night|| +15560|AAAAAAAAJMMDAAAA|15560|4|19|20|AM|third|night|| +15561|AAAAAAAAKMMDAAAA|15561|4|19|21|AM|third|night|| +15562|AAAAAAAALMMDAAAA|15562|4|19|22|AM|third|night|| +15563|AAAAAAAAMMMDAAAA|15563|4|19|23|AM|third|night|| +15564|AAAAAAAANMMDAAAA|15564|4|19|24|AM|third|night|| +15565|AAAAAAAAOMMDAAAA|15565|4|19|25|AM|third|night|| +15566|AAAAAAAAPMMDAAAA|15566|4|19|26|AM|third|night|| +15567|AAAAAAAAANMDAAAA|15567|4|19|27|AM|third|night|| +15568|AAAAAAAABNMDAAAA|15568|4|19|28|AM|third|night|| +15569|AAAAAAAACNMDAAAA|15569|4|19|29|AM|third|night|| +15570|AAAAAAAADNMDAAAA|15570|4|19|30|AM|third|night|| +15571|AAAAAAAAENMDAAAA|15571|4|19|31|AM|third|night|| +15572|AAAAAAAAFNMDAAAA|15572|4|19|32|AM|third|night|| +15573|AAAAAAAAGNMDAAAA|15573|4|19|33|AM|third|night|| +15574|AAAAAAAAHNMDAAAA|15574|4|19|34|AM|third|night|| +15575|AAAAAAAAINMDAAAA|15575|4|19|35|AM|third|night|| +15576|AAAAAAAAJNMDAAAA|15576|4|19|36|AM|third|night|| +15577|AAAAAAAAKNMDAAAA|15577|4|19|37|AM|third|night|| +15578|AAAAAAAALNMDAAAA|15578|4|19|38|AM|third|night|| +15579|AAAAAAAAMNMDAAAA|15579|4|19|39|AM|third|night|| +15580|AAAAAAAANNMDAAAA|15580|4|19|40|AM|third|night|| +15581|AAAAAAAAONMDAAAA|15581|4|19|41|AM|third|night|| +15582|AAAAAAAAPNMDAAAA|15582|4|19|42|AM|third|night|| +15583|AAAAAAAAAOMDAAAA|15583|4|19|43|AM|third|night|| +15584|AAAAAAAABOMDAAAA|15584|4|19|44|AM|third|night|| +15585|AAAAAAAACOMDAAAA|15585|4|19|45|AM|third|night|| +15586|AAAAAAAADOMDAAAA|15586|4|19|46|AM|third|night|| +15587|AAAAAAAAEOMDAAAA|15587|4|19|47|AM|third|night|| +15588|AAAAAAAAFOMDAAAA|15588|4|19|48|AM|third|night|| +15589|AAAAAAAAGOMDAAAA|15589|4|19|49|AM|third|night|| +15590|AAAAAAAAHOMDAAAA|15590|4|19|50|AM|third|night|| +15591|AAAAAAAAIOMDAAAA|15591|4|19|51|AM|third|night|| +15592|AAAAAAAAJOMDAAAA|15592|4|19|52|AM|third|night|| +15593|AAAAAAAAKOMDAAAA|15593|4|19|53|AM|third|night|| +15594|AAAAAAAALOMDAAAA|15594|4|19|54|AM|third|night|| +15595|AAAAAAAAMOMDAAAA|15595|4|19|55|AM|third|night|| +15596|AAAAAAAANOMDAAAA|15596|4|19|56|AM|third|night|| +15597|AAAAAAAAOOMDAAAA|15597|4|19|57|AM|third|night|| +15598|AAAAAAAAPOMDAAAA|15598|4|19|58|AM|third|night|| +15599|AAAAAAAAAPMDAAAA|15599|4|19|59|AM|third|night|| +15600|AAAAAAAABPMDAAAA|15600|4|20|0|AM|third|night|| +15601|AAAAAAAACPMDAAAA|15601|4|20|1|AM|third|night|| +15602|AAAAAAAADPMDAAAA|15602|4|20|2|AM|third|night|| +15603|AAAAAAAAEPMDAAAA|15603|4|20|3|AM|third|night|| +15604|AAAAAAAAFPMDAAAA|15604|4|20|4|AM|third|night|| +15605|AAAAAAAAGPMDAAAA|15605|4|20|5|AM|third|night|| +15606|AAAAAAAAHPMDAAAA|15606|4|20|6|AM|third|night|| +15607|AAAAAAAAIPMDAAAA|15607|4|20|7|AM|third|night|| +15608|AAAAAAAAJPMDAAAA|15608|4|20|8|AM|third|night|| +15609|AAAAAAAAKPMDAAAA|15609|4|20|9|AM|third|night|| +15610|AAAAAAAALPMDAAAA|15610|4|20|10|AM|third|night|| +15611|AAAAAAAAMPMDAAAA|15611|4|20|11|AM|third|night|| +15612|AAAAAAAANPMDAAAA|15612|4|20|12|AM|third|night|| +15613|AAAAAAAAOPMDAAAA|15613|4|20|13|AM|third|night|| +15614|AAAAAAAAPPMDAAAA|15614|4|20|14|AM|third|night|| +15615|AAAAAAAAAANDAAAA|15615|4|20|15|AM|third|night|| +15616|AAAAAAAABANDAAAA|15616|4|20|16|AM|third|night|| +15617|AAAAAAAACANDAAAA|15617|4|20|17|AM|third|night|| +15618|AAAAAAAADANDAAAA|15618|4|20|18|AM|third|night|| +15619|AAAAAAAAEANDAAAA|15619|4|20|19|AM|third|night|| +15620|AAAAAAAAFANDAAAA|15620|4|20|20|AM|third|night|| +15621|AAAAAAAAGANDAAAA|15621|4|20|21|AM|third|night|| +15622|AAAAAAAAHANDAAAA|15622|4|20|22|AM|third|night|| +15623|AAAAAAAAIANDAAAA|15623|4|20|23|AM|third|night|| +15624|AAAAAAAAJANDAAAA|15624|4|20|24|AM|third|night|| +15625|AAAAAAAAKANDAAAA|15625|4|20|25|AM|third|night|| +15626|AAAAAAAALANDAAAA|15626|4|20|26|AM|third|night|| +15627|AAAAAAAAMANDAAAA|15627|4|20|27|AM|third|night|| +15628|AAAAAAAANANDAAAA|15628|4|20|28|AM|third|night|| +15629|AAAAAAAAOANDAAAA|15629|4|20|29|AM|third|night|| +15630|AAAAAAAAPANDAAAA|15630|4|20|30|AM|third|night|| +15631|AAAAAAAAABNDAAAA|15631|4|20|31|AM|third|night|| +15632|AAAAAAAABBNDAAAA|15632|4|20|32|AM|third|night|| +15633|AAAAAAAACBNDAAAA|15633|4|20|33|AM|third|night|| +15634|AAAAAAAADBNDAAAA|15634|4|20|34|AM|third|night|| +15635|AAAAAAAAEBNDAAAA|15635|4|20|35|AM|third|night|| +15636|AAAAAAAAFBNDAAAA|15636|4|20|36|AM|third|night|| +15637|AAAAAAAAGBNDAAAA|15637|4|20|37|AM|third|night|| +15638|AAAAAAAAHBNDAAAA|15638|4|20|38|AM|third|night|| +15639|AAAAAAAAIBNDAAAA|15639|4|20|39|AM|third|night|| +15640|AAAAAAAAJBNDAAAA|15640|4|20|40|AM|third|night|| +15641|AAAAAAAAKBNDAAAA|15641|4|20|41|AM|third|night|| +15642|AAAAAAAALBNDAAAA|15642|4|20|42|AM|third|night|| +15643|AAAAAAAAMBNDAAAA|15643|4|20|43|AM|third|night|| +15644|AAAAAAAANBNDAAAA|15644|4|20|44|AM|third|night|| +15645|AAAAAAAAOBNDAAAA|15645|4|20|45|AM|third|night|| +15646|AAAAAAAAPBNDAAAA|15646|4|20|46|AM|third|night|| +15647|AAAAAAAAACNDAAAA|15647|4|20|47|AM|third|night|| +15648|AAAAAAAABCNDAAAA|15648|4|20|48|AM|third|night|| +15649|AAAAAAAACCNDAAAA|15649|4|20|49|AM|third|night|| +15650|AAAAAAAADCNDAAAA|15650|4|20|50|AM|third|night|| +15651|AAAAAAAAECNDAAAA|15651|4|20|51|AM|third|night|| +15652|AAAAAAAAFCNDAAAA|15652|4|20|52|AM|third|night|| +15653|AAAAAAAAGCNDAAAA|15653|4|20|53|AM|third|night|| +15654|AAAAAAAAHCNDAAAA|15654|4|20|54|AM|third|night|| +15655|AAAAAAAAICNDAAAA|15655|4|20|55|AM|third|night|| +15656|AAAAAAAAJCNDAAAA|15656|4|20|56|AM|third|night|| +15657|AAAAAAAAKCNDAAAA|15657|4|20|57|AM|third|night|| +15658|AAAAAAAALCNDAAAA|15658|4|20|58|AM|third|night|| +15659|AAAAAAAAMCNDAAAA|15659|4|20|59|AM|third|night|| +15660|AAAAAAAANCNDAAAA|15660|4|21|0|AM|third|night|| +15661|AAAAAAAAOCNDAAAA|15661|4|21|1|AM|third|night|| +15662|AAAAAAAAPCNDAAAA|15662|4|21|2|AM|third|night|| +15663|AAAAAAAAADNDAAAA|15663|4|21|3|AM|third|night|| +15664|AAAAAAAABDNDAAAA|15664|4|21|4|AM|third|night|| +15665|AAAAAAAACDNDAAAA|15665|4|21|5|AM|third|night|| +15666|AAAAAAAADDNDAAAA|15666|4|21|6|AM|third|night|| +15667|AAAAAAAAEDNDAAAA|15667|4|21|7|AM|third|night|| +15668|AAAAAAAAFDNDAAAA|15668|4|21|8|AM|third|night|| +15669|AAAAAAAAGDNDAAAA|15669|4|21|9|AM|third|night|| +15670|AAAAAAAAHDNDAAAA|15670|4|21|10|AM|third|night|| +15671|AAAAAAAAIDNDAAAA|15671|4|21|11|AM|third|night|| +15672|AAAAAAAAJDNDAAAA|15672|4|21|12|AM|third|night|| +15673|AAAAAAAAKDNDAAAA|15673|4|21|13|AM|third|night|| +15674|AAAAAAAALDNDAAAA|15674|4|21|14|AM|third|night|| +15675|AAAAAAAAMDNDAAAA|15675|4|21|15|AM|third|night|| +15676|AAAAAAAANDNDAAAA|15676|4|21|16|AM|third|night|| +15677|AAAAAAAAODNDAAAA|15677|4|21|17|AM|third|night|| +15678|AAAAAAAAPDNDAAAA|15678|4|21|18|AM|third|night|| +15679|AAAAAAAAAENDAAAA|15679|4|21|19|AM|third|night|| +15680|AAAAAAAABENDAAAA|15680|4|21|20|AM|third|night|| +15681|AAAAAAAACENDAAAA|15681|4|21|21|AM|third|night|| +15682|AAAAAAAADENDAAAA|15682|4|21|22|AM|third|night|| +15683|AAAAAAAAEENDAAAA|15683|4|21|23|AM|third|night|| +15684|AAAAAAAAFENDAAAA|15684|4|21|24|AM|third|night|| +15685|AAAAAAAAGENDAAAA|15685|4|21|25|AM|third|night|| +15686|AAAAAAAAHENDAAAA|15686|4|21|26|AM|third|night|| +15687|AAAAAAAAIENDAAAA|15687|4|21|27|AM|third|night|| +15688|AAAAAAAAJENDAAAA|15688|4|21|28|AM|third|night|| +15689|AAAAAAAAKENDAAAA|15689|4|21|29|AM|third|night|| +15690|AAAAAAAALENDAAAA|15690|4|21|30|AM|third|night|| +15691|AAAAAAAAMENDAAAA|15691|4|21|31|AM|third|night|| +15692|AAAAAAAANENDAAAA|15692|4|21|32|AM|third|night|| +15693|AAAAAAAAOENDAAAA|15693|4|21|33|AM|third|night|| +15694|AAAAAAAAPENDAAAA|15694|4|21|34|AM|third|night|| +15695|AAAAAAAAAFNDAAAA|15695|4|21|35|AM|third|night|| +15696|AAAAAAAABFNDAAAA|15696|4|21|36|AM|third|night|| +15697|AAAAAAAACFNDAAAA|15697|4|21|37|AM|third|night|| +15698|AAAAAAAADFNDAAAA|15698|4|21|38|AM|third|night|| +15699|AAAAAAAAEFNDAAAA|15699|4|21|39|AM|third|night|| +15700|AAAAAAAAFFNDAAAA|15700|4|21|40|AM|third|night|| +15701|AAAAAAAAGFNDAAAA|15701|4|21|41|AM|third|night|| +15702|AAAAAAAAHFNDAAAA|15702|4|21|42|AM|third|night|| +15703|AAAAAAAAIFNDAAAA|15703|4|21|43|AM|third|night|| +15704|AAAAAAAAJFNDAAAA|15704|4|21|44|AM|third|night|| +15705|AAAAAAAAKFNDAAAA|15705|4|21|45|AM|third|night|| +15706|AAAAAAAALFNDAAAA|15706|4|21|46|AM|third|night|| +15707|AAAAAAAAMFNDAAAA|15707|4|21|47|AM|third|night|| +15708|AAAAAAAANFNDAAAA|15708|4|21|48|AM|third|night|| +15709|AAAAAAAAOFNDAAAA|15709|4|21|49|AM|third|night|| +15710|AAAAAAAAPFNDAAAA|15710|4|21|50|AM|third|night|| +15711|AAAAAAAAAGNDAAAA|15711|4|21|51|AM|third|night|| +15712|AAAAAAAABGNDAAAA|15712|4|21|52|AM|third|night|| +15713|AAAAAAAACGNDAAAA|15713|4|21|53|AM|third|night|| +15714|AAAAAAAADGNDAAAA|15714|4|21|54|AM|third|night|| +15715|AAAAAAAAEGNDAAAA|15715|4|21|55|AM|third|night|| +15716|AAAAAAAAFGNDAAAA|15716|4|21|56|AM|third|night|| +15717|AAAAAAAAGGNDAAAA|15717|4|21|57|AM|third|night|| +15718|AAAAAAAAHGNDAAAA|15718|4|21|58|AM|third|night|| +15719|AAAAAAAAIGNDAAAA|15719|4|21|59|AM|third|night|| +15720|AAAAAAAAJGNDAAAA|15720|4|22|0|AM|third|night|| +15721|AAAAAAAAKGNDAAAA|15721|4|22|1|AM|third|night|| +15722|AAAAAAAALGNDAAAA|15722|4|22|2|AM|third|night|| +15723|AAAAAAAAMGNDAAAA|15723|4|22|3|AM|third|night|| +15724|AAAAAAAANGNDAAAA|15724|4|22|4|AM|third|night|| +15725|AAAAAAAAOGNDAAAA|15725|4|22|5|AM|third|night|| +15726|AAAAAAAAPGNDAAAA|15726|4|22|6|AM|third|night|| +15727|AAAAAAAAAHNDAAAA|15727|4|22|7|AM|third|night|| +15728|AAAAAAAABHNDAAAA|15728|4|22|8|AM|third|night|| +15729|AAAAAAAACHNDAAAA|15729|4|22|9|AM|third|night|| +15730|AAAAAAAADHNDAAAA|15730|4|22|10|AM|third|night|| +15731|AAAAAAAAEHNDAAAA|15731|4|22|11|AM|third|night|| +15732|AAAAAAAAFHNDAAAA|15732|4|22|12|AM|third|night|| +15733|AAAAAAAAGHNDAAAA|15733|4|22|13|AM|third|night|| +15734|AAAAAAAAHHNDAAAA|15734|4|22|14|AM|third|night|| +15735|AAAAAAAAIHNDAAAA|15735|4|22|15|AM|third|night|| +15736|AAAAAAAAJHNDAAAA|15736|4|22|16|AM|third|night|| +15737|AAAAAAAAKHNDAAAA|15737|4|22|17|AM|third|night|| +15738|AAAAAAAALHNDAAAA|15738|4|22|18|AM|third|night|| +15739|AAAAAAAAMHNDAAAA|15739|4|22|19|AM|third|night|| +15740|AAAAAAAANHNDAAAA|15740|4|22|20|AM|third|night|| +15741|AAAAAAAAOHNDAAAA|15741|4|22|21|AM|third|night|| +15742|AAAAAAAAPHNDAAAA|15742|4|22|22|AM|third|night|| +15743|AAAAAAAAAINDAAAA|15743|4|22|23|AM|third|night|| +15744|AAAAAAAABINDAAAA|15744|4|22|24|AM|third|night|| +15745|AAAAAAAACINDAAAA|15745|4|22|25|AM|third|night|| +15746|AAAAAAAADINDAAAA|15746|4|22|26|AM|third|night|| +15747|AAAAAAAAEINDAAAA|15747|4|22|27|AM|third|night|| +15748|AAAAAAAAFINDAAAA|15748|4|22|28|AM|third|night|| +15749|AAAAAAAAGINDAAAA|15749|4|22|29|AM|third|night|| +15750|AAAAAAAAHINDAAAA|15750|4|22|30|AM|third|night|| +15751|AAAAAAAAIINDAAAA|15751|4|22|31|AM|third|night|| +15752|AAAAAAAAJINDAAAA|15752|4|22|32|AM|third|night|| +15753|AAAAAAAAKINDAAAA|15753|4|22|33|AM|third|night|| +15754|AAAAAAAALINDAAAA|15754|4|22|34|AM|third|night|| +15755|AAAAAAAAMINDAAAA|15755|4|22|35|AM|third|night|| +15756|AAAAAAAANINDAAAA|15756|4|22|36|AM|third|night|| +15757|AAAAAAAAOINDAAAA|15757|4|22|37|AM|third|night|| +15758|AAAAAAAAPINDAAAA|15758|4|22|38|AM|third|night|| +15759|AAAAAAAAAJNDAAAA|15759|4|22|39|AM|third|night|| +15760|AAAAAAAABJNDAAAA|15760|4|22|40|AM|third|night|| +15761|AAAAAAAACJNDAAAA|15761|4|22|41|AM|third|night|| +15762|AAAAAAAADJNDAAAA|15762|4|22|42|AM|third|night|| +15763|AAAAAAAAEJNDAAAA|15763|4|22|43|AM|third|night|| +15764|AAAAAAAAFJNDAAAA|15764|4|22|44|AM|third|night|| +15765|AAAAAAAAGJNDAAAA|15765|4|22|45|AM|third|night|| +15766|AAAAAAAAHJNDAAAA|15766|4|22|46|AM|third|night|| +15767|AAAAAAAAIJNDAAAA|15767|4|22|47|AM|third|night|| +15768|AAAAAAAAJJNDAAAA|15768|4|22|48|AM|third|night|| +15769|AAAAAAAAKJNDAAAA|15769|4|22|49|AM|third|night|| +15770|AAAAAAAALJNDAAAA|15770|4|22|50|AM|third|night|| +15771|AAAAAAAAMJNDAAAA|15771|4|22|51|AM|third|night|| +15772|AAAAAAAANJNDAAAA|15772|4|22|52|AM|third|night|| +15773|AAAAAAAAOJNDAAAA|15773|4|22|53|AM|third|night|| +15774|AAAAAAAAPJNDAAAA|15774|4|22|54|AM|third|night|| +15775|AAAAAAAAAKNDAAAA|15775|4|22|55|AM|third|night|| +15776|AAAAAAAABKNDAAAA|15776|4|22|56|AM|third|night|| +15777|AAAAAAAACKNDAAAA|15777|4|22|57|AM|third|night|| +15778|AAAAAAAADKNDAAAA|15778|4|22|58|AM|third|night|| +15779|AAAAAAAAEKNDAAAA|15779|4|22|59|AM|third|night|| +15780|AAAAAAAAFKNDAAAA|15780|4|23|0|AM|third|night|| +15781|AAAAAAAAGKNDAAAA|15781|4|23|1|AM|third|night|| +15782|AAAAAAAAHKNDAAAA|15782|4|23|2|AM|third|night|| +15783|AAAAAAAAIKNDAAAA|15783|4|23|3|AM|third|night|| +15784|AAAAAAAAJKNDAAAA|15784|4|23|4|AM|third|night|| +15785|AAAAAAAAKKNDAAAA|15785|4|23|5|AM|third|night|| +15786|AAAAAAAALKNDAAAA|15786|4|23|6|AM|third|night|| +15787|AAAAAAAAMKNDAAAA|15787|4|23|7|AM|third|night|| +15788|AAAAAAAANKNDAAAA|15788|4|23|8|AM|third|night|| +15789|AAAAAAAAOKNDAAAA|15789|4|23|9|AM|third|night|| +15790|AAAAAAAAPKNDAAAA|15790|4|23|10|AM|third|night|| +15791|AAAAAAAAALNDAAAA|15791|4|23|11|AM|third|night|| +15792|AAAAAAAABLNDAAAA|15792|4|23|12|AM|third|night|| +15793|AAAAAAAACLNDAAAA|15793|4|23|13|AM|third|night|| +15794|AAAAAAAADLNDAAAA|15794|4|23|14|AM|third|night|| +15795|AAAAAAAAELNDAAAA|15795|4|23|15|AM|third|night|| +15796|AAAAAAAAFLNDAAAA|15796|4|23|16|AM|third|night|| +15797|AAAAAAAAGLNDAAAA|15797|4|23|17|AM|third|night|| +15798|AAAAAAAAHLNDAAAA|15798|4|23|18|AM|third|night|| +15799|AAAAAAAAILNDAAAA|15799|4|23|19|AM|third|night|| +15800|AAAAAAAAJLNDAAAA|15800|4|23|20|AM|third|night|| +15801|AAAAAAAAKLNDAAAA|15801|4|23|21|AM|third|night|| +15802|AAAAAAAALLNDAAAA|15802|4|23|22|AM|third|night|| +15803|AAAAAAAAMLNDAAAA|15803|4|23|23|AM|third|night|| +15804|AAAAAAAANLNDAAAA|15804|4|23|24|AM|third|night|| +15805|AAAAAAAAOLNDAAAA|15805|4|23|25|AM|third|night|| +15806|AAAAAAAAPLNDAAAA|15806|4|23|26|AM|third|night|| +15807|AAAAAAAAAMNDAAAA|15807|4|23|27|AM|third|night|| +15808|AAAAAAAABMNDAAAA|15808|4|23|28|AM|third|night|| +15809|AAAAAAAACMNDAAAA|15809|4|23|29|AM|third|night|| +15810|AAAAAAAADMNDAAAA|15810|4|23|30|AM|third|night|| +15811|AAAAAAAAEMNDAAAA|15811|4|23|31|AM|third|night|| +15812|AAAAAAAAFMNDAAAA|15812|4|23|32|AM|third|night|| +15813|AAAAAAAAGMNDAAAA|15813|4|23|33|AM|third|night|| +15814|AAAAAAAAHMNDAAAA|15814|4|23|34|AM|third|night|| +15815|AAAAAAAAIMNDAAAA|15815|4|23|35|AM|third|night|| +15816|AAAAAAAAJMNDAAAA|15816|4|23|36|AM|third|night|| +15817|AAAAAAAAKMNDAAAA|15817|4|23|37|AM|third|night|| +15818|AAAAAAAALMNDAAAA|15818|4|23|38|AM|third|night|| +15819|AAAAAAAAMMNDAAAA|15819|4|23|39|AM|third|night|| +15820|AAAAAAAANMNDAAAA|15820|4|23|40|AM|third|night|| +15821|AAAAAAAAOMNDAAAA|15821|4|23|41|AM|third|night|| +15822|AAAAAAAAPMNDAAAA|15822|4|23|42|AM|third|night|| +15823|AAAAAAAAANNDAAAA|15823|4|23|43|AM|third|night|| +15824|AAAAAAAABNNDAAAA|15824|4|23|44|AM|third|night|| +15825|AAAAAAAACNNDAAAA|15825|4|23|45|AM|third|night|| +15826|AAAAAAAADNNDAAAA|15826|4|23|46|AM|third|night|| +15827|AAAAAAAAENNDAAAA|15827|4|23|47|AM|third|night|| +15828|AAAAAAAAFNNDAAAA|15828|4|23|48|AM|third|night|| +15829|AAAAAAAAGNNDAAAA|15829|4|23|49|AM|third|night|| +15830|AAAAAAAAHNNDAAAA|15830|4|23|50|AM|third|night|| +15831|AAAAAAAAINNDAAAA|15831|4|23|51|AM|third|night|| +15832|AAAAAAAAJNNDAAAA|15832|4|23|52|AM|third|night|| +15833|AAAAAAAAKNNDAAAA|15833|4|23|53|AM|third|night|| +15834|AAAAAAAALNNDAAAA|15834|4|23|54|AM|third|night|| +15835|AAAAAAAAMNNDAAAA|15835|4|23|55|AM|third|night|| +15836|AAAAAAAANNNDAAAA|15836|4|23|56|AM|third|night|| +15837|AAAAAAAAONNDAAAA|15837|4|23|57|AM|third|night|| +15838|AAAAAAAAPNNDAAAA|15838|4|23|58|AM|third|night|| +15839|AAAAAAAAAONDAAAA|15839|4|23|59|AM|third|night|| +15840|AAAAAAAABONDAAAA|15840|4|24|0|AM|third|night|| +15841|AAAAAAAACONDAAAA|15841|4|24|1|AM|third|night|| +15842|AAAAAAAADONDAAAA|15842|4|24|2|AM|third|night|| +15843|AAAAAAAAEONDAAAA|15843|4|24|3|AM|third|night|| +15844|AAAAAAAAFONDAAAA|15844|4|24|4|AM|third|night|| +15845|AAAAAAAAGONDAAAA|15845|4|24|5|AM|third|night|| +15846|AAAAAAAAHONDAAAA|15846|4|24|6|AM|third|night|| +15847|AAAAAAAAIONDAAAA|15847|4|24|7|AM|third|night|| +15848|AAAAAAAAJONDAAAA|15848|4|24|8|AM|third|night|| +15849|AAAAAAAAKONDAAAA|15849|4|24|9|AM|third|night|| +15850|AAAAAAAALONDAAAA|15850|4|24|10|AM|third|night|| +15851|AAAAAAAAMONDAAAA|15851|4|24|11|AM|third|night|| +15852|AAAAAAAANONDAAAA|15852|4|24|12|AM|third|night|| +15853|AAAAAAAAOONDAAAA|15853|4|24|13|AM|third|night|| +15854|AAAAAAAAPONDAAAA|15854|4|24|14|AM|third|night|| +15855|AAAAAAAAAPNDAAAA|15855|4|24|15|AM|third|night|| +15856|AAAAAAAABPNDAAAA|15856|4|24|16|AM|third|night|| +15857|AAAAAAAACPNDAAAA|15857|4|24|17|AM|third|night|| +15858|AAAAAAAADPNDAAAA|15858|4|24|18|AM|third|night|| +15859|AAAAAAAAEPNDAAAA|15859|4|24|19|AM|third|night|| +15860|AAAAAAAAFPNDAAAA|15860|4|24|20|AM|third|night|| +15861|AAAAAAAAGPNDAAAA|15861|4|24|21|AM|third|night|| +15862|AAAAAAAAHPNDAAAA|15862|4|24|22|AM|third|night|| +15863|AAAAAAAAIPNDAAAA|15863|4|24|23|AM|third|night|| +15864|AAAAAAAAJPNDAAAA|15864|4|24|24|AM|third|night|| +15865|AAAAAAAAKPNDAAAA|15865|4|24|25|AM|third|night|| +15866|AAAAAAAALPNDAAAA|15866|4|24|26|AM|third|night|| +15867|AAAAAAAAMPNDAAAA|15867|4|24|27|AM|third|night|| +15868|AAAAAAAANPNDAAAA|15868|4|24|28|AM|third|night|| +15869|AAAAAAAAOPNDAAAA|15869|4|24|29|AM|third|night|| +15870|AAAAAAAAPPNDAAAA|15870|4|24|30|AM|third|night|| +15871|AAAAAAAAAAODAAAA|15871|4|24|31|AM|third|night|| +15872|AAAAAAAABAODAAAA|15872|4|24|32|AM|third|night|| +15873|AAAAAAAACAODAAAA|15873|4|24|33|AM|third|night|| +15874|AAAAAAAADAODAAAA|15874|4|24|34|AM|third|night|| +15875|AAAAAAAAEAODAAAA|15875|4|24|35|AM|third|night|| +15876|AAAAAAAAFAODAAAA|15876|4|24|36|AM|third|night|| +15877|AAAAAAAAGAODAAAA|15877|4|24|37|AM|third|night|| +15878|AAAAAAAAHAODAAAA|15878|4|24|38|AM|third|night|| +15879|AAAAAAAAIAODAAAA|15879|4|24|39|AM|third|night|| +15880|AAAAAAAAJAODAAAA|15880|4|24|40|AM|third|night|| +15881|AAAAAAAAKAODAAAA|15881|4|24|41|AM|third|night|| +15882|AAAAAAAALAODAAAA|15882|4|24|42|AM|third|night|| +15883|AAAAAAAAMAODAAAA|15883|4|24|43|AM|third|night|| +15884|AAAAAAAANAODAAAA|15884|4|24|44|AM|third|night|| +15885|AAAAAAAAOAODAAAA|15885|4|24|45|AM|third|night|| +15886|AAAAAAAAPAODAAAA|15886|4|24|46|AM|third|night|| +15887|AAAAAAAAABODAAAA|15887|4|24|47|AM|third|night|| +15888|AAAAAAAABBODAAAA|15888|4|24|48|AM|third|night|| +15889|AAAAAAAACBODAAAA|15889|4|24|49|AM|third|night|| +15890|AAAAAAAADBODAAAA|15890|4|24|50|AM|third|night|| +15891|AAAAAAAAEBODAAAA|15891|4|24|51|AM|third|night|| +15892|AAAAAAAAFBODAAAA|15892|4|24|52|AM|third|night|| +15893|AAAAAAAAGBODAAAA|15893|4|24|53|AM|third|night|| +15894|AAAAAAAAHBODAAAA|15894|4|24|54|AM|third|night|| +15895|AAAAAAAAIBODAAAA|15895|4|24|55|AM|third|night|| +15896|AAAAAAAAJBODAAAA|15896|4|24|56|AM|third|night|| +15897|AAAAAAAAKBODAAAA|15897|4|24|57|AM|third|night|| +15898|AAAAAAAALBODAAAA|15898|4|24|58|AM|third|night|| +15899|AAAAAAAAMBODAAAA|15899|4|24|59|AM|third|night|| +15900|AAAAAAAANBODAAAA|15900|4|25|0|AM|third|night|| +15901|AAAAAAAAOBODAAAA|15901|4|25|1|AM|third|night|| +15902|AAAAAAAAPBODAAAA|15902|4|25|2|AM|third|night|| +15903|AAAAAAAAACODAAAA|15903|4|25|3|AM|third|night|| +15904|AAAAAAAABCODAAAA|15904|4|25|4|AM|third|night|| +15905|AAAAAAAACCODAAAA|15905|4|25|5|AM|third|night|| +15906|AAAAAAAADCODAAAA|15906|4|25|6|AM|third|night|| +15907|AAAAAAAAECODAAAA|15907|4|25|7|AM|third|night|| +15908|AAAAAAAAFCODAAAA|15908|4|25|8|AM|third|night|| +15909|AAAAAAAAGCODAAAA|15909|4|25|9|AM|third|night|| +15910|AAAAAAAAHCODAAAA|15910|4|25|10|AM|third|night|| +15911|AAAAAAAAICODAAAA|15911|4|25|11|AM|third|night|| +15912|AAAAAAAAJCODAAAA|15912|4|25|12|AM|third|night|| +15913|AAAAAAAAKCODAAAA|15913|4|25|13|AM|third|night|| +15914|AAAAAAAALCODAAAA|15914|4|25|14|AM|third|night|| +15915|AAAAAAAAMCODAAAA|15915|4|25|15|AM|third|night|| +15916|AAAAAAAANCODAAAA|15916|4|25|16|AM|third|night|| +15917|AAAAAAAAOCODAAAA|15917|4|25|17|AM|third|night|| +15918|AAAAAAAAPCODAAAA|15918|4|25|18|AM|third|night|| +15919|AAAAAAAAADODAAAA|15919|4|25|19|AM|third|night|| +15920|AAAAAAAABDODAAAA|15920|4|25|20|AM|third|night|| +15921|AAAAAAAACDODAAAA|15921|4|25|21|AM|third|night|| +15922|AAAAAAAADDODAAAA|15922|4|25|22|AM|third|night|| +15923|AAAAAAAAEDODAAAA|15923|4|25|23|AM|third|night|| +15924|AAAAAAAAFDODAAAA|15924|4|25|24|AM|third|night|| +15925|AAAAAAAAGDODAAAA|15925|4|25|25|AM|third|night|| +15926|AAAAAAAAHDODAAAA|15926|4|25|26|AM|third|night|| +15927|AAAAAAAAIDODAAAA|15927|4|25|27|AM|third|night|| +15928|AAAAAAAAJDODAAAA|15928|4|25|28|AM|third|night|| +15929|AAAAAAAAKDODAAAA|15929|4|25|29|AM|third|night|| +15930|AAAAAAAALDODAAAA|15930|4|25|30|AM|third|night|| +15931|AAAAAAAAMDODAAAA|15931|4|25|31|AM|third|night|| +15932|AAAAAAAANDODAAAA|15932|4|25|32|AM|third|night|| +15933|AAAAAAAAODODAAAA|15933|4|25|33|AM|third|night|| +15934|AAAAAAAAPDODAAAA|15934|4|25|34|AM|third|night|| +15935|AAAAAAAAAEODAAAA|15935|4|25|35|AM|third|night|| +15936|AAAAAAAABEODAAAA|15936|4|25|36|AM|third|night|| +15937|AAAAAAAACEODAAAA|15937|4|25|37|AM|third|night|| +15938|AAAAAAAADEODAAAA|15938|4|25|38|AM|third|night|| +15939|AAAAAAAAEEODAAAA|15939|4|25|39|AM|third|night|| +15940|AAAAAAAAFEODAAAA|15940|4|25|40|AM|third|night|| +15941|AAAAAAAAGEODAAAA|15941|4|25|41|AM|third|night|| +15942|AAAAAAAAHEODAAAA|15942|4|25|42|AM|third|night|| +15943|AAAAAAAAIEODAAAA|15943|4|25|43|AM|third|night|| +15944|AAAAAAAAJEODAAAA|15944|4|25|44|AM|third|night|| +15945|AAAAAAAAKEODAAAA|15945|4|25|45|AM|third|night|| +15946|AAAAAAAALEODAAAA|15946|4|25|46|AM|third|night|| +15947|AAAAAAAAMEODAAAA|15947|4|25|47|AM|third|night|| +15948|AAAAAAAANEODAAAA|15948|4|25|48|AM|third|night|| +15949|AAAAAAAAOEODAAAA|15949|4|25|49|AM|third|night|| +15950|AAAAAAAAPEODAAAA|15950|4|25|50|AM|third|night|| +15951|AAAAAAAAAFODAAAA|15951|4|25|51|AM|third|night|| +15952|AAAAAAAABFODAAAA|15952|4|25|52|AM|third|night|| +15953|AAAAAAAACFODAAAA|15953|4|25|53|AM|third|night|| +15954|AAAAAAAADFODAAAA|15954|4|25|54|AM|third|night|| +15955|AAAAAAAAEFODAAAA|15955|4|25|55|AM|third|night|| +15956|AAAAAAAAFFODAAAA|15956|4|25|56|AM|third|night|| +15957|AAAAAAAAGFODAAAA|15957|4|25|57|AM|third|night|| +15958|AAAAAAAAHFODAAAA|15958|4|25|58|AM|third|night|| +15959|AAAAAAAAIFODAAAA|15959|4|25|59|AM|third|night|| +15960|AAAAAAAAJFODAAAA|15960|4|26|0|AM|third|night|| +15961|AAAAAAAAKFODAAAA|15961|4|26|1|AM|third|night|| +15962|AAAAAAAALFODAAAA|15962|4|26|2|AM|third|night|| +15963|AAAAAAAAMFODAAAA|15963|4|26|3|AM|third|night|| +15964|AAAAAAAANFODAAAA|15964|4|26|4|AM|third|night|| +15965|AAAAAAAAOFODAAAA|15965|4|26|5|AM|third|night|| +15966|AAAAAAAAPFODAAAA|15966|4|26|6|AM|third|night|| +15967|AAAAAAAAAGODAAAA|15967|4|26|7|AM|third|night|| +15968|AAAAAAAABGODAAAA|15968|4|26|8|AM|third|night|| +15969|AAAAAAAACGODAAAA|15969|4|26|9|AM|third|night|| +15970|AAAAAAAADGODAAAA|15970|4|26|10|AM|third|night|| +15971|AAAAAAAAEGODAAAA|15971|4|26|11|AM|third|night|| +15972|AAAAAAAAFGODAAAA|15972|4|26|12|AM|third|night|| +15973|AAAAAAAAGGODAAAA|15973|4|26|13|AM|third|night|| +15974|AAAAAAAAHGODAAAA|15974|4|26|14|AM|third|night|| +15975|AAAAAAAAIGODAAAA|15975|4|26|15|AM|third|night|| +15976|AAAAAAAAJGODAAAA|15976|4|26|16|AM|third|night|| +15977|AAAAAAAAKGODAAAA|15977|4|26|17|AM|third|night|| +15978|AAAAAAAALGODAAAA|15978|4|26|18|AM|third|night|| +15979|AAAAAAAAMGODAAAA|15979|4|26|19|AM|third|night|| +15980|AAAAAAAANGODAAAA|15980|4|26|20|AM|third|night|| +15981|AAAAAAAAOGODAAAA|15981|4|26|21|AM|third|night|| +15982|AAAAAAAAPGODAAAA|15982|4|26|22|AM|third|night|| +15983|AAAAAAAAAHODAAAA|15983|4|26|23|AM|third|night|| +15984|AAAAAAAABHODAAAA|15984|4|26|24|AM|third|night|| +15985|AAAAAAAACHODAAAA|15985|4|26|25|AM|third|night|| +15986|AAAAAAAADHODAAAA|15986|4|26|26|AM|third|night|| +15987|AAAAAAAAEHODAAAA|15987|4|26|27|AM|third|night|| +15988|AAAAAAAAFHODAAAA|15988|4|26|28|AM|third|night|| +15989|AAAAAAAAGHODAAAA|15989|4|26|29|AM|third|night|| +15990|AAAAAAAAHHODAAAA|15990|4|26|30|AM|third|night|| +15991|AAAAAAAAIHODAAAA|15991|4|26|31|AM|third|night|| +15992|AAAAAAAAJHODAAAA|15992|4|26|32|AM|third|night|| +15993|AAAAAAAAKHODAAAA|15993|4|26|33|AM|third|night|| +15994|AAAAAAAALHODAAAA|15994|4|26|34|AM|third|night|| +15995|AAAAAAAAMHODAAAA|15995|4|26|35|AM|third|night|| +15996|AAAAAAAANHODAAAA|15996|4|26|36|AM|third|night|| +15997|AAAAAAAAOHODAAAA|15997|4|26|37|AM|third|night|| +15998|AAAAAAAAPHODAAAA|15998|4|26|38|AM|third|night|| +15999|AAAAAAAAAIODAAAA|15999|4|26|39|AM|third|night|| +16000|AAAAAAAABIODAAAA|16000|4|26|40|AM|third|night|| +16001|AAAAAAAACIODAAAA|16001|4|26|41|AM|third|night|| +16002|AAAAAAAADIODAAAA|16002|4|26|42|AM|third|night|| +16003|AAAAAAAAEIODAAAA|16003|4|26|43|AM|third|night|| +16004|AAAAAAAAFIODAAAA|16004|4|26|44|AM|third|night|| +16005|AAAAAAAAGIODAAAA|16005|4|26|45|AM|third|night|| +16006|AAAAAAAAHIODAAAA|16006|4|26|46|AM|third|night|| +16007|AAAAAAAAIIODAAAA|16007|4|26|47|AM|third|night|| +16008|AAAAAAAAJIODAAAA|16008|4|26|48|AM|third|night|| +16009|AAAAAAAAKIODAAAA|16009|4|26|49|AM|third|night|| +16010|AAAAAAAALIODAAAA|16010|4|26|50|AM|third|night|| +16011|AAAAAAAAMIODAAAA|16011|4|26|51|AM|third|night|| +16012|AAAAAAAANIODAAAA|16012|4|26|52|AM|third|night|| +16013|AAAAAAAAOIODAAAA|16013|4|26|53|AM|third|night|| +16014|AAAAAAAAPIODAAAA|16014|4|26|54|AM|third|night|| +16015|AAAAAAAAAJODAAAA|16015|4|26|55|AM|third|night|| +16016|AAAAAAAABJODAAAA|16016|4|26|56|AM|third|night|| +16017|AAAAAAAACJODAAAA|16017|4|26|57|AM|third|night|| +16018|AAAAAAAADJODAAAA|16018|4|26|58|AM|third|night|| +16019|AAAAAAAAEJODAAAA|16019|4|26|59|AM|third|night|| +16020|AAAAAAAAFJODAAAA|16020|4|27|0|AM|third|night|| +16021|AAAAAAAAGJODAAAA|16021|4|27|1|AM|third|night|| +16022|AAAAAAAAHJODAAAA|16022|4|27|2|AM|third|night|| +16023|AAAAAAAAIJODAAAA|16023|4|27|3|AM|third|night|| +16024|AAAAAAAAJJODAAAA|16024|4|27|4|AM|third|night|| +16025|AAAAAAAAKJODAAAA|16025|4|27|5|AM|third|night|| +16026|AAAAAAAALJODAAAA|16026|4|27|6|AM|third|night|| +16027|AAAAAAAAMJODAAAA|16027|4|27|7|AM|third|night|| +16028|AAAAAAAANJODAAAA|16028|4|27|8|AM|third|night|| +16029|AAAAAAAAOJODAAAA|16029|4|27|9|AM|third|night|| +16030|AAAAAAAAPJODAAAA|16030|4|27|10|AM|third|night|| +16031|AAAAAAAAAKODAAAA|16031|4|27|11|AM|third|night|| +16032|AAAAAAAABKODAAAA|16032|4|27|12|AM|third|night|| +16033|AAAAAAAACKODAAAA|16033|4|27|13|AM|third|night|| +16034|AAAAAAAADKODAAAA|16034|4|27|14|AM|third|night|| +16035|AAAAAAAAEKODAAAA|16035|4|27|15|AM|third|night|| +16036|AAAAAAAAFKODAAAA|16036|4|27|16|AM|third|night|| +16037|AAAAAAAAGKODAAAA|16037|4|27|17|AM|third|night|| +16038|AAAAAAAAHKODAAAA|16038|4|27|18|AM|third|night|| +16039|AAAAAAAAIKODAAAA|16039|4|27|19|AM|third|night|| +16040|AAAAAAAAJKODAAAA|16040|4|27|20|AM|third|night|| +16041|AAAAAAAAKKODAAAA|16041|4|27|21|AM|third|night|| +16042|AAAAAAAALKODAAAA|16042|4|27|22|AM|third|night|| +16043|AAAAAAAAMKODAAAA|16043|4|27|23|AM|third|night|| +16044|AAAAAAAANKODAAAA|16044|4|27|24|AM|third|night|| +16045|AAAAAAAAOKODAAAA|16045|4|27|25|AM|third|night|| +16046|AAAAAAAAPKODAAAA|16046|4|27|26|AM|third|night|| +16047|AAAAAAAAALODAAAA|16047|4|27|27|AM|third|night|| +16048|AAAAAAAABLODAAAA|16048|4|27|28|AM|third|night|| +16049|AAAAAAAACLODAAAA|16049|4|27|29|AM|third|night|| +16050|AAAAAAAADLODAAAA|16050|4|27|30|AM|third|night|| +16051|AAAAAAAAELODAAAA|16051|4|27|31|AM|third|night|| +16052|AAAAAAAAFLODAAAA|16052|4|27|32|AM|third|night|| +16053|AAAAAAAAGLODAAAA|16053|4|27|33|AM|third|night|| +16054|AAAAAAAAHLODAAAA|16054|4|27|34|AM|third|night|| +16055|AAAAAAAAILODAAAA|16055|4|27|35|AM|third|night|| +16056|AAAAAAAAJLODAAAA|16056|4|27|36|AM|third|night|| +16057|AAAAAAAAKLODAAAA|16057|4|27|37|AM|third|night|| +16058|AAAAAAAALLODAAAA|16058|4|27|38|AM|third|night|| +16059|AAAAAAAAMLODAAAA|16059|4|27|39|AM|third|night|| +16060|AAAAAAAANLODAAAA|16060|4|27|40|AM|third|night|| +16061|AAAAAAAAOLODAAAA|16061|4|27|41|AM|third|night|| +16062|AAAAAAAAPLODAAAA|16062|4|27|42|AM|third|night|| +16063|AAAAAAAAAMODAAAA|16063|4|27|43|AM|third|night|| +16064|AAAAAAAABMODAAAA|16064|4|27|44|AM|third|night|| +16065|AAAAAAAACMODAAAA|16065|4|27|45|AM|third|night|| +16066|AAAAAAAADMODAAAA|16066|4|27|46|AM|third|night|| +16067|AAAAAAAAEMODAAAA|16067|4|27|47|AM|third|night|| +16068|AAAAAAAAFMODAAAA|16068|4|27|48|AM|third|night|| +16069|AAAAAAAAGMODAAAA|16069|4|27|49|AM|third|night|| +16070|AAAAAAAAHMODAAAA|16070|4|27|50|AM|third|night|| +16071|AAAAAAAAIMODAAAA|16071|4|27|51|AM|third|night|| +16072|AAAAAAAAJMODAAAA|16072|4|27|52|AM|third|night|| +16073|AAAAAAAAKMODAAAA|16073|4|27|53|AM|third|night|| +16074|AAAAAAAALMODAAAA|16074|4|27|54|AM|third|night|| +16075|AAAAAAAAMMODAAAA|16075|4|27|55|AM|third|night|| +16076|AAAAAAAANMODAAAA|16076|4|27|56|AM|third|night|| +16077|AAAAAAAAOMODAAAA|16077|4|27|57|AM|third|night|| +16078|AAAAAAAAPMODAAAA|16078|4|27|58|AM|third|night|| +16079|AAAAAAAAANODAAAA|16079|4|27|59|AM|third|night|| +16080|AAAAAAAABNODAAAA|16080|4|28|0|AM|third|night|| +16081|AAAAAAAACNODAAAA|16081|4|28|1|AM|third|night|| +16082|AAAAAAAADNODAAAA|16082|4|28|2|AM|third|night|| +16083|AAAAAAAAENODAAAA|16083|4|28|3|AM|third|night|| +16084|AAAAAAAAFNODAAAA|16084|4|28|4|AM|third|night|| +16085|AAAAAAAAGNODAAAA|16085|4|28|5|AM|third|night|| +16086|AAAAAAAAHNODAAAA|16086|4|28|6|AM|third|night|| +16087|AAAAAAAAINODAAAA|16087|4|28|7|AM|third|night|| +16088|AAAAAAAAJNODAAAA|16088|4|28|8|AM|third|night|| +16089|AAAAAAAAKNODAAAA|16089|4|28|9|AM|third|night|| +16090|AAAAAAAALNODAAAA|16090|4|28|10|AM|third|night|| +16091|AAAAAAAAMNODAAAA|16091|4|28|11|AM|third|night|| +16092|AAAAAAAANNODAAAA|16092|4|28|12|AM|third|night|| +16093|AAAAAAAAONODAAAA|16093|4|28|13|AM|third|night|| +16094|AAAAAAAAPNODAAAA|16094|4|28|14|AM|third|night|| +16095|AAAAAAAAAOODAAAA|16095|4|28|15|AM|third|night|| +16096|AAAAAAAABOODAAAA|16096|4|28|16|AM|third|night|| +16097|AAAAAAAACOODAAAA|16097|4|28|17|AM|third|night|| +16098|AAAAAAAADOODAAAA|16098|4|28|18|AM|third|night|| +16099|AAAAAAAAEOODAAAA|16099|4|28|19|AM|third|night|| +16100|AAAAAAAAFOODAAAA|16100|4|28|20|AM|third|night|| +16101|AAAAAAAAGOODAAAA|16101|4|28|21|AM|third|night|| +16102|AAAAAAAAHOODAAAA|16102|4|28|22|AM|third|night|| +16103|AAAAAAAAIOODAAAA|16103|4|28|23|AM|third|night|| +16104|AAAAAAAAJOODAAAA|16104|4|28|24|AM|third|night|| +16105|AAAAAAAAKOODAAAA|16105|4|28|25|AM|third|night|| +16106|AAAAAAAALOODAAAA|16106|4|28|26|AM|third|night|| +16107|AAAAAAAAMOODAAAA|16107|4|28|27|AM|third|night|| +16108|AAAAAAAANOODAAAA|16108|4|28|28|AM|third|night|| +16109|AAAAAAAAOOODAAAA|16109|4|28|29|AM|third|night|| +16110|AAAAAAAAPOODAAAA|16110|4|28|30|AM|third|night|| +16111|AAAAAAAAAPODAAAA|16111|4|28|31|AM|third|night|| +16112|AAAAAAAABPODAAAA|16112|4|28|32|AM|third|night|| +16113|AAAAAAAACPODAAAA|16113|4|28|33|AM|third|night|| +16114|AAAAAAAADPODAAAA|16114|4|28|34|AM|third|night|| +16115|AAAAAAAAEPODAAAA|16115|4|28|35|AM|third|night|| +16116|AAAAAAAAFPODAAAA|16116|4|28|36|AM|third|night|| +16117|AAAAAAAAGPODAAAA|16117|4|28|37|AM|third|night|| +16118|AAAAAAAAHPODAAAA|16118|4|28|38|AM|third|night|| +16119|AAAAAAAAIPODAAAA|16119|4|28|39|AM|third|night|| +16120|AAAAAAAAJPODAAAA|16120|4|28|40|AM|third|night|| +16121|AAAAAAAAKPODAAAA|16121|4|28|41|AM|third|night|| +16122|AAAAAAAALPODAAAA|16122|4|28|42|AM|third|night|| +16123|AAAAAAAAMPODAAAA|16123|4|28|43|AM|third|night|| +16124|AAAAAAAANPODAAAA|16124|4|28|44|AM|third|night|| +16125|AAAAAAAAOPODAAAA|16125|4|28|45|AM|third|night|| +16126|AAAAAAAAPPODAAAA|16126|4|28|46|AM|third|night|| +16127|AAAAAAAAAAPDAAAA|16127|4|28|47|AM|third|night|| +16128|AAAAAAAABAPDAAAA|16128|4|28|48|AM|third|night|| +16129|AAAAAAAACAPDAAAA|16129|4|28|49|AM|third|night|| +16130|AAAAAAAADAPDAAAA|16130|4|28|50|AM|third|night|| +16131|AAAAAAAAEAPDAAAA|16131|4|28|51|AM|third|night|| +16132|AAAAAAAAFAPDAAAA|16132|4|28|52|AM|third|night|| +16133|AAAAAAAAGAPDAAAA|16133|4|28|53|AM|third|night|| +16134|AAAAAAAAHAPDAAAA|16134|4|28|54|AM|third|night|| +16135|AAAAAAAAIAPDAAAA|16135|4|28|55|AM|third|night|| +16136|AAAAAAAAJAPDAAAA|16136|4|28|56|AM|third|night|| +16137|AAAAAAAAKAPDAAAA|16137|4|28|57|AM|third|night|| +16138|AAAAAAAALAPDAAAA|16138|4|28|58|AM|third|night|| +16139|AAAAAAAAMAPDAAAA|16139|4|28|59|AM|third|night|| +16140|AAAAAAAANAPDAAAA|16140|4|29|0|AM|third|night|| +16141|AAAAAAAAOAPDAAAA|16141|4|29|1|AM|third|night|| +16142|AAAAAAAAPAPDAAAA|16142|4|29|2|AM|third|night|| +16143|AAAAAAAAABPDAAAA|16143|4|29|3|AM|third|night|| +16144|AAAAAAAABBPDAAAA|16144|4|29|4|AM|third|night|| +16145|AAAAAAAACBPDAAAA|16145|4|29|5|AM|third|night|| +16146|AAAAAAAADBPDAAAA|16146|4|29|6|AM|third|night|| +16147|AAAAAAAAEBPDAAAA|16147|4|29|7|AM|third|night|| +16148|AAAAAAAAFBPDAAAA|16148|4|29|8|AM|third|night|| +16149|AAAAAAAAGBPDAAAA|16149|4|29|9|AM|third|night|| +16150|AAAAAAAAHBPDAAAA|16150|4|29|10|AM|third|night|| +16151|AAAAAAAAIBPDAAAA|16151|4|29|11|AM|third|night|| +16152|AAAAAAAAJBPDAAAA|16152|4|29|12|AM|third|night|| +16153|AAAAAAAAKBPDAAAA|16153|4|29|13|AM|third|night|| +16154|AAAAAAAALBPDAAAA|16154|4|29|14|AM|third|night|| +16155|AAAAAAAAMBPDAAAA|16155|4|29|15|AM|third|night|| +16156|AAAAAAAANBPDAAAA|16156|4|29|16|AM|third|night|| +16157|AAAAAAAAOBPDAAAA|16157|4|29|17|AM|third|night|| +16158|AAAAAAAAPBPDAAAA|16158|4|29|18|AM|third|night|| +16159|AAAAAAAAACPDAAAA|16159|4|29|19|AM|third|night|| +16160|AAAAAAAABCPDAAAA|16160|4|29|20|AM|third|night|| +16161|AAAAAAAACCPDAAAA|16161|4|29|21|AM|third|night|| +16162|AAAAAAAADCPDAAAA|16162|4|29|22|AM|third|night|| +16163|AAAAAAAAECPDAAAA|16163|4|29|23|AM|third|night|| +16164|AAAAAAAAFCPDAAAA|16164|4|29|24|AM|third|night|| +16165|AAAAAAAAGCPDAAAA|16165|4|29|25|AM|third|night|| +16166|AAAAAAAAHCPDAAAA|16166|4|29|26|AM|third|night|| +16167|AAAAAAAAICPDAAAA|16167|4|29|27|AM|third|night|| +16168|AAAAAAAAJCPDAAAA|16168|4|29|28|AM|third|night|| +16169|AAAAAAAAKCPDAAAA|16169|4|29|29|AM|third|night|| +16170|AAAAAAAALCPDAAAA|16170|4|29|30|AM|third|night|| +16171|AAAAAAAAMCPDAAAA|16171|4|29|31|AM|third|night|| +16172|AAAAAAAANCPDAAAA|16172|4|29|32|AM|third|night|| +16173|AAAAAAAAOCPDAAAA|16173|4|29|33|AM|third|night|| +16174|AAAAAAAAPCPDAAAA|16174|4|29|34|AM|third|night|| +16175|AAAAAAAAADPDAAAA|16175|4|29|35|AM|third|night|| +16176|AAAAAAAABDPDAAAA|16176|4|29|36|AM|third|night|| +16177|AAAAAAAACDPDAAAA|16177|4|29|37|AM|third|night|| +16178|AAAAAAAADDPDAAAA|16178|4|29|38|AM|third|night|| +16179|AAAAAAAAEDPDAAAA|16179|4|29|39|AM|third|night|| +16180|AAAAAAAAFDPDAAAA|16180|4|29|40|AM|third|night|| +16181|AAAAAAAAGDPDAAAA|16181|4|29|41|AM|third|night|| +16182|AAAAAAAAHDPDAAAA|16182|4|29|42|AM|third|night|| +16183|AAAAAAAAIDPDAAAA|16183|4|29|43|AM|third|night|| +16184|AAAAAAAAJDPDAAAA|16184|4|29|44|AM|third|night|| +16185|AAAAAAAAKDPDAAAA|16185|4|29|45|AM|third|night|| +16186|AAAAAAAALDPDAAAA|16186|4|29|46|AM|third|night|| +16187|AAAAAAAAMDPDAAAA|16187|4|29|47|AM|third|night|| +16188|AAAAAAAANDPDAAAA|16188|4|29|48|AM|third|night|| +16189|AAAAAAAAODPDAAAA|16189|4|29|49|AM|third|night|| +16190|AAAAAAAAPDPDAAAA|16190|4|29|50|AM|third|night|| +16191|AAAAAAAAAEPDAAAA|16191|4|29|51|AM|third|night|| +16192|AAAAAAAABEPDAAAA|16192|4|29|52|AM|third|night|| +16193|AAAAAAAACEPDAAAA|16193|4|29|53|AM|third|night|| +16194|AAAAAAAADEPDAAAA|16194|4|29|54|AM|third|night|| +16195|AAAAAAAAEEPDAAAA|16195|4|29|55|AM|third|night|| +16196|AAAAAAAAFEPDAAAA|16196|4|29|56|AM|third|night|| +16197|AAAAAAAAGEPDAAAA|16197|4|29|57|AM|third|night|| +16198|AAAAAAAAHEPDAAAA|16198|4|29|58|AM|third|night|| +16199|AAAAAAAAIEPDAAAA|16199|4|29|59|AM|third|night|| +16200|AAAAAAAAJEPDAAAA|16200|4|30|0|AM|third|night|| +16201|AAAAAAAAKEPDAAAA|16201|4|30|1|AM|third|night|| +16202|AAAAAAAALEPDAAAA|16202|4|30|2|AM|third|night|| +16203|AAAAAAAAMEPDAAAA|16203|4|30|3|AM|third|night|| +16204|AAAAAAAANEPDAAAA|16204|4|30|4|AM|third|night|| +16205|AAAAAAAAOEPDAAAA|16205|4|30|5|AM|third|night|| +16206|AAAAAAAAPEPDAAAA|16206|4|30|6|AM|third|night|| +16207|AAAAAAAAAFPDAAAA|16207|4|30|7|AM|third|night|| +16208|AAAAAAAABFPDAAAA|16208|4|30|8|AM|third|night|| +16209|AAAAAAAACFPDAAAA|16209|4|30|9|AM|third|night|| +16210|AAAAAAAADFPDAAAA|16210|4|30|10|AM|third|night|| +16211|AAAAAAAAEFPDAAAA|16211|4|30|11|AM|third|night|| +16212|AAAAAAAAFFPDAAAA|16212|4|30|12|AM|third|night|| +16213|AAAAAAAAGFPDAAAA|16213|4|30|13|AM|third|night|| +16214|AAAAAAAAHFPDAAAA|16214|4|30|14|AM|third|night|| +16215|AAAAAAAAIFPDAAAA|16215|4|30|15|AM|third|night|| +16216|AAAAAAAAJFPDAAAA|16216|4|30|16|AM|third|night|| +16217|AAAAAAAAKFPDAAAA|16217|4|30|17|AM|third|night|| +16218|AAAAAAAALFPDAAAA|16218|4|30|18|AM|third|night|| +16219|AAAAAAAAMFPDAAAA|16219|4|30|19|AM|third|night|| +16220|AAAAAAAANFPDAAAA|16220|4|30|20|AM|third|night|| +16221|AAAAAAAAOFPDAAAA|16221|4|30|21|AM|third|night|| +16222|AAAAAAAAPFPDAAAA|16222|4|30|22|AM|third|night|| +16223|AAAAAAAAAGPDAAAA|16223|4|30|23|AM|third|night|| +16224|AAAAAAAABGPDAAAA|16224|4|30|24|AM|third|night|| +16225|AAAAAAAACGPDAAAA|16225|4|30|25|AM|third|night|| +16226|AAAAAAAADGPDAAAA|16226|4|30|26|AM|third|night|| +16227|AAAAAAAAEGPDAAAA|16227|4|30|27|AM|third|night|| +16228|AAAAAAAAFGPDAAAA|16228|4|30|28|AM|third|night|| +16229|AAAAAAAAGGPDAAAA|16229|4|30|29|AM|third|night|| +16230|AAAAAAAAHGPDAAAA|16230|4|30|30|AM|third|night|| +16231|AAAAAAAAIGPDAAAA|16231|4|30|31|AM|third|night|| +16232|AAAAAAAAJGPDAAAA|16232|4|30|32|AM|third|night|| +16233|AAAAAAAAKGPDAAAA|16233|4|30|33|AM|third|night|| +16234|AAAAAAAALGPDAAAA|16234|4|30|34|AM|third|night|| +16235|AAAAAAAAMGPDAAAA|16235|4|30|35|AM|third|night|| +16236|AAAAAAAANGPDAAAA|16236|4|30|36|AM|third|night|| +16237|AAAAAAAAOGPDAAAA|16237|4|30|37|AM|third|night|| +16238|AAAAAAAAPGPDAAAA|16238|4|30|38|AM|third|night|| +16239|AAAAAAAAAHPDAAAA|16239|4|30|39|AM|third|night|| +16240|AAAAAAAABHPDAAAA|16240|4|30|40|AM|third|night|| +16241|AAAAAAAACHPDAAAA|16241|4|30|41|AM|third|night|| +16242|AAAAAAAADHPDAAAA|16242|4|30|42|AM|third|night|| +16243|AAAAAAAAEHPDAAAA|16243|4|30|43|AM|third|night|| +16244|AAAAAAAAFHPDAAAA|16244|4|30|44|AM|third|night|| +16245|AAAAAAAAGHPDAAAA|16245|4|30|45|AM|third|night|| +16246|AAAAAAAAHHPDAAAA|16246|4|30|46|AM|third|night|| +16247|AAAAAAAAIHPDAAAA|16247|4|30|47|AM|third|night|| +16248|AAAAAAAAJHPDAAAA|16248|4|30|48|AM|third|night|| +16249|AAAAAAAAKHPDAAAA|16249|4|30|49|AM|third|night|| +16250|AAAAAAAALHPDAAAA|16250|4|30|50|AM|third|night|| +16251|AAAAAAAAMHPDAAAA|16251|4|30|51|AM|third|night|| +16252|AAAAAAAANHPDAAAA|16252|4|30|52|AM|third|night|| +16253|AAAAAAAAOHPDAAAA|16253|4|30|53|AM|third|night|| +16254|AAAAAAAAPHPDAAAA|16254|4|30|54|AM|third|night|| +16255|AAAAAAAAAIPDAAAA|16255|4|30|55|AM|third|night|| +16256|AAAAAAAABIPDAAAA|16256|4|30|56|AM|third|night|| +16257|AAAAAAAACIPDAAAA|16257|4|30|57|AM|third|night|| +16258|AAAAAAAADIPDAAAA|16258|4|30|58|AM|third|night|| +16259|AAAAAAAAEIPDAAAA|16259|4|30|59|AM|third|night|| +16260|AAAAAAAAFIPDAAAA|16260|4|31|0|AM|third|night|| +16261|AAAAAAAAGIPDAAAA|16261|4|31|1|AM|third|night|| +16262|AAAAAAAAHIPDAAAA|16262|4|31|2|AM|third|night|| +16263|AAAAAAAAIIPDAAAA|16263|4|31|3|AM|third|night|| +16264|AAAAAAAAJIPDAAAA|16264|4|31|4|AM|third|night|| +16265|AAAAAAAAKIPDAAAA|16265|4|31|5|AM|third|night|| +16266|AAAAAAAALIPDAAAA|16266|4|31|6|AM|third|night|| +16267|AAAAAAAAMIPDAAAA|16267|4|31|7|AM|third|night|| +16268|AAAAAAAANIPDAAAA|16268|4|31|8|AM|third|night|| +16269|AAAAAAAAOIPDAAAA|16269|4|31|9|AM|third|night|| +16270|AAAAAAAAPIPDAAAA|16270|4|31|10|AM|third|night|| +16271|AAAAAAAAAJPDAAAA|16271|4|31|11|AM|third|night|| +16272|AAAAAAAABJPDAAAA|16272|4|31|12|AM|third|night|| +16273|AAAAAAAACJPDAAAA|16273|4|31|13|AM|third|night|| +16274|AAAAAAAADJPDAAAA|16274|4|31|14|AM|third|night|| +16275|AAAAAAAAEJPDAAAA|16275|4|31|15|AM|third|night|| +16276|AAAAAAAAFJPDAAAA|16276|4|31|16|AM|third|night|| +16277|AAAAAAAAGJPDAAAA|16277|4|31|17|AM|third|night|| +16278|AAAAAAAAHJPDAAAA|16278|4|31|18|AM|third|night|| +16279|AAAAAAAAIJPDAAAA|16279|4|31|19|AM|third|night|| +16280|AAAAAAAAJJPDAAAA|16280|4|31|20|AM|third|night|| +16281|AAAAAAAAKJPDAAAA|16281|4|31|21|AM|third|night|| +16282|AAAAAAAALJPDAAAA|16282|4|31|22|AM|third|night|| +16283|AAAAAAAAMJPDAAAA|16283|4|31|23|AM|third|night|| +16284|AAAAAAAANJPDAAAA|16284|4|31|24|AM|third|night|| +16285|AAAAAAAAOJPDAAAA|16285|4|31|25|AM|third|night|| +16286|AAAAAAAAPJPDAAAA|16286|4|31|26|AM|third|night|| +16287|AAAAAAAAAKPDAAAA|16287|4|31|27|AM|third|night|| +16288|AAAAAAAABKPDAAAA|16288|4|31|28|AM|third|night|| +16289|AAAAAAAACKPDAAAA|16289|4|31|29|AM|third|night|| +16290|AAAAAAAADKPDAAAA|16290|4|31|30|AM|third|night|| +16291|AAAAAAAAEKPDAAAA|16291|4|31|31|AM|third|night|| +16292|AAAAAAAAFKPDAAAA|16292|4|31|32|AM|third|night|| +16293|AAAAAAAAGKPDAAAA|16293|4|31|33|AM|third|night|| +16294|AAAAAAAAHKPDAAAA|16294|4|31|34|AM|third|night|| +16295|AAAAAAAAIKPDAAAA|16295|4|31|35|AM|third|night|| +16296|AAAAAAAAJKPDAAAA|16296|4|31|36|AM|third|night|| +16297|AAAAAAAAKKPDAAAA|16297|4|31|37|AM|third|night|| +16298|AAAAAAAALKPDAAAA|16298|4|31|38|AM|third|night|| +16299|AAAAAAAAMKPDAAAA|16299|4|31|39|AM|third|night|| +16300|AAAAAAAANKPDAAAA|16300|4|31|40|AM|third|night|| +16301|AAAAAAAAOKPDAAAA|16301|4|31|41|AM|third|night|| +16302|AAAAAAAAPKPDAAAA|16302|4|31|42|AM|third|night|| +16303|AAAAAAAAALPDAAAA|16303|4|31|43|AM|third|night|| +16304|AAAAAAAABLPDAAAA|16304|4|31|44|AM|third|night|| +16305|AAAAAAAACLPDAAAA|16305|4|31|45|AM|third|night|| +16306|AAAAAAAADLPDAAAA|16306|4|31|46|AM|third|night|| +16307|AAAAAAAAELPDAAAA|16307|4|31|47|AM|third|night|| +16308|AAAAAAAAFLPDAAAA|16308|4|31|48|AM|third|night|| +16309|AAAAAAAAGLPDAAAA|16309|4|31|49|AM|third|night|| +16310|AAAAAAAAHLPDAAAA|16310|4|31|50|AM|third|night|| +16311|AAAAAAAAILPDAAAA|16311|4|31|51|AM|third|night|| +16312|AAAAAAAAJLPDAAAA|16312|4|31|52|AM|third|night|| +16313|AAAAAAAAKLPDAAAA|16313|4|31|53|AM|third|night|| +16314|AAAAAAAALLPDAAAA|16314|4|31|54|AM|third|night|| +16315|AAAAAAAAMLPDAAAA|16315|4|31|55|AM|third|night|| +16316|AAAAAAAANLPDAAAA|16316|4|31|56|AM|third|night|| +16317|AAAAAAAAOLPDAAAA|16317|4|31|57|AM|third|night|| +16318|AAAAAAAAPLPDAAAA|16318|4|31|58|AM|third|night|| +16319|AAAAAAAAAMPDAAAA|16319|4|31|59|AM|third|night|| +16320|AAAAAAAABMPDAAAA|16320|4|32|0|AM|third|night|| +16321|AAAAAAAACMPDAAAA|16321|4|32|1|AM|third|night|| +16322|AAAAAAAADMPDAAAA|16322|4|32|2|AM|third|night|| +16323|AAAAAAAAEMPDAAAA|16323|4|32|3|AM|third|night|| +16324|AAAAAAAAFMPDAAAA|16324|4|32|4|AM|third|night|| +16325|AAAAAAAAGMPDAAAA|16325|4|32|5|AM|third|night|| +16326|AAAAAAAAHMPDAAAA|16326|4|32|6|AM|third|night|| +16327|AAAAAAAAIMPDAAAA|16327|4|32|7|AM|third|night|| +16328|AAAAAAAAJMPDAAAA|16328|4|32|8|AM|third|night|| +16329|AAAAAAAAKMPDAAAA|16329|4|32|9|AM|third|night|| +16330|AAAAAAAALMPDAAAA|16330|4|32|10|AM|third|night|| +16331|AAAAAAAAMMPDAAAA|16331|4|32|11|AM|third|night|| +16332|AAAAAAAANMPDAAAA|16332|4|32|12|AM|third|night|| +16333|AAAAAAAAOMPDAAAA|16333|4|32|13|AM|third|night|| +16334|AAAAAAAAPMPDAAAA|16334|4|32|14|AM|third|night|| +16335|AAAAAAAAANPDAAAA|16335|4|32|15|AM|third|night|| +16336|AAAAAAAABNPDAAAA|16336|4|32|16|AM|third|night|| +16337|AAAAAAAACNPDAAAA|16337|4|32|17|AM|third|night|| +16338|AAAAAAAADNPDAAAA|16338|4|32|18|AM|third|night|| +16339|AAAAAAAAENPDAAAA|16339|4|32|19|AM|third|night|| +16340|AAAAAAAAFNPDAAAA|16340|4|32|20|AM|third|night|| +16341|AAAAAAAAGNPDAAAA|16341|4|32|21|AM|third|night|| +16342|AAAAAAAAHNPDAAAA|16342|4|32|22|AM|third|night|| +16343|AAAAAAAAINPDAAAA|16343|4|32|23|AM|third|night|| +16344|AAAAAAAAJNPDAAAA|16344|4|32|24|AM|third|night|| +16345|AAAAAAAAKNPDAAAA|16345|4|32|25|AM|third|night|| +16346|AAAAAAAALNPDAAAA|16346|4|32|26|AM|third|night|| +16347|AAAAAAAAMNPDAAAA|16347|4|32|27|AM|third|night|| +16348|AAAAAAAANNPDAAAA|16348|4|32|28|AM|third|night|| +16349|AAAAAAAAONPDAAAA|16349|4|32|29|AM|third|night|| +16350|AAAAAAAAPNPDAAAA|16350|4|32|30|AM|third|night|| +16351|AAAAAAAAAOPDAAAA|16351|4|32|31|AM|third|night|| +16352|AAAAAAAABOPDAAAA|16352|4|32|32|AM|third|night|| +16353|AAAAAAAACOPDAAAA|16353|4|32|33|AM|third|night|| +16354|AAAAAAAADOPDAAAA|16354|4|32|34|AM|third|night|| +16355|AAAAAAAAEOPDAAAA|16355|4|32|35|AM|third|night|| +16356|AAAAAAAAFOPDAAAA|16356|4|32|36|AM|third|night|| +16357|AAAAAAAAGOPDAAAA|16357|4|32|37|AM|third|night|| +16358|AAAAAAAAHOPDAAAA|16358|4|32|38|AM|third|night|| +16359|AAAAAAAAIOPDAAAA|16359|4|32|39|AM|third|night|| +16360|AAAAAAAAJOPDAAAA|16360|4|32|40|AM|third|night|| +16361|AAAAAAAAKOPDAAAA|16361|4|32|41|AM|third|night|| +16362|AAAAAAAALOPDAAAA|16362|4|32|42|AM|third|night|| +16363|AAAAAAAAMOPDAAAA|16363|4|32|43|AM|third|night|| +16364|AAAAAAAANOPDAAAA|16364|4|32|44|AM|third|night|| +16365|AAAAAAAAOOPDAAAA|16365|4|32|45|AM|third|night|| +16366|AAAAAAAAPOPDAAAA|16366|4|32|46|AM|third|night|| +16367|AAAAAAAAAPPDAAAA|16367|4|32|47|AM|third|night|| +16368|AAAAAAAABPPDAAAA|16368|4|32|48|AM|third|night|| +16369|AAAAAAAACPPDAAAA|16369|4|32|49|AM|third|night|| +16370|AAAAAAAADPPDAAAA|16370|4|32|50|AM|third|night|| +16371|AAAAAAAAEPPDAAAA|16371|4|32|51|AM|third|night|| +16372|AAAAAAAAFPPDAAAA|16372|4|32|52|AM|third|night|| +16373|AAAAAAAAGPPDAAAA|16373|4|32|53|AM|third|night|| +16374|AAAAAAAAHPPDAAAA|16374|4|32|54|AM|third|night|| +16375|AAAAAAAAIPPDAAAA|16375|4|32|55|AM|third|night|| +16376|AAAAAAAAJPPDAAAA|16376|4|32|56|AM|third|night|| +16377|AAAAAAAAKPPDAAAA|16377|4|32|57|AM|third|night|| +16378|AAAAAAAALPPDAAAA|16378|4|32|58|AM|third|night|| +16379|AAAAAAAAMPPDAAAA|16379|4|32|59|AM|third|night|| +16380|AAAAAAAANPPDAAAA|16380|4|33|0|AM|third|night|| +16381|AAAAAAAAOPPDAAAA|16381|4|33|1|AM|third|night|| +16382|AAAAAAAAPPPDAAAA|16382|4|33|2|AM|third|night|| +16383|AAAAAAAAAAAEAAAA|16383|4|33|3|AM|third|night|| +16384|AAAAAAAABAAEAAAA|16384|4|33|4|AM|third|night|| +16385|AAAAAAAACAAEAAAA|16385|4|33|5|AM|third|night|| +16386|AAAAAAAADAAEAAAA|16386|4|33|6|AM|third|night|| +16387|AAAAAAAAEAAEAAAA|16387|4|33|7|AM|third|night|| +16388|AAAAAAAAFAAEAAAA|16388|4|33|8|AM|third|night|| +16389|AAAAAAAAGAAEAAAA|16389|4|33|9|AM|third|night|| +16390|AAAAAAAAHAAEAAAA|16390|4|33|10|AM|third|night|| +16391|AAAAAAAAIAAEAAAA|16391|4|33|11|AM|third|night|| +16392|AAAAAAAAJAAEAAAA|16392|4|33|12|AM|third|night|| +16393|AAAAAAAAKAAEAAAA|16393|4|33|13|AM|third|night|| +16394|AAAAAAAALAAEAAAA|16394|4|33|14|AM|third|night|| +16395|AAAAAAAAMAAEAAAA|16395|4|33|15|AM|third|night|| +16396|AAAAAAAANAAEAAAA|16396|4|33|16|AM|third|night|| +16397|AAAAAAAAOAAEAAAA|16397|4|33|17|AM|third|night|| +16398|AAAAAAAAPAAEAAAA|16398|4|33|18|AM|third|night|| +16399|AAAAAAAAABAEAAAA|16399|4|33|19|AM|third|night|| +16400|AAAAAAAABBAEAAAA|16400|4|33|20|AM|third|night|| +16401|AAAAAAAACBAEAAAA|16401|4|33|21|AM|third|night|| +16402|AAAAAAAADBAEAAAA|16402|4|33|22|AM|third|night|| +16403|AAAAAAAAEBAEAAAA|16403|4|33|23|AM|third|night|| +16404|AAAAAAAAFBAEAAAA|16404|4|33|24|AM|third|night|| +16405|AAAAAAAAGBAEAAAA|16405|4|33|25|AM|third|night|| +16406|AAAAAAAAHBAEAAAA|16406|4|33|26|AM|third|night|| +16407|AAAAAAAAIBAEAAAA|16407|4|33|27|AM|third|night|| +16408|AAAAAAAAJBAEAAAA|16408|4|33|28|AM|third|night|| +16409|AAAAAAAAKBAEAAAA|16409|4|33|29|AM|third|night|| +16410|AAAAAAAALBAEAAAA|16410|4|33|30|AM|third|night|| +16411|AAAAAAAAMBAEAAAA|16411|4|33|31|AM|third|night|| +16412|AAAAAAAANBAEAAAA|16412|4|33|32|AM|third|night|| +16413|AAAAAAAAOBAEAAAA|16413|4|33|33|AM|third|night|| +16414|AAAAAAAAPBAEAAAA|16414|4|33|34|AM|third|night|| +16415|AAAAAAAAACAEAAAA|16415|4|33|35|AM|third|night|| +16416|AAAAAAAABCAEAAAA|16416|4|33|36|AM|third|night|| +16417|AAAAAAAACCAEAAAA|16417|4|33|37|AM|third|night|| +16418|AAAAAAAADCAEAAAA|16418|4|33|38|AM|third|night|| +16419|AAAAAAAAECAEAAAA|16419|4|33|39|AM|third|night|| +16420|AAAAAAAAFCAEAAAA|16420|4|33|40|AM|third|night|| +16421|AAAAAAAAGCAEAAAA|16421|4|33|41|AM|third|night|| +16422|AAAAAAAAHCAEAAAA|16422|4|33|42|AM|third|night|| +16423|AAAAAAAAICAEAAAA|16423|4|33|43|AM|third|night|| +16424|AAAAAAAAJCAEAAAA|16424|4|33|44|AM|third|night|| +16425|AAAAAAAAKCAEAAAA|16425|4|33|45|AM|third|night|| +16426|AAAAAAAALCAEAAAA|16426|4|33|46|AM|third|night|| +16427|AAAAAAAAMCAEAAAA|16427|4|33|47|AM|third|night|| +16428|AAAAAAAANCAEAAAA|16428|4|33|48|AM|third|night|| +16429|AAAAAAAAOCAEAAAA|16429|4|33|49|AM|third|night|| +16430|AAAAAAAAPCAEAAAA|16430|4|33|50|AM|third|night|| +16431|AAAAAAAAADAEAAAA|16431|4|33|51|AM|third|night|| +16432|AAAAAAAABDAEAAAA|16432|4|33|52|AM|third|night|| +16433|AAAAAAAACDAEAAAA|16433|4|33|53|AM|third|night|| +16434|AAAAAAAADDAEAAAA|16434|4|33|54|AM|third|night|| +16435|AAAAAAAAEDAEAAAA|16435|4|33|55|AM|third|night|| +16436|AAAAAAAAFDAEAAAA|16436|4|33|56|AM|third|night|| +16437|AAAAAAAAGDAEAAAA|16437|4|33|57|AM|third|night|| +16438|AAAAAAAAHDAEAAAA|16438|4|33|58|AM|third|night|| +16439|AAAAAAAAIDAEAAAA|16439|4|33|59|AM|third|night|| +16440|AAAAAAAAJDAEAAAA|16440|4|34|0|AM|third|night|| +16441|AAAAAAAAKDAEAAAA|16441|4|34|1|AM|third|night|| +16442|AAAAAAAALDAEAAAA|16442|4|34|2|AM|third|night|| +16443|AAAAAAAAMDAEAAAA|16443|4|34|3|AM|third|night|| +16444|AAAAAAAANDAEAAAA|16444|4|34|4|AM|third|night|| +16445|AAAAAAAAODAEAAAA|16445|4|34|5|AM|third|night|| +16446|AAAAAAAAPDAEAAAA|16446|4|34|6|AM|third|night|| +16447|AAAAAAAAAEAEAAAA|16447|4|34|7|AM|third|night|| +16448|AAAAAAAABEAEAAAA|16448|4|34|8|AM|third|night|| +16449|AAAAAAAACEAEAAAA|16449|4|34|9|AM|third|night|| +16450|AAAAAAAADEAEAAAA|16450|4|34|10|AM|third|night|| +16451|AAAAAAAAEEAEAAAA|16451|4|34|11|AM|third|night|| +16452|AAAAAAAAFEAEAAAA|16452|4|34|12|AM|third|night|| +16453|AAAAAAAAGEAEAAAA|16453|4|34|13|AM|third|night|| +16454|AAAAAAAAHEAEAAAA|16454|4|34|14|AM|third|night|| +16455|AAAAAAAAIEAEAAAA|16455|4|34|15|AM|third|night|| +16456|AAAAAAAAJEAEAAAA|16456|4|34|16|AM|third|night|| +16457|AAAAAAAAKEAEAAAA|16457|4|34|17|AM|third|night|| +16458|AAAAAAAALEAEAAAA|16458|4|34|18|AM|third|night|| +16459|AAAAAAAAMEAEAAAA|16459|4|34|19|AM|third|night|| +16460|AAAAAAAANEAEAAAA|16460|4|34|20|AM|third|night|| +16461|AAAAAAAAOEAEAAAA|16461|4|34|21|AM|third|night|| +16462|AAAAAAAAPEAEAAAA|16462|4|34|22|AM|third|night|| +16463|AAAAAAAAAFAEAAAA|16463|4|34|23|AM|third|night|| +16464|AAAAAAAABFAEAAAA|16464|4|34|24|AM|third|night|| +16465|AAAAAAAACFAEAAAA|16465|4|34|25|AM|third|night|| +16466|AAAAAAAADFAEAAAA|16466|4|34|26|AM|third|night|| +16467|AAAAAAAAEFAEAAAA|16467|4|34|27|AM|third|night|| +16468|AAAAAAAAFFAEAAAA|16468|4|34|28|AM|third|night|| +16469|AAAAAAAAGFAEAAAA|16469|4|34|29|AM|third|night|| +16470|AAAAAAAAHFAEAAAA|16470|4|34|30|AM|third|night|| +16471|AAAAAAAAIFAEAAAA|16471|4|34|31|AM|third|night|| +16472|AAAAAAAAJFAEAAAA|16472|4|34|32|AM|third|night|| +16473|AAAAAAAAKFAEAAAA|16473|4|34|33|AM|third|night|| +16474|AAAAAAAALFAEAAAA|16474|4|34|34|AM|third|night|| +16475|AAAAAAAAMFAEAAAA|16475|4|34|35|AM|third|night|| +16476|AAAAAAAANFAEAAAA|16476|4|34|36|AM|third|night|| +16477|AAAAAAAAOFAEAAAA|16477|4|34|37|AM|third|night|| +16478|AAAAAAAAPFAEAAAA|16478|4|34|38|AM|third|night|| +16479|AAAAAAAAAGAEAAAA|16479|4|34|39|AM|third|night|| +16480|AAAAAAAABGAEAAAA|16480|4|34|40|AM|third|night|| +16481|AAAAAAAACGAEAAAA|16481|4|34|41|AM|third|night|| +16482|AAAAAAAADGAEAAAA|16482|4|34|42|AM|third|night|| +16483|AAAAAAAAEGAEAAAA|16483|4|34|43|AM|third|night|| +16484|AAAAAAAAFGAEAAAA|16484|4|34|44|AM|third|night|| +16485|AAAAAAAAGGAEAAAA|16485|4|34|45|AM|third|night|| +16486|AAAAAAAAHGAEAAAA|16486|4|34|46|AM|third|night|| +16487|AAAAAAAAIGAEAAAA|16487|4|34|47|AM|third|night|| +16488|AAAAAAAAJGAEAAAA|16488|4|34|48|AM|third|night|| +16489|AAAAAAAAKGAEAAAA|16489|4|34|49|AM|third|night|| +16490|AAAAAAAALGAEAAAA|16490|4|34|50|AM|third|night|| +16491|AAAAAAAAMGAEAAAA|16491|4|34|51|AM|third|night|| +16492|AAAAAAAANGAEAAAA|16492|4|34|52|AM|third|night|| +16493|AAAAAAAAOGAEAAAA|16493|4|34|53|AM|third|night|| +16494|AAAAAAAAPGAEAAAA|16494|4|34|54|AM|third|night|| +16495|AAAAAAAAAHAEAAAA|16495|4|34|55|AM|third|night|| +16496|AAAAAAAABHAEAAAA|16496|4|34|56|AM|third|night|| +16497|AAAAAAAACHAEAAAA|16497|4|34|57|AM|third|night|| +16498|AAAAAAAADHAEAAAA|16498|4|34|58|AM|third|night|| +16499|AAAAAAAAEHAEAAAA|16499|4|34|59|AM|third|night|| +16500|AAAAAAAAFHAEAAAA|16500|4|35|0|AM|third|night|| +16501|AAAAAAAAGHAEAAAA|16501|4|35|1|AM|third|night|| +16502|AAAAAAAAHHAEAAAA|16502|4|35|2|AM|third|night|| +16503|AAAAAAAAIHAEAAAA|16503|4|35|3|AM|third|night|| +16504|AAAAAAAAJHAEAAAA|16504|4|35|4|AM|third|night|| +16505|AAAAAAAAKHAEAAAA|16505|4|35|5|AM|third|night|| +16506|AAAAAAAALHAEAAAA|16506|4|35|6|AM|third|night|| +16507|AAAAAAAAMHAEAAAA|16507|4|35|7|AM|third|night|| +16508|AAAAAAAANHAEAAAA|16508|4|35|8|AM|third|night|| +16509|AAAAAAAAOHAEAAAA|16509|4|35|9|AM|third|night|| +16510|AAAAAAAAPHAEAAAA|16510|4|35|10|AM|third|night|| +16511|AAAAAAAAAIAEAAAA|16511|4|35|11|AM|third|night|| +16512|AAAAAAAABIAEAAAA|16512|4|35|12|AM|third|night|| +16513|AAAAAAAACIAEAAAA|16513|4|35|13|AM|third|night|| +16514|AAAAAAAADIAEAAAA|16514|4|35|14|AM|third|night|| +16515|AAAAAAAAEIAEAAAA|16515|4|35|15|AM|third|night|| +16516|AAAAAAAAFIAEAAAA|16516|4|35|16|AM|third|night|| +16517|AAAAAAAAGIAEAAAA|16517|4|35|17|AM|third|night|| +16518|AAAAAAAAHIAEAAAA|16518|4|35|18|AM|third|night|| +16519|AAAAAAAAIIAEAAAA|16519|4|35|19|AM|third|night|| +16520|AAAAAAAAJIAEAAAA|16520|4|35|20|AM|third|night|| +16521|AAAAAAAAKIAEAAAA|16521|4|35|21|AM|third|night|| +16522|AAAAAAAALIAEAAAA|16522|4|35|22|AM|third|night|| +16523|AAAAAAAAMIAEAAAA|16523|4|35|23|AM|third|night|| +16524|AAAAAAAANIAEAAAA|16524|4|35|24|AM|third|night|| +16525|AAAAAAAAOIAEAAAA|16525|4|35|25|AM|third|night|| +16526|AAAAAAAAPIAEAAAA|16526|4|35|26|AM|third|night|| +16527|AAAAAAAAAJAEAAAA|16527|4|35|27|AM|third|night|| +16528|AAAAAAAABJAEAAAA|16528|4|35|28|AM|third|night|| +16529|AAAAAAAACJAEAAAA|16529|4|35|29|AM|third|night|| +16530|AAAAAAAADJAEAAAA|16530|4|35|30|AM|third|night|| +16531|AAAAAAAAEJAEAAAA|16531|4|35|31|AM|third|night|| +16532|AAAAAAAAFJAEAAAA|16532|4|35|32|AM|third|night|| +16533|AAAAAAAAGJAEAAAA|16533|4|35|33|AM|third|night|| +16534|AAAAAAAAHJAEAAAA|16534|4|35|34|AM|third|night|| +16535|AAAAAAAAIJAEAAAA|16535|4|35|35|AM|third|night|| +16536|AAAAAAAAJJAEAAAA|16536|4|35|36|AM|third|night|| +16537|AAAAAAAAKJAEAAAA|16537|4|35|37|AM|third|night|| +16538|AAAAAAAALJAEAAAA|16538|4|35|38|AM|third|night|| +16539|AAAAAAAAMJAEAAAA|16539|4|35|39|AM|third|night|| +16540|AAAAAAAANJAEAAAA|16540|4|35|40|AM|third|night|| +16541|AAAAAAAAOJAEAAAA|16541|4|35|41|AM|third|night|| +16542|AAAAAAAAPJAEAAAA|16542|4|35|42|AM|third|night|| +16543|AAAAAAAAAKAEAAAA|16543|4|35|43|AM|third|night|| +16544|AAAAAAAABKAEAAAA|16544|4|35|44|AM|third|night|| +16545|AAAAAAAACKAEAAAA|16545|4|35|45|AM|third|night|| +16546|AAAAAAAADKAEAAAA|16546|4|35|46|AM|third|night|| +16547|AAAAAAAAEKAEAAAA|16547|4|35|47|AM|third|night|| +16548|AAAAAAAAFKAEAAAA|16548|4|35|48|AM|third|night|| +16549|AAAAAAAAGKAEAAAA|16549|4|35|49|AM|third|night|| +16550|AAAAAAAAHKAEAAAA|16550|4|35|50|AM|third|night|| +16551|AAAAAAAAIKAEAAAA|16551|4|35|51|AM|third|night|| +16552|AAAAAAAAJKAEAAAA|16552|4|35|52|AM|third|night|| +16553|AAAAAAAAKKAEAAAA|16553|4|35|53|AM|third|night|| +16554|AAAAAAAALKAEAAAA|16554|4|35|54|AM|third|night|| +16555|AAAAAAAAMKAEAAAA|16555|4|35|55|AM|third|night|| +16556|AAAAAAAANKAEAAAA|16556|4|35|56|AM|third|night|| +16557|AAAAAAAAOKAEAAAA|16557|4|35|57|AM|third|night|| +16558|AAAAAAAAPKAEAAAA|16558|4|35|58|AM|third|night|| +16559|AAAAAAAAALAEAAAA|16559|4|35|59|AM|third|night|| +16560|AAAAAAAABLAEAAAA|16560|4|36|0|AM|third|night|| +16561|AAAAAAAACLAEAAAA|16561|4|36|1|AM|third|night|| +16562|AAAAAAAADLAEAAAA|16562|4|36|2|AM|third|night|| +16563|AAAAAAAAELAEAAAA|16563|4|36|3|AM|third|night|| +16564|AAAAAAAAFLAEAAAA|16564|4|36|4|AM|third|night|| +16565|AAAAAAAAGLAEAAAA|16565|4|36|5|AM|third|night|| +16566|AAAAAAAAHLAEAAAA|16566|4|36|6|AM|third|night|| +16567|AAAAAAAAILAEAAAA|16567|4|36|7|AM|third|night|| +16568|AAAAAAAAJLAEAAAA|16568|4|36|8|AM|third|night|| +16569|AAAAAAAAKLAEAAAA|16569|4|36|9|AM|third|night|| +16570|AAAAAAAALLAEAAAA|16570|4|36|10|AM|third|night|| +16571|AAAAAAAAMLAEAAAA|16571|4|36|11|AM|third|night|| +16572|AAAAAAAANLAEAAAA|16572|4|36|12|AM|third|night|| +16573|AAAAAAAAOLAEAAAA|16573|4|36|13|AM|third|night|| +16574|AAAAAAAAPLAEAAAA|16574|4|36|14|AM|third|night|| +16575|AAAAAAAAAMAEAAAA|16575|4|36|15|AM|third|night|| +16576|AAAAAAAABMAEAAAA|16576|4|36|16|AM|third|night|| +16577|AAAAAAAACMAEAAAA|16577|4|36|17|AM|third|night|| +16578|AAAAAAAADMAEAAAA|16578|4|36|18|AM|third|night|| +16579|AAAAAAAAEMAEAAAA|16579|4|36|19|AM|third|night|| +16580|AAAAAAAAFMAEAAAA|16580|4|36|20|AM|third|night|| +16581|AAAAAAAAGMAEAAAA|16581|4|36|21|AM|third|night|| +16582|AAAAAAAAHMAEAAAA|16582|4|36|22|AM|third|night|| +16583|AAAAAAAAIMAEAAAA|16583|4|36|23|AM|third|night|| +16584|AAAAAAAAJMAEAAAA|16584|4|36|24|AM|third|night|| +16585|AAAAAAAAKMAEAAAA|16585|4|36|25|AM|third|night|| +16586|AAAAAAAALMAEAAAA|16586|4|36|26|AM|third|night|| +16587|AAAAAAAAMMAEAAAA|16587|4|36|27|AM|third|night|| +16588|AAAAAAAANMAEAAAA|16588|4|36|28|AM|third|night|| +16589|AAAAAAAAOMAEAAAA|16589|4|36|29|AM|third|night|| +16590|AAAAAAAAPMAEAAAA|16590|4|36|30|AM|third|night|| +16591|AAAAAAAAANAEAAAA|16591|4|36|31|AM|third|night|| +16592|AAAAAAAABNAEAAAA|16592|4|36|32|AM|third|night|| +16593|AAAAAAAACNAEAAAA|16593|4|36|33|AM|third|night|| +16594|AAAAAAAADNAEAAAA|16594|4|36|34|AM|third|night|| +16595|AAAAAAAAENAEAAAA|16595|4|36|35|AM|third|night|| +16596|AAAAAAAAFNAEAAAA|16596|4|36|36|AM|third|night|| +16597|AAAAAAAAGNAEAAAA|16597|4|36|37|AM|third|night|| +16598|AAAAAAAAHNAEAAAA|16598|4|36|38|AM|third|night|| +16599|AAAAAAAAINAEAAAA|16599|4|36|39|AM|third|night|| +16600|AAAAAAAAJNAEAAAA|16600|4|36|40|AM|third|night|| +16601|AAAAAAAAKNAEAAAA|16601|4|36|41|AM|third|night|| +16602|AAAAAAAALNAEAAAA|16602|4|36|42|AM|third|night|| +16603|AAAAAAAAMNAEAAAA|16603|4|36|43|AM|third|night|| +16604|AAAAAAAANNAEAAAA|16604|4|36|44|AM|third|night|| +16605|AAAAAAAAONAEAAAA|16605|4|36|45|AM|third|night|| +16606|AAAAAAAAPNAEAAAA|16606|4|36|46|AM|third|night|| +16607|AAAAAAAAAOAEAAAA|16607|4|36|47|AM|third|night|| +16608|AAAAAAAABOAEAAAA|16608|4|36|48|AM|third|night|| +16609|AAAAAAAACOAEAAAA|16609|4|36|49|AM|third|night|| +16610|AAAAAAAADOAEAAAA|16610|4|36|50|AM|third|night|| +16611|AAAAAAAAEOAEAAAA|16611|4|36|51|AM|third|night|| +16612|AAAAAAAAFOAEAAAA|16612|4|36|52|AM|third|night|| +16613|AAAAAAAAGOAEAAAA|16613|4|36|53|AM|third|night|| +16614|AAAAAAAAHOAEAAAA|16614|4|36|54|AM|third|night|| +16615|AAAAAAAAIOAEAAAA|16615|4|36|55|AM|third|night|| +16616|AAAAAAAAJOAEAAAA|16616|4|36|56|AM|third|night|| +16617|AAAAAAAAKOAEAAAA|16617|4|36|57|AM|third|night|| +16618|AAAAAAAALOAEAAAA|16618|4|36|58|AM|third|night|| +16619|AAAAAAAAMOAEAAAA|16619|4|36|59|AM|third|night|| +16620|AAAAAAAANOAEAAAA|16620|4|37|0|AM|third|night|| +16621|AAAAAAAAOOAEAAAA|16621|4|37|1|AM|third|night|| +16622|AAAAAAAAPOAEAAAA|16622|4|37|2|AM|third|night|| +16623|AAAAAAAAAPAEAAAA|16623|4|37|3|AM|third|night|| +16624|AAAAAAAABPAEAAAA|16624|4|37|4|AM|third|night|| +16625|AAAAAAAACPAEAAAA|16625|4|37|5|AM|third|night|| +16626|AAAAAAAADPAEAAAA|16626|4|37|6|AM|third|night|| +16627|AAAAAAAAEPAEAAAA|16627|4|37|7|AM|third|night|| +16628|AAAAAAAAFPAEAAAA|16628|4|37|8|AM|third|night|| +16629|AAAAAAAAGPAEAAAA|16629|4|37|9|AM|third|night|| +16630|AAAAAAAAHPAEAAAA|16630|4|37|10|AM|third|night|| +16631|AAAAAAAAIPAEAAAA|16631|4|37|11|AM|third|night|| +16632|AAAAAAAAJPAEAAAA|16632|4|37|12|AM|third|night|| +16633|AAAAAAAAKPAEAAAA|16633|4|37|13|AM|third|night|| +16634|AAAAAAAALPAEAAAA|16634|4|37|14|AM|third|night|| +16635|AAAAAAAAMPAEAAAA|16635|4|37|15|AM|third|night|| +16636|AAAAAAAANPAEAAAA|16636|4|37|16|AM|third|night|| +16637|AAAAAAAAOPAEAAAA|16637|4|37|17|AM|third|night|| +16638|AAAAAAAAPPAEAAAA|16638|4|37|18|AM|third|night|| +16639|AAAAAAAAAABEAAAA|16639|4|37|19|AM|third|night|| +16640|AAAAAAAABABEAAAA|16640|4|37|20|AM|third|night|| +16641|AAAAAAAACABEAAAA|16641|4|37|21|AM|third|night|| +16642|AAAAAAAADABEAAAA|16642|4|37|22|AM|third|night|| +16643|AAAAAAAAEABEAAAA|16643|4|37|23|AM|third|night|| +16644|AAAAAAAAFABEAAAA|16644|4|37|24|AM|third|night|| +16645|AAAAAAAAGABEAAAA|16645|4|37|25|AM|third|night|| +16646|AAAAAAAAHABEAAAA|16646|4|37|26|AM|third|night|| +16647|AAAAAAAAIABEAAAA|16647|4|37|27|AM|third|night|| +16648|AAAAAAAAJABEAAAA|16648|4|37|28|AM|third|night|| +16649|AAAAAAAAKABEAAAA|16649|4|37|29|AM|third|night|| +16650|AAAAAAAALABEAAAA|16650|4|37|30|AM|third|night|| +16651|AAAAAAAAMABEAAAA|16651|4|37|31|AM|third|night|| +16652|AAAAAAAANABEAAAA|16652|4|37|32|AM|third|night|| +16653|AAAAAAAAOABEAAAA|16653|4|37|33|AM|third|night|| +16654|AAAAAAAAPABEAAAA|16654|4|37|34|AM|third|night|| +16655|AAAAAAAAABBEAAAA|16655|4|37|35|AM|third|night|| +16656|AAAAAAAABBBEAAAA|16656|4|37|36|AM|third|night|| +16657|AAAAAAAACBBEAAAA|16657|4|37|37|AM|third|night|| +16658|AAAAAAAADBBEAAAA|16658|4|37|38|AM|third|night|| +16659|AAAAAAAAEBBEAAAA|16659|4|37|39|AM|third|night|| +16660|AAAAAAAAFBBEAAAA|16660|4|37|40|AM|third|night|| +16661|AAAAAAAAGBBEAAAA|16661|4|37|41|AM|third|night|| +16662|AAAAAAAAHBBEAAAA|16662|4|37|42|AM|third|night|| +16663|AAAAAAAAIBBEAAAA|16663|4|37|43|AM|third|night|| +16664|AAAAAAAAJBBEAAAA|16664|4|37|44|AM|third|night|| +16665|AAAAAAAAKBBEAAAA|16665|4|37|45|AM|third|night|| +16666|AAAAAAAALBBEAAAA|16666|4|37|46|AM|third|night|| +16667|AAAAAAAAMBBEAAAA|16667|4|37|47|AM|third|night|| +16668|AAAAAAAANBBEAAAA|16668|4|37|48|AM|third|night|| +16669|AAAAAAAAOBBEAAAA|16669|4|37|49|AM|third|night|| +16670|AAAAAAAAPBBEAAAA|16670|4|37|50|AM|third|night|| +16671|AAAAAAAAACBEAAAA|16671|4|37|51|AM|third|night|| +16672|AAAAAAAABCBEAAAA|16672|4|37|52|AM|third|night|| +16673|AAAAAAAACCBEAAAA|16673|4|37|53|AM|third|night|| +16674|AAAAAAAADCBEAAAA|16674|4|37|54|AM|third|night|| +16675|AAAAAAAAECBEAAAA|16675|4|37|55|AM|third|night|| +16676|AAAAAAAAFCBEAAAA|16676|4|37|56|AM|third|night|| +16677|AAAAAAAAGCBEAAAA|16677|4|37|57|AM|third|night|| +16678|AAAAAAAAHCBEAAAA|16678|4|37|58|AM|third|night|| +16679|AAAAAAAAICBEAAAA|16679|4|37|59|AM|third|night|| +16680|AAAAAAAAJCBEAAAA|16680|4|38|0|AM|third|night|| +16681|AAAAAAAAKCBEAAAA|16681|4|38|1|AM|third|night|| +16682|AAAAAAAALCBEAAAA|16682|4|38|2|AM|third|night|| +16683|AAAAAAAAMCBEAAAA|16683|4|38|3|AM|third|night|| +16684|AAAAAAAANCBEAAAA|16684|4|38|4|AM|third|night|| +16685|AAAAAAAAOCBEAAAA|16685|4|38|5|AM|third|night|| +16686|AAAAAAAAPCBEAAAA|16686|4|38|6|AM|third|night|| +16687|AAAAAAAAADBEAAAA|16687|4|38|7|AM|third|night|| +16688|AAAAAAAABDBEAAAA|16688|4|38|8|AM|third|night|| +16689|AAAAAAAACDBEAAAA|16689|4|38|9|AM|third|night|| +16690|AAAAAAAADDBEAAAA|16690|4|38|10|AM|third|night|| +16691|AAAAAAAAEDBEAAAA|16691|4|38|11|AM|third|night|| +16692|AAAAAAAAFDBEAAAA|16692|4|38|12|AM|third|night|| +16693|AAAAAAAAGDBEAAAA|16693|4|38|13|AM|third|night|| +16694|AAAAAAAAHDBEAAAA|16694|4|38|14|AM|third|night|| +16695|AAAAAAAAIDBEAAAA|16695|4|38|15|AM|third|night|| +16696|AAAAAAAAJDBEAAAA|16696|4|38|16|AM|third|night|| +16697|AAAAAAAAKDBEAAAA|16697|4|38|17|AM|third|night|| +16698|AAAAAAAALDBEAAAA|16698|4|38|18|AM|third|night|| +16699|AAAAAAAAMDBEAAAA|16699|4|38|19|AM|third|night|| +16700|AAAAAAAANDBEAAAA|16700|4|38|20|AM|third|night|| +16701|AAAAAAAAODBEAAAA|16701|4|38|21|AM|third|night|| +16702|AAAAAAAAPDBEAAAA|16702|4|38|22|AM|third|night|| +16703|AAAAAAAAAEBEAAAA|16703|4|38|23|AM|third|night|| +16704|AAAAAAAABEBEAAAA|16704|4|38|24|AM|third|night|| +16705|AAAAAAAACEBEAAAA|16705|4|38|25|AM|third|night|| +16706|AAAAAAAADEBEAAAA|16706|4|38|26|AM|third|night|| +16707|AAAAAAAAEEBEAAAA|16707|4|38|27|AM|third|night|| +16708|AAAAAAAAFEBEAAAA|16708|4|38|28|AM|third|night|| +16709|AAAAAAAAGEBEAAAA|16709|4|38|29|AM|third|night|| +16710|AAAAAAAAHEBEAAAA|16710|4|38|30|AM|third|night|| +16711|AAAAAAAAIEBEAAAA|16711|4|38|31|AM|third|night|| +16712|AAAAAAAAJEBEAAAA|16712|4|38|32|AM|third|night|| +16713|AAAAAAAAKEBEAAAA|16713|4|38|33|AM|third|night|| +16714|AAAAAAAALEBEAAAA|16714|4|38|34|AM|third|night|| +16715|AAAAAAAAMEBEAAAA|16715|4|38|35|AM|third|night|| +16716|AAAAAAAANEBEAAAA|16716|4|38|36|AM|third|night|| +16717|AAAAAAAAOEBEAAAA|16717|4|38|37|AM|third|night|| +16718|AAAAAAAAPEBEAAAA|16718|4|38|38|AM|third|night|| +16719|AAAAAAAAAFBEAAAA|16719|4|38|39|AM|third|night|| +16720|AAAAAAAABFBEAAAA|16720|4|38|40|AM|third|night|| +16721|AAAAAAAACFBEAAAA|16721|4|38|41|AM|third|night|| +16722|AAAAAAAADFBEAAAA|16722|4|38|42|AM|third|night|| +16723|AAAAAAAAEFBEAAAA|16723|4|38|43|AM|third|night|| +16724|AAAAAAAAFFBEAAAA|16724|4|38|44|AM|third|night|| +16725|AAAAAAAAGFBEAAAA|16725|4|38|45|AM|third|night|| +16726|AAAAAAAAHFBEAAAA|16726|4|38|46|AM|third|night|| +16727|AAAAAAAAIFBEAAAA|16727|4|38|47|AM|third|night|| +16728|AAAAAAAAJFBEAAAA|16728|4|38|48|AM|third|night|| +16729|AAAAAAAAKFBEAAAA|16729|4|38|49|AM|third|night|| +16730|AAAAAAAALFBEAAAA|16730|4|38|50|AM|third|night|| +16731|AAAAAAAAMFBEAAAA|16731|4|38|51|AM|third|night|| +16732|AAAAAAAANFBEAAAA|16732|4|38|52|AM|third|night|| +16733|AAAAAAAAOFBEAAAA|16733|4|38|53|AM|third|night|| +16734|AAAAAAAAPFBEAAAA|16734|4|38|54|AM|third|night|| +16735|AAAAAAAAAGBEAAAA|16735|4|38|55|AM|third|night|| +16736|AAAAAAAABGBEAAAA|16736|4|38|56|AM|third|night|| +16737|AAAAAAAACGBEAAAA|16737|4|38|57|AM|third|night|| +16738|AAAAAAAADGBEAAAA|16738|4|38|58|AM|third|night|| +16739|AAAAAAAAEGBEAAAA|16739|4|38|59|AM|third|night|| +16740|AAAAAAAAFGBEAAAA|16740|4|39|0|AM|third|night|| +16741|AAAAAAAAGGBEAAAA|16741|4|39|1|AM|third|night|| +16742|AAAAAAAAHGBEAAAA|16742|4|39|2|AM|third|night|| +16743|AAAAAAAAIGBEAAAA|16743|4|39|3|AM|third|night|| +16744|AAAAAAAAJGBEAAAA|16744|4|39|4|AM|third|night|| +16745|AAAAAAAAKGBEAAAA|16745|4|39|5|AM|third|night|| +16746|AAAAAAAALGBEAAAA|16746|4|39|6|AM|third|night|| +16747|AAAAAAAAMGBEAAAA|16747|4|39|7|AM|third|night|| +16748|AAAAAAAANGBEAAAA|16748|4|39|8|AM|third|night|| +16749|AAAAAAAAOGBEAAAA|16749|4|39|9|AM|third|night|| +16750|AAAAAAAAPGBEAAAA|16750|4|39|10|AM|third|night|| +16751|AAAAAAAAAHBEAAAA|16751|4|39|11|AM|third|night|| +16752|AAAAAAAABHBEAAAA|16752|4|39|12|AM|third|night|| +16753|AAAAAAAACHBEAAAA|16753|4|39|13|AM|third|night|| +16754|AAAAAAAADHBEAAAA|16754|4|39|14|AM|third|night|| +16755|AAAAAAAAEHBEAAAA|16755|4|39|15|AM|third|night|| +16756|AAAAAAAAFHBEAAAA|16756|4|39|16|AM|third|night|| +16757|AAAAAAAAGHBEAAAA|16757|4|39|17|AM|third|night|| +16758|AAAAAAAAHHBEAAAA|16758|4|39|18|AM|third|night|| +16759|AAAAAAAAIHBEAAAA|16759|4|39|19|AM|third|night|| +16760|AAAAAAAAJHBEAAAA|16760|4|39|20|AM|third|night|| +16761|AAAAAAAAKHBEAAAA|16761|4|39|21|AM|third|night|| +16762|AAAAAAAALHBEAAAA|16762|4|39|22|AM|third|night|| +16763|AAAAAAAAMHBEAAAA|16763|4|39|23|AM|third|night|| +16764|AAAAAAAANHBEAAAA|16764|4|39|24|AM|third|night|| +16765|AAAAAAAAOHBEAAAA|16765|4|39|25|AM|third|night|| +16766|AAAAAAAAPHBEAAAA|16766|4|39|26|AM|third|night|| +16767|AAAAAAAAAIBEAAAA|16767|4|39|27|AM|third|night|| +16768|AAAAAAAABIBEAAAA|16768|4|39|28|AM|third|night|| +16769|AAAAAAAACIBEAAAA|16769|4|39|29|AM|third|night|| +16770|AAAAAAAADIBEAAAA|16770|4|39|30|AM|third|night|| +16771|AAAAAAAAEIBEAAAA|16771|4|39|31|AM|third|night|| +16772|AAAAAAAAFIBEAAAA|16772|4|39|32|AM|third|night|| +16773|AAAAAAAAGIBEAAAA|16773|4|39|33|AM|third|night|| +16774|AAAAAAAAHIBEAAAA|16774|4|39|34|AM|third|night|| +16775|AAAAAAAAIIBEAAAA|16775|4|39|35|AM|third|night|| +16776|AAAAAAAAJIBEAAAA|16776|4|39|36|AM|third|night|| +16777|AAAAAAAAKIBEAAAA|16777|4|39|37|AM|third|night|| +16778|AAAAAAAALIBEAAAA|16778|4|39|38|AM|third|night|| +16779|AAAAAAAAMIBEAAAA|16779|4|39|39|AM|third|night|| +16780|AAAAAAAANIBEAAAA|16780|4|39|40|AM|third|night|| +16781|AAAAAAAAOIBEAAAA|16781|4|39|41|AM|third|night|| +16782|AAAAAAAAPIBEAAAA|16782|4|39|42|AM|third|night|| +16783|AAAAAAAAAJBEAAAA|16783|4|39|43|AM|third|night|| +16784|AAAAAAAABJBEAAAA|16784|4|39|44|AM|third|night|| +16785|AAAAAAAACJBEAAAA|16785|4|39|45|AM|third|night|| +16786|AAAAAAAADJBEAAAA|16786|4|39|46|AM|third|night|| +16787|AAAAAAAAEJBEAAAA|16787|4|39|47|AM|third|night|| +16788|AAAAAAAAFJBEAAAA|16788|4|39|48|AM|third|night|| +16789|AAAAAAAAGJBEAAAA|16789|4|39|49|AM|third|night|| +16790|AAAAAAAAHJBEAAAA|16790|4|39|50|AM|third|night|| +16791|AAAAAAAAIJBEAAAA|16791|4|39|51|AM|third|night|| +16792|AAAAAAAAJJBEAAAA|16792|4|39|52|AM|third|night|| +16793|AAAAAAAAKJBEAAAA|16793|4|39|53|AM|third|night|| +16794|AAAAAAAALJBEAAAA|16794|4|39|54|AM|third|night|| +16795|AAAAAAAAMJBEAAAA|16795|4|39|55|AM|third|night|| +16796|AAAAAAAANJBEAAAA|16796|4|39|56|AM|third|night|| +16797|AAAAAAAAOJBEAAAA|16797|4|39|57|AM|third|night|| +16798|AAAAAAAAPJBEAAAA|16798|4|39|58|AM|third|night|| +16799|AAAAAAAAAKBEAAAA|16799|4|39|59|AM|third|night|| +16800|AAAAAAAABKBEAAAA|16800|4|40|0|AM|third|night|| +16801|AAAAAAAACKBEAAAA|16801|4|40|1|AM|third|night|| +16802|AAAAAAAADKBEAAAA|16802|4|40|2|AM|third|night|| +16803|AAAAAAAAEKBEAAAA|16803|4|40|3|AM|third|night|| +16804|AAAAAAAAFKBEAAAA|16804|4|40|4|AM|third|night|| +16805|AAAAAAAAGKBEAAAA|16805|4|40|5|AM|third|night|| +16806|AAAAAAAAHKBEAAAA|16806|4|40|6|AM|third|night|| +16807|AAAAAAAAIKBEAAAA|16807|4|40|7|AM|third|night|| +16808|AAAAAAAAJKBEAAAA|16808|4|40|8|AM|third|night|| +16809|AAAAAAAAKKBEAAAA|16809|4|40|9|AM|third|night|| +16810|AAAAAAAALKBEAAAA|16810|4|40|10|AM|third|night|| +16811|AAAAAAAAMKBEAAAA|16811|4|40|11|AM|third|night|| +16812|AAAAAAAANKBEAAAA|16812|4|40|12|AM|third|night|| +16813|AAAAAAAAOKBEAAAA|16813|4|40|13|AM|third|night|| +16814|AAAAAAAAPKBEAAAA|16814|4|40|14|AM|third|night|| +16815|AAAAAAAAALBEAAAA|16815|4|40|15|AM|third|night|| +16816|AAAAAAAABLBEAAAA|16816|4|40|16|AM|third|night|| +16817|AAAAAAAACLBEAAAA|16817|4|40|17|AM|third|night|| +16818|AAAAAAAADLBEAAAA|16818|4|40|18|AM|third|night|| +16819|AAAAAAAAELBEAAAA|16819|4|40|19|AM|third|night|| +16820|AAAAAAAAFLBEAAAA|16820|4|40|20|AM|third|night|| +16821|AAAAAAAAGLBEAAAA|16821|4|40|21|AM|third|night|| +16822|AAAAAAAAHLBEAAAA|16822|4|40|22|AM|third|night|| +16823|AAAAAAAAILBEAAAA|16823|4|40|23|AM|third|night|| +16824|AAAAAAAAJLBEAAAA|16824|4|40|24|AM|third|night|| +16825|AAAAAAAAKLBEAAAA|16825|4|40|25|AM|third|night|| +16826|AAAAAAAALLBEAAAA|16826|4|40|26|AM|third|night|| +16827|AAAAAAAAMLBEAAAA|16827|4|40|27|AM|third|night|| +16828|AAAAAAAANLBEAAAA|16828|4|40|28|AM|third|night|| +16829|AAAAAAAAOLBEAAAA|16829|4|40|29|AM|third|night|| +16830|AAAAAAAAPLBEAAAA|16830|4|40|30|AM|third|night|| +16831|AAAAAAAAAMBEAAAA|16831|4|40|31|AM|third|night|| +16832|AAAAAAAABMBEAAAA|16832|4|40|32|AM|third|night|| +16833|AAAAAAAACMBEAAAA|16833|4|40|33|AM|third|night|| +16834|AAAAAAAADMBEAAAA|16834|4|40|34|AM|third|night|| +16835|AAAAAAAAEMBEAAAA|16835|4|40|35|AM|third|night|| +16836|AAAAAAAAFMBEAAAA|16836|4|40|36|AM|third|night|| +16837|AAAAAAAAGMBEAAAA|16837|4|40|37|AM|third|night|| +16838|AAAAAAAAHMBEAAAA|16838|4|40|38|AM|third|night|| +16839|AAAAAAAAIMBEAAAA|16839|4|40|39|AM|third|night|| +16840|AAAAAAAAJMBEAAAA|16840|4|40|40|AM|third|night|| +16841|AAAAAAAAKMBEAAAA|16841|4|40|41|AM|third|night|| +16842|AAAAAAAALMBEAAAA|16842|4|40|42|AM|third|night|| +16843|AAAAAAAAMMBEAAAA|16843|4|40|43|AM|third|night|| +16844|AAAAAAAANMBEAAAA|16844|4|40|44|AM|third|night|| +16845|AAAAAAAAOMBEAAAA|16845|4|40|45|AM|third|night|| +16846|AAAAAAAAPMBEAAAA|16846|4|40|46|AM|third|night|| +16847|AAAAAAAAANBEAAAA|16847|4|40|47|AM|third|night|| +16848|AAAAAAAABNBEAAAA|16848|4|40|48|AM|third|night|| +16849|AAAAAAAACNBEAAAA|16849|4|40|49|AM|third|night|| +16850|AAAAAAAADNBEAAAA|16850|4|40|50|AM|third|night|| +16851|AAAAAAAAENBEAAAA|16851|4|40|51|AM|third|night|| +16852|AAAAAAAAFNBEAAAA|16852|4|40|52|AM|third|night|| +16853|AAAAAAAAGNBEAAAA|16853|4|40|53|AM|third|night|| +16854|AAAAAAAAHNBEAAAA|16854|4|40|54|AM|third|night|| +16855|AAAAAAAAINBEAAAA|16855|4|40|55|AM|third|night|| +16856|AAAAAAAAJNBEAAAA|16856|4|40|56|AM|third|night|| +16857|AAAAAAAAKNBEAAAA|16857|4|40|57|AM|third|night|| +16858|AAAAAAAALNBEAAAA|16858|4|40|58|AM|third|night|| +16859|AAAAAAAAMNBEAAAA|16859|4|40|59|AM|third|night|| +16860|AAAAAAAANNBEAAAA|16860|4|41|0|AM|third|night|| +16861|AAAAAAAAONBEAAAA|16861|4|41|1|AM|third|night|| +16862|AAAAAAAAPNBEAAAA|16862|4|41|2|AM|third|night|| +16863|AAAAAAAAAOBEAAAA|16863|4|41|3|AM|third|night|| +16864|AAAAAAAABOBEAAAA|16864|4|41|4|AM|third|night|| +16865|AAAAAAAACOBEAAAA|16865|4|41|5|AM|third|night|| +16866|AAAAAAAADOBEAAAA|16866|4|41|6|AM|third|night|| +16867|AAAAAAAAEOBEAAAA|16867|4|41|7|AM|third|night|| +16868|AAAAAAAAFOBEAAAA|16868|4|41|8|AM|third|night|| +16869|AAAAAAAAGOBEAAAA|16869|4|41|9|AM|third|night|| +16870|AAAAAAAAHOBEAAAA|16870|4|41|10|AM|third|night|| +16871|AAAAAAAAIOBEAAAA|16871|4|41|11|AM|third|night|| +16872|AAAAAAAAJOBEAAAA|16872|4|41|12|AM|third|night|| +16873|AAAAAAAAKOBEAAAA|16873|4|41|13|AM|third|night|| +16874|AAAAAAAALOBEAAAA|16874|4|41|14|AM|third|night|| +16875|AAAAAAAAMOBEAAAA|16875|4|41|15|AM|third|night|| +16876|AAAAAAAANOBEAAAA|16876|4|41|16|AM|third|night|| +16877|AAAAAAAAOOBEAAAA|16877|4|41|17|AM|third|night|| +16878|AAAAAAAAPOBEAAAA|16878|4|41|18|AM|third|night|| +16879|AAAAAAAAAPBEAAAA|16879|4|41|19|AM|third|night|| +16880|AAAAAAAABPBEAAAA|16880|4|41|20|AM|third|night|| +16881|AAAAAAAACPBEAAAA|16881|4|41|21|AM|third|night|| +16882|AAAAAAAADPBEAAAA|16882|4|41|22|AM|third|night|| +16883|AAAAAAAAEPBEAAAA|16883|4|41|23|AM|third|night|| +16884|AAAAAAAAFPBEAAAA|16884|4|41|24|AM|third|night|| +16885|AAAAAAAAGPBEAAAA|16885|4|41|25|AM|third|night|| +16886|AAAAAAAAHPBEAAAA|16886|4|41|26|AM|third|night|| +16887|AAAAAAAAIPBEAAAA|16887|4|41|27|AM|third|night|| +16888|AAAAAAAAJPBEAAAA|16888|4|41|28|AM|third|night|| +16889|AAAAAAAAKPBEAAAA|16889|4|41|29|AM|third|night|| +16890|AAAAAAAALPBEAAAA|16890|4|41|30|AM|third|night|| +16891|AAAAAAAAMPBEAAAA|16891|4|41|31|AM|third|night|| +16892|AAAAAAAANPBEAAAA|16892|4|41|32|AM|third|night|| +16893|AAAAAAAAOPBEAAAA|16893|4|41|33|AM|third|night|| +16894|AAAAAAAAPPBEAAAA|16894|4|41|34|AM|third|night|| +16895|AAAAAAAAAACEAAAA|16895|4|41|35|AM|third|night|| +16896|AAAAAAAABACEAAAA|16896|4|41|36|AM|third|night|| +16897|AAAAAAAACACEAAAA|16897|4|41|37|AM|third|night|| +16898|AAAAAAAADACEAAAA|16898|4|41|38|AM|third|night|| +16899|AAAAAAAAEACEAAAA|16899|4|41|39|AM|third|night|| +16900|AAAAAAAAFACEAAAA|16900|4|41|40|AM|third|night|| +16901|AAAAAAAAGACEAAAA|16901|4|41|41|AM|third|night|| +16902|AAAAAAAAHACEAAAA|16902|4|41|42|AM|third|night|| +16903|AAAAAAAAIACEAAAA|16903|4|41|43|AM|third|night|| +16904|AAAAAAAAJACEAAAA|16904|4|41|44|AM|third|night|| +16905|AAAAAAAAKACEAAAA|16905|4|41|45|AM|third|night|| +16906|AAAAAAAALACEAAAA|16906|4|41|46|AM|third|night|| +16907|AAAAAAAAMACEAAAA|16907|4|41|47|AM|third|night|| +16908|AAAAAAAANACEAAAA|16908|4|41|48|AM|third|night|| +16909|AAAAAAAAOACEAAAA|16909|4|41|49|AM|third|night|| +16910|AAAAAAAAPACEAAAA|16910|4|41|50|AM|third|night|| +16911|AAAAAAAAABCEAAAA|16911|4|41|51|AM|third|night|| +16912|AAAAAAAABBCEAAAA|16912|4|41|52|AM|third|night|| +16913|AAAAAAAACBCEAAAA|16913|4|41|53|AM|third|night|| +16914|AAAAAAAADBCEAAAA|16914|4|41|54|AM|third|night|| +16915|AAAAAAAAEBCEAAAA|16915|4|41|55|AM|third|night|| +16916|AAAAAAAAFBCEAAAA|16916|4|41|56|AM|third|night|| +16917|AAAAAAAAGBCEAAAA|16917|4|41|57|AM|third|night|| +16918|AAAAAAAAHBCEAAAA|16918|4|41|58|AM|third|night|| +16919|AAAAAAAAIBCEAAAA|16919|4|41|59|AM|third|night|| +16920|AAAAAAAAJBCEAAAA|16920|4|42|0|AM|third|night|| +16921|AAAAAAAAKBCEAAAA|16921|4|42|1|AM|third|night|| +16922|AAAAAAAALBCEAAAA|16922|4|42|2|AM|third|night|| +16923|AAAAAAAAMBCEAAAA|16923|4|42|3|AM|third|night|| +16924|AAAAAAAANBCEAAAA|16924|4|42|4|AM|third|night|| +16925|AAAAAAAAOBCEAAAA|16925|4|42|5|AM|third|night|| +16926|AAAAAAAAPBCEAAAA|16926|4|42|6|AM|third|night|| +16927|AAAAAAAAACCEAAAA|16927|4|42|7|AM|third|night|| +16928|AAAAAAAABCCEAAAA|16928|4|42|8|AM|third|night|| +16929|AAAAAAAACCCEAAAA|16929|4|42|9|AM|third|night|| +16930|AAAAAAAADCCEAAAA|16930|4|42|10|AM|third|night|| +16931|AAAAAAAAECCEAAAA|16931|4|42|11|AM|third|night|| +16932|AAAAAAAAFCCEAAAA|16932|4|42|12|AM|third|night|| +16933|AAAAAAAAGCCEAAAA|16933|4|42|13|AM|third|night|| +16934|AAAAAAAAHCCEAAAA|16934|4|42|14|AM|third|night|| +16935|AAAAAAAAICCEAAAA|16935|4|42|15|AM|third|night|| +16936|AAAAAAAAJCCEAAAA|16936|4|42|16|AM|third|night|| +16937|AAAAAAAAKCCEAAAA|16937|4|42|17|AM|third|night|| +16938|AAAAAAAALCCEAAAA|16938|4|42|18|AM|third|night|| +16939|AAAAAAAAMCCEAAAA|16939|4|42|19|AM|third|night|| +16940|AAAAAAAANCCEAAAA|16940|4|42|20|AM|third|night|| +16941|AAAAAAAAOCCEAAAA|16941|4|42|21|AM|third|night|| +16942|AAAAAAAAPCCEAAAA|16942|4|42|22|AM|third|night|| +16943|AAAAAAAAADCEAAAA|16943|4|42|23|AM|third|night|| +16944|AAAAAAAABDCEAAAA|16944|4|42|24|AM|third|night|| +16945|AAAAAAAACDCEAAAA|16945|4|42|25|AM|third|night|| +16946|AAAAAAAADDCEAAAA|16946|4|42|26|AM|third|night|| +16947|AAAAAAAAEDCEAAAA|16947|4|42|27|AM|third|night|| +16948|AAAAAAAAFDCEAAAA|16948|4|42|28|AM|third|night|| +16949|AAAAAAAAGDCEAAAA|16949|4|42|29|AM|third|night|| +16950|AAAAAAAAHDCEAAAA|16950|4|42|30|AM|third|night|| +16951|AAAAAAAAIDCEAAAA|16951|4|42|31|AM|third|night|| +16952|AAAAAAAAJDCEAAAA|16952|4|42|32|AM|third|night|| +16953|AAAAAAAAKDCEAAAA|16953|4|42|33|AM|third|night|| +16954|AAAAAAAALDCEAAAA|16954|4|42|34|AM|third|night|| +16955|AAAAAAAAMDCEAAAA|16955|4|42|35|AM|third|night|| +16956|AAAAAAAANDCEAAAA|16956|4|42|36|AM|third|night|| +16957|AAAAAAAAODCEAAAA|16957|4|42|37|AM|third|night|| +16958|AAAAAAAAPDCEAAAA|16958|4|42|38|AM|third|night|| +16959|AAAAAAAAAECEAAAA|16959|4|42|39|AM|third|night|| +16960|AAAAAAAABECEAAAA|16960|4|42|40|AM|third|night|| +16961|AAAAAAAACECEAAAA|16961|4|42|41|AM|third|night|| +16962|AAAAAAAADECEAAAA|16962|4|42|42|AM|third|night|| +16963|AAAAAAAAEECEAAAA|16963|4|42|43|AM|third|night|| +16964|AAAAAAAAFECEAAAA|16964|4|42|44|AM|third|night|| +16965|AAAAAAAAGECEAAAA|16965|4|42|45|AM|third|night|| +16966|AAAAAAAAHECEAAAA|16966|4|42|46|AM|third|night|| +16967|AAAAAAAAIECEAAAA|16967|4|42|47|AM|third|night|| +16968|AAAAAAAAJECEAAAA|16968|4|42|48|AM|third|night|| +16969|AAAAAAAAKECEAAAA|16969|4|42|49|AM|third|night|| +16970|AAAAAAAALECEAAAA|16970|4|42|50|AM|third|night|| +16971|AAAAAAAAMECEAAAA|16971|4|42|51|AM|third|night|| +16972|AAAAAAAANECEAAAA|16972|4|42|52|AM|third|night|| +16973|AAAAAAAAOECEAAAA|16973|4|42|53|AM|third|night|| +16974|AAAAAAAAPECEAAAA|16974|4|42|54|AM|third|night|| +16975|AAAAAAAAAFCEAAAA|16975|4|42|55|AM|third|night|| +16976|AAAAAAAABFCEAAAA|16976|4|42|56|AM|third|night|| +16977|AAAAAAAACFCEAAAA|16977|4|42|57|AM|third|night|| +16978|AAAAAAAADFCEAAAA|16978|4|42|58|AM|third|night|| +16979|AAAAAAAAEFCEAAAA|16979|4|42|59|AM|third|night|| +16980|AAAAAAAAFFCEAAAA|16980|4|43|0|AM|third|night|| +16981|AAAAAAAAGFCEAAAA|16981|4|43|1|AM|third|night|| +16982|AAAAAAAAHFCEAAAA|16982|4|43|2|AM|third|night|| +16983|AAAAAAAAIFCEAAAA|16983|4|43|3|AM|third|night|| +16984|AAAAAAAAJFCEAAAA|16984|4|43|4|AM|third|night|| +16985|AAAAAAAAKFCEAAAA|16985|4|43|5|AM|third|night|| +16986|AAAAAAAALFCEAAAA|16986|4|43|6|AM|third|night|| +16987|AAAAAAAAMFCEAAAA|16987|4|43|7|AM|third|night|| +16988|AAAAAAAANFCEAAAA|16988|4|43|8|AM|third|night|| +16989|AAAAAAAAOFCEAAAA|16989|4|43|9|AM|third|night|| +16990|AAAAAAAAPFCEAAAA|16990|4|43|10|AM|third|night|| +16991|AAAAAAAAAGCEAAAA|16991|4|43|11|AM|third|night|| +16992|AAAAAAAABGCEAAAA|16992|4|43|12|AM|third|night|| +16993|AAAAAAAACGCEAAAA|16993|4|43|13|AM|third|night|| +16994|AAAAAAAADGCEAAAA|16994|4|43|14|AM|third|night|| +16995|AAAAAAAAEGCEAAAA|16995|4|43|15|AM|third|night|| +16996|AAAAAAAAFGCEAAAA|16996|4|43|16|AM|third|night|| +16997|AAAAAAAAGGCEAAAA|16997|4|43|17|AM|third|night|| +16998|AAAAAAAAHGCEAAAA|16998|4|43|18|AM|third|night|| +16999|AAAAAAAAIGCEAAAA|16999|4|43|19|AM|third|night|| +17000|AAAAAAAAJGCEAAAA|17000|4|43|20|AM|third|night|| +17001|AAAAAAAAKGCEAAAA|17001|4|43|21|AM|third|night|| +17002|AAAAAAAALGCEAAAA|17002|4|43|22|AM|third|night|| +17003|AAAAAAAAMGCEAAAA|17003|4|43|23|AM|third|night|| +17004|AAAAAAAANGCEAAAA|17004|4|43|24|AM|third|night|| +17005|AAAAAAAAOGCEAAAA|17005|4|43|25|AM|third|night|| +17006|AAAAAAAAPGCEAAAA|17006|4|43|26|AM|third|night|| +17007|AAAAAAAAAHCEAAAA|17007|4|43|27|AM|third|night|| +17008|AAAAAAAABHCEAAAA|17008|4|43|28|AM|third|night|| +17009|AAAAAAAACHCEAAAA|17009|4|43|29|AM|third|night|| +17010|AAAAAAAADHCEAAAA|17010|4|43|30|AM|third|night|| +17011|AAAAAAAAEHCEAAAA|17011|4|43|31|AM|third|night|| +17012|AAAAAAAAFHCEAAAA|17012|4|43|32|AM|third|night|| +17013|AAAAAAAAGHCEAAAA|17013|4|43|33|AM|third|night|| +17014|AAAAAAAAHHCEAAAA|17014|4|43|34|AM|third|night|| +17015|AAAAAAAAIHCEAAAA|17015|4|43|35|AM|third|night|| +17016|AAAAAAAAJHCEAAAA|17016|4|43|36|AM|third|night|| +17017|AAAAAAAAKHCEAAAA|17017|4|43|37|AM|third|night|| +17018|AAAAAAAALHCEAAAA|17018|4|43|38|AM|third|night|| +17019|AAAAAAAAMHCEAAAA|17019|4|43|39|AM|third|night|| +17020|AAAAAAAANHCEAAAA|17020|4|43|40|AM|third|night|| +17021|AAAAAAAAOHCEAAAA|17021|4|43|41|AM|third|night|| +17022|AAAAAAAAPHCEAAAA|17022|4|43|42|AM|third|night|| +17023|AAAAAAAAAICEAAAA|17023|4|43|43|AM|third|night|| +17024|AAAAAAAABICEAAAA|17024|4|43|44|AM|third|night|| +17025|AAAAAAAACICEAAAA|17025|4|43|45|AM|third|night|| +17026|AAAAAAAADICEAAAA|17026|4|43|46|AM|third|night|| +17027|AAAAAAAAEICEAAAA|17027|4|43|47|AM|third|night|| +17028|AAAAAAAAFICEAAAA|17028|4|43|48|AM|third|night|| +17029|AAAAAAAAGICEAAAA|17029|4|43|49|AM|third|night|| +17030|AAAAAAAAHICEAAAA|17030|4|43|50|AM|third|night|| +17031|AAAAAAAAIICEAAAA|17031|4|43|51|AM|third|night|| +17032|AAAAAAAAJICEAAAA|17032|4|43|52|AM|third|night|| +17033|AAAAAAAAKICEAAAA|17033|4|43|53|AM|third|night|| +17034|AAAAAAAALICEAAAA|17034|4|43|54|AM|third|night|| +17035|AAAAAAAAMICEAAAA|17035|4|43|55|AM|third|night|| +17036|AAAAAAAANICEAAAA|17036|4|43|56|AM|third|night|| +17037|AAAAAAAAOICEAAAA|17037|4|43|57|AM|third|night|| +17038|AAAAAAAAPICEAAAA|17038|4|43|58|AM|third|night|| +17039|AAAAAAAAAJCEAAAA|17039|4|43|59|AM|third|night|| +17040|AAAAAAAABJCEAAAA|17040|4|44|0|AM|third|night|| +17041|AAAAAAAACJCEAAAA|17041|4|44|1|AM|third|night|| +17042|AAAAAAAADJCEAAAA|17042|4|44|2|AM|third|night|| +17043|AAAAAAAAEJCEAAAA|17043|4|44|3|AM|third|night|| +17044|AAAAAAAAFJCEAAAA|17044|4|44|4|AM|third|night|| +17045|AAAAAAAAGJCEAAAA|17045|4|44|5|AM|third|night|| +17046|AAAAAAAAHJCEAAAA|17046|4|44|6|AM|third|night|| +17047|AAAAAAAAIJCEAAAA|17047|4|44|7|AM|third|night|| +17048|AAAAAAAAJJCEAAAA|17048|4|44|8|AM|third|night|| +17049|AAAAAAAAKJCEAAAA|17049|4|44|9|AM|third|night|| +17050|AAAAAAAALJCEAAAA|17050|4|44|10|AM|third|night|| +17051|AAAAAAAAMJCEAAAA|17051|4|44|11|AM|third|night|| +17052|AAAAAAAANJCEAAAA|17052|4|44|12|AM|third|night|| +17053|AAAAAAAAOJCEAAAA|17053|4|44|13|AM|third|night|| +17054|AAAAAAAAPJCEAAAA|17054|4|44|14|AM|third|night|| +17055|AAAAAAAAAKCEAAAA|17055|4|44|15|AM|third|night|| +17056|AAAAAAAABKCEAAAA|17056|4|44|16|AM|third|night|| +17057|AAAAAAAACKCEAAAA|17057|4|44|17|AM|third|night|| +17058|AAAAAAAADKCEAAAA|17058|4|44|18|AM|third|night|| +17059|AAAAAAAAEKCEAAAA|17059|4|44|19|AM|third|night|| +17060|AAAAAAAAFKCEAAAA|17060|4|44|20|AM|third|night|| +17061|AAAAAAAAGKCEAAAA|17061|4|44|21|AM|third|night|| +17062|AAAAAAAAHKCEAAAA|17062|4|44|22|AM|third|night|| +17063|AAAAAAAAIKCEAAAA|17063|4|44|23|AM|third|night|| +17064|AAAAAAAAJKCEAAAA|17064|4|44|24|AM|third|night|| +17065|AAAAAAAAKKCEAAAA|17065|4|44|25|AM|third|night|| +17066|AAAAAAAALKCEAAAA|17066|4|44|26|AM|third|night|| +17067|AAAAAAAAMKCEAAAA|17067|4|44|27|AM|third|night|| +17068|AAAAAAAANKCEAAAA|17068|4|44|28|AM|third|night|| +17069|AAAAAAAAOKCEAAAA|17069|4|44|29|AM|third|night|| +17070|AAAAAAAAPKCEAAAA|17070|4|44|30|AM|third|night|| +17071|AAAAAAAAALCEAAAA|17071|4|44|31|AM|third|night|| +17072|AAAAAAAABLCEAAAA|17072|4|44|32|AM|third|night|| +17073|AAAAAAAACLCEAAAA|17073|4|44|33|AM|third|night|| +17074|AAAAAAAADLCEAAAA|17074|4|44|34|AM|third|night|| +17075|AAAAAAAAELCEAAAA|17075|4|44|35|AM|third|night|| +17076|AAAAAAAAFLCEAAAA|17076|4|44|36|AM|third|night|| +17077|AAAAAAAAGLCEAAAA|17077|4|44|37|AM|third|night|| +17078|AAAAAAAAHLCEAAAA|17078|4|44|38|AM|third|night|| +17079|AAAAAAAAILCEAAAA|17079|4|44|39|AM|third|night|| +17080|AAAAAAAAJLCEAAAA|17080|4|44|40|AM|third|night|| +17081|AAAAAAAAKLCEAAAA|17081|4|44|41|AM|third|night|| +17082|AAAAAAAALLCEAAAA|17082|4|44|42|AM|third|night|| +17083|AAAAAAAAMLCEAAAA|17083|4|44|43|AM|third|night|| +17084|AAAAAAAANLCEAAAA|17084|4|44|44|AM|third|night|| +17085|AAAAAAAAOLCEAAAA|17085|4|44|45|AM|third|night|| +17086|AAAAAAAAPLCEAAAA|17086|4|44|46|AM|third|night|| +17087|AAAAAAAAAMCEAAAA|17087|4|44|47|AM|third|night|| +17088|AAAAAAAABMCEAAAA|17088|4|44|48|AM|third|night|| +17089|AAAAAAAACMCEAAAA|17089|4|44|49|AM|third|night|| +17090|AAAAAAAADMCEAAAA|17090|4|44|50|AM|third|night|| +17091|AAAAAAAAEMCEAAAA|17091|4|44|51|AM|third|night|| +17092|AAAAAAAAFMCEAAAA|17092|4|44|52|AM|third|night|| +17093|AAAAAAAAGMCEAAAA|17093|4|44|53|AM|third|night|| +17094|AAAAAAAAHMCEAAAA|17094|4|44|54|AM|third|night|| +17095|AAAAAAAAIMCEAAAA|17095|4|44|55|AM|third|night|| +17096|AAAAAAAAJMCEAAAA|17096|4|44|56|AM|third|night|| +17097|AAAAAAAAKMCEAAAA|17097|4|44|57|AM|third|night|| +17098|AAAAAAAALMCEAAAA|17098|4|44|58|AM|third|night|| +17099|AAAAAAAAMMCEAAAA|17099|4|44|59|AM|third|night|| +17100|AAAAAAAANMCEAAAA|17100|4|45|0|AM|third|night|| +17101|AAAAAAAAOMCEAAAA|17101|4|45|1|AM|third|night|| +17102|AAAAAAAAPMCEAAAA|17102|4|45|2|AM|third|night|| +17103|AAAAAAAAANCEAAAA|17103|4|45|3|AM|third|night|| +17104|AAAAAAAABNCEAAAA|17104|4|45|4|AM|third|night|| +17105|AAAAAAAACNCEAAAA|17105|4|45|5|AM|third|night|| +17106|AAAAAAAADNCEAAAA|17106|4|45|6|AM|third|night|| +17107|AAAAAAAAENCEAAAA|17107|4|45|7|AM|third|night|| +17108|AAAAAAAAFNCEAAAA|17108|4|45|8|AM|third|night|| +17109|AAAAAAAAGNCEAAAA|17109|4|45|9|AM|third|night|| +17110|AAAAAAAAHNCEAAAA|17110|4|45|10|AM|third|night|| +17111|AAAAAAAAINCEAAAA|17111|4|45|11|AM|third|night|| +17112|AAAAAAAAJNCEAAAA|17112|4|45|12|AM|third|night|| +17113|AAAAAAAAKNCEAAAA|17113|4|45|13|AM|third|night|| +17114|AAAAAAAALNCEAAAA|17114|4|45|14|AM|third|night|| +17115|AAAAAAAAMNCEAAAA|17115|4|45|15|AM|third|night|| +17116|AAAAAAAANNCEAAAA|17116|4|45|16|AM|third|night|| +17117|AAAAAAAAONCEAAAA|17117|4|45|17|AM|third|night|| +17118|AAAAAAAAPNCEAAAA|17118|4|45|18|AM|third|night|| +17119|AAAAAAAAAOCEAAAA|17119|4|45|19|AM|third|night|| +17120|AAAAAAAABOCEAAAA|17120|4|45|20|AM|third|night|| +17121|AAAAAAAACOCEAAAA|17121|4|45|21|AM|third|night|| +17122|AAAAAAAADOCEAAAA|17122|4|45|22|AM|third|night|| +17123|AAAAAAAAEOCEAAAA|17123|4|45|23|AM|third|night|| +17124|AAAAAAAAFOCEAAAA|17124|4|45|24|AM|third|night|| +17125|AAAAAAAAGOCEAAAA|17125|4|45|25|AM|third|night|| +17126|AAAAAAAAHOCEAAAA|17126|4|45|26|AM|third|night|| +17127|AAAAAAAAIOCEAAAA|17127|4|45|27|AM|third|night|| +17128|AAAAAAAAJOCEAAAA|17128|4|45|28|AM|third|night|| +17129|AAAAAAAAKOCEAAAA|17129|4|45|29|AM|third|night|| +17130|AAAAAAAALOCEAAAA|17130|4|45|30|AM|third|night|| +17131|AAAAAAAAMOCEAAAA|17131|4|45|31|AM|third|night|| +17132|AAAAAAAANOCEAAAA|17132|4|45|32|AM|third|night|| +17133|AAAAAAAAOOCEAAAA|17133|4|45|33|AM|third|night|| +17134|AAAAAAAAPOCEAAAA|17134|4|45|34|AM|third|night|| +17135|AAAAAAAAAPCEAAAA|17135|4|45|35|AM|third|night|| +17136|AAAAAAAABPCEAAAA|17136|4|45|36|AM|third|night|| +17137|AAAAAAAACPCEAAAA|17137|4|45|37|AM|third|night|| +17138|AAAAAAAADPCEAAAA|17138|4|45|38|AM|third|night|| +17139|AAAAAAAAEPCEAAAA|17139|4|45|39|AM|third|night|| +17140|AAAAAAAAFPCEAAAA|17140|4|45|40|AM|third|night|| +17141|AAAAAAAAGPCEAAAA|17141|4|45|41|AM|third|night|| +17142|AAAAAAAAHPCEAAAA|17142|4|45|42|AM|third|night|| +17143|AAAAAAAAIPCEAAAA|17143|4|45|43|AM|third|night|| +17144|AAAAAAAAJPCEAAAA|17144|4|45|44|AM|third|night|| +17145|AAAAAAAAKPCEAAAA|17145|4|45|45|AM|third|night|| +17146|AAAAAAAALPCEAAAA|17146|4|45|46|AM|third|night|| +17147|AAAAAAAAMPCEAAAA|17147|4|45|47|AM|third|night|| +17148|AAAAAAAANPCEAAAA|17148|4|45|48|AM|third|night|| +17149|AAAAAAAAOPCEAAAA|17149|4|45|49|AM|third|night|| +17150|AAAAAAAAPPCEAAAA|17150|4|45|50|AM|third|night|| +17151|AAAAAAAAAADEAAAA|17151|4|45|51|AM|third|night|| +17152|AAAAAAAABADEAAAA|17152|4|45|52|AM|third|night|| +17153|AAAAAAAACADEAAAA|17153|4|45|53|AM|third|night|| +17154|AAAAAAAADADEAAAA|17154|4|45|54|AM|third|night|| +17155|AAAAAAAAEADEAAAA|17155|4|45|55|AM|third|night|| +17156|AAAAAAAAFADEAAAA|17156|4|45|56|AM|third|night|| +17157|AAAAAAAAGADEAAAA|17157|4|45|57|AM|third|night|| +17158|AAAAAAAAHADEAAAA|17158|4|45|58|AM|third|night|| +17159|AAAAAAAAIADEAAAA|17159|4|45|59|AM|third|night|| +17160|AAAAAAAAJADEAAAA|17160|4|46|0|AM|third|night|| +17161|AAAAAAAAKADEAAAA|17161|4|46|1|AM|third|night|| +17162|AAAAAAAALADEAAAA|17162|4|46|2|AM|third|night|| +17163|AAAAAAAAMADEAAAA|17163|4|46|3|AM|third|night|| +17164|AAAAAAAANADEAAAA|17164|4|46|4|AM|third|night|| +17165|AAAAAAAAOADEAAAA|17165|4|46|5|AM|third|night|| +17166|AAAAAAAAPADEAAAA|17166|4|46|6|AM|third|night|| +17167|AAAAAAAAABDEAAAA|17167|4|46|7|AM|third|night|| +17168|AAAAAAAABBDEAAAA|17168|4|46|8|AM|third|night|| +17169|AAAAAAAACBDEAAAA|17169|4|46|9|AM|third|night|| +17170|AAAAAAAADBDEAAAA|17170|4|46|10|AM|third|night|| +17171|AAAAAAAAEBDEAAAA|17171|4|46|11|AM|third|night|| +17172|AAAAAAAAFBDEAAAA|17172|4|46|12|AM|third|night|| +17173|AAAAAAAAGBDEAAAA|17173|4|46|13|AM|third|night|| +17174|AAAAAAAAHBDEAAAA|17174|4|46|14|AM|third|night|| +17175|AAAAAAAAIBDEAAAA|17175|4|46|15|AM|third|night|| +17176|AAAAAAAAJBDEAAAA|17176|4|46|16|AM|third|night|| +17177|AAAAAAAAKBDEAAAA|17177|4|46|17|AM|third|night|| +17178|AAAAAAAALBDEAAAA|17178|4|46|18|AM|third|night|| +17179|AAAAAAAAMBDEAAAA|17179|4|46|19|AM|third|night|| +17180|AAAAAAAANBDEAAAA|17180|4|46|20|AM|third|night|| +17181|AAAAAAAAOBDEAAAA|17181|4|46|21|AM|third|night|| +17182|AAAAAAAAPBDEAAAA|17182|4|46|22|AM|third|night|| +17183|AAAAAAAAACDEAAAA|17183|4|46|23|AM|third|night|| +17184|AAAAAAAABCDEAAAA|17184|4|46|24|AM|third|night|| +17185|AAAAAAAACCDEAAAA|17185|4|46|25|AM|third|night|| +17186|AAAAAAAADCDEAAAA|17186|4|46|26|AM|third|night|| +17187|AAAAAAAAECDEAAAA|17187|4|46|27|AM|third|night|| +17188|AAAAAAAAFCDEAAAA|17188|4|46|28|AM|third|night|| +17189|AAAAAAAAGCDEAAAA|17189|4|46|29|AM|third|night|| +17190|AAAAAAAAHCDEAAAA|17190|4|46|30|AM|third|night|| +17191|AAAAAAAAICDEAAAA|17191|4|46|31|AM|third|night|| +17192|AAAAAAAAJCDEAAAA|17192|4|46|32|AM|third|night|| +17193|AAAAAAAAKCDEAAAA|17193|4|46|33|AM|third|night|| +17194|AAAAAAAALCDEAAAA|17194|4|46|34|AM|third|night|| +17195|AAAAAAAAMCDEAAAA|17195|4|46|35|AM|third|night|| +17196|AAAAAAAANCDEAAAA|17196|4|46|36|AM|third|night|| +17197|AAAAAAAAOCDEAAAA|17197|4|46|37|AM|third|night|| +17198|AAAAAAAAPCDEAAAA|17198|4|46|38|AM|third|night|| +17199|AAAAAAAAADDEAAAA|17199|4|46|39|AM|third|night|| +17200|AAAAAAAABDDEAAAA|17200|4|46|40|AM|third|night|| +17201|AAAAAAAACDDEAAAA|17201|4|46|41|AM|third|night|| +17202|AAAAAAAADDDEAAAA|17202|4|46|42|AM|third|night|| +17203|AAAAAAAAEDDEAAAA|17203|4|46|43|AM|third|night|| +17204|AAAAAAAAFDDEAAAA|17204|4|46|44|AM|third|night|| +17205|AAAAAAAAGDDEAAAA|17205|4|46|45|AM|third|night|| +17206|AAAAAAAAHDDEAAAA|17206|4|46|46|AM|third|night|| +17207|AAAAAAAAIDDEAAAA|17207|4|46|47|AM|third|night|| +17208|AAAAAAAAJDDEAAAA|17208|4|46|48|AM|third|night|| +17209|AAAAAAAAKDDEAAAA|17209|4|46|49|AM|third|night|| +17210|AAAAAAAALDDEAAAA|17210|4|46|50|AM|third|night|| +17211|AAAAAAAAMDDEAAAA|17211|4|46|51|AM|third|night|| +17212|AAAAAAAANDDEAAAA|17212|4|46|52|AM|third|night|| +17213|AAAAAAAAODDEAAAA|17213|4|46|53|AM|third|night|| +17214|AAAAAAAAPDDEAAAA|17214|4|46|54|AM|third|night|| +17215|AAAAAAAAAEDEAAAA|17215|4|46|55|AM|third|night|| +17216|AAAAAAAABEDEAAAA|17216|4|46|56|AM|third|night|| +17217|AAAAAAAACEDEAAAA|17217|4|46|57|AM|third|night|| +17218|AAAAAAAADEDEAAAA|17218|4|46|58|AM|third|night|| +17219|AAAAAAAAEEDEAAAA|17219|4|46|59|AM|third|night|| +17220|AAAAAAAAFEDEAAAA|17220|4|47|0|AM|third|night|| +17221|AAAAAAAAGEDEAAAA|17221|4|47|1|AM|third|night|| +17222|AAAAAAAAHEDEAAAA|17222|4|47|2|AM|third|night|| +17223|AAAAAAAAIEDEAAAA|17223|4|47|3|AM|third|night|| +17224|AAAAAAAAJEDEAAAA|17224|4|47|4|AM|third|night|| +17225|AAAAAAAAKEDEAAAA|17225|4|47|5|AM|third|night|| +17226|AAAAAAAALEDEAAAA|17226|4|47|6|AM|third|night|| +17227|AAAAAAAAMEDEAAAA|17227|4|47|7|AM|third|night|| +17228|AAAAAAAANEDEAAAA|17228|4|47|8|AM|third|night|| +17229|AAAAAAAAOEDEAAAA|17229|4|47|9|AM|third|night|| +17230|AAAAAAAAPEDEAAAA|17230|4|47|10|AM|third|night|| +17231|AAAAAAAAAFDEAAAA|17231|4|47|11|AM|third|night|| +17232|AAAAAAAABFDEAAAA|17232|4|47|12|AM|third|night|| +17233|AAAAAAAACFDEAAAA|17233|4|47|13|AM|third|night|| +17234|AAAAAAAADFDEAAAA|17234|4|47|14|AM|third|night|| +17235|AAAAAAAAEFDEAAAA|17235|4|47|15|AM|third|night|| +17236|AAAAAAAAFFDEAAAA|17236|4|47|16|AM|third|night|| +17237|AAAAAAAAGFDEAAAA|17237|4|47|17|AM|third|night|| +17238|AAAAAAAAHFDEAAAA|17238|4|47|18|AM|third|night|| +17239|AAAAAAAAIFDEAAAA|17239|4|47|19|AM|third|night|| +17240|AAAAAAAAJFDEAAAA|17240|4|47|20|AM|third|night|| +17241|AAAAAAAAKFDEAAAA|17241|4|47|21|AM|third|night|| +17242|AAAAAAAALFDEAAAA|17242|4|47|22|AM|third|night|| +17243|AAAAAAAAMFDEAAAA|17243|4|47|23|AM|third|night|| +17244|AAAAAAAANFDEAAAA|17244|4|47|24|AM|third|night|| +17245|AAAAAAAAOFDEAAAA|17245|4|47|25|AM|third|night|| +17246|AAAAAAAAPFDEAAAA|17246|4|47|26|AM|third|night|| +17247|AAAAAAAAAGDEAAAA|17247|4|47|27|AM|third|night|| +17248|AAAAAAAABGDEAAAA|17248|4|47|28|AM|third|night|| +17249|AAAAAAAACGDEAAAA|17249|4|47|29|AM|third|night|| +17250|AAAAAAAADGDEAAAA|17250|4|47|30|AM|third|night|| +17251|AAAAAAAAEGDEAAAA|17251|4|47|31|AM|third|night|| +17252|AAAAAAAAFGDEAAAA|17252|4|47|32|AM|third|night|| +17253|AAAAAAAAGGDEAAAA|17253|4|47|33|AM|third|night|| +17254|AAAAAAAAHGDEAAAA|17254|4|47|34|AM|third|night|| +17255|AAAAAAAAIGDEAAAA|17255|4|47|35|AM|third|night|| +17256|AAAAAAAAJGDEAAAA|17256|4|47|36|AM|third|night|| +17257|AAAAAAAAKGDEAAAA|17257|4|47|37|AM|third|night|| +17258|AAAAAAAALGDEAAAA|17258|4|47|38|AM|third|night|| +17259|AAAAAAAAMGDEAAAA|17259|4|47|39|AM|third|night|| +17260|AAAAAAAANGDEAAAA|17260|4|47|40|AM|third|night|| +17261|AAAAAAAAOGDEAAAA|17261|4|47|41|AM|third|night|| +17262|AAAAAAAAPGDEAAAA|17262|4|47|42|AM|third|night|| +17263|AAAAAAAAAHDEAAAA|17263|4|47|43|AM|third|night|| +17264|AAAAAAAABHDEAAAA|17264|4|47|44|AM|third|night|| +17265|AAAAAAAACHDEAAAA|17265|4|47|45|AM|third|night|| +17266|AAAAAAAADHDEAAAA|17266|4|47|46|AM|third|night|| +17267|AAAAAAAAEHDEAAAA|17267|4|47|47|AM|third|night|| +17268|AAAAAAAAFHDEAAAA|17268|4|47|48|AM|third|night|| +17269|AAAAAAAAGHDEAAAA|17269|4|47|49|AM|third|night|| +17270|AAAAAAAAHHDEAAAA|17270|4|47|50|AM|third|night|| +17271|AAAAAAAAIHDEAAAA|17271|4|47|51|AM|third|night|| +17272|AAAAAAAAJHDEAAAA|17272|4|47|52|AM|third|night|| +17273|AAAAAAAAKHDEAAAA|17273|4|47|53|AM|third|night|| +17274|AAAAAAAALHDEAAAA|17274|4|47|54|AM|third|night|| +17275|AAAAAAAAMHDEAAAA|17275|4|47|55|AM|third|night|| +17276|AAAAAAAANHDEAAAA|17276|4|47|56|AM|third|night|| +17277|AAAAAAAAOHDEAAAA|17277|4|47|57|AM|third|night|| +17278|AAAAAAAAPHDEAAAA|17278|4|47|58|AM|third|night|| +17279|AAAAAAAAAIDEAAAA|17279|4|47|59|AM|third|night|| +17280|AAAAAAAABIDEAAAA|17280|4|48|0|AM|third|night|| +17281|AAAAAAAACIDEAAAA|17281|4|48|1|AM|third|night|| +17282|AAAAAAAADIDEAAAA|17282|4|48|2|AM|third|night|| +17283|AAAAAAAAEIDEAAAA|17283|4|48|3|AM|third|night|| +17284|AAAAAAAAFIDEAAAA|17284|4|48|4|AM|third|night|| +17285|AAAAAAAAGIDEAAAA|17285|4|48|5|AM|third|night|| +17286|AAAAAAAAHIDEAAAA|17286|4|48|6|AM|third|night|| +17287|AAAAAAAAIIDEAAAA|17287|4|48|7|AM|third|night|| +17288|AAAAAAAAJIDEAAAA|17288|4|48|8|AM|third|night|| +17289|AAAAAAAAKIDEAAAA|17289|4|48|9|AM|third|night|| +17290|AAAAAAAALIDEAAAA|17290|4|48|10|AM|third|night|| +17291|AAAAAAAAMIDEAAAA|17291|4|48|11|AM|third|night|| +17292|AAAAAAAANIDEAAAA|17292|4|48|12|AM|third|night|| +17293|AAAAAAAAOIDEAAAA|17293|4|48|13|AM|third|night|| +17294|AAAAAAAAPIDEAAAA|17294|4|48|14|AM|third|night|| +17295|AAAAAAAAAJDEAAAA|17295|4|48|15|AM|third|night|| +17296|AAAAAAAABJDEAAAA|17296|4|48|16|AM|third|night|| +17297|AAAAAAAACJDEAAAA|17297|4|48|17|AM|third|night|| +17298|AAAAAAAADJDEAAAA|17298|4|48|18|AM|third|night|| +17299|AAAAAAAAEJDEAAAA|17299|4|48|19|AM|third|night|| +17300|AAAAAAAAFJDEAAAA|17300|4|48|20|AM|third|night|| +17301|AAAAAAAAGJDEAAAA|17301|4|48|21|AM|third|night|| +17302|AAAAAAAAHJDEAAAA|17302|4|48|22|AM|third|night|| +17303|AAAAAAAAIJDEAAAA|17303|4|48|23|AM|third|night|| +17304|AAAAAAAAJJDEAAAA|17304|4|48|24|AM|third|night|| +17305|AAAAAAAAKJDEAAAA|17305|4|48|25|AM|third|night|| +17306|AAAAAAAALJDEAAAA|17306|4|48|26|AM|third|night|| +17307|AAAAAAAAMJDEAAAA|17307|4|48|27|AM|third|night|| +17308|AAAAAAAANJDEAAAA|17308|4|48|28|AM|third|night|| +17309|AAAAAAAAOJDEAAAA|17309|4|48|29|AM|third|night|| +17310|AAAAAAAAPJDEAAAA|17310|4|48|30|AM|third|night|| +17311|AAAAAAAAAKDEAAAA|17311|4|48|31|AM|third|night|| +17312|AAAAAAAABKDEAAAA|17312|4|48|32|AM|third|night|| +17313|AAAAAAAACKDEAAAA|17313|4|48|33|AM|third|night|| +17314|AAAAAAAADKDEAAAA|17314|4|48|34|AM|third|night|| +17315|AAAAAAAAEKDEAAAA|17315|4|48|35|AM|third|night|| +17316|AAAAAAAAFKDEAAAA|17316|4|48|36|AM|third|night|| +17317|AAAAAAAAGKDEAAAA|17317|4|48|37|AM|third|night|| +17318|AAAAAAAAHKDEAAAA|17318|4|48|38|AM|third|night|| +17319|AAAAAAAAIKDEAAAA|17319|4|48|39|AM|third|night|| +17320|AAAAAAAAJKDEAAAA|17320|4|48|40|AM|third|night|| +17321|AAAAAAAAKKDEAAAA|17321|4|48|41|AM|third|night|| +17322|AAAAAAAALKDEAAAA|17322|4|48|42|AM|third|night|| +17323|AAAAAAAAMKDEAAAA|17323|4|48|43|AM|third|night|| +17324|AAAAAAAANKDEAAAA|17324|4|48|44|AM|third|night|| +17325|AAAAAAAAOKDEAAAA|17325|4|48|45|AM|third|night|| +17326|AAAAAAAAPKDEAAAA|17326|4|48|46|AM|third|night|| +17327|AAAAAAAAALDEAAAA|17327|4|48|47|AM|third|night|| +17328|AAAAAAAABLDEAAAA|17328|4|48|48|AM|third|night|| +17329|AAAAAAAACLDEAAAA|17329|4|48|49|AM|third|night|| +17330|AAAAAAAADLDEAAAA|17330|4|48|50|AM|third|night|| +17331|AAAAAAAAELDEAAAA|17331|4|48|51|AM|third|night|| +17332|AAAAAAAAFLDEAAAA|17332|4|48|52|AM|third|night|| +17333|AAAAAAAAGLDEAAAA|17333|4|48|53|AM|third|night|| +17334|AAAAAAAAHLDEAAAA|17334|4|48|54|AM|third|night|| +17335|AAAAAAAAILDEAAAA|17335|4|48|55|AM|third|night|| +17336|AAAAAAAAJLDEAAAA|17336|4|48|56|AM|third|night|| +17337|AAAAAAAAKLDEAAAA|17337|4|48|57|AM|third|night|| +17338|AAAAAAAALLDEAAAA|17338|4|48|58|AM|third|night|| +17339|AAAAAAAAMLDEAAAA|17339|4|48|59|AM|third|night|| +17340|AAAAAAAANLDEAAAA|17340|4|49|0|AM|third|night|| +17341|AAAAAAAAOLDEAAAA|17341|4|49|1|AM|third|night|| +17342|AAAAAAAAPLDEAAAA|17342|4|49|2|AM|third|night|| +17343|AAAAAAAAAMDEAAAA|17343|4|49|3|AM|third|night|| +17344|AAAAAAAABMDEAAAA|17344|4|49|4|AM|third|night|| +17345|AAAAAAAACMDEAAAA|17345|4|49|5|AM|third|night|| +17346|AAAAAAAADMDEAAAA|17346|4|49|6|AM|third|night|| +17347|AAAAAAAAEMDEAAAA|17347|4|49|7|AM|third|night|| +17348|AAAAAAAAFMDEAAAA|17348|4|49|8|AM|third|night|| +17349|AAAAAAAAGMDEAAAA|17349|4|49|9|AM|third|night|| +17350|AAAAAAAAHMDEAAAA|17350|4|49|10|AM|third|night|| +17351|AAAAAAAAIMDEAAAA|17351|4|49|11|AM|third|night|| +17352|AAAAAAAAJMDEAAAA|17352|4|49|12|AM|third|night|| +17353|AAAAAAAAKMDEAAAA|17353|4|49|13|AM|third|night|| +17354|AAAAAAAALMDEAAAA|17354|4|49|14|AM|third|night|| +17355|AAAAAAAAMMDEAAAA|17355|4|49|15|AM|third|night|| +17356|AAAAAAAANMDEAAAA|17356|4|49|16|AM|third|night|| +17357|AAAAAAAAOMDEAAAA|17357|4|49|17|AM|third|night|| +17358|AAAAAAAAPMDEAAAA|17358|4|49|18|AM|third|night|| +17359|AAAAAAAAANDEAAAA|17359|4|49|19|AM|third|night|| +17360|AAAAAAAABNDEAAAA|17360|4|49|20|AM|third|night|| +17361|AAAAAAAACNDEAAAA|17361|4|49|21|AM|third|night|| +17362|AAAAAAAADNDEAAAA|17362|4|49|22|AM|third|night|| +17363|AAAAAAAAENDEAAAA|17363|4|49|23|AM|third|night|| +17364|AAAAAAAAFNDEAAAA|17364|4|49|24|AM|third|night|| +17365|AAAAAAAAGNDEAAAA|17365|4|49|25|AM|third|night|| +17366|AAAAAAAAHNDEAAAA|17366|4|49|26|AM|third|night|| +17367|AAAAAAAAINDEAAAA|17367|4|49|27|AM|third|night|| +17368|AAAAAAAAJNDEAAAA|17368|4|49|28|AM|third|night|| +17369|AAAAAAAAKNDEAAAA|17369|4|49|29|AM|third|night|| +17370|AAAAAAAALNDEAAAA|17370|4|49|30|AM|third|night|| +17371|AAAAAAAAMNDEAAAA|17371|4|49|31|AM|third|night|| +17372|AAAAAAAANNDEAAAA|17372|4|49|32|AM|third|night|| +17373|AAAAAAAAONDEAAAA|17373|4|49|33|AM|third|night|| +17374|AAAAAAAAPNDEAAAA|17374|4|49|34|AM|third|night|| +17375|AAAAAAAAAODEAAAA|17375|4|49|35|AM|third|night|| +17376|AAAAAAAABODEAAAA|17376|4|49|36|AM|third|night|| +17377|AAAAAAAACODEAAAA|17377|4|49|37|AM|third|night|| +17378|AAAAAAAADODEAAAA|17378|4|49|38|AM|third|night|| +17379|AAAAAAAAEODEAAAA|17379|4|49|39|AM|third|night|| +17380|AAAAAAAAFODEAAAA|17380|4|49|40|AM|third|night|| +17381|AAAAAAAAGODEAAAA|17381|4|49|41|AM|third|night|| +17382|AAAAAAAAHODEAAAA|17382|4|49|42|AM|third|night|| +17383|AAAAAAAAIODEAAAA|17383|4|49|43|AM|third|night|| +17384|AAAAAAAAJODEAAAA|17384|4|49|44|AM|third|night|| +17385|AAAAAAAAKODEAAAA|17385|4|49|45|AM|third|night|| +17386|AAAAAAAALODEAAAA|17386|4|49|46|AM|third|night|| +17387|AAAAAAAAMODEAAAA|17387|4|49|47|AM|third|night|| +17388|AAAAAAAANODEAAAA|17388|4|49|48|AM|third|night|| +17389|AAAAAAAAOODEAAAA|17389|4|49|49|AM|third|night|| +17390|AAAAAAAAPODEAAAA|17390|4|49|50|AM|third|night|| +17391|AAAAAAAAAPDEAAAA|17391|4|49|51|AM|third|night|| +17392|AAAAAAAABPDEAAAA|17392|4|49|52|AM|third|night|| +17393|AAAAAAAACPDEAAAA|17393|4|49|53|AM|third|night|| +17394|AAAAAAAADPDEAAAA|17394|4|49|54|AM|third|night|| +17395|AAAAAAAAEPDEAAAA|17395|4|49|55|AM|third|night|| +17396|AAAAAAAAFPDEAAAA|17396|4|49|56|AM|third|night|| +17397|AAAAAAAAGPDEAAAA|17397|4|49|57|AM|third|night|| +17398|AAAAAAAAHPDEAAAA|17398|4|49|58|AM|third|night|| +17399|AAAAAAAAIPDEAAAA|17399|4|49|59|AM|third|night|| +17400|AAAAAAAAJPDEAAAA|17400|4|50|0|AM|third|night|| +17401|AAAAAAAAKPDEAAAA|17401|4|50|1|AM|third|night|| +17402|AAAAAAAALPDEAAAA|17402|4|50|2|AM|third|night|| +17403|AAAAAAAAMPDEAAAA|17403|4|50|3|AM|third|night|| +17404|AAAAAAAANPDEAAAA|17404|4|50|4|AM|third|night|| +17405|AAAAAAAAOPDEAAAA|17405|4|50|5|AM|third|night|| +17406|AAAAAAAAPPDEAAAA|17406|4|50|6|AM|third|night|| +17407|AAAAAAAAAAEEAAAA|17407|4|50|7|AM|third|night|| +17408|AAAAAAAABAEEAAAA|17408|4|50|8|AM|third|night|| +17409|AAAAAAAACAEEAAAA|17409|4|50|9|AM|third|night|| +17410|AAAAAAAADAEEAAAA|17410|4|50|10|AM|third|night|| +17411|AAAAAAAAEAEEAAAA|17411|4|50|11|AM|third|night|| +17412|AAAAAAAAFAEEAAAA|17412|4|50|12|AM|third|night|| +17413|AAAAAAAAGAEEAAAA|17413|4|50|13|AM|third|night|| +17414|AAAAAAAAHAEEAAAA|17414|4|50|14|AM|third|night|| +17415|AAAAAAAAIAEEAAAA|17415|4|50|15|AM|third|night|| +17416|AAAAAAAAJAEEAAAA|17416|4|50|16|AM|third|night|| +17417|AAAAAAAAKAEEAAAA|17417|4|50|17|AM|third|night|| +17418|AAAAAAAALAEEAAAA|17418|4|50|18|AM|third|night|| +17419|AAAAAAAAMAEEAAAA|17419|4|50|19|AM|third|night|| +17420|AAAAAAAANAEEAAAA|17420|4|50|20|AM|third|night|| +17421|AAAAAAAAOAEEAAAA|17421|4|50|21|AM|third|night|| +17422|AAAAAAAAPAEEAAAA|17422|4|50|22|AM|third|night|| +17423|AAAAAAAAABEEAAAA|17423|4|50|23|AM|third|night|| +17424|AAAAAAAABBEEAAAA|17424|4|50|24|AM|third|night|| +17425|AAAAAAAACBEEAAAA|17425|4|50|25|AM|third|night|| +17426|AAAAAAAADBEEAAAA|17426|4|50|26|AM|third|night|| +17427|AAAAAAAAEBEEAAAA|17427|4|50|27|AM|third|night|| +17428|AAAAAAAAFBEEAAAA|17428|4|50|28|AM|third|night|| +17429|AAAAAAAAGBEEAAAA|17429|4|50|29|AM|third|night|| +17430|AAAAAAAAHBEEAAAA|17430|4|50|30|AM|third|night|| +17431|AAAAAAAAIBEEAAAA|17431|4|50|31|AM|third|night|| +17432|AAAAAAAAJBEEAAAA|17432|4|50|32|AM|third|night|| +17433|AAAAAAAAKBEEAAAA|17433|4|50|33|AM|third|night|| +17434|AAAAAAAALBEEAAAA|17434|4|50|34|AM|third|night|| +17435|AAAAAAAAMBEEAAAA|17435|4|50|35|AM|third|night|| +17436|AAAAAAAANBEEAAAA|17436|4|50|36|AM|third|night|| +17437|AAAAAAAAOBEEAAAA|17437|4|50|37|AM|third|night|| +17438|AAAAAAAAPBEEAAAA|17438|4|50|38|AM|third|night|| +17439|AAAAAAAAACEEAAAA|17439|4|50|39|AM|third|night|| +17440|AAAAAAAABCEEAAAA|17440|4|50|40|AM|third|night|| +17441|AAAAAAAACCEEAAAA|17441|4|50|41|AM|third|night|| +17442|AAAAAAAADCEEAAAA|17442|4|50|42|AM|third|night|| +17443|AAAAAAAAECEEAAAA|17443|4|50|43|AM|third|night|| +17444|AAAAAAAAFCEEAAAA|17444|4|50|44|AM|third|night|| +17445|AAAAAAAAGCEEAAAA|17445|4|50|45|AM|third|night|| +17446|AAAAAAAAHCEEAAAA|17446|4|50|46|AM|third|night|| +17447|AAAAAAAAICEEAAAA|17447|4|50|47|AM|third|night|| +17448|AAAAAAAAJCEEAAAA|17448|4|50|48|AM|third|night|| +17449|AAAAAAAAKCEEAAAA|17449|4|50|49|AM|third|night|| +17450|AAAAAAAALCEEAAAA|17450|4|50|50|AM|third|night|| +17451|AAAAAAAAMCEEAAAA|17451|4|50|51|AM|third|night|| +17452|AAAAAAAANCEEAAAA|17452|4|50|52|AM|third|night|| +17453|AAAAAAAAOCEEAAAA|17453|4|50|53|AM|third|night|| +17454|AAAAAAAAPCEEAAAA|17454|4|50|54|AM|third|night|| +17455|AAAAAAAAADEEAAAA|17455|4|50|55|AM|third|night|| +17456|AAAAAAAABDEEAAAA|17456|4|50|56|AM|third|night|| +17457|AAAAAAAACDEEAAAA|17457|4|50|57|AM|third|night|| +17458|AAAAAAAADDEEAAAA|17458|4|50|58|AM|third|night|| +17459|AAAAAAAAEDEEAAAA|17459|4|50|59|AM|third|night|| +17460|AAAAAAAAFDEEAAAA|17460|4|51|0|AM|third|night|| +17461|AAAAAAAAGDEEAAAA|17461|4|51|1|AM|third|night|| +17462|AAAAAAAAHDEEAAAA|17462|4|51|2|AM|third|night|| +17463|AAAAAAAAIDEEAAAA|17463|4|51|3|AM|third|night|| +17464|AAAAAAAAJDEEAAAA|17464|4|51|4|AM|third|night|| +17465|AAAAAAAAKDEEAAAA|17465|4|51|5|AM|third|night|| +17466|AAAAAAAALDEEAAAA|17466|4|51|6|AM|third|night|| +17467|AAAAAAAAMDEEAAAA|17467|4|51|7|AM|third|night|| +17468|AAAAAAAANDEEAAAA|17468|4|51|8|AM|third|night|| +17469|AAAAAAAAODEEAAAA|17469|4|51|9|AM|third|night|| +17470|AAAAAAAAPDEEAAAA|17470|4|51|10|AM|third|night|| +17471|AAAAAAAAAEEEAAAA|17471|4|51|11|AM|third|night|| +17472|AAAAAAAABEEEAAAA|17472|4|51|12|AM|third|night|| +17473|AAAAAAAACEEEAAAA|17473|4|51|13|AM|third|night|| +17474|AAAAAAAADEEEAAAA|17474|4|51|14|AM|third|night|| +17475|AAAAAAAAEEEEAAAA|17475|4|51|15|AM|third|night|| +17476|AAAAAAAAFEEEAAAA|17476|4|51|16|AM|third|night|| +17477|AAAAAAAAGEEEAAAA|17477|4|51|17|AM|third|night|| +17478|AAAAAAAAHEEEAAAA|17478|4|51|18|AM|third|night|| +17479|AAAAAAAAIEEEAAAA|17479|4|51|19|AM|third|night|| +17480|AAAAAAAAJEEEAAAA|17480|4|51|20|AM|third|night|| +17481|AAAAAAAAKEEEAAAA|17481|4|51|21|AM|third|night|| +17482|AAAAAAAALEEEAAAA|17482|4|51|22|AM|third|night|| +17483|AAAAAAAAMEEEAAAA|17483|4|51|23|AM|third|night|| +17484|AAAAAAAANEEEAAAA|17484|4|51|24|AM|third|night|| +17485|AAAAAAAAOEEEAAAA|17485|4|51|25|AM|third|night|| +17486|AAAAAAAAPEEEAAAA|17486|4|51|26|AM|third|night|| +17487|AAAAAAAAAFEEAAAA|17487|4|51|27|AM|third|night|| +17488|AAAAAAAABFEEAAAA|17488|4|51|28|AM|third|night|| +17489|AAAAAAAACFEEAAAA|17489|4|51|29|AM|third|night|| +17490|AAAAAAAADFEEAAAA|17490|4|51|30|AM|third|night|| +17491|AAAAAAAAEFEEAAAA|17491|4|51|31|AM|third|night|| +17492|AAAAAAAAFFEEAAAA|17492|4|51|32|AM|third|night|| +17493|AAAAAAAAGFEEAAAA|17493|4|51|33|AM|third|night|| +17494|AAAAAAAAHFEEAAAA|17494|4|51|34|AM|third|night|| +17495|AAAAAAAAIFEEAAAA|17495|4|51|35|AM|third|night|| +17496|AAAAAAAAJFEEAAAA|17496|4|51|36|AM|third|night|| +17497|AAAAAAAAKFEEAAAA|17497|4|51|37|AM|third|night|| +17498|AAAAAAAALFEEAAAA|17498|4|51|38|AM|third|night|| +17499|AAAAAAAAMFEEAAAA|17499|4|51|39|AM|third|night|| +17500|AAAAAAAANFEEAAAA|17500|4|51|40|AM|third|night|| +17501|AAAAAAAAOFEEAAAA|17501|4|51|41|AM|third|night|| +17502|AAAAAAAAPFEEAAAA|17502|4|51|42|AM|third|night|| +17503|AAAAAAAAAGEEAAAA|17503|4|51|43|AM|third|night|| +17504|AAAAAAAABGEEAAAA|17504|4|51|44|AM|third|night|| +17505|AAAAAAAACGEEAAAA|17505|4|51|45|AM|third|night|| +17506|AAAAAAAADGEEAAAA|17506|4|51|46|AM|third|night|| +17507|AAAAAAAAEGEEAAAA|17507|4|51|47|AM|third|night|| +17508|AAAAAAAAFGEEAAAA|17508|4|51|48|AM|third|night|| +17509|AAAAAAAAGGEEAAAA|17509|4|51|49|AM|third|night|| +17510|AAAAAAAAHGEEAAAA|17510|4|51|50|AM|third|night|| +17511|AAAAAAAAIGEEAAAA|17511|4|51|51|AM|third|night|| +17512|AAAAAAAAJGEEAAAA|17512|4|51|52|AM|third|night|| +17513|AAAAAAAAKGEEAAAA|17513|4|51|53|AM|third|night|| +17514|AAAAAAAALGEEAAAA|17514|4|51|54|AM|third|night|| +17515|AAAAAAAAMGEEAAAA|17515|4|51|55|AM|third|night|| +17516|AAAAAAAANGEEAAAA|17516|4|51|56|AM|third|night|| +17517|AAAAAAAAOGEEAAAA|17517|4|51|57|AM|third|night|| +17518|AAAAAAAAPGEEAAAA|17518|4|51|58|AM|third|night|| +17519|AAAAAAAAAHEEAAAA|17519|4|51|59|AM|third|night|| +17520|AAAAAAAABHEEAAAA|17520|4|52|0|AM|third|night|| +17521|AAAAAAAACHEEAAAA|17521|4|52|1|AM|third|night|| +17522|AAAAAAAADHEEAAAA|17522|4|52|2|AM|third|night|| +17523|AAAAAAAAEHEEAAAA|17523|4|52|3|AM|third|night|| +17524|AAAAAAAAFHEEAAAA|17524|4|52|4|AM|third|night|| +17525|AAAAAAAAGHEEAAAA|17525|4|52|5|AM|third|night|| +17526|AAAAAAAAHHEEAAAA|17526|4|52|6|AM|third|night|| +17527|AAAAAAAAIHEEAAAA|17527|4|52|7|AM|third|night|| +17528|AAAAAAAAJHEEAAAA|17528|4|52|8|AM|third|night|| +17529|AAAAAAAAKHEEAAAA|17529|4|52|9|AM|third|night|| +17530|AAAAAAAALHEEAAAA|17530|4|52|10|AM|third|night|| +17531|AAAAAAAAMHEEAAAA|17531|4|52|11|AM|third|night|| +17532|AAAAAAAANHEEAAAA|17532|4|52|12|AM|third|night|| +17533|AAAAAAAAOHEEAAAA|17533|4|52|13|AM|third|night|| +17534|AAAAAAAAPHEEAAAA|17534|4|52|14|AM|third|night|| +17535|AAAAAAAAAIEEAAAA|17535|4|52|15|AM|third|night|| +17536|AAAAAAAABIEEAAAA|17536|4|52|16|AM|third|night|| +17537|AAAAAAAACIEEAAAA|17537|4|52|17|AM|third|night|| +17538|AAAAAAAADIEEAAAA|17538|4|52|18|AM|third|night|| +17539|AAAAAAAAEIEEAAAA|17539|4|52|19|AM|third|night|| +17540|AAAAAAAAFIEEAAAA|17540|4|52|20|AM|third|night|| +17541|AAAAAAAAGIEEAAAA|17541|4|52|21|AM|third|night|| +17542|AAAAAAAAHIEEAAAA|17542|4|52|22|AM|third|night|| +17543|AAAAAAAAIIEEAAAA|17543|4|52|23|AM|third|night|| +17544|AAAAAAAAJIEEAAAA|17544|4|52|24|AM|third|night|| +17545|AAAAAAAAKIEEAAAA|17545|4|52|25|AM|third|night|| +17546|AAAAAAAALIEEAAAA|17546|4|52|26|AM|third|night|| +17547|AAAAAAAAMIEEAAAA|17547|4|52|27|AM|third|night|| +17548|AAAAAAAANIEEAAAA|17548|4|52|28|AM|third|night|| +17549|AAAAAAAAOIEEAAAA|17549|4|52|29|AM|third|night|| +17550|AAAAAAAAPIEEAAAA|17550|4|52|30|AM|third|night|| +17551|AAAAAAAAAJEEAAAA|17551|4|52|31|AM|third|night|| +17552|AAAAAAAABJEEAAAA|17552|4|52|32|AM|third|night|| +17553|AAAAAAAACJEEAAAA|17553|4|52|33|AM|third|night|| +17554|AAAAAAAADJEEAAAA|17554|4|52|34|AM|third|night|| +17555|AAAAAAAAEJEEAAAA|17555|4|52|35|AM|third|night|| +17556|AAAAAAAAFJEEAAAA|17556|4|52|36|AM|third|night|| +17557|AAAAAAAAGJEEAAAA|17557|4|52|37|AM|third|night|| +17558|AAAAAAAAHJEEAAAA|17558|4|52|38|AM|third|night|| +17559|AAAAAAAAIJEEAAAA|17559|4|52|39|AM|third|night|| +17560|AAAAAAAAJJEEAAAA|17560|4|52|40|AM|third|night|| +17561|AAAAAAAAKJEEAAAA|17561|4|52|41|AM|third|night|| +17562|AAAAAAAALJEEAAAA|17562|4|52|42|AM|third|night|| +17563|AAAAAAAAMJEEAAAA|17563|4|52|43|AM|third|night|| +17564|AAAAAAAANJEEAAAA|17564|4|52|44|AM|third|night|| +17565|AAAAAAAAOJEEAAAA|17565|4|52|45|AM|third|night|| +17566|AAAAAAAAPJEEAAAA|17566|4|52|46|AM|third|night|| +17567|AAAAAAAAAKEEAAAA|17567|4|52|47|AM|third|night|| +17568|AAAAAAAABKEEAAAA|17568|4|52|48|AM|third|night|| +17569|AAAAAAAACKEEAAAA|17569|4|52|49|AM|third|night|| +17570|AAAAAAAADKEEAAAA|17570|4|52|50|AM|third|night|| +17571|AAAAAAAAEKEEAAAA|17571|4|52|51|AM|third|night|| +17572|AAAAAAAAFKEEAAAA|17572|4|52|52|AM|third|night|| +17573|AAAAAAAAGKEEAAAA|17573|4|52|53|AM|third|night|| +17574|AAAAAAAAHKEEAAAA|17574|4|52|54|AM|third|night|| +17575|AAAAAAAAIKEEAAAA|17575|4|52|55|AM|third|night|| +17576|AAAAAAAAJKEEAAAA|17576|4|52|56|AM|third|night|| +17577|AAAAAAAAKKEEAAAA|17577|4|52|57|AM|third|night|| +17578|AAAAAAAALKEEAAAA|17578|4|52|58|AM|third|night|| +17579|AAAAAAAAMKEEAAAA|17579|4|52|59|AM|third|night|| +17580|AAAAAAAANKEEAAAA|17580|4|53|0|AM|third|night|| +17581|AAAAAAAAOKEEAAAA|17581|4|53|1|AM|third|night|| +17582|AAAAAAAAPKEEAAAA|17582|4|53|2|AM|third|night|| +17583|AAAAAAAAALEEAAAA|17583|4|53|3|AM|third|night|| +17584|AAAAAAAABLEEAAAA|17584|4|53|4|AM|third|night|| +17585|AAAAAAAACLEEAAAA|17585|4|53|5|AM|third|night|| +17586|AAAAAAAADLEEAAAA|17586|4|53|6|AM|third|night|| +17587|AAAAAAAAELEEAAAA|17587|4|53|7|AM|third|night|| +17588|AAAAAAAAFLEEAAAA|17588|4|53|8|AM|third|night|| +17589|AAAAAAAAGLEEAAAA|17589|4|53|9|AM|third|night|| +17590|AAAAAAAAHLEEAAAA|17590|4|53|10|AM|third|night|| +17591|AAAAAAAAILEEAAAA|17591|4|53|11|AM|third|night|| +17592|AAAAAAAAJLEEAAAA|17592|4|53|12|AM|third|night|| +17593|AAAAAAAAKLEEAAAA|17593|4|53|13|AM|third|night|| +17594|AAAAAAAALLEEAAAA|17594|4|53|14|AM|third|night|| +17595|AAAAAAAAMLEEAAAA|17595|4|53|15|AM|third|night|| +17596|AAAAAAAANLEEAAAA|17596|4|53|16|AM|third|night|| +17597|AAAAAAAAOLEEAAAA|17597|4|53|17|AM|third|night|| +17598|AAAAAAAAPLEEAAAA|17598|4|53|18|AM|third|night|| +17599|AAAAAAAAAMEEAAAA|17599|4|53|19|AM|third|night|| +17600|AAAAAAAABMEEAAAA|17600|4|53|20|AM|third|night|| +17601|AAAAAAAACMEEAAAA|17601|4|53|21|AM|third|night|| +17602|AAAAAAAADMEEAAAA|17602|4|53|22|AM|third|night|| +17603|AAAAAAAAEMEEAAAA|17603|4|53|23|AM|third|night|| +17604|AAAAAAAAFMEEAAAA|17604|4|53|24|AM|third|night|| +17605|AAAAAAAAGMEEAAAA|17605|4|53|25|AM|third|night|| +17606|AAAAAAAAHMEEAAAA|17606|4|53|26|AM|third|night|| +17607|AAAAAAAAIMEEAAAA|17607|4|53|27|AM|third|night|| +17608|AAAAAAAAJMEEAAAA|17608|4|53|28|AM|third|night|| +17609|AAAAAAAAKMEEAAAA|17609|4|53|29|AM|third|night|| +17610|AAAAAAAALMEEAAAA|17610|4|53|30|AM|third|night|| +17611|AAAAAAAAMMEEAAAA|17611|4|53|31|AM|third|night|| +17612|AAAAAAAANMEEAAAA|17612|4|53|32|AM|third|night|| +17613|AAAAAAAAOMEEAAAA|17613|4|53|33|AM|third|night|| +17614|AAAAAAAAPMEEAAAA|17614|4|53|34|AM|third|night|| +17615|AAAAAAAAANEEAAAA|17615|4|53|35|AM|third|night|| +17616|AAAAAAAABNEEAAAA|17616|4|53|36|AM|third|night|| +17617|AAAAAAAACNEEAAAA|17617|4|53|37|AM|third|night|| +17618|AAAAAAAADNEEAAAA|17618|4|53|38|AM|third|night|| +17619|AAAAAAAAENEEAAAA|17619|4|53|39|AM|third|night|| +17620|AAAAAAAAFNEEAAAA|17620|4|53|40|AM|third|night|| +17621|AAAAAAAAGNEEAAAA|17621|4|53|41|AM|third|night|| +17622|AAAAAAAAHNEEAAAA|17622|4|53|42|AM|third|night|| +17623|AAAAAAAAINEEAAAA|17623|4|53|43|AM|third|night|| +17624|AAAAAAAAJNEEAAAA|17624|4|53|44|AM|third|night|| +17625|AAAAAAAAKNEEAAAA|17625|4|53|45|AM|third|night|| +17626|AAAAAAAALNEEAAAA|17626|4|53|46|AM|third|night|| +17627|AAAAAAAAMNEEAAAA|17627|4|53|47|AM|third|night|| +17628|AAAAAAAANNEEAAAA|17628|4|53|48|AM|third|night|| +17629|AAAAAAAAONEEAAAA|17629|4|53|49|AM|third|night|| +17630|AAAAAAAAPNEEAAAA|17630|4|53|50|AM|third|night|| +17631|AAAAAAAAAOEEAAAA|17631|4|53|51|AM|third|night|| +17632|AAAAAAAABOEEAAAA|17632|4|53|52|AM|third|night|| +17633|AAAAAAAACOEEAAAA|17633|4|53|53|AM|third|night|| +17634|AAAAAAAADOEEAAAA|17634|4|53|54|AM|third|night|| +17635|AAAAAAAAEOEEAAAA|17635|4|53|55|AM|third|night|| +17636|AAAAAAAAFOEEAAAA|17636|4|53|56|AM|third|night|| +17637|AAAAAAAAGOEEAAAA|17637|4|53|57|AM|third|night|| +17638|AAAAAAAAHOEEAAAA|17638|4|53|58|AM|third|night|| +17639|AAAAAAAAIOEEAAAA|17639|4|53|59|AM|third|night|| +17640|AAAAAAAAJOEEAAAA|17640|4|54|0|AM|third|night|| +17641|AAAAAAAAKOEEAAAA|17641|4|54|1|AM|third|night|| +17642|AAAAAAAALOEEAAAA|17642|4|54|2|AM|third|night|| +17643|AAAAAAAAMOEEAAAA|17643|4|54|3|AM|third|night|| +17644|AAAAAAAANOEEAAAA|17644|4|54|4|AM|third|night|| +17645|AAAAAAAAOOEEAAAA|17645|4|54|5|AM|third|night|| +17646|AAAAAAAAPOEEAAAA|17646|4|54|6|AM|third|night|| +17647|AAAAAAAAAPEEAAAA|17647|4|54|7|AM|third|night|| +17648|AAAAAAAABPEEAAAA|17648|4|54|8|AM|third|night|| +17649|AAAAAAAACPEEAAAA|17649|4|54|9|AM|third|night|| +17650|AAAAAAAADPEEAAAA|17650|4|54|10|AM|third|night|| +17651|AAAAAAAAEPEEAAAA|17651|4|54|11|AM|third|night|| +17652|AAAAAAAAFPEEAAAA|17652|4|54|12|AM|third|night|| +17653|AAAAAAAAGPEEAAAA|17653|4|54|13|AM|third|night|| +17654|AAAAAAAAHPEEAAAA|17654|4|54|14|AM|third|night|| +17655|AAAAAAAAIPEEAAAA|17655|4|54|15|AM|third|night|| +17656|AAAAAAAAJPEEAAAA|17656|4|54|16|AM|third|night|| +17657|AAAAAAAAKPEEAAAA|17657|4|54|17|AM|third|night|| +17658|AAAAAAAALPEEAAAA|17658|4|54|18|AM|third|night|| +17659|AAAAAAAAMPEEAAAA|17659|4|54|19|AM|third|night|| +17660|AAAAAAAANPEEAAAA|17660|4|54|20|AM|third|night|| +17661|AAAAAAAAOPEEAAAA|17661|4|54|21|AM|third|night|| +17662|AAAAAAAAPPEEAAAA|17662|4|54|22|AM|third|night|| +17663|AAAAAAAAAAFEAAAA|17663|4|54|23|AM|third|night|| +17664|AAAAAAAABAFEAAAA|17664|4|54|24|AM|third|night|| +17665|AAAAAAAACAFEAAAA|17665|4|54|25|AM|third|night|| +17666|AAAAAAAADAFEAAAA|17666|4|54|26|AM|third|night|| +17667|AAAAAAAAEAFEAAAA|17667|4|54|27|AM|third|night|| +17668|AAAAAAAAFAFEAAAA|17668|4|54|28|AM|third|night|| +17669|AAAAAAAAGAFEAAAA|17669|4|54|29|AM|third|night|| +17670|AAAAAAAAHAFEAAAA|17670|4|54|30|AM|third|night|| +17671|AAAAAAAAIAFEAAAA|17671|4|54|31|AM|third|night|| +17672|AAAAAAAAJAFEAAAA|17672|4|54|32|AM|third|night|| +17673|AAAAAAAAKAFEAAAA|17673|4|54|33|AM|third|night|| +17674|AAAAAAAALAFEAAAA|17674|4|54|34|AM|third|night|| +17675|AAAAAAAAMAFEAAAA|17675|4|54|35|AM|third|night|| +17676|AAAAAAAANAFEAAAA|17676|4|54|36|AM|third|night|| +17677|AAAAAAAAOAFEAAAA|17677|4|54|37|AM|third|night|| +17678|AAAAAAAAPAFEAAAA|17678|4|54|38|AM|third|night|| +17679|AAAAAAAAABFEAAAA|17679|4|54|39|AM|third|night|| +17680|AAAAAAAABBFEAAAA|17680|4|54|40|AM|third|night|| +17681|AAAAAAAACBFEAAAA|17681|4|54|41|AM|third|night|| +17682|AAAAAAAADBFEAAAA|17682|4|54|42|AM|third|night|| +17683|AAAAAAAAEBFEAAAA|17683|4|54|43|AM|third|night|| +17684|AAAAAAAAFBFEAAAA|17684|4|54|44|AM|third|night|| +17685|AAAAAAAAGBFEAAAA|17685|4|54|45|AM|third|night|| +17686|AAAAAAAAHBFEAAAA|17686|4|54|46|AM|third|night|| +17687|AAAAAAAAIBFEAAAA|17687|4|54|47|AM|third|night|| +17688|AAAAAAAAJBFEAAAA|17688|4|54|48|AM|third|night|| +17689|AAAAAAAAKBFEAAAA|17689|4|54|49|AM|third|night|| +17690|AAAAAAAALBFEAAAA|17690|4|54|50|AM|third|night|| +17691|AAAAAAAAMBFEAAAA|17691|4|54|51|AM|third|night|| +17692|AAAAAAAANBFEAAAA|17692|4|54|52|AM|third|night|| +17693|AAAAAAAAOBFEAAAA|17693|4|54|53|AM|third|night|| +17694|AAAAAAAAPBFEAAAA|17694|4|54|54|AM|third|night|| +17695|AAAAAAAAACFEAAAA|17695|4|54|55|AM|third|night|| +17696|AAAAAAAABCFEAAAA|17696|4|54|56|AM|third|night|| +17697|AAAAAAAACCFEAAAA|17697|4|54|57|AM|third|night|| +17698|AAAAAAAADCFEAAAA|17698|4|54|58|AM|third|night|| +17699|AAAAAAAAECFEAAAA|17699|4|54|59|AM|third|night|| +17700|AAAAAAAAFCFEAAAA|17700|4|55|0|AM|third|night|| +17701|AAAAAAAAGCFEAAAA|17701|4|55|1|AM|third|night|| +17702|AAAAAAAAHCFEAAAA|17702|4|55|2|AM|third|night|| +17703|AAAAAAAAICFEAAAA|17703|4|55|3|AM|third|night|| +17704|AAAAAAAAJCFEAAAA|17704|4|55|4|AM|third|night|| +17705|AAAAAAAAKCFEAAAA|17705|4|55|5|AM|third|night|| +17706|AAAAAAAALCFEAAAA|17706|4|55|6|AM|third|night|| +17707|AAAAAAAAMCFEAAAA|17707|4|55|7|AM|third|night|| +17708|AAAAAAAANCFEAAAA|17708|4|55|8|AM|third|night|| +17709|AAAAAAAAOCFEAAAA|17709|4|55|9|AM|third|night|| +17710|AAAAAAAAPCFEAAAA|17710|4|55|10|AM|third|night|| +17711|AAAAAAAAADFEAAAA|17711|4|55|11|AM|third|night|| +17712|AAAAAAAABDFEAAAA|17712|4|55|12|AM|third|night|| +17713|AAAAAAAACDFEAAAA|17713|4|55|13|AM|third|night|| +17714|AAAAAAAADDFEAAAA|17714|4|55|14|AM|third|night|| +17715|AAAAAAAAEDFEAAAA|17715|4|55|15|AM|third|night|| +17716|AAAAAAAAFDFEAAAA|17716|4|55|16|AM|third|night|| +17717|AAAAAAAAGDFEAAAA|17717|4|55|17|AM|third|night|| +17718|AAAAAAAAHDFEAAAA|17718|4|55|18|AM|third|night|| +17719|AAAAAAAAIDFEAAAA|17719|4|55|19|AM|third|night|| +17720|AAAAAAAAJDFEAAAA|17720|4|55|20|AM|third|night|| +17721|AAAAAAAAKDFEAAAA|17721|4|55|21|AM|third|night|| +17722|AAAAAAAALDFEAAAA|17722|4|55|22|AM|third|night|| +17723|AAAAAAAAMDFEAAAA|17723|4|55|23|AM|third|night|| +17724|AAAAAAAANDFEAAAA|17724|4|55|24|AM|third|night|| +17725|AAAAAAAAODFEAAAA|17725|4|55|25|AM|third|night|| +17726|AAAAAAAAPDFEAAAA|17726|4|55|26|AM|third|night|| +17727|AAAAAAAAAEFEAAAA|17727|4|55|27|AM|third|night|| +17728|AAAAAAAABEFEAAAA|17728|4|55|28|AM|third|night|| +17729|AAAAAAAACEFEAAAA|17729|4|55|29|AM|third|night|| +17730|AAAAAAAADEFEAAAA|17730|4|55|30|AM|third|night|| +17731|AAAAAAAAEEFEAAAA|17731|4|55|31|AM|third|night|| +17732|AAAAAAAAFEFEAAAA|17732|4|55|32|AM|third|night|| +17733|AAAAAAAAGEFEAAAA|17733|4|55|33|AM|third|night|| +17734|AAAAAAAAHEFEAAAA|17734|4|55|34|AM|third|night|| +17735|AAAAAAAAIEFEAAAA|17735|4|55|35|AM|third|night|| +17736|AAAAAAAAJEFEAAAA|17736|4|55|36|AM|third|night|| +17737|AAAAAAAAKEFEAAAA|17737|4|55|37|AM|third|night|| +17738|AAAAAAAALEFEAAAA|17738|4|55|38|AM|third|night|| +17739|AAAAAAAAMEFEAAAA|17739|4|55|39|AM|third|night|| +17740|AAAAAAAANEFEAAAA|17740|4|55|40|AM|third|night|| +17741|AAAAAAAAOEFEAAAA|17741|4|55|41|AM|third|night|| +17742|AAAAAAAAPEFEAAAA|17742|4|55|42|AM|third|night|| +17743|AAAAAAAAAFFEAAAA|17743|4|55|43|AM|third|night|| +17744|AAAAAAAABFFEAAAA|17744|4|55|44|AM|third|night|| +17745|AAAAAAAACFFEAAAA|17745|4|55|45|AM|third|night|| +17746|AAAAAAAADFFEAAAA|17746|4|55|46|AM|third|night|| +17747|AAAAAAAAEFFEAAAA|17747|4|55|47|AM|third|night|| +17748|AAAAAAAAFFFEAAAA|17748|4|55|48|AM|third|night|| +17749|AAAAAAAAGFFEAAAA|17749|4|55|49|AM|third|night|| +17750|AAAAAAAAHFFEAAAA|17750|4|55|50|AM|third|night|| +17751|AAAAAAAAIFFEAAAA|17751|4|55|51|AM|third|night|| +17752|AAAAAAAAJFFEAAAA|17752|4|55|52|AM|third|night|| +17753|AAAAAAAAKFFEAAAA|17753|4|55|53|AM|third|night|| +17754|AAAAAAAALFFEAAAA|17754|4|55|54|AM|third|night|| +17755|AAAAAAAAMFFEAAAA|17755|4|55|55|AM|third|night|| +17756|AAAAAAAANFFEAAAA|17756|4|55|56|AM|third|night|| +17757|AAAAAAAAOFFEAAAA|17757|4|55|57|AM|third|night|| +17758|AAAAAAAAPFFEAAAA|17758|4|55|58|AM|third|night|| +17759|AAAAAAAAAGFEAAAA|17759|4|55|59|AM|third|night|| +17760|AAAAAAAABGFEAAAA|17760|4|56|0|AM|third|night|| +17761|AAAAAAAACGFEAAAA|17761|4|56|1|AM|third|night|| +17762|AAAAAAAADGFEAAAA|17762|4|56|2|AM|third|night|| +17763|AAAAAAAAEGFEAAAA|17763|4|56|3|AM|third|night|| +17764|AAAAAAAAFGFEAAAA|17764|4|56|4|AM|third|night|| +17765|AAAAAAAAGGFEAAAA|17765|4|56|5|AM|third|night|| +17766|AAAAAAAAHGFEAAAA|17766|4|56|6|AM|third|night|| +17767|AAAAAAAAIGFEAAAA|17767|4|56|7|AM|third|night|| +17768|AAAAAAAAJGFEAAAA|17768|4|56|8|AM|third|night|| +17769|AAAAAAAAKGFEAAAA|17769|4|56|9|AM|third|night|| +17770|AAAAAAAALGFEAAAA|17770|4|56|10|AM|third|night|| +17771|AAAAAAAAMGFEAAAA|17771|4|56|11|AM|third|night|| +17772|AAAAAAAANGFEAAAA|17772|4|56|12|AM|third|night|| +17773|AAAAAAAAOGFEAAAA|17773|4|56|13|AM|third|night|| +17774|AAAAAAAAPGFEAAAA|17774|4|56|14|AM|third|night|| +17775|AAAAAAAAAHFEAAAA|17775|4|56|15|AM|third|night|| +17776|AAAAAAAABHFEAAAA|17776|4|56|16|AM|third|night|| +17777|AAAAAAAACHFEAAAA|17777|4|56|17|AM|third|night|| +17778|AAAAAAAADHFEAAAA|17778|4|56|18|AM|third|night|| +17779|AAAAAAAAEHFEAAAA|17779|4|56|19|AM|third|night|| +17780|AAAAAAAAFHFEAAAA|17780|4|56|20|AM|third|night|| +17781|AAAAAAAAGHFEAAAA|17781|4|56|21|AM|third|night|| +17782|AAAAAAAAHHFEAAAA|17782|4|56|22|AM|third|night|| +17783|AAAAAAAAIHFEAAAA|17783|4|56|23|AM|third|night|| +17784|AAAAAAAAJHFEAAAA|17784|4|56|24|AM|third|night|| +17785|AAAAAAAAKHFEAAAA|17785|4|56|25|AM|third|night|| +17786|AAAAAAAALHFEAAAA|17786|4|56|26|AM|third|night|| +17787|AAAAAAAAMHFEAAAA|17787|4|56|27|AM|third|night|| +17788|AAAAAAAANHFEAAAA|17788|4|56|28|AM|third|night|| +17789|AAAAAAAAOHFEAAAA|17789|4|56|29|AM|third|night|| +17790|AAAAAAAAPHFEAAAA|17790|4|56|30|AM|third|night|| +17791|AAAAAAAAAIFEAAAA|17791|4|56|31|AM|third|night|| +17792|AAAAAAAABIFEAAAA|17792|4|56|32|AM|third|night|| +17793|AAAAAAAACIFEAAAA|17793|4|56|33|AM|third|night|| +17794|AAAAAAAADIFEAAAA|17794|4|56|34|AM|third|night|| +17795|AAAAAAAAEIFEAAAA|17795|4|56|35|AM|third|night|| +17796|AAAAAAAAFIFEAAAA|17796|4|56|36|AM|third|night|| +17797|AAAAAAAAGIFEAAAA|17797|4|56|37|AM|third|night|| +17798|AAAAAAAAHIFEAAAA|17798|4|56|38|AM|third|night|| +17799|AAAAAAAAIIFEAAAA|17799|4|56|39|AM|third|night|| +17800|AAAAAAAAJIFEAAAA|17800|4|56|40|AM|third|night|| +17801|AAAAAAAAKIFEAAAA|17801|4|56|41|AM|third|night|| +17802|AAAAAAAALIFEAAAA|17802|4|56|42|AM|third|night|| +17803|AAAAAAAAMIFEAAAA|17803|4|56|43|AM|third|night|| +17804|AAAAAAAANIFEAAAA|17804|4|56|44|AM|third|night|| +17805|AAAAAAAAOIFEAAAA|17805|4|56|45|AM|third|night|| +17806|AAAAAAAAPIFEAAAA|17806|4|56|46|AM|third|night|| +17807|AAAAAAAAAJFEAAAA|17807|4|56|47|AM|third|night|| +17808|AAAAAAAABJFEAAAA|17808|4|56|48|AM|third|night|| +17809|AAAAAAAACJFEAAAA|17809|4|56|49|AM|third|night|| +17810|AAAAAAAADJFEAAAA|17810|4|56|50|AM|third|night|| +17811|AAAAAAAAEJFEAAAA|17811|4|56|51|AM|third|night|| +17812|AAAAAAAAFJFEAAAA|17812|4|56|52|AM|third|night|| +17813|AAAAAAAAGJFEAAAA|17813|4|56|53|AM|third|night|| +17814|AAAAAAAAHJFEAAAA|17814|4|56|54|AM|third|night|| +17815|AAAAAAAAIJFEAAAA|17815|4|56|55|AM|third|night|| +17816|AAAAAAAAJJFEAAAA|17816|4|56|56|AM|third|night|| +17817|AAAAAAAAKJFEAAAA|17817|4|56|57|AM|third|night|| +17818|AAAAAAAALJFEAAAA|17818|4|56|58|AM|third|night|| +17819|AAAAAAAAMJFEAAAA|17819|4|56|59|AM|third|night|| +17820|AAAAAAAANJFEAAAA|17820|4|57|0|AM|third|night|| +17821|AAAAAAAAOJFEAAAA|17821|4|57|1|AM|third|night|| +17822|AAAAAAAAPJFEAAAA|17822|4|57|2|AM|third|night|| +17823|AAAAAAAAAKFEAAAA|17823|4|57|3|AM|third|night|| +17824|AAAAAAAABKFEAAAA|17824|4|57|4|AM|third|night|| +17825|AAAAAAAACKFEAAAA|17825|4|57|5|AM|third|night|| +17826|AAAAAAAADKFEAAAA|17826|4|57|6|AM|third|night|| +17827|AAAAAAAAEKFEAAAA|17827|4|57|7|AM|third|night|| +17828|AAAAAAAAFKFEAAAA|17828|4|57|8|AM|third|night|| +17829|AAAAAAAAGKFEAAAA|17829|4|57|9|AM|third|night|| +17830|AAAAAAAAHKFEAAAA|17830|4|57|10|AM|third|night|| +17831|AAAAAAAAIKFEAAAA|17831|4|57|11|AM|third|night|| +17832|AAAAAAAAJKFEAAAA|17832|4|57|12|AM|third|night|| +17833|AAAAAAAAKKFEAAAA|17833|4|57|13|AM|third|night|| +17834|AAAAAAAALKFEAAAA|17834|4|57|14|AM|third|night|| +17835|AAAAAAAAMKFEAAAA|17835|4|57|15|AM|third|night|| +17836|AAAAAAAANKFEAAAA|17836|4|57|16|AM|third|night|| +17837|AAAAAAAAOKFEAAAA|17837|4|57|17|AM|third|night|| +17838|AAAAAAAAPKFEAAAA|17838|4|57|18|AM|third|night|| +17839|AAAAAAAAALFEAAAA|17839|4|57|19|AM|third|night|| +17840|AAAAAAAABLFEAAAA|17840|4|57|20|AM|third|night|| +17841|AAAAAAAACLFEAAAA|17841|4|57|21|AM|third|night|| +17842|AAAAAAAADLFEAAAA|17842|4|57|22|AM|third|night|| +17843|AAAAAAAAELFEAAAA|17843|4|57|23|AM|third|night|| +17844|AAAAAAAAFLFEAAAA|17844|4|57|24|AM|third|night|| +17845|AAAAAAAAGLFEAAAA|17845|4|57|25|AM|third|night|| +17846|AAAAAAAAHLFEAAAA|17846|4|57|26|AM|third|night|| +17847|AAAAAAAAILFEAAAA|17847|4|57|27|AM|third|night|| +17848|AAAAAAAAJLFEAAAA|17848|4|57|28|AM|third|night|| +17849|AAAAAAAAKLFEAAAA|17849|4|57|29|AM|third|night|| +17850|AAAAAAAALLFEAAAA|17850|4|57|30|AM|third|night|| +17851|AAAAAAAAMLFEAAAA|17851|4|57|31|AM|third|night|| +17852|AAAAAAAANLFEAAAA|17852|4|57|32|AM|third|night|| +17853|AAAAAAAAOLFEAAAA|17853|4|57|33|AM|third|night|| +17854|AAAAAAAAPLFEAAAA|17854|4|57|34|AM|third|night|| +17855|AAAAAAAAAMFEAAAA|17855|4|57|35|AM|third|night|| +17856|AAAAAAAABMFEAAAA|17856|4|57|36|AM|third|night|| +17857|AAAAAAAACMFEAAAA|17857|4|57|37|AM|third|night|| +17858|AAAAAAAADMFEAAAA|17858|4|57|38|AM|third|night|| +17859|AAAAAAAAEMFEAAAA|17859|4|57|39|AM|third|night|| +17860|AAAAAAAAFMFEAAAA|17860|4|57|40|AM|third|night|| +17861|AAAAAAAAGMFEAAAA|17861|4|57|41|AM|third|night|| +17862|AAAAAAAAHMFEAAAA|17862|4|57|42|AM|third|night|| +17863|AAAAAAAAIMFEAAAA|17863|4|57|43|AM|third|night|| +17864|AAAAAAAAJMFEAAAA|17864|4|57|44|AM|third|night|| +17865|AAAAAAAAKMFEAAAA|17865|4|57|45|AM|third|night|| +17866|AAAAAAAALMFEAAAA|17866|4|57|46|AM|third|night|| +17867|AAAAAAAAMMFEAAAA|17867|4|57|47|AM|third|night|| +17868|AAAAAAAANMFEAAAA|17868|4|57|48|AM|third|night|| +17869|AAAAAAAAOMFEAAAA|17869|4|57|49|AM|third|night|| +17870|AAAAAAAAPMFEAAAA|17870|4|57|50|AM|third|night|| +17871|AAAAAAAAANFEAAAA|17871|4|57|51|AM|third|night|| +17872|AAAAAAAABNFEAAAA|17872|4|57|52|AM|third|night|| +17873|AAAAAAAACNFEAAAA|17873|4|57|53|AM|third|night|| +17874|AAAAAAAADNFEAAAA|17874|4|57|54|AM|third|night|| +17875|AAAAAAAAENFEAAAA|17875|4|57|55|AM|third|night|| +17876|AAAAAAAAFNFEAAAA|17876|4|57|56|AM|third|night|| +17877|AAAAAAAAGNFEAAAA|17877|4|57|57|AM|third|night|| +17878|AAAAAAAAHNFEAAAA|17878|4|57|58|AM|third|night|| +17879|AAAAAAAAINFEAAAA|17879|4|57|59|AM|third|night|| +17880|AAAAAAAAJNFEAAAA|17880|4|58|0|AM|third|night|| +17881|AAAAAAAAKNFEAAAA|17881|4|58|1|AM|third|night|| +17882|AAAAAAAALNFEAAAA|17882|4|58|2|AM|third|night|| +17883|AAAAAAAAMNFEAAAA|17883|4|58|3|AM|third|night|| +17884|AAAAAAAANNFEAAAA|17884|4|58|4|AM|third|night|| +17885|AAAAAAAAONFEAAAA|17885|4|58|5|AM|third|night|| +17886|AAAAAAAAPNFEAAAA|17886|4|58|6|AM|third|night|| +17887|AAAAAAAAAOFEAAAA|17887|4|58|7|AM|third|night|| +17888|AAAAAAAABOFEAAAA|17888|4|58|8|AM|third|night|| +17889|AAAAAAAACOFEAAAA|17889|4|58|9|AM|third|night|| +17890|AAAAAAAADOFEAAAA|17890|4|58|10|AM|third|night|| +17891|AAAAAAAAEOFEAAAA|17891|4|58|11|AM|third|night|| +17892|AAAAAAAAFOFEAAAA|17892|4|58|12|AM|third|night|| +17893|AAAAAAAAGOFEAAAA|17893|4|58|13|AM|third|night|| +17894|AAAAAAAAHOFEAAAA|17894|4|58|14|AM|third|night|| +17895|AAAAAAAAIOFEAAAA|17895|4|58|15|AM|third|night|| +17896|AAAAAAAAJOFEAAAA|17896|4|58|16|AM|third|night|| +17897|AAAAAAAAKOFEAAAA|17897|4|58|17|AM|third|night|| +17898|AAAAAAAALOFEAAAA|17898|4|58|18|AM|third|night|| +17899|AAAAAAAAMOFEAAAA|17899|4|58|19|AM|third|night|| +17900|AAAAAAAANOFEAAAA|17900|4|58|20|AM|third|night|| +17901|AAAAAAAAOOFEAAAA|17901|4|58|21|AM|third|night|| +17902|AAAAAAAAPOFEAAAA|17902|4|58|22|AM|third|night|| +17903|AAAAAAAAAPFEAAAA|17903|4|58|23|AM|third|night|| +17904|AAAAAAAABPFEAAAA|17904|4|58|24|AM|third|night|| +17905|AAAAAAAACPFEAAAA|17905|4|58|25|AM|third|night|| +17906|AAAAAAAADPFEAAAA|17906|4|58|26|AM|third|night|| +17907|AAAAAAAAEPFEAAAA|17907|4|58|27|AM|third|night|| +17908|AAAAAAAAFPFEAAAA|17908|4|58|28|AM|third|night|| +17909|AAAAAAAAGPFEAAAA|17909|4|58|29|AM|third|night|| +17910|AAAAAAAAHPFEAAAA|17910|4|58|30|AM|third|night|| +17911|AAAAAAAAIPFEAAAA|17911|4|58|31|AM|third|night|| +17912|AAAAAAAAJPFEAAAA|17912|4|58|32|AM|third|night|| +17913|AAAAAAAAKPFEAAAA|17913|4|58|33|AM|third|night|| +17914|AAAAAAAALPFEAAAA|17914|4|58|34|AM|third|night|| +17915|AAAAAAAAMPFEAAAA|17915|4|58|35|AM|third|night|| +17916|AAAAAAAANPFEAAAA|17916|4|58|36|AM|third|night|| +17917|AAAAAAAAOPFEAAAA|17917|4|58|37|AM|third|night|| +17918|AAAAAAAAPPFEAAAA|17918|4|58|38|AM|third|night|| +17919|AAAAAAAAAAGEAAAA|17919|4|58|39|AM|third|night|| +17920|AAAAAAAABAGEAAAA|17920|4|58|40|AM|third|night|| +17921|AAAAAAAACAGEAAAA|17921|4|58|41|AM|third|night|| +17922|AAAAAAAADAGEAAAA|17922|4|58|42|AM|third|night|| +17923|AAAAAAAAEAGEAAAA|17923|4|58|43|AM|third|night|| +17924|AAAAAAAAFAGEAAAA|17924|4|58|44|AM|third|night|| +17925|AAAAAAAAGAGEAAAA|17925|4|58|45|AM|third|night|| +17926|AAAAAAAAHAGEAAAA|17926|4|58|46|AM|third|night|| +17927|AAAAAAAAIAGEAAAA|17927|4|58|47|AM|third|night|| +17928|AAAAAAAAJAGEAAAA|17928|4|58|48|AM|third|night|| +17929|AAAAAAAAKAGEAAAA|17929|4|58|49|AM|third|night|| +17930|AAAAAAAALAGEAAAA|17930|4|58|50|AM|third|night|| +17931|AAAAAAAAMAGEAAAA|17931|4|58|51|AM|third|night|| +17932|AAAAAAAANAGEAAAA|17932|4|58|52|AM|third|night|| +17933|AAAAAAAAOAGEAAAA|17933|4|58|53|AM|third|night|| +17934|AAAAAAAAPAGEAAAA|17934|4|58|54|AM|third|night|| +17935|AAAAAAAAABGEAAAA|17935|4|58|55|AM|third|night|| +17936|AAAAAAAABBGEAAAA|17936|4|58|56|AM|third|night|| +17937|AAAAAAAACBGEAAAA|17937|4|58|57|AM|third|night|| +17938|AAAAAAAADBGEAAAA|17938|4|58|58|AM|third|night|| +17939|AAAAAAAAEBGEAAAA|17939|4|58|59|AM|third|night|| +17940|AAAAAAAAFBGEAAAA|17940|4|59|0|AM|third|night|| +17941|AAAAAAAAGBGEAAAA|17941|4|59|1|AM|third|night|| +17942|AAAAAAAAHBGEAAAA|17942|4|59|2|AM|third|night|| +17943|AAAAAAAAIBGEAAAA|17943|4|59|3|AM|third|night|| +17944|AAAAAAAAJBGEAAAA|17944|4|59|4|AM|third|night|| +17945|AAAAAAAAKBGEAAAA|17945|4|59|5|AM|third|night|| +17946|AAAAAAAALBGEAAAA|17946|4|59|6|AM|third|night|| +17947|AAAAAAAAMBGEAAAA|17947|4|59|7|AM|third|night|| +17948|AAAAAAAANBGEAAAA|17948|4|59|8|AM|third|night|| +17949|AAAAAAAAOBGEAAAA|17949|4|59|9|AM|third|night|| +17950|AAAAAAAAPBGEAAAA|17950|4|59|10|AM|third|night|| +17951|AAAAAAAAACGEAAAA|17951|4|59|11|AM|third|night|| +17952|AAAAAAAABCGEAAAA|17952|4|59|12|AM|third|night|| +17953|AAAAAAAACCGEAAAA|17953|4|59|13|AM|third|night|| +17954|AAAAAAAADCGEAAAA|17954|4|59|14|AM|third|night|| +17955|AAAAAAAAECGEAAAA|17955|4|59|15|AM|third|night|| +17956|AAAAAAAAFCGEAAAA|17956|4|59|16|AM|third|night|| +17957|AAAAAAAAGCGEAAAA|17957|4|59|17|AM|third|night|| +17958|AAAAAAAAHCGEAAAA|17958|4|59|18|AM|third|night|| +17959|AAAAAAAAICGEAAAA|17959|4|59|19|AM|third|night|| +17960|AAAAAAAAJCGEAAAA|17960|4|59|20|AM|third|night|| +17961|AAAAAAAAKCGEAAAA|17961|4|59|21|AM|third|night|| +17962|AAAAAAAALCGEAAAA|17962|4|59|22|AM|third|night|| +17963|AAAAAAAAMCGEAAAA|17963|4|59|23|AM|third|night|| +17964|AAAAAAAANCGEAAAA|17964|4|59|24|AM|third|night|| +17965|AAAAAAAAOCGEAAAA|17965|4|59|25|AM|third|night|| +17966|AAAAAAAAPCGEAAAA|17966|4|59|26|AM|third|night|| +17967|AAAAAAAAADGEAAAA|17967|4|59|27|AM|third|night|| +17968|AAAAAAAABDGEAAAA|17968|4|59|28|AM|third|night|| +17969|AAAAAAAACDGEAAAA|17969|4|59|29|AM|third|night|| +17970|AAAAAAAADDGEAAAA|17970|4|59|30|AM|third|night|| +17971|AAAAAAAAEDGEAAAA|17971|4|59|31|AM|third|night|| +17972|AAAAAAAAFDGEAAAA|17972|4|59|32|AM|third|night|| +17973|AAAAAAAAGDGEAAAA|17973|4|59|33|AM|third|night|| +17974|AAAAAAAAHDGEAAAA|17974|4|59|34|AM|third|night|| +17975|AAAAAAAAIDGEAAAA|17975|4|59|35|AM|third|night|| +17976|AAAAAAAAJDGEAAAA|17976|4|59|36|AM|third|night|| +17977|AAAAAAAAKDGEAAAA|17977|4|59|37|AM|third|night|| +17978|AAAAAAAALDGEAAAA|17978|4|59|38|AM|third|night|| +17979|AAAAAAAAMDGEAAAA|17979|4|59|39|AM|third|night|| +17980|AAAAAAAANDGEAAAA|17980|4|59|40|AM|third|night|| +17981|AAAAAAAAODGEAAAA|17981|4|59|41|AM|third|night|| +17982|AAAAAAAAPDGEAAAA|17982|4|59|42|AM|third|night|| +17983|AAAAAAAAAEGEAAAA|17983|4|59|43|AM|third|night|| +17984|AAAAAAAABEGEAAAA|17984|4|59|44|AM|third|night|| +17985|AAAAAAAACEGEAAAA|17985|4|59|45|AM|third|night|| +17986|AAAAAAAADEGEAAAA|17986|4|59|46|AM|third|night|| +17987|AAAAAAAAEEGEAAAA|17987|4|59|47|AM|third|night|| +17988|AAAAAAAAFEGEAAAA|17988|4|59|48|AM|third|night|| +17989|AAAAAAAAGEGEAAAA|17989|4|59|49|AM|third|night|| +17990|AAAAAAAAHEGEAAAA|17990|4|59|50|AM|third|night|| +17991|AAAAAAAAIEGEAAAA|17991|4|59|51|AM|third|night|| +17992|AAAAAAAAJEGEAAAA|17992|4|59|52|AM|third|night|| +17993|AAAAAAAAKEGEAAAA|17993|4|59|53|AM|third|night|| +17994|AAAAAAAALEGEAAAA|17994|4|59|54|AM|third|night|| +17995|AAAAAAAAMEGEAAAA|17995|4|59|55|AM|third|night|| +17996|AAAAAAAANEGEAAAA|17996|4|59|56|AM|third|night|| +17997|AAAAAAAAOEGEAAAA|17997|4|59|57|AM|third|night|| +17998|AAAAAAAAPEGEAAAA|17998|4|59|58|AM|third|night|| +17999|AAAAAAAAAFGEAAAA|17999|4|59|59|AM|third|night|| +18000|AAAAAAAABFGEAAAA|18000|5|0|0|AM|third|night|| +18001|AAAAAAAACFGEAAAA|18001|5|0|1|AM|third|night|| +18002|AAAAAAAADFGEAAAA|18002|5|0|2|AM|third|night|| +18003|AAAAAAAAEFGEAAAA|18003|5|0|3|AM|third|night|| +18004|AAAAAAAAFFGEAAAA|18004|5|0|4|AM|third|night|| +18005|AAAAAAAAGFGEAAAA|18005|5|0|5|AM|third|night|| +18006|AAAAAAAAHFGEAAAA|18006|5|0|6|AM|third|night|| +18007|AAAAAAAAIFGEAAAA|18007|5|0|7|AM|third|night|| +18008|AAAAAAAAJFGEAAAA|18008|5|0|8|AM|third|night|| +18009|AAAAAAAAKFGEAAAA|18009|5|0|9|AM|third|night|| +18010|AAAAAAAALFGEAAAA|18010|5|0|10|AM|third|night|| +18011|AAAAAAAAMFGEAAAA|18011|5|0|11|AM|third|night|| +18012|AAAAAAAANFGEAAAA|18012|5|0|12|AM|third|night|| +18013|AAAAAAAAOFGEAAAA|18013|5|0|13|AM|third|night|| +18014|AAAAAAAAPFGEAAAA|18014|5|0|14|AM|third|night|| +18015|AAAAAAAAAGGEAAAA|18015|5|0|15|AM|third|night|| +18016|AAAAAAAABGGEAAAA|18016|5|0|16|AM|third|night|| +18017|AAAAAAAACGGEAAAA|18017|5|0|17|AM|third|night|| +18018|AAAAAAAADGGEAAAA|18018|5|0|18|AM|third|night|| +18019|AAAAAAAAEGGEAAAA|18019|5|0|19|AM|third|night|| +18020|AAAAAAAAFGGEAAAA|18020|5|0|20|AM|third|night|| +18021|AAAAAAAAGGGEAAAA|18021|5|0|21|AM|third|night|| +18022|AAAAAAAAHGGEAAAA|18022|5|0|22|AM|third|night|| +18023|AAAAAAAAIGGEAAAA|18023|5|0|23|AM|third|night|| +18024|AAAAAAAAJGGEAAAA|18024|5|0|24|AM|third|night|| +18025|AAAAAAAAKGGEAAAA|18025|5|0|25|AM|third|night|| +18026|AAAAAAAALGGEAAAA|18026|5|0|26|AM|third|night|| +18027|AAAAAAAAMGGEAAAA|18027|5|0|27|AM|third|night|| +18028|AAAAAAAANGGEAAAA|18028|5|0|28|AM|third|night|| +18029|AAAAAAAAOGGEAAAA|18029|5|0|29|AM|third|night|| +18030|AAAAAAAAPGGEAAAA|18030|5|0|30|AM|third|night|| +18031|AAAAAAAAAHGEAAAA|18031|5|0|31|AM|third|night|| +18032|AAAAAAAABHGEAAAA|18032|5|0|32|AM|third|night|| +18033|AAAAAAAACHGEAAAA|18033|5|0|33|AM|third|night|| +18034|AAAAAAAADHGEAAAA|18034|5|0|34|AM|third|night|| +18035|AAAAAAAAEHGEAAAA|18035|5|0|35|AM|third|night|| +18036|AAAAAAAAFHGEAAAA|18036|5|0|36|AM|third|night|| +18037|AAAAAAAAGHGEAAAA|18037|5|0|37|AM|third|night|| +18038|AAAAAAAAHHGEAAAA|18038|5|0|38|AM|third|night|| +18039|AAAAAAAAIHGEAAAA|18039|5|0|39|AM|third|night|| +18040|AAAAAAAAJHGEAAAA|18040|5|0|40|AM|third|night|| +18041|AAAAAAAAKHGEAAAA|18041|5|0|41|AM|third|night|| +18042|AAAAAAAALHGEAAAA|18042|5|0|42|AM|third|night|| +18043|AAAAAAAAMHGEAAAA|18043|5|0|43|AM|third|night|| +18044|AAAAAAAANHGEAAAA|18044|5|0|44|AM|third|night|| +18045|AAAAAAAAOHGEAAAA|18045|5|0|45|AM|third|night|| +18046|AAAAAAAAPHGEAAAA|18046|5|0|46|AM|third|night|| +18047|AAAAAAAAAIGEAAAA|18047|5|0|47|AM|third|night|| +18048|AAAAAAAABIGEAAAA|18048|5|0|48|AM|third|night|| +18049|AAAAAAAACIGEAAAA|18049|5|0|49|AM|third|night|| +18050|AAAAAAAADIGEAAAA|18050|5|0|50|AM|third|night|| +18051|AAAAAAAAEIGEAAAA|18051|5|0|51|AM|third|night|| +18052|AAAAAAAAFIGEAAAA|18052|5|0|52|AM|third|night|| +18053|AAAAAAAAGIGEAAAA|18053|5|0|53|AM|third|night|| +18054|AAAAAAAAHIGEAAAA|18054|5|0|54|AM|third|night|| +18055|AAAAAAAAIIGEAAAA|18055|5|0|55|AM|third|night|| +18056|AAAAAAAAJIGEAAAA|18056|5|0|56|AM|third|night|| +18057|AAAAAAAAKIGEAAAA|18057|5|0|57|AM|third|night|| +18058|AAAAAAAALIGEAAAA|18058|5|0|58|AM|third|night|| +18059|AAAAAAAAMIGEAAAA|18059|5|0|59|AM|third|night|| +18060|AAAAAAAANIGEAAAA|18060|5|1|0|AM|third|night|| +18061|AAAAAAAAOIGEAAAA|18061|5|1|1|AM|third|night|| +18062|AAAAAAAAPIGEAAAA|18062|5|1|2|AM|third|night|| +18063|AAAAAAAAAJGEAAAA|18063|5|1|3|AM|third|night|| +18064|AAAAAAAABJGEAAAA|18064|5|1|4|AM|third|night|| +18065|AAAAAAAACJGEAAAA|18065|5|1|5|AM|third|night|| +18066|AAAAAAAADJGEAAAA|18066|5|1|6|AM|third|night|| +18067|AAAAAAAAEJGEAAAA|18067|5|1|7|AM|third|night|| +18068|AAAAAAAAFJGEAAAA|18068|5|1|8|AM|third|night|| +18069|AAAAAAAAGJGEAAAA|18069|5|1|9|AM|third|night|| +18070|AAAAAAAAHJGEAAAA|18070|5|1|10|AM|third|night|| +18071|AAAAAAAAIJGEAAAA|18071|5|1|11|AM|third|night|| +18072|AAAAAAAAJJGEAAAA|18072|5|1|12|AM|third|night|| +18073|AAAAAAAAKJGEAAAA|18073|5|1|13|AM|third|night|| +18074|AAAAAAAALJGEAAAA|18074|5|1|14|AM|third|night|| +18075|AAAAAAAAMJGEAAAA|18075|5|1|15|AM|third|night|| +18076|AAAAAAAANJGEAAAA|18076|5|1|16|AM|third|night|| +18077|AAAAAAAAOJGEAAAA|18077|5|1|17|AM|third|night|| +18078|AAAAAAAAPJGEAAAA|18078|5|1|18|AM|third|night|| +18079|AAAAAAAAAKGEAAAA|18079|5|1|19|AM|third|night|| +18080|AAAAAAAABKGEAAAA|18080|5|1|20|AM|third|night|| +18081|AAAAAAAACKGEAAAA|18081|5|1|21|AM|third|night|| +18082|AAAAAAAADKGEAAAA|18082|5|1|22|AM|third|night|| +18083|AAAAAAAAEKGEAAAA|18083|5|1|23|AM|third|night|| +18084|AAAAAAAAFKGEAAAA|18084|5|1|24|AM|third|night|| +18085|AAAAAAAAGKGEAAAA|18085|5|1|25|AM|third|night|| +18086|AAAAAAAAHKGEAAAA|18086|5|1|26|AM|third|night|| +18087|AAAAAAAAIKGEAAAA|18087|5|1|27|AM|third|night|| +18088|AAAAAAAAJKGEAAAA|18088|5|1|28|AM|third|night|| +18089|AAAAAAAAKKGEAAAA|18089|5|1|29|AM|third|night|| +18090|AAAAAAAALKGEAAAA|18090|5|1|30|AM|third|night|| +18091|AAAAAAAAMKGEAAAA|18091|5|1|31|AM|third|night|| +18092|AAAAAAAANKGEAAAA|18092|5|1|32|AM|third|night|| +18093|AAAAAAAAOKGEAAAA|18093|5|1|33|AM|third|night|| +18094|AAAAAAAAPKGEAAAA|18094|5|1|34|AM|third|night|| +18095|AAAAAAAAALGEAAAA|18095|5|1|35|AM|third|night|| +18096|AAAAAAAABLGEAAAA|18096|5|1|36|AM|third|night|| +18097|AAAAAAAACLGEAAAA|18097|5|1|37|AM|third|night|| +18098|AAAAAAAADLGEAAAA|18098|5|1|38|AM|third|night|| +18099|AAAAAAAAELGEAAAA|18099|5|1|39|AM|third|night|| +18100|AAAAAAAAFLGEAAAA|18100|5|1|40|AM|third|night|| +18101|AAAAAAAAGLGEAAAA|18101|5|1|41|AM|third|night|| +18102|AAAAAAAAHLGEAAAA|18102|5|1|42|AM|third|night|| +18103|AAAAAAAAILGEAAAA|18103|5|1|43|AM|third|night|| +18104|AAAAAAAAJLGEAAAA|18104|5|1|44|AM|third|night|| +18105|AAAAAAAAKLGEAAAA|18105|5|1|45|AM|third|night|| +18106|AAAAAAAALLGEAAAA|18106|5|1|46|AM|third|night|| +18107|AAAAAAAAMLGEAAAA|18107|5|1|47|AM|third|night|| +18108|AAAAAAAANLGEAAAA|18108|5|1|48|AM|third|night|| +18109|AAAAAAAAOLGEAAAA|18109|5|1|49|AM|third|night|| +18110|AAAAAAAAPLGEAAAA|18110|5|1|50|AM|third|night|| +18111|AAAAAAAAAMGEAAAA|18111|5|1|51|AM|third|night|| +18112|AAAAAAAABMGEAAAA|18112|5|1|52|AM|third|night|| +18113|AAAAAAAACMGEAAAA|18113|5|1|53|AM|third|night|| +18114|AAAAAAAADMGEAAAA|18114|5|1|54|AM|third|night|| +18115|AAAAAAAAEMGEAAAA|18115|5|1|55|AM|third|night|| +18116|AAAAAAAAFMGEAAAA|18116|5|1|56|AM|third|night|| +18117|AAAAAAAAGMGEAAAA|18117|5|1|57|AM|third|night|| +18118|AAAAAAAAHMGEAAAA|18118|5|1|58|AM|third|night|| +18119|AAAAAAAAIMGEAAAA|18119|5|1|59|AM|third|night|| +18120|AAAAAAAAJMGEAAAA|18120|5|2|0|AM|third|night|| +18121|AAAAAAAAKMGEAAAA|18121|5|2|1|AM|third|night|| +18122|AAAAAAAALMGEAAAA|18122|5|2|2|AM|third|night|| +18123|AAAAAAAAMMGEAAAA|18123|5|2|3|AM|third|night|| +18124|AAAAAAAANMGEAAAA|18124|5|2|4|AM|third|night|| +18125|AAAAAAAAOMGEAAAA|18125|5|2|5|AM|third|night|| +18126|AAAAAAAAPMGEAAAA|18126|5|2|6|AM|third|night|| +18127|AAAAAAAAANGEAAAA|18127|5|2|7|AM|third|night|| +18128|AAAAAAAABNGEAAAA|18128|5|2|8|AM|third|night|| +18129|AAAAAAAACNGEAAAA|18129|5|2|9|AM|third|night|| +18130|AAAAAAAADNGEAAAA|18130|5|2|10|AM|third|night|| +18131|AAAAAAAAENGEAAAA|18131|5|2|11|AM|third|night|| +18132|AAAAAAAAFNGEAAAA|18132|5|2|12|AM|third|night|| +18133|AAAAAAAAGNGEAAAA|18133|5|2|13|AM|third|night|| +18134|AAAAAAAAHNGEAAAA|18134|5|2|14|AM|third|night|| +18135|AAAAAAAAINGEAAAA|18135|5|2|15|AM|third|night|| +18136|AAAAAAAAJNGEAAAA|18136|5|2|16|AM|third|night|| +18137|AAAAAAAAKNGEAAAA|18137|5|2|17|AM|third|night|| +18138|AAAAAAAALNGEAAAA|18138|5|2|18|AM|third|night|| +18139|AAAAAAAAMNGEAAAA|18139|5|2|19|AM|third|night|| +18140|AAAAAAAANNGEAAAA|18140|5|2|20|AM|third|night|| +18141|AAAAAAAAONGEAAAA|18141|5|2|21|AM|third|night|| +18142|AAAAAAAAPNGEAAAA|18142|5|2|22|AM|third|night|| +18143|AAAAAAAAAOGEAAAA|18143|5|2|23|AM|third|night|| +18144|AAAAAAAABOGEAAAA|18144|5|2|24|AM|third|night|| +18145|AAAAAAAACOGEAAAA|18145|5|2|25|AM|third|night|| +18146|AAAAAAAADOGEAAAA|18146|5|2|26|AM|third|night|| +18147|AAAAAAAAEOGEAAAA|18147|5|2|27|AM|third|night|| +18148|AAAAAAAAFOGEAAAA|18148|5|2|28|AM|third|night|| +18149|AAAAAAAAGOGEAAAA|18149|5|2|29|AM|third|night|| +18150|AAAAAAAAHOGEAAAA|18150|5|2|30|AM|third|night|| +18151|AAAAAAAAIOGEAAAA|18151|5|2|31|AM|third|night|| +18152|AAAAAAAAJOGEAAAA|18152|5|2|32|AM|third|night|| +18153|AAAAAAAAKOGEAAAA|18153|5|2|33|AM|third|night|| +18154|AAAAAAAALOGEAAAA|18154|5|2|34|AM|third|night|| +18155|AAAAAAAAMOGEAAAA|18155|5|2|35|AM|third|night|| +18156|AAAAAAAANOGEAAAA|18156|5|2|36|AM|third|night|| +18157|AAAAAAAAOOGEAAAA|18157|5|2|37|AM|third|night|| +18158|AAAAAAAAPOGEAAAA|18158|5|2|38|AM|third|night|| +18159|AAAAAAAAAPGEAAAA|18159|5|2|39|AM|third|night|| +18160|AAAAAAAABPGEAAAA|18160|5|2|40|AM|third|night|| +18161|AAAAAAAACPGEAAAA|18161|5|2|41|AM|third|night|| +18162|AAAAAAAADPGEAAAA|18162|5|2|42|AM|third|night|| +18163|AAAAAAAAEPGEAAAA|18163|5|2|43|AM|third|night|| +18164|AAAAAAAAFPGEAAAA|18164|5|2|44|AM|third|night|| +18165|AAAAAAAAGPGEAAAA|18165|5|2|45|AM|third|night|| +18166|AAAAAAAAHPGEAAAA|18166|5|2|46|AM|third|night|| +18167|AAAAAAAAIPGEAAAA|18167|5|2|47|AM|third|night|| +18168|AAAAAAAAJPGEAAAA|18168|5|2|48|AM|third|night|| +18169|AAAAAAAAKPGEAAAA|18169|5|2|49|AM|third|night|| +18170|AAAAAAAALPGEAAAA|18170|5|2|50|AM|third|night|| +18171|AAAAAAAAMPGEAAAA|18171|5|2|51|AM|third|night|| +18172|AAAAAAAANPGEAAAA|18172|5|2|52|AM|third|night|| +18173|AAAAAAAAOPGEAAAA|18173|5|2|53|AM|third|night|| +18174|AAAAAAAAPPGEAAAA|18174|5|2|54|AM|third|night|| +18175|AAAAAAAAAAHEAAAA|18175|5|2|55|AM|third|night|| +18176|AAAAAAAABAHEAAAA|18176|5|2|56|AM|third|night|| +18177|AAAAAAAACAHEAAAA|18177|5|2|57|AM|third|night|| +18178|AAAAAAAADAHEAAAA|18178|5|2|58|AM|third|night|| +18179|AAAAAAAAEAHEAAAA|18179|5|2|59|AM|third|night|| +18180|AAAAAAAAFAHEAAAA|18180|5|3|0|AM|third|night|| +18181|AAAAAAAAGAHEAAAA|18181|5|3|1|AM|third|night|| +18182|AAAAAAAAHAHEAAAA|18182|5|3|2|AM|third|night|| +18183|AAAAAAAAIAHEAAAA|18183|5|3|3|AM|third|night|| +18184|AAAAAAAAJAHEAAAA|18184|5|3|4|AM|third|night|| +18185|AAAAAAAAKAHEAAAA|18185|5|3|5|AM|third|night|| +18186|AAAAAAAALAHEAAAA|18186|5|3|6|AM|third|night|| +18187|AAAAAAAAMAHEAAAA|18187|5|3|7|AM|third|night|| +18188|AAAAAAAANAHEAAAA|18188|5|3|8|AM|third|night|| +18189|AAAAAAAAOAHEAAAA|18189|5|3|9|AM|third|night|| +18190|AAAAAAAAPAHEAAAA|18190|5|3|10|AM|third|night|| +18191|AAAAAAAAABHEAAAA|18191|5|3|11|AM|third|night|| +18192|AAAAAAAABBHEAAAA|18192|5|3|12|AM|third|night|| +18193|AAAAAAAACBHEAAAA|18193|5|3|13|AM|third|night|| +18194|AAAAAAAADBHEAAAA|18194|5|3|14|AM|third|night|| +18195|AAAAAAAAEBHEAAAA|18195|5|3|15|AM|third|night|| +18196|AAAAAAAAFBHEAAAA|18196|5|3|16|AM|third|night|| +18197|AAAAAAAAGBHEAAAA|18197|5|3|17|AM|third|night|| +18198|AAAAAAAAHBHEAAAA|18198|5|3|18|AM|third|night|| +18199|AAAAAAAAIBHEAAAA|18199|5|3|19|AM|third|night|| +18200|AAAAAAAAJBHEAAAA|18200|5|3|20|AM|third|night|| +18201|AAAAAAAAKBHEAAAA|18201|5|3|21|AM|third|night|| +18202|AAAAAAAALBHEAAAA|18202|5|3|22|AM|third|night|| +18203|AAAAAAAAMBHEAAAA|18203|5|3|23|AM|third|night|| +18204|AAAAAAAANBHEAAAA|18204|5|3|24|AM|third|night|| +18205|AAAAAAAAOBHEAAAA|18205|5|3|25|AM|third|night|| +18206|AAAAAAAAPBHEAAAA|18206|5|3|26|AM|third|night|| +18207|AAAAAAAAACHEAAAA|18207|5|3|27|AM|third|night|| +18208|AAAAAAAABCHEAAAA|18208|5|3|28|AM|third|night|| +18209|AAAAAAAACCHEAAAA|18209|5|3|29|AM|third|night|| +18210|AAAAAAAADCHEAAAA|18210|5|3|30|AM|third|night|| +18211|AAAAAAAAECHEAAAA|18211|5|3|31|AM|third|night|| +18212|AAAAAAAAFCHEAAAA|18212|5|3|32|AM|third|night|| +18213|AAAAAAAAGCHEAAAA|18213|5|3|33|AM|third|night|| +18214|AAAAAAAAHCHEAAAA|18214|5|3|34|AM|third|night|| +18215|AAAAAAAAICHEAAAA|18215|5|3|35|AM|third|night|| +18216|AAAAAAAAJCHEAAAA|18216|5|3|36|AM|third|night|| +18217|AAAAAAAAKCHEAAAA|18217|5|3|37|AM|third|night|| +18218|AAAAAAAALCHEAAAA|18218|5|3|38|AM|third|night|| +18219|AAAAAAAAMCHEAAAA|18219|5|3|39|AM|third|night|| +18220|AAAAAAAANCHEAAAA|18220|5|3|40|AM|third|night|| +18221|AAAAAAAAOCHEAAAA|18221|5|3|41|AM|third|night|| +18222|AAAAAAAAPCHEAAAA|18222|5|3|42|AM|third|night|| +18223|AAAAAAAAADHEAAAA|18223|5|3|43|AM|third|night|| +18224|AAAAAAAABDHEAAAA|18224|5|3|44|AM|third|night|| +18225|AAAAAAAACDHEAAAA|18225|5|3|45|AM|third|night|| +18226|AAAAAAAADDHEAAAA|18226|5|3|46|AM|third|night|| +18227|AAAAAAAAEDHEAAAA|18227|5|3|47|AM|third|night|| +18228|AAAAAAAAFDHEAAAA|18228|5|3|48|AM|third|night|| +18229|AAAAAAAAGDHEAAAA|18229|5|3|49|AM|third|night|| +18230|AAAAAAAAHDHEAAAA|18230|5|3|50|AM|third|night|| +18231|AAAAAAAAIDHEAAAA|18231|5|3|51|AM|third|night|| +18232|AAAAAAAAJDHEAAAA|18232|5|3|52|AM|third|night|| +18233|AAAAAAAAKDHEAAAA|18233|5|3|53|AM|third|night|| +18234|AAAAAAAALDHEAAAA|18234|5|3|54|AM|third|night|| +18235|AAAAAAAAMDHEAAAA|18235|5|3|55|AM|third|night|| +18236|AAAAAAAANDHEAAAA|18236|5|3|56|AM|third|night|| +18237|AAAAAAAAODHEAAAA|18237|5|3|57|AM|third|night|| +18238|AAAAAAAAPDHEAAAA|18238|5|3|58|AM|third|night|| +18239|AAAAAAAAAEHEAAAA|18239|5|3|59|AM|third|night|| +18240|AAAAAAAABEHEAAAA|18240|5|4|0|AM|third|night|| +18241|AAAAAAAACEHEAAAA|18241|5|4|1|AM|third|night|| +18242|AAAAAAAADEHEAAAA|18242|5|4|2|AM|third|night|| +18243|AAAAAAAAEEHEAAAA|18243|5|4|3|AM|third|night|| +18244|AAAAAAAAFEHEAAAA|18244|5|4|4|AM|third|night|| +18245|AAAAAAAAGEHEAAAA|18245|5|4|5|AM|third|night|| +18246|AAAAAAAAHEHEAAAA|18246|5|4|6|AM|third|night|| +18247|AAAAAAAAIEHEAAAA|18247|5|4|7|AM|third|night|| +18248|AAAAAAAAJEHEAAAA|18248|5|4|8|AM|third|night|| +18249|AAAAAAAAKEHEAAAA|18249|5|4|9|AM|third|night|| +18250|AAAAAAAALEHEAAAA|18250|5|4|10|AM|third|night|| +18251|AAAAAAAAMEHEAAAA|18251|5|4|11|AM|third|night|| +18252|AAAAAAAANEHEAAAA|18252|5|4|12|AM|third|night|| +18253|AAAAAAAAOEHEAAAA|18253|5|4|13|AM|third|night|| +18254|AAAAAAAAPEHEAAAA|18254|5|4|14|AM|third|night|| +18255|AAAAAAAAAFHEAAAA|18255|5|4|15|AM|third|night|| +18256|AAAAAAAABFHEAAAA|18256|5|4|16|AM|third|night|| +18257|AAAAAAAACFHEAAAA|18257|5|4|17|AM|third|night|| +18258|AAAAAAAADFHEAAAA|18258|5|4|18|AM|third|night|| +18259|AAAAAAAAEFHEAAAA|18259|5|4|19|AM|third|night|| +18260|AAAAAAAAFFHEAAAA|18260|5|4|20|AM|third|night|| +18261|AAAAAAAAGFHEAAAA|18261|5|4|21|AM|third|night|| +18262|AAAAAAAAHFHEAAAA|18262|5|4|22|AM|third|night|| +18263|AAAAAAAAIFHEAAAA|18263|5|4|23|AM|third|night|| +18264|AAAAAAAAJFHEAAAA|18264|5|4|24|AM|third|night|| +18265|AAAAAAAAKFHEAAAA|18265|5|4|25|AM|third|night|| +18266|AAAAAAAALFHEAAAA|18266|5|4|26|AM|third|night|| +18267|AAAAAAAAMFHEAAAA|18267|5|4|27|AM|third|night|| +18268|AAAAAAAANFHEAAAA|18268|5|4|28|AM|third|night|| +18269|AAAAAAAAOFHEAAAA|18269|5|4|29|AM|third|night|| +18270|AAAAAAAAPFHEAAAA|18270|5|4|30|AM|third|night|| +18271|AAAAAAAAAGHEAAAA|18271|5|4|31|AM|third|night|| +18272|AAAAAAAABGHEAAAA|18272|5|4|32|AM|third|night|| +18273|AAAAAAAACGHEAAAA|18273|5|4|33|AM|third|night|| +18274|AAAAAAAADGHEAAAA|18274|5|4|34|AM|third|night|| +18275|AAAAAAAAEGHEAAAA|18275|5|4|35|AM|third|night|| +18276|AAAAAAAAFGHEAAAA|18276|5|4|36|AM|third|night|| +18277|AAAAAAAAGGHEAAAA|18277|5|4|37|AM|third|night|| +18278|AAAAAAAAHGHEAAAA|18278|5|4|38|AM|third|night|| +18279|AAAAAAAAIGHEAAAA|18279|5|4|39|AM|third|night|| +18280|AAAAAAAAJGHEAAAA|18280|5|4|40|AM|third|night|| +18281|AAAAAAAAKGHEAAAA|18281|5|4|41|AM|third|night|| +18282|AAAAAAAALGHEAAAA|18282|5|4|42|AM|third|night|| +18283|AAAAAAAAMGHEAAAA|18283|5|4|43|AM|third|night|| +18284|AAAAAAAANGHEAAAA|18284|5|4|44|AM|third|night|| +18285|AAAAAAAAOGHEAAAA|18285|5|4|45|AM|third|night|| +18286|AAAAAAAAPGHEAAAA|18286|5|4|46|AM|third|night|| +18287|AAAAAAAAAHHEAAAA|18287|5|4|47|AM|third|night|| +18288|AAAAAAAABHHEAAAA|18288|5|4|48|AM|third|night|| +18289|AAAAAAAACHHEAAAA|18289|5|4|49|AM|third|night|| +18290|AAAAAAAADHHEAAAA|18290|5|4|50|AM|third|night|| +18291|AAAAAAAAEHHEAAAA|18291|5|4|51|AM|third|night|| +18292|AAAAAAAAFHHEAAAA|18292|5|4|52|AM|third|night|| +18293|AAAAAAAAGHHEAAAA|18293|5|4|53|AM|third|night|| +18294|AAAAAAAAHHHEAAAA|18294|5|4|54|AM|third|night|| +18295|AAAAAAAAIHHEAAAA|18295|5|4|55|AM|third|night|| +18296|AAAAAAAAJHHEAAAA|18296|5|4|56|AM|third|night|| +18297|AAAAAAAAKHHEAAAA|18297|5|4|57|AM|third|night|| +18298|AAAAAAAALHHEAAAA|18298|5|4|58|AM|third|night|| +18299|AAAAAAAAMHHEAAAA|18299|5|4|59|AM|third|night|| +18300|AAAAAAAANHHEAAAA|18300|5|5|0|AM|third|night|| +18301|AAAAAAAAOHHEAAAA|18301|5|5|1|AM|third|night|| +18302|AAAAAAAAPHHEAAAA|18302|5|5|2|AM|third|night|| +18303|AAAAAAAAAIHEAAAA|18303|5|5|3|AM|third|night|| +18304|AAAAAAAABIHEAAAA|18304|5|5|4|AM|third|night|| +18305|AAAAAAAACIHEAAAA|18305|5|5|5|AM|third|night|| +18306|AAAAAAAADIHEAAAA|18306|5|5|6|AM|third|night|| +18307|AAAAAAAAEIHEAAAA|18307|5|5|7|AM|third|night|| +18308|AAAAAAAAFIHEAAAA|18308|5|5|8|AM|third|night|| +18309|AAAAAAAAGIHEAAAA|18309|5|5|9|AM|third|night|| +18310|AAAAAAAAHIHEAAAA|18310|5|5|10|AM|third|night|| +18311|AAAAAAAAIIHEAAAA|18311|5|5|11|AM|third|night|| +18312|AAAAAAAAJIHEAAAA|18312|5|5|12|AM|third|night|| +18313|AAAAAAAAKIHEAAAA|18313|5|5|13|AM|third|night|| +18314|AAAAAAAALIHEAAAA|18314|5|5|14|AM|third|night|| +18315|AAAAAAAAMIHEAAAA|18315|5|5|15|AM|third|night|| +18316|AAAAAAAANIHEAAAA|18316|5|5|16|AM|third|night|| +18317|AAAAAAAAOIHEAAAA|18317|5|5|17|AM|third|night|| +18318|AAAAAAAAPIHEAAAA|18318|5|5|18|AM|third|night|| +18319|AAAAAAAAAJHEAAAA|18319|5|5|19|AM|third|night|| +18320|AAAAAAAABJHEAAAA|18320|5|5|20|AM|third|night|| +18321|AAAAAAAACJHEAAAA|18321|5|5|21|AM|third|night|| +18322|AAAAAAAADJHEAAAA|18322|5|5|22|AM|third|night|| +18323|AAAAAAAAEJHEAAAA|18323|5|5|23|AM|third|night|| +18324|AAAAAAAAFJHEAAAA|18324|5|5|24|AM|third|night|| +18325|AAAAAAAAGJHEAAAA|18325|5|5|25|AM|third|night|| +18326|AAAAAAAAHJHEAAAA|18326|5|5|26|AM|third|night|| +18327|AAAAAAAAIJHEAAAA|18327|5|5|27|AM|third|night|| +18328|AAAAAAAAJJHEAAAA|18328|5|5|28|AM|third|night|| +18329|AAAAAAAAKJHEAAAA|18329|5|5|29|AM|third|night|| +18330|AAAAAAAALJHEAAAA|18330|5|5|30|AM|third|night|| +18331|AAAAAAAAMJHEAAAA|18331|5|5|31|AM|third|night|| +18332|AAAAAAAANJHEAAAA|18332|5|5|32|AM|third|night|| +18333|AAAAAAAAOJHEAAAA|18333|5|5|33|AM|third|night|| +18334|AAAAAAAAPJHEAAAA|18334|5|5|34|AM|third|night|| +18335|AAAAAAAAAKHEAAAA|18335|5|5|35|AM|third|night|| +18336|AAAAAAAABKHEAAAA|18336|5|5|36|AM|third|night|| +18337|AAAAAAAACKHEAAAA|18337|5|5|37|AM|third|night|| +18338|AAAAAAAADKHEAAAA|18338|5|5|38|AM|third|night|| +18339|AAAAAAAAEKHEAAAA|18339|5|5|39|AM|third|night|| +18340|AAAAAAAAFKHEAAAA|18340|5|5|40|AM|third|night|| +18341|AAAAAAAAGKHEAAAA|18341|5|5|41|AM|third|night|| +18342|AAAAAAAAHKHEAAAA|18342|5|5|42|AM|third|night|| +18343|AAAAAAAAIKHEAAAA|18343|5|5|43|AM|third|night|| +18344|AAAAAAAAJKHEAAAA|18344|5|5|44|AM|third|night|| +18345|AAAAAAAAKKHEAAAA|18345|5|5|45|AM|third|night|| +18346|AAAAAAAALKHEAAAA|18346|5|5|46|AM|third|night|| +18347|AAAAAAAAMKHEAAAA|18347|5|5|47|AM|third|night|| +18348|AAAAAAAANKHEAAAA|18348|5|5|48|AM|third|night|| +18349|AAAAAAAAOKHEAAAA|18349|5|5|49|AM|third|night|| +18350|AAAAAAAAPKHEAAAA|18350|5|5|50|AM|third|night|| +18351|AAAAAAAAALHEAAAA|18351|5|5|51|AM|third|night|| +18352|AAAAAAAABLHEAAAA|18352|5|5|52|AM|third|night|| +18353|AAAAAAAACLHEAAAA|18353|5|5|53|AM|third|night|| +18354|AAAAAAAADLHEAAAA|18354|5|5|54|AM|third|night|| +18355|AAAAAAAAELHEAAAA|18355|5|5|55|AM|third|night|| +18356|AAAAAAAAFLHEAAAA|18356|5|5|56|AM|third|night|| +18357|AAAAAAAAGLHEAAAA|18357|5|5|57|AM|third|night|| +18358|AAAAAAAAHLHEAAAA|18358|5|5|58|AM|third|night|| +18359|AAAAAAAAILHEAAAA|18359|5|5|59|AM|third|night|| +18360|AAAAAAAAJLHEAAAA|18360|5|6|0|AM|third|night|| +18361|AAAAAAAAKLHEAAAA|18361|5|6|1|AM|third|night|| +18362|AAAAAAAALLHEAAAA|18362|5|6|2|AM|third|night|| +18363|AAAAAAAAMLHEAAAA|18363|5|6|3|AM|third|night|| +18364|AAAAAAAANLHEAAAA|18364|5|6|4|AM|third|night|| +18365|AAAAAAAAOLHEAAAA|18365|5|6|5|AM|third|night|| +18366|AAAAAAAAPLHEAAAA|18366|5|6|6|AM|third|night|| +18367|AAAAAAAAAMHEAAAA|18367|5|6|7|AM|third|night|| +18368|AAAAAAAABMHEAAAA|18368|5|6|8|AM|third|night|| +18369|AAAAAAAACMHEAAAA|18369|5|6|9|AM|third|night|| +18370|AAAAAAAADMHEAAAA|18370|5|6|10|AM|third|night|| +18371|AAAAAAAAEMHEAAAA|18371|5|6|11|AM|third|night|| +18372|AAAAAAAAFMHEAAAA|18372|5|6|12|AM|third|night|| +18373|AAAAAAAAGMHEAAAA|18373|5|6|13|AM|third|night|| +18374|AAAAAAAAHMHEAAAA|18374|5|6|14|AM|third|night|| +18375|AAAAAAAAIMHEAAAA|18375|5|6|15|AM|third|night|| +18376|AAAAAAAAJMHEAAAA|18376|5|6|16|AM|third|night|| +18377|AAAAAAAAKMHEAAAA|18377|5|6|17|AM|third|night|| +18378|AAAAAAAALMHEAAAA|18378|5|6|18|AM|third|night|| +18379|AAAAAAAAMMHEAAAA|18379|5|6|19|AM|third|night|| +18380|AAAAAAAANMHEAAAA|18380|5|6|20|AM|third|night|| +18381|AAAAAAAAOMHEAAAA|18381|5|6|21|AM|third|night|| +18382|AAAAAAAAPMHEAAAA|18382|5|6|22|AM|third|night|| +18383|AAAAAAAAANHEAAAA|18383|5|6|23|AM|third|night|| +18384|AAAAAAAABNHEAAAA|18384|5|6|24|AM|third|night|| +18385|AAAAAAAACNHEAAAA|18385|5|6|25|AM|third|night|| +18386|AAAAAAAADNHEAAAA|18386|5|6|26|AM|third|night|| +18387|AAAAAAAAENHEAAAA|18387|5|6|27|AM|third|night|| +18388|AAAAAAAAFNHEAAAA|18388|5|6|28|AM|third|night|| +18389|AAAAAAAAGNHEAAAA|18389|5|6|29|AM|third|night|| +18390|AAAAAAAAHNHEAAAA|18390|5|6|30|AM|third|night|| +18391|AAAAAAAAINHEAAAA|18391|5|6|31|AM|third|night|| +18392|AAAAAAAAJNHEAAAA|18392|5|6|32|AM|third|night|| +18393|AAAAAAAAKNHEAAAA|18393|5|6|33|AM|third|night|| +18394|AAAAAAAALNHEAAAA|18394|5|6|34|AM|third|night|| +18395|AAAAAAAAMNHEAAAA|18395|5|6|35|AM|third|night|| +18396|AAAAAAAANNHEAAAA|18396|5|6|36|AM|third|night|| +18397|AAAAAAAAONHEAAAA|18397|5|6|37|AM|third|night|| +18398|AAAAAAAAPNHEAAAA|18398|5|6|38|AM|third|night|| +18399|AAAAAAAAAOHEAAAA|18399|5|6|39|AM|third|night|| +18400|AAAAAAAABOHEAAAA|18400|5|6|40|AM|third|night|| +18401|AAAAAAAACOHEAAAA|18401|5|6|41|AM|third|night|| +18402|AAAAAAAADOHEAAAA|18402|5|6|42|AM|third|night|| +18403|AAAAAAAAEOHEAAAA|18403|5|6|43|AM|third|night|| +18404|AAAAAAAAFOHEAAAA|18404|5|6|44|AM|third|night|| +18405|AAAAAAAAGOHEAAAA|18405|5|6|45|AM|third|night|| +18406|AAAAAAAAHOHEAAAA|18406|5|6|46|AM|third|night|| +18407|AAAAAAAAIOHEAAAA|18407|5|6|47|AM|third|night|| +18408|AAAAAAAAJOHEAAAA|18408|5|6|48|AM|third|night|| +18409|AAAAAAAAKOHEAAAA|18409|5|6|49|AM|third|night|| +18410|AAAAAAAALOHEAAAA|18410|5|6|50|AM|third|night|| +18411|AAAAAAAAMOHEAAAA|18411|5|6|51|AM|third|night|| +18412|AAAAAAAANOHEAAAA|18412|5|6|52|AM|third|night|| +18413|AAAAAAAAOOHEAAAA|18413|5|6|53|AM|third|night|| +18414|AAAAAAAAPOHEAAAA|18414|5|6|54|AM|third|night|| +18415|AAAAAAAAAPHEAAAA|18415|5|6|55|AM|third|night|| +18416|AAAAAAAABPHEAAAA|18416|5|6|56|AM|third|night|| +18417|AAAAAAAACPHEAAAA|18417|5|6|57|AM|third|night|| +18418|AAAAAAAADPHEAAAA|18418|5|6|58|AM|third|night|| +18419|AAAAAAAAEPHEAAAA|18419|5|6|59|AM|third|night|| +18420|AAAAAAAAFPHEAAAA|18420|5|7|0|AM|third|night|| +18421|AAAAAAAAGPHEAAAA|18421|5|7|1|AM|third|night|| +18422|AAAAAAAAHPHEAAAA|18422|5|7|2|AM|third|night|| +18423|AAAAAAAAIPHEAAAA|18423|5|7|3|AM|third|night|| +18424|AAAAAAAAJPHEAAAA|18424|5|7|4|AM|third|night|| +18425|AAAAAAAAKPHEAAAA|18425|5|7|5|AM|third|night|| +18426|AAAAAAAALPHEAAAA|18426|5|7|6|AM|third|night|| +18427|AAAAAAAAMPHEAAAA|18427|5|7|7|AM|third|night|| +18428|AAAAAAAANPHEAAAA|18428|5|7|8|AM|third|night|| +18429|AAAAAAAAOPHEAAAA|18429|5|7|9|AM|third|night|| +18430|AAAAAAAAPPHEAAAA|18430|5|7|10|AM|third|night|| +18431|AAAAAAAAAAIEAAAA|18431|5|7|11|AM|third|night|| +18432|AAAAAAAABAIEAAAA|18432|5|7|12|AM|third|night|| +18433|AAAAAAAACAIEAAAA|18433|5|7|13|AM|third|night|| +18434|AAAAAAAADAIEAAAA|18434|5|7|14|AM|third|night|| +18435|AAAAAAAAEAIEAAAA|18435|5|7|15|AM|third|night|| +18436|AAAAAAAAFAIEAAAA|18436|5|7|16|AM|third|night|| +18437|AAAAAAAAGAIEAAAA|18437|5|7|17|AM|third|night|| +18438|AAAAAAAAHAIEAAAA|18438|5|7|18|AM|third|night|| +18439|AAAAAAAAIAIEAAAA|18439|5|7|19|AM|third|night|| +18440|AAAAAAAAJAIEAAAA|18440|5|7|20|AM|third|night|| +18441|AAAAAAAAKAIEAAAA|18441|5|7|21|AM|third|night|| +18442|AAAAAAAALAIEAAAA|18442|5|7|22|AM|third|night|| +18443|AAAAAAAAMAIEAAAA|18443|5|7|23|AM|third|night|| +18444|AAAAAAAANAIEAAAA|18444|5|7|24|AM|third|night|| +18445|AAAAAAAAOAIEAAAA|18445|5|7|25|AM|third|night|| +18446|AAAAAAAAPAIEAAAA|18446|5|7|26|AM|third|night|| +18447|AAAAAAAAABIEAAAA|18447|5|7|27|AM|third|night|| +18448|AAAAAAAABBIEAAAA|18448|5|7|28|AM|third|night|| +18449|AAAAAAAACBIEAAAA|18449|5|7|29|AM|third|night|| +18450|AAAAAAAADBIEAAAA|18450|5|7|30|AM|third|night|| +18451|AAAAAAAAEBIEAAAA|18451|5|7|31|AM|third|night|| +18452|AAAAAAAAFBIEAAAA|18452|5|7|32|AM|third|night|| +18453|AAAAAAAAGBIEAAAA|18453|5|7|33|AM|third|night|| +18454|AAAAAAAAHBIEAAAA|18454|5|7|34|AM|third|night|| +18455|AAAAAAAAIBIEAAAA|18455|5|7|35|AM|third|night|| +18456|AAAAAAAAJBIEAAAA|18456|5|7|36|AM|third|night|| +18457|AAAAAAAAKBIEAAAA|18457|5|7|37|AM|third|night|| +18458|AAAAAAAALBIEAAAA|18458|5|7|38|AM|third|night|| +18459|AAAAAAAAMBIEAAAA|18459|5|7|39|AM|third|night|| +18460|AAAAAAAANBIEAAAA|18460|5|7|40|AM|third|night|| +18461|AAAAAAAAOBIEAAAA|18461|5|7|41|AM|third|night|| +18462|AAAAAAAAPBIEAAAA|18462|5|7|42|AM|third|night|| +18463|AAAAAAAAACIEAAAA|18463|5|7|43|AM|third|night|| +18464|AAAAAAAABCIEAAAA|18464|5|7|44|AM|third|night|| +18465|AAAAAAAACCIEAAAA|18465|5|7|45|AM|third|night|| +18466|AAAAAAAADCIEAAAA|18466|5|7|46|AM|third|night|| +18467|AAAAAAAAECIEAAAA|18467|5|7|47|AM|third|night|| +18468|AAAAAAAAFCIEAAAA|18468|5|7|48|AM|third|night|| +18469|AAAAAAAAGCIEAAAA|18469|5|7|49|AM|third|night|| +18470|AAAAAAAAHCIEAAAA|18470|5|7|50|AM|third|night|| +18471|AAAAAAAAICIEAAAA|18471|5|7|51|AM|third|night|| +18472|AAAAAAAAJCIEAAAA|18472|5|7|52|AM|third|night|| +18473|AAAAAAAAKCIEAAAA|18473|5|7|53|AM|third|night|| +18474|AAAAAAAALCIEAAAA|18474|5|7|54|AM|third|night|| +18475|AAAAAAAAMCIEAAAA|18475|5|7|55|AM|third|night|| +18476|AAAAAAAANCIEAAAA|18476|5|7|56|AM|third|night|| +18477|AAAAAAAAOCIEAAAA|18477|5|7|57|AM|third|night|| +18478|AAAAAAAAPCIEAAAA|18478|5|7|58|AM|third|night|| +18479|AAAAAAAAADIEAAAA|18479|5|7|59|AM|third|night|| +18480|AAAAAAAABDIEAAAA|18480|5|8|0|AM|third|night|| +18481|AAAAAAAACDIEAAAA|18481|5|8|1|AM|third|night|| +18482|AAAAAAAADDIEAAAA|18482|5|8|2|AM|third|night|| +18483|AAAAAAAAEDIEAAAA|18483|5|8|3|AM|third|night|| +18484|AAAAAAAAFDIEAAAA|18484|5|8|4|AM|third|night|| +18485|AAAAAAAAGDIEAAAA|18485|5|8|5|AM|third|night|| +18486|AAAAAAAAHDIEAAAA|18486|5|8|6|AM|third|night|| +18487|AAAAAAAAIDIEAAAA|18487|5|8|7|AM|third|night|| +18488|AAAAAAAAJDIEAAAA|18488|5|8|8|AM|third|night|| +18489|AAAAAAAAKDIEAAAA|18489|5|8|9|AM|third|night|| +18490|AAAAAAAALDIEAAAA|18490|5|8|10|AM|third|night|| +18491|AAAAAAAAMDIEAAAA|18491|5|8|11|AM|third|night|| +18492|AAAAAAAANDIEAAAA|18492|5|8|12|AM|third|night|| +18493|AAAAAAAAODIEAAAA|18493|5|8|13|AM|third|night|| +18494|AAAAAAAAPDIEAAAA|18494|5|8|14|AM|third|night|| +18495|AAAAAAAAAEIEAAAA|18495|5|8|15|AM|third|night|| +18496|AAAAAAAABEIEAAAA|18496|5|8|16|AM|third|night|| +18497|AAAAAAAACEIEAAAA|18497|5|8|17|AM|third|night|| +18498|AAAAAAAADEIEAAAA|18498|5|8|18|AM|third|night|| +18499|AAAAAAAAEEIEAAAA|18499|5|8|19|AM|third|night|| +18500|AAAAAAAAFEIEAAAA|18500|5|8|20|AM|third|night|| +18501|AAAAAAAAGEIEAAAA|18501|5|8|21|AM|third|night|| +18502|AAAAAAAAHEIEAAAA|18502|5|8|22|AM|third|night|| +18503|AAAAAAAAIEIEAAAA|18503|5|8|23|AM|third|night|| +18504|AAAAAAAAJEIEAAAA|18504|5|8|24|AM|third|night|| +18505|AAAAAAAAKEIEAAAA|18505|5|8|25|AM|third|night|| +18506|AAAAAAAALEIEAAAA|18506|5|8|26|AM|third|night|| +18507|AAAAAAAAMEIEAAAA|18507|5|8|27|AM|third|night|| +18508|AAAAAAAANEIEAAAA|18508|5|8|28|AM|third|night|| +18509|AAAAAAAAOEIEAAAA|18509|5|8|29|AM|third|night|| +18510|AAAAAAAAPEIEAAAA|18510|5|8|30|AM|third|night|| +18511|AAAAAAAAAFIEAAAA|18511|5|8|31|AM|third|night|| +18512|AAAAAAAABFIEAAAA|18512|5|8|32|AM|third|night|| +18513|AAAAAAAACFIEAAAA|18513|5|8|33|AM|third|night|| +18514|AAAAAAAADFIEAAAA|18514|5|8|34|AM|third|night|| +18515|AAAAAAAAEFIEAAAA|18515|5|8|35|AM|third|night|| +18516|AAAAAAAAFFIEAAAA|18516|5|8|36|AM|third|night|| +18517|AAAAAAAAGFIEAAAA|18517|5|8|37|AM|third|night|| +18518|AAAAAAAAHFIEAAAA|18518|5|8|38|AM|third|night|| +18519|AAAAAAAAIFIEAAAA|18519|5|8|39|AM|third|night|| +18520|AAAAAAAAJFIEAAAA|18520|5|8|40|AM|third|night|| +18521|AAAAAAAAKFIEAAAA|18521|5|8|41|AM|third|night|| +18522|AAAAAAAALFIEAAAA|18522|5|8|42|AM|third|night|| +18523|AAAAAAAAMFIEAAAA|18523|5|8|43|AM|third|night|| +18524|AAAAAAAANFIEAAAA|18524|5|8|44|AM|third|night|| +18525|AAAAAAAAOFIEAAAA|18525|5|8|45|AM|third|night|| +18526|AAAAAAAAPFIEAAAA|18526|5|8|46|AM|third|night|| +18527|AAAAAAAAAGIEAAAA|18527|5|8|47|AM|third|night|| +18528|AAAAAAAABGIEAAAA|18528|5|8|48|AM|third|night|| +18529|AAAAAAAACGIEAAAA|18529|5|8|49|AM|third|night|| +18530|AAAAAAAADGIEAAAA|18530|5|8|50|AM|third|night|| +18531|AAAAAAAAEGIEAAAA|18531|5|8|51|AM|third|night|| +18532|AAAAAAAAFGIEAAAA|18532|5|8|52|AM|third|night|| +18533|AAAAAAAAGGIEAAAA|18533|5|8|53|AM|third|night|| +18534|AAAAAAAAHGIEAAAA|18534|5|8|54|AM|third|night|| +18535|AAAAAAAAIGIEAAAA|18535|5|8|55|AM|third|night|| +18536|AAAAAAAAJGIEAAAA|18536|5|8|56|AM|third|night|| +18537|AAAAAAAAKGIEAAAA|18537|5|8|57|AM|third|night|| +18538|AAAAAAAALGIEAAAA|18538|5|8|58|AM|third|night|| +18539|AAAAAAAAMGIEAAAA|18539|5|8|59|AM|third|night|| +18540|AAAAAAAANGIEAAAA|18540|5|9|0|AM|third|night|| +18541|AAAAAAAAOGIEAAAA|18541|5|9|1|AM|third|night|| +18542|AAAAAAAAPGIEAAAA|18542|5|9|2|AM|third|night|| +18543|AAAAAAAAAHIEAAAA|18543|5|9|3|AM|third|night|| +18544|AAAAAAAABHIEAAAA|18544|5|9|4|AM|third|night|| +18545|AAAAAAAACHIEAAAA|18545|5|9|5|AM|third|night|| +18546|AAAAAAAADHIEAAAA|18546|5|9|6|AM|third|night|| +18547|AAAAAAAAEHIEAAAA|18547|5|9|7|AM|third|night|| +18548|AAAAAAAAFHIEAAAA|18548|5|9|8|AM|third|night|| +18549|AAAAAAAAGHIEAAAA|18549|5|9|9|AM|third|night|| +18550|AAAAAAAAHHIEAAAA|18550|5|9|10|AM|third|night|| +18551|AAAAAAAAIHIEAAAA|18551|5|9|11|AM|third|night|| +18552|AAAAAAAAJHIEAAAA|18552|5|9|12|AM|third|night|| +18553|AAAAAAAAKHIEAAAA|18553|5|9|13|AM|third|night|| +18554|AAAAAAAALHIEAAAA|18554|5|9|14|AM|third|night|| +18555|AAAAAAAAMHIEAAAA|18555|5|9|15|AM|third|night|| +18556|AAAAAAAANHIEAAAA|18556|5|9|16|AM|third|night|| +18557|AAAAAAAAOHIEAAAA|18557|5|9|17|AM|third|night|| +18558|AAAAAAAAPHIEAAAA|18558|5|9|18|AM|third|night|| +18559|AAAAAAAAAIIEAAAA|18559|5|9|19|AM|third|night|| +18560|AAAAAAAABIIEAAAA|18560|5|9|20|AM|third|night|| +18561|AAAAAAAACIIEAAAA|18561|5|9|21|AM|third|night|| +18562|AAAAAAAADIIEAAAA|18562|5|9|22|AM|third|night|| +18563|AAAAAAAAEIIEAAAA|18563|5|9|23|AM|third|night|| +18564|AAAAAAAAFIIEAAAA|18564|5|9|24|AM|third|night|| +18565|AAAAAAAAGIIEAAAA|18565|5|9|25|AM|third|night|| +18566|AAAAAAAAHIIEAAAA|18566|5|9|26|AM|third|night|| +18567|AAAAAAAAIIIEAAAA|18567|5|9|27|AM|third|night|| +18568|AAAAAAAAJIIEAAAA|18568|5|9|28|AM|third|night|| +18569|AAAAAAAAKIIEAAAA|18569|5|9|29|AM|third|night|| +18570|AAAAAAAALIIEAAAA|18570|5|9|30|AM|third|night|| +18571|AAAAAAAAMIIEAAAA|18571|5|9|31|AM|third|night|| +18572|AAAAAAAANIIEAAAA|18572|5|9|32|AM|third|night|| +18573|AAAAAAAAOIIEAAAA|18573|5|9|33|AM|third|night|| +18574|AAAAAAAAPIIEAAAA|18574|5|9|34|AM|third|night|| +18575|AAAAAAAAAJIEAAAA|18575|5|9|35|AM|third|night|| +18576|AAAAAAAABJIEAAAA|18576|5|9|36|AM|third|night|| +18577|AAAAAAAACJIEAAAA|18577|5|9|37|AM|third|night|| +18578|AAAAAAAADJIEAAAA|18578|5|9|38|AM|third|night|| +18579|AAAAAAAAEJIEAAAA|18579|5|9|39|AM|third|night|| +18580|AAAAAAAAFJIEAAAA|18580|5|9|40|AM|third|night|| +18581|AAAAAAAAGJIEAAAA|18581|5|9|41|AM|third|night|| +18582|AAAAAAAAHJIEAAAA|18582|5|9|42|AM|third|night|| +18583|AAAAAAAAIJIEAAAA|18583|5|9|43|AM|third|night|| +18584|AAAAAAAAJJIEAAAA|18584|5|9|44|AM|third|night|| +18585|AAAAAAAAKJIEAAAA|18585|5|9|45|AM|third|night|| +18586|AAAAAAAALJIEAAAA|18586|5|9|46|AM|third|night|| +18587|AAAAAAAAMJIEAAAA|18587|5|9|47|AM|third|night|| +18588|AAAAAAAANJIEAAAA|18588|5|9|48|AM|third|night|| +18589|AAAAAAAAOJIEAAAA|18589|5|9|49|AM|third|night|| +18590|AAAAAAAAPJIEAAAA|18590|5|9|50|AM|third|night|| +18591|AAAAAAAAAKIEAAAA|18591|5|9|51|AM|third|night|| +18592|AAAAAAAABKIEAAAA|18592|5|9|52|AM|third|night|| +18593|AAAAAAAACKIEAAAA|18593|5|9|53|AM|third|night|| +18594|AAAAAAAADKIEAAAA|18594|5|9|54|AM|third|night|| +18595|AAAAAAAAEKIEAAAA|18595|5|9|55|AM|third|night|| +18596|AAAAAAAAFKIEAAAA|18596|5|9|56|AM|third|night|| +18597|AAAAAAAAGKIEAAAA|18597|5|9|57|AM|third|night|| +18598|AAAAAAAAHKIEAAAA|18598|5|9|58|AM|third|night|| +18599|AAAAAAAAIKIEAAAA|18599|5|9|59|AM|third|night|| +18600|AAAAAAAAJKIEAAAA|18600|5|10|0|AM|third|night|| +18601|AAAAAAAAKKIEAAAA|18601|5|10|1|AM|third|night|| +18602|AAAAAAAALKIEAAAA|18602|5|10|2|AM|third|night|| +18603|AAAAAAAAMKIEAAAA|18603|5|10|3|AM|third|night|| +18604|AAAAAAAANKIEAAAA|18604|5|10|4|AM|third|night|| +18605|AAAAAAAAOKIEAAAA|18605|5|10|5|AM|third|night|| +18606|AAAAAAAAPKIEAAAA|18606|5|10|6|AM|third|night|| +18607|AAAAAAAAALIEAAAA|18607|5|10|7|AM|third|night|| +18608|AAAAAAAABLIEAAAA|18608|5|10|8|AM|third|night|| +18609|AAAAAAAACLIEAAAA|18609|5|10|9|AM|third|night|| +18610|AAAAAAAADLIEAAAA|18610|5|10|10|AM|third|night|| +18611|AAAAAAAAELIEAAAA|18611|5|10|11|AM|third|night|| +18612|AAAAAAAAFLIEAAAA|18612|5|10|12|AM|third|night|| +18613|AAAAAAAAGLIEAAAA|18613|5|10|13|AM|third|night|| +18614|AAAAAAAAHLIEAAAA|18614|5|10|14|AM|third|night|| +18615|AAAAAAAAILIEAAAA|18615|5|10|15|AM|third|night|| +18616|AAAAAAAAJLIEAAAA|18616|5|10|16|AM|third|night|| +18617|AAAAAAAAKLIEAAAA|18617|5|10|17|AM|third|night|| +18618|AAAAAAAALLIEAAAA|18618|5|10|18|AM|third|night|| +18619|AAAAAAAAMLIEAAAA|18619|5|10|19|AM|third|night|| +18620|AAAAAAAANLIEAAAA|18620|5|10|20|AM|third|night|| +18621|AAAAAAAAOLIEAAAA|18621|5|10|21|AM|third|night|| +18622|AAAAAAAAPLIEAAAA|18622|5|10|22|AM|third|night|| +18623|AAAAAAAAAMIEAAAA|18623|5|10|23|AM|third|night|| +18624|AAAAAAAABMIEAAAA|18624|5|10|24|AM|third|night|| +18625|AAAAAAAACMIEAAAA|18625|5|10|25|AM|third|night|| +18626|AAAAAAAADMIEAAAA|18626|5|10|26|AM|third|night|| +18627|AAAAAAAAEMIEAAAA|18627|5|10|27|AM|third|night|| +18628|AAAAAAAAFMIEAAAA|18628|5|10|28|AM|third|night|| +18629|AAAAAAAAGMIEAAAA|18629|5|10|29|AM|third|night|| +18630|AAAAAAAAHMIEAAAA|18630|5|10|30|AM|third|night|| +18631|AAAAAAAAIMIEAAAA|18631|5|10|31|AM|third|night|| +18632|AAAAAAAAJMIEAAAA|18632|5|10|32|AM|third|night|| +18633|AAAAAAAAKMIEAAAA|18633|5|10|33|AM|third|night|| +18634|AAAAAAAALMIEAAAA|18634|5|10|34|AM|third|night|| +18635|AAAAAAAAMMIEAAAA|18635|5|10|35|AM|third|night|| +18636|AAAAAAAANMIEAAAA|18636|5|10|36|AM|third|night|| +18637|AAAAAAAAOMIEAAAA|18637|5|10|37|AM|third|night|| +18638|AAAAAAAAPMIEAAAA|18638|5|10|38|AM|third|night|| +18639|AAAAAAAAANIEAAAA|18639|5|10|39|AM|third|night|| +18640|AAAAAAAABNIEAAAA|18640|5|10|40|AM|third|night|| +18641|AAAAAAAACNIEAAAA|18641|5|10|41|AM|third|night|| +18642|AAAAAAAADNIEAAAA|18642|5|10|42|AM|third|night|| +18643|AAAAAAAAENIEAAAA|18643|5|10|43|AM|third|night|| +18644|AAAAAAAAFNIEAAAA|18644|5|10|44|AM|third|night|| +18645|AAAAAAAAGNIEAAAA|18645|5|10|45|AM|third|night|| +18646|AAAAAAAAHNIEAAAA|18646|5|10|46|AM|third|night|| +18647|AAAAAAAAINIEAAAA|18647|5|10|47|AM|third|night|| +18648|AAAAAAAAJNIEAAAA|18648|5|10|48|AM|third|night|| +18649|AAAAAAAAKNIEAAAA|18649|5|10|49|AM|third|night|| +18650|AAAAAAAALNIEAAAA|18650|5|10|50|AM|third|night|| +18651|AAAAAAAAMNIEAAAA|18651|5|10|51|AM|third|night|| +18652|AAAAAAAANNIEAAAA|18652|5|10|52|AM|third|night|| +18653|AAAAAAAAONIEAAAA|18653|5|10|53|AM|third|night|| +18654|AAAAAAAAPNIEAAAA|18654|5|10|54|AM|third|night|| +18655|AAAAAAAAAOIEAAAA|18655|5|10|55|AM|third|night|| +18656|AAAAAAAABOIEAAAA|18656|5|10|56|AM|third|night|| +18657|AAAAAAAACOIEAAAA|18657|5|10|57|AM|third|night|| +18658|AAAAAAAADOIEAAAA|18658|5|10|58|AM|third|night|| +18659|AAAAAAAAEOIEAAAA|18659|5|10|59|AM|third|night|| +18660|AAAAAAAAFOIEAAAA|18660|5|11|0|AM|third|night|| +18661|AAAAAAAAGOIEAAAA|18661|5|11|1|AM|third|night|| +18662|AAAAAAAAHOIEAAAA|18662|5|11|2|AM|third|night|| +18663|AAAAAAAAIOIEAAAA|18663|5|11|3|AM|third|night|| +18664|AAAAAAAAJOIEAAAA|18664|5|11|4|AM|third|night|| +18665|AAAAAAAAKOIEAAAA|18665|5|11|5|AM|third|night|| +18666|AAAAAAAALOIEAAAA|18666|5|11|6|AM|third|night|| +18667|AAAAAAAAMOIEAAAA|18667|5|11|7|AM|third|night|| +18668|AAAAAAAANOIEAAAA|18668|5|11|8|AM|third|night|| +18669|AAAAAAAAOOIEAAAA|18669|5|11|9|AM|third|night|| +18670|AAAAAAAAPOIEAAAA|18670|5|11|10|AM|third|night|| +18671|AAAAAAAAAPIEAAAA|18671|5|11|11|AM|third|night|| +18672|AAAAAAAABPIEAAAA|18672|5|11|12|AM|third|night|| +18673|AAAAAAAACPIEAAAA|18673|5|11|13|AM|third|night|| +18674|AAAAAAAADPIEAAAA|18674|5|11|14|AM|third|night|| +18675|AAAAAAAAEPIEAAAA|18675|5|11|15|AM|third|night|| +18676|AAAAAAAAFPIEAAAA|18676|5|11|16|AM|third|night|| +18677|AAAAAAAAGPIEAAAA|18677|5|11|17|AM|third|night|| +18678|AAAAAAAAHPIEAAAA|18678|5|11|18|AM|third|night|| +18679|AAAAAAAAIPIEAAAA|18679|5|11|19|AM|third|night|| +18680|AAAAAAAAJPIEAAAA|18680|5|11|20|AM|third|night|| +18681|AAAAAAAAKPIEAAAA|18681|5|11|21|AM|third|night|| +18682|AAAAAAAALPIEAAAA|18682|5|11|22|AM|third|night|| +18683|AAAAAAAAMPIEAAAA|18683|5|11|23|AM|third|night|| +18684|AAAAAAAANPIEAAAA|18684|5|11|24|AM|third|night|| +18685|AAAAAAAAOPIEAAAA|18685|5|11|25|AM|third|night|| +18686|AAAAAAAAPPIEAAAA|18686|5|11|26|AM|third|night|| +18687|AAAAAAAAAAJEAAAA|18687|5|11|27|AM|third|night|| +18688|AAAAAAAABAJEAAAA|18688|5|11|28|AM|third|night|| +18689|AAAAAAAACAJEAAAA|18689|5|11|29|AM|third|night|| +18690|AAAAAAAADAJEAAAA|18690|5|11|30|AM|third|night|| +18691|AAAAAAAAEAJEAAAA|18691|5|11|31|AM|third|night|| +18692|AAAAAAAAFAJEAAAA|18692|5|11|32|AM|third|night|| +18693|AAAAAAAAGAJEAAAA|18693|5|11|33|AM|third|night|| +18694|AAAAAAAAHAJEAAAA|18694|5|11|34|AM|third|night|| +18695|AAAAAAAAIAJEAAAA|18695|5|11|35|AM|third|night|| +18696|AAAAAAAAJAJEAAAA|18696|5|11|36|AM|third|night|| +18697|AAAAAAAAKAJEAAAA|18697|5|11|37|AM|third|night|| +18698|AAAAAAAALAJEAAAA|18698|5|11|38|AM|third|night|| +18699|AAAAAAAAMAJEAAAA|18699|5|11|39|AM|third|night|| +18700|AAAAAAAANAJEAAAA|18700|5|11|40|AM|third|night|| +18701|AAAAAAAAOAJEAAAA|18701|5|11|41|AM|third|night|| +18702|AAAAAAAAPAJEAAAA|18702|5|11|42|AM|third|night|| +18703|AAAAAAAAABJEAAAA|18703|5|11|43|AM|third|night|| +18704|AAAAAAAABBJEAAAA|18704|5|11|44|AM|third|night|| +18705|AAAAAAAACBJEAAAA|18705|5|11|45|AM|third|night|| +18706|AAAAAAAADBJEAAAA|18706|5|11|46|AM|third|night|| +18707|AAAAAAAAEBJEAAAA|18707|5|11|47|AM|third|night|| +18708|AAAAAAAAFBJEAAAA|18708|5|11|48|AM|third|night|| +18709|AAAAAAAAGBJEAAAA|18709|5|11|49|AM|third|night|| +18710|AAAAAAAAHBJEAAAA|18710|5|11|50|AM|third|night|| +18711|AAAAAAAAIBJEAAAA|18711|5|11|51|AM|third|night|| +18712|AAAAAAAAJBJEAAAA|18712|5|11|52|AM|third|night|| +18713|AAAAAAAAKBJEAAAA|18713|5|11|53|AM|third|night|| +18714|AAAAAAAALBJEAAAA|18714|5|11|54|AM|third|night|| +18715|AAAAAAAAMBJEAAAA|18715|5|11|55|AM|third|night|| +18716|AAAAAAAANBJEAAAA|18716|5|11|56|AM|third|night|| +18717|AAAAAAAAOBJEAAAA|18717|5|11|57|AM|third|night|| +18718|AAAAAAAAPBJEAAAA|18718|5|11|58|AM|third|night|| +18719|AAAAAAAAACJEAAAA|18719|5|11|59|AM|third|night|| +18720|AAAAAAAABCJEAAAA|18720|5|12|0|AM|third|night|| +18721|AAAAAAAACCJEAAAA|18721|5|12|1|AM|third|night|| +18722|AAAAAAAADCJEAAAA|18722|5|12|2|AM|third|night|| +18723|AAAAAAAAECJEAAAA|18723|5|12|3|AM|third|night|| +18724|AAAAAAAAFCJEAAAA|18724|5|12|4|AM|third|night|| +18725|AAAAAAAAGCJEAAAA|18725|5|12|5|AM|third|night|| +18726|AAAAAAAAHCJEAAAA|18726|5|12|6|AM|third|night|| +18727|AAAAAAAAICJEAAAA|18727|5|12|7|AM|third|night|| +18728|AAAAAAAAJCJEAAAA|18728|5|12|8|AM|third|night|| +18729|AAAAAAAAKCJEAAAA|18729|5|12|9|AM|third|night|| +18730|AAAAAAAALCJEAAAA|18730|5|12|10|AM|third|night|| +18731|AAAAAAAAMCJEAAAA|18731|5|12|11|AM|third|night|| +18732|AAAAAAAANCJEAAAA|18732|5|12|12|AM|third|night|| +18733|AAAAAAAAOCJEAAAA|18733|5|12|13|AM|third|night|| +18734|AAAAAAAAPCJEAAAA|18734|5|12|14|AM|third|night|| +18735|AAAAAAAAADJEAAAA|18735|5|12|15|AM|third|night|| +18736|AAAAAAAABDJEAAAA|18736|5|12|16|AM|third|night|| +18737|AAAAAAAACDJEAAAA|18737|5|12|17|AM|third|night|| +18738|AAAAAAAADDJEAAAA|18738|5|12|18|AM|third|night|| +18739|AAAAAAAAEDJEAAAA|18739|5|12|19|AM|third|night|| +18740|AAAAAAAAFDJEAAAA|18740|5|12|20|AM|third|night|| +18741|AAAAAAAAGDJEAAAA|18741|5|12|21|AM|third|night|| +18742|AAAAAAAAHDJEAAAA|18742|5|12|22|AM|third|night|| +18743|AAAAAAAAIDJEAAAA|18743|5|12|23|AM|third|night|| +18744|AAAAAAAAJDJEAAAA|18744|5|12|24|AM|third|night|| +18745|AAAAAAAAKDJEAAAA|18745|5|12|25|AM|third|night|| +18746|AAAAAAAALDJEAAAA|18746|5|12|26|AM|third|night|| +18747|AAAAAAAAMDJEAAAA|18747|5|12|27|AM|third|night|| +18748|AAAAAAAANDJEAAAA|18748|5|12|28|AM|third|night|| +18749|AAAAAAAAODJEAAAA|18749|5|12|29|AM|third|night|| +18750|AAAAAAAAPDJEAAAA|18750|5|12|30|AM|third|night|| +18751|AAAAAAAAAEJEAAAA|18751|5|12|31|AM|third|night|| +18752|AAAAAAAABEJEAAAA|18752|5|12|32|AM|third|night|| +18753|AAAAAAAACEJEAAAA|18753|5|12|33|AM|third|night|| +18754|AAAAAAAADEJEAAAA|18754|5|12|34|AM|third|night|| +18755|AAAAAAAAEEJEAAAA|18755|5|12|35|AM|third|night|| +18756|AAAAAAAAFEJEAAAA|18756|5|12|36|AM|third|night|| +18757|AAAAAAAAGEJEAAAA|18757|5|12|37|AM|third|night|| +18758|AAAAAAAAHEJEAAAA|18758|5|12|38|AM|third|night|| +18759|AAAAAAAAIEJEAAAA|18759|5|12|39|AM|third|night|| +18760|AAAAAAAAJEJEAAAA|18760|5|12|40|AM|third|night|| +18761|AAAAAAAAKEJEAAAA|18761|5|12|41|AM|third|night|| +18762|AAAAAAAALEJEAAAA|18762|5|12|42|AM|third|night|| +18763|AAAAAAAAMEJEAAAA|18763|5|12|43|AM|third|night|| +18764|AAAAAAAANEJEAAAA|18764|5|12|44|AM|third|night|| +18765|AAAAAAAAOEJEAAAA|18765|5|12|45|AM|third|night|| +18766|AAAAAAAAPEJEAAAA|18766|5|12|46|AM|third|night|| +18767|AAAAAAAAAFJEAAAA|18767|5|12|47|AM|third|night|| +18768|AAAAAAAABFJEAAAA|18768|5|12|48|AM|third|night|| +18769|AAAAAAAACFJEAAAA|18769|5|12|49|AM|third|night|| +18770|AAAAAAAADFJEAAAA|18770|5|12|50|AM|third|night|| +18771|AAAAAAAAEFJEAAAA|18771|5|12|51|AM|third|night|| +18772|AAAAAAAAFFJEAAAA|18772|5|12|52|AM|third|night|| +18773|AAAAAAAAGFJEAAAA|18773|5|12|53|AM|third|night|| +18774|AAAAAAAAHFJEAAAA|18774|5|12|54|AM|third|night|| +18775|AAAAAAAAIFJEAAAA|18775|5|12|55|AM|third|night|| +18776|AAAAAAAAJFJEAAAA|18776|5|12|56|AM|third|night|| +18777|AAAAAAAAKFJEAAAA|18777|5|12|57|AM|third|night|| +18778|AAAAAAAALFJEAAAA|18778|5|12|58|AM|third|night|| +18779|AAAAAAAAMFJEAAAA|18779|5|12|59|AM|third|night|| +18780|AAAAAAAANFJEAAAA|18780|5|13|0|AM|third|night|| +18781|AAAAAAAAOFJEAAAA|18781|5|13|1|AM|third|night|| +18782|AAAAAAAAPFJEAAAA|18782|5|13|2|AM|third|night|| +18783|AAAAAAAAAGJEAAAA|18783|5|13|3|AM|third|night|| +18784|AAAAAAAABGJEAAAA|18784|5|13|4|AM|third|night|| +18785|AAAAAAAACGJEAAAA|18785|5|13|5|AM|third|night|| +18786|AAAAAAAADGJEAAAA|18786|5|13|6|AM|third|night|| +18787|AAAAAAAAEGJEAAAA|18787|5|13|7|AM|third|night|| +18788|AAAAAAAAFGJEAAAA|18788|5|13|8|AM|third|night|| +18789|AAAAAAAAGGJEAAAA|18789|5|13|9|AM|third|night|| +18790|AAAAAAAAHGJEAAAA|18790|5|13|10|AM|third|night|| +18791|AAAAAAAAIGJEAAAA|18791|5|13|11|AM|third|night|| +18792|AAAAAAAAJGJEAAAA|18792|5|13|12|AM|third|night|| +18793|AAAAAAAAKGJEAAAA|18793|5|13|13|AM|third|night|| +18794|AAAAAAAALGJEAAAA|18794|5|13|14|AM|third|night|| +18795|AAAAAAAAMGJEAAAA|18795|5|13|15|AM|third|night|| +18796|AAAAAAAANGJEAAAA|18796|5|13|16|AM|third|night|| +18797|AAAAAAAAOGJEAAAA|18797|5|13|17|AM|third|night|| +18798|AAAAAAAAPGJEAAAA|18798|5|13|18|AM|third|night|| +18799|AAAAAAAAAHJEAAAA|18799|5|13|19|AM|third|night|| +18800|AAAAAAAABHJEAAAA|18800|5|13|20|AM|third|night|| +18801|AAAAAAAACHJEAAAA|18801|5|13|21|AM|third|night|| +18802|AAAAAAAADHJEAAAA|18802|5|13|22|AM|third|night|| +18803|AAAAAAAAEHJEAAAA|18803|5|13|23|AM|third|night|| +18804|AAAAAAAAFHJEAAAA|18804|5|13|24|AM|third|night|| +18805|AAAAAAAAGHJEAAAA|18805|5|13|25|AM|third|night|| +18806|AAAAAAAAHHJEAAAA|18806|5|13|26|AM|third|night|| +18807|AAAAAAAAIHJEAAAA|18807|5|13|27|AM|third|night|| +18808|AAAAAAAAJHJEAAAA|18808|5|13|28|AM|third|night|| +18809|AAAAAAAAKHJEAAAA|18809|5|13|29|AM|third|night|| +18810|AAAAAAAALHJEAAAA|18810|5|13|30|AM|third|night|| +18811|AAAAAAAAMHJEAAAA|18811|5|13|31|AM|third|night|| +18812|AAAAAAAANHJEAAAA|18812|5|13|32|AM|third|night|| +18813|AAAAAAAAOHJEAAAA|18813|5|13|33|AM|third|night|| +18814|AAAAAAAAPHJEAAAA|18814|5|13|34|AM|third|night|| +18815|AAAAAAAAAIJEAAAA|18815|5|13|35|AM|third|night|| +18816|AAAAAAAABIJEAAAA|18816|5|13|36|AM|third|night|| +18817|AAAAAAAACIJEAAAA|18817|5|13|37|AM|third|night|| +18818|AAAAAAAADIJEAAAA|18818|5|13|38|AM|third|night|| +18819|AAAAAAAAEIJEAAAA|18819|5|13|39|AM|third|night|| +18820|AAAAAAAAFIJEAAAA|18820|5|13|40|AM|third|night|| +18821|AAAAAAAAGIJEAAAA|18821|5|13|41|AM|third|night|| +18822|AAAAAAAAHIJEAAAA|18822|5|13|42|AM|third|night|| +18823|AAAAAAAAIIJEAAAA|18823|5|13|43|AM|third|night|| +18824|AAAAAAAAJIJEAAAA|18824|5|13|44|AM|third|night|| +18825|AAAAAAAAKIJEAAAA|18825|5|13|45|AM|third|night|| +18826|AAAAAAAALIJEAAAA|18826|5|13|46|AM|third|night|| +18827|AAAAAAAAMIJEAAAA|18827|5|13|47|AM|third|night|| +18828|AAAAAAAANIJEAAAA|18828|5|13|48|AM|third|night|| +18829|AAAAAAAAOIJEAAAA|18829|5|13|49|AM|third|night|| +18830|AAAAAAAAPIJEAAAA|18830|5|13|50|AM|third|night|| +18831|AAAAAAAAAJJEAAAA|18831|5|13|51|AM|third|night|| +18832|AAAAAAAABJJEAAAA|18832|5|13|52|AM|third|night|| +18833|AAAAAAAACJJEAAAA|18833|5|13|53|AM|third|night|| +18834|AAAAAAAADJJEAAAA|18834|5|13|54|AM|third|night|| +18835|AAAAAAAAEJJEAAAA|18835|5|13|55|AM|third|night|| +18836|AAAAAAAAFJJEAAAA|18836|5|13|56|AM|third|night|| +18837|AAAAAAAAGJJEAAAA|18837|5|13|57|AM|third|night|| +18838|AAAAAAAAHJJEAAAA|18838|5|13|58|AM|third|night|| +18839|AAAAAAAAIJJEAAAA|18839|5|13|59|AM|third|night|| +18840|AAAAAAAAJJJEAAAA|18840|5|14|0|AM|third|night|| +18841|AAAAAAAAKJJEAAAA|18841|5|14|1|AM|third|night|| +18842|AAAAAAAALJJEAAAA|18842|5|14|2|AM|third|night|| +18843|AAAAAAAAMJJEAAAA|18843|5|14|3|AM|third|night|| +18844|AAAAAAAANJJEAAAA|18844|5|14|4|AM|third|night|| +18845|AAAAAAAAOJJEAAAA|18845|5|14|5|AM|third|night|| +18846|AAAAAAAAPJJEAAAA|18846|5|14|6|AM|third|night|| +18847|AAAAAAAAAKJEAAAA|18847|5|14|7|AM|third|night|| +18848|AAAAAAAABKJEAAAA|18848|5|14|8|AM|third|night|| +18849|AAAAAAAACKJEAAAA|18849|5|14|9|AM|third|night|| +18850|AAAAAAAADKJEAAAA|18850|5|14|10|AM|third|night|| +18851|AAAAAAAAEKJEAAAA|18851|5|14|11|AM|third|night|| +18852|AAAAAAAAFKJEAAAA|18852|5|14|12|AM|third|night|| +18853|AAAAAAAAGKJEAAAA|18853|5|14|13|AM|third|night|| +18854|AAAAAAAAHKJEAAAA|18854|5|14|14|AM|third|night|| +18855|AAAAAAAAIKJEAAAA|18855|5|14|15|AM|third|night|| +18856|AAAAAAAAJKJEAAAA|18856|5|14|16|AM|third|night|| +18857|AAAAAAAAKKJEAAAA|18857|5|14|17|AM|third|night|| +18858|AAAAAAAALKJEAAAA|18858|5|14|18|AM|third|night|| +18859|AAAAAAAAMKJEAAAA|18859|5|14|19|AM|third|night|| +18860|AAAAAAAANKJEAAAA|18860|5|14|20|AM|third|night|| +18861|AAAAAAAAOKJEAAAA|18861|5|14|21|AM|third|night|| +18862|AAAAAAAAPKJEAAAA|18862|5|14|22|AM|third|night|| +18863|AAAAAAAAALJEAAAA|18863|5|14|23|AM|third|night|| +18864|AAAAAAAABLJEAAAA|18864|5|14|24|AM|third|night|| +18865|AAAAAAAACLJEAAAA|18865|5|14|25|AM|third|night|| +18866|AAAAAAAADLJEAAAA|18866|5|14|26|AM|third|night|| +18867|AAAAAAAAELJEAAAA|18867|5|14|27|AM|third|night|| +18868|AAAAAAAAFLJEAAAA|18868|5|14|28|AM|third|night|| +18869|AAAAAAAAGLJEAAAA|18869|5|14|29|AM|third|night|| +18870|AAAAAAAAHLJEAAAA|18870|5|14|30|AM|third|night|| +18871|AAAAAAAAILJEAAAA|18871|5|14|31|AM|third|night|| +18872|AAAAAAAAJLJEAAAA|18872|5|14|32|AM|third|night|| +18873|AAAAAAAAKLJEAAAA|18873|5|14|33|AM|third|night|| +18874|AAAAAAAALLJEAAAA|18874|5|14|34|AM|third|night|| +18875|AAAAAAAAMLJEAAAA|18875|5|14|35|AM|third|night|| +18876|AAAAAAAANLJEAAAA|18876|5|14|36|AM|third|night|| +18877|AAAAAAAAOLJEAAAA|18877|5|14|37|AM|third|night|| +18878|AAAAAAAAPLJEAAAA|18878|5|14|38|AM|third|night|| +18879|AAAAAAAAAMJEAAAA|18879|5|14|39|AM|third|night|| +18880|AAAAAAAABMJEAAAA|18880|5|14|40|AM|third|night|| +18881|AAAAAAAACMJEAAAA|18881|5|14|41|AM|third|night|| +18882|AAAAAAAADMJEAAAA|18882|5|14|42|AM|third|night|| +18883|AAAAAAAAEMJEAAAA|18883|5|14|43|AM|third|night|| +18884|AAAAAAAAFMJEAAAA|18884|5|14|44|AM|third|night|| +18885|AAAAAAAAGMJEAAAA|18885|5|14|45|AM|third|night|| +18886|AAAAAAAAHMJEAAAA|18886|5|14|46|AM|third|night|| +18887|AAAAAAAAIMJEAAAA|18887|5|14|47|AM|third|night|| +18888|AAAAAAAAJMJEAAAA|18888|5|14|48|AM|third|night|| +18889|AAAAAAAAKMJEAAAA|18889|5|14|49|AM|third|night|| +18890|AAAAAAAALMJEAAAA|18890|5|14|50|AM|third|night|| +18891|AAAAAAAAMMJEAAAA|18891|5|14|51|AM|third|night|| +18892|AAAAAAAANMJEAAAA|18892|5|14|52|AM|third|night|| +18893|AAAAAAAAOMJEAAAA|18893|5|14|53|AM|third|night|| +18894|AAAAAAAAPMJEAAAA|18894|5|14|54|AM|third|night|| +18895|AAAAAAAAANJEAAAA|18895|5|14|55|AM|third|night|| +18896|AAAAAAAABNJEAAAA|18896|5|14|56|AM|third|night|| +18897|AAAAAAAACNJEAAAA|18897|5|14|57|AM|third|night|| +18898|AAAAAAAADNJEAAAA|18898|5|14|58|AM|third|night|| +18899|AAAAAAAAENJEAAAA|18899|5|14|59|AM|third|night|| +18900|AAAAAAAAFNJEAAAA|18900|5|15|0|AM|third|night|| +18901|AAAAAAAAGNJEAAAA|18901|5|15|1|AM|third|night|| +18902|AAAAAAAAHNJEAAAA|18902|5|15|2|AM|third|night|| +18903|AAAAAAAAINJEAAAA|18903|5|15|3|AM|third|night|| +18904|AAAAAAAAJNJEAAAA|18904|5|15|4|AM|third|night|| +18905|AAAAAAAAKNJEAAAA|18905|5|15|5|AM|third|night|| +18906|AAAAAAAALNJEAAAA|18906|5|15|6|AM|third|night|| +18907|AAAAAAAAMNJEAAAA|18907|5|15|7|AM|third|night|| +18908|AAAAAAAANNJEAAAA|18908|5|15|8|AM|third|night|| +18909|AAAAAAAAONJEAAAA|18909|5|15|9|AM|third|night|| +18910|AAAAAAAAPNJEAAAA|18910|5|15|10|AM|third|night|| +18911|AAAAAAAAAOJEAAAA|18911|5|15|11|AM|third|night|| +18912|AAAAAAAABOJEAAAA|18912|5|15|12|AM|third|night|| +18913|AAAAAAAACOJEAAAA|18913|5|15|13|AM|third|night|| +18914|AAAAAAAADOJEAAAA|18914|5|15|14|AM|third|night|| +18915|AAAAAAAAEOJEAAAA|18915|5|15|15|AM|third|night|| +18916|AAAAAAAAFOJEAAAA|18916|5|15|16|AM|third|night|| +18917|AAAAAAAAGOJEAAAA|18917|5|15|17|AM|third|night|| +18918|AAAAAAAAHOJEAAAA|18918|5|15|18|AM|third|night|| +18919|AAAAAAAAIOJEAAAA|18919|5|15|19|AM|third|night|| +18920|AAAAAAAAJOJEAAAA|18920|5|15|20|AM|third|night|| +18921|AAAAAAAAKOJEAAAA|18921|5|15|21|AM|third|night|| +18922|AAAAAAAALOJEAAAA|18922|5|15|22|AM|third|night|| +18923|AAAAAAAAMOJEAAAA|18923|5|15|23|AM|third|night|| +18924|AAAAAAAANOJEAAAA|18924|5|15|24|AM|third|night|| +18925|AAAAAAAAOOJEAAAA|18925|5|15|25|AM|third|night|| +18926|AAAAAAAAPOJEAAAA|18926|5|15|26|AM|third|night|| +18927|AAAAAAAAAPJEAAAA|18927|5|15|27|AM|third|night|| +18928|AAAAAAAABPJEAAAA|18928|5|15|28|AM|third|night|| +18929|AAAAAAAACPJEAAAA|18929|5|15|29|AM|third|night|| +18930|AAAAAAAADPJEAAAA|18930|5|15|30|AM|third|night|| +18931|AAAAAAAAEPJEAAAA|18931|5|15|31|AM|third|night|| +18932|AAAAAAAAFPJEAAAA|18932|5|15|32|AM|third|night|| +18933|AAAAAAAAGPJEAAAA|18933|5|15|33|AM|third|night|| +18934|AAAAAAAAHPJEAAAA|18934|5|15|34|AM|third|night|| +18935|AAAAAAAAIPJEAAAA|18935|5|15|35|AM|third|night|| +18936|AAAAAAAAJPJEAAAA|18936|5|15|36|AM|third|night|| +18937|AAAAAAAAKPJEAAAA|18937|5|15|37|AM|third|night|| +18938|AAAAAAAALPJEAAAA|18938|5|15|38|AM|third|night|| +18939|AAAAAAAAMPJEAAAA|18939|5|15|39|AM|third|night|| +18940|AAAAAAAANPJEAAAA|18940|5|15|40|AM|third|night|| +18941|AAAAAAAAOPJEAAAA|18941|5|15|41|AM|third|night|| +18942|AAAAAAAAPPJEAAAA|18942|5|15|42|AM|third|night|| +18943|AAAAAAAAAAKEAAAA|18943|5|15|43|AM|third|night|| +18944|AAAAAAAABAKEAAAA|18944|5|15|44|AM|third|night|| +18945|AAAAAAAACAKEAAAA|18945|5|15|45|AM|third|night|| +18946|AAAAAAAADAKEAAAA|18946|5|15|46|AM|third|night|| +18947|AAAAAAAAEAKEAAAA|18947|5|15|47|AM|third|night|| +18948|AAAAAAAAFAKEAAAA|18948|5|15|48|AM|third|night|| +18949|AAAAAAAAGAKEAAAA|18949|5|15|49|AM|third|night|| +18950|AAAAAAAAHAKEAAAA|18950|5|15|50|AM|third|night|| +18951|AAAAAAAAIAKEAAAA|18951|5|15|51|AM|third|night|| +18952|AAAAAAAAJAKEAAAA|18952|5|15|52|AM|third|night|| +18953|AAAAAAAAKAKEAAAA|18953|5|15|53|AM|third|night|| +18954|AAAAAAAALAKEAAAA|18954|5|15|54|AM|third|night|| +18955|AAAAAAAAMAKEAAAA|18955|5|15|55|AM|third|night|| +18956|AAAAAAAANAKEAAAA|18956|5|15|56|AM|third|night|| +18957|AAAAAAAAOAKEAAAA|18957|5|15|57|AM|third|night|| +18958|AAAAAAAAPAKEAAAA|18958|5|15|58|AM|third|night|| +18959|AAAAAAAAABKEAAAA|18959|5|15|59|AM|third|night|| +18960|AAAAAAAABBKEAAAA|18960|5|16|0|AM|third|night|| +18961|AAAAAAAACBKEAAAA|18961|5|16|1|AM|third|night|| +18962|AAAAAAAADBKEAAAA|18962|5|16|2|AM|third|night|| +18963|AAAAAAAAEBKEAAAA|18963|5|16|3|AM|third|night|| +18964|AAAAAAAAFBKEAAAA|18964|5|16|4|AM|third|night|| +18965|AAAAAAAAGBKEAAAA|18965|5|16|5|AM|third|night|| +18966|AAAAAAAAHBKEAAAA|18966|5|16|6|AM|third|night|| +18967|AAAAAAAAIBKEAAAA|18967|5|16|7|AM|third|night|| +18968|AAAAAAAAJBKEAAAA|18968|5|16|8|AM|third|night|| +18969|AAAAAAAAKBKEAAAA|18969|5|16|9|AM|third|night|| +18970|AAAAAAAALBKEAAAA|18970|5|16|10|AM|third|night|| +18971|AAAAAAAAMBKEAAAA|18971|5|16|11|AM|third|night|| +18972|AAAAAAAANBKEAAAA|18972|5|16|12|AM|third|night|| +18973|AAAAAAAAOBKEAAAA|18973|5|16|13|AM|third|night|| +18974|AAAAAAAAPBKEAAAA|18974|5|16|14|AM|third|night|| +18975|AAAAAAAAACKEAAAA|18975|5|16|15|AM|third|night|| +18976|AAAAAAAABCKEAAAA|18976|5|16|16|AM|third|night|| +18977|AAAAAAAACCKEAAAA|18977|5|16|17|AM|third|night|| +18978|AAAAAAAADCKEAAAA|18978|5|16|18|AM|third|night|| +18979|AAAAAAAAECKEAAAA|18979|5|16|19|AM|third|night|| +18980|AAAAAAAAFCKEAAAA|18980|5|16|20|AM|third|night|| +18981|AAAAAAAAGCKEAAAA|18981|5|16|21|AM|third|night|| +18982|AAAAAAAAHCKEAAAA|18982|5|16|22|AM|third|night|| +18983|AAAAAAAAICKEAAAA|18983|5|16|23|AM|third|night|| +18984|AAAAAAAAJCKEAAAA|18984|5|16|24|AM|third|night|| +18985|AAAAAAAAKCKEAAAA|18985|5|16|25|AM|third|night|| +18986|AAAAAAAALCKEAAAA|18986|5|16|26|AM|third|night|| +18987|AAAAAAAAMCKEAAAA|18987|5|16|27|AM|third|night|| +18988|AAAAAAAANCKEAAAA|18988|5|16|28|AM|third|night|| +18989|AAAAAAAAOCKEAAAA|18989|5|16|29|AM|third|night|| +18990|AAAAAAAAPCKEAAAA|18990|5|16|30|AM|third|night|| +18991|AAAAAAAAADKEAAAA|18991|5|16|31|AM|third|night|| +18992|AAAAAAAABDKEAAAA|18992|5|16|32|AM|third|night|| +18993|AAAAAAAACDKEAAAA|18993|5|16|33|AM|third|night|| +18994|AAAAAAAADDKEAAAA|18994|5|16|34|AM|third|night|| +18995|AAAAAAAAEDKEAAAA|18995|5|16|35|AM|third|night|| +18996|AAAAAAAAFDKEAAAA|18996|5|16|36|AM|third|night|| +18997|AAAAAAAAGDKEAAAA|18997|5|16|37|AM|third|night|| +18998|AAAAAAAAHDKEAAAA|18998|5|16|38|AM|third|night|| +18999|AAAAAAAAIDKEAAAA|18999|5|16|39|AM|third|night|| +19000|AAAAAAAAJDKEAAAA|19000|5|16|40|AM|third|night|| +19001|AAAAAAAAKDKEAAAA|19001|5|16|41|AM|third|night|| +19002|AAAAAAAALDKEAAAA|19002|5|16|42|AM|third|night|| +19003|AAAAAAAAMDKEAAAA|19003|5|16|43|AM|third|night|| +19004|AAAAAAAANDKEAAAA|19004|5|16|44|AM|third|night|| +19005|AAAAAAAAODKEAAAA|19005|5|16|45|AM|third|night|| +19006|AAAAAAAAPDKEAAAA|19006|5|16|46|AM|third|night|| +19007|AAAAAAAAAEKEAAAA|19007|5|16|47|AM|third|night|| +19008|AAAAAAAABEKEAAAA|19008|5|16|48|AM|third|night|| +19009|AAAAAAAACEKEAAAA|19009|5|16|49|AM|third|night|| +19010|AAAAAAAADEKEAAAA|19010|5|16|50|AM|third|night|| +19011|AAAAAAAAEEKEAAAA|19011|5|16|51|AM|third|night|| +19012|AAAAAAAAFEKEAAAA|19012|5|16|52|AM|third|night|| +19013|AAAAAAAAGEKEAAAA|19013|5|16|53|AM|third|night|| +19014|AAAAAAAAHEKEAAAA|19014|5|16|54|AM|third|night|| +19015|AAAAAAAAIEKEAAAA|19015|5|16|55|AM|third|night|| +19016|AAAAAAAAJEKEAAAA|19016|5|16|56|AM|third|night|| +19017|AAAAAAAAKEKEAAAA|19017|5|16|57|AM|third|night|| +19018|AAAAAAAALEKEAAAA|19018|5|16|58|AM|third|night|| +19019|AAAAAAAAMEKEAAAA|19019|5|16|59|AM|third|night|| +19020|AAAAAAAANEKEAAAA|19020|5|17|0|AM|third|night|| +19021|AAAAAAAAOEKEAAAA|19021|5|17|1|AM|third|night|| +19022|AAAAAAAAPEKEAAAA|19022|5|17|2|AM|third|night|| +19023|AAAAAAAAAFKEAAAA|19023|5|17|3|AM|third|night|| +19024|AAAAAAAABFKEAAAA|19024|5|17|4|AM|third|night|| +19025|AAAAAAAACFKEAAAA|19025|5|17|5|AM|third|night|| +19026|AAAAAAAADFKEAAAA|19026|5|17|6|AM|third|night|| +19027|AAAAAAAAEFKEAAAA|19027|5|17|7|AM|third|night|| +19028|AAAAAAAAFFKEAAAA|19028|5|17|8|AM|third|night|| +19029|AAAAAAAAGFKEAAAA|19029|5|17|9|AM|third|night|| +19030|AAAAAAAAHFKEAAAA|19030|5|17|10|AM|third|night|| +19031|AAAAAAAAIFKEAAAA|19031|5|17|11|AM|third|night|| +19032|AAAAAAAAJFKEAAAA|19032|5|17|12|AM|third|night|| +19033|AAAAAAAAKFKEAAAA|19033|5|17|13|AM|third|night|| +19034|AAAAAAAALFKEAAAA|19034|5|17|14|AM|third|night|| +19035|AAAAAAAAMFKEAAAA|19035|5|17|15|AM|third|night|| +19036|AAAAAAAANFKEAAAA|19036|5|17|16|AM|third|night|| +19037|AAAAAAAAOFKEAAAA|19037|5|17|17|AM|third|night|| +19038|AAAAAAAAPFKEAAAA|19038|5|17|18|AM|third|night|| +19039|AAAAAAAAAGKEAAAA|19039|5|17|19|AM|third|night|| +19040|AAAAAAAABGKEAAAA|19040|5|17|20|AM|third|night|| +19041|AAAAAAAACGKEAAAA|19041|5|17|21|AM|third|night|| +19042|AAAAAAAADGKEAAAA|19042|5|17|22|AM|third|night|| +19043|AAAAAAAAEGKEAAAA|19043|5|17|23|AM|third|night|| +19044|AAAAAAAAFGKEAAAA|19044|5|17|24|AM|third|night|| +19045|AAAAAAAAGGKEAAAA|19045|5|17|25|AM|third|night|| +19046|AAAAAAAAHGKEAAAA|19046|5|17|26|AM|third|night|| +19047|AAAAAAAAIGKEAAAA|19047|5|17|27|AM|third|night|| +19048|AAAAAAAAJGKEAAAA|19048|5|17|28|AM|third|night|| +19049|AAAAAAAAKGKEAAAA|19049|5|17|29|AM|third|night|| +19050|AAAAAAAALGKEAAAA|19050|5|17|30|AM|third|night|| +19051|AAAAAAAAMGKEAAAA|19051|5|17|31|AM|third|night|| +19052|AAAAAAAANGKEAAAA|19052|5|17|32|AM|third|night|| +19053|AAAAAAAAOGKEAAAA|19053|5|17|33|AM|third|night|| +19054|AAAAAAAAPGKEAAAA|19054|5|17|34|AM|third|night|| +19055|AAAAAAAAAHKEAAAA|19055|5|17|35|AM|third|night|| +19056|AAAAAAAABHKEAAAA|19056|5|17|36|AM|third|night|| +19057|AAAAAAAACHKEAAAA|19057|5|17|37|AM|third|night|| +19058|AAAAAAAADHKEAAAA|19058|5|17|38|AM|third|night|| +19059|AAAAAAAAEHKEAAAA|19059|5|17|39|AM|third|night|| +19060|AAAAAAAAFHKEAAAA|19060|5|17|40|AM|third|night|| +19061|AAAAAAAAGHKEAAAA|19061|5|17|41|AM|third|night|| +19062|AAAAAAAAHHKEAAAA|19062|5|17|42|AM|third|night|| +19063|AAAAAAAAIHKEAAAA|19063|5|17|43|AM|third|night|| +19064|AAAAAAAAJHKEAAAA|19064|5|17|44|AM|third|night|| +19065|AAAAAAAAKHKEAAAA|19065|5|17|45|AM|third|night|| +19066|AAAAAAAALHKEAAAA|19066|5|17|46|AM|third|night|| +19067|AAAAAAAAMHKEAAAA|19067|5|17|47|AM|third|night|| +19068|AAAAAAAANHKEAAAA|19068|5|17|48|AM|third|night|| +19069|AAAAAAAAOHKEAAAA|19069|5|17|49|AM|third|night|| +19070|AAAAAAAAPHKEAAAA|19070|5|17|50|AM|third|night|| +19071|AAAAAAAAAIKEAAAA|19071|5|17|51|AM|third|night|| +19072|AAAAAAAABIKEAAAA|19072|5|17|52|AM|third|night|| +19073|AAAAAAAACIKEAAAA|19073|5|17|53|AM|third|night|| +19074|AAAAAAAADIKEAAAA|19074|5|17|54|AM|third|night|| +19075|AAAAAAAAEIKEAAAA|19075|5|17|55|AM|third|night|| +19076|AAAAAAAAFIKEAAAA|19076|5|17|56|AM|third|night|| +19077|AAAAAAAAGIKEAAAA|19077|5|17|57|AM|third|night|| +19078|AAAAAAAAHIKEAAAA|19078|5|17|58|AM|third|night|| +19079|AAAAAAAAIIKEAAAA|19079|5|17|59|AM|third|night|| +19080|AAAAAAAAJIKEAAAA|19080|5|18|0|AM|third|night|| +19081|AAAAAAAAKIKEAAAA|19081|5|18|1|AM|third|night|| +19082|AAAAAAAALIKEAAAA|19082|5|18|2|AM|third|night|| +19083|AAAAAAAAMIKEAAAA|19083|5|18|3|AM|third|night|| +19084|AAAAAAAANIKEAAAA|19084|5|18|4|AM|third|night|| +19085|AAAAAAAAOIKEAAAA|19085|5|18|5|AM|third|night|| +19086|AAAAAAAAPIKEAAAA|19086|5|18|6|AM|third|night|| +19087|AAAAAAAAAJKEAAAA|19087|5|18|7|AM|third|night|| +19088|AAAAAAAABJKEAAAA|19088|5|18|8|AM|third|night|| +19089|AAAAAAAACJKEAAAA|19089|5|18|9|AM|third|night|| +19090|AAAAAAAADJKEAAAA|19090|5|18|10|AM|third|night|| +19091|AAAAAAAAEJKEAAAA|19091|5|18|11|AM|third|night|| +19092|AAAAAAAAFJKEAAAA|19092|5|18|12|AM|third|night|| +19093|AAAAAAAAGJKEAAAA|19093|5|18|13|AM|third|night|| +19094|AAAAAAAAHJKEAAAA|19094|5|18|14|AM|third|night|| +19095|AAAAAAAAIJKEAAAA|19095|5|18|15|AM|third|night|| +19096|AAAAAAAAJJKEAAAA|19096|5|18|16|AM|third|night|| +19097|AAAAAAAAKJKEAAAA|19097|5|18|17|AM|third|night|| +19098|AAAAAAAALJKEAAAA|19098|5|18|18|AM|third|night|| +19099|AAAAAAAAMJKEAAAA|19099|5|18|19|AM|third|night|| +19100|AAAAAAAANJKEAAAA|19100|5|18|20|AM|third|night|| +19101|AAAAAAAAOJKEAAAA|19101|5|18|21|AM|third|night|| +19102|AAAAAAAAPJKEAAAA|19102|5|18|22|AM|third|night|| +19103|AAAAAAAAAKKEAAAA|19103|5|18|23|AM|third|night|| +19104|AAAAAAAABKKEAAAA|19104|5|18|24|AM|third|night|| +19105|AAAAAAAACKKEAAAA|19105|5|18|25|AM|third|night|| +19106|AAAAAAAADKKEAAAA|19106|5|18|26|AM|third|night|| +19107|AAAAAAAAEKKEAAAA|19107|5|18|27|AM|third|night|| +19108|AAAAAAAAFKKEAAAA|19108|5|18|28|AM|third|night|| +19109|AAAAAAAAGKKEAAAA|19109|5|18|29|AM|third|night|| +19110|AAAAAAAAHKKEAAAA|19110|5|18|30|AM|third|night|| +19111|AAAAAAAAIKKEAAAA|19111|5|18|31|AM|third|night|| +19112|AAAAAAAAJKKEAAAA|19112|5|18|32|AM|third|night|| +19113|AAAAAAAAKKKEAAAA|19113|5|18|33|AM|third|night|| +19114|AAAAAAAALKKEAAAA|19114|5|18|34|AM|third|night|| +19115|AAAAAAAAMKKEAAAA|19115|5|18|35|AM|third|night|| +19116|AAAAAAAANKKEAAAA|19116|5|18|36|AM|third|night|| +19117|AAAAAAAAOKKEAAAA|19117|5|18|37|AM|third|night|| +19118|AAAAAAAAPKKEAAAA|19118|5|18|38|AM|third|night|| +19119|AAAAAAAAALKEAAAA|19119|5|18|39|AM|third|night|| +19120|AAAAAAAABLKEAAAA|19120|5|18|40|AM|third|night|| +19121|AAAAAAAACLKEAAAA|19121|5|18|41|AM|third|night|| +19122|AAAAAAAADLKEAAAA|19122|5|18|42|AM|third|night|| +19123|AAAAAAAAELKEAAAA|19123|5|18|43|AM|third|night|| +19124|AAAAAAAAFLKEAAAA|19124|5|18|44|AM|third|night|| +19125|AAAAAAAAGLKEAAAA|19125|5|18|45|AM|third|night|| +19126|AAAAAAAAHLKEAAAA|19126|5|18|46|AM|third|night|| +19127|AAAAAAAAILKEAAAA|19127|5|18|47|AM|third|night|| +19128|AAAAAAAAJLKEAAAA|19128|5|18|48|AM|third|night|| +19129|AAAAAAAAKLKEAAAA|19129|5|18|49|AM|third|night|| +19130|AAAAAAAALLKEAAAA|19130|5|18|50|AM|third|night|| +19131|AAAAAAAAMLKEAAAA|19131|5|18|51|AM|third|night|| +19132|AAAAAAAANLKEAAAA|19132|5|18|52|AM|third|night|| +19133|AAAAAAAAOLKEAAAA|19133|5|18|53|AM|third|night|| +19134|AAAAAAAAPLKEAAAA|19134|5|18|54|AM|third|night|| +19135|AAAAAAAAAMKEAAAA|19135|5|18|55|AM|third|night|| +19136|AAAAAAAABMKEAAAA|19136|5|18|56|AM|third|night|| +19137|AAAAAAAACMKEAAAA|19137|5|18|57|AM|third|night|| +19138|AAAAAAAADMKEAAAA|19138|5|18|58|AM|third|night|| +19139|AAAAAAAAEMKEAAAA|19139|5|18|59|AM|third|night|| +19140|AAAAAAAAFMKEAAAA|19140|5|19|0|AM|third|night|| +19141|AAAAAAAAGMKEAAAA|19141|5|19|1|AM|third|night|| +19142|AAAAAAAAHMKEAAAA|19142|5|19|2|AM|third|night|| +19143|AAAAAAAAIMKEAAAA|19143|5|19|3|AM|third|night|| +19144|AAAAAAAAJMKEAAAA|19144|5|19|4|AM|third|night|| +19145|AAAAAAAAKMKEAAAA|19145|5|19|5|AM|third|night|| +19146|AAAAAAAALMKEAAAA|19146|5|19|6|AM|third|night|| +19147|AAAAAAAAMMKEAAAA|19147|5|19|7|AM|third|night|| +19148|AAAAAAAANMKEAAAA|19148|5|19|8|AM|third|night|| +19149|AAAAAAAAOMKEAAAA|19149|5|19|9|AM|third|night|| +19150|AAAAAAAAPMKEAAAA|19150|5|19|10|AM|third|night|| +19151|AAAAAAAAANKEAAAA|19151|5|19|11|AM|third|night|| +19152|AAAAAAAABNKEAAAA|19152|5|19|12|AM|third|night|| +19153|AAAAAAAACNKEAAAA|19153|5|19|13|AM|third|night|| +19154|AAAAAAAADNKEAAAA|19154|5|19|14|AM|third|night|| +19155|AAAAAAAAENKEAAAA|19155|5|19|15|AM|third|night|| +19156|AAAAAAAAFNKEAAAA|19156|5|19|16|AM|third|night|| +19157|AAAAAAAAGNKEAAAA|19157|5|19|17|AM|third|night|| +19158|AAAAAAAAHNKEAAAA|19158|5|19|18|AM|third|night|| +19159|AAAAAAAAINKEAAAA|19159|5|19|19|AM|third|night|| +19160|AAAAAAAAJNKEAAAA|19160|5|19|20|AM|third|night|| +19161|AAAAAAAAKNKEAAAA|19161|5|19|21|AM|third|night|| +19162|AAAAAAAALNKEAAAA|19162|5|19|22|AM|third|night|| +19163|AAAAAAAAMNKEAAAA|19163|5|19|23|AM|third|night|| +19164|AAAAAAAANNKEAAAA|19164|5|19|24|AM|third|night|| +19165|AAAAAAAAONKEAAAA|19165|5|19|25|AM|third|night|| +19166|AAAAAAAAPNKEAAAA|19166|5|19|26|AM|third|night|| +19167|AAAAAAAAAOKEAAAA|19167|5|19|27|AM|third|night|| +19168|AAAAAAAABOKEAAAA|19168|5|19|28|AM|third|night|| +19169|AAAAAAAACOKEAAAA|19169|5|19|29|AM|third|night|| +19170|AAAAAAAADOKEAAAA|19170|5|19|30|AM|third|night|| +19171|AAAAAAAAEOKEAAAA|19171|5|19|31|AM|third|night|| +19172|AAAAAAAAFOKEAAAA|19172|5|19|32|AM|third|night|| +19173|AAAAAAAAGOKEAAAA|19173|5|19|33|AM|third|night|| +19174|AAAAAAAAHOKEAAAA|19174|5|19|34|AM|third|night|| +19175|AAAAAAAAIOKEAAAA|19175|5|19|35|AM|third|night|| +19176|AAAAAAAAJOKEAAAA|19176|5|19|36|AM|third|night|| +19177|AAAAAAAAKOKEAAAA|19177|5|19|37|AM|third|night|| +19178|AAAAAAAALOKEAAAA|19178|5|19|38|AM|third|night|| +19179|AAAAAAAAMOKEAAAA|19179|5|19|39|AM|third|night|| +19180|AAAAAAAANOKEAAAA|19180|5|19|40|AM|third|night|| +19181|AAAAAAAAOOKEAAAA|19181|5|19|41|AM|third|night|| +19182|AAAAAAAAPOKEAAAA|19182|5|19|42|AM|third|night|| +19183|AAAAAAAAAPKEAAAA|19183|5|19|43|AM|third|night|| +19184|AAAAAAAABPKEAAAA|19184|5|19|44|AM|third|night|| +19185|AAAAAAAACPKEAAAA|19185|5|19|45|AM|third|night|| +19186|AAAAAAAADPKEAAAA|19186|5|19|46|AM|third|night|| +19187|AAAAAAAAEPKEAAAA|19187|5|19|47|AM|third|night|| +19188|AAAAAAAAFPKEAAAA|19188|5|19|48|AM|third|night|| +19189|AAAAAAAAGPKEAAAA|19189|5|19|49|AM|third|night|| +19190|AAAAAAAAHPKEAAAA|19190|5|19|50|AM|third|night|| +19191|AAAAAAAAIPKEAAAA|19191|5|19|51|AM|third|night|| +19192|AAAAAAAAJPKEAAAA|19192|5|19|52|AM|third|night|| +19193|AAAAAAAAKPKEAAAA|19193|5|19|53|AM|third|night|| +19194|AAAAAAAALPKEAAAA|19194|5|19|54|AM|third|night|| +19195|AAAAAAAAMPKEAAAA|19195|5|19|55|AM|third|night|| +19196|AAAAAAAANPKEAAAA|19196|5|19|56|AM|third|night|| +19197|AAAAAAAAOPKEAAAA|19197|5|19|57|AM|third|night|| +19198|AAAAAAAAPPKEAAAA|19198|5|19|58|AM|third|night|| +19199|AAAAAAAAAALEAAAA|19199|5|19|59|AM|third|night|| +19200|AAAAAAAABALEAAAA|19200|5|20|0|AM|third|night|| +19201|AAAAAAAACALEAAAA|19201|5|20|1|AM|third|night|| +19202|AAAAAAAADALEAAAA|19202|5|20|2|AM|third|night|| +19203|AAAAAAAAEALEAAAA|19203|5|20|3|AM|third|night|| +19204|AAAAAAAAFALEAAAA|19204|5|20|4|AM|third|night|| +19205|AAAAAAAAGALEAAAA|19205|5|20|5|AM|third|night|| +19206|AAAAAAAAHALEAAAA|19206|5|20|6|AM|third|night|| +19207|AAAAAAAAIALEAAAA|19207|5|20|7|AM|third|night|| +19208|AAAAAAAAJALEAAAA|19208|5|20|8|AM|third|night|| +19209|AAAAAAAAKALEAAAA|19209|5|20|9|AM|third|night|| +19210|AAAAAAAALALEAAAA|19210|5|20|10|AM|third|night|| +19211|AAAAAAAAMALEAAAA|19211|5|20|11|AM|third|night|| +19212|AAAAAAAANALEAAAA|19212|5|20|12|AM|third|night|| +19213|AAAAAAAAOALEAAAA|19213|5|20|13|AM|third|night|| +19214|AAAAAAAAPALEAAAA|19214|5|20|14|AM|third|night|| +19215|AAAAAAAAABLEAAAA|19215|5|20|15|AM|third|night|| +19216|AAAAAAAABBLEAAAA|19216|5|20|16|AM|third|night|| +19217|AAAAAAAACBLEAAAA|19217|5|20|17|AM|third|night|| +19218|AAAAAAAADBLEAAAA|19218|5|20|18|AM|third|night|| +19219|AAAAAAAAEBLEAAAA|19219|5|20|19|AM|third|night|| +19220|AAAAAAAAFBLEAAAA|19220|5|20|20|AM|third|night|| +19221|AAAAAAAAGBLEAAAA|19221|5|20|21|AM|third|night|| +19222|AAAAAAAAHBLEAAAA|19222|5|20|22|AM|third|night|| +19223|AAAAAAAAIBLEAAAA|19223|5|20|23|AM|third|night|| +19224|AAAAAAAAJBLEAAAA|19224|5|20|24|AM|third|night|| +19225|AAAAAAAAKBLEAAAA|19225|5|20|25|AM|third|night|| +19226|AAAAAAAALBLEAAAA|19226|5|20|26|AM|third|night|| +19227|AAAAAAAAMBLEAAAA|19227|5|20|27|AM|third|night|| +19228|AAAAAAAANBLEAAAA|19228|5|20|28|AM|third|night|| +19229|AAAAAAAAOBLEAAAA|19229|5|20|29|AM|third|night|| +19230|AAAAAAAAPBLEAAAA|19230|5|20|30|AM|third|night|| +19231|AAAAAAAAACLEAAAA|19231|5|20|31|AM|third|night|| +19232|AAAAAAAABCLEAAAA|19232|5|20|32|AM|third|night|| +19233|AAAAAAAACCLEAAAA|19233|5|20|33|AM|third|night|| +19234|AAAAAAAADCLEAAAA|19234|5|20|34|AM|third|night|| +19235|AAAAAAAAECLEAAAA|19235|5|20|35|AM|third|night|| +19236|AAAAAAAAFCLEAAAA|19236|5|20|36|AM|third|night|| +19237|AAAAAAAAGCLEAAAA|19237|5|20|37|AM|third|night|| +19238|AAAAAAAAHCLEAAAA|19238|5|20|38|AM|third|night|| +19239|AAAAAAAAICLEAAAA|19239|5|20|39|AM|third|night|| +19240|AAAAAAAAJCLEAAAA|19240|5|20|40|AM|third|night|| +19241|AAAAAAAAKCLEAAAA|19241|5|20|41|AM|third|night|| +19242|AAAAAAAALCLEAAAA|19242|5|20|42|AM|third|night|| +19243|AAAAAAAAMCLEAAAA|19243|5|20|43|AM|third|night|| +19244|AAAAAAAANCLEAAAA|19244|5|20|44|AM|third|night|| +19245|AAAAAAAAOCLEAAAA|19245|5|20|45|AM|third|night|| +19246|AAAAAAAAPCLEAAAA|19246|5|20|46|AM|third|night|| +19247|AAAAAAAAADLEAAAA|19247|5|20|47|AM|third|night|| +19248|AAAAAAAABDLEAAAA|19248|5|20|48|AM|third|night|| +19249|AAAAAAAACDLEAAAA|19249|5|20|49|AM|third|night|| +19250|AAAAAAAADDLEAAAA|19250|5|20|50|AM|third|night|| +19251|AAAAAAAAEDLEAAAA|19251|5|20|51|AM|third|night|| +19252|AAAAAAAAFDLEAAAA|19252|5|20|52|AM|third|night|| +19253|AAAAAAAAGDLEAAAA|19253|5|20|53|AM|third|night|| +19254|AAAAAAAAHDLEAAAA|19254|5|20|54|AM|third|night|| +19255|AAAAAAAAIDLEAAAA|19255|5|20|55|AM|third|night|| +19256|AAAAAAAAJDLEAAAA|19256|5|20|56|AM|third|night|| +19257|AAAAAAAAKDLEAAAA|19257|5|20|57|AM|third|night|| +19258|AAAAAAAALDLEAAAA|19258|5|20|58|AM|third|night|| +19259|AAAAAAAAMDLEAAAA|19259|5|20|59|AM|third|night|| +19260|AAAAAAAANDLEAAAA|19260|5|21|0|AM|third|night|| +19261|AAAAAAAAODLEAAAA|19261|5|21|1|AM|third|night|| +19262|AAAAAAAAPDLEAAAA|19262|5|21|2|AM|third|night|| +19263|AAAAAAAAAELEAAAA|19263|5|21|3|AM|third|night|| +19264|AAAAAAAABELEAAAA|19264|5|21|4|AM|third|night|| +19265|AAAAAAAACELEAAAA|19265|5|21|5|AM|third|night|| +19266|AAAAAAAADELEAAAA|19266|5|21|6|AM|third|night|| +19267|AAAAAAAAEELEAAAA|19267|5|21|7|AM|third|night|| +19268|AAAAAAAAFELEAAAA|19268|5|21|8|AM|third|night|| +19269|AAAAAAAAGELEAAAA|19269|5|21|9|AM|third|night|| +19270|AAAAAAAAHELEAAAA|19270|5|21|10|AM|third|night|| +19271|AAAAAAAAIELEAAAA|19271|5|21|11|AM|third|night|| +19272|AAAAAAAAJELEAAAA|19272|5|21|12|AM|third|night|| +19273|AAAAAAAAKELEAAAA|19273|5|21|13|AM|third|night|| +19274|AAAAAAAALELEAAAA|19274|5|21|14|AM|third|night|| +19275|AAAAAAAAMELEAAAA|19275|5|21|15|AM|third|night|| +19276|AAAAAAAANELEAAAA|19276|5|21|16|AM|third|night|| +19277|AAAAAAAAOELEAAAA|19277|5|21|17|AM|third|night|| +19278|AAAAAAAAPELEAAAA|19278|5|21|18|AM|third|night|| +19279|AAAAAAAAAFLEAAAA|19279|5|21|19|AM|third|night|| +19280|AAAAAAAABFLEAAAA|19280|5|21|20|AM|third|night|| +19281|AAAAAAAACFLEAAAA|19281|5|21|21|AM|third|night|| +19282|AAAAAAAADFLEAAAA|19282|5|21|22|AM|third|night|| +19283|AAAAAAAAEFLEAAAA|19283|5|21|23|AM|third|night|| +19284|AAAAAAAAFFLEAAAA|19284|5|21|24|AM|third|night|| +19285|AAAAAAAAGFLEAAAA|19285|5|21|25|AM|third|night|| +19286|AAAAAAAAHFLEAAAA|19286|5|21|26|AM|third|night|| +19287|AAAAAAAAIFLEAAAA|19287|5|21|27|AM|third|night|| +19288|AAAAAAAAJFLEAAAA|19288|5|21|28|AM|third|night|| +19289|AAAAAAAAKFLEAAAA|19289|5|21|29|AM|third|night|| +19290|AAAAAAAALFLEAAAA|19290|5|21|30|AM|third|night|| +19291|AAAAAAAAMFLEAAAA|19291|5|21|31|AM|third|night|| +19292|AAAAAAAANFLEAAAA|19292|5|21|32|AM|third|night|| +19293|AAAAAAAAOFLEAAAA|19293|5|21|33|AM|third|night|| +19294|AAAAAAAAPFLEAAAA|19294|5|21|34|AM|third|night|| +19295|AAAAAAAAAGLEAAAA|19295|5|21|35|AM|third|night|| +19296|AAAAAAAABGLEAAAA|19296|5|21|36|AM|third|night|| +19297|AAAAAAAACGLEAAAA|19297|5|21|37|AM|third|night|| +19298|AAAAAAAADGLEAAAA|19298|5|21|38|AM|third|night|| +19299|AAAAAAAAEGLEAAAA|19299|5|21|39|AM|third|night|| +19300|AAAAAAAAFGLEAAAA|19300|5|21|40|AM|third|night|| +19301|AAAAAAAAGGLEAAAA|19301|5|21|41|AM|third|night|| +19302|AAAAAAAAHGLEAAAA|19302|5|21|42|AM|third|night|| +19303|AAAAAAAAIGLEAAAA|19303|5|21|43|AM|third|night|| +19304|AAAAAAAAJGLEAAAA|19304|5|21|44|AM|third|night|| +19305|AAAAAAAAKGLEAAAA|19305|5|21|45|AM|third|night|| +19306|AAAAAAAALGLEAAAA|19306|5|21|46|AM|third|night|| +19307|AAAAAAAAMGLEAAAA|19307|5|21|47|AM|third|night|| +19308|AAAAAAAANGLEAAAA|19308|5|21|48|AM|third|night|| +19309|AAAAAAAAOGLEAAAA|19309|5|21|49|AM|third|night|| +19310|AAAAAAAAPGLEAAAA|19310|5|21|50|AM|third|night|| +19311|AAAAAAAAAHLEAAAA|19311|5|21|51|AM|third|night|| +19312|AAAAAAAABHLEAAAA|19312|5|21|52|AM|third|night|| +19313|AAAAAAAACHLEAAAA|19313|5|21|53|AM|third|night|| +19314|AAAAAAAADHLEAAAA|19314|5|21|54|AM|third|night|| +19315|AAAAAAAAEHLEAAAA|19315|5|21|55|AM|third|night|| +19316|AAAAAAAAFHLEAAAA|19316|5|21|56|AM|third|night|| +19317|AAAAAAAAGHLEAAAA|19317|5|21|57|AM|third|night|| +19318|AAAAAAAAHHLEAAAA|19318|5|21|58|AM|third|night|| +19319|AAAAAAAAIHLEAAAA|19319|5|21|59|AM|third|night|| +19320|AAAAAAAAJHLEAAAA|19320|5|22|0|AM|third|night|| +19321|AAAAAAAAKHLEAAAA|19321|5|22|1|AM|third|night|| +19322|AAAAAAAALHLEAAAA|19322|5|22|2|AM|third|night|| +19323|AAAAAAAAMHLEAAAA|19323|5|22|3|AM|third|night|| +19324|AAAAAAAANHLEAAAA|19324|5|22|4|AM|third|night|| +19325|AAAAAAAAOHLEAAAA|19325|5|22|5|AM|third|night|| +19326|AAAAAAAAPHLEAAAA|19326|5|22|6|AM|third|night|| +19327|AAAAAAAAAILEAAAA|19327|5|22|7|AM|third|night|| +19328|AAAAAAAABILEAAAA|19328|5|22|8|AM|third|night|| +19329|AAAAAAAACILEAAAA|19329|5|22|9|AM|third|night|| +19330|AAAAAAAADILEAAAA|19330|5|22|10|AM|third|night|| +19331|AAAAAAAAEILEAAAA|19331|5|22|11|AM|third|night|| +19332|AAAAAAAAFILEAAAA|19332|5|22|12|AM|third|night|| +19333|AAAAAAAAGILEAAAA|19333|5|22|13|AM|third|night|| +19334|AAAAAAAAHILEAAAA|19334|5|22|14|AM|third|night|| +19335|AAAAAAAAIILEAAAA|19335|5|22|15|AM|third|night|| +19336|AAAAAAAAJILEAAAA|19336|5|22|16|AM|third|night|| +19337|AAAAAAAAKILEAAAA|19337|5|22|17|AM|third|night|| +19338|AAAAAAAALILEAAAA|19338|5|22|18|AM|third|night|| +19339|AAAAAAAAMILEAAAA|19339|5|22|19|AM|third|night|| +19340|AAAAAAAANILEAAAA|19340|5|22|20|AM|third|night|| +19341|AAAAAAAAOILEAAAA|19341|5|22|21|AM|third|night|| +19342|AAAAAAAAPILEAAAA|19342|5|22|22|AM|third|night|| +19343|AAAAAAAAAJLEAAAA|19343|5|22|23|AM|third|night|| +19344|AAAAAAAABJLEAAAA|19344|5|22|24|AM|third|night|| +19345|AAAAAAAACJLEAAAA|19345|5|22|25|AM|third|night|| +19346|AAAAAAAADJLEAAAA|19346|5|22|26|AM|third|night|| +19347|AAAAAAAAEJLEAAAA|19347|5|22|27|AM|third|night|| +19348|AAAAAAAAFJLEAAAA|19348|5|22|28|AM|third|night|| +19349|AAAAAAAAGJLEAAAA|19349|5|22|29|AM|third|night|| +19350|AAAAAAAAHJLEAAAA|19350|5|22|30|AM|third|night|| +19351|AAAAAAAAIJLEAAAA|19351|5|22|31|AM|third|night|| +19352|AAAAAAAAJJLEAAAA|19352|5|22|32|AM|third|night|| +19353|AAAAAAAAKJLEAAAA|19353|5|22|33|AM|third|night|| +19354|AAAAAAAALJLEAAAA|19354|5|22|34|AM|third|night|| +19355|AAAAAAAAMJLEAAAA|19355|5|22|35|AM|third|night|| +19356|AAAAAAAANJLEAAAA|19356|5|22|36|AM|third|night|| +19357|AAAAAAAAOJLEAAAA|19357|5|22|37|AM|third|night|| +19358|AAAAAAAAPJLEAAAA|19358|5|22|38|AM|third|night|| +19359|AAAAAAAAAKLEAAAA|19359|5|22|39|AM|third|night|| +19360|AAAAAAAABKLEAAAA|19360|5|22|40|AM|third|night|| +19361|AAAAAAAACKLEAAAA|19361|5|22|41|AM|third|night|| +19362|AAAAAAAADKLEAAAA|19362|5|22|42|AM|third|night|| +19363|AAAAAAAAEKLEAAAA|19363|5|22|43|AM|third|night|| +19364|AAAAAAAAFKLEAAAA|19364|5|22|44|AM|third|night|| +19365|AAAAAAAAGKLEAAAA|19365|5|22|45|AM|third|night|| +19366|AAAAAAAAHKLEAAAA|19366|5|22|46|AM|third|night|| +19367|AAAAAAAAIKLEAAAA|19367|5|22|47|AM|third|night|| +19368|AAAAAAAAJKLEAAAA|19368|5|22|48|AM|third|night|| +19369|AAAAAAAAKKLEAAAA|19369|5|22|49|AM|third|night|| +19370|AAAAAAAALKLEAAAA|19370|5|22|50|AM|third|night|| +19371|AAAAAAAAMKLEAAAA|19371|5|22|51|AM|third|night|| +19372|AAAAAAAANKLEAAAA|19372|5|22|52|AM|third|night|| +19373|AAAAAAAAOKLEAAAA|19373|5|22|53|AM|third|night|| +19374|AAAAAAAAPKLEAAAA|19374|5|22|54|AM|third|night|| +19375|AAAAAAAAALLEAAAA|19375|5|22|55|AM|third|night|| +19376|AAAAAAAABLLEAAAA|19376|5|22|56|AM|third|night|| +19377|AAAAAAAACLLEAAAA|19377|5|22|57|AM|third|night|| +19378|AAAAAAAADLLEAAAA|19378|5|22|58|AM|third|night|| +19379|AAAAAAAAELLEAAAA|19379|5|22|59|AM|third|night|| +19380|AAAAAAAAFLLEAAAA|19380|5|23|0|AM|third|night|| +19381|AAAAAAAAGLLEAAAA|19381|5|23|1|AM|third|night|| +19382|AAAAAAAAHLLEAAAA|19382|5|23|2|AM|third|night|| +19383|AAAAAAAAILLEAAAA|19383|5|23|3|AM|third|night|| +19384|AAAAAAAAJLLEAAAA|19384|5|23|4|AM|third|night|| +19385|AAAAAAAAKLLEAAAA|19385|5|23|5|AM|third|night|| +19386|AAAAAAAALLLEAAAA|19386|5|23|6|AM|third|night|| +19387|AAAAAAAAMLLEAAAA|19387|5|23|7|AM|third|night|| +19388|AAAAAAAANLLEAAAA|19388|5|23|8|AM|third|night|| +19389|AAAAAAAAOLLEAAAA|19389|5|23|9|AM|third|night|| +19390|AAAAAAAAPLLEAAAA|19390|5|23|10|AM|third|night|| +19391|AAAAAAAAAMLEAAAA|19391|5|23|11|AM|third|night|| +19392|AAAAAAAABMLEAAAA|19392|5|23|12|AM|third|night|| +19393|AAAAAAAACMLEAAAA|19393|5|23|13|AM|third|night|| +19394|AAAAAAAADMLEAAAA|19394|5|23|14|AM|third|night|| +19395|AAAAAAAAEMLEAAAA|19395|5|23|15|AM|third|night|| +19396|AAAAAAAAFMLEAAAA|19396|5|23|16|AM|third|night|| +19397|AAAAAAAAGMLEAAAA|19397|5|23|17|AM|third|night|| +19398|AAAAAAAAHMLEAAAA|19398|5|23|18|AM|third|night|| +19399|AAAAAAAAIMLEAAAA|19399|5|23|19|AM|third|night|| +19400|AAAAAAAAJMLEAAAA|19400|5|23|20|AM|third|night|| +19401|AAAAAAAAKMLEAAAA|19401|5|23|21|AM|third|night|| +19402|AAAAAAAALMLEAAAA|19402|5|23|22|AM|third|night|| +19403|AAAAAAAAMMLEAAAA|19403|5|23|23|AM|third|night|| +19404|AAAAAAAANMLEAAAA|19404|5|23|24|AM|third|night|| +19405|AAAAAAAAOMLEAAAA|19405|5|23|25|AM|third|night|| +19406|AAAAAAAAPMLEAAAA|19406|5|23|26|AM|third|night|| +19407|AAAAAAAAANLEAAAA|19407|5|23|27|AM|third|night|| +19408|AAAAAAAABNLEAAAA|19408|5|23|28|AM|third|night|| +19409|AAAAAAAACNLEAAAA|19409|5|23|29|AM|third|night|| +19410|AAAAAAAADNLEAAAA|19410|5|23|30|AM|third|night|| +19411|AAAAAAAAENLEAAAA|19411|5|23|31|AM|third|night|| +19412|AAAAAAAAFNLEAAAA|19412|5|23|32|AM|third|night|| +19413|AAAAAAAAGNLEAAAA|19413|5|23|33|AM|third|night|| +19414|AAAAAAAAHNLEAAAA|19414|5|23|34|AM|third|night|| +19415|AAAAAAAAINLEAAAA|19415|5|23|35|AM|third|night|| +19416|AAAAAAAAJNLEAAAA|19416|5|23|36|AM|third|night|| +19417|AAAAAAAAKNLEAAAA|19417|5|23|37|AM|third|night|| +19418|AAAAAAAALNLEAAAA|19418|5|23|38|AM|third|night|| +19419|AAAAAAAAMNLEAAAA|19419|5|23|39|AM|third|night|| +19420|AAAAAAAANNLEAAAA|19420|5|23|40|AM|third|night|| +19421|AAAAAAAAONLEAAAA|19421|5|23|41|AM|third|night|| +19422|AAAAAAAAPNLEAAAA|19422|5|23|42|AM|third|night|| +19423|AAAAAAAAAOLEAAAA|19423|5|23|43|AM|third|night|| +19424|AAAAAAAABOLEAAAA|19424|5|23|44|AM|third|night|| +19425|AAAAAAAACOLEAAAA|19425|5|23|45|AM|third|night|| +19426|AAAAAAAADOLEAAAA|19426|5|23|46|AM|third|night|| +19427|AAAAAAAAEOLEAAAA|19427|5|23|47|AM|third|night|| +19428|AAAAAAAAFOLEAAAA|19428|5|23|48|AM|third|night|| +19429|AAAAAAAAGOLEAAAA|19429|5|23|49|AM|third|night|| +19430|AAAAAAAAHOLEAAAA|19430|5|23|50|AM|third|night|| +19431|AAAAAAAAIOLEAAAA|19431|5|23|51|AM|third|night|| +19432|AAAAAAAAJOLEAAAA|19432|5|23|52|AM|third|night|| +19433|AAAAAAAAKOLEAAAA|19433|5|23|53|AM|third|night|| +19434|AAAAAAAALOLEAAAA|19434|5|23|54|AM|third|night|| +19435|AAAAAAAAMOLEAAAA|19435|5|23|55|AM|third|night|| +19436|AAAAAAAANOLEAAAA|19436|5|23|56|AM|third|night|| +19437|AAAAAAAAOOLEAAAA|19437|5|23|57|AM|third|night|| +19438|AAAAAAAAPOLEAAAA|19438|5|23|58|AM|third|night|| +19439|AAAAAAAAAPLEAAAA|19439|5|23|59|AM|third|night|| +19440|AAAAAAAABPLEAAAA|19440|5|24|0|AM|third|night|| +19441|AAAAAAAACPLEAAAA|19441|5|24|1|AM|third|night|| +19442|AAAAAAAADPLEAAAA|19442|5|24|2|AM|third|night|| +19443|AAAAAAAAEPLEAAAA|19443|5|24|3|AM|third|night|| +19444|AAAAAAAAFPLEAAAA|19444|5|24|4|AM|third|night|| +19445|AAAAAAAAGPLEAAAA|19445|5|24|5|AM|third|night|| +19446|AAAAAAAAHPLEAAAA|19446|5|24|6|AM|third|night|| +19447|AAAAAAAAIPLEAAAA|19447|5|24|7|AM|third|night|| +19448|AAAAAAAAJPLEAAAA|19448|5|24|8|AM|third|night|| +19449|AAAAAAAAKPLEAAAA|19449|5|24|9|AM|third|night|| +19450|AAAAAAAALPLEAAAA|19450|5|24|10|AM|third|night|| +19451|AAAAAAAAMPLEAAAA|19451|5|24|11|AM|third|night|| +19452|AAAAAAAANPLEAAAA|19452|5|24|12|AM|third|night|| +19453|AAAAAAAAOPLEAAAA|19453|5|24|13|AM|third|night|| +19454|AAAAAAAAPPLEAAAA|19454|5|24|14|AM|third|night|| +19455|AAAAAAAAAAMEAAAA|19455|5|24|15|AM|third|night|| +19456|AAAAAAAABAMEAAAA|19456|5|24|16|AM|third|night|| +19457|AAAAAAAACAMEAAAA|19457|5|24|17|AM|third|night|| +19458|AAAAAAAADAMEAAAA|19458|5|24|18|AM|third|night|| +19459|AAAAAAAAEAMEAAAA|19459|5|24|19|AM|third|night|| +19460|AAAAAAAAFAMEAAAA|19460|5|24|20|AM|third|night|| +19461|AAAAAAAAGAMEAAAA|19461|5|24|21|AM|third|night|| +19462|AAAAAAAAHAMEAAAA|19462|5|24|22|AM|third|night|| +19463|AAAAAAAAIAMEAAAA|19463|5|24|23|AM|third|night|| +19464|AAAAAAAAJAMEAAAA|19464|5|24|24|AM|third|night|| +19465|AAAAAAAAKAMEAAAA|19465|5|24|25|AM|third|night|| +19466|AAAAAAAALAMEAAAA|19466|5|24|26|AM|third|night|| +19467|AAAAAAAAMAMEAAAA|19467|5|24|27|AM|third|night|| +19468|AAAAAAAANAMEAAAA|19468|5|24|28|AM|third|night|| +19469|AAAAAAAAOAMEAAAA|19469|5|24|29|AM|third|night|| +19470|AAAAAAAAPAMEAAAA|19470|5|24|30|AM|third|night|| +19471|AAAAAAAAABMEAAAA|19471|5|24|31|AM|third|night|| +19472|AAAAAAAABBMEAAAA|19472|5|24|32|AM|third|night|| +19473|AAAAAAAACBMEAAAA|19473|5|24|33|AM|third|night|| +19474|AAAAAAAADBMEAAAA|19474|5|24|34|AM|third|night|| +19475|AAAAAAAAEBMEAAAA|19475|5|24|35|AM|third|night|| +19476|AAAAAAAAFBMEAAAA|19476|5|24|36|AM|third|night|| +19477|AAAAAAAAGBMEAAAA|19477|5|24|37|AM|third|night|| +19478|AAAAAAAAHBMEAAAA|19478|5|24|38|AM|third|night|| +19479|AAAAAAAAIBMEAAAA|19479|5|24|39|AM|third|night|| +19480|AAAAAAAAJBMEAAAA|19480|5|24|40|AM|third|night|| +19481|AAAAAAAAKBMEAAAA|19481|5|24|41|AM|third|night|| +19482|AAAAAAAALBMEAAAA|19482|5|24|42|AM|third|night|| +19483|AAAAAAAAMBMEAAAA|19483|5|24|43|AM|third|night|| +19484|AAAAAAAANBMEAAAA|19484|5|24|44|AM|third|night|| +19485|AAAAAAAAOBMEAAAA|19485|5|24|45|AM|third|night|| +19486|AAAAAAAAPBMEAAAA|19486|5|24|46|AM|third|night|| +19487|AAAAAAAAACMEAAAA|19487|5|24|47|AM|third|night|| +19488|AAAAAAAABCMEAAAA|19488|5|24|48|AM|third|night|| +19489|AAAAAAAACCMEAAAA|19489|5|24|49|AM|third|night|| +19490|AAAAAAAADCMEAAAA|19490|5|24|50|AM|third|night|| +19491|AAAAAAAAECMEAAAA|19491|5|24|51|AM|third|night|| +19492|AAAAAAAAFCMEAAAA|19492|5|24|52|AM|third|night|| +19493|AAAAAAAAGCMEAAAA|19493|5|24|53|AM|third|night|| +19494|AAAAAAAAHCMEAAAA|19494|5|24|54|AM|third|night|| +19495|AAAAAAAAICMEAAAA|19495|5|24|55|AM|third|night|| +19496|AAAAAAAAJCMEAAAA|19496|5|24|56|AM|third|night|| +19497|AAAAAAAAKCMEAAAA|19497|5|24|57|AM|third|night|| +19498|AAAAAAAALCMEAAAA|19498|5|24|58|AM|third|night|| +19499|AAAAAAAAMCMEAAAA|19499|5|24|59|AM|third|night|| +19500|AAAAAAAANCMEAAAA|19500|5|25|0|AM|third|night|| +19501|AAAAAAAAOCMEAAAA|19501|5|25|1|AM|third|night|| +19502|AAAAAAAAPCMEAAAA|19502|5|25|2|AM|third|night|| +19503|AAAAAAAAADMEAAAA|19503|5|25|3|AM|third|night|| +19504|AAAAAAAABDMEAAAA|19504|5|25|4|AM|third|night|| +19505|AAAAAAAACDMEAAAA|19505|5|25|5|AM|third|night|| +19506|AAAAAAAADDMEAAAA|19506|5|25|6|AM|third|night|| +19507|AAAAAAAAEDMEAAAA|19507|5|25|7|AM|third|night|| +19508|AAAAAAAAFDMEAAAA|19508|5|25|8|AM|third|night|| +19509|AAAAAAAAGDMEAAAA|19509|5|25|9|AM|third|night|| +19510|AAAAAAAAHDMEAAAA|19510|5|25|10|AM|third|night|| +19511|AAAAAAAAIDMEAAAA|19511|5|25|11|AM|third|night|| +19512|AAAAAAAAJDMEAAAA|19512|5|25|12|AM|third|night|| +19513|AAAAAAAAKDMEAAAA|19513|5|25|13|AM|third|night|| +19514|AAAAAAAALDMEAAAA|19514|5|25|14|AM|third|night|| +19515|AAAAAAAAMDMEAAAA|19515|5|25|15|AM|third|night|| +19516|AAAAAAAANDMEAAAA|19516|5|25|16|AM|third|night|| +19517|AAAAAAAAODMEAAAA|19517|5|25|17|AM|third|night|| +19518|AAAAAAAAPDMEAAAA|19518|5|25|18|AM|third|night|| +19519|AAAAAAAAAEMEAAAA|19519|5|25|19|AM|third|night|| +19520|AAAAAAAABEMEAAAA|19520|5|25|20|AM|third|night|| +19521|AAAAAAAACEMEAAAA|19521|5|25|21|AM|third|night|| +19522|AAAAAAAADEMEAAAA|19522|5|25|22|AM|third|night|| +19523|AAAAAAAAEEMEAAAA|19523|5|25|23|AM|third|night|| +19524|AAAAAAAAFEMEAAAA|19524|5|25|24|AM|third|night|| +19525|AAAAAAAAGEMEAAAA|19525|5|25|25|AM|third|night|| +19526|AAAAAAAAHEMEAAAA|19526|5|25|26|AM|third|night|| +19527|AAAAAAAAIEMEAAAA|19527|5|25|27|AM|third|night|| +19528|AAAAAAAAJEMEAAAA|19528|5|25|28|AM|third|night|| +19529|AAAAAAAAKEMEAAAA|19529|5|25|29|AM|third|night|| +19530|AAAAAAAALEMEAAAA|19530|5|25|30|AM|third|night|| +19531|AAAAAAAAMEMEAAAA|19531|5|25|31|AM|third|night|| +19532|AAAAAAAANEMEAAAA|19532|5|25|32|AM|third|night|| +19533|AAAAAAAAOEMEAAAA|19533|5|25|33|AM|third|night|| +19534|AAAAAAAAPEMEAAAA|19534|5|25|34|AM|third|night|| +19535|AAAAAAAAAFMEAAAA|19535|5|25|35|AM|third|night|| +19536|AAAAAAAABFMEAAAA|19536|5|25|36|AM|third|night|| +19537|AAAAAAAACFMEAAAA|19537|5|25|37|AM|third|night|| +19538|AAAAAAAADFMEAAAA|19538|5|25|38|AM|third|night|| +19539|AAAAAAAAEFMEAAAA|19539|5|25|39|AM|third|night|| +19540|AAAAAAAAFFMEAAAA|19540|5|25|40|AM|third|night|| +19541|AAAAAAAAGFMEAAAA|19541|5|25|41|AM|third|night|| +19542|AAAAAAAAHFMEAAAA|19542|5|25|42|AM|third|night|| +19543|AAAAAAAAIFMEAAAA|19543|5|25|43|AM|third|night|| +19544|AAAAAAAAJFMEAAAA|19544|5|25|44|AM|third|night|| +19545|AAAAAAAAKFMEAAAA|19545|5|25|45|AM|third|night|| +19546|AAAAAAAALFMEAAAA|19546|5|25|46|AM|third|night|| +19547|AAAAAAAAMFMEAAAA|19547|5|25|47|AM|third|night|| +19548|AAAAAAAANFMEAAAA|19548|5|25|48|AM|third|night|| +19549|AAAAAAAAOFMEAAAA|19549|5|25|49|AM|third|night|| +19550|AAAAAAAAPFMEAAAA|19550|5|25|50|AM|third|night|| +19551|AAAAAAAAAGMEAAAA|19551|5|25|51|AM|third|night|| +19552|AAAAAAAABGMEAAAA|19552|5|25|52|AM|third|night|| +19553|AAAAAAAACGMEAAAA|19553|5|25|53|AM|third|night|| +19554|AAAAAAAADGMEAAAA|19554|5|25|54|AM|third|night|| +19555|AAAAAAAAEGMEAAAA|19555|5|25|55|AM|third|night|| +19556|AAAAAAAAFGMEAAAA|19556|5|25|56|AM|third|night|| +19557|AAAAAAAAGGMEAAAA|19557|5|25|57|AM|third|night|| +19558|AAAAAAAAHGMEAAAA|19558|5|25|58|AM|third|night|| +19559|AAAAAAAAIGMEAAAA|19559|5|25|59|AM|third|night|| +19560|AAAAAAAAJGMEAAAA|19560|5|26|0|AM|third|night|| +19561|AAAAAAAAKGMEAAAA|19561|5|26|1|AM|third|night|| +19562|AAAAAAAALGMEAAAA|19562|5|26|2|AM|third|night|| +19563|AAAAAAAAMGMEAAAA|19563|5|26|3|AM|third|night|| +19564|AAAAAAAANGMEAAAA|19564|5|26|4|AM|third|night|| +19565|AAAAAAAAOGMEAAAA|19565|5|26|5|AM|third|night|| +19566|AAAAAAAAPGMEAAAA|19566|5|26|6|AM|third|night|| +19567|AAAAAAAAAHMEAAAA|19567|5|26|7|AM|third|night|| +19568|AAAAAAAABHMEAAAA|19568|5|26|8|AM|third|night|| +19569|AAAAAAAACHMEAAAA|19569|5|26|9|AM|third|night|| +19570|AAAAAAAADHMEAAAA|19570|5|26|10|AM|third|night|| +19571|AAAAAAAAEHMEAAAA|19571|5|26|11|AM|third|night|| +19572|AAAAAAAAFHMEAAAA|19572|5|26|12|AM|third|night|| +19573|AAAAAAAAGHMEAAAA|19573|5|26|13|AM|third|night|| +19574|AAAAAAAAHHMEAAAA|19574|5|26|14|AM|third|night|| +19575|AAAAAAAAIHMEAAAA|19575|5|26|15|AM|third|night|| +19576|AAAAAAAAJHMEAAAA|19576|5|26|16|AM|third|night|| +19577|AAAAAAAAKHMEAAAA|19577|5|26|17|AM|third|night|| +19578|AAAAAAAALHMEAAAA|19578|5|26|18|AM|third|night|| +19579|AAAAAAAAMHMEAAAA|19579|5|26|19|AM|third|night|| +19580|AAAAAAAANHMEAAAA|19580|5|26|20|AM|third|night|| +19581|AAAAAAAAOHMEAAAA|19581|5|26|21|AM|third|night|| +19582|AAAAAAAAPHMEAAAA|19582|5|26|22|AM|third|night|| +19583|AAAAAAAAAIMEAAAA|19583|5|26|23|AM|third|night|| +19584|AAAAAAAABIMEAAAA|19584|5|26|24|AM|third|night|| +19585|AAAAAAAACIMEAAAA|19585|5|26|25|AM|third|night|| +19586|AAAAAAAADIMEAAAA|19586|5|26|26|AM|third|night|| +19587|AAAAAAAAEIMEAAAA|19587|5|26|27|AM|third|night|| +19588|AAAAAAAAFIMEAAAA|19588|5|26|28|AM|third|night|| +19589|AAAAAAAAGIMEAAAA|19589|5|26|29|AM|third|night|| +19590|AAAAAAAAHIMEAAAA|19590|5|26|30|AM|third|night|| +19591|AAAAAAAAIIMEAAAA|19591|5|26|31|AM|third|night|| +19592|AAAAAAAAJIMEAAAA|19592|5|26|32|AM|third|night|| +19593|AAAAAAAAKIMEAAAA|19593|5|26|33|AM|third|night|| +19594|AAAAAAAALIMEAAAA|19594|5|26|34|AM|third|night|| +19595|AAAAAAAAMIMEAAAA|19595|5|26|35|AM|third|night|| +19596|AAAAAAAANIMEAAAA|19596|5|26|36|AM|third|night|| +19597|AAAAAAAAOIMEAAAA|19597|5|26|37|AM|third|night|| +19598|AAAAAAAAPIMEAAAA|19598|5|26|38|AM|third|night|| +19599|AAAAAAAAAJMEAAAA|19599|5|26|39|AM|third|night|| +19600|AAAAAAAABJMEAAAA|19600|5|26|40|AM|third|night|| +19601|AAAAAAAACJMEAAAA|19601|5|26|41|AM|third|night|| +19602|AAAAAAAADJMEAAAA|19602|5|26|42|AM|third|night|| +19603|AAAAAAAAEJMEAAAA|19603|5|26|43|AM|third|night|| +19604|AAAAAAAAFJMEAAAA|19604|5|26|44|AM|third|night|| +19605|AAAAAAAAGJMEAAAA|19605|5|26|45|AM|third|night|| +19606|AAAAAAAAHJMEAAAA|19606|5|26|46|AM|third|night|| +19607|AAAAAAAAIJMEAAAA|19607|5|26|47|AM|third|night|| +19608|AAAAAAAAJJMEAAAA|19608|5|26|48|AM|third|night|| +19609|AAAAAAAAKJMEAAAA|19609|5|26|49|AM|third|night|| +19610|AAAAAAAALJMEAAAA|19610|5|26|50|AM|third|night|| +19611|AAAAAAAAMJMEAAAA|19611|5|26|51|AM|third|night|| +19612|AAAAAAAANJMEAAAA|19612|5|26|52|AM|third|night|| +19613|AAAAAAAAOJMEAAAA|19613|5|26|53|AM|third|night|| +19614|AAAAAAAAPJMEAAAA|19614|5|26|54|AM|third|night|| +19615|AAAAAAAAAKMEAAAA|19615|5|26|55|AM|third|night|| +19616|AAAAAAAABKMEAAAA|19616|5|26|56|AM|third|night|| +19617|AAAAAAAACKMEAAAA|19617|5|26|57|AM|third|night|| +19618|AAAAAAAADKMEAAAA|19618|5|26|58|AM|third|night|| +19619|AAAAAAAAEKMEAAAA|19619|5|26|59|AM|third|night|| +19620|AAAAAAAAFKMEAAAA|19620|5|27|0|AM|third|night|| +19621|AAAAAAAAGKMEAAAA|19621|5|27|1|AM|third|night|| +19622|AAAAAAAAHKMEAAAA|19622|5|27|2|AM|third|night|| +19623|AAAAAAAAIKMEAAAA|19623|5|27|3|AM|third|night|| +19624|AAAAAAAAJKMEAAAA|19624|5|27|4|AM|third|night|| +19625|AAAAAAAAKKMEAAAA|19625|5|27|5|AM|third|night|| +19626|AAAAAAAALKMEAAAA|19626|5|27|6|AM|third|night|| +19627|AAAAAAAAMKMEAAAA|19627|5|27|7|AM|third|night|| +19628|AAAAAAAANKMEAAAA|19628|5|27|8|AM|third|night|| +19629|AAAAAAAAOKMEAAAA|19629|5|27|9|AM|third|night|| +19630|AAAAAAAAPKMEAAAA|19630|5|27|10|AM|third|night|| +19631|AAAAAAAAALMEAAAA|19631|5|27|11|AM|third|night|| +19632|AAAAAAAABLMEAAAA|19632|5|27|12|AM|third|night|| +19633|AAAAAAAACLMEAAAA|19633|5|27|13|AM|third|night|| +19634|AAAAAAAADLMEAAAA|19634|5|27|14|AM|third|night|| +19635|AAAAAAAAELMEAAAA|19635|5|27|15|AM|third|night|| +19636|AAAAAAAAFLMEAAAA|19636|5|27|16|AM|third|night|| +19637|AAAAAAAAGLMEAAAA|19637|5|27|17|AM|third|night|| +19638|AAAAAAAAHLMEAAAA|19638|5|27|18|AM|third|night|| +19639|AAAAAAAAILMEAAAA|19639|5|27|19|AM|third|night|| +19640|AAAAAAAAJLMEAAAA|19640|5|27|20|AM|third|night|| +19641|AAAAAAAAKLMEAAAA|19641|5|27|21|AM|third|night|| +19642|AAAAAAAALLMEAAAA|19642|5|27|22|AM|third|night|| +19643|AAAAAAAAMLMEAAAA|19643|5|27|23|AM|third|night|| +19644|AAAAAAAANLMEAAAA|19644|5|27|24|AM|third|night|| +19645|AAAAAAAAOLMEAAAA|19645|5|27|25|AM|third|night|| +19646|AAAAAAAAPLMEAAAA|19646|5|27|26|AM|third|night|| +19647|AAAAAAAAAMMEAAAA|19647|5|27|27|AM|third|night|| +19648|AAAAAAAABMMEAAAA|19648|5|27|28|AM|third|night|| +19649|AAAAAAAACMMEAAAA|19649|5|27|29|AM|third|night|| +19650|AAAAAAAADMMEAAAA|19650|5|27|30|AM|third|night|| +19651|AAAAAAAAEMMEAAAA|19651|5|27|31|AM|third|night|| +19652|AAAAAAAAFMMEAAAA|19652|5|27|32|AM|third|night|| +19653|AAAAAAAAGMMEAAAA|19653|5|27|33|AM|third|night|| +19654|AAAAAAAAHMMEAAAA|19654|5|27|34|AM|third|night|| +19655|AAAAAAAAIMMEAAAA|19655|5|27|35|AM|third|night|| +19656|AAAAAAAAJMMEAAAA|19656|5|27|36|AM|third|night|| +19657|AAAAAAAAKMMEAAAA|19657|5|27|37|AM|third|night|| +19658|AAAAAAAALMMEAAAA|19658|5|27|38|AM|third|night|| +19659|AAAAAAAAMMMEAAAA|19659|5|27|39|AM|third|night|| +19660|AAAAAAAANMMEAAAA|19660|5|27|40|AM|third|night|| +19661|AAAAAAAAOMMEAAAA|19661|5|27|41|AM|third|night|| +19662|AAAAAAAAPMMEAAAA|19662|5|27|42|AM|third|night|| +19663|AAAAAAAAANMEAAAA|19663|5|27|43|AM|third|night|| +19664|AAAAAAAABNMEAAAA|19664|5|27|44|AM|third|night|| +19665|AAAAAAAACNMEAAAA|19665|5|27|45|AM|third|night|| +19666|AAAAAAAADNMEAAAA|19666|5|27|46|AM|third|night|| +19667|AAAAAAAAENMEAAAA|19667|5|27|47|AM|third|night|| +19668|AAAAAAAAFNMEAAAA|19668|5|27|48|AM|third|night|| +19669|AAAAAAAAGNMEAAAA|19669|5|27|49|AM|third|night|| +19670|AAAAAAAAHNMEAAAA|19670|5|27|50|AM|third|night|| +19671|AAAAAAAAINMEAAAA|19671|5|27|51|AM|third|night|| +19672|AAAAAAAAJNMEAAAA|19672|5|27|52|AM|third|night|| +19673|AAAAAAAAKNMEAAAA|19673|5|27|53|AM|third|night|| +19674|AAAAAAAALNMEAAAA|19674|5|27|54|AM|third|night|| +19675|AAAAAAAAMNMEAAAA|19675|5|27|55|AM|third|night|| +19676|AAAAAAAANNMEAAAA|19676|5|27|56|AM|third|night|| +19677|AAAAAAAAONMEAAAA|19677|5|27|57|AM|third|night|| +19678|AAAAAAAAPNMEAAAA|19678|5|27|58|AM|third|night|| +19679|AAAAAAAAAOMEAAAA|19679|5|27|59|AM|third|night|| +19680|AAAAAAAABOMEAAAA|19680|5|28|0|AM|third|night|| +19681|AAAAAAAACOMEAAAA|19681|5|28|1|AM|third|night|| +19682|AAAAAAAADOMEAAAA|19682|5|28|2|AM|third|night|| +19683|AAAAAAAAEOMEAAAA|19683|5|28|3|AM|third|night|| +19684|AAAAAAAAFOMEAAAA|19684|5|28|4|AM|third|night|| +19685|AAAAAAAAGOMEAAAA|19685|5|28|5|AM|third|night|| +19686|AAAAAAAAHOMEAAAA|19686|5|28|6|AM|third|night|| +19687|AAAAAAAAIOMEAAAA|19687|5|28|7|AM|third|night|| +19688|AAAAAAAAJOMEAAAA|19688|5|28|8|AM|third|night|| +19689|AAAAAAAAKOMEAAAA|19689|5|28|9|AM|third|night|| +19690|AAAAAAAALOMEAAAA|19690|5|28|10|AM|third|night|| +19691|AAAAAAAAMOMEAAAA|19691|5|28|11|AM|third|night|| +19692|AAAAAAAANOMEAAAA|19692|5|28|12|AM|third|night|| +19693|AAAAAAAAOOMEAAAA|19693|5|28|13|AM|third|night|| +19694|AAAAAAAAPOMEAAAA|19694|5|28|14|AM|third|night|| +19695|AAAAAAAAAPMEAAAA|19695|5|28|15|AM|third|night|| +19696|AAAAAAAABPMEAAAA|19696|5|28|16|AM|third|night|| +19697|AAAAAAAACPMEAAAA|19697|5|28|17|AM|third|night|| +19698|AAAAAAAADPMEAAAA|19698|5|28|18|AM|third|night|| +19699|AAAAAAAAEPMEAAAA|19699|5|28|19|AM|third|night|| +19700|AAAAAAAAFPMEAAAA|19700|5|28|20|AM|third|night|| +19701|AAAAAAAAGPMEAAAA|19701|5|28|21|AM|third|night|| +19702|AAAAAAAAHPMEAAAA|19702|5|28|22|AM|third|night|| +19703|AAAAAAAAIPMEAAAA|19703|5|28|23|AM|third|night|| +19704|AAAAAAAAJPMEAAAA|19704|5|28|24|AM|third|night|| +19705|AAAAAAAAKPMEAAAA|19705|5|28|25|AM|third|night|| +19706|AAAAAAAALPMEAAAA|19706|5|28|26|AM|third|night|| +19707|AAAAAAAAMPMEAAAA|19707|5|28|27|AM|third|night|| +19708|AAAAAAAANPMEAAAA|19708|5|28|28|AM|third|night|| +19709|AAAAAAAAOPMEAAAA|19709|5|28|29|AM|third|night|| +19710|AAAAAAAAPPMEAAAA|19710|5|28|30|AM|third|night|| +19711|AAAAAAAAAANEAAAA|19711|5|28|31|AM|third|night|| +19712|AAAAAAAABANEAAAA|19712|5|28|32|AM|third|night|| +19713|AAAAAAAACANEAAAA|19713|5|28|33|AM|third|night|| +19714|AAAAAAAADANEAAAA|19714|5|28|34|AM|third|night|| +19715|AAAAAAAAEANEAAAA|19715|5|28|35|AM|third|night|| +19716|AAAAAAAAFANEAAAA|19716|5|28|36|AM|third|night|| +19717|AAAAAAAAGANEAAAA|19717|5|28|37|AM|third|night|| +19718|AAAAAAAAHANEAAAA|19718|5|28|38|AM|third|night|| +19719|AAAAAAAAIANEAAAA|19719|5|28|39|AM|third|night|| +19720|AAAAAAAAJANEAAAA|19720|5|28|40|AM|third|night|| +19721|AAAAAAAAKANEAAAA|19721|5|28|41|AM|third|night|| +19722|AAAAAAAALANEAAAA|19722|5|28|42|AM|third|night|| +19723|AAAAAAAAMANEAAAA|19723|5|28|43|AM|third|night|| +19724|AAAAAAAANANEAAAA|19724|5|28|44|AM|third|night|| +19725|AAAAAAAAOANEAAAA|19725|5|28|45|AM|third|night|| +19726|AAAAAAAAPANEAAAA|19726|5|28|46|AM|third|night|| +19727|AAAAAAAAABNEAAAA|19727|5|28|47|AM|third|night|| +19728|AAAAAAAABBNEAAAA|19728|5|28|48|AM|third|night|| +19729|AAAAAAAACBNEAAAA|19729|5|28|49|AM|third|night|| +19730|AAAAAAAADBNEAAAA|19730|5|28|50|AM|third|night|| +19731|AAAAAAAAEBNEAAAA|19731|5|28|51|AM|third|night|| +19732|AAAAAAAAFBNEAAAA|19732|5|28|52|AM|third|night|| +19733|AAAAAAAAGBNEAAAA|19733|5|28|53|AM|third|night|| +19734|AAAAAAAAHBNEAAAA|19734|5|28|54|AM|third|night|| +19735|AAAAAAAAIBNEAAAA|19735|5|28|55|AM|third|night|| +19736|AAAAAAAAJBNEAAAA|19736|5|28|56|AM|third|night|| +19737|AAAAAAAAKBNEAAAA|19737|5|28|57|AM|third|night|| +19738|AAAAAAAALBNEAAAA|19738|5|28|58|AM|third|night|| +19739|AAAAAAAAMBNEAAAA|19739|5|28|59|AM|third|night|| +19740|AAAAAAAANBNEAAAA|19740|5|29|0|AM|third|night|| +19741|AAAAAAAAOBNEAAAA|19741|5|29|1|AM|third|night|| +19742|AAAAAAAAPBNEAAAA|19742|5|29|2|AM|third|night|| +19743|AAAAAAAAACNEAAAA|19743|5|29|3|AM|third|night|| +19744|AAAAAAAABCNEAAAA|19744|5|29|4|AM|third|night|| +19745|AAAAAAAACCNEAAAA|19745|5|29|5|AM|third|night|| +19746|AAAAAAAADCNEAAAA|19746|5|29|6|AM|third|night|| +19747|AAAAAAAAECNEAAAA|19747|5|29|7|AM|third|night|| +19748|AAAAAAAAFCNEAAAA|19748|5|29|8|AM|third|night|| +19749|AAAAAAAAGCNEAAAA|19749|5|29|9|AM|third|night|| +19750|AAAAAAAAHCNEAAAA|19750|5|29|10|AM|third|night|| +19751|AAAAAAAAICNEAAAA|19751|5|29|11|AM|third|night|| +19752|AAAAAAAAJCNEAAAA|19752|5|29|12|AM|third|night|| +19753|AAAAAAAAKCNEAAAA|19753|5|29|13|AM|third|night|| +19754|AAAAAAAALCNEAAAA|19754|5|29|14|AM|third|night|| +19755|AAAAAAAAMCNEAAAA|19755|5|29|15|AM|third|night|| +19756|AAAAAAAANCNEAAAA|19756|5|29|16|AM|third|night|| +19757|AAAAAAAAOCNEAAAA|19757|5|29|17|AM|third|night|| +19758|AAAAAAAAPCNEAAAA|19758|5|29|18|AM|third|night|| +19759|AAAAAAAAADNEAAAA|19759|5|29|19|AM|third|night|| +19760|AAAAAAAABDNEAAAA|19760|5|29|20|AM|third|night|| +19761|AAAAAAAACDNEAAAA|19761|5|29|21|AM|third|night|| +19762|AAAAAAAADDNEAAAA|19762|5|29|22|AM|third|night|| +19763|AAAAAAAAEDNEAAAA|19763|5|29|23|AM|third|night|| +19764|AAAAAAAAFDNEAAAA|19764|5|29|24|AM|third|night|| +19765|AAAAAAAAGDNEAAAA|19765|5|29|25|AM|third|night|| +19766|AAAAAAAAHDNEAAAA|19766|5|29|26|AM|third|night|| +19767|AAAAAAAAIDNEAAAA|19767|5|29|27|AM|third|night|| +19768|AAAAAAAAJDNEAAAA|19768|5|29|28|AM|third|night|| +19769|AAAAAAAAKDNEAAAA|19769|5|29|29|AM|third|night|| +19770|AAAAAAAALDNEAAAA|19770|5|29|30|AM|third|night|| +19771|AAAAAAAAMDNEAAAA|19771|5|29|31|AM|third|night|| +19772|AAAAAAAANDNEAAAA|19772|5|29|32|AM|third|night|| +19773|AAAAAAAAODNEAAAA|19773|5|29|33|AM|third|night|| +19774|AAAAAAAAPDNEAAAA|19774|5|29|34|AM|third|night|| +19775|AAAAAAAAAENEAAAA|19775|5|29|35|AM|third|night|| +19776|AAAAAAAABENEAAAA|19776|5|29|36|AM|third|night|| +19777|AAAAAAAACENEAAAA|19777|5|29|37|AM|third|night|| +19778|AAAAAAAADENEAAAA|19778|5|29|38|AM|third|night|| +19779|AAAAAAAAEENEAAAA|19779|5|29|39|AM|third|night|| +19780|AAAAAAAAFENEAAAA|19780|5|29|40|AM|third|night|| +19781|AAAAAAAAGENEAAAA|19781|5|29|41|AM|third|night|| +19782|AAAAAAAAHENEAAAA|19782|5|29|42|AM|third|night|| +19783|AAAAAAAAIENEAAAA|19783|5|29|43|AM|third|night|| +19784|AAAAAAAAJENEAAAA|19784|5|29|44|AM|third|night|| +19785|AAAAAAAAKENEAAAA|19785|5|29|45|AM|third|night|| +19786|AAAAAAAALENEAAAA|19786|5|29|46|AM|third|night|| +19787|AAAAAAAAMENEAAAA|19787|5|29|47|AM|third|night|| +19788|AAAAAAAANENEAAAA|19788|5|29|48|AM|third|night|| +19789|AAAAAAAAOENEAAAA|19789|5|29|49|AM|third|night|| +19790|AAAAAAAAPENEAAAA|19790|5|29|50|AM|third|night|| +19791|AAAAAAAAAFNEAAAA|19791|5|29|51|AM|third|night|| +19792|AAAAAAAABFNEAAAA|19792|5|29|52|AM|third|night|| +19793|AAAAAAAACFNEAAAA|19793|5|29|53|AM|third|night|| +19794|AAAAAAAADFNEAAAA|19794|5|29|54|AM|third|night|| +19795|AAAAAAAAEFNEAAAA|19795|5|29|55|AM|third|night|| +19796|AAAAAAAAFFNEAAAA|19796|5|29|56|AM|third|night|| +19797|AAAAAAAAGFNEAAAA|19797|5|29|57|AM|third|night|| +19798|AAAAAAAAHFNEAAAA|19798|5|29|58|AM|third|night|| +19799|AAAAAAAAIFNEAAAA|19799|5|29|59|AM|third|night|| +19800|AAAAAAAAJFNEAAAA|19800|5|30|0|AM|third|night|| +19801|AAAAAAAAKFNEAAAA|19801|5|30|1|AM|third|night|| +19802|AAAAAAAALFNEAAAA|19802|5|30|2|AM|third|night|| +19803|AAAAAAAAMFNEAAAA|19803|5|30|3|AM|third|night|| +19804|AAAAAAAANFNEAAAA|19804|5|30|4|AM|third|night|| +19805|AAAAAAAAOFNEAAAA|19805|5|30|5|AM|third|night|| +19806|AAAAAAAAPFNEAAAA|19806|5|30|6|AM|third|night|| +19807|AAAAAAAAAGNEAAAA|19807|5|30|7|AM|third|night|| +19808|AAAAAAAABGNEAAAA|19808|5|30|8|AM|third|night|| +19809|AAAAAAAACGNEAAAA|19809|5|30|9|AM|third|night|| +19810|AAAAAAAADGNEAAAA|19810|5|30|10|AM|third|night|| +19811|AAAAAAAAEGNEAAAA|19811|5|30|11|AM|third|night|| +19812|AAAAAAAAFGNEAAAA|19812|5|30|12|AM|third|night|| +19813|AAAAAAAAGGNEAAAA|19813|5|30|13|AM|third|night|| +19814|AAAAAAAAHGNEAAAA|19814|5|30|14|AM|third|night|| +19815|AAAAAAAAIGNEAAAA|19815|5|30|15|AM|third|night|| +19816|AAAAAAAAJGNEAAAA|19816|5|30|16|AM|third|night|| +19817|AAAAAAAAKGNEAAAA|19817|5|30|17|AM|third|night|| +19818|AAAAAAAALGNEAAAA|19818|5|30|18|AM|third|night|| +19819|AAAAAAAAMGNEAAAA|19819|5|30|19|AM|third|night|| +19820|AAAAAAAANGNEAAAA|19820|5|30|20|AM|third|night|| +19821|AAAAAAAAOGNEAAAA|19821|5|30|21|AM|third|night|| +19822|AAAAAAAAPGNEAAAA|19822|5|30|22|AM|third|night|| +19823|AAAAAAAAAHNEAAAA|19823|5|30|23|AM|third|night|| +19824|AAAAAAAABHNEAAAA|19824|5|30|24|AM|third|night|| +19825|AAAAAAAACHNEAAAA|19825|5|30|25|AM|third|night|| +19826|AAAAAAAADHNEAAAA|19826|5|30|26|AM|third|night|| +19827|AAAAAAAAEHNEAAAA|19827|5|30|27|AM|third|night|| +19828|AAAAAAAAFHNEAAAA|19828|5|30|28|AM|third|night|| +19829|AAAAAAAAGHNEAAAA|19829|5|30|29|AM|third|night|| +19830|AAAAAAAAHHNEAAAA|19830|5|30|30|AM|third|night|| +19831|AAAAAAAAIHNEAAAA|19831|5|30|31|AM|third|night|| +19832|AAAAAAAAJHNEAAAA|19832|5|30|32|AM|third|night|| +19833|AAAAAAAAKHNEAAAA|19833|5|30|33|AM|third|night|| +19834|AAAAAAAALHNEAAAA|19834|5|30|34|AM|third|night|| +19835|AAAAAAAAMHNEAAAA|19835|5|30|35|AM|third|night|| +19836|AAAAAAAANHNEAAAA|19836|5|30|36|AM|third|night|| +19837|AAAAAAAAOHNEAAAA|19837|5|30|37|AM|third|night|| +19838|AAAAAAAAPHNEAAAA|19838|5|30|38|AM|third|night|| +19839|AAAAAAAAAINEAAAA|19839|5|30|39|AM|third|night|| +19840|AAAAAAAABINEAAAA|19840|5|30|40|AM|third|night|| +19841|AAAAAAAACINEAAAA|19841|5|30|41|AM|third|night|| +19842|AAAAAAAADINEAAAA|19842|5|30|42|AM|third|night|| +19843|AAAAAAAAEINEAAAA|19843|5|30|43|AM|third|night|| +19844|AAAAAAAAFINEAAAA|19844|5|30|44|AM|third|night|| +19845|AAAAAAAAGINEAAAA|19845|5|30|45|AM|third|night|| +19846|AAAAAAAAHINEAAAA|19846|5|30|46|AM|third|night|| +19847|AAAAAAAAIINEAAAA|19847|5|30|47|AM|third|night|| +19848|AAAAAAAAJINEAAAA|19848|5|30|48|AM|third|night|| +19849|AAAAAAAAKINEAAAA|19849|5|30|49|AM|third|night|| +19850|AAAAAAAALINEAAAA|19850|5|30|50|AM|third|night|| +19851|AAAAAAAAMINEAAAA|19851|5|30|51|AM|third|night|| +19852|AAAAAAAANINEAAAA|19852|5|30|52|AM|third|night|| +19853|AAAAAAAAOINEAAAA|19853|5|30|53|AM|third|night|| +19854|AAAAAAAAPINEAAAA|19854|5|30|54|AM|third|night|| +19855|AAAAAAAAAJNEAAAA|19855|5|30|55|AM|third|night|| +19856|AAAAAAAABJNEAAAA|19856|5|30|56|AM|third|night|| +19857|AAAAAAAACJNEAAAA|19857|5|30|57|AM|third|night|| +19858|AAAAAAAADJNEAAAA|19858|5|30|58|AM|third|night|| +19859|AAAAAAAAEJNEAAAA|19859|5|30|59|AM|third|night|| +19860|AAAAAAAAFJNEAAAA|19860|5|31|0|AM|third|night|| +19861|AAAAAAAAGJNEAAAA|19861|5|31|1|AM|third|night|| +19862|AAAAAAAAHJNEAAAA|19862|5|31|2|AM|third|night|| +19863|AAAAAAAAIJNEAAAA|19863|5|31|3|AM|third|night|| +19864|AAAAAAAAJJNEAAAA|19864|5|31|4|AM|third|night|| +19865|AAAAAAAAKJNEAAAA|19865|5|31|5|AM|third|night|| +19866|AAAAAAAALJNEAAAA|19866|5|31|6|AM|third|night|| +19867|AAAAAAAAMJNEAAAA|19867|5|31|7|AM|third|night|| +19868|AAAAAAAANJNEAAAA|19868|5|31|8|AM|third|night|| +19869|AAAAAAAAOJNEAAAA|19869|5|31|9|AM|third|night|| +19870|AAAAAAAAPJNEAAAA|19870|5|31|10|AM|third|night|| +19871|AAAAAAAAAKNEAAAA|19871|5|31|11|AM|third|night|| +19872|AAAAAAAABKNEAAAA|19872|5|31|12|AM|third|night|| +19873|AAAAAAAACKNEAAAA|19873|5|31|13|AM|third|night|| +19874|AAAAAAAADKNEAAAA|19874|5|31|14|AM|third|night|| +19875|AAAAAAAAEKNEAAAA|19875|5|31|15|AM|third|night|| +19876|AAAAAAAAFKNEAAAA|19876|5|31|16|AM|third|night|| +19877|AAAAAAAAGKNEAAAA|19877|5|31|17|AM|third|night|| +19878|AAAAAAAAHKNEAAAA|19878|5|31|18|AM|third|night|| +19879|AAAAAAAAIKNEAAAA|19879|5|31|19|AM|third|night|| +19880|AAAAAAAAJKNEAAAA|19880|5|31|20|AM|third|night|| +19881|AAAAAAAAKKNEAAAA|19881|5|31|21|AM|third|night|| +19882|AAAAAAAALKNEAAAA|19882|5|31|22|AM|third|night|| +19883|AAAAAAAAMKNEAAAA|19883|5|31|23|AM|third|night|| +19884|AAAAAAAANKNEAAAA|19884|5|31|24|AM|third|night|| +19885|AAAAAAAAOKNEAAAA|19885|5|31|25|AM|third|night|| +19886|AAAAAAAAPKNEAAAA|19886|5|31|26|AM|third|night|| +19887|AAAAAAAAALNEAAAA|19887|5|31|27|AM|third|night|| +19888|AAAAAAAABLNEAAAA|19888|5|31|28|AM|third|night|| +19889|AAAAAAAACLNEAAAA|19889|5|31|29|AM|third|night|| +19890|AAAAAAAADLNEAAAA|19890|5|31|30|AM|third|night|| +19891|AAAAAAAAELNEAAAA|19891|5|31|31|AM|third|night|| +19892|AAAAAAAAFLNEAAAA|19892|5|31|32|AM|third|night|| +19893|AAAAAAAAGLNEAAAA|19893|5|31|33|AM|third|night|| +19894|AAAAAAAAHLNEAAAA|19894|5|31|34|AM|third|night|| +19895|AAAAAAAAILNEAAAA|19895|5|31|35|AM|third|night|| +19896|AAAAAAAAJLNEAAAA|19896|5|31|36|AM|third|night|| +19897|AAAAAAAAKLNEAAAA|19897|5|31|37|AM|third|night|| +19898|AAAAAAAALLNEAAAA|19898|5|31|38|AM|third|night|| +19899|AAAAAAAAMLNEAAAA|19899|5|31|39|AM|third|night|| +19900|AAAAAAAANLNEAAAA|19900|5|31|40|AM|third|night|| +19901|AAAAAAAAOLNEAAAA|19901|5|31|41|AM|third|night|| +19902|AAAAAAAAPLNEAAAA|19902|5|31|42|AM|third|night|| +19903|AAAAAAAAAMNEAAAA|19903|5|31|43|AM|third|night|| +19904|AAAAAAAABMNEAAAA|19904|5|31|44|AM|third|night|| +19905|AAAAAAAACMNEAAAA|19905|5|31|45|AM|third|night|| +19906|AAAAAAAADMNEAAAA|19906|5|31|46|AM|third|night|| +19907|AAAAAAAAEMNEAAAA|19907|5|31|47|AM|third|night|| +19908|AAAAAAAAFMNEAAAA|19908|5|31|48|AM|third|night|| +19909|AAAAAAAAGMNEAAAA|19909|5|31|49|AM|third|night|| +19910|AAAAAAAAHMNEAAAA|19910|5|31|50|AM|third|night|| +19911|AAAAAAAAIMNEAAAA|19911|5|31|51|AM|third|night|| +19912|AAAAAAAAJMNEAAAA|19912|5|31|52|AM|third|night|| +19913|AAAAAAAAKMNEAAAA|19913|5|31|53|AM|third|night|| +19914|AAAAAAAALMNEAAAA|19914|5|31|54|AM|third|night|| +19915|AAAAAAAAMMNEAAAA|19915|5|31|55|AM|third|night|| +19916|AAAAAAAANMNEAAAA|19916|5|31|56|AM|third|night|| +19917|AAAAAAAAOMNEAAAA|19917|5|31|57|AM|third|night|| +19918|AAAAAAAAPMNEAAAA|19918|5|31|58|AM|third|night|| +19919|AAAAAAAAANNEAAAA|19919|5|31|59|AM|third|night|| +19920|AAAAAAAABNNEAAAA|19920|5|32|0|AM|third|night|| +19921|AAAAAAAACNNEAAAA|19921|5|32|1|AM|third|night|| +19922|AAAAAAAADNNEAAAA|19922|5|32|2|AM|third|night|| +19923|AAAAAAAAENNEAAAA|19923|5|32|3|AM|third|night|| +19924|AAAAAAAAFNNEAAAA|19924|5|32|4|AM|third|night|| +19925|AAAAAAAAGNNEAAAA|19925|5|32|5|AM|third|night|| +19926|AAAAAAAAHNNEAAAA|19926|5|32|6|AM|third|night|| +19927|AAAAAAAAINNEAAAA|19927|5|32|7|AM|third|night|| +19928|AAAAAAAAJNNEAAAA|19928|5|32|8|AM|third|night|| +19929|AAAAAAAAKNNEAAAA|19929|5|32|9|AM|third|night|| +19930|AAAAAAAALNNEAAAA|19930|5|32|10|AM|third|night|| +19931|AAAAAAAAMNNEAAAA|19931|5|32|11|AM|third|night|| +19932|AAAAAAAANNNEAAAA|19932|5|32|12|AM|third|night|| +19933|AAAAAAAAONNEAAAA|19933|5|32|13|AM|third|night|| +19934|AAAAAAAAPNNEAAAA|19934|5|32|14|AM|third|night|| +19935|AAAAAAAAAONEAAAA|19935|5|32|15|AM|third|night|| +19936|AAAAAAAABONEAAAA|19936|5|32|16|AM|third|night|| +19937|AAAAAAAACONEAAAA|19937|5|32|17|AM|third|night|| +19938|AAAAAAAADONEAAAA|19938|5|32|18|AM|third|night|| +19939|AAAAAAAAEONEAAAA|19939|5|32|19|AM|third|night|| +19940|AAAAAAAAFONEAAAA|19940|5|32|20|AM|third|night|| +19941|AAAAAAAAGONEAAAA|19941|5|32|21|AM|third|night|| +19942|AAAAAAAAHONEAAAA|19942|5|32|22|AM|third|night|| +19943|AAAAAAAAIONEAAAA|19943|5|32|23|AM|third|night|| +19944|AAAAAAAAJONEAAAA|19944|5|32|24|AM|third|night|| +19945|AAAAAAAAKONEAAAA|19945|5|32|25|AM|third|night|| +19946|AAAAAAAALONEAAAA|19946|5|32|26|AM|third|night|| +19947|AAAAAAAAMONEAAAA|19947|5|32|27|AM|third|night|| +19948|AAAAAAAANONEAAAA|19948|5|32|28|AM|third|night|| +19949|AAAAAAAAOONEAAAA|19949|5|32|29|AM|third|night|| +19950|AAAAAAAAPONEAAAA|19950|5|32|30|AM|third|night|| +19951|AAAAAAAAAPNEAAAA|19951|5|32|31|AM|third|night|| +19952|AAAAAAAABPNEAAAA|19952|5|32|32|AM|third|night|| +19953|AAAAAAAACPNEAAAA|19953|5|32|33|AM|third|night|| +19954|AAAAAAAADPNEAAAA|19954|5|32|34|AM|third|night|| +19955|AAAAAAAAEPNEAAAA|19955|5|32|35|AM|third|night|| +19956|AAAAAAAAFPNEAAAA|19956|5|32|36|AM|third|night|| +19957|AAAAAAAAGPNEAAAA|19957|5|32|37|AM|third|night|| +19958|AAAAAAAAHPNEAAAA|19958|5|32|38|AM|third|night|| +19959|AAAAAAAAIPNEAAAA|19959|5|32|39|AM|third|night|| +19960|AAAAAAAAJPNEAAAA|19960|5|32|40|AM|third|night|| +19961|AAAAAAAAKPNEAAAA|19961|5|32|41|AM|third|night|| +19962|AAAAAAAALPNEAAAA|19962|5|32|42|AM|third|night|| +19963|AAAAAAAAMPNEAAAA|19963|5|32|43|AM|third|night|| +19964|AAAAAAAANPNEAAAA|19964|5|32|44|AM|third|night|| +19965|AAAAAAAAOPNEAAAA|19965|5|32|45|AM|third|night|| +19966|AAAAAAAAPPNEAAAA|19966|5|32|46|AM|third|night|| +19967|AAAAAAAAAAOEAAAA|19967|5|32|47|AM|third|night|| +19968|AAAAAAAABAOEAAAA|19968|5|32|48|AM|third|night|| +19969|AAAAAAAACAOEAAAA|19969|5|32|49|AM|third|night|| +19970|AAAAAAAADAOEAAAA|19970|5|32|50|AM|third|night|| +19971|AAAAAAAAEAOEAAAA|19971|5|32|51|AM|third|night|| +19972|AAAAAAAAFAOEAAAA|19972|5|32|52|AM|third|night|| +19973|AAAAAAAAGAOEAAAA|19973|5|32|53|AM|third|night|| +19974|AAAAAAAAHAOEAAAA|19974|5|32|54|AM|third|night|| +19975|AAAAAAAAIAOEAAAA|19975|5|32|55|AM|third|night|| +19976|AAAAAAAAJAOEAAAA|19976|5|32|56|AM|third|night|| +19977|AAAAAAAAKAOEAAAA|19977|5|32|57|AM|third|night|| +19978|AAAAAAAALAOEAAAA|19978|5|32|58|AM|third|night|| +19979|AAAAAAAAMAOEAAAA|19979|5|32|59|AM|third|night|| +19980|AAAAAAAANAOEAAAA|19980|5|33|0|AM|third|night|| +19981|AAAAAAAAOAOEAAAA|19981|5|33|1|AM|third|night|| +19982|AAAAAAAAPAOEAAAA|19982|5|33|2|AM|third|night|| +19983|AAAAAAAAABOEAAAA|19983|5|33|3|AM|third|night|| +19984|AAAAAAAABBOEAAAA|19984|5|33|4|AM|third|night|| +19985|AAAAAAAACBOEAAAA|19985|5|33|5|AM|third|night|| +19986|AAAAAAAADBOEAAAA|19986|5|33|6|AM|third|night|| +19987|AAAAAAAAEBOEAAAA|19987|5|33|7|AM|third|night|| +19988|AAAAAAAAFBOEAAAA|19988|5|33|8|AM|third|night|| +19989|AAAAAAAAGBOEAAAA|19989|5|33|9|AM|third|night|| +19990|AAAAAAAAHBOEAAAA|19990|5|33|10|AM|third|night|| +19991|AAAAAAAAIBOEAAAA|19991|5|33|11|AM|third|night|| +19992|AAAAAAAAJBOEAAAA|19992|5|33|12|AM|third|night|| +19993|AAAAAAAAKBOEAAAA|19993|5|33|13|AM|third|night|| +19994|AAAAAAAALBOEAAAA|19994|5|33|14|AM|third|night|| +19995|AAAAAAAAMBOEAAAA|19995|5|33|15|AM|third|night|| +19996|AAAAAAAANBOEAAAA|19996|5|33|16|AM|third|night|| +19997|AAAAAAAAOBOEAAAA|19997|5|33|17|AM|third|night|| +19998|AAAAAAAAPBOEAAAA|19998|5|33|18|AM|third|night|| +19999|AAAAAAAAACOEAAAA|19999|5|33|19|AM|third|night|| +20000|AAAAAAAABCOEAAAA|20000|5|33|20|AM|third|night|| +20001|AAAAAAAACCOEAAAA|20001|5|33|21|AM|third|night|| +20002|AAAAAAAADCOEAAAA|20002|5|33|22|AM|third|night|| +20003|AAAAAAAAECOEAAAA|20003|5|33|23|AM|third|night|| +20004|AAAAAAAAFCOEAAAA|20004|5|33|24|AM|third|night|| +20005|AAAAAAAAGCOEAAAA|20005|5|33|25|AM|third|night|| +20006|AAAAAAAAHCOEAAAA|20006|5|33|26|AM|third|night|| +20007|AAAAAAAAICOEAAAA|20007|5|33|27|AM|third|night|| +20008|AAAAAAAAJCOEAAAA|20008|5|33|28|AM|third|night|| +20009|AAAAAAAAKCOEAAAA|20009|5|33|29|AM|third|night|| +20010|AAAAAAAALCOEAAAA|20010|5|33|30|AM|third|night|| +20011|AAAAAAAAMCOEAAAA|20011|5|33|31|AM|third|night|| +20012|AAAAAAAANCOEAAAA|20012|5|33|32|AM|third|night|| +20013|AAAAAAAAOCOEAAAA|20013|5|33|33|AM|third|night|| +20014|AAAAAAAAPCOEAAAA|20014|5|33|34|AM|third|night|| +20015|AAAAAAAAADOEAAAA|20015|5|33|35|AM|third|night|| +20016|AAAAAAAABDOEAAAA|20016|5|33|36|AM|third|night|| +20017|AAAAAAAACDOEAAAA|20017|5|33|37|AM|third|night|| +20018|AAAAAAAADDOEAAAA|20018|5|33|38|AM|third|night|| +20019|AAAAAAAAEDOEAAAA|20019|5|33|39|AM|third|night|| +20020|AAAAAAAAFDOEAAAA|20020|5|33|40|AM|third|night|| +20021|AAAAAAAAGDOEAAAA|20021|5|33|41|AM|third|night|| +20022|AAAAAAAAHDOEAAAA|20022|5|33|42|AM|third|night|| +20023|AAAAAAAAIDOEAAAA|20023|5|33|43|AM|third|night|| +20024|AAAAAAAAJDOEAAAA|20024|5|33|44|AM|third|night|| +20025|AAAAAAAAKDOEAAAA|20025|5|33|45|AM|third|night|| +20026|AAAAAAAALDOEAAAA|20026|5|33|46|AM|third|night|| +20027|AAAAAAAAMDOEAAAA|20027|5|33|47|AM|third|night|| +20028|AAAAAAAANDOEAAAA|20028|5|33|48|AM|third|night|| +20029|AAAAAAAAODOEAAAA|20029|5|33|49|AM|third|night|| +20030|AAAAAAAAPDOEAAAA|20030|5|33|50|AM|third|night|| +20031|AAAAAAAAAEOEAAAA|20031|5|33|51|AM|third|night|| +20032|AAAAAAAABEOEAAAA|20032|5|33|52|AM|third|night|| +20033|AAAAAAAACEOEAAAA|20033|5|33|53|AM|third|night|| +20034|AAAAAAAADEOEAAAA|20034|5|33|54|AM|third|night|| +20035|AAAAAAAAEEOEAAAA|20035|5|33|55|AM|third|night|| +20036|AAAAAAAAFEOEAAAA|20036|5|33|56|AM|third|night|| +20037|AAAAAAAAGEOEAAAA|20037|5|33|57|AM|third|night|| +20038|AAAAAAAAHEOEAAAA|20038|5|33|58|AM|third|night|| +20039|AAAAAAAAIEOEAAAA|20039|5|33|59|AM|third|night|| +20040|AAAAAAAAJEOEAAAA|20040|5|34|0|AM|third|night|| +20041|AAAAAAAAKEOEAAAA|20041|5|34|1|AM|third|night|| +20042|AAAAAAAALEOEAAAA|20042|5|34|2|AM|third|night|| +20043|AAAAAAAAMEOEAAAA|20043|5|34|3|AM|third|night|| +20044|AAAAAAAANEOEAAAA|20044|5|34|4|AM|third|night|| +20045|AAAAAAAAOEOEAAAA|20045|5|34|5|AM|third|night|| +20046|AAAAAAAAPEOEAAAA|20046|5|34|6|AM|third|night|| +20047|AAAAAAAAAFOEAAAA|20047|5|34|7|AM|third|night|| +20048|AAAAAAAABFOEAAAA|20048|5|34|8|AM|third|night|| +20049|AAAAAAAACFOEAAAA|20049|5|34|9|AM|third|night|| +20050|AAAAAAAADFOEAAAA|20050|5|34|10|AM|third|night|| +20051|AAAAAAAAEFOEAAAA|20051|5|34|11|AM|third|night|| +20052|AAAAAAAAFFOEAAAA|20052|5|34|12|AM|third|night|| +20053|AAAAAAAAGFOEAAAA|20053|5|34|13|AM|third|night|| +20054|AAAAAAAAHFOEAAAA|20054|5|34|14|AM|third|night|| +20055|AAAAAAAAIFOEAAAA|20055|5|34|15|AM|third|night|| +20056|AAAAAAAAJFOEAAAA|20056|5|34|16|AM|third|night|| +20057|AAAAAAAAKFOEAAAA|20057|5|34|17|AM|third|night|| +20058|AAAAAAAALFOEAAAA|20058|5|34|18|AM|third|night|| +20059|AAAAAAAAMFOEAAAA|20059|5|34|19|AM|third|night|| +20060|AAAAAAAANFOEAAAA|20060|5|34|20|AM|third|night|| +20061|AAAAAAAAOFOEAAAA|20061|5|34|21|AM|third|night|| +20062|AAAAAAAAPFOEAAAA|20062|5|34|22|AM|third|night|| +20063|AAAAAAAAAGOEAAAA|20063|5|34|23|AM|third|night|| +20064|AAAAAAAABGOEAAAA|20064|5|34|24|AM|third|night|| +20065|AAAAAAAACGOEAAAA|20065|5|34|25|AM|third|night|| +20066|AAAAAAAADGOEAAAA|20066|5|34|26|AM|third|night|| +20067|AAAAAAAAEGOEAAAA|20067|5|34|27|AM|third|night|| +20068|AAAAAAAAFGOEAAAA|20068|5|34|28|AM|third|night|| +20069|AAAAAAAAGGOEAAAA|20069|5|34|29|AM|third|night|| +20070|AAAAAAAAHGOEAAAA|20070|5|34|30|AM|third|night|| +20071|AAAAAAAAIGOEAAAA|20071|5|34|31|AM|third|night|| +20072|AAAAAAAAJGOEAAAA|20072|5|34|32|AM|third|night|| +20073|AAAAAAAAKGOEAAAA|20073|5|34|33|AM|third|night|| +20074|AAAAAAAALGOEAAAA|20074|5|34|34|AM|third|night|| +20075|AAAAAAAAMGOEAAAA|20075|5|34|35|AM|third|night|| +20076|AAAAAAAANGOEAAAA|20076|5|34|36|AM|third|night|| +20077|AAAAAAAAOGOEAAAA|20077|5|34|37|AM|third|night|| +20078|AAAAAAAAPGOEAAAA|20078|5|34|38|AM|third|night|| +20079|AAAAAAAAAHOEAAAA|20079|5|34|39|AM|third|night|| +20080|AAAAAAAABHOEAAAA|20080|5|34|40|AM|third|night|| +20081|AAAAAAAACHOEAAAA|20081|5|34|41|AM|third|night|| +20082|AAAAAAAADHOEAAAA|20082|5|34|42|AM|third|night|| +20083|AAAAAAAAEHOEAAAA|20083|5|34|43|AM|third|night|| +20084|AAAAAAAAFHOEAAAA|20084|5|34|44|AM|third|night|| +20085|AAAAAAAAGHOEAAAA|20085|5|34|45|AM|third|night|| +20086|AAAAAAAAHHOEAAAA|20086|5|34|46|AM|third|night|| +20087|AAAAAAAAIHOEAAAA|20087|5|34|47|AM|third|night|| +20088|AAAAAAAAJHOEAAAA|20088|5|34|48|AM|third|night|| +20089|AAAAAAAAKHOEAAAA|20089|5|34|49|AM|third|night|| +20090|AAAAAAAALHOEAAAA|20090|5|34|50|AM|third|night|| +20091|AAAAAAAAMHOEAAAA|20091|5|34|51|AM|third|night|| +20092|AAAAAAAANHOEAAAA|20092|5|34|52|AM|third|night|| +20093|AAAAAAAAOHOEAAAA|20093|5|34|53|AM|third|night|| +20094|AAAAAAAAPHOEAAAA|20094|5|34|54|AM|third|night|| +20095|AAAAAAAAAIOEAAAA|20095|5|34|55|AM|third|night|| +20096|AAAAAAAABIOEAAAA|20096|5|34|56|AM|third|night|| +20097|AAAAAAAACIOEAAAA|20097|5|34|57|AM|third|night|| +20098|AAAAAAAADIOEAAAA|20098|5|34|58|AM|third|night|| +20099|AAAAAAAAEIOEAAAA|20099|5|34|59|AM|third|night|| +20100|AAAAAAAAFIOEAAAA|20100|5|35|0|AM|third|night|| +20101|AAAAAAAAGIOEAAAA|20101|5|35|1|AM|third|night|| +20102|AAAAAAAAHIOEAAAA|20102|5|35|2|AM|third|night|| +20103|AAAAAAAAIIOEAAAA|20103|5|35|3|AM|third|night|| +20104|AAAAAAAAJIOEAAAA|20104|5|35|4|AM|third|night|| +20105|AAAAAAAAKIOEAAAA|20105|5|35|5|AM|third|night|| +20106|AAAAAAAALIOEAAAA|20106|5|35|6|AM|third|night|| +20107|AAAAAAAAMIOEAAAA|20107|5|35|7|AM|third|night|| +20108|AAAAAAAANIOEAAAA|20108|5|35|8|AM|third|night|| +20109|AAAAAAAAOIOEAAAA|20109|5|35|9|AM|third|night|| +20110|AAAAAAAAPIOEAAAA|20110|5|35|10|AM|third|night|| +20111|AAAAAAAAAJOEAAAA|20111|5|35|11|AM|third|night|| +20112|AAAAAAAABJOEAAAA|20112|5|35|12|AM|third|night|| +20113|AAAAAAAACJOEAAAA|20113|5|35|13|AM|third|night|| +20114|AAAAAAAADJOEAAAA|20114|5|35|14|AM|third|night|| +20115|AAAAAAAAEJOEAAAA|20115|5|35|15|AM|third|night|| +20116|AAAAAAAAFJOEAAAA|20116|5|35|16|AM|third|night|| +20117|AAAAAAAAGJOEAAAA|20117|5|35|17|AM|third|night|| +20118|AAAAAAAAHJOEAAAA|20118|5|35|18|AM|third|night|| +20119|AAAAAAAAIJOEAAAA|20119|5|35|19|AM|third|night|| +20120|AAAAAAAAJJOEAAAA|20120|5|35|20|AM|third|night|| +20121|AAAAAAAAKJOEAAAA|20121|5|35|21|AM|third|night|| +20122|AAAAAAAALJOEAAAA|20122|5|35|22|AM|third|night|| +20123|AAAAAAAAMJOEAAAA|20123|5|35|23|AM|third|night|| +20124|AAAAAAAANJOEAAAA|20124|5|35|24|AM|third|night|| +20125|AAAAAAAAOJOEAAAA|20125|5|35|25|AM|third|night|| +20126|AAAAAAAAPJOEAAAA|20126|5|35|26|AM|third|night|| +20127|AAAAAAAAAKOEAAAA|20127|5|35|27|AM|third|night|| +20128|AAAAAAAABKOEAAAA|20128|5|35|28|AM|third|night|| +20129|AAAAAAAACKOEAAAA|20129|5|35|29|AM|third|night|| +20130|AAAAAAAADKOEAAAA|20130|5|35|30|AM|third|night|| +20131|AAAAAAAAEKOEAAAA|20131|5|35|31|AM|third|night|| +20132|AAAAAAAAFKOEAAAA|20132|5|35|32|AM|third|night|| +20133|AAAAAAAAGKOEAAAA|20133|5|35|33|AM|third|night|| +20134|AAAAAAAAHKOEAAAA|20134|5|35|34|AM|third|night|| +20135|AAAAAAAAIKOEAAAA|20135|5|35|35|AM|third|night|| +20136|AAAAAAAAJKOEAAAA|20136|5|35|36|AM|third|night|| +20137|AAAAAAAAKKOEAAAA|20137|5|35|37|AM|third|night|| +20138|AAAAAAAALKOEAAAA|20138|5|35|38|AM|third|night|| +20139|AAAAAAAAMKOEAAAA|20139|5|35|39|AM|third|night|| +20140|AAAAAAAANKOEAAAA|20140|5|35|40|AM|third|night|| +20141|AAAAAAAAOKOEAAAA|20141|5|35|41|AM|third|night|| +20142|AAAAAAAAPKOEAAAA|20142|5|35|42|AM|third|night|| +20143|AAAAAAAAALOEAAAA|20143|5|35|43|AM|third|night|| +20144|AAAAAAAABLOEAAAA|20144|5|35|44|AM|third|night|| +20145|AAAAAAAACLOEAAAA|20145|5|35|45|AM|third|night|| +20146|AAAAAAAADLOEAAAA|20146|5|35|46|AM|third|night|| +20147|AAAAAAAAELOEAAAA|20147|5|35|47|AM|third|night|| +20148|AAAAAAAAFLOEAAAA|20148|5|35|48|AM|third|night|| +20149|AAAAAAAAGLOEAAAA|20149|5|35|49|AM|third|night|| +20150|AAAAAAAAHLOEAAAA|20150|5|35|50|AM|third|night|| +20151|AAAAAAAAILOEAAAA|20151|5|35|51|AM|third|night|| +20152|AAAAAAAAJLOEAAAA|20152|5|35|52|AM|third|night|| +20153|AAAAAAAAKLOEAAAA|20153|5|35|53|AM|third|night|| +20154|AAAAAAAALLOEAAAA|20154|5|35|54|AM|third|night|| +20155|AAAAAAAAMLOEAAAA|20155|5|35|55|AM|third|night|| +20156|AAAAAAAANLOEAAAA|20156|5|35|56|AM|third|night|| +20157|AAAAAAAAOLOEAAAA|20157|5|35|57|AM|third|night|| +20158|AAAAAAAAPLOEAAAA|20158|5|35|58|AM|third|night|| +20159|AAAAAAAAAMOEAAAA|20159|5|35|59|AM|third|night|| +20160|AAAAAAAABMOEAAAA|20160|5|36|0|AM|third|night|| +20161|AAAAAAAACMOEAAAA|20161|5|36|1|AM|third|night|| +20162|AAAAAAAADMOEAAAA|20162|5|36|2|AM|third|night|| +20163|AAAAAAAAEMOEAAAA|20163|5|36|3|AM|third|night|| +20164|AAAAAAAAFMOEAAAA|20164|5|36|4|AM|third|night|| +20165|AAAAAAAAGMOEAAAA|20165|5|36|5|AM|third|night|| +20166|AAAAAAAAHMOEAAAA|20166|5|36|6|AM|third|night|| +20167|AAAAAAAAIMOEAAAA|20167|5|36|7|AM|third|night|| +20168|AAAAAAAAJMOEAAAA|20168|5|36|8|AM|third|night|| +20169|AAAAAAAAKMOEAAAA|20169|5|36|9|AM|third|night|| +20170|AAAAAAAALMOEAAAA|20170|5|36|10|AM|third|night|| +20171|AAAAAAAAMMOEAAAA|20171|5|36|11|AM|third|night|| +20172|AAAAAAAANMOEAAAA|20172|5|36|12|AM|third|night|| +20173|AAAAAAAAOMOEAAAA|20173|5|36|13|AM|third|night|| +20174|AAAAAAAAPMOEAAAA|20174|5|36|14|AM|third|night|| +20175|AAAAAAAAANOEAAAA|20175|5|36|15|AM|third|night|| +20176|AAAAAAAABNOEAAAA|20176|5|36|16|AM|third|night|| +20177|AAAAAAAACNOEAAAA|20177|5|36|17|AM|third|night|| +20178|AAAAAAAADNOEAAAA|20178|5|36|18|AM|third|night|| +20179|AAAAAAAAENOEAAAA|20179|5|36|19|AM|third|night|| +20180|AAAAAAAAFNOEAAAA|20180|5|36|20|AM|third|night|| +20181|AAAAAAAAGNOEAAAA|20181|5|36|21|AM|third|night|| +20182|AAAAAAAAHNOEAAAA|20182|5|36|22|AM|third|night|| +20183|AAAAAAAAINOEAAAA|20183|5|36|23|AM|third|night|| +20184|AAAAAAAAJNOEAAAA|20184|5|36|24|AM|third|night|| +20185|AAAAAAAAKNOEAAAA|20185|5|36|25|AM|third|night|| +20186|AAAAAAAALNOEAAAA|20186|5|36|26|AM|third|night|| +20187|AAAAAAAAMNOEAAAA|20187|5|36|27|AM|third|night|| +20188|AAAAAAAANNOEAAAA|20188|5|36|28|AM|third|night|| +20189|AAAAAAAAONOEAAAA|20189|5|36|29|AM|third|night|| +20190|AAAAAAAAPNOEAAAA|20190|5|36|30|AM|third|night|| +20191|AAAAAAAAAOOEAAAA|20191|5|36|31|AM|third|night|| +20192|AAAAAAAABOOEAAAA|20192|5|36|32|AM|third|night|| +20193|AAAAAAAACOOEAAAA|20193|5|36|33|AM|third|night|| +20194|AAAAAAAADOOEAAAA|20194|5|36|34|AM|third|night|| +20195|AAAAAAAAEOOEAAAA|20195|5|36|35|AM|third|night|| +20196|AAAAAAAAFOOEAAAA|20196|5|36|36|AM|third|night|| +20197|AAAAAAAAGOOEAAAA|20197|5|36|37|AM|third|night|| +20198|AAAAAAAAHOOEAAAA|20198|5|36|38|AM|third|night|| +20199|AAAAAAAAIOOEAAAA|20199|5|36|39|AM|third|night|| +20200|AAAAAAAAJOOEAAAA|20200|5|36|40|AM|third|night|| +20201|AAAAAAAAKOOEAAAA|20201|5|36|41|AM|third|night|| +20202|AAAAAAAALOOEAAAA|20202|5|36|42|AM|third|night|| +20203|AAAAAAAAMOOEAAAA|20203|5|36|43|AM|third|night|| +20204|AAAAAAAANOOEAAAA|20204|5|36|44|AM|third|night|| +20205|AAAAAAAAOOOEAAAA|20205|5|36|45|AM|third|night|| +20206|AAAAAAAAPOOEAAAA|20206|5|36|46|AM|third|night|| +20207|AAAAAAAAAPOEAAAA|20207|5|36|47|AM|third|night|| +20208|AAAAAAAABPOEAAAA|20208|5|36|48|AM|third|night|| +20209|AAAAAAAACPOEAAAA|20209|5|36|49|AM|third|night|| +20210|AAAAAAAADPOEAAAA|20210|5|36|50|AM|third|night|| +20211|AAAAAAAAEPOEAAAA|20211|5|36|51|AM|third|night|| +20212|AAAAAAAAFPOEAAAA|20212|5|36|52|AM|third|night|| +20213|AAAAAAAAGPOEAAAA|20213|5|36|53|AM|third|night|| +20214|AAAAAAAAHPOEAAAA|20214|5|36|54|AM|third|night|| +20215|AAAAAAAAIPOEAAAA|20215|5|36|55|AM|third|night|| +20216|AAAAAAAAJPOEAAAA|20216|5|36|56|AM|third|night|| +20217|AAAAAAAAKPOEAAAA|20217|5|36|57|AM|third|night|| +20218|AAAAAAAALPOEAAAA|20218|5|36|58|AM|third|night|| +20219|AAAAAAAAMPOEAAAA|20219|5|36|59|AM|third|night|| +20220|AAAAAAAANPOEAAAA|20220|5|37|0|AM|third|night|| +20221|AAAAAAAAOPOEAAAA|20221|5|37|1|AM|third|night|| +20222|AAAAAAAAPPOEAAAA|20222|5|37|2|AM|third|night|| +20223|AAAAAAAAAAPEAAAA|20223|5|37|3|AM|third|night|| +20224|AAAAAAAABAPEAAAA|20224|5|37|4|AM|third|night|| +20225|AAAAAAAACAPEAAAA|20225|5|37|5|AM|third|night|| +20226|AAAAAAAADAPEAAAA|20226|5|37|6|AM|third|night|| +20227|AAAAAAAAEAPEAAAA|20227|5|37|7|AM|third|night|| +20228|AAAAAAAAFAPEAAAA|20228|5|37|8|AM|third|night|| +20229|AAAAAAAAGAPEAAAA|20229|5|37|9|AM|third|night|| +20230|AAAAAAAAHAPEAAAA|20230|5|37|10|AM|third|night|| +20231|AAAAAAAAIAPEAAAA|20231|5|37|11|AM|third|night|| +20232|AAAAAAAAJAPEAAAA|20232|5|37|12|AM|third|night|| +20233|AAAAAAAAKAPEAAAA|20233|5|37|13|AM|third|night|| +20234|AAAAAAAALAPEAAAA|20234|5|37|14|AM|third|night|| +20235|AAAAAAAAMAPEAAAA|20235|5|37|15|AM|third|night|| +20236|AAAAAAAANAPEAAAA|20236|5|37|16|AM|third|night|| +20237|AAAAAAAAOAPEAAAA|20237|5|37|17|AM|third|night|| +20238|AAAAAAAAPAPEAAAA|20238|5|37|18|AM|third|night|| +20239|AAAAAAAAABPEAAAA|20239|5|37|19|AM|third|night|| +20240|AAAAAAAABBPEAAAA|20240|5|37|20|AM|third|night|| +20241|AAAAAAAACBPEAAAA|20241|5|37|21|AM|third|night|| +20242|AAAAAAAADBPEAAAA|20242|5|37|22|AM|third|night|| +20243|AAAAAAAAEBPEAAAA|20243|5|37|23|AM|third|night|| +20244|AAAAAAAAFBPEAAAA|20244|5|37|24|AM|third|night|| +20245|AAAAAAAAGBPEAAAA|20245|5|37|25|AM|third|night|| +20246|AAAAAAAAHBPEAAAA|20246|5|37|26|AM|third|night|| +20247|AAAAAAAAIBPEAAAA|20247|5|37|27|AM|third|night|| +20248|AAAAAAAAJBPEAAAA|20248|5|37|28|AM|third|night|| +20249|AAAAAAAAKBPEAAAA|20249|5|37|29|AM|third|night|| +20250|AAAAAAAALBPEAAAA|20250|5|37|30|AM|third|night|| +20251|AAAAAAAAMBPEAAAA|20251|5|37|31|AM|third|night|| +20252|AAAAAAAANBPEAAAA|20252|5|37|32|AM|third|night|| +20253|AAAAAAAAOBPEAAAA|20253|5|37|33|AM|third|night|| +20254|AAAAAAAAPBPEAAAA|20254|5|37|34|AM|third|night|| +20255|AAAAAAAAACPEAAAA|20255|5|37|35|AM|third|night|| +20256|AAAAAAAABCPEAAAA|20256|5|37|36|AM|third|night|| +20257|AAAAAAAACCPEAAAA|20257|5|37|37|AM|third|night|| +20258|AAAAAAAADCPEAAAA|20258|5|37|38|AM|third|night|| +20259|AAAAAAAAECPEAAAA|20259|5|37|39|AM|third|night|| +20260|AAAAAAAAFCPEAAAA|20260|5|37|40|AM|third|night|| +20261|AAAAAAAAGCPEAAAA|20261|5|37|41|AM|third|night|| +20262|AAAAAAAAHCPEAAAA|20262|5|37|42|AM|third|night|| +20263|AAAAAAAAICPEAAAA|20263|5|37|43|AM|third|night|| +20264|AAAAAAAAJCPEAAAA|20264|5|37|44|AM|third|night|| +20265|AAAAAAAAKCPEAAAA|20265|5|37|45|AM|third|night|| +20266|AAAAAAAALCPEAAAA|20266|5|37|46|AM|third|night|| +20267|AAAAAAAAMCPEAAAA|20267|5|37|47|AM|third|night|| +20268|AAAAAAAANCPEAAAA|20268|5|37|48|AM|third|night|| +20269|AAAAAAAAOCPEAAAA|20269|5|37|49|AM|third|night|| +20270|AAAAAAAAPCPEAAAA|20270|5|37|50|AM|third|night|| +20271|AAAAAAAAADPEAAAA|20271|5|37|51|AM|third|night|| +20272|AAAAAAAABDPEAAAA|20272|5|37|52|AM|third|night|| +20273|AAAAAAAACDPEAAAA|20273|5|37|53|AM|third|night|| +20274|AAAAAAAADDPEAAAA|20274|5|37|54|AM|third|night|| +20275|AAAAAAAAEDPEAAAA|20275|5|37|55|AM|third|night|| +20276|AAAAAAAAFDPEAAAA|20276|5|37|56|AM|third|night|| +20277|AAAAAAAAGDPEAAAA|20277|5|37|57|AM|third|night|| +20278|AAAAAAAAHDPEAAAA|20278|5|37|58|AM|third|night|| +20279|AAAAAAAAIDPEAAAA|20279|5|37|59|AM|third|night|| +20280|AAAAAAAAJDPEAAAA|20280|5|38|0|AM|third|night|| +20281|AAAAAAAAKDPEAAAA|20281|5|38|1|AM|third|night|| +20282|AAAAAAAALDPEAAAA|20282|5|38|2|AM|third|night|| +20283|AAAAAAAAMDPEAAAA|20283|5|38|3|AM|third|night|| +20284|AAAAAAAANDPEAAAA|20284|5|38|4|AM|third|night|| +20285|AAAAAAAAODPEAAAA|20285|5|38|5|AM|third|night|| +20286|AAAAAAAAPDPEAAAA|20286|5|38|6|AM|third|night|| +20287|AAAAAAAAAEPEAAAA|20287|5|38|7|AM|third|night|| +20288|AAAAAAAABEPEAAAA|20288|5|38|8|AM|third|night|| +20289|AAAAAAAACEPEAAAA|20289|5|38|9|AM|third|night|| +20290|AAAAAAAADEPEAAAA|20290|5|38|10|AM|third|night|| +20291|AAAAAAAAEEPEAAAA|20291|5|38|11|AM|third|night|| +20292|AAAAAAAAFEPEAAAA|20292|5|38|12|AM|third|night|| +20293|AAAAAAAAGEPEAAAA|20293|5|38|13|AM|third|night|| +20294|AAAAAAAAHEPEAAAA|20294|5|38|14|AM|third|night|| +20295|AAAAAAAAIEPEAAAA|20295|5|38|15|AM|third|night|| +20296|AAAAAAAAJEPEAAAA|20296|5|38|16|AM|third|night|| +20297|AAAAAAAAKEPEAAAA|20297|5|38|17|AM|third|night|| +20298|AAAAAAAALEPEAAAA|20298|5|38|18|AM|third|night|| +20299|AAAAAAAAMEPEAAAA|20299|5|38|19|AM|third|night|| +20300|AAAAAAAANEPEAAAA|20300|5|38|20|AM|third|night|| +20301|AAAAAAAAOEPEAAAA|20301|5|38|21|AM|third|night|| +20302|AAAAAAAAPEPEAAAA|20302|5|38|22|AM|third|night|| +20303|AAAAAAAAAFPEAAAA|20303|5|38|23|AM|third|night|| +20304|AAAAAAAABFPEAAAA|20304|5|38|24|AM|third|night|| +20305|AAAAAAAACFPEAAAA|20305|5|38|25|AM|third|night|| +20306|AAAAAAAADFPEAAAA|20306|5|38|26|AM|third|night|| +20307|AAAAAAAAEFPEAAAA|20307|5|38|27|AM|third|night|| +20308|AAAAAAAAFFPEAAAA|20308|5|38|28|AM|third|night|| +20309|AAAAAAAAGFPEAAAA|20309|5|38|29|AM|third|night|| +20310|AAAAAAAAHFPEAAAA|20310|5|38|30|AM|third|night|| +20311|AAAAAAAAIFPEAAAA|20311|5|38|31|AM|third|night|| +20312|AAAAAAAAJFPEAAAA|20312|5|38|32|AM|third|night|| +20313|AAAAAAAAKFPEAAAA|20313|5|38|33|AM|third|night|| +20314|AAAAAAAALFPEAAAA|20314|5|38|34|AM|third|night|| +20315|AAAAAAAAMFPEAAAA|20315|5|38|35|AM|third|night|| +20316|AAAAAAAANFPEAAAA|20316|5|38|36|AM|third|night|| +20317|AAAAAAAAOFPEAAAA|20317|5|38|37|AM|third|night|| +20318|AAAAAAAAPFPEAAAA|20318|5|38|38|AM|third|night|| +20319|AAAAAAAAAGPEAAAA|20319|5|38|39|AM|third|night|| +20320|AAAAAAAABGPEAAAA|20320|5|38|40|AM|third|night|| +20321|AAAAAAAACGPEAAAA|20321|5|38|41|AM|third|night|| +20322|AAAAAAAADGPEAAAA|20322|5|38|42|AM|third|night|| +20323|AAAAAAAAEGPEAAAA|20323|5|38|43|AM|third|night|| +20324|AAAAAAAAFGPEAAAA|20324|5|38|44|AM|third|night|| +20325|AAAAAAAAGGPEAAAA|20325|5|38|45|AM|third|night|| +20326|AAAAAAAAHGPEAAAA|20326|5|38|46|AM|third|night|| +20327|AAAAAAAAIGPEAAAA|20327|5|38|47|AM|third|night|| +20328|AAAAAAAAJGPEAAAA|20328|5|38|48|AM|third|night|| +20329|AAAAAAAAKGPEAAAA|20329|5|38|49|AM|third|night|| +20330|AAAAAAAALGPEAAAA|20330|5|38|50|AM|third|night|| +20331|AAAAAAAAMGPEAAAA|20331|5|38|51|AM|third|night|| +20332|AAAAAAAANGPEAAAA|20332|5|38|52|AM|third|night|| +20333|AAAAAAAAOGPEAAAA|20333|5|38|53|AM|third|night|| +20334|AAAAAAAAPGPEAAAA|20334|5|38|54|AM|third|night|| +20335|AAAAAAAAAHPEAAAA|20335|5|38|55|AM|third|night|| +20336|AAAAAAAABHPEAAAA|20336|5|38|56|AM|third|night|| +20337|AAAAAAAACHPEAAAA|20337|5|38|57|AM|third|night|| +20338|AAAAAAAADHPEAAAA|20338|5|38|58|AM|third|night|| +20339|AAAAAAAAEHPEAAAA|20339|5|38|59|AM|third|night|| +20340|AAAAAAAAFHPEAAAA|20340|5|39|0|AM|third|night|| +20341|AAAAAAAAGHPEAAAA|20341|5|39|1|AM|third|night|| +20342|AAAAAAAAHHPEAAAA|20342|5|39|2|AM|third|night|| +20343|AAAAAAAAIHPEAAAA|20343|5|39|3|AM|third|night|| +20344|AAAAAAAAJHPEAAAA|20344|5|39|4|AM|third|night|| +20345|AAAAAAAAKHPEAAAA|20345|5|39|5|AM|third|night|| +20346|AAAAAAAALHPEAAAA|20346|5|39|6|AM|third|night|| +20347|AAAAAAAAMHPEAAAA|20347|5|39|7|AM|third|night|| +20348|AAAAAAAANHPEAAAA|20348|5|39|8|AM|third|night|| +20349|AAAAAAAAOHPEAAAA|20349|5|39|9|AM|third|night|| +20350|AAAAAAAAPHPEAAAA|20350|5|39|10|AM|third|night|| +20351|AAAAAAAAAIPEAAAA|20351|5|39|11|AM|third|night|| +20352|AAAAAAAABIPEAAAA|20352|5|39|12|AM|third|night|| +20353|AAAAAAAACIPEAAAA|20353|5|39|13|AM|third|night|| +20354|AAAAAAAADIPEAAAA|20354|5|39|14|AM|third|night|| +20355|AAAAAAAAEIPEAAAA|20355|5|39|15|AM|third|night|| +20356|AAAAAAAAFIPEAAAA|20356|5|39|16|AM|third|night|| +20357|AAAAAAAAGIPEAAAA|20357|5|39|17|AM|third|night|| +20358|AAAAAAAAHIPEAAAA|20358|5|39|18|AM|third|night|| +20359|AAAAAAAAIIPEAAAA|20359|5|39|19|AM|third|night|| +20360|AAAAAAAAJIPEAAAA|20360|5|39|20|AM|third|night|| +20361|AAAAAAAAKIPEAAAA|20361|5|39|21|AM|third|night|| +20362|AAAAAAAALIPEAAAA|20362|5|39|22|AM|third|night|| +20363|AAAAAAAAMIPEAAAA|20363|5|39|23|AM|third|night|| +20364|AAAAAAAANIPEAAAA|20364|5|39|24|AM|third|night|| +20365|AAAAAAAAOIPEAAAA|20365|5|39|25|AM|third|night|| +20366|AAAAAAAAPIPEAAAA|20366|5|39|26|AM|third|night|| +20367|AAAAAAAAAJPEAAAA|20367|5|39|27|AM|third|night|| +20368|AAAAAAAABJPEAAAA|20368|5|39|28|AM|third|night|| +20369|AAAAAAAACJPEAAAA|20369|5|39|29|AM|third|night|| +20370|AAAAAAAADJPEAAAA|20370|5|39|30|AM|third|night|| +20371|AAAAAAAAEJPEAAAA|20371|5|39|31|AM|third|night|| +20372|AAAAAAAAFJPEAAAA|20372|5|39|32|AM|third|night|| +20373|AAAAAAAAGJPEAAAA|20373|5|39|33|AM|third|night|| +20374|AAAAAAAAHJPEAAAA|20374|5|39|34|AM|third|night|| +20375|AAAAAAAAIJPEAAAA|20375|5|39|35|AM|third|night|| +20376|AAAAAAAAJJPEAAAA|20376|5|39|36|AM|third|night|| +20377|AAAAAAAAKJPEAAAA|20377|5|39|37|AM|third|night|| +20378|AAAAAAAALJPEAAAA|20378|5|39|38|AM|third|night|| +20379|AAAAAAAAMJPEAAAA|20379|5|39|39|AM|third|night|| +20380|AAAAAAAANJPEAAAA|20380|5|39|40|AM|third|night|| +20381|AAAAAAAAOJPEAAAA|20381|5|39|41|AM|third|night|| +20382|AAAAAAAAPJPEAAAA|20382|5|39|42|AM|third|night|| +20383|AAAAAAAAAKPEAAAA|20383|5|39|43|AM|third|night|| +20384|AAAAAAAABKPEAAAA|20384|5|39|44|AM|third|night|| +20385|AAAAAAAACKPEAAAA|20385|5|39|45|AM|third|night|| +20386|AAAAAAAADKPEAAAA|20386|5|39|46|AM|third|night|| +20387|AAAAAAAAEKPEAAAA|20387|5|39|47|AM|third|night|| +20388|AAAAAAAAFKPEAAAA|20388|5|39|48|AM|third|night|| +20389|AAAAAAAAGKPEAAAA|20389|5|39|49|AM|third|night|| +20390|AAAAAAAAHKPEAAAA|20390|5|39|50|AM|third|night|| +20391|AAAAAAAAIKPEAAAA|20391|5|39|51|AM|third|night|| +20392|AAAAAAAAJKPEAAAA|20392|5|39|52|AM|third|night|| +20393|AAAAAAAAKKPEAAAA|20393|5|39|53|AM|third|night|| +20394|AAAAAAAALKPEAAAA|20394|5|39|54|AM|third|night|| +20395|AAAAAAAAMKPEAAAA|20395|5|39|55|AM|third|night|| +20396|AAAAAAAANKPEAAAA|20396|5|39|56|AM|third|night|| +20397|AAAAAAAAOKPEAAAA|20397|5|39|57|AM|third|night|| +20398|AAAAAAAAPKPEAAAA|20398|5|39|58|AM|third|night|| +20399|AAAAAAAAALPEAAAA|20399|5|39|59|AM|third|night|| +20400|AAAAAAAABLPEAAAA|20400|5|40|0|AM|third|night|| +20401|AAAAAAAACLPEAAAA|20401|5|40|1|AM|third|night|| +20402|AAAAAAAADLPEAAAA|20402|5|40|2|AM|third|night|| +20403|AAAAAAAAELPEAAAA|20403|5|40|3|AM|third|night|| +20404|AAAAAAAAFLPEAAAA|20404|5|40|4|AM|third|night|| +20405|AAAAAAAAGLPEAAAA|20405|5|40|5|AM|third|night|| +20406|AAAAAAAAHLPEAAAA|20406|5|40|6|AM|third|night|| +20407|AAAAAAAAILPEAAAA|20407|5|40|7|AM|third|night|| +20408|AAAAAAAAJLPEAAAA|20408|5|40|8|AM|third|night|| +20409|AAAAAAAAKLPEAAAA|20409|5|40|9|AM|third|night|| +20410|AAAAAAAALLPEAAAA|20410|5|40|10|AM|third|night|| +20411|AAAAAAAAMLPEAAAA|20411|5|40|11|AM|third|night|| +20412|AAAAAAAANLPEAAAA|20412|5|40|12|AM|third|night|| +20413|AAAAAAAAOLPEAAAA|20413|5|40|13|AM|third|night|| +20414|AAAAAAAAPLPEAAAA|20414|5|40|14|AM|third|night|| +20415|AAAAAAAAAMPEAAAA|20415|5|40|15|AM|third|night|| +20416|AAAAAAAABMPEAAAA|20416|5|40|16|AM|third|night|| +20417|AAAAAAAACMPEAAAA|20417|5|40|17|AM|third|night|| +20418|AAAAAAAADMPEAAAA|20418|5|40|18|AM|third|night|| +20419|AAAAAAAAEMPEAAAA|20419|5|40|19|AM|third|night|| +20420|AAAAAAAAFMPEAAAA|20420|5|40|20|AM|third|night|| +20421|AAAAAAAAGMPEAAAA|20421|5|40|21|AM|third|night|| +20422|AAAAAAAAHMPEAAAA|20422|5|40|22|AM|third|night|| +20423|AAAAAAAAIMPEAAAA|20423|5|40|23|AM|third|night|| +20424|AAAAAAAAJMPEAAAA|20424|5|40|24|AM|third|night|| +20425|AAAAAAAAKMPEAAAA|20425|5|40|25|AM|third|night|| +20426|AAAAAAAALMPEAAAA|20426|5|40|26|AM|third|night|| +20427|AAAAAAAAMMPEAAAA|20427|5|40|27|AM|third|night|| +20428|AAAAAAAANMPEAAAA|20428|5|40|28|AM|third|night|| +20429|AAAAAAAAOMPEAAAA|20429|5|40|29|AM|third|night|| +20430|AAAAAAAAPMPEAAAA|20430|5|40|30|AM|third|night|| +20431|AAAAAAAAANPEAAAA|20431|5|40|31|AM|third|night|| +20432|AAAAAAAABNPEAAAA|20432|5|40|32|AM|third|night|| +20433|AAAAAAAACNPEAAAA|20433|5|40|33|AM|third|night|| +20434|AAAAAAAADNPEAAAA|20434|5|40|34|AM|third|night|| +20435|AAAAAAAAENPEAAAA|20435|5|40|35|AM|third|night|| +20436|AAAAAAAAFNPEAAAA|20436|5|40|36|AM|third|night|| +20437|AAAAAAAAGNPEAAAA|20437|5|40|37|AM|third|night|| +20438|AAAAAAAAHNPEAAAA|20438|5|40|38|AM|third|night|| +20439|AAAAAAAAINPEAAAA|20439|5|40|39|AM|third|night|| +20440|AAAAAAAAJNPEAAAA|20440|5|40|40|AM|third|night|| +20441|AAAAAAAAKNPEAAAA|20441|5|40|41|AM|third|night|| +20442|AAAAAAAALNPEAAAA|20442|5|40|42|AM|third|night|| +20443|AAAAAAAAMNPEAAAA|20443|5|40|43|AM|third|night|| +20444|AAAAAAAANNPEAAAA|20444|5|40|44|AM|third|night|| +20445|AAAAAAAAONPEAAAA|20445|5|40|45|AM|third|night|| +20446|AAAAAAAAPNPEAAAA|20446|5|40|46|AM|third|night|| +20447|AAAAAAAAAOPEAAAA|20447|5|40|47|AM|third|night|| +20448|AAAAAAAABOPEAAAA|20448|5|40|48|AM|third|night|| +20449|AAAAAAAACOPEAAAA|20449|5|40|49|AM|third|night|| +20450|AAAAAAAADOPEAAAA|20450|5|40|50|AM|third|night|| +20451|AAAAAAAAEOPEAAAA|20451|5|40|51|AM|third|night|| +20452|AAAAAAAAFOPEAAAA|20452|5|40|52|AM|third|night|| +20453|AAAAAAAAGOPEAAAA|20453|5|40|53|AM|third|night|| +20454|AAAAAAAAHOPEAAAA|20454|5|40|54|AM|third|night|| +20455|AAAAAAAAIOPEAAAA|20455|5|40|55|AM|third|night|| +20456|AAAAAAAAJOPEAAAA|20456|5|40|56|AM|third|night|| +20457|AAAAAAAAKOPEAAAA|20457|5|40|57|AM|third|night|| +20458|AAAAAAAALOPEAAAA|20458|5|40|58|AM|third|night|| +20459|AAAAAAAAMOPEAAAA|20459|5|40|59|AM|third|night|| +20460|AAAAAAAANOPEAAAA|20460|5|41|0|AM|third|night|| +20461|AAAAAAAAOOPEAAAA|20461|5|41|1|AM|third|night|| +20462|AAAAAAAAPOPEAAAA|20462|5|41|2|AM|third|night|| +20463|AAAAAAAAAPPEAAAA|20463|5|41|3|AM|third|night|| +20464|AAAAAAAABPPEAAAA|20464|5|41|4|AM|third|night|| +20465|AAAAAAAACPPEAAAA|20465|5|41|5|AM|third|night|| +20466|AAAAAAAADPPEAAAA|20466|5|41|6|AM|third|night|| +20467|AAAAAAAAEPPEAAAA|20467|5|41|7|AM|third|night|| +20468|AAAAAAAAFPPEAAAA|20468|5|41|8|AM|third|night|| +20469|AAAAAAAAGPPEAAAA|20469|5|41|9|AM|third|night|| +20470|AAAAAAAAHPPEAAAA|20470|5|41|10|AM|third|night|| +20471|AAAAAAAAIPPEAAAA|20471|5|41|11|AM|third|night|| +20472|AAAAAAAAJPPEAAAA|20472|5|41|12|AM|third|night|| +20473|AAAAAAAAKPPEAAAA|20473|5|41|13|AM|third|night|| +20474|AAAAAAAALPPEAAAA|20474|5|41|14|AM|third|night|| +20475|AAAAAAAAMPPEAAAA|20475|5|41|15|AM|third|night|| +20476|AAAAAAAANPPEAAAA|20476|5|41|16|AM|third|night|| +20477|AAAAAAAAOPPEAAAA|20477|5|41|17|AM|third|night|| +20478|AAAAAAAAPPPEAAAA|20478|5|41|18|AM|third|night|| +20479|AAAAAAAAAAAFAAAA|20479|5|41|19|AM|third|night|| +20480|AAAAAAAABAAFAAAA|20480|5|41|20|AM|third|night|| +20481|AAAAAAAACAAFAAAA|20481|5|41|21|AM|third|night|| +20482|AAAAAAAADAAFAAAA|20482|5|41|22|AM|third|night|| +20483|AAAAAAAAEAAFAAAA|20483|5|41|23|AM|third|night|| +20484|AAAAAAAAFAAFAAAA|20484|5|41|24|AM|third|night|| +20485|AAAAAAAAGAAFAAAA|20485|5|41|25|AM|third|night|| +20486|AAAAAAAAHAAFAAAA|20486|5|41|26|AM|third|night|| +20487|AAAAAAAAIAAFAAAA|20487|5|41|27|AM|third|night|| +20488|AAAAAAAAJAAFAAAA|20488|5|41|28|AM|third|night|| +20489|AAAAAAAAKAAFAAAA|20489|5|41|29|AM|third|night|| +20490|AAAAAAAALAAFAAAA|20490|5|41|30|AM|third|night|| +20491|AAAAAAAAMAAFAAAA|20491|5|41|31|AM|third|night|| +20492|AAAAAAAANAAFAAAA|20492|5|41|32|AM|third|night|| +20493|AAAAAAAAOAAFAAAA|20493|5|41|33|AM|third|night|| +20494|AAAAAAAAPAAFAAAA|20494|5|41|34|AM|third|night|| +20495|AAAAAAAAABAFAAAA|20495|5|41|35|AM|third|night|| +20496|AAAAAAAABBAFAAAA|20496|5|41|36|AM|third|night|| +20497|AAAAAAAACBAFAAAA|20497|5|41|37|AM|third|night|| +20498|AAAAAAAADBAFAAAA|20498|5|41|38|AM|third|night|| +20499|AAAAAAAAEBAFAAAA|20499|5|41|39|AM|third|night|| +20500|AAAAAAAAFBAFAAAA|20500|5|41|40|AM|third|night|| +20501|AAAAAAAAGBAFAAAA|20501|5|41|41|AM|third|night|| +20502|AAAAAAAAHBAFAAAA|20502|5|41|42|AM|third|night|| +20503|AAAAAAAAIBAFAAAA|20503|5|41|43|AM|third|night|| +20504|AAAAAAAAJBAFAAAA|20504|5|41|44|AM|third|night|| +20505|AAAAAAAAKBAFAAAA|20505|5|41|45|AM|third|night|| +20506|AAAAAAAALBAFAAAA|20506|5|41|46|AM|third|night|| +20507|AAAAAAAAMBAFAAAA|20507|5|41|47|AM|third|night|| +20508|AAAAAAAANBAFAAAA|20508|5|41|48|AM|third|night|| +20509|AAAAAAAAOBAFAAAA|20509|5|41|49|AM|third|night|| +20510|AAAAAAAAPBAFAAAA|20510|5|41|50|AM|third|night|| +20511|AAAAAAAAACAFAAAA|20511|5|41|51|AM|third|night|| +20512|AAAAAAAABCAFAAAA|20512|5|41|52|AM|third|night|| +20513|AAAAAAAACCAFAAAA|20513|5|41|53|AM|third|night|| +20514|AAAAAAAADCAFAAAA|20514|5|41|54|AM|third|night|| +20515|AAAAAAAAECAFAAAA|20515|5|41|55|AM|third|night|| +20516|AAAAAAAAFCAFAAAA|20516|5|41|56|AM|third|night|| +20517|AAAAAAAAGCAFAAAA|20517|5|41|57|AM|third|night|| +20518|AAAAAAAAHCAFAAAA|20518|5|41|58|AM|third|night|| +20519|AAAAAAAAICAFAAAA|20519|5|41|59|AM|third|night|| +20520|AAAAAAAAJCAFAAAA|20520|5|42|0|AM|third|night|| +20521|AAAAAAAAKCAFAAAA|20521|5|42|1|AM|third|night|| +20522|AAAAAAAALCAFAAAA|20522|5|42|2|AM|third|night|| +20523|AAAAAAAAMCAFAAAA|20523|5|42|3|AM|third|night|| +20524|AAAAAAAANCAFAAAA|20524|5|42|4|AM|third|night|| +20525|AAAAAAAAOCAFAAAA|20525|5|42|5|AM|third|night|| +20526|AAAAAAAAPCAFAAAA|20526|5|42|6|AM|third|night|| +20527|AAAAAAAAADAFAAAA|20527|5|42|7|AM|third|night|| +20528|AAAAAAAABDAFAAAA|20528|5|42|8|AM|third|night|| +20529|AAAAAAAACDAFAAAA|20529|5|42|9|AM|third|night|| +20530|AAAAAAAADDAFAAAA|20530|5|42|10|AM|third|night|| +20531|AAAAAAAAEDAFAAAA|20531|5|42|11|AM|third|night|| +20532|AAAAAAAAFDAFAAAA|20532|5|42|12|AM|third|night|| +20533|AAAAAAAAGDAFAAAA|20533|5|42|13|AM|third|night|| +20534|AAAAAAAAHDAFAAAA|20534|5|42|14|AM|third|night|| +20535|AAAAAAAAIDAFAAAA|20535|5|42|15|AM|third|night|| +20536|AAAAAAAAJDAFAAAA|20536|5|42|16|AM|third|night|| +20537|AAAAAAAAKDAFAAAA|20537|5|42|17|AM|third|night|| +20538|AAAAAAAALDAFAAAA|20538|5|42|18|AM|third|night|| +20539|AAAAAAAAMDAFAAAA|20539|5|42|19|AM|third|night|| +20540|AAAAAAAANDAFAAAA|20540|5|42|20|AM|third|night|| +20541|AAAAAAAAODAFAAAA|20541|5|42|21|AM|third|night|| +20542|AAAAAAAAPDAFAAAA|20542|5|42|22|AM|third|night|| +20543|AAAAAAAAAEAFAAAA|20543|5|42|23|AM|third|night|| +20544|AAAAAAAABEAFAAAA|20544|5|42|24|AM|third|night|| +20545|AAAAAAAACEAFAAAA|20545|5|42|25|AM|third|night|| +20546|AAAAAAAADEAFAAAA|20546|5|42|26|AM|third|night|| +20547|AAAAAAAAEEAFAAAA|20547|5|42|27|AM|third|night|| +20548|AAAAAAAAFEAFAAAA|20548|5|42|28|AM|third|night|| +20549|AAAAAAAAGEAFAAAA|20549|5|42|29|AM|third|night|| +20550|AAAAAAAAHEAFAAAA|20550|5|42|30|AM|third|night|| +20551|AAAAAAAAIEAFAAAA|20551|5|42|31|AM|third|night|| +20552|AAAAAAAAJEAFAAAA|20552|5|42|32|AM|third|night|| +20553|AAAAAAAAKEAFAAAA|20553|5|42|33|AM|third|night|| +20554|AAAAAAAALEAFAAAA|20554|5|42|34|AM|third|night|| +20555|AAAAAAAAMEAFAAAA|20555|5|42|35|AM|third|night|| +20556|AAAAAAAANEAFAAAA|20556|5|42|36|AM|third|night|| +20557|AAAAAAAAOEAFAAAA|20557|5|42|37|AM|third|night|| +20558|AAAAAAAAPEAFAAAA|20558|5|42|38|AM|third|night|| +20559|AAAAAAAAAFAFAAAA|20559|5|42|39|AM|third|night|| +20560|AAAAAAAABFAFAAAA|20560|5|42|40|AM|third|night|| +20561|AAAAAAAACFAFAAAA|20561|5|42|41|AM|third|night|| +20562|AAAAAAAADFAFAAAA|20562|5|42|42|AM|third|night|| +20563|AAAAAAAAEFAFAAAA|20563|5|42|43|AM|third|night|| +20564|AAAAAAAAFFAFAAAA|20564|5|42|44|AM|third|night|| +20565|AAAAAAAAGFAFAAAA|20565|5|42|45|AM|third|night|| +20566|AAAAAAAAHFAFAAAA|20566|5|42|46|AM|third|night|| +20567|AAAAAAAAIFAFAAAA|20567|5|42|47|AM|third|night|| +20568|AAAAAAAAJFAFAAAA|20568|5|42|48|AM|third|night|| +20569|AAAAAAAAKFAFAAAA|20569|5|42|49|AM|third|night|| +20570|AAAAAAAALFAFAAAA|20570|5|42|50|AM|third|night|| +20571|AAAAAAAAMFAFAAAA|20571|5|42|51|AM|third|night|| +20572|AAAAAAAANFAFAAAA|20572|5|42|52|AM|third|night|| +20573|AAAAAAAAOFAFAAAA|20573|5|42|53|AM|third|night|| +20574|AAAAAAAAPFAFAAAA|20574|5|42|54|AM|third|night|| +20575|AAAAAAAAAGAFAAAA|20575|5|42|55|AM|third|night|| +20576|AAAAAAAABGAFAAAA|20576|5|42|56|AM|third|night|| +20577|AAAAAAAACGAFAAAA|20577|5|42|57|AM|third|night|| +20578|AAAAAAAADGAFAAAA|20578|5|42|58|AM|third|night|| +20579|AAAAAAAAEGAFAAAA|20579|5|42|59|AM|third|night|| +20580|AAAAAAAAFGAFAAAA|20580|5|43|0|AM|third|night|| +20581|AAAAAAAAGGAFAAAA|20581|5|43|1|AM|third|night|| +20582|AAAAAAAAHGAFAAAA|20582|5|43|2|AM|third|night|| +20583|AAAAAAAAIGAFAAAA|20583|5|43|3|AM|third|night|| +20584|AAAAAAAAJGAFAAAA|20584|5|43|4|AM|third|night|| +20585|AAAAAAAAKGAFAAAA|20585|5|43|5|AM|third|night|| +20586|AAAAAAAALGAFAAAA|20586|5|43|6|AM|third|night|| +20587|AAAAAAAAMGAFAAAA|20587|5|43|7|AM|third|night|| +20588|AAAAAAAANGAFAAAA|20588|5|43|8|AM|third|night|| +20589|AAAAAAAAOGAFAAAA|20589|5|43|9|AM|third|night|| +20590|AAAAAAAAPGAFAAAA|20590|5|43|10|AM|third|night|| +20591|AAAAAAAAAHAFAAAA|20591|5|43|11|AM|third|night|| +20592|AAAAAAAABHAFAAAA|20592|5|43|12|AM|third|night|| +20593|AAAAAAAACHAFAAAA|20593|5|43|13|AM|third|night|| +20594|AAAAAAAADHAFAAAA|20594|5|43|14|AM|third|night|| +20595|AAAAAAAAEHAFAAAA|20595|5|43|15|AM|third|night|| +20596|AAAAAAAAFHAFAAAA|20596|5|43|16|AM|third|night|| +20597|AAAAAAAAGHAFAAAA|20597|5|43|17|AM|third|night|| +20598|AAAAAAAAHHAFAAAA|20598|5|43|18|AM|third|night|| +20599|AAAAAAAAIHAFAAAA|20599|5|43|19|AM|third|night|| +20600|AAAAAAAAJHAFAAAA|20600|5|43|20|AM|third|night|| +20601|AAAAAAAAKHAFAAAA|20601|5|43|21|AM|third|night|| +20602|AAAAAAAALHAFAAAA|20602|5|43|22|AM|third|night|| +20603|AAAAAAAAMHAFAAAA|20603|5|43|23|AM|third|night|| +20604|AAAAAAAANHAFAAAA|20604|5|43|24|AM|third|night|| +20605|AAAAAAAAOHAFAAAA|20605|5|43|25|AM|third|night|| +20606|AAAAAAAAPHAFAAAA|20606|5|43|26|AM|third|night|| +20607|AAAAAAAAAIAFAAAA|20607|5|43|27|AM|third|night|| +20608|AAAAAAAABIAFAAAA|20608|5|43|28|AM|third|night|| +20609|AAAAAAAACIAFAAAA|20609|5|43|29|AM|third|night|| +20610|AAAAAAAADIAFAAAA|20610|5|43|30|AM|third|night|| +20611|AAAAAAAAEIAFAAAA|20611|5|43|31|AM|third|night|| +20612|AAAAAAAAFIAFAAAA|20612|5|43|32|AM|third|night|| +20613|AAAAAAAAGIAFAAAA|20613|5|43|33|AM|third|night|| +20614|AAAAAAAAHIAFAAAA|20614|5|43|34|AM|third|night|| +20615|AAAAAAAAIIAFAAAA|20615|5|43|35|AM|third|night|| +20616|AAAAAAAAJIAFAAAA|20616|5|43|36|AM|third|night|| +20617|AAAAAAAAKIAFAAAA|20617|5|43|37|AM|third|night|| +20618|AAAAAAAALIAFAAAA|20618|5|43|38|AM|third|night|| +20619|AAAAAAAAMIAFAAAA|20619|5|43|39|AM|third|night|| +20620|AAAAAAAANIAFAAAA|20620|5|43|40|AM|third|night|| +20621|AAAAAAAAOIAFAAAA|20621|5|43|41|AM|third|night|| +20622|AAAAAAAAPIAFAAAA|20622|5|43|42|AM|third|night|| +20623|AAAAAAAAAJAFAAAA|20623|5|43|43|AM|third|night|| +20624|AAAAAAAABJAFAAAA|20624|5|43|44|AM|third|night|| +20625|AAAAAAAACJAFAAAA|20625|5|43|45|AM|third|night|| +20626|AAAAAAAADJAFAAAA|20626|5|43|46|AM|third|night|| +20627|AAAAAAAAEJAFAAAA|20627|5|43|47|AM|third|night|| +20628|AAAAAAAAFJAFAAAA|20628|5|43|48|AM|third|night|| +20629|AAAAAAAAGJAFAAAA|20629|5|43|49|AM|third|night|| +20630|AAAAAAAAHJAFAAAA|20630|5|43|50|AM|third|night|| +20631|AAAAAAAAIJAFAAAA|20631|5|43|51|AM|third|night|| +20632|AAAAAAAAJJAFAAAA|20632|5|43|52|AM|third|night|| +20633|AAAAAAAAKJAFAAAA|20633|5|43|53|AM|third|night|| +20634|AAAAAAAALJAFAAAA|20634|5|43|54|AM|third|night|| +20635|AAAAAAAAMJAFAAAA|20635|5|43|55|AM|third|night|| +20636|AAAAAAAANJAFAAAA|20636|5|43|56|AM|third|night|| +20637|AAAAAAAAOJAFAAAA|20637|5|43|57|AM|third|night|| +20638|AAAAAAAAPJAFAAAA|20638|5|43|58|AM|third|night|| +20639|AAAAAAAAAKAFAAAA|20639|5|43|59|AM|third|night|| +20640|AAAAAAAABKAFAAAA|20640|5|44|0|AM|third|night|| +20641|AAAAAAAACKAFAAAA|20641|5|44|1|AM|third|night|| +20642|AAAAAAAADKAFAAAA|20642|5|44|2|AM|third|night|| +20643|AAAAAAAAEKAFAAAA|20643|5|44|3|AM|third|night|| +20644|AAAAAAAAFKAFAAAA|20644|5|44|4|AM|third|night|| +20645|AAAAAAAAGKAFAAAA|20645|5|44|5|AM|third|night|| +20646|AAAAAAAAHKAFAAAA|20646|5|44|6|AM|third|night|| +20647|AAAAAAAAIKAFAAAA|20647|5|44|7|AM|third|night|| +20648|AAAAAAAAJKAFAAAA|20648|5|44|8|AM|third|night|| +20649|AAAAAAAAKKAFAAAA|20649|5|44|9|AM|third|night|| +20650|AAAAAAAALKAFAAAA|20650|5|44|10|AM|third|night|| +20651|AAAAAAAAMKAFAAAA|20651|5|44|11|AM|third|night|| +20652|AAAAAAAANKAFAAAA|20652|5|44|12|AM|third|night|| +20653|AAAAAAAAOKAFAAAA|20653|5|44|13|AM|third|night|| +20654|AAAAAAAAPKAFAAAA|20654|5|44|14|AM|third|night|| +20655|AAAAAAAAALAFAAAA|20655|5|44|15|AM|third|night|| +20656|AAAAAAAABLAFAAAA|20656|5|44|16|AM|third|night|| +20657|AAAAAAAACLAFAAAA|20657|5|44|17|AM|third|night|| +20658|AAAAAAAADLAFAAAA|20658|5|44|18|AM|third|night|| +20659|AAAAAAAAELAFAAAA|20659|5|44|19|AM|third|night|| +20660|AAAAAAAAFLAFAAAA|20660|5|44|20|AM|third|night|| +20661|AAAAAAAAGLAFAAAA|20661|5|44|21|AM|third|night|| +20662|AAAAAAAAHLAFAAAA|20662|5|44|22|AM|third|night|| +20663|AAAAAAAAILAFAAAA|20663|5|44|23|AM|third|night|| +20664|AAAAAAAAJLAFAAAA|20664|5|44|24|AM|third|night|| +20665|AAAAAAAAKLAFAAAA|20665|5|44|25|AM|third|night|| +20666|AAAAAAAALLAFAAAA|20666|5|44|26|AM|third|night|| +20667|AAAAAAAAMLAFAAAA|20667|5|44|27|AM|third|night|| +20668|AAAAAAAANLAFAAAA|20668|5|44|28|AM|third|night|| +20669|AAAAAAAAOLAFAAAA|20669|5|44|29|AM|third|night|| +20670|AAAAAAAAPLAFAAAA|20670|5|44|30|AM|third|night|| +20671|AAAAAAAAAMAFAAAA|20671|5|44|31|AM|third|night|| +20672|AAAAAAAABMAFAAAA|20672|5|44|32|AM|third|night|| +20673|AAAAAAAACMAFAAAA|20673|5|44|33|AM|third|night|| +20674|AAAAAAAADMAFAAAA|20674|5|44|34|AM|third|night|| +20675|AAAAAAAAEMAFAAAA|20675|5|44|35|AM|third|night|| +20676|AAAAAAAAFMAFAAAA|20676|5|44|36|AM|third|night|| +20677|AAAAAAAAGMAFAAAA|20677|5|44|37|AM|third|night|| +20678|AAAAAAAAHMAFAAAA|20678|5|44|38|AM|third|night|| +20679|AAAAAAAAIMAFAAAA|20679|5|44|39|AM|third|night|| +20680|AAAAAAAAJMAFAAAA|20680|5|44|40|AM|third|night|| +20681|AAAAAAAAKMAFAAAA|20681|5|44|41|AM|third|night|| +20682|AAAAAAAALMAFAAAA|20682|5|44|42|AM|third|night|| +20683|AAAAAAAAMMAFAAAA|20683|5|44|43|AM|third|night|| +20684|AAAAAAAANMAFAAAA|20684|5|44|44|AM|third|night|| +20685|AAAAAAAAOMAFAAAA|20685|5|44|45|AM|third|night|| +20686|AAAAAAAAPMAFAAAA|20686|5|44|46|AM|third|night|| +20687|AAAAAAAAANAFAAAA|20687|5|44|47|AM|third|night|| +20688|AAAAAAAABNAFAAAA|20688|5|44|48|AM|third|night|| +20689|AAAAAAAACNAFAAAA|20689|5|44|49|AM|third|night|| +20690|AAAAAAAADNAFAAAA|20690|5|44|50|AM|third|night|| +20691|AAAAAAAAENAFAAAA|20691|5|44|51|AM|third|night|| +20692|AAAAAAAAFNAFAAAA|20692|5|44|52|AM|third|night|| +20693|AAAAAAAAGNAFAAAA|20693|5|44|53|AM|third|night|| +20694|AAAAAAAAHNAFAAAA|20694|5|44|54|AM|third|night|| +20695|AAAAAAAAINAFAAAA|20695|5|44|55|AM|third|night|| +20696|AAAAAAAAJNAFAAAA|20696|5|44|56|AM|third|night|| +20697|AAAAAAAAKNAFAAAA|20697|5|44|57|AM|third|night|| +20698|AAAAAAAALNAFAAAA|20698|5|44|58|AM|third|night|| +20699|AAAAAAAAMNAFAAAA|20699|5|44|59|AM|third|night|| +20700|AAAAAAAANNAFAAAA|20700|5|45|0|AM|third|night|| +20701|AAAAAAAAONAFAAAA|20701|5|45|1|AM|third|night|| +20702|AAAAAAAAPNAFAAAA|20702|5|45|2|AM|third|night|| +20703|AAAAAAAAAOAFAAAA|20703|5|45|3|AM|third|night|| +20704|AAAAAAAABOAFAAAA|20704|5|45|4|AM|third|night|| +20705|AAAAAAAACOAFAAAA|20705|5|45|5|AM|third|night|| +20706|AAAAAAAADOAFAAAA|20706|5|45|6|AM|third|night|| +20707|AAAAAAAAEOAFAAAA|20707|5|45|7|AM|third|night|| +20708|AAAAAAAAFOAFAAAA|20708|5|45|8|AM|third|night|| +20709|AAAAAAAAGOAFAAAA|20709|5|45|9|AM|third|night|| +20710|AAAAAAAAHOAFAAAA|20710|5|45|10|AM|third|night|| +20711|AAAAAAAAIOAFAAAA|20711|5|45|11|AM|third|night|| +20712|AAAAAAAAJOAFAAAA|20712|5|45|12|AM|third|night|| +20713|AAAAAAAAKOAFAAAA|20713|5|45|13|AM|third|night|| +20714|AAAAAAAALOAFAAAA|20714|5|45|14|AM|third|night|| +20715|AAAAAAAAMOAFAAAA|20715|5|45|15|AM|third|night|| +20716|AAAAAAAANOAFAAAA|20716|5|45|16|AM|third|night|| +20717|AAAAAAAAOOAFAAAA|20717|5|45|17|AM|third|night|| +20718|AAAAAAAAPOAFAAAA|20718|5|45|18|AM|third|night|| +20719|AAAAAAAAAPAFAAAA|20719|5|45|19|AM|third|night|| +20720|AAAAAAAABPAFAAAA|20720|5|45|20|AM|third|night|| +20721|AAAAAAAACPAFAAAA|20721|5|45|21|AM|third|night|| +20722|AAAAAAAADPAFAAAA|20722|5|45|22|AM|third|night|| +20723|AAAAAAAAEPAFAAAA|20723|5|45|23|AM|third|night|| +20724|AAAAAAAAFPAFAAAA|20724|5|45|24|AM|third|night|| +20725|AAAAAAAAGPAFAAAA|20725|5|45|25|AM|third|night|| +20726|AAAAAAAAHPAFAAAA|20726|5|45|26|AM|third|night|| +20727|AAAAAAAAIPAFAAAA|20727|5|45|27|AM|third|night|| +20728|AAAAAAAAJPAFAAAA|20728|5|45|28|AM|third|night|| +20729|AAAAAAAAKPAFAAAA|20729|5|45|29|AM|third|night|| +20730|AAAAAAAALPAFAAAA|20730|5|45|30|AM|third|night|| +20731|AAAAAAAAMPAFAAAA|20731|5|45|31|AM|third|night|| +20732|AAAAAAAANPAFAAAA|20732|5|45|32|AM|third|night|| +20733|AAAAAAAAOPAFAAAA|20733|5|45|33|AM|third|night|| +20734|AAAAAAAAPPAFAAAA|20734|5|45|34|AM|third|night|| +20735|AAAAAAAAAABFAAAA|20735|5|45|35|AM|third|night|| +20736|AAAAAAAABABFAAAA|20736|5|45|36|AM|third|night|| +20737|AAAAAAAACABFAAAA|20737|5|45|37|AM|third|night|| +20738|AAAAAAAADABFAAAA|20738|5|45|38|AM|third|night|| +20739|AAAAAAAAEABFAAAA|20739|5|45|39|AM|third|night|| +20740|AAAAAAAAFABFAAAA|20740|5|45|40|AM|third|night|| +20741|AAAAAAAAGABFAAAA|20741|5|45|41|AM|third|night|| +20742|AAAAAAAAHABFAAAA|20742|5|45|42|AM|third|night|| +20743|AAAAAAAAIABFAAAA|20743|5|45|43|AM|third|night|| +20744|AAAAAAAAJABFAAAA|20744|5|45|44|AM|third|night|| +20745|AAAAAAAAKABFAAAA|20745|5|45|45|AM|third|night|| +20746|AAAAAAAALABFAAAA|20746|5|45|46|AM|third|night|| +20747|AAAAAAAAMABFAAAA|20747|5|45|47|AM|third|night|| +20748|AAAAAAAANABFAAAA|20748|5|45|48|AM|third|night|| +20749|AAAAAAAAOABFAAAA|20749|5|45|49|AM|third|night|| +20750|AAAAAAAAPABFAAAA|20750|5|45|50|AM|third|night|| +20751|AAAAAAAAABBFAAAA|20751|5|45|51|AM|third|night|| +20752|AAAAAAAABBBFAAAA|20752|5|45|52|AM|third|night|| +20753|AAAAAAAACBBFAAAA|20753|5|45|53|AM|third|night|| +20754|AAAAAAAADBBFAAAA|20754|5|45|54|AM|third|night|| +20755|AAAAAAAAEBBFAAAA|20755|5|45|55|AM|third|night|| +20756|AAAAAAAAFBBFAAAA|20756|5|45|56|AM|third|night|| +20757|AAAAAAAAGBBFAAAA|20757|5|45|57|AM|third|night|| +20758|AAAAAAAAHBBFAAAA|20758|5|45|58|AM|third|night|| +20759|AAAAAAAAIBBFAAAA|20759|5|45|59|AM|third|night|| +20760|AAAAAAAAJBBFAAAA|20760|5|46|0|AM|third|night|| +20761|AAAAAAAAKBBFAAAA|20761|5|46|1|AM|third|night|| +20762|AAAAAAAALBBFAAAA|20762|5|46|2|AM|third|night|| +20763|AAAAAAAAMBBFAAAA|20763|5|46|3|AM|third|night|| +20764|AAAAAAAANBBFAAAA|20764|5|46|4|AM|third|night|| +20765|AAAAAAAAOBBFAAAA|20765|5|46|5|AM|third|night|| +20766|AAAAAAAAPBBFAAAA|20766|5|46|6|AM|third|night|| +20767|AAAAAAAAACBFAAAA|20767|5|46|7|AM|third|night|| +20768|AAAAAAAABCBFAAAA|20768|5|46|8|AM|third|night|| +20769|AAAAAAAACCBFAAAA|20769|5|46|9|AM|third|night|| +20770|AAAAAAAADCBFAAAA|20770|5|46|10|AM|third|night|| +20771|AAAAAAAAECBFAAAA|20771|5|46|11|AM|third|night|| +20772|AAAAAAAAFCBFAAAA|20772|5|46|12|AM|third|night|| +20773|AAAAAAAAGCBFAAAA|20773|5|46|13|AM|third|night|| +20774|AAAAAAAAHCBFAAAA|20774|5|46|14|AM|third|night|| +20775|AAAAAAAAICBFAAAA|20775|5|46|15|AM|third|night|| +20776|AAAAAAAAJCBFAAAA|20776|5|46|16|AM|third|night|| +20777|AAAAAAAAKCBFAAAA|20777|5|46|17|AM|third|night|| +20778|AAAAAAAALCBFAAAA|20778|5|46|18|AM|third|night|| +20779|AAAAAAAAMCBFAAAA|20779|5|46|19|AM|third|night|| +20780|AAAAAAAANCBFAAAA|20780|5|46|20|AM|third|night|| +20781|AAAAAAAAOCBFAAAA|20781|5|46|21|AM|third|night|| +20782|AAAAAAAAPCBFAAAA|20782|5|46|22|AM|third|night|| +20783|AAAAAAAAADBFAAAA|20783|5|46|23|AM|third|night|| +20784|AAAAAAAABDBFAAAA|20784|5|46|24|AM|third|night|| +20785|AAAAAAAACDBFAAAA|20785|5|46|25|AM|third|night|| +20786|AAAAAAAADDBFAAAA|20786|5|46|26|AM|third|night|| +20787|AAAAAAAAEDBFAAAA|20787|5|46|27|AM|third|night|| +20788|AAAAAAAAFDBFAAAA|20788|5|46|28|AM|third|night|| +20789|AAAAAAAAGDBFAAAA|20789|5|46|29|AM|third|night|| +20790|AAAAAAAAHDBFAAAA|20790|5|46|30|AM|third|night|| +20791|AAAAAAAAIDBFAAAA|20791|5|46|31|AM|third|night|| +20792|AAAAAAAAJDBFAAAA|20792|5|46|32|AM|third|night|| +20793|AAAAAAAAKDBFAAAA|20793|5|46|33|AM|third|night|| +20794|AAAAAAAALDBFAAAA|20794|5|46|34|AM|third|night|| +20795|AAAAAAAAMDBFAAAA|20795|5|46|35|AM|third|night|| +20796|AAAAAAAANDBFAAAA|20796|5|46|36|AM|third|night|| +20797|AAAAAAAAODBFAAAA|20797|5|46|37|AM|third|night|| +20798|AAAAAAAAPDBFAAAA|20798|5|46|38|AM|third|night|| +20799|AAAAAAAAAEBFAAAA|20799|5|46|39|AM|third|night|| +20800|AAAAAAAABEBFAAAA|20800|5|46|40|AM|third|night|| +20801|AAAAAAAACEBFAAAA|20801|5|46|41|AM|third|night|| +20802|AAAAAAAADEBFAAAA|20802|5|46|42|AM|third|night|| +20803|AAAAAAAAEEBFAAAA|20803|5|46|43|AM|third|night|| +20804|AAAAAAAAFEBFAAAA|20804|5|46|44|AM|third|night|| +20805|AAAAAAAAGEBFAAAA|20805|5|46|45|AM|third|night|| +20806|AAAAAAAAHEBFAAAA|20806|5|46|46|AM|third|night|| +20807|AAAAAAAAIEBFAAAA|20807|5|46|47|AM|third|night|| +20808|AAAAAAAAJEBFAAAA|20808|5|46|48|AM|third|night|| +20809|AAAAAAAAKEBFAAAA|20809|5|46|49|AM|third|night|| +20810|AAAAAAAALEBFAAAA|20810|5|46|50|AM|third|night|| +20811|AAAAAAAAMEBFAAAA|20811|5|46|51|AM|third|night|| +20812|AAAAAAAANEBFAAAA|20812|5|46|52|AM|third|night|| +20813|AAAAAAAAOEBFAAAA|20813|5|46|53|AM|third|night|| +20814|AAAAAAAAPEBFAAAA|20814|5|46|54|AM|third|night|| +20815|AAAAAAAAAFBFAAAA|20815|5|46|55|AM|third|night|| +20816|AAAAAAAABFBFAAAA|20816|5|46|56|AM|third|night|| +20817|AAAAAAAACFBFAAAA|20817|5|46|57|AM|third|night|| +20818|AAAAAAAADFBFAAAA|20818|5|46|58|AM|third|night|| +20819|AAAAAAAAEFBFAAAA|20819|5|46|59|AM|third|night|| +20820|AAAAAAAAFFBFAAAA|20820|5|47|0|AM|third|night|| +20821|AAAAAAAAGFBFAAAA|20821|5|47|1|AM|third|night|| +20822|AAAAAAAAHFBFAAAA|20822|5|47|2|AM|third|night|| +20823|AAAAAAAAIFBFAAAA|20823|5|47|3|AM|third|night|| +20824|AAAAAAAAJFBFAAAA|20824|5|47|4|AM|third|night|| +20825|AAAAAAAAKFBFAAAA|20825|5|47|5|AM|third|night|| +20826|AAAAAAAALFBFAAAA|20826|5|47|6|AM|third|night|| +20827|AAAAAAAAMFBFAAAA|20827|5|47|7|AM|third|night|| +20828|AAAAAAAANFBFAAAA|20828|5|47|8|AM|third|night|| +20829|AAAAAAAAOFBFAAAA|20829|5|47|9|AM|third|night|| +20830|AAAAAAAAPFBFAAAA|20830|5|47|10|AM|third|night|| +20831|AAAAAAAAAGBFAAAA|20831|5|47|11|AM|third|night|| +20832|AAAAAAAABGBFAAAA|20832|5|47|12|AM|third|night|| +20833|AAAAAAAACGBFAAAA|20833|5|47|13|AM|third|night|| +20834|AAAAAAAADGBFAAAA|20834|5|47|14|AM|third|night|| +20835|AAAAAAAAEGBFAAAA|20835|5|47|15|AM|third|night|| +20836|AAAAAAAAFGBFAAAA|20836|5|47|16|AM|third|night|| +20837|AAAAAAAAGGBFAAAA|20837|5|47|17|AM|third|night|| +20838|AAAAAAAAHGBFAAAA|20838|5|47|18|AM|third|night|| +20839|AAAAAAAAIGBFAAAA|20839|5|47|19|AM|third|night|| +20840|AAAAAAAAJGBFAAAA|20840|5|47|20|AM|third|night|| +20841|AAAAAAAAKGBFAAAA|20841|5|47|21|AM|third|night|| +20842|AAAAAAAALGBFAAAA|20842|5|47|22|AM|third|night|| +20843|AAAAAAAAMGBFAAAA|20843|5|47|23|AM|third|night|| +20844|AAAAAAAANGBFAAAA|20844|5|47|24|AM|third|night|| +20845|AAAAAAAAOGBFAAAA|20845|5|47|25|AM|third|night|| +20846|AAAAAAAAPGBFAAAA|20846|5|47|26|AM|third|night|| +20847|AAAAAAAAAHBFAAAA|20847|5|47|27|AM|third|night|| +20848|AAAAAAAABHBFAAAA|20848|5|47|28|AM|third|night|| +20849|AAAAAAAACHBFAAAA|20849|5|47|29|AM|third|night|| +20850|AAAAAAAADHBFAAAA|20850|5|47|30|AM|third|night|| +20851|AAAAAAAAEHBFAAAA|20851|5|47|31|AM|third|night|| +20852|AAAAAAAAFHBFAAAA|20852|5|47|32|AM|third|night|| +20853|AAAAAAAAGHBFAAAA|20853|5|47|33|AM|third|night|| +20854|AAAAAAAAHHBFAAAA|20854|5|47|34|AM|third|night|| +20855|AAAAAAAAIHBFAAAA|20855|5|47|35|AM|third|night|| +20856|AAAAAAAAJHBFAAAA|20856|5|47|36|AM|third|night|| +20857|AAAAAAAAKHBFAAAA|20857|5|47|37|AM|third|night|| +20858|AAAAAAAALHBFAAAA|20858|5|47|38|AM|third|night|| +20859|AAAAAAAAMHBFAAAA|20859|5|47|39|AM|third|night|| +20860|AAAAAAAANHBFAAAA|20860|5|47|40|AM|third|night|| +20861|AAAAAAAAOHBFAAAA|20861|5|47|41|AM|third|night|| +20862|AAAAAAAAPHBFAAAA|20862|5|47|42|AM|third|night|| +20863|AAAAAAAAAIBFAAAA|20863|5|47|43|AM|third|night|| +20864|AAAAAAAABIBFAAAA|20864|5|47|44|AM|third|night|| +20865|AAAAAAAACIBFAAAA|20865|5|47|45|AM|third|night|| +20866|AAAAAAAADIBFAAAA|20866|5|47|46|AM|third|night|| +20867|AAAAAAAAEIBFAAAA|20867|5|47|47|AM|third|night|| +20868|AAAAAAAAFIBFAAAA|20868|5|47|48|AM|third|night|| +20869|AAAAAAAAGIBFAAAA|20869|5|47|49|AM|third|night|| +20870|AAAAAAAAHIBFAAAA|20870|5|47|50|AM|third|night|| +20871|AAAAAAAAIIBFAAAA|20871|5|47|51|AM|third|night|| +20872|AAAAAAAAJIBFAAAA|20872|5|47|52|AM|third|night|| +20873|AAAAAAAAKIBFAAAA|20873|5|47|53|AM|third|night|| +20874|AAAAAAAALIBFAAAA|20874|5|47|54|AM|third|night|| +20875|AAAAAAAAMIBFAAAA|20875|5|47|55|AM|third|night|| +20876|AAAAAAAANIBFAAAA|20876|5|47|56|AM|third|night|| +20877|AAAAAAAAOIBFAAAA|20877|5|47|57|AM|third|night|| +20878|AAAAAAAAPIBFAAAA|20878|5|47|58|AM|third|night|| +20879|AAAAAAAAAJBFAAAA|20879|5|47|59|AM|third|night|| +20880|AAAAAAAABJBFAAAA|20880|5|48|0|AM|third|night|| +20881|AAAAAAAACJBFAAAA|20881|5|48|1|AM|third|night|| +20882|AAAAAAAADJBFAAAA|20882|5|48|2|AM|third|night|| +20883|AAAAAAAAEJBFAAAA|20883|5|48|3|AM|third|night|| +20884|AAAAAAAAFJBFAAAA|20884|5|48|4|AM|third|night|| +20885|AAAAAAAAGJBFAAAA|20885|5|48|5|AM|third|night|| +20886|AAAAAAAAHJBFAAAA|20886|5|48|6|AM|third|night|| +20887|AAAAAAAAIJBFAAAA|20887|5|48|7|AM|third|night|| +20888|AAAAAAAAJJBFAAAA|20888|5|48|8|AM|third|night|| +20889|AAAAAAAAKJBFAAAA|20889|5|48|9|AM|third|night|| +20890|AAAAAAAALJBFAAAA|20890|5|48|10|AM|third|night|| +20891|AAAAAAAAMJBFAAAA|20891|5|48|11|AM|third|night|| +20892|AAAAAAAANJBFAAAA|20892|5|48|12|AM|third|night|| +20893|AAAAAAAAOJBFAAAA|20893|5|48|13|AM|third|night|| +20894|AAAAAAAAPJBFAAAA|20894|5|48|14|AM|third|night|| +20895|AAAAAAAAAKBFAAAA|20895|5|48|15|AM|third|night|| +20896|AAAAAAAABKBFAAAA|20896|5|48|16|AM|third|night|| +20897|AAAAAAAACKBFAAAA|20897|5|48|17|AM|third|night|| +20898|AAAAAAAADKBFAAAA|20898|5|48|18|AM|third|night|| +20899|AAAAAAAAEKBFAAAA|20899|5|48|19|AM|third|night|| +20900|AAAAAAAAFKBFAAAA|20900|5|48|20|AM|third|night|| +20901|AAAAAAAAGKBFAAAA|20901|5|48|21|AM|third|night|| +20902|AAAAAAAAHKBFAAAA|20902|5|48|22|AM|third|night|| +20903|AAAAAAAAIKBFAAAA|20903|5|48|23|AM|third|night|| +20904|AAAAAAAAJKBFAAAA|20904|5|48|24|AM|third|night|| +20905|AAAAAAAAKKBFAAAA|20905|5|48|25|AM|third|night|| +20906|AAAAAAAALKBFAAAA|20906|5|48|26|AM|third|night|| +20907|AAAAAAAAMKBFAAAA|20907|5|48|27|AM|third|night|| +20908|AAAAAAAANKBFAAAA|20908|5|48|28|AM|third|night|| +20909|AAAAAAAAOKBFAAAA|20909|5|48|29|AM|third|night|| +20910|AAAAAAAAPKBFAAAA|20910|5|48|30|AM|third|night|| +20911|AAAAAAAAALBFAAAA|20911|5|48|31|AM|third|night|| +20912|AAAAAAAABLBFAAAA|20912|5|48|32|AM|third|night|| +20913|AAAAAAAACLBFAAAA|20913|5|48|33|AM|third|night|| +20914|AAAAAAAADLBFAAAA|20914|5|48|34|AM|third|night|| +20915|AAAAAAAAELBFAAAA|20915|5|48|35|AM|third|night|| +20916|AAAAAAAAFLBFAAAA|20916|5|48|36|AM|third|night|| +20917|AAAAAAAAGLBFAAAA|20917|5|48|37|AM|third|night|| +20918|AAAAAAAAHLBFAAAA|20918|5|48|38|AM|third|night|| +20919|AAAAAAAAILBFAAAA|20919|5|48|39|AM|third|night|| +20920|AAAAAAAAJLBFAAAA|20920|5|48|40|AM|third|night|| +20921|AAAAAAAAKLBFAAAA|20921|5|48|41|AM|third|night|| +20922|AAAAAAAALLBFAAAA|20922|5|48|42|AM|third|night|| +20923|AAAAAAAAMLBFAAAA|20923|5|48|43|AM|third|night|| +20924|AAAAAAAANLBFAAAA|20924|5|48|44|AM|third|night|| +20925|AAAAAAAAOLBFAAAA|20925|5|48|45|AM|third|night|| +20926|AAAAAAAAPLBFAAAA|20926|5|48|46|AM|third|night|| +20927|AAAAAAAAAMBFAAAA|20927|5|48|47|AM|third|night|| +20928|AAAAAAAABMBFAAAA|20928|5|48|48|AM|third|night|| +20929|AAAAAAAACMBFAAAA|20929|5|48|49|AM|third|night|| +20930|AAAAAAAADMBFAAAA|20930|5|48|50|AM|third|night|| +20931|AAAAAAAAEMBFAAAA|20931|5|48|51|AM|third|night|| +20932|AAAAAAAAFMBFAAAA|20932|5|48|52|AM|third|night|| +20933|AAAAAAAAGMBFAAAA|20933|5|48|53|AM|third|night|| +20934|AAAAAAAAHMBFAAAA|20934|5|48|54|AM|third|night|| +20935|AAAAAAAAIMBFAAAA|20935|5|48|55|AM|third|night|| +20936|AAAAAAAAJMBFAAAA|20936|5|48|56|AM|third|night|| +20937|AAAAAAAAKMBFAAAA|20937|5|48|57|AM|third|night|| +20938|AAAAAAAALMBFAAAA|20938|5|48|58|AM|third|night|| +20939|AAAAAAAAMMBFAAAA|20939|5|48|59|AM|third|night|| +20940|AAAAAAAANMBFAAAA|20940|5|49|0|AM|third|night|| +20941|AAAAAAAAOMBFAAAA|20941|5|49|1|AM|third|night|| +20942|AAAAAAAAPMBFAAAA|20942|5|49|2|AM|third|night|| +20943|AAAAAAAAANBFAAAA|20943|5|49|3|AM|third|night|| +20944|AAAAAAAABNBFAAAA|20944|5|49|4|AM|third|night|| +20945|AAAAAAAACNBFAAAA|20945|5|49|5|AM|third|night|| +20946|AAAAAAAADNBFAAAA|20946|5|49|6|AM|third|night|| +20947|AAAAAAAAENBFAAAA|20947|5|49|7|AM|third|night|| +20948|AAAAAAAAFNBFAAAA|20948|5|49|8|AM|third|night|| +20949|AAAAAAAAGNBFAAAA|20949|5|49|9|AM|third|night|| +20950|AAAAAAAAHNBFAAAA|20950|5|49|10|AM|third|night|| +20951|AAAAAAAAINBFAAAA|20951|5|49|11|AM|third|night|| +20952|AAAAAAAAJNBFAAAA|20952|5|49|12|AM|third|night|| +20953|AAAAAAAAKNBFAAAA|20953|5|49|13|AM|third|night|| +20954|AAAAAAAALNBFAAAA|20954|5|49|14|AM|third|night|| +20955|AAAAAAAAMNBFAAAA|20955|5|49|15|AM|third|night|| +20956|AAAAAAAANNBFAAAA|20956|5|49|16|AM|third|night|| +20957|AAAAAAAAONBFAAAA|20957|5|49|17|AM|third|night|| +20958|AAAAAAAAPNBFAAAA|20958|5|49|18|AM|third|night|| +20959|AAAAAAAAAOBFAAAA|20959|5|49|19|AM|third|night|| +20960|AAAAAAAABOBFAAAA|20960|5|49|20|AM|third|night|| +20961|AAAAAAAACOBFAAAA|20961|5|49|21|AM|third|night|| +20962|AAAAAAAADOBFAAAA|20962|5|49|22|AM|third|night|| +20963|AAAAAAAAEOBFAAAA|20963|5|49|23|AM|third|night|| +20964|AAAAAAAAFOBFAAAA|20964|5|49|24|AM|third|night|| +20965|AAAAAAAAGOBFAAAA|20965|5|49|25|AM|third|night|| +20966|AAAAAAAAHOBFAAAA|20966|5|49|26|AM|third|night|| +20967|AAAAAAAAIOBFAAAA|20967|5|49|27|AM|third|night|| +20968|AAAAAAAAJOBFAAAA|20968|5|49|28|AM|third|night|| +20969|AAAAAAAAKOBFAAAA|20969|5|49|29|AM|third|night|| +20970|AAAAAAAALOBFAAAA|20970|5|49|30|AM|third|night|| +20971|AAAAAAAAMOBFAAAA|20971|5|49|31|AM|third|night|| +20972|AAAAAAAANOBFAAAA|20972|5|49|32|AM|third|night|| +20973|AAAAAAAAOOBFAAAA|20973|5|49|33|AM|third|night|| +20974|AAAAAAAAPOBFAAAA|20974|5|49|34|AM|third|night|| +20975|AAAAAAAAAPBFAAAA|20975|5|49|35|AM|third|night|| +20976|AAAAAAAABPBFAAAA|20976|5|49|36|AM|third|night|| +20977|AAAAAAAACPBFAAAA|20977|5|49|37|AM|third|night|| +20978|AAAAAAAADPBFAAAA|20978|5|49|38|AM|third|night|| +20979|AAAAAAAAEPBFAAAA|20979|5|49|39|AM|third|night|| +20980|AAAAAAAAFPBFAAAA|20980|5|49|40|AM|third|night|| +20981|AAAAAAAAGPBFAAAA|20981|5|49|41|AM|third|night|| +20982|AAAAAAAAHPBFAAAA|20982|5|49|42|AM|third|night|| +20983|AAAAAAAAIPBFAAAA|20983|5|49|43|AM|third|night|| +20984|AAAAAAAAJPBFAAAA|20984|5|49|44|AM|third|night|| +20985|AAAAAAAAKPBFAAAA|20985|5|49|45|AM|third|night|| +20986|AAAAAAAALPBFAAAA|20986|5|49|46|AM|third|night|| +20987|AAAAAAAAMPBFAAAA|20987|5|49|47|AM|third|night|| +20988|AAAAAAAANPBFAAAA|20988|5|49|48|AM|third|night|| +20989|AAAAAAAAOPBFAAAA|20989|5|49|49|AM|third|night|| +20990|AAAAAAAAPPBFAAAA|20990|5|49|50|AM|third|night|| +20991|AAAAAAAAAACFAAAA|20991|5|49|51|AM|third|night|| +20992|AAAAAAAABACFAAAA|20992|5|49|52|AM|third|night|| +20993|AAAAAAAACACFAAAA|20993|5|49|53|AM|third|night|| +20994|AAAAAAAADACFAAAA|20994|5|49|54|AM|third|night|| +20995|AAAAAAAAEACFAAAA|20995|5|49|55|AM|third|night|| +20996|AAAAAAAAFACFAAAA|20996|5|49|56|AM|third|night|| +20997|AAAAAAAAGACFAAAA|20997|5|49|57|AM|third|night|| +20998|AAAAAAAAHACFAAAA|20998|5|49|58|AM|third|night|| +20999|AAAAAAAAIACFAAAA|20999|5|49|59|AM|third|night|| +21000|AAAAAAAAJACFAAAA|21000|5|50|0|AM|third|night|| +21001|AAAAAAAAKACFAAAA|21001|5|50|1|AM|third|night|| +21002|AAAAAAAALACFAAAA|21002|5|50|2|AM|third|night|| +21003|AAAAAAAAMACFAAAA|21003|5|50|3|AM|third|night|| +21004|AAAAAAAANACFAAAA|21004|5|50|4|AM|third|night|| +21005|AAAAAAAAOACFAAAA|21005|5|50|5|AM|third|night|| +21006|AAAAAAAAPACFAAAA|21006|5|50|6|AM|third|night|| +21007|AAAAAAAAABCFAAAA|21007|5|50|7|AM|third|night|| +21008|AAAAAAAABBCFAAAA|21008|5|50|8|AM|third|night|| +21009|AAAAAAAACBCFAAAA|21009|5|50|9|AM|third|night|| +21010|AAAAAAAADBCFAAAA|21010|5|50|10|AM|third|night|| +21011|AAAAAAAAEBCFAAAA|21011|5|50|11|AM|third|night|| +21012|AAAAAAAAFBCFAAAA|21012|5|50|12|AM|third|night|| +21013|AAAAAAAAGBCFAAAA|21013|5|50|13|AM|third|night|| +21014|AAAAAAAAHBCFAAAA|21014|5|50|14|AM|third|night|| +21015|AAAAAAAAIBCFAAAA|21015|5|50|15|AM|third|night|| +21016|AAAAAAAAJBCFAAAA|21016|5|50|16|AM|third|night|| +21017|AAAAAAAAKBCFAAAA|21017|5|50|17|AM|third|night|| +21018|AAAAAAAALBCFAAAA|21018|5|50|18|AM|third|night|| +21019|AAAAAAAAMBCFAAAA|21019|5|50|19|AM|third|night|| +21020|AAAAAAAANBCFAAAA|21020|5|50|20|AM|third|night|| +21021|AAAAAAAAOBCFAAAA|21021|5|50|21|AM|third|night|| +21022|AAAAAAAAPBCFAAAA|21022|5|50|22|AM|third|night|| +21023|AAAAAAAAACCFAAAA|21023|5|50|23|AM|third|night|| +21024|AAAAAAAABCCFAAAA|21024|5|50|24|AM|third|night|| +21025|AAAAAAAACCCFAAAA|21025|5|50|25|AM|third|night|| +21026|AAAAAAAADCCFAAAA|21026|5|50|26|AM|third|night|| +21027|AAAAAAAAECCFAAAA|21027|5|50|27|AM|third|night|| +21028|AAAAAAAAFCCFAAAA|21028|5|50|28|AM|third|night|| +21029|AAAAAAAAGCCFAAAA|21029|5|50|29|AM|third|night|| +21030|AAAAAAAAHCCFAAAA|21030|5|50|30|AM|third|night|| +21031|AAAAAAAAICCFAAAA|21031|5|50|31|AM|third|night|| +21032|AAAAAAAAJCCFAAAA|21032|5|50|32|AM|third|night|| +21033|AAAAAAAAKCCFAAAA|21033|5|50|33|AM|third|night|| +21034|AAAAAAAALCCFAAAA|21034|5|50|34|AM|third|night|| +21035|AAAAAAAAMCCFAAAA|21035|5|50|35|AM|third|night|| +21036|AAAAAAAANCCFAAAA|21036|5|50|36|AM|third|night|| +21037|AAAAAAAAOCCFAAAA|21037|5|50|37|AM|third|night|| +21038|AAAAAAAAPCCFAAAA|21038|5|50|38|AM|third|night|| +21039|AAAAAAAAADCFAAAA|21039|5|50|39|AM|third|night|| +21040|AAAAAAAABDCFAAAA|21040|5|50|40|AM|third|night|| +21041|AAAAAAAACDCFAAAA|21041|5|50|41|AM|third|night|| +21042|AAAAAAAADDCFAAAA|21042|5|50|42|AM|third|night|| +21043|AAAAAAAAEDCFAAAA|21043|5|50|43|AM|third|night|| +21044|AAAAAAAAFDCFAAAA|21044|5|50|44|AM|third|night|| +21045|AAAAAAAAGDCFAAAA|21045|5|50|45|AM|third|night|| +21046|AAAAAAAAHDCFAAAA|21046|5|50|46|AM|third|night|| +21047|AAAAAAAAIDCFAAAA|21047|5|50|47|AM|third|night|| +21048|AAAAAAAAJDCFAAAA|21048|5|50|48|AM|third|night|| +21049|AAAAAAAAKDCFAAAA|21049|5|50|49|AM|third|night|| +21050|AAAAAAAALDCFAAAA|21050|5|50|50|AM|third|night|| +21051|AAAAAAAAMDCFAAAA|21051|5|50|51|AM|third|night|| +21052|AAAAAAAANDCFAAAA|21052|5|50|52|AM|third|night|| +21053|AAAAAAAAODCFAAAA|21053|5|50|53|AM|third|night|| +21054|AAAAAAAAPDCFAAAA|21054|5|50|54|AM|third|night|| +21055|AAAAAAAAAECFAAAA|21055|5|50|55|AM|third|night|| +21056|AAAAAAAABECFAAAA|21056|5|50|56|AM|third|night|| +21057|AAAAAAAACECFAAAA|21057|5|50|57|AM|third|night|| +21058|AAAAAAAADECFAAAA|21058|5|50|58|AM|third|night|| +21059|AAAAAAAAEECFAAAA|21059|5|50|59|AM|third|night|| +21060|AAAAAAAAFECFAAAA|21060|5|51|0|AM|third|night|| +21061|AAAAAAAAGECFAAAA|21061|5|51|1|AM|third|night|| +21062|AAAAAAAAHECFAAAA|21062|5|51|2|AM|third|night|| +21063|AAAAAAAAIECFAAAA|21063|5|51|3|AM|third|night|| +21064|AAAAAAAAJECFAAAA|21064|5|51|4|AM|third|night|| +21065|AAAAAAAAKECFAAAA|21065|5|51|5|AM|third|night|| +21066|AAAAAAAALECFAAAA|21066|5|51|6|AM|third|night|| +21067|AAAAAAAAMECFAAAA|21067|5|51|7|AM|third|night|| +21068|AAAAAAAANECFAAAA|21068|5|51|8|AM|third|night|| +21069|AAAAAAAAOECFAAAA|21069|5|51|9|AM|third|night|| +21070|AAAAAAAAPECFAAAA|21070|5|51|10|AM|third|night|| +21071|AAAAAAAAAFCFAAAA|21071|5|51|11|AM|third|night|| +21072|AAAAAAAABFCFAAAA|21072|5|51|12|AM|third|night|| +21073|AAAAAAAACFCFAAAA|21073|5|51|13|AM|third|night|| +21074|AAAAAAAADFCFAAAA|21074|5|51|14|AM|third|night|| +21075|AAAAAAAAEFCFAAAA|21075|5|51|15|AM|third|night|| +21076|AAAAAAAAFFCFAAAA|21076|5|51|16|AM|third|night|| +21077|AAAAAAAAGFCFAAAA|21077|5|51|17|AM|third|night|| +21078|AAAAAAAAHFCFAAAA|21078|5|51|18|AM|third|night|| +21079|AAAAAAAAIFCFAAAA|21079|5|51|19|AM|third|night|| +21080|AAAAAAAAJFCFAAAA|21080|5|51|20|AM|third|night|| +21081|AAAAAAAAKFCFAAAA|21081|5|51|21|AM|third|night|| +21082|AAAAAAAALFCFAAAA|21082|5|51|22|AM|third|night|| +21083|AAAAAAAAMFCFAAAA|21083|5|51|23|AM|third|night|| +21084|AAAAAAAANFCFAAAA|21084|5|51|24|AM|third|night|| +21085|AAAAAAAAOFCFAAAA|21085|5|51|25|AM|third|night|| +21086|AAAAAAAAPFCFAAAA|21086|5|51|26|AM|third|night|| +21087|AAAAAAAAAGCFAAAA|21087|5|51|27|AM|third|night|| +21088|AAAAAAAABGCFAAAA|21088|5|51|28|AM|third|night|| +21089|AAAAAAAACGCFAAAA|21089|5|51|29|AM|third|night|| +21090|AAAAAAAADGCFAAAA|21090|5|51|30|AM|third|night|| +21091|AAAAAAAAEGCFAAAA|21091|5|51|31|AM|third|night|| +21092|AAAAAAAAFGCFAAAA|21092|5|51|32|AM|third|night|| +21093|AAAAAAAAGGCFAAAA|21093|5|51|33|AM|third|night|| +21094|AAAAAAAAHGCFAAAA|21094|5|51|34|AM|third|night|| +21095|AAAAAAAAIGCFAAAA|21095|5|51|35|AM|third|night|| +21096|AAAAAAAAJGCFAAAA|21096|5|51|36|AM|third|night|| +21097|AAAAAAAAKGCFAAAA|21097|5|51|37|AM|third|night|| +21098|AAAAAAAALGCFAAAA|21098|5|51|38|AM|third|night|| +21099|AAAAAAAAMGCFAAAA|21099|5|51|39|AM|third|night|| +21100|AAAAAAAANGCFAAAA|21100|5|51|40|AM|third|night|| +21101|AAAAAAAAOGCFAAAA|21101|5|51|41|AM|third|night|| +21102|AAAAAAAAPGCFAAAA|21102|5|51|42|AM|third|night|| +21103|AAAAAAAAAHCFAAAA|21103|5|51|43|AM|third|night|| +21104|AAAAAAAABHCFAAAA|21104|5|51|44|AM|third|night|| +21105|AAAAAAAACHCFAAAA|21105|5|51|45|AM|third|night|| +21106|AAAAAAAADHCFAAAA|21106|5|51|46|AM|third|night|| +21107|AAAAAAAAEHCFAAAA|21107|5|51|47|AM|third|night|| +21108|AAAAAAAAFHCFAAAA|21108|5|51|48|AM|third|night|| +21109|AAAAAAAAGHCFAAAA|21109|5|51|49|AM|third|night|| +21110|AAAAAAAAHHCFAAAA|21110|5|51|50|AM|third|night|| +21111|AAAAAAAAIHCFAAAA|21111|5|51|51|AM|third|night|| +21112|AAAAAAAAJHCFAAAA|21112|5|51|52|AM|third|night|| +21113|AAAAAAAAKHCFAAAA|21113|5|51|53|AM|third|night|| +21114|AAAAAAAALHCFAAAA|21114|5|51|54|AM|third|night|| +21115|AAAAAAAAMHCFAAAA|21115|5|51|55|AM|third|night|| +21116|AAAAAAAANHCFAAAA|21116|5|51|56|AM|third|night|| +21117|AAAAAAAAOHCFAAAA|21117|5|51|57|AM|third|night|| +21118|AAAAAAAAPHCFAAAA|21118|5|51|58|AM|third|night|| +21119|AAAAAAAAAICFAAAA|21119|5|51|59|AM|third|night|| +21120|AAAAAAAABICFAAAA|21120|5|52|0|AM|third|night|| +21121|AAAAAAAACICFAAAA|21121|5|52|1|AM|third|night|| +21122|AAAAAAAADICFAAAA|21122|5|52|2|AM|third|night|| +21123|AAAAAAAAEICFAAAA|21123|5|52|3|AM|third|night|| +21124|AAAAAAAAFICFAAAA|21124|5|52|4|AM|third|night|| +21125|AAAAAAAAGICFAAAA|21125|5|52|5|AM|third|night|| +21126|AAAAAAAAHICFAAAA|21126|5|52|6|AM|third|night|| +21127|AAAAAAAAIICFAAAA|21127|5|52|7|AM|third|night|| +21128|AAAAAAAAJICFAAAA|21128|5|52|8|AM|third|night|| +21129|AAAAAAAAKICFAAAA|21129|5|52|9|AM|third|night|| +21130|AAAAAAAALICFAAAA|21130|5|52|10|AM|third|night|| +21131|AAAAAAAAMICFAAAA|21131|5|52|11|AM|third|night|| +21132|AAAAAAAANICFAAAA|21132|5|52|12|AM|third|night|| +21133|AAAAAAAAOICFAAAA|21133|5|52|13|AM|third|night|| +21134|AAAAAAAAPICFAAAA|21134|5|52|14|AM|third|night|| +21135|AAAAAAAAAJCFAAAA|21135|5|52|15|AM|third|night|| +21136|AAAAAAAABJCFAAAA|21136|5|52|16|AM|third|night|| +21137|AAAAAAAACJCFAAAA|21137|5|52|17|AM|third|night|| +21138|AAAAAAAADJCFAAAA|21138|5|52|18|AM|third|night|| +21139|AAAAAAAAEJCFAAAA|21139|5|52|19|AM|third|night|| +21140|AAAAAAAAFJCFAAAA|21140|5|52|20|AM|third|night|| +21141|AAAAAAAAGJCFAAAA|21141|5|52|21|AM|third|night|| +21142|AAAAAAAAHJCFAAAA|21142|5|52|22|AM|third|night|| +21143|AAAAAAAAIJCFAAAA|21143|5|52|23|AM|third|night|| +21144|AAAAAAAAJJCFAAAA|21144|5|52|24|AM|third|night|| +21145|AAAAAAAAKJCFAAAA|21145|5|52|25|AM|third|night|| +21146|AAAAAAAALJCFAAAA|21146|5|52|26|AM|third|night|| +21147|AAAAAAAAMJCFAAAA|21147|5|52|27|AM|third|night|| +21148|AAAAAAAANJCFAAAA|21148|5|52|28|AM|third|night|| +21149|AAAAAAAAOJCFAAAA|21149|5|52|29|AM|third|night|| +21150|AAAAAAAAPJCFAAAA|21150|5|52|30|AM|third|night|| +21151|AAAAAAAAAKCFAAAA|21151|5|52|31|AM|third|night|| +21152|AAAAAAAABKCFAAAA|21152|5|52|32|AM|third|night|| +21153|AAAAAAAACKCFAAAA|21153|5|52|33|AM|third|night|| +21154|AAAAAAAADKCFAAAA|21154|5|52|34|AM|third|night|| +21155|AAAAAAAAEKCFAAAA|21155|5|52|35|AM|third|night|| +21156|AAAAAAAAFKCFAAAA|21156|5|52|36|AM|third|night|| +21157|AAAAAAAAGKCFAAAA|21157|5|52|37|AM|third|night|| +21158|AAAAAAAAHKCFAAAA|21158|5|52|38|AM|third|night|| +21159|AAAAAAAAIKCFAAAA|21159|5|52|39|AM|third|night|| +21160|AAAAAAAAJKCFAAAA|21160|5|52|40|AM|third|night|| +21161|AAAAAAAAKKCFAAAA|21161|5|52|41|AM|third|night|| +21162|AAAAAAAALKCFAAAA|21162|5|52|42|AM|third|night|| +21163|AAAAAAAAMKCFAAAA|21163|5|52|43|AM|third|night|| +21164|AAAAAAAANKCFAAAA|21164|5|52|44|AM|third|night|| +21165|AAAAAAAAOKCFAAAA|21165|5|52|45|AM|third|night|| +21166|AAAAAAAAPKCFAAAA|21166|5|52|46|AM|third|night|| +21167|AAAAAAAAALCFAAAA|21167|5|52|47|AM|third|night|| +21168|AAAAAAAABLCFAAAA|21168|5|52|48|AM|third|night|| +21169|AAAAAAAACLCFAAAA|21169|5|52|49|AM|third|night|| +21170|AAAAAAAADLCFAAAA|21170|5|52|50|AM|third|night|| +21171|AAAAAAAAELCFAAAA|21171|5|52|51|AM|third|night|| +21172|AAAAAAAAFLCFAAAA|21172|5|52|52|AM|third|night|| +21173|AAAAAAAAGLCFAAAA|21173|5|52|53|AM|third|night|| +21174|AAAAAAAAHLCFAAAA|21174|5|52|54|AM|third|night|| +21175|AAAAAAAAILCFAAAA|21175|5|52|55|AM|third|night|| +21176|AAAAAAAAJLCFAAAA|21176|5|52|56|AM|third|night|| +21177|AAAAAAAAKLCFAAAA|21177|5|52|57|AM|third|night|| +21178|AAAAAAAALLCFAAAA|21178|5|52|58|AM|third|night|| +21179|AAAAAAAAMLCFAAAA|21179|5|52|59|AM|third|night|| +21180|AAAAAAAANLCFAAAA|21180|5|53|0|AM|third|night|| +21181|AAAAAAAAOLCFAAAA|21181|5|53|1|AM|third|night|| +21182|AAAAAAAAPLCFAAAA|21182|5|53|2|AM|third|night|| +21183|AAAAAAAAAMCFAAAA|21183|5|53|3|AM|third|night|| +21184|AAAAAAAABMCFAAAA|21184|5|53|4|AM|third|night|| +21185|AAAAAAAACMCFAAAA|21185|5|53|5|AM|third|night|| +21186|AAAAAAAADMCFAAAA|21186|5|53|6|AM|third|night|| +21187|AAAAAAAAEMCFAAAA|21187|5|53|7|AM|third|night|| +21188|AAAAAAAAFMCFAAAA|21188|5|53|8|AM|third|night|| +21189|AAAAAAAAGMCFAAAA|21189|5|53|9|AM|third|night|| +21190|AAAAAAAAHMCFAAAA|21190|5|53|10|AM|third|night|| +21191|AAAAAAAAIMCFAAAA|21191|5|53|11|AM|third|night|| +21192|AAAAAAAAJMCFAAAA|21192|5|53|12|AM|third|night|| +21193|AAAAAAAAKMCFAAAA|21193|5|53|13|AM|third|night|| +21194|AAAAAAAALMCFAAAA|21194|5|53|14|AM|third|night|| +21195|AAAAAAAAMMCFAAAA|21195|5|53|15|AM|third|night|| +21196|AAAAAAAANMCFAAAA|21196|5|53|16|AM|third|night|| +21197|AAAAAAAAOMCFAAAA|21197|5|53|17|AM|third|night|| +21198|AAAAAAAAPMCFAAAA|21198|5|53|18|AM|third|night|| +21199|AAAAAAAAANCFAAAA|21199|5|53|19|AM|third|night|| +21200|AAAAAAAABNCFAAAA|21200|5|53|20|AM|third|night|| +21201|AAAAAAAACNCFAAAA|21201|5|53|21|AM|third|night|| +21202|AAAAAAAADNCFAAAA|21202|5|53|22|AM|third|night|| +21203|AAAAAAAAENCFAAAA|21203|5|53|23|AM|third|night|| +21204|AAAAAAAAFNCFAAAA|21204|5|53|24|AM|third|night|| +21205|AAAAAAAAGNCFAAAA|21205|5|53|25|AM|third|night|| +21206|AAAAAAAAHNCFAAAA|21206|5|53|26|AM|third|night|| +21207|AAAAAAAAINCFAAAA|21207|5|53|27|AM|third|night|| +21208|AAAAAAAAJNCFAAAA|21208|5|53|28|AM|third|night|| +21209|AAAAAAAAKNCFAAAA|21209|5|53|29|AM|third|night|| +21210|AAAAAAAALNCFAAAA|21210|5|53|30|AM|third|night|| +21211|AAAAAAAAMNCFAAAA|21211|5|53|31|AM|third|night|| +21212|AAAAAAAANNCFAAAA|21212|5|53|32|AM|third|night|| +21213|AAAAAAAAONCFAAAA|21213|5|53|33|AM|third|night|| +21214|AAAAAAAAPNCFAAAA|21214|5|53|34|AM|third|night|| +21215|AAAAAAAAAOCFAAAA|21215|5|53|35|AM|third|night|| +21216|AAAAAAAABOCFAAAA|21216|5|53|36|AM|third|night|| +21217|AAAAAAAACOCFAAAA|21217|5|53|37|AM|third|night|| +21218|AAAAAAAADOCFAAAA|21218|5|53|38|AM|third|night|| +21219|AAAAAAAAEOCFAAAA|21219|5|53|39|AM|third|night|| +21220|AAAAAAAAFOCFAAAA|21220|5|53|40|AM|third|night|| +21221|AAAAAAAAGOCFAAAA|21221|5|53|41|AM|third|night|| +21222|AAAAAAAAHOCFAAAA|21222|5|53|42|AM|third|night|| +21223|AAAAAAAAIOCFAAAA|21223|5|53|43|AM|third|night|| +21224|AAAAAAAAJOCFAAAA|21224|5|53|44|AM|third|night|| +21225|AAAAAAAAKOCFAAAA|21225|5|53|45|AM|third|night|| +21226|AAAAAAAALOCFAAAA|21226|5|53|46|AM|third|night|| +21227|AAAAAAAAMOCFAAAA|21227|5|53|47|AM|third|night|| +21228|AAAAAAAANOCFAAAA|21228|5|53|48|AM|third|night|| +21229|AAAAAAAAOOCFAAAA|21229|5|53|49|AM|third|night|| +21230|AAAAAAAAPOCFAAAA|21230|5|53|50|AM|third|night|| +21231|AAAAAAAAAPCFAAAA|21231|5|53|51|AM|third|night|| +21232|AAAAAAAABPCFAAAA|21232|5|53|52|AM|third|night|| +21233|AAAAAAAACPCFAAAA|21233|5|53|53|AM|third|night|| +21234|AAAAAAAADPCFAAAA|21234|5|53|54|AM|third|night|| +21235|AAAAAAAAEPCFAAAA|21235|5|53|55|AM|third|night|| +21236|AAAAAAAAFPCFAAAA|21236|5|53|56|AM|third|night|| +21237|AAAAAAAAGPCFAAAA|21237|5|53|57|AM|third|night|| +21238|AAAAAAAAHPCFAAAA|21238|5|53|58|AM|third|night|| +21239|AAAAAAAAIPCFAAAA|21239|5|53|59|AM|third|night|| +21240|AAAAAAAAJPCFAAAA|21240|5|54|0|AM|third|night|| +21241|AAAAAAAAKPCFAAAA|21241|5|54|1|AM|third|night|| +21242|AAAAAAAALPCFAAAA|21242|5|54|2|AM|third|night|| +21243|AAAAAAAAMPCFAAAA|21243|5|54|3|AM|third|night|| +21244|AAAAAAAANPCFAAAA|21244|5|54|4|AM|third|night|| +21245|AAAAAAAAOPCFAAAA|21245|5|54|5|AM|third|night|| +21246|AAAAAAAAPPCFAAAA|21246|5|54|6|AM|third|night|| +21247|AAAAAAAAAADFAAAA|21247|5|54|7|AM|third|night|| +21248|AAAAAAAABADFAAAA|21248|5|54|8|AM|third|night|| +21249|AAAAAAAACADFAAAA|21249|5|54|9|AM|third|night|| +21250|AAAAAAAADADFAAAA|21250|5|54|10|AM|third|night|| +21251|AAAAAAAAEADFAAAA|21251|5|54|11|AM|third|night|| +21252|AAAAAAAAFADFAAAA|21252|5|54|12|AM|third|night|| +21253|AAAAAAAAGADFAAAA|21253|5|54|13|AM|third|night|| +21254|AAAAAAAAHADFAAAA|21254|5|54|14|AM|third|night|| +21255|AAAAAAAAIADFAAAA|21255|5|54|15|AM|third|night|| +21256|AAAAAAAAJADFAAAA|21256|5|54|16|AM|third|night|| +21257|AAAAAAAAKADFAAAA|21257|5|54|17|AM|third|night|| +21258|AAAAAAAALADFAAAA|21258|5|54|18|AM|third|night|| +21259|AAAAAAAAMADFAAAA|21259|5|54|19|AM|third|night|| +21260|AAAAAAAANADFAAAA|21260|5|54|20|AM|third|night|| +21261|AAAAAAAAOADFAAAA|21261|5|54|21|AM|third|night|| +21262|AAAAAAAAPADFAAAA|21262|5|54|22|AM|third|night|| +21263|AAAAAAAAABDFAAAA|21263|5|54|23|AM|third|night|| +21264|AAAAAAAABBDFAAAA|21264|5|54|24|AM|third|night|| +21265|AAAAAAAACBDFAAAA|21265|5|54|25|AM|third|night|| +21266|AAAAAAAADBDFAAAA|21266|5|54|26|AM|third|night|| +21267|AAAAAAAAEBDFAAAA|21267|5|54|27|AM|third|night|| +21268|AAAAAAAAFBDFAAAA|21268|5|54|28|AM|third|night|| +21269|AAAAAAAAGBDFAAAA|21269|5|54|29|AM|third|night|| +21270|AAAAAAAAHBDFAAAA|21270|5|54|30|AM|third|night|| +21271|AAAAAAAAIBDFAAAA|21271|5|54|31|AM|third|night|| +21272|AAAAAAAAJBDFAAAA|21272|5|54|32|AM|third|night|| +21273|AAAAAAAAKBDFAAAA|21273|5|54|33|AM|third|night|| +21274|AAAAAAAALBDFAAAA|21274|5|54|34|AM|third|night|| +21275|AAAAAAAAMBDFAAAA|21275|5|54|35|AM|third|night|| +21276|AAAAAAAANBDFAAAA|21276|5|54|36|AM|third|night|| +21277|AAAAAAAAOBDFAAAA|21277|5|54|37|AM|third|night|| +21278|AAAAAAAAPBDFAAAA|21278|5|54|38|AM|third|night|| +21279|AAAAAAAAACDFAAAA|21279|5|54|39|AM|third|night|| +21280|AAAAAAAABCDFAAAA|21280|5|54|40|AM|third|night|| +21281|AAAAAAAACCDFAAAA|21281|5|54|41|AM|third|night|| +21282|AAAAAAAADCDFAAAA|21282|5|54|42|AM|third|night|| +21283|AAAAAAAAECDFAAAA|21283|5|54|43|AM|third|night|| +21284|AAAAAAAAFCDFAAAA|21284|5|54|44|AM|third|night|| +21285|AAAAAAAAGCDFAAAA|21285|5|54|45|AM|third|night|| +21286|AAAAAAAAHCDFAAAA|21286|5|54|46|AM|third|night|| +21287|AAAAAAAAICDFAAAA|21287|5|54|47|AM|third|night|| +21288|AAAAAAAAJCDFAAAA|21288|5|54|48|AM|third|night|| +21289|AAAAAAAAKCDFAAAA|21289|5|54|49|AM|third|night|| +21290|AAAAAAAALCDFAAAA|21290|5|54|50|AM|third|night|| +21291|AAAAAAAAMCDFAAAA|21291|5|54|51|AM|third|night|| +21292|AAAAAAAANCDFAAAA|21292|5|54|52|AM|third|night|| +21293|AAAAAAAAOCDFAAAA|21293|5|54|53|AM|third|night|| +21294|AAAAAAAAPCDFAAAA|21294|5|54|54|AM|third|night|| +21295|AAAAAAAAADDFAAAA|21295|5|54|55|AM|third|night|| +21296|AAAAAAAABDDFAAAA|21296|5|54|56|AM|third|night|| +21297|AAAAAAAACDDFAAAA|21297|5|54|57|AM|third|night|| +21298|AAAAAAAADDDFAAAA|21298|5|54|58|AM|third|night|| +21299|AAAAAAAAEDDFAAAA|21299|5|54|59|AM|third|night|| +21300|AAAAAAAAFDDFAAAA|21300|5|55|0|AM|third|night|| +21301|AAAAAAAAGDDFAAAA|21301|5|55|1|AM|third|night|| +21302|AAAAAAAAHDDFAAAA|21302|5|55|2|AM|third|night|| +21303|AAAAAAAAIDDFAAAA|21303|5|55|3|AM|third|night|| +21304|AAAAAAAAJDDFAAAA|21304|5|55|4|AM|third|night|| +21305|AAAAAAAAKDDFAAAA|21305|5|55|5|AM|third|night|| +21306|AAAAAAAALDDFAAAA|21306|5|55|6|AM|third|night|| +21307|AAAAAAAAMDDFAAAA|21307|5|55|7|AM|third|night|| +21308|AAAAAAAANDDFAAAA|21308|5|55|8|AM|third|night|| +21309|AAAAAAAAODDFAAAA|21309|5|55|9|AM|third|night|| +21310|AAAAAAAAPDDFAAAA|21310|5|55|10|AM|third|night|| +21311|AAAAAAAAAEDFAAAA|21311|5|55|11|AM|third|night|| +21312|AAAAAAAABEDFAAAA|21312|5|55|12|AM|third|night|| +21313|AAAAAAAACEDFAAAA|21313|5|55|13|AM|third|night|| +21314|AAAAAAAADEDFAAAA|21314|5|55|14|AM|third|night|| +21315|AAAAAAAAEEDFAAAA|21315|5|55|15|AM|third|night|| +21316|AAAAAAAAFEDFAAAA|21316|5|55|16|AM|third|night|| +21317|AAAAAAAAGEDFAAAA|21317|5|55|17|AM|third|night|| +21318|AAAAAAAAHEDFAAAA|21318|5|55|18|AM|third|night|| +21319|AAAAAAAAIEDFAAAA|21319|5|55|19|AM|third|night|| +21320|AAAAAAAAJEDFAAAA|21320|5|55|20|AM|third|night|| +21321|AAAAAAAAKEDFAAAA|21321|5|55|21|AM|third|night|| +21322|AAAAAAAALEDFAAAA|21322|5|55|22|AM|third|night|| +21323|AAAAAAAAMEDFAAAA|21323|5|55|23|AM|third|night|| +21324|AAAAAAAANEDFAAAA|21324|5|55|24|AM|third|night|| +21325|AAAAAAAAOEDFAAAA|21325|5|55|25|AM|third|night|| +21326|AAAAAAAAPEDFAAAA|21326|5|55|26|AM|third|night|| +21327|AAAAAAAAAFDFAAAA|21327|5|55|27|AM|third|night|| +21328|AAAAAAAABFDFAAAA|21328|5|55|28|AM|third|night|| +21329|AAAAAAAACFDFAAAA|21329|5|55|29|AM|third|night|| +21330|AAAAAAAADFDFAAAA|21330|5|55|30|AM|third|night|| +21331|AAAAAAAAEFDFAAAA|21331|5|55|31|AM|third|night|| +21332|AAAAAAAAFFDFAAAA|21332|5|55|32|AM|third|night|| +21333|AAAAAAAAGFDFAAAA|21333|5|55|33|AM|third|night|| +21334|AAAAAAAAHFDFAAAA|21334|5|55|34|AM|third|night|| +21335|AAAAAAAAIFDFAAAA|21335|5|55|35|AM|third|night|| +21336|AAAAAAAAJFDFAAAA|21336|5|55|36|AM|third|night|| +21337|AAAAAAAAKFDFAAAA|21337|5|55|37|AM|third|night|| +21338|AAAAAAAALFDFAAAA|21338|5|55|38|AM|third|night|| +21339|AAAAAAAAMFDFAAAA|21339|5|55|39|AM|third|night|| +21340|AAAAAAAANFDFAAAA|21340|5|55|40|AM|third|night|| +21341|AAAAAAAAOFDFAAAA|21341|5|55|41|AM|third|night|| +21342|AAAAAAAAPFDFAAAA|21342|5|55|42|AM|third|night|| +21343|AAAAAAAAAGDFAAAA|21343|5|55|43|AM|third|night|| +21344|AAAAAAAABGDFAAAA|21344|5|55|44|AM|third|night|| +21345|AAAAAAAACGDFAAAA|21345|5|55|45|AM|third|night|| +21346|AAAAAAAADGDFAAAA|21346|5|55|46|AM|third|night|| +21347|AAAAAAAAEGDFAAAA|21347|5|55|47|AM|third|night|| +21348|AAAAAAAAFGDFAAAA|21348|5|55|48|AM|third|night|| +21349|AAAAAAAAGGDFAAAA|21349|5|55|49|AM|third|night|| +21350|AAAAAAAAHGDFAAAA|21350|5|55|50|AM|third|night|| +21351|AAAAAAAAIGDFAAAA|21351|5|55|51|AM|third|night|| +21352|AAAAAAAAJGDFAAAA|21352|5|55|52|AM|third|night|| +21353|AAAAAAAAKGDFAAAA|21353|5|55|53|AM|third|night|| +21354|AAAAAAAALGDFAAAA|21354|5|55|54|AM|third|night|| +21355|AAAAAAAAMGDFAAAA|21355|5|55|55|AM|third|night|| +21356|AAAAAAAANGDFAAAA|21356|5|55|56|AM|third|night|| +21357|AAAAAAAAOGDFAAAA|21357|5|55|57|AM|third|night|| +21358|AAAAAAAAPGDFAAAA|21358|5|55|58|AM|third|night|| +21359|AAAAAAAAAHDFAAAA|21359|5|55|59|AM|third|night|| +21360|AAAAAAAABHDFAAAA|21360|5|56|0|AM|third|night|| +21361|AAAAAAAACHDFAAAA|21361|5|56|1|AM|third|night|| +21362|AAAAAAAADHDFAAAA|21362|5|56|2|AM|third|night|| +21363|AAAAAAAAEHDFAAAA|21363|5|56|3|AM|third|night|| +21364|AAAAAAAAFHDFAAAA|21364|5|56|4|AM|third|night|| +21365|AAAAAAAAGHDFAAAA|21365|5|56|5|AM|third|night|| +21366|AAAAAAAAHHDFAAAA|21366|5|56|6|AM|third|night|| +21367|AAAAAAAAIHDFAAAA|21367|5|56|7|AM|third|night|| +21368|AAAAAAAAJHDFAAAA|21368|5|56|8|AM|third|night|| +21369|AAAAAAAAKHDFAAAA|21369|5|56|9|AM|third|night|| +21370|AAAAAAAALHDFAAAA|21370|5|56|10|AM|third|night|| +21371|AAAAAAAAMHDFAAAA|21371|5|56|11|AM|third|night|| +21372|AAAAAAAANHDFAAAA|21372|5|56|12|AM|third|night|| +21373|AAAAAAAAOHDFAAAA|21373|5|56|13|AM|third|night|| +21374|AAAAAAAAPHDFAAAA|21374|5|56|14|AM|third|night|| +21375|AAAAAAAAAIDFAAAA|21375|5|56|15|AM|third|night|| +21376|AAAAAAAABIDFAAAA|21376|5|56|16|AM|third|night|| +21377|AAAAAAAACIDFAAAA|21377|5|56|17|AM|third|night|| +21378|AAAAAAAADIDFAAAA|21378|5|56|18|AM|third|night|| +21379|AAAAAAAAEIDFAAAA|21379|5|56|19|AM|third|night|| +21380|AAAAAAAAFIDFAAAA|21380|5|56|20|AM|third|night|| +21381|AAAAAAAAGIDFAAAA|21381|5|56|21|AM|third|night|| +21382|AAAAAAAAHIDFAAAA|21382|5|56|22|AM|third|night|| +21383|AAAAAAAAIIDFAAAA|21383|5|56|23|AM|third|night|| +21384|AAAAAAAAJIDFAAAA|21384|5|56|24|AM|third|night|| +21385|AAAAAAAAKIDFAAAA|21385|5|56|25|AM|third|night|| +21386|AAAAAAAALIDFAAAA|21386|5|56|26|AM|third|night|| +21387|AAAAAAAAMIDFAAAA|21387|5|56|27|AM|third|night|| +21388|AAAAAAAANIDFAAAA|21388|5|56|28|AM|third|night|| +21389|AAAAAAAAOIDFAAAA|21389|5|56|29|AM|third|night|| +21390|AAAAAAAAPIDFAAAA|21390|5|56|30|AM|third|night|| +21391|AAAAAAAAAJDFAAAA|21391|5|56|31|AM|third|night|| +21392|AAAAAAAABJDFAAAA|21392|5|56|32|AM|third|night|| +21393|AAAAAAAACJDFAAAA|21393|5|56|33|AM|third|night|| +21394|AAAAAAAADJDFAAAA|21394|5|56|34|AM|third|night|| +21395|AAAAAAAAEJDFAAAA|21395|5|56|35|AM|third|night|| +21396|AAAAAAAAFJDFAAAA|21396|5|56|36|AM|third|night|| +21397|AAAAAAAAGJDFAAAA|21397|5|56|37|AM|third|night|| +21398|AAAAAAAAHJDFAAAA|21398|5|56|38|AM|third|night|| +21399|AAAAAAAAIJDFAAAA|21399|5|56|39|AM|third|night|| +21400|AAAAAAAAJJDFAAAA|21400|5|56|40|AM|third|night|| +21401|AAAAAAAAKJDFAAAA|21401|5|56|41|AM|third|night|| +21402|AAAAAAAALJDFAAAA|21402|5|56|42|AM|third|night|| +21403|AAAAAAAAMJDFAAAA|21403|5|56|43|AM|third|night|| +21404|AAAAAAAANJDFAAAA|21404|5|56|44|AM|third|night|| +21405|AAAAAAAAOJDFAAAA|21405|5|56|45|AM|third|night|| +21406|AAAAAAAAPJDFAAAA|21406|5|56|46|AM|third|night|| +21407|AAAAAAAAAKDFAAAA|21407|5|56|47|AM|third|night|| +21408|AAAAAAAABKDFAAAA|21408|5|56|48|AM|third|night|| +21409|AAAAAAAACKDFAAAA|21409|5|56|49|AM|third|night|| +21410|AAAAAAAADKDFAAAA|21410|5|56|50|AM|third|night|| +21411|AAAAAAAAEKDFAAAA|21411|5|56|51|AM|third|night|| +21412|AAAAAAAAFKDFAAAA|21412|5|56|52|AM|third|night|| +21413|AAAAAAAAGKDFAAAA|21413|5|56|53|AM|third|night|| +21414|AAAAAAAAHKDFAAAA|21414|5|56|54|AM|third|night|| +21415|AAAAAAAAIKDFAAAA|21415|5|56|55|AM|third|night|| +21416|AAAAAAAAJKDFAAAA|21416|5|56|56|AM|third|night|| +21417|AAAAAAAAKKDFAAAA|21417|5|56|57|AM|third|night|| +21418|AAAAAAAALKDFAAAA|21418|5|56|58|AM|third|night|| +21419|AAAAAAAAMKDFAAAA|21419|5|56|59|AM|third|night|| +21420|AAAAAAAANKDFAAAA|21420|5|57|0|AM|third|night|| +21421|AAAAAAAAOKDFAAAA|21421|5|57|1|AM|third|night|| +21422|AAAAAAAAPKDFAAAA|21422|5|57|2|AM|third|night|| +21423|AAAAAAAAALDFAAAA|21423|5|57|3|AM|third|night|| +21424|AAAAAAAABLDFAAAA|21424|5|57|4|AM|third|night|| +21425|AAAAAAAACLDFAAAA|21425|5|57|5|AM|third|night|| +21426|AAAAAAAADLDFAAAA|21426|5|57|6|AM|third|night|| +21427|AAAAAAAAELDFAAAA|21427|5|57|7|AM|third|night|| +21428|AAAAAAAAFLDFAAAA|21428|5|57|8|AM|third|night|| +21429|AAAAAAAAGLDFAAAA|21429|5|57|9|AM|third|night|| +21430|AAAAAAAAHLDFAAAA|21430|5|57|10|AM|third|night|| +21431|AAAAAAAAILDFAAAA|21431|5|57|11|AM|third|night|| +21432|AAAAAAAAJLDFAAAA|21432|5|57|12|AM|third|night|| +21433|AAAAAAAAKLDFAAAA|21433|5|57|13|AM|third|night|| +21434|AAAAAAAALLDFAAAA|21434|5|57|14|AM|third|night|| +21435|AAAAAAAAMLDFAAAA|21435|5|57|15|AM|third|night|| +21436|AAAAAAAANLDFAAAA|21436|5|57|16|AM|third|night|| +21437|AAAAAAAAOLDFAAAA|21437|5|57|17|AM|third|night|| +21438|AAAAAAAAPLDFAAAA|21438|5|57|18|AM|third|night|| +21439|AAAAAAAAAMDFAAAA|21439|5|57|19|AM|third|night|| +21440|AAAAAAAABMDFAAAA|21440|5|57|20|AM|third|night|| +21441|AAAAAAAACMDFAAAA|21441|5|57|21|AM|third|night|| +21442|AAAAAAAADMDFAAAA|21442|5|57|22|AM|third|night|| +21443|AAAAAAAAEMDFAAAA|21443|5|57|23|AM|third|night|| +21444|AAAAAAAAFMDFAAAA|21444|5|57|24|AM|third|night|| +21445|AAAAAAAAGMDFAAAA|21445|5|57|25|AM|third|night|| +21446|AAAAAAAAHMDFAAAA|21446|5|57|26|AM|third|night|| +21447|AAAAAAAAIMDFAAAA|21447|5|57|27|AM|third|night|| +21448|AAAAAAAAJMDFAAAA|21448|5|57|28|AM|third|night|| +21449|AAAAAAAAKMDFAAAA|21449|5|57|29|AM|third|night|| +21450|AAAAAAAALMDFAAAA|21450|5|57|30|AM|third|night|| +21451|AAAAAAAAMMDFAAAA|21451|5|57|31|AM|third|night|| +21452|AAAAAAAANMDFAAAA|21452|5|57|32|AM|third|night|| +21453|AAAAAAAAOMDFAAAA|21453|5|57|33|AM|third|night|| +21454|AAAAAAAAPMDFAAAA|21454|5|57|34|AM|third|night|| +21455|AAAAAAAAANDFAAAA|21455|5|57|35|AM|third|night|| +21456|AAAAAAAABNDFAAAA|21456|5|57|36|AM|third|night|| +21457|AAAAAAAACNDFAAAA|21457|5|57|37|AM|third|night|| +21458|AAAAAAAADNDFAAAA|21458|5|57|38|AM|third|night|| +21459|AAAAAAAAENDFAAAA|21459|5|57|39|AM|third|night|| +21460|AAAAAAAAFNDFAAAA|21460|5|57|40|AM|third|night|| +21461|AAAAAAAAGNDFAAAA|21461|5|57|41|AM|third|night|| +21462|AAAAAAAAHNDFAAAA|21462|5|57|42|AM|third|night|| +21463|AAAAAAAAINDFAAAA|21463|5|57|43|AM|third|night|| +21464|AAAAAAAAJNDFAAAA|21464|5|57|44|AM|third|night|| +21465|AAAAAAAAKNDFAAAA|21465|5|57|45|AM|third|night|| +21466|AAAAAAAALNDFAAAA|21466|5|57|46|AM|third|night|| +21467|AAAAAAAAMNDFAAAA|21467|5|57|47|AM|third|night|| +21468|AAAAAAAANNDFAAAA|21468|5|57|48|AM|third|night|| +21469|AAAAAAAAONDFAAAA|21469|5|57|49|AM|third|night|| +21470|AAAAAAAAPNDFAAAA|21470|5|57|50|AM|third|night|| +21471|AAAAAAAAAODFAAAA|21471|5|57|51|AM|third|night|| +21472|AAAAAAAABODFAAAA|21472|5|57|52|AM|third|night|| +21473|AAAAAAAACODFAAAA|21473|5|57|53|AM|third|night|| +21474|AAAAAAAADODFAAAA|21474|5|57|54|AM|third|night|| +21475|AAAAAAAAEODFAAAA|21475|5|57|55|AM|third|night|| +21476|AAAAAAAAFODFAAAA|21476|5|57|56|AM|third|night|| +21477|AAAAAAAAGODFAAAA|21477|5|57|57|AM|third|night|| +21478|AAAAAAAAHODFAAAA|21478|5|57|58|AM|third|night|| +21479|AAAAAAAAIODFAAAA|21479|5|57|59|AM|third|night|| +21480|AAAAAAAAJODFAAAA|21480|5|58|0|AM|third|night|| +21481|AAAAAAAAKODFAAAA|21481|5|58|1|AM|third|night|| +21482|AAAAAAAALODFAAAA|21482|5|58|2|AM|third|night|| +21483|AAAAAAAAMODFAAAA|21483|5|58|3|AM|third|night|| +21484|AAAAAAAANODFAAAA|21484|5|58|4|AM|third|night|| +21485|AAAAAAAAOODFAAAA|21485|5|58|5|AM|third|night|| +21486|AAAAAAAAPODFAAAA|21486|5|58|6|AM|third|night|| +21487|AAAAAAAAAPDFAAAA|21487|5|58|7|AM|third|night|| +21488|AAAAAAAABPDFAAAA|21488|5|58|8|AM|third|night|| +21489|AAAAAAAACPDFAAAA|21489|5|58|9|AM|third|night|| +21490|AAAAAAAADPDFAAAA|21490|5|58|10|AM|third|night|| +21491|AAAAAAAAEPDFAAAA|21491|5|58|11|AM|third|night|| +21492|AAAAAAAAFPDFAAAA|21492|5|58|12|AM|third|night|| +21493|AAAAAAAAGPDFAAAA|21493|5|58|13|AM|third|night|| +21494|AAAAAAAAHPDFAAAA|21494|5|58|14|AM|third|night|| +21495|AAAAAAAAIPDFAAAA|21495|5|58|15|AM|third|night|| +21496|AAAAAAAAJPDFAAAA|21496|5|58|16|AM|third|night|| +21497|AAAAAAAAKPDFAAAA|21497|5|58|17|AM|third|night|| +21498|AAAAAAAALPDFAAAA|21498|5|58|18|AM|third|night|| +21499|AAAAAAAAMPDFAAAA|21499|5|58|19|AM|third|night|| +21500|AAAAAAAANPDFAAAA|21500|5|58|20|AM|third|night|| +21501|AAAAAAAAOPDFAAAA|21501|5|58|21|AM|third|night|| +21502|AAAAAAAAPPDFAAAA|21502|5|58|22|AM|third|night|| +21503|AAAAAAAAAAEFAAAA|21503|5|58|23|AM|third|night|| +21504|AAAAAAAABAEFAAAA|21504|5|58|24|AM|third|night|| +21505|AAAAAAAACAEFAAAA|21505|5|58|25|AM|third|night|| +21506|AAAAAAAADAEFAAAA|21506|5|58|26|AM|third|night|| +21507|AAAAAAAAEAEFAAAA|21507|5|58|27|AM|third|night|| +21508|AAAAAAAAFAEFAAAA|21508|5|58|28|AM|third|night|| +21509|AAAAAAAAGAEFAAAA|21509|5|58|29|AM|third|night|| +21510|AAAAAAAAHAEFAAAA|21510|5|58|30|AM|third|night|| +21511|AAAAAAAAIAEFAAAA|21511|5|58|31|AM|third|night|| +21512|AAAAAAAAJAEFAAAA|21512|5|58|32|AM|third|night|| +21513|AAAAAAAAKAEFAAAA|21513|5|58|33|AM|third|night|| +21514|AAAAAAAALAEFAAAA|21514|5|58|34|AM|third|night|| +21515|AAAAAAAAMAEFAAAA|21515|5|58|35|AM|third|night|| +21516|AAAAAAAANAEFAAAA|21516|5|58|36|AM|third|night|| +21517|AAAAAAAAOAEFAAAA|21517|5|58|37|AM|third|night|| +21518|AAAAAAAAPAEFAAAA|21518|5|58|38|AM|third|night|| +21519|AAAAAAAAABEFAAAA|21519|5|58|39|AM|third|night|| +21520|AAAAAAAABBEFAAAA|21520|5|58|40|AM|third|night|| +21521|AAAAAAAACBEFAAAA|21521|5|58|41|AM|third|night|| +21522|AAAAAAAADBEFAAAA|21522|5|58|42|AM|third|night|| +21523|AAAAAAAAEBEFAAAA|21523|5|58|43|AM|third|night|| +21524|AAAAAAAAFBEFAAAA|21524|5|58|44|AM|third|night|| +21525|AAAAAAAAGBEFAAAA|21525|5|58|45|AM|third|night|| +21526|AAAAAAAAHBEFAAAA|21526|5|58|46|AM|third|night|| +21527|AAAAAAAAIBEFAAAA|21527|5|58|47|AM|third|night|| +21528|AAAAAAAAJBEFAAAA|21528|5|58|48|AM|third|night|| +21529|AAAAAAAAKBEFAAAA|21529|5|58|49|AM|third|night|| +21530|AAAAAAAALBEFAAAA|21530|5|58|50|AM|third|night|| +21531|AAAAAAAAMBEFAAAA|21531|5|58|51|AM|third|night|| +21532|AAAAAAAANBEFAAAA|21532|5|58|52|AM|third|night|| +21533|AAAAAAAAOBEFAAAA|21533|5|58|53|AM|third|night|| +21534|AAAAAAAAPBEFAAAA|21534|5|58|54|AM|third|night|| +21535|AAAAAAAAACEFAAAA|21535|5|58|55|AM|third|night|| +21536|AAAAAAAABCEFAAAA|21536|5|58|56|AM|third|night|| +21537|AAAAAAAACCEFAAAA|21537|5|58|57|AM|third|night|| +21538|AAAAAAAADCEFAAAA|21538|5|58|58|AM|third|night|| +21539|AAAAAAAAECEFAAAA|21539|5|58|59|AM|third|night|| +21540|AAAAAAAAFCEFAAAA|21540|5|59|0|AM|third|night|| +21541|AAAAAAAAGCEFAAAA|21541|5|59|1|AM|third|night|| +21542|AAAAAAAAHCEFAAAA|21542|5|59|2|AM|third|night|| +21543|AAAAAAAAICEFAAAA|21543|5|59|3|AM|third|night|| +21544|AAAAAAAAJCEFAAAA|21544|5|59|4|AM|third|night|| +21545|AAAAAAAAKCEFAAAA|21545|5|59|5|AM|third|night|| +21546|AAAAAAAALCEFAAAA|21546|5|59|6|AM|third|night|| +21547|AAAAAAAAMCEFAAAA|21547|5|59|7|AM|third|night|| +21548|AAAAAAAANCEFAAAA|21548|5|59|8|AM|third|night|| +21549|AAAAAAAAOCEFAAAA|21549|5|59|9|AM|third|night|| +21550|AAAAAAAAPCEFAAAA|21550|5|59|10|AM|third|night|| +21551|AAAAAAAAADEFAAAA|21551|5|59|11|AM|third|night|| +21552|AAAAAAAABDEFAAAA|21552|5|59|12|AM|third|night|| +21553|AAAAAAAACDEFAAAA|21553|5|59|13|AM|third|night|| +21554|AAAAAAAADDEFAAAA|21554|5|59|14|AM|third|night|| +21555|AAAAAAAAEDEFAAAA|21555|5|59|15|AM|third|night|| +21556|AAAAAAAAFDEFAAAA|21556|5|59|16|AM|third|night|| +21557|AAAAAAAAGDEFAAAA|21557|5|59|17|AM|third|night|| +21558|AAAAAAAAHDEFAAAA|21558|5|59|18|AM|third|night|| +21559|AAAAAAAAIDEFAAAA|21559|5|59|19|AM|third|night|| +21560|AAAAAAAAJDEFAAAA|21560|5|59|20|AM|third|night|| +21561|AAAAAAAAKDEFAAAA|21561|5|59|21|AM|third|night|| +21562|AAAAAAAALDEFAAAA|21562|5|59|22|AM|third|night|| +21563|AAAAAAAAMDEFAAAA|21563|5|59|23|AM|third|night|| +21564|AAAAAAAANDEFAAAA|21564|5|59|24|AM|third|night|| +21565|AAAAAAAAODEFAAAA|21565|5|59|25|AM|third|night|| +21566|AAAAAAAAPDEFAAAA|21566|5|59|26|AM|third|night|| +21567|AAAAAAAAAEEFAAAA|21567|5|59|27|AM|third|night|| +21568|AAAAAAAABEEFAAAA|21568|5|59|28|AM|third|night|| +21569|AAAAAAAACEEFAAAA|21569|5|59|29|AM|third|night|| +21570|AAAAAAAADEEFAAAA|21570|5|59|30|AM|third|night|| +21571|AAAAAAAAEEEFAAAA|21571|5|59|31|AM|third|night|| +21572|AAAAAAAAFEEFAAAA|21572|5|59|32|AM|third|night|| +21573|AAAAAAAAGEEFAAAA|21573|5|59|33|AM|third|night|| +21574|AAAAAAAAHEEFAAAA|21574|5|59|34|AM|third|night|| +21575|AAAAAAAAIEEFAAAA|21575|5|59|35|AM|third|night|| +21576|AAAAAAAAJEEFAAAA|21576|5|59|36|AM|third|night|| +21577|AAAAAAAAKEEFAAAA|21577|5|59|37|AM|third|night|| +21578|AAAAAAAALEEFAAAA|21578|5|59|38|AM|third|night|| +21579|AAAAAAAAMEEFAAAA|21579|5|59|39|AM|third|night|| +21580|AAAAAAAANEEFAAAA|21580|5|59|40|AM|third|night|| +21581|AAAAAAAAOEEFAAAA|21581|5|59|41|AM|third|night|| +21582|AAAAAAAAPEEFAAAA|21582|5|59|42|AM|third|night|| +21583|AAAAAAAAAFEFAAAA|21583|5|59|43|AM|third|night|| +21584|AAAAAAAABFEFAAAA|21584|5|59|44|AM|third|night|| +21585|AAAAAAAACFEFAAAA|21585|5|59|45|AM|third|night|| +21586|AAAAAAAADFEFAAAA|21586|5|59|46|AM|third|night|| +21587|AAAAAAAAEFEFAAAA|21587|5|59|47|AM|third|night|| +21588|AAAAAAAAFFEFAAAA|21588|5|59|48|AM|third|night|| +21589|AAAAAAAAGFEFAAAA|21589|5|59|49|AM|third|night|| +21590|AAAAAAAAHFEFAAAA|21590|5|59|50|AM|third|night|| +21591|AAAAAAAAIFEFAAAA|21591|5|59|51|AM|third|night|| +21592|AAAAAAAAJFEFAAAA|21592|5|59|52|AM|third|night|| +21593|AAAAAAAAKFEFAAAA|21593|5|59|53|AM|third|night|| +21594|AAAAAAAALFEFAAAA|21594|5|59|54|AM|third|night|| +21595|AAAAAAAAMFEFAAAA|21595|5|59|55|AM|third|night|| +21596|AAAAAAAANFEFAAAA|21596|5|59|56|AM|third|night|| +21597|AAAAAAAAOFEFAAAA|21597|5|59|57|AM|third|night|| +21598|AAAAAAAAPFEFAAAA|21598|5|59|58|AM|third|night|| +21599|AAAAAAAAAGEFAAAA|21599|5|59|59|AM|third|night|| +21600|AAAAAAAABGEFAAAA|21600|6|0|0|AM|third|morning|breakfast| +21601|AAAAAAAACGEFAAAA|21601|6|0|1|AM|third|morning|breakfast| +21602|AAAAAAAADGEFAAAA|21602|6|0|2|AM|third|morning|breakfast| +21603|AAAAAAAAEGEFAAAA|21603|6|0|3|AM|third|morning|breakfast| +21604|AAAAAAAAFGEFAAAA|21604|6|0|4|AM|third|morning|breakfast| +21605|AAAAAAAAGGEFAAAA|21605|6|0|5|AM|third|morning|breakfast| +21606|AAAAAAAAHGEFAAAA|21606|6|0|6|AM|third|morning|breakfast| +21607|AAAAAAAAIGEFAAAA|21607|6|0|7|AM|third|morning|breakfast| +21608|AAAAAAAAJGEFAAAA|21608|6|0|8|AM|third|morning|breakfast| +21609|AAAAAAAAKGEFAAAA|21609|6|0|9|AM|third|morning|breakfast| +21610|AAAAAAAALGEFAAAA|21610|6|0|10|AM|third|morning|breakfast| +21611|AAAAAAAAMGEFAAAA|21611|6|0|11|AM|third|morning|breakfast| +21612|AAAAAAAANGEFAAAA|21612|6|0|12|AM|third|morning|breakfast| +21613|AAAAAAAAOGEFAAAA|21613|6|0|13|AM|third|morning|breakfast| +21614|AAAAAAAAPGEFAAAA|21614|6|0|14|AM|third|morning|breakfast| +21615|AAAAAAAAAHEFAAAA|21615|6|0|15|AM|third|morning|breakfast| +21616|AAAAAAAABHEFAAAA|21616|6|0|16|AM|third|morning|breakfast| +21617|AAAAAAAACHEFAAAA|21617|6|0|17|AM|third|morning|breakfast| +21618|AAAAAAAADHEFAAAA|21618|6|0|18|AM|third|morning|breakfast| +21619|AAAAAAAAEHEFAAAA|21619|6|0|19|AM|third|morning|breakfast| +21620|AAAAAAAAFHEFAAAA|21620|6|0|20|AM|third|morning|breakfast| +21621|AAAAAAAAGHEFAAAA|21621|6|0|21|AM|third|morning|breakfast| +21622|AAAAAAAAHHEFAAAA|21622|6|0|22|AM|third|morning|breakfast| +21623|AAAAAAAAIHEFAAAA|21623|6|0|23|AM|third|morning|breakfast| +21624|AAAAAAAAJHEFAAAA|21624|6|0|24|AM|third|morning|breakfast| +21625|AAAAAAAAKHEFAAAA|21625|6|0|25|AM|third|morning|breakfast| +21626|AAAAAAAALHEFAAAA|21626|6|0|26|AM|third|morning|breakfast| +21627|AAAAAAAAMHEFAAAA|21627|6|0|27|AM|third|morning|breakfast| +21628|AAAAAAAANHEFAAAA|21628|6|0|28|AM|third|morning|breakfast| +21629|AAAAAAAAOHEFAAAA|21629|6|0|29|AM|third|morning|breakfast| +21630|AAAAAAAAPHEFAAAA|21630|6|0|30|AM|third|morning|breakfast| +21631|AAAAAAAAAIEFAAAA|21631|6|0|31|AM|third|morning|breakfast| +21632|AAAAAAAABIEFAAAA|21632|6|0|32|AM|third|morning|breakfast| +21633|AAAAAAAACIEFAAAA|21633|6|0|33|AM|third|morning|breakfast| +21634|AAAAAAAADIEFAAAA|21634|6|0|34|AM|third|morning|breakfast| +21635|AAAAAAAAEIEFAAAA|21635|6|0|35|AM|third|morning|breakfast| +21636|AAAAAAAAFIEFAAAA|21636|6|0|36|AM|third|morning|breakfast| +21637|AAAAAAAAGIEFAAAA|21637|6|0|37|AM|third|morning|breakfast| +21638|AAAAAAAAHIEFAAAA|21638|6|0|38|AM|third|morning|breakfast| +21639|AAAAAAAAIIEFAAAA|21639|6|0|39|AM|third|morning|breakfast| +21640|AAAAAAAAJIEFAAAA|21640|6|0|40|AM|third|morning|breakfast| +21641|AAAAAAAAKIEFAAAA|21641|6|0|41|AM|third|morning|breakfast| +21642|AAAAAAAALIEFAAAA|21642|6|0|42|AM|third|morning|breakfast| +21643|AAAAAAAAMIEFAAAA|21643|6|0|43|AM|third|morning|breakfast| +21644|AAAAAAAANIEFAAAA|21644|6|0|44|AM|third|morning|breakfast| +21645|AAAAAAAAOIEFAAAA|21645|6|0|45|AM|third|morning|breakfast| +21646|AAAAAAAAPIEFAAAA|21646|6|0|46|AM|third|morning|breakfast| +21647|AAAAAAAAAJEFAAAA|21647|6|0|47|AM|third|morning|breakfast| +21648|AAAAAAAABJEFAAAA|21648|6|0|48|AM|third|morning|breakfast| +21649|AAAAAAAACJEFAAAA|21649|6|0|49|AM|third|morning|breakfast| +21650|AAAAAAAADJEFAAAA|21650|6|0|50|AM|third|morning|breakfast| +21651|AAAAAAAAEJEFAAAA|21651|6|0|51|AM|third|morning|breakfast| +21652|AAAAAAAAFJEFAAAA|21652|6|0|52|AM|third|morning|breakfast| +21653|AAAAAAAAGJEFAAAA|21653|6|0|53|AM|third|morning|breakfast| +21654|AAAAAAAAHJEFAAAA|21654|6|0|54|AM|third|morning|breakfast| +21655|AAAAAAAAIJEFAAAA|21655|6|0|55|AM|third|morning|breakfast| +21656|AAAAAAAAJJEFAAAA|21656|6|0|56|AM|third|morning|breakfast| +21657|AAAAAAAAKJEFAAAA|21657|6|0|57|AM|third|morning|breakfast| +21658|AAAAAAAALJEFAAAA|21658|6|0|58|AM|third|morning|breakfast| +21659|AAAAAAAAMJEFAAAA|21659|6|0|59|AM|third|morning|breakfast| +21660|AAAAAAAANJEFAAAA|21660|6|1|0|AM|third|morning|breakfast| +21661|AAAAAAAAOJEFAAAA|21661|6|1|1|AM|third|morning|breakfast| +21662|AAAAAAAAPJEFAAAA|21662|6|1|2|AM|third|morning|breakfast| +21663|AAAAAAAAAKEFAAAA|21663|6|1|3|AM|third|morning|breakfast| +21664|AAAAAAAABKEFAAAA|21664|6|1|4|AM|third|morning|breakfast| +21665|AAAAAAAACKEFAAAA|21665|6|1|5|AM|third|morning|breakfast| +21666|AAAAAAAADKEFAAAA|21666|6|1|6|AM|third|morning|breakfast| +21667|AAAAAAAAEKEFAAAA|21667|6|1|7|AM|third|morning|breakfast| +21668|AAAAAAAAFKEFAAAA|21668|6|1|8|AM|third|morning|breakfast| +21669|AAAAAAAAGKEFAAAA|21669|6|1|9|AM|third|morning|breakfast| +21670|AAAAAAAAHKEFAAAA|21670|6|1|10|AM|third|morning|breakfast| +21671|AAAAAAAAIKEFAAAA|21671|6|1|11|AM|third|morning|breakfast| +21672|AAAAAAAAJKEFAAAA|21672|6|1|12|AM|third|morning|breakfast| +21673|AAAAAAAAKKEFAAAA|21673|6|1|13|AM|third|morning|breakfast| +21674|AAAAAAAALKEFAAAA|21674|6|1|14|AM|third|morning|breakfast| +21675|AAAAAAAAMKEFAAAA|21675|6|1|15|AM|third|morning|breakfast| +21676|AAAAAAAANKEFAAAA|21676|6|1|16|AM|third|morning|breakfast| +21677|AAAAAAAAOKEFAAAA|21677|6|1|17|AM|third|morning|breakfast| +21678|AAAAAAAAPKEFAAAA|21678|6|1|18|AM|third|morning|breakfast| +21679|AAAAAAAAALEFAAAA|21679|6|1|19|AM|third|morning|breakfast| +21680|AAAAAAAABLEFAAAA|21680|6|1|20|AM|third|morning|breakfast| +21681|AAAAAAAACLEFAAAA|21681|6|1|21|AM|third|morning|breakfast| +21682|AAAAAAAADLEFAAAA|21682|6|1|22|AM|third|morning|breakfast| +21683|AAAAAAAAELEFAAAA|21683|6|1|23|AM|third|morning|breakfast| +21684|AAAAAAAAFLEFAAAA|21684|6|1|24|AM|third|morning|breakfast| +21685|AAAAAAAAGLEFAAAA|21685|6|1|25|AM|third|morning|breakfast| +21686|AAAAAAAAHLEFAAAA|21686|6|1|26|AM|third|morning|breakfast| +21687|AAAAAAAAILEFAAAA|21687|6|1|27|AM|third|morning|breakfast| +21688|AAAAAAAAJLEFAAAA|21688|6|1|28|AM|third|morning|breakfast| +21689|AAAAAAAAKLEFAAAA|21689|6|1|29|AM|third|morning|breakfast| +21690|AAAAAAAALLEFAAAA|21690|6|1|30|AM|third|morning|breakfast| +21691|AAAAAAAAMLEFAAAA|21691|6|1|31|AM|third|morning|breakfast| +21692|AAAAAAAANLEFAAAA|21692|6|1|32|AM|third|morning|breakfast| +21693|AAAAAAAAOLEFAAAA|21693|6|1|33|AM|third|morning|breakfast| +21694|AAAAAAAAPLEFAAAA|21694|6|1|34|AM|third|morning|breakfast| +21695|AAAAAAAAAMEFAAAA|21695|6|1|35|AM|third|morning|breakfast| +21696|AAAAAAAABMEFAAAA|21696|6|1|36|AM|third|morning|breakfast| +21697|AAAAAAAACMEFAAAA|21697|6|1|37|AM|third|morning|breakfast| +21698|AAAAAAAADMEFAAAA|21698|6|1|38|AM|third|morning|breakfast| +21699|AAAAAAAAEMEFAAAA|21699|6|1|39|AM|third|morning|breakfast| +21700|AAAAAAAAFMEFAAAA|21700|6|1|40|AM|third|morning|breakfast| +21701|AAAAAAAAGMEFAAAA|21701|6|1|41|AM|third|morning|breakfast| +21702|AAAAAAAAHMEFAAAA|21702|6|1|42|AM|third|morning|breakfast| +21703|AAAAAAAAIMEFAAAA|21703|6|1|43|AM|third|morning|breakfast| +21704|AAAAAAAAJMEFAAAA|21704|6|1|44|AM|third|morning|breakfast| +21705|AAAAAAAAKMEFAAAA|21705|6|1|45|AM|third|morning|breakfast| +21706|AAAAAAAALMEFAAAA|21706|6|1|46|AM|third|morning|breakfast| +21707|AAAAAAAAMMEFAAAA|21707|6|1|47|AM|third|morning|breakfast| +21708|AAAAAAAANMEFAAAA|21708|6|1|48|AM|third|morning|breakfast| +21709|AAAAAAAAOMEFAAAA|21709|6|1|49|AM|third|morning|breakfast| +21710|AAAAAAAAPMEFAAAA|21710|6|1|50|AM|third|morning|breakfast| +21711|AAAAAAAAANEFAAAA|21711|6|1|51|AM|third|morning|breakfast| +21712|AAAAAAAABNEFAAAA|21712|6|1|52|AM|third|morning|breakfast| +21713|AAAAAAAACNEFAAAA|21713|6|1|53|AM|third|morning|breakfast| +21714|AAAAAAAADNEFAAAA|21714|6|1|54|AM|third|morning|breakfast| +21715|AAAAAAAAENEFAAAA|21715|6|1|55|AM|third|morning|breakfast| +21716|AAAAAAAAFNEFAAAA|21716|6|1|56|AM|third|morning|breakfast| +21717|AAAAAAAAGNEFAAAA|21717|6|1|57|AM|third|morning|breakfast| +21718|AAAAAAAAHNEFAAAA|21718|6|1|58|AM|third|morning|breakfast| +21719|AAAAAAAAINEFAAAA|21719|6|1|59|AM|third|morning|breakfast| +21720|AAAAAAAAJNEFAAAA|21720|6|2|0|AM|third|morning|breakfast| +21721|AAAAAAAAKNEFAAAA|21721|6|2|1|AM|third|morning|breakfast| +21722|AAAAAAAALNEFAAAA|21722|6|2|2|AM|third|morning|breakfast| +21723|AAAAAAAAMNEFAAAA|21723|6|2|3|AM|third|morning|breakfast| +21724|AAAAAAAANNEFAAAA|21724|6|2|4|AM|third|morning|breakfast| +21725|AAAAAAAAONEFAAAA|21725|6|2|5|AM|third|morning|breakfast| +21726|AAAAAAAAPNEFAAAA|21726|6|2|6|AM|third|morning|breakfast| +21727|AAAAAAAAAOEFAAAA|21727|6|2|7|AM|third|morning|breakfast| +21728|AAAAAAAABOEFAAAA|21728|6|2|8|AM|third|morning|breakfast| +21729|AAAAAAAACOEFAAAA|21729|6|2|9|AM|third|morning|breakfast| +21730|AAAAAAAADOEFAAAA|21730|6|2|10|AM|third|morning|breakfast| +21731|AAAAAAAAEOEFAAAA|21731|6|2|11|AM|third|morning|breakfast| +21732|AAAAAAAAFOEFAAAA|21732|6|2|12|AM|third|morning|breakfast| +21733|AAAAAAAAGOEFAAAA|21733|6|2|13|AM|third|morning|breakfast| +21734|AAAAAAAAHOEFAAAA|21734|6|2|14|AM|third|morning|breakfast| +21735|AAAAAAAAIOEFAAAA|21735|6|2|15|AM|third|morning|breakfast| +21736|AAAAAAAAJOEFAAAA|21736|6|2|16|AM|third|morning|breakfast| +21737|AAAAAAAAKOEFAAAA|21737|6|2|17|AM|third|morning|breakfast| +21738|AAAAAAAALOEFAAAA|21738|6|2|18|AM|third|morning|breakfast| +21739|AAAAAAAAMOEFAAAA|21739|6|2|19|AM|third|morning|breakfast| +21740|AAAAAAAANOEFAAAA|21740|6|2|20|AM|third|morning|breakfast| +21741|AAAAAAAAOOEFAAAA|21741|6|2|21|AM|third|morning|breakfast| +21742|AAAAAAAAPOEFAAAA|21742|6|2|22|AM|third|morning|breakfast| +21743|AAAAAAAAAPEFAAAA|21743|6|2|23|AM|third|morning|breakfast| +21744|AAAAAAAABPEFAAAA|21744|6|2|24|AM|third|morning|breakfast| +21745|AAAAAAAACPEFAAAA|21745|6|2|25|AM|third|morning|breakfast| +21746|AAAAAAAADPEFAAAA|21746|6|2|26|AM|third|morning|breakfast| +21747|AAAAAAAAEPEFAAAA|21747|6|2|27|AM|third|morning|breakfast| +21748|AAAAAAAAFPEFAAAA|21748|6|2|28|AM|third|morning|breakfast| +21749|AAAAAAAAGPEFAAAA|21749|6|2|29|AM|third|morning|breakfast| +21750|AAAAAAAAHPEFAAAA|21750|6|2|30|AM|third|morning|breakfast| +21751|AAAAAAAAIPEFAAAA|21751|6|2|31|AM|third|morning|breakfast| +21752|AAAAAAAAJPEFAAAA|21752|6|2|32|AM|third|morning|breakfast| +21753|AAAAAAAAKPEFAAAA|21753|6|2|33|AM|third|morning|breakfast| +21754|AAAAAAAALPEFAAAA|21754|6|2|34|AM|third|morning|breakfast| +21755|AAAAAAAAMPEFAAAA|21755|6|2|35|AM|third|morning|breakfast| +21756|AAAAAAAANPEFAAAA|21756|6|2|36|AM|third|morning|breakfast| +21757|AAAAAAAAOPEFAAAA|21757|6|2|37|AM|third|morning|breakfast| +21758|AAAAAAAAPPEFAAAA|21758|6|2|38|AM|third|morning|breakfast| +21759|AAAAAAAAAAFFAAAA|21759|6|2|39|AM|third|morning|breakfast| +21760|AAAAAAAABAFFAAAA|21760|6|2|40|AM|third|morning|breakfast| +21761|AAAAAAAACAFFAAAA|21761|6|2|41|AM|third|morning|breakfast| +21762|AAAAAAAADAFFAAAA|21762|6|2|42|AM|third|morning|breakfast| +21763|AAAAAAAAEAFFAAAA|21763|6|2|43|AM|third|morning|breakfast| +21764|AAAAAAAAFAFFAAAA|21764|6|2|44|AM|third|morning|breakfast| +21765|AAAAAAAAGAFFAAAA|21765|6|2|45|AM|third|morning|breakfast| +21766|AAAAAAAAHAFFAAAA|21766|6|2|46|AM|third|morning|breakfast| +21767|AAAAAAAAIAFFAAAA|21767|6|2|47|AM|third|morning|breakfast| +21768|AAAAAAAAJAFFAAAA|21768|6|2|48|AM|third|morning|breakfast| +21769|AAAAAAAAKAFFAAAA|21769|6|2|49|AM|third|morning|breakfast| +21770|AAAAAAAALAFFAAAA|21770|6|2|50|AM|third|morning|breakfast| +21771|AAAAAAAAMAFFAAAA|21771|6|2|51|AM|third|morning|breakfast| +21772|AAAAAAAANAFFAAAA|21772|6|2|52|AM|third|morning|breakfast| +21773|AAAAAAAAOAFFAAAA|21773|6|2|53|AM|third|morning|breakfast| +21774|AAAAAAAAPAFFAAAA|21774|6|2|54|AM|third|morning|breakfast| +21775|AAAAAAAAABFFAAAA|21775|6|2|55|AM|third|morning|breakfast| +21776|AAAAAAAABBFFAAAA|21776|6|2|56|AM|third|morning|breakfast| +21777|AAAAAAAACBFFAAAA|21777|6|2|57|AM|third|morning|breakfast| +21778|AAAAAAAADBFFAAAA|21778|6|2|58|AM|third|morning|breakfast| +21779|AAAAAAAAEBFFAAAA|21779|6|2|59|AM|third|morning|breakfast| +21780|AAAAAAAAFBFFAAAA|21780|6|3|0|AM|third|morning|breakfast| +21781|AAAAAAAAGBFFAAAA|21781|6|3|1|AM|third|morning|breakfast| +21782|AAAAAAAAHBFFAAAA|21782|6|3|2|AM|third|morning|breakfast| +21783|AAAAAAAAIBFFAAAA|21783|6|3|3|AM|third|morning|breakfast| +21784|AAAAAAAAJBFFAAAA|21784|6|3|4|AM|third|morning|breakfast| +21785|AAAAAAAAKBFFAAAA|21785|6|3|5|AM|third|morning|breakfast| +21786|AAAAAAAALBFFAAAA|21786|6|3|6|AM|third|morning|breakfast| +21787|AAAAAAAAMBFFAAAA|21787|6|3|7|AM|third|morning|breakfast| +21788|AAAAAAAANBFFAAAA|21788|6|3|8|AM|third|morning|breakfast| +21789|AAAAAAAAOBFFAAAA|21789|6|3|9|AM|third|morning|breakfast| +21790|AAAAAAAAPBFFAAAA|21790|6|3|10|AM|third|morning|breakfast| +21791|AAAAAAAAACFFAAAA|21791|6|3|11|AM|third|morning|breakfast| +21792|AAAAAAAABCFFAAAA|21792|6|3|12|AM|third|morning|breakfast| +21793|AAAAAAAACCFFAAAA|21793|6|3|13|AM|third|morning|breakfast| +21794|AAAAAAAADCFFAAAA|21794|6|3|14|AM|third|morning|breakfast| +21795|AAAAAAAAECFFAAAA|21795|6|3|15|AM|third|morning|breakfast| +21796|AAAAAAAAFCFFAAAA|21796|6|3|16|AM|third|morning|breakfast| +21797|AAAAAAAAGCFFAAAA|21797|6|3|17|AM|third|morning|breakfast| +21798|AAAAAAAAHCFFAAAA|21798|6|3|18|AM|third|morning|breakfast| +21799|AAAAAAAAICFFAAAA|21799|6|3|19|AM|third|morning|breakfast| +21800|AAAAAAAAJCFFAAAA|21800|6|3|20|AM|third|morning|breakfast| +21801|AAAAAAAAKCFFAAAA|21801|6|3|21|AM|third|morning|breakfast| +21802|AAAAAAAALCFFAAAA|21802|6|3|22|AM|third|morning|breakfast| +21803|AAAAAAAAMCFFAAAA|21803|6|3|23|AM|third|morning|breakfast| +21804|AAAAAAAANCFFAAAA|21804|6|3|24|AM|third|morning|breakfast| +21805|AAAAAAAAOCFFAAAA|21805|6|3|25|AM|third|morning|breakfast| +21806|AAAAAAAAPCFFAAAA|21806|6|3|26|AM|third|morning|breakfast| +21807|AAAAAAAAADFFAAAA|21807|6|3|27|AM|third|morning|breakfast| +21808|AAAAAAAABDFFAAAA|21808|6|3|28|AM|third|morning|breakfast| +21809|AAAAAAAACDFFAAAA|21809|6|3|29|AM|third|morning|breakfast| +21810|AAAAAAAADDFFAAAA|21810|6|3|30|AM|third|morning|breakfast| +21811|AAAAAAAAEDFFAAAA|21811|6|3|31|AM|third|morning|breakfast| +21812|AAAAAAAAFDFFAAAA|21812|6|3|32|AM|third|morning|breakfast| +21813|AAAAAAAAGDFFAAAA|21813|6|3|33|AM|third|morning|breakfast| +21814|AAAAAAAAHDFFAAAA|21814|6|3|34|AM|third|morning|breakfast| +21815|AAAAAAAAIDFFAAAA|21815|6|3|35|AM|third|morning|breakfast| +21816|AAAAAAAAJDFFAAAA|21816|6|3|36|AM|third|morning|breakfast| +21817|AAAAAAAAKDFFAAAA|21817|6|3|37|AM|third|morning|breakfast| +21818|AAAAAAAALDFFAAAA|21818|6|3|38|AM|third|morning|breakfast| +21819|AAAAAAAAMDFFAAAA|21819|6|3|39|AM|third|morning|breakfast| +21820|AAAAAAAANDFFAAAA|21820|6|3|40|AM|third|morning|breakfast| +21821|AAAAAAAAODFFAAAA|21821|6|3|41|AM|third|morning|breakfast| +21822|AAAAAAAAPDFFAAAA|21822|6|3|42|AM|third|morning|breakfast| +21823|AAAAAAAAAEFFAAAA|21823|6|3|43|AM|third|morning|breakfast| +21824|AAAAAAAABEFFAAAA|21824|6|3|44|AM|third|morning|breakfast| +21825|AAAAAAAACEFFAAAA|21825|6|3|45|AM|third|morning|breakfast| +21826|AAAAAAAADEFFAAAA|21826|6|3|46|AM|third|morning|breakfast| +21827|AAAAAAAAEEFFAAAA|21827|6|3|47|AM|third|morning|breakfast| +21828|AAAAAAAAFEFFAAAA|21828|6|3|48|AM|third|morning|breakfast| +21829|AAAAAAAAGEFFAAAA|21829|6|3|49|AM|third|morning|breakfast| +21830|AAAAAAAAHEFFAAAA|21830|6|3|50|AM|third|morning|breakfast| +21831|AAAAAAAAIEFFAAAA|21831|6|3|51|AM|third|morning|breakfast| +21832|AAAAAAAAJEFFAAAA|21832|6|3|52|AM|third|morning|breakfast| +21833|AAAAAAAAKEFFAAAA|21833|6|3|53|AM|third|morning|breakfast| +21834|AAAAAAAALEFFAAAA|21834|6|3|54|AM|third|morning|breakfast| +21835|AAAAAAAAMEFFAAAA|21835|6|3|55|AM|third|morning|breakfast| +21836|AAAAAAAANEFFAAAA|21836|6|3|56|AM|third|morning|breakfast| +21837|AAAAAAAAOEFFAAAA|21837|6|3|57|AM|third|morning|breakfast| +21838|AAAAAAAAPEFFAAAA|21838|6|3|58|AM|third|morning|breakfast| +21839|AAAAAAAAAFFFAAAA|21839|6|3|59|AM|third|morning|breakfast| +21840|AAAAAAAABFFFAAAA|21840|6|4|0|AM|third|morning|breakfast| +21841|AAAAAAAACFFFAAAA|21841|6|4|1|AM|third|morning|breakfast| +21842|AAAAAAAADFFFAAAA|21842|6|4|2|AM|third|morning|breakfast| +21843|AAAAAAAAEFFFAAAA|21843|6|4|3|AM|third|morning|breakfast| +21844|AAAAAAAAFFFFAAAA|21844|6|4|4|AM|third|morning|breakfast| +21845|AAAAAAAAGFFFAAAA|21845|6|4|5|AM|third|morning|breakfast| +21846|AAAAAAAAHFFFAAAA|21846|6|4|6|AM|third|morning|breakfast| +21847|AAAAAAAAIFFFAAAA|21847|6|4|7|AM|third|morning|breakfast| +21848|AAAAAAAAJFFFAAAA|21848|6|4|8|AM|third|morning|breakfast| +21849|AAAAAAAAKFFFAAAA|21849|6|4|9|AM|third|morning|breakfast| +21850|AAAAAAAALFFFAAAA|21850|6|4|10|AM|third|morning|breakfast| +21851|AAAAAAAAMFFFAAAA|21851|6|4|11|AM|third|morning|breakfast| +21852|AAAAAAAANFFFAAAA|21852|6|4|12|AM|third|morning|breakfast| +21853|AAAAAAAAOFFFAAAA|21853|6|4|13|AM|third|morning|breakfast| +21854|AAAAAAAAPFFFAAAA|21854|6|4|14|AM|third|morning|breakfast| +21855|AAAAAAAAAGFFAAAA|21855|6|4|15|AM|third|morning|breakfast| +21856|AAAAAAAABGFFAAAA|21856|6|4|16|AM|third|morning|breakfast| +21857|AAAAAAAACGFFAAAA|21857|6|4|17|AM|third|morning|breakfast| +21858|AAAAAAAADGFFAAAA|21858|6|4|18|AM|third|morning|breakfast| +21859|AAAAAAAAEGFFAAAA|21859|6|4|19|AM|third|morning|breakfast| +21860|AAAAAAAAFGFFAAAA|21860|6|4|20|AM|third|morning|breakfast| +21861|AAAAAAAAGGFFAAAA|21861|6|4|21|AM|third|morning|breakfast| +21862|AAAAAAAAHGFFAAAA|21862|6|4|22|AM|third|morning|breakfast| +21863|AAAAAAAAIGFFAAAA|21863|6|4|23|AM|third|morning|breakfast| +21864|AAAAAAAAJGFFAAAA|21864|6|4|24|AM|third|morning|breakfast| +21865|AAAAAAAAKGFFAAAA|21865|6|4|25|AM|third|morning|breakfast| +21866|AAAAAAAALGFFAAAA|21866|6|4|26|AM|third|morning|breakfast| +21867|AAAAAAAAMGFFAAAA|21867|6|4|27|AM|third|morning|breakfast| +21868|AAAAAAAANGFFAAAA|21868|6|4|28|AM|third|morning|breakfast| +21869|AAAAAAAAOGFFAAAA|21869|6|4|29|AM|third|morning|breakfast| +21870|AAAAAAAAPGFFAAAA|21870|6|4|30|AM|third|morning|breakfast| +21871|AAAAAAAAAHFFAAAA|21871|6|4|31|AM|third|morning|breakfast| +21872|AAAAAAAABHFFAAAA|21872|6|4|32|AM|third|morning|breakfast| +21873|AAAAAAAACHFFAAAA|21873|6|4|33|AM|third|morning|breakfast| +21874|AAAAAAAADHFFAAAA|21874|6|4|34|AM|third|morning|breakfast| +21875|AAAAAAAAEHFFAAAA|21875|6|4|35|AM|third|morning|breakfast| +21876|AAAAAAAAFHFFAAAA|21876|6|4|36|AM|third|morning|breakfast| +21877|AAAAAAAAGHFFAAAA|21877|6|4|37|AM|third|morning|breakfast| +21878|AAAAAAAAHHFFAAAA|21878|6|4|38|AM|third|morning|breakfast| +21879|AAAAAAAAIHFFAAAA|21879|6|4|39|AM|third|morning|breakfast| +21880|AAAAAAAAJHFFAAAA|21880|6|4|40|AM|third|morning|breakfast| +21881|AAAAAAAAKHFFAAAA|21881|6|4|41|AM|third|morning|breakfast| +21882|AAAAAAAALHFFAAAA|21882|6|4|42|AM|third|morning|breakfast| +21883|AAAAAAAAMHFFAAAA|21883|6|4|43|AM|third|morning|breakfast| +21884|AAAAAAAANHFFAAAA|21884|6|4|44|AM|third|morning|breakfast| +21885|AAAAAAAAOHFFAAAA|21885|6|4|45|AM|third|morning|breakfast| +21886|AAAAAAAAPHFFAAAA|21886|6|4|46|AM|third|morning|breakfast| +21887|AAAAAAAAAIFFAAAA|21887|6|4|47|AM|third|morning|breakfast| +21888|AAAAAAAABIFFAAAA|21888|6|4|48|AM|third|morning|breakfast| +21889|AAAAAAAACIFFAAAA|21889|6|4|49|AM|third|morning|breakfast| +21890|AAAAAAAADIFFAAAA|21890|6|4|50|AM|third|morning|breakfast| +21891|AAAAAAAAEIFFAAAA|21891|6|4|51|AM|third|morning|breakfast| +21892|AAAAAAAAFIFFAAAA|21892|6|4|52|AM|third|morning|breakfast| +21893|AAAAAAAAGIFFAAAA|21893|6|4|53|AM|third|morning|breakfast| +21894|AAAAAAAAHIFFAAAA|21894|6|4|54|AM|third|morning|breakfast| +21895|AAAAAAAAIIFFAAAA|21895|6|4|55|AM|third|morning|breakfast| +21896|AAAAAAAAJIFFAAAA|21896|6|4|56|AM|third|morning|breakfast| +21897|AAAAAAAAKIFFAAAA|21897|6|4|57|AM|third|morning|breakfast| +21898|AAAAAAAALIFFAAAA|21898|6|4|58|AM|third|morning|breakfast| +21899|AAAAAAAAMIFFAAAA|21899|6|4|59|AM|third|morning|breakfast| +21900|AAAAAAAANIFFAAAA|21900|6|5|0|AM|third|morning|breakfast| +21901|AAAAAAAAOIFFAAAA|21901|6|5|1|AM|third|morning|breakfast| +21902|AAAAAAAAPIFFAAAA|21902|6|5|2|AM|third|morning|breakfast| +21903|AAAAAAAAAJFFAAAA|21903|6|5|3|AM|third|morning|breakfast| +21904|AAAAAAAABJFFAAAA|21904|6|5|4|AM|third|morning|breakfast| +21905|AAAAAAAACJFFAAAA|21905|6|5|5|AM|third|morning|breakfast| +21906|AAAAAAAADJFFAAAA|21906|6|5|6|AM|third|morning|breakfast| +21907|AAAAAAAAEJFFAAAA|21907|6|5|7|AM|third|morning|breakfast| +21908|AAAAAAAAFJFFAAAA|21908|6|5|8|AM|third|morning|breakfast| +21909|AAAAAAAAGJFFAAAA|21909|6|5|9|AM|third|morning|breakfast| +21910|AAAAAAAAHJFFAAAA|21910|6|5|10|AM|third|morning|breakfast| +21911|AAAAAAAAIJFFAAAA|21911|6|5|11|AM|third|morning|breakfast| +21912|AAAAAAAAJJFFAAAA|21912|6|5|12|AM|third|morning|breakfast| +21913|AAAAAAAAKJFFAAAA|21913|6|5|13|AM|third|morning|breakfast| +21914|AAAAAAAALJFFAAAA|21914|6|5|14|AM|third|morning|breakfast| +21915|AAAAAAAAMJFFAAAA|21915|6|5|15|AM|third|morning|breakfast| +21916|AAAAAAAANJFFAAAA|21916|6|5|16|AM|third|morning|breakfast| +21917|AAAAAAAAOJFFAAAA|21917|6|5|17|AM|third|morning|breakfast| +21918|AAAAAAAAPJFFAAAA|21918|6|5|18|AM|third|morning|breakfast| +21919|AAAAAAAAAKFFAAAA|21919|6|5|19|AM|third|morning|breakfast| +21920|AAAAAAAABKFFAAAA|21920|6|5|20|AM|third|morning|breakfast| +21921|AAAAAAAACKFFAAAA|21921|6|5|21|AM|third|morning|breakfast| +21922|AAAAAAAADKFFAAAA|21922|6|5|22|AM|third|morning|breakfast| +21923|AAAAAAAAEKFFAAAA|21923|6|5|23|AM|third|morning|breakfast| +21924|AAAAAAAAFKFFAAAA|21924|6|5|24|AM|third|morning|breakfast| +21925|AAAAAAAAGKFFAAAA|21925|6|5|25|AM|third|morning|breakfast| +21926|AAAAAAAAHKFFAAAA|21926|6|5|26|AM|third|morning|breakfast| +21927|AAAAAAAAIKFFAAAA|21927|6|5|27|AM|third|morning|breakfast| +21928|AAAAAAAAJKFFAAAA|21928|6|5|28|AM|third|morning|breakfast| +21929|AAAAAAAAKKFFAAAA|21929|6|5|29|AM|third|morning|breakfast| +21930|AAAAAAAALKFFAAAA|21930|6|5|30|AM|third|morning|breakfast| +21931|AAAAAAAAMKFFAAAA|21931|6|5|31|AM|third|morning|breakfast| +21932|AAAAAAAANKFFAAAA|21932|6|5|32|AM|third|morning|breakfast| +21933|AAAAAAAAOKFFAAAA|21933|6|5|33|AM|third|morning|breakfast| +21934|AAAAAAAAPKFFAAAA|21934|6|5|34|AM|third|morning|breakfast| +21935|AAAAAAAAALFFAAAA|21935|6|5|35|AM|third|morning|breakfast| +21936|AAAAAAAABLFFAAAA|21936|6|5|36|AM|third|morning|breakfast| +21937|AAAAAAAACLFFAAAA|21937|6|5|37|AM|third|morning|breakfast| +21938|AAAAAAAADLFFAAAA|21938|6|5|38|AM|third|morning|breakfast| +21939|AAAAAAAAELFFAAAA|21939|6|5|39|AM|third|morning|breakfast| +21940|AAAAAAAAFLFFAAAA|21940|6|5|40|AM|third|morning|breakfast| +21941|AAAAAAAAGLFFAAAA|21941|6|5|41|AM|third|morning|breakfast| +21942|AAAAAAAAHLFFAAAA|21942|6|5|42|AM|third|morning|breakfast| +21943|AAAAAAAAILFFAAAA|21943|6|5|43|AM|third|morning|breakfast| +21944|AAAAAAAAJLFFAAAA|21944|6|5|44|AM|third|morning|breakfast| +21945|AAAAAAAAKLFFAAAA|21945|6|5|45|AM|third|morning|breakfast| +21946|AAAAAAAALLFFAAAA|21946|6|5|46|AM|third|morning|breakfast| +21947|AAAAAAAAMLFFAAAA|21947|6|5|47|AM|third|morning|breakfast| +21948|AAAAAAAANLFFAAAA|21948|6|5|48|AM|third|morning|breakfast| +21949|AAAAAAAAOLFFAAAA|21949|6|5|49|AM|third|morning|breakfast| +21950|AAAAAAAAPLFFAAAA|21950|6|5|50|AM|third|morning|breakfast| +21951|AAAAAAAAAMFFAAAA|21951|6|5|51|AM|third|morning|breakfast| +21952|AAAAAAAABMFFAAAA|21952|6|5|52|AM|third|morning|breakfast| +21953|AAAAAAAACMFFAAAA|21953|6|5|53|AM|third|morning|breakfast| +21954|AAAAAAAADMFFAAAA|21954|6|5|54|AM|third|morning|breakfast| +21955|AAAAAAAAEMFFAAAA|21955|6|5|55|AM|third|morning|breakfast| +21956|AAAAAAAAFMFFAAAA|21956|6|5|56|AM|third|morning|breakfast| +21957|AAAAAAAAGMFFAAAA|21957|6|5|57|AM|third|morning|breakfast| +21958|AAAAAAAAHMFFAAAA|21958|6|5|58|AM|third|morning|breakfast| +21959|AAAAAAAAIMFFAAAA|21959|6|5|59|AM|third|morning|breakfast| +21960|AAAAAAAAJMFFAAAA|21960|6|6|0|AM|third|morning|breakfast| +21961|AAAAAAAAKMFFAAAA|21961|6|6|1|AM|third|morning|breakfast| +21962|AAAAAAAALMFFAAAA|21962|6|6|2|AM|third|morning|breakfast| +21963|AAAAAAAAMMFFAAAA|21963|6|6|3|AM|third|morning|breakfast| +21964|AAAAAAAANMFFAAAA|21964|6|6|4|AM|third|morning|breakfast| +21965|AAAAAAAAOMFFAAAA|21965|6|6|5|AM|third|morning|breakfast| +21966|AAAAAAAAPMFFAAAA|21966|6|6|6|AM|third|morning|breakfast| +21967|AAAAAAAAANFFAAAA|21967|6|6|7|AM|third|morning|breakfast| +21968|AAAAAAAABNFFAAAA|21968|6|6|8|AM|third|morning|breakfast| +21969|AAAAAAAACNFFAAAA|21969|6|6|9|AM|third|morning|breakfast| +21970|AAAAAAAADNFFAAAA|21970|6|6|10|AM|third|morning|breakfast| +21971|AAAAAAAAENFFAAAA|21971|6|6|11|AM|third|morning|breakfast| +21972|AAAAAAAAFNFFAAAA|21972|6|6|12|AM|third|morning|breakfast| +21973|AAAAAAAAGNFFAAAA|21973|6|6|13|AM|third|morning|breakfast| +21974|AAAAAAAAHNFFAAAA|21974|6|6|14|AM|third|morning|breakfast| +21975|AAAAAAAAINFFAAAA|21975|6|6|15|AM|third|morning|breakfast| +21976|AAAAAAAAJNFFAAAA|21976|6|6|16|AM|third|morning|breakfast| +21977|AAAAAAAAKNFFAAAA|21977|6|6|17|AM|third|morning|breakfast| +21978|AAAAAAAALNFFAAAA|21978|6|6|18|AM|third|morning|breakfast| +21979|AAAAAAAAMNFFAAAA|21979|6|6|19|AM|third|morning|breakfast| +21980|AAAAAAAANNFFAAAA|21980|6|6|20|AM|third|morning|breakfast| +21981|AAAAAAAAONFFAAAA|21981|6|6|21|AM|third|morning|breakfast| +21982|AAAAAAAAPNFFAAAA|21982|6|6|22|AM|third|morning|breakfast| +21983|AAAAAAAAAOFFAAAA|21983|6|6|23|AM|third|morning|breakfast| +21984|AAAAAAAABOFFAAAA|21984|6|6|24|AM|third|morning|breakfast| +21985|AAAAAAAACOFFAAAA|21985|6|6|25|AM|third|morning|breakfast| +21986|AAAAAAAADOFFAAAA|21986|6|6|26|AM|third|morning|breakfast| +21987|AAAAAAAAEOFFAAAA|21987|6|6|27|AM|third|morning|breakfast| +21988|AAAAAAAAFOFFAAAA|21988|6|6|28|AM|third|morning|breakfast| +21989|AAAAAAAAGOFFAAAA|21989|6|6|29|AM|third|morning|breakfast| +21990|AAAAAAAAHOFFAAAA|21990|6|6|30|AM|third|morning|breakfast| +21991|AAAAAAAAIOFFAAAA|21991|6|6|31|AM|third|morning|breakfast| +21992|AAAAAAAAJOFFAAAA|21992|6|6|32|AM|third|morning|breakfast| +21993|AAAAAAAAKOFFAAAA|21993|6|6|33|AM|third|morning|breakfast| +21994|AAAAAAAALOFFAAAA|21994|6|6|34|AM|third|morning|breakfast| +21995|AAAAAAAAMOFFAAAA|21995|6|6|35|AM|third|morning|breakfast| +21996|AAAAAAAANOFFAAAA|21996|6|6|36|AM|third|morning|breakfast| +21997|AAAAAAAAOOFFAAAA|21997|6|6|37|AM|third|morning|breakfast| +21998|AAAAAAAAPOFFAAAA|21998|6|6|38|AM|third|morning|breakfast| +21999|AAAAAAAAAPFFAAAA|21999|6|6|39|AM|third|morning|breakfast| +22000|AAAAAAAABPFFAAAA|22000|6|6|40|AM|third|morning|breakfast| +22001|AAAAAAAACPFFAAAA|22001|6|6|41|AM|third|morning|breakfast| +22002|AAAAAAAADPFFAAAA|22002|6|6|42|AM|third|morning|breakfast| +22003|AAAAAAAAEPFFAAAA|22003|6|6|43|AM|third|morning|breakfast| +22004|AAAAAAAAFPFFAAAA|22004|6|6|44|AM|third|morning|breakfast| +22005|AAAAAAAAGPFFAAAA|22005|6|6|45|AM|third|morning|breakfast| +22006|AAAAAAAAHPFFAAAA|22006|6|6|46|AM|third|morning|breakfast| +22007|AAAAAAAAIPFFAAAA|22007|6|6|47|AM|third|morning|breakfast| +22008|AAAAAAAAJPFFAAAA|22008|6|6|48|AM|third|morning|breakfast| +22009|AAAAAAAAKPFFAAAA|22009|6|6|49|AM|third|morning|breakfast| +22010|AAAAAAAALPFFAAAA|22010|6|6|50|AM|third|morning|breakfast| +22011|AAAAAAAAMPFFAAAA|22011|6|6|51|AM|third|morning|breakfast| +22012|AAAAAAAANPFFAAAA|22012|6|6|52|AM|third|morning|breakfast| +22013|AAAAAAAAOPFFAAAA|22013|6|6|53|AM|third|morning|breakfast| +22014|AAAAAAAAPPFFAAAA|22014|6|6|54|AM|third|morning|breakfast| +22015|AAAAAAAAAAGFAAAA|22015|6|6|55|AM|third|morning|breakfast| +22016|AAAAAAAABAGFAAAA|22016|6|6|56|AM|third|morning|breakfast| +22017|AAAAAAAACAGFAAAA|22017|6|6|57|AM|third|morning|breakfast| +22018|AAAAAAAADAGFAAAA|22018|6|6|58|AM|third|morning|breakfast| +22019|AAAAAAAAEAGFAAAA|22019|6|6|59|AM|third|morning|breakfast| +22020|AAAAAAAAFAGFAAAA|22020|6|7|0|AM|third|morning|breakfast| +22021|AAAAAAAAGAGFAAAA|22021|6|7|1|AM|third|morning|breakfast| +22022|AAAAAAAAHAGFAAAA|22022|6|7|2|AM|third|morning|breakfast| +22023|AAAAAAAAIAGFAAAA|22023|6|7|3|AM|third|morning|breakfast| +22024|AAAAAAAAJAGFAAAA|22024|6|7|4|AM|third|morning|breakfast| +22025|AAAAAAAAKAGFAAAA|22025|6|7|5|AM|third|morning|breakfast| +22026|AAAAAAAALAGFAAAA|22026|6|7|6|AM|third|morning|breakfast| +22027|AAAAAAAAMAGFAAAA|22027|6|7|7|AM|third|morning|breakfast| +22028|AAAAAAAANAGFAAAA|22028|6|7|8|AM|third|morning|breakfast| +22029|AAAAAAAAOAGFAAAA|22029|6|7|9|AM|third|morning|breakfast| +22030|AAAAAAAAPAGFAAAA|22030|6|7|10|AM|third|morning|breakfast| +22031|AAAAAAAAABGFAAAA|22031|6|7|11|AM|third|morning|breakfast| +22032|AAAAAAAABBGFAAAA|22032|6|7|12|AM|third|morning|breakfast| +22033|AAAAAAAACBGFAAAA|22033|6|7|13|AM|third|morning|breakfast| +22034|AAAAAAAADBGFAAAA|22034|6|7|14|AM|third|morning|breakfast| +22035|AAAAAAAAEBGFAAAA|22035|6|7|15|AM|third|morning|breakfast| +22036|AAAAAAAAFBGFAAAA|22036|6|7|16|AM|third|morning|breakfast| +22037|AAAAAAAAGBGFAAAA|22037|6|7|17|AM|third|morning|breakfast| +22038|AAAAAAAAHBGFAAAA|22038|6|7|18|AM|third|morning|breakfast| +22039|AAAAAAAAIBGFAAAA|22039|6|7|19|AM|third|morning|breakfast| +22040|AAAAAAAAJBGFAAAA|22040|6|7|20|AM|third|morning|breakfast| +22041|AAAAAAAAKBGFAAAA|22041|6|7|21|AM|third|morning|breakfast| +22042|AAAAAAAALBGFAAAA|22042|6|7|22|AM|third|morning|breakfast| +22043|AAAAAAAAMBGFAAAA|22043|6|7|23|AM|third|morning|breakfast| +22044|AAAAAAAANBGFAAAA|22044|6|7|24|AM|third|morning|breakfast| +22045|AAAAAAAAOBGFAAAA|22045|6|7|25|AM|third|morning|breakfast| +22046|AAAAAAAAPBGFAAAA|22046|6|7|26|AM|third|morning|breakfast| +22047|AAAAAAAAACGFAAAA|22047|6|7|27|AM|third|morning|breakfast| +22048|AAAAAAAABCGFAAAA|22048|6|7|28|AM|third|morning|breakfast| +22049|AAAAAAAACCGFAAAA|22049|6|7|29|AM|third|morning|breakfast| +22050|AAAAAAAADCGFAAAA|22050|6|7|30|AM|third|morning|breakfast| +22051|AAAAAAAAECGFAAAA|22051|6|7|31|AM|third|morning|breakfast| +22052|AAAAAAAAFCGFAAAA|22052|6|7|32|AM|third|morning|breakfast| +22053|AAAAAAAAGCGFAAAA|22053|6|7|33|AM|third|morning|breakfast| +22054|AAAAAAAAHCGFAAAA|22054|6|7|34|AM|third|morning|breakfast| +22055|AAAAAAAAICGFAAAA|22055|6|7|35|AM|third|morning|breakfast| +22056|AAAAAAAAJCGFAAAA|22056|6|7|36|AM|third|morning|breakfast| +22057|AAAAAAAAKCGFAAAA|22057|6|7|37|AM|third|morning|breakfast| +22058|AAAAAAAALCGFAAAA|22058|6|7|38|AM|third|morning|breakfast| +22059|AAAAAAAAMCGFAAAA|22059|6|7|39|AM|third|morning|breakfast| +22060|AAAAAAAANCGFAAAA|22060|6|7|40|AM|third|morning|breakfast| +22061|AAAAAAAAOCGFAAAA|22061|6|7|41|AM|third|morning|breakfast| +22062|AAAAAAAAPCGFAAAA|22062|6|7|42|AM|third|morning|breakfast| +22063|AAAAAAAAADGFAAAA|22063|6|7|43|AM|third|morning|breakfast| +22064|AAAAAAAABDGFAAAA|22064|6|7|44|AM|third|morning|breakfast| +22065|AAAAAAAACDGFAAAA|22065|6|7|45|AM|third|morning|breakfast| +22066|AAAAAAAADDGFAAAA|22066|6|7|46|AM|third|morning|breakfast| +22067|AAAAAAAAEDGFAAAA|22067|6|7|47|AM|third|morning|breakfast| +22068|AAAAAAAAFDGFAAAA|22068|6|7|48|AM|third|morning|breakfast| +22069|AAAAAAAAGDGFAAAA|22069|6|7|49|AM|third|morning|breakfast| +22070|AAAAAAAAHDGFAAAA|22070|6|7|50|AM|third|morning|breakfast| +22071|AAAAAAAAIDGFAAAA|22071|6|7|51|AM|third|morning|breakfast| +22072|AAAAAAAAJDGFAAAA|22072|6|7|52|AM|third|morning|breakfast| +22073|AAAAAAAAKDGFAAAA|22073|6|7|53|AM|third|morning|breakfast| +22074|AAAAAAAALDGFAAAA|22074|6|7|54|AM|third|morning|breakfast| +22075|AAAAAAAAMDGFAAAA|22075|6|7|55|AM|third|morning|breakfast| +22076|AAAAAAAANDGFAAAA|22076|6|7|56|AM|third|morning|breakfast| +22077|AAAAAAAAODGFAAAA|22077|6|7|57|AM|third|morning|breakfast| +22078|AAAAAAAAPDGFAAAA|22078|6|7|58|AM|third|morning|breakfast| +22079|AAAAAAAAAEGFAAAA|22079|6|7|59|AM|third|morning|breakfast| +22080|AAAAAAAABEGFAAAA|22080|6|8|0|AM|third|morning|breakfast| +22081|AAAAAAAACEGFAAAA|22081|6|8|1|AM|third|morning|breakfast| +22082|AAAAAAAADEGFAAAA|22082|6|8|2|AM|third|morning|breakfast| +22083|AAAAAAAAEEGFAAAA|22083|6|8|3|AM|third|morning|breakfast| +22084|AAAAAAAAFEGFAAAA|22084|6|8|4|AM|third|morning|breakfast| +22085|AAAAAAAAGEGFAAAA|22085|6|8|5|AM|third|morning|breakfast| +22086|AAAAAAAAHEGFAAAA|22086|6|8|6|AM|third|morning|breakfast| +22087|AAAAAAAAIEGFAAAA|22087|6|8|7|AM|third|morning|breakfast| +22088|AAAAAAAAJEGFAAAA|22088|6|8|8|AM|third|morning|breakfast| +22089|AAAAAAAAKEGFAAAA|22089|6|8|9|AM|third|morning|breakfast| +22090|AAAAAAAALEGFAAAA|22090|6|8|10|AM|third|morning|breakfast| +22091|AAAAAAAAMEGFAAAA|22091|6|8|11|AM|third|morning|breakfast| +22092|AAAAAAAANEGFAAAA|22092|6|8|12|AM|third|morning|breakfast| +22093|AAAAAAAAOEGFAAAA|22093|6|8|13|AM|third|morning|breakfast| +22094|AAAAAAAAPEGFAAAA|22094|6|8|14|AM|third|morning|breakfast| +22095|AAAAAAAAAFGFAAAA|22095|6|8|15|AM|third|morning|breakfast| +22096|AAAAAAAABFGFAAAA|22096|6|8|16|AM|third|morning|breakfast| +22097|AAAAAAAACFGFAAAA|22097|6|8|17|AM|third|morning|breakfast| +22098|AAAAAAAADFGFAAAA|22098|6|8|18|AM|third|morning|breakfast| +22099|AAAAAAAAEFGFAAAA|22099|6|8|19|AM|third|morning|breakfast| +22100|AAAAAAAAFFGFAAAA|22100|6|8|20|AM|third|morning|breakfast| +22101|AAAAAAAAGFGFAAAA|22101|6|8|21|AM|third|morning|breakfast| +22102|AAAAAAAAHFGFAAAA|22102|6|8|22|AM|third|morning|breakfast| +22103|AAAAAAAAIFGFAAAA|22103|6|8|23|AM|third|morning|breakfast| +22104|AAAAAAAAJFGFAAAA|22104|6|8|24|AM|third|morning|breakfast| +22105|AAAAAAAAKFGFAAAA|22105|6|8|25|AM|third|morning|breakfast| +22106|AAAAAAAALFGFAAAA|22106|6|8|26|AM|third|morning|breakfast| +22107|AAAAAAAAMFGFAAAA|22107|6|8|27|AM|third|morning|breakfast| +22108|AAAAAAAANFGFAAAA|22108|6|8|28|AM|third|morning|breakfast| +22109|AAAAAAAAOFGFAAAA|22109|6|8|29|AM|third|morning|breakfast| +22110|AAAAAAAAPFGFAAAA|22110|6|8|30|AM|third|morning|breakfast| +22111|AAAAAAAAAGGFAAAA|22111|6|8|31|AM|third|morning|breakfast| +22112|AAAAAAAABGGFAAAA|22112|6|8|32|AM|third|morning|breakfast| +22113|AAAAAAAACGGFAAAA|22113|6|8|33|AM|third|morning|breakfast| +22114|AAAAAAAADGGFAAAA|22114|6|8|34|AM|third|morning|breakfast| +22115|AAAAAAAAEGGFAAAA|22115|6|8|35|AM|third|morning|breakfast| +22116|AAAAAAAAFGGFAAAA|22116|6|8|36|AM|third|morning|breakfast| +22117|AAAAAAAAGGGFAAAA|22117|6|8|37|AM|third|morning|breakfast| +22118|AAAAAAAAHGGFAAAA|22118|6|8|38|AM|third|morning|breakfast| +22119|AAAAAAAAIGGFAAAA|22119|6|8|39|AM|third|morning|breakfast| +22120|AAAAAAAAJGGFAAAA|22120|6|8|40|AM|third|morning|breakfast| +22121|AAAAAAAAKGGFAAAA|22121|6|8|41|AM|third|morning|breakfast| +22122|AAAAAAAALGGFAAAA|22122|6|8|42|AM|third|morning|breakfast| +22123|AAAAAAAAMGGFAAAA|22123|6|8|43|AM|third|morning|breakfast| +22124|AAAAAAAANGGFAAAA|22124|6|8|44|AM|third|morning|breakfast| +22125|AAAAAAAAOGGFAAAA|22125|6|8|45|AM|third|morning|breakfast| +22126|AAAAAAAAPGGFAAAA|22126|6|8|46|AM|third|morning|breakfast| +22127|AAAAAAAAAHGFAAAA|22127|6|8|47|AM|third|morning|breakfast| +22128|AAAAAAAABHGFAAAA|22128|6|8|48|AM|third|morning|breakfast| +22129|AAAAAAAACHGFAAAA|22129|6|8|49|AM|third|morning|breakfast| +22130|AAAAAAAADHGFAAAA|22130|6|8|50|AM|third|morning|breakfast| +22131|AAAAAAAAEHGFAAAA|22131|6|8|51|AM|third|morning|breakfast| +22132|AAAAAAAAFHGFAAAA|22132|6|8|52|AM|third|morning|breakfast| +22133|AAAAAAAAGHGFAAAA|22133|6|8|53|AM|third|morning|breakfast| +22134|AAAAAAAAHHGFAAAA|22134|6|8|54|AM|third|morning|breakfast| +22135|AAAAAAAAIHGFAAAA|22135|6|8|55|AM|third|morning|breakfast| +22136|AAAAAAAAJHGFAAAA|22136|6|8|56|AM|third|morning|breakfast| +22137|AAAAAAAAKHGFAAAA|22137|6|8|57|AM|third|morning|breakfast| +22138|AAAAAAAALHGFAAAA|22138|6|8|58|AM|third|morning|breakfast| +22139|AAAAAAAAMHGFAAAA|22139|6|8|59|AM|third|morning|breakfast| +22140|AAAAAAAANHGFAAAA|22140|6|9|0|AM|third|morning|breakfast| +22141|AAAAAAAAOHGFAAAA|22141|6|9|1|AM|third|morning|breakfast| +22142|AAAAAAAAPHGFAAAA|22142|6|9|2|AM|third|morning|breakfast| +22143|AAAAAAAAAIGFAAAA|22143|6|9|3|AM|third|morning|breakfast| +22144|AAAAAAAABIGFAAAA|22144|6|9|4|AM|third|morning|breakfast| +22145|AAAAAAAACIGFAAAA|22145|6|9|5|AM|third|morning|breakfast| +22146|AAAAAAAADIGFAAAA|22146|6|9|6|AM|third|morning|breakfast| +22147|AAAAAAAAEIGFAAAA|22147|6|9|7|AM|third|morning|breakfast| +22148|AAAAAAAAFIGFAAAA|22148|6|9|8|AM|third|morning|breakfast| +22149|AAAAAAAAGIGFAAAA|22149|6|9|9|AM|third|morning|breakfast| +22150|AAAAAAAAHIGFAAAA|22150|6|9|10|AM|third|morning|breakfast| +22151|AAAAAAAAIIGFAAAA|22151|6|9|11|AM|third|morning|breakfast| +22152|AAAAAAAAJIGFAAAA|22152|6|9|12|AM|third|morning|breakfast| +22153|AAAAAAAAKIGFAAAA|22153|6|9|13|AM|third|morning|breakfast| +22154|AAAAAAAALIGFAAAA|22154|6|9|14|AM|third|morning|breakfast| +22155|AAAAAAAAMIGFAAAA|22155|6|9|15|AM|third|morning|breakfast| +22156|AAAAAAAANIGFAAAA|22156|6|9|16|AM|third|morning|breakfast| +22157|AAAAAAAAOIGFAAAA|22157|6|9|17|AM|third|morning|breakfast| +22158|AAAAAAAAPIGFAAAA|22158|6|9|18|AM|third|morning|breakfast| +22159|AAAAAAAAAJGFAAAA|22159|6|9|19|AM|third|morning|breakfast| +22160|AAAAAAAABJGFAAAA|22160|6|9|20|AM|third|morning|breakfast| +22161|AAAAAAAACJGFAAAA|22161|6|9|21|AM|third|morning|breakfast| +22162|AAAAAAAADJGFAAAA|22162|6|9|22|AM|third|morning|breakfast| +22163|AAAAAAAAEJGFAAAA|22163|6|9|23|AM|third|morning|breakfast| +22164|AAAAAAAAFJGFAAAA|22164|6|9|24|AM|third|morning|breakfast| +22165|AAAAAAAAGJGFAAAA|22165|6|9|25|AM|third|morning|breakfast| +22166|AAAAAAAAHJGFAAAA|22166|6|9|26|AM|third|morning|breakfast| +22167|AAAAAAAAIJGFAAAA|22167|6|9|27|AM|third|morning|breakfast| +22168|AAAAAAAAJJGFAAAA|22168|6|9|28|AM|third|morning|breakfast| +22169|AAAAAAAAKJGFAAAA|22169|6|9|29|AM|third|morning|breakfast| +22170|AAAAAAAALJGFAAAA|22170|6|9|30|AM|third|morning|breakfast| +22171|AAAAAAAAMJGFAAAA|22171|6|9|31|AM|third|morning|breakfast| +22172|AAAAAAAANJGFAAAA|22172|6|9|32|AM|third|morning|breakfast| +22173|AAAAAAAAOJGFAAAA|22173|6|9|33|AM|third|morning|breakfast| +22174|AAAAAAAAPJGFAAAA|22174|6|9|34|AM|third|morning|breakfast| +22175|AAAAAAAAAKGFAAAA|22175|6|9|35|AM|third|morning|breakfast| +22176|AAAAAAAABKGFAAAA|22176|6|9|36|AM|third|morning|breakfast| +22177|AAAAAAAACKGFAAAA|22177|6|9|37|AM|third|morning|breakfast| +22178|AAAAAAAADKGFAAAA|22178|6|9|38|AM|third|morning|breakfast| +22179|AAAAAAAAEKGFAAAA|22179|6|9|39|AM|third|morning|breakfast| +22180|AAAAAAAAFKGFAAAA|22180|6|9|40|AM|third|morning|breakfast| +22181|AAAAAAAAGKGFAAAA|22181|6|9|41|AM|third|morning|breakfast| +22182|AAAAAAAAHKGFAAAA|22182|6|9|42|AM|third|morning|breakfast| +22183|AAAAAAAAIKGFAAAA|22183|6|9|43|AM|third|morning|breakfast| +22184|AAAAAAAAJKGFAAAA|22184|6|9|44|AM|third|morning|breakfast| +22185|AAAAAAAAKKGFAAAA|22185|6|9|45|AM|third|morning|breakfast| +22186|AAAAAAAALKGFAAAA|22186|6|9|46|AM|third|morning|breakfast| +22187|AAAAAAAAMKGFAAAA|22187|6|9|47|AM|third|morning|breakfast| +22188|AAAAAAAANKGFAAAA|22188|6|9|48|AM|third|morning|breakfast| +22189|AAAAAAAAOKGFAAAA|22189|6|9|49|AM|third|morning|breakfast| +22190|AAAAAAAAPKGFAAAA|22190|6|9|50|AM|third|morning|breakfast| +22191|AAAAAAAAALGFAAAA|22191|6|9|51|AM|third|morning|breakfast| +22192|AAAAAAAABLGFAAAA|22192|6|9|52|AM|third|morning|breakfast| +22193|AAAAAAAACLGFAAAA|22193|6|9|53|AM|third|morning|breakfast| +22194|AAAAAAAADLGFAAAA|22194|6|9|54|AM|third|morning|breakfast| +22195|AAAAAAAAELGFAAAA|22195|6|9|55|AM|third|morning|breakfast| +22196|AAAAAAAAFLGFAAAA|22196|6|9|56|AM|third|morning|breakfast| +22197|AAAAAAAAGLGFAAAA|22197|6|9|57|AM|third|morning|breakfast| +22198|AAAAAAAAHLGFAAAA|22198|6|9|58|AM|third|morning|breakfast| +22199|AAAAAAAAILGFAAAA|22199|6|9|59|AM|third|morning|breakfast| +22200|AAAAAAAAJLGFAAAA|22200|6|10|0|AM|third|morning|breakfast| +22201|AAAAAAAAKLGFAAAA|22201|6|10|1|AM|third|morning|breakfast| +22202|AAAAAAAALLGFAAAA|22202|6|10|2|AM|third|morning|breakfast| +22203|AAAAAAAAMLGFAAAA|22203|6|10|3|AM|third|morning|breakfast| +22204|AAAAAAAANLGFAAAA|22204|6|10|4|AM|third|morning|breakfast| +22205|AAAAAAAAOLGFAAAA|22205|6|10|5|AM|third|morning|breakfast| +22206|AAAAAAAAPLGFAAAA|22206|6|10|6|AM|third|morning|breakfast| +22207|AAAAAAAAAMGFAAAA|22207|6|10|7|AM|third|morning|breakfast| +22208|AAAAAAAABMGFAAAA|22208|6|10|8|AM|third|morning|breakfast| +22209|AAAAAAAACMGFAAAA|22209|6|10|9|AM|third|morning|breakfast| +22210|AAAAAAAADMGFAAAA|22210|6|10|10|AM|third|morning|breakfast| +22211|AAAAAAAAEMGFAAAA|22211|6|10|11|AM|third|morning|breakfast| +22212|AAAAAAAAFMGFAAAA|22212|6|10|12|AM|third|morning|breakfast| +22213|AAAAAAAAGMGFAAAA|22213|6|10|13|AM|third|morning|breakfast| +22214|AAAAAAAAHMGFAAAA|22214|6|10|14|AM|third|morning|breakfast| +22215|AAAAAAAAIMGFAAAA|22215|6|10|15|AM|third|morning|breakfast| +22216|AAAAAAAAJMGFAAAA|22216|6|10|16|AM|third|morning|breakfast| +22217|AAAAAAAAKMGFAAAA|22217|6|10|17|AM|third|morning|breakfast| +22218|AAAAAAAALMGFAAAA|22218|6|10|18|AM|third|morning|breakfast| +22219|AAAAAAAAMMGFAAAA|22219|6|10|19|AM|third|morning|breakfast| +22220|AAAAAAAANMGFAAAA|22220|6|10|20|AM|third|morning|breakfast| +22221|AAAAAAAAOMGFAAAA|22221|6|10|21|AM|third|morning|breakfast| +22222|AAAAAAAAPMGFAAAA|22222|6|10|22|AM|third|morning|breakfast| +22223|AAAAAAAAANGFAAAA|22223|6|10|23|AM|third|morning|breakfast| +22224|AAAAAAAABNGFAAAA|22224|6|10|24|AM|third|morning|breakfast| +22225|AAAAAAAACNGFAAAA|22225|6|10|25|AM|third|morning|breakfast| +22226|AAAAAAAADNGFAAAA|22226|6|10|26|AM|third|morning|breakfast| +22227|AAAAAAAAENGFAAAA|22227|6|10|27|AM|third|morning|breakfast| +22228|AAAAAAAAFNGFAAAA|22228|6|10|28|AM|third|morning|breakfast| +22229|AAAAAAAAGNGFAAAA|22229|6|10|29|AM|third|morning|breakfast| +22230|AAAAAAAAHNGFAAAA|22230|6|10|30|AM|third|morning|breakfast| +22231|AAAAAAAAINGFAAAA|22231|6|10|31|AM|third|morning|breakfast| +22232|AAAAAAAAJNGFAAAA|22232|6|10|32|AM|third|morning|breakfast| +22233|AAAAAAAAKNGFAAAA|22233|6|10|33|AM|third|morning|breakfast| +22234|AAAAAAAALNGFAAAA|22234|6|10|34|AM|third|morning|breakfast| +22235|AAAAAAAAMNGFAAAA|22235|6|10|35|AM|third|morning|breakfast| +22236|AAAAAAAANNGFAAAA|22236|6|10|36|AM|third|morning|breakfast| +22237|AAAAAAAAONGFAAAA|22237|6|10|37|AM|third|morning|breakfast| +22238|AAAAAAAAPNGFAAAA|22238|6|10|38|AM|third|morning|breakfast| +22239|AAAAAAAAAOGFAAAA|22239|6|10|39|AM|third|morning|breakfast| +22240|AAAAAAAABOGFAAAA|22240|6|10|40|AM|third|morning|breakfast| +22241|AAAAAAAACOGFAAAA|22241|6|10|41|AM|third|morning|breakfast| +22242|AAAAAAAADOGFAAAA|22242|6|10|42|AM|third|morning|breakfast| +22243|AAAAAAAAEOGFAAAA|22243|6|10|43|AM|third|morning|breakfast| +22244|AAAAAAAAFOGFAAAA|22244|6|10|44|AM|third|morning|breakfast| +22245|AAAAAAAAGOGFAAAA|22245|6|10|45|AM|third|morning|breakfast| +22246|AAAAAAAAHOGFAAAA|22246|6|10|46|AM|third|morning|breakfast| +22247|AAAAAAAAIOGFAAAA|22247|6|10|47|AM|third|morning|breakfast| +22248|AAAAAAAAJOGFAAAA|22248|6|10|48|AM|third|morning|breakfast| +22249|AAAAAAAAKOGFAAAA|22249|6|10|49|AM|third|morning|breakfast| +22250|AAAAAAAALOGFAAAA|22250|6|10|50|AM|third|morning|breakfast| +22251|AAAAAAAAMOGFAAAA|22251|6|10|51|AM|third|morning|breakfast| +22252|AAAAAAAANOGFAAAA|22252|6|10|52|AM|third|morning|breakfast| +22253|AAAAAAAAOOGFAAAA|22253|6|10|53|AM|third|morning|breakfast| +22254|AAAAAAAAPOGFAAAA|22254|6|10|54|AM|third|morning|breakfast| +22255|AAAAAAAAAPGFAAAA|22255|6|10|55|AM|third|morning|breakfast| +22256|AAAAAAAABPGFAAAA|22256|6|10|56|AM|third|morning|breakfast| +22257|AAAAAAAACPGFAAAA|22257|6|10|57|AM|third|morning|breakfast| +22258|AAAAAAAADPGFAAAA|22258|6|10|58|AM|third|morning|breakfast| +22259|AAAAAAAAEPGFAAAA|22259|6|10|59|AM|third|morning|breakfast| +22260|AAAAAAAAFPGFAAAA|22260|6|11|0|AM|third|morning|breakfast| +22261|AAAAAAAAGPGFAAAA|22261|6|11|1|AM|third|morning|breakfast| +22262|AAAAAAAAHPGFAAAA|22262|6|11|2|AM|third|morning|breakfast| +22263|AAAAAAAAIPGFAAAA|22263|6|11|3|AM|third|morning|breakfast| +22264|AAAAAAAAJPGFAAAA|22264|6|11|4|AM|third|morning|breakfast| +22265|AAAAAAAAKPGFAAAA|22265|6|11|5|AM|third|morning|breakfast| +22266|AAAAAAAALPGFAAAA|22266|6|11|6|AM|third|morning|breakfast| +22267|AAAAAAAAMPGFAAAA|22267|6|11|7|AM|third|morning|breakfast| +22268|AAAAAAAANPGFAAAA|22268|6|11|8|AM|third|morning|breakfast| +22269|AAAAAAAAOPGFAAAA|22269|6|11|9|AM|third|morning|breakfast| +22270|AAAAAAAAPPGFAAAA|22270|6|11|10|AM|third|morning|breakfast| +22271|AAAAAAAAAAHFAAAA|22271|6|11|11|AM|third|morning|breakfast| +22272|AAAAAAAABAHFAAAA|22272|6|11|12|AM|third|morning|breakfast| +22273|AAAAAAAACAHFAAAA|22273|6|11|13|AM|third|morning|breakfast| +22274|AAAAAAAADAHFAAAA|22274|6|11|14|AM|third|morning|breakfast| +22275|AAAAAAAAEAHFAAAA|22275|6|11|15|AM|third|morning|breakfast| +22276|AAAAAAAAFAHFAAAA|22276|6|11|16|AM|third|morning|breakfast| +22277|AAAAAAAAGAHFAAAA|22277|6|11|17|AM|third|morning|breakfast| +22278|AAAAAAAAHAHFAAAA|22278|6|11|18|AM|third|morning|breakfast| +22279|AAAAAAAAIAHFAAAA|22279|6|11|19|AM|third|morning|breakfast| +22280|AAAAAAAAJAHFAAAA|22280|6|11|20|AM|third|morning|breakfast| +22281|AAAAAAAAKAHFAAAA|22281|6|11|21|AM|third|morning|breakfast| +22282|AAAAAAAALAHFAAAA|22282|6|11|22|AM|third|morning|breakfast| +22283|AAAAAAAAMAHFAAAA|22283|6|11|23|AM|third|morning|breakfast| +22284|AAAAAAAANAHFAAAA|22284|6|11|24|AM|third|morning|breakfast| +22285|AAAAAAAAOAHFAAAA|22285|6|11|25|AM|third|morning|breakfast| +22286|AAAAAAAAPAHFAAAA|22286|6|11|26|AM|third|morning|breakfast| +22287|AAAAAAAAABHFAAAA|22287|6|11|27|AM|third|morning|breakfast| +22288|AAAAAAAABBHFAAAA|22288|6|11|28|AM|third|morning|breakfast| +22289|AAAAAAAACBHFAAAA|22289|6|11|29|AM|third|morning|breakfast| +22290|AAAAAAAADBHFAAAA|22290|6|11|30|AM|third|morning|breakfast| +22291|AAAAAAAAEBHFAAAA|22291|6|11|31|AM|third|morning|breakfast| +22292|AAAAAAAAFBHFAAAA|22292|6|11|32|AM|third|morning|breakfast| +22293|AAAAAAAAGBHFAAAA|22293|6|11|33|AM|third|morning|breakfast| +22294|AAAAAAAAHBHFAAAA|22294|6|11|34|AM|third|morning|breakfast| +22295|AAAAAAAAIBHFAAAA|22295|6|11|35|AM|third|morning|breakfast| +22296|AAAAAAAAJBHFAAAA|22296|6|11|36|AM|third|morning|breakfast| +22297|AAAAAAAAKBHFAAAA|22297|6|11|37|AM|third|morning|breakfast| +22298|AAAAAAAALBHFAAAA|22298|6|11|38|AM|third|morning|breakfast| +22299|AAAAAAAAMBHFAAAA|22299|6|11|39|AM|third|morning|breakfast| +22300|AAAAAAAANBHFAAAA|22300|6|11|40|AM|third|morning|breakfast| +22301|AAAAAAAAOBHFAAAA|22301|6|11|41|AM|third|morning|breakfast| +22302|AAAAAAAAPBHFAAAA|22302|6|11|42|AM|third|morning|breakfast| +22303|AAAAAAAAACHFAAAA|22303|6|11|43|AM|third|morning|breakfast| +22304|AAAAAAAABCHFAAAA|22304|6|11|44|AM|third|morning|breakfast| +22305|AAAAAAAACCHFAAAA|22305|6|11|45|AM|third|morning|breakfast| +22306|AAAAAAAADCHFAAAA|22306|6|11|46|AM|third|morning|breakfast| +22307|AAAAAAAAECHFAAAA|22307|6|11|47|AM|third|morning|breakfast| +22308|AAAAAAAAFCHFAAAA|22308|6|11|48|AM|third|morning|breakfast| +22309|AAAAAAAAGCHFAAAA|22309|6|11|49|AM|third|morning|breakfast| +22310|AAAAAAAAHCHFAAAA|22310|6|11|50|AM|third|morning|breakfast| +22311|AAAAAAAAICHFAAAA|22311|6|11|51|AM|third|morning|breakfast| +22312|AAAAAAAAJCHFAAAA|22312|6|11|52|AM|third|morning|breakfast| +22313|AAAAAAAAKCHFAAAA|22313|6|11|53|AM|third|morning|breakfast| +22314|AAAAAAAALCHFAAAA|22314|6|11|54|AM|third|morning|breakfast| +22315|AAAAAAAAMCHFAAAA|22315|6|11|55|AM|third|morning|breakfast| +22316|AAAAAAAANCHFAAAA|22316|6|11|56|AM|third|morning|breakfast| +22317|AAAAAAAAOCHFAAAA|22317|6|11|57|AM|third|morning|breakfast| +22318|AAAAAAAAPCHFAAAA|22318|6|11|58|AM|third|morning|breakfast| +22319|AAAAAAAAADHFAAAA|22319|6|11|59|AM|third|morning|breakfast| +22320|AAAAAAAABDHFAAAA|22320|6|12|0|AM|third|morning|breakfast| +22321|AAAAAAAACDHFAAAA|22321|6|12|1|AM|third|morning|breakfast| +22322|AAAAAAAADDHFAAAA|22322|6|12|2|AM|third|morning|breakfast| +22323|AAAAAAAAEDHFAAAA|22323|6|12|3|AM|third|morning|breakfast| +22324|AAAAAAAAFDHFAAAA|22324|6|12|4|AM|third|morning|breakfast| +22325|AAAAAAAAGDHFAAAA|22325|6|12|5|AM|third|morning|breakfast| +22326|AAAAAAAAHDHFAAAA|22326|6|12|6|AM|third|morning|breakfast| +22327|AAAAAAAAIDHFAAAA|22327|6|12|7|AM|third|morning|breakfast| +22328|AAAAAAAAJDHFAAAA|22328|6|12|8|AM|third|morning|breakfast| +22329|AAAAAAAAKDHFAAAA|22329|6|12|9|AM|third|morning|breakfast| +22330|AAAAAAAALDHFAAAA|22330|6|12|10|AM|third|morning|breakfast| +22331|AAAAAAAAMDHFAAAA|22331|6|12|11|AM|third|morning|breakfast| +22332|AAAAAAAANDHFAAAA|22332|6|12|12|AM|third|morning|breakfast| +22333|AAAAAAAAODHFAAAA|22333|6|12|13|AM|third|morning|breakfast| +22334|AAAAAAAAPDHFAAAA|22334|6|12|14|AM|third|morning|breakfast| +22335|AAAAAAAAAEHFAAAA|22335|6|12|15|AM|third|morning|breakfast| +22336|AAAAAAAABEHFAAAA|22336|6|12|16|AM|third|morning|breakfast| +22337|AAAAAAAACEHFAAAA|22337|6|12|17|AM|third|morning|breakfast| +22338|AAAAAAAADEHFAAAA|22338|6|12|18|AM|third|morning|breakfast| +22339|AAAAAAAAEEHFAAAA|22339|6|12|19|AM|third|morning|breakfast| +22340|AAAAAAAAFEHFAAAA|22340|6|12|20|AM|third|morning|breakfast| +22341|AAAAAAAAGEHFAAAA|22341|6|12|21|AM|third|morning|breakfast| +22342|AAAAAAAAHEHFAAAA|22342|6|12|22|AM|third|morning|breakfast| +22343|AAAAAAAAIEHFAAAA|22343|6|12|23|AM|third|morning|breakfast| +22344|AAAAAAAAJEHFAAAA|22344|6|12|24|AM|third|morning|breakfast| +22345|AAAAAAAAKEHFAAAA|22345|6|12|25|AM|third|morning|breakfast| +22346|AAAAAAAALEHFAAAA|22346|6|12|26|AM|third|morning|breakfast| +22347|AAAAAAAAMEHFAAAA|22347|6|12|27|AM|third|morning|breakfast| +22348|AAAAAAAANEHFAAAA|22348|6|12|28|AM|third|morning|breakfast| +22349|AAAAAAAAOEHFAAAA|22349|6|12|29|AM|third|morning|breakfast| +22350|AAAAAAAAPEHFAAAA|22350|6|12|30|AM|third|morning|breakfast| +22351|AAAAAAAAAFHFAAAA|22351|6|12|31|AM|third|morning|breakfast| +22352|AAAAAAAABFHFAAAA|22352|6|12|32|AM|third|morning|breakfast| +22353|AAAAAAAACFHFAAAA|22353|6|12|33|AM|third|morning|breakfast| +22354|AAAAAAAADFHFAAAA|22354|6|12|34|AM|third|morning|breakfast| +22355|AAAAAAAAEFHFAAAA|22355|6|12|35|AM|third|morning|breakfast| +22356|AAAAAAAAFFHFAAAA|22356|6|12|36|AM|third|morning|breakfast| +22357|AAAAAAAAGFHFAAAA|22357|6|12|37|AM|third|morning|breakfast| +22358|AAAAAAAAHFHFAAAA|22358|6|12|38|AM|third|morning|breakfast| +22359|AAAAAAAAIFHFAAAA|22359|6|12|39|AM|third|morning|breakfast| +22360|AAAAAAAAJFHFAAAA|22360|6|12|40|AM|third|morning|breakfast| +22361|AAAAAAAAKFHFAAAA|22361|6|12|41|AM|third|morning|breakfast| +22362|AAAAAAAALFHFAAAA|22362|6|12|42|AM|third|morning|breakfast| +22363|AAAAAAAAMFHFAAAA|22363|6|12|43|AM|third|morning|breakfast| +22364|AAAAAAAANFHFAAAA|22364|6|12|44|AM|third|morning|breakfast| +22365|AAAAAAAAOFHFAAAA|22365|6|12|45|AM|third|morning|breakfast| +22366|AAAAAAAAPFHFAAAA|22366|6|12|46|AM|third|morning|breakfast| +22367|AAAAAAAAAGHFAAAA|22367|6|12|47|AM|third|morning|breakfast| +22368|AAAAAAAABGHFAAAA|22368|6|12|48|AM|third|morning|breakfast| +22369|AAAAAAAACGHFAAAA|22369|6|12|49|AM|third|morning|breakfast| +22370|AAAAAAAADGHFAAAA|22370|6|12|50|AM|third|morning|breakfast| +22371|AAAAAAAAEGHFAAAA|22371|6|12|51|AM|third|morning|breakfast| +22372|AAAAAAAAFGHFAAAA|22372|6|12|52|AM|third|morning|breakfast| +22373|AAAAAAAAGGHFAAAA|22373|6|12|53|AM|third|morning|breakfast| +22374|AAAAAAAAHGHFAAAA|22374|6|12|54|AM|third|morning|breakfast| +22375|AAAAAAAAIGHFAAAA|22375|6|12|55|AM|third|morning|breakfast| +22376|AAAAAAAAJGHFAAAA|22376|6|12|56|AM|third|morning|breakfast| +22377|AAAAAAAAKGHFAAAA|22377|6|12|57|AM|third|morning|breakfast| +22378|AAAAAAAALGHFAAAA|22378|6|12|58|AM|third|morning|breakfast| +22379|AAAAAAAAMGHFAAAA|22379|6|12|59|AM|third|morning|breakfast| +22380|AAAAAAAANGHFAAAA|22380|6|13|0|AM|third|morning|breakfast| +22381|AAAAAAAAOGHFAAAA|22381|6|13|1|AM|third|morning|breakfast| +22382|AAAAAAAAPGHFAAAA|22382|6|13|2|AM|third|morning|breakfast| +22383|AAAAAAAAAHHFAAAA|22383|6|13|3|AM|third|morning|breakfast| +22384|AAAAAAAABHHFAAAA|22384|6|13|4|AM|third|morning|breakfast| +22385|AAAAAAAACHHFAAAA|22385|6|13|5|AM|third|morning|breakfast| +22386|AAAAAAAADHHFAAAA|22386|6|13|6|AM|third|morning|breakfast| +22387|AAAAAAAAEHHFAAAA|22387|6|13|7|AM|third|morning|breakfast| +22388|AAAAAAAAFHHFAAAA|22388|6|13|8|AM|third|morning|breakfast| +22389|AAAAAAAAGHHFAAAA|22389|6|13|9|AM|third|morning|breakfast| +22390|AAAAAAAAHHHFAAAA|22390|6|13|10|AM|third|morning|breakfast| +22391|AAAAAAAAIHHFAAAA|22391|6|13|11|AM|third|morning|breakfast| +22392|AAAAAAAAJHHFAAAA|22392|6|13|12|AM|third|morning|breakfast| +22393|AAAAAAAAKHHFAAAA|22393|6|13|13|AM|third|morning|breakfast| +22394|AAAAAAAALHHFAAAA|22394|6|13|14|AM|third|morning|breakfast| +22395|AAAAAAAAMHHFAAAA|22395|6|13|15|AM|third|morning|breakfast| +22396|AAAAAAAANHHFAAAA|22396|6|13|16|AM|third|morning|breakfast| +22397|AAAAAAAAOHHFAAAA|22397|6|13|17|AM|third|morning|breakfast| +22398|AAAAAAAAPHHFAAAA|22398|6|13|18|AM|third|morning|breakfast| +22399|AAAAAAAAAIHFAAAA|22399|6|13|19|AM|third|morning|breakfast| +22400|AAAAAAAABIHFAAAA|22400|6|13|20|AM|third|morning|breakfast| +22401|AAAAAAAACIHFAAAA|22401|6|13|21|AM|third|morning|breakfast| +22402|AAAAAAAADIHFAAAA|22402|6|13|22|AM|third|morning|breakfast| +22403|AAAAAAAAEIHFAAAA|22403|6|13|23|AM|third|morning|breakfast| +22404|AAAAAAAAFIHFAAAA|22404|6|13|24|AM|third|morning|breakfast| +22405|AAAAAAAAGIHFAAAA|22405|6|13|25|AM|third|morning|breakfast| +22406|AAAAAAAAHIHFAAAA|22406|6|13|26|AM|third|morning|breakfast| +22407|AAAAAAAAIIHFAAAA|22407|6|13|27|AM|third|morning|breakfast| +22408|AAAAAAAAJIHFAAAA|22408|6|13|28|AM|third|morning|breakfast| +22409|AAAAAAAAKIHFAAAA|22409|6|13|29|AM|third|morning|breakfast| +22410|AAAAAAAALIHFAAAA|22410|6|13|30|AM|third|morning|breakfast| +22411|AAAAAAAAMIHFAAAA|22411|6|13|31|AM|third|morning|breakfast| +22412|AAAAAAAANIHFAAAA|22412|6|13|32|AM|third|morning|breakfast| +22413|AAAAAAAAOIHFAAAA|22413|6|13|33|AM|third|morning|breakfast| +22414|AAAAAAAAPIHFAAAA|22414|6|13|34|AM|third|morning|breakfast| +22415|AAAAAAAAAJHFAAAA|22415|6|13|35|AM|third|morning|breakfast| +22416|AAAAAAAABJHFAAAA|22416|6|13|36|AM|third|morning|breakfast| +22417|AAAAAAAACJHFAAAA|22417|6|13|37|AM|third|morning|breakfast| +22418|AAAAAAAADJHFAAAA|22418|6|13|38|AM|third|morning|breakfast| +22419|AAAAAAAAEJHFAAAA|22419|6|13|39|AM|third|morning|breakfast| +22420|AAAAAAAAFJHFAAAA|22420|6|13|40|AM|third|morning|breakfast| +22421|AAAAAAAAGJHFAAAA|22421|6|13|41|AM|third|morning|breakfast| +22422|AAAAAAAAHJHFAAAA|22422|6|13|42|AM|third|morning|breakfast| +22423|AAAAAAAAIJHFAAAA|22423|6|13|43|AM|third|morning|breakfast| +22424|AAAAAAAAJJHFAAAA|22424|6|13|44|AM|third|morning|breakfast| +22425|AAAAAAAAKJHFAAAA|22425|6|13|45|AM|third|morning|breakfast| +22426|AAAAAAAALJHFAAAA|22426|6|13|46|AM|third|morning|breakfast| +22427|AAAAAAAAMJHFAAAA|22427|6|13|47|AM|third|morning|breakfast| +22428|AAAAAAAANJHFAAAA|22428|6|13|48|AM|third|morning|breakfast| +22429|AAAAAAAAOJHFAAAA|22429|6|13|49|AM|third|morning|breakfast| +22430|AAAAAAAAPJHFAAAA|22430|6|13|50|AM|third|morning|breakfast| +22431|AAAAAAAAAKHFAAAA|22431|6|13|51|AM|third|morning|breakfast| +22432|AAAAAAAABKHFAAAA|22432|6|13|52|AM|third|morning|breakfast| +22433|AAAAAAAACKHFAAAA|22433|6|13|53|AM|third|morning|breakfast| +22434|AAAAAAAADKHFAAAA|22434|6|13|54|AM|third|morning|breakfast| +22435|AAAAAAAAEKHFAAAA|22435|6|13|55|AM|third|morning|breakfast| +22436|AAAAAAAAFKHFAAAA|22436|6|13|56|AM|third|morning|breakfast| +22437|AAAAAAAAGKHFAAAA|22437|6|13|57|AM|third|morning|breakfast| +22438|AAAAAAAAHKHFAAAA|22438|6|13|58|AM|third|morning|breakfast| +22439|AAAAAAAAIKHFAAAA|22439|6|13|59|AM|third|morning|breakfast| +22440|AAAAAAAAJKHFAAAA|22440|6|14|0|AM|third|morning|breakfast| +22441|AAAAAAAAKKHFAAAA|22441|6|14|1|AM|third|morning|breakfast| +22442|AAAAAAAALKHFAAAA|22442|6|14|2|AM|third|morning|breakfast| +22443|AAAAAAAAMKHFAAAA|22443|6|14|3|AM|third|morning|breakfast| +22444|AAAAAAAANKHFAAAA|22444|6|14|4|AM|third|morning|breakfast| +22445|AAAAAAAAOKHFAAAA|22445|6|14|5|AM|third|morning|breakfast| +22446|AAAAAAAAPKHFAAAA|22446|6|14|6|AM|third|morning|breakfast| +22447|AAAAAAAAALHFAAAA|22447|6|14|7|AM|third|morning|breakfast| +22448|AAAAAAAABLHFAAAA|22448|6|14|8|AM|third|morning|breakfast| +22449|AAAAAAAACLHFAAAA|22449|6|14|9|AM|third|morning|breakfast| +22450|AAAAAAAADLHFAAAA|22450|6|14|10|AM|third|morning|breakfast| +22451|AAAAAAAAELHFAAAA|22451|6|14|11|AM|third|morning|breakfast| +22452|AAAAAAAAFLHFAAAA|22452|6|14|12|AM|third|morning|breakfast| +22453|AAAAAAAAGLHFAAAA|22453|6|14|13|AM|third|morning|breakfast| +22454|AAAAAAAAHLHFAAAA|22454|6|14|14|AM|third|morning|breakfast| +22455|AAAAAAAAILHFAAAA|22455|6|14|15|AM|third|morning|breakfast| +22456|AAAAAAAAJLHFAAAA|22456|6|14|16|AM|third|morning|breakfast| +22457|AAAAAAAAKLHFAAAA|22457|6|14|17|AM|third|morning|breakfast| +22458|AAAAAAAALLHFAAAA|22458|6|14|18|AM|third|morning|breakfast| +22459|AAAAAAAAMLHFAAAA|22459|6|14|19|AM|third|morning|breakfast| +22460|AAAAAAAANLHFAAAA|22460|6|14|20|AM|third|morning|breakfast| +22461|AAAAAAAAOLHFAAAA|22461|6|14|21|AM|third|morning|breakfast| +22462|AAAAAAAAPLHFAAAA|22462|6|14|22|AM|third|morning|breakfast| +22463|AAAAAAAAAMHFAAAA|22463|6|14|23|AM|third|morning|breakfast| +22464|AAAAAAAABMHFAAAA|22464|6|14|24|AM|third|morning|breakfast| +22465|AAAAAAAACMHFAAAA|22465|6|14|25|AM|third|morning|breakfast| +22466|AAAAAAAADMHFAAAA|22466|6|14|26|AM|third|morning|breakfast| +22467|AAAAAAAAEMHFAAAA|22467|6|14|27|AM|third|morning|breakfast| +22468|AAAAAAAAFMHFAAAA|22468|6|14|28|AM|third|morning|breakfast| +22469|AAAAAAAAGMHFAAAA|22469|6|14|29|AM|third|morning|breakfast| +22470|AAAAAAAAHMHFAAAA|22470|6|14|30|AM|third|morning|breakfast| +22471|AAAAAAAAIMHFAAAA|22471|6|14|31|AM|third|morning|breakfast| +22472|AAAAAAAAJMHFAAAA|22472|6|14|32|AM|third|morning|breakfast| +22473|AAAAAAAAKMHFAAAA|22473|6|14|33|AM|third|morning|breakfast| +22474|AAAAAAAALMHFAAAA|22474|6|14|34|AM|third|morning|breakfast| +22475|AAAAAAAAMMHFAAAA|22475|6|14|35|AM|third|morning|breakfast| +22476|AAAAAAAANMHFAAAA|22476|6|14|36|AM|third|morning|breakfast| +22477|AAAAAAAAOMHFAAAA|22477|6|14|37|AM|third|morning|breakfast| +22478|AAAAAAAAPMHFAAAA|22478|6|14|38|AM|third|morning|breakfast| +22479|AAAAAAAAANHFAAAA|22479|6|14|39|AM|third|morning|breakfast| +22480|AAAAAAAABNHFAAAA|22480|6|14|40|AM|third|morning|breakfast| +22481|AAAAAAAACNHFAAAA|22481|6|14|41|AM|third|morning|breakfast| +22482|AAAAAAAADNHFAAAA|22482|6|14|42|AM|third|morning|breakfast| +22483|AAAAAAAAENHFAAAA|22483|6|14|43|AM|third|morning|breakfast| +22484|AAAAAAAAFNHFAAAA|22484|6|14|44|AM|third|morning|breakfast| +22485|AAAAAAAAGNHFAAAA|22485|6|14|45|AM|third|morning|breakfast| +22486|AAAAAAAAHNHFAAAA|22486|6|14|46|AM|third|morning|breakfast| +22487|AAAAAAAAINHFAAAA|22487|6|14|47|AM|third|morning|breakfast| +22488|AAAAAAAAJNHFAAAA|22488|6|14|48|AM|third|morning|breakfast| +22489|AAAAAAAAKNHFAAAA|22489|6|14|49|AM|third|morning|breakfast| +22490|AAAAAAAALNHFAAAA|22490|6|14|50|AM|third|morning|breakfast| +22491|AAAAAAAAMNHFAAAA|22491|6|14|51|AM|third|morning|breakfast| +22492|AAAAAAAANNHFAAAA|22492|6|14|52|AM|third|morning|breakfast| +22493|AAAAAAAAONHFAAAA|22493|6|14|53|AM|third|morning|breakfast| +22494|AAAAAAAAPNHFAAAA|22494|6|14|54|AM|third|morning|breakfast| +22495|AAAAAAAAAOHFAAAA|22495|6|14|55|AM|third|morning|breakfast| +22496|AAAAAAAABOHFAAAA|22496|6|14|56|AM|third|morning|breakfast| +22497|AAAAAAAACOHFAAAA|22497|6|14|57|AM|third|morning|breakfast| +22498|AAAAAAAADOHFAAAA|22498|6|14|58|AM|third|morning|breakfast| +22499|AAAAAAAAEOHFAAAA|22499|6|14|59|AM|third|morning|breakfast| +22500|AAAAAAAAFOHFAAAA|22500|6|15|0|AM|third|morning|breakfast| +22501|AAAAAAAAGOHFAAAA|22501|6|15|1|AM|third|morning|breakfast| +22502|AAAAAAAAHOHFAAAA|22502|6|15|2|AM|third|morning|breakfast| +22503|AAAAAAAAIOHFAAAA|22503|6|15|3|AM|third|morning|breakfast| +22504|AAAAAAAAJOHFAAAA|22504|6|15|4|AM|third|morning|breakfast| +22505|AAAAAAAAKOHFAAAA|22505|6|15|5|AM|third|morning|breakfast| +22506|AAAAAAAALOHFAAAA|22506|6|15|6|AM|third|morning|breakfast| +22507|AAAAAAAAMOHFAAAA|22507|6|15|7|AM|third|morning|breakfast| +22508|AAAAAAAANOHFAAAA|22508|6|15|8|AM|third|morning|breakfast| +22509|AAAAAAAAOOHFAAAA|22509|6|15|9|AM|third|morning|breakfast| +22510|AAAAAAAAPOHFAAAA|22510|6|15|10|AM|third|morning|breakfast| +22511|AAAAAAAAAPHFAAAA|22511|6|15|11|AM|third|morning|breakfast| +22512|AAAAAAAABPHFAAAA|22512|6|15|12|AM|third|morning|breakfast| +22513|AAAAAAAACPHFAAAA|22513|6|15|13|AM|third|morning|breakfast| +22514|AAAAAAAADPHFAAAA|22514|6|15|14|AM|third|morning|breakfast| +22515|AAAAAAAAEPHFAAAA|22515|6|15|15|AM|third|morning|breakfast| +22516|AAAAAAAAFPHFAAAA|22516|6|15|16|AM|third|morning|breakfast| +22517|AAAAAAAAGPHFAAAA|22517|6|15|17|AM|third|morning|breakfast| +22518|AAAAAAAAHPHFAAAA|22518|6|15|18|AM|third|morning|breakfast| +22519|AAAAAAAAIPHFAAAA|22519|6|15|19|AM|third|morning|breakfast| +22520|AAAAAAAAJPHFAAAA|22520|6|15|20|AM|third|morning|breakfast| +22521|AAAAAAAAKPHFAAAA|22521|6|15|21|AM|third|morning|breakfast| +22522|AAAAAAAALPHFAAAA|22522|6|15|22|AM|third|morning|breakfast| +22523|AAAAAAAAMPHFAAAA|22523|6|15|23|AM|third|morning|breakfast| +22524|AAAAAAAANPHFAAAA|22524|6|15|24|AM|third|morning|breakfast| +22525|AAAAAAAAOPHFAAAA|22525|6|15|25|AM|third|morning|breakfast| +22526|AAAAAAAAPPHFAAAA|22526|6|15|26|AM|third|morning|breakfast| +22527|AAAAAAAAAAIFAAAA|22527|6|15|27|AM|third|morning|breakfast| +22528|AAAAAAAABAIFAAAA|22528|6|15|28|AM|third|morning|breakfast| +22529|AAAAAAAACAIFAAAA|22529|6|15|29|AM|third|morning|breakfast| +22530|AAAAAAAADAIFAAAA|22530|6|15|30|AM|third|morning|breakfast| +22531|AAAAAAAAEAIFAAAA|22531|6|15|31|AM|third|morning|breakfast| +22532|AAAAAAAAFAIFAAAA|22532|6|15|32|AM|third|morning|breakfast| +22533|AAAAAAAAGAIFAAAA|22533|6|15|33|AM|third|morning|breakfast| +22534|AAAAAAAAHAIFAAAA|22534|6|15|34|AM|third|morning|breakfast| +22535|AAAAAAAAIAIFAAAA|22535|6|15|35|AM|third|morning|breakfast| +22536|AAAAAAAAJAIFAAAA|22536|6|15|36|AM|third|morning|breakfast| +22537|AAAAAAAAKAIFAAAA|22537|6|15|37|AM|third|morning|breakfast| +22538|AAAAAAAALAIFAAAA|22538|6|15|38|AM|third|morning|breakfast| +22539|AAAAAAAAMAIFAAAA|22539|6|15|39|AM|third|morning|breakfast| +22540|AAAAAAAANAIFAAAA|22540|6|15|40|AM|third|morning|breakfast| +22541|AAAAAAAAOAIFAAAA|22541|6|15|41|AM|third|morning|breakfast| +22542|AAAAAAAAPAIFAAAA|22542|6|15|42|AM|third|morning|breakfast| +22543|AAAAAAAAABIFAAAA|22543|6|15|43|AM|third|morning|breakfast| +22544|AAAAAAAABBIFAAAA|22544|6|15|44|AM|third|morning|breakfast| +22545|AAAAAAAACBIFAAAA|22545|6|15|45|AM|third|morning|breakfast| +22546|AAAAAAAADBIFAAAA|22546|6|15|46|AM|third|morning|breakfast| +22547|AAAAAAAAEBIFAAAA|22547|6|15|47|AM|third|morning|breakfast| +22548|AAAAAAAAFBIFAAAA|22548|6|15|48|AM|third|morning|breakfast| +22549|AAAAAAAAGBIFAAAA|22549|6|15|49|AM|third|morning|breakfast| +22550|AAAAAAAAHBIFAAAA|22550|6|15|50|AM|third|morning|breakfast| +22551|AAAAAAAAIBIFAAAA|22551|6|15|51|AM|third|morning|breakfast| +22552|AAAAAAAAJBIFAAAA|22552|6|15|52|AM|third|morning|breakfast| +22553|AAAAAAAAKBIFAAAA|22553|6|15|53|AM|third|morning|breakfast| +22554|AAAAAAAALBIFAAAA|22554|6|15|54|AM|third|morning|breakfast| +22555|AAAAAAAAMBIFAAAA|22555|6|15|55|AM|third|morning|breakfast| +22556|AAAAAAAANBIFAAAA|22556|6|15|56|AM|third|morning|breakfast| +22557|AAAAAAAAOBIFAAAA|22557|6|15|57|AM|third|morning|breakfast| +22558|AAAAAAAAPBIFAAAA|22558|6|15|58|AM|third|morning|breakfast| +22559|AAAAAAAAACIFAAAA|22559|6|15|59|AM|third|morning|breakfast| +22560|AAAAAAAABCIFAAAA|22560|6|16|0|AM|third|morning|breakfast| +22561|AAAAAAAACCIFAAAA|22561|6|16|1|AM|third|morning|breakfast| +22562|AAAAAAAADCIFAAAA|22562|6|16|2|AM|third|morning|breakfast| +22563|AAAAAAAAECIFAAAA|22563|6|16|3|AM|third|morning|breakfast| +22564|AAAAAAAAFCIFAAAA|22564|6|16|4|AM|third|morning|breakfast| +22565|AAAAAAAAGCIFAAAA|22565|6|16|5|AM|third|morning|breakfast| +22566|AAAAAAAAHCIFAAAA|22566|6|16|6|AM|third|morning|breakfast| +22567|AAAAAAAAICIFAAAA|22567|6|16|7|AM|third|morning|breakfast| +22568|AAAAAAAAJCIFAAAA|22568|6|16|8|AM|third|morning|breakfast| +22569|AAAAAAAAKCIFAAAA|22569|6|16|9|AM|third|morning|breakfast| +22570|AAAAAAAALCIFAAAA|22570|6|16|10|AM|third|morning|breakfast| +22571|AAAAAAAAMCIFAAAA|22571|6|16|11|AM|third|morning|breakfast| +22572|AAAAAAAANCIFAAAA|22572|6|16|12|AM|third|morning|breakfast| +22573|AAAAAAAAOCIFAAAA|22573|6|16|13|AM|third|morning|breakfast| +22574|AAAAAAAAPCIFAAAA|22574|6|16|14|AM|third|morning|breakfast| +22575|AAAAAAAAADIFAAAA|22575|6|16|15|AM|third|morning|breakfast| +22576|AAAAAAAABDIFAAAA|22576|6|16|16|AM|third|morning|breakfast| +22577|AAAAAAAACDIFAAAA|22577|6|16|17|AM|third|morning|breakfast| +22578|AAAAAAAADDIFAAAA|22578|6|16|18|AM|third|morning|breakfast| +22579|AAAAAAAAEDIFAAAA|22579|6|16|19|AM|third|morning|breakfast| +22580|AAAAAAAAFDIFAAAA|22580|6|16|20|AM|third|morning|breakfast| +22581|AAAAAAAAGDIFAAAA|22581|6|16|21|AM|third|morning|breakfast| +22582|AAAAAAAAHDIFAAAA|22582|6|16|22|AM|third|morning|breakfast| +22583|AAAAAAAAIDIFAAAA|22583|6|16|23|AM|third|morning|breakfast| +22584|AAAAAAAAJDIFAAAA|22584|6|16|24|AM|third|morning|breakfast| +22585|AAAAAAAAKDIFAAAA|22585|6|16|25|AM|third|morning|breakfast| +22586|AAAAAAAALDIFAAAA|22586|6|16|26|AM|third|morning|breakfast| +22587|AAAAAAAAMDIFAAAA|22587|6|16|27|AM|third|morning|breakfast| +22588|AAAAAAAANDIFAAAA|22588|6|16|28|AM|third|morning|breakfast| +22589|AAAAAAAAODIFAAAA|22589|6|16|29|AM|third|morning|breakfast| +22590|AAAAAAAAPDIFAAAA|22590|6|16|30|AM|third|morning|breakfast| +22591|AAAAAAAAAEIFAAAA|22591|6|16|31|AM|third|morning|breakfast| +22592|AAAAAAAABEIFAAAA|22592|6|16|32|AM|third|morning|breakfast| +22593|AAAAAAAACEIFAAAA|22593|6|16|33|AM|third|morning|breakfast| +22594|AAAAAAAADEIFAAAA|22594|6|16|34|AM|third|morning|breakfast| +22595|AAAAAAAAEEIFAAAA|22595|6|16|35|AM|third|morning|breakfast| +22596|AAAAAAAAFEIFAAAA|22596|6|16|36|AM|third|morning|breakfast| +22597|AAAAAAAAGEIFAAAA|22597|6|16|37|AM|third|morning|breakfast| +22598|AAAAAAAAHEIFAAAA|22598|6|16|38|AM|third|morning|breakfast| +22599|AAAAAAAAIEIFAAAA|22599|6|16|39|AM|third|morning|breakfast| +22600|AAAAAAAAJEIFAAAA|22600|6|16|40|AM|third|morning|breakfast| +22601|AAAAAAAAKEIFAAAA|22601|6|16|41|AM|third|morning|breakfast| +22602|AAAAAAAALEIFAAAA|22602|6|16|42|AM|third|morning|breakfast| +22603|AAAAAAAAMEIFAAAA|22603|6|16|43|AM|third|morning|breakfast| +22604|AAAAAAAANEIFAAAA|22604|6|16|44|AM|third|morning|breakfast| +22605|AAAAAAAAOEIFAAAA|22605|6|16|45|AM|third|morning|breakfast| +22606|AAAAAAAAPEIFAAAA|22606|6|16|46|AM|third|morning|breakfast| +22607|AAAAAAAAAFIFAAAA|22607|6|16|47|AM|third|morning|breakfast| +22608|AAAAAAAABFIFAAAA|22608|6|16|48|AM|third|morning|breakfast| +22609|AAAAAAAACFIFAAAA|22609|6|16|49|AM|third|morning|breakfast| +22610|AAAAAAAADFIFAAAA|22610|6|16|50|AM|third|morning|breakfast| +22611|AAAAAAAAEFIFAAAA|22611|6|16|51|AM|third|morning|breakfast| +22612|AAAAAAAAFFIFAAAA|22612|6|16|52|AM|third|morning|breakfast| +22613|AAAAAAAAGFIFAAAA|22613|6|16|53|AM|third|morning|breakfast| +22614|AAAAAAAAHFIFAAAA|22614|6|16|54|AM|third|morning|breakfast| +22615|AAAAAAAAIFIFAAAA|22615|6|16|55|AM|third|morning|breakfast| +22616|AAAAAAAAJFIFAAAA|22616|6|16|56|AM|third|morning|breakfast| +22617|AAAAAAAAKFIFAAAA|22617|6|16|57|AM|third|morning|breakfast| +22618|AAAAAAAALFIFAAAA|22618|6|16|58|AM|third|morning|breakfast| +22619|AAAAAAAAMFIFAAAA|22619|6|16|59|AM|third|morning|breakfast| +22620|AAAAAAAANFIFAAAA|22620|6|17|0|AM|third|morning|breakfast| +22621|AAAAAAAAOFIFAAAA|22621|6|17|1|AM|third|morning|breakfast| +22622|AAAAAAAAPFIFAAAA|22622|6|17|2|AM|third|morning|breakfast| +22623|AAAAAAAAAGIFAAAA|22623|6|17|3|AM|third|morning|breakfast| +22624|AAAAAAAABGIFAAAA|22624|6|17|4|AM|third|morning|breakfast| +22625|AAAAAAAACGIFAAAA|22625|6|17|5|AM|third|morning|breakfast| +22626|AAAAAAAADGIFAAAA|22626|6|17|6|AM|third|morning|breakfast| +22627|AAAAAAAAEGIFAAAA|22627|6|17|7|AM|third|morning|breakfast| +22628|AAAAAAAAFGIFAAAA|22628|6|17|8|AM|third|morning|breakfast| +22629|AAAAAAAAGGIFAAAA|22629|6|17|9|AM|third|morning|breakfast| +22630|AAAAAAAAHGIFAAAA|22630|6|17|10|AM|third|morning|breakfast| +22631|AAAAAAAAIGIFAAAA|22631|6|17|11|AM|third|morning|breakfast| +22632|AAAAAAAAJGIFAAAA|22632|6|17|12|AM|third|morning|breakfast| +22633|AAAAAAAAKGIFAAAA|22633|6|17|13|AM|third|morning|breakfast| +22634|AAAAAAAALGIFAAAA|22634|6|17|14|AM|third|morning|breakfast| +22635|AAAAAAAAMGIFAAAA|22635|6|17|15|AM|third|morning|breakfast| +22636|AAAAAAAANGIFAAAA|22636|6|17|16|AM|third|morning|breakfast| +22637|AAAAAAAAOGIFAAAA|22637|6|17|17|AM|third|morning|breakfast| +22638|AAAAAAAAPGIFAAAA|22638|6|17|18|AM|third|morning|breakfast| +22639|AAAAAAAAAHIFAAAA|22639|6|17|19|AM|third|morning|breakfast| +22640|AAAAAAAABHIFAAAA|22640|6|17|20|AM|third|morning|breakfast| +22641|AAAAAAAACHIFAAAA|22641|6|17|21|AM|third|morning|breakfast| +22642|AAAAAAAADHIFAAAA|22642|6|17|22|AM|third|morning|breakfast| +22643|AAAAAAAAEHIFAAAA|22643|6|17|23|AM|third|morning|breakfast| +22644|AAAAAAAAFHIFAAAA|22644|6|17|24|AM|third|morning|breakfast| +22645|AAAAAAAAGHIFAAAA|22645|6|17|25|AM|third|morning|breakfast| +22646|AAAAAAAAHHIFAAAA|22646|6|17|26|AM|third|morning|breakfast| +22647|AAAAAAAAIHIFAAAA|22647|6|17|27|AM|third|morning|breakfast| +22648|AAAAAAAAJHIFAAAA|22648|6|17|28|AM|third|morning|breakfast| +22649|AAAAAAAAKHIFAAAA|22649|6|17|29|AM|third|morning|breakfast| +22650|AAAAAAAALHIFAAAA|22650|6|17|30|AM|third|morning|breakfast| +22651|AAAAAAAAMHIFAAAA|22651|6|17|31|AM|third|morning|breakfast| +22652|AAAAAAAANHIFAAAA|22652|6|17|32|AM|third|morning|breakfast| +22653|AAAAAAAAOHIFAAAA|22653|6|17|33|AM|third|morning|breakfast| +22654|AAAAAAAAPHIFAAAA|22654|6|17|34|AM|third|morning|breakfast| +22655|AAAAAAAAAIIFAAAA|22655|6|17|35|AM|third|morning|breakfast| +22656|AAAAAAAABIIFAAAA|22656|6|17|36|AM|third|morning|breakfast| +22657|AAAAAAAACIIFAAAA|22657|6|17|37|AM|third|morning|breakfast| +22658|AAAAAAAADIIFAAAA|22658|6|17|38|AM|third|morning|breakfast| +22659|AAAAAAAAEIIFAAAA|22659|6|17|39|AM|third|morning|breakfast| +22660|AAAAAAAAFIIFAAAA|22660|6|17|40|AM|third|morning|breakfast| +22661|AAAAAAAAGIIFAAAA|22661|6|17|41|AM|third|morning|breakfast| +22662|AAAAAAAAHIIFAAAA|22662|6|17|42|AM|third|morning|breakfast| +22663|AAAAAAAAIIIFAAAA|22663|6|17|43|AM|third|morning|breakfast| +22664|AAAAAAAAJIIFAAAA|22664|6|17|44|AM|third|morning|breakfast| +22665|AAAAAAAAKIIFAAAA|22665|6|17|45|AM|third|morning|breakfast| +22666|AAAAAAAALIIFAAAA|22666|6|17|46|AM|third|morning|breakfast| +22667|AAAAAAAAMIIFAAAA|22667|6|17|47|AM|third|morning|breakfast| +22668|AAAAAAAANIIFAAAA|22668|6|17|48|AM|third|morning|breakfast| +22669|AAAAAAAAOIIFAAAA|22669|6|17|49|AM|third|morning|breakfast| +22670|AAAAAAAAPIIFAAAA|22670|6|17|50|AM|third|morning|breakfast| +22671|AAAAAAAAAJIFAAAA|22671|6|17|51|AM|third|morning|breakfast| +22672|AAAAAAAABJIFAAAA|22672|6|17|52|AM|third|morning|breakfast| +22673|AAAAAAAACJIFAAAA|22673|6|17|53|AM|third|morning|breakfast| +22674|AAAAAAAADJIFAAAA|22674|6|17|54|AM|third|morning|breakfast| +22675|AAAAAAAAEJIFAAAA|22675|6|17|55|AM|third|morning|breakfast| +22676|AAAAAAAAFJIFAAAA|22676|6|17|56|AM|third|morning|breakfast| +22677|AAAAAAAAGJIFAAAA|22677|6|17|57|AM|third|morning|breakfast| +22678|AAAAAAAAHJIFAAAA|22678|6|17|58|AM|third|morning|breakfast| +22679|AAAAAAAAIJIFAAAA|22679|6|17|59|AM|third|morning|breakfast| +22680|AAAAAAAAJJIFAAAA|22680|6|18|0|AM|third|morning|breakfast| +22681|AAAAAAAAKJIFAAAA|22681|6|18|1|AM|third|morning|breakfast| +22682|AAAAAAAALJIFAAAA|22682|6|18|2|AM|third|morning|breakfast| +22683|AAAAAAAAMJIFAAAA|22683|6|18|3|AM|third|morning|breakfast| +22684|AAAAAAAANJIFAAAA|22684|6|18|4|AM|third|morning|breakfast| +22685|AAAAAAAAOJIFAAAA|22685|6|18|5|AM|third|morning|breakfast| +22686|AAAAAAAAPJIFAAAA|22686|6|18|6|AM|third|morning|breakfast| +22687|AAAAAAAAAKIFAAAA|22687|6|18|7|AM|third|morning|breakfast| +22688|AAAAAAAABKIFAAAA|22688|6|18|8|AM|third|morning|breakfast| +22689|AAAAAAAACKIFAAAA|22689|6|18|9|AM|third|morning|breakfast| +22690|AAAAAAAADKIFAAAA|22690|6|18|10|AM|third|morning|breakfast| +22691|AAAAAAAAEKIFAAAA|22691|6|18|11|AM|third|morning|breakfast| +22692|AAAAAAAAFKIFAAAA|22692|6|18|12|AM|third|morning|breakfast| +22693|AAAAAAAAGKIFAAAA|22693|6|18|13|AM|third|morning|breakfast| +22694|AAAAAAAAHKIFAAAA|22694|6|18|14|AM|third|morning|breakfast| +22695|AAAAAAAAIKIFAAAA|22695|6|18|15|AM|third|morning|breakfast| +22696|AAAAAAAAJKIFAAAA|22696|6|18|16|AM|third|morning|breakfast| +22697|AAAAAAAAKKIFAAAA|22697|6|18|17|AM|third|morning|breakfast| +22698|AAAAAAAALKIFAAAA|22698|6|18|18|AM|third|morning|breakfast| +22699|AAAAAAAAMKIFAAAA|22699|6|18|19|AM|third|morning|breakfast| +22700|AAAAAAAANKIFAAAA|22700|6|18|20|AM|third|morning|breakfast| +22701|AAAAAAAAOKIFAAAA|22701|6|18|21|AM|third|morning|breakfast| +22702|AAAAAAAAPKIFAAAA|22702|6|18|22|AM|third|morning|breakfast| +22703|AAAAAAAAALIFAAAA|22703|6|18|23|AM|third|morning|breakfast| +22704|AAAAAAAABLIFAAAA|22704|6|18|24|AM|third|morning|breakfast| +22705|AAAAAAAACLIFAAAA|22705|6|18|25|AM|third|morning|breakfast| +22706|AAAAAAAADLIFAAAA|22706|6|18|26|AM|third|morning|breakfast| +22707|AAAAAAAAELIFAAAA|22707|6|18|27|AM|third|morning|breakfast| +22708|AAAAAAAAFLIFAAAA|22708|6|18|28|AM|third|morning|breakfast| +22709|AAAAAAAAGLIFAAAA|22709|6|18|29|AM|third|morning|breakfast| +22710|AAAAAAAAHLIFAAAA|22710|6|18|30|AM|third|morning|breakfast| +22711|AAAAAAAAILIFAAAA|22711|6|18|31|AM|third|morning|breakfast| +22712|AAAAAAAAJLIFAAAA|22712|6|18|32|AM|third|morning|breakfast| +22713|AAAAAAAAKLIFAAAA|22713|6|18|33|AM|third|morning|breakfast| +22714|AAAAAAAALLIFAAAA|22714|6|18|34|AM|third|morning|breakfast| +22715|AAAAAAAAMLIFAAAA|22715|6|18|35|AM|third|morning|breakfast| +22716|AAAAAAAANLIFAAAA|22716|6|18|36|AM|third|morning|breakfast| +22717|AAAAAAAAOLIFAAAA|22717|6|18|37|AM|third|morning|breakfast| +22718|AAAAAAAAPLIFAAAA|22718|6|18|38|AM|third|morning|breakfast| +22719|AAAAAAAAAMIFAAAA|22719|6|18|39|AM|third|morning|breakfast| +22720|AAAAAAAABMIFAAAA|22720|6|18|40|AM|third|morning|breakfast| +22721|AAAAAAAACMIFAAAA|22721|6|18|41|AM|third|morning|breakfast| +22722|AAAAAAAADMIFAAAA|22722|6|18|42|AM|third|morning|breakfast| +22723|AAAAAAAAEMIFAAAA|22723|6|18|43|AM|third|morning|breakfast| +22724|AAAAAAAAFMIFAAAA|22724|6|18|44|AM|third|morning|breakfast| +22725|AAAAAAAAGMIFAAAA|22725|6|18|45|AM|third|morning|breakfast| +22726|AAAAAAAAHMIFAAAA|22726|6|18|46|AM|third|morning|breakfast| +22727|AAAAAAAAIMIFAAAA|22727|6|18|47|AM|third|morning|breakfast| +22728|AAAAAAAAJMIFAAAA|22728|6|18|48|AM|third|morning|breakfast| +22729|AAAAAAAAKMIFAAAA|22729|6|18|49|AM|third|morning|breakfast| +22730|AAAAAAAALMIFAAAA|22730|6|18|50|AM|third|morning|breakfast| +22731|AAAAAAAAMMIFAAAA|22731|6|18|51|AM|third|morning|breakfast| +22732|AAAAAAAANMIFAAAA|22732|6|18|52|AM|third|morning|breakfast| +22733|AAAAAAAAOMIFAAAA|22733|6|18|53|AM|third|morning|breakfast| +22734|AAAAAAAAPMIFAAAA|22734|6|18|54|AM|third|morning|breakfast| +22735|AAAAAAAAANIFAAAA|22735|6|18|55|AM|third|morning|breakfast| +22736|AAAAAAAABNIFAAAA|22736|6|18|56|AM|third|morning|breakfast| +22737|AAAAAAAACNIFAAAA|22737|6|18|57|AM|third|morning|breakfast| +22738|AAAAAAAADNIFAAAA|22738|6|18|58|AM|third|morning|breakfast| +22739|AAAAAAAAENIFAAAA|22739|6|18|59|AM|third|morning|breakfast| +22740|AAAAAAAAFNIFAAAA|22740|6|19|0|AM|third|morning|breakfast| +22741|AAAAAAAAGNIFAAAA|22741|6|19|1|AM|third|morning|breakfast| +22742|AAAAAAAAHNIFAAAA|22742|6|19|2|AM|third|morning|breakfast| +22743|AAAAAAAAINIFAAAA|22743|6|19|3|AM|third|morning|breakfast| +22744|AAAAAAAAJNIFAAAA|22744|6|19|4|AM|third|morning|breakfast| +22745|AAAAAAAAKNIFAAAA|22745|6|19|5|AM|third|morning|breakfast| +22746|AAAAAAAALNIFAAAA|22746|6|19|6|AM|third|morning|breakfast| +22747|AAAAAAAAMNIFAAAA|22747|6|19|7|AM|third|morning|breakfast| +22748|AAAAAAAANNIFAAAA|22748|6|19|8|AM|third|morning|breakfast| +22749|AAAAAAAAONIFAAAA|22749|6|19|9|AM|third|morning|breakfast| +22750|AAAAAAAAPNIFAAAA|22750|6|19|10|AM|third|morning|breakfast| +22751|AAAAAAAAAOIFAAAA|22751|6|19|11|AM|third|morning|breakfast| +22752|AAAAAAAABOIFAAAA|22752|6|19|12|AM|third|morning|breakfast| +22753|AAAAAAAACOIFAAAA|22753|6|19|13|AM|third|morning|breakfast| +22754|AAAAAAAADOIFAAAA|22754|6|19|14|AM|third|morning|breakfast| +22755|AAAAAAAAEOIFAAAA|22755|6|19|15|AM|third|morning|breakfast| +22756|AAAAAAAAFOIFAAAA|22756|6|19|16|AM|third|morning|breakfast| +22757|AAAAAAAAGOIFAAAA|22757|6|19|17|AM|third|morning|breakfast| +22758|AAAAAAAAHOIFAAAA|22758|6|19|18|AM|third|morning|breakfast| +22759|AAAAAAAAIOIFAAAA|22759|6|19|19|AM|third|morning|breakfast| +22760|AAAAAAAAJOIFAAAA|22760|6|19|20|AM|third|morning|breakfast| +22761|AAAAAAAAKOIFAAAA|22761|6|19|21|AM|third|morning|breakfast| +22762|AAAAAAAALOIFAAAA|22762|6|19|22|AM|third|morning|breakfast| +22763|AAAAAAAAMOIFAAAA|22763|6|19|23|AM|third|morning|breakfast| +22764|AAAAAAAANOIFAAAA|22764|6|19|24|AM|third|morning|breakfast| +22765|AAAAAAAAOOIFAAAA|22765|6|19|25|AM|third|morning|breakfast| +22766|AAAAAAAAPOIFAAAA|22766|6|19|26|AM|third|morning|breakfast| +22767|AAAAAAAAAPIFAAAA|22767|6|19|27|AM|third|morning|breakfast| +22768|AAAAAAAABPIFAAAA|22768|6|19|28|AM|third|morning|breakfast| +22769|AAAAAAAACPIFAAAA|22769|6|19|29|AM|third|morning|breakfast| +22770|AAAAAAAADPIFAAAA|22770|6|19|30|AM|third|morning|breakfast| +22771|AAAAAAAAEPIFAAAA|22771|6|19|31|AM|third|morning|breakfast| +22772|AAAAAAAAFPIFAAAA|22772|6|19|32|AM|third|morning|breakfast| +22773|AAAAAAAAGPIFAAAA|22773|6|19|33|AM|third|morning|breakfast| +22774|AAAAAAAAHPIFAAAA|22774|6|19|34|AM|third|morning|breakfast| +22775|AAAAAAAAIPIFAAAA|22775|6|19|35|AM|third|morning|breakfast| +22776|AAAAAAAAJPIFAAAA|22776|6|19|36|AM|third|morning|breakfast| +22777|AAAAAAAAKPIFAAAA|22777|6|19|37|AM|third|morning|breakfast| +22778|AAAAAAAALPIFAAAA|22778|6|19|38|AM|third|morning|breakfast| +22779|AAAAAAAAMPIFAAAA|22779|6|19|39|AM|third|morning|breakfast| +22780|AAAAAAAANPIFAAAA|22780|6|19|40|AM|third|morning|breakfast| +22781|AAAAAAAAOPIFAAAA|22781|6|19|41|AM|third|morning|breakfast| +22782|AAAAAAAAPPIFAAAA|22782|6|19|42|AM|third|morning|breakfast| +22783|AAAAAAAAAAJFAAAA|22783|6|19|43|AM|third|morning|breakfast| +22784|AAAAAAAABAJFAAAA|22784|6|19|44|AM|third|morning|breakfast| +22785|AAAAAAAACAJFAAAA|22785|6|19|45|AM|third|morning|breakfast| +22786|AAAAAAAADAJFAAAA|22786|6|19|46|AM|third|morning|breakfast| +22787|AAAAAAAAEAJFAAAA|22787|6|19|47|AM|third|morning|breakfast| +22788|AAAAAAAAFAJFAAAA|22788|6|19|48|AM|third|morning|breakfast| +22789|AAAAAAAAGAJFAAAA|22789|6|19|49|AM|third|morning|breakfast| +22790|AAAAAAAAHAJFAAAA|22790|6|19|50|AM|third|morning|breakfast| +22791|AAAAAAAAIAJFAAAA|22791|6|19|51|AM|third|morning|breakfast| +22792|AAAAAAAAJAJFAAAA|22792|6|19|52|AM|third|morning|breakfast| +22793|AAAAAAAAKAJFAAAA|22793|6|19|53|AM|third|morning|breakfast| +22794|AAAAAAAALAJFAAAA|22794|6|19|54|AM|third|morning|breakfast| +22795|AAAAAAAAMAJFAAAA|22795|6|19|55|AM|third|morning|breakfast| +22796|AAAAAAAANAJFAAAA|22796|6|19|56|AM|third|morning|breakfast| +22797|AAAAAAAAOAJFAAAA|22797|6|19|57|AM|third|morning|breakfast| +22798|AAAAAAAAPAJFAAAA|22798|6|19|58|AM|third|morning|breakfast| +22799|AAAAAAAAABJFAAAA|22799|6|19|59|AM|third|morning|breakfast| +22800|AAAAAAAABBJFAAAA|22800|6|20|0|AM|third|morning|breakfast| +22801|AAAAAAAACBJFAAAA|22801|6|20|1|AM|third|morning|breakfast| +22802|AAAAAAAADBJFAAAA|22802|6|20|2|AM|third|morning|breakfast| +22803|AAAAAAAAEBJFAAAA|22803|6|20|3|AM|third|morning|breakfast| +22804|AAAAAAAAFBJFAAAA|22804|6|20|4|AM|third|morning|breakfast| +22805|AAAAAAAAGBJFAAAA|22805|6|20|5|AM|third|morning|breakfast| +22806|AAAAAAAAHBJFAAAA|22806|6|20|6|AM|third|morning|breakfast| +22807|AAAAAAAAIBJFAAAA|22807|6|20|7|AM|third|morning|breakfast| +22808|AAAAAAAAJBJFAAAA|22808|6|20|8|AM|third|morning|breakfast| +22809|AAAAAAAAKBJFAAAA|22809|6|20|9|AM|third|morning|breakfast| +22810|AAAAAAAALBJFAAAA|22810|6|20|10|AM|third|morning|breakfast| +22811|AAAAAAAAMBJFAAAA|22811|6|20|11|AM|third|morning|breakfast| +22812|AAAAAAAANBJFAAAA|22812|6|20|12|AM|third|morning|breakfast| +22813|AAAAAAAAOBJFAAAA|22813|6|20|13|AM|third|morning|breakfast| +22814|AAAAAAAAPBJFAAAA|22814|6|20|14|AM|third|morning|breakfast| +22815|AAAAAAAAACJFAAAA|22815|6|20|15|AM|third|morning|breakfast| +22816|AAAAAAAABCJFAAAA|22816|6|20|16|AM|third|morning|breakfast| +22817|AAAAAAAACCJFAAAA|22817|6|20|17|AM|third|morning|breakfast| +22818|AAAAAAAADCJFAAAA|22818|6|20|18|AM|third|morning|breakfast| +22819|AAAAAAAAECJFAAAA|22819|6|20|19|AM|third|morning|breakfast| +22820|AAAAAAAAFCJFAAAA|22820|6|20|20|AM|third|morning|breakfast| +22821|AAAAAAAAGCJFAAAA|22821|6|20|21|AM|third|morning|breakfast| +22822|AAAAAAAAHCJFAAAA|22822|6|20|22|AM|third|morning|breakfast| +22823|AAAAAAAAICJFAAAA|22823|6|20|23|AM|third|morning|breakfast| +22824|AAAAAAAAJCJFAAAA|22824|6|20|24|AM|third|morning|breakfast| +22825|AAAAAAAAKCJFAAAA|22825|6|20|25|AM|third|morning|breakfast| +22826|AAAAAAAALCJFAAAA|22826|6|20|26|AM|third|morning|breakfast| +22827|AAAAAAAAMCJFAAAA|22827|6|20|27|AM|third|morning|breakfast| +22828|AAAAAAAANCJFAAAA|22828|6|20|28|AM|third|morning|breakfast| +22829|AAAAAAAAOCJFAAAA|22829|6|20|29|AM|third|morning|breakfast| +22830|AAAAAAAAPCJFAAAA|22830|6|20|30|AM|third|morning|breakfast| +22831|AAAAAAAAADJFAAAA|22831|6|20|31|AM|third|morning|breakfast| +22832|AAAAAAAABDJFAAAA|22832|6|20|32|AM|third|morning|breakfast| +22833|AAAAAAAACDJFAAAA|22833|6|20|33|AM|third|morning|breakfast| +22834|AAAAAAAADDJFAAAA|22834|6|20|34|AM|third|morning|breakfast| +22835|AAAAAAAAEDJFAAAA|22835|6|20|35|AM|third|morning|breakfast| +22836|AAAAAAAAFDJFAAAA|22836|6|20|36|AM|third|morning|breakfast| +22837|AAAAAAAAGDJFAAAA|22837|6|20|37|AM|third|morning|breakfast| +22838|AAAAAAAAHDJFAAAA|22838|6|20|38|AM|third|morning|breakfast| +22839|AAAAAAAAIDJFAAAA|22839|6|20|39|AM|third|morning|breakfast| +22840|AAAAAAAAJDJFAAAA|22840|6|20|40|AM|third|morning|breakfast| +22841|AAAAAAAAKDJFAAAA|22841|6|20|41|AM|third|morning|breakfast| +22842|AAAAAAAALDJFAAAA|22842|6|20|42|AM|third|morning|breakfast| +22843|AAAAAAAAMDJFAAAA|22843|6|20|43|AM|third|morning|breakfast| +22844|AAAAAAAANDJFAAAA|22844|6|20|44|AM|third|morning|breakfast| +22845|AAAAAAAAODJFAAAA|22845|6|20|45|AM|third|morning|breakfast| +22846|AAAAAAAAPDJFAAAA|22846|6|20|46|AM|third|morning|breakfast| +22847|AAAAAAAAAEJFAAAA|22847|6|20|47|AM|third|morning|breakfast| +22848|AAAAAAAABEJFAAAA|22848|6|20|48|AM|third|morning|breakfast| +22849|AAAAAAAACEJFAAAA|22849|6|20|49|AM|third|morning|breakfast| +22850|AAAAAAAADEJFAAAA|22850|6|20|50|AM|third|morning|breakfast| +22851|AAAAAAAAEEJFAAAA|22851|6|20|51|AM|third|morning|breakfast| +22852|AAAAAAAAFEJFAAAA|22852|6|20|52|AM|third|morning|breakfast| +22853|AAAAAAAAGEJFAAAA|22853|6|20|53|AM|third|morning|breakfast| +22854|AAAAAAAAHEJFAAAA|22854|6|20|54|AM|third|morning|breakfast| +22855|AAAAAAAAIEJFAAAA|22855|6|20|55|AM|third|morning|breakfast| +22856|AAAAAAAAJEJFAAAA|22856|6|20|56|AM|third|morning|breakfast| +22857|AAAAAAAAKEJFAAAA|22857|6|20|57|AM|third|morning|breakfast| +22858|AAAAAAAALEJFAAAA|22858|6|20|58|AM|third|morning|breakfast| +22859|AAAAAAAAMEJFAAAA|22859|6|20|59|AM|third|morning|breakfast| +22860|AAAAAAAANEJFAAAA|22860|6|21|0|AM|third|morning|breakfast| +22861|AAAAAAAAOEJFAAAA|22861|6|21|1|AM|third|morning|breakfast| +22862|AAAAAAAAPEJFAAAA|22862|6|21|2|AM|third|morning|breakfast| +22863|AAAAAAAAAFJFAAAA|22863|6|21|3|AM|third|morning|breakfast| +22864|AAAAAAAABFJFAAAA|22864|6|21|4|AM|third|morning|breakfast| +22865|AAAAAAAACFJFAAAA|22865|6|21|5|AM|third|morning|breakfast| +22866|AAAAAAAADFJFAAAA|22866|6|21|6|AM|third|morning|breakfast| +22867|AAAAAAAAEFJFAAAA|22867|6|21|7|AM|third|morning|breakfast| +22868|AAAAAAAAFFJFAAAA|22868|6|21|8|AM|third|morning|breakfast| +22869|AAAAAAAAGFJFAAAA|22869|6|21|9|AM|third|morning|breakfast| +22870|AAAAAAAAHFJFAAAA|22870|6|21|10|AM|third|morning|breakfast| +22871|AAAAAAAAIFJFAAAA|22871|6|21|11|AM|third|morning|breakfast| +22872|AAAAAAAAJFJFAAAA|22872|6|21|12|AM|third|morning|breakfast| +22873|AAAAAAAAKFJFAAAA|22873|6|21|13|AM|third|morning|breakfast| +22874|AAAAAAAALFJFAAAA|22874|6|21|14|AM|third|morning|breakfast| +22875|AAAAAAAAMFJFAAAA|22875|6|21|15|AM|third|morning|breakfast| +22876|AAAAAAAANFJFAAAA|22876|6|21|16|AM|third|morning|breakfast| +22877|AAAAAAAAOFJFAAAA|22877|6|21|17|AM|third|morning|breakfast| +22878|AAAAAAAAPFJFAAAA|22878|6|21|18|AM|third|morning|breakfast| +22879|AAAAAAAAAGJFAAAA|22879|6|21|19|AM|third|morning|breakfast| +22880|AAAAAAAABGJFAAAA|22880|6|21|20|AM|third|morning|breakfast| +22881|AAAAAAAACGJFAAAA|22881|6|21|21|AM|third|morning|breakfast| +22882|AAAAAAAADGJFAAAA|22882|6|21|22|AM|third|morning|breakfast| +22883|AAAAAAAAEGJFAAAA|22883|6|21|23|AM|third|morning|breakfast| +22884|AAAAAAAAFGJFAAAA|22884|6|21|24|AM|third|morning|breakfast| +22885|AAAAAAAAGGJFAAAA|22885|6|21|25|AM|third|morning|breakfast| +22886|AAAAAAAAHGJFAAAA|22886|6|21|26|AM|third|morning|breakfast| +22887|AAAAAAAAIGJFAAAA|22887|6|21|27|AM|third|morning|breakfast| +22888|AAAAAAAAJGJFAAAA|22888|6|21|28|AM|third|morning|breakfast| +22889|AAAAAAAAKGJFAAAA|22889|6|21|29|AM|third|morning|breakfast| +22890|AAAAAAAALGJFAAAA|22890|6|21|30|AM|third|morning|breakfast| +22891|AAAAAAAAMGJFAAAA|22891|6|21|31|AM|third|morning|breakfast| +22892|AAAAAAAANGJFAAAA|22892|6|21|32|AM|third|morning|breakfast| +22893|AAAAAAAAOGJFAAAA|22893|6|21|33|AM|third|morning|breakfast| +22894|AAAAAAAAPGJFAAAA|22894|6|21|34|AM|third|morning|breakfast| +22895|AAAAAAAAAHJFAAAA|22895|6|21|35|AM|third|morning|breakfast| +22896|AAAAAAAABHJFAAAA|22896|6|21|36|AM|third|morning|breakfast| +22897|AAAAAAAACHJFAAAA|22897|6|21|37|AM|third|morning|breakfast| +22898|AAAAAAAADHJFAAAA|22898|6|21|38|AM|third|morning|breakfast| +22899|AAAAAAAAEHJFAAAA|22899|6|21|39|AM|third|morning|breakfast| +22900|AAAAAAAAFHJFAAAA|22900|6|21|40|AM|third|morning|breakfast| +22901|AAAAAAAAGHJFAAAA|22901|6|21|41|AM|third|morning|breakfast| +22902|AAAAAAAAHHJFAAAA|22902|6|21|42|AM|third|morning|breakfast| +22903|AAAAAAAAIHJFAAAA|22903|6|21|43|AM|third|morning|breakfast| +22904|AAAAAAAAJHJFAAAA|22904|6|21|44|AM|third|morning|breakfast| +22905|AAAAAAAAKHJFAAAA|22905|6|21|45|AM|third|morning|breakfast| +22906|AAAAAAAALHJFAAAA|22906|6|21|46|AM|third|morning|breakfast| +22907|AAAAAAAAMHJFAAAA|22907|6|21|47|AM|third|morning|breakfast| +22908|AAAAAAAANHJFAAAA|22908|6|21|48|AM|third|morning|breakfast| +22909|AAAAAAAAOHJFAAAA|22909|6|21|49|AM|third|morning|breakfast| +22910|AAAAAAAAPHJFAAAA|22910|6|21|50|AM|third|morning|breakfast| +22911|AAAAAAAAAIJFAAAA|22911|6|21|51|AM|third|morning|breakfast| +22912|AAAAAAAABIJFAAAA|22912|6|21|52|AM|third|morning|breakfast| +22913|AAAAAAAACIJFAAAA|22913|6|21|53|AM|third|morning|breakfast| +22914|AAAAAAAADIJFAAAA|22914|6|21|54|AM|third|morning|breakfast| +22915|AAAAAAAAEIJFAAAA|22915|6|21|55|AM|third|morning|breakfast| +22916|AAAAAAAAFIJFAAAA|22916|6|21|56|AM|third|morning|breakfast| +22917|AAAAAAAAGIJFAAAA|22917|6|21|57|AM|third|morning|breakfast| +22918|AAAAAAAAHIJFAAAA|22918|6|21|58|AM|third|morning|breakfast| +22919|AAAAAAAAIIJFAAAA|22919|6|21|59|AM|third|morning|breakfast| +22920|AAAAAAAAJIJFAAAA|22920|6|22|0|AM|third|morning|breakfast| +22921|AAAAAAAAKIJFAAAA|22921|6|22|1|AM|third|morning|breakfast| +22922|AAAAAAAALIJFAAAA|22922|6|22|2|AM|third|morning|breakfast| +22923|AAAAAAAAMIJFAAAA|22923|6|22|3|AM|third|morning|breakfast| +22924|AAAAAAAANIJFAAAA|22924|6|22|4|AM|third|morning|breakfast| +22925|AAAAAAAAOIJFAAAA|22925|6|22|5|AM|third|morning|breakfast| +22926|AAAAAAAAPIJFAAAA|22926|6|22|6|AM|third|morning|breakfast| +22927|AAAAAAAAAJJFAAAA|22927|6|22|7|AM|third|morning|breakfast| +22928|AAAAAAAABJJFAAAA|22928|6|22|8|AM|third|morning|breakfast| +22929|AAAAAAAACJJFAAAA|22929|6|22|9|AM|third|morning|breakfast| +22930|AAAAAAAADJJFAAAA|22930|6|22|10|AM|third|morning|breakfast| +22931|AAAAAAAAEJJFAAAA|22931|6|22|11|AM|third|morning|breakfast| +22932|AAAAAAAAFJJFAAAA|22932|6|22|12|AM|third|morning|breakfast| +22933|AAAAAAAAGJJFAAAA|22933|6|22|13|AM|third|morning|breakfast| +22934|AAAAAAAAHJJFAAAA|22934|6|22|14|AM|third|morning|breakfast| +22935|AAAAAAAAIJJFAAAA|22935|6|22|15|AM|third|morning|breakfast| +22936|AAAAAAAAJJJFAAAA|22936|6|22|16|AM|third|morning|breakfast| +22937|AAAAAAAAKJJFAAAA|22937|6|22|17|AM|third|morning|breakfast| +22938|AAAAAAAALJJFAAAA|22938|6|22|18|AM|third|morning|breakfast| +22939|AAAAAAAAMJJFAAAA|22939|6|22|19|AM|third|morning|breakfast| +22940|AAAAAAAANJJFAAAA|22940|6|22|20|AM|third|morning|breakfast| +22941|AAAAAAAAOJJFAAAA|22941|6|22|21|AM|third|morning|breakfast| +22942|AAAAAAAAPJJFAAAA|22942|6|22|22|AM|third|morning|breakfast| +22943|AAAAAAAAAKJFAAAA|22943|6|22|23|AM|third|morning|breakfast| +22944|AAAAAAAABKJFAAAA|22944|6|22|24|AM|third|morning|breakfast| +22945|AAAAAAAACKJFAAAA|22945|6|22|25|AM|third|morning|breakfast| +22946|AAAAAAAADKJFAAAA|22946|6|22|26|AM|third|morning|breakfast| +22947|AAAAAAAAEKJFAAAA|22947|6|22|27|AM|third|morning|breakfast| +22948|AAAAAAAAFKJFAAAA|22948|6|22|28|AM|third|morning|breakfast| +22949|AAAAAAAAGKJFAAAA|22949|6|22|29|AM|third|morning|breakfast| +22950|AAAAAAAAHKJFAAAA|22950|6|22|30|AM|third|morning|breakfast| +22951|AAAAAAAAIKJFAAAA|22951|6|22|31|AM|third|morning|breakfast| +22952|AAAAAAAAJKJFAAAA|22952|6|22|32|AM|third|morning|breakfast| +22953|AAAAAAAAKKJFAAAA|22953|6|22|33|AM|third|morning|breakfast| +22954|AAAAAAAALKJFAAAA|22954|6|22|34|AM|third|morning|breakfast| +22955|AAAAAAAAMKJFAAAA|22955|6|22|35|AM|third|morning|breakfast| +22956|AAAAAAAANKJFAAAA|22956|6|22|36|AM|third|morning|breakfast| +22957|AAAAAAAAOKJFAAAA|22957|6|22|37|AM|third|morning|breakfast| +22958|AAAAAAAAPKJFAAAA|22958|6|22|38|AM|third|morning|breakfast| +22959|AAAAAAAAALJFAAAA|22959|6|22|39|AM|third|morning|breakfast| +22960|AAAAAAAABLJFAAAA|22960|6|22|40|AM|third|morning|breakfast| +22961|AAAAAAAACLJFAAAA|22961|6|22|41|AM|third|morning|breakfast| +22962|AAAAAAAADLJFAAAA|22962|6|22|42|AM|third|morning|breakfast| +22963|AAAAAAAAELJFAAAA|22963|6|22|43|AM|third|morning|breakfast| +22964|AAAAAAAAFLJFAAAA|22964|6|22|44|AM|third|morning|breakfast| +22965|AAAAAAAAGLJFAAAA|22965|6|22|45|AM|third|morning|breakfast| +22966|AAAAAAAAHLJFAAAA|22966|6|22|46|AM|third|morning|breakfast| +22967|AAAAAAAAILJFAAAA|22967|6|22|47|AM|third|morning|breakfast| +22968|AAAAAAAAJLJFAAAA|22968|6|22|48|AM|third|morning|breakfast| +22969|AAAAAAAAKLJFAAAA|22969|6|22|49|AM|third|morning|breakfast| +22970|AAAAAAAALLJFAAAA|22970|6|22|50|AM|third|morning|breakfast| +22971|AAAAAAAAMLJFAAAA|22971|6|22|51|AM|third|morning|breakfast| +22972|AAAAAAAANLJFAAAA|22972|6|22|52|AM|third|morning|breakfast| +22973|AAAAAAAAOLJFAAAA|22973|6|22|53|AM|third|morning|breakfast| +22974|AAAAAAAAPLJFAAAA|22974|6|22|54|AM|third|morning|breakfast| +22975|AAAAAAAAAMJFAAAA|22975|6|22|55|AM|third|morning|breakfast| +22976|AAAAAAAABMJFAAAA|22976|6|22|56|AM|third|morning|breakfast| +22977|AAAAAAAACMJFAAAA|22977|6|22|57|AM|third|morning|breakfast| +22978|AAAAAAAADMJFAAAA|22978|6|22|58|AM|third|morning|breakfast| +22979|AAAAAAAAEMJFAAAA|22979|6|22|59|AM|third|morning|breakfast| +22980|AAAAAAAAFMJFAAAA|22980|6|23|0|AM|third|morning|breakfast| +22981|AAAAAAAAGMJFAAAA|22981|6|23|1|AM|third|morning|breakfast| +22982|AAAAAAAAHMJFAAAA|22982|6|23|2|AM|third|morning|breakfast| +22983|AAAAAAAAIMJFAAAA|22983|6|23|3|AM|third|morning|breakfast| +22984|AAAAAAAAJMJFAAAA|22984|6|23|4|AM|third|morning|breakfast| +22985|AAAAAAAAKMJFAAAA|22985|6|23|5|AM|third|morning|breakfast| +22986|AAAAAAAALMJFAAAA|22986|6|23|6|AM|third|morning|breakfast| +22987|AAAAAAAAMMJFAAAA|22987|6|23|7|AM|third|morning|breakfast| +22988|AAAAAAAANMJFAAAA|22988|6|23|8|AM|third|morning|breakfast| +22989|AAAAAAAAOMJFAAAA|22989|6|23|9|AM|third|morning|breakfast| +22990|AAAAAAAAPMJFAAAA|22990|6|23|10|AM|third|morning|breakfast| +22991|AAAAAAAAANJFAAAA|22991|6|23|11|AM|third|morning|breakfast| +22992|AAAAAAAABNJFAAAA|22992|6|23|12|AM|third|morning|breakfast| +22993|AAAAAAAACNJFAAAA|22993|6|23|13|AM|third|morning|breakfast| +22994|AAAAAAAADNJFAAAA|22994|6|23|14|AM|third|morning|breakfast| +22995|AAAAAAAAENJFAAAA|22995|6|23|15|AM|third|morning|breakfast| +22996|AAAAAAAAFNJFAAAA|22996|6|23|16|AM|third|morning|breakfast| +22997|AAAAAAAAGNJFAAAA|22997|6|23|17|AM|third|morning|breakfast| +22998|AAAAAAAAHNJFAAAA|22998|6|23|18|AM|third|morning|breakfast| +22999|AAAAAAAAINJFAAAA|22999|6|23|19|AM|third|morning|breakfast| +23000|AAAAAAAAJNJFAAAA|23000|6|23|20|AM|third|morning|breakfast| +23001|AAAAAAAAKNJFAAAA|23001|6|23|21|AM|third|morning|breakfast| +23002|AAAAAAAALNJFAAAA|23002|6|23|22|AM|third|morning|breakfast| +23003|AAAAAAAAMNJFAAAA|23003|6|23|23|AM|third|morning|breakfast| +23004|AAAAAAAANNJFAAAA|23004|6|23|24|AM|third|morning|breakfast| +23005|AAAAAAAAONJFAAAA|23005|6|23|25|AM|third|morning|breakfast| +23006|AAAAAAAAPNJFAAAA|23006|6|23|26|AM|third|morning|breakfast| +23007|AAAAAAAAAOJFAAAA|23007|6|23|27|AM|third|morning|breakfast| +23008|AAAAAAAABOJFAAAA|23008|6|23|28|AM|third|morning|breakfast| +23009|AAAAAAAACOJFAAAA|23009|6|23|29|AM|third|morning|breakfast| +23010|AAAAAAAADOJFAAAA|23010|6|23|30|AM|third|morning|breakfast| +23011|AAAAAAAAEOJFAAAA|23011|6|23|31|AM|third|morning|breakfast| +23012|AAAAAAAAFOJFAAAA|23012|6|23|32|AM|third|morning|breakfast| +23013|AAAAAAAAGOJFAAAA|23013|6|23|33|AM|third|morning|breakfast| +23014|AAAAAAAAHOJFAAAA|23014|6|23|34|AM|third|morning|breakfast| +23015|AAAAAAAAIOJFAAAA|23015|6|23|35|AM|third|morning|breakfast| +23016|AAAAAAAAJOJFAAAA|23016|6|23|36|AM|third|morning|breakfast| +23017|AAAAAAAAKOJFAAAA|23017|6|23|37|AM|third|morning|breakfast| +23018|AAAAAAAALOJFAAAA|23018|6|23|38|AM|third|morning|breakfast| +23019|AAAAAAAAMOJFAAAA|23019|6|23|39|AM|third|morning|breakfast| +23020|AAAAAAAANOJFAAAA|23020|6|23|40|AM|third|morning|breakfast| +23021|AAAAAAAAOOJFAAAA|23021|6|23|41|AM|third|morning|breakfast| +23022|AAAAAAAAPOJFAAAA|23022|6|23|42|AM|third|morning|breakfast| +23023|AAAAAAAAAPJFAAAA|23023|6|23|43|AM|third|morning|breakfast| +23024|AAAAAAAABPJFAAAA|23024|6|23|44|AM|third|morning|breakfast| +23025|AAAAAAAACPJFAAAA|23025|6|23|45|AM|third|morning|breakfast| +23026|AAAAAAAADPJFAAAA|23026|6|23|46|AM|third|morning|breakfast| +23027|AAAAAAAAEPJFAAAA|23027|6|23|47|AM|third|morning|breakfast| +23028|AAAAAAAAFPJFAAAA|23028|6|23|48|AM|third|morning|breakfast| +23029|AAAAAAAAGPJFAAAA|23029|6|23|49|AM|third|morning|breakfast| +23030|AAAAAAAAHPJFAAAA|23030|6|23|50|AM|third|morning|breakfast| +23031|AAAAAAAAIPJFAAAA|23031|6|23|51|AM|third|morning|breakfast| +23032|AAAAAAAAJPJFAAAA|23032|6|23|52|AM|third|morning|breakfast| +23033|AAAAAAAAKPJFAAAA|23033|6|23|53|AM|third|morning|breakfast| +23034|AAAAAAAALPJFAAAA|23034|6|23|54|AM|third|morning|breakfast| +23035|AAAAAAAAMPJFAAAA|23035|6|23|55|AM|third|morning|breakfast| +23036|AAAAAAAANPJFAAAA|23036|6|23|56|AM|third|morning|breakfast| +23037|AAAAAAAAOPJFAAAA|23037|6|23|57|AM|third|morning|breakfast| +23038|AAAAAAAAPPJFAAAA|23038|6|23|58|AM|third|morning|breakfast| +23039|AAAAAAAAAAKFAAAA|23039|6|23|59|AM|third|morning|breakfast| +23040|AAAAAAAABAKFAAAA|23040|6|24|0|AM|third|morning|breakfast| +23041|AAAAAAAACAKFAAAA|23041|6|24|1|AM|third|morning|breakfast| +23042|AAAAAAAADAKFAAAA|23042|6|24|2|AM|third|morning|breakfast| +23043|AAAAAAAAEAKFAAAA|23043|6|24|3|AM|third|morning|breakfast| +23044|AAAAAAAAFAKFAAAA|23044|6|24|4|AM|third|morning|breakfast| +23045|AAAAAAAAGAKFAAAA|23045|6|24|5|AM|third|morning|breakfast| +23046|AAAAAAAAHAKFAAAA|23046|6|24|6|AM|third|morning|breakfast| +23047|AAAAAAAAIAKFAAAA|23047|6|24|7|AM|third|morning|breakfast| +23048|AAAAAAAAJAKFAAAA|23048|6|24|8|AM|third|morning|breakfast| +23049|AAAAAAAAKAKFAAAA|23049|6|24|9|AM|third|morning|breakfast| +23050|AAAAAAAALAKFAAAA|23050|6|24|10|AM|third|morning|breakfast| +23051|AAAAAAAAMAKFAAAA|23051|6|24|11|AM|third|morning|breakfast| +23052|AAAAAAAANAKFAAAA|23052|6|24|12|AM|third|morning|breakfast| +23053|AAAAAAAAOAKFAAAA|23053|6|24|13|AM|third|morning|breakfast| +23054|AAAAAAAAPAKFAAAA|23054|6|24|14|AM|third|morning|breakfast| +23055|AAAAAAAAABKFAAAA|23055|6|24|15|AM|third|morning|breakfast| +23056|AAAAAAAABBKFAAAA|23056|6|24|16|AM|third|morning|breakfast| +23057|AAAAAAAACBKFAAAA|23057|6|24|17|AM|third|morning|breakfast| +23058|AAAAAAAADBKFAAAA|23058|6|24|18|AM|third|morning|breakfast| +23059|AAAAAAAAEBKFAAAA|23059|6|24|19|AM|third|morning|breakfast| +23060|AAAAAAAAFBKFAAAA|23060|6|24|20|AM|third|morning|breakfast| +23061|AAAAAAAAGBKFAAAA|23061|6|24|21|AM|third|morning|breakfast| +23062|AAAAAAAAHBKFAAAA|23062|6|24|22|AM|third|morning|breakfast| +23063|AAAAAAAAIBKFAAAA|23063|6|24|23|AM|third|morning|breakfast| +23064|AAAAAAAAJBKFAAAA|23064|6|24|24|AM|third|morning|breakfast| +23065|AAAAAAAAKBKFAAAA|23065|6|24|25|AM|third|morning|breakfast| +23066|AAAAAAAALBKFAAAA|23066|6|24|26|AM|third|morning|breakfast| +23067|AAAAAAAAMBKFAAAA|23067|6|24|27|AM|third|morning|breakfast| +23068|AAAAAAAANBKFAAAA|23068|6|24|28|AM|third|morning|breakfast| +23069|AAAAAAAAOBKFAAAA|23069|6|24|29|AM|third|morning|breakfast| +23070|AAAAAAAAPBKFAAAA|23070|6|24|30|AM|third|morning|breakfast| +23071|AAAAAAAAACKFAAAA|23071|6|24|31|AM|third|morning|breakfast| +23072|AAAAAAAABCKFAAAA|23072|6|24|32|AM|third|morning|breakfast| +23073|AAAAAAAACCKFAAAA|23073|6|24|33|AM|third|morning|breakfast| +23074|AAAAAAAADCKFAAAA|23074|6|24|34|AM|third|morning|breakfast| +23075|AAAAAAAAECKFAAAA|23075|6|24|35|AM|third|morning|breakfast| +23076|AAAAAAAAFCKFAAAA|23076|6|24|36|AM|third|morning|breakfast| +23077|AAAAAAAAGCKFAAAA|23077|6|24|37|AM|third|morning|breakfast| +23078|AAAAAAAAHCKFAAAA|23078|6|24|38|AM|third|morning|breakfast| +23079|AAAAAAAAICKFAAAA|23079|6|24|39|AM|third|morning|breakfast| +23080|AAAAAAAAJCKFAAAA|23080|6|24|40|AM|third|morning|breakfast| +23081|AAAAAAAAKCKFAAAA|23081|6|24|41|AM|third|morning|breakfast| +23082|AAAAAAAALCKFAAAA|23082|6|24|42|AM|third|morning|breakfast| +23083|AAAAAAAAMCKFAAAA|23083|6|24|43|AM|third|morning|breakfast| +23084|AAAAAAAANCKFAAAA|23084|6|24|44|AM|third|morning|breakfast| +23085|AAAAAAAAOCKFAAAA|23085|6|24|45|AM|third|morning|breakfast| +23086|AAAAAAAAPCKFAAAA|23086|6|24|46|AM|third|morning|breakfast| +23087|AAAAAAAAADKFAAAA|23087|6|24|47|AM|third|morning|breakfast| +23088|AAAAAAAABDKFAAAA|23088|6|24|48|AM|third|morning|breakfast| +23089|AAAAAAAACDKFAAAA|23089|6|24|49|AM|third|morning|breakfast| +23090|AAAAAAAADDKFAAAA|23090|6|24|50|AM|third|morning|breakfast| +23091|AAAAAAAAEDKFAAAA|23091|6|24|51|AM|third|morning|breakfast| +23092|AAAAAAAAFDKFAAAA|23092|6|24|52|AM|third|morning|breakfast| +23093|AAAAAAAAGDKFAAAA|23093|6|24|53|AM|third|morning|breakfast| +23094|AAAAAAAAHDKFAAAA|23094|6|24|54|AM|third|morning|breakfast| +23095|AAAAAAAAIDKFAAAA|23095|6|24|55|AM|third|morning|breakfast| +23096|AAAAAAAAJDKFAAAA|23096|6|24|56|AM|third|morning|breakfast| +23097|AAAAAAAAKDKFAAAA|23097|6|24|57|AM|third|morning|breakfast| +23098|AAAAAAAALDKFAAAA|23098|6|24|58|AM|third|morning|breakfast| +23099|AAAAAAAAMDKFAAAA|23099|6|24|59|AM|third|morning|breakfast| +23100|AAAAAAAANDKFAAAA|23100|6|25|0|AM|third|morning|breakfast| +23101|AAAAAAAAODKFAAAA|23101|6|25|1|AM|third|morning|breakfast| +23102|AAAAAAAAPDKFAAAA|23102|6|25|2|AM|third|morning|breakfast| +23103|AAAAAAAAAEKFAAAA|23103|6|25|3|AM|third|morning|breakfast| +23104|AAAAAAAABEKFAAAA|23104|6|25|4|AM|third|morning|breakfast| +23105|AAAAAAAACEKFAAAA|23105|6|25|5|AM|third|morning|breakfast| +23106|AAAAAAAADEKFAAAA|23106|6|25|6|AM|third|morning|breakfast| +23107|AAAAAAAAEEKFAAAA|23107|6|25|7|AM|third|morning|breakfast| +23108|AAAAAAAAFEKFAAAA|23108|6|25|8|AM|third|morning|breakfast| +23109|AAAAAAAAGEKFAAAA|23109|6|25|9|AM|third|morning|breakfast| +23110|AAAAAAAAHEKFAAAA|23110|6|25|10|AM|third|morning|breakfast| +23111|AAAAAAAAIEKFAAAA|23111|6|25|11|AM|third|morning|breakfast| +23112|AAAAAAAAJEKFAAAA|23112|6|25|12|AM|third|morning|breakfast| +23113|AAAAAAAAKEKFAAAA|23113|6|25|13|AM|third|morning|breakfast| +23114|AAAAAAAALEKFAAAA|23114|6|25|14|AM|third|morning|breakfast| +23115|AAAAAAAAMEKFAAAA|23115|6|25|15|AM|third|morning|breakfast| +23116|AAAAAAAANEKFAAAA|23116|6|25|16|AM|third|morning|breakfast| +23117|AAAAAAAAOEKFAAAA|23117|6|25|17|AM|third|morning|breakfast| +23118|AAAAAAAAPEKFAAAA|23118|6|25|18|AM|third|morning|breakfast| +23119|AAAAAAAAAFKFAAAA|23119|6|25|19|AM|third|morning|breakfast| +23120|AAAAAAAABFKFAAAA|23120|6|25|20|AM|third|morning|breakfast| +23121|AAAAAAAACFKFAAAA|23121|6|25|21|AM|third|morning|breakfast| +23122|AAAAAAAADFKFAAAA|23122|6|25|22|AM|third|morning|breakfast| +23123|AAAAAAAAEFKFAAAA|23123|6|25|23|AM|third|morning|breakfast| +23124|AAAAAAAAFFKFAAAA|23124|6|25|24|AM|third|morning|breakfast| +23125|AAAAAAAAGFKFAAAA|23125|6|25|25|AM|third|morning|breakfast| +23126|AAAAAAAAHFKFAAAA|23126|6|25|26|AM|third|morning|breakfast| +23127|AAAAAAAAIFKFAAAA|23127|6|25|27|AM|third|morning|breakfast| +23128|AAAAAAAAJFKFAAAA|23128|6|25|28|AM|third|morning|breakfast| +23129|AAAAAAAAKFKFAAAA|23129|6|25|29|AM|third|morning|breakfast| +23130|AAAAAAAALFKFAAAA|23130|6|25|30|AM|third|morning|breakfast| +23131|AAAAAAAAMFKFAAAA|23131|6|25|31|AM|third|morning|breakfast| +23132|AAAAAAAANFKFAAAA|23132|6|25|32|AM|third|morning|breakfast| +23133|AAAAAAAAOFKFAAAA|23133|6|25|33|AM|third|morning|breakfast| +23134|AAAAAAAAPFKFAAAA|23134|6|25|34|AM|third|morning|breakfast| +23135|AAAAAAAAAGKFAAAA|23135|6|25|35|AM|third|morning|breakfast| +23136|AAAAAAAABGKFAAAA|23136|6|25|36|AM|third|morning|breakfast| +23137|AAAAAAAACGKFAAAA|23137|6|25|37|AM|third|morning|breakfast| +23138|AAAAAAAADGKFAAAA|23138|6|25|38|AM|third|morning|breakfast| +23139|AAAAAAAAEGKFAAAA|23139|6|25|39|AM|third|morning|breakfast| +23140|AAAAAAAAFGKFAAAA|23140|6|25|40|AM|third|morning|breakfast| +23141|AAAAAAAAGGKFAAAA|23141|6|25|41|AM|third|morning|breakfast| +23142|AAAAAAAAHGKFAAAA|23142|6|25|42|AM|third|morning|breakfast| +23143|AAAAAAAAIGKFAAAA|23143|6|25|43|AM|third|morning|breakfast| +23144|AAAAAAAAJGKFAAAA|23144|6|25|44|AM|third|morning|breakfast| +23145|AAAAAAAAKGKFAAAA|23145|6|25|45|AM|third|morning|breakfast| +23146|AAAAAAAALGKFAAAA|23146|6|25|46|AM|third|morning|breakfast| +23147|AAAAAAAAMGKFAAAA|23147|6|25|47|AM|third|morning|breakfast| +23148|AAAAAAAANGKFAAAA|23148|6|25|48|AM|third|morning|breakfast| +23149|AAAAAAAAOGKFAAAA|23149|6|25|49|AM|third|morning|breakfast| +23150|AAAAAAAAPGKFAAAA|23150|6|25|50|AM|third|morning|breakfast| +23151|AAAAAAAAAHKFAAAA|23151|6|25|51|AM|third|morning|breakfast| +23152|AAAAAAAABHKFAAAA|23152|6|25|52|AM|third|morning|breakfast| +23153|AAAAAAAACHKFAAAA|23153|6|25|53|AM|third|morning|breakfast| +23154|AAAAAAAADHKFAAAA|23154|6|25|54|AM|third|morning|breakfast| +23155|AAAAAAAAEHKFAAAA|23155|6|25|55|AM|third|morning|breakfast| +23156|AAAAAAAAFHKFAAAA|23156|6|25|56|AM|third|morning|breakfast| +23157|AAAAAAAAGHKFAAAA|23157|6|25|57|AM|third|morning|breakfast| +23158|AAAAAAAAHHKFAAAA|23158|6|25|58|AM|third|morning|breakfast| +23159|AAAAAAAAIHKFAAAA|23159|6|25|59|AM|third|morning|breakfast| +23160|AAAAAAAAJHKFAAAA|23160|6|26|0|AM|third|morning|breakfast| +23161|AAAAAAAAKHKFAAAA|23161|6|26|1|AM|third|morning|breakfast| +23162|AAAAAAAALHKFAAAA|23162|6|26|2|AM|third|morning|breakfast| +23163|AAAAAAAAMHKFAAAA|23163|6|26|3|AM|third|morning|breakfast| +23164|AAAAAAAANHKFAAAA|23164|6|26|4|AM|third|morning|breakfast| +23165|AAAAAAAAOHKFAAAA|23165|6|26|5|AM|third|morning|breakfast| +23166|AAAAAAAAPHKFAAAA|23166|6|26|6|AM|third|morning|breakfast| +23167|AAAAAAAAAIKFAAAA|23167|6|26|7|AM|third|morning|breakfast| +23168|AAAAAAAABIKFAAAA|23168|6|26|8|AM|third|morning|breakfast| +23169|AAAAAAAACIKFAAAA|23169|6|26|9|AM|third|morning|breakfast| +23170|AAAAAAAADIKFAAAA|23170|6|26|10|AM|third|morning|breakfast| +23171|AAAAAAAAEIKFAAAA|23171|6|26|11|AM|third|morning|breakfast| +23172|AAAAAAAAFIKFAAAA|23172|6|26|12|AM|third|morning|breakfast| +23173|AAAAAAAAGIKFAAAA|23173|6|26|13|AM|third|morning|breakfast| +23174|AAAAAAAAHIKFAAAA|23174|6|26|14|AM|third|morning|breakfast| +23175|AAAAAAAAIIKFAAAA|23175|6|26|15|AM|third|morning|breakfast| +23176|AAAAAAAAJIKFAAAA|23176|6|26|16|AM|third|morning|breakfast| +23177|AAAAAAAAKIKFAAAA|23177|6|26|17|AM|third|morning|breakfast| +23178|AAAAAAAALIKFAAAA|23178|6|26|18|AM|third|morning|breakfast| +23179|AAAAAAAAMIKFAAAA|23179|6|26|19|AM|third|morning|breakfast| +23180|AAAAAAAANIKFAAAA|23180|6|26|20|AM|third|morning|breakfast| +23181|AAAAAAAAOIKFAAAA|23181|6|26|21|AM|third|morning|breakfast| +23182|AAAAAAAAPIKFAAAA|23182|6|26|22|AM|third|morning|breakfast| +23183|AAAAAAAAAJKFAAAA|23183|6|26|23|AM|third|morning|breakfast| +23184|AAAAAAAABJKFAAAA|23184|6|26|24|AM|third|morning|breakfast| +23185|AAAAAAAACJKFAAAA|23185|6|26|25|AM|third|morning|breakfast| +23186|AAAAAAAADJKFAAAA|23186|6|26|26|AM|third|morning|breakfast| +23187|AAAAAAAAEJKFAAAA|23187|6|26|27|AM|third|morning|breakfast| +23188|AAAAAAAAFJKFAAAA|23188|6|26|28|AM|third|morning|breakfast| +23189|AAAAAAAAGJKFAAAA|23189|6|26|29|AM|third|morning|breakfast| +23190|AAAAAAAAHJKFAAAA|23190|6|26|30|AM|third|morning|breakfast| +23191|AAAAAAAAIJKFAAAA|23191|6|26|31|AM|third|morning|breakfast| +23192|AAAAAAAAJJKFAAAA|23192|6|26|32|AM|third|morning|breakfast| +23193|AAAAAAAAKJKFAAAA|23193|6|26|33|AM|third|morning|breakfast| +23194|AAAAAAAALJKFAAAA|23194|6|26|34|AM|third|morning|breakfast| +23195|AAAAAAAAMJKFAAAA|23195|6|26|35|AM|third|morning|breakfast| +23196|AAAAAAAANJKFAAAA|23196|6|26|36|AM|third|morning|breakfast| +23197|AAAAAAAAOJKFAAAA|23197|6|26|37|AM|third|morning|breakfast| +23198|AAAAAAAAPJKFAAAA|23198|6|26|38|AM|third|morning|breakfast| +23199|AAAAAAAAAKKFAAAA|23199|6|26|39|AM|third|morning|breakfast| +23200|AAAAAAAABKKFAAAA|23200|6|26|40|AM|third|morning|breakfast| +23201|AAAAAAAACKKFAAAA|23201|6|26|41|AM|third|morning|breakfast| +23202|AAAAAAAADKKFAAAA|23202|6|26|42|AM|third|morning|breakfast| +23203|AAAAAAAAEKKFAAAA|23203|6|26|43|AM|third|morning|breakfast| +23204|AAAAAAAAFKKFAAAA|23204|6|26|44|AM|third|morning|breakfast| +23205|AAAAAAAAGKKFAAAA|23205|6|26|45|AM|third|morning|breakfast| +23206|AAAAAAAAHKKFAAAA|23206|6|26|46|AM|third|morning|breakfast| +23207|AAAAAAAAIKKFAAAA|23207|6|26|47|AM|third|morning|breakfast| +23208|AAAAAAAAJKKFAAAA|23208|6|26|48|AM|third|morning|breakfast| +23209|AAAAAAAAKKKFAAAA|23209|6|26|49|AM|third|morning|breakfast| +23210|AAAAAAAALKKFAAAA|23210|6|26|50|AM|third|morning|breakfast| +23211|AAAAAAAAMKKFAAAA|23211|6|26|51|AM|third|morning|breakfast| +23212|AAAAAAAANKKFAAAA|23212|6|26|52|AM|third|morning|breakfast| +23213|AAAAAAAAOKKFAAAA|23213|6|26|53|AM|third|morning|breakfast| +23214|AAAAAAAAPKKFAAAA|23214|6|26|54|AM|third|morning|breakfast| +23215|AAAAAAAAALKFAAAA|23215|6|26|55|AM|third|morning|breakfast| +23216|AAAAAAAABLKFAAAA|23216|6|26|56|AM|third|morning|breakfast| +23217|AAAAAAAACLKFAAAA|23217|6|26|57|AM|third|morning|breakfast| +23218|AAAAAAAADLKFAAAA|23218|6|26|58|AM|third|morning|breakfast| +23219|AAAAAAAAELKFAAAA|23219|6|26|59|AM|third|morning|breakfast| +23220|AAAAAAAAFLKFAAAA|23220|6|27|0|AM|third|morning|breakfast| +23221|AAAAAAAAGLKFAAAA|23221|6|27|1|AM|third|morning|breakfast| +23222|AAAAAAAAHLKFAAAA|23222|6|27|2|AM|third|morning|breakfast| +23223|AAAAAAAAILKFAAAA|23223|6|27|3|AM|third|morning|breakfast| +23224|AAAAAAAAJLKFAAAA|23224|6|27|4|AM|third|morning|breakfast| +23225|AAAAAAAAKLKFAAAA|23225|6|27|5|AM|third|morning|breakfast| +23226|AAAAAAAALLKFAAAA|23226|6|27|6|AM|third|morning|breakfast| +23227|AAAAAAAAMLKFAAAA|23227|6|27|7|AM|third|morning|breakfast| +23228|AAAAAAAANLKFAAAA|23228|6|27|8|AM|third|morning|breakfast| +23229|AAAAAAAAOLKFAAAA|23229|6|27|9|AM|third|morning|breakfast| +23230|AAAAAAAAPLKFAAAA|23230|6|27|10|AM|third|morning|breakfast| +23231|AAAAAAAAAMKFAAAA|23231|6|27|11|AM|third|morning|breakfast| +23232|AAAAAAAABMKFAAAA|23232|6|27|12|AM|third|morning|breakfast| +23233|AAAAAAAACMKFAAAA|23233|6|27|13|AM|third|morning|breakfast| +23234|AAAAAAAADMKFAAAA|23234|6|27|14|AM|third|morning|breakfast| +23235|AAAAAAAAEMKFAAAA|23235|6|27|15|AM|third|morning|breakfast| +23236|AAAAAAAAFMKFAAAA|23236|6|27|16|AM|third|morning|breakfast| +23237|AAAAAAAAGMKFAAAA|23237|6|27|17|AM|third|morning|breakfast| +23238|AAAAAAAAHMKFAAAA|23238|6|27|18|AM|third|morning|breakfast| +23239|AAAAAAAAIMKFAAAA|23239|6|27|19|AM|third|morning|breakfast| +23240|AAAAAAAAJMKFAAAA|23240|6|27|20|AM|third|morning|breakfast| +23241|AAAAAAAAKMKFAAAA|23241|6|27|21|AM|third|morning|breakfast| +23242|AAAAAAAALMKFAAAA|23242|6|27|22|AM|third|morning|breakfast| +23243|AAAAAAAAMMKFAAAA|23243|6|27|23|AM|third|morning|breakfast| +23244|AAAAAAAANMKFAAAA|23244|6|27|24|AM|third|morning|breakfast| +23245|AAAAAAAAOMKFAAAA|23245|6|27|25|AM|third|morning|breakfast| +23246|AAAAAAAAPMKFAAAA|23246|6|27|26|AM|third|morning|breakfast| +23247|AAAAAAAAANKFAAAA|23247|6|27|27|AM|third|morning|breakfast| +23248|AAAAAAAABNKFAAAA|23248|6|27|28|AM|third|morning|breakfast| +23249|AAAAAAAACNKFAAAA|23249|6|27|29|AM|third|morning|breakfast| +23250|AAAAAAAADNKFAAAA|23250|6|27|30|AM|third|morning|breakfast| +23251|AAAAAAAAENKFAAAA|23251|6|27|31|AM|third|morning|breakfast| +23252|AAAAAAAAFNKFAAAA|23252|6|27|32|AM|third|morning|breakfast| +23253|AAAAAAAAGNKFAAAA|23253|6|27|33|AM|third|morning|breakfast| +23254|AAAAAAAAHNKFAAAA|23254|6|27|34|AM|third|morning|breakfast| +23255|AAAAAAAAINKFAAAA|23255|6|27|35|AM|third|morning|breakfast| +23256|AAAAAAAAJNKFAAAA|23256|6|27|36|AM|third|morning|breakfast| +23257|AAAAAAAAKNKFAAAA|23257|6|27|37|AM|third|morning|breakfast| +23258|AAAAAAAALNKFAAAA|23258|6|27|38|AM|third|morning|breakfast| +23259|AAAAAAAAMNKFAAAA|23259|6|27|39|AM|third|morning|breakfast| +23260|AAAAAAAANNKFAAAA|23260|6|27|40|AM|third|morning|breakfast| +23261|AAAAAAAAONKFAAAA|23261|6|27|41|AM|third|morning|breakfast| +23262|AAAAAAAAPNKFAAAA|23262|6|27|42|AM|third|morning|breakfast| +23263|AAAAAAAAAOKFAAAA|23263|6|27|43|AM|third|morning|breakfast| +23264|AAAAAAAABOKFAAAA|23264|6|27|44|AM|third|morning|breakfast| +23265|AAAAAAAACOKFAAAA|23265|6|27|45|AM|third|morning|breakfast| +23266|AAAAAAAADOKFAAAA|23266|6|27|46|AM|third|morning|breakfast| +23267|AAAAAAAAEOKFAAAA|23267|6|27|47|AM|third|morning|breakfast| +23268|AAAAAAAAFOKFAAAA|23268|6|27|48|AM|third|morning|breakfast| +23269|AAAAAAAAGOKFAAAA|23269|6|27|49|AM|third|morning|breakfast| +23270|AAAAAAAAHOKFAAAA|23270|6|27|50|AM|third|morning|breakfast| +23271|AAAAAAAAIOKFAAAA|23271|6|27|51|AM|third|morning|breakfast| +23272|AAAAAAAAJOKFAAAA|23272|6|27|52|AM|third|morning|breakfast| +23273|AAAAAAAAKOKFAAAA|23273|6|27|53|AM|third|morning|breakfast| +23274|AAAAAAAALOKFAAAA|23274|6|27|54|AM|third|morning|breakfast| +23275|AAAAAAAAMOKFAAAA|23275|6|27|55|AM|third|morning|breakfast| +23276|AAAAAAAANOKFAAAA|23276|6|27|56|AM|third|morning|breakfast| +23277|AAAAAAAAOOKFAAAA|23277|6|27|57|AM|third|morning|breakfast| +23278|AAAAAAAAPOKFAAAA|23278|6|27|58|AM|third|morning|breakfast| +23279|AAAAAAAAAPKFAAAA|23279|6|27|59|AM|third|morning|breakfast| +23280|AAAAAAAABPKFAAAA|23280|6|28|0|AM|third|morning|breakfast| +23281|AAAAAAAACPKFAAAA|23281|6|28|1|AM|third|morning|breakfast| +23282|AAAAAAAADPKFAAAA|23282|6|28|2|AM|third|morning|breakfast| +23283|AAAAAAAAEPKFAAAA|23283|6|28|3|AM|third|morning|breakfast| +23284|AAAAAAAAFPKFAAAA|23284|6|28|4|AM|third|morning|breakfast| +23285|AAAAAAAAGPKFAAAA|23285|6|28|5|AM|third|morning|breakfast| +23286|AAAAAAAAHPKFAAAA|23286|6|28|6|AM|third|morning|breakfast| +23287|AAAAAAAAIPKFAAAA|23287|6|28|7|AM|third|morning|breakfast| +23288|AAAAAAAAJPKFAAAA|23288|6|28|8|AM|third|morning|breakfast| +23289|AAAAAAAAKPKFAAAA|23289|6|28|9|AM|third|morning|breakfast| +23290|AAAAAAAALPKFAAAA|23290|6|28|10|AM|third|morning|breakfast| +23291|AAAAAAAAMPKFAAAA|23291|6|28|11|AM|third|morning|breakfast| +23292|AAAAAAAANPKFAAAA|23292|6|28|12|AM|third|morning|breakfast| +23293|AAAAAAAAOPKFAAAA|23293|6|28|13|AM|third|morning|breakfast| +23294|AAAAAAAAPPKFAAAA|23294|6|28|14|AM|third|morning|breakfast| +23295|AAAAAAAAAALFAAAA|23295|6|28|15|AM|third|morning|breakfast| +23296|AAAAAAAABALFAAAA|23296|6|28|16|AM|third|morning|breakfast| +23297|AAAAAAAACALFAAAA|23297|6|28|17|AM|third|morning|breakfast| +23298|AAAAAAAADALFAAAA|23298|6|28|18|AM|third|morning|breakfast| +23299|AAAAAAAAEALFAAAA|23299|6|28|19|AM|third|morning|breakfast| +23300|AAAAAAAAFALFAAAA|23300|6|28|20|AM|third|morning|breakfast| +23301|AAAAAAAAGALFAAAA|23301|6|28|21|AM|third|morning|breakfast| +23302|AAAAAAAAHALFAAAA|23302|6|28|22|AM|third|morning|breakfast| +23303|AAAAAAAAIALFAAAA|23303|6|28|23|AM|third|morning|breakfast| +23304|AAAAAAAAJALFAAAA|23304|6|28|24|AM|third|morning|breakfast| +23305|AAAAAAAAKALFAAAA|23305|6|28|25|AM|third|morning|breakfast| +23306|AAAAAAAALALFAAAA|23306|6|28|26|AM|third|morning|breakfast| +23307|AAAAAAAAMALFAAAA|23307|6|28|27|AM|third|morning|breakfast| +23308|AAAAAAAANALFAAAA|23308|6|28|28|AM|third|morning|breakfast| +23309|AAAAAAAAOALFAAAA|23309|6|28|29|AM|third|morning|breakfast| +23310|AAAAAAAAPALFAAAA|23310|6|28|30|AM|third|morning|breakfast| +23311|AAAAAAAAABLFAAAA|23311|6|28|31|AM|third|morning|breakfast| +23312|AAAAAAAABBLFAAAA|23312|6|28|32|AM|third|morning|breakfast| +23313|AAAAAAAACBLFAAAA|23313|6|28|33|AM|third|morning|breakfast| +23314|AAAAAAAADBLFAAAA|23314|6|28|34|AM|third|morning|breakfast| +23315|AAAAAAAAEBLFAAAA|23315|6|28|35|AM|third|morning|breakfast| +23316|AAAAAAAAFBLFAAAA|23316|6|28|36|AM|third|morning|breakfast| +23317|AAAAAAAAGBLFAAAA|23317|6|28|37|AM|third|morning|breakfast| +23318|AAAAAAAAHBLFAAAA|23318|6|28|38|AM|third|morning|breakfast| +23319|AAAAAAAAIBLFAAAA|23319|6|28|39|AM|third|morning|breakfast| +23320|AAAAAAAAJBLFAAAA|23320|6|28|40|AM|third|morning|breakfast| +23321|AAAAAAAAKBLFAAAA|23321|6|28|41|AM|third|morning|breakfast| +23322|AAAAAAAALBLFAAAA|23322|6|28|42|AM|third|morning|breakfast| +23323|AAAAAAAAMBLFAAAA|23323|6|28|43|AM|third|morning|breakfast| +23324|AAAAAAAANBLFAAAA|23324|6|28|44|AM|third|morning|breakfast| +23325|AAAAAAAAOBLFAAAA|23325|6|28|45|AM|third|morning|breakfast| +23326|AAAAAAAAPBLFAAAA|23326|6|28|46|AM|third|morning|breakfast| +23327|AAAAAAAAACLFAAAA|23327|6|28|47|AM|third|morning|breakfast| +23328|AAAAAAAABCLFAAAA|23328|6|28|48|AM|third|morning|breakfast| +23329|AAAAAAAACCLFAAAA|23329|6|28|49|AM|third|morning|breakfast| +23330|AAAAAAAADCLFAAAA|23330|6|28|50|AM|third|morning|breakfast| +23331|AAAAAAAAECLFAAAA|23331|6|28|51|AM|third|morning|breakfast| +23332|AAAAAAAAFCLFAAAA|23332|6|28|52|AM|third|morning|breakfast| +23333|AAAAAAAAGCLFAAAA|23333|6|28|53|AM|third|morning|breakfast| +23334|AAAAAAAAHCLFAAAA|23334|6|28|54|AM|third|morning|breakfast| +23335|AAAAAAAAICLFAAAA|23335|6|28|55|AM|third|morning|breakfast| +23336|AAAAAAAAJCLFAAAA|23336|6|28|56|AM|third|morning|breakfast| +23337|AAAAAAAAKCLFAAAA|23337|6|28|57|AM|third|morning|breakfast| +23338|AAAAAAAALCLFAAAA|23338|6|28|58|AM|third|morning|breakfast| +23339|AAAAAAAAMCLFAAAA|23339|6|28|59|AM|third|morning|breakfast| +23340|AAAAAAAANCLFAAAA|23340|6|29|0|AM|third|morning|breakfast| +23341|AAAAAAAAOCLFAAAA|23341|6|29|1|AM|third|morning|breakfast| +23342|AAAAAAAAPCLFAAAA|23342|6|29|2|AM|third|morning|breakfast| +23343|AAAAAAAAADLFAAAA|23343|6|29|3|AM|third|morning|breakfast| +23344|AAAAAAAABDLFAAAA|23344|6|29|4|AM|third|morning|breakfast| +23345|AAAAAAAACDLFAAAA|23345|6|29|5|AM|third|morning|breakfast| +23346|AAAAAAAADDLFAAAA|23346|6|29|6|AM|third|morning|breakfast| +23347|AAAAAAAAEDLFAAAA|23347|6|29|7|AM|third|morning|breakfast| +23348|AAAAAAAAFDLFAAAA|23348|6|29|8|AM|third|morning|breakfast| +23349|AAAAAAAAGDLFAAAA|23349|6|29|9|AM|third|morning|breakfast| +23350|AAAAAAAAHDLFAAAA|23350|6|29|10|AM|third|morning|breakfast| +23351|AAAAAAAAIDLFAAAA|23351|6|29|11|AM|third|morning|breakfast| +23352|AAAAAAAAJDLFAAAA|23352|6|29|12|AM|third|morning|breakfast| +23353|AAAAAAAAKDLFAAAA|23353|6|29|13|AM|third|morning|breakfast| +23354|AAAAAAAALDLFAAAA|23354|6|29|14|AM|third|morning|breakfast| +23355|AAAAAAAAMDLFAAAA|23355|6|29|15|AM|third|morning|breakfast| +23356|AAAAAAAANDLFAAAA|23356|6|29|16|AM|third|morning|breakfast| +23357|AAAAAAAAODLFAAAA|23357|6|29|17|AM|third|morning|breakfast| +23358|AAAAAAAAPDLFAAAA|23358|6|29|18|AM|third|morning|breakfast| +23359|AAAAAAAAAELFAAAA|23359|6|29|19|AM|third|morning|breakfast| +23360|AAAAAAAABELFAAAA|23360|6|29|20|AM|third|morning|breakfast| +23361|AAAAAAAACELFAAAA|23361|6|29|21|AM|third|morning|breakfast| +23362|AAAAAAAADELFAAAA|23362|6|29|22|AM|third|morning|breakfast| +23363|AAAAAAAAEELFAAAA|23363|6|29|23|AM|third|morning|breakfast| +23364|AAAAAAAAFELFAAAA|23364|6|29|24|AM|third|morning|breakfast| +23365|AAAAAAAAGELFAAAA|23365|6|29|25|AM|third|morning|breakfast| +23366|AAAAAAAAHELFAAAA|23366|6|29|26|AM|third|morning|breakfast| +23367|AAAAAAAAIELFAAAA|23367|6|29|27|AM|third|morning|breakfast| +23368|AAAAAAAAJELFAAAA|23368|6|29|28|AM|third|morning|breakfast| +23369|AAAAAAAAKELFAAAA|23369|6|29|29|AM|third|morning|breakfast| +23370|AAAAAAAALELFAAAA|23370|6|29|30|AM|third|morning|breakfast| +23371|AAAAAAAAMELFAAAA|23371|6|29|31|AM|third|morning|breakfast| +23372|AAAAAAAANELFAAAA|23372|6|29|32|AM|third|morning|breakfast| +23373|AAAAAAAAOELFAAAA|23373|6|29|33|AM|third|morning|breakfast| +23374|AAAAAAAAPELFAAAA|23374|6|29|34|AM|third|morning|breakfast| +23375|AAAAAAAAAFLFAAAA|23375|6|29|35|AM|third|morning|breakfast| +23376|AAAAAAAABFLFAAAA|23376|6|29|36|AM|third|morning|breakfast| +23377|AAAAAAAACFLFAAAA|23377|6|29|37|AM|third|morning|breakfast| +23378|AAAAAAAADFLFAAAA|23378|6|29|38|AM|third|morning|breakfast| +23379|AAAAAAAAEFLFAAAA|23379|6|29|39|AM|third|morning|breakfast| +23380|AAAAAAAAFFLFAAAA|23380|6|29|40|AM|third|morning|breakfast| +23381|AAAAAAAAGFLFAAAA|23381|6|29|41|AM|third|morning|breakfast| +23382|AAAAAAAAHFLFAAAA|23382|6|29|42|AM|third|morning|breakfast| +23383|AAAAAAAAIFLFAAAA|23383|6|29|43|AM|third|morning|breakfast| +23384|AAAAAAAAJFLFAAAA|23384|6|29|44|AM|third|morning|breakfast| +23385|AAAAAAAAKFLFAAAA|23385|6|29|45|AM|third|morning|breakfast| +23386|AAAAAAAALFLFAAAA|23386|6|29|46|AM|third|morning|breakfast| +23387|AAAAAAAAMFLFAAAA|23387|6|29|47|AM|third|morning|breakfast| +23388|AAAAAAAANFLFAAAA|23388|6|29|48|AM|third|morning|breakfast| +23389|AAAAAAAAOFLFAAAA|23389|6|29|49|AM|third|morning|breakfast| +23390|AAAAAAAAPFLFAAAA|23390|6|29|50|AM|third|morning|breakfast| +23391|AAAAAAAAAGLFAAAA|23391|6|29|51|AM|third|morning|breakfast| +23392|AAAAAAAABGLFAAAA|23392|6|29|52|AM|third|morning|breakfast| +23393|AAAAAAAACGLFAAAA|23393|6|29|53|AM|third|morning|breakfast| +23394|AAAAAAAADGLFAAAA|23394|6|29|54|AM|third|morning|breakfast| +23395|AAAAAAAAEGLFAAAA|23395|6|29|55|AM|third|morning|breakfast| +23396|AAAAAAAAFGLFAAAA|23396|6|29|56|AM|third|morning|breakfast| +23397|AAAAAAAAGGLFAAAA|23397|6|29|57|AM|third|morning|breakfast| +23398|AAAAAAAAHGLFAAAA|23398|6|29|58|AM|third|morning|breakfast| +23399|AAAAAAAAIGLFAAAA|23399|6|29|59|AM|third|morning|breakfast| +23400|AAAAAAAAJGLFAAAA|23400|6|30|0|AM|third|morning|breakfast| +23401|AAAAAAAAKGLFAAAA|23401|6|30|1|AM|third|morning|breakfast| +23402|AAAAAAAALGLFAAAA|23402|6|30|2|AM|third|morning|breakfast| +23403|AAAAAAAAMGLFAAAA|23403|6|30|3|AM|third|morning|breakfast| +23404|AAAAAAAANGLFAAAA|23404|6|30|4|AM|third|morning|breakfast| +23405|AAAAAAAAOGLFAAAA|23405|6|30|5|AM|third|morning|breakfast| +23406|AAAAAAAAPGLFAAAA|23406|6|30|6|AM|third|morning|breakfast| +23407|AAAAAAAAAHLFAAAA|23407|6|30|7|AM|third|morning|breakfast| +23408|AAAAAAAABHLFAAAA|23408|6|30|8|AM|third|morning|breakfast| +23409|AAAAAAAACHLFAAAA|23409|6|30|9|AM|third|morning|breakfast| +23410|AAAAAAAADHLFAAAA|23410|6|30|10|AM|third|morning|breakfast| +23411|AAAAAAAAEHLFAAAA|23411|6|30|11|AM|third|morning|breakfast| +23412|AAAAAAAAFHLFAAAA|23412|6|30|12|AM|third|morning|breakfast| +23413|AAAAAAAAGHLFAAAA|23413|6|30|13|AM|third|morning|breakfast| +23414|AAAAAAAAHHLFAAAA|23414|6|30|14|AM|third|morning|breakfast| +23415|AAAAAAAAIHLFAAAA|23415|6|30|15|AM|third|morning|breakfast| +23416|AAAAAAAAJHLFAAAA|23416|6|30|16|AM|third|morning|breakfast| +23417|AAAAAAAAKHLFAAAA|23417|6|30|17|AM|third|morning|breakfast| +23418|AAAAAAAALHLFAAAA|23418|6|30|18|AM|third|morning|breakfast| +23419|AAAAAAAAMHLFAAAA|23419|6|30|19|AM|third|morning|breakfast| +23420|AAAAAAAANHLFAAAA|23420|6|30|20|AM|third|morning|breakfast| +23421|AAAAAAAAOHLFAAAA|23421|6|30|21|AM|third|morning|breakfast| +23422|AAAAAAAAPHLFAAAA|23422|6|30|22|AM|third|morning|breakfast| +23423|AAAAAAAAAILFAAAA|23423|6|30|23|AM|third|morning|breakfast| +23424|AAAAAAAABILFAAAA|23424|6|30|24|AM|third|morning|breakfast| +23425|AAAAAAAACILFAAAA|23425|6|30|25|AM|third|morning|breakfast| +23426|AAAAAAAADILFAAAA|23426|6|30|26|AM|third|morning|breakfast| +23427|AAAAAAAAEILFAAAA|23427|6|30|27|AM|third|morning|breakfast| +23428|AAAAAAAAFILFAAAA|23428|6|30|28|AM|third|morning|breakfast| +23429|AAAAAAAAGILFAAAA|23429|6|30|29|AM|third|morning|breakfast| +23430|AAAAAAAAHILFAAAA|23430|6|30|30|AM|third|morning|breakfast| +23431|AAAAAAAAIILFAAAA|23431|6|30|31|AM|third|morning|breakfast| +23432|AAAAAAAAJILFAAAA|23432|6|30|32|AM|third|morning|breakfast| +23433|AAAAAAAAKILFAAAA|23433|6|30|33|AM|third|morning|breakfast| +23434|AAAAAAAALILFAAAA|23434|6|30|34|AM|third|morning|breakfast| +23435|AAAAAAAAMILFAAAA|23435|6|30|35|AM|third|morning|breakfast| +23436|AAAAAAAANILFAAAA|23436|6|30|36|AM|third|morning|breakfast| +23437|AAAAAAAAOILFAAAA|23437|6|30|37|AM|third|morning|breakfast| +23438|AAAAAAAAPILFAAAA|23438|6|30|38|AM|third|morning|breakfast| +23439|AAAAAAAAAJLFAAAA|23439|6|30|39|AM|third|morning|breakfast| +23440|AAAAAAAABJLFAAAA|23440|6|30|40|AM|third|morning|breakfast| +23441|AAAAAAAACJLFAAAA|23441|6|30|41|AM|third|morning|breakfast| +23442|AAAAAAAADJLFAAAA|23442|6|30|42|AM|third|morning|breakfast| +23443|AAAAAAAAEJLFAAAA|23443|6|30|43|AM|third|morning|breakfast| +23444|AAAAAAAAFJLFAAAA|23444|6|30|44|AM|third|morning|breakfast| +23445|AAAAAAAAGJLFAAAA|23445|6|30|45|AM|third|morning|breakfast| +23446|AAAAAAAAHJLFAAAA|23446|6|30|46|AM|third|morning|breakfast| +23447|AAAAAAAAIJLFAAAA|23447|6|30|47|AM|third|morning|breakfast| +23448|AAAAAAAAJJLFAAAA|23448|6|30|48|AM|third|morning|breakfast| +23449|AAAAAAAAKJLFAAAA|23449|6|30|49|AM|third|morning|breakfast| +23450|AAAAAAAALJLFAAAA|23450|6|30|50|AM|third|morning|breakfast| +23451|AAAAAAAAMJLFAAAA|23451|6|30|51|AM|third|morning|breakfast| +23452|AAAAAAAANJLFAAAA|23452|6|30|52|AM|third|morning|breakfast| +23453|AAAAAAAAOJLFAAAA|23453|6|30|53|AM|third|morning|breakfast| +23454|AAAAAAAAPJLFAAAA|23454|6|30|54|AM|third|morning|breakfast| +23455|AAAAAAAAAKLFAAAA|23455|6|30|55|AM|third|morning|breakfast| +23456|AAAAAAAABKLFAAAA|23456|6|30|56|AM|third|morning|breakfast| +23457|AAAAAAAACKLFAAAA|23457|6|30|57|AM|third|morning|breakfast| +23458|AAAAAAAADKLFAAAA|23458|6|30|58|AM|third|morning|breakfast| +23459|AAAAAAAAEKLFAAAA|23459|6|30|59|AM|third|morning|breakfast| +23460|AAAAAAAAFKLFAAAA|23460|6|31|0|AM|third|morning|breakfast| +23461|AAAAAAAAGKLFAAAA|23461|6|31|1|AM|third|morning|breakfast| +23462|AAAAAAAAHKLFAAAA|23462|6|31|2|AM|third|morning|breakfast| +23463|AAAAAAAAIKLFAAAA|23463|6|31|3|AM|third|morning|breakfast| +23464|AAAAAAAAJKLFAAAA|23464|6|31|4|AM|third|morning|breakfast| +23465|AAAAAAAAKKLFAAAA|23465|6|31|5|AM|third|morning|breakfast| +23466|AAAAAAAALKLFAAAA|23466|6|31|6|AM|third|morning|breakfast| +23467|AAAAAAAAMKLFAAAA|23467|6|31|7|AM|third|morning|breakfast| +23468|AAAAAAAANKLFAAAA|23468|6|31|8|AM|third|morning|breakfast| +23469|AAAAAAAAOKLFAAAA|23469|6|31|9|AM|third|morning|breakfast| +23470|AAAAAAAAPKLFAAAA|23470|6|31|10|AM|third|morning|breakfast| +23471|AAAAAAAAALLFAAAA|23471|6|31|11|AM|third|morning|breakfast| +23472|AAAAAAAABLLFAAAA|23472|6|31|12|AM|third|morning|breakfast| +23473|AAAAAAAACLLFAAAA|23473|6|31|13|AM|third|morning|breakfast| +23474|AAAAAAAADLLFAAAA|23474|6|31|14|AM|third|morning|breakfast| +23475|AAAAAAAAELLFAAAA|23475|6|31|15|AM|third|morning|breakfast| +23476|AAAAAAAAFLLFAAAA|23476|6|31|16|AM|third|morning|breakfast| +23477|AAAAAAAAGLLFAAAA|23477|6|31|17|AM|third|morning|breakfast| +23478|AAAAAAAAHLLFAAAA|23478|6|31|18|AM|third|morning|breakfast| +23479|AAAAAAAAILLFAAAA|23479|6|31|19|AM|third|morning|breakfast| +23480|AAAAAAAAJLLFAAAA|23480|6|31|20|AM|third|morning|breakfast| +23481|AAAAAAAAKLLFAAAA|23481|6|31|21|AM|third|morning|breakfast| +23482|AAAAAAAALLLFAAAA|23482|6|31|22|AM|third|morning|breakfast| +23483|AAAAAAAAMLLFAAAA|23483|6|31|23|AM|third|morning|breakfast| +23484|AAAAAAAANLLFAAAA|23484|6|31|24|AM|third|morning|breakfast| +23485|AAAAAAAAOLLFAAAA|23485|6|31|25|AM|third|morning|breakfast| +23486|AAAAAAAAPLLFAAAA|23486|6|31|26|AM|third|morning|breakfast| +23487|AAAAAAAAAMLFAAAA|23487|6|31|27|AM|third|morning|breakfast| +23488|AAAAAAAABMLFAAAA|23488|6|31|28|AM|third|morning|breakfast| +23489|AAAAAAAACMLFAAAA|23489|6|31|29|AM|third|morning|breakfast| +23490|AAAAAAAADMLFAAAA|23490|6|31|30|AM|third|morning|breakfast| +23491|AAAAAAAAEMLFAAAA|23491|6|31|31|AM|third|morning|breakfast| +23492|AAAAAAAAFMLFAAAA|23492|6|31|32|AM|third|morning|breakfast| +23493|AAAAAAAAGMLFAAAA|23493|6|31|33|AM|third|morning|breakfast| +23494|AAAAAAAAHMLFAAAA|23494|6|31|34|AM|third|morning|breakfast| +23495|AAAAAAAAIMLFAAAA|23495|6|31|35|AM|third|morning|breakfast| +23496|AAAAAAAAJMLFAAAA|23496|6|31|36|AM|third|morning|breakfast| +23497|AAAAAAAAKMLFAAAA|23497|6|31|37|AM|third|morning|breakfast| +23498|AAAAAAAALMLFAAAA|23498|6|31|38|AM|third|morning|breakfast| +23499|AAAAAAAAMMLFAAAA|23499|6|31|39|AM|third|morning|breakfast| +23500|AAAAAAAANMLFAAAA|23500|6|31|40|AM|third|morning|breakfast| +23501|AAAAAAAAOMLFAAAA|23501|6|31|41|AM|third|morning|breakfast| +23502|AAAAAAAAPMLFAAAA|23502|6|31|42|AM|third|morning|breakfast| +23503|AAAAAAAAANLFAAAA|23503|6|31|43|AM|third|morning|breakfast| +23504|AAAAAAAABNLFAAAA|23504|6|31|44|AM|third|morning|breakfast| +23505|AAAAAAAACNLFAAAA|23505|6|31|45|AM|third|morning|breakfast| +23506|AAAAAAAADNLFAAAA|23506|6|31|46|AM|third|morning|breakfast| +23507|AAAAAAAAENLFAAAA|23507|6|31|47|AM|third|morning|breakfast| +23508|AAAAAAAAFNLFAAAA|23508|6|31|48|AM|third|morning|breakfast| +23509|AAAAAAAAGNLFAAAA|23509|6|31|49|AM|third|morning|breakfast| +23510|AAAAAAAAHNLFAAAA|23510|6|31|50|AM|third|morning|breakfast| +23511|AAAAAAAAINLFAAAA|23511|6|31|51|AM|third|morning|breakfast| +23512|AAAAAAAAJNLFAAAA|23512|6|31|52|AM|third|morning|breakfast| +23513|AAAAAAAAKNLFAAAA|23513|6|31|53|AM|third|morning|breakfast| +23514|AAAAAAAALNLFAAAA|23514|6|31|54|AM|third|morning|breakfast| +23515|AAAAAAAAMNLFAAAA|23515|6|31|55|AM|third|morning|breakfast| +23516|AAAAAAAANNLFAAAA|23516|6|31|56|AM|third|morning|breakfast| +23517|AAAAAAAAONLFAAAA|23517|6|31|57|AM|third|morning|breakfast| +23518|AAAAAAAAPNLFAAAA|23518|6|31|58|AM|third|morning|breakfast| +23519|AAAAAAAAAOLFAAAA|23519|6|31|59|AM|third|morning|breakfast| +23520|AAAAAAAABOLFAAAA|23520|6|32|0|AM|third|morning|breakfast| +23521|AAAAAAAACOLFAAAA|23521|6|32|1|AM|third|morning|breakfast| +23522|AAAAAAAADOLFAAAA|23522|6|32|2|AM|third|morning|breakfast| +23523|AAAAAAAAEOLFAAAA|23523|6|32|3|AM|third|morning|breakfast| +23524|AAAAAAAAFOLFAAAA|23524|6|32|4|AM|third|morning|breakfast| +23525|AAAAAAAAGOLFAAAA|23525|6|32|5|AM|third|morning|breakfast| +23526|AAAAAAAAHOLFAAAA|23526|6|32|6|AM|third|morning|breakfast| +23527|AAAAAAAAIOLFAAAA|23527|6|32|7|AM|third|morning|breakfast| +23528|AAAAAAAAJOLFAAAA|23528|6|32|8|AM|third|morning|breakfast| +23529|AAAAAAAAKOLFAAAA|23529|6|32|9|AM|third|morning|breakfast| +23530|AAAAAAAALOLFAAAA|23530|6|32|10|AM|third|morning|breakfast| +23531|AAAAAAAAMOLFAAAA|23531|6|32|11|AM|third|morning|breakfast| +23532|AAAAAAAANOLFAAAA|23532|6|32|12|AM|third|morning|breakfast| +23533|AAAAAAAAOOLFAAAA|23533|6|32|13|AM|third|morning|breakfast| +23534|AAAAAAAAPOLFAAAA|23534|6|32|14|AM|third|morning|breakfast| +23535|AAAAAAAAAPLFAAAA|23535|6|32|15|AM|third|morning|breakfast| +23536|AAAAAAAABPLFAAAA|23536|6|32|16|AM|third|morning|breakfast| +23537|AAAAAAAACPLFAAAA|23537|6|32|17|AM|third|morning|breakfast| +23538|AAAAAAAADPLFAAAA|23538|6|32|18|AM|third|morning|breakfast| +23539|AAAAAAAAEPLFAAAA|23539|6|32|19|AM|third|morning|breakfast| +23540|AAAAAAAAFPLFAAAA|23540|6|32|20|AM|third|morning|breakfast| +23541|AAAAAAAAGPLFAAAA|23541|6|32|21|AM|third|morning|breakfast| +23542|AAAAAAAAHPLFAAAA|23542|6|32|22|AM|third|morning|breakfast| +23543|AAAAAAAAIPLFAAAA|23543|6|32|23|AM|third|morning|breakfast| +23544|AAAAAAAAJPLFAAAA|23544|6|32|24|AM|third|morning|breakfast| +23545|AAAAAAAAKPLFAAAA|23545|6|32|25|AM|third|morning|breakfast| +23546|AAAAAAAALPLFAAAA|23546|6|32|26|AM|third|morning|breakfast| +23547|AAAAAAAAMPLFAAAA|23547|6|32|27|AM|third|morning|breakfast| +23548|AAAAAAAANPLFAAAA|23548|6|32|28|AM|third|morning|breakfast| +23549|AAAAAAAAOPLFAAAA|23549|6|32|29|AM|third|morning|breakfast| +23550|AAAAAAAAPPLFAAAA|23550|6|32|30|AM|third|morning|breakfast| +23551|AAAAAAAAAAMFAAAA|23551|6|32|31|AM|third|morning|breakfast| +23552|AAAAAAAABAMFAAAA|23552|6|32|32|AM|third|morning|breakfast| +23553|AAAAAAAACAMFAAAA|23553|6|32|33|AM|third|morning|breakfast| +23554|AAAAAAAADAMFAAAA|23554|6|32|34|AM|third|morning|breakfast| +23555|AAAAAAAAEAMFAAAA|23555|6|32|35|AM|third|morning|breakfast| +23556|AAAAAAAAFAMFAAAA|23556|6|32|36|AM|third|morning|breakfast| +23557|AAAAAAAAGAMFAAAA|23557|6|32|37|AM|third|morning|breakfast| +23558|AAAAAAAAHAMFAAAA|23558|6|32|38|AM|third|morning|breakfast| +23559|AAAAAAAAIAMFAAAA|23559|6|32|39|AM|third|morning|breakfast| +23560|AAAAAAAAJAMFAAAA|23560|6|32|40|AM|third|morning|breakfast| +23561|AAAAAAAAKAMFAAAA|23561|6|32|41|AM|third|morning|breakfast| +23562|AAAAAAAALAMFAAAA|23562|6|32|42|AM|third|morning|breakfast| +23563|AAAAAAAAMAMFAAAA|23563|6|32|43|AM|third|morning|breakfast| +23564|AAAAAAAANAMFAAAA|23564|6|32|44|AM|third|morning|breakfast| +23565|AAAAAAAAOAMFAAAA|23565|6|32|45|AM|third|morning|breakfast| +23566|AAAAAAAAPAMFAAAA|23566|6|32|46|AM|third|morning|breakfast| +23567|AAAAAAAAABMFAAAA|23567|6|32|47|AM|third|morning|breakfast| +23568|AAAAAAAABBMFAAAA|23568|6|32|48|AM|third|morning|breakfast| +23569|AAAAAAAACBMFAAAA|23569|6|32|49|AM|third|morning|breakfast| +23570|AAAAAAAADBMFAAAA|23570|6|32|50|AM|third|morning|breakfast| +23571|AAAAAAAAEBMFAAAA|23571|6|32|51|AM|third|morning|breakfast| +23572|AAAAAAAAFBMFAAAA|23572|6|32|52|AM|third|morning|breakfast| +23573|AAAAAAAAGBMFAAAA|23573|6|32|53|AM|third|morning|breakfast| +23574|AAAAAAAAHBMFAAAA|23574|6|32|54|AM|third|morning|breakfast| +23575|AAAAAAAAIBMFAAAA|23575|6|32|55|AM|third|morning|breakfast| +23576|AAAAAAAAJBMFAAAA|23576|6|32|56|AM|third|morning|breakfast| +23577|AAAAAAAAKBMFAAAA|23577|6|32|57|AM|third|morning|breakfast| +23578|AAAAAAAALBMFAAAA|23578|6|32|58|AM|third|morning|breakfast| +23579|AAAAAAAAMBMFAAAA|23579|6|32|59|AM|third|morning|breakfast| +23580|AAAAAAAANBMFAAAA|23580|6|33|0|AM|third|morning|breakfast| +23581|AAAAAAAAOBMFAAAA|23581|6|33|1|AM|third|morning|breakfast| +23582|AAAAAAAAPBMFAAAA|23582|6|33|2|AM|third|morning|breakfast| +23583|AAAAAAAAACMFAAAA|23583|6|33|3|AM|third|morning|breakfast| +23584|AAAAAAAABCMFAAAA|23584|6|33|4|AM|third|morning|breakfast| +23585|AAAAAAAACCMFAAAA|23585|6|33|5|AM|third|morning|breakfast| +23586|AAAAAAAADCMFAAAA|23586|6|33|6|AM|third|morning|breakfast| +23587|AAAAAAAAECMFAAAA|23587|6|33|7|AM|third|morning|breakfast| +23588|AAAAAAAAFCMFAAAA|23588|6|33|8|AM|third|morning|breakfast| +23589|AAAAAAAAGCMFAAAA|23589|6|33|9|AM|third|morning|breakfast| +23590|AAAAAAAAHCMFAAAA|23590|6|33|10|AM|third|morning|breakfast| +23591|AAAAAAAAICMFAAAA|23591|6|33|11|AM|third|morning|breakfast| +23592|AAAAAAAAJCMFAAAA|23592|6|33|12|AM|third|morning|breakfast| +23593|AAAAAAAAKCMFAAAA|23593|6|33|13|AM|third|morning|breakfast| +23594|AAAAAAAALCMFAAAA|23594|6|33|14|AM|third|morning|breakfast| +23595|AAAAAAAAMCMFAAAA|23595|6|33|15|AM|third|morning|breakfast| +23596|AAAAAAAANCMFAAAA|23596|6|33|16|AM|third|morning|breakfast| +23597|AAAAAAAAOCMFAAAA|23597|6|33|17|AM|third|morning|breakfast| +23598|AAAAAAAAPCMFAAAA|23598|6|33|18|AM|third|morning|breakfast| +23599|AAAAAAAAADMFAAAA|23599|6|33|19|AM|third|morning|breakfast| +23600|AAAAAAAABDMFAAAA|23600|6|33|20|AM|third|morning|breakfast| +23601|AAAAAAAACDMFAAAA|23601|6|33|21|AM|third|morning|breakfast| +23602|AAAAAAAADDMFAAAA|23602|6|33|22|AM|third|morning|breakfast| +23603|AAAAAAAAEDMFAAAA|23603|6|33|23|AM|third|morning|breakfast| +23604|AAAAAAAAFDMFAAAA|23604|6|33|24|AM|third|morning|breakfast| +23605|AAAAAAAAGDMFAAAA|23605|6|33|25|AM|third|morning|breakfast| +23606|AAAAAAAAHDMFAAAA|23606|6|33|26|AM|third|morning|breakfast| +23607|AAAAAAAAIDMFAAAA|23607|6|33|27|AM|third|morning|breakfast| +23608|AAAAAAAAJDMFAAAA|23608|6|33|28|AM|third|morning|breakfast| +23609|AAAAAAAAKDMFAAAA|23609|6|33|29|AM|third|morning|breakfast| +23610|AAAAAAAALDMFAAAA|23610|6|33|30|AM|third|morning|breakfast| +23611|AAAAAAAAMDMFAAAA|23611|6|33|31|AM|third|morning|breakfast| +23612|AAAAAAAANDMFAAAA|23612|6|33|32|AM|third|morning|breakfast| +23613|AAAAAAAAODMFAAAA|23613|6|33|33|AM|third|morning|breakfast| +23614|AAAAAAAAPDMFAAAA|23614|6|33|34|AM|third|morning|breakfast| +23615|AAAAAAAAAEMFAAAA|23615|6|33|35|AM|third|morning|breakfast| +23616|AAAAAAAABEMFAAAA|23616|6|33|36|AM|third|morning|breakfast| +23617|AAAAAAAACEMFAAAA|23617|6|33|37|AM|third|morning|breakfast| +23618|AAAAAAAADEMFAAAA|23618|6|33|38|AM|third|morning|breakfast| +23619|AAAAAAAAEEMFAAAA|23619|6|33|39|AM|third|morning|breakfast| +23620|AAAAAAAAFEMFAAAA|23620|6|33|40|AM|third|morning|breakfast| +23621|AAAAAAAAGEMFAAAA|23621|6|33|41|AM|third|morning|breakfast| +23622|AAAAAAAAHEMFAAAA|23622|6|33|42|AM|third|morning|breakfast| +23623|AAAAAAAAIEMFAAAA|23623|6|33|43|AM|third|morning|breakfast| +23624|AAAAAAAAJEMFAAAA|23624|6|33|44|AM|third|morning|breakfast| +23625|AAAAAAAAKEMFAAAA|23625|6|33|45|AM|third|morning|breakfast| +23626|AAAAAAAALEMFAAAA|23626|6|33|46|AM|third|morning|breakfast| +23627|AAAAAAAAMEMFAAAA|23627|6|33|47|AM|third|morning|breakfast| +23628|AAAAAAAANEMFAAAA|23628|6|33|48|AM|third|morning|breakfast| +23629|AAAAAAAAOEMFAAAA|23629|6|33|49|AM|third|morning|breakfast| +23630|AAAAAAAAPEMFAAAA|23630|6|33|50|AM|third|morning|breakfast| +23631|AAAAAAAAAFMFAAAA|23631|6|33|51|AM|third|morning|breakfast| +23632|AAAAAAAABFMFAAAA|23632|6|33|52|AM|third|morning|breakfast| +23633|AAAAAAAACFMFAAAA|23633|6|33|53|AM|third|morning|breakfast| +23634|AAAAAAAADFMFAAAA|23634|6|33|54|AM|third|morning|breakfast| +23635|AAAAAAAAEFMFAAAA|23635|6|33|55|AM|third|morning|breakfast| +23636|AAAAAAAAFFMFAAAA|23636|6|33|56|AM|third|morning|breakfast| +23637|AAAAAAAAGFMFAAAA|23637|6|33|57|AM|third|morning|breakfast| +23638|AAAAAAAAHFMFAAAA|23638|6|33|58|AM|third|morning|breakfast| +23639|AAAAAAAAIFMFAAAA|23639|6|33|59|AM|third|morning|breakfast| +23640|AAAAAAAAJFMFAAAA|23640|6|34|0|AM|third|morning|breakfast| +23641|AAAAAAAAKFMFAAAA|23641|6|34|1|AM|third|morning|breakfast| +23642|AAAAAAAALFMFAAAA|23642|6|34|2|AM|third|morning|breakfast| +23643|AAAAAAAAMFMFAAAA|23643|6|34|3|AM|third|morning|breakfast| +23644|AAAAAAAANFMFAAAA|23644|6|34|4|AM|third|morning|breakfast| +23645|AAAAAAAAOFMFAAAA|23645|6|34|5|AM|third|morning|breakfast| +23646|AAAAAAAAPFMFAAAA|23646|6|34|6|AM|third|morning|breakfast| +23647|AAAAAAAAAGMFAAAA|23647|6|34|7|AM|third|morning|breakfast| +23648|AAAAAAAABGMFAAAA|23648|6|34|8|AM|third|morning|breakfast| +23649|AAAAAAAACGMFAAAA|23649|6|34|9|AM|third|morning|breakfast| +23650|AAAAAAAADGMFAAAA|23650|6|34|10|AM|third|morning|breakfast| +23651|AAAAAAAAEGMFAAAA|23651|6|34|11|AM|third|morning|breakfast| +23652|AAAAAAAAFGMFAAAA|23652|6|34|12|AM|third|morning|breakfast| +23653|AAAAAAAAGGMFAAAA|23653|6|34|13|AM|third|morning|breakfast| +23654|AAAAAAAAHGMFAAAA|23654|6|34|14|AM|third|morning|breakfast| +23655|AAAAAAAAIGMFAAAA|23655|6|34|15|AM|third|morning|breakfast| +23656|AAAAAAAAJGMFAAAA|23656|6|34|16|AM|third|morning|breakfast| +23657|AAAAAAAAKGMFAAAA|23657|6|34|17|AM|third|morning|breakfast| +23658|AAAAAAAALGMFAAAA|23658|6|34|18|AM|third|morning|breakfast| +23659|AAAAAAAAMGMFAAAA|23659|6|34|19|AM|third|morning|breakfast| +23660|AAAAAAAANGMFAAAA|23660|6|34|20|AM|third|morning|breakfast| +23661|AAAAAAAAOGMFAAAA|23661|6|34|21|AM|third|morning|breakfast| +23662|AAAAAAAAPGMFAAAA|23662|6|34|22|AM|third|morning|breakfast| +23663|AAAAAAAAAHMFAAAA|23663|6|34|23|AM|third|morning|breakfast| +23664|AAAAAAAABHMFAAAA|23664|6|34|24|AM|third|morning|breakfast| +23665|AAAAAAAACHMFAAAA|23665|6|34|25|AM|third|morning|breakfast| +23666|AAAAAAAADHMFAAAA|23666|6|34|26|AM|third|morning|breakfast| +23667|AAAAAAAAEHMFAAAA|23667|6|34|27|AM|third|morning|breakfast| +23668|AAAAAAAAFHMFAAAA|23668|6|34|28|AM|third|morning|breakfast| +23669|AAAAAAAAGHMFAAAA|23669|6|34|29|AM|third|morning|breakfast| +23670|AAAAAAAAHHMFAAAA|23670|6|34|30|AM|third|morning|breakfast| +23671|AAAAAAAAIHMFAAAA|23671|6|34|31|AM|third|morning|breakfast| +23672|AAAAAAAAJHMFAAAA|23672|6|34|32|AM|third|morning|breakfast| +23673|AAAAAAAAKHMFAAAA|23673|6|34|33|AM|third|morning|breakfast| +23674|AAAAAAAALHMFAAAA|23674|6|34|34|AM|third|morning|breakfast| +23675|AAAAAAAAMHMFAAAA|23675|6|34|35|AM|third|morning|breakfast| +23676|AAAAAAAANHMFAAAA|23676|6|34|36|AM|third|morning|breakfast| +23677|AAAAAAAAOHMFAAAA|23677|6|34|37|AM|third|morning|breakfast| +23678|AAAAAAAAPHMFAAAA|23678|6|34|38|AM|third|morning|breakfast| +23679|AAAAAAAAAIMFAAAA|23679|6|34|39|AM|third|morning|breakfast| +23680|AAAAAAAABIMFAAAA|23680|6|34|40|AM|third|morning|breakfast| +23681|AAAAAAAACIMFAAAA|23681|6|34|41|AM|third|morning|breakfast| +23682|AAAAAAAADIMFAAAA|23682|6|34|42|AM|third|morning|breakfast| +23683|AAAAAAAAEIMFAAAA|23683|6|34|43|AM|third|morning|breakfast| +23684|AAAAAAAAFIMFAAAA|23684|6|34|44|AM|third|morning|breakfast| +23685|AAAAAAAAGIMFAAAA|23685|6|34|45|AM|third|morning|breakfast| +23686|AAAAAAAAHIMFAAAA|23686|6|34|46|AM|third|morning|breakfast| +23687|AAAAAAAAIIMFAAAA|23687|6|34|47|AM|third|morning|breakfast| +23688|AAAAAAAAJIMFAAAA|23688|6|34|48|AM|third|morning|breakfast| +23689|AAAAAAAAKIMFAAAA|23689|6|34|49|AM|third|morning|breakfast| +23690|AAAAAAAALIMFAAAA|23690|6|34|50|AM|third|morning|breakfast| +23691|AAAAAAAAMIMFAAAA|23691|6|34|51|AM|third|morning|breakfast| +23692|AAAAAAAANIMFAAAA|23692|6|34|52|AM|third|morning|breakfast| +23693|AAAAAAAAOIMFAAAA|23693|6|34|53|AM|third|morning|breakfast| +23694|AAAAAAAAPIMFAAAA|23694|6|34|54|AM|third|morning|breakfast| +23695|AAAAAAAAAJMFAAAA|23695|6|34|55|AM|third|morning|breakfast| +23696|AAAAAAAABJMFAAAA|23696|6|34|56|AM|third|morning|breakfast| +23697|AAAAAAAACJMFAAAA|23697|6|34|57|AM|third|morning|breakfast| +23698|AAAAAAAADJMFAAAA|23698|6|34|58|AM|third|morning|breakfast| +23699|AAAAAAAAEJMFAAAA|23699|6|34|59|AM|third|morning|breakfast| +23700|AAAAAAAAFJMFAAAA|23700|6|35|0|AM|third|morning|breakfast| +23701|AAAAAAAAGJMFAAAA|23701|6|35|1|AM|third|morning|breakfast| +23702|AAAAAAAAHJMFAAAA|23702|6|35|2|AM|third|morning|breakfast| +23703|AAAAAAAAIJMFAAAA|23703|6|35|3|AM|third|morning|breakfast| +23704|AAAAAAAAJJMFAAAA|23704|6|35|4|AM|third|morning|breakfast| +23705|AAAAAAAAKJMFAAAA|23705|6|35|5|AM|third|morning|breakfast| +23706|AAAAAAAALJMFAAAA|23706|6|35|6|AM|third|morning|breakfast| +23707|AAAAAAAAMJMFAAAA|23707|6|35|7|AM|third|morning|breakfast| +23708|AAAAAAAANJMFAAAA|23708|6|35|8|AM|third|morning|breakfast| +23709|AAAAAAAAOJMFAAAA|23709|6|35|9|AM|third|morning|breakfast| +23710|AAAAAAAAPJMFAAAA|23710|6|35|10|AM|third|morning|breakfast| +23711|AAAAAAAAAKMFAAAA|23711|6|35|11|AM|third|morning|breakfast| +23712|AAAAAAAABKMFAAAA|23712|6|35|12|AM|third|morning|breakfast| +23713|AAAAAAAACKMFAAAA|23713|6|35|13|AM|third|morning|breakfast| +23714|AAAAAAAADKMFAAAA|23714|6|35|14|AM|third|morning|breakfast| +23715|AAAAAAAAEKMFAAAA|23715|6|35|15|AM|third|morning|breakfast| +23716|AAAAAAAAFKMFAAAA|23716|6|35|16|AM|third|morning|breakfast| +23717|AAAAAAAAGKMFAAAA|23717|6|35|17|AM|third|morning|breakfast| +23718|AAAAAAAAHKMFAAAA|23718|6|35|18|AM|third|morning|breakfast| +23719|AAAAAAAAIKMFAAAA|23719|6|35|19|AM|third|morning|breakfast| +23720|AAAAAAAAJKMFAAAA|23720|6|35|20|AM|third|morning|breakfast| +23721|AAAAAAAAKKMFAAAA|23721|6|35|21|AM|third|morning|breakfast| +23722|AAAAAAAALKMFAAAA|23722|6|35|22|AM|third|morning|breakfast| +23723|AAAAAAAAMKMFAAAA|23723|6|35|23|AM|third|morning|breakfast| +23724|AAAAAAAANKMFAAAA|23724|6|35|24|AM|third|morning|breakfast| +23725|AAAAAAAAOKMFAAAA|23725|6|35|25|AM|third|morning|breakfast| +23726|AAAAAAAAPKMFAAAA|23726|6|35|26|AM|third|morning|breakfast| +23727|AAAAAAAAALMFAAAA|23727|6|35|27|AM|third|morning|breakfast| +23728|AAAAAAAABLMFAAAA|23728|6|35|28|AM|third|morning|breakfast| +23729|AAAAAAAACLMFAAAA|23729|6|35|29|AM|third|morning|breakfast| +23730|AAAAAAAADLMFAAAA|23730|6|35|30|AM|third|morning|breakfast| +23731|AAAAAAAAELMFAAAA|23731|6|35|31|AM|third|morning|breakfast| +23732|AAAAAAAAFLMFAAAA|23732|6|35|32|AM|third|morning|breakfast| +23733|AAAAAAAAGLMFAAAA|23733|6|35|33|AM|third|morning|breakfast| +23734|AAAAAAAAHLMFAAAA|23734|6|35|34|AM|third|morning|breakfast| +23735|AAAAAAAAILMFAAAA|23735|6|35|35|AM|third|morning|breakfast| +23736|AAAAAAAAJLMFAAAA|23736|6|35|36|AM|third|morning|breakfast| +23737|AAAAAAAAKLMFAAAA|23737|6|35|37|AM|third|morning|breakfast| +23738|AAAAAAAALLMFAAAA|23738|6|35|38|AM|third|morning|breakfast| +23739|AAAAAAAAMLMFAAAA|23739|6|35|39|AM|third|morning|breakfast| +23740|AAAAAAAANLMFAAAA|23740|6|35|40|AM|third|morning|breakfast| +23741|AAAAAAAAOLMFAAAA|23741|6|35|41|AM|third|morning|breakfast| +23742|AAAAAAAAPLMFAAAA|23742|6|35|42|AM|third|morning|breakfast| +23743|AAAAAAAAAMMFAAAA|23743|6|35|43|AM|third|morning|breakfast| +23744|AAAAAAAABMMFAAAA|23744|6|35|44|AM|third|morning|breakfast| +23745|AAAAAAAACMMFAAAA|23745|6|35|45|AM|third|morning|breakfast| +23746|AAAAAAAADMMFAAAA|23746|6|35|46|AM|third|morning|breakfast| +23747|AAAAAAAAEMMFAAAA|23747|6|35|47|AM|third|morning|breakfast| +23748|AAAAAAAAFMMFAAAA|23748|6|35|48|AM|third|morning|breakfast| +23749|AAAAAAAAGMMFAAAA|23749|6|35|49|AM|third|morning|breakfast| +23750|AAAAAAAAHMMFAAAA|23750|6|35|50|AM|third|morning|breakfast| +23751|AAAAAAAAIMMFAAAA|23751|6|35|51|AM|third|morning|breakfast| +23752|AAAAAAAAJMMFAAAA|23752|6|35|52|AM|third|morning|breakfast| +23753|AAAAAAAAKMMFAAAA|23753|6|35|53|AM|third|morning|breakfast| +23754|AAAAAAAALMMFAAAA|23754|6|35|54|AM|third|morning|breakfast| +23755|AAAAAAAAMMMFAAAA|23755|6|35|55|AM|third|morning|breakfast| +23756|AAAAAAAANMMFAAAA|23756|6|35|56|AM|third|morning|breakfast| +23757|AAAAAAAAOMMFAAAA|23757|6|35|57|AM|third|morning|breakfast| +23758|AAAAAAAAPMMFAAAA|23758|6|35|58|AM|third|morning|breakfast| +23759|AAAAAAAAANMFAAAA|23759|6|35|59|AM|third|morning|breakfast| +23760|AAAAAAAABNMFAAAA|23760|6|36|0|AM|third|morning|breakfast| +23761|AAAAAAAACNMFAAAA|23761|6|36|1|AM|third|morning|breakfast| +23762|AAAAAAAADNMFAAAA|23762|6|36|2|AM|third|morning|breakfast| +23763|AAAAAAAAENMFAAAA|23763|6|36|3|AM|third|morning|breakfast| +23764|AAAAAAAAFNMFAAAA|23764|6|36|4|AM|third|morning|breakfast| +23765|AAAAAAAAGNMFAAAA|23765|6|36|5|AM|third|morning|breakfast| +23766|AAAAAAAAHNMFAAAA|23766|6|36|6|AM|third|morning|breakfast| +23767|AAAAAAAAINMFAAAA|23767|6|36|7|AM|third|morning|breakfast| +23768|AAAAAAAAJNMFAAAA|23768|6|36|8|AM|third|morning|breakfast| +23769|AAAAAAAAKNMFAAAA|23769|6|36|9|AM|third|morning|breakfast| +23770|AAAAAAAALNMFAAAA|23770|6|36|10|AM|third|morning|breakfast| +23771|AAAAAAAAMNMFAAAA|23771|6|36|11|AM|third|morning|breakfast| +23772|AAAAAAAANNMFAAAA|23772|6|36|12|AM|third|morning|breakfast| +23773|AAAAAAAAONMFAAAA|23773|6|36|13|AM|third|morning|breakfast| +23774|AAAAAAAAPNMFAAAA|23774|6|36|14|AM|third|morning|breakfast| +23775|AAAAAAAAAOMFAAAA|23775|6|36|15|AM|third|morning|breakfast| +23776|AAAAAAAABOMFAAAA|23776|6|36|16|AM|third|morning|breakfast| +23777|AAAAAAAACOMFAAAA|23777|6|36|17|AM|third|morning|breakfast| +23778|AAAAAAAADOMFAAAA|23778|6|36|18|AM|third|morning|breakfast| +23779|AAAAAAAAEOMFAAAA|23779|6|36|19|AM|third|morning|breakfast| +23780|AAAAAAAAFOMFAAAA|23780|6|36|20|AM|third|morning|breakfast| +23781|AAAAAAAAGOMFAAAA|23781|6|36|21|AM|third|morning|breakfast| +23782|AAAAAAAAHOMFAAAA|23782|6|36|22|AM|third|morning|breakfast| +23783|AAAAAAAAIOMFAAAA|23783|6|36|23|AM|third|morning|breakfast| +23784|AAAAAAAAJOMFAAAA|23784|6|36|24|AM|third|morning|breakfast| +23785|AAAAAAAAKOMFAAAA|23785|6|36|25|AM|third|morning|breakfast| +23786|AAAAAAAALOMFAAAA|23786|6|36|26|AM|third|morning|breakfast| +23787|AAAAAAAAMOMFAAAA|23787|6|36|27|AM|third|morning|breakfast| +23788|AAAAAAAANOMFAAAA|23788|6|36|28|AM|third|morning|breakfast| +23789|AAAAAAAAOOMFAAAA|23789|6|36|29|AM|third|morning|breakfast| +23790|AAAAAAAAPOMFAAAA|23790|6|36|30|AM|third|morning|breakfast| +23791|AAAAAAAAAPMFAAAA|23791|6|36|31|AM|third|morning|breakfast| +23792|AAAAAAAABPMFAAAA|23792|6|36|32|AM|third|morning|breakfast| +23793|AAAAAAAACPMFAAAA|23793|6|36|33|AM|third|morning|breakfast| +23794|AAAAAAAADPMFAAAA|23794|6|36|34|AM|third|morning|breakfast| +23795|AAAAAAAAEPMFAAAA|23795|6|36|35|AM|third|morning|breakfast| +23796|AAAAAAAAFPMFAAAA|23796|6|36|36|AM|third|morning|breakfast| +23797|AAAAAAAAGPMFAAAA|23797|6|36|37|AM|third|morning|breakfast| +23798|AAAAAAAAHPMFAAAA|23798|6|36|38|AM|third|morning|breakfast| +23799|AAAAAAAAIPMFAAAA|23799|6|36|39|AM|third|morning|breakfast| +23800|AAAAAAAAJPMFAAAA|23800|6|36|40|AM|third|morning|breakfast| +23801|AAAAAAAAKPMFAAAA|23801|6|36|41|AM|third|morning|breakfast| +23802|AAAAAAAALPMFAAAA|23802|6|36|42|AM|third|morning|breakfast| +23803|AAAAAAAAMPMFAAAA|23803|6|36|43|AM|third|morning|breakfast| +23804|AAAAAAAANPMFAAAA|23804|6|36|44|AM|third|morning|breakfast| +23805|AAAAAAAAOPMFAAAA|23805|6|36|45|AM|third|morning|breakfast| +23806|AAAAAAAAPPMFAAAA|23806|6|36|46|AM|third|morning|breakfast| +23807|AAAAAAAAAANFAAAA|23807|6|36|47|AM|third|morning|breakfast| +23808|AAAAAAAABANFAAAA|23808|6|36|48|AM|third|morning|breakfast| +23809|AAAAAAAACANFAAAA|23809|6|36|49|AM|third|morning|breakfast| +23810|AAAAAAAADANFAAAA|23810|6|36|50|AM|third|morning|breakfast| +23811|AAAAAAAAEANFAAAA|23811|6|36|51|AM|third|morning|breakfast| +23812|AAAAAAAAFANFAAAA|23812|6|36|52|AM|third|morning|breakfast| +23813|AAAAAAAAGANFAAAA|23813|6|36|53|AM|third|morning|breakfast| +23814|AAAAAAAAHANFAAAA|23814|6|36|54|AM|third|morning|breakfast| +23815|AAAAAAAAIANFAAAA|23815|6|36|55|AM|third|morning|breakfast| +23816|AAAAAAAAJANFAAAA|23816|6|36|56|AM|third|morning|breakfast| +23817|AAAAAAAAKANFAAAA|23817|6|36|57|AM|third|morning|breakfast| +23818|AAAAAAAALANFAAAA|23818|6|36|58|AM|third|morning|breakfast| +23819|AAAAAAAAMANFAAAA|23819|6|36|59|AM|third|morning|breakfast| +23820|AAAAAAAANANFAAAA|23820|6|37|0|AM|third|morning|breakfast| +23821|AAAAAAAAOANFAAAA|23821|6|37|1|AM|third|morning|breakfast| +23822|AAAAAAAAPANFAAAA|23822|6|37|2|AM|third|morning|breakfast| +23823|AAAAAAAAABNFAAAA|23823|6|37|3|AM|third|morning|breakfast| +23824|AAAAAAAABBNFAAAA|23824|6|37|4|AM|third|morning|breakfast| +23825|AAAAAAAACBNFAAAA|23825|6|37|5|AM|third|morning|breakfast| +23826|AAAAAAAADBNFAAAA|23826|6|37|6|AM|third|morning|breakfast| +23827|AAAAAAAAEBNFAAAA|23827|6|37|7|AM|third|morning|breakfast| +23828|AAAAAAAAFBNFAAAA|23828|6|37|8|AM|third|morning|breakfast| +23829|AAAAAAAAGBNFAAAA|23829|6|37|9|AM|third|morning|breakfast| +23830|AAAAAAAAHBNFAAAA|23830|6|37|10|AM|third|morning|breakfast| +23831|AAAAAAAAIBNFAAAA|23831|6|37|11|AM|third|morning|breakfast| +23832|AAAAAAAAJBNFAAAA|23832|6|37|12|AM|third|morning|breakfast| +23833|AAAAAAAAKBNFAAAA|23833|6|37|13|AM|third|morning|breakfast| +23834|AAAAAAAALBNFAAAA|23834|6|37|14|AM|third|morning|breakfast| +23835|AAAAAAAAMBNFAAAA|23835|6|37|15|AM|third|morning|breakfast| +23836|AAAAAAAANBNFAAAA|23836|6|37|16|AM|third|morning|breakfast| +23837|AAAAAAAAOBNFAAAA|23837|6|37|17|AM|third|morning|breakfast| +23838|AAAAAAAAPBNFAAAA|23838|6|37|18|AM|third|morning|breakfast| +23839|AAAAAAAAACNFAAAA|23839|6|37|19|AM|third|morning|breakfast| +23840|AAAAAAAABCNFAAAA|23840|6|37|20|AM|third|morning|breakfast| +23841|AAAAAAAACCNFAAAA|23841|6|37|21|AM|third|morning|breakfast| +23842|AAAAAAAADCNFAAAA|23842|6|37|22|AM|third|morning|breakfast| +23843|AAAAAAAAECNFAAAA|23843|6|37|23|AM|third|morning|breakfast| +23844|AAAAAAAAFCNFAAAA|23844|6|37|24|AM|third|morning|breakfast| +23845|AAAAAAAAGCNFAAAA|23845|6|37|25|AM|third|morning|breakfast| +23846|AAAAAAAAHCNFAAAA|23846|6|37|26|AM|third|morning|breakfast| +23847|AAAAAAAAICNFAAAA|23847|6|37|27|AM|third|morning|breakfast| +23848|AAAAAAAAJCNFAAAA|23848|6|37|28|AM|third|morning|breakfast| +23849|AAAAAAAAKCNFAAAA|23849|6|37|29|AM|third|morning|breakfast| +23850|AAAAAAAALCNFAAAA|23850|6|37|30|AM|third|morning|breakfast| +23851|AAAAAAAAMCNFAAAA|23851|6|37|31|AM|third|morning|breakfast| +23852|AAAAAAAANCNFAAAA|23852|6|37|32|AM|third|morning|breakfast| +23853|AAAAAAAAOCNFAAAA|23853|6|37|33|AM|third|morning|breakfast| +23854|AAAAAAAAPCNFAAAA|23854|6|37|34|AM|third|morning|breakfast| +23855|AAAAAAAAADNFAAAA|23855|6|37|35|AM|third|morning|breakfast| +23856|AAAAAAAABDNFAAAA|23856|6|37|36|AM|third|morning|breakfast| +23857|AAAAAAAACDNFAAAA|23857|6|37|37|AM|third|morning|breakfast| +23858|AAAAAAAADDNFAAAA|23858|6|37|38|AM|third|morning|breakfast| +23859|AAAAAAAAEDNFAAAA|23859|6|37|39|AM|third|morning|breakfast| +23860|AAAAAAAAFDNFAAAA|23860|6|37|40|AM|third|morning|breakfast| +23861|AAAAAAAAGDNFAAAA|23861|6|37|41|AM|third|morning|breakfast| +23862|AAAAAAAAHDNFAAAA|23862|6|37|42|AM|third|morning|breakfast| +23863|AAAAAAAAIDNFAAAA|23863|6|37|43|AM|third|morning|breakfast| +23864|AAAAAAAAJDNFAAAA|23864|6|37|44|AM|third|morning|breakfast| +23865|AAAAAAAAKDNFAAAA|23865|6|37|45|AM|third|morning|breakfast| +23866|AAAAAAAALDNFAAAA|23866|6|37|46|AM|third|morning|breakfast| +23867|AAAAAAAAMDNFAAAA|23867|6|37|47|AM|third|morning|breakfast| +23868|AAAAAAAANDNFAAAA|23868|6|37|48|AM|third|morning|breakfast| +23869|AAAAAAAAODNFAAAA|23869|6|37|49|AM|third|morning|breakfast| +23870|AAAAAAAAPDNFAAAA|23870|6|37|50|AM|third|morning|breakfast| +23871|AAAAAAAAAENFAAAA|23871|6|37|51|AM|third|morning|breakfast| +23872|AAAAAAAABENFAAAA|23872|6|37|52|AM|third|morning|breakfast| +23873|AAAAAAAACENFAAAA|23873|6|37|53|AM|third|morning|breakfast| +23874|AAAAAAAADENFAAAA|23874|6|37|54|AM|third|morning|breakfast| +23875|AAAAAAAAEENFAAAA|23875|6|37|55|AM|third|morning|breakfast| +23876|AAAAAAAAFENFAAAA|23876|6|37|56|AM|third|morning|breakfast| +23877|AAAAAAAAGENFAAAA|23877|6|37|57|AM|third|morning|breakfast| +23878|AAAAAAAAHENFAAAA|23878|6|37|58|AM|third|morning|breakfast| +23879|AAAAAAAAIENFAAAA|23879|6|37|59|AM|third|morning|breakfast| +23880|AAAAAAAAJENFAAAA|23880|6|38|0|AM|third|morning|breakfast| +23881|AAAAAAAAKENFAAAA|23881|6|38|1|AM|third|morning|breakfast| +23882|AAAAAAAALENFAAAA|23882|6|38|2|AM|third|morning|breakfast| +23883|AAAAAAAAMENFAAAA|23883|6|38|3|AM|third|morning|breakfast| +23884|AAAAAAAANENFAAAA|23884|6|38|4|AM|third|morning|breakfast| +23885|AAAAAAAAOENFAAAA|23885|6|38|5|AM|third|morning|breakfast| +23886|AAAAAAAAPENFAAAA|23886|6|38|6|AM|third|morning|breakfast| +23887|AAAAAAAAAFNFAAAA|23887|6|38|7|AM|third|morning|breakfast| +23888|AAAAAAAABFNFAAAA|23888|6|38|8|AM|third|morning|breakfast| +23889|AAAAAAAACFNFAAAA|23889|6|38|9|AM|third|morning|breakfast| +23890|AAAAAAAADFNFAAAA|23890|6|38|10|AM|third|morning|breakfast| +23891|AAAAAAAAEFNFAAAA|23891|6|38|11|AM|third|morning|breakfast| +23892|AAAAAAAAFFNFAAAA|23892|6|38|12|AM|third|morning|breakfast| +23893|AAAAAAAAGFNFAAAA|23893|6|38|13|AM|third|morning|breakfast| +23894|AAAAAAAAHFNFAAAA|23894|6|38|14|AM|third|morning|breakfast| +23895|AAAAAAAAIFNFAAAA|23895|6|38|15|AM|third|morning|breakfast| +23896|AAAAAAAAJFNFAAAA|23896|6|38|16|AM|third|morning|breakfast| +23897|AAAAAAAAKFNFAAAA|23897|6|38|17|AM|third|morning|breakfast| +23898|AAAAAAAALFNFAAAA|23898|6|38|18|AM|third|morning|breakfast| +23899|AAAAAAAAMFNFAAAA|23899|6|38|19|AM|third|morning|breakfast| +23900|AAAAAAAANFNFAAAA|23900|6|38|20|AM|third|morning|breakfast| +23901|AAAAAAAAOFNFAAAA|23901|6|38|21|AM|third|morning|breakfast| +23902|AAAAAAAAPFNFAAAA|23902|6|38|22|AM|third|morning|breakfast| +23903|AAAAAAAAAGNFAAAA|23903|6|38|23|AM|third|morning|breakfast| +23904|AAAAAAAABGNFAAAA|23904|6|38|24|AM|third|morning|breakfast| +23905|AAAAAAAACGNFAAAA|23905|6|38|25|AM|third|morning|breakfast| +23906|AAAAAAAADGNFAAAA|23906|6|38|26|AM|third|morning|breakfast| +23907|AAAAAAAAEGNFAAAA|23907|6|38|27|AM|third|morning|breakfast| +23908|AAAAAAAAFGNFAAAA|23908|6|38|28|AM|third|morning|breakfast| +23909|AAAAAAAAGGNFAAAA|23909|6|38|29|AM|third|morning|breakfast| +23910|AAAAAAAAHGNFAAAA|23910|6|38|30|AM|third|morning|breakfast| +23911|AAAAAAAAIGNFAAAA|23911|6|38|31|AM|third|morning|breakfast| +23912|AAAAAAAAJGNFAAAA|23912|6|38|32|AM|third|morning|breakfast| +23913|AAAAAAAAKGNFAAAA|23913|6|38|33|AM|third|morning|breakfast| +23914|AAAAAAAALGNFAAAA|23914|6|38|34|AM|third|morning|breakfast| +23915|AAAAAAAAMGNFAAAA|23915|6|38|35|AM|third|morning|breakfast| +23916|AAAAAAAANGNFAAAA|23916|6|38|36|AM|third|morning|breakfast| +23917|AAAAAAAAOGNFAAAA|23917|6|38|37|AM|third|morning|breakfast| +23918|AAAAAAAAPGNFAAAA|23918|6|38|38|AM|third|morning|breakfast| +23919|AAAAAAAAAHNFAAAA|23919|6|38|39|AM|third|morning|breakfast| +23920|AAAAAAAABHNFAAAA|23920|6|38|40|AM|third|morning|breakfast| +23921|AAAAAAAACHNFAAAA|23921|6|38|41|AM|third|morning|breakfast| +23922|AAAAAAAADHNFAAAA|23922|6|38|42|AM|third|morning|breakfast| +23923|AAAAAAAAEHNFAAAA|23923|6|38|43|AM|third|morning|breakfast| +23924|AAAAAAAAFHNFAAAA|23924|6|38|44|AM|third|morning|breakfast| +23925|AAAAAAAAGHNFAAAA|23925|6|38|45|AM|third|morning|breakfast| +23926|AAAAAAAAHHNFAAAA|23926|6|38|46|AM|third|morning|breakfast| +23927|AAAAAAAAIHNFAAAA|23927|6|38|47|AM|third|morning|breakfast| +23928|AAAAAAAAJHNFAAAA|23928|6|38|48|AM|third|morning|breakfast| +23929|AAAAAAAAKHNFAAAA|23929|6|38|49|AM|third|morning|breakfast| +23930|AAAAAAAALHNFAAAA|23930|6|38|50|AM|third|morning|breakfast| +23931|AAAAAAAAMHNFAAAA|23931|6|38|51|AM|third|morning|breakfast| +23932|AAAAAAAANHNFAAAA|23932|6|38|52|AM|third|morning|breakfast| +23933|AAAAAAAAOHNFAAAA|23933|6|38|53|AM|third|morning|breakfast| +23934|AAAAAAAAPHNFAAAA|23934|6|38|54|AM|third|morning|breakfast| +23935|AAAAAAAAAINFAAAA|23935|6|38|55|AM|third|morning|breakfast| +23936|AAAAAAAABINFAAAA|23936|6|38|56|AM|third|morning|breakfast| +23937|AAAAAAAACINFAAAA|23937|6|38|57|AM|third|morning|breakfast| +23938|AAAAAAAADINFAAAA|23938|6|38|58|AM|third|morning|breakfast| +23939|AAAAAAAAEINFAAAA|23939|6|38|59|AM|third|morning|breakfast| +23940|AAAAAAAAFINFAAAA|23940|6|39|0|AM|third|morning|breakfast| +23941|AAAAAAAAGINFAAAA|23941|6|39|1|AM|third|morning|breakfast| +23942|AAAAAAAAHINFAAAA|23942|6|39|2|AM|third|morning|breakfast| +23943|AAAAAAAAIINFAAAA|23943|6|39|3|AM|third|morning|breakfast| +23944|AAAAAAAAJINFAAAA|23944|6|39|4|AM|third|morning|breakfast| +23945|AAAAAAAAKINFAAAA|23945|6|39|5|AM|third|morning|breakfast| +23946|AAAAAAAALINFAAAA|23946|6|39|6|AM|third|morning|breakfast| +23947|AAAAAAAAMINFAAAA|23947|6|39|7|AM|third|morning|breakfast| +23948|AAAAAAAANINFAAAA|23948|6|39|8|AM|third|morning|breakfast| +23949|AAAAAAAAOINFAAAA|23949|6|39|9|AM|third|morning|breakfast| +23950|AAAAAAAAPINFAAAA|23950|6|39|10|AM|third|morning|breakfast| +23951|AAAAAAAAAJNFAAAA|23951|6|39|11|AM|third|morning|breakfast| +23952|AAAAAAAABJNFAAAA|23952|6|39|12|AM|third|morning|breakfast| +23953|AAAAAAAACJNFAAAA|23953|6|39|13|AM|third|morning|breakfast| +23954|AAAAAAAADJNFAAAA|23954|6|39|14|AM|third|morning|breakfast| +23955|AAAAAAAAEJNFAAAA|23955|6|39|15|AM|third|morning|breakfast| +23956|AAAAAAAAFJNFAAAA|23956|6|39|16|AM|third|morning|breakfast| +23957|AAAAAAAAGJNFAAAA|23957|6|39|17|AM|third|morning|breakfast| +23958|AAAAAAAAHJNFAAAA|23958|6|39|18|AM|third|morning|breakfast| +23959|AAAAAAAAIJNFAAAA|23959|6|39|19|AM|third|morning|breakfast| +23960|AAAAAAAAJJNFAAAA|23960|6|39|20|AM|third|morning|breakfast| +23961|AAAAAAAAKJNFAAAA|23961|6|39|21|AM|third|morning|breakfast| +23962|AAAAAAAALJNFAAAA|23962|6|39|22|AM|third|morning|breakfast| +23963|AAAAAAAAMJNFAAAA|23963|6|39|23|AM|third|morning|breakfast| +23964|AAAAAAAANJNFAAAA|23964|6|39|24|AM|third|morning|breakfast| +23965|AAAAAAAAOJNFAAAA|23965|6|39|25|AM|third|morning|breakfast| +23966|AAAAAAAAPJNFAAAA|23966|6|39|26|AM|third|morning|breakfast| +23967|AAAAAAAAAKNFAAAA|23967|6|39|27|AM|third|morning|breakfast| +23968|AAAAAAAABKNFAAAA|23968|6|39|28|AM|third|morning|breakfast| +23969|AAAAAAAACKNFAAAA|23969|6|39|29|AM|third|morning|breakfast| +23970|AAAAAAAADKNFAAAA|23970|6|39|30|AM|third|morning|breakfast| +23971|AAAAAAAAEKNFAAAA|23971|6|39|31|AM|third|morning|breakfast| +23972|AAAAAAAAFKNFAAAA|23972|6|39|32|AM|third|morning|breakfast| +23973|AAAAAAAAGKNFAAAA|23973|6|39|33|AM|third|morning|breakfast| +23974|AAAAAAAAHKNFAAAA|23974|6|39|34|AM|third|morning|breakfast| +23975|AAAAAAAAIKNFAAAA|23975|6|39|35|AM|third|morning|breakfast| +23976|AAAAAAAAJKNFAAAA|23976|6|39|36|AM|third|morning|breakfast| +23977|AAAAAAAAKKNFAAAA|23977|6|39|37|AM|third|morning|breakfast| +23978|AAAAAAAALKNFAAAA|23978|6|39|38|AM|third|morning|breakfast| +23979|AAAAAAAAMKNFAAAA|23979|6|39|39|AM|third|morning|breakfast| +23980|AAAAAAAANKNFAAAA|23980|6|39|40|AM|third|morning|breakfast| +23981|AAAAAAAAOKNFAAAA|23981|6|39|41|AM|third|morning|breakfast| +23982|AAAAAAAAPKNFAAAA|23982|6|39|42|AM|third|morning|breakfast| +23983|AAAAAAAAALNFAAAA|23983|6|39|43|AM|third|morning|breakfast| +23984|AAAAAAAABLNFAAAA|23984|6|39|44|AM|third|morning|breakfast| +23985|AAAAAAAACLNFAAAA|23985|6|39|45|AM|third|morning|breakfast| +23986|AAAAAAAADLNFAAAA|23986|6|39|46|AM|third|morning|breakfast| +23987|AAAAAAAAELNFAAAA|23987|6|39|47|AM|third|morning|breakfast| +23988|AAAAAAAAFLNFAAAA|23988|6|39|48|AM|third|morning|breakfast| +23989|AAAAAAAAGLNFAAAA|23989|6|39|49|AM|third|morning|breakfast| +23990|AAAAAAAAHLNFAAAA|23990|6|39|50|AM|third|morning|breakfast| +23991|AAAAAAAAILNFAAAA|23991|6|39|51|AM|third|morning|breakfast| +23992|AAAAAAAAJLNFAAAA|23992|6|39|52|AM|third|morning|breakfast| +23993|AAAAAAAAKLNFAAAA|23993|6|39|53|AM|third|morning|breakfast| +23994|AAAAAAAALLNFAAAA|23994|6|39|54|AM|third|morning|breakfast| +23995|AAAAAAAAMLNFAAAA|23995|6|39|55|AM|third|morning|breakfast| +23996|AAAAAAAANLNFAAAA|23996|6|39|56|AM|third|morning|breakfast| +23997|AAAAAAAAOLNFAAAA|23997|6|39|57|AM|third|morning|breakfast| +23998|AAAAAAAAPLNFAAAA|23998|6|39|58|AM|third|morning|breakfast| +23999|AAAAAAAAAMNFAAAA|23999|6|39|59|AM|third|morning|breakfast| +24000|AAAAAAAABMNFAAAA|24000|6|40|0|AM|third|morning|breakfast| +24001|AAAAAAAACMNFAAAA|24001|6|40|1|AM|third|morning|breakfast| +24002|AAAAAAAADMNFAAAA|24002|6|40|2|AM|third|morning|breakfast| +24003|AAAAAAAAEMNFAAAA|24003|6|40|3|AM|third|morning|breakfast| +24004|AAAAAAAAFMNFAAAA|24004|6|40|4|AM|third|morning|breakfast| +24005|AAAAAAAAGMNFAAAA|24005|6|40|5|AM|third|morning|breakfast| +24006|AAAAAAAAHMNFAAAA|24006|6|40|6|AM|third|morning|breakfast| +24007|AAAAAAAAIMNFAAAA|24007|6|40|7|AM|third|morning|breakfast| +24008|AAAAAAAAJMNFAAAA|24008|6|40|8|AM|third|morning|breakfast| +24009|AAAAAAAAKMNFAAAA|24009|6|40|9|AM|third|morning|breakfast| +24010|AAAAAAAALMNFAAAA|24010|6|40|10|AM|third|morning|breakfast| +24011|AAAAAAAAMMNFAAAA|24011|6|40|11|AM|third|morning|breakfast| +24012|AAAAAAAANMNFAAAA|24012|6|40|12|AM|third|morning|breakfast| +24013|AAAAAAAAOMNFAAAA|24013|6|40|13|AM|third|morning|breakfast| +24014|AAAAAAAAPMNFAAAA|24014|6|40|14|AM|third|morning|breakfast| +24015|AAAAAAAAANNFAAAA|24015|6|40|15|AM|third|morning|breakfast| +24016|AAAAAAAABNNFAAAA|24016|6|40|16|AM|third|morning|breakfast| +24017|AAAAAAAACNNFAAAA|24017|6|40|17|AM|third|morning|breakfast| +24018|AAAAAAAADNNFAAAA|24018|6|40|18|AM|third|morning|breakfast| +24019|AAAAAAAAENNFAAAA|24019|6|40|19|AM|third|morning|breakfast| +24020|AAAAAAAAFNNFAAAA|24020|6|40|20|AM|third|morning|breakfast| +24021|AAAAAAAAGNNFAAAA|24021|6|40|21|AM|third|morning|breakfast| +24022|AAAAAAAAHNNFAAAA|24022|6|40|22|AM|third|morning|breakfast| +24023|AAAAAAAAINNFAAAA|24023|6|40|23|AM|third|morning|breakfast| +24024|AAAAAAAAJNNFAAAA|24024|6|40|24|AM|third|morning|breakfast| +24025|AAAAAAAAKNNFAAAA|24025|6|40|25|AM|third|morning|breakfast| +24026|AAAAAAAALNNFAAAA|24026|6|40|26|AM|third|morning|breakfast| +24027|AAAAAAAAMNNFAAAA|24027|6|40|27|AM|third|morning|breakfast| +24028|AAAAAAAANNNFAAAA|24028|6|40|28|AM|third|morning|breakfast| +24029|AAAAAAAAONNFAAAA|24029|6|40|29|AM|third|morning|breakfast| +24030|AAAAAAAAPNNFAAAA|24030|6|40|30|AM|third|morning|breakfast| +24031|AAAAAAAAAONFAAAA|24031|6|40|31|AM|third|morning|breakfast| +24032|AAAAAAAABONFAAAA|24032|6|40|32|AM|third|morning|breakfast| +24033|AAAAAAAACONFAAAA|24033|6|40|33|AM|third|morning|breakfast| +24034|AAAAAAAADONFAAAA|24034|6|40|34|AM|third|morning|breakfast| +24035|AAAAAAAAEONFAAAA|24035|6|40|35|AM|third|morning|breakfast| +24036|AAAAAAAAFONFAAAA|24036|6|40|36|AM|third|morning|breakfast| +24037|AAAAAAAAGONFAAAA|24037|6|40|37|AM|third|morning|breakfast| +24038|AAAAAAAAHONFAAAA|24038|6|40|38|AM|third|morning|breakfast| +24039|AAAAAAAAIONFAAAA|24039|6|40|39|AM|third|morning|breakfast| +24040|AAAAAAAAJONFAAAA|24040|6|40|40|AM|third|morning|breakfast| +24041|AAAAAAAAKONFAAAA|24041|6|40|41|AM|third|morning|breakfast| +24042|AAAAAAAALONFAAAA|24042|6|40|42|AM|third|morning|breakfast| +24043|AAAAAAAAMONFAAAA|24043|6|40|43|AM|third|morning|breakfast| +24044|AAAAAAAANONFAAAA|24044|6|40|44|AM|third|morning|breakfast| +24045|AAAAAAAAOONFAAAA|24045|6|40|45|AM|third|morning|breakfast| +24046|AAAAAAAAPONFAAAA|24046|6|40|46|AM|third|morning|breakfast| +24047|AAAAAAAAAPNFAAAA|24047|6|40|47|AM|third|morning|breakfast| +24048|AAAAAAAABPNFAAAA|24048|6|40|48|AM|third|morning|breakfast| +24049|AAAAAAAACPNFAAAA|24049|6|40|49|AM|third|morning|breakfast| +24050|AAAAAAAADPNFAAAA|24050|6|40|50|AM|third|morning|breakfast| +24051|AAAAAAAAEPNFAAAA|24051|6|40|51|AM|third|morning|breakfast| +24052|AAAAAAAAFPNFAAAA|24052|6|40|52|AM|third|morning|breakfast| +24053|AAAAAAAAGPNFAAAA|24053|6|40|53|AM|third|morning|breakfast| +24054|AAAAAAAAHPNFAAAA|24054|6|40|54|AM|third|morning|breakfast| +24055|AAAAAAAAIPNFAAAA|24055|6|40|55|AM|third|morning|breakfast| +24056|AAAAAAAAJPNFAAAA|24056|6|40|56|AM|third|morning|breakfast| +24057|AAAAAAAAKPNFAAAA|24057|6|40|57|AM|third|morning|breakfast| +24058|AAAAAAAALPNFAAAA|24058|6|40|58|AM|third|morning|breakfast| +24059|AAAAAAAAMPNFAAAA|24059|6|40|59|AM|third|morning|breakfast| +24060|AAAAAAAANPNFAAAA|24060|6|41|0|AM|third|morning|breakfast| +24061|AAAAAAAAOPNFAAAA|24061|6|41|1|AM|third|morning|breakfast| +24062|AAAAAAAAPPNFAAAA|24062|6|41|2|AM|third|morning|breakfast| +24063|AAAAAAAAAAOFAAAA|24063|6|41|3|AM|third|morning|breakfast| +24064|AAAAAAAABAOFAAAA|24064|6|41|4|AM|third|morning|breakfast| +24065|AAAAAAAACAOFAAAA|24065|6|41|5|AM|third|morning|breakfast| +24066|AAAAAAAADAOFAAAA|24066|6|41|6|AM|third|morning|breakfast| +24067|AAAAAAAAEAOFAAAA|24067|6|41|7|AM|third|morning|breakfast| +24068|AAAAAAAAFAOFAAAA|24068|6|41|8|AM|third|morning|breakfast| +24069|AAAAAAAAGAOFAAAA|24069|6|41|9|AM|third|morning|breakfast| +24070|AAAAAAAAHAOFAAAA|24070|6|41|10|AM|third|morning|breakfast| +24071|AAAAAAAAIAOFAAAA|24071|6|41|11|AM|third|morning|breakfast| +24072|AAAAAAAAJAOFAAAA|24072|6|41|12|AM|third|morning|breakfast| +24073|AAAAAAAAKAOFAAAA|24073|6|41|13|AM|third|morning|breakfast| +24074|AAAAAAAALAOFAAAA|24074|6|41|14|AM|third|morning|breakfast| +24075|AAAAAAAAMAOFAAAA|24075|6|41|15|AM|third|morning|breakfast| +24076|AAAAAAAANAOFAAAA|24076|6|41|16|AM|third|morning|breakfast| +24077|AAAAAAAAOAOFAAAA|24077|6|41|17|AM|third|morning|breakfast| +24078|AAAAAAAAPAOFAAAA|24078|6|41|18|AM|third|morning|breakfast| +24079|AAAAAAAAABOFAAAA|24079|6|41|19|AM|third|morning|breakfast| +24080|AAAAAAAABBOFAAAA|24080|6|41|20|AM|third|morning|breakfast| +24081|AAAAAAAACBOFAAAA|24081|6|41|21|AM|third|morning|breakfast| +24082|AAAAAAAADBOFAAAA|24082|6|41|22|AM|third|morning|breakfast| +24083|AAAAAAAAEBOFAAAA|24083|6|41|23|AM|third|morning|breakfast| +24084|AAAAAAAAFBOFAAAA|24084|6|41|24|AM|third|morning|breakfast| +24085|AAAAAAAAGBOFAAAA|24085|6|41|25|AM|third|morning|breakfast| +24086|AAAAAAAAHBOFAAAA|24086|6|41|26|AM|third|morning|breakfast| +24087|AAAAAAAAIBOFAAAA|24087|6|41|27|AM|third|morning|breakfast| +24088|AAAAAAAAJBOFAAAA|24088|6|41|28|AM|third|morning|breakfast| +24089|AAAAAAAAKBOFAAAA|24089|6|41|29|AM|third|morning|breakfast| +24090|AAAAAAAALBOFAAAA|24090|6|41|30|AM|third|morning|breakfast| +24091|AAAAAAAAMBOFAAAA|24091|6|41|31|AM|third|morning|breakfast| +24092|AAAAAAAANBOFAAAA|24092|6|41|32|AM|third|morning|breakfast| +24093|AAAAAAAAOBOFAAAA|24093|6|41|33|AM|third|morning|breakfast| +24094|AAAAAAAAPBOFAAAA|24094|6|41|34|AM|third|morning|breakfast| +24095|AAAAAAAAACOFAAAA|24095|6|41|35|AM|third|morning|breakfast| +24096|AAAAAAAABCOFAAAA|24096|6|41|36|AM|third|morning|breakfast| +24097|AAAAAAAACCOFAAAA|24097|6|41|37|AM|third|morning|breakfast| +24098|AAAAAAAADCOFAAAA|24098|6|41|38|AM|third|morning|breakfast| +24099|AAAAAAAAECOFAAAA|24099|6|41|39|AM|third|morning|breakfast| +24100|AAAAAAAAFCOFAAAA|24100|6|41|40|AM|third|morning|breakfast| +24101|AAAAAAAAGCOFAAAA|24101|6|41|41|AM|third|morning|breakfast| +24102|AAAAAAAAHCOFAAAA|24102|6|41|42|AM|third|morning|breakfast| +24103|AAAAAAAAICOFAAAA|24103|6|41|43|AM|third|morning|breakfast| +24104|AAAAAAAAJCOFAAAA|24104|6|41|44|AM|third|morning|breakfast| +24105|AAAAAAAAKCOFAAAA|24105|6|41|45|AM|third|morning|breakfast| +24106|AAAAAAAALCOFAAAA|24106|6|41|46|AM|third|morning|breakfast| +24107|AAAAAAAAMCOFAAAA|24107|6|41|47|AM|third|morning|breakfast| +24108|AAAAAAAANCOFAAAA|24108|6|41|48|AM|third|morning|breakfast| +24109|AAAAAAAAOCOFAAAA|24109|6|41|49|AM|third|morning|breakfast| +24110|AAAAAAAAPCOFAAAA|24110|6|41|50|AM|third|morning|breakfast| +24111|AAAAAAAAADOFAAAA|24111|6|41|51|AM|third|morning|breakfast| +24112|AAAAAAAABDOFAAAA|24112|6|41|52|AM|third|morning|breakfast| +24113|AAAAAAAACDOFAAAA|24113|6|41|53|AM|third|morning|breakfast| +24114|AAAAAAAADDOFAAAA|24114|6|41|54|AM|third|morning|breakfast| +24115|AAAAAAAAEDOFAAAA|24115|6|41|55|AM|third|morning|breakfast| +24116|AAAAAAAAFDOFAAAA|24116|6|41|56|AM|third|morning|breakfast| +24117|AAAAAAAAGDOFAAAA|24117|6|41|57|AM|third|morning|breakfast| +24118|AAAAAAAAHDOFAAAA|24118|6|41|58|AM|third|morning|breakfast| +24119|AAAAAAAAIDOFAAAA|24119|6|41|59|AM|third|morning|breakfast| +24120|AAAAAAAAJDOFAAAA|24120|6|42|0|AM|third|morning|breakfast| +24121|AAAAAAAAKDOFAAAA|24121|6|42|1|AM|third|morning|breakfast| +24122|AAAAAAAALDOFAAAA|24122|6|42|2|AM|third|morning|breakfast| +24123|AAAAAAAAMDOFAAAA|24123|6|42|3|AM|third|morning|breakfast| +24124|AAAAAAAANDOFAAAA|24124|6|42|4|AM|third|morning|breakfast| +24125|AAAAAAAAODOFAAAA|24125|6|42|5|AM|third|morning|breakfast| +24126|AAAAAAAAPDOFAAAA|24126|6|42|6|AM|third|morning|breakfast| +24127|AAAAAAAAAEOFAAAA|24127|6|42|7|AM|third|morning|breakfast| +24128|AAAAAAAABEOFAAAA|24128|6|42|8|AM|third|morning|breakfast| +24129|AAAAAAAACEOFAAAA|24129|6|42|9|AM|third|morning|breakfast| +24130|AAAAAAAADEOFAAAA|24130|6|42|10|AM|third|morning|breakfast| +24131|AAAAAAAAEEOFAAAA|24131|6|42|11|AM|third|morning|breakfast| +24132|AAAAAAAAFEOFAAAA|24132|6|42|12|AM|third|morning|breakfast| +24133|AAAAAAAAGEOFAAAA|24133|6|42|13|AM|third|morning|breakfast| +24134|AAAAAAAAHEOFAAAA|24134|6|42|14|AM|third|morning|breakfast| +24135|AAAAAAAAIEOFAAAA|24135|6|42|15|AM|third|morning|breakfast| +24136|AAAAAAAAJEOFAAAA|24136|6|42|16|AM|third|morning|breakfast| +24137|AAAAAAAAKEOFAAAA|24137|6|42|17|AM|third|morning|breakfast| +24138|AAAAAAAALEOFAAAA|24138|6|42|18|AM|third|morning|breakfast| +24139|AAAAAAAAMEOFAAAA|24139|6|42|19|AM|third|morning|breakfast| +24140|AAAAAAAANEOFAAAA|24140|6|42|20|AM|third|morning|breakfast| +24141|AAAAAAAAOEOFAAAA|24141|6|42|21|AM|third|morning|breakfast| +24142|AAAAAAAAPEOFAAAA|24142|6|42|22|AM|third|morning|breakfast| +24143|AAAAAAAAAFOFAAAA|24143|6|42|23|AM|third|morning|breakfast| +24144|AAAAAAAABFOFAAAA|24144|6|42|24|AM|third|morning|breakfast| +24145|AAAAAAAACFOFAAAA|24145|6|42|25|AM|third|morning|breakfast| +24146|AAAAAAAADFOFAAAA|24146|6|42|26|AM|third|morning|breakfast| +24147|AAAAAAAAEFOFAAAA|24147|6|42|27|AM|third|morning|breakfast| +24148|AAAAAAAAFFOFAAAA|24148|6|42|28|AM|third|morning|breakfast| +24149|AAAAAAAAGFOFAAAA|24149|6|42|29|AM|third|morning|breakfast| +24150|AAAAAAAAHFOFAAAA|24150|6|42|30|AM|third|morning|breakfast| +24151|AAAAAAAAIFOFAAAA|24151|6|42|31|AM|third|morning|breakfast| +24152|AAAAAAAAJFOFAAAA|24152|6|42|32|AM|third|morning|breakfast| +24153|AAAAAAAAKFOFAAAA|24153|6|42|33|AM|third|morning|breakfast| +24154|AAAAAAAALFOFAAAA|24154|6|42|34|AM|third|morning|breakfast| +24155|AAAAAAAAMFOFAAAA|24155|6|42|35|AM|third|morning|breakfast| +24156|AAAAAAAANFOFAAAA|24156|6|42|36|AM|third|morning|breakfast| +24157|AAAAAAAAOFOFAAAA|24157|6|42|37|AM|third|morning|breakfast| +24158|AAAAAAAAPFOFAAAA|24158|6|42|38|AM|third|morning|breakfast| +24159|AAAAAAAAAGOFAAAA|24159|6|42|39|AM|third|morning|breakfast| +24160|AAAAAAAABGOFAAAA|24160|6|42|40|AM|third|morning|breakfast| +24161|AAAAAAAACGOFAAAA|24161|6|42|41|AM|third|morning|breakfast| +24162|AAAAAAAADGOFAAAA|24162|6|42|42|AM|third|morning|breakfast| +24163|AAAAAAAAEGOFAAAA|24163|6|42|43|AM|third|morning|breakfast| +24164|AAAAAAAAFGOFAAAA|24164|6|42|44|AM|third|morning|breakfast| +24165|AAAAAAAAGGOFAAAA|24165|6|42|45|AM|third|morning|breakfast| +24166|AAAAAAAAHGOFAAAA|24166|6|42|46|AM|third|morning|breakfast| +24167|AAAAAAAAIGOFAAAA|24167|6|42|47|AM|third|morning|breakfast| +24168|AAAAAAAAJGOFAAAA|24168|6|42|48|AM|third|morning|breakfast| +24169|AAAAAAAAKGOFAAAA|24169|6|42|49|AM|third|morning|breakfast| +24170|AAAAAAAALGOFAAAA|24170|6|42|50|AM|third|morning|breakfast| +24171|AAAAAAAAMGOFAAAA|24171|6|42|51|AM|third|morning|breakfast| +24172|AAAAAAAANGOFAAAA|24172|6|42|52|AM|third|morning|breakfast| +24173|AAAAAAAAOGOFAAAA|24173|6|42|53|AM|third|morning|breakfast| +24174|AAAAAAAAPGOFAAAA|24174|6|42|54|AM|third|morning|breakfast| +24175|AAAAAAAAAHOFAAAA|24175|6|42|55|AM|third|morning|breakfast| +24176|AAAAAAAABHOFAAAA|24176|6|42|56|AM|third|morning|breakfast| +24177|AAAAAAAACHOFAAAA|24177|6|42|57|AM|third|morning|breakfast| +24178|AAAAAAAADHOFAAAA|24178|6|42|58|AM|third|morning|breakfast| +24179|AAAAAAAAEHOFAAAA|24179|6|42|59|AM|third|morning|breakfast| +24180|AAAAAAAAFHOFAAAA|24180|6|43|0|AM|third|morning|breakfast| +24181|AAAAAAAAGHOFAAAA|24181|6|43|1|AM|third|morning|breakfast| +24182|AAAAAAAAHHOFAAAA|24182|6|43|2|AM|third|morning|breakfast| +24183|AAAAAAAAIHOFAAAA|24183|6|43|3|AM|third|morning|breakfast| +24184|AAAAAAAAJHOFAAAA|24184|6|43|4|AM|third|morning|breakfast| +24185|AAAAAAAAKHOFAAAA|24185|6|43|5|AM|third|morning|breakfast| +24186|AAAAAAAALHOFAAAA|24186|6|43|6|AM|third|morning|breakfast| +24187|AAAAAAAAMHOFAAAA|24187|6|43|7|AM|third|morning|breakfast| +24188|AAAAAAAANHOFAAAA|24188|6|43|8|AM|third|morning|breakfast| +24189|AAAAAAAAOHOFAAAA|24189|6|43|9|AM|third|morning|breakfast| +24190|AAAAAAAAPHOFAAAA|24190|6|43|10|AM|third|morning|breakfast| +24191|AAAAAAAAAIOFAAAA|24191|6|43|11|AM|third|morning|breakfast| +24192|AAAAAAAABIOFAAAA|24192|6|43|12|AM|third|morning|breakfast| +24193|AAAAAAAACIOFAAAA|24193|6|43|13|AM|third|morning|breakfast| +24194|AAAAAAAADIOFAAAA|24194|6|43|14|AM|third|morning|breakfast| +24195|AAAAAAAAEIOFAAAA|24195|6|43|15|AM|third|morning|breakfast| +24196|AAAAAAAAFIOFAAAA|24196|6|43|16|AM|third|morning|breakfast| +24197|AAAAAAAAGIOFAAAA|24197|6|43|17|AM|third|morning|breakfast| +24198|AAAAAAAAHIOFAAAA|24198|6|43|18|AM|third|morning|breakfast| +24199|AAAAAAAAIIOFAAAA|24199|6|43|19|AM|third|morning|breakfast| +24200|AAAAAAAAJIOFAAAA|24200|6|43|20|AM|third|morning|breakfast| +24201|AAAAAAAAKIOFAAAA|24201|6|43|21|AM|third|morning|breakfast| +24202|AAAAAAAALIOFAAAA|24202|6|43|22|AM|third|morning|breakfast| +24203|AAAAAAAAMIOFAAAA|24203|6|43|23|AM|third|morning|breakfast| +24204|AAAAAAAANIOFAAAA|24204|6|43|24|AM|third|morning|breakfast| +24205|AAAAAAAAOIOFAAAA|24205|6|43|25|AM|third|morning|breakfast| +24206|AAAAAAAAPIOFAAAA|24206|6|43|26|AM|third|morning|breakfast| +24207|AAAAAAAAAJOFAAAA|24207|6|43|27|AM|third|morning|breakfast| +24208|AAAAAAAABJOFAAAA|24208|6|43|28|AM|third|morning|breakfast| +24209|AAAAAAAACJOFAAAA|24209|6|43|29|AM|third|morning|breakfast| +24210|AAAAAAAADJOFAAAA|24210|6|43|30|AM|third|morning|breakfast| +24211|AAAAAAAAEJOFAAAA|24211|6|43|31|AM|third|morning|breakfast| +24212|AAAAAAAAFJOFAAAA|24212|6|43|32|AM|third|morning|breakfast| +24213|AAAAAAAAGJOFAAAA|24213|6|43|33|AM|third|morning|breakfast| +24214|AAAAAAAAHJOFAAAA|24214|6|43|34|AM|third|morning|breakfast| +24215|AAAAAAAAIJOFAAAA|24215|6|43|35|AM|third|morning|breakfast| +24216|AAAAAAAAJJOFAAAA|24216|6|43|36|AM|third|morning|breakfast| +24217|AAAAAAAAKJOFAAAA|24217|6|43|37|AM|third|morning|breakfast| +24218|AAAAAAAALJOFAAAA|24218|6|43|38|AM|third|morning|breakfast| +24219|AAAAAAAAMJOFAAAA|24219|6|43|39|AM|third|morning|breakfast| +24220|AAAAAAAANJOFAAAA|24220|6|43|40|AM|third|morning|breakfast| +24221|AAAAAAAAOJOFAAAA|24221|6|43|41|AM|third|morning|breakfast| +24222|AAAAAAAAPJOFAAAA|24222|6|43|42|AM|third|morning|breakfast| +24223|AAAAAAAAAKOFAAAA|24223|6|43|43|AM|third|morning|breakfast| +24224|AAAAAAAABKOFAAAA|24224|6|43|44|AM|third|morning|breakfast| +24225|AAAAAAAACKOFAAAA|24225|6|43|45|AM|third|morning|breakfast| +24226|AAAAAAAADKOFAAAA|24226|6|43|46|AM|third|morning|breakfast| +24227|AAAAAAAAEKOFAAAA|24227|6|43|47|AM|third|morning|breakfast| +24228|AAAAAAAAFKOFAAAA|24228|6|43|48|AM|third|morning|breakfast| +24229|AAAAAAAAGKOFAAAA|24229|6|43|49|AM|third|morning|breakfast| +24230|AAAAAAAAHKOFAAAA|24230|6|43|50|AM|third|morning|breakfast| +24231|AAAAAAAAIKOFAAAA|24231|6|43|51|AM|third|morning|breakfast| +24232|AAAAAAAAJKOFAAAA|24232|6|43|52|AM|third|morning|breakfast| +24233|AAAAAAAAKKOFAAAA|24233|6|43|53|AM|third|morning|breakfast| +24234|AAAAAAAALKOFAAAA|24234|6|43|54|AM|third|morning|breakfast| +24235|AAAAAAAAMKOFAAAA|24235|6|43|55|AM|third|morning|breakfast| +24236|AAAAAAAANKOFAAAA|24236|6|43|56|AM|third|morning|breakfast| +24237|AAAAAAAAOKOFAAAA|24237|6|43|57|AM|third|morning|breakfast| +24238|AAAAAAAAPKOFAAAA|24238|6|43|58|AM|third|morning|breakfast| +24239|AAAAAAAAALOFAAAA|24239|6|43|59|AM|third|morning|breakfast| +24240|AAAAAAAABLOFAAAA|24240|6|44|0|AM|third|morning|breakfast| +24241|AAAAAAAACLOFAAAA|24241|6|44|1|AM|third|morning|breakfast| +24242|AAAAAAAADLOFAAAA|24242|6|44|2|AM|third|morning|breakfast| +24243|AAAAAAAAELOFAAAA|24243|6|44|3|AM|third|morning|breakfast| +24244|AAAAAAAAFLOFAAAA|24244|6|44|4|AM|third|morning|breakfast| +24245|AAAAAAAAGLOFAAAA|24245|6|44|5|AM|third|morning|breakfast| +24246|AAAAAAAAHLOFAAAA|24246|6|44|6|AM|third|morning|breakfast| +24247|AAAAAAAAILOFAAAA|24247|6|44|7|AM|third|morning|breakfast| +24248|AAAAAAAAJLOFAAAA|24248|6|44|8|AM|third|morning|breakfast| +24249|AAAAAAAAKLOFAAAA|24249|6|44|9|AM|third|morning|breakfast| +24250|AAAAAAAALLOFAAAA|24250|6|44|10|AM|third|morning|breakfast| +24251|AAAAAAAAMLOFAAAA|24251|6|44|11|AM|third|morning|breakfast| +24252|AAAAAAAANLOFAAAA|24252|6|44|12|AM|third|morning|breakfast| +24253|AAAAAAAAOLOFAAAA|24253|6|44|13|AM|third|morning|breakfast| +24254|AAAAAAAAPLOFAAAA|24254|6|44|14|AM|third|morning|breakfast| +24255|AAAAAAAAAMOFAAAA|24255|6|44|15|AM|third|morning|breakfast| +24256|AAAAAAAABMOFAAAA|24256|6|44|16|AM|third|morning|breakfast| +24257|AAAAAAAACMOFAAAA|24257|6|44|17|AM|third|morning|breakfast| +24258|AAAAAAAADMOFAAAA|24258|6|44|18|AM|third|morning|breakfast| +24259|AAAAAAAAEMOFAAAA|24259|6|44|19|AM|third|morning|breakfast| +24260|AAAAAAAAFMOFAAAA|24260|6|44|20|AM|third|morning|breakfast| +24261|AAAAAAAAGMOFAAAA|24261|6|44|21|AM|third|morning|breakfast| +24262|AAAAAAAAHMOFAAAA|24262|6|44|22|AM|third|morning|breakfast| +24263|AAAAAAAAIMOFAAAA|24263|6|44|23|AM|third|morning|breakfast| +24264|AAAAAAAAJMOFAAAA|24264|6|44|24|AM|third|morning|breakfast| +24265|AAAAAAAAKMOFAAAA|24265|6|44|25|AM|third|morning|breakfast| +24266|AAAAAAAALMOFAAAA|24266|6|44|26|AM|third|morning|breakfast| +24267|AAAAAAAAMMOFAAAA|24267|6|44|27|AM|third|morning|breakfast| +24268|AAAAAAAANMOFAAAA|24268|6|44|28|AM|third|morning|breakfast| +24269|AAAAAAAAOMOFAAAA|24269|6|44|29|AM|third|morning|breakfast| +24270|AAAAAAAAPMOFAAAA|24270|6|44|30|AM|third|morning|breakfast| +24271|AAAAAAAAANOFAAAA|24271|6|44|31|AM|third|morning|breakfast| +24272|AAAAAAAABNOFAAAA|24272|6|44|32|AM|third|morning|breakfast| +24273|AAAAAAAACNOFAAAA|24273|6|44|33|AM|third|morning|breakfast| +24274|AAAAAAAADNOFAAAA|24274|6|44|34|AM|third|morning|breakfast| +24275|AAAAAAAAENOFAAAA|24275|6|44|35|AM|third|morning|breakfast| +24276|AAAAAAAAFNOFAAAA|24276|6|44|36|AM|third|morning|breakfast| +24277|AAAAAAAAGNOFAAAA|24277|6|44|37|AM|third|morning|breakfast| +24278|AAAAAAAAHNOFAAAA|24278|6|44|38|AM|third|morning|breakfast| +24279|AAAAAAAAINOFAAAA|24279|6|44|39|AM|third|morning|breakfast| +24280|AAAAAAAAJNOFAAAA|24280|6|44|40|AM|third|morning|breakfast| +24281|AAAAAAAAKNOFAAAA|24281|6|44|41|AM|third|morning|breakfast| +24282|AAAAAAAALNOFAAAA|24282|6|44|42|AM|third|morning|breakfast| +24283|AAAAAAAAMNOFAAAA|24283|6|44|43|AM|third|morning|breakfast| +24284|AAAAAAAANNOFAAAA|24284|6|44|44|AM|third|morning|breakfast| +24285|AAAAAAAAONOFAAAA|24285|6|44|45|AM|third|morning|breakfast| +24286|AAAAAAAAPNOFAAAA|24286|6|44|46|AM|third|morning|breakfast| +24287|AAAAAAAAAOOFAAAA|24287|6|44|47|AM|third|morning|breakfast| +24288|AAAAAAAABOOFAAAA|24288|6|44|48|AM|third|morning|breakfast| +24289|AAAAAAAACOOFAAAA|24289|6|44|49|AM|third|morning|breakfast| +24290|AAAAAAAADOOFAAAA|24290|6|44|50|AM|third|morning|breakfast| +24291|AAAAAAAAEOOFAAAA|24291|6|44|51|AM|third|morning|breakfast| +24292|AAAAAAAAFOOFAAAA|24292|6|44|52|AM|third|morning|breakfast| +24293|AAAAAAAAGOOFAAAA|24293|6|44|53|AM|third|morning|breakfast| +24294|AAAAAAAAHOOFAAAA|24294|6|44|54|AM|third|morning|breakfast| +24295|AAAAAAAAIOOFAAAA|24295|6|44|55|AM|third|morning|breakfast| +24296|AAAAAAAAJOOFAAAA|24296|6|44|56|AM|third|morning|breakfast| +24297|AAAAAAAAKOOFAAAA|24297|6|44|57|AM|third|morning|breakfast| +24298|AAAAAAAALOOFAAAA|24298|6|44|58|AM|third|morning|breakfast| +24299|AAAAAAAAMOOFAAAA|24299|6|44|59|AM|third|morning|breakfast| +24300|AAAAAAAANOOFAAAA|24300|6|45|0|AM|third|morning|breakfast| +24301|AAAAAAAAOOOFAAAA|24301|6|45|1|AM|third|morning|breakfast| +24302|AAAAAAAAPOOFAAAA|24302|6|45|2|AM|third|morning|breakfast| +24303|AAAAAAAAAPOFAAAA|24303|6|45|3|AM|third|morning|breakfast| +24304|AAAAAAAABPOFAAAA|24304|6|45|4|AM|third|morning|breakfast| +24305|AAAAAAAACPOFAAAA|24305|6|45|5|AM|third|morning|breakfast| +24306|AAAAAAAADPOFAAAA|24306|6|45|6|AM|third|morning|breakfast| +24307|AAAAAAAAEPOFAAAA|24307|6|45|7|AM|third|morning|breakfast| +24308|AAAAAAAAFPOFAAAA|24308|6|45|8|AM|third|morning|breakfast| +24309|AAAAAAAAGPOFAAAA|24309|6|45|9|AM|third|morning|breakfast| +24310|AAAAAAAAHPOFAAAA|24310|6|45|10|AM|third|morning|breakfast| +24311|AAAAAAAAIPOFAAAA|24311|6|45|11|AM|third|morning|breakfast| +24312|AAAAAAAAJPOFAAAA|24312|6|45|12|AM|third|morning|breakfast| +24313|AAAAAAAAKPOFAAAA|24313|6|45|13|AM|third|morning|breakfast| +24314|AAAAAAAALPOFAAAA|24314|6|45|14|AM|third|morning|breakfast| +24315|AAAAAAAAMPOFAAAA|24315|6|45|15|AM|third|morning|breakfast| +24316|AAAAAAAANPOFAAAA|24316|6|45|16|AM|third|morning|breakfast| +24317|AAAAAAAAOPOFAAAA|24317|6|45|17|AM|third|morning|breakfast| +24318|AAAAAAAAPPOFAAAA|24318|6|45|18|AM|third|morning|breakfast| +24319|AAAAAAAAAAPFAAAA|24319|6|45|19|AM|third|morning|breakfast| +24320|AAAAAAAABAPFAAAA|24320|6|45|20|AM|third|morning|breakfast| +24321|AAAAAAAACAPFAAAA|24321|6|45|21|AM|third|morning|breakfast| +24322|AAAAAAAADAPFAAAA|24322|6|45|22|AM|third|morning|breakfast| +24323|AAAAAAAAEAPFAAAA|24323|6|45|23|AM|third|morning|breakfast| +24324|AAAAAAAAFAPFAAAA|24324|6|45|24|AM|third|morning|breakfast| +24325|AAAAAAAAGAPFAAAA|24325|6|45|25|AM|third|morning|breakfast| +24326|AAAAAAAAHAPFAAAA|24326|6|45|26|AM|third|morning|breakfast| +24327|AAAAAAAAIAPFAAAA|24327|6|45|27|AM|third|morning|breakfast| +24328|AAAAAAAAJAPFAAAA|24328|6|45|28|AM|third|morning|breakfast| +24329|AAAAAAAAKAPFAAAA|24329|6|45|29|AM|third|morning|breakfast| +24330|AAAAAAAALAPFAAAA|24330|6|45|30|AM|third|morning|breakfast| +24331|AAAAAAAAMAPFAAAA|24331|6|45|31|AM|third|morning|breakfast| +24332|AAAAAAAANAPFAAAA|24332|6|45|32|AM|third|morning|breakfast| +24333|AAAAAAAAOAPFAAAA|24333|6|45|33|AM|third|morning|breakfast| +24334|AAAAAAAAPAPFAAAA|24334|6|45|34|AM|third|morning|breakfast| +24335|AAAAAAAAABPFAAAA|24335|6|45|35|AM|third|morning|breakfast| +24336|AAAAAAAABBPFAAAA|24336|6|45|36|AM|third|morning|breakfast| +24337|AAAAAAAACBPFAAAA|24337|6|45|37|AM|third|morning|breakfast| +24338|AAAAAAAADBPFAAAA|24338|6|45|38|AM|third|morning|breakfast| +24339|AAAAAAAAEBPFAAAA|24339|6|45|39|AM|third|morning|breakfast| +24340|AAAAAAAAFBPFAAAA|24340|6|45|40|AM|third|morning|breakfast| +24341|AAAAAAAAGBPFAAAA|24341|6|45|41|AM|third|morning|breakfast| +24342|AAAAAAAAHBPFAAAA|24342|6|45|42|AM|third|morning|breakfast| +24343|AAAAAAAAIBPFAAAA|24343|6|45|43|AM|third|morning|breakfast| +24344|AAAAAAAAJBPFAAAA|24344|6|45|44|AM|third|morning|breakfast| +24345|AAAAAAAAKBPFAAAA|24345|6|45|45|AM|third|morning|breakfast| +24346|AAAAAAAALBPFAAAA|24346|6|45|46|AM|third|morning|breakfast| +24347|AAAAAAAAMBPFAAAA|24347|6|45|47|AM|third|morning|breakfast| +24348|AAAAAAAANBPFAAAA|24348|6|45|48|AM|third|morning|breakfast| +24349|AAAAAAAAOBPFAAAA|24349|6|45|49|AM|third|morning|breakfast| +24350|AAAAAAAAPBPFAAAA|24350|6|45|50|AM|third|morning|breakfast| +24351|AAAAAAAAACPFAAAA|24351|6|45|51|AM|third|morning|breakfast| +24352|AAAAAAAABCPFAAAA|24352|6|45|52|AM|third|morning|breakfast| +24353|AAAAAAAACCPFAAAA|24353|6|45|53|AM|third|morning|breakfast| +24354|AAAAAAAADCPFAAAA|24354|6|45|54|AM|third|morning|breakfast| +24355|AAAAAAAAECPFAAAA|24355|6|45|55|AM|third|morning|breakfast| +24356|AAAAAAAAFCPFAAAA|24356|6|45|56|AM|third|morning|breakfast| +24357|AAAAAAAAGCPFAAAA|24357|6|45|57|AM|third|morning|breakfast| +24358|AAAAAAAAHCPFAAAA|24358|6|45|58|AM|third|morning|breakfast| +24359|AAAAAAAAICPFAAAA|24359|6|45|59|AM|third|morning|breakfast| +24360|AAAAAAAAJCPFAAAA|24360|6|46|0|AM|third|morning|breakfast| +24361|AAAAAAAAKCPFAAAA|24361|6|46|1|AM|third|morning|breakfast| +24362|AAAAAAAALCPFAAAA|24362|6|46|2|AM|third|morning|breakfast| +24363|AAAAAAAAMCPFAAAA|24363|6|46|3|AM|third|morning|breakfast| +24364|AAAAAAAANCPFAAAA|24364|6|46|4|AM|third|morning|breakfast| +24365|AAAAAAAAOCPFAAAA|24365|6|46|5|AM|third|morning|breakfast| +24366|AAAAAAAAPCPFAAAA|24366|6|46|6|AM|third|morning|breakfast| +24367|AAAAAAAAADPFAAAA|24367|6|46|7|AM|third|morning|breakfast| +24368|AAAAAAAABDPFAAAA|24368|6|46|8|AM|third|morning|breakfast| +24369|AAAAAAAACDPFAAAA|24369|6|46|9|AM|third|morning|breakfast| +24370|AAAAAAAADDPFAAAA|24370|6|46|10|AM|third|morning|breakfast| +24371|AAAAAAAAEDPFAAAA|24371|6|46|11|AM|third|morning|breakfast| +24372|AAAAAAAAFDPFAAAA|24372|6|46|12|AM|third|morning|breakfast| +24373|AAAAAAAAGDPFAAAA|24373|6|46|13|AM|third|morning|breakfast| +24374|AAAAAAAAHDPFAAAA|24374|6|46|14|AM|third|morning|breakfast| +24375|AAAAAAAAIDPFAAAA|24375|6|46|15|AM|third|morning|breakfast| +24376|AAAAAAAAJDPFAAAA|24376|6|46|16|AM|third|morning|breakfast| +24377|AAAAAAAAKDPFAAAA|24377|6|46|17|AM|third|morning|breakfast| +24378|AAAAAAAALDPFAAAA|24378|6|46|18|AM|third|morning|breakfast| +24379|AAAAAAAAMDPFAAAA|24379|6|46|19|AM|third|morning|breakfast| +24380|AAAAAAAANDPFAAAA|24380|6|46|20|AM|third|morning|breakfast| +24381|AAAAAAAAODPFAAAA|24381|6|46|21|AM|third|morning|breakfast| +24382|AAAAAAAAPDPFAAAA|24382|6|46|22|AM|third|morning|breakfast| +24383|AAAAAAAAAEPFAAAA|24383|6|46|23|AM|third|morning|breakfast| +24384|AAAAAAAABEPFAAAA|24384|6|46|24|AM|third|morning|breakfast| +24385|AAAAAAAACEPFAAAA|24385|6|46|25|AM|third|morning|breakfast| +24386|AAAAAAAADEPFAAAA|24386|6|46|26|AM|third|morning|breakfast| +24387|AAAAAAAAEEPFAAAA|24387|6|46|27|AM|third|morning|breakfast| +24388|AAAAAAAAFEPFAAAA|24388|6|46|28|AM|third|morning|breakfast| +24389|AAAAAAAAGEPFAAAA|24389|6|46|29|AM|third|morning|breakfast| +24390|AAAAAAAAHEPFAAAA|24390|6|46|30|AM|third|morning|breakfast| +24391|AAAAAAAAIEPFAAAA|24391|6|46|31|AM|third|morning|breakfast| +24392|AAAAAAAAJEPFAAAA|24392|6|46|32|AM|third|morning|breakfast| +24393|AAAAAAAAKEPFAAAA|24393|6|46|33|AM|third|morning|breakfast| +24394|AAAAAAAALEPFAAAA|24394|6|46|34|AM|third|morning|breakfast| +24395|AAAAAAAAMEPFAAAA|24395|6|46|35|AM|third|morning|breakfast| +24396|AAAAAAAANEPFAAAA|24396|6|46|36|AM|third|morning|breakfast| +24397|AAAAAAAAOEPFAAAA|24397|6|46|37|AM|third|morning|breakfast| +24398|AAAAAAAAPEPFAAAA|24398|6|46|38|AM|third|morning|breakfast| +24399|AAAAAAAAAFPFAAAA|24399|6|46|39|AM|third|morning|breakfast| +24400|AAAAAAAABFPFAAAA|24400|6|46|40|AM|third|morning|breakfast| +24401|AAAAAAAACFPFAAAA|24401|6|46|41|AM|third|morning|breakfast| +24402|AAAAAAAADFPFAAAA|24402|6|46|42|AM|third|morning|breakfast| +24403|AAAAAAAAEFPFAAAA|24403|6|46|43|AM|third|morning|breakfast| +24404|AAAAAAAAFFPFAAAA|24404|6|46|44|AM|third|morning|breakfast| +24405|AAAAAAAAGFPFAAAA|24405|6|46|45|AM|third|morning|breakfast| +24406|AAAAAAAAHFPFAAAA|24406|6|46|46|AM|third|morning|breakfast| +24407|AAAAAAAAIFPFAAAA|24407|6|46|47|AM|third|morning|breakfast| +24408|AAAAAAAAJFPFAAAA|24408|6|46|48|AM|third|morning|breakfast| +24409|AAAAAAAAKFPFAAAA|24409|6|46|49|AM|third|morning|breakfast| +24410|AAAAAAAALFPFAAAA|24410|6|46|50|AM|third|morning|breakfast| +24411|AAAAAAAAMFPFAAAA|24411|6|46|51|AM|third|morning|breakfast| +24412|AAAAAAAANFPFAAAA|24412|6|46|52|AM|third|morning|breakfast| +24413|AAAAAAAAOFPFAAAA|24413|6|46|53|AM|third|morning|breakfast| +24414|AAAAAAAAPFPFAAAA|24414|6|46|54|AM|third|morning|breakfast| +24415|AAAAAAAAAGPFAAAA|24415|6|46|55|AM|third|morning|breakfast| +24416|AAAAAAAABGPFAAAA|24416|6|46|56|AM|third|morning|breakfast| +24417|AAAAAAAACGPFAAAA|24417|6|46|57|AM|third|morning|breakfast| +24418|AAAAAAAADGPFAAAA|24418|6|46|58|AM|third|morning|breakfast| +24419|AAAAAAAAEGPFAAAA|24419|6|46|59|AM|third|morning|breakfast| +24420|AAAAAAAAFGPFAAAA|24420|6|47|0|AM|third|morning|breakfast| +24421|AAAAAAAAGGPFAAAA|24421|6|47|1|AM|third|morning|breakfast| +24422|AAAAAAAAHGPFAAAA|24422|6|47|2|AM|third|morning|breakfast| +24423|AAAAAAAAIGPFAAAA|24423|6|47|3|AM|third|morning|breakfast| +24424|AAAAAAAAJGPFAAAA|24424|6|47|4|AM|third|morning|breakfast| +24425|AAAAAAAAKGPFAAAA|24425|6|47|5|AM|third|morning|breakfast| +24426|AAAAAAAALGPFAAAA|24426|6|47|6|AM|third|morning|breakfast| +24427|AAAAAAAAMGPFAAAA|24427|6|47|7|AM|third|morning|breakfast| +24428|AAAAAAAANGPFAAAA|24428|6|47|8|AM|third|morning|breakfast| +24429|AAAAAAAAOGPFAAAA|24429|6|47|9|AM|third|morning|breakfast| +24430|AAAAAAAAPGPFAAAA|24430|6|47|10|AM|third|morning|breakfast| +24431|AAAAAAAAAHPFAAAA|24431|6|47|11|AM|third|morning|breakfast| +24432|AAAAAAAABHPFAAAA|24432|6|47|12|AM|third|morning|breakfast| +24433|AAAAAAAACHPFAAAA|24433|6|47|13|AM|third|morning|breakfast| +24434|AAAAAAAADHPFAAAA|24434|6|47|14|AM|third|morning|breakfast| +24435|AAAAAAAAEHPFAAAA|24435|6|47|15|AM|third|morning|breakfast| +24436|AAAAAAAAFHPFAAAA|24436|6|47|16|AM|third|morning|breakfast| +24437|AAAAAAAAGHPFAAAA|24437|6|47|17|AM|third|morning|breakfast| +24438|AAAAAAAAHHPFAAAA|24438|6|47|18|AM|third|morning|breakfast| +24439|AAAAAAAAIHPFAAAA|24439|6|47|19|AM|third|morning|breakfast| +24440|AAAAAAAAJHPFAAAA|24440|6|47|20|AM|third|morning|breakfast| +24441|AAAAAAAAKHPFAAAA|24441|6|47|21|AM|third|morning|breakfast| +24442|AAAAAAAALHPFAAAA|24442|6|47|22|AM|third|morning|breakfast| +24443|AAAAAAAAMHPFAAAA|24443|6|47|23|AM|third|morning|breakfast| +24444|AAAAAAAANHPFAAAA|24444|6|47|24|AM|third|morning|breakfast| +24445|AAAAAAAAOHPFAAAA|24445|6|47|25|AM|third|morning|breakfast| +24446|AAAAAAAAPHPFAAAA|24446|6|47|26|AM|third|morning|breakfast| +24447|AAAAAAAAAIPFAAAA|24447|6|47|27|AM|third|morning|breakfast| +24448|AAAAAAAABIPFAAAA|24448|6|47|28|AM|third|morning|breakfast| +24449|AAAAAAAACIPFAAAA|24449|6|47|29|AM|third|morning|breakfast| +24450|AAAAAAAADIPFAAAA|24450|6|47|30|AM|third|morning|breakfast| +24451|AAAAAAAAEIPFAAAA|24451|6|47|31|AM|third|morning|breakfast| +24452|AAAAAAAAFIPFAAAA|24452|6|47|32|AM|third|morning|breakfast| +24453|AAAAAAAAGIPFAAAA|24453|6|47|33|AM|third|morning|breakfast| +24454|AAAAAAAAHIPFAAAA|24454|6|47|34|AM|third|morning|breakfast| +24455|AAAAAAAAIIPFAAAA|24455|6|47|35|AM|third|morning|breakfast| +24456|AAAAAAAAJIPFAAAA|24456|6|47|36|AM|third|morning|breakfast| +24457|AAAAAAAAKIPFAAAA|24457|6|47|37|AM|third|morning|breakfast| +24458|AAAAAAAALIPFAAAA|24458|6|47|38|AM|third|morning|breakfast| +24459|AAAAAAAAMIPFAAAA|24459|6|47|39|AM|third|morning|breakfast| +24460|AAAAAAAANIPFAAAA|24460|6|47|40|AM|third|morning|breakfast| +24461|AAAAAAAAOIPFAAAA|24461|6|47|41|AM|third|morning|breakfast| +24462|AAAAAAAAPIPFAAAA|24462|6|47|42|AM|third|morning|breakfast| +24463|AAAAAAAAAJPFAAAA|24463|6|47|43|AM|third|morning|breakfast| +24464|AAAAAAAABJPFAAAA|24464|6|47|44|AM|third|morning|breakfast| +24465|AAAAAAAACJPFAAAA|24465|6|47|45|AM|third|morning|breakfast| +24466|AAAAAAAADJPFAAAA|24466|6|47|46|AM|third|morning|breakfast| +24467|AAAAAAAAEJPFAAAA|24467|6|47|47|AM|third|morning|breakfast| +24468|AAAAAAAAFJPFAAAA|24468|6|47|48|AM|third|morning|breakfast| +24469|AAAAAAAAGJPFAAAA|24469|6|47|49|AM|third|morning|breakfast| +24470|AAAAAAAAHJPFAAAA|24470|6|47|50|AM|third|morning|breakfast| +24471|AAAAAAAAIJPFAAAA|24471|6|47|51|AM|third|morning|breakfast| +24472|AAAAAAAAJJPFAAAA|24472|6|47|52|AM|third|morning|breakfast| +24473|AAAAAAAAKJPFAAAA|24473|6|47|53|AM|third|morning|breakfast| +24474|AAAAAAAALJPFAAAA|24474|6|47|54|AM|third|morning|breakfast| +24475|AAAAAAAAMJPFAAAA|24475|6|47|55|AM|third|morning|breakfast| +24476|AAAAAAAANJPFAAAA|24476|6|47|56|AM|third|morning|breakfast| +24477|AAAAAAAAOJPFAAAA|24477|6|47|57|AM|third|morning|breakfast| +24478|AAAAAAAAPJPFAAAA|24478|6|47|58|AM|third|morning|breakfast| +24479|AAAAAAAAAKPFAAAA|24479|6|47|59|AM|third|morning|breakfast| +24480|AAAAAAAABKPFAAAA|24480|6|48|0|AM|third|morning|breakfast| +24481|AAAAAAAACKPFAAAA|24481|6|48|1|AM|third|morning|breakfast| +24482|AAAAAAAADKPFAAAA|24482|6|48|2|AM|third|morning|breakfast| +24483|AAAAAAAAEKPFAAAA|24483|6|48|3|AM|third|morning|breakfast| +24484|AAAAAAAAFKPFAAAA|24484|6|48|4|AM|third|morning|breakfast| +24485|AAAAAAAAGKPFAAAA|24485|6|48|5|AM|third|morning|breakfast| +24486|AAAAAAAAHKPFAAAA|24486|6|48|6|AM|third|morning|breakfast| +24487|AAAAAAAAIKPFAAAA|24487|6|48|7|AM|third|morning|breakfast| +24488|AAAAAAAAJKPFAAAA|24488|6|48|8|AM|third|morning|breakfast| +24489|AAAAAAAAKKPFAAAA|24489|6|48|9|AM|third|morning|breakfast| +24490|AAAAAAAALKPFAAAA|24490|6|48|10|AM|third|morning|breakfast| +24491|AAAAAAAAMKPFAAAA|24491|6|48|11|AM|third|morning|breakfast| +24492|AAAAAAAANKPFAAAA|24492|6|48|12|AM|third|morning|breakfast| +24493|AAAAAAAAOKPFAAAA|24493|6|48|13|AM|third|morning|breakfast| +24494|AAAAAAAAPKPFAAAA|24494|6|48|14|AM|third|morning|breakfast| +24495|AAAAAAAAALPFAAAA|24495|6|48|15|AM|third|morning|breakfast| +24496|AAAAAAAABLPFAAAA|24496|6|48|16|AM|third|morning|breakfast| +24497|AAAAAAAACLPFAAAA|24497|6|48|17|AM|third|morning|breakfast| +24498|AAAAAAAADLPFAAAA|24498|6|48|18|AM|third|morning|breakfast| +24499|AAAAAAAAELPFAAAA|24499|6|48|19|AM|third|morning|breakfast| +24500|AAAAAAAAFLPFAAAA|24500|6|48|20|AM|third|morning|breakfast| +24501|AAAAAAAAGLPFAAAA|24501|6|48|21|AM|third|morning|breakfast| +24502|AAAAAAAAHLPFAAAA|24502|6|48|22|AM|third|morning|breakfast| +24503|AAAAAAAAILPFAAAA|24503|6|48|23|AM|third|morning|breakfast| +24504|AAAAAAAAJLPFAAAA|24504|6|48|24|AM|third|morning|breakfast| +24505|AAAAAAAAKLPFAAAA|24505|6|48|25|AM|third|morning|breakfast| +24506|AAAAAAAALLPFAAAA|24506|6|48|26|AM|third|morning|breakfast| +24507|AAAAAAAAMLPFAAAA|24507|6|48|27|AM|third|morning|breakfast| +24508|AAAAAAAANLPFAAAA|24508|6|48|28|AM|third|morning|breakfast| +24509|AAAAAAAAOLPFAAAA|24509|6|48|29|AM|third|morning|breakfast| +24510|AAAAAAAAPLPFAAAA|24510|6|48|30|AM|third|morning|breakfast| +24511|AAAAAAAAAMPFAAAA|24511|6|48|31|AM|third|morning|breakfast| +24512|AAAAAAAABMPFAAAA|24512|6|48|32|AM|third|morning|breakfast| +24513|AAAAAAAACMPFAAAA|24513|6|48|33|AM|third|morning|breakfast| +24514|AAAAAAAADMPFAAAA|24514|6|48|34|AM|third|morning|breakfast| +24515|AAAAAAAAEMPFAAAA|24515|6|48|35|AM|third|morning|breakfast| +24516|AAAAAAAAFMPFAAAA|24516|6|48|36|AM|third|morning|breakfast| +24517|AAAAAAAAGMPFAAAA|24517|6|48|37|AM|third|morning|breakfast| +24518|AAAAAAAAHMPFAAAA|24518|6|48|38|AM|third|morning|breakfast| +24519|AAAAAAAAIMPFAAAA|24519|6|48|39|AM|third|morning|breakfast| +24520|AAAAAAAAJMPFAAAA|24520|6|48|40|AM|third|morning|breakfast| +24521|AAAAAAAAKMPFAAAA|24521|6|48|41|AM|third|morning|breakfast| +24522|AAAAAAAALMPFAAAA|24522|6|48|42|AM|third|morning|breakfast| +24523|AAAAAAAAMMPFAAAA|24523|6|48|43|AM|third|morning|breakfast| +24524|AAAAAAAANMPFAAAA|24524|6|48|44|AM|third|morning|breakfast| +24525|AAAAAAAAOMPFAAAA|24525|6|48|45|AM|third|morning|breakfast| +24526|AAAAAAAAPMPFAAAA|24526|6|48|46|AM|third|morning|breakfast| +24527|AAAAAAAAANPFAAAA|24527|6|48|47|AM|third|morning|breakfast| +24528|AAAAAAAABNPFAAAA|24528|6|48|48|AM|third|morning|breakfast| +24529|AAAAAAAACNPFAAAA|24529|6|48|49|AM|third|morning|breakfast| +24530|AAAAAAAADNPFAAAA|24530|6|48|50|AM|third|morning|breakfast| +24531|AAAAAAAAENPFAAAA|24531|6|48|51|AM|third|morning|breakfast| +24532|AAAAAAAAFNPFAAAA|24532|6|48|52|AM|third|morning|breakfast| +24533|AAAAAAAAGNPFAAAA|24533|6|48|53|AM|third|morning|breakfast| +24534|AAAAAAAAHNPFAAAA|24534|6|48|54|AM|third|morning|breakfast| +24535|AAAAAAAAINPFAAAA|24535|6|48|55|AM|third|morning|breakfast| +24536|AAAAAAAAJNPFAAAA|24536|6|48|56|AM|third|morning|breakfast| +24537|AAAAAAAAKNPFAAAA|24537|6|48|57|AM|third|morning|breakfast| +24538|AAAAAAAALNPFAAAA|24538|6|48|58|AM|third|morning|breakfast| +24539|AAAAAAAAMNPFAAAA|24539|6|48|59|AM|third|morning|breakfast| +24540|AAAAAAAANNPFAAAA|24540|6|49|0|AM|third|morning|breakfast| +24541|AAAAAAAAONPFAAAA|24541|6|49|1|AM|third|morning|breakfast| +24542|AAAAAAAAPNPFAAAA|24542|6|49|2|AM|third|morning|breakfast| +24543|AAAAAAAAAOPFAAAA|24543|6|49|3|AM|third|morning|breakfast| +24544|AAAAAAAABOPFAAAA|24544|6|49|4|AM|third|morning|breakfast| +24545|AAAAAAAACOPFAAAA|24545|6|49|5|AM|third|morning|breakfast| +24546|AAAAAAAADOPFAAAA|24546|6|49|6|AM|third|morning|breakfast| +24547|AAAAAAAAEOPFAAAA|24547|6|49|7|AM|third|morning|breakfast| +24548|AAAAAAAAFOPFAAAA|24548|6|49|8|AM|third|morning|breakfast| +24549|AAAAAAAAGOPFAAAA|24549|6|49|9|AM|third|morning|breakfast| +24550|AAAAAAAAHOPFAAAA|24550|6|49|10|AM|third|morning|breakfast| +24551|AAAAAAAAIOPFAAAA|24551|6|49|11|AM|third|morning|breakfast| +24552|AAAAAAAAJOPFAAAA|24552|6|49|12|AM|third|morning|breakfast| +24553|AAAAAAAAKOPFAAAA|24553|6|49|13|AM|third|morning|breakfast| +24554|AAAAAAAALOPFAAAA|24554|6|49|14|AM|third|morning|breakfast| +24555|AAAAAAAAMOPFAAAA|24555|6|49|15|AM|third|morning|breakfast| +24556|AAAAAAAANOPFAAAA|24556|6|49|16|AM|third|morning|breakfast| +24557|AAAAAAAAOOPFAAAA|24557|6|49|17|AM|third|morning|breakfast| +24558|AAAAAAAAPOPFAAAA|24558|6|49|18|AM|third|morning|breakfast| +24559|AAAAAAAAAPPFAAAA|24559|6|49|19|AM|third|morning|breakfast| +24560|AAAAAAAABPPFAAAA|24560|6|49|20|AM|third|morning|breakfast| +24561|AAAAAAAACPPFAAAA|24561|6|49|21|AM|third|morning|breakfast| +24562|AAAAAAAADPPFAAAA|24562|6|49|22|AM|third|morning|breakfast| +24563|AAAAAAAAEPPFAAAA|24563|6|49|23|AM|third|morning|breakfast| +24564|AAAAAAAAFPPFAAAA|24564|6|49|24|AM|third|morning|breakfast| +24565|AAAAAAAAGPPFAAAA|24565|6|49|25|AM|third|morning|breakfast| +24566|AAAAAAAAHPPFAAAA|24566|6|49|26|AM|third|morning|breakfast| +24567|AAAAAAAAIPPFAAAA|24567|6|49|27|AM|third|morning|breakfast| +24568|AAAAAAAAJPPFAAAA|24568|6|49|28|AM|third|morning|breakfast| +24569|AAAAAAAAKPPFAAAA|24569|6|49|29|AM|third|morning|breakfast| +24570|AAAAAAAALPPFAAAA|24570|6|49|30|AM|third|morning|breakfast| +24571|AAAAAAAAMPPFAAAA|24571|6|49|31|AM|third|morning|breakfast| +24572|AAAAAAAANPPFAAAA|24572|6|49|32|AM|third|morning|breakfast| +24573|AAAAAAAAOPPFAAAA|24573|6|49|33|AM|third|morning|breakfast| +24574|AAAAAAAAPPPFAAAA|24574|6|49|34|AM|third|morning|breakfast| +24575|AAAAAAAAAAAGAAAA|24575|6|49|35|AM|third|morning|breakfast| +24576|AAAAAAAABAAGAAAA|24576|6|49|36|AM|third|morning|breakfast| +24577|AAAAAAAACAAGAAAA|24577|6|49|37|AM|third|morning|breakfast| +24578|AAAAAAAADAAGAAAA|24578|6|49|38|AM|third|morning|breakfast| +24579|AAAAAAAAEAAGAAAA|24579|6|49|39|AM|third|morning|breakfast| +24580|AAAAAAAAFAAGAAAA|24580|6|49|40|AM|third|morning|breakfast| +24581|AAAAAAAAGAAGAAAA|24581|6|49|41|AM|third|morning|breakfast| +24582|AAAAAAAAHAAGAAAA|24582|6|49|42|AM|third|morning|breakfast| +24583|AAAAAAAAIAAGAAAA|24583|6|49|43|AM|third|morning|breakfast| +24584|AAAAAAAAJAAGAAAA|24584|6|49|44|AM|third|morning|breakfast| +24585|AAAAAAAAKAAGAAAA|24585|6|49|45|AM|third|morning|breakfast| +24586|AAAAAAAALAAGAAAA|24586|6|49|46|AM|third|morning|breakfast| +24587|AAAAAAAAMAAGAAAA|24587|6|49|47|AM|third|morning|breakfast| +24588|AAAAAAAANAAGAAAA|24588|6|49|48|AM|third|morning|breakfast| +24589|AAAAAAAAOAAGAAAA|24589|6|49|49|AM|third|morning|breakfast| +24590|AAAAAAAAPAAGAAAA|24590|6|49|50|AM|third|morning|breakfast| +24591|AAAAAAAAABAGAAAA|24591|6|49|51|AM|third|morning|breakfast| +24592|AAAAAAAABBAGAAAA|24592|6|49|52|AM|third|morning|breakfast| +24593|AAAAAAAACBAGAAAA|24593|6|49|53|AM|third|morning|breakfast| +24594|AAAAAAAADBAGAAAA|24594|6|49|54|AM|third|morning|breakfast| +24595|AAAAAAAAEBAGAAAA|24595|6|49|55|AM|third|morning|breakfast| +24596|AAAAAAAAFBAGAAAA|24596|6|49|56|AM|third|morning|breakfast| +24597|AAAAAAAAGBAGAAAA|24597|6|49|57|AM|third|morning|breakfast| +24598|AAAAAAAAHBAGAAAA|24598|6|49|58|AM|third|morning|breakfast| +24599|AAAAAAAAIBAGAAAA|24599|6|49|59|AM|third|morning|breakfast| +24600|AAAAAAAAJBAGAAAA|24600|6|50|0|AM|third|morning|breakfast| +24601|AAAAAAAAKBAGAAAA|24601|6|50|1|AM|third|morning|breakfast| +24602|AAAAAAAALBAGAAAA|24602|6|50|2|AM|third|morning|breakfast| +24603|AAAAAAAAMBAGAAAA|24603|6|50|3|AM|third|morning|breakfast| +24604|AAAAAAAANBAGAAAA|24604|6|50|4|AM|third|morning|breakfast| +24605|AAAAAAAAOBAGAAAA|24605|6|50|5|AM|third|morning|breakfast| +24606|AAAAAAAAPBAGAAAA|24606|6|50|6|AM|third|morning|breakfast| +24607|AAAAAAAAACAGAAAA|24607|6|50|7|AM|third|morning|breakfast| +24608|AAAAAAAABCAGAAAA|24608|6|50|8|AM|third|morning|breakfast| +24609|AAAAAAAACCAGAAAA|24609|6|50|9|AM|third|morning|breakfast| +24610|AAAAAAAADCAGAAAA|24610|6|50|10|AM|third|morning|breakfast| +24611|AAAAAAAAECAGAAAA|24611|6|50|11|AM|third|morning|breakfast| +24612|AAAAAAAAFCAGAAAA|24612|6|50|12|AM|third|morning|breakfast| +24613|AAAAAAAAGCAGAAAA|24613|6|50|13|AM|third|morning|breakfast| +24614|AAAAAAAAHCAGAAAA|24614|6|50|14|AM|third|morning|breakfast| +24615|AAAAAAAAICAGAAAA|24615|6|50|15|AM|third|morning|breakfast| +24616|AAAAAAAAJCAGAAAA|24616|6|50|16|AM|third|morning|breakfast| +24617|AAAAAAAAKCAGAAAA|24617|6|50|17|AM|third|morning|breakfast| +24618|AAAAAAAALCAGAAAA|24618|6|50|18|AM|third|morning|breakfast| +24619|AAAAAAAAMCAGAAAA|24619|6|50|19|AM|third|morning|breakfast| +24620|AAAAAAAANCAGAAAA|24620|6|50|20|AM|third|morning|breakfast| +24621|AAAAAAAAOCAGAAAA|24621|6|50|21|AM|third|morning|breakfast| +24622|AAAAAAAAPCAGAAAA|24622|6|50|22|AM|third|morning|breakfast| +24623|AAAAAAAAADAGAAAA|24623|6|50|23|AM|third|morning|breakfast| +24624|AAAAAAAABDAGAAAA|24624|6|50|24|AM|third|morning|breakfast| +24625|AAAAAAAACDAGAAAA|24625|6|50|25|AM|third|morning|breakfast| +24626|AAAAAAAADDAGAAAA|24626|6|50|26|AM|third|morning|breakfast| +24627|AAAAAAAAEDAGAAAA|24627|6|50|27|AM|third|morning|breakfast| +24628|AAAAAAAAFDAGAAAA|24628|6|50|28|AM|third|morning|breakfast| +24629|AAAAAAAAGDAGAAAA|24629|6|50|29|AM|third|morning|breakfast| +24630|AAAAAAAAHDAGAAAA|24630|6|50|30|AM|third|morning|breakfast| +24631|AAAAAAAAIDAGAAAA|24631|6|50|31|AM|third|morning|breakfast| +24632|AAAAAAAAJDAGAAAA|24632|6|50|32|AM|third|morning|breakfast| +24633|AAAAAAAAKDAGAAAA|24633|6|50|33|AM|third|morning|breakfast| +24634|AAAAAAAALDAGAAAA|24634|6|50|34|AM|third|morning|breakfast| +24635|AAAAAAAAMDAGAAAA|24635|6|50|35|AM|third|morning|breakfast| +24636|AAAAAAAANDAGAAAA|24636|6|50|36|AM|third|morning|breakfast| +24637|AAAAAAAAODAGAAAA|24637|6|50|37|AM|third|morning|breakfast| +24638|AAAAAAAAPDAGAAAA|24638|6|50|38|AM|third|morning|breakfast| +24639|AAAAAAAAAEAGAAAA|24639|6|50|39|AM|third|morning|breakfast| +24640|AAAAAAAABEAGAAAA|24640|6|50|40|AM|third|morning|breakfast| +24641|AAAAAAAACEAGAAAA|24641|6|50|41|AM|third|morning|breakfast| +24642|AAAAAAAADEAGAAAA|24642|6|50|42|AM|third|morning|breakfast| +24643|AAAAAAAAEEAGAAAA|24643|6|50|43|AM|third|morning|breakfast| +24644|AAAAAAAAFEAGAAAA|24644|6|50|44|AM|third|morning|breakfast| +24645|AAAAAAAAGEAGAAAA|24645|6|50|45|AM|third|morning|breakfast| +24646|AAAAAAAAHEAGAAAA|24646|6|50|46|AM|third|morning|breakfast| +24647|AAAAAAAAIEAGAAAA|24647|6|50|47|AM|third|morning|breakfast| +24648|AAAAAAAAJEAGAAAA|24648|6|50|48|AM|third|morning|breakfast| +24649|AAAAAAAAKEAGAAAA|24649|6|50|49|AM|third|morning|breakfast| +24650|AAAAAAAALEAGAAAA|24650|6|50|50|AM|third|morning|breakfast| +24651|AAAAAAAAMEAGAAAA|24651|6|50|51|AM|third|morning|breakfast| +24652|AAAAAAAANEAGAAAA|24652|6|50|52|AM|third|morning|breakfast| +24653|AAAAAAAAOEAGAAAA|24653|6|50|53|AM|third|morning|breakfast| +24654|AAAAAAAAPEAGAAAA|24654|6|50|54|AM|third|morning|breakfast| +24655|AAAAAAAAAFAGAAAA|24655|6|50|55|AM|third|morning|breakfast| +24656|AAAAAAAABFAGAAAA|24656|6|50|56|AM|third|morning|breakfast| +24657|AAAAAAAACFAGAAAA|24657|6|50|57|AM|third|morning|breakfast| +24658|AAAAAAAADFAGAAAA|24658|6|50|58|AM|third|morning|breakfast| +24659|AAAAAAAAEFAGAAAA|24659|6|50|59|AM|third|morning|breakfast| +24660|AAAAAAAAFFAGAAAA|24660|6|51|0|AM|third|morning|breakfast| +24661|AAAAAAAAGFAGAAAA|24661|6|51|1|AM|third|morning|breakfast| +24662|AAAAAAAAHFAGAAAA|24662|6|51|2|AM|third|morning|breakfast| +24663|AAAAAAAAIFAGAAAA|24663|6|51|3|AM|third|morning|breakfast| +24664|AAAAAAAAJFAGAAAA|24664|6|51|4|AM|third|morning|breakfast| +24665|AAAAAAAAKFAGAAAA|24665|6|51|5|AM|third|morning|breakfast| +24666|AAAAAAAALFAGAAAA|24666|6|51|6|AM|third|morning|breakfast| +24667|AAAAAAAAMFAGAAAA|24667|6|51|7|AM|third|morning|breakfast| +24668|AAAAAAAANFAGAAAA|24668|6|51|8|AM|third|morning|breakfast| +24669|AAAAAAAAOFAGAAAA|24669|6|51|9|AM|third|morning|breakfast| +24670|AAAAAAAAPFAGAAAA|24670|6|51|10|AM|third|morning|breakfast| +24671|AAAAAAAAAGAGAAAA|24671|6|51|11|AM|third|morning|breakfast| +24672|AAAAAAAABGAGAAAA|24672|6|51|12|AM|third|morning|breakfast| +24673|AAAAAAAACGAGAAAA|24673|6|51|13|AM|third|morning|breakfast| +24674|AAAAAAAADGAGAAAA|24674|6|51|14|AM|third|morning|breakfast| +24675|AAAAAAAAEGAGAAAA|24675|6|51|15|AM|third|morning|breakfast| +24676|AAAAAAAAFGAGAAAA|24676|6|51|16|AM|third|morning|breakfast| +24677|AAAAAAAAGGAGAAAA|24677|6|51|17|AM|third|morning|breakfast| +24678|AAAAAAAAHGAGAAAA|24678|6|51|18|AM|third|morning|breakfast| +24679|AAAAAAAAIGAGAAAA|24679|6|51|19|AM|third|morning|breakfast| +24680|AAAAAAAAJGAGAAAA|24680|6|51|20|AM|third|morning|breakfast| +24681|AAAAAAAAKGAGAAAA|24681|6|51|21|AM|third|morning|breakfast| +24682|AAAAAAAALGAGAAAA|24682|6|51|22|AM|third|morning|breakfast| +24683|AAAAAAAAMGAGAAAA|24683|6|51|23|AM|third|morning|breakfast| +24684|AAAAAAAANGAGAAAA|24684|6|51|24|AM|third|morning|breakfast| +24685|AAAAAAAAOGAGAAAA|24685|6|51|25|AM|third|morning|breakfast| +24686|AAAAAAAAPGAGAAAA|24686|6|51|26|AM|third|morning|breakfast| +24687|AAAAAAAAAHAGAAAA|24687|6|51|27|AM|third|morning|breakfast| +24688|AAAAAAAABHAGAAAA|24688|6|51|28|AM|third|morning|breakfast| +24689|AAAAAAAACHAGAAAA|24689|6|51|29|AM|third|morning|breakfast| +24690|AAAAAAAADHAGAAAA|24690|6|51|30|AM|third|morning|breakfast| +24691|AAAAAAAAEHAGAAAA|24691|6|51|31|AM|third|morning|breakfast| +24692|AAAAAAAAFHAGAAAA|24692|6|51|32|AM|third|morning|breakfast| +24693|AAAAAAAAGHAGAAAA|24693|6|51|33|AM|third|morning|breakfast| +24694|AAAAAAAAHHAGAAAA|24694|6|51|34|AM|third|morning|breakfast| +24695|AAAAAAAAIHAGAAAA|24695|6|51|35|AM|third|morning|breakfast| +24696|AAAAAAAAJHAGAAAA|24696|6|51|36|AM|third|morning|breakfast| +24697|AAAAAAAAKHAGAAAA|24697|6|51|37|AM|third|morning|breakfast| +24698|AAAAAAAALHAGAAAA|24698|6|51|38|AM|third|morning|breakfast| +24699|AAAAAAAAMHAGAAAA|24699|6|51|39|AM|third|morning|breakfast| +24700|AAAAAAAANHAGAAAA|24700|6|51|40|AM|third|morning|breakfast| +24701|AAAAAAAAOHAGAAAA|24701|6|51|41|AM|third|morning|breakfast| +24702|AAAAAAAAPHAGAAAA|24702|6|51|42|AM|third|morning|breakfast| +24703|AAAAAAAAAIAGAAAA|24703|6|51|43|AM|third|morning|breakfast| +24704|AAAAAAAABIAGAAAA|24704|6|51|44|AM|third|morning|breakfast| +24705|AAAAAAAACIAGAAAA|24705|6|51|45|AM|third|morning|breakfast| +24706|AAAAAAAADIAGAAAA|24706|6|51|46|AM|third|morning|breakfast| +24707|AAAAAAAAEIAGAAAA|24707|6|51|47|AM|third|morning|breakfast| +24708|AAAAAAAAFIAGAAAA|24708|6|51|48|AM|third|morning|breakfast| +24709|AAAAAAAAGIAGAAAA|24709|6|51|49|AM|third|morning|breakfast| +24710|AAAAAAAAHIAGAAAA|24710|6|51|50|AM|third|morning|breakfast| +24711|AAAAAAAAIIAGAAAA|24711|6|51|51|AM|third|morning|breakfast| +24712|AAAAAAAAJIAGAAAA|24712|6|51|52|AM|third|morning|breakfast| +24713|AAAAAAAAKIAGAAAA|24713|6|51|53|AM|third|morning|breakfast| +24714|AAAAAAAALIAGAAAA|24714|6|51|54|AM|third|morning|breakfast| +24715|AAAAAAAAMIAGAAAA|24715|6|51|55|AM|third|morning|breakfast| +24716|AAAAAAAANIAGAAAA|24716|6|51|56|AM|third|morning|breakfast| +24717|AAAAAAAAOIAGAAAA|24717|6|51|57|AM|third|morning|breakfast| +24718|AAAAAAAAPIAGAAAA|24718|6|51|58|AM|third|morning|breakfast| +24719|AAAAAAAAAJAGAAAA|24719|6|51|59|AM|third|morning|breakfast| +24720|AAAAAAAABJAGAAAA|24720|6|52|0|AM|third|morning|breakfast| +24721|AAAAAAAACJAGAAAA|24721|6|52|1|AM|third|morning|breakfast| +24722|AAAAAAAADJAGAAAA|24722|6|52|2|AM|third|morning|breakfast| +24723|AAAAAAAAEJAGAAAA|24723|6|52|3|AM|third|morning|breakfast| +24724|AAAAAAAAFJAGAAAA|24724|6|52|4|AM|third|morning|breakfast| +24725|AAAAAAAAGJAGAAAA|24725|6|52|5|AM|third|morning|breakfast| +24726|AAAAAAAAHJAGAAAA|24726|6|52|6|AM|third|morning|breakfast| +24727|AAAAAAAAIJAGAAAA|24727|6|52|7|AM|third|morning|breakfast| +24728|AAAAAAAAJJAGAAAA|24728|6|52|8|AM|third|morning|breakfast| +24729|AAAAAAAAKJAGAAAA|24729|6|52|9|AM|third|morning|breakfast| +24730|AAAAAAAALJAGAAAA|24730|6|52|10|AM|third|morning|breakfast| +24731|AAAAAAAAMJAGAAAA|24731|6|52|11|AM|third|morning|breakfast| +24732|AAAAAAAANJAGAAAA|24732|6|52|12|AM|third|morning|breakfast| +24733|AAAAAAAAOJAGAAAA|24733|6|52|13|AM|third|morning|breakfast| +24734|AAAAAAAAPJAGAAAA|24734|6|52|14|AM|third|morning|breakfast| +24735|AAAAAAAAAKAGAAAA|24735|6|52|15|AM|third|morning|breakfast| +24736|AAAAAAAABKAGAAAA|24736|6|52|16|AM|third|morning|breakfast| +24737|AAAAAAAACKAGAAAA|24737|6|52|17|AM|third|morning|breakfast| +24738|AAAAAAAADKAGAAAA|24738|6|52|18|AM|third|morning|breakfast| +24739|AAAAAAAAEKAGAAAA|24739|6|52|19|AM|third|morning|breakfast| +24740|AAAAAAAAFKAGAAAA|24740|6|52|20|AM|third|morning|breakfast| +24741|AAAAAAAAGKAGAAAA|24741|6|52|21|AM|third|morning|breakfast| +24742|AAAAAAAAHKAGAAAA|24742|6|52|22|AM|third|morning|breakfast| +24743|AAAAAAAAIKAGAAAA|24743|6|52|23|AM|third|morning|breakfast| +24744|AAAAAAAAJKAGAAAA|24744|6|52|24|AM|third|morning|breakfast| +24745|AAAAAAAAKKAGAAAA|24745|6|52|25|AM|third|morning|breakfast| +24746|AAAAAAAALKAGAAAA|24746|6|52|26|AM|third|morning|breakfast| +24747|AAAAAAAAMKAGAAAA|24747|6|52|27|AM|third|morning|breakfast| +24748|AAAAAAAANKAGAAAA|24748|6|52|28|AM|third|morning|breakfast| +24749|AAAAAAAAOKAGAAAA|24749|6|52|29|AM|third|morning|breakfast| +24750|AAAAAAAAPKAGAAAA|24750|6|52|30|AM|third|morning|breakfast| +24751|AAAAAAAAALAGAAAA|24751|6|52|31|AM|third|morning|breakfast| +24752|AAAAAAAABLAGAAAA|24752|6|52|32|AM|third|morning|breakfast| +24753|AAAAAAAACLAGAAAA|24753|6|52|33|AM|third|morning|breakfast| +24754|AAAAAAAADLAGAAAA|24754|6|52|34|AM|third|morning|breakfast| +24755|AAAAAAAAELAGAAAA|24755|6|52|35|AM|third|morning|breakfast| +24756|AAAAAAAAFLAGAAAA|24756|6|52|36|AM|third|morning|breakfast| +24757|AAAAAAAAGLAGAAAA|24757|6|52|37|AM|third|morning|breakfast| +24758|AAAAAAAAHLAGAAAA|24758|6|52|38|AM|third|morning|breakfast| +24759|AAAAAAAAILAGAAAA|24759|6|52|39|AM|third|morning|breakfast| +24760|AAAAAAAAJLAGAAAA|24760|6|52|40|AM|third|morning|breakfast| +24761|AAAAAAAAKLAGAAAA|24761|6|52|41|AM|third|morning|breakfast| +24762|AAAAAAAALLAGAAAA|24762|6|52|42|AM|third|morning|breakfast| +24763|AAAAAAAAMLAGAAAA|24763|6|52|43|AM|third|morning|breakfast| +24764|AAAAAAAANLAGAAAA|24764|6|52|44|AM|third|morning|breakfast| +24765|AAAAAAAAOLAGAAAA|24765|6|52|45|AM|third|morning|breakfast| +24766|AAAAAAAAPLAGAAAA|24766|6|52|46|AM|third|morning|breakfast| +24767|AAAAAAAAAMAGAAAA|24767|6|52|47|AM|third|morning|breakfast| +24768|AAAAAAAABMAGAAAA|24768|6|52|48|AM|third|morning|breakfast| +24769|AAAAAAAACMAGAAAA|24769|6|52|49|AM|third|morning|breakfast| +24770|AAAAAAAADMAGAAAA|24770|6|52|50|AM|third|morning|breakfast| +24771|AAAAAAAAEMAGAAAA|24771|6|52|51|AM|third|morning|breakfast| +24772|AAAAAAAAFMAGAAAA|24772|6|52|52|AM|third|morning|breakfast| +24773|AAAAAAAAGMAGAAAA|24773|6|52|53|AM|third|morning|breakfast| +24774|AAAAAAAAHMAGAAAA|24774|6|52|54|AM|third|morning|breakfast| +24775|AAAAAAAAIMAGAAAA|24775|6|52|55|AM|third|morning|breakfast| +24776|AAAAAAAAJMAGAAAA|24776|6|52|56|AM|third|morning|breakfast| +24777|AAAAAAAAKMAGAAAA|24777|6|52|57|AM|third|morning|breakfast| +24778|AAAAAAAALMAGAAAA|24778|6|52|58|AM|third|morning|breakfast| +24779|AAAAAAAAMMAGAAAA|24779|6|52|59|AM|third|morning|breakfast| +24780|AAAAAAAANMAGAAAA|24780|6|53|0|AM|third|morning|breakfast| +24781|AAAAAAAAOMAGAAAA|24781|6|53|1|AM|third|morning|breakfast| +24782|AAAAAAAAPMAGAAAA|24782|6|53|2|AM|third|morning|breakfast| +24783|AAAAAAAAANAGAAAA|24783|6|53|3|AM|third|morning|breakfast| +24784|AAAAAAAABNAGAAAA|24784|6|53|4|AM|third|morning|breakfast| +24785|AAAAAAAACNAGAAAA|24785|6|53|5|AM|third|morning|breakfast| +24786|AAAAAAAADNAGAAAA|24786|6|53|6|AM|third|morning|breakfast| +24787|AAAAAAAAENAGAAAA|24787|6|53|7|AM|third|morning|breakfast| +24788|AAAAAAAAFNAGAAAA|24788|6|53|8|AM|third|morning|breakfast| +24789|AAAAAAAAGNAGAAAA|24789|6|53|9|AM|third|morning|breakfast| +24790|AAAAAAAAHNAGAAAA|24790|6|53|10|AM|third|morning|breakfast| +24791|AAAAAAAAINAGAAAA|24791|6|53|11|AM|third|morning|breakfast| +24792|AAAAAAAAJNAGAAAA|24792|6|53|12|AM|third|morning|breakfast| +24793|AAAAAAAAKNAGAAAA|24793|6|53|13|AM|third|morning|breakfast| +24794|AAAAAAAALNAGAAAA|24794|6|53|14|AM|third|morning|breakfast| +24795|AAAAAAAAMNAGAAAA|24795|6|53|15|AM|third|morning|breakfast| +24796|AAAAAAAANNAGAAAA|24796|6|53|16|AM|third|morning|breakfast| +24797|AAAAAAAAONAGAAAA|24797|6|53|17|AM|third|morning|breakfast| +24798|AAAAAAAAPNAGAAAA|24798|6|53|18|AM|third|morning|breakfast| +24799|AAAAAAAAAOAGAAAA|24799|6|53|19|AM|third|morning|breakfast| +24800|AAAAAAAABOAGAAAA|24800|6|53|20|AM|third|morning|breakfast| +24801|AAAAAAAACOAGAAAA|24801|6|53|21|AM|third|morning|breakfast| +24802|AAAAAAAADOAGAAAA|24802|6|53|22|AM|third|morning|breakfast| +24803|AAAAAAAAEOAGAAAA|24803|6|53|23|AM|third|morning|breakfast| +24804|AAAAAAAAFOAGAAAA|24804|6|53|24|AM|third|morning|breakfast| +24805|AAAAAAAAGOAGAAAA|24805|6|53|25|AM|third|morning|breakfast| +24806|AAAAAAAAHOAGAAAA|24806|6|53|26|AM|third|morning|breakfast| +24807|AAAAAAAAIOAGAAAA|24807|6|53|27|AM|third|morning|breakfast| +24808|AAAAAAAAJOAGAAAA|24808|6|53|28|AM|third|morning|breakfast| +24809|AAAAAAAAKOAGAAAA|24809|6|53|29|AM|third|morning|breakfast| +24810|AAAAAAAALOAGAAAA|24810|6|53|30|AM|third|morning|breakfast| +24811|AAAAAAAAMOAGAAAA|24811|6|53|31|AM|third|morning|breakfast| +24812|AAAAAAAANOAGAAAA|24812|6|53|32|AM|third|morning|breakfast| +24813|AAAAAAAAOOAGAAAA|24813|6|53|33|AM|third|morning|breakfast| +24814|AAAAAAAAPOAGAAAA|24814|6|53|34|AM|third|morning|breakfast| +24815|AAAAAAAAAPAGAAAA|24815|6|53|35|AM|third|morning|breakfast| +24816|AAAAAAAABPAGAAAA|24816|6|53|36|AM|third|morning|breakfast| +24817|AAAAAAAACPAGAAAA|24817|6|53|37|AM|third|morning|breakfast| +24818|AAAAAAAADPAGAAAA|24818|6|53|38|AM|third|morning|breakfast| +24819|AAAAAAAAEPAGAAAA|24819|6|53|39|AM|third|morning|breakfast| +24820|AAAAAAAAFPAGAAAA|24820|6|53|40|AM|third|morning|breakfast| +24821|AAAAAAAAGPAGAAAA|24821|6|53|41|AM|third|morning|breakfast| +24822|AAAAAAAAHPAGAAAA|24822|6|53|42|AM|third|morning|breakfast| +24823|AAAAAAAAIPAGAAAA|24823|6|53|43|AM|third|morning|breakfast| +24824|AAAAAAAAJPAGAAAA|24824|6|53|44|AM|third|morning|breakfast| +24825|AAAAAAAAKPAGAAAA|24825|6|53|45|AM|third|morning|breakfast| +24826|AAAAAAAALPAGAAAA|24826|6|53|46|AM|third|morning|breakfast| +24827|AAAAAAAAMPAGAAAA|24827|6|53|47|AM|third|morning|breakfast| +24828|AAAAAAAANPAGAAAA|24828|6|53|48|AM|third|morning|breakfast| +24829|AAAAAAAAOPAGAAAA|24829|6|53|49|AM|third|morning|breakfast| +24830|AAAAAAAAPPAGAAAA|24830|6|53|50|AM|third|morning|breakfast| +24831|AAAAAAAAAABGAAAA|24831|6|53|51|AM|third|morning|breakfast| +24832|AAAAAAAABABGAAAA|24832|6|53|52|AM|third|morning|breakfast| +24833|AAAAAAAACABGAAAA|24833|6|53|53|AM|third|morning|breakfast| +24834|AAAAAAAADABGAAAA|24834|6|53|54|AM|third|morning|breakfast| +24835|AAAAAAAAEABGAAAA|24835|6|53|55|AM|third|morning|breakfast| +24836|AAAAAAAAFABGAAAA|24836|6|53|56|AM|third|morning|breakfast| +24837|AAAAAAAAGABGAAAA|24837|6|53|57|AM|third|morning|breakfast| +24838|AAAAAAAAHABGAAAA|24838|6|53|58|AM|third|morning|breakfast| +24839|AAAAAAAAIABGAAAA|24839|6|53|59|AM|third|morning|breakfast| +24840|AAAAAAAAJABGAAAA|24840|6|54|0|AM|third|morning|breakfast| +24841|AAAAAAAAKABGAAAA|24841|6|54|1|AM|third|morning|breakfast| +24842|AAAAAAAALABGAAAA|24842|6|54|2|AM|third|morning|breakfast| +24843|AAAAAAAAMABGAAAA|24843|6|54|3|AM|third|morning|breakfast| +24844|AAAAAAAANABGAAAA|24844|6|54|4|AM|third|morning|breakfast| +24845|AAAAAAAAOABGAAAA|24845|6|54|5|AM|third|morning|breakfast| +24846|AAAAAAAAPABGAAAA|24846|6|54|6|AM|third|morning|breakfast| +24847|AAAAAAAAABBGAAAA|24847|6|54|7|AM|third|morning|breakfast| +24848|AAAAAAAABBBGAAAA|24848|6|54|8|AM|third|morning|breakfast| +24849|AAAAAAAACBBGAAAA|24849|6|54|9|AM|third|morning|breakfast| +24850|AAAAAAAADBBGAAAA|24850|6|54|10|AM|third|morning|breakfast| +24851|AAAAAAAAEBBGAAAA|24851|6|54|11|AM|third|morning|breakfast| +24852|AAAAAAAAFBBGAAAA|24852|6|54|12|AM|third|morning|breakfast| +24853|AAAAAAAAGBBGAAAA|24853|6|54|13|AM|third|morning|breakfast| +24854|AAAAAAAAHBBGAAAA|24854|6|54|14|AM|third|morning|breakfast| +24855|AAAAAAAAIBBGAAAA|24855|6|54|15|AM|third|morning|breakfast| +24856|AAAAAAAAJBBGAAAA|24856|6|54|16|AM|third|morning|breakfast| +24857|AAAAAAAAKBBGAAAA|24857|6|54|17|AM|third|morning|breakfast| +24858|AAAAAAAALBBGAAAA|24858|6|54|18|AM|third|morning|breakfast| +24859|AAAAAAAAMBBGAAAA|24859|6|54|19|AM|third|morning|breakfast| +24860|AAAAAAAANBBGAAAA|24860|6|54|20|AM|third|morning|breakfast| +24861|AAAAAAAAOBBGAAAA|24861|6|54|21|AM|third|morning|breakfast| +24862|AAAAAAAAPBBGAAAA|24862|6|54|22|AM|third|morning|breakfast| +24863|AAAAAAAAACBGAAAA|24863|6|54|23|AM|third|morning|breakfast| +24864|AAAAAAAABCBGAAAA|24864|6|54|24|AM|third|morning|breakfast| +24865|AAAAAAAACCBGAAAA|24865|6|54|25|AM|third|morning|breakfast| +24866|AAAAAAAADCBGAAAA|24866|6|54|26|AM|third|morning|breakfast| +24867|AAAAAAAAECBGAAAA|24867|6|54|27|AM|third|morning|breakfast| +24868|AAAAAAAAFCBGAAAA|24868|6|54|28|AM|third|morning|breakfast| +24869|AAAAAAAAGCBGAAAA|24869|6|54|29|AM|third|morning|breakfast| +24870|AAAAAAAAHCBGAAAA|24870|6|54|30|AM|third|morning|breakfast| +24871|AAAAAAAAICBGAAAA|24871|6|54|31|AM|third|morning|breakfast| +24872|AAAAAAAAJCBGAAAA|24872|6|54|32|AM|third|morning|breakfast| +24873|AAAAAAAAKCBGAAAA|24873|6|54|33|AM|third|morning|breakfast| +24874|AAAAAAAALCBGAAAA|24874|6|54|34|AM|third|morning|breakfast| +24875|AAAAAAAAMCBGAAAA|24875|6|54|35|AM|third|morning|breakfast| +24876|AAAAAAAANCBGAAAA|24876|6|54|36|AM|third|morning|breakfast| +24877|AAAAAAAAOCBGAAAA|24877|6|54|37|AM|third|morning|breakfast| +24878|AAAAAAAAPCBGAAAA|24878|6|54|38|AM|third|morning|breakfast| +24879|AAAAAAAAADBGAAAA|24879|6|54|39|AM|third|morning|breakfast| +24880|AAAAAAAABDBGAAAA|24880|6|54|40|AM|third|morning|breakfast| +24881|AAAAAAAACDBGAAAA|24881|6|54|41|AM|third|morning|breakfast| +24882|AAAAAAAADDBGAAAA|24882|6|54|42|AM|third|morning|breakfast| +24883|AAAAAAAAEDBGAAAA|24883|6|54|43|AM|third|morning|breakfast| +24884|AAAAAAAAFDBGAAAA|24884|6|54|44|AM|third|morning|breakfast| +24885|AAAAAAAAGDBGAAAA|24885|6|54|45|AM|third|morning|breakfast| +24886|AAAAAAAAHDBGAAAA|24886|6|54|46|AM|third|morning|breakfast| +24887|AAAAAAAAIDBGAAAA|24887|6|54|47|AM|third|morning|breakfast| +24888|AAAAAAAAJDBGAAAA|24888|6|54|48|AM|third|morning|breakfast| +24889|AAAAAAAAKDBGAAAA|24889|6|54|49|AM|third|morning|breakfast| +24890|AAAAAAAALDBGAAAA|24890|6|54|50|AM|third|morning|breakfast| +24891|AAAAAAAAMDBGAAAA|24891|6|54|51|AM|third|morning|breakfast| +24892|AAAAAAAANDBGAAAA|24892|6|54|52|AM|third|morning|breakfast| +24893|AAAAAAAAODBGAAAA|24893|6|54|53|AM|third|morning|breakfast| +24894|AAAAAAAAPDBGAAAA|24894|6|54|54|AM|third|morning|breakfast| +24895|AAAAAAAAAEBGAAAA|24895|6|54|55|AM|third|morning|breakfast| +24896|AAAAAAAABEBGAAAA|24896|6|54|56|AM|third|morning|breakfast| +24897|AAAAAAAACEBGAAAA|24897|6|54|57|AM|third|morning|breakfast| +24898|AAAAAAAADEBGAAAA|24898|6|54|58|AM|third|morning|breakfast| +24899|AAAAAAAAEEBGAAAA|24899|6|54|59|AM|third|morning|breakfast| +24900|AAAAAAAAFEBGAAAA|24900|6|55|0|AM|third|morning|breakfast| +24901|AAAAAAAAGEBGAAAA|24901|6|55|1|AM|third|morning|breakfast| +24902|AAAAAAAAHEBGAAAA|24902|6|55|2|AM|third|morning|breakfast| +24903|AAAAAAAAIEBGAAAA|24903|6|55|3|AM|third|morning|breakfast| +24904|AAAAAAAAJEBGAAAA|24904|6|55|4|AM|third|morning|breakfast| +24905|AAAAAAAAKEBGAAAA|24905|6|55|5|AM|third|morning|breakfast| +24906|AAAAAAAALEBGAAAA|24906|6|55|6|AM|third|morning|breakfast| +24907|AAAAAAAAMEBGAAAA|24907|6|55|7|AM|third|morning|breakfast| +24908|AAAAAAAANEBGAAAA|24908|6|55|8|AM|third|morning|breakfast| +24909|AAAAAAAAOEBGAAAA|24909|6|55|9|AM|third|morning|breakfast| +24910|AAAAAAAAPEBGAAAA|24910|6|55|10|AM|third|morning|breakfast| +24911|AAAAAAAAAFBGAAAA|24911|6|55|11|AM|third|morning|breakfast| +24912|AAAAAAAABFBGAAAA|24912|6|55|12|AM|third|morning|breakfast| +24913|AAAAAAAACFBGAAAA|24913|6|55|13|AM|third|morning|breakfast| +24914|AAAAAAAADFBGAAAA|24914|6|55|14|AM|third|morning|breakfast| +24915|AAAAAAAAEFBGAAAA|24915|6|55|15|AM|third|morning|breakfast| +24916|AAAAAAAAFFBGAAAA|24916|6|55|16|AM|third|morning|breakfast| +24917|AAAAAAAAGFBGAAAA|24917|6|55|17|AM|third|morning|breakfast| +24918|AAAAAAAAHFBGAAAA|24918|6|55|18|AM|third|morning|breakfast| +24919|AAAAAAAAIFBGAAAA|24919|6|55|19|AM|third|morning|breakfast| +24920|AAAAAAAAJFBGAAAA|24920|6|55|20|AM|third|morning|breakfast| +24921|AAAAAAAAKFBGAAAA|24921|6|55|21|AM|third|morning|breakfast| +24922|AAAAAAAALFBGAAAA|24922|6|55|22|AM|third|morning|breakfast| +24923|AAAAAAAAMFBGAAAA|24923|6|55|23|AM|third|morning|breakfast| +24924|AAAAAAAANFBGAAAA|24924|6|55|24|AM|third|morning|breakfast| +24925|AAAAAAAAOFBGAAAA|24925|6|55|25|AM|third|morning|breakfast| +24926|AAAAAAAAPFBGAAAA|24926|6|55|26|AM|third|morning|breakfast| +24927|AAAAAAAAAGBGAAAA|24927|6|55|27|AM|third|morning|breakfast| +24928|AAAAAAAABGBGAAAA|24928|6|55|28|AM|third|morning|breakfast| +24929|AAAAAAAACGBGAAAA|24929|6|55|29|AM|third|morning|breakfast| +24930|AAAAAAAADGBGAAAA|24930|6|55|30|AM|third|morning|breakfast| +24931|AAAAAAAAEGBGAAAA|24931|6|55|31|AM|third|morning|breakfast| +24932|AAAAAAAAFGBGAAAA|24932|6|55|32|AM|third|morning|breakfast| +24933|AAAAAAAAGGBGAAAA|24933|6|55|33|AM|third|morning|breakfast| +24934|AAAAAAAAHGBGAAAA|24934|6|55|34|AM|third|morning|breakfast| +24935|AAAAAAAAIGBGAAAA|24935|6|55|35|AM|third|morning|breakfast| +24936|AAAAAAAAJGBGAAAA|24936|6|55|36|AM|third|morning|breakfast| +24937|AAAAAAAAKGBGAAAA|24937|6|55|37|AM|third|morning|breakfast| +24938|AAAAAAAALGBGAAAA|24938|6|55|38|AM|third|morning|breakfast| +24939|AAAAAAAAMGBGAAAA|24939|6|55|39|AM|third|morning|breakfast| +24940|AAAAAAAANGBGAAAA|24940|6|55|40|AM|third|morning|breakfast| +24941|AAAAAAAAOGBGAAAA|24941|6|55|41|AM|third|morning|breakfast| +24942|AAAAAAAAPGBGAAAA|24942|6|55|42|AM|third|morning|breakfast| +24943|AAAAAAAAAHBGAAAA|24943|6|55|43|AM|third|morning|breakfast| +24944|AAAAAAAABHBGAAAA|24944|6|55|44|AM|third|morning|breakfast| +24945|AAAAAAAACHBGAAAA|24945|6|55|45|AM|third|morning|breakfast| +24946|AAAAAAAADHBGAAAA|24946|6|55|46|AM|third|morning|breakfast| +24947|AAAAAAAAEHBGAAAA|24947|6|55|47|AM|third|morning|breakfast| +24948|AAAAAAAAFHBGAAAA|24948|6|55|48|AM|third|morning|breakfast| +24949|AAAAAAAAGHBGAAAA|24949|6|55|49|AM|third|morning|breakfast| +24950|AAAAAAAAHHBGAAAA|24950|6|55|50|AM|third|morning|breakfast| +24951|AAAAAAAAIHBGAAAA|24951|6|55|51|AM|third|morning|breakfast| +24952|AAAAAAAAJHBGAAAA|24952|6|55|52|AM|third|morning|breakfast| +24953|AAAAAAAAKHBGAAAA|24953|6|55|53|AM|third|morning|breakfast| +24954|AAAAAAAALHBGAAAA|24954|6|55|54|AM|third|morning|breakfast| +24955|AAAAAAAAMHBGAAAA|24955|6|55|55|AM|third|morning|breakfast| +24956|AAAAAAAANHBGAAAA|24956|6|55|56|AM|third|morning|breakfast| +24957|AAAAAAAAOHBGAAAA|24957|6|55|57|AM|third|morning|breakfast| +24958|AAAAAAAAPHBGAAAA|24958|6|55|58|AM|third|morning|breakfast| +24959|AAAAAAAAAIBGAAAA|24959|6|55|59|AM|third|morning|breakfast| +24960|AAAAAAAABIBGAAAA|24960|6|56|0|AM|third|morning|breakfast| +24961|AAAAAAAACIBGAAAA|24961|6|56|1|AM|third|morning|breakfast| +24962|AAAAAAAADIBGAAAA|24962|6|56|2|AM|third|morning|breakfast| +24963|AAAAAAAAEIBGAAAA|24963|6|56|3|AM|third|morning|breakfast| +24964|AAAAAAAAFIBGAAAA|24964|6|56|4|AM|third|morning|breakfast| +24965|AAAAAAAAGIBGAAAA|24965|6|56|5|AM|third|morning|breakfast| +24966|AAAAAAAAHIBGAAAA|24966|6|56|6|AM|third|morning|breakfast| +24967|AAAAAAAAIIBGAAAA|24967|6|56|7|AM|third|morning|breakfast| +24968|AAAAAAAAJIBGAAAA|24968|6|56|8|AM|third|morning|breakfast| +24969|AAAAAAAAKIBGAAAA|24969|6|56|9|AM|third|morning|breakfast| +24970|AAAAAAAALIBGAAAA|24970|6|56|10|AM|third|morning|breakfast| +24971|AAAAAAAAMIBGAAAA|24971|6|56|11|AM|third|morning|breakfast| +24972|AAAAAAAANIBGAAAA|24972|6|56|12|AM|third|morning|breakfast| +24973|AAAAAAAAOIBGAAAA|24973|6|56|13|AM|third|morning|breakfast| +24974|AAAAAAAAPIBGAAAA|24974|6|56|14|AM|third|morning|breakfast| +24975|AAAAAAAAAJBGAAAA|24975|6|56|15|AM|third|morning|breakfast| +24976|AAAAAAAABJBGAAAA|24976|6|56|16|AM|third|morning|breakfast| +24977|AAAAAAAACJBGAAAA|24977|6|56|17|AM|third|morning|breakfast| +24978|AAAAAAAADJBGAAAA|24978|6|56|18|AM|third|morning|breakfast| +24979|AAAAAAAAEJBGAAAA|24979|6|56|19|AM|third|morning|breakfast| +24980|AAAAAAAAFJBGAAAA|24980|6|56|20|AM|third|morning|breakfast| +24981|AAAAAAAAGJBGAAAA|24981|6|56|21|AM|third|morning|breakfast| +24982|AAAAAAAAHJBGAAAA|24982|6|56|22|AM|third|morning|breakfast| +24983|AAAAAAAAIJBGAAAA|24983|6|56|23|AM|third|morning|breakfast| +24984|AAAAAAAAJJBGAAAA|24984|6|56|24|AM|third|morning|breakfast| +24985|AAAAAAAAKJBGAAAA|24985|6|56|25|AM|third|morning|breakfast| +24986|AAAAAAAALJBGAAAA|24986|6|56|26|AM|third|morning|breakfast| +24987|AAAAAAAAMJBGAAAA|24987|6|56|27|AM|third|morning|breakfast| +24988|AAAAAAAANJBGAAAA|24988|6|56|28|AM|third|morning|breakfast| +24989|AAAAAAAAOJBGAAAA|24989|6|56|29|AM|third|morning|breakfast| +24990|AAAAAAAAPJBGAAAA|24990|6|56|30|AM|third|morning|breakfast| +24991|AAAAAAAAAKBGAAAA|24991|6|56|31|AM|third|morning|breakfast| +24992|AAAAAAAABKBGAAAA|24992|6|56|32|AM|third|morning|breakfast| +24993|AAAAAAAACKBGAAAA|24993|6|56|33|AM|third|morning|breakfast| +24994|AAAAAAAADKBGAAAA|24994|6|56|34|AM|third|morning|breakfast| +24995|AAAAAAAAEKBGAAAA|24995|6|56|35|AM|third|morning|breakfast| +24996|AAAAAAAAFKBGAAAA|24996|6|56|36|AM|third|morning|breakfast| +24997|AAAAAAAAGKBGAAAA|24997|6|56|37|AM|third|morning|breakfast| +24998|AAAAAAAAHKBGAAAA|24998|6|56|38|AM|third|morning|breakfast| +24999|AAAAAAAAIKBGAAAA|24999|6|56|39|AM|third|morning|breakfast| +25000|AAAAAAAAJKBGAAAA|25000|6|56|40|AM|third|morning|breakfast| +25001|AAAAAAAAKKBGAAAA|25001|6|56|41|AM|third|morning|breakfast| +25002|AAAAAAAALKBGAAAA|25002|6|56|42|AM|third|morning|breakfast| +25003|AAAAAAAAMKBGAAAA|25003|6|56|43|AM|third|morning|breakfast| +25004|AAAAAAAANKBGAAAA|25004|6|56|44|AM|third|morning|breakfast| +25005|AAAAAAAAOKBGAAAA|25005|6|56|45|AM|third|morning|breakfast| +25006|AAAAAAAAPKBGAAAA|25006|6|56|46|AM|third|morning|breakfast| +25007|AAAAAAAAALBGAAAA|25007|6|56|47|AM|third|morning|breakfast| +25008|AAAAAAAABLBGAAAA|25008|6|56|48|AM|third|morning|breakfast| +25009|AAAAAAAACLBGAAAA|25009|6|56|49|AM|third|morning|breakfast| +25010|AAAAAAAADLBGAAAA|25010|6|56|50|AM|third|morning|breakfast| +25011|AAAAAAAAELBGAAAA|25011|6|56|51|AM|third|morning|breakfast| +25012|AAAAAAAAFLBGAAAA|25012|6|56|52|AM|third|morning|breakfast| +25013|AAAAAAAAGLBGAAAA|25013|6|56|53|AM|third|morning|breakfast| +25014|AAAAAAAAHLBGAAAA|25014|6|56|54|AM|third|morning|breakfast| +25015|AAAAAAAAILBGAAAA|25015|6|56|55|AM|third|morning|breakfast| +25016|AAAAAAAAJLBGAAAA|25016|6|56|56|AM|third|morning|breakfast| +25017|AAAAAAAAKLBGAAAA|25017|6|56|57|AM|third|morning|breakfast| +25018|AAAAAAAALLBGAAAA|25018|6|56|58|AM|third|morning|breakfast| +25019|AAAAAAAAMLBGAAAA|25019|6|56|59|AM|third|morning|breakfast| +25020|AAAAAAAANLBGAAAA|25020|6|57|0|AM|third|morning|breakfast| +25021|AAAAAAAAOLBGAAAA|25021|6|57|1|AM|third|morning|breakfast| +25022|AAAAAAAAPLBGAAAA|25022|6|57|2|AM|third|morning|breakfast| +25023|AAAAAAAAAMBGAAAA|25023|6|57|3|AM|third|morning|breakfast| +25024|AAAAAAAABMBGAAAA|25024|6|57|4|AM|third|morning|breakfast| +25025|AAAAAAAACMBGAAAA|25025|6|57|5|AM|third|morning|breakfast| +25026|AAAAAAAADMBGAAAA|25026|6|57|6|AM|third|morning|breakfast| +25027|AAAAAAAAEMBGAAAA|25027|6|57|7|AM|third|morning|breakfast| +25028|AAAAAAAAFMBGAAAA|25028|6|57|8|AM|third|morning|breakfast| +25029|AAAAAAAAGMBGAAAA|25029|6|57|9|AM|third|morning|breakfast| +25030|AAAAAAAAHMBGAAAA|25030|6|57|10|AM|third|morning|breakfast| +25031|AAAAAAAAIMBGAAAA|25031|6|57|11|AM|third|morning|breakfast| +25032|AAAAAAAAJMBGAAAA|25032|6|57|12|AM|third|morning|breakfast| +25033|AAAAAAAAKMBGAAAA|25033|6|57|13|AM|third|morning|breakfast| +25034|AAAAAAAALMBGAAAA|25034|6|57|14|AM|third|morning|breakfast| +25035|AAAAAAAAMMBGAAAA|25035|6|57|15|AM|third|morning|breakfast| +25036|AAAAAAAANMBGAAAA|25036|6|57|16|AM|third|morning|breakfast| +25037|AAAAAAAAOMBGAAAA|25037|6|57|17|AM|third|morning|breakfast| +25038|AAAAAAAAPMBGAAAA|25038|6|57|18|AM|third|morning|breakfast| +25039|AAAAAAAAANBGAAAA|25039|6|57|19|AM|third|morning|breakfast| +25040|AAAAAAAABNBGAAAA|25040|6|57|20|AM|third|morning|breakfast| +25041|AAAAAAAACNBGAAAA|25041|6|57|21|AM|third|morning|breakfast| +25042|AAAAAAAADNBGAAAA|25042|6|57|22|AM|third|morning|breakfast| +25043|AAAAAAAAENBGAAAA|25043|6|57|23|AM|third|morning|breakfast| +25044|AAAAAAAAFNBGAAAA|25044|6|57|24|AM|third|morning|breakfast| +25045|AAAAAAAAGNBGAAAA|25045|6|57|25|AM|third|morning|breakfast| +25046|AAAAAAAAHNBGAAAA|25046|6|57|26|AM|third|morning|breakfast| +25047|AAAAAAAAINBGAAAA|25047|6|57|27|AM|third|morning|breakfast| +25048|AAAAAAAAJNBGAAAA|25048|6|57|28|AM|third|morning|breakfast| +25049|AAAAAAAAKNBGAAAA|25049|6|57|29|AM|third|morning|breakfast| +25050|AAAAAAAALNBGAAAA|25050|6|57|30|AM|third|morning|breakfast| +25051|AAAAAAAAMNBGAAAA|25051|6|57|31|AM|third|morning|breakfast| +25052|AAAAAAAANNBGAAAA|25052|6|57|32|AM|third|morning|breakfast| +25053|AAAAAAAAONBGAAAA|25053|6|57|33|AM|third|morning|breakfast| +25054|AAAAAAAAPNBGAAAA|25054|6|57|34|AM|third|morning|breakfast| +25055|AAAAAAAAAOBGAAAA|25055|6|57|35|AM|third|morning|breakfast| +25056|AAAAAAAABOBGAAAA|25056|6|57|36|AM|third|morning|breakfast| +25057|AAAAAAAACOBGAAAA|25057|6|57|37|AM|third|morning|breakfast| +25058|AAAAAAAADOBGAAAA|25058|6|57|38|AM|third|morning|breakfast| +25059|AAAAAAAAEOBGAAAA|25059|6|57|39|AM|third|morning|breakfast| +25060|AAAAAAAAFOBGAAAA|25060|6|57|40|AM|third|morning|breakfast| +25061|AAAAAAAAGOBGAAAA|25061|6|57|41|AM|third|morning|breakfast| +25062|AAAAAAAAHOBGAAAA|25062|6|57|42|AM|third|morning|breakfast| +25063|AAAAAAAAIOBGAAAA|25063|6|57|43|AM|third|morning|breakfast| +25064|AAAAAAAAJOBGAAAA|25064|6|57|44|AM|third|morning|breakfast| +25065|AAAAAAAAKOBGAAAA|25065|6|57|45|AM|third|morning|breakfast| +25066|AAAAAAAALOBGAAAA|25066|6|57|46|AM|third|morning|breakfast| +25067|AAAAAAAAMOBGAAAA|25067|6|57|47|AM|third|morning|breakfast| +25068|AAAAAAAANOBGAAAA|25068|6|57|48|AM|third|morning|breakfast| +25069|AAAAAAAAOOBGAAAA|25069|6|57|49|AM|third|morning|breakfast| +25070|AAAAAAAAPOBGAAAA|25070|6|57|50|AM|third|morning|breakfast| +25071|AAAAAAAAAPBGAAAA|25071|6|57|51|AM|third|morning|breakfast| +25072|AAAAAAAABPBGAAAA|25072|6|57|52|AM|third|morning|breakfast| +25073|AAAAAAAACPBGAAAA|25073|6|57|53|AM|third|morning|breakfast| +25074|AAAAAAAADPBGAAAA|25074|6|57|54|AM|third|morning|breakfast| +25075|AAAAAAAAEPBGAAAA|25075|6|57|55|AM|third|morning|breakfast| +25076|AAAAAAAAFPBGAAAA|25076|6|57|56|AM|third|morning|breakfast| +25077|AAAAAAAAGPBGAAAA|25077|6|57|57|AM|third|morning|breakfast| +25078|AAAAAAAAHPBGAAAA|25078|6|57|58|AM|third|morning|breakfast| +25079|AAAAAAAAIPBGAAAA|25079|6|57|59|AM|third|morning|breakfast| +25080|AAAAAAAAJPBGAAAA|25080|6|58|0|AM|third|morning|breakfast| +25081|AAAAAAAAKPBGAAAA|25081|6|58|1|AM|third|morning|breakfast| +25082|AAAAAAAALPBGAAAA|25082|6|58|2|AM|third|morning|breakfast| +25083|AAAAAAAAMPBGAAAA|25083|6|58|3|AM|third|morning|breakfast| +25084|AAAAAAAANPBGAAAA|25084|6|58|4|AM|third|morning|breakfast| +25085|AAAAAAAAOPBGAAAA|25085|6|58|5|AM|third|morning|breakfast| +25086|AAAAAAAAPPBGAAAA|25086|6|58|6|AM|third|morning|breakfast| +25087|AAAAAAAAAACGAAAA|25087|6|58|7|AM|third|morning|breakfast| +25088|AAAAAAAABACGAAAA|25088|6|58|8|AM|third|morning|breakfast| +25089|AAAAAAAACACGAAAA|25089|6|58|9|AM|third|morning|breakfast| +25090|AAAAAAAADACGAAAA|25090|6|58|10|AM|third|morning|breakfast| +25091|AAAAAAAAEACGAAAA|25091|6|58|11|AM|third|morning|breakfast| +25092|AAAAAAAAFACGAAAA|25092|6|58|12|AM|third|morning|breakfast| +25093|AAAAAAAAGACGAAAA|25093|6|58|13|AM|third|morning|breakfast| +25094|AAAAAAAAHACGAAAA|25094|6|58|14|AM|third|morning|breakfast| +25095|AAAAAAAAIACGAAAA|25095|6|58|15|AM|third|morning|breakfast| +25096|AAAAAAAAJACGAAAA|25096|6|58|16|AM|third|morning|breakfast| +25097|AAAAAAAAKACGAAAA|25097|6|58|17|AM|third|morning|breakfast| +25098|AAAAAAAALACGAAAA|25098|6|58|18|AM|third|morning|breakfast| +25099|AAAAAAAAMACGAAAA|25099|6|58|19|AM|third|morning|breakfast| +25100|AAAAAAAANACGAAAA|25100|6|58|20|AM|third|morning|breakfast| +25101|AAAAAAAAOACGAAAA|25101|6|58|21|AM|third|morning|breakfast| +25102|AAAAAAAAPACGAAAA|25102|6|58|22|AM|third|morning|breakfast| +25103|AAAAAAAAABCGAAAA|25103|6|58|23|AM|third|morning|breakfast| +25104|AAAAAAAABBCGAAAA|25104|6|58|24|AM|third|morning|breakfast| +25105|AAAAAAAACBCGAAAA|25105|6|58|25|AM|third|morning|breakfast| +25106|AAAAAAAADBCGAAAA|25106|6|58|26|AM|third|morning|breakfast| +25107|AAAAAAAAEBCGAAAA|25107|6|58|27|AM|third|morning|breakfast| +25108|AAAAAAAAFBCGAAAA|25108|6|58|28|AM|third|morning|breakfast| +25109|AAAAAAAAGBCGAAAA|25109|6|58|29|AM|third|morning|breakfast| +25110|AAAAAAAAHBCGAAAA|25110|6|58|30|AM|third|morning|breakfast| +25111|AAAAAAAAIBCGAAAA|25111|6|58|31|AM|third|morning|breakfast| +25112|AAAAAAAAJBCGAAAA|25112|6|58|32|AM|third|morning|breakfast| +25113|AAAAAAAAKBCGAAAA|25113|6|58|33|AM|third|morning|breakfast| +25114|AAAAAAAALBCGAAAA|25114|6|58|34|AM|third|morning|breakfast| +25115|AAAAAAAAMBCGAAAA|25115|6|58|35|AM|third|morning|breakfast| +25116|AAAAAAAANBCGAAAA|25116|6|58|36|AM|third|morning|breakfast| +25117|AAAAAAAAOBCGAAAA|25117|6|58|37|AM|third|morning|breakfast| +25118|AAAAAAAAPBCGAAAA|25118|6|58|38|AM|third|morning|breakfast| +25119|AAAAAAAAACCGAAAA|25119|6|58|39|AM|third|morning|breakfast| +25120|AAAAAAAABCCGAAAA|25120|6|58|40|AM|third|morning|breakfast| +25121|AAAAAAAACCCGAAAA|25121|6|58|41|AM|third|morning|breakfast| +25122|AAAAAAAADCCGAAAA|25122|6|58|42|AM|third|morning|breakfast| +25123|AAAAAAAAECCGAAAA|25123|6|58|43|AM|third|morning|breakfast| +25124|AAAAAAAAFCCGAAAA|25124|6|58|44|AM|third|morning|breakfast| +25125|AAAAAAAAGCCGAAAA|25125|6|58|45|AM|third|morning|breakfast| +25126|AAAAAAAAHCCGAAAA|25126|6|58|46|AM|third|morning|breakfast| +25127|AAAAAAAAICCGAAAA|25127|6|58|47|AM|third|morning|breakfast| +25128|AAAAAAAAJCCGAAAA|25128|6|58|48|AM|third|morning|breakfast| +25129|AAAAAAAAKCCGAAAA|25129|6|58|49|AM|third|morning|breakfast| +25130|AAAAAAAALCCGAAAA|25130|6|58|50|AM|third|morning|breakfast| +25131|AAAAAAAAMCCGAAAA|25131|6|58|51|AM|third|morning|breakfast| +25132|AAAAAAAANCCGAAAA|25132|6|58|52|AM|third|morning|breakfast| +25133|AAAAAAAAOCCGAAAA|25133|6|58|53|AM|third|morning|breakfast| +25134|AAAAAAAAPCCGAAAA|25134|6|58|54|AM|third|morning|breakfast| +25135|AAAAAAAAADCGAAAA|25135|6|58|55|AM|third|morning|breakfast| +25136|AAAAAAAABDCGAAAA|25136|6|58|56|AM|third|morning|breakfast| +25137|AAAAAAAACDCGAAAA|25137|6|58|57|AM|third|morning|breakfast| +25138|AAAAAAAADDCGAAAA|25138|6|58|58|AM|third|morning|breakfast| +25139|AAAAAAAAEDCGAAAA|25139|6|58|59|AM|third|morning|breakfast| +25140|AAAAAAAAFDCGAAAA|25140|6|59|0|AM|third|morning|breakfast| +25141|AAAAAAAAGDCGAAAA|25141|6|59|1|AM|third|morning|breakfast| +25142|AAAAAAAAHDCGAAAA|25142|6|59|2|AM|third|morning|breakfast| +25143|AAAAAAAAIDCGAAAA|25143|6|59|3|AM|third|morning|breakfast| +25144|AAAAAAAAJDCGAAAA|25144|6|59|4|AM|third|morning|breakfast| +25145|AAAAAAAAKDCGAAAA|25145|6|59|5|AM|third|morning|breakfast| +25146|AAAAAAAALDCGAAAA|25146|6|59|6|AM|third|morning|breakfast| +25147|AAAAAAAAMDCGAAAA|25147|6|59|7|AM|third|morning|breakfast| +25148|AAAAAAAANDCGAAAA|25148|6|59|8|AM|third|morning|breakfast| +25149|AAAAAAAAODCGAAAA|25149|6|59|9|AM|third|morning|breakfast| +25150|AAAAAAAAPDCGAAAA|25150|6|59|10|AM|third|morning|breakfast| +25151|AAAAAAAAAECGAAAA|25151|6|59|11|AM|third|morning|breakfast| +25152|AAAAAAAABECGAAAA|25152|6|59|12|AM|third|morning|breakfast| +25153|AAAAAAAACECGAAAA|25153|6|59|13|AM|third|morning|breakfast| +25154|AAAAAAAADECGAAAA|25154|6|59|14|AM|third|morning|breakfast| +25155|AAAAAAAAEECGAAAA|25155|6|59|15|AM|third|morning|breakfast| +25156|AAAAAAAAFECGAAAA|25156|6|59|16|AM|third|morning|breakfast| +25157|AAAAAAAAGECGAAAA|25157|6|59|17|AM|third|morning|breakfast| +25158|AAAAAAAAHECGAAAA|25158|6|59|18|AM|third|morning|breakfast| +25159|AAAAAAAAIECGAAAA|25159|6|59|19|AM|third|morning|breakfast| +25160|AAAAAAAAJECGAAAA|25160|6|59|20|AM|third|morning|breakfast| +25161|AAAAAAAAKECGAAAA|25161|6|59|21|AM|third|morning|breakfast| +25162|AAAAAAAALECGAAAA|25162|6|59|22|AM|third|morning|breakfast| +25163|AAAAAAAAMECGAAAA|25163|6|59|23|AM|third|morning|breakfast| +25164|AAAAAAAANECGAAAA|25164|6|59|24|AM|third|morning|breakfast| +25165|AAAAAAAAOECGAAAA|25165|6|59|25|AM|third|morning|breakfast| +25166|AAAAAAAAPECGAAAA|25166|6|59|26|AM|third|morning|breakfast| +25167|AAAAAAAAAFCGAAAA|25167|6|59|27|AM|third|morning|breakfast| +25168|AAAAAAAABFCGAAAA|25168|6|59|28|AM|third|morning|breakfast| +25169|AAAAAAAACFCGAAAA|25169|6|59|29|AM|third|morning|breakfast| +25170|AAAAAAAADFCGAAAA|25170|6|59|30|AM|third|morning|breakfast| +25171|AAAAAAAAEFCGAAAA|25171|6|59|31|AM|third|morning|breakfast| +25172|AAAAAAAAFFCGAAAA|25172|6|59|32|AM|third|morning|breakfast| +25173|AAAAAAAAGFCGAAAA|25173|6|59|33|AM|third|morning|breakfast| +25174|AAAAAAAAHFCGAAAA|25174|6|59|34|AM|third|morning|breakfast| +25175|AAAAAAAAIFCGAAAA|25175|6|59|35|AM|third|morning|breakfast| +25176|AAAAAAAAJFCGAAAA|25176|6|59|36|AM|third|morning|breakfast| +25177|AAAAAAAAKFCGAAAA|25177|6|59|37|AM|third|morning|breakfast| +25178|AAAAAAAALFCGAAAA|25178|6|59|38|AM|third|morning|breakfast| +25179|AAAAAAAAMFCGAAAA|25179|6|59|39|AM|third|morning|breakfast| +25180|AAAAAAAANFCGAAAA|25180|6|59|40|AM|third|morning|breakfast| +25181|AAAAAAAAOFCGAAAA|25181|6|59|41|AM|third|morning|breakfast| +25182|AAAAAAAAPFCGAAAA|25182|6|59|42|AM|third|morning|breakfast| +25183|AAAAAAAAAGCGAAAA|25183|6|59|43|AM|third|morning|breakfast| +25184|AAAAAAAABGCGAAAA|25184|6|59|44|AM|third|morning|breakfast| +25185|AAAAAAAACGCGAAAA|25185|6|59|45|AM|third|morning|breakfast| +25186|AAAAAAAADGCGAAAA|25186|6|59|46|AM|third|morning|breakfast| +25187|AAAAAAAAEGCGAAAA|25187|6|59|47|AM|third|morning|breakfast| +25188|AAAAAAAAFGCGAAAA|25188|6|59|48|AM|third|morning|breakfast| +25189|AAAAAAAAGGCGAAAA|25189|6|59|49|AM|third|morning|breakfast| +25190|AAAAAAAAHGCGAAAA|25190|6|59|50|AM|third|morning|breakfast| +25191|AAAAAAAAIGCGAAAA|25191|6|59|51|AM|third|morning|breakfast| +25192|AAAAAAAAJGCGAAAA|25192|6|59|52|AM|third|morning|breakfast| +25193|AAAAAAAAKGCGAAAA|25193|6|59|53|AM|third|morning|breakfast| +25194|AAAAAAAALGCGAAAA|25194|6|59|54|AM|third|morning|breakfast| +25195|AAAAAAAAMGCGAAAA|25195|6|59|55|AM|third|morning|breakfast| +25196|AAAAAAAANGCGAAAA|25196|6|59|56|AM|third|morning|breakfast| +25197|AAAAAAAAOGCGAAAA|25197|6|59|57|AM|third|morning|breakfast| +25198|AAAAAAAAPGCGAAAA|25198|6|59|58|AM|third|morning|breakfast| +25199|AAAAAAAAAHCGAAAA|25199|6|59|59|AM|third|morning|breakfast| +25200|AAAAAAAABHCGAAAA|25200|7|0|0|AM|first|morning|breakfast| +25201|AAAAAAAACHCGAAAA|25201|7|0|1|AM|first|morning|breakfast| +25202|AAAAAAAADHCGAAAA|25202|7|0|2|AM|first|morning|breakfast| +25203|AAAAAAAAEHCGAAAA|25203|7|0|3|AM|first|morning|breakfast| +25204|AAAAAAAAFHCGAAAA|25204|7|0|4|AM|first|morning|breakfast| +25205|AAAAAAAAGHCGAAAA|25205|7|0|5|AM|first|morning|breakfast| +25206|AAAAAAAAHHCGAAAA|25206|7|0|6|AM|first|morning|breakfast| +25207|AAAAAAAAIHCGAAAA|25207|7|0|7|AM|first|morning|breakfast| +25208|AAAAAAAAJHCGAAAA|25208|7|0|8|AM|first|morning|breakfast| +25209|AAAAAAAAKHCGAAAA|25209|7|0|9|AM|first|morning|breakfast| +25210|AAAAAAAALHCGAAAA|25210|7|0|10|AM|first|morning|breakfast| +25211|AAAAAAAAMHCGAAAA|25211|7|0|11|AM|first|morning|breakfast| +25212|AAAAAAAANHCGAAAA|25212|7|0|12|AM|first|morning|breakfast| +25213|AAAAAAAAOHCGAAAA|25213|7|0|13|AM|first|morning|breakfast| +25214|AAAAAAAAPHCGAAAA|25214|7|0|14|AM|first|morning|breakfast| +25215|AAAAAAAAAICGAAAA|25215|7|0|15|AM|first|morning|breakfast| +25216|AAAAAAAABICGAAAA|25216|7|0|16|AM|first|morning|breakfast| +25217|AAAAAAAACICGAAAA|25217|7|0|17|AM|first|morning|breakfast| +25218|AAAAAAAADICGAAAA|25218|7|0|18|AM|first|morning|breakfast| +25219|AAAAAAAAEICGAAAA|25219|7|0|19|AM|first|morning|breakfast| +25220|AAAAAAAAFICGAAAA|25220|7|0|20|AM|first|morning|breakfast| +25221|AAAAAAAAGICGAAAA|25221|7|0|21|AM|first|morning|breakfast| +25222|AAAAAAAAHICGAAAA|25222|7|0|22|AM|first|morning|breakfast| +25223|AAAAAAAAIICGAAAA|25223|7|0|23|AM|first|morning|breakfast| +25224|AAAAAAAAJICGAAAA|25224|7|0|24|AM|first|morning|breakfast| +25225|AAAAAAAAKICGAAAA|25225|7|0|25|AM|first|morning|breakfast| +25226|AAAAAAAALICGAAAA|25226|7|0|26|AM|first|morning|breakfast| +25227|AAAAAAAAMICGAAAA|25227|7|0|27|AM|first|morning|breakfast| +25228|AAAAAAAANICGAAAA|25228|7|0|28|AM|first|morning|breakfast| +25229|AAAAAAAAOICGAAAA|25229|7|0|29|AM|first|morning|breakfast| +25230|AAAAAAAAPICGAAAA|25230|7|0|30|AM|first|morning|breakfast| +25231|AAAAAAAAAJCGAAAA|25231|7|0|31|AM|first|morning|breakfast| +25232|AAAAAAAABJCGAAAA|25232|7|0|32|AM|first|morning|breakfast| +25233|AAAAAAAACJCGAAAA|25233|7|0|33|AM|first|morning|breakfast| +25234|AAAAAAAADJCGAAAA|25234|7|0|34|AM|first|morning|breakfast| +25235|AAAAAAAAEJCGAAAA|25235|7|0|35|AM|first|morning|breakfast| +25236|AAAAAAAAFJCGAAAA|25236|7|0|36|AM|first|morning|breakfast| +25237|AAAAAAAAGJCGAAAA|25237|7|0|37|AM|first|morning|breakfast| +25238|AAAAAAAAHJCGAAAA|25238|7|0|38|AM|first|morning|breakfast| +25239|AAAAAAAAIJCGAAAA|25239|7|0|39|AM|first|morning|breakfast| +25240|AAAAAAAAJJCGAAAA|25240|7|0|40|AM|first|morning|breakfast| +25241|AAAAAAAAKJCGAAAA|25241|7|0|41|AM|first|morning|breakfast| +25242|AAAAAAAALJCGAAAA|25242|7|0|42|AM|first|morning|breakfast| +25243|AAAAAAAAMJCGAAAA|25243|7|0|43|AM|first|morning|breakfast| +25244|AAAAAAAANJCGAAAA|25244|7|0|44|AM|first|morning|breakfast| +25245|AAAAAAAAOJCGAAAA|25245|7|0|45|AM|first|morning|breakfast| +25246|AAAAAAAAPJCGAAAA|25246|7|0|46|AM|first|morning|breakfast| +25247|AAAAAAAAAKCGAAAA|25247|7|0|47|AM|first|morning|breakfast| +25248|AAAAAAAABKCGAAAA|25248|7|0|48|AM|first|morning|breakfast| +25249|AAAAAAAACKCGAAAA|25249|7|0|49|AM|first|morning|breakfast| +25250|AAAAAAAADKCGAAAA|25250|7|0|50|AM|first|morning|breakfast| +25251|AAAAAAAAEKCGAAAA|25251|7|0|51|AM|first|morning|breakfast| +25252|AAAAAAAAFKCGAAAA|25252|7|0|52|AM|first|morning|breakfast| +25253|AAAAAAAAGKCGAAAA|25253|7|0|53|AM|first|morning|breakfast| +25254|AAAAAAAAHKCGAAAA|25254|7|0|54|AM|first|morning|breakfast| +25255|AAAAAAAAIKCGAAAA|25255|7|0|55|AM|first|morning|breakfast| +25256|AAAAAAAAJKCGAAAA|25256|7|0|56|AM|first|morning|breakfast| +25257|AAAAAAAAKKCGAAAA|25257|7|0|57|AM|first|morning|breakfast| +25258|AAAAAAAALKCGAAAA|25258|7|0|58|AM|first|morning|breakfast| +25259|AAAAAAAAMKCGAAAA|25259|7|0|59|AM|first|morning|breakfast| +25260|AAAAAAAANKCGAAAA|25260|7|1|0|AM|first|morning|breakfast| +25261|AAAAAAAAOKCGAAAA|25261|7|1|1|AM|first|morning|breakfast| +25262|AAAAAAAAPKCGAAAA|25262|7|1|2|AM|first|morning|breakfast| +25263|AAAAAAAAALCGAAAA|25263|7|1|3|AM|first|morning|breakfast| +25264|AAAAAAAABLCGAAAA|25264|7|1|4|AM|first|morning|breakfast| +25265|AAAAAAAACLCGAAAA|25265|7|1|5|AM|first|morning|breakfast| +25266|AAAAAAAADLCGAAAA|25266|7|1|6|AM|first|morning|breakfast| +25267|AAAAAAAAELCGAAAA|25267|7|1|7|AM|first|morning|breakfast| +25268|AAAAAAAAFLCGAAAA|25268|7|1|8|AM|first|morning|breakfast| +25269|AAAAAAAAGLCGAAAA|25269|7|1|9|AM|first|morning|breakfast| +25270|AAAAAAAAHLCGAAAA|25270|7|1|10|AM|first|morning|breakfast| +25271|AAAAAAAAILCGAAAA|25271|7|1|11|AM|first|morning|breakfast| +25272|AAAAAAAAJLCGAAAA|25272|7|1|12|AM|first|morning|breakfast| +25273|AAAAAAAAKLCGAAAA|25273|7|1|13|AM|first|morning|breakfast| +25274|AAAAAAAALLCGAAAA|25274|7|1|14|AM|first|morning|breakfast| +25275|AAAAAAAAMLCGAAAA|25275|7|1|15|AM|first|morning|breakfast| +25276|AAAAAAAANLCGAAAA|25276|7|1|16|AM|first|morning|breakfast| +25277|AAAAAAAAOLCGAAAA|25277|7|1|17|AM|first|morning|breakfast| +25278|AAAAAAAAPLCGAAAA|25278|7|1|18|AM|first|morning|breakfast| +25279|AAAAAAAAAMCGAAAA|25279|7|1|19|AM|first|morning|breakfast| +25280|AAAAAAAABMCGAAAA|25280|7|1|20|AM|first|morning|breakfast| +25281|AAAAAAAACMCGAAAA|25281|7|1|21|AM|first|morning|breakfast| +25282|AAAAAAAADMCGAAAA|25282|7|1|22|AM|first|morning|breakfast| +25283|AAAAAAAAEMCGAAAA|25283|7|1|23|AM|first|morning|breakfast| +25284|AAAAAAAAFMCGAAAA|25284|7|1|24|AM|first|morning|breakfast| +25285|AAAAAAAAGMCGAAAA|25285|7|1|25|AM|first|morning|breakfast| +25286|AAAAAAAAHMCGAAAA|25286|7|1|26|AM|first|morning|breakfast| +25287|AAAAAAAAIMCGAAAA|25287|7|1|27|AM|first|morning|breakfast| +25288|AAAAAAAAJMCGAAAA|25288|7|1|28|AM|first|morning|breakfast| +25289|AAAAAAAAKMCGAAAA|25289|7|1|29|AM|first|morning|breakfast| +25290|AAAAAAAALMCGAAAA|25290|7|1|30|AM|first|morning|breakfast| +25291|AAAAAAAAMMCGAAAA|25291|7|1|31|AM|first|morning|breakfast| +25292|AAAAAAAANMCGAAAA|25292|7|1|32|AM|first|morning|breakfast| +25293|AAAAAAAAOMCGAAAA|25293|7|1|33|AM|first|morning|breakfast| +25294|AAAAAAAAPMCGAAAA|25294|7|1|34|AM|first|morning|breakfast| +25295|AAAAAAAAANCGAAAA|25295|7|1|35|AM|first|morning|breakfast| +25296|AAAAAAAABNCGAAAA|25296|7|1|36|AM|first|morning|breakfast| +25297|AAAAAAAACNCGAAAA|25297|7|1|37|AM|first|morning|breakfast| +25298|AAAAAAAADNCGAAAA|25298|7|1|38|AM|first|morning|breakfast| +25299|AAAAAAAAENCGAAAA|25299|7|1|39|AM|first|morning|breakfast| +25300|AAAAAAAAFNCGAAAA|25300|7|1|40|AM|first|morning|breakfast| +25301|AAAAAAAAGNCGAAAA|25301|7|1|41|AM|first|morning|breakfast| +25302|AAAAAAAAHNCGAAAA|25302|7|1|42|AM|first|morning|breakfast| +25303|AAAAAAAAINCGAAAA|25303|7|1|43|AM|first|morning|breakfast| +25304|AAAAAAAAJNCGAAAA|25304|7|1|44|AM|first|morning|breakfast| +25305|AAAAAAAAKNCGAAAA|25305|7|1|45|AM|first|morning|breakfast| +25306|AAAAAAAALNCGAAAA|25306|7|1|46|AM|first|morning|breakfast| +25307|AAAAAAAAMNCGAAAA|25307|7|1|47|AM|first|morning|breakfast| +25308|AAAAAAAANNCGAAAA|25308|7|1|48|AM|first|morning|breakfast| +25309|AAAAAAAAONCGAAAA|25309|7|1|49|AM|first|morning|breakfast| +25310|AAAAAAAAPNCGAAAA|25310|7|1|50|AM|first|morning|breakfast| +25311|AAAAAAAAAOCGAAAA|25311|7|1|51|AM|first|morning|breakfast| +25312|AAAAAAAABOCGAAAA|25312|7|1|52|AM|first|morning|breakfast| +25313|AAAAAAAACOCGAAAA|25313|7|1|53|AM|first|morning|breakfast| +25314|AAAAAAAADOCGAAAA|25314|7|1|54|AM|first|morning|breakfast| +25315|AAAAAAAAEOCGAAAA|25315|7|1|55|AM|first|morning|breakfast| +25316|AAAAAAAAFOCGAAAA|25316|7|1|56|AM|first|morning|breakfast| +25317|AAAAAAAAGOCGAAAA|25317|7|1|57|AM|first|morning|breakfast| +25318|AAAAAAAAHOCGAAAA|25318|7|1|58|AM|first|morning|breakfast| +25319|AAAAAAAAIOCGAAAA|25319|7|1|59|AM|first|morning|breakfast| +25320|AAAAAAAAJOCGAAAA|25320|7|2|0|AM|first|morning|breakfast| +25321|AAAAAAAAKOCGAAAA|25321|7|2|1|AM|first|morning|breakfast| +25322|AAAAAAAALOCGAAAA|25322|7|2|2|AM|first|morning|breakfast| +25323|AAAAAAAAMOCGAAAA|25323|7|2|3|AM|first|morning|breakfast| +25324|AAAAAAAANOCGAAAA|25324|7|2|4|AM|first|morning|breakfast| +25325|AAAAAAAAOOCGAAAA|25325|7|2|5|AM|first|morning|breakfast| +25326|AAAAAAAAPOCGAAAA|25326|7|2|6|AM|first|morning|breakfast| +25327|AAAAAAAAAPCGAAAA|25327|7|2|7|AM|first|morning|breakfast| +25328|AAAAAAAABPCGAAAA|25328|7|2|8|AM|first|morning|breakfast| +25329|AAAAAAAACPCGAAAA|25329|7|2|9|AM|first|morning|breakfast| +25330|AAAAAAAADPCGAAAA|25330|7|2|10|AM|first|morning|breakfast| +25331|AAAAAAAAEPCGAAAA|25331|7|2|11|AM|first|morning|breakfast| +25332|AAAAAAAAFPCGAAAA|25332|7|2|12|AM|first|morning|breakfast| +25333|AAAAAAAAGPCGAAAA|25333|7|2|13|AM|first|morning|breakfast| +25334|AAAAAAAAHPCGAAAA|25334|7|2|14|AM|first|morning|breakfast| +25335|AAAAAAAAIPCGAAAA|25335|7|2|15|AM|first|morning|breakfast| +25336|AAAAAAAAJPCGAAAA|25336|7|2|16|AM|first|morning|breakfast| +25337|AAAAAAAAKPCGAAAA|25337|7|2|17|AM|first|morning|breakfast| +25338|AAAAAAAALPCGAAAA|25338|7|2|18|AM|first|morning|breakfast| +25339|AAAAAAAAMPCGAAAA|25339|7|2|19|AM|first|morning|breakfast| +25340|AAAAAAAANPCGAAAA|25340|7|2|20|AM|first|morning|breakfast| +25341|AAAAAAAAOPCGAAAA|25341|7|2|21|AM|first|morning|breakfast| +25342|AAAAAAAAPPCGAAAA|25342|7|2|22|AM|first|morning|breakfast| +25343|AAAAAAAAAADGAAAA|25343|7|2|23|AM|first|morning|breakfast| +25344|AAAAAAAABADGAAAA|25344|7|2|24|AM|first|morning|breakfast| +25345|AAAAAAAACADGAAAA|25345|7|2|25|AM|first|morning|breakfast| +25346|AAAAAAAADADGAAAA|25346|7|2|26|AM|first|morning|breakfast| +25347|AAAAAAAAEADGAAAA|25347|7|2|27|AM|first|morning|breakfast| +25348|AAAAAAAAFADGAAAA|25348|7|2|28|AM|first|morning|breakfast| +25349|AAAAAAAAGADGAAAA|25349|7|2|29|AM|first|morning|breakfast| +25350|AAAAAAAAHADGAAAA|25350|7|2|30|AM|first|morning|breakfast| +25351|AAAAAAAAIADGAAAA|25351|7|2|31|AM|first|morning|breakfast| +25352|AAAAAAAAJADGAAAA|25352|7|2|32|AM|first|morning|breakfast| +25353|AAAAAAAAKADGAAAA|25353|7|2|33|AM|first|morning|breakfast| +25354|AAAAAAAALADGAAAA|25354|7|2|34|AM|first|morning|breakfast| +25355|AAAAAAAAMADGAAAA|25355|7|2|35|AM|first|morning|breakfast| +25356|AAAAAAAANADGAAAA|25356|7|2|36|AM|first|morning|breakfast| +25357|AAAAAAAAOADGAAAA|25357|7|2|37|AM|first|morning|breakfast| +25358|AAAAAAAAPADGAAAA|25358|7|2|38|AM|first|morning|breakfast| +25359|AAAAAAAAABDGAAAA|25359|7|2|39|AM|first|morning|breakfast| +25360|AAAAAAAABBDGAAAA|25360|7|2|40|AM|first|morning|breakfast| +25361|AAAAAAAACBDGAAAA|25361|7|2|41|AM|first|morning|breakfast| +25362|AAAAAAAADBDGAAAA|25362|7|2|42|AM|first|morning|breakfast| +25363|AAAAAAAAEBDGAAAA|25363|7|2|43|AM|first|morning|breakfast| +25364|AAAAAAAAFBDGAAAA|25364|7|2|44|AM|first|morning|breakfast| +25365|AAAAAAAAGBDGAAAA|25365|7|2|45|AM|first|morning|breakfast| +25366|AAAAAAAAHBDGAAAA|25366|7|2|46|AM|first|morning|breakfast| +25367|AAAAAAAAIBDGAAAA|25367|7|2|47|AM|first|morning|breakfast| +25368|AAAAAAAAJBDGAAAA|25368|7|2|48|AM|first|morning|breakfast| +25369|AAAAAAAAKBDGAAAA|25369|7|2|49|AM|first|morning|breakfast| +25370|AAAAAAAALBDGAAAA|25370|7|2|50|AM|first|morning|breakfast| +25371|AAAAAAAAMBDGAAAA|25371|7|2|51|AM|first|morning|breakfast| +25372|AAAAAAAANBDGAAAA|25372|7|2|52|AM|first|morning|breakfast| +25373|AAAAAAAAOBDGAAAA|25373|7|2|53|AM|first|morning|breakfast| +25374|AAAAAAAAPBDGAAAA|25374|7|2|54|AM|first|morning|breakfast| +25375|AAAAAAAAACDGAAAA|25375|7|2|55|AM|first|morning|breakfast| +25376|AAAAAAAABCDGAAAA|25376|7|2|56|AM|first|morning|breakfast| +25377|AAAAAAAACCDGAAAA|25377|7|2|57|AM|first|morning|breakfast| +25378|AAAAAAAADCDGAAAA|25378|7|2|58|AM|first|morning|breakfast| +25379|AAAAAAAAECDGAAAA|25379|7|2|59|AM|first|morning|breakfast| +25380|AAAAAAAAFCDGAAAA|25380|7|3|0|AM|first|morning|breakfast| +25381|AAAAAAAAGCDGAAAA|25381|7|3|1|AM|first|morning|breakfast| +25382|AAAAAAAAHCDGAAAA|25382|7|3|2|AM|first|morning|breakfast| +25383|AAAAAAAAICDGAAAA|25383|7|3|3|AM|first|morning|breakfast| +25384|AAAAAAAAJCDGAAAA|25384|7|3|4|AM|first|morning|breakfast| +25385|AAAAAAAAKCDGAAAA|25385|7|3|5|AM|first|morning|breakfast| +25386|AAAAAAAALCDGAAAA|25386|7|3|6|AM|first|morning|breakfast| +25387|AAAAAAAAMCDGAAAA|25387|7|3|7|AM|first|morning|breakfast| +25388|AAAAAAAANCDGAAAA|25388|7|3|8|AM|first|morning|breakfast| +25389|AAAAAAAAOCDGAAAA|25389|7|3|9|AM|first|morning|breakfast| +25390|AAAAAAAAPCDGAAAA|25390|7|3|10|AM|first|morning|breakfast| +25391|AAAAAAAAADDGAAAA|25391|7|3|11|AM|first|morning|breakfast| +25392|AAAAAAAABDDGAAAA|25392|7|3|12|AM|first|morning|breakfast| +25393|AAAAAAAACDDGAAAA|25393|7|3|13|AM|first|morning|breakfast| +25394|AAAAAAAADDDGAAAA|25394|7|3|14|AM|first|morning|breakfast| +25395|AAAAAAAAEDDGAAAA|25395|7|3|15|AM|first|morning|breakfast| +25396|AAAAAAAAFDDGAAAA|25396|7|3|16|AM|first|morning|breakfast| +25397|AAAAAAAAGDDGAAAA|25397|7|3|17|AM|first|morning|breakfast| +25398|AAAAAAAAHDDGAAAA|25398|7|3|18|AM|first|morning|breakfast| +25399|AAAAAAAAIDDGAAAA|25399|7|3|19|AM|first|morning|breakfast| +25400|AAAAAAAAJDDGAAAA|25400|7|3|20|AM|first|morning|breakfast| +25401|AAAAAAAAKDDGAAAA|25401|7|3|21|AM|first|morning|breakfast| +25402|AAAAAAAALDDGAAAA|25402|7|3|22|AM|first|morning|breakfast| +25403|AAAAAAAAMDDGAAAA|25403|7|3|23|AM|first|morning|breakfast| +25404|AAAAAAAANDDGAAAA|25404|7|3|24|AM|first|morning|breakfast| +25405|AAAAAAAAODDGAAAA|25405|7|3|25|AM|first|morning|breakfast| +25406|AAAAAAAAPDDGAAAA|25406|7|3|26|AM|first|morning|breakfast| +25407|AAAAAAAAAEDGAAAA|25407|7|3|27|AM|first|morning|breakfast| +25408|AAAAAAAABEDGAAAA|25408|7|3|28|AM|first|morning|breakfast| +25409|AAAAAAAACEDGAAAA|25409|7|3|29|AM|first|morning|breakfast| +25410|AAAAAAAADEDGAAAA|25410|7|3|30|AM|first|morning|breakfast| +25411|AAAAAAAAEEDGAAAA|25411|7|3|31|AM|first|morning|breakfast| +25412|AAAAAAAAFEDGAAAA|25412|7|3|32|AM|first|morning|breakfast| +25413|AAAAAAAAGEDGAAAA|25413|7|3|33|AM|first|morning|breakfast| +25414|AAAAAAAAHEDGAAAA|25414|7|3|34|AM|first|morning|breakfast| +25415|AAAAAAAAIEDGAAAA|25415|7|3|35|AM|first|morning|breakfast| +25416|AAAAAAAAJEDGAAAA|25416|7|3|36|AM|first|morning|breakfast| +25417|AAAAAAAAKEDGAAAA|25417|7|3|37|AM|first|morning|breakfast| +25418|AAAAAAAALEDGAAAA|25418|7|3|38|AM|first|morning|breakfast| +25419|AAAAAAAAMEDGAAAA|25419|7|3|39|AM|first|morning|breakfast| +25420|AAAAAAAANEDGAAAA|25420|7|3|40|AM|first|morning|breakfast| +25421|AAAAAAAAOEDGAAAA|25421|7|3|41|AM|first|morning|breakfast| +25422|AAAAAAAAPEDGAAAA|25422|7|3|42|AM|first|morning|breakfast| +25423|AAAAAAAAAFDGAAAA|25423|7|3|43|AM|first|morning|breakfast| +25424|AAAAAAAABFDGAAAA|25424|7|3|44|AM|first|morning|breakfast| +25425|AAAAAAAACFDGAAAA|25425|7|3|45|AM|first|morning|breakfast| +25426|AAAAAAAADFDGAAAA|25426|7|3|46|AM|first|morning|breakfast| +25427|AAAAAAAAEFDGAAAA|25427|7|3|47|AM|first|morning|breakfast| +25428|AAAAAAAAFFDGAAAA|25428|7|3|48|AM|first|morning|breakfast| +25429|AAAAAAAAGFDGAAAA|25429|7|3|49|AM|first|morning|breakfast| +25430|AAAAAAAAHFDGAAAA|25430|7|3|50|AM|first|morning|breakfast| +25431|AAAAAAAAIFDGAAAA|25431|7|3|51|AM|first|morning|breakfast| +25432|AAAAAAAAJFDGAAAA|25432|7|3|52|AM|first|morning|breakfast| +25433|AAAAAAAAKFDGAAAA|25433|7|3|53|AM|first|morning|breakfast| +25434|AAAAAAAALFDGAAAA|25434|7|3|54|AM|first|morning|breakfast| +25435|AAAAAAAAMFDGAAAA|25435|7|3|55|AM|first|morning|breakfast| +25436|AAAAAAAANFDGAAAA|25436|7|3|56|AM|first|morning|breakfast| +25437|AAAAAAAAOFDGAAAA|25437|7|3|57|AM|first|morning|breakfast| +25438|AAAAAAAAPFDGAAAA|25438|7|3|58|AM|first|morning|breakfast| +25439|AAAAAAAAAGDGAAAA|25439|7|3|59|AM|first|morning|breakfast| +25440|AAAAAAAABGDGAAAA|25440|7|4|0|AM|first|morning|breakfast| +25441|AAAAAAAACGDGAAAA|25441|7|4|1|AM|first|morning|breakfast| +25442|AAAAAAAADGDGAAAA|25442|7|4|2|AM|first|morning|breakfast| +25443|AAAAAAAAEGDGAAAA|25443|7|4|3|AM|first|morning|breakfast| +25444|AAAAAAAAFGDGAAAA|25444|7|4|4|AM|first|morning|breakfast| +25445|AAAAAAAAGGDGAAAA|25445|7|4|5|AM|first|morning|breakfast| +25446|AAAAAAAAHGDGAAAA|25446|7|4|6|AM|first|morning|breakfast| +25447|AAAAAAAAIGDGAAAA|25447|7|4|7|AM|first|morning|breakfast| +25448|AAAAAAAAJGDGAAAA|25448|7|4|8|AM|first|morning|breakfast| +25449|AAAAAAAAKGDGAAAA|25449|7|4|9|AM|first|morning|breakfast| +25450|AAAAAAAALGDGAAAA|25450|7|4|10|AM|first|morning|breakfast| +25451|AAAAAAAAMGDGAAAA|25451|7|4|11|AM|first|morning|breakfast| +25452|AAAAAAAANGDGAAAA|25452|7|4|12|AM|first|morning|breakfast| +25453|AAAAAAAAOGDGAAAA|25453|7|4|13|AM|first|morning|breakfast| +25454|AAAAAAAAPGDGAAAA|25454|7|4|14|AM|first|morning|breakfast| +25455|AAAAAAAAAHDGAAAA|25455|7|4|15|AM|first|morning|breakfast| +25456|AAAAAAAABHDGAAAA|25456|7|4|16|AM|first|morning|breakfast| +25457|AAAAAAAACHDGAAAA|25457|7|4|17|AM|first|morning|breakfast| +25458|AAAAAAAADHDGAAAA|25458|7|4|18|AM|first|morning|breakfast| +25459|AAAAAAAAEHDGAAAA|25459|7|4|19|AM|first|morning|breakfast| +25460|AAAAAAAAFHDGAAAA|25460|7|4|20|AM|first|morning|breakfast| +25461|AAAAAAAAGHDGAAAA|25461|7|4|21|AM|first|morning|breakfast| +25462|AAAAAAAAHHDGAAAA|25462|7|4|22|AM|first|morning|breakfast| +25463|AAAAAAAAIHDGAAAA|25463|7|4|23|AM|first|morning|breakfast| +25464|AAAAAAAAJHDGAAAA|25464|7|4|24|AM|first|morning|breakfast| +25465|AAAAAAAAKHDGAAAA|25465|7|4|25|AM|first|morning|breakfast| +25466|AAAAAAAALHDGAAAA|25466|7|4|26|AM|first|morning|breakfast| +25467|AAAAAAAAMHDGAAAA|25467|7|4|27|AM|first|morning|breakfast| +25468|AAAAAAAANHDGAAAA|25468|7|4|28|AM|first|morning|breakfast| +25469|AAAAAAAAOHDGAAAA|25469|7|4|29|AM|first|morning|breakfast| +25470|AAAAAAAAPHDGAAAA|25470|7|4|30|AM|first|morning|breakfast| +25471|AAAAAAAAAIDGAAAA|25471|7|4|31|AM|first|morning|breakfast| +25472|AAAAAAAABIDGAAAA|25472|7|4|32|AM|first|morning|breakfast| +25473|AAAAAAAACIDGAAAA|25473|7|4|33|AM|first|morning|breakfast| +25474|AAAAAAAADIDGAAAA|25474|7|4|34|AM|first|morning|breakfast| +25475|AAAAAAAAEIDGAAAA|25475|7|4|35|AM|first|morning|breakfast| +25476|AAAAAAAAFIDGAAAA|25476|7|4|36|AM|first|morning|breakfast| +25477|AAAAAAAAGIDGAAAA|25477|7|4|37|AM|first|morning|breakfast| +25478|AAAAAAAAHIDGAAAA|25478|7|4|38|AM|first|morning|breakfast| +25479|AAAAAAAAIIDGAAAA|25479|7|4|39|AM|first|morning|breakfast| +25480|AAAAAAAAJIDGAAAA|25480|7|4|40|AM|first|morning|breakfast| +25481|AAAAAAAAKIDGAAAA|25481|7|4|41|AM|first|morning|breakfast| +25482|AAAAAAAALIDGAAAA|25482|7|4|42|AM|first|morning|breakfast| +25483|AAAAAAAAMIDGAAAA|25483|7|4|43|AM|first|morning|breakfast| +25484|AAAAAAAANIDGAAAA|25484|7|4|44|AM|first|morning|breakfast| +25485|AAAAAAAAOIDGAAAA|25485|7|4|45|AM|first|morning|breakfast| +25486|AAAAAAAAPIDGAAAA|25486|7|4|46|AM|first|morning|breakfast| +25487|AAAAAAAAAJDGAAAA|25487|7|4|47|AM|first|morning|breakfast| +25488|AAAAAAAABJDGAAAA|25488|7|4|48|AM|first|morning|breakfast| +25489|AAAAAAAACJDGAAAA|25489|7|4|49|AM|first|morning|breakfast| +25490|AAAAAAAADJDGAAAA|25490|7|4|50|AM|first|morning|breakfast| +25491|AAAAAAAAEJDGAAAA|25491|7|4|51|AM|first|morning|breakfast| +25492|AAAAAAAAFJDGAAAA|25492|7|4|52|AM|first|morning|breakfast| +25493|AAAAAAAAGJDGAAAA|25493|7|4|53|AM|first|morning|breakfast| +25494|AAAAAAAAHJDGAAAA|25494|7|4|54|AM|first|morning|breakfast| +25495|AAAAAAAAIJDGAAAA|25495|7|4|55|AM|first|morning|breakfast| +25496|AAAAAAAAJJDGAAAA|25496|7|4|56|AM|first|morning|breakfast| +25497|AAAAAAAAKJDGAAAA|25497|7|4|57|AM|first|morning|breakfast| +25498|AAAAAAAALJDGAAAA|25498|7|4|58|AM|first|morning|breakfast| +25499|AAAAAAAAMJDGAAAA|25499|7|4|59|AM|first|morning|breakfast| +25500|AAAAAAAANJDGAAAA|25500|7|5|0|AM|first|morning|breakfast| +25501|AAAAAAAAOJDGAAAA|25501|7|5|1|AM|first|morning|breakfast| +25502|AAAAAAAAPJDGAAAA|25502|7|5|2|AM|first|morning|breakfast| +25503|AAAAAAAAAKDGAAAA|25503|7|5|3|AM|first|morning|breakfast| +25504|AAAAAAAABKDGAAAA|25504|7|5|4|AM|first|morning|breakfast| +25505|AAAAAAAACKDGAAAA|25505|7|5|5|AM|first|morning|breakfast| +25506|AAAAAAAADKDGAAAA|25506|7|5|6|AM|first|morning|breakfast| +25507|AAAAAAAAEKDGAAAA|25507|7|5|7|AM|first|morning|breakfast| +25508|AAAAAAAAFKDGAAAA|25508|7|5|8|AM|first|morning|breakfast| +25509|AAAAAAAAGKDGAAAA|25509|7|5|9|AM|first|morning|breakfast| +25510|AAAAAAAAHKDGAAAA|25510|7|5|10|AM|first|morning|breakfast| +25511|AAAAAAAAIKDGAAAA|25511|7|5|11|AM|first|morning|breakfast| +25512|AAAAAAAAJKDGAAAA|25512|7|5|12|AM|first|morning|breakfast| +25513|AAAAAAAAKKDGAAAA|25513|7|5|13|AM|first|morning|breakfast| +25514|AAAAAAAALKDGAAAA|25514|7|5|14|AM|first|morning|breakfast| +25515|AAAAAAAAMKDGAAAA|25515|7|5|15|AM|first|morning|breakfast| +25516|AAAAAAAANKDGAAAA|25516|7|5|16|AM|first|morning|breakfast| +25517|AAAAAAAAOKDGAAAA|25517|7|5|17|AM|first|morning|breakfast| +25518|AAAAAAAAPKDGAAAA|25518|7|5|18|AM|first|morning|breakfast| +25519|AAAAAAAAALDGAAAA|25519|7|5|19|AM|first|morning|breakfast| +25520|AAAAAAAABLDGAAAA|25520|7|5|20|AM|first|morning|breakfast| +25521|AAAAAAAACLDGAAAA|25521|7|5|21|AM|first|morning|breakfast| +25522|AAAAAAAADLDGAAAA|25522|7|5|22|AM|first|morning|breakfast| +25523|AAAAAAAAELDGAAAA|25523|7|5|23|AM|first|morning|breakfast| +25524|AAAAAAAAFLDGAAAA|25524|7|5|24|AM|first|morning|breakfast| +25525|AAAAAAAAGLDGAAAA|25525|7|5|25|AM|first|morning|breakfast| +25526|AAAAAAAAHLDGAAAA|25526|7|5|26|AM|first|morning|breakfast| +25527|AAAAAAAAILDGAAAA|25527|7|5|27|AM|first|morning|breakfast| +25528|AAAAAAAAJLDGAAAA|25528|7|5|28|AM|first|morning|breakfast| +25529|AAAAAAAAKLDGAAAA|25529|7|5|29|AM|first|morning|breakfast| +25530|AAAAAAAALLDGAAAA|25530|7|5|30|AM|first|morning|breakfast| +25531|AAAAAAAAMLDGAAAA|25531|7|5|31|AM|first|morning|breakfast| +25532|AAAAAAAANLDGAAAA|25532|7|5|32|AM|first|morning|breakfast| +25533|AAAAAAAAOLDGAAAA|25533|7|5|33|AM|first|morning|breakfast| +25534|AAAAAAAAPLDGAAAA|25534|7|5|34|AM|first|morning|breakfast| +25535|AAAAAAAAAMDGAAAA|25535|7|5|35|AM|first|morning|breakfast| +25536|AAAAAAAABMDGAAAA|25536|7|5|36|AM|first|morning|breakfast| +25537|AAAAAAAACMDGAAAA|25537|7|5|37|AM|first|morning|breakfast| +25538|AAAAAAAADMDGAAAA|25538|7|5|38|AM|first|morning|breakfast| +25539|AAAAAAAAEMDGAAAA|25539|7|5|39|AM|first|morning|breakfast| +25540|AAAAAAAAFMDGAAAA|25540|7|5|40|AM|first|morning|breakfast| +25541|AAAAAAAAGMDGAAAA|25541|7|5|41|AM|first|morning|breakfast| +25542|AAAAAAAAHMDGAAAA|25542|7|5|42|AM|first|morning|breakfast| +25543|AAAAAAAAIMDGAAAA|25543|7|5|43|AM|first|morning|breakfast| +25544|AAAAAAAAJMDGAAAA|25544|7|5|44|AM|first|morning|breakfast| +25545|AAAAAAAAKMDGAAAA|25545|7|5|45|AM|first|morning|breakfast| +25546|AAAAAAAALMDGAAAA|25546|7|5|46|AM|first|morning|breakfast| +25547|AAAAAAAAMMDGAAAA|25547|7|5|47|AM|first|morning|breakfast| +25548|AAAAAAAANMDGAAAA|25548|7|5|48|AM|first|morning|breakfast| +25549|AAAAAAAAOMDGAAAA|25549|7|5|49|AM|first|morning|breakfast| +25550|AAAAAAAAPMDGAAAA|25550|7|5|50|AM|first|morning|breakfast| +25551|AAAAAAAAANDGAAAA|25551|7|5|51|AM|first|morning|breakfast| +25552|AAAAAAAABNDGAAAA|25552|7|5|52|AM|first|morning|breakfast| +25553|AAAAAAAACNDGAAAA|25553|7|5|53|AM|first|morning|breakfast| +25554|AAAAAAAADNDGAAAA|25554|7|5|54|AM|first|morning|breakfast| +25555|AAAAAAAAENDGAAAA|25555|7|5|55|AM|first|morning|breakfast| +25556|AAAAAAAAFNDGAAAA|25556|7|5|56|AM|first|morning|breakfast| +25557|AAAAAAAAGNDGAAAA|25557|7|5|57|AM|first|morning|breakfast| +25558|AAAAAAAAHNDGAAAA|25558|7|5|58|AM|first|morning|breakfast| +25559|AAAAAAAAINDGAAAA|25559|7|5|59|AM|first|morning|breakfast| +25560|AAAAAAAAJNDGAAAA|25560|7|6|0|AM|first|morning|breakfast| +25561|AAAAAAAAKNDGAAAA|25561|7|6|1|AM|first|morning|breakfast| +25562|AAAAAAAALNDGAAAA|25562|7|6|2|AM|first|morning|breakfast| +25563|AAAAAAAAMNDGAAAA|25563|7|6|3|AM|first|morning|breakfast| +25564|AAAAAAAANNDGAAAA|25564|7|6|4|AM|first|morning|breakfast| +25565|AAAAAAAAONDGAAAA|25565|7|6|5|AM|first|morning|breakfast| +25566|AAAAAAAAPNDGAAAA|25566|7|6|6|AM|first|morning|breakfast| +25567|AAAAAAAAAODGAAAA|25567|7|6|7|AM|first|morning|breakfast| +25568|AAAAAAAABODGAAAA|25568|7|6|8|AM|first|morning|breakfast| +25569|AAAAAAAACODGAAAA|25569|7|6|9|AM|first|morning|breakfast| +25570|AAAAAAAADODGAAAA|25570|7|6|10|AM|first|morning|breakfast| +25571|AAAAAAAAEODGAAAA|25571|7|6|11|AM|first|morning|breakfast| +25572|AAAAAAAAFODGAAAA|25572|7|6|12|AM|first|morning|breakfast| +25573|AAAAAAAAGODGAAAA|25573|7|6|13|AM|first|morning|breakfast| +25574|AAAAAAAAHODGAAAA|25574|7|6|14|AM|first|morning|breakfast| +25575|AAAAAAAAIODGAAAA|25575|7|6|15|AM|first|morning|breakfast| +25576|AAAAAAAAJODGAAAA|25576|7|6|16|AM|first|morning|breakfast| +25577|AAAAAAAAKODGAAAA|25577|7|6|17|AM|first|morning|breakfast| +25578|AAAAAAAALODGAAAA|25578|7|6|18|AM|first|morning|breakfast| +25579|AAAAAAAAMODGAAAA|25579|7|6|19|AM|first|morning|breakfast| +25580|AAAAAAAANODGAAAA|25580|7|6|20|AM|first|morning|breakfast| +25581|AAAAAAAAOODGAAAA|25581|7|6|21|AM|first|morning|breakfast| +25582|AAAAAAAAPODGAAAA|25582|7|6|22|AM|first|morning|breakfast| +25583|AAAAAAAAAPDGAAAA|25583|7|6|23|AM|first|morning|breakfast| +25584|AAAAAAAABPDGAAAA|25584|7|6|24|AM|first|morning|breakfast| +25585|AAAAAAAACPDGAAAA|25585|7|6|25|AM|first|morning|breakfast| +25586|AAAAAAAADPDGAAAA|25586|7|6|26|AM|first|morning|breakfast| +25587|AAAAAAAAEPDGAAAA|25587|7|6|27|AM|first|morning|breakfast| +25588|AAAAAAAAFPDGAAAA|25588|7|6|28|AM|first|morning|breakfast| +25589|AAAAAAAAGPDGAAAA|25589|7|6|29|AM|first|morning|breakfast| +25590|AAAAAAAAHPDGAAAA|25590|7|6|30|AM|first|morning|breakfast| +25591|AAAAAAAAIPDGAAAA|25591|7|6|31|AM|first|morning|breakfast| +25592|AAAAAAAAJPDGAAAA|25592|7|6|32|AM|first|morning|breakfast| +25593|AAAAAAAAKPDGAAAA|25593|7|6|33|AM|first|morning|breakfast| +25594|AAAAAAAALPDGAAAA|25594|7|6|34|AM|first|morning|breakfast| +25595|AAAAAAAAMPDGAAAA|25595|7|6|35|AM|first|morning|breakfast| +25596|AAAAAAAANPDGAAAA|25596|7|6|36|AM|first|morning|breakfast| +25597|AAAAAAAAOPDGAAAA|25597|7|6|37|AM|first|morning|breakfast| +25598|AAAAAAAAPPDGAAAA|25598|7|6|38|AM|first|morning|breakfast| +25599|AAAAAAAAAAEGAAAA|25599|7|6|39|AM|first|morning|breakfast| +25600|AAAAAAAABAEGAAAA|25600|7|6|40|AM|first|morning|breakfast| +25601|AAAAAAAACAEGAAAA|25601|7|6|41|AM|first|morning|breakfast| +25602|AAAAAAAADAEGAAAA|25602|7|6|42|AM|first|morning|breakfast| +25603|AAAAAAAAEAEGAAAA|25603|7|6|43|AM|first|morning|breakfast| +25604|AAAAAAAAFAEGAAAA|25604|7|6|44|AM|first|morning|breakfast| +25605|AAAAAAAAGAEGAAAA|25605|7|6|45|AM|first|morning|breakfast| +25606|AAAAAAAAHAEGAAAA|25606|7|6|46|AM|first|morning|breakfast| +25607|AAAAAAAAIAEGAAAA|25607|7|6|47|AM|first|morning|breakfast| +25608|AAAAAAAAJAEGAAAA|25608|7|6|48|AM|first|morning|breakfast| +25609|AAAAAAAAKAEGAAAA|25609|7|6|49|AM|first|morning|breakfast| +25610|AAAAAAAALAEGAAAA|25610|7|6|50|AM|first|morning|breakfast| +25611|AAAAAAAAMAEGAAAA|25611|7|6|51|AM|first|morning|breakfast| +25612|AAAAAAAANAEGAAAA|25612|7|6|52|AM|first|morning|breakfast| +25613|AAAAAAAAOAEGAAAA|25613|7|6|53|AM|first|morning|breakfast| +25614|AAAAAAAAPAEGAAAA|25614|7|6|54|AM|first|morning|breakfast| +25615|AAAAAAAAABEGAAAA|25615|7|6|55|AM|first|morning|breakfast| +25616|AAAAAAAABBEGAAAA|25616|7|6|56|AM|first|morning|breakfast| +25617|AAAAAAAACBEGAAAA|25617|7|6|57|AM|first|morning|breakfast| +25618|AAAAAAAADBEGAAAA|25618|7|6|58|AM|first|morning|breakfast| +25619|AAAAAAAAEBEGAAAA|25619|7|6|59|AM|first|morning|breakfast| +25620|AAAAAAAAFBEGAAAA|25620|7|7|0|AM|first|morning|breakfast| +25621|AAAAAAAAGBEGAAAA|25621|7|7|1|AM|first|morning|breakfast| +25622|AAAAAAAAHBEGAAAA|25622|7|7|2|AM|first|morning|breakfast| +25623|AAAAAAAAIBEGAAAA|25623|7|7|3|AM|first|morning|breakfast| +25624|AAAAAAAAJBEGAAAA|25624|7|7|4|AM|first|morning|breakfast| +25625|AAAAAAAAKBEGAAAA|25625|7|7|5|AM|first|morning|breakfast| +25626|AAAAAAAALBEGAAAA|25626|7|7|6|AM|first|morning|breakfast| +25627|AAAAAAAAMBEGAAAA|25627|7|7|7|AM|first|morning|breakfast| +25628|AAAAAAAANBEGAAAA|25628|7|7|8|AM|first|morning|breakfast| +25629|AAAAAAAAOBEGAAAA|25629|7|7|9|AM|first|morning|breakfast| +25630|AAAAAAAAPBEGAAAA|25630|7|7|10|AM|first|morning|breakfast| +25631|AAAAAAAAACEGAAAA|25631|7|7|11|AM|first|morning|breakfast| +25632|AAAAAAAABCEGAAAA|25632|7|7|12|AM|first|morning|breakfast| +25633|AAAAAAAACCEGAAAA|25633|7|7|13|AM|first|morning|breakfast| +25634|AAAAAAAADCEGAAAA|25634|7|7|14|AM|first|morning|breakfast| +25635|AAAAAAAAECEGAAAA|25635|7|7|15|AM|first|morning|breakfast| +25636|AAAAAAAAFCEGAAAA|25636|7|7|16|AM|first|morning|breakfast| +25637|AAAAAAAAGCEGAAAA|25637|7|7|17|AM|first|morning|breakfast| +25638|AAAAAAAAHCEGAAAA|25638|7|7|18|AM|first|morning|breakfast| +25639|AAAAAAAAICEGAAAA|25639|7|7|19|AM|first|morning|breakfast| +25640|AAAAAAAAJCEGAAAA|25640|7|7|20|AM|first|morning|breakfast| +25641|AAAAAAAAKCEGAAAA|25641|7|7|21|AM|first|morning|breakfast| +25642|AAAAAAAALCEGAAAA|25642|7|7|22|AM|first|morning|breakfast| +25643|AAAAAAAAMCEGAAAA|25643|7|7|23|AM|first|morning|breakfast| +25644|AAAAAAAANCEGAAAA|25644|7|7|24|AM|first|morning|breakfast| +25645|AAAAAAAAOCEGAAAA|25645|7|7|25|AM|first|morning|breakfast| +25646|AAAAAAAAPCEGAAAA|25646|7|7|26|AM|first|morning|breakfast| +25647|AAAAAAAAADEGAAAA|25647|7|7|27|AM|first|morning|breakfast| +25648|AAAAAAAABDEGAAAA|25648|7|7|28|AM|first|morning|breakfast| +25649|AAAAAAAACDEGAAAA|25649|7|7|29|AM|first|morning|breakfast| +25650|AAAAAAAADDEGAAAA|25650|7|7|30|AM|first|morning|breakfast| +25651|AAAAAAAAEDEGAAAA|25651|7|7|31|AM|first|morning|breakfast| +25652|AAAAAAAAFDEGAAAA|25652|7|7|32|AM|first|morning|breakfast| +25653|AAAAAAAAGDEGAAAA|25653|7|7|33|AM|first|morning|breakfast| +25654|AAAAAAAAHDEGAAAA|25654|7|7|34|AM|first|morning|breakfast| +25655|AAAAAAAAIDEGAAAA|25655|7|7|35|AM|first|morning|breakfast| +25656|AAAAAAAAJDEGAAAA|25656|7|7|36|AM|first|morning|breakfast| +25657|AAAAAAAAKDEGAAAA|25657|7|7|37|AM|first|morning|breakfast| +25658|AAAAAAAALDEGAAAA|25658|7|7|38|AM|first|morning|breakfast| +25659|AAAAAAAAMDEGAAAA|25659|7|7|39|AM|first|morning|breakfast| +25660|AAAAAAAANDEGAAAA|25660|7|7|40|AM|first|morning|breakfast| +25661|AAAAAAAAODEGAAAA|25661|7|7|41|AM|first|morning|breakfast| +25662|AAAAAAAAPDEGAAAA|25662|7|7|42|AM|first|morning|breakfast| +25663|AAAAAAAAAEEGAAAA|25663|7|7|43|AM|first|morning|breakfast| +25664|AAAAAAAABEEGAAAA|25664|7|7|44|AM|first|morning|breakfast| +25665|AAAAAAAACEEGAAAA|25665|7|7|45|AM|first|morning|breakfast| +25666|AAAAAAAADEEGAAAA|25666|7|7|46|AM|first|morning|breakfast| +25667|AAAAAAAAEEEGAAAA|25667|7|7|47|AM|first|morning|breakfast| +25668|AAAAAAAAFEEGAAAA|25668|7|7|48|AM|first|morning|breakfast| +25669|AAAAAAAAGEEGAAAA|25669|7|7|49|AM|first|morning|breakfast| +25670|AAAAAAAAHEEGAAAA|25670|7|7|50|AM|first|morning|breakfast| +25671|AAAAAAAAIEEGAAAA|25671|7|7|51|AM|first|morning|breakfast| +25672|AAAAAAAAJEEGAAAA|25672|7|7|52|AM|first|morning|breakfast| +25673|AAAAAAAAKEEGAAAA|25673|7|7|53|AM|first|morning|breakfast| +25674|AAAAAAAALEEGAAAA|25674|7|7|54|AM|first|morning|breakfast| +25675|AAAAAAAAMEEGAAAA|25675|7|7|55|AM|first|morning|breakfast| +25676|AAAAAAAANEEGAAAA|25676|7|7|56|AM|first|morning|breakfast| +25677|AAAAAAAAOEEGAAAA|25677|7|7|57|AM|first|morning|breakfast| +25678|AAAAAAAAPEEGAAAA|25678|7|7|58|AM|first|morning|breakfast| +25679|AAAAAAAAAFEGAAAA|25679|7|7|59|AM|first|morning|breakfast| +25680|AAAAAAAABFEGAAAA|25680|7|8|0|AM|first|morning|breakfast| +25681|AAAAAAAACFEGAAAA|25681|7|8|1|AM|first|morning|breakfast| +25682|AAAAAAAADFEGAAAA|25682|7|8|2|AM|first|morning|breakfast| +25683|AAAAAAAAEFEGAAAA|25683|7|8|3|AM|first|morning|breakfast| +25684|AAAAAAAAFFEGAAAA|25684|7|8|4|AM|first|morning|breakfast| +25685|AAAAAAAAGFEGAAAA|25685|7|8|5|AM|first|morning|breakfast| +25686|AAAAAAAAHFEGAAAA|25686|7|8|6|AM|first|morning|breakfast| +25687|AAAAAAAAIFEGAAAA|25687|7|8|7|AM|first|morning|breakfast| +25688|AAAAAAAAJFEGAAAA|25688|7|8|8|AM|first|morning|breakfast| +25689|AAAAAAAAKFEGAAAA|25689|7|8|9|AM|first|morning|breakfast| +25690|AAAAAAAALFEGAAAA|25690|7|8|10|AM|first|morning|breakfast| +25691|AAAAAAAAMFEGAAAA|25691|7|8|11|AM|first|morning|breakfast| +25692|AAAAAAAANFEGAAAA|25692|7|8|12|AM|first|morning|breakfast| +25693|AAAAAAAAOFEGAAAA|25693|7|8|13|AM|first|morning|breakfast| +25694|AAAAAAAAPFEGAAAA|25694|7|8|14|AM|first|morning|breakfast| +25695|AAAAAAAAAGEGAAAA|25695|7|8|15|AM|first|morning|breakfast| +25696|AAAAAAAABGEGAAAA|25696|7|8|16|AM|first|morning|breakfast| +25697|AAAAAAAACGEGAAAA|25697|7|8|17|AM|first|morning|breakfast| +25698|AAAAAAAADGEGAAAA|25698|7|8|18|AM|first|morning|breakfast| +25699|AAAAAAAAEGEGAAAA|25699|7|8|19|AM|first|morning|breakfast| +25700|AAAAAAAAFGEGAAAA|25700|7|8|20|AM|first|morning|breakfast| +25701|AAAAAAAAGGEGAAAA|25701|7|8|21|AM|first|morning|breakfast| +25702|AAAAAAAAHGEGAAAA|25702|7|8|22|AM|first|morning|breakfast| +25703|AAAAAAAAIGEGAAAA|25703|7|8|23|AM|first|morning|breakfast| +25704|AAAAAAAAJGEGAAAA|25704|7|8|24|AM|first|morning|breakfast| +25705|AAAAAAAAKGEGAAAA|25705|7|8|25|AM|first|morning|breakfast| +25706|AAAAAAAALGEGAAAA|25706|7|8|26|AM|first|morning|breakfast| +25707|AAAAAAAAMGEGAAAA|25707|7|8|27|AM|first|morning|breakfast| +25708|AAAAAAAANGEGAAAA|25708|7|8|28|AM|first|morning|breakfast| +25709|AAAAAAAAOGEGAAAA|25709|7|8|29|AM|first|morning|breakfast| +25710|AAAAAAAAPGEGAAAA|25710|7|8|30|AM|first|morning|breakfast| +25711|AAAAAAAAAHEGAAAA|25711|7|8|31|AM|first|morning|breakfast| +25712|AAAAAAAABHEGAAAA|25712|7|8|32|AM|first|morning|breakfast| +25713|AAAAAAAACHEGAAAA|25713|7|8|33|AM|first|morning|breakfast| +25714|AAAAAAAADHEGAAAA|25714|7|8|34|AM|first|morning|breakfast| +25715|AAAAAAAAEHEGAAAA|25715|7|8|35|AM|first|morning|breakfast| +25716|AAAAAAAAFHEGAAAA|25716|7|8|36|AM|first|morning|breakfast| +25717|AAAAAAAAGHEGAAAA|25717|7|8|37|AM|first|morning|breakfast| +25718|AAAAAAAAHHEGAAAA|25718|7|8|38|AM|first|morning|breakfast| +25719|AAAAAAAAIHEGAAAA|25719|7|8|39|AM|first|morning|breakfast| +25720|AAAAAAAAJHEGAAAA|25720|7|8|40|AM|first|morning|breakfast| +25721|AAAAAAAAKHEGAAAA|25721|7|8|41|AM|first|morning|breakfast| +25722|AAAAAAAALHEGAAAA|25722|7|8|42|AM|first|morning|breakfast| +25723|AAAAAAAAMHEGAAAA|25723|7|8|43|AM|first|morning|breakfast| +25724|AAAAAAAANHEGAAAA|25724|7|8|44|AM|first|morning|breakfast| +25725|AAAAAAAAOHEGAAAA|25725|7|8|45|AM|first|morning|breakfast| +25726|AAAAAAAAPHEGAAAA|25726|7|8|46|AM|first|morning|breakfast| +25727|AAAAAAAAAIEGAAAA|25727|7|8|47|AM|first|morning|breakfast| +25728|AAAAAAAABIEGAAAA|25728|7|8|48|AM|first|morning|breakfast| +25729|AAAAAAAACIEGAAAA|25729|7|8|49|AM|first|morning|breakfast| +25730|AAAAAAAADIEGAAAA|25730|7|8|50|AM|first|morning|breakfast| +25731|AAAAAAAAEIEGAAAA|25731|7|8|51|AM|first|morning|breakfast| +25732|AAAAAAAAFIEGAAAA|25732|7|8|52|AM|first|morning|breakfast| +25733|AAAAAAAAGIEGAAAA|25733|7|8|53|AM|first|morning|breakfast| +25734|AAAAAAAAHIEGAAAA|25734|7|8|54|AM|first|morning|breakfast| +25735|AAAAAAAAIIEGAAAA|25735|7|8|55|AM|first|morning|breakfast| +25736|AAAAAAAAJIEGAAAA|25736|7|8|56|AM|first|morning|breakfast| +25737|AAAAAAAAKIEGAAAA|25737|7|8|57|AM|first|morning|breakfast| +25738|AAAAAAAALIEGAAAA|25738|7|8|58|AM|first|morning|breakfast| +25739|AAAAAAAAMIEGAAAA|25739|7|8|59|AM|first|morning|breakfast| +25740|AAAAAAAANIEGAAAA|25740|7|9|0|AM|first|morning|breakfast| +25741|AAAAAAAAOIEGAAAA|25741|7|9|1|AM|first|morning|breakfast| +25742|AAAAAAAAPIEGAAAA|25742|7|9|2|AM|first|morning|breakfast| +25743|AAAAAAAAAJEGAAAA|25743|7|9|3|AM|first|morning|breakfast| +25744|AAAAAAAABJEGAAAA|25744|7|9|4|AM|first|morning|breakfast| +25745|AAAAAAAACJEGAAAA|25745|7|9|5|AM|first|morning|breakfast| +25746|AAAAAAAADJEGAAAA|25746|7|9|6|AM|first|morning|breakfast| +25747|AAAAAAAAEJEGAAAA|25747|7|9|7|AM|first|morning|breakfast| +25748|AAAAAAAAFJEGAAAA|25748|7|9|8|AM|first|morning|breakfast| +25749|AAAAAAAAGJEGAAAA|25749|7|9|9|AM|first|morning|breakfast| +25750|AAAAAAAAHJEGAAAA|25750|7|9|10|AM|first|morning|breakfast| +25751|AAAAAAAAIJEGAAAA|25751|7|9|11|AM|first|morning|breakfast| +25752|AAAAAAAAJJEGAAAA|25752|7|9|12|AM|first|morning|breakfast| +25753|AAAAAAAAKJEGAAAA|25753|7|9|13|AM|first|morning|breakfast| +25754|AAAAAAAALJEGAAAA|25754|7|9|14|AM|first|morning|breakfast| +25755|AAAAAAAAMJEGAAAA|25755|7|9|15|AM|first|morning|breakfast| +25756|AAAAAAAANJEGAAAA|25756|7|9|16|AM|first|morning|breakfast| +25757|AAAAAAAAOJEGAAAA|25757|7|9|17|AM|first|morning|breakfast| +25758|AAAAAAAAPJEGAAAA|25758|7|9|18|AM|first|morning|breakfast| +25759|AAAAAAAAAKEGAAAA|25759|7|9|19|AM|first|morning|breakfast| +25760|AAAAAAAABKEGAAAA|25760|7|9|20|AM|first|morning|breakfast| +25761|AAAAAAAACKEGAAAA|25761|7|9|21|AM|first|morning|breakfast| +25762|AAAAAAAADKEGAAAA|25762|7|9|22|AM|first|morning|breakfast| +25763|AAAAAAAAEKEGAAAA|25763|7|9|23|AM|first|morning|breakfast| +25764|AAAAAAAAFKEGAAAA|25764|7|9|24|AM|first|morning|breakfast| +25765|AAAAAAAAGKEGAAAA|25765|7|9|25|AM|first|morning|breakfast| +25766|AAAAAAAAHKEGAAAA|25766|7|9|26|AM|first|morning|breakfast| +25767|AAAAAAAAIKEGAAAA|25767|7|9|27|AM|first|morning|breakfast| +25768|AAAAAAAAJKEGAAAA|25768|7|9|28|AM|first|morning|breakfast| +25769|AAAAAAAAKKEGAAAA|25769|7|9|29|AM|first|morning|breakfast| +25770|AAAAAAAALKEGAAAA|25770|7|9|30|AM|first|morning|breakfast| +25771|AAAAAAAAMKEGAAAA|25771|7|9|31|AM|first|morning|breakfast| +25772|AAAAAAAANKEGAAAA|25772|7|9|32|AM|first|morning|breakfast| +25773|AAAAAAAAOKEGAAAA|25773|7|9|33|AM|first|morning|breakfast| +25774|AAAAAAAAPKEGAAAA|25774|7|9|34|AM|first|morning|breakfast| +25775|AAAAAAAAALEGAAAA|25775|7|9|35|AM|first|morning|breakfast| +25776|AAAAAAAABLEGAAAA|25776|7|9|36|AM|first|morning|breakfast| +25777|AAAAAAAACLEGAAAA|25777|7|9|37|AM|first|morning|breakfast| +25778|AAAAAAAADLEGAAAA|25778|7|9|38|AM|first|morning|breakfast| +25779|AAAAAAAAELEGAAAA|25779|7|9|39|AM|first|morning|breakfast| +25780|AAAAAAAAFLEGAAAA|25780|7|9|40|AM|first|morning|breakfast| +25781|AAAAAAAAGLEGAAAA|25781|7|9|41|AM|first|morning|breakfast| +25782|AAAAAAAAHLEGAAAA|25782|7|9|42|AM|first|morning|breakfast| +25783|AAAAAAAAILEGAAAA|25783|7|9|43|AM|first|morning|breakfast| +25784|AAAAAAAAJLEGAAAA|25784|7|9|44|AM|first|morning|breakfast| +25785|AAAAAAAAKLEGAAAA|25785|7|9|45|AM|first|morning|breakfast| +25786|AAAAAAAALLEGAAAA|25786|7|9|46|AM|first|morning|breakfast| +25787|AAAAAAAAMLEGAAAA|25787|7|9|47|AM|first|morning|breakfast| +25788|AAAAAAAANLEGAAAA|25788|7|9|48|AM|first|morning|breakfast| +25789|AAAAAAAAOLEGAAAA|25789|7|9|49|AM|first|morning|breakfast| +25790|AAAAAAAAPLEGAAAA|25790|7|9|50|AM|first|morning|breakfast| +25791|AAAAAAAAAMEGAAAA|25791|7|9|51|AM|first|morning|breakfast| +25792|AAAAAAAABMEGAAAA|25792|7|9|52|AM|first|morning|breakfast| +25793|AAAAAAAACMEGAAAA|25793|7|9|53|AM|first|morning|breakfast| +25794|AAAAAAAADMEGAAAA|25794|7|9|54|AM|first|morning|breakfast| +25795|AAAAAAAAEMEGAAAA|25795|7|9|55|AM|first|morning|breakfast| +25796|AAAAAAAAFMEGAAAA|25796|7|9|56|AM|first|morning|breakfast| +25797|AAAAAAAAGMEGAAAA|25797|7|9|57|AM|first|morning|breakfast| +25798|AAAAAAAAHMEGAAAA|25798|7|9|58|AM|first|morning|breakfast| +25799|AAAAAAAAIMEGAAAA|25799|7|9|59|AM|first|morning|breakfast| +25800|AAAAAAAAJMEGAAAA|25800|7|10|0|AM|first|morning|breakfast| +25801|AAAAAAAAKMEGAAAA|25801|7|10|1|AM|first|morning|breakfast| +25802|AAAAAAAALMEGAAAA|25802|7|10|2|AM|first|morning|breakfast| +25803|AAAAAAAAMMEGAAAA|25803|7|10|3|AM|first|morning|breakfast| +25804|AAAAAAAANMEGAAAA|25804|7|10|4|AM|first|morning|breakfast| +25805|AAAAAAAAOMEGAAAA|25805|7|10|5|AM|first|morning|breakfast| +25806|AAAAAAAAPMEGAAAA|25806|7|10|6|AM|first|morning|breakfast| +25807|AAAAAAAAANEGAAAA|25807|7|10|7|AM|first|morning|breakfast| +25808|AAAAAAAABNEGAAAA|25808|7|10|8|AM|first|morning|breakfast| +25809|AAAAAAAACNEGAAAA|25809|7|10|9|AM|first|morning|breakfast| +25810|AAAAAAAADNEGAAAA|25810|7|10|10|AM|first|morning|breakfast| +25811|AAAAAAAAENEGAAAA|25811|7|10|11|AM|first|morning|breakfast| +25812|AAAAAAAAFNEGAAAA|25812|7|10|12|AM|first|morning|breakfast| +25813|AAAAAAAAGNEGAAAA|25813|7|10|13|AM|first|morning|breakfast| +25814|AAAAAAAAHNEGAAAA|25814|7|10|14|AM|first|morning|breakfast| +25815|AAAAAAAAINEGAAAA|25815|7|10|15|AM|first|morning|breakfast| +25816|AAAAAAAAJNEGAAAA|25816|7|10|16|AM|first|morning|breakfast| +25817|AAAAAAAAKNEGAAAA|25817|7|10|17|AM|first|morning|breakfast| +25818|AAAAAAAALNEGAAAA|25818|7|10|18|AM|first|morning|breakfast| +25819|AAAAAAAAMNEGAAAA|25819|7|10|19|AM|first|morning|breakfast| +25820|AAAAAAAANNEGAAAA|25820|7|10|20|AM|first|morning|breakfast| +25821|AAAAAAAAONEGAAAA|25821|7|10|21|AM|first|morning|breakfast| +25822|AAAAAAAAPNEGAAAA|25822|7|10|22|AM|first|morning|breakfast| +25823|AAAAAAAAAOEGAAAA|25823|7|10|23|AM|first|morning|breakfast| +25824|AAAAAAAABOEGAAAA|25824|7|10|24|AM|first|morning|breakfast| +25825|AAAAAAAACOEGAAAA|25825|7|10|25|AM|first|morning|breakfast| +25826|AAAAAAAADOEGAAAA|25826|7|10|26|AM|first|morning|breakfast| +25827|AAAAAAAAEOEGAAAA|25827|7|10|27|AM|first|morning|breakfast| +25828|AAAAAAAAFOEGAAAA|25828|7|10|28|AM|first|morning|breakfast| +25829|AAAAAAAAGOEGAAAA|25829|7|10|29|AM|first|morning|breakfast| +25830|AAAAAAAAHOEGAAAA|25830|7|10|30|AM|first|morning|breakfast| +25831|AAAAAAAAIOEGAAAA|25831|7|10|31|AM|first|morning|breakfast| +25832|AAAAAAAAJOEGAAAA|25832|7|10|32|AM|first|morning|breakfast| +25833|AAAAAAAAKOEGAAAA|25833|7|10|33|AM|first|morning|breakfast| +25834|AAAAAAAALOEGAAAA|25834|7|10|34|AM|first|morning|breakfast| +25835|AAAAAAAAMOEGAAAA|25835|7|10|35|AM|first|morning|breakfast| +25836|AAAAAAAANOEGAAAA|25836|7|10|36|AM|first|morning|breakfast| +25837|AAAAAAAAOOEGAAAA|25837|7|10|37|AM|first|morning|breakfast| +25838|AAAAAAAAPOEGAAAA|25838|7|10|38|AM|first|morning|breakfast| +25839|AAAAAAAAAPEGAAAA|25839|7|10|39|AM|first|morning|breakfast| +25840|AAAAAAAABPEGAAAA|25840|7|10|40|AM|first|morning|breakfast| +25841|AAAAAAAACPEGAAAA|25841|7|10|41|AM|first|morning|breakfast| +25842|AAAAAAAADPEGAAAA|25842|7|10|42|AM|first|morning|breakfast| +25843|AAAAAAAAEPEGAAAA|25843|7|10|43|AM|first|morning|breakfast| +25844|AAAAAAAAFPEGAAAA|25844|7|10|44|AM|first|morning|breakfast| +25845|AAAAAAAAGPEGAAAA|25845|7|10|45|AM|first|morning|breakfast| +25846|AAAAAAAAHPEGAAAA|25846|7|10|46|AM|first|morning|breakfast| +25847|AAAAAAAAIPEGAAAA|25847|7|10|47|AM|first|morning|breakfast| +25848|AAAAAAAAJPEGAAAA|25848|7|10|48|AM|first|morning|breakfast| +25849|AAAAAAAAKPEGAAAA|25849|7|10|49|AM|first|morning|breakfast| +25850|AAAAAAAALPEGAAAA|25850|7|10|50|AM|first|morning|breakfast| +25851|AAAAAAAAMPEGAAAA|25851|7|10|51|AM|first|morning|breakfast| +25852|AAAAAAAANPEGAAAA|25852|7|10|52|AM|first|morning|breakfast| +25853|AAAAAAAAOPEGAAAA|25853|7|10|53|AM|first|morning|breakfast| +25854|AAAAAAAAPPEGAAAA|25854|7|10|54|AM|first|morning|breakfast| +25855|AAAAAAAAAAFGAAAA|25855|7|10|55|AM|first|morning|breakfast| +25856|AAAAAAAABAFGAAAA|25856|7|10|56|AM|first|morning|breakfast| +25857|AAAAAAAACAFGAAAA|25857|7|10|57|AM|first|morning|breakfast| +25858|AAAAAAAADAFGAAAA|25858|7|10|58|AM|first|morning|breakfast| +25859|AAAAAAAAEAFGAAAA|25859|7|10|59|AM|first|morning|breakfast| +25860|AAAAAAAAFAFGAAAA|25860|7|11|0|AM|first|morning|breakfast| +25861|AAAAAAAAGAFGAAAA|25861|7|11|1|AM|first|morning|breakfast| +25862|AAAAAAAAHAFGAAAA|25862|7|11|2|AM|first|morning|breakfast| +25863|AAAAAAAAIAFGAAAA|25863|7|11|3|AM|first|morning|breakfast| +25864|AAAAAAAAJAFGAAAA|25864|7|11|4|AM|first|morning|breakfast| +25865|AAAAAAAAKAFGAAAA|25865|7|11|5|AM|first|morning|breakfast| +25866|AAAAAAAALAFGAAAA|25866|7|11|6|AM|first|morning|breakfast| +25867|AAAAAAAAMAFGAAAA|25867|7|11|7|AM|first|morning|breakfast| +25868|AAAAAAAANAFGAAAA|25868|7|11|8|AM|first|morning|breakfast| +25869|AAAAAAAAOAFGAAAA|25869|7|11|9|AM|first|morning|breakfast| +25870|AAAAAAAAPAFGAAAA|25870|7|11|10|AM|first|morning|breakfast| +25871|AAAAAAAAABFGAAAA|25871|7|11|11|AM|first|morning|breakfast| +25872|AAAAAAAABBFGAAAA|25872|7|11|12|AM|first|morning|breakfast| +25873|AAAAAAAACBFGAAAA|25873|7|11|13|AM|first|morning|breakfast| +25874|AAAAAAAADBFGAAAA|25874|7|11|14|AM|first|morning|breakfast| +25875|AAAAAAAAEBFGAAAA|25875|7|11|15|AM|first|morning|breakfast| +25876|AAAAAAAAFBFGAAAA|25876|7|11|16|AM|first|morning|breakfast| +25877|AAAAAAAAGBFGAAAA|25877|7|11|17|AM|first|morning|breakfast| +25878|AAAAAAAAHBFGAAAA|25878|7|11|18|AM|first|morning|breakfast| +25879|AAAAAAAAIBFGAAAA|25879|7|11|19|AM|first|morning|breakfast| +25880|AAAAAAAAJBFGAAAA|25880|7|11|20|AM|first|morning|breakfast| +25881|AAAAAAAAKBFGAAAA|25881|7|11|21|AM|first|morning|breakfast| +25882|AAAAAAAALBFGAAAA|25882|7|11|22|AM|first|morning|breakfast| +25883|AAAAAAAAMBFGAAAA|25883|7|11|23|AM|first|morning|breakfast| +25884|AAAAAAAANBFGAAAA|25884|7|11|24|AM|first|morning|breakfast| +25885|AAAAAAAAOBFGAAAA|25885|7|11|25|AM|first|morning|breakfast| +25886|AAAAAAAAPBFGAAAA|25886|7|11|26|AM|first|morning|breakfast| +25887|AAAAAAAAACFGAAAA|25887|7|11|27|AM|first|morning|breakfast| +25888|AAAAAAAABCFGAAAA|25888|7|11|28|AM|first|morning|breakfast| +25889|AAAAAAAACCFGAAAA|25889|7|11|29|AM|first|morning|breakfast| +25890|AAAAAAAADCFGAAAA|25890|7|11|30|AM|first|morning|breakfast| +25891|AAAAAAAAECFGAAAA|25891|7|11|31|AM|first|morning|breakfast| +25892|AAAAAAAAFCFGAAAA|25892|7|11|32|AM|first|morning|breakfast| +25893|AAAAAAAAGCFGAAAA|25893|7|11|33|AM|first|morning|breakfast| +25894|AAAAAAAAHCFGAAAA|25894|7|11|34|AM|first|morning|breakfast| +25895|AAAAAAAAICFGAAAA|25895|7|11|35|AM|first|morning|breakfast| +25896|AAAAAAAAJCFGAAAA|25896|7|11|36|AM|first|morning|breakfast| +25897|AAAAAAAAKCFGAAAA|25897|7|11|37|AM|first|morning|breakfast| +25898|AAAAAAAALCFGAAAA|25898|7|11|38|AM|first|morning|breakfast| +25899|AAAAAAAAMCFGAAAA|25899|7|11|39|AM|first|morning|breakfast| +25900|AAAAAAAANCFGAAAA|25900|7|11|40|AM|first|morning|breakfast| +25901|AAAAAAAAOCFGAAAA|25901|7|11|41|AM|first|morning|breakfast| +25902|AAAAAAAAPCFGAAAA|25902|7|11|42|AM|first|morning|breakfast| +25903|AAAAAAAAADFGAAAA|25903|7|11|43|AM|first|morning|breakfast| +25904|AAAAAAAABDFGAAAA|25904|7|11|44|AM|first|morning|breakfast| +25905|AAAAAAAACDFGAAAA|25905|7|11|45|AM|first|morning|breakfast| +25906|AAAAAAAADDFGAAAA|25906|7|11|46|AM|first|morning|breakfast| +25907|AAAAAAAAEDFGAAAA|25907|7|11|47|AM|first|morning|breakfast| +25908|AAAAAAAAFDFGAAAA|25908|7|11|48|AM|first|morning|breakfast| +25909|AAAAAAAAGDFGAAAA|25909|7|11|49|AM|first|morning|breakfast| +25910|AAAAAAAAHDFGAAAA|25910|7|11|50|AM|first|morning|breakfast| +25911|AAAAAAAAIDFGAAAA|25911|7|11|51|AM|first|morning|breakfast| +25912|AAAAAAAAJDFGAAAA|25912|7|11|52|AM|first|morning|breakfast| +25913|AAAAAAAAKDFGAAAA|25913|7|11|53|AM|first|morning|breakfast| +25914|AAAAAAAALDFGAAAA|25914|7|11|54|AM|first|morning|breakfast| +25915|AAAAAAAAMDFGAAAA|25915|7|11|55|AM|first|morning|breakfast| +25916|AAAAAAAANDFGAAAA|25916|7|11|56|AM|first|morning|breakfast| +25917|AAAAAAAAODFGAAAA|25917|7|11|57|AM|first|morning|breakfast| +25918|AAAAAAAAPDFGAAAA|25918|7|11|58|AM|first|morning|breakfast| +25919|AAAAAAAAAEFGAAAA|25919|7|11|59|AM|first|morning|breakfast| +25920|AAAAAAAABEFGAAAA|25920|7|12|0|AM|first|morning|breakfast| +25921|AAAAAAAACEFGAAAA|25921|7|12|1|AM|first|morning|breakfast| +25922|AAAAAAAADEFGAAAA|25922|7|12|2|AM|first|morning|breakfast| +25923|AAAAAAAAEEFGAAAA|25923|7|12|3|AM|first|morning|breakfast| +25924|AAAAAAAAFEFGAAAA|25924|7|12|4|AM|first|morning|breakfast| +25925|AAAAAAAAGEFGAAAA|25925|7|12|5|AM|first|morning|breakfast| +25926|AAAAAAAAHEFGAAAA|25926|7|12|6|AM|first|morning|breakfast| +25927|AAAAAAAAIEFGAAAA|25927|7|12|7|AM|first|morning|breakfast| +25928|AAAAAAAAJEFGAAAA|25928|7|12|8|AM|first|morning|breakfast| +25929|AAAAAAAAKEFGAAAA|25929|7|12|9|AM|first|morning|breakfast| +25930|AAAAAAAALEFGAAAA|25930|7|12|10|AM|first|morning|breakfast| +25931|AAAAAAAAMEFGAAAA|25931|7|12|11|AM|first|morning|breakfast| +25932|AAAAAAAANEFGAAAA|25932|7|12|12|AM|first|morning|breakfast| +25933|AAAAAAAAOEFGAAAA|25933|7|12|13|AM|first|morning|breakfast| +25934|AAAAAAAAPEFGAAAA|25934|7|12|14|AM|first|morning|breakfast| +25935|AAAAAAAAAFFGAAAA|25935|7|12|15|AM|first|morning|breakfast| +25936|AAAAAAAABFFGAAAA|25936|7|12|16|AM|first|morning|breakfast| +25937|AAAAAAAACFFGAAAA|25937|7|12|17|AM|first|morning|breakfast| +25938|AAAAAAAADFFGAAAA|25938|7|12|18|AM|first|morning|breakfast| +25939|AAAAAAAAEFFGAAAA|25939|7|12|19|AM|first|morning|breakfast| +25940|AAAAAAAAFFFGAAAA|25940|7|12|20|AM|first|morning|breakfast| +25941|AAAAAAAAGFFGAAAA|25941|7|12|21|AM|first|morning|breakfast| +25942|AAAAAAAAHFFGAAAA|25942|7|12|22|AM|first|morning|breakfast| +25943|AAAAAAAAIFFGAAAA|25943|7|12|23|AM|first|morning|breakfast| +25944|AAAAAAAAJFFGAAAA|25944|7|12|24|AM|first|morning|breakfast| +25945|AAAAAAAAKFFGAAAA|25945|7|12|25|AM|first|morning|breakfast| +25946|AAAAAAAALFFGAAAA|25946|7|12|26|AM|first|morning|breakfast| +25947|AAAAAAAAMFFGAAAA|25947|7|12|27|AM|first|morning|breakfast| +25948|AAAAAAAANFFGAAAA|25948|7|12|28|AM|first|morning|breakfast| +25949|AAAAAAAAOFFGAAAA|25949|7|12|29|AM|first|morning|breakfast| +25950|AAAAAAAAPFFGAAAA|25950|7|12|30|AM|first|morning|breakfast| +25951|AAAAAAAAAGFGAAAA|25951|7|12|31|AM|first|morning|breakfast| +25952|AAAAAAAABGFGAAAA|25952|7|12|32|AM|first|morning|breakfast| +25953|AAAAAAAACGFGAAAA|25953|7|12|33|AM|first|morning|breakfast| +25954|AAAAAAAADGFGAAAA|25954|7|12|34|AM|first|morning|breakfast| +25955|AAAAAAAAEGFGAAAA|25955|7|12|35|AM|first|morning|breakfast| +25956|AAAAAAAAFGFGAAAA|25956|7|12|36|AM|first|morning|breakfast| +25957|AAAAAAAAGGFGAAAA|25957|7|12|37|AM|first|morning|breakfast| +25958|AAAAAAAAHGFGAAAA|25958|7|12|38|AM|first|morning|breakfast| +25959|AAAAAAAAIGFGAAAA|25959|7|12|39|AM|first|morning|breakfast| +25960|AAAAAAAAJGFGAAAA|25960|7|12|40|AM|first|morning|breakfast| +25961|AAAAAAAAKGFGAAAA|25961|7|12|41|AM|first|morning|breakfast| +25962|AAAAAAAALGFGAAAA|25962|7|12|42|AM|first|morning|breakfast| +25963|AAAAAAAAMGFGAAAA|25963|7|12|43|AM|first|morning|breakfast| +25964|AAAAAAAANGFGAAAA|25964|7|12|44|AM|first|morning|breakfast| +25965|AAAAAAAAOGFGAAAA|25965|7|12|45|AM|first|morning|breakfast| +25966|AAAAAAAAPGFGAAAA|25966|7|12|46|AM|first|morning|breakfast| +25967|AAAAAAAAAHFGAAAA|25967|7|12|47|AM|first|morning|breakfast| +25968|AAAAAAAABHFGAAAA|25968|7|12|48|AM|first|morning|breakfast| +25969|AAAAAAAACHFGAAAA|25969|7|12|49|AM|first|morning|breakfast| +25970|AAAAAAAADHFGAAAA|25970|7|12|50|AM|first|morning|breakfast| +25971|AAAAAAAAEHFGAAAA|25971|7|12|51|AM|first|morning|breakfast| +25972|AAAAAAAAFHFGAAAA|25972|7|12|52|AM|first|morning|breakfast| +25973|AAAAAAAAGHFGAAAA|25973|7|12|53|AM|first|morning|breakfast| +25974|AAAAAAAAHHFGAAAA|25974|7|12|54|AM|first|morning|breakfast| +25975|AAAAAAAAIHFGAAAA|25975|7|12|55|AM|first|morning|breakfast| +25976|AAAAAAAAJHFGAAAA|25976|7|12|56|AM|first|morning|breakfast| +25977|AAAAAAAAKHFGAAAA|25977|7|12|57|AM|first|morning|breakfast| +25978|AAAAAAAALHFGAAAA|25978|7|12|58|AM|first|morning|breakfast| +25979|AAAAAAAAMHFGAAAA|25979|7|12|59|AM|first|morning|breakfast| +25980|AAAAAAAANHFGAAAA|25980|7|13|0|AM|first|morning|breakfast| +25981|AAAAAAAAOHFGAAAA|25981|7|13|1|AM|first|morning|breakfast| +25982|AAAAAAAAPHFGAAAA|25982|7|13|2|AM|first|morning|breakfast| +25983|AAAAAAAAAIFGAAAA|25983|7|13|3|AM|first|morning|breakfast| +25984|AAAAAAAABIFGAAAA|25984|7|13|4|AM|first|morning|breakfast| +25985|AAAAAAAACIFGAAAA|25985|7|13|5|AM|first|morning|breakfast| +25986|AAAAAAAADIFGAAAA|25986|7|13|6|AM|first|morning|breakfast| +25987|AAAAAAAAEIFGAAAA|25987|7|13|7|AM|first|morning|breakfast| +25988|AAAAAAAAFIFGAAAA|25988|7|13|8|AM|first|morning|breakfast| +25989|AAAAAAAAGIFGAAAA|25989|7|13|9|AM|first|morning|breakfast| +25990|AAAAAAAAHIFGAAAA|25990|7|13|10|AM|first|morning|breakfast| +25991|AAAAAAAAIIFGAAAA|25991|7|13|11|AM|first|morning|breakfast| +25992|AAAAAAAAJIFGAAAA|25992|7|13|12|AM|first|morning|breakfast| +25993|AAAAAAAAKIFGAAAA|25993|7|13|13|AM|first|morning|breakfast| +25994|AAAAAAAALIFGAAAA|25994|7|13|14|AM|first|morning|breakfast| +25995|AAAAAAAAMIFGAAAA|25995|7|13|15|AM|first|morning|breakfast| +25996|AAAAAAAANIFGAAAA|25996|7|13|16|AM|first|morning|breakfast| +25997|AAAAAAAAOIFGAAAA|25997|7|13|17|AM|first|morning|breakfast| +25998|AAAAAAAAPIFGAAAA|25998|7|13|18|AM|first|morning|breakfast| +25999|AAAAAAAAAJFGAAAA|25999|7|13|19|AM|first|morning|breakfast| +26000|AAAAAAAABJFGAAAA|26000|7|13|20|AM|first|morning|breakfast| +26001|AAAAAAAACJFGAAAA|26001|7|13|21|AM|first|morning|breakfast| +26002|AAAAAAAADJFGAAAA|26002|7|13|22|AM|first|morning|breakfast| +26003|AAAAAAAAEJFGAAAA|26003|7|13|23|AM|first|morning|breakfast| +26004|AAAAAAAAFJFGAAAA|26004|7|13|24|AM|first|morning|breakfast| +26005|AAAAAAAAGJFGAAAA|26005|7|13|25|AM|first|morning|breakfast| +26006|AAAAAAAAHJFGAAAA|26006|7|13|26|AM|first|morning|breakfast| +26007|AAAAAAAAIJFGAAAA|26007|7|13|27|AM|first|morning|breakfast| +26008|AAAAAAAAJJFGAAAA|26008|7|13|28|AM|first|morning|breakfast| +26009|AAAAAAAAKJFGAAAA|26009|7|13|29|AM|first|morning|breakfast| +26010|AAAAAAAALJFGAAAA|26010|7|13|30|AM|first|morning|breakfast| +26011|AAAAAAAAMJFGAAAA|26011|7|13|31|AM|first|morning|breakfast| +26012|AAAAAAAANJFGAAAA|26012|7|13|32|AM|first|morning|breakfast| +26013|AAAAAAAAOJFGAAAA|26013|7|13|33|AM|first|morning|breakfast| +26014|AAAAAAAAPJFGAAAA|26014|7|13|34|AM|first|morning|breakfast| +26015|AAAAAAAAAKFGAAAA|26015|7|13|35|AM|first|morning|breakfast| +26016|AAAAAAAABKFGAAAA|26016|7|13|36|AM|first|morning|breakfast| +26017|AAAAAAAACKFGAAAA|26017|7|13|37|AM|first|morning|breakfast| +26018|AAAAAAAADKFGAAAA|26018|7|13|38|AM|first|morning|breakfast| +26019|AAAAAAAAEKFGAAAA|26019|7|13|39|AM|first|morning|breakfast| +26020|AAAAAAAAFKFGAAAA|26020|7|13|40|AM|first|morning|breakfast| +26021|AAAAAAAAGKFGAAAA|26021|7|13|41|AM|first|morning|breakfast| +26022|AAAAAAAAHKFGAAAA|26022|7|13|42|AM|first|morning|breakfast| +26023|AAAAAAAAIKFGAAAA|26023|7|13|43|AM|first|morning|breakfast| +26024|AAAAAAAAJKFGAAAA|26024|7|13|44|AM|first|morning|breakfast| +26025|AAAAAAAAKKFGAAAA|26025|7|13|45|AM|first|morning|breakfast| +26026|AAAAAAAALKFGAAAA|26026|7|13|46|AM|first|morning|breakfast| +26027|AAAAAAAAMKFGAAAA|26027|7|13|47|AM|first|morning|breakfast| +26028|AAAAAAAANKFGAAAA|26028|7|13|48|AM|first|morning|breakfast| +26029|AAAAAAAAOKFGAAAA|26029|7|13|49|AM|first|morning|breakfast| +26030|AAAAAAAAPKFGAAAA|26030|7|13|50|AM|first|morning|breakfast| +26031|AAAAAAAAALFGAAAA|26031|7|13|51|AM|first|morning|breakfast| +26032|AAAAAAAABLFGAAAA|26032|7|13|52|AM|first|morning|breakfast| +26033|AAAAAAAACLFGAAAA|26033|7|13|53|AM|first|morning|breakfast| +26034|AAAAAAAADLFGAAAA|26034|7|13|54|AM|first|morning|breakfast| +26035|AAAAAAAAELFGAAAA|26035|7|13|55|AM|first|morning|breakfast| +26036|AAAAAAAAFLFGAAAA|26036|7|13|56|AM|first|morning|breakfast| +26037|AAAAAAAAGLFGAAAA|26037|7|13|57|AM|first|morning|breakfast| +26038|AAAAAAAAHLFGAAAA|26038|7|13|58|AM|first|morning|breakfast| +26039|AAAAAAAAILFGAAAA|26039|7|13|59|AM|first|morning|breakfast| +26040|AAAAAAAAJLFGAAAA|26040|7|14|0|AM|first|morning|breakfast| +26041|AAAAAAAAKLFGAAAA|26041|7|14|1|AM|first|morning|breakfast| +26042|AAAAAAAALLFGAAAA|26042|7|14|2|AM|first|morning|breakfast| +26043|AAAAAAAAMLFGAAAA|26043|7|14|3|AM|first|morning|breakfast| +26044|AAAAAAAANLFGAAAA|26044|7|14|4|AM|first|morning|breakfast| +26045|AAAAAAAAOLFGAAAA|26045|7|14|5|AM|first|morning|breakfast| +26046|AAAAAAAAPLFGAAAA|26046|7|14|6|AM|first|morning|breakfast| +26047|AAAAAAAAAMFGAAAA|26047|7|14|7|AM|first|morning|breakfast| +26048|AAAAAAAABMFGAAAA|26048|7|14|8|AM|first|morning|breakfast| +26049|AAAAAAAACMFGAAAA|26049|7|14|9|AM|first|morning|breakfast| +26050|AAAAAAAADMFGAAAA|26050|7|14|10|AM|first|morning|breakfast| +26051|AAAAAAAAEMFGAAAA|26051|7|14|11|AM|first|morning|breakfast| +26052|AAAAAAAAFMFGAAAA|26052|7|14|12|AM|first|morning|breakfast| +26053|AAAAAAAAGMFGAAAA|26053|7|14|13|AM|first|morning|breakfast| +26054|AAAAAAAAHMFGAAAA|26054|7|14|14|AM|first|morning|breakfast| +26055|AAAAAAAAIMFGAAAA|26055|7|14|15|AM|first|morning|breakfast| +26056|AAAAAAAAJMFGAAAA|26056|7|14|16|AM|first|morning|breakfast| +26057|AAAAAAAAKMFGAAAA|26057|7|14|17|AM|first|morning|breakfast| +26058|AAAAAAAALMFGAAAA|26058|7|14|18|AM|first|morning|breakfast| +26059|AAAAAAAAMMFGAAAA|26059|7|14|19|AM|first|morning|breakfast| +26060|AAAAAAAANMFGAAAA|26060|7|14|20|AM|first|morning|breakfast| +26061|AAAAAAAAOMFGAAAA|26061|7|14|21|AM|first|morning|breakfast| +26062|AAAAAAAAPMFGAAAA|26062|7|14|22|AM|first|morning|breakfast| +26063|AAAAAAAAANFGAAAA|26063|7|14|23|AM|first|morning|breakfast| +26064|AAAAAAAABNFGAAAA|26064|7|14|24|AM|first|morning|breakfast| +26065|AAAAAAAACNFGAAAA|26065|7|14|25|AM|first|morning|breakfast| +26066|AAAAAAAADNFGAAAA|26066|7|14|26|AM|first|morning|breakfast| +26067|AAAAAAAAENFGAAAA|26067|7|14|27|AM|first|morning|breakfast| +26068|AAAAAAAAFNFGAAAA|26068|7|14|28|AM|first|morning|breakfast| +26069|AAAAAAAAGNFGAAAA|26069|7|14|29|AM|first|morning|breakfast| +26070|AAAAAAAAHNFGAAAA|26070|7|14|30|AM|first|morning|breakfast| +26071|AAAAAAAAINFGAAAA|26071|7|14|31|AM|first|morning|breakfast| +26072|AAAAAAAAJNFGAAAA|26072|7|14|32|AM|first|morning|breakfast| +26073|AAAAAAAAKNFGAAAA|26073|7|14|33|AM|first|morning|breakfast| +26074|AAAAAAAALNFGAAAA|26074|7|14|34|AM|first|morning|breakfast| +26075|AAAAAAAAMNFGAAAA|26075|7|14|35|AM|first|morning|breakfast| +26076|AAAAAAAANNFGAAAA|26076|7|14|36|AM|first|morning|breakfast| +26077|AAAAAAAAONFGAAAA|26077|7|14|37|AM|first|morning|breakfast| +26078|AAAAAAAAPNFGAAAA|26078|7|14|38|AM|first|morning|breakfast| +26079|AAAAAAAAAOFGAAAA|26079|7|14|39|AM|first|morning|breakfast| +26080|AAAAAAAABOFGAAAA|26080|7|14|40|AM|first|morning|breakfast| +26081|AAAAAAAACOFGAAAA|26081|7|14|41|AM|first|morning|breakfast| +26082|AAAAAAAADOFGAAAA|26082|7|14|42|AM|first|morning|breakfast| +26083|AAAAAAAAEOFGAAAA|26083|7|14|43|AM|first|morning|breakfast| +26084|AAAAAAAAFOFGAAAA|26084|7|14|44|AM|first|morning|breakfast| +26085|AAAAAAAAGOFGAAAA|26085|7|14|45|AM|first|morning|breakfast| +26086|AAAAAAAAHOFGAAAA|26086|7|14|46|AM|first|morning|breakfast| +26087|AAAAAAAAIOFGAAAA|26087|7|14|47|AM|first|morning|breakfast| +26088|AAAAAAAAJOFGAAAA|26088|7|14|48|AM|first|morning|breakfast| +26089|AAAAAAAAKOFGAAAA|26089|7|14|49|AM|first|morning|breakfast| +26090|AAAAAAAALOFGAAAA|26090|7|14|50|AM|first|morning|breakfast| +26091|AAAAAAAAMOFGAAAA|26091|7|14|51|AM|first|morning|breakfast| +26092|AAAAAAAANOFGAAAA|26092|7|14|52|AM|first|morning|breakfast| +26093|AAAAAAAAOOFGAAAA|26093|7|14|53|AM|first|morning|breakfast| +26094|AAAAAAAAPOFGAAAA|26094|7|14|54|AM|first|morning|breakfast| +26095|AAAAAAAAAPFGAAAA|26095|7|14|55|AM|first|morning|breakfast| +26096|AAAAAAAABPFGAAAA|26096|7|14|56|AM|first|morning|breakfast| +26097|AAAAAAAACPFGAAAA|26097|7|14|57|AM|first|morning|breakfast| +26098|AAAAAAAADPFGAAAA|26098|7|14|58|AM|first|morning|breakfast| +26099|AAAAAAAAEPFGAAAA|26099|7|14|59|AM|first|morning|breakfast| +26100|AAAAAAAAFPFGAAAA|26100|7|15|0|AM|first|morning|breakfast| +26101|AAAAAAAAGPFGAAAA|26101|7|15|1|AM|first|morning|breakfast| +26102|AAAAAAAAHPFGAAAA|26102|7|15|2|AM|first|morning|breakfast| +26103|AAAAAAAAIPFGAAAA|26103|7|15|3|AM|first|morning|breakfast| +26104|AAAAAAAAJPFGAAAA|26104|7|15|4|AM|first|morning|breakfast| +26105|AAAAAAAAKPFGAAAA|26105|7|15|5|AM|first|morning|breakfast| +26106|AAAAAAAALPFGAAAA|26106|7|15|6|AM|first|morning|breakfast| +26107|AAAAAAAAMPFGAAAA|26107|7|15|7|AM|first|morning|breakfast| +26108|AAAAAAAANPFGAAAA|26108|7|15|8|AM|first|morning|breakfast| +26109|AAAAAAAAOPFGAAAA|26109|7|15|9|AM|first|morning|breakfast| +26110|AAAAAAAAPPFGAAAA|26110|7|15|10|AM|first|morning|breakfast| +26111|AAAAAAAAAAGGAAAA|26111|7|15|11|AM|first|morning|breakfast| +26112|AAAAAAAABAGGAAAA|26112|7|15|12|AM|first|morning|breakfast| +26113|AAAAAAAACAGGAAAA|26113|7|15|13|AM|first|morning|breakfast| +26114|AAAAAAAADAGGAAAA|26114|7|15|14|AM|first|morning|breakfast| +26115|AAAAAAAAEAGGAAAA|26115|7|15|15|AM|first|morning|breakfast| +26116|AAAAAAAAFAGGAAAA|26116|7|15|16|AM|first|morning|breakfast| +26117|AAAAAAAAGAGGAAAA|26117|7|15|17|AM|first|morning|breakfast| +26118|AAAAAAAAHAGGAAAA|26118|7|15|18|AM|first|morning|breakfast| +26119|AAAAAAAAIAGGAAAA|26119|7|15|19|AM|first|morning|breakfast| +26120|AAAAAAAAJAGGAAAA|26120|7|15|20|AM|first|morning|breakfast| +26121|AAAAAAAAKAGGAAAA|26121|7|15|21|AM|first|morning|breakfast| +26122|AAAAAAAALAGGAAAA|26122|7|15|22|AM|first|morning|breakfast| +26123|AAAAAAAAMAGGAAAA|26123|7|15|23|AM|first|morning|breakfast| +26124|AAAAAAAANAGGAAAA|26124|7|15|24|AM|first|morning|breakfast| +26125|AAAAAAAAOAGGAAAA|26125|7|15|25|AM|first|morning|breakfast| +26126|AAAAAAAAPAGGAAAA|26126|7|15|26|AM|first|morning|breakfast| +26127|AAAAAAAAABGGAAAA|26127|7|15|27|AM|first|morning|breakfast| +26128|AAAAAAAABBGGAAAA|26128|7|15|28|AM|first|morning|breakfast| +26129|AAAAAAAACBGGAAAA|26129|7|15|29|AM|first|morning|breakfast| +26130|AAAAAAAADBGGAAAA|26130|7|15|30|AM|first|morning|breakfast| +26131|AAAAAAAAEBGGAAAA|26131|7|15|31|AM|first|morning|breakfast| +26132|AAAAAAAAFBGGAAAA|26132|7|15|32|AM|first|morning|breakfast| +26133|AAAAAAAAGBGGAAAA|26133|7|15|33|AM|first|morning|breakfast| +26134|AAAAAAAAHBGGAAAA|26134|7|15|34|AM|first|morning|breakfast| +26135|AAAAAAAAIBGGAAAA|26135|7|15|35|AM|first|morning|breakfast| +26136|AAAAAAAAJBGGAAAA|26136|7|15|36|AM|first|morning|breakfast| +26137|AAAAAAAAKBGGAAAA|26137|7|15|37|AM|first|morning|breakfast| +26138|AAAAAAAALBGGAAAA|26138|7|15|38|AM|first|morning|breakfast| +26139|AAAAAAAAMBGGAAAA|26139|7|15|39|AM|first|morning|breakfast| +26140|AAAAAAAANBGGAAAA|26140|7|15|40|AM|first|morning|breakfast| +26141|AAAAAAAAOBGGAAAA|26141|7|15|41|AM|first|morning|breakfast| +26142|AAAAAAAAPBGGAAAA|26142|7|15|42|AM|first|morning|breakfast| +26143|AAAAAAAAACGGAAAA|26143|7|15|43|AM|first|morning|breakfast| +26144|AAAAAAAABCGGAAAA|26144|7|15|44|AM|first|morning|breakfast| +26145|AAAAAAAACCGGAAAA|26145|7|15|45|AM|first|morning|breakfast| +26146|AAAAAAAADCGGAAAA|26146|7|15|46|AM|first|morning|breakfast| +26147|AAAAAAAAECGGAAAA|26147|7|15|47|AM|first|morning|breakfast| +26148|AAAAAAAAFCGGAAAA|26148|7|15|48|AM|first|morning|breakfast| +26149|AAAAAAAAGCGGAAAA|26149|7|15|49|AM|first|morning|breakfast| +26150|AAAAAAAAHCGGAAAA|26150|7|15|50|AM|first|morning|breakfast| +26151|AAAAAAAAICGGAAAA|26151|7|15|51|AM|first|morning|breakfast| +26152|AAAAAAAAJCGGAAAA|26152|7|15|52|AM|first|morning|breakfast| +26153|AAAAAAAAKCGGAAAA|26153|7|15|53|AM|first|morning|breakfast| +26154|AAAAAAAALCGGAAAA|26154|7|15|54|AM|first|morning|breakfast| +26155|AAAAAAAAMCGGAAAA|26155|7|15|55|AM|first|morning|breakfast| +26156|AAAAAAAANCGGAAAA|26156|7|15|56|AM|first|morning|breakfast| +26157|AAAAAAAAOCGGAAAA|26157|7|15|57|AM|first|morning|breakfast| +26158|AAAAAAAAPCGGAAAA|26158|7|15|58|AM|first|morning|breakfast| +26159|AAAAAAAAADGGAAAA|26159|7|15|59|AM|first|morning|breakfast| +26160|AAAAAAAABDGGAAAA|26160|7|16|0|AM|first|morning|breakfast| +26161|AAAAAAAACDGGAAAA|26161|7|16|1|AM|first|morning|breakfast| +26162|AAAAAAAADDGGAAAA|26162|7|16|2|AM|first|morning|breakfast| +26163|AAAAAAAAEDGGAAAA|26163|7|16|3|AM|first|morning|breakfast| +26164|AAAAAAAAFDGGAAAA|26164|7|16|4|AM|first|morning|breakfast| +26165|AAAAAAAAGDGGAAAA|26165|7|16|5|AM|first|morning|breakfast| +26166|AAAAAAAAHDGGAAAA|26166|7|16|6|AM|first|morning|breakfast| +26167|AAAAAAAAIDGGAAAA|26167|7|16|7|AM|first|morning|breakfast| +26168|AAAAAAAAJDGGAAAA|26168|7|16|8|AM|first|morning|breakfast| +26169|AAAAAAAAKDGGAAAA|26169|7|16|9|AM|first|morning|breakfast| +26170|AAAAAAAALDGGAAAA|26170|7|16|10|AM|first|morning|breakfast| +26171|AAAAAAAAMDGGAAAA|26171|7|16|11|AM|first|morning|breakfast| +26172|AAAAAAAANDGGAAAA|26172|7|16|12|AM|first|morning|breakfast| +26173|AAAAAAAAODGGAAAA|26173|7|16|13|AM|first|morning|breakfast| +26174|AAAAAAAAPDGGAAAA|26174|7|16|14|AM|first|morning|breakfast| +26175|AAAAAAAAAEGGAAAA|26175|7|16|15|AM|first|morning|breakfast| +26176|AAAAAAAABEGGAAAA|26176|7|16|16|AM|first|morning|breakfast| +26177|AAAAAAAACEGGAAAA|26177|7|16|17|AM|first|morning|breakfast| +26178|AAAAAAAADEGGAAAA|26178|7|16|18|AM|first|morning|breakfast| +26179|AAAAAAAAEEGGAAAA|26179|7|16|19|AM|first|morning|breakfast| +26180|AAAAAAAAFEGGAAAA|26180|7|16|20|AM|first|morning|breakfast| +26181|AAAAAAAAGEGGAAAA|26181|7|16|21|AM|first|morning|breakfast| +26182|AAAAAAAAHEGGAAAA|26182|7|16|22|AM|first|morning|breakfast| +26183|AAAAAAAAIEGGAAAA|26183|7|16|23|AM|first|morning|breakfast| +26184|AAAAAAAAJEGGAAAA|26184|7|16|24|AM|first|morning|breakfast| +26185|AAAAAAAAKEGGAAAA|26185|7|16|25|AM|first|morning|breakfast| +26186|AAAAAAAALEGGAAAA|26186|7|16|26|AM|first|morning|breakfast| +26187|AAAAAAAAMEGGAAAA|26187|7|16|27|AM|first|morning|breakfast| +26188|AAAAAAAANEGGAAAA|26188|7|16|28|AM|first|morning|breakfast| +26189|AAAAAAAAOEGGAAAA|26189|7|16|29|AM|first|morning|breakfast| +26190|AAAAAAAAPEGGAAAA|26190|7|16|30|AM|first|morning|breakfast| +26191|AAAAAAAAAFGGAAAA|26191|7|16|31|AM|first|morning|breakfast| +26192|AAAAAAAABFGGAAAA|26192|7|16|32|AM|first|morning|breakfast| +26193|AAAAAAAACFGGAAAA|26193|7|16|33|AM|first|morning|breakfast| +26194|AAAAAAAADFGGAAAA|26194|7|16|34|AM|first|morning|breakfast| +26195|AAAAAAAAEFGGAAAA|26195|7|16|35|AM|first|morning|breakfast| +26196|AAAAAAAAFFGGAAAA|26196|7|16|36|AM|first|morning|breakfast| +26197|AAAAAAAAGFGGAAAA|26197|7|16|37|AM|first|morning|breakfast| +26198|AAAAAAAAHFGGAAAA|26198|7|16|38|AM|first|morning|breakfast| +26199|AAAAAAAAIFGGAAAA|26199|7|16|39|AM|first|morning|breakfast| +26200|AAAAAAAAJFGGAAAA|26200|7|16|40|AM|first|morning|breakfast| +26201|AAAAAAAAKFGGAAAA|26201|7|16|41|AM|first|morning|breakfast| +26202|AAAAAAAALFGGAAAA|26202|7|16|42|AM|first|morning|breakfast| +26203|AAAAAAAAMFGGAAAA|26203|7|16|43|AM|first|morning|breakfast| +26204|AAAAAAAANFGGAAAA|26204|7|16|44|AM|first|morning|breakfast| +26205|AAAAAAAAOFGGAAAA|26205|7|16|45|AM|first|morning|breakfast| +26206|AAAAAAAAPFGGAAAA|26206|7|16|46|AM|first|morning|breakfast| +26207|AAAAAAAAAGGGAAAA|26207|7|16|47|AM|first|morning|breakfast| +26208|AAAAAAAABGGGAAAA|26208|7|16|48|AM|first|morning|breakfast| +26209|AAAAAAAACGGGAAAA|26209|7|16|49|AM|first|morning|breakfast| +26210|AAAAAAAADGGGAAAA|26210|7|16|50|AM|first|morning|breakfast| +26211|AAAAAAAAEGGGAAAA|26211|7|16|51|AM|first|morning|breakfast| +26212|AAAAAAAAFGGGAAAA|26212|7|16|52|AM|first|morning|breakfast| +26213|AAAAAAAAGGGGAAAA|26213|7|16|53|AM|first|morning|breakfast| +26214|AAAAAAAAHGGGAAAA|26214|7|16|54|AM|first|morning|breakfast| +26215|AAAAAAAAIGGGAAAA|26215|7|16|55|AM|first|morning|breakfast| +26216|AAAAAAAAJGGGAAAA|26216|7|16|56|AM|first|morning|breakfast| +26217|AAAAAAAAKGGGAAAA|26217|7|16|57|AM|first|morning|breakfast| +26218|AAAAAAAALGGGAAAA|26218|7|16|58|AM|first|morning|breakfast| +26219|AAAAAAAAMGGGAAAA|26219|7|16|59|AM|first|morning|breakfast| +26220|AAAAAAAANGGGAAAA|26220|7|17|0|AM|first|morning|breakfast| +26221|AAAAAAAAOGGGAAAA|26221|7|17|1|AM|first|morning|breakfast| +26222|AAAAAAAAPGGGAAAA|26222|7|17|2|AM|first|morning|breakfast| +26223|AAAAAAAAAHGGAAAA|26223|7|17|3|AM|first|morning|breakfast| +26224|AAAAAAAABHGGAAAA|26224|7|17|4|AM|first|morning|breakfast| +26225|AAAAAAAACHGGAAAA|26225|7|17|5|AM|first|morning|breakfast| +26226|AAAAAAAADHGGAAAA|26226|7|17|6|AM|first|morning|breakfast| +26227|AAAAAAAAEHGGAAAA|26227|7|17|7|AM|first|morning|breakfast| +26228|AAAAAAAAFHGGAAAA|26228|7|17|8|AM|first|morning|breakfast| +26229|AAAAAAAAGHGGAAAA|26229|7|17|9|AM|first|morning|breakfast| +26230|AAAAAAAAHHGGAAAA|26230|7|17|10|AM|first|morning|breakfast| +26231|AAAAAAAAIHGGAAAA|26231|7|17|11|AM|first|morning|breakfast| +26232|AAAAAAAAJHGGAAAA|26232|7|17|12|AM|first|morning|breakfast| +26233|AAAAAAAAKHGGAAAA|26233|7|17|13|AM|first|morning|breakfast| +26234|AAAAAAAALHGGAAAA|26234|7|17|14|AM|first|morning|breakfast| +26235|AAAAAAAAMHGGAAAA|26235|7|17|15|AM|first|morning|breakfast| +26236|AAAAAAAANHGGAAAA|26236|7|17|16|AM|first|morning|breakfast| +26237|AAAAAAAAOHGGAAAA|26237|7|17|17|AM|first|morning|breakfast| +26238|AAAAAAAAPHGGAAAA|26238|7|17|18|AM|first|morning|breakfast| +26239|AAAAAAAAAIGGAAAA|26239|7|17|19|AM|first|morning|breakfast| +26240|AAAAAAAABIGGAAAA|26240|7|17|20|AM|first|morning|breakfast| +26241|AAAAAAAACIGGAAAA|26241|7|17|21|AM|first|morning|breakfast| +26242|AAAAAAAADIGGAAAA|26242|7|17|22|AM|first|morning|breakfast| +26243|AAAAAAAAEIGGAAAA|26243|7|17|23|AM|first|morning|breakfast| +26244|AAAAAAAAFIGGAAAA|26244|7|17|24|AM|first|morning|breakfast| +26245|AAAAAAAAGIGGAAAA|26245|7|17|25|AM|first|morning|breakfast| +26246|AAAAAAAAHIGGAAAA|26246|7|17|26|AM|first|morning|breakfast| +26247|AAAAAAAAIIGGAAAA|26247|7|17|27|AM|first|morning|breakfast| +26248|AAAAAAAAJIGGAAAA|26248|7|17|28|AM|first|morning|breakfast| +26249|AAAAAAAAKIGGAAAA|26249|7|17|29|AM|first|morning|breakfast| +26250|AAAAAAAALIGGAAAA|26250|7|17|30|AM|first|morning|breakfast| +26251|AAAAAAAAMIGGAAAA|26251|7|17|31|AM|first|morning|breakfast| +26252|AAAAAAAANIGGAAAA|26252|7|17|32|AM|first|morning|breakfast| +26253|AAAAAAAAOIGGAAAA|26253|7|17|33|AM|first|morning|breakfast| +26254|AAAAAAAAPIGGAAAA|26254|7|17|34|AM|first|morning|breakfast| +26255|AAAAAAAAAJGGAAAA|26255|7|17|35|AM|first|morning|breakfast| +26256|AAAAAAAABJGGAAAA|26256|7|17|36|AM|first|morning|breakfast| +26257|AAAAAAAACJGGAAAA|26257|7|17|37|AM|first|morning|breakfast| +26258|AAAAAAAADJGGAAAA|26258|7|17|38|AM|first|morning|breakfast| +26259|AAAAAAAAEJGGAAAA|26259|7|17|39|AM|first|morning|breakfast| +26260|AAAAAAAAFJGGAAAA|26260|7|17|40|AM|first|morning|breakfast| +26261|AAAAAAAAGJGGAAAA|26261|7|17|41|AM|first|morning|breakfast| +26262|AAAAAAAAHJGGAAAA|26262|7|17|42|AM|first|morning|breakfast| +26263|AAAAAAAAIJGGAAAA|26263|7|17|43|AM|first|morning|breakfast| +26264|AAAAAAAAJJGGAAAA|26264|7|17|44|AM|first|morning|breakfast| +26265|AAAAAAAAKJGGAAAA|26265|7|17|45|AM|first|morning|breakfast| +26266|AAAAAAAALJGGAAAA|26266|7|17|46|AM|first|morning|breakfast| +26267|AAAAAAAAMJGGAAAA|26267|7|17|47|AM|first|morning|breakfast| +26268|AAAAAAAANJGGAAAA|26268|7|17|48|AM|first|morning|breakfast| +26269|AAAAAAAAOJGGAAAA|26269|7|17|49|AM|first|morning|breakfast| +26270|AAAAAAAAPJGGAAAA|26270|7|17|50|AM|first|morning|breakfast| +26271|AAAAAAAAAKGGAAAA|26271|7|17|51|AM|first|morning|breakfast| +26272|AAAAAAAABKGGAAAA|26272|7|17|52|AM|first|morning|breakfast| +26273|AAAAAAAACKGGAAAA|26273|7|17|53|AM|first|morning|breakfast| +26274|AAAAAAAADKGGAAAA|26274|7|17|54|AM|first|morning|breakfast| +26275|AAAAAAAAEKGGAAAA|26275|7|17|55|AM|first|morning|breakfast| +26276|AAAAAAAAFKGGAAAA|26276|7|17|56|AM|first|morning|breakfast| +26277|AAAAAAAAGKGGAAAA|26277|7|17|57|AM|first|morning|breakfast| +26278|AAAAAAAAHKGGAAAA|26278|7|17|58|AM|first|morning|breakfast| +26279|AAAAAAAAIKGGAAAA|26279|7|17|59|AM|first|morning|breakfast| +26280|AAAAAAAAJKGGAAAA|26280|7|18|0|AM|first|morning|breakfast| +26281|AAAAAAAAKKGGAAAA|26281|7|18|1|AM|first|morning|breakfast| +26282|AAAAAAAALKGGAAAA|26282|7|18|2|AM|first|morning|breakfast| +26283|AAAAAAAAMKGGAAAA|26283|7|18|3|AM|first|morning|breakfast| +26284|AAAAAAAANKGGAAAA|26284|7|18|4|AM|first|morning|breakfast| +26285|AAAAAAAAOKGGAAAA|26285|7|18|5|AM|first|morning|breakfast| +26286|AAAAAAAAPKGGAAAA|26286|7|18|6|AM|first|morning|breakfast| +26287|AAAAAAAAALGGAAAA|26287|7|18|7|AM|first|morning|breakfast| +26288|AAAAAAAABLGGAAAA|26288|7|18|8|AM|first|morning|breakfast| +26289|AAAAAAAACLGGAAAA|26289|7|18|9|AM|first|morning|breakfast| +26290|AAAAAAAADLGGAAAA|26290|7|18|10|AM|first|morning|breakfast| +26291|AAAAAAAAELGGAAAA|26291|7|18|11|AM|first|morning|breakfast| +26292|AAAAAAAAFLGGAAAA|26292|7|18|12|AM|first|morning|breakfast| +26293|AAAAAAAAGLGGAAAA|26293|7|18|13|AM|first|morning|breakfast| +26294|AAAAAAAAHLGGAAAA|26294|7|18|14|AM|first|morning|breakfast| +26295|AAAAAAAAILGGAAAA|26295|7|18|15|AM|first|morning|breakfast| +26296|AAAAAAAAJLGGAAAA|26296|7|18|16|AM|first|morning|breakfast| +26297|AAAAAAAAKLGGAAAA|26297|7|18|17|AM|first|morning|breakfast| +26298|AAAAAAAALLGGAAAA|26298|7|18|18|AM|first|morning|breakfast| +26299|AAAAAAAAMLGGAAAA|26299|7|18|19|AM|first|morning|breakfast| +26300|AAAAAAAANLGGAAAA|26300|7|18|20|AM|first|morning|breakfast| +26301|AAAAAAAAOLGGAAAA|26301|7|18|21|AM|first|morning|breakfast| +26302|AAAAAAAAPLGGAAAA|26302|7|18|22|AM|first|morning|breakfast| +26303|AAAAAAAAAMGGAAAA|26303|7|18|23|AM|first|morning|breakfast| +26304|AAAAAAAABMGGAAAA|26304|7|18|24|AM|first|morning|breakfast| +26305|AAAAAAAACMGGAAAA|26305|7|18|25|AM|first|morning|breakfast| +26306|AAAAAAAADMGGAAAA|26306|7|18|26|AM|first|morning|breakfast| +26307|AAAAAAAAEMGGAAAA|26307|7|18|27|AM|first|morning|breakfast| +26308|AAAAAAAAFMGGAAAA|26308|7|18|28|AM|first|morning|breakfast| +26309|AAAAAAAAGMGGAAAA|26309|7|18|29|AM|first|morning|breakfast| +26310|AAAAAAAAHMGGAAAA|26310|7|18|30|AM|first|morning|breakfast| +26311|AAAAAAAAIMGGAAAA|26311|7|18|31|AM|first|morning|breakfast| +26312|AAAAAAAAJMGGAAAA|26312|7|18|32|AM|first|morning|breakfast| +26313|AAAAAAAAKMGGAAAA|26313|7|18|33|AM|first|morning|breakfast| +26314|AAAAAAAALMGGAAAA|26314|7|18|34|AM|first|morning|breakfast| +26315|AAAAAAAAMMGGAAAA|26315|7|18|35|AM|first|morning|breakfast| +26316|AAAAAAAANMGGAAAA|26316|7|18|36|AM|first|morning|breakfast| +26317|AAAAAAAAOMGGAAAA|26317|7|18|37|AM|first|morning|breakfast| +26318|AAAAAAAAPMGGAAAA|26318|7|18|38|AM|first|morning|breakfast| +26319|AAAAAAAAANGGAAAA|26319|7|18|39|AM|first|morning|breakfast| +26320|AAAAAAAABNGGAAAA|26320|7|18|40|AM|first|morning|breakfast| +26321|AAAAAAAACNGGAAAA|26321|7|18|41|AM|first|morning|breakfast| +26322|AAAAAAAADNGGAAAA|26322|7|18|42|AM|first|morning|breakfast| +26323|AAAAAAAAENGGAAAA|26323|7|18|43|AM|first|morning|breakfast| +26324|AAAAAAAAFNGGAAAA|26324|7|18|44|AM|first|morning|breakfast| +26325|AAAAAAAAGNGGAAAA|26325|7|18|45|AM|first|morning|breakfast| +26326|AAAAAAAAHNGGAAAA|26326|7|18|46|AM|first|morning|breakfast| +26327|AAAAAAAAINGGAAAA|26327|7|18|47|AM|first|morning|breakfast| +26328|AAAAAAAAJNGGAAAA|26328|7|18|48|AM|first|morning|breakfast| +26329|AAAAAAAAKNGGAAAA|26329|7|18|49|AM|first|morning|breakfast| +26330|AAAAAAAALNGGAAAA|26330|7|18|50|AM|first|morning|breakfast| +26331|AAAAAAAAMNGGAAAA|26331|7|18|51|AM|first|morning|breakfast| +26332|AAAAAAAANNGGAAAA|26332|7|18|52|AM|first|morning|breakfast| +26333|AAAAAAAAONGGAAAA|26333|7|18|53|AM|first|morning|breakfast| +26334|AAAAAAAAPNGGAAAA|26334|7|18|54|AM|first|morning|breakfast| +26335|AAAAAAAAAOGGAAAA|26335|7|18|55|AM|first|morning|breakfast| +26336|AAAAAAAABOGGAAAA|26336|7|18|56|AM|first|morning|breakfast| +26337|AAAAAAAACOGGAAAA|26337|7|18|57|AM|first|morning|breakfast| +26338|AAAAAAAADOGGAAAA|26338|7|18|58|AM|first|morning|breakfast| +26339|AAAAAAAAEOGGAAAA|26339|7|18|59|AM|first|morning|breakfast| +26340|AAAAAAAAFOGGAAAA|26340|7|19|0|AM|first|morning|breakfast| +26341|AAAAAAAAGOGGAAAA|26341|7|19|1|AM|first|morning|breakfast| +26342|AAAAAAAAHOGGAAAA|26342|7|19|2|AM|first|morning|breakfast| +26343|AAAAAAAAIOGGAAAA|26343|7|19|3|AM|first|morning|breakfast| +26344|AAAAAAAAJOGGAAAA|26344|7|19|4|AM|first|morning|breakfast| +26345|AAAAAAAAKOGGAAAA|26345|7|19|5|AM|first|morning|breakfast| +26346|AAAAAAAALOGGAAAA|26346|7|19|6|AM|first|morning|breakfast| +26347|AAAAAAAAMOGGAAAA|26347|7|19|7|AM|first|morning|breakfast| +26348|AAAAAAAANOGGAAAA|26348|7|19|8|AM|first|morning|breakfast| +26349|AAAAAAAAOOGGAAAA|26349|7|19|9|AM|first|morning|breakfast| +26350|AAAAAAAAPOGGAAAA|26350|7|19|10|AM|first|morning|breakfast| +26351|AAAAAAAAAPGGAAAA|26351|7|19|11|AM|first|morning|breakfast| +26352|AAAAAAAABPGGAAAA|26352|7|19|12|AM|first|morning|breakfast| +26353|AAAAAAAACPGGAAAA|26353|7|19|13|AM|first|morning|breakfast| +26354|AAAAAAAADPGGAAAA|26354|7|19|14|AM|first|morning|breakfast| +26355|AAAAAAAAEPGGAAAA|26355|7|19|15|AM|first|morning|breakfast| +26356|AAAAAAAAFPGGAAAA|26356|7|19|16|AM|first|morning|breakfast| +26357|AAAAAAAAGPGGAAAA|26357|7|19|17|AM|first|morning|breakfast| +26358|AAAAAAAAHPGGAAAA|26358|7|19|18|AM|first|morning|breakfast| +26359|AAAAAAAAIPGGAAAA|26359|7|19|19|AM|first|morning|breakfast| +26360|AAAAAAAAJPGGAAAA|26360|7|19|20|AM|first|morning|breakfast| +26361|AAAAAAAAKPGGAAAA|26361|7|19|21|AM|first|morning|breakfast| +26362|AAAAAAAALPGGAAAA|26362|7|19|22|AM|first|morning|breakfast| +26363|AAAAAAAAMPGGAAAA|26363|7|19|23|AM|first|morning|breakfast| +26364|AAAAAAAANPGGAAAA|26364|7|19|24|AM|first|morning|breakfast| +26365|AAAAAAAAOPGGAAAA|26365|7|19|25|AM|first|morning|breakfast| +26366|AAAAAAAAPPGGAAAA|26366|7|19|26|AM|first|morning|breakfast| +26367|AAAAAAAAAAHGAAAA|26367|7|19|27|AM|first|morning|breakfast| +26368|AAAAAAAABAHGAAAA|26368|7|19|28|AM|first|morning|breakfast| +26369|AAAAAAAACAHGAAAA|26369|7|19|29|AM|first|morning|breakfast| +26370|AAAAAAAADAHGAAAA|26370|7|19|30|AM|first|morning|breakfast| +26371|AAAAAAAAEAHGAAAA|26371|7|19|31|AM|first|morning|breakfast| +26372|AAAAAAAAFAHGAAAA|26372|7|19|32|AM|first|morning|breakfast| +26373|AAAAAAAAGAHGAAAA|26373|7|19|33|AM|first|morning|breakfast| +26374|AAAAAAAAHAHGAAAA|26374|7|19|34|AM|first|morning|breakfast| +26375|AAAAAAAAIAHGAAAA|26375|7|19|35|AM|first|morning|breakfast| +26376|AAAAAAAAJAHGAAAA|26376|7|19|36|AM|first|morning|breakfast| +26377|AAAAAAAAKAHGAAAA|26377|7|19|37|AM|first|morning|breakfast| +26378|AAAAAAAALAHGAAAA|26378|7|19|38|AM|first|morning|breakfast| +26379|AAAAAAAAMAHGAAAA|26379|7|19|39|AM|first|morning|breakfast| +26380|AAAAAAAANAHGAAAA|26380|7|19|40|AM|first|morning|breakfast| +26381|AAAAAAAAOAHGAAAA|26381|7|19|41|AM|first|morning|breakfast| +26382|AAAAAAAAPAHGAAAA|26382|7|19|42|AM|first|morning|breakfast| +26383|AAAAAAAAABHGAAAA|26383|7|19|43|AM|first|morning|breakfast| +26384|AAAAAAAABBHGAAAA|26384|7|19|44|AM|first|morning|breakfast| +26385|AAAAAAAACBHGAAAA|26385|7|19|45|AM|first|morning|breakfast| +26386|AAAAAAAADBHGAAAA|26386|7|19|46|AM|first|morning|breakfast| +26387|AAAAAAAAEBHGAAAA|26387|7|19|47|AM|first|morning|breakfast| +26388|AAAAAAAAFBHGAAAA|26388|7|19|48|AM|first|morning|breakfast| +26389|AAAAAAAAGBHGAAAA|26389|7|19|49|AM|first|morning|breakfast| +26390|AAAAAAAAHBHGAAAA|26390|7|19|50|AM|first|morning|breakfast| +26391|AAAAAAAAIBHGAAAA|26391|7|19|51|AM|first|morning|breakfast| +26392|AAAAAAAAJBHGAAAA|26392|7|19|52|AM|first|morning|breakfast| +26393|AAAAAAAAKBHGAAAA|26393|7|19|53|AM|first|morning|breakfast| +26394|AAAAAAAALBHGAAAA|26394|7|19|54|AM|first|morning|breakfast| +26395|AAAAAAAAMBHGAAAA|26395|7|19|55|AM|first|morning|breakfast| +26396|AAAAAAAANBHGAAAA|26396|7|19|56|AM|first|morning|breakfast| +26397|AAAAAAAAOBHGAAAA|26397|7|19|57|AM|first|morning|breakfast| +26398|AAAAAAAAPBHGAAAA|26398|7|19|58|AM|first|morning|breakfast| +26399|AAAAAAAAACHGAAAA|26399|7|19|59|AM|first|morning|breakfast| +26400|AAAAAAAABCHGAAAA|26400|7|20|0|AM|first|morning|breakfast| +26401|AAAAAAAACCHGAAAA|26401|7|20|1|AM|first|morning|breakfast| +26402|AAAAAAAADCHGAAAA|26402|7|20|2|AM|first|morning|breakfast| +26403|AAAAAAAAECHGAAAA|26403|7|20|3|AM|first|morning|breakfast| +26404|AAAAAAAAFCHGAAAA|26404|7|20|4|AM|first|morning|breakfast| +26405|AAAAAAAAGCHGAAAA|26405|7|20|5|AM|first|morning|breakfast| +26406|AAAAAAAAHCHGAAAA|26406|7|20|6|AM|first|morning|breakfast| +26407|AAAAAAAAICHGAAAA|26407|7|20|7|AM|first|morning|breakfast| +26408|AAAAAAAAJCHGAAAA|26408|7|20|8|AM|first|morning|breakfast| +26409|AAAAAAAAKCHGAAAA|26409|7|20|9|AM|first|morning|breakfast| +26410|AAAAAAAALCHGAAAA|26410|7|20|10|AM|first|morning|breakfast| +26411|AAAAAAAAMCHGAAAA|26411|7|20|11|AM|first|morning|breakfast| +26412|AAAAAAAANCHGAAAA|26412|7|20|12|AM|first|morning|breakfast| +26413|AAAAAAAAOCHGAAAA|26413|7|20|13|AM|first|morning|breakfast| +26414|AAAAAAAAPCHGAAAA|26414|7|20|14|AM|first|morning|breakfast| +26415|AAAAAAAAADHGAAAA|26415|7|20|15|AM|first|morning|breakfast| +26416|AAAAAAAABDHGAAAA|26416|7|20|16|AM|first|morning|breakfast| +26417|AAAAAAAACDHGAAAA|26417|7|20|17|AM|first|morning|breakfast| +26418|AAAAAAAADDHGAAAA|26418|7|20|18|AM|first|morning|breakfast| +26419|AAAAAAAAEDHGAAAA|26419|7|20|19|AM|first|morning|breakfast| +26420|AAAAAAAAFDHGAAAA|26420|7|20|20|AM|first|morning|breakfast| +26421|AAAAAAAAGDHGAAAA|26421|7|20|21|AM|first|morning|breakfast| +26422|AAAAAAAAHDHGAAAA|26422|7|20|22|AM|first|morning|breakfast| +26423|AAAAAAAAIDHGAAAA|26423|7|20|23|AM|first|morning|breakfast| +26424|AAAAAAAAJDHGAAAA|26424|7|20|24|AM|first|morning|breakfast| +26425|AAAAAAAAKDHGAAAA|26425|7|20|25|AM|first|morning|breakfast| +26426|AAAAAAAALDHGAAAA|26426|7|20|26|AM|first|morning|breakfast| +26427|AAAAAAAAMDHGAAAA|26427|7|20|27|AM|first|morning|breakfast| +26428|AAAAAAAANDHGAAAA|26428|7|20|28|AM|first|morning|breakfast| +26429|AAAAAAAAODHGAAAA|26429|7|20|29|AM|first|morning|breakfast| +26430|AAAAAAAAPDHGAAAA|26430|7|20|30|AM|first|morning|breakfast| +26431|AAAAAAAAAEHGAAAA|26431|7|20|31|AM|first|morning|breakfast| +26432|AAAAAAAABEHGAAAA|26432|7|20|32|AM|first|morning|breakfast| +26433|AAAAAAAACEHGAAAA|26433|7|20|33|AM|first|morning|breakfast| +26434|AAAAAAAADEHGAAAA|26434|7|20|34|AM|first|morning|breakfast| +26435|AAAAAAAAEEHGAAAA|26435|7|20|35|AM|first|morning|breakfast| +26436|AAAAAAAAFEHGAAAA|26436|7|20|36|AM|first|morning|breakfast| +26437|AAAAAAAAGEHGAAAA|26437|7|20|37|AM|first|morning|breakfast| +26438|AAAAAAAAHEHGAAAA|26438|7|20|38|AM|first|morning|breakfast| +26439|AAAAAAAAIEHGAAAA|26439|7|20|39|AM|first|morning|breakfast| +26440|AAAAAAAAJEHGAAAA|26440|7|20|40|AM|first|morning|breakfast| +26441|AAAAAAAAKEHGAAAA|26441|7|20|41|AM|first|morning|breakfast| +26442|AAAAAAAALEHGAAAA|26442|7|20|42|AM|first|morning|breakfast| +26443|AAAAAAAAMEHGAAAA|26443|7|20|43|AM|first|morning|breakfast| +26444|AAAAAAAANEHGAAAA|26444|7|20|44|AM|first|morning|breakfast| +26445|AAAAAAAAOEHGAAAA|26445|7|20|45|AM|first|morning|breakfast| +26446|AAAAAAAAPEHGAAAA|26446|7|20|46|AM|first|morning|breakfast| +26447|AAAAAAAAAFHGAAAA|26447|7|20|47|AM|first|morning|breakfast| +26448|AAAAAAAABFHGAAAA|26448|7|20|48|AM|first|morning|breakfast| +26449|AAAAAAAACFHGAAAA|26449|7|20|49|AM|first|morning|breakfast| +26450|AAAAAAAADFHGAAAA|26450|7|20|50|AM|first|morning|breakfast| +26451|AAAAAAAAEFHGAAAA|26451|7|20|51|AM|first|morning|breakfast| +26452|AAAAAAAAFFHGAAAA|26452|7|20|52|AM|first|morning|breakfast| +26453|AAAAAAAAGFHGAAAA|26453|7|20|53|AM|first|morning|breakfast| +26454|AAAAAAAAHFHGAAAA|26454|7|20|54|AM|first|morning|breakfast| +26455|AAAAAAAAIFHGAAAA|26455|7|20|55|AM|first|morning|breakfast| +26456|AAAAAAAAJFHGAAAA|26456|7|20|56|AM|first|morning|breakfast| +26457|AAAAAAAAKFHGAAAA|26457|7|20|57|AM|first|morning|breakfast| +26458|AAAAAAAALFHGAAAA|26458|7|20|58|AM|first|morning|breakfast| +26459|AAAAAAAAMFHGAAAA|26459|7|20|59|AM|first|morning|breakfast| +26460|AAAAAAAANFHGAAAA|26460|7|21|0|AM|first|morning|breakfast| +26461|AAAAAAAAOFHGAAAA|26461|7|21|1|AM|first|morning|breakfast| +26462|AAAAAAAAPFHGAAAA|26462|7|21|2|AM|first|morning|breakfast| +26463|AAAAAAAAAGHGAAAA|26463|7|21|3|AM|first|morning|breakfast| +26464|AAAAAAAABGHGAAAA|26464|7|21|4|AM|first|morning|breakfast| +26465|AAAAAAAACGHGAAAA|26465|7|21|5|AM|first|morning|breakfast| +26466|AAAAAAAADGHGAAAA|26466|7|21|6|AM|first|morning|breakfast| +26467|AAAAAAAAEGHGAAAA|26467|7|21|7|AM|first|morning|breakfast| +26468|AAAAAAAAFGHGAAAA|26468|7|21|8|AM|first|morning|breakfast| +26469|AAAAAAAAGGHGAAAA|26469|7|21|9|AM|first|morning|breakfast| +26470|AAAAAAAAHGHGAAAA|26470|7|21|10|AM|first|morning|breakfast| +26471|AAAAAAAAIGHGAAAA|26471|7|21|11|AM|first|morning|breakfast| +26472|AAAAAAAAJGHGAAAA|26472|7|21|12|AM|first|morning|breakfast| +26473|AAAAAAAAKGHGAAAA|26473|7|21|13|AM|first|morning|breakfast| +26474|AAAAAAAALGHGAAAA|26474|7|21|14|AM|first|morning|breakfast| +26475|AAAAAAAAMGHGAAAA|26475|7|21|15|AM|first|morning|breakfast| +26476|AAAAAAAANGHGAAAA|26476|7|21|16|AM|first|morning|breakfast| +26477|AAAAAAAAOGHGAAAA|26477|7|21|17|AM|first|morning|breakfast| +26478|AAAAAAAAPGHGAAAA|26478|7|21|18|AM|first|morning|breakfast| +26479|AAAAAAAAAHHGAAAA|26479|7|21|19|AM|first|morning|breakfast| +26480|AAAAAAAABHHGAAAA|26480|7|21|20|AM|first|morning|breakfast| +26481|AAAAAAAACHHGAAAA|26481|7|21|21|AM|first|morning|breakfast| +26482|AAAAAAAADHHGAAAA|26482|7|21|22|AM|first|morning|breakfast| +26483|AAAAAAAAEHHGAAAA|26483|7|21|23|AM|first|morning|breakfast| +26484|AAAAAAAAFHHGAAAA|26484|7|21|24|AM|first|morning|breakfast| +26485|AAAAAAAAGHHGAAAA|26485|7|21|25|AM|first|morning|breakfast| +26486|AAAAAAAAHHHGAAAA|26486|7|21|26|AM|first|morning|breakfast| +26487|AAAAAAAAIHHGAAAA|26487|7|21|27|AM|first|morning|breakfast| +26488|AAAAAAAAJHHGAAAA|26488|7|21|28|AM|first|morning|breakfast| +26489|AAAAAAAAKHHGAAAA|26489|7|21|29|AM|first|morning|breakfast| +26490|AAAAAAAALHHGAAAA|26490|7|21|30|AM|first|morning|breakfast| +26491|AAAAAAAAMHHGAAAA|26491|7|21|31|AM|first|morning|breakfast| +26492|AAAAAAAANHHGAAAA|26492|7|21|32|AM|first|morning|breakfast| +26493|AAAAAAAAOHHGAAAA|26493|7|21|33|AM|first|morning|breakfast| +26494|AAAAAAAAPHHGAAAA|26494|7|21|34|AM|first|morning|breakfast| +26495|AAAAAAAAAIHGAAAA|26495|7|21|35|AM|first|morning|breakfast| +26496|AAAAAAAABIHGAAAA|26496|7|21|36|AM|first|morning|breakfast| +26497|AAAAAAAACIHGAAAA|26497|7|21|37|AM|first|morning|breakfast| +26498|AAAAAAAADIHGAAAA|26498|7|21|38|AM|first|morning|breakfast| +26499|AAAAAAAAEIHGAAAA|26499|7|21|39|AM|first|morning|breakfast| +26500|AAAAAAAAFIHGAAAA|26500|7|21|40|AM|first|morning|breakfast| +26501|AAAAAAAAGIHGAAAA|26501|7|21|41|AM|first|morning|breakfast| +26502|AAAAAAAAHIHGAAAA|26502|7|21|42|AM|first|morning|breakfast| +26503|AAAAAAAAIIHGAAAA|26503|7|21|43|AM|first|morning|breakfast| +26504|AAAAAAAAJIHGAAAA|26504|7|21|44|AM|first|morning|breakfast| +26505|AAAAAAAAKIHGAAAA|26505|7|21|45|AM|first|morning|breakfast| +26506|AAAAAAAALIHGAAAA|26506|7|21|46|AM|first|morning|breakfast| +26507|AAAAAAAAMIHGAAAA|26507|7|21|47|AM|first|morning|breakfast| +26508|AAAAAAAANIHGAAAA|26508|7|21|48|AM|first|morning|breakfast| +26509|AAAAAAAAOIHGAAAA|26509|7|21|49|AM|first|morning|breakfast| +26510|AAAAAAAAPIHGAAAA|26510|7|21|50|AM|first|morning|breakfast| +26511|AAAAAAAAAJHGAAAA|26511|7|21|51|AM|first|morning|breakfast| +26512|AAAAAAAABJHGAAAA|26512|7|21|52|AM|first|morning|breakfast| +26513|AAAAAAAACJHGAAAA|26513|7|21|53|AM|first|morning|breakfast| +26514|AAAAAAAADJHGAAAA|26514|7|21|54|AM|first|morning|breakfast| +26515|AAAAAAAAEJHGAAAA|26515|7|21|55|AM|first|morning|breakfast| +26516|AAAAAAAAFJHGAAAA|26516|7|21|56|AM|first|morning|breakfast| +26517|AAAAAAAAGJHGAAAA|26517|7|21|57|AM|first|morning|breakfast| +26518|AAAAAAAAHJHGAAAA|26518|7|21|58|AM|first|morning|breakfast| +26519|AAAAAAAAIJHGAAAA|26519|7|21|59|AM|first|morning|breakfast| +26520|AAAAAAAAJJHGAAAA|26520|7|22|0|AM|first|morning|breakfast| +26521|AAAAAAAAKJHGAAAA|26521|7|22|1|AM|first|morning|breakfast| +26522|AAAAAAAALJHGAAAA|26522|7|22|2|AM|first|morning|breakfast| +26523|AAAAAAAAMJHGAAAA|26523|7|22|3|AM|first|morning|breakfast| +26524|AAAAAAAANJHGAAAA|26524|7|22|4|AM|first|morning|breakfast| +26525|AAAAAAAAOJHGAAAA|26525|7|22|5|AM|first|morning|breakfast| +26526|AAAAAAAAPJHGAAAA|26526|7|22|6|AM|first|morning|breakfast| +26527|AAAAAAAAAKHGAAAA|26527|7|22|7|AM|first|morning|breakfast| +26528|AAAAAAAABKHGAAAA|26528|7|22|8|AM|first|morning|breakfast| +26529|AAAAAAAACKHGAAAA|26529|7|22|9|AM|first|morning|breakfast| +26530|AAAAAAAADKHGAAAA|26530|7|22|10|AM|first|morning|breakfast| +26531|AAAAAAAAEKHGAAAA|26531|7|22|11|AM|first|morning|breakfast| +26532|AAAAAAAAFKHGAAAA|26532|7|22|12|AM|first|morning|breakfast| +26533|AAAAAAAAGKHGAAAA|26533|7|22|13|AM|first|morning|breakfast| +26534|AAAAAAAAHKHGAAAA|26534|7|22|14|AM|first|morning|breakfast| +26535|AAAAAAAAIKHGAAAA|26535|7|22|15|AM|first|morning|breakfast| +26536|AAAAAAAAJKHGAAAA|26536|7|22|16|AM|first|morning|breakfast| +26537|AAAAAAAAKKHGAAAA|26537|7|22|17|AM|first|morning|breakfast| +26538|AAAAAAAALKHGAAAA|26538|7|22|18|AM|first|morning|breakfast| +26539|AAAAAAAAMKHGAAAA|26539|7|22|19|AM|first|morning|breakfast| +26540|AAAAAAAANKHGAAAA|26540|7|22|20|AM|first|morning|breakfast| +26541|AAAAAAAAOKHGAAAA|26541|7|22|21|AM|first|morning|breakfast| +26542|AAAAAAAAPKHGAAAA|26542|7|22|22|AM|first|morning|breakfast| +26543|AAAAAAAAALHGAAAA|26543|7|22|23|AM|first|morning|breakfast| +26544|AAAAAAAABLHGAAAA|26544|7|22|24|AM|first|morning|breakfast| +26545|AAAAAAAACLHGAAAA|26545|7|22|25|AM|first|morning|breakfast| +26546|AAAAAAAADLHGAAAA|26546|7|22|26|AM|first|morning|breakfast| +26547|AAAAAAAAELHGAAAA|26547|7|22|27|AM|first|morning|breakfast| +26548|AAAAAAAAFLHGAAAA|26548|7|22|28|AM|first|morning|breakfast| +26549|AAAAAAAAGLHGAAAA|26549|7|22|29|AM|first|morning|breakfast| +26550|AAAAAAAAHLHGAAAA|26550|7|22|30|AM|first|morning|breakfast| +26551|AAAAAAAAILHGAAAA|26551|7|22|31|AM|first|morning|breakfast| +26552|AAAAAAAAJLHGAAAA|26552|7|22|32|AM|first|morning|breakfast| +26553|AAAAAAAAKLHGAAAA|26553|7|22|33|AM|first|morning|breakfast| +26554|AAAAAAAALLHGAAAA|26554|7|22|34|AM|first|morning|breakfast| +26555|AAAAAAAAMLHGAAAA|26555|7|22|35|AM|first|morning|breakfast| +26556|AAAAAAAANLHGAAAA|26556|7|22|36|AM|first|morning|breakfast| +26557|AAAAAAAAOLHGAAAA|26557|7|22|37|AM|first|morning|breakfast| +26558|AAAAAAAAPLHGAAAA|26558|7|22|38|AM|first|morning|breakfast| +26559|AAAAAAAAAMHGAAAA|26559|7|22|39|AM|first|morning|breakfast| +26560|AAAAAAAABMHGAAAA|26560|7|22|40|AM|first|morning|breakfast| +26561|AAAAAAAACMHGAAAA|26561|7|22|41|AM|first|morning|breakfast| +26562|AAAAAAAADMHGAAAA|26562|7|22|42|AM|first|morning|breakfast| +26563|AAAAAAAAEMHGAAAA|26563|7|22|43|AM|first|morning|breakfast| +26564|AAAAAAAAFMHGAAAA|26564|7|22|44|AM|first|morning|breakfast| +26565|AAAAAAAAGMHGAAAA|26565|7|22|45|AM|first|morning|breakfast| +26566|AAAAAAAAHMHGAAAA|26566|7|22|46|AM|first|morning|breakfast| +26567|AAAAAAAAIMHGAAAA|26567|7|22|47|AM|first|morning|breakfast| +26568|AAAAAAAAJMHGAAAA|26568|7|22|48|AM|first|morning|breakfast| +26569|AAAAAAAAKMHGAAAA|26569|7|22|49|AM|first|morning|breakfast| +26570|AAAAAAAALMHGAAAA|26570|7|22|50|AM|first|morning|breakfast| +26571|AAAAAAAAMMHGAAAA|26571|7|22|51|AM|first|morning|breakfast| +26572|AAAAAAAANMHGAAAA|26572|7|22|52|AM|first|morning|breakfast| +26573|AAAAAAAAOMHGAAAA|26573|7|22|53|AM|first|morning|breakfast| +26574|AAAAAAAAPMHGAAAA|26574|7|22|54|AM|first|morning|breakfast| +26575|AAAAAAAAANHGAAAA|26575|7|22|55|AM|first|morning|breakfast| +26576|AAAAAAAABNHGAAAA|26576|7|22|56|AM|first|morning|breakfast| +26577|AAAAAAAACNHGAAAA|26577|7|22|57|AM|first|morning|breakfast| +26578|AAAAAAAADNHGAAAA|26578|7|22|58|AM|first|morning|breakfast| +26579|AAAAAAAAENHGAAAA|26579|7|22|59|AM|first|morning|breakfast| +26580|AAAAAAAAFNHGAAAA|26580|7|23|0|AM|first|morning|breakfast| +26581|AAAAAAAAGNHGAAAA|26581|7|23|1|AM|first|morning|breakfast| +26582|AAAAAAAAHNHGAAAA|26582|7|23|2|AM|first|morning|breakfast| +26583|AAAAAAAAINHGAAAA|26583|7|23|3|AM|first|morning|breakfast| +26584|AAAAAAAAJNHGAAAA|26584|7|23|4|AM|first|morning|breakfast| +26585|AAAAAAAAKNHGAAAA|26585|7|23|5|AM|first|morning|breakfast| +26586|AAAAAAAALNHGAAAA|26586|7|23|6|AM|first|morning|breakfast| +26587|AAAAAAAAMNHGAAAA|26587|7|23|7|AM|first|morning|breakfast| +26588|AAAAAAAANNHGAAAA|26588|7|23|8|AM|first|morning|breakfast| +26589|AAAAAAAAONHGAAAA|26589|7|23|9|AM|first|morning|breakfast| +26590|AAAAAAAAPNHGAAAA|26590|7|23|10|AM|first|morning|breakfast| +26591|AAAAAAAAAOHGAAAA|26591|7|23|11|AM|first|morning|breakfast| +26592|AAAAAAAABOHGAAAA|26592|7|23|12|AM|first|morning|breakfast| +26593|AAAAAAAACOHGAAAA|26593|7|23|13|AM|first|morning|breakfast| +26594|AAAAAAAADOHGAAAA|26594|7|23|14|AM|first|morning|breakfast| +26595|AAAAAAAAEOHGAAAA|26595|7|23|15|AM|first|morning|breakfast| +26596|AAAAAAAAFOHGAAAA|26596|7|23|16|AM|first|morning|breakfast| +26597|AAAAAAAAGOHGAAAA|26597|7|23|17|AM|first|morning|breakfast| +26598|AAAAAAAAHOHGAAAA|26598|7|23|18|AM|first|morning|breakfast| +26599|AAAAAAAAIOHGAAAA|26599|7|23|19|AM|first|morning|breakfast| +26600|AAAAAAAAJOHGAAAA|26600|7|23|20|AM|first|morning|breakfast| +26601|AAAAAAAAKOHGAAAA|26601|7|23|21|AM|first|morning|breakfast| +26602|AAAAAAAALOHGAAAA|26602|7|23|22|AM|first|morning|breakfast| +26603|AAAAAAAAMOHGAAAA|26603|7|23|23|AM|first|morning|breakfast| +26604|AAAAAAAANOHGAAAA|26604|7|23|24|AM|first|morning|breakfast| +26605|AAAAAAAAOOHGAAAA|26605|7|23|25|AM|first|morning|breakfast| +26606|AAAAAAAAPOHGAAAA|26606|7|23|26|AM|first|morning|breakfast| +26607|AAAAAAAAAPHGAAAA|26607|7|23|27|AM|first|morning|breakfast| +26608|AAAAAAAABPHGAAAA|26608|7|23|28|AM|first|morning|breakfast| +26609|AAAAAAAACPHGAAAA|26609|7|23|29|AM|first|morning|breakfast| +26610|AAAAAAAADPHGAAAA|26610|7|23|30|AM|first|morning|breakfast| +26611|AAAAAAAAEPHGAAAA|26611|7|23|31|AM|first|morning|breakfast| +26612|AAAAAAAAFPHGAAAA|26612|7|23|32|AM|first|morning|breakfast| +26613|AAAAAAAAGPHGAAAA|26613|7|23|33|AM|first|morning|breakfast| +26614|AAAAAAAAHPHGAAAA|26614|7|23|34|AM|first|morning|breakfast| +26615|AAAAAAAAIPHGAAAA|26615|7|23|35|AM|first|morning|breakfast| +26616|AAAAAAAAJPHGAAAA|26616|7|23|36|AM|first|morning|breakfast| +26617|AAAAAAAAKPHGAAAA|26617|7|23|37|AM|first|morning|breakfast| +26618|AAAAAAAALPHGAAAA|26618|7|23|38|AM|first|morning|breakfast| +26619|AAAAAAAAMPHGAAAA|26619|7|23|39|AM|first|morning|breakfast| +26620|AAAAAAAANPHGAAAA|26620|7|23|40|AM|first|morning|breakfast| +26621|AAAAAAAAOPHGAAAA|26621|7|23|41|AM|first|morning|breakfast| +26622|AAAAAAAAPPHGAAAA|26622|7|23|42|AM|first|morning|breakfast| +26623|AAAAAAAAAAIGAAAA|26623|7|23|43|AM|first|morning|breakfast| +26624|AAAAAAAABAIGAAAA|26624|7|23|44|AM|first|morning|breakfast| +26625|AAAAAAAACAIGAAAA|26625|7|23|45|AM|first|morning|breakfast| +26626|AAAAAAAADAIGAAAA|26626|7|23|46|AM|first|morning|breakfast| +26627|AAAAAAAAEAIGAAAA|26627|7|23|47|AM|first|morning|breakfast| +26628|AAAAAAAAFAIGAAAA|26628|7|23|48|AM|first|morning|breakfast| +26629|AAAAAAAAGAIGAAAA|26629|7|23|49|AM|first|morning|breakfast| +26630|AAAAAAAAHAIGAAAA|26630|7|23|50|AM|first|morning|breakfast| +26631|AAAAAAAAIAIGAAAA|26631|7|23|51|AM|first|morning|breakfast| +26632|AAAAAAAAJAIGAAAA|26632|7|23|52|AM|first|morning|breakfast| +26633|AAAAAAAAKAIGAAAA|26633|7|23|53|AM|first|morning|breakfast| +26634|AAAAAAAALAIGAAAA|26634|7|23|54|AM|first|morning|breakfast| +26635|AAAAAAAAMAIGAAAA|26635|7|23|55|AM|first|morning|breakfast| +26636|AAAAAAAANAIGAAAA|26636|7|23|56|AM|first|morning|breakfast| +26637|AAAAAAAAOAIGAAAA|26637|7|23|57|AM|first|morning|breakfast| +26638|AAAAAAAAPAIGAAAA|26638|7|23|58|AM|first|morning|breakfast| +26639|AAAAAAAAABIGAAAA|26639|7|23|59|AM|first|morning|breakfast| +26640|AAAAAAAABBIGAAAA|26640|7|24|0|AM|first|morning|breakfast| +26641|AAAAAAAACBIGAAAA|26641|7|24|1|AM|first|morning|breakfast| +26642|AAAAAAAADBIGAAAA|26642|7|24|2|AM|first|morning|breakfast| +26643|AAAAAAAAEBIGAAAA|26643|7|24|3|AM|first|morning|breakfast| +26644|AAAAAAAAFBIGAAAA|26644|7|24|4|AM|first|morning|breakfast| +26645|AAAAAAAAGBIGAAAA|26645|7|24|5|AM|first|morning|breakfast| +26646|AAAAAAAAHBIGAAAA|26646|7|24|6|AM|first|morning|breakfast| +26647|AAAAAAAAIBIGAAAA|26647|7|24|7|AM|first|morning|breakfast| +26648|AAAAAAAAJBIGAAAA|26648|7|24|8|AM|first|morning|breakfast| +26649|AAAAAAAAKBIGAAAA|26649|7|24|9|AM|first|morning|breakfast| +26650|AAAAAAAALBIGAAAA|26650|7|24|10|AM|first|morning|breakfast| +26651|AAAAAAAAMBIGAAAA|26651|7|24|11|AM|first|morning|breakfast| +26652|AAAAAAAANBIGAAAA|26652|7|24|12|AM|first|morning|breakfast| +26653|AAAAAAAAOBIGAAAA|26653|7|24|13|AM|first|morning|breakfast| +26654|AAAAAAAAPBIGAAAA|26654|7|24|14|AM|first|morning|breakfast| +26655|AAAAAAAAACIGAAAA|26655|7|24|15|AM|first|morning|breakfast| +26656|AAAAAAAABCIGAAAA|26656|7|24|16|AM|first|morning|breakfast| +26657|AAAAAAAACCIGAAAA|26657|7|24|17|AM|first|morning|breakfast| +26658|AAAAAAAADCIGAAAA|26658|7|24|18|AM|first|morning|breakfast| +26659|AAAAAAAAECIGAAAA|26659|7|24|19|AM|first|morning|breakfast| +26660|AAAAAAAAFCIGAAAA|26660|7|24|20|AM|first|morning|breakfast| +26661|AAAAAAAAGCIGAAAA|26661|7|24|21|AM|first|morning|breakfast| +26662|AAAAAAAAHCIGAAAA|26662|7|24|22|AM|first|morning|breakfast| +26663|AAAAAAAAICIGAAAA|26663|7|24|23|AM|first|morning|breakfast| +26664|AAAAAAAAJCIGAAAA|26664|7|24|24|AM|first|morning|breakfast| +26665|AAAAAAAAKCIGAAAA|26665|7|24|25|AM|first|morning|breakfast| +26666|AAAAAAAALCIGAAAA|26666|7|24|26|AM|first|morning|breakfast| +26667|AAAAAAAAMCIGAAAA|26667|7|24|27|AM|first|morning|breakfast| +26668|AAAAAAAANCIGAAAA|26668|7|24|28|AM|first|morning|breakfast| +26669|AAAAAAAAOCIGAAAA|26669|7|24|29|AM|first|morning|breakfast| +26670|AAAAAAAAPCIGAAAA|26670|7|24|30|AM|first|morning|breakfast| +26671|AAAAAAAAADIGAAAA|26671|7|24|31|AM|first|morning|breakfast| +26672|AAAAAAAABDIGAAAA|26672|7|24|32|AM|first|morning|breakfast| +26673|AAAAAAAACDIGAAAA|26673|7|24|33|AM|first|morning|breakfast| +26674|AAAAAAAADDIGAAAA|26674|7|24|34|AM|first|morning|breakfast| +26675|AAAAAAAAEDIGAAAA|26675|7|24|35|AM|first|morning|breakfast| +26676|AAAAAAAAFDIGAAAA|26676|7|24|36|AM|first|morning|breakfast| +26677|AAAAAAAAGDIGAAAA|26677|7|24|37|AM|first|morning|breakfast| +26678|AAAAAAAAHDIGAAAA|26678|7|24|38|AM|first|morning|breakfast| +26679|AAAAAAAAIDIGAAAA|26679|7|24|39|AM|first|morning|breakfast| +26680|AAAAAAAAJDIGAAAA|26680|7|24|40|AM|first|morning|breakfast| +26681|AAAAAAAAKDIGAAAA|26681|7|24|41|AM|first|morning|breakfast| +26682|AAAAAAAALDIGAAAA|26682|7|24|42|AM|first|morning|breakfast| +26683|AAAAAAAAMDIGAAAA|26683|7|24|43|AM|first|morning|breakfast| +26684|AAAAAAAANDIGAAAA|26684|7|24|44|AM|first|morning|breakfast| +26685|AAAAAAAAODIGAAAA|26685|7|24|45|AM|first|morning|breakfast| +26686|AAAAAAAAPDIGAAAA|26686|7|24|46|AM|first|morning|breakfast| +26687|AAAAAAAAAEIGAAAA|26687|7|24|47|AM|first|morning|breakfast| +26688|AAAAAAAABEIGAAAA|26688|7|24|48|AM|first|morning|breakfast| +26689|AAAAAAAACEIGAAAA|26689|7|24|49|AM|first|morning|breakfast| +26690|AAAAAAAADEIGAAAA|26690|7|24|50|AM|first|morning|breakfast| +26691|AAAAAAAAEEIGAAAA|26691|7|24|51|AM|first|morning|breakfast| +26692|AAAAAAAAFEIGAAAA|26692|7|24|52|AM|first|morning|breakfast| +26693|AAAAAAAAGEIGAAAA|26693|7|24|53|AM|first|morning|breakfast| +26694|AAAAAAAAHEIGAAAA|26694|7|24|54|AM|first|morning|breakfast| +26695|AAAAAAAAIEIGAAAA|26695|7|24|55|AM|first|morning|breakfast| +26696|AAAAAAAAJEIGAAAA|26696|7|24|56|AM|first|morning|breakfast| +26697|AAAAAAAAKEIGAAAA|26697|7|24|57|AM|first|morning|breakfast| +26698|AAAAAAAALEIGAAAA|26698|7|24|58|AM|first|morning|breakfast| +26699|AAAAAAAAMEIGAAAA|26699|7|24|59|AM|first|morning|breakfast| +26700|AAAAAAAANEIGAAAA|26700|7|25|0|AM|first|morning|breakfast| +26701|AAAAAAAAOEIGAAAA|26701|7|25|1|AM|first|morning|breakfast| +26702|AAAAAAAAPEIGAAAA|26702|7|25|2|AM|first|morning|breakfast| +26703|AAAAAAAAAFIGAAAA|26703|7|25|3|AM|first|morning|breakfast| +26704|AAAAAAAABFIGAAAA|26704|7|25|4|AM|first|morning|breakfast| +26705|AAAAAAAACFIGAAAA|26705|7|25|5|AM|first|morning|breakfast| +26706|AAAAAAAADFIGAAAA|26706|7|25|6|AM|first|morning|breakfast| +26707|AAAAAAAAEFIGAAAA|26707|7|25|7|AM|first|morning|breakfast| +26708|AAAAAAAAFFIGAAAA|26708|7|25|8|AM|first|morning|breakfast| +26709|AAAAAAAAGFIGAAAA|26709|7|25|9|AM|first|morning|breakfast| +26710|AAAAAAAAHFIGAAAA|26710|7|25|10|AM|first|morning|breakfast| +26711|AAAAAAAAIFIGAAAA|26711|7|25|11|AM|first|morning|breakfast| +26712|AAAAAAAAJFIGAAAA|26712|7|25|12|AM|first|morning|breakfast| +26713|AAAAAAAAKFIGAAAA|26713|7|25|13|AM|first|morning|breakfast| +26714|AAAAAAAALFIGAAAA|26714|7|25|14|AM|first|morning|breakfast| +26715|AAAAAAAAMFIGAAAA|26715|7|25|15|AM|first|morning|breakfast| +26716|AAAAAAAANFIGAAAA|26716|7|25|16|AM|first|morning|breakfast| +26717|AAAAAAAAOFIGAAAA|26717|7|25|17|AM|first|morning|breakfast| +26718|AAAAAAAAPFIGAAAA|26718|7|25|18|AM|first|morning|breakfast| +26719|AAAAAAAAAGIGAAAA|26719|7|25|19|AM|first|morning|breakfast| +26720|AAAAAAAABGIGAAAA|26720|7|25|20|AM|first|morning|breakfast| +26721|AAAAAAAACGIGAAAA|26721|7|25|21|AM|first|morning|breakfast| +26722|AAAAAAAADGIGAAAA|26722|7|25|22|AM|first|morning|breakfast| +26723|AAAAAAAAEGIGAAAA|26723|7|25|23|AM|first|morning|breakfast| +26724|AAAAAAAAFGIGAAAA|26724|7|25|24|AM|first|morning|breakfast| +26725|AAAAAAAAGGIGAAAA|26725|7|25|25|AM|first|morning|breakfast| +26726|AAAAAAAAHGIGAAAA|26726|7|25|26|AM|first|morning|breakfast| +26727|AAAAAAAAIGIGAAAA|26727|7|25|27|AM|first|morning|breakfast| +26728|AAAAAAAAJGIGAAAA|26728|7|25|28|AM|first|morning|breakfast| +26729|AAAAAAAAKGIGAAAA|26729|7|25|29|AM|first|morning|breakfast| +26730|AAAAAAAALGIGAAAA|26730|7|25|30|AM|first|morning|breakfast| +26731|AAAAAAAAMGIGAAAA|26731|7|25|31|AM|first|morning|breakfast| +26732|AAAAAAAANGIGAAAA|26732|7|25|32|AM|first|morning|breakfast| +26733|AAAAAAAAOGIGAAAA|26733|7|25|33|AM|first|morning|breakfast| +26734|AAAAAAAAPGIGAAAA|26734|7|25|34|AM|first|morning|breakfast| +26735|AAAAAAAAAHIGAAAA|26735|7|25|35|AM|first|morning|breakfast| +26736|AAAAAAAABHIGAAAA|26736|7|25|36|AM|first|morning|breakfast| +26737|AAAAAAAACHIGAAAA|26737|7|25|37|AM|first|morning|breakfast| +26738|AAAAAAAADHIGAAAA|26738|7|25|38|AM|first|morning|breakfast| +26739|AAAAAAAAEHIGAAAA|26739|7|25|39|AM|first|morning|breakfast| +26740|AAAAAAAAFHIGAAAA|26740|7|25|40|AM|first|morning|breakfast| +26741|AAAAAAAAGHIGAAAA|26741|7|25|41|AM|first|morning|breakfast| +26742|AAAAAAAAHHIGAAAA|26742|7|25|42|AM|first|morning|breakfast| +26743|AAAAAAAAIHIGAAAA|26743|7|25|43|AM|first|morning|breakfast| +26744|AAAAAAAAJHIGAAAA|26744|7|25|44|AM|first|morning|breakfast| +26745|AAAAAAAAKHIGAAAA|26745|7|25|45|AM|first|morning|breakfast| +26746|AAAAAAAALHIGAAAA|26746|7|25|46|AM|first|morning|breakfast| +26747|AAAAAAAAMHIGAAAA|26747|7|25|47|AM|first|morning|breakfast| +26748|AAAAAAAANHIGAAAA|26748|7|25|48|AM|first|morning|breakfast| +26749|AAAAAAAAOHIGAAAA|26749|7|25|49|AM|first|morning|breakfast| +26750|AAAAAAAAPHIGAAAA|26750|7|25|50|AM|first|morning|breakfast| +26751|AAAAAAAAAIIGAAAA|26751|7|25|51|AM|first|morning|breakfast| +26752|AAAAAAAABIIGAAAA|26752|7|25|52|AM|first|morning|breakfast| +26753|AAAAAAAACIIGAAAA|26753|7|25|53|AM|first|morning|breakfast| +26754|AAAAAAAADIIGAAAA|26754|7|25|54|AM|first|morning|breakfast| +26755|AAAAAAAAEIIGAAAA|26755|7|25|55|AM|first|morning|breakfast| +26756|AAAAAAAAFIIGAAAA|26756|7|25|56|AM|first|morning|breakfast| +26757|AAAAAAAAGIIGAAAA|26757|7|25|57|AM|first|morning|breakfast| +26758|AAAAAAAAHIIGAAAA|26758|7|25|58|AM|first|morning|breakfast| +26759|AAAAAAAAIIIGAAAA|26759|7|25|59|AM|first|morning|breakfast| +26760|AAAAAAAAJIIGAAAA|26760|7|26|0|AM|first|morning|breakfast| +26761|AAAAAAAAKIIGAAAA|26761|7|26|1|AM|first|morning|breakfast| +26762|AAAAAAAALIIGAAAA|26762|7|26|2|AM|first|morning|breakfast| +26763|AAAAAAAAMIIGAAAA|26763|7|26|3|AM|first|morning|breakfast| +26764|AAAAAAAANIIGAAAA|26764|7|26|4|AM|first|morning|breakfast| +26765|AAAAAAAAOIIGAAAA|26765|7|26|5|AM|first|morning|breakfast| +26766|AAAAAAAAPIIGAAAA|26766|7|26|6|AM|first|morning|breakfast| +26767|AAAAAAAAAJIGAAAA|26767|7|26|7|AM|first|morning|breakfast| +26768|AAAAAAAABJIGAAAA|26768|7|26|8|AM|first|morning|breakfast| +26769|AAAAAAAACJIGAAAA|26769|7|26|9|AM|first|morning|breakfast| +26770|AAAAAAAADJIGAAAA|26770|7|26|10|AM|first|morning|breakfast| +26771|AAAAAAAAEJIGAAAA|26771|7|26|11|AM|first|morning|breakfast| +26772|AAAAAAAAFJIGAAAA|26772|7|26|12|AM|first|morning|breakfast| +26773|AAAAAAAAGJIGAAAA|26773|7|26|13|AM|first|morning|breakfast| +26774|AAAAAAAAHJIGAAAA|26774|7|26|14|AM|first|morning|breakfast| +26775|AAAAAAAAIJIGAAAA|26775|7|26|15|AM|first|morning|breakfast| +26776|AAAAAAAAJJIGAAAA|26776|7|26|16|AM|first|morning|breakfast| +26777|AAAAAAAAKJIGAAAA|26777|7|26|17|AM|first|morning|breakfast| +26778|AAAAAAAALJIGAAAA|26778|7|26|18|AM|first|morning|breakfast| +26779|AAAAAAAAMJIGAAAA|26779|7|26|19|AM|first|morning|breakfast| +26780|AAAAAAAANJIGAAAA|26780|7|26|20|AM|first|morning|breakfast| +26781|AAAAAAAAOJIGAAAA|26781|7|26|21|AM|first|morning|breakfast| +26782|AAAAAAAAPJIGAAAA|26782|7|26|22|AM|first|morning|breakfast| +26783|AAAAAAAAAKIGAAAA|26783|7|26|23|AM|first|morning|breakfast| +26784|AAAAAAAABKIGAAAA|26784|7|26|24|AM|first|morning|breakfast| +26785|AAAAAAAACKIGAAAA|26785|7|26|25|AM|first|morning|breakfast| +26786|AAAAAAAADKIGAAAA|26786|7|26|26|AM|first|morning|breakfast| +26787|AAAAAAAAEKIGAAAA|26787|7|26|27|AM|first|morning|breakfast| +26788|AAAAAAAAFKIGAAAA|26788|7|26|28|AM|first|morning|breakfast| +26789|AAAAAAAAGKIGAAAA|26789|7|26|29|AM|first|morning|breakfast| +26790|AAAAAAAAHKIGAAAA|26790|7|26|30|AM|first|morning|breakfast| +26791|AAAAAAAAIKIGAAAA|26791|7|26|31|AM|first|morning|breakfast| +26792|AAAAAAAAJKIGAAAA|26792|7|26|32|AM|first|morning|breakfast| +26793|AAAAAAAAKKIGAAAA|26793|7|26|33|AM|first|morning|breakfast| +26794|AAAAAAAALKIGAAAA|26794|7|26|34|AM|first|morning|breakfast| +26795|AAAAAAAAMKIGAAAA|26795|7|26|35|AM|first|morning|breakfast| +26796|AAAAAAAANKIGAAAA|26796|7|26|36|AM|first|morning|breakfast| +26797|AAAAAAAAOKIGAAAA|26797|7|26|37|AM|first|morning|breakfast| +26798|AAAAAAAAPKIGAAAA|26798|7|26|38|AM|first|morning|breakfast| +26799|AAAAAAAAALIGAAAA|26799|7|26|39|AM|first|morning|breakfast| +26800|AAAAAAAABLIGAAAA|26800|7|26|40|AM|first|morning|breakfast| +26801|AAAAAAAACLIGAAAA|26801|7|26|41|AM|first|morning|breakfast| +26802|AAAAAAAADLIGAAAA|26802|7|26|42|AM|first|morning|breakfast| +26803|AAAAAAAAELIGAAAA|26803|7|26|43|AM|first|morning|breakfast| +26804|AAAAAAAAFLIGAAAA|26804|7|26|44|AM|first|morning|breakfast| +26805|AAAAAAAAGLIGAAAA|26805|7|26|45|AM|first|morning|breakfast| +26806|AAAAAAAAHLIGAAAA|26806|7|26|46|AM|first|morning|breakfast| +26807|AAAAAAAAILIGAAAA|26807|7|26|47|AM|first|morning|breakfast| +26808|AAAAAAAAJLIGAAAA|26808|7|26|48|AM|first|morning|breakfast| +26809|AAAAAAAAKLIGAAAA|26809|7|26|49|AM|first|morning|breakfast| +26810|AAAAAAAALLIGAAAA|26810|7|26|50|AM|first|morning|breakfast| +26811|AAAAAAAAMLIGAAAA|26811|7|26|51|AM|first|morning|breakfast| +26812|AAAAAAAANLIGAAAA|26812|7|26|52|AM|first|morning|breakfast| +26813|AAAAAAAAOLIGAAAA|26813|7|26|53|AM|first|morning|breakfast| +26814|AAAAAAAAPLIGAAAA|26814|7|26|54|AM|first|morning|breakfast| +26815|AAAAAAAAAMIGAAAA|26815|7|26|55|AM|first|morning|breakfast| +26816|AAAAAAAABMIGAAAA|26816|7|26|56|AM|first|morning|breakfast| +26817|AAAAAAAACMIGAAAA|26817|7|26|57|AM|first|morning|breakfast| +26818|AAAAAAAADMIGAAAA|26818|7|26|58|AM|first|morning|breakfast| +26819|AAAAAAAAEMIGAAAA|26819|7|26|59|AM|first|morning|breakfast| +26820|AAAAAAAAFMIGAAAA|26820|7|27|0|AM|first|morning|breakfast| +26821|AAAAAAAAGMIGAAAA|26821|7|27|1|AM|first|morning|breakfast| +26822|AAAAAAAAHMIGAAAA|26822|7|27|2|AM|first|morning|breakfast| +26823|AAAAAAAAIMIGAAAA|26823|7|27|3|AM|first|morning|breakfast| +26824|AAAAAAAAJMIGAAAA|26824|7|27|4|AM|first|morning|breakfast| +26825|AAAAAAAAKMIGAAAA|26825|7|27|5|AM|first|morning|breakfast| +26826|AAAAAAAALMIGAAAA|26826|7|27|6|AM|first|morning|breakfast| +26827|AAAAAAAAMMIGAAAA|26827|7|27|7|AM|first|morning|breakfast| +26828|AAAAAAAANMIGAAAA|26828|7|27|8|AM|first|morning|breakfast| +26829|AAAAAAAAOMIGAAAA|26829|7|27|9|AM|first|morning|breakfast| +26830|AAAAAAAAPMIGAAAA|26830|7|27|10|AM|first|morning|breakfast| +26831|AAAAAAAAANIGAAAA|26831|7|27|11|AM|first|morning|breakfast| +26832|AAAAAAAABNIGAAAA|26832|7|27|12|AM|first|morning|breakfast| +26833|AAAAAAAACNIGAAAA|26833|7|27|13|AM|first|morning|breakfast| +26834|AAAAAAAADNIGAAAA|26834|7|27|14|AM|first|morning|breakfast| +26835|AAAAAAAAENIGAAAA|26835|7|27|15|AM|first|morning|breakfast| +26836|AAAAAAAAFNIGAAAA|26836|7|27|16|AM|first|morning|breakfast| +26837|AAAAAAAAGNIGAAAA|26837|7|27|17|AM|first|morning|breakfast| +26838|AAAAAAAAHNIGAAAA|26838|7|27|18|AM|first|morning|breakfast| +26839|AAAAAAAAINIGAAAA|26839|7|27|19|AM|first|morning|breakfast| +26840|AAAAAAAAJNIGAAAA|26840|7|27|20|AM|first|morning|breakfast| +26841|AAAAAAAAKNIGAAAA|26841|7|27|21|AM|first|morning|breakfast| +26842|AAAAAAAALNIGAAAA|26842|7|27|22|AM|first|morning|breakfast| +26843|AAAAAAAAMNIGAAAA|26843|7|27|23|AM|first|morning|breakfast| +26844|AAAAAAAANNIGAAAA|26844|7|27|24|AM|first|morning|breakfast| +26845|AAAAAAAAONIGAAAA|26845|7|27|25|AM|first|morning|breakfast| +26846|AAAAAAAAPNIGAAAA|26846|7|27|26|AM|first|morning|breakfast| +26847|AAAAAAAAAOIGAAAA|26847|7|27|27|AM|first|morning|breakfast| +26848|AAAAAAAABOIGAAAA|26848|7|27|28|AM|first|morning|breakfast| +26849|AAAAAAAACOIGAAAA|26849|7|27|29|AM|first|morning|breakfast| +26850|AAAAAAAADOIGAAAA|26850|7|27|30|AM|first|morning|breakfast| +26851|AAAAAAAAEOIGAAAA|26851|7|27|31|AM|first|morning|breakfast| +26852|AAAAAAAAFOIGAAAA|26852|7|27|32|AM|first|morning|breakfast| +26853|AAAAAAAAGOIGAAAA|26853|7|27|33|AM|first|morning|breakfast| +26854|AAAAAAAAHOIGAAAA|26854|7|27|34|AM|first|morning|breakfast| +26855|AAAAAAAAIOIGAAAA|26855|7|27|35|AM|first|morning|breakfast| +26856|AAAAAAAAJOIGAAAA|26856|7|27|36|AM|first|morning|breakfast| +26857|AAAAAAAAKOIGAAAA|26857|7|27|37|AM|first|morning|breakfast| +26858|AAAAAAAALOIGAAAA|26858|7|27|38|AM|first|morning|breakfast| +26859|AAAAAAAAMOIGAAAA|26859|7|27|39|AM|first|morning|breakfast| +26860|AAAAAAAANOIGAAAA|26860|7|27|40|AM|first|morning|breakfast| +26861|AAAAAAAAOOIGAAAA|26861|7|27|41|AM|first|morning|breakfast| +26862|AAAAAAAAPOIGAAAA|26862|7|27|42|AM|first|morning|breakfast| +26863|AAAAAAAAAPIGAAAA|26863|7|27|43|AM|first|morning|breakfast| +26864|AAAAAAAABPIGAAAA|26864|7|27|44|AM|first|morning|breakfast| +26865|AAAAAAAACPIGAAAA|26865|7|27|45|AM|first|morning|breakfast| +26866|AAAAAAAADPIGAAAA|26866|7|27|46|AM|first|morning|breakfast| +26867|AAAAAAAAEPIGAAAA|26867|7|27|47|AM|first|morning|breakfast| +26868|AAAAAAAAFPIGAAAA|26868|7|27|48|AM|first|morning|breakfast| +26869|AAAAAAAAGPIGAAAA|26869|7|27|49|AM|first|morning|breakfast| +26870|AAAAAAAAHPIGAAAA|26870|7|27|50|AM|first|morning|breakfast| +26871|AAAAAAAAIPIGAAAA|26871|7|27|51|AM|first|morning|breakfast| +26872|AAAAAAAAJPIGAAAA|26872|7|27|52|AM|first|morning|breakfast| +26873|AAAAAAAAKPIGAAAA|26873|7|27|53|AM|first|morning|breakfast| +26874|AAAAAAAALPIGAAAA|26874|7|27|54|AM|first|morning|breakfast| +26875|AAAAAAAAMPIGAAAA|26875|7|27|55|AM|first|morning|breakfast| +26876|AAAAAAAANPIGAAAA|26876|7|27|56|AM|first|morning|breakfast| +26877|AAAAAAAAOPIGAAAA|26877|7|27|57|AM|first|morning|breakfast| +26878|AAAAAAAAPPIGAAAA|26878|7|27|58|AM|first|morning|breakfast| +26879|AAAAAAAAAAJGAAAA|26879|7|27|59|AM|first|morning|breakfast| +26880|AAAAAAAABAJGAAAA|26880|7|28|0|AM|first|morning|breakfast| +26881|AAAAAAAACAJGAAAA|26881|7|28|1|AM|first|morning|breakfast| +26882|AAAAAAAADAJGAAAA|26882|7|28|2|AM|first|morning|breakfast| +26883|AAAAAAAAEAJGAAAA|26883|7|28|3|AM|first|morning|breakfast| +26884|AAAAAAAAFAJGAAAA|26884|7|28|4|AM|first|morning|breakfast| +26885|AAAAAAAAGAJGAAAA|26885|7|28|5|AM|first|morning|breakfast| +26886|AAAAAAAAHAJGAAAA|26886|7|28|6|AM|first|morning|breakfast| +26887|AAAAAAAAIAJGAAAA|26887|7|28|7|AM|first|morning|breakfast| +26888|AAAAAAAAJAJGAAAA|26888|7|28|8|AM|first|morning|breakfast| +26889|AAAAAAAAKAJGAAAA|26889|7|28|9|AM|first|morning|breakfast| +26890|AAAAAAAALAJGAAAA|26890|7|28|10|AM|first|morning|breakfast| +26891|AAAAAAAAMAJGAAAA|26891|7|28|11|AM|first|morning|breakfast| +26892|AAAAAAAANAJGAAAA|26892|7|28|12|AM|first|morning|breakfast| +26893|AAAAAAAAOAJGAAAA|26893|7|28|13|AM|first|morning|breakfast| +26894|AAAAAAAAPAJGAAAA|26894|7|28|14|AM|first|morning|breakfast| +26895|AAAAAAAAABJGAAAA|26895|7|28|15|AM|first|morning|breakfast| +26896|AAAAAAAABBJGAAAA|26896|7|28|16|AM|first|morning|breakfast| +26897|AAAAAAAACBJGAAAA|26897|7|28|17|AM|first|morning|breakfast| +26898|AAAAAAAADBJGAAAA|26898|7|28|18|AM|first|morning|breakfast| +26899|AAAAAAAAEBJGAAAA|26899|7|28|19|AM|first|morning|breakfast| +26900|AAAAAAAAFBJGAAAA|26900|7|28|20|AM|first|morning|breakfast| +26901|AAAAAAAAGBJGAAAA|26901|7|28|21|AM|first|morning|breakfast| +26902|AAAAAAAAHBJGAAAA|26902|7|28|22|AM|first|morning|breakfast| +26903|AAAAAAAAIBJGAAAA|26903|7|28|23|AM|first|morning|breakfast| +26904|AAAAAAAAJBJGAAAA|26904|7|28|24|AM|first|morning|breakfast| +26905|AAAAAAAAKBJGAAAA|26905|7|28|25|AM|first|morning|breakfast| +26906|AAAAAAAALBJGAAAA|26906|7|28|26|AM|first|morning|breakfast| +26907|AAAAAAAAMBJGAAAA|26907|7|28|27|AM|first|morning|breakfast| +26908|AAAAAAAANBJGAAAA|26908|7|28|28|AM|first|morning|breakfast| +26909|AAAAAAAAOBJGAAAA|26909|7|28|29|AM|first|morning|breakfast| +26910|AAAAAAAAPBJGAAAA|26910|7|28|30|AM|first|morning|breakfast| +26911|AAAAAAAAACJGAAAA|26911|7|28|31|AM|first|morning|breakfast| +26912|AAAAAAAABCJGAAAA|26912|7|28|32|AM|first|morning|breakfast| +26913|AAAAAAAACCJGAAAA|26913|7|28|33|AM|first|morning|breakfast| +26914|AAAAAAAADCJGAAAA|26914|7|28|34|AM|first|morning|breakfast| +26915|AAAAAAAAECJGAAAA|26915|7|28|35|AM|first|morning|breakfast| +26916|AAAAAAAAFCJGAAAA|26916|7|28|36|AM|first|morning|breakfast| +26917|AAAAAAAAGCJGAAAA|26917|7|28|37|AM|first|morning|breakfast| +26918|AAAAAAAAHCJGAAAA|26918|7|28|38|AM|first|morning|breakfast| +26919|AAAAAAAAICJGAAAA|26919|7|28|39|AM|first|morning|breakfast| +26920|AAAAAAAAJCJGAAAA|26920|7|28|40|AM|first|morning|breakfast| +26921|AAAAAAAAKCJGAAAA|26921|7|28|41|AM|first|morning|breakfast| +26922|AAAAAAAALCJGAAAA|26922|7|28|42|AM|first|morning|breakfast| +26923|AAAAAAAAMCJGAAAA|26923|7|28|43|AM|first|morning|breakfast| +26924|AAAAAAAANCJGAAAA|26924|7|28|44|AM|first|morning|breakfast| +26925|AAAAAAAAOCJGAAAA|26925|7|28|45|AM|first|morning|breakfast| +26926|AAAAAAAAPCJGAAAA|26926|7|28|46|AM|first|morning|breakfast| +26927|AAAAAAAAADJGAAAA|26927|7|28|47|AM|first|morning|breakfast| +26928|AAAAAAAABDJGAAAA|26928|7|28|48|AM|first|morning|breakfast| +26929|AAAAAAAACDJGAAAA|26929|7|28|49|AM|first|morning|breakfast| +26930|AAAAAAAADDJGAAAA|26930|7|28|50|AM|first|morning|breakfast| +26931|AAAAAAAAEDJGAAAA|26931|7|28|51|AM|first|morning|breakfast| +26932|AAAAAAAAFDJGAAAA|26932|7|28|52|AM|first|morning|breakfast| +26933|AAAAAAAAGDJGAAAA|26933|7|28|53|AM|first|morning|breakfast| +26934|AAAAAAAAHDJGAAAA|26934|7|28|54|AM|first|morning|breakfast| +26935|AAAAAAAAIDJGAAAA|26935|7|28|55|AM|first|morning|breakfast| +26936|AAAAAAAAJDJGAAAA|26936|7|28|56|AM|first|morning|breakfast| +26937|AAAAAAAAKDJGAAAA|26937|7|28|57|AM|first|morning|breakfast| +26938|AAAAAAAALDJGAAAA|26938|7|28|58|AM|first|morning|breakfast| +26939|AAAAAAAAMDJGAAAA|26939|7|28|59|AM|first|morning|breakfast| +26940|AAAAAAAANDJGAAAA|26940|7|29|0|AM|first|morning|breakfast| +26941|AAAAAAAAODJGAAAA|26941|7|29|1|AM|first|morning|breakfast| +26942|AAAAAAAAPDJGAAAA|26942|7|29|2|AM|first|morning|breakfast| +26943|AAAAAAAAAEJGAAAA|26943|7|29|3|AM|first|morning|breakfast| +26944|AAAAAAAABEJGAAAA|26944|7|29|4|AM|first|morning|breakfast| +26945|AAAAAAAACEJGAAAA|26945|7|29|5|AM|first|morning|breakfast| +26946|AAAAAAAADEJGAAAA|26946|7|29|6|AM|first|morning|breakfast| +26947|AAAAAAAAEEJGAAAA|26947|7|29|7|AM|first|morning|breakfast| +26948|AAAAAAAAFEJGAAAA|26948|7|29|8|AM|first|morning|breakfast| +26949|AAAAAAAAGEJGAAAA|26949|7|29|9|AM|first|morning|breakfast| +26950|AAAAAAAAHEJGAAAA|26950|7|29|10|AM|first|morning|breakfast| +26951|AAAAAAAAIEJGAAAA|26951|7|29|11|AM|first|morning|breakfast| +26952|AAAAAAAAJEJGAAAA|26952|7|29|12|AM|first|morning|breakfast| +26953|AAAAAAAAKEJGAAAA|26953|7|29|13|AM|first|morning|breakfast| +26954|AAAAAAAALEJGAAAA|26954|7|29|14|AM|first|morning|breakfast| +26955|AAAAAAAAMEJGAAAA|26955|7|29|15|AM|first|morning|breakfast| +26956|AAAAAAAANEJGAAAA|26956|7|29|16|AM|first|morning|breakfast| +26957|AAAAAAAAOEJGAAAA|26957|7|29|17|AM|first|morning|breakfast| +26958|AAAAAAAAPEJGAAAA|26958|7|29|18|AM|first|morning|breakfast| +26959|AAAAAAAAAFJGAAAA|26959|7|29|19|AM|first|morning|breakfast| +26960|AAAAAAAABFJGAAAA|26960|7|29|20|AM|first|morning|breakfast| +26961|AAAAAAAACFJGAAAA|26961|7|29|21|AM|first|morning|breakfast| +26962|AAAAAAAADFJGAAAA|26962|7|29|22|AM|first|morning|breakfast| +26963|AAAAAAAAEFJGAAAA|26963|7|29|23|AM|first|morning|breakfast| +26964|AAAAAAAAFFJGAAAA|26964|7|29|24|AM|first|morning|breakfast| +26965|AAAAAAAAGFJGAAAA|26965|7|29|25|AM|first|morning|breakfast| +26966|AAAAAAAAHFJGAAAA|26966|7|29|26|AM|first|morning|breakfast| +26967|AAAAAAAAIFJGAAAA|26967|7|29|27|AM|first|morning|breakfast| +26968|AAAAAAAAJFJGAAAA|26968|7|29|28|AM|first|morning|breakfast| +26969|AAAAAAAAKFJGAAAA|26969|7|29|29|AM|first|morning|breakfast| +26970|AAAAAAAALFJGAAAA|26970|7|29|30|AM|first|morning|breakfast| +26971|AAAAAAAAMFJGAAAA|26971|7|29|31|AM|first|morning|breakfast| +26972|AAAAAAAANFJGAAAA|26972|7|29|32|AM|first|morning|breakfast| +26973|AAAAAAAAOFJGAAAA|26973|7|29|33|AM|first|morning|breakfast| +26974|AAAAAAAAPFJGAAAA|26974|7|29|34|AM|first|morning|breakfast| +26975|AAAAAAAAAGJGAAAA|26975|7|29|35|AM|first|morning|breakfast| +26976|AAAAAAAABGJGAAAA|26976|7|29|36|AM|first|morning|breakfast| +26977|AAAAAAAACGJGAAAA|26977|7|29|37|AM|first|morning|breakfast| +26978|AAAAAAAADGJGAAAA|26978|7|29|38|AM|first|morning|breakfast| +26979|AAAAAAAAEGJGAAAA|26979|7|29|39|AM|first|morning|breakfast| +26980|AAAAAAAAFGJGAAAA|26980|7|29|40|AM|first|morning|breakfast| +26981|AAAAAAAAGGJGAAAA|26981|7|29|41|AM|first|morning|breakfast| +26982|AAAAAAAAHGJGAAAA|26982|7|29|42|AM|first|morning|breakfast| +26983|AAAAAAAAIGJGAAAA|26983|7|29|43|AM|first|morning|breakfast| +26984|AAAAAAAAJGJGAAAA|26984|7|29|44|AM|first|morning|breakfast| +26985|AAAAAAAAKGJGAAAA|26985|7|29|45|AM|first|morning|breakfast| +26986|AAAAAAAALGJGAAAA|26986|7|29|46|AM|first|morning|breakfast| +26987|AAAAAAAAMGJGAAAA|26987|7|29|47|AM|first|morning|breakfast| +26988|AAAAAAAANGJGAAAA|26988|7|29|48|AM|first|morning|breakfast| +26989|AAAAAAAAOGJGAAAA|26989|7|29|49|AM|first|morning|breakfast| +26990|AAAAAAAAPGJGAAAA|26990|7|29|50|AM|first|morning|breakfast| +26991|AAAAAAAAAHJGAAAA|26991|7|29|51|AM|first|morning|breakfast| +26992|AAAAAAAABHJGAAAA|26992|7|29|52|AM|first|morning|breakfast| +26993|AAAAAAAACHJGAAAA|26993|7|29|53|AM|first|morning|breakfast| +26994|AAAAAAAADHJGAAAA|26994|7|29|54|AM|first|morning|breakfast| +26995|AAAAAAAAEHJGAAAA|26995|7|29|55|AM|first|morning|breakfast| +26996|AAAAAAAAFHJGAAAA|26996|7|29|56|AM|first|morning|breakfast| +26997|AAAAAAAAGHJGAAAA|26997|7|29|57|AM|first|morning|breakfast| +26998|AAAAAAAAHHJGAAAA|26998|7|29|58|AM|first|morning|breakfast| +26999|AAAAAAAAIHJGAAAA|26999|7|29|59|AM|first|morning|breakfast| +27000|AAAAAAAAJHJGAAAA|27000|7|30|0|AM|first|morning|breakfast| +27001|AAAAAAAAKHJGAAAA|27001|7|30|1|AM|first|morning|breakfast| +27002|AAAAAAAALHJGAAAA|27002|7|30|2|AM|first|morning|breakfast| +27003|AAAAAAAAMHJGAAAA|27003|7|30|3|AM|first|morning|breakfast| +27004|AAAAAAAANHJGAAAA|27004|7|30|4|AM|first|morning|breakfast| +27005|AAAAAAAAOHJGAAAA|27005|7|30|5|AM|first|morning|breakfast| +27006|AAAAAAAAPHJGAAAA|27006|7|30|6|AM|first|morning|breakfast| +27007|AAAAAAAAAIJGAAAA|27007|7|30|7|AM|first|morning|breakfast| +27008|AAAAAAAABIJGAAAA|27008|7|30|8|AM|first|morning|breakfast| +27009|AAAAAAAACIJGAAAA|27009|7|30|9|AM|first|morning|breakfast| +27010|AAAAAAAADIJGAAAA|27010|7|30|10|AM|first|morning|breakfast| +27011|AAAAAAAAEIJGAAAA|27011|7|30|11|AM|first|morning|breakfast| +27012|AAAAAAAAFIJGAAAA|27012|7|30|12|AM|first|morning|breakfast| +27013|AAAAAAAAGIJGAAAA|27013|7|30|13|AM|first|morning|breakfast| +27014|AAAAAAAAHIJGAAAA|27014|7|30|14|AM|first|morning|breakfast| +27015|AAAAAAAAIIJGAAAA|27015|7|30|15|AM|first|morning|breakfast| +27016|AAAAAAAAJIJGAAAA|27016|7|30|16|AM|first|morning|breakfast| +27017|AAAAAAAAKIJGAAAA|27017|7|30|17|AM|first|morning|breakfast| +27018|AAAAAAAALIJGAAAA|27018|7|30|18|AM|first|morning|breakfast| +27019|AAAAAAAAMIJGAAAA|27019|7|30|19|AM|first|morning|breakfast| +27020|AAAAAAAANIJGAAAA|27020|7|30|20|AM|first|morning|breakfast| +27021|AAAAAAAAOIJGAAAA|27021|7|30|21|AM|first|morning|breakfast| +27022|AAAAAAAAPIJGAAAA|27022|7|30|22|AM|first|morning|breakfast| +27023|AAAAAAAAAJJGAAAA|27023|7|30|23|AM|first|morning|breakfast| +27024|AAAAAAAABJJGAAAA|27024|7|30|24|AM|first|morning|breakfast| +27025|AAAAAAAACJJGAAAA|27025|7|30|25|AM|first|morning|breakfast| +27026|AAAAAAAADJJGAAAA|27026|7|30|26|AM|first|morning|breakfast| +27027|AAAAAAAAEJJGAAAA|27027|7|30|27|AM|first|morning|breakfast| +27028|AAAAAAAAFJJGAAAA|27028|7|30|28|AM|first|morning|breakfast| +27029|AAAAAAAAGJJGAAAA|27029|7|30|29|AM|first|morning|breakfast| +27030|AAAAAAAAHJJGAAAA|27030|7|30|30|AM|first|morning|breakfast| +27031|AAAAAAAAIJJGAAAA|27031|7|30|31|AM|first|morning|breakfast| +27032|AAAAAAAAJJJGAAAA|27032|7|30|32|AM|first|morning|breakfast| +27033|AAAAAAAAKJJGAAAA|27033|7|30|33|AM|first|morning|breakfast| +27034|AAAAAAAALJJGAAAA|27034|7|30|34|AM|first|morning|breakfast| +27035|AAAAAAAAMJJGAAAA|27035|7|30|35|AM|first|morning|breakfast| +27036|AAAAAAAANJJGAAAA|27036|7|30|36|AM|first|morning|breakfast| +27037|AAAAAAAAOJJGAAAA|27037|7|30|37|AM|first|morning|breakfast| +27038|AAAAAAAAPJJGAAAA|27038|7|30|38|AM|first|morning|breakfast| +27039|AAAAAAAAAKJGAAAA|27039|7|30|39|AM|first|morning|breakfast| +27040|AAAAAAAABKJGAAAA|27040|7|30|40|AM|first|morning|breakfast| +27041|AAAAAAAACKJGAAAA|27041|7|30|41|AM|first|morning|breakfast| +27042|AAAAAAAADKJGAAAA|27042|7|30|42|AM|first|morning|breakfast| +27043|AAAAAAAAEKJGAAAA|27043|7|30|43|AM|first|morning|breakfast| +27044|AAAAAAAAFKJGAAAA|27044|7|30|44|AM|first|morning|breakfast| +27045|AAAAAAAAGKJGAAAA|27045|7|30|45|AM|first|morning|breakfast| +27046|AAAAAAAAHKJGAAAA|27046|7|30|46|AM|first|morning|breakfast| +27047|AAAAAAAAIKJGAAAA|27047|7|30|47|AM|first|morning|breakfast| +27048|AAAAAAAAJKJGAAAA|27048|7|30|48|AM|first|morning|breakfast| +27049|AAAAAAAAKKJGAAAA|27049|7|30|49|AM|first|morning|breakfast| +27050|AAAAAAAALKJGAAAA|27050|7|30|50|AM|first|morning|breakfast| +27051|AAAAAAAAMKJGAAAA|27051|7|30|51|AM|first|morning|breakfast| +27052|AAAAAAAANKJGAAAA|27052|7|30|52|AM|first|morning|breakfast| +27053|AAAAAAAAOKJGAAAA|27053|7|30|53|AM|first|morning|breakfast| +27054|AAAAAAAAPKJGAAAA|27054|7|30|54|AM|first|morning|breakfast| +27055|AAAAAAAAALJGAAAA|27055|7|30|55|AM|first|morning|breakfast| +27056|AAAAAAAABLJGAAAA|27056|7|30|56|AM|first|morning|breakfast| +27057|AAAAAAAACLJGAAAA|27057|7|30|57|AM|first|morning|breakfast| +27058|AAAAAAAADLJGAAAA|27058|7|30|58|AM|first|morning|breakfast| +27059|AAAAAAAAELJGAAAA|27059|7|30|59|AM|first|morning|breakfast| +27060|AAAAAAAAFLJGAAAA|27060|7|31|0|AM|first|morning|breakfast| +27061|AAAAAAAAGLJGAAAA|27061|7|31|1|AM|first|morning|breakfast| +27062|AAAAAAAAHLJGAAAA|27062|7|31|2|AM|first|morning|breakfast| +27063|AAAAAAAAILJGAAAA|27063|7|31|3|AM|first|morning|breakfast| +27064|AAAAAAAAJLJGAAAA|27064|7|31|4|AM|first|morning|breakfast| +27065|AAAAAAAAKLJGAAAA|27065|7|31|5|AM|first|morning|breakfast| +27066|AAAAAAAALLJGAAAA|27066|7|31|6|AM|first|morning|breakfast| +27067|AAAAAAAAMLJGAAAA|27067|7|31|7|AM|first|morning|breakfast| +27068|AAAAAAAANLJGAAAA|27068|7|31|8|AM|first|morning|breakfast| +27069|AAAAAAAAOLJGAAAA|27069|7|31|9|AM|first|morning|breakfast| +27070|AAAAAAAAPLJGAAAA|27070|7|31|10|AM|first|morning|breakfast| +27071|AAAAAAAAAMJGAAAA|27071|7|31|11|AM|first|morning|breakfast| +27072|AAAAAAAABMJGAAAA|27072|7|31|12|AM|first|morning|breakfast| +27073|AAAAAAAACMJGAAAA|27073|7|31|13|AM|first|morning|breakfast| +27074|AAAAAAAADMJGAAAA|27074|7|31|14|AM|first|morning|breakfast| +27075|AAAAAAAAEMJGAAAA|27075|7|31|15|AM|first|morning|breakfast| +27076|AAAAAAAAFMJGAAAA|27076|7|31|16|AM|first|morning|breakfast| +27077|AAAAAAAAGMJGAAAA|27077|7|31|17|AM|first|morning|breakfast| +27078|AAAAAAAAHMJGAAAA|27078|7|31|18|AM|first|morning|breakfast| +27079|AAAAAAAAIMJGAAAA|27079|7|31|19|AM|first|morning|breakfast| +27080|AAAAAAAAJMJGAAAA|27080|7|31|20|AM|first|morning|breakfast| +27081|AAAAAAAAKMJGAAAA|27081|7|31|21|AM|first|morning|breakfast| +27082|AAAAAAAALMJGAAAA|27082|7|31|22|AM|first|morning|breakfast| +27083|AAAAAAAAMMJGAAAA|27083|7|31|23|AM|first|morning|breakfast| +27084|AAAAAAAANMJGAAAA|27084|7|31|24|AM|first|morning|breakfast| +27085|AAAAAAAAOMJGAAAA|27085|7|31|25|AM|first|morning|breakfast| +27086|AAAAAAAAPMJGAAAA|27086|7|31|26|AM|first|morning|breakfast| +27087|AAAAAAAAANJGAAAA|27087|7|31|27|AM|first|morning|breakfast| +27088|AAAAAAAABNJGAAAA|27088|7|31|28|AM|first|morning|breakfast| +27089|AAAAAAAACNJGAAAA|27089|7|31|29|AM|first|morning|breakfast| +27090|AAAAAAAADNJGAAAA|27090|7|31|30|AM|first|morning|breakfast| +27091|AAAAAAAAENJGAAAA|27091|7|31|31|AM|first|morning|breakfast| +27092|AAAAAAAAFNJGAAAA|27092|7|31|32|AM|first|morning|breakfast| +27093|AAAAAAAAGNJGAAAA|27093|7|31|33|AM|first|morning|breakfast| +27094|AAAAAAAAHNJGAAAA|27094|7|31|34|AM|first|morning|breakfast| +27095|AAAAAAAAINJGAAAA|27095|7|31|35|AM|first|morning|breakfast| +27096|AAAAAAAAJNJGAAAA|27096|7|31|36|AM|first|morning|breakfast| +27097|AAAAAAAAKNJGAAAA|27097|7|31|37|AM|first|morning|breakfast| +27098|AAAAAAAALNJGAAAA|27098|7|31|38|AM|first|morning|breakfast| +27099|AAAAAAAAMNJGAAAA|27099|7|31|39|AM|first|morning|breakfast| +27100|AAAAAAAANNJGAAAA|27100|7|31|40|AM|first|morning|breakfast| +27101|AAAAAAAAONJGAAAA|27101|7|31|41|AM|first|morning|breakfast| +27102|AAAAAAAAPNJGAAAA|27102|7|31|42|AM|first|morning|breakfast| +27103|AAAAAAAAAOJGAAAA|27103|7|31|43|AM|first|morning|breakfast| +27104|AAAAAAAABOJGAAAA|27104|7|31|44|AM|first|morning|breakfast| +27105|AAAAAAAACOJGAAAA|27105|7|31|45|AM|first|morning|breakfast| +27106|AAAAAAAADOJGAAAA|27106|7|31|46|AM|first|morning|breakfast| +27107|AAAAAAAAEOJGAAAA|27107|7|31|47|AM|first|morning|breakfast| +27108|AAAAAAAAFOJGAAAA|27108|7|31|48|AM|first|morning|breakfast| +27109|AAAAAAAAGOJGAAAA|27109|7|31|49|AM|first|morning|breakfast| +27110|AAAAAAAAHOJGAAAA|27110|7|31|50|AM|first|morning|breakfast| +27111|AAAAAAAAIOJGAAAA|27111|7|31|51|AM|first|morning|breakfast| +27112|AAAAAAAAJOJGAAAA|27112|7|31|52|AM|first|morning|breakfast| +27113|AAAAAAAAKOJGAAAA|27113|7|31|53|AM|first|morning|breakfast| +27114|AAAAAAAALOJGAAAA|27114|7|31|54|AM|first|morning|breakfast| +27115|AAAAAAAAMOJGAAAA|27115|7|31|55|AM|first|morning|breakfast| +27116|AAAAAAAANOJGAAAA|27116|7|31|56|AM|first|morning|breakfast| +27117|AAAAAAAAOOJGAAAA|27117|7|31|57|AM|first|morning|breakfast| +27118|AAAAAAAAPOJGAAAA|27118|7|31|58|AM|first|morning|breakfast| +27119|AAAAAAAAAPJGAAAA|27119|7|31|59|AM|first|morning|breakfast| +27120|AAAAAAAABPJGAAAA|27120|7|32|0|AM|first|morning|breakfast| +27121|AAAAAAAACPJGAAAA|27121|7|32|1|AM|first|morning|breakfast| +27122|AAAAAAAADPJGAAAA|27122|7|32|2|AM|first|morning|breakfast| +27123|AAAAAAAAEPJGAAAA|27123|7|32|3|AM|first|morning|breakfast| +27124|AAAAAAAAFPJGAAAA|27124|7|32|4|AM|first|morning|breakfast| +27125|AAAAAAAAGPJGAAAA|27125|7|32|5|AM|first|morning|breakfast| +27126|AAAAAAAAHPJGAAAA|27126|7|32|6|AM|first|morning|breakfast| +27127|AAAAAAAAIPJGAAAA|27127|7|32|7|AM|first|morning|breakfast| +27128|AAAAAAAAJPJGAAAA|27128|7|32|8|AM|first|morning|breakfast| +27129|AAAAAAAAKPJGAAAA|27129|7|32|9|AM|first|morning|breakfast| +27130|AAAAAAAALPJGAAAA|27130|7|32|10|AM|first|morning|breakfast| +27131|AAAAAAAAMPJGAAAA|27131|7|32|11|AM|first|morning|breakfast| +27132|AAAAAAAANPJGAAAA|27132|7|32|12|AM|first|morning|breakfast| +27133|AAAAAAAAOPJGAAAA|27133|7|32|13|AM|first|morning|breakfast| +27134|AAAAAAAAPPJGAAAA|27134|7|32|14|AM|first|morning|breakfast| +27135|AAAAAAAAAAKGAAAA|27135|7|32|15|AM|first|morning|breakfast| +27136|AAAAAAAABAKGAAAA|27136|7|32|16|AM|first|morning|breakfast| +27137|AAAAAAAACAKGAAAA|27137|7|32|17|AM|first|morning|breakfast| +27138|AAAAAAAADAKGAAAA|27138|7|32|18|AM|first|morning|breakfast| +27139|AAAAAAAAEAKGAAAA|27139|7|32|19|AM|first|morning|breakfast| +27140|AAAAAAAAFAKGAAAA|27140|7|32|20|AM|first|morning|breakfast| +27141|AAAAAAAAGAKGAAAA|27141|7|32|21|AM|first|morning|breakfast| +27142|AAAAAAAAHAKGAAAA|27142|7|32|22|AM|first|morning|breakfast| +27143|AAAAAAAAIAKGAAAA|27143|7|32|23|AM|first|morning|breakfast| +27144|AAAAAAAAJAKGAAAA|27144|7|32|24|AM|first|morning|breakfast| +27145|AAAAAAAAKAKGAAAA|27145|7|32|25|AM|first|morning|breakfast| +27146|AAAAAAAALAKGAAAA|27146|7|32|26|AM|first|morning|breakfast| +27147|AAAAAAAAMAKGAAAA|27147|7|32|27|AM|first|morning|breakfast| +27148|AAAAAAAANAKGAAAA|27148|7|32|28|AM|first|morning|breakfast| +27149|AAAAAAAAOAKGAAAA|27149|7|32|29|AM|first|morning|breakfast| +27150|AAAAAAAAPAKGAAAA|27150|7|32|30|AM|first|morning|breakfast| +27151|AAAAAAAAABKGAAAA|27151|7|32|31|AM|first|morning|breakfast| +27152|AAAAAAAABBKGAAAA|27152|7|32|32|AM|first|morning|breakfast| +27153|AAAAAAAACBKGAAAA|27153|7|32|33|AM|first|morning|breakfast| +27154|AAAAAAAADBKGAAAA|27154|7|32|34|AM|first|morning|breakfast| +27155|AAAAAAAAEBKGAAAA|27155|7|32|35|AM|first|morning|breakfast| +27156|AAAAAAAAFBKGAAAA|27156|7|32|36|AM|first|morning|breakfast| +27157|AAAAAAAAGBKGAAAA|27157|7|32|37|AM|first|morning|breakfast| +27158|AAAAAAAAHBKGAAAA|27158|7|32|38|AM|first|morning|breakfast| +27159|AAAAAAAAIBKGAAAA|27159|7|32|39|AM|first|morning|breakfast| +27160|AAAAAAAAJBKGAAAA|27160|7|32|40|AM|first|morning|breakfast| +27161|AAAAAAAAKBKGAAAA|27161|7|32|41|AM|first|morning|breakfast| +27162|AAAAAAAALBKGAAAA|27162|7|32|42|AM|first|morning|breakfast| +27163|AAAAAAAAMBKGAAAA|27163|7|32|43|AM|first|morning|breakfast| +27164|AAAAAAAANBKGAAAA|27164|7|32|44|AM|first|morning|breakfast| +27165|AAAAAAAAOBKGAAAA|27165|7|32|45|AM|first|morning|breakfast| +27166|AAAAAAAAPBKGAAAA|27166|7|32|46|AM|first|morning|breakfast| +27167|AAAAAAAAACKGAAAA|27167|7|32|47|AM|first|morning|breakfast| +27168|AAAAAAAABCKGAAAA|27168|7|32|48|AM|first|morning|breakfast| +27169|AAAAAAAACCKGAAAA|27169|7|32|49|AM|first|morning|breakfast| +27170|AAAAAAAADCKGAAAA|27170|7|32|50|AM|first|morning|breakfast| +27171|AAAAAAAAECKGAAAA|27171|7|32|51|AM|first|morning|breakfast| +27172|AAAAAAAAFCKGAAAA|27172|7|32|52|AM|first|morning|breakfast| +27173|AAAAAAAAGCKGAAAA|27173|7|32|53|AM|first|morning|breakfast| +27174|AAAAAAAAHCKGAAAA|27174|7|32|54|AM|first|morning|breakfast| +27175|AAAAAAAAICKGAAAA|27175|7|32|55|AM|first|morning|breakfast| +27176|AAAAAAAAJCKGAAAA|27176|7|32|56|AM|first|morning|breakfast| +27177|AAAAAAAAKCKGAAAA|27177|7|32|57|AM|first|morning|breakfast| +27178|AAAAAAAALCKGAAAA|27178|7|32|58|AM|first|morning|breakfast| +27179|AAAAAAAAMCKGAAAA|27179|7|32|59|AM|first|morning|breakfast| +27180|AAAAAAAANCKGAAAA|27180|7|33|0|AM|first|morning|breakfast| +27181|AAAAAAAAOCKGAAAA|27181|7|33|1|AM|first|morning|breakfast| +27182|AAAAAAAAPCKGAAAA|27182|7|33|2|AM|first|morning|breakfast| +27183|AAAAAAAAADKGAAAA|27183|7|33|3|AM|first|morning|breakfast| +27184|AAAAAAAABDKGAAAA|27184|7|33|4|AM|first|morning|breakfast| +27185|AAAAAAAACDKGAAAA|27185|7|33|5|AM|first|morning|breakfast| +27186|AAAAAAAADDKGAAAA|27186|7|33|6|AM|first|morning|breakfast| +27187|AAAAAAAAEDKGAAAA|27187|7|33|7|AM|first|morning|breakfast| +27188|AAAAAAAAFDKGAAAA|27188|7|33|8|AM|first|morning|breakfast| +27189|AAAAAAAAGDKGAAAA|27189|7|33|9|AM|first|morning|breakfast| +27190|AAAAAAAAHDKGAAAA|27190|7|33|10|AM|first|morning|breakfast| +27191|AAAAAAAAIDKGAAAA|27191|7|33|11|AM|first|morning|breakfast| +27192|AAAAAAAAJDKGAAAA|27192|7|33|12|AM|first|morning|breakfast| +27193|AAAAAAAAKDKGAAAA|27193|7|33|13|AM|first|morning|breakfast| +27194|AAAAAAAALDKGAAAA|27194|7|33|14|AM|first|morning|breakfast| +27195|AAAAAAAAMDKGAAAA|27195|7|33|15|AM|first|morning|breakfast| +27196|AAAAAAAANDKGAAAA|27196|7|33|16|AM|first|morning|breakfast| +27197|AAAAAAAAODKGAAAA|27197|7|33|17|AM|first|morning|breakfast| +27198|AAAAAAAAPDKGAAAA|27198|7|33|18|AM|first|morning|breakfast| +27199|AAAAAAAAAEKGAAAA|27199|7|33|19|AM|first|morning|breakfast| +27200|AAAAAAAABEKGAAAA|27200|7|33|20|AM|first|morning|breakfast| +27201|AAAAAAAACEKGAAAA|27201|7|33|21|AM|first|morning|breakfast| +27202|AAAAAAAADEKGAAAA|27202|7|33|22|AM|first|morning|breakfast| +27203|AAAAAAAAEEKGAAAA|27203|7|33|23|AM|first|morning|breakfast| +27204|AAAAAAAAFEKGAAAA|27204|7|33|24|AM|first|morning|breakfast| +27205|AAAAAAAAGEKGAAAA|27205|7|33|25|AM|first|morning|breakfast| +27206|AAAAAAAAHEKGAAAA|27206|7|33|26|AM|first|morning|breakfast| +27207|AAAAAAAAIEKGAAAA|27207|7|33|27|AM|first|morning|breakfast| +27208|AAAAAAAAJEKGAAAA|27208|7|33|28|AM|first|morning|breakfast| +27209|AAAAAAAAKEKGAAAA|27209|7|33|29|AM|first|morning|breakfast| +27210|AAAAAAAALEKGAAAA|27210|7|33|30|AM|first|morning|breakfast| +27211|AAAAAAAAMEKGAAAA|27211|7|33|31|AM|first|morning|breakfast| +27212|AAAAAAAANEKGAAAA|27212|7|33|32|AM|first|morning|breakfast| +27213|AAAAAAAAOEKGAAAA|27213|7|33|33|AM|first|morning|breakfast| +27214|AAAAAAAAPEKGAAAA|27214|7|33|34|AM|first|morning|breakfast| +27215|AAAAAAAAAFKGAAAA|27215|7|33|35|AM|first|morning|breakfast| +27216|AAAAAAAABFKGAAAA|27216|7|33|36|AM|first|morning|breakfast| +27217|AAAAAAAACFKGAAAA|27217|7|33|37|AM|first|morning|breakfast| +27218|AAAAAAAADFKGAAAA|27218|7|33|38|AM|first|morning|breakfast| +27219|AAAAAAAAEFKGAAAA|27219|7|33|39|AM|first|morning|breakfast| +27220|AAAAAAAAFFKGAAAA|27220|7|33|40|AM|first|morning|breakfast| +27221|AAAAAAAAGFKGAAAA|27221|7|33|41|AM|first|morning|breakfast| +27222|AAAAAAAAHFKGAAAA|27222|7|33|42|AM|first|morning|breakfast| +27223|AAAAAAAAIFKGAAAA|27223|7|33|43|AM|first|morning|breakfast| +27224|AAAAAAAAJFKGAAAA|27224|7|33|44|AM|first|morning|breakfast| +27225|AAAAAAAAKFKGAAAA|27225|7|33|45|AM|first|morning|breakfast| +27226|AAAAAAAALFKGAAAA|27226|7|33|46|AM|first|morning|breakfast| +27227|AAAAAAAAMFKGAAAA|27227|7|33|47|AM|first|morning|breakfast| +27228|AAAAAAAANFKGAAAA|27228|7|33|48|AM|first|morning|breakfast| +27229|AAAAAAAAOFKGAAAA|27229|7|33|49|AM|first|morning|breakfast| +27230|AAAAAAAAPFKGAAAA|27230|7|33|50|AM|first|morning|breakfast| +27231|AAAAAAAAAGKGAAAA|27231|7|33|51|AM|first|morning|breakfast| +27232|AAAAAAAABGKGAAAA|27232|7|33|52|AM|first|morning|breakfast| +27233|AAAAAAAACGKGAAAA|27233|7|33|53|AM|first|morning|breakfast| +27234|AAAAAAAADGKGAAAA|27234|7|33|54|AM|first|morning|breakfast| +27235|AAAAAAAAEGKGAAAA|27235|7|33|55|AM|first|morning|breakfast| +27236|AAAAAAAAFGKGAAAA|27236|7|33|56|AM|first|morning|breakfast| +27237|AAAAAAAAGGKGAAAA|27237|7|33|57|AM|first|morning|breakfast| +27238|AAAAAAAAHGKGAAAA|27238|7|33|58|AM|first|morning|breakfast| +27239|AAAAAAAAIGKGAAAA|27239|7|33|59|AM|first|morning|breakfast| +27240|AAAAAAAAJGKGAAAA|27240|7|34|0|AM|first|morning|breakfast| +27241|AAAAAAAAKGKGAAAA|27241|7|34|1|AM|first|morning|breakfast| +27242|AAAAAAAALGKGAAAA|27242|7|34|2|AM|first|morning|breakfast| +27243|AAAAAAAAMGKGAAAA|27243|7|34|3|AM|first|morning|breakfast| +27244|AAAAAAAANGKGAAAA|27244|7|34|4|AM|first|morning|breakfast| +27245|AAAAAAAAOGKGAAAA|27245|7|34|5|AM|first|morning|breakfast| +27246|AAAAAAAAPGKGAAAA|27246|7|34|6|AM|first|morning|breakfast| +27247|AAAAAAAAAHKGAAAA|27247|7|34|7|AM|first|morning|breakfast| +27248|AAAAAAAABHKGAAAA|27248|7|34|8|AM|first|morning|breakfast| +27249|AAAAAAAACHKGAAAA|27249|7|34|9|AM|first|morning|breakfast| +27250|AAAAAAAADHKGAAAA|27250|7|34|10|AM|first|morning|breakfast| +27251|AAAAAAAAEHKGAAAA|27251|7|34|11|AM|first|morning|breakfast| +27252|AAAAAAAAFHKGAAAA|27252|7|34|12|AM|first|morning|breakfast| +27253|AAAAAAAAGHKGAAAA|27253|7|34|13|AM|first|morning|breakfast| +27254|AAAAAAAAHHKGAAAA|27254|7|34|14|AM|first|morning|breakfast| +27255|AAAAAAAAIHKGAAAA|27255|7|34|15|AM|first|morning|breakfast| +27256|AAAAAAAAJHKGAAAA|27256|7|34|16|AM|first|morning|breakfast| +27257|AAAAAAAAKHKGAAAA|27257|7|34|17|AM|first|morning|breakfast| +27258|AAAAAAAALHKGAAAA|27258|7|34|18|AM|first|morning|breakfast| +27259|AAAAAAAAMHKGAAAA|27259|7|34|19|AM|first|morning|breakfast| +27260|AAAAAAAANHKGAAAA|27260|7|34|20|AM|first|morning|breakfast| +27261|AAAAAAAAOHKGAAAA|27261|7|34|21|AM|first|morning|breakfast| +27262|AAAAAAAAPHKGAAAA|27262|7|34|22|AM|first|morning|breakfast| +27263|AAAAAAAAAIKGAAAA|27263|7|34|23|AM|first|morning|breakfast| +27264|AAAAAAAABIKGAAAA|27264|7|34|24|AM|first|morning|breakfast| +27265|AAAAAAAACIKGAAAA|27265|7|34|25|AM|first|morning|breakfast| +27266|AAAAAAAADIKGAAAA|27266|7|34|26|AM|first|morning|breakfast| +27267|AAAAAAAAEIKGAAAA|27267|7|34|27|AM|first|morning|breakfast| +27268|AAAAAAAAFIKGAAAA|27268|7|34|28|AM|first|morning|breakfast| +27269|AAAAAAAAGIKGAAAA|27269|7|34|29|AM|first|morning|breakfast| +27270|AAAAAAAAHIKGAAAA|27270|7|34|30|AM|first|morning|breakfast| +27271|AAAAAAAAIIKGAAAA|27271|7|34|31|AM|first|morning|breakfast| +27272|AAAAAAAAJIKGAAAA|27272|7|34|32|AM|first|morning|breakfast| +27273|AAAAAAAAKIKGAAAA|27273|7|34|33|AM|first|morning|breakfast| +27274|AAAAAAAALIKGAAAA|27274|7|34|34|AM|first|morning|breakfast| +27275|AAAAAAAAMIKGAAAA|27275|7|34|35|AM|first|morning|breakfast| +27276|AAAAAAAANIKGAAAA|27276|7|34|36|AM|first|morning|breakfast| +27277|AAAAAAAAOIKGAAAA|27277|7|34|37|AM|first|morning|breakfast| +27278|AAAAAAAAPIKGAAAA|27278|7|34|38|AM|first|morning|breakfast| +27279|AAAAAAAAAJKGAAAA|27279|7|34|39|AM|first|morning|breakfast| +27280|AAAAAAAABJKGAAAA|27280|7|34|40|AM|first|morning|breakfast| +27281|AAAAAAAACJKGAAAA|27281|7|34|41|AM|first|morning|breakfast| +27282|AAAAAAAADJKGAAAA|27282|7|34|42|AM|first|morning|breakfast| +27283|AAAAAAAAEJKGAAAA|27283|7|34|43|AM|first|morning|breakfast| +27284|AAAAAAAAFJKGAAAA|27284|7|34|44|AM|first|morning|breakfast| +27285|AAAAAAAAGJKGAAAA|27285|7|34|45|AM|first|morning|breakfast| +27286|AAAAAAAAHJKGAAAA|27286|7|34|46|AM|first|morning|breakfast| +27287|AAAAAAAAIJKGAAAA|27287|7|34|47|AM|first|morning|breakfast| +27288|AAAAAAAAJJKGAAAA|27288|7|34|48|AM|first|morning|breakfast| +27289|AAAAAAAAKJKGAAAA|27289|7|34|49|AM|first|morning|breakfast| +27290|AAAAAAAALJKGAAAA|27290|7|34|50|AM|first|morning|breakfast| +27291|AAAAAAAAMJKGAAAA|27291|7|34|51|AM|first|morning|breakfast| +27292|AAAAAAAANJKGAAAA|27292|7|34|52|AM|first|morning|breakfast| +27293|AAAAAAAAOJKGAAAA|27293|7|34|53|AM|first|morning|breakfast| +27294|AAAAAAAAPJKGAAAA|27294|7|34|54|AM|first|morning|breakfast| +27295|AAAAAAAAAKKGAAAA|27295|7|34|55|AM|first|morning|breakfast| +27296|AAAAAAAABKKGAAAA|27296|7|34|56|AM|first|morning|breakfast| +27297|AAAAAAAACKKGAAAA|27297|7|34|57|AM|first|morning|breakfast| +27298|AAAAAAAADKKGAAAA|27298|7|34|58|AM|first|morning|breakfast| +27299|AAAAAAAAEKKGAAAA|27299|7|34|59|AM|first|morning|breakfast| +27300|AAAAAAAAFKKGAAAA|27300|7|35|0|AM|first|morning|breakfast| +27301|AAAAAAAAGKKGAAAA|27301|7|35|1|AM|first|morning|breakfast| +27302|AAAAAAAAHKKGAAAA|27302|7|35|2|AM|first|morning|breakfast| +27303|AAAAAAAAIKKGAAAA|27303|7|35|3|AM|first|morning|breakfast| +27304|AAAAAAAAJKKGAAAA|27304|7|35|4|AM|first|morning|breakfast| +27305|AAAAAAAAKKKGAAAA|27305|7|35|5|AM|first|morning|breakfast| +27306|AAAAAAAALKKGAAAA|27306|7|35|6|AM|first|morning|breakfast| +27307|AAAAAAAAMKKGAAAA|27307|7|35|7|AM|first|morning|breakfast| +27308|AAAAAAAANKKGAAAA|27308|7|35|8|AM|first|morning|breakfast| +27309|AAAAAAAAOKKGAAAA|27309|7|35|9|AM|first|morning|breakfast| +27310|AAAAAAAAPKKGAAAA|27310|7|35|10|AM|first|morning|breakfast| +27311|AAAAAAAAALKGAAAA|27311|7|35|11|AM|first|morning|breakfast| +27312|AAAAAAAABLKGAAAA|27312|7|35|12|AM|first|morning|breakfast| +27313|AAAAAAAACLKGAAAA|27313|7|35|13|AM|first|morning|breakfast| +27314|AAAAAAAADLKGAAAA|27314|7|35|14|AM|first|morning|breakfast| +27315|AAAAAAAAELKGAAAA|27315|7|35|15|AM|first|morning|breakfast| +27316|AAAAAAAAFLKGAAAA|27316|7|35|16|AM|first|morning|breakfast| +27317|AAAAAAAAGLKGAAAA|27317|7|35|17|AM|first|morning|breakfast| +27318|AAAAAAAAHLKGAAAA|27318|7|35|18|AM|first|morning|breakfast| +27319|AAAAAAAAILKGAAAA|27319|7|35|19|AM|first|morning|breakfast| +27320|AAAAAAAAJLKGAAAA|27320|7|35|20|AM|first|morning|breakfast| +27321|AAAAAAAAKLKGAAAA|27321|7|35|21|AM|first|morning|breakfast| +27322|AAAAAAAALLKGAAAA|27322|7|35|22|AM|first|morning|breakfast| +27323|AAAAAAAAMLKGAAAA|27323|7|35|23|AM|first|morning|breakfast| +27324|AAAAAAAANLKGAAAA|27324|7|35|24|AM|first|morning|breakfast| +27325|AAAAAAAAOLKGAAAA|27325|7|35|25|AM|first|morning|breakfast| +27326|AAAAAAAAPLKGAAAA|27326|7|35|26|AM|first|morning|breakfast| +27327|AAAAAAAAAMKGAAAA|27327|7|35|27|AM|first|morning|breakfast| +27328|AAAAAAAABMKGAAAA|27328|7|35|28|AM|first|morning|breakfast| +27329|AAAAAAAACMKGAAAA|27329|7|35|29|AM|first|morning|breakfast| +27330|AAAAAAAADMKGAAAA|27330|7|35|30|AM|first|morning|breakfast| +27331|AAAAAAAAEMKGAAAA|27331|7|35|31|AM|first|morning|breakfast| +27332|AAAAAAAAFMKGAAAA|27332|7|35|32|AM|first|morning|breakfast| +27333|AAAAAAAAGMKGAAAA|27333|7|35|33|AM|first|morning|breakfast| +27334|AAAAAAAAHMKGAAAA|27334|7|35|34|AM|first|morning|breakfast| +27335|AAAAAAAAIMKGAAAA|27335|7|35|35|AM|first|morning|breakfast| +27336|AAAAAAAAJMKGAAAA|27336|7|35|36|AM|first|morning|breakfast| +27337|AAAAAAAAKMKGAAAA|27337|7|35|37|AM|first|morning|breakfast| +27338|AAAAAAAALMKGAAAA|27338|7|35|38|AM|first|morning|breakfast| +27339|AAAAAAAAMMKGAAAA|27339|7|35|39|AM|first|morning|breakfast| +27340|AAAAAAAANMKGAAAA|27340|7|35|40|AM|first|morning|breakfast| +27341|AAAAAAAAOMKGAAAA|27341|7|35|41|AM|first|morning|breakfast| +27342|AAAAAAAAPMKGAAAA|27342|7|35|42|AM|first|morning|breakfast| +27343|AAAAAAAAANKGAAAA|27343|7|35|43|AM|first|morning|breakfast| +27344|AAAAAAAABNKGAAAA|27344|7|35|44|AM|first|morning|breakfast| +27345|AAAAAAAACNKGAAAA|27345|7|35|45|AM|first|morning|breakfast| +27346|AAAAAAAADNKGAAAA|27346|7|35|46|AM|first|morning|breakfast| +27347|AAAAAAAAENKGAAAA|27347|7|35|47|AM|first|morning|breakfast| +27348|AAAAAAAAFNKGAAAA|27348|7|35|48|AM|first|morning|breakfast| +27349|AAAAAAAAGNKGAAAA|27349|7|35|49|AM|first|morning|breakfast| +27350|AAAAAAAAHNKGAAAA|27350|7|35|50|AM|first|morning|breakfast| +27351|AAAAAAAAINKGAAAA|27351|7|35|51|AM|first|morning|breakfast| +27352|AAAAAAAAJNKGAAAA|27352|7|35|52|AM|first|morning|breakfast| +27353|AAAAAAAAKNKGAAAA|27353|7|35|53|AM|first|morning|breakfast| +27354|AAAAAAAALNKGAAAA|27354|7|35|54|AM|first|morning|breakfast| +27355|AAAAAAAAMNKGAAAA|27355|7|35|55|AM|first|morning|breakfast| +27356|AAAAAAAANNKGAAAA|27356|7|35|56|AM|first|morning|breakfast| +27357|AAAAAAAAONKGAAAA|27357|7|35|57|AM|first|morning|breakfast| +27358|AAAAAAAAPNKGAAAA|27358|7|35|58|AM|first|morning|breakfast| +27359|AAAAAAAAAOKGAAAA|27359|7|35|59|AM|first|morning|breakfast| +27360|AAAAAAAABOKGAAAA|27360|7|36|0|AM|first|morning|breakfast| +27361|AAAAAAAACOKGAAAA|27361|7|36|1|AM|first|morning|breakfast| +27362|AAAAAAAADOKGAAAA|27362|7|36|2|AM|first|morning|breakfast| +27363|AAAAAAAAEOKGAAAA|27363|7|36|3|AM|first|morning|breakfast| +27364|AAAAAAAAFOKGAAAA|27364|7|36|4|AM|first|morning|breakfast| +27365|AAAAAAAAGOKGAAAA|27365|7|36|5|AM|first|morning|breakfast| +27366|AAAAAAAAHOKGAAAA|27366|7|36|6|AM|first|morning|breakfast| +27367|AAAAAAAAIOKGAAAA|27367|7|36|7|AM|first|morning|breakfast| +27368|AAAAAAAAJOKGAAAA|27368|7|36|8|AM|first|morning|breakfast| +27369|AAAAAAAAKOKGAAAA|27369|7|36|9|AM|first|morning|breakfast| +27370|AAAAAAAALOKGAAAA|27370|7|36|10|AM|first|morning|breakfast| +27371|AAAAAAAAMOKGAAAA|27371|7|36|11|AM|first|morning|breakfast| +27372|AAAAAAAANOKGAAAA|27372|7|36|12|AM|first|morning|breakfast| +27373|AAAAAAAAOOKGAAAA|27373|7|36|13|AM|first|morning|breakfast| +27374|AAAAAAAAPOKGAAAA|27374|7|36|14|AM|first|morning|breakfast| +27375|AAAAAAAAAPKGAAAA|27375|7|36|15|AM|first|morning|breakfast| +27376|AAAAAAAABPKGAAAA|27376|7|36|16|AM|first|morning|breakfast| +27377|AAAAAAAACPKGAAAA|27377|7|36|17|AM|first|morning|breakfast| +27378|AAAAAAAADPKGAAAA|27378|7|36|18|AM|first|morning|breakfast| +27379|AAAAAAAAEPKGAAAA|27379|7|36|19|AM|first|morning|breakfast| +27380|AAAAAAAAFPKGAAAA|27380|7|36|20|AM|first|morning|breakfast| +27381|AAAAAAAAGPKGAAAA|27381|7|36|21|AM|first|morning|breakfast| +27382|AAAAAAAAHPKGAAAA|27382|7|36|22|AM|first|morning|breakfast| +27383|AAAAAAAAIPKGAAAA|27383|7|36|23|AM|first|morning|breakfast| +27384|AAAAAAAAJPKGAAAA|27384|7|36|24|AM|first|morning|breakfast| +27385|AAAAAAAAKPKGAAAA|27385|7|36|25|AM|first|morning|breakfast| +27386|AAAAAAAALPKGAAAA|27386|7|36|26|AM|first|morning|breakfast| +27387|AAAAAAAAMPKGAAAA|27387|7|36|27|AM|first|morning|breakfast| +27388|AAAAAAAANPKGAAAA|27388|7|36|28|AM|first|morning|breakfast| +27389|AAAAAAAAOPKGAAAA|27389|7|36|29|AM|first|morning|breakfast| +27390|AAAAAAAAPPKGAAAA|27390|7|36|30|AM|first|morning|breakfast| +27391|AAAAAAAAAALGAAAA|27391|7|36|31|AM|first|morning|breakfast| +27392|AAAAAAAABALGAAAA|27392|7|36|32|AM|first|morning|breakfast| +27393|AAAAAAAACALGAAAA|27393|7|36|33|AM|first|morning|breakfast| +27394|AAAAAAAADALGAAAA|27394|7|36|34|AM|first|morning|breakfast| +27395|AAAAAAAAEALGAAAA|27395|7|36|35|AM|first|morning|breakfast| +27396|AAAAAAAAFALGAAAA|27396|7|36|36|AM|first|morning|breakfast| +27397|AAAAAAAAGALGAAAA|27397|7|36|37|AM|first|morning|breakfast| +27398|AAAAAAAAHALGAAAA|27398|7|36|38|AM|first|morning|breakfast| +27399|AAAAAAAAIALGAAAA|27399|7|36|39|AM|first|morning|breakfast| +27400|AAAAAAAAJALGAAAA|27400|7|36|40|AM|first|morning|breakfast| +27401|AAAAAAAAKALGAAAA|27401|7|36|41|AM|first|morning|breakfast| +27402|AAAAAAAALALGAAAA|27402|7|36|42|AM|first|morning|breakfast| +27403|AAAAAAAAMALGAAAA|27403|7|36|43|AM|first|morning|breakfast| +27404|AAAAAAAANALGAAAA|27404|7|36|44|AM|first|morning|breakfast| +27405|AAAAAAAAOALGAAAA|27405|7|36|45|AM|first|morning|breakfast| +27406|AAAAAAAAPALGAAAA|27406|7|36|46|AM|first|morning|breakfast| +27407|AAAAAAAAABLGAAAA|27407|7|36|47|AM|first|morning|breakfast| +27408|AAAAAAAABBLGAAAA|27408|7|36|48|AM|first|morning|breakfast| +27409|AAAAAAAACBLGAAAA|27409|7|36|49|AM|first|morning|breakfast| +27410|AAAAAAAADBLGAAAA|27410|7|36|50|AM|first|morning|breakfast| +27411|AAAAAAAAEBLGAAAA|27411|7|36|51|AM|first|morning|breakfast| +27412|AAAAAAAAFBLGAAAA|27412|7|36|52|AM|first|morning|breakfast| +27413|AAAAAAAAGBLGAAAA|27413|7|36|53|AM|first|morning|breakfast| +27414|AAAAAAAAHBLGAAAA|27414|7|36|54|AM|first|morning|breakfast| +27415|AAAAAAAAIBLGAAAA|27415|7|36|55|AM|first|morning|breakfast| +27416|AAAAAAAAJBLGAAAA|27416|7|36|56|AM|first|morning|breakfast| +27417|AAAAAAAAKBLGAAAA|27417|7|36|57|AM|first|morning|breakfast| +27418|AAAAAAAALBLGAAAA|27418|7|36|58|AM|first|morning|breakfast| +27419|AAAAAAAAMBLGAAAA|27419|7|36|59|AM|first|morning|breakfast| +27420|AAAAAAAANBLGAAAA|27420|7|37|0|AM|first|morning|breakfast| +27421|AAAAAAAAOBLGAAAA|27421|7|37|1|AM|first|morning|breakfast| +27422|AAAAAAAAPBLGAAAA|27422|7|37|2|AM|first|morning|breakfast| +27423|AAAAAAAAACLGAAAA|27423|7|37|3|AM|first|morning|breakfast| +27424|AAAAAAAABCLGAAAA|27424|7|37|4|AM|first|morning|breakfast| +27425|AAAAAAAACCLGAAAA|27425|7|37|5|AM|first|morning|breakfast| +27426|AAAAAAAADCLGAAAA|27426|7|37|6|AM|first|morning|breakfast| +27427|AAAAAAAAECLGAAAA|27427|7|37|7|AM|first|morning|breakfast| +27428|AAAAAAAAFCLGAAAA|27428|7|37|8|AM|first|morning|breakfast| +27429|AAAAAAAAGCLGAAAA|27429|7|37|9|AM|first|morning|breakfast| +27430|AAAAAAAAHCLGAAAA|27430|7|37|10|AM|first|morning|breakfast| +27431|AAAAAAAAICLGAAAA|27431|7|37|11|AM|first|morning|breakfast| +27432|AAAAAAAAJCLGAAAA|27432|7|37|12|AM|first|morning|breakfast| +27433|AAAAAAAAKCLGAAAA|27433|7|37|13|AM|first|morning|breakfast| +27434|AAAAAAAALCLGAAAA|27434|7|37|14|AM|first|morning|breakfast| +27435|AAAAAAAAMCLGAAAA|27435|7|37|15|AM|first|morning|breakfast| +27436|AAAAAAAANCLGAAAA|27436|7|37|16|AM|first|morning|breakfast| +27437|AAAAAAAAOCLGAAAA|27437|7|37|17|AM|first|morning|breakfast| +27438|AAAAAAAAPCLGAAAA|27438|7|37|18|AM|first|morning|breakfast| +27439|AAAAAAAAADLGAAAA|27439|7|37|19|AM|first|morning|breakfast| +27440|AAAAAAAABDLGAAAA|27440|7|37|20|AM|first|morning|breakfast| +27441|AAAAAAAACDLGAAAA|27441|7|37|21|AM|first|morning|breakfast| +27442|AAAAAAAADDLGAAAA|27442|7|37|22|AM|first|morning|breakfast| +27443|AAAAAAAAEDLGAAAA|27443|7|37|23|AM|first|morning|breakfast| +27444|AAAAAAAAFDLGAAAA|27444|7|37|24|AM|first|morning|breakfast| +27445|AAAAAAAAGDLGAAAA|27445|7|37|25|AM|first|morning|breakfast| +27446|AAAAAAAAHDLGAAAA|27446|7|37|26|AM|first|morning|breakfast| +27447|AAAAAAAAIDLGAAAA|27447|7|37|27|AM|first|morning|breakfast| +27448|AAAAAAAAJDLGAAAA|27448|7|37|28|AM|first|morning|breakfast| +27449|AAAAAAAAKDLGAAAA|27449|7|37|29|AM|first|morning|breakfast| +27450|AAAAAAAALDLGAAAA|27450|7|37|30|AM|first|morning|breakfast| +27451|AAAAAAAAMDLGAAAA|27451|7|37|31|AM|first|morning|breakfast| +27452|AAAAAAAANDLGAAAA|27452|7|37|32|AM|first|morning|breakfast| +27453|AAAAAAAAODLGAAAA|27453|7|37|33|AM|first|morning|breakfast| +27454|AAAAAAAAPDLGAAAA|27454|7|37|34|AM|first|morning|breakfast| +27455|AAAAAAAAAELGAAAA|27455|7|37|35|AM|first|morning|breakfast| +27456|AAAAAAAABELGAAAA|27456|7|37|36|AM|first|morning|breakfast| +27457|AAAAAAAACELGAAAA|27457|7|37|37|AM|first|morning|breakfast| +27458|AAAAAAAADELGAAAA|27458|7|37|38|AM|first|morning|breakfast| +27459|AAAAAAAAEELGAAAA|27459|7|37|39|AM|first|morning|breakfast| +27460|AAAAAAAAFELGAAAA|27460|7|37|40|AM|first|morning|breakfast| +27461|AAAAAAAAGELGAAAA|27461|7|37|41|AM|first|morning|breakfast| +27462|AAAAAAAAHELGAAAA|27462|7|37|42|AM|first|morning|breakfast| +27463|AAAAAAAAIELGAAAA|27463|7|37|43|AM|first|morning|breakfast| +27464|AAAAAAAAJELGAAAA|27464|7|37|44|AM|first|morning|breakfast| +27465|AAAAAAAAKELGAAAA|27465|7|37|45|AM|first|morning|breakfast| +27466|AAAAAAAALELGAAAA|27466|7|37|46|AM|first|morning|breakfast| +27467|AAAAAAAAMELGAAAA|27467|7|37|47|AM|first|morning|breakfast| +27468|AAAAAAAANELGAAAA|27468|7|37|48|AM|first|morning|breakfast| +27469|AAAAAAAAOELGAAAA|27469|7|37|49|AM|first|morning|breakfast| +27470|AAAAAAAAPELGAAAA|27470|7|37|50|AM|first|morning|breakfast| +27471|AAAAAAAAAFLGAAAA|27471|7|37|51|AM|first|morning|breakfast| +27472|AAAAAAAABFLGAAAA|27472|7|37|52|AM|first|morning|breakfast| +27473|AAAAAAAACFLGAAAA|27473|7|37|53|AM|first|morning|breakfast| +27474|AAAAAAAADFLGAAAA|27474|7|37|54|AM|first|morning|breakfast| +27475|AAAAAAAAEFLGAAAA|27475|7|37|55|AM|first|morning|breakfast| +27476|AAAAAAAAFFLGAAAA|27476|7|37|56|AM|first|morning|breakfast| +27477|AAAAAAAAGFLGAAAA|27477|7|37|57|AM|first|morning|breakfast| +27478|AAAAAAAAHFLGAAAA|27478|7|37|58|AM|first|morning|breakfast| +27479|AAAAAAAAIFLGAAAA|27479|7|37|59|AM|first|morning|breakfast| +27480|AAAAAAAAJFLGAAAA|27480|7|38|0|AM|first|morning|breakfast| +27481|AAAAAAAAKFLGAAAA|27481|7|38|1|AM|first|morning|breakfast| +27482|AAAAAAAALFLGAAAA|27482|7|38|2|AM|first|morning|breakfast| +27483|AAAAAAAAMFLGAAAA|27483|7|38|3|AM|first|morning|breakfast| +27484|AAAAAAAANFLGAAAA|27484|7|38|4|AM|first|morning|breakfast| +27485|AAAAAAAAOFLGAAAA|27485|7|38|5|AM|first|morning|breakfast| +27486|AAAAAAAAPFLGAAAA|27486|7|38|6|AM|first|morning|breakfast| +27487|AAAAAAAAAGLGAAAA|27487|7|38|7|AM|first|morning|breakfast| +27488|AAAAAAAABGLGAAAA|27488|7|38|8|AM|first|morning|breakfast| +27489|AAAAAAAACGLGAAAA|27489|7|38|9|AM|first|morning|breakfast| +27490|AAAAAAAADGLGAAAA|27490|7|38|10|AM|first|morning|breakfast| +27491|AAAAAAAAEGLGAAAA|27491|7|38|11|AM|first|morning|breakfast| +27492|AAAAAAAAFGLGAAAA|27492|7|38|12|AM|first|morning|breakfast| +27493|AAAAAAAAGGLGAAAA|27493|7|38|13|AM|first|morning|breakfast| +27494|AAAAAAAAHGLGAAAA|27494|7|38|14|AM|first|morning|breakfast| +27495|AAAAAAAAIGLGAAAA|27495|7|38|15|AM|first|morning|breakfast| +27496|AAAAAAAAJGLGAAAA|27496|7|38|16|AM|first|morning|breakfast| +27497|AAAAAAAAKGLGAAAA|27497|7|38|17|AM|first|morning|breakfast| +27498|AAAAAAAALGLGAAAA|27498|7|38|18|AM|first|morning|breakfast| +27499|AAAAAAAAMGLGAAAA|27499|7|38|19|AM|first|morning|breakfast| +27500|AAAAAAAANGLGAAAA|27500|7|38|20|AM|first|morning|breakfast| +27501|AAAAAAAAOGLGAAAA|27501|7|38|21|AM|first|morning|breakfast| +27502|AAAAAAAAPGLGAAAA|27502|7|38|22|AM|first|morning|breakfast| +27503|AAAAAAAAAHLGAAAA|27503|7|38|23|AM|first|morning|breakfast| +27504|AAAAAAAABHLGAAAA|27504|7|38|24|AM|first|morning|breakfast| +27505|AAAAAAAACHLGAAAA|27505|7|38|25|AM|first|morning|breakfast| +27506|AAAAAAAADHLGAAAA|27506|7|38|26|AM|first|morning|breakfast| +27507|AAAAAAAAEHLGAAAA|27507|7|38|27|AM|first|morning|breakfast| +27508|AAAAAAAAFHLGAAAA|27508|7|38|28|AM|first|morning|breakfast| +27509|AAAAAAAAGHLGAAAA|27509|7|38|29|AM|first|morning|breakfast| +27510|AAAAAAAAHHLGAAAA|27510|7|38|30|AM|first|morning|breakfast| +27511|AAAAAAAAIHLGAAAA|27511|7|38|31|AM|first|morning|breakfast| +27512|AAAAAAAAJHLGAAAA|27512|7|38|32|AM|first|morning|breakfast| +27513|AAAAAAAAKHLGAAAA|27513|7|38|33|AM|first|morning|breakfast| +27514|AAAAAAAALHLGAAAA|27514|7|38|34|AM|first|morning|breakfast| +27515|AAAAAAAAMHLGAAAA|27515|7|38|35|AM|first|morning|breakfast| +27516|AAAAAAAANHLGAAAA|27516|7|38|36|AM|first|morning|breakfast| +27517|AAAAAAAAOHLGAAAA|27517|7|38|37|AM|first|morning|breakfast| +27518|AAAAAAAAPHLGAAAA|27518|7|38|38|AM|first|morning|breakfast| +27519|AAAAAAAAAILGAAAA|27519|7|38|39|AM|first|morning|breakfast| +27520|AAAAAAAABILGAAAA|27520|7|38|40|AM|first|morning|breakfast| +27521|AAAAAAAACILGAAAA|27521|7|38|41|AM|first|morning|breakfast| +27522|AAAAAAAADILGAAAA|27522|7|38|42|AM|first|morning|breakfast| +27523|AAAAAAAAEILGAAAA|27523|7|38|43|AM|first|morning|breakfast| +27524|AAAAAAAAFILGAAAA|27524|7|38|44|AM|first|morning|breakfast| +27525|AAAAAAAAGILGAAAA|27525|7|38|45|AM|first|morning|breakfast| +27526|AAAAAAAAHILGAAAA|27526|7|38|46|AM|first|morning|breakfast| +27527|AAAAAAAAIILGAAAA|27527|7|38|47|AM|first|morning|breakfast| +27528|AAAAAAAAJILGAAAA|27528|7|38|48|AM|first|morning|breakfast| +27529|AAAAAAAAKILGAAAA|27529|7|38|49|AM|first|morning|breakfast| +27530|AAAAAAAALILGAAAA|27530|7|38|50|AM|first|morning|breakfast| +27531|AAAAAAAAMILGAAAA|27531|7|38|51|AM|first|morning|breakfast| +27532|AAAAAAAANILGAAAA|27532|7|38|52|AM|first|morning|breakfast| +27533|AAAAAAAAOILGAAAA|27533|7|38|53|AM|first|morning|breakfast| +27534|AAAAAAAAPILGAAAA|27534|7|38|54|AM|first|morning|breakfast| +27535|AAAAAAAAAJLGAAAA|27535|7|38|55|AM|first|morning|breakfast| +27536|AAAAAAAABJLGAAAA|27536|7|38|56|AM|first|morning|breakfast| +27537|AAAAAAAACJLGAAAA|27537|7|38|57|AM|first|morning|breakfast| +27538|AAAAAAAADJLGAAAA|27538|7|38|58|AM|first|morning|breakfast| +27539|AAAAAAAAEJLGAAAA|27539|7|38|59|AM|first|morning|breakfast| +27540|AAAAAAAAFJLGAAAA|27540|7|39|0|AM|first|morning|breakfast| +27541|AAAAAAAAGJLGAAAA|27541|7|39|1|AM|first|morning|breakfast| +27542|AAAAAAAAHJLGAAAA|27542|7|39|2|AM|first|morning|breakfast| +27543|AAAAAAAAIJLGAAAA|27543|7|39|3|AM|first|morning|breakfast| +27544|AAAAAAAAJJLGAAAA|27544|7|39|4|AM|first|morning|breakfast| +27545|AAAAAAAAKJLGAAAA|27545|7|39|5|AM|first|morning|breakfast| +27546|AAAAAAAALJLGAAAA|27546|7|39|6|AM|first|morning|breakfast| +27547|AAAAAAAAMJLGAAAA|27547|7|39|7|AM|first|morning|breakfast| +27548|AAAAAAAANJLGAAAA|27548|7|39|8|AM|first|morning|breakfast| +27549|AAAAAAAAOJLGAAAA|27549|7|39|9|AM|first|morning|breakfast| +27550|AAAAAAAAPJLGAAAA|27550|7|39|10|AM|first|morning|breakfast| +27551|AAAAAAAAAKLGAAAA|27551|7|39|11|AM|first|morning|breakfast| +27552|AAAAAAAABKLGAAAA|27552|7|39|12|AM|first|morning|breakfast| +27553|AAAAAAAACKLGAAAA|27553|7|39|13|AM|first|morning|breakfast| +27554|AAAAAAAADKLGAAAA|27554|7|39|14|AM|first|morning|breakfast| +27555|AAAAAAAAEKLGAAAA|27555|7|39|15|AM|first|morning|breakfast| +27556|AAAAAAAAFKLGAAAA|27556|7|39|16|AM|first|morning|breakfast| +27557|AAAAAAAAGKLGAAAA|27557|7|39|17|AM|first|morning|breakfast| +27558|AAAAAAAAHKLGAAAA|27558|7|39|18|AM|first|morning|breakfast| +27559|AAAAAAAAIKLGAAAA|27559|7|39|19|AM|first|morning|breakfast| +27560|AAAAAAAAJKLGAAAA|27560|7|39|20|AM|first|morning|breakfast| +27561|AAAAAAAAKKLGAAAA|27561|7|39|21|AM|first|morning|breakfast| +27562|AAAAAAAALKLGAAAA|27562|7|39|22|AM|first|morning|breakfast| +27563|AAAAAAAAMKLGAAAA|27563|7|39|23|AM|first|morning|breakfast| +27564|AAAAAAAANKLGAAAA|27564|7|39|24|AM|first|morning|breakfast| +27565|AAAAAAAAOKLGAAAA|27565|7|39|25|AM|first|morning|breakfast| +27566|AAAAAAAAPKLGAAAA|27566|7|39|26|AM|first|morning|breakfast| +27567|AAAAAAAAALLGAAAA|27567|7|39|27|AM|first|morning|breakfast| +27568|AAAAAAAABLLGAAAA|27568|7|39|28|AM|first|morning|breakfast| +27569|AAAAAAAACLLGAAAA|27569|7|39|29|AM|first|morning|breakfast| +27570|AAAAAAAADLLGAAAA|27570|7|39|30|AM|first|morning|breakfast| +27571|AAAAAAAAELLGAAAA|27571|7|39|31|AM|first|morning|breakfast| +27572|AAAAAAAAFLLGAAAA|27572|7|39|32|AM|first|morning|breakfast| +27573|AAAAAAAAGLLGAAAA|27573|7|39|33|AM|first|morning|breakfast| +27574|AAAAAAAAHLLGAAAA|27574|7|39|34|AM|first|morning|breakfast| +27575|AAAAAAAAILLGAAAA|27575|7|39|35|AM|first|morning|breakfast| +27576|AAAAAAAAJLLGAAAA|27576|7|39|36|AM|first|morning|breakfast| +27577|AAAAAAAAKLLGAAAA|27577|7|39|37|AM|first|morning|breakfast| +27578|AAAAAAAALLLGAAAA|27578|7|39|38|AM|first|morning|breakfast| +27579|AAAAAAAAMLLGAAAA|27579|7|39|39|AM|first|morning|breakfast| +27580|AAAAAAAANLLGAAAA|27580|7|39|40|AM|first|morning|breakfast| +27581|AAAAAAAAOLLGAAAA|27581|7|39|41|AM|first|morning|breakfast| +27582|AAAAAAAAPLLGAAAA|27582|7|39|42|AM|first|morning|breakfast| +27583|AAAAAAAAAMLGAAAA|27583|7|39|43|AM|first|morning|breakfast| +27584|AAAAAAAABMLGAAAA|27584|7|39|44|AM|first|morning|breakfast| +27585|AAAAAAAACMLGAAAA|27585|7|39|45|AM|first|morning|breakfast| +27586|AAAAAAAADMLGAAAA|27586|7|39|46|AM|first|morning|breakfast| +27587|AAAAAAAAEMLGAAAA|27587|7|39|47|AM|first|morning|breakfast| +27588|AAAAAAAAFMLGAAAA|27588|7|39|48|AM|first|morning|breakfast| +27589|AAAAAAAAGMLGAAAA|27589|7|39|49|AM|first|morning|breakfast| +27590|AAAAAAAAHMLGAAAA|27590|7|39|50|AM|first|morning|breakfast| +27591|AAAAAAAAIMLGAAAA|27591|7|39|51|AM|first|morning|breakfast| +27592|AAAAAAAAJMLGAAAA|27592|7|39|52|AM|first|morning|breakfast| +27593|AAAAAAAAKMLGAAAA|27593|7|39|53|AM|first|morning|breakfast| +27594|AAAAAAAALMLGAAAA|27594|7|39|54|AM|first|morning|breakfast| +27595|AAAAAAAAMMLGAAAA|27595|7|39|55|AM|first|morning|breakfast| +27596|AAAAAAAANMLGAAAA|27596|7|39|56|AM|first|morning|breakfast| +27597|AAAAAAAAOMLGAAAA|27597|7|39|57|AM|first|morning|breakfast| +27598|AAAAAAAAPMLGAAAA|27598|7|39|58|AM|first|morning|breakfast| +27599|AAAAAAAAANLGAAAA|27599|7|39|59|AM|first|morning|breakfast| +27600|AAAAAAAABNLGAAAA|27600|7|40|0|AM|first|morning|breakfast| +27601|AAAAAAAACNLGAAAA|27601|7|40|1|AM|first|morning|breakfast| +27602|AAAAAAAADNLGAAAA|27602|7|40|2|AM|first|morning|breakfast| +27603|AAAAAAAAENLGAAAA|27603|7|40|3|AM|first|morning|breakfast| +27604|AAAAAAAAFNLGAAAA|27604|7|40|4|AM|first|morning|breakfast| +27605|AAAAAAAAGNLGAAAA|27605|7|40|5|AM|first|morning|breakfast| +27606|AAAAAAAAHNLGAAAA|27606|7|40|6|AM|first|morning|breakfast| +27607|AAAAAAAAINLGAAAA|27607|7|40|7|AM|first|morning|breakfast| +27608|AAAAAAAAJNLGAAAA|27608|7|40|8|AM|first|morning|breakfast| +27609|AAAAAAAAKNLGAAAA|27609|7|40|9|AM|first|morning|breakfast| +27610|AAAAAAAALNLGAAAA|27610|7|40|10|AM|first|morning|breakfast| +27611|AAAAAAAAMNLGAAAA|27611|7|40|11|AM|first|morning|breakfast| +27612|AAAAAAAANNLGAAAA|27612|7|40|12|AM|first|morning|breakfast| +27613|AAAAAAAAONLGAAAA|27613|7|40|13|AM|first|morning|breakfast| +27614|AAAAAAAAPNLGAAAA|27614|7|40|14|AM|first|morning|breakfast| +27615|AAAAAAAAAOLGAAAA|27615|7|40|15|AM|first|morning|breakfast| +27616|AAAAAAAABOLGAAAA|27616|7|40|16|AM|first|morning|breakfast| +27617|AAAAAAAACOLGAAAA|27617|7|40|17|AM|first|morning|breakfast| +27618|AAAAAAAADOLGAAAA|27618|7|40|18|AM|first|morning|breakfast| +27619|AAAAAAAAEOLGAAAA|27619|7|40|19|AM|first|morning|breakfast| +27620|AAAAAAAAFOLGAAAA|27620|7|40|20|AM|first|morning|breakfast| +27621|AAAAAAAAGOLGAAAA|27621|7|40|21|AM|first|morning|breakfast| +27622|AAAAAAAAHOLGAAAA|27622|7|40|22|AM|first|morning|breakfast| +27623|AAAAAAAAIOLGAAAA|27623|7|40|23|AM|first|morning|breakfast| +27624|AAAAAAAAJOLGAAAA|27624|7|40|24|AM|first|morning|breakfast| +27625|AAAAAAAAKOLGAAAA|27625|7|40|25|AM|first|morning|breakfast| +27626|AAAAAAAALOLGAAAA|27626|7|40|26|AM|first|morning|breakfast| +27627|AAAAAAAAMOLGAAAA|27627|7|40|27|AM|first|morning|breakfast| +27628|AAAAAAAANOLGAAAA|27628|7|40|28|AM|first|morning|breakfast| +27629|AAAAAAAAOOLGAAAA|27629|7|40|29|AM|first|morning|breakfast| +27630|AAAAAAAAPOLGAAAA|27630|7|40|30|AM|first|morning|breakfast| +27631|AAAAAAAAAPLGAAAA|27631|7|40|31|AM|first|morning|breakfast| +27632|AAAAAAAABPLGAAAA|27632|7|40|32|AM|first|morning|breakfast| +27633|AAAAAAAACPLGAAAA|27633|7|40|33|AM|first|morning|breakfast| +27634|AAAAAAAADPLGAAAA|27634|7|40|34|AM|first|morning|breakfast| +27635|AAAAAAAAEPLGAAAA|27635|7|40|35|AM|first|morning|breakfast| +27636|AAAAAAAAFPLGAAAA|27636|7|40|36|AM|first|morning|breakfast| +27637|AAAAAAAAGPLGAAAA|27637|7|40|37|AM|first|morning|breakfast| +27638|AAAAAAAAHPLGAAAA|27638|7|40|38|AM|first|morning|breakfast| +27639|AAAAAAAAIPLGAAAA|27639|7|40|39|AM|first|morning|breakfast| +27640|AAAAAAAAJPLGAAAA|27640|7|40|40|AM|first|morning|breakfast| +27641|AAAAAAAAKPLGAAAA|27641|7|40|41|AM|first|morning|breakfast| +27642|AAAAAAAALPLGAAAA|27642|7|40|42|AM|first|morning|breakfast| +27643|AAAAAAAAMPLGAAAA|27643|7|40|43|AM|first|morning|breakfast| +27644|AAAAAAAANPLGAAAA|27644|7|40|44|AM|first|morning|breakfast| +27645|AAAAAAAAOPLGAAAA|27645|7|40|45|AM|first|morning|breakfast| +27646|AAAAAAAAPPLGAAAA|27646|7|40|46|AM|first|morning|breakfast| +27647|AAAAAAAAAAMGAAAA|27647|7|40|47|AM|first|morning|breakfast| +27648|AAAAAAAABAMGAAAA|27648|7|40|48|AM|first|morning|breakfast| +27649|AAAAAAAACAMGAAAA|27649|7|40|49|AM|first|morning|breakfast| +27650|AAAAAAAADAMGAAAA|27650|7|40|50|AM|first|morning|breakfast| +27651|AAAAAAAAEAMGAAAA|27651|7|40|51|AM|first|morning|breakfast| +27652|AAAAAAAAFAMGAAAA|27652|7|40|52|AM|first|morning|breakfast| +27653|AAAAAAAAGAMGAAAA|27653|7|40|53|AM|first|morning|breakfast| +27654|AAAAAAAAHAMGAAAA|27654|7|40|54|AM|first|morning|breakfast| +27655|AAAAAAAAIAMGAAAA|27655|7|40|55|AM|first|morning|breakfast| +27656|AAAAAAAAJAMGAAAA|27656|7|40|56|AM|first|morning|breakfast| +27657|AAAAAAAAKAMGAAAA|27657|7|40|57|AM|first|morning|breakfast| +27658|AAAAAAAALAMGAAAA|27658|7|40|58|AM|first|morning|breakfast| +27659|AAAAAAAAMAMGAAAA|27659|7|40|59|AM|first|morning|breakfast| +27660|AAAAAAAANAMGAAAA|27660|7|41|0|AM|first|morning|breakfast| +27661|AAAAAAAAOAMGAAAA|27661|7|41|1|AM|first|morning|breakfast| +27662|AAAAAAAAPAMGAAAA|27662|7|41|2|AM|first|morning|breakfast| +27663|AAAAAAAAABMGAAAA|27663|7|41|3|AM|first|morning|breakfast| +27664|AAAAAAAABBMGAAAA|27664|7|41|4|AM|first|morning|breakfast| +27665|AAAAAAAACBMGAAAA|27665|7|41|5|AM|first|morning|breakfast| +27666|AAAAAAAADBMGAAAA|27666|7|41|6|AM|first|morning|breakfast| +27667|AAAAAAAAEBMGAAAA|27667|7|41|7|AM|first|morning|breakfast| +27668|AAAAAAAAFBMGAAAA|27668|7|41|8|AM|first|morning|breakfast| +27669|AAAAAAAAGBMGAAAA|27669|7|41|9|AM|first|morning|breakfast| +27670|AAAAAAAAHBMGAAAA|27670|7|41|10|AM|first|morning|breakfast| +27671|AAAAAAAAIBMGAAAA|27671|7|41|11|AM|first|morning|breakfast| +27672|AAAAAAAAJBMGAAAA|27672|7|41|12|AM|first|morning|breakfast| +27673|AAAAAAAAKBMGAAAA|27673|7|41|13|AM|first|morning|breakfast| +27674|AAAAAAAALBMGAAAA|27674|7|41|14|AM|first|morning|breakfast| +27675|AAAAAAAAMBMGAAAA|27675|7|41|15|AM|first|morning|breakfast| +27676|AAAAAAAANBMGAAAA|27676|7|41|16|AM|first|morning|breakfast| +27677|AAAAAAAAOBMGAAAA|27677|7|41|17|AM|first|morning|breakfast| +27678|AAAAAAAAPBMGAAAA|27678|7|41|18|AM|first|morning|breakfast| +27679|AAAAAAAAACMGAAAA|27679|7|41|19|AM|first|morning|breakfast| +27680|AAAAAAAABCMGAAAA|27680|7|41|20|AM|first|morning|breakfast| +27681|AAAAAAAACCMGAAAA|27681|7|41|21|AM|first|morning|breakfast| +27682|AAAAAAAADCMGAAAA|27682|7|41|22|AM|first|morning|breakfast| +27683|AAAAAAAAECMGAAAA|27683|7|41|23|AM|first|morning|breakfast| +27684|AAAAAAAAFCMGAAAA|27684|7|41|24|AM|first|morning|breakfast| +27685|AAAAAAAAGCMGAAAA|27685|7|41|25|AM|first|morning|breakfast| +27686|AAAAAAAAHCMGAAAA|27686|7|41|26|AM|first|morning|breakfast| +27687|AAAAAAAAICMGAAAA|27687|7|41|27|AM|first|morning|breakfast| +27688|AAAAAAAAJCMGAAAA|27688|7|41|28|AM|first|morning|breakfast| +27689|AAAAAAAAKCMGAAAA|27689|7|41|29|AM|first|morning|breakfast| +27690|AAAAAAAALCMGAAAA|27690|7|41|30|AM|first|morning|breakfast| +27691|AAAAAAAAMCMGAAAA|27691|7|41|31|AM|first|morning|breakfast| +27692|AAAAAAAANCMGAAAA|27692|7|41|32|AM|first|morning|breakfast| +27693|AAAAAAAAOCMGAAAA|27693|7|41|33|AM|first|morning|breakfast| +27694|AAAAAAAAPCMGAAAA|27694|7|41|34|AM|first|morning|breakfast| +27695|AAAAAAAAADMGAAAA|27695|7|41|35|AM|first|morning|breakfast| +27696|AAAAAAAABDMGAAAA|27696|7|41|36|AM|first|morning|breakfast| +27697|AAAAAAAACDMGAAAA|27697|7|41|37|AM|first|morning|breakfast| +27698|AAAAAAAADDMGAAAA|27698|7|41|38|AM|first|morning|breakfast| +27699|AAAAAAAAEDMGAAAA|27699|7|41|39|AM|first|morning|breakfast| +27700|AAAAAAAAFDMGAAAA|27700|7|41|40|AM|first|morning|breakfast| +27701|AAAAAAAAGDMGAAAA|27701|7|41|41|AM|first|morning|breakfast| +27702|AAAAAAAAHDMGAAAA|27702|7|41|42|AM|first|morning|breakfast| +27703|AAAAAAAAIDMGAAAA|27703|7|41|43|AM|first|morning|breakfast| +27704|AAAAAAAAJDMGAAAA|27704|7|41|44|AM|first|morning|breakfast| +27705|AAAAAAAAKDMGAAAA|27705|7|41|45|AM|first|morning|breakfast| +27706|AAAAAAAALDMGAAAA|27706|7|41|46|AM|first|morning|breakfast| +27707|AAAAAAAAMDMGAAAA|27707|7|41|47|AM|first|morning|breakfast| +27708|AAAAAAAANDMGAAAA|27708|7|41|48|AM|first|morning|breakfast| +27709|AAAAAAAAODMGAAAA|27709|7|41|49|AM|first|morning|breakfast| +27710|AAAAAAAAPDMGAAAA|27710|7|41|50|AM|first|morning|breakfast| +27711|AAAAAAAAAEMGAAAA|27711|7|41|51|AM|first|morning|breakfast| +27712|AAAAAAAABEMGAAAA|27712|7|41|52|AM|first|morning|breakfast| +27713|AAAAAAAACEMGAAAA|27713|7|41|53|AM|first|morning|breakfast| +27714|AAAAAAAADEMGAAAA|27714|7|41|54|AM|first|morning|breakfast| +27715|AAAAAAAAEEMGAAAA|27715|7|41|55|AM|first|morning|breakfast| +27716|AAAAAAAAFEMGAAAA|27716|7|41|56|AM|first|morning|breakfast| +27717|AAAAAAAAGEMGAAAA|27717|7|41|57|AM|first|morning|breakfast| +27718|AAAAAAAAHEMGAAAA|27718|7|41|58|AM|first|morning|breakfast| +27719|AAAAAAAAIEMGAAAA|27719|7|41|59|AM|first|morning|breakfast| +27720|AAAAAAAAJEMGAAAA|27720|7|42|0|AM|first|morning|breakfast| +27721|AAAAAAAAKEMGAAAA|27721|7|42|1|AM|first|morning|breakfast| +27722|AAAAAAAALEMGAAAA|27722|7|42|2|AM|first|morning|breakfast| +27723|AAAAAAAAMEMGAAAA|27723|7|42|3|AM|first|morning|breakfast| +27724|AAAAAAAANEMGAAAA|27724|7|42|4|AM|first|morning|breakfast| +27725|AAAAAAAAOEMGAAAA|27725|7|42|5|AM|first|morning|breakfast| +27726|AAAAAAAAPEMGAAAA|27726|7|42|6|AM|first|morning|breakfast| +27727|AAAAAAAAAFMGAAAA|27727|7|42|7|AM|first|morning|breakfast| +27728|AAAAAAAABFMGAAAA|27728|7|42|8|AM|first|morning|breakfast| +27729|AAAAAAAACFMGAAAA|27729|7|42|9|AM|first|morning|breakfast| +27730|AAAAAAAADFMGAAAA|27730|7|42|10|AM|first|morning|breakfast| +27731|AAAAAAAAEFMGAAAA|27731|7|42|11|AM|first|morning|breakfast| +27732|AAAAAAAAFFMGAAAA|27732|7|42|12|AM|first|morning|breakfast| +27733|AAAAAAAAGFMGAAAA|27733|7|42|13|AM|first|morning|breakfast| +27734|AAAAAAAAHFMGAAAA|27734|7|42|14|AM|first|morning|breakfast| +27735|AAAAAAAAIFMGAAAA|27735|7|42|15|AM|first|morning|breakfast| +27736|AAAAAAAAJFMGAAAA|27736|7|42|16|AM|first|morning|breakfast| +27737|AAAAAAAAKFMGAAAA|27737|7|42|17|AM|first|morning|breakfast| +27738|AAAAAAAALFMGAAAA|27738|7|42|18|AM|first|morning|breakfast| +27739|AAAAAAAAMFMGAAAA|27739|7|42|19|AM|first|morning|breakfast| +27740|AAAAAAAANFMGAAAA|27740|7|42|20|AM|first|morning|breakfast| +27741|AAAAAAAAOFMGAAAA|27741|7|42|21|AM|first|morning|breakfast| +27742|AAAAAAAAPFMGAAAA|27742|7|42|22|AM|first|morning|breakfast| +27743|AAAAAAAAAGMGAAAA|27743|7|42|23|AM|first|morning|breakfast| +27744|AAAAAAAABGMGAAAA|27744|7|42|24|AM|first|morning|breakfast| +27745|AAAAAAAACGMGAAAA|27745|7|42|25|AM|first|morning|breakfast| +27746|AAAAAAAADGMGAAAA|27746|7|42|26|AM|first|morning|breakfast| +27747|AAAAAAAAEGMGAAAA|27747|7|42|27|AM|first|morning|breakfast| +27748|AAAAAAAAFGMGAAAA|27748|7|42|28|AM|first|morning|breakfast| +27749|AAAAAAAAGGMGAAAA|27749|7|42|29|AM|first|morning|breakfast| +27750|AAAAAAAAHGMGAAAA|27750|7|42|30|AM|first|morning|breakfast| +27751|AAAAAAAAIGMGAAAA|27751|7|42|31|AM|first|morning|breakfast| +27752|AAAAAAAAJGMGAAAA|27752|7|42|32|AM|first|morning|breakfast| +27753|AAAAAAAAKGMGAAAA|27753|7|42|33|AM|first|morning|breakfast| +27754|AAAAAAAALGMGAAAA|27754|7|42|34|AM|first|morning|breakfast| +27755|AAAAAAAAMGMGAAAA|27755|7|42|35|AM|first|morning|breakfast| +27756|AAAAAAAANGMGAAAA|27756|7|42|36|AM|first|morning|breakfast| +27757|AAAAAAAAOGMGAAAA|27757|7|42|37|AM|first|morning|breakfast| +27758|AAAAAAAAPGMGAAAA|27758|7|42|38|AM|first|morning|breakfast| +27759|AAAAAAAAAHMGAAAA|27759|7|42|39|AM|first|morning|breakfast| +27760|AAAAAAAABHMGAAAA|27760|7|42|40|AM|first|morning|breakfast| +27761|AAAAAAAACHMGAAAA|27761|7|42|41|AM|first|morning|breakfast| +27762|AAAAAAAADHMGAAAA|27762|7|42|42|AM|first|morning|breakfast| +27763|AAAAAAAAEHMGAAAA|27763|7|42|43|AM|first|morning|breakfast| +27764|AAAAAAAAFHMGAAAA|27764|7|42|44|AM|first|morning|breakfast| +27765|AAAAAAAAGHMGAAAA|27765|7|42|45|AM|first|morning|breakfast| +27766|AAAAAAAAHHMGAAAA|27766|7|42|46|AM|first|morning|breakfast| +27767|AAAAAAAAIHMGAAAA|27767|7|42|47|AM|first|morning|breakfast| +27768|AAAAAAAAJHMGAAAA|27768|7|42|48|AM|first|morning|breakfast| +27769|AAAAAAAAKHMGAAAA|27769|7|42|49|AM|first|morning|breakfast| +27770|AAAAAAAALHMGAAAA|27770|7|42|50|AM|first|morning|breakfast| +27771|AAAAAAAAMHMGAAAA|27771|7|42|51|AM|first|morning|breakfast| +27772|AAAAAAAANHMGAAAA|27772|7|42|52|AM|first|morning|breakfast| +27773|AAAAAAAAOHMGAAAA|27773|7|42|53|AM|first|morning|breakfast| +27774|AAAAAAAAPHMGAAAA|27774|7|42|54|AM|first|morning|breakfast| +27775|AAAAAAAAAIMGAAAA|27775|7|42|55|AM|first|morning|breakfast| +27776|AAAAAAAABIMGAAAA|27776|7|42|56|AM|first|morning|breakfast| +27777|AAAAAAAACIMGAAAA|27777|7|42|57|AM|first|morning|breakfast| +27778|AAAAAAAADIMGAAAA|27778|7|42|58|AM|first|morning|breakfast| +27779|AAAAAAAAEIMGAAAA|27779|7|42|59|AM|first|morning|breakfast| +27780|AAAAAAAAFIMGAAAA|27780|7|43|0|AM|first|morning|breakfast| +27781|AAAAAAAAGIMGAAAA|27781|7|43|1|AM|first|morning|breakfast| +27782|AAAAAAAAHIMGAAAA|27782|7|43|2|AM|first|morning|breakfast| +27783|AAAAAAAAIIMGAAAA|27783|7|43|3|AM|first|morning|breakfast| +27784|AAAAAAAAJIMGAAAA|27784|7|43|4|AM|first|morning|breakfast| +27785|AAAAAAAAKIMGAAAA|27785|7|43|5|AM|first|morning|breakfast| +27786|AAAAAAAALIMGAAAA|27786|7|43|6|AM|first|morning|breakfast| +27787|AAAAAAAAMIMGAAAA|27787|7|43|7|AM|first|morning|breakfast| +27788|AAAAAAAANIMGAAAA|27788|7|43|8|AM|first|morning|breakfast| +27789|AAAAAAAAOIMGAAAA|27789|7|43|9|AM|first|morning|breakfast| +27790|AAAAAAAAPIMGAAAA|27790|7|43|10|AM|first|morning|breakfast| +27791|AAAAAAAAAJMGAAAA|27791|7|43|11|AM|first|morning|breakfast| +27792|AAAAAAAABJMGAAAA|27792|7|43|12|AM|first|morning|breakfast| +27793|AAAAAAAACJMGAAAA|27793|7|43|13|AM|first|morning|breakfast| +27794|AAAAAAAADJMGAAAA|27794|7|43|14|AM|first|morning|breakfast| +27795|AAAAAAAAEJMGAAAA|27795|7|43|15|AM|first|morning|breakfast| +27796|AAAAAAAAFJMGAAAA|27796|7|43|16|AM|first|morning|breakfast| +27797|AAAAAAAAGJMGAAAA|27797|7|43|17|AM|first|morning|breakfast| +27798|AAAAAAAAHJMGAAAA|27798|7|43|18|AM|first|morning|breakfast| +27799|AAAAAAAAIJMGAAAA|27799|7|43|19|AM|first|morning|breakfast| +27800|AAAAAAAAJJMGAAAA|27800|7|43|20|AM|first|morning|breakfast| +27801|AAAAAAAAKJMGAAAA|27801|7|43|21|AM|first|morning|breakfast| +27802|AAAAAAAALJMGAAAA|27802|7|43|22|AM|first|morning|breakfast| +27803|AAAAAAAAMJMGAAAA|27803|7|43|23|AM|first|morning|breakfast| +27804|AAAAAAAANJMGAAAA|27804|7|43|24|AM|first|morning|breakfast| +27805|AAAAAAAAOJMGAAAA|27805|7|43|25|AM|first|morning|breakfast| +27806|AAAAAAAAPJMGAAAA|27806|7|43|26|AM|first|morning|breakfast| +27807|AAAAAAAAAKMGAAAA|27807|7|43|27|AM|first|morning|breakfast| +27808|AAAAAAAABKMGAAAA|27808|7|43|28|AM|first|morning|breakfast| +27809|AAAAAAAACKMGAAAA|27809|7|43|29|AM|first|morning|breakfast| +27810|AAAAAAAADKMGAAAA|27810|7|43|30|AM|first|morning|breakfast| +27811|AAAAAAAAEKMGAAAA|27811|7|43|31|AM|first|morning|breakfast| +27812|AAAAAAAAFKMGAAAA|27812|7|43|32|AM|first|morning|breakfast| +27813|AAAAAAAAGKMGAAAA|27813|7|43|33|AM|first|morning|breakfast| +27814|AAAAAAAAHKMGAAAA|27814|7|43|34|AM|first|morning|breakfast| +27815|AAAAAAAAIKMGAAAA|27815|7|43|35|AM|first|morning|breakfast| +27816|AAAAAAAAJKMGAAAA|27816|7|43|36|AM|first|morning|breakfast| +27817|AAAAAAAAKKMGAAAA|27817|7|43|37|AM|first|morning|breakfast| +27818|AAAAAAAALKMGAAAA|27818|7|43|38|AM|first|morning|breakfast| +27819|AAAAAAAAMKMGAAAA|27819|7|43|39|AM|first|morning|breakfast| +27820|AAAAAAAANKMGAAAA|27820|7|43|40|AM|first|morning|breakfast| +27821|AAAAAAAAOKMGAAAA|27821|7|43|41|AM|first|morning|breakfast| +27822|AAAAAAAAPKMGAAAA|27822|7|43|42|AM|first|morning|breakfast| +27823|AAAAAAAAALMGAAAA|27823|7|43|43|AM|first|morning|breakfast| +27824|AAAAAAAABLMGAAAA|27824|7|43|44|AM|first|morning|breakfast| +27825|AAAAAAAACLMGAAAA|27825|7|43|45|AM|first|morning|breakfast| +27826|AAAAAAAADLMGAAAA|27826|7|43|46|AM|first|morning|breakfast| +27827|AAAAAAAAELMGAAAA|27827|7|43|47|AM|first|morning|breakfast| +27828|AAAAAAAAFLMGAAAA|27828|7|43|48|AM|first|morning|breakfast| +27829|AAAAAAAAGLMGAAAA|27829|7|43|49|AM|first|morning|breakfast| +27830|AAAAAAAAHLMGAAAA|27830|7|43|50|AM|first|morning|breakfast| +27831|AAAAAAAAILMGAAAA|27831|7|43|51|AM|first|morning|breakfast| +27832|AAAAAAAAJLMGAAAA|27832|7|43|52|AM|first|morning|breakfast| +27833|AAAAAAAAKLMGAAAA|27833|7|43|53|AM|first|morning|breakfast| +27834|AAAAAAAALLMGAAAA|27834|7|43|54|AM|first|morning|breakfast| +27835|AAAAAAAAMLMGAAAA|27835|7|43|55|AM|first|morning|breakfast| +27836|AAAAAAAANLMGAAAA|27836|7|43|56|AM|first|morning|breakfast| +27837|AAAAAAAAOLMGAAAA|27837|7|43|57|AM|first|morning|breakfast| +27838|AAAAAAAAPLMGAAAA|27838|7|43|58|AM|first|morning|breakfast| +27839|AAAAAAAAAMMGAAAA|27839|7|43|59|AM|first|morning|breakfast| +27840|AAAAAAAABMMGAAAA|27840|7|44|0|AM|first|morning|breakfast| +27841|AAAAAAAACMMGAAAA|27841|7|44|1|AM|first|morning|breakfast| +27842|AAAAAAAADMMGAAAA|27842|7|44|2|AM|first|morning|breakfast| +27843|AAAAAAAAEMMGAAAA|27843|7|44|3|AM|first|morning|breakfast| +27844|AAAAAAAAFMMGAAAA|27844|7|44|4|AM|first|morning|breakfast| +27845|AAAAAAAAGMMGAAAA|27845|7|44|5|AM|first|morning|breakfast| +27846|AAAAAAAAHMMGAAAA|27846|7|44|6|AM|first|morning|breakfast| +27847|AAAAAAAAIMMGAAAA|27847|7|44|7|AM|first|morning|breakfast| +27848|AAAAAAAAJMMGAAAA|27848|7|44|8|AM|first|morning|breakfast| +27849|AAAAAAAAKMMGAAAA|27849|7|44|9|AM|first|morning|breakfast| +27850|AAAAAAAALMMGAAAA|27850|7|44|10|AM|first|morning|breakfast| +27851|AAAAAAAAMMMGAAAA|27851|7|44|11|AM|first|morning|breakfast| +27852|AAAAAAAANMMGAAAA|27852|7|44|12|AM|first|morning|breakfast| +27853|AAAAAAAAOMMGAAAA|27853|7|44|13|AM|first|morning|breakfast| +27854|AAAAAAAAPMMGAAAA|27854|7|44|14|AM|first|morning|breakfast| +27855|AAAAAAAAANMGAAAA|27855|7|44|15|AM|first|morning|breakfast| +27856|AAAAAAAABNMGAAAA|27856|7|44|16|AM|first|morning|breakfast| +27857|AAAAAAAACNMGAAAA|27857|7|44|17|AM|first|morning|breakfast| +27858|AAAAAAAADNMGAAAA|27858|7|44|18|AM|first|morning|breakfast| +27859|AAAAAAAAENMGAAAA|27859|7|44|19|AM|first|morning|breakfast| +27860|AAAAAAAAFNMGAAAA|27860|7|44|20|AM|first|morning|breakfast| +27861|AAAAAAAAGNMGAAAA|27861|7|44|21|AM|first|morning|breakfast| +27862|AAAAAAAAHNMGAAAA|27862|7|44|22|AM|first|morning|breakfast| +27863|AAAAAAAAINMGAAAA|27863|7|44|23|AM|first|morning|breakfast| +27864|AAAAAAAAJNMGAAAA|27864|7|44|24|AM|first|morning|breakfast| +27865|AAAAAAAAKNMGAAAA|27865|7|44|25|AM|first|morning|breakfast| +27866|AAAAAAAALNMGAAAA|27866|7|44|26|AM|first|morning|breakfast| +27867|AAAAAAAAMNMGAAAA|27867|7|44|27|AM|first|morning|breakfast| +27868|AAAAAAAANNMGAAAA|27868|7|44|28|AM|first|morning|breakfast| +27869|AAAAAAAAONMGAAAA|27869|7|44|29|AM|first|morning|breakfast| +27870|AAAAAAAAPNMGAAAA|27870|7|44|30|AM|first|morning|breakfast| +27871|AAAAAAAAAOMGAAAA|27871|7|44|31|AM|first|morning|breakfast| +27872|AAAAAAAABOMGAAAA|27872|7|44|32|AM|first|morning|breakfast| +27873|AAAAAAAACOMGAAAA|27873|7|44|33|AM|first|morning|breakfast| +27874|AAAAAAAADOMGAAAA|27874|7|44|34|AM|first|morning|breakfast| +27875|AAAAAAAAEOMGAAAA|27875|7|44|35|AM|first|morning|breakfast| +27876|AAAAAAAAFOMGAAAA|27876|7|44|36|AM|first|morning|breakfast| +27877|AAAAAAAAGOMGAAAA|27877|7|44|37|AM|first|morning|breakfast| +27878|AAAAAAAAHOMGAAAA|27878|7|44|38|AM|first|morning|breakfast| +27879|AAAAAAAAIOMGAAAA|27879|7|44|39|AM|first|morning|breakfast| +27880|AAAAAAAAJOMGAAAA|27880|7|44|40|AM|first|morning|breakfast| +27881|AAAAAAAAKOMGAAAA|27881|7|44|41|AM|first|morning|breakfast| +27882|AAAAAAAALOMGAAAA|27882|7|44|42|AM|first|morning|breakfast| +27883|AAAAAAAAMOMGAAAA|27883|7|44|43|AM|first|morning|breakfast| +27884|AAAAAAAANOMGAAAA|27884|7|44|44|AM|first|morning|breakfast| +27885|AAAAAAAAOOMGAAAA|27885|7|44|45|AM|first|morning|breakfast| +27886|AAAAAAAAPOMGAAAA|27886|7|44|46|AM|first|morning|breakfast| +27887|AAAAAAAAAPMGAAAA|27887|7|44|47|AM|first|morning|breakfast| +27888|AAAAAAAABPMGAAAA|27888|7|44|48|AM|first|morning|breakfast| +27889|AAAAAAAACPMGAAAA|27889|7|44|49|AM|first|morning|breakfast| +27890|AAAAAAAADPMGAAAA|27890|7|44|50|AM|first|morning|breakfast| +27891|AAAAAAAAEPMGAAAA|27891|7|44|51|AM|first|morning|breakfast| +27892|AAAAAAAAFPMGAAAA|27892|7|44|52|AM|first|morning|breakfast| +27893|AAAAAAAAGPMGAAAA|27893|7|44|53|AM|first|morning|breakfast| +27894|AAAAAAAAHPMGAAAA|27894|7|44|54|AM|first|morning|breakfast| +27895|AAAAAAAAIPMGAAAA|27895|7|44|55|AM|first|morning|breakfast| +27896|AAAAAAAAJPMGAAAA|27896|7|44|56|AM|first|morning|breakfast| +27897|AAAAAAAAKPMGAAAA|27897|7|44|57|AM|first|morning|breakfast| +27898|AAAAAAAALPMGAAAA|27898|7|44|58|AM|first|morning|breakfast| +27899|AAAAAAAAMPMGAAAA|27899|7|44|59|AM|first|morning|breakfast| +27900|AAAAAAAANPMGAAAA|27900|7|45|0|AM|first|morning|breakfast| +27901|AAAAAAAAOPMGAAAA|27901|7|45|1|AM|first|morning|breakfast| +27902|AAAAAAAAPPMGAAAA|27902|7|45|2|AM|first|morning|breakfast| +27903|AAAAAAAAAANGAAAA|27903|7|45|3|AM|first|morning|breakfast| +27904|AAAAAAAABANGAAAA|27904|7|45|4|AM|first|morning|breakfast| +27905|AAAAAAAACANGAAAA|27905|7|45|5|AM|first|morning|breakfast| +27906|AAAAAAAADANGAAAA|27906|7|45|6|AM|first|morning|breakfast| +27907|AAAAAAAAEANGAAAA|27907|7|45|7|AM|first|morning|breakfast| +27908|AAAAAAAAFANGAAAA|27908|7|45|8|AM|first|morning|breakfast| +27909|AAAAAAAAGANGAAAA|27909|7|45|9|AM|first|morning|breakfast| +27910|AAAAAAAAHANGAAAA|27910|7|45|10|AM|first|morning|breakfast| +27911|AAAAAAAAIANGAAAA|27911|7|45|11|AM|first|morning|breakfast| +27912|AAAAAAAAJANGAAAA|27912|7|45|12|AM|first|morning|breakfast| +27913|AAAAAAAAKANGAAAA|27913|7|45|13|AM|first|morning|breakfast| +27914|AAAAAAAALANGAAAA|27914|7|45|14|AM|first|morning|breakfast| +27915|AAAAAAAAMANGAAAA|27915|7|45|15|AM|first|morning|breakfast| +27916|AAAAAAAANANGAAAA|27916|7|45|16|AM|first|morning|breakfast| +27917|AAAAAAAAOANGAAAA|27917|7|45|17|AM|first|morning|breakfast| +27918|AAAAAAAAPANGAAAA|27918|7|45|18|AM|first|morning|breakfast| +27919|AAAAAAAAABNGAAAA|27919|7|45|19|AM|first|morning|breakfast| +27920|AAAAAAAABBNGAAAA|27920|7|45|20|AM|first|morning|breakfast| +27921|AAAAAAAACBNGAAAA|27921|7|45|21|AM|first|morning|breakfast| +27922|AAAAAAAADBNGAAAA|27922|7|45|22|AM|first|morning|breakfast| +27923|AAAAAAAAEBNGAAAA|27923|7|45|23|AM|first|morning|breakfast| +27924|AAAAAAAAFBNGAAAA|27924|7|45|24|AM|first|morning|breakfast| +27925|AAAAAAAAGBNGAAAA|27925|7|45|25|AM|first|morning|breakfast| +27926|AAAAAAAAHBNGAAAA|27926|7|45|26|AM|first|morning|breakfast| +27927|AAAAAAAAIBNGAAAA|27927|7|45|27|AM|first|morning|breakfast| +27928|AAAAAAAAJBNGAAAA|27928|7|45|28|AM|first|morning|breakfast| +27929|AAAAAAAAKBNGAAAA|27929|7|45|29|AM|first|morning|breakfast| +27930|AAAAAAAALBNGAAAA|27930|7|45|30|AM|first|morning|breakfast| +27931|AAAAAAAAMBNGAAAA|27931|7|45|31|AM|first|morning|breakfast| +27932|AAAAAAAANBNGAAAA|27932|7|45|32|AM|first|morning|breakfast| +27933|AAAAAAAAOBNGAAAA|27933|7|45|33|AM|first|morning|breakfast| +27934|AAAAAAAAPBNGAAAA|27934|7|45|34|AM|first|morning|breakfast| +27935|AAAAAAAAACNGAAAA|27935|7|45|35|AM|first|morning|breakfast| +27936|AAAAAAAABCNGAAAA|27936|7|45|36|AM|first|morning|breakfast| +27937|AAAAAAAACCNGAAAA|27937|7|45|37|AM|first|morning|breakfast| +27938|AAAAAAAADCNGAAAA|27938|7|45|38|AM|first|morning|breakfast| +27939|AAAAAAAAECNGAAAA|27939|7|45|39|AM|first|morning|breakfast| +27940|AAAAAAAAFCNGAAAA|27940|7|45|40|AM|first|morning|breakfast| +27941|AAAAAAAAGCNGAAAA|27941|7|45|41|AM|first|morning|breakfast| +27942|AAAAAAAAHCNGAAAA|27942|7|45|42|AM|first|morning|breakfast| +27943|AAAAAAAAICNGAAAA|27943|7|45|43|AM|first|morning|breakfast| +27944|AAAAAAAAJCNGAAAA|27944|7|45|44|AM|first|morning|breakfast| +27945|AAAAAAAAKCNGAAAA|27945|7|45|45|AM|first|morning|breakfast| +27946|AAAAAAAALCNGAAAA|27946|7|45|46|AM|first|morning|breakfast| +27947|AAAAAAAAMCNGAAAA|27947|7|45|47|AM|first|morning|breakfast| +27948|AAAAAAAANCNGAAAA|27948|7|45|48|AM|first|morning|breakfast| +27949|AAAAAAAAOCNGAAAA|27949|7|45|49|AM|first|morning|breakfast| +27950|AAAAAAAAPCNGAAAA|27950|7|45|50|AM|first|morning|breakfast| +27951|AAAAAAAAADNGAAAA|27951|7|45|51|AM|first|morning|breakfast| +27952|AAAAAAAABDNGAAAA|27952|7|45|52|AM|first|morning|breakfast| +27953|AAAAAAAACDNGAAAA|27953|7|45|53|AM|first|morning|breakfast| +27954|AAAAAAAADDNGAAAA|27954|7|45|54|AM|first|morning|breakfast| +27955|AAAAAAAAEDNGAAAA|27955|7|45|55|AM|first|morning|breakfast| +27956|AAAAAAAAFDNGAAAA|27956|7|45|56|AM|first|morning|breakfast| +27957|AAAAAAAAGDNGAAAA|27957|7|45|57|AM|first|morning|breakfast| +27958|AAAAAAAAHDNGAAAA|27958|7|45|58|AM|first|morning|breakfast| +27959|AAAAAAAAIDNGAAAA|27959|7|45|59|AM|first|morning|breakfast| +27960|AAAAAAAAJDNGAAAA|27960|7|46|0|AM|first|morning|breakfast| +27961|AAAAAAAAKDNGAAAA|27961|7|46|1|AM|first|morning|breakfast| +27962|AAAAAAAALDNGAAAA|27962|7|46|2|AM|first|morning|breakfast| +27963|AAAAAAAAMDNGAAAA|27963|7|46|3|AM|first|morning|breakfast| +27964|AAAAAAAANDNGAAAA|27964|7|46|4|AM|first|morning|breakfast| +27965|AAAAAAAAODNGAAAA|27965|7|46|5|AM|first|morning|breakfast| +27966|AAAAAAAAPDNGAAAA|27966|7|46|6|AM|first|morning|breakfast| +27967|AAAAAAAAAENGAAAA|27967|7|46|7|AM|first|morning|breakfast| +27968|AAAAAAAABENGAAAA|27968|7|46|8|AM|first|morning|breakfast| +27969|AAAAAAAACENGAAAA|27969|7|46|9|AM|first|morning|breakfast| +27970|AAAAAAAADENGAAAA|27970|7|46|10|AM|first|morning|breakfast| +27971|AAAAAAAAEENGAAAA|27971|7|46|11|AM|first|morning|breakfast| +27972|AAAAAAAAFENGAAAA|27972|7|46|12|AM|first|morning|breakfast| +27973|AAAAAAAAGENGAAAA|27973|7|46|13|AM|first|morning|breakfast| +27974|AAAAAAAAHENGAAAA|27974|7|46|14|AM|first|morning|breakfast| +27975|AAAAAAAAIENGAAAA|27975|7|46|15|AM|first|morning|breakfast| +27976|AAAAAAAAJENGAAAA|27976|7|46|16|AM|first|morning|breakfast| +27977|AAAAAAAAKENGAAAA|27977|7|46|17|AM|first|morning|breakfast| +27978|AAAAAAAALENGAAAA|27978|7|46|18|AM|first|morning|breakfast| +27979|AAAAAAAAMENGAAAA|27979|7|46|19|AM|first|morning|breakfast| +27980|AAAAAAAANENGAAAA|27980|7|46|20|AM|first|morning|breakfast| +27981|AAAAAAAAOENGAAAA|27981|7|46|21|AM|first|morning|breakfast| +27982|AAAAAAAAPENGAAAA|27982|7|46|22|AM|first|morning|breakfast| +27983|AAAAAAAAAFNGAAAA|27983|7|46|23|AM|first|morning|breakfast| +27984|AAAAAAAABFNGAAAA|27984|7|46|24|AM|first|morning|breakfast| +27985|AAAAAAAACFNGAAAA|27985|7|46|25|AM|first|morning|breakfast| +27986|AAAAAAAADFNGAAAA|27986|7|46|26|AM|first|morning|breakfast| +27987|AAAAAAAAEFNGAAAA|27987|7|46|27|AM|first|morning|breakfast| +27988|AAAAAAAAFFNGAAAA|27988|7|46|28|AM|first|morning|breakfast| +27989|AAAAAAAAGFNGAAAA|27989|7|46|29|AM|first|morning|breakfast| +27990|AAAAAAAAHFNGAAAA|27990|7|46|30|AM|first|morning|breakfast| +27991|AAAAAAAAIFNGAAAA|27991|7|46|31|AM|first|morning|breakfast| +27992|AAAAAAAAJFNGAAAA|27992|7|46|32|AM|first|morning|breakfast| +27993|AAAAAAAAKFNGAAAA|27993|7|46|33|AM|first|morning|breakfast| +27994|AAAAAAAALFNGAAAA|27994|7|46|34|AM|first|morning|breakfast| +27995|AAAAAAAAMFNGAAAA|27995|7|46|35|AM|first|morning|breakfast| +27996|AAAAAAAANFNGAAAA|27996|7|46|36|AM|first|morning|breakfast| +27997|AAAAAAAAOFNGAAAA|27997|7|46|37|AM|first|morning|breakfast| +27998|AAAAAAAAPFNGAAAA|27998|7|46|38|AM|first|morning|breakfast| +27999|AAAAAAAAAGNGAAAA|27999|7|46|39|AM|first|morning|breakfast| +28000|AAAAAAAABGNGAAAA|28000|7|46|40|AM|first|morning|breakfast| +28001|AAAAAAAACGNGAAAA|28001|7|46|41|AM|first|morning|breakfast| +28002|AAAAAAAADGNGAAAA|28002|7|46|42|AM|first|morning|breakfast| +28003|AAAAAAAAEGNGAAAA|28003|7|46|43|AM|first|morning|breakfast| +28004|AAAAAAAAFGNGAAAA|28004|7|46|44|AM|first|morning|breakfast| +28005|AAAAAAAAGGNGAAAA|28005|7|46|45|AM|first|morning|breakfast| +28006|AAAAAAAAHGNGAAAA|28006|7|46|46|AM|first|morning|breakfast| +28007|AAAAAAAAIGNGAAAA|28007|7|46|47|AM|first|morning|breakfast| +28008|AAAAAAAAJGNGAAAA|28008|7|46|48|AM|first|morning|breakfast| +28009|AAAAAAAAKGNGAAAA|28009|7|46|49|AM|first|morning|breakfast| +28010|AAAAAAAALGNGAAAA|28010|7|46|50|AM|first|morning|breakfast| +28011|AAAAAAAAMGNGAAAA|28011|7|46|51|AM|first|morning|breakfast| +28012|AAAAAAAANGNGAAAA|28012|7|46|52|AM|first|morning|breakfast| +28013|AAAAAAAAOGNGAAAA|28013|7|46|53|AM|first|morning|breakfast| +28014|AAAAAAAAPGNGAAAA|28014|7|46|54|AM|first|morning|breakfast| +28015|AAAAAAAAAHNGAAAA|28015|7|46|55|AM|first|morning|breakfast| +28016|AAAAAAAABHNGAAAA|28016|7|46|56|AM|first|morning|breakfast| +28017|AAAAAAAACHNGAAAA|28017|7|46|57|AM|first|morning|breakfast| +28018|AAAAAAAADHNGAAAA|28018|7|46|58|AM|first|morning|breakfast| +28019|AAAAAAAAEHNGAAAA|28019|7|46|59|AM|first|morning|breakfast| +28020|AAAAAAAAFHNGAAAA|28020|7|47|0|AM|first|morning|breakfast| +28021|AAAAAAAAGHNGAAAA|28021|7|47|1|AM|first|morning|breakfast| +28022|AAAAAAAAHHNGAAAA|28022|7|47|2|AM|first|morning|breakfast| +28023|AAAAAAAAIHNGAAAA|28023|7|47|3|AM|first|morning|breakfast| +28024|AAAAAAAAJHNGAAAA|28024|7|47|4|AM|first|morning|breakfast| +28025|AAAAAAAAKHNGAAAA|28025|7|47|5|AM|first|morning|breakfast| +28026|AAAAAAAALHNGAAAA|28026|7|47|6|AM|first|morning|breakfast| +28027|AAAAAAAAMHNGAAAA|28027|7|47|7|AM|first|morning|breakfast| +28028|AAAAAAAANHNGAAAA|28028|7|47|8|AM|first|morning|breakfast| +28029|AAAAAAAAOHNGAAAA|28029|7|47|9|AM|first|morning|breakfast| +28030|AAAAAAAAPHNGAAAA|28030|7|47|10|AM|first|morning|breakfast| +28031|AAAAAAAAAINGAAAA|28031|7|47|11|AM|first|morning|breakfast| +28032|AAAAAAAABINGAAAA|28032|7|47|12|AM|first|morning|breakfast| +28033|AAAAAAAACINGAAAA|28033|7|47|13|AM|first|morning|breakfast| +28034|AAAAAAAADINGAAAA|28034|7|47|14|AM|first|morning|breakfast| +28035|AAAAAAAAEINGAAAA|28035|7|47|15|AM|first|morning|breakfast| +28036|AAAAAAAAFINGAAAA|28036|7|47|16|AM|first|morning|breakfast| +28037|AAAAAAAAGINGAAAA|28037|7|47|17|AM|first|morning|breakfast| +28038|AAAAAAAAHINGAAAA|28038|7|47|18|AM|first|morning|breakfast| +28039|AAAAAAAAIINGAAAA|28039|7|47|19|AM|first|morning|breakfast| +28040|AAAAAAAAJINGAAAA|28040|7|47|20|AM|first|morning|breakfast| +28041|AAAAAAAAKINGAAAA|28041|7|47|21|AM|first|morning|breakfast| +28042|AAAAAAAALINGAAAA|28042|7|47|22|AM|first|morning|breakfast| +28043|AAAAAAAAMINGAAAA|28043|7|47|23|AM|first|morning|breakfast| +28044|AAAAAAAANINGAAAA|28044|7|47|24|AM|first|morning|breakfast| +28045|AAAAAAAAOINGAAAA|28045|7|47|25|AM|first|morning|breakfast| +28046|AAAAAAAAPINGAAAA|28046|7|47|26|AM|first|morning|breakfast| +28047|AAAAAAAAAJNGAAAA|28047|7|47|27|AM|first|morning|breakfast| +28048|AAAAAAAABJNGAAAA|28048|7|47|28|AM|first|morning|breakfast| +28049|AAAAAAAACJNGAAAA|28049|7|47|29|AM|first|morning|breakfast| +28050|AAAAAAAADJNGAAAA|28050|7|47|30|AM|first|morning|breakfast| +28051|AAAAAAAAEJNGAAAA|28051|7|47|31|AM|first|morning|breakfast| +28052|AAAAAAAAFJNGAAAA|28052|7|47|32|AM|first|morning|breakfast| +28053|AAAAAAAAGJNGAAAA|28053|7|47|33|AM|first|morning|breakfast| +28054|AAAAAAAAHJNGAAAA|28054|7|47|34|AM|first|morning|breakfast| +28055|AAAAAAAAIJNGAAAA|28055|7|47|35|AM|first|morning|breakfast| +28056|AAAAAAAAJJNGAAAA|28056|7|47|36|AM|first|morning|breakfast| +28057|AAAAAAAAKJNGAAAA|28057|7|47|37|AM|first|morning|breakfast| +28058|AAAAAAAALJNGAAAA|28058|7|47|38|AM|first|morning|breakfast| +28059|AAAAAAAAMJNGAAAA|28059|7|47|39|AM|first|morning|breakfast| +28060|AAAAAAAANJNGAAAA|28060|7|47|40|AM|first|morning|breakfast| +28061|AAAAAAAAOJNGAAAA|28061|7|47|41|AM|first|morning|breakfast| +28062|AAAAAAAAPJNGAAAA|28062|7|47|42|AM|first|morning|breakfast| +28063|AAAAAAAAAKNGAAAA|28063|7|47|43|AM|first|morning|breakfast| +28064|AAAAAAAABKNGAAAA|28064|7|47|44|AM|first|morning|breakfast| +28065|AAAAAAAACKNGAAAA|28065|7|47|45|AM|first|morning|breakfast| +28066|AAAAAAAADKNGAAAA|28066|7|47|46|AM|first|morning|breakfast| +28067|AAAAAAAAEKNGAAAA|28067|7|47|47|AM|first|morning|breakfast| +28068|AAAAAAAAFKNGAAAA|28068|7|47|48|AM|first|morning|breakfast| +28069|AAAAAAAAGKNGAAAA|28069|7|47|49|AM|first|morning|breakfast| +28070|AAAAAAAAHKNGAAAA|28070|7|47|50|AM|first|morning|breakfast| +28071|AAAAAAAAIKNGAAAA|28071|7|47|51|AM|first|morning|breakfast| +28072|AAAAAAAAJKNGAAAA|28072|7|47|52|AM|first|morning|breakfast| +28073|AAAAAAAAKKNGAAAA|28073|7|47|53|AM|first|morning|breakfast| +28074|AAAAAAAALKNGAAAA|28074|7|47|54|AM|first|morning|breakfast| +28075|AAAAAAAAMKNGAAAA|28075|7|47|55|AM|first|morning|breakfast| +28076|AAAAAAAANKNGAAAA|28076|7|47|56|AM|first|morning|breakfast| +28077|AAAAAAAAOKNGAAAA|28077|7|47|57|AM|first|morning|breakfast| +28078|AAAAAAAAPKNGAAAA|28078|7|47|58|AM|first|morning|breakfast| +28079|AAAAAAAAALNGAAAA|28079|7|47|59|AM|first|morning|breakfast| +28080|AAAAAAAABLNGAAAA|28080|7|48|0|AM|first|morning|breakfast| +28081|AAAAAAAACLNGAAAA|28081|7|48|1|AM|first|morning|breakfast| +28082|AAAAAAAADLNGAAAA|28082|7|48|2|AM|first|morning|breakfast| +28083|AAAAAAAAELNGAAAA|28083|7|48|3|AM|first|morning|breakfast| +28084|AAAAAAAAFLNGAAAA|28084|7|48|4|AM|first|morning|breakfast| +28085|AAAAAAAAGLNGAAAA|28085|7|48|5|AM|first|morning|breakfast| +28086|AAAAAAAAHLNGAAAA|28086|7|48|6|AM|first|morning|breakfast| +28087|AAAAAAAAILNGAAAA|28087|7|48|7|AM|first|morning|breakfast| +28088|AAAAAAAAJLNGAAAA|28088|7|48|8|AM|first|morning|breakfast| +28089|AAAAAAAAKLNGAAAA|28089|7|48|9|AM|first|morning|breakfast| +28090|AAAAAAAALLNGAAAA|28090|7|48|10|AM|first|morning|breakfast| +28091|AAAAAAAAMLNGAAAA|28091|7|48|11|AM|first|morning|breakfast| +28092|AAAAAAAANLNGAAAA|28092|7|48|12|AM|first|morning|breakfast| +28093|AAAAAAAAOLNGAAAA|28093|7|48|13|AM|first|morning|breakfast| +28094|AAAAAAAAPLNGAAAA|28094|7|48|14|AM|first|morning|breakfast| +28095|AAAAAAAAAMNGAAAA|28095|7|48|15|AM|first|morning|breakfast| +28096|AAAAAAAABMNGAAAA|28096|7|48|16|AM|first|morning|breakfast| +28097|AAAAAAAACMNGAAAA|28097|7|48|17|AM|first|morning|breakfast| +28098|AAAAAAAADMNGAAAA|28098|7|48|18|AM|first|morning|breakfast| +28099|AAAAAAAAEMNGAAAA|28099|7|48|19|AM|first|morning|breakfast| +28100|AAAAAAAAFMNGAAAA|28100|7|48|20|AM|first|morning|breakfast| +28101|AAAAAAAAGMNGAAAA|28101|7|48|21|AM|first|morning|breakfast| +28102|AAAAAAAAHMNGAAAA|28102|7|48|22|AM|first|morning|breakfast| +28103|AAAAAAAAIMNGAAAA|28103|7|48|23|AM|first|morning|breakfast| +28104|AAAAAAAAJMNGAAAA|28104|7|48|24|AM|first|morning|breakfast| +28105|AAAAAAAAKMNGAAAA|28105|7|48|25|AM|first|morning|breakfast| +28106|AAAAAAAALMNGAAAA|28106|7|48|26|AM|first|morning|breakfast| +28107|AAAAAAAAMMNGAAAA|28107|7|48|27|AM|first|morning|breakfast| +28108|AAAAAAAANMNGAAAA|28108|7|48|28|AM|first|morning|breakfast| +28109|AAAAAAAAOMNGAAAA|28109|7|48|29|AM|first|morning|breakfast| +28110|AAAAAAAAPMNGAAAA|28110|7|48|30|AM|first|morning|breakfast| +28111|AAAAAAAAANNGAAAA|28111|7|48|31|AM|first|morning|breakfast| +28112|AAAAAAAABNNGAAAA|28112|7|48|32|AM|first|morning|breakfast| +28113|AAAAAAAACNNGAAAA|28113|7|48|33|AM|first|morning|breakfast| +28114|AAAAAAAADNNGAAAA|28114|7|48|34|AM|first|morning|breakfast| +28115|AAAAAAAAENNGAAAA|28115|7|48|35|AM|first|morning|breakfast| +28116|AAAAAAAAFNNGAAAA|28116|7|48|36|AM|first|morning|breakfast| +28117|AAAAAAAAGNNGAAAA|28117|7|48|37|AM|first|morning|breakfast| +28118|AAAAAAAAHNNGAAAA|28118|7|48|38|AM|first|morning|breakfast| +28119|AAAAAAAAINNGAAAA|28119|7|48|39|AM|first|morning|breakfast| +28120|AAAAAAAAJNNGAAAA|28120|7|48|40|AM|first|morning|breakfast| +28121|AAAAAAAAKNNGAAAA|28121|7|48|41|AM|first|morning|breakfast| +28122|AAAAAAAALNNGAAAA|28122|7|48|42|AM|first|morning|breakfast| +28123|AAAAAAAAMNNGAAAA|28123|7|48|43|AM|first|morning|breakfast| +28124|AAAAAAAANNNGAAAA|28124|7|48|44|AM|first|morning|breakfast| +28125|AAAAAAAAONNGAAAA|28125|7|48|45|AM|first|morning|breakfast| +28126|AAAAAAAAPNNGAAAA|28126|7|48|46|AM|first|morning|breakfast| +28127|AAAAAAAAAONGAAAA|28127|7|48|47|AM|first|morning|breakfast| +28128|AAAAAAAABONGAAAA|28128|7|48|48|AM|first|morning|breakfast| +28129|AAAAAAAACONGAAAA|28129|7|48|49|AM|first|morning|breakfast| +28130|AAAAAAAADONGAAAA|28130|7|48|50|AM|first|morning|breakfast| +28131|AAAAAAAAEONGAAAA|28131|7|48|51|AM|first|morning|breakfast| +28132|AAAAAAAAFONGAAAA|28132|7|48|52|AM|first|morning|breakfast| +28133|AAAAAAAAGONGAAAA|28133|7|48|53|AM|first|morning|breakfast| +28134|AAAAAAAAHONGAAAA|28134|7|48|54|AM|first|morning|breakfast| +28135|AAAAAAAAIONGAAAA|28135|7|48|55|AM|first|morning|breakfast| +28136|AAAAAAAAJONGAAAA|28136|7|48|56|AM|first|morning|breakfast| +28137|AAAAAAAAKONGAAAA|28137|7|48|57|AM|first|morning|breakfast| +28138|AAAAAAAALONGAAAA|28138|7|48|58|AM|first|morning|breakfast| +28139|AAAAAAAAMONGAAAA|28139|7|48|59|AM|first|morning|breakfast| +28140|AAAAAAAANONGAAAA|28140|7|49|0|AM|first|morning|breakfast| +28141|AAAAAAAAOONGAAAA|28141|7|49|1|AM|first|morning|breakfast| +28142|AAAAAAAAPONGAAAA|28142|7|49|2|AM|first|morning|breakfast| +28143|AAAAAAAAAPNGAAAA|28143|7|49|3|AM|first|morning|breakfast| +28144|AAAAAAAABPNGAAAA|28144|7|49|4|AM|first|morning|breakfast| +28145|AAAAAAAACPNGAAAA|28145|7|49|5|AM|first|morning|breakfast| +28146|AAAAAAAADPNGAAAA|28146|7|49|6|AM|first|morning|breakfast| +28147|AAAAAAAAEPNGAAAA|28147|7|49|7|AM|first|morning|breakfast| +28148|AAAAAAAAFPNGAAAA|28148|7|49|8|AM|first|morning|breakfast| +28149|AAAAAAAAGPNGAAAA|28149|7|49|9|AM|first|morning|breakfast| +28150|AAAAAAAAHPNGAAAA|28150|7|49|10|AM|first|morning|breakfast| +28151|AAAAAAAAIPNGAAAA|28151|7|49|11|AM|first|morning|breakfast| +28152|AAAAAAAAJPNGAAAA|28152|7|49|12|AM|first|morning|breakfast| +28153|AAAAAAAAKPNGAAAA|28153|7|49|13|AM|first|morning|breakfast| +28154|AAAAAAAALPNGAAAA|28154|7|49|14|AM|first|morning|breakfast| +28155|AAAAAAAAMPNGAAAA|28155|7|49|15|AM|first|morning|breakfast| +28156|AAAAAAAANPNGAAAA|28156|7|49|16|AM|first|morning|breakfast| +28157|AAAAAAAAOPNGAAAA|28157|7|49|17|AM|first|morning|breakfast| +28158|AAAAAAAAPPNGAAAA|28158|7|49|18|AM|first|morning|breakfast| +28159|AAAAAAAAAAOGAAAA|28159|7|49|19|AM|first|morning|breakfast| +28160|AAAAAAAABAOGAAAA|28160|7|49|20|AM|first|morning|breakfast| +28161|AAAAAAAACAOGAAAA|28161|7|49|21|AM|first|morning|breakfast| +28162|AAAAAAAADAOGAAAA|28162|7|49|22|AM|first|morning|breakfast| +28163|AAAAAAAAEAOGAAAA|28163|7|49|23|AM|first|morning|breakfast| +28164|AAAAAAAAFAOGAAAA|28164|7|49|24|AM|first|morning|breakfast| +28165|AAAAAAAAGAOGAAAA|28165|7|49|25|AM|first|morning|breakfast| +28166|AAAAAAAAHAOGAAAA|28166|7|49|26|AM|first|morning|breakfast| +28167|AAAAAAAAIAOGAAAA|28167|7|49|27|AM|first|morning|breakfast| +28168|AAAAAAAAJAOGAAAA|28168|7|49|28|AM|first|morning|breakfast| +28169|AAAAAAAAKAOGAAAA|28169|7|49|29|AM|first|morning|breakfast| +28170|AAAAAAAALAOGAAAA|28170|7|49|30|AM|first|morning|breakfast| +28171|AAAAAAAAMAOGAAAA|28171|7|49|31|AM|first|morning|breakfast| +28172|AAAAAAAANAOGAAAA|28172|7|49|32|AM|first|morning|breakfast| +28173|AAAAAAAAOAOGAAAA|28173|7|49|33|AM|first|morning|breakfast| +28174|AAAAAAAAPAOGAAAA|28174|7|49|34|AM|first|morning|breakfast| +28175|AAAAAAAAABOGAAAA|28175|7|49|35|AM|first|morning|breakfast| +28176|AAAAAAAABBOGAAAA|28176|7|49|36|AM|first|morning|breakfast| +28177|AAAAAAAACBOGAAAA|28177|7|49|37|AM|first|morning|breakfast| +28178|AAAAAAAADBOGAAAA|28178|7|49|38|AM|first|morning|breakfast| +28179|AAAAAAAAEBOGAAAA|28179|7|49|39|AM|first|morning|breakfast| +28180|AAAAAAAAFBOGAAAA|28180|7|49|40|AM|first|morning|breakfast| +28181|AAAAAAAAGBOGAAAA|28181|7|49|41|AM|first|morning|breakfast| +28182|AAAAAAAAHBOGAAAA|28182|7|49|42|AM|first|morning|breakfast| +28183|AAAAAAAAIBOGAAAA|28183|7|49|43|AM|first|morning|breakfast| +28184|AAAAAAAAJBOGAAAA|28184|7|49|44|AM|first|morning|breakfast| +28185|AAAAAAAAKBOGAAAA|28185|7|49|45|AM|first|morning|breakfast| +28186|AAAAAAAALBOGAAAA|28186|7|49|46|AM|first|morning|breakfast| +28187|AAAAAAAAMBOGAAAA|28187|7|49|47|AM|first|morning|breakfast| +28188|AAAAAAAANBOGAAAA|28188|7|49|48|AM|first|morning|breakfast| +28189|AAAAAAAAOBOGAAAA|28189|7|49|49|AM|first|morning|breakfast| +28190|AAAAAAAAPBOGAAAA|28190|7|49|50|AM|first|morning|breakfast| +28191|AAAAAAAAACOGAAAA|28191|7|49|51|AM|first|morning|breakfast| +28192|AAAAAAAABCOGAAAA|28192|7|49|52|AM|first|morning|breakfast| +28193|AAAAAAAACCOGAAAA|28193|7|49|53|AM|first|morning|breakfast| +28194|AAAAAAAADCOGAAAA|28194|7|49|54|AM|first|morning|breakfast| +28195|AAAAAAAAECOGAAAA|28195|7|49|55|AM|first|morning|breakfast| +28196|AAAAAAAAFCOGAAAA|28196|7|49|56|AM|first|morning|breakfast| +28197|AAAAAAAAGCOGAAAA|28197|7|49|57|AM|first|morning|breakfast| +28198|AAAAAAAAHCOGAAAA|28198|7|49|58|AM|first|morning|breakfast| +28199|AAAAAAAAICOGAAAA|28199|7|49|59|AM|first|morning|breakfast| +28200|AAAAAAAAJCOGAAAA|28200|7|50|0|AM|first|morning|breakfast| +28201|AAAAAAAAKCOGAAAA|28201|7|50|1|AM|first|morning|breakfast| +28202|AAAAAAAALCOGAAAA|28202|7|50|2|AM|first|morning|breakfast| +28203|AAAAAAAAMCOGAAAA|28203|7|50|3|AM|first|morning|breakfast| +28204|AAAAAAAANCOGAAAA|28204|7|50|4|AM|first|morning|breakfast| +28205|AAAAAAAAOCOGAAAA|28205|7|50|5|AM|first|morning|breakfast| +28206|AAAAAAAAPCOGAAAA|28206|7|50|6|AM|first|morning|breakfast| +28207|AAAAAAAAADOGAAAA|28207|7|50|7|AM|first|morning|breakfast| +28208|AAAAAAAABDOGAAAA|28208|7|50|8|AM|first|morning|breakfast| +28209|AAAAAAAACDOGAAAA|28209|7|50|9|AM|first|morning|breakfast| +28210|AAAAAAAADDOGAAAA|28210|7|50|10|AM|first|morning|breakfast| +28211|AAAAAAAAEDOGAAAA|28211|7|50|11|AM|first|morning|breakfast| +28212|AAAAAAAAFDOGAAAA|28212|7|50|12|AM|first|morning|breakfast| +28213|AAAAAAAAGDOGAAAA|28213|7|50|13|AM|first|morning|breakfast| +28214|AAAAAAAAHDOGAAAA|28214|7|50|14|AM|first|morning|breakfast| +28215|AAAAAAAAIDOGAAAA|28215|7|50|15|AM|first|morning|breakfast| +28216|AAAAAAAAJDOGAAAA|28216|7|50|16|AM|first|morning|breakfast| +28217|AAAAAAAAKDOGAAAA|28217|7|50|17|AM|first|morning|breakfast| +28218|AAAAAAAALDOGAAAA|28218|7|50|18|AM|first|morning|breakfast| +28219|AAAAAAAAMDOGAAAA|28219|7|50|19|AM|first|morning|breakfast| +28220|AAAAAAAANDOGAAAA|28220|7|50|20|AM|first|morning|breakfast| +28221|AAAAAAAAODOGAAAA|28221|7|50|21|AM|first|morning|breakfast| +28222|AAAAAAAAPDOGAAAA|28222|7|50|22|AM|first|morning|breakfast| +28223|AAAAAAAAAEOGAAAA|28223|7|50|23|AM|first|morning|breakfast| +28224|AAAAAAAABEOGAAAA|28224|7|50|24|AM|first|morning|breakfast| +28225|AAAAAAAACEOGAAAA|28225|7|50|25|AM|first|morning|breakfast| +28226|AAAAAAAADEOGAAAA|28226|7|50|26|AM|first|morning|breakfast| +28227|AAAAAAAAEEOGAAAA|28227|7|50|27|AM|first|morning|breakfast| +28228|AAAAAAAAFEOGAAAA|28228|7|50|28|AM|first|morning|breakfast| +28229|AAAAAAAAGEOGAAAA|28229|7|50|29|AM|first|morning|breakfast| +28230|AAAAAAAAHEOGAAAA|28230|7|50|30|AM|first|morning|breakfast| +28231|AAAAAAAAIEOGAAAA|28231|7|50|31|AM|first|morning|breakfast| +28232|AAAAAAAAJEOGAAAA|28232|7|50|32|AM|first|morning|breakfast| +28233|AAAAAAAAKEOGAAAA|28233|7|50|33|AM|first|morning|breakfast| +28234|AAAAAAAALEOGAAAA|28234|7|50|34|AM|first|morning|breakfast| +28235|AAAAAAAAMEOGAAAA|28235|7|50|35|AM|first|morning|breakfast| +28236|AAAAAAAANEOGAAAA|28236|7|50|36|AM|first|morning|breakfast| +28237|AAAAAAAAOEOGAAAA|28237|7|50|37|AM|first|morning|breakfast| +28238|AAAAAAAAPEOGAAAA|28238|7|50|38|AM|first|morning|breakfast| +28239|AAAAAAAAAFOGAAAA|28239|7|50|39|AM|first|morning|breakfast| +28240|AAAAAAAABFOGAAAA|28240|7|50|40|AM|first|morning|breakfast| +28241|AAAAAAAACFOGAAAA|28241|7|50|41|AM|first|morning|breakfast| +28242|AAAAAAAADFOGAAAA|28242|7|50|42|AM|first|morning|breakfast| +28243|AAAAAAAAEFOGAAAA|28243|7|50|43|AM|first|morning|breakfast| +28244|AAAAAAAAFFOGAAAA|28244|7|50|44|AM|first|morning|breakfast| +28245|AAAAAAAAGFOGAAAA|28245|7|50|45|AM|first|morning|breakfast| +28246|AAAAAAAAHFOGAAAA|28246|7|50|46|AM|first|morning|breakfast| +28247|AAAAAAAAIFOGAAAA|28247|7|50|47|AM|first|morning|breakfast| +28248|AAAAAAAAJFOGAAAA|28248|7|50|48|AM|first|morning|breakfast| +28249|AAAAAAAAKFOGAAAA|28249|7|50|49|AM|first|morning|breakfast| +28250|AAAAAAAALFOGAAAA|28250|7|50|50|AM|first|morning|breakfast| +28251|AAAAAAAAMFOGAAAA|28251|7|50|51|AM|first|morning|breakfast| +28252|AAAAAAAANFOGAAAA|28252|7|50|52|AM|first|morning|breakfast| +28253|AAAAAAAAOFOGAAAA|28253|7|50|53|AM|first|morning|breakfast| +28254|AAAAAAAAPFOGAAAA|28254|7|50|54|AM|first|morning|breakfast| +28255|AAAAAAAAAGOGAAAA|28255|7|50|55|AM|first|morning|breakfast| +28256|AAAAAAAABGOGAAAA|28256|7|50|56|AM|first|morning|breakfast| +28257|AAAAAAAACGOGAAAA|28257|7|50|57|AM|first|morning|breakfast| +28258|AAAAAAAADGOGAAAA|28258|7|50|58|AM|first|morning|breakfast| +28259|AAAAAAAAEGOGAAAA|28259|7|50|59|AM|first|morning|breakfast| +28260|AAAAAAAAFGOGAAAA|28260|7|51|0|AM|first|morning|breakfast| +28261|AAAAAAAAGGOGAAAA|28261|7|51|1|AM|first|morning|breakfast| +28262|AAAAAAAAHGOGAAAA|28262|7|51|2|AM|first|morning|breakfast| +28263|AAAAAAAAIGOGAAAA|28263|7|51|3|AM|first|morning|breakfast| +28264|AAAAAAAAJGOGAAAA|28264|7|51|4|AM|first|morning|breakfast| +28265|AAAAAAAAKGOGAAAA|28265|7|51|5|AM|first|morning|breakfast| +28266|AAAAAAAALGOGAAAA|28266|7|51|6|AM|first|morning|breakfast| +28267|AAAAAAAAMGOGAAAA|28267|7|51|7|AM|first|morning|breakfast| +28268|AAAAAAAANGOGAAAA|28268|7|51|8|AM|first|morning|breakfast| +28269|AAAAAAAAOGOGAAAA|28269|7|51|9|AM|first|morning|breakfast| +28270|AAAAAAAAPGOGAAAA|28270|7|51|10|AM|first|morning|breakfast| +28271|AAAAAAAAAHOGAAAA|28271|7|51|11|AM|first|morning|breakfast| +28272|AAAAAAAABHOGAAAA|28272|7|51|12|AM|first|morning|breakfast| +28273|AAAAAAAACHOGAAAA|28273|7|51|13|AM|first|morning|breakfast| +28274|AAAAAAAADHOGAAAA|28274|7|51|14|AM|first|morning|breakfast| +28275|AAAAAAAAEHOGAAAA|28275|7|51|15|AM|first|morning|breakfast| +28276|AAAAAAAAFHOGAAAA|28276|7|51|16|AM|first|morning|breakfast| +28277|AAAAAAAAGHOGAAAA|28277|7|51|17|AM|first|morning|breakfast| +28278|AAAAAAAAHHOGAAAA|28278|7|51|18|AM|first|morning|breakfast| +28279|AAAAAAAAIHOGAAAA|28279|7|51|19|AM|first|morning|breakfast| +28280|AAAAAAAAJHOGAAAA|28280|7|51|20|AM|first|morning|breakfast| +28281|AAAAAAAAKHOGAAAA|28281|7|51|21|AM|first|morning|breakfast| +28282|AAAAAAAALHOGAAAA|28282|7|51|22|AM|first|morning|breakfast| +28283|AAAAAAAAMHOGAAAA|28283|7|51|23|AM|first|morning|breakfast| +28284|AAAAAAAANHOGAAAA|28284|7|51|24|AM|first|morning|breakfast| +28285|AAAAAAAAOHOGAAAA|28285|7|51|25|AM|first|morning|breakfast| +28286|AAAAAAAAPHOGAAAA|28286|7|51|26|AM|first|morning|breakfast| +28287|AAAAAAAAAIOGAAAA|28287|7|51|27|AM|first|morning|breakfast| +28288|AAAAAAAABIOGAAAA|28288|7|51|28|AM|first|morning|breakfast| +28289|AAAAAAAACIOGAAAA|28289|7|51|29|AM|first|morning|breakfast| +28290|AAAAAAAADIOGAAAA|28290|7|51|30|AM|first|morning|breakfast| +28291|AAAAAAAAEIOGAAAA|28291|7|51|31|AM|first|morning|breakfast| +28292|AAAAAAAAFIOGAAAA|28292|7|51|32|AM|first|morning|breakfast| +28293|AAAAAAAAGIOGAAAA|28293|7|51|33|AM|first|morning|breakfast| +28294|AAAAAAAAHIOGAAAA|28294|7|51|34|AM|first|morning|breakfast| +28295|AAAAAAAAIIOGAAAA|28295|7|51|35|AM|first|morning|breakfast| +28296|AAAAAAAAJIOGAAAA|28296|7|51|36|AM|first|morning|breakfast| +28297|AAAAAAAAKIOGAAAA|28297|7|51|37|AM|first|morning|breakfast| +28298|AAAAAAAALIOGAAAA|28298|7|51|38|AM|first|morning|breakfast| +28299|AAAAAAAAMIOGAAAA|28299|7|51|39|AM|first|morning|breakfast| +28300|AAAAAAAANIOGAAAA|28300|7|51|40|AM|first|morning|breakfast| +28301|AAAAAAAAOIOGAAAA|28301|7|51|41|AM|first|morning|breakfast| +28302|AAAAAAAAPIOGAAAA|28302|7|51|42|AM|first|morning|breakfast| +28303|AAAAAAAAAJOGAAAA|28303|7|51|43|AM|first|morning|breakfast| +28304|AAAAAAAABJOGAAAA|28304|7|51|44|AM|first|morning|breakfast| +28305|AAAAAAAACJOGAAAA|28305|7|51|45|AM|first|morning|breakfast| +28306|AAAAAAAADJOGAAAA|28306|7|51|46|AM|first|morning|breakfast| +28307|AAAAAAAAEJOGAAAA|28307|7|51|47|AM|first|morning|breakfast| +28308|AAAAAAAAFJOGAAAA|28308|7|51|48|AM|first|morning|breakfast| +28309|AAAAAAAAGJOGAAAA|28309|7|51|49|AM|first|morning|breakfast| +28310|AAAAAAAAHJOGAAAA|28310|7|51|50|AM|first|morning|breakfast| +28311|AAAAAAAAIJOGAAAA|28311|7|51|51|AM|first|morning|breakfast| +28312|AAAAAAAAJJOGAAAA|28312|7|51|52|AM|first|morning|breakfast| +28313|AAAAAAAAKJOGAAAA|28313|7|51|53|AM|first|morning|breakfast| +28314|AAAAAAAALJOGAAAA|28314|7|51|54|AM|first|morning|breakfast| +28315|AAAAAAAAMJOGAAAA|28315|7|51|55|AM|first|morning|breakfast| +28316|AAAAAAAANJOGAAAA|28316|7|51|56|AM|first|morning|breakfast| +28317|AAAAAAAAOJOGAAAA|28317|7|51|57|AM|first|morning|breakfast| +28318|AAAAAAAAPJOGAAAA|28318|7|51|58|AM|first|morning|breakfast| +28319|AAAAAAAAAKOGAAAA|28319|7|51|59|AM|first|morning|breakfast| +28320|AAAAAAAABKOGAAAA|28320|7|52|0|AM|first|morning|breakfast| +28321|AAAAAAAACKOGAAAA|28321|7|52|1|AM|first|morning|breakfast| +28322|AAAAAAAADKOGAAAA|28322|7|52|2|AM|first|morning|breakfast| +28323|AAAAAAAAEKOGAAAA|28323|7|52|3|AM|first|morning|breakfast| +28324|AAAAAAAAFKOGAAAA|28324|7|52|4|AM|first|morning|breakfast| +28325|AAAAAAAAGKOGAAAA|28325|7|52|5|AM|first|morning|breakfast| +28326|AAAAAAAAHKOGAAAA|28326|7|52|6|AM|first|morning|breakfast| +28327|AAAAAAAAIKOGAAAA|28327|7|52|7|AM|first|morning|breakfast| +28328|AAAAAAAAJKOGAAAA|28328|7|52|8|AM|first|morning|breakfast| +28329|AAAAAAAAKKOGAAAA|28329|7|52|9|AM|first|morning|breakfast| +28330|AAAAAAAALKOGAAAA|28330|7|52|10|AM|first|morning|breakfast| +28331|AAAAAAAAMKOGAAAA|28331|7|52|11|AM|first|morning|breakfast| +28332|AAAAAAAANKOGAAAA|28332|7|52|12|AM|first|morning|breakfast| +28333|AAAAAAAAOKOGAAAA|28333|7|52|13|AM|first|morning|breakfast| +28334|AAAAAAAAPKOGAAAA|28334|7|52|14|AM|first|morning|breakfast| +28335|AAAAAAAAALOGAAAA|28335|7|52|15|AM|first|morning|breakfast| +28336|AAAAAAAABLOGAAAA|28336|7|52|16|AM|first|morning|breakfast| +28337|AAAAAAAACLOGAAAA|28337|7|52|17|AM|first|morning|breakfast| +28338|AAAAAAAADLOGAAAA|28338|7|52|18|AM|first|morning|breakfast| +28339|AAAAAAAAELOGAAAA|28339|7|52|19|AM|first|morning|breakfast| +28340|AAAAAAAAFLOGAAAA|28340|7|52|20|AM|first|morning|breakfast| +28341|AAAAAAAAGLOGAAAA|28341|7|52|21|AM|first|morning|breakfast| +28342|AAAAAAAAHLOGAAAA|28342|7|52|22|AM|first|morning|breakfast| +28343|AAAAAAAAILOGAAAA|28343|7|52|23|AM|first|morning|breakfast| +28344|AAAAAAAAJLOGAAAA|28344|7|52|24|AM|first|morning|breakfast| +28345|AAAAAAAAKLOGAAAA|28345|7|52|25|AM|first|morning|breakfast| +28346|AAAAAAAALLOGAAAA|28346|7|52|26|AM|first|morning|breakfast| +28347|AAAAAAAAMLOGAAAA|28347|7|52|27|AM|first|morning|breakfast| +28348|AAAAAAAANLOGAAAA|28348|7|52|28|AM|first|morning|breakfast| +28349|AAAAAAAAOLOGAAAA|28349|7|52|29|AM|first|morning|breakfast| +28350|AAAAAAAAPLOGAAAA|28350|7|52|30|AM|first|morning|breakfast| +28351|AAAAAAAAAMOGAAAA|28351|7|52|31|AM|first|morning|breakfast| +28352|AAAAAAAABMOGAAAA|28352|7|52|32|AM|first|morning|breakfast| +28353|AAAAAAAACMOGAAAA|28353|7|52|33|AM|first|morning|breakfast| +28354|AAAAAAAADMOGAAAA|28354|7|52|34|AM|first|morning|breakfast| +28355|AAAAAAAAEMOGAAAA|28355|7|52|35|AM|first|morning|breakfast| +28356|AAAAAAAAFMOGAAAA|28356|7|52|36|AM|first|morning|breakfast| +28357|AAAAAAAAGMOGAAAA|28357|7|52|37|AM|first|morning|breakfast| +28358|AAAAAAAAHMOGAAAA|28358|7|52|38|AM|first|morning|breakfast| +28359|AAAAAAAAIMOGAAAA|28359|7|52|39|AM|first|morning|breakfast| +28360|AAAAAAAAJMOGAAAA|28360|7|52|40|AM|first|morning|breakfast| +28361|AAAAAAAAKMOGAAAA|28361|7|52|41|AM|first|morning|breakfast| +28362|AAAAAAAALMOGAAAA|28362|7|52|42|AM|first|morning|breakfast| +28363|AAAAAAAAMMOGAAAA|28363|7|52|43|AM|first|morning|breakfast| +28364|AAAAAAAANMOGAAAA|28364|7|52|44|AM|first|morning|breakfast| +28365|AAAAAAAAOMOGAAAA|28365|7|52|45|AM|first|morning|breakfast| +28366|AAAAAAAAPMOGAAAA|28366|7|52|46|AM|first|morning|breakfast| +28367|AAAAAAAAANOGAAAA|28367|7|52|47|AM|first|morning|breakfast| +28368|AAAAAAAABNOGAAAA|28368|7|52|48|AM|first|morning|breakfast| +28369|AAAAAAAACNOGAAAA|28369|7|52|49|AM|first|morning|breakfast| +28370|AAAAAAAADNOGAAAA|28370|7|52|50|AM|first|morning|breakfast| +28371|AAAAAAAAENOGAAAA|28371|7|52|51|AM|first|morning|breakfast| +28372|AAAAAAAAFNOGAAAA|28372|7|52|52|AM|first|morning|breakfast| +28373|AAAAAAAAGNOGAAAA|28373|7|52|53|AM|first|morning|breakfast| +28374|AAAAAAAAHNOGAAAA|28374|7|52|54|AM|first|morning|breakfast| +28375|AAAAAAAAINOGAAAA|28375|7|52|55|AM|first|morning|breakfast| +28376|AAAAAAAAJNOGAAAA|28376|7|52|56|AM|first|morning|breakfast| +28377|AAAAAAAAKNOGAAAA|28377|7|52|57|AM|first|morning|breakfast| +28378|AAAAAAAALNOGAAAA|28378|7|52|58|AM|first|morning|breakfast| +28379|AAAAAAAAMNOGAAAA|28379|7|52|59|AM|first|morning|breakfast| +28380|AAAAAAAANNOGAAAA|28380|7|53|0|AM|first|morning|breakfast| +28381|AAAAAAAAONOGAAAA|28381|7|53|1|AM|first|morning|breakfast| +28382|AAAAAAAAPNOGAAAA|28382|7|53|2|AM|first|morning|breakfast| +28383|AAAAAAAAAOOGAAAA|28383|7|53|3|AM|first|morning|breakfast| +28384|AAAAAAAABOOGAAAA|28384|7|53|4|AM|first|morning|breakfast| +28385|AAAAAAAACOOGAAAA|28385|7|53|5|AM|first|morning|breakfast| +28386|AAAAAAAADOOGAAAA|28386|7|53|6|AM|first|morning|breakfast| +28387|AAAAAAAAEOOGAAAA|28387|7|53|7|AM|first|morning|breakfast| +28388|AAAAAAAAFOOGAAAA|28388|7|53|8|AM|first|morning|breakfast| +28389|AAAAAAAAGOOGAAAA|28389|7|53|9|AM|first|morning|breakfast| +28390|AAAAAAAAHOOGAAAA|28390|7|53|10|AM|first|morning|breakfast| +28391|AAAAAAAAIOOGAAAA|28391|7|53|11|AM|first|morning|breakfast| +28392|AAAAAAAAJOOGAAAA|28392|7|53|12|AM|first|morning|breakfast| +28393|AAAAAAAAKOOGAAAA|28393|7|53|13|AM|first|morning|breakfast| +28394|AAAAAAAALOOGAAAA|28394|7|53|14|AM|first|morning|breakfast| +28395|AAAAAAAAMOOGAAAA|28395|7|53|15|AM|first|morning|breakfast| +28396|AAAAAAAANOOGAAAA|28396|7|53|16|AM|first|morning|breakfast| +28397|AAAAAAAAOOOGAAAA|28397|7|53|17|AM|first|morning|breakfast| +28398|AAAAAAAAPOOGAAAA|28398|7|53|18|AM|first|morning|breakfast| +28399|AAAAAAAAAPOGAAAA|28399|7|53|19|AM|first|morning|breakfast| +28400|AAAAAAAABPOGAAAA|28400|7|53|20|AM|first|morning|breakfast| +28401|AAAAAAAACPOGAAAA|28401|7|53|21|AM|first|morning|breakfast| +28402|AAAAAAAADPOGAAAA|28402|7|53|22|AM|first|morning|breakfast| +28403|AAAAAAAAEPOGAAAA|28403|7|53|23|AM|first|morning|breakfast| +28404|AAAAAAAAFPOGAAAA|28404|7|53|24|AM|first|morning|breakfast| +28405|AAAAAAAAGPOGAAAA|28405|7|53|25|AM|first|morning|breakfast| +28406|AAAAAAAAHPOGAAAA|28406|7|53|26|AM|first|morning|breakfast| +28407|AAAAAAAAIPOGAAAA|28407|7|53|27|AM|first|morning|breakfast| +28408|AAAAAAAAJPOGAAAA|28408|7|53|28|AM|first|morning|breakfast| +28409|AAAAAAAAKPOGAAAA|28409|7|53|29|AM|first|morning|breakfast| +28410|AAAAAAAALPOGAAAA|28410|7|53|30|AM|first|morning|breakfast| +28411|AAAAAAAAMPOGAAAA|28411|7|53|31|AM|first|morning|breakfast| +28412|AAAAAAAANPOGAAAA|28412|7|53|32|AM|first|morning|breakfast| +28413|AAAAAAAAOPOGAAAA|28413|7|53|33|AM|first|morning|breakfast| +28414|AAAAAAAAPPOGAAAA|28414|7|53|34|AM|first|morning|breakfast| +28415|AAAAAAAAAAPGAAAA|28415|7|53|35|AM|first|morning|breakfast| +28416|AAAAAAAABAPGAAAA|28416|7|53|36|AM|first|morning|breakfast| +28417|AAAAAAAACAPGAAAA|28417|7|53|37|AM|first|morning|breakfast| +28418|AAAAAAAADAPGAAAA|28418|7|53|38|AM|first|morning|breakfast| +28419|AAAAAAAAEAPGAAAA|28419|7|53|39|AM|first|morning|breakfast| +28420|AAAAAAAAFAPGAAAA|28420|7|53|40|AM|first|morning|breakfast| +28421|AAAAAAAAGAPGAAAA|28421|7|53|41|AM|first|morning|breakfast| +28422|AAAAAAAAHAPGAAAA|28422|7|53|42|AM|first|morning|breakfast| +28423|AAAAAAAAIAPGAAAA|28423|7|53|43|AM|first|morning|breakfast| +28424|AAAAAAAAJAPGAAAA|28424|7|53|44|AM|first|morning|breakfast| +28425|AAAAAAAAKAPGAAAA|28425|7|53|45|AM|first|morning|breakfast| +28426|AAAAAAAALAPGAAAA|28426|7|53|46|AM|first|morning|breakfast| +28427|AAAAAAAAMAPGAAAA|28427|7|53|47|AM|first|morning|breakfast| +28428|AAAAAAAANAPGAAAA|28428|7|53|48|AM|first|morning|breakfast| +28429|AAAAAAAAOAPGAAAA|28429|7|53|49|AM|first|morning|breakfast| +28430|AAAAAAAAPAPGAAAA|28430|7|53|50|AM|first|morning|breakfast| +28431|AAAAAAAAABPGAAAA|28431|7|53|51|AM|first|morning|breakfast| +28432|AAAAAAAABBPGAAAA|28432|7|53|52|AM|first|morning|breakfast| +28433|AAAAAAAACBPGAAAA|28433|7|53|53|AM|first|morning|breakfast| +28434|AAAAAAAADBPGAAAA|28434|7|53|54|AM|first|morning|breakfast| +28435|AAAAAAAAEBPGAAAA|28435|7|53|55|AM|first|morning|breakfast| +28436|AAAAAAAAFBPGAAAA|28436|7|53|56|AM|first|morning|breakfast| +28437|AAAAAAAAGBPGAAAA|28437|7|53|57|AM|first|morning|breakfast| +28438|AAAAAAAAHBPGAAAA|28438|7|53|58|AM|first|morning|breakfast| +28439|AAAAAAAAIBPGAAAA|28439|7|53|59|AM|first|morning|breakfast| +28440|AAAAAAAAJBPGAAAA|28440|7|54|0|AM|first|morning|breakfast| +28441|AAAAAAAAKBPGAAAA|28441|7|54|1|AM|first|morning|breakfast| +28442|AAAAAAAALBPGAAAA|28442|7|54|2|AM|first|morning|breakfast| +28443|AAAAAAAAMBPGAAAA|28443|7|54|3|AM|first|morning|breakfast| +28444|AAAAAAAANBPGAAAA|28444|7|54|4|AM|first|morning|breakfast| +28445|AAAAAAAAOBPGAAAA|28445|7|54|5|AM|first|morning|breakfast| +28446|AAAAAAAAPBPGAAAA|28446|7|54|6|AM|first|morning|breakfast| +28447|AAAAAAAAACPGAAAA|28447|7|54|7|AM|first|morning|breakfast| +28448|AAAAAAAABCPGAAAA|28448|7|54|8|AM|first|morning|breakfast| +28449|AAAAAAAACCPGAAAA|28449|7|54|9|AM|first|morning|breakfast| +28450|AAAAAAAADCPGAAAA|28450|7|54|10|AM|first|morning|breakfast| +28451|AAAAAAAAECPGAAAA|28451|7|54|11|AM|first|morning|breakfast| +28452|AAAAAAAAFCPGAAAA|28452|7|54|12|AM|first|morning|breakfast| +28453|AAAAAAAAGCPGAAAA|28453|7|54|13|AM|first|morning|breakfast| +28454|AAAAAAAAHCPGAAAA|28454|7|54|14|AM|first|morning|breakfast| +28455|AAAAAAAAICPGAAAA|28455|7|54|15|AM|first|morning|breakfast| +28456|AAAAAAAAJCPGAAAA|28456|7|54|16|AM|first|morning|breakfast| +28457|AAAAAAAAKCPGAAAA|28457|7|54|17|AM|first|morning|breakfast| +28458|AAAAAAAALCPGAAAA|28458|7|54|18|AM|first|morning|breakfast| +28459|AAAAAAAAMCPGAAAA|28459|7|54|19|AM|first|morning|breakfast| +28460|AAAAAAAANCPGAAAA|28460|7|54|20|AM|first|morning|breakfast| +28461|AAAAAAAAOCPGAAAA|28461|7|54|21|AM|first|morning|breakfast| +28462|AAAAAAAAPCPGAAAA|28462|7|54|22|AM|first|morning|breakfast| +28463|AAAAAAAAADPGAAAA|28463|7|54|23|AM|first|morning|breakfast| +28464|AAAAAAAABDPGAAAA|28464|7|54|24|AM|first|morning|breakfast| +28465|AAAAAAAACDPGAAAA|28465|7|54|25|AM|first|morning|breakfast| +28466|AAAAAAAADDPGAAAA|28466|7|54|26|AM|first|morning|breakfast| +28467|AAAAAAAAEDPGAAAA|28467|7|54|27|AM|first|morning|breakfast| +28468|AAAAAAAAFDPGAAAA|28468|7|54|28|AM|first|morning|breakfast| +28469|AAAAAAAAGDPGAAAA|28469|7|54|29|AM|first|morning|breakfast| +28470|AAAAAAAAHDPGAAAA|28470|7|54|30|AM|first|morning|breakfast| +28471|AAAAAAAAIDPGAAAA|28471|7|54|31|AM|first|morning|breakfast| +28472|AAAAAAAAJDPGAAAA|28472|7|54|32|AM|first|morning|breakfast| +28473|AAAAAAAAKDPGAAAA|28473|7|54|33|AM|first|morning|breakfast| +28474|AAAAAAAALDPGAAAA|28474|7|54|34|AM|first|morning|breakfast| +28475|AAAAAAAAMDPGAAAA|28475|7|54|35|AM|first|morning|breakfast| +28476|AAAAAAAANDPGAAAA|28476|7|54|36|AM|first|morning|breakfast| +28477|AAAAAAAAODPGAAAA|28477|7|54|37|AM|first|morning|breakfast| +28478|AAAAAAAAPDPGAAAA|28478|7|54|38|AM|first|morning|breakfast| +28479|AAAAAAAAAEPGAAAA|28479|7|54|39|AM|first|morning|breakfast| +28480|AAAAAAAABEPGAAAA|28480|7|54|40|AM|first|morning|breakfast| +28481|AAAAAAAACEPGAAAA|28481|7|54|41|AM|first|morning|breakfast| +28482|AAAAAAAADEPGAAAA|28482|7|54|42|AM|first|morning|breakfast| +28483|AAAAAAAAEEPGAAAA|28483|7|54|43|AM|first|morning|breakfast| +28484|AAAAAAAAFEPGAAAA|28484|7|54|44|AM|first|morning|breakfast| +28485|AAAAAAAAGEPGAAAA|28485|7|54|45|AM|first|morning|breakfast| +28486|AAAAAAAAHEPGAAAA|28486|7|54|46|AM|first|morning|breakfast| +28487|AAAAAAAAIEPGAAAA|28487|7|54|47|AM|first|morning|breakfast| +28488|AAAAAAAAJEPGAAAA|28488|7|54|48|AM|first|morning|breakfast| +28489|AAAAAAAAKEPGAAAA|28489|7|54|49|AM|first|morning|breakfast| +28490|AAAAAAAALEPGAAAA|28490|7|54|50|AM|first|morning|breakfast| +28491|AAAAAAAAMEPGAAAA|28491|7|54|51|AM|first|morning|breakfast| +28492|AAAAAAAANEPGAAAA|28492|7|54|52|AM|first|morning|breakfast| +28493|AAAAAAAAOEPGAAAA|28493|7|54|53|AM|first|morning|breakfast| +28494|AAAAAAAAPEPGAAAA|28494|7|54|54|AM|first|morning|breakfast| +28495|AAAAAAAAAFPGAAAA|28495|7|54|55|AM|first|morning|breakfast| +28496|AAAAAAAABFPGAAAA|28496|7|54|56|AM|first|morning|breakfast| +28497|AAAAAAAACFPGAAAA|28497|7|54|57|AM|first|morning|breakfast| +28498|AAAAAAAADFPGAAAA|28498|7|54|58|AM|first|morning|breakfast| +28499|AAAAAAAAEFPGAAAA|28499|7|54|59|AM|first|morning|breakfast| +28500|AAAAAAAAFFPGAAAA|28500|7|55|0|AM|first|morning|breakfast| +28501|AAAAAAAAGFPGAAAA|28501|7|55|1|AM|first|morning|breakfast| +28502|AAAAAAAAHFPGAAAA|28502|7|55|2|AM|first|morning|breakfast| +28503|AAAAAAAAIFPGAAAA|28503|7|55|3|AM|first|morning|breakfast| +28504|AAAAAAAAJFPGAAAA|28504|7|55|4|AM|first|morning|breakfast| +28505|AAAAAAAAKFPGAAAA|28505|7|55|5|AM|first|morning|breakfast| +28506|AAAAAAAALFPGAAAA|28506|7|55|6|AM|first|morning|breakfast| +28507|AAAAAAAAMFPGAAAA|28507|7|55|7|AM|first|morning|breakfast| +28508|AAAAAAAANFPGAAAA|28508|7|55|8|AM|first|morning|breakfast| +28509|AAAAAAAAOFPGAAAA|28509|7|55|9|AM|first|morning|breakfast| +28510|AAAAAAAAPFPGAAAA|28510|7|55|10|AM|first|morning|breakfast| +28511|AAAAAAAAAGPGAAAA|28511|7|55|11|AM|first|morning|breakfast| +28512|AAAAAAAABGPGAAAA|28512|7|55|12|AM|first|morning|breakfast| +28513|AAAAAAAACGPGAAAA|28513|7|55|13|AM|first|morning|breakfast| +28514|AAAAAAAADGPGAAAA|28514|7|55|14|AM|first|morning|breakfast| +28515|AAAAAAAAEGPGAAAA|28515|7|55|15|AM|first|morning|breakfast| +28516|AAAAAAAAFGPGAAAA|28516|7|55|16|AM|first|morning|breakfast| +28517|AAAAAAAAGGPGAAAA|28517|7|55|17|AM|first|morning|breakfast| +28518|AAAAAAAAHGPGAAAA|28518|7|55|18|AM|first|morning|breakfast| +28519|AAAAAAAAIGPGAAAA|28519|7|55|19|AM|first|morning|breakfast| +28520|AAAAAAAAJGPGAAAA|28520|7|55|20|AM|first|morning|breakfast| +28521|AAAAAAAAKGPGAAAA|28521|7|55|21|AM|first|morning|breakfast| +28522|AAAAAAAALGPGAAAA|28522|7|55|22|AM|first|morning|breakfast| +28523|AAAAAAAAMGPGAAAA|28523|7|55|23|AM|first|morning|breakfast| +28524|AAAAAAAANGPGAAAA|28524|7|55|24|AM|first|morning|breakfast| +28525|AAAAAAAAOGPGAAAA|28525|7|55|25|AM|first|morning|breakfast| +28526|AAAAAAAAPGPGAAAA|28526|7|55|26|AM|first|morning|breakfast| +28527|AAAAAAAAAHPGAAAA|28527|7|55|27|AM|first|morning|breakfast| +28528|AAAAAAAABHPGAAAA|28528|7|55|28|AM|first|morning|breakfast| +28529|AAAAAAAACHPGAAAA|28529|7|55|29|AM|first|morning|breakfast| +28530|AAAAAAAADHPGAAAA|28530|7|55|30|AM|first|morning|breakfast| +28531|AAAAAAAAEHPGAAAA|28531|7|55|31|AM|first|morning|breakfast| +28532|AAAAAAAAFHPGAAAA|28532|7|55|32|AM|first|morning|breakfast| +28533|AAAAAAAAGHPGAAAA|28533|7|55|33|AM|first|morning|breakfast| +28534|AAAAAAAAHHPGAAAA|28534|7|55|34|AM|first|morning|breakfast| +28535|AAAAAAAAIHPGAAAA|28535|7|55|35|AM|first|morning|breakfast| +28536|AAAAAAAAJHPGAAAA|28536|7|55|36|AM|first|morning|breakfast| +28537|AAAAAAAAKHPGAAAA|28537|7|55|37|AM|first|morning|breakfast| +28538|AAAAAAAALHPGAAAA|28538|7|55|38|AM|first|morning|breakfast| +28539|AAAAAAAAMHPGAAAA|28539|7|55|39|AM|first|morning|breakfast| +28540|AAAAAAAANHPGAAAA|28540|7|55|40|AM|first|morning|breakfast| +28541|AAAAAAAAOHPGAAAA|28541|7|55|41|AM|first|morning|breakfast| +28542|AAAAAAAAPHPGAAAA|28542|7|55|42|AM|first|morning|breakfast| +28543|AAAAAAAAAIPGAAAA|28543|7|55|43|AM|first|morning|breakfast| +28544|AAAAAAAABIPGAAAA|28544|7|55|44|AM|first|morning|breakfast| +28545|AAAAAAAACIPGAAAA|28545|7|55|45|AM|first|morning|breakfast| +28546|AAAAAAAADIPGAAAA|28546|7|55|46|AM|first|morning|breakfast| +28547|AAAAAAAAEIPGAAAA|28547|7|55|47|AM|first|morning|breakfast| +28548|AAAAAAAAFIPGAAAA|28548|7|55|48|AM|first|morning|breakfast| +28549|AAAAAAAAGIPGAAAA|28549|7|55|49|AM|first|morning|breakfast| +28550|AAAAAAAAHIPGAAAA|28550|7|55|50|AM|first|morning|breakfast| +28551|AAAAAAAAIIPGAAAA|28551|7|55|51|AM|first|morning|breakfast| +28552|AAAAAAAAJIPGAAAA|28552|7|55|52|AM|first|morning|breakfast| +28553|AAAAAAAAKIPGAAAA|28553|7|55|53|AM|first|morning|breakfast| +28554|AAAAAAAALIPGAAAA|28554|7|55|54|AM|first|morning|breakfast| +28555|AAAAAAAAMIPGAAAA|28555|7|55|55|AM|first|morning|breakfast| +28556|AAAAAAAANIPGAAAA|28556|7|55|56|AM|first|morning|breakfast| +28557|AAAAAAAAOIPGAAAA|28557|7|55|57|AM|first|morning|breakfast| +28558|AAAAAAAAPIPGAAAA|28558|7|55|58|AM|first|morning|breakfast| +28559|AAAAAAAAAJPGAAAA|28559|7|55|59|AM|first|morning|breakfast| +28560|AAAAAAAABJPGAAAA|28560|7|56|0|AM|first|morning|breakfast| +28561|AAAAAAAACJPGAAAA|28561|7|56|1|AM|first|morning|breakfast| +28562|AAAAAAAADJPGAAAA|28562|7|56|2|AM|first|morning|breakfast| +28563|AAAAAAAAEJPGAAAA|28563|7|56|3|AM|first|morning|breakfast| +28564|AAAAAAAAFJPGAAAA|28564|7|56|4|AM|first|morning|breakfast| +28565|AAAAAAAAGJPGAAAA|28565|7|56|5|AM|first|morning|breakfast| +28566|AAAAAAAAHJPGAAAA|28566|7|56|6|AM|first|morning|breakfast| +28567|AAAAAAAAIJPGAAAA|28567|7|56|7|AM|first|morning|breakfast| +28568|AAAAAAAAJJPGAAAA|28568|7|56|8|AM|first|morning|breakfast| +28569|AAAAAAAAKJPGAAAA|28569|7|56|9|AM|first|morning|breakfast| +28570|AAAAAAAALJPGAAAA|28570|7|56|10|AM|first|morning|breakfast| +28571|AAAAAAAAMJPGAAAA|28571|7|56|11|AM|first|morning|breakfast| +28572|AAAAAAAANJPGAAAA|28572|7|56|12|AM|first|morning|breakfast| +28573|AAAAAAAAOJPGAAAA|28573|7|56|13|AM|first|morning|breakfast| +28574|AAAAAAAAPJPGAAAA|28574|7|56|14|AM|first|morning|breakfast| +28575|AAAAAAAAAKPGAAAA|28575|7|56|15|AM|first|morning|breakfast| +28576|AAAAAAAABKPGAAAA|28576|7|56|16|AM|first|morning|breakfast| +28577|AAAAAAAACKPGAAAA|28577|7|56|17|AM|first|morning|breakfast| +28578|AAAAAAAADKPGAAAA|28578|7|56|18|AM|first|morning|breakfast| +28579|AAAAAAAAEKPGAAAA|28579|7|56|19|AM|first|morning|breakfast| +28580|AAAAAAAAFKPGAAAA|28580|7|56|20|AM|first|morning|breakfast| +28581|AAAAAAAAGKPGAAAA|28581|7|56|21|AM|first|morning|breakfast| +28582|AAAAAAAAHKPGAAAA|28582|7|56|22|AM|first|morning|breakfast| +28583|AAAAAAAAIKPGAAAA|28583|7|56|23|AM|first|morning|breakfast| +28584|AAAAAAAAJKPGAAAA|28584|7|56|24|AM|first|morning|breakfast| +28585|AAAAAAAAKKPGAAAA|28585|7|56|25|AM|first|morning|breakfast| +28586|AAAAAAAALKPGAAAA|28586|7|56|26|AM|first|morning|breakfast| +28587|AAAAAAAAMKPGAAAA|28587|7|56|27|AM|first|morning|breakfast| +28588|AAAAAAAANKPGAAAA|28588|7|56|28|AM|first|morning|breakfast| +28589|AAAAAAAAOKPGAAAA|28589|7|56|29|AM|first|morning|breakfast| +28590|AAAAAAAAPKPGAAAA|28590|7|56|30|AM|first|morning|breakfast| +28591|AAAAAAAAALPGAAAA|28591|7|56|31|AM|first|morning|breakfast| +28592|AAAAAAAABLPGAAAA|28592|7|56|32|AM|first|morning|breakfast| +28593|AAAAAAAACLPGAAAA|28593|7|56|33|AM|first|morning|breakfast| +28594|AAAAAAAADLPGAAAA|28594|7|56|34|AM|first|morning|breakfast| +28595|AAAAAAAAELPGAAAA|28595|7|56|35|AM|first|morning|breakfast| +28596|AAAAAAAAFLPGAAAA|28596|7|56|36|AM|first|morning|breakfast| +28597|AAAAAAAAGLPGAAAA|28597|7|56|37|AM|first|morning|breakfast| +28598|AAAAAAAAHLPGAAAA|28598|7|56|38|AM|first|morning|breakfast| +28599|AAAAAAAAILPGAAAA|28599|7|56|39|AM|first|morning|breakfast| +28600|AAAAAAAAJLPGAAAA|28600|7|56|40|AM|first|morning|breakfast| +28601|AAAAAAAAKLPGAAAA|28601|7|56|41|AM|first|morning|breakfast| +28602|AAAAAAAALLPGAAAA|28602|7|56|42|AM|first|morning|breakfast| +28603|AAAAAAAAMLPGAAAA|28603|7|56|43|AM|first|morning|breakfast| +28604|AAAAAAAANLPGAAAA|28604|7|56|44|AM|first|morning|breakfast| +28605|AAAAAAAAOLPGAAAA|28605|7|56|45|AM|first|morning|breakfast| +28606|AAAAAAAAPLPGAAAA|28606|7|56|46|AM|first|morning|breakfast| +28607|AAAAAAAAAMPGAAAA|28607|7|56|47|AM|first|morning|breakfast| +28608|AAAAAAAABMPGAAAA|28608|7|56|48|AM|first|morning|breakfast| +28609|AAAAAAAACMPGAAAA|28609|7|56|49|AM|first|morning|breakfast| +28610|AAAAAAAADMPGAAAA|28610|7|56|50|AM|first|morning|breakfast| +28611|AAAAAAAAEMPGAAAA|28611|7|56|51|AM|first|morning|breakfast| +28612|AAAAAAAAFMPGAAAA|28612|7|56|52|AM|first|morning|breakfast| +28613|AAAAAAAAGMPGAAAA|28613|7|56|53|AM|first|morning|breakfast| +28614|AAAAAAAAHMPGAAAA|28614|7|56|54|AM|first|morning|breakfast| +28615|AAAAAAAAIMPGAAAA|28615|7|56|55|AM|first|morning|breakfast| +28616|AAAAAAAAJMPGAAAA|28616|7|56|56|AM|first|morning|breakfast| +28617|AAAAAAAAKMPGAAAA|28617|7|56|57|AM|first|morning|breakfast| +28618|AAAAAAAALMPGAAAA|28618|7|56|58|AM|first|morning|breakfast| +28619|AAAAAAAAMMPGAAAA|28619|7|56|59|AM|first|morning|breakfast| +28620|AAAAAAAANMPGAAAA|28620|7|57|0|AM|first|morning|breakfast| +28621|AAAAAAAAOMPGAAAA|28621|7|57|1|AM|first|morning|breakfast| +28622|AAAAAAAAPMPGAAAA|28622|7|57|2|AM|first|morning|breakfast| +28623|AAAAAAAAANPGAAAA|28623|7|57|3|AM|first|morning|breakfast| +28624|AAAAAAAABNPGAAAA|28624|7|57|4|AM|first|morning|breakfast| +28625|AAAAAAAACNPGAAAA|28625|7|57|5|AM|first|morning|breakfast| +28626|AAAAAAAADNPGAAAA|28626|7|57|6|AM|first|morning|breakfast| +28627|AAAAAAAAENPGAAAA|28627|7|57|7|AM|first|morning|breakfast| +28628|AAAAAAAAFNPGAAAA|28628|7|57|8|AM|first|morning|breakfast| +28629|AAAAAAAAGNPGAAAA|28629|7|57|9|AM|first|morning|breakfast| +28630|AAAAAAAAHNPGAAAA|28630|7|57|10|AM|first|morning|breakfast| +28631|AAAAAAAAINPGAAAA|28631|7|57|11|AM|first|morning|breakfast| +28632|AAAAAAAAJNPGAAAA|28632|7|57|12|AM|first|morning|breakfast| +28633|AAAAAAAAKNPGAAAA|28633|7|57|13|AM|first|morning|breakfast| +28634|AAAAAAAALNPGAAAA|28634|7|57|14|AM|first|morning|breakfast| +28635|AAAAAAAAMNPGAAAA|28635|7|57|15|AM|first|morning|breakfast| +28636|AAAAAAAANNPGAAAA|28636|7|57|16|AM|first|morning|breakfast| +28637|AAAAAAAAONPGAAAA|28637|7|57|17|AM|first|morning|breakfast| +28638|AAAAAAAAPNPGAAAA|28638|7|57|18|AM|first|morning|breakfast| +28639|AAAAAAAAAOPGAAAA|28639|7|57|19|AM|first|morning|breakfast| +28640|AAAAAAAABOPGAAAA|28640|7|57|20|AM|first|morning|breakfast| +28641|AAAAAAAACOPGAAAA|28641|7|57|21|AM|first|morning|breakfast| +28642|AAAAAAAADOPGAAAA|28642|7|57|22|AM|first|morning|breakfast| +28643|AAAAAAAAEOPGAAAA|28643|7|57|23|AM|first|morning|breakfast| +28644|AAAAAAAAFOPGAAAA|28644|7|57|24|AM|first|morning|breakfast| +28645|AAAAAAAAGOPGAAAA|28645|7|57|25|AM|first|morning|breakfast| +28646|AAAAAAAAHOPGAAAA|28646|7|57|26|AM|first|morning|breakfast| +28647|AAAAAAAAIOPGAAAA|28647|7|57|27|AM|first|morning|breakfast| +28648|AAAAAAAAJOPGAAAA|28648|7|57|28|AM|first|morning|breakfast| +28649|AAAAAAAAKOPGAAAA|28649|7|57|29|AM|first|morning|breakfast| +28650|AAAAAAAALOPGAAAA|28650|7|57|30|AM|first|morning|breakfast| +28651|AAAAAAAAMOPGAAAA|28651|7|57|31|AM|first|morning|breakfast| +28652|AAAAAAAANOPGAAAA|28652|7|57|32|AM|first|morning|breakfast| +28653|AAAAAAAAOOPGAAAA|28653|7|57|33|AM|first|morning|breakfast| +28654|AAAAAAAAPOPGAAAA|28654|7|57|34|AM|first|morning|breakfast| +28655|AAAAAAAAAPPGAAAA|28655|7|57|35|AM|first|morning|breakfast| +28656|AAAAAAAABPPGAAAA|28656|7|57|36|AM|first|morning|breakfast| +28657|AAAAAAAACPPGAAAA|28657|7|57|37|AM|first|morning|breakfast| +28658|AAAAAAAADPPGAAAA|28658|7|57|38|AM|first|morning|breakfast| +28659|AAAAAAAAEPPGAAAA|28659|7|57|39|AM|first|morning|breakfast| +28660|AAAAAAAAFPPGAAAA|28660|7|57|40|AM|first|morning|breakfast| +28661|AAAAAAAAGPPGAAAA|28661|7|57|41|AM|first|morning|breakfast| +28662|AAAAAAAAHPPGAAAA|28662|7|57|42|AM|first|morning|breakfast| +28663|AAAAAAAAIPPGAAAA|28663|7|57|43|AM|first|morning|breakfast| +28664|AAAAAAAAJPPGAAAA|28664|7|57|44|AM|first|morning|breakfast| +28665|AAAAAAAAKPPGAAAA|28665|7|57|45|AM|first|morning|breakfast| +28666|AAAAAAAALPPGAAAA|28666|7|57|46|AM|first|morning|breakfast| +28667|AAAAAAAAMPPGAAAA|28667|7|57|47|AM|first|morning|breakfast| +28668|AAAAAAAANPPGAAAA|28668|7|57|48|AM|first|morning|breakfast| +28669|AAAAAAAAOPPGAAAA|28669|7|57|49|AM|first|morning|breakfast| +28670|AAAAAAAAPPPGAAAA|28670|7|57|50|AM|first|morning|breakfast| +28671|AAAAAAAAAAAHAAAA|28671|7|57|51|AM|first|morning|breakfast| +28672|AAAAAAAABAAHAAAA|28672|7|57|52|AM|first|morning|breakfast| +28673|AAAAAAAACAAHAAAA|28673|7|57|53|AM|first|morning|breakfast| +28674|AAAAAAAADAAHAAAA|28674|7|57|54|AM|first|morning|breakfast| +28675|AAAAAAAAEAAHAAAA|28675|7|57|55|AM|first|morning|breakfast| +28676|AAAAAAAAFAAHAAAA|28676|7|57|56|AM|first|morning|breakfast| +28677|AAAAAAAAGAAHAAAA|28677|7|57|57|AM|first|morning|breakfast| +28678|AAAAAAAAHAAHAAAA|28678|7|57|58|AM|first|morning|breakfast| +28679|AAAAAAAAIAAHAAAA|28679|7|57|59|AM|first|morning|breakfast| +28680|AAAAAAAAJAAHAAAA|28680|7|58|0|AM|first|morning|breakfast| +28681|AAAAAAAAKAAHAAAA|28681|7|58|1|AM|first|morning|breakfast| +28682|AAAAAAAALAAHAAAA|28682|7|58|2|AM|first|morning|breakfast| +28683|AAAAAAAAMAAHAAAA|28683|7|58|3|AM|first|morning|breakfast| +28684|AAAAAAAANAAHAAAA|28684|7|58|4|AM|first|morning|breakfast| +28685|AAAAAAAAOAAHAAAA|28685|7|58|5|AM|first|morning|breakfast| +28686|AAAAAAAAPAAHAAAA|28686|7|58|6|AM|first|morning|breakfast| +28687|AAAAAAAAABAHAAAA|28687|7|58|7|AM|first|morning|breakfast| +28688|AAAAAAAABBAHAAAA|28688|7|58|8|AM|first|morning|breakfast| +28689|AAAAAAAACBAHAAAA|28689|7|58|9|AM|first|morning|breakfast| +28690|AAAAAAAADBAHAAAA|28690|7|58|10|AM|first|morning|breakfast| +28691|AAAAAAAAEBAHAAAA|28691|7|58|11|AM|first|morning|breakfast| +28692|AAAAAAAAFBAHAAAA|28692|7|58|12|AM|first|morning|breakfast| +28693|AAAAAAAAGBAHAAAA|28693|7|58|13|AM|first|morning|breakfast| +28694|AAAAAAAAHBAHAAAA|28694|7|58|14|AM|first|morning|breakfast| +28695|AAAAAAAAIBAHAAAA|28695|7|58|15|AM|first|morning|breakfast| +28696|AAAAAAAAJBAHAAAA|28696|7|58|16|AM|first|morning|breakfast| +28697|AAAAAAAAKBAHAAAA|28697|7|58|17|AM|first|morning|breakfast| +28698|AAAAAAAALBAHAAAA|28698|7|58|18|AM|first|morning|breakfast| +28699|AAAAAAAAMBAHAAAA|28699|7|58|19|AM|first|morning|breakfast| +28700|AAAAAAAANBAHAAAA|28700|7|58|20|AM|first|morning|breakfast| +28701|AAAAAAAAOBAHAAAA|28701|7|58|21|AM|first|morning|breakfast| +28702|AAAAAAAAPBAHAAAA|28702|7|58|22|AM|first|morning|breakfast| +28703|AAAAAAAAACAHAAAA|28703|7|58|23|AM|first|morning|breakfast| +28704|AAAAAAAABCAHAAAA|28704|7|58|24|AM|first|morning|breakfast| +28705|AAAAAAAACCAHAAAA|28705|7|58|25|AM|first|morning|breakfast| +28706|AAAAAAAADCAHAAAA|28706|7|58|26|AM|first|morning|breakfast| +28707|AAAAAAAAECAHAAAA|28707|7|58|27|AM|first|morning|breakfast| +28708|AAAAAAAAFCAHAAAA|28708|7|58|28|AM|first|morning|breakfast| +28709|AAAAAAAAGCAHAAAA|28709|7|58|29|AM|first|morning|breakfast| +28710|AAAAAAAAHCAHAAAA|28710|7|58|30|AM|first|morning|breakfast| +28711|AAAAAAAAICAHAAAA|28711|7|58|31|AM|first|morning|breakfast| +28712|AAAAAAAAJCAHAAAA|28712|7|58|32|AM|first|morning|breakfast| +28713|AAAAAAAAKCAHAAAA|28713|7|58|33|AM|first|morning|breakfast| +28714|AAAAAAAALCAHAAAA|28714|7|58|34|AM|first|morning|breakfast| +28715|AAAAAAAAMCAHAAAA|28715|7|58|35|AM|first|morning|breakfast| +28716|AAAAAAAANCAHAAAA|28716|7|58|36|AM|first|morning|breakfast| +28717|AAAAAAAAOCAHAAAA|28717|7|58|37|AM|first|morning|breakfast| +28718|AAAAAAAAPCAHAAAA|28718|7|58|38|AM|first|morning|breakfast| +28719|AAAAAAAAADAHAAAA|28719|7|58|39|AM|first|morning|breakfast| +28720|AAAAAAAABDAHAAAA|28720|7|58|40|AM|first|morning|breakfast| +28721|AAAAAAAACDAHAAAA|28721|7|58|41|AM|first|morning|breakfast| +28722|AAAAAAAADDAHAAAA|28722|7|58|42|AM|first|morning|breakfast| +28723|AAAAAAAAEDAHAAAA|28723|7|58|43|AM|first|morning|breakfast| +28724|AAAAAAAAFDAHAAAA|28724|7|58|44|AM|first|morning|breakfast| +28725|AAAAAAAAGDAHAAAA|28725|7|58|45|AM|first|morning|breakfast| +28726|AAAAAAAAHDAHAAAA|28726|7|58|46|AM|first|morning|breakfast| +28727|AAAAAAAAIDAHAAAA|28727|7|58|47|AM|first|morning|breakfast| +28728|AAAAAAAAJDAHAAAA|28728|7|58|48|AM|first|morning|breakfast| +28729|AAAAAAAAKDAHAAAA|28729|7|58|49|AM|first|morning|breakfast| +28730|AAAAAAAALDAHAAAA|28730|7|58|50|AM|first|morning|breakfast| +28731|AAAAAAAAMDAHAAAA|28731|7|58|51|AM|first|morning|breakfast| +28732|AAAAAAAANDAHAAAA|28732|7|58|52|AM|first|morning|breakfast| +28733|AAAAAAAAODAHAAAA|28733|7|58|53|AM|first|morning|breakfast| +28734|AAAAAAAAPDAHAAAA|28734|7|58|54|AM|first|morning|breakfast| +28735|AAAAAAAAAEAHAAAA|28735|7|58|55|AM|first|morning|breakfast| +28736|AAAAAAAABEAHAAAA|28736|7|58|56|AM|first|morning|breakfast| +28737|AAAAAAAACEAHAAAA|28737|7|58|57|AM|first|morning|breakfast| +28738|AAAAAAAADEAHAAAA|28738|7|58|58|AM|first|morning|breakfast| +28739|AAAAAAAAEEAHAAAA|28739|7|58|59|AM|first|morning|breakfast| +28740|AAAAAAAAFEAHAAAA|28740|7|59|0|AM|first|morning|breakfast| +28741|AAAAAAAAGEAHAAAA|28741|7|59|1|AM|first|morning|breakfast| +28742|AAAAAAAAHEAHAAAA|28742|7|59|2|AM|first|morning|breakfast| +28743|AAAAAAAAIEAHAAAA|28743|7|59|3|AM|first|morning|breakfast| +28744|AAAAAAAAJEAHAAAA|28744|7|59|4|AM|first|morning|breakfast| +28745|AAAAAAAAKEAHAAAA|28745|7|59|5|AM|first|morning|breakfast| +28746|AAAAAAAALEAHAAAA|28746|7|59|6|AM|first|morning|breakfast| +28747|AAAAAAAAMEAHAAAA|28747|7|59|7|AM|first|morning|breakfast| +28748|AAAAAAAANEAHAAAA|28748|7|59|8|AM|first|morning|breakfast| +28749|AAAAAAAAOEAHAAAA|28749|7|59|9|AM|first|morning|breakfast| +28750|AAAAAAAAPEAHAAAA|28750|7|59|10|AM|first|morning|breakfast| +28751|AAAAAAAAAFAHAAAA|28751|7|59|11|AM|first|morning|breakfast| +28752|AAAAAAAABFAHAAAA|28752|7|59|12|AM|first|morning|breakfast| +28753|AAAAAAAACFAHAAAA|28753|7|59|13|AM|first|morning|breakfast| +28754|AAAAAAAADFAHAAAA|28754|7|59|14|AM|first|morning|breakfast| +28755|AAAAAAAAEFAHAAAA|28755|7|59|15|AM|first|morning|breakfast| +28756|AAAAAAAAFFAHAAAA|28756|7|59|16|AM|first|morning|breakfast| +28757|AAAAAAAAGFAHAAAA|28757|7|59|17|AM|first|morning|breakfast| +28758|AAAAAAAAHFAHAAAA|28758|7|59|18|AM|first|morning|breakfast| +28759|AAAAAAAAIFAHAAAA|28759|7|59|19|AM|first|morning|breakfast| +28760|AAAAAAAAJFAHAAAA|28760|7|59|20|AM|first|morning|breakfast| +28761|AAAAAAAAKFAHAAAA|28761|7|59|21|AM|first|morning|breakfast| +28762|AAAAAAAALFAHAAAA|28762|7|59|22|AM|first|morning|breakfast| +28763|AAAAAAAAMFAHAAAA|28763|7|59|23|AM|first|morning|breakfast| +28764|AAAAAAAANFAHAAAA|28764|7|59|24|AM|first|morning|breakfast| +28765|AAAAAAAAOFAHAAAA|28765|7|59|25|AM|first|morning|breakfast| +28766|AAAAAAAAPFAHAAAA|28766|7|59|26|AM|first|morning|breakfast| +28767|AAAAAAAAAGAHAAAA|28767|7|59|27|AM|first|morning|breakfast| +28768|AAAAAAAABGAHAAAA|28768|7|59|28|AM|first|morning|breakfast| +28769|AAAAAAAACGAHAAAA|28769|7|59|29|AM|first|morning|breakfast| +28770|AAAAAAAADGAHAAAA|28770|7|59|30|AM|first|morning|breakfast| +28771|AAAAAAAAEGAHAAAA|28771|7|59|31|AM|first|morning|breakfast| +28772|AAAAAAAAFGAHAAAA|28772|7|59|32|AM|first|morning|breakfast| +28773|AAAAAAAAGGAHAAAA|28773|7|59|33|AM|first|morning|breakfast| +28774|AAAAAAAAHGAHAAAA|28774|7|59|34|AM|first|morning|breakfast| +28775|AAAAAAAAIGAHAAAA|28775|7|59|35|AM|first|morning|breakfast| +28776|AAAAAAAAJGAHAAAA|28776|7|59|36|AM|first|morning|breakfast| +28777|AAAAAAAAKGAHAAAA|28777|7|59|37|AM|first|morning|breakfast| +28778|AAAAAAAALGAHAAAA|28778|7|59|38|AM|first|morning|breakfast| +28779|AAAAAAAAMGAHAAAA|28779|7|59|39|AM|first|morning|breakfast| +28780|AAAAAAAANGAHAAAA|28780|7|59|40|AM|first|morning|breakfast| +28781|AAAAAAAAOGAHAAAA|28781|7|59|41|AM|first|morning|breakfast| +28782|AAAAAAAAPGAHAAAA|28782|7|59|42|AM|first|morning|breakfast| +28783|AAAAAAAAAHAHAAAA|28783|7|59|43|AM|first|morning|breakfast| +28784|AAAAAAAABHAHAAAA|28784|7|59|44|AM|first|morning|breakfast| +28785|AAAAAAAACHAHAAAA|28785|7|59|45|AM|first|morning|breakfast| +28786|AAAAAAAADHAHAAAA|28786|7|59|46|AM|first|morning|breakfast| +28787|AAAAAAAAEHAHAAAA|28787|7|59|47|AM|first|morning|breakfast| +28788|AAAAAAAAFHAHAAAA|28788|7|59|48|AM|first|morning|breakfast| +28789|AAAAAAAAGHAHAAAA|28789|7|59|49|AM|first|morning|breakfast| +28790|AAAAAAAAHHAHAAAA|28790|7|59|50|AM|first|morning|breakfast| +28791|AAAAAAAAIHAHAAAA|28791|7|59|51|AM|first|morning|breakfast| +28792|AAAAAAAAJHAHAAAA|28792|7|59|52|AM|first|morning|breakfast| +28793|AAAAAAAAKHAHAAAA|28793|7|59|53|AM|first|morning|breakfast| +28794|AAAAAAAALHAHAAAA|28794|7|59|54|AM|first|morning|breakfast| +28795|AAAAAAAAMHAHAAAA|28795|7|59|55|AM|first|morning|breakfast| +28796|AAAAAAAANHAHAAAA|28796|7|59|56|AM|first|morning|breakfast| +28797|AAAAAAAAOHAHAAAA|28797|7|59|57|AM|first|morning|breakfast| +28798|AAAAAAAAPHAHAAAA|28798|7|59|58|AM|first|morning|breakfast| +28799|AAAAAAAAAIAHAAAA|28799|7|59|59|AM|first|morning|breakfast| +28800|AAAAAAAABIAHAAAA|28800|8|0|0|AM|first|morning|breakfast| +28801|AAAAAAAACIAHAAAA|28801|8|0|1|AM|first|morning|breakfast| +28802|AAAAAAAADIAHAAAA|28802|8|0|2|AM|first|morning|breakfast| +28803|AAAAAAAAEIAHAAAA|28803|8|0|3|AM|first|morning|breakfast| +28804|AAAAAAAAFIAHAAAA|28804|8|0|4|AM|first|morning|breakfast| +28805|AAAAAAAAGIAHAAAA|28805|8|0|5|AM|first|morning|breakfast| +28806|AAAAAAAAHIAHAAAA|28806|8|0|6|AM|first|morning|breakfast| +28807|AAAAAAAAIIAHAAAA|28807|8|0|7|AM|first|morning|breakfast| +28808|AAAAAAAAJIAHAAAA|28808|8|0|8|AM|first|morning|breakfast| +28809|AAAAAAAAKIAHAAAA|28809|8|0|9|AM|first|morning|breakfast| +28810|AAAAAAAALIAHAAAA|28810|8|0|10|AM|first|morning|breakfast| +28811|AAAAAAAAMIAHAAAA|28811|8|0|11|AM|first|morning|breakfast| +28812|AAAAAAAANIAHAAAA|28812|8|0|12|AM|first|morning|breakfast| +28813|AAAAAAAAOIAHAAAA|28813|8|0|13|AM|first|morning|breakfast| +28814|AAAAAAAAPIAHAAAA|28814|8|0|14|AM|first|morning|breakfast| +28815|AAAAAAAAAJAHAAAA|28815|8|0|15|AM|first|morning|breakfast| +28816|AAAAAAAABJAHAAAA|28816|8|0|16|AM|first|morning|breakfast| +28817|AAAAAAAACJAHAAAA|28817|8|0|17|AM|first|morning|breakfast| +28818|AAAAAAAADJAHAAAA|28818|8|0|18|AM|first|morning|breakfast| +28819|AAAAAAAAEJAHAAAA|28819|8|0|19|AM|first|morning|breakfast| +28820|AAAAAAAAFJAHAAAA|28820|8|0|20|AM|first|morning|breakfast| +28821|AAAAAAAAGJAHAAAA|28821|8|0|21|AM|first|morning|breakfast| +28822|AAAAAAAAHJAHAAAA|28822|8|0|22|AM|first|morning|breakfast| +28823|AAAAAAAAIJAHAAAA|28823|8|0|23|AM|first|morning|breakfast| +28824|AAAAAAAAJJAHAAAA|28824|8|0|24|AM|first|morning|breakfast| +28825|AAAAAAAAKJAHAAAA|28825|8|0|25|AM|first|morning|breakfast| +28826|AAAAAAAALJAHAAAA|28826|8|0|26|AM|first|morning|breakfast| +28827|AAAAAAAAMJAHAAAA|28827|8|0|27|AM|first|morning|breakfast| +28828|AAAAAAAANJAHAAAA|28828|8|0|28|AM|first|morning|breakfast| +28829|AAAAAAAAOJAHAAAA|28829|8|0|29|AM|first|morning|breakfast| +28830|AAAAAAAAPJAHAAAA|28830|8|0|30|AM|first|morning|breakfast| +28831|AAAAAAAAAKAHAAAA|28831|8|0|31|AM|first|morning|breakfast| +28832|AAAAAAAABKAHAAAA|28832|8|0|32|AM|first|morning|breakfast| +28833|AAAAAAAACKAHAAAA|28833|8|0|33|AM|first|morning|breakfast| +28834|AAAAAAAADKAHAAAA|28834|8|0|34|AM|first|morning|breakfast| +28835|AAAAAAAAEKAHAAAA|28835|8|0|35|AM|first|morning|breakfast| +28836|AAAAAAAAFKAHAAAA|28836|8|0|36|AM|first|morning|breakfast| +28837|AAAAAAAAGKAHAAAA|28837|8|0|37|AM|first|morning|breakfast| +28838|AAAAAAAAHKAHAAAA|28838|8|0|38|AM|first|morning|breakfast| +28839|AAAAAAAAIKAHAAAA|28839|8|0|39|AM|first|morning|breakfast| +28840|AAAAAAAAJKAHAAAA|28840|8|0|40|AM|first|morning|breakfast| +28841|AAAAAAAAKKAHAAAA|28841|8|0|41|AM|first|morning|breakfast| +28842|AAAAAAAALKAHAAAA|28842|8|0|42|AM|first|morning|breakfast| +28843|AAAAAAAAMKAHAAAA|28843|8|0|43|AM|first|morning|breakfast| +28844|AAAAAAAANKAHAAAA|28844|8|0|44|AM|first|morning|breakfast| +28845|AAAAAAAAOKAHAAAA|28845|8|0|45|AM|first|morning|breakfast| +28846|AAAAAAAAPKAHAAAA|28846|8|0|46|AM|first|morning|breakfast| +28847|AAAAAAAAALAHAAAA|28847|8|0|47|AM|first|morning|breakfast| +28848|AAAAAAAABLAHAAAA|28848|8|0|48|AM|first|morning|breakfast| +28849|AAAAAAAACLAHAAAA|28849|8|0|49|AM|first|morning|breakfast| +28850|AAAAAAAADLAHAAAA|28850|8|0|50|AM|first|morning|breakfast| +28851|AAAAAAAAELAHAAAA|28851|8|0|51|AM|first|morning|breakfast| +28852|AAAAAAAAFLAHAAAA|28852|8|0|52|AM|first|morning|breakfast| +28853|AAAAAAAAGLAHAAAA|28853|8|0|53|AM|first|morning|breakfast| +28854|AAAAAAAAHLAHAAAA|28854|8|0|54|AM|first|morning|breakfast| +28855|AAAAAAAAILAHAAAA|28855|8|0|55|AM|first|morning|breakfast| +28856|AAAAAAAAJLAHAAAA|28856|8|0|56|AM|first|morning|breakfast| +28857|AAAAAAAAKLAHAAAA|28857|8|0|57|AM|first|morning|breakfast| +28858|AAAAAAAALLAHAAAA|28858|8|0|58|AM|first|morning|breakfast| +28859|AAAAAAAAMLAHAAAA|28859|8|0|59|AM|first|morning|breakfast| +28860|AAAAAAAANLAHAAAA|28860|8|1|0|AM|first|morning|breakfast| +28861|AAAAAAAAOLAHAAAA|28861|8|1|1|AM|first|morning|breakfast| +28862|AAAAAAAAPLAHAAAA|28862|8|1|2|AM|first|morning|breakfast| +28863|AAAAAAAAAMAHAAAA|28863|8|1|3|AM|first|morning|breakfast| +28864|AAAAAAAABMAHAAAA|28864|8|1|4|AM|first|morning|breakfast| +28865|AAAAAAAACMAHAAAA|28865|8|1|5|AM|first|morning|breakfast| +28866|AAAAAAAADMAHAAAA|28866|8|1|6|AM|first|morning|breakfast| +28867|AAAAAAAAEMAHAAAA|28867|8|1|7|AM|first|morning|breakfast| +28868|AAAAAAAAFMAHAAAA|28868|8|1|8|AM|first|morning|breakfast| +28869|AAAAAAAAGMAHAAAA|28869|8|1|9|AM|first|morning|breakfast| +28870|AAAAAAAAHMAHAAAA|28870|8|1|10|AM|first|morning|breakfast| +28871|AAAAAAAAIMAHAAAA|28871|8|1|11|AM|first|morning|breakfast| +28872|AAAAAAAAJMAHAAAA|28872|8|1|12|AM|first|morning|breakfast| +28873|AAAAAAAAKMAHAAAA|28873|8|1|13|AM|first|morning|breakfast| +28874|AAAAAAAALMAHAAAA|28874|8|1|14|AM|first|morning|breakfast| +28875|AAAAAAAAMMAHAAAA|28875|8|1|15|AM|first|morning|breakfast| +28876|AAAAAAAANMAHAAAA|28876|8|1|16|AM|first|morning|breakfast| +28877|AAAAAAAAOMAHAAAA|28877|8|1|17|AM|first|morning|breakfast| +28878|AAAAAAAAPMAHAAAA|28878|8|1|18|AM|first|morning|breakfast| +28879|AAAAAAAAANAHAAAA|28879|8|1|19|AM|first|morning|breakfast| +28880|AAAAAAAABNAHAAAA|28880|8|1|20|AM|first|morning|breakfast| +28881|AAAAAAAACNAHAAAA|28881|8|1|21|AM|first|morning|breakfast| +28882|AAAAAAAADNAHAAAA|28882|8|1|22|AM|first|morning|breakfast| +28883|AAAAAAAAENAHAAAA|28883|8|1|23|AM|first|morning|breakfast| +28884|AAAAAAAAFNAHAAAA|28884|8|1|24|AM|first|morning|breakfast| +28885|AAAAAAAAGNAHAAAA|28885|8|1|25|AM|first|morning|breakfast| +28886|AAAAAAAAHNAHAAAA|28886|8|1|26|AM|first|morning|breakfast| +28887|AAAAAAAAINAHAAAA|28887|8|1|27|AM|first|morning|breakfast| +28888|AAAAAAAAJNAHAAAA|28888|8|1|28|AM|first|morning|breakfast| +28889|AAAAAAAAKNAHAAAA|28889|8|1|29|AM|first|morning|breakfast| +28890|AAAAAAAALNAHAAAA|28890|8|1|30|AM|first|morning|breakfast| +28891|AAAAAAAAMNAHAAAA|28891|8|1|31|AM|first|morning|breakfast| +28892|AAAAAAAANNAHAAAA|28892|8|1|32|AM|first|morning|breakfast| +28893|AAAAAAAAONAHAAAA|28893|8|1|33|AM|first|morning|breakfast| +28894|AAAAAAAAPNAHAAAA|28894|8|1|34|AM|first|morning|breakfast| +28895|AAAAAAAAAOAHAAAA|28895|8|1|35|AM|first|morning|breakfast| +28896|AAAAAAAABOAHAAAA|28896|8|1|36|AM|first|morning|breakfast| +28897|AAAAAAAACOAHAAAA|28897|8|1|37|AM|first|morning|breakfast| +28898|AAAAAAAADOAHAAAA|28898|8|1|38|AM|first|morning|breakfast| +28899|AAAAAAAAEOAHAAAA|28899|8|1|39|AM|first|morning|breakfast| +28900|AAAAAAAAFOAHAAAA|28900|8|1|40|AM|first|morning|breakfast| +28901|AAAAAAAAGOAHAAAA|28901|8|1|41|AM|first|morning|breakfast| +28902|AAAAAAAAHOAHAAAA|28902|8|1|42|AM|first|morning|breakfast| +28903|AAAAAAAAIOAHAAAA|28903|8|1|43|AM|first|morning|breakfast| +28904|AAAAAAAAJOAHAAAA|28904|8|1|44|AM|first|morning|breakfast| +28905|AAAAAAAAKOAHAAAA|28905|8|1|45|AM|first|morning|breakfast| +28906|AAAAAAAALOAHAAAA|28906|8|1|46|AM|first|morning|breakfast| +28907|AAAAAAAAMOAHAAAA|28907|8|1|47|AM|first|morning|breakfast| +28908|AAAAAAAANOAHAAAA|28908|8|1|48|AM|first|morning|breakfast| +28909|AAAAAAAAOOAHAAAA|28909|8|1|49|AM|first|morning|breakfast| +28910|AAAAAAAAPOAHAAAA|28910|8|1|50|AM|first|morning|breakfast| +28911|AAAAAAAAAPAHAAAA|28911|8|1|51|AM|first|morning|breakfast| +28912|AAAAAAAABPAHAAAA|28912|8|1|52|AM|first|morning|breakfast| +28913|AAAAAAAACPAHAAAA|28913|8|1|53|AM|first|morning|breakfast| +28914|AAAAAAAADPAHAAAA|28914|8|1|54|AM|first|morning|breakfast| +28915|AAAAAAAAEPAHAAAA|28915|8|1|55|AM|first|morning|breakfast| +28916|AAAAAAAAFPAHAAAA|28916|8|1|56|AM|first|morning|breakfast| +28917|AAAAAAAAGPAHAAAA|28917|8|1|57|AM|first|morning|breakfast| +28918|AAAAAAAAHPAHAAAA|28918|8|1|58|AM|first|morning|breakfast| +28919|AAAAAAAAIPAHAAAA|28919|8|1|59|AM|first|morning|breakfast| +28920|AAAAAAAAJPAHAAAA|28920|8|2|0|AM|first|morning|breakfast| +28921|AAAAAAAAKPAHAAAA|28921|8|2|1|AM|first|morning|breakfast| +28922|AAAAAAAALPAHAAAA|28922|8|2|2|AM|first|morning|breakfast| +28923|AAAAAAAAMPAHAAAA|28923|8|2|3|AM|first|morning|breakfast| +28924|AAAAAAAANPAHAAAA|28924|8|2|4|AM|first|morning|breakfast| +28925|AAAAAAAAOPAHAAAA|28925|8|2|5|AM|first|morning|breakfast| +28926|AAAAAAAAPPAHAAAA|28926|8|2|6|AM|first|morning|breakfast| +28927|AAAAAAAAAABHAAAA|28927|8|2|7|AM|first|morning|breakfast| +28928|AAAAAAAABABHAAAA|28928|8|2|8|AM|first|morning|breakfast| +28929|AAAAAAAACABHAAAA|28929|8|2|9|AM|first|morning|breakfast| +28930|AAAAAAAADABHAAAA|28930|8|2|10|AM|first|morning|breakfast| +28931|AAAAAAAAEABHAAAA|28931|8|2|11|AM|first|morning|breakfast| +28932|AAAAAAAAFABHAAAA|28932|8|2|12|AM|first|morning|breakfast| +28933|AAAAAAAAGABHAAAA|28933|8|2|13|AM|first|morning|breakfast| +28934|AAAAAAAAHABHAAAA|28934|8|2|14|AM|first|morning|breakfast| +28935|AAAAAAAAIABHAAAA|28935|8|2|15|AM|first|morning|breakfast| +28936|AAAAAAAAJABHAAAA|28936|8|2|16|AM|first|morning|breakfast| +28937|AAAAAAAAKABHAAAA|28937|8|2|17|AM|first|morning|breakfast| +28938|AAAAAAAALABHAAAA|28938|8|2|18|AM|first|morning|breakfast| +28939|AAAAAAAAMABHAAAA|28939|8|2|19|AM|first|morning|breakfast| +28940|AAAAAAAANABHAAAA|28940|8|2|20|AM|first|morning|breakfast| +28941|AAAAAAAAOABHAAAA|28941|8|2|21|AM|first|morning|breakfast| +28942|AAAAAAAAPABHAAAA|28942|8|2|22|AM|first|morning|breakfast| +28943|AAAAAAAAABBHAAAA|28943|8|2|23|AM|first|morning|breakfast| +28944|AAAAAAAABBBHAAAA|28944|8|2|24|AM|first|morning|breakfast| +28945|AAAAAAAACBBHAAAA|28945|8|2|25|AM|first|morning|breakfast| +28946|AAAAAAAADBBHAAAA|28946|8|2|26|AM|first|morning|breakfast| +28947|AAAAAAAAEBBHAAAA|28947|8|2|27|AM|first|morning|breakfast| +28948|AAAAAAAAFBBHAAAA|28948|8|2|28|AM|first|morning|breakfast| +28949|AAAAAAAAGBBHAAAA|28949|8|2|29|AM|first|morning|breakfast| +28950|AAAAAAAAHBBHAAAA|28950|8|2|30|AM|first|morning|breakfast| +28951|AAAAAAAAIBBHAAAA|28951|8|2|31|AM|first|morning|breakfast| +28952|AAAAAAAAJBBHAAAA|28952|8|2|32|AM|first|morning|breakfast| +28953|AAAAAAAAKBBHAAAA|28953|8|2|33|AM|first|morning|breakfast| +28954|AAAAAAAALBBHAAAA|28954|8|2|34|AM|first|morning|breakfast| +28955|AAAAAAAAMBBHAAAA|28955|8|2|35|AM|first|morning|breakfast| +28956|AAAAAAAANBBHAAAA|28956|8|2|36|AM|first|morning|breakfast| +28957|AAAAAAAAOBBHAAAA|28957|8|2|37|AM|first|morning|breakfast| +28958|AAAAAAAAPBBHAAAA|28958|8|2|38|AM|first|morning|breakfast| +28959|AAAAAAAAACBHAAAA|28959|8|2|39|AM|first|morning|breakfast| +28960|AAAAAAAABCBHAAAA|28960|8|2|40|AM|first|morning|breakfast| +28961|AAAAAAAACCBHAAAA|28961|8|2|41|AM|first|morning|breakfast| +28962|AAAAAAAADCBHAAAA|28962|8|2|42|AM|first|morning|breakfast| +28963|AAAAAAAAECBHAAAA|28963|8|2|43|AM|first|morning|breakfast| +28964|AAAAAAAAFCBHAAAA|28964|8|2|44|AM|first|morning|breakfast| +28965|AAAAAAAAGCBHAAAA|28965|8|2|45|AM|first|morning|breakfast| +28966|AAAAAAAAHCBHAAAA|28966|8|2|46|AM|first|morning|breakfast| +28967|AAAAAAAAICBHAAAA|28967|8|2|47|AM|first|morning|breakfast| +28968|AAAAAAAAJCBHAAAA|28968|8|2|48|AM|first|morning|breakfast| +28969|AAAAAAAAKCBHAAAA|28969|8|2|49|AM|first|morning|breakfast| +28970|AAAAAAAALCBHAAAA|28970|8|2|50|AM|first|morning|breakfast| +28971|AAAAAAAAMCBHAAAA|28971|8|2|51|AM|first|morning|breakfast| +28972|AAAAAAAANCBHAAAA|28972|8|2|52|AM|first|morning|breakfast| +28973|AAAAAAAAOCBHAAAA|28973|8|2|53|AM|first|morning|breakfast| +28974|AAAAAAAAPCBHAAAA|28974|8|2|54|AM|first|morning|breakfast| +28975|AAAAAAAAADBHAAAA|28975|8|2|55|AM|first|morning|breakfast| +28976|AAAAAAAABDBHAAAA|28976|8|2|56|AM|first|morning|breakfast| +28977|AAAAAAAACDBHAAAA|28977|8|2|57|AM|first|morning|breakfast| +28978|AAAAAAAADDBHAAAA|28978|8|2|58|AM|first|morning|breakfast| +28979|AAAAAAAAEDBHAAAA|28979|8|2|59|AM|first|morning|breakfast| +28980|AAAAAAAAFDBHAAAA|28980|8|3|0|AM|first|morning|breakfast| +28981|AAAAAAAAGDBHAAAA|28981|8|3|1|AM|first|morning|breakfast| +28982|AAAAAAAAHDBHAAAA|28982|8|3|2|AM|first|morning|breakfast| +28983|AAAAAAAAIDBHAAAA|28983|8|3|3|AM|first|morning|breakfast| +28984|AAAAAAAAJDBHAAAA|28984|8|3|4|AM|first|morning|breakfast| +28985|AAAAAAAAKDBHAAAA|28985|8|3|5|AM|first|morning|breakfast| +28986|AAAAAAAALDBHAAAA|28986|8|3|6|AM|first|morning|breakfast| +28987|AAAAAAAAMDBHAAAA|28987|8|3|7|AM|first|morning|breakfast| +28988|AAAAAAAANDBHAAAA|28988|8|3|8|AM|first|morning|breakfast| +28989|AAAAAAAAODBHAAAA|28989|8|3|9|AM|first|morning|breakfast| +28990|AAAAAAAAPDBHAAAA|28990|8|3|10|AM|first|morning|breakfast| +28991|AAAAAAAAAEBHAAAA|28991|8|3|11|AM|first|morning|breakfast| +28992|AAAAAAAABEBHAAAA|28992|8|3|12|AM|first|morning|breakfast| +28993|AAAAAAAACEBHAAAA|28993|8|3|13|AM|first|morning|breakfast| +28994|AAAAAAAADEBHAAAA|28994|8|3|14|AM|first|morning|breakfast| +28995|AAAAAAAAEEBHAAAA|28995|8|3|15|AM|first|morning|breakfast| +28996|AAAAAAAAFEBHAAAA|28996|8|3|16|AM|first|morning|breakfast| +28997|AAAAAAAAGEBHAAAA|28997|8|3|17|AM|first|morning|breakfast| +28998|AAAAAAAAHEBHAAAA|28998|8|3|18|AM|first|morning|breakfast| +28999|AAAAAAAAIEBHAAAA|28999|8|3|19|AM|first|morning|breakfast| +29000|AAAAAAAAJEBHAAAA|29000|8|3|20|AM|first|morning|breakfast| +29001|AAAAAAAAKEBHAAAA|29001|8|3|21|AM|first|morning|breakfast| +29002|AAAAAAAALEBHAAAA|29002|8|3|22|AM|first|morning|breakfast| +29003|AAAAAAAAMEBHAAAA|29003|8|3|23|AM|first|morning|breakfast| +29004|AAAAAAAANEBHAAAA|29004|8|3|24|AM|first|morning|breakfast| +29005|AAAAAAAAOEBHAAAA|29005|8|3|25|AM|first|morning|breakfast| +29006|AAAAAAAAPEBHAAAA|29006|8|3|26|AM|first|morning|breakfast| +29007|AAAAAAAAAFBHAAAA|29007|8|3|27|AM|first|morning|breakfast| +29008|AAAAAAAABFBHAAAA|29008|8|3|28|AM|first|morning|breakfast| +29009|AAAAAAAACFBHAAAA|29009|8|3|29|AM|first|morning|breakfast| +29010|AAAAAAAADFBHAAAA|29010|8|3|30|AM|first|morning|breakfast| +29011|AAAAAAAAEFBHAAAA|29011|8|3|31|AM|first|morning|breakfast| +29012|AAAAAAAAFFBHAAAA|29012|8|3|32|AM|first|morning|breakfast| +29013|AAAAAAAAGFBHAAAA|29013|8|3|33|AM|first|morning|breakfast| +29014|AAAAAAAAHFBHAAAA|29014|8|3|34|AM|first|morning|breakfast| +29015|AAAAAAAAIFBHAAAA|29015|8|3|35|AM|first|morning|breakfast| +29016|AAAAAAAAJFBHAAAA|29016|8|3|36|AM|first|morning|breakfast| +29017|AAAAAAAAKFBHAAAA|29017|8|3|37|AM|first|morning|breakfast| +29018|AAAAAAAALFBHAAAA|29018|8|3|38|AM|first|morning|breakfast| +29019|AAAAAAAAMFBHAAAA|29019|8|3|39|AM|first|morning|breakfast| +29020|AAAAAAAANFBHAAAA|29020|8|3|40|AM|first|morning|breakfast| +29021|AAAAAAAAOFBHAAAA|29021|8|3|41|AM|first|morning|breakfast| +29022|AAAAAAAAPFBHAAAA|29022|8|3|42|AM|first|morning|breakfast| +29023|AAAAAAAAAGBHAAAA|29023|8|3|43|AM|first|morning|breakfast| +29024|AAAAAAAABGBHAAAA|29024|8|3|44|AM|first|morning|breakfast| +29025|AAAAAAAACGBHAAAA|29025|8|3|45|AM|first|morning|breakfast| +29026|AAAAAAAADGBHAAAA|29026|8|3|46|AM|first|morning|breakfast| +29027|AAAAAAAAEGBHAAAA|29027|8|3|47|AM|first|morning|breakfast| +29028|AAAAAAAAFGBHAAAA|29028|8|3|48|AM|first|morning|breakfast| +29029|AAAAAAAAGGBHAAAA|29029|8|3|49|AM|first|morning|breakfast| +29030|AAAAAAAAHGBHAAAA|29030|8|3|50|AM|first|morning|breakfast| +29031|AAAAAAAAIGBHAAAA|29031|8|3|51|AM|first|morning|breakfast| +29032|AAAAAAAAJGBHAAAA|29032|8|3|52|AM|first|morning|breakfast| +29033|AAAAAAAAKGBHAAAA|29033|8|3|53|AM|first|morning|breakfast| +29034|AAAAAAAALGBHAAAA|29034|8|3|54|AM|first|morning|breakfast| +29035|AAAAAAAAMGBHAAAA|29035|8|3|55|AM|first|morning|breakfast| +29036|AAAAAAAANGBHAAAA|29036|8|3|56|AM|first|morning|breakfast| +29037|AAAAAAAAOGBHAAAA|29037|8|3|57|AM|first|morning|breakfast| +29038|AAAAAAAAPGBHAAAA|29038|8|3|58|AM|first|morning|breakfast| +29039|AAAAAAAAAHBHAAAA|29039|8|3|59|AM|first|morning|breakfast| +29040|AAAAAAAABHBHAAAA|29040|8|4|0|AM|first|morning|breakfast| +29041|AAAAAAAACHBHAAAA|29041|8|4|1|AM|first|morning|breakfast| +29042|AAAAAAAADHBHAAAA|29042|8|4|2|AM|first|morning|breakfast| +29043|AAAAAAAAEHBHAAAA|29043|8|4|3|AM|first|morning|breakfast| +29044|AAAAAAAAFHBHAAAA|29044|8|4|4|AM|first|morning|breakfast| +29045|AAAAAAAAGHBHAAAA|29045|8|4|5|AM|first|morning|breakfast| +29046|AAAAAAAAHHBHAAAA|29046|8|4|6|AM|first|morning|breakfast| +29047|AAAAAAAAIHBHAAAA|29047|8|4|7|AM|first|morning|breakfast| +29048|AAAAAAAAJHBHAAAA|29048|8|4|8|AM|first|morning|breakfast| +29049|AAAAAAAAKHBHAAAA|29049|8|4|9|AM|first|morning|breakfast| +29050|AAAAAAAALHBHAAAA|29050|8|4|10|AM|first|morning|breakfast| +29051|AAAAAAAAMHBHAAAA|29051|8|4|11|AM|first|morning|breakfast| +29052|AAAAAAAANHBHAAAA|29052|8|4|12|AM|first|morning|breakfast| +29053|AAAAAAAAOHBHAAAA|29053|8|4|13|AM|first|morning|breakfast| +29054|AAAAAAAAPHBHAAAA|29054|8|4|14|AM|first|morning|breakfast| +29055|AAAAAAAAAIBHAAAA|29055|8|4|15|AM|first|morning|breakfast| +29056|AAAAAAAABIBHAAAA|29056|8|4|16|AM|first|morning|breakfast| +29057|AAAAAAAACIBHAAAA|29057|8|4|17|AM|first|morning|breakfast| +29058|AAAAAAAADIBHAAAA|29058|8|4|18|AM|first|morning|breakfast| +29059|AAAAAAAAEIBHAAAA|29059|8|4|19|AM|first|morning|breakfast| +29060|AAAAAAAAFIBHAAAA|29060|8|4|20|AM|first|morning|breakfast| +29061|AAAAAAAAGIBHAAAA|29061|8|4|21|AM|first|morning|breakfast| +29062|AAAAAAAAHIBHAAAA|29062|8|4|22|AM|first|morning|breakfast| +29063|AAAAAAAAIIBHAAAA|29063|8|4|23|AM|first|morning|breakfast| +29064|AAAAAAAAJIBHAAAA|29064|8|4|24|AM|first|morning|breakfast| +29065|AAAAAAAAKIBHAAAA|29065|8|4|25|AM|first|morning|breakfast| +29066|AAAAAAAALIBHAAAA|29066|8|4|26|AM|first|morning|breakfast| +29067|AAAAAAAAMIBHAAAA|29067|8|4|27|AM|first|morning|breakfast| +29068|AAAAAAAANIBHAAAA|29068|8|4|28|AM|first|morning|breakfast| +29069|AAAAAAAAOIBHAAAA|29069|8|4|29|AM|first|morning|breakfast| +29070|AAAAAAAAPIBHAAAA|29070|8|4|30|AM|first|morning|breakfast| +29071|AAAAAAAAAJBHAAAA|29071|8|4|31|AM|first|morning|breakfast| +29072|AAAAAAAABJBHAAAA|29072|8|4|32|AM|first|morning|breakfast| +29073|AAAAAAAACJBHAAAA|29073|8|4|33|AM|first|morning|breakfast| +29074|AAAAAAAADJBHAAAA|29074|8|4|34|AM|first|morning|breakfast| +29075|AAAAAAAAEJBHAAAA|29075|8|4|35|AM|first|morning|breakfast| +29076|AAAAAAAAFJBHAAAA|29076|8|4|36|AM|first|morning|breakfast| +29077|AAAAAAAAGJBHAAAA|29077|8|4|37|AM|first|morning|breakfast| +29078|AAAAAAAAHJBHAAAA|29078|8|4|38|AM|first|morning|breakfast| +29079|AAAAAAAAIJBHAAAA|29079|8|4|39|AM|first|morning|breakfast| +29080|AAAAAAAAJJBHAAAA|29080|8|4|40|AM|first|morning|breakfast| +29081|AAAAAAAAKJBHAAAA|29081|8|4|41|AM|first|morning|breakfast| +29082|AAAAAAAALJBHAAAA|29082|8|4|42|AM|first|morning|breakfast| +29083|AAAAAAAAMJBHAAAA|29083|8|4|43|AM|first|morning|breakfast| +29084|AAAAAAAANJBHAAAA|29084|8|4|44|AM|first|morning|breakfast| +29085|AAAAAAAAOJBHAAAA|29085|8|4|45|AM|first|morning|breakfast| +29086|AAAAAAAAPJBHAAAA|29086|8|4|46|AM|first|morning|breakfast| +29087|AAAAAAAAAKBHAAAA|29087|8|4|47|AM|first|morning|breakfast| +29088|AAAAAAAABKBHAAAA|29088|8|4|48|AM|first|morning|breakfast| +29089|AAAAAAAACKBHAAAA|29089|8|4|49|AM|first|morning|breakfast| +29090|AAAAAAAADKBHAAAA|29090|8|4|50|AM|first|morning|breakfast| +29091|AAAAAAAAEKBHAAAA|29091|8|4|51|AM|first|morning|breakfast| +29092|AAAAAAAAFKBHAAAA|29092|8|4|52|AM|first|morning|breakfast| +29093|AAAAAAAAGKBHAAAA|29093|8|4|53|AM|first|morning|breakfast| +29094|AAAAAAAAHKBHAAAA|29094|8|4|54|AM|first|morning|breakfast| +29095|AAAAAAAAIKBHAAAA|29095|8|4|55|AM|first|morning|breakfast| +29096|AAAAAAAAJKBHAAAA|29096|8|4|56|AM|first|morning|breakfast| +29097|AAAAAAAAKKBHAAAA|29097|8|4|57|AM|first|morning|breakfast| +29098|AAAAAAAALKBHAAAA|29098|8|4|58|AM|first|morning|breakfast| +29099|AAAAAAAAMKBHAAAA|29099|8|4|59|AM|first|morning|breakfast| +29100|AAAAAAAANKBHAAAA|29100|8|5|0|AM|first|morning|breakfast| +29101|AAAAAAAAOKBHAAAA|29101|8|5|1|AM|first|morning|breakfast| +29102|AAAAAAAAPKBHAAAA|29102|8|5|2|AM|first|morning|breakfast| +29103|AAAAAAAAALBHAAAA|29103|8|5|3|AM|first|morning|breakfast| +29104|AAAAAAAABLBHAAAA|29104|8|5|4|AM|first|morning|breakfast| +29105|AAAAAAAACLBHAAAA|29105|8|5|5|AM|first|morning|breakfast| +29106|AAAAAAAADLBHAAAA|29106|8|5|6|AM|first|morning|breakfast| +29107|AAAAAAAAELBHAAAA|29107|8|5|7|AM|first|morning|breakfast| +29108|AAAAAAAAFLBHAAAA|29108|8|5|8|AM|first|morning|breakfast| +29109|AAAAAAAAGLBHAAAA|29109|8|5|9|AM|first|morning|breakfast| +29110|AAAAAAAAHLBHAAAA|29110|8|5|10|AM|first|morning|breakfast| +29111|AAAAAAAAILBHAAAA|29111|8|5|11|AM|first|morning|breakfast| +29112|AAAAAAAAJLBHAAAA|29112|8|5|12|AM|first|morning|breakfast| +29113|AAAAAAAAKLBHAAAA|29113|8|5|13|AM|first|morning|breakfast| +29114|AAAAAAAALLBHAAAA|29114|8|5|14|AM|first|morning|breakfast| +29115|AAAAAAAAMLBHAAAA|29115|8|5|15|AM|first|morning|breakfast| +29116|AAAAAAAANLBHAAAA|29116|8|5|16|AM|first|morning|breakfast| +29117|AAAAAAAAOLBHAAAA|29117|8|5|17|AM|first|morning|breakfast| +29118|AAAAAAAAPLBHAAAA|29118|8|5|18|AM|first|morning|breakfast| +29119|AAAAAAAAAMBHAAAA|29119|8|5|19|AM|first|morning|breakfast| +29120|AAAAAAAABMBHAAAA|29120|8|5|20|AM|first|morning|breakfast| +29121|AAAAAAAACMBHAAAA|29121|8|5|21|AM|first|morning|breakfast| +29122|AAAAAAAADMBHAAAA|29122|8|5|22|AM|first|morning|breakfast| +29123|AAAAAAAAEMBHAAAA|29123|8|5|23|AM|first|morning|breakfast| +29124|AAAAAAAAFMBHAAAA|29124|8|5|24|AM|first|morning|breakfast| +29125|AAAAAAAAGMBHAAAA|29125|8|5|25|AM|first|morning|breakfast| +29126|AAAAAAAAHMBHAAAA|29126|8|5|26|AM|first|morning|breakfast| +29127|AAAAAAAAIMBHAAAA|29127|8|5|27|AM|first|morning|breakfast| +29128|AAAAAAAAJMBHAAAA|29128|8|5|28|AM|first|morning|breakfast| +29129|AAAAAAAAKMBHAAAA|29129|8|5|29|AM|first|morning|breakfast| +29130|AAAAAAAALMBHAAAA|29130|8|5|30|AM|first|morning|breakfast| +29131|AAAAAAAAMMBHAAAA|29131|8|5|31|AM|first|morning|breakfast| +29132|AAAAAAAANMBHAAAA|29132|8|5|32|AM|first|morning|breakfast| +29133|AAAAAAAAOMBHAAAA|29133|8|5|33|AM|first|morning|breakfast| +29134|AAAAAAAAPMBHAAAA|29134|8|5|34|AM|first|morning|breakfast| +29135|AAAAAAAAANBHAAAA|29135|8|5|35|AM|first|morning|breakfast| +29136|AAAAAAAABNBHAAAA|29136|8|5|36|AM|first|morning|breakfast| +29137|AAAAAAAACNBHAAAA|29137|8|5|37|AM|first|morning|breakfast| +29138|AAAAAAAADNBHAAAA|29138|8|5|38|AM|first|morning|breakfast| +29139|AAAAAAAAENBHAAAA|29139|8|5|39|AM|first|morning|breakfast| +29140|AAAAAAAAFNBHAAAA|29140|8|5|40|AM|first|morning|breakfast| +29141|AAAAAAAAGNBHAAAA|29141|8|5|41|AM|first|morning|breakfast| +29142|AAAAAAAAHNBHAAAA|29142|8|5|42|AM|first|morning|breakfast| +29143|AAAAAAAAINBHAAAA|29143|8|5|43|AM|first|morning|breakfast| +29144|AAAAAAAAJNBHAAAA|29144|8|5|44|AM|first|morning|breakfast| +29145|AAAAAAAAKNBHAAAA|29145|8|5|45|AM|first|morning|breakfast| +29146|AAAAAAAALNBHAAAA|29146|8|5|46|AM|first|morning|breakfast| +29147|AAAAAAAAMNBHAAAA|29147|8|5|47|AM|first|morning|breakfast| +29148|AAAAAAAANNBHAAAA|29148|8|5|48|AM|first|morning|breakfast| +29149|AAAAAAAAONBHAAAA|29149|8|5|49|AM|first|morning|breakfast| +29150|AAAAAAAAPNBHAAAA|29150|8|5|50|AM|first|morning|breakfast| +29151|AAAAAAAAAOBHAAAA|29151|8|5|51|AM|first|morning|breakfast| +29152|AAAAAAAABOBHAAAA|29152|8|5|52|AM|first|morning|breakfast| +29153|AAAAAAAACOBHAAAA|29153|8|5|53|AM|first|morning|breakfast| +29154|AAAAAAAADOBHAAAA|29154|8|5|54|AM|first|morning|breakfast| +29155|AAAAAAAAEOBHAAAA|29155|8|5|55|AM|first|morning|breakfast| +29156|AAAAAAAAFOBHAAAA|29156|8|5|56|AM|first|morning|breakfast| +29157|AAAAAAAAGOBHAAAA|29157|8|5|57|AM|first|morning|breakfast| +29158|AAAAAAAAHOBHAAAA|29158|8|5|58|AM|first|morning|breakfast| +29159|AAAAAAAAIOBHAAAA|29159|8|5|59|AM|first|morning|breakfast| +29160|AAAAAAAAJOBHAAAA|29160|8|6|0|AM|first|morning|breakfast| +29161|AAAAAAAAKOBHAAAA|29161|8|6|1|AM|first|morning|breakfast| +29162|AAAAAAAALOBHAAAA|29162|8|6|2|AM|first|morning|breakfast| +29163|AAAAAAAAMOBHAAAA|29163|8|6|3|AM|first|morning|breakfast| +29164|AAAAAAAANOBHAAAA|29164|8|6|4|AM|first|morning|breakfast| +29165|AAAAAAAAOOBHAAAA|29165|8|6|5|AM|first|morning|breakfast| +29166|AAAAAAAAPOBHAAAA|29166|8|6|6|AM|first|morning|breakfast| +29167|AAAAAAAAAPBHAAAA|29167|8|6|7|AM|first|morning|breakfast| +29168|AAAAAAAABPBHAAAA|29168|8|6|8|AM|first|morning|breakfast| +29169|AAAAAAAACPBHAAAA|29169|8|6|9|AM|first|morning|breakfast| +29170|AAAAAAAADPBHAAAA|29170|8|6|10|AM|first|morning|breakfast| +29171|AAAAAAAAEPBHAAAA|29171|8|6|11|AM|first|morning|breakfast| +29172|AAAAAAAAFPBHAAAA|29172|8|6|12|AM|first|morning|breakfast| +29173|AAAAAAAAGPBHAAAA|29173|8|6|13|AM|first|morning|breakfast| +29174|AAAAAAAAHPBHAAAA|29174|8|6|14|AM|first|morning|breakfast| +29175|AAAAAAAAIPBHAAAA|29175|8|6|15|AM|first|morning|breakfast| +29176|AAAAAAAAJPBHAAAA|29176|8|6|16|AM|first|morning|breakfast| +29177|AAAAAAAAKPBHAAAA|29177|8|6|17|AM|first|morning|breakfast| +29178|AAAAAAAALPBHAAAA|29178|8|6|18|AM|first|morning|breakfast| +29179|AAAAAAAAMPBHAAAA|29179|8|6|19|AM|first|morning|breakfast| +29180|AAAAAAAANPBHAAAA|29180|8|6|20|AM|first|morning|breakfast| +29181|AAAAAAAAOPBHAAAA|29181|8|6|21|AM|first|morning|breakfast| +29182|AAAAAAAAPPBHAAAA|29182|8|6|22|AM|first|morning|breakfast| +29183|AAAAAAAAAACHAAAA|29183|8|6|23|AM|first|morning|breakfast| +29184|AAAAAAAABACHAAAA|29184|8|6|24|AM|first|morning|breakfast| +29185|AAAAAAAACACHAAAA|29185|8|6|25|AM|first|morning|breakfast| +29186|AAAAAAAADACHAAAA|29186|8|6|26|AM|first|morning|breakfast| +29187|AAAAAAAAEACHAAAA|29187|8|6|27|AM|first|morning|breakfast| +29188|AAAAAAAAFACHAAAA|29188|8|6|28|AM|first|morning|breakfast| +29189|AAAAAAAAGACHAAAA|29189|8|6|29|AM|first|morning|breakfast| +29190|AAAAAAAAHACHAAAA|29190|8|6|30|AM|first|morning|breakfast| +29191|AAAAAAAAIACHAAAA|29191|8|6|31|AM|first|morning|breakfast| +29192|AAAAAAAAJACHAAAA|29192|8|6|32|AM|first|morning|breakfast| +29193|AAAAAAAAKACHAAAA|29193|8|6|33|AM|first|morning|breakfast| +29194|AAAAAAAALACHAAAA|29194|8|6|34|AM|first|morning|breakfast| +29195|AAAAAAAAMACHAAAA|29195|8|6|35|AM|first|morning|breakfast| +29196|AAAAAAAANACHAAAA|29196|8|6|36|AM|first|morning|breakfast| +29197|AAAAAAAAOACHAAAA|29197|8|6|37|AM|first|morning|breakfast| +29198|AAAAAAAAPACHAAAA|29198|8|6|38|AM|first|morning|breakfast| +29199|AAAAAAAAABCHAAAA|29199|8|6|39|AM|first|morning|breakfast| +29200|AAAAAAAABBCHAAAA|29200|8|6|40|AM|first|morning|breakfast| +29201|AAAAAAAACBCHAAAA|29201|8|6|41|AM|first|morning|breakfast| +29202|AAAAAAAADBCHAAAA|29202|8|6|42|AM|first|morning|breakfast| +29203|AAAAAAAAEBCHAAAA|29203|8|6|43|AM|first|morning|breakfast| +29204|AAAAAAAAFBCHAAAA|29204|8|6|44|AM|first|morning|breakfast| +29205|AAAAAAAAGBCHAAAA|29205|8|6|45|AM|first|morning|breakfast| +29206|AAAAAAAAHBCHAAAA|29206|8|6|46|AM|first|morning|breakfast| +29207|AAAAAAAAIBCHAAAA|29207|8|6|47|AM|first|morning|breakfast| +29208|AAAAAAAAJBCHAAAA|29208|8|6|48|AM|first|morning|breakfast| +29209|AAAAAAAAKBCHAAAA|29209|8|6|49|AM|first|morning|breakfast| +29210|AAAAAAAALBCHAAAA|29210|8|6|50|AM|first|morning|breakfast| +29211|AAAAAAAAMBCHAAAA|29211|8|6|51|AM|first|morning|breakfast| +29212|AAAAAAAANBCHAAAA|29212|8|6|52|AM|first|morning|breakfast| +29213|AAAAAAAAOBCHAAAA|29213|8|6|53|AM|first|morning|breakfast| +29214|AAAAAAAAPBCHAAAA|29214|8|6|54|AM|first|morning|breakfast| +29215|AAAAAAAAACCHAAAA|29215|8|6|55|AM|first|morning|breakfast| +29216|AAAAAAAABCCHAAAA|29216|8|6|56|AM|first|morning|breakfast| +29217|AAAAAAAACCCHAAAA|29217|8|6|57|AM|first|morning|breakfast| +29218|AAAAAAAADCCHAAAA|29218|8|6|58|AM|first|morning|breakfast| +29219|AAAAAAAAECCHAAAA|29219|8|6|59|AM|first|morning|breakfast| +29220|AAAAAAAAFCCHAAAA|29220|8|7|0|AM|first|morning|breakfast| +29221|AAAAAAAAGCCHAAAA|29221|8|7|1|AM|first|morning|breakfast| +29222|AAAAAAAAHCCHAAAA|29222|8|7|2|AM|first|morning|breakfast| +29223|AAAAAAAAICCHAAAA|29223|8|7|3|AM|first|morning|breakfast| +29224|AAAAAAAAJCCHAAAA|29224|8|7|4|AM|first|morning|breakfast| +29225|AAAAAAAAKCCHAAAA|29225|8|7|5|AM|first|morning|breakfast| +29226|AAAAAAAALCCHAAAA|29226|8|7|6|AM|first|morning|breakfast| +29227|AAAAAAAAMCCHAAAA|29227|8|7|7|AM|first|morning|breakfast| +29228|AAAAAAAANCCHAAAA|29228|8|7|8|AM|first|morning|breakfast| +29229|AAAAAAAAOCCHAAAA|29229|8|7|9|AM|first|morning|breakfast| +29230|AAAAAAAAPCCHAAAA|29230|8|7|10|AM|first|morning|breakfast| +29231|AAAAAAAAADCHAAAA|29231|8|7|11|AM|first|morning|breakfast| +29232|AAAAAAAABDCHAAAA|29232|8|7|12|AM|first|morning|breakfast| +29233|AAAAAAAACDCHAAAA|29233|8|7|13|AM|first|morning|breakfast| +29234|AAAAAAAADDCHAAAA|29234|8|7|14|AM|first|morning|breakfast| +29235|AAAAAAAAEDCHAAAA|29235|8|7|15|AM|first|morning|breakfast| +29236|AAAAAAAAFDCHAAAA|29236|8|7|16|AM|first|morning|breakfast| +29237|AAAAAAAAGDCHAAAA|29237|8|7|17|AM|first|morning|breakfast| +29238|AAAAAAAAHDCHAAAA|29238|8|7|18|AM|first|morning|breakfast| +29239|AAAAAAAAIDCHAAAA|29239|8|7|19|AM|first|morning|breakfast| +29240|AAAAAAAAJDCHAAAA|29240|8|7|20|AM|first|morning|breakfast| +29241|AAAAAAAAKDCHAAAA|29241|8|7|21|AM|first|morning|breakfast| +29242|AAAAAAAALDCHAAAA|29242|8|7|22|AM|first|morning|breakfast| +29243|AAAAAAAAMDCHAAAA|29243|8|7|23|AM|first|morning|breakfast| +29244|AAAAAAAANDCHAAAA|29244|8|7|24|AM|first|morning|breakfast| +29245|AAAAAAAAODCHAAAA|29245|8|7|25|AM|first|morning|breakfast| +29246|AAAAAAAAPDCHAAAA|29246|8|7|26|AM|first|morning|breakfast| +29247|AAAAAAAAAECHAAAA|29247|8|7|27|AM|first|morning|breakfast| +29248|AAAAAAAABECHAAAA|29248|8|7|28|AM|first|morning|breakfast| +29249|AAAAAAAACECHAAAA|29249|8|7|29|AM|first|morning|breakfast| +29250|AAAAAAAADECHAAAA|29250|8|7|30|AM|first|morning|breakfast| +29251|AAAAAAAAEECHAAAA|29251|8|7|31|AM|first|morning|breakfast| +29252|AAAAAAAAFECHAAAA|29252|8|7|32|AM|first|morning|breakfast| +29253|AAAAAAAAGECHAAAA|29253|8|7|33|AM|first|morning|breakfast| +29254|AAAAAAAAHECHAAAA|29254|8|7|34|AM|first|morning|breakfast| +29255|AAAAAAAAIECHAAAA|29255|8|7|35|AM|first|morning|breakfast| +29256|AAAAAAAAJECHAAAA|29256|8|7|36|AM|first|morning|breakfast| +29257|AAAAAAAAKECHAAAA|29257|8|7|37|AM|first|morning|breakfast| +29258|AAAAAAAALECHAAAA|29258|8|7|38|AM|first|morning|breakfast| +29259|AAAAAAAAMECHAAAA|29259|8|7|39|AM|first|morning|breakfast| +29260|AAAAAAAANECHAAAA|29260|8|7|40|AM|first|morning|breakfast| +29261|AAAAAAAAOECHAAAA|29261|8|7|41|AM|first|morning|breakfast| +29262|AAAAAAAAPECHAAAA|29262|8|7|42|AM|first|morning|breakfast| +29263|AAAAAAAAAFCHAAAA|29263|8|7|43|AM|first|morning|breakfast| +29264|AAAAAAAABFCHAAAA|29264|8|7|44|AM|first|morning|breakfast| +29265|AAAAAAAACFCHAAAA|29265|8|7|45|AM|first|morning|breakfast| +29266|AAAAAAAADFCHAAAA|29266|8|7|46|AM|first|morning|breakfast| +29267|AAAAAAAAEFCHAAAA|29267|8|7|47|AM|first|morning|breakfast| +29268|AAAAAAAAFFCHAAAA|29268|8|7|48|AM|first|morning|breakfast| +29269|AAAAAAAAGFCHAAAA|29269|8|7|49|AM|first|morning|breakfast| +29270|AAAAAAAAHFCHAAAA|29270|8|7|50|AM|first|morning|breakfast| +29271|AAAAAAAAIFCHAAAA|29271|8|7|51|AM|first|morning|breakfast| +29272|AAAAAAAAJFCHAAAA|29272|8|7|52|AM|first|morning|breakfast| +29273|AAAAAAAAKFCHAAAA|29273|8|7|53|AM|first|morning|breakfast| +29274|AAAAAAAALFCHAAAA|29274|8|7|54|AM|first|morning|breakfast| +29275|AAAAAAAAMFCHAAAA|29275|8|7|55|AM|first|morning|breakfast| +29276|AAAAAAAANFCHAAAA|29276|8|7|56|AM|first|morning|breakfast| +29277|AAAAAAAAOFCHAAAA|29277|8|7|57|AM|first|morning|breakfast| +29278|AAAAAAAAPFCHAAAA|29278|8|7|58|AM|first|morning|breakfast| +29279|AAAAAAAAAGCHAAAA|29279|8|7|59|AM|first|morning|breakfast| +29280|AAAAAAAABGCHAAAA|29280|8|8|0|AM|first|morning|breakfast| +29281|AAAAAAAACGCHAAAA|29281|8|8|1|AM|first|morning|breakfast| +29282|AAAAAAAADGCHAAAA|29282|8|8|2|AM|first|morning|breakfast| +29283|AAAAAAAAEGCHAAAA|29283|8|8|3|AM|first|morning|breakfast| +29284|AAAAAAAAFGCHAAAA|29284|8|8|4|AM|first|morning|breakfast| +29285|AAAAAAAAGGCHAAAA|29285|8|8|5|AM|first|morning|breakfast| +29286|AAAAAAAAHGCHAAAA|29286|8|8|6|AM|first|morning|breakfast| +29287|AAAAAAAAIGCHAAAA|29287|8|8|7|AM|first|morning|breakfast| +29288|AAAAAAAAJGCHAAAA|29288|8|8|8|AM|first|morning|breakfast| +29289|AAAAAAAAKGCHAAAA|29289|8|8|9|AM|first|morning|breakfast| +29290|AAAAAAAALGCHAAAA|29290|8|8|10|AM|first|morning|breakfast| +29291|AAAAAAAAMGCHAAAA|29291|8|8|11|AM|first|morning|breakfast| +29292|AAAAAAAANGCHAAAA|29292|8|8|12|AM|first|morning|breakfast| +29293|AAAAAAAAOGCHAAAA|29293|8|8|13|AM|first|morning|breakfast| +29294|AAAAAAAAPGCHAAAA|29294|8|8|14|AM|first|morning|breakfast| +29295|AAAAAAAAAHCHAAAA|29295|8|8|15|AM|first|morning|breakfast| +29296|AAAAAAAABHCHAAAA|29296|8|8|16|AM|first|morning|breakfast| +29297|AAAAAAAACHCHAAAA|29297|8|8|17|AM|first|morning|breakfast| +29298|AAAAAAAADHCHAAAA|29298|8|8|18|AM|first|morning|breakfast| +29299|AAAAAAAAEHCHAAAA|29299|8|8|19|AM|first|morning|breakfast| +29300|AAAAAAAAFHCHAAAA|29300|8|8|20|AM|first|morning|breakfast| +29301|AAAAAAAAGHCHAAAA|29301|8|8|21|AM|first|morning|breakfast| +29302|AAAAAAAAHHCHAAAA|29302|8|8|22|AM|first|morning|breakfast| +29303|AAAAAAAAIHCHAAAA|29303|8|8|23|AM|first|morning|breakfast| +29304|AAAAAAAAJHCHAAAA|29304|8|8|24|AM|first|morning|breakfast| +29305|AAAAAAAAKHCHAAAA|29305|8|8|25|AM|first|morning|breakfast| +29306|AAAAAAAALHCHAAAA|29306|8|8|26|AM|first|morning|breakfast| +29307|AAAAAAAAMHCHAAAA|29307|8|8|27|AM|first|morning|breakfast| +29308|AAAAAAAANHCHAAAA|29308|8|8|28|AM|first|morning|breakfast| +29309|AAAAAAAAOHCHAAAA|29309|8|8|29|AM|first|morning|breakfast| +29310|AAAAAAAAPHCHAAAA|29310|8|8|30|AM|first|morning|breakfast| +29311|AAAAAAAAAICHAAAA|29311|8|8|31|AM|first|morning|breakfast| +29312|AAAAAAAABICHAAAA|29312|8|8|32|AM|first|morning|breakfast| +29313|AAAAAAAACICHAAAA|29313|8|8|33|AM|first|morning|breakfast| +29314|AAAAAAAADICHAAAA|29314|8|8|34|AM|first|morning|breakfast| +29315|AAAAAAAAEICHAAAA|29315|8|8|35|AM|first|morning|breakfast| +29316|AAAAAAAAFICHAAAA|29316|8|8|36|AM|first|morning|breakfast| +29317|AAAAAAAAGICHAAAA|29317|8|8|37|AM|first|morning|breakfast| +29318|AAAAAAAAHICHAAAA|29318|8|8|38|AM|first|morning|breakfast| +29319|AAAAAAAAIICHAAAA|29319|8|8|39|AM|first|morning|breakfast| +29320|AAAAAAAAJICHAAAA|29320|8|8|40|AM|first|morning|breakfast| +29321|AAAAAAAAKICHAAAA|29321|8|8|41|AM|first|morning|breakfast| +29322|AAAAAAAALICHAAAA|29322|8|8|42|AM|first|morning|breakfast| +29323|AAAAAAAAMICHAAAA|29323|8|8|43|AM|first|morning|breakfast| +29324|AAAAAAAANICHAAAA|29324|8|8|44|AM|first|morning|breakfast| +29325|AAAAAAAAOICHAAAA|29325|8|8|45|AM|first|morning|breakfast| +29326|AAAAAAAAPICHAAAA|29326|8|8|46|AM|first|morning|breakfast| +29327|AAAAAAAAAJCHAAAA|29327|8|8|47|AM|first|morning|breakfast| +29328|AAAAAAAABJCHAAAA|29328|8|8|48|AM|first|morning|breakfast| +29329|AAAAAAAACJCHAAAA|29329|8|8|49|AM|first|morning|breakfast| +29330|AAAAAAAADJCHAAAA|29330|8|8|50|AM|first|morning|breakfast| +29331|AAAAAAAAEJCHAAAA|29331|8|8|51|AM|first|morning|breakfast| +29332|AAAAAAAAFJCHAAAA|29332|8|8|52|AM|first|morning|breakfast| +29333|AAAAAAAAGJCHAAAA|29333|8|8|53|AM|first|morning|breakfast| +29334|AAAAAAAAHJCHAAAA|29334|8|8|54|AM|first|morning|breakfast| +29335|AAAAAAAAIJCHAAAA|29335|8|8|55|AM|first|morning|breakfast| +29336|AAAAAAAAJJCHAAAA|29336|8|8|56|AM|first|morning|breakfast| +29337|AAAAAAAAKJCHAAAA|29337|8|8|57|AM|first|morning|breakfast| +29338|AAAAAAAALJCHAAAA|29338|8|8|58|AM|first|morning|breakfast| +29339|AAAAAAAAMJCHAAAA|29339|8|8|59|AM|first|morning|breakfast| +29340|AAAAAAAANJCHAAAA|29340|8|9|0|AM|first|morning|breakfast| +29341|AAAAAAAAOJCHAAAA|29341|8|9|1|AM|first|morning|breakfast| +29342|AAAAAAAAPJCHAAAA|29342|8|9|2|AM|first|morning|breakfast| +29343|AAAAAAAAAKCHAAAA|29343|8|9|3|AM|first|morning|breakfast| +29344|AAAAAAAABKCHAAAA|29344|8|9|4|AM|first|morning|breakfast| +29345|AAAAAAAACKCHAAAA|29345|8|9|5|AM|first|morning|breakfast| +29346|AAAAAAAADKCHAAAA|29346|8|9|6|AM|first|morning|breakfast| +29347|AAAAAAAAEKCHAAAA|29347|8|9|7|AM|first|morning|breakfast| +29348|AAAAAAAAFKCHAAAA|29348|8|9|8|AM|first|morning|breakfast| +29349|AAAAAAAAGKCHAAAA|29349|8|9|9|AM|first|morning|breakfast| +29350|AAAAAAAAHKCHAAAA|29350|8|9|10|AM|first|morning|breakfast| +29351|AAAAAAAAIKCHAAAA|29351|8|9|11|AM|first|morning|breakfast| +29352|AAAAAAAAJKCHAAAA|29352|8|9|12|AM|first|morning|breakfast| +29353|AAAAAAAAKKCHAAAA|29353|8|9|13|AM|first|morning|breakfast| +29354|AAAAAAAALKCHAAAA|29354|8|9|14|AM|first|morning|breakfast| +29355|AAAAAAAAMKCHAAAA|29355|8|9|15|AM|first|morning|breakfast| +29356|AAAAAAAANKCHAAAA|29356|8|9|16|AM|first|morning|breakfast| +29357|AAAAAAAAOKCHAAAA|29357|8|9|17|AM|first|morning|breakfast| +29358|AAAAAAAAPKCHAAAA|29358|8|9|18|AM|first|morning|breakfast| +29359|AAAAAAAAALCHAAAA|29359|8|9|19|AM|first|morning|breakfast| +29360|AAAAAAAABLCHAAAA|29360|8|9|20|AM|first|morning|breakfast| +29361|AAAAAAAACLCHAAAA|29361|8|9|21|AM|first|morning|breakfast| +29362|AAAAAAAADLCHAAAA|29362|8|9|22|AM|first|morning|breakfast| +29363|AAAAAAAAELCHAAAA|29363|8|9|23|AM|first|morning|breakfast| +29364|AAAAAAAAFLCHAAAA|29364|8|9|24|AM|first|morning|breakfast| +29365|AAAAAAAAGLCHAAAA|29365|8|9|25|AM|first|morning|breakfast| +29366|AAAAAAAAHLCHAAAA|29366|8|9|26|AM|first|morning|breakfast| +29367|AAAAAAAAILCHAAAA|29367|8|9|27|AM|first|morning|breakfast| +29368|AAAAAAAAJLCHAAAA|29368|8|9|28|AM|first|morning|breakfast| +29369|AAAAAAAAKLCHAAAA|29369|8|9|29|AM|first|morning|breakfast| +29370|AAAAAAAALLCHAAAA|29370|8|9|30|AM|first|morning|breakfast| +29371|AAAAAAAAMLCHAAAA|29371|8|9|31|AM|first|morning|breakfast| +29372|AAAAAAAANLCHAAAA|29372|8|9|32|AM|first|morning|breakfast| +29373|AAAAAAAAOLCHAAAA|29373|8|9|33|AM|first|morning|breakfast| +29374|AAAAAAAAPLCHAAAA|29374|8|9|34|AM|first|morning|breakfast| +29375|AAAAAAAAAMCHAAAA|29375|8|9|35|AM|first|morning|breakfast| +29376|AAAAAAAABMCHAAAA|29376|8|9|36|AM|first|morning|breakfast| +29377|AAAAAAAACMCHAAAA|29377|8|9|37|AM|first|morning|breakfast| +29378|AAAAAAAADMCHAAAA|29378|8|9|38|AM|first|morning|breakfast| +29379|AAAAAAAAEMCHAAAA|29379|8|9|39|AM|first|morning|breakfast| +29380|AAAAAAAAFMCHAAAA|29380|8|9|40|AM|first|morning|breakfast| +29381|AAAAAAAAGMCHAAAA|29381|8|9|41|AM|first|morning|breakfast| +29382|AAAAAAAAHMCHAAAA|29382|8|9|42|AM|first|morning|breakfast| +29383|AAAAAAAAIMCHAAAA|29383|8|9|43|AM|first|morning|breakfast| +29384|AAAAAAAAJMCHAAAA|29384|8|9|44|AM|first|morning|breakfast| +29385|AAAAAAAAKMCHAAAA|29385|8|9|45|AM|first|morning|breakfast| +29386|AAAAAAAALMCHAAAA|29386|8|9|46|AM|first|morning|breakfast| +29387|AAAAAAAAMMCHAAAA|29387|8|9|47|AM|first|morning|breakfast| +29388|AAAAAAAANMCHAAAA|29388|8|9|48|AM|first|morning|breakfast| +29389|AAAAAAAAOMCHAAAA|29389|8|9|49|AM|first|morning|breakfast| +29390|AAAAAAAAPMCHAAAA|29390|8|9|50|AM|first|morning|breakfast| +29391|AAAAAAAAANCHAAAA|29391|8|9|51|AM|first|morning|breakfast| +29392|AAAAAAAABNCHAAAA|29392|8|9|52|AM|first|morning|breakfast| +29393|AAAAAAAACNCHAAAA|29393|8|9|53|AM|first|morning|breakfast| +29394|AAAAAAAADNCHAAAA|29394|8|9|54|AM|first|morning|breakfast| +29395|AAAAAAAAENCHAAAA|29395|8|9|55|AM|first|morning|breakfast| +29396|AAAAAAAAFNCHAAAA|29396|8|9|56|AM|first|morning|breakfast| +29397|AAAAAAAAGNCHAAAA|29397|8|9|57|AM|first|morning|breakfast| +29398|AAAAAAAAHNCHAAAA|29398|8|9|58|AM|first|morning|breakfast| +29399|AAAAAAAAINCHAAAA|29399|8|9|59|AM|first|morning|breakfast| +29400|AAAAAAAAJNCHAAAA|29400|8|10|0|AM|first|morning|breakfast| +29401|AAAAAAAAKNCHAAAA|29401|8|10|1|AM|first|morning|breakfast| +29402|AAAAAAAALNCHAAAA|29402|8|10|2|AM|first|morning|breakfast| +29403|AAAAAAAAMNCHAAAA|29403|8|10|3|AM|first|morning|breakfast| +29404|AAAAAAAANNCHAAAA|29404|8|10|4|AM|first|morning|breakfast| +29405|AAAAAAAAONCHAAAA|29405|8|10|5|AM|first|morning|breakfast| +29406|AAAAAAAAPNCHAAAA|29406|8|10|6|AM|first|morning|breakfast| +29407|AAAAAAAAAOCHAAAA|29407|8|10|7|AM|first|morning|breakfast| +29408|AAAAAAAABOCHAAAA|29408|8|10|8|AM|first|morning|breakfast| +29409|AAAAAAAACOCHAAAA|29409|8|10|9|AM|first|morning|breakfast| +29410|AAAAAAAADOCHAAAA|29410|8|10|10|AM|first|morning|breakfast| +29411|AAAAAAAAEOCHAAAA|29411|8|10|11|AM|first|morning|breakfast| +29412|AAAAAAAAFOCHAAAA|29412|8|10|12|AM|first|morning|breakfast| +29413|AAAAAAAAGOCHAAAA|29413|8|10|13|AM|first|morning|breakfast| +29414|AAAAAAAAHOCHAAAA|29414|8|10|14|AM|first|morning|breakfast| +29415|AAAAAAAAIOCHAAAA|29415|8|10|15|AM|first|morning|breakfast| +29416|AAAAAAAAJOCHAAAA|29416|8|10|16|AM|first|morning|breakfast| +29417|AAAAAAAAKOCHAAAA|29417|8|10|17|AM|first|morning|breakfast| +29418|AAAAAAAALOCHAAAA|29418|8|10|18|AM|first|morning|breakfast| +29419|AAAAAAAAMOCHAAAA|29419|8|10|19|AM|first|morning|breakfast| +29420|AAAAAAAANOCHAAAA|29420|8|10|20|AM|first|morning|breakfast| +29421|AAAAAAAAOOCHAAAA|29421|8|10|21|AM|first|morning|breakfast| +29422|AAAAAAAAPOCHAAAA|29422|8|10|22|AM|first|morning|breakfast| +29423|AAAAAAAAAPCHAAAA|29423|8|10|23|AM|first|morning|breakfast| +29424|AAAAAAAABPCHAAAA|29424|8|10|24|AM|first|morning|breakfast| +29425|AAAAAAAACPCHAAAA|29425|8|10|25|AM|first|morning|breakfast| +29426|AAAAAAAADPCHAAAA|29426|8|10|26|AM|first|morning|breakfast| +29427|AAAAAAAAEPCHAAAA|29427|8|10|27|AM|first|morning|breakfast| +29428|AAAAAAAAFPCHAAAA|29428|8|10|28|AM|first|morning|breakfast| +29429|AAAAAAAAGPCHAAAA|29429|8|10|29|AM|first|morning|breakfast| +29430|AAAAAAAAHPCHAAAA|29430|8|10|30|AM|first|morning|breakfast| +29431|AAAAAAAAIPCHAAAA|29431|8|10|31|AM|first|morning|breakfast| +29432|AAAAAAAAJPCHAAAA|29432|8|10|32|AM|first|morning|breakfast| +29433|AAAAAAAAKPCHAAAA|29433|8|10|33|AM|first|morning|breakfast| +29434|AAAAAAAALPCHAAAA|29434|8|10|34|AM|first|morning|breakfast| +29435|AAAAAAAAMPCHAAAA|29435|8|10|35|AM|first|morning|breakfast| +29436|AAAAAAAANPCHAAAA|29436|8|10|36|AM|first|morning|breakfast| +29437|AAAAAAAAOPCHAAAA|29437|8|10|37|AM|first|morning|breakfast| +29438|AAAAAAAAPPCHAAAA|29438|8|10|38|AM|first|morning|breakfast| +29439|AAAAAAAAAADHAAAA|29439|8|10|39|AM|first|morning|breakfast| +29440|AAAAAAAABADHAAAA|29440|8|10|40|AM|first|morning|breakfast| +29441|AAAAAAAACADHAAAA|29441|8|10|41|AM|first|morning|breakfast| +29442|AAAAAAAADADHAAAA|29442|8|10|42|AM|first|morning|breakfast| +29443|AAAAAAAAEADHAAAA|29443|8|10|43|AM|first|morning|breakfast| +29444|AAAAAAAAFADHAAAA|29444|8|10|44|AM|first|morning|breakfast| +29445|AAAAAAAAGADHAAAA|29445|8|10|45|AM|first|morning|breakfast| +29446|AAAAAAAAHADHAAAA|29446|8|10|46|AM|first|morning|breakfast| +29447|AAAAAAAAIADHAAAA|29447|8|10|47|AM|first|morning|breakfast| +29448|AAAAAAAAJADHAAAA|29448|8|10|48|AM|first|morning|breakfast| +29449|AAAAAAAAKADHAAAA|29449|8|10|49|AM|first|morning|breakfast| +29450|AAAAAAAALADHAAAA|29450|8|10|50|AM|first|morning|breakfast| +29451|AAAAAAAAMADHAAAA|29451|8|10|51|AM|first|morning|breakfast| +29452|AAAAAAAANADHAAAA|29452|8|10|52|AM|first|morning|breakfast| +29453|AAAAAAAAOADHAAAA|29453|8|10|53|AM|first|morning|breakfast| +29454|AAAAAAAAPADHAAAA|29454|8|10|54|AM|first|morning|breakfast| +29455|AAAAAAAAABDHAAAA|29455|8|10|55|AM|first|morning|breakfast| +29456|AAAAAAAABBDHAAAA|29456|8|10|56|AM|first|morning|breakfast| +29457|AAAAAAAACBDHAAAA|29457|8|10|57|AM|first|morning|breakfast| +29458|AAAAAAAADBDHAAAA|29458|8|10|58|AM|first|morning|breakfast| +29459|AAAAAAAAEBDHAAAA|29459|8|10|59|AM|first|morning|breakfast| +29460|AAAAAAAAFBDHAAAA|29460|8|11|0|AM|first|morning|breakfast| +29461|AAAAAAAAGBDHAAAA|29461|8|11|1|AM|first|morning|breakfast| +29462|AAAAAAAAHBDHAAAA|29462|8|11|2|AM|first|morning|breakfast| +29463|AAAAAAAAIBDHAAAA|29463|8|11|3|AM|first|morning|breakfast| +29464|AAAAAAAAJBDHAAAA|29464|8|11|4|AM|first|morning|breakfast| +29465|AAAAAAAAKBDHAAAA|29465|8|11|5|AM|first|morning|breakfast| +29466|AAAAAAAALBDHAAAA|29466|8|11|6|AM|first|morning|breakfast| +29467|AAAAAAAAMBDHAAAA|29467|8|11|7|AM|first|morning|breakfast| +29468|AAAAAAAANBDHAAAA|29468|8|11|8|AM|first|morning|breakfast| +29469|AAAAAAAAOBDHAAAA|29469|8|11|9|AM|first|morning|breakfast| +29470|AAAAAAAAPBDHAAAA|29470|8|11|10|AM|first|morning|breakfast| +29471|AAAAAAAAACDHAAAA|29471|8|11|11|AM|first|morning|breakfast| +29472|AAAAAAAABCDHAAAA|29472|8|11|12|AM|first|morning|breakfast| +29473|AAAAAAAACCDHAAAA|29473|8|11|13|AM|first|morning|breakfast| +29474|AAAAAAAADCDHAAAA|29474|8|11|14|AM|first|morning|breakfast| +29475|AAAAAAAAECDHAAAA|29475|8|11|15|AM|first|morning|breakfast| +29476|AAAAAAAAFCDHAAAA|29476|8|11|16|AM|first|morning|breakfast| +29477|AAAAAAAAGCDHAAAA|29477|8|11|17|AM|first|morning|breakfast| +29478|AAAAAAAAHCDHAAAA|29478|8|11|18|AM|first|morning|breakfast| +29479|AAAAAAAAICDHAAAA|29479|8|11|19|AM|first|morning|breakfast| +29480|AAAAAAAAJCDHAAAA|29480|8|11|20|AM|first|morning|breakfast| +29481|AAAAAAAAKCDHAAAA|29481|8|11|21|AM|first|morning|breakfast| +29482|AAAAAAAALCDHAAAA|29482|8|11|22|AM|first|morning|breakfast| +29483|AAAAAAAAMCDHAAAA|29483|8|11|23|AM|first|morning|breakfast| +29484|AAAAAAAANCDHAAAA|29484|8|11|24|AM|first|morning|breakfast| +29485|AAAAAAAAOCDHAAAA|29485|8|11|25|AM|first|morning|breakfast| +29486|AAAAAAAAPCDHAAAA|29486|8|11|26|AM|first|morning|breakfast| +29487|AAAAAAAAADDHAAAA|29487|8|11|27|AM|first|morning|breakfast| +29488|AAAAAAAABDDHAAAA|29488|8|11|28|AM|first|morning|breakfast| +29489|AAAAAAAACDDHAAAA|29489|8|11|29|AM|first|morning|breakfast| +29490|AAAAAAAADDDHAAAA|29490|8|11|30|AM|first|morning|breakfast| +29491|AAAAAAAAEDDHAAAA|29491|8|11|31|AM|first|morning|breakfast| +29492|AAAAAAAAFDDHAAAA|29492|8|11|32|AM|first|morning|breakfast| +29493|AAAAAAAAGDDHAAAA|29493|8|11|33|AM|first|morning|breakfast| +29494|AAAAAAAAHDDHAAAA|29494|8|11|34|AM|first|morning|breakfast| +29495|AAAAAAAAIDDHAAAA|29495|8|11|35|AM|first|morning|breakfast| +29496|AAAAAAAAJDDHAAAA|29496|8|11|36|AM|first|morning|breakfast| +29497|AAAAAAAAKDDHAAAA|29497|8|11|37|AM|first|morning|breakfast| +29498|AAAAAAAALDDHAAAA|29498|8|11|38|AM|first|morning|breakfast| +29499|AAAAAAAAMDDHAAAA|29499|8|11|39|AM|first|morning|breakfast| +29500|AAAAAAAANDDHAAAA|29500|8|11|40|AM|first|morning|breakfast| +29501|AAAAAAAAODDHAAAA|29501|8|11|41|AM|first|morning|breakfast| +29502|AAAAAAAAPDDHAAAA|29502|8|11|42|AM|first|morning|breakfast| +29503|AAAAAAAAAEDHAAAA|29503|8|11|43|AM|first|morning|breakfast| +29504|AAAAAAAABEDHAAAA|29504|8|11|44|AM|first|morning|breakfast| +29505|AAAAAAAACEDHAAAA|29505|8|11|45|AM|first|morning|breakfast| +29506|AAAAAAAADEDHAAAA|29506|8|11|46|AM|first|morning|breakfast| +29507|AAAAAAAAEEDHAAAA|29507|8|11|47|AM|first|morning|breakfast| +29508|AAAAAAAAFEDHAAAA|29508|8|11|48|AM|first|morning|breakfast| +29509|AAAAAAAAGEDHAAAA|29509|8|11|49|AM|first|morning|breakfast| +29510|AAAAAAAAHEDHAAAA|29510|8|11|50|AM|first|morning|breakfast| +29511|AAAAAAAAIEDHAAAA|29511|8|11|51|AM|first|morning|breakfast| +29512|AAAAAAAAJEDHAAAA|29512|8|11|52|AM|first|morning|breakfast| +29513|AAAAAAAAKEDHAAAA|29513|8|11|53|AM|first|morning|breakfast| +29514|AAAAAAAALEDHAAAA|29514|8|11|54|AM|first|morning|breakfast| +29515|AAAAAAAAMEDHAAAA|29515|8|11|55|AM|first|morning|breakfast| +29516|AAAAAAAANEDHAAAA|29516|8|11|56|AM|first|morning|breakfast| +29517|AAAAAAAAOEDHAAAA|29517|8|11|57|AM|first|morning|breakfast| +29518|AAAAAAAAPEDHAAAA|29518|8|11|58|AM|first|morning|breakfast| +29519|AAAAAAAAAFDHAAAA|29519|8|11|59|AM|first|morning|breakfast| +29520|AAAAAAAABFDHAAAA|29520|8|12|0|AM|first|morning|breakfast| +29521|AAAAAAAACFDHAAAA|29521|8|12|1|AM|first|morning|breakfast| +29522|AAAAAAAADFDHAAAA|29522|8|12|2|AM|first|morning|breakfast| +29523|AAAAAAAAEFDHAAAA|29523|8|12|3|AM|first|morning|breakfast| +29524|AAAAAAAAFFDHAAAA|29524|8|12|4|AM|first|morning|breakfast| +29525|AAAAAAAAGFDHAAAA|29525|8|12|5|AM|first|morning|breakfast| +29526|AAAAAAAAHFDHAAAA|29526|8|12|6|AM|first|morning|breakfast| +29527|AAAAAAAAIFDHAAAA|29527|8|12|7|AM|first|morning|breakfast| +29528|AAAAAAAAJFDHAAAA|29528|8|12|8|AM|first|morning|breakfast| +29529|AAAAAAAAKFDHAAAA|29529|8|12|9|AM|first|morning|breakfast| +29530|AAAAAAAALFDHAAAA|29530|8|12|10|AM|first|morning|breakfast| +29531|AAAAAAAAMFDHAAAA|29531|8|12|11|AM|first|morning|breakfast| +29532|AAAAAAAANFDHAAAA|29532|8|12|12|AM|first|morning|breakfast| +29533|AAAAAAAAOFDHAAAA|29533|8|12|13|AM|first|morning|breakfast| +29534|AAAAAAAAPFDHAAAA|29534|8|12|14|AM|first|morning|breakfast| +29535|AAAAAAAAAGDHAAAA|29535|8|12|15|AM|first|morning|breakfast| +29536|AAAAAAAABGDHAAAA|29536|8|12|16|AM|first|morning|breakfast| +29537|AAAAAAAACGDHAAAA|29537|8|12|17|AM|first|morning|breakfast| +29538|AAAAAAAADGDHAAAA|29538|8|12|18|AM|first|morning|breakfast| +29539|AAAAAAAAEGDHAAAA|29539|8|12|19|AM|first|morning|breakfast| +29540|AAAAAAAAFGDHAAAA|29540|8|12|20|AM|first|morning|breakfast| +29541|AAAAAAAAGGDHAAAA|29541|8|12|21|AM|first|morning|breakfast| +29542|AAAAAAAAHGDHAAAA|29542|8|12|22|AM|first|morning|breakfast| +29543|AAAAAAAAIGDHAAAA|29543|8|12|23|AM|first|morning|breakfast| +29544|AAAAAAAAJGDHAAAA|29544|8|12|24|AM|first|morning|breakfast| +29545|AAAAAAAAKGDHAAAA|29545|8|12|25|AM|first|morning|breakfast| +29546|AAAAAAAALGDHAAAA|29546|8|12|26|AM|first|morning|breakfast| +29547|AAAAAAAAMGDHAAAA|29547|8|12|27|AM|first|morning|breakfast| +29548|AAAAAAAANGDHAAAA|29548|8|12|28|AM|first|morning|breakfast| +29549|AAAAAAAAOGDHAAAA|29549|8|12|29|AM|first|morning|breakfast| +29550|AAAAAAAAPGDHAAAA|29550|8|12|30|AM|first|morning|breakfast| +29551|AAAAAAAAAHDHAAAA|29551|8|12|31|AM|first|morning|breakfast| +29552|AAAAAAAABHDHAAAA|29552|8|12|32|AM|first|morning|breakfast| +29553|AAAAAAAACHDHAAAA|29553|8|12|33|AM|first|morning|breakfast| +29554|AAAAAAAADHDHAAAA|29554|8|12|34|AM|first|morning|breakfast| +29555|AAAAAAAAEHDHAAAA|29555|8|12|35|AM|first|morning|breakfast| +29556|AAAAAAAAFHDHAAAA|29556|8|12|36|AM|first|morning|breakfast| +29557|AAAAAAAAGHDHAAAA|29557|8|12|37|AM|first|morning|breakfast| +29558|AAAAAAAAHHDHAAAA|29558|8|12|38|AM|first|morning|breakfast| +29559|AAAAAAAAIHDHAAAA|29559|8|12|39|AM|first|morning|breakfast| +29560|AAAAAAAAJHDHAAAA|29560|8|12|40|AM|first|morning|breakfast| +29561|AAAAAAAAKHDHAAAA|29561|8|12|41|AM|first|morning|breakfast| +29562|AAAAAAAALHDHAAAA|29562|8|12|42|AM|first|morning|breakfast| +29563|AAAAAAAAMHDHAAAA|29563|8|12|43|AM|first|morning|breakfast| +29564|AAAAAAAANHDHAAAA|29564|8|12|44|AM|first|morning|breakfast| +29565|AAAAAAAAOHDHAAAA|29565|8|12|45|AM|first|morning|breakfast| +29566|AAAAAAAAPHDHAAAA|29566|8|12|46|AM|first|morning|breakfast| +29567|AAAAAAAAAIDHAAAA|29567|8|12|47|AM|first|morning|breakfast| +29568|AAAAAAAABIDHAAAA|29568|8|12|48|AM|first|morning|breakfast| +29569|AAAAAAAACIDHAAAA|29569|8|12|49|AM|first|morning|breakfast| +29570|AAAAAAAADIDHAAAA|29570|8|12|50|AM|first|morning|breakfast| +29571|AAAAAAAAEIDHAAAA|29571|8|12|51|AM|first|morning|breakfast| +29572|AAAAAAAAFIDHAAAA|29572|8|12|52|AM|first|morning|breakfast| +29573|AAAAAAAAGIDHAAAA|29573|8|12|53|AM|first|morning|breakfast| +29574|AAAAAAAAHIDHAAAA|29574|8|12|54|AM|first|morning|breakfast| +29575|AAAAAAAAIIDHAAAA|29575|8|12|55|AM|first|morning|breakfast| +29576|AAAAAAAAJIDHAAAA|29576|8|12|56|AM|first|morning|breakfast| +29577|AAAAAAAAKIDHAAAA|29577|8|12|57|AM|first|morning|breakfast| +29578|AAAAAAAALIDHAAAA|29578|8|12|58|AM|first|morning|breakfast| +29579|AAAAAAAAMIDHAAAA|29579|8|12|59|AM|first|morning|breakfast| +29580|AAAAAAAANIDHAAAA|29580|8|13|0|AM|first|morning|breakfast| +29581|AAAAAAAAOIDHAAAA|29581|8|13|1|AM|first|morning|breakfast| +29582|AAAAAAAAPIDHAAAA|29582|8|13|2|AM|first|morning|breakfast| +29583|AAAAAAAAAJDHAAAA|29583|8|13|3|AM|first|morning|breakfast| +29584|AAAAAAAABJDHAAAA|29584|8|13|4|AM|first|morning|breakfast| +29585|AAAAAAAACJDHAAAA|29585|8|13|5|AM|first|morning|breakfast| +29586|AAAAAAAADJDHAAAA|29586|8|13|6|AM|first|morning|breakfast| +29587|AAAAAAAAEJDHAAAA|29587|8|13|7|AM|first|morning|breakfast| +29588|AAAAAAAAFJDHAAAA|29588|8|13|8|AM|first|morning|breakfast| +29589|AAAAAAAAGJDHAAAA|29589|8|13|9|AM|first|morning|breakfast| +29590|AAAAAAAAHJDHAAAA|29590|8|13|10|AM|first|morning|breakfast| +29591|AAAAAAAAIJDHAAAA|29591|8|13|11|AM|first|morning|breakfast| +29592|AAAAAAAAJJDHAAAA|29592|8|13|12|AM|first|morning|breakfast| +29593|AAAAAAAAKJDHAAAA|29593|8|13|13|AM|first|morning|breakfast| +29594|AAAAAAAALJDHAAAA|29594|8|13|14|AM|first|morning|breakfast| +29595|AAAAAAAAMJDHAAAA|29595|8|13|15|AM|first|morning|breakfast| +29596|AAAAAAAANJDHAAAA|29596|8|13|16|AM|first|morning|breakfast| +29597|AAAAAAAAOJDHAAAA|29597|8|13|17|AM|first|morning|breakfast| +29598|AAAAAAAAPJDHAAAA|29598|8|13|18|AM|first|morning|breakfast| +29599|AAAAAAAAAKDHAAAA|29599|8|13|19|AM|first|morning|breakfast| +29600|AAAAAAAABKDHAAAA|29600|8|13|20|AM|first|morning|breakfast| +29601|AAAAAAAACKDHAAAA|29601|8|13|21|AM|first|morning|breakfast| +29602|AAAAAAAADKDHAAAA|29602|8|13|22|AM|first|morning|breakfast| +29603|AAAAAAAAEKDHAAAA|29603|8|13|23|AM|first|morning|breakfast| +29604|AAAAAAAAFKDHAAAA|29604|8|13|24|AM|first|morning|breakfast| +29605|AAAAAAAAGKDHAAAA|29605|8|13|25|AM|first|morning|breakfast| +29606|AAAAAAAAHKDHAAAA|29606|8|13|26|AM|first|morning|breakfast| +29607|AAAAAAAAIKDHAAAA|29607|8|13|27|AM|first|morning|breakfast| +29608|AAAAAAAAJKDHAAAA|29608|8|13|28|AM|first|morning|breakfast| +29609|AAAAAAAAKKDHAAAA|29609|8|13|29|AM|first|morning|breakfast| +29610|AAAAAAAALKDHAAAA|29610|8|13|30|AM|first|morning|breakfast| +29611|AAAAAAAAMKDHAAAA|29611|8|13|31|AM|first|morning|breakfast| +29612|AAAAAAAANKDHAAAA|29612|8|13|32|AM|first|morning|breakfast| +29613|AAAAAAAAOKDHAAAA|29613|8|13|33|AM|first|morning|breakfast| +29614|AAAAAAAAPKDHAAAA|29614|8|13|34|AM|first|morning|breakfast| +29615|AAAAAAAAALDHAAAA|29615|8|13|35|AM|first|morning|breakfast| +29616|AAAAAAAABLDHAAAA|29616|8|13|36|AM|first|morning|breakfast| +29617|AAAAAAAACLDHAAAA|29617|8|13|37|AM|first|morning|breakfast| +29618|AAAAAAAADLDHAAAA|29618|8|13|38|AM|first|morning|breakfast| +29619|AAAAAAAAELDHAAAA|29619|8|13|39|AM|first|morning|breakfast| +29620|AAAAAAAAFLDHAAAA|29620|8|13|40|AM|first|morning|breakfast| +29621|AAAAAAAAGLDHAAAA|29621|8|13|41|AM|first|morning|breakfast| +29622|AAAAAAAAHLDHAAAA|29622|8|13|42|AM|first|morning|breakfast| +29623|AAAAAAAAILDHAAAA|29623|8|13|43|AM|first|morning|breakfast| +29624|AAAAAAAAJLDHAAAA|29624|8|13|44|AM|first|morning|breakfast| +29625|AAAAAAAAKLDHAAAA|29625|8|13|45|AM|first|morning|breakfast| +29626|AAAAAAAALLDHAAAA|29626|8|13|46|AM|first|morning|breakfast| +29627|AAAAAAAAMLDHAAAA|29627|8|13|47|AM|first|morning|breakfast| +29628|AAAAAAAANLDHAAAA|29628|8|13|48|AM|first|morning|breakfast| +29629|AAAAAAAAOLDHAAAA|29629|8|13|49|AM|first|morning|breakfast| +29630|AAAAAAAAPLDHAAAA|29630|8|13|50|AM|first|morning|breakfast| +29631|AAAAAAAAAMDHAAAA|29631|8|13|51|AM|first|morning|breakfast| +29632|AAAAAAAABMDHAAAA|29632|8|13|52|AM|first|morning|breakfast| +29633|AAAAAAAACMDHAAAA|29633|8|13|53|AM|first|morning|breakfast| +29634|AAAAAAAADMDHAAAA|29634|8|13|54|AM|first|morning|breakfast| +29635|AAAAAAAAEMDHAAAA|29635|8|13|55|AM|first|morning|breakfast| +29636|AAAAAAAAFMDHAAAA|29636|8|13|56|AM|first|morning|breakfast| +29637|AAAAAAAAGMDHAAAA|29637|8|13|57|AM|first|morning|breakfast| +29638|AAAAAAAAHMDHAAAA|29638|8|13|58|AM|first|morning|breakfast| +29639|AAAAAAAAIMDHAAAA|29639|8|13|59|AM|first|morning|breakfast| +29640|AAAAAAAAJMDHAAAA|29640|8|14|0|AM|first|morning|breakfast| +29641|AAAAAAAAKMDHAAAA|29641|8|14|1|AM|first|morning|breakfast| +29642|AAAAAAAALMDHAAAA|29642|8|14|2|AM|first|morning|breakfast| +29643|AAAAAAAAMMDHAAAA|29643|8|14|3|AM|first|morning|breakfast| +29644|AAAAAAAANMDHAAAA|29644|8|14|4|AM|first|morning|breakfast| +29645|AAAAAAAAOMDHAAAA|29645|8|14|5|AM|first|morning|breakfast| +29646|AAAAAAAAPMDHAAAA|29646|8|14|6|AM|first|morning|breakfast| +29647|AAAAAAAAANDHAAAA|29647|8|14|7|AM|first|morning|breakfast| +29648|AAAAAAAABNDHAAAA|29648|8|14|8|AM|first|morning|breakfast| +29649|AAAAAAAACNDHAAAA|29649|8|14|9|AM|first|morning|breakfast| +29650|AAAAAAAADNDHAAAA|29650|8|14|10|AM|first|morning|breakfast| +29651|AAAAAAAAENDHAAAA|29651|8|14|11|AM|first|morning|breakfast| +29652|AAAAAAAAFNDHAAAA|29652|8|14|12|AM|first|morning|breakfast| +29653|AAAAAAAAGNDHAAAA|29653|8|14|13|AM|first|morning|breakfast| +29654|AAAAAAAAHNDHAAAA|29654|8|14|14|AM|first|morning|breakfast| +29655|AAAAAAAAINDHAAAA|29655|8|14|15|AM|first|morning|breakfast| +29656|AAAAAAAAJNDHAAAA|29656|8|14|16|AM|first|morning|breakfast| +29657|AAAAAAAAKNDHAAAA|29657|8|14|17|AM|first|morning|breakfast| +29658|AAAAAAAALNDHAAAA|29658|8|14|18|AM|first|morning|breakfast| +29659|AAAAAAAAMNDHAAAA|29659|8|14|19|AM|first|morning|breakfast| +29660|AAAAAAAANNDHAAAA|29660|8|14|20|AM|first|morning|breakfast| +29661|AAAAAAAAONDHAAAA|29661|8|14|21|AM|first|morning|breakfast| +29662|AAAAAAAAPNDHAAAA|29662|8|14|22|AM|first|morning|breakfast| +29663|AAAAAAAAAODHAAAA|29663|8|14|23|AM|first|morning|breakfast| +29664|AAAAAAAABODHAAAA|29664|8|14|24|AM|first|morning|breakfast| +29665|AAAAAAAACODHAAAA|29665|8|14|25|AM|first|morning|breakfast| +29666|AAAAAAAADODHAAAA|29666|8|14|26|AM|first|morning|breakfast| +29667|AAAAAAAAEODHAAAA|29667|8|14|27|AM|first|morning|breakfast| +29668|AAAAAAAAFODHAAAA|29668|8|14|28|AM|first|morning|breakfast| +29669|AAAAAAAAGODHAAAA|29669|8|14|29|AM|first|morning|breakfast| +29670|AAAAAAAAHODHAAAA|29670|8|14|30|AM|first|morning|breakfast| +29671|AAAAAAAAIODHAAAA|29671|8|14|31|AM|first|morning|breakfast| +29672|AAAAAAAAJODHAAAA|29672|8|14|32|AM|first|morning|breakfast| +29673|AAAAAAAAKODHAAAA|29673|8|14|33|AM|first|morning|breakfast| +29674|AAAAAAAALODHAAAA|29674|8|14|34|AM|first|morning|breakfast| +29675|AAAAAAAAMODHAAAA|29675|8|14|35|AM|first|morning|breakfast| +29676|AAAAAAAANODHAAAA|29676|8|14|36|AM|first|morning|breakfast| +29677|AAAAAAAAOODHAAAA|29677|8|14|37|AM|first|morning|breakfast| +29678|AAAAAAAAPODHAAAA|29678|8|14|38|AM|first|morning|breakfast| +29679|AAAAAAAAAPDHAAAA|29679|8|14|39|AM|first|morning|breakfast| +29680|AAAAAAAABPDHAAAA|29680|8|14|40|AM|first|morning|breakfast| +29681|AAAAAAAACPDHAAAA|29681|8|14|41|AM|first|morning|breakfast| +29682|AAAAAAAADPDHAAAA|29682|8|14|42|AM|first|morning|breakfast| +29683|AAAAAAAAEPDHAAAA|29683|8|14|43|AM|first|morning|breakfast| +29684|AAAAAAAAFPDHAAAA|29684|8|14|44|AM|first|morning|breakfast| +29685|AAAAAAAAGPDHAAAA|29685|8|14|45|AM|first|morning|breakfast| +29686|AAAAAAAAHPDHAAAA|29686|8|14|46|AM|first|morning|breakfast| +29687|AAAAAAAAIPDHAAAA|29687|8|14|47|AM|first|morning|breakfast| +29688|AAAAAAAAJPDHAAAA|29688|8|14|48|AM|first|morning|breakfast| +29689|AAAAAAAAKPDHAAAA|29689|8|14|49|AM|first|morning|breakfast| +29690|AAAAAAAALPDHAAAA|29690|8|14|50|AM|first|morning|breakfast| +29691|AAAAAAAAMPDHAAAA|29691|8|14|51|AM|first|morning|breakfast| +29692|AAAAAAAANPDHAAAA|29692|8|14|52|AM|first|morning|breakfast| +29693|AAAAAAAAOPDHAAAA|29693|8|14|53|AM|first|morning|breakfast| +29694|AAAAAAAAPPDHAAAA|29694|8|14|54|AM|first|morning|breakfast| +29695|AAAAAAAAAAEHAAAA|29695|8|14|55|AM|first|morning|breakfast| +29696|AAAAAAAABAEHAAAA|29696|8|14|56|AM|first|morning|breakfast| +29697|AAAAAAAACAEHAAAA|29697|8|14|57|AM|first|morning|breakfast| +29698|AAAAAAAADAEHAAAA|29698|8|14|58|AM|first|morning|breakfast| +29699|AAAAAAAAEAEHAAAA|29699|8|14|59|AM|first|morning|breakfast| +29700|AAAAAAAAFAEHAAAA|29700|8|15|0|AM|first|morning|breakfast| +29701|AAAAAAAAGAEHAAAA|29701|8|15|1|AM|first|morning|breakfast| +29702|AAAAAAAAHAEHAAAA|29702|8|15|2|AM|first|morning|breakfast| +29703|AAAAAAAAIAEHAAAA|29703|8|15|3|AM|first|morning|breakfast| +29704|AAAAAAAAJAEHAAAA|29704|8|15|4|AM|first|morning|breakfast| +29705|AAAAAAAAKAEHAAAA|29705|8|15|5|AM|first|morning|breakfast| +29706|AAAAAAAALAEHAAAA|29706|8|15|6|AM|first|morning|breakfast| +29707|AAAAAAAAMAEHAAAA|29707|8|15|7|AM|first|morning|breakfast| +29708|AAAAAAAANAEHAAAA|29708|8|15|8|AM|first|morning|breakfast| +29709|AAAAAAAAOAEHAAAA|29709|8|15|9|AM|first|morning|breakfast| +29710|AAAAAAAAPAEHAAAA|29710|8|15|10|AM|first|morning|breakfast| +29711|AAAAAAAAABEHAAAA|29711|8|15|11|AM|first|morning|breakfast| +29712|AAAAAAAABBEHAAAA|29712|8|15|12|AM|first|morning|breakfast| +29713|AAAAAAAACBEHAAAA|29713|8|15|13|AM|first|morning|breakfast| +29714|AAAAAAAADBEHAAAA|29714|8|15|14|AM|first|morning|breakfast| +29715|AAAAAAAAEBEHAAAA|29715|8|15|15|AM|first|morning|breakfast| +29716|AAAAAAAAFBEHAAAA|29716|8|15|16|AM|first|morning|breakfast| +29717|AAAAAAAAGBEHAAAA|29717|8|15|17|AM|first|morning|breakfast| +29718|AAAAAAAAHBEHAAAA|29718|8|15|18|AM|first|morning|breakfast| +29719|AAAAAAAAIBEHAAAA|29719|8|15|19|AM|first|morning|breakfast| +29720|AAAAAAAAJBEHAAAA|29720|8|15|20|AM|first|morning|breakfast| +29721|AAAAAAAAKBEHAAAA|29721|8|15|21|AM|first|morning|breakfast| +29722|AAAAAAAALBEHAAAA|29722|8|15|22|AM|first|morning|breakfast| +29723|AAAAAAAAMBEHAAAA|29723|8|15|23|AM|first|morning|breakfast| +29724|AAAAAAAANBEHAAAA|29724|8|15|24|AM|first|morning|breakfast| +29725|AAAAAAAAOBEHAAAA|29725|8|15|25|AM|first|morning|breakfast| +29726|AAAAAAAAPBEHAAAA|29726|8|15|26|AM|first|morning|breakfast| +29727|AAAAAAAAACEHAAAA|29727|8|15|27|AM|first|morning|breakfast| +29728|AAAAAAAABCEHAAAA|29728|8|15|28|AM|first|morning|breakfast| +29729|AAAAAAAACCEHAAAA|29729|8|15|29|AM|first|morning|breakfast| +29730|AAAAAAAADCEHAAAA|29730|8|15|30|AM|first|morning|breakfast| +29731|AAAAAAAAECEHAAAA|29731|8|15|31|AM|first|morning|breakfast| +29732|AAAAAAAAFCEHAAAA|29732|8|15|32|AM|first|morning|breakfast| +29733|AAAAAAAAGCEHAAAA|29733|8|15|33|AM|first|morning|breakfast| +29734|AAAAAAAAHCEHAAAA|29734|8|15|34|AM|first|morning|breakfast| +29735|AAAAAAAAICEHAAAA|29735|8|15|35|AM|first|morning|breakfast| +29736|AAAAAAAAJCEHAAAA|29736|8|15|36|AM|first|morning|breakfast| +29737|AAAAAAAAKCEHAAAA|29737|8|15|37|AM|first|morning|breakfast| +29738|AAAAAAAALCEHAAAA|29738|8|15|38|AM|first|morning|breakfast| +29739|AAAAAAAAMCEHAAAA|29739|8|15|39|AM|first|morning|breakfast| +29740|AAAAAAAANCEHAAAA|29740|8|15|40|AM|first|morning|breakfast| +29741|AAAAAAAAOCEHAAAA|29741|8|15|41|AM|first|morning|breakfast| +29742|AAAAAAAAPCEHAAAA|29742|8|15|42|AM|first|morning|breakfast| +29743|AAAAAAAAADEHAAAA|29743|8|15|43|AM|first|morning|breakfast| +29744|AAAAAAAABDEHAAAA|29744|8|15|44|AM|first|morning|breakfast| +29745|AAAAAAAACDEHAAAA|29745|8|15|45|AM|first|morning|breakfast| +29746|AAAAAAAADDEHAAAA|29746|8|15|46|AM|first|morning|breakfast| +29747|AAAAAAAAEDEHAAAA|29747|8|15|47|AM|first|morning|breakfast| +29748|AAAAAAAAFDEHAAAA|29748|8|15|48|AM|first|morning|breakfast| +29749|AAAAAAAAGDEHAAAA|29749|8|15|49|AM|first|morning|breakfast| +29750|AAAAAAAAHDEHAAAA|29750|8|15|50|AM|first|morning|breakfast| +29751|AAAAAAAAIDEHAAAA|29751|8|15|51|AM|first|morning|breakfast| +29752|AAAAAAAAJDEHAAAA|29752|8|15|52|AM|first|morning|breakfast| +29753|AAAAAAAAKDEHAAAA|29753|8|15|53|AM|first|morning|breakfast| +29754|AAAAAAAALDEHAAAA|29754|8|15|54|AM|first|morning|breakfast| +29755|AAAAAAAAMDEHAAAA|29755|8|15|55|AM|first|morning|breakfast| +29756|AAAAAAAANDEHAAAA|29756|8|15|56|AM|first|morning|breakfast| +29757|AAAAAAAAODEHAAAA|29757|8|15|57|AM|first|morning|breakfast| +29758|AAAAAAAAPDEHAAAA|29758|8|15|58|AM|first|morning|breakfast| +29759|AAAAAAAAAEEHAAAA|29759|8|15|59|AM|first|morning|breakfast| +29760|AAAAAAAABEEHAAAA|29760|8|16|0|AM|first|morning|breakfast| +29761|AAAAAAAACEEHAAAA|29761|8|16|1|AM|first|morning|breakfast| +29762|AAAAAAAADEEHAAAA|29762|8|16|2|AM|first|morning|breakfast| +29763|AAAAAAAAEEEHAAAA|29763|8|16|3|AM|first|morning|breakfast| +29764|AAAAAAAAFEEHAAAA|29764|8|16|4|AM|first|morning|breakfast| +29765|AAAAAAAAGEEHAAAA|29765|8|16|5|AM|first|morning|breakfast| +29766|AAAAAAAAHEEHAAAA|29766|8|16|6|AM|first|morning|breakfast| +29767|AAAAAAAAIEEHAAAA|29767|8|16|7|AM|first|morning|breakfast| +29768|AAAAAAAAJEEHAAAA|29768|8|16|8|AM|first|morning|breakfast| +29769|AAAAAAAAKEEHAAAA|29769|8|16|9|AM|first|morning|breakfast| +29770|AAAAAAAALEEHAAAA|29770|8|16|10|AM|first|morning|breakfast| +29771|AAAAAAAAMEEHAAAA|29771|8|16|11|AM|first|morning|breakfast| +29772|AAAAAAAANEEHAAAA|29772|8|16|12|AM|first|morning|breakfast| +29773|AAAAAAAAOEEHAAAA|29773|8|16|13|AM|first|morning|breakfast| +29774|AAAAAAAAPEEHAAAA|29774|8|16|14|AM|first|morning|breakfast| +29775|AAAAAAAAAFEHAAAA|29775|8|16|15|AM|first|morning|breakfast| +29776|AAAAAAAABFEHAAAA|29776|8|16|16|AM|first|morning|breakfast| +29777|AAAAAAAACFEHAAAA|29777|8|16|17|AM|first|morning|breakfast| +29778|AAAAAAAADFEHAAAA|29778|8|16|18|AM|first|morning|breakfast| +29779|AAAAAAAAEFEHAAAA|29779|8|16|19|AM|first|morning|breakfast| +29780|AAAAAAAAFFEHAAAA|29780|8|16|20|AM|first|morning|breakfast| +29781|AAAAAAAAGFEHAAAA|29781|8|16|21|AM|first|morning|breakfast| +29782|AAAAAAAAHFEHAAAA|29782|8|16|22|AM|first|morning|breakfast| +29783|AAAAAAAAIFEHAAAA|29783|8|16|23|AM|first|morning|breakfast| +29784|AAAAAAAAJFEHAAAA|29784|8|16|24|AM|first|morning|breakfast| +29785|AAAAAAAAKFEHAAAA|29785|8|16|25|AM|first|morning|breakfast| +29786|AAAAAAAALFEHAAAA|29786|8|16|26|AM|first|morning|breakfast| +29787|AAAAAAAAMFEHAAAA|29787|8|16|27|AM|first|morning|breakfast| +29788|AAAAAAAANFEHAAAA|29788|8|16|28|AM|first|morning|breakfast| +29789|AAAAAAAAOFEHAAAA|29789|8|16|29|AM|first|morning|breakfast| +29790|AAAAAAAAPFEHAAAA|29790|8|16|30|AM|first|morning|breakfast| +29791|AAAAAAAAAGEHAAAA|29791|8|16|31|AM|first|morning|breakfast| +29792|AAAAAAAABGEHAAAA|29792|8|16|32|AM|first|morning|breakfast| +29793|AAAAAAAACGEHAAAA|29793|8|16|33|AM|first|morning|breakfast| +29794|AAAAAAAADGEHAAAA|29794|8|16|34|AM|first|morning|breakfast| +29795|AAAAAAAAEGEHAAAA|29795|8|16|35|AM|first|morning|breakfast| +29796|AAAAAAAAFGEHAAAA|29796|8|16|36|AM|first|morning|breakfast| +29797|AAAAAAAAGGEHAAAA|29797|8|16|37|AM|first|morning|breakfast| +29798|AAAAAAAAHGEHAAAA|29798|8|16|38|AM|first|morning|breakfast| +29799|AAAAAAAAIGEHAAAA|29799|8|16|39|AM|first|morning|breakfast| +29800|AAAAAAAAJGEHAAAA|29800|8|16|40|AM|first|morning|breakfast| +29801|AAAAAAAAKGEHAAAA|29801|8|16|41|AM|first|morning|breakfast| +29802|AAAAAAAALGEHAAAA|29802|8|16|42|AM|first|morning|breakfast| +29803|AAAAAAAAMGEHAAAA|29803|8|16|43|AM|first|morning|breakfast| +29804|AAAAAAAANGEHAAAA|29804|8|16|44|AM|first|morning|breakfast| +29805|AAAAAAAAOGEHAAAA|29805|8|16|45|AM|first|morning|breakfast| +29806|AAAAAAAAPGEHAAAA|29806|8|16|46|AM|first|morning|breakfast| +29807|AAAAAAAAAHEHAAAA|29807|8|16|47|AM|first|morning|breakfast| +29808|AAAAAAAABHEHAAAA|29808|8|16|48|AM|first|morning|breakfast| +29809|AAAAAAAACHEHAAAA|29809|8|16|49|AM|first|morning|breakfast| +29810|AAAAAAAADHEHAAAA|29810|8|16|50|AM|first|morning|breakfast| +29811|AAAAAAAAEHEHAAAA|29811|8|16|51|AM|first|morning|breakfast| +29812|AAAAAAAAFHEHAAAA|29812|8|16|52|AM|first|morning|breakfast| +29813|AAAAAAAAGHEHAAAA|29813|8|16|53|AM|first|morning|breakfast| +29814|AAAAAAAAHHEHAAAA|29814|8|16|54|AM|first|morning|breakfast| +29815|AAAAAAAAIHEHAAAA|29815|8|16|55|AM|first|morning|breakfast| +29816|AAAAAAAAJHEHAAAA|29816|8|16|56|AM|first|morning|breakfast| +29817|AAAAAAAAKHEHAAAA|29817|8|16|57|AM|first|morning|breakfast| +29818|AAAAAAAALHEHAAAA|29818|8|16|58|AM|first|morning|breakfast| +29819|AAAAAAAAMHEHAAAA|29819|8|16|59|AM|first|morning|breakfast| +29820|AAAAAAAANHEHAAAA|29820|8|17|0|AM|first|morning|breakfast| +29821|AAAAAAAAOHEHAAAA|29821|8|17|1|AM|first|morning|breakfast| +29822|AAAAAAAAPHEHAAAA|29822|8|17|2|AM|first|morning|breakfast| +29823|AAAAAAAAAIEHAAAA|29823|8|17|3|AM|first|morning|breakfast| +29824|AAAAAAAABIEHAAAA|29824|8|17|4|AM|first|morning|breakfast| +29825|AAAAAAAACIEHAAAA|29825|8|17|5|AM|first|morning|breakfast| +29826|AAAAAAAADIEHAAAA|29826|8|17|6|AM|first|morning|breakfast| +29827|AAAAAAAAEIEHAAAA|29827|8|17|7|AM|first|morning|breakfast| +29828|AAAAAAAAFIEHAAAA|29828|8|17|8|AM|first|morning|breakfast| +29829|AAAAAAAAGIEHAAAA|29829|8|17|9|AM|first|morning|breakfast| +29830|AAAAAAAAHIEHAAAA|29830|8|17|10|AM|first|morning|breakfast| +29831|AAAAAAAAIIEHAAAA|29831|8|17|11|AM|first|morning|breakfast| +29832|AAAAAAAAJIEHAAAA|29832|8|17|12|AM|first|morning|breakfast| +29833|AAAAAAAAKIEHAAAA|29833|8|17|13|AM|first|morning|breakfast| +29834|AAAAAAAALIEHAAAA|29834|8|17|14|AM|first|morning|breakfast| +29835|AAAAAAAAMIEHAAAA|29835|8|17|15|AM|first|morning|breakfast| +29836|AAAAAAAANIEHAAAA|29836|8|17|16|AM|first|morning|breakfast| +29837|AAAAAAAAOIEHAAAA|29837|8|17|17|AM|first|morning|breakfast| +29838|AAAAAAAAPIEHAAAA|29838|8|17|18|AM|first|morning|breakfast| +29839|AAAAAAAAAJEHAAAA|29839|8|17|19|AM|first|morning|breakfast| +29840|AAAAAAAABJEHAAAA|29840|8|17|20|AM|first|morning|breakfast| +29841|AAAAAAAACJEHAAAA|29841|8|17|21|AM|first|morning|breakfast| +29842|AAAAAAAADJEHAAAA|29842|8|17|22|AM|first|morning|breakfast| +29843|AAAAAAAAEJEHAAAA|29843|8|17|23|AM|first|morning|breakfast| +29844|AAAAAAAAFJEHAAAA|29844|8|17|24|AM|first|morning|breakfast| +29845|AAAAAAAAGJEHAAAA|29845|8|17|25|AM|first|morning|breakfast| +29846|AAAAAAAAHJEHAAAA|29846|8|17|26|AM|first|morning|breakfast| +29847|AAAAAAAAIJEHAAAA|29847|8|17|27|AM|first|morning|breakfast| +29848|AAAAAAAAJJEHAAAA|29848|8|17|28|AM|first|morning|breakfast| +29849|AAAAAAAAKJEHAAAA|29849|8|17|29|AM|first|morning|breakfast| +29850|AAAAAAAALJEHAAAA|29850|8|17|30|AM|first|morning|breakfast| +29851|AAAAAAAAMJEHAAAA|29851|8|17|31|AM|first|morning|breakfast| +29852|AAAAAAAANJEHAAAA|29852|8|17|32|AM|first|morning|breakfast| +29853|AAAAAAAAOJEHAAAA|29853|8|17|33|AM|first|morning|breakfast| +29854|AAAAAAAAPJEHAAAA|29854|8|17|34|AM|first|morning|breakfast| +29855|AAAAAAAAAKEHAAAA|29855|8|17|35|AM|first|morning|breakfast| +29856|AAAAAAAABKEHAAAA|29856|8|17|36|AM|first|morning|breakfast| +29857|AAAAAAAACKEHAAAA|29857|8|17|37|AM|first|morning|breakfast| +29858|AAAAAAAADKEHAAAA|29858|8|17|38|AM|first|morning|breakfast| +29859|AAAAAAAAEKEHAAAA|29859|8|17|39|AM|first|morning|breakfast| +29860|AAAAAAAAFKEHAAAA|29860|8|17|40|AM|first|morning|breakfast| +29861|AAAAAAAAGKEHAAAA|29861|8|17|41|AM|first|morning|breakfast| +29862|AAAAAAAAHKEHAAAA|29862|8|17|42|AM|first|morning|breakfast| +29863|AAAAAAAAIKEHAAAA|29863|8|17|43|AM|first|morning|breakfast| +29864|AAAAAAAAJKEHAAAA|29864|8|17|44|AM|first|morning|breakfast| +29865|AAAAAAAAKKEHAAAA|29865|8|17|45|AM|first|morning|breakfast| +29866|AAAAAAAALKEHAAAA|29866|8|17|46|AM|first|morning|breakfast| +29867|AAAAAAAAMKEHAAAA|29867|8|17|47|AM|first|morning|breakfast| +29868|AAAAAAAANKEHAAAA|29868|8|17|48|AM|first|morning|breakfast| +29869|AAAAAAAAOKEHAAAA|29869|8|17|49|AM|first|morning|breakfast| +29870|AAAAAAAAPKEHAAAA|29870|8|17|50|AM|first|morning|breakfast| +29871|AAAAAAAAALEHAAAA|29871|8|17|51|AM|first|morning|breakfast| +29872|AAAAAAAABLEHAAAA|29872|8|17|52|AM|first|morning|breakfast| +29873|AAAAAAAACLEHAAAA|29873|8|17|53|AM|first|morning|breakfast| +29874|AAAAAAAADLEHAAAA|29874|8|17|54|AM|first|morning|breakfast| +29875|AAAAAAAAELEHAAAA|29875|8|17|55|AM|first|morning|breakfast| +29876|AAAAAAAAFLEHAAAA|29876|8|17|56|AM|first|morning|breakfast| +29877|AAAAAAAAGLEHAAAA|29877|8|17|57|AM|first|morning|breakfast| +29878|AAAAAAAAHLEHAAAA|29878|8|17|58|AM|first|morning|breakfast| +29879|AAAAAAAAILEHAAAA|29879|8|17|59|AM|first|morning|breakfast| +29880|AAAAAAAAJLEHAAAA|29880|8|18|0|AM|first|morning|breakfast| +29881|AAAAAAAAKLEHAAAA|29881|8|18|1|AM|first|morning|breakfast| +29882|AAAAAAAALLEHAAAA|29882|8|18|2|AM|first|morning|breakfast| +29883|AAAAAAAAMLEHAAAA|29883|8|18|3|AM|first|morning|breakfast| +29884|AAAAAAAANLEHAAAA|29884|8|18|4|AM|first|morning|breakfast| +29885|AAAAAAAAOLEHAAAA|29885|8|18|5|AM|first|morning|breakfast| +29886|AAAAAAAAPLEHAAAA|29886|8|18|6|AM|first|morning|breakfast| +29887|AAAAAAAAAMEHAAAA|29887|8|18|7|AM|first|morning|breakfast| +29888|AAAAAAAABMEHAAAA|29888|8|18|8|AM|first|morning|breakfast| +29889|AAAAAAAACMEHAAAA|29889|8|18|9|AM|first|morning|breakfast| +29890|AAAAAAAADMEHAAAA|29890|8|18|10|AM|first|morning|breakfast| +29891|AAAAAAAAEMEHAAAA|29891|8|18|11|AM|first|morning|breakfast| +29892|AAAAAAAAFMEHAAAA|29892|8|18|12|AM|first|morning|breakfast| +29893|AAAAAAAAGMEHAAAA|29893|8|18|13|AM|first|morning|breakfast| +29894|AAAAAAAAHMEHAAAA|29894|8|18|14|AM|first|morning|breakfast| +29895|AAAAAAAAIMEHAAAA|29895|8|18|15|AM|first|morning|breakfast| +29896|AAAAAAAAJMEHAAAA|29896|8|18|16|AM|first|morning|breakfast| +29897|AAAAAAAAKMEHAAAA|29897|8|18|17|AM|first|morning|breakfast| +29898|AAAAAAAALMEHAAAA|29898|8|18|18|AM|first|morning|breakfast| +29899|AAAAAAAAMMEHAAAA|29899|8|18|19|AM|first|morning|breakfast| +29900|AAAAAAAANMEHAAAA|29900|8|18|20|AM|first|morning|breakfast| +29901|AAAAAAAAOMEHAAAA|29901|8|18|21|AM|first|morning|breakfast| +29902|AAAAAAAAPMEHAAAA|29902|8|18|22|AM|first|morning|breakfast| +29903|AAAAAAAAANEHAAAA|29903|8|18|23|AM|first|morning|breakfast| +29904|AAAAAAAABNEHAAAA|29904|8|18|24|AM|first|morning|breakfast| +29905|AAAAAAAACNEHAAAA|29905|8|18|25|AM|first|morning|breakfast| +29906|AAAAAAAADNEHAAAA|29906|8|18|26|AM|first|morning|breakfast| +29907|AAAAAAAAENEHAAAA|29907|8|18|27|AM|first|morning|breakfast| +29908|AAAAAAAAFNEHAAAA|29908|8|18|28|AM|first|morning|breakfast| +29909|AAAAAAAAGNEHAAAA|29909|8|18|29|AM|first|morning|breakfast| +29910|AAAAAAAAHNEHAAAA|29910|8|18|30|AM|first|morning|breakfast| +29911|AAAAAAAAINEHAAAA|29911|8|18|31|AM|first|morning|breakfast| +29912|AAAAAAAAJNEHAAAA|29912|8|18|32|AM|first|morning|breakfast| +29913|AAAAAAAAKNEHAAAA|29913|8|18|33|AM|first|morning|breakfast| +29914|AAAAAAAALNEHAAAA|29914|8|18|34|AM|first|morning|breakfast| +29915|AAAAAAAAMNEHAAAA|29915|8|18|35|AM|first|morning|breakfast| +29916|AAAAAAAANNEHAAAA|29916|8|18|36|AM|first|morning|breakfast| +29917|AAAAAAAAONEHAAAA|29917|8|18|37|AM|first|morning|breakfast| +29918|AAAAAAAAPNEHAAAA|29918|8|18|38|AM|first|morning|breakfast| +29919|AAAAAAAAAOEHAAAA|29919|8|18|39|AM|first|morning|breakfast| +29920|AAAAAAAABOEHAAAA|29920|8|18|40|AM|first|morning|breakfast| +29921|AAAAAAAACOEHAAAA|29921|8|18|41|AM|first|morning|breakfast| +29922|AAAAAAAADOEHAAAA|29922|8|18|42|AM|first|morning|breakfast| +29923|AAAAAAAAEOEHAAAA|29923|8|18|43|AM|first|morning|breakfast| +29924|AAAAAAAAFOEHAAAA|29924|8|18|44|AM|first|morning|breakfast| +29925|AAAAAAAAGOEHAAAA|29925|8|18|45|AM|first|morning|breakfast| +29926|AAAAAAAAHOEHAAAA|29926|8|18|46|AM|first|morning|breakfast| +29927|AAAAAAAAIOEHAAAA|29927|8|18|47|AM|first|morning|breakfast| +29928|AAAAAAAAJOEHAAAA|29928|8|18|48|AM|first|morning|breakfast| +29929|AAAAAAAAKOEHAAAA|29929|8|18|49|AM|first|morning|breakfast| +29930|AAAAAAAALOEHAAAA|29930|8|18|50|AM|first|morning|breakfast| +29931|AAAAAAAAMOEHAAAA|29931|8|18|51|AM|first|morning|breakfast| +29932|AAAAAAAANOEHAAAA|29932|8|18|52|AM|first|morning|breakfast| +29933|AAAAAAAAOOEHAAAA|29933|8|18|53|AM|first|morning|breakfast| +29934|AAAAAAAAPOEHAAAA|29934|8|18|54|AM|first|morning|breakfast| +29935|AAAAAAAAAPEHAAAA|29935|8|18|55|AM|first|morning|breakfast| +29936|AAAAAAAABPEHAAAA|29936|8|18|56|AM|first|morning|breakfast| +29937|AAAAAAAACPEHAAAA|29937|8|18|57|AM|first|morning|breakfast| +29938|AAAAAAAADPEHAAAA|29938|8|18|58|AM|first|morning|breakfast| +29939|AAAAAAAAEPEHAAAA|29939|8|18|59|AM|first|morning|breakfast| +29940|AAAAAAAAFPEHAAAA|29940|8|19|0|AM|first|morning|breakfast| +29941|AAAAAAAAGPEHAAAA|29941|8|19|1|AM|first|morning|breakfast| +29942|AAAAAAAAHPEHAAAA|29942|8|19|2|AM|first|morning|breakfast| +29943|AAAAAAAAIPEHAAAA|29943|8|19|3|AM|first|morning|breakfast| +29944|AAAAAAAAJPEHAAAA|29944|8|19|4|AM|first|morning|breakfast| +29945|AAAAAAAAKPEHAAAA|29945|8|19|5|AM|first|morning|breakfast| +29946|AAAAAAAALPEHAAAA|29946|8|19|6|AM|first|morning|breakfast| +29947|AAAAAAAAMPEHAAAA|29947|8|19|7|AM|first|morning|breakfast| +29948|AAAAAAAANPEHAAAA|29948|8|19|8|AM|first|morning|breakfast| +29949|AAAAAAAAOPEHAAAA|29949|8|19|9|AM|first|morning|breakfast| +29950|AAAAAAAAPPEHAAAA|29950|8|19|10|AM|first|morning|breakfast| +29951|AAAAAAAAAAFHAAAA|29951|8|19|11|AM|first|morning|breakfast| +29952|AAAAAAAABAFHAAAA|29952|8|19|12|AM|first|morning|breakfast| +29953|AAAAAAAACAFHAAAA|29953|8|19|13|AM|first|morning|breakfast| +29954|AAAAAAAADAFHAAAA|29954|8|19|14|AM|first|morning|breakfast| +29955|AAAAAAAAEAFHAAAA|29955|8|19|15|AM|first|morning|breakfast| +29956|AAAAAAAAFAFHAAAA|29956|8|19|16|AM|first|morning|breakfast| +29957|AAAAAAAAGAFHAAAA|29957|8|19|17|AM|first|morning|breakfast| +29958|AAAAAAAAHAFHAAAA|29958|8|19|18|AM|first|morning|breakfast| +29959|AAAAAAAAIAFHAAAA|29959|8|19|19|AM|first|morning|breakfast| +29960|AAAAAAAAJAFHAAAA|29960|8|19|20|AM|first|morning|breakfast| +29961|AAAAAAAAKAFHAAAA|29961|8|19|21|AM|first|morning|breakfast| +29962|AAAAAAAALAFHAAAA|29962|8|19|22|AM|first|morning|breakfast| +29963|AAAAAAAAMAFHAAAA|29963|8|19|23|AM|first|morning|breakfast| +29964|AAAAAAAANAFHAAAA|29964|8|19|24|AM|first|morning|breakfast| +29965|AAAAAAAAOAFHAAAA|29965|8|19|25|AM|first|morning|breakfast| +29966|AAAAAAAAPAFHAAAA|29966|8|19|26|AM|first|morning|breakfast| +29967|AAAAAAAAABFHAAAA|29967|8|19|27|AM|first|morning|breakfast| +29968|AAAAAAAABBFHAAAA|29968|8|19|28|AM|first|morning|breakfast| +29969|AAAAAAAACBFHAAAA|29969|8|19|29|AM|first|morning|breakfast| +29970|AAAAAAAADBFHAAAA|29970|8|19|30|AM|first|morning|breakfast| +29971|AAAAAAAAEBFHAAAA|29971|8|19|31|AM|first|morning|breakfast| +29972|AAAAAAAAFBFHAAAA|29972|8|19|32|AM|first|morning|breakfast| +29973|AAAAAAAAGBFHAAAA|29973|8|19|33|AM|first|morning|breakfast| +29974|AAAAAAAAHBFHAAAA|29974|8|19|34|AM|first|morning|breakfast| +29975|AAAAAAAAIBFHAAAA|29975|8|19|35|AM|first|morning|breakfast| +29976|AAAAAAAAJBFHAAAA|29976|8|19|36|AM|first|morning|breakfast| +29977|AAAAAAAAKBFHAAAA|29977|8|19|37|AM|first|morning|breakfast| +29978|AAAAAAAALBFHAAAA|29978|8|19|38|AM|first|morning|breakfast| +29979|AAAAAAAAMBFHAAAA|29979|8|19|39|AM|first|morning|breakfast| +29980|AAAAAAAANBFHAAAA|29980|8|19|40|AM|first|morning|breakfast| +29981|AAAAAAAAOBFHAAAA|29981|8|19|41|AM|first|morning|breakfast| +29982|AAAAAAAAPBFHAAAA|29982|8|19|42|AM|first|morning|breakfast| +29983|AAAAAAAAACFHAAAA|29983|8|19|43|AM|first|morning|breakfast| +29984|AAAAAAAABCFHAAAA|29984|8|19|44|AM|first|morning|breakfast| +29985|AAAAAAAACCFHAAAA|29985|8|19|45|AM|first|morning|breakfast| +29986|AAAAAAAADCFHAAAA|29986|8|19|46|AM|first|morning|breakfast| +29987|AAAAAAAAECFHAAAA|29987|8|19|47|AM|first|morning|breakfast| +29988|AAAAAAAAFCFHAAAA|29988|8|19|48|AM|first|morning|breakfast| +29989|AAAAAAAAGCFHAAAA|29989|8|19|49|AM|first|morning|breakfast| +29990|AAAAAAAAHCFHAAAA|29990|8|19|50|AM|first|morning|breakfast| +29991|AAAAAAAAICFHAAAA|29991|8|19|51|AM|first|morning|breakfast| +29992|AAAAAAAAJCFHAAAA|29992|8|19|52|AM|first|morning|breakfast| +29993|AAAAAAAAKCFHAAAA|29993|8|19|53|AM|first|morning|breakfast| +29994|AAAAAAAALCFHAAAA|29994|8|19|54|AM|first|morning|breakfast| +29995|AAAAAAAAMCFHAAAA|29995|8|19|55|AM|first|morning|breakfast| +29996|AAAAAAAANCFHAAAA|29996|8|19|56|AM|first|morning|breakfast| +29997|AAAAAAAAOCFHAAAA|29997|8|19|57|AM|first|morning|breakfast| +29998|AAAAAAAAPCFHAAAA|29998|8|19|58|AM|first|morning|breakfast| +29999|AAAAAAAAADFHAAAA|29999|8|19|59|AM|first|morning|breakfast| +30000|AAAAAAAABDFHAAAA|30000|8|20|0|AM|first|morning|breakfast| +30001|AAAAAAAACDFHAAAA|30001|8|20|1|AM|first|morning|breakfast| +30002|AAAAAAAADDFHAAAA|30002|8|20|2|AM|first|morning|breakfast| +30003|AAAAAAAAEDFHAAAA|30003|8|20|3|AM|first|morning|breakfast| +30004|AAAAAAAAFDFHAAAA|30004|8|20|4|AM|first|morning|breakfast| +30005|AAAAAAAAGDFHAAAA|30005|8|20|5|AM|first|morning|breakfast| +30006|AAAAAAAAHDFHAAAA|30006|8|20|6|AM|first|morning|breakfast| +30007|AAAAAAAAIDFHAAAA|30007|8|20|7|AM|first|morning|breakfast| +30008|AAAAAAAAJDFHAAAA|30008|8|20|8|AM|first|morning|breakfast| +30009|AAAAAAAAKDFHAAAA|30009|8|20|9|AM|first|morning|breakfast| +30010|AAAAAAAALDFHAAAA|30010|8|20|10|AM|first|morning|breakfast| +30011|AAAAAAAAMDFHAAAA|30011|8|20|11|AM|first|morning|breakfast| +30012|AAAAAAAANDFHAAAA|30012|8|20|12|AM|first|morning|breakfast| +30013|AAAAAAAAODFHAAAA|30013|8|20|13|AM|first|morning|breakfast| +30014|AAAAAAAAPDFHAAAA|30014|8|20|14|AM|first|morning|breakfast| +30015|AAAAAAAAAEFHAAAA|30015|8|20|15|AM|first|morning|breakfast| +30016|AAAAAAAABEFHAAAA|30016|8|20|16|AM|first|morning|breakfast| +30017|AAAAAAAACEFHAAAA|30017|8|20|17|AM|first|morning|breakfast| +30018|AAAAAAAADEFHAAAA|30018|8|20|18|AM|first|morning|breakfast| +30019|AAAAAAAAEEFHAAAA|30019|8|20|19|AM|first|morning|breakfast| +30020|AAAAAAAAFEFHAAAA|30020|8|20|20|AM|first|morning|breakfast| +30021|AAAAAAAAGEFHAAAA|30021|8|20|21|AM|first|morning|breakfast| +30022|AAAAAAAAHEFHAAAA|30022|8|20|22|AM|first|morning|breakfast| +30023|AAAAAAAAIEFHAAAA|30023|8|20|23|AM|first|morning|breakfast| +30024|AAAAAAAAJEFHAAAA|30024|8|20|24|AM|first|morning|breakfast| +30025|AAAAAAAAKEFHAAAA|30025|8|20|25|AM|first|morning|breakfast| +30026|AAAAAAAALEFHAAAA|30026|8|20|26|AM|first|morning|breakfast| +30027|AAAAAAAAMEFHAAAA|30027|8|20|27|AM|first|morning|breakfast| +30028|AAAAAAAANEFHAAAA|30028|8|20|28|AM|first|morning|breakfast| +30029|AAAAAAAAOEFHAAAA|30029|8|20|29|AM|first|morning|breakfast| +30030|AAAAAAAAPEFHAAAA|30030|8|20|30|AM|first|morning|breakfast| +30031|AAAAAAAAAFFHAAAA|30031|8|20|31|AM|first|morning|breakfast| +30032|AAAAAAAABFFHAAAA|30032|8|20|32|AM|first|morning|breakfast| +30033|AAAAAAAACFFHAAAA|30033|8|20|33|AM|first|morning|breakfast| +30034|AAAAAAAADFFHAAAA|30034|8|20|34|AM|first|morning|breakfast| +30035|AAAAAAAAEFFHAAAA|30035|8|20|35|AM|first|morning|breakfast| +30036|AAAAAAAAFFFHAAAA|30036|8|20|36|AM|first|morning|breakfast| +30037|AAAAAAAAGFFHAAAA|30037|8|20|37|AM|first|morning|breakfast| +30038|AAAAAAAAHFFHAAAA|30038|8|20|38|AM|first|morning|breakfast| +30039|AAAAAAAAIFFHAAAA|30039|8|20|39|AM|first|morning|breakfast| +30040|AAAAAAAAJFFHAAAA|30040|8|20|40|AM|first|morning|breakfast| +30041|AAAAAAAAKFFHAAAA|30041|8|20|41|AM|first|morning|breakfast| +30042|AAAAAAAALFFHAAAA|30042|8|20|42|AM|first|morning|breakfast| +30043|AAAAAAAAMFFHAAAA|30043|8|20|43|AM|first|morning|breakfast| +30044|AAAAAAAANFFHAAAA|30044|8|20|44|AM|first|morning|breakfast| +30045|AAAAAAAAOFFHAAAA|30045|8|20|45|AM|first|morning|breakfast| +30046|AAAAAAAAPFFHAAAA|30046|8|20|46|AM|first|morning|breakfast| +30047|AAAAAAAAAGFHAAAA|30047|8|20|47|AM|first|morning|breakfast| +30048|AAAAAAAABGFHAAAA|30048|8|20|48|AM|first|morning|breakfast| +30049|AAAAAAAACGFHAAAA|30049|8|20|49|AM|first|morning|breakfast| +30050|AAAAAAAADGFHAAAA|30050|8|20|50|AM|first|morning|breakfast| +30051|AAAAAAAAEGFHAAAA|30051|8|20|51|AM|first|morning|breakfast| +30052|AAAAAAAAFGFHAAAA|30052|8|20|52|AM|first|morning|breakfast| +30053|AAAAAAAAGGFHAAAA|30053|8|20|53|AM|first|morning|breakfast| +30054|AAAAAAAAHGFHAAAA|30054|8|20|54|AM|first|morning|breakfast| +30055|AAAAAAAAIGFHAAAA|30055|8|20|55|AM|first|morning|breakfast| +30056|AAAAAAAAJGFHAAAA|30056|8|20|56|AM|first|morning|breakfast| +30057|AAAAAAAAKGFHAAAA|30057|8|20|57|AM|first|morning|breakfast| +30058|AAAAAAAALGFHAAAA|30058|8|20|58|AM|first|morning|breakfast| +30059|AAAAAAAAMGFHAAAA|30059|8|20|59|AM|first|morning|breakfast| +30060|AAAAAAAANGFHAAAA|30060|8|21|0|AM|first|morning|breakfast| +30061|AAAAAAAAOGFHAAAA|30061|8|21|1|AM|first|morning|breakfast| +30062|AAAAAAAAPGFHAAAA|30062|8|21|2|AM|first|morning|breakfast| +30063|AAAAAAAAAHFHAAAA|30063|8|21|3|AM|first|morning|breakfast| +30064|AAAAAAAABHFHAAAA|30064|8|21|4|AM|first|morning|breakfast| +30065|AAAAAAAACHFHAAAA|30065|8|21|5|AM|first|morning|breakfast| +30066|AAAAAAAADHFHAAAA|30066|8|21|6|AM|first|morning|breakfast| +30067|AAAAAAAAEHFHAAAA|30067|8|21|7|AM|first|morning|breakfast| +30068|AAAAAAAAFHFHAAAA|30068|8|21|8|AM|first|morning|breakfast| +30069|AAAAAAAAGHFHAAAA|30069|8|21|9|AM|first|morning|breakfast| +30070|AAAAAAAAHHFHAAAA|30070|8|21|10|AM|first|morning|breakfast| +30071|AAAAAAAAIHFHAAAA|30071|8|21|11|AM|first|morning|breakfast| +30072|AAAAAAAAJHFHAAAA|30072|8|21|12|AM|first|morning|breakfast| +30073|AAAAAAAAKHFHAAAA|30073|8|21|13|AM|first|morning|breakfast| +30074|AAAAAAAALHFHAAAA|30074|8|21|14|AM|first|morning|breakfast| +30075|AAAAAAAAMHFHAAAA|30075|8|21|15|AM|first|morning|breakfast| +30076|AAAAAAAANHFHAAAA|30076|8|21|16|AM|first|morning|breakfast| +30077|AAAAAAAAOHFHAAAA|30077|8|21|17|AM|first|morning|breakfast| +30078|AAAAAAAAPHFHAAAA|30078|8|21|18|AM|first|morning|breakfast| +30079|AAAAAAAAAIFHAAAA|30079|8|21|19|AM|first|morning|breakfast| +30080|AAAAAAAABIFHAAAA|30080|8|21|20|AM|first|morning|breakfast| +30081|AAAAAAAACIFHAAAA|30081|8|21|21|AM|first|morning|breakfast| +30082|AAAAAAAADIFHAAAA|30082|8|21|22|AM|first|morning|breakfast| +30083|AAAAAAAAEIFHAAAA|30083|8|21|23|AM|first|morning|breakfast| +30084|AAAAAAAAFIFHAAAA|30084|8|21|24|AM|first|morning|breakfast| +30085|AAAAAAAAGIFHAAAA|30085|8|21|25|AM|first|morning|breakfast| +30086|AAAAAAAAHIFHAAAA|30086|8|21|26|AM|first|morning|breakfast| +30087|AAAAAAAAIIFHAAAA|30087|8|21|27|AM|first|morning|breakfast| +30088|AAAAAAAAJIFHAAAA|30088|8|21|28|AM|first|morning|breakfast| +30089|AAAAAAAAKIFHAAAA|30089|8|21|29|AM|first|morning|breakfast| +30090|AAAAAAAALIFHAAAA|30090|8|21|30|AM|first|morning|breakfast| +30091|AAAAAAAAMIFHAAAA|30091|8|21|31|AM|first|morning|breakfast| +30092|AAAAAAAANIFHAAAA|30092|8|21|32|AM|first|morning|breakfast| +30093|AAAAAAAAOIFHAAAA|30093|8|21|33|AM|first|morning|breakfast| +30094|AAAAAAAAPIFHAAAA|30094|8|21|34|AM|first|morning|breakfast| +30095|AAAAAAAAAJFHAAAA|30095|8|21|35|AM|first|morning|breakfast| +30096|AAAAAAAABJFHAAAA|30096|8|21|36|AM|first|morning|breakfast| +30097|AAAAAAAACJFHAAAA|30097|8|21|37|AM|first|morning|breakfast| +30098|AAAAAAAADJFHAAAA|30098|8|21|38|AM|first|morning|breakfast| +30099|AAAAAAAAEJFHAAAA|30099|8|21|39|AM|first|morning|breakfast| +30100|AAAAAAAAFJFHAAAA|30100|8|21|40|AM|first|morning|breakfast| +30101|AAAAAAAAGJFHAAAA|30101|8|21|41|AM|first|morning|breakfast| +30102|AAAAAAAAHJFHAAAA|30102|8|21|42|AM|first|morning|breakfast| +30103|AAAAAAAAIJFHAAAA|30103|8|21|43|AM|first|morning|breakfast| +30104|AAAAAAAAJJFHAAAA|30104|8|21|44|AM|first|morning|breakfast| +30105|AAAAAAAAKJFHAAAA|30105|8|21|45|AM|first|morning|breakfast| +30106|AAAAAAAALJFHAAAA|30106|8|21|46|AM|first|morning|breakfast| +30107|AAAAAAAAMJFHAAAA|30107|8|21|47|AM|first|morning|breakfast| +30108|AAAAAAAANJFHAAAA|30108|8|21|48|AM|first|morning|breakfast| +30109|AAAAAAAAOJFHAAAA|30109|8|21|49|AM|first|morning|breakfast| +30110|AAAAAAAAPJFHAAAA|30110|8|21|50|AM|first|morning|breakfast| +30111|AAAAAAAAAKFHAAAA|30111|8|21|51|AM|first|morning|breakfast| +30112|AAAAAAAABKFHAAAA|30112|8|21|52|AM|first|morning|breakfast| +30113|AAAAAAAACKFHAAAA|30113|8|21|53|AM|first|morning|breakfast| +30114|AAAAAAAADKFHAAAA|30114|8|21|54|AM|first|morning|breakfast| +30115|AAAAAAAAEKFHAAAA|30115|8|21|55|AM|first|morning|breakfast| +30116|AAAAAAAAFKFHAAAA|30116|8|21|56|AM|first|morning|breakfast| +30117|AAAAAAAAGKFHAAAA|30117|8|21|57|AM|first|morning|breakfast| +30118|AAAAAAAAHKFHAAAA|30118|8|21|58|AM|first|morning|breakfast| +30119|AAAAAAAAIKFHAAAA|30119|8|21|59|AM|first|morning|breakfast| +30120|AAAAAAAAJKFHAAAA|30120|8|22|0|AM|first|morning|breakfast| +30121|AAAAAAAAKKFHAAAA|30121|8|22|1|AM|first|morning|breakfast| +30122|AAAAAAAALKFHAAAA|30122|8|22|2|AM|first|morning|breakfast| +30123|AAAAAAAAMKFHAAAA|30123|8|22|3|AM|first|morning|breakfast| +30124|AAAAAAAANKFHAAAA|30124|8|22|4|AM|first|morning|breakfast| +30125|AAAAAAAAOKFHAAAA|30125|8|22|5|AM|first|morning|breakfast| +30126|AAAAAAAAPKFHAAAA|30126|8|22|6|AM|first|morning|breakfast| +30127|AAAAAAAAALFHAAAA|30127|8|22|7|AM|first|morning|breakfast| +30128|AAAAAAAABLFHAAAA|30128|8|22|8|AM|first|morning|breakfast| +30129|AAAAAAAACLFHAAAA|30129|8|22|9|AM|first|morning|breakfast| +30130|AAAAAAAADLFHAAAA|30130|8|22|10|AM|first|morning|breakfast| +30131|AAAAAAAAELFHAAAA|30131|8|22|11|AM|first|morning|breakfast| +30132|AAAAAAAAFLFHAAAA|30132|8|22|12|AM|first|morning|breakfast| +30133|AAAAAAAAGLFHAAAA|30133|8|22|13|AM|first|morning|breakfast| +30134|AAAAAAAAHLFHAAAA|30134|8|22|14|AM|first|morning|breakfast| +30135|AAAAAAAAILFHAAAA|30135|8|22|15|AM|first|morning|breakfast| +30136|AAAAAAAAJLFHAAAA|30136|8|22|16|AM|first|morning|breakfast| +30137|AAAAAAAAKLFHAAAA|30137|8|22|17|AM|first|morning|breakfast| +30138|AAAAAAAALLFHAAAA|30138|8|22|18|AM|first|morning|breakfast| +30139|AAAAAAAAMLFHAAAA|30139|8|22|19|AM|first|morning|breakfast| +30140|AAAAAAAANLFHAAAA|30140|8|22|20|AM|first|morning|breakfast| +30141|AAAAAAAAOLFHAAAA|30141|8|22|21|AM|first|morning|breakfast| +30142|AAAAAAAAPLFHAAAA|30142|8|22|22|AM|first|morning|breakfast| +30143|AAAAAAAAAMFHAAAA|30143|8|22|23|AM|first|morning|breakfast| +30144|AAAAAAAABMFHAAAA|30144|8|22|24|AM|first|morning|breakfast| +30145|AAAAAAAACMFHAAAA|30145|8|22|25|AM|first|morning|breakfast| +30146|AAAAAAAADMFHAAAA|30146|8|22|26|AM|first|morning|breakfast| +30147|AAAAAAAAEMFHAAAA|30147|8|22|27|AM|first|morning|breakfast| +30148|AAAAAAAAFMFHAAAA|30148|8|22|28|AM|first|morning|breakfast| +30149|AAAAAAAAGMFHAAAA|30149|8|22|29|AM|first|morning|breakfast| +30150|AAAAAAAAHMFHAAAA|30150|8|22|30|AM|first|morning|breakfast| +30151|AAAAAAAAIMFHAAAA|30151|8|22|31|AM|first|morning|breakfast| +30152|AAAAAAAAJMFHAAAA|30152|8|22|32|AM|first|morning|breakfast| +30153|AAAAAAAAKMFHAAAA|30153|8|22|33|AM|first|morning|breakfast| +30154|AAAAAAAALMFHAAAA|30154|8|22|34|AM|first|morning|breakfast| +30155|AAAAAAAAMMFHAAAA|30155|8|22|35|AM|first|morning|breakfast| +30156|AAAAAAAANMFHAAAA|30156|8|22|36|AM|first|morning|breakfast| +30157|AAAAAAAAOMFHAAAA|30157|8|22|37|AM|first|morning|breakfast| +30158|AAAAAAAAPMFHAAAA|30158|8|22|38|AM|first|morning|breakfast| +30159|AAAAAAAAANFHAAAA|30159|8|22|39|AM|first|morning|breakfast| +30160|AAAAAAAABNFHAAAA|30160|8|22|40|AM|first|morning|breakfast| +30161|AAAAAAAACNFHAAAA|30161|8|22|41|AM|first|morning|breakfast| +30162|AAAAAAAADNFHAAAA|30162|8|22|42|AM|first|morning|breakfast| +30163|AAAAAAAAENFHAAAA|30163|8|22|43|AM|first|morning|breakfast| +30164|AAAAAAAAFNFHAAAA|30164|8|22|44|AM|first|morning|breakfast| +30165|AAAAAAAAGNFHAAAA|30165|8|22|45|AM|first|morning|breakfast| +30166|AAAAAAAAHNFHAAAA|30166|8|22|46|AM|first|morning|breakfast| +30167|AAAAAAAAINFHAAAA|30167|8|22|47|AM|first|morning|breakfast| +30168|AAAAAAAAJNFHAAAA|30168|8|22|48|AM|first|morning|breakfast| +30169|AAAAAAAAKNFHAAAA|30169|8|22|49|AM|first|morning|breakfast| +30170|AAAAAAAALNFHAAAA|30170|8|22|50|AM|first|morning|breakfast| +30171|AAAAAAAAMNFHAAAA|30171|8|22|51|AM|first|morning|breakfast| +30172|AAAAAAAANNFHAAAA|30172|8|22|52|AM|first|morning|breakfast| +30173|AAAAAAAAONFHAAAA|30173|8|22|53|AM|first|morning|breakfast| +30174|AAAAAAAAPNFHAAAA|30174|8|22|54|AM|first|morning|breakfast| +30175|AAAAAAAAAOFHAAAA|30175|8|22|55|AM|first|morning|breakfast| +30176|AAAAAAAABOFHAAAA|30176|8|22|56|AM|first|morning|breakfast| +30177|AAAAAAAACOFHAAAA|30177|8|22|57|AM|first|morning|breakfast| +30178|AAAAAAAADOFHAAAA|30178|8|22|58|AM|first|morning|breakfast| +30179|AAAAAAAAEOFHAAAA|30179|8|22|59|AM|first|morning|breakfast| +30180|AAAAAAAAFOFHAAAA|30180|8|23|0|AM|first|morning|breakfast| +30181|AAAAAAAAGOFHAAAA|30181|8|23|1|AM|first|morning|breakfast| +30182|AAAAAAAAHOFHAAAA|30182|8|23|2|AM|first|morning|breakfast| +30183|AAAAAAAAIOFHAAAA|30183|8|23|3|AM|first|morning|breakfast| +30184|AAAAAAAAJOFHAAAA|30184|8|23|4|AM|first|morning|breakfast| +30185|AAAAAAAAKOFHAAAA|30185|8|23|5|AM|first|morning|breakfast| +30186|AAAAAAAALOFHAAAA|30186|8|23|6|AM|first|morning|breakfast| +30187|AAAAAAAAMOFHAAAA|30187|8|23|7|AM|first|morning|breakfast| +30188|AAAAAAAANOFHAAAA|30188|8|23|8|AM|first|morning|breakfast| +30189|AAAAAAAAOOFHAAAA|30189|8|23|9|AM|first|morning|breakfast| +30190|AAAAAAAAPOFHAAAA|30190|8|23|10|AM|first|morning|breakfast| +30191|AAAAAAAAAPFHAAAA|30191|8|23|11|AM|first|morning|breakfast| +30192|AAAAAAAABPFHAAAA|30192|8|23|12|AM|first|morning|breakfast| +30193|AAAAAAAACPFHAAAA|30193|8|23|13|AM|first|morning|breakfast| +30194|AAAAAAAADPFHAAAA|30194|8|23|14|AM|first|morning|breakfast| +30195|AAAAAAAAEPFHAAAA|30195|8|23|15|AM|first|morning|breakfast| +30196|AAAAAAAAFPFHAAAA|30196|8|23|16|AM|first|morning|breakfast| +30197|AAAAAAAAGPFHAAAA|30197|8|23|17|AM|first|morning|breakfast| +30198|AAAAAAAAHPFHAAAA|30198|8|23|18|AM|first|morning|breakfast| +30199|AAAAAAAAIPFHAAAA|30199|8|23|19|AM|first|morning|breakfast| +30200|AAAAAAAAJPFHAAAA|30200|8|23|20|AM|first|morning|breakfast| +30201|AAAAAAAAKPFHAAAA|30201|8|23|21|AM|first|morning|breakfast| +30202|AAAAAAAALPFHAAAA|30202|8|23|22|AM|first|morning|breakfast| +30203|AAAAAAAAMPFHAAAA|30203|8|23|23|AM|first|morning|breakfast| +30204|AAAAAAAANPFHAAAA|30204|8|23|24|AM|first|morning|breakfast| +30205|AAAAAAAAOPFHAAAA|30205|8|23|25|AM|first|morning|breakfast| +30206|AAAAAAAAPPFHAAAA|30206|8|23|26|AM|first|morning|breakfast| +30207|AAAAAAAAAAGHAAAA|30207|8|23|27|AM|first|morning|breakfast| +30208|AAAAAAAABAGHAAAA|30208|8|23|28|AM|first|morning|breakfast| +30209|AAAAAAAACAGHAAAA|30209|8|23|29|AM|first|morning|breakfast| +30210|AAAAAAAADAGHAAAA|30210|8|23|30|AM|first|morning|breakfast| +30211|AAAAAAAAEAGHAAAA|30211|8|23|31|AM|first|morning|breakfast| +30212|AAAAAAAAFAGHAAAA|30212|8|23|32|AM|first|morning|breakfast| +30213|AAAAAAAAGAGHAAAA|30213|8|23|33|AM|first|morning|breakfast| +30214|AAAAAAAAHAGHAAAA|30214|8|23|34|AM|first|morning|breakfast| +30215|AAAAAAAAIAGHAAAA|30215|8|23|35|AM|first|morning|breakfast| +30216|AAAAAAAAJAGHAAAA|30216|8|23|36|AM|first|morning|breakfast| +30217|AAAAAAAAKAGHAAAA|30217|8|23|37|AM|first|morning|breakfast| +30218|AAAAAAAALAGHAAAA|30218|8|23|38|AM|first|morning|breakfast| +30219|AAAAAAAAMAGHAAAA|30219|8|23|39|AM|first|morning|breakfast| +30220|AAAAAAAANAGHAAAA|30220|8|23|40|AM|first|morning|breakfast| +30221|AAAAAAAAOAGHAAAA|30221|8|23|41|AM|first|morning|breakfast| +30222|AAAAAAAAPAGHAAAA|30222|8|23|42|AM|first|morning|breakfast| +30223|AAAAAAAAABGHAAAA|30223|8|23|43|AM|first|morning|breakfast| +30224|AAAAAAAABBGHAAAA|30224|8|23|44|AM|first|morning|breakfast| +30225|AAAAAAAACBGHAAAA|30225|8|23|45|AM|first|morning|breakfast| +30226|AAAAAAAADBGHAAAA|30226|8|23|46|AM|first|morning|breakfast| +30227|AAAAAAAAEBGHAAAA|30227|8|23|47|AM|first|morning|breakfast| +30228|AAAAAAAAFBGHAAAA|30228|8|23|48|AM|first|morning|breakfast| +30229|AAAAAAAAGBGHAAAA|30229|8|23|49|AM|first|morning|breakfast| +30230|AAAAAAAAHBGHAAAA|30230|8|23|50|AM|first|morning|breakfast| +30231|AAAAAAAAIBGHAAAA|30231|8|23|51|AM|first|morning|breakfast| +30232|AAAAAAAAJBGHAAAA|30232|8|23|52|AM|first|morning|breakfast| +30233|AAAAAAAAKBGHAAAA|30233|8|23|53|AM|first|morning|breakfast| +30234|AAAAAAAALBGHAAAA|30234|8|23|54|AM|first|morning|breakfast| +30235|AAAAAAAAMBGHAAAA|30235|8|23|55|AM|first|morning|breakfast| +30236|AAAAAAAANBGHAAAA|30236|8|23|56|AM|first|morning|breakfast| +30237|AAAAAAAAOBGHAAAA|30237|8|23|57|AM|first|morning|breakfast| +30238|AAAAAAAAPBGHAAAA|30238|8|23|58|AM|first|morning|breakfast| +30239|AAAAAAAAACGHAAAA|30239|8|23|59|AM|first|morning|breakfast| +30240|AAAAAAAABCGHAAAA|30240|8|24|0|AM|first|morning|breakfast| +30241|AAAAAAAACCGHAAAA|30241|8|24|1|AM|first|morning|breakfast| +30242|AAAAAAAADCGHAAAA|30242|8|24|2|AM|first|morning|breakfast| +30243|AAAAAAAAECGHAAAA|30243|8|24|3|AM|first|morning|breakfast| +30244|AAAAAAAAFCGHAAAA|30244|8|24|4|AM|first|morning|breakfast| +30245|AAAAAAAAGCGHAAAA|30245|8|24|5|AM|first|morning|breakfast| +30246|AAAAAAAAHCGHAAAA|30246|8|24|6|AM|first|morning|breakfast| +30247|AAAAAAAAICGHAAAA|30247|8|24|7|AM|first|morning|breakfast| +30248|AAAAAAAAJCGHAAAA|30248|8|24|8|AM|first|morning|breakfast| +30249|AAAAAAAAKCGHAAAA|30249|8|24|9|AM|first|morning|breakfast| +30250|AAAAAAAALCGHAAAA|30250|8|24|10|AM|first|morning|breakfast| +30251|AAAAAAAAMCGHAAAA|30251|8|24|11|AM|first|morning|breakfast| +30252|AAAAAAAANCGHAAAA|30252|8|24|12|AM|first|morning|breakfast| +30253|AAAAAAAAOCGHAAAA|30253|8|24|13|AM|first|morning|breakfast| +30254|AAAAAAAAPCGHAAAA|30254|8|24|14|AM|first|morning|breakfast| +30255|AAAAAAAAADGHAAAA|30255|8|24|15|AM|first|morning|breakfast| +30256|AAAAAAAABDGHAAAA|30256|8|24|16|AM|first|morning|breakfast| +30257|AAAAAAAACDGHAAAA|30257|8|24|17|AM|first|morning|breakfast| +30258|AAAAAAAADDGHAAAA|30258|8|24|18|AM|first|morning|breakfast| +30259|AAAAAAAAEDGHAAAA|30259|8|24|19|AM|first|morning|breakfast| +30260|AAAAAAAAFDGHAAAA|30260|8|24|20|AM|first|morning|breakfast| +30261|AAAAAAAAGDGHAAAA|30261|8|24|21|AM|first|morning|breakfast| +30262|AAAAAAAAHDGHAAAA|30262|8|24|22|AM|first|morning|breakfast| +30263|AAAAAAAAIDGHAAAA|30263|8|24|23|AM|first|morning|breakfast| +30264|AAAAAAAAJDGHAAAA|30264|8|24|24|AM|first|morning|breakfast| +30265|AAAAAAAAKDGHAAAA|30265|8|24|25|AM|first|morning|breakfast| +30266|AAAAAAAALDGHAAAA|30266|8|24|26|AM|first|morning|breakfast| +30267|AAAAAAAAMDGHAAAA|30267|8|24|27|AM|first|morning|breakfast| +30268|AAAAAAAANDGHAAAA|30268|8|24|28|AM|first|morning|breakfast| +30269|AAAAAAAAODGHAAAA|30269|8|24|29|AM|first|morning|breakfast| +30270|AAAAAAAAPDGHAAAA|30270|8|24|30|AM|first|morning|breakfast| +30271|AAAAAAAAAEGHAAAA|30271|8|24|31|AM|first|morning|breakfast| +30272|AAAAAAAABEGHAAAA|30272|8|24|32|AM|first|morning|breakfast| +30273|AAAAAAAACEGHAAAA|30273|8|24|33|AM|first|morning|breakfast| +30274|AAAAAAAADEGHAAAA|30274|8|24|34|AM|first|morning|breakfast| +30275|AAAAAAAAEEGHAAAA|30275|8|24|35|AM|first|morning|breakfast| +30276|AAAAAAAAFEGHAAAA|30276|8|24|36|AM|first|morning|breakfast| +30277|AAAAAAAAGEGHAAAA|30277|8|24|37|AM|first|morning|breakfast| +30278|AAAAAAAAHEGHAAAA|30278|8|24|38|AM|first|morning|breakfast| +30279|AAAAAAAAIEGHAAAA|30279|8|24|39|AM|first|morning|breakfast| +30280|AAAAAAAAJEGHAAAA|30280|8|24|40|AM|first|morning|breakfast| +30281|AAAAAAAAKEGHAAAA|30281|8|24|41|AM|first|morning|breakfast| +30282|AAAAAAAALEGHAAAA|30282|8|24|42|AM|first|morning|breakfast| +30283|AAAAAAAAMEGHAAAA|30283|8|24|43|AM|first|morning|breakfast| +30284|AAAAAAAANEGHAAAA|30284|8|24|44|AM|first|morning|breakfast| +30285|AAAAAAAAOEGHAAAA|30285|8|24|45|AM|first|morning|breakfast| +30286|AAAAAAAAPEGHAAAA|30286|8|24|46|AM|first|morning|breakfast| +30287|AAAAAAAAAFGHAAAA|30287|8|24|47|AM|first|morning|breakfast| +30288|AAAAAAAABFGHAAAA|30288|8|24|48|AM|first|morning|breakfast| +30289|AAAAAAAACFGHAAAA|30289|8|24|49|AM|first|morning|breakfast| +30290|AAAAAAAADFGHAAAA|30290|8|24|50|AM|first|morning|breakfast| +30291|AAAAAAAAEFGHAAAA|30291|8|24|51|AM|first|morning|breakfast| +30292|AAAAAAAAFFGHAAAA|30292|8|24|52|AM|first|morning|breakfast| +30293|AAAAAAAAGFGHAAAA|30293|8|24|53|AM|first|morning|breakfast| +30294|AAAAAAAAHFGHAAAA|30294|8|24|54|AM|first|morning|breakfast| +30295|AAAAAAAAIFGHAAAA|30295|8|24|55|AM|first|morning|breakfast| +30296|AAAAAAAAJFGHAAAA|30296|8|24|56|AM|first|morning|breakfast| +30297|AAAAAAAAKFGHAAAA|30297|8|24|57|AM|first|morning|breakfast| +30298|AAAAAAAALFGHAAAA|30298|8|24|58|AM|first|morning|breakfast| +30299|AAAAAAAAMFGHAAAA|30299|8|24|59|AM|first|morning|breakfast| +30300|AAAAAAAANFGHAAAA|30300|8|25|0|AM|first|morning|breakfast| +30301|AAAAAAAAOFGHAAAA|30301|8|25|1|AM|first|morning|breakfast| +30302|AAAAAAAAPFGHAAAA|30302|8|25|2|AM|first|morning|breakfast| +30303|AAAAAAAAAGGHAAAA|30303|8|25|3|AM|first|morning|breakfast| +30304|AAAAAAAABGGHAAAA|30304|8|25|4|AM|first|morning|breakfast| +30305|AAAAAAAACGGHAAAA|30305|8|25|5|AM|first|morning|breakfast| +30306|AAAAAAAADGGHAAAA|30306|8|25|6|AM|first|morning|breakfast| +30307|AAAAAAAAEGGHAAAA|30307|8|25|7|AM|first|morning|breakfast| +30308|AAAAAAAAFGGHAAAA|30308|8|25|8|AM|first|morning|breakfast| +30309|AAAAAAAAGGGHAAAA|30309|8|25|9|AM|first|morning|breakfast| +30310|AAAAAAAAHGGHAAAA|30310|8|25|10|AM|first|morning|breakfast| +30311|AAAAAAAAIGGHAAAA|30311|8|25|11|AM|first|morning|breakfast| +30312|AAAAAAAAJGGHAAAA|30312|8|25|12|AM|first|morning|breakfast| +30313|AAAAAAAAKGGHAAAA|30313|8|25|13|AM|first|morning|breakfast| +30314|AAAAAAAALGGHAAAA|30314|8|25|14|AM|first|morning|breakfast| +30315|AAAAAAAAMGGHAAAA|30315|8|25|15|AM|first|morning|breakfast| +30316|AAAAAAAANGGHAAAA|30316|8|25|16|AM|first|morning|breakfast| +30317|AAAAAAAAOGGHAAAA|30317|8|25|17|AM|first|morning|breakfast| +30318|AAAAAAAAPGGHAAAA|30318|8|25|18|AM|first|morning|breakfast| +30319|AAAAAAAAAHGHAAAA|30319|8|25|19|AM|first|morning|breakfast| +30320|AAAAAAAABHGHAAAA|30320|8|25|20|AM|first|morning|breakfast| +30321|AAAAAAAACHGHAAAA|30321|8|25|21|AM|first|morning|breakfast| +30322|AAAAAAAADHGHAAAA|30322|8|25|22|AM|first|morning|breakfast| +30323|AAAAAAAAEHGHAAAA|30323|8|25|23|AM|first|morning|breakfast| +30324|AAAAAAAAFHGHAAAA|30324|8|25|24|AM|first|morning|breakfast| +30325|AAAAAAAAGHGHAAAA|30325|8|25|25|AM|first|morning|breakfast| +30326|AAAAAAAAHHGHAAAA|30326|8|25|26|AM|first|morning|breakfast| +30327|AAAAAAAAIHGHAAAA|30327|8|25|27|AM|first|morning|breakfast| +30328|AAAAAAAAJHGHAAAA|30328|8|25|28|AM|first|morning|breakfast| +30329|AAAAAAAAKHGHAAAA|30329|8|25|29|AM|first|morning|breakfast| +30330|AAAAAAAALHGHAAAA|30330|8|25|30|AM|first|morning|breakfast| +30331|AAAAAAAAMHGHAAAA|30331|8|25|31|AM|first|morning|breakfast| +30332|AAAAAAAANHGHAAAA|30332|8|25|32|AM|first|morning|breakfast| +30333|AAAAAAAAOHGHAAAA|30333|8|25|33|AM|first|morning|breakfast| +30334|AAAAAAAAPHGHAAAA|30334|8|25|34|AM|first|morning|breakfast| +30335|AAAAAAAAAIGHAAAA|30335|8|25|35|AM|first|morning|breakfast| +30336|AAAAAAAABIGHAAAA|30336|8|25|36|AM|first|morning|breakfast| +30337|AAAAAAAACIGHAAAA|30337|8|25|37|AM|first|morning|breakfast| +30338|AAAAAAAADIGHAAAA|30338|8|25|38|AM|first|morning|breakfast| +30339|AAAAAAAAEIGHAAAA|30339|8|25|39|AM|first|morning|breakfast| +30340|AAAAAAAAFIGHAAAA|30340|8|25|40|AM|first|morning|breakfast| +30341|AAAAAAAAGIGHAAAA|30341|8|25|41|AM|first|morning|breakfast| +30342|AAAAAAAAHIGHAAAA|30342|8|25|42|AM|first|morning|breakfast| +30343|AAAAAAAAIIGHAAAA|30343|8|25|43|AM|first|morning|breakfast| +30344|AAAAAAAAJIGHAAAA|30344|8|25|44|AM|first|morning|breakfast| +30345|AAAAAAAAKIGHAAAA|30345|8|25|45|AM|first|morning|breakfast| +30346|AAAAAAAALIGHAAAA|30346|8|25|46|AM|first|morning|breakfast| +30347|AAAAAAAAMIGHAAAA|30347|8|25|47|AM|first|morning|breakfast| +30348|AAAAAAAANIGHAAAA|30348|8|25|48|AM|first|morning|breakfast| +30349|AAAAAAAAOIGHAAAA|30349|8|25|49|AM|first|morning|breakfast| +30350|AAAAAAAAPIGHAAAA|30350|8|25|50|AM|first|morning|breakfast| +30351|AAAAAAAAAJGHAAAA|30351|8|25|51|AM|first|morning|breakfast| +30352|AAAAAAAABJGHAAAA|30352|8|25|52|AM|first|morning|breakfast| +30353|AAAAAAAACJGHAAAA|30353|8|25|53|AM|first|morning|breakfast| +30354|AAAAAAAADJGHAAAA|30354|8|25|54|AM|first|morning|breakfast| +30355|AAAAAAAAEJGHAAAA|30355|8|25|55|AM|first|morning|breakfast| +30356|AAAAAAAAFJGHAAAA|30356|8|25|56|AM|first|morning|breakfast| +30357|AAAAAAAAGJGHAAAA|30357|8|25|57|AM|first|morning|breakfast| +30358|AAAAAAAAHJGHAAAA|30358|8|25|58|AM|first|morning|breakfast| +30359|AAAAAAAAIJGHAAAA|30359|8|25|59|AM|first|morning|breakfast| +30360|AAAAAAAAJJGHAAAA|30360|8|26|0|AM|first|morning|breakfast| +30361|AAAAAAAAKJGHAAAA|30361|8|26|1|AM|first|morning|breakfast| +30362|AAAAAAAALJGHAAAA|30362|8|26|2|AM|first|morning|breakfast| +30363|AAAAAAAAMJGHAAAA|30363|8|26|3|AM|first|morning|breakfast| +30364|AAAAAAAANJGHAAAA|30364|8|26|4|AM|first|morning|breakfast| +30365|AAAAAAAAOJGHAAAA|30365|8|26|5|AM|first|morning|breakfast| +30366|AAAAAAAAPJGHAAAA|30366|8|26|6|AM|first|morning|breakfast| +30367|AAAAAAAAAKGHAAAA|30367|8|26|7|AM|first|morning|breakfast| +30368|AAAAAAAABKGHAAAA|30368|8|26|8|AM|first|morning|breakfast| +30369|AAAAAAAACKGHAAAA|30369|8|26|9|AM|first|morning|breakfast| +30370|AAAAAAAADKGHAAAA|30370|8|26|10|AM|first|morning|breakfast| +30371|AAAAAAAAEKGHAAAA|30371|8|26|11|AM|first|morning|breakfast| +30372|AAAAAAAAFKGHAAAA|30372|8|26|12|AM|first|morning|breakfast| +30373|AAAAAAAAGKGHAAAA|30373|8|26|13|AM|first|morning|breakfast| +30374|AAAAAAAAHKGHAAAA|30374|8|26|14|AM|first|morning|breakfast| +30375|AAAAAAAAIKGHAAAA|30375|8|26|15|AM|first|morning|breakfast| +30376|AAAAAAAAJKGHAAAA|30376|8|26|16|AM|first|morning|breakfast| +30377|AAAAAAAAKKGHAAAA|30377|8|26|17|AM|first|morning|breakfast| +30378|AAAAAAAALKGHAAAA|30378|8|26|18|AM|first|morning|breakfast| +30379|AAAAAAAAMKGHAAAA|30379|8|26|19|AM|first|morning|breakfast| +30380|AAAAAAAANKGHAAAA|30380|8|26|20|AM|first|morning|breakfast| +30381|AAAAAAAAOKGHAAAA|30381|8|26|21|AM|first|morning|breakfast| +30382|AAAAAAAAPKGHAAAA|30382|8|26|22|AM|first|morning|breakfast| +30383|AAAAAAAAALGHAAAA|30383|8|26|23|AM|first|morning|breakfast| +30384|AAAAAAAABLGHAAAA|30384|8|26|24|AM|first|morning|breakfast| +30385|AAAAAAAACLGHAAAA|30385|8|26|25|AM|first|morning|breakfast| +30386|AAAAAAAADLGHAAAA|30386|8|26|26|AM|first|morning|breakfast| +30387|AAAAAAAAELGHAAAA|30387|8|26|27|AM|first|morning|breakfast| +30388|AAAAAAAAFLGHAAAA|30388|8|26|28|AM|first|morning|breakfast| +30389|AAAAAAAAGLGHAAAA|30389|8|26|29|AM|first|morning|breakfast| +30390|AAAAAAAAHLGHAAAA|30390|8|26|30|AM|first|morning|breakfast| +30391|AAAAAAAAILGHAAAA|30391|8|26|31|AM|first|morning|breakfast| +30392|AAAAAAAAJLGHAAAA|30392|8|26|32|AM|first|morning|breakfast| +30393|AAAAAAAAKLGHAAAA|30393|8|26|33|AM|first|morning|breakfast| +30394|AAAAAAAALLGHAAAA|30394|8|26|34|AM|first|morning|breakfast| +30395|AAAAAAAAMLGHAAAA|30395|8|26|35|AM|first|morning|breakfast| +30396|AAAAAAAANLGHAAAA|30396|8|26|36|AM|first|morning|breakfast| +30397|AAAAAAAAOLGHAAAA|30397|8|26|37|AM|first|morning|breakfast| +30398|AAAAAAAAPLGHAAAA|30398|8|26|38|AM|first|morning|breakfast| +30399|AAAAAAAAAMGHAAAA|30399|8|26|39|AM|first|morning|breakfast| +30400|AAAAAAAABMGHAAAA|30400|8|26|40|AM|first|morning|breakfast| +30401|AAAAAAAACMGHAAAA|30401|8|26|41|AM|first|morning|breakfast| +30402|AAAAAAAADMGHAAAA|30402|8|26|42|AM|first|morning|breakfast| +30403|AAAAAAAAEMGHAAAA|30403|8|26|43|AM|first|morning|breakfast| +30404|AAAAAAAAFMGHAAAA|30404|8|26|44|AM|first|morning|breakfast| +30405|AAAAAAAAGMGHAAAA|30405|8|26|45|AM|first|morning|breakfast| +30406|AAAAAAAAHMGHAAAA|30406|8|26|46|AM|first|morning|breakfast| +30407|AAAAAAAAIMGHAAAA|30407|8|26|47|AM|first|morning|breakfast| +30408|AAAAAAAAJMGHAAAA|30408|8|26|48|AM|first|morning|breakfast| +30409|AAAAAAAAKMGHAAAA|30409|8|26|49|AM|first|morning|breakfast| +30410|AAAAAAAALMGHAAAA|30410|8|26|50|AM|first|morning|breakfast| +30411|AAAAAAAAMMGHAAAA|30411|8|26|51|AM|first|morning|breakfast| +30412|AAAAAAAANMGHAAAA|30412|8|26|52|AM|first|morning|breakfast| +30413|AAAAAAAAOMGHAAAA|30413|8|26|53|AM|first|morning|breakfast| +30414|AAAAAAAAPMGHAAAA|30414|8|26|54|AM|first|morning|breakfast| +30415|AAAAAAAAANGHAAAA|30415|8|26|55|AM|first|morning|breakfast| +30416|AAAAAAAABNGHAAAA|30416|8|26|56|AM|first|morning|breakfast| +30417|AAAAAAAACNGHAAAA|30417|8|26|57|AM|first|morning|breakfast| +30418|AAAAAAAADNGHAAAA|30418|8|26|58|AM|first|morning|breakfast| +30419|AAAAAAAAENGHAAAA|30419|8|26|59|AM|first|morning|breakfast| +30420|AAAAAAAAFNGHAAAA|30420|8|27|0|AM|first|morning|breakfast| +30421|AAAAAAAAGNGHAAAA|30421|8|27|1|AM|first|morning|breakfast| +30422|AAAAAAAAHNGHAAAA|30422|8|27|2|AM|first|morning|breakfast| +30423|AAAAAAAAINGHAAAA|30423|8|27|3|AM|first|morning|breakfast| +30424|AAAAAAAAJNGHAAAA|30424|8|27|4|AM|first|morning|breakfast| +30425|AAAAAAAAKNGHAAAA|30425|8|27|5|AM|first|morning|breakfast| +30426|AAAAAAAALNGHAAAA|30426|8|27|6|AM|first|morning|breakfast| +30427|AAAAAAAAMNGHAAAA|30427|8|27|7|AM|first|morning|breakfast| +30428|AAAAAAAANNGHAAAA|30428|8|27|8|AM|first|morning|breakfast| +30429|AAAAAAAAONGHAAAA|30429|8|27|9|AM|first|morning|breakfast| +30430|AAAAAAAAPNGHAAAA|30430|8|27|10|AM|first|morning|breakfast| +30431|AAAAAAAAAOGHAAAA|30431|8|27|11|AM|first|morning|breakfast| +30432|AAAAAAAABOGHAAAA|30432|8|27|12|AM|first|morning|breakfast| +30433|AAAAAAAACOGHAAAA|30433|8|27|13|AM|first|morning|breakfast| +30434|AAAAAAAADOGHAAAA|30434|8|27|14|AM|first|morning|breakfast| +30435|AAAAAAAAEOGHAAAA|30435|8|27|15|AM|first|morning|breakfast| +30436|AAAAAAAAFOGHAAAA|30436|8|27|16|AM|first|morning|breakfast| +30437|AAAAAAAAGOGHAAAA|30437|8|27|17|AM|first|morning|breakfast| +30438|AAAAAAAAHOGHAAAA|30438|8|27|18|AM|first|morning|breakfast| +30439|AAAAAAAAIOGHAAAA|30439|8|27|19|AM|first|morning|breakfast| +30440|AAAAAAAAJOGHAAAA|30440|8|27|20|AM|first|morning|breakfast| +30441|AAAAAAAAKOGHAAAA|30441|8|27|21|AM|first|morning|breakfast| +30442|AAAAAAAALOGHAAAA|30442|8|27|22|AM|first|morning|breakfast| +30443|AAAAAAAAMOGHAAAA|30443|8|27|23|AM|first|morning|breakfast| +30444|AAAAAAAANOGHAAAA|30444|8|27|24|AM|first|morning|breakfast| +30445|AAAAAAAAOOGHAAAA|30445|8|27|25|AM|first|morning|breakfast| +30446|AAAAAAAAPOGHAAAA|30446|8|27|26|AM|first|morning|breakfast| +30447|AAAAAAAAAPGHAAAA|30447|8|27|27|AM|first|morning|breakfast| +30448|AAAAAAAABPGHAAAA|30448|8|27|28|AM|first|morning|breakfast| +30449|AAAAAAAACPGHAAAA|30449|8|27|29|AM|first|morning|breakfast| +30450|AAAAAAAADPGHAAAA|30450|8|27|30|AM|first|morning|breakfast| +30451|AAAAAAAAEPGHAAAA|30451|8|27|31|AM|first|morning|breakfast| +30452|AAAAAAAAFPGHAAAA|30452|8|27|32|AM|first|morning|breakfast| +30453|AAAAAAAAGPGHAAAA|30453|8|27|33|AM|first|morning|breakfast| +30454|AAAAAAAAHPGHAAAA|30454|8|27|34|AM|first|morning|breakfast| +30455|AAAAAAAAIPGHAAAA|30455|8|27|35|AM|first|morning|breakfast| +30456|AAAAAAAAJPGHAAAA|30456|8|27|36|AM|first|morning|breakfast| +30457|AAAAAAAAKPGHAAAA|30457|8|27|37|AM|first|morning|breakfast| +30458|AAAAAAAALPGHAAAA|30458|8|27|38|AM|first|morning|breakfast| +30459|AAAAAAAAMPGHAAAA|30459|8|27|39|AM|first|morning|breakfast| +30460|AAAAAAAANPGHAAAA|30460|8|27|40|AM|first|morning|breakfast| +30461|AAAAAAAAOPGHAAAA|30461|8|27|41|AM|first|morning|breakfast| +30462|AAAAAAAAPPGHAAAA|30462|8|27|42|AM|first|morning|breakfast| +30463|AAAAAAAAAAHHAAAA|30463|8|27|43|AM|first|morning|breakfast| +30464|AAAAAAAABAHHAAAA|30464|8|27|44|AM|first|morning|breakfast| +30465|AAAAAAAACAHHAAAA|30465|8|27|45|AM|first|morning|breakfast| +30466|AAAAAAAADAHHAAAA|30466|8|27|46|AM|first|morning|breakfast| +30467|AAAAAAAAEAHHAAAA|30467|8|27|47|AM|first|morning|breakfast| +30468|AAAAAAAAFAHHAAAA|30468|8|27|48|AM|first|morning|breakfast| +30469|AAAAAAAAGAHHAAAA|30469|8|27|49|AM|first|morning|breakfast| +30470|AAAAAAAAHAHHAAAA|30470|8|27|50|AM|first|morning|breakfast| +30471|AAAAAAAAIAHHAAAA|30471|8|27|51|AM|first|morning|breakfast| +30472|AAAAAAAAJAHHAAAA|30472|8|27|52|AM|first|morning|breakfast| +30473|AAAAAAAAKAHHAAAA|30473|8|27|53|AM|first|morning|breakfast| +30474|AAAAAAAALAHHAAAA|30474|8|27|54|AM|first|morning|breakfast| +30475|AAAAAAAAMAHHAAAA|30475|8|27|55|AM|first|morning|breakfast| +30476|AAAAAAAANAHHAAAA|30476|8|27|56|AM|first|morning|breakfast| +30477|AAAAAAAAOAHHAAAA|30477|8|27|57|AM|first|morning|breakfast| +30478|AAAAAAAAPAHHAAAA|30478|8|27|58|AM|first|morning|breakfast| +30479|AAAAAAAAABHHAAAA|30479|8|27|59|AM|first|morning|breakfast| +30480|AAAAAAAABBHHAAAA|30480|8|28|0|AM|first|morning|breakfast| +30481|AAAAAAAACBHHAAAA|30481|8|28|1|AM|first|morning|breakfast| +30482|AAAAAAAADBHHAAAA|30482|8|28|2|AM|first|morning|breakfast| +30483|AAAAAAAAEBHHAAAA|30483|8|28|3|AM|first|morning|breakfast| +30484|AAAAAAAAFBHHAAAA|30484|8|28|4|AM|first|morning|breakfast| +30485|AAAAAAAAGBHHAAAA|30485|8|28|5|AM|first|morning|breakfast| +30486|AAAAAAAAHBHHAAAA|30486|8|28|6|AM|first|morning|breakfast| +30487|AAAAAAAAIBHHAAAA|30487|8|28|7|AM|first|morning|breakfast| +30488|AAAAAAAAJBHHAAAA|30488|8|28|8|AM|first|morning|breakfast| +30489|AAAAAAAAKBHHAAAA|30489|8|28|9|AM|first|morning|breakfast| +30490|AAAAAAAALBHHAAAA|30490|8|28|10|AM|first|morning|breakfast| +30491|AAAAAAAAMBHHAAAA|30491|8|28|11|AM|first|morning|breakfast| +30492|AAAAAAAANBHHAAAA|30492|8|28|12|AM|first|morning|breakfast| +30493|AAAAAAAAOBHHAAAA|30493|8|28|13|AM|first|morning|breakfast| +30494|AAAAAAAAPBHHAAAA|30494|8|28|14|AM|first|morning|breakfast| +30495|AAAAAAAAACHHAAAA|30495|8|28|15|AM|first|morning|breakfast| +30496|AAAAAAAABCHHAAAA|30496|8|28|16|AM|first|morning|breakfast| +30497|AAAAAAAACCHHAAAA|30497|8|28|17|AM|first|morning|breakfast| +30498|AAAAAAAADCHHAAAA|30498|8|28|18|AM|first|morning|breakfast| +30499|AAAAAAAAECHHAAAA|30499|8|28|19|AM|first|morning|breakfast| +30500|AAAAAAAAFCHHAAAA|30500|8|28|20|AM|first|morning|breakfast| +30501|AAAAAAAAGCHHAAAA|30501|8|28|21|AM|first|morning|breakfast| +30502|AAAAAAAAHCHHAAAA|30502|8|28|22|AM|first|morning|breakfast| +30503|AAAAAAAAICHHAAAA|30503|8|28|23|AM|first|morning|breakfast| +30504|AAAAAAAAJCHHAAAA|30504|8|28|24|AM|first|morning|breakfast| +30505|AAAAAAAAKCHHAAAA|30505|8|28|25|AM|first|morning|breakfast| +30506|AAAAAAAALCHHAAAA|30506|8|28|26|AM|first|morning|breakfast| +30507|AAAAAAAAMCHHAAAA|30507|8|28|27|AM|first|morning|breakfast| +30508|AAAAAAAANCHHAAAA|30508|8|28|28|AM|first|morning|breakfast| +30509|AAAAAAAAOCHHAAAA|30509|8|28|29|AM|first|morning|breakfast| +30510|AAAAAAAAPCHHAAAA|30510|8|28|30|AM|first|morning|breakfast| +30511|AAAAAAAAADHHAAAA|30511|8|28|31|AM|first|morning|breakfast| +30512|AAAAAAAABDHHAAAA|30512|8|28|32|AM|first|morning|breakfast| +30513|AAAAAAAACDHHAAAA|30513|8|28|33|AM|first|morning|breakfast| +30514|AAAAAAAADDHHAAAA|30514|8|28|34|AM|first|morning|breakfast| +30515|AAAAAAAAEDHHAAAA|30515|8|28|35|AM|first|morning|breakfast| +30516|AAAAAAAAFDHHAAAA|30516|8|28|36|AM|first|morning|breakfast| +30517|AAAAAAAAGDHHAAAA|30517|8|28|37|AM|first|morning|breakfast| +30518|AAAAAAAAHDHHAAAA|30518|8|28|38|AM|first|morning|breakfast| +30519|AAAAAAAAIDHHAAAA|30519|8|28|39|AM|first|morning|breakfast| +30520|AAAAAAAAJDHHAAAA|30520|8|28|40|AM|first|morning|breakfast| +30521|AAAAAAAAKDHHAAAA|30521|8|28|41|AM|first|morning|breakfast| +30522|AAAAAAAALDHHAAAA|30522|8|28|42|AM|first|morning|breakfast| +30523|AAAAAAAAMDHHAAAA|30523|8|28|43|AM|first|morning|breakfast| +30524|AAAAAAAANDHHAAAA|30524|8|28|44|AM|first|morning|breakfast| +30525|AAAAAAAAODHHAAAA|30525|8|28|45|AM|first|morning|breakfast| +30526|AAAAAAAAPDHHAAAA|30526|8|28|46|AM|first|morning|breakfast| +30527|AAAAAAAAAEHHAAAA|30527|8|28|47|AM|first|morning|breakfast| +30528|AAAAAAAABEHHAAAA|30528|8|28|48|AM|first|morning|breakfast| +30529|AAAAAAAACEHHAAAA|30529|8|28|49|AM|first|morning|breakfast| +30530|AAAAAAAADEHHAAAA|30530|8|28|50|AM|first|morning|breakfast| +30531|AAAAAAAAEEHHAAAA|30531|8|28|51|AM|first|morning|breakfast| +30532|AAAAAAAAFEHHAAAA|30532|8|28|52|AM|first|morning|breakfast| +30533|AAAAAAAAGEHHAAAA|30533|8|28|53|AM|first|morning|breakfast| +30534|AAAAAAAAHEHHAAAA|30534|8|28|54|AM|first|morning|breakfast| +30535|AAAAAAAAIEHHAAAA|30535|8|28|55|AM|first|morning|breakfast| +30536|AAAAAAAAJEHHAAAA|30536|8|28|56|AM|first|morning|breakfast| +30537|AAAAAAAAKEHHAAAA|30537|8|28|57|AM|first|morning|breakfast| +30538|AAAAAAAALEHHAAAA|30538|8|28|58|AM|first|morning|breakfast| +30539|AAAAAAAAMEHHAAAA|30539|8|28|59|AM|first|morning|breakfast| +30540|AAAAAAAANEHHAAAA|30540|8|29|0|AM|first|morning|breakfast| +30541|AAAAAAAAOEHHAAAA|30541|8|29|1|AM|first|morning|breakfast| +30542|AAAAAAAAPEHHAAAA|30542|8|29|2|AM|first|morning|breakfast| +30543|AAAAAAAAAFHHAAAA|30543|8|29|3|AM|first|morning|breakfast| +30544|AAAAAAAABFHHAAAA|30544|8|29|4|AM|first|morning|breakfast| +30545|AAAAAAAACFHHAAAA|30545|8|29|5|AM|first|morning|breakfast| +30546|AAAAAAAADFHHAAAA|30546|8|29|6|AM|first|morning|breakfast| +30547|AAAAAAAAEFHHAAAA|30547|8|29|7|AM|first|morning|breakfast| +30548|AAAAAAAAFFHHAAAA|30548|8|29|8|AM|first|morning|breakfast| +30549|AAAAAAAAGFHHAAAA|30549|8|29|9|AM|first|morning|breakfast| +30550|AAAAAAAAHFHHAAAA|30550|8|29|10|AM|first|morning|breakfast| +30551|AAAAAAAAIFHHAAAA|30551|8|29|11|AM|first|morning|breakfast| +30552|AAAAAAAAJFHHAAAA|30552|8|29|12|AM|first|morning|breakfast| +30553|AAAAAAAAKFHHAAAA|30553|8|29|13|AM|first|morning|breakfast| +30554|AAAAAAAALFHHAAAA|30554|8|29|14|AM|first|morning|breakfast| +30555|AAAAAAAAMFHHAAAA|30555|8|29|15|AM|first|morning|breakfast| +30556|AAAAAAAANFHHAAAA|30556|8|29|16|AM|first|morning|breakfast| +30557|AAAAAAAAOFHHAAAA|30557|8|29|17|AM|first|morning|breakfast| +30558|AAAAAAAAPFHHAAAA|30558|8|29|18|AM|first|morning|breakfast| +30559|AAAAAAAAAGHHAAAA|30559|8|29|19|AM|first|morning|breakfast| +30560|AAAAAAAABGHHAAAA|30560|8|29|20|AM|first|morning|breakfast| +30561|AAAAAAAACGHHAAAA|30561|8|29|21|AM|first|morning|breakfast| +30562|AAAAAAAADGHHAAAA|30562|8|29|22|AM|first|morning|breakfast| +30563|AAAAAAAAEGHHAAAA|30563|8|29|23|AM|first|morning|breakfast| +30564|AAAAAAAAFGHHAAAA|30564|8|29|24|AM|first|morning|breakfast| +30565|AAAAAAAAGGHHAAAA|30565|8|29|25|AM|first|morning|breakfast| +30566|AAAAAAAAHGHHAAAA|30566|8|29|26|AM|first|morning|breakfast| +30567|AAAAAAAAIGHHAAAA|30567|8|29|27|AM|first|morning|breakfast| +30568|AAAAAAAAJGHHAAAA|30568|8|29|28|AM|first|morning|breakfast| +30569|AAAAAAAAKGHHAAAA|30569|8|29|29|AM|first|morning|breakfast| +30570|AAAAAAAALGHHAAAA|30570|8|29|30|AM|first|morning|breakfast| +30571|AAAAAAAAMGHHAAAA|30571|8|29|31|AM|first|morning|breakfast| +30572|AAAAAAAANGHHAAAA|30572|8|29|32|AM|first|morning|breakfast| +30573|AAAAAAAAOGHHAAAA|30573|8|29|33|AM|first|morning|breakfast| +30574|AAAAAAAAPGHHAAAA|30574|8|29|34|AM|first|morning|breakfast| +30575|AAAAAAAAAHHHAAAA|30575|8|29|35|AM|first|morning|breakfast| +30576|AAAAAAAABHHHAAAA|30576|8|29|36|AM|first|morning|breakfast| +30577|AAAAAAAACHHHAAAA|30577|8|29|37|AM|first|morning|breakfast| +30578|AAAAAAAADHHHAAAA|30578|8|29|38|AM|first|morning|breakfast| +30579|AAAAAAAAEHHHAAAA|30579|8|29|39|AM|first|morning|breakfast| +30580|AAAAAAAAFHHHAAAA|30580|8|29|40|AM|first|morning|breakfast| +30581|AAAAAAAAGHHHAAAA|30581|8|29|41|AM|first|morning|breakfast| +30582|AAAAAAAAHHHHAAAA|30582|8|29|42|AM|first|morning|breakfast| +30583|AAAAAAAAIHHHAAAA|30583|8|29|43|AM|first|morning|breakfast| +30584|AAAAAAAAJHHHAAAA|30584|8|29|44|AM|first|morning|breakfast| +30585|AAAAAAAAKHHHAAAA|30585|8|29|45|AM|first|morning|breakfast| +30586|AAAAAAAALHHHAAAA|30586|8|29|46|AM|first|morning|breakfast| +30587|AAAAAAAAMHHHAAAA|30587|8|29|47|AM|first|morning|breakfast| +30588|AAAAAAAANHHHAAAA|30588|8|29|48|AM|first|morning|breakfast| +30589|AAAAAAAAOHHHAAAA|30589|8|29|49|AM|first|morning|breakfast| +30590|AAAAAAAAPHHHAAAA|30590|8|29|50|AM|first|morning|breakfast| +30591|AAAAAAAAAIHHAAAA|30591|8|29|51|AM|first|morning|breakfast| +30592|AAAAAAAABIHHAAAA|30592|8|29|52|AM|first|morning|breakfast| +30593|AAAAAAAACIHHAAAA|30593|8|29|53|AM|first|morning|breakfast| +30594|AAAAAAAADIHHAAAA|30594|8|29|54|AM|first|morning|breakfast| +30595|AAAAAAAAEIHHAAAA|30595|8|29|55|AM|first|morning|breakfast| +30596|AAAAAAAAFIHHAAAA|30596|8|29|56|AM|first|morning|breakfast| +30597|AAAAAAAAGIHHAAAA|30597|8|29|57|AM|first|morning|breakfast| +30598|AAAAAAAAHIHHAAAA|30598|8|29|58|AM|first|morning|breakfast| +30599|AAAAAAAAIIHHAAAA|30599|8|29|59|AM|first|morning|breakfast| +30600|AAAAAAAAJIHHAAAA|30600|8|30|0|AM|first|morning|breakfast| +30601|AAAAAAAAKIHHAAAA|30601|8|30|1|AM|first|morning|breakfast| +30602|AAAAAAAALIHHAAAA|30602|8|30|2|AM|first|morning|breakfast| +30603|AAAAAAAAMIHHAAAA|30603|8|30|3|AM|first|morning|breakfast| +30604|AAAAAAAANIHHAAAA|30604|8|30|4|AM|first|morning|breakfast| +30605|AAAAAAAAOIHHAAAA|30605|8|30|5|AM|first|morning|breakfast| +30606|AAAAAAAAPIHHAAAA|30606|8|30|6|AM|first|morning|breakfast| +30607|AAAAAAAAAJHHAAAA|30607|8|30|7|AM|first|morning|breakfast| +30608|AAAAAAAABJHHAAAA|30608|8|30|8|AM|first|morning|breakfast| +30609|AAAAAAAACJHHAAAA|30609|8|30|9|AM|first|morning|breakfast| +30610|AAAAAAAADJHHAAAA|30610|8|30|10|AM|first|morning|breakfast| +30611|AAAAAAAAEJHHAAAA|30611|8|30|11|AM|first|morning|breakfast| +30612|AAAAAAAAFJHHAAAA|30612|8|30|12|AM|first|morning|breakfast| +30613|AAAAAAAAGJHHAAAA|30613|8|30|13|AM|first|morning|breakfast| +30614|AAAAAAAAHJHHAAAA|30614|8|30|14|AM|first|morning|breakfast| +30615|AAAAAAAAIJHHAAAA|30615|8|30|15|AM|first|morning|breakfast| +30616|AAAAAAAAJJHHAAAA|30616|8|30|16|AM|first|morning|breakfast| +30617|AAAAAAAAKJHHAAAA|30617|8|30|17|AM|first|morning|breakfast| +30618|AAAAAAAALJHHAAAA|30618|8|30|18|AM|first|morning|breakfast| +30619|AAAAAAAAMJHHAAAA|30619|8|30|19|AM|first|morning|breakfast| +30620|AAAAAAAANJHHAAAA|30620|8|30|20|AM|first|morning|breakfast| +30621|AAAAAAAAOJHHAAAA|30621|8|30|21|AM|first|morning|breakfast| +30622|AAAAAAAAPJHHAAAA|30622|8|30|22|AM|first|morning|breakfast| +30623|AAAAAAAAAKHHAAAA|30623|8|30|23|AM|first|morning|breakfast| +30624|AAAAAAAABKHHAAAA|30624|8|30|24|AM|first|morning|breakfast| +30625|AAAAAAAACKHHAAAA|30625|8|30|25|AM|first|morning|breakfast| +30626|AAAAAAAADKHHAAAA|30626|8|30|26|AM|first|morning|breakfast| +30627|AAAAAAAAEKHHAAAA|30627|8|30|27|AM|first|morning|breakfast| +30628|AAAAAAAAFKHHAAAA|30628|8|30|28|AM|first|morning|breakfast| +30629|AAAAAAAAGKHHAAAA|30629|8|30|29|AM|first|morning|breakfast| +30630|AAAAAAAAHKHHAAAA|30630|8|30|30|AM|first|morning|breakfast| +30631|AAAAAAAAIKHHAAAA|30631|8|30|31|AM|first|morning|breakfast| +30632|AAAAAAAAJKHHAAAA|30632|8|30|32|AM|first|morning|breakfast| +30633|AAAAAAAAKKHHAAAA|30633|8|30|33|AM|first|morning|breakfast| +30634|AAAAAAAALKHHAAAA|30634|8|30|34|AM|first|morning|breakfast| +30635|AAAAAAAAMKHHAAAA|30635|8|30|35|AM|first|morning|breakfast| +30636|AAAAAAAANKHHAAAA|30636|8|30|36|AM|first|morning|breakfast| +30637|AAAAAAAAOKHHAAAA|30637|8|30|37|AM|first|morning|breakfast| +30638|AAAAAAAAPKHHAAAA|30638|8|30|38|AM|first|morning|breakfast| +30639|AAAAAAAAALHHAAAA|30639|8|30|39|AM|first|morning|breakfast| +30640|AAAAAAAABLHHAAAA|30640|8|30|40|AM|first|morning|breakfast| +30641|AAAAAAAACLHHAAAA|30641|8|30|41|AM|first|morning|breakfast| +30642|AAAAAAAADLHHAAAA|30642|8|30|42|AM|first|morning|breakfast| +30643|AAAAAAAAELHHAAAA|30643|8|30|43|AM|first|morning|breakfast| +30644|AAAAAAAAFLHHAAAA|30644|8|30|44|AM|first|morning|breakfast| +30645|AAAAAAAAGLHHAAAA|30645|8|30|45|AM|first|morning|breakfast| +30646|AAAAAAAAHLHHAAAA|30646|8|30|46|AM|first|morning|breakfast| +30647|AAAAAAAAILHHAAAA|30647|8|30|47|AM|first|morning|breakfast| +30648|AAAAAAAAJLHHAAAA|30648|8|30|48|AM|first|morning|breakfast| +30649|AAAAAAAAKLHHAAAA|30649|8|30|49|AM|first|morning|breakfast| +30650|AAAAAAAALLHHAAAA|30650|8|30|50|AM|first|morning|breakfast| +30651|AAAAAAAAMLHHAAAA|30651|8|30|51|AM|first|morning|breakfast| +30652|AAAAAAAANLHHAAAA|30652|8|30|52|AM|first|morning|breakfast| +30653|AAAAAAAAOLHHAAAA|30653|8|30|53|AM|first|morning|breakfast| +30654|AAAAAAAAPLHHAAAA|30654|8|30|54|AM|first|morning|breakfast| +30655|AAAAAAAAAMHHAAAA|30655|8|30|55|AM|first|morning|breakfast| +30656|AAAAAAAABMHHAAAA|30656|8|30|56|AM|first|morning|breakfast| +30657|AAAAAAAACMHHAAAA|30657|8|30|57|AM|first|morning|breakfast| +30658|AAAAAAAADMHHAAAA|30658|8|30|58|AM|first|morning|breakfast| +30659|AAAAAAAAEMHHAAAA|30659|8|30|59|AM|first|morning|breakfast| +30660|AAAAAAAAFMHHAAAA|30660|8|31|0|AM|first|morning|breakfast| +30661|AAAAAAAAGMHHAAAA|30661|8|31|1|AM|first|morning|breakfast| +30662|AAAAAAAAHMHHAAAA|30662|8|31|2|AM|first|morning|breakfast| +30663|AAAAAAAAIMHHAAAA|30663|8|31|3|AM|first|morning|breakfast| +30664|AAAAAAAAJMHHAAAA|30664|8|31|4|AM|first|morning|breakfast| +30665|AAAAAAAAKMHHAAAA|30665|8|31|5|AM|first|morning|breakfast| +30666|AAAAAAAALMHHAAAA|30666|8|31|6|AM|first|morning|breakfast| +30667|AAAAAAAAMMHHAAAA|30667|8|31|7|AM|first|morning|breakfast| +30668|AAAAAAAANMHHAAAA|30668|8|31|8|AM|first|morning|breakfast| +30669|AAAAAAAAOMHHAAAA|30669|8|31|9|AM|first|morning|breakfast| +30670|AAAAAAAAPMHHAAAA|30670|8|31|10|AM|first|morning|breakfast| +30671|AAAAAAAAANHHAAAA|30671|8|31|11|AM|first|morning|breakfast| +30672|AAAAAAAABNHHAAAA|30672|8|31|12|AM|first|morning|breakfast| +30673|AAAAAAAACNHHAAAA|30673|8|31|13|AM|first|morning|breakfast| +30674|AAAAAAAADNHHAAAA|30674|8|31|14|AM|first|morning|breakfast| +30675|AAAAAAAAENHHAAAA|30675|8|31|15|AM|first|morning|breakfast| +30676|AAAAAAAAFNHHAAAA|30676|8|31|16|AM|first|morning|breakfast| +30677|AAAAAAAAGNHHAAAA|30677|8|31|17|AM|first|morning|breakfast| +30678|AAAAAAAAHNHHAAAA|30678|8|31|18|AM|first|morning|breakfast| +30679|AAAAAAAAINHHAAAA|30679|8|31|19|AM|first|morning|breakfast| +30680|AAAAAAAAJNHHAAAA|30680|8|31|20|AM|first|morning|breakfast| +30681|AAAAAAAAKNHHAAAA|30681|8|31|21|AM|first|morning|breakfast| +30682|AAAAAAAALNHHAAAA|30682|8|31|22|AM|first|morning|breakfast| +30683|AAAAAAAAMNHHAAAA|30683|8|31|23|AM|first|morning|breakfast| +30684|AAAAAAAANNHHAAAA|30684|8|31|24|AM|first|morning|breakfast| +30685|AAAAAAAAONHHAAAA|30685|8|31|25|AM|first|morning|breakfast| +30686|AAAAAAAAPNHHAAAA|30686|8|31|26|AM|first|morning|breakfast| +30687|AAAAAAAAAOHHAAAA|30687|8|31|27|AM|first|morning|breakfast| +30688|AAAAAAAABOHHAAAA|30688|8|31|28|AM|first|morning|breakfast| +30689|AAAAAAAACOHHAAAA|30689|8|31|29|AM|first|morning|breakfast| +30690|AAAAAAAADOHHAAAA|30690|8|31|30|AM|first|morning|breakfast| +30691|AAAAAAAAEOHHAAAA|30691|8|31|31|AM|first|morning|breakfast| +30692|AAAAAAAAFOHHAAAA|30692|8|31|32|AM|first|morning|breakfast| +30693|AAAAAAAAGOHHAAAA|30693|8|31|33|AM|first|morning|breakfast| +30694|AAAAAAAAHOHHAAAA|30694|8|31|34|AM|first|morning|breakfast| +30695|AAAAAAAAIOHHAAAA|30695|8|31|35|AM|first|morning|breakfast| +30696|AAAAAAAAJOHHAAAA|30696|8|31|36|AM|first|morning|breakfast| +30697|AAAAAAAAKOHHAAAA|30697|8|31|37|AM|first|morning|breakfast| +30698|AAAAAAAALOHHAAAA|30698|8|31|38|AM|first|morning|breakfast| +30699|AAAAAAAAMOHHAAAA|30699|8|31|39|AM|first|morning|breakfast| +30700|AAAAAAAANOHHAAAA|30700|8|31|40|AM|first|morning|breakfast| +30701|AAAAAAAAOOHHAAAA|30701|8|31|41|AM|first|morning|breakfast| +30702|AAAAAAAAPOHHAAAA|30702|8|31|42|AM|first|morning|breakfast| +30703|AAAAAAAAAPHHAAAA|30703|8|31|43|AM|first|morning|breakfast| +30704|AAAAAAAABPHHAAAA|30704|8|31|44|AM|first|morning|breakfast| +30705|AAAAAAAACPHHAAAA|30705|8|31|45|AM|first|morning|breakfast| +30706|AAAAAAAADPHHAAAA|30706|8|31|46|AM|first|morning|breakfast| +30707|AAAAAAAAEPHHAAAA|30707|8|31|47|AM|first|morning|breakfast| +30708|AAAAAAAAFPHHAAAA|30708|8|31|48|AM|first|morning|breakfast| +30709|AAAAAAAAGPHHAAAA|30709|8|31|49|AM|first|morning|breakfast| +30710|AAAAAAAAHPHHAAAA|30710|8|31|50|AM|first|morning|breakfast| +30711|AAAAAAAAIPHHAAAA|30711|8|31|51|AM|first|morning|breakfast| +30712|AAAAAAAAJPHHAAAA|30712|8|31|52|AM|first|morning|breakfast| +30713|AAAAAAAAKPHHAAAA|30713|8|31|53|AM|first|morning|breakfast| +30714|AAAAAAAALPHHAAAA|30714|8|31|54|AM|first|morning|breakfast| +30715|AAAAAAAAMPHHAAAA|30715|8|31|55|AM|first|morning|breakfast| +30716|AAAAAAAANPHHAAAA|30716|8|31|56|AM|first|morning|breakfast| +30717|AAAAAAAAOPHHAAAA|30717|8|31|57|AM|first|morning|breakfast| +30718|AAAAAAAAPPHHAAAA|30718|8|31|58|AM|first|morning|breakfast| +30719|AAAAAAAAAAIHAAAA|30719|8|31|59|AM|first|morning|breakfast| +30720|AAAAAAAABAIHAAAA|30720|8|32|0|AM|first|morning|breakfast| +30721|AAAAAAAACAIHAAAA|30721|8|32|1|AM|first|morning|breakfast| +30722|AAAAAAAADAIHAAAA|30722|8|32|2|AM|first|morning|breakfast| +30723|AAAAAAAAEAIHAAAA|30723|8|32|3|AM|first|morning|breakfast| +30724|AAAAAAAAFAIHAAAA|30724|8|32|4|AM|first|morning|breakfast| +30725|AAAAAAAAGAIHAAAA|30725|8|32|5|AM|first|morning|breakfast| +30726|AAAAAAAAHAIHAAAA|30726|8|32|6|AM|first|morning|breakfast| +30727|AAAAAAAAIAIHAAAA|30727|8|32|7|AM|first|morning|breakfast| +30728|AAAAAAAAJAIHAAAA|30728|8|32|8|AM|first|morning|breakfast| +30729|AAAAAAAAKAIHAAAA|30729|8|32|9|AM|first|morning|breakfast| +30730|AAAAAAAALAIHAAAA|30730|8|32|10|AM|first|morning|breakfast| +30731|AAAAAAAAMAIHAAAA|30731|8|32|11|AM|first|morning|breakfast| +30732|AAAAAAAANAIHAAAA|30732|8|32|12|AM|first|morning|breakfast| +30733|AAAAAAAAOAIHAAAA|30733|8|32|13|AM|first|morning|breakfast| +30734|AAAAAAAAPAIHAAAA|30734|8|32|14|AM|first|morning|breakfast| +30735|AAAAAAAAABIHAAAA|30735|8|32|15|AM|first|morning|breakfast| +30736|AAAAAAAABBIHAAAA|30736|8|32|16|AM|first|morning|breakfast| +30737|AAAAAAAACBIHAAAA|30737|8|32|17|AM|first|morning|breakfast| +30738|AAAAAAAADBIHAAAA|30738|8|32|18|AM|first|morning|breakfast| +30739|AAAAAAAAEBIHAAAA|30739|8|32|19|AM|first|morning|breakfast| +30740|AAAAAAAAFBIHAAAA|30740|8|32|20|AM|first|morning|breakfast| +30741|AAAAAAAAGBIHAAAA|30741|8|32|21|AM|first|morning|breakfast| +30742|AAAAAAAAHBIHAAAA|30742|8|32|22|AM|first|morning|breakfast| +30743|AAAAAAAAIBIHAAAA|30743|8|32|23|AM|first|morning|breakfast| +30744|AAAAAAAAJBIHAAAA|30744|8|32|24|AM|first|morning|breakfast| +30745|AAAAAAAAKBIHAAAA|30745|8|32|25|AM|first|morning|breakfast| +30746|AAAAAAAALBIHAAAA|30746|8|32|26|AM|first|morning|breakfast| +30747|AAAAAAAAMBIHAAAA|30747|8|32|27|AM|first|morning|breakfast| +30748|AAAAAAAANBIHAAAA|30748|8|32|28|AM|first|morning|breakfast| +30749|AAAAAAAAOBIHAAAA|30749|8|32|29|AM|first|morning|breakfast| +30750|AAAAAAAAPBIHAAAA|30750|8|32|30|AM|first|morning|breakfast| +30751|AAAAAAAAACIHAAAA|30751|8|32|31|AM|first|morning|breakfast| +30752|AAAAAAAABCIHAAAA|30752|8|32|32|AM|first|morning|breakfast| +30753|AAAAAAAACCIHAAAA|30753|8|32|33|AM|first|morning|breakfast| +30754|AAAAAAAADCIHAAAA|30754|8|32|34|AM|first|morning|breakfast| +30755|AAAAAAAAECIHAAAA|30755|8|32|35|AM|first|morning|breakfast| +30756|AAAAAAAAFCIHAAAA|30756|8|32|36|AM|first|morning|breakfast| +30757|AAAAAAAAGCIHAAAA|30757|8|32|37|AM|first|morning|breakfast| +30758|AAAAAAAAHCIHAAAA|30758|8|32|38|AM|first|morning|breakfast| +30759|AAAAAAAAICIHAAAA|30759|8|32|39|AM|first|morning|breakfast| +30760|AAAAAAAAJCIHAAAA|30760|8|32|40|AM|first|morning|breakfast| +30761|AAAAAAAAKCIHAAAA|30761|8|32|41|AM|first|morning|breakfast| +30762|AAAAAAAALCIHAAAA|30762|8|32|42|AM|first|morning|breakfast| +30763|AAAAAAAAMCIHAAAA|30763|8|32|43|AM|first|morning|breakfast| +30764|AAAAAAAANCIHAAAA|30764|8|32|44|AM|first|morning|breakfast| +30765|AAAAAAAAOCIHAAAA|30765|8|32|45|AM|first|morning|breakfast| +30766|AAAAAAAAPCIHAAAA|30766|8|32|46|AM|first|morning|breakfast| +30767|AAAAAAAAADIHAAAA|30767|8|32|47|AM|first|morning|breakfast| +30768|AAAAAAAABDIHAAAA|30768|8|32|48|AM|first|morning|breakfast| +30769|AAAAAAAACDIHAAAA|30769|8|32|49|AM|first|morning|breakfast| +30770|AAAAAAAADDIHAAAA|30770|8|32|50|AM|first|morning|breakfast| +30771|AAAAAAAAEDIHAAAA|30771|8|32|51|AM|first|morning|breakfast| +30772|AAAAAAAAFDIHAAAA|30772|8|32|52|AM|first|morning|breakfast| +30773|AAAAAAAAGDIHAAAA|30773|8|32|53|AM|first|morning|breakfast| +30774|AAAAAAAAHDIHAAAA|30774|8|32|54|AM|first|morning|breakfast| +30775|AAAAAAAAIDIHAAAA|30775|8|32|55|AM|first|morning|breakfast| +30776|AAAAAAAAJDIHAAAA|30776|8|32|56|AM|first|morning|breakfast| +30777|AAAAAAAAKDIHAAAA|30777|8|32|57|AM|first|morning|breakfast| +30778|AAAAAAAALDIHAAAA|30778|8|32|58|AM|first|morning|breakfast| +30779|AAAAAAAAMDIHAAAA|30779|8|32|59|AM|first|morning|breakfast| +30780|AAAAAAAANDIHAAAA|30780|8|33|0|AM|first|morning|breakfast| +30781|AAAAAAAAODIHAAAA|30781|8|33|1|AM|first|morning|breakfast| +30782|AAAAAAAAPDIHAAAA|30782|8|33|2|AM|first|morning|breakfast| +30783|AAAAAAAAAEIHAAAA|30783|8|33|3|AM|first|morning|breakfast| +30784|AAAAAAAABEIHAAAA|30784|8|33|4|AM|first|morning|breakfast| +30785|AAAAAAAACEIHAAAA|30785|8|33|5|AM|first|morning|breakfast| +30786|AAAAAAAADEIHAAAA|30786|8|33|6|AM|first|morning|breakfast| +30787|AAAAAAAAEEIHAAAA|30787|8|33|7|AM|first|morning|breakfast| +30788|AAAAAAAAFEIHAAAA|30788|8|33|8|AM|first|morning|breakfast| +30789|AAAAAAAAGEIHAAAA|30789|8|33|9|AM|first|morning|breakfast| +30790|AAAAAAAAHEIHAAAA|30790|8|33|10|AM|first|morning|breakfast| +30791|AAAAAAAAIEIHAAAA|30791|8|33|11|AM|first|morning|breakfast| +30792|AAAAAAAAJEIHAAAA|30792|8|33|12|AM|first|morning|breakfast| +30793|AAAAAAAAKEIHAAAA|30793|8|33|13|AM|first|morning|breakfast| +30794|AAAAAAAALEIHAAAA|30794|8|33|14|AM|first|morning|breakfast| +30795|AAAAAAAAMEIHAAAA|30795|8|33|15|AM|first|morning|breakfast| +30796|AAAAAAAANEIHAAAA|30796|8|33|16|AM|first|morning|breakfast| +30797|AAAAAAAAOEIHAAAA|30797|8|33|17|AM|first|morning|breakfast| +30798|AAAAAAAAPEIHAAAA|30798|8|33|18|AM|first|morning|breakfast| +30799|AAAAAAAAAFIHAAAA|30799|8|33|19|AM|first|morning|breakfast| +30800|AAAAAAAABFIHAAAA|30800|8|33|20|AM|first|morning|breakfast| +30801|AAAAAAAACFIHAAAA|30801|8|33|21|AM|first|morning|breakfast| +30802|AAAAAAAADFIHAAAA|30802|8|33|22|AM|first|morning|breakfast| +30803|AAAAAAAAEFIHAAAA|30803|8|33|23|AM|first|morning|breakfast| +30804|AAAAAAAAFFIHAAAA|30804|8|33|24|AM|first|morning|breakfast| +30805|AAAAAAAAGFIHAAAA|30805|8|33|25|AM|first|morning|breakfast| +30806|AAAAAAAAHFIHAAAA|30806|8|33|26|AM|first|morning|breakfast| +30807|AAAAAAAAIFIHAAAA|30807|8|33|27|AM|first|morning|breakfast| +30808|AAAAAAAAJFIHAAAA|30808|8|33|28|AM|first|morning|breakfast| +30809|AAAAAAAAKFIHAAAA|30809|8|33|29|AM|first|morning|breakfast| +30810|AAAAAAAALFIHAAAA|30810|8|33|30|AM|first|morning|breakfast| +30811|AAAAAAAAMFIHAAAA|30811|8|33|31|AM|first|morning|breakfast| +30812|AAAAAAAANFIHAAAA|30812|8|33|32|AM|first|morning|breakfast| +30813|AAAAAAAAOFIHAAAA|30813|8|33|33|AM|first|morning|breakfast| +30814|AAAAAAAAPFIHAAAA|30814|8|33|34|AM|first|morning|breakfast| +30815|AAAAAAAAAGIHAAAA|30815|8|33|35|AM|first|morning|breakfast| +30816|AAAAAAAABGIHAAAA|30816|8|33|36|AM|first|morning|breakfast| +30817|AAAAAAAACGIHAAAA|30817|8|33|37|AM|first|morning|breakfast| +30818|AAAAAAAADGIHAAAA|30818|8|33|38|AM|first|morning|breakfast| +30819|AAAAAAAAEGIHAAAA|30819|8|33|39|AM|first|morning|breakfast| +30820|AAAAAAAAFGIHAAAA|30820|8|33|40|AM|first|morning|breakfast| +30821|AAAAAAAAGGIHAAAA|30821|8|33|41|AM|first|morning|breakfast| +30822|AAAAAAAAHGIHAAAA|30822|8|33|42|AM|first|morning|breakfast| +30823|AAAAAAAAIGIHAAAA|30823|8|33|43|AM|first|morning|breakfast| +30824|AAAAAAAAJGIHAAAA|30824|8|33|44|AM|first|morning|breakfast| +30825|AAAAAAAAKGIHAAAA|30825|8|33|45|AM|first|morning|breakfast| +30826|AAAAAAAALGIHAAAA|30826|8|33|46|AM|first|morning|breakfast| +30827|AAAAAAAAMGIHAAAA|30827|8|33|47|AM|first|morning|breakfast| +30828|AAAAAAAANGIHAAAA|30828|8|33|48|AM|first|morning|breakfast| +30829|AAAAAAAAOGIHAAAA|30829|8|33|49|AM|first|morning|breakfast| +30830|AAAAAAAAPGIHAAAA|30830|8|33|50|AM|first|morning|breakfast| +30831|AAAAAAAAAHIHAAAA|30831|8|33|51|AM|first|morning|breakfast| +30832|AAAAAAAABHIHAAAA|30832|8|33|52|AM|first|morning|breakfast| +30833|AAAAAAAACHIHAAAA|30833|8|33|53|AM|first|morning|breakfast| +30834|AAAAAAAADHIHAAAA|30834|8|33|54|AM|first|morning|breakfast| +30835|AAAAAAAAEHIHAAAA|30835|8|33|55|AM|first|morning|breakfast| +30836|AAAAAAAAFHIHAAAA|30836|8|33|56|AM|first|morning|breakfast| +30837|AAAAAAAAGHIHAAAA|30837|8|33|57|AM|first|morning|breakfast| +30838|AAAAAAAAHHIHAAAA|30838|8|33|58|AM|first|morning|breakfast| +30839|AAAAAAAAIHIHAAAA|30839|8|33|59|AM|first|morning|breakfast| +30840|AAAAAAAAJHIHAAAA|30840|8|34|0|AM|first|morning|breakfast| +30841|AAAAAAAAKHIHAAAA|30841|8|34|1|AM|first|morning|breakfast| +30842|AAAAAAAALHIHAAAA|30842|8|34|2|AM|first|morning|breakfast| +30843|AAAAAAAAMHIHAAAA|30843|8|34|3|AM|first|morning|breakfast| +30844|AAAAAAAANHIHAAAA|30844|8|34|4|AM|first|morning|breakfast| +30845|AAAAAAAAOHIHAAAA|30845|8|34|5|AM|first|morning|breakfast| +30846|AAAAAAAAPHIHAAAA|30846|8|34|6|AM|first|morning|breakfast| +30847|AAAAAAAAAIIHAAAA|30847|8|34|7|AM|first|morning|breakfast| +30848|AAAAAAAABIIHAAAA|30848|8|34|8|AM|first|morning|breakfast| +30849|AAAAAAAACIIHAAAA|30849|8|34|9|AM|first|morning|breakfast| +30850|AAAAAAAADIIHAAAA|30850|8|34|10|AM|first|morning|breakfast| +30851|AAAAAAAAEIIHAAAA|30851|8|34|11|AM|first|morning|breakfast| +30852|AAAAAAAAFIIHAAAA|30852|8|34|12|AM|first|morning|breakfast| +30853|AAAAAAAAGIIHAAAA|30853|8|34|13|AM|first|morning|breakfast| +30854|AAAAAAAAHIIHAAAA|30854|8|34|14|AM|first|morning|breakfast| +30855|AAAAAAAAIIIHAAAA|30855|8|34|15|AM|first|morning|breakfast| +30856|AAAAAAAAJIIHAAAA|30856|8|34|16|AM|first|morning|breakfast| +30857|AAAAAAAAKIIHAAAA|30857|8|34|17|AM|first|morning|breakfast| +30858|AAAAAAAALIIHAAAA|30858|8|34|18|AM|first|morning|breakfast| +30859|AAAAAAAAMIIHAAAA|30859|8|34|19|AM|first|morning|breakfast| +30860|AAAAAAAANIIHAAAA|30860|8|34|20|AM|first|morning|breakfast| +30861|AAAAAAAAOIIHAAAA|30861|8|34|21|AM|first|morning|breakfast| +30862|AAAAAAAAPIIHAAAA|30862|8|34|22|AM|first|morning|breakfast| +30863|AAAAAAAAAJIHAAAA|30863|8|34|23|AM|first|morning|breakfast| +30864|AAAAAAAABJIHAAAA|30864|8|34|24|AM|first|morning|breakfast| +30865|AAAAAAAACJIHAAAA|30865|8|34|25|AM|first|morning|breakfast| +30866|AAAAAAAADJIHAAAA|30866|8|34|26|AM|first|morning|breakfast| +30867|AAAAAAAAEJIHAAAA|30867|8|34|27|AM|first|morning|breakfast| +30868|AAAAAAAAFJIHAAAA|30868|8|34|28|AM|first|morning|breakfast| +30869|AAAAAAAAGJIHAAAA|30869|8|34|29|AM|first|morning|breakfast| +30870|AAAAAAAAHJIHAAAA|30870|8|34|30|AM|first|morning|breakfast| +30871|AAAAAAAAIJIHAAAA|30871|8|34|31|AM|first|morning|breakfast| +30872|AAAAAAAAJJIHAAAA|30872|8|34|32|AM|first|morning|breakfast| +30873|AAAAAAAAKJIHAAAA|30873|8|34|33|AM|first|morning|breakfast| +30874|AAAAAAAALJIHAAAA|30874|8|34|34|AM|first|morning|breakfast| +30875|AAAAAAAAMJIHAAAA|30875|8|34|35|AM|first|morning|breakfast| +30876|AAAAAAAANJIHAAAA|30876|8|34|36|AM|first|morning|breakfast| +30877|AAAAAAAAOJIHAAAA|30877|8|34|37|AM|first|morning|breakfast| +30878|AAAAAAAAPJIHAAAA|30878|8|34|38|AM|first|morning|breakfast| +30879|AAAAAAAAAKIHAAAA|30879|8|34|39|AM|first|morning|breakfast| +30880|AAAAAAAABKIHAAAA|30880|8|34|40|AM|first|morning|breakfast| +30881|AAAAAAAACKIHAAAA|30881|8|34|41|AM|first|morning|breakfast| +30882|AAAAAAAADKIHAAAA|30882|8|34|42|AM|first|morning|breakfast| +30883|AAAAAAAAEKIHAAAA|30883|8|34|43|AM|first|morning|breakfast| +30884|AAAAAAAAFKIHAAAA|30884|8|34|44|AM|first|morning|breakfast| +30885|AAAAAAAAGKIHAAAA|30885|8|34|45|AM|first|morning|breakfast| +30886|AAAAAAAAHKIHAAAA|30886|8|34|46|AM|first|morning|breakfast| +30887|AAAAAAAAIKIHAAAA|30887|8|34|47|AM|first|morning|breakfast| +30888|AAAAAAAAJKIHAAAA|30888|8|34|48|AM|first|morning|breakfast| +30889|AAAAAAAAKKIHAAAA|30889|8|34|49|AM|first|morning|breakfast| +30890|AAAAAAAALKIHAAAA|30890|8|34|50|AM|first|morning|breakfast| +30891|AAAAAAAAMKIHAAAA|30891|8|34|51|AM|first|morning|breakfast| +30892|AAAAAAAANKIHAAAA|30892|8|34|52|AM|first|morning|breakfast| +30893|AAAAAAAAOKIHAAAA|30893|8|34|53|AM|first|morning|breakfast| +30894|AAAAAAAAPKIHAAAA|30894|8|34|54|AM|first|morning|breakfast| +30895|AAAAAAAAALIHAAAA|30895|8|34|55|AM|first|morning|breakfast| +30896|AAAAAAAABLIHAAAA|30896|8|34|56|AM|first|morning|breakfast| +30897|AAAAAAAACLIHAAAA|30897|8|34|57|AM|first|morning|breakfast| +30898|AAAAAAAADLIHAAAA|30898|8|34|58|AM|first|morning|breakfast| +30899|AAAAAAAAELIHAAAA|30899|8|34|59|AM|first|morning|breakfast| +30900|AAAAAAAAFLIHAAAA|30900|8|35|0|AM|first|morning|breakfast| +30901|AAAAAAAAGLIHAAAA|30901|8|35|1|AM|first|morning|breakfast| +30902|AAAAAAAAHLIHAAAA|30902|8|35|2|AM|first|morning|breakfast| +30903|AAAAAAAAILIHAAAA|30903|8|35|3|AM|first|morning|breakfast| +30904|AAAAAAAAJLIHAAAA|30904|8|35|4|AM|first|morning|breakfast| +30905|AAAAAAAAKLIHAAAA|30905|8|35|5|AM|first|morning|breakfast| +30906|AAAAAAAALLIHAAAA|30906|8|35|6|AM|first|morning|breakfast| +30907|AAAAAAAAMLIHAAAA|30907|8|35|7|AM|first|morning|breakfast| +30908|AAAAAAAANLIHAAAA|30908|8|35|8|AM|first|morning|breakfast| +30909|AAAAAAAAOLIHAAAA|30909|8|35|9|AM|first|morning|breakfast| +30910|AAAAAAAAPLIHAAAA|30910|8|35|10|AM|first|morning|breakfast| +30911|AAAAAAAAAMIHAAAA|30911|8|35|11|AM|first|morning|breakfast| +30912|AAAAAAAABMIHAAAA|30912|8|35|12|AM|first|morning|breakfast| +30913|AAAAAAAACMIHAAAA|30913|8|35|13|AM|first|morning|breakfast| +30914|AAAAAAAADMIHAAAA|30914|8|35|14|AM|first|morning|breakfast| +30915|AAAAAAAAEMIHAAAA|30915|8|35|15|AM|first|morning|breakfast| +30916|AAAAAAAAFMIHAAAA|30916|8|35|16|AM|first|morning|breakfast| +30917|AAAAAAAAGMIHAAAA|30917|8|35|17|AM|first|morning|breakfast| +30918|AAAAAAAAHMIHAAAA|30918|8|35|18|AM|first|morning|breakfast| +30919|AAAAAAAAIMIHAAAA|30919|8|35|19|AM|first|morning|breakfast| +30920|AAAAAAAAJMIHAAAA|30920|8|35|20|AM|first|morning|breakfast| +30921|AAAAAAAAKMIHAAAA|30921|8|35|21|AM|first|morning|breakfast| +30922|AAAAAAAALMIHAAAA|30922|8|35|22|AM|first|morning|breakfast| +30923|AAAAAAAAMMIHAAAA|30923|8|35|23|AM|first|morning|breakfast| +30924|AAAAAAAANMIHAAAA|30924|8|35|24|AM|first|morning|breakfast| +30925|AAAAAAAAOMIHAAAA|30925|8|35|25|AM|first|morning|breakfast| +30926|AAAAAAAAPMIHAAAA|30926|8|35|26|AM|first|morning|breakfast| +30927|AAAAAAAAANIHAAAA|30927|8|35|27|AM|first|morning|breakfast| +30928|AAAAAAAABNIHAAAA|30928|8|35|28|AM|first|morning|breakfast| +30929|AAAAAAAACNIHAAAA|30929|8|35|29|AM|first|morning|breakfast| +30930|AAAAAAAADNIHAAAA|30930|8|35|30|AM|first|morning|breakfast| +30931|AAAAAAAAENIHAAAA|30931|8|35|31|AM|first|morning|breakfast| +30932|AAAAAAAAFNIHAAAA|30932|8|35|32|AM|first|morning|breakfast| +30933|AAAAAAAAGNIHAAAA|30933|8|35|33|AM|first|morning|breakfast| +30934|AAAAAAAAHNIHAAAA|30934|8|35|34|AM|first|morning|breakfast| +30935|AAAAAAAAINIHAAAA|30935|8|35|35|AM|first|morning|breakfast| +30936|AAAAAAAAJNIHAAAA|30936|8|35|36|AM|first|morning|breakfast| +30937|AAAAAAAAKNIHAAAA|30937|8|35|37|AM|first|morning|breakfast| +30938|AAAAAAAALNIHAAAA|30938|8|35|38|AM|first|morning|breakfast| +30939|AAAAAAAAMNIHAAAA|30939|8|35|39|AM|first|morning|breakfast| +30940|AAAAAAAANNIHAAAA|30940|8|35|40|AM|first|morning|breakfast| +30941|AAAAAAAAONIHAAAA|30941|8|35|41|AM|first|morning|breakfast| +30942|AAAAAAAAPNIHAAAA|30942|8|35|42|AM|first|morning|breakfast| +30943|AAAAAAAAAOIHAAAA|30943|8|35|43|AM|first|morning|breakfast| +30944|AAAAAAAABOIHAAAA|30944|8|35|44|AM|first|morning|breakfast| +30945|AAAAAAAACOIHAAAA|30945|8|35|45|AM|first|morning|breakfast| +30946|AAAAAAAADOIHAAAA|30946|8|35|46|AM|first|morning|breakfast| +30947|AAAAAAAAEOIHAAAA|30947|8|35|47|AM|first|morning|breakfast| +30948|AAAAAAAAFOIHAAAA|30948|8|35|48|AM|first|morning|breakfast| +30949|AAAAAAAAGOIHAAAA|30949|8|35|49|AM|first|morning|breakfast| +30950|AAAAAAAAHOIHAAAA|30950|8|35|50|AM|first|morning|breakfast| +30951|AAAAAAAAIOIHAAAA|30951|8|35|51|AM|first|morning|breakfast| +30952|AAAAAAAAJOIHAAAA|30952|8|35|52|AM|first|morning|breakfast| +30953|AAAAAAAAKOIHAAAA|30953|8|35|53|AM|first|morning|breakfast| +30954|AAAAAAAALOIHAAAA|30954|8|35|54|AM|first|morning|breakfast| +30955|AAAAAAAAMOIHAAAA|30955|8|35|55|AM|first|morning|breakfast| +30956|AAAAAAAANOIHAAAA|30956|8|35|56|AM|first|morning|breakfast| +30957|AAAAAAAAOOIHAAAA|30957|8|35|57|AM|first|morning|breakfast| +30958|AAAAAAAAPOIHAAAA|30958|8|35|58|AM|first|morning|breakfast| +30959|AAAAAAAAAPIHAAAA|30959|8|35|59|AM|first|morning|breakfast| +30960|AAAAAAAABPIHAAAA|30960|8|36|0|AM|first|morning|breakfast| +30961|AAAAAAAACPIHAAAA|30961|8|36|1|AM|first|morning|breakfast| +30962|AAAAAAAADPIHAAAA|30962|8|36|2|AM|first|morning|breakfast| +30963|AAAAAAAAEPIHAAAA|30963|8|36|3|AM|first|morning|breakfast| +30964|AAAAAAAAFPIHAAAA|30964|8|36|4|AM|first|morning|breakfast| +30965|AAAAAAAAGPIHAAAA|30965|8|36|5|AM|first|morning|breakfast| +30966|AAAAAAAAHPIHAAAA|30966|8|36|6|AM|first|morning|breakfast| +30967|AAAAAAAAIPIHAAAA|30967|8|36|7|AM|first|morning|breakfast| +30968|AAAAAAAAJPIHAAAA|30968|8|36|8|AM|first|morning|breakfast| +30969|AAAAAAAAKPIHAAAA|30969|8|36|9|AM|first|morning|breakfast| +30970|AAAAAAAALPIHAAAA|30970|8|36|10|AM|first|morning|breakfast| +30971|AAAAAAAAMPIHAAAA|30971|8|36|11|AM|first|morning|breakfast| +30972|AAAAAAAANPIHAAAA|30972|8|36|12|AM|first|morning|breakfast| +30973|AAAAAAAAOPIHAAAA|30973|8|36|13|AM|first|morning|breakfast| +30974|AAAAAAAAPPIHAAAA|30974|8|36|14|AM|first|morning|breakfast| +30975|AAAAAAAAAAJHAAAA|30975|8|36|15|AM|first|morning|breakfast| +30976|AAAAAAAABAJHAAAA|30976|8|36|16|AM|first|morning|breakfast| +30977|AAAAAAAACAJHAAAA|30977|8|36|17|AM|first|morning|breakfast| +30978|AAAAAAAADAJHAAAA|30978|8|36|18|AM|first|morning|breakfast| +30979|AAAAAAAAEAJHAAAA|30979|8|36|19|AM|first|morning|breakfast| +30980|AAAAAAAAFAJHAAAA|30980|8|36|20|AM|first|morning|breakfast| +30981|AAAAAAAAGAJHAAAA|30981|8|36|21|AM|first|morning|breakfast| +30982|AAAAAAAAHAJHAAAA|30982|8|36|22|AM|first|morning|breakfast| +30983|AAAAAAAAIAJHAAAA|30983|8|36|23|AM|first|morning|breakfast| +30984|AAAAAAAAJAJHAAAA|30984|8|36|24|AM|first|morning|breakfast| +30985|AAAAAAAAKAJHAAAA|30985|8|36|25|AM|first|morning|breakfast| +30986|AAAAAAAALAJHAAAA|30986|8|36|26|AM|first|morning|breakfast| +30987|AAAAAAAAMAJHAAAA|30987|8|36|27|AM|first|morning|breakfast| +30988|AAAAAAAANAJHAAAA|30988|8|36|28|AM|first|morning|breakfast| +30989|AAAAAAAAOAJHAAAA|30989|8|36|29|AM|first|morning|breakfast| +30990|AAAAAAAAPAJHAAAA|30990|8|36|30|AM|first|morning|breakfast| +30991|AAAAAAAAABJHAAAA|30991|8|36|31|AM|first|morning|breakfast| +30992|AAAAAAAABBJHAAAA|30992|8|36|32|AM|first|morning|breakfast| +30993|AAAAAAAACBJHAAAA|30993|8|36|33|AM|first|morning|breakfast| +30994|AAAAAAAADBJHAAAA|30994|8|36|34|AM|first|morning|breakfast| +30995|AAAAAAAAEBJHAAAA|30995|8|36|35|AM|first|morning|breakfast| +30996|AAAAAAAAFBJHAAAA|30996|8|36|36|AM|first|morning|breakfast| +30997|AAAAAAAAGBJHAAAA|30997|8|36|37|AM|first|morning|breakfast| +30998|AAAAAAAAHBJHAAAA|30998|8|36|38|AM|first|morning|breakfast| +30999|AAAAAAAAIBJHAAAA|30999|8|36|39|AM|first|morning|breakfast| +31000|AAAAAAAAJBJHAAAA|31000|8|36|40|AM|first|morning|breakfast| +31001|AAAAAAAAKBJHAAAA|31001|8|36|41|AM|first|morning|breakfast| +31002|AAAAAAAALBJHAAAA|31002|8|36|42|AM|first|morning|breakfast| +31003|AAAAAAAAMBJHAAAA|31003|8|36|43|AM|first|morning|breakfast| +31004|AAAAAAAANBJHAAAA|31004|8|36|44|AM|first|morning|breakfast| +31005|AAAAAAAAOBJHAAAA|31005|8|36|45|AM|first|morning|breakfast| +31006|AAAAAAAAPBJHAAAA|31006|8|36|46|AM|first|morning|breakfast| +31007|AAAAAAAAACJHAAAA|31007|8|36|47|AM|first|morning|breakfast| +31008|AAAAAAAABCJHAAAA|31008|8|36|48|AM|first|morning|breakfast| +31009|AAAAAAAACCJHAAAA|31009|8|36|49|AM|first|morning|breakfast| +31010|AAAAAAAADCJHAAAA|31010|8|36|50|AM|first|morning|breakfast| +31011|AAAAAAAAECJHAAAA|31011|8|36|51|AM|first|morning|breakfast| +31012|AAAAAAAAFCJHAAAA|31012|8|36|52|AM|first|morning|breakfast| +31013|AAAAAAAAGCJHAAAA|31013|8|36|53|AM|first|morning|breakfast| +31014|AAAAAAAAHCJHAAAA|31014|8|36|54|AM|first|morning|breakfast| +31015|AAAAAAAAICJHAAAA|31015|8|36|55|AM|first|morning|breakfast| +31016|AAAAAAAAJCJHAAAA|31016|8|36|56|AM|first|morning|breakfast| +31017|AAAAAAAAKCJHAAAA|31017|8|36|57|AM|first|morning|breakfast| +31018|AAAAAAAALCJHAAAA|31018|8|36|58|AM|first|morning|breakfast| +31019|AAAAAAAAMCJHAAAA|31019|8|36|59|AM|first|morning|breakfast| +31020|AAAAAAAANCJHAAAA|31020|8|37|0|AM|first|morning|breakfast| +31021|AAAAAAAAOCJHAAAA|31021|8|37|1|AM|first|morning|breakfast| +31022|AAAAAAAAPCJHAAAA|31022|8|37|2|AM|first|morning|breakfast| +31023|AAAAAAAAADJHAAAA|31023|8|37|3|AM|first|morning|breakfast| +31024|AAAAAAAABDJHAAAA|31024|8|37|4|AM|first|morning|breakfast| +31025|AAAAAAAACDJHAAAA|31025|8|37|5|AM|first|morning|breakfast| +31026|AAAAAAAADDJHAAAA|31026|8|37|6|AM|first|morning|breakfast| +31027|AAAAAAAAEDJHAAAA|31027|8|37|7|AM|first|morning|breakfast| +31028|AAAAAAAAFDJHAAAA|31028|8|37|8|AM|first|morning|breakfast| +31029|AAAAAAAAGDJHAAAA|31029|8|37|9|AM|first|morning|breakfast| +31030|AAAAAAAAHDJHAAAA|31030|8|37|10|AM|first|morning|breakfast| +31031|AAAAAAAAIDJHAAAA|31031|8|37|11|AM|first|morning|breakfast| +31032|AAAAAAAAJDJHAAAA|31032|8|37|12|AM|first|morning|breakfast| +31033|AAAAAAAAKDJHAAAA|31033|8|37|13|AM|first|morning|breakfast| +31034|AAAAAAAALDJHAAAA|31034|8|37|14|AM|first|morning|breakfast| +31035|AAAAAAAAMDJHAAAA|31035|8|37|15|AM|first|morning|breakfast| +31036|AAAAAAAANDJHAAAA|31036|8|37|16|AM|first|morning|breakfast| +31037|AAAAAAAAODJHAAAA|31037|8|37|17|AM|first|morning|breakfast| +31038|AAAAAAAAPDJHAAAA|31038|8|37|18|AM|first|morning|breakfast| +31039|AAAAAAAAAEJHAAAA|31039|8|37|19|AM|first|morning|breakfast| +31040|AAAAAAAABEJHAAAA|31040|8|37|20|AM|first|morning|breakfast| +31041|AAAAAAAACEJHAAAA|31041|8|37|21|AM|first|morning|breakfast| +31042|AAAAAAAADEJHAAAA|31042|8|37|22|AM|first|morning|breakfast| +31043|AAAAAAAAEEJHAAAA|31043|8|37|23|AM|first|morning|breakfast| +31044|AAAAAAAAFEJHAAAA|31044|8|37|24|AM|first|morning|breakfast| +31045|AAAAAAAAGEJHAAAA|31045|8|37|25|AM|first|morning|breakfast| +31046|AAAAAAAAHEJHAAAA|31046|8|37|26|AM|first|morning|breakfast| +31047|AAAAAAAAIEJHAAAA|31047|8|37|27|AM|first|morning|breakfast| +31048|AAAAAAAAJEJHAAAA|31048|8|37|28|AM|first|morning|breakfast| +31049|AAAAAAAAKEJHAAAA|31049|8|37|29|AM|first|morning|breakfast| +31050|AAAAAAAALEJHAAAA|31050|8|37|30|AM|first|morning|breakfast| +31051|AAAAAAAAMEJHAAAA|31051|8|37|31|AM|first|morning|breakfast| +31052|AAAAAAAANEJHAAAA|31052|8|37|32|AM|first|morning|breakfast| +31053|AAAAAAAAOEJHAAAA|31053|8|37|33|AM|first|morning|breakfast| +31054|AAAAAAAAPEJHAAAA|31054|8|37|34|AM|first|morning|breakfast| +31055|AAAAAAAAAFJHAAAA|31055|8|37|35|AM|first|morning|breakfast| +31056|AAAAAAAABFJHAAAA|31056|8|37|36|AM|first|morning|breakfast| +31057|AAAAAAAACFJHAAAA|31057|8|37|37|AM|first|morning|breakfast| +31058|AAAAAAAADFJHAAAA|31058|8|37|38|AM|first|morning|breakfast| +31059|AAAAAAAAEFJHAAAA|31059|8|37|39|AM|first|morning|breakfast| +31060|AAAAAAAAFFJHAAAA|31060|8|37|40|AM|first|morning|breakfast| +31061|AAAAAAAAGFJHAAAA|31061|8|37|41|AM|first|morning|breakfast| +31062|AAAAAAAAHFJHAAAA|31062|8|37|42|AM|first|morning|breakfast| +31063|AAAAAAAAIFJHAAAA|31063|8|37|43|AM|first|morning|breakfast| +31064|AAAAAAAAJFJHAAAA|31064|8|37|44|AM|first|morning|breakfast| +31065|AAAAAAAAKFJHAAAA|31065|8|37|45|AM|first|morning|breakfast| +31066|AAAAAAAALFJHAAAA|31066|8|37|46|AM|first|morning|breakfast| +31067|AAAAAAAAMFJHAAAA|31067|8|37|47|AM|first|morning|breakfast| +31068|AAAAAAAANFJHAAAA|31068|8|37|48|AM|first|morning|breakfast| +31069|AAAAAAAAOFJHAAAA|31069|8|37|49|AM|first|morning|breakfast| +31070|AAAAAAAAPFJHAAAA|31070|8|37|50|AM|first|morning|breakfast| +31071|AAAAAAAAAGJHAAAA|31071|8|37|51|AM|first|morning|breakfast| +31072|AAAAAAAABGJHAAAA|31072|8|37|52|AM|first|morning|breakfast| +31073|AAAAAAAACGJHAAAA|31073|8|37|53|AM|first|morning|breakfast| +31074|AAAAAAAADGJHAAAA|31074|8|37|54|AM|first|morning|breakfast| +31075|AAAAAAAAEGJHAAAA|31075|8|37|55|AM|first|morning|breakfast| +31076|AAAAAAAAFGJHAAAA|31076|8|37|56|AM|first|morning|breakfast| +31077|AAAAAAAAGGJHAAAA|31077|8|37|57|AM|first|morning|breakfast| +31078|AAAAAAAAHGJHAAAA|31078|8|37|58|AM|first|morning|breakfast| +31079|AAAAAAAAIGJHAAAA|31079|8|37|59|AM|first|morning|breakfast| +31080|AAAAAAAAJGJHAAAA|31080|8|38|0|AM|first|morning|breakfast| +31081|AAAAAAAAKGJHAAAA|31081|8|38|1|AM|first|morning|breakfast| +31082|AAAAAAAALGJHAAAA|31082|8|38|2|AM|first|morning|breakfast| +31083|AAAAAAAAMGJHAAAA|31083|8|38|3|AM|first|morning|breakfast| +31084|AAAAAAAANGJHAAAA|31084|8|38|4|AM|first|morning|breakfast| +31085|AAAAAAAAOGJHAAAA|31085|8|38|5|AM|first|morning|breakfast| +31086|AAAAAAAAPGJHAAAA|31086|8|38|6|AM|first|morning|breakfast| +31087|AAAAAAAAAHJHAAAA|31087|8|38|7|AM|first|morning|breakfast| +31088|AAAAAAAABHJHAAAA|31088|8|38|8|AM|first|morning|breakfast| +31089|AAAAAAAACHJHAAAA|31089|8|38|9|AM|first|morning|breakfast| +31090|AAAAAAAADHJHAAAA|31090|8|38|10|AM|first|morning|breakfast| +31091|AAAAAAAAEHJHAAAA|31091|8|38|11|AM|first|morning|breakfast| +31092|AAAAAAAAFHJHAAAA|31092|8|38|12|AM|first|morning|breakfast| +31093|AAAAAAAAGHJHAAAA|31093|8|38|13|AM|first|morning|breakfast| +31094|AAAAAAAAHHJHAAAA|31094|8|38|14|AM|first|morning|breakfast| +31095|AAAAAAAAIHJHAAAA|31095|8|38|15|AM|first|morning|breakfast| +31096|AAAAAAAAJHJHAAAA|31096|8|38|16|AM|first|morning|breakfast| +31097|AAAAAAAAKHJHAAAA|31097|8|38|17|AM|first|morning|breakfast| +31098|AAAAAAAALHJHAAAA|31098|8|38|18|AM|first|morning|breakfast| +31099|AAAAAAAAMHJHAAAA|31099|8|38|19|AM|first|morning|breakfast| +31100|AAAAAAAANHJHAAAA|31100|8|38|20|AM|first|morning|breakfast| +31101|AAAAAAAAOHJHAAAA|31101|8|38|21|AM|first|morning|breakfast| +31102|AAAAAAAAPHJHAAAA|31102|8|38|22|AM|first|morning|breakfast| +31103|AAAAAAAAAIJHAAAA|31103|8|38|23|AM|first|morning|breakfast| +31104|AAAAAAAABIJHAAAA|31104|8|38|24|AM|first|morning|breakfast| +31105|AAAAAAAACIJHAAAA|31105|8|38|25|AM|first|morning|breakfast| +31106|AAAAAAAADIJHAAAA|31106|8|38|26|AM|first|morning|breakfast| +31107|AAAAAAAAEIJHAAAA|31107|8|38|27|AM|first|morning|breakfast| +31108|AAAAAAAAFIJHAAAA|31108|8|38|28|AM|first|morning|breakfast| +31109|AAAAAAAAGIJHAAAA|31109|8|38|29|AM|first|morning|breakfast| +31110|AAAAAAAAHIJHAAAA|31110|8|38|30|AM|first|morning|breakfast| +31111|AAAAAAAAIIJHAAAA|31111|8|38|31|AM|first|morning|breakfast| +31112|AAAAAAAAJIJHAAAA|31112|8|38|32|AM|first|morning|breakfast| +31113|AAAAAAAAKIJHAAAA|31113|8|38|33|AM|first|morning|breakfast| +31114|AAAAAAAALIJHAAAA|31114|8|38|34|AM|first|morning|breakfast| +31115|AAAAAAAAMIJHAAAA|31115|8|38|35|AM|first|morning|breakfast| +31116|AAAAAAAANIJHAAAA|31116|8|38|36|AM|first|morning|breakfast| +31117|AAAAAAAAOIJHAAAA|31117|8|38|37|AM|first|morning|breakfast| +31118|AAAAAAAAPIJHAAAA|31118|8|38|38|AM|first|morning|breakfast| +31119|AAAAAAAAAJJHAAAA|31119|8|38|39|AM|first|morning|breakfast| +31120|AAAAAAAABJJHAAAA|31120|8|38|40|AM|first|morning|breakfast| +31121|AAAAAAAACJJHAAAA|31121|8|38|41|AM|first|morning|breakfast| +31122|AAAAAAAADJJHAAAA|31122|8|38|42|AM|first|morning|breakfast| +31123|AAAAAAAAEJJHAAAA|31123|8|38|43|AM|first|morning|breakfast| +31124|AAAAAAAAFJJHAAAA|31124|8|38|44|AM|first|morning|breakfast| +31125|AAAAAAAAGJJHAAAA|31125|8|38|45|AM|first|morning|breakfast| +31126|AAAAAAAAHJJHAAAA|31126|8|38|46|AM|first|morning|breakfast| +31127|AAAAAAAAIJJHAAAA|31127|8|38|47|AM|first|morning|breakfast| +31128|AAAAAAAAJJJHAAAA|31128|8|38|48|AM|first|morning|breakfast| +31129|AAAAAAAAKJJHAAAA|31129|8|38|49|AM|first|morning|breakfast| +31130|AAAAAAAALJJHAAAA|31130|8|38|50|AM|first|morning|breakfast| +31131|AAAAAAAAMJJHAAAA|31131|8|38|51|AM|first|morning|breakfast| +31132|AAAAAAAANJJHAAAA|31132|8|38|52|AM|first|morning|breakfast| +31133|AAAAAAAAOJJHAAAA|31133|8|38|53|AM|first|morning|breakfast| +31134|AAAAAAAAPJJHAAAA|31134|8|38|54|AM|first|morning|breakfast| +31135|AAAAAAAAAKJHAAAA|31135|8|38|55|AM|first|morning|breakfast| +31136|AAAAAAAABKJHAAAA|31136|8|38|56|AM|first|morning|breakfast| +31137|AAAAAAAACKJHAAAA|31137|8|38|57|AM|first|morning|breakfast| +31138|AAAAAAAADKJHAAAA|31138|8|38|58|AM|first|morning|breakfast| +31139|AAAAAAAAEKJHAAAA|31139|8|38|59|AM|first|morning|breakfast| +31140|AAAAAAAAFKJHAAAA|31140|8|39|0|AM|first|morning|breakfast| +31141|AAAAAAAAGKJHAAAA|31141|8|39|1|AM|first|morning|breakfast| +31142|AAAAAAAAHKJHAAAA|31142|8|39|2|AM|first|morning|breakfast| +31143|AAAAAAAAIKJHAAAA|31143|8|39|3|AM|first|morning|breakfast| +31144|AAAAAAAAJKJHAAAA|31144|8|39|4|AM|first|morning|breakfast| +31145|AAAAAAAAKKJHAAAA|31145|8|39|5|AM|first|morning|breakfast| +31146|AAAAAAAALKJHAAAA|31146|8|39|6|AM|first|morning|breakfast| +31147|AAAAAAAAMKJHAAAA|31147|8|39|7|AM|first|morning|breakfast| +31148|AAAAAAAANKJHAAAA|31148|8|39|8|AM|first|morning|breakfast| +31149|AAAAAAAAOKJHAAAA|31149|8|39|9|AM|first|morning|breakfast| +31150|AAAAAAAAPKJHAAAA|31150|8|39|10|AM|first|morning|breakfast| +31151|AAAAAAAAALJHAAAA|31151|8|39|11|AM|first|morning|breakfast| +31152|AAAAAAAABLJHAAAA|31152|8|39|12|AM|first|morning|breakfast| +31153|AAAAAAAACLJHAAAA|31153|8|39|13|AM|first|morning|breakfast| +31154|AAAAAAAADLJHAAAA|31154|8|39|14|AM|first|morning|breakfast| +31155|AAAAAAAAELJHAAAA|31155|8|39|15|AM|first|morning|breakfast| +31156|AAAAAAAAFLJHAAAA|31156|8|39|16|AM|first|morning|breakfast| +31157|AAAAAAAAGLJHAAAA|31157|8|39|17|AM|first|morning|breakfast| +31158|AAAAAAAAHLJHAAAA|31158|8|39|18|AM|first|morning|breakfast| +31159|AAAAAAAAILJHAAAA|31159|8|39|19|AM|first|morning|breakfast| +31160|AAAAAAAAJLJHAAAA|31160|8|39|20|AM|first|morning|breakfast| +31161|AAAAAAAAKLJHAAAA|31161|8|39|21|AM|first|morning|breakfast| +31162|AAAAAAAALLJHAAAA|31162|8|39|22|AM|first|morning|breakfast| +31163|AAAAAAAAMLJHAAAA|31163|8|39|23|AM|first|morning|breakfast| +31164|AAAAAAAANLJHAAAA|31164|8|39|24|AM|first|morning|breakfast| +31165|AAAAAAAAOLJHAAAA|31165|8|39|25|AM|first|morning|breakfast| +31166|AAAAAAAAPLJHAAAA|31166|8|39|26|AM|first|morning|breakfast| +31167|AAAAAAAAAMJHAAAA|31167|8|39|27|AM|first|morning|breakfast| +31168|AAAAAAAABMJHAAAA|31168|8|39|28|AM|first|morning|breakfast| +31169|AAAAAAAACMJHAAAA|31169|8|39|29|AM|first|morning|breakfast| +31170|AAAAAAAADMJHAAAA|31170|8|39|30|AM|first|morning|breakfast| +31171|AAAAAAAAEMJHAAAA|31171|8|39|31|AM|first|morning|breakfast| +31172|AAAAAAAAFMJHAAAA|31172|8|39|32|AM|first|morning|breakfast| +31173|AAAAAAAAGMJHAAAA|31173|8|39|33|AM|first|morning|breakfast| +31174|AAAAAAAAHMJHAAAA|31174|8|39|34|AM|first|morning|breakfast| +31175|AAAAAAAAIMJHAAAA|31175|8|39|35|AM|first|morning|breakfast| +31176|AAAAAAAAJMJHAAAA|31176|8|39|36|AM|first|morning|breakfast| +31177|AAAAAAAAKMJHAAAA|31177|8|39|37|AM|first|morning|breakfast| +31178|AAAAAAAALMJHAAAA|31178|8|39|38|AM|first|morning|breakfast| +31179|AAAAAAAAMMJHAAAA|31179|8|39|39|AM|first|morning|breakfast| +31180|AAAAAAAANMJHAAAA|31180|8|39|40|AM|first|morning|breakfast| +31181|AAAAAAAAOMJHAAAA|31181|8|39|41|AM|first|morning|breakfast| +31182|AAAAAAAAPMJHAAAA|31182|8|39|42|AM|first|morning|breakfast| +31183|AAAAAAAAANJHAAAA|31183|8|39|43|AM|first|morning|breakfast| +31184|AAAAAAAABNJHAAAA|31184|8|39|44|AM|first|morning|breakfast| +31185|AAAAAAAACNJHAAAA|31185|8|39|45|AM|first|morning|breakfast| +31186|AAAAAAAADNJHAAAA|31186|8|39|46|AM|first|morning|breakfast| +31187|AAAAAAAAENJHAAAA|31187|8|39|47|AM|first|morning|breakfast| +31188|AAAAAAAAFNJHAAAA|31188|8|39|48|AM|first|morning|breakfast| +31189|AAAAAAAAGNJHAAAA|31189|8|39|49|AM|first|morning|breakfast| +31190|AAAAAAAAHNJHAAAA|31190|8|39|50|AM|first|morning|breakfast| +31191|AAAAAAAAINJHAAAA|31191|8|39|51|AM|first|morning|breakfast| +31192|AAAAAAAAJNJHAAAA|31192|8|39|52|AM|first|morning|breakfast| +31193|AAAAAAAAKNJHAAAA|31193|8|39|53|AM|first|morning|breakfast| +31194|AAAAAAAALNJHAAAA|31194|8|39|54|AM|first|morning|breakfast| +31195|AAAAAAAAMNJHAAAA|31195|8|39|55|AM|first|morning|breakfast| +31196|AAAAAAAANNJHAAAA|31196|8|39|56|AM|first|morning|breakfast| +31197|AAAAAAAAONJHAAAA|31197|8|39|57|AM|first|morning|breakfast| +31198|AAAAAAAAPNJHAAAA|31198|8|39|58|AM|first|morning|breakfast| +31199|AAAAAAAAAOJHAAAA|31199|8|39|59|AM|first|morning|breakfast| +31200|AAAAAAAABOJHAAAA|31200|8|40|0|AM|first|morning|breakfast| +31201|AAAAAAAACOJHAAAA|31201|8|40|1|AM|first|morning|breakfast| +31202|AAAAAAAADOJHAAAA|31202|8|40|2|AM|first|morning|breakfast| +31203|AAAAAAAAEOJHAAAA|31203|8|40|3|AM|first|morning|breakfast| +31204|AAAAAAAAFOJHAAAA|31204|8|40|4|AM|first|morning|breakfast| +31205|AAAAAAAAGOJHAAAA|31205|8|40|5|AM|first|morning|breakfast| +31206|AAAAAAAAHOJHAAAA|31206|8|40|6|AM|first|morning|breakfast| +31207|AAAAAAAAIOJHAAAA|31207|8|40|7|AM|first|morning|breakfast| +31208|AAAAAAAAJOJHAAAA|31208|8|40|8|AM|first|morning|breakfast| +31209|AAAAAAAAKOJHAAAA|31209|8|40|9|AM|first|morning|breakfast| +31210|AAAAAAAALOJHAAAA|31210|8|40|10|AM|first|morning|breakfast| +31211|AAAAAAAAMOJHAAAA|31211|8|40|11|AM|first|morning|breakfast| +31212|AAAAAAAANOJHAAAA|31212|8|40|12|AM|first|morning|breakfast| +31213|AAAAAAAAOOJHAAAA|31213|8|40|13|AM|first|morning|breakfast| +31214|AAAAAAAAPOJHAAAA|31214|8|40|14|AM|first|morning|breakfast| +31215|AAAAAAAAAPJHAAAA|31215|8|40|15|AM|first|morning|breakfast| +31216|AAAAAAAABPJHAAAA|31216|8|40|16|AM|first|morning|breakfast| +31217|AAAAAAAACPJHAAAA|31217|8|40|17|AM|first|morning|breakfast| +31218|AAAAAAAADPJHAAAA|31218|8|40|18|AM|first|morning|breakfast| +31219|AAAAAAAAEPJHAAAA|31219|8|40|19|AM|first|morning|breakfast| +31220|AAAAAAAAFPJHAAAA|31220|8|40|20|AM|first|morning|breakfast| +31221|AAAAAAAAGPJHAAAA|31221|8|40|21|AM|first|morning|breakfast| +31222|AAAAAAAAHPJHAAAA|31222|8|40|22|AM|first|morning|breakfast| +31223|AAAAAAAAIPJHAAAA|31223|8|40|23|AM|first|morning|breakfast| +31224|AAAAAAAAJPJHAAAA|31224|8|40|24|AM|first|morning|breakfast| +31225|AAAAAAAAKPJHAAAA|31225|8|40|25|AM|first|morning|breakfast| +31226|AAAAAAAALPJHAAAA|31226|8|40|26|AM|first|morning|breakfast| +31227|AAAAAAAAMPJHAAAA|31227|8|40|27|AM|first|morning|breakfast| +31228|AAAAAAAANPJHAAAA|31228|8|40|28|AM|first|morning|breakfast| +31229|AAAAAAAAOPJHAAAA|31229|8|40|29|AM|first|morning|breakfast| +31230|AAAAAAAAPPJHAAAA|31230|8|40|30|AM|first|morning|breakfast| +31231|AAAAAAAAAAKHAAAA|31231|8|40|31|AM|first|morning|breakfast| +31232|AAAAAAAABAKHAAAA|31232|8|40|32|AM|first|morning|breakfast| +31233|AAAAAAAACAKHAAAA|31233|8|40|33|AM|first|morning|breakfast| +31234|AAAAAAAADAKHAAAA|31234|8|40|34|AM|first|morning|breakfast| +31235|AAAAAAAAEAKHAAAA|31235|8|40|35|AM|first|morning|breakfast| +31236|AAAAAAAAFAKHAAAA|31236|8|40|36|AM|first|morning|breakfast| +31237|AAAAAAAAGAKHAAAA|31237|8|40|37|AM|first|morning|breakfast| +31238|AAAAAAAAHAKHAAAA|31238|8|40|38|AM|first|morning|breakfast| +31239|AAAAAAAAIAKHAAAA|31239|8|40|39|AM|first|morning|breakfast| +31240|AAAAAAAAJAKHAAAA|31240|8|40|40|AM|first|morning|breakfast| +31241|AAAAAAAAKAKHAAAA|31241|8|40|41|AM|first|morning|breakfast| +31242|AAAAAAAALAKHAAAA|31242|8|40|42|AM|first|morning|breakfast| +31243|AAAAAAAAMAKHAAAA|31243|8|40|43|AM|first|morning|breakfast| +31244|AAAAAAAANAKHAAAA|31244|8|40|44|AM|first|morning|breakfast| +31245|AAAAAAAAOAKHAAAA|31245|8|40|45|AM|first|morning|breakfast| +31246|AAAAAAAAPAKHAAAA|31246|8|40|46|AM|first|morning|breakfast| +31247|AAAAAAAAABKHAAAA|31247|8|40|47|AM|first|morning|breakfast| +31248|AAAAAAAABBKHAAAA|31248|8|40|48|AM|first|morning|breakfast| +31249|AAAAAAAACBKHAAAA|31249|8|40|49|AM|first|morning|breakfast| +31250|AAAAAAAADBKHAAAA|31250|8|40|50|AM|first|morning|breakfast| +31251|AAAAAAAAEBKHAAAA|31251|8|40|51|AM|first|morning|breakfast| +31252|AAAAAAAAFBKHAAAA|31252|8|40|52|AM|first|morning|breakfast| +31253|AAAAAAAAGBKHAAAA|31253|8|40|53|AM|first|morning|breakfast| +31254|AAAAAAAAHBKHAAAA|31254|8|40|54|AM|first|morning|breakfast| +31255|AAAAAAAAIBKHAAAA|31255|8|40|55|AM|first|morning|breakfast| +31256|AAAAAAAAJBKHAAAA|31256|8|40|56|AM|first|morning|breakfast| +31257|AAAAAAAAKBKHAAAA|31257|8|40|57|AM|first|morning|breakfast| +31258|AAAAAAAALBKHAAAA|31258|8|40|58|AM|first|morning|breakfast| +31259|AAAAAAAAMBKHAAAA|31259|8|40|59|AM|first|morning|breakfast| +31260|AAAAAAAANBKHAAAA|31260|8|41|0|AM|first|morning|breakfast| +31261|AAAAAAAAOBKHAAAA|31261|8|41|1|AM|first|morning|breakfast| +31262|AAAAAAAAPBKHAAAA|31262|8|41|2|AM|first|morning|breakfast| +31263|AAAAAAAAACKHAAAA|31263|8|41|3|AM|first|morning|breakfast| +31264|AAAAAAAABCKHAAAA|31264|8|41|4|AM|first|morning|breakfast| +31265|AAAAAAAACCKHAAAA|31265|8|41|5|AM|first|morning|breakfast| +31266|AAAAAAAADCKHAAAA|31266|8|41|6|AM|first|morning|breakfast| +31267|AAAAAAAAECKHAAAA|31267|8|41|7|AM|first|morning|breakfast| +31268|AAAAAAAAFCKHAAAA|31268|8|41|8|AM|first|morning|breakfast| +31269|AAAAAAAAGCKHAAAA|31269|8|41|9|AM|first|morning|breakfast| +31270|AAAAAAAAHCKHAAAA|31270|8|41|10|AM|first|morning|breakfast| +31271|AAAAAAAAICKHAAAA|31271|8|41|11|AM|first|morning|breakfast| +31272|AAAAAAAAJCKHAAAA|31272|8|41|12|AM|first|morning|breakfast| +31273|AAAAAAAAKCKHAAAA|31273|8|41|13|AM|first|morning|breakfast| +31274|AAAAAAAALCKHAAAA|31274|8|41|14|AM|first|morning|breakfast| +31275|AAAAAAAAMCKHAAAA|31275|8|41|15|AM|first|morning|breakfast| +31276|AAAAAAAANCKHAAAA|31276|8|41|16|AM|first|morning|breakfast| +31277|AAAAAAAAOCKHAAAA|31277|8|41|17|AM|first|morning|breakfast| +31278|AAAAAAAAPCKHAAAA|31278|8|41|18|AM|first|morning|breakfast| +31279|AAAAAAAAADKHAAAA|31279|8|41|19|AM|first|morning|breakfast| +31280|AAAAAAAABDKHAAAA|31280|8|41|20|AM|first|morning|breakfast| +31281|AAAAAAAACDKHAAAA|31281|8|41|21|AM|first|morning|breakfast| +31282|AAAAAAAADDKHAAAA|31282|8|41|22|AM|first|morning|breakfast| +31283|AAAAAAAAEDKHAAAA|31283|8|41|23|AM|first|morning|breakfast| +31284|AAAAAAAAFDKHAAAA|31284|8|41|24|AM|first|morning|breakfast| +31285|AAAAAAAAGDKHAAAA|31285|8|41|25|AM|first|morning|breakfast| +31286|AAAAAAAAHDKHAAAA|31286|8|41|26|AM|first|morning|breakfast| +31287|AAAAAAAAIDKHAAAA|31287|8|41|27|AM|first|morning|breakfast| +31288|AAAAAAAAJDKHAAAA|31288|8|41|28|AM|first|morning|breakfast| +31289|AAAAAAAAKDKHAAAA|31289|8|41|29|AM|first|morning|breakfast| +31290|AAAAAAAALDKHAAAA|31290|8|41|30|AM|first|morning|breakfast| +31291|AAAAAAAAMDKHAAAA|31291|8|41|31|AM|first|morning|breakfast| +31292|AAAAAAAANDKHAAAA|31292|8|41|32|AM|first|morning|breakfast| +31293|AAAAAAAAODKHAAAA|31293|8|41|33|AM|first|morning|breakfast| +31294|AAAAAAAAPDKHAAAA|31294|8|41|34|AM|first|morning|breakfast| +31295|AAAAAAAAAEKHAAAA|31295|8|41|35|AM|first|morning|breakfast| +31296|AAAAAAAABEKHAAAA|31296|8|41|36|AM|first|morning|breakfast| +31297|AAAAAAAACEKHAAAA|31297|8|41|37|AM|first|morning|breakfast| +31298|AAAAAAAADEKHAAAA|31298|8|41|38|AM|first|morning|breakfast| +31299|AAAAAAAAEEKHAAAA|31299|8|41|39|AM|first|morning|breakfast| +31300|AAAAAAAAFEKHAAAA|31300|8|41|40|AM|first|morning|breakfast| +31301|AAAAAAAAGEKHAAAA|31301|8|41|41|AM|first|morning|breakfast| +31302|AAAAAAAAHEKHAAAA|31302|8|41|42|AM|first|morning|breakfast| +31303|AAAAAAAAIEKHAAAA|31303|8|41|43|AM|first|morning|breakfast| +31304|AAAAAAAAJEKHAAAA|31304|8|41|44|AM|first|morning|breakfast| +31305|AAAAAAAAKEKHAAAA|31305|8|41|45|AM|first|morning|breakfast| +31306|AAAAAAAALEKHAAAA|31306|8|41|46|AM|first|morning|breakfast| +31307|AAAAAAAAMEKHAAAA|31307|8|41|47|AM|first|morning|breakfast| +31308|AAAAAAAANEKHAAAA|31308|8|41|48|AM|first|morning|breakfast| +31309|AAAAAAAAOEKHAAAA|31309|8|41|49|AM|first|morning|breakfast| +31310|AAAAAAAAPEKHAAAA|31310|8|41|50|AM|first|morning|breakfast| +31311|AAAAAAAAAFKHAAAA|31311|8|41|51|AM|first|morning|breakfast| +31312|AAAAAAAABFKHAAAA|31312|8|41|52|AM|first|morning|breakfast| +31313|AAAAAAAACFKHAAAA|31313|8|41|53|AM|first|morning|breakfast| +31314|AAAAAAAADFKHAAAA|31314|8|41|54|AM|first|morning|breakfast| +31315|AAAAAAAAEFKHAAAA|31315|8|41|55|AM|first|morning|breakfast| +31316|AAAAAAAAFFKHAAAA|31316|8|41|56|AM|first|morning|breakfast| +31317|AAAAAAAAGFKHAAAA|31317|8|41|57|AM|first|morning|breakfast| +31318|AAAAAAAAHFKHAAAA|31318|8|41|58|AM|first|morning|breakfast| +31319|AAAAAAAAIFKHAAAA|31319|8|41|59|AM|first|morning|breakfast| +31320|AAAAAAAAJFKHAAAA|31320|8|42|0|AM|first|morning|breakfast| +31321|AAAAAAAAKFKHAAAA|31321|8|42|1|AM|first|morning|breakfast| +31322|AAAAAAAALFKHAAAA|31322|8|42|2|AM|first|morning|breakfast| +31323|AAAAAAAAMFKHAAAA|31323|8|42|3|AM|first|morning|breakfast| +31324|AAAAAAAANFKHAAAA|31324|8|42|4|AM|first|morning|breakfast| +31325|AAAAAAAAOFKHAAAA|31325|8|42|5|AM|first|morning|breakfast| +31326|AAAAAAAAPFKHAAAA|31326|8|42|6|AM|first|morning|breakfast| +31327|AAAAAAAAAGKHAAAA|31327|8|42|7|AM|first|morning|breakfast| +31328|AAAAAAAABGKHAAAA|31328|8|42|8|AM|first|morning|breakfast| +31329|AAAAAAAACGKHAAAA|31329|8|42|9|AM|first|morning|breakfast| +31330|AAAAAAAADGKHAAAA|31330|8|42|10|AM|first|morning|breakfast| +31331|AAAAAAAAEGKHAAAA|31331|8|42|11|AM|first|morning|breakfast| +31332|AAAAAAAAFGKHAAAA|31332|8|42|12|AM|first|morning|breakfast| +31333|AAAAAAAAGGKHAAAA|31333|8|42|13|AM|first|morning|breakfast| +31334|AAAAAAAAHGKHAAAA|31334|8|42|14|AM|first|morning|breakfast| +31335|AAAAAAAAIGKHAAAA|31335|8|42|15|AM|first|morning|breakfast| +31336|AAAAAAAAJGKHAAAA|31336|8|42|16|AM|first|morning|breakfast| +31337|AAAAAAAAKGKHAAAA|31337|8|42|17|AM|first|morning|breakfast| +31338|AAAAAAAALGKHAAAA|31338|8|42|18|AM|first|morning|breakfast| +31339|AAAAAAAAMGKHAAAA|31339|8|42|19|AM|first|morning|breakfast| +31340|AAAAAAAANGKHAAAA|31340|8|42|20|AM|first|morning|breakfast| +31341|AAAAAAAAOGKHAAAA|31341|8|42|21|AM|first|morning|breakfast| +31342|AAAAAAAAPGKHAAAA|31342|8|42|22|AM|first|morning|breakfast| +31343|AAAAAAAAAHKHAAAA|31343|8|42|23|AM|first|morning|breakfast| +31344|AAAAAAAABHKHAAAA|31344|8|42|24|AM|first|morning|breakfast| +31345|AAAAAAAACHKHAAAA|31345|8|42|25|AM|first|morning|breakfast| +31346|AAAAAAAADHKHAAAA|31346|8|42|26|AM|first|morning|breakfast| +31347|AAAAAAAAEHKHAAAA|31347|8|42|27|AM|first|morning|breakfast| +31348|AAAAAAAAFHKHAAAA|31348|8|42|28|AM|first|morning|breakfast| +31349|AAAAAAAAGHKHAAAA|31349|8|42|29|AM|first|morning|breakfast| +31350|AAAAAAAAHHKHAAAA|31350|8|42|30|AM|first|morning|breakfast| +31351|AAAAAAAAIHKHAAAA|31351|8|42|31|AM|first|morning|breakfast| +31352|AAAAAAAAJHKHAAAA|31352|8|42|32|AM|first|morning|breakfast| +31353|AAAAAAAAKHKHAAAA|31353|8|42|33|AM|first|morning|breakfast| +31354|AAAAAAAALHKHAAAA|31354|8|42|34|AM|first|morning|breakfast| +31355|AAAAAAAAMHKHAAAA|31355|8|42|35|AM|first|morning|breakfast| +31356|AAAAAAAANHKHAAAA|31356|8|42|36|AM|first|morning|breakfast| +31357|AAAAAAAAOHKHAAAA|31357|8|42|37|AM|first|morning|breakfast| +31358|AAAAAAAAPHKHAAAA|31358|8|42|38|AM|first|morning|breakfast| +31359|AAAAAAAAAIKHAAAA|31359|8|42|39|AM|first|morning|breakfast| +31360|AAAAAAAABIKHAAAA|31360|8|42|40|AM|first|morning|breakfast| +31361|AAAAAAAACIKHAAAA|31361|8|42|41|AM|first|morning|breakfast| +31362|AAAAAAAADIKHAAAA|31362|8|42|42|AM|first|morning|breakfast| +31363|AAAAAAAAEIKHAAAA|31363|8|42|43|AM|first|morning|breakfast| +31364|AAAAAAAAFIKHAAAA|31364|8|42|44|AM|first|morning|breakfast| +31365|AAAAAAAAGIKHAAAA|31365|8|42|45|AM|first|morning|breakfast| +31366|AAAAAAAAHIKHAAAA|31366|8|42|46|AM|first|morning|breakfast| +31367|AAAAAAAAIIKHAAAA|31367|8|42|47|AM|first|morning|breakfast| +31368|AAAAAAAAJIKHAAAA|31368|8|42|48|AM|first|morning|breakfast| +31369|AAAAAAAAKIKHAAAA|31369|8|42|49|AM|first|morning|breakfast| +31370|AAAAAAAALIKHAAAA|31370|8|42|50|AM|first|morning|breakfast| +31371|AAAAAAAAMIKHAAAA|31371|8|42|51|AM|first|morning|breakfast| +31372|AAAAAAAANIKHAAAA|31372|8|42|52|AM|first|morning|breakfast| +31373|AAAAAAAAOIKHAAAA|31373|8|42|53|AM|first|morning|breakfast| +31374|AAAAAAAAPIKHAAAA|31374|8|42|54|AM|first|morning|breakfast| +31375|AAAAAAAAAJKHAAAA|31375|8|42|55|AM|first|morning|breakfast| +31376|AAAAAAAABJKHAAAA|31376|8|42|56|AM|first|morning|breakfast| +31377|AAAAAAAACJKHAAAA|31377|8|42|57|AM|first|morning|breakfast| +31378|AAAAAAAADJKHAAAA|31378|8|42|58|AM|first|morning|breakfast| +31379|AAAAAAAAEJKHAAAA|31379|8|42|59|AM|first|morning|breakfast| +31380|AAAAAAAAFJKHAAAA|31380|8|43|0|AM|first|morning|breakfast| +31381|AAAAAAAAGJKHAAAA|31381|8|43|1|AM|first|morning|breakfast| +31382|AAAAAAAAHJKHAAAA|31382|8|43|2|AM|first|morning|breakfast| +31383|AAAAAAAAIJKHAAAA|31383|8|43|3|AM|first|morning|breakfast| +31384|AAAAAAAAJJKHAAAA|31384|8|43|4|AM|first|morning|breakfast| +31385|AAAAAAAAKJKHAAAA|31385|8|43|5|AM|first|morning|breakfast| +31386|AAAAAAAALJKHAAAA|31386|8|43|6|AM|first|morning|breakfast| +31387|AAAAAAAAMJKHAAAA|31387|8|43|7|AM|first|morning|breakfast| +31388|AAAAAAAANJKHAAAA|31388|8|43|8|AM|first|morning|breakfast| +31389|AAAAAAAAOJKHAAAA|31389|8|43|9|AM|first|morning|breakfast| +31390|AAAAAAAAPJKHAAAA|31390|8|43|10|AM|first|morning|breakfast| +31391|AAAAAAAAAKKHAAAA|31391|8|43|11|AM|first|morning|breakfast| +31392|AAAAAAAABKKHAAAA|31392|8|43|12|AM|first|morning|breakfast| +31393|AAAAAAAACKKHAAAA|31393|8|43|13|AM|first|morning|breakfast| +31394|AAAAAAAADKKHAAAA|31394|8|43|14|AM|first|morning|breakfast| +31395|AAAAAAAAEKKHAAAA|31395|8|43|15|AM|first|morning|breakfast| +31396|AAAAAAAAFKKHAAAA|31396|8|43|16|AM|first|morning|breakfast| +31397|AAAAAAAAGKKHAAAA|31397|8|43|17|AM|first|morning|breakfast| +31398|AAAAAAAAHKKHAAAA|31398|8|43|18|AM|first|morning|breakfast| +31399|AAAAAAAAIKKHAAAA|31399|8|43|19|AM|first|morning|breakfast| +31400|AAAAAAAAJKKHAAAA|31400|8|43|20|AM|first|morning|breakfast| +31401|AAAAAAAAKKKHAAAA|31401|8|43|21|AM|first|morning|breakfast| +31402|AAAAAAAALKKHAAAA|31402|8|43|22|AM|first|morning|breakfast| +31403|AAAAAAAAMKKHAAAA|31403|8|43|23|AM|first|morning|breakfast| +31404|AAAAAAAANKKHAAAA|31404|8|43|24|AM|first|morning|breakfast| +31405|AAAAAAAAOKKHAAAA|31405|8|43|25|AM|first|morning|breakfast| +31406|AAAAAAAAPKKHAAAA|31406|8|43|26|AM|first|morning|breakfast| +31407|AAAAAAAAALKHAAAA|31407|8|43|27|AM|first|morning|breakfast| +31408|AAAAAAAABLKHAAAA|31408|8|43|28|AM|first|morning|breakfast| +31409|AAAAAAAACLKHAAAA|31409|8|43|29|AM|first|morning|breakfast| +31410|AAAAAAAADLKHAAAA|31410|8|43|30|AM|first|morning|breakfast| +31411|AAAAAAAAELKHAAAA|31411|8|43|31|AM|first|morning|breakfast| +31412|AAAAAAAAFLKHAAAA|31412|8|43|32|AM|first|morning|breakfast| +31413|AAAAAAAAGLKHAAAA|31413|8|43|33|AM|first|morning|breakfast| +31414|AAAAAAAAHLKHAAAA|31414|8|43|34|AM|first|morning|breakfast| +31415|AAAAAAAAILKHAAAA|31415|8|43|35|AM|first|morning|breakfast| +31416|AAAAAAAAJLKHAAAA|31416|8|43|36|AM|first|morning|breakfast| +31417|AAAAAAAAKLKHAAAA|31417|8|43|37|AM|first|morning|breakfast| +31418|AAAAAAAALLKHAAAA|31418|8|43|38|AM|first|morning|breakfast| +31419|AAAAAAAAMLKHAAAA|31419|8|43|39|AM|first|morning|breakfast| +31420|AAAAAAAANLKHAAAA|31420|8|43|40|AM|first|morning|breakfast| +31421|AAAAAAAAOLKHAAAA|31421|8|43|41|AM|first|morning|breakfast| +31422|AAAAAAAAPLKHAAAA|31422|8|43|42|AM|first|morning|breakfast| +31423|AAAAAAAAAMKHAAAA|31423|8|43|43|AM|first|morning|breakfast| +31424|AAAAAAAABMKHAAAA|31424|8|43|44|AM|first|morning|breakfast| +31425|AAAAAAAACMKHAAAA|31425|8|43|45|AM|first|morning|breakfast| +31426|AAAAAAAADMKHAAAA|31426|8|43|46|AM|first|morning|breakfast| +31427|AAAAAAAAEMKHAAAA|31427|8|43|47|AM|first|morning|breakfast| +31428|AAAAAAAAFMKHAAAA|31428|8|43|48|AM|first|morning|breakfast| +31429|AAAAAAAAGMKHAAAA|31429|8|43|49|AM|first|morning|breakfast| +31430|AAAAAAAAHMKHAAAA|31430|8|43|50|AM|first|morning|breakfast| +31431|AAAAAAAAIMKHAAAA|31431|8|43|51|AM|first|morning|breakfast| +31432|AAAAAAAAJMKHAAAA|31432|8|43|52|AM|first|morning|breakfast| +31433|AAAAAAAAKMKHAAAA|31433|8|43|53|AM|first|morning|breakfast| +31434|AAAAAAAALMKHAAAA|31434|8|43|54|AM|first|morning|breakfast| +31435|AAAAAAAAMMKHAAAA|31435|8|43|55|AM|first|morning|breakfast| +31436|AAAAAAAANMKHAAAA|31436|8|43|56|AM|first|morning|breakfast| +31437|AAAAAAAAOMKHAAAA|31437|8|43|57|AM|first|morning|breakfast| +31438|AAAAAAAAPMKHAAAA|31438|8|43|58|AM|first|morning|breakfast| +31439|AAAAAAAAANKHAAAA|31439|8|43|59|AM|first|morning|breakfast| +31440|AAAAAAAABNKHAAAA|31440|8|44|0|AM|first|morning|breakfast| +31441|AAAAAAAACNKHAAAA|31441|8|44|1|AM|first|morning|breakfast| +31442|AAAAAAAADNKHAAAA|31442|8|44|2|AM|first|morning|breakfast| +31443|AAAAAAAAENKHAAAA|31443|8|44|3|AM|first|morning|breakfast| +31444|AAAAAAAAFNKHAAAA|31444|8|44|4|AM|first|morning|breakfast| +31445|AAAAAAAAGNKHAAAA|31445|8|44|5|AM|first|morning|breakfast| +31446|AAAAAAAAHNKHAAAA|31446|8|44|6|AM|first|morning|breakfast| +31447|AAAAAAAAINKHAAAA|31447|8|44|7|AM|first|morning|breakfast| +31448|AAAAAAAAJNKHAAAA|31448|8|44|8|AM|first|morning|breakfast| +31449|AAAAAAAAKNKHAAAA|31449|8|44|9|AM|first|morning|breakfast| +31450|AAAAAAAALNKHAAAA|31450|8|44|10|AM|first|morning|breakfast| +31451|AAAAAAAAMNKHAAAA|31451|8|44|11|AM|first|morning|breakfast| +31452|AAAAAAAANNKHAAAA|31452|8|44|12|AM|first|morning|breakfast| +31453|AAAAAAAAONKHAAAA|31453|8|44|13|AM|first|morning|breakfast| +31454|AAAAAAAAPNKHAAAA|31454|8|44|14|AM|first|morning|breakfast| +31455|AAAAAAAAAOKHAAAA|31455|8|44|15|AM|first|morning|breakfast| +31456|AAAAAAAABOKHAAAA|31456|8|44|16|AM|first|morning|breakfast| +31457|AAAAAAAACOKHAAAA|31457|8|44|17|AM|first|morning|breakfast| +31458|AAAAAAAADOKHAAAA|31458|8|44|18|AM|first|morning|breakfast| +31459|AAAAAAAAEOKHAAAA|31459|8|44|19|AM|first|morning|breakfast| +31460|AAAAAAAAFOKHAAAA|31460|8|44|20|AM|first|morning|breakfast| +31461|AAAAAAAAGOKHAAAA|31461|8|44|21|AM|first|morning|breakfast| +31462|AAAAAAAAHOKHAAAA|31462|8|44|22|AM|first|morning|breakfast| +31463|AAAAAAAAIOKHAAAA|31463|8|44|23|AM|first|morning|breakfast| +31464|AAAAAAAAJOKHAAAA|31464|8|44|24|AM|first|morning|breakfast| +31465|AAAAAAAAKOKHAAAA|31465|8|44|25|AM|first|morning|breakfast| +31466|AAAAAAAALOKHAAAA|31466|8|44|26|AM|first|morning|breakfast| +31467|AAAAAAAAMOKHAAAA|31467|8|44|27|AM|first|morning|breakfast| +31468|AAAAAAAANOKHAAAA|31468|8|44|28|AM|first|morning|breakfast| +31469|AAAAAAAAOOKHAAAA|31469|8|44|29|AM|first|morning|breakfast| +31470|AAAAAAAAPOKHAAAA|31470|8|44|30|AM|first|morning|breakfast| +31471|AAAAAAAAAPKHAAAA|31471|8|44|31|AM|first|morning|breakfast| +31472|AAAAAAAABPKHAAAA|31472|8|44|32|AM|first|morning|breakfast| +31473|AAAAAAAACPKHAAAA|31473|8|44|33|AM|first|morning|breakfast| +31474|AAAAAAAADPKHAAAA|31474|8|44|34|AM|first|morning|breakfast| +31475|AAAAAAAAEPKHAAAA|31475|8|44|35|AM|first|morning|breakfast| +31476|AAAAAAAAFPKHAAAA|31476|8|44|36|AM|first|morning|breakfast| +31477|AAAAAAAAGPKHAAAA|31477|8|44|37|AM|first|morning|breakfast| +31478|AAAAAAAAHPKHAAAA|31478|8|44|38|AM|first|morning|breakfast| +31479|AAAAAAAAIPKHAAAA|31479|8|44|39|AM|first|morning|breakfast| +31480|AAAAAAAAJPKHAAAA|31480|8|44|40|AM|first|morning|breakfast| +31481|AAAAAAAAKPKHAAAA|31481|8|44|41|AM|first|morning|breakfast| +31482|AAAAAAAALPKHAAAA|31482|8|44|42|AM|first|morning|breakfast| +31483|AAAAAAAAMPKHAAAA|31483|8|44|43|AM|first|morning|breakfast| +31484|AAAAAAAANPKHAAAA|31484|8|44|44|AM|first|morning|breakfast| +31485|AAAAAAAAOPKHAAAA|31485|8|44|45|AM|first|morning|breakfast| +31486|AAAAAAAAPPKHAAAA|31486|8|44|46|AM|first|morning|breakfast| +31487|AAAAAAAAAALHAAAA|31487|8|44|47|AM|first|morning|breakfast| +31488|AAAAAAAABALHAAAA|31488|8|44|48|AM|first|morning|breakfast| +31489|AAAAAAAACALHAAAA|31489|8|44|49|AM|first|morning|breakfast| +31490|AAAAAAAADALHAAAA|31490|8|44|50|AM|first|morning|breakfast| +31491|AAAAAAAAEALHAAAA|31491|8|44|51|AM|first|morning|breakfast| +31492|AAAAAAAAFALHAAAA|31492|8|44|52|AM|first|morning|breakfast| +31493|AAAAAAAAGALHAAAA|31493|8|44|53|AM|first|morning|breakfast| +31494|AAAAAAAAHALHAAAA|31494|8|44|54|AM|first|morning|breakfast| +31495|AAAAAAAAIALHAAAA|31495|8|44|55|AM|first|morning|breakfast| +31496|AAAAAAAAJALHAAAA|31496|8|44|56|AM|first|morning|breakfast| +31497|AAAAAAAAKALHAAAA|31497|8|44|57|AM|first|morning|breakfast| +31498|AAAAAAAALALHAAAA|31498|8|44|58|AM|first|morning|breakfast| +31499|AAAAAAAAMALHAAAA|31499|8|44|59|AM|first|morning|breakfast| +31500|AAAAAAAANALHAAAA|31500|8|45|0|AM|first|morning|breakfast| +31501|AAAAAAAAOALHAAAA|31501|8|45|1|AM|first|morning|breakfast| +31502|AAAAAAAAPALHAAAA|31502|8|45|2|AM|first|morning|breakfast| +31503|AAAAAAAAABLHAAAA|31503|8|45|3|AM|first|morning|breakfast| +31504|AAAAAAAABBLHAAAA|31504|8|45|4|AM|first|morning|breakfast| +31505|AAAAAAAACBLHAAAA|31505|8|45|5|AM|first|morning|breakfast| +31506|AAAAAAAADBLHAAAA|31506|8|45|6|AM|first|morning|breakfast| +31507|AAAAAAAAEBLHAAAA|31507|8|45|7|AM|first|morning|breakfast| +31508|AAAAAAAAFBLHAAAA|31508|8|45|8|AM|first|morning|breakfast| +31509|AAAAAAAAGBLHAAAA|31509|8|45|9|AM|first|morning|breakfast| +31510|AAAAAAAAHBLHAAAA|31510|8|45|10|AM|first|morning|breakfast| +31511|AAAAAAAAIBLHAAAA|31511|8|45|11|AM|first|morning|breakfast| +31512|AAAAAAAAJBLHAAAA|31512|8|45|12|AM|first|morning|breakfast| +31513|AAAAAAAAKBLHAAAA|31513|8|45|13|AM|first|morning|breakfast| +31514|AAAAAAAALBLHAAAA|31514|8|45|14|AM|first|morning|breakfast| +31515|AAAAAAAAMBLHAAAA|31515|8|45|15|AM|first|morning|breakfast| +31516|AAAAAAAANBLHAAAA|31516|8|45|16|AM|first|morning|breakfast| +31517|AAAAAAAAOBLHAAAA|31517|8|45|17|AM|first|morning|breakfast| +31518|AAAAAAAAPBLHAAAA|31518|8|45|18|AM|first|morning|breakfast| +31519|AAAAAAAAACLHAAAA|31519|8|45|19|AM|first|morning|breakfast| +31520|AAAAAAAABCLHAAAA|31520|8|45|20|AM|first|morning|breakfast| +31521|AAAAAAAACCLHAAAA|31521|8|45|21|AM|first|morning|breakfast| +31522|AAAAAAAADCLHAAAA|31522|8|45|22|AM|first|morning|breakfast| +31523|AAAAAAAAECLHAAAA|31523|8|45|23|AM|first|morning|breakfast| +31524|AAAAAAAAFCLHAAAA|31524|8|45|24|AM|first|morning|breakfast| +31525|AAAAAAAAGCLHAAAA|31525|8|45|25|AM|first|morning|breakfast| +31526|AAAAAAAAHCLHAAAA|31526|8|45|26|AM|first|morning|breakfast| +31527|AAAAAAAAICLHAAAA|31527|8|45|27|AM|first|morning|breakfast| +31528|AAAAAAAAJCLHAAAA|31528|8|45|28|AM|first|morning|breakfast| +31529|AAAAAAAAKCLHAAAA|31529|8|45|29|AM|first|morning|breakfast| +31530|AAAAAAAALCLHAAAA|31530|8|45|30|AM|first|morning|breakfast| +31531|AAAAAAAAMCLHAAAA|31531|8|45|31|AM|first|morning|breakfast| +31532|AAAAAAAANCLHAAAA|31532|8|45|32|AM|first|morning|breakfast| +31533|AAAAAAAAOCLHAAAA|31533|8|45|33|AM|first|morning|breakfast| +31534|AAAAAAAAPCLHAAAA|31534|8|45|34|AM|first|morning|breakfast| +31535|AAAAAAAAADLHAAAA|31535|8|45|35|AM|first|morning|breakfast| +31536|AAAAAAAABDLHAAAA|31536|8|45|36|AM|first|morning|breakfast| +31537|AAAAAAAACDLHAAAA|31537|8|45|37|AM|first|morning|breakfast| +31538|AAAAAAAADDLHAAAA|31538|8|45|38|AM|first|morning|breakfast| +31539|AAAAAAAAEDLHAAAA|31539|8|45|39|AM|first|morning|breakfast| +31540|AAAAAAAAFDLHAAAA|31540|8|45|40|AM|first|morning|breakfast| +31541|AAAAAAAAGDLHAAAA|31541|8|45|41|AM|first|morning|breakfast| +31542|AAAAAAAAHDLHAAAA|31542|8|45|42|AM|first|morning|breakfast| +31543|AAAAAAAAIDLHAAAA|31543|8|45|43|AM|first|morning|breakfast| +31544|AAAAAAAAJDLHAAAA|31544|8|45|44|AM|first|morning|breakfast| +31545|AAAAAAAAKDLHAAAA|31545|8|45|45|AM|first|morning|breakfast| +31546|AAAAAAAALDLHAAAA|31546|8|45|46|AM|first|morning|breakfast| +31547|AAAAAAAAMDLHAAAA|31547|8|45|47|AM|first|morning|breakfast| +31548|AAAAAAAANDLHAAAA|31548|8|45|48|AM|first|morning|breakfast| +31549|AAAAAAAAODLHAAAA|31549|8|45|49|AM|first|morning|breakfast| +31550|AAAAAAAAPDLHAAAA|31550|8|45|50|AM|first|morning|breakfast| +31551|AAAAAAAAAELHAAAA|31551|8|45|51|AM|first|morning|breakfast| +31552|AAAAAAAABELHAAAA|31552|8|45|52|AM|first|morning|breakfast| +31553|AAAAAAAACELHAAAA|31553|8|45|53|AM|first|morning|breakfast| +31554|AAAAAAAADELHAAAA|31554|8|45|54|AM|first|morning|breakfast| +31555|AAAAAAAAEELHAAAA|31555|8|45|55|AM|first|morning|breakfast| +31556|AAAAAAAAFELHAAAA|31556|8|45|56|AM|first|morning|breakfast| +31557|AAAAAAAAGELHAAAA|31557|8|45|57|AM|first|morning|breakfast| +31558|AAAAAAAAHELHAAAA|31558|8|45|58|AM|first|morning|breakfast| +31559|AAAAAAAAIELHAAAA|31559|8|45|59|AM|first|morning|breakfast| +31560|AAAAAAAAJELHAAAA|31560|8|46|0|AM|first|morning|breakfast| +31561|AAAAAAAAKELHAAAA|31561|8|46|1|AM|first|morning|breakfast| +31562|AAAAAAAALELHAAAA|31562|8|46|2|AM|first|morning|breakfast| +31563|AAAAAAAAMELHAAAA|31563|8|46|3|AM|first|morning|breakfast| +31564|AAAAAAAANELHAAAA|31564|8|46|4|AM|first|morning|breakfast| +31565|AAAAAAAAOELHAAAA|31565|8|46|5|AM|first|morning|breakfast| +31566|AAAAAAAAPELHAAAA|31566|8|46|6|AM|first|morning|breakfast| +31567|AAAAAAAAAFLHAAAA|31567|8|46|7|AM|first|morning|breakfast| +31568|AAAAAAAABFLHAAAA|31568|8|46|8|AM|first|morning|breakfast| +31569|AAAAAAAACFLHAAAA|31569|8|46|9|AM|first|morning|breakfast| +31570|AAAAAAAADFLHAAAA|31570|8|46|10|AM|first|morning|breakfast| +31571|AAAAAAAAEFLHAAAA|31571|8|46|11|AM|first|morning|breakfast| +31572|AAAAAAAAFFLHAAAA|31572|8|46|12|AM|first|morning|breakfast| +31573|AAAAAAAAGFLHAAAA|31573|8|46|13|AM|first|morning|breakfast| +31574|AAAAAAAAHFLHAAAA|31574|8|46|14|AM|first|morning|breakfast| +31575|AAAAAAAAIFLHAAAA|31575|8|46|15|AM|first|morning|breakfast| +31576|AAAAAAAAJFLHAAAA|31576|8|46|16|AM|first|morning|breakfast| +31577|AAAAAAAAKFLHAAAA|31577|8|46|17|AM|first|morning|breakfast| +31578|AAAAAAAALFLHAAAA|31578|8|46|18|AM|first|morning|breakfast| +31579|AAAAAAAAMFLHAAAA|31579|8|46|19|AM|first|morning|breakfast| +31580|AAAAAAAANFLHAAAA|31580|8|46|20|AM|first|morning|breakfast| +31581|AAAAAAAAOFLHAAAA|31581|8|46|21|AM|first|morning|breakfast| +31582|AAAAAAAAPFLHAAAA|31582|8|46|22|AM|first|morning|breakfast| +31583|AAAAAAAAAGLHAAAA|31583|8|46|23|AM|first|morning|breakfast| +31584|AAAAAAAABGLHAAAA|31584|8|46|24|AM|first|morning|breakfast| +31585|AAAAAAAACGLHAAAA|31585|8|46|25|AM|first|morning|breakfast| +31586|AAAAAAAADGLHAAAA|31586|8|46|26|AM|first|morning|breakfast| +31587|AAAAAAAAEGLHAAAA|31587|8|46|27|AM|first|morning|breakfast| +31588|AAAAAAAAFGLHAAAA|31588|8|46|28|AM|first|morning|breakfast| +31589|AAAAAAAAGGLHAAAA|31589|8|46|29|AM|first|morning|breakfast| +31590|AAAAAAAAHGLHAAAA|31590|8|46|30|AM|first|morning|breakfast| +31591|AAAAAAAAIGLHAAAA|31591|8|46|31|AM|first|morning|breakfast| +31592|AAAAAAAAJGLHAAAA|31592|8|46|32|AM|first|morning|breakfast| +31593|AAAAAAAAKGLHAAAA|31593|8|46|33|AM|first|morning|breakfast| +31594|AAAAAAAALGLHAAAA|31594|8|46|34|AM|first|morning|breakfast| +31595|AAAAAAAAMGLHAAAA|31595|8|46|35|AM|first|morning|breakfast| +31596|AAAAAAAANGLHAAAA|31596|8|46|36|AM|first|morning|breakfast| +31597|AAAAAAAAOGLHAAAA|31597|8|46|37|AM|first|morning|breakfast| +31598|AAAAAAAAPGLHAAAA|31598|8|46|38|AM|first|morning|breakfast| +31599|AAAAAAAAAHLHAAAA|31599|8|46|39|AM|first|morning|breakfast| +31600|AAAAAAAABHLHAAAA|31600|8|46|40|AM|first|morning|breakfast| +31601|AAAAAAAACHLHAAAA|31601|8|46|41|AM|first|morning|breakfast| +31602|AAAAAAAADHLHAAAA|31602|8|46|42|AM|first|morning|breakfast| +31603|AAAAAAAAEHLHAAAA|31603|8|46|43|AM|first|morning|breakfast| +31604|AAAAAAAAFHLHAAAA|31604|8|46|44|AM|first|morning|breakfast| +31605|AAAAAAAAGHLHAAAA|31605|8|46|45|AM|first|morning|breakfast| +31606|AAAAAAAAHHLHAAAA|31606|8|46|46|AM|first|morning|breakfast| +31607|AAAAAAAAIHLHAAAA|31607|8|46|47|AM|first|morning|breakfast| +31608|AAAAAAAAJHLHAAAA|31608|8|46|48|AM|first|morning|breakfast| +31609|AAAAAAAAKHLHAAAA|31609|8|46|49|AM|first|morning|breakfast| +31610|AAAAAAAALHLHAAAA|31610|8|46|50|AM|first|morning|breakfast| +31611|AAAAAAAAMHLHAAAA|31611|8|46|51|AM|first|morning|breakfast| +31612|AAAAAAAANHLHAAAA|31612|8|46|52|AM|first|morning|breakfast| +31613|AAAAAAAAOHLHAAAA|31613|8|46|53|AM|first|morning|breakfast| +31614|AAAAAAAAPHLHAAAA|31614|8|46|54|AM|first|morning|breakfast| +31615|AAAAAAAAAILHAAAA|31615|8|46|55|AM|first|morning|breakfast| +31616|AAAAAAAABILHAAAA|31616|8|46|56|AM|first|morning|breakfast| +31617|AAAAAAAACILHAAAA|31617|8|46|57|AM|first|morning|breakfast| +31618|AAAAAAAADILHAAAA|31618|8|46|58|AM|first|morning|breakfast| +31619|AAAAAAAAEILHAAAA|31619|8|46|59|AM|first|morning|breakfast| +31620|AAAAAAAAFILHAAAA|31620|8|47|0|AM|first|morning|breakfast| +31621|AAAAAAAAGILHAAAA|31621|8|47|1|AM|first|morning|breakfast| +31622|AAAAAAAAHILHAAAA|31622|8|47|2|AM|first|morning|breakfast| +31623|AAAAAAAAIILHAAAA|31623|8|47|3|AM|first|morning|breakfast| +31624|AAAAAAAAJILHAAAA|31624|8|47|4|AM|first|morning|breakfast| +31625|AAAAAAAAKILHAAAA|31625|8|47|5|AM|first|morning|breakfast| +31626|AAAAAAAALILHAAAA|31626|8|47|6|AM|first|morning|breakfast| +31627|AAAAAAAAMILHAAAA|31627|8|47|7|AM|first|morning|breakfast| +31628|AAAAAAAANILHAAAA|31628|8|47|8|AM|first|morning|breakfast| +31629|AAAAAAAAOILHAAAA|31629|8|47|9|AM|first|morning|breakfast| +31630|AAAAAAAAPILHAAAA|31630|8|47|10|AM|first|morning|breakfast| +31631|AAAAAAAAAJLHAAAA|31631|8|47|11|AM|first|morning|breakfast| +31632|AAAAAAAABJLHAAAA|31632|8|47|12|AM|first|morning|breakfast| +31633|AAAAAAAACJLHAAAA|31633|8|47|13|AM|first|morning|breakfast| +31634|AAAAAAAADJLHAAAA|31634|8|47|14|AM|first|morning|breakfast| +31635|AAAAAAAAEJLHAAAA|31635|8|47|15|AM|first|morning|breakfast| +31636|AAAAAAAAFJLHAAAA|31636|8|47|16|AM|first|morning|breakfast| +31637|AAAAAAAAGJLHAAAA|31637|8|47|17|AM|first|morning|breakfast| +31638|AAAAAAAAHJLHAAAA|31638|8|47|18|AM|first|morning|breakfast| +31639|AAAAAAAAIJLHAAAA|31639|8|47|19|AM|first|morning|breakfast| +31640|AAAAAAAAJJLHAAAA|31640|8|47|20|AM|first|morning|breakfast| +31641|AAAAAAAAKJLHAAAA|31641|8|47|21|AM|first|morning|breakfast| +31642|AAAAAAAALJLHAAAA|31642|8|47|22|AM|first|morning|breakfast| +31643|AAAAAAAAMJLHAAAA|31643|8|47|23|AM|first|morning|breakfast| +31644|AAAAAAAANJLHAAAA|31644|8|47|24|AM|first|morning|breakfast| +31645|AAAAAAAAOJLHAAAA|31645|8|47|25|AM|first|morning|breakfast| +31646|AAAAAAAAPJLHAAAA|31646|8|47|26|AM|first|morning|breakfast| +31647|AAAAAAAAAKLHAAAA|31647|8|47|27|AM|first|morning|breakfast| +31648|AAAAAAAABKLHAAAA|31648|8|47|28|AM|first|morning|breakfast| +31649|AAAAAAAACKLHAAAA|31649|8|47|29|AM|first|morning|breakfast| +31650|AAAAAAAADKLHAAAA|31650|8|47|30|AM|first|morning|breakfast| +31651|AAAAAAAAEKLHAAAA|31651|8|47|31|AM|first|morning|breakfast| +31652|AAAAAAAAFKLHAAAA|31652|8|47|32|AM|first|morning|breakfast| +31653|AAAAAAAAGKLHAAAA|31653|8|47|33|AM|first|morning|breakfast| +31654|AAAAAAAAHKLHAAAA|31654|8|47|34|AM|first|morning|breakfast| +31655|AAAAAAAAIKLHAAAA|31655|8|47|35|AM|first|morning|breakfast| +31656|AAAAAAAAJKLHAAAA|31656|8|47|36|AM|first|morning|breakfast| +31657|AAAAAAAAKKLHAAAA|31657|8|47|37|AM|first|morning|breakfast| +31658|AAAAAAAALKLHAAAA|31658|8|47|38|AM|first|morning|breakfast| +31659|AAAAAAAAMKLHAAAA|31659|8|47|39|AM|first|morning|breakfast| +31660|AAAAAAAANKLHAAAA|31660|8|47|40|AM|first|morning|breakfast| +31661|AAAAAAAAOKLHAAAA|31661|8|47|41|AM|first|morning|breakfast| +31662|AAAAAAAAPKLHAAAA|31662|8|47|42|AM|first|morning|breakfast| +31663|AAAAAAAAALLHAAAA|31663|8|47|43|AM|first|morning|breakfast| +31664|AAAAAAAABLLHAAAA|31664|8|47|44|AM|first|morning|breakfast| +31665|AAAAAAAACLLHAAAA|31665|8|47|45|AM|first|morning|breakfast| +31666|AAAAAAAADLLHAAAA|31666|8|47|46|AM|first|morning|breakfast| +31667|AAAAAAAAELLHAAAA|31667|8|47|47|AM|first|morning|breakfast| +31668|AAAAAAAAFLLHAAAA|31668|8|47|48|AM|first|morning|breakfast| +31669|AAAAAAAAGLLHAAAA|31669|8|47|49|AM|first|morning|breakfast| +31670|AAAAAAAAHLLHAAAA|31670|8|47|50|AM|first|morning|breakfast| +31671|AAAAAAAAILLHAAAA|31671|8|47|51|AM|first|morning|breakfast| +31672|AAAAAAAAJLLHAAAA|31672|8|47|52|AM|first|morning|breakfast| +31673|AAAAAAAAKLLHAAAA|31673|8|47|53|AM|first|morning|breakfast| +31674|AAAAAAAALLLHAAAA|31674|8|47|54|AM|first|morning|breakfast| +31675|AAAAAAAAMLLHAAAA|31675|8|47|55|AM|first|morning|breakfast| +31676|AAAAAAAANLLHAAAA|31676|8|47|56|AM|first|morning|breakfast| +31677|AAAAAAAAOLLHAAAA|31677|8|47|57|AM|first|morning|breakfast| +31678|AAAAAAAAPLLHAAAA|31678|8|47|58|AM|first|morning|breakfast| +31679|AAAAAAAAAMLHAAAA|31679|8|47|59|AM|first|morning|breakfast| +31680|AAAAAAAABMLHAAAA|31680|8|48|0|AM|first|morning|breakfast| +31681|AAAAAAAACMLHAAAA|31681|8|48|1|AM|first|morning|breakfast| +31682|AAAAAAAADMLHAAAA|31682|8|48|2|AM|first|morning|breakfast| +31683|AAAAAAAAEMLHAAAA|31683|8|48|3|AM|first|morning|breakfast| +31684|AAAAAAAAFMLHAAAA|31684|8|48|4|AM|first|morning|breakfast| +31685|AAAAAAAAGMLHAAAA|31685|8|48|5|AM|first|morning|breakfast| +31686|AAAAAAAAHMLHAAAA|31686|8|48|6|AM|first|morning|breakfast| +31687|AAAAAAAAIMLHAAAA|31687|8|48|7|AM|first|morning|breakfast| +31688|AAAAAAAAJMLHAAAA|31688|8|48|8|AM|first|morning|breakfast| +31689|AAAAAAAAKMLHAAAA|31689|8|48|9|AM|first|morning|breakfast| +31690|AAAAAAAALMLHAAAA|31690|8|48|10|AM|first|morning|breakfast| +31691|AAAAAAAAMMLHAAAA|31691|8|48|11|AM|first|morning|breakfast| +31692|AAAAAAAANMLHAAAA|31692|8|48|12|AM|first|morning|breakfast| +31693|AAAAAAAAOMLHAAAA|31693|8|48|13|AM|first|morning|breakfast| +31694|AAAAAAAAPMLHAAAA|31694|8|48|14|AM|first|morning|breakfast| +31695|AAAAAAAAANLHAAAA|31695|8|48|15|AM|first|morning|breakfast| +31696|AAAAAAAABNLHAAAA|31696|8|48|16|AM|first|morning|breakfast| +31697|AAAAAAAACNLHAAAA|31697|8|48|17|AM|first|morning|breakfast| +31698|AAAAAAAADNLHAAAA|31698|8|48|18|AM|first|morning|breakfast| +31699|AAAAAAAAENLHAAAA|31699|8|48|19|AM|first|morning|breakfast| +31700|AAAAAAAAFNLHAAAA|31700|8|48|20|AM|first|morning|breakfast| +31701|AAAAAAAAGNLHAAAA|31701|8|48|21|AM|first|morning|breakfast| +31702|AAAAAAAAHNLHAAAA|31702|8|48|22|AM|first|morning|breakfast| +31703|AAAAAAAAINLHAAAA|31703|8|48|23|AM|first|morning|breakfast| +31704|AAAAAAAAJNLHAAAA|31704|8|48|24|AM|first|morning|breakfast| +31705|AAAAAAAAKNLHAAAA|31705|8|48|25|AM|first|morning|breakfast| +31706|AAAAAAAALNLHAAAA|31706|8|48|26|AM|first|morning|breakfast| +31707|AAAAAAAAMNLHAAAA|31707|8|48|27|AM|first|morning|breakfast| +31708|AAAAAAAANNLHAAAA|31708|8|48|28|AM|first|morning|breakfast| +31709|AAAAAAAAONLHAAAA|31709|8|48|29|AM|first|morning|breakfast| +31710|AAAAAAAAPNLHAAAA|31710|8|48|30|AM|first|morning|breakfast| +31711|AAAAAAAAAOLHAAAA|31711|8|48|31|AM|first|morning|breakfast| +31712|AAAAAAAABOLHAAAA|31712|8|48|32|AM|first|morning|breakfast| +31713|AAAAAAAACOLHAAAA|31713|8|48|33|AM|first|morning|breakfast| +31714|AAAAAAAADOLHAAAA|31714|8|48|34|AM|first|morning|breakfast| +31715|AAAAAAAAEOLHAAAA|31715|8|48|35|AM|first|morning|breakfast| +31716|AAAAAAAAFOLHAAAA|31716|8|48|36|AM|first|morning|breakfast| +31717|AAAAAAAAGOLHAAAA|31717|8|48|37|AM|first|morning|breakfast| +31718|AAAAAAAAHOLHAAAA|31718|8|48|38|AM|first|morning|breakfast| +31719|AAAAAAAAIOLHAAAA|31719|8|48|39|AM|first|morning|breakfast| +31720|AAAAAAAAJOLHAAAA|31720|8|48|40|AM|first|morning|breakfast| +31721|AAAAAAAAKOLHAAAA|31721|8|48|41|AM|first|morning|breakfast| +31722|AAAAAAAALOLHAAAA|31722|8|48|42|AM|first|morning|breakfast| +31723|AAAAAAAAMOLHAAAA|31723|8|48|43|AM|first|morning|breakfast| +31724|AAAAAAAANOLHAAAA|31724|8|48|44|AM|first|morning|breakfast| +31725|AAAAAAAAOOLHAAAA|31725|8|48|45|AM|first|morning|breakfast| +31726|AAAAAAAAPOLHAAAA|31726|8|48|46|AM|first|morning|breakfast| +31727|AAAAAAAAAPLHAAAA|31727|8|48|47|AM|first|morning|breakfast| +31728|AAAAAAAABPLHAAAA|31728|8|48|48|AM|first|morning|breakfast| +31729|AAAAAAAACPLHAAAA|31729|8|48|49|AM|first|morning|breakfast| +31730|AAAAAAAADPLHAAAA|31730|8|48|50|AM|first|morning|breakfast| +31731|AAAAAAAAEPLHAAAA|31731|8|48|51|AM|first|morning|breakfast| +31732|AAAAAAAAFPLHAAAA|31732|8|48|52|AM|first|morning|breakfast| +31733|AAAAAAAAGPLHAAAA|31733|8|48|53|AM|first|morning|breakfast| +31734|AAAAAAAAHPLHAAAA|31734|8|48|54|AM|first|morning|breakfast| +31735|AAAAAAAAIPLHAAAA|31735|8|48|55|AM|first|morning|breakfast| +31736|AAAAAAAAJPLHAAAA|31736|8|48|56|AM|first|morning|breakfast| +31737|AAAAAAAAKPLHAAAA|31737|8|48|57|AM|first|morning|breakfast| +31738|AAAAAAAALPLHAAAA|31738|8|48|58|AM|first|morning|breakfast| +31739|AAAAAAAAMPLHAAAA|31739|8|48|59|AM|first|morning|breakfast| +31740|AAAAAAAANPLHAAAA|31740|8|49|0|AM|first|morning|breakfast| +31741|AAAAAAAAOPLHAAAA|31741|8|49|1|AM|first|morning|breakfast| +31742|AAAAAAAAPPLHAAAA|31742|8|49|2|AM|first|morning|breakfast| +31743|AAAAAAAAAAMHAAAA|31743|8|49|3|AM|first|morning|breakfast| +31744|AAAAAAAABAMHAAAA|31744|8|49|4|AM|first|morning|breakfast| +31745|AAAAAAAACAMHAAAA|31745|8|49|5|AM|first|morning|breakfast| +31746|AAAAAAAADAMHAAAA|31746|8|49|6|AM|first|morning|breakfast| +31747|AAAAAAAAEAMHAAAA|31747|8|49|7|AM|first|morning|breakfast| +31748|AAAAAAAAFAMHAAAA|31748|8|49|8|AM|first|morning|breakfast| +31749|AAAAAAAAGAMHAAAA|31749|8|49|9|AM|first|morning|breakfast| +31750|AAAAAAAAHAMHAAAA|31750|8|49|10|AM|first|morning|breakfast| +31751|AAAAAAAAIAMHAAAA|31751|8|49|11|AM|first|morning|breakfast| +31752|AAAAAAAAJAMHAAAA|31752|8|49|12|AM|first|morning|breakfast| +31753|AAAAAAAAKAMHAAAA|31753|8|49|13|AM|first|morning|breakfast| +31754|AAAAAAAALAMHAAAA|31754|8|49|14|AM|first|morning|breakfast| +31755|AAAAAAAAMAMHAAAA|31755|8|49|15|AM|first|morning|breakfast| +31756|AAAAAAAANAMHAAAA|31756|8|49|16|AM|first|morning|breakfast| +31757|AAAAAAAAOAMHAAAA|31757|8|49|17|AM|first|morning|breakfast| +31758|AAAAAAAAPAMHAAAA|31758|8|49|18|AM|first|morning|breakfast| +31759|AAAAAAAAABMHAAAA|31759|8|49|19|AM|first|morning|breakfast| +31760|AAAAAAAABBMHAAAA|31760|8|49|20|AM|first|morning|breakfast| +31761|AAAAAAAACBMHAAAA|31761|8|49|21|AM|first|morning|breakfast| +31762|AAAAAAAADBMHAAAA|31762|8|49|22|AM|first|morning|breakfast| +31763|AAAAAAAAEBMHAAAA|31763|8|49|23|AM|first|morning|breakfast| +31764|AAAAAAAAFBMHAAAA|31764|8|49|24|AM|first|morning|breakfast| +31765|AAAAAAAAGBMHAAAA|31765|8|49|25|AM|first|morning|breakfast| +31766|AAAAAAAAHBMHAAAA|31766|8|49|26|AM|first|morning|breakfast| +31767|AAAAAAAAIBMHAAAA|31767|8|49|27|AM|first|morning|breakfast| +31768|AAAAAAAAJBMHAAAA|31768|8|49|28|AM|first|morning|breakfast| +31769|AAAAAAAAKBMHAAAA|31769|8|49|29|AM|first|morning|breakfast| +31770|AAAAAAAALBMHAAAA|31770|8|49|30|AM|first|morning|breakfast| +31771|AAAAAAAAMBMHAAAA|31771|8|49|31|AM|first|morning|breakfast| +31772|AAAAAAAANBMHAAAA|31772|8|49|32|AM|first|morning|breakfast| +31773|AAAAAAAAOBMHAAAA|31773|8|49|33|AM|first|morning|breakfast| +31774|AAAAAAAAPBMHAAAA|31774|8|49|34|AM|first|morning|breakfast| +31775|AAAAAAAAACMHAAAA|31775|8|49|35|AM|first|morning|breakfast| +31776|AAAAAAAABCMHAAAA|31776|8|49|36|AM|first|morning|breakfast| +31777|AAAAAAAACCMHAAAA|31777|8|49|37|AM|first|morning|breakfast| +31778|AAAAAAAADCMHAAAA|31778|8|49|38|AM|first|morning|breakfast| +31779|AAAAAAAAECMHAAAA|31779|8|49|39|AM|first|morning|breakfast| +31780|AAAAAAAAFCMHAAAA|31780|8|49|40|AM|first|morning|breakfast| +31781|AAAAAAAAGCMHAAAA|31781|8|49|41|AM|first|morning|breakfast| +31782|AAAAAAAAHCMHAAAA|31782|8|49|42|AM|first|morning|breakfast| +31783|AAAAAAAAICMHAAAA|31783|8|49|43|AM|first|morning|breakfast| +31784|AAAAAAAAJCMHAAAA|31784|8|49|44|AM|first|morning|breakfast| +31785|AAAAAAAAKCMHAAAA|31785|8|49|45|AM|first|morning|breakfast| +31786|AAAAAAAALCMHAAAA|31786|8|49|46|AM|first|morning|breakfast| +31787|AAAAAAAAMCMHAAAA|31787|8|49|47|AM|first|morning|breakfast| +31788|AAAAAAAANCMHAAAA|31788|8|49|48|AM|first|morning|breakfast| +31789|AAAAAAAAOCMHAAAA|31789|8|49|49|AM|first|morning|breakfast| +31790|AAAAAAAAPCMHAAAA|31790|8|49|50|AM|first|morning|breakfast| +31791|AAAAAAAAADMHAAAA|31791|8|49|51|AM|first|morning|breakfast| +31792|AAAAAAAABDMHAAAA|31792|8|49|52|AM|first|morning|breakfast| +31793|AAAAAAAACDMHAAAA|31793|8|49|53|AM|first|morning|breakfast| +31794|AAAAAAAADDMHAAAA|31794|8|49|54|AM|first|morning|breakfast| +31795|AAAAAAAAEDMHAAAA|31795|8|49|55|AM|first|morning|breakfast| +31796|AAAAAAAAFDMHAAAA|31796|8|49|56|AM|first|morning|breakfast| +31797|AAAAAAAAGDMHAAAA|31797|8|49|57|AM|first|morning|breakfast| +31798|AAAAAAAAHDMHAAAA|31798|8|49|58|AM|first|morning|breakfast| +31799|AAAAAAAAIDMHAAAA|31799|8|49|59|AM|first|morning|breakfast| +31800|AAAAAAAAJDMHAAAA|31800|8|50|0|AM|first|morning|breakfast| +31801|AAAAAAAAKDMHAAAA|31801|8|50|1|AM|first|morning|breakfast| +31802|AAAAAAAALDMHAAAA|31802|8|50|2|AM|first|morning|breakfast| +31803|AAAAAAAAMDMHAAAA|31803|8|50|3|AM|first|morning|breakfast| +31804|AAAAAAAANDMHAAAA|31804|8|50|4|AM|first|morning|breakfast| +31805|AAAAAAAAODMHAAAA|31805|8|50|5|AM|first|morning|breakfast| +31806|AAAAAAAAPDMHAAAA|31806|8|50|6|AM|first|morning|breakfast| +31807|AAAAAAAAAEMHAAAA|31807|8|50|7|AM|first|morning|breakfast| +31808|AAAAAAAABEMHAAAA|31808|8|50|8|AM|first|morning|breakfast| +31809|AAAAAAAACEMHAAAA|31809|8|50|9|AM|first|morning|breakfast| +31810|AAAAAAAADEMHAAAA|31810|8|50|10|AM|first|morning|breakfast| +31811|AAAAAAAAEEMHAAAA|31811|8|50|11|AM|first|morning|breakfast| +31812|AAAAAAAAFEMHAAAA|31812|8|50|12|AM|first|morning|breakfast| +31813|AAAAAAAAGEMHAAAA|31813|8|50|13|AM|first|morning|breakfast| +31814|AAAAAAAAHEMHAAAA|31814|8|50|14|AM|first|morning|breakfast| +31815|AAAAAAAAIEMHAAAA|31815|8|50|15|AM|first|morning|breakfast| +31816|AAAAAAAAJEMHAAAA|31816|8|50|16|AM|first|morning|breakfast| +31817|AAAAAAAAKEMHAAAA|31817|8|50|17|AM|first|morning|breakfast| +31818|AAAAAAAALEMHAAAA|31818|8|50|18|AM|first|morning|breakfast| +31819|AAAAAAAAMEMHAAAA|31819|8|50|19|AM|first|morning|breakfast| +31820|AAAAAAAANEMHAAAA|31820|8|50|20|AM|first|morning|breakfast| +31821|AAAAAAAAOEMHAAAA|31821|8|50|21|AM|first|morning|breakfast| +31822|AAAAAAAAPEMHAAAA|31822|8|50|22|AM|first|morning|breakfast| +31823|AAAAAAAAAFMHAAAA|31823|8|50|23|AM|first|morning|breakfast| +31824|AAAAAAAABFMHAAAA|31824|8|50|24|AM|first|morning|breakfast| +31825|AAAAAAAACFMHAAAA|31825|8|50|25|AM|first|morning|breakfast| +31826|AAAAAAAADFMHAAAA|31826|8|50|26|AM|first|morning|breakfast| +31827|AAAAAAAAEFMHAAAA|31827|8|50|27|AM|first|morning|breakfast| +31828|AAAAAAAAFFMHAAAA|31828|8|50|28|AM|first|morning|breakfast| +31829|AAAAAAAAGFMHAAAA|31829|8|50|29|AM|first|morning|breakfast| +31830|AAAAAAAAHFMHAAAA|31830|8|50|30|AM|first|morning|breakfast| +31831|AAAAAAAAIFMHAAAA|31831|8|50|31|AM|first|morning|breakfast| +31832|AAAAAAAAJFMHAAAA|31832|8|50|32|AM|first|morning|breakfast| +31833|AAAAAAAAKFMHAAAA|31833|8|50|33|AM|first|morning|breakfast| +31834|AAAAAAAALFMHAAAA|31834|8|50|34|AM|first|morning|breakfast| +31835|AAAAAAAAMFMHAAAA|31835|8|50|35|AM|first|morning|breakfast| +31836|AAAAAAAANFMHAAAA|31836|8|50|36|AM|first|morning|breakfast| +31837|AAAAAAAAOFMHAAAA|31837|8|50|37|AM|first|morning|breakfast| +31838|AAAAAAAAPFMHAAAA|31838|8|50|38|AM|first|morning|breakfast| +31839|AAAAAAAAAGMHAAAA|31839|8|50|39|AM|first|morning|breakfast| +31840|AAAAAAAABGMHAAAA|31840|8|50|40|AM|first|morning|breakfast| +31841|AAAAAAAACGMHAAAA|31841|8|50|41|AM|first|morning|breakfast| +31842|AAAAAAAADGMHAAAA|31842|8|50|42|AM|first|morning|breakfast| +31843|AAAAAAAAEGMHAAAA|31843|8|50|43|AM|first|morning|breakfast| +31844|AAAAAAAAFGMHAAAA|31844|8|50|44|AM|first|morning|breakfast| +31845|AAAAAAAAGGMHAAAA|31845|8|50|45|AM|first|morning|breakfast| +31846|AAAAAAAAHGMHAAAA|31846|8|50|46|AM|first|morning|breakfast| +31847|AAAAAAAAIGMHAAAA|31847|8|50|47|AM|first|morning|breakfast| +31848|AAAAAAAAJGMHAAAA|31848|8|50|48|AM|first|morning|breakfast| +31849|AAAAAAAAKGMHAAAA|31849|8|50|49|AM|first|morning|breakfast| +31850|AAAAAAAALGMHAAAA|31850|8|50|50|AM|first|morning|breakfast| +31851|AAAAAAAAMGMHAAAA|31851|8|50|51|AM|first|morning|breakfast| +31852|AAAAAAAANGMHAAAA|31852|8|50|52|AM|first|morning|breakfast| +31853|AAAAAAAAOGMHAAAA|31853|8|50|53|AM|first|morning|breakfast| +31854|AAAAAAAAPGMHAAAA|31854|8|50|54|AM|first|morning|breakfast| +31855|AAAAAAAAAHMHAAAA|31855|8|50|55|AM|first|morning|breakfast| +31856|AAAAAAAABHMHAAAA|31856|8|50|56|AM|first|morning|breakfast| +31857|AAAAAAAACHMHAAAA|31857|8|50|57|AM|first|morning|breakfast| +31858|AAAAAAAADHMHAAAA|31858|8|50|58|AM|first|morning|breakfast| +31859|AAAAAAAAEHMHAAAA|31859|8|50|59|AM|first|morning|breakfast| +31860|AAAAAAAAFHMHAAAA|31860|8|51|0|AM|first|morning|breakfast| +31861|AAAAAAAAGHMHAAAA|31861|8|51|1|AM|first|morning|breakfast| +31862|AAAAAAAAHHMHAAAA|31862|8|51|2|AM|first|morning|breakfast| +31863|AAAAAAAAIHMHAAAA|31863|8|51|3|AM|first|morning|breakfast| +31864|AAAAAAAAJHMHAAAA|31864|8|51|4|AM|first|morning|breakfast| +31865|AAAAAAAAKHMHAAAA|31865|8|51|5|AM|first|morning|breakfast| +31866|AAAAAAAALHMHAAAA|31866|8|51|6|AM|first|morning|breakfast| +31867|AAAAAAAAMHMHAAAA|31867|8|51|7|AM|first|morning|breakfast| +31868|AAAAAAAANHMHAAAA|31868|8|51|8|AM|first|morning|breakfast| +31869|AAAAAAAAOHMHAAAA|31869|8|51|9|AM|first|morning|breakfast| +31870|AAAAAAAAPHMHAAAA|31870|8|51|10|AM|first|morning|breakfast| +31871|AAAAAAAAAIMHAAAA|31871|8|51|11|AM|first|morning|breakfast| +31872|AAAAAAAABIMHAAAA|31872|8|51|12|AM|first|morning|breakfast| +31873|AAAAAAAACIMHAAAA|31873|8|51|13|AM|first|morning|breakfast| +31874|AAAAAAAADIMHAAAA|31874|8|51|14|AM|first|morning|breakfast| +31875|AAAAAAAAEIMHAAAA|31875|8|51|15|AM|first|morning|breakfast| +31876|AAAAAAAAFIMHAAAA|31876|8|51|16|AM|first|morning|breakfast| +31877|AAAAAAAAGIMHAAAA|31877|8|51|17|AM|first|morning|breakfast| +31878|AAAAAAAAHIMHAAAA|31878|8|51|18|AM|first|morning|breakfast| +31879|AAAAAAAAIIMHAAAA|31879|8|51|19|AM|first|morning|breakfast| +31880|AAAAAAAAJIMHAAAA|31880|8|51|20|AM|first|morning|breakfast| +31881|AAAAAAAAKIMHAAAA|31881|8|51|21|AM|first|morning|breakfast| +31882|AAAAAAAALIMHAAAA|31882|8|51|22|AM|first|morning|breakfast| +31883|AAAAAAAAMIMHAAAA|31883|8|51|23|AM|first|morning|breakfast| +31884|AAAAAAAANIMHAAAA|31884|8|51|24|AM|first|morning|breakfast| +31885|AAAAAAAAOIMHAAAA|31885|8|51|25|AM|first|morning|breakfast| +31886|AAAAAAAAPIMHAAAA|31886|8|51|26|AM|first|morning|breakfast| +31887|AAAAAAAAAJMHAAAA|31887|8|51|27|AM|first|morning|breakfast| +31888|AAAAAAAABJMHAAAA|31888|8|51|28|AM|first|morning|breakfast| +31889|AAAAAAAACJMHAAAA|31889|8|51|29|AM|first|morning|breakfast| +31890|AAAAAAAADJMHAAAA|31890|8|51|30|AM|first|morning|breakfast| +31891|AAAAAAAAEJMHAAAA|31891|8|51|31|AM|first|morning|breakfast| +31892|AAAAAAAAFJMHAAAA|31892|8|51|32|AM|first|morning|breakfast| +31893|AAAAAAAAGJMHAAAA|31893|8|51|33|AM|first|morning|breakfast| +31894|AAAAAAAAHJMHAAAA|31894|8|51|34|AM|first|morning|breakfast| +31895|AAAAAAAAIJMHAAAA|31895|8|51|35|AM|first|morning|breakfast| +31896|AAAAAAAAJJMHAAAA|31896|8|51|36|AM|first|morning|breakfast| +31897|AAAAAAAAKJMHAAAA|31897|8|51|37|AM|first|morning|breakfast| +31898|AAAAAAAALJMHAAAA|31898|8|51|38|AM|first|morning|breakfast| +31899|AAAAAAAAMJMHAAAA|31899|8|51|39|AM|first|morning|breakfast| +31900|AAAAAAAANJMHAAAA|31900|8|51|40|AM|first|morning|breakfast| +31901|AAAAAAAAOJMHAAAA|31901|8|51|41|AM|first|morning|breakfast| +31902|AAAAAAAAPJMHAAAA|31902|8|51|42|AM|first|morning|breakfast| +31903|AAAAAAAAAKMHAAAA|31903|8|51|43|AM|first|morning|breakfast| +31904|AAAAAAAABKMHAAAA|31904|8|51|44|AM|first|morning|breakfast| +31905|AAAAAAAACKMHAAAA|31905|8|51|45|AM|first|morning|breakfast| +31906|AAAAAAAADKMHAAAA|31906|8|51|46|AM|first|morning|breakfast| +31907|AAAAAAAAEKMHAAAA|31907|8|51|47|AM|first|morning|breakfast| +31908|AAAAAAAAFKMHAAAA|31908|8|51|48|AM|first|morning|breakfast| +31909|AAAAAAAAGKMHAAAA|31909|8|51|49|AM|first|morning|breakfast| +31910|AAAAAAAAHKMHAAAA|31910|8|51|50|AM|first|morning|breakfast| +31911|AAAAAAAAIKMHAAAA|31911|8|51|51|AM|first|morning|breakfast| +31912|AAAAAAAAJKMHAAAA|31912|8|51|52|AM|first|morning|breakfast| +31913|AAAAAAAAKKMHAAAA|31913|8|51|53|AM|first|morning|breakfast| +31914|AAAAAAAALKMHAAAA|31914|8|51|54|AM|first|morning|breakfast| +31915|AAAAAAAAMKMHAAAA|31915|8|51|55|AM|first|morning|breakfast| +31916|AAAAAAAANKMHAAAA|31916|8|51|56|AM|first|morning|breakfast| +31917|AAAAAAAAOKMHAAAA|31917|8|51|57|AM|first|morning|breakfast| +31918|AAAAAAAAPKMHAAAA|31918|8|51|58|AM|first|morning|breakfast| +31919|AAAAAAAAALMHAAAA|31919|8|51|59|AM|first|morning|breakfast| +31920|AAAAAAAABLMHAAAA|31920|8|52|0|AM|first|morning|breakfast| +31921|AAAAAAAACLMHAAAA|31921|8|52|1|AM|first|morning|breakfast| +31922|AAAAAAAADLMHAAAA|31922|8|52|2|AM|first|morning|breakfast| +31923|AAAAAAAAELMHAAAA|31923|8|52|3|AM|first|morning|breakfast| +31924|AAAAAAAAFLMHAAAA|31924|8|52|4|AM|first|morning|breakfast| +31925|AAAAAAAAGLMHAAAA|31925|8|52|5|AM|first|morning|breakfast| +31926|AAAAAAAAHLMHAAAA|31926|8|52|6|AM|first|morning|breakfast| +31927|AAAAAAAAILMHAAAA|31927|8|52|7|AM|first|morning|breakfast| +31928|AAAAAAAAJLMHAAAA|31928|8|52|8|AM|first|morning|breakfast| +31929|AAAAAAAAKLMHAAAA|31929|8|52|9|AM|first|morning|breakfast| +31930|AAAAAAAALLMHAAAA|31930|8|52|10|AM|first|morning|breakfast| +31931|AAAAAAAAMLMHAAAA|31931|8|52|11|AM|first|morning|breakfast| +31932|AAAAAAAANLMHAAAA|31932|8|52|12|AM|first|morning|breakfast| +31933|AAAAAAAAOLMHAAAA|31933|8|52|13|AM|first|morning|breakfast| +31934|AAAAAAAAPLMHAAAA|31934|8|52|14|AM|first|morning|breakfast| +31935|AAAAAAAAAMMHAAAA|31935|8|52|15|AM|first|morning|breakfast| +31936|AAAAAAAABMMHAAAA|31936|8|52|16|AM|first|morning|breakfast| +31937|AAAAAAAACMMHAAAA|31937|8|52|17|AM|first|morning|breakfast| +31938|AAAAAAAADMMHAAAA|31938|8|52|18|AM|first|morning|breakfast| +31939|AAAAAAAAEMMHAAAA|31939|8|52|19|AM|first|morning|breakfast| +31940|AAAAAAAAFMMHAAAA|31940|8|52|20|AM|first|morning|breakfast| +31941|AAAAAAAAGMMHAAAA|31941|8|52|21|AM|first|morning|breakfast| +31942|AAAAAAAAHMMHAAAA|31942|8|52|22|AM|first|morning|breakfast| +31943|AAAAAAAAIMMHAAAA|31943|8|52|23|AM|first|morning|breakfast| +31944|AAAAAAAAJMMHAAAA|31944|8|52|24|AM|first|morning|breakfast| +31945|AAAAAAAAKMMHAAAA|31945|8|52|25|AM|first|morning|breakfast| +31946|AAAAAAAALMMHAAAA|31946|8|52|26|AM|first|morning|breakfast| +31947|AAAAAAAAMMMHAAAA|31947|8|52|27|AM|first|morning|breakfast| +31948|AAAAAAAANMMHAAAA|31948|8|52|28|AM|first|morning|breakfast| +31949|AAAAAAAAOMMHAAAA|31949|8|52|29|AM|first|morning|breakfast| +31950|AAAAAAAAPMMHAAAA|31950|8|52|30|AM|first|morning|breakfast| +31951|AAAAAAAAANMHAAAA|31951|8|52|31|AM|first|morning|breakfast| +31952|AAAAAAAABNMHAAAA|31952|8|52|32|AM|first|morning|breakfast| +31953|AAAAAAAACNMHAAAA|31953|8|52|33|AM|first|morning|breakfast| +31954|AAAAAAAADNMHAAAA|31954|8|52|34|AM|first|morning|breakfast| +31955|AAAAAAAAENMHAAAA|31955|8|52|35|AM|first|morning|breakfast| +31956|AAAAAAAAFNMHAAAA|31956|8|52|36|AM|first|morning|breakfast| +31957|AAAAAAAAGNMHAAAA|31957|8|52|37|AM|first|morning|breakfast| +31958|AAAAAAAAHNMHAAAA|31958|8|52|38|AM|first|morning|breakfast| +31959|AAAAAAAAINMHAAAA|31959|8|52|39|AM|first|morning|breakfast| +31960|AAAAAAAAJNMHAAAA|31960|8|52|40|AM|first|morning|breakfast| +31961|AAAAAAAAKNMHAAAA|31961|8|52|41|AM|first|morning|breakfast| +31962|AAAAAAAALNMHAAAA|31962|8|52|42|AM|first|morning|breakfast| +31963|AAAAAAAAMNMHAAAA|31963|8|52|43|AM|first|morning|breakfast| +31964|AAAAAAAANNMHAAAA|31964|8|52|44|AM|first|morning|breakfast| +31965|AAAAAAAAONMHAAAA|31965|8|52|45|AM|first|morning|breakfast| +31966|AAAAAAAAPNMHAAAA|31966|8|52|46|AM|first|morning|breakfast| +31967|AAAAAAAAAOMHAAAA|31967|8|52|47|AM|first|morning|breakfast| +31968|AAAAAAAABOMHAAAA|31968|8|52|48|AM|first|morning|breakfast| +31969|AAAAAAAACOMHAAAA|31969|8|52|49|AM|first|morning|breakfast| +31970|AAAAAAAADOMHAAAA|31970|8|52|50|AM|first|morning|breakfast| +31971|AAAAAAAAEOMHAAAA|31971|8|52|51|AM|first|morning|breakfast| +31972|AAAAAAAAFOMHAAAA|31972|8|52|52|AM|first|morning|breakfast| +31973|AAAAAAAAGOMHAAAA|31973|8|52|53|AM|first|morning|breakfast| +31974|AAAAAAAAHOMHAAAA|31974|8|52|54|AM|first|morning|breakfast| +31975|AAAAAAAAIOMHAAAA|31975|8|52|55|AM|first|morning|breakfast| +31976|AAAAAAAAJOMHAAAA|31976|8|52|56|AM|first|morning|breakfast| +31977|AAAAAAAAKOMHAAAA|31977|8|52|57|AM|first|morning|breakfast| +31978|AAAAAAAALOMHAAAA|31978|8|52|58|AM|first|morning|breakfast| +31979|AAAAAAAAMOMHAAAA|31979|8|52|59|AM|first|morning|breakfast| +31980|AAAAAAAANOMHAAAA|31980|8|53|0|AM|first|morning|breakfast| +31981|AAAAAAAAOOMHAAAA|31981|8|53|1|AM|first|morning|breakfast| +31982|AAAAAAAAPOMHAAAA|31982|8|53|2|AM|first|morning|breakfast| +31983|AAAAAAAAAPMHAAAA|31983|8|53|3|AM|first|morning|breakfast| +31984|AAAAAAAABPMHAAAA|31984|8|53|4|AM|first|morning|breakfast| +31985|AAAAAAAACPMHAAAA|31985|8|53|5|AM|first|morning|breakfast| +31986|AAAAAAAADPMHAAAA|31986|8|53|6|AM|first|morning|breakfast| +31987|AAAAAAAAEPMHAAAA|31987|8|53|7|AM|first|morning|breakfast| +31988|AAAAAAAAFPMHAAAA|31988|8|53|8|AM|first|morning|breakfast| +31989|AAAAAAAAGPMHAAAA|31989|8|53|9|AM|first|morning|breakfast| +31990|AAAAAAAAHPMHAAAA|31990|8|53|10|AM|first|morning|breakfast| +31991|AAAAAAAAIPMHAAAA|31991|8|53|11|AM|first|morning|breakfast| +31992|AAAAAAAAJPMHAAAA|31992|8|53|12|AM|first|morning|breakfast| +31993|AAAAAAAAKPMHAAAA|31993|8|53|13|AM|first|morning|breakfast| +31994|AAAAAAAALPMHAAAA|31994|8|53|14|AM|first|morning|breakfast| +31995|AAAAAAAAMPMHAAAA|31995|8|53|15|AM|first|morning|breakfast| +31996|AAAAAAAANPMHAAAA|31996|8|53|16|AM|first|morning|breakfast| +31997|AAAAAAAAOPMHAAAA|31997|8|53|17|AM|first|morning|breakfast| +31998|AAAAAAAAPPMHAAAA|31998|8|53|18|AM|first|morning|breakfast| +31999|AAAAAAAAAANHAAAA|31999|8|53|19|AM|first|morning|breakfast| +32000|AAAAAAAABANHAAAA|32000|8|53|20|AM|first|morning|breakfast| +32001|AAAAAAAACANHAAAA|32001|8|53|21|AM|first|morning|breakfast| +32002|AAAAAAAADANHAAAA|32002|8|53|22|AM|first|morning|breakfast| +32003|AAAAAAAAEANHAAAA|32003|8|53|23|AM|first|morning|breakfast| +32004|AAAAAAAAFANHAAAA|32004|8|53|24|AM|first|morning|breakfast| +32005|AAAAAAAAGANHAAAA|32005|8|53|25|AM|first|morning|breakfast| +32006|AAAAAAAAHANHAAAA|32006|8|53|26|AM|first|morning|breakfast| +32007|AAAAAAAAIANHAAAA|32007|8|53|27|AM|first|morning|breakfast| +32008|AAAAAAAAJANHAAAA|32008|8|53|28|AM|first|morning|breakfast| +32009|AAAAAAAAKANHAAAA|32009|8|53|29|AM|first|morning|breakfast| +32010|AAAAAAAALANHAAAA|32010|8|53|30|AM|first|morning|breakfast| +32011|AAAAAAAAMANHAAAA|32011|8|53|31|AM|first|morning|breakfast| +32012|AAAAAAAANANHAAAA|32012|8|53|32|AM|first|morning|breakfast| +32013|AAAAAAAAOANHAAAA|32013|8|53|33|AM|first|morning|breakfast| +32014|AAAAAAAAPANHAAAA|32014|8|53|34|AM|first|morning|breakfast| +32015|AAAAAAAAABNHAAAA|32015|8|53|35|AM|first|morning|breakfast| +32016|AAAAAAAABBNHAAAA|32016|8|53|36|AM|first|morning|breakfast| +32017|AAAAAAAACBNHAAAA|32017|8|53|37|AM|first|morning|breakfast| +32018|AAAAAAAADBNHAAAA|32018|8|53|38|AM|first|morning|breakfast| +32019|AAAAAAAAEBNHAAAA|32019|8|53|39|AM|first|morning|breakfast| +32020|AAAAAAAAFBNHAAAA|32020|8|53|40|AM|first|morning|breakfast| +32021|AAAAAAAAGBNHAAAA|32021|8|53|41|AM|first|morning|breakfast| +32022|AAAAAAAAHBNHAAAA|32022|8|53|42|AM|first|morning|breakfast| +32023|AAAAAAAAIBNHAAAA|32023|8|53|43|AM|first|morning|breakfast| +32024|AAAAAAAAJBNHAAAA|32024|8|53|44|AM|first|morning|breakfast| +32025|AAAAAAAAKBNHAAAA|32025|8|53|45|AM|first|morning|breakfast| +32026|AAAAAAAALBNHAAAA|32026|8|53|46|AM|first|morning|breakfast| +32027|AAAAAAAAMBNHAAAA|32027|8|53|47|AM|first|morning|breakfast| +32028|AAAAAAAANBNHAAAA|32028|8|53|48|AM|first|morning|breakfast| +32029|AAAAAAAAOBNHAAAA|32029|8|53|49|AM|first|morning|breakfast| +32030|AAAAAAAAPBNHAAAA|32030|8|53|50|AM|first|morning|breakfast| +32031|AAAAAAAAACNHAAAA|32031|8|53|51|AM|first|morning|breakfast| +32032|AAAAAAAABCNHAAAA|32032|8|53|52|AM|first|morning|breakfast| +32033|AAAAAAAACCNHAAAA|32033|8|53|53|AM|first|morning|breakfast| +32034|AAAAAAAADCNHAAAA|32034|8|53|54|AM|first|morning|breakfast| +32035|AAAAAAAAECNHAAAA|32035|8|53|55|AM|first|morning|breakfast| +32036|AAAAAAAAFCNHAAAA|32036|8|53|56|AM|first|morning|breakfast| +32037|AAAAAAAAGCNHAAAA|32037|8|53|57|AM|first|morning|breakfast| +32038|AAAAAAAAHCNHAAAA|32038|8|53|58|AM|first|morning|breakfast| +32039|AAAAAAAAICNHAAAA|32039|8|53|59|AM|first|morning|breakfast| +32040|AAAAAAAAJCNHAAAA|32040|8|54|0|AM|first|morning|breakfast| +32041|AAAAAAAAKCNHAAAA|32041|8|54|1|AM|first|morning|breakfast| +32042|AAAAAAAALCNHAAAA|32042|8|54|2|AM|first|morning|breakfast| +32043|AAAAAAAAMCNHAAAA|32043|8|54|3|AM|first|morning|breakfast| +32044|AAAAAAAANCNHAAAA|32044|8|54|4|AM|first|morning|breakfast| +32045|AAAAAAAAOCNHAAAA|32045|8|54|5|AM|first|morning|breakfast| +32046|AAAAAAAAPCNHAAAA|32046|8|54|6|AM|first|morning|breakfast| +32047|AAAAAAAAADNHAAAA|32047|8|54|7|AM|first|morning|breakfast| +32048|AAAAAAAABDNHAAAA|32048|8|54|8|AM|first|morning|breakfast| +32049|AAAAAAAACDNHAAAA|32049|8|54|9|AM|first|morning|breakfast| +32050|AAAAAAAADDNHAAAA|32050|8|54|10|AM|first|morning|breakfast| +32051|AAAAAAAAEDNHAAAA|32051|8|54|11|AM|first|morning|breakfast| +32052|AAAAAAAAFDNHAAAA|32052|8|54|12|AM|first|morning|breakfast| +32053|AAAAAAAAGDNHAAAA|32053|8|54|13|AM|first|morning|breakfast| +32054|AAAAAAAAHDNHAAAA|32054|8|54|14|AM|first|morning|breakfast| +32055|AAAAAAAAIDNHAAAA|32055|8|54|15|AM|first|morning|breakfast| +32056|AAAAAAAAJDNHAAAA|32056|8|54|16|AM|first|morning|breakfast| +32057|AAAAAAAAKDNHAAAA|32057|8|54|17|AM|first|morning|breakfast| +32058|AAAAAAAALDNHAAAA|32058|8|54|18|AM|first|morning|breakfast| +32059|AAAAAAAAMDNHAAAA|32059|8|54|19|AM|first|morning|breakfast| +32060|AAAAAAAANDNHAAAA|32060|8|54|20|AM|first|morning|breakfast| +32061|AAAAAAAAODNHAAAA|32061|8|54|21|AM|first|morning|breakfast| +32062|AAAAAAAAPDNHAAAA|32062|8|54|22|AM|first|morning|breakfast| +32063|AAAAAAAAAENHAAAA|32063|8|54|23|AM|first|morning|breakfast| +32064|AAAAAAAABENHAAAA|32064|8|54|24|AM|first|morning|breakfast| +32065|AAAAAAAACENHAAAA|32065|8|54|25|AM|first|morning|breakfast| +32066|AAAAAAAADENHAAAA|32066|8|54|26|AM|first|morning|breakfast| +32067|AAAAAAAAEENHAAAA|32067|8|54|27|AM|first|morning|breakfast| +32068|AAAAAAAAFENHAAAA|32068|8|54|28|AM|first|morning|breakfast| +32069|AAAAAAAAGENHAAAA|32069|8|54|29|AM|first|morning|breakfast| +32070|AAAAAAAAHENHAAAA|32070|8|54|30|AM|first|morning|breakfast| +32071|AAAAAAAAIENHAAAA|32071|8|54|31|AM|first|morning|breakfast| +32072|AAAAAAAAJENHAAAA|32072|8|54|32|AM|first|morning|breakfast| +32073|AAAAAAAAKENHAAAA|32073|8|54|33|AM|first|morning|breakfast| +32074|AAAAAAAALENHAAAA|32074|8|54|34|AM|first|morning|breakfast| +32075|AAAAAAAAMENHAAAA|32075|8|54|35|AM|first|morning|breakfast| +32076|AAAAAAAANENHAAAA|32076|8|54|36|AM|first|morning|breakfast| +32077|AAAAAAAAOENHAAAA|32077|8|54|37|AM|first|morning|breakfast| +32078|AAAAAAAAPENHAAAA|32078|8|54|38|AM|first|morning|breakfast| +32079|AAAAAAAAAFNHAAAA|32079|8|54|39|AM|first|morning|breakfast| +32080|AAAAAAAABFNHAAAA|32080|8|54|40|AM|first|morning|breakfast| +32081|AAAAAAAACFNHAAAA|32081|8|54|41|AM|first|morning|breakfast| +32082|AAAAAAAADFNHAAAA|32082|8|54|42|AM|first|morning|breakfast| +32083|AAAAAAAAEFNHAAAA|32083|8|54|43|AM|first|morning|breakfast| +32084|AAAAAAAAFFNHAAAA|32084|8|54|44|AM|first|morning|breakfast| +32085|AAAAAAAAGFNHAAAA|32085|8|54|45|AM|first|morning|breakfast| +32086|AAAAAAAAHFNHAAAA|32086|8|54|46|AM|first|morning|breakfast| +32087|AAAAAAAAIFNHAAAA|32087|8|54|47|AM|first|morning|breakfast| +32088|AAAAAAAAJFNHAAAA|32088|8|54|48|AM|first|morning|breakfast| +32089|AAAAAAAAKFNHAAAA|32089|8|54|49|AM|first|morning|breakfast| +32090|AAAAAAAALFNHAAAA|32090|8|54|50|AM|first|morning|breakfast| +32091|AAAAAAAAMFNHAAAA|32091|8|54|51|AM|first|morning|breakfast| +32092|AAAAAAAANFNHAAAA|32092|8|54|52|AM|first|morning|breakfast| +32093|AAAAAAAAOFNHAAAA|32093|8|54|53|AM|first|morning|breakfast| +32094|AAAAAAAAPFNHAAAA|32094|8|54|54|AM|first|morning|breakfast| +32095|AAAAAAAAAGNHAAAA|32095|8|54|55|AM|first|morning|breakfast| +32096|AAAAAAAABGNHAAAA|32096|8|54|56|AM|first|morning|breakfast| +32097|AAAAAAAACGNHAAAA|32097|8|54|57|AM|first|morning|breakfast| +32098|AAAAAAAADGNHAAAA|32098|8|54|58|AM|first|morning|breakfast| +32099|AAAAAAAAEGNHAAAA|32099|8|54|59|AM|first|morning|breakfast| +32100|AAAAAAAAFGNHAAAA|32100|8|55|0|AM|first|morning|breakfast| +32101|AAAAAAAAGGNHAAAA|32101|8|55|1|AM|first|morning|breakfast| +32102|AAAAAAAAHGNHAAAA|32102|8|55|2|AM|first|morning|breakfast| +32103|AAAAAAAAIGNHAAAA|32103|8|55|3|AM|first|morning|breakfast| +32104|AAAAAAAAJGNHAAAA|32104|8|55|4|AM|first|morning|breakfast| +32105|AAAAAAAAKGNHAAAA|32105|8|55|5|AM|first|morning|breakfast| +32106|AAAAAAAALGNHAAAA|32106|8|55|6|AM|first|morning|breakfast| +32107|AAAAAAAAMGNHAAAA|32107|8|55|7|AM|first|morning|breakfast| +32108|AAAAAAAANGNHAAAA|32108|8|55|8|AM|first|morning|breakfast| +32109|AAAAAAAAOGNHAAAA|32109|8|55|9|AM|first|morning|breakfast| +32110|AAAAAAAAPGNHAAAA|32110|8|55|10|AM|first|morning|breakfast| +32111|AAAAAAAAAHNHAAAA|32111|8|55|11|AM|first|morning|breakfast| +32112|AAAAAAAABHNHAAAA|32112|8|55|12|AM|first|morning|breakfast| +32113|AAAAAAAACHNHAAAA|32113|8|55|13|AM|first|morning|breakfast| +32114|AAAAAAAADHNHAAAA|32114|8|55|14|AM|first|morning|breakfast| +32115|AAAAAAAAEHNHAAAA|32115|8|55|15|AM|first|morning|breakfast| +32116|AAAAAAAAFHNHAAAA|32116|8|55|16|AM|first|morning|breakfast| +32117|AAAAAAAAGHNHAAAA|32117|8|55|17|AM|first|morning|breakfast| +32118|AAAAAAAAHHNHAAAA|32118|8|55|18|AM|first|morning|breakfast| +32119|AAAAAAAAIHNHAAAA|32119|8|55|19|AM|first|morning|breakfast| +32120|AAAAAAAAJHNHAAAA|32120|8|55|20|AM|first|morning|breakfast| +32121|AAAAAAAAKHNHAAAA|32121|8|55|21|AM|first|morning|breakfast| +32122|AAAAAAAALHNHAAAA|32122|8|55|22|AM|first|morning|breakfast| +32123|AAAAAAAAMHNHAAAA|32123|8|55|23|AM|first|morning|breakfast| +32124|AAAAAAAANHNHAAAA|32124|8|55|24|AM|first|morning|breakfast| +32125|AAAAAAAAOHNHAAAA|32125|8|55|25|AM|first|morning|breakfast| +32126|AAAAAAAAPHNHAAAA|32126|8|55|26|AM|first|morning|breakfast| +32127|AAAAAAAAAINHAAAA|32127|8|55|27|AM|first|morning|breakfast| +32128|AAAAAAAABINHAAAA|32128|8|55|28|AM|first|morning|breakfast| +32129|AAAAAAAACINHAAAA|32129|8|55|29|AM|first|morning|breakfast| +32130|AAAAAAAADINHAAAA|32130|8|55|30|AM|first|morning|breakfast| +32131|AAAAAAAAEINHAAAA|32131|8|55|31|AM|first|morning|breakfast| +32132|AAAAAAAAFINHAAAA|32132|8|55|32|AM|first|morning|breakfast| +32133|AAAAAAAAGINHAAAA|32133|8|55|33|AM|first|morning|breakfast| +32134|AAAAAAAAHINHAAAA|32134|8|55|34|AM|first|morning|breakfast| +32135|AAAAAAAAIINHAAAA|32135|8|55|35|AM|first|morning|breakfast| +32136|AAAAAAAAJINHAAAA|32136|8|55|36|AM|first|morning|breakfast| +32137|AAAAAAAAKINHAAAA|32137|8|55|37|AM|first|morning|breakfast| +32138|AAAAAAAALINHAAAA|32138|8|55|38|AM|first|morning|breakfast| +32139|AAAAAAAAMINHAAAA|32139|8|55|39|AM|first|morning|breakfast| +32140|AAAAAAAANINHAAAA|32140|8|55|40|AM|first|morning|breakfast| +32141|AAAAAAAAOINHAAAA|32141|8|55|41|AM|first|morning|breakfast| +32142|AAAAAAAAPINHAAAA|32142|8|55|42|AM|first|morning|breakfast| +32143|AAAAAAAAAJNHAAAA|32143|8|55|43|AM|first|morning|breakfast| +32144|AAAAAAAABJNHAAAA|32144|8|55|44|AM|first|morning|breakfast| +32145|AAAAAAAACJNHAAAA|32145|8|55|45|AM|first|morning|breakfast| +32146|AAAAAAAADJNHAAAA|32146|8|55|46|AM|first|morning|breakfast| +32147|AAAAAAAAEJNHAAAA|32147|8|55|47|AM|first|morning|breakfast| +32148|AAAAAAAAFJNHAAAA|32148|8|55|48|AM|first|morning|breakfast| +32149|AAAAAAAAGJNHAAAA|32149|8|55|49|AM|first|morning|breakfast| +32150|AAAAAAAAHJNHAAAA|32150|8|55|50|AM|first|morning|breakfast| +32151|AAAAAAAAIJNHAAAA|32151|8|55|51|AM|first|morning|breakfast| +32152|AAAAAAAAJJNHAAAA|32152|8|55|52|AM|first|morning|breakfast| +32153|AAAAAAAAKJNHAAAA|32153|8|55|53|AM|first|morning|breakfast| +32154|AAAAAAAALJNHAAAA|32154|8|55|54|AM|first|morning|breakfast| +32155|AAAAAAAAMJNHAAAA|32155|8|55|55|AM|first|morning|breakfast| +32156|AAAAAAAANJNHAAAA|32156|8|55|56|AM|first|morning|breakfast| +32157|AAAAAAAAOJNHAAAA|32157|8|55|57|AM|first|morning|breakfast| +32158|AAAAAAAAPJNHAAAA|32158|8|55|58|AM|first|morning|breakfast| +32159|AAAAAAAAAKNHAAAA|32159|8|55|59|AM|first|morning|breakfast| +32160|AAAAAAAABKNHAAAA|32160|8|56|0|AM|first|morning|breakfast| +32161|AAAAAAAACKNHAAAA|32161|8|56|1|AM|first|morning|breakfast| +32162|AAAAAAAADKNHAAAA|32162|8|56|2|AM|first|morning|breakfast| +32163|AAAAAAAAEKNHAAAA|32163|8|56|3|AM|first|morning|breakfast| +32164|AAAAAAAAFKNHAAAA|32164|8|56|4|AM|first|morning|breakfast| +32165|AAAAAAAAGKNHAAAA|32165|8|56|5|AM|first|morning|breakfast| +32166|AAAAAAAAHKNHAAAA|32166|8|56|6|AM|first|morning|breakfast| +32167|AAAAAAAAIKNHAAAA|32167|8|56|7|AM|first|morning|breakfast| +32168|AAAAAAAAJKNHAAAA|32168|8|56|8|AM|first|morning|breakfast| +32169|AAAAAAAAKKNHAAAA|32169|8|56|9|AM|first|morning|breakfast| +32170|AAAAAAAALKNHAAAA|32170|8|56|10|AM|first|morning|breakfast| +32171|AAAAAAAAMKNHAAAA|32171|8|56|11|AM|first|morning|breakfast| +32172|AAAAAAAANKNHAAAA|32172|8|56|12|AM|first|morning|breakfast| +32173|AAAAAAAAOKNHAAAA|32173|8|56|13|AM|first|morning|breakfast| +32174|AAAAAAAAPKNHAAAA|32174|8|56|14|AM|first|morning|breakfast| +32175|AAAAAAAAALNHAAAA|32175|8|56|15|AM|first|morning|breakfast| +32176|AAAAAAAABLNHAAAA|32176|8|56|16|AM|first|morning|breakfast| +32177|AAAAAAAACLNHAAAA|32177|8|56|17|AM|first|morning|breakfast| +32178|AAAAAAAADLNHAAAA|32178|8|56|18|AM|first|morning|breakfast| +32179|AAAAAAAAELNHAAAA|32179|8|56|19|AM|first|morning|breakfast| +32180|AAAAAAAAFLNHAAAA|32180|8|56|20|AM|first|morning|breakfast| +32181|AAAAAAAAGLNHAAAA|32181|8|56|21|AM|first|morning|breakfast| +32182|AAAAAAAAHLNHAAAA|32182|8|56|22|AM|first|morning|breakfast| +32183|AAAAAAAAILNHAAAA|32183|8|56|23|AM|first|morning|breakfast| +32184|AAAAAAAAJLNHAAAA|32184|8|56|24|AM|first|morning|breakfast| +32185|AAAAAAAAKLNHAAAA|32185|8|56|25|AM|first|morning|breakfast| +32186|AAAAAAAALLNHAAAA|32186|8|56|26|AM|first|morning|breakfast| +32187|AAAAAAAAMLNHAAAA|32187|8|56|27|AM|first|morning|breakfast| +32188|AAAAAAAANLNHAAAA|32188|8|56|28|AM|first|morning|breakfast| +32189|AAAAAAAAOLNHAAAA|32189|8|56|29|AM|first|morning|breakfast| +32190|AAAAAAAAPLNHAAAA|32190|8|56|30|AM|first|morning|breakfast| +32191|AAAAAAAAAMNHAAAA|32191|8|56|31|AM|first|morning|breakfast| +32192|AAAAAAAABMNHAAAA|32192|8|56|32|AM|first|morning|breakfast| +32193|AAAAAAAACMNHAAAA|32193|8|56|33|AM|first|morning|breakfast| +32194|AAAAAAAADMNHAAAA|32194|8|56|34|AM|first|morning|breakfast| +32195|AAAAAAAAEMNHAAAA|32195|8|56|35|AM|first|morning|breakfast| +32196|AAAAAAAAFMNHAAAA|32196|8|56|36|AM|first|morning|breakfast| +32197|AAAAAAAAGMNHAAAA|32197|8|56|37|AM|first|morning|breakfast| +32198|AAAAAAAAHMNHAAAA|32198|8|56|38|AM|first|morning|breakfast| +32199|AAAAAAAAIMNHAAAA|32199|8|56|39|AM|first|morning|breakfast| +32200|AAAAAAAAJMNHAAAA|32200|8|56|40|AM|first|morning|breakfast| +32201|AAAAAAAAKMNHAAAA|32201|8|56|41|AM|first|morning|breakfast| +32202|AAAAAAAALMNHAAAA|32202|8|56|42|AM|first|morning|breakfast| +32203|AAAAAAAAMMNHAAAA|32203|8|56|43|AM|first|morning|breakfast| +32204|AAAAAAAANMNHAAAA|32204|8|56|44|AM|first|morning|breakfast| +32205|AAAAAAAAOMNHAAAA|32205|8|56|45|AM|first|morning|breakfast| +32206|AAAAAAAAPMNHAAAA|32206|8|56|46|AM|first|morning|breakfast| +32207|AAAAAAAAANNHAAAA|32207|8|56|47|AM|first|morning|breakfast| +32208|AAAAAAAABNNHAAAA|32208|8|56|48|AM|first|morning|breakfast| +32209|AAAAAAAACNNHAAAA|32209|8|56|49|AM|first|morning|breakfast| +32210|AAAAAAAADNNHAAAA|32210|8|56|50|AM|first|morning|breakfast| +32211|AAAAAAAAENNHAAAA|32211|8|56|51|AM|first|morning|breakfast| +32212|AAAAAAAAFNNHAAAA|32212|8|56|52|AM|first|morning|breakfast| +32213|AAAAAAAAGNNHAAAA|32213|8|56|53|AM|first|morning|breakfast| +32214|AAAAAAAAHNNHAAAA|32214|8|56|54|AM|first|morning|breakfast| +32215|AAAAAAAAINNHAAAA|32215|8|56|55|AM|first|morning|breakfast| +32216|AAAAAAAAJNNHAAAA|32216|8|56|56|AM|first|morning|breakfast| +32217|AAAAAAAAKNNHAAAA|32217|8|56|57|AM|first|morning|breakfast| +32218|AAAAAAAALNNHAAAA|32218|8|56|58|AM|first|morning|breakfast| +32219|AAAAAAAAMNNHAAAA|32219|8|56|59|AM|first|morning|breakfast| +32220|AAAAAAAANNNHAAAA|32220|8|57|0|AM|first|morning|breakfast| +32221|AAAAAAAAONNHAAAA|32221|8|57|1|AM|first|morning|breakfast| +32222|AAAAAAAAPNNHAAAA|32222|8|57|2|AM|first|morning|breakfast| +32223|AAAAAAAAAONHAAAA|32223|8|57|3|AM|first|morning|breakfast| +32224|AAAAAAAABONHAAAA|32224|8|57|4|AM|first|morning|breakfast| +32225|AAAAAAAACONHAAAA|32225|8|57|5|AM|first|morning|breakfast| +32226|AAAAAAAADONHAAAA|32226|8|57|6|AM|first|morning|breakfast| +32227|AAAAAAAAEONHAAAA|32227|8|57|7|AM|first|morning|breakfast| +32228|AAAAAAAAFONHAAAA|32228|8|57|8|AM|first|morning|breakfast| +32229|AAAAAAAAGONHAAAA|32229|8|57|9|AM|first|morning|breakfast| +32230|AAAAAAAAHONHAAAA|32230|8|57|10|AM|first|morning|breakfast| +32231|AAAAAAAAIONHAAAA|32231|8|57|11|AM|first|morning|breakfast| +32232|AAAAAAAAJONHAAAA|32232|8|57|12|AM|first|morning|breakfast| +32233|AAAAAAAAKONHAAAA|32233|8|57|13|AM|first|morning|breakfast| +32234|AAAAAAAALONHAAAA|32234|8|57|14|AM|first|morning|breakfast| +32235|AAAAAAAAMONHAAAA|32235|8|57|15|AM|first|morning|breakfast| +32236|AAAAAAAANONHAAAA|32236|8|57|16|AM|first|morning|breakfast| +32237|AAAAAAAAOONHAAAA|32237|8|57|17|AM|first|morning|breakfast| +32238|AAAAAAAAPONHAAAA|32238|8|57|18|AM|first|morning|breakfast| +32239|AAAAAAAAAPNHAAAA|32239|8|57|19|AM|first|morning|breakfast| +32240|AAAAAAAABPNHAAAA|32240|8|57|20|AM|first|morning|breakfast| +32241|AAAAAAAACPNHAAAA|32241|8|57|21|AM|first|morning|breakfast| +32242|AAAAAAAADPNHAAAA|32242|8|57|22|AM|first|morning|breakfast| +32243|AAAAAAAAEPNHAAAA|32243|8|57|23|AM|first|morning|breakfast| +32244|AAAAAAAAFPNHAAAA|32244|8|57|24|AM|first|morning|breakfast| +32245|AAAAAAAAGPNHAAAA|32245|8|57|25|AM|first|morning|breakfast| +32246|AAAAAAAAHPNHAAAA|32246|8|57|26|AM|first|morning|breakfast| +32247|AAAAAAAAIPNHAAAA|32247|8|57|27|AM|first|morning|breakfast| +32248|AAAAAAAAJPNHAAAA|32248|8|57|28|AM|first|morning|breakfast| +32249|AAAAAAAAKPNHAAAA|32249|8|57|29|AM|first|morning|breakfast| +32250|AAAAAAAALPNHAAAA|32250|8|57|30|AM|first|morning|breakfast| +32251|AAAAAAAAMPNHAAAA|32251|8|57|31|AM|first|morning|breakfast| +32252|AAAAAAAANPNHAAAA|32252|8|57|32|AM|first|morning|breakfast| +32253|AAAAAAAAOPNHAAAA|32253|8|57|33|AM|first|morning|breakfast| +32254|AAAAAAAAPPNHAAAA|32254|8|57|34|AM|first|morning|breakfast| +32255|AAAAAAAAAAOHAAAA|32255|8|57|35|AM|first|morning|breakfast| +32256|AAAAAAAABAOHAAAA|32256|8|57|36|AM|first|morning|breakfast| +32257|AAAAAAAACAOHAAAA|32257|8|57|37|AM|first|morning|breakfast| +32258|AAAAAAAADAOHAAAA|32258|8|57|38|AM|first|morning|breakfast| +32259|AAAAAAAAEAOHAAAA|32259|8|57|39|AM|first|morning|breakfast| +32260|AAAAAAAAFAOHAAAA|32260|8|57|40|AM|first|morning|breakfast| +32261|AAAAAAAAGAOHAAAA|32261|8|57|41|AM|first|morning|breakfast| +32262|AAAAAAAAHAOHAAAA|32262|8|57|42|AM|first|morning|breakfast| +32263|AAAAAAAAIAOHAAAA|32263|8|57|43|AM|first|morning|breakfast| +32264|AAAAAAAAJAOHAAAA|32264|8|57|44|AM|first|morning|breakfast| +32265|AAAAAAAAKAOHAAAA|32265|8|57|45|AM|first|morning|breakfast| +32266|AAAAAAAALAOHAAAA|32266|8|57|46|AM|first|morning|breakfast| +32267|AAAAAAAAMAOHAAAA|32267|8|57|47|AM|first|morning|breakfast| +32268|AAAAAAAANAOHAAAA|32268|8|57|48|AM|first|morning|breakfast| +32269|AAAAAAAAOAOHAAAA|32269|8|57|49|AM|first|morning|breakfast| +32270|AAAAAAAAPAOHAAAA|32270|8|57|50|AM|first|morning|breakfast| +32271|AAAAAAAAABOHAAAA|32271|8|57|51|AM|first|morning|breakfast| +32272|AAAAAAAABBOHAAAA|32272|8|57|52|AM|first|morning|breakfast| +32273|AAAAAAAACBOHAAAA|32273|8|57|53|AM|first|morning|breakfast| +32274|AAAAAAAADBOHAAAA|32274|8|57|54|AM|first|morning|breakfast| +32275|AAAAAAAAEBOHAAAA|32275|8|57|55|AM|first|morning|breakfast| +32276|AAAAAAAAFBOHAAAA|32276|8|57|56|AM|first|morning|breakfast| +32277|AAAAAAAAGBOHAAAA|32277|8|57|57|AM|first|morning|breakfast| +32278|AAAAAAAAHBOHAAAA|32278|8|57|58|AM|first|morning|breakfast| +32279|AAAAAAAAIBOHAAAA|32279|8|57|59|AM|first|morning|breakfast| +32280|AAAAAAAAJBOHAAAA|32280|8|58|0|AM|first|morning|breakfast| +32281|AAAAAAAAKBOHAAAA|32281|8|58|1|AM|first|morning|breakfast| +32282|AAAAAAAALBOHAAAA|32282|8|58|2|AM|first|morning|breakfast| +32283|AAAAAAAAMBOHAAAA|32283|8|58|3|AM|first|morning|breakfast| +32284|AAAAAAAANBOHAAAA|32284|8|58|4|AM|first|morning|breakfast| +32285|AAAAAAAAOBOHAAAA|32285|8|58|5|AM|first|morning|breakfast| +32286|AAAAAAAAPBOHAAAA|32286|8|58|6|AM|first|morning|breakfast| +32287|AAAAAAAAACOHAAAA|32287|8|58|7|AM|first|morning|breakfast| +32288|AAAAAAAABCOHAAAA|32288|8|58|8|AM|first|morning|breakfast| +32289|AAAAAAAACCOHAAAA|32289|8|58|9|AM|first|morning|breakfast| +32290|AAAAAAAADCOHAAAA|32290|8|58|10|AM|first|morning|breakfast| +32291|AAAAAAAAECOHAAAA|32291|8|58|11|AM|first|morning|breakfast| +32292|AAAAAAAAFCOHAAAA|32292|8|58|12|AM|first|morning|breakfast| +32293|AAAAAAAAGCOHAAAA|32293|8|58|13|AM|first|morning|breakfast| +32294|AAAAAAAAHCOHAAAA|32294|8|58|14|AM|first|morning|breakfast| +32295|AAAAAAAAICOHAAAA|32295|8|58|15|AM|first|morning|breakfast| +32296|AAAAAAAAJCOHAAAA|32296|8|58|16|AM|first|morning|breakfast| +32297|AAAAAAAAKCOHAAAA|32297|8|58|17|AM|first|morning|breakfast| +32298|AAAAAAAALCOHAAAA|32298|8|58|18|AM|first|morning|breakfast| +32299|AAAAAAAAMCOHAAAA|32299|8|58|19|AM|first|morning|breakfast| +32300|AAAAAAAANCOHAAAA|32300|8|58|20|AM|first|morning|breakfast| +32301|AAAAAAAAOCOHAAAA|32301|8|58|21|AM|first|morning|breakfast| +32302|AAAAAAAAPCOHAAAA|32302|8|58|22|AM|first|morning|breakfast| +32303|AAAAAAAAADOHAAAA|32303|8|58|23|AM|first|morning|breakfast| +32304|AAAAAAAABDOHAAAA|32304|8|58|24|AM|first|morning|breakfast| +32305|AAAAAAAACDOHAAAA|32305|8|58|25|AM|first|morning|breakfast| +32306|AAAAAAAADDOHAAAA|32306|8|58|26|AM|first|morning|breakfast| +32307|AAAAAAAAEDOHAAAA|32307|8|58|27|AM|first|morning|breakfast| +32308|AAAAAAAAFDOHAAAA|32308|8|58|28|AM|first|morning|breakfast| +32309|AAAAAAAAGDOHAAAA|32309|8|58|29|AM|first|morning|breakfast| +32310|AAAAAAAAHDOHAAAA|32310|8|58|30|AM|first|morning|breakfast| +32311|AAAAAAAAIDOHAAAA|32311|8|58|31|AM|first|morning|breakfast| +32312|AAAAAAAAJDOHAAAA|32312|8|58|32|AM|first|morning|breakfast| +32313|AAAAAAAAKDOHAAAA|32313|8|58|33|AM|first|morning|breakfast| +32314|AAAAAAAALDOHAAAA|32314|8|58|34|AM|first|morning|breakfast| +32315|AAAAAAAAMDOHAAAA|32315|8|58|35|AM|first|morning|breakfast| +32316|AAAAAAAANDOHAAAA|32316|8|58|36|AM|first|morning|breakfast| +32317|AAAAAAAAODOHAAAA|32317|8|58|37|AM|first|morning|breakfast| +32318|AAAAAAAAPDOHAAAA|32318|8|58|38|AM|first|morning|breakfast| +32319|AAAAAAAAAEOHAAAA|32319|8|58|39|AM|first|morning|breakfast| +32320|AAAAAAAABEOHAAAA|32320|8|58|40|AM|first|morning|breakfast| +32321|AAAAAAAACEOHAAAA|32321|8|58|41|AM|first|morning|breakfast| +32322|AAAAAAAADEOHAAAA|32322|8|58|42|AM|first|morning|breakfast| +32323|AAAAAAAAEEOHAAAA|32323|8|58|43|AM|first|morning|breakfast| +32324|AAAAAAAAFEOHAAAA|32324|8|58|44|AM|first|morning|breakfast| +32325|AAAAAAAAGEOHAAAA|32325|8|58|45|AM|first|morning|breakfast| +32326|AAAAAAAAHEOHAAAA|32326|8|58|46|AM|first|morning|breakfast| +32327|AAAAAAAAIEOHAAAA|32327|8|58|47|AM|first|morning|breakfast| +32328|AAAAAAAAJEOHAAAA|32328|8|58|48|AM|first|morning|breakfast| +32329|AAAAAAAAKEOHAAAA|32329|8|58|49|AM|first|morning|breakfast| +32330|AAAAAAAALEOHAAAA|32330|8|58|50|AM|first|morning|breakfast| +32331|AAAAAAAAMEOHAAAA|32331|8|58|51|AM|first|morning|breakfast| +32332|AAAAAAAANEOHAAAA|32332|8|58|52|AM|first|morning|breakfast| +32333|AAAAAAAAOEOHAAAA|32333|8|58|53|AM|first|morning|breakfast| +32334|AAAAAAAAPEOHAAAA|32334|8|58|54|AM|first|morning|breakfast| +32335|AAAAAAAAAFOHAAAA|32335|8|58|55|AM|first|morning|breakfast| +32336|AAAAAAAABFOHAAAA|32336|8|58|56|AM|first|morning|breakfast| +32337|AAAAAAAACFOHAAAA|32337|8|58|57|AM|first|morning|breakfast| +32338|AAAAAAAADFOHAAAA|32338|8|58|58|AM|first|morning|breakfast| +32339|AAAAAAAAEFOHAAAA|32339|8|58|59|AM|first|morning|breakfast| +32340|AAAAAAAAFFOHAAAA|32340|8|59|0|AM|first|morning|breakfast| +32341|AAAAAAAAGFOHAAAA|32341|8|59|1|AM|first|morning|breakfast| +32342|AAAAAAAAHFOHAAAA|32342|8|59|2|AM|first|morning|breakfast| +32343|AAAAAAAAIFOHAAAA|32343|8|59|3|AM|first|morning|breakfast| +32344|AAAAAAAAJFOHAAAA|32344|8|59|4|AM|first|morning|breakfast| +32345|AAAAAAAAKFOHAAAA|32345|8|59|5|AM|first|morning|breakfast| +32346|AAAAAAAALFOHAAAA|32346|8|59|6|AM|first|morning|breakfast| +32347|AAAAAAAAMFOHAAAA|32347|8|59|7|AM|first|morning|breakfast| +32348|AAAAAAAANFOHAAAA|32348|8|59|8|AM|first|morning|breakfast| +32349|AAAAAAAAOFOHAAAA|32349|8|59|9|AM|first|morning|breakfast| +32350|AAAAAAAAPFOHAAAA|32350|8|59|10|AM|first|morning|breakfast| +32351|AAAAAAAAAGOHAAAA|32351|8|59|11|AM|first|morning|breakfast| +32352|AAAAAAAABGOHAAAA|32352|8|59|12|AM|first|morning|breakfast| +32353|AAAAAAAACGOHAAAA|32353|8|59|13|AM|first|morning|breakfast| +32354|AAAAAAAADGOHAAAA|32354|8|59|14|AM|first|morning|breakfast| +32355|AAAAAAAAEGOHAAAA|32355|8|59|15|AM|first|morning|breakfast| +32356|AAAAAAAAFGOHAAAA|32356|8|59|16|AM|first|morning|breakfast| +32357|AAAAAAAAGGOHAAAA|32357|8|59|17|AM|first|morning|breakfast| +32358|AAAAAAAAHGOHAAAA|32358|8|59|18|AM|first|morning|breakfast| +32359|AAAAAAAAIGOHAAAA|32359|8|59|19|AM|first|morning|breakfast| +32360|AAAAAAAAJGOHAAAA|32360|8|59|20|AM|first|morning|breakfast| +32361|AAAAAAAAKGOHAAAA|32361|8|59|21|AM|first|morning|breakfast| +32362|AAAAAAAALGOHAAAA|32362|8|59|22|AM|first|morning|breakfast| +32363|AAAAAAAAMGOHAAAA|32363|8|59|23|AM|first|morning|breakfast| +32364|AAAAAAAANGOHAAAA|32364|8|59|24|AM|first|morning|breakfast| +32365|AAAAAAAAOGOHAAAA|32365|8|59|25|AM|first|morning|breakfast| +32366|AAAAAAAAPGOHAAAA|32366|8|59|26|AM|first|morning|breakfast| +32367|AAAAAAAAAHOHAAAA|32367|8|59|27|AM|first|morning|breakfast| +32368|AAAAAAAABHOHAAAA|32368|8|59|28|AM|first|morning|breakfast| +32369|AAAAAAAACHOHAAAA|32369|8|59|29|AM|first|morning|breakfast| +32370|AAAAAAAADHOHAAAA|32370|8|59|30|AM|first|morning|breakfast| +32371|AAAAAAAAEHOHAAAA|32371|8|59|31|AM|first|morning|breakfast| +32372|AAAAAAAAFHOHAAAA|32372|8|59|32|AM|first|morning|breakfast| +32373|AAAAAAAAGHOHAAAA|32373|8|59|33|AM|first|morning|breakfast| +32374|AAAAAAAAHHOHAAAA|32374|8|59|34|AM|first|morning|breakfast| +32375|AAAAAAAAIHOHAAAA|32375|8|59|35|AM|first|morning|breakfast| +32376|AAAAAAAAJHOHAAAA|32376|8|59|36|AM|first|morning|breakfast| +32377|AAAAAAAAKHOHAAAA|32377|8|59|37|AM|first|morning|breakfast| +32378|AAAAAAAALHOHAAAA|32378|8|59|38|AM|first|morning|breakfast| +32379|AAAAAAAAMHOHAAAA|32379|8|59|39|AM|first|morning|breakfast| +32380|AAAAAAAANHOHAAAA|32380|8|59|40|AM|first|morning|breakfast| +32381|AAAAAAAAOHOHAAAA|32381|8|59|41|AM|first|morning|breakfast| +32382|AAAAAAAAPHOHAAAA|32382|8|59|42|AM|first|morning|breakfast| +32383|AAAAAAAAAIOHAAAA|32383|8|59|43|AM|first|morning|breakfast| +32384|AAAAAAAABIOHAAAA|32384|8|59|44|AM|first|morning|breakfast| +32385|AAAAAAAACIOHAAAA|32385|8|59|45|AM|first|morning|breakfast| +32386|AAAAAAAADIOHAAAA|32386|8|59|46|AM|first|morning|breakfast| +32387|AAAAAAAAEIOHAAAA|32387|8|59|47|AM|first|morning|breakfast| +32388|AAAAAAAAFIOHAAAA|32388|8|59|48|AM|first|morning|breakfast| +32389|AAAAAAAAGIOHAAAA|32389|8|59|49|AM|first|morning|breakfast| +32390|AAAAAAAAHIOHAAAA|32390|8|59|50|AM|first|morning|breakfast| +32391|AAAAAAAAIIOHAAAA|32391|8|59|51|AM|first|morning|breakfast| +32392|AAAAAAAAJIOHAAAA|32392|8|59|52|AM|first|morning|breakfast| +32393|AAAAAAAAKIOHAAAA|32393|8|59|53|AM|first|morning|breakfast| +32394|AAAAAAAALIOHAAAA|32394|8|59|54|AM|first|morning|breakfast| +32395|AAAAAAAAMIOHAAAA|32395|8|59|55|AM|first|morning|breakfast| +32396|AAAAAAAANIOHAAAA|32396|8|59|56|AM|first|morning|breakfast| +32397|AAAAAAAAOIOHAAAA|32397|8|59|57|AM|first|morning|breakfast| +32398|AAAAAAAAPIOHAAAA|32398|8|59|58|AM|first|morning|breakfast| +32399|AAAAAAAAAJOHAAAA|32399|8|59|59|AM|first|morning|breakfast| +32400|AAAAAAAABJOHAAAA|32400|9|0|0|AM|first|morning|breakfast| +32401|AAAAAAAACJOHAAAA|32401|9|0|1|AM|first|morning|breakfast| +32402|AAAAAAAADJOHAAAA|32402|9|0|2|AM|first|morning|breakfast| +32403|AAAAAAAAEJOHAAAA|32403|9|0|3|AM|first|morning|breakfast| +32404|AAAAAAAAFJOHAAAA|32404|9|0|4|AM|first|morning|breakfast| +32405|AAAAAAAAGJOHAAAA|32405|9|0|5|AM|first|morning|breakfast| +32406|AAAAAAAAHJOHAAAA|32406|9|0|6|AM|first|morning|breakfast| +32407|AAAAAAAAIJOHAAAA|32407|9|0|7|AM|first|morning|breakfast| +32408|AAAAAAAAJJOHAAAA|32408|9|0|8|AM|first|morning|breakfast| +32409|AAAAAAAAKJOHAAAA|32409|9|0|9|AM|first|morning|breakfast| +32410|AAAAAAAALJOHAAAA|32410|9|0|10|AM|first|morning|breakfast| +32411|AAAAAAAAMJOHAAAA|32411|9|0|11|AM|first|morning|breakfast| +32412|AAAAAAAANJOHAAAA|32412|9|0|12|AM|first|morning|breakfast| +32413|AAAAAAAAOJOHAAAA|32413|9|0|13|AM|first|morning|breakfast| +32414|AAAAAAAAPJOHAAAA|32414|9|0|14|AM|first|morning|breakfast| +32415|AAAAAAAAAKOHAAAA|32415|9|0|15|AM|first|morning|breakfast| +32416|AAAAAAAABKOHAAAA|32416|9|0|16|AM|first|morning|breakfast| +32417|AAAAAAAACKOHAAAA|32417|9|0|17|AM|first|morning|breakfast| +32418|AAAAAAAADKOHAAAA|32418|9|0|18|AM|first|morning|breakfast| +32419|AAAAAAAAEKOHAAAA|32419|9|0|19|AM|first|morning|breakfast| +32420|AAAAAAAAFKOHAAAA|32420|9|0|20|AM|first|morning|breakfast| +32421|AAAAAAAAGKOHAAAA|32421|9|0|21|AM|first|morning|breakfast| +32422|AAAAAAAAHKOHAAAA|32422|9|0|22|AM|first|morning|breakfast| +32423|AAAAAAAAIKOHAAAA|32423|9|0|23|AM|first|morning|breakfast| +32424|AAAAAAAAJKOHAAAA|32424|9|0|24|AM|first|morning|breakfast| +32425|AAAAAAAAKKOHAAAA|32425|9|0|25|AM|first|morning|breakfast| +32426|AAAAAAAALKOHAAAA|32426|9|0|26|AM|first|morning|breakfast| +32427|AAAAAAAAMKOHAAAA|32427|9|0|27|AM|first|morning|breakfast| +32428|AAAAAAAANKOHAAAA|32428|9|0|28|AM|first|morning|breakfast| +32429|AAAAAAAAOKOHAAAA|32429|9|0|29|AM|first|morning|breakfast| +32430|AAAAAAAAPKOHAAAA|32430|9|0|30|AM|first|morning|breakfast| +32431|AAAAAAAAALOHAAAA|32431|9|0|31|AM|first|morning|breakfast| +32432|AAAAAAAABLOHAAAA|32432|9|0|32|AM|first|morning|breakfast| +32433|AAAAAAAACLOHAAAA|32433|9|0|33|AM|first|morning|breakfast| +32434|AAAAAAAADLOHAAAA|32434|9|0|34|AM|first|morning|breakfast| +32435|AAAAAAAAELOHAAAA|32435|9|0|35|AM|first|morning|breakfast| +32436|AAAAAAAAFLOHAAAA|32436|9|0|36|AM|first|morning|breakfast| +32437|AAAAAAAAGLOHAAAA|32437|9|0|37|AM|first|morning|breakfast| +32438|AAAAAAAAHLOHAAAA|32438|9|0|38|AM|first|morning|breakfast| +32439|AAAAAAAAILOHAAAA|32439|9|0|39|AM|first|morning|breakfast| +32440|AAAAAAAAJLOHAAAA|32440|9|0|40|AM|first|morning|breakfast| +32441|AAAAAAAAKLOHAAAA|32441|9|0|41|AM|first|morning|breakfast| +32442|AAAAAAAALLOHAAAA|32442|9|0|42|AM|first|morning|breakfast| +32443|AAAAAAAAMLOHAAAA|32443|9|0|43|AM|first|morning|breakfast| +32444|AAAAAAAANLOHAAAA|32444|9|0|44|AM|first|morning|breakfast| +32445|AAAAAAAAOLOHAAAA|32445|9|0|45|AM|first|morning|breakfast| +32446|AAAAAAAAPLOHAAAA|32446|9|0|46|AM|first|morning|breakfast| +32447|AAAAAAAAAMOHAAAA|32447|9|0|47|AM|first|morning|breakfast| +32448|AAAAAAAABMOHAAAA|32448|9|0|48|AM|first|morning|breakfast| +32449|AAAAAAAACMOHAAAA|32449|9|0|49|AM|first|morning|breakfast| +32450|AAAAAAAADMOHAAAA|32450|9|0|50|AM|first|morning|breakfast| +32451|AAAAAAAAEMOHAAAA|32451|9|0|51|AM|first|morning|breakfast| +32452|AAAAAAAAFMOHAAAA|32452|9|0|52|AM|first|morning|breakfast| +32453|AAAAAAAAGMOHAAAA|32453|9|0|53|AM|first|morning|breakfast| +32454|AAAAAAAAHMOHAAAA|32454|9|0|54|AM|first|morning|breakfast| +32455|AAAAAAAAIMOHAAAA|32455|9|0|55|AM|first|morning|breakfast| +32456|AAAAAAAAJMOHAAAA|32456|9|0|56|AM|first|morning|breakfast| +32457|AAAAAAAAKMOHAAAA|32457|9|0|57|AM|first|morning|breakfast| +32458|AAAAAAAALMOHAAAA|32458|9|0|58|AM|first|morning|breakfast| +32459|AAAAAAAAMMOHAAAA|32459|9|0|59|AM|first|morning|breakfast| +32460|AAAAAAAANMOHAAAA|32460|9|1|0|AM|first|morning|breakfast| +32461|AAAAAAAAOMOHAAAA|32461|9|1|1|AM|first|morning|breakfast| +32462|AAAAAAAAPMOHAAAA|32462|9|1|2|AM|first|morning|breakfast| +32463|AAAAAAAAANOHAAAA|32463|9|1|3|AM|first|morning|breakfast| +32464|AAAAAAAABNOHAAAA|32464|9|1|4|AM|first|morning|breakfast| +32465|AAAAAAAACNOHAAAA|32465|9|1|5|AM|first|morning|breakfast| +32466|AAAAAAAADNOHAAAA|32466|9|1|6|AM|first|morning|breakfast| +32467|AAAAAAAAENOHAAAA|32467|9|1|7|AM|first|morning|breakfast| +32468|AAAAAAAAFNOHAAAA|32468|9|1|8|AM|first|morning|breakfast| +32469|AAAAAAAAGNOHAAAA|32469|9|1|9|AM|first|morning|breakfast| +32470|AAAAAAAAHNOHAAAA|32470|9|1|10|AM|first|morning|breakfast| +32471|AAAAAAAAINOHAAAA|32471|9|1|11|AM|first|morning|breakfast| +32472|AAAAAAAAJNOHAAAA|32472|9|1|12|AM|first|morning|breakfast| +32473|AAAAAAAAKNOHAAAA|32473|9|1|13|AM|first|morning|breakfast| +32474|AAAAAAAALNOHAAAA|32474|9|1|14|AM|first|morning|breakfast| +32475|AAAAAAAAMNOHAAAA|32475|9|1|15|AM|first|morning|breakfast| +32476|AAAAAAAANNOHAAAA|32476|9|1|16|AM|first|morning|breakfast| +32477|AAAAAAAAONOHAAAA|32477|9|1|17|AM|first|morning|breakfast| +32478|AAAAAAAAPNOHAAAA|32478|9|1|18|AM|first|morning|breakfast| +32479|AAAAAAAAAOOHAAAA|32479|9|1|19|AM|first|morning|breakfast| +32480|AAAAAAAABOOHAAAA|32480|9|1|20|AM|first|morning|breakfast| +32481|AAAAAAAACOOHAAAA|32481|9|1|21|AM|first|morning|breakfast| +32482|AAAAAAAADOOHAAAA|32482|9|1|22|AM|first|morning|breakfast| +32483|AAAAAAAAEOOHAAAA|32483|9|1|23|AM|first|morning|breakfast| +32484|AAAAAAAAFOOHAAAA|32484|9|1|24|AM|first|morning|breakfast| +32485|AAAAAAAAGOOHAAAA|32485|9|1|25|AM|first|morning|breakfast| +32486|AAAAAAAAHOOHAAAA|32486|9|1|26|AM|first|morning|breakfast| +32487|AAAAAAAAIOOHAAAA|32487|9|1|27|AM|first|morning|breakfast| +32488|AAAAAAAAJOOHAAAA|32488|9|1|28|AM|first|morning|breakfast| +32489|AAAAAAAAKOOHAAAA|32489|9|1|29|AM|first|morning|breakfast| +32490|AAAAAAAALOOHAAAA|32490|9|1|30|AM|first|morning|breakfast| +32491|AAAAAAAAMOOHAAAA|32491|9|1|31|AM|first|morning|breakfast| +32492|AAAAAAAANOOHAAAA|32492|9|1|32|AM|first|morning|breakfast| +32493|AAAAAAAAOOOHAAAA|32493|9|1|33|AM|first|morning|breakfast| +32494|AAAAAAAAPOOHAAAA|32494|9|1|34|AM|first|morning|breakfast| +32495|AAAAAAAAAPOHAAAA|32495|9|1|35|AM|first|morning|breakfast| +32496|AAAAAAAABPOHAAAA|32496|9|1|36|AM|first|morning|breakfast| +32497|AAAAAAAACPOHAAAA|32497|9|1|37|AM|first|morning|breakfast| +32498|AAAAAAAADPOHAAAA|32498|9|1|38|AM|first|morning|breakfast| +32499|AAAAAAAAEPOHAAAA|32499|9|1|39|AM|first|morning|breakfast| +32500|AAAAAAAAFPOHAAAA|32500|9|1|40|AM|first|morning|breakfast| +32501|AAAAAAAAGPOHAAAA|32501|9|1|41|AM|first|morning|breakfast| +32502|AAAAAAAAHPOHAAAA|32502|9|1|42|AM|first|morning|breakfast| +32503|AAAAAAAAIPOHAAAA|32503|9|1|43|AM|first|morning|breakfast| +32504|AAAAAAAAJPOHAAAA|32504|9|1|44|AM|first|morning|breakfast| +32505|AAAAAAAAKPOHAAAA|32505|9|1|45|AM|first|morning|breakfast| +32506|AAAAAAAALPOHAAAA|32506|9|1|46|AM|first|morning|breakfast| +32507|AAAAAAAAMPOHAAAA|32507|9|1|47|AM|first|morning|breakfast| +32508|AAAAAAAANPOHAAAA|32508|9|1|48|AM|first|morning|breakfast| +32509|AAAAAAAAOPOHAAAA|32509|9|1|49|AM|first|morning|breakfast| +32510|AAAAAAAAPPOHAAAA|32510|9|1|50|AM|first|morning|breakfast| +32511|AAAAAAAAAAPHAAAA|32511|9|1|51|AM|first|morning|breakfast| +32512|AAAAAAAABAPHAAAA|32512|9|1|52|AM|first|morning|breakfast| +32513|AAAAAAAACAPHAAAA|32513|9|1|53|AM|first|morning|breakfast| +32514|AAAAAAAADAPHAAAA|32514|9|1|54|AM|first|morning|breakfast| +32515|AAAAAAAAEAPHAAAA|32515|9|1|55|AM|first|morning|breakfast| +32516|AAAAAAAAFAPHAAAA|32516|9|1|56|AM|first|morning|breakfast| +32517|AAAAAAAAGAPHAAAA|32517|9|1|57|AM|first|morning|breakfast| +32518|AAAAAAAAHAPHAAAA|32518|9|1|58|AM|first|morning|breakfast| +32519|AAAAAAAAIAPHAAAA|32519|9|1|59|AM|first|morning|breakfast| +32520|AAAAAAAAJAPHAAAA|32520|9|2|0|AM|first|morning|breakfast| +32521|AAAAAAAAKAPHAAAA|32521|9|2|1|AM|first|morning|breakfast| +32522|AAAAAAAALAPHAAAA|32522|9|2|2|AM|first|morning|breakfast| +32523|AAAAAAAAMAPHAAAA|32523|9|2|3|AM|first|morning|breakfast| +32524|AAAAAAAANAPHAAAA|32524|9|2|4|AM|first|morning|breakfast| +32525|AAAAAAAAOAPHAAAA|32525|9|2|5|AM|first|morning|breakfast| +32526|AAAAAAAAPAPHAAAA|32526|9|2|6|AM|first|morning|breakfast| +32527|AAAAAAAAABPHAAAA|32527|9|2|7|AM|first|morning|breakfast| +32528|AAAAAAAABBPHAAAA|32528|9|2|8|AM|first|morning|breakfast| +32529|AAAAAAAACBPHAAAA|32529|9|2|9|AM|first|morning|breakfast| +32530|AAAAAAAADBPHAAAA|32530|9|2|10|AM|first|morning|breakfast| +32531|AAAAAAAAEBPHAAAA|32531|9|2|11|AM|first|morning|breakfast| +32532|AAAAAAAAFBPHAAAA|32532|9|2|12|AM|first|morning|breakfast| +32533|AAAAAAAAGBPHAAAA|32533|9|2|13|AM|first|morning|breakfast| +32534|AAAAAAAAHBPHAAAA|32534|9|2|14|AM|first|morning|breakfast| +32535|AAAAAAAAIBPHAAAA|32535|9|2|15|AM|first|morning|breakfast| +32536|AAAAAAAAJBPHAAAA|32536|9|2|16|AM|first|morning|breakfast| +32537|AAAAAAAAKBPHAAAA|32537|9|2|17|AM|first|morning|breakfast| +32538|AAAAAAAALBPHAAAA|32538|9|2|18|AM|first|morning|breakfast| +32539|AAAAAAAAMBPHAAAA|32539|9|2|19|AM|first|morning|breakfast| +32540|AAAAAAAANBPHAAAA|32540|9|2|20|AM|first|morning|breakfast| +32541|AAAAAAAAOBPHAAAA|32541|9|2|21|AM|first|morning|breakfast| +32542|AAAAAAAAPBPHAAAA|32542|9|2|22|AM|first|morning|breakfast| +32543|AAAAAAAAACPHAAAA|32543|9|2|23|AM|first|morning|breakfast| +32544|AAAAAAAABCPHAAAA|32544|9|2|24|AM|first|morning|breakfast| +32545|AAAAAAAACCPHAAAA|32545|9|2|25|AM|first|morning|breakfast| +32546|AAAAAAAADCPHAAAA|32546|9|2|26|AM|first|morning|breakfast| +32547|AAAAAAAAECPHAAAA|32547|9|2|27|AM|first|morning|breakfast| +32548|AAAAAAAAFCPHAAAA|32548|9|2|28|AM|first|morning|breakfast| +32549|AAAAAAAAGCPHAAAA|32549|9|2|29|AM|first|morning|breakfast| +32550|AAAAAAAAHCPHAAAA|32550|9|2|30|AM|first|morning|breakfast| +32551|AAAAAAAAICPHAAAA|32551|9|2|31|AM|first|morning|breakfast| +32552|AAAAAAAAJCPHAAAA|32552|9|2|32|AM|first|morning|breakfast| +32553|AAAAAAAAKCPHAAAA|32553|9|2|33|AM|first|morning|breakfast| +32554|AAAAAAAALCPHAAAA|32554|9|2|34|AM|first|morning|breakfast| +32555|AAAAAAAAMCPHAAAA|32555|9|2|35|AM|first|morning|breakfast| +32556|AAAAAAAANCPHAAAA|32556|9|2|36|AM|first|morning|breakfast| +32557|AAAAAAAAOCPHAAAA|32557|9|2|37|AM|first|morning|breakfast| +32558|AAAAAAAAPCPHAAAA|32558|9|2|38|AM|first|morning|breakfast| +32559|AAAAAAAAADPHAAAA|32559|9|2|39|AM|first|morning|breakfast| +32560|AAAAAAAABDPHAAAA|32560|9|2|40|AM|first|morning|breakfast| +32561|AAAAAAAACDPHAAAA|32561|9|2|41|AM|first|morning|breakfast| +32562|AAAAAAAADDPHAAAA|32562|9|2|42|AM|first|morning|breakfast| +32563|AAAAAAAAEDPHAAAA|32563|9|2|43|AM|first|morning|breakfast| +32564|AAAAAAAAFDPHAAAA|32564|9|2|44|AM|first|morning|breakfast| +32565|AAAAAAAAGDPHAAAA|32565|9|2|45|AM|first|morning|breakfast| +32566|AAAAAAAAHDPHAAAA|32566|9|2|46|AM|first|morning|breakfast| +32567|AAAAAAAAIDPHAAAA|32567|9|2|47|AM|first|morning|breakfast| +32568|AAAAAAAAJDPHAAAA|32568|9|2|48|AM|first|morning|breakfast| +32569|AAAAAAAAKDPHAAAA|32569|9|2|49|AM|first|morning|breakfast| +32570|AAAAAAAALDPHAAAA|32570|9|2|50|AM|first|morning|breakfast| +32571|AAAAAAAAMDPHAAAA|32571|9|2|51|AM|first|morning|breakfast| +32572|AAAAAAAANDPHAAAA|32572|9|2|52|AM|first|morning|breakfast| +32573|AAAAAAAAODPHAAAA|32573|9|2|53|AM|first|morning|breakfast| +32574|AAAAAAAAPDPHAAAA|32574|9|2|54|AM|first|morning|breakfast| +32575|AAAAAAAAAEPHAAAA|32575|9|2|55|AM|first|morning|breakfast| +32576|AAAAAAAABEPHAAAA|32576|9|2|56|AM|first|morning|breakfast| +32577|AAAAAAAACEPHAAAA|32577|9|2|57|AM|first|morning|breakfast| +32578|AAAAAAAADEPHAAAA|32578|9|2|58|AM|first|morning|breakfast| +32579|AAAAAAAAEEPHAAAA|32579|9|2|59|AM|first|morning|breakfast| +32580|AAAAAAAAFEPHAAAA|32580|9|3|0|AM|first|morning|breakfast| +32581|AAAAAAAAGEPHAAAA|32581|9|3|1|AM|first|morning|breakfast| +32582|AAAAAAAAHEPHAAAA|32582|9|3|2|AM|first|morning|breakfast| +32583|AAAAAAAAIEPHAAAA|32583|9|3|3|AM|first|morning|breakfast| +32584|AAAAAAAAJEPHAAAA|32584|9|3|4|AM|first|morning|breakfast| +32585|AAAAAAAAKEPHAAAA|32585|9|3|5|AM|first|morning|breakfast| +32586|AAAAAAAALEPHAAAA|32586|9|3|6|AM|first|morning|breakfast| +32587|AAAAAAAAMEPHAAAA|32587|9|3|7|AM|first|morning|breakfast| +32588|AAAAAAAANEPHAAAA|32588|9|3|8|AM|first|morning|breakfast| +32589|AAAAAAAAOEPHAAAA|32589|9|3|9|AM|first|morning|breakfast| +32590|AAAAAAAAPEPHAAAA|32590|9|3|10|AM|first|morning|breakfast| +32591|AAAAAAAAAFPHAAAA|32591|9|3|11|AM|first|morning|breakfast| +32592|AAAAAAAABFPHAAAA|32592|9|3|12|AM|first|morning|breakfast| +32593|AAAAAAAACFPHAAAA|32593|9|3|13|AM|first|morning|breakfast| +32594|AAAAAAAADFPHAAAA|32594|9|3|14|AM|first|morning|breakfast| +32595|AAAAAAAAEFPHAAAA|32595|9|3|15|AM|first|morning|breakfast| +32596|AAAAAAAAFFPHAAAA|32596|9|3|16|AM|first|morning|breakfast| +32597|AAAAAAAAGFPHAAAA|32597|9|3|17|AM|first|morning|breakfast| +32598|AAAAAAAAHFPHAAAA|32598|9|3|18|AM|first|morning|breakfast| +32599|AAAAAAAAIFPHAAAA|32599|9|3|19|AM|first|morning|breakfast| +32600|AAAAAAAAJFPHAAAA|32600|9|3|20|AM|first|morning|breakfast| +32601|AAAAAAAAKFPHAAAA|32601|9|3|21|AM|first|morning|breakfast| +32602|AAAAAAAALFPHAAAA|32602|9|3|22|AM|first|morning|breakfast| +32603|AAAAAAAAMFPHAAAA|32603|9|3|23|AM|first|morning|breakfast| +32604|AAAAAAAANFPHAAAA|32604|9|3|24|AM|first|morning|breakfast| +32605|AAAAAAAAOFPHAAAA|32605|9|3|25|AM|first|morning|breakfast| +32606|AAAAAAAAPFPHAAAA|32606|9|3|26|AM|first|morning|breakfast| +32607|AAAAAAAAAGPHAAAA|32607|9|3|27|AM|first|morning|breakfast| +32608|AAAAAAAABGPHAAAA|32608|9|3|28|AM|first|morning|breakfast| +32609|AAAAAAAACGPHAAAA|32609|9|3|29|AM|first|morning|breakfast| +32610|AAAAAAAADGPHAAAA|32610|9|3|30|AM|first|morning|breakfast| +32611|AAAAAAAAEGPHAAAA|32611|9|3|31|AM|first|morning|breakfast| +32612|AAAAAAAAFGPHAAAA|32612|9|3|32|AM|first|morning|breakfast| +32613|AAAAAAAAGGPHAAAA|32613|9|3|33|AM|first|morning|breakfast| +32614|AAAAAAAAHGPHAAAA|32614|9|3|34|AM|first|morning|breakfast| +32615|AAAAAAAAIGPHAAAA|32615|9|3|35|AM|first|morning|breakfast| +32616|AAAAAAAAJGPHAAAA|32616|9|3|36|AM|first|morning|breakfast| +32617|AAAAAAAAKGPHAAAA|32617|9|3|37|AM|first|morning|breakfast| +32618|AAAAAAAALGPHAAAA|32618|9|3|38|AM|first|morning|breakfast| +32619|AAAAAAAAMGPHAAAA|32619|9|3|39|AM|first|morning|breakfast| +32620|AAAAAAAANGPHAAAA|32620|9|3|40|AM|first|morning|breakfast| +32621|AAAAAAAAOGPHAAAA|32621|9|3|41|AM|first|morning|breakfast| +32622|AAAAAAAAPGPHAAAA|32622|9|3|42|AM|first|morning|breakfast| +32623|AAAAAAAAAHPHAAAA|32623|9|3|43|AM|first|morning|breakfast| +32624|AAAAAAAABHPHAAAA|32624|9|3|44|AM|first|morning|breakfast| +32625|AAAAAAAACHPHAAAA|32625|9|3|45|AM|first|morning|breakfast| +32626|AAAAAAAADHPHAAAA|32626|9|3|46|AM|first|morning|breakfast| +32627|AAAAAAAAEHPHAAAA|32627|9|3|47|AM|first|morning|breakfast| +32628|AAAAAAAAFHPHAAAA|32628|9|3|48|AM|first|morning|breakfast| +32629|AAAAAAAAGHPHAAAA|32629|9|3|49|AM|first|morning|breakfast| +32630|AAAAAAAAHHPHAAAA|32630|9|3|50|AM|first|morning|breakfast| +32631|AAAAAAAAIHPHAAAA|32631|9|3|51|AM|first|morning|breakfast| +32632|AAAAAAAAJHPHAAAA|32632|9|3|52|AM|first|morning|breakfast| +32633|AAAAAAAAKHPHAAAA|32633|9|3|53|AM|first|morning|breakfast| +32634|AAAAAAAALHPHAAAA|32634|9|3|54|AM|first|morning|breakfast| +32635|AAAAAAAAMHPHAAAA|32635|9|3|55|AM|first|morning|breakfast| +32636|AAAAAAAANHPHAAAA|32636|9|3|56|AM|first|morning|breakfast| +32637|AAAAAAAAOHPHAAAA|32637|9|3|57|AM|first|morning|breakfast| +32638|AAAAAAAAPHPHAAAA|32638|9|3|58|AM|first|morning|breakfast| +32639|AAAAAAAAAIPHAAAA|32639|9|3|59|AM|first|morning|breakfast| +32640|AAAAAAAABIPHAAAA|32640|9|4|0|AM|first|morning|breakfast| +32641|AAAAAAAACIPHAAAA|32641|9|4|1|AM|first|morning|breakfast| +32642|AAAAAAAADIPHAAAA|32642|9|4|2|AM|first|morning|breakfast| +32643|AAAAAAAAEIPHAAAA|32643|9|4|3|AM|first|morning|breakfast| +32644|AAAAAAAAFIPHAAAA|32644|9|4|4|AM|first|morning|breakfast| +32645|AAAAAAAAGIPHAAAA|32645|9|4|5|AM|first|morning|breakfast| +32646|AAAAAAAAHIPHAAAA|32646|9|4|6|AM|first|morning|breakfast| +32647|AAAAAAAAIIPHAAAA|32647|9|4|7|AM|first|morning|breakfast| +32648|AAAAAAAAJIPHAAAA|32648|9|4|8|AM|first|morning|breakfast| +32649|AAAAAAAAKIPHAAAA|32649|9|4|9|AM|first|morning|breakfast| +32650|AAAAAAAALIPHAAAA|32650|9|4|10|AM|first|morning|breakfast| +32651|AAAAAAAAMIPHAAAA|32651|9|4|11|AM|first|morning|breakfast| +32652|AAAAAAAANIPHAAAA|32652|9|4|12|AM|first|morning|breakfast| +32653|AAAAAAAAOIPHAAAA|32653|9|4|13|AM|first|morning|breakfast| +32654|AAAAAAAAPIPHAAAA|32654|9|4|14|AM|first|morning|breakfast| +32655|AAAAAAAAAJPHAAAA|32655|9|4|15|AM|first|morning|breakfast| +32656|AAAAAAAABJPHAAAA|32656|9|4|16|AM|first|morning|breakfast| +32657|AAAAAAAACJPHAAAA|32657|9|4|17|AM|first|morning|breakfast| +32658|AAAAAAAADJPHAAAA|32658|9|4|18|AM|first|morning|breakfast| +32659|AAAAAAAAEJPHAAAA|32659|9|4|19|AM|first|morning|breakfast| +32660|AAAAAAAAFJPHAAAA|32660|9|4|20|AM|first|morning|breakfast| +32661|AAAAAAAAGJPHAAAA|32661|9|4|21|AM|first|morning|breakfast| +32662|AAAAAAAAHJPHAAAA|32662|9|4|22|AM|first|morning|breakfast| +32663|AAAAAAAAIJPHAAAA|32663|9|4|23|AM|first|morning|breakfast| +32664|AAAAAAAAJJPHAAAA|32664|9|4|24|AM|first|morning|breakfast| +32665|AAAAAAAAKJPHAAAA|32665|9|4|25|AM|first|morning|breakfast| +32666|AAAAAAAALJPHAAAA|32666|9|4|26|AM|first|morning|breakfast| +32667|AAAAAAAAMJPHAAAA|32667|9|4|27|AM|first|morning|breakfast| +32668|AAAAAAAANJPHAAAA|32668|9|4|28|AM|first|morning|breakfast| +32669|AAAAAAAAOJPHAAAA|32669|9|4|29|AM|first|morning|breakfast| +32670|AAAAAAAAPJPHAAAA|32670|9|4|30|AM|first|morning|breakfast| +32671|AAAAAAAAAKPHAAAA|32671|9|4|31|AM|first|morning|breakfast| +32672|AAAAAAAABKPHAAAA|32672|9|4|32|AM|first|morning|breakfast| +32673|AAAAAAAACKPHAAAA|32673|9|4|33|AM|first|morning|breakfast| +32674|AAAAAAAADKPHAAAA|32674|9|4|34|AM|first|morning|breakfast| +32675|AAAAAAAAEKPHAAAA|32675|9|4|35|AM|first|morning|breakfast| +32676|AAAAAAAAFKPHAAAA|32676|9|4|36|AM|first|morning|breakfast| +32677|AAAAAAAAGKPHAAAA|32677|9|4|37|AM|first|morning|breakfast| +32678|AAAAAAAAHKPHAAAA|32678|9|4|38|AM|first|morning|breakfast| +32679|AAAAAAAAIKPHAAAA|32679|9|4|39|AM|first|morning|breakfast| +32680|AAAAAAAAJKPHAAAA|32680|9|4|40|AM|first|morning|breakfast| +32681|AAAAAAAAKKPHAAAA|32681|9|4|41|AM|first|morning|breakfast| +32682|AAAAAAAALKPHAAAA|32682|9|4|42|AM|first|morning|breakfast| +32683|AAAAAAAAMKPHAAAA|32683|9|4|43|AM|first|morning|breakfast| +32684|AAAAAAAANKPHAAAA|32684|9|4|44|AM|first|morning|breakfast| +32685|AAAAAAAAOKPHAAAA|32685|9|4|45|AM|first|morning|breakfast| +32686|AAAAAAAAPKPHAAAA|32686|9|4|46|AM|first|morning|breakfast| +32687|AAAAAAAAALPHAAAA|32687|9|4|47|AM|first|morning|breakfast| +32688|AAAAAAAABLPHAAAA|32688|9|4|48|AM|first|morning|breakfast| +32689|AAAAAAAACLPHAAAA|32689|9|4|49|AM|first|morning|breakfast| +32690|AAAAAAAADLPHAAAA|32690|9|4|50|AM|first|morning|breakfast| +32691|AAAAAAAAELPHAAAA|32691|9|4|51|AM|first|morning|breakfast| +32692|AAAAAAAAFLPHAAAA|32692|9|4|52|AM|first|morning|breakfast| +32693|AAAAAAAAGLPHAAAA|32693|9|4|53|AM|first|morning|breakfast| +32694|AAAAAAAAHLPHAAAA|32694|9|4|54|AM|first|morning|breakfast| +32695|AAAAAAAAILPHAAAA|32695|9|4|55|AM|first|morning|breakfast| +32696|AAAAAAAAJLPHAAAA|32696|9|4|56|AM|first|morning|breakfast| +32697|AAAAAAAAKLPHAAAA|32697|9|4|57|AM|first|morning|breakfast| +32698|AAAAAAAALLPHAAAA|32698|9|4|58|AM|first|morning|breakfast| +32699|AAAAAAAAMLPHAAAA|32699|9|4|59|AM|first|morning|breakfast| +32700|AAAAAAAANLPHAAAA|32700|9|5|0|AM|first|morning|breakfast| +32701|AAAAAAAAOLPHAAAA|32701|9|5|1|AM|first|morning|breakfast| +32702|AAAAAAAAPLPHAAAA|32702|9|5|2|AM|first|morning|breakfast| +32703|AAAAAAAAAMPHAAAA|32703|9|5|3|AM|first|morning|breakfast| +32704|AAAAAAAABMPHAAAA|32704|9|5|4|AM|first|morning|breakfast| +32705|AAAAAAAACMPHAAAA|32705|9|5|5|AM|first|morning|breakfast| +32706|AAAAAAAADMPHAAAA|32706|9|5|6|AM|first|morning|breakfast| +32707|AAAAAAAAEMPHAAAA|32707|9|5|7|AM|first|morning|breakfast| +32708|AAAAAAAAFMPHAAAA|32708|9|5|8|AM|first|morning|breakfast| +32709|AAAAAAAAGMPHAAAA|32709|9|5|9|AM|first|morning|breakfast| +32710|AAAAAAAAHMPHAAAA|32710|9|5|10|AM|first|morning|breakfast| +32711|AAAAAAAAIMPHAAAA|32711|9|5|11|AM|first|morning|breakfast| +32712|AAAAAAAAJMPHAAAA|32712|9|5|12|AM|first|morning|breakfast| +32713|AAAAAAAAKMPHAAAA|32713|9|5|13|AM|first|morning|breakfast| +32714|AAAAAAAALMPHAAAA|32714|9|5|14|AM|first|morning|breakfast| +32715|AAAAAAAAMMPHAAAA|32715|9|5|15|AM|first|morning|breakfast| +32716|AAAAAAAANMPHAAAA|32716|9|5|16|AM|first|morning|breakfast| +32717|AAAAAAAAOMPHAAAA|32717|9|5|17|AM|first|morning|breakfast| +32718|AAAAAAAAPMPHAAAA|32718|9|5|18|AM|first|morning|breakfast| +32719|AAAAAAAAANPHAAAA|32719|9|5|19|AM|first|morning|breakfast| +32720|AAAAAAAABNPHAAAA|32720|9|5|20|AM|first|morning|breakfast| +32721|AAAAAAAACNPHAAAA|32721|9|5|21|AM|first|morning|breakfast| +32722|AAAAAAAADNPHAAAA|32722|9|5|22|AM|first|morning|breakfast| +32723|AAAAAAAAENPHAAAA|32723|9|5|23|AM|first|morning|breakfast| +32724|AAAAAAAAFNPHAAAA|32724|9|5|24|AM|first|morning|breakfast| +32725|AAAAAAAAGNPHAAAA|32725|9|5|25|AM|first|morning|breakfast| +32726|AAAAAAAAHNPHAAAA|32726|9|5|26|AM|first|morning|breakfast| +32727|AAAAAAAAINPHAAAA|32727|9|5|27|AM|first|morning|breakfast| +32728|AAAAAAAAJNPHAAAA|32728|9|5|28|AM|first|morning|breakfast| +32729|AAAAAAAAKNPHAAAA|32729|9|5|29|AM|first|morning|breakfast| +32730|AAAAAAAALNPHAAAA|32730|9|5|30|AM|first|morning|breakfast| +32731|AAAAAAAAMNPHAAAA|32731|9|5|31|AM|first|morning|breakfast| +32732|AAAAAAAANNPHAAAA|32732|9|5|32|AM|first|morning|breakfast| +32733|AAAAAAAAONPHAAAA|32733|9|5|33|AM|first|morning|breakfast| +32734|AAAAAAAAPNPHAAAA|32734|9|5|34|AM|first|morning|breakfast| +32735|AAAAAAAAAOPHAAAA|32735|9|5|35|AM|first|morning|breakfast| +32736|AAAAAAAABOPHAAAA|32736|9|5|36|AM|first|morning|breakfast| +32737|AAAAAAAACOPHAAAA|32737|9|5|37|AM|first|morning|breakfast| +32738|AAAAAAAADOPHAAAA|32738|9|5|38|AM|first|morning|breakfast| +32739|AAAAAAAAEOPHAAAA|32739|9|5|39|AM|first|morning|breakfast| +32740|AAAAAAAAFOPHAAAA|32740|9|5|40|AM|first|morning|breakfast| +32741|AAAAAAAAGOPHAAAA|32741|9|5|41|AM|first|morning|breakfast| +32742|AAAAAAAAHOPHAAAA|32742|9|5|42|AM|first|morning|breakfast| +32743|AAAAAAAAIOPHAAAA|32743|9|5|43|AM|first|morning|breakfast| +32744|AAAAAAAAJOPHAAAA|32744|9|5|44|AM|first|morning|breakfast| +32745|AAAAAAAAKOPHAAAA|32745|9|5|45|AM|first|morning|breakfast| +32746|AAAAAAAALOPHAAAA|32746|9|5|46|AM|first|morning|breakfast| +32747|AAAAAAAAMOPHAAAA|32747|9|5|47|AM|first|morning|breakfast| +32748|AAAAAAAANOPHAAAA|32748|9|5|48|AM|first|morning|breakfast| +32749|AAAAAAAAOOPHAAAA|32749|9|5|49|AM|first|morning|breakfast| +32750|AAAAAAAAPOPHAAAA|32750|9|5|50|AM|first|morning|breakfast| +32751|AAAAAAAAAPPHAAAA|32751|9|5|51|AM|first|morning|breakfast| +32752|AAAAAAAABPPHAAAA|32752|9|5|52|AM|first|morning|breakfast| +32753|AAAAAAAACPPHAAAA|32753|9|5|53|AM|first|morning|breakfast| +32754|AAAAAAAADPPHAAAA|32754|9|5|54|AM|first|morning|breakfast| +32755|AAAAAAAAEPPHAAAA|32755|9|5|55|AM|first|morning|breakfast| +32756|AAAAAAAAFPPHAAAA|32756|9|5|56|AM|first|morning|breakfast| +32757|AAAAAAAAGPPHAAAA|32757|9|5|57|AM|first|morning|breakfast| +32758|AAAAAAAAHPPHAAAA|32758|9|5|58|AM|first|morning|breakfast| +32759|AAAAAAAAIPPHAAAA|32759|9|5|59|AM|first|morning|breakfast| +32760|AAAAAAAAJPPHAAAA|32760|9|6|0|AM|first|morning|breakfast| +32761|AAAAAAAAKPPHAAAA|32761|9|6|1|AM|first|morning|breakfast| +32762|AAAAAAAALPPHAAAA|32762|9|6|2|AM|first|morning|breakfast| +32763|AAAAAAAAMPPHAAAA|32763|9|6|3|AM|first|morning|breakfast| +32764|AAAAAAAANPPHAAAA|32764|9|6|4|AM|first|morning|breakfast| +32765|AAAAAAAAOPPHAAAA|32765|9|6|5|AM|first|morning|breakfast| +32766|AAAAAAAAPPPHAAAA|32766|9|6|6|AM|first|morning|breakfast| +32767|AAAAAAAAAAAIAAAA|32767|9|6|7|AM|first|morning|breakfast| +32768|AAAAAAAABAAIAAAA|32768|9|6|8|AM|first|morning|breakfast| +32769|AAAAAAAACAAIAAAA|32769|9|6|9|AM|first|morning|breakfast| +32770|AAAAAAAADAAIAAAA|32770|9|6|10|AM|first|morning|breakfast| +32771|AAAAAAAAEAAIAAAA|32771|9|6|11|AM|first|morning|breakfast| +32772|AAAAAAAAFAAIAAAA|32772|9|6|12|AM|first|morning|breakfast| +32773|AAAAAAAAGAAIAAAA|32773|9|6|13|AM|first|morning|breakfast| +32774|AAAAAAAAHAAIAAAA|32774|9|6|14|AM|first|morning|breakfast| +32775|AAAAAAAAIAAIAAAA|32775|9|6|15|AM|first|morning|breakfast| +32776|AAAAAAAAJAAIAAAA|32776|9|6|16|AM|first|morning|breakfast| +32777|AAAAAAAAKAAIAAAA|32777|9|6|17|AM|first|morning|breakfast| +32778|AAAAAAAALAAIAAAA|32778|9|6|18|AM|first|morning|breakfast| +32779|AAAAAAAAMAAIAAAA|32779|9|6|19|AM|first|morning|breakfast| +32780|AAAAAAAANAAIAAAA|32780|9|6|20|AM|first|morning|breakfast| +32781|AAAAAAAAOAAIAAAA|32781|9|6|21|AM|first|morning|breakfast| +32782|AAAAAAAAPAAIAAAA|32782|9|6|22|AM|first|morning|breakfast| +32783|AAAAAAAAABAIAAAA|32783|9|6|23|AM|first|morning|breakfast| +32784|AAAAAAAABBAIAAAA|32784|9|6|24|AM|first|morning|breakfast| +32785|AAAAAAAACBAIAAAA|32785|9|6|25|AM|first|morning|breakfast| +32786|AAAAAAAADBAIAAAA|32786|9|6|26|AM|first|morning|breakfast| +32787|AAAAAAAAEBAIAAAA|32787|9|6|27|AM|first|morning|breakfast| +32788|AAAAAAAAFBAIAAAA|32788|9|6|28|AM|first|morning|breakfast| +32789|AAAAAAAAGBAIAAAA|32789|9|6|29|AM|first|morning|breakfast| +32790|AAAAAAAAHBAIAAAA|32790|9|6|30|AM|first|morning|breakfast| +32791|AAAAAAAAIBAIAAAA|32791|9|6|31|AM|first|morning|breakfast| +32792|AAAAAAAAJBAIAAAA|32792|9|6|32|AM|first|morning|breakfast| +32793|AAAAAAAAKBAIAAAA|32793|9|6|33|AM|first|morning|breakfast| +32794|AAAAAAAALBAIAAAA|32794|9|6|34|AM|first|morning|breakfast| +32795|AAAAAAAAMBAIAAAA|32795|9|6|35|AM|first|morning|breakfast| +32796|AAAAAAAANBAIAAAA|32796|9|6|36|AM|first|morning|breakfast| +32797|AAAAAAAAOBAIAAAA|32797|9|6|37|AM|first|morning|breakfast| +32798|AAAAAAAAPBAIAAAA|32798|9|6|38|AM|first|morning|breakfast| +32799|AAAAAAAAACAIAAAA|32799|9|6|39|AM|first|morning|breakfast| +32800|AAAAAAAABCAIAAAA|32800|9|6|40|AM|first|morning|breakfast| +32801|AAAAAAAACCAIAAAA|32801|9|6|41|AM|first|morning|breakfast| +32802|AAAAAAAADCAIAAAA|32802|9|6|42|AM|first|morning|breakfast| +32803|AAAAAAAAECAIAAAA|32803|9|6|43|AM|first|morning|breakfast| +32804|AAAAAAAAFCAIAAAA|32804|9|6|44|AM|first|morning|breakfast| +32805|AAAAAAAAGCAIAAAA|32805|9|6|45|AM|first|morning|breakfast| +32806|AAAAAAAAHCAIAAAA|32806|9|6|46|AM|first|morning|breakfast| +32807|AAAAAAAAICAIAAAA|32807|9|6|47|AM|first|morning|breakfast| +32808|AAAAAAAAJCAIAAAA|32808|9|6|48|AM|first|morning|breakfast| +32809|AAAAAAAAKCAIAAAA|32809|9|6|49|AM|first|morning|breakfast| +32810|AAAAAAAALCAIAAAA|32810|9|6|50|AM|first|morning|breakfast| +32811|AAAAAAAAMCAIAAAA|32811|9|6|51|AM|first|morning|breakfast| +32812|AAAAAAAANCAIAAAA|32812|9|6|52|AM|first|morning|breakfast| +32813|AAAAAAAAOCAIAAAA|32813|9|6|53|AM|first|morning|breakfast| +32814|AAAAAAAAPCAIAAAA|32814|9|6|54|AM|first|morning|breakfast| +32815|AAAAAAAAADAIAAAA|32815|9|6|55|AM|first|morning|breakfast| +32816|AAAAAAAABDAIAAAA|32816|9|6|56|AM|first|morning|breakfast| +32817|AAAAAAAACDAIAAAA|32817|9|6|57|AM|first|morning|breakfast| +32818|AAAAAAAADDAIAAAA|32818|9|6|58|AM|first|morning|breakfast| +32819|AAAAAAAAEDAIAAAA|32819|9|6|59|AM|first|morning|breakfast| +32820|AAAAAAAAFDAIAAAA|32820|9|7|0|AM|first|morning|breakfast| +32821|AAAAAAAAGDAIAAAA|32821|9|7|1|AM|first|morning|breakfast| +32822|AAAAAAAAHDAIAAAA|32822|9|7|2|AM|first|morning|breakfast| +32823|AAAAAAAAIDAIAAAA|32823|9|7|3|AM|first|morning|breakfast| +32824|AAAAAAAAJDAIAAAA|32824|9|7|4|AM|first|morning|breakfast| +32825|AAAAAAAAKDAIAAAA|32825|9|7|5|AM|first|morning|breakfast| +32826|AAAAAAAALDAIAAAA|32826|9|7|6|AM|first|morning|breakfast| +32827|AAAAAAAAMDAIAAAA|32827|9|7|7|AM|first|morning|breakfast| +32828|AAAAAAAANDAIAAAA|32828|9|7|8|AM|first|morning|breakfast| +32829|AAAAAAAAODAIAAAA|32829|9|7|9|AM|first|morning|breakfast| +32830|AAAAAAAAPDAIAAAA|32830|9|7|10|AM|first|morning|breakfast| +32831|AAAAAAAAAEAIAAAA|32831|9|7|11|AM|first|morning|breakfast| +32832|AAAAAAAABEAIAAAA|32832|9|7|12|AM|first|morning|breakfast| +32833|AAAAAAAACEAIAAAA|32833|9|7|13|AM|first|morning|breakfast| +32834|AAAAAAAADEAIAAAA|32834|9|7|14|AM|first|morning|breakfast| +32835|AAAAAAAAEEAIAAAA|32835|9|7|15|AM|first|morning|breakfast| +32836|AAAAAAAAFEAIAAAA|32836|9|7|16|AM|first|morning|breakfast| +32837|AAAAAAAAGEAIAAAA|32837|9|7|17|AM|first|morning|breakfast| +32838|AAAAAAAAHEAIAAAA|32838|9|7|18|AM|first|morning|breakfast| +32839|AAAAAAAAIEAIAAAA|32839|9|7|19|AM|first|morning|breakfast| +32840|AAAAAAAAJEAIAAAA|32840|9|7|20|AM|first|morning|breakfast| +32841|AAAAAAAAKEAIAAAA|32841|9|7|21|AM|first|morning|breakfast| +32842|AAAAAAAALEAIAAAA|32842|9|7|22|AM|first|morning|breakfast| +32843|AAAAAAAAMEAIAAAA|32843|9|7|23|AM|first|morning|breakfast| +32844|AAAAAAAANEAIAAAA|32844|9|7|24|AM|first|morning|breakfast| +32845|AAAAAAAAOEAIAAAA|32845|9|7|25|AM|first|morning|breakfast| +32846|AAAAAAAAPEAIAAAA|32846|9|7|26|AM|first|morning|breakfast| +32847|AAAAAAAAAFAIAAAA|32847|9|7|27|AM|first|morning|breakfast| +32848|AAAAAAAABFAIAAAA|32848|9|7|28|AM|first|morning|breakfast| +32849|AAAAAAAACFAIAAAA|32849|9|7|29|AM|first|morning|breakfast| +32850|AAAAAAAADFAIAAAA|32850|9|7|30|AM|first|morning|breakfast| +32851|AAAAAAAAEFAIAAAA|32851|9|7|31|AM|first|morning|breakfast| +32852|AAAAAAAAFFAIAAAA|32852|9|7|32|AM|first|morning|breakfast| +32853|AAAAAAAAGFAIAAAA|32853|9|7|33|AM|first|morning|breakfast| +32854|AAAAAAAAHFAIAAAA|32854|9|7|34|AM|first|morning|breakfast| +32855|AAAAAAAAIFAIAAAA|32855|9|7|35|AM|first|morning|breakfast| +32856|AAAAAAAAJFAIAAAA|32856|9|7|36|AM|first|morning|breakfast| +32857|AAAAAAAAKFAIAAAA|32857|9|7|37|AM|first|morning|breakfast| +32858|AAAAAAAALFAIAAAA|32858|9|7|38|AM|first|morning|breakfast| +32859|AAAAAAAAMFAIAAAA|32859|9|7|39|AM|first|morning|breakfast| +32860|AAAAAAAANFAIAAAA|32860|9|7|40|AM|first|morning|breakfast| +32861|AAAAAAAAOFAIAAAA|32861|9|7|41|AM|first|morning|breakfast| +32862|AAAAAAAAPFAIAAAA|32862|9|7|42|AM|first|morning|breakfast| +32863|AAAAAAAAAGAIAAAA|32863|9|7|43|AM|first|morning|breakfast| +32864|AAAAAAAABGAIAAAA|32864|9|7|44|AM|first|morning|breakfast| +32865|AAAAAAAACGAIAAAA|32865|9|7|45|AM|first|morning|breakfast| +32866|AAAAAAAADGAIAAAA|32866|9|7|46|AM|first|morning|breakfast| +32867|AAAAAAAAEGAIAAAA|32867|9|7|47|AM|first|morning|breakfast| +32868|AAAAAAAAFGAIAAAA|32868|9|7|48|AM|first|morning|breakfast| +32869|AAAAAAAAGGAIAAAA|32869|9|7|49|AM|first|morning|breakfast| +32870|AAAAAAAAHGAIAAAA|32870|9|7|50|AM|first|morning|breakfast| +32871|AAAAAAAAIGAIAAAA|32871|9|7|51|AM|first|morning|breakfast| +32872|AAAAAAAAJGAIAAAA|32872|9|7|52|AM|first|morning|breakfast| +32873|AAAAAAAAKGAIAAAA|32873|9|7|53|AM|first|morning|breakfast| +32874|AAAAAAAALGAIAAAA|32874|9|7|54|AM|first|morning|breakfast| +32875|AAAAAAAAMGAIAAAA|32875|9|7|55|AM|first|morning|breakfast| +32876|AAAAAAAANGAIAAAA|32876|9|7|56|AM|first|morning|breakfast| +32877|AAAAAAAAOGAIAAAA|32877|9|7|57|AM|first|morning|breakfast| +32878|AAAAAAAAPGAIAAAA|32878|9|7|58|AM|first|morning|breakfast| +32879|AAAAAAAAAHAIAAAA|32879|9|7|59|AM|first|morning|breakfast| +32880|AAAAAAAABHAIAAAA|32880|9|8|0|AM|first|morning|breakfast| +32881|AAAAAAAACHAIAAAA|32881|9|8|1|AM|first|morning|breakfast| +32882|AAAAAAAADHAIAAAA|32882|9|8|2|AM|first|morning|breakfast| +32883|AAAAAAAAEHAIAAAA|32883|9|8|3|AM|first|morning|breakfast| +32884|AAAAAAAAFHAIAAAA|32884|9|8|4|AM|first|morning|breakfast| +32885|AAAAAAAAGHAIAAAA|32885|9|8|5|AM|first|morning|breakfast| +32886|AAAAAAAAHHAIAAAA|32886|9|8|6|AM|first|morning|breakfast| +32887|AAAAAAAAIHAIAAAA|32887|9|8|7|AM|first|morning|breakfast| +32888|AAAAAAAAJHAIAAAA|32888|9|8|8|AM|first|morning|breakfast| +32889|AAAAAAAAKHAIAAAA|32889|9|8|9|AM|first|morning|breakfast| +32890|AAAAAAAALHAIAAAA|32890|9|8|10|AM|first|morning|breakfast| +32891|AAAAAAAAMHAIAAAA|32891|9|8|11|AM|first|morning|breakfast| +32892|AAAAAAAANHAIAAAA|32892|9|8|12|AM|first|morning|breakfast| +32893|AAAAAAAAOHAIAAAA|32893|9|8|13|AM|first|morning|breakfast| +32894|AAAAAAAAPHAIAAAA|32894|9|8|14|AM|first|morning|breakfast| +32895|AAAAAAAAAIAIAAAA|32895|9|8|15|AM|first|morning|breakfast| +32896|AAAAAAAABIAIAAAA|32896|9|8|16|AM|first|morning|breakfast| +32897|AAAAAAAACIAIAAAA|32897|9|8|17|AM|first|morning|breakfast| +32898|AAAAAAAADIAIAAAA|32898|9|8|18|AM|first|morning|breakfast| +32899|AAAAAAAAEIAIAAAA|32899|9|8|19|AM|first|morning|breakfast| +32900|AAAAAAAAFIAIAAAA|32900|9|8|20|AM|first|morning|breakfast| +32901|AAAAAAAAGIAIAAAA|32901|9|8|21|AM|first|morning|breakfast| +32902|AAAAAAAAHIAIAAAA|32902|9|8|22|AM|first|morning|breakfast| +32903|AAAAAAAAIIAIAAAA|32903|9|8|23|AM|first|morning|breakfast| +32904|AAAAAAAAJIAIAAAA|32904|9|8|24|AM|first|morning|breakfast| +32905|AAAAAAAAKIAIAAAA|32905|9|8|25|AM|first|morning|breakfast| +32906|AAAAAAAALIAIAAAA|32906|9|8|26|AM|first|morning|breakfast| +32907|AAAAAAAAMIAIAAAA|32907|9|8|27|AM|first|morning|breakfast| +32908|AAAAAAAANIAIAAAA|32908|9|8|28|AM|first|morning|breakfast| +32909|AAAAAAAAOIAIAAAA|32909|9|8|29|AM|first|morning|breakfast| +32910|AAAAAAAAPIAIAAAA|32910|9|8|30|AM|first|morning|breakfast| +32911|AAAAAAAAAJAIAAAA|32911|9|8|31|AM|first|morning|breakfast| +32912|AAAAAAAABJAIAAAA|32912|9|8|32|AM|first|morning|breakfast| +32913|AAAAAAAACJAIAAAA|32913|9|8|33|AM|first|morning|breakfast| +32914|AAAAAAAADJAIAAAA|32914|9|8|34|AM|first|morning|breakfast| +32915|AAAAAAAAEJAIAAAA|32915|9|8|35|AM|first|morning|breakfast| +32916|AAAAAAAAFJAIAAAA|32916|9|8|36|AM|first|morning|breakfast| +32917|AAAAAAAAGJAIAAAA|32917|9|8|37|AM|first|morning|breakfast| +32918|AAAAAAAAHJAIAAAA|32918|9|8|38|AM|first|morning|breakfast| +32919|AAAAAAAAIJAIAAAA|32919|9|8|39|AM|first|morning|breakfast| +32920|AAAAAAAAJJAIAAAA|32920|9|8|40|AM|first|morning|breakfast| +32921|AAAAAAAAKJAIAAAA|32921|9|8|41|AM|first|morning|breakfast| +32922|AAAAAAAALJAIAAAA|32922|9|8|42|AM|first|morning|breakfast| +32923|AAAAAAAAMJAIAAAA|32923|9|8|43|AM|first|morning|breakfast| +32924|AAAAAAAANJAIAAAA|32924|9|8|44|AM|first|morning|breakfast| +32925|AAAAAAAAOJAIAAAA|32925|9|8|45|AM|first|morning|breakfast| +32926|AAAAAAAAPJAIAAAA|32926|9|8|46|AM|first|morning|breakfast| +32927|AAAAAAAAAKAIAAAA|32927|9|8|47|AM|first|morning|breakfast| +32928|AAAAAAAABKAIAAAA|32928|9|8|48|AM|first|morning|breakfast| +32929|AAAAAAAACKAIAAAA|32929|9|8|49|AM|first|morning|breakfast| +32930|AAAAAAAADKAIAAAA|32930|9|8|50|AM|first|morning|breakfast| +32931|AAAAAAAAEKAIAAAA|32931|9|8|51|AM|first|morning|breakfast| +32932|AAAAAAAAFKAIAAAA|32932|9|8|52|AM|first|morning|breakfast| +32933|AAAAAAAAGKAIAAAA|32933|9|8|53|AM|first|morning|breakfast| +32934|AAAAAAAAHKAIAAAA|32934|9|8|54|AM|first|morning|breakfast| +32935|AAAAAAAAIKAIAAAA|32935|9|8|55|AM|first|morning|breakfast| +32936|AAAAAAAAJKAIAAAA|32936|9|8|56|AM|first|morning|breakfast| +32937|AAAAAAAAKKAIAAAA|32937|9|8|57|AM|first|morning|breakfast| +32938|AAAAAAAALKAIAAAA|32938|9|8|58|AM|first|morning|breakfast| +32939|AAAAAAAAMKAIAAAA|32939|9|8|59|AM|first|morning|breakfast| +32940|AAAAAAAANKAIAAAA|32940|9|9|0|AM|first|morning|breakfast| +32941|AAAAAAAAOKAIAAAA|32941|9|9|1|AM|first|morning|breakfast| +32942|AAAAAAAAPKAIAAAA|32942|9|9|2|AM|first|morning|breakfast| +32943|AAAAAAAAALAIAAAA|32943|9|9|3|AM|first|morning|breakfast| +32944|AAAAAAAABLAIAAAA|32944|9|9|4|AM|first|morning|breakfast| +32945|AAAAAAAACLAIAAAA|32945|9|9|5|AM|first|morning|breakfast| +32946|AAAAAAAADLAIAAAA|32946|9|9|6|AM|first|morning|breakfast| +32947|AAAAAAAAELAIAAAA|32947|9|9|7|AM|first|morning|breakfast| +32948|AAAAAAAAFLAIAAAA|32948|9|9|8|AM|first|morning|breakfast| +32949|AAAAAAAAGLAIAAAA|32949|9|9|9|AM|first|morning|breakfast| +32950|AAAAAAAAHLAIAAAA|32950|9|9|10|AM|first|morning|breakfast| +32951|AAAAAAAAILAIAAAA|32951|9|9|11|AM|first|morning|breakfast| +32952|AAAAAAAAJLAIAAAA|32952|9|9|12|AM|first|morning|breakfast| +32953|AAAAAAAAKLAIAAAA|32953|9|9|13|AM|first|morning|breakfast| +32954|AAAAAAAALLAIAAAA|32954|9|9|14|AM|first|morning|breakfast| +32955|AAAAAAAAMLAIAAAA|32955|9|9|15|AM|first|morning|breakfast| +32956|AAAAAAAANLAIAAAA|32956|9|9|16|AM|first|morning|breakfast| +32957|AAAAAAAAOLAIAAAA|32957|9|9|17|AM|first|morning|breakfast| +32958|AAAAAAAAPLAIAAAA|32958|9|9|18|AM|first|morning|breakfast| +32959|AAAAAAAAAMAIAAAA|32959|9|9|19|AM|first|morning|breakfast| +32960|AAAAAAAABMAIAAAA|32960|9|9|20|AM|first|morning|breakfast| +32961|AAAAAAAACMAIAAAA|32961|9|9|21|AM|first|morning|breakfast| +32962|AAAAAAAADMAIAAAA|32962|9|9|22|AM|first|morning|breakfast| +32963|AAAAAAAAEMAIAAAA|32963|9|9|23|AM|first|morning|breakfast| +32964|AAAAAAAAFMAIAAAA|32964|9|9|24|AM|first|morning|breakfast| +32965|AAAAAAAAGMAIAAAA|32965|9|9|25|AM|first|morning|breakfast| +32966|AAAAAAAAHMAIAAAA|32966|9|9|26|AM|first|morning|breakfast| +32967|AAAAAAAAIMAIAAAA|32967|9|9|27|AM|first|morning|breakfast| +32968|AAAAAAAAJMAIAAAA|32968|9|9|28|AM|first|morning|breakfast| +32969|AAAAAAAAKMAIAAAA|32969|9|9|29|AM|first|morning|breakfast| +32970|AAAAAAAALMAIAAAA|32970|9|9|30|AM|first|morning|breakfast| +32971|AAAAAAAAMMAIAAAA|32971|9|9|31|AM|first|morning|breakfast| +32972|AAAAAAAANMAIAAAA|32972|9|9|32|AM|first|morning|breakfast| +32973|AAAAAAAAOMAIAAAA|32973|9|9|33|AM|first|morning|breakfast| +32974|AAAAAAAAPMAIAAAA|32974|9|9|34|AM|first|morning|breakfast| +32975|AAAAAAAAANAIAAAA|32975|9|9|35|AM|first|morning|breakfast| +32976|AAAAAAAABNAIAAAA|32976|9|9|36|AM|first|morning|breakfast| +32977|AAAAAAAACNAIAAAA|32977|9|9|37|AM|first|morning|breakfast| +32978|AAAAAAAADNAIAAAA|32978|9|9|38|AM|first|morning|breakfast| +32979|AAAAAAAAENAIAAAA|32979|9|9|39|AM|first|morning|breakfast| +32980|AAAAAAAAFNAIAAAA|32980|9|9|40|AM|first|morning|breakfast| +32981|AAAAAAAAGNAIAAAA|32981|9|9|41|AM|first|morning|breakfast| +32982|AAAAAAAAHNAIAAAA|32982|9|9|42|AM|first|morning|breakfast| +32983|AAAAAAAAINAIAAAA|32983|9|9|43|AM|first|morning|breakfast| +32984|AAAAAAAAJNAIAAAA|32984|9|9|44|AM|first|morning|breakfast| +32985|AAAAAAAAKNAIAAAA|32985|9|9|45|AM|first|morning|breakfast| +32986|AAAAAAAALNAIAAAA|32986|9|9|46|AM|first|morning|breakfast| +32987|AAAAAAAAMNAIAAAA|32987|9|9|47|AM|first|morning|breakfast| +32988|AAAAAAAANNAIAAAA|32988|9|9|48|AM|first|morning|breakfast| +32989|AAAAAAAAONAIAAAA|32989|9|9|49|AM|first|morning|breakfast| +32990|AAAAAAAAPNAIAAAA|32990|9|9|50|AM|first|morning|breakfast| +32991|AAAAAAAAAOAIAAAA|32991|9|9|51|AM|first|morning|breakfast| +32992|AAAAAAAABOAIAAAA|32992|9|9|52|AM|first|morning|breakfast| +32993|AAAAAAAACOAIAAAA|32993|9|9|53|AM|first|morning|breakfast| +32994|AAAAAAAADOAIAAAA|32994|9|9|54|AM|first|morning|breakfast| +32995|AAAAAAAAEOAIAAAA|32995|9|9|55|AM|first|morning|breakfast| +32996|AAAAAAAAFOAIAAAA|32996|9|9|56|AM|first|morning|breakfast| +32997|AAAAAAAAGOAIAAAA|32997|9|9|57|AM|first|morning|breakfast| +32998|AAAAAAAAHOAIAAAA|32998|9|9|58|AM|first|morning|breakfast| +32999|AAAAAAAAIOAIAAAA|32999|9|9|59|AM|first|morning|breakfast| +33000|AAAAAAAAJOAIAAAA|33000|9|10|0|AM|first|morning|breakfast| +33001|AAAAAAAAKOAIAAAA|33001|9|10|1|AM|first|morning|breakfast| +33002|AAAAAAAALOAIAAAA|33002|9|10|2|AM|first|morning|breakfast| +33003|AAAAAAAAMOAIAAAA|33003|9|10|3|AM|first|morning|breakfast| +33004|AAAAAAAANOAIAAAA|33004|9|10|4|AM|first|morning|breakfast| +33005|AAAAAAAAOOAIAAAA|33005|9|10|5|AM|first|morning|breakfast| +33006|AAAAAAAAPOAIAAAA|33006|9|10|6|AM|first|morning|breakfast| +33007|AAAAAAAAAPAIAAAA|33007|9|10|7|AM|first|morning|breakfast| +33008|AAAAAAAABPAIAAAA|33008|9|10|8|AM|first|morning|breakfast| +33009|AAAAAAAACPAIAAAA|33009|9|10|9|AM|first|morning|breakfast| +33010|AAAAAAAADPAIAAAA|33010|9|10|10|AM|first|morning|breakfast| +33011|AAAAAAAAEPAIAAAA|33011|9|10|11|AM|first|morning|breakfast| +33012|AAAAAAAAFPAIAAAA|33012|9|10|12|AM|first|morning|breakfast| +33013|AAAAAAAAGPAIAAAA|33013|9|10|13|AM|first|morning|breakfast| +33014|AAAAAAAAHPAIAAAA|33014|9|10|14|AM|first|morning|breakfast| +33015|AAAAAAAAIPAIAAAA|33015|9|10|15|AM|first|morning|breakfast| +33016|AAAAAAAAJPAIAAAA|33016|9|10|16|AM|first|morning|breakfast| +33017|AAAAAAAAKPAIAAAA|33017|9|10|17|AM|first|morning|breakfast| +33018|AAAAAAAALPAIAAAA|33018|9|10|18|AM|first|morning|breakfast| +33019|AAAAAAAAMPAIAAAA|33019|9|10|19|AM|first|morning|breakfast| +33020|AAAAAAAANPAIAAAA|33020|9|10|20|AM|first|morning|breakfast| +33021|AAAAAAAAOPAIAAAA|33021|9|10|21|AM|first|morning|breakfast| +33022|AAAAAAAAPPAIAAAA|33022|9|10|22|AM|first|morning|breakfast| +33023|AAAAAAAAAABIAAAA|33023|9|10|23|AM|first|morning|breakfast| +33024|AAAAAAAABABIAAAA|33024|9|10|24|AM|first|morning|breakfast| +33025|AAAAAAAACABIAAAA|33025|9|10|25|AM|first|morning|breakfast| +33026|AAAAAAAADABIAAAA|33026|9|10|26|AM|first|morning|breakfast| +33027|AAAAAAAAEABIAAAA|33027|9|10|27|AM|first|morning|breakfast| +33028|AAAAAAAAFABIAAAA|33028|9|10|28|AM|first|morning|breakfast| +33029|AAAAAAAAGABIAAAA|33029|9|10|29|AM|first|morning|breakfast| +33030|AAAAAAAAHABIAAAA|33030|9|10|30|AM|first|morning|breakfast| +33031|AAAAAAAAIABIAAAA|33031|9|10|31|AM|first|morning|breakfast| +33032|AAAAAAAAJABIAAAA|33032|9|10|32|AM|first|morning|breakfast| +33033|AAAAAAAAKABIAAAA|33033|9|10|33|AM|first|morning|breakfast| +33034|AAAAAAAALABIAAAA|33034|9|10|34|AM|first|morning|breakfast| +33035|AAAAAAAAMABIAAAA|33035|9|10|35|AM|first|morning|breakfast| +33036|AAAAAAAANABIAAAA|33036|9|10|36|AM|first|morning|breakfast| +33037|AAAAAAAAOABIAAAA|33037|9|10|37|AM|first|morning|breakfast| +33038|AAAAAAAAPABIAAAA|33038|9|10|38|AM|first|morning|breakfast| +33039|AAAAAAAAABBIAAAA|33039|9|10|39|AM|first|morning|breakfast| +33040|AAAAAAAABBBIAAAA|33040|9|10|40|AM|first|morning|breakfast| +33041|AAAAAAAACBBIAAAA|33041|9|10|41|AM|first|morning|breakfast| +33042|AAAAAAAADBBIAAAA|33042|9|10|42|AM|first|morning|breakfast| +33043|AAAAAAAAEBBIAAAA|33043|9|10|43|AM|first|morning|breakfast| +33044|AAAAAAAAFBBIAAAA|33044|9|10|44|AM|first|morning|breakfast| +33045|AAAAAAAAGBBIAAAA|33045|9|10|45|AM|first|morning|breakfast| +33046|AAAAAAAAHBBIAAAA|33046|9|10|46|AM|first|morning|breakfast| +33047|AAAAAAAAIBBIAAAA|33047|9|10|47|AM|first|morning|breakfast| +33048|AAAAAAAAJBBIAAAA|33048|9|10|48|AM|first|morning|breakfast| +33049|AAAAAAAAKBBIAAAA|33049|9|10|49|AM|first|morning|breakfast| +33050|AAAAAAAALBBIAAAA|33050|9|10|50|AM|first|morning|breakfast| +33051|AAAAAAAAMBBIAAAA|33051|9|10|51|AM|first|morning|breakfast| +33052|AAAAAAAANBBIAAAA|33052|9|10|52|AM|first|morning|breakfast| +33053|AAAAAAAAOBBIAAAA|33053|9|10|53|AM|first|morning|breakfast| +33054|AAAAAAAAPBBIAAAA|33054|9|10|54|AM|first|morning|breakfast| +33055|AAAAAAAAACBIAAAA|33055|9|10|55|AM|first|morning|breakfast| +33056|AAAAAAAABCBIAAAA|33056|9|10|56|AM|first|morning|breakfast| +33057|AAAAAAAACCBIAAAA|33057|9|10|57|AM|first|morning|breakfast| +33058|AAAAAAAADCBIAAAA|33058|9|10|58|AM|first|morning|breakfast| +33059|AAAAAAAAECBIAAAA|33059|9|10|59|AM|first|morning|breakfast| +33060|AAAAAAAAFCBIAAAA|33060|9|11|0|AM|first|morning|breakfast| +33061|AAAAAAAAGCBIAAAA|33061|9|11|1|AM|first|morning|breakfast| +33062|AAAAAAAAHCBIAAAA|33062|9|11|2|AM|first|morning|breakfast| +33063|AAAAAAAAICBIAAAA|33063|9|11|3|AM|first|morning|breakfast| +33064|AAAAAAAAJCBIAAAA|33064|9|11|4|AM|first|morning|breakfast| +33065|AAAAAAAAKCBIAAAA|33065|9|11|5|AM|first|morning|breakfast| +33066|AAAAAAAALCBIAAAA|33066|9|11|6|AM|first|morning|breakfast| +33067|AAAAAAAAMCBIAAAA|33067|9|11|7|AM|first|morning|breakfast| +33068|AAAAAAAANCBIAAAA|33068|9|11|8|AM|first|morning|breakfast| +33069|AAAAAAAAOCBIAAAA|33069|9|11|9|AM|first|morning|breakfast| +33070|AAAAAAAAPCBIAAAA|33070|9|11|10|AM|first|morning|breakfast| +33071|AAAAAAAAADBIAAAA|33071|9|11|11|AM|first|morning|breakfast| +33072|AAAAAAAABDBIAAAA|33072|9|11|12|AM|first|morning|breakfast| +33073|AAAAAAAACDBIAAAA|33073|9|11|13|AM|first|morning|breakfast| +33074|AAAAAAAADDBIAAAA|33074|9|11|14|AM|first|morning|breakfast| +33075|AAAAAAAAEDBIAAAA|33075|9|11|15|AM|first|morning|breakfast| +33076|AAAAAAAAFDBIAAAA|33076|9|11|16|AM|first|morning|breakfast| +33077|AAAAAAAAGDBIAAAA|33077|9|11|17|AM|first|morning|breakfast| +33078|AAAAAAAAHDBIAAAA|33078|9|11|18|AM|first|morning|breakfast| +33079|AAAAAAAAIDBIAAAA|33079|9|11|19|AM|first|morning|breakfast| +33080|AAAAAAAAJDBIAAAA|33080|9|11|20|AM|first|morning|breakfast| +33081|AAAAAAAAKDBIAAAA|33081|9|11|21|AM|first|morning|breakfast| +33082|AAAAAAAALDBIAAAA|33082|9|11|22|AM|first|morning|breakfast| +33083|AAAAAAAAMDBIAAAA|33083|9|11|23|AM|first|morning|breakfast| +33084|AAAAAAAANDBIAAAA|33084|9|11|24|AM|first|morning|breakfast| +33085|AAAAAAAAODBIAAAA|33085|9|11|25|AM|first|morning|breakfast| +33086|AAAAAAAAPDBIAAAA|33086|9|11|26|AM|first|morning|breakfast| +33087|AAAAAAAAAEBIAAAA|33087|9|11|27|AM|first|morning|breakfast| +33088|AAAAAAAABEBIAAAA|33088|9|11|28|AM|first|morning|breakfast| +33089|AAAAAAAACEBIAAAA|33089|9|11|29|AM|first|morning|breakfast| +33090|AAAAAAAADEBIAAAA|33090|9|11|30|AM|first|morning|breakfast| +33091|AAAAAAAAEEBIAAAA|33091|9|11|31|AM|first|morning|breakfast| +33092|AAAAAAAAFEBIAAAA|33092|9|11|32|AM|first|morning|breakfast| +33093|AAAAAAAAGEBIAAAA|33093|9|11|33|AM|first|morning|breakfast| +33094|AAAAAAAAHEBIAAAA|33094|9|11|34|AM|first|morning|breakfast| +33095|AAAAAAAAIEBIAAAA|33095|9|11|35|AM|first|morning|breakfast| +33096|AAAAAAAAJEBIAAAA|33096|9|11|36|AM|first|morning|breakfast| +33097|AAAAAAAAKEBIAAAA|33097|9|11|37|AM|first|morning|breakfast| +33098|AAAAAAAALEBIAAAA|33098|9|11|38|AM|first|morning|breakfast| +33099|AAAAAAAAMEBIAAAA|33099|9|11|39|AM|first|morning|breakfast| +33100|AAAAAAAANEBIAAAA|33100|9|11|40|AM|first|morning|breakfast| +33101|AAAAAAAAOEBIAAAA|33101|9|11|41|AM|first|morning|breakfast| +33102|AAAAAAAAPEBIAAAA|33102|9|11|42|AM|first|morning|breakfast| +33103|AAAAAAAAAFBIAAAA|33103|9|11|43|AM|first|morning|breakfast| +33104|AAAAAAAABFBIAAAA|33104|9|11|44|AM|first|morning|breakfast| +33105|AAAAAAAACFBIAAAA|33105|9|11|45|AM|first|morning|breakfast| +33106|AAAAAAAADFBIAAAA|33106|9|11|46|AM|first|morning|breakfast| +33107|AAAAAAAAEFBIAAAA|33107|9|11|47|AM|first|morning|breakfast| +33108|AAAAAAAAFFBIAAAA|33108|9|11|48|AM|first|morning|breakfast| +33109|AAAAAAAAGFBIAAAA|33109|9|11|49|AM|first|morning|breakfast| +33110|AAAAAAAAHFBIAAAA|33110|9|11|50|AM|first|morning|breakfast| +33111|AAAAAAAAIFBIAAAA|33111|9|11|51|AM|first|morning|breakfast| +33112|AAAAAAAAJFBIAAAA|33112|9|11|52|AM|first|morning|breakfast| +33113|AAAAAAAAKFBIAAAA|33113|9|11|53|AM|first|morning|breakfast| +33114|AAAAAAAALFBIAAAA|33114|9|11|54|AM|first|morning|breakfast| +33115|AAAAAAAAMFBIAAAA|33115|9|11|55|AM|first|morning|breakfast| +33116|AAAAAAAANFBIAAAA|33116|9|11|56|AM|first|morning|breakfast| +33117|AAAAAAAAOFBIAAAA|33117|9|11|57|AM|first|morning|breakfast| +33118|AAAAAAAAPFBIAAAA|33118|9|11|58|AM|first|morning|breakfast| +33119|AAAAAAAAAGBIAAAA|33119|9|11|59|AM|first|morning|breakfast| +33120|AAAAAAAABGBIAAAA|33120|9|12|0|AM|first|morning|breakfast| +33121|AAAAAAAACGBIAAAA|33121|9|12|1|AM|first|morning|breakfast| +33122|AAAAAAAADGBIAAAA|33122|9|12|2|AM|first|morning|breakfast| +33123|AAAAAAAAEGBIAAAA|33123|9|12|3|AM|first|morning|breakfast| +33124|AAAAAAAAFGBIAAAA|33124|9|12|4|AM|first|morning|breakfast| +33125|AAAAAAAAGGBIAAAA|33125|9|12|5|AM|first|morning|breakfast| +33126|AAAAAAAAHGBIAAAA|33126|9|12|6|AM|first|morning|breakfast| +33127|AAAAAAAAIGBIAAAA|33127|9|12|7|AM|first|morning|breakfast| +33128|AAAAAAAAJGBIAAAA|33128|9|12|8|AM|first|morning|breakfast| +33129|AAAAAAAAKGBIAAAA|33129|9|12|9|AM|first|morning|breakfast| +33130|AAAAAAAALGBIAAAA|33130|9|12|10|AM|first|morning|breakfast| +33131|AAAAAAAAMGBIAAAA|33131|9|12|11|AM|first|morning|breakfast| +33132|AAAAAAAANGBIAAAA|33132|9|12|12|AM|first|morning|breakfast| +33133|AAAAAAAAOGBIAAAA|33133|9|12|13|AM|first|morning|breakfast| +33134|AAAAAAAAPGBIAAAA|33134|9|12|14|AM|first|morning|breakfast| +33135|AAAAAAAAAHBIAAAA|33135|9|12|15|AM|first|morning|breakfast| +33136|AAAAAAAABHBIAAAA|33136|9|12|16|AM|first|morning|breakfast| +33137|AAAAAAAACHBIAAAA|33137|9|12|17|AM|first|morning|breakfast| +33138|AAAAAAAADHBIAAAA|33138|9|12|18|AM|first|morning|breakfast| +33139|AAAAAAAAEHBIAAAA|33139|9|12|19|AM|first|morning|breakfast| +33140|AAAAAAAAFHBIAAAA|33140|9|12|20|AM|first|morning|breakfast| +33141|AAAAAAAAGHBIAAAA|33141|9|12|21|AM|first|morning|breakfast| +33142|AAAAAAAAHHBIAAAA|33142|9|12|22|AM|first|morning|breakfast| +33143|AAAAAAAAIHBIAAAA|33143|9|12|23|AM|first|morning|breakfast| +33144|AAAAAAAAJHBIAAAA|33144|9|12|24|AM|first|morning|breakfast| +33145|AAAAAAAAKHBIAAAA|33145|9|12|25|AM|first|morning|breakfast| +33146|AAAAAAAALHBIAAAA|33146|9|12|26|AM|first|morning|breakfast| +33147|AAAAAAAAMHBIAAAA|33147|9|12|27|AM|first|morning|breakfast| +33148|AAAAAAAANHBIAAAA|33148|9|12|28|AM|first|morning|breakfast| +33149|AAAAAAAAOHBIAAAA|33149|9|12|29|AM|first|morning|breakfast| +33150|AAAAAAAAPHBIAAAA|33150|9|12|30|AM|first|morning|breakfast| +33151|AAAAAAAAAIBIAAAA|33151|9|12|31|AM|first|morning|breakfast| +33152|AAAAAAAABIBIAAAA|33152|9|12|32|AM|first|morning|breakfast| +33153|AAAAAAAACIBIAAAA|33153|9|12|33|AM|first|morning|breakfast| +33154|AAAAAAAADIBIAAAA|33154|9|12|34|AM|first|morning|breakfast| +33155|AAAAAAAAEIBIAAAA|33155|9|12|35|AM|first|morning|breakfast| +33156|AAAAAAAAFIBIAAAA|33156|9|12|36|AM|first|morning|breakfast| +33157|AAAAAAAAGIBIAAAA|33157|9|12|37|AM|first|morning|breakfast| +33158|AAAAAAAAHIBIAAAA|33158|9|12|38|AM|first|morning|breakfast| +33159|AAAAAAAAIIBIAAAA|33159|9|12|39|AM|first|morning|breakfast| +33160|AAAAAAAAJIBIAAAA|33160|9|12|40|AM|first|morning|breakfast| +33161|AAAAAAAAKIBIAAAA|33161|9|12|41|AM|first|morning|breakfast| +33162|AAAAAAAALIBIAAAA|33162|9|12|42|AM|first|morning|breakfast| +33163|AAAAAAAAMIBIAAAA|33163|9|12|43|AM|first|morning|breakfast| +33164|AAAAAAAANIBIAAAA|33164|9|12|44|AM|first|morning|breakfast| +33165|AAAAAAAAOIBIAAAA|33165|9|12|45|AM|first|morning|breakfast| +33166|AAAAAAAAPIBIAAAA|33166|9|12|46|AM|first|morning|breakfast| +33167|AAAAAAAAAJBIAAAA|33167|9|12|47|AM|first|morning|breakfast| +33168|AAAAAAAABJBIAAAA|33168|9|12|48|AM|first|morning|breakfast| +33169|AAAAAAAACJBIAAAA|33169|9|12|49|AM|first|morning|breakfast| +33170|AAAAAAAADJBIAAAA|33170|9|12|50|AM|first|morning|breakfast| +33171|AAAAAAAAEJBIAAAA|33171|9|12|51|AM|first|morning|breakfast| +33172|AAAAAAAAFJBIAAAA|33172|9|12|52|AM|first|morning|breakfast| +33173|AAAAAAAAGJBIAAAA|33173|9|12|53|AM|first|morning|breakfast| +33174|AAAAAAAAHJBIAAAA|33174|9|12|54|AM|first|morning|breakfast| +33175|AAAAAAAAIJBIAAAA|33175|9|12|55|AM|first|morning|breakfast| +33176|AAAAAAAAJJBIAAAA|33176|9|12|56|AM|first|morning|breakfast| +33177|AAAAAAAAKJBIAAAA|33177|9|12|57|AM|first|morning|breakfast| +33178|AAAAAAAALJBIAAAA|33178|9|12|58|AM|first|morning|breakfast| +33179|AAAAAAAAMJBIAAAA|33179|9|12|59|AM|first|morning|breakfast| +33180|AAAAAAAANJBIAAAA|33180|9|13|0|AM|first|morning|breakfast| +33181|AAAAAAAAOJBIAAAA|33181|9|13|1|AM|first|morning|breakfast| +33182|AAAAAAAAPJBIAAAA|33182|9|13|2|AM|first|morning|breakfast| +33183|AAAAAAAAAKBIAAAA|33183|9|13|3|AM|first|morning|breakfast| +33184|AAAAAAAABKBIAAAA|33184|9|13|4|AM|first|morning|breakfast| +33185|AAAAAAAACKBIAAAA|33185|9|13|5|AM|first|morning|breakfast| +33186|AAAAAAAADKBIAAAA|33186|9|13|6|AM|first|morning|breakfast| +33187|AAAAAAAAEKBIAAAA|33187|9|13|7|AM|first|morning|breakfast| +33188|AAAAAAAAFKBIAAAA|33188|9|13|8|AM|first|morning|breakfast| +33189|AAAAAAAAGKBIAAAA|33189|9|13|9|AM|first|morning|breakfast| +33190|AAAAAAAAHKBIAAAA|33190|9|13|10|AM|first|morning|breakfast| +33191|AAAAAAAAIKBIAAAA|33191|9|13|11|AM|first|morning|breakfast| +33192|AAAAAAAAJKBIAAAA|33192|9|13|12|AM|first|morning|breakfast| +33193|AAAAAAAAKKBIAAAA|33193|9|13|13|AM|first|morning|breakfast| +33194|AAAAAAAALKBIAAAA|33194|9|13|14|AM|first|morning|breakfast| +33195|AAAAAAAAMKBIAAAA|33195|9|13|15|AM|first|morning|breakfast| +33196|AAAAAAAANKBIAAAA|33196|9|13|16|AM|first|morning|breakfast| +33197|AAAAAAAAOKBIAAAA|33197|9|13|17|AM|first|morning|breakfast| +33198|AAAAAAAAPKBIAAAA|33198|9|13|18|AM|first|morning|breakfast| +33199|AAAAAAAAALBIAAAA|33199|9|13|19|AM|first|morning|breakfast| +33200|AAAAAAAABLBIAAAA|33200|9|13|20|AM|first|morning|breakfast| +33201|AAAAAAAACLBIAAAA|33201|9|13|21|AM|first|morning|breakfast| +33202|AAAAAAAADLBIAAAA|33202|9|13|22|AM|first|morning|breakfast| +33203|AAAAAAAAELBIAAAA|33203|9|13|23|AM|first|morning|breakfast| +33204|AAAAAAAAFLBIAAAA|33204|9|13|24|AM|first|morning|breakfast| +33205|AAAAAAAAGLBIAAAA|33205|9|13|25|AM|first|morning|breakfast| +33206|AAAAAAAAHLBIAAAA|33206|9|13|26|AM|first|morning|breakfast| +33207|AAAAAAAAILBIAAAA|33207|9|13|27|AM|first|morning|breakfast| +33208|AAAAAAAAJLBIAAAA|33208|9|13|28|AM|first|morning|breakfast| +33209|AAAAAAAAKLBIAAAA|33209|9|13|29|AM|first|morning|breakfast| +33210|AAAAAAAALLBIAAAA|33210|9|13|30|AM|first|morning|breakfast| +33211|AAAAAAAAMLBIAAAA|33211|9|13|31|AM|first|morning|breakfast| +33212|AAAAAAAANLBIAAAA|33212|9|13|32|AM|first|morning|breakfast| +33213|AAAAAAAAOLBIAAAA|33213|9|13|33|AM|first|morning|breakfast| +33214|AAAAAAAAPLBIAAAA|33214|9|13|34|AM|first|morning|breakfast| +33215|AAAAAAAAAMBIAAAA|33215|9|13|35|AM|first|morning|breakfast| +33216|AAAAAAAABMBIAAAA|33216|9|13|36|AM|first|morning|breakfast| +33217|AAAAAAAACMBIAAAA|33217|9|13|37|AM|first|morning|breakfast| +33218|AAAAAAAADMBIAAAA|33218|9|13|38|AM|first|morning|breakfast| +33219|AAAAAAAAEMBIAAAA|33219|9|13|39|AM|first|morning|breakfast| +33220|AAAAAAAAFMBIAAAA|33220|9|13|40|AM|first|morning|breakfast| +33221|AAAAAAAAGMBIAAAA|33221|9|13|41|AM|first|morning|breakfast| +33222|AAAAAAAAHMBIAAAA|33222|9|13|42|AM|first|morning|breakfast| +33223|AAAAAAAAIMBIAAAA|33223|9|13|43|AM|first|morning|breakfast| +33224|AAAAAAAAJMBIAAAA|33224|9|13|44|AM|first|morning|breakfast| +33225|AAAAAAAAKMBIAAAA|33225|9|13|45|AM|first|morning|breakfast| +33226|AAAAAAAALMBIAAAA|33226|9|13|46|AM|first|morning|breakfast| +33227|AAAAAAAAMMBIAAAA|33227|9|13|47|AM|first|morning|breakfast| +33228|AAAAAAAANMBIAAAA|33228|9|13|48|AM|first|morning|breakfast| +33229|AAAAAAAAOMBIAAAA|33229|9|13|49|AM|first|morning|breakfast| +33230|AAAAAAAAPMBIAAAA|33230|9|13|50|AM|first|morning|breakfast| +33231|AAAAAAAAANBIAAAA|33231|9|13|51|AM|first|morning|breakfast| +33232|AAAAAAAABNBIAAAA|33232|9|13|52|AM|first|morning|breakfast| +33233|AAAAAAAACNBIAAAA|33233|9|13|53|AM|first|morning|breakfast| +33234|AAAAAAAADNBIAAAA|33234|9|13|54|AM|first|morning|breakfast| +33235|AAAAAAAAENBIAAAA|33235|9|13|55|AM|first|morning|breakfast| +33236|AAAAAAAAFNBIAAAA|33236|9|13|56|AM|first|morning|breakfast| +33237|AAAAAAAAGNBIAAAA|33237|9|13|57|AM|first|morning|breakfast| +33238|AAAAAAAAHNBIAAAA|33238|9|13|58|AM|first|morning|breakfast| +33239|AAAAAAAAINBIAAAA|33239|9|13|59|AM|first|morning|breakfast| +33240|AAAAAAAAJNBIAAAA|33240|9|14|0|AM|first|morning|breakfast| +33241|AAAAAAAAKNBIAAAA|33241|9|14|1|AM|first|morning|breakfast| +33242|AAAAAAAALNBIAAAA|33242|9|14|2|AM|first|morning|breakfast| +33243|AAAAAAAAMNBIAAAA|33243|9|14|3|AM|first|morning|breakfast| +33244|AAAAAAAANNBIAAAA|33244|9|14|4|AM|first|morning|breakfast| +33245|AAAAAAAAONBIAAAA|33245|9|14|5|AM|first|morning|breakfast| +33246|AAAAAAAAPNBIAAAA|33246|9|14|6|AM|first|morning|breakfast| +33247|AAAAAAAAAOBIAAAA|33247|9|14|7|AM|first|morning|breakfast| +33248|AAAAAAAABOBIAAAA|33248|9|14|8|AM|first|morning|breakfast| +33249|AAAAAAAACOBIAAAA|33249|9|14|9|AM|first|morning|breakfast| +33250|AAAAAAAADOBIAAAA|33250|9|14|10|AM|first|morning|breakfast| +33251|AAAAAAAAEOBIAAAA|33251|9|14|11|AM|first|morning|breakfast| +33252|AAAAAAAAFOBIAAAA|33252|9|14|12|AM|first|morning|breakfast| +33253|AAAAAAAAGOBIAAAA|33253|9|14|13|AM|first|morning|breakfast| +33254|AAAAAAAAHOBIAAAA|33254|9|14|14|AM|first|morning|breakfast| +33255|AAAAAAAAIOBIAAAA|33255|9|14|15|AM|first|morning|breakfast| +33256|AAAAAAAAJOBIAAAA|33256|9|14|16|AM|first|morning|breakfast| +33257|AAAAAAAAKOBIAAAA|33257|9|14|17|AM|first|morning|breakfast| +33258|AAAAAAAALOBIAAAA|33258|9|14|18|AM|first|morning|breakfast| +33259|AAAAAAAAMOBIAAAA|33259|9|14|19|AM|first|morning|breakfast| +33260|AAAAAAAANOBIAAAA|33260|9|14|20|AM|first|morning|breakfast| +33261|AAAAAAAAOOBIAAAA|33261|9|14|21|AM|first|morning|breakfast| +33262|AAAAAAAAPOBIAAAA|33262|9|14|22|AM|first|morning|breakfast| +33263|AAAAAAAAAPBIAAAA|33263|9|14|23|AM|first|morning|breakfast| +33264|AAAAAAAABPBIAAAA|33264|9|14|24|AM|first|morning|breakfast| +33265|AAAAAAAACPBIAAAA|33265|9|14|25|AM|first|morning|breakfast| +33266|AAAAAAAADPBIAAAA|33266|9|14|26|AM|first|morning|breakfast| +33267|AAAAAAAAEPBIAAAA|33267|9|14|27|AM|first|morning|breakfast| +33268|AAAAAAAAFPBIAAAA|33268|9|14|28|AM|first|morning|breakfast| +33269|AAAAAAAAGPBIAAAA|33269|9|14|29|AM|first|morning|breakfast| +33270|AAAAAAAAHPBIAAAA|33270|9|14|30|AM|first|morning|breakfast| +33271|AAAAAAAAIPBIAAAA|33271|9|14|31|AM|first|morning|breakfast| +33272|AAAAAAAAJPBIAAAA|33272|9|14|32|AM|first|morning|breakfast| +33273|AAAAAAAAKPBIAAAA|33273|9|14|33|AM|first|morning|breakfast| +33274|AAAAAAAALPBIAAAA|33274|9|14|34|AM|first|morning|breakfast| +33275|AAAAAAAAMPBIAAAA|33275|9|14|35|AM|first|morning|breakfast| +33276|AAAAAAAANPBIAAAA|33276|9|14|36|AM|first|morning|breakfast| +33277|AAAAAAAAOPBIAAAA|33277|9|14|37|AM|first|morning|breakfast| +33278|AAAAAAAAPPBIAAAA|33278|9|14|38|AM|first|morning|breakfast| +33279|AAAAAAAAAACIAAAA|33279|9|14|39|AM|first|morning|breakfast| +33280|AAAAAAAABACIAAAA|33280|9|14|40|AM|first|morning|breakfast| +33281|AAAAAAAACACIAAAA|33281|9|14|41|AM|first|morning|breakfast| +33282|AAAAAAAADACIAAAA|33282|9|14|42|AM|first|morning|breakfast| +33283|AAAAAAAAEACIAAAA|33283|9|14|43|AM|first|morning|breakfast| +33284|AAAAAAAAFACIAAAA|33284|9|14|44|AM|first|morning|breakfast| +33285|AAAAAAAAGACIAAAA|33285|9|14|45|AM|first|morning|breakfast| +33286|AAAAAAAAHACIAAAA|33286|9|14|46|AM|first|morning|breakfast| +33287|AAAAAAAAIACIAAAA|33287|9|14|47|AM|first|morning|breakfast| +33288|AAAAAAAAJACIAAAA|33288|9|14|48|AM|first|morning|breakfast| +33289|AAAAAAAAKACIAAAA|33289|9|14|49|AM|first|morning|breakfast| +33290|AAAAAAAALACIAAAA|33290|9|14|50|AM|first|morning|breakfast| +33291|AAAAAAAAMACIAAAA|33291|9|14|51|AM|first|morning|breakfast| +33292|AAAAAAAANACIAAAA|33292|9|14|52|AM|first|morning|breakfast| +33293|AAAAAAAAOACIAAAA|33293|9|14|53|AM|first|morning|breakfast| +33294|AAAAAAAAPACIAAAA|33294|9|14|54|AM|first|morning|breakfast| +33295|AAAAAAAAABCIAAAA|33295|9|14|55|AM|first|morning|breakfast| +33296|AAAAAAAABBCIAAAA|33296|9|14|56|AM|first|morning|breakfast| +33297|AAAAAAAACBCIAAAA|33297|9|14|57|AM|first|morning|breakfast| +33298|AAAAAAAADBCIAAAA|33298|9|14|58|AM|first|morning|breakfast| +33299|AAAAAAAAEBCIAAAA|33299|9|14|59|AM|first|morning|breakfast| +33300|AAAAAAAAFBCIAAAA|33300|9|15|0|AM|first|morning|breakfast| +33301|AAAAAAAAGBCIAAAA|33301|9|15|1|AM|first|morning|breakfast| +33302|AAAAAAAAHBCIAAAA|33302|9|15|2|AM|first|morning|breakfast| +33303|AAAAAAAAIBCIAAAA|33303|9|15|3|AM|first|morning|breakfast| +33304|AAAAAAAAJBCIAAAA|33304|9|15|4|AM|first|morning|breakfast| +33305|AAAAAAAAKBCIAAAA|33305|9|15|5|AM|first|morning|breakfast| +33306|AAAAAAAALBCIAAAA|33306|9|15|6|AM|first|morning|breakfast| +33307|AAAAAAAAMBCIAAAA|33307|9|15|7|AM|first|morning|breakfast| +33308|AAAAAAAANBCIAAAA|33308|9|15|8|AM|first|morning|breakfast| +33309|AAAAAAAAOBCIAAAA|33309|9|15|9|AM|first|morning|breakfast| +33310|AAAAAAAAPBCIAAAA|33310|9|15|10|AM|first|morning|breakfast| +33311|AAAAAAAAACCIAAAA|33311|9|15|11|AM|first|morning|breakfast| +33312|AAAAAAAABCCIAAAA|33312|9|15|12|AM|first|morning|breakfast| +33313|AAAAAAAACCCIAAAA|33313|9|15|13|AM|first|morning|breakfast| +33314|AAAAAAAADCCIAAAA|33314|9|15|14|AM|first|morning|breakfast| +33315|AAAAAAAAECCIAAAA|33315|9|15|15|AM|first|morning|breakfast| +33316|AAAAAAAAFCCIAAAA|33316|9|15|16|AM|first|morning|breakfast| +33317|AAAAAAAAGCCIAAAA|33317|9|15|17|AM|first|morning|breakfast| +33318|AAAAAAAAHCCIAAAA|33318|9|15|18|AM|first|morning|breakfast| +33319|AAAAAAAAICCIAAAA|33319|9|15|19|AM|first|morning|breakfast| +33320|AAAAAAAAJCCIAAAA|33320|9|15|20|AM|first|morning|breakfast| +33321|AAAAAAAAKCCIAAAA|33321|9|15|21|AM|first|morning|breakfast| +33322|AAAAAAAALCCIAAAA|33322|9|15|22|AM|first|morning|breakfast| +33323|AAAAAAAAMCCIAAAA|33323|9|15|23|AM|first|morning|breakfast| +33324|AAAAAAAANCCIAAAA|33324|9|15|24|AM|first|morning|breakfast| +33325|AAAAAAAAOCCIAAAA|33325|9|15|25|AM|first|morning|breakfast| +33326|AAAAAAAAPCCIAAAA|33326|9|15|26|AM|first|morning|breakfast| +33327|AAAAAAAAADCIAAAA|33327|9|15|27|AM|first|morning|breakfast| +33328|AAAAAAAABDCIAAAA|33328|9|15|28|AM|first|morning|breakfast| +33329|AAAAAAAACDCIAAAA|33329|9|15|29|AM|first|morning|breakfast| +33330|AAAAAAAADDCIAAAA|33330|9|15|30|AM|first|morning|breakfast| +33331|AAAAAAAAEDCIAAAA|33331|9|15|31|AM|first|morning|breakfast| +33332|AAAAAAAAFDCIAAAA|33332|9|15|32|AM|first|morning|breakfast| +33333|AAAAAAAAGDCIAAAA|33333|9|15|33|AM|first|morning|breakfast| +33334|AAAAAAAAHDCIAAAA|33334|9|15|34|AM|first|morning|breakfast| +33335|AAAAAAAAIDCIAAAA|33335|9|15|35|AM|first|morning|breakfast| +33336|AAAAAAAAJDCIAAAA|33336|9|15|36|AM|first|morning|breakfast| +33337|AAAAAAAAKDCIAAAA|33337|9|15|37|AM|first|morning|breakfast| +33338|AAAAAAAALDCIAAAA|33338|9|15|38|AM|first|morning|breakfast| +33339|AAAAAAAAMDCIAAAA|33339|9|15|39|AM|first|morning|breakfast| +33340|AAAAAAAANDCIAAAA|33340|9|15|40|AM|first|morning|breakfast| +33341|AAAAAAAAODCIAAAA|33341|9|15|41|AM|first|morning|breakfast| +33342|AAAAAAAAPDCIAAAA|33342|9|15|42|AM|first|morning|breakfast| +33343|AAAAAAAAAECIAAAA|33343|9|15|43|AM|first|morning|breakfast| +33344|AAAAAAAABECIAAAA|33344|9|15|44|AM|first|morning|breakfast| +33345|AAAAAAAACECIAAAA|33345|9|15|45|AM|first|morning|breakfast| +33346|AAAAAAAADECIAAAA|33346|9|15|46|AM|first|morning|breakfast| +33347|AAAAAAAAEECIAAAA|33347|9|15|47|AM|first|morning|breakfast| +33348|AAAAAAAAFECIAAAA|33348|9|15|48|AM|first|morning|breakfast| +33349|AAAAAAAAGECIAAAA|33349|9|15|49|AM|first|morning|breakfast| +33350|AAAAAAAAHECIAAAA|33350|9|15|50|AM|first|morning|breakfast| +33351|AAAAAAAAIECIAAAA|33351|9|15|51|AM|first|morning|breakfast| +33352|AAAAAAAAJECIAAAA|33352|9|15|52|AM|first|morning|breakfast| +33353|AAAAAAAAKECIAAAA|33353|9|15|53|AM|first|morning|breakfast| +33354|AAAAAAAALECIAAAA|33354|9|15|54|AM|first|morning|breakfast| +33355|AAAAAAAAMECIAAAA|33355|9|15|55|AM|first|morning|breakfast| +33356|AAAAAAAANECIAAAA|33356|9|15|56|AM|first|morning|breakfast| +33357|AAAAAAAAOECIAAAA|33357|9|15|57|AM|first|morning|breakfast| +33358|AAAAAAAAPECIAAAA|33358|9|15|58|AM|first|morning|breakfast| +33359|AAAAAAAAAFCIAAAA|33359|9|15|59|AM|first|morning|breakfast| +33360|AAAAAAAABFCIAAAA|33360|9|16|0|AM|first|morning|breakfast| +33361|AAAAAAAACFCIAAAA|33361|9|16|1|AM|first|morning|breakfast| +33362|AAAAAAAADFCIAAAA|33362|9|16|2|AM|first|morning|breakfast| +33363|AAAAAAAAEFCIAAAA|33363|9|16|3|AM|first|morning|breakfast| +33364|AAAAAAAAFFCIAAAA|33364|9|16|4|AM|first|morning|breakfast| +33365|AAAAAAAAGFCIAAAA|33365|9|16|5|AM|first|morning|breakfast| +33366|AAAAAAAAHFCIAAAA|33366|9|16|6|AM|first|morning|breakfast| +33367|AAAAAAAAIFCIAAAA|33367|9|16|7|AM|first|morning|breakfast| +33368|AAAAAAAAJFCIAAAA|33368|9|16|8|AM|first|morning|breakfast| +33369|AAAAAAAAKFCIAAAA|33369|9|16|9|AM|first|morning|breakfast| +33370|AAAAAAAALFCIAAAA|33370|9|16|10|AM|first|morning|breakfast| +33371|AAAAAAAAMFCIAAAA|33371|9|16|11|AM|first|morning|breakfast| +33372|AAAAAAAANFCIAAAA|33372|9|16|12|AM|first|morning|breakfast| +33373|AAAAAAAAOFCIAAAA|33373|9|16|13|AM|first|morning|breakfast| +33374|AAAAAAAAPFCIAAAA|33374|9|16|14|AM|first|morning|breakfast| +33375|AAAAAAAAAGCIAAAA|33375|9|16|15|AM|first|morning|breakfast| +33376|AAAAAAAABGCIAAAA|33376|9|16|16|AM|first|morning|breakfast| +33377|AAAAAAAACGCIAAAA|33377|9|16|17|AM|first|morning|breakfast| +33378|AAAAAAAADGCIAAAA|33378|9|16|18|AM|first|morning|breakfast| +33379|AAAAAAAAEGCIAAAA|33379|9|16|19|AM|first|morning|breakfast| +33380|AAAAAAAAFGCIAAAA|33380|9|16|20|AM|first|morning|breakfast| +33381|AAAAAAAAGGCIAAAA|33381|9|16|21|AM|first|morning|breakfast| +33382|AAAAAAAAHGCIAAAA|33382|9|16|22|AM|first|morning|breakfast| +33383|AAAAAAAAIGCIAAAA|33383|9|16|23|AM|first|morning|breakfast| +33384|AAAAAAAAJGCIAAAA|33384|9|16|24|AM|first|morning|breakfast| +33385|AAAAAAAAKGCIAAAA|33385|9|16|25|AM|first|morning|breakfast| +33386|AAAAAAAALGCIAAAA|33386|9|16|26|AM|first|morning|breakfast| +33387|AAAAAAAAMGCIAAAA|33387|9|16|27|AM|first|morning|breakfast| +33388|AAAAAAAANGCIAAAA|33388|9|16|28|AM|first|morning|breakfast| +33389|AAAAAAAAOGCIAAAA|33389|9|16|29|AM|first|morning|breakfast| +33390|AAAAAAAAPGCIAAAA|33390|9|16|30|AM|first|morning|breakfast| +33391|AAAAAAAAAHCIAAAA|33391|9|16|31|AM|first|morning|breakfast| +33392|AAAAAAAABHCIAAAA|33392|9|16|32|AM|first|morning|breakfast| +33393|AAAAAAAACHCIAAAA|33393|9|16|33|AM|first|morning|breakfast| +33394|AAAAAAAADHCIAAAA|33394|9|16|34|AM|first|morning|breakfast| +33395|AAAAAAAAEHCIAAAA|33395|9|16|35|AM|first|morning|breakfast| +33396|AAAAAAAAFHCIAAAA|33396|9|16|36|AM|first|morning|breakfast| +33397|AAAAAAAAGHCIAAAA|33397|9|16|37|AM|first|morning|breakfast| +33398|AAAAAAAAHHCIAAAA|33398|9|16|38|AM|first|morning|breakfast| +33399|AAAAAAAAIHCIAAAA|33399|9|16|39|AM|first|morning|breakfast| +33400|AAAAAAAAJHCIAAAA|33400|9|16|40|AM|first|morning|breakfast| +33401|AAAAAAAAKHCIAAAA|33401|9|16|41|AM|first|morning|breakfast| +33402|AAAAAAAALHCIAAAA|33402|9|16|42|AM|first|morning|breakfast| +33403|AAAAAAAAMHCIAAAA|33403|9|16|43|AM|first|morning|breakfast| +33404|AAAAAAAANHCIAAAA|33404|9|16|44|AM|first|morning|breakfast| +33405|AAAAAAAAOHCIAAAA|33405|9|16|45|AM|first|morning|breakfast| +33406|AAAAAAAAPHCIAAAA|33406|9|16|46|AM|first|morning|breakfast| +33407|AAAAAAAAAICIAAAA|33407|9|16|47|AM|first|morning|breakfast| +33408|AAAAAAAABICIAAAA|33408|9|16|48|AM|first|morning|breakfast| +33409|AAAAAAAACICIAAAA|33409|9|16|49|AM|first|morning|breakfast| +33410|AAAAAAAADICIAAAA|33410|9|16|50|AM|first|morning|breakfast| +33411|AAAAAAAAEICIAAAA|33411|9|16|51|AM|first|morning|breakfast| +33412|AAAAAAAAFICIAAAA|33412|9|16|52|AM|first|morning|breakfast| +33413|AAAAAAAAGICIAAAA|33413|9|16|53|AM|first|morning|breakfast| +33414|AAAAAAAAHICIAAAA|33414|9|16|54|AM|first|morning|breakfast| +33415|AAAAAAAAIICIAAAA|33415|9|16|55|AM|first|morning|breakfast| +33416|AAAAAAAAJICIAAAA|33416|9|16|56|AM|first|morning|breakfast| +33417|AAAAAAAAKICIAAAA|33417|9|16|57|AM|first|morning|breakfast| +33418|AAAAAAAALICIAAAA|33418|9|16|58|AM|first|morning|breakfast| +33419|AAAAAAAAMICIAAAA|33419|9|16|59|AM|first|morning|breakfast| +33420|AAAAAAAANICIAAAA|33420|9|17|0|AM|first|morning|breakfast| +33421|AAAAAAAAOICIAAAA|33421|9|17|1|AM|first|morning|breakfast| +33422|AAAAAAAAPICIAAAA|33422|9|17|2|AM|first|morning|breakfast| +33423|AAAAAAAAAJCIAAAA|33423|9|17|3|AM|first|morning|breakfast| +33424|AAAAAAAABJCIAAAA|33424|9|17|4|AM|first|morning|breakfast| +33425|AAAAAAAACJCIAAAA|33425|9|17|5|AM|first|morning|breakfast| +33426|AAAAAAAADJCIAAAA|33426|9|17|6|AM|first|morning|breakfast| +33427|AAAAAAAAEJCIAAAA|33427|9|17|7|AM|first|morning|breakfast| +33428|AAAAAAAAFJCIAAAA|33428|9|17|8|AM|first|morning|breakfast| +33429|AAAAAAAAGJCIAAAA|33429|9|17|9|AM|first|morning|breakfast| +33430|AAAAAAAAHJCIAAAA|33430|9|17|10|AM|first|morning|breakfast| +33431|AAAAAAAAIJCIAAAA|33431|9|17|11|AM|first|morning|breakfast| +33432|AAAAAAAAJJCIAAAA|33432|9|17|12|AM|first|morning|breakfast| +33433|AAAAAAAAKJCIAAAA|33433|9|17|13|AM|first|morning|breakfast| +33434|AAAAAAAALJCIAAAA|33434|9|17|14|AM|first|morning|breakfast| +33435|AAAAAAAAMJCIAAAA|33435|9|17|15|AM|first|morning|breakfast| +33436|AAAAAAAANJCIAAAA|33436|9|17|16|AM|first|morning|breakfast| +33437|AAAAAAAAOJCIAAAA|33437|9|17|17|AM|first|morning|breakfast| +33438|AAAAAAAAPJCIAAAA|33438|9|17|18|AM|first|morning|breakfast| +33439|AAAAAAAAAKCIAAAA|33439|9|17|19|AM|first|morning|breakfast| +33440|AAAAAAAABKCIAAAA|33440|9|17|20|AM|first|morning|breakfast| +33441|AAAAAAAACKCIAAAA|33441|9|17|21|AM|first|morning|breakfast| +33442|AAAAAAAADKCIAAAA|33442|9|17|22|AM|first|morning|breakfast| +33443|AAAAAAAAEKCIAAAA|33443|9|17|23|AM|first|morning|breakfast| +33444|AAAAAAAAFKCIAAAA|33444|9|17|24|AM|first|morning|breakfast| +33445|AAAAAAAAGKCIAAAA|33445|9|17|25|AM|first|morning|breakfast| +33446|AAAAAAAAHKCIAAAA|33446|9|17|26|AM|first|morning|breakfast| +33447|AAAAAAAAIKCIAAAA|33447|9|17|27|AM|first|morning|breakfast| +33448|AAAAAAAAJKCIAAAA|33448|9|17|28|AM|first|morning|breakfast| +33449|AAAAAAAAKKCIAAAA|33449|9|17|29|AM|first|morning|breakfast| +33450|AAAAAAAALKCIAAAA|33450|9|17|30|AM|first|morning|breakfast| +33451|AAAAAAAAMKCIAAAA|33451|9|17|31|AM|first|morning|breakfast| +33452|AAAAAAAANKCIAAAA|33452|9|17|32|AM|first|morning|breakfast| +33453|AAAAAAAAOKCIAAAA|33453|9|17|33|AM|first|morning|breakfast| +33454|AAAAAAAAPKCIAAAA|33454|9|17|34|AM|first|morning|breakfast| +33455|AAAAAAAAALCIAAAA|33455|9|17|35|AM|first|morning|breakfast| +33456|AAAAAAAABLCIAAAA|33456|9|17|36|AM|first|morning|breakfast| +33457|AAAAAAAACLCIAAAA|33457|9|17|37|AM|first|morning|breakfast| +33458|AAAAAAAADLCIAAAA|33458|9|17|38|AM|first|morning|breakfast| +33459|AAAAAAAAELCIAAAA|33459|9|17|39|AM|first|morning|breakfast| +33460|AAAAAAAAFLCIAAAA|33460|9|17|40|AM|first|morning|breakfast| +33461|AAAAAAAAGLCIAAAA|33461|9|17|41|AM|first|morning|breakfast| +33462|AAAAAAAAHLCIAAAA|33462|9|17|42|AM|first|morning|breakfast| +33463|AAAAAAAAILCIAAAA|33463|9|17|43|AM|first|morning|breakfast| +33464|AAAAAAAAJLCIAAAA|33464|9|17|44|AM|first|morning|breakfast| +33465|AAAAAAAAKLCIAAAA|33465|9|17|45|AM|first|morning|breakfast| +33466|AAAAAAAALLCIAAAA|33466|9|17|46|AM|first|morning|breakfast| +33467|AAAAAAAAMLCIAAAA|33467|9|17|47|AM|first|morning|breakfast| +33468|AAAAAAAANLCIAAAA|33468|9|17|48|AM|first|morning|breakfast| +33469|AAAAAAAAOLCIAAAA|33469|9|17|49|AM|first|morning|breakfast| +33470|AAAAAAAAPLCIAAAA|33470|9|17|50|AM|first|morning|breakfast| +33471|AAAAAAAAAMCIAAAA|33471|9|17|51|AM|first|morning|breakfast| +33472|AAAAAAAABMCIAAAA|33472|9|17|52|AM|first|morning|breakfast| +33473|AAAAAAAACMCIAAAA|33473|9|17|53|AM|first|morning|breakfast| +33474|AAAAAAAADMCIAAAA|33474|9|17|54|AM|first|morning|breakfast| +33475|AAAAAAAAEMCIAAAA|33475|9|17|55|AM|first|morning|breakfast| +33476|AAAAAAAAFMCIAAAA|33476|9|17|56|AM|first|morning|breakfast| +33477|AAAAAAAAGMCIAAAA|33477|9|17|57|AM|first|morning|breakfast| +33478|AAAAAAAAHMCIAAAA|33478|9|17|58|AM|first|morning|breakfast| +33479|AAAAAAAAIMCIAAAA|33479|9|17|59|AM|first|morning|breakfast| +33480|AAAAAAAAJMCIAAAA|33480|9|18|0|AM|first|morning|breakfast| +33481|AAAAAAAAKMCIAAAA|33481|9|18|1|AM|first|morning|breakfast| +33482|AAAAAAAALMCIAAAA|33482|9|18|2|AM|first|morning|breakfast| +33483|AAAAAAAAMMCIAAAA|33483|9|18|3|AM|first|morning|breakfast| +33484|AAAAAAAANMCIAAAA|33484|9|18|4|AM|first|morning|breakfast| +33485|AAAAAAAAOMCIAAAA|33485|9|18|5|AM|first|morning|breakfast| +33486|AAAAAAAAPMCIAAAA|33486|9|18|6|AM|first|morning|breakfast| +33487|AAAAAAAAANCIAAAA|33487|9|18|7|AM|first|morning|breakfast| +33488|AAAAAAAABNCIAAAA|33488|9|18|8|AM|first|morning|breakfast| +33489|AAAAAAAACNCIAAAA|33489|9|18|9|AM|first|morning|breakfast| +33490|AAAAAAAADNCIAAAA|33490|9|18|10|AM|first|morning|breakfast| +33491|AAAAAAAAENCIAAAA|33491|9|18|11|AM|first|morning|breakfast| +33492|AAAAAAAAFNCIAAAA|33492|9|18|12|AM|first|morning|breakfast| +33493|AAAAAAAAGNCIAAAA|33493|9|18|13|AM|first|morning|breakfast| +33494|AAAAAAAAHNCIAAAA|33494|9|18|14|AM|first|morning|breakfast| +33495|AAAAAAAAINCIAAAA|33495|9|18|15|AM|first|morning|breakfast| +33496|AAAAAAAAJNCIAAAA|33496|9|18|16|AM|first|morning|breakfast| +33497|AAAAAAAAKNCIAAAA|33497|9|18|17|AM|first|morning|breakfast| +33498|AAAAAAAALNCIAAAA|33498|9|18|18|AM|first|morning|breakfast| +33499|AAAAAAAAMNCIAAAA|33499|9|18|19|AM|first|morning|breakfast| +33500|AAAAAAAANNCIAAAA|33500|9|18|20|AM|first|morning|breakfast| +33501|AAAAAAAAONCIAAAA|33501|9|18|21|AM|first|morning|breakfast| +33502|AAAAAAAAPNCIAAAA|33502|9|18|22|AM|first|morning|breakfast| +33503|AAAAAAAAAOCIAAAA|33503|9|18|23|AM|first|morning|breakfast| +33504|AAAAAAAABOCIAAAA|33504|9|18|24|AM|first|morning|breakfast| +33505|AAAAAAAACOCIAAAA|33505|9|18|25|AM|first|morning|breakfast| +33506|AAAAAAAADOCIAAAA|33506|9|18|26|AM|first|morning|breakfast| +33507|AAAAAAAAEOCIAAAA|33507|9|18|27|AM|first|morning|breakfast| +33508|AAAAAAAAFOCIAAAA|33508|9|18|28|AM|first|morning|breakfast| +33509|AAAAAAAAGOCIAAAA|33509|9|18|29|AM|first|morning|breakfast| +33510|AAAAAAAAHOCIAAAA|33510|9|18|30|AM|first|morning|breakfast| +33511|AAAAAAAAIOCIAAAA|33511|9|18|31|AM|first|morning|breakfast| +33512|AAAAAAAAJOCIAAAA|33512|9|18|32|AM|first|morning|breakfast| +33513|AAAAAAAAKOCIAAAA|33513|9|18|33|AM|first|morning|breakfast| +33514|AAAAAAAALOCIAAAA|33514|9|18|34|AM|first|morning|breakfast| +33515|AAAAAAAAMOCIAAAA|33515|9|18|35|AM|first|morning|breakfast| +33516|AAAAAAAANOCIAAAA|33516|9|18|36|AM|first|morning|breakfast| +33517|AAAAAAAAOOCIAAAA|33517|9|18|37|AM|first|morning|breakfast| +33518|AAAAAAAAPOCIAAAA|33518|9|18|38|AM|first|morning|breakfast| +33519|AAAAAAAAAPCIAAAA|33519|9|18|39|AM|first|morning|breakfast| +33520|AAAAAAAABPCIAAAA|33520|9|18|40|AM|first|morning|breakfast| +33521|AAAAAAAACPCIAAAA|33521|9|18|41|AM|first|morning|breakfast| +33522|AAAAAAAADPCIAAAA|33522|9|18|42|AM|first|morning|breakfast| +33523|AAAAAAAAEPCIAAAA|33523|9|18|43|AM|first|morning|breakfast| +33524|AAAAAAAAFPCIAAAA|33524|9|18|44|AM|first|morning|breakfast| +33525|AAAAAAAAGPCIAAAA|33525|9|18|45|AM|first|morning|breakfast| +33526|AAAAAAAAHPCIAAAA|33526|9|18|46|AM|first|morning|breakfast| +33527|AAAAAAAAIPCIAAAA|33527|9|18|47|AM|first|morning|breakfast| +33528|AAAAAAAAJPCIAAAA|33528|9|18|48|AM|first|morning|breakfast| +33529|AAAAAAAAKPCIAAAA|33529|9|18|49|AM|first|morning|breakfast| +33530|AAAAAAAALPCIAAAA|33530|9|18|50|AM|first|morning|breakfast| +33531|AAAAAAAAMPCIAAAA|33531|9|18|51|AM|first|morning|breakfast| +33532|AAAAAAAANPCIAAAA|33532|9|18|52|AM|first|morning|breakfast| +33533|AAAAAAAAOPCIAAAA|33533|9|18|53|AM|first|morning|breakfast| +33534|AAAAAAAAPPCIAAAA|33534|9|18|54|AM|first|morning|breakfast| +33535|AAAAAAAAAADIAAAA|33535|9|18|55|AM|first|morning|breakfast| +33536|AAAAAAAABADIAAAA|33536|9|18|56|AM|first|morning|breakfast| +33537|AAAAAAAACADIAAAA|33537|9|18|57|AM|first|morning|breakfast| +33538|AAAAAAAADADIAAAA|33538|9|18|58|AM|first|morning|breakfast| +33539|AAAAAAAAEADIAAAA|33539|9|18|59|AM|first|morning|breakfast| +33540|AAAAAAAAFADIAAAA|33540|9|19|0|AM|first|morning|breakfast| +33541|AAAAAAAAGADIAAAA|33541|9|19|1|AM|first|morning|breakfast| +33542|AAAAAAAAHADIAAAA|33542|9|19|2|AM|first|morning|breakfast| +33543|AAAAAAAAIADIAAAA|33543|9|19|3|AM|first|morning|breakfast| +33544|AAAAAAAAJADIAAAA|33544|9|19|4|AM|first|morning|breakfast| +33545|AAAAAAAAKADIAAAA|33545|9|19|5|AM|first|morning|breakfast| +33546|AAAAAAAALADIAAAA|33546|9|19|6|AM|first|morning|breakfast| +33547|AAAAAAAAMADIAAAA|33547|9|19|7|AM|first|morning|breakfast| +33548|AAAAAAAANADIAAAA|33548|9|19|8|AM|first|morning|breakfast| +33549|AAAAAAAAOADIAAAA|33549|9|19|9|AM|first|morning|breakfast| +33550|AAAAAAAAPADIAAAA|33550|9|19|10|AM|first|morning|breakfast| +33551|AAAAAAAAABDIAAAA|33551|9|19|11|AM|first|morning|breakfast| +33552|AAAAAAAABBDIAAAA|33552|9|19|12|AM|first|morning|breakfast| +33553|AAAAAAAACBDIAAAA|33553|9|19|13|AM|first|morning|breakfast| +33554|AAAAAAAADBDIAAAA|33554|9|19|14|AM|first|morning|breakfast| +33555|AAAAAAAAEBDIAAAA|33555|9|19|15|AM|first|morning|breakfast| +33556|AAAAAAAAFBDIAAAA|33556|9|19|16|AM|first|morning|breakfast| +33557|AAAAAAAAGBDIAAAA|33557|9|19|17|AM|first|morning|breakfast| +33558|AAAAAAAAHBDIAAAA|33558|9|19|18|AM|first|morning|breakfast| +33559|AAAAAAAAIBDIAAAA|33559|9|19|19|AM|first|morning|breakfast| +33560|AAAAAAAAJBDIAAAA|33560|9|19|20|AM|first|morning|breakfast| +33561|AAAAAAAAKBDIAAAA|33561|9|19|21|AM|first|morning|breakfast| +33562|AAAAAAAALBDIAAAA|33562|9|19|22|AM|first|morning|breakfast| +33563|AAAAAAAAMBDIAAAA|33563|9|19|23|AM|first|morning|breakfast| +33564|AAAAAAAANBDIAAAA|33564|9|19|24|AM|first|morning|breakfast| +33565|AAAAAAAAOBDIAAAA|33565|9|19|25|AM|first|morning|breakfast| +33566|AAAAAAAAPBDIAAAA|33566|9|19|26|AM|first|morning|breakfast| +33567|AAAAAAAAACDIAAAA|33567|9|19|27|AM|first|morning|breakfast| +33568|AAAAAAAABCDIAAAA|33568|9|19|28|AM|first|morning|breakfast| +33569|AAAAAAAACCDIAAAA|33569|9|19|29|AM|first|morning|breakfast| +33570|AAAAAAAADCDIAAAA|33570|9|19|30|AM|first|morning|breakfast| +33571|AAAAAAAAECDIAAAA|33571|9|19|31|AM|first|morning|breakfast| +33572|AAAAAAAAFCDIAAAA|33572|9|19|32|AM|first|morning|breakfast| +33573|AAAAAAAAGCDIAAAA|33573|9|19|33|AM|first|morning|breakfast| +33574|AAAAAAAAHCDIAAAA|33574|9|19|34|AM|first|morning|breakfast| +33575|AAAAAAAAICDIAAAA|33575|9|19|35|AM|first|morning|breakfast| +33576|AAAAAAAAJCDIAAAA|33576|9|19|36|AM|first|morning|breakfast| +33577|AAAAAAAAKCDIAAAA|33577|9|19|37|AM|first|morning|breakfast| +33578|AAAAAAAALCDIAAAA|33578|9|19|38|AM|first|morning|breakfast| +33579|AAAAAAAAMCDIAAAA|33579|9|19|39|AM|first|morning|breakfast| +33580|AAAAAAAANCDIAAAA|33580|9|19|40|AM|first|morning|breakfast| +33581|AAAAAAAAOCDIAAAA|33581|9|19|41|AM|first|morning|breakfast| +33582|AAAAAAAAPCDIAAAA|33582|9|19|42|AM|first|morning|breakfast| +33583|AAAAAAAAADDIAAAA|33583|9|19|43|AM|first|morning|breakfast| +33584|AAAAAAAABDDIAAAA|33584|9|19|44|AM|first|morning|breakfast| +33585|AAAAAAAACDDIAAAA|33585|9|19|45|AM|first|morning|breakfast| +33586|AAAAAAAADDDIAAAA|33586|9|19|46|AM|first|morning|breakfast| +33587|AAAAAAAAEDDIAAAA|33587|9|19|47|AM|first|morning|breakfast| +33588|AAAAAAAAFDDIAAAA|33588|9|19|48|AM|first|morning|breakfast| +33589|AAAAAAAAGDDIAAAA|33589|9|19|49|AM|first|morning|breakfast| +33590|AAAAAAAAHDDIAAAA|33590|9|19|50|AM|first|morning|breakfast| +33591|AAAAAAAAIDDIAAAA|33591|9|19|51|AM|first|morning|breakfast| +33592|AAAAAAAAJDDIAAAA|33592|9|19|52|AM|first|morning|breakfast| +33593|AAAAAAAAKDDIAAAA|33593|9|19|53|AM|first|morning|breakfast| +33594|AAAAAAAALDDIAAAA|33594|9|19|54|AM|first|morning|breakfast| +33595|AAAAAAAAMDDIAAAA|33595|9|19|55|AM|first|morning|breakfast| +33596|AAAAAAAANDDIAAAA|33596|9|19|56|AM|first|morning|breakfast| +33597|AAAAAAAAODDIAAAA|33597|9|19|57|AM|first|morning|breakfast| +33598|AAAAAAAAPDDIAAAA|33598|9|19|58|AM|first|morning|breakfast| +33599|AAAAAAAAAEDIAAAA|33599|9|19|59|AM|first|morning|breakfast| +33600|AAAAAAAABEDIAAAA|33600|9|20|0|AM|first|morning|breakfast| +33601|AAAAAAAACEDIAAAA|33601|9|20|1|AM|first|morning|breakfast| +33602|AAAAAAAADEDIAAAA|33602|9|20|2|AM|first|morning|breakfast| +33603|AAAAAAAAEEDIAAAA|33603|9|20|3|AM|first|morning|breakfast| +33604|AAAAAAAAFEDIAAAA|33604|9|20|4|AM|first|morning|breakfast| +33605|AAAAAAAAGEDIAAAA|33605|9|20|5|AM|first|morning|breakfast| +33606|AAAAAAAAHEDIAAAA|33606|9|20|6|AM|first|morning|breakfast| +33607|AAAAAAAAIEDIAAAA|33607|9|20|7|AM|first|morning|breakfast| +33608|AAAAAAAAJEDIAAAA|33608|9|20|8|AM|first|morning|breakfast| +33609|AAAAAAAAKEDIAAAA|33609|9|20|9|AM|first|morning|breakfast| +33610|AAAAAAAALEDIAAAA|33610|9|20|10|AM|first|morning|breakfast| +33611|AAAAAAAAMEDIAAAA|33611|9|20|11|AM|first|morning|breakfast| +33612|AAAAAAAANEDIAAAA|33612|9|20|12|AM|first|morning|breakfast| +33613|AAAAAAAAOEDIAAAA|33613|9|20|13|AM|first|morning|breakfast| +33614|AAAAAAAAPEDIAAAA|33614|9|20|14|AM|first|morning|breakfast| +33615|AAAAAAAAAFDIAAAA|33615|9|20|15|AM|first|morning|breakfast| +33616|AAAAAAAABFDIAAAA|33616|9|20|16|AM|first|morning|breakfast| +33617|AAAAAAAACFDIAAAA|33617|9|20|17|AM|first|morning|breakfast| +33618|AAAAAAAADFDIAAAA|33618|9|20|18|AM|first|morning|breakfast| +33619|AAAAAAAAEFDIAAAA|33619|9|20|19|AM|first|morning|breakfast| +33620|AAAAAAAAFFDIAAAA|33620|9|20|20|AM|first|morning|breakfast| +33621|AAAAAAAAGFDIAAAA|33621|9|20|21|AM|first|morning|breakfast| +33622|AAAAAAAAHFDIAAAA|33622|9|20|22|AM|first|morning|breakfast| +33623|AAAAAAAAIFDIAAAA|33623|9|20|23|AM|first|morning|breakfast| +33624|AAAAAAAAJFDIAAAA|33624|9|20|24|AM|first|morning|breakfast| +33625|AAAAAAAAKFDIAAAA|33625|9|20|25|AM|first|morning|breakfast| +33626|AAAAAAAALFDIAAAA|33626|9|20|26|AM|first|morning|breakfast| +33627|AAAAAAAAMFDIAAAA|33627|9|20|27|AM|first|morning|breakfast| +33628|AAAAAAAANFDIAAAA|33628|9|20|28|AM|first|morning|breakfast| +33629|AAAAAAAAOFDIAAAA|33629|9|20|29|AM|first|morning|breakfast| +33630|AAAAAAAAPFDIAAAA|33630|9|20|30|AM|first|morning|breakfast| +33631|AAAAAAAAAGDIAAAA|33631|9|20|31|AM|first|morning|breakfast| +33632|AAAAAAAABGDIAAAA|33632|9|20|32|AM|first|morning|breakfast| +33633|AAAAAAAACGDIAAAA|33633|9|20|33|AM|first|morning|breakfast| +33634|AAAAAAAADGDIAAAA|33634|9|20|34|AM|first|morning|breakfast| +33635|AAAAAAAAEGDIAAAA|33635|9|20|35|AM|first|morning|breakfast| +33636|AAAAAAAAFGDIAAAA|33636|9|20|36|AM|first|morning|breakfast| +33637|AAAAAAAAGGDIAAAA|33637|9|20|37|AM|first|morning|breakfast| +33638|AAAAAAAAHGDIAAAA|33638|9|20|38|AM|first|morning|breakfast| +33639|AAAAAAAAIGDIAAAA|33639|9|20|39|AM|first|morning|breakfast| +33640|AAAAAAAAJGDIAAAA|33640|9|20|40|AM|first|morning|breakfast| +33641|AAAAAAAAKGDIAAAA|33641|9|20|41|AM|first|morning|breakfast| +33642|AAAAAAAALGDIAAAA|33642|9|20|42|AM|first|morning|breakfast| +33643|AAAAAAAAMGDIAAAA|33643|9|20|43|AM|first|morning|breakfast| +33644|AAAAAAAANGDIAAAA|33644|9|20|44|AM|first|morning|breakfast| +33645|AAAAAAAAOGDIAAAA|33645|9|20|45|AM|first|morning|breakfast| +33646|AAAAAAAAPGDIAAAA|33646|9|20|46|AM|first|morning|breakfast| +33647|AAAAAAAAAHDIAAAA|33647|9|20|47|AM|first|morning|breakfast| +33648|AAAAAAAABHDIAAAA|33648|9|20|48|AM|first|morning|breakfast| +33649|AAAAAAAACHDIAAAA|33649|9|20|49|AM|first|morning|breakfast| +33650|AAAAAAAADHDIAAAA|33650|9|20|50|AM|first|morning|breakfast| +33651|AAAAAAAAEHDIAAAA|33651|9|20|51|AM|first|morning|breakfast| +33652|AAAAAAAAFHDIAAAA|33652|9|20|52|AM|first|morning|breakfast| +33653|AAAAAAAAGHDIAAAA|33653|9|20|53|AM|first|morning|breakfast| +33654|AAAAAAAAHHDIAAAA|33654|9|20|54|AM|first|morning|breakfast| +33655|AAAAAAAAIHDIAAAA|33655|9|20|55|AM|first|morning|breakfast| +33656|AAAAAAAAJHDIAAAA|33656|9|20|56|AM|first|morning|breakfast| +33657|AAAAAAAAKHDIAAAA|33657|9|20|57|AM|first|morning|breakfast| +33658|AAAAAAAALHDIAAAA|33658|9|20|58|AM|first|morning|breakfast| +33659|AAAAAAAAMHDIAAAA|33659|9|20|59|AM|first|morning|breakfast| +33660|AAAAAAAANHDIAAAA|33660|9|21|0|AM|first|morning|breakfast| +33661|AAAAAAAAOHDIAAAA|33661|9|21|1|AM|first|morning|breakfast| +33662|AAAAAAAAPHDIAAAA|33662|9|21|2|AM|first|morning|breakfast| +33663|AAAAAAAAAIDIAAAA|33663|9|21|3|AM|first|morning|breakfast| +33664|AAAAAAAABIDIAAAA|33664|9|21|4|AM|first|morning|breakfast| +33665|AAAAAAAACIDIAAAA|33665|9|21|5|AM|first|morning|breakfast| +33666|AAAAAAAADIDIAAAA|33666|9|21|6|AM|first|morning|breakfast| +33667|AAAAAAAAEIDIAAAA|33667|9|21|7|AM|first|morning|breakfast| +33668|AAAAAAAAFIDIAAAA|33668|9|21|8|AM|first|morning|breakfast| +33669|AAAAAAAAGIDIAAAA|33669|9|21|9|AM|first|morning|breakfast| +33670|AAAAAAAAHIDIAAAA|33670|9|21|10|AM|first|morning|breakfast| +33671|AAAAAAAAIIDIAAAA|33671|9|21|11|AM|first|morning|breakfast| +33672|AAAAAAAAJIDIAAAA|33672|9|21|12|AM|first|morning|breakfast| +33673|AAAAAAAAKIDIAAAA|33673|9|21|13|AM|first|morning|breakfast| +33674|AAAAAAAALIDIAAAA|33674|9|21|14|AM|first|morning|breakfast| +33675|AAAAAAAAMIDIAAAA|33675|9|21|15|AM|first|morning|breakfast| +33676|AAAAAAAANIDIAAAA|33676|9|21|16|AM|first|morning|breakfast| +33677|AAAAAAAAOIDIAAAA|33677|9|21|17|AM|first|morning|breakfast| +33678|AAAAAAAAPIDIAAAA|33678|9|21|18|AM|first|morning|breakfast| +33679|AAAAAAAAAJDIAAAA|33679|9|21|19|AM|first|morning|breakfast| +33680|AAAAAAAABJDIAAAA|33680|9|21|20|AM|first|morning|breakfast| +33681|AAAAAAAACJDIAAAA|33681|9|21|21|AM|first|morning|breakfast| +33682|AAAAAAAADJDIAAAA|33682|9|21|22|AM|first|morning|breakfast| +33683|AAAAAAAAEJDIAAAA|33683|9|21|23|AM|first|morning|breakfast| +33684|AAAAAAAAFJDIAAAA|33684|9|21|24|AM|first|morning|breakfast| +33685|AAAAAAAAGJDIAAAA|33685|9|21|25|AM|first|morning|breakfast| +33686|AAAAAAAAHJDIAAAA|33686|9|21|26|AM|first|morning|breakfast| +33687|AAAAAAAAIJDIAAAA|33687|9|21|27|AM|first|morning|breakfast| +33688|AAAAAAAAJJDIAAAA|33688|9|21|28|AM|first|morning|breakfast| +33689|AAAAAAAAKJDIAAAA|33689|9|21|29|AM|first|morning|breakfast| +33690|AAAAAAAALJDIAAAA|33690|9|21|30|AM|first|morning|breakfast| +33691|AAAAAAAAMJDIAAAA|33691|9|21|31|AM|first|morning|breakfast| +33692|AAAAAAAANJDIAAAA|33692|9|21|32|AM|first|morning|breakfast| +33693|AAAAAAAAOJDIAAAA|33693|9|21|33|AM|first|morning|breakfast| +33694|AAAAAAAAPJDIAAAA|33694|9|21|34|AM|first|morning|breakfast| +33695|AAAAAAAAAKDIAAAA|33695|9|21|35|AM|first|morning|breakfast| +33696|AAAAAAAABKDIAAAA|33696|9|21|36|AM|first|morning|breakfast| +33697|AAAAAAAACKDIAAAA|33697|9|21|37|AM|first|morning|breakfast| +33698|AAAAAAAADKDIAAAA|33698|9|21|38|AM|first|morning|breakfast| +33699|AAAAAAAAEKDIAAAA|33699|9|21|39|AM|first|morning|breakfast| +33700|AAAAAAAAFKDIAAAA|33700|9|21|40|AM|first|morning|breakfast| +33701|AAAAAAAAGKDIAAAA|33701|9|21|41|AM|first|morning|breakfast| +33702|AAAAAAAAHKDIAAAA|33702|9|21|42|AM|first|morning|breakfast| +33703|AAAAAAAAIKDIAAAA|33703|9|21|43|AM|first|morning|breakfast| +33704|AAAAAAAAJKDIAAAA|33704|9|21|44|AM|first|morning|breakfast| +33705|AAAAAAAAKKDIAAAA|33705|9|21|45|AM|first|morning|breakfast| +33706|AAAAAAAALKDIAAAA|33706|9|21|46|AM|first|morning|breakfast| +33707|AAAAAAAAMKDIAAAA|33707|9|21|47|AM|first|morning|breakfast| +33708|AAAAAAAANKDIAAAA|33708|9|21|48|AM|first|morning|breakfast| +33709|AAAAAAAAOKDIAAAA|33709|9|21|49|AM|first|morning|breakfast| +33710|AAAAAAAAPKDIAAAA|33710|9|21|50|AM|first|morning|breakfast| +33711|AAAAAAAAALDIAAAA|33711|9|21|51|AM|first|morning|breakfast| +33712|AAAAAAAABLDIAAAA|33712|9|21|52|AM|first|morning|breakfast| +33713|AAAAAAAACLDIAAAA|33713|9|21|53|AM|first|morning|breakfast| +33714|AAAAAAAADLDIAAAA|33714|9|21|54|AM|first|morning|breakfast| +33715|AAAAAAAAELDIAAAA|33715|9|21|55|AM|first|morning|breakfast| +33716|AAAAAAAAFLDIAAAA|33716|9|21|56|AM|first|morning|breakfast| +33717|AAAAAAAAGLDIAAAA|33717|9|21|57|AM|first|morning|breakfast| +33718|AAAAAAAAHLDIAAAA|33718|9|21|58|AM|first|morning|breakfast| +33719|AAAAAAAAILDIAAAA|33719|9|21|59|AM|first|morning|breakfast| +33720|AAAAAAAAJLDIAAAA|33720|9|22|0|AM|first|morning|breakfast| +33721|AAAAAAAAKLDIAAAA|33721|9|22|1|AM|first|morning|breakfast| +33722|AAAAAAAALLDIAAAA|33722|9|22|2|AM|first|morning|breakfast| +33723|AAAAAAAAMLDIAAAA|33723|9|22|3|AM|first|morning|breakfast| +33724|AAAAAAAANLDIAAAA|33724|9|22|4|AM|first|morning|breakfast| +33725|AAAAAAAAOLDIAAAA|33725|9|22|5|AM|first|morning|breakfast| +33726|AAAAAAAAPLDIAAAA|33726|9|22|6|AM|first|morning|breakfast| +33727|AAAAAAAAAMDIAAAA|33727|9|22|7|AM|first|morning|breakfast| +33728|AAAAAAAABMDIAAAA|33728|9|22|8|AM|first|morning|breakfast| +33729|AAAAAAAACMDIAAAA|33729|9|22|9|AM|first|morning|breakfast| +33730|AAAAAAAADMDIAAAA|33730|9|22|10|AM|first|morning|breakfast| +33731|AAAAAAAAEMDIAAAA|33731|9|22|11|AM|first|morning|breakfast| +33732|AAAAAAAAFMDIAAAA|33732|9|22|12|AM|first|morning|breakfast| +33733|AAAAAAAAGMDIAAAA|33733|9|22|13|AM|first|morning|breakfast| +33734|AAAAAAAAHMDIAAAA|33734|9|22|14|AM|first|morning|breakfast| +33735|AAAAAAAAIMDIAAAA|33735|9|22|15|AM|first|morning|breakfast| +33736|AAAAAAAAJMDIAAAA|33736|9|22|16|AM|first|morning|breakfast| +33737|AAAAAAAAKMDIAAAA|33737|9|22|17|AM|first|morning|breakfast| +33738|AAAAAAAALMDIAAAA|33738|9|22|18|AM|first|morning|breakfast| +33739|AAAAAAAAMMDIAAAA|33739|9|22|19|AM|first|morning|breakfast| +33740|AAAAAAAANMDIAAAA|33740|9|22|20|AM|first|morning|breakfast| +33741|AAAAAAAAOMDIAAAA|33741|9|22|21|AM|first|morning|breakfast| +33742|AAAAAAAAPMDIAAAA|33742|9|22|22|AM|first|morning|breakfast| +33743|AAAAAAAAANDIAAAA|33743|9|22|23|AM|first|morning|breakfast| +33744|AAAAAAAABNDIAAAA|33744|9|22|24|AM|first|morning|breakfast| +33745|AAAAAAAACNDIAAAA|33745|9|22|25|AM|first|morning|breakfast| +33746|AAAAAAAADNDIAAAA|33746|9|22|26|AM|first|morning|breakfast| +33747|AAAAAAAAENDIAAAA|33747|9|22|27|AM|first|morning|breakfast| +33748|AAAAAAAAFNDIAAAA|33748|9|22|28|AM|first|morning|breakfast| +33749|AAAAAAAAGNDIAAAA|33749|9|22|29|AM|first|morning|breakfast| +33750|AAAAAAAAHNDIAAAA|33750|9|22|30|AM|first|morning|breakfast| +33751|AAAAAAAAINDIAAAA|33751|9|22|31|AM|first|morning|breakfast| +33752|AAAAAAAAJNDIAAAA|33752|9|22|32|AM|first|morning|breakfast| +33753|AAAAAAAAKNDIAAAA|33753|9|22|33|AM|first|morning|breakfast| +33754|AAAAAAAALNDIAAAA|33754|9|22|34|AM|first|morning|breakfast| +33755|AAAAAAAAMNDIAAAA|33755|9|22|35|AM|first|morning|breakfast| +33756|AAAAAAAANNDIAAAA|33756|9|22|36|AM|first|morning|breakfast| +33757|AAAAAAAAONDIAAAA|33757|9|22|37|AM|first|morning|breakfast| +33758|AAAAAAAAPNDIAAAA|33758|9|22|38|AM|first|morning|breakfast| +33759|AAAAAAAAAODIAAAA|33759|9|22|39|AM|first|morning|breakfast| +33760|AAAAAAAABODIAAAA|33760|9|22|40|AM|first|morning|breakfast| +33761|AAAAAAAACODIAAAA|33761|9|22|41|AM|first|morning|breakfast| +33762|AAAAAAAADODIAAAA|33762|9|22|42|AM|first|morning|breakfast| +33763|AAAAAAAAEODIAAAA|33763|9|22|43|AM|first|morning|breakfast| +33764|AAAAAAAAFODIAAAA|33764|9|22|44|AM|first|morning|breakfast| +33765|AAAAAAAAGODIAAAA|33765|9|22|45|AM|first|morning|breakfast| +33766|AAAAAAAAHODIAAAA|33766|9|22|46|AM|first|morning|breakfast| +33767|AAAAAAAAIODIAAAA|33767|9|22|47|AM|first|morning|breakfast| +33768|AAAAAAAAJODIAAAA|33768|9|22|48|AM|first|morning|breakfast| +33769|AAAAAAAAKODIAAAA|33769|9|22|49|AM|first|morning|breakfast| +33770|AAAAAAAALODIAAAA|33770|9|22|50|AM|first|morning|breakfast| +33771|AAAAAAAAMODIAAAA|33771|9|22|51|AM|first|morning|breakfast| +33772|AAAAAAAANODIAAAA|33772|9|22|52|AM|first|morning|breakfast| +33773|AAAAAAAAOODIAAAA|33773|9|22|53|AM|first|morning|breakfast| +33774|AAAAAAAAPODIAAAA|33774|9|22|54|AM|first|morning|breakfast| +33775|AAAAAAAAAPDIAAAA|33775|9|22|55|AM|first|morning|breakfast| +33776|AAAAAAAABPDIAAAA|33776|9|22|56|AM|first|morning|breakfast| +33777|AAAAAAAACPDIAAAA|33777|9|22|57|AM|first|morning|breakfast| +33778|AAAAAAAADPDIAAAA|33778|9|22|58|AM|first|morning|breakfast| +33779|AAAAAAAAEPDIAAAA|33779|9|22|59|AM|first|morning|breakfast| +33780|AAAAAAAAFPDIAAAA|33780|9|23|0|AM|first|morning|breakfast| +33781|AAAAAAAAGPDIAAAA|33781|9|23|1|AM|first|morning|breakfast| +33782|AAAAAAAAHPDIAAAA|33782|9|23|2|AM|first|morning|breakfast| +33783|AAAAAAAAIPDIAAAA|33783|9|23|3|AM|first|morning|breakfast| +33784|AAAAAAAAJPDIAAAA|33784|9|23|4|AM|first|morning|breakfast| +33785|AAAAAAAAKPDIAAAA|33785|9|23|5|AM|first|morning|breakfast| +33786|AAAAAAAALPDIAAAA|33786|9|23|6|AM|first|morning|breakfast| +33787|AAAAAAAAMPDIAAAA|33787|9|23|7|AM|first|morning|breakfast| +33788|AAAAAAAANPDIAAAA|33788|9|23|8|AM|first|morning|breakfast| +33789|AAAAAAAAOPDIAAAA|33789|9|23|9|AM|first|morning|breakfast| +33790|AAAAAAAAPPDIAAAA|33790|9|23|10|AM|first|morning|breakfast| +33791|AAAAAAAAAAEIAAAA|33791|9|23|11|AM|first|morning|breakfast| +33792|AAAAAAAABAEIAAAA|33792|9|23|12|AM|first|morning|breakfast| +33793|AAAAAAAACAEIAAAA|33793|9|23|13|AM|first|morning|breakfast| +33794|AAAAAAAADAEIAAAA|33794|9|23|14|AM|first|morning|breakfast| +33795|AAAAAAAAEAEIAAAA|33795|9|23|15|AM|first|morning|breakfast| +33796|AAAAAAAAFAEIAAAA|33796|9|23|16|AM|first|morning|breakfast| +33797|AAAAAAAAGAEIAAAA|33797|9|23|17|AM|first|morning|breakfast| +33798|AAAAAAAAHAEIAAAA|33798|9|23|18|AM|first|morning|breakfast| +33799|AAAAAAAAIAEIAAAA|33799|9|23|19|AM|first|morning|breakfast| +33800|AAAAAAAAJAEIAAAA|33800|9|23|20|AM|first|morning|breakfast| +33801|AAAAAAAAKAEIAAAA|33801|9|23|21|AM|first|morning|breakfast| +33802|AAAAAAAALAEIAAAA|33802|9|23|22|AM|first|morning|breakfast| +33803|AAAAAAAAMAEIAAAA|33803|9|23|23|AM|first|morning|breakfast| +33804|AAAAAAAANAEIAAAA|33804|9|23|24|AM|first|morning|breakfast| +33805|AAAAAAAAOAEIAAAA|33805|9|23|25|AM|first|morning|breakfast| +33806|AAAAAAAAPAEIAAAA|33806|9|23|26|AM|first|morning|breakfast| +33807|AAAAAAAAABEIAAAA|33807|9|23|27|AM|first|morning|breakfast| +33808|AAAAAAAABBEIAAAA|33808|9|23|28|AM|first|morning|breakfast| +33809|AAAAAAAACBEIAAAA|33809|9|23|29|AM|first|morning|breakfast| +33810|AAAAAAAADBEIAAAA|33810|9|23|30|AM|first|morning|breakfast| +33811|AAAAAAAAEBEIAAAA|33811|9|23|31|AM|first|morning|breakfast| +33812|AAAAAAAAFBEIAAAA|33812|9|23|32|AM|first|morning|breakfast| +33813|AAAAAAAAGBEIAAAA|33813|9|23|33|AM|first|morning|breakfast| +33814|AAAAAAAAHBEIAAAA|33814|9|23|34|AM|first|morning|breakfast| +33815|AAAAAAAAIBEIAAAA|33815|9|23|35|AM|first|morning|breakfast| +33816|AAAAAAAAJBEIAAAA|33816|9|23|36|AM|first|morning|breakfast| +33817|AAAAAAAAKBEIAAAA|33817|9|23|37|AM|first|morning|breakfast| +33818|AAAAAAAALBEIAAAA|33818|9|23|38|AM|first|morning|breakfast| +33819|AAAAAAAAMBEIAAAA|33819|9|23|39|AM|first|morning|breakfast| +33820|AAAAAAAANBEIAAAA|33820|9|23|40|AM|first|morning|breakfast| +33821|AAAAAAAAOBEIAAAA|33821|9|23|41|AM|first|morning|breakfast| +33822|AAAAAAAAPBEIAAAA|33822|9|23|42|AM|first|morning|breakfast| +33823|AAAAAAAAACEIAAAA|33823|9|23|43|AM|first|morning|breakfast| +33824|AAAAAAAABCEIAAAA|33824|9|23|44|AM|first|morning|breakfast| +33825|AAAAAAAACCEIAAAA|33825|9|23|45|AM|first|morning|breakfast| +33826|AAAAAAAADCEIAAAA|33826|9|23|46|AM|first|morning|breakfast| +33827|AAAAAAAAECEIAAAA|33827|9|23|47|AM|first|morning|breakfast| +33828|AAAAAAAAFCEIAAAA|33828|9|23|48|AM|first|morning|breakfast| +33829|AAAAAAAAGCEIAAAA|33829|9|23|49|AM|first|morning|breakfast| +33830|AAAAAAAAHCEIAAAA|33830|9|23|50|AM|first|morning|breakfast| +33831|AAAAAAAAICEIAAAA|33831|9|23|51|AM|first|morning|breakfast| +33832|AAAAAAAAJCEIAAAA|33832|9|23|52|AM|first|morning|breakfast| +33833|AAAAAAAAKCEIAAAA|33833|9|23|53|AM|first|morning|breakfast| +33834|AAAAAAAALCEIAAAA|33834|9|23|54|AM|first|morning|breakfast| +33835|AAAAAAAAMCEIAAAA|33835|9|23|55|AM|first|morning|breakfast| +33836|AAAAAAAANCEIAAAA|33836|9|23|56|AM|first|morning|breakfast| +33837|AAAAAAAAOCEIAAAA|33837|9|23|57|AM|first|morning|breakfast| +33838|AAAAAAAAPCEIAAAA|33838|9|23|58|AM|first|morning|breakfast| +33839|AAAAAAAAADEIAAAA|33839|9|23|59|AM|first|morning|breakfast| +33840|AAAAAAAABDEIAAAA|33840|9|24|0|AM|first|morning|breakfast| +33841|AAAAAAAACDEIAAAA|33841|9|24|1|AM|first|morning|breakfast| +33842|AAAAAAAADDEIAAAA|33842|9|24|2|AM|first|morning|breakfast| +33843|AAAAAAAAEDEIAAAA|33843|9|24|3|AM|first|morning|breakfast| +33844|AAAAAAAAFDEIAAAA|33844|9|24|4|AM|first|morning|breakfast| +33845|AAAAAAAAGDEIAAAA|33845|9|24|5|AM|first|morning|breakfast| +33846|AAAAAAAAHDEIAAAA|33846|9|24|6|AM|first|morning|breakfast| +33847|AAAAAAAAIDEIAAAA|33847|9|24|7|AM|first|morning|breakfast| +33848|AAAAAAAAJDEIAAAA|33848|9|24|8|AM|first|morning|breakfast| +33849|AAAAAAAAKDEIAAAA|33849|9|24|9|AM|first|morning|breakfast| +33850|AAAAAAAALDEIAAAA|33850|9|24|10|AM|first|morning|breakfast| +33851|AAAAAAAAMDEIAAAA|33851|9|24|11|AM|first|morning|breakfast| +33852|AAAAAAAANDEIAAAA|33852|9|24|12|AM|first|morning|breakfast| +33853|AAAAAAAAODEIAAAA|33853|9|24|13|AM|first|morning|breakfast| +33854|AAAAAAAAPDEIAAAA|33854|9|24|14|AM|first|morning|breakfast| +33855|AAAAAAAAAEEIAAAA|33855|9|24|15|AM|first|morning|breakfast| +33856|AAAAAAAABEEIAAAA|33856|9|24|16|AM|first|morning|breakfast| +33857|AAAAAAAACEEIAAAA|33857|9|24|17|AM|first|morning|breakfast| +33858|AAAAAAAADEEIAAAA|33858|9|24|18|AM|first|morning|breakfast| +33859|AAAAAAAAEEEIAAAA|33859|9|24|19|AM|first|morning|breakfast| +33860|AAAAAAAAFEEIAAAA|33860|9|24|20|AM|first|morning|breakfast| +33861|AAAAAAAAGEEIAAAA|33861|9|24|21|AM|first|morning|breakfast| +33862|AAAAAAAAHEEIAAAA|33862|9|24|22|AM|first|morning|breakfast| +33863|AAAAAAAAIEEIAAAA|33863|9|24|23|AM|first|morning|breakfast| +33864|AAAAAAAAJEEIAAAA|33864|9|24|24|AM|first|morning|breakfast| +33865|AAAAAAAAKEEIAAAA|33865|9|24|25|AM|first|morning|breakfast| +33866|AAAAAAAALEEIAAAA|33866|9|24|26|AM|first|morning|breakfast| +33867|AAAAAAAAMEEIAAAA|33867|9|24|27|AM|first|morning|breakfast| +33868|AAAAAAAANEEIAAAA|33868|9|24|28|AM|first|morning|breakfast| +33869|AAAAAAAAOEEIAAAA|33869|9|24|29|AM|first|morning|breakfast| +33870|AAAAAAAAPEEIAAAA|33870|9|24|30|AM|first|morning|breakfast| +33871|AAAAAAAAAFEIAAAA|33871|9|24|31|AM|first|morning|breakfast| +33872|AAAAAAAABFEIAAAA|33872|9|24|32|AM|first|morning|breakfast| +33873|AAAAAAAACFEIAAAA|33873|9|24|33|AM|first|morning|breakfast| +33874|AAAAAAAADFEIAAAA|33874|9|24|34|AM|first|morning|breakfast| +33875|AAAAAAAAEFEIAAAA|33875|9|24|35|AM|first|morning|breakfast| +33876|AAAAAAAAFFEIAAAA|33876|9|24|36|AM|first|morning|breakfast| +33877|AAAAAAAAGFEIAAAA|33877|9|24|37|AM|first|morning|breakfast| +33878|AAAAAAAAHFEIAAAA|33878|9|24|38|AM|first|morning|breakfast| +33879|AAAAAAAAIFEIAAAA|33879|9|24|39|AM|first|morning|breakfast| +33880|AAAAAAAAJFEIAAAA|33880|9|24|40|AM|first|morning|breakfast| +33881|AAAAAAAAKFEIAAAA|33881|9|24|41|AM|first|morning|breakfast| +33882|AAAAAAAALFEIAAAA|33882|9|24|42|AM|first|morning|breakfast| +33883|AAAAAAAAMFEIAAAA|33883|9|24|43|AM|first|morning|breakfast| +33884|AAAAAAAANFEIAAAA|33884|9|24|44|AM|first|morning|breakfast| +33885|AAAAAAAAOFEIAAAA|33885|9|24|45|AM|first|morning|breakfast| +33886|AAAAAAAAPFEIAAAA|33886|9|24|46|AM|first|morning|breakfast| +33887|AAAAAAAAAGEIAAAA|33887|9|24|47|AM|first|morning|breakfast| +33888|AAAAAAAABGEIAAAA|33888|9|24|48|AM|first|morning|breakfast| +33889|AAAAAAAACGEIAAAA|33889|9|24|49|AM|first|morning|breakfast| +33890|AAAAAAAADGEIAAAA|33890|9|24|50|AM|first|morning|breakfast| +33891|AAAAAAAAEGEIAAAA|33891|9|24|51|AM|first|morning|breakfast| +33892|AAAAAAAAFGEIAAAA|33892|9|24|52|AM|first|morning|breakfast| +33893|AAAAAAAAGGEIAAAA|33893|9|24|53|AM|first|morning|breakfast| +33894|AAAAAAAAHGEIAAAA|33894|9|24|54|AM|first|morning|breakfast| +33895|AAAAAAAAIGEIAAAA|33895|9|24|55|AM|first|morning|breakfast| +33896|AAAAAAAAJGEIAAAA|33896|9|24|56|AM|first|morning|breakfast| +33897|AAAAAAAAKGEIAAAA|33897|9|24|57|AM|first|morning|breakfast| +33898|AAAAAAAALGEIAAAA|33898|9|24|58|AM|first|morning|breakfast| +33899|AAAAAAAAMGEIAAAA|33899|9|24|59|AM|first|morning|breakfast| +33900|AAAAAAAANGEIAAAA|33900|9|25|0|AM|first|morning|breakfast| +33901|AAAAAAAAOGEIAAAA|33901|9|25|1|AM|first|morning|breakfast| +33902|AAAAAAAAPGEIAAAA|33902|9|25|2|AM|first|morning|breakfast| +33903|AAAAAAAAAHEIAAAA|33903|9|25|3|AM|first|morning|breakfast| +33904|AAAAAAAABHEIAAAA|33904|9|25|4|AM|first|morning|breakfast| +33905|AAAAAAAACHEIAAAA|33905|9|25|5|AM|first|morning|breakfast| +33906|AAAAAAAADHEIAAAA|33906|9|25|6|AM|first|morning|breakfast| +33907|AAAAAAAAEHEIAAAA|33907|9|25|7|AM|first|morning|breakfast| +33908|AAAAAAAAFHEIAAAA|33908|9|25|8|AM|first|morning|breakfast| +33909|AAAAAAAAGHEIAAAA|33909|9|25|9|AM|first|morning|breakfast| +33910|AAAAAAAAHHEIAAAA|33910|9|25|10|AM|first|morning|breakfast| +33911|AAAAAAAAIHEIAAAA|33911|9|25|11|AM|first|morning|breakfast| +33912|AAAAAAAAJHEIAAAA|33912|9|25|12|AM|first|morning|breakfast| +33913|AAAAAAAAKHEIAAAA|33913|9|25|13|AM|first|morning|breakfast| +33914|AAAAAAAALHEIAAAA|33914|9|25|14|AM|first|morning|breakfast| +33915|AAAAAAAAMHEIAAAA|33915|9|25|15|AM|first|morning|breakfast| +33916|AAAAAAAANHEIAAAA|33916|9|25|16|AM|first|morning|breakfast| +33917|AAAAAAAAOHEIAAAA|33917|9|25|17|AM|first|morning|breakfast| +33918|AAAAAAAAPHEIAAAA|33918|9|25|18|AM|first|morning|breakfast| +33919|AAAAAAAAAIEIAAAA|33919|9|25|19|AM|first|morning|breakfast| +33920|AAAAAAAABIEIAAAA|33920|9|25|20|AM|first|morning|breakfast| +33921|AAAAAAAACIEIAAAA|33921|9|25|21|AM|first|morning|breakfast| +33922|AAAAAAAADIEIAAAA|33922|9|25|22|AM|first|morning|breakfast| +33923|AAAAAAAAEIEIAAAA|33923|9|25|23|AM|first|morning|breakfast| +33924|AAAAAAAAFIEIAAAA|33924|9|25|24|AM|first|morning|breakfast| +33925|AAAAAAAAGIEIAAAA|33925|9|25|25|AM|first|morning|breakfast| +33926|AAAAAAAAHIEIAAAA|33926|9|25|26|AM|first|morning|breakfast| +33927|AAAAAAAAIIEIAAAA|33927|9|25|27|AM|first|morning|breakfast| +33928|AAAAAAAAJIEIAAAA|33928|9|25|28|AM|first|morning|breakfast| +33929|AAAAAAAAKIEIAAAA|33929|9|25|29|AM|first|morning|breakfast| +33930|AAAAAAAALIEIAAAA|33930|9|25|30|AM|first|morning|breakfast| +33931|AAAAAAAAMIEIAAAA|33931|9|25|31|AM|first|morning|breakfast| +33932|AAAAAAAANIEIAAAA|33932|9|25|32|AM|first|morning|breakfast| +33933|AAAAAAAAOIEIAAAA|33933|9|25|33|AM|first|morning|breakfast| +33934|AAAAAAAAPIEIAAAA|33934|9|25|34|AM|first|morning|breakfast| +33935|AAAAAAAAAJEIAAAA|33935|9|25|35|AM|first|morning|breakfast| +33936|AAAAAAAABJEIAAAA|33936|9|25|36|AM|first|morning|breakfast| +33937|AAAAAAAACJEIAAAA|33937|9|25|37|AM|first|morning|breakfast| +33938|AAAAAAAADJEIAAAA|33938|9|25|38|AM|first|morning|breakfast| +33939|AAAAAAAAEJEIAAAA|33939|9|25|39|AM|first|morning|breakfast| +33940|AAAAAAAAFJEIAAAA|33940|9|25|40|AM|first|morning|breakfast| +33941|AAAAAAAAGJEIAAAA|33941|9|25|41|AM|first|morning|breakfast| +33942|AAAAAAAAHJEIAAAA|33942|9|25|42|AM|first|morning|breakfast| +33943|AAAAAAAAIJEIAAAA|33943|9|25|43|AM|first|morning|breakfast| +33944|AAAAAAAAJJEIAAAA|33944|9|25|44|AM|first|morning|breakfast| +33945|AAAAAAAAKJEIAAAA|33945|9|25|45|AM|first|morning|breakfast| +33946|AAAAAAAALJEIAAAA|33946|9|25|46|AM|first|morning|breakfast| +33947|AAAAAAAAMJEIAAAA|33947|9|25|47|AM|first|morning|breakfast| +33948|AAAAAAAANJEIAAAA|33948|9|25|48|AM|first|morning|breakfast| +33949|AAAAAAAAOJEIAAAA|33949|9|25|49|AM|first|morning|breakfast| +33950|AAAAAAAAPJEIAAAA|33950|9|25|50|AM|first|morning|breakfast| +33951|AAAAAAAAAKEIAAAA|33951|9|25|51|AM|first|morning|breakfast| +33952|AAAAAAAABKEIAAAA|33952|9|25|52|AM|first|morning|breakfast| +33953|AAAAAAAACKEIAAAA|33953|9|25|53|AM|first|morning|breakfast| +33954|AAAAAAAADKEIAAAA|33954|9|25|54|AM|first|morning|breakfast| +33955|AAAAAAAAEKEIAAAA|33955|9|25|55|AM|first|morning|breakfast| +33956|AAAAAAAAFKEIAAAA|33956|9|25|56|AM|first|morning|breakfast| +33957|AAAAAAAAGKEIAAAA|33957|9|25|57|AM|first|morning|breakfast| +33958|AAAAAAAAHKEIAAAA|33958|9|25|58|AM|first|morning|breakfast| +33959|AAAAAAAAIKEIAAAA|33959|9|25|59|AM|first|morning|breakfast| +33960|AAAAAAAAJKEIAAAA|33960|9|26|0|AM|first|morning|breakfast| +33961|AAAAAAAAKKEIAAAA|33961|9|26|1|AM|first|morning|breakfast| +33962|AAAAAAAALKEIAAAA|33962|9|26|2|AM|first|morning|breakfast| +33963|AAAAAAAAMKEIAAAA|33963|9|26|3|AM|first|morning|breakfast| +33964|AAAAAAAANKEIAAAA|33964|9|26|4|AM|first|morning|breakfast| +33965|AAAAAAAAOKEIAAAA|33965|9|26|5|AM|first|morning|breakfast| +33966|AAAAAAAAPKEIAAAA|33966|9|26|6|AM|first|morning|breakfast| +33967|AAAAAAAAALEIAAAA|33967|9|26|7|AM|first|morning|breakfast| +33968|AAAAAAAABLEIAAAA|33968|9|26|8|AM|first|morning|breakfast| +33969|AAAAAAAACLEIAAAA|33969|9|26|9|AM|first|morning|breakfast| +33970|AAAAAAAADLEIAAAA|33970|9|26|10|AM|first|morning|breakfast| +33971|AAAAAAAAELEIAAAA|33971|9|26|11|AM|first|morning|breakfast| +33972|AAAAAAAAFLEIAAAA|33972|9|26|12|AM|first|morning|breakfast| +33973|AAAAAAAAGLEIAAAA|33973|9|26|13|AM|first|morning|breakfast| +33974|AAAAAAAAHLEIAAAA|33974|9|26|14|AM|first|morning|breakfast| +33975|AAAAAAAAILEIAAAA|33975|9|26|15|AM|first|morning|breakfast| +33976|AAAAAAAAJLEIAAAA|33976|9|26|16|AM|first|morning|breakfast| +33977|AAAAAAAAKLEIAAAA|33977|9|26|17|AM|first|morning|breakfast| +33978|AAAAAAAALLEIAAAA|33978|9|26|18|AM|first|morning|breakfast| +33979|AAAAAAAAMLEIAAAA|33979|9|26|19|AM|first|morning|breakfast| +33980|AAAAAAAANLEIAAAA|33980|9|26|20|AM|first|morning|breakfast| +33981|AAAAAAAAOLEIAAAA|33981|9|26|21|AM|first|morning|breakfast| +33982|AAAAAAAAPLEIAAAA|33982|9|26|22|AM|first|morning|breakfast| +33983|AAAAAAAAAMEIAAAA|33983|9|26|23|AM|first|morning|breakfast| +33984|AAAAAAAABMEIAAAA|33984|9|26|24|AM|first|morning|breakfast| +33985|AAAAAAAACMEIAAAA|33985|9|26|25|AM|first|morning|breakfast| +33986|AAAAAAAADMEIAAAA|33986|9|26|26|AM|first|morning|breakfast| +33987|AAAAAAAAEMEIAAAA|33987|9|26|27|AM|first|morning|breakfast| +33988|AAAAAAAAFMEIAAAA|33988|9|26|28|AM|first|morning|breakfast| +33989|AAAAAAAAGMEIAAAA|33989|9|26|29|AM|first|morning|breakfast| +33990|AAAAAAAAHMEIAAAA|33990|9|26|30|AM|first|morning|breakfast| +33991|AAAAAAAAIMEIAAAA|33991|9|26|31|AM|first|morning|breakfast| +33992|AAAAAAAAJMEIAAAA|33992|9|26|32|AM|first|morning|breakfast| +33993|AAAAAAAAKMEIAAAA|33993|9|26|33|AM|first|morning|breakfast| +33994|AAAAAAAALMEIAAAA|33994|9|26|34|AM|first|morning|breakfast| +33995|AAAAAAAAMMEIAAAA|33995|9|26|35|AM|first|morning|breakfast| +33996|AAAAAAAANMEIAAAA|33996|9|26|36|AM|first|morning|breakfast| +33997|AAAAAAAAOMEIAAAA|33997|9|26|37|AM|first|morning|breakfast| +33998|AAAAAAAAPMEIAAAA|33998|9|26|38|AM|first|morning|breakfast| +33999|AAAAAAAAANEIAAAA|33999|9|26|39|AM|first|morning|breakfast| +34000|AAAAAAAABNEIAAAA|34000|9|26|40|AM|first|morning|breakfast| +34001|AAAAAAAACNEIAAAA|34001|9|26|41|AM|first|morning|breakfast| +34002|AAAAAAAADNEIAAAA|34002|9|26|42|AM|first|morning|breakfast| +34003|AAAAAAAAENEIAAAA|34003|9|26|43|AM|first|morning|breakfast| +34004|AAAAAAAAFNEIAAAA|34004|9|26|44|AM|first|morning|breakfast| +34005|AAAAAAAAGNEIAAAA|34005|9|26|45|AM|first|morning|breakfast| +34006|AAAAAAAAHNEIAAAA|34006|9|26|46|AM|first|morning|breakfast| +34007|AAAAAAAAINEIAAAA|34007|9|26|47|AM|first|morning|breakfast| +34008|AAAAAAAAJNEIAAAA|34008|9|26|48|AM|first|morning|breakfast| +34009|AAAAAAAAKNEIAAAA|34009|9|26|49|AM|first|morning|breakfast| +34010|AAAAAAAALNEIAAAA|34010|9|26|50|AM|first|morning|breakfast| +34011|AAAAAAAAMNEIAAAA|34011|9|26|51|AM|first|morning|breakfast| +34012|AAAAAAAANNEIAAAA|34012|9|26|52|AM|first|morning|breakfast| +34013|AAAAAAAAONEIAAAA|34013|9|26|53|AM|first|morning|breakfast| +34014|AAAAAAAAPNEIAAAA|34014|9|26|54|AM|first|morning|breakfast| +34015|AAAAAAAAAOEIAAAA|34015|9|26|55|AM|first|morning|breakfast| +34016|AAAAAAAABOEIAAAA|34016|9|26|56|AM|first|morning|breakfast| +34017|AAAAAAAACOEIAAAA|34017|9|26|57|AM|first|morning|breakfast| +34018|AAAAAAAADOEIAAAA|34018|9|26|58|AM|first|morning|breakfast| +34019|AAAAAAAAEOEIAAAA|34019|9|26|59|AM|first|morning|breakfast| +34020|AAAAAAAAFOEIAAAA|34020|9|27|0|AM|first|morning|breakfast| +34021|AAAAAAAAGOEIAAAA|34021|9|27|1|AM|first|morning|breakfast| +34022|AAAAAAAAHOEIAAAA|34022|9|27|2|AM|first|morning|breakfast| +34023|AAAAAAAAIOEIAAAA|34023|9|27|3|AM|first|morning|breakfast| +34024|AAAAAAAAJOEIAAAA|34024|9|27|4|AM|first|morning|breakfast| +34025|AAAAAAAAKOEIAAAA|34025|9|27|5|AM|first|morning|breakfast| +34026|AAAAAAAALOEIAAAA|34026|9|27|6|AM|first|morning|breakfast| +34027|AAAAAAAAMOEIAAAA|34027|9|27|7|AM|first|morning|breakfast| +34028|AAAAAAAANOEIAAAA|34028|9|27|8|AM|first|morning|breakfast| +34029|AAAAAAAAOOEIAAAA|34029|9|27|9|AM|first|morning|breakfast| +34030|AAAAAAAAPOEIAAAA|34030|9|27|10|AM|first|morning|breakfast| +34031|AAAAAAAAAPEIAAAA|34031|9|27|11|AM|first|morning|breakfast| +34032|AAAAAAAABPEIAAAA|34032|9|27|12|AM|first|morning|breakfast| +34033|AAAAAAAACPEIAAAA|34033|9|27|13|AM|first|morning|breakfast| +34034|AAAAAAAADPEIAAAA|34034|9|27|14|AM|first|morning|breakfast| +34035|AAAAAAAAEPEIAAAA|34035|9|27|15|AM|first|morning|breakfast| +34036|AAAAAAAAFPEIAAAA|34036|9|27|16|AM|first|morning|breakfast| +34037|AAAAAAAAGPEIAAAA|34037|9|27|17|AM|first|morning|breakfast| +34038|AAAAAAAAHPEIAAAA|34038|9|27|18|AM|first|morning|breakfast| +34039|AAAAAAAAIPEIAAAA|34039|9|27|19|AM|first|morning|breakfast| +34040|AAAAAAAAJPEIAAAA|34040|9|27|20|AM|first|morning|breakfast| +34041|AAAAAAAAKPEIAAAA|34041|9|27|21|AM|first|morning|breakfast| +34042|AAAAAAAALPEIAAAA|34042|9|27|22|AM|first|morning|breakfast| +34043|AAAAAAAAMPEIAAAA|34043|9|27|23|AM|first|morning|breakfast| +34044|AAAAAAAANPEIAAAA|34044|9|27|24|AM|first|morning|breakfast| +34045|AAAAAAAAOPEIAAAA|34045|9|27|25|AM|first|morning|breakfast| +34046|AAAAAAAAPPEIAAAA|34046|9|27|26|AM|first|morning|breakfast| +34047|AAAAAAAAAAFIAAAA|34047|9|27|27|AM|first|morning|breakfast| +34048|AAAAAAAABAFIAAAA|34048|9|27|28|AM|first|morning|breakfast| +34049|AAAAAAAACAFIAAAA|34049|9|27|29|AM|first|morning|breakfast| +34050|AAAAAAAADAFIAAAA|34050|9|27|30|AM|first|morning|breakfast| +34051|AAAAAAAAEAFIAAAA|34051|9|27|31|AM|first|morning|breakfast| +34052|AAAAAAAAFAFIAAAA|34052|9|27|32|AM|first|morning|breakfast| +34053|AAAAAAAAGAFIAAAA|34053|9|27|33|AM|first|morning|breakfast| +34054|AAAAAAAAHAFIAAAA|34054|9|27|34|AM|first|morning|breakfast| +34055|AAAAAAAAIAFIAAAA|34055|9|27|35|AM|first|morning|breakfast| +34056|AAAAAAAAJAFIAAAA|34056|9|27|36|AM|first|morning|breakfast| +34057|AAAAAAAAKAFIAAAA|34057|9|27|37|AM|first|morning|breakfast| +34058|AAAAAAAALAFIAAAA|34058|9|27|38|AM|first|morning|breakfast| +34059|AAAAAAAAMAFIAAAA|34059|9|27|39|AM|first|morning|breakfast| +34060|AAAAAAAANAFIAAAA|34060|9|27|40|AM|first|morning|breakfast| +34061|AAAAAAAAOAFIAAAA|34061|9|27|41|AM|first|morning|breakfast| +34062|AAAAAAAAPAFIAAAA|34062|9|27|42|AM|first|morning|breakfast| +34063|AAAAAAAAABFIAAAA|34063|9|27|43|AM|first|morning|breakfast| +34064|AAAAAAAABBFIAAAA|34064|9|27|44|AM|first|morning|breakfast| +34065|AAAAAAAACBFIAAAA|34065|9|27|45|AM|first|morning|breakfast| +34066|AAAAAAAADBFIAAAA|34066|9|27|46|AM|first|morning|breakfast| +34067|AAAAAAAAEBFIAAAA|34067|9|27|47|AM|first|morning|breakfast| +34068|AAAAAAAAFBFIAAAA|34068|9|27|48|AM|first|morning|breakfast| +34069|AAAAAAAAGBFIAAAA|34069|9|27|49|AM|first|morning|breakfast| +34070|AAAAAAAAHBFIAAAA|34070|9|27|50|AM|first|morning|breakfast| +34071|AAAAAAAAIBFIAAAA|34071|9|27|51|AM|first|morning|breakfast| +34072|AAAAAAAAJBFIAAAA|34072|9|27|52|AM|first|morning|breakfast| +34073|AAAAAAAAKBFIAAAA|34073|9|27|53|AM|first|morning|breakfast| +34074|AAAAAAAALBFIAAAA|34074|9|27|54|AM|first|morning|breakfast| +34075|AAAAAAAAMBFIAAAA|34075|9|27|55|AM|first|morning|breakfast| +34076|AAAAAAAANBFIAAAA|34076|9|27|56|AM|first|morning|breakfast| +34077|AAAAAAAAOBFIAAAA|34077|9|27|57|AM|first|morning|breakfast| +34078|AAAAAAAAPBFIAAAA|34078|9|27|58|AM|first|morning|breakfast| +34079|AAAAAAAAACFIAAAA|34079|9|27|59|AM|first|morning|breakfast| +34080|AAAAAAAABCFIAAAA|34080|9|28|0|AM|first|morning|breakfast| +34081|AAAAAAAACCFIAAAA|34081|9|28|1|AM|first|morning|breakfast| +34082|AAAAAAAADCFIAAAA|34082|9|28|2|AM|first|morning|breakfast| +34083|AAAAAAAAECFIAAAA|34083|9|28|3|AM|first|morning|breakfast| +34084|AAAAAAAAFCFIAAAA|34084|9|28|4|AM|first|morning|breakfast| +34085|AAAAAAAAGCFIAAAA|34085|9|28|5|AM|first|morning|breakfast| +34086|AAAAAAAAHCFIAAAA|34086|9|28|6|AM|first|morning|breakfast| +34087|AAAAAAAAICFIAAAA|34087|9|28|7|AM|first|morning|breakfast| +34088|AAAAAAAAJCFIAAAA|34088|9|28|8|AM|first|morning|breakfast| +34089|AAAAAAAAKCFIAAAA|34089|9|28|9|AM|first|morning|breakfast| +34090|AAAAAAAALCFIAAAA|34090|9|28|10|AM|first|morning|breakfast| +34091|AAAAAAAAMCFIAAAA|34091|9|28|11|AM|first|morning|breakfast| +34092|AAAAAAAANCFIAAAA|34092|9|28|12|AM|first|morning|breakfast| +34093|AAAAAAAAOCFIAAAA|34093|9|28|13|AM|first|morning|breakfast| +34094|AAAAAAAAPCFIAAAA|34094|9|28|14|AM|first|morning|breakfast| +34095|AAAAAAAAADFIAAAA|34095|9|28|15|AM|first|morning|breakfast| +34096|AAAAAAAABDFIAAAA|34096|9|28|16|AM|first|morning|breakfast| +34097|AAAAAAAACDFIAAAA|34097|9|28|17|AM|first|morning|breakfast| +34098|AAAAAAAADDFIAAAA|34098|9|28|18|AM|first|morning|breakfast| +34099|AAAAAAAAEDFIAAAA|34099|9|28|19|AM|first|morning|breakfast| +34100|AAAAAAAAFDFIAAAA|34100|9|28|20|AM|first|morning|breakfast| +34101|AAAAAAAAGDFIAAAA|34101|9|28|21|AM|first|morning|breakfast| +34102|AAAAAAAAHDFIAAAA|34102|9|28|22|AM|first|morning|breakfast| +34103|AAAAAAAAIDFIAAAA|34103|9|28|23|AM|first|morning|breakfast| +34104|AAAAAAAAJDFIAAAA|34104|9|28|24|AM|first|morning|breakfast| +34105|AAAAAAAAKDFIAAAA|34105|9|28|25|AM|first|morning|breakfast| +34106|AAAAAAAALDFIAAAA|34106|9|28|26|AM|first|morning|breakfast| +34107|AAAAAAAAMDFIAAAA|34107|9|28|27|AM|first|morning|breakfast| +34108|AAAAAAAANDFIAAAA|34108|9|28|28|AM|first|morning|breakfast| +34109|AAAAAAAAODFIAAAA|34109|9|28|29|AM|first|morning|breakfast| +34110|AAAAAAAAPDFIAAAA|34110|9|28|30|AM|first|morning|breakfast| +34111|AAAAAAAAAEFIAAAA|34111|9|28|31|AM|first|morning|breakfast| +34112|AAAAAAAABEFIAAAA|34112|9|28|32|AM|first|morning|breakfast| +34113|AAAAAAAACEFIAAAA|34113|9|28|33|AM|first|morning|breakfast| +34114|AAAAAAAADEFIAAAA|34114|9|28|34|AM|first|morning|breakfast| +34115|AAAAAAAAEEFIAAAA|34115|9|28|35|AM|first|morning|breakfast| +34116|AAAAAAAAFEFIAAAA|34116|9|28|36|AM|first|morning|breakfast| +34117|AAAAAAAAGEFIAAAA|34117|9|28|37|AM|first|morning|breakfast| +34118|AAAAAAAAHEFIAAAA|34118|9|28|38|AM|first|morning|breakfast| +34119|AAAAAAAAIEFIAAAA|34119|9|28|39|AM|first|morning|breakfast| +34120|AAAAAAAAJEFIAAAA|34120|9|28|40|AM|first|morning|breakfast| +34121|AAAAAAAAKEFIAAAA|34121|9|28|41|AM|first|morning|breakfast| +34122|AAAAAAAALEFIAAAA|34122|9|28|42|AM|first|morning|breakfast| +34123|AAAAAAAAMEFIAAAA|34123|9|28|43|AM|first|morning|breakfast| +34124|AAAAAAAANEFIAAAA|34124|9|28|44|AM|first|morning|breakfast| +34125|AAAAAAAAOEFIAAAA|34125|9|28|45|AM|first|morning|breakfast| +34126|AAAAAAAAPEFIAAAA|34126|9|28|46|AM|first|morning|breakfast| +34127|AAAAAAAAAFFIAAAA|34127|9|28|47|AM|first|morning|breakfast| +34128|AAAAAAAABFFIAAAA|34128|9|28|48|AM|first|morning|breakfast| +34129|AAAAAAAACFFIAAAA|34129|9|28|49|AM|first|morning|breakfast| +34130|AAAAAAAADFFIAAAA|34130|9|28|50|AM|first|morning|breakfast| +34131|AAAAAAAAEFFIAAAA|34131|9|28|51|AM|first|morning|breakfast| +34132|AAAAAAAAFFFIAAAA|34132|9|28|52|AM|first|morning|breakfast| +34133|AAAAAAAAGFFIAAAA|34133|9|28|53|AM|first|morning|breakfast| +34134|AAAAAAAAHFFIAAAA|34134|9|28|54|AM|first|morning|breakfast| +34135|AAAAAAAAIFFIAAAA|34135|9|28|55|AM|first|morning|breakfast| +34136|AAAAAAAAJFFIAAAA|34136|9|28|56|AM|first|morning|breakfast| +34137|AAAAAAAAKFFIAAAA|34137|9|28|57|AM|first|morning|breakfast| +34138|AAAAAAAALFFIAAAA|34138|9|28|58|AM|first|morning|breakfast| +34139|AAAAAAAAMFFIAAAA|34139|9|28|59|AM|first|morning|breakfast| +34140|AAAAAAAANFFIAAAA|34140|9|29|0|AM|first|morning|breakfast| +34141|AAAAAAAAOFFIAAAA|34141|9|29|1|AM|first|morning|breakfast| +34142|AAAAAAAAPFFIAAAA|34142|9|29|2|AM|first|morning|breakfast| +34143|AAAAAAAAAGFIAAAA|34143|9|29|3|AM|first|morning|breakfast| +34144|AAAAAAAABGFIAAAA|34144|9|29|4|AM|first|morning|breakfast| +34145|AAAAAAAACGFIAAAA|34145|9|29|5|AM|first|morning|breakfast| +34146|AAAAAAAADGFIAAAA|34146|9|29|6|AM|first|morning|breakfast| +34147|AAAAAAAAEGFIAAAA|34147|9|29|7|AM|first|morning|breakfast| +34148|AAAAAAAAFGFIAAAA|34148|9|29|8|AM|first|morning|breakfast| +34149|AAAAAAAAGGFIAAAA|34149|9|29|9|AM|first|morning|breakfast| +34150|AAAAAAAAHGFIAAAA|34150|9|29|10|AM|first|morning|breakfast| +34151|AAAAAAAAIGFIAAAA|34151|9|29|11|AM|first|morning|breakfast| +34152|AAAAAAAAJGFIAAAA|34152|9|29|12|AM|first|morning|breakfast| +34153|AAAAAAAAKGFIAAAA|34153|9|29|13|AM|first|morning|breakfast| +34154|AAAAAAAALGFIAAAA|34154|9|29|14|AM|first|morning|breakfast| +34155|AAAAAAAAMGFIAAAA|34155|9|29|15|AM|first|morning|breakfast| +34156|AAAAAAAANGFIAAAA|34156|9|29|16|AM|first|morning|breakfast| +34157|AAAAAAAAOGFIAAAA|34157|9|29|17|AM|first|morning|breakfast| +34158|AAAAAAAAPGFIAAAA|34158|9|29|18|AM|first|morning|breakfast| +34159|AAAAAAAAAHFIAAAA|34159|9|29|19|AM|first|morning|breakfast| +34160|AAAAAAAABHFIAAAA|34160|9|29|20|AM|first|morning|breakfast| +34161|AAAAAAAACHFIAAAA|34161|9|29|21|AM|first|morning|breakfast| +34162|AAAAAAAADHFIAAAA|34162|9|29|22|AM|first|morning|breakfast| +34163|AAAAAAAAEHFIAAAA|34163|9|29|23|AM|first|morning|breakfast| +34164|AAAAAAAAFHFIAAAA|34164|9|29|24|AM|first|morning|breakfast| +34165|AAAAAAAAGHFIAAAA|34165|9|29|25|AM|first|morning|breakfast| +34166|AAAAAAAAHHFIAAAA|34166|9|29|26|AM|first|morning|breakfast| +34167|AAAAAAAAIHFIAAAA|34167|9|29|27|AM|first|morning|breakfast| +34168|AAAAAAAAJHFIAAAA|34168|9|29|28|AM|first|morning|breakfast| +34169|AAAAAAAAKHFIAAAA|34169|9|29|29|AM|first|morning|breakfast| +34170|AAAAAAAALHFIAAAA|34170|9|29|30|AM|first|morning|breakfast| +34171|AAAAAAAAMHFIAAAA|34171|9|29|31|AM|first|morning|breakfast| +34172|AAAAAAAANHFIAAAA|34172|9|29|32|AM|first|morning|breakfast| +34173|AAAAAAAAOHFIAAAA|34173|9|29|33|AM|first|morning|breakfast| +34174|AAAAAAAAPHFIAAAA|34174|9|29|34|AM|first|morning|breakfast| +34175|AAAAAAAAAIFIAAAA|34175|9|29|35|AM|first|morning|breakfast| +34176|AAAAAAAABIFIAAAA|34176|9|29|36|AM|first|morning|breakfast| +34177|AAAAAAAACIFIAAAA|34177|9|29|37|AM|first|morning|breakfast| +34178|AAAAAAAADIFIAAAA|34178|9|29|38|AM|first|morning|breakfast| +34179|AAAAAAAAEIFIAAAA|34179|9|29|39|AM|first|morning|breakfast| +34180|AAAAAAAAFIFIAAAA|34180|9|29|40|AM|first|morning|breakfast| +34181|AAAAAAAAGIFIAAAA|34181|9|29|41|AM|first|morning|breakfast| +34182|AAAAAAAAHIFIAAAA|34182|9|29|42|AM|first|morning|breakfast| +34183|AAAAAAAAIIFIAAAA|34183|9|29|43|AM|first|morning|breakfast| +34184|AAAAAAAAJIFIAAAA|34184|9|29|44|AM|first|morning|breakfast| +34185|AAAAAAAAKIFIAAAA|34185|9|29|45|AM|first|morning|breakfast| +34186|AAAAAAAALIFIAAAA|34186|9|29|46|AM|first|morning|breakfast| +34187|AAAAAAAAMIFIAAAA|34187|9|29|47|AM|first|morning|breakfast| +34188|AAAAAAAANIFIAAAA|34188|9|29|48|AM|first|morning|breakfast| +34189|AAAAAAAAOIFIAAAA|34189|9|29|49|AM|first|morning|breakfast| +34190|AAAAAAAAPIFIAAAA|34190|9|29|50|AM|first|morning|breakfast| +34191|AAAAAAAAAJFIAAAA|34191|9|29|51|AM|first|morning|breakfast| +34192|AAAAAAAABJFIAAAA|34192|9|29|52|AM|first|morning|breakfast| +34193|AAAAAAAACJFIAAAA|34193|9|29|53|AM|first|morning|breakfast| +34194|AAAAAAAADJFIAAAA|34194|9|29|54|AM|first|morning|breakfast| +34195|AAAAAAAAEJFIAAAA|34195|9|29|55|AM|first|morning|breakfast| +34196|AAAAAAAAFJFIAAAA|34196|9|29|56|AM|first|morning|breakfast| +34197|AAAAAAAAGJFIAAAA|34197|9|29|57|AM|first|morning|breakfast| +34198|AAAAAAAAHJFIAAAA|34198|9|29|58|AM|first|morning|breakfast| +34199|AAAAAAAAIJFIAAAA|34199|9|29|59|AM|first|morning|breakfast| +34200|AAAAAAAAJJFIAAAA|34200|9|30|0|AM|first|morning|breakfast| +34201|AAAAAAAAKJFIAAAA|34201|9|30|1|AM|first|morning|breakfast| +34202|AAAAAAAALJFIAAAA|34202|9|30|2|AM|first|morning|breakfast| +34203|AAAAAAAAMJFIAAAA|34203|9|30|3|AM|first|morning|breakfast| +34204|AAAAAAAANJFIAAAA|34204|9|30|4|AM|first|morning|breakfast| +34205|AAAAAAAAOJFIAAAA|34205|9|30|5|AM|first|morning|breakfast| +34206|AAAAAAAAPJFIAAAA|34206|9|30|6|AM|first|morning|breakfast| +34207|AAAAAAAAAKFIAAAA|34207|9|30|7|AM|first|morning|breakfast| +34208|AAAAAAAABKFIAAAA|34208|9|30|8|AM|first|morning|breakfast| +34209|AAAAAAAACKFIAAAA|34209|9|30|9|AM|first|morning|breakfast| +34210|AAAAAAAADKFIAAAA|34210|9|30|10|AM|first|morning|breakfast| +34211|AAAAAAAAEKFIAAAA|34211|9|30|11|AM|first|morning|breakfast| +34212|AAAAAAAAFKFIAAAA|34212|9|30|12|AM|first|morning|breakfast| +34213|AAAAAAAAGKFIAAAA|34213|9|30|13|AM|first|morning|breakfast| +34214|AAAAAAAAHKFIAAAA|34214|9|30|14|AM|first|morning|breakfast| +34215|AAAAAAAAIKFIAAAA|34215|9|30|15|AM|first|morning|breakfast| +34216|AAAAAAAAJKFIAAAA|34216|9|30|16|AM|first|morning|breakfast| +34217|AAAAAAAAKKFIAAAA|34217|9|30|17|AM|first|morning|breakfast| +34218|AAAAAAAALKFIAAAA|34218|9|30|18|AM|first|morning|breakfast| +34219|AAAAAAAAMKFIAAAA|34219|9|30|19|AM|first|morning|breakfast| +34220|AAAAAAAANKFIAAAA|34220|9|30|20|AM|first|morning|breakfast| +34221|AAAAAAAAOKFIAAAA|34221|9|30|21|AM|first|morning|breakfast| +34222|AAAAAAAAPKFIAAAA|34222|9|30|22|AM|first|morning|breakfast| +34223|AAAAAAAAALFIAAAA|34223|9|30|23|AM|first|morning|breakfast| +34224|AAAAAAAABLFIAAAA|34224|9|30|24|AM|first|morning|breakfast| +34225|AAAAAAAACLFIAAAA|34225|9|30|25|AM|first|morning|breakfast| +34226|AAAAAAAADLFIAAAA|34226|9|30|26|AM|first|morning|breakfast| +34227|AAAAAAAAELFIAAAA|34227|9|30|27|AM|first|morning|breakfast| +34228|AAAAAAAAFLFIAAAA|34228|9|30|28|AM|first|morning|breakfast| +34229|AAAAAAAAGLFIAAAA|34229|9|30|29|AM|first|morning|breakfast| +34230|AAAAAAAAHLFIAAAA|34230|9|30|30|AM|first|morning|breakfast| +34231|AAAAAAAAILFIAAAA|34231|9|30|31|AM|first|morning|breakfast| +34232|AAAAAAAAJLFIAAAA|34232|9|30|32|AM|first|morning|breakfast| +34233|AAAAAAAAKLFIAAAA|34233|9|30|33|AM|first|morning|breakfast| +34234|AAAAAAAALLFIAAAA|34234|9|30|34|AM|first|morning|breakfast| +34235|AAAAAAAAMLFIAAAA|34235|9|30|35|AM|first|morning|breakfast| +34236|AAAAAAAANLFIAAAA|34236|9|30|36|AM|first|morning|breakfast| +34237|AAAAAAAAOLFIAAAA|34237|9|30|37|AM|first|morning|breakfast| +34238|AAAAAAAAPLFIAAAA|34238|9|30|38|AM|first|morning|breakfast| +34239|AAAAAAAAAMFIAAAA|34239|9|30|39|AM|first|morning|breakfast| +34240|AAAAAAAABMFIAAAA|34240|9|30|40|AM|first|morning|breakfast| +34241|AAAAAAAACMFIAAAA|34241|9|30|41|AM|first|morning|breakfast| +34242|AAAAAAAADMFIAAAA|34242|9|30|42|AM|first|morning|breakfast| +34243|AAAAAAAAEMFIAAAA|34243|9|30|43|AM|first|morning|breakfast| +34244|AAAAAAAAFMFIAAAA|34244|9|30|44|AM|first|morning|breakfast| +34245|AAAAAAAAGMFIAAAA|34245|9|30|45|AM|first|morning|breakfast| +34246|AAAAAAAAHMFIAAAA|34246|9|30|46|AM|first|morning|breakfast| +34247|AAAAAAAAIMFIAAAA|34247|9|30|47|AM|first|morning|breakfast| +34248|AAAAAAAAJMFIAAAA|34248|9|30|48|AM|first|morning|breakfast| +34249|AAAAAAAAKMFIAAAA|34249|9|30|49|AM|first|morning|breakfast| +34250|AAAAAAAALMFIAAAA|34250|9|30|50|AM|first|morning|breakfast| +34251|AAAAAAAAMMFIAAAA|34251|9|30|51|AM|first|morning|breakfast| +34252|AAAAAAAANMFIAAAA|34252|9|30|52|AM|first|morning|breakfast| +34253|AAAAAAAAOMFIAAAA|34253|9|30|53|AM|first|morning|breakfast| +34254|AAAAAAAAPMFIAAAA|34254|9|30|54|AM|first|morning|breakfast| +34255|AAAAAAAAANFIAAAA|34255|9|30|55|AM|first|morning|breakfast| +34256|AAAAAAAABNFIAAAA|34256|9|30|56|AM|first|morning|breakfast| +34257|AAAAAAAACNFIAAAA|34257|9|30|57|AM|first|morning|breakfast| +34258|AAAAAAAADNFIAAAA|34258|9|30|58|AM|first|morning|breakfast| +34259|AAAAAAAAENFIAAAA|34259|9|30|59|AM|first|morning|breakfast| +34260|AAAAAAAAFNFIAAAA|34260|9|31|0|AM|first|morning|breakfast| +34261|AAAAAAAAGNFIAAAA|34261|9|31|1|AM|first|morning|breakfast| +34262|AAAAAAAAHNFIAAAA|34262|9|31|2|AM|first|morning|breakfast| +34263|AAAAAAAAINFIAAAA|34263|9|31|3|AM|first|morning|breakfast| +34264|AAAAAAAAJNFIAAAA|34264|9|31|4|AM|first|morning|breakfast| +34265|AAAAAAAAKNFIAAAA|34265|9|31|5|AM|first|morning|breakfast| +34266|AAAAAAAALNFIAAAA|34266|9|31|6|AM|first|morning|breakfast| +34267|AAAAAAAAMNFIAAAA|34267|9|31|7|AM|first|morning|breakfast| +34268|AAAAAAAANNFIAAAA|34268|9|31|8|AM|first|morning|breakfast| +34269|AAAAAAAAONFIAAAA|34269|9|31|9|AM|first|morning|breakfast| +34270|AAAAAAAAPNFIAAAA|34270|9|31|10|AM|first|morning|breakfast| +34271|AAAAAAAAAOFIAAAA|34271|9|31|11|AM|first|morning|breakfast| +34272|AAAAAAAABOFIAAAA|34272|9|31|12|AM|first|morning|breakfast| +34273|AAAAAAAACOFIAAAA|34273|9|31|13|AM|first|morning|breakfast| +34274|AAAAAAAADOFIAAAA|34274|9|31|14|AM|first|morning|breakfast| +34275|AAAAAAAAEOFIAAAA|34275|9|31|15|AM|first|morning|breakfast| +34276|AAAAAAAAFOFIAAAA|34276|9|31|16|AM|first|morning|breakfast| +34277|AAAAAAAAGOFIAAAA|34277|9|31|17|AM|first|morning|breakfast| +34278|AAAAAAAAHOFIAAAA|34278|9|31|18|AM|first|morning|breakfast| +34279|AAAAAAAAIOFIAAAA|34279|9|31|19|AM|first|morning|breakfast| +34280|AAAAAAAAJOFIAAAA|34280|9|31|20|AM|first|morning|breakfast| +34281|AAAAAAAAKOFIAAAA|34281|9|31|21|AM|first|morning|breakfast| +34282|AAAAAAAALOFIAAAA|34282|9|31|22|AM|first|morning|breakfast| +34283|AAAAAAAAMOFIAAAA|34283|9|31|23|AM|first|morning|breakfast| +34284|AAAAAAAANOFIAAAA|34284|9|31|24|AM|first|morning|breakfast| +34285|AAAAAAAAOOFIAAAA|34285|9|31|25|AM|first|morning|breakfast| +34286|AAAAAAAAPOFIAAAA|34286|9|31|26|AM|first|morning|breakfast| +34287|AAAAAAAAAPFIAAAA|34287|9|31|27|AM|first|morning|breakfast| +34288|AAAAAAAABPFIAAAA|34288|9|31|28|AM|first|morning|breakfast| +34289|AAAAAAAACPFIAAAA|34289|9|31|29|AM|first|morning|breakfast| +34290|AAAAAAAADPFIAAAA|34290|9|31|30|AM|first|morning|breakfast| +34291|AAAAAAAAEPFIAAAA|34291|9|31|31|AM|first|morning|breakfast| +34292|AAAAAAAAFPFIAAAA|34292|9|31|32|AM|first|morning|breakfast| +34293|AAAAAAAAGPFIAAAA|34293|9|31|33|AM|first|morning|breakfast| +34294|AAAAAAAAHPFIAAAA|34294|9|31|34|AM|first|morning|breakfast| +34295|AAAAAAAAIPFIAAAA|34295|9|31|35|AM|first|morning|breakfast| +34296|AAAAAAAAJPFIAAAA|34296|9|31|36|AM|first|morning|breakfast| +34297|AAAAAAAAKPFIAAAA|34297|9|31|37|AM|first|morning|breakfast| +34298|AAAAAAAALPFIAAAA|34298|9|31|38|AM|first|morning|breakfast| +34299|AAAAAAAAMPFIAAAA|34299|9|31|39|AM|first|morning|breakfast| +34300|AAAAAAAANPFIAAAA|34300|9|31|40|AM|first|morning|breakfast| +34301|AAAAAAAAOPFIAAAA|34301|9|31|41|AM|first|morning|breakfast| +34302|AAAAAAAAPPFIAAAA|34302|9|31|42|AM|first|morning|breakfast| +34303|AAAAAAAAAAGIAAAA|34303|9|31|43|AM|first|morning|breakfast| +34304|AAAAAAAABAGIAAAA|34304|9|31|44|AM|first|morning|breakfast| +34305|AAAAAAAACAGIAAAA|34305|9|31|45|AM|first|morning|breakfast| +34306|AAAAAAAADAGIAAAA|34306|9|31|46|AM|first|morning|breakfast| +34307|AAAAAAAAEAGIAAAA|34307|9|31|47|AM|first|morning|breakfast| +34308|AAAAAAAAFAGIAAAA|34308|9|31|48|AM|first|morning|breakfast| +34309|AAAAAAAAGAGIAAAA|34309|9|31|49|AM|first|morning|breakfast| +34310|AAAAAAAAHAGIAAAA|34310|9|31|50|AM|first|morning|breakfast| +34311|AAAAAAAAIAGIAAAA|34311|9|31|51|AM|first|morning|breakfast| +34312|AAAAAAAAJAGIAAAA|34312|9|31|52|AM|first|morning|breakfast| +34313|AAAAAAAAKAGIAAAA|34313|9|31|53|AM|first|morning|breakfast| +34314|AAAAAAAALAGIAAAA|34314|9|31|54|AM|first|morning|breakfast| +34315|AAAAAAAAMAGIAAAA|34315|9|31|55|AM|first|morning|breakfast| +34316|AAAAAAAANAGIAAAA|34316|9|31|56|AM|first|morning|breakfast| +34317|AAAAAAAAOAGIAAAA|34317|9|31|57|AM|first|morning|breakfast| +34318|AAAAAAAAPAGIAAAA|34318|9|31|58|AM|first|morning|breakfast| +34319|AAAAAAAAABGIAAAA|34319|9|31|59|AM|first|morning|breakfast| +34320|AAAAAAAABBGIAAAA|34320|9|32|0|AM|first|morning|breakfast| +34321|AAAAAAAACBGIAAAA|34321|9|32|1|AM|first|morning|breakfast| +34322|AAAAAAAADBGIAAAA|34322|9|32|2|AM|first|morning|breakfast| +34323|AAAAAAAAEBGIAAAA|34323|9|32|3|AM|first|morning|breakfast| +34324|AAAAAAAAFBGIAAAA|34324|9|32|4|AM|first|morning|breakfast| +34325|AAAAAAAAGBGIAAAA|34325|9|32|5|AM|first|morning|breakfast| +34326|AAAAAAAAHBGIAAAA|34326|9|32|6|AM|first|morning|breakfast| +34327|AAAAAAAAIBGIAAAA|34327|9|32|7|AM|first|morning|breakfast| +34328|AAAAAAAAJBGIAAAA|34328|9|32|8|AM|first|morning|breakfast| +34329|AAAAAAAAKBGIAAAA|34329|9|32|9|AM|first|morning|breakfast| +34330|AAAAAAAALBGIAAAA|34330|9|32|10|AM|first|morning|breakfast| +34331|AAAAAAAAMBGIAAAA|34331|9|32|11|AM|first|morning|breakfast| +34332|AAAAAAAANBGIAAAA|34332|9|32|12|AM|first|morning|breakfast| +34333|AAAAAAAAOBGIAAAA|34333|9|32|13|AM|first|morning|breakfast| +34334|AAAAAAAAPBGIAAAA|34334|9|32|14|AM|first|morning|breakfast| +34335|AAAAAAAAACGIAAAA|34335|9|32|15|AM|first|morning|breakfast| +34336|AAAAAAAABCGIAAAA|34336|9|32|16|AM|first|morning|breakfast| +34337|AAAAAAAACCGIAAAA|34337|9|32|17|AM|first|morning|breakfast| +34338|AAAAAAAADCGIAAAA|34338|9|32|18|AM|first|morning|breakfast| +34339|AAAAAAAAECGIAAAA|34339|9|32|19|AM|first|morning|breakfast| +34340|AAAAAAAAFCGIAAAA|34340|9|32|20|AM|first|morning|breakfast| +34341|AAAAAAAAGCGIAAAA|34341|9|32|21|AM|first|morning|breakfast| +34342|AAAAAAAAHCGIAAAA|34342|9|32|22|AM|first|morning|breakfast| +34343|AAAAAAAAICGIAAAA|34343|9|32|23|AM|first|morning|breakfast| +34344|AAAAAAAAJCGIAAAA|34344|9|32|24|AM|first|morning|breakfast| +34345|AAAAAAAAKCGIAAAA|34345|9|32|25|AM|first|morning|breakfast| +34346|AAAAAAAALCGIAAAA|34346|9|32|26|AM|first|morning|breakfast| +34347|AAAAAAAAMCGIAAAA|34347|9|32|27|AM|first|morning|breakfast| +34348|AAAAAAAANCGIAAAA|34348|9|32|28|AM|first|morning|breakfast| +34349|AAAAAAAAOCGIAAAA|34349|9|32|29|AM|first|morning|breakfast| +34350|AAAAAAAAPCGIAAAA|34350|9|32|30|AM|first|morning|breakfast| +34351|AAAAAAAAADGIAAAA|34351|9|32|31|AM|first|morning|breakfast| +34352|AAAAAAAABDGIAAAA|34352|9|32|32|AM|first|morning|breakfast| +34353|AAAAAAAACDGIAAAA|34353|9|32|33|AM|first|morning|breakfast| +34354|AAAAAAAADDGIAAAA|34354|9|32|34|AM|first|morning|breakfast| +34355|AAAAAAAAEDGIAAAA|34355|9|32|35|AM|first|morning|breakfast| +34356|AAAAAAAAFDGIAAAA|34356|9|32|36|AM|first|morning|breakfast| +34357|AAAAAAAAGDGIAAAA|34357|9|32|37|AM|first|morning|breakfast| +34358|AAAAAAAAHDGIAAAA|34358|9|32|38|AM|first|morning|breakfast| +34359|AAAAAAAAIDGIAAAA|34359|9|32|39|AM|first|morning|breakfast| +34360|AAAAAAAAJDGIAAAA|34360|9|32|40|AM|first|morning|breakfast| +34361|AAAAAAAAKDGIAAAA|34361|9|32|41|AM|first|morning|breakfast| +34362|AAAAAAAALDGIAAAA|34362|9|32|42|AM|first|morning|breakfast| +34363|AAAAAAAAMDGIAAAA|34363|9|32|43|AM|first|morning|breakfast| +34364|AAAAAAAANDGIAAAA|34364|9|32|44|AM|first|morning|breakfast| +34365|AAAAAAAAODGIAAAA|34365|9|32|45|AM|first|morning|breakfast| +34366|AAAAAAAAPDGIAAAA|34366|9|32|46|AM|first|morning|breakfast| +34367|AAAAAAAAAEGIAAAA|34367|9|32|47|AM|first|morning|breakfast| +34368|AAAAAAAABEGIAAAA|34368|9|32|48|AM|first|morning|breakfast| +34369|AAAAAAAACEGIAAAA|34369|9|32|49|AM|first|morning|breakfast| +34370|AAAAAAAADEGIAAAA|34370|9|32|50|AM|first|morning|breakfast| +34371|AAAAAAAAEEGIAAAA|34371|9|32|51|AM|first|morning|breakfast| +34372|AAAAAAAAFEGIAAAA|34372|9|32|52|AM|first|morning|breakfast| +34373|AAAAAAAAGEGIAAAA|34373|9|32|53|AM|first|morning|breakfast| +34374|AAAAAAAAHEGIAAAA|34374|9|32|54|AM|first|morning|breakfast| +34375|AAAAAAAAIEGIAAAA|34375|9|32|55|AM|first|morning|breakfast| +34376|AAAAAAAAJEGIAAAA|34376|9|32|56|AM|first|morning|breakfast| +34377|AAAAAAAAKEGIAAAA|34377|9|32|57|AM|first|morning|breakfast| +34378|AAAAAAAALEGIAAAA|34378|9|32|58|AM|first|morning|breakfast| +34379|AAAAAAAAMEGIAAAA|34379|9|32|59|AM|first|morning|breakfast| +34380|AAAAAAAANEGIAAAA|34380|9|33|0|AM|first|morning|breakfast| +34381|AAAAAAAAOEGIAAAA|34381|9|33|1|AM|first|morning|breakfast| +34382|AAAAAAAAPEGIAAAA|34382|9|33|2|AM|first|morning|breakfast| +34383|AAAAAAAAAFGIAAAA|34383|9|33|3|AM|first|morning|breakfast| +34384|AAAAAAAABFGIAAAA|34384|9|33|4|AM|first|morning|breakfast| +34385|AAAAAAAACFGIAAAA|34385|9|33|5|AM|first|morning|breakfast| +34386|AAAAAAAADFGIAAAA|34386|9|33|6|AM|first|morning|breakfast| +34387|AAAAAAAAEFGIAAAA|34387|9|33|7|AM|first|morning|breakfast| +34388|AAAAAAAAFFGIAAAA|34388|9|33|8|AM|first|morning|breakfast| +34389|AAAAAAAAGFGIAAAA|34389|9|33|9|AM|first|morning|breakfast| +34390|AAAAAAAAHFGIAAAA|34390|9|33|10|AM|first|morning|breakfast| +34391|AAAAAAAAIFGIAAAA|34391|9|33|11|AM|first|morning|breakfast| +34392|AAAAAAAAJFGIAAAA|34392|9|33|12|AM|first|morning|breakfast| +34393|AAAAAAAAKFGIAAAA|34393|9|33|13|AM|first|morning|breakfast| +34394|AAAAAAAALFGIAAAA|34394|9|33|14|AM|first|morning|breakfast| +34395|AAAAAAAAMFGIAAAA|34395|9|33|15|AM|first|morning|breakfast| +34396|AAAAAAAANFGIAAAA|34396|9|33|16|AM|first|morning|breakfast| +34397|AAAAAAAAOFGIAAAA|34397|9|33|17|AM|first|morning|breakfast| +34398|AAAAAAAAPFGIAAAA|34398|9|33|18|AM|first|morning|breakfast| +34399|AAAAAAAAAGGIAAAA|34399|9|33|19|AM|first|morning|breakfast| +34400|AAAAAAAABGGIAAAA|34400|9|33|20|AM|first|morning|breakfast| +34401|AAAAAAAACGGIAAAA|34401|9|33|21|AM|first|morning|breakfast| +34402|AAAAAAAADGGIAAAA|34402|9|33|22|AM|first|morning|breakfast| +34403|AAAAAAAAEGGIAAAA|34403|9|33|23|AM|first|morning|breakfast| +34404|AAAAAAAAFGGIAAAA|34404|9|33|24|AM|first|morning|breakfast| +34405|AAAAAAAAGGGIAAAA|34405|9|33|25|AM|first|morning|breakfast| +34406|AAAAAAAAHGGIAAAA|34406|9|33|26|AM|first|morning|breakfast| +34407|AAAAAAAAIGGIAAAA|34407|9|33|27|AM|first|morning|breakfast| +34408|AAAAAAAAJGGIAAAA|34408|9|33|28|AM|first|morning|breakfast| +34409|AAAAAAAAKGGIAAAA|34409|9|33|29|AM|first|morning|breakfast| +34410|AAAAAAAALGGIAAAA|34410|9|33|30|AM|first|morning|breakfast| +34411|AAAAAAAAMGGIAAAA|34411|9|33|31|AM|first|morning|breakfast| +34412|AAAAAAAANGGIAAAA|34412|9|33|32|AM|first|morning|breakfast| +34413|AAAAAAAAOGGIAAAA|34413|9|33|33|AM|first|morning|breakfast| +34414|AAAAAAAAPGGIAAAA|34414|9|33|34|AM|first|morning|breakfast| +34415|AAAAAAAAAHGIAAAA|34415|9|33|35|AM|first|morning|breakfast| +34416|AAAAAAAABHGIAAAA|34416|9|33|36|AM|first|morning|breakfast| +34417|AAAAAAAACHGIAAAA|34417|9|33|37|AM|first|morning|breakfast| +34418|AAAAAAAADHGIAAAA|34418|9|33|38|AM|first|morning|breakfast| +34419|AAAAAAAAEHGIAAAA|34419|9|33|39|AM|first|morning|breakfast| +34420|AAAAAAAAFHGIAAAA|34420|9|33|40|AM|first|morning|breakfast| +34421|AAAAAAAAGHGIAAAA|34421|9|33|41|AM|first|morning|breakfast| +34422|AAAAAAAAHHGIAAAA|34422|9|33|42|AM|first|morning|breakfast| +34423|AAAAAAAAIHGIAAAA|34423|9|33|43|AM|first|morning|breakfast| +34424|AAAAAAAAJHGIAAAA|34424|9|33|44|AM|first|morning|breakfast| +34425|AAAAAAAAKHGIAAAA|34425|9|33|45|AM|first|morning|breakfast| +34426|AAAAAAAALHGIAAAA|34426|9|33|46|AM|first|morning|breakfast| +34427|AAAAAAAAMHGIAAAA|34427|9|33|47|AM|first|morning|breakfast| +34428|AAAAAAAANHGIAAAA|34428|9|33|48|AM|first|morning|breakfast| +34429|AAAAAAAAOHGIAAAA|34429|9|33|49|AM|first|morning|breakfast| +34430|AAAAAAAAPHGIAAAA|34430|9|33|50|AM|first|morning|breakfast| +34431|AAAAAAAAAIGIAAAA|34431|9|33|51|AM|first|morning|breakfast| +34432|AAAAAAAABIGIAAAA|34432|9|33|52|AM|first|morning|breakfast| +34433|AAAAAAAACIGIAAAA|34433|9|33|53|AM|first|morning|breakfast| +34434|AAAAAAAADIGIAAAA|34434|9|33|54|AM|first|morning|breakfast| +34435|AAAAAAAAEIGIAAAA|34435|9|33|55|AM|first|morning|breakfast| +34436|AAAAAAAAFIGIAAAA|34436|9|33|56|AM|first|morning|breakfast| +34437|AAAAAAAAGIGIAAAA|34437|9|33|57|AM|first|morning|breakfast| +34438|AAAAAAAAHIGIAAAA|34438|9|33|58|AM|first|morning|breakfast| +34439|AAAAAAAAIIGIAAAA|34439|9|33|59|AM|first|morning|breakfast| +34440|AAAAAAAAJIGIAAAA|34440|9|34|0|AM|first|morning|breakfast| +34441|AAAAAAAAKIGIAAAA|34441|9|34|1|AM|first|morning|breakfast| +34442|AAAAAAAALIGIAAAA|34442|9|34|2|AM|first|morning|breakfast| +34443|AAAAAAAAMIGIAAAA|34443|9|34|3|AM|first|morning|breakfast| +34444|AAAAAAAANIGIAAAA|34444|9|34|4|AM|first|morning|breakfast| +34445|AAAAAAAAOIGIAAAA|34445|9|34|5|AM|first|morning|breakfast| +34446|AAAAAAAAPIGIAAAA|34446|9|34|6|AM|first|morning|breakfast| +34447|AAAAAAAAAJGIAAAA|34447|9|34|7|AM|first|morning|breakfast| +34448|AAAAAAAABJGIAAAA|34448|9|34|8|AM|first|morning|breakfast| +34449|AAAAAAAACJGIAAAA|34449|9|34|9|AM|first|morning|breakfast| +34450|AAAAAAAADJGIAAAA|34450|9|34|10|AM|first|morning|breakfast| +34451|AAAAAAAAEJGIAAAA|34451|9|34|11|AM|first|morning|breakfast| +34452|AAAAAAAAFJGIAAAA|34452|9|34|12|AM|first|morning|breakfast| +34453|AAAAAAAAGJGIAAAA|34453|9|34|13|AM|first|morning|breakfast| +34454|AAAAAAAAHJGIAAAA|34454|9|34|14|AM|first|morning|breakfast| +34455|AAAAAAAAIJGIAAAA|34455|9|34|15|AM|first|morning|breakfast| +34456|AAAAAAAAJJGIAAAA|34456|9|34|16|AM|first|morning|breakfast| +34457|AAAAAAAAKJGIAAAA|34457|9|34|17|AM|first|morning|breakfast| +34458|AAAAAAAALJGIAAAA|34458|9|34|18|AM|first|morning|breakfast| +34459|AAAAAAAAMJGIAAAA|34459|9|34|19|AM|first|morning|breakfast| +34460|AAAAAAAANJGIAAAA|34460|9|34|20|AM|first|morning|breakfast| +34461|AAAAAAAAOJGIAAAA|34461|9|34|21|AM|first|morning|breakfast| +34462|AAAAAAAAPJGIAAAA|34462|9|34|22|AM|first|morning|breakfast| +34463|AAAAAAAAAKGIAAAA|34463|9|34|23|AM|first|morning|breakfast| +34464|AAAAAAAABKGIAAAA|34464|9|34|24|AM|first|morning|breakfast| +34465|AAAAAAAACKGIAAAA|34465|9|34|25|AM|first|morning|breakfast| +34466|AAAAAAAADKGIAAAA|34466|9|34|26|AM|first|morning|breakfast| +34467|AAAAAAAAEKGIAAAA|34467|9|34|27|AM|first|morning|breakfast| +34468|AAAAAAAAFKGIAAAA|34468|9|34|28|AM|first|morning|breakfast| +34469|AAAAAAAAGKGIAAAA|34469|9|34|29|AM|first|morning|breakfast| +34470|AAAAAAAAHKGIAAAA|34470|9|34|30|AM|first|morning|breakfast| +34471|AAAAAAAAIKGIAAAA|34471|9|34|31|AM|first|morning|breakfast| +34472|AAAAAAAAJKGIAAAA|34472|9|34|32|AM|first|morning|breakfast| +34473|AAAAAAAAKKGIAAAA|34473|9|34|33|AM|first|morning|breakfast| +34474|AAAAAAAALKGIAAAA|34474|9|34|34|AM|first|morning|breakfast| +34475|AAAAAAAAMKGIAAAA|34475|9|34|35|AM|first|morning|breakfast| +34476|AAAAAAAANKGIAAAA|34476|9|34|36|AM|first|morning|breakfast| +34477|AAAAAAAAOKGIAAAA|34477|9|34|37|AM|first|morning|breakfast| +34478|AAAAAAAAPKGIAAAA|34478|9|34|38|AM|first|morning|breakfast| +34479|AAAAAAAAALGIAAAA|34479|9|34|39|AM|first|morning|breakfast| +34480|AAAAAAAABLGIAAAA|34480|9|34|40|AM|first|morning|breakfast| +34481|AAAAAAAACLGIAAAA|34481|9|34|41|AM|first|morning|breakfast| +34482|AAAAAAAADLGIAAAA|34482|9|34|42|AM|first|morning|breakfast| +34483|AAAAAAAAELGIAAAA|34483|9|34|43|AM|first|morning|breakfast| +34484|AAAAAAAAFLGIAAAA|34484|9|34|44|AM|first|morning|breakfast| +34485|AAAAAAAAGLGIAAAA|34485|9|34|45|AM|first|morning|breakfast| +34486|AAAAAAAAHLGIAAAA|34486|9|34|46|AM|first|morning|breakfast| +34487|AAAAAAAAILGIAAAA|34487|9|34|47|AM|first|morning|breakfast| +34488|AAAAAAAAJLGIAAAA|34488|9|34|48|AM|first|morning|breakfast| +34489|AAAAAAAAKLGIAAAA|34489|9|34|49|AM|first|morning|breakfast| +34490|AAAAAAAALLGIAAAA|34490|9|34|50|AM|first|morning|breakfast| +34491|AAAAAAAAMLGIAAAA|34491|9|34|51|AM|first|morning|breakfast| +34492|AAAAAAAANLGIAAAA|34492|9|34|52|AM|first|morning|breakfast| +34493|AAAAAAAAOLGIAAAA|34493|9|34|53|AM|first|morning|breakfast| +34494|AAAAAAAAPLGIAAAA|34494|9|34|54|AM|first|morning|breakfast| +34495|AAAAAAAAAMGIAAAA|34495|9|34|55|AM|first|morning|breakfast| +34496|AAAAAAAABMGIAAAA|34496|9|34|56|AM|first|morning|breakfast| +34497|AAAAAAAACMGIAAAA|34497|9|34|57|AM|first|morning|breakfast| +34498|AAAAAAAADMGIAAAA|34498|9|34|58|AM|first|morning|breakfast| +34499|AAAAAAAAEMGIAAAA|34499|9|34|59|AM|first|morning|breakfast| +34500|AAAAAAAAFMGIAAAA|34500|9|35|0|AM|first|morning|breakfast| +34501|AAAAAAAAGMGIAAAA|34501|9|35|1|AM|first|morning|breakfast| +34502|AAAAAAAAHMGIAAAA|34502|9|35|2|AM|first|morning|breakfast| +34503|AAAAAAAAIMGIAAAA|34503|9|35|3|AM|first|morning|breakfast| +34504|AAAAAAAAJMGIAAAA|34504|9|35|4|AM|first|morning|breakfast| +34505|AAAAAAAAKMGIAAAA|34505|9|35|5|AM|first|morning|breakfast| +34506|AAAAAAAALMGIAAAA|34506|9|35|6|AM|first|morning|breakfast| +34507|AAAAAAAAMMGIAAAA|34507|9|35|7|AM|first|morning|breakfast| +34508|AAAAAAAANMGIAAAA|34508|9|35|8|AM|first|morning|breakfast| +34509|AAAAAAAAOMGIAAAA|34509|9|35|9|AM|first|morning|breakfast| +34510|AAAAAAAAPMGIAAAA|34510|9|35|10|AM|first|morning|breakfast| +34511|AAAAAAAAANGIAAAA|34511|9|35|11|AM|first|morning|breakfast| +34512|AAAAAAAABNGIAAAA|34512|9|35|12|AM|first|morning|breakfast| +34513|AAAAAAAACNGIAAAA|34513|9|35|13|AM|first|morning|breakfast| +34514|AAAAAAAADNGIAAAA|34514|9|35|14|AM|first|morning|breakfast| +34515|AAAAAAAAENGIAAAA|34515|9|35|15|AM|first|morning|breakfast| +34516|AAAAAAAAFNGIAAAA|34516|9|35|16|AM|first|morning|breakfast| +34517|AAAAAAAAGNGIAAAA|34517|9|35|17|AM|first|morning|breakfast| +34518|AAAAAAAAHNGIAAAA|34518|9|35|18|AM|first|morning|breakfast| +34519|AAAAAAAAINGIAAAA|34519|9|35|19|AM|first|morning|breakfast| +34520|AAAAAAAAJNGIAAAA|34520|9|35|20|AM|first|morning|breakfast| +34521|AAAAAAAAKNGIAAAA|34521|9|35|21|AM|first|morning|breakfast| +34522|AAAAAAAALNGIAAAA|34522|9|35|22|AM|first|morning|breakfast| +34523|AAAAAAAAMNGIAAAA|34523|9|35|23|AM|first|morning|breakfast| +34524|AAAAAAAANNGIAAAA|34524|9|35|24|AM|first|morning|breakfast| +34525|AAAAAAAAONGIAAAA|34525|9|35|25|AM|first|morning|breakfast| +34526|AAAAAAAAPNGIAAAA|34526|9|35|26|AM|first|morning|breakfast| +34527|AAAAAAAAAOGIAAAA|34527|9|35|27|AM|first|morning|breakfast| +34528|AAAAAAAABOGIAAAA|34528|9|35|28|AM|first|morning|breakfast| +34529|AAAAAAAACOGIAAAA|34529|9|35|29|AM|first|morning|breakfast| +34530|AAAAAAAADOGIAAAA|34530|9|35|30|AM|first|morning|breakfast| +34531|AAAAAAAAEOGIAAAA|34531|9|35|31|AM|first|morning|breakfast| +34532|AAAAAAAAFOGIAAAA|34532|9|35|32|AM|first|morning|breakfast| +34533|AAAAAAAAGOGIAAAA|34533|9|35|33|AM|first|morning|breakfast| +34534|AAAAAAAAHOGIAAAA|34534|9|35|34|AM|first|morning|breakfast| +34535|AAAAAAAAIOGIAAAA|34535|9|35|35|AM|first|morning|breakfast| +34536|AAAAAAAAJOGIAAAA|34536|9|35|36|AM|first|morning|breakfast| +34537|AAAAAAAAKOGIAAAA|34537|9|35|37|AM|first|morning|breakfast| +34538|AAAAAAAALOGIAAAA|34538|9|35|38|AM|first|morning|breakfast| +34539|AAAAAAAAMOGIAAAA|34539|9|35|39|AM|first|morning|breakfast| +34540|AAAAAAAANOGIAAAA|34540|9|35|40|AM|first|morning|breakfast| +34541|AAAAAAAAOOGIAAAA|34541|9|35|41|AM|first|morning|breakfast| +34542|AAAAAAAAPOGIAAAA|34542|9|35|42|AM|first|morning|breakfast| +34543|AAAAAAAAAPGIAAAA|34543|9|35|43|AM|first|morning|breakfast| +34544|AAAAAAAABPGIAAAA|34544|9|35|44|AM|first|morning|breakfast| +34545|AAAAAAAACPGIAAAA|34545|9|35|45|AM|first|morning|breakfast| +34546|AAAAAAAADPGIAAAA|34546|9|35|46|AM|first|morning|breakfast| +34547|AAAAAAAAEPGIAAAA|34547|9|35|47|AM|first|morning|breakfast| +34548|AAAAAAAAFPGIAAAA|34548|9|35|48|AM|first|morning|breakfast| +34549|AAAAAAAAGPGIAAAA|34549|9|35|49|AM|first|morning|breakfast| +34550|AAAAAAAAHPGIAAAA|34550|9|35|50|AM|first|morning|breakfast| +34551|AAAAAAAAIPGIAAAA|34551|9|35|51|AM|first|morning|breakfast| +34552|AAAAAAAAJPGIAAAA|34552|9|35|52|AM|first|morning|breakfast| +34553|AAAAAAAAKPGIAAAA|34553|9|35|53|AM|first|morning|breakfast| +34554|AAAAAAAALPGIAAAA|34554|9|35|54|AM|first|morning|breakfast| +34555|AAAAAAAAMPGIAAAA|34555|9|35|55|AM|first|morning|breakfast| +34556|AAAAAAAANPGIAAAA|34556|9|35|56|AM|first|morning|breakfast| +34557|AAAAAAAAOPGIAAAA|34557|9|35|57|AM|first|morning|breakfast| +34558|AAAAAAAAPPGIAAAA|34558|9|35|58|AM|first|morning|breakfast| +34559|AAAAAAAAAAHIAAAA|34559|9|35|59|AM|first|morning|breakfast| +34560|AAAAAAAABAHIAAAA|34560|9|36|0|AM|first|morning|breakfast| +34561|AAAAAAAACAHIAAAA|34561|9|36|1|AM|first|morning|breakfast| +34562|AAAAAAAADAHIAAAA|34562|9|36|2|AM|first|morning|breakfast| +34563|AAAAAAAAEAHIAAAA|34563|9|36|3|AM|first|morning|breakfast| +34564|AAAAAAAAFAHIAAAA|34564|9|36|4|AM|first|morning|breakfast| +34565|AAAAAAAAGAHIAAAA|34565|9|36|5|AM|first|morning|breakfast| +34566|AAAAAAAAHAHIAAAA|34566|9|36|6|AM|first|morning|breakfast| +34567|AAAAAAAAIAHIAAAA|34567|9|36|7|AM|first|morning|breakfast| +34568|AAAAAAAAJAHIAAAA|34568|9|36|8|AM|first|morning|breakfast| +34569|AAAAAAAAKAHIAAAA|34569|9|36|9|AM|first|morning|breakfast| +34570|AAAAAAAALAHIAAAA|34570|9|36|10|AM|first|morning|breakfast| +34571|AAAAAAAAMAHIAAAA|34571|9|36|11|AM|first|morning|breakfast| +34572|AAAAAAAANAHIAAAA|34572|9|36|12|AM|first|morning|breakfast| +34573|AAAAAAAAOAHIAAAA|34573|9|36|13|AM|first|morning|breakfast| +34574|AAAAAAAAPAHIAAAA|34574|9|36|14|AM|first|morning|breakfast| +34575|AAAAAAAAABHIAAAA|34575|9|36|15|AM|first|morning|breakfast| +34576|AAAAAAAABBHIAAAA|34576|9|36|16|AM|first|morning|breakfast| +34577|AAAAAAAACBHIAAAA|34577|9|36|17|AM|first|morning|breakfast| +34578|AAAAAAAADBHIAAAA|34578|9|36|18|AM|first|morning|breakfast| +34579|AAAAAAAAEBHIAAAA|34579|9|36|19|AM|first|morning|breakfast| +34580|AAAAAAAAFBHIAAAA|34580|9|36|20|AM|first|morning|breakfast| +34581|AAAAAAAAGBHIAAAA|34581|9|36|21|AM|first|morning|breakfast| +34582|AAAAAAAAHBHIAAAA|34582|9|36|22|AM|first|morning|breakfast| +34583|AAAAAAAAIBHIAAAA|34583|9|36|23|AM|first|morning|breakfast| +34584|AAAAAAAAJBHIAAAA|34584|9|36|24|AM|first|morning|breakfast| +34585|AAAAAAAAKBHIAAAA|34585|9|36|25|AM|first|morning|breakfast| +34586|AAAAAAAALBHIAAAA|34586|9|36|26|AM|first|morning|breakfast| +34587|AAAAAAAAMBHIAAAA|34587|9|36|27|AM|first|morning|breakfast| +34588|AAAAAAAANBHIAAAA|34588|9|36|28|AM|first|morning|breakfast| +34589|AAAAAAAAOBHIAAAA|34589|9|36|29|AM|first|morning|breakfast| +34590|AAAAAAAAPBHIAAAA|34590|9|36|30|AM|first|morning|breakfast| +34591|AAAAAAAAACHIAAAA|34591|9|36|31|AM|first|morning|breakfast| +34592|AAAAAAAABCHIAAAA|34592|9|36|32|AM|first|morning|breakfast| +34593|AAAAAAAACCHIAAAA|34593|9|36|33|AM|first|morning|breakfast| +34594|AAAAAAAADCHIAAAA|34594|9|36|34|AM|first|morning|breakfast| +34595|AAAAAAAAECHIAAAA|34595|9|36|35|AM|first|morning|breakfast| +34596|AAAAAAAAFCHIAAAA|34596|9|36|36|AM|first|morning|breakfast| +34597|AAAAAAAAGCHIAAAA|34597|9|36|37|AM|first|morning|breakfast| +34598|AAAAAAAAHCHIAAAA|34598|9|36|38|AM|first|morning|breakfast| +34599|AAAAAAAAICHIAAAA|34599|9|36|39|AM|first|morning|breakfast| +34600|AAAAAAAAJCHIAAAA|34600|9|36|40|AM|first|morning|breakfast| +34601|AAAAAAAAKCHIAAAA|34601|9|36|41|AM|first|morning|breakfast| +34602|AAAAAAAALCHIAAAA|34602|9|36|42|AM|first|morning|breakfast| +34603|AAAAAAAAMCHIAAAA|34603|9|36|43|AM|first|morning|breakfast| +34604|AAAAAAAANCHIAAAA|34604|9|36|44|AM|first|morning|breakfast| +34605|AAAAAAAAOCHIAAAA|34605|9|36|45|AM|first|morning|breakfast| +34606|AAAAAAAAPCHIAAAA|34606|9|36|46|AM|first|morning|breakfast| +34607|AAAAAAAAADHIAAAA|34607|9|36|47|AM|first|morning|breakfast| +34608|AAAAAAAABDHIAAAA|34608|9|36|48|AM|first|morning|breakfast| +34609|AAAAAAAACDHIAAAA|34609|9|36|49|AM|first|morning|breakfast| +34610|AAAAAAAADDHIAAAA|34610|9|36|50|AM|first|morning|breakfast| +34611|AAAAAAAAEDHIAAAA|34611|9|36|51|AM|first|morning|breakfast| +34612|AAAAAAAAFDHIAAAA|34612|9|36|52|AM|first|morning|breakfast| +34613|AAAAAAAAGDHIAAAA|34613|9|36|53|AM|first|morning|breakfast| +34614|AAAAAAAAHDHIAAAA|34614|9|36|54|AM|first|morning|breakfast| +34615|AAAAAAAAIDHIAAAA|34615|9|36|55|AM|first|morning|breakfast| +34616|AAAAAAAAJDHIAAAA|34616|9|36|56|AM|first|morning|breakfast| +34617|AAAAAAAAKDHIAAAA|34617|9|36|57|AM|first|morning|breakfast| +34618|AAAAAAAALDHIAAAA|34618|9|36|58|AM|first|morning|breakfast| +34619|AAAAAAAAMDHIAAAA|34619|9|36|59|AM|first|morning|breakfast| +34620|AAAAAAAANDHIAAAA|34620|9|37|0|AM|first|morning|breakfast| +34621|AAAAAAAAODHIAAAA|34621|9|37|1|AM|first|morning|breakfast| +34622|AAAAAAAAPDHIAAAA|34622|9|37|2|AM|first|morning|breakfast| +34623|AAAAAAAAAEHIAAAA|34623|9|37|3|AM|first|morning|breakfast| +34624|AAAAAAAABEHIAAAA|34624|9|37|4|AM|first|morning|breakfast| +34625|AAAAAAAACEHIAAAA|34625|9|37|5|AM|first|morning|breakfast| +34626|AAAAAAAADEHIAAAA|34626|9|37|6|AM|first|morning|breakfast| +34627|AAAAAAAAEEHIAAAA|34627|9|37|7|AM|first|morning|breakfast| +34628|AAAAAAAAFEHIAAAA|34628|9|37|8|AM|first|morning|breakfast| +34629|AAAAAAAAGEHIAAAA|34629|9|37|9|AM|first|morning|breakfast| +34630|AAAAAAAAHEHIAAAA|34630|9|37|10|AM|first|morning|breakfast| +34631|AAAAAAAAIEHIAAAA|34631|9|37|11|AM|first|morning|breakfast| +34632|AAAAAAAAJEHIAAAA|34632|9|37|12|AM|first|morning|breakfast| +34633|AAAAAAAAKEHIAAAA|34633|9|37|13|AM|first|morning|breakfast| +34634|AAAAAAAALEHIAAAA|34634|9|37|14|AM|first|morning|breakfast| +34635|AAAAAAAAMEHIAAAA|34635|9|37|15|AM|first|morning|breakfast| +34636|AAAAAAAANEHIAAAA|34636|9|37|16|AM|first|morning|breakfast| +34637|AAAAAAAAOEHIAAAA|34637|9|37|17|AM|first|morning|breakfast| +34638|AAAAAAAAPEHIAAAA|34638|9|37|18|AM|first|morning|breakfast| +34639|AAAAAAAAAFHIAAAA|34639|9|37|19|AM|first|morning|breakfast| +34640|AAAAAAAABFHIAAAA|34640|9|37|20|AM|first|morning|breakfast| +34641|AAAAAAAACFHIAAAA|34641|9|37|21|AM|first|morning|breakfast| +34642|AAAAAAAADFHIAAAA|34642|9|37|22|AM|first|morning|breakfast| +34643|AAAAAAAAEFHIAAAA|34643|9|37|23|AM|first|morning|breakfast| +34644|AAAAAAAAFFHIAAAA|34644|9|37|24|AM|first|morning|breakfast| +34645|AAAAAAAAGFHIAAAA|34645|9|37|25|AM|first|morning|breakfast| +34646|AAAAAAAAHFHIAAAA|34646|9|37|26|AM|first|morning|breakfast| +34647|AAAAAAAAIFHIAAAA|34647|9|37|27|AM|first|morning|breakfast| +34648|AAAAAAAAJFHIAAAA|34648|9|37|28|AM|first|morning|breakfast| +34649|AAAAAAAAKFHIAAAA|34649|9|37|29|AM|first|morning|breakfast| +34650|AAAAAAAALFHIAAAA|34650|9|37|30|AM|first|morning|breakfast| +34651|AAAAAAAAMFHIAAAA|34651|9|37|31|AM|first|morning|breakfast| +34652|AAAAAAAANFHIAAAA|34652|9|37|32|AM|first|morning|breakfast| +34653|AAAAAAAAOFHIAAAA|34653|9|37|33|AM|first|morning|breakfast| +34654|AAAAAAAAPFHIAAAA|34654|9|37|34|AM|first|morning|breakfast| +34655|AAAAAAAAAGHIAAAA|34655|9|37|35|AM|first|morning|breakfast| +34656|AAAAAAAABGHIAAAA|34656|9|37|36|AM|first|morning|breakfast| +34657|AAAAAAAACGHIAAAA|34657|9|37|37|AM|first|morning|breakfast| +34658|AAAAAAAADGHIAAAA|34658|9|37|38|AM|first|morning|breakfast| +34659|AAAAAAAAEGHIAAAA|34659|9|37|39|AM|first|morning|breakfast| +34660|AAAAAAAAFGHIAAAA|34660|9|37|40|AM|first|morning|breakfast| +34661|AAAAAAAAGGHIAAAA|34661|9|37|41|AM|first|morning|breakfast| +34662|AAAAAAAAHGHIAAAA|34662|9|37|42|AM|first|morning|breakfast| +34663|AAAAAAAAIGHIAAAA|34663|9|37|43|AM|first|morning|breakfast| +34664|AAAAAAAAJGHIAAAA|34664|9|37|44|AM|first|morning|breakfast| +34665|AAAAAAAAKGHIAAAA|34665|9|37|45|AM|first|morning|breakfast| +34666|AAAAAAAALGHIAAAA|34666|9|37|46|AM|first|morning|breakfast| +34667|AAAAAAAAMGHIAAAA|34667|9|37|47|AM|first|morning|breakfast| +34668|AAAAAAAANGHIAAAA|34668|9|37|48|AM|first|morning|breakfast| +34669|AAAAAAAAOGHIAAAA|34669|9|37|49|AM|first|morning|breakfast| +34670|AAAAAAAAPGHIAAAA|34670|9|37|50|AM|first|morning|breakfast| +34671|AAAAAAAAAHHIAAAA|34671|9|37|51|AM|first|morning|breakfast| +34672|AAAAAAAABHHIAAAA|34672|9|37|52|AM|first|morning|breakfast| +34673|AAAAAAAACHHIAAAA|34673|9|37|53|AM|first|morning|breakfast| +34674|AAAAAAAADHHIAAAA|34674|9|37|54|AM|first|morning|breakfast| +34675|AAAAAAAAEHHIAAAA|34675|9|37|55|AM|first|morning|breakfast| +34676|AAAAAAAAFHHIAAAA|34676|9|37|56|AM|first|morning|breakfast| +34677|AAAAAAAAGHHIAAAA|34677|9|37|57|AM|first|morning|breakfast| +34678|AAAAAAAAHHHIAAAA|34678|9|37|58|AM|first|morning|breakfast| +34679|AAAAAAAAIHHIAAAA|34679|9|37|59|AM|first|morning|breakfast| +34680|AAAAAAAAJHHIAAAA|34680|9|38|0|AM|first|morning|breakfast| +34681|AAAAAAAAKHHIAAAA|34681|9|38|1|AM|first|morning|breakfast| +34682|AAAAAAAALHHIAAAA|34682|9|38|2|AM|first|morning|breakfast| +34683|AAAAAAAAMHHIAAAA|34683|9|38|3|AM|first|morning|breakfast| +34684|AAAAAAAANHHIAAAA|34684|9|38|4|AM|first|morning|breakfast| +34685|AAAAAAAAOHHIAAAA|34685|9|38|5|AM|first|morning|breakfast| +34686|AAAAAAAAPHHIAAAA|34686|9|38|6|AM|first|morning|breakfast| +34687|AAAAAAAAAIHIAAAA|34687|9|38|7|AM|first|morning|breakfast| +34688|AAAAAAAABIHIAAAA|34688|9|38|8|AM|first|morning|breakfast| +34689|AAAAAAAACIHIAAAA|34689|9|38|9|AM|first|morning|breakfast| +34690|AAAAAAAADIHIAAAA|34690|9|38|10|AM|first|morning|breakfast| +34691|AAAAAAAAEIHIAAAA|34691|9|38|11|AM|first|morning|breakfast| +34692|AAAAAAAAFIHIAAAA|34692|9|38|12|AM|first|morning|breakfast| +34693|AAAAAAAAGIHIAAAA|34693|9|38|13|AM|first|morning|breakfast| +34694|AAAAAAAAHIHIAAAA|34694|9|38|14|AM|first|morning|breakfast| +34695|AAAAAAAAIIHIAAAA|34695|9|38|15|AM|first|morning|breakfast| +34696|AAAAAAAAJIHIAAAA|34696|9|38|16|AM|first|morning|breakfast| +34697|AAAAAAAAKIHIAAAA|34697|9|38|17|AM|first|morning|breakfast| +34698|AAAAAAAALIHIAAAA|34698|9|38|18|AM|first|morning|breakfast| +34699|AAAAAAAAMIHIAAAA|34699|9|38|19|AM|first|morning|breakfast| +34700|AAAAAAAANIHIAAAA|34700|9|38|20|AM|first|morning|breakfast| +34701|AAAAAAAAOIHIAAAA|34701|9|38|21|AM|first|morning|breakfast| +34702|AAAAAAAAPIHIAAAA|34702|9|38|22|AM|first|morning|breakfast| +34703|AAAAAAAAAJHIAAAA|34703|9|38|23|AM|first|morning|breakfast| +34704|AAAAAAAABJHIAAAA|34704|9|38|24|AM|first|morning|breakfast| +34705|AAAAAAAACJHIAAAA|34705|9|38|25|AM|first|morning|breakfast| +34706|AAAAAAAADJHIAAAA|34706|9|38|26|AM|first|morning|breakfast| +34707|AAAAAAAAEJHIAAAA|34707|9|38|27|AM|first|morning|breakfast| +34708|AAAAAAAAFJHIAAAA|34708|9|38|28|AM|first|morning|breakfast| +34709|AAAAAAAAGJHIAAAA|34709|9|38|29|AM|first|morning|breakfast| +34710|AAAAAAAAHJHIAAAA|34710|9|38|30|AM|first|morning|breakfast| +34711|AAAAAAAAIJHIAAAA|34711|9|38|31|AM|first|morning|breakfast| +34712|AAAAAAAAJJHIAAAA|34712|9|38|32|AM|first|morning|breakfast| +34713|AAAAAAAAKJHIAAAA|34713|9|38|33|AM|first|morning|breakfast| +34714|AAAAAAAALJHIAAAA|34714|9|38|34|AM|first|morning|breakfast| +34715|AAAAAAAAMJHIAAAA|34715|9|38|35|AM|first|morning|breakfast| +34716|AAAAAAAANJHIAAAA|34716|9|38|36|AM|first|morning|breakfast| +34717|AAAAAAAAOJHIAAAA|34717|9|38|37|AM|first|morning|breakfast| +34718|AAAAAAAAPJHIAAAA|34718|9|38|38|AM|first|morning|breakfast| +34719|AAAAAAAAAKHIAAAA|34719|9|38|39|AM|first|morning|breakfast| +34720|AAAAAAAABKHIAAAA|34720|9|38|40|AM|first|morning|breakfast| +34721|AAAAAAAACKHIAAAA|34721|9|38|41|AM|first|morning|breakfast| +34722|AAAAAAAADKHIAAAA|34722|9|38|42|AM|first|morning|breakfast| +34723|AAAAAAAAEKHIAAAA|34723|9|38|43|AM|first|morning|breakfast| +34724|AAAAAAAAFKHIAAAA|34724|9|38|44|AM|first|morning|breakfast| +34725|AAAAAAAAGKHIAAAA|34725|9|38|45|AM|first|morning|breakfast| +34726|AAAAAAAAHKHIAAAA|34726|9|38|46|AM|first|morning|breakfast| +34727|AAAAAAAAIKHIAAAA|34727|9|38|47|AM|first|morning|breakfast| +34728|AAAAAAAAJKHIAAAA|34728|9|38|48|AM|first|morning|breakfast| +34729|AAAAAAAAKKHIAAAA|34729|9|38|49|AM|first|morning|breakfast| +34730|AAAAAAAALKHIAAAA|34730|9|38|50|AM|first|morning|breakfast| +34731|AAAAAAAAMKHIAAAA|34731|9|38|51|AM|first|morning|breakfast| +34732|AAAAAAAANKHIAAAA|34732|9|38|52|AM|first|morning|breakfast| +34733|AAAAAAAAOKHIAAAA|34733|9|38|53|AM|first|morning|breakfast| +34734|AAAAAAAAPKHIAAAA|34734|9|38|54|AM|first|morning|breakfast| +34735|AAAAAAAAALHIAAAA|34735|9|38|55|AM|first|morning|breakfast| +34736|AAAAAAAABLHIAAAA|34736|9|38|56|AM|first|morning|breakfast| +34737|AAAAAAAACLHIAAAA|34737|9|38|57|AM|first|morning|breakfast| +34738|AAAAAAAADLHIAAAA|34738|9|38|58|AM|first|morning|breakfast| +34739|AAAAAAAAELHIAAAA|34739|9|38|59|AM|first|morning|breakfast| +34740|AAAAAAAAFLHIAAAA|34740|9|39|0|AM|first|morning|breakfast| +34741|AAAAAAAAGLHIAAAA|34741|9|39|1|AM|first|morning|breakfast| +34742|AAAAAAAAHLHIAAAA|34742|9|39|2|AM|first|morning|breakfast| +34743|AAAAAAAAILHIAAAA|34743|9|39|3|AM|first|morning|breakfast| +34744|AAAAAAAAJLHIAAAA|34744|9|39|4|AM|first|morning|breakfast| +34745|AAAAAAAAKLHIAAAA|34745|9|39|5|AM|first|morning|breakfast| +34746|AAAAAAAALLHIAAAA|34746|9|39|6|AM|first|morning|breakfast| +34747|AAAAAAAAMLHIAAAA|34747|9|39|7|AM|first|morning|breakfast| +34748|AAAAAAAANLHIAAAA|34748|9|39|8|AM|first|morning|breakfast| +34749|AAAAAAAAOLHIAAAA|34749|9|39|9|AM|first|morning|breakfast| +34750|AAAAAAAAPLHIAAAA|34750|9|39|10|AM|first|morning|breakfast| +34751|AAAAAAAAAMHIAAAA|34751|9|39|11|AM|first|morning|breakfast| +34752|AAAAAAAABMHIAAAA|34752|9|39|12|AM|first|morning|breakfast| +34753|AAAAAAAACMHIAAAA|34753|9|39|13|AM|first|morning|breakfast| +34754|AAAAAAAADMHIAAAA|34754|9|39|14|AM|first|morning|breakfast| +34755|AAAAAAAAEMHIAAAA|34755|9|39|15|AM|first|morning|breakfast| +34756|AAAAAAAAFMHIAAAA|34756|9|39|16|AM|first|morning|breakfast| +34757|AAAAAAAAGMHIAAAA|34757|9|39|17|AM|first|morning|breakfast| +34758|AAAAAAAAHMHIAAAA|34758|9|39|18|AM|first|morning|breakfast| +34759|AAAAAAAAIMHIAAAA|34759|9|39|19|AM|first|morning|breakfast| +34760|AAAAAAAAJMHIAAAA|34760|9|39|20|AM|first|morning|breakfast| +34761|AAAAAAAAKMHIAAAA|34761|9|39|21|AM|first|morning|breakfast| +34762|AAAAAAAALMHIAAAA|34762|9|39|22|AM|first|morning|breakfast| +34763|AAAAAAAAMMHIAAAA|34763|9|39|23|AM|first|morning|breakfast| +34764|AAAAAAAANMHIAAAA|34764|9|39|24|AM|first|morning|breakfast| +34765|AAAAAAAAOMHIAAAA|34765|9|39|25|AM|first|morning|breakfast| +34766|AAAAAAAAPMHIAAAA|34766|9|39|26|AM|first|morning|breakfast| +34767|AAAAAAAAANHIAAAA|34767|9|39|27|AM|first|morning|breakfast| +34768|AAAAAAAABNHIAAAA|34768|9|39|28|AM|first|morning|breakfast| +34769|AAAAAAAACNHIAAAA|34769|9|39|29|AM|first|morning|breakfast| +34770|AAAAAAAADNHIAAAA|34770|9|39|30|AM|first|morning|breakfast| +34771|AAAAAAAAENHIAAAA|34771|9|39|31|AM|first|morning|breakfast| +34772|AAAAAAAAFNHIAAAA|34772|9|39|32|AM|first|morning|breakfast| +34773|AAAAAAAAGNHIAAAA|34773|9|39|33|AM|first|morning|breakfast| +34774|AAAAAAAAHNHIAAAA|34774|9|39|34|AM|first|morning|breakfast| +34775|AAAAAAAAINHIAAAA|34775|9|39|35|AM|first|morning|breakfast| +34776|AAAAAAAAJNHIAAAA|34776|9|39|36|AM|first|morning|breakfast| +34777|AAAAAAAAKNHIAAAA|34777|9|39|37|AM|first|morning|breakfast| +34778|AAAAAAAALNHIAAAA|34778|9|39|38|AM|first|morning|breakfast| +34779|AAAAAAAAMNHIAAAA|34779|9|39|39|AM|first|morning|breakfast| +34780|AAAAAAAANNHIAAAA|34780|9|39|40|AM|first|morning|breakfast| +34781|AAAAAAAAONHIAAAA|34781|9|39|41|AM|first|morning|breakfast| +34782|AAAAAAAAPNHIAAAA|34782|9|39|42|AM|first|morning|breakfast| +34783|AAAAAAAAAOHIAAAA|34783|9|39|43|AM|first|morning|breakfast| +34784|AAAAAAAABOHIAAAA|34784|9|39|44|AM|first|morning|breakfast| +34785|AAAAAAAACOHIAAAA|34785|9|39|45|AM|first|morning|breakfast| +34786|AAAAAAAADOHIAAAA|34786|9|39|46|AM|first|morning|breakfast| +34787|AAAAAAAAEOHIAAAA|34787|9|39|47|AM|first|morning|breakfast| +34788|AAAAAAAAFOHIAAAA|34788|9|39|48|AM|first|morning|breakfast| +34789|AAAAAAAAGOHIAAAA|34789|9|39|49|AM|first|morning|breakfast| +34790|AAAAAAAAHOHIAAAA|34790|9|39|50|AM|first|morning|breakfast| +34791|AAAAAAAAIOHIAAAA|34791|9|39|51|AM|first|morning|breakfast| +34792|AAAAAAAAJOHIAAAA|34792|9|39|52|AM|first|morning|breakfast| +34793|AAAAAAAAKOHIAAAA|34793|9|39|53|AM|first|morning|breakfast| +34794|AAAAAAAALOHIAAAA|34794|9|39|54|AM|first|morning|breakfast| +34795|AAAAAAAAMOHIAAAA|34795|9|39|55|AM|first|morning|breakfast| +34796|AAAAAAAANOHIAAAA|34796|9|39|56|AM|first|morning|breakfast| +34797|AAAAAAAAOOHIAAAA|34797|9|39|57|AM|first|morning|breakfast| +34798|AAAAAAAAPOHIAAAA|34798|9|39|58|AM|first|morning|breakfast| +34799|AAAAAAAAAPHIAAAA|34799|9|39|59|AM|first|morning|breakfast| +34800|AAAAAAAABPHIAAAA|34800|9|40|0|AM|first|morning|breakfast| +34801|AAAAAAAACPHIAAAA|34801|9|40|1|AM|first|morning|breakfast| +34802|AAAAAAAADPHIAAAA|34802|9|40|2|AM|first|morning|breakfast| +34803|AAAAAAAAEPHIAAAA|34803|9|40|3|AM|first|morning|breakfast| +34804|AAAAAAAAFPHIAAAA|34804|9|40|4|AM|first|morning|breakfast| +34805|AAAAAAAAGPHIAAAA|34805|9|40|5|AM|first|morning|breakfast| +34806|AAAAAAAAHPHIAAAA|34806|9|40|6|AM|first|morning|breakfast| +34807|AAAAAAAAIPHIAAAA|34807|9|40|7|AM|first|morning|breakfast| +34808|AAAAAAAAJPHIAAAA|34808|9|40|8|AM|first|morning|breakfast| +34809|AAAAAAAAKPHIAAAA|34809|9|40|9|AM|first|morning|breakfast| +34810|AAAAAAAALPHIAAAA|34810|9|40|10|AM|first|morning|breakfast| +34811|AAAAAAAAMPHIAAAA|34811|9|40|11|AM|first|morning|breakfast| +34812|AAAAAAAANPHIAAAA|34812|9|40|12|AM|first|morning|breakfast| +34813|AAAAAAAAOPHIAAAA|34813|9|40|13|AM|first|morning|breakfast| +34814|AAAAAAAAPPHIAAAA|34814|9|40|14|AM|first|morning|breakfast| +34815|AAAAAAAAAAIIAAAA|34815|9|40|15|AM|first|morning|breakfast| +34816|AAAAAAAABAIIAAAA|34816|9|40|16|AM|first|morning|breakfast| +34817|AAAAAAAACAIIAAAA|34817|9|40|17|AM|first|morning|breakfast| +34818|AAAAAAAADAIIAAAA|34818|9|40|18|AM|first|morning|breakfast| +34819|AAAAAAAAEAIIAAAA|34819|9|40|19|AM|first|morning|breakfast| +34820|AAAAAAAAFAIIAAAA|34820|9|40|20|AM|first|morning|breakfast| +34821|AAAAAAAAGAIIAAAA|34821|9|40|21|AM|first|morning|breakfast| +34822|AAAAAAAAHAIIAAAA|34822|9|40|22|AM|first|morning|breakfast| +34823|AAAAAAAAIAIIAAAA|34823|9|40|23|AM|first|morning|breakfast| +34824|AAAAAAAAJAIIAAAA|34824|9|40|24|AM|first|morning|breakfast| +34825|AAAAAAAAKAIIAAAA|34825|9|40|25|AM|first|morning|breakfast| +34826|AAAAAAAALAIIAAAA|34826|9|40|26|AM|first|morning|breakfast| +34827|AAAAAAAAMAIIAAAA|34827|9|40|27|AM|first|morning|breakfast| +34828|AAAAAAAANAIIAAAA|34828|9|40|28|AM|first|morning|breakfast| +34829|AAAAAAAAOAIIAAAA|34829|9|40|29|AM|first|morning|breakfast| +34830|AAAAAAAAPAIIAAAA|34830|9|40|30|AM|first|morning|breakfast| +34831|AAAAAAAAABIIAAAA|34831|9|40|31|AM|first|morning|breakfast| +34832|AAAAAAAABBIIAAAA|34832|9|40|32|AM|first|morning|breakfast| +34833|AAAAAAAACBIIAAAA|34833|9|40|33|AM|first|morning|breakfast| +34834|AAAAAAAADBIIAAAA|34834|9|40|34|AM|first|morning|breakfast| +34835|AAAAAAAAEBIIAAAA|34835|9|40|35|AM|first|morning|breakfast| +34836|AAAAAAAAFBIIAAAA|34836|9|40|36|AM|first|morning|breakfast| +34837|AAAAAAAAGBIIAAAA|34837|9|40|37|AM|first|morning|breakfast| +34838|AAAAAAAAHBIIAAAA|34838|9|40|38|AM|first|morning|breakfast| +34839|AAAAAAAAIBIIAAAA|34839|9|40|39|AM|first|morning|breakfast| +34840|AAAAAAAAJBIIAAAA|34840|9|40|40|AM|first|morning|breakfast| +34841|AAAAAAAAKBIIAAAA|34841|9|40|41|AM|first|morning|breakfast| +34842|AAAAAAAALBIIAAAA|34842|9|40|42|AM|first|morning|breakfast| +34843|AAAAAAAAMBIIAAAA|34843|9|40|43|AM|first|morning|breakfast| +34844|AAAAAAAANBIIAAAA|34844|9|40|44|AM|first|morning|breakfast| +34845|AAAAAAAAOBIIAAAA|34845|9|40|45|AM|first|morning|breakfast| +34846|AAAAAAAAPBIIAAAA|34846|9|40|46|AM|first|morning|breakfast| +34847|AAAAAAAAACIIAAAA|34847|9|40|47|AM|first|morning|breakfast| +34848|AAAAAAAABCIIAAAA|34848|9|40|48|AM|first|morning|breakfast| +34849|AAAAAAAACCIIAAAA|34849|9|40|49|AM|first|morning|breakfast| +34850|AAAAAAAADCIIAAAA|34850|9|40|50|AM|first|morning|breakfast| +34851|AAAAAAAAECIIAAAA|34851|9|40|51|AM|first|morning|breakfast| +34852|AAAAAAAAFCIIAAAA|34852|9|40|52|AM|first|morning|breakfast| +34853|AAAAAAAAGCIIAAAA|34853|9|40|53|AM|first|morning|breakfast| +34854|AAAAAAAAHCIIAAAA|34854|9|40|54|AM|first|morning|breakfast| +34855|AAAAAAAAICIIAAAA|34855|9|40|55|AM|first|morning|breakfast| +34856|AAAAAAAAJCIIAAAA|34856|9|40|56|AM|first|morning|breakfast| +34857|AAAAAAAAKCIIAAAA|34857|9|40|57|AM|first|morning|breakfast| +34858|AAAAAAAALCIIAAAA|34858|9|40|58|AM|first|morning|breakfast| +34859|AAAAAAAAMCIIAAAA|34859|9|40|59|AM|first|morning|breakfast| +34860|AAAAAAAANCIIAAAA|34860|9|41|0|AM|first|morning|breakfast| +34861|AAAAAAAAOCIIAAAA|34861|9|41|1|AM|first|morning|breakfast| +34862|AAAAAAAAPCIIAAAA|34862|9|41|2|AM|first|morning|breakfast| +34863|AAAAAAAAADIIAAAA|34863|9|41|3|AM|first|morning|breakfast| +34864|AAAAAAAABDIIAAAA|34864|9|41|4|AM|first|morning|breakfast| +34865|AAAAAAAACDIIAAAA|34865|9|41|5|AM|first|morning|breakfast| +34866|AAAAAAAADDIIAAAA|34866|9|41|6|AM|first|morning|breakfast| +34867|AAAAAAAAEDIIAAAA|34867|9|41|7|AM|first|morning|breakfast| +34868|AAAAAAAAFDIIAAAA|34868|9|41|8|AM|first|morning|breakfast| +34869|AAAAAAAAGDIIAAAA|34869|9|41|9|AM|first|morning|breakfast| +34870|AAAAAAAAHDIIAAAA|34870|9|41|10|AM|first|morning|breakfast| +34871|AAAAAAAAIDIIAAAA|34871|9|41|11|AM|first|morning|breakfast| +34872|AAAAAAAAJDIIAAAA|34872|9|41|12|AM|first|morning|breakfast| +34873|AAAAAAAAKDIIAAAA|34873|9|41|13|AM|first|morning|breakfast| +34874|AAAAAAAALDIIAAAA|34874|9|41|14|AM|first|morning|breakfast| +34875|AAAAAAAAMDIIAAAA|34875|9|41|15|AM|first|morning|breakfast| +34876|AAAAAAAANDIIAAAA|34876|9|41|16|AM|first|morning|breakfast| +34877|AAAAAAAAODIIAAAA|34877|9|41|17|AM|first|morning|breakfast| +34878|AAAAAAAAPDIIAAAA|34878|9|41|18|AM|first|morning|breakfast| +34879|AAAAAAAAAEIIAAAA|34879|9|41|19|AM|first|morning|breakfast| +34880|AAAAAAAABEIIAAAA|34880|9|41|20|AM|first|morning|breakfast| +34881|AAAAAAAACEIIAAAA|34881|9|41|21|AM|first|morning|breakfast| +34882|AAAAAAAADEIIAAAA|34882|9|41|22|AM|first|morning|breakfast| +34883|AAAAAAAAEEIIAAAA|34883|9|41|23|AM|first|morning|breakfast| +34884|AAAAAAAAFEIIAAAA|34884|9|41|24|AM|first|morning|breakfast| +34885|AAAAAAAAGEIIAAAA|34885|9|41|25|AM|first|morning|breakfast| +34886|AAAAAAAAHEIIAAAA|34886|9|41|26|AM|first|morning|breakfast| +34887|AAAAAAAAIEIIAAAA|34887|9|41|27|AM|first|morning|breakfast| +34888|AAAAAAAAJEIIAAAA|34888|9|41|28|AM|first|morning|breakfast| +34889|AAAAAAAAKEIIAAAA|34889|9|41|29|AM|first|morning|breakfast| +34890|AAAAAAAALEIIAAAA|34890|9|41|30|AM|first|morning|breakfast| +34891|AAAAAAAAMEIIAAAA|34891|9|41|31|AM|first|morning|breakfast| +34892|AAAAAAAANEIIAAAA|34892|9|41|32|AM|first|morning|breakfast| +34893|AAAAAAAAOEIIAAAA|34893|9|41|33|AM|first|morning|breakfast| +34894|AAAAAAAAPEIIAAAA|34894|9|41|34|AM|first|morning|breakfast| +34895|AAAAAAAAAFIIAAAA|34895|9|41|35|AM|first|morning|breakfast| +34896|AAAAAAAABFIIAAAA|34896|9|41|36|AM|first|morning|breakfast| +34897|AAAAAAAACFIIAAAA|34897|9|41|37|AM|first|morning|breakfast| +34898|AAAAAAAADFIIAAAA|34898|9|41|38|AM|first|morning|breakfast| +34899|AAAAAAAAEFIIAAAA|34899|9|41|39|AM|first|morning|breakfast| +34900|AAAAAAAAFFIIAAAA|34900|9|41|40|AM|first|morning|breakfast| +34901|AAAAAAAAGFIIAAAA|34901|9|41|41|AM|first|morning|breakfast| +34902|AAAAAAAAHFIIAAAA|34902|9|41|42|AM|first|morning|breakfast| +34903|AAAAAAAAIFIIAAAA|34903|9|41|43|AM|first|morning|breakfast| +34904|AAAAAAAAJFIIAAAA|34904|9|41|44|AM|first|morning|breakfast| +34905|AAAAAAAAKFIIAAAA|34905|9|41|45|AM|first|morning|breakfast| +34906|AAAAAAAALFIIAAAA|34906|9|41|46|AM|first|morning|breakfast| +34907|AAAAAAAAMFIIAAAA|34907|9|41|47|AM|first|morning|breakfast| +34908|AAAAAAAANFIIAAAA|34908|9|41|48|AM|first|morning|breakfast| +34909|AAAAAAAAOFIIAAAA|34909|9|41|49|AM|first|morning|breakfast| +34910|AAAAAAAAPFIIAAAA|34910|9|41|50|AM|first|morning|breakfast| +34911|AAAAAAAAAGIIAAAA|34911|9|41|51|AM|first|morning|breakfast| +34912|AAAAAAAABGIIAAAA|34912|9|41|52|AM|first|morning|breakfast| +34913|AAAAAAAACGIIAAAA|34913|9|41|53|AM|first|morning|breakfast| +34914|AAAAAAAADGIIAAAA|34914|9|41|54|AM|first|morning|breakfast| +34915|AAAAAAAAEGIIAAAA|34915|9|41|55|AM|first|morning|breakfast| +34916|AAAAAAAAFGIIAAAA|34916|9|41|56|AM|first|morning|breakfast| +34917|AAAAAAAAGGIIAAAA|34917|9|41|57|AM|first|morning|breakfast| +34918|AAAAAAAAHGIIAAAA|34918|9|41|58|AM|first|morning|breakfast| +34919|AAAAAAAAIGIIAAAA|34919|9|41|59|AM|first|morning|breakfast| +34920|AAAAAAAAJGIIAAAA|34920|9|42|0|AM|first|morning|breakfast| +34921|AAAAAAAAKGIIAAAA|34921|9|42|1|AM|first|morning|breakfast| +34922|AAAAAAAALGIIAAAA|34922|9|42|2|AM|first|morning|breakfast| +34923|AAAAAAAAMGIIAAAA|34923|9|42|3|AM|first|morning|breakfast| +34924|AAAAAAAANGIIAAAA|34924|9|42|4|AM|first|morning|breakfast| +34925|AAAAAAAAOGIIAAAA|34925|9|42|5|AM|first|morning|breakfast| +34926|AAAAAAAAPGIIAAAA|34926|9|42|6|AM|first|morning|breakfast| +34927|AAAAAAAAAHIIAAAA|34927|9|42|7|AM|first|morning|breakfast| +34928|AAAAAAAABHIIAAAA|34928|9|42|8|AM|first|morning|breakfast| +34929|AAAAAAAACHIIAAAA|34929|9|42|9|AM|first|morning|breakfast| +34930|AAAAAAAADHIIAAAA|34930|9|42|10|AM|first|morning|breakfast| +34931|AAAAAAAAEHIIAAAA|34931|9|42|11|AM|first|morning|breakfast| +34932|AAAAAAAAFHIIAAAA|34932|9|42|12|AM|first|morning|breakfast| +34933|AAAAAAAAGHIIAAAA|34933|9|42|13|AM|first|morning|breakfast| +34934|AAAAAAAAHHIIAAAA|34934|9|42|14|AM|first|morning|breakfast| +34935|AAAAAAAAIHIIAAAA|34935|9|42|15|AM|first|morning|breakfast| +34936|AAAAAAAAJHIIAAAA|34936|9|42|16|AM|first|morning|breakfast| +34937|AAAAAAAAKHIIAAAA|34937|9|42|17|AM|first|morning|breakfast| +34938|AAAAAAAALHIIAAAA|34938|9|42|18|AM|first|morning|breakfast| +34939|AAAAAAAAMHIIAAAA|34939|9|42|19|AM|first|morning|breakfast| +34940|AAAAAAAANHIIAAAA|34940|9|42|20|AM|first|morning|breakfast| +34941|AAAAAAAAOHIIAAAA|34941|9|42|21|AM|first|morning|breakfast| +34942|AAAAAAAAPHIIAAAA|34942|9|42|22|AM|first|morning|breakfast| +34943|AAAAAAAAAIIIAAAA|34943|9|42|23|AM|first|morning|breakfast| +34944|AAAAAAAABIIIAAAA|34944|9|42|24|AM|first|morning|breakfast| +34945|AAAAAAAACIIIAAAA|34945|9|42|25|AM|first|morning|breakfast| +34946|AAAAAAAADIIIAAAA|34946|9|42|26|AM|first|morning|breakfast| +34947|AAAAAAAAEIIIAAAA|34947|9|42|27|AM|first|morning|breakfast| +34948|AAAAAAAAFIIIAAAA|34948|9|42|28|AM|first|morning|breakfast| +34949|AAAAAAAAGIIIAAAA|34949|9|42|29|AM|first|morning|breakfast| +34950|AAAAAAAAHIIIAAAA|34950|9|42|30|AM|first|morning|breakfast| +34951|AAAAAAAAIIIIAAAA|34951|9|42|31|AM|first|morning|breakfast| +34952|AAAAAAAAJIIIAAAA|34952|9|42|32|AM|first|morning|breakfast| +34953|AAAAAAAAKIIIAAAA|34953|9|42|33|AM|first|morning|breakfast| +34954|AAAAAAAALIIIAAAA|34954|9|42|34|AM|first|morning|breakfast| +34955|AAAAAAAAMIIIAAAA|34955|9|42|35|AM|first|morning|breakfast| +34956|AAAAAAAANIIIAAAA|34956|9|42|36|AM|first|morning|breakfast| +34957|AAAAAAAAOIIIAAAA|34957|9|42|37|AM|first|morning|breakfast| +34958|AAAAAAAAPIIIAAAA|34958|9|42|38|AM|first|morning|breakfast| +34959|AAAAAAAAAJIIAAAA|34959|9|42|39|AM|first|morning|breakfast| +34960|AAAAAAAABJIIAAAA|34960|9|42|40|AM|first|morning|breakfast| +34961|AAAAAAAACJIIAAAA|34961|9|42|41|AM|first|morning|breakfast| +34962|AAAAAAAADJIIAAAA|34962|9|42|42|AM|first|morning|breakfast| +34963|AAAAAAAAEJIIAAAA|34963|9|42|43|AM|first|morning|breakfast| +34964|AAAAAAAAFJIIAAAA|34964|9|42|44|AM|first|morning|breakfast| +34965|AAAAAAAAGJIIAAAA|34965|9|42|45|AM|first|morning|breakfast| +34966|AAAAAAAAHJIIAAAA|34966|9|42|46|AM|first|morning|breakfast| +34967|AAAAAAAAIJIIAAAA|34967|9|42|47|AM|first|morning|breakfast| +34968|AAAAAAAAJJIIAAAA|34968|9|42|48|AM|first|morning|breakfast| +34969|AAAAAAAAKJIIAAAA|34969|9|42|49|AM|first|morning|breakfast| +34970|AAAAAAAALJIIAAAA|34970|9|42|50|AM|first|morning|breakfast| +34971|AAAAAAAAMJIIAAAA|34971|9|42|51|AM|first|morning|breakfast| +34972|AAAAAAAANJIIAAAA|34972|9|42|52|AM|first|morning|breakfast| +34973|AAAAAAAAOJIIAAAA|34973|9|42|53|AM|first|morning|breakfast| +34974|AAAAAAAAPJIIAAAA|34974|9|42|54|AM|first|morning|breakfast| +34975|AAAAAAAAAKIIAAAA|34975|9|42|55|AM|first|morning|breakfast| +34976|AAAAAAAABKIIAAAA|34976|9|42|56|AM|first|morning|breakfast| +34977|AAAAAAAACKIIAAAA|34977|9|42|57|AM|first|morning|breakfast| +34978|AAAAAAAADKIIAAAA|34978|9|42|58|AM|first|morning|breakfast| +34979|AAAAAAAAEKIIAAAA|34979|9|42|59|AM|first|morning|breakfast| +34980|AAAAAAAAFKIIAAAA|34980|9|43|0|AM|first|morning|breakfast| +34981|AAAAAAAAGKIIAAAA|34981|9|43|1|AM|first|morning|breakfast| +34982|AAAAAAAAHKIIAAAA|34982|9|43|2|AM|first|morning|breakfast| +34983|AAAAAAAAIKIIAAAA|34983|9|43|3|AM|first|morning|breakfast| +34984|AAAAAAAAJKIIAAAA|34984|9|43|4|AM|first|morning|breakfast| +34985|AAAAAAAAKKIIAAAA|34985|9|43|5|AM|first|morning|breakfast| +34986|AAAAAAAALKIIAAAA|34986|9|43|6|AM|first|morning|breakfast| +34987|AAAAAAAAMKIIAAAA|34987|9|43|7|AM|first|morning|breakfast| +34988|AAAAAAAANKIIAAAA|34988|9|43|8|AM|first|morning|breakfast| +34989|AAAAAAAAOKIIAAAA|34989|9|43|9|AM|first|morning|breakfast| +34990|AAAAAAAAPKIIAAAA|34990|9|43|10|AM|first|morning|breakfast| +34991|AAAAAAAAALIIAAAA|34991|9|43|11|AM|first|morning|breakfast| +34992|AAAAAAAABLIIAAAA|34992|9|43|12|AM|first|morning|breakfast| +34993|AAAAAAAACLIIAAAA|34993|9|43|13|AM|first|morning|breakfast| +34994|AAAAAAAADLIIAAAA|34994|9|43|14|AM|first|morning|breakfast| +34995|AAAAAAAAELIIAAAA|34995|9|43|15|AM|first|morning|breakfast| +34996|AAAAAAAAFLIIAAAA|34996|9|43|16|AM|first|morning|breakfast| +34997|AAAAAAAAGLIIAAAA|34997|9|43|17|AM|first|morning|breakfast| +34998|AAAAAAAAHLIIAAAA|34998|9|43|18|AM|first|morning|breakfast| +34999|AAAAAAAAILIIAAAA|34999|9|43|19|AM|first|morning|breakfast| +35000|AAAAAAAAJLIIAAAA|35000|9|43|20|AM|first|morning|breakfast| +35001|AAAAAAAAKLIIAAAA|35001|9|43|21|AM|first|morning|breakfast| +35002|AAAAAAAALLIIAAAA|35002|9|43|22|AM|first|morning|breakfast| +35003|AAAAAAAAMLIIAAAA|35003|9|43|23|AM|first|morning|breakfast| +35004|AAAAAAAANLIIAAAA|35004|9|43|24|AM|first|morning|breakfast| +35005|AAAAAAAAOLIIAAAA|35005|9|43|25|AM|first|morning|breakfast| +35006|AAAAAAAAPLIIAAAA|35006|9|43|26|AM|first|morning|breakfast| +35007|AAAAAAAAAMIIAAAA|35007|9|43|27|AM|first|morning|breakfast| +35008|AAAAAAAABMIIAAAA|35008|9|43|28|AM|first|morning|breakfast| +35009|AAAAAAAACMIIAAAA|35009|9|43|29|AM|first|morning|breakfast| +35010|AAAAAAAADMIIAAAA|35010|9|43|30|AM|first|morning|breakfast| +35011|AAAAAAAAEMIIAAAA|35011|9|43|31|AM|first|morning|breakfast| +35012|AAAAAAAAFMIIAAAA|35012|9|43|32|AM|first|morning|breakfast| +35013|AAAAAAAAGMIIAAAA|35013|9|43|33|AM|first|morning|breakfast| +35014|AAAAAAAAHMIIAAAA|35014|9|43|34|AM|first|morning|breakfast| +35015|AAAAAAAAIMIIAAAA|35015|9|43|35|AM|first|morning|breakfast| +35016|AAAAAAAAJMIIAAAA|35016|9|43|36|AM|first|morning|breakfast| +35017|AAAAAAAAKMIIAAAA|35017|9|43|37|AM|first|morning|breakfast| +35018|AAAAAAAALMIIAAAA|35018|9|43|38|AM|first|morning|breakfast| +35019|AAAAAAAAMMIIAAAA|35019|9|43|39|AM|first|morning|breakfast| +35020|AAAAAAAANMIIAAAA|35020|9|43|40|AM|first|morning|breakfast| +35021|AAAAAAAAOMIIAAAA|35021|9|43|41|AM|first|morning|breakfast| +35022|AAAAAAAAPMIIAAAA|35022|9|43|42|AM|first|morning|breakfast| +35023|AAAAAAAAANIIAAAA|35023|9|43|43|AM|first|morning|breakfast| +35024|AAAAAAAABNIIAAAA|35024|9|43|44|AM|first|morning|breakfast| +35025|AAAAAAAACNIIAAAA|35025|9|43|45|AM|first|morning|breakfast| +35026|AAAAAAAADNIIAAAA|35026|9|43|46|AM|first|morning|breakfast| +35027|AAAAAAAAENIIAAAA|35027|9|43|47|AM|first|morning|breakfast| +35028|AAAAAAAAFNIIAAAA|35028|9|43|48|AM|first|morning|breakfast| +35029|AAAAAAAAGNIIAAAA|35029|9|43|49|AM|first|morning|breakfast| +35030|AAAAAAAAHNIIAAAA|35030|9|43|50|AM|first|morning|breakfast| +35031|AAAAAAAAINIIAAAA|35031|9|43|51|AM|first|morning|breakfast| +35032|AAAAAAAAJNIIAAAA|35032|9|43|52|AM|first|morning|breakfast| +35033|AAAAAAAAKNIIAAAA|35033|9|43|53|AM|first|morning|breakfast| +35034|AAAAAAAALNIIAAAA|35034|9|43|54|AM|first|morning|breakfast| +35035|AAAAAAAAMNIIAAAA|35035|9|43|55|AM|first|morning|breakfast| +35036|AAAAAAAANNIIAAAA|35036|9|43|56|AM|first|morning|breakfast| +35037|AAAAAAAAONIIAAAA|35037|9|43|57|AM|first|morning|breakfast| +35038|AAAAAAAAPNIIAAAA|35038|9|43|58|AM|first|morning|breakfast| +35039|AAAAAAAAAOIIAAAA|35039|9|43|59|AM|first|morning|breakfast| +35040|AAAAAAAABOIIAAAA|35040|9|44|0|AM|first|morning|breakfast| +35041|AAAAAAAACOIIAAAA|35041|9|44|1|AM|first|morning|breakfast| +35042|AAAAAAAADOIIAAAA|35042|9|44|2|AM|first|morning|breakfast| +35043|AAAAAAAAEOIIAAAA|35043|9|44|3|AM|first|morning|breakfast| +35044|AAAAAAAAFOIIAAAA|35044|9|44|4|AM|first|morning|breakfast| +35045|AAAAAAAAGOIIAAAA|35045|9|44|5|AM|first|morning|breakfast| +35046|AAAAAAAAHOIIAAAA|35046|9|44|6|AM|first|morning|breakfast| +35047|AAAAAAAAIOIIAAAA|35047|9|44|7|AM|first|morning|breakfast| +35048|AAAAAAAAJOIIAAAA|35048|9|44|8|AM|first|morning|breakfast| +35049|AAAAAAAAKOIIAAAA|35049|9|44|9|AM|first|morning|breakfast| +35050|AAAAAAAALOIIAAAA|35050|9|44|10|AM|first|morning|breakfast| +35051|AAAAAAAAMOIIAAAA|35051|9|44|11|AM|first|morning|breakfast| +35052|AAAAAAAANOIIAAAA|35052|9|44|12|AM|first|morning|breakfast| +35053|AAAAAAAAOOIIAAAA|35053|9|44|13|AM|first|morning|breakfast| +35054|AAAAAAAAPOIIAAAA|35054|9|44|14|AM|first|morning|breakfast| +35055|AAAAAAAAAPIIAAAA|35055|9|44|15|AM|first|morning|breakfast| +35056|AAAAAAAABPIIAAAA|35056|9|44|16|AM|first|morning|breakfast| +35057|AAAAAAAACPIIAAAA|35057|9|44|17|AM|first|morning|breakfast| +35058|AAAAAAAADPIIAAAA|35058|9|44|18|AM|first|morning|breakfast| +35059|AAAAAAAAEPIIAAAA|35059|9|44|19|AM|first|morning|breakfast| +35060|AAAAAAAAFPIIAAAA|35060|9|44|20|AM|first|morning|breakfast| +35061|AAAAAAAAGPIIAAAA|35061|9|44|21|AM|first|morning|breakfast| +35062|AAAAAAAAHPIIAAAA|35062|9|44|22|AM|first|morning|breakfast| +35063|AAAAAAAAIPIIAAAA|35063|9|44|23|AM|first|morning|breakfast| +35064|AAAAAAAAJPIIAAAA|35064|9|44|24|AM|first|morning|breakfast| +35065|AAAAAAAAKPIIAAAA|35065|9|44|25|AM|first|morning|breakfast| +35066|AAAAAAAALPIIAAAA|35066|9|44|26|AM|first|morning|breakfast| +35067|AAAAAAAAMPIIAAAA|35067|9|44|27|AM|first|morning|breakfast| +35068|AAAAAAAANPIIAAAA|35068|9|44|28|AM|first|morning|breakfast| +35069|AAAAAAAAOPIIAAAA|35069|9|44|29|AM|first|morning|breakfast| +35070|AAAAAAAAPPIIAAAA|35070|9|44|30|AM|first|morning|breakfast| +35071|AAAAAAAAAAJIAAAA|35071|9|44|31|AM|first|morning|breakfast| +35072|AAAAAAAABAJIAAAA|35072|9|44|32|AM|first|morning|breakfast| +35073|AAAAAAAACAJIAAAA|35073|9|44|33|AM|first|morning|breakfast| +35074|AAAAAAAADAJIAAAA|35074|9|44|34|AM|first|morning|breakfast| +35075|AAAAAAAAEAJIAAAA|35075|9|44|35|AM|first|morning|breakfast| +35076|AAAAAAAAFAJIAAAA|35076|9|44|36|AM|first|morning|breakfast| +35077|AAAAAAAAGAJIAAAA|35077|9|44|37|AM|first|morning|breakfast| +35078|AAAAAAAAHAJIAAAA|35078|9|44|38|AM|first|morning|breakfast| +35079|AAAAAAAAIAJIAAAA|35079|9|44|39|AM|first|morning|breakfast| +35080|AAAAAAAAJAJIAAAA|35080|9|44|40|AM|first|morning|breakfast| +35081|AAAAAAAAKAJIAAAA|35081|9|44|41|AM|first|morning|breakfast| +35082|AAAAAAAALAJIAAAA|35082|9|44|42|AM|first|morning|breakfast| +35083|AAAAAAAAMAJIAAAA|35083|9|44|43|AM|first|morning|breakfast| +35084|AAAAAAAANAJIAAAA|35084|9|44|44|AM|first|morning|breakfast| +35085|AAAAAAAAOAJIAAAA|35085|9|44|45|AM|first|morning|breakfast| +35086|AAAAAAAAPAJIAAAA|35086|9|44|46|AM|first|morning|breakfast| +35087|AAAAAAAAABJIAAAA|35087|9|44|47|AM|first|morning|breakfast| +35088|AAAAAAAABBJIAAAA|35088|9|44|48|AM|first|morning|breakfast| +35089|AAAAAAAACBJIAAAA|35089|9|44|49|AM|first|morning|breakfast| +35090|AAAAAAAADBJIAAAA|35090|9|44|50|AM|first|morning|breakfast| +35091|AAAAAAAAEBJIAAAA|35091|9|44|51|AM|first|morning|breakfast| +35092|AAAAAAAAFBJIAAAA|35092|9|44|52|AM|first|morning|breakfast| +35093|AAAAAAAAGBJIAAAA|35093|9|44|53|AM|first|morning|breakfast| +35094|AAAAAAAAHBJIAAAA|35094|9|44|54|AM|first|morning|breakfast| +35095|AAAAAAAAIBJIAAAA|35095|9|44|55|AM|first|morning|breakfast| +35096|AAAAAAAAJBJIAAAA|35096|9|44|56|AM|first|morning|breakfast| +35097|AAAAAAAAKBJIAAAA|35097|9|44|57|AM|first|morning|breakfast| +35098|AAAAAAAALBJIAAAA|35098|9|44|58|AM|first|morning|breakfast| +35099|AAAAAAAAMBJIAAAA|35099|9|44|59|AM|first|morning|breakfast| +35100|AAAAAAAANBJIAAAA|35100|9|45|0|AM|first|morning|breakfast| +35101|AAAAAAAAOBJIAAAA|35101|9|45|1|AM|first|morning|breakfast| +35102|AAAAAAAAPBJIAAAA|35102|9|45|2|AM|first|morning|breakfast| +35103|AAAAAAAAACJIAAAA|35103|9|45|3|AM|first|morning|breakfast| +35104|AAAAAAAABCJIAAAA|35104|9|45|4|AM|first|morning|breakfast| +35105|AAAAAAAACCJIAAAA|35105|9|45|5|AM|first|morning|breakfast| +35106|AAAAAAAADCJIAAAA|35106|9|45|6|AM|first|morning|breakfast| +35107|AAAAAAAAECJIAAAA|35107|9|45|7|AM|first|morning|breakfast| +35108|AAAAAAAAFCJIAAAA|35108|9|45|8|AM|first|morning|breakfast| +35109|AAAAAAAAGCJIAAAA|35109|9|45|9|AM|first|morning|breakfast| +35110|AAAAAAAAHCJIAAAA|35110|9|45|10|AM|first|morning|breakfast| +35111|AAAAAAAAICJIAAAA|35111|9|45|11|AM|first|morning|breakfast| +35112|AAAAAAAAJCJIAAAA|35112|9|45|12|AM|first|morning|breakfast| +35113|AAAAAAAAKCJIAAAA|35113|9|45|13|AM|first|morning|breakfast| +35114|AAAAAAAALCJIAAAA|35114|9|45|14|AM|first|morning|breakfast| +35115|AAAAAAAAMCJIAAAA|35115|9|45|15|AM|first|morning|breakfast| +35116|AAAAAAAANCJIAAAA|35116|9|45|16|AM|first|morning|breakfast| +35117|AAAAAAAAOCJIAAAA|35117|9|45|17|AM|first|morning|breakfast| +35118|AAAAAAAAPCJIAAAA|35118|9|45|18|AM|first|morning|breakfast| +35119|AAAAAAAAADJIAAAA|35119|9|45|19|AM|first|morning|breakfast| +35120|AAAAAAAABDJIAAAA|35120|9|45|20|AM|first|morning|breakfast| +35121|AAAAAAAACDJIAAAA|35121|9|45|21|AM|first|morning|breakfast| +35122|AAAAAAAADDJIAAAA|35122|9|45|22|AM|first|morning|breakfast| +35123|AAAAAAAAEDJIAAAA|35123|9|45|23|AM|first|morning|breakfast| +35124|AAAAAAAAFDJIAAAA|35124|9|45|24|AM|first|morning|breakfast| +35125|AAAAAAAAGDJIAAAA|35125|9|45|25|AM|first|morning|breakfast| +35126|AAAAAAAAHDJIAAAA|35126|9|45|26|AM|first|morning|breakfast| +35127|AAAAAAAAIDJIAAAA|35127|9|45|27|AM|first|morning|breakfast| +35128|AAAAAAAAJDJIAAAA|35128|9|45|28|AM|first|morning|breakfast| +35129|AAAAAAAAKDJIAAAA|35129|9|45|29|AM|first|morning|breakfast| +35130|AAAAAAAALDJIAAAA|35130|9|45|30|AM|first|morning|breakfast| +35131|AAAAAAAAMDJIAAAA|35131|9|45|31|AM|first|morning|breakfast| +35132|AAAAAAAANDJIAAAA|35132|9|45|32|AM|first|morning|breakfast| +35133|AAAAAAAAODJIAAAA|35133|9|45|33|AM|first|morning|breakfast| +35134|AAAAAAAAPDJIAAAA|35134|9|45|34|AM|first|morning|breakfast| +35135|AAAAAAAAAEJIAAAA|35135|9|45|35|AM|first|morning|breakfast| +35136|AAAAAAAABEJIAAAA|35136|9|45|36|AM|first|morning|breakfast| +35137|AAAAAAAACEJIAAAA|35137|9|45|37|AM|first|morning|breakfast| +35138|AAAAAAAADEJIAAAA|35138|9|45|38|AM|first|morning|breakfast| +35139|AAAAAAAAEEJIAAAA|35139|9|45|39|AM|first|morning|breakfast| +35140|AAAAAAAAFEJIAAAA|35140|9|45|40|AM|first|morning|breakfast| +35141|AAAAAAAAGEJIAAAA|35141|9|45|41|AM|first|morning|breakfast| +35142|AAAAAAAAHEJIAAAA|35142|9|45|42|AM|first|morning|breakfast| +35143|AAAAAAAAIEJIAAAA|35143|9|45|43|AM|first|morning|breakfast| +35144|AAAAAAAAJEJIAAAA|35144|9|45|44|AM|first|morning|breakfast| +35145|AAAAAAAAKEJIAAAA|35145|9|45|45|AM|first|morning|breakfast| +35146|AAAAAAAALEJIAAAA|35146|9|45|46|AM|first|morning|breakfast| +35147|AAAAAAAAMEJIAAAA|35147|9|45|47|AM|first|morning|breakfast| +35148|AAAAAAAANEJIAAAA|35148|9|45|48|AM|first|morning|breakfast| +35149|AAAAAAAAOEJIAAAA|35149|9|45|49|AM|first|morning|breakfast| +35150|AAAAAAAAPEJIAAAA|35150|9|45|50|AM|first|morning|breakfast| +35151|AAAAAAAAAFJIAAAA|35151|9|45|51|AM|first|morning|breakfast| +35152|AAAAAAAABFJIAAAA|35152|9|45|52|AM|first|morning|breakfast| +35153|AAAAAAAACFJIAAAA|35153|9|45|53|AM|first|morning|breakfast| +35154|AAAAAAAADFJIAAAA|35154|9|45|54|AM|first|morning|breakfast| +35155|AAAAAAAAEFJIAAAA|35155|9|45|55|AM|first|morning|breakfast| +35156|AAAAAAAAFFJIAAAA|35156|9|45|56|AM|first|morning|breakfast| +35157|AAAAAAAAGFJIAAAA|35157|9|45|57|AM|first|morning|breakfast| +35158|AAAAAAAAHFJIAAAA|35158|9|45|58|AM|first|morning|breakfast| +35159|AAAAAAAAIFJIAAAA|35159|9|45|59|AM|first|morning|breakfast| +35160|AAAAAAAAJFJIAAAA|35160|9|46|0|AM|first|morning|breakfast| +35161|AAAAAAAAKFJIAAAA|35161|9|46|1|AM|first|morning|breakfast| +35162|AAAAAAAALFJIAAAA|35162|9|46|2|AM|first|morning|breakfast| +35163|AAAAAAAAMFJIAAAA|35163|9|46|3|AM|first|morning|breakfast| +35164|AAAAAAAANFJIAAAA|35164|9|46|4|AM|first|morning|breakfast| +35165|AAAAAAAAOFJIAAAA|35165|9|46|5|AM|first|morning|breakfast| +35166|AAAAAAAAPFJIAAAA|35166|9|46|6|AM|first|morning|breakfast| +35167|AAAAAAAAAGJIAAAA|35167|9|46|7|AM|first|morning|breakfast| +35168|AAAAAAAABGJIAAAA|35168|9|46|8|AM|first|morning|breakfast| +35169|AAAAAAAACGJIAAAA|35169|9|46|9|AM|first|morning|breakfast| +35170|AAAAAAAADGJIAAAA|35170|9|46|10|AM|first|morning|breakfast| +35171|AAAAAAAAEGJIAAAA|35171|9|46|11|AM|first|morning|breakfast| +35172|AAAAAAAAFGJIAAAA|35172|9|46|12|AM|first|morning|breakfast| +35173|AAAAAAAAGGJIAAAA|35173|9|46|13|AM|first|morning|breakfast| +35174|AAAAAAAAHGJIAAAA|35174|9|46|14|AM|first|morning|breakfast| +35175|AAAAAAAAIGJIAAAA|35175|9|46|15|AM|first|morning|breakfast| +35176|AAAAAAAAJGJIAAAA|35176|9|46|16|AM|first|morning|breakfast| +35177|AAAAAAAAKGJIAAAA|35177|9|46|17|AM|first|morning|breakfast| +35178|AAAAAAAALGJIAAAA|35178|9|46|18|AM|first|morning|breakfast| +35179|AAAAAAAAMGJIAAAA|35179|9|46|19|AM|first|morning|breakfast| +35180|AAAAAAAANGJIAAAA|35180|9|46|20|AM|first|morning|breakfast| +35181|AAAAAAAAOGJIAAAA|35181|9|46|21|AM|first|morning|breakfast| +35182|AAAAAAAAPGJIAAAA|35182|9|46|22|AM|first|morning|breakfast| +35183|AAAAAAAAAHJIAAAA|35183|9|46|23|AM|first|morning|breakfast| +35184|AAAAAAAABHJIAAAA|35184|9|46|24|AM|first|morning|breakfast| +35185|AAAAAAAACHJIAAAA|35185|9|46|25|AM|first|morning|breakfast| +35186|AAAAAAAADHJIAAAA|35186|9|46|26|AM|first|morning|breakfast| +35187|AAAAAAAAEHJIAAAA|35187|9|46|27|AM|first|morning|breakfast| +35188|AAAAAAAAFHJIAAAA|35188|9|46|28|AM|first|morning|breakfast| +35189|AAAAAAAAGHJIAAAA|35189|9|46|29|AM|first|morning|breakfast| +35190|AAAAAAAAHHJIAAAA|35190|9|46|30|AM|first|morning|breakfast| +35191|AAAAAAAAIHJIAAAA|35191|9|46|31|AM|first|morning|breakfast| +35192|AAAAAAAAJHJIAAAA|35192|9|46|32|AM|first|morning|breakfast| +35193|AAAAAAAAKHJIAAAA|35193|9|46|33|AM|first|morning|breakfast| +35194|AAAAAAAALHJIAAAA|35194|9|46|34|AM|first|morning|breakfast| +35195|AAAAAAAAMHJIAAAA|35195|9|46|35|AM|first|morning|breakfast| +35196|AAAAAAAANHJIAAAA|35196|9|46|36|AM|first|morning|breakfast| +35197|AAAAAAAAOHJIAAAA|35197|9|46|37|AM|first|morning|breakfast| +35198|AAAAAAAAPHJIAAAA|35198|9|46|38|AM|first|morning|breakfast| +35199|AAAAAAAAAIJIAAAA|35199|9|46|39|AM|first|morning|breakfast| +35200|AAAAAAAABIJIAAAA|35200|9|46|40|AM|first|morning|breakfast| +35201|AAAAAAAACIJIAAAA|35201|9|46|41|AM|first|morning|breakfast| +35202|AAAAAAAADIJIAAAA|35202|9|46|42|AM|first|morning|breakfast| +35203|AAAAAAAAEIJIAAAA|35203|9|46|43|AM|first|morning|breakfast| +35204|AAAAAAAAFIJIAAAA|35204|9|46|44|AM|first|morning|breakfast| +35205|AAAAAAAAGIJIAAAA|35205|9|46|45|AM|first|morning|breakfast| +35206|AAAAAAAAHIJIAAAA|35206|9|46|46|AM|first|morning|breakfast| +35207|AAAAAAAAIIJIAAAA|35207|9|46|47|AM|first|morning|breakfast| +35208|AAAAAAAAJIJIAAAA|35208|9|46|48|AM|first|morning|breakfast| +35209|AAAAAAAAKIJIAAAA|35209|9|46|49|AM|first|morning|breakfast| +35210|AAAAAAAALIJIAAAA|35210|9|46|50|AM|first|morning|breakfast| +35211|AAAAAAAAMIJIAAAA|35211|9|46|51|AM|first|morning|breakfast| +35212|AAAAAAAANIJIAAAA|35212|9|46|52|AM|first|morning|breakfast| +35213|AAAAAAAAOIJIAAAA|35213|9|46|53|AM|first|morning|breakfast| +35214|AAAAAAAAPIJIAAAA|35214|9|46|54|AM|first|morning|breakfast| +35215|AAAAAAAAAJJIAAAA|35215|9|46|55|AM|first|morning|breakfast| +35216|AAAAAAAABJJIAAAA|35216|9|46|56|AM|first|morning|breakfast| +35217|AAAAAAAACJJIAAAA|35217|9|46|57|AM|first|morning|breakfast| +35218|AAAAAAAADJJIAAAA|35218|9|46|58|AM|first|morning|breakfast| +35219|AAAAAAAAEJJIAAAA|35219|9|46|59|AM|first|morning|breakfast| +35220|AAAAAAAAFJJIAAAA|35220|9|47|0|AM|first|morning|breakfast| +35221|AAAAAAAAGJJIAAAA|35221|9|47|1|AM|first|morning|breakfast| +35222|AAAAAAAAHJJIAAAA|35222|9|47|2|AM|first|morning|breakfast| +35223|AAAAAAAAIJJIAAAA|35223|9|47|3|AM|first|morning|breakfast| +35224|AAAAAAAAJJJIAAAA|35224|9|47|4|AM|first|morning|breakfast| +35225|AAAAAAAAKJJIAAAA|35225|9|47|5|AM|first|morning|breakfast| +35226|AAAAAAAALJJIAAAA|35226|9|47|6|AM|first|morning|breakfast| +35227|AAAAAAAAMJJIAAAA|35227|9|47|7|AM|first|morning|breakfast| +35228|AAAAAAAANJJIAAAA|35228|9|47|8|AM|first|morning|breakfast| +35229|AAAAAAAAOJJIAAAA|35229|9|47|9|AM|first|morning|breakfast| +35230|AAAAAAAAPJJIAAAA|35230|9|47|10|AM|first|morning|breakfast| +35231|AAAAAAAAAKJIAAAA|35231|9|47|11|AM|first|morning|breakfast| +35232|AAAAAAAABKJIAAAA|35232|9|47|12|AM|first|morning|breakfast| +35233|AAAAAAAACKJIAAAA|35233|9|47|13|AM|first|morning|breakfast| +35234|AAAAAAAADKJIAAAA|35234|9|47|14|AM|first|morning|breakfast| +35235|AAAAAAAAEKJIAAAA|35235|9|47|15|AM|first|morning|breakfast| +35236|AAAAAAAAFKJIAAAA|35236|9|47|16|AM|first|morning|breakfast| +35237|AAAAAAAAGKJIAAAA|35237|9|47|17|AM|first|morning|breakfast| +35238|AAAAAAAAHKJIAAAA|35238|9|47|18|AM|first|morning|breakfast| +35239|AAAAAAAAIKJIAAAA|35239|9|47|19|AM|first|morning|breakfast| +35240|AAAAAAAAJKJIAAAA|35240|9|47|20|AM|first|morning|breakfast| +35241|AAAAAAAAKKJIAAAA|35241|9|47|21|AM|first|morning|breakfast| +35242|AAAAAAAALKJIAAAA|35242|9|47|22|AM|first|morning|breakfast| +35243|AAAAAAAAMKJIAAAA|35243|9|47|23|AM|first|morning|breakfast| +35244|AAAAAAAANKJIAAAA|35244|9|47|24|AM|first|morning|breakfast| +35245|AAAAAAAAOKJIAAAA|35245|9|47|25|AM|first|morning|breakfast| +35246|AAAAAAAAPKJIAAAA|35246|9|47|26|AM|first|morning|breakfast| +35247|AAAAAAAAALJIAAAA|35247|9|47|27|AM|first|morning|breakfast| +35248|AAAAAAAABLJIAAAA|35248|9|47|28|AM|first|morning|breakfast| +35249|AAAAAAAACLJIAAAA|35249|9|47|29|AM|first|morning|breakfast| +35250|AAAAAAAADLJIAAAA|35250|9|47|30|AM|first|morning|breakfast| +35251|AAAAAAAAELJIAAAA|35251|9|47|31|AM|first|morning|breakfast| +35252|AAAAAAAAFLJIAAAA|35252|9|47|32|AM|first|morning|breakfast| +35253|AAAAAAAAGLJIAAAA|35253|9|47|33|AM|first|morning|breakfast| +35254|AAAAAAAAHLJIAAAA|35254|9|47|34|AM|first|morning|breakfast| +35255|AAAAAAAAILJIAAAA|35255|9|47|35|AM|first|morning|breakfast| +35256|AAAAAAAAJLJIAAAA|35256|9|47|36|AM|first|morning|breakfast| +35257|AAAAAAAAKLJIAAAA|35257|9|47|37|AM|first|morning|breakfast| +35258|AAAAAAAALLJIAAAA|35258|9|47|38|AM|first|morning|breakfast| +35259|AAAAAAAAMLJIAAAA|35259|9|47|39|AM|first|morning|breakfast| +35260|AAAAAAAANLJIAAAA|35260|9|47|40|AM|first|morning|breakfast| +35261|AAAAAAAAOLJIAAAA|35261|9|47|41|AM|first|morning|breakfast| +35262|AAAAAAAAPLJIAAAA|35262|9|47|42|AM|first|morning|breakfast| +35263|AAAAAAAAAMJIAAAA|35263|9|47|43|AM|first|morning|breakfast| +35264|AAAAAAAABMJIAAAA|35264|9|47|44|AM|first|morning|breakfast| +35265|AAAAAAAACMJIAAAA|35265|9|47|45|AM|first|morning|breakfast| +35266|AAAAAAAADMJIAAAA|35266|9|47|46|AM|first|morning|breakfast| +35267|AAAAAAAAEMJIAAAA|35267|9|47|47|AM|first|morning|breakfast| +35268|AAAAAAAAFMJIAAAA|35268|9|47|48|AM|first|morning|breakfast| +35269|AAAAAAAAGMJIAAAA|35269|9|47|49|AM|first|morning|breakfast| +35270|AAAAAAAAHMJIAAAA|35270|9|47|50|AM|first|morning|breakfast| +35271|AAAAAAAAIMJIAAAA|35271|9|47|51|AM|first|morning|breakfast| +35272|AAAAAAAAJMJIAAAA|35272|9|47|52|AM|first|morning|breakfast| +35273|AAAAAAAAKMJIAAAA|35273|9|47|53|AM|first|morning|breakfast| +35274|AAAAAAAALMJIAAAA|35274|9|47|54|AM|first|morning|breakfast| +35275|AAAAAAAAMMJIAAAA|35275|9|47|55|AM|first|morning|breakfast| +35276|AAAAAAAANMJIAAAA|35276|9|47|56|AM|first|morning|breakfast| +35277|AAAAAAAAOMJIAAAA|35277|9|47|57|AM|first|morning|breakfast| +35278|AAAAAAAAPMJIAAAA|35278|9|47|58|AM|first|morning|breakfast| +35279|AAAAAAAAANJIAAAA|35279|9|47|59|AM|first|morning|breakfast| +35280|AAAAAAAABNJIAAAA|35280|9|48|0|AM|first|morning|breakfast| +35281|AAAAAAAACNJIAAAA|35281|9|48|1|AM|first|morning|breakfast| +35282|AAAAAAAADNJIAAAA|35282|9|48|2|AM|first|morning|breakfast| +35283|AAAAAAAAENJIAAAA|35283|9|48|3|AM|first|morning|breakfast| +35284|AAAAAAAAFNJIAAAA|35284|9|48|4|AM|first|morning|breakfast| +35285|AAAAAAAAGNJIAAAA|35285|9|48|5|AM|first|morning|breakfast| +35286|AAAAAAAAHNJIAAAA|35286|9|48|6|AM|first|morning|breakfast| +35287|AAAAAAAAINJIAAAA|35287|9|48|7|AM|first|morning|breakfast| +35288|AAAAAAAAJNJIAAAA|35288|9|48|8|AM|first|morning|breakfast| +35289|AAAAAAAAKNJIAAAA|35289|9|48|9|AM|first|morning|breakfast| +35290|AAAAAAAALNJIAAAA|35290|9|48|10|AM|first|morning|breakfast| +35291|AAAAAAAAMNJIAAAA|35291|9|48|11|AM|first|morning|breakfast| +35292|AAAAAAAANNJIAAAA|35292|9|48|12|AM|first|morning|breakfast| +35293|AAAAAAAAONJIAAAA|35293|9|48|13|AM|first|morning|breakfast| +35294|AAAAAAAAPNJIAAAA|35294|9|48|14|AM|first|morning|breakfast| +35295|AAAAAAAAAOJIAAAA|35295|9|48|15|AM|first|morning|breakfast| +35296|AAAAAAAABOJIAAAA|35296|9|48|16|AM|first|morning|breakfast| +35297|AAAAAAAACOJIAAAA|35297|9|48|17|AM|first|morning|breakfast| +35298|AAAAAAAADOJIAAAA|35298|9|48|18|AM|first|morning|breakfast| +35299|AAAAAAAAEOJIAAAA|35299|9|48|19|AM|first|morning|breakfast| +35300|AAAAAAAAFOJIAAAA|35300|9|48|20|AM|first|morning|breakfast| +35301|AAAAAAAAGOJIAAAA|35301|9|48|21|AM|first|morning|breakfast| +35302|AAAAAAAAHOJIAAAA|35302|9|48|22|AM|first|morning|breakfast| +35303|AAAAAAAAIOJIAAAA|35303|9|48|23|AM|first|morning|breakfast| +35304|AAAAAAAAJOJIAAAA|35304|9|48|24|AM|first|morning|breakfast| +35305|AAAAAAAAKOJIAAAA|35305|9|48|25|AM|first|morning|breakfast| +35306|AAAAAAAALOJIAAAA|35306|9|48|26|AM|first|morning|breakfast| +35307|AAAAAAAAMOJIAAAA|35307|9|48|27|AM|first|morning|breakfast| +35308|AAAAAAAANOJIAAAA|35308|9|48|28|AM|first|morning|breakfast| +35309|AAAAAAAAOOJIAAAA|35309|9|48|29|AM|first|morning|breakfast| +35310|AAAAAAAAPOJIAAAA|35310|9|48|30|AM|first|morning|breakfast| +35311|AAAAAAAAAPJIAAAA|35311|9|48|31|AM|first|morning|breakfast| +35312|AAAAAAAABPJIAAAA|35312|9|48|32|AM|first|morning|breakfast| +35313|AAAAAAAACPJIAAAA|35313|9|48|33|AM|first|morning|breakfast| +35314|AAAAAAAADPJIAAAA|35314|9|48|34|AM|first|morning|breakfast| +35315|AAAAAAAAEPJIAAAA|35315|9|48|35|AM|first|morning|breakfast| +35316|AAAAAAAAFPJIAAAA|35316|9|48|36|AM|first|morning|breakfast| +35317|AAAAAAAAGPJIAAAA|35317|9|48|37|AM|first|morning|breakfast| +35318|AAAAAAAAHPJIAAAA|35318|9|48|38|AM|first|morning|breakfast| +35319|AAAAAAAAIPJIAAAA|35319|9|48|39|AM|first|morning|breakfast| +35320|AAAAAAAAJPJIAAAA|35320|9|48|40|AM|first|morning|breakfast| +35321|AAAAAAAAKPJIAAAA|35321|9|48|41|AM|first|morning|breakfast| +35322|AAAAAAAALPJIAAAA|35322|9|48|42|AM|first|morning|breakfast| +35323|AAAAAAAAMPJIAAAA|35323|9|48|43|AM|first|morning|breakfast| +35324|AAAAAAAANPJIAAAA|35324|9|48|44|AM|first|morning|breakfast| +35325|AAAAAAAAOPJIAAAA|35325|9|48|45|AM|first|morning|breakfast| +35326|AAAAAAAAPPJIAAAA|35326|9|48|46|AM|first|morning|breakfast| +35327|AAAAAAAAAAKIAAAA|35327|9|48|47|AM|first|morning|breakfast| +35328|AAAAAAAABAKIAAAA|35328|9|48|48|AM|first|morning|breakfast| +35329|AAAAAAAACAKIAAAA|35329|9|48|49|AM|first|morning|breakfast| +35330|AAAAAAAADAKIAAAA|35330|9|48|50|AM|first|morning|breakfast| +35331|AAAAAAAAEAKIAAAA|35331|9|48|51|AM|first|morning|breakfast| +35332|AAAAAAAAFAKIAAAA|35332|9|48|52|AM|first|morning|breakfast| +35333|AAAAAAAAGAKIAAAA|35333|9|48|53|AM|first|morning|breakfast| +35334|AAAAAAAAHAKIAAAA|35334|9|48|54|AM|first|morning|breakfast| +35335|AAAAAAAAIAKIAAAA|35335|9|48|55|AM|first|morning|breakfast| +35336|AAAAAAAAJAKIAAAA|35336|9|48|56|AM|first|morning|breakfast| +35337|AAAAAAAAKAKIAAAA|35337|9|48|57|AM|first|morning|breakfast| +35338|AAAAAAAALAKIAAAA|35338|9|48|58|AM|first|morning|breakfast| +35339|AAAAAAAAMAKIAAAA|35339|9|48|59|AM|first|morning|breakfast| +35340|AAAAAAAANAKIAAAA|35340|9|49|0|AM|first|morning|breakfast| +35341|AAAAAAAAOAKIAAAA|35341|9|49|1|AM|first|morning|breakfast| +35342|AAAAAAAAPAKIAAAA|35342|9|49|2|AM|first|morning|breakfast| +35343|AAAAAAAAABKIAAAA|35343|9|49|3|AM|first|morning|breakfast| +35344|AAAAAAAABBKIAAAA|35344|9|49|4|AM|first|morning|breakfast| +35345|AAAAAAAACBKIAAAA|35345|9|49|5|AM|first|morning|breakfast| +35346|AAAAAAAADBKIAAAA|35346|9|49|6|AM|first|morning|breakfast| +35347|AAAAAAAAEBKIAAAA|35347|9|49|7|AM|first|morning|breakfast| +35348|AAAAAAAAFBKIAAAA|35348|9|49|8|AM|first|morning|breakfast| +35349|AAAAAAAAGBKIAAAA|35349|9|49|9|AM|first|morning|breakfast| +35350|AAAAAAAAHBKIAAAA|35350|9|49|10|AM|first|morning|breakfast| +35351|AAAAAAAAIBKIAAAA|35351|9|49|11|AM|first|morning|breakfast| +35352|AAAAAAAAJBKIAAAA|35352|9|49|12|AM|first|morning|breakfast| +35353|AAAAAAAAKBKIAAAA|35353|9|49|13|AM|first|morning|breakfast| +35354|AAAAAAAALBKIAAAA|35354|9|49|14|AM|first|morning|breakfast| +35355|AAAAAAAAMBKIAAAA|35355|9|49|15|AM|first|morning|breakfast| +35356|AAAAAAAANBKIAAAA|35356|9|49|16|AM|first|morning|breakfast| +35357|AAAAAAAAOBKIAAAA|35357|9|49|17|AM|first|morning|breakfast| +35358|AAAAAAAAPBKIAAAA|35358|9|49|18|AM|first|morning|breakfast| +35359|AAAAAAAAACKIAAAA|35359|9|49|19|AM|first|morning|breakfast| +35360|AAAAAAAABCKIAAAA|35360|9|49|20|AM|first|morning|breakfast| +35361|AAAAAAAACCKIAAAA|35361|9|49|21|AM|first|morning|breakfast| +35362|AAAAAAAADCKIAAAA|35362|9|49|22|AM|first|morning|breakfast| +35363|AAAAAAAAECKIAAAA|35363|9|49|23|AM|first|morning|breakfast| +35364|AAAAAAAAFCKIAAAA|35364|9|49|24|AM|first|morning|breakfast| +35365|AAAAAAAAGCKIAAAA|35365|9|49|25|AM|first|morning|breakfast| +35366|AAAAAAAAHCKIAAAA|35366|9|49|26|AM|first|morning|breakfast| +35367|AAAAAAAAICKIAAAA|35367|9|49|27|AM|first|morning|breakfast| +35368|AAAAAAAAJCKIAAAA|35368|9|49|28|AM|first|morning|breakfast| +35369|AAAAAAAAKCKIAAAA|35369|9|49|29|AM|first|morning|breakfast| +35370|AAAAAAAALCKIAAAA|35370|9|49|30|AM|first|morning|breakfast| +35371|AAAAAAAAMCKIAAAA|35371|9|49|31|AM|first|morning|breakfast| +35372|AAAAAAAANCKIAAAA|35372|9|49|32|AM|first|morning|breakfast| +35373|AAAAAAAAOCKIAAAA|35373|9|49|33|AM|first|morning|breakfast| +35374|AAAAAAAAPCKIAAAA|35374|9|49|34|AM|first|morning|breakfast| +35375|AAAAAAAAADKIAAAA|35375|9|49|35|AM|first|morning|breakfast| +35376|AAAAAAAABDKIAAAA|35376|9|49|36|AM|first|morning|breakfast| +35377|AAAAAAAACDKIAAAA|35377|9|49|37|AM|first|morning|breakfast| +35378|AAAAAAAADDKIAAAA|35378|9|49|38|AM|first|morning|breakfast| +35379|AAAAAAAAEDKIAAAA|35379|9|49|39|AM|first|morning|breakfast| +35380|AAAAAAAAFDKIAAAA|35380|9|49|40|AM|first|morning|breakfast| +35381|AAAAAAAAGDKIAAAA|35381|9|49|41|AM|first|morning|breakfast| +35382|AAAAAAAAHDKIAAAA|35382|9|49|42|AM|first|morning|breakfast| +35383|AAAAAAAAIDKIAAAA|35383|9|49|43|AM|first|morning|breakfast| +35384|AAAAAAAAJDKIAAAA|35384|9|49|44|AM|first|morning|breakfast| +35385|AAAAAAAAKDKIAAAA|35385|9|49|45|AM|first|morning|breakfast| +35386|AAAAAAAALDKIAAAA|35386|9|49|46|AM|first|morning|breakfast| +35387|AAAAAAAAMDKIAAAA|35387|9|49|47|AM|first|morning|breakfast| +35388|AAAAAAAANDKIAAAA|35388|9|49|48|AM|first|morning|breakfast| +35389|AAAAAAAAODKIAAAA|35389|9|49|49|AM|first|morning|breakfast| +35390|AAAAAAAAPDKIAAAA|35390|9|49|50|AM|first|morning|breakfast| +35391|AAAAAAAAAEKIAAAA|35391|9|49|51|AM|first|morning|breakfast| +35392|AAAAAAAABEKIAAAA|35392|9|49|52|AM|first|morning|breakfast| +35393|AAAAAAAACEKIAAAA|35393|9|49|53|AM|first|morning|breakfast| +35394|AAAAAAAADEKIAAAA|35394|9|49|54|AM|first|morning|breakfast| +35395|AAAAAAAAEEKIAAAA|35395|9|49|55|AM|first|morning|breakfast| +35396|AAAAAAAAFEKIAAAA|35396|9|49|56|AM|first|morning|breakfast| +35397|AAAAAAAAGEKIAAAA|35397|9|49|57|AM|first|morning|breakfast| +35398|AAAAAAAAHEKIAAAA|35398|9|49|58|AM|first|morning|breakfast| +35399|AAAAAAAAIEKIAAAA|35399|9|49|59|AM|first|morning|breakfast| +35400|AAAAAAAAJEKIAAAA|35400|9|50|0|AM|first|morning|breakfast| +35401|AAAAAAAAKEKIAAAA|35401|9|50|1|AM|first|morning|breakfast| +35402|AAAAAAAALEKIAAAA|35402|9|50|2|AM|first|morning|breakfast| +35403|AAAAAAAAMEKIAAAA|35403|9|50|3|AM|first|morning|breakfast| +35404|AAAAAAAANEKIAAAA|35404|9|50|4|AM|first|morning|breakfast| +35405|AAAAAAAAOEKIAAAA|35405|9|50|5|AM|first|morning|breakfast| +35406|AAAAAAAAPEKIAAAA|35406|9|50|6|AM|first|morning|breakfast| +35407|AAAAAAAAAFKIAAAA|35407|9|50|7|AM|first|morning|breakfast| +35408|AAAAAAAABFKIAAAA|35408|9|50|8|AM|first|morning|breakfast| +35409|AAAAAAAACFKIAAAA|35409|9|50|9|AM|first|morning|breakfast| +35410|AAAAAAAADFKIAAAA|35410|9|50|10|AM|first|morning|breakfast| +35411|AAAAAAAAEFKIAAAA|35411|9|50|11|AM|first|morning|breakfast| +35412|AAAAAAAAFFKIAAAA|35412|9|50|12|AM|first|morning|breakfast| +35413|AAAAAAAAGFKIAAAA|35413|9|50|13|AM|first|morning|breakfast| +35414|AAAAAAAAHFKIAAAA|35414|9|50|14|AM|first|morning|breakfast| +35415|AAAAAAAAIFKIAAAA|35415|9|50|15|AM|first|morning|breakfast| +35416|AAAAAAAAJFKIAAAA|35416|9|50|16|AM|first|morning|breakfast| +35417|AAAAAAAAKFKIAAAA|35417|9|50|17|AM|first|morning|breakfast| +35418|AAAAAAAALFKIAAAA|35418|9|50|18|AM|first|morning|breakfast| +35419|AAAAAAAAMFKIAAAA|35419|9|50|19|AM|first|morning|breakfast| +35420|AAAAAAAANFKIAAAA|35420|9|50|20|AM|first|morning|breakfast| +35421|AAAAAAAAOFKIAAAA|35421|9|50|21|AM|first|morning|breakfast| +35422|AAAAAAAAPFKIAAAA|35422|9|50|22|AM|first|morning|breakfast| +35423|AAAAAAAAAGKIAAAA|35423|9|50|23|AM|first|morning|breakfast| +35424|AAAAAAAABGKIAAAA|35424|9|50|24|AM|first|morning|breakfast| +35425|AAAAAAAACGKIAAAA|35425|9|50|25|AM|first|morning|breakfast| +35426|AAAAAAAADGKIAAAA|35426|9|50|26|AM|first|morning|breakfast| +35427|AAAAAAAAEGKIAAAA|35427|9|50|27|AM|first|morning|breakfast| +35428|AAAAAAAAFGKIAAAA|35428|9|50|28|AM|first|morning|breakfast| +35429|AAAAAAAAGGKIAAAA|35429|9|50|29|AM|first|morning|breakfast| +35430|AAAAAAAAHGKIAAAA|35430|9|50|30|AM|first|morning|breakfast| +35431|AAAAAAAAIGKIAAAA|35431|9|50|31|AM|first|morning|breakfast| +35432|AAAAAAAAJGKIAAAA|35432|9|50|32|AM|first|morning|breakfast| +35433|AAAAAAAAKGKIAAAA|35433|9|50|33|AM|first|morning|breakfast| +35434|AAAAAAAALGKIAAAA|35434|9|50|34|AM|first|morning|breakfast| +35435|AAAAAAAAMGKIAAAA|35435|9|50|35|AM|first|morning|breakfast| +35436|AAAAAAAANGKIAAAA|35436|9|50|36|AM|first|morning|breakfast| +35437|AAAAAAAAOGKIAAAA|35437|9|50|37|AM|first|morning|breakfast| +35438|AAAAAAAAPGKIAAAA|35438|9|50|38|AM|first|morning|breakfast| +35439|AAAAAAAAAHKIAAAA|35439|9|50|39|AM|first|morning|breakfast| +35440|AAAAAAAABHKIAAAA|35440|9|50|40|AM|first|morning|breakfast| +35441|AAAAAAAACHKIAAAA|35441|9|50|41|AM|first|morning|breakfast| +35442|AAAAAAAADHKIAAAA|35442|9|50|42|AM|first|morning|breakfast| +35443|AAAAAAAAEHKIAAAA|35443|9|50|43|AM|first|morning|breakfast| +35444|AAAAAAAAFHKIAAAA|35444|9|50|44|AM|first|morning|breakfast| +35445|AAAAAAAAGHKIAAAA|35445|9|50|45|AM|first|morning|breakfast| +35446|AAAAAAAAHHKIAAAA|35446|9|50|46|AM|first|morning|breakfast| +35447|AAAAAAAAIHKIAAAA|35447|9|50|47|AM|first|morning|breakfast| +35448|AAAAAAAAJHKIAAAA|35448|9|50|48|AM|first|morning|breakfast| +35449|AAAAAAAAKHKIAAAA|35449|9|50|49|AM|first|morning|breakfast| +35450|AAAAAAAALHKIAAAA|35450|9|50|50|AM|first|morning|breakfast| +35451|AAAAAAAAMHKIAAAA|35451|9|50|51|AM|first|morning|breakfast| +35452|AAAAAAAANHKIAAAA|35452|9|50|52|AM|first|morning|breakfast| +35453|AAAAAAAAOHKIAAAA|35453|9|50|53|AM|first|morning|breakfast| +35454|AAAAAAAAPHKIAAAA|35454|9|50|54|AM|first|morning|breakfast| +35455|AAAAAAAAAIKIAAAA|35455|9|50|55|AM|first|morning|breakfast| +35456|AAAAAAAABIKIAAAA|35456|9|50|56|AM|first|morning|breakfast| +35457|AAAAAAAACIKIAAAA|35457|9|50|57|AM|first|morning|breakfast| +35458|AAAAAAAADIKIAAAA|35458|9|50|58|AM|first|morning|breakfast| +35459|AAAAAAAAEIKIAAAA|35459|9|50|59|AM|first|morning|breakfast| +35460|AAAAAAAAFIKIAAAA|35460|9|51|0|AM|first|morning|breakfast| +35461|AAAAAAAAGIKIAAAA|35461|9|51|1|AM|first|morning|breakfast| +35462|AAAAAAAAHIKIAAAA|35462|9|51|2|AM|first|morning|breakfast| +35463|AAAAAAAAIIKIAAAA|35463|9|51|3|AM|first|morning|breakfast| +35464|AAAAAAAAJIKIAAAA|35464|9|51|4|AM|first|morning|breakfast| +35465|AAAAAAAAKIKIAAAA|35465|9|51|5|AM|first|morning|breakfast| +35466|AAAAAAAALIKIAAAA|35466|9|51|6|AM|first|morning|breakfast| +35467|AAAAAAAAMIKIAAAA|35467|9|51|7|AM|first|morning|breakfast| +35468|AAAAAAAANIKIAAAA|35468|9|51|8|AM|first|morning|breakfast| +35469|AAAAAAAAOIKIAAAA|35469|9|51|9|AM|first|morning|breakfast| +35470|AAAAAAAAPIKIAAAA|35470|9|51|10|AM|first|morning|breakfast| +35471|AAAAAAAAAJKIAAAA|35471|9|51|11|AM|first|morning|breakfast| +35472|AAAAAAAABJKIAAAA|35472|9|51|12|AM|first|morning|breakfast| +35473|AAAAAAAACJKIAAAA|35473|9|51|13|AM|first|morning|breakfast| +35474|AAAAAAAADJKIAAAA|35474|9|51|14|AM|first|morning|breakfast| +35475|AAAAAAAAEJKIAAAA|35475|9|51|15|AM|first|morning|breakfast| +35476|AAAAAAAAFJKIAAAA|35476|9|51|16|AM|first|morning|breakfast| +35477|AAAAAAAAGJKIAAAA|35477|9|51|17|AM|first|morning|breakfast| +35478|AAAAAAAAHJKIAAAA|35478|9|51|18|AM|first|morning|breakfast| +35479|AAAAAAAAIJKIAAAA|35479|9|51|19|AM|first|morning|breakfast| +35480|AAAAAAAAJJKIAAAA|35480|9|51|20|AM|first|morning|breakfast| +35481|AAAAAAAAKJKIAAAA|35481|9|51|21|AM|first|morning|breakfast| +35482|AAAAAAAALJKIAAAA|35482|9|51|22|AM|first|morning|breakfast| +35483|AAAAAAAAMJKIAAAA|35483|9|51|23|AM|first|morning|breakfast| +35484|AAAAAAAANJKIAAAA|35484|9|51|24|AM|first|morning|breakfast| +35485|AAAAAAAAOJKIAAAA|35485|9|51|25|AM|first|morning|breakfast| +35486|AAAAAAAAPJKIAAAA|35486|9|51|26|AM|first|morning|breakfast| +35487|AAAAAAAAAKKIAAAA|35487|9|51|27|AM|first|morning|breakfast| +35488|AAAAAAAABKKIAAAA|35488|9|51|28|AM|first|morning|breakfast| +35489|AAAAAAAACKKIAAAA|35489|9|51|29|AM|first|morning|breakfast| +35490|AAAAAAAADKKIAAAA|35490|9|51|30|AM|first|morning|breakfast| +35491|AAAAAAAAEKKIAAAA|35491|9|51|31|AM|first|morning|breakfast| +35492|AAAAAAAAFKKIAAAA|35492|9|51|32|AM|first|morning|breakfast| +35493|AAAAAAAAGKKIAAAA|35493|9|51|33|AM|first|morning|breakfast| +35494|AAAAAAAAHKKIAAAA|35494|9|51|34|AM|first|morning|breakfast| +35495|AAAAAAAAIKKIAAAA|35495|9|51|35|AM|first|morning|breakfast| +35496|AAAAAAAAJKKIAAAA|35496|9|51|36|AM|first|morning|breakfast| +35497|AAAAAAAAKKKIAAAA|35497|9|51|37|AM|first|morning|breakfast| +35498|AAAAAAAALKKIAAAA|35498|9|51|38|AM|first|morning|breakfast| +35499|AAAAAAAAMKKIAAAA|35499|9|51|39|AM|first|morning|breakfast| +35500|AAAAAAAANKKIAAAA|35500|9|51|40|AM|first|morning|breakfast| +35501|AAAAAAAAOKKIAAAA|35501|9|51|41|AM|first|morning|breakfast| +35502|AAAAAAAAPKKIAAAA|35502|9|51|42|AM|first|morning|breakfast| +35503|AAAAAAAAALKIAAAA|35503|9|51|43|AM|first|morning|breakfast| +35504|AAAAAAAABLKIAAAA|35504|9|51|44|AM|first|morning|breakfast| +35505|AAAAAAAACLKIAAAA|35505|9|51|45|AM|first|morning|breakfast| +35506|AAAAAAAADLKIAAAA|35506|9|51|46|AM|first|morning|breakfast| +35507|AAAAAAAAELKIAAAA|35507|9|51|47|AM|first|morning|breakfast| +35508|AAAAAAAAFLKIAAAA|35508|9|51|48|AM|first|morning|breakfast| +35509|AAAAAAAAGLKIAAAA|35509|9|51|49|AM|first|morning|breakfast| +35510|AAAAAAAAHLKIAAAA|35510|9|51|50|AM|first|morning|breakfast| +35511|AAAAAAAAILKIAAAA|35511|9|51|51|AM|first|morning|breakfast| +35512|AAAAAAAAJLKIAAAA|35512|9|51|52|AM|first|morning|breakfast| +35513|AAAAAAAAKLKIAAAA|35513|9|51|53|AM|first|morning|breakfast| +35514|AAAAAAAALLKIAAAA|35514|9|51|54|AM|first|morning|breakfast| +35515|AAAAAAAAMLKIAAAA|35515|9|51|55|AM|first|morning|breakfast| +35516|AAAAAAAANLKIAAAA|35516|9|51|56|AM|first|morning|breakfast| +35517|AAAAAAAAOLKIAAAA|35517|9|51|57|AM|first|morning|breakfast| +35518|AAAAAAAAPLKIAAAA|35518|9|51|58|AM|first|morning|breakfast| +35519|AAAAAAAAAMKIAAAA|35519|9|51|59|AM|first|morning|breakfast| +35520|AAAAAAAABMKIAAAA|35520|9|52|0|AM|first|morning|breakfast| +35521|AAAAAAAACMKIAAAA|35521|9|52|1|AM|first|morning|breakfast| +35522|AAAAAAAADMKIAAAA|35522|9|52|2|AM|first|morning|breakfast| +35523|AAAAAAAAEMKIAAAA|35523|9|52|3|AM|first|morning|breakfast| +35524|AAAAAAAAFMKIAAAA|35524|9|52|4|AM|first|morning|breakfast| +35525|AAAAAAAAGMKIAAAA|35525|9|52|5|AM|first|morning|breakfast| +35526|AAAAAAAAHMKIAAAA|35526|9|52|6|AM|first|morning|breakfast| +35527|AAAAAAAAIMKIAAAA|35527|9|52|7|AM|first|morning|breakfast| +35528|AAAAAAAAJMKIAAAA|35528|9|52|8|AM|first|morning|breakfast| +35529|AAAAAAAAKMKIAAAA|35529|9|52|9|AM|first|morning|breakfast| +35530|AAAAAAAALMKIAAAA|35530|9|52|10|AM|first|morning|breakfast| +35531|AAAAAAAAMMKIAAAA|35531|9|52|11|AM|first|morning|breakfast| +35532|AAAAAAAANMKIAAAA|35532|9|52|12|AM|first|morning|breakfast| +35533|AAAAAAAAOMKIAAAA|35533|9|52|13|AM|first|morning|breakfast| +35534|AAAAAAAAPMKIAAAA|35534|9|52|14|AM|first|morning|breakfast| +35535|AAAAAAAAANKIAAAA|35535|9|52|15|AM|first|morning|breakfast| +35536|AAAAAAAABNKIAAAA|35536|9|52|16|AM|first|morning|breakfast| +35537|AAAAAAAACNKIAAAA|35537|9|52|17|AM|first|morning|breakfast| +35538|AAAAAAAADNKIAAAA|35538|9|52|18|AM|first|morning|breakfast| +35539|AAAAAAAAENKIAAAA|35539|9|52|19|AM|first|morning|breakfast| +35540|AAAAAAAAFNKIAAAA|35540|9|52|20|AM|first|morning|breakfast| +35541|AAAAAAAAGNKIAAAA|35541|9|52|21|AM|first|morning|breakfast| +35542|AAAAAAAAHNKIAAAA|35542|9|52|22|AM|first|morning|breakfast| +35543|AAAAAAAAINKIAAAA|35543|9|52|23|AM|first|morning|breakfast| +35544|AAAAAAAAJNKIAAAA|35544|9|52|24|AM|first|morning|breakfast| +35545|AAAAAAAAKNKIAAAA|35545|9|52|25|AM|first|morning|breakfast| +35546|AAAAAAAALNKIAAAA|35546|9|52|26|AM|first|morning|breakfast| +35547|AAAAAAAAMNKIAAAA|35547|9|52|27|AM|first|morning|breakfast| +35548|AAAAAAAANNKIAAAA|35548|9|52|28|AM|first|morning|breakfast| +35549|AAAAAAAAONKIAAAA|35549|9|52|29|AM|first|morning|breakfast| +35550|AAAAAAAAPNKIAAAA|35550|9|52|30|AM|first|morning|breakfast| +35551|AAAAAAAAAOKIAAAA|35551|9|52|31|AM|first|morning|breakfast| +35552|AAAAAAAABOKIAAAA|35552|9|52|32|AM|first|morning|breakfast| +35553|AAAAAAAACOKIAAAA|35553|9|52|33|AM|first|morning|breakfast| +35554|AAAAAAAADOKIAAAA|35554|9|52|34|AM|first|morning|breakfast| +35555|AAAAAAAAEOKIAAAA|35555|9|52|35|AM|first|morning|breakfast| +35556|AAAAAAAAFOKIAAAA|35556|9|52|36|AM|first|morning|breakfast| +35557|AAAAAAAAGOKIAAAA|35557|9|52|37|AM|first|morning|breakfast| +35558|AAAAAAAAHOKIAAAA|35558|9|52|38|AM|first|morning|breakfast| +35559|AAAAAAAAIOKIAAAA|35559|9|52|39|AM|first|morning|breakfast| +35560|AAAAAAAAJOKIAAAA|35560|9|52|40|AM|first|morning|breakfast| +35561|AAAAAAAAKOKIAAAA|35561|9|52|41|AM|first|morning|breakfast| +35562|AAAAAAAALOKIAAAA|35562|9|52|42|AM|first|morning|breakfast| +35563|AAAAAAAAMOKIAAAA|35563|9|52|43|AM|first|morning|breakfast| +35564|AAAAAAAANOKIAAAA|35564|9|52|44|AM|first|morning|breakfast| +35565|AAAAAAAAOOKIAAAA|35565|9|52|45|AM|first|morning|breakfast| +35566|AAAAAAAAPOKIAAAA|35566|9|52|46|AM|first|morning|breakfast| +35567|AAAAAAAAAPKIAAAA|35567|9|52|47|AM|first|morning|breakfast| +35568|AAAAAAAABPKIAAAA|35568|9|52|48|AM|first|morning|breakfast| +35569|AAAAAAAACPKIAAAA|35569|9|52|49|AM|first|morning|breakfast| +35570|AAAAAAAADPKIAAAA|35570|9|52|50|AM|first|morning|breakfast| +35571|AAAAAAAAEPKIAAAA|35571|9|52|51|AM|first|morning|breakfast| +35572|AAAAAAAAFPKIAAAA|35572|9|52|52|AM|first|morning|breakfast| +35573|AAAAAAAAGPKIAAAA|35573|9|52|53|AM|first|morning|breakfast| +35574|AAAAAAAAHPKIAAAA|35574|9|52|54|AM|first|morning|breakfast| +35575|AAAAAAAAIPKIAAAA|35575|9|52|55|AM|first|morning|breakfast| +35576|AAAAAAAAJPKIAAAA|35576|9|52|56|AM|first|morning|breakfast| +35577|AAAAAAAAKPKIAAAA|35577|9|52|57|AM|first|morning|breakfast| +35578|AAAAAAAALPKIAAAA|35578|9|52|58|AM|first|morning|breakfast| +35579|AAAAAAAAMPKIAAAA|35579|9|52|59|AM|first|morning|breakfast| +35580|AAAAAAAANPKIAAAA|35580|9|53|0|AM|first|morning|breakfast| +35581|AAAAAAAAOPKIAAAA|35581|9|53|1|AM|first|morning|breakfast| +35582|AAAAAAAAPPKIAAAA|35582|9|53|2|AM|first|morning|breakfast| +35583|AAAAAAAAAALIAAAA|35583|9|53|3|AM|first|morning|breakfast| +35584|AAAAAAAABALIAAAA|35584|9|53|4|AM|first|morning|breakfast| +35585|AAAAAAAACALIAAAA|35585|9|53|5|AM|first|morning|breakfast| +35586|AAAAAAAADALIAAAA|35586|9|53|6|AM|first|morning|breakfast| +35587|AAAAAAAAEALIAAAA|35587|9|53|7|AM|first|morning|breakfast| +35588|AAAAAAAAFALIAAAA|35588|9|53|8|AM|first|morning|breakfast| +35589|AAAAAAAAGALIAAAA|35589|9|53|9|AM|first|morning|breakfast| +35590|AAAAAAAAHALIAAAA|35590|9|53|10|AM|first|morning|breakfast| +35591|AAAAAAAAIALIAAAA|35591|9|53|11|AM|first|morning|breakfast| +35592|AAAAAAAAJALIAAAA|35592|9|53|12|AM|first|morning|breakfast| +35593|AAAAAAAAKALIAAAA|35593|9|53|13|AM|first|morning|breakfast| +35594|AAAAAAAALALIAAAA|35594|9|53|14|AM|first|morning|breakfast| +35595|AAAAAAAAMALIAAAA|35595|9|53|15|AM|first|morning|breakfast| +35596|AAAAAAAANALIAAAA|35596|9|53|16|AM|first|morning|breakfast| +35597|AAAAAAAAOALIAAAA|35597|9|53|17|AM|first|morning|breakfast| +35598|AAAAAAAAPALIAAAA|35598|9|53|18|AM|first|morning|breakfast| +35599|AAAAAAAAABLIAAAA|35599|9|53|19|AM|first|morning|breakfast| +35600|AAAAAAAABBLIAAAA|35600|9|53|20|AM|first|morning|breakfast| +35601|AAAAAAAACBLIAAAA|35601|9|53|21|AM|first|morning|breakfast| +35602|AAAAAAAADBLIAAAA|35602|9|53|22|AM|first|morning|breakfast| +35603|AAAAAAAAEBLIAAAA|35603|9|53|23|AM|first|morning|breakfast| +35604|AAAAAAAAFBLIAAAA|35604|9|53|24|AM|first|morning|breakfast| +35605|AAAAAAAAGBLIAAAA|35605|9|53|25|AM|first|morning|breakfast| +35606|AAAAAAAAHBLIAAAA|35606|9|53|26|AM|first|morning|breakfast| +35607|AAAAAAAAIBLIAAAA|35607|9|53|27|AM|first|morning|breakfast| +35608|AAAAAAAAJBLIAAAA|35608|9|53|28|AM|first|morning|breakfast| +35609|AAAAAAAAKBLIAAAA|35609|9|53|29|AM|first|morning|breakfast| +35610|AAAAAAAALBLIAAAA|35610|9|53|30|AM|first|morning|breakfast| +35611|AAAAAAAAMBLIAAAA|35611|9|53|31|AM|first|morning|breakfast| +35612|AAAAAAAANBLIAAAA|35612|9|53|32|AM|first|morning|breakfast| +35613|AAAAAAAAOBLIAAAA|35613|9|53|33|AM|first|morning|breakfast| +35614|AAAAAAAAPBLIAAAA|35614|9|53|34|AM|first|morning|breakfast| +35615|AAAAAAAAACLIAAAA|35615|9|53|35|AM|first|morning|breakfast| +35616|AAAAAAAABCLIAAAA|35616|9|53|36|AM|first|morning|breakfast| +35617|AAAAAAAACCLIAAAA|35617|9|53|37|AM|first|morning|breakfast| +35618|AAAAAAAADCLIAAAA|35618|9|53|38|AM|first|morning|breakfast| +35619|AAAAAAAAECLIAAAA|35619|9|53|39|AM|first|morning|breakfast| +35620|AAAAAAAAFCLIAAAA|35620|9|53|40|AM|first|morning|breakfast| +35621|AAAAAAAAGCLIAAAA|35621|9|53|41|AM|first|morning|breakfast| +35622|AAAAAAAAHCLIAAAA|35622|9|53|42|AM|first|morning|breakfast| +35623|AAAAAAAAICLIAAAA|35623|9|53|43|AM|first|morning|breakfast| +35624|AAAAAAAAJCLIAAAA|35624|9|53|44|AM|first|morning|breakfast| +35625|AAAAAAAAKCLIAAAA|35625|9|53|45|AM|first|morning|breakfast| +35626|AAAAAAAALCLIAAAA|35626|9|53|46|AM|first|morning|breakfast| +35627|AAAAAAAAMCLIAAAA|35627|9|53|47|AM|first|morning|breakfast| +35628|AAAAAAAANCLIAAAA|35628|9|53|48|AM|first|morning|breakfast| +35629|AAAAAAAAOCLIAAAA|35629|9|53|49|AM|first|morning|breakfast| +35630|AAAAAAAAPCLIAAAA|35630|9|53|50|AM|first|morning|breakfast| +35631|AAAAAAAAADLIAAAA|35631|9|53|51|AM|first|morning|breakfast| +35632|AAAAAAAABDLIAAAA|35632|9|53|52|AM|first|morning|breakfast| +35633|AAAAAAAACDLIAAAA|35633|9|53|53|AM|first|morning|breakfast| +35634|AAAAAAAADDLIAAAA|35634|9|53|54|AM|first|morning|breakfast| +35635|AAAAAAAAEDLIAAAA|35635|9|53|55|AM|first|morning|breakfast| +35636|AAAAAAAAFDLIAAAA|35636|9|53|56|AM|first|morning|breakfast| +35637|AAAAAAAAGDLIAAAA|35637|9|53|57|AM|first|morning|breakfast| +35638|AAAAAAAAHDLIAAAA|35638|9|53|58|AM|first|morning|breakfast| +35639|AAAAAAAAIDLIAAAA|35639|9|53|59|AM|first|morning|breakfast| +35640|AAAAAAAAJDLIAAAA|35640|9|54|0|AM|first|morning|breakfast| +35641|AAAAAAAAKDLIAAAA|35641|9|54|1|AM|first|morning|breakfast| +35642|AAAAAAAALDLIAAAA|35642|9|54|2|AM|first|morning|breakfast| +35643|AAAAAAAAMDLIAAAA|35643|9|54|3|AM|first|morning|breakfast| +35644|AAAAAAAANDLIAAAA|35644|9|54|4|AM|first|morning|breakfast| +35645|AAAAAAAAODLIAAAA|35645|9|54|5|AM|first|morning|breakfast| +35646|AAAAAAAAPDLIAAAA|35646|9|54|6|AM|first|morning|breakfast| +35647|AAAAAAAAAELIAAAA|35647|9|54|7|AM|first|morning|breakfast| +35648|AAAAAAAABELIAAAA|35648|9|54|8|AM|first|morning|breakfast| +35649|AAAAAAAACELIAAAA|35649|9|54|9|AM|first|morning|breakfast| +35650|AAAAAAAADELIAAAA|35650|9|54|10|AM|first|morning|breakfast| +35651|AAAAAAAAEELIAAAA|35651|9|54|11|AM|first|morning|breakfast| +35652|AAAAAAAAFELIAAAA|35652|9|54|12|AM|first|morning|breakfast| +35653|AAAAAAAAGELIAAAA|35653|9|54|13|AM|first|morning|breakfast| +35654|AAAAAAAAHELIAAAA|35654|9|54|14|AM|first|morning|breakfast| +35655|AAAAAAAAIELIAAAA|35655|9|54|15|AM|first|morning|breakfast| +35656|AAAAAAAAJELIAAAA|35656|9|54|16|AM|first|morning|breakfast| +35657|AAAAAAAAKELIAAAA|35657|9|54|17|AM|first|morning|breakfast| +35658|AAAAAAAALELIAAAA|35658|9|54|18|AM|first|morning|breakfast| +35659|AAAAAAAAMELIAAAA|35659|9|54|19|AM|first|morning|breakfast| +35660|AAAAAAAANELIAAAA|35660|9|54|20|AM|first|morning|breakfast| +35661|AAAAAAAAOELIAAAA|35661|9|54|21|AM|first|morning|breakfast| +35662|AAAAAAAAPELIAAAA|35662|9|54|22|AM|first|morning|breakfast| +35663|AAAAAAAAAFLIAAAA|35663|9|54|23|AM|first|morning|breakfast| +35664|AAAAAAAABFLIAAAA|35664|9|54|24|AM|first|morning|breakfast| +35665|AAAAAAAACFLIAAAA|35665|9|54|25|AM|first|morning|breakfast| +35666|AAAAAAAADFLIAAAA|35666|9|54|26|AM|first|morning|breakfast| +35667|AAAAAAAAEFLIAAAA|35667|9|54|27|AM|first|morning|breakfast| +35668|AAAAAAAAFFLIAAAA|35668|9|54|28|AM|first|morning|breakfast| +35669|AAAAAAAAGFLIAAAA|35669|9|54|29|AM|first|morning|breakfast| +35670|AAAAAAAAHFLIAAAA|35670|9|54|30|AM|first|morning|breakfast| +35671|AAAAAAAAIFLIAAAA|35671|9|54|31|AM|first|morning|breakfast| +35672|AAAAAAAAJFLIAAAA|35672|9|54|32|AM|first|morning|breakfast| +35673|AAAAAAAAKFLIAAAA|35673|9|54|33|AM|first|morning|breakfast| +35674|AAAAAAAALFLIAAAA|35674|9|54|34|AM|first|morning|breakfast| +35675|AAAAAAAAMFLIAAAA|35675|9|54|35|AM|first|morning|breakfast| +35676|AAAAAAAANFLIAAAA|35676|9|54|36|AM|first|morning|breakfast| +35677|AAAAAAAAOFLIAAAA|35677|9|54|37|AM|first|morning|breakfast| +35678|AAAAAAAAPFLIAAAA|35678|9|54|38|AM|first|morning|breakfast| +35679|AAAAAAAAAGLIAAAA|35679|9|54|39|AM|first|morning|breakfast| +35680|AAAAAAAABGLIAAAA|35680|9|54|40|AM|first|morning|breakfast| +35681|AAAAAAAACGLIAAAA|35681|9|54|41|AM|first|morning|breakfast| +35682|AAAAAAAADGLIAAAA|35682|9|54|42|AM|first|morning|breakfast| +35683|AAAAAAAAEGLIAAAA|35683|9|54|43|AM|first|morning|breakfast| +35684|AAAAAAAAFGLIAAAA|35684|9|54|44|AM|first|morning|breakfast| +35685|AAAAAAAAGGLIAAAA|35685|9|54|45|AM|first|morning|breakfast| +35686|AAAAAAAAHGLIAAAA|35686|9|54|46|AM|first|morning|breakfast| +35687|AAAAAAAAIGLIAAAA|35687|9|54|47|AM|first|morning|breakfast| +35688|AAAAAAAAJGLIAAAA|35688|9|54|48|AM|first|morning|breakfast| +35689|AAAAAAAAKGLIAAAA|35689|9|54|49|AM|first|morning|breakfast| +35690|AAAAAAAALGLIAAAA|35690|9|54|50|AM|first|morning|breakfast| +35691|AAAAAAAAMGLIAAAA|35691|9|54|51|AM|first|morning|breakfast| +35692|AAAAAAAANGLIAAAA|35692|9|54|52|AM|first|morning|breakfast| +35693|AAAAAAAAOGLIAAAA|35693|9|54|53|AM|first|morning|breakfast| +35694|AAAAAAAAPGLIAAAA|35694|9|54|54|AM|first|morning|breakfast| +35695|AAAAAAAAAHLIAAAA|35695|9|54|55|AM|first|morning|breakfast| +35696|AAAAAAAABHLIAAAA|35696|9|54|56|AM|first|morning|breakfast| +35697|AAAAAAAACHLIAAAA|35697|9|54|57|AM|first|morning|breakfast| +35698|AAAAAAAADHLIAAAA|35698|9|54|58|AM|first|morning|breakfast| +35699|AAAAAAAAEHLIAAAA|35699|9|54|59|AM|first|morning|breakfast| +35700|AAAAAAAAFHLIAAAA|35700|9|55|0|AM|first|morning|breakfast| +35701|AAAAAAAAGHLIAAAA|35701|9|55|1|AM|first|morning|breakfast| +35702|AAAAAAAAHHLIAAAA|35702|9|55|2|AM|first|morning|breakfast| +35703|AAAAAAAAIHLIAAAA|35703|9|55|3|AM|first|morning|breakfast| +35704|AAAAAAAAJHLIAAAA|35704|9|55|4|AM|first|morning|breakfast| +35705|AAAAAAAAKHLIAAAA|35705|9|55|5|AM|first|morning|breakfast| +35706|AAAAAAAALHLIAAAA|35706|9|55|6|AM|first|morning|breakfast| +35707|AAAAAAAAMHLIAAAA|35707|9|55|7|AM|first|morning|breakfast| +35708|AAAAAAAANHLIAAAA|35708|9|55|8|AM|first|morning|breakfast| +35709|AAAAAAAAOHLIAAAA|35709|9|55|9|AM|first|morning|breakfast| +35710|AAAAAAAAPHLIAAAA|35710|9|55|10|AM|first|morning|breakfast| +35711|AAAAAAAAAILIAAAA|35711|9|55|11|AM|first|morning|breakfast| +35712|AAAAAAAABILIAAAA|35712|9|55|12|AM|first|morning|breakfast| +35713|AAAAAAAACILIAAAA|35713|9|55|13|AM|first|morning|breakfast| +35714|AAAAAAAADILIAAAA|35714|9|55|14|AM|first|morning|breakfast| +35715|AAAAAAAAEILIAAAA|35715|9|55|15|AM|first|morning|breakfast| +35716|AAAAAAAAFILIAAAA|35716|9|55|16|AM|first|morning|breakfast| +35717|AAAAAAAAGILIAAAA|35717|9|55|17|AM|first|morning|breakfast| +35718|AAAAAAAAHILIAAAA|35718|9|55|18|AM|first|morning|breakfast| +35719|AAAAAAAAIILIAAAA|35719|9|55|19|AM|first|morning|breakfast| +35720|AAAAAAAAJILIAAAA|35720|9|55|20|AM|first|morning|breakfast| +35721|AAAAAAAAKILIAAAA|35721|9|55|21|AM|first|morning|breakfast| +35722|AAAAAAAALILIAAAA|35722|9|55|22|AM|first|morning|breakfast| +35723|AAAAAAAAMILIAAAA|35723|9|55|23|AM|first|morning|breakfast| +35724|AAAAAAAANILIAAAA|35724|9|55|24|AM|first|morning|breakfast| +35725|AAAAAAAAOILIAAAA|35725|9|55|25|AM|first|morning|breakfast| +35726|AAAAAAAAPILIAAAA|35726|9|55|26|AM|first|morning|breakfast| +35727|AAAAAAAAAJLIAAAA|35727|9|55|27|AM|first|morning|breakfast| +35728|AAAAAAAABJLIAAAA|35728|9|55|28|AM|first|morning|breakfast| +35729|AAAAAAAACJLIAAAA|35729|9|55|29|AM|first|morning|breakfast| +35730|AAAAAAAADJLIAAAA|35730|9|55|30|AM|first|morning|breakfast| +35731|AAAAAAAAEJLIAAAA|35731|9|55|31|AM|first|morning|breakfast| +35732|AAAAAAAAFJLIAAAA|35732|9|55|32|AM|first|morning|breakfast| +35733|AAAAAAAAGJLIAAAA|35733|9|55|33|AM|first|morning|breakfast| +35734|AAAAAAAAHJLIAAAA|35734|9|55|34|AM|first|morning|breakfast| +35735|AAAAAAAAIJLIAAAA|35735|9|55|35|AM|first|morning|breakfast| +35736|AAAAAAAAJJLIAAAA|35736|9|55|36|AM|first|morning|breakfast| +35737|AAAAAAAAKJLIAAAA|35737|9|55|37|AM|first|morning|breakfast| +35738|AAAAAAAALJLIAAAA|35738|9|55|38|AM|first|morning|breakfast| +35739|AAAAAAAAMJLIAAAA|35739|9|55|39|AM|first|morning|breakfast| +35740|AAAAAAAANJLIAAAA|35740|9|55|40|AM|first|morning|breakfast| +35741|AAAAAAAAOJLIAAAA|35741|9|55|41|AM|first|morning|breakfast| +35742|AAAAAAAAPJLIAAAA|35742|9|55|42|AM|first|morning|breakfast| +35743|AAAAAAAAAKLIAAAA|35743|9|55|43|AM|first|morning|breakfast| +35744|AAAAAAAABKLIAAAA|35744|9|55|44|AM|first|morning|breakfast| +35745|AAAAAAAACKLIAAAA|35745|9|55|45|AM|first|morning|breakfast| +35746|AAAAAAAADKLIAAAA|35746|9|55|46|AM|first|morning|breakfast| +35747|AAAAAAAAEKLIAAAA|35747|9|55|47|AM|first|morning|breakfast| +35748|AAAAAAAAFKLIAAAA|35748|9|55|48|AM|first|morning|breakfast| +35749|AAAAAAAAGKLIAAAA|35749|9|55|49|AM|first|morning|breakfast| +35750|AAAAAAAAHKLIAAAA|35750|9|55|50|AM|first|morning|breakfast| +35751|AAAAAAAAIKLIAAAA|35751|9|55|51|AM|first|morning|breakfast| +35752|AAAAAAAAJKLIAAAA|35752|9|55|52|AM|first|morning|breakfast| +35753|AAAAAAAAKKLIAAAA|35753|9|55|53|AM|first|morning|breakfast| +35754|AAAAAAAALKLIAAAA|35754|9|55|54|AM|first|morning|breakfast| +35755|AAAAAAAAMKLIAAAA|35755|9|55|55|AM|first|morning|breakfast| +35756|AAAAAAAANKLIAAAA|35756|9|55|56|AM|first|morning|breakfast| +35757|AAAAAAAAOKLIAAAA|35757|9|55|57|AM|first|morning|breakfast| +35758|AAAAAAAAPKLIAAAA|35758|9|55|58|AM|first|morning|breakfast| +35759|AAAAAAAAALLIAAAA|35759|9|55|59|AM|first|morning|breakfast| +35760|AAAAAAAABLLIAAAA|35760|9|56|0|AM|first|morning|breakfast| +35761|AAAAAAAACLLIAAAA|35761|9|56|1|AM|first|morning|breakfast| +35762|AAAAAAAADLLIAAAA|35762|9|56|2|AM|first|morning|breakfast| +35763|AAAAAAAAELLIAAAA|35763|9|56|3|AM|first|morning|breakfast| +35764|AAAAAAAAFLLIAAAA|35764|9|56|4|AM|first|morning|breakfast| +35765|AAAAAAAAGLLIAAAA|35765|9|56|5|AM|first|morning|breakfast| +35766|AAAAAAAAHLLIAAAA|35766|9|56|6|AM|first|morning|breakfast| +35767|AAAAAAAAILLIAAAA|35767|9|56|7|AM|first|morning|breakfast| +35768|AAAAAAAAJLLIAAAA|35768|9|56|8|AM|first|morning|breakfast| +35769|AAAAAAAAKLLIAAAA|35769|9|56|9|AM|first|morning|breakfast| +35770|AAAAAAAALLLIAAAA|35770|9|56|10|AM|first|morning|breakfast| +35771|AAAAAAAAMLLIAAAA|35771|9|56|11|AM|first|morning|breakfast| +35772|AAAAAAAANLLIAAAA|35772|9|56|12|AM|first|morning|breakfast| +35773|AAAAAAAAOLLIAAAA|35773|9|56|13|AM|first|morning|breakfast| +35774|AAAAAAAAPLLIAAAA|35774|9|56|14|AM|first|morning|breakfast| +35775|AAAAAAAAAMLIAAAA|35775|9|56|15|AM|first|morning|breakfast| +35776|AAAAAAAABMLIAAAA|35776|9|56|16|AM|first|morning|breakfast| +35777|AAAAAAAACMLIAAAA|35777|9|56|17|AM|first|morning|breakfast| +35778|AAAAAAAADMLIAAAA|35778|9|56|18|AM|first|morning|breakfast| +35779|AAAAAAAAEMLIAAAA|35779|9|56|19|AM|first|morning|breakfast| +35780|AAAAAAAAFMLIAAAA|35780|9|56|20|AM|first|morning|breakfast| +35781|AAAAAAAAGMLIAAAA|35781|9|56|21|AM|first|morning|breakfast| +35782|AAAAAAAAHMLIAAAA|35782|9|56|22|AM|first|morning|breakfast| +35783|AAAAAAAAIMLIAAAA|35783|9|56|23|AM|first|morning|breakfast| +35784|AAAAAAAAJMLIAAAA|35784|9|56|24|AM|first|morning|breakfast| +35785|AAAAAAAAKMLIAAAA|35785|9|56|25|AM|first|morning|breakfast| +35786|AAAAAAAALMLIAAAA|35786|9|56|26|AM|first|morning|breakfast| +35787|AAAAAAAAMMLIAAAA|35787|9|56|27|AM|first|morning|breakfast| +35788|AAAAAAAANMLIAAAA|35788|9|56|28|AM|first|morning|breakfast| +35789|AAAAAAAAOMLIAAAA|35789|9|56|29|AM|first|morning|breakfast| +35790|AAAAAAAAPMLIAAAA|35790|9|56|30|AM|first|morning|breakfast| +35791|AAAAAAAAANLIAAAA|35791|9|56|31|AM|first|morning|breakfast| +35792|AAAAAAAABNLIAAAA|35792|9|56|32|AM|first|morning|breakfast| +35793|AAAAAAAACNLIAAAA|35793|9|56|33|AM|first|morning|breakfast| +35794|AAAAAAAADNLIAAAA|35794|9|56|34|AM|first|morning|breakfast| +35795|AAAAAAAAENLIAAAA|35795|9|56|35|AM|first|morning|breakfast| +35796|AAAAAAAAFNLIAAAA|35796|9|56|36|AM|first|morning|breakfast| +35797|AAAAAAAAGNLIAAAA|35797|9|56|37|AM|first|morning|breakfast| +35798|AAAAAAAAHNLIAAAA|35798|9|56|38|AM|first|morning|breakfast| +35799|AAAAAAAAINLIAAAA|35799|9|56|39|AM|first|morning|breakfast| +35800|AAAAAAAAJNLIAAAA|35800|9|56|40|AM|first|morning|breakfast| +35801|AAAAAAAAKNLIAAAA|35801|9|56|41|AM|first|morning|breakfast| +35802|AAAAAAAALNLIAAAA|35802|9|56|42|AM|first|morning|breakfast| +35803|AAAAAAAAMNLIAAAA|35803|9|56|43|AM|first|morning|breakfast| +35804|AAAAAAAANNLIAAAA|35804|9|56|44|AM|first|morning|breakfast| +35805|AAAAAAAAONLIAAAA|35805|9|56|45|AM|first|morning|breakfast| +35806|AAAAAAAAPNLIAAAA|35806|9|56|46|AM|first|morning|breakfast| +35807|AAAAAAAAAOLIAAAA|35807|9|56|47|AM|first|morning|breakfast| +35808|AAAAAAAABOLIAAAA|35808|9|56|48|AM|first|morning|breakfast| +35809|AAAAAAAACOLIAAAA|35809|9|56|49|AM|first|morning|breakfast| +35810|AAAAAAAADOLIAAAA|35810|9|56|50|AM|first|morning|breakfast| +35811|AAAAAAAAEOLIAAAA|35811|9|56|51|AM|first|morning|breakfast| +35812|AAAAAAAAFOLIAAAA|35812|9|56|52|AM|first|morning|breakfast| +35813|AAAAAAAAGOLIAAAA|35813|9|56|53|AM|first|morning|breakfast| +35814|AAAAAAAAHOLIAAAA|35814|9|56|54|AM|first|morning|breakfast| +35815|AAAAAAAAIOLIAAAA|35815|9|56|55|AM|first|morning|breakfast| +35816|AAAAAAAAJOLIAAAA|35816|9|56|56|AM|first|morning|breakfast| +35817|AAAAAAAAKOLIAAAA|35817|9|56|57|AM|first|morning|breakfast| +35818|AAAAAAAALOLIAAAA|35818|9|56|58|AM|first|morning|breakfast| +35819|AAAAAAAAMOLIAAAA|35819|9|56|59|AM|first|morning|breakfast| +35820|AAAAAAAANOLIAAAA|35820|9|57|0|AM|first|morning|breakfast| +35821|AAAAAAAAOOLIAAAA|35821|9|57|1|AM|first|morning|breakfast| +35822|AAAAAAAAPOLIAAAA|35822|9|57|2|AM|first|morning|breakfast| +35823|AAAAAAAAAPLIAAAA|35823|9|57|3|AM|first|morning|breakfast| +35824|AAAAAAAABPLIAAAA|35824|9|57|4|AM|first|morning|breakfast| +35825|AAAAAAAACPLIAAAA|35825|9|57|5|AM|first|morning|breakfast| +35826|AAAAAAAADPLIAAAA|35826|9|57|6|AM|first|morning|breakfast| +35827|AAAAAAAAEPLIAAAA|35827|9|57|7|AM|first|morning|breakfast| +35828|AAAAAAAAFPLIAAAA|35828|9|57|8|AM|first|morning|breakfast| +35829|AAAAAAAAGPLIAAAA|35829|9|57|9|AM|first|morning|breakfast| +35830|AAAAAAAAHPLIAAAA|35830|9|57|10|AM|first|morning|breakfast| +35831|AAAAAAAAIPLIAAAA|35831|9|57|11|AM|first|morning|breakfast| +35832|AAAAAAAAJPLIAAAA|35832|9|57|12|AM|first|morning|breakfast| +35833|AAAAAAAAKPLIAAAA|35833|9|57|13|AM|first|morning|breakfast| +35834|AAAAAAAALPLIAAAA|35834|9|57|14|AM|first|morning|breakfast| +35835|AAAAAAAAMPLIAAAA|35835|9|57|15|AM|first|morning|breakfast| +35836|AAAAAAAANPLIAAAA|35836|9|57|16|AM|first|morning|breakfast| +35837|AAAAAAAAOPLIAAAA|35837|9|57|17|AM|first|morning|breakfast| +35838|AAAAAAAAPPLIAAAA|35838|9|57|18|AM|first|morning|breakfast| +35839|AAAAAAAAAAMIAAAA|35839|9|57|19|AM|first|morning|breakfast| +35840|AAAAAAAABAMIAAAA|35840|9|57|20|AM|first|morning|breakfast| +35841|AAAAAAAACAMIAAAA|35841|9|57|21|AM|first|morning|breakfast| +35842|AAAAAAAADAMIAAAA|35842|9|57|22|AM|first|morning|breakfast| +35843|AAAAAAAAEAMIAAAA|35843|9|57|23|AM|first|morning|breakfast| +35844|AAAAAAAAFAMIAAAA|35844|9|57|24|AM|first|morning|breakfast| +35845|AAAAAAAAGAMIAAAA|35845|9|57|25|AM|first|morning|breakfast| +35846|AAAAAAAAHAMIAAAA|35846|9|57|26|AM|first|morning|breakfast| +35847|AAAAAAAAIAMIAAAA|35847|9|57|27|AM|first|morning|breakfast| +35848|AAAAAAAAJAMIAAAA|35848|9|57|28|AM|first|morning|breakfast| +35849|AAAAAAAAKAMIAAAA|35849|9|57|29|AM|first|morning|breakfast| +35850|AAAAAAAALAMIAAAA|35850|9|57|30|AM|first|morning|breakfast| +35851|AAAAAAAAMAMIAAAA|35851|9|57|31|AM|first|morning|breakfast| +35852|AAAAAAAANAMIAAAA|35852|9|57|32|AM|first|morning|breakfast| +35853|AAAAAAAAOAMIAAAA|35853|9|57|33|AM|first|morning|breakfast| +35854|AAAAAAAAPAMIAAAA|35854|9|57|34|AM|first|morning|breakfast| +35855|AAAAAAAAABMIAAAA|35855|9|57|35|AM|first|morning|breakfast| +35856|AAAAAAAABBMIAAAA|35856|9|57|36|AM|first|morning|breakfast| +35857|AAAAAAAACBMIAAAA|35857|9|57|37|AM|first|morning|breakfast| +35858|AAAAAAAADBMIAAAA|35858|9|57|38|AM|first|morning|breakfast| +35859|AAAAAAAAEBMIAAAA|35859|9|57|39|AM|first|morning|breakfast| +35860|AAAAAAAAFBMIAAAA|35860|9|57|40|AM|first|morning|breakfast| +35861|AAAAAAAAGBMIAAAA|35861|9|57|41|AM|first|morning|breakfast| +35862|AAAAAAAAHBMIAAAA|35862|9|57|42|AM|first|morning|breakfast| +35863|AAAAAAAAIBMIAAAA|35863|9|57|43|AM|first|morning|breakfast| +35864|AAAAAAAAJBMIAAAA|35864|9|57|44|AM|first|morning|breakfast| +35865|AAAAAAAAKBMIAAAA|35865|9|57|45|AM|first|morning|breakfast| +35866|AAAAAAAALBMIAAAA|35866|9|57|46|AM|first|morning|breakfast| +35867|AAAAAAAAMBMIAAAA|35867|9|57|47|AM|first|morning|breakfast| +35868|AAAAAAAANBMIAAAA|35868|9|57|48|AM|first|morning|breakfast| +35869|AAAAAAAAOBMIAAAA|35869|9|57|49|AM|first|morning|breakfast| +35870|AAAAAAAAPBMIAAAA|35870|9|57|50|AM|first|morning|breakfast| +35871|AAAAAAAAACMIAAAA|35871|9|57|51|AM|first|morning|breakfast| +35872|AAAAAAAABCMIAAAA|35872|9|57|52|AM|first|morning|breakfast| +35873|AAAAAAAACCMIAAAA|35873|9|57|53|AM|first|morning|breakfast| +35874|AAAAAAAADCMIAAAA|35874|9|57|54|AM|first|morning|breakfast| +35875|AAAAAAAAECMIAAAA|35875|9|57|55|AM|first|morning|breakfast| +35876|AAAAAAAAFCMIAAAA|35876|9|57|56|AM|first|morning|breakfast| +35877|AAAAAAAAGCMIAAAA|35877|9|57|57|AM|first|morning|breakfast| +35878|AAAAAAAAHCMIAAAA|35878|9|57|58|AM|first|morning|breakfast| +35879|AAAAAAAAICMIAAAA|35879|9|57|59|AM|first|morning|breakfast| +35880|AAAAAAAAJCMIAAAA|35880|9|58|0|AM|first|morning|breakfast| +35881|AAAAAAAAKCMIAAAA|35881|9|58|1|AM|first|morning|breakfast| +35882|AAAAAAAALCMIAAAA|35882|9|58|2|AM|first|morning|breakfast| +35883|AAAAAAAAMCMIAAAA|35883|9|58|3|AM|first|morning|breakfast| +35884|AAAAAAAANCMIAAAA|35884|9|58|4|AM|first|morning|breakfast| +35885|AAAAAAAAOCMIAAAA|35885|9|58|5|AM|first|morning|breakfast| +35886|AAAAAAAAPCMIAAAA|35886|9|58|6|AM|first|morning|breakfast| +35887|AAAAAAAAADMIAAAA|35887|9|58|7|AM|first|morning|breakfast| +35888|AAAAAAAABDMIAAAA|35888|9|58|8|AM|first|morning|breakfast| +35889|AAAAAAAACDMIAAAA|35889|9|58|9|AM|first|morning|breakfast| +35890|AAAAAAAADDMIAAAA|35890|9|58|10|AM|first|morning|breakfast| +35891|AAAAAAAAEDMIAAAA|35891|9|58|11|AM|first|morning|breakfast| +35892|AAAAAAAAFDMIAAAA|35892|9|58|12|AM|first|morning|breakfast| +35893|AAAAAAAAGDMIAAAA|35893|9|58|13|AM|first|morning|breakfast| +35894|AAAAAAAAHDMIAAAA|35894|9|58|14|AM|first|morning|breakfast| +35895|AAAAAAAAIDMIAAAA|35895|9|58|15|AM|first|morning|breakfast| +35896|AAAAAAAAJDMIAAAA|35896|9|58|16|AM|first|morning|breakfast| +35897|AAAAAAAAKDMIAAAA|35897|9|58|17|AM|first|morning|breakfast| +35898|AAAAAAAALDMIAAAA|35898|9|58|18|AM|first|morning|breakfast| +35899|AAAAAAAAMDMIAAAA|35899|9|58|19|AM|first|morning|breakfast| +35900|AAAAAAAANDMIAAAA|35900|9|58|20|AM|first|morning|breakfast| +35901|AAAAAAAAODMIAAAA|35901|9|58|21|AM|first|morning|breakfast| +35902|AAAAAAAAPDMIAAAA|35902|9|58|22|AM|first|morning|breakfast| +35903|AAAAAAAAAEMIAAAA|35903|9|58|23|AM|first|morning|breakfast| +35904|AAAAAAAABEMIAAAA|35904|9|58|24|AM|first|morning|breakfast| +35905|AAAAAAAACEMIAAAA|35905|9|58|25|AM|first|morning|breakfast| +35906|AAAAAAAADEMIAAAA|35906|9|58|26|AM|first|morning|breakfast| +35907|AAAAAAAAEEMIAAAA|35907|9|58|27|AM|first|morning|breakfast| +35908|AAAAAAAAFEMIAAAA|35908|9|58|28|AM|first|morning|breakfast| +35909|AAAAAAAAGEMIAAAA|35909|9|58|29|AM|first|morning|breakfast| +35910|AAAAAAAAHEMIAAAA|35910|9|58|30|AM|first|morning|breakfast| +35911|AAAAAAAAIEMIAAAA|35911|9|58|31|AM|first|morning|breakfast| +35912|AAAAAAAAJEMIAAAA|35912|9|58|32|AM|first|morning|breakfast| +35913|AAAAAAAAKEMIAAAA|35913|9|58|33|AM|first|morning|breakfast| +35914|AAAAAAAALEMIAAAA|35914|9|58|34|AM|first|morning|breakfast| +35915|AAAAAAAAMEMIAAAA|35915|9|58|35|AM|first|morning|breakfast| +35916|AAAAAAAANEMIAAAA|35916|9|58|36|AM|first|morning|breakfast| +35917|AAAAAAAAOEMIAAAA|35917|9|58|37|AM|first|morning|breakfast| +35918|AAAAAAAAPEMIAAAA|35918|9|58|38|AM|first|morning|breakfast| +35919|AAAAAAAAAFMIAAAA|35919|9|58|39|AM|first|morning|breakfast| +35920|AAAAAAAABFMIAAAA|35920|9|58|40|AM|first|morning|breakfast| +35921|AAAAAAAACFMIAAAA|35921|9|58|41|AM|first|morning|breakfast| +35922|AAAAAAAADFMIAAAA|35922|9|58|42|AM|first|morning|breakfast| +35923|AAAAAAAAEFMIAAAA|35923|9|58|43|AM|first|morning|breakfast| +35924|AAAAAAAAFFMIAAAA|35924|9|58|44|AM|first|morning|breakfast| +35925|AAAAAAAAGFMIAAAA|35925|9|58|45|AM|first|morning|breakfast| +35926|AAAAAAAAHFMIAAAA|35926|9|58|46|AM|first|morning|breakfast| +35927|AAAAAAAAIFMIAAAA|35927|9|58|47|AM|first|morning|breakfast| +35928|AAAAAAAAJFMIAAAA|35928|9|58|48|AM|first|morning|breakfast| +35929|AAAAAAAAKFMIAAAA|35929|9|58|49|AM|first|morning|breakfast| +35930|AAAAAAAALFMIAAAA|35930|9|58|50|AM|first|morning|breakfast| +35931|AAAAAAAAMFMIAAAA|35931|9|58|51|AM|first|morning|breakfast| +35932|AAAAAAAANFMIAAAA|35932|9|58|52|AM|first|morning|breakfast| +35933|AAAAAAAAOFMIAAAA|35933|9|58|53|AM|first|morning|breakfast| +35934|AAAAAAAAPFMIAAAA|35934|9|58|54|AM|first|morning|breakfast| +35935|AAAAAAAAAGMIAAAA|35935|9|58|55|AM|first|morning|breakfast| +35936|AAAAAAAABGMIAAAA|35936|9|58|56|AM|first|morning|breakfast| +35937|AAAAAAAACGMIAAAA|35937|9|58|57|AM|first|morning|breakfast| +35938|AAAAAAAADGMIAAAA|35938|9|58|58|AM|first|morning|breakfast| +35939|AAAAAAAAEGMIAAAA|35939|9|58|59|AM|first|morning|breakfast| +35940|AAAAAAAAFGMIAAAA|35940|9|59|0|AM|first|morning|breakfast| +35941|AAAAAAAAGGMIAAAA|35941|9|59|1|AM|first|morning|breakfast| +35942|AAAAAAAAHGMIAAAA|35942|9|59|2|AM|first|morning|breakfast| +35943|AAAAAAAAIGMIAAAA|35943|9|59|3|AM|first|morning|breakfast| +35944|AAAAAAAAJGMIAAAA|35944|9|59|4|AM|first|morning|breakfast| +35945|AAAAAAAAKGMIAAAA|35945|9|59|5|AM|first|morning|breakfast| +35946|AAAAAAAALGMIAAAA|35946|9|59|6|AM|first|morning|breakfast| +35947|AAAAAAAAMGMIAAAA|35947|9|59|7|AM|first|morning|breakfast| +35948|AAAAAAAANGMIAAAA|35948|9|59|8|AM|first|morning|breakfast| +35949|AAAAAAAAOGMIAAAA|35949|9|59|9|AM|first|morning|breakfast| +35950|AAAAAAAAPGMIAAAA|35950|9|59|10|AM|first|morning|breakfast| +35951|AAAAAAAAAHMIAAAA|35951|9|59|11|AM|first|morning|breakfast| +35952|AAAAAAAABHMIAAAA|35952|9|59|12|AM|first|morning|breakfast| +35953|AAAAAAAACHMIAAAA|35953|9|59|13|AM|first|morning|breakfast| +35954|AAAAAAAADHMIAAAA|35954|9|59|14|AM|first|morning|breakfast| +35955|AAAAAAAAEHMIAAAA|35955|9|59|15|AM|first|morning|breakfast| +35956|AAAAAAAAFHMIAAAA|35956|9|59|16|AM|first|morning|breakfast| +35957|AAAAAAAAGHMIAAAA|35957|9|59|17|AM|first|morning|breakfast| +35958|AAAAAAAAHHMIAAAA|35958|9|59|18|AM|first|morning|breakfast| +35959|AAAAAAAAIHMIAAAA|35959|9|59|19|AM|first|morning|breakfast| +35960|AAAAAAAAJHMIAAAA|35960|9|59|20|AM|first|morning|breakfast| +35961|AAAAAAAAKHMIAAAA|35961|9|59|21|AM|first|morning|breakfast| +35962|AAAAAAAALHMIAAAA|35962|9|59|22|AM|first|morning|breakfast| +35963|AAAAAAAAMHMIAAAA|35963|9|59|23|AM|first|morning|breakfast| +35964|AAAAAAAANHMIAAAA|35964|9|59|24|AM|first|morning|breakfast| +35965|AAAAAAAAOHMIAAAA|35965|9|59|25|AM|first|morning|breakfast| +35966|AAAAAAAAPHMIAAAA|35966|9|59|26|AM|first|morning|breakfast| +35967|AAAAAAAAAIMIAAAA|35967|9|59|27|AM|first|morning|breakfast| +35968|AAAAAAAABIMIAAAA|35968|9|59|28|AM|first|morning|breakfast| +35969|AAAAAAAACIMIAAAA|35969|9|59|29|AM|first|morning|breakfast| +35970|AAAAAAAADIMIAAAA|35970|9|59|30|AM|first|morning|breakfast| +35971|AAAAAAAAEIMIAAAA|35971|9|59|31|AM|first|morning|breakfast| +35972|AAAAAAAAFIMIAAAA|35972|9|59|32|AM|first|morning|breakfast| +35973|AAAAAAAAGIMIAAAA|35973|9|59|33|AM|first|morning|breakfast| +35974|AAAAAAAAHIMIAAAA|35974|9|59|34|AM|first|morning|breakfast| +35975|AAAAAAAAIIMIAAAA|35975|9|59|35|AM|first|morning|breakfast| +35976|AAAAAAAAJIMIAAAA|35976|9|59|36|AM|first|morning|breakfast| +35977|AAAAAAAAKIMIAAAA|35977|9|59|37|AM|first|morning|breakfast| +35978|AAAAAAAALIMIAAAA|35978|9|59|38|AM|first|morning|breakfast| +35979|AAAAAAAAMIMIAAAA|35979|9|59|39|AM|first|morning|breakfast| +35980|AAAAAAAANIMIAAAA|35980|9|59|40|AM|first|morning|breakfast| +35981|AAAAAAAAOIMIAAAA|35981|9|59|41|AM|first|morning|breakfast| +35982|AAAAAAAAPIMIAAAA|35982|9|59|42|AM|first|morning|breakfast| +35983|AAAAAAAAAJMIAAAA|35983|9|59|43|AM|first|morning|breakfast| +35984|AAAAAAAABJMIAAAA|35984|9|59|44|AM|first|morning|breakfast| +35985|AAAAAAAACJMIAAAA|35985|9|59|45|AM|first|morning|breakfast| +35986|AAAAAAAADJMIAAAA|35986|9|59|46|AM|first|morning|breakfast| +35987|AAAAAAAAEJMIAAAA|35987|9|59|47|AM|first|morning|breakfast| +35988|AAAAAAAAFJMIAAAA|35988|9|59|48|AM|first|morning|breakfast| +35989|AAAAAAAAGJMIAAAA|35989|9|59|49|AM|first|morning|breakfast| +35990|AAAAAAAAHJMIAAAA|35990|9|59|50|AM|first|morning|breakfast| +35991|AAAAAAAAIJMIAAAA|35991|9|59|51|AM|first|morning|breakfast| +35992|AAAAAAAAJJMIAAAA|35992|9|59|52|AM|first|morning|breakfast| +35993|AAAAAAAAKJMIAAAA|35993|9|59|53|AM|first|morning|breakfast| +35994|AAAAAAAALJMIAAAA|35994|9|59|54|AM|first|morning|breakfast| +35995|AAAAAAAAMJMIAAAA|35995|9|59|55|AM|first|morning|breakfast| +35996|AAAAAAAANJMIAAAA|35996|9|59|56|AM|first|morning|breakfast| +35997|AAAAAAAAOJMIAAAA|35997|9|59|57|AM|first|morning|breakfast| +35998|AAAAAAAAPJMIAAAA|35998|9|59|58|AM|first|morning|breakfast| +35999|AAAAAAAAAKMIAAAA|35999|9|59|59|AM|first|morning|breakfast| +36000|AAAAAAAABKMIAAAA|36000|10|0|0|AM|first|morning|| +36001|AAAAAAAACKMIAAAA|36001|10|0|1|AM|first|morning|| +36002|AAAAAAAADKMIAAAA|36002|10|0|2|AM|first|morning|| +36003|AAAAAAAAEKMIAAAA|36003|10|0|3|AM|first|morning|| +36004|AAAAAAAAFKMIAAAA|36004|10|0|4|AM|first|morning|| +36005|AAAAAAAAGKMIAAAA|36005|10|0|5|AM|first|morning|| +36006|AAAAAAAAHKMIAAAA|36006|10|0|6|AM|first|morning|| +36007|AAAAAAAAIKMIAAAA|36007|10|0|7|AM|first|morning|| +36008|AAAAAAAAJKMIAAAA|36008|10|0|8|AM|first|morning|| +36009|AAAAAAAAKKMIAAAA|36009|10|0|9|AM|first|morning|| +36010|AAAAAAAALKMIAAAA|36010|10|0|10|AM|first|morning|| +36011|AAAAAAAAMKMIAAAA|36011|10|0|11|AM|first|morning|| +36012|AAAAAAAANKMIAAAA|36012|10|0|12|AM|first|morning|| +36013|AAAAAAAAOKMIAAAA|36013|10|0|13|AM|first|morning|| +36014|AAAAAAAAPKMIAAAA|36014|10|0|14|AM|first|morning|| +36015|AAAAAAAAALMIAAAA|36015|10|0|15|AM|first|morning|| +36016|AAAAAAAABLMIAAAA|36016|10|0|16|AM|first|morning|| +36017|AAAAAAAACLMIAAAA|36017|10|0|17|AM|first|morning|| +36018|AAAAAAAADLMIAAAA|36018|10|0|18|AM|first|morning|| +36019|AAAAAAAAELMIAAAA|36019|10|0|19|AM|first|morning|| +36020|AAAAAAAAFLMIAAAA|36020|10|0|20|AM|first|morning|| +36021|AAAAAAAAGLMIAAAA|36021|10|0|21|AM|first|morning|| +36022|AAAAAAAAHLMIAAAA|36022|10|0|22|AM|first|morning|| +36023|AAAAAAAAILMIAAAA|36023|10|0|23|AM|first|morning|| +36024|AAAAAAAAJLMIAAAA|36024|10|0|24|AM|first|morning|| +36025|AAAAAAAAKLMIAAAA|36025|10|0|25|AM|first|morning|| +36026|AAAAAAAALLMIAAAA|36026|10|0|26|AM|first|morning|| +36027|AAAAAAAAMLMIAAAA|36027|10|0|27|AM|first|morning|| +36028|AAAAAAAANLMIAAAA|36028|10|0|28|AM|first|morning|| +36029|AAAAAAAAOLMIAAAA|36029|10|0|29|AM|first|morning|| +36030|AAAAAAAAPLMIAAAA|36030|10|0|30|AM|first|morning|| +36031|AAAAAAAAAMMIAAAA|36031|10|0|31|AM|first|morning|| +36032|AAAAAAAABMMIAAAA|36032|10|0|32|AM|first|morning|| +36033|AAAAAAAACMMIAAAA|36033|10|0|33|AM|first|morning|| +36034|AAAAAAAADMMIAAAA|36034|10|0|34|AM|first|morning|| +36035|AAAAAAAAEMMIAAAA|36035|10|0|35|AM|first|morning|| +36036|AAAAAAAAFMMIAAAA|36036|10|0|36|AM|first|morning|| +36037|AAAAAAAAGMMIAAAA|36037|10|0|37|AM|first|morning|| +36038|AAAAAAAAHMMIAAAA|36038|10|0|38|AM|first|morning|| +36039|AAAAAAAAIMMIAAAA|36039|10|0|39|AM|first|morning|| +36040|AAAAAAAAJMMIAAAA|36040|10|0|40|AM|first|morning|| +36041|AAAAAAAAKMMIAAAA|36041|10|0|41|AM|first|morning|| +36042|AAAAAAAALMMIAAAA|36042|10|0|42|AM|first|morning|| +36043|AAAAAAAAMMMIAAAA|36043|10|0|43|AM|first|morning|| +36044|AAAAAAAANMMIAAAA|36044|10|0|44|AM|first|morning|| +36045|AAAAAAAAOMMIAAAA|36045|10|0|45|AM|first|morning|| +36046|AAAAAAAAPMMIAAAA|36046|10|0|46|AM|first|morning|| +36047|AAAAAAAAANMIAAAA|36047|10|0|47|AM|first|morning|| +36048|AAAAAAAABNMIAAAA|36048|10|0|48|AM|first|morning|| +36049|AAAAAAAACNMIAAAA|36049|10|0|49|AM|first|morning|| +36050|AAAAAAAADNMIAAAA|36050|10|0|50|AM|first|morning|| +36051|AAAAAAAAENMIAAAA|36051|10|0|51|AM|first|morning|| +36052|AAAAAAAAFNMIAAAA|36052|10|0|52|AM|first|morning|| +36053|AAAAAAAAGNMIAAAA|36053|10|0|53|AM|first|morning|| +36054|AAAAAAAAHNMIAAAA|36054|10|0|54|AM|first|morning|| +36055|AAAAAAAAINMIAAAA|36055|10|0|55|AM|first|morning|| +36056|AAAAAAAAJNMIAAAA|36056|10|0|56|AM|first|morning|| +36057|AAAAAAAAKNMIAAAA|36057|10|0|57|AM|first|morning|| +36058|AAAAAAAALNMIAAAA|36058|10|0|58|AM|first|morning|| +36059|AAAAAAAAMNMIAAAA|36059|10|0|59|AM|first|morning|| +36060|AAAAAAAANNMIAAAA|36060|10|1|0|AM|first|morning|| +36061|AAAAAAAAONMIAAAA|36061|10|1|1|AM|first|morning|| +36062|AAAAAAAAPNMIAAAA|36062|10|1|2|AM|first|morning|| +36063|AAAAAAAAAOMIAAAA|36063|10|1|3|AM|first|morning|| +36064|AAAAAAAABOMIAAAA|36064|10|1|4|AM|first|morning|| +36065|AAAAAAAACOMIAAAA|36065|10|1|5|AM|first|morning|| +36066|AAAAAAAADOMIAAAA|36066|10|1|6|AM|first|morning|| +36067|AAAAAAAAEOMIAAAA|36067|10|1|7|AM|first|morning|| +36068|AAAAAAAAFOMIAAAA|36068|10|1|8|AM|first|morning|| +36069|AAAAAAAAGOMIAAAA|36069|10|1|9|AM|first|morning|| +36070|AAAAAAAAHOMIAAAA|36070|10|1|10|AM|first|morning|| +36071|AAAAAAAAIOMIAAAA|36071|10|1|11|AM|first|morning|| +36072|AAAAAAAAJOMIAAAA|36072|10|1|12|AM|first|morning|| +36073|AAAAAAAAKOMIAAAA|36073|10|1|13|AM|first|morning|| +36074|AAAAAAAALOMIAAAA|36074|10|1|14|AM|first|morning|| +36075|AAAAAAAAMOMIAAAA|36075|10|1|15|AM|first|morning|| +36076|AAAAAAAANOMIAAAA|36076|10|1|16|AM|first|morning|| +36077|AAAAAAAAOOMIAAAA|36077|10|1|17|AM|first|morning|| +36078|AAAAAAAAPOMIAAAA|36078|10|1|18|AM|first|morning|| +36079|AAAAAAAAAPMIAAAA|36079|10|1|19|AM|first|morning|| +36080|AAAAAAAABPMIAAAA|36080|10|1|20|AM|first|morning|| +36081|AAAAAAAACPMIAAAA|36081|10|1|21|AM|first|morning|| +36082|AAAAAAAADPMIAAAA|36082|10|1|22|AM|first|morning|| +36083|AAAAAAAAEPMIAAAA|36083|10|1|23|AM|first|morning|| +36084|AAAAAAAAFPMIAAAA|36084|10|1|24|AM|first|morning|| +36085|AAAAAAAAGPMIAAAA|36085|10|1|25|AM|first|morning|| +36086|AAAAAAAAHPMIAAAA|36086|10|1|26|AM|first|morning|| +36087|AAAAAAAAIPMIAAAA|36087|10|1|27|AM|first|morning|| +36088|AAAAAAAAJPMIAAAA|36088|10|1|28|AM|first|morning|| +36089|AAAAAAAAKPMIAAAA|36089|10|1|29|AM|first|morning|| +36090|AAAAAAAALPMIAAAA|36090|10|1|30|AM|first|morning|| +36091|AAAAAAAAMPMIAAAA|36091|10|1|31|AM|first|morning|| +36092|AAAAAAAANPMIAAAA|36092|10|1|32|AM|first|morning|| +36093|AAAAAAAAOPMIAAAA|36093|10|1|33|AM|first|morning|| +36094|AAAAAAAAPPMIAAAA|36094|10|1|34|AM|first|morning|| +36095|AAAAAAAAAANIAAAA|36095|10|1|35|AM|first|morning|| +36096|AAAAAAAABANIAAAA|36096|10|1|36|AM|first|morning|| +36097|AAAAAAAACANIAAAA|36097|10|1|37|AM|first|morning|| +36098|AAAAAAAADANIAAAA|36098|10|1|38|AM|first|morning|| +36099|AAAAAAAAEANIAAAA|36099|10|1|39|AM|first|morning|| +36100|AAAAAAAAFANIAAAA|36100|10|1|40|AM|first|morning|| +36101|AAAAAAAAGANIAAAA|36101|10|1|41|AM|first|morning|| +36102|AAAAAAAAHANIAAAA|36102|10|1|42|AM|first|morning|| +36103|AAAAAAAAIANIAAAA|36103|10|1|43|AM|first|morning|| +36104|AAAAAAAAJANIAAAA|36104|10|1|44|AM|first|morning|| +36105|AAAAAAAAKANIAAAA|36105|10|1|45|AM|first|morning|| +36106|AAAAAAAALANIAAAA|36106|10|1|46|AM|first|morning|| +36107|AAAAAAAAMANIAAAA|36107|10|1|47|AM|first|morning|| +36108|AAAAAAAANANIAAAA|36108|10|1|48|AM|first|morning|| +36109|AAAAAAAAOANIAAAA|36109|10|1|49|AM|first|morning|| +36110|AAAAAAAAPANIAAAA|36110|10|1|50|AM|first|morning|| +36111|AAAAAAAAABNIAAAA|36111|10|1|51|AM|first|morning|| +36112|AAAAAAAABBNIAAAA|36112|10|1|52|AM|first|morning|| +36113|AAAAAAAACBNIAAAA|36113|10|1|53|AM|first|morning|| +36114|AAAAAAAADBNIAAAA|36114|10|1|54|AM|first|morning|| +36115|AAAAAAAAEBNIAAAA|36115|10|1|55|AM|first|morning|| +36116|AAAAAAAAFBNIAAAA|36116|10|1|56|AM|first|morning|| +36117|AAAAAAAAGBNIAAAA|36117|10|1|57|AM|first|morning|| +36118|AAAAAAAAHBNIAAAA|36118|10|1|58|AM|first|morning|| +36119|AAAAAAAAIBNIAAAA|36119|10|1|59|AM|first|morning|| +36120|AAAAAAAAJBNIAAAA|36120|10|2|0|AM|first|morning|| +36121|AAAAAAAAKBNIAAAA|36121|10|2|1|AM|first|morning|| +36122|AAAAAAAALBNIAAAA|36122|10|2|2|AM|first|morning|| +36123|AAAAAAAAMBNIAAAA|36123|10|2|3|AM|first|morning|| +36124|AAAAAAAANBNIAAAA|36124|10|2|4|AM|first|morning|| +36125|AAAAAAAAOBNIAAAA|36125|10|2|5|AM|first|morning|| +36126|AAAAAAAAPBNIAAAA|36126|10|2|6|AM|first|morning|| +36127|AAAAAAAAACNIAAAA|36127|10|2|7|AM|first|morning|| +36128|AAAAAAAABCNIAAAA|36128|10|2|8|AM|first|morning|| +36129|AAAAAAAACCNIAAAA|36129|10|2|9|AM|first|morning|| +36130|AAAAAAAADCNIAAAA|36130|10|2|10|AM|first|morning|| +36131|AAAAAAAAECNIAAAA|36131|10|2|11|AM|first|morning|| +36132|AAAAAAAAFCNIAAAA|36132|10|2|12|AM|first|morning|| +36133|AAAAAAAAGCNIAAAA|36133|10|2|13|AM|first|morning|| +36134|AAAAAAAAHCNIAAAA|36134|10|2|14|AM|first|morning|| +36135|AAAAAAAAICNIAAAA|36135|10|2|15|AM|first|morning|| +36136|AAAAAAAAJCNIAAAA|36136|10|2|16|AM|first|morning|| +36137|AAAAAAAAKCNIAAAA|36137|10|2|17|AM|first|morning|| +36138|AAAAAAAALCNIAAAA|36138|10|2|18|AM|first|morning|| +36139|AAAAAAAAMCNIAAAA|36139|10|2|19|AM|first|morning|| +36140|AAAAAAAANCNIAAAA|36140|10|2|20|AM|first|morning|| +36141|AAAAAAAAOCNIAAAA|36141|10|2|21|AM|first|morning|| +36142|AAAAAAAAPCNIAAAA|36142|10|2|22|AM|first|morning|| +36143|AAAAAAAAADNIAAAA|36143|10|2|23|AM|first|morning|| +36144|AAAAAAAABDNIAAAA|36144|10|2|24|AM|first|morning|| +36145|AAAAAAAACDNIAAAA|36145|10|2|25|AM|first|morning|| +36146|AAAAAAAADDNIAAAA|36146|10|2|26|AM|first|morning|| +36147|AAAAAAAAEDNIAAAA|36147|10|2|27|AM|first|morning|| +36148|AAAAAAAAFDNIAAAA|36148|10|2|28|AM|first|morning|| +36149|AAAAAAAAGDNIAAAA|36149|10|2|29|AM|first|morning|| +36150|AAAAAAAAHDNIAAAA|36150|10|2|30|AM|first|morning|| +36151|AAAAAAAAIDNIAAAA|36151|10|2|31|AM|first|morning|| +36152|AAAAAAAAJDNIAAAA|36152|10|2|32|AM|first|morning|| +36153|AAAAAAAAKDNIAAAA|36153|10|2|33|AM|first|morning|| +36154|AAAAAAAALDNIAAAA|36154|10|2|34|AM|first|morning|| +36155|AAAAAAAAMDNIAAAA|36155|10|2|35|AM|first|morning|| +36156|AAAAAAAANDNIAAAA|36156|10|2|36|AM|first|morning|| +36157|AAAAAAAAODNIAAAA|36157|10|2|37|AM|first|morning|| +36158|AAAAAAAAPDNIAAAA|36158|10|2|38|AM|first|morning|| +36159|AAAAAAAAAENIAAAA|36159|10|2|39|AM|first|morning|| +36160|AAAAAAAABENIAAAA|36160|10|2|40|AM|first|morning|| +36161|AAAAAAAACENIAAAA|36161|10|2|41|AM|first|morning|| +36162|AAAAAAAADENIAAAA|36162|10|2|42|AM|first|morning|| +36163|AAAAAAAAEENIAAAA|36163|10|2|43|AM|first|morning|| +36164|AAAAAAAAFENIAAAA|36164|10|2|44|AM|first|morning|| +36165|AAAAAAAAGENIAAAA|36165|10|2|45|AM|first|morning|| +36166|AAAAAAAAHENIAAAA|36166|10|2|46|AM|first|morning|| +36167|AAAAAAAAIENIAAAA|36167|10|2|47|AM|first|morning|| +36168|AAAAAAAAJENIAAAA|36168|10|2|48|AM|first|morning|| +36169|AAAAAAAAKENIAAAA|36169|10|2|49|AM|first|morning|| +36170|AAAAAAAALENIAAAA|36170|10|2|50|AM|first|morning|| +36171|AAAAAAAAMENIAAAA|36171|10|2|51|AM|first|morning|| +36172|AAAAAAAANENIAAAA|36172|10|2|52|AM|first|morning|| +36173|AAAAAAAAOENIAAAA|36173|10|2|53|AM|first|morning|| +36174|AAAAAAAAPENIAAAA|36174|10|2|54|AM|first|morning|| +36175|AAAAAAAAAFNIAAAA|36175|10|2|55|AM|first|morning|| +36176|AAAAAAAABFNIAAAA|36176|10|2|56|AM|first|morning|| +36177|AAAAAAAACFNIAAAA|36177|10|2|57|AM|first|morning|| +36178|AAAAAAAADFNIAAAA|36178|10|2|58|AM|first|morning|| +36179|AAAAAAAAEFNIAAAA|36179|10|2|59|AM|first|morning|| +36180|AAAAAAAAFFNIAAAA|36180|10|3|0|AM|first|morning|| +36181|AAAAAAAAGFNIAAAA|36181|10|3|1|AM|first|morning|| +36182|AAAAAAAAHFNIAAAA|36182|10|3|2|AM|first|morning|| +36183|AAAAAAAAIFNIAAAA|36183|10|3|3|AM|first|morning|| +36184|AAAAAAAAJFNIAAAA|36184|10|3|4|AM|first|morning|| +36185|AAAAAAAAKFNIAAAA|36185|10|3|5|AM|first|morning|| +36186|AAAAAAAALFNIAAAA|36186|10|3|6|AM|first|morning|| +36187|AAAAAAAAMFNIAAAA|36187|10|3|7|AM|first|morning|| +36188|AAAAAAAANFNIAAAA|36188|10|3|8|AM|first|morning|| +36189|AAAAAAAAOFNIAAAA|36189|10|3|9|AM|first|morning|| +36190|AAAAAAAAPFNIAAAA|36190|10|3|10|AM|first|morning|| +36191|AAAAAAAAAGNIAAAA|36191|10|3|11|AM|first|morning|| +36192|AAAAAAAABGNIAAAA|36192|10|3|12|AM|first|morning|| +36193|AAAAAAAACGNIAAAA|36193|10|3|13|AM|first|morning|| +36194|AAAAAAAADGNIAAAA|36194|10|3|14|AM|first|morning|| +36195|AAAAAAAAEGNIAAAA|36195|10|3|15|AM|first|morning|| +36196|AAAAAAAAFGNIAAAA|36196|10|3|16|AM|first|morning|| +36197|AAAAAAAAGGNIAAAA|36197|10|3|17|AM|first|morning|| +36198|AAAAAAAAHGNIAAAA|36198|10|3|18|AM|first|morning|| +36199|AAAAAAAAIGNIAAAA|36199|10|3|19|AM|first|morning|| +36200|AAAAAAAAJGNIAAAA|36200|10|3|20|AM|first|morning|| +36201|AAAAAAAAKGNIAAAA|36201|10|3|21|AM|first|morning|| +36202|AAAAAAAALGNIAAAA|36202|10|3|22|AM|first|morning|| +36203|AAAAAAAAMGNIAAAA|36203|10|3|23|AM|first|morning|| +36204|AAAAAAAANGNIAAAA|36204|10|3|24|AM|first|morning|| +36205|AAAAAAAAOGNIAAAA|36205|10|3|25|AM|first|morning|| +36206|AAAAAAAAPGNIAAAA|36206|10|3|26|AM|first|morning|| +36207|AAAAAAAAAHNIAAAA|36207|10|3|27|AM|first|morning|| +36208|AAAAAAAABHNIAAAA|36208|10|3|28|AM|first|morning|| +36209|AAAAAAAACHNIAAAA|36209|10|3|29|AM|first|morning|| +36210|AAAAAAAADHNIAAAA|36210|10|3|30|AM|first|morning|| +36211|AAAAAAAAEHNIAAAA|36211|10|3|31|AM|first|morning|| +36212|AAAAAAAAFHNIAAAA|36212|10|3|32|AM|first|morning|| +36213|AAAAAAAAGHNIAAAA|36213|10|3|33|AM|first|morning|| +36214|AAAAAAAAHHNIAAAA|36214|10|3|34|AM|first|morning|| +36215|AAAAAAAAIHNIAAAA|36215|10|3|35|AM|first|morning|| +36216|AAAAAAAAJHNIAAAA|36216|10|3|36|AM|first|morning|| +36217|AAAAAAAAKHNIAAAA|36217|10|3|37|AM|first|morning|| +36218|AAAAAAAALHNIAAAA|36218|10|3|38|AM|first|morning|| +36219|AAAAAAAAMHNIAAAA|36219|10|3|39|AM|first|morning|| +36220|AAAAAAAANHNIAAAA|36220|10|3|40|AM|first|morning|| +36221|AAAAAAAAOHNIAAAA|36221|10|3|41|AM|first|morning|| +36222|AAAAAAAAPHNIAAAA|36222|10|3|42|AM|first|morning|| +36223|AAAAAAAAAINIAAAA|36223|10|3|43|AM|first|morning|| +36224|AAAAAAAABINIAAAA|36224|10|3|44|AM|first|morning|| +36225|AAAAAAAACINIAAAA|36225|10|3|45|AM|first|morning|| +36226|AAAAAAAADINIAAAA|36226|10|3|46|AM|first|morning|| +36227|AAAAAAAAEINIAAAA|36227|10|3|47|AM|first|morning|| +36228|AAAAAAAAFINIAAAA|36228|10|3|48|AM|first|morning|| +36229|AAAAAAAAGINIAAAA|36229|10|3|49|AM|first|morning|| +36230|AAAAAAAAHINIAAAA|36230|10|3|50|AM|first|morning|| +36231|AAAAAAAAIINIAAAA|36231|10|3|51|AM|first|morning|| +36232|AAAAAAAAJINIAAAA|36232|10|3|52|AM|first|morning|| +36233|AAAAAAAAKINIAAAA|36233|10|3|53|AM|first|morning|| +36234|AAAAAAAALINIAAAA|36234|10|3|54|AM|first|morning|| +36235|AAAAAAAAMINIAAAA|36235|10|3|55|AM|first|morning|| +36236|AAAAAAAANINIAAAA|36236|10|3|56|AM|first|morning|| +36237|AAAAAAAAOINIAAAA|36237|10|3|57|AM|first|morning|| +36238|AAAAAAAAPINIAAAA|36238|10|3|58|AM|first|morning|| +36239|AAAAAAAAAJNIAAAA|36239|10|3|59|AM|first|morning|| +36240|AAAAAAAABJNIAAAA|36240|10|4|0|AM|first|morning|| +36241|AAAAAAAACJNIAAAA|36241|10|4|1|AM|first|morning|| +36242|AAAAAAAADJNIAAAA|36242|10|4|2|AM|first|morning|| +36243|AAAAAAAAEJNIAAAA|36243|10|4|3|AM|first|morning|| +36244|AAAAAAAAFJNIAAAA|36244|10|4|4|AM|first|morning|| +36245|AAAAAAAAGJNIAAAA|36245|10|4|5|AM|first|morning|| +36246|AAAAAAAAHJNIAAAA|36246|10|4|6|AM|first|morning|| +36247|AAAAAAAAIJNIAAAA|36247|10|4|7|AM|first|morning|| +36248|AAAAAAAAJJNIAAAA|36248|10|4|8|AM|first|morning|| +36249|AAAAAAAAKJNIAAAA|36249|10|4|9|AM|first|morning|| +36250|AAAAAAAALJNIAAAA|36250|10|4|10|AM|first|morning|| +36251|AAAAAAAAMJNIAAAA|36251|10|4|11|AM|first|morning|| +36252|AAAAAAAANJNIAAAA|36252|10|4|12|AM|first|morning|| +36253|AAAAAAAAOJNIAAAA|36253|10|4|13|AM|first|morning|| +36254|AAAAAAAAPJNIAAAA|36254|10|4|14|AM|first|morning|| +36255|AAAAAAAAAKNIAAAA|36255|10|4|15|AM|first|morning|| +36256|AAAAAAAABKNIAAAA|36256|10|4|16|AM|first|morning|| +36257|AAAAAAAACKNIAAAA|36257|10|4|17|AM|first|morning|| +36258|AAAAAAAADKNIAAAA|36258|10|4|18|AM|first|morning|| +36259|AAAAAAAAEKNIAAAA|36259|10|4|19|AM|first|morning|| +36260|AAAAAAAAFKNIAAAA|36260|10|4|20|AM|first|morning|| +36261|AAAAAAAAGKNIAAAA|36261|10|4|21|AM|first|morning|| +36262|AAAAAAAAHKNIAAAA|36262|10|4|22|AM|first|morning|| +36263|AAAAAAAAIKNIAAAA|36263|10|4|23|AM|first|morning|| +36264|AAAAAAAAJKNIAAAA|36264|10|4|24|AM|first|morning|| +36265|AAAAAAAAKKNIAAAA|36265|10|4|25|AM|first|morning|| +36266|AAAAAAAALKNIAAAA|36266|10|4|26|AM|first|morning|| +36267|AAAAAAAAMKNIAAAA|36267|10|4|27|AM|first|morning|| +36268|AAAAAAAANKNIAAAA|36268|10|4|28|AM|first|morning|| +36269|AAAAAAAAOKNIAAAA|36269|10|4|29|AM|first|morning|| +36270|AAAAAAAAPKNIAAAA|36270|10|4|30|AM|first|morning|| +36271|AAAAAAAAALNIAAAA|36271|10|4|31|AM|first|morning|| +36272|AAAAAAAABLNIAAAA|36272|10|4|32|AM|first|morning|| +36273|AAAAAAAACLNIAAAA|36273|10|4|33|AM|first|morning|| +36274|AAAAAAAADLNIAAAA|36274|10|4|34|AM|first|morning|| +36275|AAAAAAAAELNIAAAA|36275|10|4|35|AM|first|morning|| +36276|AAAAAAAAFLNIAAAA|36276|10|4|36|AM|first|morning|| +36277|AAAAAAAAGLNIAAAA|36277|10|4|37|AM|first|morning|| +36278|AAAAAAAAHLNIAAAA|36278|10|4|38|AM|first|morning|| +36279|AAAAAAAAILNIAAAA|36279|10|4|39|AM|first|morning|| +36280|AAAAAAAAJLNIAAAA|36280|10|4|40|AM|first|morning|| +36281|AAAAAAAAKLNIAAAA|36281|10|4|41|AM|first|morning|| +36282|AAAAAAAALLNIAAAA|36282|10|4|42|AM|first|morning|| +36283|AAAAAAAAMLNIAAAA|36283|10|4|43|AM|first|morning|| +36284|AAAAAAAANLNIAAAA|36284|10|4|44|AM|first|morning|| +36285|AAAAAAAAOLNIAAAA|36285|10|4|45|AM|first|morning|| +36286|AAAAAAAAPLNIAAAA|36286|10|4|46|AM|first|morning|| +36287|AAAAAAAAAMNIAAAA|36287|10|4|47|AM|first|morning|| +36288|AAAAAAAABMNIAAAA|36288|10|4|48|AM|first|morning|| +36289|AAAAAAAACMNIAAAA|36289|10|4|49|AM|first|morning|| +36290|AAAAAAAADMNIAAAA|36290|10|4|50|AM|first|morning|| +36291|AAAAAAAAEMNIAAAA|36291|10|4|51|AM|first|morning|| +36292|AAAAAAAAFMNIAAAA|36292|10|4|52|AM|first|morning|| +36293|AAAAAAAAGMNIAAAA|36293|10|4|53|AM|first|morning|| +36294|AAAAAAAAHMNIAAAA|36294|10|4|54|AM|first|morning|| +36295|AAAAAAAAIMNIAAAA|36295|10|4|55|AM|first|morning|| +36296|AAAAAAAAJMNIAAAA|36296|10|4|56|AM|first|morning|| +36297|AAAAAAAAKMNIAAAA|36297|10|4|57|AM|first|morning|| +36298|AAAAAAAALMNIAAAA|36298|10|4|58|AM|first|morning|| +36299|AAAAAAAAMMNIAAAA|36299|10|4|59|AM|first|morning|| +36300|AAAAAAAANMNIAAAA|36300|10|5|0|AM|first|morning|| +36301|AAAAAAAAOMNIAAAA|36301|10|5|1|AM|first|morning|| +36302|AAAAAAAAPMNIAAAA|36302|10|5|2|AM|first|morning|| +36303|AAAAAAAAANNIAAAA|36303|10|5|3|AM|first|morning|| +36304|AAAAAAAABNNIAAAA|36304|10|5|4|AM|first|morning|| +36305|AAAAAAAACNNIAAAA|36305|10|5|5|AM|first|morning|| +36306|AAAAAAAADNNIAAAA|36306|10|5|6|AM|first|morning|| +36307|AAAAAAAAENNIAAAA|36307|10|5|7|AM|first|morning|| +36308|AAAAAAAAFNNIAAAA|36308|10|5|8|AM|first|morning|| +36309|AAAAAAAAGNNIAAAA|36309|10|5|9|AM|first|morning|| +36310|AAAAAAAAHNNIAAAA|36310|10|5|10|AM|first|morning|| +36311|AAAAAAAAINNIAAAA|36311|10|5|11|AM|first|morning|| +36312|AAAAAAAAJNNIAAAA|36312|10|5|12|AM|first|morning|| +36313|AAAAAAAAKNNIAAAA|36313|10|5|13|AM|first|morning|| +36314|AAAAAAAALNNIAAAA|36314|10|5|14|AM|first|morning|| +36315|AAAAAAAAMNNIAAAA|36315|10|5|15|AM|first|morning|| +36316|AAAAAAAANNNIAAAA|36316|10|5|16|AM|first|morning|| +36317|AAAAAAAAONNIAAAA|36317|10|5|17|AM|first|morning|| +36318|AAAAAAAAPNNIAAAA|36318|10|5|18|AM|first|morning|| +36319|AAAAAAAAAONIAAAA|36319|10|5|19|AM|first|morning|| +36320|AAAAAAAABONIAAAA|36320|10|5|20|AM|first|morning|| +36321|AAAAAAAACONIAAAA|36321|10|5|21|AM|first|morning|| +36322|AAAAAAAADONIAAAA|36322|10|5|22|AM|first|morning|| +36323|AAAAAAAAEONIAAAA|36323|10|5|23|AM|first|morning|| +36324|AAAAAAAAFONIAAAA|36324|10|5|24|AM|first|morning|| +36325|AAAAAAAAGONIAAAA|36325|10|5|25|AM|first|morning|| +36326|AAAAAAAAHONIAAAA|36326|10|5|26|AM|first|morning|| +36327|AAAAAAAAIONIAAAA|36327|10|5|27|AM|first|morning|| +36328|AAAAAAAAJONIAAAA|36328|10|5|28|AM|first|morning|| +36329|AAAAAAAAKONIAAAA|36329|10|5|29|AM|first|morning|| +36330|AAAAAAAALONIAAAA|36330|10|5|30|AM|first|morning|| +36331|AAAAAAAAMONIAAAA|36331|10|5|31|AM|first|morning|| +36332|AAAAAAAANONIAAAA|36332|10|5|32|AM|first|morning|| +36333|AAAAAAAAOONIAAAA|36333|10|5|33|AM|first|morning|| +36334|AAAAAAAAPONIAAAA|36334|10|5|34|AM|first|morning|| +36335|AAAAAAAAAPNIAAAA|36335|10|5|35|AM|first|morning|| +36336|AAAAAAAABPNIAAAA|36336|10|5|36|AM|first|morning|| +36337|AAAAAAAACPNIAAAA|36337|10|5|37|AM|first|morning|| +36338|AAAAAAAADPNIAAAA|36338|10|5|38|AM|first|morning|| +36339|AAAAAAAAEPNIAAAA|36339|10|5|39|AM|first|morning|| +36340|AAAAAAAAFPNIAAAA|36340|10|5|40|AM|first|morning|| +36341|AAAAAAAAGPNIAAAA|36341|10|5|41|AM|first|morning|| +36342|AAAAAAAAHPNIAAAA|36342|10|5|42|AM|first|morning|| +36343|AAAAAAAAIPNIAAAA|36343|10|5|43|AM|first|morning|| +36344|AAAAAAAAJPNIAAAA|36344|10|5|44|AM|first|morning|| +36345|AAAAAAAAKPNIAAAA|36345|10|5|45|AM|first|morning|| +36346|AAAAAAAALPNIAAAA|36346|10|5|46|AM|first|morning|| +36347|AAAAAAAAMPNIAAAA|36347|10|5|47|AM|first|morning|| +36348|AAAAAAAANPNIAAAA|36348|10|5|48|AM|first|morning|| +36349|AAAAAAAAOPNIAAAA|36349|10|5|49|AM|first|morning|| +36350|AAAAAAAAPPNIAAAA|36350|10|5|50|AM|first|morning|| +36351|AAAAAAAAAAOIAAAA|36351|10|5|51|AM|first|morning|| +36352|AAAAAAAABAOIAAAA|36352|10|5|52|AM|first|morning|| +36353|AAAAAAAACAOIAAAA|36353|10|5|53|AM|first|morning|| +36354|AAAAAAAADAOIAAAA|36354|10|5|54|AM|first|morning|| +36355|AAAAAAAAEAOIAAAA|36355|10|5|55|AM|first|morning|| +36356|AAAAAAAAFAOIAAAA|36356|10|5|56|AM|first|morning|| +36357|AAAAAAAAGAOIAAAA|36357|10|5|57|AM|first|morning|| +36358|AAAAAAAAHAOIAAAA|36358|10|5|58|AM|first|morning|| +36359|AAAAAAAAIAOIAAAA|36359|10|5|59|AM|first|morning|| +36360|AAAAAAAAJAOIAAAA|36360|10|6|0|AM|first|morning|| +36361|AAAAAAAAKAOIAAAA|36361|10|6|1|AM|first|morning|| +36362|AAAAAAAALAOIAAAA|36362|10|6|2|AM|first|morning|| +36363|AAAAAAAAMAOIAAAA|36363|10|6|3|AM|first|morning|| +36364|AAAAAAAANAOIAAAA|36364|10|6|4|AM|first|morning|| +36365|AAAAAAAAOAOIAAAA|36365|10|6|5|AM|first|morning|| +36366|AAAAAAAAPAOIAAAA|36366|10|6|6|AM|first|morning|| +36367|AAAAAAAAABOIAAAA|36367|10|6|7|AM|first|morning|| +36368|AAAAAAAABBOIAAAA|36368|10|6|8|AM|first|morning|| +36369|AAAAAAAACBOIAAAA|36369|10|6|9|AM|first|morning|| +36370|AAAAAAAADBOIAAAA|36370|10|6|10|AM|first|morning|| +36371|AAAAAAAAEBOIAAAA|36371|10|6|11|AM|first|morning|| +36372|AAAAAAAAFBOIAAAA|36372|10|6|12|AM|first|morning|| +36373|AAAAAAAAGBOIAAAA|36373|10|6|13|AM|first|morning|| +36374|AAAAAAAAHBOIAAAA|36374|10|6|14|AM|first|morning|| +36375|AAAAAAAAIBOIAAAA|36375|10|6|15|AM|first|morning|| +36376|AAAAAAAAJBOIAAAA|36376|10|6|16|AM|first|morning|| +36377|AAAAAAAAKBOIAAAA|36377|10|6|17|AM|first|morning|| +36378|AAAAAAAALBOIAAAA|36378|10|6|18|AM|first|morning|| +36379|AAAAAAAAMBOIAAAA|36379|10|6|19|AM|first|morning|| +36380|AAAAAAAANBOIAAAA|36380|10|6|20|AM|first|morning|| +36381|AAAAAAAAOBOIAAAA|36381|10|6|21|AM|first|morning|| +36382|AAAAAAAAPBOIAAAA|36382|10|6|22|AM|first|morning|| +36383|AAAAAAAAACOIAAAA|36383|10|6|23|AM|first|morning|| +36384|AAAAAAAABCOIAAAA|36384|10|6|24|AM|first|morning|| +36385|AAAAAAAACCOIAAAA|36385|10|6|25|AM|first|morning|| +36386|AAAAAAAADCOIAAAA|36386|10|6|26|AM|first|morning|| +36387|AAAAAAAAECOIAAAA|36387|10|6|27|AM|first|morning|| +36388|AAAAAAAAFCOIAAAA|36388|10|6|28|AM|first|morning|| +36389|AAAAAAAAGCOIAAAA|36389|10|6|29|AM|first|morning|| +36390|AAAAAAAAHCOIAAAA|36390|10|6|30|AM|first|morning|| +36391|AAAAAAAAICOIAAAA|36391|10|6|31|AM|first|morning|| +36392|AAAAAAAAJCOIAAAA|36392|10|6|32|AM|first|morning|| +36393|AAAAAAAAKCOIAAAA|36393|10|6|33|AM|first|morning|| +36394|AAAAAAAALCOIAAAA|36394|10|6|34|AM|first|morning|| +36395|AAAAAAAAMCOIAAAA|36395|10|6|35|AM|first|morning|| +36396|AAAAAAAANCOIAAAA|36396|10|6|36|AM|first|morning|| +36397|AAAAAAAAOCOIAAAA|36397|10|6|37|AM|first|morning|| +36398|AAAAAAAAPCOIAAAA|36398|10|6|38|AM|first|morning|| +36399|AAAAAAAAADOIAAAA|36399|10|6|39|AM|first|morning|| +36400|AAAAAAAABDOIAAAA|36400|10|6|40|AM|first|morning|| +36401|AAAAAAAACDOIAAAA|36401|10|6|41|AM|first|morning|| +36402|AAAAAAAADDOIAAAA|36402|10|6|42|AM|first|morning|| +36403|AAAAAAAAEDOIAAAA|36403|10|6|43|AM|first|morning|| +36404|AAAAAAAAFDOIAAAA|36404|10|6|44|AM|first|morning|| +36405|AAAAAAAAGDOIAAAA|36405|10|6|45|AM|first|morning|| +36406|AAAAAAAAHDOIAAAA|36406|10|6|46|AM|first|morning|| +36407|AAAAAAAAIDOIAAAA|36407|10|6|47|AM|first|morning|| +36408|AAAAAAAAJDOIAAAA|36408|10|6|48|AM|first|morning|| +36409|AAAAAAAAKDOIAAAA|36409|10|6|49|AM|first|morning|| +36410|AAAAAAAALDOIAAAA|36410|10|6|50|AM|first|morning|| +36411|AAAAAAAAMDOIAAAA|36411|10|6|51|AM|first|morning|| +36412|AAAAAAAANDOIAAAA|36412|10|6|52|AM|first|morning|| +36413|AAAAAAAAODOIAAAA|36413|10|6|53|AM|first|morning|| +36414|AAAAAAAAPDOIAAAA|36414|10|6|54|AM|first|morning|| +36415|AAAAAAAAAEOIAAAA|36415|10|6|55|AM|first|morning|| +36416|AAAAAAAABEOIAAAA|36416|10|6|56|AM|first|morning|| +36417|AAAAAAAACEOIAAAA|36417|10|6|57|AM|first|morning|| +36418|AAAAAAAADEOIAAAA|36418|10|6|58|AM|first|morning|| +36419|AAAAAAAAEEOIAAAA|36419|10|6|59|AM|first|morning|| +36420|AAAAAAAAFEOIAAAA|36420|10|7|0|AM|first|morning|| +36421|AAAAAAAAGEOIAAAA|36421|10|7|1|AM|first|morning|| +36422|AAAAAAAAHEOIAAAA|36422|10|7|2|AM|first|morning|| +36423|AAAAAAAAIEOIAAAA|36423|10|7|3|AM|first|morning|| +36424|AAAAAAAAJEOIAAAA|36424|10|7|4|AM|first|morning|| +36425|AAAAAAAAKEOIAAAA|36425|10|7|5|AM|first|morning|| +36426|AAAAAAAALEOIAAAA|36426|10|7|6|AM|first|morning|| +36427|AAAAAAAAMEOIAAAA|36427|10|7|7|AM|first|morning|| +36428|AAAAAAAANEOIAAAA|36428|10|7|8|AM|first|morning|| +36429|AAAAAAAAOEOIAAAA|36429|10|7|9|AM|first|morning|| +36430|AAAAAAAAPEOIAAAA|36430|10|7|10|AM|first|morning|| +36431|AAAAAAAAAFOIAAAA|36431|10|7|11|AM|first|morning|| +36432|AAAAAAAABFOIAAAA|36432|10|7|12|AM|first|morning|| +36433|AAAAAAAACFOIAAAA|36433|10|7|13|AM|first|morning|| +36434|AAAAAAAADFOIAAAA|36434|10|7|14|AM|first|morning|| +36435|AAAAAAAAEFOIAAAA|36435|10|7|15|AM|first|morning|| +36436|AAAAAAAAFFOIAAAA|36436|10|7|16|AM|first|morning|| +36437|AAAAAAAAGFOIAAAA|36437|10|7|17|AM|first|morning|| +36438|AAAAAAAAHFOIAAAA|36438|10|7|18|AM|first|morning|| +36439|AAAAAAAAIFOIAAAA|36439|10|7|19|AM|first|morning|| +36440|AAAAAAAAJFOIAAAA|36440|10|7|20|AM|first|morning|| +36441|AAAAAAAAKFOIAAAA|36441|10|7|21|AM|first|morning|| +36442|AAAAAAAALFOIAAAA|36442|10|7|22|AM|first|morning|| +36443|AAAAAAAAMFOIAAAA|36443|10|7|23|AM|first|morning|| +36444|AAAAAAAANFOIAAAA|36444|10|7|24|AM|first|morning|| +36445|AAAAAAAAOFOIAAAA|36445|10|7|25|AM|first|morning|| +36446|AAAAAAAAPFOIAAAA|36446|10|7|26|AM|first|morning|| +36447|AAAAAAAAAGOIAAAA|36447|10|7|27|AM|first|morning|| +36448|AAAAAAAABGOIAAAA|36448|10|7|28|AM|first|morning|| +36449|AAAAAAAACGOIAAAA|36449|10|7|29|AM|first|morning|| +36450|AAAAAAAADGOIAAAA|36450|10|7|30|AM|first|morning|| +36451|AAAAAAAAEGOIAAAA|36451|10|7|31|AM|first|morning|| +36452|AAAAAAAAFGOIAAAA|36452|10|7|32|AM|first|morning|| +36453|AAAAAAAAGGOIAAAA|36453|10|7|33|AM|first|morning|| +36454|AAAAAAAAHGOIAAAA|36454|10|7|34|AM|first|morning|| +36455|AAAAAAAAIGOIAAAA|36455|10|7|35|AM|first|morning|| +36456|AAAAAAAAJGOIAAAA|36456|10|7|36|AM|first|morning|| +36457|AAAAAAAAKGOIAAAA|36457|10|7|37|AM|first|morning|| +36458|AAAAAAAALGOIAAAA|36458|10|7|38|AM|first|morning|| +36459|AAAAAAAAMGOIAAAA|36459|10|7|39|AM|first|morning|| +36460|AAAAAAAANGOIAAAA|36460|10|7|40|AM|first|morning|| +36461|AAAAAAAAOGOIAAAA|36461|10|7|41|AM|first|morning|| +36462|AAAAAAAAPGOIAAAA|36462|10|7|42|AM|first|morning|| +36463|AAAAAAAAAHOIAAAA|36463|10|7|43|AM|first|morning|| +36464|AAAAAAAABHOIAAAA|36464|10|7|44|AM|first|morning|| +36465|AAAAAAAACHOIAAAA|36465|10|7|45|AM|first|morning|| +36466|AAAAAAAADHOIAAAA|36466|10|7|46|AM|first|morning|| +36467|AAAAAAAAEHOIAAAA|36467|10|7|47|AM|first|morning|| +36468|AAAAAAAAFHOIAAAA|36468|10|7|48|AM|first|morning|| +36469|AAAAAAAAGHOIAAAA|36469|10|7|49|AM|first|morning|| +36470|AAAAAAAAHHOIAAAA|36470|10|7|50|AM|first|morning|| +36471|AAAAAAAAIHOIAAAA|36471|10|7|51|AM|first|morning|| +36472|AAAAAAAAJHOIAAAA|36472|10|7|52|AM|first|morning|| +36473|AAAAAAAAKHOIAAAA|36473|10|7|53|AM|first|morning|| +36474|AAAAAAAALHOIAAAA|36474|10|7|54|AM|first|morning|| +36475|AAAAAAAAMHOIAAAA|36475|10|7|55|AM|first|morning|| +36476|AAAAAAAANHOIAAAA|36476|10|7|56|AM|first|morning|| +36477|AAAAAAAAOHOIAAAA|36477|10|7|57|AM|first|morning|| +36478|AAAAAAAAPHOIAAAA|36478|10|7|58|AM|first|morning|| +36479|AAAAAAAAAIOIAAAA|36479|10|7|59|AM|first|morning|| +36480|AAAAAAAABIOIAAAA|36480|10|8|0|AM|first|morning|| +36481|AAAAAAAACIOIAAAA|36481|10|8|1|AM|first|morning|| +36482|AAAAAAAADIOIAAAA|36482|10|8|2|AM|first|morning|| +36483|AAAAAAAAEIOIAAAA|36483|10|8|3|AM|first|morning|| +36484|AAAAAAAAFIOIAAAA|36484|10|8|4|AM|first|morning|| +36485|AAAAAAAAGIOIAAAA|36485|10|8|5|AM|first|morning|| +36486|AAAAAAAAHIOIAAAA|36486|10|8|6|AM|first|morning|| +36487|AAAAAAAAIIOIAAAA|36487|10|8|7|AM|first|morning|| +36488|AAAAAAAAJIOIAAAA|36488|10|8|8|AM|first|morning|| +36489|AAAAAAAAKIOIAAAA|36489|10|8|9|AM|first|morning|| +36490|AAAAAAAALIOIAAAA|36490|10|8|10|AM|first|morning|| +36491|AAAAAAAAMIOIAAAA|36491|10|8|11|AM|first|morning|| +36492|AAAAAAAANIOIAAAA|36492|10|8|12|AM|first|morning|| +36493|AAAAAAAAOIOIAAAA|36493|10|8|13|AM|first|morning|| +36494|AAAAAAAAPIOIAAAA|36494|10|8|14|AM|first|morning|| +36495|AAAAAAAAAJOIAAAA|36495|10|8|15|AM|first|morning|| +36496|AAAAAAAABJOIAAAA|36496|10|8|16|AM|first|morning|| +36497|AAAAAAAACJOIAAAA|36497|10|8|17|AM|first|morning|| +36498|AAAAAAAADJOIAAAA|36498|10|8|18|AM|first|morning|| +36499|AAAAAAAAEJOIAAAA|36499|10|8|19|AM|first|morning|| +36500|AAAAAAAAFJOIAAAA|36500|10|8|20|AM|first|morning|| +36501|AAAAAAAAGJOIAAAA|36501|10|8|21|AM|first|morning|| +36502|AAAAAAAAHJOIAAAA|36502|10|8|22|AM|first|morning|| +36503|AAAAAAAAIJOIAAAA|36503|10|8|23|AM|first|morning|| +36504|AAAAAAAAJJOIAAAA|36504|10|8|24|AM|first|morning|| +36505|AAAAAAAAKJOIAAAA|36505|10|8|25|AM|first|morning|| +36506|AAAAAAAALJOIAAAA|36506|10|8|26|AM|first|morning|| +36507|AAAAAAAAMJOIAAAA|36507|10|8|27|AM|first|morning|| +36508|AAAAAAAANJOIAAAA|36508|10|8|28|AM|first|morning|| +36509|AAAAAAAAOJOIAAAA|36509|10|8|29|AM|first|morning|| +36510|AAAAAAAAPJOIAAAA|36510|10|8|30|AM|first|morning|| +36511|AAAAAAAAAKOIAAAA|36511|10|8|31|AM|first|morning|| +36512|AAAAAAAABKOIAAAA|36512|10|8|32|AM|first|morning|| +36513|AAAAAAAACKOIAAAA|36513|10|8|33|AM|first|morning|| +36514|AAAAAAAADKOIAAAA|36514|10|8|34|AM|first|morning|| +36515|AAAAAAAAEKOIAAAA|36515|10|8|35|AM|first|morning|| +36516|AAAAAAAAFKOIAAAA|36516|10|8|36|AM|first|morning|| +36517|AAAAAAAAGKOIAAAA|36517|10|8|37|AM|first|morning|| +36518|AAAAAAAAHKOIAAAA|36518|10|8|38|AM|first|morning|| +36519|AAAAAAAAIKOIAAAA|36519|10|8|39|AM|first|morning|| +36520|AAAAAAAAJKOIAAAA|36520|10|8|40|AM|first|morning|| +36521|AAAAAAAAKKOIAAAA|36521|10|8|41|AM|first|morning|| +36522|AAAAAAAALKOIAAAA|36522|10|8|42|AM|first|morning|| +36523|AAAAAAAAMKOIAAAA|36523|10|8|43|AM|first|morning|| +36524|AAAAAAAANKOIAAAA|36524|10|8|44|AM|first|morning|| +36525|AAAAAAAAOKOIAAAA|36525|10|8|45|AM|first|morning|| +36526|AAAAAAAAPKOIAAAA|36526|10|8|46|AM|first|morning|| +36527|AAAAAAAAALOIAAAA|36527|10|8|47|AM|first|morning|| +36528|AAAAAAAABLOIAAAA|36528|10|8|48|AM|first|morning|| +36529|AAAAAAAACLOIAAAA|36529|10|8|49|AM|first|morning|| +36530|AAAAAAAADLOIAAAA|36530|10|8|50|AM|first|morning|| +36531|AAAAAAAAELOIAAAA|36531|10|8|51|AM|first|morning|| +36532|AAAAAAAAFLOIAAAA|36532|10|8|52|AM|first|morning|| +36533|AAAAAAAAGLOIAAAA|36533|10|8|53|AM|first|morning|| +36534|AAAAAAAAHLOIAAAA|36534|10|8|54|AM|first|morning|| +36535|AAAAAAAAILOIAAAA|36535|10|8|55|AM|first|morning|| +36536|AAAAAAAAJLOIAAAA|36536|10|8|56|AM|first|morning|| +36537|AAAAAAAAKLOIAAAA|36537|10|8|57|AM|first|morning|| +36538|AAAAAAAALLOIAAAA|36538|10|8|58|AM|first|morning|| +36539|AAAAAAAAMLOIAAAA|36539|10|8|59|AM|first|morning|| +36540|AAAAAAAANLOIAAAA|36540|10|9|0|AM|first|morning|| +36541|AAAAAAAAOLOIAAAA|36541|10|9|1|AM|first|morning|| +36542|AAAAAAAAPLOIAAAA|36542|10|9|2|AM|first|morning|| +36543|AAAAAAAAAMOIAAAA|36543|10|9|3|AM|first|morning|| +36544|AAAAAAAABMOIAAAA|36544|10|9|4|AM|first|morning|| +36545|AAAAAAAACMOIAAAA|36545|10|9|5|AM|first|morning|| +36546|AAAAAAAADMOIAAAA|36546|10|9|6|AM|first|morning|| +36547|AAAAAAAAEMOIAAAA|36547|10|9|7|AM|first|morning|| +36548|AAAAAAAAFMOIAAAA|36548|10|9|8|AM|first|morning|| +36549|AAAAAAAAGMOIAAAA|36549|10|9|9|AM|first|morning|| +36550|AAAAAAAAHMOIAAAA|36550|10|9|10|AM|first|morning|| +36551|AAAAAAAAIMOIAAAA|36551|10|9|11|AM|first|morning|| +36552|AAAAAAAAJMOIAAAA|36552|10|9|12|AM|first|morning|| +36553|AAAAAAAAKMOIAAAA|36553|10|9|13|AM|first|morning|| +36554|AAAAAAAALMOIAAAA|36554|10|9|14|AM|first|morning|| +36555|AAAAAAAAMMOIAAAA|36555|10|9|15|AM|first|morning|| +36556|AAAAAAAANMOIAAAA|36556|10|9|16|AM|first|morning|| +36557|AAAAAAAAOMOIAAAA|36557|10|9|17|AM|first|morning|| +36558|AAAAAAAAPMOIAAAA|36558|10|9|18|AM|first|morning|| +36559|AAAAAAAAANOIAAAA|36559|10|9|19|AM|first|morning|| +36560|AAAAAAAABNOIAAAA|36560|10|9|20|AM|first|morning|| +36561|AAAAAAAACNOIAAAA|36561|10|9|21|AM|first|morning|| +36562|AAAAAAAADNOIAAAA|36562|10|9|22|AM|first|morning|| +36563|AAAAAAAAENOIAAAA|36563|10|9|23|AM|first|morning|| +36564|AAAAAAAAFNOIAAAA|36564|10|9|24|AM|first|morning|| +36565|AAAAAAAAGNOIAAAA|36565|10|9|25|AM|first|morning|| +36566|AAAAAAAAHNOIAAAA|36566|10|9|26|AM|first|morning|| +36567|AAAAAAAAINOIAAAA|36567|10|9|27|AM|first|morning|| +36568|AAAAAAAAJNOIAAAA|36568|10|9|28|AM|first|morning|| +36569|AAAAAAAAKNOIAAAA|36569|10|9|29|AM|first|morning|| +36570|AAAAAAAALNOIAAAA|36570|10|9|30|AM|first|morning|| +36571|AAAAAAAAMNOIAAAA|36571|10|9|31|AM|first|morning|| +36572|AAAAAAAANNOIAAAA|36572|10|9|32|AM|first|morning|| +36573|AAAAAAAAONOIAAAA|36573|10|9|33|AM|first|morning|| +36574|AAAAAAAAPNOIAAAA|36574|10|9|34|AM|first|morning|| +36575|AAAAAAAAAOOIAAAA|36575|10|9|35|AM|first|morning|| +36576|AAAAAAAABOOIAAAA|36576|10|9|36|AM|first|morning|| +36577|AAAAAAAACOOIAAAA|36577|10|9|37|AM|first|morning|| +36578|AAAAAAAADOOIAAAA|36578|10|9|38|AM|first|morning|| +36579|AAAAAAAAEOOIAAAA|36579|10|9|39|AM|first|morning|| +36580|AAAAAAAAFOOIAAAA|36580|10|9|40|AM|first|morning|| +36581|AAAAAAAAGOOIAAAA|36581|10|9|41|AM|first|morning|| +36582|AAAAAAAAHOOIAAAA|36582|10|9|42|AM|first|morning|| +36583|AAAAAAAAIOOIAAAA|36583|10|9|43|AM|first|morning|| +36584|AAAAAAAAJOOIAAAA|36584|10|9|44|AM|first|morning|| +36585|AAAAAAAAKOOIAAAA|36585|10|9|45|AM|first|morning|| +36586|AAAAAAAALOOIAAAA|36586|10|9|46|AM|first|morning|| +36587|AAAAAAAAMOOIAAAA|36587|10|9|47|AM|first|morning|| +36588|AAAAAAAANOOIAAAA|36588|10|9|48|AM|first|morning|| +36589|AAAAAAAAOOOIAAAA|36589|10|9|49|AM|first|morning|| +36590|AAAAAAAAPOOIAAAA|36590|10|9|50|AM|first|morning|| +36591|AAAAAAAAAPOIAAAA|36591|10|9|51|AM|first|morning|| +36592|AAAAAAAABPOIAAAA|36592|10|9|52|AM|first|morning|| +36593|AAAAAAAACPOIAAAA|36593|10|9|53|AM|first|morning|| +36594|AAAAAAAADPOIAAAA|36594|10|9|54|AM|first|morning|| +36595|AAAAAAAAEPOIAAAA|36595|10|9|55|AM|first|morning|| +36596|AAAAAAAAFPOIAAAA|36596|10|9|56|AM|first|morning|| +36597|AAAAAAAAGPOIAAAA|36597|10|9|57|AM|first|morning|| +36598|AAAAAAAAHPOIAAAA|36598|10|9|58|AM|first|morning|| +36599|AAAAAAAAIPOIAAAA|36599|10|9|59|AM|first|morning|| +36600|AAAAAAAAJPOIAAAA|36600|10|10|0|AM|first|morning|| +36601|AAAAAAAAKPOIAAAA|36601|10|10|1|AM|first|morning|| +36602|AAAAAAAALPOIAAAA|36602|10|10|2|AM|first|morning|| +36603|AAAAAAAAMPOIAAAA|36603|10|10|3|AM|first|morning|| +36604|AAAAAAAANPOIAAAA|36604|10|10|4|AM|first|morning|| +36605|AAAAAAAAOPOIAAAA|36605|10|10|5|AM|first|morning|| +36606|AAAAAAAAPPOIAAAA|36606|10|10|6|AM|first|morning|| +36607|AAAAAAAAAAPIAAAA|36607|10|10|7|AM|first|morning|| +36608|AAAAAAAABAPIAAAA|36608|10|10|8|AM|first|morning|| +36609|AAAAAAAACAPIAAAA|36609|10|10|9|AM|first|morning|| +36610|AAAAAAAADAPIAAAA|36610|10|10|10|AM|first|morning|| +36611|AAAAAAAAEAPIAAAA|36611|10|10|11|AM|first|morning|| +36612|AAAAAAAAFAPIAAAA|36612|10|10|12|AM|first|morning|| +36613|AAAAAAAAGAPIAAAA|36613|10|10|13|AM|first|morning|| +36614|AAAAAAAAHAPIAAAA|36614|10|10|14|AM|first|morning|| +36615|AAAAAAAAIAPIAAAA|36615|10|10|15|AM|first|morning|| +36616|AAAAAAAAJAPIAAAA|36616|10|10|16|AM|first|morning|| +36617|AAAAAAAAKAPIAAAA|36617|10|10|17|AM|first|morning|| +36618|AAAAAAAALAPIAAAA|36618|10|10|18|AM|first|morning|| +36619|AAAAAAAAMAPIAAAA|36619|10|10|19|AM|first|morning|| +36620|AAAAAAAANAPIAAAA|36620|10|10|20|AM|first|morning|| +36621|AAAAAAAAOAPIAAAA|36621|10|10|21|AM|first|morning|| +36622|AAAAAAAAPAPIAAAA|36622|10|10|22|AM|first|morning|| +36623|AAAAAAAAABPIAAAA|36623|10|10|23|AM|first|morning|| +36624|AAAAAAAABBPIAAAA|36624|10|10|24|AM|first|morning|| +36625|AAAAAAAACBPIAAAA|36625|10|10|25|AM|first|morning|| +36626|AAAAAAAADBPIAAAA|36626|10|10|26|AM|first|morning|| +36627|AAAAAAAAEBPIAAAA|36627|10|10|27|AM|first|morning|| +36628|AAAAAAAAFBPIAAAA|36628|10|10|28|AM|first|morning|| +36629|AAAAAAAAGBPIAAAA|36629|10|10|29|AM|first|morning|| +36630|AAAAAAAAHBPIAAAA|36630|10|10|30|AM|first|morning|| +36631|AAAAAAAAIBPIAAAA|36631|10|10|31|AM|first|morning|| +36632|AAAAAAAAJBPIAAAA|36632|10|10|32|AM|first|morning|| +36633|AAAAAAAAKBPIAAAA|36633|10|10|33|AM|first|morning|| +36634|AAAAAAAALBPIAAAA|36634|10|10|34|AM|first|morning|| +36635|AAAAAAAAMBPIAAAA|36635|10|10|35|AM|first|morning|| +36636|AAAAAAAANBPIAAAA|36636|10|10|36|AM|first|morning|| +36637|AAAAAAAAOBPIAAAA|36637|10|10|37|AM|first|morning|| +36638|AAAAAAAAPBPIAAAA|36638|10|10|38|AM|first|morning|| +36639|AAAAAAAAACPIAAAA|36639|10|10|39|AM|first|morning|| +36640|AAAAAAAABCPIAAAA|36640|10|10|40|AM|first|morning|| +36641|AAAAAAAACCPIAAAA|36641|10|10|41|AM|first|morning|| +36642|AAAAAAAADCPIAAAA|36642|10|10|42|AM|first|morning|| +36643|AAAAAAAAECPIAAAA|36643|10|10|43|AM|first|morning|| +36644|AAAAAAAAFCPIAAAA|36644|10|10|44|AM|first|morning|| +36645|AAAAAAAAGCPIAAAA|36645|10|10|45|AM|first|morning|| +36646|AAAAAAAAHCPIAAAA|36646|10|10|46|AM|first|morning|| +36647|AAAAAAAAICPIAAAA|36647|10|10|47|AM|first|morning|| +36648|AAAAAAAAJCPIAAAA|36648|10|10|48|AM|first|morning|| +36649|AAAAAAAAKCPIAAAA|36649|10|10|49|AM|first|morning|| +36650|AAAAAAAALCPIAAAA|36650|10|10|50|AM|first|morning|| +36651|AAAAAAAAMCPIAAAA|36651|10|10|51|AM|first|morning|| +36652|AAAAAAAANCPIAAAA|36652|10|10|52|AM|first|morning|| +36653|AAAAAAAAOCPIAAAA|36653|10|10|53|AM|first|morning|| +36654|AAAAAAAAPCPIAAAA|36654|10|10|54|AM|first|morning|| +36655|AAAAAAAAADPIAAAA|36655|10|10|55|AM|first|morning|| +36656|AAAAAAAABDPIAAAA|36656|10|10|56|AM|first|morning|| +36657|AAAAAAAACDPIAAAA|36657|10|10|57|AM|first|morning|| +36658|AAAAAAAADDPIAAAA|36658|10|10|58|AM|first|morning|| +36659|AAAAAAAAEDPIAAAA|36659|10|10|59|AM|first|morning|| +36660|AAAAAAAAFDPIAAAA|36660|10|11|0|AM|first|morning|| +36661|AAAAAAAAGDPIAAAA|36661|10|11|1|AM|first|morning|| +36662|AAAAAAAAHDPIAAAA|36662|10|11|2|AM|first|morning|| +36663|AAAAAAAAIDPIAAAA|36663|10|11|3|AM|first|morning|| +36664|AAAAAAAAJDPIAAAA|36664|10|11|4|AM|first|morning|| +36665|AAAAAAAAKDPIAAAA|36665|10|11|5|AM|first|morning|| +36666|AAAAAAAALDPIAAAA|36666|10|11|6|AM|first|morning|| +36667|AAAAAAAAMDPIAAAA|36667|10|11|7|AM|first|morning|| +36668|AAAAAAAANDPIAAAA|36668|10|11|8|AM|first|morning|| +36669|AAAAAAAAODPIAAAA|36669|10|11|9|AM|first|morning|| +36670|AAAAAAAAPDPIAAAA|36670|10|11|10|AM|first|morning|| +36671|AAAAAAAAAEPIAAAA|36671|10|11|11|AM|first|morning|| +36672|AAAAAAAABEPIAAAA|36672|10|11|12|AM|first|morning|| +36673|AAAAAAAACEPIAAAA|36673|10|11|13|AM|first|morning|| +36674|AAAAAAAADEPIAAAA|36674|10|11|14|AM|first|morning|| +36675|AAAAAAAAEEPIAAAA|36675|10|11|15|AM|first|morning|| +36676|AAAAAAAAFEPIAAAA|36676|10|11|16|AM|first|morning|| +36677|AAAAAAAAGEPIAAAA|36677|10|11|17|AM|first|morning|| +36678|AAAAAAAAHEPIAAAA|36678|10|11|18|AM|first|morning|| +36679|AAAAAAAAIEPIAAAA|36679|10|11|19|AM|first|morning|| +36680|AAAAAAAAJEPIAAAA|36680|10|11|20|AM|first|morning|| +36681|AAAAAAAAKEPIAAAA|36681|10|11|21|AM|first|morning|| +36682|AAAAAAAALEPIAAAA|36682|10|11|22|AM|first|morning|| +36683|AAAAAAAAMEPIAAAA|36683|10|11|23|AM|first|morning|| +36684|AAAAAAAANEPIAAAA|36684|10|11|24|AM|first|morning|| +36685|AAAAAAAAOEPIAAAA|36685|10|11|25|AM|first|morning|| +36686|AAAAAAAAPEPIAAAA|36686|10|11|26|AM|first|morning|| +36687|AAAAAAAAAFPIAAAA|36687|10|11|27|AM|first|morning|| +36688|AAAAAAAABFPIAAAA|36688|10|11|28|AM|first|morning|| +36689|AAAAAAAACFPIAAAA|36689|10|11|29|AM|first|morning|| +36690|AAAAAAAADFPIAAAA|36690|10|11|30|AM|first|morning|| +36691|AAAAAAAAEFPIAAAA|36691|10|11|31|AM|first|morning|| +36692|AAAAAAAAFFPIAAAA|36692|10|11|32|AM|first|morning|| +36693|AAAAAAAAGFPIAAAA|36693|10|11|33|AM|first|morning|| +36694|AAAAAAAAHFPIAAAA|36694|10|11|34|AM|first|morning|| +36695|AAAAAAAAIFPIAAAA|36695|10|11|35|AM|first|morning|| +36696|AAAAAAAAJFPIAAAA|36696|10|11|36|AM|first|morning|| +36697|AAAAAAAAKFPIAAAA|36697|10|11|37|AM|first|morning|| +36698|AAAAAAAALFPIAAAA|36698|10|11|38|AM|first|morning|| +36699|AAAAAAAAMFPIAAAA|36699|10|11|39|AM|first|morning|| +36700|AAAAAAAANFPIAAAA|36700|10|11|40|AM|first|morning|| +36701|AAAAAAAAOFPIAAAA|36701|10|11|41|AM|first|morning|| +36702|AAAAAAAAPFPIAAAA|36702|10|11|42|AM|first|morning|| +36703|AAAAAAAAAGPIAAAA|36703|10|11|43|AM|first|morning|| +36704|AAAAAAAABGPIAAAA|36704|10|11|44|AM|first|morning|| +36705|AAAAAAAACGPIAAAA|36705|10|11|45|AM|first|morning|| +36706|AAAAAAAADGPIAAAA|36706|10|11|46|AM|first|morning|| +36707|AAAAAAAAEGPIAAAA|36707|10|11|47|AM|first|morning|| +36708|AAAAAAAAFGPIAAAA|36708|10|11|48|AM|first|morning|| +36709|AAAAAAAAGGPIAAAA|36709|10|11|49|AM|first|morning|| +36710|AAAAAAAAHGPIAAAA|36710|10|11|50|AM|first|morning|| +36711|AAAAAAAAIGPIAAAA|36711|10|11|51|AM|first|morning|| +36712|AAAAAAAAJGPIAAAA|36712|10|11|52|AM|first|morning|| +36713|AAAAAAAAKGPIAAAA|36713|10|11|53|AM|first|morning|| +36714|AAAAAAAALGPIAAAA|36714|10|11|54|AM|first|morning|| +36715|AAAAAAAAMGPIAAAA|36715|10|11|55|AM|first|morning|| +36716|AAAAAAAANGPIAAAA|36716|10|11|56|AM|first|morning|| +36717|AAAAAAAAOGPIAAAA|36717|10|11|57|AM|first|morning|| +36718|AAAAAAAAPGPIAAAA|36718|10|11|58|AM|first|morning|| +36719|AAAAAAAAAHPIAAAA|36719|10|11|59|AM|first|morning|| +36720|AAAAAAAABHPIAAAA|36720|10|12|0|AM|first|morning|| +36721|AAAAAAAACHPIAAAA|36721|10|12|1|AM|first|morning|| +36722|AAAAAAAADHPIAAAA|36722|10|12|2|AM|first|morning|| +36723|AAAAAAAAEHPIAAAA|36723|10|12|3|AM|first|morning|| +36724|AAAAAAAAFHPIAAAA|36724|10|12|4|AM|first|morning|| +36725|AAAAAAAAGHPIAAAA|36725|10|12|5|AM|first|morning|| +36726|AAAAAAAAHHPIAAAA|36726|10|12|6|AM|first|morning|| +36727|AAAAAAAAIHPIAAAA|36727|10|12|7|AM|first|morning|| +36728|AAAAAAAAJHPIAAAA|36728|10|12|8|AM|first|morning|| +36729|AAAAAAAAKHPIAAAA|36729|10|12|9|AM|first|morning|| +36730|AAAAAAAALHPIAAAA|36730|10|12|10|AM|first|morning|| +36731|AAAAAAAAMHPIAAAA|36731|10|12|11|AM|first|morning|| +36732|AAAAAAAANHPIAAAA|36732|10|12|12|AM|first|morning|| +36733|AAAAAAAAOHPIAAAA|36733|10|12|13|AM|first|morning|| +36734|AAAAAAAAPHPIAAAA|36734|10|12|14|AM|first|morning|| +36735|AAAAAAAAAIPIAAAA|36735|10|12|15|AM|first|morning|| +36736|AAAAAAAABIPIAAAA|36736|10|12|16|AM|first|morning|| +36737|AAAAAAAACIPIAAAA|36737|10|12|17|AM|first|morning|| +36738|AAAAAAAADIPIAAAA|36738|10|12|18|AM|first|morning|| +36739|AAAAAAAAEIPIAAAA|36739|10|12|19|AM|first|morning|| +36740|AAAAAAAAFIPIAAAA|36740|10|12|20|AM|first|morning|| +36741|AAAAAAAAGIPIAAAA|36741|10|12|21|AM|first|morning|| +36742|AAAAAAAAHIPIAAAA|36742|10|12|22|AM|first|morning|| +36743|AAAAAAAAIIPIAAAA|36743|10|12|23|AM|first|morning|| +36744|AAAAAAAAJIPIAAAA|36744|10|12|24|AM|first|morning|| +36745|AAAAAAAAKIPIAAAA|36745|10|12|25|AM|first|morning|| +36746|AAAAAAAALIPIAAAA|36746|10|12|26|AM|first|morning|| +36747|AAAAAAAAMIPIAAAA|36747|10|12|27|AM|first|morning|| +36748|AAAAAAAANIPIAAAA|36748|10|12|28|AM|first|morning|| +36749|AAAAAAAAOIPIAAAA|36749|10|12|29|AM|first|morning|| +36750|AAAAAAAAPIPIAAAA|36750|10|12|30|AM|first|morning|| +36751|AAAAAAAAAJPIAAAA|36751|10|12|31|AM|first|morning|| +36752|AAAAAAAABJPIAAAA|36752|10|12|32|AM|first|morning|| +36753|AAAAAAAACJPIAAAA|36753|10|12|33|AM|first|morning|| +36754|AAAAAAAADJPIAAAA|36754|10|12|34|AM|first|morning|| +36755|AAAAAAAAEJPIAAAA|36755|10|12|35|AM|first|morning|| +36756|AAAAAAAAFJPIAAAA|36756|10|12|36|AM|first|morning|| +36757|AAAAAAAAGJPIAAAA|36757|10|12|37|AM|first|morning|| +36758|AAAAAAAAHJPIAAAA|36758|10|12|38|AM|first|morning|| +36759|AAAAAAAAIJPIAAAA|36759|10|12|39|AM|first|morning|| +36760|AAAAAAAAJJPIAAAA|36760|10|12|40|AM|first|morning|| +36761|AAAAAAAAKJPIAAAA|36761|10|12|41|AM|first|morning|| +36762|AAAAAAAALJPIAAAA|36762|10|12|42|AM|first|morning|| +36763|AAAAAAAAMJPIAAAA|36763|10|12|43|AM|first|morning|| +36764|AAAAAAAANJPIAAAA|36764|10|12|44|AM|first|morning|| +36765|AAAAAAAAOJPIAAAA|36765|10|12|45|AM|first|morning|| +36766|AAAAAAAAPJPIAAAA|36766|10|12|46|AM|first|morning|| +36767|AAAAAAAAAKPIAAAA|36767|10|12|47|AM|first|morning|| +36768|AAAAAAAABKPIAAAA|36768|10|12|48|AM|first|morning|| +36769|AAAAAAAACKPIAAAA|36769|10|12|49|AM|first|morning|| +36770|AAAAAAAADKPIAAAA|36770|10|12|50|AM|first|morning|| +36771|AAAAAAAAEKPIAAAA|36771|10|12|51|AM|first|morning|| +36772|AAAAAAAAFKPIAAAA|36772|10|12|52|AM|first|morning|| +36773|AAAAAAAAGKPIAAAA|36773|10|12|53|AM|first|morning|| +36774|AAAAAAAAHKPIAAAA|36774|10|12|54|AM|first|morning|| +36775|AAAAAAAAIKPIAAAA|36775|10|12|55|AM|first|morning|| +36776|AAAAAAAAJKPIAAAA|36776|10|12|56|AM|first|morning|| +36777|AAAAAAAAKKPIAAAA|36777|10|12|57|AM|first|morning|| +36778|AAAAAAAALKPIAAAA|36778|10|12|58|AM|first|morning|| +36779|AAAAAAAAMKPIAAAA|36779|10|12|59|AM|first|morning|| +36780|AAAAAAAANKPIAAAA|36780|10|13|0|AM|first|morning|| +36781|AAAAAAAAOKPIAAAA|36781|10|13|1|AM|first|morning|| +36782|AAAAAAAAPKPIAAAA|36782|10|13|2|AM|first|morning|| +36783|AAAAAAAAALPIAAAA|36783|10|13|3|AM|first|morning|| +36784|AAAAAAAABLPIAAAA|36784|10|13|4|AM|first|morning|| +36785|AAAAAAAACLPIAAAA|36785|10|13|5|AM|first|morning|| +36786|AAAAAAAADLPIAAAA|36786|10|13|6|AM|first|morning|| +36787|AAAAAAAAELPIAAAA|36787|10|13|7|AM|first|morning|| +36788|AAAAAAAAFLPIAAAA|36788|10|13|8|AM|first|morning|| +36789|AAAAAAAAGLPIAAAA|36789|10|13|9|AM|first|morning|| +36790|AAAAAAAAHLPIAAAA|36790|10|13|10|AM|first|morning|| +36791|AAAAAAAAILPIAAAA|36791|10|13|11|AM|first|morning|| +36792|AAAAAAAAJLPIAAAA|36792|10|13|12|AM|first|morning|| +36793|AAAAAAAAKLPIAAAA|36793|10|13|13|AM|first|morning|| +36794|AAAAAAAALLPIAAAA|36794|10|13|14|AM|first|morning|| +36795|AAAAAAAAMLPIAAAA|36795|10|13|15|AM|first|morning|| +36796|AAAAAAAANLPIAAAA|36796|10|13|16|AM|first|morning|| +36797|AAAAAAAAOLPIAAAA|36797|10|13|17|AM|first|morning|| +36798|AAAAAAAAPLPIAAAA|36798|10|13|18|AM|first|morning|| +36799|AAAAAAAAAMPIAAAA|36799|10|13|19|AM|first|morning|| +36800|AAAAAAAABMPIAAAA|36800|10|13|20|AM|first|morning|| +36801|AAAAAAAACMPIAAAA|36801|10|13|21|AM|first|morning|| +36802|AAAAAAAADMPIAAAA|36802|10|13|22|AM|first|morning|| +36803|AAAAAAAAEMPIAAAA|36803|10|13|23|AM|first|morning|| +36804|AAAAAAAAFMPIAAAA|36804|10|13|24|AM|first|morning|| +36805|AAAAAAAAGMPIAAAA|36805|10|13|25|AM|first|morning|| +36806|AAAAAAAAHMPIAAAA|36806|10|13|26|AM|first|morning|| +36807|AAAAAAAAIMPIAAAA|36807|10|13|27|AM|first|morning|| +36808|AAAAAAAAJMPIAAAA|36808|10|13|28|AM|first|morning|| +36809|AAAAAAAAKMPIAAAA|36809|10|13|29|AM|first|morning|| +36810|AAAAAAAALMPIAAAA|36810|10|13|30|AM|first|morning|| +36811|AAAAAAAAMMPIAAAA|36811|10|13|31|AM|first|morning|| +36812|AAAAAAAANMPIAAAA|36812|10|13|32|AM|first|morning|| +36813|AAAAAAAAOMPIAAAA|36813|10|13|33|AM|first|morning|| +36814|AAAAAAAAPMPIAAAA|36814|10|13|34|AM|first|morning|| +36815|AAAAAAAAANPIAAAA|36815|10|13|35|AM|first|morning|| +36816|AAAAAAAABNPIAAAA|36816|10|13|36|AM|first|morning|| +36817|AAAAAAAACNPIAAAA|36817|10|13|37|AM|first|morning|| +36818|AAAAAAAADNPIAAAA|36818|10|13|38|AM|first|morning|| +36819|AAAAAAAAENPIAAAA|36819|10|13|39|AM|first|morning|| +36820|AAAAAAAAFNPIAAAA|36820|10|13|40|AM|first|morning|| +36821|AAAAAAAAGNPIAAAA|36821|10|13|41|AM|first|morning|| +36822|AAAAAAAAHNPIAAAA|36822|10|13|42|AM|first|morning|| +36823|AAAAAAAAINPIAAAA|36823|10|13|43|AM|first|morning|| +36824|AAAAAAAAJNPIAAAA|36824|10|13|44|AM|first|morning|| +36825|AAAAAAAAKNPIAAAA|36825|10|13|45|AM|first|morning|| +36826|AAAAAAAALNPIAAAA|36826|10|13|46|AM|first|morning|| +36827|AAAAAAAAMNPIAAAA|36827|10|13|47|AM|first|morning|| +36828|AAAAAAAANNPIAAAA|36828|10|13|48|AM|first|morning|| +36829|AAAAAAAAONPIAAAA|36829|10|13|49|AM|first|morning|| +36830|AAAAAAAAPNPIAAAA|36830|10|13|50|AM|first|morning|| +36831|AAAAAAAAAOPIAAAA|36831|10|13|51|AM|first|morning|| +36832|AAAAAAAABOPIAAAA|36832|10|13|52|AM|first|morning|| +36833|AAAAAAAACOPIAAAA|36833|10|13|53|AM|first|morning|| +36834|AAAAAAAADOPIAAAA|36834|10|13|54|AM|first|morning|| +36835|AAAAAAAAEOPIAAAA|36835|10|13|55|AM|first|morning|| +36836|AAAAAAAAFOPIAAAA|36836|10|13|56|AM|first|morning|| +36837|AAAAAAAAGOPIAAAA|36837|10|13|57|AM|first|morning|| +36838|AAAAAAAAHOPIAAAA|36838|10|13|58|AM|first|morning|| +36839|AAAAAAAAIOPIAAAA|36839|10|13|59|AM|first|morning|| +36840|AAAAAAAAJOPIAAAA|36840|10|14|0|AM|first|morning|| +36841|AAAAAAAAKOPIAAAA|36841|10|14|1|AM|first|morning|| +36842|AAAAAAAALOPIAAAA|36842|10|14|2|AM|first|morning|| +36843|AAAAAAAAMOPIAAAA|36843|10|14|3|AM|first|morning|| +36844|AAAAAAAANOPIAAAA|36844|10|14|4|AM|first|morning|| +36845|AAAAAAAAOOPIAAAA|36845|10|14|5|AM|first|morning|| +36846|AAAAAAAAPOPIAAAA|36846|10|14|6|AM|first|morning|| +36847|AAAAAAAAAPPIAAAA|36847|10|14|7|AM|first|morning|| +36848|AAAAAAAABPPIAAAA|36848|10|14|8|AM|first|morning|| +36849|AAAAAAAACPPIAAAA|36849|10|14|9|AM|first|morning|| +36850|AAAAAAAADPPIAAAA|36850|10|14|10|AM|first|morning|| +36851|AAAAAAAAEPPIAAAA|36851|10|14|11|AM|first|morning|| +36852|AAAAAAAAFPPIAAAA|36852|10|14|12|AM|first|morning|| +36853|AAAAAAAAGPPIAAAA|36853|10|14|13|AM|first|morning|| +36854|AAAAAAAAHPPIAAAA|36854|10|14|14|AM|first|morning|| +36855|AAAAAAAAIPPIAAAA|36855|10|14|15|AM|first|morning|| +36856|AAAAAAAAJPPIAAAA|36856|10|14|16|AM|first|morning|| +36857|AAAAAAAAKPPIAAAA|36857|10|14|17|AM|first|morning|| +36858|AAAAAAAALPPIAAAA|36858|10|14|18|AM|first|morning|| +36859|AAAAAAAAMPPIAAAA|36859|10|14|19|AM|first|morning|| +36860|AAAAAAAANPPIAAAA|36860|10|14|20|AM|first|morning|| +36861|AAAAAAAAOPPIAAAA|36861|10|14|21|AM|first|morning|| +36862|AAAAAAAAPPPIAAAA|36862|10|14|22|AM|first|morning|| +36863|AAAAAAAAAAAJAAAA|36863|10|14|23|AM|first|morning|| +36864|AAAAAAAABAAJAAAA|36864|10|14|24|AM|first|morning|| +36865|AAAAAAAACAAJAAAA|36865|10|14|25|AM|first|morning|| +36866|AAAAAAAADAAJAAAA|36866|10|14|26|AM|first|morning|| +36867|AAAAAAAAEAAJAAAA|36867|10|14|27|AM|first|morning|| +36868|AAAAAAAAFAAJAAAA|36868|10|14|28|AM|first|morning|| +36869|AAAAAAAAGAAJAAAA|36869|10|14|29|AM|first|morning|| +36870|AAAAAAAAHAAJAAAA|36870|10|14|30|AM|first|morning|| +36871|AAAAAAAAIAAJAAAA|36871|10|14|31|AM|first|morning|| +36872|AAAAAAAAJAAJAAAA|36872|10|14|32|AM|first|morning|| +36873|AAAAAAAAKAAJAAAA|36873|10|14|33|AM|first|morning|| +36874|AAAAAAAALAAJAAAA|36874|10|14|34|AM|first|morning|| +36875|AAAAAAAAMAAJAAAA|36875|10|14|35|AM|first|morning|| +36876|AAAAAAAANAAJAAAA|36876|10|14|36|AM|first|morning|| +36877|AAAAAAAAOAAJAAAA|36877|10|14|37|AM|first|morning|| +36878|AAAAAAAAPAAJAAAA|36878|10|14|38|AM|first|morning|| +36879|AAAAAAAAABAJAAAA|36879|10|14|39|AM|first|morning|| +36880|AAAAAAAABBAJAAAA|36880|10|14|40|AM|first|morning|| +36881|AAAAAAAACBAJAAAA|36881|10|14|41|AM|first|morning|| +36882|AAAAAAAADBAJAAAA|36882|10|14|42|AM|first|morning|| +36883|AAAAAAAAEBAJAAAA|36883|10|14|43|AM|first|morning|| +36884|AAAAAAAAFBAJAAAA|36884|10|14|44|AM|first|morning|| +36885|AAAAAAAAGBAJAAAA|36885|10|14|45|AM|first|morning|| +36886|AAAAAAAAHBAJAAAA|36886|10|14|46|AM|first|morning|| +36887|AAAAAAAAIBAJAAAA|36887|10|14|47|AM|first|morning|| +36888|AAAAAAAAJBAJAAAA|36888|10|14|48|AM|first|morning|| +36889|AAAAAAAAKBAJAAAA|36889|10|14|49|AM|first|morning|| +36890|AAAAAAAALBAJAAAA|36890|10|14|50|AM|first|morning|| +36891|AAAAAAAAMBAJAAAA|36891|10|14|51|AM|first|morning|| +36892|AAAAAAAANBAJAAAA|36892|10|14|52|AM|first|morning|| +36893|AAAAAAAAOBAJAAAA|36893|10|14|53|AM|first|morning|| +36894|AAAAAAAAPBAJAAAA|36894|10|14|54|AM|first|morning|| +36895|AAAAAAAAACAJAAAA|36895|10|14|55|AM|first|morning|| +36896|AAAAAAAABCAJAAAA|36896|10|14|56|AM|first|morning|| +36897|AAAAAAAACCAJAAAA|36897|10|14|57|AM|first|morning|| +36898|AAAAAAAADCAJAAAA|36898|10|14|58|AM|first|morning|| +36899|AAAAAAAAECAJAAAA|36899|10|14|59|AM|first|morning|| +36900|AAAAAAAAFCAJAAAA|36900|10|15|0|AM|first|morning|| +36901|AAAAAAAAGCAJAAAA|36901|10|15|1|AM|first|morning|| +36902|AAAAAAAAHCAJAAAA|36902|10|15|2|AM|first|morning|| +36903|AAAAAAAAICAJAAAA|36903|10|15|3|AM|first|morning|| +36904|AAAAAAAAJCAJAAAA|36904|10|15|4|AM|first|morning|| +36905|AAAAAAAAKCAJAAAA|36905|10|15|5|AM|first|morning|| +36906|AAAAAAAALCAJAAAA|36906|10|15|6|AM|first|morning|| +36907|AAAAAAAAMCAJAAAA|36907|10|15|7|AM|first|morning|| +36908|AAAAAAAANCAJAAAA|36908|10|15|8|AM|first|morning|| +36909|AAAAAAAAOCAJAAAA|36909|10|15|9|AM|first|morning|| +36910|AAAAAAAAPCAJAAAA|36910|10|15|10|AM|first|morning|| +36911|AAAAAAAAADAJAAAA|36911|10|15|11|AM|first|morning|| +36912|AAAAAAAABDAJAAAA|36912|10|15|12|AM|first|morning|| +36913|AAAAAAAACDAJAAAA|36913|10|15|13|AM|first|morning|| +36914|AAAAAAAADDAJAAAA|36914|10|15|14|AM|first|morning|| +36915|AAAAAAAAEDAJAAAA|36915|10|15|15|AM|first|morning|| +36916|AAAAAAAAFDAJAAAA|36916|10|15|16|AM|first|morning|| +36917|AAAAAAAAGDAJAAAA|36917|10|15|17|AM|first|morning|| +36918|AAAAAAAAHDAJAAAA|36918|10|15|18|AM|first|morning|| +36919|AAAAAAAAIDAJAAAA|36919|10|15|19|AM|first|morning|| +36920|AAAAAAAAJDAJAAAA|36920|10|15|20|AM|first|morning|| +36921|AAAAAAAAKDAJAAAA|36921|10|15|21|AM|first|morning|| +36922|AAAAAAAALDAJAAAA|36922|10|15|22|AM|first|morning|| +36923|AAAAAAAAMDAJAAAA|36923|10|15|23|AM|first|morning|| +36924|AAAAAAAANDAJAAAA|36924|10|15|24|AM|first|morning|| +36925|AAAAAAAAODAJAAAA|36925|10|15|25|AM|first|morning|| +36926|AAAAAAAAPDAJAAAA|36926|10|15|26|AM|first|morning|| +36927|AAAAAAAAAEAJAAAA|36927|10|15|27|AM|first|morning|| +36928|AAAAAAAABEAJAAAA|36928|10|15|28|AM|first|morning|| +36929|AAAAAAAACEAJAAAA|36929|10|15|29|AM|first|morning|| +36930|AAAAAAAADEAJAAAA|36930|10|15|30|AM|first|morning|| +36931|AAAAAAAAEEAJAAAA|36931|10|15|31|AM|first|morning|| +36932|AAAAAAAAFEAJAAAA|36932|10|15|32|AM|first|morning|| +36933|AAAAAAAAGEAJAAAA|36933|10|15|33|AM|first|morning|| +36934|AAAAAAAAHEAJAAAA|36934|10|15|34|AM|first|morning|| +36935|AAAAAAAAIEAJAAAA|36935|10|15|35|AM|first|morning|| +36936|AAAAAAAAJEAJAAAA|36936|10|15|36|AM|first|morning|| +36937|AAAAAAAAKEAJAAAA|36937|10|15|37|AM|first|morning|| +36938|AAAAAAAALEAJAAAA|36938|10|15|38|AM|first|morning|| +36939|AAAAAAAAMEAJAAAA|36939|10|15|39|AM|first|morning|| +36940|AAAAAAAANEAJAAAA|36940|10|15|40|AM|first|morning|| +36941|AAAAAAAAOEAJAAAA|36941|10|15|41|AM|first|morning|| +36942|AAAAAAAAPEAJAAAA|36942|10|15|42|AM|first|morning|| +36943|AAAAAAAAAFAJAAAA|36943|10|15|43|AM|first|morning|| +36944|AAAAAAAABFAJAAAA|36944|10|15|44|AM|first|morning|| +36945|AAAAAAAACFAJAAAA|36945|10|15|45|AM|first|morning|| +36946|AAAAAAAADFAJAAAA|36946|10|15|46|AM|first|morning|| +36947|AAAAAAAAEFAJAAAA|36947|10|15|47|AM|first|morning|| +36948|AAAAAAAAFFAJAAAA|36948|10|15|48|AM|first|morning|| +36949|AAAAAAAAGFAJAAAA|36949|10|15|49|AM|first|morning|| +36950|AAAAAAAAHFAJAAAA|36950|10|15|50|AM|first|morning|| +36951|AAAAAAAAIFAJAAAA|36951|10|15|51|AM|first|morning|| +36952|AAAAAAAAJFAJAAAA|36952|10|15|52|AM|first|morning|| +36953|AAAAAAAAKFAJAAAA|36953|10|15|53|AM|first|morning|| +36954|AAAAAAAALFAJAAAA|36954|10|15|54|AM|first|morning|| +36955|AAAAAAAAMFAJAAAA|36955|10|15|55|AM|first|morning|| +36956|AAAAAAAANFAJAAAA|36956|10|15|56|AM|first|morning|| +36957|AAAAAAAAOFAJAAAA|36957|10|15|57|AM|first|morning|| +36958|AAAAAAAAPFAJAAAA|36958|10|15|58|AM|first|morning|| +36959|AAAAAAAAAGAJAAAA|36959|10|15|59|AM|first|morning|| +36960|AAAAAAAABGAJAAAA|36960|10|16|0|AM|first|morning|| +36961|AAAAAAAACGAJAAAA|36961|10|16|1|AM|first|morning|| +36962|AAAAAAAADGAJAAAA|36962|10|16|2|AM|first|morning|| +36963|AAAAAAAAEGAJAAAA|36963|10|16|3|AM|first|morning|| +36964|AAAAAAAAFGAJAAAA|36964|10|16|4|AM|first|morning|| +36965|AAAAAAAAGGAJAAAA|36965|10|16|5|AM|first|morning|| +36966|AAAAAAAAHGAJAAAA|36966|10|16|6|AM|first|morning|| +36967|AAAAAAAAIGAJAAAA|36967|10|16|7|AM|first|morning|| +36968|AAAAAAAAJGAJAAAA|36968|10|16|8|AM|first|morning|| +36969|AAAAAAAAKGAJAAAA|36969|10|16|9|AM|first|morning|| +36970|AAAAAAAALGAJAAAA|36970|10|16|10|AM|first|morning|| +36971|AAAAAAAAMGAJAAAA|36971|10|16|11|AM|first|morning|| +36972|AAAAAAAANGAJAAAA|36972|10|16|12|AM|first|morning|| +36973|AAAAAAAAOGAJAAAA|36973|10|16|13|AM|first|morning|| +36974|AAAAAAAAPGAJAAAA|36974|10|16|14|AM|first|morning|| +36975|AAAAAAAAAHAJAAAA|36975|10|16|15|AM|first|morning|| +36976|AAAAAAAABHAJAAAA|36976|10|16|16|AM|first|morning|| +36977|AAAAAAAACHAJAAAA|36977|10|16|17|AM|first|morning|| +36978|AAAAAAAADHAJAAAA|36978|10|16|18|AM|first|morning|| +36979|AAAAAAAAEHAJAAAA|36979|10|16|19|AM|first|morning|| +36980|AAAAAAAAFHAJAAAA|36980|10|16|20|AM|first|morning|| +36981|AAAAAAAAGHAJAAAA|36981|10|16|21|AM|first|morning|| +36982|AAAAAAAAHHAJAAAA|36982|10|16|22|AM|first|morning|| +36983|AAAAAAAAIHAJAAAA|36983|10|16|23|AM|first|morning|| +36984|AAAAAAAAJHAJAAAA|36984|10|16|24|AM|first|morning|| +36985|AAAAAAAAKHAJAAAA|36985|10|16|25|AM|first|morning|| +36986|AAAAAAAALHAJAAAA|36986|10|16|26|AM|first|morning|| +36987|AAAAAAAAMHAJAAAA|36987|10|16|27|AM|first|morning|| +36988|AAAAAAAANHAJAAAA|36988|10|16|28|AM|first|morning|| +36989|AAAAAAAAOHAJAAAA|36989|10|16|29|AM|first|morning|| +36990|AAAAAAAAPHAJAAAA|36990|10|16|30|AM|first|morning|| +36991|AAAAAAAAAIAJAAAA|36991|10|16|31|AM|first|morning|| +36992|AAAAAAAABIAJAAAA|36992|10|16|32|AM|first|morning|| +36993|AAAAAAAACIAJAAAA|36993|10|16|33|AM|first|morning|| +36994|AAAAAAAADIAJAAAA|36994|10|16|34|AM|first|morning|| +36995|AAAAAAAAEIAJAAAA|36995|10|16|35|AM|first|morning|| +36996|AAAAAAAAFIAJAAAA|36996|10|16|36|AM|first|morning|| +36997|AAAAAAAAGIAJAAAA|36997|10|16|37|AM|first|morning|| +36998|AAAAAAAAHIAJAAAA|36998|10|16|38|AM|first|morning|| +36999|AAAAAAAAIIAJAAAA|36999|10|16|39|AM|first|morning|| +37000|AAAAAAAAJIAJAAAA|37000|10|16|40|AM|first|morning|| +37001|AAAAAAAAKIAJAAAA|37001|10|16|41|AM|first|morning|| +37002|AAAAAAAALIAJAAAA|37002|10|16|42|AM|first|morning|| +37003|AAAAAAAAMIAJAAAA|37003|10|16|43|AM|first|morning|| +37004|AAAAAAAANIAJAAAA|37004|10|16|44|AM|first|morning|| +37005|AAAAAAAAOIAJAAAA|37005|10|16|45|AM|first|morning|| +37006|AAAAAAAAPIAJAAAA|37006|10|16|46|AM|first|morning|| +37007|AAAAAAAAAJAJAAAA|37007|10|16|47|AM|first|morning|| +37008|AAAAAAAABJAJAAAA|37008|10|16|48|AM|first|morning|| +37009|AAAAAAAACJAJAAAA|37009|10|16|49|AM|first|morning|| +37010|AAAAAAAADJAJAAAA|37010|10|16|50|AM|first|morning|| +37011|AAAAAAAAEJAJAAAA|37011|10|16|51|AM|first|morning|| +37012|AAAAAAAAFJAJAAAA|37012|10|16|52|AM|first|morning|| +37013|AAAAAAAAGJAJAAAA|37013|10|16|53|AM|first|morning|| +37014|AAAAAAAAHJAJAAAA|37014|10|16|54|AM|first|morning|| +37015|AAAAAAAAIJAJAAAA|37015|10|16|55|AM|first|morning|| +37016|AAAAAAAAJJAJAAAA|37016|10|16|56|AM|first|morning|| +37017|AAAAAAAAKJAJAAAA|37017|10|16|57|AM|first|morning|| +37018|AAAAAAAALJAJAAAA|37018|10|16|58|AM|first|morning|| +37019|AAAAAAAAMJAJAAAA|37019|10|16|59|AM|first|morning|| +37020|AAAAAAAANJAJAAAA|37020|10|17|0|AM|first|morning|| +37021|AAAAAAAAOJAJAAAA|37021|10|17|1|AM|first|morning|| +37022|AAAAAAAAPJAJAAAA|37022|10|17|2|AM|first|morning|| +37023|AAAAAAAAAKAJAAAA|37023|10|17|3|AM|first|morning|| +37024|AAAAAAAABKAJAAAA|37024|10|17|4|AM|first|morning|| +37025|AAAAAAAACKAJAAAA|37025|10|17|5|AM|first|morning|| +37026|AAAAAAAADKAJAAAA|37026|10|17|6|AM|first|morning|| +37027|AAAAAAAAEKAJAAAA|37027|10|17|7|AM|first|morning|| +37028|AAAAAAAAFKAJAAAA|37028|10|17|8|AM|first|morning|| +37029|AAAAAAAAGKAJAAAA|37029|10|17|9|AM|first|morning|| +37030|AAAAAAAAHKAJAAAA|37030|10|17|10|AM|first|morning|| +37031|AAAAAAAAIKAJAAAA|37031|10|17|11|AM|first|morning|| +37032|AAAAAAAAJKAJAAAA|37032|10|17|12|AM|first|morning|| +37033|AAAAAAAAKKAJAAAA|37033|10|17|13|AM|first|morning|| +37034|AAAAAAAALKAJAAAA|37034|10|17|14|AM|first|morning|| +37035|AAAAAAAAMKAJAAAA|37035|10|17|15|AM|first|morning|| +37036|AAAAAAAANKAJAAAA|37036|10|17|16|AM|first|morning|| +37037|AAAAAAAAOKAJAAAA|37037|10|17|17|AM|first|morning|| +37038|AAAAAAAAPKAJAAAA|37038|10|17|18|AM|first|morning|| +37039|AAAAAAAAALAJAAAA|37039|10|17|19|AM|first|morning|| +37040|AAAAAAAABLAJAAAA|37040|10|17|20|AM|first|morning|| +37041|AAAAAAAACLAJAAAA|37041|10|17|21|AM|first|morning|| +37042|AAAAAAAADLAJAAAA|37042|10|17|22|AM|first|morning|| +37043|AAAAAAAAELAJAAAA|37043|10|17|23|AM|first|morning|| +37044|AAAAAAAAFLAJAAAA|37044|10|17|24|AM|first|morning|| +37045|AAAAAAAAGLAJAAAA|37045|10|17|25|AM|first|morning|| +37046|AAAAAAAAHLAJAAAA|37046|10|17|26|AM|first|morning|| +37047|AAAAAAAAILAJAAAA|37047|10|17|27|AM|first|morning|| +37048|AAAAAAAAJLAJAAAA|37048|10|17|28|AM|first|morning|| +37049|AAAAAAAAKLAJAAAA|37049|10|17|29|AM|first|morning|| +37050|AAAAAAAALLAJAAAA|37050|10|17|30|AM|first|morning|| +37051|AAAAAAAAMLAJAAAA|37051|10|17|31|AM|first|morning|| +37052|AAAAAAAANLAJAAAA|37052|10|17|32|AM|first|morning|| +37053|AAAAAAAAOLAJAAAA|37053|10|17|33|AM|first|morning|| +37054|AAAAAAAAPLAJAAAA|37054|10|17|34|AM|first|morning|| +37055|AAAAAAAAAMAJAAAA|37055|10|17|35|AM|first|morning|| +37056|AAAAAAAABMAJAAAA|37056|10|17|36|AM|first|morning|| +37057|AAAAAAAACMAJAAAA|37057|10|17|37|AM|first|morning|| +37058|AAAAAAAADMAJAAAA|37058|10|17|38|AM|first|morning|| +37059|AAAAAAAAEMAJAAAA|37059|10|17|39|AM|first|morning|| +37060|AAAAAAAAFMAJAAAA|37060|10|17|40|AM|first|morning|| +37061|AAAAAAAAGMAJAAAA|37061|10|17|41|AM|first|morning|| +37062|AAAAAAAAHMAJAAAA|37062|10|17|42|AM|first|morning|| +37063|AAAAAAAAIMAJAAAA|37063|10|17|43|AM|first|morning|| +37064|AAAAAAAAJMAJAAAA|37064|10|17|44|AM|first|morning|| +37065|AAAAAAAAKMAJAAAA|37065|10|17|45|AM|first|morning|| +37066|AAAAAAAALMAJAAAA|37066|10|17|46|AM|first|morning|| +37067|AAAAAAAAMMAJAAAA|37067|10|17|47|AM|first|morning|| +37068|AAAAAAAANMAJAAAA|37068|10|17|48|AM|first|morning|| +37069|AAAAAAAAOMAJAAAA|37069|10|17|49|AM|first|morning|| +37070|AAAAAAAAPMAJAAAA|37070|10|17|50|AM|first|morning|| +37071|AAAAAAAAANAJAAAA|37071|10|17|51|AM|first|morning|| +37072|AAAAAAAABNAJAAAA|37072|10|17|52|AM|first|morning|| +37073|AAAAAAAACNAJAAAA|37073|10|17|53|AM|first|morning|| +37074|AAAAAAAADNAJAAAA|37074|10|17|54|AM|first|morning|| +37075|AAAAAAAAENAJAAAA|37075|10|17|55|AM|first|morning|| +37076|AAAAAAAAFNAJAAAA|37076|10|17|56|AM|first|morning|| +37077|AAAAAAAAGNAJAAAA|37077|10|17|57|AM|first|morning|| +37078|AAAAAAAAHNAJAAAA|37078|10|17|58|AM|first|morning|| +37079|AAAAAAAAINAJAAAA|37079|10|17|59|AM|first|morning|| +37080|AAAAAAAAJNAJAAAA|37080|10|18|0|AM|first|morning|| +37081|AAAAAAAAKNAJAAAA|37081|10|18|1|AM|first|morning|| +37082|AAAAAAAALNAJAAAA|37082|10|18|2|AM|first|morning|| +37083|AAAAAAAAMNAJAAAA|37083|10|18|3|AM|first|morning|| +37084|AAAAAAAANNAJAAAA|37084|10|18|4|AM|first|morning|| +37085|AAAAAAAAONAJAAAA|37085|10|18|5|AM|first|morning|| +37086|AAAAAAAAPNAJAAAA|37086|10|18|6|AM|first|morning|| +37087|AAAAAAAAAOAJAAAA|37087|10|18|7|AM|first|morning|| +37088|AAAAAAAABOAJAAAA|37088|10|18|8|AM|first|morning|| +37089|AAAAAAAACOAJAAAA|37089|10|18|9|AM|first|morning|| +37090|AAAAAAAADOAJAAAA|37090|10|18|10|AM|first|morning|| +37091|AAAAAAAAEOAJAAAA|37091|10|18|11|AM|first|morning|| +37092|AAAAAAAAFOAJAAAA|37092|10|18|12|AM|first|morning|| +37093|AAAAAAAAGOAJAAAA|37093|10|18|13|AM|first|morning|| +37094|AAAAAAAAHOAJAAAA|37094|10|18|14|AM|first|morning|| +37095|AAAAAAAAIOAJAAAA|37095|10|18|15|AM|first|morning|| +37096|AAAAAAAAJOAJAAAA|37096|10|18|16|AM|first|morning|| +37097|AAAAAAAAKOAJAAAA|37097|10|18|17|AM|first|morning|| +37098|AAAAAAAALOAJAAAA|37098|10|18|18|AM|first|morning|| +37099|AAAAAAAAMOAJAAAA|37099|10|18|19|AM|first|morning|| +37100|AAAAAAAANOAJAAAA|37100|10|18|20|AM|first|morning|| +37101|AAAAAAAAOOAJAAAA|37101|10|18|21|AM|first|morning|| +37102|AAAAAAAAPOAJAAAA|37102|10|18|22|AM|first|morning|| +37103|AAAAAAAAAPAJAAAA|37103|10|18|23|AM|first|morning|| +37104|AAAAAAAABPAJAAAA|37104|10|18|24|AM|first|morning|| +37105|AAAAAAAACPAJAAAA|37105|10|18|25|AM|first|morning|| +37106|AAAAAAAADPAJAAAA|37106|10|18|26|AM|first|morning|| +37107|AAAAAAAAEPAJAAAA|37107|10|18|27|AM|first|morning|| +37108|AAAAAAAAFPAJAAAA|37108|10|18|28|AM|first|morning|| +37109|AAAAAAAAGPAJAAAA|37109|10|18|29|AM|first|morning|| +37110|AAAAAAAAHPAJAAAA|37110|10|18|30|AM|first|morning|| +37111|AAAAAAAAIPAJAAAA|37111|10|18|31|AM|first|morning|| +37112|AAAAAAAAJPAJAAAA|37112|10|18|32|AM|first|morning|| +37113|AAAAAAAAKPAJAAAA|37113|10|18|33|AM|first|morning|| +37114|AAAAAAAALPAJAAAA|37114|10|18|34|AM|first|morning|| +37115|AAAAAAAAMPAJAAAA|37115|10|18|35|AM|first|morning|| +37116|AAAAAAAANPAJAAAA|37116|10|18|36|AM|first|morning|| +37117|AAAAAAAAOPAJAAAA|37117|10|18|37|AM|first|morning|| +37118|AAAAAAAAPPAJAAAA|37118|10|18|38|AM|first|morning|| +37119|AAAAAAAAAABJAAAA|37119|10|18|39|AM|first|morning|| +37120|AAAAAAAABABJAAAA|37120|10|18|40|AM|first|morning|| +37121|AAAAAAAACABJAAAA|37121|10|18|41|AM|first|morning|| +37122|AAAAAAAADABJAAAA|37122|10|18|42|AM|first|morning|| +37123|AAAAAAAAEABJAAAA|37123|10|18|43|AM|first|morning|| +37124|AAAAAAAAFABJAAAA|37124|10|18|44|AM|first|morning|| +37125|AAAAAAAAGABJAAAA|37125|10|18|45|AM|first|morning|| +37126|AAAAAAAAHABJAAAA|37126|10|18|46|AM|first|morning|| +37127|AAAAAAAAIABJAAAA|37127|10|18|47|AM|first|morning|| +37128|AAAAAAAAJABJAAAA|37128|10|18|48|AM|first|morning|| +37129|AAAAAAAAKABJAAAA|37129|10|18|49|AM|first|morning|| +37130|AAAAAAAALABJAAAA|37130|10|18|50|AM|first|morning|| +37131|AAAAAAAAMABJAAAA|37131|10|18|51|AM|first|morning|| +37132|AAAAAAAANABJAAAA|37132|10|18|52|AM|first|morning|| +37133|AAAAAAAAOABJAAAA|37133|10|18|53|AM|first|morning|| +37134|AAAAAAAAPABJAAAA|37134|10|18|54|AM|first|morning|| +37135|AAAAAAAAABBJAAAA|37135|10|18|55|AM|first|morning|| +37136|AAAAAAAABBBJAAAA|37136|10|18|56|AM|first|morning|| +37137|AAAAAAAACBBJAAAA|37137|10|18|57|AM|first|morning|| +37138|AAAAAAAADBBJAAAA|37138|10|18|58|AM|first|morning|| +37139|AAAAAAAAEBBJAAAA|37139|10|18|59|AM|first|morning|| +37140|AAAAAAAAFBBJAAAA|37140|10|19|0|AM|first|morning|| +37141|AAAAAAAAGBBJAAAA|37141|10|19|1|AM|first|morning|| +37142|AAAAAAAAHBBJAAAA|37142|10|19|2|AM|first|morning|| +37143|AAAAAAAAIBBJAAAA|37143|10|19|3|AM|first|morning|| +37144|AAAAAAAAJBBJAAAA|37144|10|19|4|AM|first|morning|| +37145|AAAAAAAAKBBJAAAA|37145|10|19|5|AM|first|morning|| +37146|AAAAAAAALBBJAAAA|37146|10|19|6|AM|first|morning|| +37147|AAAAAAAAMBBJAAAA|37147|10|19|7|AM|first|morning|| +37148|AAAAAAAANBBJAAAA|37148|10|19|8|AM|first|morning|| +37149|AAAAAAAAOBBJAAAA|37149|10|19|9|AM|first|morning|| +37150|AAAAAAAAPBBJAAAA|37150|10|19|10|AM|first|morning|| +37151|AAAAAAAAACBJAAAA|37151|10|19|11|AM|first|morning|| +37152|AAAAAAAABCBJAAAA|37152|10|19|12|AM|first|morning|| +37153|AAAAAAAACCBJAAAA|37153|10|19|13|AM|first|morning|| +37154|AAAAAAAADCBJAAAA|37154|10|19|14|AM|first|morning|| +37155|AAAAAAAAECBJAAAA|37155|10|19|15|AM|first|morning|| +37156|AAAAAAAAFCBJAAAA|37156|10|19|16|AM|first|morning|| +37157|AAAAAAAAGCBJAAAA|37157|10|19|17|AM|first|morning|| +37158|AAAAAAAAHCBJAAAA|37158|10|19|18|AM|first|morning|| +37159|AAAAAAAAICBJAAAA|37159|10|19|19|AM|first|morning|| +37160|AAAAAAAAJCBJAAAA|37160|10|19|20|AM|first|morning|| +37161|AAAAAAAAKCBJAAAA|37161|10|19|21|AM|first|morning|| +37162|AAAAAAAALCBJAAAA|37162|10|19|22|AM|first|morning|| +37163|AAAAAAAAMCBJAAAA|37163|10|19|23|AM|first|morning|| +37164|AAAAAAAANCBJAAAA|37164|10|19|24|AM|first|morning|| +37165|AAAAAAAAOCBJAAAA|37165|10|19|25|AM|first|morning|| +37166|AAAAAAAAPCBJAAAA|37166|10|19|26|AM|first|morning|| +37167|AAAAAAAAADBJAAAA|37167|10|19|27|AM|first|morning|| +37168|AAAAAAAABDBJAAAA|37168|10|19|28|AM|first|morning|| +37169|AAAAAAAACDBJAAAA|37169|10|19|29|AM|first|morning|| +37170|AAAAAAAADDBJAAAA|37170|10|19|30|AM|first|morning|| +37171|AAAAAAAAEDBJAAAA|37171|10|19|31|AM|first|morning|| +37172|AAAAAAAAFDBJAAAA|37172|10|19|32|AM|first|morning|| +37173|AAAAAAAAGDBJAAAA|37173|10|19|33|AM|first|morning|| +37174|AAAAAAAAHDBJAAAA|37174|10|19|34|AM|first|morning|| +37175|AAAAAAAAIDBJAAAA|37175|10|19|35|AM|first|morning|| +37176|AAAAAAAAJDBJAAAA|37176|10|19|36|AM|first|morning|| +37177|AAAAAAAAKDBJAAAA|37177|10|19|37|AM|first|morning|| +37178|AAAAAAAALDBJAAAA|37178|10|19|38|AM|first|morning|| +37179|AAAAAAAAMDBJAAAA|37179|10|19|39|AM|first|morning|| +37180|AAAAAAAANDBJAAAA|37180|10|19|40|AM|first|morning|| +37181|AAAAAAAAODBJAAAA|37181|10|19|41|AM|first|morning|| +37182|AAAAAAAAPDBJAAAA|37182|10|19|42|AM|first|morning|| +37183|AAAAAAAAAEBJAAAA|37183|10|19|43|AM|first|morning|| +37184|AAAAAAAABEBJAAAA|37184|10|19|44|AM|first|morning|| +37185|AAAAAAAACEBJAAAA|37185|10|19|45|AM|first|morning|| +37186|AAAAAAAADEBJAAAA|37186|10|19|46|AM|first|morning|| +37187|AAAAAAAAEEBJAAAA|37187|10|19|47|AM|first|morning|| +37188|AAAAAAAAFEBJAAAA|37188|10|19|48|AM|first|morning|| +37189|AAAAAAAAGEBJAAAA|37189|10|19|49|AM|first|morning|| +37190|AAAAAAAAHEBJAAAA|37190|10|19|50|AM|first|morning|| +37191|AAAAAAAAIEBJAAAA|37191|10|19|51|AM|first|morning|| +37192|AAAAAAAAJEBJAAAA|37192|10|19|52|AM|first|morning|| +37193|AAAAAAAAKEBJAAAA|37193|10|19|53|AM|first|morning|| +37194|AAAAAAAALEBJAAAA|37194|10|19|54|AM|first|morning|| +37195|AAAAAAAAMEBJAAAA|37195|10|19|55|AM|first|morning|| +37196|AAAAAAAANEBJAAAA|37196|10|19|56|AM|first|morning|| +37197|AAAAAAAAOEBJAAAA|37197|10|19|57|AM|first|morning|| +37198|AAAAAAAAPEBJAAAA|37198|10|19|58|AM|first|morning|| +37199|AAAAAAAAAFBJAAAA|37199|10|19|59|AM|first|morning|| +37200|AAAAAAAABFBJAAAA|37200|10|20|0|AM|first|morning|| +37201|AAAAAAAACFBJAAAA|37201|10|20|1|AM|first|morning|| +37202|AAAAAAAADFBJAAAA|37202|10|20|2|AM|first|morning|| +37203|AAAAAAAAEFBJAAAA|37203|10|20|3|AM|first|morning|| +37204|AAAAAAAAFFBJAAAA|37204|10|20|4|AM|first|morning|| +37205|AAAAAAAAGFBJAAAA|37205|10|20|5|AM|first|morning|| +37206|AAAAAAAAHFBJAAAA|37206|10|20|6|AM|first|morning|| +37207|AAAAAAAAIFBJAAAA|37207|10|20|7|AM|first|morning|| +37208|AAAAAAAAJFBJAAAA|37208|10|20|8|AM|first|morning|| +37209|AAAAAAAAKFBJAAAA|37209|10|20|9|AM|first|morning|| +37210|AAAAAAAALFBJAAAA|37210|10|20|10|AM|first|morning|| +37211|AAAAAAAAMFBJAAAA|37211|10|20|11|AM|first|morning|| +37212|AAAAAAAANFBJAAAA|37212|10|20|12|AM|first|morning|| +37213|AAAAAAAAOFBJAAAA|37213|10|20|13|AM|first|morning|| +37214|AAAAAAAAPFBJAAAA|37214|10|20|14|AM|first|morning|| +37215|AAAAAAAAAGBJAAAA|37215|10|20|15|AM|first|morning|| +37216|AAAAAAAABGBJAAAA|37216|10|20|16|AM|first|morning|| +37217|AAAAAAAACGBJAAAA|37217|10|20|17|AM|first|morning|| +37218|AAAAAAAADGBJAAAA|37218|10|20|18|AM|first|morning|| +37219|AAAAAAAAEGBJAAAA|37219|10|20|19|AM|first|morning|| +37220|AAAAAAAAFGBJAAAA|37220|10|20|20|AM|first|morning|| +37221|AAAAAAAAGGBJAAAA|37221|10|20|21|AM|first|morning|| +37222|AAAAAAAAHGBJAAAA|37222|10|20|22|AM|first|morning|| +37223|AAAAAAAAIGBJAAAA|37223|10|20|23|AM|first|morning|| +37224|AAAAAAAAJGBJAAAA|37224|10|20|24|AM|first|morning|| +37225|AAAAAAAAKGBJAAAA|37225|10|20|25|AM|first|morning|| +37226|AAAAAAAALGBJAAAA|37226|10|20|26|AM|first|morning|| +37227|AAAAAAAAMGBJAAAA|37227|10|20|27|AM|first|morning|| +37228|AAAAAAAANGBJAAAA|37228|10|20|28|AM|first|morning|| +37229|AAAAAAAAOGBJAAAA|37229|10|20|29|AM|first|morning|| +37230|AAAAAAAAPGBJAAAA|37230|10|20|30|AM|first|morning|| +37231|AAAAAAAAAHBJAAAA|37231|10|20|31|AM|first|morning|| +37232|AAAAAAAABHBJAAAA|37232|10|20|32|AM|first|morning|| +37233|AAAAAAAACHBJAAAA|37233|10|20|33|AM|first|morning|| +37234|AAAAAAAADHBJAAAA|37234|10|20|34|AM|first|morning|| +37235|AAAAAAAAEHBJAAAA|37235|10|20|35|AM|first|morning|| +37236|AAAAAAAAFHBJAAAA|37236|10|20|36|AM|first|morning|| +37237|AAAAAAAAGHBJAAAA|37237|10|20|37|AM|first|morning|| +37238|AAAAAAAAHHBJAAAA|37238|10|20|38|AM|first|morning|| +37239|AAAAAAAAIHBJAAAA|37239|10|20|39|AM|first|morning|| +37240|AAAAAAAAJHBJAAAA|37240|10|20|40|AM|first|morning|| +37241|AAAAAAAAKHBJAAAA|37241|10|20|41|AM|first|morning|| +37242|AAAAAAAALHBJAAAA|37242|10|20|42|AM|first|morning|| +37243|AAAAAAAAMHBJAAAA|37243|10|20|43|AM|first|morning|| +37244|AAAAAAAANHBJAAAA|37244|10|20|44|AM|first|morning|| +37245|AAAAAAAAOHBJAAAA|37245|10|20|45|AM|first|morning|| +37246|AAAAAAAAPHBJAAAA|37246|10|20|46|AM|first|morning|| +37247|AAAAAAAAAIBJAAAA|37247|10|20|47|AM|first|morning|| +37248|AAAAAAAABIBJAAAA|37248|10|20|48|AM|first|morning|| +37249|AAAAAAAACIBJAAAA|37249|10|20|49|AM|first|morning|| +37250|AAAAAAAADIBJAAAA|37250|10|20|50|AM|first|morning|| +37251|AAAAAAAAEIBJAAAA|37251|10|20|51|AM|first|morning|| +37252|AAAAAAAAFIBJAAAA|37252|10|20|52|AM|first|morning|| +37253|AAAAAAAAGIBJAAAA|37253|10|20|53|AM|first|morning|| +37254|AAAAAAAAHIBJAAAA|37254|10|20|54|AM|first|morning|| +37255|AAAAAAAAIIBJAAAA|37255|10|20|55|AM|first|morning|| +37256|AAAAAAAAJIBJAAAA|37256|10|20|56|AM|first|morning|| +37257|AAAAAAAAKIBJAAAA|37257|10|20|57|AM|first|morning|| +37258|AAAAAAAALIBJAAAA|37258|10|20|58|AM|first|morning|| +37259|AAAAAAAAMIBJAAAA|37259|10|20|59|AM|first|morning|| +37260|AAAAAAAANIBJAAAA|37260|10|21|0|AM|first|morning|| +37261|AAAAAAAAOIBJAAAA|37261|10|21|1|AM|first|morning|| +37262|AAAAAAAAPIBJAAAA|37262|10|21|2|AM|first|morning|| +37263|AAAAAAAAAJBJAAAA|37263|10|21|3|AM|first|morning|| +37264|AAAAAAAABJBJAAAA|37264|10|21|4|AM|first|morning|| +37265|AAAAAAAACJBJAAAA|37265|10|21|5|AM|first|morning|| +37266|AAAAAAAADJBJAAAA|37266|10|21|6|AM|first|morning|| +37267|AAAAAAAAEJBJAAAA|37267|10|21|7|AM|first|morning|| +37268|AAAAAAAAFJBJAAAA|37268|10|21|8|AM|first|morning|| +37269|AAAAAAAAGJBJAAAA|37269|10|21|9|AM|first|morning|| +37270|AAAAAAAAHJBJAAAA|37270|10|21|10|AM|first|morning|| +37271|AAAAAAAAIJBJAAAA|37271|10|21|11|AM|first|morning|| +37272|AAAAAAAAJJBJAAAA|37272|10|21|12|AM|first|morning|| +37273|AAAAAAAAKJBJAAAA|37273|10|21|13|AM|first|morning|| +37274|AAAAAAAALJBJAAAA|37274|10|21|14|AM|first|morning|| +37275|AAAAAAAAMJBJAAAA|37275|10|21|15|AM|first|morning|| +37276|AAAAAAAANJBJAAAA|37276|10|21|16|AM|first|morning|| +37277|AAAAAAAAOJBJAAAA|37277|10|21|17|AM|first|morning|| +37278|AAAAAAAAPJBJAAAA|37278|10|21|18|AM|first|morning|| +37279|AAAAAAAAAKBJAAAA|37279|10|21|19|AM|first|morning|| +37280|AAAAAAAABKBJAAAA|37280|10|21|20|AM|first|morning|| +37281|AAAAAAAACKBJAAAA|37281|10|21|21|AM|first|morning|| +37282|AAAAAAAADKBJAAAA|37282|10|21|22|AM|first|morning|| +37283|AAAAAAAAEKBJAAAA|37283|10|21|23|AM|first|morning|| +37284|AAAAAAAAFKBJAAAA|37284|10|21|24|AM|first|morning|| +37285|AAAAAAAAGKBJAAAA|37285|10|21|25|AM|first|morning|| +37286|AAAAAAAAHKBJAAAA|37286|10|21|26|AM|first|morning|| +37287|AAAAAAAAIKBJAAAA|37287|10|21|27|AM|first|morning|| +37288|AAAAAAAAJKBJAAAA|37288|10|21|28|AM|first|morning|| +37289|AAAAAAAAKKBJAAAA|37289|10|21|29|AM|first|morning|| +37290|AAAAAAAALKBJAAAA|37290|10|21|30|AM|first|morning|| +37291|AAAAAAAAMKBJAAAA|37291|10|21|31|AM|first|morning|| +37292|AAAAAAAANKBJAAAA|37292|10|21|32|AM|first|morning|| +37293|AAAAAAAAOKBJAAAA|37293|10|21|33|AM|first|morning|| +37294|AAAAAAAAPKBJAAAA|37294|10|21|34|AM|first|morning|| +37295|AAAAAAAAALBJAAAA|37295|10|21|35|AM|first|morning|| +37296|AAAAAAAABLBJAAAA|37296|10|21|36|AM|first|morning|| +37297|AAAAAAAACLBJAAAA|37297|10|21|37|AM|first|morning|| +37298|AAAAAAAADLBJAAAA|37298|10|21|38|AM|first|morning|| +37299|AAAAAAAAELBJAAAA|37299|10|21|39|AM|first|morning|| +37300|AAAAAAAAFLBJAAAA|37300|10|21|40|AM|first|morning|| +37301|AAAAAAAAGLBJAAAA|37301|10|21|41|AM|first|morning|| +37302|AAAAAAAAHLBJAAAA|37302|10|21|42|AM|first|morning|| +37303|AAAAAAAAILBJAAAA|37303|10|21|43|AM|first|morning|| +37304|AAAAAAAAJLBJAAAA|37304|10|21|44|AM|first|morning|| +37305|AAAAAAAAKLBJAAAA|37305|10|21|45|AM|first|morning|| +37306|AAAAAAAALLBJAAAA|37306|10|21|46|AM|first|morning|| +37307|AAAAAAAAMLBJAAAA|37307|10|21|47|AM|first|morning|| +37308|AAAAAAAANLBJAAAA|37308|10|21|48|AM|first|morning|| +37309|AAAAAAAAOLBJAAAA|37309|10|21|49|AM|first|morning|| +37310|AAAAAAAAPLBJAAAA|37310|10|21|50|AM|first|morning|| +37311|AAAAAAAAAMBJAAAA|37311|10|21|51|AM|first|morning|| +37312|AAAAAAAABMBJAAAA|37312|10|21|52|AM|first|morning|| +37313|AAAAAAAACMBJAAAA|37313|10|21|53|AM|first|morning|| +37314|AAAAAAAADMBJAAAA|37314|10|21|54|AM|first|morning|| +37315|AAAAAAAAEMBJAAAA|37315|10|21|55|AM|first|morning|| +37316|AAAAAAAAFMBJAAAA|37316|10|21|56|AM|first|morning|| +37317|AAAAAAAAGMBJAAAA|37317|10|21|57|AM|first|morning|| +37318|AAAAAAAAHMBJAAAA|37318|10|21|58|AM|first|morning|| +37319|AAAAAAAAIMBJAAAA|37319|10|21|59|AM|first|morning|| +37320|AAAAAAAAJMBJAAAA|37320|10|22|0|AM|first|morning|| +37321|AAAAAAAAKMBJAAAA|37321|10|22|1|AM|first|morning|| +37322|AAAAAAAALMBJAAAA|37322|10|22|2|AM|first|morning|| +37323|AAAAAAAAMMBJAAAA|37323|10|22|3|AM|first|morning|| +37324|AAAAAAAANMBJAAAA|37324|10|22|4|AM|first|morning|| +37325|AAAAAAAAOMBJAAAA|37325|10|22|5|AM|first|morning|| +37326|AAAAAAAAPMBJAAAA|37326|10|22|6|AM|first|morning|| +37327|AAAAAAAAANBJAAAA|37327|10|22|7|AM|first|morning|| +37328|AAAAAAAABNBJAAAA|37328|10|22|8|AM|first|morning|| +37329|AAAAAAAACNBJAAAA|37329|10|22|9|AM|first|morning|| +37330|AAAAAAAADNBJAAAA|37330|10|22|10|AM|first|morning|| +37331|AAAAAAAAENBJAAAA|37331|10|22|11|AM|first|morning|| +37332|AAAAAAAAFNBJAAAA|37332|10|22|12|AM|first|morning|| +37333|AAAAAAAAGNBJAAAA|37333|10|22|13|AM|first|morning|| +37334|AAAAAAAAHNBJAAAA|37334|10|22|14|AM|first|morning|| +37335|AAAAAAAAINBJAAAA|37335|10|22|15|AM|first|morning|| +37336|AAAAAAAAJNBJAAAA|37336|10|22|16|AM|first|morning|| +37337|AAAAAAAAKNBJAAAA|37337|10|22|17|AM|first|morning|| +37338|AAAAAAAALNBJAAAA|37338|10|22|18|AM|first|morning|| +37339|AAAAAAAAMNBJAAAA|37339|10|22|19|AM|first|morning|| +37340|AAAAAAAANNBJAAAA|37340|10|22|20|AM|first|morning|| +37341|AAAAAAAAONBJAAAA|37341|10|22|21|AM|first|morning|| +37342|AAAAAAAAPNBJAAAA|37342|10|22|22|AM|first|morning|| +37343|AAAAAAAAAOBJAAAA|37343|10|22|23|AM|first|morning|| +37344|AAAAAAAABOBJAAAA|37344|10|22|24|AM|first|morning|| +37345|AAAAAAAACOBJAAAA|37345|10|22|25|AM|first|morning|| +37346|AAAAAAAADOBJAAAA|37346|10|22|26|AM|first|morning|| +37347|AAAAAAAAEOBJAAAA|37347|10|22|27|AM|first|morning|| +37348|AAAAAAAAFOBJAAAA|37348|10|22|28|AM|first|morning|| +37349|AAAAAAAAGOBJAAAA|37349|10|22|29|AM|first|morning|| +37350|AAAAAAAAHOBJAAAA|37350|10|22|30|AM|first|morning|| +37351|AAAAAAAAIOBJAAAA|37351|10|22|31|AM|first|morning|| +37352|AAAAAAAAJOBJAAAA|37352|10|22|32|AM|first|morning|| +37353|AAAAAAAAKOBJAAAA|37353|10|22|33|AM|first|morning|| +37354|AAAAAAAALOBJAAAA|37354|10|22|34|AM|first|morning|| +37355|AAAAAAAAMOBJAAAA|37355|10|22|35|AM|first|morning|| +37356|AAAAAAAANOBJAAAA|37356|10|22|36|AM|first|morning|| +37357|AAAAAAAAOOBJAAAA|37357|10|22|37|AM|first|morning|| +37358|AAAAAAAAPOBJAAAA|37358|10|22|38|AM|first|morning|| +37359|AAAAAAAAAPBJAAAA|37359|10|22|39|AM|first|morning|| +37360|AAAAAAAABPBJAAAA|37360|10|22|40|AM|first|morning|| +37361|AAAAAAAACPBJAAAA|37361|10|22|41|AM|first|morning|| +37362|AAAAAAAADPBJAAAA|37362|10|22|42|AM|first|morning|| +37363|AAAAAAAAEPBJAAAA|37363|10|22|43|AM|first|morning|| +37364|AAAAAAAAFPBJAAAA|37364|10|22|44|AM|first|morning|| +37365|AAAAAAAAGPBJAAAA|37365|10|22|45|AM|first|morning|| +37366|AAAAAAAAHPBJAAAA|37366|10|22|46|AM|first|morning|| +37367|AAAAAAAAIPBJAAAA|37367|10|22|47|AM|first|morning|| +37368|AAAAAAAAJPBJAAAA|37368|10|22|48|AM|first|morning|| +37369|AAAAAAAAKPBJAAAA|37369|10|22|49|AM|first|morning|| +37370|AAAAAAAALPBJAAAA|37370|10|22|50|AM|first|morning|| +37371|AAAAAAAAMPBJAAAA|37371|10|22|51|AM|first|morning|| +37372|AAAAAAAANPBJAAAA|37372|10|22|52|AM|first|morning|| +37373|AAAAAAAAOPBJAAAA|37373|10|22|53|AM|first|morning|| +37374|AAAAAAAAPPBJAAAA|37374|10|22|54|AM|first|morning|| +37375|AAAAAAAAAACJAAAA|37375|10|22|55|AM|first|morning|| +37376|AAAAAAAABACJAAAA|37376|10|22|56|AM|first|morning|| +37377|AAAAAAAACACJAAAA|37377|10|22|57|AM|first|morning|| +37378|AAAAAAAADACJAAAA|37378|10|22|58|AM|first|morning|| +37379|AAAAAAAAEACJAAAA|37379|10|22|59|AM|first|morning|| +37380|AAAAAAAAFACJAAAA|37380|10|23|0|AM|first|morning|| +37381|AAAAAAAAGACJAAAA|37381|10|23|1|AM|first|morning|| +37382|AAAAAAAAHACJAAAA|37382|10|23|2|AM|first|morning|| +37383|AAAAAAAAIACJAAAA|37383|10|23|3|AM|first|morning|| +37384|AAAAAAAAJACJAAAA|37384|10|23|4|AM|first|morning|| +37385|AAAAAAAAKACJAAAA|37385|10|23|5|AM|first|morning|| +37386|AAAAAAAALACJAAAA|37386|10|23|6|AM|first|morning|| +37387|AAAAAAAAMACJAAAA|37387|10|23|7|AM|first|morning|| +37388|AAAAAAAANACJAAAA|37388|10|23|8|AM|first|morning|| +37389|AAAAAAAAOACJAAAA|37389|10|23|9|AM|first|morning|| +37390|AAAAAAAAPACJAAAA|37390|10|23|10|AM|first|morning|| +37391|AAAAAAAAABCJAAAA|37391|10|23|11|AM|first|morning|| +37392|AAAAAAAABBCJAAAA|37392|10|23|12|AM|first|morning|| +37393|AAAAAAAACBCJAAAA|37393|10|23|13|AM|first|morning|| +37394|AAAAAAAADBCJAAAA|37394|10|23|14|AM|first|morning|| +37395|AAAAAAAAEBCJAAAA|37395|10|23|15|AM|first|morning|| +37396|AAAAAAAAFBCJAAAA|37396|10|23|16|AM|first|morning|| +37397|AAAAAAAAGBCJAAAA|37397|10|23|17|AM|first|morning|| +37398|AAAAAAAAHBCJAAAA|37398|10|23|18|AM|first|morning|| +37399|AAAAAAAAIBCJAAAA|37399|10|23|19|AM|first|morning|| +37400|AAAAAAAAJBCJAAAA|37400|10|23|20|AM|first|morning|| +37401|AAAAAAAAKBCJAAAA|37401|10|23|21|AM|first|morning|| +37402|AAAAAAAALBCJAAAA|37402|10|23|22|AM|first|morning|| +37403|AAAAAAAAMBCJAAAA|37403|10|23|23|AM|first|morning|| +37404|AAAAAAAANBCJAAAA|37404|10|23|24|AM|first|morning|| +37405|AAAAAAAAOBCJAAAA|37405|10|23|25|AM|first|morning|| +37406|AAAAAAAAPBCJAAAA|37406|10|23|26|AM|first|morning|| +37407|AAAAAAAAACCJAAAA|37407|10|23|27|AM|first|morning|| +37408|AAAAAAAABCCJAAAA|37408|10|23|28|AM|first|morning|| +37409|AAAAAAAACCCJAAAA|37409|10|23|29|AM|first|morning|| +37410|AAAAAAAADCCJAAAA|37410|10|23|30|AM|first|morning|| +37411|AAAAAAAAECCJAAAA|37411|10|23|31|AM|first|morning|| +37412|AAAAAAAAFCCJAAAA|37412|10|23|32|AM|first|morning|| +37413|AAAAAAAAGCCJAAAA|37413|10|23|33|AM|first|morning|| +37414|AAAAAAAAHCCJAAAA|37414|10|23|34|AM|first|morning|| +37415|AAAAAAAAICCJAAAA|37415|10|23|35|AM|first|morning|| +37416|AAAAAAAAJCCJAAAA|37416|10|23|36|AM|first|morning|| +37417|AAAAAAAAKCCJAAAA|37417|10|23|37|AM|first|morning|| +37418|AAAAAAAALCCJAAAA|37418|10|23|38|AM|first|morning|| +37419|AAAAAAAAMCCJAAAA|37419|10|23|39|AM|first|morning|| +37420|AAAAAAAANCCJAAAA|37420|10|23|40|AM|first|morning|| +37421|AAAAAAAAOCCJAAAA|37421|10|23|41|AM|first|morning|| +37422|AAAAAAAAPCCJAAAA|37422|10|23|42|AM|first|morning|| +37423|AAAAAAAAADCJAAAA|37423|10|23|43|AM|first|morning|| +37424|AAAAAAAABDCJAAAA|37424|10|23|44|AM|first|morning|| +37425|AAAAAAAACDCJAAAA|37425|10|23|45|AM|first|morning|| +37426|AAAAAAAADDCJAAAA|37426|10|23|46|AM|first|morning|| +37427|AAAAAAAAEDCJAAAA|37427|10|23|47|AM|first|morning|| +37428|AAAAAAAAFDCJAAAA|37428|10|23|48|AM|first|morning|| +37429|AAAAAAAAGDCJAAAA|37429|10|23|49|AM|first|morning|| +37430|AAAAAAAAHDCJAAAA|37430|10|23|50|AM|first|morning|| +37431|AAAAAAAAIDCJAAAA|37431|10|23|51|AM|first|morning|| +37432|AAAAAAAAJDCJAAAA|37432|10|23|52|AM|first|morning|| +37433|AAAAAAAAKDCJAAAA|37433|10|23|53|AM|first|morning|| +37434|AAAAAAAALDCJAAAA|37434|10|23|54|AM|first|morning|| +37435|AAAAAAAAMDCJAAAA|37435|10|23|55|AM|first|morning|| +37436|AAAAAAAANDCJAAAA|37436|10|23|56|AM|first|morning|| +37437|AAAAAAAAODCJAAAA|37437|10|23|57|AM|first|morning|| +37438|AAAAAAAAPDCJAAAA|37438|10|23|58|AM|first|morning|| +37439|AAAAAAAAAECJAAAA|37439|10|23|59|AM|first|morning|| +37440|AAAAAAAABECJAAAA|37440|10|24|0|AM|first|morning|| +37441|AAAAAAAACECJAAAA|37441|10|24|1|AM|first|morning|| +37442|AAAAAAAADECJAAAA|37442|10|24|2|AM|first|morning|| +37443|AAAAAAAAEECJAAAA|37443|10|24|3|AM|first|morning|| +37444|AAAAAAAAFECJAAAA|37444|10|24|4|AM|first|morning|| +37445|AAAAAAAAGECJAAAA|37445|10|24|5|AM|first|morning|| +37446|AAAAAAAAHECJAAAA|37446|10|24|6|AM|first|morning|| +37447|AAAAAAAAIECJAAAA|37447|10|24|7|AM|first|morning|| +37448|AAAAAAAAJECJAAAA|37448|10|24|8|AM|first|morning|| +37449|AAAAAAAAKECJAAAA|37449|10|24|9|AM|first|morning|| +37450|AAAAAAAALECJAAAA|37450|10|24|10|AM|first|morning|| +37451|AAAAAAAAMECJAAAA|37451|10|24|11|AM|first|morning|| +37452|AAAAAAAANECJAAAA|37452|10|24|12|AM|first|morning|| +37453|AAAAAAAAOECJAAAA|37453|10|24|13|AM|first|morning|| +37454|AAAAAAAAPECJAAAA|37454|10|24|14|AM|first|morning|| +37455|AAAAAAAAAFCJAAAA|37455|10|24|15|AM|first|morning|| +37456|AAAAAAAABFCJAAAA|37456|10|24|16|AM|first|morning|| +37457|AAAAAAAACFCJAAAA|37457|10|24|17|AM|first|morning|| +37458|AAAAAAAADFCJAAAA|37458|10|24|18|AM|first|morning|| +37459|AAAAAAAAEFCJAAAA|37459|10|24|19|AM|first|morning|| +37460|AAAAAAAAFFCJAAAA|37460|10|24|20|AM|first|morning|| +37461|AAAAAAAAGFCJAAAA|37461|10|24|21|AM|first|morning|| +37462|AAAAAAAAHFCJAAAA|37462|10|24|22|AM|first|morning|| +37463|AAAAAAAAIFCJAAAA|37463|10|24|23|AM|first|morning|| +37464|AAAAAAAAJFCJAAAA|37464|10|24|24|AM|first|morning|| +37465|AAAAAAAAKFCJAAAA|37465|10|24|25|AM|first|morning|| +37466|AAAAAAAALFCJAAAA|37466|10|24|26|AM|first|morning|| +37467|AAAAAAAAMFCJAAAA|37467|10|24|27|AM|first|morning|| +37468|AAAAAAAANFCJAAAA|37468|10|24|28|AM|first|morning|| +37469|AAAAAAAAOFCJAAAA|37469|10|24|29|AM|first|morning|| +37470|AAAAAAAAPFCJAAAA|37470|10|24|30|AM|first|morning|| +37471|AAAAAAAAAGCJAAAA|37471|10|24|31|AM|first|morning|| +37472|AAAAAAAABGCJAAAA|37472|10|24|32|AM|first|morning|| +37473|AAAAAAAACGCJAAAA|37473|10|24|33|AM|first|morning|| +37474|AAAAAAAADGCJAAAA|37474|10|24|34|AM|first|morning|| +37475|AAAAAAAAEGCJAAAA|37475|10|24|35|AM|first|morning|| +37476|AAAAAAAAFGCJAAAA|37476|10|24|36|AM|first|morning|| +37477|AAAAAAAAGGCJAAAA|37477|10|24|37|AM|first|morning|| +37478|AAAAAAAAHGCJAAAA|37478|10|24|38|AM|first|morning|| +37479|AAAAAAAAIGCJAAAA|37479|10|24|39|AM|first|morning|| +37480|AAAAAAAAJGCJAAAA|37480|10|24|40|AM|first|morning|| +37481|AAAAAAAAKGCJAAAA|37481|10|24|41|AM|first|morning|| +37482|AAAAAAAALGCJAAAA|37482|10|24|42|AM|first|morning|| +37483|AAAAAAAAMGCJAAAA|37483|10|24|43|AM|first|morning|| +37484|AAAAAAAANGCJAAAA|37484|10|24|44|AM|first|morning|| +37485|AAAAAAAAOGCJAAAA|37485|10|24|45|AM|first|morning|| +37486|AAAAAAAAPGCJAAAA|37486|10|24|46|AM|first|morning|| +37487|AAAAAAAAAHCJAAAA|37487|10|24|47|AM|first|morning|| +37488|AAAAAAAABHCJAAAA|37488|10|24|48|AM|first|morning|| +37489|AAAAAAAACHCJAAAA|37489|10|24|49|AM|first|morning|| +37490|AAAAAAAADHCJAAAA|37490|10|24|50|AM|first|morning|| +37491|AAAAAAAAEHCJAAAA|37491|10|24|51|AM|first|morning|| +37492|AAAAAAAAFHCJAAAA|37492|10|24|52|AM|first|morning|| +37493|AAAAAAAAGHCJAAAA|37493|10|24|53|AM|first|morning|| +37494|AAAAAAAAHHCJAAAA|37494|10|24|54|AM|first|morning|| +37495|AAAAAAAAIHCJAAAA|37495|10|24|55|AM|first|morning|| +37496|AAAAAAAAJHCJAAAA|37496|10|24|56|AM|first|morning|| +37497|AAAAAAAAKHCJAAAA|37497|10|24|57|AM|first|morning|| +37498|AAAAAAAALHCJAAAA|37498|10|24|58|AM|first|morning|| +37499|AAAAAAAAMHCJAAAA|37499|10|24|59|AM|first|morning|| +37500|AAAAAAAANHCJAAAA|37500|10|25|0|AM|first|morning|| +37501|AAAAAAAAOHCJAAAA|37501|10|25|1|AM|first|morning|| +37502|AAAAAAAAPHCJAAAA|37502|10|25|2|AM|first|morning|| +37503|AAAAAAAAAICJAAAA|37503|10|25|3|AM|first|morning|| +37504|AAAAAAAABICJAAAA|37504|10|25|4|AM|first|morning|| +37505|AAAAAAAACICJAAAA|37505|10|25|5|AM|first|morning|| +37506|AAAAAAAADICJAAAA|37506|10|25|6|AM|first|morning|| +37507|AAAAAAAAEICJAAAA|37507|10|25|7|AM|first|morning|| +37508|AAAAAAAAFICJAAAA|37508|10|25|8|AM|first|morning|| +37509|AAAAAAAAGICJAAAA|37509|10|25|9|AM|first|morning|| +37510|AAAAAAAAHICJAAAA|37510|10|25|10|AM|first|morning|| +37511|AAAAAAAAIICJAAAA|37511|10|25|11|AM|first|morning|| +37512|AAAAAAAAJICJAAAA|37512|10|25|12|AM|first|morning|| +37513|AAAAAAAAKICJAAAA|37513|10|25|13|AM|first|morning|| +37514|AAAAAAAALICJAAAA|37514|10|25|14|AM|first|morning|| +37515|AAAAAAAAMICJAAAA|37515|10|25|15|AM|first|morning|| +37516|AAAAAAAANICJAAAA|37516|10|25|16|AM|first|morning|| +37517|AAAAAAAAOICJAAAA|37517|10|25|17|AM|first|morning|| +37518|AAAAAAAAPICJAAAA|37518|10|25|18|AM|first|morning|| +37519|AAAAAAAAAJCJAAAA|37519|10|25|19|AM|first|morning|| +37520|AAAAAAAABJCJAAAA|37520|10|25|20|AM|first|morning|| +37521|AAAAAAAACJCJAAAA|37521|10|25|21|AM|first|morning|| +37522|AAAAAAAADJCJAAAA|37522|10|25|22|AM|first|morning|| +37523|AAAAAAAAEJCJAAAA|37523|10|25|23|AM|first|morning|| +37524|AAAAAAAAFJCJAAAA|37524|10|25|24|AM|first|morning|| +37525|AAAAAAAAGJCJAAAA|37525|10|25|25|AM|first|morning|| +37526|AAAAAAAAHJCJAAAA|37526|10|25|26|AM|first|morning|| +37527|AAAAAAAAIJCJAAAA|37527|10|25|27|AM|first|morning|| +37528|AAAAAAAAJJCJAAAA|37528|10|25|28|AM|first|morning|| +37529|AAAAAAAAKJCJAAAA|37529|10|25|29|AM|first|morning|| +37530|AAAAAAAALJCJAAAA|37530|10|25|30|AM|first|morning|| +37531|AAAAAAAAMJCJAAAA|37531|10|25|31|AM|first|morning|| +37532|AAAAAAAANJCJAAAA|37532|10|25|32|AM|first|morning|| +37533|AAAAAAAAOJCJAAAA|37533|10|25|33|AM|first|morning|| +37534|AAAAAAAAPJCJAAAA|37534|10|25|34|AM|first|morning|| +37535|AAAAAAAAAKCJAAAA|37535|10|25|35|AM|first|morning|| +37536|AAAAAAAABKCJAAAA|37536|10|25|36|AM|first|morning|| +37537|AAAAAAAACKCJAAAA|37537|10|25|37|AM|first|morning|| +37538|AAAAAAAADKCJAAAA|37538|10|25|38|AM|first|morning|| +37539|AAAAAAAAEKCJAAAA|37539|10|25|39|AM|first|morning|| +37540|AAAAAAAAFKCJAAAA|37540|10|25|40|AM|first|morning|| +37541|AAAAAAAAGKCJAAAA|37541|10|25|41|AM|first|morning|| +37542|AAAAAAAAHKCJAAAA|37542|10|25|42|AM|first|morning|| +37543|AAAAAAAAIKCJAAAA|37543|10|25|43|AM|first|morning|| +37544|AAAAAAAAJKCJAAAA|37544|10|25|44|AM|first|morning|| +37545|AAAAAAAAKKCJAAAA|37545|10|25|45|AM|first|morning|| +37546|AAAAAAAALKCJAAAA|37546|10|25|46|AM|first|morning|| +37547|AAAAAAAAMKCJAAAA|37547|10|25|47|AM|first|morning|| +37548|AAAAAAAANKCJAAAA|37548|10|25|48|AM|first|morning|| +37549|AAAAAAAAOKCJAAAA|37549|10|25|49|AM|first|morning|| +37550|AAAAAAAAPKCJAAAA|37550|10|25|50|AM|first|morning|| +37551|AAAAAAAAALCJAAAA|37551|10|25|51|AM|first|morning|| +37552|AAAAAAAABLCJAAAA|37552|10|25|52|AM|first|morning|| +37553|AAAAAAAACLCJAAAA|37553|10|25|53|AM|first|morning|| +37554|AAAAAAAADLCJAAAA|37554|10|25|54|AM|first|morning|| +37555|AAAAAAAAELCJAAAA|37555|10|25|55|AM|first|morning|| +37556|AAAAAAAAFLCJAAAA|37556|10|25|56|AM|first|morning|| +37557|AAAAAAAAGLCJAAAA|37557|10|25|57|AM|first|morning|| +37558|AAAAAAAAHLCJAAAA|37558|10|25|58|AM|first|morning|| +37559|AAAAAAAAILCJAAAA|37559|10|25|59|AM|first|morning|| +37560|AAAAAAAAJLCJAAAA|37560|10|26|0|AM|first|morning|| +37561|AAAAAAAAKLCJAAAA|37561|10|26|1|AM|first|morning|| +37562|AAAAAAAALLCJAAAA|37562|10|26|2|AM|first|morning|| +37563|AAAAAAAAMLCJAAAA|37563|10|26|3|AM|first|morning|| +37564|AAAAAAAANLCJAAAA|37564|10|26|4|AM|first|morning|| +37565|AAAAAAAAOLCJAAAA|37565|10|26|5|AM|first|morning|| +37566|AAAAAAAAPLCJAAAA|37566|10|26|6|AM|first|morning|| +37567|AAAAAAAAAMCJAAAA|37567|10|26|7|AM|first|morning|| +37568|AAAAAAAABMCJAAAA|37568|10|26|8|AM|first|morning|| +37569|AAAAAAAACMCJAAAA|37569|10|26|9|AM|first|morning|| +37570|AAAAAAAADMCJAAAA|37570|10|26|10|AM|first|morning|| +37571|AAAAAAAAEMCJAAAA|37571|10|26|11|AM|first|morning|| +37572|AAAAAAAAFMCJAAAA|37572|10|26|12|AM|first|morning|| +37573|AAAAAAAAGMCJAAAA|37573|10|26|13|AM|first|morning|| +37574|AAAAAAAAHMCJAAAA|37574|10|26|14|AM|first|morning|| +37575|AAAAAAAAIMCJAAAA|37575|10|26|15|AM|first|morning|| +37576|AAAAAAAAJMCJAAAA|37576|10|26|16|AM|first|morning|| +37577|AAAAAAAAKMCJAAAA|37577|10|26|17|AM|first|morning|| +37578|AAAAAAAALMCJAAAA|37578|10|26|18|AM|first|morning|| +37579|AAAAAAAAMMCJAAAA|37579|10|26|19|AM|first|morning|| +37580|AAAAAAAANMCJAAAA|37580|10|26|20|AM|first|morning|| +37581|AAAAAAAAOMCJAAAA|37581|10|26|21|AM|first|morning|| +37582|AAAAAAAAPMCJAAAA|37582|10|26|22|AM|first|morning|| +37583|AAAAAAAAANCJAAAA|37583|10|26|23|AM|first|morning|| +37584|AAAAAAAABNCJAAAA|37584|10|26|24|AM|first|morning|| +37585|AAAAAAAACNCJAAAA|37585|10|26|25|AM|first|morning|| +37586|AAAAAAAADNCJAAAA|37586|10|26|26|AM|first|morning|| +37587|AAAAAAAAENCJAAAA|37587|10|26|27|AM|first|morning|| +37588|AAAAAAAAFNCJAAAA|37588|10|26|28|AM|first|morning|| +37589|AAAAAAAAGNCJAAAA|37589|10|26|29|AM|first|morning|| +37590|AAAAAAAAHNCJAAAA|37590|10|26|30|AM|first|morning|| +37591|AAAAAAAAINCJAAAA|37591|10|26|31|AM|first|morning|| +37592|AAAAAAAAJNCJAAAA|37592|10|26|32|AM|first|morning|| +37593|AAAAAAAAKNCJAAAA|37593|10|26|33|AM|first|morning|| +37594|AAAAAAAALNCJAAAA|37594|10|26|34|AM|first|morning|| +37595|AAAAAAAAMNCJAAAA|37595|10|26|35|AM|first|morning|| +37596|AAAAAAAANNCJAAAA|37596|10|26|36|AM|first|morning|| +37597|AAAAAAAAONCJAAAA|37597|10|26|37|AM|first|morning|| +37598|AAAAAAAAPNCJAAAA|37598|10|26|38|AM|first|morning|| +37599|AAAAAAAAAOCJAAAA|37599|10|26|39|AM|first|morning|| +37600|AAAAAAAABOCJAAAA|37600|10|26|40|AM|first|morning|| +37601|AAAAAAAACOCJAAAA|37601|10|26|41|AM|first|morning|| +37602|AAAAAAAADOCJAAAA|37602|10|26|42|AM|first|morning|| +37603|AAAAAAAAEOCJAAAA|37603|10|26|43|AM|first|morning|| +37604|AAAAAAAAFOCJAAAA|37604|10|26|44|AM|first|morning|| +37605|AAAAAAAAGOCJAAAA|37605|10|26|45|AM|first|morning|| +37606|AAAAAAAAHOCJAAAA|37606|10|26|46|AM|first|morning|| +37607|AAAAAAAAIOCJAAAA|37607|10|26|47|AM|first|morning|| +37608|AAAAAAAAJOCJAAAA|37608|10|26|48|AM|first|morning|| +37609|AAAAAAAAKOCJAAAA|37609|10|26|49|AM|first|morning|| +37610|AAAAAAAALOCJAAAA|37610|10|26|50|AM|first|morning|| +37611|AAAAAAAAMOCJAAAA|37611|10|26|51|AM|first|morning|| +37612|AAAAAAAANOCJAAAA|37612|10|26|52|AM|first|morning|| +37613|AAAAAAAAOOCJAAAA|37613|10|26|53|AM|first|morning|| +37614|AAAAAAAAPOCJAAAA|37614|10|26|54|AM|first|morning|| +37615|AAAAAAAAAPCJAAAA|37615|10|26|55|AM|first|morning|| +37616|AAAAAAAABPCJAAAA|37616|10|26|56|AM|first|morning|| +37617|AAAAAAAACPCJAAAA|37617|10|26|57|AM|first|morning|| +37618|AAAAAAAADPCJAAAA|37618|10|26|58|AM|first|morning|| +37619|AAAAAAAAEPCJAAAA|37619|10|26|59|AM|first|morning|| +37620|AAAAAAAAFPCJAAAA|37620|10|27|0|AM|first|morning|| +37621|AAAAAAAAGPCJAAAA|37621|10|27|1|AM|first|morning|| +37622|AAAAAAAAHPCJAAAA|37622|10|27|2|AM|first|morning|| +37623|AAAAAAAAIPCJAAAA|37623|10|27|3|AM|first|morning|| +37624|AAAAAAAAJPCJAAAA|37624|10|27|4|AM|first|morning|| +37625|AAAAAAAAKPCJAAAA|37625|10|27|5|AM|first|morning|| +37626|AAAAAAAALPCJAAAA|37626|10|27|6|AM|first|morning|| +37627|AAAAAAAAMPCJAAAA|37627|10|27|7|AM|first|morning|| +37628|AAAAAAAANPCJAAAA|37628|10|27|8|AM|first|morning|| +37629|AAAAAAAAOPCJAAAA|37629|10|27|9|AM|first|morning|| +37630|AAAAAAAAPPCJAAAA|37630|10|27|10|AM|first|morning|| +37631|AAAAAAAAAADJAAAA|37631|10|27|11|AM|first|morning|| +37632|AAAAAAAABADJAAAA|37632|10|27|12|AM|first|morning|| +37633|AAAAAAAACADJAAAA|37633|10|27|13|AM|first|morning|| +37634|AAAAAAAADADJAAAA|37634|10|27|14|AM|first|morning|| +37635|AAAAAAAAEADJAAAA|37635|10|27|15|AM|first|morning|| +37636|AAAAAAAAFADJAAAA|37636|10|27|16|AM|first|morning|| +37637|AAAAAAAAGADJAAAA|37637|10|27|17|AM|first|morning|| +37638|AAAAAAAAHADJAAAA|37638|10|27|18|AM|first|morning|| +37639|AAAAAAAAIADJAAAA|37639|10|27|19|AM|first|morning|| +37640|AAAAAAAAJADJAAAA|37640|10|27|20|AM|first|morning|| +37641|AAAAAAAAKADJAAAA|37641|10|27|21|AM|first|morning|| +37642|AAAAAAAALADJAAAA|37642|10|27|22|AM|first|morning|| +37643|AAAAAAAAMADJAAAA|37643|10|27|23|AM|first|morning|| +37644|AAAAAAAANADJAAAA|37644|10|27|24|AM|first|morning|| +37645|AAAAAAAAOADJAAAA|37645|10|27|25|AM|first|morning|| +37646|AAAAAAAAPADJAAAA|37646|10|27|26|AM|first|morning|| +37647|AAAAAAAAABDJAAAA|37647|10|27|27|AM|first|morning|| +37648|AAAAAAAABBDJAAAA|37648|10|27|28|AM|first|morning|| +37649|AAAAAAAACBDJAAAA|37649|10|27|29|AM|first|morning|| +37650|AAAAAAAADBDJAAAA|37650|10|27|30|AM|first|morning|| +37651|AAAAAAAAEBDJAAAA|37651|10|27|31|AM|first|morning|| +37652|AAAAAAAAFBDJAAAA|37652|10|27|32|AM|first|morning|| +37653|AAAAAAAAGBDJAAAA|37653|10|27|33|AM|first|morning|| +37654|AAAAAAAAHBDJAAAA|37654|10|27|34|AM|first|morning|| +37655|AAAAAAAAIBDJAAAA|37655|10|27|35|AM|first|morning|| +37656|AAAAAAAAJBDJAAAA|37656|10|27|36|AM|first|morning|| +37657|AAAAAAAAKBDJAAAA|37657|10|27|37|AM|first|morning|| +37658|AAAAAAAALBDJAAAA|37658|10|27|38|AM|first|morning|| +37659|AAAAAAAAMBDJAAAA|37659|10|27|39|AM|first|morning|| +37660|AAAAAAAANBDJAAAA|37660|10|27|40|AM|first|morning|| +37661|AAAAAAAAOBDJAAAA|37661|10|27|41|AM|first|morning|| +37662|AAAAAAAAPBDJAAAA|37662|10|27|42|AM|first|morning|| +37663|AAAAAAAAACDJAAAA|37663|10|27|43|AM|first|morning|| +37664|AAAAAAAABCDJAAAA|37664|10|27|44|AM|first|morning|| +37665|AAAAAAAACCDJAAAA|37665|10|27|45|AM|first|morning|| +37666|AAAAAAAADCDJAAAA|37666|10|27|46|AM|first|morning|| +37667|AAAAAAAAECDJAAAA|37667|10|27|47|AM|first|morning|| +37668|AAAAAAAAFCDJAAAA|37668|10|27|48|AM|first|morning|| +37669|AAAAAAAAGCDJAAAA|37669|10|27|49|AM|first|morning|| +37670|AAAAAAAAHCDJAAAA|37670|10|27|50|AM|first|morning|| +37671|AAAAAAAAICDJAAAA|37671|10|27|51|AM|first|morning|| +37672|AAAAAAAAJCDJAAAA|37672|10|27|52|AM|first|morning|| +37673|AAAAAAAAKCDJAAAA|37673|10|27|53|AM|first|morning|| +37674|AAAAAAAALCDJAAAA|37674|10|27|54|AM|first|morning|| +37675|AAAAAAAAMCDJAAAA|37675|10|27|55|AM|first|morning|| +37676|AAAAAAAANCDJAAAA|37676|10|27|56|AM|first|morning|| +37677|AAAAAAAAOCDJAAAA|37677|10|27|57|AM|first|morning|| +37678|AAAAAAAAPCDJAAAA|37678|10|27|58|AM|first|morning|| +37679|AAAAAAAAADDJAAAA|37679|10|27|59|AM|first|morning|| +37680|AAAAAAAABDDJAAAA|37680|10|28|0|AM|first|morning|| +37681|AAAAAAAACDDJAAAA|37681|10|28|1|AM|first|morning|| +37682|AAAAAAAADDDJAAAA|37682|10|28|2|AM|first|morning|| +37683|AAAAAAAAEDDJAAAA|37683|10|28|3|AM|first|morning|| +37684|AAAAAAAAFDDJAAAA|37684|10|28|4|AM|first|morning|| +37685|AAAAAAAAGDDJAAAA|37685|10|28|5|AM|first|morning|| +37686|AAAAAAAAHDDJAAAA|37686|10|28|6|AM|first|morning|| +37687|AAAAAAAAIDDJAAAA|37687|10|28|7|AM|first|morning|| +37688|AAAAAAAAJDDJAAAA|37688|10|28|8|AM|first|morning|| +37689|AAAAAAAAKDDJAAAA|37689|10|28|9|AM|first|morning|| +37690|AAAAAAAALDDJAAAA|37690|10|28|10|AM|first|morning|| +37691|AAAAAAAAMDDJAAAA|37691|10|28|11|AM|first|morning|| +37692|AAAAAAAANDDJAAAA|37692|10|28|12|AM|first|morning|| +37693|AAAAAAAAODDJAAAA|37693|10|28|13|AM|first|morning|| +37694|AAAAAAAAPDDJAAAA|37694|10|28|14|AM|first|morning|| +37695|AAAAAAAAAEDJAAAA|37695|10|28|15|AM|first|morning|| +37696|AAAAAAAABEDJAAAA|37696|10|28|16|AM|first|morning|| +37697|AAAAAAAACEDJAAAA|37697|10|28|17|AM|first|morning|| +37698|AAAAAAAADEDJAAAA|37698|10|28|18|AM|first|morning|| +37699|AAAAAAAAEEDJAAAA|37699|10|28|19|AM|first|morning|| +37700|AAAAAAAAFEDJAAAA|37700|10|28|20|AM|first|morning|| +37701|AAAAAAAAGEDJAAAA|37701|10|28|21|AM|first|morning|| +37702|AAAAAAAAHEDJAAAA|37702|10|28|22|AM|first|morning|| +37703|AAAAAAAAIEDJAAAA|37703|10|28|23|AM|first|morning|| +37704|AAAAAAAAJEDJAAAA|37704|10|28|24|AM|first|morning|| +37705|AAAAAAAAKEDJAAAA|37705|10|28|25|AM|first|morning|| +37706|AAAAAAAALEDJAAAA|37706|10|28|26|AM|first|morning|| +37707|AAAAAAAAMEDJAAAA|37707|10|28|27|AM|first|morning|| +37708|AAAAAAAANEDJAAAA|37708|10|28|28|AM|first|morning|| +37709|AAAAAAAAOEDJAAAA|37709|10|28|29|AM|first|morning|| +37710|AAAAAAAAPEDJAAAA|37710|10|28|30|AM|first|morning|| +37711|AAAAAAAAAFDJAAAA|37711|10|28|31|AM|first|morning|| +37712|AAAAAAAABFDJAAAA|37712|10|28|32|AM|first|morning|| +37713|AAAAAAAACFDJAAAA|37713|10|28|33|AM|first|morning|| +37714|AAAAAAAADFDJAAAA|37714|10|28|34|AM|first|morning|| +37715|AAAAAAAAEFDJAAAA|37715|10|28|35|AM|first|morning|| +37716|AAAAAAAAFFDJAAAA|37716|10|28|36|AM|first|morning|| +37717|AAAAAAAAGFDJAAAA|37717|10|28|37|AM|first|morning|| +37718|AAAAAAAAHFDJAAAA|37718|10|28|38|AM|first|morning|| +37719|AAAAAAAAIFDJAAAA|37719|10|28|39|AM|first|morning|| +37720|AAAAAAAAJFDJAAAA|37720|10|28|40|AM|first|morning|| +37721|AAAAAAAAKFDJAAAA|37721|10|28|41|AM|first|morning|| +37722|AAAAAAAALFDJAAAA|37722|10|28|42|AM|first|morning|| +37723|AAAAAAAAMFDJAAAA|37723|10|28|43|AM|first|morning|| +37724|AAAAAAAANFDJAAAA|37724|10|28|44|AM|first|morning|| +37725|AAAAAAAAOFDJAAAA|37725|10|28|45|AM|first|morning|| +37726|AAAAAAAAPFDJAAAA|37726|10|28|46|AM|first|morning|| +37727|AAAAAAAAAGDJAAAA|37727|10|28|47|AM|first|morning|| +37728|AAAAAAAABGDJAAAA|37728|10|28|48|AM|first|morning|| +37729|AAAAAAAACGDJAAAA|37729|10|28|49|AM|first|morning|| +37730|AAAAAAAADGDJAAAA|37730|10|28|50|AM|first|morning|| +37731|AAAAAAAAEGDJAAAA|37731|10|28|51|AM|first|morning|| +37732|AAAAAAAAFGDJAAAA|37732|10|28|52|AM|first|morning|| +37733|AAAAAAAAGGDJAAAA|37733|10|28|53|AM|first|morning|| +37734|AAAAAAAAHGDJAAAA|37734|10|28|54|AM|first|morning|| +37735|AAAAAAAAIGDJAAAA|37735|10|28|55|AM|first|morning|| +37736|AAAAAAAAJGDJAAAA|37736|10|28|56|AM|first|morning|| +37737|AAAAAAAAKGDJAAAA|37737|10|28|57|AM|first|morning|| +37738|AAAAAAAALGDJAAAA|37738|10|28|58|AM|first|morning|| +37739|AAAAAAAAMGDJAAAA|37739|10|28|59|AM|first|morning|| +37740|AAAAAAAANGDJAAAA|37740|10|29|0|AM|first|morning|| +37741|AAAAAAAAOGDJAAAA|37741|10|29|1|AM|first|morning|| +37742|AAAAAAAAPGDJAAAA|37742|10|29|2|AM|first|morning|| +37743|AAAAAAAAAHDJAAAA|37743|10|29|3|AM|first|morning|| +37744|AAAAAAAABHDJAAAA|37744|10|29|4|AM|first|morning|| +37745|AAAAAAAACHDJAAAA|37745|10|29|5|AM|first|morning|| +37746|AAAAAAAADHDJAAAA|37746|10|29|6|AM|first|morning|| +37747|AAAAAAAAEHDJAAAA|37747|10|29|7|AM|first|morning|| +37748|AAAAAAAAFHDJAAAA|37748|10|29|8|AM|first|morning|| +37749|AAAAAAAAGHDJAAAA|37749|10|29|9|AM|first|morning|| +37750|AAAAAAAAHHDJAAAA|37750|10|29|10|AM|first|morning|| +37751|AAAAAAAAIHDJAAAA|37751|10|29|11|AM|first|morning|| +37752|AAAAAAAAJHDJAAAA|37752|10|29|12|AM|first|morning|| +37753|AAAAAAAAKHDJAAAA|37753|10|29|13|AM|first|morning|| +37754|AAAAAAAALHDJAAAA|37754|10|29|14|AM|first|morning|| +37755|AAAAAAAAMHDJAAAA|37755|10|29|15|AM|first|morning|| +37756|AAAAAAAANHDJAAAA|37756|10|29|16|AM|first|morning|| +37757|AAAAAAAAOHDJAAAA|37757|10|29|17|AM|first|morning|| +37758|AAAAAAAAPHDJAAAA|37758|10|29|18|AM|first|morning|| +37759|AAAAAAAAAIDJAAAA|37759|10|29|19|AM|first|morning|| +37760|AAAAAAAABIDJAAAA|37760|10|29|20|AM|first|morning|| +37761|AAAAAAAACIDJAAAA|37761|10|29|21|AM|first|morning|| +37762|AAAAAAAADIDJAAAA|37762|10|29|22|AM|first|morning|| +37763|AAAAAAAAEIDJAAAA|37763|10|29|23|AM|first|morning|| +37764|AAAAAAAAFIDJAAAA|37764|10|29|24|AM|first|morning|| +37765|AAAAAAAAGIDJAAAA|37765|10|29|25|AM|first|morning|| +37766|AAAAAAAAHIDJAAAA|37766|10|29|26|AM|first|morning|| +37767|AAAAAAAAIIDJAAAA|37767|10|29|27|AM|first|morning|| +37768|AAAAAAAAJIDJAAAA|37768|10|29|28|AM|first|morning|| +37769|AAAAAAAAKIDJAAAA|37769|10|29|29|AM|first|morning|| +37770|AAAAAAAALIDJAAAA|37770|10|29|30|AM|first|morning|| +37771|AAAAAAAAMIDJAAAA|37771|10|29|31|AM|first|morning|| +37772|AAAAAAAANIDJAAAA|37772|10|29|32|AM|first|morning|| +37773|AAAAAAAAOIDJAAAA|37773|10|29|33|AM|first|morning|| +37774|AAAAAAAAPIDJAAAA|37774|10|29|34|AM|first|morning|| +37775|AAAAAAAAAJDJAAAA|37775|10|29|35|AM|first|morning|| +37776|AAAAAAAABJDJAAAA|37776|10|29|36|AM|first|morning|| +37777|AAAAAAAACJDJAAAA|37777|10|29|37|AM|first|morning|| +37778|AAAAAAAADJDJAAAA|37778|10|29|38|AM|first|morning|| +37779|AAAAAAAAEJDJAAAA|37779|10|29|39|AM|first|morning|| +37780|AAAAAAAAFJDJAAAA|37780|10|29|40|AM|first|morning|| +37781|AAAAAAAAGJDJAAAA|37781|10|29|41|AM|first|morning|| +37782|AAAAAAAAHJDJAAAA|37782|10|29|42|AM|first|morning|| +37783|AAAAAAAAIJDJAAAA|37783|10|29|43|AM|first|morning|| +37784|AAAAAAAAJJDJAAAA|37784|10|29|44|AM|first|morning|| +37785|AAAAAAAAKJDJAAAA|37785|10|29|45|AM|first|morning|| +37786|AAAAAAAALJDJAAAA|37786|10|29|46|AM|first|morning|| +37787|AAAAAAAAMJDJAAAA|37787|10|29|47|AM|first|morning|| +37788|AAAAAAAANJDJAAAA|37788|10|29|48|AM|first|morning|| +37789|AAAAAAAAOJDJAAAA|37789|10|29|49|AM|first|morning|| +37790|AAAAAAAAPJDJAAAA|37790|10|29|50|AM|first|morning|| +37791|AAAAAAAAAKDJAAAA|37791|10|29|51|AM|first|morning|| +37792|AAAAAAAABKDJAAAA|37792|10|29|52|AM|first|morning|| +37793|AAAAAAAACKDJAAAA|37793|10|29|53|AM|first|morning|| +37794|AAAAAAAADKDJAAAA|37794|10|29|54|AM|first|morning|| +37795|AAAAAAAAEKDJAAAA|37795|10|29|55|AM|first|morning|| +37796|AAAAAAAAFKDJAAAA|37796|10|29|56|AM|first|morning|| +37797|AAAAAAAAGKDJAAAA|37797|10|29|57|AM|first|morning|| +37798|AAAAAAAAHKDJAAAA|37798|10|29|58|AM|first|morning|| +37799|AAAAAAAAIKDJAAAA|37799|10|29|59|AM|first|morning|| +37800|AAAAAAAAJKDJAAAA|37800|10|30|0|AM|first|morning|| +37801|AAAAAAAAKKDJAAAA|37801|10|30|1|AM|first|morning|| +37802|AAAAAAAALKDJAAAA|37802|10|30|2|AM|first|morning|| +37803|AAAAAAAAMKDJAAAA|37803|10|30|3|AM|first|morning|| +37804|AAAAAAAANKDJAAAA|37804|10|30|4|AM|first|morning|| +37805|AAAAAAAAOKDJAAAA|37805|10|30|5|AM|first|morning|| +37806|AAAAAAAAPKDJAAAA|37806|10|30|6|AM|first|morning|| +37807|AAAAAAAAALDJAAAA|37807|10|30|7|AM|first|morning|| +37808|AAAAAAAABLDJAAAA|37808|10|30|8|AM|first|morning|| +37809|AAAAAAAACLDJAAAA|37809|10|30|9|AM|first|morning|| +37810|AAAAAAAADLDJAAAA|37810|10|30|10|AM|first|morning|| +37811|AAAAAAAAELDJAAAA|37811|10|30|11|AM|first|morning|| +37812|AAAAAAAAFLDJAAAA|37812|10|30|12|AM|first|morning|| +37813|AAAAAAAAGLDJAAAA|37813|10|30|13|AM|first|morning|| +37814|AAAAAAAAHLDJAAAA|37814|10|30|14|AM|first|morning|| +37815|AAAAAAAAILDJAAAA|37815|10|30|15|AM|first|morning|| +37816|AAAAAAAAJLDJAAAA|37816|10|30|16|AM|first|morning|| +37817|AAAAAAAAKLDJAAAA|37817|10|30|17|AM|first|morning|| +37818|AAAAAAAALLDJAAAA|37818|10|30|18|AM|first|morning|| +37819|AAAAAAAAMLDJAAAA|37819|10|30|19|AM|first|morning|| +37820|AAAAAAAANLDJAAAA|37820|10|30|20|AM|first|morning|| +37821|AAAAAAAAOLDJAAAA|37821|10|30|21|AM|first|morning|| +37822|AAAAAAAAPLDJAAAA|37822|10|30|22|AM|first|morning|| +37823|AAAAAAAAAMDJAAAA|37823|10|30|23|AM|first|morning|| +37824|AAAAAAAABMDJAAAA|37824|10|30|24|AM|first|morning|| +37825|AAAAAAAACMDJAAAA|37825|10|30|25|AM|first|morning|| +37826|AAAAAAAADMDJAAAA|37826|10|30|26|AM|first|morning|| +37827|AAAAAAAAEMDJAAAA|37827|10|30|27|AM|first|morning|| +37828|AAAAAAAAFMDJAAAA|37828|10|30|28|AM|first|morning|| +37829|AAAAAAAAGMDJAAAA|37829|10|30|29|AM|first|morning|| +37830|AAAAAAAAHMDJAAAA|37830|10|30|30|AM|first|morning|| +37831|AAAAAAAAIMDJAAAA|37831|10|30|31|AM|first|morning|| +37832|AAAAAAAAJMDJAAAA|37832|10|30|32|AM|first|morning|| +37833|AAAAAAAAKMDJAAAA|37833|10|30|33|AM|first|morning|| +37834|AAAAAAAALMDJAAAA|37834|10|30|34|AM|first|morning|| +37835|AAAAAAAAMMDJAAAA|37835|10|30|35|AM|first|morning|| +37836|AAAAAAAANMDJAAAA|37836|10|30|36|AM|first|morning|| +37837|AAAAAAAAOMDJAAAA|37837|10|30|37|AM|first|morning|| +37838|AAAAAAAAPMDJAAAA|37838|10|30|38|AM|first|morning|| +37839|AAAAAAAAANDJAAAA|37839|10|30|39|AM|first|morning|| +37840|AAAAAAAABNDJAAAA|37840|10|30|40|AM|first|morning|| +37841|AAAAAAAACNDJAAAA|37841|10|30|41|AM|first|morning|| +37842|AAAAAAAADNDJAAAA|37842|10|30|42|AM|first|morning|| +37843|AAAAAAAAENDJAAAA|37843|10|30|43|AM|first|morning|| +37844|AAAAAAAAFNDJAAAA|37844|10|30|44|AM|first|morning|| +37845|AAAAAAAAGNDJAAAA|37845|10|30|45|AM|first|morning|| +37846|AAAAAAAAHNDJAAAA|37846|10|30|46|AM|first|morning|| +37847|AAAAAAAAINDJAAAA|37847|10|30|47|AM|first|morning|| +37848|AAAAAAAAJNDJAAAA|37848|10|30|48|AM|first|morning|| +37849|AAAAAAAAKNDJAAAA|37849|10|30|49|AM|first|morning|| +37850|AAAAAAAALNDJAAAA|37850|10|30|50|AM|first|morning|| +37851|AAAAAAAAMNDJAAAA|37851|10|30|51|AM|first|morning|| +37852|AAAAAAAANNDJAAAA|37852|10|30|52|AM|first|morning|| +37853|AAAAAAAAONDJAAAA|37853|10|30|53|AM|first|morning|| +37854|AAAAAAAAPNDJAAAA|37854|10|30|54|AM|first|morning|| +37855|AAAAAAAAAODJAAAA|37855|10|30|55|AM|first|morning|| +37856|AAAAAAAABODJAAAA|37856|10|30|56|AM|first|morning|| +37857|AAAAAAAACODJAAAA|37857|10|30|57|AM|first|morning|| +37858|AAAAAAAADODJAAAA|37858|10|30|58|AM|first|morning|| +37859|AAAAAAAAEODJAAAA|37859|10|30|59|AM|first|morning|| +37860|AAAAAAAAFODJAAAA|37860|10|31|0|AM|first|morning|| +37861|AAAAAAAAGODJAAAA|37861|10|31|1|AM|first|morning|| +37862|AAAAAAAAHODJAAAA|37862|10|31|2|AM|first|morning|| +37863|AAAAAAAAIODJAAAA|37863|10|31|3|AM|first|morning|| +37864|AAAAAAAAJODJAAAA|37864|10|31|4|AM|first|morning|| +37865|AAAAAAAAKODJAAAA|37865|10|31|5|AM|first|morning|| +37866|AAAAAAAALODJAAAA|37866|10|31|6|AM|first|morning|| +37867|AAAAAAAAMODJAAAA|37867|10|31|7|AM|first|morning|| +37868|AAAAAAAANODJAAAA|37868|10|31|8|AM|first|morning|| +37869|AAAAAAAAOODJAAAA|37869|10|31|9|AM|first|morning|| +37870|AAAAAAAAPODJAAAA|37870|10|31|10|AM|first|morning|| +37871|AAAAAAAAAPDJAAAA|37871|10|31|11|AM|first|morning|| +37872|AAAAAAAABPDJAAAA|37872|10|31|12|AM|first|morning|| +37873|AAAAAAAACPDJAAAA|37873|10|31|13|AM|first|morning|| +37874|AAAAAAAADPDJAAAA|37874|10|31|14|AM|first|morning|| +37875|AAAAAAAAEPDJAAAA|37875|10|31|15|AM|first|morning|| +37876|AAAAAAAAFPDJAAAA|37876|10|31|16|AM|first|morning|| +37877|AAAAAAAAGPDJAAAA|37877|10|31|17|AM|first|morning|| +37878|AAAAAAAAHPDJAAAA|37878|10|31|18|AM|first|morning|| +37879|AAAAAAAAIPDJAAAA|37879|10|31|19|AM|first|morning|| +37880|AAAAAAAAJPDJAAAA|37880|10|31|20|AM|first|morning|| +37881|AAAAAAAAKPDJAAAA|37881|10|31|21|AM|first|morning|| +37882|AAAAAAAALPDJAAAA|37882|10|31|22|AM|first|morning|| +37883|AAAAAAAAMPDJAAAA|37883|10|31|23|AM|first|morning|| +37884|AAAAAAAANPDJAAAA|37884|10|31|24|AM|first|morning|| +37885|AAAAAAAAOPDJAAAA|37885|10|31|25|AM|first|morning|| +37886|AAAAAAAAPPDJAAAA|37886|10|31|26|AM|first|morning|| +37887|AAAAAAAAAAEJAAAA|37887|10|31|27|AM|first|morning|| +37888|AAAAAAAABAEJAAAA|37888|10|31|28|AM|first|morning|| +37889|AAAAAAAACAEJAAAA|37889|10|31|29|AM|first|morning|| +37890|AAAAAAAADAEJAAAA|37890|10|31|30|AM|first|morning|| +37891|AAAAAAAAEAEJAAAA|37891|10|31|31|AM|first|morning|| +37892|AAAAAAAAFAEJAAAA|37892|10|31|32|AM|first|morning|| +37893|AAAAAAAAGAEJAAAA|37893|10|31|33|AM|first|morning|| +37894|AAAAAAAAHAEJAAAA|37894|10|31|34|AM|first|morning|| +37895|AAAAAAAAIAEJAAAA|37895|10|31|35|AM|first|morning|| +37896|AAAAAAAAJAEJAAAA|37896|10|31|36|AM|first|morning|| +37897|AAAAAAAAKAEJAAAA|37897|10|31|37|AM|first|morning|| +37898|AAAAAAAALAEJAAAA|37898|10|31|38|AM|first|morning|| +37899|AAAAAAAAMAEJAAAA|37899|10|31|39|AM|first|morning|| +37900|AAAAAAAANAEJAAAA|37900|10|31|40|AM|first|morning|| +37901|AAAAAAAAOAEJAAAA|37901|10|31|41|AM|first|morning|| +37902|AAAAAAAAPAEJAAAA|37902|10|31|42|AM|first|morning|| +37903|AAAAAAAAABEJAAAA|37903|10|31|43|AM|first|morning|| +37904|AAAAAAAABBEJAAAA|37904|10|31|44|AM|first|morning|| +37905|AAAAAAAACBEJAAAA|37905|10|31|45|AM|first|morning|| +37906|AAAAAAAADBEJAAAA|37906|10|31|46|AM|first|morning|| +37907|AAAAAAAAEBEJAAAA|37907|10|31|47|AM|first|morning|| +37908|AAAAAAAAFBEJAAAA|37908|10|31|48|AM|first|morning|| +37909|AAAAAAAAGBEJAAAA|37909|10|31|49|AM|first|morning|| +37910|AAAAAAAAHBEJAAAA|37910|10|31|50|AM|first|morning|| +37911|AAAAAAAAIBEJAAAA|37911|10|31|51|AM|first|morning|| +37912|AAAAAAAAJBEJAAAA|37912|10|31|52|AM|first|morning|| +37913|AAAAAAAAKBEJAAAA|37913|10|31|53|AM|first|morning|| +37914|AAAAAAAALBEJAAAA|37914|10|31|54|AM|first|morning|| +37915|AAAAAAAAMBEJAAAA|37915|10|31|55|AM|first|morning|| +37916|AAAAAAAANBEJAAAA|37916|10|31|56|AM|first|morning|| +37917|AAAAAAAAOBEJAAAA|37917|10|31|57|AM|first|morning|| +37918|AAAAAAAAPBEJAAAA|37918|10|31|58|AM|first|morning|| +37919|AAAAAAAAACEJAAAA|37919|10|31|59|AM|first|morning|| +37920|AAAAAAAABCEJAAAA|37920|10|32|0|AM|first|morning|| +37921|AAAAAAAACCEJAAAA|37921|10|32|1|AM|first|morning|| +37922|AAAAAAAADCEJAAAA|37922|10|32|2|AM|first|morning|| +37923|AAAAAAAAECEJAAAA|37923|10|32|3|AM|first|morning|| +37924|AAAAAAAAFCEJAAAA|37924|10|32|4|AM|first|morning|| +37925|AAAAAAAAGCEJAAAA|37925|10|32|5|AM|first|morning|| +37926|AAAAAAAAHCEJAAAA|37926|10|32|6|AM|first|morning|| +37927|AAAAAAAAICEJAAAA|37927|10|32|7|AM|first|morning|| +37928|AAAAAAAAJCEJAAAA|37928|10|32|8|AM|first|morning|| +37929|AAAAAAAAKCEJAAAA|37929|10|32|9|AM|first|morning|| +37930|AAAAAAAALCEJAAAA|37930|10|32|10|AM|first|morning|| +37931|AAAAAAAAMCEJAAAA|37931|10|32|11|AM|first|morning|| +37932|AAAAAAAANCEJAAAA|37932|10|32|12|AM|first|morning|| +37933|AAAAAAAAOCEJAAAA|37933|10|32|13|AM|first|morning|| +37934|AAAAAAAAPCEJAAAA|37934|10|32|14|AM|first|morning|| +37935|AAAAAAAAADEJAAAA|37935|10|32|15|AM|first|morning|| +37936|AAAAAAAABDEJAAAA|37936|10|32|16|AM|first|morning|| +37937|AAAAAAAACDEJAAAA|37937|10|32|17|AM|first|morning|| +37938|AAAAAAAADDEJAAAA|37938|10|32|18|AM|first|morning|| +37939|AAAAAAAAEDEJAAAA|37939|10|32|19|AM|first|morning|| +37940|AAAAAAAAFDEJAAAA|37940|10|32|20|AM|first|morning|| +37941|AAAAAAAAGDEJAAAA|37941|10|32|21|AM|first|morning|| +37942|AAAAAAAAHDEJAAAA|37942|10|32|22|AM|first|morning|| +37943|AAAAAAAAIDEJAAAA|37943|10|32|23|AM|first|morning|| +37944|AAAAAAAAJDEJAAAA|37944|10|32|24|AM|first|morning|| +37945|AAAAAAAAKDEJAAAA|37945|10|32|25|AM|first|morning|| +37946|AAAAAAAALDEJAAAA|37946|10|32|26|AM|first|morning|| +37947|AAAAAAAAMDEJAAAA|37947|10|32|27|AM|first|morning|| +37948|AAAAAAAANDEJAAAA|37948|10|32|28|AM|first|morning|| +37949|AAAAAAAAODEJAAAA|37949|10|32|29|AM|first|morning|| +37950|AAAAAAAAPDEJAAAA|37950|10|32|30|AM|first|morning|| +37951|AAAAAAAAAEEJAAAA|37951|10|32|31|AM|first|morning|| +37952|AAAAAAAABEEJAAAA|37952|10|32|32|AM|first|morning|| +37953|AAAAAAAACEEJAAAA|37953|10|32|33|AM|first|morning|| +37954|AAAAAAAADEEJAAAA|37954|10|32|34|AM|first|morning|| +37955|AAAAAAAAEEEJAAAA|37955|10|32|35|AM|first|morning|| +37956|AAAAAAAAFEEJAAAA|37956|10|32|36|AM|first|morning|| +37957|AAAAAAAAGEEJAAAA|37957|10|32|37|AM|first|morning|| +37958|AAAAAAAAHEEJAAAA|37958|10|32|38|AM|first|morning|| +37959|AAAAAAAAIEEJAAAA|37959|10|32|39|AM|first|morning|| +37960|AAAAAAAAJEEJAAAA|37960|10|32|40|AM|first|morning|| +37961|AAAAAAAAKEEJAAAA|37961|10|32|41|AM|first|morning|| +37962|AAAAAAAALEEJAAAA|37962|10|32|42|AM|first|morning|| +37963|AAAAAAAAMEEJAAAA|37963|10|32|43|AM|first|morning|| +37964|AAAAAAAANEEJAAAA|37964|10|32|44|AM|first|morning|| +37965|AAAAAAAAOEEJAAAA|37965|10|32|45|AM|first|morning|| +37966|AAAAAAAAPEEJAAAA|37966|10|32|46|AM|first|morning|| +37967|AAAAAAAAAFEJAAAA|37967|10|32|47|AM|first|morning|| +37968|AAAAAAAABFEJAAAA|37968|10|32|48|AM|first|morning|| +37969|AAAAAAAACFEJAAAA|37969|10|32|49|AM|first|morning|| +37970|AAAAAAAADFEJAAAA|37970|10|32|50|AM|first|morning|| +37971|AAAAAAAAEFEJAAAA|37971|10|32|51|AM|first|morning|| +37972|AAAAAAAAFFEJAAAA|37972|10|32|52|AM|first|morning|| +37973|AAAAAAAAGFEJAAAA|37973|10|32|53|AM|first|morning|| +37974|AAAAAAAAHFEJAAAA|37974|10|32|54|AM|first|morning|| +37975|AAAAAAAAIFEJAAAA|37975|10|32|55|AM|first|morning|| +37976|AAAAAAAAJFEJAAAA|37976|10|32|56|AM|first|morning|| +37977|AAAAAAAAKFEJAAAA|37977|10|32|57|AM|first|morning|| +37978|AAAAAAAALFEJAAAA|37978|10|32|58|AM|first|morning|| +37979|AAAAAAAAMFEJAAAA|37979|10|32|59|AM|first|morning|| +37980|AAAAAAAANFEJAAAA|37980|10|33|0|AM|first|morning|| +37981|AAAAAAAAOFEJAAAA|37981|10|33|1|AM|first|morning|| +37982|AAAAAAAAPFEJAAAA|37982|10|33|2|AM|first|morning|| +37983|AAAAAAAAAGEJAAAA|37983|10|33|3|AM|first|morning|| +37984|AAAAAAAABGEJAAAA|37984|10|33|4|AM|first|morning|| +37985|AAAAAAAACGEJAAAA|37985|10|33|5|AM|first|morning|| +37986|AAAAAAAADGEJAAAA|37986|10|33|6|AM|first|morning|| +37987|AAAAAAAAEGEJAAAA|37987|10|33|7|AM|first|morning|| +37988|AAAAAAAAFGEJAAAA|37988|10|33|8|AM|first|morning|| +37989|AAAAAAAAGGEJAAAA|37989|10|33|9|AM|first|morning|| +37990|AAAAAAAAHGEJAAAA|37990|10|33|10|AM|first|morning|| +37991|AAAAAAAAIGEJAAAA|37991|10|33|11|AM|first|morning|| +37992|AAAAAAAAJGEJAAAA|37992|10|33|12|AM|first|morning|| +37993|AAAAAAAAKGEJAAAA|37993|10|33|13|AM|first|morning|| +37994|AAAAAAAALGEJAAAA|37994|10|33|14|AM|first|morning|| +37995|AAAAAAAAMGEJAAAA|37995|10|33|15|AM|first|morning|| +37996|AAAAAAAANGEJAAAA|37996|10|33|16|AM|first|morning|| +37997|AAAAAAAAOGEJAAAA|37997|10|33|17|AM|first|morning|| +37998|AAAAAAAAPGEJAAAA|37998|10|33|18|AM|first|morning|| +37999|AAAAAAAAAHEJAAAA|37999|10|33|19|AM|first|morning|| +38000|AAAAAAAABHEJAAAA|38000|10|33|20|AM|first|morning|| +38001|AAAAAAAACHEJAAAA|38001|10|33|21|AM|first|morning|| +38002|AAAAAAAADHEJAAAA|38002|10|33|22|AM|first|morning|| +38003|AAAAAAAAEHEJAAAA|38003|10|33|23|AM|first|morning|| +38004|AAAAAAAAFHEJAAAA|38004|10|33|24|AM|first|morning|| +38005|AAAAAAAAGHEJAAAA|38005|10|33|25|AM|first|morning|| +38006|AAAAAAAAHHEJAAAA|38006|10|33|26|AM|first|morning|| +38007|AAAAAAAAIHEJAAAA|38007|10|33|27|AM|first|morning|| +38008|AAAAAAAAJHEJAAAA|38008|10|33|28|AM|first|morning|| +38009|AAAAAAAAKHEJAAAA|38009|10|33|29|AM|first|morning|| +38010|AAAAAAAALHEJAAAA|38010|10|33|30|AM|first|morning|| +38011|AAAAAAAAMHEJAAAA|38011|10|33|31|AM|first|morning|| +38012|AAAAAAAANHEJAAAA|38012|10|33|32|AM|first|morning|| +38013|AAAAAAAAOHEJAAAA|38013|10|33|33|AM|first|morning|| +38014|AAAAAAAAPHEJAAAA|38014|10|33|34|AM|first|morning|| +38015|AAAAAAAAAIEJAAAA|38015|10|33|35|AM|first|morning|| +38016|AAAAAAAABIEJAAAA|38016|10|33|36|AM|first|morning|| +38017|AAAAAAAACIEJAAAA|38017|10|33|37|AM|first|morning|| +38018|AAAAAAAADIEJAAAA|38018|10|33|38|AM|first|morning|| +38019|AAAAAAAAEIEJAAAA|38019|10|33|39|AM|first|morning|| +38020|AAAAAAAAFIEJAAAA|38020|10|33|40|AM|first|morning|| +38021|AAAAAAAAGIEJAAAA|38021|10|33|41|AM|first|morning|| +38022|AAAAAAAAHIEJAAAA|38022|10|33|42|AM|first|morning|| +38023|AAAAAAAAIIEJAAAA|38023|10|33|43|AM|first|morning|| +38024|AAAAAAAAJIEJAAAA|38024|10|33|44|AM|first|morning|| +38025|AAAAAAAAKIEJAAAA|38025|10|33|45|AM|first|morning|| +38026|AAAAAAAALIEJAAAA|38026|10|33|46|AM|first|morning|| +38027|AAAAAAAAMIEJAAAA|38027|10|33|47|AM|first|morning|| +38028|AAAAAAAANIEJAAAA|38028|10|33|48|AM|first|morning|| +38029|AAAAAAAAOIEJAAAA|38029|10|33|49|AM|first|morning|| +38030|AAAAAAAAPIEJAAAA|38030|10|33|50|AM|first|morning|| +38031|AAAAAAAAAJEJAAAA|38031|10|33|51|AM|first|morning|| +38032|AAAAAAAABJEJAAAA|38032|10|33|52|AM|first|morning|| +38033|AAAAAAAACJEJAAAA|38033|10|33|53|AM|first|morning|| +38034|AAAAAAAADJEJAAAA|38034|10|33|54|AM|first|morning|| +38035|AAAAAAAAEJEJAAAA|38035|10|33|55|AM|first|morning|| +38036|AAAAAAAAFJEJAAAA|38036|10|33|56|AM|first|morning|| +38037|AAAAAAAAGJEJAAAA|38037|10|33|57|AM|first|morning|| +38038|AAAAAAAAHJEJAAAA|38038|10|33|58|AM|first|morning|| +38039|AAAAAAAAIJEJAAAA|38039|10|33|59|AM|first|morning|| +38040|AAAAAAAAJJEJAAAA|38040|10|34|0|AM|first|morning|| +38041|AAAAAAAAKJEJAAAA|38041|10|34|1|AM|first|morning|| +38042|AAAAAAAALJEJAAAA|38042|10|34|2|AM|first|morning|| +38043|AAAAAAAAMJEJAAAA|38043|10|34|3|AM|first|morning|| +38044|AAAAAAAANJEJAAAA|38044|10|34|4|AM|first|morning|| +38045|AAAAAAAAOJEJAAAA|38045|10|34|5|AM|first|morning|| +38046|AAAAAAAAPJEJAAAA|38046|10|34|6|AM|first|morning|| +38047|AAAAAAAAAKEJAAAA|38047|10|34|7|AM|first|morning|| +38048|AAAAAAAABKEJAAAA|38048|10|34|8|AM|first|morning|| +38049|AAAAAAAACKEJAAAA|38049|10|34|9|AM|first|morning|| +38050|AAAAAAAADKEJAAAA|38050|10|34|10|AM|first|morning|| +38051|AAAAAAAAEKEJAAAA|38051|10|34|11|AM|first|morning|| +38052|AAAAAAAAFKEJAAAA|38052|10|34|12|AM|first|morning|| +38053|AAAAAAAAGKEJAAAA|38053|10|34|13|AM|first|morning|| +38054|AAAAAAAAHKEJAAAA|38054|10|34|14|AM|first|morning|| +38055|AAAAAAAAIKEJAAAA|38055|10|34|15|AM|first|morning|| +38056|AAAAAAAAJKEJAAAA|38056|10|34|16|AM|first|morning|| +38057|AAAAAAAAKKEJAAAA|38057|10|34|17|AM|first|morning|| +38058|AAAAAAAALKEJAAAA|38058|10|34|18|AM|first|morning|| +38059|AAAAAAAAMKEJAAAA|38059|10|34|19|AM|first|morning|| +38060|AAAAAAAANKEJAAAA|38060|10|34|20|AM|first|morning|| +38061|AAAAAAAAOKEJAAAA|38061|10|34|21|AM|first|morning|| +38062|AAAAAAAAPKEJAAAA|38062|10|34|22|AM|first|morning|| +38063|AAAAAAAAALEJAAAA|38063|10|34|23|AM|first|morning|| +38064|AAAAAAAABLEJAAAA|38064|10|34|24|AM|first|morning|| +38065|AAAAAAAACLEJAAAA|38065|10|34|25|AM|first|morning|| +38066|AAAAAAAADLEJAAAA|38066|10|34|26|AM|first|morning|| +38067|AAAAAAAAELEJAAAA|38067|10|34|27|AM|first|morning|| +38068|AAAAAAAAFLEJAAAA|38068|10|34|28|AM|first|morning|| +38069|AAAAAAAAGLEJAAAA|38069|10|34|29|AM|first|morning|| +38070|AAAAAAAAHLEJAAAA|38070|10|34|30|AM|first|morning|| +38071|AAAAAAAAILEJAAAA|38071|10|34|31|AM|first|morning|| +38072|AAAAAAAAJLEJAAAA|38072|10|34|32|AM|first|morning|| +38073|AAAAAAAAKLEJAAAA|38073|10|34|33|AM|first|morning|| +38074|AAAAAAAALLEJAAAA|38074|10|34|34|AM|first|morning|| +38075|AAAAAAAAMLEJAAAA|38075|10|34|35|AM|first|morning|| +38076|AAAAAAAANLEJAAAA|38076|10|34|36|AM|first|morning|| +38077|AAAAAAAAOLEJAAAA|38077|10|34|37|AM|first|morning|| +38078|AAAAAAAAPLEJAAAA|38078|10|34|38|AM|first|morning|| +38079|AAAAAAAAAMEJAAAA|38079|10|34|39|AM|first|morning|| +38080|AAAAAAAABMEJAAAA|38080|10|34|40|AM|first|morning|| +38081|AAAAAAAACMEJAAAA|38081|10|34|41|AM|first|morning|| +38082|AAAAAAAADMEJAAAA|38082|10|34|42|AM|first|morning|| +38083|AAAAAAAAEMEJAAAA|38083|10|34|43|AM|first|morning|| +38084|AAAAAAAAFMEJAAAA|38084|10|34|44|AM|first|morning|| +38085|AAAAAAAAGMEJAAAA|38085|10|34|45|AM|first|morning|| +38086|AAAAAAAAHMEJAAAA|38086|10|34|46|AM|first|morning|| +38087|AAAAAAAAIMEJAAAA|38087|10|34|47|AM|first|morning|| +38088|AAAAAAAAJMEJAAAA|38088|10|34|48|AM|first|morning|| +38089|AAAAAAAAKMEJAAAA|38089|10|34|49|AM|first|morning|| +38090|AAAAAAAALMEJAAAA|38090|10|34|50|AM|first|morning|| +38091|AAAAAAAAMMEJAAAA|38091|10|34|51|AM|first|morning|| +38092|AAAAAAAANMEJAAAA|38092|10|34|52|AM|first|morning|| +38093|AAAAAAAAOMEJAAAA|38093|10|34|53|AM|first|morning|| +38094|AAAAAAAAPMEJAAAA|38094|10|34|54|AM|first|morning|| +38095|AAAAAAAAANEJAAAA|38095|10|34|55|AM|first|morning|| +38096|AAAAAAAABNEJAAAA|38096|10|34|56|AM|first|morning|| +38097|AAAAAAAACNEJAAAA|38097|10|34|57|AM|first|morning|| +38098|AAAAAAAADNEJAAAA|38098|10|34|58|AM|first|morning|| +38099|AAAAAAAAENEJAAAA|38099|10|34|59|AM|first|morning|| +38100|AAAAAAAAFNEJAAAA|38100|10|35|0|AM|first|morning|| +38101|AAAAAAAAGNEJAAAA|38101|10|35|1|AM|first|morning|| +38102|AAAAAAAAHNEJAAAA|38102|10|35|2|AM|first|morning|| +38103|AAAAAAAAINEJAAAA|38103|10|35|3|AM|first|morning|| +38104|AAAAAAAAJNEJAAAA|38104|10|35|4|AM|first|morning|| +38105|AAAAAAAAKNEJAAAA|38105|10|35|5|AM|first|morning|| +38106|AAAAAAAALNEJAAAA|38106|10|35|6|AM|first|morning|| +38107|AAAAAAAAMNEJAAAA|38107|10|35|7|AM|first|morning|| +38108|AAAAAAAANNEJAAAA|38108|10|35|8|AM|first|morning|| +38109|AAAAAAAAONEJAAAA|38109|10|35|9|AM|first|morning|| +38110|AAAAAAAAPNEJAAAA|38110|10|35|10|AM|first|morning|| +38111|AAAAAAAAAOEJAAAA|38111|10|35|11|AM|first|morning|| +38112|AAAAAAAABOEJAAAA|38112|10|35|12|AM|first|morning|| +38113|AAAAAAAACOEJAAAA|38113|10|35|13|AM|first|morning|| +38114|AAAAAAAADOEJAAAA|38114|10|35|14|AM|first|morning|| +38115|AAAAAAAAEOEJAAAA|38115|10|35|15|AM|first|morning|| +38116|AAAAAAAAFOEJAAAA|38116|10|35|16|AM|first|morning|| +38117|AAAAAAAAGOEJAAAA|38117|10|35|17|AM|first|morning|| +38118|AAAAAAAAHOEJAAAA|38118|10|35|18|AM|first|morning|| +38119|AAAAAAAAIOEJAAAA|38119|10|35|19|AM|first|morning|| +38120|AAAAAAAAJOEJAAAA|38120|10|35|20|AM|first|morning|| +38121|AAAAAAAAKOEJAAAA|38121|10|35|21|AM|first|morning|| +38122|AAAAAAAALOEJAAAA|38122|10|35|22|AM|first|morning|| +38123|AAAAAAAAMOEJAAAA|38123|10|35|23|AM|first|morning|| +38124|AAAAAAAANOEJAAAA|38124|10|35|24|AM|first|morning|| +38125|AAAAAAAAOOEJAAAA|38125|10|35|25|AM|first|morning|| +38126|AAAAAAAAPOEJAAAA|38126|10|35|26|AM|first|morning|| +38127|AAAAAAAAAPEJAAAA|38127|10|35|27|AM|first|morning|| +38128|AAAAAAAABPEJAAAA|38128|10|35|28|AM|first|morning|| +38129|AAAAAAAACPEJAAAA|38129|10|35|29|AM|first|morning|| +38130|AAAAAAAADPEJAAAA|38130|10|35|30|AM|first|morning|| +38131|AAAAAAAAEPEJAAAA|38131|10|35|31|AM|first|morning|| +38132|AAAAAAAAFPEJAAAA|38132|10|35|32|AM|first|morning|| +38133|AAAAAAAAGPEJAAAA|38133|10|35|33|AM|first|morning|| +38134|AAAAAAAAHPEJAAAA|38134|10|35|34|AM|first|morning|| +38135|AAAAAAAAIPEJAAAA|38135|10|35|35|AM|first|morning|| +38136|AAAAAAAAJPEJAAAA|38136|10|35|36|AM|first|morning|| +38137|AAAAAAAAKPEJAAAA|38137|10|35|37|AM|first|morning|| +38138|AAAAAAAALPEJAAAA|38138|10|35|38|AM|first|morning|| +38139|AAAAAAAAMPEJAAAA|38139|10|35|39|AM|first|morning|| +38140|AAAAAAAANPEJAAAA|38140|10|35|40|AM|first|morning|| +38141|AAAAAAAAOPEJAAAA|38141|10|35|41|AM|first|morning|| +38142|AAAAAAAAPPEJAAAA|38142|10|35|42|AM|first|morning|| +38143|AAAAAAAAAAFJAAAA|38143|10|35|43|AM|first|morning|| +38144|AAAAAAAABAFJAAAA|38144|10|35|44|AM|first|morning|| +38145|AAAAAAAACAFJAAAA|38145|10|35|45|AM|first|morning|| +38146|AAAAAAAADAFJAAAA|38146|10|35|46|AM|first|morning|| +38147|AAAAAAAAEAFJAAAA|38147|10|35|47|AM|first|morning|| +38148|AAAAAAAAFAFJAAAA|38148|10|35|48|AM|first|morning|| +38149|AAAAAAAAGAFJAAAA|38149|10|35|49|AM|first|morning|| +38150|AAAAAAAAHAFJAAAA|38150|10|35|50|AM|first|morning|| +38151|AAAAAAAAIAFJAAAA|38151|10|35|51|AM|first|morning|| +38152|AAAAAAAAJAFJAAAA|38152|10|35|52|AM|first|morning|| +38153|AAAAAAAAKAFJAAAA|38153|10|35|53|AM|first|morning|| +38154|AAAAAAAALAFJAAAA|38154|10|35|54|AM|first|morning|| +38155|AAAAAAAAMAFJAAAA|38155|10|35|55|AM|first|morning|| +38156|AAAAAAAANAFJAAAA|38156|10|35|56|AM|first|morning|| +38157|AAAAAAAAOAFJAAAA|38157|10|35|57|AM|first|morning|| +38158|AAAAAAAAPAFJAAAA|38158|10|35|58|AM|first|morning|| +38159|AAAAAAAAABFJAAAA|38159|10|35|59|AM|first|morning|| +38160|AAAAAAAABBFJAAAA|38160|10|36|0|AM|first|morning|| +38161|AAAAAAAACBFJAAAA|38161|10|36|1|AM|first|morning|| +38162|AAAAAAAADBFJAAAA|38162|10|36|2|AM|first|morning|| +38163|AAAAAAAAEBFJAAAA|38163|10|36|3|AM|first|morning|| +38164|AAAAAAAAFBFJAAAA|38164|10|36|4|AM|first|morning|| +38165|AAAAAAAAGBFJAAAA|38165|10|36|5|AM|first|morning|| +38166|AAAAAAAAHBFJAAAA|38166|10|36|6|AM|first|morning|| +38167|AAAAAAAAIBFJAAAA|38167|10|36|7|AM|first|morning|| +38168|AAAAAAAAJBFJAAAA|38168|10|36|8|AM|first|morning|| +38169|AAAAAAAAKBFJAAAA|38169|10|36|9|AM|first|morning|| +38170|AAAAAAAALBFJAAAA|38170|10|36|10|AM|first|morning|| +38171|AAAAAAAAMBFJAAAA|38171|10|36|11|AM|first|morning|| +38172|AAAAAAAANBFJAAAA|38172|10|36|12|AM|first|morning|| +38173|AAAAAAAAOBFJAAAA|38173|10|36|13|AM|first|morning|| +38174|AAAAAAAAPBFJAAAA|38174|10|36|14|AM|first|morning|| +38175|AAAAAAAAACFJAAAA|38175|10|36|15|AM|first|morning|| +38176|AAAAAAAABCFJAAAA|38176|10|36|16|AM|first|morning|| +38177|AAAAAAAACCFJAAAA|38177|10|36|17|AM|first|morning|| +38178|AAAAAAAADCFJAAAA|38178|10|36|18|AM|first|morning|| +38179|AAAAAAAAECFJAAAA|38179|10|36|19|AM|first|morning|| +38180|AAAAAAAAFCFJAAAA|38180|10|36|20|AM|first|morning|| +38181|AAAAAAAAGCFJAAAA|38181|10|36|21|AM|first|morning|| +38182|AAAAAAAAHCFJAAAA|38182|10|36|22|AM|first|morning|| +38183|AAAAAAAAICFJAAAA|38183|10|36|23|AM|first|morning|| +38184|AAAAAAAAJCFJAAAA|38184|10|36|24|AM|first|morning|| +38185|AAAAAAAAKCFJAAAA|38185|10|36|25|AM|first|morning|| +38186|AAAAAAAALCFJAAAA|38186|10|36|26|AM|first|morning|| +38187|AAAAAAAAMCFJAAAA|38187|10|36|27|AM|first|morning|| +38188|AAAAAAAANCFJAAAA|38188|10|36|28|AM|first|morning|| +38189|AAAAAAAAOCFJAAAA|38189|10|36|29|AM|first|morning|| +38190|AAAAAAAAPCFJAAAA|38190|10|36|30|AM|first|morning|| +38191|AAAAAAAAADFJAAAA|38191|10|36|31|AM|first|morning|| +38192|AAAAAAAABDFJAAAA|38192|10|36|32|AM|first|morning|| +38193|AAAAAAAACDFJAAAA|38193|10|36|33|AM|first|morning|| +38194|AAAAAAAADDFJAAAA|38194|10|36|34|AM|first|morning|| +38195|AAAAAAAAEDFJAAAA|38195|10|36|35|AM|first|morning|| +38196|AAAAAAAAFDFJAAAA|38196|10|36|36|AM|first|morning|| +38197|AAAAAAAAGDFJAAAA|38197|10|36|37|AM|first|morning|| +38198|AAAAAAAAHDFJAAAA|38198|10|36|38|AM|first|morning|| +38199|AAAAAAAAIDFJAAAA|38199|10|36|39|AM|first|morning|| +38200|AAAAAAAAJDFJAAAA|38200|10|36|40|AM|first|morning|| +38201|AAAAAAAAKDFJAAAA|38201|10|36|41|AM|first|morning|| +38202|AAAAAAAALDFJAAAA|38202|10|36|42|AM|first|morning|| +38203|AAAAAAAAMDFJAAAA|38203|10|36|43|AM|first|morning|| +38204|AAAAAAAANDFJAAAA|38204|10|36|44|AM|first|morning|| +38205|AAAAAAAAODFJAAAA|38205|10|36|45|AM|first|morning|| +38206|AAAAAAAAPDFJAAAA|38206|10|36|46|AM|first|morning|| +38207|AAAAAAAAAEFJAAAA|38207|10|36|47|AM|first|morning|| +38208|AAAAAAAABEFJAAAA|38208|10|36|48|AM|first|morning|| +38209|AAAAAAAACEFJAAAA|38209|10|36|49|AM|first|morning|| +38210|AAAAAAAADEFJAAAA|38210|10|36|50|AM|first|morning|| +38211|AAAAAAAAEEFJAAAA|38211|10|36|51|AM|first|morning|| +38212|AAAAAAAAFEFJAAAA|38212|10|36|52|AM|first|morning|| +38213|AAAAAAAAGEFJAAAA|38213|10|36|53|AM|first|morning|| +38214|AAAAAAAAHEFJAAAA|38214|10|36|54|AM|first|morning|| +38215|AAAAAAAAIEFJAAAA|38215|10|36|55|AM|first|morning|| +38216|AAAAAAAAJEFJAAAA|38216|10|36|56|AM|first|morning|| +38217|AAAAAAAAKEFJAAAA|38217|10|36|57|AM|first|morning|| +38218|AAAAAAAALEFJAAAA|38218|10|36|58|AM|first|morning|| +38219|AAAAAAAAMEFJAAAA|38219|10|36|59|AM|first|morning|| +38220|AAAAAAAANEFJAAAA|38220|10|37|0|AM|first|morning|| +38221|AAAAAAAAOEFJAAAA|38221|10|37|1|AM|first|morning|| +38222|AAAAAAAAPEFJAAAA|38222|10|37|2|AM|first|morning|| +38223|AAAAAAAAAFFJAAAA|38223|10|37|3|AM|first|morning|| +38224|AAAAAAAABFFJAAAA|38224|10|37|4|AM|first|morning|| +38225|AAAAAAAACFFJAAAA|38225|10|37|5|AM|first|morning|| +38226|AAAAAAAADFFJAAAA|38226|10|37|6|AM|first|morning|| +38227|AAAAAAAAEFFJAAAA|38227|10|37|7|AM|first|morning|| +38228|AAAAAAAAFFFJAAAA|38228|10|37|8|AM|first|morning|| +38229|AAAAAAAAGFFJAAAA|38229|10|37|9|AM|first|morning|| +38230|AAAAAAAAHFFJAAAA|38230|10|37|10|AM|first|morning|| +38231|AAAAAAAAIFFJAAAA|38231|10|37|11|AM|first|morning|| +38232|AAAAAAAAJFFJAAAA|38232|10|37|12|AM|first|morning|| +38233|AAAAAAAAKFFJAAAA|38233|10|37|13|AM|first|morning|| +38234|AAAAAAAALFFJAAAA|38234|10|37|14|AM|first|morning|| +38235|AAAAAAAAMFFJAAAA|38235|10|37|15|AM|first|morning|| +38236|AAAAAAAANFFJAAAA|38236|10|37|16|AM|first|morning|| +38237|AAAAAAAAOFFJAAAA|38237|10|37|17|AM|first|morning|| +38238|AAAAAAAAPFFJAAAA|38238|10|37|18|AM|first|morning|| +38239|AAAAAAAAAGFJAAAA|38239|10|37|19|AM|first|morning|| +38240|AAAAAAAABGFJAAAA|38240|10|37|20|AM|first|morning|| +38241|AAAAAAAACGFJAAAA|38241|10|37|21|AM|first|morning|| +38242|AAAAAAAADGFJAAAA|38242|10|37|22|AM|first|morning|| +38243|AAAAAAAAEGFJAAAA|38243|10|37|23|AM|first|morning|| +38244|AAAAAAAAFGFJAAAA|38244|10|37|24|AM|first|morning|| +38245|AAAAAAAAGGFJAAAA|38245|10|37|25|AM|first|morning|| +38246|AAAAAAAAHGFJAAAA|38246|10|37|26|AM|first|morning|| +38247|AAAAAAAAIGFJAAAA|38247|10|37|27|AM|first|morning|| +38248|AAAAAAAAJGFJAAAA|38248|10|37|28|AM|first|morning|| +38249|AAAAAAAAKGFJAAAA|38249|10|37|29|AM|first|morning|| +38250|AAAAAAAALGFJAAAA|38250|10|37|30|AM|first|morning|| +38251|AAAAAAAAMGFJAAAA|38251|10|37|31|AM|first|morning|| +38252|AAAAAAAANGFJAAAA|38252|10|37|32|AM|first|morning|| +38253|AAAAAAAAOGFJAAAA|38253|10|37|33|AM|first|morning|| +38254|AAAAAAAAPGFJAAAA|38254|10|37|34|AM|first|morning|| +38255|AAAAAAAAAHFJAAAA|38255|10|37|35|AM|first|morning|| +38256|AAAAAAAABHFJAAAA|38256|10|37|36|AM|first|morning|| +38257|AAAAAAAACHFJAAAA|38257|10|37|37|AM|first|morning|| +38258|AAAAAAAADHFJAAAA|38258|10|37|38|AM|first|morning|| +38259|AAAAAAAAEHFJAAAA|38259|10|37|39|AM|first|morning|| +38260|AAAAAAAAFHFJAAAA|38260|10|37|40|AM|first|morning|| +38261|AAAAAAAAGHFJAAAA|38261|10|37|41|AM|first|morning|| +38262|AAAAAAAAHHFJAAAA|38262|10|37|42|AM|first|morning|| +38263|AAAAAAAAIHFJAAAA|38263|10|37|43|AM|first|morning|| +38264|AAAAAAAAJHFJAAAA|38264|10|37|44|AM|first|morning|| +38265|AAAAAAAAKHFJAAAA|38265|10|37|45|AM|first|morning|| +38266|AAAAAAAALHFJAAAA|38266|10|37|46|AM|first|morning|| +38267|AAAAAAAAMHFJAAAA|38267|10|37|47|AM|first|morning|| +38268|AAAAAAAANHFJAAAA|38268|10|37|48|AM|first|morning|| +38269|AAAAAAAAOHFJAAAA|38269|10|37|49|AM|first|morning|| +38270|AAAAAAAAPHFJAAAA|38270|10|37|50|AM|first|morning|| +38271|AAAAAAAAAIFJAAAA|38271|10|37|51|AM|first|morning|| +38272|AAAAAAAABIFJAAAA|38272|10|37|52|AM|first|morning|| +38273|AAAAAAAACIFJAAAA|38273|10|37|53|AM|first|morning|| +38274|AAAAAAAADIFJAAAA|38274|10|37|54|AM|first|morning|| +38275|AAAAAAAAEIFJAAAA|38275|10|37|55|AM|first|morning|| +38276|AAAAAAAAFIFJAAAA|38276|10|37|56|AM|first|morning|| +38277|AAAAAAAAGIFJAAAA|38277|10|37|57|AM|first|morning|| +38278|AAAAAAAAHIFJAAAA|38278|10|37|58|AM|first|morning|| +38279|AAAAAAAAIIFJAAAA|38279|10|37|59|AM|first|morning|| +38280|AAAAAAAAJIFJAAAA|38280|10|38|0|AM|first|morning|| +38281|AAAAAAAAKIFJAAAA|38281|10|38|1|AM|first|morning|| +38282|AAAAAAAALIFJAAAA|38282|10|38|2|AM|first|morning|| +38283|AAAAAAAAMIFJAAAA|38283|10|38|3|AM|first|morning|| +38284|AAAAAAAANIFJAAAA|38284|10|38|4|AM|first|morning|| +38285|AAAAAAAAOIFJAAAA|38285|10|38|5|AM|first|morning|| +38286|AAAAAAAAPIFJAAAA|38286|10|38|6|AM|first|morning|| +38287|AAAAAAAAAJFJAAAA|38287|10|38|7|AM|first|morning|| +38288|AAAAAAAABJFJAAAA|38288|10|38|8|AM|first|morning|| +38289|AAAAAAAACJFJAAAA|38289|10|38|9|AM|first|morning|| +38290|AAAAAAAADJFJAAAA|38290|10|38|10|AM|first|morning|| +38291|AAAAAAAAEJFJAAAA|38291|10|38|11|AM|first|morning|| +38292|AAAAAAAAFJFJAAAA|38292|10|38|12|AM|first|morning|| +38293|AAAAAAAAGJFJAAAA|38293|10|38|13|AM|first|morning|| +38294|AAAAAAAAHJFJAAAA|38294|10|38|14|AM|first|morning|| +38295|AAAAAAAAIJFJAAAA|38295|10|38|15|AM|first|morning|| +38296|AAAAAAAAJJFJAAAA|38296|10|38|16|AM|first|morning|| +38297|AAAAAAAAKJFJAAAA|38297|10|38|17|AM|first|morning|| +38298|AAAAAAAALJFJAAAA|38298|10|38|18|AM|first|morning|| +38299|AAAAAAAAMJFJAAAA|38299|10|38|19|AM|first|morning|| +38300|AAAAAAAANJFJAAAA|38300|10|38|20|AM|first|morning|| +38301|AAAAAAAAOJFJAAAA|38301|10|38|21|AM|first|morning|| +38302|AAAAAAAAPJFJAAAA|38302|10|38|22|AM|first|morning|| +38303|AAAAAAAAAKFJAAAA|38303|10|38|23|AM|first|morning|| +38304|AAAAAAAABKFJAAAA|38304|10|38|24|AM|first|morning|| +38305|AAAAAAAACKFJAAAA|38305|10|38|25|AM|first|morning|| +38306|AAAAAAAADKFJAAAA|38306|10|38|26|AM|first|morning|| +38307|AAAAAAAAEKFJAAAA|38307|10|38|27|AM|first|morning|| +38308|AAAAAAAAFKFJAAAA|38308|10|38|28|AM|first|morning|| +38309|AAAAAAAAGKFJAAAA|38309|10|38|29|AM|first|morning|| +38310|AAAAAAAAHKFJAAAA|38310|10|38|30|AM|first|morning|| +38311|AAAAAAAAIKFJAAAA|38311|10|38|31|AM|first|morning|| +38312|AAAAAAAAJKFJAAAA|38312|10|38|32|AM|first|morning|| +38313|AAAAAAAAKKFJAAAA|38313|10|38|33|AM|first|morning|| +38314|AAAAAAAALKFJAAAA|38314|10|38|34|AM|first|morning|| +38315|AAAAAAAAMKFJAAAA|38315|10|38|35|AM|first|morning|| +38316|AAAAAAAANKFJAAAA|38316|10|38|36|AM|first|morning|| +38317|AAAAAAAAOKFJAAAA|38317|10|38|37|AM|first|morning|| +38318|AAAAAAAAPKFJAAAA|38318|10|38|38|AM|first|morning|| +38319|AAAAAAAAALFJAAAA|38319|10|38|39|AM|first|morning|| +38320|AAAAAAAABLFJAAAA|38320|10|38|40|AM|first|morning|| +38321|AAAAAAAACLFJAAAA|38321|10|38|41|AM|first|morning|| +38322|AAAAAAAADLFJAAAA|38322|10|38|42|AM|first|morning|| +38323|AAAAAAAAELFJAAAA|38323|10|38|43|AM|first|morning|| +38324|AAAAAAAAFLFJAAAA|38324|10|38|44|AM|first|morning|| +38325|AAAAAAAAGLFJAAAA|38325|10|38|45|AM|first|morning|| +38326|AAAAAAAAHLFJAAAA|38326|10|38|46|AM|first|morning|| +38327|AAAAAAAAILFJAAAA|38327|10|38|47|AM|first|morning|| +38328|AAAAAAAAJLFJAAAA|38328|10|38|48|AM|first|morning|| +38329|AAAAAAAAKLFJAAAA|38329|10|38|49|AM|first|morning|| +38330|AAAAAAAALLFJAAAA|38330|10|38|50|AM|first|morning|| +38331|AAAAAAAAMLFJAAAA|38331|10|38|51|AM|first|morning|| +38332|AAAAAAAANLFJAAAA|38332|10|38|52|AM|first|morning|| +38333|AAAAAAAAOLFJAAAA|38333|10|38|53|AM|first|morning|| +38334|AAAAAAAAPLFJAAAA|38334|10|38|54|AM|first|morning|| +38335|AAAAAAAAAMFJAAAA|38335|10|38|55|AM|first|morning|| +38336|AAAAAAAABMFJAAAA|38336|10|38|56|AM|first|morning|| +38337|AAAAAAAACMFJAAAA|38337|10|38|57|AM|first|morning|| +38338|AAAAAAAADMFJAAAA|38338|10|38|58|AM|first|morning|| +38339|AAAAAAAAEMFJAAAA|38339|10|38|59|AM|first|morning|| +38340|AAAAAAAAFMFJAAAA|38340|10|39|0|AM|first|morning|| +38341|AAAAAAAAGMFJAAAA|38341|10|39|1|AM|first|morning|| +38342|AAAAAAAAHMFJAAAA|38342|10|39|2|AM|first|morning|| +38343|AAAAAAAAIMFJAAAA|38343|10|39|3|AM|first|morning|| +38344|AAAAAAAAJMFJAAAA|38344|10|39|4|AM|first|morning|| +38345|AAAAAAAAKMFJAAAA|38345|10|39|5|AM|first|morning|| +38346|AAAAAAAALMFJAAAA|38346|10|39|6|AM|first|morning|| +38347|AAAAAAAAMMFJAAAA|38347|10|39|7|AM|first|morning|| +38348|AAAAAAAANMFJAAAA|38348|10|39|8|AM|first|morning|| +38349|AAAAAAAAOMFJAAAA|38349|10|39|9|AM|first|morning|| +38350|AAAAAAAAPMFJAAAA|38350|10|39|10|AM|first|morning|| +38351|AAAAAAAAANFJAAAA|38351|10|39|11|AM|first|morning|| +38352|AAAAAAAABNFJAAAA|38352|10|39|12|AM|first|morning|| +38353|AAAAAAAACNFJAAAA|38353|10|39|13|AM|first|morning|| +38354|AAAAAAAADNFJAAAA|38354|10|39|14|AM|first|morning|| +38355|AAAAAAAAENFJAAAA|38355|10|39|15|AM|first|morning|| +38356|AAAAAAAAFNFJAAAA|38356|10|39|16|AM|first|morning|| +38357|AAAAAAAAGNFJAAAA|38357|10|39|17|AM|first|morning|| +38358|AAAAAAAAHNFJAAAA|38358|10|39|18|AM|first|morning|| +38359|AAAAAAAAINFJAAAA|38359|10|39|19|AM|first|morning|| +38360|AAAAAAAAJNFJAAAA|38360|10|39|20|AM|first|morning|| +38361|AAAAAAAAKNFJAAAA|38361|10|39|21|AM|first|morning|| +38362|AAAAAAAALNFJAAAA|38362|10|39|22|AM|first|morning|| +38363|AAAAAAAAMNFJAAAA|38363|10|39|23|AM|first|morning|| +38364|AAAAAAAANNFJAAAA|38364|10|39|24|AM|first|morning|| +38365|AAAAAAAAONFJAAAA|38365|10|39|25|AM|first|morning|| +38366|AAAAAAAAPNFJAAAA|38366|10|39|26|AM|first|morning|| +38367|AAAAAAAAAOFJAAAA|38367|10|39|27|AM|first|morning|| +38368|AAAAAAAABOFJAAAA|38368|10|39|28|AM|first|morning|| +38369|AAAAAAAACOFJAAAA|38369|10|39|29|AM|first|morning|| +38370|AAAAAAAADOFJAAAA|38370|10|39|30|AM|first|morning|| +38371|AAAAAAAAEOFJAAAA|38371|10|39|31|AM|first|morning|| +38372|AAAAAAAAFOFJAAAA|38372|10|39|32|AM|first|morning|| +38373|AAAAAAAAGOFJAAAA|38373|10|39|33|AM|first|morning|| +38374|AAAAAAAAHOFJAAAA|38374|10|39|34|AM|first|morning|| +38375|AAAAAAAAIOFJAAAA|38375|10|39|35|AM|first|morning|| +38376|AAAAAAAAJOFJAAAA|38376|10|39|36|AM|first|morning|| +38377|AAAAAAAAKOFJAAAA|38377|10|39|37|AM|first|morning|| +38378|AAAAAAAALOFJAAAA|38378|10|39|38|AM|first|morning|| +38379|AAAAAAAAMOFJAAAA|38379|10|39|39|AM|first|morning|| +38380|AAAAAAAANOFJAAAA|38380|10|39|40|AM|first|morning|| +38381|AAAAAAAAOOFJAAAA|38381|10|39|41|AM|first|morning|| +38382|AAAAAAAAPOFJAAAA|38382|10|39|42|AM|first|morning|| +38383|AAAAAAAAAPFJAAAA|38383|10|39|43|AM|first|morning|| +38384|AAAAAAAABPFJAAAA|38384|10|39|44|AM|first|morning|| +38385|AAAAAAAACPFJAAAA|38385|10|39|45|AM|first|morning|| +38386|AAAAAAAADPFJAAAA|38386|10|39|46|AM|first|morning|| +38387|AAAAAAAAEPFJAAAA|38387|10|39|47|AM|first|morning|| +38388|AAAAAAAAFPFJAAAA|38388|10|39|48|AM|first|morning|| +38389|AAAAAAAAGPFJAAAA|38389|10|39|49|AM|first|morning|| +38390|AAAAAAAAHPFJAAAA|38390|10|39|50|AM|first|morning|| +38391|AAAAAAAAIPFJAAAA|38391|10|39|51|AM|first|morning|| +38392|AAAAAAAAJPFJAAAA|38392|10|39|52|AM|first|morning|| +38393|AAAAAAAAKPFJAAAA|38393|10|39|53|AM|first|morning|| +38394|AAAAAAAALPFJAAAA|38394|10|39|54|AM|first|morning|| +38395|AAAAAAAAMPFJAAAA|38395|10|39|55|AM|first|morning|| +38396|AAAAAAAANPFJAAAA|38396|10|39|56|AM|first|morning|| +38397|AAAAAAAAOPFJAAAA|38397|10|39|57|AM|first|morning|| +38398|AAAAAAAAPPFJAAAA|38398|10|39|58|AM|first|morning|| +38399|AAAAAAAAAAGJAAAA|38399|10|39|59|AM|first|morning|| +38400|AAAAAAAABAGJAAAA|38400|10|40|0|AM|first|morning|| +38401|AAAAAAAACAGJAAAA|38401|10|40|1|AM|first|morning|| +38402|AAAAAAAADAGJAAAA|38402|10|40|2|AM|first|morning|| +38403|AAAAAAAAEAGJAAAA|38403|10|40|3|AM|first|morning|| +38404|AAAAAAAAFAGJAAAA|38404|10|40|4|AM|first|morning|| +38405|AAAAAAAAGAGJAAAA|38405|10|40|5|AM|first|morning|| +38406|AAAAAAAAHAGJAAAA|38406|10|40|6|AM|first|morning|| +38407|AAAAAAAAIAGJAAAA|38407|10|40|7|AM|first|morning|| +38408|AAAAAAAAJAGJAAAA|38408|10|40|8|AM|first|morning|| +38409|AAAAAAAAKAGJAAAA|38409|10|40|9|AM|first|morning|| +38410|AAAAAAAALAGJAAAA|38410|10|40|10|AM|first|morning|| +38411|AAAAAAAAMAGJAAAA|38411|10|40|11|AM|first|morning|| +38412|AAAAAAAANAGJAAAA|38412|10|40|12|AM|first|morning|| +38413|AAAAAAAAOAGJAAAA|38413|10|40|13|AM|first|morning|| +38414|AAAAAAAAPAGJAAAA|38414|10|40|14|AM|first|morning|| +38415|AAAAAAAAABGJAAAA|38415|10|40|15|AM|first|morning|| +38416|AAAAAAAABBGJAAAA|38416|10|40|16|AM|first|morning|| +38417|AAAAAAAACBGJAAAA|38417|10|40|17|AM|first|morning|| +38418|AAAAAAAADBGJAAAA|38418|10|40|18|AM|first|morning|| +38419|AAAAAAAAEBGJAAAA|38419|10|40|19|AM|first|morning|| +38420|AAAAAAAAFBGJAAAA|38420|10|40|20|AM|first|morning|| +38421|AAAAAAAAGBGJAAAA|38421|10|40|21|AM|first|morning|| +38422|AAAAAAAAHBGJAAAA|38422|10|40|22|AM|first|morning|| +38423|AAAAAAAAIBGJAAAA|38423|10|40|23|AM|first|morning|| +38424|AAAAAAAAJBGJAAAA|38424|10|40|24|AM|first|morning|| +38425|AAAAAAAAKBGJAAAA|38425|10|40|25|AM|first|morning|| +38426|AAAAAAAALBGJAAAA|38426|10|40|26|AM|first|morning|| +38427|AAAAAAAAMBGJAAAA|38427|10|40|27|AM|first|morning|| +38428|AAAAAAAANBGJAAAA|38428|10|40|28|AM|first|morning|| +38429|AAAAAAAAOBGJAAAA|38429|10|40|29|AM|first|morning|| +38430|AAAAAAAAPBGJAAAA|38430|10|40|30|AM|first|morning|| +38431|AAAAAAAAACGJAAAA|38431|10|40|31|AM|first|morning|| +38432|AAAAAAAABCGJAAAA|38432|10|40|32|AM|first|morning|| +38433|AAAAAAAACCGJAAAA|38433|10|40|33|AM|first|morning|| +38434|AAAAAAAADCGJAAAA|38434|10|40|34|AM|first|morning|| +38435|AAAAAAAAECGJAAAA|38435|10|40|35|AM|first|morning|| +38436|AAAAAAAAFCGJAAAA|38436|10|40|36|AM|first|morning|| +38437|AAAAAAAAGCGJAAAA|38437|10|40|37|AM|first|morning|| +38438|AAAAAAAAHCGJAAAA|38438|10|40|38|AM|first|morning|| +38439|AAAAAAAAICGJAAAA|38439|10|40|39|AM|first|morning|| +38440|AAAAAAAAJCGJAAAA|38440|10|40|40|AM|first|morning|| +38441|AAAAAAAAKCGJAAAA|38441|10|40|41|AM|first|morning|| +38442|AAAAAAAALCGJAAAA|38442|10|40|42|AM|first|morning|| +38443|AAAAAAAAMCGJAAAA|38443|10|40|43|AM|first|morning|| +38444|AAAAAAAANCGJAAAA|38444|10|40|44|AM|first|morning|| +38445|AAAAAAAAOCGJAAAA|38445|10|40|45|AM|first|morning|| +38446|AAAAAAAAPCGJAAAA|38446|10|40|46|AM|first|morning|| +38447|AAAAAAAAADGJAAAA|38447|10|40|47|AM|first|morning|| +38448|AAAAAAAABDGJAAAA|38448|10|40|48|AM|first|morning|| +38449|AAAAAAAACDGJAAAA|38449|10|40|49|AM|first|morning|| +38450|AAAAAAAADDGJAAAA|38450|10|40|50|AM|first|morning|| +38451|AAAAAAAAEDGJAAAA|38451|10|40|51|AM|first|morning|| +38452|AAAAAAAAFDGJAAAA|38452|10|40|52|AM|first|morning|| +38453|AAAAAAAAGDGJAAAA|38453|10|40|53|AM|first|morning|| +38454|AAAAAAAAHDGJAAAA|38454|10|40|54|AM|first|morning|| +38455|AAAAAAAAIDGJAAAA|38455|10|40|55|AM|first|morning|| +38456|AAAAAAAAJDGJAAAA|38456|10|40|56|AM|first|morning|| +38457|AAAAAAAAKDGJAAAA|38457|10|40|57|AM|first|morning|| +38458|AAAAAAAALDGJAAAA|38458|10|40|58|AM|first|morning|| +38459|AAAAAAAAMDGJAAAA|38459|10|40|59|AM|first|morning|| +38460|AAAAAAAANDGJAAAA|38460|10|41|0|AM|first|morning|| +38461|AAAAAAAAODGJAAAA|38461|10|41|1|AM|first|morning|| +38462|AAAAAAAAPDGJAAAA|38462|10|41|2|AM|first|morning|| +38463|AAAAAAAAAEGJAAAA|38463|10|41|3|AM|first|morning|| +38464|AAAAAAAABEGJAAAA|38464|10|41|4|AM|first|morning|| +38465|AAAAAAAACEGJAAAA|38465|10|41|5|AM|first|morning|| +38466|AAAAAAAADEGJAAAA|38466|10|41|6|AM|first|morning|| +38467|AAAAAAAAEEGJAAAA|38467|10|41|7|AM|first|morning|| +38468|AAAAAAAAFEGJAAAA|38468|10|41|8|AM|first|morning|| +38469|AAAAAAAAGEGJAAAA|38469|10|41|9|AM|first|morning|| +38470|AAAAAAAAHEGJAAAA|38470|10|41|10|AM|first|morning|| +38471|AAAAAAAAIEGJAAAA|38471|10|41|11|AM|first|morning|| +38472|AAAAAAAAJEGJAAAA|38472|10|41|12|AM|first|morning|| +38473|AAAAAAAAKEGJAAAA|38473|10|41|13|AM|first|morning|| +38474|AAAAAAAALEGJAAAA|38474|10|41|14|AM|first|morning|| +38475|AAAAAAAAMEGJAAAA|38475|10|41|15|AM|first|morning|| +38476|AAAAAAAANEGJAAAA|38476|10|41|16|AM|first|morning|| +38477|AAAAAAAAOEGJAAAA|38477|10|41|17|AM|first|morning|| +38478|AAAAAAAAPEGJAAAA|38478|10|41|18|AM|first|morning|| +38479|AAAAAAAAAFGJAAAA|38479|10|41|19|AM|first|morning|| +38480|AAAAAAAABFGJAAAA|38480|10|41|20|AM|first|morning|| +38481|AAAAAAAACFGJAAAA|38481|10|41|21|AM|first|morning|| +38482|AAAAAAAADFGJAAAA|38482|10|41|22|AM|first|morning|| +38483|AAAAAAAAEFGJAAAA|38483|10|41|23|AM|first|morning|| +38484|AAAAAAAAFFGJAAAA|38484|10|41|24|AM|first|morning|| +38485|AAAAAAAAGFGJAAAA|38485|10|41|25|AM|first|morning|| +38486|AAAAAAAAHFGJAAAA|38486|10|41|26|AM|first|morning|| +38487|AAAAAAAAIFGJAAAA|38487|10|41|27|AM|first|morning|| +38488|AAAAAAAAJFGJAAAA|38488|10|41|28|AM|first|morning|| +38489|AAAAAAAAKFGJAAAA|38489|10|41|29|AM|first|morning|| +38490|AAAAAAAALFGJAAAA|38490|10|41|30|AM|first|morning|| +38491|AAAAAAAAMFGJAAAA|38491|10|41|31|AM|first|morning|| +38492|AAAAAAAANFGJAAAA|38492|10|41|32|AM|first|morning|| +38493|AAAAAAAAOFGJAAAA|38493|10|41|33|AM|first|morning|| +38494|AAAAAAAAPFGJAAAA|38494|10|41|34|AM|first|morning|| +38495|AAAAAAAAAGGJAAAA|38495|10|41|35|AM|first|morning|| +38496|AAAAAAAABGGJAAAA|38496|10|41|36|AM|first|morning|| +38497|AAAAAAAACGGJAAAA|38497|10|41|37|AM|first|morning|| +38498|AAAAAAAADGGJAAAA|38498|10|41|38|AM|first|morning|| +38499|AAAAAAAAEGGJAAAA|38499|10|41|39|AM|first|morning|| +38500|AAAAAAAAFGGJAAAA|38500|10|41|40|AM|first|morning|| +38501|AAAAAAAAGGGJAAAA|38501|10|41|41|AM|first|morning|| +38502|AAAAAAAAHGGJAAAA|38502|10|41|42|AM|first|morning|| +38503|AAAAAAAAIGGJAAAA|38503|10|41|43|AM|first|morning|| +38504|AAAAAAAAJGGJAAAA|38504|10|41|44|AM|first|morning|| +38505|AAAAAAAAKGGJAAAA|38505|10|41|45|AM|first|morning|| +38506|AAAAAAAALGGJAAAA|38506|10|41|46|AM|first|morning|| +38507|AAAAAAAAMGGJAAAA|38507|10|41|47|AM|first|morning|| +38508|AAAAAAAANGGJAAAA|38508|10|41|48|AM|first|morning|| +38509|AAAAAAAAOGGJAAAA|38509|10|41|49|AM|first|morning|| +38510|AAAAAAAAPGGJAAAA|38510|10|41|50|AM|first|morning|| +38511|AAAAAAAAAHGJAAAA|38511|10|41|51|AM|first|morning|| +38512|AAAAAAAABHGJAAAA|38512|10|41|52|AM|first|morning|| +38513|AAAAAAAACHGJAAAA|38513|10|41|53|AM|first|morning|| +38514|AAAAAAAADHGJAAAA|38514|10|41|54|AM|first|morning|| +38515|AAAAAAAAEHGJAAAA|38515|10|41|55|AM|first|morning|| +38516|AAAAAAAAFHGJAAAA|38516|10|41|56|AM|first|morning|| +38517|AAAAAAAAGHGJAAAA|38517|10|41|57|AM|first|morning|| +38518|AAAAAAAAHHGJAAAA|38518|10|41|58|AM|first|morning|| +38519|AAAAAAAAIHGJAAAA|38519|10|41|59|AM|first|morning|| +38520|AAAAAAAAJHGJAAAA|38520|10|42|0|AM|first|morning|| +38521|AAAAAAAAKHGJAAAA|38521|10|42|1|AM|first|morning|| +38522|AAAAAAAALHGJAAAA|38522|10|42|2|AM|first|morning|| +38523|AAAAAAAAMHGJAAAA|38523|10|42|3|AM|first|morning|| +38524|AAAAAAAANHGJAAAA|38524|10|42|4|AM|first|morning|| +38525|AAAAAAAAOHGJAAAA|38525|10|42|5|AM|first|morning|| +38526|AAAAAAAAPHGJAAAA|38526|10|42|6|AM|first|morning|| +38527|AAAAAAAAAIGJAAAA|38527|10|42|7|AM|first|morning|| +38528|AAAAAAAABIGJAAAA|38528|10|42|8|AM|first|morning|| +38529|AAAAAAAACIGJAAAA|38529|10|42|9|AM|first|morning|| +38530|AAAAAAAADIGJAAAA|38530|10|42|10|AM|first|morning|| +38531|AAAAAAAAEIGJAAAA|38531|10|42|11|AM|first|morning|| +38532|AAAAAAAAFIGJAAAA|38532|10|42|12|AM|first|morning|| +38533|AAAAAAAAGIGJAAAA|38533|10|42|13|AM|first|morning|| +38534|AAAAAAAAHIGJAAAA|38534|10|42|14|AM|first|morning|| +38535|AAAAAAAAIIGJAAAA|38535|10|42|15|AM|first|morning|| +38536|AAAAAAAAJIGJAAAA|38536|10|42|16|AM|first|morning|| +38537|AAAAAAAAKIGJAAAA|38537|10|42|17|AM|first|morning|| +38538|AAAAAAAALIGJAAAA|38538|10|42|18|AM|first|morning|| +38539|AAAAAAAAMIGJAAAA|38539|10|42|19|AM|first|morning|| +38540|AAAAAAAANIGJAAAA|38540|10|42|20|AM|first|morning|| +38541|AAAAAAAAOIGJAAAA|38541|10|42|21|AM|first|morning|| +38542|AAAAAAAAPIGJAAAA|38542|10|42|22|AM|first|morning|| +38543|AAAAAAAAAJGJAAAA|38543|10|42|23|AM|first|morning|| +38544|AAAAAAAABJGJAAAA|38544|10|42|24|AM|first|morning|| +38545|AAAAAAAACJGJAAAA|38545|10|42|25|AM|first|morning|| +38546|AAAAAAAADJGJAAAA|38546|10|42|26|AM|first|morning|| +38547|AAAAAAAAEJGJAAAA|38547|10|42|27|AM|first|morning|| +38548|AAAAAAAAFJGJAAAA|38548|10|42|28|AM|first|morning|| +38549|AAAAAAAAGJGJAAAA|38549|10|42|29|AM|first|morning|| +38550|AAAAAAAAHJGJAAAA|38550|10|42|30|AM|first|morning|| +38551|AAAAAAAAIJGJAAAA|38551|10|42|31|AM|first|morning|| +38552|AAAAAAAAJJGJAAAA|38552|10|42|32|AM|first|morning|| +38553|AAAAAAAAKJGJAAAA|38553|10|42|33|AM|first|morning|| +38554|AAAAAAAALJGJAAAA|38554|10|42|34|AM|first|morning|| +38555|AAAAAAAAMJGJAAAA|38555|10|42|35|AM|first|morning|| +38556|AAAAAAAANJGJAAAA|38556|10|42|36|AM|first|morning|| +38557|AAAAAAAAOJGJAAAA|38557|10|42|37|AM|first|morning|| +38558|AAAAAAAAPJGJAAAA|38558|10|42|38|AM|first|morning|| +38559|AAAAAAAAAKGJAAAA|38559|10|42|39|AM|first|morning|| +38560|AAAAAAAABKGJAAAA|38560|10|42|40|AM|first|morning|| +38561|AAAAAAAACKGJAAAA|38561|10|42|41|AM|first|morning|| +38562|AAAAAAAADKGJAAAA|38562|10|42|42|AM|first|morning|| +38563|AAAAAAAAEKGJAAAA|38563|10|42|43|AM|first|morning|| +38564|AAAAAAAAFKGJAAAA|38564|10|42|44|AM|first|morning|| +38565|AAAAAAAAGKGJAAAA|38565|10|42|45|AM|first|morning|| +38566|AAAAAAAAHKGJAAAA|38566|10|42|46|AM|first|morning|| +38567|AAAAAAAAIKGJAAAA|38567|10|42|47|AM|first|morning|| +38568|AAAAAAAAJKGJAAAA|38568|10|42|48|AM|first|morning|| +38569|AAAAAAAAKKGJAAAA|38569|10|42|49|AM|first|morning|| +38570|AAAAAAAALKGJAAAA|38570|10|42|50|AM|first|morning|| +38571|AAAAAAAAMKGJAAAA|38571|10|42|51|AM|first|morning|| +38572|AAAAAAAANKGJAAAA|38572|10|42|52|AM|first|morning|| +38573|AAAAAAAAOKGJAAAA|38573|10|42|53|AM|first|morning|| +38574|AAAAAAAAPKGJAAAA|38574|10|42|54|AM|first|morning|| +38575|AAAAAAAAALGJAAAA|38575|10|42|55|AM|first|morning|| +38576|AAAAAAAABLGJAAAA|38576|10|42|56|AM|first|morning|| +38577|AAAAAAAACLGJAAAA|38577|10|42|57|AM|first|morning|| +38578|AAAAAAAADLGJAAAA|38578|10|42|58|AM|first|morning|| +38579|AAAAAAAAELGJAAAA|38579|10|42|59|AM|first|morning|| +38580|AAAAAAAAFLGJAAAA|38580|10|43|0|AM|first|morning|| +38581|AAAAAAAAGLGJAAAA|38581|10|43|1|AM|first|morning|| +38582|AAAAAAAAHLGJAAAA|38582|10|43|2|AM|first|morning|| +38583|AAAAAAAAILGJAAAA|38583|10|43|3|AM|first|morning|| +38584|AAAAAAAAJLGJAAAA|38584|10|43|4|AM|first|morning|| +38585|AAAAAAAAKLGJAAAA|38585|10|43|5|AM|first|morning|| +38586|AAAAAAAALLGJAAAA|38586|10|43|6|AM|first|morning|| +38587|AAAAAAAAMLGJAAAA|38587|10|43|7|AM|first|morning|| +38588|AAAAAAAANLGJAAAA|38588|10|43|8|AM|first|morning|| +38589|AAAAAAAAOLGJAAAA|38589|10|43|9|AM|first|morning|| +38590|AAAAAAAAPLGJAAAA|38590|10|43|10|AM|first|morning|| +38591|AAAAAAAAAMGJAAAA|38591|10|43|11|AM|first|morning|| +38592|AAAAAAAABMGJAAAA|38592|10|43|12|AM|first|morning|| +38593|AAAAAAAACMGJAAAA|38593|10|43|13|AM|first|morning|| +38594|AAAAAAAADMGJAAAA|38594|10|43|14|AM|first|morning|| +38595|AAAAAAAAEMGJAAAA|38595|10|43|15|AM|first|morning|| +38596|AAAAAAAAFMGJAAAA|38596|10|43|16|AM|first|morning|| +38597|AAAAAAAAGMGJAAAA|38597|10|43|17|AM|first|morning|| +38598|AAAAAAAAHMGJAAAA|38598|10|43|18|AM|first|morning|| +38599|AAAAAAAAIMGJAAAA|38599|10|43|19|AM|first|morning|| +38600|AAAAAAAAJMGJAAAA|38600|10|43|20|AM|first|morning|| +38601|AAAAAAAAKMGJAAAA|38601|10|43|21|AM|first|morning|| +38602|AAAAAAAALMGJAAAA|38602|10|43|22|AM|first|morning|| +38603|AAAAAAAAMMGJAAAA|38603|10|43|23|AM|first|morning|| +38604|AAAAAAAANMGJAAAA|38604|10|43|24|AM|first|morning|| +38605|AAAAAAAAOMGJAAAA|38605|10|43|25|AM|first|morning|| +38606|AAAAAAAAPMGJAAAA|38606|10|43|26|AM|first|morning|| +38607|AAAAAAAAANGJAAAA|38607|10|43|27|AM|first|morning|| +38608|AAAAAAAABNGJAAAA|38608|10|43|28|AM|first|morning|| +38609|AAAAAAAACNGJAAAA|38609|10|43|29|AM|first|morning|| +38610|AAAAAAAADNGJAAAA|38610|10|43|30|AM|first|morning|| +38611|AAAAAAAAENGJAAAA|38611|10|43|31|AM|first|morning|| +38612|AAAAAAAAFNGJAAAA|38612|10|43|32|AM|first|morning|| +38613|AAAAAAAAGNGJAAAA|38613|10|43|33|AM|first|morning|| +38614|AAAAAAAAHNGJAAAA|38614|10|43|34|AM|first|morning|| +38615|AAAAAAAAINGJAAAA|38615|10|43|35|AM|first|morning|| +38616|AAAAAAAAJNGJAAAA|38616|10|43|36|AM|first|morning|| +38617|AAAAAAAAKNGJAAAA|38617|10|43|37|AM|first|morning|| +38618|AAAAAAAALNGJAAAA|38618|10|43|38|AM|first|morning|| +38619|AAAAAAAAMNGJAAAA|38619|10|43|39|AM|first|morning|| +38620|AAAAAAAANNGJAAAA|38620|10|43|40|AM|first|morning|| +38621|AAAAAAAAONGJAAAA|38621|10|43|41|AM|first|morning|| +38622|AAAAAAAAPNGJAAAA|38622|10|43|42|AM|first|morning|| +38623|AAAAAAAAAOGJAAAA|38623|10|43|43|AM|first|morning|| +38624|AAAAAAAABOGJAAAA|38624|10|43|44|AM|first|morning|| +38625|AAAAAAAACOGJAAAA|38625|10|43|45|AM|first|morning|| +38626|AAAAAAAADOGJAAAA|38626|10|43|46|AM|first|morning|| +38627|AAAAAAAAEOGJAAAA|38627|10|43|47|AM|first|morning|| +38628|AAAAAAAAFOGJAAAA|38628|10|43|48|AM|first|morning|| +38629|AAAAAAAAGOGJAAAA|38629|10|43|49|AM|first|morning|| +38630|AAAAAAAAHOGJAAAA|38630|10|43|50|AM|first|morning|| +38631|AAAAAAAAIOGJAAAA|38631|10|43|51|AM|first|morning|| +38632|AAAAAAAAJOGJAAAA|38632|10|43|52|AM|first|morning|| +38633|AAAAAAAAKOGJAAAA|38633|10|43|53|AM|first|morning|| +38634|AAAAAAAALOGJAAAA|38634|10|43|54|AM|first|morning|| +38635|AAAAAAAAMOGJAAAA|38635|10|43|55|AM|first|morning|| +38636|AAAAAAAANOGJAAAA|38636|10|43|56|AM|first|morning|| +38637|AAAAAAAAOOGJAAAA|38637|10|43|57|AM|first|morning|| +38638|AAAAAAAAPOGJAAAA|38638|10|43|58|AM|first|morning|| +38639|AAAAAAAAAPGJAAAA|38639|10|43|59|AM|first|morning|| +38640|AAAAAAAABPGJAAAA|38640|10|44|0|AM|first|morning|| +38641|AAAAAAAACPGJAAAA|38641|10|44|1|AM|first|morning|| +38642|AAAAAAAADPGJAAAA|38642|10|44|2|AM|first|morning|| +38643|AAAAAAAAEPGJAAAA|38643|10|44|3|AM|first|morning|| +38644|AAAAAAAAFPGJAAAA|38644|10|44|4|AM|first|morning|| +38645|AAAAAAAAGPGJAAAA|38645|10|44|5|AM|first|morning|| +38646|AAAAAAAAHPGJAAAA|38646|10|44|6|AM|first|morning|| +38647|AAAAAAAAIPGJAAAA|38647|10|44|7|AM|first|morning|| +38648|AAAAAAAAJPGJAAAA|38648|10|44|8|AM|first|morning|| +38649|AAAAAAAAKPGJAAAA|38649|10|44|9|AM|first|morning|| +38650|AAAAAAAALPGJAAAA|38650|10|44|10|AM|first|morning|| +38651|AAAAAAAAMPGJAAAA|38651|10|44|11|AM|first|morning|| +38652|AAAAAAAANPGJAAAA|38652|10|44|12|AM|first|morning|| +38653|AAAAAAAAOPGJAAAA|38653|10|44|13|AM|first|morning|| +38654|AAAAAAAAPPGJAAAA|38654|10|44|14|AM|first|morning|| +38655|AAAAAAAAAAHJAAAA|38655|10|44|15|AM|first|morning|| +38656|AAAAAAAABAHJAAAA|38656|10|44|16|AM|first|morning|| +38657|AAAAAAAACAHJAAAA|38657|10|44|17|AM|first|morning|| +38658|AAAAAAAADAHJAAAA|38658|10|44|18|AM|first|morning|| +38659|AAAAAAAAEAHJAAAA|38659|10|44|19|AM|first|morning|| +38660|AAAAAAAAFAHJAAAA|38660|10|44|20|AM|first|morning|| +38661|AAAAAAAAGAHJAAAA|38661|10|44|21|AM|first|morning|| +38662|AAAAAAAAHAHJAAAA|38662|10|44|22|AM|first|morning|| +38663|AAAAAAAAIAHJAAAA|38663|10|44|23|AM|first|morning|| +38664|AAAAAAAAJAHJAAAA|38664|10|44|24|AM|first|morning|| +38665|AAAAAAAAKAHJAAAA|38665|10|44|25|AM|first|morning|| +38666|AAAAAAAALAHJAAAA|38666|10|44|26|AM|first|morning|| +38667|AAAAAAAAMAHJAAAA|38667|10|44|27|AM|first|morning|| +38668|AAAAAAAANAHJAAAA|38668|10|44|28|AM|first|morning|| +38669|AAAAAAAAOAHJAAAA|38669|10|44|29|AM|first|morning|| +38670|AAAAAAAAPAHJAAAA|38670|10|44|30|AM|first|morning|| +38671|AAAAAAAAABHJAAAA|38671|10|44|31|AM|first|morning|| +38672|AAAAAAAABBHJAAAA|38672|10|44|32|AM|first|morning|| +38673|AAAAAAAACBHJAAAA|38673|10|44|33|AM|first|morning|| +38674|AAAAAAAADBHJAAAA|38674|10|44|34|AM|first|morning|| +38675|AAAAAAAAEBHJAAAA|38675|10|44|35|AM|first|morning|| +38676|AAAAAAAAFBHJAAAA|38676|10|44|36|AM|first|morning|| +38677|AAAAAAAAGBHJAAAA|38677|10|44|37|AM|first|morning|| +38678|AAAAAAAAHBHJAAAA|38678|10|44|38|AM|first|morning|| +38679|AAAAAAAAIBHJAAAA|38679|10|44|39|AM|first|morning|| +38680|AAAAAAAAJBHJAAAA|38680|10|44|40|AM|first|morning|| +38681|AAAAAAAAKBHJAAAA|38681|10|44|41|AM|first|morning|| +38682|AAAAAAAALBHJAAAA|38682|10|44|42|AM|first|morning|| +38683|AAAAAAAAMBHJAAAA|38683|10|44|43|AM|first|morning|| +38684|AAAAAAAANBHJAAAA|38684|10|44|44|AM|first|morning|| +38685|AAAAAAAAOBHJAAAA|38685|10|44|45|AM|first|morning|| +38686|AAAAAAAAPBHJAAAA|38686|10|44|46|AM|first|morning|| +38687|AAAAAAAAACHJAAAA|38687|10|44|47|AM|first|morning|| +38688|AAAAAAAABCHJAAAA|38688|10|44|48|AM|first|morning|| +38689|AAAAAAAACCHJAAAA|38689|10|44|49|AM|first|morning|| +38690|AAAAAAAADCHJAAAA|38690|10|44|50|AM|first|morning|| +38691|AAAAAAAAECHJAAAA|38691|10|44|51|AM|first|morning|| +38692|AAAAAAAAFCHJAAAA|38692|10|44|52|AM|first|morning|| +38693|AAAAAAAAGCHJAAAA|38693|10|44|53|AM|first|morning|| +38694|AAAAAAAAHCHJAAAA|38694|10|44|54|AM|first|morning|| +38695|AAAAAAAAICHJAAAA|38695|10|44|55|AM|first|morning|| +38696|AAAAAAAAJCHJAAAA|38696|10|44|56|AM|first|morning|| +38697|AAAAAAAAKCHJAAAA|38697|10|44|57|AM|first|morning|| +38698|AAAAAAAALCHJAAAA|38698|10|44|58|AM|first|morning|| +38699|AAAAAAAAMCHJAAAA|38699|10|44|59|AM|first|morning|| +38700|AAAAAAAANCHJAAAA|38700|10|45|0|AM|first|morning|| +38701|AAAAAAAAOCHJAAAA|38701|10|45|1|AM|first|morning|| +38702|AAAAAAAAPCHJAAAA|38702|10|45|2|AM|first|morning|| +38703|AAAAAAAAADHJAAAA|38703|10|45|3|AM|first|morning|| +38704|AAAAAAAABDHJAAAA|38704|10|45|4|AM|first|morning|| +38705|AAAAAAAACDHJAAAA|38705|10|45|5|AM|first|morning|| +38706|AAAAAAAADDHJAAAA|38706|10|45|6|AM|first|morning|| +38707|AAAAAAAAEDHJAAAA|38707|10|45|7|AM|first|morning|| +38708|AAAAAAAAFDHJAAAA|38708|10|45|8|AM|first|morning|| +38709|AAAAAAAAGDHJAAAA|38709|10|45|9|AM|first|morning|| +38710|AAAAAAAAHDHJAAAA|38710|10|45|10|AM|first|morning|| +38711|AAAAAAAAIDHJAAAA|38711|10|45|11|AM|first|morning|| +38712|AAAAAAAAJDHJAAAA|38712|10|45|12|AM|first|morning|| +38713|AAAAAAAAKDHJAAAA|38713|10|45|13|AM|first|morning|| +38714|AAAAAAAALDHJAAAA|38714|10|45|14|AM|first|morning|| +38715|AAAAAAAAMDHJAAAA|38715|10|45|15|AM|first|morning|| +38716|AAAAAAAANDHJAAAA|38716|10|45|16|AM|first|morning|| +38717|AAAAAAAAODHJAAAA|38717|10|45|17|AM|first|morning|| +38718|AAAAAAAAPDHJAAAA|38718|10|45|18|AM|first|morning|| +38719|AAAAAAAAAEHJAAAA|38719|10|45|19|AM|first|morning|| +38720|AAAAAAAABEHJAAAA|38720|10|45|20|AM|first|morning|| +38721|AAAAAAAACEHJAAAA|38721|10|45|21|AM|first|morning|| +38722|AAAAAAAADEHJAAAA|38722|10|45|22|AM|first|morning|| +38723|AAAAAAAAEEHJAAAA|38723|10|45|23|AM|first|morning|| +38724|AAAAAAAAFEHJAAAA|38724|10|45|24|AM|first|morning|| +38725|AAAAAAAAGEHJAAAA|38725|10|45|25|AM|first|morning|| +38726|AAAAAAAAHEHJAAAA|38726|10|45|26|AM|first|morning|| +38727|AAAAAAAAIEHJAAAA|38727|10|45|27|AM|first|morning|| +38728|AAAAAAAAJEHJAAAA|38728|10|45|28|AM|first|morning|| +38729|AAAAAAAAKEHJAAAA|38729|10|45|29|AM|first|morning|| +38730|AAAAAAAALEHJAAAA|38730|10|45|30|AM|first|morning|| +38731|AAAAAAAAMEHJAAAA|38731|10|45|31|AM|first|morning|| +38732|AAAAAAAANEHJAAAA|38732|10|45|32|AM|first|morning|| +38733|AAAAAAAAOEHJAAAA|38733|10|45|33|AM|first|morning|| +38734|AAAAAAAAPEHJAAAA|38734|10|45|34|AM|first|morning|| +38735|AAAAAAAAAFHJAAAA|38735|10|45|35|AM|first|morning|| +38736|AAAAAAAABFHJAAAA|38736|10|45|36|AM|first|morning|| +38737|AAAAAAAACFHJAAAA|38737|10|45|37|AM|first|morning|| +38738|AAAAAAAADFHJAAAA|38738|10|45|38|AM|first|morning|| +38739|AAAAAAAAEFHJAAAA|38739|10|45|39|AM|first|morning|| +38740|AAAAAAAAFFHJAAAA|38740|10|45|40|AM|first|morning|| +38741|AAAAAAAAGFHJAAAA|38741|10|45|41|AM|first|morning|| +38742|AAAAAAAAHFHJAAAA|38742|10|45|42|AM|first|morning|| +38743|AAAAAAAAIFHJAAAA|38743|10|45|43|AM|first|morning|| +38744|AAAAAAAAJFHJAAAA|38744|10|45|44|AM|first|morning|| +38745|AAAAAAAAKFHJAAAA|38745|10|45|45|AM|first|morning|| +38746|AAAAAAAALFHJAAAA|38746|10|45|46|AM|first|morning|| +38747|AAAAAAAAMFHJAAAA|38747|10|45|47|AM|first|morning|| +38748|AAAAAAAANFHJAAAA|38748|10|45|48|AM|first|morning|| +38749|AAAAAAAAOFHJAAAA|38749|10|45|49|AM|first|morning|| +38750|AAAAAAAAPFHJAAAA|38750|10|45|50|AM|first|morning|| +38751|AAAAAAAAAGHJAAAA|38751|10|45|51|AM|first|morning|| +38752|AAAAAAAABGHJAAAA|38752|10|45|52|AM|first|morning|| +38753|AAAAAAAACGHJAAAA|38753|10|45|53|AM|first|morning|| +38754|AAAAAAAADGHJAAAA|38754|10|45|54|AM|first|morning|| +38755|AAAAAAAAEGHJAAAA|38755|10|45|55|AM|first|morning|| +38756|AAAAAAAAFGHJAAAA|38756|10|45|56|AM|first|morning|| +38757|AAAAAAAAGGHJAAAA|38757|10|45|57|AM|first|morning|| +38758|AAAAAAAAHGHJAAAA|38758|10|45|58|AM|first|morning|| +38759|AAAAAAAAIGHJAAAA|38759|10|45|59|AM|first|morning|| +38760|AAAAAAAAJGHJAAAA|38760|10|46|0|AM|first|morning|| +38761|AAAAAAAAKGHJAAAA|38761|10|46|1|AM|first|morning|| +38762|AAAAAAAALGHJAAAA|38762|10|46|2|AM|first|morning|| +38763|AAAAAAAAMGHJAAAA|38763|10|46|3|AM|first|morning|| +38764|AAAAAAAANGHJAAAA|38764|10|46|4|AM|first|morning|| +38765|AAAAAAAAOGHJAAAA|38765|10|46|5|AM|first|morning|| +38766|AAAAAAAAPGHJAAAA|38766|10|46|6|AM|first|morning|| +38767|AAAAAAAAAHHJAAAA|38767|10|46|7|AM|first|morning|| +38768|AAAAAAAABHHJAAAA|38768|10|46|8|AM|first|morning|| +38769|AAAAAAAACHHJAAAA|38769|10|46|9|AM|first|morning|| +38770|AAAAAAAADHHJAAAA|38770|10|46|10|AM|first|morning|| +38771|AAAAAAAAEHHJAAAA|38771|10|46|11|AM|first|morning|| +38772|AAAAAAAAFHHJAAAA|38772|10|46|12|AM|first|morning|| +38773|AAAAAAAAGHHJAAAA|38773|10|46|13|AM|first|morning|| +38774|AAAAAAAAHHHJAAAA|38774|10|46|14|AM|first|morning|| +38775|AAAAAAAAIHHJAAAA|38775|10|46|15|AM|first|morning|| +38776|AAAAAAAAJHHJAAAA|38776|10|46|16|AM|first|morning|| +38777|AAAAAAAAKHHJAAAA|38777|10|46|17|AM|first|morning|| +38778|AAAAAAAALHHJAAAA|38778|10|46|18|AM|first|morning|| +38779|AAAAAAAAMHHJAAAA|38779|10|46|19|AM|first|morning|| +38780|AAAAAAAANHHJAAAA|38780|10|46|20|AM|first|morning|| +38781|AAAAAAAAOHHJAAAA|38781|10|46|21|AM|first|morning|| +38782|AAAAAAAAPHHJAAAA|38782|10|46|22|AM|first|morning|| +38783|AAAAAAAAAIHJAAAA|38783|10|46|23|AM|first|morning|| +38784|AAAAAAAABIHJAAAA|38784|10|46|24|AM|first|morning|| +38785|AAAAAAAACIHJAAAA|38785|10|46|25|AM|first|morning|| +38786|AAAAAAAADIHJAAAA|38786|10|46|26|AM|first|morning|| +38787|AAAAAAAAEIHJAAAA|38787|10|46|27|AM|first|morning|| +38788|AAAAAAAAFIHJAAAA|38788|10|46|28|AM|first|morning|| +38789|AAAAAAAAGIHJAAAA|38789|10|46|29|AM|first|morning|| +38790|AAAAAAAAHIHJAAAA|38790|10|46|30|AM|first|morning|| +38791|AAAAAAAAIIHJAAAA|38791|10|46|31|AM|first|morning|| +38792|AAAAAAAAJIHJAAAA|38792|10|46|32|AM|first|morning|| +38793|AAAAAAAAKIHJAAAA|38793|10|46|33|AM|first|morning|| +38794|AAAAAAAALIHJAAAA|38794|10|46|34|AM|first|morning|| +38795|AAAAAAAAMIHJAAAA|38795|10|46|35|AM|first|morning|| +38796|AAAAAAAANIHJAAAA|38796|10|46|36|AM|first|morning|| +38797|AAAAAAAAOIHJAAAA|38797|10|46|37|AM|first|morning|| +38798|AAAAAAAAPIHJAAAA|38798|10|46|38|AM|first|morning|| +38799|AAAAAAAAAJHJAAAA|38799|10|46|39|AM|first|morning|| +38800|AAAAAAAABJHJAAAA|38800|10|46|40|AM|first|morning|| +38801|AAAAAAAACJHJAAAA|38801|10|46|41|AM|first|morning|| +38802|AAAAAAAADJHJAAAA|38802|10|46|42|AM|first|morning|| +38803|AAAAAAAAEJHJAAAA|38803|10|46|43|AM|first|morning|| +38804|AAAAAAAAFJHJAAAA|38804|10|46|44|AM|first|morning|| +38805|AAAAAAAAGJHJAAAA|38805|10|46|45|AM|first|morning|| +38806|AAAAAAAAHJHJAAAA|38806|10|46|46|AM|first|morning|| +38807|AAAAAAAAIJHJAAAA|38807|10|46|47|AM|first|morning|| +38808|AAAAAAAAJJHJAAAA|38808|10|46|48|AM|first|morning|| +38809|AAAAAAAAKJHJAAAA|38809|10|46|49|AM|first|morning|| +38810|AAAAAAAALJHJAAAA|38810|10|46|50|AM|first|morning|| +38811|AAAAAAAAMJHJAAAA|38811|10|46|51|AM|first|morning|| +38812|AAAAAAAANJHJAAAA|38812|10|46|52|AM|first|morning|| +38813|AAAAAAAAOJHJAAAA|38813|10|46|53|AM|first|morning|| +38814|AAAAAAAAPJHJAAAA|38814|10|46|54|AM|first|morning|| +38815|AAAAAAAAAKHJAAAA|38815|10|46|55|AM|first|morning|| +38816|AAAAAAAABKHJAAAA|38816|10|46|56|AM|first|morning|| +38817|AAAAAAAACKHJAAAA|38817|10|46|57|AM|first|morning|| +38818|AAAAAAAADKHJAAAA|38818|10|46|58|AM|first|morning|| +38819|AAAAAAAAEKHJAAAA|38819|10|46|59|AM|first|morning|| +38820|AAAAAAAAFKHJAAAA|38820|10|47|0|AM|first|morning|| +38821|AAAAAAAAGKHJAAAA|38821|10|47|1|AM|first|morning|| +38822|AAAAAAAAHKHJAAAA|38822|10|47|2|AM|first|morning|| +38823|AAAAAAAAIKHJAAAA|38823|10|47|3|AM|first|morning|| +38824|AAAAAAAAJKHJAAAA|38824|10|47|4|AM|first|morning|| +38825|AAAAAAAAKKHJAAAA|38825|10|47|5|AM|first|morning|| +38826|AAAAAAAALKHJAAAA|38826|10|47|6|AM|first|morning|| +38827|AAAAAAAAMKHJAAAA|38827|10|47|7|AM|first|morning|| +38828|AAAAAAAANKHJAAAA|38828|10|47|8|AM|first|morning|| +38829|AAAAAAAAOKHJAAAA|38829|10|47|9|AM|first|morning|| +38830|AAAAAAAAPKHJAAAA|38830|10|47|10|AM|first|morning|| +38831|AAAAAAAAALHJAAAA|38831|10|47|11|AM|first|morning|| +38832|AAAAAAAABLHJAAAA|38832|10|47|12|AM|first|morning|| +38833|AAAAAAAACLHJAAAA|38833|10|47|13|AM|first|morning|| +38834|AAAAAAAADLHJAAAA|38834|10|47|14|AM|first|morning|| +38835|AAAAAAAAELHJAAAA|38835|10|47|15|AM|first|morning|| +38836|AAAAAAAAFLHJAAAA|38836|10|47|16|AM|first|morning|| +38837|AAAAAAAAGLHJAAAA|38837|10|47|17|AM|first|morning|| +38838|AAAAAAAAHLHJAAAA|38838|10|47|18|AM|first|morning|| +38839|AAAAAAAAILHJAAAA|38839|10|47|19|AM|first|morning|| +38840|AAAAAAAAJLHJAAAA|38840|10|47|20|AM|first|morning|| +38841|AAAAAAAAKLHJAAAA|38841|10|47|21|AM|first|morning|| +38842|AAAAAAAALLHJAAAA|38842|10|47|22|AM|first|morning|| +38843|AAAAAAAAMLHJAAAA|38843|10|47|23|AM|first|morning|| +38844|AAAAAAAANLHJAAAA|38844|10|47|24|AM|first|morning|| +38845|AAAAAAAAOLHJAAAA|38845|10|47|25|AM|first|morning|| +38846|AAAAAAAAPLHJAAAA|38846|10|47|26|AM|first|morning|| +38847|AAAAAAAAAMHJAAAA|38847|10|47|27|AM|first|morning|| +38848|AAAAAAAABMHJAAAA|38848|10|47|28|AM|first|morning|| +38849|AAAAAAAACMHJAAAA|38849|10|47|29|AM|first|morning|| +38850|AAAAAAAADMHJAAAA|38850|10|47|30|AM|first|morning|| +38851|AAAAAAAAEMHJAAAA|38851|10|47|31|AM|first|morning|| +38852|AAAAAAAAFMHJAAAA|38852|10|47|32|AM|first|morning|| +38853|AAAAAAAAGMHJAAAA|38853|10|47|33|AM|first|morning|| +38854|AAAAAAAAHMHJAAAA|38854|10|47|34|AM|first|morning|| +38855|AAAAAAAAIMHJAAAA|38855|10|47|35|AM|first|morning|| +38856|AAAAAAAAJMHJAAAA|38856|10|47|36|AM|first|morning|| +38857|AAAAAAAAKMHJAAAA|38857|10|47|37|AM|first|morning|| +38858|AAAAAAAALMHJAAAA|38858|10|47|38|AM|first|morning|| +38859|AAAAAAAAMMHJAAAA|38859|10|47|39|AM|first|morning|| +38860|AAAAAAAANMHJAAAA|38860|10|47|40|AM|first|morning|| +38861|AAAAAAAAOMHJAAAA|38861|10|47|41|AM|first|morning|| +38862|AAAAAAAAPMHJAAAA|38862|10|47|42|AM|first|morning|| +38863|AAAAAAAAANHJAAAA|38863|10|47|43|AM|first|morning|| +38864|AAAAAAAABNHJAAAA|38864|10|47|44|AM|first|morning|| +38865|AAAAAAAACNHJAAAA|38865|10|47|45|AM|first|morning|| +38866|AAAAAAAADNHJAAAA|38866|10|47|46|AM|first|morning|| +38867|AAAAAAAAENHJAAAA|38867|10|47|47|AM|first|morning|| +38868|AAAAAAAAFNHJAAAA|38868|10|47|48|AM|first|morning|| +38869|AAAAAAAAGNHJAAAA|38869|10|47|49|AM|first|morning|| +38870|AAAAAAAAHNHJAAAA|38870|10|47|50|AM|first|morning|| +38871|AAAAAAAAINHJAAAA|38871|10|47|51|AM|first|morning|| +38872|AAAAAAAAJNHJAAAA|38872|10|47|52|AM|first|morning|| +38873|AAAAAAAAKNHJAAAA|38873|10|47|53|AM|first|morning|| +38874|AAAAAAAALNHJAAAA|38874|10|47|54|AM|first|morning|| +38875|AAAAAAAAMNHJAAAA|38875|10|47|55|AM|first|morning|| +38876|AAAAAAAANNHJAAAA|38876|10|47|56|AM|first|morning|| +38877|AAAAAAAAONHJAAAA|38877|10|47|57|AM|first|morning|| +38878|AAAAAAAAPNHJAAAA|38878|10|47|58|AM|first|morning|| +38879|AAAAAAAAAOHJAAAA|38879|10|47|59|AM|first|morning|| +38880|AAAAAAAABOHJAAAA|38880|10|48|0|AM|first|morning|| +38881|AAAAAAAACOHJAAAA|38881|10|48|1|AM|first|morning|| +38882|AAAAAAAADOHJAAAA|38882|10|48|2|AM|first|morning|| +38883|AAAAAAAAEOHJAAAA|38883|10|48|3|AM|first|morning|| +38884|AAAAAAAAFOHJAAAA|38884|10|48|4|AM|first|morning|| +38885|AAAAAAAAGOHJAAAA|38885|10|48|5|AM|first|morning|| +38886|AAAAAAAAHOHJAAAA|38886|10|48|6|AM|first|morning|| +38887|AAAAAAAAIOHJAAAA|38887|10|48|7|AM|first|morning|| +38888|AAAAAAAAJOHJAAAA|38888|10|48|8|AM|first|morning|| +38889|AAAAAAAAKOHJAAAA|38889|10|48|9|AM|first|morning|| +38890|AAAAAAAALOHJAAAA|38890|10|48|10|AM|first|morning|| +38891|AAAAAAAAMOHJAAAA|38891|10|48|11|AM|first|morning|| +38892|AAAAAAAANOHJAAAA|38892|10|48|12|AM|first|morning|| +38893|AAAAAAAAOOHJAAAA|38893|10|48|13|AM|first|morning|| +38894|AAAAAAAAPOHJAAAA|38894|10|48|14|AM|first|morning|| +38895|AAAAAAAAAPHJAAAA|38895|10|48|15|AM|first|morning|| +38896|AAAAAAAABPHJAAAA|38896|10|48|16|AM|first|morning|| +38897|AAAAAAAACPHJAAAA|38897|10|48|17|AM|first|morning|| +38898|AAAAAAAADPHJAAAA|38898|10|48|18|AM|first|morning|| +38899|AAAAAAAAEPHJAAAA|38899|10|48|19|AM|first|morning|| +38900|AAAAAAAAFPHJAAAA|38900|10|48|20|AM|first|morning|| +38901|AAAAAAAAGPHJAAAA|38901|10|48|21|AM|first|morning|| +38902|AAAAAAAAHPHJAAAA|38902|10|48|22|AM|first|morning|| +38903|AAAAAAAAIPHJAAAA|38903|10|48|23|AM|first|morning|| +38904|AAAAAAAAJPHJAAAA|38904|10|48|24|AM|first|morning|| +38905|AAAAAAAAKPHJAAAA|38905|10|48|25|AM|first|morning|| +38906|AAAAAAAALPHJAAAA|38906|10|48|26|AM|first|morning|| +38907|AAAAAAAAMPHJAAAA|38907|10|48|27|AM|first|morning|| +38908|AAAAAAAANPHJAAAA|38908|10|48|28|AM|first|morning|| +38909|AAAAAAAAOPHJAAAA|38909|10|48|29|AM|first|morning|| +38910|AAAAAAAAPPHJAAAA|38910|10|48|30|AM|first|morning|| +38911|AAAAAAAAAAIJAAAA|38911|10|48|31|AM|first|morning|| +38912|AAAAAAAABAIJAAAA|38912|10|48|32|AM|first|morning|| +38913|AAAAAAAACAIJAAAA|38913|10|48|33|AM|first|morning|| +38914|AAAAAAAADAIJAAAA|38914|10|48|34|AM|first|morning|| +38915|AAAAAAAAEAIJAAAA|38915|10|48|35|AM|first|morning|| +38916|AAAAAAAAFAIJAAAA|38916|10|48|36|AM|first|morning|| +38917|AAAAAAAAGAIJAAAA|38917|10|48|37|AM|first|morning|| +38918|AAAAAAAAHAIJAAAA|38918|10|48|38|AM|first|morning|| +38919|AAAAAAAAIAIJAAAA|38919|10|48|39|AM|first|morning|| +38920|AAAAAAAAJAIJAAAA|38920|10|48|40|AM|first|morning|| +38921|AAAAAAAAKAIJAAAA|38921|10|48|41|AM|first|morning|| +38922|AAAAAAAALAIJAAAA|38922|10|48|42|AM|first|morning|| +38923|AAAAAAAAMAIJAAAA|38923|10|48|43|AM|first|morning|| +38924|AAAAAAAANAIJAAAA|38924|10|48|44|AM|first|morning|| +38925|AAAAAAAAOAIJAAAA|38925|10|48|45|AM|first|morning|| +38926|AAAAAAAAPAIJAAAA|38926|10|48|46|AM|first|morning|| +38927|AAAAAAAAABIJAAAA|38927|10|48|47|AM|first|morning|| +38928|AAAAAAAABBIJAAAA|38928|10|48|48|AM|first|morning|| +38929|AAAAAAAACBIJAAAA|38929|10|48|49|AM|first|morning|| +38930|AAAAAAAADBIJAAAA|38930|10|48|50|AM|first|morning|| +38931|AAAAAAAAEBIJAAAA|38931|10|48|51|AM|first|morning|| +38932|AAAAAAAAFBIJAAAA|38932|10|48|52|AM|first|morning|| +38933|AAAAAAAAGBIJAAAA|38933|10|48|53|AM|first|morning|| +38934|AAAAAAAAHBIJAAAA|38934|10|48|54|AM|first|morning|| +38935|AAAAAAAAIBIJAAAA|38935|10|48|55|AM|first|morning|| +38936|AAAAAAAAJBIJAAAA|38936|10|48|56|AM|first|morning|| +38937|AAAAAAAAKBIJAAAA|38937|10|48|57|AM|first|morning|| +38938|AAAAAAAALBIJAAAA|38938|10|48|58|AM|first|morning|| +38939|AAAAAAAAMBIJAAAA|38939|10|48|59|AM|first|morning|| +38940|AAAAAAAANBIJAAAA|38940|10|49|0|AM|first|morning|| +38941|AAAAAAAAOBIJAAAA|38941|10|49|1|AM|first|morning|| +38942|AAAAAAAAPBIJAAAA|38942|10|49|2|AM|first|morning|| +38943|AAAAAAAAACIJAAAA|38943|10|49|3|AM|first|morning|| +38944|AAAAAAAABCIJAAAA|38944|10|49|4|AM|first|morning|| +38945|AAAAAAAACCIJAAAA|38945|10|49|5|AM|first|morning|| +38946|AAAAAAAADCIJAAAA|38946|10|49|6|AM|first|morning|| +38947|AAAAAAAAECIJAAAA|38947|10|49|7|AM|first|morning|| +38948|AAAAAAAAFCIJAAAA|38948|10|49|8|AM|first|morning|| +38949|AAAAAAAAGCIJAAAA|38949|10|49|9|AM|first|morning|| +38950|AAAAAAAAHCIJAAAA|38950|10|49|10|AM|first|morning|| +38951|AAAAAAAAICIJAAAA|38951|10|49|11|AM|first|morning|| +38952|AAAAAAAAJCIJAAAA|38952|10|49|12|AM|first|morning|| +38953|AAAAAAAAKCIJAAAA|38953|10|49|13|AM|first|morning|| +38954|AAAAAAAALCIJAAAA|38954|10|49|14|AM|first|morning|| +38955|AAAAAAAAMCIJAAAA|38955|10|49|15|AM|first|morning|| +38956|AAAAAAAANCIJAAAA|38956|10|49|16|AM|first|morning|| +38957|AAAAAAAAOCIJAAAA|38957|10|49|17|AM|first|morning|| +38958|AAAAAAAAPCIJAAAA|38958|10|49|18|AM|first|morning|| +38959|AAAAAAAAADIJAAAA|38959|10|49|19|AM|first|morning|| +38960|AAAAAAAABDIJAAAA|38960|10|49|20|AM|first|morning|| +38961|AAAAAAAACDIJAAAA|38961|10|49|21|AM|first|morning|| +38962|AAAAAAAADDIJAAAA|38962|10|49|22|AM|first|morning|| +38963|AAAAAAAAEDIJAAAA|38963|10|49|23|AM|first|morning|| +38964|AAAAAAAAFDIJAAAA|38964|10|49|24|AM|first|morning|| +38965|AAAAAAAAGDIJAAAA|38965|10|49|25|AM|first|morning|| +38966|AAAAAAAAHDIJAAAA|38966|10|49|26|AM|first|morning|| +38967|AAAAAAAAIDIJAAAA|38967|10|49|27|AM|first|morning|| +38968|AAAAAAAAJDIJAAAA|38968|10|49|28|AM|first|morning|| +38969|AAAAAAAAKDIJAAAA|38969|10|49|29|AM|first|morning|| +38970|AAAAAAAALDIJAAAA|38970|10|49|30|AM|first|morning|| +38971|AAAAAAAAMDIJAAAA|38971|10|49|31|AM|first|morning|| +38972|AAAAAAAANDIJAAAA|38972|10|49|32|AM|first|morning|| +38973|AAAAAAAAODIJAAAA|38973|10|49|33|AM|first|morning|| +38974|AAAAAAAAPDIJAAAA|38974|10|49|34|AM|first|morning|| +38975|AAAAAAAAAEIJAAAA|38975|10|49|35|AM|first|morning|| +38976|AAAAAAAABEIJAAAA|38976|10|49|36|AM|first|morning|| +38977|AAAAAAAACEIJAAAA|38977|10|49|37|AM|first|morning|| +38978|AAAAAAAADEIJAAAA|38978|10|49|38|AM|first|morning|| +38979|AAAAAAAAEEIJAAAA|38979|10|49|39|AM|first|morning|| +38980|AAAAAAAAFEIJAAAA|38980|10|49|40|AM|first|morning|| +38981|AAAAAAAAGEIJAAAA|38981|10|49|41|AM|first|morning|| +38982|AAAAAAAAHEIJAAAA|38982|10|49|42|AM|first|morning|| +38983|AAAAAAAAIEIJAAAA|38983|10|49|43|AM|first|morning|| +38984|AAAAAAAAJEIJAAAA|38984|10|49|44|AM|first|morning|| +38985|AAAAAAAAKEIJAAAA|38985|10|49|45|AM|first|morning|| +38986|AAAAAAAALEIJAAAA|38986|10|49|46|AM|first|morning|| +38987|AAAAAAAAMEIJAAAA|38987|10|49|47|AM|first|morning|| +38988|AAAAAAAANEIJAAAA|38988|10|49|48|AM|first|morning|| +38989|AAAAAAAAOEIJAAAA|38989|10|49|49|AM|first|morning|| +38990|AAAAAAAAPEIJAAAA|38990|10|49|50|AM|first|morning|| +38991|AAAAAAAAAFIJAAAA|38991|10|49|51|AM|first|morning|| +38992|AAAAAAAABFIJAAAA|38992|10|49|52|AM|first|morning|| +38993|AAAAAAAACFIJAAAA|38993|10|49|53|AM|first|morning|| +38994|AAAAAAAADFIJAAAA|38994|10|49|54|AM|first|morning|| +38995|AAAAAAAAEFIJAAAA|38995|10|49|55|AM|first|morning|| +38996|AAAAAAAAFFIJAAAA|38996|10|49|56|AM|first|morning|| +38997|AAAAAAAAGFIJAAAA|38997|10|49|57|AM|first|morning|| +38998|AAAAAAAAHFIJAAAA|38998|10|49|58|AM|first|morning|| +38999|AAAAAAAAIFIJAAAA|38999|10|49|59|AM|first|morning|| +39000|AAAAAAAAJFIJAAAA|39000|10|50|0|AM|first|morning|| +39001|AAAAAAAAKFIJAAAA|39001|10|50|1|AM|first|morning|| +39002|AAAAAAAALFIJAAAA|39002|10|50|2|AM|first|morning|| +39003|AAAAAAAAMFIJAAAA|39003|10|50|3|AM|first|morning|| +39004|AAAAAAAANFIJAAAA|39004|10|50|4|AM|first|morning|| +39005|AAAAAAAAOFIJAAAA|39005|10|50|5|AM|first|morning|| +39006|AAAAAAAAPFIJAAAA|39006|10|50|6|AM|first|morning|| +39007|AAAAAAAAAGIJAAAA|39007|10|50|7|AM|first|morning|| +39008|AAAAAAAABGIJAAAA|39008|10|50|8|AM|first|morning|| +39009|AAAAAAAACGIJAAAA|39009|10|50|9|AM|first|morning|| +39010|AAAAAAAADGIJAAAA|39010|10|50|10|AM|first|morning|| +39011|AAAAAAAAEGIJAAAA|39011|10|50|11|AM|first|morning|| +39012|AAAAAAAAFGIJAAAA|39012|10|50|12|AM|first|morning|| +39013|AAAAAAAAGGIJAAAA|39013|10|50|13|AM|first|morning|| +39014|AAAAAAAAHGIJAAAA|39014|10|50|14|AM|first|morning|| +39015|AAAAAAAAIGIJAAAA|39015|10|50|15|AM|first|morning|| +39016|AAAAAAAAJGIJAAAA|39016|10|50|16|AM|first|morning|| +39017|AAAAAAAAKGIJAAAA|39017|10|50|17|AM|first|morning|| +39018|AAAAAAAALGIJAAAA|39018|10|50|18|AM|first|morning|| +39019|AAAAAAAAMGIJAAAA|39019|10|50|19|AM|first|morning|| +39020|AAAAAAAANGIJAAAA|39020|10|50|20|AM|first|morning|| +39021|AAAAAAAAOGIJAAAA|39021|10|50|21|AM|first|morning|| +39022|AAAAAAAAPGIJAAAA|39022|10|50|22|AM|first|morning|| +39023|AAAAAAAAAHIJAAAA|39023|10|50|23|AM|first|morning|| +39024|AAAAAAAABHIJAAAA|39024|10|50|24|AM|first|morning|| +39025|AAAAAAAACHIJAAAA|39025|10|50|25|AM|first|morning|| +39026|AAAAAAAADHIJAAAA|39026|10|50|26|AM|first|morning|| +39027|AAAAAAAAEHIJAAAA|39027|10|50|27|AM|first|morning|| +39028|AAAAAAAAFHIJAAAA|39028|10|50|28|AM|first|morning|| +39029|AAAAAAAAGHIJAAAA|39029|10|50|29|AM|first|morning|| +39030|AAAAAAAAHHIJAAAA|39030|10|50|30|AM|first|morning|| +39031|AAAAAAAAIHIJAAAA|39031|10|50|31|AM|first|morning|| +39032|AAAAAAAAJHIJAAAA|39032|10|50|32|AM|first|morning|| +39033|AAAAAAAAKHIJAAAA|39033|10|50|33|AM|first|morning|| +39034|AAAAAAAALHIJAAAA|39034|10|50|34|AM|first|morning|| +39035|AAAAAAAAMHIJAAAA|39035|10|50|35|AM|first|morning|| +39036|AAAAAAAANHIJAAAA|39036|10|50|36|AM|first|morning|| +39037|AAAAAAAAOHIJAAAA|39037|10|50|37|AM|first|morning|| +39038|AAAAAAAAPHIJAAAA|39038|10|50|38|AM|first|morning|| +39039|AAAAAAAAAIIJAAAA|39039|10|50|39|AM|first|morning|| +39040|AAAAAAAABIIJAAAA|39040|10|50|40|AM|first|morning|| +39041|AAAAAAAACIIJAAAA|39041|10|50|41|AM|first|morning|| +39042|AAAAAAAADIIJAAAA|39042|10|50|42|AM|first|morning|| +39043|AAAAAAAAEIIJAAAA|39043|10|50|43|AM|first|morning|| +39044|AAAAAAAAFIIJAAAA|39044|10|50|44|AM|first|morning|| +39045|AAAAAAAAGIIJAAAA|39045|10|50|45|AM|first|morning|| +39046|AAAAAAAAHIIJAAAA|39046|10|50|46|AM|first|morning|| +39047|AAAAAAAAIIIJAAAA|39047|10|50|47|AM|first|morning|| +39048|AAAAAAAAJIIJAAAA|39048|10|50|48|AM|first|morning|| +39049|AAAAAAAAKIIJAAAA|39049|10|50|49|AM|first|morning|| +39050|AAAAAAAALIIJAAAA|39050|10|50|50|AM|first|morning|| +39051|AAAAAAAAMIIJAAAA|39051|10|50|51|AM|first|morning|| +39052|AAAAAAAANIIJAAAA|39052|10|50|52|AM|first|morning|| +39053|AAAAAAAAOIIJAAAA|39053|10|50|53|AM|first|morning|| +39054|AAAAAAAAPIIJAAAA|39054|10|50|54|AM|first|morning|| +39055|AAAAAAAAAJIJAAAA|39055|10|50|55|AM|first|morning|| +39056|AAAAAAAABJIJAAAA|39056|10|50|56|AM|first|morning|| +39057|AAAAAAAACJIJAAAA|39057|10|50|57|AM|first|morning|| +39058|AAAAAAAADJIJAAAA|39058|10|50|58|AM|first|morning|| +39059|AAAAAAAAEJIJAAAA|39059|10|50|59|AM|first|morning|| +39060|AAAAAAAAFJIJAAAA|39060|10|51|0|AM|first|morning|| +39061|AAAAAAAAGJIJAAAA|39061|10|51|1|AM|first|morning|| +39062|AAAAAAAAHJIJAAAA|39062|10|51|2|AM|first|morning|| +39063|AAAAAAAAIJIJAAAA|39063|10|51|3|AM|first|morning|| +39064|AAAAAAAAJJIJAAAA|39064|10|51|4|AM|first|morning|| +39065|AAAAAAAAKJIJAAAA|39065|10|51|5|AM|first|morning|| +39066|AAAAAAAALJIJAAAA|39066|10|51|6|AM|first|morning|| +39067|AAAAAAAAMJIJAAAA|39067|10|51|7|AM|first|morning|| +39068|AAAAAAAANJIJAAAA|39068|10|51|8|AM|first|morning|| +39069|AAAAAAAAOJIJAAAA|39069|10|51|9|AM|first|morning|| +39070|AAAAAAAAPJIJAAAA|39070|10|51|10|AM|first|morning|| +39071|AAAAAAAAAKIJAAAA|39071|10|51|11|AM|first|morning|| +39072|AAAAAAAABKIJAAAA|39072|10|51|12|AM|first|morning|| +39073|AAAAAAAACKIJAAAA|39073|10|51|13|AM|first|morning|| +39074|AAAAAAAADKIJAAAA|39074|10|51|14|AM|first|morning|| +39075|AAAAAAAAEKIJAAAA|39075|10|51|15|AM|first|morning|| +39076|AAAAAAAAFKIJAAAA|39076|10|51|16|AM|first|morning|| +39077|AAAAAAAAGKIJAAAA|39077|10|51|17|AM|first|morning|| +39078|AAAAAAAAHKIJAAAA|39078|10|51|18|AM|first|morning|| +39079|AAAAAAAAIKIJAAAA|39079|10|51|19|AM|first|morning|| +39080|AAAAAAAAJKIJAAAA|39080|10|51|20|AM|first|morning|| +39081|AAAAAAAAKKIJAAAA|39081|10|51|21|AM|first|morning|| +39082|AAAAAAAALKIJAAAA|39082|10|51|22|AM|first|morning|| +39083|AAAAAAAAMKIJAAAA|39083|10|51|23|AM|first|morning|| +39084|AAAAAAAANKIJAAAA|39084|10|51|24|AM|first|morning|| +39085|AAAAAAAAOKIJAAAA|39085|10|51|25|AM|first|morning|| +39086|AAAAAAAAPKIJAAAA|39086|10|51|26|AM|first|morning|| +39087|AAAAAAAAALIJAAAA|39087|10|51|27|AM|first|morning|| +39088|AAAAAAAABLIJAAAA|39088|10|51|28|AM|first|morning|| +39089|AAAAAAAACLIJAAAA|39089|10|51|29|AM|first|morning|| +39090|AAAAAAAADLIJAAAA|39090|10|51|30|AM|first|morning|| +39091|AAAAAAAAELIJAAAA|39091|10|51|31|AM|first|morning|| +39092|AAAAAAAAFLIJAAAA|39092|10|51|32|AM|first|morning|| +39093|AAAAAAAAGLIJAAAA|39093|10|51|33|AM|first|morning|| +39094|AAAAAAAAHLIJAAAA|39094|10|51|34|AM|first|morning|| +39095|AAAAAAAAILIJAAAA|39095|10|51|35|AM|first|morning|| +39096|AAAAAAAAJLIJAAAA|39096|10|51|36|AM|first|morning|| +39097|AAAAAAAAKLIJAAAA|39097|10|51|37|AM|first|morning|| +39098|AAAAAAAALLIJAAAA|39098|10|51|38|AM|first|morning|| +39099|AAAAAAAAMLIJAAAA|39099|10|51|39|AM|first|morning|| +39100|AAAAAAAANLIJAAAA|39100|10|51|40|AM|first|morning|| +39101|AAAAAAAAOLIJAAAA|39101|10|51|41|AM|first|morning|| +39102|AAAAAAAAPLIJAAAA|39102|10|51|42|AM|first|morning|| +39103|AAAAAAAAAMIJAAAA|39103|10|51|43|AM|first|morning|| +39104|AAAAAAAABMIJAAAA|39104|10|51|44|AM|first|morning|| +39105|AAAAAAAACMIJAAAA|39105|10|51|45|AM|first|morning|| +39106|AAAAAAAADMIJAAAA|39106|10|51|46|AM|first|morning|| +39107|AAAAAAAAEMIJAAAA|39107|10|51|47|AM|first|morning|| +39108|AAAAAAAAFMIJAAAA|39108|10|51|48|AM|first|morning|| +39109|AAAAAAAAGMIJAAAA|39109|10|51|49|AM|first|morning|| +39110|AAAAAAAAHMIJAAAA|39110|10|51|50|AM|first|morning|| +39111|AAAAAAAAIMIJAAAA|39111|10|51|51|AM|first|morning|| +39112|AAAAAAAAJMIJAAAA|39112|10|51|52|AM|first|morning|| +39113|AAAAAAAAKMIJAAAA|39113|10|51|53|AM|first|morning|| +39114|AAAAAAAALMIJAAAA|39114|10|51|54|AM|first|morning|| +39115|AAAAAAAAMMIJAAAA|39115|10|51|55|AM|first|morning|| +39116|AAAAAAAANMIJAAAA|39116|10|51|56|AM|first|morning|| +39117|AAAAAAAAOMIJAAAA|39117|10|51|57|AM|first|morning|| +39118|AAAAAAAAPMIJAAAA|39118|10|51|58|AM|first|morning|| +39119|AAAAAAAAANIJAAAA|39119|10|51|59|AM|first|morning|| +39120|AAAAAAAABNIJAAAA|39120|10|52|0|AM|first|morning|| +39121|AAAAAAAACNIJAAAA|39121|10|52|1|AM|first|morning|| +39122|AAAAAAAADNIJAAAA|39122|10|52|2|AM|first|morning|| +39123|AAAAAAAAENIJAAAA|39123|10|52|3|AM|first|morning|| +39124|AAAAAAAAFNIJAAAA|39124|10|52|4|AM|first|morning|| +39125|AAAAAAAAGNIJAAAA|39125|10|52|5|AM|first|morning|| +39126|AAAAAAAAHNIJAAAA|39126|10|52|6|AM|first|morning|| +39127|AAAAAAAAINIJAAAA|39127|10|52|7|AM|first|morning|| +39128|AAAAAAAAJNIJAAAA|39128|10|52|8|AM|first|morning|| +39129|AAAAAAAAKNIJAAAA|39129|10|52|9|AM|first|morning|| +39130|AAAAAAAALNIJAAAA|39130|10|52|10|AM|first|morning|| +39131|AAAAAAAAMNIJAAAA|39131|10|52|11|AM|first|morning|| +39132|AAAAAAAANNIJAAAA|39132|10|52|12|AM|first|morning|| +39133|AAAAAAAAONIJAAAA|39133|10|52|13|AM|first|morning|| +39134|AAAAAAAAPNIJAAAA|39134|10|52|14|AM|first|morning|| +39135|AAAAAAAAAOIJAAAA|39135|10|52|15|AM|first|morning|| +39136|AAAAAAAABOIJAAAA|39136|10|52|16|AM|first|morning|| +39137|AAAAAAAACOIJAAAA|39137|10|52|17|AM|first|morning|| +39138|AAAAAAAADOIJAAAA|39138|10|52|18|AM|first|morning|| +39139|AAAAAAAAEOIJAAAA|39139|10|52|19|AM|first|morning|| +39140|AAAAAAAAFOIJAAAA|39140|10|52|20|AM|first|morning|| +39141|AAAAAAAAGOIJAAAA|39141|10|52|21|AM|first|morning|| +39142|AAAAAAAAHOIJAAAA|39142|10|52|22|AM|first|morning|| +39143|AAAAAAAAIOIJAAAA|39143|10|52|23|AM|first|morning|| +39144|AAAAAAAAJOIJAAAA|39144|10|52|24|AM|first|morning|| +39145|AAAAAAAAKOIJAAAA|39145|10|52|25|AM|first|morning|| +39146|AAAAAAAALOIJAAAA|39146|10|52|26|AM|first|morning|| +39147|AAAAAAAAMOIJAAAA|39147|10|52|27|AM|first|morning|| +39148|AAAAAAAANOIJAAAA|39148|10|52|28|AM|first|morning|| +39149|AAAAAAAAOOIJAAAA|39149|10|52|29|AM|first|morning|| +39150|AAAAAAAAPOIJAAAA|39150|10|52|30|AM|first|morning|| +39151|AAAAAAAAAPIJAAAA|39151|10|52|31|AM|first|morning|| +39152|AAAAAAAABPIJAAAA|39152|10|52|32|AM|first|morning|| +39153|AAAAAAAACPIJAAAA|39153|10|52|33|AM|first|morning|| +39154|AAAAAAAADPIJAAAA|39154|10|52|34|AM|first|morning|| +39155|AAAAAAAAEPIJAAAA|39155|10|52|35|AM|first|morning|| +39156|AAAAAAAAFPIJAAAA|39156|10|52|36|AM|first|morning|| +39157|AAAAAAAAGPIJAAAA|39157|10|52|37|AM|first|morning|| +39158|AAAAAAAAHPIJAAAA|39158|10|52|38|AM|first|morning|| +39159|AAAAAAAAIPIJAAAA|39159|10|52|39|AM|first|morning|| +39160|AAAAAAAAJPIJAAAA|39160|10|52|40|AM|first|morning|| +39161|AAAAAAAAKPIJAAAA|39161|10|52|41|AM|first|morning|| +39162|AAAAAAAALPIJAAAA|39162|10|52|42|AM|first|morning|| +39163|AAAAAAAAMPIJAAAA|39163|10|52|43|AM|first|morning|| +39164|AAAAAAAANPIJAAAA|39164|10|52|44|AM|first|morning|| +39165|AAAAAAAAOPIJAAAA|39165|10|52|45|AM|first|morning|| +39166|AAAAAAAAPPIJAAAA|39166|10|52|46|AM|first|morning|| +39167|AAAAAAAAAAJJAAAA|39167|10|52|47|AM|first|morning|| +39168|AAAAAAAABAJJAAAA|39168|10|52|48|AM|first|morning|| +39169|AAAAAAAACAJJAAAA|39169|10|52|49|AM|first|morning|| +39170|AAAAAAAADAJJAAAA|39170|10|52|50|AM|first|morning|| +39171|AAAAAAAAEAJJAAAA|39171|10|52|51|AM|first|morning|| +39172|AAAAAAAAFAJJAAAA|39172|10|52|52|AM|first|morning|| +39173|AAAAAAAAGAJJAAAA|39173|10|52|53|AM|first|morning|| +39174|AAAAAAAAHAJJAAAA|39174|10|52|54|AM|first|morning|| +39175|AAAAAAAAIAJJAAAA|39175|10|52|55|AM|first|morning|| +39176|AAAAAAAAJAJJAAAA|39176|10|52|56|AM|first|morning|| +39177|AAAAAAAAKAJJAAAA|39177|10|52|57|AM|first|morning|| +39178|AAAAAAAALAJJAAAA|39178|10|52|58|AM|first|morning|| +39179|AAAAAAAAMAJJAAAA|39179|10|52|59|AM|first|morning|| +39180|AAAAAAAANAJJAAAA|39180|10|53|0|AM|first|morning|| +39181|AAAAAAAAOAJJAAAA|39181|10|53|1|AM|first|morning|| +39182|AAAAAAAAPAJJAAAA|39182|10|53|2|AM|first|morning|| +39183|AAAAAAAAABJJAAAA|39183|10|53|3|AM|first|morning|| +39184|AAAAAAAABBJJAAAA|39184|10|53|4|AM|first|morning|| +39185|AAAAAAAACBJJAAAA|39185|10|53|5|AM|first|morning|| +39186|AAAAAAAADBJJAAAA|39186|10|53|6|AM|first|morning|| +39187|AAAAAAAAEBJJAAAA|39187|10|53|7|AM|first|morning|| +39188|AAAAAAAAFBJJAAAA|39188|10|53|8|AM|first|morning|| +39189|AAAAAAAAGBJJAAAA|39189|10|53|9|AM|first|morning|| +39190|AAAAAAAAHBJJAAAA|39190|10|53|10|AM|first|morning|| +39191|AAAAAAAAIBJJAAAA|39191|10|53|11|AM|first|morning|| +39192|AAAAAAAAJBJJAAAA|39192|10|53|12|AM|first|morning|| +39193|AAAAAAAAKBJJAAAA|39193|10|53|13|AM|first|morning|| +39194|AAAAAAAALBJJAAAA|39194|10|53|14|AM|first|morning|| +39195|AAAAAAAAMBJJAAAA|39195|10|53|15|AM|first|morning|| +39196|AAAAAAAANBJJAAAA|39196|10|53|16|AM|first|morning|| +39197|AAAAAAAAOBJJAAAA|39197|10|53|17|AM|first|morning|| +39198|AAAAAAAAPBJJAAAA|39198|10|53|18|AM|first|morning|| +39199|AAAAAAAAACJJAAAA|39199|10|53|19|AM|first|morning|| +39200|AAAAAAAABCJJAAAA|39200|10|53|20|AM|first|morning|| +39201|AAAAAAAACCJJAAAA|39201|10|53|21|AM|first|morning|| +39202|AAAAAAAADCJJAAAA|39202|10|53|22|AM|first|morning|| +39203|AAAAAAAAECJJAAAA|39203|10|53|23|AM|first|morning|| +39204|AAAAAAAAFCJJAAAA|39204|10|53|24|AM|first|morning|| +39205|AAAAAAAAGCJJAAAA|39205|10|53|25|AM|first|morning|| +39206|AAAAAAAAHCJJAAAA|39206|10|53|26|AM|first|morning|| +39207|AAAAAAAAICJJAAAA|39207|10|53|27|AM|first|morning|| +39208|AAAAAAAAJCJJAAAA|39208|10|53|28|AM|first|morning|| +39209|AAAAAAAAKCJJAAAA|39209|10|53|29|AM|first|morning|| +39210|AAAAAAAALCJJAAAA|39210|10|53|30|AM|first|morning|| +39211|AAAAAAAAMCJJAAAA|39211|10|53|31|AM|first|morning|| +39212|AAAAAAAANCJJAAAA|39212|10|53|32|AM|first|morning|| +39213|AAAAAAAAOCJJAAAA|39213|10|53|33|AM|first|morning|| +39214|AAAAAAAAPCJJAAAA|39214|10|53|34|AM|first|morning|| +39215|AAAAAAAAADJJAAAA|39215|10|53|35|AM|first|morning|| +39216|AAAAAAAABDJJAAAA|39216|10|53|36|AM|first|morning|| +39217|AAAAAAAACDJJAAAA|39217|10|53|37|AM|first|morning|| +39218|AAAAAAAADDJJAAAA|39218|10|53|38|AM|first|morning|| +39219|AAAAAAAAEDJJAAAA|39219|10|53|39|AM|first|morning|| +39220|AAAAAAAAFDJJAAAA|39220|10|53|40|AM|first|morning|| +39221|AAAAAAAAGDJJAAAA|39221|10|53|41|AM|first|morning|| +39222|AAAAAAAAHDJJAAAA|39222|10|53|42|AM|first|morning|| +39223|AAAAAAAAIDJJAAAA|39223|10|53|43|AM|first|morning|| +39224|AAAAAAAAJDJJAAAA|39224|10|53|44|AM|first|morning|| +39225|AAAAAAAAKDJJAAAA|39225|10|53|45|AM|first|morning|| +39226|AAAAAAAALDJJAAAA|39226|10|53|46|AM|first|morning|| +39227|AAAAAAAAMDJJAAAA|39227|10|53|47|AM|first|morning|| +39228|AAAAAAAANDJJAAAA|39228|10|53|48|AM|first|morning|| +39229|AAAAAAAAODJJAAAA|39229|10|53|49|AM|first|morning|| +39230|AAAAAAAAPDJJAAAA|39230|10|53|50|AM|first|morning|| +39231|AAAAAAAAAEJJAAAA|39231|10|53|51|AM|first|morning|| +39232|AAAAAAAABEJJAAAA|39232|10|53|52|AM|first|morning|| +39233|AAAAAAAACEJJAAAA|39233|10|53|53|AM|first|morning|| +39234|AAAAAAAADEJJAAAA|39234|10|53|54|AM|first|morning|| +39235|AAAAAAAAEEJJAAAA|39235|10|53|55|AM|first|morning|| +39236|AAAAAAAAFEJJAAAA|39236|10|53|56|AM|first|morning|| +39237|AAAAAAAAGEJJAAAA|39237|10|53|57|AM|first|morning|| +39238|AAAAAAAAHEJJAAAA|39238|10|53|58|AM|first|morning|| +39239|AAAAAAAAIEJJAAAA|39239|10|53|59|AM|first|morning|| +39240|AAAAAAAAJEJJAAAA|39240|10|54|0|AM|first|morning|| +39241|AAAAAAAAKEJJAAAA|39241|10|54|1|AM|first|morning|| +39242|AAAAAAAALEJJAAAA|39242|10|54|2|AM|first|morning|| +39243|AAAAAAAAMEJJAAAA|39243|10|54|3|AM|first|morning|| +39244|AAAAAAAANEJJAAAA|39244|10|54|4|AM|first|morning|| +39245|AAAAAAAAOEJJAAAA|39245|10|54|5|AM|first|morning|| +39246|AAAAAAAAPEJJAAAA|39246|10|54|6|AM|first|morning|| +39247|AAAAAAAAAFJJAAAA|39247|10|54|7|AM|first|morning|| +39248|AAAAAAAABFJJAAAA|39248|10|54|8|AM|first|morning|| +39249|AAAAAAAACFJJAAAA|39249|10|54|9|AM|first|morning|| +39250|AAAAAAAADFJJAAAA|39250|10|54|10|AM|first|morning|| +39251|AAAAAAAAEFJJAAAA|39251|10|54|11|AM|first|morning|| +39252|AAAAAAAAFFJJAAAA|39252|10|54|12|AM|first|morning|| +39253|AAAAAAAAGFJJAAAA|39253|10|54|13|AM|first|morning|| +39254|AAAAAAAAHFJJAAAA|39254|10|54|14|AM|first|morning|| +39255|AAAAAAAAIFJJAAAA|39255|10|54|15|AM|first|morning|| +39256|AAAAAAAAJFJJAAAA|39256|10|54|16|AM|first|morning|| +39257|AAAAAAAAKFJJAAAA|39257|10|54|17|AM|first|morning|| +39258|AAAAAAAALFJJAAAA|39258|10|54|18|AM|first|morning|| +39259|AAAAAAAAMFJJAAAA|39259|10|54|19|AM|first|morning|| +39260|AAAAAAAANFJJAAAA|39260|10|54|20|AM|first|morning|| +39261|AAAAAAAAOFJJAAAA|39261|10|54|21|AM|first|morning|| +39262|AAAAAAAAPFJJAAAA|39262|10|54|22|AM|first|morning|| +39263|AAAAAAAAAGJJAAAA|39263|10|54|23|AM|first|morning|| +39264|AAAAAAAABGJJAAAA|39264|10|54|24|AM|first|morning|| +39265|AAAAAAAACGJJAAAA|39265|10|54|25|AM|first|morning|| +39266|AAAAAAAADGJJAAAA|39266|10|54|26|AM|first|morning|| +39267|AAAAAAAAEGJJAAAA|39267|10|54|27|AM|first|morning|| +39268|AAAAAAAAFGJJAAAA|39268|10|54|28|AM|first|morning|| +39269|AAAAAAAAGGJJAAAA|39269|10|54|29|AM|first|morning|| +39270|AAAAAAAAHGJJAAAA|39270|10|54|30|AM|first|morning|| +39271|AAAAAAAAIGJJAAAA|39271|10|54|31|AM|first|morning|| +39272|AAAAAAAAJGJJAAAA|39272|10|54|32|AM|first|morning|| +39273|AAAAAAAAKGJJAAAA|39273|10|54|33|AM|first|morning|| +39274|AAAAAAAALGJJAAAA|39274|10|54|34|AM|first|morning|| +39275|AAAAAAAAMGJJAAAA|39275|10|54|35|AM|first|morning|| +39276|AAAAAAAANGJJAAAA|39276|10|54|36|AM|first|morning|| +39277|AAAAAAAAOGJJAAAA|39277|10|54|37|AM|first|morning|| +39278|AAAAAAAAPGJJAAAA|39278|10|54|38|AM|first|morning|| +39279|AAAAAAAAAHJJAAAA|39279|10|54|39|AM|first|morning|| +39280|AAAAAAAABHJJAAAA|39280|10|54|40|AM|first|morning|| +39281|AAAAAAAACHJJAAAA|39281|10|54|41|AM|first|morning|| +39282|AAAAAAAADHJJAAAA|39282|10|54|42|AM|first|morning|| +39283|AAAAAAAAEHJJAAAA|39283|10|54|43|AM|first|morning|| +39284|AAAAAAAAFHJJAAAA|39284|10|54|44|AM|first|morning|| +39285|AAAAAAAAGHJJAAAA|39285|10|54|45|AM|first|morning|| +39286|AAAAAAAAHHJJAAAA|39286|10|54|46|AM|first|morning|| +39287|AAAAAAAAIHJJAAAA|39287|10|54|47|AM|first|morning|| +39288|AAAAAAAAJHJJAAAA|39288|10|54|48|AM|first|morning|| +39289|AAAAAAAAKHJJAAAA|39289|10|54|49|AM|first|morning|| +39290|AAAAAAAALHJJAAAA|39290|10|54|50|AM|first|morning|| +39291|AAAAAAAAMHJJAAAA|39291|10|54|51|AM|first|morning|| +39292|AAAAAAAANHJJAAAA|39292|10|54|52|AM|first|morning|| +39293|AAAAAAAAOHJJAAAA|39293|10|54|53|AM|first|morning|| +39294|AAAAAAAAPHJJAAAA|39294|10|54|54|AM|first|morning|| +39295|AAAAAAAAAIJJAAAA|39295|10|54|55|AM|first|morning|| +39296|AAAAAAAABIJJAAAA|39296|10|54|56|AM|first|morning|| +39297|AAAAAAAACIJJAAAA|39297|10|54|57|AM|first|morning|| +39298|AAAAAAAADIJJAAAA|39298|10|54|58|AM|first|morning|| +39299|AAAAAAAAEIJJAAAA|39299|10|54|59|AM|first|morning|| +39300|AAAAAAAAFIJJAAAA|39300|10|55|0|AM|first|morning|| +39301|AAAAAAAAGIJJAAAA|39301|10|55|1|AM|first|morning|| +39302|AAAAAAAAHIJJAAAA|39302|10|55|2|AM|first|morning|| +39303|AAAAAAAAIIJJAAAA|39303|10|55|3|AM|first|morning|| +39304|AAAAAAAAJIJJAAAA|39304|10|55|4|AM|first|morning|| +39305|AAAAAAAAKIJJAAAA|39305|10|55|5|AM|first|morning|| +39306|AAAAAAAALIJJAAAA|39306|10|55|6|AM|first|morning|| +39307|AAAAAAAAMIJJAAAA|39307|10|55|7|AM|first|morning|| +39308|AAAAAAAANIJJAAAA|39308|10|55|8|AM|first|morning|| +39309|AAAAAAAAOIJJAAAA|39309|10|55|9|AM|first|morning|| +39310|AAAAAAAAPIJJAAAA|39310|10|55|10|AM|first|morning|| +39311|AAAAAAAAAJJJAAAA|39311|10|55|11|AM|first|morning|| +39312|AAAAAAAABJJJAAAA|39312|10|55|12|AM|first|morning|| +39313|AAAAAAAACJJJAAAA|39313|10|55|13|AM|first|morning|| +39314|AAAAAAAADJJJAAAA|39314|10|55|14|AM|first|morning|| +39315|AAAAAAAAEJJJAAAA|39315|10|55|15|AM|first|morning|| +39316|AAAAAAAAFJJJAAAA|39316|10|55|16|AM|first|morning|| +39317|AAAAAAAAGJJJAAAA|39317|10|55|17|AM|first|morning|| +39318|AAAAAAAAHJJJAAAA|39318|10|55|18|AM|first|morning|| +39319|AAAAAAAAIJJJAAAA|39319|10|55|19|AM|first|morning|| +39320|AAAAAAAAJJJJAAAA|39320|10|55|20|AM|first|morning|| +39321|AAAAAAAAKJJJAAAA|39321|10|55|21|AM|first|morning|| +39322|AAAAAAAALJJJAAAA|39322|10|55|22|AM|first|morning|| +39323|AAAAAAAAMJJJAAAA|39323|10|55|23|AM|first|morning|| +39324|AAAAAAAANJJJAAAA|39324|10|55|24|AM|first|morning|| +39325|AAAAAAAAOJJJAAAA|39325|10|55|25|AM|first|morning|| +39326|AAAAAAAAPJJJAAAA|39326|10|55|26|AM|first|morning|| +39327|AAAAAAAAAKJJAAAA|39327|10|55|27|AM|first|morning|| +39328|AAAAAAAABKJJAAAA|39328|10|55|28|AM|first|morning|| +39329|AAAAAAAACKJJAAAA|39329|10|55|29|AM|first|morning|| +39330|AAAAAAAADKJJAAAA|39330|10|55|30|AM|first|morning|| +39331|AAAAAAAAEKJJAAAA|39331|10|55|31|AM|first|morning|| +39332|AAAAAAAAFKJJAAAA|39332|10|55|32|AM|first|morning|| +39333|AAAAAAAAGKJJAAAA|39333|10|55|33|AM|first|morning|| +39334|AAAAAAAAHKJJAAAA|39334|10|55|34|AM|first|morning|| +39335|AAAAAAAAIKJJAAAA|39335|10|55|35|AM|first|morning|| +39336|AAAAAAAAJKJJAAAA|39336|10|55|36|AM|first|morning|| +39337|AAAAAAAAKKJJAAAA|39337|10|55|37|AM|first|morning|| +39338|AAAAAAAALKJJAAAA|39338|10|55|38|AM|first|morning|| +39339|AAAAAAAAMKJJAAAA|39339|10|55|39|AM|first|morning|| +39340|AAAAAAAANKJJAAAA|39340|10|55|40|AM|first|morning|| +39341|AAAAAAAAOKJJAAAA|39341|10|55|41|AM|first|morning|| +39342|AAAAAAAAPKJJAAAA|39342|10|55|42|AM|first|morning|| +39343|AAAAAAAAALJJAAAA|39343|10|55|43|AM|first|morning|| +39344|AAAAAAAABLJJAAAA|39344|10|55|44|AM|first|morning|| +39345|AAAAAAAACLJJAAAA|39345|10|55|45|AM|first|morning|| +39346|AAAAAAAADLJJAAAA|39346|10|55|46|AM|first|morning|| +39347|AAAAAAAAELJJAAAA|39347|10|55|47|AM|first|morning|| +39348|AAAAAAAAFLJJAAAA|39348|10|55|48|AM|first|morning|| +39349|AAAAAAAAGLJJAAAA|39349|10|55|49|AM|first|morning|| +39350|AAAAAAAAHLJJAAAA|39350|10|55|50|AM|first|morning|| +39351|AAAAAAAAILJJAAAA|39351|10|55|51|AM|first|morning|| +39352|AAAAAAAAJLJJAAAA|39352|10|55|52|AM|first|morning|| +39353|AAAAAAAAKLJJAAAA|39353|10|55|53|AM|first|morning|| +39354|AAAAAAAALLJJAAAA|39354|10|55|54|AM|first|morning|| +39355|AAAAAAAAMLJJAAAA|39355|10|55|55|AM|first|morning|| +39356|AAAAAAAANLJJAAAA|39356|10|55|56|AM|first|morning|| +39357|AAAAAAAAOLJJAAAA|39357|10|55|57|AM|first|morning|| +39358|AAAAAAAAPLJJAAAA|39358|10|55|58|AM|first|morning|| +39359|AAAAAAAAAMJJAAAA|39359|10|55|59|AM|first|morning|| +39360|AAAAAAAABMJJAAAA|39360|10|56|0|AM|first|morning|| +39361|AAAAAAAACMJJAAAA|39361|10|56|1|AM|first|morning|| +39362|AAAAAAAADMJJAAAA|39362|10|56|2|AM|first|morning|| +39363|AAAAAAAAEMJJAAAA|39363|10|56|3|AM|first|morning|| +39364|AAAAAAAAFMJJAAAA|39364|10|56|4|AM|first|morning|| +39365|AAAAAAAAGMJJAAAA|39365|10|56|5|AM|first|morning|| +39366|AAAAAAAAHMJJAAAA|39366|10|56|6|AM|first|morning|| +39367|AAAAAAAAIMJJAAAA|39367|10|56|7|AM|first|morning|| +39368|AAAAAAAAJMJJAAAA|39368|10|56|8|AM|first|morning|| +39369|AAAAAAAAKMJJAAAA|39369|10|56|9|AM|first|morning|| +39370|AAAAAAAALMJJAAAA|39370|10|56|10|AM|first|morning|| +39371|AAAAAAAAMMJJAAAA|39371|10|56|11|AM|first|morning|| +39372|AAAAAAAANMJJAAAA|39372|10|56|12|AM|first|morning|| +39373|AAAAAAAAOMJJAAAA|39373|10|56|13|AM|first|morning|| +39374|AAAAAAAAPMJJAAAA|39374|10|56|14|AM|first|morning|| +39375|AAAAAAAAANJJAAAA|39375|10|56|15|AM|first|morning|| +39376|AAAAAAAABNJJAAAA|39376|10|56|16|AM|first|morning|| +39377|AAAAAAAACNJJAAAA|39377|10|56|17|AM|first|morning|| +39378|AAAAAAAADNJJAAAA|39378|10|56|18|AM|first|morning|| +39379|AAAAAAAAENJJAAAA|39379|10|56|19|AM|first|morning|| +39380|AAAAAAAAFNJJAAAA|39380|10|56|20|AM|first|morning|| +39381|AAAAAAAAGNJJAAAA|39381|10|56|21|AM|first|morning|| +39382|AAAAAAAAHNJJAAAA|39382|10|56|22|AM|first|morning|| +39383|AAAAAAAAINJJAAAA|39383|10|56|23|AM|first|morning|| +39384|AAAAAAAAJNJJAAAA|39384|10|56|24|AM|first|morning|| +39385|AAAAAAAAKNJJAAAA|39385|10|56|25|AM|first|morning|| +39386|AAAAAAAALNJJAAAA|39386|10|56|26|AM|first|morning|| +39387|AAAAAAAAMNJJAAAA|39387|10|56|27|AM|first|morning|| +39388|AAAAAAAANNJJAAAA|39388|10|56|28|AM|first|morning|| +39389|AAAAAAAAONJJAAAA|39389|10|56|29|AM|first|morning|| +39390|AAAAAAAAPNJJAAAA|39390|10|56|30|AM|first|morning|| +39391|AAAAAAAAAOJJAAAA|39391|10|56|31|AM|first|morning|| +39392|AAAAAAAABOJJAAAA|39392|10|56|32|AM|first|morning|| +39393|AAAAAAAACOJJAAAA|39393|10|56|33|AM|first|morning|| +39394|AAAAAAAADOJJAAAA|39394|10|56|34|AM|first|morning|| +39395|AAAAAAAAEOJJAAAA|39395|10|56|35|AM|first|morning|| +39396|AAAAAAAAFOJJAAAA|39396|10|56|36|AM|first|morning|| +39397|AAAAAAAAGOJJAAAA|39397|10|56|37|AM|first|morning|| +39398|AAAAAAAAHOJJAAAA|39398|10|56|38|AM|first|morning|| +39399|AAAAAAAAIOJJAAAA|39399|10|56|39|AM|first|morning|| +39400|AAAAAAAAJOJJAAAA|39400|10|56|40|AM|first|morning|| +39401|AAAAAAAAKOJJAAAA|39401|10|56|41|AM|first|morning|| +39402|AAAAAAAALOJJAAAA|39402|10|56|42|AM|first|morning|| +39403|AAAAAAAAMOJJAAAA|39403|10|56|43|AM|first|morning|| +39404|AAAAAAAANOJJAAAA|39404|10|56|44|AM|first|morning|| +39405|AAAAAAAAOOJJAAAA|39405|10|56|45|AM|first|morning|| +39406|AAAAAAAAPOJJAAAA|39406|10|56|46|AM|first|morning|| +39407|AAAAAAAAAPJJAAAA|39407|10|56|47|AM|first|morning|| +39408|AAAAAAAABPJJAAAA|39408|10|56|48|AM|first|morning|| +39409|AAAAAAAACPJJAAAA|39409|10|56|49|AM|first|morning|| +39410|AAAAAAAADPJJAAAA|39410|10|56|50|AM|first|morning|| +39411|AAAAAAAAEPJJAAAA|39411|10|56|51|AM|first|morning|| +39412|AAAAAAAAFPJJAAAA|39412|10|56|52|AM|first|morning|| +39413|AAAAAAAAGPJJAAAA|39413|10|56|53|AM|first|morning|| +39414|AAAAAAAAHPJJAAAA|39414|10|56|54|AM|first|morning|| +39415|AAAAAAAAIPJJAAAA|39415|10|56|55|AM|first|morning|| +39416|AAAAAAAAJPJJAAAA|39416|10|56|56|AM|first|morning|| +39417|AAAAAAAAKPJJAAAA|39417|10|56|57|AM|first|morning|| +39418|AAAAAAAALPJJAAAA|39418|10|56|58|AM|first|morning|| +39419|AAAAAAAAMPJJAAAA|39419|10|56|59|AM|first|morning|| +39420|AAAAAAAANPJJAAAA|39420|10|57|0|AM|first|morning|| +39421|AAAAAAAAOPJJAAAA|39421|10|57|1|AM|first|morning|| +39422|AAAAAAAAPPJJAAAA|39422|10|57|2|AM|first|morning|| +39423|AAAAAAAAAAKJAAAA|39423|10|57|3|AM|first|morning|| +39424|AAAAAAAABAKJAAAA|39424|10|57|4|AM|first|morning|| +39425|AAAAAAAACAKJAAAA|39425|10|57|5|AM|first|morning|| +39426|AAAAAAAADAKJAAAA|39426|10|57|6|AM|first|morning|| +39427|AAAAAAAAEAKJAAAA|39427|10|57|7|AM|first|morning|| +39428|AAAAAAAAFAKJAAAA|39428|10|57|8|AM|first|morning|| +39429|AAAAAAAAGAKJAAAA|39429|10|57|9|AM|first|morning|| +39430|AAAAAAAAHAKJAAAA|39430|10|57|10|AM|first|morning|| +39431|AAAAAAAAIAKJAAAA|39431|10|57|11|AM|first|morning|| +39432|AAAAAAAAJAKJAAAA|39432|10|57|12|AM|first|morning|| +39433|AAAAAAAAKAKJAAAA|39433|10|57|13|AM|first|morning|| +39434|AAAAAAAALAKJAAAA|39434|10|57|14|AM|first|morning|| +39435|AAAAAAAAMAKJAAAA|39435|10|57|15|AM|first|morning|| +39436|AAAAAAAANAKJAAAA|39436|10|57|16|AM|first|morning|| +39437|AAAAAAAAOAKJAAAA|39437|10|57|17|AM|first|morning|| +39438|AAAAAAAAPAKJAAAA|39438|10|57|18|AM|first|morning|| +39439|AAAAAAAAABKJAAAA|39439|10|57|19|AM|first|morning|| +39440|AAAAAAAABBKJAAAA|39440|10|57|20|AM|first|morning|| +39441|AAAAAAAACBKJAAAA|39441|10|57|21|AM|first|morning|| +39442|AAAAAAAADBKJAAAA|39442|10|57|22|AM|first|morning|| +39443|AAAAAAAAEBKJAAAA|39443|10|57|23|AM|first|morning|| +39444|AAAAAAAAFBKJAAAA|39444|10|57|24|AM|first|morning|| +39445|AAAAAAAAGBKJAAAA|39445|10|57|25|AM|first|morning|| +39446|AAAAAAAAHBKJAAAA|39446|10|57|26|AM|first|morning|| +39447|AAAAAAAAIBKJAAAA|39447|10|57|27|AM|first|morning|| +39448|AAAAAAAAJBKJAAAA|39448|10|57|28|AM|first|morning|| +39449|AAAAAAAAKBKJAAAA|39449|10|57|29|AM|first|morning|| +39450|AAAAAAAALBKJAAAA|39450|10|57|30|AM|first|morning|| +39451|AAAAAAAAMBKJAAAA|39451|10|57|31|AM|first|morning|| +39452|AAAAAAAANBKJAAAA|39452|10|57|32|AM|first|morning|| +39453|AAAAAAAAOBKJAAAA|39453|10|57|33|AM|first|morning|| +39454|AAAAAAAAPBKJAAAA|39454|10|57|34|AM|first|morning|| +39455|AAAAAAAAACKJAAAA|39455|10|57|35|AM|first|morning|| +39456|AAAAAAAABCKJAAAA|39456|10|57|36|AM|first|morning|| +39457|AAAAAAAACCKJAAAA|39457|10|57|37|AM|first|morning|| +39458|AAAAAAAADCKJAAAA|39458|10|57|38|AM|first|morning|| +39459|AAAAAAAAECKJAAAA|39459|10|57|39|AM|first|morning|| +39460|AAAAAAAAFCKJAAAA|39460|10|57|40|AM|first|morning|| +39461|AAAAAAAAGCKJAAAA|39461|10|57|41|AM|first|morning|| +39462|AAAAAAAAHCKJAAAA|39462|10|57|42|AM|first|morning|| +39463|AAAAAAAAICKJAAAA|39463|10|57|43|AM|first|morning|| +39464|AAAAAAAAJCKJAAAA|39464|10|57|44|AM|first|morning|| +39465|AAAAAAAAKCKJAAAA|39465|10|57|45|AM|first|morning|| +39466|AAAAAAAALCKJAAAA|39466|10|57|46|AM|first|morning|| +39467|AAAAAAAAMCKJAAAA|39467|10|57|47|AM|first|morning|| +39468|AAAAAAAANCKJAAAA|39468|10|57|48|AM|first|morning|| +39469|AAAAAAAAOCKJAAAA|39469|10|57|49|AM|first|morning|| +39470|AAAAAAAAPCKJAAAA|39470|10|57|50|AM|first|morning|| +39471|AAAAAAAAADKJAAAA|39471|10|57|51|AM|first|morning|| +39472|AAAAAAAABDKJAAAA|39472|10|57|52|AM|first|morning|| +39473|AAAAAAAACDKJAAAA|39473|10|57|53|AM|first|morning|| +39474|AAAAAAAADDKJAAAA|39474|10|57|54|AM|first|morning|| +39475|AAAAAAAAEDKJAAAA|39475|10|57|55|AM|first|morning|| +39476|AAAAAAAAFDKJAAAA|39476|10|57|56|AM|first|morning|| +39477|AAAAAAAAGDKJAAAA|39477|10|57|57|AM|first|morning|| +39478|AAAAAAAAHDKJAAAA|39478|10|57|58|AM|first|morning|| +39479|AAAAAAAAIDKJAAAA|39479|10|57|59|AM|first|morning|| +39480|AAAAAAAAJDKJAAAA|39480|10|58|0|AM|first|morning|| +39481|AAAAAAAAKDKJAAAA|39481|10|58|1|AM|first|morning|| +39482|AAAAAAAALDKJAAAA|39482|10|58|2|AM|first|morning|| +39483|AAAAAAAAMDKJAAAA|39483|10|58|3|AM|first|morning|| +39484|AAAAAAAANDKJAAAA|39484|10|58|4|AM|first|morning|| +39485|AAAAAAAAODKJAAAA|39485|10|58|5|AM|first|morning|| +39486|AAAAAAAAPDKJAAAA|39486|10|58|6|AM|first|morning|| +39487|AAAAAAAAAEKJAAAA|39487|10|58|7|AM|first|morning|| +39488|AAAAAAAABEKJAAAA|39488|10|58|8|AM|first|morning|| +39489|AAAAAAAACEKJAAAA|39489|10|58|9|AM|first|morning|| +39490|AAAAAAAADEKJAAAA|39490|10|58|10|AM|first|morning|| +39491|AAAAAAAAEEKJAAAA|39491|10|58|11|AM|first|morning|| +39492|AAAAAAAAFEKJAAAA|39492|10|58|12|AM|first|morning|| +39493|AAAAAAAAGEKJAAAA|39493|10|58|13|AM|first|morning|| +39494|AAAAAAAAHEKJAAAA|39494|10|58|14|AM|first|morning|| +39495|AAAAAAAAIEKJAAAA|39495|10|58|15|AM|first|morning|| +39496|AAAAAAAAJEKJAAAA|39496|10|58|16|AM|first|morning|| +39497|AAAAAAAAKEKJAAAA|39497|10|58|17|AM|first|morning|| +39498|AAAAAAAALEKJAAAA|39498|10|58|18|AM|first|morning|| +39499|AAAAAAAAMEKJAAAA|39499|10|58|19|AM|first|morning|| +39500|AAAAAAAANEKJAAAA|39500|10|58|20|AM|first|morning|| +39501|AAAAAAAAOEKJAAAA|39501|10|58|21|AM|first|morning|| +39502|AAAAAAAAPEKJAAAA|39502|10|58|22|AM|first|morning|| +39503|AAAAAAAAAFKJAAAA|39503|10|58|23|AM|first|morning|| +39504|AAAAAAAABFKJAAAA|39504|10|58|24|AM|first|morning|| +39505|AAAAAAAACFKJAAAA|39505|10|58|25|AM|first|morning|| +39506|AAAAAAAADFKJAAAA|39506|10|58|26|AM|first|morning|| +39507|AAAAAAAAEFKJAAAA|39507|10|58|27|AM|first|morning|| +39508|AAAAAAAAFFKJAAAA|39508|10|58|28|AM|first|morning|| +39509|AAAAAAAAGFKJAAAA|39509|10|58|29|AM|first|morning|| +39510|AAAAAAAAHFKJAAAA|39510|10|58|30|AM|first|morning|| +39511|AAAAAAAAIFKJAAAA|39511|10|58|31|AM|first|morning|| +39512|AAAAAAAAJFKJAAAA|39512|10|58|32|AM|first|morning|| +39513|AAAAAAAAKFKJAAAA|39513|10|58|33|AM|first|morning|| +39514|AAAAAAAALFKJAAAA|39514|10|58|34|AM|first|morning|| +39515|AAAAAAAAMFKJAAAA|39515|10|58|35|AM|first|morning|| +39516|AAAAAAAANFKJAAAA|39516|10|58|36|AM|first|morning|| +39517|AAAAAAAAOFKJAAAA|39517|10|58|37|AM|first|morning|| +39518|AAAAAAAAPFKJAAAA|39518|10|58|38|AM|first|morning|| +39519|AAAAAAAAAGKJAAAA|39519|10|58|39|AM|first|morning|| +39520|AAAAAAAABGKJAAAA|39520|10|58|40|AM|first|morning|| +39521|AAAAAAAACGKJAAAA|39521|10|58|41|AM|first|morning|| +39522|AAAAAAAADGKJAAAA|39522|10|58|42|AM|first|morning|| +39523|AAAAAAAAEGKJAAAA|39523|10|58|43|AM|first|morning|| +39524|AAAAAAAAFGKJAAAA|39524|10|58|44|AM|first|morning|| +39525|AAAAAAAAGGKJAAAA|39525|10|58|45|AM|first|morning|| +39526|AAAAAAAAHGKJAAAA|39526|10|58|46|AM|first|morning|| +39527|AAAAAAAAIGKJAAAA|39527|10|58|47|AM|first|morning|| +39528|AAAAAAAAJGKJAAAA|39528|10|58|48|AM|first|morning|| +39529|AAAAAAAAKGKJAAAA|39529|10|58|49|AM|first|morning|| +39530|AAAAAAAALGKJAAAA|39530|10|58|50|AM|first|morning|| +39531|AAAAAAAAMGKJAAAA|39531|10|58|51|AM|first|morning|| +39532|AAAAAAAANGKJAAAA|39532|10|58|52|AM|first|morning|| +39533|AAAAAAAAOGKJAAAA|39533|10|58|53|AM|first|morning|| +39534|AAAAAAAAPGKJAAAA|39534|10|58|54|AM|first|morning|| +39535|AAAAAAAAAHKJAAAA|39535|10|58|55|AM|first|morning|| +39536|AAAAAAAABHKJAAAA|39536|10|58|56|AM|first|morning|| +39537|AAAAAAAACHKJAAAA|39537|10|58|57|AM|first|morning|| +39538|AAAAAAAADHKJAAAA|39538|10|58|58|AM|first|morning|| +39539|AAAAAAAAEHKJAAAA|39539|10|58|59|AM|first|morning|| +39540|AAAAAAAAFHKJAAAA|39540|10|59|0|AM|first|morning|| +39541|AAAAAAAAGHKJAAAA|39541|10|59|1|AM|first|morning|| +39542|AAAAAAAAHHKJAAAA|39542|10|59|2|AM|first|morning|| +39543|AAAAAAAAIHKJAAAA|39543|10|59|3|AM|first|morning|| +39544|AAAAAAAAJHKJAAAA|39544|10|59|4|AM|first|morning|| +39545|AAAAAAAAKHKJAAAA|39545|10|59|5|AM|first|morning|| +39546|AAAAAAAALHKJAAAA|39546|10|59|6|AM|first|morning|| +39547|AAAAAAAAMHKJAAAA|39547|10|59|7|AM|first|morning|| +39548|AAAAAAAANHKJAAAA|39548|10|59|8|AM|first|morning|| +39549|AAAAAAAAOHKJAAAA|39549|10|59|9|AM|first|morning|| +39550|AAAAAAAAPHKJAAAA|39550|10|59|10|AM|first|morning|| +39551|AAAAAAAAAIKJAAAA|39551|10|59|11|AM|first|morning|| +39552|AAAAAAAABIKJAAAA|39552|10|59|12|AM|first|morning|| +39553|AAAAAAAACIKJAAAA|39553|10|59|13|AM|first|morning|| +39554|AAAAAAAADIKJAAAA|39554|10|59|14|AM|first|morning|| +39555|AAAAAAAAEIKJAAAA|39555|10|59|15|AM|first|morning|| +39556|AAAAAAAAFIKJAAAA|39556|10|59|16|AM|first|morning|| +39557|AAAAAAAAGIKJAAAA|39557|10|59|17|AM|first|morning|| +39558|AAAAAAAAHIKJAAAA|39558|10|59|18|AM|first|morning|| +39559|AAAAAAAAIIKJAAAA|39559|10|59|19|AM|first|morning|| +39560|AAAAAAAAJIKJAAAA|39560|10|59|20|AM|first|morning|| +39561|AAAAAAAAKIKJAAAA|39561|10|59|21|AM|first|morning|| +39562|AAAAAAAALIKJAAAA|39562|10|59|22|AM|first|morning|| +39563|AAAAAAAAMIKJAAAA|39563|10|59|23|AM|first|morning|| +39564|AAAAAAAANIKJAAAA|39564|10|59|24|AM|first|morning|| +39565|AAAAAAAAOIKJAAAA|39565|10|59|25|AM|first|morning|| +39566|AAAAAAAAPIKJAAAA|39566|10|59|26|AM|first|morning|| +39567|AAAAAAAAAJKJAAAA|39567|10|59|27|AM|first|morning|| +39568|AAAAAAAABJKJAAAA|39568|10|59|28|AM|first|morning|| +39569|AAAAAAAACJKJAAAA|39569|10|59|29|AM|first|morning|| +39570|AAAAAAAADJKJAAAA|39570|10|59|30|AM|first|morning|| +39571|AAAAAAAAEJKJAAAA|39571|10|59|31|AM|first|morning|| +39572|AAAAAAAAFJKJAAAA|39572|10|59|32|AM|first|morning|| +39573|AAAAAAAAGJKJAAAA|39573|10|59|33|AM|first|morning|| +39574|AAAAAAAAHJKJAAAA|39574|10|59|34|AM|first|morning|| +39575|AAAAAAAAIJKJAAAA|39575|10|59|35|AM|first|morning|| +39576|AAAAAAAAJJKJAAAA|39576|10|59|36|AM|first|morning|| +39577|AAAAAAAAKJKJAAAA|39577|10|59|37|AM|first|morning|| +39578|AAAAAAAALJKJAAAA|39578|10|59|38|AM|first|morning|| +39579|AAAAAAAAMJKJAAAA|39579|10|59|39|AM|first|morning|| +39580|AAAAAAAANJKJAAAA|39580|10|59|40|AM|first|morning|| +39581|AAAAAAAAOJKJAAAA|39581|10|59|41|AM|first|morning|| +39582|AAAAAAAAPJKJAAAA|39582|10|59|42|AM|first|morning|| +39583|AAAAAAAAAKKJAAAA|39583|10|59|43|AM|first|morning|| +39584|AAAAAAAABKKJAAAA|39584|10|59|44|AM|first|morning|| +39585|AAAAAAAACKKJAAAA|39585|10|59|45|AM|first|morning|| +39586|AAAAAAAADKKJAAAA|39586|10|59|46|AM|first|morning|| +39587|AAAAAAAAEKKJAAAA|39587|10|59|47|AM|first|morning|| +39588|AAAAAAAAFKKJAAAA|39588|10|59|48|AM|first|morning|| +39589|AAAAAAAAGKKJAAAA|39589|10|59|49|AM|first|morning|| +39590|AAAAAAAAHKKJAAAA|39590|10|59|50|AM|first|morning|| +39591|AAAAAAAAIKKJAAAA|39591|10|59|51|AM|first|morning|| +39592|AAAAAAAAJKKJAAAA|39592|10|59|52|AM|first|morning|| +39593|AAAAAAAAKKKJAAAA|39593|10|59|53|AM|first|morning|| +39594|AAAAAAAALKKJAAAA|39594|10|59|54|AM|first|morning|| +39595|AAAAAAAAMKKJAAAA|39595|10|59|55|AM|first|morning|| +39596|AAAAAAAANKKJAAAA|39596|10|59|56|AM|first|morning|| +39597|AAAAAAAAOKKJAAAA|39597|10|59|57|AM|first|morning|| +39598|AAAAAAAAPKKJAAAA|39598|10|59|58|AM|first|morning|| +39599|AAAAAAAAALKJAAAA|39599|10|59|59|AM|first|morning|| +39600|AAAAAAAABLKJAAAA|39600|11|0|0|AM|first|morning|| +39601|AAAAAAAACLKJAAAA|39601|11|0|1|AM|first|morning|| +39602|AAAAAAAADLKJAAAA|39602|11|0|2|AM|first|morning|| +39603|AAAAAAAAELKJAAAA|39603|11|0|3|AM|first|morning|| +39604|AAAAAAAAFLKJAAAA|39604|11|0|4|AM|first|morning|| +39605|AAAAAAAAGLKJAAAA|39605|11|0|5|AM|first|morning|| +39606|AAAAAAAAHLKJAAAA|39606|11|0|6|AM|first|morning|| +39607|AAAAAAAAILKJAAAA|39607|11|0|7|AM|first|morning|| +39608|AAAAAAAAJLKJAAAA|39608|11|0|8|AM|first|morning|| +39609|AAAAAAAAKLKJAAAA|39609|11|0|9|AM|first|morning|| +39610|AAAAAAAALLKJAAAA|39610|11|0|10|AM|first|morning|| +39611|AAAAAAAAMLKJAAAA|39611|11|0|11|AM|first|morning|| +39612|AAAAAAAANLKJAAAA|39612|11|0|12|AM|first|morning|| +39613|AAAAAAAAOLKJAAAA|39613|11|0|13|AM|first|morning|| +39614|AAAAAAAAPLKJAAAA|39614|11|0|14|AM|first|morning|| +39615|AAAAAAAAAMKJAAAA|39615|11|0|15|AM|first|morning|| +39616|AAAAAAAABMKJAAAA|39616|11|0|16|AM|first|morning|| +39617|AAAAAAAACMKJAAAA|39617|11|0|17|AM|first|morning|| +39618|AAAAAAAADMKJAAAA|39618|11|0|18|AM|first|morning|| +39619|AAAAAAAAEMKJAAAA|39619|11|0|19|AM|first|morning|| +39620|AAAAAAAAFMKJAAAA|39620|11|0|20|AM|first|morning|| +39621|AAAAAAAAGMKJAAAA|39621|11|0|21|AM|first|morning|| +39622|AAAAAAAAHMKJAAAA|39622|11|0|22|AM|first|morning|| +39623|AAAAAAAAIMKJAAAA|39623|11|0|23|AM|first|morning|| +39624|AAAAAAAAJMKJAAAA|39624|11|0|24|AM|first|morning|| +39625|AAAAAAAAKMKJAAAA|39625|11|0|25|AM|first|morning|| +39626|AAAAAAAALMKJAAAA|39626|11|0|26|AM|first|morning|| +39627|AAAAAAAAMMKJAAAA|39627|11|0|27|AM|first|morning|| +39628|AAAAAAAANMKJAAAA|39628|11|0|28|AM|first|morning|| +39629|AAAAAAAAOMKJAAAA|39629|11|0|29|AM|first|morning|| +39630|AAAAAAAAPMKJAAAA|39630|11|0|30|AM|first|morning|| +39631|AAAAAAAAANKJAAAA|39631|11|0|31|AM|first|morning|| +39632|AAAAAAAABNKJAAAA|39632|11|0|32|AM|first|morning|| +39633|AAAAAAAACNKJAAAA|39633|11|0|33|AM|first|morning|| +39634|AAAAAAAADNKJAAAA|39634|11|0|34|AM|first|morning|| +39635|AAAAAAAAENKJAAAA|39635|11|0|35|AM|first|morning|| +39636|AAAAAAAAFNKJAAAA|39636|11|0|36|AM|first|morning|| +39637|AAAAAAAAGNKJAAAA|39637|11|0|37|AM|first|morning|| +39638|AAAAAAAAHNKJAAAA|39638|11|0|38|AM|first|morning|| +39639|AAAAAAAAINKJAAAA|39639|11|0|39|AM|first|morning|| +39640|AAAAAAAAJNKJAAAA|39640|11|0|40|AM|first|morning|| +39641|AAAAAAAAKNKJAAAA|39641|11|0|41|AM|first|morning|| +39642|AAAAAAAALNKJAAAA|39642|11|0|42|AM|first|morning|| +39643|AAAAAAAAMNKJAAAA|39643|11|0|43|AM|first|morning|| +39644|AAAAAAAANNKJAAAA|39644|11|0|44|AM|first|morning|| +39645|AAAAAAAAONKJAAAA|39645|11|0|45|AM|first|morning|| +39646|AAAAAAAAPNKJAAAA|39646|11|0|46|AM|first|morning|| +39647|AAAAAAAAAOKJAAAA|39647|11|0|47|AM|first|morning|| +39648|AAAAAAAABOKJAAAA|39648|11|0|48|AM|first|morning|| +39649|AAAAAAAACOKJAAAA|39649|11|0|49|AM|first|morning|| +39650|AAAAAAAADOKJAAAA|39650|11|0|50|AM|first|morning|| +39651|AAAAAAAAEOKJAAAA|39651|11|0|51|AM|first|morning|| +39652|AAAAAAAAFOKJAAAA|39652|11|0|52|AM|first|morning|| +39653|AAAAAAAAGOKJAAAA|39653|11|0|53|AM|first|morning|| +39654|AAAAAAAAHOKJAAAA|39654|11|0|54|AM|first|morning|| +39655|AAAAAAAAIOKJAAAA|39655|11|0|55|AM|first|morning|| +39656|AAAAAAAAJOKJAAAA|39656|11|0|56|AM|first|morning|| +39657|AAAAAAAAKOKJAAAA|39657|11|0|57|AM|first|morning|| +39658|AAAAAAAALOKJAAAA|39658|11|0|58|AM|first|morning|| +39659|AAAAAAAAMOKJAAAA|39659|11|0|59|AM|first|morning|| +39660|AAAAAAAANOKJAAAA|39660|11|1|0|AM|first|morning|| +39661|AAAAAAAAOOKJAAAA|39661|11|1|1|AM|first|morning|| +39662|AAAAAAAAPOKJAAAA|39662|11|1|2|AM|first|morning|| +39663|AAAAAAAAAPKJAAAA|39663|11|1|3|AM|first|morning|| +39664|AAAAAAAABPKJAAAA|39664|11|1|4|AM|first|morning|| +39665|AAAAAAAACPKJAAAA|39665|11|1|5|AM|first|morning|| +39666|AAAAAAAADPKJAAAA|39666|11|1|6|AM|first|morning|| +39667|AAAAAAAAEPKJAAAA|39667|11|1|7|AM|first|morning|| +39668|AAAAAAAAFPKJAAAA|39668|11|1|8|AM|first|morning|| +39669|AAAAAAAAGPKJAAAA|39669|11|1|9|AM|first|morning|| +39670|AAAAAAAAHPKJAAAA|39670|11|1|10|AM|first|morning|| +39671|AAAAAAAAIPKJAAAA|39671|11|1|11|AM|first|morning|| +39672|AAAAAAAAJPKJAAAA|39672|11|1|12|AM|first|morning|| +39673|AAAAAAAAKPKJAAAA|39673|11|1|13|AM|first|morning|| +39674|AAAAAAAALPKJAAAA|39674|11|1|14|AM|first|morning|| +39675|AAAAAAAAMPKJAAAA|39675|11|1|15|AM|first|morning|| +39676|AAAAAAAANPKJAAAA|39676|11|1|16|AM|first|morning|| +39677|AAAAAAAAOPKJAAAA|39677|11|1|17|AM|first|morning|| +39678|AAAAAAAAPPKJAAAA|39678|11|1|18|AM|first|morning|| +39679|AAAAAAAAAALJAAAA|39679|11|1|19|AM|first|morning|| +39680|AAAAAAAABALJAAAA|39680|11|1|20|AM|first|morning|| +39681|AAAAAAAACALJAAAA|39681|11|1|21|AM|first|morning|| +39682|AAAAAAAADALJAAAA|39682|11|1|22|AM|first|morning|| +39683|AAAAAAAAEALJAAAA|39683|11|1|23|AM|first|morning|| +39684|AAAAAAAAFALJAAAA|39684|11|1|24|AM|first|morning|| +39685|AAAAAAAAGALJAAAA|39685|11|1|25|AM|first|morning|| +39686|AAAAAAAAHALJAAAA|39686|11|1|26|AM|first|morning|| +39687|AAAAAAAAIALJAAAA|39687|11|1|27|AM|first|morning|| +39688|AAAAAAAAJALJAAAA|39688|11|1|28|AM|first|morning|| +39689|AAAAAAAAKALJAAAA|39689|11|1|29|AM|first|morning|| +39690|AAAAAAAALALJAAAA|39690|11|1|30|AM|first|morning|| +39691|AAAAAAAAMALJAAAA|39691|11|1|31|AM|first|morning|| +39692|AAAAAAAANALJAAAA|39692|11|1|32|AM|first|morning|| +39693|AAAAAAAAOALJAAAA|39693|11|1|33|AM|first|morning|| +39694|AAAAAAAAPALJAAAA|39694|11|1|34|AM|first|morning|| +39695|AAAAAAAAABLJAAAA|39695|11|1|35|AM|first|morning|| +39696|AAAAAAAABBLJAAAA|39696|11|1|36|AM|first|morning|| +39697|AAAAAAAACBLJAAAA|39697|11|1|37|AM|first|morning|| +39698|AAAAAAAADBLJAAAA|39698|11|1|38|AM|first|morning|| +39699|AAAAAAAAEBLJAAAA|39699|11|1|39|AM|first|morning|| +39700|AAAAAAAAFBLJAAAA|39700|11|1|40|AM|first|morning|| +39701|AAAAAAAAGBLJAAAA|39701|11|1|41|AM|first|morning|| +39702|AAAAAAAAHBLJAAAA|39702|11|1|42|AM|first|morning|| +39703|AAAAAAAAIBLJAAAA|39703|11|1|43|AM|first|morning|| +39704|AAAAAAAAJBLJAAAA|39704|11|1|44|AM|first|morning|| +39705|AAAAAAAAKBLJAAAA|39705|11|1|45|AM|first|morning|| +39706|AAAAAAAALBLJAAAA|39706|11|1|46|AM|first|morning|| +39707|AAAAAAAAMBLJAAAA|39707|11|1|47|AM|first|morning|| +39708|AAAAAAAANBLJAAAA|39708|11|1|48|AM|first|morning|| +39709|AAAAAAAAOBLJAAAA|39709|11|1|49|AM|first|morning|| +39710|AAAAAAAAPBLJAAAA|39710|11|1|50|AM|first|morning|| +39711|AAAAAAAAACLJAAAA|39711|11|1|51|AM|first|morning|| +39712|AAAAAAAABCLJAAAA|39712|11|1|52|AM|first|morning|| +39713|AAAAAAAACCLJAAAA|39713|11|1|53|AM|first|morning|| +39714|AAAAAAAADCLJAAAA|39714|11|1|54|AM|first|morning|| +39715|AAAAAAAAECLJAAAA|39715|11|1|55|AM|first|morning|| +39716|AAAAAAAAFCLJAAAA|39716|11|1|56|AM|first|morning|| +39717|AAAAAAAAGCLJAAAA|39717|11|1|57|AM|first|morning|| +39718|AAAAAAAAHCLJAAAA|39718|11|1|58|AM|first|morning|| +39719|AAAAAAAAICLJAAAA|39719|11|1|59|AM|first|morning|| +39720|AAAAAAAAJCLJAAAA|39720|11|2|0|AM|first|morning|| +39721|AAAAAAAAKCLJAAAA|39721|11|2|1|AM|first|morning|| +39722|AAAAAAAALCLJAAAA|39722|11|2|2|AM|first|morning|| +39723|AAAAAAAAMCLJAAAA|39723|11|2|3|AM|first|morning|| +39724|AAAAAAAANCLJAAAA|39724|11|2|4|AM|first|morning|| +39725|AAAAAAAAOCLJAAAA|39725|11|2|5|AM|first|morning|| +39726|AAAAAAAAPCLJAAAA|39726|11|2|6|AM|first|morning|| +39727|AAAAAAAAADLJAAAA|39727|11|2|7|AM|first|morning|| +39728|AAAAAAAABDLJAAAA|39728|11|2|8|AM|first|morning|| +39729|AAAAAAAACDLJAAAA|39729|11|2|9|AM|first|morning|| +39730|AAAAAAAADDLJAAAA|39730|11|2|10|AM|first|morning|| +39731|AAAAAAAAEDLJAAAA|39731|11|2|11|AM|first|morning|| +39732|AAAAAAAAFDLJAAAA|39732|11|2|12|AM|first|morning|| +39733|AAAAAAAAGDLJAAAA|39733|11|2|13|AM|first|morning|| +39734|AAAAAAAAHDLJAAAA|39734|11|2|14|AM|first|morning|| +39735|AAAAAAAAIDLJAAAA|39735|11|2|15|AM|first|morning|| +39736|AAAAAAAAJDLJAAAA|39736|11|2|16|AM|first|morning|| +39737|AAAAAAAAKDLJAAAA|39737|11|2|17|AM|first|morning|| +39738|AAAAAAAALDLJAAAA|39738|11|2|18|AM|first|morning|| +39739|AAAAAAAAMDLJAAAA|39739|11|2|19|AM|first|morning|| +39740|AAAAAAAANDLJAAAA|39740|11|2|20|AM|first|morning|| +39741|AAAAAAAAODLJAAAA|39741|11|2|21|AM|first|morning|| +39742|AAAAAAAAPDLJAAAA|39742|11|2|22|AM|first|morning|| +39743|AAAAAAAAAELJAAAA|39743|11|2|23|AM|first|morning|| +39744|AAAAAAAABELJAAAA|39744|11|2|24|AM|first|morning|| +39745|AAAAAAAACELJAAAA|39745|11|2|25|AM|first|morning|| +39746|AAAAAAAADELJAAAA|39746|11|2|26|AM|first|morning|| +39747|AAAAAAAAEELJAAAA|39747|11|2|27|AM|first|morning|| +39748|AAAAAAAAFELJAAAA|39748|11|2|28|AM|first|morning|| +39749|AAAAAAAAGELJAAAA|39749|11|2|29|AM|first|morning|| +39750|AAAAAAAAHELJAAAA|39750|11|2|30|AM|first|morning|| +39751|AAAAAAAAIELJAAAA|39751|11|2|31|AM|first|morning|| +39752|AAAAAAAAJELJAAAA|39752|11|2|32|AM|first|morning|| +39753|AAAAAAAAKELJAAAA|39753|11|2|33|AM|first|morning|| +39754|AAAAAAAALELJAAAA|39754|11|2|34|AM|first|morning|| +39755|AAAAAAAAMELJAAAA|39755|11|2|35|AM|first|morning|| +39756|AAAAAAAANELJAAAA|39756|11|2|36|AM|first|morning|| +39757|AAAAAAAAOELJAAAA|39757|11|2|37|AM|first|morning|| +39758|AAAAAAAAPELJAAAA|39758|11|2|38|AM|first|morning|| +39759|AAAAAAAAAFLJAAAA|39759|11|2|39|AM|first|morning|| +39760|AAAAAAAABFLJAAAA|39760|11|2|40|AM|first|morning|| +39761|AAAAAAAACFLJAAAA|39761|11|2|41|AM|first|morning|| +39762|AAAAAAAADFLJAAAA|39762|11|2|42|AM|first|morning|| +39763|AAAAAAAAEFLJAAAA|39763|11|2|43|AM|first|morning|| +39764|AAAAAAAAFFLJAAAA|39764|11|2|44|AM|first|morning|| +39765|AAAAAAAAGFLJAAAA|39765|11|2|45|AM|first|morning|| +39766|AAAAAAAAHFLJAAAA|39766|11|2|46|AM|first|morning|| +39767|AAAAAAAAIFLJAAAA|39767|11|2|47|AM|first|morning|| +39768|AAAAAAAAJFLJAAAA|39768|11|2|48|AM|first|morning|| +39769|AAAAAAAAKFLJAAAA|39769|11|2|49|AM|first|morning|| +39770|AAAAAAAALFLJAAAA|39770|11|2|50|AM|first|morning|| +39771|AAAAAAAAMFLJAAAA|39771|11|2|51|AM|first|morning|| +39772|AAAAAAAANFLJAAAA|39772|11|2|52|AM|first|morning|| +39773|AAAAAAAAOFLJAAAA|39773|11|2|53|AM|first|morning|| +39774|AAAAAAAAPFLJAAAA|39774|11|2|54|AM|first|morning|| +39775|AAAAAAAAAGLJAAAA|39775|11|2|55|AM|first|morning|| +39776|AAAAAAAABGLJAAAA|39776|11|2|56|AM|first|morning|| +39777|AAAAAAAACGLJAAAA|39777|11|2|57|AM|first|morning|| +39778|AAAAAAAADGLJAAAA|39778|11|2|58|AM|first|morning|| +39779|AAAAAAAAEGLJAAAA|39779|11|2|59|AM|first|morning|| +39780|AAAAAAAAFGLJAAAA|39780|11|3|0|AM|first|morning|| +39781|AAAAAAAAGGLJAAAA|39781|11|3|1|AM|first|morning|| +39782|AAAAAAAAHGLJAAAA|39782|11|3|2|AM|first|morning|| +39783|AAAAAAAAIGLJAAAA|39783|11|3|3|AM|first|morning|| +39784|AAAAAAAAJGLJAAAA|39784|11|3|4|AM|first|morning|| +39785|AAAAAAAAKGLJAAAA|39785|11|3|5|AM|first|morning|| +39786|AAAAAAAALGLJAAAA|39786|11|3|6|AM|first|morning|| +39787|AAAAAAAAMGLJAAAA|39787|11|3|7|AM|first|morning|| +39788|AAAAAAAANGLJAAAA|39788|11|3|8|AM|first|morning|| +39789|AAAAAAAAOGLJAAAA|39789|11|3|9|AM|first|morning|| +39790|AAAAAAAAPGLJAAAA|39790|11|3|10|AM|first|morning|| +39791|AAAAAAAAAHLJAAAA|39791|11|3|11|AM|first|morning|| +39792|AAAAAAAABHLJAAAA|39792|11|3|12|AM|first|morning|| +39793|AAAAAAAACHLJAAAA|39793|11|3|13|AM|first|morning|| +39794|AAAAAAAADHLJAAAA|39794|11|3|14|AM|first|morning|| +39795|AAAAAAAAEHLJAAAA|39795|11|3|15|AM|first|morning|| +39796|AAAAAAAAFHLJAAAA|39796|11|3|16|AM|first|morning|| +39797|AAAAAAAAGHLJAAAA|39797|11|3|17|AM|first|morning|| +39798|AAAAAAAAHHLJAAAA|39798|11|3|18|AM|first|morning|| +39799|AAAAAAAAIHLJAAAA|39799|11|3|19|AM|first|morning|| +39800|AAAAAAAAJHLJAAAA|39800|11|3|20|AM|first|morning|| +39801|AAAAAAAAKHLJAAAA|39801|11|3|21|AM|first|morning|| +39802|AAAAAAAALHLJAAAA|39802|11|3|22|AM|first|morning|| +39803|AAAAAAAAMHLJAAAA|39803|11|3|23|AM|first|morning|| +39804|AAAAAAAANHLJAAAA|39804|11|3|24|AM|first|morning|| +39805|AAAAAAAAOHLJAAAA|39805|11|3|25|AM|first|morning|| +39806|AAAAAAAAPHLJAAAA|39806|11|3|26|AM|first|morning|| +39807|AAAAAAAAAILJAAAA|39807|11|3|27|AM|first|morning|| +39808|AAAAAAAABILJAAAA|39808|11|3|28|AM|first|morning|| +39809|AAAAAAAACILJAAAA|39809|11|3|29|AM|first|morning|| +39810|AAAAAAAADILJAAAA|39810|11|3|30|AM|first|morning|| +39811|AAAAAAAAEILJAAAA|39811|11|3|31|AM|first|morning|| +39812|AAAAAAAAFILJAAAA|39812|11|3|32|AM|first|morning|| +39813|AAAAAAAAGILJAAAA|39813|11|3|33|AM|first|morning|| +39814|AAAAAAAAHILJAAAA|39814|11|3|34|AM|first|morning|| +39815|AAAAAAAAIILJAAAA|39815|11|3|35|AM|first|morning|| +39816|AAAAAAAAJILJAAAA|39816|11|3|36|AM|first|morning|| +39817|AAAAAAAAKILJAAAA|39817|11|3|37|AM|first|morning|| +39818|AAAAAAAALILJAAAA|39818|11|3|38|AM|first|morning|| +39819|AAAAAAAAMILJAAAA|39819|11|3|39|AM|first|morning|| +39820|AAAAAAAANILJAAAA|39820|11|3|40|AM|first|morning|| +39821|AAAAAAAAOILJAAAA|39821|11|3|41|AM|first|morning|| +39822|AAAAAAAAPILJAAAA|39822|11|3|42|AM|first|morning|| +39823|AAAAAAAAAJLJAAAA|39823|11|3|43|AM|first|morning|| +39824|AAAAAAAABJLJAAAA|39824|11|3|44|AM|first|morning|| +39825|AAAAAAAACJLJAAAA|39825|11|3|45|AM|first|morning|| +39826|AAAAAAAADJLJAAAA|39826|11|3|46|AM|first|morning|| +39827|AAAAAAAAEJLJAAAA|39827|11|3|47|AM|first|morning|| +39828|AAAAAAAAFJLJAAAA|39828|11|3|48|AM|first|morning|| +39829|AAAAAAAAGJLJAAAA|39829|11|3|49|AM|first|morning|| +39830|AAAAAAAAHJLJAAAA|39830|11|3|50|AM|first|morning|| +39831|AAAAAAAAIJLJAAAA|39831|11|3|51|AM|first|morning|| +39832|AAAAAAAAJJLJAAAA|39832|11|3|52|AM|first|morning|| +39833|AAAAAAAAKJLJAAAA|39833|11|3|53|AM|first|morning|| +39834|AAAAAAAALJLJAAAA|39834|11|3|54|AM|first|morning|| +39835|AAAAAAAAMJLJAAAA|39835|11|3|55|AM|first|morning|| +39836|AAAAAAAANJLJAAAA|39836|11|3|56|AM|first|morning|| +39837|AAAAAAAAOJLJAAAA|39837|11|3|57|AM|first|morning|| +39838|AAAAAAAAPJLJAAAA|39838|11|3|58|AM|first|morning|| +39839|AAAAAAAAAKLJAAAA|39839|11|3|59|AM|first|morning|| +39840|AAAAAAAABKLJAAAA|39840|11|4|0|AM|first|morning|| +39841|AAAAAAAACKLJAAAA|39841|11|4|1|AM|first|morning|| +39842|AAAAAAAADKLJAAAA|39842|11|4|2|AM|first|morning|| +39843|AAAAAAAAEKLJAAAA|39843|11|4|3|AM|first|morning|| +39844|AAAAAAAAFKLJAAAA|39844|11|4|4|AM|first|morning|| +39845|AAAAAAAAGKLJAAAA|39845|11|4|5|AM|first|morning|| +39846|AAAAAAAAHKLJAAAA|39846|11|4|6|AM|first|morning|| +39847|AAAAAAAAIKLJAAAA|39847|11|4|7|AM|first|morning|| +39848|AAAAAAAAJKLJAAAA|39848|11|4|8|AM|first|morning|| +39849|AAAAAAAAKKLJAAAA|39849|11|4|9|AM|first|morning|| +39850|AAAAAAAALKLJAAAA|39850|11|4|10|AM|first|morning|| +39851|AAAAAAAAMKLJAAAA|39851|11|4|11|AM|first|morning|| +39852|AAAAAAAANKLJAAAA|39852|11|4|12|AM|first|morning|| +39853|AAAAAAAAOKLJAAAA|39853|11|4|13|AM|first|morning|| +39854|AAAAAAAAPKLJAAAA|39854|11|4|14|AM|first|morning|| +39855|AAAAAAAAALLJAAAA|39855|11|4|15|AM|first|morning|| +39856|AAAAAAAABLLJAAAA|39856|11|4|16|AM|first|morning|| +39857|AAAAAAAACLLJAAAA|39857|11|4|17|AM|first|morning|| +39858|AAAAAAAADLLJAAAA|39858|11|4|18|AM|first|morning|| +39859|AAAAAAAAELLJAAAA|39859|11|4|19|AM|first|morning|| +39860|AAAAAAAAFLLJAAAA|39860|11|4|20|AM|first|morning|| +39861|AAAAAAAAGLLJAAAA|39861|11|4|21|AM|first|morning|| +39862|AAAAAAAAHLLJAAAA|39862|11|4|22|AM|first|morning|| +39863|AAAAAAAAILLJAAAA|39863|11|4|23|AM|first|morning|| +39864|AAAAAAAAJLLJAAAA|39864|11|4|24|AM|first|morning|| +39865|AAAAAAAAKLLJAAAA|39865|11|4|25|AM|first|morning|| +39866|AAAAAAAALLLJAAAA|39866|11|4|26|AM|first|morning|| +39867|AAAAAAAAMLLJAAAA|39867|11|4|27|AM|first|morning|| +39868|AAAAAAAANLLJAAAA|39868|11|4|28|AM|first|morning|| +39869|AAAAAAAAOLLJAAAA|39869|11|4|29|AM|first|morning|| +39870|AAAAAAAAPLLJAAAA|39870|11|4|30|AM|first|morning|| +39871|AAAAAAAAAMLJAAAA|39871|11|4|31|AM|first|morning|| +39872|AAAAAAAABMLJAAAA|39872|11|4|32|AM|first|morning|| +39873|AAAAAAAACMLJAAAA|39873|11|4|33|AM|first|morning|| +39874|AAAAAAAADMLJAAAA|39874|11|4|34|AM|first|morning|| +39875|AAAAAAAAEMLJAAAA|39875|11|4|35|AM|first|morning|| +39876|AAAAAAAAFMLJAAAA|39876|11|4|36|AM|first|morning|| +39877|AAAAAAAAGMLJAAAA|39877|11|4|37|AM|first|morning|| +39878|AAAAAAAAHMLJAAAA|39878|11|4|38|AM|first|morning|| +39879|AAAAAAAAIMLJAAAA|39879|11|4|39|AM|first|morning|| +39880|AAAAAAAAJMLJAAAA|39880|11|4|40|AM|first|morning|| +39881|AAAAAAAAKMLJAAAA|39881|11|4|41|AM|first|morning|| +39882|AAAAAAAALMLJAAAA|39882|11|4|42|AM|first|morning|| +39883|AAAAAAAAMMLJAAAA|39883|11|4|43|AM|first|morning|| +39884|AAAAAAAANMLJAAAA|39884|11|4|44|AM|first|morning|| +39885|AAAAAAAAOMLJAAAA|39885|11|4|45|AM|first|morning|| +39886|AAAAAAAAPMLJAAAA|39886|11|4|46|AM|first|morning|| +39887|AAAAAAAAANLJAAAA|39887|11|4|47|AM|first|morning|| +39888|AAAAAAAABNLJAAAA|39888|11|4|48|AM|first|morning|| +39889|AAAAAAAACNLJAAAA|39889|11|4|49|AM|first|morning|| +39890|AAAAAAAADNLJAAAA|39890|11|4|50|AM|first|morning|| +39891|AAAAAAAAENLJAAAA|39891|11|4|51|AM|first|morning|| +39892|AAAAAAAAFNLJAAAA|39892|11|4|52|AM|first|morning|| +39893|AAAAAAAAGNLJAAAA|39893|11|4|53|AM|first|morning|| +39894|AAAAAAAAHNLJAAAA|39894|11|4|54|AM|first|morning|| +39895|AAAAAAAAINLJAAAA|39895|11|4|55|AM|first|morning|| +39896|AAAAAAAAJNLJAAAA|39896|11|4|56|AM|first|morning|| +39897|AAAAAAAAKNLJAAAA|39897|11|4|57|AM|first|morning|| +39898|AAAAAAAALNLJAAAA|39898|11|4|58|AM|first|morning|| +39899|AAAAAAAAMNLJAAAA|39899|11|4|59|AM|first|morning|| +39900|AAAAAAAANNLJAAAA|39900|11|5|0|AM|first|morning|| +39901|AAAAAAAAONLJAAAA|39901|11|5|1|AM|first|morning|| +39902|AAAAAAAAPNLJAAAA|39902|11|5|2|AM|first|morning|| +39903|AAAAAAAAAOLJAAAA|39903|11|5|3|AM|first|morning|| +39904|AAAAAAAABOLJAAAA|39904|11|5|4|AM|first|morning|| +39905|AAAAAAAACOLJAAAA|39905|11|5|5|AM|first|morning|| +39906|AAAAAAAADOLJAAAA|39906|11|5|6|AM|first|morning|| +39907|AAAAAAAAEOLJAAAA|39907|11|5|7|AM|first|morning|| +39908|AAAAAAAAFOLJAAAA|39908|11|5|8|AM|first|morning|| +39909|AAAAAAAAGOLJAAAA|39909|11|5|9|AM|first|morning|| +39910|AAAAAAAAHOLJAAAA|39910|11|5|10|AM|first|morning|| +39911|AAAAAAAAIOLJAAAA|39911|11|5|11|AM|first|morning|| +39912|AAAAAAAAJOLJAAAA|39912|11|5|12|AM|first|morning|| +39913|AAAAAAAAKOLJAAAA|39913|11|5|13|AM|first|morning|| +39914|AAAAAAAALOLJAAAA|39914|11|5|14|AM|first|morning|| +39915|AAAAAAAAMOLJAAAA|39915|11|5|15|AM|first|morning|| +39916|AAAAAAAANOLJAAAA|39916|11|5|16|AM|first|morning|| +39917|AAAAAAAAOOLJAAAA|39917|11|5|17|AM|first|morning|| +39918|AAAAAAAAPOLJAAAA|39918|11|5|18|AM|first|morning|| +39919|AAAAAAAAAPLJAAAA|39919|11|5|19|AM|first|morning|| +39920|AAAAAAAABPLJAAAA|39920|11|5|20|AM|first|morning|| +39921|AAAAAAAACPLJAAAA|39921|11|5|21|AM|first|morning|| +39922|AAAAAAAADPLJAAAA|39922|11|5|22|AM|first|morning|| +39923|AAAAAAAAEPLJAAAA|39923|11|5|23|AM|first|morning|| +39924|AAAAAAAAFPLJAAAA|39924|11|5|24|AM|first|morning|| +39925|AAAAAAAAGPLJAAAA|39925|11|5|25|AM|first|morning|| +39926|AAAAAAAAHPLJAAAA|39926|11|5|26|AM|first|morning|| +39927|AAAAAAAAIPLJAAAA|39927|11|5|27|AM|first|morning|| +39928|AAAAAAAAJPLJAAAA|39928|11|5|28|AM|first|morning|| +39929|AAAAAAAAKPLJAAAA|39929|11|5|29|AM|first|morning|| +39930|AAAAAAAALPLJAAAA|39930|11|5|30|AM|first|morning|| +39931|AAAAAAAAMPLJAAAA|39931|11|5|31|AM|first|morning|| +39932|AAAAAAAANPLJAAAA|39932|11|5|32|AM|first|morning|| +39933|AAAAAAAAOPLJAAAA|39933|11|5|33|AM|first|morning|| +39934|AAAAAAAAPPLJAAAA|39934|11|5|34|AM|first|morning|| +39935|AAAAAAAAAAMJAAAA|39935|11|5|35|AM|first|morning|| +39936|AAAAAAAABAMJAAAA|39936|11|5|36|AM|first|morning|| +39937|AAAAAAAACAMJAAAA|39937|11|5|37|AM|first|morning|| +39938|AAAAAAAADAMJAAAA|39938|11|5|38|AM|first|morning|| +39939|AAAAAAAAEAMJAAAA|39939|11|5|39|AM|first|morning|| +39940|AAAAAAAAFAMJAAAA|39940|11|5|40|AM|first|morning|| +39941|AAAAAAAAGAMJAAAA|39941|11|5|41|AM|first|morning|| +39942|AAAAAAAAHAMJAAAA|39942|11|5|42|AM|first|morning|| +39943|AAAAAAAAIAMJAAAA|39943|11|5|43|AM|first|morning|| +39944|AAAAAAAAJAMJAAAA|39944|11|5|44|AM|first|morning|| +39945|AAAAAAAAKAMJAAAA|39945|11|5|45|AM|first|morning|| +39946|AAAAAAAALAMJAAAA|39946|11|5|46|AM|first|morning|| +39947|AAAAAAAAMAMJAAAA|39947|11|5|47|AM|first|morning|| +39948|AAAAAAAANAMJAAAA|39948|11|5|48|AM|first|morning|| +39949|AAAAAAAAOAMJAAAA|39949|11|5|49|AM|first|morning|| +39950|AAAAAAAAPAMJAAAA|39950|11|5|50|AM|first|morning|| +39951|AAAAAAAAABMJAAAA|39951|11|5|51|AM|first|morning|| +39952|AAAAAAAABBMJAAAA|39952|11|5|52|AM|first|morning|| +39953|AAAAAAAACBMJAAAA|39953|11|5|53|AM|first|morning|| +39954|AAAAAAAADBMJAAAA|39954|11|5|54|AM|first|morning|| +39955|AAAAAAAAEBMJAAAA|39955|11|5|55|AM|first|morning|| +39956|AAAAAAAAFBMJAAAA|39956|11|5|56|AM|first|morning|| +39957|AAAAAAAAGBMJAAAA|39957|11|5|57|AM|first|morning|| +39958|AAAAAAAAHBMJAAAA|39958|11|5|58|AM|first|morning|| +39959|AAAAAAAAIBMJAAAA|39959|11|5|59|AM|first|morning|| +39960|AAAAAAAAJBMJAAAA|39960|11|6|0|AM|first|morning|| +39961|AAAAAAAAKBMJAAAA|39961|11|6|1|AM|first|morning|| +39962|AAAAAAAALBMJAAAA|39962|11|6|2|AM|first|morning|| +39963|AAAAAAAAMBMJAAAA|39963|11|6|3|AM|first|morning|| +39964|AAAAAAAANBMJAAAA|39964|11|6|4|AM|first|morning|| +39965|AAAAAAAAOBMJAAAA|39965|11|6|5|AM|first|morning|| +39966|AAAAAAAAPBMJAAAA|39966|11|6|6|AM|first|morning|| +39967|AAAAAAAAACMJAAAA|39967|11|6|7|AM|first|morning|| +39968|AAAAAAAABCMJAAAA|39968|11|6|8|AM|first|morning|| +39969|AAAAAAAACCMJAAAA|39969|11|6|9|AM|first|morning|| +39970|AAAAAAAADCMJAAAA|39970|11|6|10|AM|first|morning|| +39971|AAAAAAAAECMJAAAA|39971|11|6|11|AM|first|morning|| +39972|AAAAAAAAFCMJAAAA|39972|11|6|12|AM|first|morning|| +39973|AAAAAAAAGCMJAAAA|39973|11|6|13|AM|first|morning|| +39974|AAAAAAAAHCMJAAAA|39974|11|6|14|AM|first|morning|| +39975|AAAAAAAAICMJAAAA|39975|11|6|15|AM|first|morning|| +39976|AAAAAAAAJCMJAAAA|39976|11|6|16|AM|first|morning|| +39977|AAAAAAAAKCMJAAAA|39977|11|6|17|AM|first|morning|| +39978|AAAAAAAALCMJAAAA|39978|11|6|18|AM|first|morning|| +39979|AAAAAAAAMCMJAAAA|39979|11|6|19|AM|first|morning|| +39980|AAAAAAAANCMJAAAA|39980|11|6|20|AM|first|morning|| +39981|AAAAAAAAOCMJAAAA|39981|11|6|21|AM|first|morning|| +39982|AAAAAAAAPCMJAAAA|39982|11|6|22|AM|first|morning|| +39983|AAAAAAAAADMJAAAA|39983|11|6|23|AM|first|morning|| +39984|AAAAAAAABDMJAAAA|39984|11|6|24|AM|first|morning|| +39985|AAAAAAAACDMJAAAA|39985|11|6|25|AM|first|morning|| +39986|AAAAAAAADDMJAAAA|39986|11|6|26|AM|first|morning|| +39987|AAAAAAAAEDMJAAAA|39987|11|6|27|AM|first|morning|| +39988|AAAAAAAAFDMJAAAA|39988|11|6|28|AM|first|morning|| +39989|AAAAAAAAGDMJAAAA|39989|11|6|29|AM|first|morning|| +39990|AAAAAAAAHDMJAAAA|39990|11|6|30|AM|first|morning|| +39991|AAAAAAAAIDMJAAAA|39991|11|6|31|AM|first|morning|| +39992|AAAAAAAAJDMJAAAA|39992|11|6|32|AM|first|morning|| +39993|AAAAAAAAKDMJAAAA|39993|11|6|33|AM|first|morning|| +39994|AAAAAAAALDMJAAAA|39994|11|6|34|AM|first|morning|| +39995|AAAAAAAAMDMJAAAA|39995|11|6|35|AM|first|morning|| +39996|AAAAAAAANDMJAAAA|39996|11|6|36|AM|first|morning|| +39997|AAAAAAAAODMJAAAA|39997|11|6|37|AM|first|morning|| +39998|AAAAAAAAPDMJAAAA|39998|11|6|38|AM|first|morning|| +39999|AAAAAAAAAEMJAAAA|39999|11|6|39|AM|first|morning|| +40000|AAAAAAAABEMJAAAA|40000|11|6|40|AM|first|morning|| +40001|AAAAAAAACEMJAAAA|40001|11|6|41|AM|first|morning|| +40002|AAAAAAAADEMJAAAA|40002|11|6|42|AM|first|morning|| +40003|AAAAAAAAEEMJAAAA|40003|11|6|43|AM|first|morning|| +40004|AAAAAAAAFEMJAAAA|40004|11|6|44|AM|first|morning|| +40005|AAAAAAAAGEMJAAAA|40005|11|6|45|AM|first|morning|| +40006|AAAAAAAAHEMJAAAA|40006|11|6|46|AM|first|morning|| +40007|AAAAAAAAIEMJAAAA|40007|11|6|47|AM|first|morning|| +40008|AAAAAAAAJEMJAAAA|40008|11|6|48|AM|first|morning|| +40009|AAAAAAAAKEMJAAAA|40009|11|6|49|AM|first|morning|| +40010|AAAAAAAALEMJAAAA|40010|11|6|50|AM|first|morning|| +40011|AAAAAAAAMEMJAAAA|40011|11|6|51|AM|first|morning|| +40012|AAAAAAAANEMJAAAA|40012|11|6|52|AM|first|morning|| +40013|AAAAAAAAOEMJAAAA|40013|11|6|53|AM|first|morning|| +40014|AAAAAAAAPEMJAAAA|40014|11|6|54|AM|first|morning|| +40015|AAAAAAAAAFMJAAAA|40015|11|6|55|AM|first|morning|| +40016|AAAAAAAABFMJAAAA|40016|11|6|56|AM|first|morning|| +40017|AAAAAAAACFMJAAAA|40017|11|6|57|AM|first|morning|| +40018|AAAAAAAADFMJAAAA|40018|11|6|58|AM|first|morning|| +40019|AAAAAAAAEFMJAAAA|40019|11|6|59|AM|first|morning|| +40020|AAAAAAAAFFMJAAAA|40020|11|7|0|AM|first|morning|| +40021|AAAAAAAAGFMJAAAA|40021|11|7|1|AM|first|morning|| +40022|AAAAAAAAHFMJAAAA|40022|11|7|2|AM|first|morning|| +40023|AAAAAAAAIFMJAAAA|40023|11|7|3|AM|first|morning|| +40024|AAAAAAAAJFMJAAAA|40024|11|7|4|AM|first|morning|| +40025|AAAAAAAAKFMJAAAA|40025|11|7|5|AM|first|morning|| +40026|AAAAAAAALFMJAAAA|40026|11|7|6|AM|first|morning|| +40027|AAAAAAAAMFMJAAAA|40027|11|7|7|AM|first|morning|| +40028|AAAAAAAANFMJAAAA|40028|11|7|8|AM|first|morning|| +40029|AAAAAAAAOFMJAAAA|40029|11|7|9|AM|first|morning|| +40030|AAAAAAAAPFMJAAAA|40030|11|7|10|AM|first|morning|| +40031|AAAAAAAAAGMJAAAA|40031|11|7|11|AM|first|morning|| +40032|AAAAAAAABGMJAAAA|40032|11|7|12|AM|first|morning|| +40033|AAAAAAAACGMJAAAA|40033|11|7|13|AM|first|morning|| +40034|AAAAAAAADGMJAAAA|40034|11|7|14|AM|first|morning|| +40035|AAAAAAAAEGMJAAAA|40035|11|7|15|AM|first|morning|| +40036|AAAAAAAAFGMJAAAA|40036|11|7|16|AM|first|morning|| +40037|AAAAAAAAGGMJAAAA|40037|11|7|17|AM|first|morning|| +40038|AAAAAAAAHGMJAAAA|40038|11|7|18|AM|first|morning|| +40039|AAAAAAAAIGMJAAAA|40039|11|7|19|AM|first|morning|| +40040|AAAAAAAAJGMJAAAA|40040|11|7|20|AM|first|morning|| +40041|AAAAAAAAKGMJAAAA|40041|11|7|21|AM|first|morning|| +40042|AAAAAAAALGMJAAAA|40042|11|7|22|AM|first|morning|| +40043|AAAAAAAAMGMJAAAA|40043|11|7|23|AM|first|morning|| +40044|AAAAAAAANGMJAAAA|40044|11|7|24|AM|first|morning|| +40045|AAAAAAAAOGMJAAAA|40045|11|7|25|AM|first|morning|| +40046|AAAAAAAAPGMJAAAA|40046|11|7|26|AM|first|morning|| +40047|AAAAAAAAAHMJAAAA|40047|11|7|27|AM|first|morning|| +40048|AAAAAAAABHMJAAAA|40048|11|7|28|AM|first|morning|| +40049|AAAAAAAACHMJAAAA|40049|11|7|29|AM|first|morning|| +40050|AAAAAAAADHMJAAAA|40050|11|7|30|AM|first|morning|| +40051|AAAAAAAAEHMJAAAA|40051|11|7|31|AM|first|morning|| +40052|AAAAAAAAFHMJAAAA|40052|11|7|32|AM|first|morning|| +40053|AAAAAAAAGHMJAAAA|40053|11|7|33|AM|first|morning|| +40054|AAAAAAAAHHMJAAAA|40054|11|7|34|AM|first|morning|| +40055|AAAAAAAAIHMJAAAA|40055|11|7|35|AM|first|morning|| +40056|AAAAAAAAJHMJAAAA|40056|11|7|36|AM|first|morning|| +40057|AAAAAAAAKHMJAAAA|40057|11|7|37|AM|first|morning|| +40058|AAAAAAAALHMJAAAA|40058|11|7|38|AM|first|morning|| +40059|AAAAAAAAMHMJAAAA|40059|11|7|39|AM|first|morning|| +40060|AAAAAAAANHMJAAAA|40060|11|7|40|AM|first|morning|| +40061|AAAAAAAAOHMJAAAA|40061|11|7|41|AM|first|morning|| +40062|AAAAAAAAPHMJAAAA|40062|11|7|42|AM|first|morning|| +40063|AAAAAAAAAIMJAAAA|40063|11|7|43|AM|first|morning|| +40064|AAAAAAAABIMJAAAA|40064|11|7|44|AM|first|morning|| +40065|AAAAAAAACIMJAAAA|40065|11|7|45|AM|first|morning|| +40066|AAAAAAAADIMJAAAA|40066|11|7|46|AM|first|morning|| +40067|AAAAAAAAEIMJAAAA|40067|11|7|47|AM|first|morning|| +40068|AAAAAAAAFIMJAAAA|40068|11|7|48|AM|first|morning|| +40069|AAAAAAAAGIMJAAAA|40069|11|7|49|AM|first|morning|| +40070|AAAAAAAAHIMJAAAA|40070|11|7|50|AM|first|morning|| +40071|AAAAAAAAIIMJAAAA|40071|11|7|51|AM|first|morning|| +40072|AAAAAAAAJIMJAAAA|40072|11|7|52|AM|first|morning|| +40073|AAAAAAAAKIMJAAAA|40073|11|7|53|AM|first|morning|| +40074|AAAAAAAALIMJAAAA|40074|11|7|54|AM|first|morning|| +40075|AAAAAAAAMIMJAAAA|40075|11|7|55|AM|first|morning|| +40076|AAAAAAAANIMJAAAA|40076|11|7|56|AM|first|morning|| +40077|AAAAAAAAOIMJAAAA|40077|11|7|57|AM|first|morning|| +40078|AAAAAAAAPIMJAAAA|40078|11|7|58|AM|first|morning|| +40079|AAAAAAAAAJMJAAAA|40079|11|7|59|AM|first|morning|| +40080|AAAAAAAABJMJAAAA|40080|11|8|0|AM|first|morning|| +40081|AAAAAAAACJMJAAAA|40081|11|8|1|AM|first|morning|| +40082|AAAAAAAADJMJAAAA|40082|11|8|2|AM|first|morning|| +40083|AAAAAAAAEJMJAAAA|40083|11|8|3|AM|first|morning|| +40084|AAAAAAAAFJMJAAAA|40084|11|8|4|AM|first|morning|| +40085|AAAAAAAAGJMJAAAA|40085|11|8|5|AM|first|morning|| +40086|AAAAAAAAHJMJAAAA|40086|11|8|6|AM|first|morning|| +40087|AAAAAAAAIJMJAAAA|40087|11|8|7|AM|first|morning|| +40088|AAAAAAAAJJMJAAAA|40088|11|8|8|AM|first|morning|| +40089|AAAAAAAAKJMJAAAA|40089|11|8|9|AM|first|morning|| +40090|AAAAAAAALJMJAAAA|40090|11|8|10|AM|first|morning|| +40091|AAAAAAAAMJMJAAAA|40091|11|8|11|AM|first|morning|| +40092|AAAAAAAANJMJAAAA|40092|11|8|12|AM|first|morning|| +40093|AAAAAAAAOJMJAAAA|40093|11|8|13|AM|first|morning|| +40094|AAAAAAAAPJMJAAAA|40094|11|8|14|AM|first|morning|| +40095|AAAAAAAAAKMJAAAA|40095|11|8|15|AM|first|morning|| +40096|AAAAAAAABKMJAAAA|40096|11|8|16|AM|first|morning|| +40097|AAAAAAAACKMJAAAA|40097|11|8|17|AM|first|morning|| +40098|AAAAAAAADKMJAAAA|40098|11|8|18|AM|first|morning|| +40099|AAAAAAAAEKMJAAAA|40099|11|8|19|AM|first|morning|| +40100|AAAAAAAAFKMJAAAA|40100|11|8|20|AM|first|morning|| +40101|AAAAAAAAGKMJAAAA|40101|11|8|21|AM|first|morning|| +40102|AAAAAAAAHKMJAAAA|40102|11|8|22|AM|first|morning|| +40103|AAAAAAAAIKMJAAAA|40103|11|8|23|AM|first|morning|| +40104|AAAAAAAAJKMJAAAA|40104|11|8|24|AM|first|morning|| +40105|AAAAAAAAKKMJAAAA|40105|11|8|25|AM|first|morning|| +40106|AAAAAAAALKMJAAAA|40106|11|8|26|AM|first|morning|| +40107|AAAAAAAAMKMJAAAA|40107|11|8|27|AM|first|morning|| +40108|AAAAAAAANKMJAAAA|40108|11|8|28|AM|first|morning|| +40109|AAAAAAAAOKMJAAAA|40109|11|8|29|AM|first|morning|| +40110|AAAAAAAAPKMJAAAA|40110|11|8|30|AM|first|morning|| +40111|AAAAAAAAALMJAAAA|40111|11|8|31|AM|first|morning|| +40112|AAAAAAAABLMJAAAA|40112|11|8|32|AM|first|morning|| +40113|AAAAAAAACLMJAAAA|40113|11|8|33|AM|first|morning|| +40114|AAAAAAAADLMJAAAA|40114|11|8|34|AM|first|morning|| +40115|AAAAAAAAELMJAAAA|40115|11|8|35|AM|first|morning|| +40116|AAAAAAAAFLMJAAAA|40116|11|8|36|AM|first|morning|| +40117|AAAAAAAAGLMJAAAA|40117|11|8|37|AM|first|morning|| +40118|AAAAAAAAHLMJAAAA|40118|11|8|38|AM|first|morning|| +40119|AAAAAAAAILMJAAAA|40119|11|8|39|AM|first|morning|| +40120|AAAAAAAAJLMJAAAA|40120|11|8|40|AM|first|morning|| +40121|AAAAAAAAKLMJAAAA|40121|11|8|41|AM|first|morning|| +40122|AAAAAAAALLMJAAAA|40122|11|8|42|AM|first|morning|| +40123|AAAAAAAAMLMJAAAA|40123|11|8|43|AM|first|morning|| +40124|AAAAAAAANLMJAAAA|40124|11|8|44|AM|first|morning|| +40125|AAAAAAAAOLMJAAAA|40125|11|8|45|AM|first|morning|| +40126|AAAAAAAAPLMJAAAA|40126|11|8|46|AM|first|morning|| +40127|AAAAAAAAAMMJAAAA|40127|11|8|47|AM|first|morning|| +40128|AAAAAAAABMMJAAAA|40128|11|8|48|AM|first|morning|| +40129|AAAAAAAACMMJAAAA|40129|11|8|49|AM|first|morning|| +40130|AAAAAAAADMMJAAAA|40130|11|8|50|AM|first|morning|| +40131|AAAAAAAAEMMJAAAA|40131|11|8|51|AM|first|morning|| +40132|AAAAAAAAFMMJAAAA|40132|11|8|52|AM|first|morning|| +40133|AAAAAAAAGMMJAAAA|40133|11|8|53|AM|first|morning|| +40134|AAAAAAAAHMMJAAAA|40134|11|8|54|AM|first|morning|| +40135|AAAAAAAAIMMJAAAA|40135|11|8|55|AM|first|morning|| +40136|AAAAAAAAJMMJAAAA|40136|11|8|56|AM|first|morning|| +40137|AAAAAAAAKMMJAAAA|40137|11|8|57|AM|first|morning|| +40138|AAAAAAAALMMJAAAA|40138|11|8|58|AM|first|morning|| +40139|AAAAAAAAMMMJAAAA|40139|11|8|59|AM|first|morning|| +40140|AAAAAAAANMMJAAAA|40140|11|9|0|AM|first|morning|| +40141|AAAAAAAAOMMJAAAA|40141|11|9|1|AM|first|morning|| +40142|AAAAAAAAPMMJAAAA|40142|11|9|2|AM|first|morning|| +40143|AAAAAAAAANMJAAAA|40143|11|9|3|AM|first|morning|| +40144|AAAAAAAABNMJAAAA|40144|11|9|4|AM|first|morning|| +40145|AAAAAAAACNMJAAAA|40145|11|9|5|AM|first|morning|| +40146|AAAAAAAADNMJAAAA|40146|11|9|6|AM|first|morning|| +40147|AAAAAAAAENMJAAAA|40147|11|9|7|AM|first|morning|| +40148|AAAAAAAAFNMJAAAA|40148|11|9|8|AM|first|morning|| +40149|AAAAAAAAGNMJAAAA|40149|11|9|9|AM|first|morning|| +40150|AAAAAAAAHNMJAAAA|40150|11|9|10|AM|first|morning|| +40151|AAAAAAAAINMJAAAA|40151|11|9|11|AM|first|morning|| +40152|AAAAAAAAJNMJAAAA|40152|11|9|12|AM|first|morning|| +40153|AAAAAAAAKNMJAAAA|40153|11|9|13|AM|first|morning|| +40154|AAAAAAAALNMJAAAA|40154|11|9|14|AM|first|morning|| +40155|AAAAAAAAMNMJAAAA|40155|11|9|15|AM|first|morning|| +40156|AAAAAAAANNMJAAAA|40156|11|9|16|AM|first|morning|| +40157|AAAAAAAAONMJAAAA|40157|11|9|17|AM|first|morning|| +40158|AAAAAAAAPNMJAAAA|40158|11|9|18|AM|first|morning|| +40159|AAAAAAAAAOMJAAAA|40159|11|9|19|AM|first|morning|| +40160|AAAAAAAABOMJAAAA|40160|11|9|20|AM|first|morning|| +40161|AAAAAAAACOMJAAAA|40161|11|9|21|AM|first|morning|| +40162|AAAAAAAADOMJAAAA|40162|11|9|22|AM|first|morning|| +40163|AAAAAAAAEOMJAAAA|40163|11|9|23|AM|first|morning|| +40164|AAAAAAAAFOMJAAAA|40164|11|9|24|AM|first|morning|| +40165|AAAAAAAAGOMJAAAA|40165|11|9|25|AM|first|morning|| +40166|AAAAAAAAHOMJAAAA|40166|11|9|26|AM|first|morning|| +40167|AAAAAAAAIOMJAAAA|40167|11|9|27|AM|first|morning|| +40168|AAAAAAAAJOMJAAAA|40168|11|9|28|AM|first|morning|| +40169|AAAAAAAAKOMJAAAA|40169|11|9|29|AM|first|morning|| +40170|AAAAAAAALOMJAAAA|40170|11|9|30|AM|first|morning|| +40171|AAAAAAAAMOMJAAAA|40171|11|9|31|AM|first|morning|| +40172|AAAAAAAANOMJAAAA|40172|11|9|32|AM|first|morning|| +40173|AAAAAAAAOOMJAAAA|40173|11|9|33|AM|first|morning|| +40174|AAAAAAAAPOMJAAAA|40174|11|9|34|AM|first|morning|| +40175|AAAAAAAAAPMJAAAA|40175|11|9|35|AM|first|morning|| +40176|AAAAAAAABPMJAAAA|40176|11|9|36|AM|first|morning|| +40177|AAAAAAAACPMJAAAA|40177|11|9|37|AM|first|morning|| +40178|AAAAAAAADPMJAAAA|40178|11|9|38|AM|first|morning|| +40179|AAAAAAAAEPMJAAAA|40179|11|9|39|AM|first|morning|| +40180|AAAAAAAAFPMJAAAA|40180|11|9|40|AM|first|morning|| +40181|AAAAAAAAGPMJAAAA|40181|11|9|41|AM|first|morning|| +40182|AAAAAAAAHPMJAAAA|40182|11|9|42|AM|first|morning|| +40183|AAAAAAAAIPMJAAAA|40183|11|9|43|AM|first|morning|| +40184|AAAAAAAAJPMJAAAA|40184|11|9|44|AM|first|morning|| +40185|AAAAAAAAKPMJAAAA|40185|11|9|45|AM|first|morning|| +40186|AAAAAAAALPMJAAAA|40186|11|9|46|AM|first|morning|| +40187|AAAAAAAAMPMJAAAA|40187|11|9|47|AM|first|morning|| +40188|AAAAAAAANPMJAAAA|40188|11|9|48|AM|first|morning|| +40189|AAAAAAAAOPMJAAAA|40189|11|9|49|AM|first|morning|| +40190|AAAAAAAAPPMJAAAA|40190|11|9|50|AM|first|morning|| +40191|AAAAAAAAAANJAAAA|40191|11|9|51|AM|first|morning|| +40192|AAAAAAAABANJAAAA|40192|11|9|52|AM|first|morning|| +40193|AAAAAAAACANJAAAA|40193|11|9|53|AM|first|morning|| +40194|AAAAAAAADANJAAAA|40194|11|9|54|AM|first|morning|| +40195|AAAAAAAAEANJAAAA|40195|11|9|55|AM|first|morning|| +40196|AAAAAAAAFANJAAAA|40196|11|9|56|AM|first|morning|| +40197|AAAAAAAAGANJAAAA|40197|11|9|57|AM|first|morning|| +40198|AAAAAAAAHANJAAAA|40198|11|9|58|AM|first|morning|| +40199|AAAAAAAAIANJAAAA|40199|11|9|59|AM|first|morning|| +40200|AAAAAAAAJANJAAAA|40200|11|10|0|AM|first|morning|| +40201|AAAAAAAAKANJAAAA|40201|11|10|1|AM|first|morning|| +40202|AAAAAAAALANJAAAA|40202|11|10|2|AM|first|morning|| +40203|AAAAAAAAMANJAAAA|40203|11|10|3|AM|first|morning|| +40204|AAAAAAAANANJAAAA|40204|11|10|4|AM|first|morning|| +40205|AAAAAAAAOANJAAAA|40205|11|10|5|AM|first|morning|| +40206|AAAAAAAAPANJAAAA|40206|11|10|6|AM|first|morning|| +40207|AAAAAAAAABNJAAAA|40207|11|10|7|AM|first|morning|| +40208|AAAAAAAABBNJAAAA|40208|11|10|8|AM|first|morning|| +40209|AAAAAAAACBNJAAAA|40209|11|10|9|AM|first|morning|| +40210|AAAAAAAADBNJAAAA|40210|11|10|10|AM|first|morning|| +40211|AAAAAAAAEBNJAAAA|40211|11|10|11|AM|first|morning|| +40212|AAAAAAAAFBNJAAAA|40212|11|10|12|AM|first|morning|| +40213|AAAAAAAAGBNJAAAA|40213|11|10|13|AM|first|morning|| +40214|AAAAAAAAHBNJAAAA|40214|11|10|14|AM|first|morning|| +40215|AAAAAAAAIBNJAAAA|40215|11|10|15|AM|first|morning|| +40216|AAAAAAAAJBNJAAAA|40216|11|10|16|AM|first|morning|| +40217|AAAAAAAAKBNJAAAA|40217|11|10|17|AM|first|morning|| +40218|AAAAAAAALBNJAAAA|40218|11|10|18|AM|first|morning|| +40219|AAAAAAAAMBNJAAAA|40219|11|10|19|AM|first|morning|| +40220|AAAAAAAANBNJAAAA|40220|11|10|20|AM|first|morning|| +40221|AAAAAAAAOBNJAAAA|40221|11|10|21|AM|first|morning|| +40222|AAAAAAAAPBNJAAAA|40222|11|10|22|AM|first|morning|| +40223|AAAAAAAAACNJAAAA|40223|11|10|23|AM|first|morning|| +40224|AAAAAAAABCNJAAAA|40224|11|10|24|AM|first|morning|| +40225|AAAAAAAACCNJAAAA|40225|11|10|25|AM|first|morning|| +40226|AAAAAAAADCNJAAAA|40226|11|10|26|AM|first|morning|| +40227|AAAAAAAAECNJAAAA|40227|11|10|27|AM|first|morning|| +40228|AAAAAAAAFCNJAAAA|40228|11|10|28|AM|first|morning|| +40229|AAAAAAAAGCNJAAAA|40229|11|10|29|AM|first|morning|| +40230|AAAAAAAAHCNJAAAA|40230|11|10|30|AM|first|morning|| +40231|AAAAAAAAICNJAAAA|40231|11|10|31|AM|first|morning|| +40232|AAAAAAAAJCNJAAAA|40232|11|10|32|AM|first|morning|| +40233|AAAAAAAAKCNJAAAA|40233|11|10|33|AM|first|morning|| +40234|AAAAAAAALCNJAAAA|40234|11|10|34|AM|first|morning|| +40235|AAAAAAAAMCNJAAAA|40235|11|10|35|AM|first|morning|| +40236|AAAAAAAANCNJAAAA|40236|11|10|36|AM|first|morning|| +40237|AAAAAAAAOCNJAAAA|40237|11|10|37|AM|first|morning|| +40238|AAAAAAAAPCNJAAAA|40238|11|10|38|AM|first|morning|| +40239|AAAAAAAAADNJAAAA|40239|11|10|39|AM|first|morning|| +40240|AAAAAAAABDNJAAAA|40240|11|10|40|AM|first|morning|| +40241|AAAAAAAACDNJAAAA|40241|11|10|41|AM|first|morning|| +40242|AAAAAAAADDNJAAAA|40242|11|10|42|AM|first|morning|| +40243|AAAAAAAAEDNJAAAA|40243|11|10|43|AM|first|morning|| +40244|AAAAAAAAFDNJAAAA|40244|11|10|44|AM|first|morning|| +40245|AAAAAAAAGDNJAAAA|40245|11|10|45|AM|first|morning|| +40246|AAAAAAAAHDNJAAAA|40246|11|10|46|AM|first|morning|| +40247|AAAAAAAAIDNJAAAA|40247|11|10|47|AM|first|morning|| +40248|AAAAAAAAJDNJAAAA|40248|11|10|48|AM|first|morning|| +40249|AAAAAAAAKDNJAAAA|40249|11|10|49|AM|first|morning|| +40250|AAAAAAAALDNJAAAA|40250|11|10|50|AM|first|morning|| +40251|AAAAAAAAMDNJAAAA|40251|11|10|51|AM|first|morning|| +40252|AAAAAAAANDNJAAAA|40252|11|10|52|AM|first|morning|| +40253|AAAAAAAAODNJAAAA|40253|11|10|53|AM|first|morning|| +40254|AAAAAAAAPDNJAAAA|40254|11|10|54|AM|first|morning|| +40255|AAAAAAAAAENJAAAA|40255|11|10|55|AM|first|morning|| +40256|AAAAAAAABENJAAAA|40256|11|10|56|AM|first|morning|| +40257|AAAAAAAACENJAAAA|40257|11|10|57|AM|first|morning|| +40258|AAAAAAAADENJAAAA|40258|11|10|58|AM|first|morning|| +40259|AAAAAAAAEENJAAAA|40259|11|10|59|AM|first|morning|| +40260|AAAAAAAAFENJAAAA|40260|11|11|0|AM|first|morning|| +40261|AAAAAAAAGENJAAAA|40261|11|11|1|AM|first|morning|| +40262|AAAAAAAAHENJAAAA|40262|11|11|2|AM|first|morning|| +40263|AAAAAAAAIENJAAAA|40263|11|11|3|AM|first|morning|| +40264|AAAAAAAAJENJAAAA|40264|11|11|4|AM|first|morning|| +40265|AAAAAAAAKENJAAAA|40265|11|11|5|AM|first|morning|| +40266|AAAAAAAALENJAAAA|40266|11|11|6|AM|first|morning|| +40267|AAAAAAAAMENJAAAA|40267|11|11|7|AM|first|morning|| +40268|AAAAAAAANENJAAAA|40268|11|11|8|AM|first|morning|| +40269|AAAAAAAAOENJAAAA|40269|11|11|9|AM|first|morning|| +40270|AAAAAAAAPENJAAAA|40270|11|11|10|AM|first|morning|| +40271|AAAAAAAAAFNJAAAA|40271|11|11|11|AM|first|morning|| +40272|AAAAAAAABFNJAAAA|40272|11|11|12|AM|first|morning|| +40273|AAAAAAAACFNJAAAA|40273|11|11|13|AM|first|morning|| +40274|AAAAAAAADFNJAAAA|40274|11|11|14|AM|first|morning|| +40275|AAAAAAAAEFNJAAAA|40275|11|11|15|AM|first|morning|| +40276|AAAAAAAAFFNJAAAA|40276|11|11|16|AM|first|morning|| +40277|AAAAAAAAGFNJAAAA|40277|11|11|17|AM|first|morning|| +40278|AAAAAAAAHFNJAAAA|40278|11|11|18|AM|first|morning|| +40279|AAAAAAAAIFNJAAAA|40279|11|11|19|AM|first|morning|| +40280|AAAAAAAAJFNJAAAA|40280|11|11|20|AM|first|morning|| +40281|AAAAAAAAKFNJAAAA|40281|11|11|21|AM|first|morning|| +40282|AAAAAAAALFNJAAAA|40282|11|11|22|AM|first|morning|| +40283|AAAAAAAAMFNJAAAA|40283|11|11|23|AM|first|morning|| +40284|AAAAAAAANFNJAAAA|40284|11|11|24|AM|first|morning|| +40285|AAAAAAAAOFNJAAAA|40285|11|11|25|AM|first|morning|| +40286|AAAAAAAAPFNJAAAA|40286|11|11|26|AM|first|morning|| +40287|AAAAAAAAAGNJAAAA|40287|11|11|27|AM|first|morning|| +40288|AAAAAAAABGNJAAAA|40288|11|11|28|AM|first|morning|| +40289|AAAAAAAACGNJAAAA|40289|11|11|29|AM|first|morning|| +40290|AAAAAAAADGNJAAAA|40290|11|11|30|AM|first|morning|| +40291|AAAAAAAAEGNJAAAA|40291|11|11|31|AM|first|morning|| +40292|AAAAAAAAFGNJAAAA|40292|11|11|32|AM|first|morning|| +40293|AAAAAAAAGGNJAAAA|40293|11|11|33|AM|first|morning|| +40294|AAAAAAAAHGNJAAAA|40294|11|11|34|AM|first|morning|| +40295|AAAAAAAAIGNJAAAA|40295|11|11|35|AM|first|morning|| +40296|AAAAAAAAJGNJAAAA|40296|11|11|36|AM|first|morning|| +40297|AAAAAAAAKGNJAAAA|40297|11|11|37|AM|first|morning|| +40298|AAAAAAAALGNJAAAA|40298|11|11|38|AM|first|morning|| +40299|AAAAAAAAMGNJAAAA|40299|11|11|39|AM|first|morning|| +40300|AAAAAAAANGNJAAAA|40300|11|11|40|AM|first|morning|| +40301|AAAAAAAAOGNJAAAA|40301|11|11|41|AM|first|morning|| +40302|AAAAAAAAPGNJAAAA|40302|11|11|42|AM|first|morning|| +40303|AAAAAAAAAHNJAAAA|40303|11|11|43|AM|first|morning|| +40304|AAAAAAAABHNJAAAA|40304|11|11|44|AM|first|morning|| +40305|AAAAAAAACHNJAAAA|40305|11|11|45|AM|first|morning|| +40306|AAAAAAAADHNJAAAA|40306|11|11|46|AM|first|morning|| +40307|AAAAAAAAEHNJAAAA|40307|11|11|47|AM|first|morning|| +40308|AAAAAAAAFHNJAAAA|40308|11|11|48|AM|first|morning|| +40309|AAAAAAAAGHNJAAAA|40309|11|11|49|AM|first|morning|| +40310|AAAAAAAAHHNJAAAA|40310|11|11|50|AM|first|morning|| +40311|AAAAAAAAIHNJAAAA|40311|11|11|51|AM|first|morning|| +40312|AAAAAAAAJHNJAAAA|40312|11|11|52|AM|first|morning|| +40313|AAAAAAAAKHNJAAAA|40313|11|11|53|AM|first|morning|| +40314|AAAAAAAALHNJAAAA|40314|11|11|54|AM|first|morning|| +40315|AAAAAAAAMHNJAAAA|40315|11|11|55|AM|first|morning|| +40316|AAAAAAAANHNJAAAA|40316|11|11|56|AM|first|morning|| +40317|AAAAAAAAOHNJAAAA|40317|11|11|57|AM|first|morning|| +40318|AAAAAAAAPHNJAAAA|40318|11|11|58|AM|first|morning|| +40319|AAAAAAAAAINJAAAA|40319|11|11|59|AM|first|morning|| +40320|AAAAAAAABINJAAAA|40320|11|12|0|AM|first|morning|| +40321|AAAAAAAACINJAAAA|40321|11|12|1|AM|first|morning|| +40322|AAAAAAAADINJAAAA|40322|11|12|2|AM|first|morning|| +40323|AAAAAAAAEINJAAAA|40323|11|12|3|AM|first|morning|| +40324|AAAAAAAAFINJAAAA|40324|11|12|4|AM|first|morning|| +40325|AAAAAAAAGINJAAAA|40325|11|12|5|AM|first|morning|| +40326|AAAAAAAAHINJAAAA|40326|11|12|6|AM|first|morning|| +40327|AAAAAAAAIINJAAAA|40327|11|12|7|AM|first|morning|| +40328|AAAAAAAAJINJAAAA|40328|11|12|8|AM|first|morning|| +40329|AAAAAAAAKINJAAAA|40329|11|12|9|AM|first|morning|| +40330|AAAAAAAALINJAAAA|40330|11|12|10|AM|first|morning|| +40331|AAAAAAAAMINJAAAA|40331|11|12|11|AM|first|morning|| +40332|AAAAAAAANINJAAAA|40332|11|12|12|AM|first|morning|| +40333|AAAAAAAAOINJAAAA|40333|11|12|13|AM|first|morning|| +40334|AAAAAAAAPINJAAAA|40334|11|12|14|AM|first|morning|| +40335|AAAAAAAAAJNJAAAA|40335|11|12|15|AM|first|morning|| +40336|AAAAAAAABJNJAAAA|40336|11|12|16|AM|first|morning|| +40337|AAAAAAAACJNJAAAA|40337|11|12|17|AM|first|morning|| +40338|AAAAAAAADJNJAAAA|40338|11|12|18|AM|first|morning|| +40339|AAAAAAAAEJNJAAAA|40339|11|12|19|AM|first|morning|| +40340|AAAAAAAAFJNJAAAA|40340|11|12|20|AM|first|morning|| +40341|AAAAAAAAGJNJAAAA|40341|11|12|21|AM|first|morning|| +40342|AAAAAAAAHJNJAAAA|40342|11|12|22|AM|first|morning|| +40343|AAAAAAAAIJNJAAAA|40343|11|12|23|AM|first|morning|| +40344|AAAAAAAAJJNJAAAA|40344|11|12|24|AM|first|morning|| +40345|AAAAAAAAKJNJAAAA|40345|11|12|25|AM|first|morning|| +40346|AAAAAAAALJNJAAAA|40346|11|12|26|AM|first|morning|| +40347|AAAAAAAAMJNJAAAA|40347|11|12|27|AM|first|morning|| +40348|AAAAAAAANJNJAAAA|40348|11|12|28|AM|first|morning|| +40349|AAAAAAAAOJNJAAAA|40349|11|12|29|AM|first|morning|| +40350|AAAAAAAAPJNJAAAA|40350|11|12|30|AM|first|morning|| +40351|AAAAAAAAAKNJAAAA|40351|11|12|31|AM|first|morning|| +40352|AAAAAAAABKNJAAAA|40352|11|12|32|AM|first|morning|| +40353|AAAAAAAACKNJAAAA|40353|11|12|33|AM|first|morning|| +40354|AAAAAAAADKNJAAAA|40354|11|12|34|AM|first|morning|| +40355|AAAAAAAAEKNJAAAA|40355|11|12|35|AM|first|morning|| +40356|AAAAAAAAFKNJAAAA|40356|11|12|36|AM|first|morning|| +40357|AAAAAAAAGKNJAAAA|40357|11|12|37|AM|first|morning|| +40358|AAAAAAAAHKNJAAAA|40358|11|12|38|AM|first|morning|| +40359|AAAAAAAAIKNJAAAA|40359|11|12|39|AM|first|morning|| +40360|AAAAAAAAJKNJAAAA|40360|11|12|40|AM|first|morning|| +40361|AAAAAAAAKKNJAAAA|40361|11|12|41|AM|first|morning|| +40362|AAAAAAAALKNJAAAA|40362|11|12|42|AM|first|morning|| +40363|AAAAAAAAMKNJAAAA|40363|11|12|43|AM|first|morning|| +40364|AAAAAAAANKNJAAAA|40364|11|12|44|AM|first|morning|| +40365|AAAAAAAAOKNJAAAA|40365|11|12|45|AM|first|morning|| +40366|AAAAAAAAPKNJAAAA|40366|11|12|46|AM|first|morning|| +40367|AAAAAAAAALNJAAAA|40367|11|12|47|AM|first|morning|| +40368|AAAAAAAABLNJAAAA|40368|11|12|48|AM|first|morning|| +40369|AAAAAAAACLNJAAAA|40369|11|12|49|AM|first|morning|| +40370|AAAAAAAADLNJAAAA|40370|11|12|50|AM|first|morning|| +40371|AAAAAAAAELNJAAAA|40371|11|12|51|AM|first|morning|| +40372|AAAAAAAAFLNJAAAA|40372|11|12|52|AM|first|morning|| +40373|AAAAAAAAGLNJAAAA|40373|11|12|53|AM|first|morning|| +40374|AAAAAAAAHLNJAAAA|40374|11|12|54|AM|first|morning|| +40375|AAAAAAAAILNJAAAA|40375|11|12|55|AM|first|morning|| +40376|AAAAAAAAJLNJAAAA|40376|11|12|56|AM|first|morning|| +40377|AAAAAAAAKLNJAAAA|40377|11|12|57|AM|first|morning|| +40378|AAAAAAAALLNJAAAA|40378|11|12|58|AM|first|morning|| +40379|AAAAAAAAMLNJAAAA|40379|11|12|59|AM|first|morning|| +40380|AAAAAAAANLNJAAAA|40380|11|13|0|AM|first|morning|| +40381|AAAAAAAAOLNJAAAA|40381|11|13|1|AM|first|morning|| +40382|AAAAAAAAPLNJAAAA|40382|11|13|2|AM|first|morning|| +40383|AAAAAAAAAMNJAAAA|40383|11|13|3|AM|first|morning|| +40384|AAAAAAAABMNJAAAA|40384|11|13|4|AM|first|morning|| +40385|AAAAAAAACMNJAAAA|40385|11|13|5|AM|first|morning|| +40386|AAAAAAAADMNJAAAA|40386|11|13|6|AM|first|morning|| +40387|AAAAAAAAEMNJAAAA|40387|11|13|7|AM|first|morning|| +40388|AAAAAAAAFMNJAAAA|40388|11|13|8|AM|first|morning|| +40389|AAAAAAAAGMNJAAAA|40389|11|13|9|AM|first|morning|| +40390|AAAAAAAAHMNJAAAA|40390|11|13|10|AM|first|morning|| +40391|AAAAAAAAIMNJAAAA|40391|11|13|11|AM|first|morning|| +40392|AAAAAAAAJMNJAAAA|40392|11|13|12|AM|first|morning|| +40393|AAAAAAAAKMNJAAAA|40393|11|13|13|AM|first|morning|| +40394|AAAAAAAALMNJAAAA|40394|11|13|14|AM|first|morning|| +40395|AAAAAAAAMMNJAAAA|40395|11|13|15|AM|first|morning|| +40396|AAAAAAAANMNJAAAA|40396|11|13|16|AM|first|morning|| +40397|AAAAAAAAOMNJAAAA|40397|11|13|17|AM|first|morning|| +40398|AAAAAAAAPMNJAAAA|40398|11|13|18|AM|first|morning|| +40399|AAAAAAAAANNJAAAA|40399|11|13|19|AM|first|morning|| +40400|AAAAAAAABNNJAAAA|40400|11|13|20|AM|first|morning|| +40401|AAAAAAAACNNJAAAA|40401|11|13|21|AM|first|morning|| +40402|AAAAAAAADNNJAAAA|40402|11|13|22|AM|first|morning|| +40403|AAAAAAAAENNJAAAA|40403|11|13|23|AM|first|morning|| +40404|AAAAAAAAFNNJAAAA|40404|11|13|24|AM|first|morning|| +40405|AAAAAAAAGNNJAAAA|40405|11|13|25|AM|first|morning|| +40406|AAAAAAAAHNNJAAAA|40406|11|13|26|AM|first|morning|| +40407|AAAAAAAAINNJAAAA|40407|11|13|27|AM|first|morning|| +40408|AAAAAAAAJNNJAAAA|40408|11|13|28|AM|first|morning|| +40409|AAAAAAAAKNNJAAAA|40409|11|13|29|AM|first|morning|| +40410|AAAAAAAALNNJAAAA|40410|11|13|30|AM|first|morning|| +40411|AAAAAAAAMNNJAAAA|40411|11|13|31|AM|first|morning|| +40412|AAAAAAAANNNJAAAA|40412|11|13|32|AM|first|morning|| +40413|AAAAAAAAONNJAAAA|40413|11|13|33|AM|first|morning|| +40414|AAAAAAAAPNNJAAAA|40414|11|13|34|AM|first|morning|| +40415|AAAAAAAAAONJAAAA|40415|11|13|35|AM|first|morning|| +40416|AAAAAAAABONJAAAA|40416|11|13|36|AM|first|morning|| +40417|AAAAAAAACONJAAAA|40417|11|13|37|AM|first|morning|| +40418|AAAAAAAADONJAAAA|40418|11|13|38|AM|first|morning|| +40419|AAAAAAAAEONJAAAA|40419|11|13|39|AM|first|morning|| +40420|AAAAAAAAFONJAAAA|40420|11|13|40|AM|first|morning|| +40421|AAAAAAAAGONJAAAA|40421|11|13|41|AM|first|morning|| +40422|AAAAAAAAHONJAAAA|40422|11|13|42|AM|first|morning|| +40423|AAAAAAAAIONJAAAA|40423|11|13|43|AM|first|morning|| +40424|AAAAAAAAJONJAAAA|40424|11|13|44|AM|first|morning|| +40425|AAAAAAAAKONJAAAA|40425|11|13|45|AM|first|morning|| +40426|AAAAAAAALONJAAAA|40426|11|13|46|AM|first|morning|| +40427|AAAAAAAAMONJAAAA|40427|11|13|47|AM|first|morning|| +40428|AAAAAAAANONJAAAA|40428|11|13|48|AM|first|morning|| +40429|AAAAAAAAOONJAAAA|40429|11|13|49|AM|first|morning|| +40430|AAAAAAAAPONJAAAA|40430|11|13|50|AM|first|morning|| +40431|AAAAAAAAAPNJAAAA|40431|11|13|51|AM|first|morning|| +40432|AAAAAAAABPNJAAAA|40432|11|13|52|AM|first|morning|| +40433|AAAAAAAACPNJAAAA|40433|11|13|53|AM|first|morning|| +40434|AAAAAAAADPNJAAAA|40434|11|13|54|AM|first|morning|| +40435|AAAAAAAAEPNJAAAA|40435|11|13|55|AM|first|morning|| +40436|AAAAAAAAFPNJAAAA|40436|11|13|56|AM|first|morning|| +40437|AAAAAAAAGPNJAAAA|40437|11|13|57|AM|first|morning|| +40438|AAAAAAAAHPNJAAAA|40438|11|13|58|AM|first|morning|| +40439|AAAAAAAAIPNJAAAA|40439|11|13|59|AM|first|morning|| +40440|AAAAAAAAJPNJAAAA|40440|11|14|0|AM|first|morning|| +40441|AAAAAAAAKPNJAAAA|40441|11|14|1|AM|first|morning|| +40442|AAAAAAAALPNJAAAA|40442|11|14|2|AM|first|morning|| +40443|AAAAAAAAMPNJAAAA|40443|11|14|3|AM|first|morning|| +40444|AAAAAAAANPNJAAAA|40444|11|14|4|AM|first|morning|| +40445|AAAAAAAAOPNJAAAA|40445|11|14|5|AM|first|morning|| +40446|AAAAAAAAPPNJAAAA|40446|11|14|6|AM|first|morning|| +40447|AAAAAAAAAAOJAAAA|40447|11|14|7|AM|first|morning|| +40448|AAAAAAAABAOJAAAA|40448|11|14|8|AM|first|morning|| +40449|AAAAAAAACAOJAAAA|40449|11|14|9|AM|first|morning|| +40450|AAAAAAAADAOJAAAA|40450|11|14|10|AM|first|morning|| +40451|AAAAAAAAEAOJAAAA|40451|11|14|11|AM|first|morning|| +40452|AAAAAAAAFAOJAAAA|40452|11|14|12|AM|first|morning|| +40453|AAAAAAAAGAOJAAAA|40453|11|14|13|AM|first|morning|| +40454|AAAAAAAAHAOJAAAA|40454|11|14|14|AM|first|morning|| +40455|AAAAAAAAIAOJAAAA|40455|11|14|15|AM|first|morning|| +40456|AAAAAAAAJAOJAAAA|40456|11|14|16|AM|first|morning|| +40457|AAAAAAAAKAOJAAAA|40457|11|14|17|AM|first|morning|| +40458|AAAAAAAALAOJAAAA|40458|11|14|18|AM|first|morning|| +40459|AAAAAAAAMAOJAAAA|40459|11|14|19|AM|first|morning|| +40460|AAAAAAAANAOJAAAA|40460|11|14|20|AM|first|morning|| +40461|AAAAAAAAOAOJAAAA|40461|11|14|21|AM|first|morning|| +40462|AAAAAAAAPAOJAAAA|40462|11|14|22|AM|first|morning|| +40463|AAAAAAAAABOJAAAA|40463|11|14|23|AM|first|morning|| +40464|AAAAAAAABBOJAAAA|40464|11|14|24|AM|first|morning|| +40465|AAAAAAAACBOJAAAA|40465|11|14|25|AM|first|morning|| +40466|AAAAAAAADBOJAAAA|40466|11|14|26|AM|first|morning|| +40467|AAAAAAAAEBOJAAAA|40467|11|14|27|AM|first|morning|| +40468|AAAAAAAAFBOJAAAA|40468|11|14|28|AM|first|morning|| +40469|AAAAAAAAGBOJAAAA|40469|11|14|29|AM|first|morning|| +40470|AAAAAAAAHBOJAAAA|40470|11|14|30|AM|first|morning|| +40471|AAAAAAAAIBOJAAAA|40471|11|14|31|AM|first|morning|| +40472|AAAAAAAAJBOJAAAA|40472|11|14|32|AM|first|morning|| +40473|AAAAAAAAKBOJAAAA|40473|11|14|33|AM|first|morning|| +40474|AAAAAAAALBOJAAAA|40474|11|14|34|AM|first|morning|| +40475|AAAAAAAAMBOJAAAA|40475|11|14|35|AM|first|morning|| +40476|AAAAAAAANBOJAAAA|40476|11|14|36|AM|first|morning|| +40477|AAAAAAAAOBOJAAAA|40477|11|14|37|AM|first|morning|| +40478|AAAAAAAAPBOJAAAA|40478|11|14|38|AM|first|morning|| +40479|AAAAAAAAACOJAAAA|40479|11|14|39|AM|first|morning|| +40480|AAAAAAAABCOJAAAA|40480|11|14|40|AM|first|morning|| +40481|AAAAAAAACCOJAAAA|40481|11|14|41|AM|first|morning|| +40482|AAAAAAAADCOJAAAA|40482|11|14|42|AM|first|morning|| +40483|AAAAAAAAECOJAAAA|40483|11|14|43|AM|first|morning|| +40484|AAAAAAAAFCOJAAAA|40484|11|14|44|AM|first|morning|| +40485|AAAAAAAAGCOJAAAA|40485|11|14|45|AM|first|morning|| +40486|AAAAAAAAHCOJAAAA|40486|11|14|46|AM|first|morning|| +40487|AAAAAAAAICOJAAAA|40487|11|14|47|AM|first|morning|| +40488|AAAAAAAAJCOJAAAA|40488|11|14|48|AM|first|morning|| +40489|AAAAAAAAKCOJAAAA|40489|11|14|49|AM|first|morning|| +40490|AAAAAAAALCOJAAAA|40490|11|14|50|AM|first|morning|| +40491|AAAAAAAAMCOJAAAA|40491|11|14|51|AM|first|morning|| +40492|AAAAAAAANCOJAAAA|40492|11|14|52|AM|first|morning|| +40493|AAAAAAAAOCOJAAAA|40493|11|14|53|AM|first|morning|| +40494|AAAAAAAAPCOJAAAA|40494|11|14|54|AM|first|morning|| +40495|AAAAAAAAADOJAAAA|40495|11|14|55|AM|first|morning|| +40496|AAAAAAAABDOJAAAA|40496|11|14|56|AM|first|morning|| +40497|AAAAAAAACDOJAAAA|40497|11|14|57|AM|first|morning|| +40498|AAAAAAAADDOJAAAA|40498|11|14|58|AM|first|morning|| +40499|AAAAAAAAEDOJAAAA|40499|11|14|59|AM|first|morning|| +40500|AAAAAAAAFDOJAAAA|40500|11|15|0|AM|first|morning|| +40501|AAAAAAAAGDOJAAAA|40501|11|15|1|AM|first|morning|| +40502|AAAAAAAAHDOJAAAA|40502|11|15|2|AM|first|morning|| +40503|AAAAAAAAIDOJAAAA|40503|11|15|3|AM|first|morning|| +40504|AAAAAAAAJDOJAAAA|40504|11|15|4|AM|first|morning|| +40505|AAAAAAAAKDOJAAAA|40505|11|15|5|AM|first|morning|| +40506|AAAAAAAALDOJAAAA|40506|11|15|6|AM|first|morning|| +40507|AAAAAAAAMDOJAAAA|40507|11|15|7|AM|first|morning|| +40508|AAAAAAAANDOJAAAA|40508|11|15|8|AM|first|morning|| +40509|AAAAAAAAODOJAAAA|40509|11|15|9|AM|first|morning|| +40510|AAAAAAAAPDOJAAAA|40510|11|15|10|AM|first|morning|| +40511|AAAAAAAAAEOJAAAA|40511|11|15|11|AM|first|morning|| +40512|AAAAAAAABEOJAAAA|40512|11|15|12|AM|first|morning|| +40513|AAAAAAAACEOJAAAA|40513|11|15|13|AM|first|morning|| +40514|AAAAAAAADEOJAAAA|40514|11|15|14|AM|first|morning|| +40515|AAAAAAAAEEOJAAAA|40515|11|15|15|AM|first|morning|| +40516|AAAAAAAAFEOJAAAA|40516|11|15|16|AM|first|morning|| +40517|AAAAAAAAGEOJAAAA|40517|11|15|17|AM|first|morning|| +40518|AAAAAAAAHEOJAAAA|40518|11|15|18|AM|first|morning|| +40519|AAAAAAAAIEOJAAAA|40519|11|15|19|AM|first|morning|| +40520|AAAAAAAAJEOJAAAA|40520|11|15|20|AM|first|morning|| +40521|AAAAAAAAKEOJAAAA|40521|11|15|21|AM|first|morning|| +40522|AAAAAAAALEOJAAAA|40522|11|15|22|AM|first|morning|| +40523|AAAAAAAAMEOJAAAA|40523|11|15|23|AM|first|morning|| +40524|AAAAAAAANEOJAAAA|40524|11|15|24|AM|first|morning|| +40525|AAAAAAAAOEOJAAAA|40525|11|15|25|AM|first|morning|| +40526|AAAAAAAAPEOJAAAA|40526|11|15|26|AM|first|morning|| +40527|AAAAAAAAAFOJAAAA|40527|11|15|27|AM|first|morning|| +40528|AAAAAAAABFOJAAAA|40528|11|15|28|AM|first|morning|| +40529|AAAAAAAACFOJAAAA|40529|11|15|29|AM|first|morning|| +40530|AAAAAAAADFOJAAAA|40530|11|15|30|AM|first|morning|| +40531|AAAAAAAAEFOJAAAA|40531|11|15|31|AM|first|morning|| +40532|AAAAAAAAFFOJAAAA|40532|11|15|32|AM|first|morning|| +40533|AAAAAAAAGFOJAAAA|40533|11|15|33|AM|first|morning|| +40534|AAAAAAAAHFOJAAAA|40534|11|15|34|AM|first|morning|| +40535|AAAAAAAAIFOJAAAA|40535|11|15|35|AM|first|morning|| +40536|AAAAAAAAJFOJAAAA|40536|11|15|36|AM|first|morning|| +40537|AAAAAAAAKFOJAAAA|40537|11|15|37|AM|first|morning|| +40538|AAAAAAAALFOJAAAA|40538|11|15|38|AM|first|morning|| +40539|AAAAAAAAMFOJAAAA|40539|11|15|39|AM|first|morning|| +40540|AAAAAAAANFOJAAAA|40540|11|15|40|AM|first|morning|| +40541|AAAAAAAAOFOJAAAA|40541|11|15|41|AM|first|morning|| +40542|AAAAAAAAPFOJAAAA|40542|11|15|42|AM|first|morning|| +40543|AAAAAAAAAGOJAAAA|40543|11|15|43|AM|first|morning|| +40544|AAAAAAAABGOJAAAA|40544|11|15|44|AM|first|morning|| +40545|AAAAAAAACGOJAAAA|40545|11|15|45|AM|first|morning|| +40546|AAAAAAAADGOJAAAA|40546|11|15|46|AM|first|morning|| +40547|AAAAAAAAEGOJAAAA|40547|11|15|47|AM|first|morning|| +40548|AAAAAAAAFGOJAAAA|40548|11|15|48|AM|first|morning|| +40549|AAAAAAAAGGOJAAAA|40549|11|15|49|AM|first|morning|| +40550|AAAAAAAAHGOJAAAA|40550|11|15|50|AM|first|morning|| +40551|AAAAAAAAIGOJAAAA|40551|11|15|51|AM|first|morning|| +40552|AAAAAAAAJGOJAAAA|40552|11|15|52|AM|first|morning|| +40553|AAAAAAAAKGOJAAAA|40553|11|15|53|AM|first|morning|| +40554|AAAAAAAALGOJAAAA|40554|11|15|54|AM|first|morning|| +40555|AAAAAAAAMGOJAAAA|40555|11|15|55|AM|first|morning|| +40556|AAAAAAAANGOJAAAA|40556|11|15|56|AM|first|morning|| +40557|AAAAAAAAOGOJAAAA|40557|11|15|57|AM|first|morning|| +40558|AAAAAAAAPGOJAAAA|40558|11|15|58|AM|first|morning|| +40559|AAAAAAAAAHOJAAAA|40559|11|15|59|AM|first|morning|| +40560|AAAAAAAABHOJAAAA|40560|11|16|0|AM|first|morning|| +40561|AAAAAAAACHOJAAAA|40561|11|16|1|AM|first|morning|| +40562|AAAAAAAADHOJAAAA|40562|11|16|2|AM|first|morning|| +40563|AAAAAAAAEHOJAAAA|40563|11|16|3|AM|first|morning|| +40564|AAAAAAAAFHOJAAAA|40564|11|16|4|AM|first|morning|| +40565|AAAAAAAAGHOJAAAA|40565|11|16|5|AM|first|morning|| +40566|AAAAAAAAHHOJAAAA|40566|11|16|6|AM|first|morning|| +40567|AAAAAAAAIHOJAAAA|40567|11|16|7|AM|first|morning|| +40568|AAAAAAAAJHOJAAAA|40568|11|16|8|AM|first|morning|| +40569|AAAAAAAAKHOJAAAA|40569|11|16|9|AM|first|morning|| +40570|AAAAAAAALHOJAAAA|40570|11|16|10|AM|first|morning|| +40571|AAAAAAAAMHOJAAAA|40571|11|16|11|AM|first|morning|| +40572|AAAAAAAANHOJAAAA|40572|11|16|12|AM|first|morning|| +40573|AAAAAAAAOHOJAAAA|40573|11|16|13|AM|first|morning|| +40574|AAAAAAAAPHOJAAAA|40574|11|16|14|AM|first|morning|| +40575|AAAAAAAAAIOJAAAA|40575|11|16|15|AM|first|morning|| +40576|AAAAAAAABIOJAAAA|40576|11|16|16|AM|first|morning|| +40577|AAAAAAAACIOJAAAA|40577|11|16|17|AM|first|morning|| +40578|AAAAAAAADIOJAAAA|40578|11|16|18|AM|first|morning|| +40579|AAAAAAAAEIOJAAAA|40579|11|16|19|AM|first|morning|| +40580|AAAAAAAAFIOJAAAA|40580|11|16|20|AM|first|morning|| +40581|AAAAAAAAGIOJAAAA|40581|11|16|21|AM|first|morning|| +40582|AAAAAAAAHIOJAAAA|40582|11|16|22|AM|first|morning|| +40583|AAAAAAAAIIOJAAAA|40583|11|16|23|AM|first|morning|| +40584|AAAAAAAAJIOJAAAA|40584|11|16|24|AM|first|morning|| +40585|AAAAAAAAKIOJAAAA|40585|11|16|25|AM|first|morning|| +40586|AAAAAAAALIOJAAAA|40586|11|16|26|AM|first|morning|| +40587|AAAAAAAAMIOJAAAA|40587|11|16|27|AM|first|morning|| +40588|AAAAAAAANIOJAAAA|40588|11|16|28|AM|first|morning|| +40589|AAAAAAAAOIOJAAAA|40589|11|16|29|AM|first|morning|| +40590|AAAAAAAAPIOJAAAA|40590|11|16|30|AM|first|morning|| +40591|AAAAAAAAAJOJAAAA|40591|11|16|31|AM|first|morning|| +40592|AAAAAAAABJOJAAAA|40592|11|16|32|AM|first|morning|| +40593|AAAAAAAACJOJAAAA|40593|11|16|33|AM|first|morning|| +40594|AAAAAAAADJOJAAAA|40594|11|16|34|AM|first|morning|| +40595|AAAAAAAAEJOJAAAA|40595|11|16|35|AM|first|morning|| +40596|AAAAAAAAFJOJAAAA|40596|11|16|36|AM|first|morning|| +40597|AAAAAAAAGJOJAAAA|40597|11|16|37|AM|first|morning|| +40598|AAAAAAAAHJOJAAAA|40598|11|16|38|AM|first|morning|| +40599|AAAAAAAAIJOJAAAA|40599|11|16|39|AM|first|morning|| +40600|AAAAAAAAJJOJAAAA|40600|11|16|40|AM|first|morning|| +40601|AAAAAAAAKJOJAAAA|40601|11|16|41|AM|first|morning|| +40602|AAAAAAAALJOJAAAA|40602|11|16|42|AM|first|morning|| +40603|AAAAAAAAMJOJAAAA|40603|11|16|43|AM|first|morning|| +40604|AAAAAAAANJOJAAAA|40604|11|16|44|AM|first|morning|| +40605|AAAAAAAAOJOJAAAA|40605|11|16|45|AM|first|morning|| +40606|AAAAAAAAPJOJAAAA|40606|11|16|46|AM|first|morning|| +40607|AAAAAAAAAKOJAAAA|40607|11|16|47|AM|first|morning|| +40608|AAAAAAAABKOJAAAA|40608|11|16|48|AM|first|morning|| +40609|AAAAAAAACKOJAAAA|40609|11|16|49|AM|first|morning|| +40610|AAAAAAAADKOJAAAA|40610|11|16|50|AM|first|morning|| +40611|AAAAAAAAEKOJAAAA|40611|11|16|51|AM|first|morning|| +40612|AAAAAAAAFKOJAAAA|40612|11|16|52|AM|first|morning|| +40613|AAAAAAAAGKOJAAAA|40613|11|16|53|AM|first|morning|| +40614|AAAAAAAAHKOJAAAA|40614|11|16|54|AM|first|morning|| +40615|AAAAAAAAIKOJAAAA|40615|11|16|55|AM|first|morning|| +40616|AAAAAAAAJKOJAAAA|40616|11|16|56|AM|first|morning|| +40617|AAAAAAAAKKOJAAAA|40617|11|16|57|AM|first|morning|| +40618|AAAAAAAALKOJAAAA|40618|11|16|58|AM|first|morning|| +40619|AAAAAAAAMKOJAAAA|40619|11|16|59|AM|first|morning|| +40620|AAAAAAAANKOJAAAA|40620|11|17|0|AM|first|morning|| +40621|AAAAAAAAOKOJAAAA|40621|11|17|1|AM|first|morning|| +40622|AAAAAAAAPKOJAAAA|40622|11|17|2|AM|first|morning|| +40623|AAAAAAAAALOJAAAA|40623|11|17|3|AM|first|morning|| +40624|AAAAAAAABLOJAAAA|40624|11|17|4|AM|first|morning|| +40625|AAAAAAAACLOJAAAA|40625|11|17|5|AM|first|morning|| +40626|AAAAAAAADLOJAAAA|40626|11|17|6|AM|first|morning|| +40627|AAAAAAAAELOJAAAA|40627|11|17|7|AM|first|morning|| +40628|AAAAAAAAFLOJAAAA|40628|11|17|8|AM|first|morning|| +40629|AAAAAAAAGLOJAAAA|40629|11|17|9|AM|first|morning|| +40630|AAAAAAAAHLOJAAAA|40630|11|17|10|AM|first|morning|| +40631|AAAAAAAAILOJAAAA|40631|11|17|11|AM|first|morning|| +40632|AAAAAAAAJLOJAAAA|40632|11|17|12|AM|first|morning|| +40633|AAAAAAAAKLOJAAAA|40633|11|17|13|AM|first|morning|| +40634|AAAAAAAALLOJAAAA|40634|11|17|14|AM|first|morning|| +40635|AAAAAAAAMLOJAAAA|40635|11|17|15|AM|first|morning|| +40636|AAAAAAAANLOJAAAA|40636|11|17|16|AM|first|morning|| +40637|AAAAAAAAOLOJAAAA|40637|11|17|17|AM|first|morning|| +40638|AAAAAAAAPLOJAAAA|40638|11|17|18|AM|first|morning|| +40639|AAAAAAAAAMOJAAAA|40639|11|17|19|AM|first|morning|| +40640|AAAAAAAABMOJAAAA|40640|11|17|20|AM|first|morning|| +40641|AAAAAAAACMOJAAAA|40641|11|17|21|AM|first|morning|| +40642|AAAAAAAADMOJAAAA|40642|11|17|22|AM|first|morning|| +40643|AAAAAAAAEMOJAAAA|40643|11|17|23|AM|first|morning|| +40644|AAAAAAAAFMOJAAAA|40644|11|17|24|AM|first|morning|| +40645|AAAAAAAAGMOJAAAA|40645|11|17|25|AM|first|morning|| +40646|AAAAAAAAHMOJAAAA|40646|11|17|26|AM|first|morning|| +40647|AAAAAAAAIMOJAAAA|40647|11|17|27|AM|first|morning|| +40648|AAAAAAAAJMOJAAAA|40648|11|17|28|AM|first|morning|| +40649|AAAAAAAAKMOJAAAA|40649|11|17|29|AM|first|morning|| +40650|AAAAAAAALMOJAAAA|40650|11|17|30|AM|first|morning|| +40651|AAAAAAAAMMOJAAAA|40651|11|17|31|AM|first|morning|| +40652|AAAAAAAANMOJAAAA|40652|11|17|32|AM|first|morning|| +40653|AAAAAAAAOMOJAAAA|40653|11|17|33|AM|first|morning|| +40654|AAAAAAAAPMOJAAAA|40654|11|17|34|AM|first|morning|| +40655|AAAAAAAAANOJAAAA|40655|11|17|35|AM|first|morning|| +40656|AAAAAAAABNOJAAAA|40656|11|17|36|AM|first|morning|| +40657|AAAAAAAACNOJAAAA|40657|11|17|37|AM|first|morning|| +40658|AAAAAAAADNOJAAAA|40658|11|17|38|AM|first|morning|| +40659|AAAAAAAAENOJAAAA|40659|11|17|39|AM|first|morning|| +40660|AAAAAAAAFNOJAAAA|40660|11|17|40|AM|first|morning|| +40661|AAAAAAAAGNOJAAAA|40661|11|17|41|AM|first|morning|| +40662|AAAAAAAAHNOJAAAA|40662|11|17|42|AM|first|morning|| +40663|AAAAAAAAINOJAAAA|40663|11|17|43|AM|first|morning|| +40664|AAAAAAAAJNOJAAAA|40664|11|17|44|AM|first|morning|| +40665|AAAAAAAAKNOJAAAA|40665|11|17|45|AM|first|morning|| +40666|AAAAAAAALNOJAAAA|40666|11|17|46|AM|first|morning|| +40667|AAAAAAAAMNOJAAAA|40667|11|17|47|AM|first|morning|| +40668|AAAAAAAANNOJAAAA|40668|11|17|48|AM|first|morning|| +40669|AAAAAAAAONOJAAAA|40669|11|17|49|AM|first|morning|| +40670|AAAAAAAAPNOJAAAA|40670|11|17|50|AM|first|morning|| +40671|AAAAAAAAAOOJAAAA|40671|11|17|51|AM|first|morning|| +40672|AAAAAAAABOOJAAAA|40672|11|17|52|AM|first|morning|| +40673|AAAAAAAACOOJAAAA|40673|11|17|53|AM|first|morning|| +40674|AAAAAAAADOOJAAAA|40674|11|17|54|AM|first|morning|| +40675|AAAAAAAAEOOJAAAA|40675|11|17|55|AM|first|morning|| +40676|AAAAAAAAFOOJAAAA|40676|11|17|56|AM|first|morning|| +40677|AAAAAAAAGOOJAAAA|40677|11|17|57|AM|first|morning|| +40678|AAAAAAAAHOOJAAAA|40678|11|17|58|AM|first|morning|| +40679|AAAAAAAAIOOJAAAA|40679|11|17|59|AM|first|morning|| +40680|AAAAAAAAJOOJAAAA|40680|11|18|0|AM|first|morning|| +40681|AAAAAAAAKOOJAAAA|40681|11|18|1|AM|first|morning|| +40682|AAAAAAAALOOJAAAA|40682|11|18|2|AM|first|morning|| +40683|AAAAAAAAMOOJAAAA|40683|11|18|3|AM|first|morning|| +40684|AAAAAAAANOOJAAAA|40684|11|18|4|AM|first|morning|| +40685|AAAAAAAAOOOJAAAA|40685|11|18|5|AM|first|morning|| +40686|AAAAAAAAPOOJAAAA|40686|11|18|6|AM|first|morning|| +40687|AAAAAAAAAPOJAAAA|40687|11|18|7|AM|first|morning|| +40688|AAAAAAAABPOJAAAA|40688|11|18|8|AM|first|morning|| +40689|AAAAAAAACPOJAAAA|40689|11|18|9|AM|first|morning|| +40690|AAAAAAAADPOJAAAA|40690|11|18|10|AM|first|morning|| +40691|AAAAAAAAEPOJAAAA|40691|11|18|11|AM|first|morning|| +40692|AAAAAAAAFPOJAAAA|40692|11|18|12|AM|first|morning|| +40693|AAAAAAAAGPOJAAAA|40693|11|18|13|AM|first|morning|| +40694|AAAAAAAAHPOJAAAA|40694|11|18|14|AM|first|morning|| +40695|AAAAAAAAIPOJAAAA|40695|11|18|15|AM|first|morning|| +40696|AAAAAAAAJPOJAAAA|40696|11|18|16|AM|first|morning|| +40697|AAAAAAAAKPOJAAAA|40697|11|18|17|AM|first|morning|| +40698|AAAAAAAALPOJAAAA|40698|11|18|18|AM|first|morning|| +40699|AAAAAAAAMPOJAAAA|40699|11|18|19|AM|first|morning|| +40700|AAAAAAAANPOJAAAA|40700|11|18|20|AM|first|morning|| +40701|AAAAAAAAOPOJAAAA|40701|11|18|21|AM|first|morning|| +40702|AAAAAAAAPPOJAAAA|40702|11|18|22|AM|first|morning|| +40703|AAAAAAAAAAPJAAAA|40703|11|18|23|AM|first|morning|| +40704|AAAAAAAABAPJAAAA|40704|11|18|24|AM|first|morning|| +40705|AAAAAAAACAPJAAAA|40705|11|18|25|AM|first|morning|| +40706|AAAAAAAADAPJAAAA|40706|11|18|26|AM|first|morning|| +40707|AAAAAAAAEAPJAAAA|40707|11|18|27|AM|first|morning|| +40708|AAAAAAAAFAPJAAAA|40708|11|18|28|AM|first|morning|| +40709|AAAAAAAAGAPJAAAA|40709|11|18|29|AM|first|morning|| +40710|AAAAAAAAHAPJAAAA|40710|11|18|30|AM|first|morning|| +40711|AAAAAAAAIAPJAAAA|40711|11|18|31|AM|first|morning|| +40712|AAAAAAAAJAPJAAAA|40712|11|18|32|AM|first|morning|| +40713|AAAAAAAAKAPJAAAA|40713|11|18|33|AM|first|morning|| +40714|AAAAAAAALAPJAAAA|40714|11|18|34|AM|first|morning|| +40715|AAAAAAAAMAPJAAAA|40715|11|18|35|AM|first|morning|| +40716|AAAAAAAANAPJAAAA|40716|11|18|36|AM|first|morning|| +40717|AAAAAAAAOAPJAAAA|40717|11|18|37|AM|first|morning|| +40718|AAAAAAAAPAPJAAAA|40718|11|18|38|AM|first|morning|| +40719|AAAAAAAAABPJAAAA|40719|11|18|39|AM|first|morning|| +40720|AAAAAAAABBPJAAAA|40720|11|18|40|AM|first|morning|| +40721|AAAAAAAACBPJAAAA|40721|11|18|41|AM|first|morning|| +40722|AAAAAAAADBPJAAAA|40722|11|18|42|AM|first|morning|| +40723|AAAAAAAAEBPJAAAA|40723|11|18|43|AM|first|morning|| +40724|AAAAAAAAFBPJAAAA|40724|11|18|44|AM|first|morning|| +40725|AAAAAAAAGBPJAAAA|40725|11|18|45|AM|first|morning|| +40726|AAAAAAAAHBPJAAAA|40726|11|18|46|AM|first|morning|| +40727|AAAAAAAAIBPJAAAA|40727|11|18|47|AM|first|morning|| +40728|AAAAAAAAJBPJAAAA|40728|11|18|48|AM|first|morning|| +40729|AAAAAAAAKBPJAAAA|40729|11|18|49|AM|first|morning|| +40730|AAAAAAAALBPJAAAA|40730|11|18|50|AM|first|morning|| +40731|AAAAAAAAMBPJAAAA|40731|11|18|51|AM|first|morning|| +40732|AAAAAAAANBPJAAAA|40732|11|18|52|AM|first|morning|| +40733|AAAAAAAAOBPJAAAA|40733|11|18|53|AM|first|morning|| +40734|AAAAAAAAPBPJAAAA|40734|11|18|54|AM|first|morning|| +40735|AAAAAAAAACPJAAAA|40735|11|18|55|AM|first|morning|| +40736|AAAAAAAABCPJAAAA|40736|11|18|56|AM|first|morning|| +40737|AAAAAAAACCPJAAAA|40737|11|18|57|AM|first|morning|| +40738|AAAAAAAADCPJAAAA|40738|11|18|58|AM|first|morning|| +40739|AAAAAAAAECPJAAAA|40739|11|18|59|AM|first|morning|| +40740|AAAAAAAAFCPJAAAA|40740|11|19|0|AM|first|morning|| +40741|AAAAAAAAGCPJAAAA|40741|11|19|1|AM|first|morning|| +40742|AAAAAAAAHCPJAAAA|40742|11|19|2|AM|first|morning|| +40743|AAAAAAAAICPJAAAA|40743|11|19|3|AM|first|morning|| +40744|AAAAAAAAJCPJAAAA|40744|11|19|4|AM|first|morning|| +40745|AAAAAAAAKCPJAAAA|40745|11|19|5|AM|first|morning|| +40746|AAAAAAAALCPJAAAA|40746|11|19|6|AM|first|morning|| +40747|AAAAAAAAMCPJAAAA|40747|11|19|7|AM|first|morning|| +40748|AAAAAAAANCPJAAAA|40748|11|19|8|AM|first|morning|| +40749|AAAAAAAAOCPJAAAA|40749|11|19|9|AM|first|morning|| +40750|AAAAAAAAPCPJAAAA|40750|11|19|10|AM|first|morning|| +40751|AAAAAAAAADPJAAAA|40751|11|19|11|AM|first|morning|| +40752|AAAAAAAABDPJAAAA|40752|11|19|12|AM|first|morning|| +40753|AAAAAAAACDPJAAAA|40753|11|19|13|AM|first|morning|| +40754|AAAAAAAADDPJAAAA|40754|11|19|14|AM|first|morning|| +40755|AAAAAAAAEDPJAAAA|40755|11|19|15|AM|first|morning|| +40756|AAAAAAAAFDPJAAAA|40756|11|19|16|AM|first|morning|| +40757|AAAAAAAAGDPJAAAA|40757|11|19|17|AM|first|morning|| +40758|AAAAAAAAHDPJAAAA|40758|11|19|18|AM|first|morning|| +40759|AAAAAAAAIDPJAAAA|40759|11|19|19|AM|first|morning|| +40760|AAAAAAAAJDPJAAAA|40760|11|19|20|AM|first|morning|| +40761|AAAAAAAAKDPJAAAA|40761|11|19|21|AM|first|morning|| +40762|AAAAAAAALDPJAAAA|40762|11|19|22|AM|first|morning|| +40763|AAAAAAAAMDPJAAAA|40763|11|19|23|AM|first|morning|| +40764|AAAAAAAANDPJAAAA|40764|11|19|24|AM|first|morning|| +40765|AAAAAAAAODPJAAAA|40765|11|19|25|AM|first|morning|| +40766|AAAAAAAAPDPJAAAA|40766|11|19|26|AM|first|morning|| +40767|AAAAAAAAAEPJAAAA|40767|11|19|27|AM|first|morning|| +40768|AAAAAAAABEPJAAAA|40768|11|19|28|AM|first|morning|| +40769|AAAAAAAACEPJAAAA|40769|11|19|29|AM|first|morning|| +40770|AAAAAAAADEPJAAAA|40770|11|19|30|AM|first|morning|| +40771|AAAAAAAAEEPJAAAA|40771|11|19|31|AM|first|morning|| +40772|AAAAAAAAFEPJAAAA|40772|11|19|32|AM|first|morning|| +40773|AAAAAAAAGEPJAAAA|40773|11|19|33|AM|first|morning|| +40774|AAAAAAAAHEPJAAAA|40774|11|19|34|AM|first|morning|| +40775|AAAAAAAAIEPJAAAA|40775|11|19|35|AM|first|morning|| +40776|AAAAAAAAJEPJAAAA|40776|11|19|36|AM|first|morning|| +40777|AAAAAAAAKEPJAAAA|40777|11|19|37|AM|first|morning|| +40778|AAAAAAAALEPJAAAA|40778|11|19|38|AM|first|morning|| +40779|AAAAAAAAMEPJAAAA|40779|11|19|39|AM|first|morning|| +40780|AAAAAAAANEPJAAAA|40780|11|19|40|AM|first|morning|| +40781|AAAAAAAAOEPJAAAA|40781|11|19|41|AM|first|morning|| +40782|AAAAAAAAPEPJAAAA|40782|11|19|42|AM|first|morning|| +40783|AAAAAAAAAFPJAAAA|40783|11|19|43|AM|first|morning|| +40784|AAAAAAAABFPJAAAA|40784|11|19|44|AM|first|morning|| +40785|AAAAAAAACFPJAAAA|40785|11|19|45|AM|first|morning|| +40786|AAAAAAAADFPJAAAA|40786|11|19|46|AM|first|morning|| +40787|AAAAAAAAEFPJAAAA|40787|11|19|47|AM|first|morning|| +40788|AAAAAAAAFFPJAAAA|40788|11|19|48|AM|first|morning|| +40789|AAAAAAAAGFPJAAAA|40789|11|19|49|AM|first|morning|| +40790|AAAAAAAAHFPJAAAA|40790|11|19|50|AM|first|morning|| +40791|AAAAAAAAIFPJAAAA|40791|11|19|51|AM|first|morning|| +40792|AAAAAAAAJFPJAAAA|40792|11|19|52|AM|first|morning|| +40793|AAAAAAAAKFPJAAAA|40793|11|19|53|AM|first|morning|| +40794|AAAAAAAALFPJAAAA|40794|11|19|54|AM|first|morning|| +40795|AAAAAAAAMFPJAAAA|40795|11|19|55|AM|first|morning|| +40796|AAAAAAAANFPJAAAA|40796|11|19|56|AM|first|morning|| +40797|AAAAAAAAOFPJAAAA|40797|11|19|57|AM|first|morning|| +40798|AAAAAAAAPFPJAAAA|40798|11|19|58|AM|first|morning|| +40799|AAAAAAAAAGPJAAAA|40799|11|19|59|AM|first|morning|| +40800|AAAAAAAABGPJAAAA|40800|11|20|0|AM|first|morning|| +40801|AAAAAAAACGPJAAAA|40801|11|20|1|AM|first|morning|| +40802|AAAAAAAADGPJAAAA|40802|11|20|2|AM|first|morning|| +40803|AAAAAAAAEGPJAAAA|40803|11|20|3|AM|first|morning|| +40804|AAAAAAAAFGPJAAAA|40804|11|20|4|AM|first|morning|| +40805|AAAAAAAAGGPJAAAA|40805|11|20|5|AM|first|morning|| +40806|AAAAAAAAHGPJAAAA|40806|11|20|6|AM|first|morning|| +40807|AAAAAAAAIGPJAAAA|40807|11|20|7|AM|first|morning|| +40808|AAAAAAAAJGPJAAAA|40808|11|20|8|AM|first|morning|| +40809|AAAAAAAAKGPJAAAA|40809|11|20|9|AM|first|morning|| +40810|AAAAAAAALGPJAAAA|40810|11|20|10|AM|first|morning|| +40811|AAAAAAAAMGPJAAAA|40811|11|20|11|AM|first|morning|| +40812|AAAAAAAANGPJAAAA|40812|11|20|12|AM|first|morning|| +40813|AAAAAAAAOGPJAAAA|40813|11|20|13|AM|first|morning|| +40814|AAAAAAAAPGPJAAAA|40814|11|20|14|AM|first|morning|| +40815|AAAAAAAAAHPJAAAA|40815|11|20|15|AM|first|morning|| +40816|AAAAAAAABHPJAAAA|40816|11|20|16|AM|first|morning|| +40817|AAAAAAAACHPJAAAA|40817|11|20|17|AM|first|morning|| +40818|AAAAAAAADHPJAAAA|40818|11|20|18|AM|first|morning|| +40819|AAAAAAAAEHPJAAAA|40819|11|20|19|AM|first|morning|| +40820|AAAAAAAAFHPJAAAA|40820|11|20|20|AM|first|morning|| +40821|AAAAAAAAGHPJAAAA|40821|11|20|21|AM|first|morning|| +40822|AAAAAAAAHHPJAAAA|40822|11|20|22|AM|first|morning|| +40823|AAAAAAAAIHPJAAAA|40823|11|20|23|AM|first|morning|| +40824|AAAAAAAAJHPJAAAA|40824|11|20|24|AM|first|morning|| +40825|AAAAAAAAKHPJAAAA|40825|11|20|25|AM|first|morning|| +40826|AAAAAAAALHPJAAAA|40826|11|20|26|AM|first|morning|| +40827|AAAAAAAAMHPJAAAA|40827|11|20|27|AM|first|morning|| +40828|AAAAAAAANHPJAAAA|40828|11|20|28|AM|first|morning|| +40829|AAAAAAAAOHPJAAAA|40829|11|20|29|AM|first|morning|| +40830|AAAAAAAAPHPJAAAA|40830|11|20|30|AM|first|morning|| +40831|AAAAAAAAAIPJAAAA|40831|11|20|31|AM|first|morning|| +40832|AAAAAAAABIPJAAAA|40832|11|20|32|AM|first|morning|| +40833|AAAAAAAACIPJAAAA|40833|11|20|33|AM|first|morning|| +40834|AAAAAAAADIPJAAAA|40834|11|20|34|AM|first|morning|| +40835|AAAAAAAAEIPJAAAA|40835|11|20|35|AM|first|morning|| +40836|AAAAAAAAFIPJAAAA|40836|11|20|36|AM|first|morning|| +40837|AAAAAAAAGIPJAAAA|40837|11|20|37|AM|first|morning|| +40838|AAAAAAAAHIPJAAAA|40838|11|20|38|AM|first|morning|| +40839|AAAAAAAAIIPJAAAA|40839|11|20|39|AM|first|morning|| +40840|AAAAAAAAJIPJAAAA|40840|11|20|40|AM|first|morning|| +40841|AAAAAAAAKIPJAAAA|40841|11|20|41|AM|first|morning|| +40842|AAAAAAAALIPJAAAA|40842|11|20|42|AM|first|morning|| +40843|AAAAAAAAMIPJAAAA|40843|11|20|43|AM|first|morning|| +40844|AAAAAAAANIPJAAAA|40844|11|20|44|AM|first|morning|| +40845|AAAAAAAAOIPJAAAA|40845|11|20|45|AM|first|morning|| +40846|AAAAAAAAPIPJAAAA|40846|11|20|46|AM|first|morning|| +40847|AAAAAAAAAJPJAAAA|40847|11|20|47|AM|first|morning|| +40848|AAAAAAAABJPJAAAA|40848|11|20|48|AM|first|morning|| +40849|AAAAAAAACJPJAAAA|40849|11|20|49|AM|first|morning|| +40850|AAAAAAAADJPJAAAA|40850|11|20|50|AM|first|morning|| +40851|AAAAAAAAEJPJAAAA|40851|11|20|51|AM|first|morning|| +40852|AAAAAAAAFJPJAAAA|40852|11|20|52|AM|first|morning|| +40853|AAAAAAAAGJPJAAAA|40853|11|20|53|AM|first|morning|| +40854|AAAAAAAAHJPJAAAA|40854|11|20|54|AM|first|morning|| +40855|AAAAAAAAIJPJAAAA|40855|11|20|55|AM|first|morning|| +40856|AAAAAAAAJJPJAAAA|40856|11|20|56|AM|first|morning|| +40857|AAAAAAAAKJPJAAAA|40857|11|20|57|AM|first|morning|| +40858|AAAAAAAALJPJAAAA|40858|11|20|58|AM|first|morning|| +40859|AAAAAAAAMJPJAAAA|40859|11|20|59|AM|first|morning|| +40860|AAAAAAAANJPJAAAA|40860|11|21|0|AM|first|morning|| +40861|AAAAAAAAOJPJAAAA|40861|11|21|1|AM|first|morning|| +40862|AAAAAAAAPJPJAAAA|40862|11|21|2|AM|first|morning|| +40863|AAAAAAAAAKPJAAAA|40863|11|21|3|AM|first|morning|| +40864|AAAAAAAABKPJAAAA|40864|11|21|4|AM|first|morning|| +40865|AAAAAAAACKPJAAAA|40865|11|21|5|AM|first|morning|| +40866|AAAAAAAADKPJAAAA|40866|11|21|6|AM|first|morning|| +40867|AAAAAAAAEKPJAAAA|40867|11|21|7|AM|first|morning|| +40868|AAAAAAAAFKPJAAAA|40868|11|21|8|AM|first|morning|| +40869|AAAAAAAAGKPJAAAA|40869|11|21|9|AM|first|morning|| +40870|AAAAAAAAHKPJAAAA|40870|11|21|10|AM|first|morning|| +40871|AAAAAAAAIKPJAAAA|40871|11|21|11|AM|first|morning|| +40872|AAAAAAAAJKPJAAAA|40872|11|21|12|AM|first|morning|| +40873|AAAAAAAAKKPJAAAA|40873|11|21|13|AM|first|morning|| +40874|AAAAAAAALKPJAAAA|40874|11|21|14|AM|first|morning|| +40875|AAAAAAAAMKPJAAAA|40875|11|21|15|AM|first|morning|| +40876|AAAAAAAANKPJAAAA|40876|11|21|16|AM|first|morning|| +40877|AAAAAAAAOKPJAAAA|40877|11|21|17|AM|first|morning|| +40878|AAAAAAAAPKPJAAAA|40878|11|21|18|AM|first|morning|| +40879|AAAAAAAAALPJAAAA|40879|11|21|19|AM|first|morning|| +40880|AAAAAAAABLPJAAAA|40880|11|21|20|AM|first|morning|| +40881|AAAAAAAACLPJAAAA|40881|11|21|21|AM|first|morning|| +40882|AAAAAAAADLPJAAAA|40882|11|21|22|AM|first|morning|| +40883|AAAAAAAAELPJAAAA|40883|11|21|23|AM|first|morning|| +40884|AAAAAAAAFLPJAAAA|40884|11|21|24|AM|first|morning|| +40885|AAAAAAAAGLPJAAAA|40885|11|21|25|AM|first|morning|| +40886|AAAAAAAAHLPJAAAA|40886|11|21|26|AM|first|morning|| +40887|AAAAAAAAILPJAAAA|40887|11|21|27|AM|first|morning|| +40888|AAAAAAAAJLPJAAAA|40888|11|21|28|AM|first|morning|| +40889|AAAAAAAAKLPJAAAA|40889|11|21|29|AM|first|morning|| +40890|AAAAAAAALLPJAAAA|40890|11|21|30|AM|first|morning|| +40891|AAAAAAAAMLPJAAAA|40891|11|21|31|AM|first|morning|| +40892|AAAAAAAANLPJAAAA|40892|11|21|32|AM|first|morning|| +40893|AAAAAAAAOLPJAAAA|40893|11|21|33|AM|first|morning|| +40894|AAAAAAAAPLPJAAAA|40894|11|21|34|AM|first|morning|| +40895|AAAAAAAAAMPJAAAA|40895|11|21|35|AM|first|morning|| +40896|AAAAAAAABMPJAAAA|40896|11|21|36|AM|first|morning|| +40897|AAAAAAAACMPJAAAA|40897|11|21|37|AM|first|morning|| +40898|AAAAAAAADMPJAAAA|40898|11|21|38|AM|first|morning|| +40899|AAAAAAAAEMPJAAAA|40899|11|21|39|AM|first|morning|| +40900|AAAAAAAAFMPJAAAA|40900|11|21|40|AM|first|morning|| +40901|AAAAAAAAGMPJAAAA|40901|11|21|41|AM|first|morning|| +40902|AAAAAAAAHMPJAAAA|40902|11|21|42|AM|first|morning|| +40903|AAAAAAAAIMPJAAAA|40903|11|21|43|AM|first|morning|| +40904|AAAAAAAAJMPJAAAA|40904|11|21|44|AM|first|morning|| +40905|AAAAAAAAKMPJAAAA|40905|11|21|45|AM|first|morning|| +40906|AAAAAAAALMPJAAAA|40906|11|21|46|AM|first|morning|| +40907|AAAAAAAAMMPJAAAA|40907|11|21|47|AM|first|morning|| +40908|AAAAAAAANMPJAAAA|40908|11|21|48|AM|first|morning|| +40909|AAAAAAAAOMPJAAAA|40909|11|21|49|AM|first|morning|| +40910|AAAAAAAAPMPJAAAA|40910|11|21|50|AM|first|morning|| +40911|AAAAAAAAANPJAAAA|40911|11|21|51|AM|first|morning|| +40912|AAAAAAAABNPJAAAA|40912|11|21|52|AM|first|morning|| +40913|AAAAAAAACNPJAAAA|40913|11|21|53|AM|first|morning|| +40914|AAAAAAAADNPJAAAA|40914|11|21|54|AM|first|morning|| +40915|AAAAAAAAENPJAAAA|40915|11|21|55|AM|first|morning|| +40916|AAAAAAAAFNPJAAAA|40916|11|21|56|AM|first|morning|| +40917|AAAAAAAAGNPJAAAA|40917|11|21|57|AM|first|morning|| +40918|AAAAAAAAHNPJAAAA|40918|11|21|58|AM|first|morning|| +40919|AAAAAAAAINPJAAAA|40919|11|21|59|AM|first|morning|| +40920|AAAAAAAAJNPJAAAA|40920|11|22|0|AM|first|morning|| +40921|AAAAAAAAKNPJAAAA|40921|11|22|1|AM|first|morning|| +40922|AAAAAAAALNPJAAAA|40922|11|22|2|AM|first|morning|| +40923|AAAAAAAAMNPJAAAA|40923|11|22|3|AM|first|morning|| +40924|AAAAAAAANNPJAAAA|40924|11|22|4|AM|first|morning|| +40925|AAAAAAAAONPJAAAA|40925|11|22|5|AM|first|morning|| +40926|AAAAAAAAPNPJAAAA|40926|11|22|6|AM|first|morning|| +40927|AAAAAAAAAOPJAAAA|40927|11|22|7|AM|first|morning|| +40928|AAAAAAAABOPJAAAA|40928|11|22|8|AM|first|morning|| +40929|AAAAAAAACOPJAAAA|40929|11|22|9|AM|first|morning|| +40930|AAAAAAAADOPJAAAA|40930|11|22|10|AM|first|morning|| +40931|AAAAAAAAEOPJAAAA|40931|11|22|11|AM|first|morning|| +40932|AAAAAAAAFOPJAAAA|40932|11|22|12|AM|first|morning|| +40933|AAAAAAAAGOPJAAAA|40933|11|22|13|AM|first|morning|| +40934|AAAAAAAAHOPJAAAA|40934|11|22|14|AM|first|morning|| +40935|AAAAAAAAIOPJAAAA|40935|11|22|15|AM|first|morning|| +40936|AAAAAAAAJOPJAAAA|40936|11|22|16|AM|first|morning|| +40937|AAAAAAAAKOPJAAAA|40937|11|22|17|AM|first|morning|| +40938|AAAAAAAALOPJAAAA|40938|11|22|18|AM|first|morning|| +40939|AAAAAAAAMOPJAAAA|40939|11|22|19|AM|first|morning|| +40940|AAAAAAAANOPJAAAA|40940|11|22|20|AM|first|morning|| +40941|AAAAAAAAOOPJAAAA|40941|11|22|21|AM|first|morning|| +40942|AAAAAAAAPOPJAAAA|40942|11|22|22|AM|first|morning|| +40943|AAAAAAAAAPPJAAAA|40943|11|22|23|AM|first|morning|| +40944|AAAAAAAABPPJAAAA|40944|11|22|24|AM|first|morning|| +40945|AAAAAAAACPPJAAAA|40945|11|22|25|AM|first|morning|| +40946|AAAAAAAADPPJAAAA|40946|11|22|26|AM|first|morning|| +40947|AAAAAAAAEPPJAAAA|40947|11|22|27|AM|first|morning|| +40948|AAAAAAAAFPPJAAAA|40948|11|22|28|AM|first|morning|| +40949|AAAAAAAAGPPJAAAA|40949|11|22|29|AM|first|morning|| +40950|AAAAAAAAHPPJAAAA|40950|11|22|30|AM|first|morning|| +40951|AAAAAAAAIPPJAAAA|40951|11|22|31|AM|first|morning|| +40952|AAAAAAAAJPPJAAAA|40952|11|22|32|AM|first|morning|| +40953|AAAAAAAAKPPJAAAA|40953|11|22|33|AM|first|morning|| +40954|AAAAAAAALPPJAAAA|40954|11|22|34|AM|first|morning|| +40955|AAAAAAAAMPPJAAAA|40955|11|22|35|AM|first|morning|| +40956|AAAAAAAANPPJAAAA|40956|11|22|36|AM|first|morning|| +40957|AAAAAAAAOPPJAAAA|40957|11|22|37|AM|first|morning|| +40958|AAAAAAAAPPPJAAAA|40958|11|22|38|AM|first|morning|| +40959|AAAAAAAAAAAKAAAA|40959|11|22|39|AM|first|morning|| +40960|AAAAAAAABAAKAAAA|40960|11|22|40|AM|first|morning|| +40961|AAAAAAAACAAKAAAA|40961|11|22|41|AM|first|morning|| +40962|AAAAAAAADAAKAAAA|40962|11|22|42|AM|first|morning|| +40963|AAAAAAAAEAAKAAAA|40963|11|22|43|AM|first|morning|| +40964|AAAAAAAAFAAKAAAA|40964|11|22|44|AM|first|morning|| +40965|AAAAAAAAGAAKAAAA|40965|11|22|45|AM|first|morning|| +40966|AAAAAAAAHAAKAAAA|40966|11|22|46|AM|first|morning|| +40967|AAAAAAAAIAAKAAAA|40967|11|22|47|AM|first|morning|| +40968|AAAAAAAAJAAKAAAA|40968|11|22|48|AM|first|morning|| +40969|AAAAAAAAKAAKAAAA|40969|11|22|49|AM|first|morning|| +40970|AAAAAAAALAAKAAAA|40970|11|22|50|AM|first|morning|| +40971|AAAAAAAAMAAKAAAA|40971|11|22|51|AM|first|morning|| +40972|AAAAAAAANAAKAAAA|40972|11|22|52|AM|first|morning|| +40973|AAAAAAAAOAAKAAAA|40973|11|22|53|AM|first|morning|| +40974|AAAAAAAAPAAKAAAA|40974|11|22|54|AM|first|morning|| +40975|AAAAAAAAABAKAAAA|40975|11|22|55|AM|first|morning|| +40976|AAAAAAAABBAKAAAA|40976|11|22|56|AM|first|morning|| +40977|AAAAAAAACBAKAAAA|40977|11|22|57|AM|first|morning|| +40978|AAAAAAAADBAKAAAA|40978|11|22|58|AM|first|morning|| +40979|AAAAAAAAEBAKAAAA|40979|11|22|59|AM|first|morning|| +40980|AAAAAAAAFBAKAAAA|40980|11|23|0|AM|first|morning|| +40981|AAAAAAAAGBAKAAAA|40981|11|23|1|AM|first|morning|| +40982|AAAAAAAAHBAKAAAA|40982|11|23|2|AM|first|morning|| +40983|AAAAAAAAIBAKAAAA|40983|11|23|3|AM|first|morning|| +40984|AAAAAAAAJBAKAAAA|40984|11|23|4|AM|first|morning|| +40985|AAAAAAAAKBAKAAAA|40985|11|23|5|AM|first|morning|| +40986|AAAAAAAALBAKAAAA|40986|11|23|6|AM|first|morning|| +40987|AAAAAAAAMBAKAAAA|40987|11|23|7|AM|first|morning|| +40988|AAAAAAAANBAKAAAA|40988|11|23|8|AM|first|morning|| +40989|AAAAAAAAOBAKAAAA|40989|11|23|9|AM|first|morning|| +40990|AAAAAAAAPBAKAAAA|40990|11|23|10|AM|first|morning|| +40991|AAAAAAAAACAKAAAA|40991|11|23|11|AM|first|morning|| +40992|AAAAAAAABCAKAAAA|40992|11|23|12|AM|first|morning|| +40993|AAAAAAAACCAKAAAA|40993|11|23|13|AM|first|morning|| +40994|AAAAAAAADCAKAAAA|40994|11|23|14|AM|first|morning|| +40995|AAAAAAAAECAKAAAA|40995|11|23|15|AM|first|morning|| +40996|AAAAAAAAFCAKAAAA|40996|11|23|16|AM|first|morning|| +40997|AAAAAAAAGCAKAAAA|40997|11|23|17|AM|first|morning|| +40998|AAAAAAAAHCAKAAAA|40998|11|23|18|AM|first|morning|| +40999|AAAAAAAAICAKAAAA|40999|11|23|19|AM|first|morning|| +41000|AAAAAAAAJCAKAAAA|41000|11|23|20|AM|first|morning|| +41001|AAAAAAAAKCAKAAAA|41001|11|23|21|AM|first|morning|| +41002|AAAAAAAALCAKAAAA|41002|11|23|22|AM|first|morning|| +41003|AAAAAAAAMCAKAAAA|41003|11|23|23|AM|first|morning|| +41004|AAAAAAAANCAKAAAA|41004|11|23|24|AM|first|morning|| +41005|AAAAAAAAOCAKAAAA|41005|11|23|25|AM|first|morning|| +41006|AAAAAAAAPCAKAAAA|41006|11|23|26|AM|first|morning|| +41007|AAAAAAAAADAKAAAA|41007|11|23|27|AM|first|morning|| +41008|AAAAAAAABDAKAAAA|41008|11|23|28|AM|first|morning|| +41009|AAAAAAAACDAKAAAA|41009|11|23|29|AM|first|morning|| +41010|AAAAAAAADDAKAAAA|41010|11|23|30|AM|first|morning|| +41011|AAAAAAAAEDAKAAAA|41011|11|23|31|AM|first|morning|| +41012|AAAAAAAAFDAKAAAA|41012|11|23|32|AM|first|morning|| +41013|AAAAAAAAGDAKAAAA|41013|11|23|33|AM|first|morning|| +41014|AAAAAAAAHDAKAAAA|41014|11|23|34|AM|first|morning|| +41015|AAAAAAAAIDAKAAAA|41015|11|23|35|AM|first|morning|| +41016|AAAAAAAAJDAKAAAA|41016|11|23|36|AM|first|morning|| +41017|AAAAAAAAKDAKAAAA|41017|11|23|37|AM|first|morning|| +41018|AAAAAAAALDAKAAAA|41018|11|23|38|AM|first|morning|| +41019|AAAAAAAAMDAKAAAA|41019|11|23|39|AM|first|morning|| +41020|AAAAAAAANDAKAAAA|41020|11|23|40|AM|first|morning|| +41021|AAAAAAAAODAKAAAA|41021|11|23|41|AM|first|morning|| +41022|AAAAAAAAPDAKAAAA|41022|11|23|42|AM|first|morning|| +41023|AAAAAAAAAEAKAAAA|41023|11|23|43|AM|first|morning|| +41024|AAAAAAAABEAKAAAA|41024|11|23|44|AM|first|morning|| +41025|AAAAAAAACEAKAAAA|41025|11|23|45|AM|first|morning|| +41026|AAAAAAAADEAKAAAA|41026|11|23|46|AM|first|morning|| +41027|AAAAAAAAEEAKAAAA|41027|11|23|47|AM|first|morning|| +41028|AAAAAAAAFEAKAAAA|41028|11|23|48|AM|first|morning|| +41029|AAAAAAAAGEAKAAAA|41029|11|23|49|AM|first|morning|| +41030|AAAAAAAAHEAKAAAA|41030|11|23|50|AM|first|morning|| +41031|AAAAAAAAIEAKAAAA|41031|11|23|51|AM|first|morning|| +41032|AAAAAAAAJEAKAAAA|41032|11|23|52|AM|first|morning|| +41033|AAAAAAAAKEAKAAAA|41033|11|23|53|AM|first|morning|| +41034|AAAAAAAALEAKAAAA|41034|11|23|54|AM|first|morning|| +41035|AAAAAAAAMEAKAAAA|41035|11|23|55|AM|first|morning|| +41036|AAAAAAAANEAKAAAA|41036|11|23|56|AM|first|morning|| +41037|AAAAAAAAOEAKAAAA|41037|11|23|57|AM|first|morning|| +41038|AAAAAAAAPEAKAAAA|41038|11|23|58|AM|first|morning|| +41039|AAAAAAAAAFAKAAAA|41039|11|23|59|AM|first|morning|| +41040|AAAAAAAABFAKAAAA|41040|11|24|0|AM|first|morning|| +41041|AAAAAAAACFAKAAAA|41041|11|24|1|AM|first|morning|| +41042|AAAAAAAADFAKAAAA|41042|11|24|2|AM|first|morning|| +41043|AAAAAAAAEFAKAAAA|41043|11|24|3|AM|first|morning|| +41044|AAAAAAAAFFAKAAAA|41044|11|24|4|AM|first|morning|| +41045|AAAAAAAAGFAKAAAA|41045|11|24|5|AM|first|morning|| +41046|AAAAAAAAHFAKAAAA|41046|11|24|6|AM|first|morning|| +41047|AAAAAAAAIFAKAAAA|41047|11|24|7|AM|first|morning|| +41048|AAAAAAAAJFAKAAAA|41048|11|24|8|AM|first|morning|| +41049|AAAAAAAAKFAKAAAA|41049|11|24|9|AM|first|morning|| +41050|AAAAAAAALFAKAAAA|41050|11|24|10|AM|first|morning|| +41051|AAAAAAAAMFAKAAAA|41051|11|24|11|AM|first|morning|| +41052|AAAAAAAANFAKAAAA|41052|11|24|12|AM|first|morning|| +41053|AAAAAAAAOFAKAAAA|41053|11|24|13|AM|first|morning|| +41054|AAAAAAAAPFAKAAAA|41054|11|24|14|AM|first|morning|| +41055|AAAAAAAAAGAKAAAA|41055|11|24|15|AM|first|morning|| +41056|AAAAAAAABGAKAAAA|41056|11|24|16|AM|first|morning|| +41057|AAAAAAAACGAKAAAA|41057|11|24|17|AM|first|morning|| +41058|AAAAAAAADGAKAAAA|41058|11|24|18|AM|first|morning|| +41059|AAAAAAAAEGAKAAAA|41059|11|24|19|AM|first|morning|| +41060|AAAAAAAAFGAKAAAA|41060|11|24|20|AM|first|morning|| +41061|AAAAAAAAGGAKAAAA|41061|11|24|21|AM|first|morning|| +41062|AAAAAAAAHGAKAAAA|41062|11|24|22|AM|first|morning|| +41063|AAAAAAAAIGAKAAAA|41063|11|24|23|AM|first|morning|| +41064|AAAAAAAAJGAKAAAA|41064|11|24|24|AM|first|morning|| +41065|AAAAAAAAKGAKAAAA|41065|11|24|25|AM|first|morning|| +41066|AAAAAAAALGAKAAAA|41066|11|24|26|AM|first|morning|| +41067|AAAAAAAAMGAKAAAA|41067|11|24|27|AM|first|morning|| +41068|AAAAAAAANGAKAAAA|41068|11|24|28|AM|first|morning|| +41069|AAAAAAAAOGAKAAAA|41069|11|24|29|AM|first|morning|| +41070|AAAAAAAAPGAKAAAA|41070|11|24|30|AM|first|morning|| +41071|AAAAAAAAAHAKAAAA|41071|11|24|31|AM|first|morning|| +41072|AAAAAAAABHAKAAAA|41072|11|24|32|AM|first|morning|| +41073|AAAAAAAACHAKAAAA|41073|11|24|33|AM|first|morning|| +41074|AAAAAAAADHAKAAAA|41074|11|24|34|AM|first|morning|| +41075|AAAAAAAAEHAKAAAA|41075|11|24|35|AM|first|morning|| +41076|AAAAAAAAFHAKAAAA|41076|11|24|36|AM|first|morning|| +41077|AAAAAAAAGHAKAAAA|41077|11|24|37|AM|first|morning|| +41078|AAAAAAAAHHAKAAAA|41078|11|24|38|AM|first|morning|| +41079|AAAAAAAAIHAKAAAA|41079|11|24|39|AM|first|morning|| +41080|AAAAAAAAJHAKAAAA|41080|11|24|40|AM|first|morning|| +41081|AAAAAAAAKHAKAAAA|41081|11|24|41|AM|first|morning|| +41082|AAAAAAAALHAKAAAA|41082|11|24|42|AM|first|morning|| +41083|AAAAAAAAMHAKAAAA|41083|11|24|43|AM|first|morning|| +41084|AAAAAAAANHAKAAAA|41084|11|24|44|AM|first|morning|| +41085|AAAAAAAAOHAKAAAA|41085|11|24|45|AM|first|morning|| +41086|AAAAAAAAPHAKAAAA|41086|11|24|46|AM|first|morning|| +41087|AAAAAAAAAIAKAAAA|41087|11|24|47|AM|first|morning|| +41088|AAAAAAAABIAKAAAA|41088|11|24|48|AM|first|morning|| +41089|AAAAAAAACIAKAAAA|41089|11|24|49|AM|first|morning|| +41090|AAAAAAAADIAKAAAA|41090|11|24|50|AM|first|morning|| +41091|AAAAAAAAEIAKAAAA|41091|11|24|51|AM|first|morning|| +41092|AAAAAAAAFIAKAAAA|41092|11|24|52|AM|first|morning|| +41093|AAAAAAAAGIAKAAAA|41093|11|24|53|AM|first|morning|| +41094|AAAAAAAAHIAKAAAA|41094|11|24|54|AM|first|morning|| +41095|AAAAAAAAIIAKAAAA|41095|11|24|55|AM|first|morning|| +41096|AAAAAAAAJIAKAAAA|41096|11|24|56|AM|first|morning|| +41097|AAAAAAAAKIAKAAAA|41097|11|24|57|AM|first|morning|| +41098|AAAAAAAALIAKAAAA|41098|11|24|58|AM|first|morning|| +41099|AAAAAAAAMIAKAAAA|41099|11|24|59|AM|first|morning|| +41100|AAAAAAAANIAKAAAA|41100|11|25|0|AM|first|morning|| +41101|AAAAAAAAOIAKAAAA|41101|11|25|1|AM|first|morning|| +41102|AAAAAAAAPIAKAAAA|41102|11|25|2|AM|first|morning|| +41103|AAAAAAAAAJAKAAAA|41103|11|25|3|AM|first|morning|| +41104|AAAAAAAABJAKAAAA|41104|11|25|4|AM|first|morning|| +41105|AAAAAAAACJAKAAAA|41105|11|25|5|AM|first|morning|| +41106|AAAAAAAADJAKAAAA|41106|11|25|6|AM|first|morning|| +41107|AAAAAAAAEJAKAAAA|41107|11|25|7|AM|first|morning|| +41108|AAAAAAAAFJAKAAAA|41108|11|25|8|AM|first|morning|| +41109|AAAAAAAAGJAKAAAA|41109|11|25|9|AM|first|morning|| +41110|AAAAAAAAHJAKAAAA|41110|11|25|10|AM|first|morning|| +41111|AAAAAAAAIJAKAAAA|41111|11|25|11|AM|first|morning|| +41112|AAAAAAAAJJAKAAAA|41112|11|25|12|AM|first|morning|| +41113|AAAAAAAAKJAKAAAA|41113|11|25|13|AM|first|morning|| +41114|AAAAAAAALJAKAAAA|41114|11|25|14|AM|first|morning|| +41115|AAAAAAAAMJAKAAAA|41115|11|25|15|AM|first|morning|| +41116|AAAAAAAANJAKAAAA|41116|11|25|16|AM|first|morning|| +41117|AAAAAAAAOJAKAAAA|41117|11|25|17|AM|first|morning|| +41118|AAAAAAAAPJAKAAAA|41118|11|25|18|AM|first|morning|| +41119|AAAAAAAAAKAKAAAA|41119|11|25|19|AM|first|morning|| +41120|AAAAAAAABKAKAAAA|41120|11|25|20|AM|first|morning|| +41121|AAAAAAAACKAKAAAA|41121|11|25|21|AM|first|morning|| +41122|AAAAAAAADKAKAAAA|41122|11|25|22|AM|first|morning|| +41123|AAAAAAAAEKAKAAAA|41123|11|25|23|AM|first|morning|| +41124|AAAAAAAAFKAKAAAA|41124|11|25|24|AM|first|morning|| +41125|AAAAAAAAGKAKAAAA|41125|11|25|25|AM|first|morning|| +41126|AAAAAAAAHKAKAAAA|41126|11|25|26|AM|first|morning|| +41127|AAAAAAAAIKAKAAAA|41127|11|25|27|AM|first|morning|| +41128|AAAAAAAAJKAKAAAA|41128|11|25|28|AM|first|morning|| +41129|AAAAAAAAKKAKAAAA|41129|11|25|29|AM|first|morning|| +41130|AAAAAAAALKAKAAAA|41130|11|25|30|AM|first|morning|| +41131|AAAAAAAAMKAKAAAA|41131|11|25|31|AM|first|morning|| +41132|AAAAAAAANKAKAAAA|41132|11|25|32|AM|first|morning|| +41133|AAAAAAAAOKAKAAAA|41133|11|25|33|AM|first|morning|| +41134|AAAAAAAAPKAKAAAA|41134|11|25|34|AM|first|morning|| +41135|AAAAAAAAALAKAAAA|41135|11|25|35|AM|first|morning|| +41136|AAAAAAAABLAKAAAA|41136|11|25|36|AM|first|morning|| +41137|AAAAAAAACLAKAAAA|41137|11|25|37|AM|first|morning|| +41138|AAAAAAAADLAKAAAA|41138|11|25|38|AM|first|morning|| +41139|AAAAAAAAELAKAAAA|41139|11|25|39|AM|first|morning|| +41140|AAAAAAAAFLAKAAAA|41140|11|25|40|AM|first|morning|| +41141|AAAAAAAAGLAKAAAA|41141|11|25|41|AM|first|morning|| +41142|AAAAAAAAHLAKAAAA|41142|11|25|42|AM|first|morning|| +41143|AAAAAAAAILAKAAAA|41143|11|25|43|AM|first|morning|| +41144|AAAAAAAAJLAKAAAA|41144|11|25|44|AM|first|morning|| +41145|AAAAAAAAKLAKAAAA|41145|11|25|45|AM|first|morning|| +41146|AAAAAAAALLAKAAAA|41146|11|25|46|AM|first|morning|| +41147|AAAAAAAAMLAKAAAA|41147|11|25|47|AM|first|morning|| +41148|AAAAAAAANLAKAAAA|41148|11|25|48|AM|first|morning|| +41149|AAAAAAAAOLAKAAAA|41149|11|25|49|AM|first|morning|| +41150|AAAAAAAAPLAKAAAA|41150|11|25|50|AM|first|morning|| +41151|AAAAAAAAAMAKAAAA|41151|11|25|51|AM|first|morning|| +41152|AAAAAAAABMAKAAAA|41152|11|25|52|AM|first|morning|| +41153|AAAAAAAACMAKAAAA|41153|11|25|53|AM|first|morning|| +41154|AAAAAAAADMAKAAAA|41154|11|25|54|AM|first|morning|| +41155|AAAAAAAAEMAKAAAA|41155|11|25|55|AM|first|morning|| +41156|AAAAAAAAFMAKAAAA|41156|11|25|56|AM|first|morning|| +41157|AAAAAAAAGMAKAAAA|41157|11|25|57|AM|first|morning|| +41158|AAAAAAAAHMAKAAAA|41158|11|25|58|AM|first|morning|| +41159|AAAAAAAAIMAKAAAA|41159|11|25|59|AM|first|morning|| +41160|AAAAAAAAJMAKAAAA|41160|11|26|0|AM|first|morning|| +41161|AAAAAAAAKMAKAAAA|41161|11|26|1|AM|first|morning|| +41162|AAAAAAAALMAKAAAA|41162|11|26|2|AM|first|morning|| +41163|AAAAAAAAMMAKAAAA|41163|11|26|3|AM|first|morning|| +41164|AAAAAAAANMAKAAAA|41164|11|26|4|AM|first|morning|| +41165|AAAAAAAAOMAKAAAA|41165|11|26|5|AM|first|morning|| +41166|AAAAAAAAPMAKAAAA|41166|11|26|6|AM|first|morning|| +41167|AAAAAAAAANAKAAAA|41167|11|26|7|AM|first|morning|| +41168|AAAAAAAABNAKAAAA|41168|11|26|8|AM|first|morning|| +41169|AAAAAAAACNAKAAAA|41169|11|26|9|AM|first|morning|| +41170|AAAAAAAADNAKAAAA|41170|11|26|10|AM|first|morning|| +41171|AAAAAAAAENAKAAAA|41171|11|26|11|AM|first|morning|| +41172|AAAAAAAAFNAKAAAA|41172|11|26|12|AM|first|morning|| +41173|AAAAAAAAGNAKAAAA|41173|11|26|13|AM|first|morning|| +41174|AAAAAAAAHNAKAAAA|41174|11|26|14|AM|first|morning|| +41175|AAAAAAAAINAKAAAA|41175|11|26|15|AM|first|morning|| +41176|AAAAAAAAJNAKAAAA|41176|11|26|16|AM|first|morning|| +41177|AAAAAAAAKNAKAAAA|41177|11|26|17|AM|first|morning|| +41178|AAAAAAAALNAKAAAA|41178|11|26|18|AM|first|morning|| +41179|AAAAAAAAMNAKAAAA|41179|11|26|19|AM|first|morning|| +41180|AAAAAAAANNAKAAAA|41180|11|26|20|AM|first|morning|| +41181|AAAAAAAAONAKAAAA|41181|11|26|21|AM|first|morning|| +41182|AAAAAAAAPNAKAAAA|41182|11|26|22|AM|first|morning|| +41183|AAAAAAAAAOAKAAAA|41183|11|26|23|AM|first|morning|| +41184|AAAAAAAABOAKAAAA|41184|11|26|24|AM|first|morning|| +41185|AAAAAAAACOAKAAAA|41185|11|26|25|AM|first|morning|| +41186|AAAAAAAADOAKAAAA|41186|11|26|26|AM|first|morning|| +41187|AAAAAAAAEOAKAAAA|41187|11|26|27|AM|first|morning|| +41188|AAAAAAAAFOAKAAAA|41188|11|26|28|AM|first|morning|| +41189|AAAAAAAAGOAKAAAA|41189|11|26|29|AM|first|morning|| +41190|AAAAAAAAHOAKAAAA|41190|11|26|30|AM|first|morning|| +41191|AAAAAAAAIOAKAAAA|41191|11|26|31|AM|first|morning|| +41192|AAAAAAAAJOAKAAAA|41192|11|26|32|AM|first|morning|| +41193|AAAAAAAAKOAKAAAA|41193|11|26|33|AM|first|morning|| +41194|AAAAAAAALOAKAAAA|41194|11|26|34|AM|first|morning|| +41195|AAAAAAAAMOAKAAAA|41195|11|26|35|AM|first|morning|| +41196|AAAAAAAANOAKAAAA|41196|11|26|36|AM|first|morning|| +41197|AAAAAAAAOOAKAAAA|41197|11|26|37|AM|first|morning|| +41198|AAAAAAAAPOAKAAAA|41198|11|26|38|AM|first|morning|| +41199|AAAAAAAAAPAKAAAA|41199|11|26|39|AM|first|morning|| +41200|AAAAAAAABPAKAAAA|41200|11|26|40|AM|first|morning|| +41201|AAAAAAAACPAKAAAA|41201|11|26|41|AM|first|morning|| +41202|AAAAAAAADPAKAAAA|41202|11|26|42|AM|first|morning|| +41203|AAAAAAAAEPAKAAAA|41203|11|26|43|AM|first|morning|| +41204|AAAAAAAAFPAKAAAA|41204|11|26|44|AM|first|morning|| +41205|AAAAAAAAGPAKAAAA|41205|11|26|45|AM|first|morning|| +41206|AAAAAAAAHPAKAAAA|41206|11|26|46|AM|first|morning|| +41207|AAAAAAAAIPAKAAAA|41207|11|26|47|AM|first|morning|| +41208|AAAAAAAAJPAKAAAA|41208|11|26|48|AM|first|morning|| +41209|AAAAAAAAKPAKAAAA|41209|11|26|49|AM|first|morning|| +41210|AAAAAAAALPAKAAAA|41210|11|26|50|AM|first|morning|| +41211|AAAAAAAAMPAKAAAA|41211|11|26|51|AM|first|morning|| +41212|AAAAAAAANPAKAAAA|41212|11|26|52|AM|first|morning|| +41213|AAAAAAAAOPAKAAAA|41213|11|26|53|AM|first|morning|| +41214|AAAAAAAAPPAKAAAA|41214|11|26|54|AM|first|morning|| +41215|AAAAAAAAAABKAAAA|41215|11|26|55|AM|first|morning|| +41216|AAAAAAAABABKAAAA|41216|11|26|56|AM|first|morning|| +41217|AAAAAAAACABKAAAA|41217|11|26|57|AM|first|morning|| +41218|AAAAAAAADABKAAAA|41218|11|26|58|AM|first|morning|| +41219|AAAAAAAAEABKAAAA|41219|11|26|59|AM|first|morning|| +41220|AAAAAAAAFABKAAAA|41220|11|27|0|AM|first|morning|| +41221|AAAAAAAAGABKAAAA|41221|11|27|1|AM|first|morning|| +41222|AAAAAAAAHABKAAAA|41222|11|27|2|AM|first|morning|| +41223|AAAAAAAAIABKAAAA|41223|11|27|3|AM|first|morning|| +41224|AAAAAAAAJABKAAAA|41224|11|27|4|AM|first|morning|| +41225|AAAAAAAAKABKAAAA|41225|11|27|5|AM|first|morning|| +41226|AAAAAAAALABKAAAA|41226|11|27|6|AM|first|morning|| +41227|AAAAAAAAMABKAAAA|41227|11|27|7|AM|first|morning|| +41228|AAAAAAAANABKAAAA|41228|11|27|8|AM|first|morning|| +41229|AAAAAAAAOABKAAAA|41229|11|27|9|AM|first|morning|| +41230|AAAAAAAAPABKAAAA|41230|11|27|10|AM|first|morning|| +41231|AAAAAAAAABBKAAAA|41231|11|27|11|AM|first|morning|| +41232|AAAAAAAABBBKAAAA|41232|11|27|12|AM|first|morning|| +41233|AAAAAAAACBBKAAAA|41233|11|27|13|AM|first|morning|| +41234|AAAAAAAADBBKAAAA|41234|11|27|14|AM|first|morning|| +41235|AAAAAAAAEBBKAAAA|41235|11|27|15|AM|first|morning|| +41236|AAAAAAAAFBBKAAAA|41236|11|27|16|AM|first|morning|| +41237|AAAAAAAAGBBKAAAA|41237|11|27|17|AM|first|morning|| +41238|AAAAAAAAHBBKAAAA|41238|11|27|18|AM|first|morning|| +41239|AAAAAAAAIBBKAAAA|41239|11|27|19|AM|first|morning|| +41240|AAAAAAAAJBBKAAAA|41240|11|27|20|AM|first|morning|| +41241|AAAAAAAAKBBKAAAA|41241|11|27|21|AM|first|morning|| +41242|AAAAAAAALBBKAAAA|41242|11|27|22|AM|first|morning|| +41243|AAAAAAAAMBBKAAAA|41243|11|27|23|AM|first|morning|| +41244|AAAAAAAANBBKAAAA|41244|11|27|24|AM|first|morning|| +41245|AAAAAAAAOBBKAAAA|41245|11|27|25|AM|first|morning|| +41246|AAAAAAAAPBBKAAAA|41246|11|27|26|AM|first|morning|| +41247|AAAAAAAAACBKAAAA|41247|11|27|27|AM|first|morning|| +41248|AAAAAAAABCBKAAAA|41248|11|27|28|AM|first|morning|| +41249|AAAAAAAACCBKAAAA|41249|11|27|29|AM|first|morning|| +41250|AAAAAAAADCBKAAAA|41250|11|27|30|AM|first|morning|| +41251|AAAAAAAAECBKAAAA|41251|11|27|31|AM|first|morning|| +41252|AAAAAAAAFCBKAAAA|41252|11|27|32|AM|first|morning|| +41253|AAAAAAAAGCBKAAAA|41253|11|27|33|AM|first|morning|| +41254|AAAAAAAAHCBKAAAA|41254|11|27|34|AM|first|morning|| +41255|AAAAAAAAICBKAAAA|41255|11|27|35|AM|first|morning|| +41256|AAAAAAAAJCBKAAAA|41256|11|27|36|AM|first|morning|| +41257|AAAAAAAAKCBKAAAA|41257|11|27|37|AM|first|morning|| +41258|AAAAAAAALCBKAAAA|41258|11|27|38|AM|first|morning|| +41259|AAAAAAAAMCBKAAAA|41259|11|27|39|AM|first|morning|| +41260|AAAAAAAANCBKAAAA|41260|11|27|40|AM|first|morning|| +41261|AAAAAAAAOCBKAAAA|41261|11|27|41|AM|first|morning|| +41262|AAAAAAAAPCBKAAAA|41262|11|27|42|AM|first|morning|| +41263|AAAAAAAAADBKAAAA|41263|11|27|43|AM|first|morning|| +41264|AAAAAAAABDBKAAAA|41264|11|27|44|AM|first|morning|| +41265|AAAAAAAACDBKAAAA|41265|11|27|45|AM|first|morning|| +41266|AAAAAAAADDBKAAAA|41266|11|27|46|AM|first|morning|| +41267|AAAAAAAAEDBKAAAA|41267|11|27|47|AM|first|morning|| +41268|AAAAAAAAFDBKAAAA|41268|11|27|48|AM|first|morning|| +41269|AAAAAAAAGDBKAAAA|41269|11|27|49|AM|first|morning|| +41270|AAAAAAAAHDBKAAAA|41270|11|27|50|AM|first|morning|| +41271|AAAAAAAAIDBKAAAA|41271|11|27|51|AM|first|morning|| +41272|AAAAAAAAJDBKAAAA|41272|11|27|52|AM|first|morning|| +41273|AAAAAAAAKDBKAAAA|41273|11|27|53|AM|first|morning|| +41274|AAAAAAAALDBKAAAA|41274|11|27|54|AM|first|morning|| +41275|AAAAAAAAMDBKAAAA|41275|11|27|55|AM|first|morning|| +41276|AAAAAAAANDBKAAAA|41276|11|27|56|AM|first|morning|| +41277|AAAAAAAAODBKAAAA|41277|11|27|57|AM|first|morning|| +41278|AAAAAAAAPDBKAAAA|41278|11|27|58|AM|first|morning|| +41279|AAAAAAAAAEBKAAAA|41279|11|27|59|AM|first|morning|| +41280|AAAAAAAABEBKAAAA|41280|11|28|0|AM|first|morning|| +41281|AAAAAAAACEBKAAAA|41281|11|28|1|AM|first|morning|| +41282|AAAAAAAADEBKAAAA|41282|11|28|2|AM|first|morning|| +41283|AAAAAAAAEEBKAAAA|41283|11|28|3|AM|first|morning|| +41284|AAAAAAAAFEBKAAAA|41284|11|28|4|AM|first|morning|| +41285|AAAAAAAAGEBKAAAA|41285|11|28|5|AM|first|morning|| +41286|AAAAAAAAHEBKAAAA|41286|11|28|6|AM|first|morning|| +41287|AAAAAAAAIEBKAAAA|41287|11|28|7|AM|first|morning|| +41288|AAAAAAAAJEBKAAAA|41288|11|28|8|AM|first|morning|| +41289|AAAAAAAAKEBKAAAA|41289|11|28|9|AM|first|morning|| +41290|AAAAAAAALEBKAAAA|41290|11|28|10|AM|first|morning|| +41291|AAAAAAAAMEBKAAAA|41291|11|28|11|AM|first|morning|| +41292|AAAAAAAANEBKAAAA|41292|11|28|12|AM|first|morning|| +41293|AAAAAAAAOEBKAAAA|41293|11|28|13|AM|first|morning|| +41294|AAAAAAAAPEBKAAAA|41294|11|28|14|AM|first|morning|| +41295|AAAAAAAAAFBKAAAA|41295|11|28|15|AM|first|morning|| +41296|AAAAAAAABFBKAAAA|41296|11|28|16|AM|first|morning|| +41297|AAAAAAAACFBKAAAA|41297|11|28|17|AM|first|morning|| +41298|AAAAAAAADFBKAAAA|41298|11|28|18|AM|first|morning|| +41299|AAAAAAAAEFBKAAAA|41299|11|28|19|AM|first|morning|| +41300|AAAAAAAAFFBKAAAA|41300|11|28|20|AM|first|morning|| +41301|AAAAAAAAGFBKAAAA|41301|11|28|21|AM|first|morning|| +41302|AAAAAAAAHFBKAAAA|41302|11|28|22|AM|first|morning|| +41303|AAAAAAAAIFBKAAAA|41303|11|28|23|AM|first|morning|| +41304|AAAAAAAAJFBKAAAA|41304|11|28|24|AM|first|morning|| +41305|AAAAAAAAKFBKAAAA|41305|11|28|25|AM|first|morning|| +41306|AAAAAAAALFBKAAAA|41306|11|28|26|AM|first|morning|| +41307|AAAAAAAAMFBKAAAA|41307|11|28|27|AM|first|morning|| +41308|AAAAAAAANFBKAAAA|41308|11|28|28|AM|first|morning|| +41309|AAAAAAAAOFBKAAAA|41309|11|28|29|AM|first|morning|| +41310|AAAAAAAAPFBKAAAA|41310|11|28|30|AM|first|morning|| +41311|AAAAAAAAAGBKAAAA|41311|11|28|31|AM|first|morning|| +41312|AAAAAAAABGBKAAAA|41312|11|28|32|AM|first|morning|| +41313|AAAAAAAACGBKAAAA|41313|11|28|33|AM|first|morning|| +41314|AAAAAAAADGBKAAAA|41314|11|28|34|AM|first|morning|| +41315|AAAAAAAAEGBKAAAA|41315|11|28|35|AM|first|morning|| +41316|AAAAAAAAFGBKAAAA|41316|11|28|36|AM|first|morning|| +41317|AAAAAAAAGGBKAAAA|41317|11|28|37|AM|first|morning|| +41318|AAAAAAAAHGBKAAAA|41318|11|28|38|AM|first|morning|| +41319|AAAAAAAAIGBKAAAA|41319|11|28|39|AM|first|morning|| +41320|AAAAAAAAJGBKAAAA|41320|11|28|40|AM|first|morning|| +41321|AAAAAAAAKGBKAAAA|41321|11|28|41|AM|first|morning|| +41322|AAAAAAAALGBKAAAA|41322|11|28|42|AM|first|morning|| +41323|AAAAAAAAMGBKAAAA|41323|11|28|43|AM|first|morning|| +41324|AAAAAAAANGBKAAAA|41324|11|28|44|AM|first|morning|| +41325|AAAAAAAAOGBKAAAA|41325|11|28|45|AM|first|morning|| +41326|AAAAAAAAPGBKAAAA|41326|11|28|46|AM|first|morning|| +41327|AAAAAAAAAHBKAAAA|41327|11|28|47|AM|first|morning|| +41328|AAAAAAAABHBKAAAA|41328|11|28|48|AM|first|morning|| +41329|AAAAAAAACHBKAAAA|41329|11|28|49|AM|first|morning|| +41330|AAAAAAAADHBKAAAA|41330|11|28|50|AM|first|morning|| +41331|AAAAAAAAEHBKAAAA|41331|11|28|51|AM|first|morning|| +41332|AAAAAAAAFHBKAAAA|41332|11|28|52|AM|first|morning|| +41333|AAAAAAAAGHBKAAAA|41333|11|28|53|AM|first|morning|| +41334|AAAAAAAAHHBKAAAA|41334|11|28|54|AM|first|morning|| +41335|AAAAAAAAIHBKAAAA|41335|11|28|55|AM|first|morning|| +41336|AAAAAAAAJHBKAAAA|41336|11|28|56|AM|first|morning|| +41337|AAAAAAAAKHBKAAAA|41337|11|28|57|AM|first|morning|| +41338|AAAAAAAALHBKAAAA|41338|11|28|58|AM|first|morning|| +41339|AAAAAAAAMHBKAAAA|41339|11|28|59|AM|first|morning|| +41340|AAAAAAAANHBKAAAA|41340|11|29|0|AM|first|morning|| +41341|AAAAAAAAOHBKAAAA|41341|11|29|1|AM|first|morning|| +41342|AAAAAAAAPHBKAAAA|41342|11|29|2|AM|first|morning|| +41343|AAAAAAAAAIBKAAAA|41343|11|29|3|AM|first|morning|| +41344|AAAAAAAABIBKAAAA|41344|11|29|4|AM|first|morning|| +41345|AAAAAAAACIBKAAAA|41345|11|29|5|AM|first|morning|| +41346|AAAAAAAADIBKAAAA|41346|11|29|6|AM|first|morning|| +41347|AAAAAAAAEIBKAAAA|41347|11|29|7|AM|first|morning|| +41348|AAAAAAAAFIBKAAAA|41348|11|29|8|AM|first|morning|| +41349|AAAAAAAAGIBKAAAA|41349|11|29|9|AM|first|morning|| +41350|AAAAAAAAHIBKAAAA|41350|11|29|10|AM|first|morning|| +41351|AAAAAAAAIIBKAAAA|41351|11|29|11|AM|first|morning|| +41352|AAAAAAAAJIBKAAAA|41352|11|29|12|AM|first|morning|| +41353|AAAAAAAAKIBKAAAA|41353|11|29|13|AM|first|morning|| +41354|AAAAAAAALIBKAAAA|41354|11|29|14|AM|first|morning|| +41355|AAAAAAAAMIBKAAAA|41355|11|29|15|AM|first|morning|| +41356|AAAAAAAANIBKAAAA|41356|11|29|16|AM|first|morning|| +41357|AAAAAAAAOIBKAAAA|41357|11|29|17|AM|first|morning|| +41358|AAAAAAAAPIBKAAAA|41358|11|29|18|AM|first|morning|| +41359|AAAAAAAAAJBKAAAA|41359|11|29|19|AM|first|morning|| +41360|AAAAAAAABJBKAAAA|41360|11|29|20|AM|first|morning|| +41361|AAAAAAAACJBKAAAA|41361|11|29|21|AM|first|morning|| +41362|AAAAAAAADJBKAAAA|41362|11|29|22|AM|first|morning|| +41363|AAAAAAAAEJBKAAAA|41363|11|29|23|AM|first|morning|| +41364|AAAAAAAAFJBKAAAA|41364|11|29|24|AM|first|morning|| +41365|AAAAAAAAGJBKAAAA|41365|11|29|25|AM|first|morning|| +41366|AAAAAAAAHJBKAAAA|41366|11|29|26|AM|first|morning|| +41367|AAAAAAAAIJBKAAAA|41367|11|29|27|AM|first|morning|| +41368|AAAAAAAAJJBKAAAA|41368|11|29|28|AM|first|morning|| +41369|AAAAAAAAKJBKAAAA|41369|11|29|29|AM|first|morning|| +41370|AAAAAAAALJBKAAAA|41370|11|29|30|AM|first|morning|| +41371|AAAAAAAAMJBKAAAA|41371|11|29|31|AM|first|morning|| +41372|AAAAAAAANJBKAAAA|41372|11|29|32|AM|first|morning|| +41373|AAAAAAAAOJBKAAAA|41373|11|29|33|AM|first|morning|| +41374|AAAAAAAAPJBKAAAA|41374|11|29|34|AM|first|morning|| +41375|AAAAAAAAAKBKAAAA|41375|11|29|35|AM|first|morning|| +41376|AAAAAAAABKBKAAAA|41376|11|29|36|AM|first|morning|| +41377|AAAAAAAACKBKAAAA|41377|11|29|37|AM|first|morning|| +41378|AAAAAAAADKBKAAAA|41378|11|29|38|AM|first|morning|| +41379|AAAAAAAAEKBKAAAA|41379|11|29|39|AM|first|morning|| +41380|AAAAAAAAFKBKAAAA|41380|11|29|40|AM|first|morning|| +41381|AAAAAAAAGKBKAAAA|41381|11|29|41|AM|first|morning|| +41382|AAAAAAAAHKBKAAAA|41382|11|29|42|AM|first|morning|| +41383|AAAAAAAAIKBKAAAA|41383|11|29|43|AM|first|morning|| +41384|AAAAAAAAJKBKAAAA|41384|11|29|44|AM|first|morning|| +41385|AAAAAAAAKKBKAAAA|41385|11|29|45|AM|first|morning|| +41386|AAAAAAAALKBKAAAA|41386|11|29|46|AM|first|morning|| +41387|AAAAAAAAMKBKAAAA|41387|11|29|47|AM|first|morning|| +41388|AAAAAAAANKBKAAAA|41388|11|29|48|AM|first|morning|| +41389|AAAAAAAAOKBKAAAA|41389|11|29|49|AM|first|morning|| +41390|AAAAAAAAPKBKAAAA|41390|11|29|50|AM|first|morning|| +41391|AAAAAAAAALBKAAAA|41391|11|29|51|AM|first|morning|| +41392|AAAAAAAABLBKAAAA|41392|11|29|52|AM|first|morning|| +41393|AAAAAAAACLBKAAAA|41393|11|29|53|AM|first|morning|| +41394|AAAAAAAADLBKAAAA|41394|11|29|54|AM|first|morning|| +41395|AAAAAAAAELBKAAAA|41395|11|29|55|AM|first|morning|| +41396|AAAAAAAAFLBKAAAA|41396|11|29|56|AM|first|morning|| +41397|AAAAAAAAGLBKAAAA|41397|11|29|57|AM|first|morning|| +41398|AAAAAAAAHLBKAAAA|41398|11|29|58|AM|first|morning|| +41399|AAAAAAAAILBKAAAA|41399|11|29|59|AM|first|morning|| +41400|AAAAAAAAJLBKAAAA|41400|11|30|0|AM|first|morning|| +41401|AAAAAAAAKLBKAAAA|41401|11|30|1|AM|first|morning|| +41402|AAAAAAAALLBKAAAA|41402|11|30|2|AM|first|morning|| +41403|AAAAAAAAMLBKAAAA|41403|11|30|3|AM|first|morning|| +41404|AAAAAAAANLBKAAAA|41404|11|30|4|AM|first|morning|| +41405|AAAAAAAAOLBKAAAA|41405|11|30|5|AM|first|morning|| +41406|AAAAAAAAPLBKAAAA|41406|11|30|6|AM|first|morning|| +41407|AAAAAAAAAMBKAAAA|41407|11|30|7|AM|first|morning|| +41408|AAAAAAAABMBKAAAA|41408|11|30|8|AM|first|morning|| +41409|AAAAAAAACMBKAAAA|41409|11|30|9|AM|first|morning|| +41410|AAAAAAAADMBKAAAA|41410|11|30|10|AM|first|morning|| +41411|AAAAAAAAEMBKAAAA|41411|11|30|11|AM|first|morning|| +41412|AAAAAAAAFMBKAAAA|41412|11|30|12|AM|first|morning|| +41413|AAAAAAAAGMBKAAAA|41413|11|30|13|AM|first|morning|| +41414|AAAAAAAAHMBKAAAA|41414|11|30|14|AM|first|morning|| +41415|AAAAAAAAIMBKAAAA|41415|11|30|15|AM|first|morning|| +41416|AAAAAAAAJMBKAAAA|41416|11|30|16|AM|first|morning|| +41417|AAAAAAAAKMBKAAAA|41417|11|30|17|AM|first|morning|| +41418|AAAAAAAALMBKAAAA|41418|11|30|18|AM|first|morning|| +41419|AAAAAAAAMMBKAAAA|41419|11|30|19|AM|first|morning|| +41420|AAAAAAAANMBKAAAA|41420|11|30|20|AM|first|morning|| +41421|AAAAAAAAOMBKAAAA|41421|11|30|21|AM|first|morning|| +41422|AAAAAAAAPMBKAAAA|41422|11|30|22|AM|first|morning|| +41423|AAAAAAAAANBKAAAA|41423|11|30|23|AM|first|morning|| +41424|AAAAAAAABNBKAAAA|41424|11|30|24|AM|first|morning|| +41425|AAAAAAAACNBKAAAA|41425|11|30|25|AM|first|morning|| +41426|AAAAAAAADNBKAAAA|41426|11|30|26|AM|first|morning|| +41427|AAAAAAAAENBKAAAA|41427|11|30|27|AM|first|morning|| +41428|AAAAAAAAFNBKAAAA|41428|11|30|28|AM|first|morning|| +41429|AAAAAAAAGNBKAAAA|41429|11|30|29|AM|first|morning|| +41430|AAAAAAAAHNBKAAAA|41430|11|30|30|AM|first|morning|| +41431|AAAAAAAAINBKAAAA|41431|11|30|31|AM|first|morning|| +41432|AAAAAAAAJNBKAAAA|41432|11|30|32|AM|first|morning|| +41433|AAAAAAAAKNBKAAAA|41433|11|30|33|AM|first|morning|| +41434|AAAAAAAALNBKAAAA|41434|11|30|34|AM|first|morning|| +41435|AAAAAAAAMNBKAAAA|41435|11|30|35|AM|first|morning|| +41436|AAAAAAAANNBKAAAA|41436|11|30|36|AM|first|morning|| +41437|AAAAAAAAONBKAAAA|41437|11|30|37|AM|first|morning|| +41438|AAAAAAAAPNBKAAAA|41438|11|30|38|AM|first|morning|| +41439|AAAAAAAAAOBKAAAA|41439|11|30|39|AM|first|morning|| +41440|AAAAAAAABOBKAAAA|41440|11|30|40|AM|first|morning|| +41441|AAAAAAAACOBKAAAA|41441|11|30|41|AM|first|morning|| +41442|AAAAAAAADOBKAAAA|41442|11|30|42|AM|first|morning|| +41443|AAAAAAAAEOBKAAAA|41443|11|30|43|AM|first|morning|| +41444|AAAAAAAAFOBKAAAA|41444|11|30|44|AM|first|morning|| +41445|AAAAAAAAGOBKAAAA|41445|11|30|45|AM|first|morning|| +41446|AAAAAAAAHOBKAAAA|41446|11|30|46|AM|first|morning|| +41447|AAAAAAAAIOBKAAAA|41447|11|30|47|AM|first|morning|| +41448|AAAAAAAAJOBKAAAA|41448|11|30|48|AM|first|morning|| +41449|AAAAAAAAKOBKAAAA|41449|11|30|49|AM|first|morning|| +41450|AAAAAAAALOBKAAAA|41450|11|30|50|AM|first|morning|| +41451|AAAAAAAAMOBKAAAA|41451|11|30|51|AM|first|morning|| +41452|AAAAAAAANOBKAAAA|41452|11|30|52|AM|first|morning|| +41453|AAAAAAAAOOBKAAAA|41453|11|30|53|AM|first|morning|| +41454|AAAAAAAAPOBKAAAA|41454|11|30|54|AM|first|morning|| +41455|AAAAAAAAAPBKAAAA|41455|11|30|55|AM|first|morning|| +41456|AAAAAAAABPBKAAAA|41456|11|30|56|AM|first|morning|| +41457|AAAAAAAACPBKAAAA|41457|11|30|57|AM|first|morning|| +41458|AAAAAAAADPBKAAAA|41458|11|30|58|AM|first|morning|| +41459|AAAAAAAAEPBKAAAA|41459|11|30|59|AM|first|morning|| +41460|AAAAAAAAFPBKAAAA|41460|11|31|0|AM|first|morning|| +41461|AAAAAAAAGPBKAAAA|41461|11|31|1|AM|first|morning|| +41462|AAAAAAAAHPBKAAAA|41462|11|31|2|AM|first|morning|| +41463|AAAAAAAAIPBKAAAA|41463|11|31|3|AM|first|morning|| +41464|AAAAAAAAJPBKAAAA|41464|11|31|4|AM|first|morning|| +41465|AAAAAAAAKPBKAAAA|41465|11|31|5|AM|first|morning|| +41466|AAAAAAAALPBKAAAA|41466|11|31|6|AM|first|morning|| +41467|AAAAAAAAMPBKAAAA|41467|11|31|7|AM|first|morning|| +41468|AAAAAAAANPBKAAAA|41468|11|31|8|AM|first|morning|| +41469|AAAAAAAAOPBKAAAA|41469|11|31|9|AM|first|morning|| +41470|AAAAAAAAPPBKAAAA|41470|11|31|10|AM|first|morning|| +41471|AAAAAAAAAACKAAAA|41471|11|31|11|AM|first|morning|| +41472|AAAAAAAABACKAAAA|41472|11|31|12|AM|first|morning|| +41473|AAAAAAAACACKAAAA|41473|11|31|13|AM|first|morning|| +41474|AAAAAAAADACKAAAA|41474|11|31|14|AM|first|morning|| +41475|AAAAAAAAEACKAAAA|41475|11|31|15|AM|first|morning|| +41476|AAAAAAAAFACKAAAA|41476|11|31|16|AM|first|morning|| +41477|AAAAAAAAGACKAAAA|41477|11|31|17|AM|first|morning|| +41478|AAAAAAAAHACKAAAA|41478|11|31|18|AM|first|morning|| +41479|AAAAAAAAIACKAAAA|41479|11|31|19|AM|first|morning|| +41480|AAAAAAAAJACKAAAA|41480|11|31|20|AM|first|morning|| +41481|AAAAAAAAKACKAAAA|41481|11|31|21|AM|first|morning|| +41482|AAAAAAAALACKAAAA|41482|11|31|22|AM|first|morning|| +41483|AAAAAAAAMACKAAAA|41483|11|31|23|AM|first|morning|| +41484|AAAAAAAANACKAAAA|41484|11|31|24|AM|first|morning|| +41485|AAAAAAAAOACKAAAA|41485|11|31|25|AM|first|morning|| +41486|AAAAAAAAPACKAAAA|41486|11|31|26|AM|first|morning|| +41487|AAAAAAAAABCKAAAA|41487|11|31|27|AM|first|morning|| +41488|AAAAAAAABBCKAAAA|41488|11|31|28|AM|first|morning|| +41489|AAAAAAAACBCKAAAA|41489|11|31|29|AM|first|morning|| +41490|AAAAAAAADBCKAAAA|41490|11|31|30|AM|first|morning|| +41491|AAAAAAAAEBCKAAAA|41491|11|31|31|AM|first|morning|| +41492|AAAAAAAAFBCKAAAA|41492|11|31|32|AM|first|morning|| +41493|AAAAAAAAGBCKAAAA|41493|11|31|33|AM|first|morning|| +41494|AAAAAAAAHBCKAAAA|41494|11|31|34|AM|first|morning|| +41495|AAAAAAAAIBCKAAAA|41495|11|31|35|AM|first|morning|| +41496|AAAAAAAAJBCKAAAA|41496|11|31|36|AM|first|morning|| +41497|AAAAAAAAKBCKAAAA|41497|11|31|37|AM|first|morning|| +41498|AAAAAAAALBCKAAAA|41498|11|31|38|AM|first|morning|| +41499|AAAAAAAAMBCKAAAA|41499|11|31|39|AM|first|morning|| +41500|AAAAAAAANBCKAAAA|41500|11|31|40|AM|first|morning|| +41501|AAAAAAAAOBCKAAAA|41501|11|31|41|AM|first|morning|| +41502|AAAAAAAAPBCKAAAA|41502|11|31|42|AM|first|morning|| +41503|AAAAAAAAACCKAAAA|41503|11|31|43|AM|first|morning|| +41504|AAAAAAAABCCKAAAA|41504|11|31|44|AM|first|morning|| +41505|AAAAAAAACCCKAAAA|41505|11|31|45|AM|first|morning|| +41506|AAAAAAAADCCKAAAA|41506|11|31|46|AM|first|morning|| +41507|AAAAAAAAECCKAAAA|41507|11|31|47|AM|first|morning|| +41508|AAAAAAAAFCCKAAAA|41508|11|31|48|AM|first|morning|| +41509|AAAAAAAAGCCKAAAA|41509|11|31|49|AM|first|morning|| +41510|AAAAAAAAHCCKAAAA|41510|11|31|50|AM|first|morning|| +41511|AAAAAAAAICCKAAAA|41511|11|31|51|AM|first|morning|| +41512|AAAAAAAAJCCKAAAA|41512|11|31|52|AM|first|morning|| +41513|AAAAAAAAKCCKAAAA|41513|11|31|53|AM|first|morning|| +41514|AAAAAAAALCCKAAAA|41514|11|31|54|AM|first|morning|| +41515|AAAAAAAAMCCKAAAA|41515|11|31|55|AM|first|morning|| +41516|AAAAAAAANCCKAAAA|41516|11|31|56|AM|first|morning|| +41517|AAAAAAAAOCCKAAAA|41517|11|31|57|AM|first|morning|| +41518|AAAAAAAAPCCKAAAA|41518|11|31|58|AM|first|morning|| +41519|AAAAAAAAADCKAAAA|41519|11|31|59|AM|first|morning|| +41520|AAAAAAAABDCKAAAA|41520|11|32|0|AM|first|morning|| +41521|AAAAAAAACDCKAAAA|41521|11|32|1|AM|first|morning|| +41522|AAAAAAAADDCKAAAA|41522|11|32|2|AM|first|morning|| +41523|AAAAAAAAEDCKAAAA|41523|11|32|3|AM|first|morning|| +41524|AAAAAAAAFDCKAAAA|41524|11|32|4|AM|first|morning|| +41525|AAAAAAAAGDCKAAAA|41525|11|32|5|AM|first|morning|| +41526|AAAAAAAAHDCKAAAA|41526|11|32|6|AM|first|morning|| +41527|AAAAAAAAIDCKAAAA|41527|11|32|7|AM|first|morning|| +41528|AAAAAAAAJDCKAAAA|41528|11|32|8|AM|first|morning|| +41529|AAAAAAAAKDCKAAAA|41529|11|32|9|AM|first|morning|| +41530|AAAAAAAALDCKAAAA|41530|11|32|10|AM|first|morning|| +41531|AAAAAAAAMDCKAAAA|41531|11|32|11|AM|first|morning|| +41532|AAAAAAAANDCKAAAA|41532|11|32|12|AM|first|morning|| +41533|AAAAAAAAODCKAAAA|41533|11|32|13|AM|first|morning|| +41534|AAAAAAAAPDCKAAAA|41534|11|32|14|AM|first|morning|| +41535|AAAAAAAAAECKAAAA|41535|11|32|15|AM|first|morning|| +41536|AAAAAAAABECKAAAA|41536|11|32|16|AM|first|morning|| +41537|AAAAAAAACECKAAAA|41537|11|32|17|AM|first|morning|| +41538|AAAAAAAADECKAAAA|41538|11|32|18|AM|first|morning|| +41539|AAAAAAAAEECKAAAA|41539|11|32|19|AM|first|morning|| +41540|AAAAAAAAFECKAAAA|41540|11|32|20|AM|first|morning|| +41541|AAAAAAAAGECKAAAA|41541|11|32|21|AM|first|morning|| +41542|AAAAAAAAHECKAAAA|41542|11|32|22|AM|first|morning|| +41543|AAAAAAAAIECKAAAA|41543|11|32|23|AM|first|morning|| +41544|AAAAAAAAJECKAAAA|41544|11|32|24|AM|first|morning|| +41545|AAAAAAAAKECKAAAA|41545|11|32|25|AM|first|morning|| +41546|AAAAAAAALECKAAAA|41546|11|32|26|AM|first|morning|| +41547|AAAAAAAAMECKAAAA|41547|11|32|27|AM|first|morning|| +41548|AAAAAAAANECKAAAA|41548|11|32|28|AM|first|morning|| +41549|AAAAAAAAOECKAAAA|41549|11|32|29|AM|first|morning|| +41550|AAAAAAAAPECKAAAA|41550|11|32|30|AM|first|morning|| +41551|AAAAAAAAAFCKAAAA|41551|11|32|31|AM|first|morning|| +41552|AAAAAAAABFCKAAAA|41552|11|32|32|AM|first|morning|| +41553|AAAAAAAACFCKAAAA|41553|11|32|33|AM|first|morning|| +41554|AAAAAAAADFCKAAAA|41554|11|32|34|AM|first|morning|| +41555|AAAAAAAAEFCKAAAA|41555|11|32|35|AM|first|morning|| +41556|AAAAAAAAFFCKAAAA|41556|11|32|36|AM|first|morning|| +41557|AAAAAAAAGFCKAAAA|41557|11|32|37|AM|first|morning|| +41558|AAAAAAAAHFCKAAAA|41558|11|32|38|AM|first|morning|| +41559|AAAAAAAAIFCKAAAA|41559|11|32|39|AM|first|morning|| +41560|AAAAAAAAJFCKAAAA|41560|11|32|40|AM|first|morning|| +41561|AAAAAAAAKFCKAAAA|41561|11|32|41|AM|first|morning|| +41562|AAAAAAAALFCKAAAA|41562|11|32|42|AM|first|morning|| +41563|AAAAAAAAMFCKAAAA|41563|11|32|43|AM|first|morning|| +41564|AAAAAAAANFCKAAAA|41564|11|32|44|AM|first|morning|| +41565|AAAAAAAAOFCKAAAA|41565|11|32|45|AM|first|morning|| +41566|AAAAAAAAPFCKAAAA|41566|11|32|46|AM|first|morning|| +41567|AAAAAAAAAGCKAAAA|41567|11|32|47|AM|first|morning|| +41568|AAAAAAAABGCKAAAA|41568|11|32|48|AM|first|morning|| +41569|AAAAAAAACGCKAAAA|41569|11|32|49|AM|first|morning|| +41570|AAAAAAAADGCKAAAA|41570|11|32|50|AM|first|morning|| +41571|AAAAAAAAEGCKAAAA|41571|11|32|51|AM|first|morning|| +41572|AAAAAAAAFGCKAAAA|41572|11|32|52|AM|first|morning|| +41573|AAAAAAAAGGCKAAAA|41573|11|32|53|AM|first|morning|| +41574|AAAAAAAAHGCKAAAA|41574|11|32|54|AM|first|morning|| +41575|AAAAAAAAIGCKAAAA|41575|11|32|55|AM|first|morning|| +41576|AAAAAAAAJGCKAAAA|41576|11|32|56|AM|first|morning|| +41577|AAAAAAAAKGCKAAAA|41577|11|32|57|AM|first|morning|| +41578|AAAAAAAALGCKAAAA|41578|11|32|58|AM|first|morning|| +41579|AAAAAAAAMGCKAAAA|41579|11|32|59|AM|first|morning|| +41580|AAAAAAAANGCKAAAA|41580|11|33|0|AM|first|morning|| +41581|AAAAAAAAOGCKAAAA|41581|11|33|1|AM|first|morning|| +41582|AAAAAAAAPGCKAAAA|41582|11|33|2|AM|first|morning|| +41583|AAAAAAAAAHCKAAAA|41583|11|33|3|AM|first|morning|| +41584|AAAAAAAABHCKAAAA|41584|11|33|4|AM|first|morning|| +41585|AAAAAAAACHCKAAAA|41585|11|33|5|AM|first|morning|| +41586|AAAAAAAADHCKAAAA|41586|11|33|6|AM|first|morning|| +41587|AAAAAAAAEHCKAAAA|41587|11|33|7|AM|first|morning|| +41588|AAAAAAAAFHCKAAAA|41588|11|33|8|AM|first|morning|| +41589|AAAAAAAAGHCKAAAA|41589|11|33|9|AM|first|morning|| +41590|AAAAAAAAHHCKAAAA|41590|11|33|10|AM|first|morning|| +41591|AAAAAAAAIHCKAAAA|41591|11|33|11|AM|first|morning|| +41592|AAAAAAAAJHCKAAAA|41592|11|33|12|AM|first|morning|| +41593|AAAAAAAAKHCKAAAA|41593|11|33|13|AM|first|morning|| +41594|AAAAAAAALHCKAAAA|41594|11|33|14|AM|first|morning|| +41595|AAAAAAAAMHCKAAAA|41595|11|33|15|AM|first|morning|| +41596|AAAAAAAANHCKAAAA|41596|11|33|16|AM|first|morning|| +41597|AAAAAAAAOHCKAAAA|41597|11|33|17|AM|first|morning|| +41598|AAAAAAAAPHCKAAAA|41598|11|33|18|AM|first|morning|| +41599|AAAAAAAAAICKAAAA|41599|11|33|19|AM|first|morning|| +41600|AAAAAAAABICKAAAA|41600|11|33|20|AM|first|morning|| +41601|AAAAAAAACICKAAAA|41601|11|33|21|AM|first|morning|| +41602|AAAAAAAADICKAAAA|41602|11|33|22|AM|first|morning|| +41603|AAAAAAAAEICKAAAA|41603|11|33|23|AM|first|morning|| +41604|AAAAAAAAFICKAAAA|41604|11|33|24|AM|first|morning|| +41605|AAAAAAAAGICKAAAA|41605|11|33|25|AM|first|morning|| +41606|AAAAAAAAHICKAAAA|41606|11|33|26|AM|first|morning|| +41607|AAAAAAAAIICKAAAA|41607|11|33|27|AM|first|morning|| +41608|AAAAAAAAJICKAAAA|41608|11|33|28|AM|first|morning|| +41609|AAAAAAAAKICKAAAA|41609|11|33|29|AM|first|morning|| +41610|AAAAAAAALICKAAAA|41610|11|33|30|AM|first|morning|| +41611|AAAAAAAAMICKAAAA|41611|11|33|31|AM|first|morning|| +41612|AAAAAAAANICKAAAA|41612|11|33|32|AM|first|morning|| +41613|AAAAAAAAOICKAAAA|41613|11|33|33|AM|first|morning|| +41614|AAAAAAAAPICKAAAA|41614|11|33|34|AM|first|morning|| +41615|AAAAAAAAAJCKAAAA|41615|11|33|35|AM|first|morning|| +41616|AAAAAAAABJCKAAAA|41616|11|33|36|AM|first|morning|| +41617|AAAAAAAACJCKAAAA|41617|11|33|37|AM|first|morning|| +41618|AAAAAAAADJCKAAAA|41618|11|33|38|AM|first|morning|| +41619|AAAAAAAAEJCKAAAA|41619|11|33|39|AM|first|morning|| +41620|AAAAAAAAFJCKAAAA|41620|11|33|40|AM|first|morning|| +41621|AAAAAAAAGJCKAAAA|41621|11|33|41|AM|first|morning|| +41622|AAAAAAAAHJCKAAAA|41622|11|33|42|AM|first|morning|| +41623|AAAAAAAAIJCKAAAA|41623|11|33|43|AM|first|morning|| +41624|AAAAAAAAJJCKAAAA|41624|11|33|44|AM|first|morning|| +41625|AAAAAAAAKJCKAAAA|41625|11|33|45|AM|first|morning|| +41626|AAAAAAAALJCKAAAA|41626|11|33|46|AM|first|morning|| +41627|AAAAAAAAMJCKAAAA|41627|11|33|47|AM|first|morning|| +41628|AAAAAAAANJCKAAAA|41628|11|33|48|AM|first|morning|| +41629|AAAAAAAAOJCKAAAA|41629|11|33|49|AM|first|morning|| +41630|AAAAAAAAPJCKAAAA|41630|11|33|50|AM|first|morning|| +41631|AAAAAAAAAKCKAAAA|41631|11|33|51|AM|first|morning|| +41632|AAAAAAAABKCKAAAA|41632|11|33|52|AM|first|morning|| +41633|AAAAAAAACKCKAAAA|41633|11|33|53|AM|first|morning|| +41634|AAAAAAAADKCKAAAA|41634|11|33|54|AM|first|morning|| +41635|AAAAAAAAEKCKAAAA|41635|11|33|55|AM|first|morning|| +41636|AAAAAAAAFKCKAAAA|41636|11|33|56|AM|first|morning|| +41637|AAAAAAAAGKCKAAAA|41637|11|33|57|AM|first|morning|| +41638|AAAAAAAAHKCKAAAA|41638|11|33|58|AM|first|morning|| +41639|AAAAAAAAIKCKAAAA|41639|11|33|59|AM|first|morning|| +41640|AAAAAAAAJKCKAAAA|41640|11|34|0|AM|first|morning|| +41641|AAAAAAAAKKCKAAAA|41641|11|34|1|AM|first|morning|| +41642|AAAAAAAALKCKAAAA|41642|11|34|2|AM|first|morning|| +41643|AAAAAAAAMKCKAAAA|41643|11|34|3|AM|first|morning|| +41644|AAAAAAAANKCKAAAA|41644|11|34|4|AM|first|morning|| +41645|AAAAAAAAOKCKAAAA|41645|11|34|5|AM|first|morning|| +41646|AAAAAAAAPKCKAAAA|41646|11|34|6|AM|first|morning|| +41647|AAAAAAAAALCKAAAA|41647|11|34|7|AM|first|morning|| +41648|AAAAAAAABLCKAAAA|41648|11|34|8|AM|first|morning|| +41649|AAAAAAAACLCKAAAA|41649|11|34|9|AM|first|morning|| +41650|AAAAAAAADLCKAAAA|41650|11|34|10|AM|first|morning|| +41651|AAAAAAAAELCKAAAA|41651|11|34|11|AM|first|morning|| +41652|AAAAAAAAFLCKAAAA|41652|11|34|12|AM|first|morning|| +41653|AAAAAAAAGLCKAAAA|41653|11|34|13|AM|first|morning|| +41654|AAAAAAAAHLCKAAAA|41654|11|34|14|AM|first|morning|| +41655|AAAAAAAAILCKAAAA|41655|11|34|15|AM|first|morning|| +41656|AAAAAAAAJLCKAAAA|41656|11|34|16|AM|first|morning|| +41657|AAAAAAAAKLCKAAAA|41657|11|34|17|AM|first|morning|| +41658|AAAAAAAALLCKAAAA|41658|11|34|18|AM|first|morning|| +41659|AAAAAAAAMLCKAAAA|41659|11|34|19|AM|first|morning|| +41660|AAAAAAAANLCKAAAA|41660|11|34|20|AM|first|morning|| +41661|AAAAAAAAOLCKAAAA|41661|11|34|21|AM|first|morning|| +41662|AAAAAAAAPLCKAAAA|41662|11|34|22|AM|first|morning|| +41663|AAAAAAAAAMCKAAAA|41663|11|34|23|AM|first|morning|| +41664|AAAAAAAABMCKAAAA|41664|11|34|24|AM|first|morning|| +41665|AAAAAAAACMCKAAAA|41665|11|34|25|AM|first|morning|| +41666|AAAAAAAADMCKAAAA|41666|11|34|26|AM|first|morning|| +41667|AAAAAAAAEMCKAAAA|41667|11|34|27|AM|first|morning|| +41668|AAAAAAAAFMCKAAAA|41668|11|34|28|AM|first|morning|| +41669|AAAAAAAAGMCKAAAA|41669|11|34|29|AM|first|morning|| +41670|AAAAAAAAHMCKAAAA|41670|11|34|30|AM|first|morning|| +41671|AAAAAAAAIMCKAAAA|41671|11|34|31|AM|first|morning|| +41672|AAAAAAAAJMCKAAAA|41672|11|34|32|AM|first|morning|| +41673|AAAAAAAAKMCKAAAA|41673|11|34|33|AM|first|morning|| +41674|AAAAAAAALMCKAAAA|41674|11|34|34|AM|first|morning|| +41675|AAAAAAAAMMCKAAAA|41675|11|34|35|AM|first|morning|| +41676|AAAAAAAANMCKAAAA|41676|11|34|36|AM|first|morning|| +41677|AAAAAAAAOMCKAAAA|41677|11|34|37|AM|first|morning|| +41678|AAAAAAAAPMCKAAAA|41678|11|34|38|AM|first|morning|| +41679|AAAAAAAAANCKAAAA|41679|11|34|39|AM|first|morning|| +41680|AAAAAAAABNCKAAAA|41680|11|34|40|AM|first|morning|| +41681|AAAAAAAACNCKAAAA|41681|11|34|41|AM|first|morning|| +41682|AAAAAAAADNCKAAAA|41682|11|34|42|AM|first|morning|| +41683|AAAAAAAAENCKAAAA|41683|11|34|43|AM|first|morning|| +41684|AAAAAAAAFNCKAAAA|41684|11|34|44|AM|first|morning|| +41685|AAAAAAAAGNCKAAAA|41685|11|34|45|AM|first|morning|| +41686|AAAAAAAAHNCKAAAA|41686|11|34|46|AM|first|morning|| +41687|AAAAAAAAINCKAAAA|41687|11|34|47|AM|first|morning|| +41688|AAAAAAAAJNCKAAAA|41688|11|34|48|AM|first|morning|| +41689|AAAAAAAAKNCKAAAA|41689|11|34|49|AM|first|morning|| +41690|AAAAAAAALNCKAAAA|41690|11|34|50|AM|first|morning|| +41691|AAAAAAAAMNCKAAAA|41691|11|34|51|AM|first|morning|| +41692|AAAAAAAANNCKAAAA|41692|11|34|52|AM|first|morning|| +41693|AAAAAAAAONCKAAAA|41693|11|34|53|AM|first|morning|| +41694|AAAAAAAAPNCKAAAA|41694|11|34|54|AM|first|morning|| +41695|AAAAAAAAAOCKAAAA|41695|11|34|55|AM|first|morning|| +41696|AAAAAAAABOCKAAAA|41696|11|34|56|AM|first|morning|| +41697|AAAAAAAACOCKAAAA|41697|11|34|57|AM|first|morning|| +41698|AAAAAAAADOCKAAAA|41698|11|34|58|AM|first|morning|| +41699|AAAAAAAAEOCKAAAA|41699|11|34|59|AM|first|morning|| +41700|AAAAAAAAFOCKAAAA|41700|11|35|0|AM|first|morning|| +41701|AAAAAAAAGOCKAAAA|41701|11|35|1|AM|first|morning|| +41702|AAAAAAAAHOCKAAAA|41702|11|35|2|AM|first|morning|| +41703|AAAAAAAAIOCKAAAA|41703|11|35|3|AM|first|morning|| +41704|AAAAAAAAJOCKAAAA|41704|11|35|4|AM|first|morning|| +41705|AAAAAAAAKOCKAAAA|41705|11|35|5|AM|first|morning|| +41706|AAAAAAAALOCKAAAA|41706|11|35|6|AM|first|morning|| +41707|AAAAAAAAMOCKAAAA|41707|11|35|7|AM|first|morning|| +41708|AAAAAAAANOCKAAAA|41708|11|35|8|AM|first|morning|| +41709|AAAAAAAAOOCKAAAA|41709|11|35|9|AM|first|morning|| +41710|AAAAAAAAPOCKAAAA|41710|11|35|10|AM|first|morning|| +41711|AAAAAAAAAPCKAAAA|41711|11|35|11|AM|first|morning|| +41712|AAAAAAAABPCKAAAA|41712|11|35|12|AM|first|morning|| +41713|AAAAAAAACPCKAAAA|41713|11|35|13|AM|first|morning|| +41714|AAAAAAAADPCKAAAA|41714|11|35|14|AM|first|morning|| +41715|AAAAAAAAEPCKAAAA|41715|11|35|15|AM|first|morning|| +41716|AAAAAAAAFPCKAAAA|41716|11|35|16|AM|first|morning|| +41717|AAAAAAAAGPCKAAAA|41717|11|35|17|AM|first|morning|| +41718|AAAAAAAAHPCKAAAA|41718|11|35|18|AM|first|morning|| +41719|AAAAAAAAIPCKAAAA|41719|11|35|19|AM|first|morning|| +41720|AAAAAAAAJPCKAAAA|41720|11|35|20|AM|first|morning|| +41721|AAAAAAAAKPCKAAAA|41721|11|35|21|AM|first|morning|| +41722|AAAAAAAALPCKAAAA|41722|11|35|22|AM|first|morning|| +41723|AAAAAAAAMPCKAAAA|41723|11|35|23|AM|first|morning|| +41724|AAAAAAAANPCKAAAA|41724|11|35|24|AM|first|morning|| +41725|AAAAAAAAOPCKAAAA|41725|11|35|25|AM|first|morning|| +41726|AAAAAAAAPPCKAAAA|41726|11|35|26|AM|first|morning|| +41727|AAAAAAAAAADKAAAA|41727|11|35|27|AM|first|morning|| +41728|AAAAAAAABADKAAAA|41728|11|35|28|AM|first|morning|| +41729|AAAAAAAACADKAAAA|41729|11|35|29|AM|first|morning|| +41730|AAAAAAAADADKAAAA|41730|11|35|30|AM|first|morning|| +41731|AAAAAAAAEADKAAAA|41731|11|35|31|AM|first|morning|| +41732|AAAAAAAAFADKAAAA|41732|11|35|32|AM|first|morning|| +41733|AAAAAAAAGADKAAAA|41733|11|35|33|AM|first|morning|| +41734|AAAAAAAAHADKAAAA|41734|11|35|34|AM|first|morning|| +41735|AAAAAAAAIADKAAAA|41735|11|35|35|AM|first|morning|| +41736|AAAAAAAAJADKAAAA|41736|11|35|36|AM|first|morning|| +41737|AAAAAAAAKADKAAAA|41737|11|35|37|AM|first|morning|| +41738|AAAAAAAALADKAAAA|41738|11|35|38|AM|first|morning|| +41739|AAAAAAAAMADKAAAA|41739|11|35|39|AM|first|morning|| +41740|AAAAAAAANADKAAAA|41740|11|35|40|AM|first|morning|| +41741|AAAAAAAAOADKAAAA|41741|11|35|41|AM|first|morning|| +41742|AAAAAAAAPADKAAAA|41742|11|35|42|AM|first|morning|| +41743|AAAAAAAAABDKAAAA|41743|11|35|43|AM|first|morning|| +41744|AAAAAAAABBDKAAAA|41744|11|35|44|AM|first|morning|| +41745|AAAAAAAACBDKAAAA|41745|11|35|45|AM|first|morning|| +41746|AAAAAAAADBDKAAAA|41746|11|35|46|AM|first|morning|| +41747|AAAAAAAAEBDKAAAA|41747|11|35|47|AM|first|morning|| +41748|AAAAAAAAFBDKAAAA|41748|11|35|48|AM|first|morning|| +41749|AAAAAAAAGBDKAAAA|41749|11|35|49|AM|first|morning|| +41750|AAAAAAAAHBDKAAAA|41750|11|35|50|AM|first|morning|| +41751|AAAAAAAAIBDKAAAA|41751|11|35|51|AM|first|morning|| +41752|AAAAAAAAJBDKAAAA|41752|11|35|52|AM|first|morning|| +41753|AAAAAAAAKBDKAAAA|41753|11|35|53|AM|first|morning|| +41754|AAAAAAAALBDKAAAA|41754|11|35|54|AM|first|morning|| +41755|AAAAAAAAMBDKAAAA|41755|11|35|55|AM|first|morning|| +41756|AAAAAAAANBDKAAAA|41756|11|35|56|AM|first|morning|| +41757|AAAAAAAAOBDKAAAA|41757|11|35|57|AM|first|morning|| +41758|AAAAAAAAPBDKAAAA|41758|11|35|58|AM|first|morning|| +41759|AAAAAAAAACDKAAAA|41759|11|35|59|AM|first|morning|| +41760|AAAAAAAABCDKAAAA|41760|11|36|0|AM|first|morning|| +41761|AAAAAAAACCDKAAAA|41761|11|36|1|AM|first|morning|| +41762|AAAAAAAADCDKAAAA|41762|11|36|2|AM|first|morning|| +41763|AAAAAAAAECDKAAAA|41763|11|36|3|AM|first|morning|| +41764|AAAAAAAAFCDKAAAA|41764|11|36|4|AM|first|morning|| +41765|AAAAAAAAGCDKAAAA|41765|11|36|5|AM|first|morning|| +41766|AAAAAAAAHCDKAAAA|41766|11|36|6|AM|first|morning|| +41767|AAAAAAAAICDKAAAA|41767|11|36|7|AM|first|morning|| +41768|AAAAAAAAJCDKAAAA|41768|11|36|8|AM|first|morning|| +41769|AAAAAAAAKCDKAAAA|41769|11|36|9|AM|first|morning|| +41770|AAAAAAAALCDKAAAA|41770|11|36|10|AM|first|morning|| +41771|AAAAAAAAMCDKAAAA|41771|11|36|11|AM|first|morning|| +41772|AAAAAAAANCDKAAAA|41772|11|36|12|AM|first|morning|| +41773|AAAAAAAAOCDKAAAA|41773|11|36|13|AM|first|morning|| +41774|AAAAAAAAPCDKAAAA|41774|11|36|14|AM|first|morning|| +41775|AAAAAAAAADDKAAAA|41775|11|36|15|AM|first|morning|| +41776|AAAAAAAABDDKAAAA|41776|11|36|16|AM|first|morning|| +41777|AAAAAAAACDDKAAAA|41777|11|36|17|AM|first|morning|| +41778|AAAAAAAADDDKAAAA|41778|11|36|18|AM|first|morning|| +41779|AAAAAAAAEDDKAAAA|41779|11|36|19|AM|first|morning|| +41780|AAAAAAAAFDDKAAAA|41780|11|36|20|AM|first|morning|| +41781|AAAAAAAAGDDKAAAA|41781|11|36|21|AM|first|morning|| +41782|AAAAAAAAHDDKAAAA|41782|11|36|22|AM|first|morning|| +41783|AAAAAAAAIDDKAAAA|41783|11|36|23|AM|first|morning|| +41784|AAAAAAAAJDDKAAAA|41784|11|36|24|AM|first|morning|| +41785|AAAAAAAAKDDKAAAA|41785|11|36|25|AM|first|morning|| +41786|AAAAAAAALDDKAAAA|41786|11|36|26|AM|first|morning|| +41787|AAAAAAAAMDDKAAAA|41787|11|36|27|AM|first|morning|| +41788|AAAAAAAANDDKAAAA|41788|11|36|28|AM|first|morning|| +41789|AAAAAAAAODDKAAAA|41789|11|36|29|AM|first|morning|| +41790|AAAAAAAAPDDKAAAA|41790|11|36|30|AM|first|morning|| +41791|AAAAAAAAAEDKAAAA|41791|11|36|31|AM|first|morning|| +41792|AAAAAAAABEDKAAAA|41792|11|36|32|AM|first|morning|| +41793|AAAAAAAACEDKAAAA|41793|11|36|33|AM|first|morning|| +41794|AAAAAAAADEDKAAAA|41794|11|36|34|AM|first|morning|| +41795|AAAAAAAAEEDKAAAA|41795|11|36|35|AM|first|morning|| +41796|AAAAAAAAFEDKAAAA|41796|11|36|36|AM|first|morning|| +41797|AAAAAAAAGEDKAAAA|41797|11|36|37|AM|first|morning|| +41798|AAAAAAAAHEDKAAAA|41798|11|36|38|AM|first|morning|| +41799|AAAAAAAAIEDKAAAA|41799|11|36|39|AM|first|morning|| +41800|AAAAAAAAJEDKAAAA|41800|11|36|40|AM|first|morning|| +41801|AAAAAAAAKEDKAAAA|41801|11|36|41|AM|first|morning|| +41802|AAAAAAAALEDKAAAA|41802|11|36|42|AM|first|morning|| +41803|AAAAAAAAMEDKAAAA|41803|11|36|43|AM|first|morning|| +41804|AAAAAAAANEDKAAAA|41804|11|36|44|AM|first|morning|| +41805|AAAAAAAAOEDKAAAA|41805|11|36|45|AM|first|morning|| +41806|AAAAAAAAPEDKAAAA|41806|11|36|46|AM|first|morning|| +41807|AAAAAAAAAFDKAAAA|41807|11|36|47|AM|first|morning|| +41808|AAAAAAAABFDKAAAA|41808|11|36|48|AM|first|morning|| +41809|AAAAAAAACFDKAAAA|41809|11|36|49|AM|first|morning|| +41810|AAAAAAAADFDKAAAA|41810|11|36|50|AM|first|morning|| +41811|AAAAAAAAEFDKAAAA|41811|11|36|51|AM|first|morning|| +41812|AAAAAAAAFFDKAAAA|41812|11|36|52|AM|first|morning|| +41813|AAAAAAAAGFDKAAAA|41813|11|36|53|AM|first|morning|| +41814|AAAAAAAAHFDKAAAA|41814|11|36|54|AM|first|morning|| +41815|AAAAAAAAIFDKAAAA|41815|11|36|55|AM|first|morning|| +41816|AAAAAAAAJFDKAAAA|41816|11|36|56|AM|first|morning|| +41817|AAAAAAAAKFDKAAAA|41817|11|36|57|AM|first|morning|| +41818|AAAAAAAALFDKAAAA|41818|11|36|58|AM|first|morning|| +41819|AAAAAAAAMFDKAAAA|41819|11|36|59|AM|first|morning|| +41820|AAAAAAAANFDKAAAA|41820|11|37|0|AM|first|morning|| +41821|AAAAAAAAOFDKAAAA|41821|11|37|1|AM|first|morning|| +41822|AAAAAAAAPFDKAAAA|41822|11|37|2|AM|first|morning|| +41823|AAAAAAAAAGDKAAAA|41823|11|37|3|AM|first|morning|| +41824|AAAAAAAABGDKAAAA|41824|11|37|4|AM|first|morning|| +41825|AAAAAAAACGDKAAAA|41825|11|37|5|AM|first|morning|| +41826|AAAAAAAADGDKAAAA|41826|11|37|6|AM|first|morning|| +41827|AAAAAAAAEGDKAAAA|41827|11|37|7|AM|first|morning|| +41828|AAAAAAAAFGDKAAAA|41828|11|37|8|AM|first|morning|| +41829|AAAAAAAAGGDKAAAA|41829|11|37|9|AM|first|morning|| +41830|AAAAAAAAHGDKAAAA|41830|11|37|10|AM|first|morning|| +41831|AAAAAAAAIGDKAAAA|41831|11|37|11|AM|first|morning|| +41832|AAAAAAAAJGDKAAAA|41832|11|37|12|AM|first|morning|| +41833|AAAAAAAAKGDKAAAA|41833|11|37|13|AM|first|morning|| +41834|AAAAAAAALGDKAAAA|41834|11|37|14|AM|first|morning|| +41835|AAAAAAAAMGDKAAAA|41835|11|37|15|AM|first|morning|| +41836|AAAAAAAANGDKAAAA|41836|11|37|16|AM|first|morning|| +41837|AAAAAAAAOGDKAAAA|41837|11|37|17|AM|first|morning|| +41838|AAAAAAAAPGDKAAAA|41838|11|37|18|AM|first|morning|| +41839|AAAAAAAAAHDKAAAA|41839|11|37|19|AM|first|morning|| +41840|AAAAAAAABHDKAAAA|41840|11|37|20|AM|first|morning|| +41841|AAAAAAAACHDKAAAA|41841|11|37|21|AM|first|morning|| +41842|AAAAAAAADHDKAAAA|41842|11|37|22|AM|first|morning|| +41843|AAAAAAAAEHDKAAAA|41843|11|37|23|AM|first|morning|| +41844|AAAAAAAAFHDKAAAA|41844|11|37|24|AM|first|morning|| +41845|AAAAAAAAGHDKAAAA|41845|11|37|25|AM|first|morning|| +41846|AAAAAAAAHHDKAAAA|41846|11|37|26|AM|first|morning|| +41847|AAAAAAAAIHDKAAAA|41847|11|37|27|AM|first|morning|| +41848|AAAAAAAAJHDKAAAA|41848|11|37|28|AM|first|morning|| +41849|AAAAAAAAKHDKAAAA|41849|11|37|29|AM|first|morning|| +41850|AAAAAAAALHDKAAAA|41850|11|37|30|AM|first|morning|| +41851|AAAAAAAAMHDKAAAA|41851|11|37|31|AM|first|morning|| +41852|AAAAAAAANHDKAAAA|41852|11|37|32|AM|first|morning|| +41853|AAAAAAAAOHDKAAAA|41853|11|37|33|AM|first|morning|| +41854|AAAAAAAAPHDKAAAA|41854|11|37|34|AM|first|morning|| +41855|AAAAAAAAAIDKAAAA|41855|11|37|35|AM|first|morning|| +41856|AAAAAAAABIDKAAAA|41856|11|37|36|AM|first|morning|| +41857|AAAAAAAACIDKAAAA|41857|11|37|37|AM|first|morning|| +41858|AAAAAAAADIDKAAAA|41858|11|37|38|AM|first|morning|| +41859|AAAAAAAAEIDKAAAA|41859|11|37|39|AM|first|morning|| +41860|AAAAAAAAFIDKAAAA|41860|11|37|40|AM|first|morning|| +41861|AAAAAAAAGIDKAAAA|41861|11|37|41|AM|first|morning|| +41862|AAAAAAAAHIDKAAAA|41862|11|37|42|AM|first|morning|| +41863|AAAAAAAAIIDKAAAA|41863|11|37|43|AM|first|morning|| +41864|AAAAAAAAJIDKAAAA|41864|11|37|44|AM|first|morning|| +41865|AAAAAAAAKIDKAAAA|41865|11|37|45|AM|first|morning|| +41866|AAAAAAAALIDKAAAA|41866|11|37|46|AM|first|morning|| +41867|AAAAAAAAMIDKAAAA|41867|11|37|47|AM|first|morning|| +41868|AAAAAAAANIDKAAAA|41868|11|37|48|AM|first|morning|| +41869|AAAAAAAAOIDKAAAA|41869|11|37|49|AM|first|morning|| +41870|AAAAAAAAPIDKAAAA|41870|11|37|50|AM|first|morning|| +41871|AAAAAAAAAJDKAAAA|41871|11|37|51|AM|first|morning|| +41872|AAAAAAAABJDKAAAA|41872|11|37|52|AM|first|morning|| +41873|AAAAAAAACJDKAAAA|41873|11|37|53|AM|first|morning|| +41874|AAAAAAAADJDKAAAA|41874|11|37|54|AM|first|morning|| +41875|AAAAAAAAEJDKAAAA|41875|11|37|55|AM|first|morning|| +41876|AAAAAAAAFJDKAAAA|41876|11|37|56|AM|first|morning|| +41877|AAAAAAAAGJDKAAAA|41877|11|37|57|AM|first|morning|| +41878|AAAAAAAAHJDKAAAA|41878|11|37|58|AM|first|morning|| +41879|AAAAAAAAIJDKAAAA|41879|11|37|59|AM|first|morning|| +41880|AAAAAAAAJJDKAAAA|41880|11|38|0|AM|first|morning|| +41881|AAAAAAAAKJDKAAAA|41881|11|38|1|AM|first|morning|| +41882|AAAAAAAALJDKAAAA|41882|11|38|2|AM|first|morning|| +41883|AAAAAAAAMJDKAAAA|41883|11|38|3|AM|first|morning|| +41884|AAAAAAAANJDKAAAA|41884|11|38|4|AM|first|morning|| +41885|AAAAAAAAOJDKAAAA|41885|11|38|5|AM|first|morning|| +41886|AAAAAAAAPJDKAAAA|41886|11|38|6|AM|first|morning|| +41887|AAAAAAAAAKDKAAAA|41887|11|38|7|AM|first|morning|| +41888|AAAAAAAABKDKAAAA|41888|11|38|8|AM|first|morning|| +41889|AAAAAAAACKDKAAAA|41889|11|38|9|AM|first|morning|| +41890|AAAAAAAADKDKAAAA|41890|11|38|10|AM|first|morning|| +41891|AAAAAAAAEKDKAAAA|41891|11|38|11|AM|first|morning|| +41892|AAAAAAAAFKDKAAAA|41892|11|38|12|AM|first|morning|| +41893|AAAAAAAAGKDKAAAA|41893|11|38|13|AM|first|morning|| +41894|AAAAAAAAHKDKAAAA|41894|11|38|14|AM|first|morning|| +41895|AAAAAAAAIKDKAAAA|41895|11|38|15|AM|first|morning|| +41896|AAAAAAAAJKDKAAAA|41896|11|38|16|AM|first|morning|| +41897|AAAAAAAAKKDKAAAA|41897|11|38|17|AM|first|morning|| +41898|AAAAAAAALKDKAAAA|41898|11|38|18|AM|first|morning|| +41899|AAAAAAAAMKDKAAAA|41899|11|38|19|AM|first|morning|| +41900|AAAAAAAANKDKAAAA|41900|11|38|20|AM|first|morning|| +41901|AAAAAAAAOKDKAAAA|41901|11|38|21|AM|first|morning|| +41902|AAAAAAAAPKDKAAAA|41902|11|38|22|AM|first|morning|| +41903|AAAAAAAAALDKAAAA|41903|11|38|23|AM|first|morning|| +41904|AAAAAAAABLDKAAAA|41904|11|38|24|AM|first|morning|| +41905|AAAAAAAACLDKAAAA|41905|11|38|25|AM|first|morning|| +41906|AAAAAAAADLDKAAAA|41906|11|38|26|AM|first|morning|| +41907|AAAAAAAAELDKAAAA|41907|11|38|27|AM|first|morning|| +41908|AAAAAAAAFLDKAAAA|41908|11|38|28|AM|first|morning|| +41909|AAAAAAAAGLDKAAAA|41909|11|38|29|AM|first|morning|| +41910|AAAAAAAAHLDKAAAA|41910|11|38|30|AM|first|morning|| +41911|AAAAAAAAILDKAAAA|41911|11|38|31|AM|first|morning|| +41912|AAAAAAAAJLDKAAAA|41912|11|38|32|AM|first|morning|| +41913|AAAAAAAAKLDKAAAA|41913|11|38|33|AM|first|morning|| +41914|AAAAAAAALLDKAAAA|41914|11|38|34|AM|first|morning|| +41915|AAAAAAAAMLDKAAAA|41915|11|38|35|AM|first|morning|| +41916|AAAAAAAANLDKAAAA|41916|11|38|36|AM|first|morning|| +41917|AAAAAAAAOLDKAAAA|41917|11|38|37|AM|first|morning|| +41918|AAAAAAAAPLDKAAAA|41918|11|38|38|AM|first|morning|| +41919|AAAAAAAAAMDKAAAA|41919|11|38|39|AM|first|morning|| +41920|AAAAAAAABMDKAAAA|41920|11|38|40|AM|first|morning|| +41921|AAAAAAAACMDKAAAA|41921|11|38|41|AM|first|morning|| +41922|AAAAAAAADMDKAAAA|41922|11|38|42|AM|first|morning|| +41923|AAAAAAAAEMDKAAAA|41923|11|38|43|AM|first|morning|| +41924|AAAAAAAAFMDKAAAA|41924|11|38|44|AM|first|morning|| +41925|AAAAAAAAGMDKAAAA|41925|11|38|45|AM|first|morning|| +41926|AAAAAAAAHMDKAAAA|41926|11|38|46|AM|first|morning|| +41927|AAAAAAAAIMDKAAAA|41927|11|38|47|AM|first|morning|| +41928|AAAAAAAAJMDKAAAA|41928|11|38|48|AM|first|morning|| +41929|AAAAAAAAKMDKAAAA|41929|11|38|49|AM|first|morning|| +41930|AAAAAAAALMDKAAAA|41930|11|38|50|AM|first|morning|| +41931|AAAAAAAAMMDKAAAA|41931|11|38|51|AM|first|morning|| +41932|AAAAAAAANMDKAAAA|41932|11|38|52|AM|first|morning|| +41933|AAAAAAAAOMDKAAAA|41933|11|38|53|AM|first|morning|| +41934|AAAAAAAAPMDKAAAA|41934|11|38|54|AM|first|morning|| +41935|AAAAAAAAANDKAAAA|41935|11|38|55|AM|first|morning|| +41936|AAAAAAAABNDKAAAA|41936|11|38|56|AM|first|morning|| +41937|AAAAAAAACNDKAAAA|41937|11|38|57|AM|first|morning|| +41938|AAAAAAAADNDKAAAA|41938|11|38|58|AM|first|morning|| +41939|AAAAAAAAENDKAAAA|41939|11|38|59|AM|first|morning|| +41940|AAAAAAAAFNDKAAAA|41940|11|39|0|AM|first|morning|| +41941|AAAAAAAAGNDKAAAA|41941|11|39|1|AM|first|morning|| +41942|AAAAAAAAHNDKAAAA|41942|11|39|2|AM|first|morning|| +41943|AAAAAAAAINDKAAAA|41943|11|39|3|AM|first|morning|| +41944|AAAAAAAAJNDKAAAA|41944|11|39|4|AM|first|morning|| +41945|AAAAAAAAKNDKAAAA|41945|11|39|5|AM|first|morning|| +41946|AAAAAAAALNDKAAAA|41946|11|39|6|AM|first|morning|| +41947|AAAAAAAAMNDKAAAA|41947|11|39|7|AM|first|morning|| +41948|AAAAAAAANNDKAAAA|41948|11|39|8|AM|first|morning|| +41949|AAAAAAAAONDKAAAA|41949|11|39|9|AM|first|morning|| +41950|AAAAAAAAPNDKAAAA|41950|11|39|10|AM|first|morning|| +41951|AAAAAAAAAODKAAAA|41951|11|39|11|AM|first|morning|| +41952|AAAAAAAABODKAAAA|41952|11|39|12|AM|first|morning|| +41953|AAAAAAAACODKAAAA|41953|11|39|13|AM|first|morning|| +41954|AAAAAAAADODKAAAA|41954|11|39|14|AM|first|morning|| +41955|AAAAAAAAEODKAAAA|41955|11|39|15|AM|first|morning|| +41956|AAAAAAAAFODKAAAA|41956|11|39|16|AM|first|morning|| +41957|AAAAAAAAGODKAAAA|41957|11|39|17|AM|first|morning|| +41958|AAAAAAAAHODKAAAA|41958|11|39|18|AM|first|morning|| +41959|AAAAAAAAIODKAAAA|41959|11|39|19|AM|first|morning|| +41960|AAAAAAAAJODKAAAA|41960|11|39|20|AM|first|morning|| +41961|AAAAAAAAKODKAAAA|41961|11|39|21|AM|first|morning|| +41962|AAAAAAAALODKAAAA|41962|11|39|22|AM|first|morning|| +41963|AAAAAAAAMODKAAAA|41963|11|39|23|AM|first|morning|| +41964|AAAAAAAANODKAAAA|41964|11|39|24|AM|first|morning|| +41965|AAAAAAAAOODKAAAA|41965|11|39|25|AM|first|morning|| +41966|AAAAAAAAPODKAAAA|41966|11|39|26|AM|first|morning|| +41967|AAAAAAAAAPDKAAAA|41967|11|39|27|AM|first|morning|| +41968|AAAAAAAABPDKAAAA|41968|11|39|28|AM|first|morning|| +41969|AAAAAAAACPDKAAAA|41969|11|39|29|AM|first|morning|| +41970|AAAAAAAADPDKAAAA|41970|11|39|30|AM|first|morning|| +41971|AAAAAAAAEPDKAAAA|41971|11|39|31|AM|first|morning|| +41972|AAAAAAAAFPDKAAAA|41972|11|39|32|AM|first|morning|| +41973|AAAAAAAAGPDKAAAA|41973|11|39|33|AM|first|morning|| +41974|AAAAAAAAHPDKAAAA|41974|11|39|34|AM|first|morning|| +41975|AAAAAAAAIPDKAAAA|41975|11|39|35|AM|first|morning|| +41976|AAAAAAAAJPDKAAAA|41976|11|39|36|AM|first|morning|| +41977|AAAAAAAAKPDKAAAA|41977|11|39|37|AM|first|morning|| +41978|AAAAAAAALPDKAAAA|41978|11|39|38|AM|first|morning|| +41979|AAAAAAAAMPDKAAAA|41979|11|39|39|AM|first|morning|| +41980|AAAAAAAANPDKAAAA|41980|11|39|40|AM|first|morning|| +41981|AAAAAAAAOPDKAAAA|41981|11|39|41|AM|first|morning|| +41982|AAAAAAAAPPDKAAAA|41982|11|39|42|AM|first|morning|| +41983|AAAAAAAAAAEKAAAA|41983|11|39|43|AM|first|morning|| +41984|AAAAAAAABAEKAAAA|41984|11|39|44|AM|first|morning|| +41985|AAAAAAAACAEKAAAA|41985|11|39|45|AM|first|morning|| +41986|AAAAAAAADAEKAAAA|41986|11|39|46|AM|first|morning|| +41987|AAAAAAAAEAEKAAAA|41987|11|39|47|AM|first|morning|| +41988|AAAAAAAAFAEKAAAA|41988|11|39|48|AM|first|morning|| +41989|AAAAAAAAGAEKAAAA|41989|11|39|49|AM|first|morning|| +41990|AAAAAAAAHAEKAAAA|41990|11|39|50|AM|first|morning|| +41991|AAAAAAAAIAEKAAAA|41991|11|39|51|AM|first|morning|| +41992|AAAAAAAAJAEKAAAA|41992|11|39|52|AM|first|morning|| +41993|AAAAAAAAKAEKAAAA|41993|11|39|53|AM|first|morning|| +41994|AAAAAAAALAEKAAAA|41994|11|39|54|AM|first|morning|| +41995|AAAAAAAAMAEKAAAA|41995|11|39|55|AM|first|morning|| +41996|AAAAAAAANAEKAAAA|41996|11|39|56|AM|first|morning|| +41997|AAAAAAAAOAEKAAAA|41997|11|39|57|AM|first|morning|| +41998|AAAAAAAAPAEKAAAA|41998|11|39|58|AM|first|morning|| +41999|AAAAAAAAABEKAAAA|41999|11|39|59|AM|first|morning|| +42000|AAAAAAAABBEKAAAA|42000|11|40|0|AM|first|morning|| +42001|AAAAAAAACBEKAAAA|42001|11|40|1|AM|first|morning|| +42002|AAAAAAAADBEKAAAA|42002|11|40|2|AM|first|morning|| +42003|AAAAAAAAEBEKAAAA|42003|11|40|3|AM|first|morning|| +42004|AAAAAAAAFBEKAAAA|42004|11|40|4|AM|first|morning|| +42005|AAAAAAAAGBEKAAAA|42005|11|40|5|AM|first|morning|| +42006|AAAAAAAAHBEKAAAA|42006|11|40|6|AM|first|morning|| +42007|AAAAAAAAIBEKAAAA|42007|11|40|7|AM|first|morning|| +42008|AAAAAAAAJBEKAAAA|42008|11|40|8|AM|first|morning|| +42009|AAAAAAAAKBEKAAAA|42009|11|40|9|AM|first|morning|| +42010|AAAAAAAALBEKAAAA|42010|11|40|10|AM|first|morning|| +42011|AAAAAAAAMBEKAAAA|42011|11|40|11|AM|first|morning|| +42012|AAAAAAAANBEKAAAA|42012|11|40|12|AM|first|morning|| +42013|AAAAAAAAOBEKAAAA|42013|11|40|13|AM|first|morning|| +42014|AAAAAAAAPBEKAAAA|42014|11|40|14|AM|first|morning|| +42015|AAAAAAAAACEKAAAA|42015|11|40|15|AM|first|morning|| +42016|AAAAAAAABCEKAAAA|42016|11|40|16|AM|first|morning|| +42017|AAAAAAAACCEKAAAA|42017|11|40|17|AM|first|morning|| +42018|AAAAAAAADCEKAAAA|42018|11|40|18|AM|first|morning|| +42019|AAAAAAAAECEKAAAA|42019|11|40|19|AM|first|morning|| +42020|AAAAAAAAFCEKAAAA|42020|11|40|20|AM|first|morning|| +42021|AAAAAAAAGCEKAAAA|42021|11|40|21|AM|first|morning|| +42022|AAAAAAAAHCEKAAAA|42022|11|40|22|AM|first|morning|| +42023|AAAAAAAAICEKAAAA|42023|11|40|23|AM|first|morning|| +42024|AAAAAAAAJCEKAAAA|42024|11|40|24|AM|first|morning|| +42025|AAAAAAAAKCEKAAAA|42025|11|40|25|AM|first|morning|| +42026|AAAAAAAALCEKAAAA|42026|11|40|26|AM|first|morning|| +42027|AAAAAAAAMCEKAAAA|42027|11|40|27|AM|first|morning|| +42028|AAAAAAAANCEKAAAA|42028|11|40|28|AM|first|morning|| +42029|AAAAAAAAOCEKAAAA|42029|11|40|29|AM|first|morning|| +42030|AAAAAAAAPCEKAAAA|42030|11|40|30|AM|first|morning|| +42031|AAAAAAAAADEKAAAA|42031|11|40|31|AM|first|morning|| +42032|AAAAAAAABDEKAAAA|42032|11|40|32|AM|first|morning|| +42033|AAAAAAAACDEKAAAA|42033|11|40|33|AM|first|morning|| +42034|AAAAAAAADDEKAAAA|42034|11|40|34|AM|first|morning|| +42035|AAAAAAAAEDEKAAAA|42035|11|40|35|AM|first|morning|| +42036|AAAAAAAAFDEKAAAA|42036|11|40|36|AM|first|morning|| +42037|AAAAAAAAGDEKAAAA|42037|11|40|37|AM|first|morning|| +42038|AAAAAAAAHDEKAAAA|42038|11|40|38|AM|first|morning|| +42039|AAAAAAAAIDEKAAAA|42039|11|40|39|AM|first|morning|| +42040|AAAAAAAAJDEKAAAA|42040|11|40|40|AM|first|morning|| +42041|AAAAAAAAKDEKAAAA|42041|11|40|41|AM|first|morning|| +42042|AAAAAAAALDEKAAAA|42042|11|40|42|AM|first|morning|| +42043|AAAAAAAAMDEKAAAA|42043|11|40|43|AM|first|morning|| +42044|AAAAAAAANDEKAAAA|42044|11|40|44|AM|first|morning|| +42045|AAAAAAAAODEKAAAA|42045|11|40|45|AM|first|morning|| +42046|AAAAAAAAPDEKAAAA|42046|11|40|46|AM|first|morning|| +42047|AAAAAAAAAEEKAAAA|42047|11|40|47|AM|first|morning|| +42048|AAAAAAAABEEKAAAA|42048|11|40|48|AM|first|morning|| +42049|AAAAAAAACEEKAAAA|42049|11|40|49|AM|first|morning|| +42050|AAAAAAAADEEKAAAA|42050|11|40|50|AM|first|morning|| +42051|AAAAAAAAEEEKAAAA|42051|11|40|51|AM|first|morning|| +42052|AAAAAAAAFEEKAAAA|42052|11|40|52|AM|first|morning|| +42053|AAAAAAAAGEEKAAAA|42053|11|40|53|AM|first|morning|| +42054|AAAAAAAAHEEKAAAA|42054|11|40|54|AM|first|morning|| +42055|AAAAAAAAIEEKAAAA|42055|11|40|55|AM|first|morning|| +42056|AAAAAAAAJEEKAAAA|42056|11|40|56|AM|first|morning|| +42057|AAAAAAAAKEEKAAAA|42057|11|40|57|AM|first|morning|| +42058|AAAAAAAALEEKAAAA|42058|11|40|58|AM|first|morning|| +42059|AAAAAAAAMEEKAAAA|42059|11|40|59|AM|first|morning|| +42060|AAAAAAAANEEKAAAA|42060|11|41|0|AM|first|morning|| +42061|AAAAAAAAOEEKAAAA|42061|11|41|1|AM|first|morning|| +42062|AAAAAAAAPEEKAAAA|42062|11|41|2|AM|first|morning|| +42063|AAAAAAAAAFEKAAAA|42063|11|41|3|AM|first|morning|| +42064|AAAAAAAABFEKAAAA|42064|11|41|4|AM|first|morning|| +42065|AAAAAAAACFEKAAAA|42065|11|41|5|AM|first|morning|| +42066|AAAAAAAADFEKAAAA|42066|11|41|6|AM|first|morning|| +42067|AAAAAAAAEFEKAAAA|42067|11|41|7|AM|first|morning|| +42068|AAAAAAAAFFEKAAAA|42068|11|41|8|AM|first|morning|| +42069|AAAAAAAAGFEKAAAA|42069|11|41|9|AM|first|morning|| +42070|AAAAAAAAHFEKAAAA|42070|11|41|10|AM|first|morning|| +42071|AAAAAAAAIFEKAAAA|42071|11|41|11|AM|first|morning|| +42072|AAAAAAAAJFEKAAAA|42072|11|41|12|AM|first|morning|| +42073|AAAAAAAAKFEKAAAA|42073|11|41|13|AM|first|morning|| +42074|AAAAAAAALFEKAAAA|42074|11|41|14|AM|first|morning|| +42075|AAAAAAAAMFEKAAAA|42075|11|41|15|AM|first|morning|| +42076|AAAAAAAANFEKAAAA|42076|11|41|16|AM|first|morning|| +42077|AAAAAAAAOFEKAAAA|42077|11|41|17|AM|first|morning|| +42078|AAAAAAAAPFEKAAAA|42078|11|41|18|AM|first|morning|| +42079|AAAAAAAAAGEKAAAA|42079|11|41|19|AM|first|morning|| +42080|AAAAAAAABGEKAAAA|42080|11|41|20|AM|first|morning|| +42081|AAAAAAAACGEKAAAA|42081|11|41|21|AM|first|morning|| +42082|AAAAAAAADGEKAAAA|42082|11|41|22|AM|first|morning|| +42083|AAAAAAAAEGEKAAAA|42083|11|41|23|AM|first|morning|| +42084|AAAAAAAAFGEKAAAA|42084|11|41|24|AM|first|morning|| +42085|AAAAAAAAGGEKAAAA|42085|11|41|25|AM|first|morning|| +42086|AAAAAAAAHGEKAAAA|42086|11|41|26|AM|first|morning|| +42087|AAAAAAAAIGEKAAAA|42087|11|41|27|AM|first|morning|| +42088|AAAAAAAAJGEKAAAA|42088|11|41|28|AM|first|morning|| +42089|AAAAAAAAKGEKAAAA|42089|11|41|29|AM|first|morning|| +42090|AAAAAAAALGEKAAAA|42090|11|41|30|AM|first|morning|| +42091|AAAAAAAAMGEKAAAA|42091|11|41|31|AM|first|morning|| +42092|AAAAAAAANGEKAAAA|42092|11|41|32|AM|first|morning|| +42093|AAAAAAAAOGEKAAAA|42093|11|41|33|AM|first|morning|| +42094|AAAAAAAAPGEKAAAA|42094|11|41|34|AM|first|morning|| +42095|AAAAAAAAAHEKAAAA|42095|11|41|35|AM|first|morning|| +42096|AAAAAAAABHEKAAAA|42096|11|41|36|AM|first|morning|| +42097|AAAAAAAACHEKAAAA|42097|11|41|37|AM|first|morning|| +42098|AAAAAAAADHEKAAAA|42098|11|41|38|AM|first|morning|| +42099|AAAAAAAAEHEKAAAA|42099|11|41|39|AM|first|morning|| +42100|AAAAAAAAFHEKAAAA|42100|11|41|40|AM|first|morning|| +42101|AAAAAAAAGHEKAAAA|42101|11|41|41|AM|first|morning|| +42102|AAAAAAAAHHEKAAAA|42102|11|41|42|AM|first|morning|| +42103|AAAAAAAAIHEKAAAA|42103|11|41|43|AM|first|morning|| +42104|AAAAAAAAJHEKAAAA|42104|11|41|44|AM|first|morning|| +42105|AAAAAAAAKHEKAAAA|42105|11|41|45|AM|first|morning|| +42106|AAAAAAAALHEKAAAA|42106|11|41|46|AM|first|morning|| +42107|AAAAAAAAMHEKAAAA|42107|11|41|47|AM|first|morning|| +42108|AAAAAAAANHEKAAAA|42108|11|41|48|AM|first|morning|| +42109|AAAAAAAAOHEKAAAA|42109|11|41|49|AM|first|morning|| +42110|AAAAAAAAPHEKAAAA|42110|11|41|50|AM|first|morning|| +42111|AAAAAAAAAIEKAAAA|42111|11|41|51|AM|first|morning|| +42112|AAAAAAAABIEKAAAA|42112|11|41|52|AM|first|morning|| +42113|AAAAAAAACIEKAAAA|42113|11|41|53|AM|first|morning|| +42114|AAAAAAAADIEKAAAA|42114|11|41|54|AM|first|morning|| +42115|AAAAAAAAEIEKAAAA|42115|11|41|55|AM|first|morning|| +42116|AAAAAAAAFIEKAAAA|42116|11|41|56|AM|first|morning|| +42117|AAAAAAAAGIEKAAAA|42117|11|41|57|AM|first|morning|| +42118|AAAAAAAAHIEKAAAA|42118|11|41|58|AM|first|morning|| +42119|AAAAAAAAIIEKAAAA|42119|11|41|59|AM|first|morning|| +42120|AAAAAAAAJIEKAAAA|42120|11|42|0|AM|first|morning|| +42121|AAAAAAAAKIEKAAAA|42121|11|42|1|AM|first|morning|| +42122|AAAAAAAALIEKAAAA|42122|11|42|2|AM|first|morning|| +42123|AAAAAAAAMIEKAAAA|42123|11|42|3|AM|first|morning|| +42124|AAAAAAAANIEKAAAA|42124|11|42|4|AM|first|morning|| +42125|AAAAAAAAOIEKAAAA|42125|11|42|5|AM|first|morning|| +42126|AAAAAAAAPIEKAAAA|42126|11|42|6|AM|first|morning|| +42127|AAAAAAAAAJEKAAAA|42127|11|42|7|AM|first|morning|| +42128|AAAAAAAABJEKAAAA|42128|11|42|8|AM|first|morning|| +42129|AAAAAAAACJEKAAAA|42129|11|42|9|AM|first|morning|| +42130|AAAAAAAADJEKAAAA|42130|11|42|10|AM|first|morning|| +42131|AAAAAAAAEJEKAAAA|42131|11|42|11|AM|first|morning|| +42132|AAAAAAAAFJEKAAAA|42132|11|42|12|AM|first|morning|| +42133|AAAAAAAAGJEKAAAA|42133|11|42|13|AM|first|morning|| +42134|AAAAAAAAHJEKAAAA|42134|11|42|14|AM|first|morning|| +42135|AAAAAAAAIJEKAAAA|42135|11|42|15|AM|first|morning|| +42136|AAAAAAAAJJEKAAAA|42136|11|42|16|AM|first|morning|| +42137|AAAAAAAAKJEKAAAA|42137|11|42|17|AM|first|morning|| +42138|AAAAAAAALJEKAAAA|42138|11|42|18|AM|first|morning|| +42139|AAAAAAAAMJEKAAAA|42139|11|42|19|AM|first|morning|| +42140|AAAAAAAANJEKAAAA|42140|11|42|20|AM|first|morning|| +42141|AAAAAAAAOJEKAAAA|42141|11|42|21|AM|first|morning|| +42142|AAAAAAAAPJEKAAAA|42142|11|42|22|AM|first|morning|| +42143|AAAAAAAAAKEKAAAA|42143|11|42|23|AM|first|morning|| +42144|AAAAAAAABKEKAAAA|42144|11|42|24|AM|first|morning|| +42145|AAAAAAAACKEKAAAA|42145|11|42|25|AM|first|morning|| +42146|AAAAAAAADKEKAAAA|42146|11|42|26|AM|first|morning|| +42147|AAAAAAAAEKEKAAAA|42147|11|42|27|AM|first|morning|| +42148|AAAAAAAAFKEKAAAA|42148|11|42|28|AM|first|morning|| +42149|AAAAAAAAGKEKAAAA|42149|11|42|29|AM|first|morning|| +42150|AAAAAAAAHKEKAAAA|42150|11|42|30|AM|first|morning|| +42151|AAAAAAAAIKEKAAAA|42151|11|42|31|AM|first|morning|| +42152|AAAAAAAAJKEKAAAA|42152|11|42|32|AM|first|morning|| +42153|AAAAAAAAKKEKAAAA|42153|11|42|33|AM|first|morning|| +42154|AAAAAAAALKEKAAAA|42154|11|42|34|AM|first|morning|| +42155|AAAAAAAAMKEKAAAA|42155|11|42|35|AM|first|morning|| +42156|AAAAAAAANKEKAAAA|42156|11|42|36|AM|first|morning|| +42157|AAAAAAAAOKEKAAAA|42157|11|42|37|AM|first|morning|| +42158|AAAAAAAAPKEKAAAA|42158|11|42|38|AM|first|morning|| +42159|AAAAAAAAALEKAAAA|42159|11|42|39|AM|first|morning|| +42160|AAAAAAAABLEKAAAA|42160|11|42|40|AM|first|morning|| +42161|AAAAAAAACLEKAAAA|42161|11|42|41|AM|first|morning|| +42162|AAAAAAAADLEKAAAA|42162|11|42|42|AM|first|morning|| +42163|AAAAAAAAELEKAAAA|42163|11|42|43|AM|first|morning|| +42164|AAAAAAAAFLEKAAAA|42164|11|42|44|AM|first|morning|| +42165|AAAAAAAAGLEKAAAA|42165|11|42|45|AM|first|morning|| +42166|AAAAAAAAHLEKAAAA|42166|11|42|46|AM|first|morning|| +42167|AAAAAAAAILEKAAAA|42167|11|42|47|AM|first|morning|| +42168|AAAAAAAAJLEKAAAA|42168|11|42|48|AM|first|morning|| +42169|AAAAAAAAKLEKAAAA|42169|11|42|49|AM|first|morning|| +42170|AAAAAAAALLEKAAAA|42170|11|42|50|AM|first|morning|| +42171|AAAAAAAAMLEKAAAA|42171|11|42|51|AM|first|morning|| +42172|AAAAAAAANLEKAAAA|42172|11|42|52|AM|first|morning|| +42173|AAAAAAAAOLEKAAAA|42173|11|42|53|AM|first|morning|| +42174|AAAAAAAAPLEKAAAA|42174|11|42|54|AM|first|morning|| +42175|AAAAAAAAAMEKAAAA|42175|11|42|55|AM|first|morning|| +42176|AAAAAAAABMEKAAAA|42176|11|42|56|AM|first|morning|| +42177|AAAAAAAACMEKAAAA|42177|11|42|57|AM|first|morning|| +42178|AAAAAAAADMEKAAAA|42178|11|42|58|AM|first|morning|| +42179|AAAAAAAAEMEKAAAA|42179|11|42|59|AM|first|morning|| +42180|AAAAAAAAFMEKAAAA|42180|11|43|0|AM|first|morning|| +42181|AAAAAAAAGMEKAAAA|42181|11|43|1|AM|first|morning|| +42182|AAAAAAAAHMEKAAAA|42182|11|43|2|AM|first|morning|| +42183|AAAAAAAAIMEKAAAA|42183|11|43|3|AM|first|morning|| +42184|AAAAAAAAJMEKAAAA|42184|11|43|4|AM|first|morning|| +42185|AAAAAAAAKMEKAAAA|42185|11|43|5|AM|first|morning|| +42186|AAAAAAAALMEKAAAA|42186|11|43|6|AM|first|morning|| +42187|AAAAAAAAMMEKAAAA|42187|11|43|7|AM|first|morning|| +42188|AAAAAAAANMEKAAAA|42188|11|43|8|AM|first|morning|| +42189|AAAAAAAAOMEKAAAA|42189|11|43|9|AM|first|morning|| +42190|AAAAAAAAPMEKAAAA|42190|11|43|10|AM|first|morning|| +42191|AAAAAAAAANEKAAAA|42191|11|43|11|AM|first|morning|| +42192|AAAAAAAABNEKAAAA|42192|11|43|12|AM|first|morning|| +42193|AAAAAAAACNEKAAAA|42193|11|43|13|AM|first|morning|| +42194|AAAAAAAADNEKAAAA|42194|11|43|14|AM|first|morning|| +42195|AAAAAAAAENEKAAAA|42195|11|43|15|AM|first|morning|| +42196|AAAAAAAAFNEKAAAA|42196|11|43|16|AM|first|morning|| +42197|AAAAAAAAGNEKAAAA|42197|11|43|17|AM|first|morning|| +42198|AAAAAAAAHNEKAAAA|42198|11|43|18|AM|first|morning|| +42199|AAAAAAAAINEKAAAA|42199|11|43|19|AM|first|morning|| +42200|AAAAAAAAJNEKAAAA|42200|11|43|20|AM|first|morning|| +42201|AAAAAAAAKNEKAAAA|42201|11|43|21|AM|first|morning|| +42202|AAAAAAAALNEKAAAA|42202|11|43|22|AM|first|morning|| +42203|AAAAAAAAMNEKAAAA|42203|11|43|23|AM|first|morning|| +42204|AAAAAAAANNEKAAAA|42204|11|43|24|AM|first|morning|| +42205|AAAAAAAAONEKAAAA|42205|11|43|25|AM|first|morning|| +42206|AAAAAAAAPNEKAAAA|42206|11|43|26|AM|first|morning|| +42207|AAAAAAAAAOEKAAAA|42207|11|43|27|AM|first|morning|| +42208|AAAAAAAABOEKAAAA|42208|11|43|28|AM|first|morning|| +42209|AAAAAAAACOEKAAAA|42209|11|43|29|AM|first|morning|| +42210|AAAAAAAADOEKAAAA|42210|11|43|30|AM|first|morning|| +42211|AAAAAAAAEOEKAAAA|42211|11|43|31|AM|first|morning|| +42212|AAAAAAAAFOEKAAAA|42212|11|43|32|AM|first|morning|| +42213|AAAAAAAAGOEKAAAA|42213|11|43|33|AM|first|morning|| +42214|AAAAAAAAHOEKAAAA|42214|11|43|34|AM|first|morning|| +42215|AAAAAAAAIOEKAAAA|42215|11|43|35|AM|first|morning|| +42216|AAAAAAAAJOEKAAAA|42216|11|43|36|AM|first|morning|| +42217|AAAAAAAAKOEKAAAA|42217|11|43|37|AM|first|morning|| +42218|AAAAAAAALOEKAAAA|42218|11|43|38|AM|first|morning|| +42219|AAAAAAAAMOEKAAAA|42219|11|43|39|AM|first|morning|| +42220|AAAAAAAANOEKAAAA|42220|11|43|40|AM|first|morning|| +42221|AAAAAAAAOOEKAAAA|42221|11|43|41|AM|first|morning|| +42222|AAAAAAAAPOEKAAAA|42222|11|43|42|AM|first|morning|| +42223|AAAAAAAAAPEKAAAA|42223|11|43|43|AM|first|morning|| +42224|AAAAAAAABPEKAAAA|42224|11|43|44|AM|first|morning|| +42225|AAAAAAAACPEKAAAA|42225|11|43|45|AM|first|morning|| +42226|AAAAAAAADPEKAAAA|42226|11|43|46|AM|first|morning|| +42227|AAAAAAAAEPEKAAAA|42227|11|43|47|AM|first|morning|| +42228|AAAAAAAAFPEKAAAA|42228|11|43|48|AM|first|morning|| +42229|AAAAAAAAGPEKAAAA|42229|11|43|49|AM|first|morning|| +42230|AAAAAAAAHPEKAAAA|42230|11|43|50|AM|first|morning|| +42231|AAAAAAAAIPEKAAAA|42231|11|43|51|AM|first|morning|| +42232|AAAAAAAAJPEKAAAA|42232|11|43|52|AM|first|morning|| +42233|AAAAAAAAKPEKAAAA|42233|11|43|53|AM|first|morning|| +42234|AAAAAAAALPEKAAAA|42234|11|43|54|AM|first|morning|| +42235|AAAAAAAAMPEKAAAA|42235|11|43|55|AM|first|morning|| +42236|AAAAAAAANPEKAAAA|42236|11|43|56|AM|first|morning|| +42237|AAAAAAAAOPEKAAAA|42237|11|43|57|AM|first|morning|| +42238|AAAAAAAAPPEKAAAA|42238|11|43|58|AM|first|morning|| +42239|AAAAAAAAAAFKAAAA|42239|11|43|59|AM|first|morning|| +42240|AAAAAAAABAFKAAAA|42240|11|44|0|AM|first|morning|| +42241|AAAAAAAACAFKAAAA|42241|11|44|1|AM|first|morning|| +42242|AAAAAAAADAFKAAAA|42242|11|44|2|AM|first|morning|| +42243|AAAAAAAAEAFKAAAA|42243|11|44|3|AM|first|morning|| +42244|AAAAAAAAFAFKAAAA|42244|11|44|4|AM|first|morning|| +42245|AAAAAAAAGAFKAAAA|42245|11|44|5|AM|first|morning|| +42246|AAAAAAAAHAFKAAAA|42246|11|44|6|AM|first|morning|| +42247|AAAAAAAAIAFKAAAA|42247|11|44|7|AM|first|morning|| +42248|AAAAAAAAJAFKAAAA|42248|11|44|8|AM|first|morning|| +42249|AAAAAAAAKAFKAAAA|42249|11|44|9|AM|first|morning|| +42250|AAAAAAAALAFKAAAA|42250|11|44|10|AM|first|morning|| +42251|AAAAAAAAMAFKAAAA|42251|11|44|11|AM|first|morning|| +42252|AAAAAAAANAFKAAAA|42252|11|44|12|AM|first|morning|| +42253|AAAAAAAAOAFKAAAA|42253|11|44|13|AM|first|morning|| +42254|AAAAAAAAPAFKAAAA|42254|11|44|14|AM|first|morning|| +42255|AAAAAAAAABFKAAAA|42255|11|44|15|AM|first|morning|| +42256|AAAAAAAABBFKAAAA|42256|11|44|16|AM|first|morning|| +42257|AAAAAAAACBFKAAAA|42257|11|44|17|AM|first|morning|| +42258|AAAAAAAADBFKAAAA|42258|11|44|18|AM|first|morning|| +42259|AAAAAAAAEBFKAAAA|42259|11|44|19|AM|first|morning|| +42260|AAAAAAAAFBFKAAAA|42260|11|44|20|AM|first|morning|| +42261|AAAAAAAAGBFKAAAA|42261|11|44|21|AM|first|morning|| +42262|AAAAAAAAHBFKAAAA|42262|11|44|22|AM|first|morning|| +42263|AAAAAAAAIBFKAAAA|42263|11|44|23|AM|first|morning|| +42264|AAAAAAAAJBFKAAAA|42264|11|44|24|AM|first|morning|| +42265|AAAAAAAAKBFKAAAA|42265|11|44|25|AM|first|morning|| +42266|AAAAAAAALBFKAAAA|42266|11|44|26|AM|first|morning|| +42267|AAAAAAAAMBFKAAAA|42267|11|44|27|AM|first|morning|| +42268|AAAAAAAANBFKAAAA|42268|11|44|28|AM|first|morning|| +42269|AAAAAAAAOBFKAAAA|42269|11|44|29|AM|first|morning|| +42270|AAAAAAAAPBFKAAAA|42270|11|44|30|AM|first|morning|| +42271|AAAAAAAAACFKAAAA|42271|11|44|31|AM|first|morning|| +42272|AAAAAAAABCFKAAAA|42272|11|44|32|AM|first|morning|| +42273|AAAAAAAACCFKAAAA|42273|11|44|33|AM|first|morning|| +42274|AAAAAAAADCFKAAAA|42274|11|44|34|AM|first|morning|| +42275|AAAAAAAAECFKAAAA|42275|11|44|35|AM|first|morning|| +42276|AAAAAAAAFCFKAAAA|42276|11|44|36|AM|first|morning|| +42277|AAAAAAAAGCFKAAAA|42277|11|44|37|AM|first|morning|| +42278|AAAAAAAAHCFKAAAA|42278|11|44|38|AM|first|morning|| +42279|AAAAAAAAICFKAAAA|42279|11|44|39|AM|first|morning|| +42280|AAAAAAAAJCFKAAAA|42280|11|44|40|AM|first|morning|| +42281|AAAAAAAAKCFKAAAA|42281|11|44|41|AM|first|morning|| +42282|AAAAAAAALCFKAAAA|42282|11|44|42|AM|first|morning|| +42283|AAAAAAAAMCFKAAAA|42283|11|44|43|AM|first|morning|| +42284|AAAAAAAANCFKAAAA|42284|11|44|44|AM|first|morning|| +42285|AAAAAAAAOCFKAAAA|42285|11|44|45|AM|first|morning|| +42286|AAAAAAAAPCFKAAAA|42286|11|44|46|AM|first|morning|| +42287|AAAAAAAAADFKAAAA|42287|11|44|47|AM|first|morning|| +42288|AAAAAAAABDFKAAAA|42288|11|44|48|AM|first|morning|| +42289|AAAAAAAACDFKAAAA|42289|11|44|49|AM|first|morning|| +42290|AAAAAAAADDFKAAAA|42290|11|44|50|AM|first|morning|| +42291|AAAAAAAAEDFKAAAA|42291|11|44|51|AM|first|morning|| +42292|AAAAAAAAFDFKAAAA|42292|11|44|52|AM|first|morning|| +42293|AAAAAAAAGDFKAAAA|42293|11|44|53|AM|first|morning|| +42294|AAAAAAAAHDFKAAAA|42294|11|44|54|AM|first|morning|| +42295|AAAAAAAAIDFKAAAA|42295|11|44|55|AM|first|morning|| +42296|AAAAAAAAJDFKAAAA|42296|11|44|56|AM|first|morning|| +42297|AAAAAAAAKDFKAAAA|42297|11|44|57|AM|first|morning|| +42298|AAAAAAAALDFKAAAA|42298|11|44|58|AM|first|morning|| +42299|AAAAAAAAMDFKAAAA|42299|11|44|59|AM|first|morning|| +42300|AAAAAAAANDFKAAAA|42300|11|45|0|AM|first|morning|| +42301|AAAAAAAAODFKAAAA|42301|11|45|1|AM|first|morning|| +42302|AAAAAAAAPDFKAAAA|42302|11|45|2|AM|first|morning|| +42303|AAAAAAAAAEFKAAAA|42303|11|45|3|AM|first|morning|| +42304|AAAAAAAABEFKAAAA|42304|11|45|4|AM|first|morning|| +42305|AAAAAAAACEFKAAAA|42305|11|45|5|AM|first|morning|| +42306|AAAAAAAADEFKAAAA|42306|11|45|6|AM|first|morning|| +42307|AAAAAAAAEEFKAAAA|42307|11|45|7|AM|first|morning|| +42308|AAAAAAAAFEFKAAAA|42308|11|45|8|AM|first|morning|| +42309|AAAAAAAAGEFKAAAA|42309|11|45|9|AM|first|morning|| +42310|AAAAAAAAHEFKAAAA|42310|11|45|10|AM|first|morning|| +42311|AAAAAAAAIEFKAAAA|42311|11|45|11|AM|first|morning|| +42312|AAAAAAAAJEFKAAAA|42312|11|45|12|AM|first|morning|| +42313|AAAAAAAAKEFKAAAA|42313|11|45|13|AM|first|morning|| +42314|AAAAAAAALEFKAAAA|42314|11|45|14|AM|first|morning|| +42315|AAAAAAAAMEFKAAAA|42315|11|45|15|AM|first|morning|| +42316|AAAAAAAANEFKAAAA|42316|11|45|16|AM|first|morning|| +42317|AAAAAAAAOEFKAAAA|42317|11|45|17|AM|first|morning|| +42318|AAAAAAAAPEFKAAAA|42318|11|45|18|AM|first|morning|| +42319|AAAAAAAAAFFKAAAA|42319|11|45|19|AM|first|morning|| +42320|AAAAAAAABFFKAAAA|42320|11|45|20|AM|first|morning|| +42321|AAAAAAAACFFKAAAA|42321|11|45|21|AM|first|morning|| +42322|AAAAAAAADFFKAAAA|42322|11|45|22|AM|first|morning|| +42323|AAAAAAAAEFFKAAAA|42323|11|45|23|AM|first|morning|| +42324|AAAAAAAAFFFKAAAA|42324|11|45|24|AM|first|morning|| +42325|AAAAAAAAGFFKAAAA|42325|11|45|25|AM|first|morning|| +42326|AAAAAAAAHFFKAAAA|42326|11|45|26|AM|first|morning|| +42327|AAAAAAAAIFFKAAAA|42327|11|45|27|AM|first|morning|| +42328|AAAAAAAAJFFKAAAA|42328|11|45|28|AM|first|morning|| +42329|AAAAAAAAKFFKAAAA|42329|11|45|29|AM|first|morning|| +42330|AAAAAAAALFFKAAAA|42330|11|45|30|AM|first|morning|| +42331|AAAAAAAAMFFKAAAA|42331|11|45|31|AM|first|morning|| +42332|AAAAAAAANFFKAAAA|42332|11|45|32|AM|first|morning|| +42333|AAAAAAAAOFFKAAAA|42333|11|45|33|AM|first|morning|| +42334|AAAAAAAAPFFKAAAA|42334|11|45|34|AM|first|morning|| +42335|AAAAAAAAAGFKAAAA|42335|11|45|35|AM|first|morning|| +42336|AAAAAAAABGFKAAAA|42336|11|45|36|AM|first|morning|| +42337|AAAAAAAACGFKAAAA|42337|11|45|37|AM|first|morning|| +42338|AAAAAAAADGFKAAAA|42338|11|45|38|AM|first|morning|| +42339|AAAAAAAAEGFKAAAA|42339|11|45|39|AM|first|morning|| +42340|AAAAAAAAFGFKAAAA|42340|11|45|40|AM|first|morning|| +42341|AAAAAAAAGGFKAAAA|42341|11|45|41|AM|first|morning|| +42342|AAAAAAAAHGFKAAAA|42342|11|45|42|AM|first|morning|| +42343|AAAAAAAAIGFKAAAA|42343|11|45|43|AM|first|morning|| +42344|AAAAAAAAJGFKAAAA|42344|11|45|44|AM|first|morning|| +42345|AAAAAAAAKGFKAAAA|42345|11|45|45|AM|first|morning|| +42346|AAAAAAAALGFKAAAA|42346|11|45|46|AM|first|morning|| +42347|AAAAAAAAMGFKAAAA|42347|11|45|47|AM|first|morning|| +42348|AAAAAAAANGFKAAAA|42348|11|45|48|AM|first|morning|| +42349|AAAAAAAAOGFKAAAA|42349|11|45|49|AM|first|morning|| +42350|AAAAAAAAPGFKAAAA|42350|11|45|50|AM|first|morning|| +42351|AAAAAAAAAHFKAAAA|42351|11|45|51|AM|first|morning|| +42352|AAAAAAAABHFKAAAA|42352|11|45|52|AM|first|morning|| +42353|AAAAAAAACHFKAAAA|42353|11|45|53|AM|first|morning|| +42354|AAAAAAAADHFKAAAA|42354|11|45|54|AM|first|morning|| +42355|AAAAAAAAEHFKAAAA|42355|11|45|55|AM|first|morning|| +42356|AAAAAAAAFHFKAAAA|42356|11|45|56|AM|first|morning|| +42357|AAAAAAAAGHFKAAAA|42357|11|45|57|AM|first|morning|| +42358|AAAAAAAAHHFKAAAA|42358|11|45|58|AM|first|morning|| +42359|AAAAAAAAIHFKAAAA|42359|11|45|59|AM|first|morning|| +42360|AAAAAAAAJHFKAAAA|42360|11|46|0|AM|first|morning|| +42361|AAAAAAAAKHFKAAAA|42361|11|46|1|AM|first|morning|| +42362|AAAAAAAALHFKAAAA|42362|11|46|2|AM|first|morning|| +42363|AAAAAAAAMHFKAAAA|42363|11|46|3|AM|first|morning|| +42364|AAAAAAAANHFKAAAA|42364|11|46|4|AM|first|morning|| +42365|AAAAAAAAOHFKAAAA|42365|11|46|5|AM|first|morning|| +42366|AAAAAAAAPHFKAAAA|42366|11|46|6|AM|first|morning|| +42367|AAAAAAAAAIFKAAAA|42367|11|46|7|AM|first|morning|| +42368|AAAAAAAABIFKAAAA|42368|11|46|8|AM|first|morning|| +42369|AAAAAAAACIFKAAAA|42369|11|46|9|AM|first|morning|| +42370|AAAAAAAADIFKAAAA|42370|11|46|10|AM|first|morning|| +42371|AAAAAAAAEIFKAAAA|42371|11|46|11|AM|first|morning|| +42372|AAAAAAAAFIFKAAAA|42372|11|46|12|AM|first|morning|| +42373|AAAAAAAAGIFKAAAA|42373|11|46|13|AM|first|morning|| +42374|AAAAAAAAHIFKAAAA|42374|11|46|14|AM|first|morning|| +42375|AAAAAAAAIIFKAAAA|42375|11|46|15|AM|first|morning|| +42376|AAAAAAAAJIFKAAAA|42376|11|46|16|AM|first|morning|| +42377|AAAAAAAAKIFKAAAA|42377|11|46|17|AM|first|morning|| +42378|AAAAAAAALIFKAAAA|42378|11|46|18|AM|first|morning|| +42379|AAAAAAAAMIFKAAAA|42379|11|46|19|AM|first|morning|| +42380|AAAAAAAANIFKAAAA|42380|11|46|20|AM|first|morning|| +42381|AAAAAAAAOIFKAAAA|42381|11|46|21|AM|first|morning|| +42382|AAAAAAAAPIFKAAAA|42382|11|46|22|AM|first|morning|| +42383|AAAAAAAAAJFKAAAA|42383|11|46|23|AM|first|morning|| +42384|AAAAAAAABJFKAAAA|42384|11|46|24|AM|first|morning|| +42385|AAAAAAAACJFKAAAA|42385|11|46|25|AM|first|morning|| +42386|AAAAAAAADJFKAAAA|42386|11|46|26|AM|first|morning|| +42387|AAAAAAAAEJFKAAAA|42387|11|46|27|AM|first|morning|| +42388|AAAAAAAAFJFKAAAA|42388|11|46|28|AM|first|morning|| +42389|AAAAAAAAGJFKAAAA|42389|11|46|29|AM|first|morning|| +42390|AAAAAAAAHJFKAAAA|42390|11|46|30|AM|first|morning|| +42391|AAAAAAAAIJFKAAAA|42391|11|46|31|AM|first|morning|| +42392|AAAAAAAAJJFKAAAA|42392|11|46|32|AM|first|morning|| +42393|AAAAAAAAKJFKAAAA|42393|11|46|33|AM|first|morning|| +42394|AAAAAAAALJFKAAAA|42394|11|46|34|AM|first|morning|| +42395|AAAAAAAAMJFKAAAA|42395|11|46|35|AM|first|morning|| +42396|AAAAAAAANJFKAAAA|42396|11|46|36|AM|first|morning|| +42397|AAAAAAAAOJFKAAAA|42397|11|46|37|AM|first|morning|| +42398|AAAAAAAAPJFKAAAA|42398|11|46|38|AM|first|morning|| +42399|AAAAAAAAAKFKAAAA|42399|11|46|39|AM|first|morning|| +42400|AAAAAAAABKFKAAAA|42400|11|46|40|AM|first|morning|| +42401|AAAAAAAACKFKAAAA|42401|11|46|41|AM|first|morning|| +42402|AAAAAAAADKFKAAAA|42402|11|46|42|AM|first|morning|| +42403|AAAAAAAAEKFKAAAA|42403|11|46|43|AM|first|morning|| +42404|AAAAAAAAFKFKAAAA|42404|11|46|44|AM|first|morning|| +42405|AAAAAAAAGKFKAAAA|42405|11|46|45|AM|first|morning|| +42406|AAAAAAAAHKFKAAAA|42406|11|46|46|AM|first|morning|| +42407|AAAAAAAAIKFKAAAA|42407|11|46|47|AM|first|morning|| +42408|AAAAAAAAJKFKAAAA|42408|11|46|48|AM|first|morning|| +42409|AAAAAAAAKKFKAAAA|42409|11|46|49|AM|first|morning|| +42410|AAAAAAAALKFKAAAA|42410|11|46|50|AM|first|morning|| +42411|AAAAAAAAMKFKAAAA|42411|11|46|51|AM|first|morning|| +42412|AAAAAAAANKFKAAAA|42412|11|46|52|AM|first|morning|| +42413|AAAAAAAAOKFKAAAA|42413|11|46|53|AM|first|morning|| +42414|AAAAAAAAPKFKAAAA|42414|11|46|54|AM|first|morning|| +42415|AAAAAAAAALFKAAAA|42415|11|46|55|AM|first|morning|| +42416|AAAAAAAABLFKAAAA|42416|11|46|56|AM|first|morning|| +42417|AAAAAAAACLFKAAAA|42417|11|46|57|AM|first|morning|| +42418|AAAAAAAADLFKAAAA|42418|11|46|58|AM|first|morning|| +42419|AAAAAAAAELFKAAAA|42419|11|46|59|AM|first|morning|| +42420|AAAAAAAAFLFKAAAA|42420|11|47|0|AM|first|morning|| +42421|AAAAAAAAGLFKAAAA|42421|11|47|1|AM|first|morning|| +42422|AAAAAAAAHLFKAAAA|42422|11|47|2|AM|first|morning|| +42423|AAAAAAAAILFKAAAA|42423|11|47|3|AM|first|morning|| +42424|AAAAAAAAJLFKAAAA|42424|11|47|4|AM|first|morning|| +42425|AAAAAAAAKLFKAAAA|42425|11|47|5|AM|first|morning|| +42426|AAAAAAAALLFKAAAA|42426|11|47|6|AM|first|morning|| +42427|AAAAAAAAMLFKAAAA|42427|11|47|7|AM|first|morning|| +42428|AAAAAAAANLFKAAAA|42428|11|47|8|AM|first|morning|| +42429|AAAAAAAAOLFKAAAA|42429|11|47|9|AM|first|morning|| +42430|AAAAAAAAPLFKAAAA|42430|11|47|10|AM|first|morning|| +42431|AAAAAAAAAMFKAAAA|42431|11|47|11|AM|first|morning|| +42432|AAAAAAAABMFKAAAA|42432|11|47|12|AM|first|morning|| +42433|AAAAAAAACMFKAAAA|42433|11|47|13|AM|first|morning|| +42434|AAAAAAAADMFKAAAA|42434|11|47|14|AM|first|morning|| +42435|AAAAAAAAEMFKAAAA|42435|11|47|15|AM|first|morning|| +42436|AAAAAAAAFMFKAAAA|42436|11|47|16|AM|first|morning|| +42437|AAAAAAAAGMFKAAAA|42437|11|47|17|AM|first|morning|| +42438|AAAAAAAAHMFKAAAA|42438|11|47|18|AM|first|morning|| +42439|AAAAAAAAIMFKAAAA|42439|11|47|19|AM|first|morning|| +42440|AAAAAAAAJMFKAAAA|42440|11|47|20|AM|first|morning|| +42441|AAAAAAAAKMFKAAAA|42441|11|47|21|AM|first|morning|| +42442|AAAAAAAALMFKAAAA|42442|11|47|22|AM|first|morning|| +42443|AAAAAAAAMMFKAAAA|42443|11|47|23|AM|first|morning|| +42444|AAAAAAAANMFKAAAA|42444|11|47|24|AM|first|morning|| +42445|AAAAAAAAOMFKAAAA|42445|11|47|25|AM|first|morning|| +42446|AAAAAAAAPMFKAAAA|42446|11|47|26|AM|first|morning|| +42447|AAAAAAAAANFKAAAA|42447|11|47|27|AM|first|morning|| +42448|AAAAAAAABNFKAAAA|42448|11|47|28|AM|first|morning|| +42449|AAAAAAAACNFKAAAA|42449|11|47|29|AM|first|morning|| +42450|AAAAAAAADNFKAAAA|42450|11|47|30|AM|first|morning|| +42451|AAAAAAAAENFKAAAA|42451|11|47|31|AM|first|morning|| +42452|AAAAAAAAFNFKAAAA|42452|11|47|32|AM|first|morning|| +42453|AAAAAAAAGNFKAAAA|42453|11|47|33|AM|first|morning|| +42454|AAAAAAAAHNFKAAAA|42454|11|47|34|AM|first|morning|| +42455|AAAAAAAAINFKAAAA|42455|11|47|35|AM|first|morning|| +42456|AAAAAAAAJNFKAAAA|42456|11|47|36|AM|first|morning|| +42457|AAAAAAAAKNFKAAAA|42457|11|47|37|AM|first|morning|| +42458|AAAAAAAALNFKAAAA|42458|11|47|38|AM|first|morning|| +42459|AAAAAAAAMNFKAAAA|42459|11|47|39|AM|first|morning|| +42460|AAAAAAAANNFKAAAA|42460|11|47|40|AM|first|morning|| +42461|AAAAAAAAONFKAAAA|42461|11|47|41|AM|first|morning|| +42462|AAAAAAAAPNFKAAAA|42462|11|47|42|AM|first|morning|| +42463|AAAAAAAAAOFKAAAA|42463|11|47|43|AM|first|morning|| +42464|AAAAAAAABOFKAAAA|42464|11|47|44|AM|first|morning|| +42465|AAAAAAAACOFKAAAA|42465|11|47|45|AM|first|morning|| +42466|AAAAAAAADOFKAAAA|42466|11|47|46|AM|first|morning|| +42467|AAAAAAAAEOFKAAAA|42467|11|47|47|AM|first|morning|| +42468|AAAAAAAAFOFKAAAA|42468|11|47|48|AM|first|morning|| +42469|AAAAAAAAGOFKAAAA|42469|11|47|49|AM|first|morning|| +42470|AAAAAAAAHOFKAAAA|42470|11|47|50|AM|first|morning|| +42471|AAAAAAAAIOFKAAAA|42471|11|47|51|AM|first|morning|| +42472|AAAAAAAAJOFKAAAA|42472|11|47|52|AM|first|morning|| +42473|AAAAAAAAKOFKAAAA|42473|11|47|53|AM|first|morning|| +42474|AAAAAAAALOFKAAAA|42474|11|47|54|AM|first|morning|| +42475|AAAAAAAAMOFKAAAA|42475|11|47|55|AM|first|morning|| +42476|AAAAAAAANOFKAAAA|42476|11|47|56|AM|first|morning|| +42477|AAAAAAAAOOFKAAAA|42477|11|47|57|AM|first|morning|| +42478|AAAAAAAAPOFKAAAA|42478|11|47|58|AM|first|morning|| +42479|AAAAAAAAAPFKAAAA|42479|11|47|59|AM|first|morning|| +42480|AAAAAAAABPFKAAAA|42480|11|48|0|AM|first|morning|| +42481|AAAAAAAACPFKAAAA|42481|11|48|1|AM|first|morning|| +42482|AAAAAAAADPFKAAAA|42482|11|48|2|AM|first|morning|| +42483|AAAAAAAAEPFKAAAA|42483|11|48|3|AM|first|morning|| +42484|AAAAAAAAFPFKAAAA|42484|11|48|4|AM|first|morning|| +42485|AAAAAAAAGPFKAAAA|42485|11|48|5|AM|first|morning|| +42486|AAAAAAAAHPFKAAAA|42486|11|48|6|AM|first|morning|| +42487|AAAAAAAAIPFKAAAA|42487|11|48|7|AM|first|morning|| +42488|AAAAAAAAJPFKAAAA|42488|11|48|8|AM|first|morning|| +42489|AAAAAAAAKPFKAAAA|42489|11|48|9|AM|first|morning|| +42490|AAAAAAAALPFKAAAA|42490|11|48|10|AM|first|morning|| +42491|AAAAAAAAMPFKAAAA|42491|11|48|11|AM|first|morning|| +42492|AAAAAAAANPFKAAAA|42492|11|48|12|AM|first|morning|| +42493|AAAAAAAAOPFKAAAA|42493|11|48|13|AM|first|morning|| +42494|AAAAAAAAPPFKAAAA|42494|11|48|14|AM|first|morning|| +42495|AAAAAAAAAAGKAAAA|42495|11|48|15|AM|first|morning|| +42496|AAAAAAAABAGKAAAA|42496|11|48|16|AM|first|morning|| +42497|AAAAAAAACAGKAAAA|42497|11|48|17|AM|first|morning|| +42498|AAAAAAAADAGKAAAA|42498|11|48|18|AM|first|morning|| +42499|AAAAAAAAEAGKAAAA|42499|11|48|19|AM|first|morning|| +42500|AAAAAAAAFAGKAAAA|42500|11|48|20|AM|first|morning|| +42501|AAAAAAAAGAGKAAAA|42501|11|48|21|AM|first|morning|| +42502|AAAAAAAAHAGKAAAA|42502|11|48|22|AM|first|morning|| +42503|AAAAAAAAIAGKAAAA|42503|11|48|23|AM|first|morning|| +42504|AAAAAAAAJAGKAAAA|42504|11|48|24|AM|first|morning|| +42505|AAAAAAAAKAGKAAAA|42505|11|48|25|AM|first|morning|| +42506|AAAAAAAALAGKAAAA|42506|11|48|26|AM|first|morning|| +42507|AAAAAAAAMAGKAAAA|42507|11|48|27|AM|first|morning|| +42508|AAAAAAAANAGKAAAA|42508|11|48|28|AM|first|morning|| +42509|AAAAAAAAOAGKAAAA|42509|11|48|29|AM|first|morning|| +42510|AAAAAAAAPAGKAAAA|42510|11|48|30|AM|first|morning|| +42511|AAAAAAAAABGKAAAA|42511|11|48|31|AM|first|morning|| +42512|AAAAAAAABBGKAAAA|42512|11|48|32|AM|first|morning|| +42513|AAAAAAAACBGKAAAA|42513|11|48|33|AM|first|morning|| +42514|AAAAAAAADBGKAAAA|42514|11|48|34|AM|first|morning|| +42515|AAAAAAAAEBGKAAAA|42515|11|48|35|AM|first|morning|| +42516|AAAAAAAAFBGKAAAA|42516|11|48|36|AM|first|morning|| +42517|AAAAAAAAGBGKAAAA|42517|11|48|37|AM|first|morning|| +42518|AAAAAAAAHBGKAAAA|42518|11|48|38|AM|first|morning|| +42519|AAAAAAAAIBGKAAAA|42519|11|48|39|AM|first|morning|| +42520|AAAAAAAAJBGKAAAA|42520|11|48|40|AM|first|morning|| +42521|AAAAAAAAKBGKAAAA|42521|11|48|41|AM|first|morning|| +42522|AAAAAAAALBGKAAAA|42522|11|48|42|AM|first|morning|| +42523|AAAAAAAAMBGKAAAA|42523|11|48|43|AM|first|morning|| +42524|AAAAAAAANBGKAAAA|42524|11|48|44|AM|first|morning|| +42525|AAAAAAAAOBGKAAAA|42525|11|48|45|AM|first|morning|| +42526|AAAAAAAAPBGKAAAA|42526|11|48|46|AM|first|morning|| +42527|AAAAAAAAACGKAAAA|42527|11|48|47|AM|first|morning|| +42528|AAAAAAAABCGKAAAA|42528|11|48|48|AM|first|morning|| +42529|AAAAAAAACCGKAAAA|42529|11|48|49|AM|first|morning|| +42530|AAAAAAAADCGKAAAA|42530|11|48|50|AM|first|morning|| +42531|AAAAAAAAECGKAAAA|42531|11|48|51|AM|first|morning|| +42532|AAAAAAAAFCGKAAAA|42532|11|48|52|AM|first|morning|| +42533|AAAAAAAAGCGKAAAA|42533|11|48|53|AM|first|morning|| +42534|AAAAAAAAHCGKAAAA|42534|11|48|54|AM|first|morning|| +42535|AAAAAAAAICGKAAAA|42535|11|48|55|AM|first|morning|| +42536|AAAAAAAAJCGKAAAA|42536|11|48|56|AM|first|morning|| +42537|AAAAAAAAKCGKAAAA|42537|11|48|57|AM|first|morning|| +42538|AAAAAAAALCGKAAAA|42538|11|48|58|AM|first|morning|| +42539|AAAAAAAAMCGKAAAA|42539|11|48|59|AM|first|morning|| +42540|AAAAAAAANCGKAAAA|42540|11|49|0|AM|first|morning|| +42541|AAAAAAAAOCGKAAAA|42541|11|49|1|AM|first|morning|| +42542|AAAAAAAAPCGKAAAA|42542|11|49|2|AM|first|morning|| +42543|AAAAAAAAADGKAAAA|42543|11|49|3|AM|first|morning|| +42544|AAAAAAAABDGKAAAA|42544|11|49|4|AM|first|morning|| +42545|AAAAAAAACDGKAAAA|42545|11|49|5|AM|first|morning|| +42546|AAAAAAAADDGKAAAA|42546|11|49|6|AM|first|morning|| +42547|AAAAAAAAEDGKAAAA|42547|11|49|7|AM|first|morning|| +42548|AAAAAAAAFDGKAAAA|42548|11|49|8|AM|first|morning|| +42549|AAAAAAAAGDGKAAAA|42549|11|49|9|AM|first|morning|| +42550|AAAAAAAAHDGKAAAA|42550|11|49|10|AM|first|morning|| +42551|AAAAAAAAIDGKAAAA|42551|11|49|11|AM|first|morning|| +42552|AAAAAAAAJDGKAAAA|42552|11|49|12|AM|first|morning|| +42553|AAAAAAAAKDGKAAAA|42553|11|49|13|AM|first|morning|| +42554|AAAAAAAALDGKAAAA|42554|11|49|14|AM|first|morning|| +42555|AAAAAAAAMDGKAAAA|42555|11|49|15|AM|first|morning|| +42556|AAAAAAAANDGKAAAA|42556|11|49|16|AM|first|morning|| +42557|AAAAAAAAODGKAAAA|42557|11|49|17|AM|first|morning|| +42558|AAAAAAAAPDGKAAAA|42558|11|49|18|AM|first|morning|| +42559|AAAAAAAAAEGKAAAA|42559|11|49|19|AM|first|morning|| +42560|AAAAAAAABEGKAAAA|42560|11|49|20|AM|first|morning|| +42561|AAAAAAAACEGKAAAA|42561|11|49|21|AM|first|morning|| +42562|AAAAAAAADEGKAAAA|42562|11|49|22|AM|first|morning|| +42563|AAAAAAAAEEGKAAAA|42563|11|49|23|AM|first|morning|| +42564|AAAAAAAAFEGKAAAA|42564|11|49|24|AM|first|morning|| +42565|AAAAAAAAGEGKAAAA|42565|11|49|25|AM|first|morning|| +42566|AAAAAAAAHEGKAAAA|42566|11|49|26|AM|first|morning|| +42567|AAAAAAAAIEGKAAAA|42567|11|49|27|AM|first|morning|| +42568|AAAAAAAAJEGKAAAA|42568|11|49|28|AM|first|morning|| +42569|AAAAAAAAKEGKAAAA|42569|11|49|29|AM|first|morning|| +42570|AAAAAAAALEGKAAAA|42570|11|49|30|AM|first|morning|| +42571|AAAAAAAAMEGKAAAA|42571|11|49|31|AM|first|morning|| +42572|AAAAAAAANEGKAAAA|42572|11|49|32|AM|first|morning|| +42573|AAAAAAAAOEGKAAAA|42573|11|49|33|AM|first|morning|| +42574|AAAAAAAAPEGKAAAA|42574|11|49|34|AM|first|morning|| +42575|AAAAAAAAAFGKAAAA|42575|11|49|35|AM|first|morning|| +42576|AAAAAAAABFGKAAAA|42576|11|49|36|AM|first|morning|| +42577|AAAAAAAACFGKAAAA|42577|11|49|37|AM|first|morning|| +42578|AAAAAAAADFGKAAAA|42578|11|49|38|AM|first|morning|| +42579|AAAAAAAAEFGKAAAA|42579|11|49|39|AM|first|morning|| +42580|AAAAAAAAFFGKAAAA|42580|11|49|40|AM|first|morning|| +42581|AAAAAAAAGFGKAAAA|42581|11|49|41|AM|first|morning|| +42582|AAAAAAAAHFGKAAAA|42582|11|49|42|AM|first|morning|| +42583|AAAAAAAAIFGKAAAA|42583|11|49|43|AM|first|morning|| +42584|AAAAAAAAJFGKAAAA|42584|11|49|44|AM|first|morning|| +42585|AAAAAAAAKFGKAAAA|42585|11|49|45|AM|first|morning|| +42586|AAAAAAAALFGKAAAA|42586|11|49|46|AM|first|morning|| +42587|AAAAAAAAMFGKAAAA|42587|11|49|47|AM|first|morning|| +42588|AAAAAAAANFGKAAAA|42588|11|49|48|AM|first|morning|| +42589|AAAAAAAAOFGKAAAA|42589|11|49|49|AM|first|morning|| +42590|AAAAAAAAPFGKAAAA|42590|11|49|50|AM|first|morning|| +42591|AAAAAAAAAGGKAAAA|42591|11|49|51|AM|first|morning|| +42592|AAAAAAAABGGKAAAA|42592|11|49|52|AM|first|morning|| +42593|AAAAAAAACGGKAAAA|42593|11|49|53|AM|first|morning|| +42594|AAAAAAAADGGKAAAA|42594|11|49|54|AM|first|morning|| +42595|AAAAAAAAEGGKAAAA|42595|11|49|55|AM|first|morning|| +42596|AAAAAAAAFGGKAAAA|42596|11|49|56|AM|first|morning|| +42597|AAAAAAAAGGGKAAAA|42597|11|49|57|AM|first|morning|| +42598|AAAAAAAAHGGKAAAA|42598|11|49|58|AM|first|morning|| +42599|AAAAAAAAIGGKAAAA|42599|11|49|59|AM|first|morning|| +42600|AAAAAAAAJGGKAAAA|42600|11|50|0|AM|first|morning|| +42601|AAAAAAAAKGGKAAAA|42601|11|50|1|AM|first|morning|| +42602|AAAAAAAALGGKAAAA|42602|11|50|2|AM|first|morning|| +42603|AAAAAAAAMGGKAAAA|42603|11|50|3|AM|first|morning|| +42604|AAAAAAAANGGKAAAA|42604|11|50|4|AM|first|morning|| +42605|AAAAAAAAOGGKAAAA|42605|11|50|5|AM|first|morning|| +42606|AAAAAAAAPGGKAAAA|42606|11|50|6|AM|first|morning|| +42607|AAAAAAAAAHGKAAAA|42607|11|50|7|AM|first|morning|| +42608|AAAAAAAABHGKAAAA|42608|11|50|8|AM|first|morning|| +42609|AAAAAAAACHGKAAAA|42609|11|50|9|AM|first|morning|| +42610|AAAAAAAADHGKAAAA|42610|11|50|10|AM|first|morning|| +42611|AAAAAAAAEHGKAAAA|42611|11|50|11|AM|first|morning|| +42612|AAAAAAAAFHGKAAAA|42612|11|50|12|AM|first|morning|| +42613|AAAAAAAAGHGKAAAA|42613|11|50|13|AM|first|morning|| +42614|AAAAAAAAHHGKAAAA|42614|11|50|14|AM|first|morning|| +42615|AAAAAAAAIHGKAAAA|42615|11|50|15|AM|first|morning|| +42616|AAAAAAAAJHGKAAAA|42616|11|50|16|AM|first|morning|| +42617|AAAAAAAAKHGKAAAA|42617|11|50|17|AM|first|morning|| +42618|AAAAAAAALHGKAAAA|42618|11|50|18|AM|first|morning|| +42619|AAAAAAAAMHGKAAAA|42619|11|50|19|AM|first|morning|| +42620|AAAAAAAANHGKAAAA|42620|11|50|20|AM|first|morning|| +42621|AAAAAAAAOHGKAAAA|42621|11|50|21|AM|first|morning|| +42622|AAAAAAAAPHGKAAAA|42622|11|50|22|AM|first|morning|| +42623|AAAAAAAAAIGKAAAA|42623|11|50|23|AM|first|morning|| +42624|AAAAAAAABIGKAAAA|42624|11|50|24|AM|first|morning|| +42625|AAAAAAAACIGKAAAA|42625|11|50|25|AM|first|morning|| +42626|AAAAAAAADIGKAAAA|42626|11|50|26|AM|first|morning|| +42627|AAAAAAAAEIGKAAAA|42627|11|50|27|AM|first|morning|| +42628|AAAAAAAAFIGKAAAA|42628|11|50|28|AM|first|morning|| +42629|AAAAAAAAGIGKAAAA|42629|11|50|29|AM|first|morning|| +42630|AAAAAAAAHIGKAAAA|42630|11|50|30|AM|first|morning|| +42631|AAAAAAAAIIGKAAAA|42631|11|50|31|AM|first|morning|| +42632|AAAAAAAAJIGKAAAA|42632|11|50|32|AM|first|morning|| +42633|AAAAAAAAKIGKAAAA|42633|11|50|33|AM|first|morning|| +42634|AAAAAAAALIGKAAAA|42634|11|50|34|AM|first|morning|| +42635|AAAAAAAAMIGKAAAA|42635|11|50|35|AM|first|morning|| +42636|AAAAAAAANIGKAAAA|42636|11|50|36|AM|first|morning|| +42637|AAAAAAAAOIGKAAAA|42637|11|50|37|AM|first|morning|| +42638|AAAAAAAAPIGKAAAA|42638|11|50|38|AM|first|morning|| +42639|AAAAAAAAAJGKAAAA|42639|11|50|39|AM|first|morning|| +42640|AAAAAAAABJGKAAAA|42640|11|50|40|AM|first|morning|| +42641|AAAAAAAACJGKAAAA|42641|11|50|41|AM|first|morning|| +42642|AAAAAAAADJGKAAAA|42642|11|50|42|AM|first|morning|| +42643|AAAAAAAAEJGKAAAA|42643|11|50|43|AM|first|morning|| +42644|AAAAAAAAFJGKAAAA|42644|11|50|44|AM|first|morning|| +42645|AAAAAAAAGJGKAAAA|42645|11|50|45|AM|first|morning|| +42646|AAAAAAAAHJGKAAAA|42646|11|50|46|AM|first|morning|| +42647|AAAAAAAAIJGKAAAA|42647|11|50|47|AM|first|morning|| +42648|AAAAAAAAJJGKAAAA|42648|11|50|48|AM|first|morning|| +42649|AAAAAAAAKJGKAAAA|42649|11|50|49|AM|first|morning|| +42650|AAAAAAAALJGKAAAA|42650|11|50|50|AM|first|morning|| +42651|AAAAAAAAMJGKAAAA|42651|11|50|51|AM|first|morning|| +42652|AAAAAAAANJGKAAAA|42652|11|50|52|AM|first|morning|| +42653|AAAAAAAAOJGKAAAA|42653|11|50|53|AM|first|morning|| +42654|AAAAAAAAPJGKAAAA|42654|11|50|54|AM|first|morning|| +42655|AAAAAAAAAKGKAAAA|42655|11|50|55|AM|first|morning|| +42656|AAAAAAAABKGKAAAA|42656|11|50|56|AM|first|morning|| +42657|AAAAAAAACKGKAAAA|42657|11|50|57|AM|first|morning|| +42658|AAAAAAAADKGKAAAA|42658|11|50|58|AM|first|morning|| +42659|AAAAAAAAEKGKAAAA|42659|11|50|59|AM|first|morning|| +42660|AAAAAAAAFKGKAAAA|42660|11|51|0|AM|first|morning|| +42661|AAAAAAAAGKGKAAAA|42661|11|51|1|AM|first|morning|| +42662|AAAAAAAAHKGKAAAA|42662|11|51|2|AM|first|morning|| +42663|AAAAAAAAIKGKAAAA|42663|11|51|3|AM|first|morning|| +42664|AAAAAAAAJKGKAAAA|42664|11|51|4|AM|first|morning|| +42665|AAAAAAAAKKGKAAAA|42665|11|51|5|AM|first|morning|| +42666|AAAAAAAALKGKAAAA|42666|11|51|6|AM|first|morning|| +42667|AAAAAAAAMKGKAAAA|42667|11|51|7|AM|first|morning|| +42668|AAAAAAAANKGKAAAA|42668|11|51|8|AM|first|morning|| +42669|AAAAAAAAOKGKAAAA|42669|11|51|9|AM|first|morning|| +42670|AAAAAAAAPKGKAAAA|42670|11|51|10|AM|first|morning|| +42671|AAAAAAAAALGKAAAA|42671|11|51|11|AM|first|morning|| +42672|AAAAAAAABLGKAAAA|42672|11|51|12|AM|first|morning|| +42673|AAAAAAAACLGKAAAA|42673|11|51|13|AM|first|morning|| +42674|AAAAAAAADLGKAAAA|42674|11|51|14|AM|first|morning|| +42675|AAAAAAAAELGKAAAA|42675|11|51|15|AM|first|morning|| +42676|AAAAAAAAFLGKAAAA|42676|11|51|16|AM|first|morning|| +42677|AAAAAAAAGLGKAAAA|42677|11|51|17|AM|first|morning|| +42678|AAAAAAAAHLGKAAAA|42678|11|51|18|AM|first|morning|| +42679|AAAAAAAAILGKAAAA|42679|11|51|19|AM|first|morning|| +42680|AAAAAAAAJLGKAAAA|42680|11|51|20|AM|first|morning|| +42681|AAAAAAAAKLGKAAAA|42681|11|51|21|AM|first|morning|| +42682|AAAAAAAALLGKAAAA|42682|11|51|22|AM|first|morning|| +42683|AAAAAAAAMLGKAAAA|42683|11|51|23|AM|first|morning|| +42684|AAAAAAAANLGKAAAA|42684|11|51|24|AM|first|morning|| +42685|AAAAAAAAOLGKAAAA|42685|11|51|25|AM|first|morning|| +42686|AAAAAAAAPLGKAAAA|42686|11|51|26|AM|first|morning|| +42687|AAAAAAAAAMGKAAAA|42687|11|51|27|AM|first|morning|| +42688|AAAAAAAABMGKAAAA|42688|11|51|28|AM|first|morning|| +42689|AAAAAAAACMGKAAAA|42689|11|51|29|AM|first|morning|| +42690|AAAAAAAADMGKAAAA|42690|11|51|30|AM|first|morning|| +42691|AAAAAAAAEMGKAAAA|42691|11|51|31|AM|first|morning|| +42692|AAAAAAAAFMGKAAAA|42692|11|51|32|AM|first|morning|| +42693|AAAAAAAAGMGKAAAA|42693|11|51|33|AM|first|morning|| +42694|AAAAAAAAHMGKAAAA|42694|11|51|34|AM|first|morning|| +42695|AAAAAAAAIMGKAAAA|42695|11|51|35|AM|first|morning|| +42696|AAAAAAAAJMGKAAAA|42696|11|51|36|AM|first|morning|| +42697|AAAAAAAAKMGKAAAA|42697|11|51|37|AM|first|morning|| +42698|AAAAAAAALMGKAAAA|42698|11|51|38|AM|first|morning|| +42699|AAAAAAAAMMGKAAAA|42699|11|51|39|AM|first|morning|| +42700|AAAAAAAANMGKAAAA|42700|11|51|40|AM|first|morning|| +42701|AAAAAAAAOMGKAAAA|42701|11|51|41|AM|first|morning|| +42702|AAAAAAAAPMGKAAAA|42702|11|51|42|AM|first|morning|| +42703|AAAAAAAAANGKAAAA|42703|11|51|43|AM|first|morning|| +42704|AAAAAAAABNGKAAAA|42704|11|51|44|AM|first|morning|| +42705|AAAAAAAACNGKAAAA|42705|11|51|45|AM|first|morning|| +42706|AAAAAAAADNGKAAAA|42706|11|51|46|AM|first|morning|| +42707|AAAAAAAAENGKAAAA|42707|11|51|47|AM|first|morning|| +42708|AAAAAAAAFNGKAAAA|42708|11|51|48|AM|first|morning|| +42709|AAAAAAAAGNGKAAAA|42709|11|51|49|AM|first|morning|| +42710|AAAAAAAAHNGKAAAA|42710|11|51|50|AM|first|morning|| +42711|AAAAAAAAINGKAAAA|42711|11|51|51|AM|first|morning|| +42712|AAAAAAAAJNGKAAAA|42712|11|51|52|AM|first|morning|| +42713|AAAAAAAAKNGKAAAA|42713|11|51|53|AM|first|morning|| +42714|AAAAAAAALNGKAAAA|42714|11|51|54|AM|first|morning|| +42715|AAAAAAAAMNGKAAAA|42715|11|51|55|AM|first|morning|| +42716|AAAAAAAANNGKAAAA|42716|11|51|56|AM|first|morning|| +42717|AAAAAAAAONGKAAAA|42717|11|51|57|AM|first|morning|| +42718|AAAAAAAAPNGKAAAA|42718|11|51|58|AM|first|morning|| +42719|AAAAAAAAAOGKAAAA|42719|11|51|59|AM|first|morning|| +42720|AAAAAAAABOGKAAAA|42720|11|52|0|AM|first|morning|| +42721|AAAAAAAACOGKAAAA|42721|11|52|1|AM|first|morning|| +42722|AAAAAAAADOGKAAAA|42722|11|52|2|AM|first|morning|| +42723|AAAAAAAAEOGKAAAA|42723|11|52|3|AM|first|morning|| +42724|AAAAAAAAFOGKAAAA|42724|11|52|4|AM|first|morning|| +42725|AAAAAAAAGOGKAAAA|42725|11|52|5|AM|first|morning|| +42726|AAAAAAAAHOGKAAAA|42726|11|52|6|AM|first|morning|| +42727|AAAAAAAAIOGKAAAA|42727|11|52|7|AM|first|morning|| +42728|AAAAAAAAJOGKAAAA|42728|11|52|8|AM|first|morning|| +42729|AAAAAAAAKOGKAAAA|42729|11|52|9|AM|first|morning|| +42730|AAAAAAAALOGKAAAA|42730|11|52|10|AM|first|morning|| +42731|AAAAAAAAMOGKAAAA|42731|11|52|11|AM|first|morning|| +42732|AAAAAAAANOGKAAAA|42732|11|52|12|AM|first|morning|| +42733|AAAAAAAAOOGKAAAA|42733|11|52|13|AM|first|morning|| +42734|AAAAAAAAPOGKAAAA|42734|11|52|14|AM|first|morning|| +42735|AAAAAAAAAPGKAAAA|42735|11|52|15|AM|first|morning|| +42736|AAAAAAAABPGKAAAA|42736|11|52|16|AM|first|morning|| +42737|AAAAAAAACPGKAAAA|42737|11|52|17|AM|first|morning|| +42738|AAAAAAAADPGKAAAA|42738|11|52|18|AM|first|morning|| +42739|AAAAAAAAEPGKAAAA|42739|11|52|19|AM|first|morning|| +42740|AAAAAAAAFPGKAAAA|42740|11|52|20|AM|first|morning|| +42741|AAAAAAAAGPGKAAAA|42741|11|52|21|AM|first|morning|| +42742|AAAAAAAAHPGKAAAA|42742|11|52|22|AM|first|morning|| +42743|AAAAAAAAIPGKAAAA|42743|11|52|23|AM|first|morning|| +42744|AAAAAAAAJPGKAAAA|42744|11|52|24|AM|first|morning|| +42745|AAAAAAAAKPGKAAAA|42745|11|52|25|AM|first|morning|| +42746|AAAAAAAALPGKAAAA|42746|11|52|26|AM|first|morning|| +42747|AAAAAAAAMPGKAAAA|42747|11|52|27|AM|first|morning|| +42748|AAAAAAAANPGKAAAA|42748|11|52|28|AM|first|morning|| +42749|AAAAAAAAOPGKAAAA|42749|11|52|29|AM|first|morning|| +42750|AAAAAAAAPPGKAAAA|42750|11|52|30|AM|first|morning|| +42751|AAAAAAAAAAHKAAAA|42751|11|52|31|AM|first|morning|| +42752|AAAAAAAABAHKAAAA|42752|11|52|32|AM|first|morning|| +42753|AAAAAAAACAHKAAAA|42753|11|52|33|AM|first|morning|| +42754|AAAAAAAADAHKAAAA|42754|11|52|34|AM|first|morning|| +42755|AAAAAAAAEAHKAAAA|42755|11|52|35|AM|first|morning|| +42756|AAAAAAAAFAHKAAAA|42756|11|52|36|AM|first|morning|| +42757|AAAAAAAAGAHKAAAA|42757|11|52|37|AM|first|morning|| +42758|AAAAAAAAHAHKAAAA|42758|11|52|38|AM|first|morning|| +42759|AAAAAAAAIAHKAAAA|42759|11|52|39|AM|first|morning|| +42760|AAAAAAAAJAHKAAAA|42760|11|52|40|AM|first|morning|| +42761|AAAAAAAAKAHKAAAA|42761|11|52|41|AM|first|morning|| +42762|AAAAAAAALAHKAAAA|42762|11|52|42|AM|first|morning|| +42763|AAAAAAAAMAHKAAAA|42763|11|52|43|AM|first|morning|| +42764|AAAAAAAANAHKAAAA|42764|11|52|44|AM|first|morning|| +42765|AAAAAAAAOAHKAAAA|42765|11|52|45|AM|first|morning|| +42766|AAAAAAAAPAHKAAAA|42766|11|52|46|AM|first|morning|| +42767|AAAAAAAAABHKAAAA|42767|11|52|47|AM|first|morning|| +42768|AAAAAAAABBHKAAAA|42768|11|52|48|AM|first|morning|| +42769|AAAAAAAACBHKAAAA|42769|11|52|49|AM|first|morning|| +42770|AAAAAAAADBHKAAAA|42770|11|52|50|AM|first|morning|| +42771|AAAAAAAAEBHKAAAA|42771|11|52|51|AM|first|morning|| +42772|AAAAAAAAFBHKAAAA|42772|11|52|52|AM|first|morning|| +42773|AAAAAAAAGBHKAAAA|42773|11|52|53|AM|first|morning|| +42774|AAAAAAAAHBHKAAAA|42774|11|52|54|AM|first|morning|| +42775|AAAAAAAAIBHKAAAA|42775|11|52|55|AM|first|morning|| +42776|AAAAAAAAJBHKAAAA|42776|11|52|56|AM|first|morning|| +42777|AAAAAAAAKBHKAAAA|42777|11|52|57|AM|first|morning|| +42778|AAAAAAAALBHKAAAA|42778|11|52|58|AM|first|morning|| +42779|AAAAAAAAMBHKAAAA|42779|11|52|59|AM|first|morning|| +42780|AAAAAAAANBHKAAAA|42780|11|53|0|AM|first|morning|| +42781|AAAAAAAAOBHKAAAA|42781|11|53|1|AM|first|morning|| +42782|AAAAAAAAPBHKAAAA|42782|11|53|2|AM|first|morning|| +42783|AAAAAAAAACHKAAAA|42783|11|53|3|AM|first|morning|| +42784|AAAAAAAABCHKAAAA|42784|11|53|4|AM|first|morning|| +42785|AAAAAAAACCHKAAAA|42785|11|53|5|AM|first|morning|| +42786|AAAAAAAADCHKAAAA|42786|11|53|6|AM|first|morning|| +42787|AAAAAAAAECHKAAAA|42787|11|53|7|AM|first|morning|| +42788|AAAAAAAAFCHKAAAA|42788|11|53|8|AM|first|morning|| +42789|AAAAAAAAGCHKAAAA|42789|11|53|9|AM|first|morning|| +42790|AAAAAAAAHCHKAAAA|42790|11|53|10|AM|first|morning|| +42791|AAAAAAAAICHKAAAA|42791|11|53|11|AM|first|morning|| +42792|AAAAAAAAJCHKAAAA|42792|11|53|12|AM|first|morning|| +42793|AAAAAAAAKCHKAAAA|42793|11|53|13|AM|first|morning|| +42794|AAAAAAAALCHKAAAA|42794|11|53|14|AM|first|morning|| +42795|AAAAAAAAMCHKAAAA|42795|11|53|15|AM|first|morning|| +42796|AAAAAAAANCHKAAAA|42796|11|53|16|AM|first|morning|| +42797|AAAAAAAAOCHKAAAA|42797|11|53|17|AM|first|morning|| +42798|AAAAAAAAPCHKAAAA|42798|11|53|18|AM|first|morning|| +42799|AAAAAAAAADHKAAAA|42799|11|53|19|AM|first|morning|| +42800|AAAAAAAABDHKAAAA|42800|11|53|20|AM|first|morning|| +42801|AAAAAAAACDHKAAAA|42801|11|53|21|AM|first|morning|| +42802|AAAAAAAADDHKAAAA|42802|11|53|22|AM|first|morning|| +42803|AAAAAAAAEDHKAAAA|42803|11|53|23|AM|first|morning|| +42804|AAAAAAAAFDHKAAAA|42804|11|53|24|AM|first|morning|| +42805|AAAAAAAAGDHKAAAA|42805|11|53|25|AM|first|morning|| +42806|AAAAAAAAHDHKAAAA|42806|11|53|26|AM|first|morning|| +42807|AAAAAAAAIDHKAAAA|42807|11|53|27|AM|first|morning|| +42808|AAAAAAAAJDHKAAAA|42808|11|53|28|AM|first|morning|| +42809|AAAAAAAAKDHKAAAA|42809|11|53|29|AM|first|morning|| +42810|AAAAAAAALDHKAAAA|42810|11|53|30|AM|first|morning|| +42811|AAAAAAAAMDHKAAAA|42811|11|53|31|AM|first|morning|| +42812|AAAAAAAANDHKAAAA|42812|11|53|32|AM|first|morning|| +42813|AAAAAAAAODHKAAAA|42813|11|53|33|AM|first|morning|| +42814|AAAAAAAAPDHKAAAA|42814|11|53|34|AM|first|morning|| +42815|AAAAAAAAAEHKAAAA|42815|11|53|35|AM|first|morning|| +42816|AAAAAAAABEHKAAAA|42816|11|53|36|AM|first|morning|| +42817|AAAAAAAACEHKAAAA|42817|11|53|37|AM|first|morning|| +42818|AAAAAAAADEHKAAAA|42818|11|53|38|AM|first|morning|| +42819|AAAAAAAAEEHKAAAA|42819|11|53|39|AM|first|morning|| +42820|AAAAAAAAFEHKAAAA|42820|11|53|40|AM|first|morning|| +42821|AAAAAAAAGEHKAAAA|42821|11|53|41|AM|first|morning|| +42822|AAAAAAAAHEHKAAAA|42822|11|53|42|AM|first|morning|| +42823|AAAAAAAAIEHKAAAA|42823|11|53|43|AM|first|morning|| +42824|AAAAAAAAJEHKAAAA|42824|11|53|44|AM|first|morning|| +42825|AAAAAAAAKEHKAAAA|42825|11|53|45|AM|first|morning|| +42826|AAAAAAAALEHKAAAA|42826|11|53|46|AM|first|morning|| +42827|AAAAAAAAMEHKAAAA|42827|11|53|47|AM|first|morning|| +42828|AAAAAAAANEHKAAAA|42828|11|53|48|AM|first|morning|| +42829|AAAAAAAAOEHKAAAA|42829|11|53|49|AM|first|morning|| +42830|AAAAAAAAPEHKAAAA|42830|11|53|50|AM|first|morning|| +42831|AAAAAAAAAFHKAAAA|42831|11|53|51|AM|first|morning|| +42832|AAAAAAAABFHKAAAA|42832|11|53|52|AM|first|morning|| +42833|AAAAAAAACFHKAAAA|42833|11|53|53|AM|first|morning|| +42834|AAAAAAAADFHKAAAA|42834|11|53|54|AM|first|morning|| +42835|AAAAAAAAEFHKAAAA|42835|11|53|55|AM|first|morning|| +42836|AAAAAAAAFFHKAAAA|42836|11|53|56|AM|first|morning|| +42837|AAAAAAAAGFHKAAAA|42837|11|53|57|AM|first|morning|| +42838|AAAAAAAAHFHKAAAA|42838|11|53|58|AM|first|morning|| +42839|AAAAAAAAIFHKAAAA|42839|11|53|59|AM|first|morning|| +42840|AAAAAAAAJFHKAAAA|42840|11|54|0|AM|first|morning|| +42841|AAAAAAAAKFHKAAAA|42841|11|54|1|AM|first|morning|| +42842|AAAAAAAALFHKAAAA|42842|11|54|2|AM|first|morning|| +42843|AAAAAAAAMFHKAAAA|42843|11|54|3|AM|first|morning|| +42844|AAAAAAAANFHKAAAA|42844|11|54|4|AM|first|morning|| +42845|AAAAAAAAOFHKAAAA|42845|11|54|5|AM|first|morning|| +42846|AAAAAAAAPFHKAAAA|42846|11|54|6|AM|first|morning|| +42847|AAAAAAAAAGHKAAAA|42847|11|54|7|AM|first|morning|| +42848|AAAAAAAABGHKAAAA|42848|11|54|8|AM|first|morning|| +42849|AAAAAAAACGHKAAAA|42849|11|54|9|AM|first|morning|| +42850|AAAAAAAADGHKAAAA|42850|11|54|10|AM|first|morning|| +42851|AAAAAAAAEGHKAAAA|42851|11|54|11|AM|first|morning|| +42852|AAAAAAAAFGHKAAAA|42852|11|54|12|AM|first|morning|| +42853|AAAAAAAAGGHKAAAA|42853|11|54|13|AM|first|morning|| +42854|AAAAAAAAHGHKAAAA|42854|11|54|14|AM|first|morning|| +42855|AAAAAAAAIGHKAAAA|42855|11|54|15|AM|first|morning|| +42856|AAAAAAAAJGHKAAAA|42856|11|54|16|AM|first|morning|| +42857|AAAAAAAAKGHKAAAA|42857|11|54|17|AM|first|morning|| +42858|AAAAAAAALGHKAAAA|42858|11|54|18|AM|first|morning|| +42859|AAAAAAAAMGHKAAAA|42859|11|54|19|AM|first|morning|| +42860|AAAAAAAANGHKAAAA|42860|11|54|20|AM|first|morning|| +42861|AAAAAAAAOGHKAAAA|42861|11|54|21|AM|first|morning|| +42862|AAAAAAAAPGHKAAAA|42862|11|54|22|AM|first|morning|| +42863|AAAAAAAAAHHKAAAA|42863|11|54|23|AM|first|morning|| +42864|AAAAAAAABHHKAAAA|42864|11|54|24|AM|first|morning|| +42865|AAAAAAAACHHKAAAA|42865|11|54|25|AM|first|morning|| +42866|AAAAAAAADHHKAAAA|42866|11|54|26|AM|first|morning|| +42867|AAAAAAAAEHHKAAAA|42867|11|54|27|AM|first|morning|| +42868|AAAAAAAAFHHKAAAA|42868|11|54|28|AM|first|morning|| +42869|AAAAAAAAGHHKAAAA|42869|11|54|29|AM|first|morning|| +42870|AAAAAAAAHHHKAAAA|42870|11|54|30|AM|first|morning|| +42871|AAAAAAAAIHHKAAAA|42871|11|54|31|AM|first|morning|| +42872|AAAAAAAAJHHKAAAA|42872|11|54|32|AM|first|morning|| +42873|AAAAAAAAKHHKAAAA|42873|11|54|33|AM|first|morning|| +42874|AAAAAAAALHHKAAAA|42874|11|54|34|AM|first|morning|| +42875|AAAAAAAAMHHKAAAA|42875|11|54|35|AM|first|morning|| +42876|AAAAAAAANHHKAAAA|42876|11|54|36|AM|first|morning|| +42877|AAAAAAAAOHHKAAAA|42877|11|54|37|AM|first|morning|| +42878|AAAAAAAAPHHKAAAA|42878|11|54|38|AM|first|morning|| +42879|AAAAAAAAAIHKAAAA|42879|11|54|39|AM|first|morning|| +42880|AAAAAAAABIHKAAAA|42880|11|54|40|AM|first|morning|| +42881|AAAAAAAACIHKAAAA|42881|11|54|41|AM|first|morning|| +42882|AAAAAAAADIHKAAAA|42882|11|54|42|AM|first|morning|| +42883|AAAAAAAAEIHKAAAA|42883|11|54|43|AM|first|morning|| +42884|AAAAAAAAFIHKAAAA|42884|11|54|44|AM|first|morning|| +42885|AAAAAAAAGIHKAAAA|42885|11|54|45|AM|first|morning|| +42886|AAAAAAAAHIHKAAAA|42886|11|54|46|AM|first|morning|| +42887|AAAAAAAAIIHKAAAA|42887|11|54|47|AM|first|morning|| +42888|AAAAAAAAJIHKAAAA|42888|11|54|48|AM|first|morning|| +42889|AAAAAAAAKIHKAAAA|42889|11|54|49|AM|first|morning|| +42890|AAAAAAAALIHKAAAA|42890|11|54|50|AM|first|morning|| +42891|AAAAAAAAMIHKAAAA|42891|11|54|51|AM|first|morning|| +42892|AAAAAAAANIHKAAAA|42892|11|54|52|AM|first|morning|| +42893|AAAAAAAAOIHKAAAA|42893|11|54|53|AM|first|morning|| +42894|AAAAAAAAPIHKAAAA|42894|11|54|54|AM|first|morning|| +42895|AAAAAAAAAJHKAAAA|42895|11|54|55|AM|first|morning|| +42896|AAAAAAAABJHKAAAA|42896|11|54|56|AM|first|morning|| +42897|AAAAAAAACJHKAAAA|42897|11|54|57|AM|first|morning|| +42898|AAAAAAAADJHKAAAA|42898|11|54|58|AM|first|morning|| +42899|AAAAAAAAEJHKAAAA|42899|11|54|59|AM|first|morning|| +42900|AAAAAAAAFJHKAAAA|42900|11|55|0|AM|first|morning|| +42901|AAAAAAAAGJHKAAAA|42901|11|55|1|AM|first|morning|| +42902|AAAAAAAAHJHKAAAA|42902|11|55|2|AM|first|morning|| +42903|AAAAAAAAIJHKAAAA|42903|11|55|3|AM|first|morning|| +42904|AAAAAAAAJJHKAAAA|42904|11|55|4|AM|first|morning|| +42905|AAAAAAAAKJHKAAAA|42905|11|55|5|AM|first|morning|| +42906|AAAAAAAALJHKAAAA|42906|11|55|6|AM|first|morning|| +42907|AAAAAAAAMJHKAAAA|42907|11|55|7|AM|first|morning|| +42908|AAAAAAAANJHKAAAA|42908|11|55|8|AM|first|morning|| +42909|AAAAAAAAOJHKAAAA|42909|11|55|9|AM|first|morning|| +42910|AAAAAAAAPJHKAAAA|42910|11|55|10|AM|first|morning|| +42911|AAAAAAAAAKHKAAAA|42911|11|55|11|AM|first|morning|| +42912|AAAAAAAABKHKAAAA|42912|11|55|12|AM|first|morning|| +42913|AAAAAAAACKHKAAAA|42913|11|55|13|AM|first|morning|| +42914|AAAAAAAADKHKAAAA|42914|11|55|14|AM|first|morning|| +42915|AAAAAAAAEKHKAAAA|42915|11|55|15|AM|first|morning|| +42916|AAAAAAAAFKHKAAAA|42916|11|55|16|AM|first|morning|| +42917|AAAAAAAAGKHKAAAA|42917|11|55|17|AM|first|morning|| +42918|AAAAAAAAHKHKAAAA|42918|11|55|18|AM|first|morning|| +42919|AAAAAAAAIKHKAAAA|42919|11|55|19|AM|first|morning|| +42920|AAAAAAAAJKHKAAAA|42920|11|55|20|AM|first|morning|| +42921|AAAAAAAAKKHKAAAA|42921|11|55|21|AM|first|morning|| +42922|AAAAAAAALKHKAAAA|42922|11|55|22|AM|first|morning|| +42923|AAAAAAAAMKHKAAAA|42923|11|55|23|AM|first|morning|| +42924|AAAAAAAANKHKAAAA|42924|11|55|24|AM|first|morning|| +42925|AAAAAAAAOKHKAAAA|42925|11|55|25|AM|first|morning|| +42926|AAAAAAAAPKHKAAAA|42926|11|55|26|AM|first|morning|| +42927|AAAAAAAAALHKAAAA|42927|11|55|27|AM|first|morning|| +42928|AAAAAAAABLHKAAAA|42928|11|55|28|AM|first|morning|| +42929|AAAAAAAACLHKAAAA|42929|11|55|29|AM|first|morning|| +42930|AAAAAAAADLHKAAAA|42930|11|55|30|AM|first|morning|| +42931|AAAAAAAAELHKAAAA|42931|11|55|31|AM|first|morning|| +42932|AAAAAAAAFLHKAAAA|42932|11|55|32|AM|first|morning|| +42933|AAAAAAAAGLHKAAAA|42933|11|55|33|AM|first|morning|| +42934|AAAAAAAAHLHKAAAA|42934|11|55|34|AM|first|morning|| +42935|AAAAAAAAILHKAAAA|42935|11|55|35|AM|first|morning|| +42936|AAAAAAAAJLHKAAAA|42936|11|55|36|AM|first|morning|| +42937|AAAAAAAAKLHKAAAA|42937|11|55|37|AM|first|morning|| +42938|AAAAAAAALLHKAAAA|42938|11|55|38|AM|first|morning|| +42939|AAAAAAAAMLHKAAAA|42939|11|55|39|AM|first|morning|| +42940|AAAAAAAANLHKAAAA|42940|11|55|40|AM|first|morning|| +42941|AAAAAAAAOLHKAAAA|42941|11|55|41|AM|first|morning|| +42942|AAAAAAAAPLHKAAAA|42942|11|55|42|AM|first|morning|| +42943|AAAAAAAAAMHKAAAA|42943|11|55|43|AM|first|morning|| +42944|AAAAAAAABMHKAAAA|42944|11|55|44|AM|first|morning|| +42945|AAAAAAAACMHKAAAA|42945|11|55|45|AM|first|morning|| +42946|AAAAAAAADMHKAAAA|42946|11|55|46|AM|first|morning|| +42947|AAAAAAAAEMHKAAAA|42947|11|55|47|AM|first|morning|| +42948|AAAAAAAAFMHKAAAA|42948|11|55|48|AM|first|morning|| +42949|AAAAAAAAGMHKAAAA|42949|11|55|49|AM|first|morning|| +42950|AAAAAAAAHMHKAAAA|42950|11|55|50|AM|first|morning|| +42951|AAAAAAAAIMHKAAAA|42951|11|55|51|AM|first|morning|| +42952|AAAAAAAAJMHKAAAA|42952|11|55|52|AM|first|morning|| +42953|AAAAAAAAKMHKAAAA|42953|11|55|53|AM|first|morning|| +42954|AAAAAAAALMHKAAAA|42954|11|55|54|AM|first|morning|| +42955|AAAAAAAAMMHKAAAA|42955|11|55|55|AM|first|morning|| +42956|AAAAAAAANMHKAAAA|42956|11|55|56|AM|first|morning|| +42957|AAAAAAAAOMHKAAAA|42957|11|55|57|AM|first|morning|| +42958|AAAAAAAAPMHKAAAA|42958|11|55|58|AM|first|morning|| +42959|AAAAAAAAANHKAAAA|42959|11|55|59|AM|first|morning|| +42960|AAAAAAAABNHKAAAA|42960|11|56|0|AM|first|morning|| +42961|AAAAAAAACNHKAAAA|42961|11|56|1|AM|first|morning|| +42962|AAAAAAAADNHKAAAA|42962|11|56|2|AM|first|morning|| +42963|AAAAAAAAENHKAAAA|42963|11|56|3|AM|first|morning|| +42964|AAAAAAAAFNHKAAAA|42964|11|56|4|AM|first|morning|| +42965|AAAAAAAAGNHKAAAA|42965|11|56|5|AM|first|morning|| +42966|AAAAAAAAHNHKAAAA|42966|11|56|6|AM|first|morning|| +42967|AAAAAAAAINHKAAAA|42967|11|56|7|AM|first|morning|| +42968|AAAAAAAAJNHKAAAA|42968|11|56|8|AM|first|morning|| +42969|AAAAAAAAKNHKAAAA|42969|11|56|9|AM|first|morning|| +42970|AAAAAAAALNHKAAAA|42970|11|56|10|AM|first|morning|| +42971|AAAAAAAAMNHKAAAA|42971|11|56|11|AM|first|morning|| +42972|AAAAAAAANNHKAAAA|42972|11|56|12|AM|first|morning|| +42973|AAAAAAAAONHKAAAA|42973|11|56|13|AM|first|morning|| +42974|AAAAAAAAPNHKAAAA|42974|11|56|14|AM|first|morning|| +42975|AAAAAAAAAOHKAAAA|42975|11|56|15|AM|first|morning|| +42976|AAAAAAAABOHKAAAA|42976|11|56|16|AM|first|morning|| +42977|AAAAAAAACOHKAAAA|42977|11|56|17|AM|first|morning|| +42978|AAAAAAAADOHKAAAA|42978|11|56|18|AM|first|morning|| +42979|AAAAAAAAEOHKAAAA|42979|11|56|19|AM|first|morning|| +42980|AAAAAAAAFOHKAAAA|42980|11|56|20|AM|first|morning|| +42981|AAAAAAAAGOHKAAAA|42981|11|56|21|AM|first|morning|| +42982|AAAAAAAAHOHKAAAA|42982|11|56|22|AM|first|morning|| +42983|AAAAAAAAIOHKAAAA|42983|11|56|23|AM|first|morning|| +42984|AAAAAAAAJOHKAAAA|42984|11|56|24|AM|first|morning|| +42985|AAAAAAAAKOHKAAAA|42985|11|56|25|AM|first|morning|| +42986|AAAAAAAALOHKAAAA|42986|11|56|26|AM|first|morning|| +42987|AAAAAAAAMOHKAAAA|42987|11|56|27|AM|first|morning|| +42988|AAAAAAAANOHKAAAA|42988|11|56|28|AM|first|morning|| +42989|AAAAAAAAOOHKAAAA|42989|11|56|29|AM|first|morning|| +42990|AAAAAAAAPOHKAAAA|42990|11|56|30|AM|first|morning|| +42991|AAAAAAAAAPHKAAAA|42991|11|56|31|AM|first|morning|| +42992|AAAAAAAABPHKAAAA|42992|11|56|32|AM|first|morning|| +42993|AAAAAAAACPHKAAAA|42993|11|56|33|AM|first|morning|| +42994|AAAAAAAADPHKAAAA|42994|11|56|34|AM|first|morning|| +42995|AAAAAAAAEPHKAAAA|42995|11|56|35|AM|first|morning|| +42996|AAAAAAAAFPHKAAAA|42996|11|56|36|AM|first|morning|| +42997|AAAAAAAAGPHKAAAA|42997|11|56|37|AM|first|morning|| +42998|AAAAAAAAHPHKAAAA|42998|11|56|38|AM|first|morning|| +42999|AAAAAAAAIPHKAAAA|42999|11|56|39|AM|first|morning|| +43000|AAAAAAAAJPHKAAAA|43000|11|56|40|AM|first|morning|| +43001|AAAAAAAAKPHKAAAA|43001|11|56|41|AM|first|morning|| +43002|AAAAAAAALPHKAAAA|43002|11|56|42|AM|first|morning|| +43003|AAAAAAAAMPHKAAAA|43003|11|56|43|AM|first|morning|| +43004|AAAAAAAANPHKAAAA|43004|11|56|44|AM|first|morning|| +43005|AAAAAAAAOPHKAAAA|43005|11|56|45|AM|first|morning|| +43006|AAAAAAAAPPHKAAAA|43006|11|56|46|AM|first|morning|| +43007|AAAAAAAAAAIKAAAA|43007|11|56|47|AM|first|morning|| +43008|AAAAAAAABAIKAAAA|43008|11|56|48|AM|first|morning|| +43009|AAAAAAAACAIKAAAA|43009|11|56|49|AM|first|morning|| +43010|AAAAAAAADAIKAAAA|43010|11|56|50|AM|first|morning|| +43011|AAAAAAAAEAIKAAAA|43011|11|56|51|AM|first|morning|| +43012|AAAAAAAAFAIKAAAA|43012|11|56|52|AM|first|morning|| +43013|AAAAAAAAGAIKAAAA|43013|11|56|53|AM|first|morning|| +43014|AAAAAAAAHAIKAAAA|43014|11|56|54|AM|first|morning|| +43015|AAAAAAAAIAIKAAAA|43015|11|56|55|AM|first|morning|| +43016|AAAAAAAAJAIKAAAA|43016|11|56|56|AM|first|morning|| +43017|AAAAAAAAKAIKAAAA|43017|11|56|57|AM|first|morning|| +43018|AAAAAAAALAIKAAAA|43018|11|56|58|AM|first|morning|| +43019|AAAAAAAAMAIKAAAA|43019|11|56|59|AM|first|morning|| +43020|AAAAAAAANAIKAAAA|43020|11|57|0|AM|first|morning|| +43021|AAAAAAAAOAIKAAAA|43021|11|57|1|AM|first|morning|| +43022|AAAAAAAAPAIKAAAA|43022|11|57|2|AM|first|morning|| +43023|AAAAAAAAABIKAAAA|43023|11|57|3|AM|first|morning|| +43024|AAAAAAAABBIKAAAA|43024|11|57|4|AM|first|morning|| +43025|AAAAAAAACBIKAAAA|43025|11|57|5|AM|first|morning|| +43026|AAAAAAAADBIKAAAA|43026|11|57|6|AM|first|morning|| +43027|AAAAAAAAEBIKAAAA|43027|11|57|7|AM|first|morning|| +43028|AAAAAAAAFBIKAAAA|43028|11|57|8|AM|first|morning|| +43029|AAAAAAAAGBIKAAAA|43029|11|57|9|AM|first|morning|| +43030|AAAAAAAAHBIKAAAA|43030|11|57|10|AM|first|morning|| +43031|AAAAAAAAIBIKAAAA|43031|11|57|11|AM|first|morning|| +43032|AAAAAAAAJBIKAAAA|43032|11|57|12|AM|first|morning|| +43033|AAAAAAAAKBIKAAAA|43033|11|57|13|AM|first|morning|| +43034|AAAAAAAALBIKAAAA|43034|11|57|14|AM|first|morning|| +43035|AAAAAAAAMBIKAAAA|43035|11|57|15|AM|first|morning|| +43036|AAAAAAAANBIKAAAA|43036|11|57|16|AM|first|morning|| +43037|AAAAAAAAOBIKAAAA|43037|11|57|17|AM|first|morning|| +43038|AAAAAAAAPBIKAAAA|43038|11|57|18|AM|first|morning|| +43039|AAAAAAAAACIKAAAA|43039|11|57|19|AM|first|morning|| +43040|AAAAAAAABCIKAAAA|43040|11|57|20|AM|first|morning|| +43041|AAAAAAAACCIKAAAA|43041|11|57|21|AM|first|morning|| +43042|AAAAAAAADCIKAAAA|43042|11|57|22|AM|first|morning|| +43043|AAAAAAAAECIKAAAA|43043|11|57|23|AM|first|morning|| +43044|AAAAAAAAFCIKAAAA|43044|11|57|24|AM|first|morning|| +43045|AAAAAAAAGCIKAAAA|43045|11|57|25|AM|first|morning|| +43046|AAAAAAAAHCIKAAAA|43046|11|57|26|AM|first|morning|| +43047|AAAAAAAAICIKAAAA|43047|11|57|27|AM|first|morning|| +43048|AAAAAAAAJCIKAAAA|43048|11|57|28|AM|first|morning|| +43049|AAAAAAAAKCIKAAAA|43049|11|57|29|AM|first|morning|| +43050|AAAAAAAALCIKAAAA|43050|11|57|30|AM|first|morning|| +43051|AAAAAAAAMCIKAAAA|43051|11|57|31|AM|first|morning|| +43052|AAAAAAAANCIKAAAA|43052|11|57|32|AM|first|morning|| +43053|AAAAAAAAOCIKAAAA|43053|11|57|33|AM|first|morning|| +43054|AAAAAAAAPCIKAAAA|43054|11|57|34|AM|first|morning|| +43055|AAAAAAAAADIKAAAA|43055|11|57|35|AM|first|morning|| +43056|AAAAAAAABDIKAAAA|43056|11|57|36|AM|first|morning|| +43057|AAAAAAAACDIKAAAA|43057|11|57|37|AM|first|morning|| +43058|AAAAAAAADDIKAAAA|43058|11|57|38|AM|first|morning|| +43059|AAAAAAAAEDIKAAAA|43059|11|57|39|AM|first|morning|| +43060|AAAAAAAAFDIKAAAA|43060|11|57|40|AM|first|morning|| +43061|AAAAAAAAGDIKAAAA|43061|11|57|41|AM|first|morning|| +43062|AAAAAAAAHDIKAAAA|43062|11|57|42|AM|first|morning|| +43063|AAAAAAAAIDIKAAAA|43063|11|57|43|AM|first|morning|| +43064|AAAAAAAAJDIKAAAA|43064|11|57|44|AM|first|morning|| +43065|AAAAAAAAKDIKAAAA|43065|11|57|45|AM|first|morning|| +43066|AAAAAAAALDIKAAAA|43066|11|57|46|AM|first|morning|| +43067|AAAAAAAAMDIKAAAA|43067|11|57|47|AM|first|morning|| +43068|AAAAAAAANDIKAAAA|43068|11|57|48|AM|first|morning|| +43069|AAAAAAAAODIKAAAA|43069|11|57|49|AM|first|morning|| +43070|AAAAAAAAPDIKAAAA|43070|11|57|50|AM|first|morning|| +43071|AAAAAAAAAEIKAAAA|43071|11|57|51|AM|first|morning|| +43072|AAAAAAAABEIKAAAA|43072|11|57|52|AM|first|morning|| +43073|AAAAAAAACEIKAAAA|43073|11|57|53|AM|first|morning|| +43074|AAAAAAAADEIKAAAA|43074|11|57|54|AM|first|morning|| +43075|AAAAAAAAEEIKAAAA|43075|11|57|55|AM|first|morning|| +43076|AAAAAAAAFEIKAAAA|43076|11|57|56|AM|first|morning|| +43077|AAAAAAAAGEIKAAAA|43077|11|57|57|AM|first|morning|| +43078|AAAAAAAAHEIKAAAA|43078|11|57|58|AM|first|morning|| +43079|AAAAAAAAIEIKAAAA|43079|11|57|59|AM|first|morning|| +43080|AAAAAAAAJEIKAAAA|43080|11|58|0|AM|first|morning|| +43081|AAAAAAAAKEIKAAAA|43081|11|58|1|AM|first|morning|| +43082|AAAAAAAALEIKAAAA|43082|11|58|2|AM|first|morning|| +43083|AAAAAAAAMEIKAAAA|43083|11|58|3|AM|first|morning|| +43084|AAAAAAAANEIKAAAA|43084|11|58|4|AM|first|morning|| +43085|AAAAAAAAOEIKAAAA|43085|11|58|5|AM|first|morning|| +43086|AAAAAAAAPEIKAAAA|43086|11|58|6|AM|first|morning|| +43087|AAAAAAAAAFIKAAAA|43087|11|58|7|AM|first|morning|| +43088|AAAAAAAABFIKAAAA|43088|11|58|8|AM|first|morning|| +43089|AAAAAAAACFIKAAAA|43089|11|58|9|AM|first|morning|| +43090|AAAAAAAADFIKAAAA|43090|11|58|10|AM|first|morning|| +43091|AAAAAAAAEFIKAAAA|43091|11|58|11|AM|first|morning|| +43092|AAAAAAAAFFIKAAAA|43092|11|58|12|AM|first|morning|| +43093|AAAAAAAAGFIKAAAA|43093|11|58|13|AM|first|morning|| +43094|AAAAAAAAHFIKAAAA|43094|11|58|14|AM|first|morning|| +43095|AAAAAAAAIFIKAAAA|43095|11|58|15|AM|first|morning|| +43096|AAAAAAAAJFIKAAAA|43096|11|58|16|AM|first|morning|| +43097|AAAAAAAAKFIKAAAA|43097|11|58|17|AM|first|morning|| +43098|AAAAAAAALFIKAAAA|43098|11|58|18|AM|first|morning|| +43099|AAAAAAAAMFIKAAAA|43099|11|58|19|AM|first|morning|| +43100|AAAAAAAANFIKAAAA|43100|11|58|20|AM|first|morning|| +43101|AAAAAAAAOFIKAAAA|43101|11|58|21|AM|first|morning|| +43102|AAAAAAAAPFIKAAAA|43102|11|58|22|AM|first|morning|| +43103|AAAAAAAAAGIKAAAA|43103|11|58|23|AM|first|morning|| +43104|AAAAAAAABGIKAAAA|43104|11|58|24|AM|first|morning|| +43105|AAAAAAAACGIKAAAA|43105|11|58|25|AM|first|morning|| +43106|AAAAAAAADGIKAAAA|43106|11|58|26|AM|first|morning|| +43107|AAAAAAAAEGIKAAAA|43107|11|58|27|AM|first|morning|| +43108|AAAAAAAAFGIKAAAA|43108|11|58|28|AM|first|morning|| +43109|AAAAAAAAGGIKAAAA|43109|11|58|29|AM|first|morning|| +43110|AAAAAAAAHGIKAAAA|43110|11|58|30|AM|first|morning|| +43111|AAAAAAAAIGIKAAAA|43111|11|58|31|AM|first|morning|| +43112|AAAAAAAAJGIKAAAA|43112|11|58|32|AM|first|morning|| +43113|AAAAAAAAKGIKAAAA|43113|11|58|33|AM|first|morning|| +43114|AAAAAAAALGIKAAAA|43114|11|58|34|AM|first|morning|| +43115|AAAAAAAAMGIKAAAA|43115|11|58|35|AM|first|morning|| +43116|AAAAAAAANGIKAAAA|43116|11|58|36|AM|first|morning|| +43117|AAAAAAAAOGIKAAAA|43117|11|58|37|AM|first|morning|| +43118|AAAAAAAAPGIKAAAA|43118|11|58|38|AM|first|morning|| +43119|AAAAAAAAAHIKAAAA|43119|11|58|39|AM|first|morning|| +43120|AAAAAAAABHIKAAAA|43120|11|58|40|AM|first|morning|| +43121|AAAAAAAACHIKAAAA|43121|11|58|41|AM|first|morning|| +43122|AAAAAAAADHIKAAAA|43122|11|58|42|AM|first|morning|| +43123|AAAAAAAAEHIKAAAA|43123|11|58|43|AM|first|morning|| +43124|AAAAAAAAFHIKAAAA|43124|11|58|44|AM|first|morning|| +43125|AAAAAAAAGHIKAAAA|43125|11|58|45|AM|first|morning|| +43126|AAAAAAAAHHIKAAAA|43126|11|58|46|AM|first|morning|| +43127|AAAAAAAAIHIKAAAA|43127|11|58|47|AM|first|morning|| +43128|AAAAAAAAJHIKAAAA|43128|11|58|48|AM|first|morning|| +43129|AAAAAAAAKHIKAAAA|43129|11|58|49|AM|first|morning|| +43130|AAAAAAAALHIKAAAA|43130|11|58|50|AM|first|morning|| +43131|AAAAAAAAMHIKAAAA|43131|11|58|51|AM|first|morning|| +43132|AAAAAAAANHIKAAAA|43132|11|58|52|AM|first|morning|| +43133|AAAAAAAAOHIKAAAA|43133|11|58|53|AM|first|morning|| +43134|AAAAAAAAPHIKAAAA|43134|11|58|54|AM|first|morning|| +43135|AAAAAAAAAIIKAAAA|43135|11|58|55|AM|first|morning|| +43136|AAAAAAAABIIKAAAA|43136|11|58|56|AM|first|morning|| +43137|AAAAAAAACIIKAAAA|43137|11|58|57|AM|first|morning|| +43138|AAAAAAAADIIKAAAA|43138|11|58|58|AM|first|morning|| +43139|AAAAAAAAEIIKAAAA|43139|11|58|59|AM|first|morning|| +43140|AAAAAAAAFIIKAAAA|43140|11|59|0|AM|first|morning|| +43141|AAAAAAAAGIIKAAAA|43141|11|59|1|AM|first|morning|| +43142|AAAAAAAAHIIKAAAA|43142|11|59|2|AM|first|morning|| +43143|AAAAAAAAIIIKAAAA|43143|11|59|3|AM|first|morning|| +43144|AAAAAAAAJIIKAAAA|43144|11|59|4|AM|first|morning|| +43145|AAAAAAAAKIIKAAAA|43145|11|59|5|AM|first|morning|| +43146|AAAAAAAALIIKAAAA|43146|11|59|6|AM|first|morning|| +43147|AAAAAAAAMIIKAAAA|43147|11|59|7|AM|first|morning|| +43148|AAAAAAAANIIKAAAA|43148|11|59|8|AM|first|morning|| +43149|AAAAAAAAOIIKAAAA|43149|11|59|9|AM|first|morning|| +43150|AAAAAAAAPIIKAAAA|43150|11|59|10|AM|first|morning|| +43151|AAAAAAAAAJIKAAAA|43151|11|59|11|AM|first|morning|| +43152|AAAAAAAABJIKAAAA|43152|11|59|12|AM|first|morning|| +43153|AAAAAAAACJIKAAAA|43153|11|59|13|AM|first|morning|| +43154|AAAAAAAADJIKAAAA|43154|11|59|14|AM|first|morning|| +43155|AAAAAAAAEJIKAAAA|43155|11|59|15|AM|first|morning|| +43156|AAAAAAAAFJIKAAAA|43156|11|59|16|AM|first|morning|| +43157|AAAAAAAAGJIKAAAA|43157|11|59|17|AM|first|morning|| +43158|AAAAAAAAHJIKAAAA|43158|11|59|18|AM|first|morning|| +43159|AAAAAAAAIJIKAAAA|43159|11|59|19|AM|first|morning|| +43160|AAAAAAAAJJIKAAAA|43160|11|59|20|AM|first|morning|| +43161|AAAAAAAAKJIKAAAA|43161|11|59|21|AM|first|morning|| +43162|AAAAAAAALJIKAAAA|43162|11|59|22|AM|first|morning|| +43163|AAAAAAAAMJIKAAAA|43163|11|59|23|AM|first|morning|| +43164|AAAAAAAANJIKAAAA|43164|11|59|24|AM|first|morning|| +43165|AAAAAAAAOJIKAAAA|43165|11|59|25|AM|first|morning|| +43166|AAAAAAAAPJIKAAAA|43166|11|59|26|AM|first|morning|| +43167|AAAAAAAAAKIKAAAA|43167|11|59|27|AM|first|morning|| +43168|AAAAAAAABKIKAAAA|43168|11|59|28|AM|first|morning|| +43169|AAAAAAAACKIKAAAA|43169|11|59|29|AM|first|morning|| +43170|AAAAAAAADKIKAAAA|43170|11|59|30|AM|first|morning|| +43171|AAAAAAAAEKIKAAAA|43171|11|59|31|AM|first|morning|| +43172|AAAAAAAAFKIKAAAA|43172|11|59|32|AM|first|morning|| +43173|AAAAAAAAGKIKAAAA|43173|11|59|33|AM|first|morning|| +43174|AAAAAAAAHKIKAAAA|43174|11|59|34|AM|first|morning|| +43175|AAAAAAAAIKIKAAAA|43175|11|59|35|AM|first|morning|| +43176|AAAAAAAAJKIKAAAA|43176|11|59|36|AM|first|morning|| +43177|AAAAAAAAKKIKAAAA|43177|11|59|37|AM|first|morning|| +43178|AAAAAAAALKIKAAAA|43178|11|59|38|AM|first|morning|| +43179|AAAAAAAAMKIKAAAA|43179|11|59|39|AM|first|morning|| +43180|AAAAAAAANKIKAAAA|43180|11|59|40|AM|first|morning|| +43181|AAAAAAAAOKIKAAAA|43181|11|59|41|AM|first|morning|| +43182|AAAAAAAAPKIKAAAA|43182|11|59|42|AM|first|morning|| +43183|AAAAAAAAALIKAAAA|43183|11|59|43|AM|first|morning|| +43184|AAAAAAAABLIKAAAA|43184|11|59|44|AM|first|morning|| +43185|AAAAAAAACLIKAAAA|43185|11|59|45|AM|first|morning|| +43186|AAAAAAAADLIKAAAA|43186|11|59|46|AM|first|morning|| +43187|AAAAAAAAELIKAAAA|43187|11|59|47|AM|first|morning|| +43188|AAAAAAAAFLIKAAAA|43188|11|59|48|AM|first|morning|| +43189|AAAAAAAAGLIKAAAA|43189|11|59|49|AM|first|morning|| +43190|AAAAAAAAHLIKAAAA|43190|11|59|50|AM|first|morning|| +43191|AAAAAAAAILIKAAAA|43191|11|59|51|AM|first|morning|| +43192|AAAAAAAAJLIKAAAA|43192|11|59|52|AM|first|morning|| +43193|AAAAAAAAKLIKAAAA|43193|11|59|53|AM|first|morning|| +43194|AAAAAAAALLIKAAAA|43194|11|59|54|AM|first|morning|| +43195|AAAAAAAAMLIKAAAA|43195|11|59|55|AM|first|morning|| +43196|AAAAAAAANLIKAAAA|43196|11|59|56|AM|first|morning|| +43197|AAAAAAAAOLIKAAAA|43197|11|59|57|AM|first|morning|| +43198|AAAAAAAAPLIKAAAA|43198|11|59|58|AM|first|morning|| +43199|AAAAAAAAAMIKAAAA|43199|11|59|59|AM|first|morning|| +43200|AAAAAAAABMIKAAAA|43200|12|0|0|PM|first|afternoon|lunch| +43201|AAAAAAAACMIKAAAA|43201|12|0|1|PM|first|afternoon|lunch| +43202|AAAAAAAADMIKAAAA|43202|12|0|2|PM|first|afternoon|lunch| +43203|AAAAAAAAEMIKAAAA|43203|12|0|3|PM|first|afternoon|lunch| +43204|AAAAAAAAFMIKAAAA|43204|12|0|4|PM|first|afternoon|lunch| +43205|AAAAAAAAGMIKAAAA|43205|12|0|5|PM|first|afternoon|lunch| +43206|AAAAAAAAHMIKAAAA|43206|12|0|6|PM|first|afternoon|lunch| +43207|AAAAAAAAIMIKAAAA|43207|12|0|7|PM|first|afternoon|lunch| +43208|AAAAAAAAJMIKAAAA|43208|12|0|8|PM|first|afternoon|lunch| +43209|AAAAAAAAKMIKAAAA|43209|12|0|9|PM|first|afternoon|lunch| +43210|AAAAAAAALMIKAAAA|43210|12|0|10|PM|first|afternoon|lunch| +43211|AAAAAAAAMMIKAAAA|43211|12|0|11|PM|first|afternoon|lunch| +43212|AAAAAAAANMIKAAAA|43212|12|0|12|PM|first|afternoon|lunch| +43213|AAAAAAAAOMIKAAAA|43213|12|0|13|PM|first|afternoon|lunch| +43214|AAAAAAAAPMIKAAAA|43214|12|0|14|PM|first|afternoon|lunch| +43215|AAAAAAAAANIKAAAA|43215|12|0|15|PM|first|afternoon|lunch| +43216|AAAAAAAABNIKAAAA|43216|12|0|16|PM|first|afternoon|lunch| +43217|AAAAAAAACNIKAAAA|43217|12|0|17|PM|first|afternoon|lunch| +43218|AAAAAAAADNIKAAAA|43218|12|0|18|PM|first|afternoon|lunch| +43219|AAAAAAAAENIKAAAA|43219|12|0|19|PM|first|afternoon|lunch| +43220|AAAAAAAAFNIKAAAA|43220|12|0|20|PM|first|afternoon|lunch| +43221|AAAAAAAAGNIKAAAA|43221|12|0|21|PM|first|afternoon|lunch| +43222|AAAAAAAAHNIKAAAA|43222|12|0|22|PM|first|afternoon|lunch| +43223|AAAAAAAAINIKAAAA|43223|12|0|23|PM|first|afternoon|lunch| +43224|AAAAAAAAJNIKAAAA|43224|12|0|24|PM|first|afternoon|lunch| +43225|AAAAAAAAKNIKAAAA|43225|12|0|25|PM|first|afternoon|lunch| +43226|AAAAAAAALNIKAAAA|43226|12|0|26|PM|first|afternoon|lunch| +43227|AAAAAAAAMNIKAAAA|43227|12|0|27|PM|first|afternoon|lunch| +43228|AAAAAAAANNIKAAAA|43228|12|0|28|PM|first|afternoon|lunch| +43229|AAAAAAAAONIKAAAA|43229|12|0|29|PM|first|afternoon|lunch| +43230|AAAAAAAAPNIKAAAA|43230|12|0|30|PM|first|afternoon|lunch| +43231|AAAAAAAAAOIKAAAA|43231|12|0|31|PM|first|afternoon|lunch| +43232|AAAAAAAABOIKAAAA|43232|12|0|32|PM|first|afternoon|lunch| +43233|AAAAAAAACOIKAAAA|43233|12|0|33|PM|first|afternoon|lunch| +43234|AAAAAAAADOIKAAAA|43234|12|0|34|PM|first|afternoon|lunch| +43235|AAAAAAAAEOIKAAAA|43235|12|0|35|PM|first|afternoon|lunch| +43236|AAAAAAAAFOIKAAAA|43236|12|0|36|PM|first|afternoon|lunch| +43237|AAAAAAAAGOIKAAAA|43237|12|0|37|PM|first|afternoon|lunch| +43238|AAAAAAAAHOIKAAAA|43238|12|0|38|PM|first|afternoon|lunch| +43239|AAAAAAAAIOIKAAAA|43239|12|0|39|PM|first|afternoon|lunch| +43240|AAAAAAAAJOIKAAAA|43240|12|0|40|PM|first|afternoon|lunch| +43241|AAAAAAAAKOIKAAAA|43241|12|0|41|PM|first|afternoon|lunch| +43242|AAAAAAAALOIKAAAA|43242|12|0|42|PM|first|afternoon|lunch| +43243|AAAAAAAAMOIKAAAA|43243|12|0|43|PM|first|afternoon|lunch| +43244|AAAAAAAANOIKAAAA|43244|12|0|44|PM|first|afternoon|lunch| +43245|AAAAAAAAOOIKAAAA|43245|12|0|45|PM|first|afternoon|lunch| +43246|AAAAAAAAPOIKAAAA|43246|12|0|46|PM|first|afternoon|lunch| +43247|AAAAAAAAAPIKAAAA|43247|12|0|47|PM|first|afternoon|lunch| +43248|AAAAAAAABPIKAAAA|43248|12|0|48|PM|first|afternoon|lunch| +43249|AAAAAAAACPIKAAAA|43249|12|0|49|PM|first|afternoon|lunch| +43250|AAAAAAAADPIKAAAA|43250|12|0|50|PM|first|afternoon|lunch| +43251|AAAAAAAAEPIKAAAA|43251|12|0|51|PM|first|afternoon|lunch| +43252|AAAAAAAAFPIKAAAA|43252|12|0|52|PM|first|afternoon|lunch| +43253|AAAAAAAAGPIKAAAA|43253|12|0|53|PM|first|afternoon|lunch| +43254|AAAAAAAAHPIKAAAA|43254|12|0|54|PM|first|afternoon|lunch| +43255|AAAAAAAAIPIKAAAA|43255|12|0|55|PM|first|afternoon|lunch| +43256|AAAAAAAAJPIKAAAA|43256|12|0|56|PM|first|afternoon|lunch| +43257|AAAAAAAAKPIKAAAA|43257|12|0|57|PM|first|afternoon|lunch| +43258|AAAAAAAALPIKAAAA|43258|12|0|58|PM|first|afternoon|lunch| +43259|AAAAAAAAMPIKAAAA|43259|12|0|59|PM|first|afternoon|lunch| +43260|AAAAAAAANPIKAAAA|43260|12|1|0|PM|first|afternoon|lunch| +43261|AAAAAAAAOPIKAAAA|43261|12|1|1|PM|first|afternoon|lunch| +43262|AAAAAAAAPPIKAAAA|43262|12|1|2|PM|first|afternoon|lunch| +43263|AAAAAAAAAAJKAAAA|43263|12|1|3|PM|first|afternoon|lunch| +43264|AAAAAAAABAJKAAAA|43264|12|1|4|PM|first|afternoon|lunch| +43265|AAAAAAAACAJKAAAA|43265|12|1|5|PM|first|afternoon|lunch| +43266|AAAAAAAADAJKAAAA|43266|12|1|6|PM|first|afternoon|lunch| +43267|AAAAAAAAEAJKAAAA|43267|12|1|7|PM|first|afternoon|lunch| +43268|AAAAAAAAFAJKAAAA|43268|12|1|8|PM|first|afternoon|lunch| +43269|AAAAAAAAGAJKAAAA|43269|12|1|9|PM|first|afternoon|lunch| +43270|AAAAAAAAHAJKAAAA|43270|12|1|10|PM|first|afternoon|lunch| +43271|AAAAAAAAIAJKAAAA|43271|12|1|11|PM|first|afternoon|lunch| +43272|AAAAAAAAJAJKAAAA|43272|12|1|12|PM|first|afternoon|lunch| +43273|AAAAAAAAKAJKAAAA|43273|12|1|13|PM|first|afternoon|lunch| +43274|AAAAAAAALAJKAAAA|43274|12|1|14|PM|first|afternoon|lunch| +43275|AAAAAAAAMAJKAAAA|43275|12|1|15|PM|first|afternoon|lunch| +43276|AAAAAAAANAJKAAAA|43276|12|1|16|PM|first|afternoon|lunch| +43277|AAAAAAAAOAJKAAAA|43277|12|1|17|PM|first|afternoon|lunch| +43278|AAAAAAAAPAJKAAAA|43278|12|1|18|PM|first|afternoon|lunch| +43279|AAAAAAAAABJKAAAA|43279|12|1|19|PM|first|afternoon|lunch| +43280|AAAAAAAABBJKAAAA|43280|12|1|20|PM|first|afternoon|lunch| +43281|AAAAAAAACBJKAAAA|43281|12|1|21|PM|first|afternoon|lunch| +43282|AAAAAAAADBJKAAAA|43282|12|1|22|PM|first|afternoon|lunch| +43283|AAAAAAAAEBJKAAAA|43283|12|1|23|PM|first|afternoon|lunch| +43284|AAAAAAAAFBJKAAAA|43284|12|1|24|PM|first|afternoon|lunch| +43285|AAAAAAAAGBJKAAAA|43285|12|1|25|PM|first|afternoon|lunch| +43286|AAAAAAAAHBJKAAAA|43286|12|1|26|PM|first|afternoon|lunch| +43287|AAAAAAAAIBJKAAAA|43287|12|1|27|PM|first|afternoon|lunch| +43288|AAAAAAAAJBJKAAAA|43288|12|1|28|PM|first|afternoon|lunch| +43289|AAAAAAAAKBJKAAAA|43289|12|1|29|PM|first|afternoon|lunch| +43290|AAAAAAAALBJKAAAA|43290|12|1|30|PM|first|afternoon|lunch| +43291|AAAAAAAAMBJKAAAA|43291|12|1|31|PM|first|afternoon|lunch| +43292|AAAAAAAANBJKAAAA|43292|12|1|32|PM|first|afternoon|lunch| +43293|AAAAAAAAOBJKAAAA|43293|12|1|33|PM|first|afternoon|lunch| +43294|AAAAAAAAPBJKAAAA|43294|12|1|34|PM|first|afternoon|lunch| +43295|AAAAAAAAACJKAAAA|43295|12|1|35|PM|first|afternoon|lunch| +43296|AAAAAAAABCJKAAAA|43296|12|1|36|PM|first|afternoon|lunch| +43297|AAAAAAAACCJKAAAA|43297|12|1|37|PM|first|afternoon|lunch| +43298|AAAAAAAADCJKAAAA|43298|12|1|38|PM|first|afternoon|lunch| +43299|AAAAAAAAECJKAAAA|43299|12|1|39|PM|first|afternoon|lunch| +43300|AAAAAAAAFCJKAAAA|43300|12|1|40|PM|first|afternoon|lunch| +43301|AAAAAAAAGCJKAAAA|43301|12|1|41|PM|first|afternoon|lunch| +43302|AAAAAAAAHCJKAAAA|43302|12|1|42|PM|first|afternoon|lunch| +43303|AAAAAAAAICJKAAAA|43303|12|1|43|PM|first|afternoon|lunch| +43304|AAAAAAAAJCJKAAAA|43304|12|1|44|PM|first|afternoon|lunch| +43305|AAAAAAAAKCJKAAAA|43305|12|1|45|PM|first|afternoon|lunch| +43306|AAAAAAAALCJKAAAA|43306|12|1|46|PM|first|afternoon|lunch| +43307|AAAAAAAAMCJKAAAA|43307|12|1|47|PM|first|afternoon|lunch| +43308|AAAAAAAANCJKAAAA|43308|12|1|48|PM|first|afternoon|lunch| +43309|AAAAAAAAOCJKAAAA|43309|12|1|49|PM|first|afternoon|lunch| +43310|AAAAAAAAPCJKAAAA|43310|12|1|50|PM|first|afternoon|lunch| +43311|AAAAAAAAADJKAAAA|43311|12|1|51|PM|first|afternoon|lunch| +43312|AAAAAAAABDJKAAAA|43312|12|1|52|PM|first|afternoon|lunch| +43313|AAAAAAAACDJKAAAA|43313|12|1|53|PM|first|afternoon|lunch| +43314|AAAAAAAADDJKAAAA|43314|12|1|54|PM|first|afternoon|lunch| +43315|AAAAAAAAEDJKAAAA|43315|12|1|55|PM|first|afternoon|lunch| +43316|AAAAAAAAFDJKAAAA|43316|12|1|56|PM|first|afternoon|lunch| +43317|AAAAAAAAGDJKAAAA|43317|12|1|57|PM|first|afternoon|lunch| +43318|AAAAAAAAHDJKAAAA|43318|12|1|58|PM|first|afternoon|lunch| +43319|AAAAAAAAIDJKAAAA|43319|12|1|59|PM|first|afternoon|lunch| +43320|AAAAAAAAJDJKAAAA|43320|12|2|0|PM|first|afternoon|lunch| +43321|AAAAAAAAKDJKAAAA|43321|12|2|1|PM|first|afternoon|lunch| +43322|AAAAAAAALDJKAAAA|43322|12|2|2|PM|first|afternoon|lunch| +43323|AAAAAAAAMDJKAAAA|43323|12|2|3|PM|first|afternoon|lunch| +43324|AAAAAAAANDJKAAAA|43324|12|2|4|PM|first|afternoon|lunch| +43325|AAAAAAAAODJKAAAA|43325|12|2|5|PM|first|afternoon|lunch| +43326|AAAAAAAAPDJKAAAA|43326|12|2|6|PM|first|afternoon|lunch| +43327|AAAAAAAAAEJKAAAA|43327|12|2|7|PM|first|afternoon|lunch| +43328|AAAAAAAABEJKAAAA|43328|12|2|8|PM|first|afternoon|lunch| +43329|AAAAAAAACEJKAAAA|43329|12|2|9|PM|first|afternoon|lunch| +43330|AAAAAAAADEJKAAAA|43330|12|2|10|PM|first|afternoon|lunch| +43331|AAAAAAAAEEJKAAAA|43331|12|2|11|PM|first|afternoon|lunch| +43332|AAAAAAAAFEJKAAAA|43332|12|2|12|PM|first|afternoon|lunch| +43333|AAAAAAAAGEJKAAAA|43333|12|2|13|PM|first|afternoon|lunch| +43334|AAAAAAAAHEJKAAAA|43334|12|2|14|PM|first|afternoon|lunch| +43335|AAAAAAAAIEJKAAAA|43335|12|2|15|PM|first|afternoon|lunch| +43336|AAAAAAAAJEJKAAAA|43336|12|2|16|PM|first|afternoon|lunch| +43337|AAAAAAAAKEJKAAAA|43337|12|2|17|PM|first|afternoon|lunch| +43338|AAAAAAAALEJKAAAA|43338|12|2|18|PM|first|afternoon|lunch| +43339|AAAAAAAAMEJKAAAA|43339|12|2|19|PM|first|afternoon|lunch| +43340|AAAAAAAANEJKAAAA|43340|12|2|20|PM|first|afternoon|lunch| +43341|AAAAAAAAOEJKAAAA|43341|12|2|21|PM|first|afternoon|lunch| +43342|AAAAAAAAPEJKAAAA|43342|12|2|22|PM|first|afternoon|lunch| +43343|AAAAAAAAAFJKAAAA|43343|12|2|23|PM|first|afternoon|lunch| +43344|AAAAAAAABFJKAAAA|43344|12|2|24|PM|first|afternoon|lunch| +43345|AAAAAAAACFJKAAAA|43345|12|2|25|PM|first|afternoon|lunch| +43346|AAAAAAAADFJKAAAA|43346|12|2|26|PM|first|afternoon|lunch| +43347|AAAAAAAAEFJKAAAA|43347|12|2|27|PM|first|afternoon|lunch| +43348|AAAAAAAAFFJKAAAA|43348|12|2|28|PM|first|afternoon|lunch| +43349|AAAAAAAAGFJKAAAA|43349|12|2|29|PM|first|afternoon|lunch| +43350|AAAAAAAAHFJKAAAA|43350|12|2|30|PM|first|afternoon|lunch| +43351|AAAAAAAAIFJKAAAA|43351|12|2|31|PM|first|afternoon|lunch| +43352|AAAAAAAAJFJKAAAA|43352|12|2|32|PM|first|afternoon|lunch| +43353|AAAAAAAAKFJKAAAA|43353|12|2|33|PM|first|afternoon|lunch| +43354|AAAAAAAALFJKAAAA|43354|12|2|34|PM|first|afternoon|lunch| +43355|AAAAAAAAMFJKAAAA|43355|12|2|35|PM|first|afternoon|lunch| +43356|AAAAAAAANFJKAAAA|43356|12|2|36|PM|first|afternoon|lunch| +43357|AAAAAAAAOFJKAAAA|43357|12|2|37|PM|first|afternoon|lunch| +43358|AAAAAAAAPFJKAAAA|43358|12|2|38|PM|first|afternoon|lunch| +43359|AAAAAAAAAGJKAAAA|43359|12|2|39|PM|first|afternoon|lunch| +43360|AAAAAAAABGJKAAAA|43360|12|2|40|PM|first|afternoon|lunch| +43361|AAAAAAAACGJKAAAA|43361|12|2|41|PM|first|afternoon|lunch| +43362|AAAAAAAADGJKAAAA|43362|12|2|42|PM|first|afternoon|lunch| +43363|AAAAAAAAEGJKAAAA|43363|12|2|43|PM|first|afternoon|lunch| +43364|AAAAAAAAFGJKAAAA|43364|12|2|44|PM|first|afternoon|lunch| +43365|AAAAAAAAGGJKAAAA|43365|12|2|45|PM|first|afternoon|lunch| +43366|AAAAAAAAHGJKAAAA|43366|12|2|46|PM|first|afternoon|lunch| +43367|AAAAAAAAIGJKAAAA|43367|12|2|47|PM|first|afternoon|lunch| +43368|AAAAAAAAJGJKAAAA|43368|12|2|48|PM|first|afternoon|lunch| +43369|AAAAAAAAKGJKAAAA|43369|12|2|49|PM|first|afternoon|lunch| +43370|AAAAAAAALGJKAAAA|43370|12|2|50|PM|first|afternoon|lunch| +43371|AAAAAAAAMGJKAAAA|43371|12|2|51|PM|first|afternoon|lunch| +43372|AAAAAAAANGJKAAAA|43372|12|2|52|PM|first|afternoon|lunch| +43373|AAAAAAAAOGJKAAAA|43373|12|2|53|PM|first|afternoon|lunch| +43374|AAAAAAAAPGJKAAAA|43374|12|2|54|PM|first|afternoon|lunch| +43375|AAAAAAAAAHJKAAAA|43375|12|2|55|PM|first|afternoon|lunch| +43376|AAAAAAAABHJKAAAA|43376|12|2|56|PM|first|afternoon|lunch| +43377|AAAAAAAACHJKAAAA|43377|12|2|57|PM|first|afternoon|lunch| +43378|AAAAAAAADHJKAAAA|43378|12|2|58|PM|first|afternoon|lunch| +43379|AAAAAAAAEHJKAAAA|43379|12|2|59|PM|first|afternoon|lunch| +43380|AAAAAAAAFHJKAAAA|43380|12|3|0|PM|first|afternoon|lunch| +43381|AAAAAAAAGHJKAAAA|43381|12|3|1|PM|first|afternoon|lunch| +43382|AAAAAAAAHHJKAAAA|43382|12|3|2|PM|first|afternoon|lunch| +43383|AAAAAAAAIHJKAAAA|43383|12|3|3|PM|first|afternoon|lunch| +43384|AAAAAAAAJHJKAAAA|43384|12|3|4|PM|first|afternoon|lunch| +43385|AAAAAAAAKHJKAAAA|43385|12|3|5|PM|first|afternoon|lunch| +43386|AAAAAAAALHJKAAAA|43386|12|3|6|PM|first|afternoon|lunch| +43387|AAAAAAAAMHJKAAAA|43387|12|3|7|PM|first|afternoon|lunch| +43388|AAAAAAAANHJKAAAA|43388|12|3|8|PM|first|afternoon|lunch| +43389|AAAAAAAAOHJKAAAA|43389|12|3|9|PM|first|afternoon|lunch| +43390|AAAAAAAAPHJKAAAA|43390|12|3|10|PM|first|afternoon|lunch| +43391|AAAAAAAAAIJKAAAA|43391|12|3|11|PM|first|afternoon|lunch| +43392|AAAAAAAABIJKAAAA|43392|12|3|12|PM|first|afternoon|lunch| +43393|AAAAAAAACIJKAAAA|43393|12|3|13|PM|first|afternoon|lunch| +43394|AAAAAAAADIJKAAAA|43394|12|3|14|PM|first|afternoon|lunch| +43395|AAAAAAAAEIJKAAAA|43395|12|3|15|PM|first|afternoon|lunch| +43396|AAAAAAAAFIJKAAAA|43396|12|3|16|PM|first|afternoon|lunch| +43397|AAAAAAAAGIJKAAAA|43397|12|3|17|PM|first|afternoon|lunch| +43398|AAAAAAAAHIJKAAAA|43398|12|3|18|PM|first|afternoon|lunch| +43399|AAAAAAAAIIJKAAAA|43399|12|3|19|PM|first|afternoon|lunch| +43400|AAAAAAAAJIJKAAAA|43400|12|3|20|PM|first|afternoon|lunch| +43401|AAAAAAAAKIJKAAAA|43401|12|3|21|PM|first|afternoon|lunch| +43402|AAAAAAAALIJKAAAA|43402|12|3|22|PM|first|afternoon|lunch| +43403|AAAAAAAAMIJKAAAA|43403|12|3|23|PM|first|afternoon|lunch| +43404|AAAAAAAANIJKAAAA|43404|12|3|24|PM|first|afternoon|lunch| +43405|AAAAAAAAOIJKAAAA|43405|12|3|25|PM|first|afternoon|lunch| +43406|AAAAAAAAPIJKAAAA|43406|12|3|26|PM|first|afternoon|lunch| +43407|AAAAAAAAAJJKAAAA|43407|12|3|27|PM|first|afternoon|lunch| +43408|AAAAAAAABJJKAAAA|43408|12|3|28|PM|first|afternoon|lunch| +43409|AAAAAAAACJJKAAAA|43409|12|3|29|PM|first|afternoon|lunch| +43410|AAAAAAAADJJKAAAA|43410|12|3|30|PM|first|afternoon|lunch| +43411|AAAAAAAAEJJKAAAA|43411|12|3|31|PM|first|afternoon|lunch| +43412|AAAAAAAAFJJKAAAA|43412|12|3|32|PM|first|afternoon|lunch| +43413|AAAAAAAAGJJKAAAA|43413|12|3|33|PM|first|afternoon|lunch| +43414|AAAAAAAAHJJKAAAA|43414|12|3|34|PM|first|afternoon|lunch| +43415|AAAAAAAAIJJKAAAA|43415|12|3|35|PM|first|afternoon|lunch| +43416|AAAAAAAAJJJKAAAA|43416|12|3|36|PM|first|afternoon|lunch| +43417|AAAAAAAAKJJKAAAA|43417|12|3|37|PM|first|afternoon|lunch| +43418|AAAAAAAALJJKAAAA|43418|12|3|38|PM|first|afternoon|lunch| +43419|AAAAAAAAMJJKAAAA|43419|12|3|39|PM|first|afternoon|lunch| +43420|AAAAAAAANJJKAAAA|43420|12|3|40|PM|first|afternoon|lunch| +43421|AAAAAAAAOJJKAAAA|43421|12|3|41|PM|first|afternoon|lunch| +43422|AAAAAAAAPJJKAAAA|43422|12|3|42|PM|first|afternoon|lunch| +43423|AAAAAAAAAKJKAAAA|43423|12|3|43|PM|first|afternoon|lunch| +43424|AAAAAAAABKJKAAAA|43424|12|3|44|PM|first|afternoon|lunch| +43425|AAAAAAAACKJKAAAA|43425|12|3|45|PM|first|afternoon|lunch| +43426|AAAAAAAADKJKAAAA|43426|12|3|46|PM|first|afternoon|lunch| +43427|AAAAAAAAEKJKAAAA|43427|12|3|47|PM|first|afternoon|lunch| +43428|AAAAAAAAFKJKAAAA|43428|12|3|48|PM|first|afternoon|lunch| +43429|AAAAAAAAGKJKAAAA|43429|12|3|49|PM|first|afternoon|lunch| +43430|AAAAAAAAHKJKAAAA|43430|12|3|50|PM|first|afternoon|lunch| +43431|AAAAAAAAIKJKAAAA|43431|12|3|51|PM|first|afternoon|lunch| +43432|AAAAAAAAJKJKAAAA|43432|12|3|52|PM|first|afternoon|lunch| +43433|AAAAAAAAKKJKAAAA|43433|12|3|53|PM|first|afternoon|lunch| +43434|AAAAAAAALKJKAAAA|43434|12|3|54|PM|first|afternoon|lunch| +43435|AAAAAAAAMKJKAAAA|43435|12|3|55|PM|first|afternoon|lunch| +43436|AAAAAAAANKJKAAAA|43436|12|3|56|PM|first|afternoon|lunch| +43437|AAAAAAAAOKJKAAAA|43437|12|3|57|PM|first|afternoon|lunch| +43438|AAAAAAAAPKJKAAAA|43438|12|3|58|PM|first|afternoon|lunch| +43439|AAAAAAAAALJKAAAA|43439|12|3|59|PM|first|afternoon|lunch| +43440|AAAAAAAABLJKAAAA|43440|12|4|0|PM|first|afternoon|lunch| +43441|AAAAAAAACLJKAAAA|43441|12|4|1|PM|first|afternoon|lunch| +43442|AAAAAAAADLJKAAAA|43442|12|4|2|PM|first|afternoon|lunch| +43443|AAAAAAAAELJKAAAA|43443|12|4|3|PM|first|afternoon|lunch| +43444|AAAAAAAAFLJKAAAA|43444|12|4|4|PM|first|afternoon|lunch| +43445|AAAAAAAAGLJKAAAA|43445|12|4|5|PM|first|afternoon|lunch| +43446|AAAAAAAAHLJKAAAA|43446|12|4|6|PM|first|afternoon|lunch| +43447|AAAAAAAAILJKAAAA|43447|12|4|7|PM|first|afternoon|lunch| +43448|AAAAAAAAJLJKAAAA|43448|12|4|8|PM|first|afternoon|lunch| +43449|AAAAAAAAKLJKAAAA|43449|12|4|9|PM|first|afternoon|lunch| +43450|AAAAAAAALLJKAAAA|43450|12|4|10|PM|first|afternoon|lunch| +43451|AAAAAAAAMLJKAAAA|43451|12|4|11|PM|first|afternoon|lunch| +43452|AAAAAAAANLJKAAAA|43452|12|4|12|PM|first|afternoon|lunch| +43453|AAAAAAAAOLJKAAAA|43453|12|4|13|PM|first|afternoon|lunch| +43454|AAAAAAAAPLJKAAAA|43454|12|4|14|PM|first|afternoon|lunch| +43455|AAAAAAAAAMJKAAAA|43455|12|4|15|PM|first|afternoon|lunch| +43456|AAAAAAAABMJKAAAA|43456|12|4|16|PM|first|afternoon|lunch| +43457|AAAAAAAACMJKAAAA|43457|12|4|17|PM|first|afternoon|lunch| +43458|AAAAAAAADMJKAAAA|43458|12|4|18|PM|first|afternoon|lunch| +43459|AAAAAAAAEMJKAAAA|43459|12|4|19|PM|first|afternoon|lunch| +43460|AAAAAAAAFMJKAAAA|43460|12|4|20|PM|first|afternoon|lunch| +43461|AAAAAAAAGMJKAAAA|43461|12|4|21|PM|first|afternoon|lunch| +43462|AAAAAAAAHMJKAAAA|43462|12|4|22|PM|first|afternoon|lunch| +43463|AAAAAAAAIMJKAAAA|43463|12|4|23|PM|first|afternoon|lunch| +43464|AAAAAAAAJMJKAAAA|43464|12|4|24|PM|first|afternoon|lunch| +43465|AAAAAAAAKMJKAAAA|43465|12|4|25|PM|first|afternoon|lunch| +43466|AAAAAAAALMJKAAAA|43466|12|4|26|PM|first|afternoon|lunch| +43467|AAAAAAAAMMJKAAAA|43467|12|4|27|PM|first|afternoon|lunch| +43468|AAAAAAAANMJKAAAA|43468|12|4|28|PM|first|afternoon|lunch| +43469|AAAAAAAAOMJKAAAA|43469|12|4|29|PM|first|afternoon|lunch| +43470|AAAAAAAAPMJKAAAA|43470|12|4|30|PM|first|afternoon|lunch| +43471|AAAAAAAAANJKAAAA|43471|12|4|31|PM|first|afternoon|lunch| +43472|AAAAAAAABNJKAAAA|43472|12|4|32|PM|first|afternoon|lunch| +43473|AAAAAAAACNJKAAAA|43473|12|4|33|PM|first|afternoon|lunch| +43474|AAAAAAAADNJKAAAA|43474|12|4|34|PM|first|afternoon|lunch| +43475|AAAAAAAAENJKAAAA|43475|12|4|35|PM|first|afternoon|lunch| +43476|AAAAAAAAFNJKAAAA|43476|12|4|36|PM|first|afternoon|lunch| +43477|AAAAAAAAGNJKAAAA|43477|12|4|37|PM|first|afternoon|lunch| +43478|AAAAAAAAHNJKAAAA|43478|12|4|38|PM|first|afternoon|lunch| +43479|AAAAAAAAINJKAAAA|43479|12|4|39|PM|first|afternoon|lunch| +43480|AAAAAAAAJNJKAAAA|43480|12|4|40|PM|first|afternoon|lunch| +43481|AAAAAAAAKNJKAAAA|43481|12|4|41|PM|first|afternoon|lunch| +43482|AAAAAAAALNJKAAAA|43482|12|4|42|PM|first|afternoon|lunch| +43483|AAAAAAAAMNJKAAAA|43483|12|4|43|PM|first|afternoon|lunch| +43484|AAAAAAAANNJKAAAA|43484|12|4|44|PM|first|afternoon|lunch| +43485|AAAAAAAAONJKAAAA|43485|12|4|45|PM|first|afternoon|lunch| +43486|AAAAAAAAPNJKAAAA|43486|12|4|46|PM|first|afternoon|lunch| +43487|AAAAAAAAAOJKAAAA|43487|12|4|47|PM|first|afternoon|lunch| +43488|AAAAAAAABOJKAAAA|43488|12|4|48|PM|first|afternoon|lunch| +43489|AAAAAAAACOJKAAAA|43489|12|4|49|PM|first|afternoon|lunch| +43490|AAAAAAAADOJKAAAA|43490|12|4|50|PM|first|afternoon|lunch| +43491|AAAAAAAAEOJKAAAA|43491|12|4|51|PM|first|afternoon|lunch| +43492|AAAAAAAAFOJKAAAA|43492|12|4|52|PM|first|afternoon|lunch| +43493|AAAAAAAAGOJKAAAA|43493|12|4|53|PM|first|afternoon|lunch| +43494|AAAAAAAAHOJKAAAA|43494|12|4|54|PM|first|afternoon|lunch| +43495|AAAAAAAAIOJKAAAA|43495|12|4|55|PM|first|afternoon|lunch| +43496|AAAAAAAAJOJKAAAA|43496|12|4|56|PM|first|afternoon|lunch| +43497|AAAAAAAAKOJKAAAA|43497|12|4|57|PM|first|afternoon|lunch| +43498|AAAAAAAALOJKAAAA|43498|12|4|58|PM|first|afternoon|lunch| +43499|AAAAAAAAMOJKAAAA|43499|12|4|59|PM|first|afternoon|lunch| +43500|AAAAAAAANOJKAAAA|43500|12|5|0|PM|first|afternoon|lunch| +43501|AAAAAAAAOOJKAAAA|43501|12|5|1|PM|first|afternoon|lunch| +43502|AAAAAAAAPOJKAAAA|43502|12|5|2|PM|first|afternoon|lunch| +43503|AAAAAAAAAPJKAAAA|43503|12|5|3|PM|first|afternoon|lunch| +43504|AAAAAAAABPJKAAAA|43504|12|5|4|PM|first|afternoon|lunch| +43505|AAAAAAAACPJKAAAA|43505|12|5|5|PM|first|afternoon|lunch| +43506|AAAAAAAADPJKAAAA|43506|12|5|6|PM|first|afternoon|lunch| +43507|AAAAAAAAEPJKAAAA|43507|12|5|7|PM|first|afternoon|lunch| +43508|AAAAAAAAFPJKAAAA|43508|12|5|8|PM|first|afternoon|lunch| +43509|AAAAAAAAGPJKAAAA|43509|12|5|9|PM|first|afternoon|lunch| +43510|AAAAAAAAHPJKAAAA|43510|12|5|10|PM|first|afternoon|lunch| +43511|AAAAAAAAIPJKAAAA|43511|12|5|11|PM|first|afternoon|lunch| +43512|AAAAAAAAJPJKAAAA|43512|12|5|12|PM|first|afternoon|lunch| +43513|AAAAAAAAKPJKAAAA|43513|12|5|13|PM|first|afternoon|lunch| +43514|AAAAAAAALPJKAAAA|43514|12|5|14|PM|first|afternoon|lunch| +43515|AAAAAAAAMPJKAAAA|43515|12|5|15|PM|first|afternoon|lunch| +43516|AAAAAAAANPJKAAAA|43516|12|5|16|PM|first|afternoon|lunch| +43517|AAAAAAAAOPJKAAAA|43517|12|5|17|PM|first|afternoon|lunch| +43518|AAAAAAAAPPJKAAAA|43518|12|5|18|PM|first|afternoon|lunch| +43519|AAAAAAAAAAKKAAAA|43519|12|5|19|PM|first|afternoon|lunch| +43520|AAAAAAAABAKKAAAA|43520|12|5|20|PM|first|afternoon|lunch| +43521|AAAAAAAACAKKAAAA|43521|12|5|21|PM|first|afternoon|lunch| +43522|AAAAAAAADAKKAAAA|43522|12|5|22|PM|first|afternoon|lunch| +43523|AAAAAAAAEAKKAAAA|43523|12|5|23|PM|first|afternoon|lunch| +43524|AAAAAAAAFAKKAAAA|43524|12|5|24|PM|first|afternoon|lunch| +43525|AAAAAAAAGAKKAAAA|43525|12|5|25|PM|first|afternoon|lunch| +43526|AAAAAAAAHAKKAAAA|43526|12|5|26|PM|first|afternoon|lunch| +43527|AAAAAAAAIAKKAAAA|43527|12|5|27|PM|first|afternoon|lunch| +43528|AAAAAAAAJAKKAAAA|43528|12|5|28|PM|first|afternoon|lunch| +43529|AAAAAAAAKAKKAAAA|43529|12|5|29|PM|first|afternoon|lunch| +43530|AAAAAAAALAKKAAAA|43530|12|5|30|PM|first|afternoon|lunch| +43531|AAAAAAAAMAKKAAAA|43531|12|5|31|PM|first|afternoon|lunch| +43532|AAAAAAAANAKKAAAA|43532|12|5|32|PM|first|afternoon|lunch| +43533|AAAAAAAAOAKKAAAA|43533|12|5|33|PM|first|afternoon|lunch| +43534|AAAAAAAAPAKKAAAA|43534|12|5|34|PM|first|afternoon|lunch| +43535|AAAAAAAAABKKAAAA|43535|12|5|35|PM|first|afternoon|lunch| +43536|AAAAAAAABBKKAAAA|43536|12|5|36|PM|first|afternoon|lunch| +43537|AAAAAAAACBKKAAAA|43537|12|5|37|PM|first|afternoon|lunch| +43538|AAAAAAAADBKKAAAA|43538|12|5|38|PM|first|afternoon|lunch| +43539|AAAAAAAAEBKKAAAA|43539|12|5|39|PM|first|afternoon|lunch| +43540|AAAAAAAAFBKKAAAA|43540|12|5|40|PM|first|afternoon|lunch| +43541|AAAAAAAAGBKKAAAA|43541|12|5|41|PM|first|afternoon|lunch| +43542|AAAAAAAAHBKKAAAA|43542|12|5|42|PM|first|afternoon|lunch| +43543|AAAAAAAAIBKKAAAA|43543|12|5|43|PM|first|afternoon|lunch| +43544|AAAAAAAAJBKKAAAA|43544|12|5|44|PM|first|afternoon|lunch| +43545|AAAAAAAAKBKKAAAA|43545|12|5|45|PM|first|afternoon|lunch| +43546|AAAAAAAALBKKAAAA|43546|12|5|46|PM|first|afternoon|lunch| +43547|AAAAAAAAMBKKAAAA|43547|12|5|47|PM|first|afternoon|lunch| +43548|AAAAAAAANBKKAAAA|43548|12|5|48|PM|first|afternoon|lunch| +43549|AAAAAAAAOBKKAAAA|43549|12|5|49|PM|first|afternoon|lunch| +43550|AAAAAAAAPBKKAAAA|43550|12|5|50|PM|first|afternoon|lunch| +43551|AAAAAAAAACKKAAAA|43551|12|5|51|PM|first|afternoon|lunch| +43552|AAAAAAAABCKKAAAA|43552|12|5|52|PM|first|afternoon|lunch| +43553|AAAAAAAACCKKAAAA|43553|12|5|53|PM|first|afternoon|lunch| +43554|AAAAAAAADCKKAAAA|43554|12|5|54|PM|first|afternoon|lunch| +43555|AAAAAAAAECKKAAAA|43555|12|5|55|PM|first|afternoon|lunch| +43556|AAAAAAAAFCKKAAAA|43556|12|5|56|PM|first|afternoon|lunch| +43557|AAAAAAAAGCKKAAAA|43557|12|5|57|PM|first|afternoon|lunch| +43558|AAAAAAAAHCKKAAAA|43558|12|5|58|PM|first|afternoon|lunch| +43559|AAAAAAAAICKKAAAA|43559|12|5|59|PM|first|afternoon|lunch| +43560|AAAAAAAAJCKKAAAA|43560|12|6|0|PM|first|afternoon|lunch| +43561|AAAAAAAAKCKKAAAA|43561|12|6|1|PM|first|afternoon|lunch| +43562|AAAAAAAALCKKAAAA|43562|12|6|2|PM|first|afternoon|lunch| +43563|AAAAAAAAMCKKAAAA|43563|12|6|3|PM|first|afternoon|lunch| +43564|AAAAAAAANCKKAAAA|43564|12|6|4|PM|first|afternoon|lunch| +43565|AAAAAAAAOCKKAAAA|43565|12|6|5|PM|first|afternoon|lunch| +43566|AAAAAAAAPCKKAAAA|43566|12|6|6|PM|first|afternoon|lunch| +43567|AAAAAAAAADKKAAAA|43567|12|6|7|PM|first|afternoon|lunch| +43568|AAAAAAAABDKKAAAA|43568|12|6|8|PM|first|afternoon|lunch| +43569|AAAAAAAACDKKAAAA|43569|12|6|9|PM|first|afternoon|lunch| +43570|AAAAAAAADDKKAAAA|43570|12|6|10|PM|first|afternoon|lunch| +43571|AAAAAAAAEDKKAAAA|43571|12|6|11|PM|first|afternoon|lunch| +43572|AAAAAAAAFDKKAAAA|43572|12|6|12|PM|first|afternoon|lunch| +43573|AAAAAAAAGDKKAAAA|43573|12|6|13|PM|first|afternoon|lunch| +43574|AAAAAAAAHDKKAAAA|43574|12|6|14|PM|first|afternoon|lunch| +43575|AAAAAAAAIDKKAAAA|43575|12|6|15|PM|first|afternoon|lunch| +43576|AAAAAAAAJDKKAAAA|43576|12|6|16|PM|first|afternoon|lunch| +43577|AAAAAAAAKDKKAAAA|43577|12|6|17|PM|first|afternoon|lunch| +43578|AAAAAAAALDKKAAAA|43578|12|6|18|PM|first|afternoon|lunch| +43579|AAAAAAAAMDKKAAAA|43579|12|6|19|PM|first|afternoon|lunch| +43580|AAAAAAAANDKKAAAA|43580|12|6|20|PM|first|afternoon|lunch| +43581|AAAAAAAAODKKAAAA|43581|12|6|21|PM|first|afternoon|lunch| +43582|AAAAAAAAPDKKAAAA|43582|12|6|22|PM|first|afternoon|lunch| +43583|AAAAAAAAAEKKAAAA|43583|12|6|23|PM|first|afternoon|lunch| +43584|AAAAAAAABEKKAAAA|43584|12|6|24|PM|first|afternoon|lunch| +43585|AAAAAAAACEKKAAAA|43585|12|6|25|PM|first|afternoon|lunch| +43586|AAAAAAAADEKKAAAA|43586|12|6|26|PM|first|afternoon|lunch| +43587|AAAAAAAAEEKKAAAA|43587|12|6|27|PM|first|afternoon|lunch| +43588|AAAAAAAAFEKKAAAA|43588|12|6|28|PM|first|afternoon|lunch| +43589|AAAAAAAAGEKKAAAA|43589|12|6|29|PM|first|afternoon|lunch| +43590|AAAAAAAAHEKKAAAA|43590|12|6|30|PM|first|afternoon|lunch| +43591|AAAAAAAAIEKKAAAA|43591|12|6|31|PM|first|afternoon|lunch| +43592|AAAAAAAAJEKKAAAA|43592|12|6|32|PM|first|afternoon|lunch| +43593|AAAAAAAAKEKKAAAA|43593|12|6|33|PM|first|afternoon|lunch| +43594|AAAAAAAALEKKAAAA|43594|12|6|34|PM|first|afternoon|lunch| +43595|AAAAAAAAMEKKAAAA|43595|12|6|35|PM|first|afternoon|lunch| +43596|AAAAAAAANEKKAAAA|43596|12|6|36|PM|first|afternoon|lunch| +43597|AAAAAAAAOEKKAAAA|43597|12|6|37|PM|first|afternoon|lunch| +43598|AAAAAAAAPEKKAAAA|43598|12|6|38|PM|first|afternoon|lunch| +43599|AAAAAAAAAFKKAAAA|43599|12|6|39|PM|first|afternoon|lunch| +43600|AAAAAAAABFKKAAAA|43600|12|6|40|PM|first|afternoon|lunch| +43601|AAAAAAAACFKKAAAA|43601|12|6|41|PM|first|afternoon|lunch| +43602|AAAAAAAADFKKAAAA|43602|12|6|42|PM|first|afternoon|lunch| +43603|AAAAAAAAEFKKAAAA|43603|12|6|43|PM|first|afternoon|lunch| +43604|AAAAAAAAFFKKAAAA|43604|12|6|44|PM|first|afternoon|lunch| +43605|AAAAAAAAGFKKAAAA|43605|12|6|45|PM|first|afternoon|lunch| +43606|AAAAAAAAHFKKAAAA|43606|12|6|46|PM|first|afternoon|lunch| +43607|AAAAAAAAIFKKAAAA|43607|12|6|47|PM|first|afternoon|lunch| +43608|AAAAAAAAJFKKAAAA|43608|12|6|48|PM|first|afternoon|lunch| +43609|AAAAAAAAKFKKAAAA|43609|12|6|49|PM|first|afternoon|lunch| +43610|AAAAAAAALFKKAAAA|43610|12|6|50|PM|first|afternoon|lunch| +43611|AAAAAAAAMFKKAAAA|43611|12|6|51|PM|first|afternoon|lunch| +43612|AAAAAAAANFKKAAAA|43612|12|6|52|PM|first|afternoon|lunch| +43613|AAAAAAAAOFKKAAAA|43613|12|6|53|PM|first|afternoon|lunch| +43614|AAAAAAAAPFKKAAAA|43614|12|6|54|PM|first|afternoon|lunch| +43615|AAAAAAAAAGKKAAAA|43615|12|6|55|PM|first|afternoon|lunch| +43616|AAAAAAAABGKKAAAA|43616|12|6|56|PM|first|afternoon|lunch| +43617|AAAAAAAACGKKAAAA|43617|12|6|57|PM|first|afternoon|lunch| +43618|AAAAAAAADGKKAAAA|43618|12|6|58|PM|first|afternoon|lunch| +43619|AAAAAAAAEGKKAAAA|43619|12|6|59|PM|first|afternoon|lunch| +43620|AAAAAAAAFGKKAAAA|43620|12|7|0|PM|first|afternoon|lunch| +43621|AAAAAAAAGGKKAAAA|43621|12|7|1|PM|first|afternoon|lunch| +43622|AAAAAAAAHGKKAAAA|43622|12|7|2|PM|first|afternoon|lunch| +43623|AAAAAAAAIGKKAAAA|43623|12|7|3|PM|first|afternoon|lunch| +43624|AAAAAAAAJGKKAAAA|43624|12|7|4|PM|first|afternoon|lunch| +43625|AAAAAAAAKGKKAAAA|43625|12|7|5|PM|first|afternoon|lunch| +43626|AAAAAAAALGKKAAAA|43626|12|7|6|PM|first|afternoon|lunch| +43627|AAAAAAAAMGKKAAAA|43627|12|7|7|PM|first|afternoon|lunch| +43628|AAAAAAAANGKKAAAA|43628|12|7|8|PM|first|afternoon|lunch| +43629|AAAAAAAAOGKKAAAA|43629|12|7|9|PM|first|afternoon|lunch| +43630|AAAAAAAAPGKKAAAA|43630|12|7|10|PM|first|afternoon|lunch| +43631|AAAAAAAAAHKKAAAA|43631|12|7|11|PM|first|afternoon|lunch| +43632|AAAAAAAABHKKAAAA|43632|12|7|12|PM|first|afternoon|lunch| +43633|AAAAAAAACHKKAAAA|43633|12|7|13|PM|first|afternoon|lunch| +43634|AAAAAAAADHKKAAAA|43634|12|7|14|PM|first|afternoon|lunch| +43635|AAAAAAAAEHKKAAAA|43635|12|7|15|PM|first|afternoon|lunch| +43636|AAAAAAAAFHKKAAAA|43636|12|7|16|PM|first|afternoon|lunch| +43637|AAAAAAAAGHKKAAAA|43637|12|7|17|PM|first|afternoon|lunch| +43638|AAAAAAAAHHKKAAAA|43638|12|7|18|PM|first|afternoon|lunch| +43639|AAAAAAAAIHKKAAAA|43639|12|7|19|PM|first|afternoon|lunch| +43640|AAAAAAAAJHKKAAAA|43640|12|7|20|PM|first|afternoon|lunch| +43641|AAAAAAAAKHKKAAAA|43641|12|7|21|PM|first|afternoon|lunch| +43642|AAAAAAAALHKKAAAA|43642|12|7|22|PM|first|afternoon|lunch| +43643|AAAAAAAAMHKKAAAA|43643|12|7|23|PM|first|afternoon|lunch| +43644|AAAAAAAANHKKAAAA|43644|12|7|24|PM|first|afternoon|lunch| +43645|AAAAAAAAOHKKAAAA|43645|12|7|25|PM|first|afternoon|lunch| +43646|AAAAAAAAPHKKAAAA|43646|12|7|26|PM|first|afternoon|lunch| +43647|AAAAAAAAAIKKAAAA|43647|12|7|27|PM|first|afternoon|lunch| +43648|AAAAAAAABIKKAAAA|43648|12|7|28|PM|first|afternoon|lunch| +43649|AAAAAAAACIKKAAAA|43649|12|7|29|PM|first|afternoon|lunch| +43650|AAAAAAAADIKKAAAA|43650|12|7|30|PM|first|afternoon|lunch| +43651|AAAAAAAAEIKKAAAA|43651|12|7|31|PM|first|afternoon|lunch| +43652|AAAAAAAAFIKKAAAA|43652|12|7|32|PM|first|afternoon|lunch| +43653|AAAAAAAAGIKKAAAA|43653|12|7|33|PM|first|afternoon|lunch| +43654|AAAAAAAAHIKKAAAA|43654|12|7|34|PM|first|afternoon|lunch| +43655|AAAAAAAAIIKKAAAA|43655|12|7|35|PM|first|afternoon|lunch| +43656|AAAAAAAAJIKKAAAA|43656|12|7|36|PM|first|afternoon|lunch| +43657|AAAAAAAAKIKKAAAA|43657|12|7|37|PM|first|afternoon|lunch| +43658|AAAAAAAALIKKAAAA|43658|12|7|38|PM|first|afternoon|lunch| +43659|AAAAAAAAMIKKAAAA|43659|12|7|39|PM|first|afternoon|lunch| +43660|AAAAAAAANIKKAAAA|43660|12|7|40|PM|first|afternoon|lunch| +43661|AAAAAAAAOIKKAAAA|43661|12|7|41|PM|first|afternoon|lunch| +43662|AAAAAAAAPIKKAAAA|43662|12|7|42|PM|first|afternoon|lunch| +43663|AAAAAAAAAJKKAAAA|43663|12|7|43|PM|first|afternoon|lunch| +43664|AAAAAAAABJKKAAAA|43664|12|7|44|PM|first|afternoon|lunch| +43665|AAAAAAAACJKKAAAA|43665|12|7|45|PM|first|afternoon|lunch| +43666|AAAAAAAADJKKAAAA|43666|12|7|46|PM|first|afternoon|lunch| +43667|AAAAAAAAEJKKAAAA|43667|12|7|47|PM|first|afternoon|lunch| +43668|AAAAAAAAFJKKAAAA|43668|12|7|48|PM|first|afternoon|lunch| +43669|AAAAAAAAGJKKAAAA|43669|12|7|49|PM|first|afternoon|lunch| +43670|AAAAAAAAHJKKAAAA|43670|12|7|50|PM|first|afternoon|lunch| +43671|AAAAAAAAIJKKAAAA|43671|12|7|51|PM|first|afternoon|lunch| +43672|AAAAAAAAJJKKAAAA|43672|12|7|52|PM|first|afternoon|lunch| +43673|AAAAAAAAKJKKAAAA|43673|12|7|53|PM|first|afternoon|lunch| +43674|AAAAAAAALJKKAAAA|43674|12|7|54|PM|first|afternoon|lunch| +43675|AAAAAAAAMJKKAAAA|43675|12|7|55|PM|first|afternoon|lunch| +43676|AAAAAAAANJKKAAAA|43676|12|7|56|PM|first|afternoon|lunch| +43677|AAAAAAAAOJKKAAAA|43677|12|7|57|PM|first|afternoon|lunch| +43678|AAAAAAAAPJKKAAAA|43678|12|7|58|PM|first|afternoon|lunch| +43679|AAAAAAAAAKKKAAAA|43679|12|7|59|PM|first|afternoon|lunch| +43680|AAAAAAAABKKKAAAA|43680|12|8|0|PM|first|afternoon|lunch| +43681|AAAAAAAACKKKAAAA|43681|12|8|1|PM|first|afternoon|lunch| +43682|AAAAAAAADKKKAAAA|43682|12|8|2|PM|first|afternoon|lunch| +43683|AAAAAAAAEKKKAAAA|43683|12|8|3|PM|first|afternoon|lunch| +43684|AAAAAAAAFKKKAAAA|43684|12|8|4|PM|first|afternoon|lunch| +43685|AAAAAAAAGKKKAAAA|43685|12|8|5|PM|first|afternoon|lunch| +43686|AAAAAAAAHKKKAAAA|43686|12|8|6|PM|first|afternoon|lunch| +43687|AAAAAAAAIKKKAAAA|43687|12|8|7|PM|first|afternoon|lunch| +43688|AAAAAAAAJKKKAAAA|43688|12|8|8|PM|first|afternoon|lunch| +43689|AAAAAAAAKKKKAAAA|43689|12|8|9|PM|first|afternoon|lunch| +43690|AAAAAAAALKKKAAAA|43690|12|8|10|PM|first|afternoon|lunch| +43691|AAAAAAAAMKKKAAAA|43691|12|8|11|PM|first|afternoon|lunch| +43692|AAAAAAAANKKKAAAA|43692|12|8|12|PM|first|afternoon|lunch| +43693|AAAAAAAAOKKKAAAA|43693|12|8|13|PM|first|afternoon|lunch| +43694|AAAAAAAAPKKKAAAA|43694|12|8|14|PM|first|afternoon|lunch| +43695|AAAAAAAAALKKAAAA|43695|12|8|15|PM|first|afternoon|lunch| +43696|AAAAAAAABLKKAAAA|43696|12|8|16|PM|first|afternoon|lunch| +43697|AAAAAAAACLKKAAAA|43697|12|8|17|PM|first|afternoon|lunch| +43698|AAAAAAAADLKKAAAA|43698|12|8|18|PM|first|afternoon|lunch| +43699|AAAAAAAAELKKAAAA|43699|12|8|19|PM|first|afternoon|lunch| +43700|AAAAAAAAFLKKAAAA|43700|12|8|20|PM|first|afternoon|lunch| +43701|AAAAAAAAGLKKAAAA|43701|12|8|21|PM|first|afternoon|lunch| +43702|AAAAAAAAHLKKAAAA|43702|12|8|22|PM|first|afternoon|lunch| +43703|AAAAAAAAILKKAAAA|43703|12|8|23|PM|first|afternoon|lunch| +43704|AAAAAAAAJLKKAAAA|43704|12|8|24|PM|first|afternoon|lunch| +43705|AAAAAAAAKLKKAAAA|43705|12|8|25|PM|first|afternoon|lunch| +43706|AAAAAAAALLKKAAAA|43706|12|8|26|PM|first|afternoon|lunch| +43707|AAAAAAAAMLKKAAAA|43707|12|8|27|PM|first|afternoon|lunch| +43708|AAAAAAAANLKKAAAA|43708|12|8|28|PM|first|afternoon|lunch| +43709|AAAAAAAAOLKKAAAA|43709|12|8|29|PM|first|afternoon|lunch| +43710|AAAAAAAAPLKKAAAA|43710|12|8|30|PM|first|afternoon|lunch| +43711|AAAAAAAAAMKKAAAA|43711|12|8|31|PM|first|afternoon|lunch| +43712|AAAAAAAABMKKAAAA|43712|12|8|32|PM|first|afternoon|lunch| +43713|AAAAAAAACMKKAAAA|43713|12|8|33|PM|first|afternoon|lunch| +43714|AAAAAAAADMKKAAAA|43714|12|8|34|PM|first|afternoon|lunch| +43715|AAAAAAAAEMKKAAAA|43715|12|8|35|PM|first|afternoon|lunch| +43716|AAAAAAAAFMKKAAAA|43716|12|8|36|PM|first|afternoon|lunch| +43717|AAAAAAAAGMKKAAAA|43717|12|8|37|PM|first|afternoon|lunch| +43718|AAAAAAAAHMKKAAAA|43718|12|8|38|PM|first|afternoon|lunch| +43719|AAAAAAAAIMKKAAAA|43719|12|8|39|PM|first|afternoon|lunch| +43720|AAAAAAAAJMKKAAAA|43720|12|8|40|PM|first|afternoon|lunch| +43721|AAAAAAAAKMKKAAAA|43721|12|8|41|PM|first|afternoon|lunch| +43722|AAAAAAAALMKKAAAA|43722|12|8|42|PM|first|afternoon|lunch| +43723|AAAAAAAAMMKKAAAA|43723|12|8|43|PM|first|afternoon|lunch| +43724|AAAAAAAANMKKAAAA|43724|12|8|44|PM|first|afternoon|lunch| +43725|AAAAAAAAOMKKAAAA|43725|12|8|45|PM|first|afternoon|lunch| +43726|AAAAAAAAPMKKAAAA|43726|12|8|46|PM|first|afternoon|lunch| +43727|AAAAAAAAANKKAAAA|43727|12|8|47|PM|first|afternoon|lunch| +43728|AAAAAAAABNKKAAAA|43728|12|8|48|PM|first|afternoon|lunch| +43729|AAAAAAAACNKKAAAA|43729|12|8|49|PM|first|afternoon|lunch| +43730|AAAAAAAADNKKAAAA|43730|12|8|50|PM|first|afternoon|lunch| +43731|AAAAAAAAENKKAAAA|43731|12|8|51|PM|first|afternoon|lunch| +43732|AAAAAAAAFNKKAAAA|43732|12|8|52|PM|first|afternoon|lunch| +43733|AAAAAAAAGNKKAAAA|43733|12|8|53|PM|first|afternoon|lunch| +43734|AAAAAAAAHNKKAAAA|43734|12|8|54|PM|first|afternoon|lunch| +43735|AAAAAAAAINKKAAAA|43735|12|8|55|PM|first|afternoon|lunch| +43736|AAAAAAAAJNKKAAAA|43736|12|8|56|PM|first|afternoon|lunch| +43737|AAAAAAAAKNKKAAAA|43737|12|8|57|PM|first|afternoon|lunch| +43738|AAAAAAAALNKKAAAA|43738|12|8|58|PM|first|afternoon|lunch| +43739|AAAAAAAAMNKKAAAA|43739|12|8|59|PM|first|afternoon|lunch| +43740|AAAAAAAANNKKAAAA|43740|12|9|0|PM|first|afternoon|lunch| +43741|AAAAAAAAONKKAAAA|43741|12|9|1|PM|first|afternoon|lunch| +43742|AAAAAAAAPNKKAAAA|43742|12|9|2|PM|first|afternoon|lunch| +43743|AAAAAAAAAOKKAAAA|43743|12|9|3|PM|first|afternoon|lunch| +43744|AAAAAAAABOKKAAAA|43744|12|9|4|PM|first|afternoon|lunch| +43745|AAAAAAAACOKKAAAA|43745|12|9|5|PM|first|afternoon|lunch| +43746|AAAAAAAADOKKAAAA|43746|12|9|6|PM|first|afternoon|lunch| +43747|AAAAAAAAEOKKAAAA|43747|12|9|7|PM|first|afternoon|lunch| +43748|AAAAAAAAFOKKAAAA|43748|12|9|8|PM|first|afternoon|lunch| +43749|AAAAAAAAGOKKAAAA|43749|12|9|9|PM|first|afternoon|lunch| +43750|AAAAAAAAHOKKAAAA|43750|12|9|10|PM|first|afternoon|lunch| +43751|AAAAAAAAIOKKAAAA|43751|12|9|11|PM|first|afternoon|lunch| +43752|AAAAAAAAJOKKAAAA|43752|12|9|12|PM|first|afternoon|lunch| +43753|AAAAAAAAKOKKAAAA|43753|12|9|13|PM|first|afternoon|lunch| +43754|AAAAAAAALOKKAAAA|43754|12|9|14|PM|first|afternoon|lunch| +43755|AAAAAAAAMOKKAAAA|43755|12|9|15|PM|first|afternoon|lunch| +43756|AAAAAAAANOKKAAAA|43756|12|9|16|PM|first|afternoon|lunch| +43757|AAAAAAAAOOKKAAAA|43757|12|9|17|PM|first|afternoon|lunch| +43758|AAAAAAAAPOKKAAAA|43758|12|9|18|PM|first|afternoon|lunch| +43759|AAAAAAAAAPKKAAAA|43759|12|9|19|PM|first|afternoon|lunch| +43760|AAAAAAAABPKKAAAA|43760|12|9|20|PM|first|afternoon|lunch| +43761|AAAAAAAACPKKAAAA|43761|12|9|21|PM|first|afternoon|lunch| +43762|AAAAAAAADPKKAAAA|43762|12|9|22|PM|first|afternoon|lunch| +43763|AAAAAAAAEPKKAAAA|43763|12|9|23|PM|first|afternoon|lunch| +43764|AAAAAAAAFPKKAAAA|43764|12|9|24|PM|first|afternoon|lunch| +43765|AAAAAAAAGPKKAAAA|43765|12|9|25|PM|first|afternoon|lunch| +43766|AAAAAAAAHPKKAAAA|43766|12|9|26|PM|first|afternoon|lunch| +43767|AAAAAAAAIPKKAAAA|43767|12|9|27|PM|first|afternoon|lunch| +43768|AAAAAAAAJPKKAAAA|43768|12|9|28|PM|first|afternoon|lunch| +43769|AAAAAAAAKPKKAAAA|43769|12|9|29|PM|first|afternoon|lunch| +43770|AAAAAAAALPKKAAAA|43770|12|9|30|PM|first|afternoon|lunch| +43771|AAAAAAAAMPKKAAAA|43771|12|9|31|PM|first|afternoon|lunch| +43772|AAAAAAAANPKKAAAA|43772|12|9|32|PM|first|afternoon|lunch| +43773|AAAAAAAAOPKKAAAA|43773|12|9|33|PM|first|afternoon|lunch| +43774|AAAAAAAAPPKKAAAA|43774|12|9|34|PM|first|afternoon|lunch| +43775|AAAAAAAAAALKAAAA|43775|12|9|35|PM|first|afternoon|lunch| +43776|AAAAAAAABALKAAAA|43776|12|9|36|PM|first|afternoon|lunch| +43777|AAAAAAAACALKAAAA|43777|12|9|37|PM|first|afternoon|lunch| +43778|AAAAAAAADALKAAAA|43778|12|9|38|PM|first|afternoon|lunch| +43779|AAAAAAAAEALKAAAA|43779|12|9|39|PM|first|afternoon|lunch| +43780|AAAAAAAAFALKAAAA|43780|12|9|40|PM|first|afternoon|lunch| +43781|AAAAAAAAGALKAAAA|43781|12|9|41|PM|first|afternoon|lunch| +43782|AAAAAAAAHALKAAAA|43782|12|9|42|PM|first|afternoon|lunch| +43783|AAAAAAAAIALKAAAA|43783|12|9|43|PM|first|afternoon|lunch| +43784|AAAAAAAAJALKAAAA|43784|12|9|44|PM|first|afternoon|lunch| +43785|AAAAAAAAKALKAAAA|43785|12|9|45|PM|first|afternoon|lunch| +43786|AAAAAAAALALKAAAA|43786|12|9|46|PM|first|afternoon|lunch| +43787|AAAAAAAAMALKAAAA|43787|12|9|47|PM|first|afternoon|lunch| +43788|AAAAAAAANALKAAAA|43788|12|9|48|PM|first|afternoon|lunch| +43789|AAAAAAAAOALKAAAA|43789|12|9|49|PM|first|afternoon|lunch| +43790|AAAAAAAAPALKAAAA|43790|12|9|50|PM|first|afternoon|lunch| +43791|AAAAAAAAABLKAAAA|43791|12|9|51|PM|first|afternoon|lunch| +43792|AAAAAAAABBLKAAAA|43792|12|9|52|PM|first|afternoon|lunch| +43793|AAAAAAAACBLKAAAA|43793|12|9|53|PM|first|afternoon|lunch| +43794|AAAAAAAADBLKAAAA|43794|12|9|54|PM|first|afternoon|lunch| +43795|AAAAAAAAEBLKAAAA|43795|12|9|55|PM|first|afternoon|lunch| +43796|AAAAAAAAFBLKAAAA|43796|12|9|56|PM|first|afternoon|lunch| +43797|AAAAAAAAGBLKAAAA|43797|12|9|57|PM|first|afternoon|lunch| +43798|AAAAAAAAHBLKAAAA|43798|12|9|58|PM|first|afternoon|lunch| +43799|AAAAAAAAIBLKAAAA|43799|12|9|59|PM|first|afternoon|lunch| +43800|AAAAAAAAJBLKAAAA|43800|12|10|0|PM|first|afternoon|lunch| +43801|AAAAAAAAKBLKAAAA|43801|12|10|1|PM|first|afternoon|lunch| +43802|AAAAAAAALBLKAAAA|43802|12|10|2|PM|first|afternoon|lunch| +43803|AAAAAAAAMBLKAAAA|43803|12|10|3|PM|first|afternoon|lunch| +43804|AAAAAAAANBLKAAAA|43804|12|10|4|PM|first|afternoon|lunch| +43805|AAAAAAAAOBLKAAAA|43805|12|10|5|PM|first|afternoon|lunch| +43806|AAAAAAAAPBLKAAAA|43806|12|10|6|PM|first|afternoon|lunch| +43807|AAAAAAAAACLKAAAA|43807|12|10|7|PM|first|afternoon|lunch| +43808|AAAAAAAABCLKAAAA|43808|12|10|8|PM|first|afternoon|lunch| +43809|AAAAAAAACCLKAAAA|43809|12|10|9|PM|first|afternoon|lunch| +43810|AAAAAAAADCLKAAAA|43810|12|10|10|PM|first|afternoon|lunch| +43811|AAAAAAAAECLKAAAA|43811|12|10|11|PM|first|afternoon|lunch| +43812|AAAAAAAAFCLKAAAA|43812|12|10|12|PM|first|afternoon|lunch| +43813|AAAAAAAAGCLKAAAA|43813|12|10|13|PM|first|afternoon|lunch| +43814|AAAAAAAAHCLKAAAA|43814|12|10|14|PM|first|afternoon|lunch| +43815|AAAAAAAAICLKAAAA|43815|12|10|15|PM|first|afternoon|lunch| +43816|AAAAAAAAJCLKAAAA|43816|12|10|16|PM|first|afternoon|lunch| +43817|AAAAAAAAKCLKAAAA|43817|12|10|17|PM|first|afternoon|lunch| +43818|AAAAAAAALCLKAAAA|43818|12|10|18|PM|first|afternoon|lunch| +43819|AAAAAAAAMCLKAAAA|43819|12|10|19|PM|first|afternoon|lunch| +43820|AAAAAAAANCLKAAAA|43820|12|10|20|PM|first|afternoon|lunch| +43821|AAAAAAAAOCLKAAAA|43821|12|10|21|PM|first|afternoon|lunch| +43822|AAAAAAAAPCLKAAAA|43822|12|10|22|PM|first|afternoon|lunch| +43823|AAAAAAAAADLKAAAA|43823|12|10|23|PM|first|afternoon|lunch| +43824|AAAAAAAABDLKAAAA|43824|12|10|24|PM|first|afternoon|lunch| +43825|AAAAAAAACDLKAAAA|43825|12|10|25|PM|first|afternoon|lunch| +43826|AAAAAAAADDLKAAAA|43826|12|10|26|PM|first|afternoon|lunch| +43827|AAAAAAAAEDLKAAAA|43827|12|10|27|PM|first|afternoon|lunch| +43828|AAAAAAAAFDLKAAAA|43828|12|10|28|PM|first|afternoon|lunch| +43829|AAAAAAAAGDLKAAAA|43829|12|10|29|PM|first|afternoon|lunch| +43830|AAAAAAAAHDLKAAAA|43830|12|10|30|PM|first|afternoon|lunch| +43831|AAAAAAAAIDLKAAAA|43831|12|10|31|PM|first|afternoon|lunch| +43832|AAAAAAAAJDLKAAAA|43832|12|10|32|PM|first|afternoon|lunch| +43833|AAAAAAAAKDLKAAAA|43833|12|10|33|PM|first|afternoon|lunch| +43834|AAAAAAAALDLKAAAA|43834|12|10|34|PM|first|afternoon|lunch| +43835|AAAAAAAAMDLKAAAA|43835|12|10|35|PM|first|afternoon|lunch| +43836|AAAAAAAANDLKAAAA|43836|12|10|36|PM|first|afternoon|lunch| +43837|AAAAAAAAODLKAAAA|43837|12|10|37|PM|first|afternoon|lunch| +43838|AAAAAAAAPDLKAAAA|43838|12|10|38|PM|first|afternoon|lunch| +43839|AAAAAAAAAELKAAAA|43839|12|10|39|PM|first|afternoon|lunch| +43840|AAAAAAAABELKAAAA|43840|12|10|40|PM|first|afternoon|lunch| +43841|AAAAAAAACELKAAAA|43841|12|10|41|PM|first|afternoon|lunch| +43842|AAAAAAAADELKAAAA|43842|12|10|42|PM|first|afternoon|lunch| +43843|AAAAAAAAEELKAAAA|43843|12|10|43|PM|first|afternoon|lunch| +43844|AAAAAAAAFELKAAAA|43844|12|10|44|PM|first|afternoon|lunch| +43845|AAAAAAAAGELKAAAA|43845|12|10|45|PM|first|afternoon|lunch| +43846|AAAAAAAAHELKAAAA|43846|12|10|46|PM|first|afternoon|lunch| +43847|AAAAAAAAIELKAAAA|43847|12|10|47|PM|first|afternoon|lunch| +43848|AAAAAAAAJELKAAAA|43848|12|10|48|PM|first|afternoon|lunch| +43849|AAAAAAAAKELKAAAA|43849|12|10|49|PM|first|afternoon|lunch| +43850|AAAAAAAALELKAAAA|43850|12|10|50|PM|first|afternoon|lunch| +43851|AAAAAAAAMELKAAAA|43851|12|10|51|PM|first|afternoon|lunch| +43852|AAAAAAAANELKAAAA|43852|12|10|52|PM|first|afternoon|lunch| +43853|AAAAAAAAOELKAAAA|43853|12|10|53|PM|first|afternoon|lunch| +43854|AAAAAAAAPELKAAAA|43854|12|10|54|PM|first|afternoon|lunch| +43855|AAAAAAAAAFLKAAAA|43855|12|10|55|PM|first|afternoon|lunch| +43856|AAAAAAAABFLKAAAA|43856|12|10|56|PM|first|afternoon|lunch| +43857|AAAAAAAACFLKAAAA|43857|12|10|57|PM|first|afternoon|lunch| +43858|AAAAAAAADFLKAAAA|43858|12|10|58|PM|first|afternoon|lunch| +43859|AAAAAAAAEFLKAAAA|43859|12|10|59|PM|first|afternoon|lunch| +43860|AAAAAAAAFFLKAAAA|43860|12|11|0|PM|first|afternoon|lunch| +43861|AAAAAAAAGFLKAAAA|43861|12|11|1|PM|first|afternoon|lunch| +43862|AAAAAAAAHFLKAAAA|43862|12|11|2|PM|first|afternoon|lunch| +43863|AAAAAAAAIFLKAAAA|43863|12|11|3|PM|first|afternoon|lunch| +43864|AAAAAAAAJFLKAAAA|43864|12|11|4|PM|first|afternoon|lunch| +43865|AAAAAAAAKFLKAAAA|43865|12|11|5|PM|first|afternoon|lunch| +43866|AAAAAAAALFLKAAAA|43866|12|11|6|PM|first|afternoon|lunch| +43867|AAAAAAAAMFLKAAAA|43867|12|11|7|PM|first|afternoon|lunch| +43868|AAAAAAAANFLKAAAA|43868|12|11|8|PM|first|afternoon|lunch| +43869|AAAAAAAAOFLKAAAA|43869|12|11|9|PM|first|afternoon|lunch| +43870|AAAAAAAAPFLKAAAA|43870|12|11|10|PM|first|afternoon|lunch| +43871|AAAAAAAAAGLKAAAA|43871|12|11|11|PM|first|afternoon|lunch| +43872|AAAAAAAABGLKAAAA|43872|12|11|12|PM|first|afternoon|lunch| +43873|AAAAAAAACGLKAAAA|43873|12|11|13|PM|first|afternoon|lunch| +43874|AAAAAAAADGLKAAAA|43874|12|11|14|PM|first|afternoon|lunch| +43875|AAAAAAAAEGLKAAAA|43875|12|11|15|PM|first|afternoon|lunch| +43876|AAAAAAAAFGLKAAAA|43876|12|11|16|PM|first|afternoon|lunch| +43877|AAAAAAAAGGLKAAAA|43877|12|11|17|PM|first|afternoon|lunch| +43878|AAAAAAAAHGLKAAAA|43878|12|11|18|PM|first|afternoon|lunch| +43879|AAAAAAAAIGLKAAAA|43879|12|11|19|PM|first|afternoon|lunch| +43880|AAAAAAAAJGLKAAAA|43880|12|11|20|PM|first|afternoon|lunch| +43881|AAAAAAAAKGLKAAAA|43881|12|11|21|PM|first|afternoon|lunch| +43882|AAAAAAAALGLKAAAA|43882|12|11|22|PM|first|afternoon|lunch| +43883|AAAAAAAAMGLKAAAA|43883|12|11|23|PM|first|afternoon|lunch| +43884|AAAAAAAANGLKAAAA|43884|12|11|24|PM|first|afternoon|lunch| +43885|AAAAAAAAOGLKAAAA|43885|12|11|25|PM|first|afternoon|lunch| +43886|AAAAAAAAPGLKAAAA|43886|12|11|26|PM|first|afternoon|lunch| +43887|AAAAAAAAAHLKAAAA|43887|12|11|27|PM|first|afternoon|lunch| +43888|AAAAAAAABHLKAAAA|43888|12|11|28|PM|first|afternoon|lunch| +43889|AAAAAAAACHLKAAAA|43889|12|11|29|PM|first|afternoon|lunch| +43890|AAAAAAAADHLKAAAA|43890|12|11|30|PM|first|afternoon|lunch| +43891|AAAAAAAAEHLKAAAA|43891|12|11|31|PM|first|afternoon|lunch| +43892|AAAAAAAAFHLKAAAA|43892|12|11|32|PM|first|afternoon|lunch| +43893|AAAAAAAAGHLKAAAA|43893|12|11|33|PM|first|afternoon|lunch| +43894|AAAAAAAAHHLKAAAA|43894|12|11|34|PM|first|afternoon|lunch| +43895|AAAAAAAAIHLKAAAA|43895|12|11|35|PM|first|afternoon|lunch| +43896|AAAAAAAAJHLKAAAA|43896|12|11|36|PM|first|afternoon|lunch| +43897|AAAAAAAAKHLKAAAA|43897|12|11|37|PM|first|afternoon|lunch| +43898|AAAAAAAALHLKAAAA|43898|12|11|38|PM|first|afternoon|lunch| +43899|AAAAAAAAMHLKAAAA|43899|12|11|39|PM|first|afternoon|lunch| +43900|AAAAAAAANHLKAAAA|43900|12|11|40|PM|first|afternoon|lunch| +43901|AAAAAAAAOHLKAAAA|43901|12|11|41|PM|first|afternoon|lunch| +43902|AAAAAAAAPHLKAAAA|43902|12|11|42|PM|first|afternoon|lunch| +43903|AAAAAAAAAILKAAAA|43903|12|11|43|PM|first|afternoon|lunch| +43904|AAAAAAAABILKAAAA|43904|12|11|44|PM|first|afternoon|lunch| +43905|AAAAAAAACILKAAAA|43905|12|11|45|PM|first|afternoon|lunch| +43906|AAAAAAAADILKAAAA|43906|12|11|46|PM|first|afternoon|lunch| +43907|AAAAAAAAEILKAAAA|43907|12|11|47|PM|first|afternoon|lunch| +43908|AAAAAAAAFILKAAAA|43908|12|11|48|PM|first|afternoon|lunch| +43909|AAAAAAAAGILKAAAA|43909|12|11|49|PM|first|afternoon|lunch| +43910|AAAAAAAAHILKAAAA|43910|12|11|50|PM|first|afternoon|lunch| +43911|AAAAAAAAIILKAAAA|43911|12|11|51|PM|first|afternoon|lunch| +43912|AAAAAAAAJILKAAAA|43912|12|11|52|PM|first|afternoon|lunch| +43913|AAAAAAAAKILKAAAA|43913|12|11|53|PM|first|afternoon|lunch| +43914|AAAAAAAALILKAAAA|43914|12|11|54|PM|first|afternoon|lunch| +43915|AAAAAAAAMILKAAAA|43915|12|11|55|PM|first|afternoon|lunch| +43916|AAAAAAAANILKAAAA|43916|12|11|56|PM|first|afternoon|lunch| +43917|AAAAAAAAOILKAAAA|43917|12|11|57|PM|first|afternoon|lunch| +43918|AAAAAAAAPILKAAAA|43918|12|11|58|PM|first|afternoon|lunch| +43919|AAAAAAAAAJLKAAAA|43919|12|11|59|PM|first|afternoon|lunch| +43920|AAAAAAAABJLKAAAA|43920|12|12|0|PM|first|afternoon|lunch| +43921|AAAAAAAACJLKAAAA|43921|12|12|1|PM|first|afternoon|lunch| +43922|AAAAAAAADJLKAAAA|43922|12|12|2|PM|first|afternoon|lunch| +43923|AAAAAAAAEJLKAAAA|43923|12|12|3|PM|first|afternoon|lunch| +43924|AAAAAAAAFJLKAAAA|43924|12|12|4|PM|first|afternoon|lunch| +43925|AAAAAAAAGJLKAAAA|43925|12|12|5|PM|first|afternoon|lunch| +43926|AAAAAAAAHJLKAAAA|43926|12|12|6|PM|first|afternoon|lunch| +43927|AAAAAAAAIJLKAAAA|43927|12|12|7|PM|first|afternoon|lunch| +43928|AAAAAAAAJJLKAAAA|43928|12|12|8|PM|first|afternoon|lunch| +43929|AAAAAAAAKJLKAAAA|43929|12|12|9|PM|first|afternoon|lunch| +43930|AAAAAAAALJLKAAAA|43930|12|12|10|PM|first|afternoon|lunch| +43931|AAAAAAAAMJLKAAAA|43931|12|12|11|PM|first|afternoon|lunch| +43932|AAAAAAAANJLKAAAA|43932|12|12|12|PM|first|afternoon|lunch| +43933|AAAAAAAAOJLKAAAA|43933|12|12|13|PM|first|afternoon|lunch| +43934|AAAAAAAAPJLKAAAA|43934|12|12|14|PM|first|afternoon|lunch| +43935|AAAAAAAAAKLKAAAA|43935|12|12|15|PM|first|afternoon|lunch| +43936|AAAAAAAABKLKAAAA|43936|12|12|16|PM|first|afternoon|lunch| +43937|AAAAAAAACKLKAAAA|43937|12|12|17|PM|first|afternoon|lunch| +43938|AAAAAAAADKLKAAAA|43938|12|12|18|PM|first|afternoon|lunch| +43939|AAAAAAAAEKLKAAAA|43939|12|12|19|PM|first|afternoon|lunch| +43940|AAAAAAAAFKLKAAAA|43940|12|12|20|PM|first|afternoon|lunch| +43941|AAAAAAAAGKLKAAAA|43941|12|12|21|PM|first|afternoon|lunch| +43942|AAAAAAAAHKLKAAAA|43942|12|12|22|PM|first|afternoon|lunch| +43943|AAAAAAAAIKLKAAAA|43943|12|12|23|PM|first|afternoon|lunch| +43944|AAAAAAAAJKLKAAAA|43944|12|12|24|PM|first|afternoon|lunch| +43945|AAAAAAAAKKLKAAAA|43945|12|12|25|PM|first|afternoon|lunch| +43946|AAAAAAAALKLKAAAA|43946|12|12|26|PM|first|afternoon|lunch| +43947|AAAAAAAAMKLKAAAA|43947|12|12|27|PM|first|afternoon|lunch| +43948|AAAAAAAANKLKAAAA|43948|12|12|28|PM|first|afternoon|lunch| +43949|AAAAAAAAOKLKAAAA|43949|12|12|29|PM|first|afternoon|lunch| +43950|AAAAAAAAPKLKAAAA|43950|12|12|30|PM|first|afternoon|lunch| +43951|AAAAAAAAALLKAAAA|43951|12|12|31|PM|first|afternoon|lunch| +43952|AAAAAAAABLLKAAAA|43952|12|12|32|PM|first|afternoon|lunch| +43953|AAAAAAAACLLKAAAA|43953|12|12|33|PM|first|afternoon|lunch| +43954|AAAAAAAADLLKAAAA|43954|12|12|34|PM|first|afternoon|lunch| +43955|AAAAAAAAELLKAAAA|43955|12|12|35|PM|first|afternoon|lunch| +43956|AAAAAAAAFLLKAAAA|43956|12|12|36|PM|first|afternoon|lunch| +43957|AAAAAAAAGLLKAAAA|43957|12|12|37|PM|first|afternoon|lunch| +43958|AAAAAAAAHLLKAAAA|43958|12|12|38|PM|first|afternoon|lunch| +43959|AAAAAAAAILLKAAAA|43959|12|12|39|PM|first|afternoon|lunch| +43960|AAAAAAAAJLLKAAAA|43960|12|12|40|PM|first|afternoon|lunch| +43961|AAAAAAAAKLLKAAAA|43961|12|12|41|PM|first|afternoon|lunch| +43962|AAAAAAAALLLKAAAA|43962|12|12|42|PM|first|afternoon|lunch| +43963|AAAAAAAAMLLKAAAA|43963|12|12|43|PM|first|afternoon|lunch| +43964|AAAAAAAANLLKAAAA|43964|12|12|44|PM|first|afternoon|lunch| +43965|AAAAAAAAOLLKAAAA|43965|12|12|45|PM|first|afternoon|lunch| +43966|AAAAAAAAPLLKAAAA|43966|12|12|46|PM|first|afternoon|lunch| +43967|AAAAAAAAAMLKAAAA|43967|12|12|47|PM|first|afternoon|lunch| +43968|AAAAAAAABMLKAAAA|43968|12|12|48|PM|first|afternoon|lunch| +43969|AAAAAAAACMLKAAAA|43969|12|12|49|PM|first|afternoon|lunch| +43970|AAAAAAAADMLKAAAA|43970|12|12|50|PM|first|afternoon|lunch| +43971|AAAAAAAAEMLKAAAA|43971|12|12|51|PM|first|afternoon|lunch| +43972|AAAAAAAAFMLKAAAA|43972|12|12|52|PM|first|afternoon|lunch| +43973|AAAAAAAAGMLKAAAA|43973|12|12|53|PM|first|afternoon|lunch| +43974|AAAAAAAAHMLKAAAA|43974|12|12|54|PM|first|afternoon|lunch| +43975|AAAAAAAAIMLKAAAA|43975|12|12|55|PM|first|afternoon|lunch| +43976|AAAAAAAAJMLKAAAA|43976|12|12|56|PM|first|afternoon|lunch| +43977|AAAAAAAAKMLKAAAA|43977|12|12|57|PM|first|afternoon|lunch| +43978|AAAAAAAALMLKAAAA|43978|12|12|58|PM|first|afternoon|lunch| +43979|AAAAAAAAMMLKAAAA|43979|12|12|59|PM|first|afternoon|lunch| +43980|AAAAAAAANMLKAAAA|43980|12|13|0|PM|first|afternoon|lunch| +43981|AAAAAAAAOMLKAAAA|43981|12|13|1|PM|first|afternoon|lunch| +43982|AAAAAAAAPMLKAAAA|43982|12|13|2|PM|first|afternoon|lunch| +43983|AAAAAAAAANLKAAAA|43983|12|13|3|PM|first|afternoon|lunch| +43984|AAAAAAAABNLKAAAA|43984|12|13|4|PM|first|afternoon|lunch| +43985|AAAAAAAACNLKAAAA|43985|12|13|5|PM|first|afternoon|lunch| +43986|AAAAAAAADNLKAAAA|43986|12|13|6|PM|first|afternoon|lunch| +43987|AAAAAAAAENLKAAAA|43987|12|13|7|PM|first|afternoon|lunch| +43988|AAAAAAAAFNLKAAAA|43988|12|13|8|PM|first|afternoon|lunch| +43989|AAAAAAAAGNLKAAAA|43989|12|13|9|PM|first|afternoon|lunch| +43990|AAAAAAAAHNLKAAAA|43990|12|13|10|PM|first|afternoon|lunch| +43991|AAAAAAAAINLKAAAA|43991|12|13|11|PM|first|afternoon|lunch| +43992|AAAAAAAAJNLKAAAA|43992|12|13|12|PM|first|afternoon|lunch| +43993|AAAAAAAAKNLKAAAA|43993|12|13|13|PM|first|afternoon|lunch| +43994|AAAAAAAALNLKAAAA|43994|12|13|14|PM|first|afternoon|lunch| +43995|AAAAAAAAMNLKAAAA|43995|12|13|15|PM|first|afternoon|lunch| +43996|AAAAAAAANNLKAAAA|43996|12|13|16|PM|first|afternoon|lunch| +43997|AAAAAAAAONLKAAAA|43997|12|13|17|PM|first|afternoon|lunch| +43998|AAAAAAAAPNLKAAAA|43998|12|13|18|PM|first|afternoon|lunch| +43999|AAAAAAAAAOLKAAAA|43999|12|13|19|PM|first|afternoon|lunch| +44000|AAAAAAAABOLKAAAA|44000|12|13|20|PM|first|afternoon|lunch| +44001|AAAAAAAACOLKAAAA|44001|12|13|21|PM|first|afternoon|lunch| +44002|AAAAAAAADOLKAAAA|44002|12|13|22|PM|first|afternoon|lunch| +44003|AAAAAAAAEOLKAAAA|44003|12|13|23|PM|first|afternoon|lunch| +44004|AAAAAAAAFOLKAAAA|44004|12|13|24|PM|first|afternoon|lunch| +44005|AAAAAAAAGOLKAAAA|44005|12|13|25|PM|first|afternoon|lunch| +44006|AAAAAAAAHOLKAAAA|44006|12|13|26|PM|first|afternoon|lunch| +44007|AAAAAAAAIOLKAAAA|44007|12|13|27|PM|first|afternoon|lunch| +44008|AAAAAAAAJOLKAAAA|44008|12|13|28|PM|first|afternoon|lunch| +44009|AAAAAAAAKOLKAAAA|44009|12|13|29|PM|first|afternoon|lunch| +44010|AAAAAAAALOLKAAAA|44010|12|13|30|PM|first|afternoon|lunch| +44011|AAAAAAAAMOLKAAAA|44011|12|13|31|PM|first|afternoon|lunch| +44012|AAAAAAAANOLKAAAA|44012|12|13|32|PM|first|afternoon|lunch| +44013|AAAAAAAAOOLKAAAA|44013|12|13|33|PM|first|afternoon|lunch| +44014|AAAAAAAAPOLKAAAA|44014|12|13|34|PM|first|afternoon|lunch| +44015|AAAAAAAAAPLKAAAA|44015|12|13|35|PM|first|afternoon|lunch| +44016|AAAAAAAABPLKAAAA|44016|12|13|36|PM|first|afternoon|lunch| +44017|AAAAAAAACPLKAAAA|44017|12|13|37|PM|first|afternoon|lunch| +44018|AAAAAAAADPLKAAAA|44018|12|13|38|PM|first|afternoon|lunch| +44019|AAAAAAAAEPLKAAAA|44019|12|13|39|PM|first|afternoon|lunch| +44020|AAAAAAAAFPLKAAAA|44020|12|13|40|PM|first|afternoon|lunch| +44021|AAAAAAAAGPLKAAAA|44021|12|13|41|PM|first|afternoon|lunch| +44022|AAAAAAAAHPLKAAAA|44022|12|13|42|PM|first|afternoon|lunch| +44023|AAAAAAAAIPLKAAAA|44023|12|13|43|PM|first|afternoon|lunch| +44024|AAAAAAAAJPLKAAAA|44024|12|13|44|PM|first|afternoon|lunch| +44025|AAAAAAAAKPLKAAAA|44025|12|13|45|PM|first|afternoon|lunch| +44026|AAAAAAAALPLKAAAA|44026|12|13|46|PM|first|afternoon|lunch| +44027|AAAAAAAAMPLKAAAA|44027|12|13|47|PM|first|afternoon|lunch| +44028|AAAAAAAANPLKAAAA|44028|12|13|48|PM|first|afternoon|lunch| +44029|AAAAAAAAOPLKAAAA|44029|12|13|49|PM|first|afternoon|lunch| +44030|AAAAAAAAPPLKAAAA|44030|12|13|50|PM|first|afternoon|lunch| +44031|AAAAAAAAAAMKAAAA|44031|12|13|51|PM|first|afternoon|lunch| +44032|AAAAAAAABAMKAAAA|44032|12|13|52|PM|first|afternoon|lunch| +44033|AAAAAAAACAMKAAAA|44033|12|13|53|PM|first|afternoon|lunch| +44034|AAAAAAAADAMKAAAA|44034|12|13|54|PM|first|afternoon|lunch| +44035|AAAAAAAAEAMKAAAA|44035|12|13|55|PM|first|afternoon|lunch| +44036|AAAAAAAAFAMKAAAA|44036|12|13|56|PM|first|afternoon|lunch| +44037|AAAAAAAAGAMKAAAA|44037|12|13|57|PM|first|afternoon|lunch| +44038|AAAAAAAAHAMKAAAA|44038|12|13|58|PM|first|afternoon|lunch| +44039|AAAAAAAAIAMKAAAA|44039|12|13|59|PM|first|afternoon|lunch| +44040|AAAAAAAAJAMKAAAA|44040|12|14|0|PM|first|afternoon|lunch| +44041|AAAAAAAAKAMKAAAA|44041|12|14|1|PM|first|afternoon|lunch| +44042|AAAAAAAALAMKAAAA|44042|12|14|2|PM|first|afternoon|lunch| +44043|AAAAAAAAMAMKAAAA|44043|12|14|3|PM|first|afternoon|lunch| +44044|AAAAAAAANAMKAAAA|44044|12|14|4|PM|first|afternoon|lunch| +44045|AAAAAAAAOAMKAAAA|44045|12|14|5|PM|first|afternoon|lunch| +44046|AAAAAAAAPAMKAAAA|44046|12|14|6|PM|first|afternoon|lunch| +44047|AAAAAAAAABMKAAAA|44047|12|14|7|PM|first|afternoon|lunch| +44048|AAAAAAAABBMKAAAA|44048|12|14|8|PM|first|afternoon|lunch| +44049|AAAAAAAACBMKAAAA|44049|12|14|9|PM|first|afternoon|lunch| +44050|AAAAAAAADBMKAAAA|44050|12|14|10|PM|first|afternoon|lunch| +44051|AAAAAAAAEBMKAAAA|44051|12|14|11|PM|first|afternoon|lunch| +44052|AAAAAAAAFBMKAAAA|44052|12|14|12|PM|first|afternoon|lunch| +44053|AAAAAAAAGBMKAAAA|44053|12|14|13|PM|first|afternoon|lunch| +44054|AAAAAAAAHBMKAAAA|44054|12|14|14|PM|first|afternoon|lunch| +44055|AAAAAAAAIBMKAAAA|44055|12|14|15|PM|first|afternoon|lunch| +44056|AAAAAAAAJBMKAAAA|44056|12|14|16|PM|first|afternoon|lunch| +44057|AAAAAAAAKBMKAAAA|44057|12|14|17|PM|first|afternoon|lunch| +44058|AAAAAAAALBMKAAAA|44058|12|14|18|PM|first|afternoon|lunch| +44059|AAAAAAAAMBMKAAAA|44059|12|14|19|PM|first|afternoon|lunch| +44060|AAAAAAAANBMKAAAA|44060|12|14|20|PM|first|afternoon|lunch| +44061|AAAAAAAAOBMKAAAA|44061|12|14|21|PM|first|afternoon|lunch| +44062|AAAAAAAAPBMKAAAA|44062|12|14|22|PM|first|afternoon|lunch| +44063|AAAAAAAAACMKAAAA|44063|12|14|23|PM|first|afternoon|lunch| +44064|AAAAAAAABCMKAAAA|44064|12|14|24|PM|first|afternoon|lunch| +44065|AAAAAAAACCMKAAAA|44065|12|14|25|PM|first|afternoon|lunch| +44066|AAAAAAAADCMKAAAA|44066|12|14|26|PM|first|afternoon|lunch| +44067|AAAAAAAAECMKAAAA|44067|12|14|27|PM|first|afternoon|lunch| +44068|AAAAAAAAFCMKAAAA|44068|12|14|28|PM|first|afternoon|lunch| +44069|AAAAAAAAGCMKAAAA|44069|12|14|29|PM|first|afternoon|lunch| +44070|AAAAAAAAHCMKAAAA|44070|12|14|30|PM|first|afternoon|lunch| +44071|AAAAAAAAICMKAAAA|44071|12|14|31|PM|first|afternoon|lunch| +44072|AAAAAAAAJCMKAAAA|44072|12|14|32|PM|first|afternoon|lunch| +44073|AAAAAAAAKCMKAAAA|44073|12|14|33|PM|first|afternoon|lunch| +44074|AAAAAAAALCMKAAAA|44074|12|14|34|PM|first|afternoon|lunch| +44075|AAAAAAAAMCMKAAAA|44075|12|14|35|PM|first|afternoon|lunch| +44076|AAAAAAAANCMKAAAA|44076|12|14|36|PM|first|afternoon|lunch| +44077|AAAAAAAAOCMKAAAA|44077|12|14|37|PM|first|afternoon|lunch| +44078|AAAAAAAAPCMKAAAA|44078|12|14|38|PM|first|afternoon|lunch| +44079|AAAAAAAAADMKAAAA|44079|12|14|39|PM|first|afternoon|lunch| +44080|AAAAAAAABDMKAAAA|44080|12|14|40|PM|first|afternoon|lunch| +44081|AAAAAAAACDMKAAAA|44081|12|14|41|PM|first|afternoon|lunch| +44082|AAAAAAAADDMKAAAA|44082|12|14|42|PM|first|afternoon|lunch| +44083|AAAAAAAAEDMKAAAA|44083|12|14|43|PM|first|afternoon|lunch| +44084|AAAAAAAAFDMKAAAA|44084|12|14|44|PM|first|afternoon|lunch| +44085|AAAAAAAAGDMKAAAA|44085|12|14|45|PM|first|afternoon|lunch| +44086|AAAAAAAAHDMKAAAA|44086|12|14|46|PM|first|afternoon|lunch| +44087|AAAAAAAAIDMKAAAA|44087|12|14|47|PM|first|afternoon|lunch| +44088|AAAAAAAAJDMKAAAA|44088|12|14|48|PM|first|afternoon|lunch| +44089|AAAAAAAAKDMKAAAA|44089|12|14|49|PM|first|afternoon|lunch| +44090|AAAAAAAALDMKAAAA|44090|12|14|50|PM|first|afternoon|lunch| +44091|AAAAAAAAMDMKAAAA|44091|12|14|51|PM|first|afternoon|lunch| +44092|AAAAAAAANDMKAAAA|44092|12|14|52|PM|first|afternoon|lunch| +44093|AAAAAAAAODMKAAAA|44093|12|14|53|PM|first|afternoon|lunch| +44094|AAAAAAAAPDMKAAAA|44094|12|14|54|PM|first|afternoon|lunch| +44095|AAAAAAAAAEMKAAAA|44095|12|14|55|PM|first|afternoon|lunch| +44096|AAAAAAAABEMKAAAA|44096|12|14|56|PM|first|afternoon|lunch| +44097|AAAAAAAACEMKAAAA|44097|12|14|57|PM|first|afternoon|lunch| +44098|AAAAAAAADEMKAAAA|44098|12|14|58|PM|first|afternoon|lunch| +44099|AAAAAAAAEEMKAAAA|44099|12|14|59|PM|first|afternoon|lunch| +44100|AAAAAAAAFEMKAAAA|44100|12|15|0|PM|first|afternoon|lunch| +44101|AAAAAAAAGEMKAAAA|44101|12|15|1|PM|first|afternoon|lunch| +44102|AAAAAAAAHEMKAAAA|44102|12|15|2|PM|first|afternoon|lunch| +44103|AAAAAAAAIEMKAAAA|44103|12|15|3|PM|first|afternoon|lunch| +44104|AAAAAAAAJEMKAAAA|44104|12|15|4|PM|first|afternoon|lunch| +44105|AAAAAAAAKEMKAAAA|44105|12|15|5|PM|first|afternoon|lunch| +44106|AAAAAAAALEMKAAAA|44106|12|15|6|PM|first|afternoon|lunch| +44107|AAAAAAAAMEMKAAAA|44107|12|15|7|PM|first|afternoon|lunch| +44108|AAAAAAAANEMKAAAA|44108|12|15|8|PM|first|afternoon|lunch| +44109|AAAAAAAAOEMKAAAA|44109|12|15|9|PM|first|afternoon|lunch| +44110|AAAAAAAAPEMKAAAA|44110|12|15|10|PM|first|afternoon|lunch| +44111|AAAAAAAAAFMKAAAA|44111|12|15|11|PM|first|afternoon|lunch| +44112|AAAAAAAABFMKAAAA|44112|12|15|12|PM|first|afternoon|lunch| +44113|AAAAAAAACFMKAAAA|44113|12|15|13|PM|first|afternoon|lunch| +44114|AAAAAAAADFMKAAAA|44114|12|15|14|PM|first|afternoon|lunch| +44115|AAAAAAAAEFMKAAAA|44115|12|15|15|PM|first|afternoon|lunch| +44116|AAAAAAAAFFMKAAAA|44116|12|15|16|PM|first|afternoon|lunch| +44117|AAAAAAAAGFMKAAAA|44117|12|15|17|PM|first|afternoon|lunch| +44118|AAAAAAAAHFMKAAAA|44118|12|15|18|PM|first|afternoon|lunch| +44119|AAAAAAAAIFMKAAAA|44119|12|15|19|PM|first|afternoon|lunch| +44120|AAAAAAAAJFMKAAAA|44120|12|15|20|PM|first|afternoon|lunch| +44121|AAAAAAAAKFMKAAAA|44121|12|15|21|PM|first|afternoon|lunch| +44122|AAAAAAAALFMKAAAA|44122|12|15|22|PM|first|afternoon|lunch| +44123|AAAAAAAAMFMKAAAA|44123|12|15|23|PM|first|afternoon|lunch| +44124|AAAAAAAANFMKAAAA|44124|12|15|24|PM|first|afternoon|lunch| +44125|AAAAAAAAOFMKAAAA|44125|12|15|25|PM|first|afternoon|lunch| +44126|AAAAAAAAPFMKAAAA|44126|12|15|26|PM|first|afternoon|lunch| +44127|AAAAAAAAAGMKAAAA|44127|12|15|27|PM|first|afternoon|lunch| +44128|AAAAAAAABGMKAAAA|44128|12|15|28|PM|first|afternoon|lunch| +44129|AAAAAAAACGMKAAAA|44129|12|15|29|PM|first|afternoon|lunch| +44130|AAAAAAAADGMKAAAA|44130|12|15|30|PM|first|afternoon|lunch| +44131|AAAAAAAAEGMKAAAA|44131|12|15|31|PM|first|afternoon|lunch| +44132|AAAAAAAAFGMKAAAA|44132|12|15|32|PM|first|afternoon|lunch| +44133|AAAAAAAAGGMKAAAA|44133|12|15|33|PM|first|afternoon|lunch| +44134|AAAAAAAAHGMKAAAA|44134|12|15|34|PM|first|afternoon|lunch| +44135|AAAAAAAAIGMKAAAA|44135|12|15|35|PM|first|afternoon|lunch| +44136|AAAAAAAAJGMKAAAA|44136|12|15|36|PM|first|afternoon|lunch| +44137|AAAAAAAAKGMKAAAA|44137|12|15|37|PM|first|afternoon|lunch| +44138|AAAAAAAALGMKAAAA|44138|12|15|38|PM|first|afternoon|lunch| +44139|AAAAAAAAMGMKAAAA|44139|12|15|39|PM|first|afternoon|lunch| +44140|AAAAAAAANGMKAAAA|44140|12|15|40|PM|first|afternoon|lunch| +44141|AAAAAAAAOGMKAAAA|44141|12|15|41|PM|first|afternoon|lunch| +44142|AAAAAAAAPGMKAAAA|44142|12|15|42|PM|first|afternoon|lunch| +44143|AAAAAAAAAHMKAAAA|44143|12|15|43|PM|first|afternoon|lunch| +44144|AAAAAAAABHMKAAAA|44144|12|15|44|PM|first|afternoon|lunch| +44145|AAAAAAAACHMKAAAA|44145|12|15|45|PM|first|afternoon|lunch| +44146|AAAAAAAADHMKAAAA|44146|12|15|46|PM|first|afternoon|lunch| +44147|AAAAAAAAEHMKAAAA|44147|12|15|47|PM|first|afternoon|lunch| +44148|AAAAAAAAFHMKAAAA|44148|12|15|48|PM|first|afternoon|lunch| +44149|AAAAAAAAGHMKAAAA|44149|12|15|49|PM|first|afternoon|lunch| +44150|AAAAAAAAHHMKAAAA|44150|12|15|50|PM|first|afternoon|lunch| +44151|AAAAAAAAIHMKAAAA|44151|12|15|51|PM|first|afternoon|lunch| +44152|AAAAAAAAJHMKAAAA|44152|12|15|52|PM|first|afternoon|lunch| +44153|AAAAAAAAKHMKAAAA|44153|12|15|53|PM|first|afternoon|lunch| +44154|AAAAAAAALHMKAAAA|44154|12|15|54|PM|first|afternoon|lunch| +44155|AAAAAAAAMHMKAAAA|44155|12|15|55|PM|first|afternoon|lunch| +44156|AAAAAAAANHMKAAAA|44156|12|15|56|PM|first|afternoon|lunch| +44157|AAAAAAAAOHMKAAAA|44157|12|15|57|PM|first|afternoon|lunch| +44158|AAAAAAAAPHMKAAAA|44158|12|15|58|PM|first|afternoon|lunch| +44159|AAAAAAAAAIMKAAAA|44159|12|15|59|PM|first|afternoon|lunch| +44160|AAAAAAAABIMKAAAA|44160|12|16|0|PM|first|afternoon|lunch| +44161|AAAAAAAACIMKAAAA|44161|12|16|1|PM|first|afternoon|lunch| +44162|AAAAAAAADIMKAAAA|44162|12|16|2|PM|first|afternoon|lunch| +44163|AAAAAAAAEIMKAAAA|44163|12|16|3|PM|first|afternoon|lunch| +44164|AAAAAAAAFIMKAAAA|44164|12|16|4|PM|first|afternoon|lunch| +44165|AAAAAAAAGIMKAAAA|44165|12|16|5|PM|first|afternoon|lunch| +44166|AAAAAAAAHIMKAAAA|44166|12|16|6|PM|first|afternoon|lunch| +44167|AAAAAAAAIIMKAAAA|44167|12|16|7|PM|first|afternoon|lunch| +44168|AAAAAAAAJIMKAAAA|44168|12|16|8|PM|first|afternoon|lunch| +44169|AAAAAAAAKIMKAAAA|44169|12|16|9|PM|first|afternoon|lunch| +44170|AAAAAAAALIMKAAAA|44170|12|16|10|PM|first|afternoon|lunch| +44171|AAAAAAAAMIMKAAAA|44171|12|16|11|PM|first|afternoon|lunch| +44172|AAAAAAAANIMKAAAA|44172|12|16|12|PM|first|afternoon|lunch| +44173|AAAAAAAAOIMKAAAA|44173|12|16|13|PM|first|afternoon|lunch| +44174|AAAAAAAAPIMKAAAA|44174|12|16|14|PM|first|afternoon|lunch| +44175|AAAAAAAAAJMKAAAA|44175|12|16|15|PM|first|afternoon|lunch| +44176|AAAAAAAABJMKAAAA|44176|12|16|16|PM|first|afternoon|lunch| +44177|AAAAAAAACJMKAAAA|44177|12|16|17|PM|first|afternoon|lunch| +44178|AAAAAAAADJMKAAAA|44178|12|16|18|PM|first|afternoon|lunch| +44179|AAAAAAAAEJMKAAAA|44179|12|16|19|PM|first|afternoon|lunch| +44180|AAAAAAAAFJMKAAAA|44180|12|16|20|PM|first|afternoon|lunch| +44181|AAAAAAAAGJMKAAAA|44181|12|16|21|PM|first|afternoon|lunch| +44182|AAAAAAAAHJMKAAAA|44182|12|16|22|PM|first|afternoon|lunch| +44183|AAAAAAAAIJMKAAAA|44183|12|16|23|PM|first|afternoon|lunch| +44184|AAAAAAAAJJMKAAAA|44184|12|16|24|PM|first|afternoon|lunch| +44185|AAAAAAAAKJMKAAAA|44185|12|16|25|PM|first|afternoon|lunch| +44186|AAAAAAAALJMKAAAA|44186|12|16|26|PM|first|afternoon|lunch| +44187|AAAAAAAAMJMKAAAA|44187|12|16|27|PM|first|afternoon|lunch| +44188|AAAAAAAANJMKAAAA|44188|12|16|28|PM|first|afternoon|lunch| +44189|AAAAAAAAOJMKAAAA|44189|12|16|29|PM|first|afternoon|lunch| +44190|AAAAAAAAPJMKAAAA|44190|12|16|30|PM|first|afternoon|lunch| +44191|AAAAAAAAAKMKAAAA|44191|12|16|31|PM|first|afternoon|lunch| +44192|AAAAAAAABKMKAAAA|44192|12|16|32|PM|first|afternoon|lunch| +44193|AAAAAAAACKMKAAAA|44193|12|16|33|PM|first|afternoon|lunch| +44194|AAAAAAAADKMKAAAA|44194|12|16|34|PM|first|afternoon|lunch| +44195|AAAAAAAAEKMKAAAA|44195|12|16|35|PM|first|afternoon|lunch| +44196|AAAAAAAAFKMKAAAA|44196|12|16|36|PM|first|afternoon|lunch| +44197|AAAAAAAAGKMKAAAA|44197|12|16|37|PM|first|afternoon|lunch| +44198|AAAAAAAAHKMKAAAA|44198|12|16|38|PM|first|afternoon|lunch| +44199|AAAAAAAAIKMKAAAA|44199|12|16|39|PM|first|afternoon|lunch| +44200|AAAAAAAAJKMKAAAA|44200|12|16|40|PM|first|afternoon|lunch| +44201|AAAAAAAAKKMKAAAA|44201|12|16|41|PM|first|afternoon|lunch| +44202|AAAAAAAALKMKAAAA|44202|12|16|42|PM|first|afternoon|lunch| +44203|AAAAAAAAMKMKAAAA|44203|12|16|43|PM|first|afternoon|lunch| +44204|AAAAAAAANKMKAAAA|44204|12|16|44|PM|first|afternoon|lunch| +44205|AAAAAAAAOKMKAAAA|44205|12|16|45|PM|first|afternoon|lunch| +44206|AAAAAAAAPKMKAAAA|44206|12|16|46|PM|first|afternoon|lunch| +44207|AAAAAAAAALMKAAAA|44207|12|16|47|PM|first|afternoon|lunch| +44208|AAAAAAAABLMKAAAA|44208|12|16|48|PM|first|afternoon|lunch| +44209|AAAAAAAACLMKAAAA|44209|12|16|49|PM|first|afternoon|lunch| +44210|AAAAAAAADLMKAAAA|44210|12|16|50|PM|first|afternoon|lunch| +44211|AAAAAAAAELMKAAAA|44211|12|16|51|PM|first|afternoon|lunch| +44212|AAAAAAAAFLMKAAAA|44212|12|16|52|PM|first|afternoon|lunch| +44213|AAAAAAAAGLMKAAAA|44213|12|16|53|PM|first|afternoon|lunch| +44214|AAAAAAAAHLMKAAAA|44214|12|16|54|PM|first|afternoon|lunch| +44215|AAAAAAAAILMKAAAA|44215|12|16|55|PM|first|afternoon|lunch| +44216|AAAAAAAAJLMKAAAA|44216|12|16|56|PM|first|afternoon|lunch| +44217|AAAAAAAAKLMKAAAA|44217|12|16|57|PM|first|afternoon|lunch| +44218|AAAAAAAALLMKAAAA|44218|12|16|58|PM|first|afternoon|lunch| +44219|AAAAAAAAMLMKAAAA|44219|12|16|59|PM|first|afternoon|lunch| +44220|AAAAAAAANLMKAAAA|44220|12|17|0|PM|first|afternoon|lunch| +44221|AAAAAAAAOLMKAAAA|44221|12|17|1|PM|first|afternoon|lunch| +44222|AAAAAAAAPLMKAAAA|44222|12|17|2|PM|first|afternoon|lunch| +44223|AAAAAAAAAMMKAAAA|44223|12|17|3|PM|first|afternoon|lunch| +44224|AAAAAAAABMMKAAAA|44224|12|17|4|PM|first|afternoon|lunch| +44225|AAAAAAAACMMKAAAA|44225|12|17|5|PM|first|afternoon|lunch| +44226|AAAAAAAADMMKAAAA|44226|12|17|6|PM|first|afternoon|lunch| +44227|AAAAAAAAEMMKAAAA|44227|12|17|7|PM|first|afternoon|lunch| +44228|AAAAAAAAFMMKAAAA|44228|12|17|8|PM|first|afternoon|lunch| +44229|AAAAAAAAGMMKAAAA|44229|12|17|9|PM|first|afternoon|lunch| +44230|AAAAAAAAHMMKAAAA|44230|12|17|10|PM|first|afternoon|lunch| +44231|AAAAAAAAIMMKAAAA|44231|12|17|11|PM|first|afternoon|lunch| +44232|AAAAAAAAJMMKAAAA|44232|12|17|12|PM|first|afternoon|lunch| +44233|AAAAAAAAKMMKAAAA|44233|12|17|13|PM|first|afternoon|lunch| +44234|AAAAAAAALMMKAAAA|44234|12|17|14|PM|first|afternoon|lunch| +44235|AAAAAAAAMMMKAAAA|44235|12|17|15|PM|first|afternoon|lunch| +44236|AAAAAAAANMMKAAAA|44236|12|17|16|PM|first|afternoon|lunch| +44237|AAAAAAAAOMMKAAAA|44237|12|17|17|PM|first|afternoon|lunch| +44238|AAAAAAAAPMMKAAAA|44238|12|17|18|PM|first|afternoon|lunch| +44239|AAAAAAAAANMKAAAA|44239|12|17|19|PM|first|afternoon|lunch| +44240|AAAAAAAABNMKAAAA|44240|12|17|20|PM|first|afternoon|lunch| +44241|AAAAAAAACNMKAAAA|44241|12|17|21|PM|first|afternoon|lunch| +44242|AAAAAAAADNMKAAAA|44242|12|17|22|PM|first|afternoon|lunch| +44243|AAAAAAAAENMKAAAA|44243|12|17|23|PM|first|afternoon|lunch| +44244|AAAAAAAAFNMKAAAA|44244|12|17|24|PM|first|afternoon|lunch| +44245|AAAAAAAAGNMKAAAA|44245|12|17|25|PM|first|afternoon|lunch| +44246|AAAAAAAAHNMKAAAA|44246|12|17|26|PM|first|afternoon|lunch| +44247|AAAAAAAAINMKAAAA|44247|12|17|27|PM|first|afternoon|lunch| +44248|AAAAAAAAJNMKAAAA|44248|12|17|28|PM|first|afternoon|lunch| +44249|AAAAAAAAKNMKAAAA|44249|12|17|29|PM|first|afternoon|lunch| +44250|AAAAAAAALNMKAAAA|44250|12|17|30|PM|first|afternoon|lunch| +44251|AAAAAAAAMNMKAAAA|44251|12|17|31|PM|first|afternoon|lunch| +44252|AAAAAAAANNMKAAAA|44252|12|17|32|PM|first|afternoon|lunch| +44253|AAAAAAAAONMKAAAA|44253|12|17|33|PM|first|afternoon|lunch| +44254|AAAAAAAAPNMKAAAA|44254|12|17|34|PM|first|afternoon|lunch| +44255|AAAAAAAAAOMKAAAA|44255|12|17|35|PM|first|afternoon|lunch| +44256|AAAAAAAABOMKAAAA|44256|12|17|36|PM|first|afternoon|lunch| +44257|AAAAAAAACOMKAAAA|44257|12|17|37|PM|first|afternoon|lunch| +44258|AAAAAAAADOMKAAAA|44258|12|17|38|PM|first|afternoon|lunch| +44259|AAAAAAAAEOMKAAAA|44259|12|17|39|PM|first|afternoon|lunch| +44260|AAAAAAAAFOMKAAAA|44260|12|17|40|PM|first|afternoon|lunch| +44261|AAAAAAAAGOMKAAAA|44261|12|17|41|PM|first|afternoon|lunch| +44262|AAAAAAAAHOMKAAAA|44262|12|17|42|PM|first|afternoon|lunch| +44263|AAAAAAAAIOMKAAAA|44263|12|17|43|PM|first|afternoon|lunch| +44264|AAAAAAAAJOMKAAAA|44264|12|17|44|PM|first|afternoon|lunch| +44265|AAAAAAAAKOMKAAAA|44265|12|17|45|PM|first|afternoon|lunch| +44266|AAAAAAAALOMKAAAA|44266|12|17|46|PM|first|afternoon|lunch| +44267|AAAAAAAAMOMKAAAA|44267|12|17|47|PM|first|afternoon|lunch| +44268|AAAAAAAANOMKAAAA|44268|12|17|48|PM|first|afternoon|lunch| +44269|AAAAAAAAOOMKAAAA|44269|12|17|49|PM|first|afternoon|lunch| +44270|AAAAAAAAPOMKAAAA|44270|12|17|50|PM|first|afternoon|lunch| +44271|AAAAAAAAAPMKAAAA|44271|12|17|51|PM|first|afternoon|lunch| +44272|AAAAAAAABPMKAAAA|44272|12|17|52|PM|first|afternoon|lunch| +44273|AAAAAAAACPMKAAAA|44273|12|17|53|PM|first|afternoon|lunch| +44274|AAAAAAAADPMKAAAA|44274|12|17|54|PM|first|afternoon|lunch| +44275|AAAAAAAAEPMKAAAA|44275|12|17|55|PM|first|afternoon|lunch| +44276|AAAAAAAAFPMKAAAA|44276|12|17|56|PM|first|afternoon|lunch| +44277|AAAAAAAAGPMKAAAA|44277|12|17|57|PM|first|afternoon|lunch| +44278|AAAAAAAAHPMKAAAA|44278|12|17|58|PM|first|afternoon|lunch| +44279|AAAAAAAAIPMKAAAA|44279|12|17|59|PM|first|afternoon|lunch| +44280|AAAAAAAAJPMKAAAA|44280|12|18|0|PM|first|afternoon|lunch| +44281|AAAAAAAAKPMKAAAA|44281|12|18|1|PM|first|afternoon|lunch| +44282|AAAAAAAALPMKAAAA|44282|12|18|2|PM|first|afternoon|lunch| +44283|AAAAAAAAMPMKAAAA|44283|12|18|3|PM|first|afternoon|lunch| +44284|AAAAAAAANPMKAAAA|44284|12|18|4|PM|first|afternoon|lunch| +44285|AAAAAAAAOPMKAAAA|44285|12|18|5|PM|first|afternoon|lunch| +44286|AAAAAAAAPPMKAAAA|44286|12|18|6|PM|first|afternoon|lunch| +44287|AAAAAAAAAANKAAAA|44287|12|18|7|PM|first|afternoon|lunch| +44288|AAAAAAAABANKAAAA|44288|12|18|8|PM|first|afternoon|lunch| +44289|AAAAAAAACANKAAAA|44289|12|18|9|PM|first|afternoon|lunch| +44290|AAAAAAAADANKAAAA|44290|12|18|10|PM|first|afternoon|lunch| +44291|AAAAAAAAEANKAAAA|44291|12|18|11|PM|first|afternoon|lunch| +44292|AAAAAAAAFANKAAAA|44292|12|18|12|PM|first|afternoon|lunch| +44293|AAAAAAAAGANKAAAA|44293|12|18|13|PM|first|afternoon|lunch| +44294|AAAAAAAAHANKAAAA|44294|12|18|14|PM|first|afternoon|lunch| +44295|AAAAAAAAIANKAAAA|44295|12|18|15|PM|first|afternoon|lunch| +44296|AAAAAAAAJANKAAAA|44296|12|18|16|PM|first|afternoon|lunch| +44297|AAAAAAAAKANKAAAA|44297|12|18|17|PM|first|afternoon|lunch| +44298|AAAAAAAALANKAAAA|44298|12|18|18|PM|first|afternoon|lunch| +44299|AAAAAAAAMANKAAAA|44299|12|18|19|PM|first|afternoon|lunch| +44300|AAAAAAAANANKAAAA|44300|12|18|20|PM|first|afternoon|lunch| +44301|AAAAAAAAOANKAAAA|44301|12|18|21|PM|first|afternoon|lunch| +44302|AAAAAAAAPANKAAAA|44302|12|18|22|PM|first|afternoon|lunch| +44303|AAAAAAAAABNKAAAA|44303|12|18|23|PM|first|afternoon|lunch| +44304|AAAAAAAABBNKAAAA|44304|12|18|24|PM|first|afternoon|lunch| +44305|AAAAAAAACBNKAAAA|44305|12|18|25|PM|first|afternoon|lunch| +44306|AAAAAAAADBNKAAAA|44306|12|18|26|PM|first|afternoon|lunch| +44307|AAAAAAAAEBNKAAAA|44307|12|18|27|PM|first|afternoon|lunch| +44308|AAAAAAAAFBNKAAAA|44308|12|18|28|PM|first|afternoon|lunch| +44309|AAAAAAAAGBNKAAAA|44309|12|18|29|PM|first|afternoon|lunch| +44310|AAAAAAAAHBNKAAAA|44310|12|18|30|PM|first|afternoon|lunch| +44311|AAAAAAAAIBNKAAAA|44311|12|18|31|PM|first|afternoon|lunch| +44312|AAAAAAAAJBNKAAAA|44312|12|18|32|PM|first|afternoon|lunch| +44313|AAAAAAAAKBNKAAAA|44313|12|18|33|PM|first|afternoon|lunch| +44314|AAAAAAAALBNKAAAA|44314|12|18|34|PM|first|afternoon|lunch| +44315|AAAAAAAAMBNKAAAA|44315|12|18|35|PM|first|afternoon|lunch| +44316|AAAAAAAANBNKAAAA|44316|12|18|36|PM|first|afternoon|lunch| +44317|AAAAAAAAOBNKAAAA|44317|12|18|37|PM|first|afternoon|lunch| +44318|AAAAAAAAPBNKAAAA|44318|12|18|38|PM|first|afternoon|lunch| +44319|AAAAAAAAACNKAAAA|44319|12|18|39|PM|first|afternoon|lunch| +44320|AAAAAAAABCNKAAAA|44320|12|18|40|PM|first|afternoon|lunch| +44321|AAAAAAAACCNKAAAA|44321|12|18|41|PM|first|afternoon|lunch| +44322|AAAAAAAADCNKAAAA|44322|12|18|42|PM|first|afternoon|lunch| +44323|AAAAAAAAECNKAAAA|44323|12|18|43|PM|first|afternoon|lunch| +44324|AAAAAAAAFCNKAAAA|44324|12|18|44|PM|first|afternoon|lunch| +44325|AAAAAAAAGCNKAAAA|44325|12|18|45|PM|first|afternoon|lunch| +44326|AAAAAAAAHCNKAAAA|44326|12|18|46|PM|first|afternoon|lunch| +44327|AAAAAAAAICNKAAAA|44327|12|18|47|PM|first|afternoon|lunch| +44328|AAAAAAAAJCNKAAAA|44328|12|18|48|PM|first|afternoon|lunch| +44329|AAAAAAAAKCNKAAAA|44329|12|18|49|PM|first|afternoon|lunch| +44330|AAAAAAAALCNKAAAA|44330|12|18|50|PM|first|afternoon|lunch| +44331|AAAAAAAAMCNKAAAA|44331|12|18|51|PM|first|afternoon|lunch| +44332|AAAAAAAANCNKAAAA|44332|12|18|52|PM|first|afternoon|lunch| +44333|AAAAAAAAOCNKAAAA|44333|12|18|53|PM|first|afternoon|lunch| +44334|AAAAAAAAPCNKAAAA|44334|12|18|54|PM|first|afternoon|lunch| +44335|AAAAAAAAADNKAAAA|44335|12|18|55|PM|first|afternoon|lunch| +44336|AAAAAAAABDNKAAAA|44336|12|18|56|PM|first|afternoon|lunch| +44337|AAAAAAAACDNKAAAA|44337|12|18|57|PM|first|afternoon|lunch| +44338|AAAAAAAADDNKAAAA|44338|12|18|58|PM|first|afternoon|lunch| +44339|AAAAAAAAEDNKAAAA|44339|12|18|59|PM|first|afternoon|lunch| +44340|AAAAAAAAFDNKAAAA|44340|12|19|0|PM|first|afternoon|lunch| +44341|AAAAAAAAGDNKAAAA|44341|12|19|1|PM|first|afternoon|lunch| +44342|AAAAAAAAHDNKAAAA|44342|12|19|2|PM|first|afternoon|lunch| +44343|AAAAAAAAIDNKAAAA|44343|12|19|3|PM|first|afternoon|lunch| +44344|AAAAAAAAJDNKAAAA|44344|12|19|4|PM|first|afternoon|lunch| +44345|AAAAAAAAKDNKAAAA|44345|12|19|5|PM|first|afternoon|lunch| +44346|AAAAAAAALDNKAAAA|44346|12|19|6|PM|first|afternoon|lunch| +44347|AAAAAAAAMDNKAAAA|44347|12|19|7|PM|first|afternoon|lunch| +44348|AAAAAAAANDNKAAAA|44348|12|19|8|PM|first|afternoon|lunch| +44349|AAAAAAAAODNKAAAA|44349|12|19|9|PM|first|afternoon|lunch| +44350|AAAAAAAAPDNKAAAA|44350|12|19|10|PM|first|afternoon|lunch| +44351|AAAAAAAAAENKAAAA|44351|12|19|11|PM|first|afternoon|lunch| +44352|AAAAAAAABENKAAAA|44352|12|19|12|PM|first|afternoon|lunch| +44353|AAAAAAAACENKAAAA|44353|12|19|13|PM|first|afternoon|lunch| +44354|AAAAAAAADENKAAAA|44354|12|19|14|PM|first|afternoon|lunch| +44355|AAAAAAAAEENKAAAA|44355|12|19|15|PM|first|afternoon|lunch| +44356|AAAAAAAAFENKAAAA|44356|12|19|16|PM|first|afternoon|lunch| +44357|AAAAAAAAGENKAAAA|44357|12|19|17|PM|first|afternoon|lunch| +44358|AAAAAAAAHENKAAAA|44358|12|19|18|PM|first|afternoon|lunch| +44359|AAAAAAAAIENKAAAA|44359|12|19|19|PM|first|afternoon|lunch| +44360|AAAAAAAAJENKAAAA|44360|12|19|20|PM|first|afternoon|lunch| +44361|AAAAAAAAKENKAAAA|44361|12|19|21|PM|first|afternoon|lunch| +44362|AAAAAAAALENKAAAA|44362|12|19|22|PM|first|afternoon|lunch| +44363|AAAAAAAAMENKAAAA|44363|12|19|23|PM|first|afternoon|lunch| +44364|AAAAAAAANENKAAAA|44364|12|19|24|PM|first|afternoon|lunch| +44365|AAAAAAAAOENKAAAA|44365|12|19|25|PM|first|afternoon|lunch| +44366|AAAAAAAAPENKAAAA|44366|12|19|26|PM|first|afternoon|lunch| +44367|AAAAAAAAAFNKAAAA|44367|12|19|27|PM|first|afternoon|lunch| +44368|AAAAAAAABFNKAAAA|44368|12|19|28|PM|first|afternoon|lunch| +44369|AAAAAAAACFNKAAAA|44369|12|19|29|PM|first|afternoon|lunch| +44370|AAAAAAAADFNKAAAA|44370|12|19|30|PM|first|afternoon|lunch| +44371|AAAAAAAAEFNKAAAA|44371|12|19|31|PM|first|afternoon|lunch| +44372|AAAAAAAAFFNKAAAA|44372|12|19|32|PM|first|afternoon|lunch| +44373|AAAAAAAAGFNKAAAA|44373|12|19|33|PM|first|afternoon|lunch| +44374|AAAAAAAAHFNKAAAA|44374|12|19|34|PM|first|afternoon|lunch| +44375|AAAAAAAAIFNKAAAA|44375|12|19|35|PM|first|afternoon|lunch| +44376|AAAAAAAAJFNKAAAA|44376|12|19|36|PM|first|afternoon|lunch| +44377|AAAAAAAAKFNKAAAA|44377|12|19|37|PM|first|afternoon|lunch| +44378|AAAAAAAALFNKAAAA|44378|12|19|38|PM|first|afternoon|lunch| +44379|AAAAAAAAMFNKAAAA|44379|12|19|39|PM|first|afternoon|lunch| +44380|AAAAAAAANFNKAAAA|44380|12|19|40|PM|first|afternoon|lunch| +44381|AAAAAAAAOFNKAAAA|44381|12|19|41|PM|first|afternoon|lunch| +44382|AAAAAAAAPFNKAAAA|44382|12|19|42|PM|first|afternoon|lunch| +44383|AAAAAAAAAGNKAAAA|44383|12|19|43|PM|first|afternoon|lunch| +44384|AAAAAAAABGNKAAAA|44384|12|19|44|PM|first|afternoon|lunch| +44385|AAAAAAAACGNKAAAA|44385|12|19|45|PM|first|afternoon|lunch| +44386|AAAAAAAADGNKAAAA|44386|12|19|46|PM|first|afternoon|lunch| +44387|AAAAAAAAEGNKAAAA|44387|12|19|47|PM|first|afternoon|lunch| +44388|AAAAAAAAFGNKAAAA|44388|12|19|48|PM|first|afternoon|lunch| +44389|AAAAAAAAGGNKAAAA|44389|12|19|49|PM|first|afternoon|lunch| +44390|AAAAAAAAHGNKAAAA|44390|12|19|50|PM|first|afternoon|lunch| +44391|AAAAAAAAIGNKAAAA|44391|12|19|51|PM|first|afternoon|lunch| +44392|AAAAAAAAJGNKAAAA|44392|12|19|52|PM|first|afternoon|lunch| +44393|AAAAAAAAKGNKAAAA|44393|12|19|53|PM|first|afternoon|lunch| +44394|AAAAAAAALGNKAAAA|44394|12|19|54|PM|first|afternoon|lunch| +44395|AAAAAAAAMGNKAAAA|44395|12|19|55|PM|first|afternoon|lunch| +44396|AAAAAAAANGNKAAAA|44396|12|19|56|PM|first|afternoon|lunch| +44397|AAAAAAAAOGNKAAAA|44397|12|19|57|PM|first|afternoon|lunch| +44398|AAAAAAAAPGNKAAAA|44398|12|19|58|PM|first|afternoon|lunch| +44399|AAAAAAAAAHNKAAAA|44399|12|19|59|PM|first|afternoon|lunch| +44400|AAAAAAAABHNKAAAA|44400|12|20|0|PM|first|afternoon|lunch| +44401|AAAAAAAACHNKAAAA|44401|12|20|1|PM|first|afternoon|lunch| +44402|AAAAAAAADHNKAAAA|44402|12|20|2|PM|first|afternoon|lunch| +44403|AAAAAAAAEHNKAAAA|44403|12|20|3|PM|first|afternoon|lunch| +44404|AAAAAAAAFHNKAAAA|44404|12|20|4|PM|first|afternoon|lunch| +44405|AAAAAAAAGHNKAAAA|44405|12|20|5|PM|first|afternoon|lunch| +44406|AAAAAAAAHHNKAAAA|44406|12|20|6|PM|first|afternoon|lunch| +44407|AAAAAAAAIHNKAAAA|44407|12|20|7|PM|first|afternoon|lunch| +44408|AAAAAAAAJHNKAAAA|44408|12|20|8|PM|first|afternoon|lunch| +44409|AAAAAAAAKHNKAAAA|44409|12|20|9|PM|first|afternoon|lunch| +44410|AAAAAAAALHNKAAAA|44410|12|20|10|PM|first|afternoon|lunch| +44411|AAAAAAAAMHNKAAAA|44411|12|20|11|PM|first|afternoon|lunch| +44412|AAAAAAAANHNKAAAA|44412|12|20|12|PM|first|afternoon|lunch| +44413|AAAAAAAAOHNKAAAA|44413|12|20|13|PM|first|afternoon|lunch| +44414|AAAAAAAAPHNKAAAA|44414|12|20|14|PM|first|afternoon|lunch| +44415|AAAAAAAAAINKAAAA|44415|12|20|15|PM|first|afternoon|lunch| +44416|AAAAAAAABINKAAAA|44416|12|20|16|PM|first|afternoon|lunch| +44417|AAAAAAAACINKAAAA|44417|12|20|17|PM|first|afternoon|lunch| +44418|AAAAAAAADINKAAAA|44418|12|20|18|PM|first|afternoon|lunch| +44419|AAAAAAAAEINKAAAA|44419|12|20|19|PM|first|afternoon|lunch| +44420|AAAAAAAAFINKAAAA|44420|12|20|20|PM|first|afternoon|lunch| +44421|AAAAAAAAGINKAAAA|44421|12|20|21|PM|first|afternoon|lunch| +44422|AAAAAAAAHINKAAAA|44422|12|20|22|PM|first|afternoon|lunch| +44423|AAAAAAAAIINKAAAA|44423|12|20|23|PM|first|afternoon|lunch| +44424|AAAAAAAAJINKAAAA|44424|12|20|24|PM|first|afternoon|lunch| +44425|AAAAAAAAKINKAAAA|44425|12|20|25|PM|first|afternoon|lunch| +44426|AAAAAAAALINKAAAA|44426|12|20|26|PM|first|afternoon|lunch| +44427|AAAAAAAAMINKAAAA|44427|12|20|27|PM|first|afternoon|lunch| +44428|AAAAAAAANINKAAAA|44428|12|20|28|PM|first|afternoon|lunch| +44429|AAAAAAAAOINKAAAA|44429|12|20|29|PM|first|afternoon|lunch| +44430|AAAAAAAAPINKAAAA|44430|12|20|30|PM|first|afternoon|lunch| +44431|AAAAAAAAAJNKAAAA|44431|12|20|31|PM|first|afternoon|lunch| +44432|AAAAAAAABJNKAAAA|44432|12|20|32|PM|first|afternoon|lunch| +44433|AAAAAAAACJNKAAAA|44433|12|20|33|PM|first|afternoon|lunch| +44434|AAAAAAAADJNKAAAA|44434|12|20|34|PM|first|afternoon|lunch| +44435|AAAAAAAAEJNKAAAA|44435|12|20|35|PM|first|afternoon|lunch| +44436|AAAAAAAAFJNKAAAA|44436|12|20|36|PM|first|afternoon|lunch| +44437|AAAAAAAAGJNKAAAA|44437|12|20|37|PM|first|afternoon|lunch| +44438|AAAAAAAAHJNKAAAA|44438|12|20|38|PM|first|afternoon|lunch| +44439|AAAAAAAAIJNKAAAA|44439|12|20|39|PM|first|afternoon|lunch| +44440|AAAAAAAAJJNKAAAA|44440|12|20|40|PM|first|afternoon|lunch| +44441|AAAAAAAAKJNKAAAA|44441|12|20|41|PM|first|afternoon|lunch| +44442|AAAAAAAALJNKAAAA|44442|12|20|42|PM|first|afternoon|lunch| +44443|AAAAAAAAMJNKAAAA|44443|12|20|43|PM|first|afternoon|lunch| +44444|AAAAAAAANJNKAAAA|44444|12|20|44|PM|first|afternoon|lunch| +44445|AAAAAAAAOJNKAAAA|44445|12|20|45|PM|first|afternoon|lunch| +44446|AAAAAAAAPJNKAAAA|44446|12|20|46|PM|first|afternoon|lunch| +44447|AAAAAAAAAKNKAAAA|44447|12|20|47|PM|first|afternoon|lunch| +44448|AAAAAAAABKNKAAAA|44448|12|20|48|PM|first|afternoon|lunch| +44449|AAAAAAAACKNKAAAA|44449|12|20|49|PM|first|afternoon|lunch| +44450|AAAAAAAADKNKAAAA|44450|12|20|50|PM|first|afternoon|lunch| +44451|AAAAAAAAEKNKAAAA|44451|12|20|51|PM|first|afternoon|lunch| +44452|AAAAAAAAFKNKAAAA|44452|12|20|52|PM|first|afternoon|lunch| +44453|AAAAAAAAGKNKAAAA|44453|12|20|53|PM|first|afternoon|lunch| +44454|AAAAAAAAHKNKAAAA|44454|12|20|54|PM|first|afternoon|lunch| +44455|AAAAAAAAIKNKAAAA|44455|12|20|55|PM|first|afternoon|lunch| +44456|AAAAAAAAJKNKAAAA|44456|12|20|56|PM|first|afternoon|lunch| +44457|AAAAAAAAKKNKAAAA|44457|12|20|57|PM|first|afternoon|lunch| +44458|AAAAAAAALKNKAAAA|44458|12|20|58|PM|first|afternoon|lunch| +44459|AAAAAAAAMKNKAAAA|44459|12|20|59|PM|first|afternoon|lunch| +44460|AAAAAAAANKNKAAAA|44460|12|21|0|PM|first|afternoon|lunch| +44461|AAAAAAAAOKNKAAAA|44461|12|21|1|PM|first|afternoon|lunch| +44462|AAAAAAAAPKNKAAAA|44462|12|21|2|PM|first|afternoon|lunch| +44463|AAAAAAAAALNKAAAA|44463|12|21|3|PM|first|afternoon|lunch| +44464|AAAAAAAABLNKAAAA|44464|12|21|4|PM|first|afternoon|lunch| +44465|AAAAAAAACLNKAAAA|44465|12|21|5|PM|first|afternoon|lunch| +44466|AAAAAAAADLNKAAAA|44466|12|21|6|PM|first|afternoon|lunch| +44467|AAAAAAAAELNKAAAA|44467|12|21|7|PM|first|afternoon|lunch| +44468|AAAAAAAAFLNKAAAA|44468|12|21|8|PM|first|afternoon|lunch| +44469|AAAAAAAAGLNKAAAA|44469|12|21|9|PM|first|afternoon|lunch| +44470|AAAAAAAAHLNKAAAA|44470|12|21|10|PM|first|afternoon|lunch| +44471|AAAAAAAAILNKAAAA|44471|12|21|11|PM|first|afternoon|lunch| +44472|AAAAAAAAJLNKAAAA|44472|12|21|12|PM|first|afternoon|lunch| +44473|AAAAAAAAKLNKAAAA|44473|12|21|13|PM|first|afternoon|lunch| +44474|AAAAAAAALLNKAAAA|44474|12|21|14|PM|first|afternoon|lunch| +44475|AAAAAAAAMLNKAAAA|44475|12|21|15|PM|first|afternoon|lunch| +44476|AAAAAAAANLNKAAAA|44476|12|21|16|PM|first|afternoon|lunch| +44477|AAAAAAAAOLNKAAAA|44477|12|21|17|PM|first|afternoon|lunch| +44478|AAAAAAAAPLNKAAAA|44478|12|21|18|PM|first|afternoon|lunch| +44479|AAAAAAAAAMNKAAAA|44479|12|21|19|PM|first|afternoon|lunch| +44480|AAAAAAAABMNKAAAA|44480|12|21|20|PM|first|afternoon|lunch| +44481|AAAAAAAACMNKAAAA|44481|12|21|21|PM|first|afternoon|lunch| +44482|AAAAAAAADMNKAAAA|44482|12|21|22|PM|first|afternoon|lunch| +44483|AAAAAAAAEMNKAAAA|44483|12|21|23|PM|first|afternoon|lunch| +44484|AAAAAAAAFMNKAAAA|44484|12|21|24|PM|first|afternoon|lunch| +44485|AAAAAAAAGMNKAAAA|44485|12|21|25|PM|first|afternoon|lunch| +44486|AAAAAAAAHMNKAAAA|44486|12|21|26|PM|first|afternoon|lunch| +44487|AAAAAAAAIMNKAAAA|44487|12|21|27|PM|first|afternoon|lunch| +44488|AAAAAAAAJMNKAAAA|44488|12|21|28|PM|first|afternoon|lunch| +44489|AAAAAAAAKMNKAAAA|44489|12|21|29|PM|first|afternoon|lunch| +44490|AAAAAAAALMNKAAAA|44490|12|21|30|PM|first|afternoon|lunch| +44491|AAAAAAAAMMNKAAAA|44491|12|21|31|PM|first|afternoon|lunch| +44492|AAAAAAAANMNKAAAA|44492|12|21|32|PM|first|afternoon|lunch| +44493|AAAAAAAAOMNKAAAA|44493|12|21|33|PM|first|afternoon|lunch| +44494|AAAAAAAAPMNKAAAA|44494|12|21|34|PM|first|afternoon|lunch| +44495|AAAAAAAAANNKAAAA|44495|12|21|35|PM|first|afternoon|lunch| +44496|AAAAAAAABNNKAAAA|44496|12|21|36|PM|first|afternoon|lunch| +44497|AAAAAAAACNNKAAAA|44497|12|21|37|PM|first|afternoon|lunch| +44498|AAAAAAAADNNKAAAA|44498|12|21|38|PM|first|afternoon|lunch| +44499|AAAAAAAAENNKAAAA|44499|12|21|39|PM|first|afternoon|lunch| +44500|AAAAAAAAFNNKAAAA|44500|12|21|40|PM|first|afternoon|lunch| +44501|AAAAAAAAGNNKAAAA|44501|12|21|41|PM|first|afternoon|lunch| +44502|AAAAAAAAHNNKAAAA|44502|12|21|42|PM|first|afternoon|lunch| +44503|AAAAAAAAINNKAAAA|44503|12|21|43|PM|first|afternoon|lunch| +44504|AAAAAAAAJNNKAAAA|44504|12|21|44|PM|first|afternoon|lunch| +44505|AAAAAAAAKNNKAAAA|44505|12|21|45|PM|first|afternoon|lunch| +44506|AAAAAAAALNNKAAAA|44506|12|21|46|PM|first|afternoon|lunch| +44507|AAAAAAAAMNNKAAAA|44507|12|21|47|PM|first|afternoon|lunch| +44508|AAAAAAAANNNKAAAA|44508|12|21|48|PM|first|afternoon|lunch| +44509|AAAAAAAAONNKAAAA|44509|12|21|49|PM|first|afternoon|lunch| +44510|AAAAAAAAPNNKAAAA|44510|12|21|50|PM|first|afternoon|lunch| +44511|AAAAAAAAAONKAAAA|44511|12|21|51|PM|first|afternoon|lunch| +44512|AAAAAAAABONKAAAA|44512|12|21|52|PM|first|afternoon|lunch| +44513|AAAAAAAACONKAAAA|44513|12|21|53|PM|first|afternoon|lunch| +44514|AAAAAAAADONKAAAA|44514|12|21|54|PM|first|afternoon|lunch| +44515|AAAAAAAAEONKAAAA|44515|12|21|55|PM|first|afternoon|lunch| +44516|AAAAAAAAFONKAAAA|44516|12|21|56|PM|first|afternoon|lunch| +44517|AAAAAAAAGONKAAAA|44517|12|21|57|PM|first|afternoon|lunch| +44518|AAAAAAAAHONKAAAA|44518|12|21|58|PM|first|afternoon|lunch| +44519|AAAAAAAAIONKAAAA|44519|12|21|59|PM|first|afternoon|lunch| +44520|AAAAAAAAJONKAAAA|44520|12|22|0|PM|first|afternoon|lunch| +44521|AAAAAAAAKONKAAAA|44521|12|22|1|PM|first|afternoon|lunch| +44522|AAAAAAAALONKAAAA|44522|12|22|2|PM|first|afternoon|lunch| +44523|AAAAAAAAMONKAAAA|44523|12|22|3|PM|first|afternoon|lunch| +44524|AAAAAAAANONKAAAA|44524|12|22|4|PM|first|afternoon|lunch| +44525|AAAAAAAAOONKAAAA|44525|12|22|5|PM|first|afternoon|lunch| +44526|AAAAAAAAPONKAAAA|44526|12|22|6|PM|first|afternoon|lunch| +44527|AAAAAAAAAPNKAAAA|44527|12|22|7|PM|first|afternoon|lunch| +44528|AAAAAAAABPNKAAAA|44528|12|22|8|PM|first|afternoon|lunch| +44529|AAAAAAAACPNKAAAA|44529|12|22|9|PM|first|afternoon|lunch| +44530|AAAAAAAADPNKAAAA|44530|12|22|10|PM|first|afternoon|lunch| +44531|AAAAAAAAEPNKAAAA|44531|12|22|11|PM|first|afternoon|lunch| +44532|AAAAAAAAFPNKAAAA|44532|12|22|12|PM|first|afternoon|lunch| +44533|AAAAAAAAGPNKAAAA|44533|12|22|13|PM|first|afternoon|lunch| +44534|AAAAAAAAHPNKAAAA|44534|12|22|14|PM|first|afternoon|lunch| +44535|AAAAAAAAIPNKAAAA|44535|12|22|15|PM|first|afternoon|lunch| +44536|AAAAAAAAJPNKAAAA|44536|12|22|16|PM|first|afternoon|lunch| +44537|AAAAAAAAKPNKAAAA|44537|12|22|17|PM|first|afternoon|lunch| +44538|AAAAAAAALPNKAAAA|44538|12|22|18|PM|first|afternoon|lunch| +44539|AAAAAAAAMPNKAAAA|44539|12|22|19|PM|first|afternoon|lunch| +44540|AAAAAAAANPNKAAAA|44540|12|22|20|PM|first|afternoon|lunch| +44541|AAAAAAAAOPNKAAAA|44541|12|22|21|PM|first|afternoon|lunch| +44542|AAAAAAAAPPNKAAAA|44542|12|22|22|PM|first|afternoon|lunch| +44543|AAAAAAAAAAOKAAAA|44543|12|22|23|PM|first|afternoon|lunch| +44544|AAAAAAAABAOKAAAA|44544|12|22|24|PM|first|afternoon|lunch| +44545|AAAAAAAACAOKAAAA|44545|12|22|25|PM|first|afternoon|lunch| +44546|AAAAAAAADAOKAAAA|44546|12|22|26|PM|first|afternoon|lunch| +44547|AAAAAAAAEAOKAAAA|44547|12|22|27|PM|first|afternoon|lunch| +44548|AAAAAAAAFAOKAAAA|44548|12|22|28|PM|first|afternoon|lunch| +44549|AAAAAAAAGAOKAAAA|44549|12|22|29|PM|first|afternoon|lunch| +44550|AAAAAAAAHAOKAAAA|44550|12|22|30|PM|first|afternoon|lunch| +44551|AAAAAAAAIAOKAAAA|44551|12|22|31|PM|first|afternoon|lunch| +44552|AAAAAAAAJAOKAAAA|44552|12|22|32|PM|first|afternoon|lunch| +44553|AAAAAAAAKAOKAAAA|44553|12|22|33|PM|first|afternoon|lunch| +44554|AAAAAAAALAOKAAAA|44554|12|22|34|PM|first|afternoon|lunch| +44555|AAAAAAAAMAOKAAAA|44555|12|22|35|PM|first|afternoon|lunch| +44556|AAAAAAAANAOKAAAA|44556|12|22|36|PM|first|afternoon|lunch| +44557|AAAAAAAAOAOKAAAA|44557|12|22|37|PM|first|afternoon|lunch| +44558|AAAAAAAAPAOKAAAA|44558|12|22|38|PM|first|afternoon|lunch| +44559|AAAAAAAAABOKAAAA|44559|12|22|39|PM|first|afternoon|lunch| +44560|AAAAAAAABBOKAAAA|44560|12|22|40|PM|first|afternoon|lunch| +44561|AAAAAAAACBOKAAAA|44561|12|22|41|PM|first|afternoon|lunch| +44562|AAAAAAAADBOKAAAA|44562|12|22|42|PM|first|afternoon|lunch| +44563|AAAAAAAAEBOKAAAA|44563|12|22|43|PM|first|afternoon|lunch| +44564|AAAAAAAAFBOKAAAA|44564|12|22|44|PM|first|afternoon|lunch| +44565|AAAAAAAAGBOKAAAA|44565|12|22|45|PM|first|afternoon|lunch| +44566|AAAAAAAAHBOKAAAA|44566|12|22|46|PM|first|afternoon|lunch| +44567|AAAAAAAAIBOKAAAA|44567|12|22|47|PM|first|afternoon|lunch| +44568|AAAAAAAAJBOKAAAA|44568|12|22|48|PM|first|afternoon|lunch| +44569|AAAAAAAAKBOKAAAA|44569|12|22|49|PM|first|afternoon|lunch| +44570|AAAAAAAALBOKAAAA|44570|12|22|50|PM|first|afternoon|lunch| +44571|AAAAAAAAMBOKAAAA|44571|12|22|51|PM|first|afternoon|lunch| +44572|AAAAAAAANBOKAAAA|44572|12|22|52|PM|first|afternoon|lunch| +44573|AAAAAAAAOBOKAAAA|44573|12|22|53|PM|first|afternoon|lunch| +44574|AAAAAAAAPBOKAAAA|44574|12|22|54|PM|first|afternoon|lunch| +44575|AAAAAAAAACOKAAAA|44575|12|22|55|PM|first|afternoon|lunch| +44576|AAAAAAAABCOKAAAA|44576|12|22|56|PM|first|afternoon|lunch| +44577|AAAAAAAACCOKAAAA|44577|12|22|57|PM|first|afternoon|lunch| +44578|AAAAAAAADCOKAAAA|44578|12|22|58|PM|first|afternoon|lunch| +44579|AAAAAAAAECOKAAAA|44579|12|22|59|PM|first|afternoon|lunch| +44580|AAAAAAAAFCOKAAAA|44580|12|23|0|PM|first|afternoon|lunch| +44581|AAAAAAAAGCOKAAAA|44581|12|23|1|PM|first|afternoon|lunch| +44582|AAAAAAAAHCOKAAAA|44582|12|23|2|PM|first|afternoon|lunch| +44583|AAAAAAAAICOKAAAA|44583|12|23|3|PM|first|afternoon|lunch| +44584|AAAAAAAAJCOKAAAA|44584|12|23|4|PM|first|afternoon|lunch| +44585|AAAAAAAAKCOKAAAA|44585|12|23|5|PM|first|afternoon|lunch| +44586|AAAAAAAALCOKAAAA|44586|12|23|6|PM|first|afternoon|lunch| +44587|AAAAAAAAMCOKAAAA|44587|12|23|7|PM|first|afternoon|lunch| +44588|AAAAAAAANCOKAAAA|44588|12|23|8|PM|first|afternoon|lunch| +44589|AAAAAAAAOCOKAAAA|44589|12|23|9|PM|first|afternoon|lunch| +44590|AAAAAAAAPCOKAAAA|44590|12|23|10|PM|first|afternoon|lunch| +44591|AAAAAAAAADOKAAAA|44591|12|23|11|PM|first|afternoon|lunch| +44592|AAAAAAAABDOKAAAA|44592|12|23|12|PM|first|afternoon|lunch| +44593|AAAAAAAACDOKAAAA|44593|12|23|13|PM|first|afternoon|lunch| +44594|AAAAAAAADDOKAAAA|44594|12|23|14|PM|first|afternoon|lunch| +44595|AAAAAAAAEDOKAAAA|44595|12|23|15|PM|first|afternoon|lunch| +44596|AAAAAAAAFDOKAAAA|44596|12|23|16|PM|first|afternoon|lunch| +44597|AAAAAAAAGDOKAAAA|44597|12|23|17|PM|first|afternoon|lunch| +44598|AAAAAAAAHDOKAAAA|44598|12|23|18|PM|first|afternoon|lunch| +44599|AAAAAAAAIDOKAAAA|44599|12|23|19|PM|first|afternoon|lunch| +44600|AAAAAAAAJDOKAAAA|44600|12|23|20|PM|first|afternoon|lunch| +44601|AAAAAAAAKDOKAAAA|44601|12|23|21|PM|first|afternoon|lunch| +44602|AAAAAAAALDOKAAAA|44602|12|23|22|PM|first|afternoon|lunch| +44603|AAAAAAAAMDOKAAAA|44603|12|23|23|PM|first|afternoon|lunch| +44604|AAAAAAAANDOKAAAA|44604|12|23|24|PM|first|afternoon|lunch| +44605|AAAAAAAAODOKAAAA|44605|12|23|25|PM|first|afternoon|lunch| +44606|AAAAAAAAPDOKAAAA|44606|12|23|26|PM|first|afternoon|lunch| +44607|AAAAAAAAAEOKAAAA|44607|12|23|27|PM|first|afternoon|lunch| +44608|AAAAAAAABEOKAAAA|44608|12|23|28|PM|first|afternoon|lunch| +44609|AAAAAAAACEOKAAAA|44609|12|23|29|PM|first|afternoon|lunch| +44610|AAAAAAAADEOKAAAA|44610|12|23|30|PM|first|afternoon|lunch| +44611|AAAAAAAAEEOKAAAA|44611|12|23|31|PM|first|afternoon|lunch| +44612|AAAAAAAAFEOKAAAA|44612|12|23|32|PM|first|afternoon|lunch| +44613|AAAAAAAAGEOKAAAA|44613|12|23|33|PM|first|afternoon|lunch| +44614|AAAAAAAAHEOKAAAA|44614|12|23|34|PM|first|afternoon|lunch| +44615|AAAAAAAAIEOKAAAA|44615|12|23|35|PM|first|afternoon|lunch| +44616|AAAAAAAAJEOKAAAA|44616|12|23|36|PM|first|afternoon|lunch| +44617|AAAAAAAAKEOKAAAA|44617|12|23|37|PM|first|afternoon|lunch| +44618|AAAAAAAALEOKAAAA|44618|12|23|38|PM|first|afternoon|lunch| +44619|AAAAAAAAMEOKAAAA|44619|12|23|39|PM|first|afternoon|lunch| +44620|AAAAAAAANEOKAAAA|44620|12|23|40|PM|first|afternoon|lunch| +44621|AAAAAAAAOEOKAAAA|44621|12|23|41|PM|first|afternoon|lunch| +44622|AAAAAAAAPEOKAAAA|44622|12|23|42|PM|first|afternoon|lunch| +44623|AAAAAAAAAFOKAAAA|44623|12|23|43|PM|first|afternoon|lunch| +44624|AAAAAAAABFOKAAAA|44624|12|23|44|PM|first|afternoon|lunch| +44625|AAAAAAAACFOKAAAA|44625|12|23|45|PM|first|afternoon|lunch| +44626|AAAAAAAADFOKAAAA|44626|12|23|46|PM|first|afternoon|lunch| +44627|AAAAAAAAEFOKAAAA|44627|12|23|47|PM|first|afternoon|lunch| +44628|AAAAAAAAFFOKAAAA|44628|12|23|48|PM|first|afternoon|lunch| +44629|AAAAAAAAGFOKAAAA|44629|12|23|49|PM|first|afternoon|lunch| +44630|AAAAAAAAHFOKAAAA|44630|12|23|50|PM|first|afternoon|lunch| +44631|AAAAAAAAIFOKAAAA|44631|12|23|51|PM|first|afternoon|lunch| +44632|AAAAAAAAJFOKAAAA|44632|12|23|52|PM|first|afternoon|lunch| +44633|AAAAAAAAKFOKAAAA|44633|12|23|53|PM|first|afternoon|lunch| +44634|AAAAAAAALFOKAAAA|44634|12|23|54|PM|first|afternoon|lunch| +44635|AAAAAAAAMFOKAAAA|44635|12|23|55|PM|first|afternoon|lunch| +44636|AAAAAAAANFOKAAAA|44636|12|23|56|PM|first|afternoon|lunch| +44637|AAAAAAAAOFOKAAAA|44637|12|23|57|PM|first|afternoon|lunch| +44638|AAAAAAAAPFOKAAAA|44638|12|23|58|PM|first|afternoon|lunch| +44639|AAAAAAAAAGOKAAAA|44639|12|23|59|PM|first|afternoon|lunch| +44640|AAAAAAAABGOKAAAA|44640|12|24|0|PM|first|afternoon|lunch| +44641|AAAAAAAACGOKAAAA|44641|12|24|1|PM|first|afternoon|lunch| +44642|AAAAAAAADGOKAAAA|44642|12|24|2|PM|first|afternoon|lunch| +44643|AAAAAAAAEGOKAAAA|44643|12|24|3|PM|first|afternoon|lunch| +44644|AAAAAAAAFGOKAAAA|44644|12|24|4|PM|first|afternoon|lunch| +44645|AAAAAAAAGGOKAAAA|44645|12|24|5|PM|first|afternoon|lunch| +44646|AAAAAAAAHGOKAAAA|44646|12|24|6|PM|first|afternoon|lunch| +44647|AAAAAAAAIGOKAAAA|44647|12|24|7|PM|first|afternoon|lunch| +44648|AAAAAAAAJGOKAAAA|44648|12|24|8|PM|first|afternoon|lunch| +44649|AAAAAAAAKGOKAAAA|44649|12|24|9|PM|first|afternoon|lunch| +44650|AAAAAAAALGOKAAAA|44650|12|24|10|PM|first|afternoon|lunch| +44651|AAAAAAAAMGOKAAAA|44651|12|24|11|PM|first|afternoon|lunch| +44652|AAAAAAAANGOKAAAA|44652|12|24|12|PM|first|afternoon|lunch| +44653|AAAAAAAAOGOKAAAA|44653|12|24|13|PM|first|afternoon|lunch| +44654|AAAAAAAAPGOKAAAA|44654|12|24|14|PM|first|afternoon|lunch| +44655|AAAAAAAAAHOKAAAA|44655|12|24|15|PM|first|afternoon|lunch| +44656|AAAAAAAABHOKAAAA|44656|12|24|16|PM|first|afternoon|lunch| +44657|AAAAAAAACHOKAAAA|44657|12|24|17|PM|first|afternoon|lunch| +44658|AAAAAAAADHOKAAAA|44658|12|24|18|PM|first|afternoon|lunch| +44659|AAAAAAAAEHOKAAAA|44659|12|24|19|PM|first|afternoon|lunch| +44660|AAAAAAAAFHOKAAAA|44660|12|24|20|PM|first|afternoon|lunch| +44661|AAAAAAAAGHOKAAAA|44661|12|24|21|PM|first|afternoon|lunch| +44662|AAAAAAAAHHOKAAAA|44662|12|24|22|PM|first|afternoon|lunch| +44663|AAAAAAAAIHOKAAAA|44663|12|24|23|PM|first|afternoon|lunch| +44664|AAAAAAAAJHOKAAAA|44664|12|24|24|PM|first|afternoon|lunch| +44665|AAAAAAAAKHOKAAAA|44665|12|24|25|PM|first|afternoon|lunch| +44666|AAAAAAAALHOKAAAA|44666|12|24|26|PM|first|afternoon|lunch| +44667|AAAAAAAAMHOKAAAA|44667|12|24|27|PM|first|afternoon|lunch| +44668|AAAAAAAANHOKAAAA|44668|12|24|28|PM|first|afternoon|lunch| +44669|AAAAAAAAOHOKAAAA|44669|12|24|29|PM|first|afternoon|lunch| +44670|AAAAAAAAPHOKAAAA|44670|12|24|30|PM|first|afternoon|lunch| +44671|AAAAAAAAAIOKAAAA|44671|12|24|31|PM|first|afternoon|lunch| +44672|AAAAAAAABIOKAAAA|44672|12|24|32|PM|first|afternoon|lunch| +44673|AAAAAAAACIOKAAAA|44673|12|24|33|PM|first|afternoon|lunch| +44674|AAAAAAAADIOKAAAA|44674|12|24|34|PM|first|afternoon|lunch| +44675|AAAAAAAAEIOKAAAA|44675|12|24|35|PM|first|afternoon|lunch| +44676|AAAAAAAAFIOKAAAA|44676|12|24|36|PM|first|afternoon|lunch| +44677|AAAAAAAAGIOKAAAA|44677|12|24|37|PM|first|afternoon|lunch| +44678|AAAAAAAAHIOKAAAA|44678|12|24|38|PM|first|afternoon|lunch| +44679|AAAAAAAAIIOKAAAA|44679|12|24|39|PM|first|afternoon|lunch| +44680|AAAAAAAAJIOKAAAA|44680|12|24|40|PM|first|afternoon|lunch| +44681|AAAAAAAAKIOKAAAA|44681|12|24|41|PM|first|afternoon|lunch| +44682|AAAAAAAALIOKAAAA|44682|12|24|42|PM|first|afternoon|lunch| +44683|AAAAAAAAMIOKAAAA|44683|12|24|43|PM|first|afternoon|lunch| +44684|AAAAAAAANIOKAAAA|44684|12|24|44|PM|first|afternoon|lunch| +44685|AAAAAAAAOIOKAAAA|44685|12|24|45|PM|first|afternoon|lunch| +44686|AAAAAAAAPIOKAAAA|44686|12|24|46|PM|first|afternoon|lunch| +44687|AAAAAAAAAJOKAAAA|44687|12|24|47|PM|first|afternoon|lunch| +44688|AAAAAAAABJOKAAAA|44688|12|24|48|PM|first|afternoon|lunch| +44689|AAAAAAAACJOKAAAA|44689|12|24|49|PM|first|afternoon|lunch| +44690|AAAAAAAADJOKAAAA|44690|12|24|50|PM|first|afternoon|lunch| +44691|AAAAAAAAEJOKAAAA|44691|12|24|51|PM|first|afternoon|lunch| +44692|AAAAAAAAFJOKAAAA|44692|12|24|52|PM|first|afternoon|lunch| +44693|AAAAAAAAGJOKAAAA|44693|12|24|53|PM|first|afternoon|lunch| +44694|AAAAAAAAHJOKAAAA|44694|12|24|54|PM|first|afternoon|lunch| +44695|AAAAAAAAIJOKAAAA|44695|12|24|55|PM|first|afternoon|lunch| +44696|AAAAAAAAJJOKAAAA|44696|12|24|56|PM|first|afternoon|lunch| +44697|AAAAAAAAKJOKAAAA|44697|12|24|57|PM|first|afternoon|lunch| +44698|AAAAAAAALJOKAAAA|44698|12|24|58|PM|first|afternoon|lunch| +44699|AAAAAAAAMJOKAAAA|44699|12|24|59|PM|first|afternoon|lunch| +44700|AAAAAAAANJOKAAAA|44700|12|25|0|PM|first|afternoon|lunch| +44701|AAAAAAAAOJOKAAAA|44701|12|25|1|PM|first|afternoon|lunch| +44702|AAAAAAAAPJOKAAAA|44702|12|25|2|PM|first|afternoon|lunch| +44703|AAAAAAAAAKOKAAAA|44703|12|25|3|PM|first|afternoon|lunch| +44704|AAAAAAAABKOKAAAA|44704|12|25|4|PM|first|afternoon|lunch| +44705|AAAAAAAACKOKAAAA|44705|12|25|5|PM|first|afternoon|lunch| +44706|AAAAAAAADKOKAAAA|44706|12|25|6|PM|first|afternoon|lunch| +44707|AAAAAAAAEKOKAAAA|44707|12|25|7|PM|first|afternoon|lunch| +44708|AAAAAAAAFKOKAAAA|44708|12|25|8|PM|first|afternoon|lunch| +44709|AAAAAAAAGKOKAAAA|44709|12|25|9|PM|first|afternoon|lunch| +44710|AAAAAAAAHKOKAAAA|44710|12|25|10|PM|first|afternoon|lunch| +44711|AAAAAAAAIKOKAAAA|44711|12|25|11|PM|first|afternoon|lunch| +44712|AAAAAAAAJKOKAAAA|44712|12|25|12|PM|first|afternoon|lunch| +44713|AAAAAAAAKKOKAAAA|44713|12|25|13|PM|first|afternoon|lunch| +44714|AAAAAAAALKOKAAAA|44714|12|25|14|PM|first|afternoon|lunch| +44715|AAAAAAAAMKOKAAAA|44715|12|25|15|PM|first|afternoon|lunch| +44716|AAAAAAAANKOKAAAA|44716|12|25|16|PM|first|afternoon|lunch| +44717|AAAAAAAAOKOKAAAA|44717|12|25|17|PM|first|afternoon|lunch| +44718|AAAAAAAAPKOKAAAA|44718|12|25|18|PM|first|afternoon|lunch| +44719|AAAAAAAAALOKAAAA|44719|12|25|19|PM|first|afternoon|lunch| +44720|AAAAAAAABLOKAAAA|44720|12|25|20|PM|first|afternoon|lunch| +44721|AAAAAAAACLOKAAAA|44721|12|25|21|PM|first|afternoon|lunch| +44722|AAAAAAAADLOKAAAA|44722|12|25|22|PM|first|afternoon|lunch| +44723|AAAAAAAAELOKAAAA|44723|12|25|23|PM|first|afternoon|lunch| +44724|AAAAAAAAFLOKAAAA|44724|12|25|24|PM|first|afternoon|lunch| +44725|AAAAAAAAGLOKAAAA|44725|12|25|25|PM|first|afternoon|lunch| +44726|AAAAAAAAHLOKAAAA|44726|12|25|26|PM|first|afternoon|lunch| +44727|AAAAAAAAILOKAAAA|44727|12|25|27|PM|first|afternoon|lunch| +44728|AAAAAAAAJLOKAAAA|44728|12|25|28|PM|first|afternoon|lunch| +44729|AAAAAAAAKLOKAAAA|44729|12|25|29|PM|first|afternoon|lunch| +44730|AAAAAAAALLOKAAAA|44730|12|25|30|PM|first|afternoon|lunch| +44731|AAAAAAAAMLOKAAAA|44731|12|25|31|PM|first|afternoon|lunch| +44732|AAAAAAAANLOKAAAA|44732|12|25|32|PM|first|afternoon|lunch| +44733|AAAAAAAAOLOKAAAA|44733|12|25|33|PM|first|afternoon|lunch| +44734|AAAAAAAAPLOKAAAA|44734|12|25|34|PM|first|afternoon|lunch| +44735|AAAAAAAAAMOKAAAA|44735|12|25|35|PM|first|afternoon|lunch| +44736|AAAAAAAABMOKAAAA|44736|12|25|36|PM|first|afternoon|lunch| +44737|AAAAAAAACMOKAAAA|44737|12|25|37|PM|first|afternoon|lunch| +44738|AAAAAAAADMOKAAAA|44738|12|25|38|PM|first|afternoon|lunch| +44739|AAAAAAAAEMOKAAAA|44739|12|25|39|PM|first|afternoon|lunch| +44740|AAAAAAAAFMOKAAAA|44740|12|25|40|PM|first|afternoon|lunch| +44741|AAAAAAAAGMOKAAAA|44741|12|25|41|PM|first|afternoon|lunch| +44742|AAAAAAAAHMOKAAAA|44742|12|25|42|PM|first|afternoon|lunch| +44743|AAAAAAAAIMOKAAAA|44743|12|25|43|PM|first|afternoon|lunch| +44744|AAAAAAAAJMOKAAAA|44744|12|25|44|PM|first|afternoon|lunch| +44745|AAAAAAAAKMOKAAAA|44745|12|25|45|PM|first|afternoon|lunch| +44746|AAAAAAAALMOKAAAA|44746|12|25|46|PM|first|afternoon|lunch| +44747|AAAAAAAAMMOKAAAA|44747|12|25|47|PM|first|afternoon|lunch| +44748|AAAAAAAANMOKAAAA|44748|12|25|48|PM|first|afternoon|lunch| +44749|AAAAAAAAOMOKAAAA|44749|12|25|49|PM|first|afternoon|lunch| +44750|AAAAAAAAPMOKAAAA|44750|12|25|50|PM|first|afternoon|lunch| +44751|AAAAAAAAANOKAAAA|44751|12|25|51|PM|first|afternoon|lunch| +44752|AAAAAAAABNOKAAAA|44752|12|25|52|PM|first|afternoon|lunch| +44753|AAAAAAAACNOKAAAA|44753|12|25|53|PM|first|afternoon|lunch| +44754|AAAAAAAADNOKAAAA|44754|12|25|54|PM|first|afternoon|lunch| +44755|AAAAAAAAENOKAAAA|44755|12|25|55|PM|first|afternoon|lunch| +44756|AAAAAAAAFNOKAAAA|44756|12|25|56|PM|first|afternoon|lunch| +44757|AAAAAAAAGNOKAAAA|44757|12|25|57|PM|first|afternoon|lunch| +44758|AAAAAAAAHNOKAAAA|44758|12|25|58|PM|first|afternoon|lunch| +44759|AAAAAAAAINOKAAAA|44759|12|25|59|PM|first|afternoon|lunch| +44760|AAAAAAAAJNOKAAAA|44760|12|26|0|PM|first|afternoon|lunch| +44761|AAAAAAAAKNOKAAAA|44761|12|26|1|PM|first|afternoon|lunch| +44762|AAAAAAAALNOKAAAA|44762|12|26|2|PM|first|afternoon|lunch| +44763|AAAAAAAAMNOKAAAA|44763|12|26|3|PM|first|afternoon|lunch| +44764|AAAAAAAANNOKAAAA|44764|12|26|4|PM|first|afternoon|lunch| +44765|AAAAAAAAONOKAAAA|44765|12|26|5|PM|first|afternoon|lunch| +44766|AAAAAAAAPNOKAAAA|44766|12|26|6|PM|first|afternoon|lunch| +44767|AAAAAAAAAOOKAAAA|44767|12|26|7|PM|first|afternoon|lunch| +44768|AAAAAAAABOOKAAAA|44768|12|26|8|PM|first|afternoon|lunch| +44769|AAAAAAAACOOKAAAA|44769|12|26|9|PM|first|afternoon|lunch| +44770|AAAAAAAADOOKAAAA|44770|12|26|10|PM|first|afternoon|lunch| +44771|AAAAAAAAEOOKAAAA|44771|12|26|11|PM|first|afternoon|lunch| +44772|AAAAAAAAFOOKAAAA|44772|12|26|12|PM|first|afternoon|lunch| +44773|AAAAAAAAGOOKAAAA|44773|12|26|13|PM|first|afternoon|lunch| +44774|AAAAAAAAHOOKAAAA|44774|12|26|14|PM|first|afternoon|lunch| +44775|AAAAAAAAIOOKAAAA|44775|12|26|15|PM|first|afternoon|lunch| +44776|AAAAAAAAJOOKAAAA|44776|12|26|16|PM|first|afternoon|lunch| +44777|AAAAAAAAKOOKAAAA|44777|12|26|17|PM|first|afternoon|lunch| +44778|AAAAAAAALOOKAAAA|44778|12|26|18|PM|first|afternoon|lunch| +44779|AAAAAAAAMOOKAAAA|44779|12|26|19|PM|first|afternoon|lunch| +44780|AAAAAAAANOOKAAAA|44780|12|26|20|PM|first|afternoon|lunch| +44781|AAAAAAAAOOOKAAAA|44781|12|26|21|PM|first|afternoon|lunch| +44782|AAAAAAAAPOOKAAAA|44782|12|26|22|PM|first|afternoon|lunch| +44783|AAAAAAAAAPOKAAAA|44783|12|26|23|PM|first|afternoon|lunch| +44784|AAAAAAAABPOKAAAA|44784|12|26|24|PM|first|afternoon|lunch| +44785|AAAAAAAACPOKAAAA|44785|12|26|25|PM|first|afternoon|lunch| +44786|AAAAAAAADPOKAAAA|44786|12|26|26|PM|first|afternoon|lunch| +44787|AAAAAAAAEPOKAAAA|44787|12|26|27|PM|first|afternoon|lunch| +44788|AAAAAAAAFPOKAAAA|44788|12|26|28|PM|first|afternoon|lunch| +44789|AAAAAAAAGPOKAAAA|44789|12|26|29|PM|first|afternoon|lunch| +44790|AAAAAAAAHPOKAAAA|44790|12|26|30|PM|first|afternoon|lunch| +44791|AAAAAAAAIPOKAAAA|44791|12|26|31|PM|first|afternoon|lunch| +44792|AAAAAAAAJPOKAAAA|44792|12|26|32|PM|first|afternoon|lunch| +44793|AAAAAAAAKPOKAAAA|44793|12|26|33|PM|first|afternoon|lunch| +44794|AAAAAAAALPOKAAAA|44794|12|26|34|PM|first|afternoon|lunch| +44795|AAAAAAAAMPOKAAAA|44795|12|26|35|PM|first|afternoon|lunch| +44796|AAAAAAAANPOKAAAA|44796|12|26|36|PM|first|afternoon|lunch| +44797|AAAAAAAAOPOKAAAA|44797|12|26|37|PM|first|afternoon|lunch| +44798|AAAAAAAAPPOKAAAA|44798|12|26|38|PM|first|afternoon|lunch| +44799|AAAAAAAAAAPKAAAA|44799|12|26|39|PM|first|afternoon|lunch| +44800|AAAAAAAABAPKAAAA|44800|12|26|40|PM|first|afternoon|lunch| +44801|AAAAAAAACAPKAAAA|44801|12|26|41|PM|first|afternoon|lunch| +44802|AAAAAAAADAPKAAAA|44802|12|26|42|PM|first|afternoon|lunch| +44803|AAAAAAAAEAPKAAAA|44803|12|26|43|PM|first|afternoon|lunch| +44804|AAAAAAAAFAPKAAAA|44804|12|26|44|PM|first|afternoon|lunch| +44805|AAAAAAAAGAPKAAAA|44805|12|26|45|PM|first|afternoon|lunch| +44806|AAAAAAAAHAPKAAAA|44806|12|26|46|PM|first|afternoon|lunch| +44807|AAAAAAAAIAPKAAAA|44807|12|26|47|PM|first|afternoon|lunch| +44808|AAAAAAAAJAPKAAAA|44808|12|26|48|PM|first|afternoon|lunch| +44809|AAAAAAAAKAPKAAAA|44809|12|26|49|PM|first|afternoon|lunch| +44810|AAAAAAAALAPKAAAA|44810|12|26|50|PM|first|afternoon|lunch| +44811|AAAAAAAAMAPKAAAA|44811|12|26|51|PM|first|afternoon|lunch| +44812|AAAAAAAANAPKAAAA|44812|12|26|52|PM|first|afternoon|lunch| +44813|AAAAAAAAOAPKAAAA|44813|12|26|53|PM|first|afternoon|lunch| +44814|AAAAAAAAPAPKAAAA|44814|12|26|54|PM|first|afternoon|lunch| +44815|AAAAAAAAABPKAAAA|44815|12|26|55|PM|first|afternoon|lunch| +44816|AAAAAAAABBPKAAAA|44816|12|26|56|PM|first|afternoon|lunch| +44817|AAAAAAAACBPKAAAA|44817|12|26|57|PM|first|afternoon|lunch| +44818|AAAAAAAADBPKAAAA|44818|12|26|58|PM|first|afternoon|lunch| +44819|AAAAAAAAEBPKAAAA|44819|12|26|59|PM|first|afternoon|lunch| +44820|AAAAAAAAFBPKAAAA|44820|12|27|0|PM|first|afternoon|lunch| +44821|AAAAAAAAGBPKAAAA|44821|12|27|1|PM|first|afternoon|lunch| +44822|AAAAAAAAHBPKAAAA|44822|12|27|2|PM|first|afternoon|lunch| +44823|AAAAAAAAIBPKAAAA|44823|12|27|3|PM|first|afternoon|lunch| +44824|AAAAAAAAJBPKAAAA|44824|12|27|4|PM|first|afternoon|lunch| +44825|AAAAAAAAKBPKAAAA|44825|12|27|5|PM|first|afternoon|lunch| +44826|AAAAAAAALBPKAAAA|44826|12|27|6|PM|first|afternoon|lunch| +44827|AAAAAAAAMBPKAAAA|44827|12|27|7|PM|first|afternoon|lunch| +44828|AAAAAAAANBPKAAAA|44828|12|27|8|PM|first|afternoon|lunch| +44829|AAAAAAAAOBPKAAAA|44829|12|27|9|PM|first|afternoon|lunch| +44830|AAAAAAAAPBPKAAAA|44830|12|27|10|PM|first|afternoon|lunch| +44831|AAAAAAAAACPKAAAA|44831|12|27|11|PM|first|afternoon|lunch| +44832|AAAAAAAABCPKAAAA|44832|12|27|12|PM|first|afternoon|lunch| +44833|AAAAAAAACCPKAAAA|44833|12|27|13|PM|first|afternoon|lunch| +44834|AAAAAAAADCPKAAAA|44834|12|27|14|PM|first|afternoon|lunch| +44835|AAAAAAAAECPKAAAA|44835|12|27|15|PM|first|afternoon|lunch| +44836|AAAAAAAAFCPKAAAA|44836|12|27|16|PM|first|afternoon|lunch| +44837|AAAAAAAAGCPKAAAA|44837|12|27|17|PM|first|afternoon|lunch| +44838|AAAAAAAAHCPKAAAA|44838|12|27|18|PM|first|afternoon|lunch| +44839|AAAAAAAAICPKAAAA|44839|12|27|19|PM|first|afternoon|lunch| +44840|AAAAAAAAJCPKAAAA|44840|12|27|20|PM|first|afternoon|lunch| +44841|AAAAAAAAKCPKAAAA|44841|12|27|21|PM|first|afternoon|lunch| +44842|AAAAAAAALCPKAAAA|44842|12|27|22|PM|first|afternoon|lunch| +44843|AAAAAAAAMCPKAAAA|44843|12|27|23|PM|first|afternoon|lunch| +44844|AAAAAAAANCPKAAAA|44844|12|27|24|PM|first|afternoon|lunch| +44845|AAAAAAAAOCPKAAAA|44845|12|27|25|PM|first|afternoon|lunch| +44846|AAAAAAAAPCPKAAAA|44846|12|27|26|PM|first|afternoon|lunch| +44847|AAAAAAAAADPKAAAA|44847|12|27|27|PM|first|afternoon|lunch| +44848|AAAAAAAABDPKAAAA|44848|12|27|28|PM|first|afternoon|lunch| +44849|AAAAAAAACDPKAAAA|44849|12|27|29|PM|first|afternoon|lunch| +44850|AAAAAAAADDPKAAAA|44850|12|27|30|PM|first|afternoon|lunch| +44851|AAAAAAAAEDPKAAAA|44851|12|27|31|PM|first|afternoon|lunch| +44852|AAAAAAAAFDPKAAAA|44852|12|27|32|PM|first|afternoon|lunch| +44853|AAAAAAAAGDPKAAAA|44853|12|27|33|PM|first|afternoon|lunch| +44854|AAAAAAAAHDPKAAAA|44854|12|27|34|PM|first|afternoon|lunch| +44855|AAAAAAAAIDPKAAAA|44855|12|27|35|PM|first|afternoon|lunch| +44856|AAAAAAAAJDPKAAAA|44856|12|27|36|PM|first|afternoon|lunch| +44857|AAAAAAAAKDPKAAAA|44857|12|27|37|PM|first|afternoon|lunch| +44858|AAAAAAAALDPKAAAA|44858|12|27|38|PM|first|afternoon|lunch| +44859|AAAAAAAAMDPKAAAA|44859|12|27|39|PM|first|afternoon|lunch| +44860|AAAAAAAANDPKAAAA|44860|12|27|40|PM|first|afternoon|lunch| +44861|AAAAAAAAODPKAAAA|44861|12|27|41|PM|first|afternoon|lunch| +44862|AAAAAAAAPDPKAAAA|44862|12|27|42|PM|first|afternoon|lunch| +44863|AAAAAAAAAEPKAAAA|44863|12|27|43|PM|first|afternoon|lunch| +44864|AAAAAAAABEPKAAAA|44864|12|27|44|PM|first|afternoon|lunch| +44865|AAAAAAAACEPKAAAA|44865|12|27|45|PM|first|afternoon|lunch| +44866|AAAAAAAADEPKAAAA|44866|12|27|46|PM|first|afternoon|lunch| +44867|AAAAAAAAEEPKAAAA|44867|12|27|47|PM|first|afternoon|lunch| +44868|AAAAAAAAFEPKAAAA|44868|12|27|48|PM|first|afternoon|lunch| +44869|AAAAAAAAGEPKAAAA|44869|12|27|49|PM|first|afternoon|lunch| +44870|AAAAAAAAHEPKAAAA|44870|12|27|50|PM|first|afternoon|lunch| +44871|AAAAAAAAIEPKAAAA|44871|12|27|51|PM|first|afternoon|lunch| +44872|AAAAAAAAJEPKAAAA|44872|12|27|52|PM|first|afternoon|lunch| +44873|AAAAAAAAKEPKAAAA|44873|12|27|53|PM|first|afternoon|lunch| +44874|AAAAAAAALEPKAAAA|44874|12|27|54|PM|first|afternoon|lunch| +44875|AAAAAAAAMEPKAAAA|44875|12|27|55|PM|first|afternoon|lunch| +44876|AAAAAAAANEPKAAAA|44876|12|27|56|PM|first|afternoon|lunch| +44877|AAAAAAAAOEPKAAAA|44877|12|27|57|PM|first|afternoon|lunch| +44878|AAAAAAAAPEPKAAAA|44878|12|27|58|PM|first|afternoon|lunch| +44879|AAAAAAAAAFPKAAAA|44879|12|27|59|PM|first|afternoon|lunch| +44880|AAAAAAAABFPKAAAA|44880|12|28|0|PM|first|afternoon|lunch| +44881|AAAAAAAACFPKAAAA|44881|12|28|1|PM|first|afternoon|lunch| +44882|AAAAAAAADFPKAAAA|44882|12|28|2|PM|first|afternoon|lunch| +44883|AAAAAAAAEFPKAAAA|44883|12|28|3|PM|first|afternoon|lunch| +44884|AAAAAAAAFFPKAAAA|44884|12|28|4|PM|first|afternoon|lunch| +44885|AAAAAAAAGFPKAAAA|44885|12|28|5|PM|first|afternoon|lunch| +44886|AAAAAAAAHFPKAAAA|44886|12|28|6|PM|first|afternoon|lunch| +44887|AAAAAAAAIFPKAAAA|44887|12|28|7|PM|first|afternoon|lunch| +44888|AAAAAAAAJFPKAAAA|44888|12|28|8|PM|first|afternoon|lunch| +44889|AAAAAAAAKFPKAAAA|44889|12|28|9|PM|first|afternoon|lunch| +44890|AAAAAAAALFPKAAAA|44890|12|28|10|PM|first|afternoon|lunch| +44891|AAAAAAAAMFPKAAAA|44891|12|28|11|PM|first|afternoon|lunch| +44892|AAAAAAAANFPKAAAA|44892|12|28|12|PM|first|afternoon|lunch| +44893|AAAAAAAAOFPKAAAA|44893|12|28|13|PM|first|afternoon|lunch| +44894|AAAAAAAAPFPKAAAA|44894|12|28|14|PM|first|afternoon|lunch| +44895|AAAAAAAAAGPKAAAA|44895|12|28|15|PM|first|afternoon|lunch| +44896|AAAAAAAABGPKAAAA|44896|12|28|16|PM|first|afternoon|lunch| +44897|AAAAAAAACGPKAAAA|44897|12|28|17|PM|first|afternoon|lunch| +44898|AAAAAAAADGPKAAAA|44898|12|28|18|PM|first|afternoon|lunch| +44899|AAAAAAAAEGPKAAAA|44899|12|28|19|PM|first|afternoon|lunch| +44900|AAAAAAAAFGPKAAAA|44900|12|28|20|PM|first|afternoon|lunch| +44901|AAAAAAAAGGPKAAAA|44901|12|28|21|PM|first|afternoon|lunch| +44902|AAAAAAAAHGPKAAAA|44902|12|28|22|PM|first|afternoon|lunch| +44903|AAAAAAAAIGPKAAAA|44903|12|28|23|PM|first|afternoon|lunch| +44904|AAAAAAAAJGPKAAAA|44904|12|28|24|PM|first|afternoon|lunch| +44905|AAAAAAAAKGPKAAAA|44905|12|28|25|PM|first|afternoon|lunch| +44906|AAAAAAAALGPKAAAA|44906|12|28|26|PM|first|afternoon|lunch| +44907|AAAAAAAAMGPKAAAA|44907|12|28|27|PM|first|afternoon|lunch| +44908|AAAAAAAANGPKAAAA|44908|12|28|28|PM|first|afternoon|lunch| +44909|AAAAAAAAOGPKAAAA|44909|12|28|29|PM|first|afternoon|lunch| +44910|AAAAAAAAPGPKAAAA|44910|12|28|30|PM|first|afternoon|lunch| +44911|AAAAAAAAAHPKAAAA|44911|12|28|31|PM|first|afternoon|lunch| +44912|AAAAAAAABHPKAAAA|44912|12|28|32|PM|first|afternoon|lunch| +44913|AAAAAAAACHPKAAAA|44913|12|28|33|PM|first|afternoon|lunch| +44914|AAAAAAAADHPKAAAA|44914|12|28|34|PM|first|afternoon|lunch| +44915|AAAAAAAAEHPKAAAA|44915|12|28|35|PM|first|afternoon|lunch| +44916|AAAAAAAAFHPKAAAA|44916|12|28|36|PM|first|afternoon|lunch| +44917|AAAAAAAAGHPKAAAA|44917|12|28|37|PM|first|afternoon|lunch| +44918|AAAAAAAAHHPKAAAA|44918|12|28|38|PM|first|afternoon|lunch| +44919|AAAAAAAAIHPKAAAA|44919|12|28|39|PM|first|afternoon|lunch| +44920|AAAAAAAAJHPKAAAA|44920|12|28|40|PM|first|afternoon|lunch| +44921|AAAAAAAAKHPKAAAA|44921|12|28|41|PM|first|afternoon|lunch| +44922|AAAAAAAALHPKAAAA|44922|12|28|42|PM|first|afternoon|lunch| +44923|AAAAAAAAMHPKAAAA|44923|12|28|43|PM|first|afternoon|lunch| +44924|AAAAAAAANHPKAAAA|44924|12|28|44|PM|first|afternoon|lunch| +44925|AAAAAAAAOHPKAAAA|44925|12|28|45|PM|first|afternoon|lunch| +44926|AAAAAAAAPHPKAAAA|44926|12|28|46|PM|first|afternoon|lunch| +44927|AAAAAAAAAIPKAAAA|44927|12|28|47|PM|first|afternoon|lunch| +44928|AAAAAAAABIPKAAAA|44928|12|28|48|PM|first|afternoon|lunch| +44929|AAAAAAAACIPKAAAA|44929|12|28|49|PM|first|afternoon|lunch| +44930|AAAAAAAADIPKAAAA|44930|12|28|50|PM|first|afternoon|lunch| +44931|AAAAAAAAEIPKAAAA|44931|12|28|51|PM|first|afternoon|lunch| +44932|AAAAAAAAFIPKAAAA|44932|12|28|52|PM|first|afternoon|lunch| +44933|AAAAAAAAGIPKAAAA|44933|12|28|53|PM|first|afternoon|lunch| +44934|AAAAAAAAHIPKAAAA|44934|12|28|54|PM|first|afternoon|lunch| +44935|AAAAAAAAIIPKAAAA|44935|12|28|55|PM|first|afternoon|lunch| +44936|AAAAAAAAJIPKAAAA|44936|12|28|56|PM|first|afternoon|lunch| +44937|AAAAAAAAKIPKAAAA|44937|12|28|57|PM|first|afternoon|lunch| +44938|AAAAAAAALIPKAAAA|44938|12|28|58|PM|first|afternoon|lunch| +44939|AAAAAAAAMIPKAAAA|44939|12|28|59|PM|first|afternoon|lunch| +44940|AAAAAAAANIPKAAAA|44940|12|29|0|PM|first|afternoon|lunch| +44941|AAAAAAAAOIPKAAAA|44941|12|29|1|PM|first|afternoon|lunch| +44942|AAAAAAAAPIPKAAAA|44942|12|29|2|PM|first|afternoon|lunch| +44943|AAAAAAAAAJPKAAAA|44943|12|29|3|PM|first|afternoon|lunch| +44944|AAAAAAAABJPKAAAA|44944|12|29|4|PM|first|afternoon|lunch| +44945|AAAAAAAACJPKAAAA|44945|12|29|5|PM|first|afternoon|lunch| +44946|AAAAAAAADJPKAAAA|44946|12|29|6|PM|first|afternoon|lunch| +44947|AAAAAAAAEJPKAAAA|44947|12|29|7|PM|first|afternoon|lunch| +44948|AAAAAAAAFJPKAAAA|44948|12|29|8|PM|first|afternoon|lunch| +44949|AAAAAAAAGJPKAAAA|44949|12|29|9|PM|first|afternoon|lunch| +44950|AAAAAAAAHJPKAAAA|44950|12|29|10|PM|first|afternoon|lunch| +44951|AAAAAAAAIJPKAAAA|44951|12|29|11|PM|first|afternoon|lunch| +44952|AAAAAAAAJJPKAAAA|44952|12|29|12|PM|first|afternoon|lunch| +44953|AAAAAAAAKJPKAAAA|44953|12|29|13|PM|first|afternoon|lunch| +44954|AAAAAAAALJPKAAAA|44954|12|29|14|PM|first|afternoon|lunch| +44955|AAAAAAAAMJPKAAAA|44955|12|29|15|PM|first|afternoon|lunch| +44956|AAAAAAAANJPKAAAA|44956|12|29|16|PM|first|afternoon|lunch| +44957|AAAAAAAAOJPKAAAA|44957|12|29|17|PM|first|afternoon|lunch| +44958|AAAAAAAAPJPKAAAA|44958|12|29|18|PM|first|afternoon|lunch| +44959|AAAAAAAAAKPKAAAA|44959|12|29|19|PM|first|afternoon|lunch| +44960|AAAAAAAABKPKAAAA|44960|12|29|20|PM|first|afternoon|lunch| +44961|AAAAAAAACKPKAAAA|44961|12|29|21|PM|first|afternoon|lunch| +44962|AAAAAAAADKPKAAAA|44962|12|29|22|PM|first|afternoon|lunch| +44963|AAAAAAAAEKPKAAAA|44963|12|29|23|PM|first|afternoon|lunch| +44964|AAAAAAAAFKPKAAAA|44964|12|29|24|PM|first|afternoon|lunch| +44965|AAAAAAAAGKPKAAAA|44965|12|29|25|PM|first|afternoon|lunch| +44966|AAAAAAAAHKPKAAAA|44966|12|29|26|PM|first|afternoon|lunch| +44967|AAAAAAAAIKPKAAAA|44967|12|29|27|PM|first|afternoon|lunch| +44968|AAAAAAAAJKPKAAAA|44968|12|29|28|PM|first|afternoon|lunch| +44969|AAAAAAAAKKPKAAAA|44969|12|29|29|PM|first|afternoon|lunch| +44970|AAAAAAAALKPKAAAA|44970|12|29|30|PM|first|afternoon|lunch| +44971|AAAAAAAAMKPKAAAA|44971|12|29|31|PM|first|afternoon|lunch| +44972|AAAAAAAANKPKAAAA|44972|12|29|32|PM|first|afternoon|lunch| +44973|AAAAAAAAOKPKAAAA|44973|12|29|33|PM|first|afternoon|lunch| +44974|AAAAAAAAPKPKAAAA|44974|12|29|34|PM|first|afternoon|lunch| +44975|AAAAAAAAALPKAAAA|44975|12|29|35|PM|first|afternoon|lunch| +44976|AAAAAAAABLPKAAAA|44976|12|29|36|PM|first|afternoon|lunch| +44977|AAAAAAAACLPKAAAA|44977|12|29|37|PM|first|afternoon|lunch| +44978|AAAAAAAADLPKAAAA|44978|12|29|38|PM|first|afternoon|lunch| +44979|AAAAAAAAELPKAAAA|44979|12|29|39|PM|first|afternoon|lunch| +44980|AAAAAAAAFLPKAAAA|44980|12|29|40|PM|first|afternoon|lunch| +44981|AAAAAAAAGLPKAAAA|44981|12|29|41|PM|first|afternoon|lunch| +44982|AAAAAAAAHLPKAAAA|44982|12|29|42|PM|first|afternoon|lunch| +44983|AAAAAAAAILPKAAAA|44983|12|29|43|PM|first|afternoon|lunch| +44984|AAAAAAAAJLPKAAAA|44984|12|29|44|PM|first|afternoon|lunch| +44985|AAAAAAAAKLPKAAAA|44985|12|29|45|PM|first|afternoon|lunch| +44986|AAAAAAAALLPKAAAA|44986|12|29|46|PM|first|afternoon|lunch| +44987|AAAAAAAAMLPKAAAA|44987|12|29|47|PM|first|afternoon|lunch| +44988|AAAAAAAANLPKAAAA|44988|12|29|48|PM|first|afternoon|lunch| +44989|AAAAAAAAOLPKAAAA|44989|12|29|49|PM|first|afternoon|lunch| +44990|AAAAAAAAPLPKAAAA|44990|12|29|50|PM|first|afternoon|lunch| +44991|AAAAAAAAAMPKAAAA|44991|12|29|51|PM|first|afternoon|lunch| +44992|AAAAAAAABMPKAAAA|44992|12|29|52|PM|first|afternoon|lunch| +44993|AAAAAAAACMPKAAAA|44993|12|29|53|PM|first|afternoon|lunch| +44994|AAAAAAAADMPKAAAA|44994|12|29|54|PM|first|afternoon|lunch| +44995|AAAAAAAAEMPKAAAA|44995|12|29|55|PM|first|afternoon|lunch| +44996|AAAAAAAAFMPKAAAA|44996|12|29|56|PM|first|afternoon|lunch| +44997|AAAAAAAAGMPKAAAA|44997|12|29|57|PM|first|afternoon|lunch| +44998|AAAAAAAAHMPKAAAA|44998|12|29|58|PM|first|afternoon|lunch| +44999|AAAAAAAAIMPKAAAA|44999|12|29|59|PM|first|afternoon|lunch| +45000|AAAAAAAAJMPKAAAA|45000|12|30|0|PM|first|afternoon|lunch| +45001|AAAAAAAAKMPKAAAA|45001|12|30|1|PM|first|afternoon|lunch| +45002|AAAAAAAALMPKAAAA|45002|12|30|2|PM|first|afternoon|lunch| +45003|AAAAAAAAMMPKAAAA|45003|12|30|3|PM|first|afternoon|lunch| +45004|AAAAAAAANMPKAAAA|45004|12|30|4|PM|first|afternoon|lunch| +45005|AAAAAAAAOMPKAAAA|45005|12|30|5|PM|first|afternoon|lunch| +45006|AAAAAAAAPMPKAAAA|45006|12|30|6|PM|first|afternoon|lunch| +45007|AAAAAAAAANPKAAAA|45007|12|30|7|PM|first|afternoon|lunch| +45008|AAAAAAAABNPKAAAA|45008|12|30|8|PM|first|afternoon|lunch| +45009|AAAAAAAACNPKAAAA|45009|12|30|9|PM|first|afternoon|lunch| +45010|AAAAAAAADNPKAAAA|45010|12|30|10|PM|first|afternoon|lunch| +45011|AAAAAAAAENPKAAAA|45011|12|30|11|PM|first|afternoon|lunch| +45012|AAAAAAAAFNPKAAAA|45012|12|30|12|PM|first|afternoon|lunch| +45013|AAAAAAAAGNPKAAAA|45013|12|30|13|PM|first|afternoon|lunch| +45014|AAAAAAAAHNPKAAAA|45014|12|30|14|PM|first|afternoon|lunch| +45015|AAAAAAAAINPKAAAA|45015|12|30|15|PM|first|afternoon|lunch| +45016|AAAAAAAAJNPKAAAA|45016|12|30|16|PM|first|afternoon|lunch| +45017|AAAAAAAAKNPKAAAA|45017|12|30|17|PM|first|afternoon|lunch| +45018|AAAAAAAALNPKAAAA|45018|12|30|18|PM|first|afternoon|lunch| +45019|AAAAAAAAMNPKAAAA|45019|12|30|19|PM|first|afternoon|lunch| +45020|AAAAAAAANNPKAAAA|45020|12|30|20|PM|first|afternoon|lunch| +45021|AAAAAAAAONPKAAAA|45021|12|30|21|PM|first|afternoon|lunch| +45022|AAAAAAAAPNPKAAAA|45022|12|30|22|PM|first|afternoon|lunch| +45023|AAAAAAAAAOPKAAAA|45023|12|30|23|PM|first|afternoon|lunch| +45024|AAAAAAAABOPKAAAA|45024|12|30|24|PM|first|afternoon|lunch| +45025|AAAAAAAACOPKAAAA|45025|12|30|25|PM|first|afternoon|lunch| +45026|AAAAAAAADOPKAAAA|45026|12|30|26|PM|first|afternoon|lunch| +45027|AAAAAAAAEOPKAAAA|45027|12|30|27|PM|first|afternoon|lunch| +45028|AAAAAAAAFOPKAAAA|45028|12|30|28|PM|first|afternoon|lunch| +45029|AAAAAAAAGOPKAAAA|45029|12|30|29|PM|first|afternoon|lunch| +45030|AAAAAAAAHOPKAAAA|45030|12|30|30|PM|first|afternoon|lunch| +45031|AAAAAAAAIOPKAAAA|45031|12|30|31|PM|first|afternoon|lunch| +45032|AAAAAAAAJOPKAAAA|45032|12|30|32|PM|first|afternoon|lunch| +45033|AAAAAAAAKOPKAAAA|45033|12|30|33|PM|first|afternoon|lunch| +45034|AAAAAAAALOPKAAAA|45034|12|30|34|PM|first|afternoon|lunch| +45035|AAAAAAAAMOPKAAAA|45035|12|30|35|PM|first|afternoon|lunch| +45036|AAAAAAAANOPKAAAA|45036|12|30|36|PM|first|afternoon|lunch| +45037|AAAAAAAAOOPKAAAA|45037|12|30|37|PM|first|afternoon|lunch| +45038|AAAAAAAAPOPKAAAA|45038|12|30|38|PM|first|afternoon|lunch| +45039|AAAAAAAAAPPKAAAA|45039|12|30|39|PM|first|afternoon|lunch| +45040|AAAAAAAABPPKAAAA|45040|12|30|40|PM|first|afternoon|lunch| +45041|AAAAAAAACPPKAAAA|45041|12|30|41|PM|first|afternoon|lunch| +45042|AAAAAAAADPPKAAAA|45042|12|30|42|PM|first|afternoon|lunch| +45043|AAAAAAAAEPPKAAAA|45043|12|30|43|PM|first|afternoon|lunch| +45044|AAAAAAAAFPPKAAAA|45044|12|30|44|PM|first|afternoon|lunch| +45045|AAAAAAAAGPPKAAAA|45045|12|30|45|PM|first|afternoon|lunch| +45046|AAAAAAAAHPPKAAAA|45046|12|30|46|PM|first|afternoon|lunch| +45047|AAAAAAAAIPPKAAAA|45047|12|30|47|PM|first|afternoon|lunch| +45048|AAAAAAAAJPPKAAAA|45048|12|30|48|PM|first|afternoon|lunch| +45049|AAAAAAAAKPPKAAAA|45049|12|30|49|PM|first|afternoon|lunch| +45050|AAAAAAAALPPKAAAA|45050|12|30|50|PM|first|afternoon|lunch| +45051|AAAAAAAAMPPKAAAA|45051|12|30|51|PM|first|afternoon|lunch| +45052|AAAAAAAANPPKAAAA|45052|12|30|52|PM|first|afternoon|lunch| +45053|AAAAAAAAOPPKAAAA|45053|12|30|53|PM|first|afternoon|lunch| +45054|AAAAAAAAPPPKAAAA|45054|12|30|54|PM|first|afternoon|lunch| +45055|AAAAAAAAAAALAAAA|45055|12|30|55|PM|first|afternoon|lunch| +45056|AAAAAAAABAALAAAA|45056|12|30|56|PM|first|afternoon|lunch| +45057|AAAAAAAACAALAAAA|45057|12|30|57|PM|first|afternoon|lunch| +45058|AAAAAAAADAALAAAA|45058|12|30|58|PM|first|afternoon|lunch| +45059|AAAAAAAAEAALAAAA|45059|12|30|59|PM|first|afternoon|lunch| +45060|AAAAAAAAFAALAAAA|45060|12|31|0|PM|first|afternoon|lunch| +45061|AAAAAAAAGAALAAAA|45061|12|31|1|PM|first|afternoon|lunch| +45062|AAAAAAAAHAALAAAA|45062|12|31|2|PM|first|afternoon|lunch| +45063|AAAAAAAAIAALAAAA|45063|12|31|3|PM|first|afternoon|lunch| +45064|AAAAAAAAJAALAAAA|45064|12|31|4|PM|first|afternoon|lunch| +45065|AAAAAAAAKAALAAAA|45065|12|31|5|PM|first|afternoon|lunch| +45066|AAAAAAAALAALAAAA|45066|12|31|6|PM|first|afternoon|lunch| +45067|AAAAAAAAMAALAAAA|45067|12|31|7|PM|first|afternoon|lunch| +45068|AAAAAAAANAALAAAA|45068|12|31|8|PM|first|afternoon|lunch| +45069|AAAAAAAAOAALAAAA|45069|12|31|9|PM|first|afternoon|lunch| +45070|AAAAAAAAPAALAAAA|45070|12|31|10|PM|first|afternoon|lunch| +45071|AAAAAAAAABALAAAA|45071|12|31|11|PM|first|afternoon|lunch| +45072|AAAAAAAABBALAAAA|45072|12|31|12|PM|first|afternoon|lunch| +45073|AAAAAAAACBALAAAA|45073|12|31|13|PM|first|afternoon|lunch| +45074|AAAAAAAADBALAAAA|45074|12|31|14|PM|first|afternoon|lunch| +45075|AAAAAAAAEBALAAAA|45075|12|31|15|PM|first|afternoon|lunch| +45076|AAAAAAAAFBALAAAA|45076|12|31|16|PM|first|afternoon|lunch| +45077|AAAAAAAAGBALAAAA|45077|12|31|17|PM|first|afternoon|lunch| +45078|AAAAAAAAHBALAAAA|45078|12|31|18|PM|first|afternoon|lunch| +45079|AAAAAAAAIBALAAAA|45079|12|31|19|PM|first|afternoon|lunch| +45080|AAAAAAAAJBALAAAA|45080|12|31|20|PM|first|afternoon|lunch| +45081|AAAAAAAAKBALAAAA|45081|12|31|21|PM|first|afternoon|lunch| +45082|AAAAAAAALBALAAAA|45082|12|31|22|PM|first|afternoon|lunch| +45083|AAAAAAAAMBALAAAA|45083|12|31|23|PM|first|afternoon|lunch| +45084|AAAAAAAANBALAAAA|45084|12|31|24|PM|first|afternoon|lunch| +45085|AAAAAAAAOBALAAAA|45085|12|31|25|PM|first|afternoon|lunch| +45086|AAAAAAAAPBALAAAA|45086|12|31|26|PM|first|afternoon|lunch| +45087|AAAAAAAAACALAAAA|45087|12|31|27|PM|first|afternoon|lunch| +45088|AAAAAAAABCALAAAA|45088|12|31|28|PM|first|afternoon|lunch| +45089|AAAAAAAACCALAAAA|45089|12|31|29|PM|first|afternoon|lunch| +45090|AAAAAAAADCALAAAA|45090|12|31|30|PM|first|afternoon|lunch| +45091|AAAAAAAAECALAAAA|45091|12|31|31|PM|first|afternoon|lunch| +45092|AAAAAAAAFCALAAAA|45092|12|31|32|PM|first|afternoon|lunch| +45093|AAAAAAAAGCALAAAA|45093|12|31|33|PM|first|afternoon|lunch| +45094|AAAAAAAAHCALAAAA|45094|12|31|34|PM|first|afternoon|lunch| +45095|AAAAAAAAICALAAAA|45095|12|31|35|PM|first|afternoon|lunch| +45096|AAAAAAAAJCALAAAA|45096|12|31|36|PM|first|afternoon|lunch| +45097|AAAAAAAAKCALAAAA|45097|12|31|37|PM|first|afternoon|lunch| +45098|AAAAAAAALCALAAAA|45098|12|31|38|PM|first|afternoon|lunch| +45099|AAAAAAAAMCALAAAA|45099|12|31|39|PM|first|afternoon|lunch| +45100|AAAAAAAANCALAAAA|45100|12|31|40|PM|first|afternoon|lunch| +45101|AAAAAAAAOCALAAAA|45101|12|31|41|PM|first|afternoon|lunch| +45102|AAAAAAAAPCALAAAA|45102|12|31|42|PM|first|afternoon|lunch| +45103|AAAAAAAAADALAAAA|45103|12|31|43|PM|first|afternoon|lunch| +45104|AAAAAAAABDALAAAA|45104|12|31|44|PM|first|afternoon|lunch| +45105|AAAAAAAACDALAAAA|45105|12|31|45|PM|first|afternoon|lunch| +45106|AAAAAAAADDALAAAA|45106|12|31|46|PM|first|afternoon|lunch| +45107|AAAAAAAAEDALAAAA|45107|12|31|47|PM|first|afternoon|lunch| +45108|AAAAAAAAFDALAAAA|45108|12|31|48|PM|first|afternoon|lunch| +45109|AAAAAAAAGDALAAAA|45109|12|31|49|PM|first|afternoon|lunch| +45110|AAAAAAAAHDALAAAA|45110|12|31|50|PM|first|afternoon|lunch| +45111|AAAAAAAAIDALAAAA|45111|12|31|51|PM|first|afternoon|lunch| +45112|AAAAAAAAJDALAAAA|45112|12|31|52|PM|first|afternoon|lunch| +45113|AAAAAAAAKDALAAAA|45113|12|31|53|PM|first|afternoon|lunch| +45114|AAAAAAAALDALAAAA|45114|12|31|54|PM|first|afternoon|lunch| +45115|AAAAAAAAMDALAAAA|45115|12|31|55|PM|first|afternoon|lunch| +45116|AAAAAAAANDALAAAA|45116|12|31|56|PM|first|afternoon|lunch| +45117|AAAAAAAAODALAAAA|45117|12|31|57|PM|first|afternoon|lunch| +45118|AAAAAAAAPDALAAAA|45118|12|31|58|PM|first|afternoon|lunch| +45119|AAAAAAAAAEALAAAA|45119|12|31|59|PM|first|afternoon|lunch| +45120|AAAAAAAABEALAAAA|45120|12|32|0|PM|first|afternoon|lunch| +45121|AAAAAAAACEALAAAA|45121|12|32|1|PM|first|afternoon|lunch| +45122|AAAAAAAADEALAAAA|45122|12|32|2|PM|first|afternoon|lunch| +45123|AAAAAAAAEEALAAAA|45123|12|32|3|PM|first|afternoon|lunch| +45124|AAAAAAAAFEALAAAA|45124|12|32|4|PM|first|afternoon|lunch| +45125|AAAAAAAAGEALAAAA|45125|12|32|5|PM|first|afternoon|lunch| +45126|AAAAAAAAHEALAAAA|45126|12|32|6|PM|first|afternoon|lunch| +45127|AAAAAAAAIEALAAAA|45127|12|32|7|PM|first|afternoon|lunch| +45128|AAAAAAAAJEALAAAA|45128|12|32|8|PM|first|afternoon|lunch| +45129|AAAAAAAAKEALAAAA|45129|12|32|9|PM|first|afternoon|lunch| +45130|AAAAAAAALEALAAAA|45130|12|32|10|PM|first|afternoon|lunch| +45131|AAAAAAAAMEALAAAA|45131|12|32|11|PM|first|afternoon|lunch| +45132|AAAAAAAANEALAAAA|45132|12|32|12|PM|first|afternoon|lunch| +45133|AAAAAAAAOEALAAAA|45133|12|32|13|PM|first|afternoon|lunch| +45134|AAAAAAAAPEALAAAA|45134|12|32|14|PM|first|afternoon|lunch| +45135|AAAAAAAAAFALAAAA|45135|12|32|15|PM|first|afternoon|lunch| +45136|AAAAAAAABFALAAAA|45136|12|32|16|PM|first|afternoon|lunch| +45137|AAAAAAAACFALAAAA|45137|12|32|17|PM|first|afternoon|lunch| +45138|AAAAAAAADFALAAAA|45138|12|32|18|PM|first|afternoon|lunch| +45139|AAAAAAAAEFALAAAA|45139|12|32|19|PM|first|afternoon|lunch| +45140|AAAAAAAAFFALAAAA|45140|12|32|20|PM|first|afternoon|lunch| +45141|AAAAAAAAGFALAAAA|45141|12|32|21|PM|first|afternoon|lunch| +45142|AAAAAAAAHFALAAAA|45142|12|32|22|PM|first|afternoon|lunch| +45143|AAAAAAAAIFALAAAA|45143|12|32|23|PM|first|afternoon|lunch| +45144|AAAAAAAAJFALAAAA|45144|12|32|24|PM|first|afternoon|lunch| +45145|AAAAAAAAKFALAAAA|45145|12|32|25|PM|first|afternoon|lunch| +45146|AAAAAAAALFALAAAA|45146|12|32|26|PM|first|afternoon|lunch| +45147|AAAAAAAAMFALAAAA|45147|12|32|27|PM|first|afternoon|lunch| +45148|AAAAAAAANFALAAAA|45148|12|32|28|PM|first|afternoon|lunch| +45149|AAAAAAAAOFALAAAA|45149|12|32|29|PM|first|afternoon|lunch| +45150|AAAAAAAAPFALAAAA|45150|12|32|30|PM|first|afternoon|lunch| +45151|AAAAAAAAAGALAAAA|45151|12|32|31|PM|first|afternoon|lunch| +45152|AAAAAAAABGALAAAA|45152|12|32|32|PM|first|afternoon|lunch| +45153|AAAAAAAACGALAAAA|45153|12|32|33|PM|first|afternoon|lunch| +45154|AAAAAAAADGALAAAA|45154|12|32|34|PM|first|afternoon|lunch| +45155|AAAAAAAAEGALAAAA|45155|12|32|35|PM|first|afternoon|lunch| +45156|AAAAAAAAFGALAAAA|45156|12|32|36|PM|first|afternoon|lunch| +45157|AAAAAAAAGGALAAAA|45157|12|32|37|PM|first|afternoon|lunch| +45158|AAAAAAAAHGALAAAA|45158|12|32|38|PM|first|afternoon|lunch| +45159|AAAAAAAAIGALAAAA|45159|12|32|39|PM|first|afternoon|lunch| +45160|AAAAAAAAJGALAAAA|45160|12|32|40|PM|first|afternoon|lunch| +45161|AAAAAAAAKGALAAAA|45161|12|32|41|PM|first|afternoon|lunch| +45162|AAAAAAAALGALAAAA|45162|12|32|42|PM|first|afternoon|lunch| +45163|AAAAAAAAMGALAAAA|45163|12|32|43|PM|first|afternoon|lunch| +45164|AAAAAAAANGALAAAA|45164|12|32|44|PM|first|afternoon|lunch| +45165|AAAAAAAAOGALAAAA|45165|12|32|45|PM|first|afternoon|lunch| +45166|AAAAAAAAPGALAAAA|45166|12|32|46|PM|first|afternoon|lunch| +45167|AAAAAAAAAHALAAAA|45167|12|32|47|PM|first|afternoon|lunch| +45168|AAAAAAAABHALAAAA|45168|12|32|48|PM|first|afternoon|lunch| +45169|AAAAAAAACHALAAAA|45169|12|32|49|PM|first|afternoon|lunch| +45170|AAAAAAAADHALAAAA|45170|12|32|50|PM|first|afternoon|lunch| +45171|AAAAAAAAEHALAAAA|45171|12|32|51|PM|first|afternoon|lunch| +45172|AAAAAAAAFHALAAAA|45172|12|32|52|PM|first|afternoon|lunch| +45173|AAAAAAAAGHALAAAA|45173|12|32|53|PM|first|afternoon|lunch| +45174|AAAAAAAAHHALAAAA|45174|12|32|54|PM|first|afternoon|lunch| +45175|AAAAAAAAIHALAAAA|45175|12|32|55|PM|first|afternoon|lunch| +45176|AAAAAAAAJHALAAAA|45176|12|32|56|PM|first|afternoon|lunch| +45177|AAAAAAAAKHALAAAA|45177|12|32|57|PM|first|afternoon|lunch| +45178|AAAAAAAALHALAAAA|45178|12|32|58|PM|first|afternoon|lunch| +45179|AAAAAAAAMHALAAAA|45179|12|32|59|PM|first|afternoon|lunch| +45180|AAAAAAAANHALAAAA|45180|12|33|0|PM|first|afternoon|lunch| +45181|AAAAAAAAOHALAAAA|45181|12|33|1|PM|first|afternoon|lunch| +45182|AAAAAAAAPHALAAAA|45182|12|33|2|PM|first|afternoon|lunch| +45183|AAAAAAAAAIALAAAA|45183|12|33|3|PM|first|afternoon|lunch| +45184|AAAAAAAABIALAAAA|45184|12|33|4|PM|first|afternoon|lunch| +45185|AAAAAAAACIALAAAA|45185|12|33|5|PM|first|afternoon|lunch| +45186|AAAAAAAADIALAAAA|45186|12|33|6|PM|first|afternoon|lunch| +45187|AAAAAAAAEIALAAAA|45187|12|33|7|PM|first|afternoon|lunch| +45188|AAAAAAAAFIALAAAA|45188|12|33|8|PM|first|afternoon|lunch| +45189|AAAAAAAAGIALAAAA|45189|12|33|9|PM|first|afternoon|lunch| +45190|AAAAAAAAHIALAAAA|45190|12|33|10|PM|first|afternoon|lunch| +45191|AAAAAAAAIIALAAAA|45191|12|33|11|PM|first|afternoon|lunch| +45192|AAAAAAAAJIALAAAA|45192|12|33|12|PM|first|afternoon|lunch| +45193|AAAAAAAAKIALAAAA|45193|12|33|13|PM|first|afternoon|lunch| +45194|AAAAAAAALIALAAAA|45194|12|33|14|PM|first|afternoon|lunch| +45195|AAAAAAAAMIALAAAA|45195|12|33|15|PM|first|afternoon|lunch| +45196|AAAAAAAANIALAAAA|45196|12|33|16|PM|first|afternoon|lunch| +45197|AAAAAAAAOIALAAAA|45197|12|33|17|PM|first|afternoon|lunch| +45198|AAAAAAAAPIALAAAA|45198|12|33|18|PM|first|afternoon|lunch| +45199|AAAAAAAAAJALAAAA|45199|12|33|19|PM|first|afternoon|lunch| +45200|AAAAAAAABJALAAAA|45200|12|33|20|PM|first|afternoon|lunch| +45201|AAAAAAAACJALAAAA|45201|12|33|21|PM|first|afternoon|lunch| +45202|AAAAAAAADJALAAAA|45202|12|33|22|PM|first|afternoon|lunch| +45203|AAAAAAAAEJALAAAA|45203|12|33|23|PM|first|afternoon|lunch| +45204|AAAAAAAAFJALAAAA|45204|12|33|24|PM|first|afternoon|lunch| +45205|AAAAAAAAGJALAAAA|45205|12|33|25|PM|first|afternoon|lunch| +45206|AAAAAAAAHJALAAAA|45206|12|33|26|PM|first|afternoon|lunch| +45207|AAAAAAAAIJALAAAA|45207|12|33|27|PM|first|afternoon|lunch| +45208|AAAAAAAAJJALAAAA|45208|12|33|28|PM|first|afternoon|lunch| +45209|AAAAAAAAKJALAAAA|45209|12|33|29|PM|first|afternoon|lunch| +45210|AAAAAAAALJALAAAA|45210|12|33|30|PM|first|afternoon|lunch| +45211|AAAAAAAAMJALAAAA|45211|12|33|31|PM|first|afternoon|lunch| +45212|AAAAAAAANJALAAAA|45212|12|33|32|PM|first|afternoon|lunch| +45213|AAAAAAAAOJALAAAA|45213|12|33|33|PM|first|afternoon|lunch| +45214|AAAAAAAAPJALAAAA|45214|12|33|34|PM|first|afternoon|lunch| +45215|AAAAAAAAAKALAAAA|45215|12|33|35|PM|first|afternoon|lunch| +45216|AAAAAAAABKALAAAA|45216|12|33|36|PM|first|afternoon|lunch| +45217|AAAAAAAACKALAAAA|45217|12|33|37|PM|first|afternoon|lunch| +45218|AAAAAAAADKALAAAA|45218|12|33|38|PM|first|afternoon|lunch| +45219|AAAAAAAAEKALAAAA|45219|12|33|39|PM|first|afternoon|lunch| +45220|AAAAAAAAFKALAAAA|45220|12|33|40|PM|first|afternoon|lunch| +45221|AAAAAAAAGKALAAAA|45221|12|33|41|PM|first|afternoon|lunch| +45222|AAAAAAAAHKALAAAA|45222|12|33|42|PM|first|afternoon|lunch| +45223|AAAAAAAAIKALAAAA|45223|12|33|43|PM|first|afternoon|lunch| +45224|AAAAAAAAJKALAAAA|45224|12|33|44|PM|first|afternoon|lunch| +45225|AAAAAAAAKKALAAAA|45225|12|33|45|PM|first|afternoon|lunch| +45226|AAAAAAAALKALAAAA|45226|12|33|46|PM|first|afternoon|lunch| +45227|AAAAAAAAMKALAAAA|45227|12|33|47|PM|first|afternoon|lunch| +45228|AAAAAAAANKALAAAA|45228|12|33|48|PM|first|afternoon|lunch| +45229|AAAAAAAAOKALAAAA|45229|12|33|49|PM|first|afternoon|lunch| +45230|AAAAAAAAPKALAAAA|45230|12|33|50|PM|first|afternoon|lunch| +45231|AAAAAAAAALALAAAA|45231|12|33|51|PM|first|afternoon|lunch| +45232|AAAAAAAABLALAAAA|45232|12|33|52|PM|first|afternoon|lunch| +45233|AAAAAAAACLALAAAA|45233|12|33|53|PM|first|afternoon|lunch| +45234|AAAAAAAADLALAAAA|45234|12|33|54|PM|first|afternoon|lunch| +45235|AAAAAAAAELALAAAA|45235|12|33|55|PM|first|afternoon|lunch| +45236|AAAAAAAAFLALAAAA|45236|12|33|56|PM|first|afternoon|lunch| +45237|AAAAAAAAGLALAAAA|45237|12|33|57|PM|first|afternoon|lunch| +45238|AAAAAAAAHLALAAAA|45238|12|33|58|PM|first|afternoon|lunch| +45239|AAAAAAAAILALAAAA|45239|12|33|59|PM|first|afternoon|lunch| +45240|AAAAAAAAJLALAAAA|45240|12|34|0|PM|first|afternoon|lunch| +45241|AAAAAAAAKLALAAAA|45241|12|34|1|PM|first|afternoon|lunch| +45242|AAAAAAAALLALAAAA|45242|12|34|2|PM|first|afternoon|lunch| +45243|AAAAAAAAMLALAAAA|45243|12|34|3|PM|first|afternoon|lunch| +45244|AAAAAAAANLALAAAA|45244|12|34|4|PM|first|afternoon|lunch| +45245|AAAAAAAAOLALAAAA|45245|12|34|5|PM|first|afternoon|lunch| +45246|AAAAAAAAPLALAAAA|45246|12|34|6|PM|first|afternoon|lunch| +45247|AAAAAAAAAMALAAAA|45247|12|34|7|PM|first|afternoon|lunch| +45248|AAAAAAAABMALAAAA|45248|12|34|8|PM|first|afternoon|lunch| +45249|AAAAAAAACMALAAAA|45249|12|34|9|PM|first|afternoon|lunch| +45250|AAAAAAAADMALAAAA|45250|12|34|10|PM|first|afternoon|lunch| +45251|AAAAAAAAEMALAAAA|45251|12|34|11|PM|first|afternoon|lunch| +45252|AAAAAAAAFMALAAAA|45252|12|34|12|PM|first|afternoon|lunch| +45253|AAAAAAAAGMALAAAA|45253|12|34|13|PM|first|afternoon|lunch| +45254|AAAAAAAAHMALAAAA|45254|12|34|14|PM|first|afternoon|lunch| +45255|AAAAAAAAIMALAAAA|45255|12|34|15|PM|first|afternoon|lunch| +45256|AAAAAAAAJMALAAAA|45256|12|34|16|PM|first|afternoon|lunch| +45257|AAAAAAAAKMALAAAA|45257|12|34|17|PM|first|afternoon|lunch| +45258|AAAAAAAALMALAAAA|45258|12|34|18|PM|first|afternoon|lunch| +45259|AAAAAAAAMMALAAAA|45259|12|34|19|PM|first|afternoon|lunch| +45260|AAAAAAAANMALAAAA|45260|12|34|20|PM|first|afternoon|lunch| +45261|AAAAAAAAOMALAAAA|45261|12|34|21|PM|first|afternoon|lunch| +45262|AAAAAAAAPMALAAAA|45262|12|34|22|PM|first|afternoon|lunch| +45263|AAAAAAAAANALAAAA|45263|12|34|23|PM|first|afternoon|lunch| +45264|AAAAAAAABNALAAAA|45264|12|34|24|PM|first|afternoon|lunch| +45265|AAAAAAAACNALAAAA|45265|12|34|25|PM|first|afternoon|lunch| +45266|AAAAAAAADNALAAAA|45266|12|34|26|PM|first|afternoon|lunch| +45267|AAAAAAAAENALAAAA|45267|12|34|27|PM|first|afternoon|lunch| +45268|AAAAAAAAFNALAAAA|45268|12|34|28|PM|first|afternoon|lunch| +45269|AAAAAAAAGNALAAAA|45269|12|34|29|PM|first|afternoon|lunch| +45270|AAAAAAAAHNALAAAA|45270|12|34|30|PM|first|afternoon|lunch| +45271|AAAAAAAAINALAAAA|45271|12|34|31|PM|first|afternoon|lunch| +45272|AAAAAAAAJNALAAAA|45272|12|34|32|PM|first|afternoon|lunch| +45273|AAAAAAAAKNALAAAA|45273|12|34|33|PM|first|afternoon|lunch| +45274|AAAAAAAALNALAAAA|45274|12|34|34|PM|first|afternoon|lunch| +45275|AAAAAAAAMNALAAAA|45275|12|34|35|PM|first|afternoon|lunch| +45276|AAAAAAAANNALAAAA|45276|12|34|36|PM|first|afternoon|lunch| +45277|AAAAAAAAONALAAAA|45277|12|34|37|PM|first|afternoon|lunch| +45278|AAAAAAAAPNALAAAA|45278|12|34|38|PM|first|afternoon|lunch| +45279|AAAAAAAAAOALAAAA|45279|12|34|39|PM|first|afternoon|lunch| +45280|AAAAAAAABOALAAAA|45280|12|34|40|PM|first|afternoon|lunch| +45281|AAAAAAAACOALAAAA|45281|12|34|41|PM|first|afternoon|lunch| +45282|AAAAAAAADOALAAAA|45282|12|34|42|PM|first|afternoon|lunch| +45283|AAAAAAAAEOALAAAA|45283|12|34|43|PM|first|afternoon|lunch| +45284|AAAAAAAAFOALAAAA|45284|12|34|44|PM|first|afternoon|lunch| +45285|AAAAAAAAGOALAAAA|45285|12|34|45|PM|first|afternoon|lunch| +45286|AAAAAAAAHOALAAAA|45286|12|34|46|PM|first|afternoon|lunch| +45287|AAAAAAAAIOALAAAA|45287|12|34|47|PM|first|afternoon|lunch| +45288|AAAAAAAAJOALAAAA|45288|12|34|48|PM|first|afternoon|lunch| +45289|AAAAAAAAKOALAAAA|45289|12|34|49|PM|first|afternoon|lunch| +45290|AAAAAAAALOALAAAA|45290|12|34|50|PM|first|afternoon|lunch| +45291|AAAAAAAAMOALAAAA|45291|12|34|51|PM|first|afternoon|lunch| +45292|AAAAAAAANOALAAAA|45292|12|34|52|PM|first|afternoon|lunch| +45293|AAAAAAAAOOALAAAA|45293|12|34|53|PM|first|afternoon|lunch| +45294|AAAAAAAAPOALAAAA|45294|12|34|54|PM|first|afternoon|lunch| +45295|AAAAAAAAAPALAAAA|45295|12|34|55|PM|first|afternoon|lunch| +45296|AAAAAAAABPALAAAA|45296|12|34|56|PM|first|afternoon|lunch| +45297|AAAAAAAACPALAAAA|45297|12|34|57|PM|first|afternoon|lunch| +45298|AAAAAAAADPALAAAA|45298|12|34|58|PM|first|afternoon|lunch| +45299|AAAAAAAAEPALAAAA|45299|12|34|59|PM|first|afternoon|lunch| +45300|AAAAAAAAFPALAAAA|45300|12|35|0|PM|first|afternoon|lunch| +45301|AAAAAAAAGPALAAAA|45301|12|35|1|PM|first|afternoon|lunch| +45302|AAAAAAAAHPALAAAA|45302|12|35|2|PM|first|afternoon|lunch| +45303|AAAAAAAAIPALAAAA|45303|12|35|3|PM|first|afternoon|lunch| +45304|AAAAAAAAJPALAAAA|45304|12|35|4|PM|first|afternoon|lunch| +45305|AAAAAAAAKPALAAAA|45305|12|35|5|PM|first|afternoon|lunch| +45306|AAAAAAAALPALAAAA|45306|12|35|6|PM|first|afternoon|lunch| +45307|AAAAAAAAMPALAAAA|45307|12|35|7|PM|first|afternoon|lunch| +45308|AAAAAAAANPALAAAA|45308|12|35|8|PM|first|afternoon|lunch| +45309|AAAAAAAAOPALAAAA|45309|12|35|9|PM|first|afternoon|lunch| +45310|AAAAAAAAPPALAAAA|45310|12|35|10|PM|first|afternoon|lunch| +45311|AAAAAAAAAABLAAAA|45311|12|35|11|PM|first|afternoon|lunch| +45312|AAAAAAAABABLAAAA|45312|12|35|12|PM|first|afternoon|lunch| +45313|AAAAAAAACABLAAAA|45313|12|35|13|PM|first|afternoon|lunch| +45314|AAAAAAAADABLAAAA|45314|12|35|14|PM|first|afternoon|lunch| +45315|AAAAAAAAEABLAAAA|45315|12|35|15|PM|first|afternoon|lunch| +45316|AAAAAAAAFABLAAAA|45316|12|35|16|PM|first|afternoon|lunch| +45317|AAAAAAAAGABLAAAA|45317|12|35|17|PM|first|afternoon|lunch| +45318|AAAAAAAAHABLAAAA|45318|12|35|18|PM|first|afternoon|lunch| +45319|AAAAAAAAIABLAAAA|45319|12|35|19|PM|first|afternoon|lunch| +45320|AAAAAAAAJABLAAAA|45320|12|35|20|PM|first|afternoon|lunch| +45321|AAAAAAAAKABLAAAA|45321|12|35|21|PM|first|afternoon|lunch| +45322|AAAAAAAALABLAAAA|45322|12|35|22|PM|first|afternoon|lunch| +45323|AAAAAAAAMABLAAAA|45323|12|35|23|PM|first|afternoon|lunch| +45324|AAAAAAAANABLAAAA|45324|12|35|24|PM|first|afternoon|lunch| +45325|AAAAAAAAOABLAAAA|45325|12|35|25|PM|first|afternoon|lunch| +45326|AAAAAAAAPABLAAAA|45326|12|35|26|PM|first|afternoon|lunch| +45327|AAAAAAAAABBLAAAA|45327|12|35|27|PM|first|afternoon|lunch| +45328|AAAAAAAABBBLAAAA|45328|12|35|28|PM|first|afternoon|lunch| +45329|AAAAAAAACBBLAAAA|45329|12|35|29|PM|first|afternoon|lunch| +45330|AAAAAAAADBBLAAAA|45330|12|35|30|PM|first|afternoon|lunch| +45331|AAAAAAAAEBBLAAAA|45331|12|35|31|PM|first|afternoon|lunch| +45332|AAAAAAAAFBBLAAAA|45332|12|35|32|PM|first|afternoon|lunch| +45333|AAAAAAAAGBBLAAAA|45333|12|35|33|PM|first|afternoon|lunch| +45334|AAAAAAAAHBBLAAAA|45334|12|35|34|PM|first|afternoon|lunch| +45335|AAAAAAAAIBBLAAAA|45335|12|35|35|PM|first|afternoon|lunch| +45336|AAAAAAAAJBBLAAAA|45336|12|35|36|PM|first|afternoon|lunch| +45337|AAAAAAAAKBBLAAAA|45337|12|35|37|PM|first|afternoon|lunch| +45338|AAAAAAAALBBLAAAA|45338|12|35|38|PM|first|afternoon|lunch| +45339|AAAAAAAAMBBLAAAA|45339|12|35|39|PM|first|afternoon|lunch| +45340|AAAAAAAANBBLAAAA|45340|12|35|40|PM|first|afternoon|lunch| +45341|AAAAAAAAOBBLAAAA|45341|12|35|41|PM|first|afternoon|lunch| +45342|AAAAAAAAPBBLAAAA|45342|12|35|42|PM|first|afternoon|lunch| +45343|AAAAAAAAACBLAAAA|45343|12|35|43|PM|first|afternoon|lunch| +45344|AAAAAAAABCBLAAAA|45344|12|35|44|PM|first|afternoon|lunch| +45345|AAAAAAAACCBLAAAA|45345|12|35|45|PM|first|afternoon|lunch| +45346|AAAAAAAADCBLAAAA|45346|12|35|46|PM|first|afternoon|lunch| +45347|AAAAAAAAECBLAAAA|45347|12|35|47|PM|first|afternoon|lunch| +45348|AAAAAAAAFCBLAAAA|45348|12|35|48|PM|first|afternoon|lunch| +45349|AAAAAAAAGCBLAAAA|45349|12|35|49|PM|first|afternoon|lunch| +45350|AAAAAAAAHCBLAAAA|45350|12|35|50|PM|first|afternoon|lunch| +45351|AAAAAAAAICBLAAAA|45351|12|35|51|PM|first|afternoon|lunch| +45352|AAAAAAAAJCBLAAAA|45352|12|35|52|PM|first|afternoon|lunch| +45353|AAAAAAAAKCBLAAAA|45353|12|35|53|PM|first|afternoon|lunch| +45354|AAAAAAAALCBLAAAA|45354|12|35|54|PM|first|afternoon|lunch| +45355|AAAAAAAAMCBLAAAA|45355|12|35|55|PM|first|afternoon|lunch| +45356|AAAAAAAANCBLAAAA|45356|12|35|56|PM|first|afternoon|lunch| +45357|AAAAAAAAOCBLAAAA|45357|12|35|57|PM|first|afternoon|lunch| +45358|AAAAAAAAPCBLAAAA|45358|12|35|58|PM|first|afternoon|lunch| +45359|AAAAAAAAADBLAAAA|45359|12|35|59|PM|first|afternoon|lunch| +45360|AAAAAAAABDBLAAAA|45360|12|36|0|PM|first|afternoon|lunch| +45361|AAAAAAAACDBLAAAA|45361|12|36|1|PM|first|afternoon|lunch| +45362|AAAAAAAADDBLAAAA|45362|12|36|2|PM|first|afternoon|lunch| +45363|AAAAAAAAEDBLAAAA|45363|12|36|3|PM|first|afternoon|lunch| +45364|AAAAAAAAFDBLAAAA|45364|12|36|4|PM|first|afternoon|lunch| +45365|AAAAAAAAGDBLAAAA|45365|12|36|5|PM|first|afternoon|lunch| +45366|AAAAAAAAHDBLAAAA|45366|12|36|6|PM|first|afternoon|lunch| +45367|AAAAAAAAIDBLAAAA|45367|12|36|7|PM|first|afternoon|lunch| +45368|AAAAAAAAJDBLAAAA|45368|12|36|8|PM|first|afternoon|lunch| +45369|AAAAAAAAKDBLAAAA|45369|12|36|9|PM|first|afternoon|lunch| +45370|AAAAAAAALDBLAAAA|45370|12|36|10|PM|first|afternoon|lunch| +45371|AAAAAAAAMDBLAAAA|45371|12|36|11|PM|first|afternoon|lunch| +45372|AAAAAAAANDBLAAAA|45372|12|36|12|PM|first|afternoon|lunch| +45373|AAAAAAAAODBLAAAA|45373|12|36|13|PM|first|afternoon|lunch| +45374|AAAAAAAAPDBLAAAA|45374|12|36|14|PM|first|afternoon|lunch| +45375|AAAAAAAAAEBLAAAA|45375|12|36|15|PM|first|afternoon|lunch| +45376|AAAAAAAABEBLAAAA|45376|12|36|16|PM|first|afternoon|lunch| +45377|AAAAAAAACEBLAAAA|45377|12|36|17|PM|first|afternoon|lunch| +45378|AAAAAAAADEBLAAAA|45378|12|36|18|PM|first|afternoon|lunch| +45379|AAAAAAAAEEBLAAAA|45379|12|36|19|PM|first|afternoon|lunch| +45380|AAAAAAAAFEBLAAAA|45380|12|36|20|PM|first|afternoon|lunch| +45381|AAAAAAAAGEBLAAAA|45381|12|36|21|PM|first|afternoon|lunch| +45382|AAAAAAAAHEBLAAAA|45382|12|36|22|PM|first|afternoon|lunch| +45383|AAAAAAAAIEBLAAAA|45383|12|36|23|PM|first|afternoon|lunch| +45384|AAAAAAAAJEBLAAAA|45384|12|36|24|PM|first|afternoon|lunch| +45385|AAAAAAAAKEBLAAAA|45385|12|36|25|PM|first|afternoon|lunch| +45386|AAAAAAAALEBLAAAA|45386|12|36|26|PM|first|afternoon|lunch| +45387|AAAAAAAAMEBLAAAA|45387|12|36|27|PM|first|afternoon|lunch| +45388|AAAAAAAANEBLAAAA|45388|12|36|28|PM|first|afternoon|lunch| +45389|AAAAAAAAOEBLAAAA|45389|12|36|29|PM|first|afternoon|lunch| +45390|AAAAAAAAPEBLAAAA|45390|12|36|30|PM|first|afternoon|lunch| +45391|AAAAAAAAAFBLAAAA|45391|12|36|31|PM|first|afternoon|lunch| +45392|AAAAAAAABFBLAAAA|45392|12|36|32|PM|first|afternoon|lunch| +45393|AAAAAAAACFBLAAAA|45393|12|36|33|PM|first|afternoon|lunch| +45394|AAAAAAAADFBLAAAA|45394|12|36|34|PM|first|afternoon|lunch| +45395|AAAAAAAAEFBLAAAA|45395|12|36|35|PM|first|afternoon|lunch| +45396|AAAAAAAAFFBLAAAA|45396|12|36|36|PM|first|afternoon|lunch| +45397|AAAAAAAAGFBLAAAA|45397|12|36|37|PM|first|afternoon|lunch| +45398|AAAAAAAAHFBLAAAA|45398|12|36|38|PM|first|afternoon|lunch| +45399|AAAAAAAAIFBLAAAA|45399|12|36|39|PM|first|afternoon|lunch| +45400|AAAAAAAAJFBLAAAA|45400|12|36|40|PM|first|afternoon|lunch| +45401|AAAAAAAAKFBLAAAA|45401|12|36|41|PM|first|afternoon|lunch| +45402|AAAAAAAALFBLAAAA|45402|12|36|42|PM|first|afternoon|lunch| +45403|AAAAAAAAMFBLAAAA|45403|12|36|43|PM|first|afternoon|lunch| +45404|AAAAAAAANFBLAAAA|45404|12|36|44|PM|first|afternoon|lunch| +45405|AAAAAAAAOFBLAAAA|45405|12|36|45|PM|first|afternoon|lunch| +45406|AAAAAAAAPFBLAAAA|45406|12|36|46|PM|first|afternoon|lunch| +45407|AAAAAAAAAGBLAAAA|45407|12|36|47|PM|first|afternoon|lunch| +45408|AAAAAAAABGBLAAAA|45408|12|36|48|PM|first|afternoon|lunch| +45409|AAAAAAAACGBLAAAA|45409|12|36|49|PM|first|afternoon|lunch| +45410|AAAAAAAADGBLAAAA|45410|12|36|50|PM|first|afternoon|lunch| +45411|AAAAAAAAEGBLAAAA|45411|12|36|51|PM|first|afternoon|lunch| +45412|AAAAAAAAFGBLAAAA|45412|12|36|52|PM|first|afternoon|lunch| +45413|AAAAAAAAGGBLAAAA|45413|12|36|53|PM|first|afternoon|lunch| +45414|AAAAAAAAHGBLAAAA|45414|12|36|54|PM|first|afternoon|lunch| +45415|AAAAAAAAIGBLAAAA|45415|12|36|55|PM|first|afternoon|lunch| +45416|AAAAAAAAJGBLAAAA|45416|12|36|56|PM|first|afternoon|lunch| +45417|AAAAAAAAKGBLAAAA|45417|12|36|57|PM|first|afternoon|lunch| +45418|AAAAAAAALGBLAAAA|45418|12|36|58|PM|first|afternoon|lunch| +45419|AAAAAAAAMGBLAAAA|45419|12|36|59|PM|first|afternoon|lunch| +45420|AAAAAAAANGBLAAAA|45420|12|37|0|PM|first|afternoon|lunch| +45421|AAAAAAAAOGBLAAAA|45421|12|37|1|PM|first|afternoon|lunch| +45422|AAAAAAAAPGBLAAAA|45422|12|37|2|PM|first|afternoon|lunch| +45423|AAAAAAAAAHBLAAAA|45423|12|37|3|PM|first|afternoon|lunch| +45424|AAAAAAAABHBLAAAA|45424|12|37|4|PM|first|afternoon|lunch| +45425|AAAAAAAACHBLAAAA|45425|12|37|5|PM|first|afternoon|lunch| +45426|AAAAAAAADHBLAAAA|45426|12|37|6|PM|first|afternoon|lunch| +45427|AAAAAAAAEHBLAAAA|45427|12|37|7|PM|first|afternoon|lunch| +45428|AAAAAAAAFHBLAAAA|45428|12|37|8|PM|first|afternoon|lunch| +45429|AAAAAAAAGHBLAAAA|45429|12|37|9|PM|first|afternoon|lunch| +45430|AAAAAAAAHHBLAAAA|45430|12|37|10|PM|first|afternoon|lunch| +45431|AAAAAAAAIHBLAAAA|45431|12|37|11|PM|first|afternoon|lunch| +45432|AAAAAAAAJHBLAAAA|45432|12|37|12|PM|first|afternoon|lunch| +45433|AAAAAAAAKHBLAAAA|45433|12|37|13|PM|first|afternoon|lunch| +45434|AAAAAAAALHBLAAAA|45434|12|37|14|PM|first|afternoon|lunch| +45435|AAAAAAAAMHBLAAAA|45435|12|37|15|PM|first|afternoon|lunch| +45436|AAAAAAAANHBLAAAA|45436|12|37|16|PM|first|afternoon|lunch| +45437|AAAAAAAAOHBLAAAA|45437|12|37|17|PM|first|afternoon|lunch| +45438|AAAAAAAAPHBLAAAA|45438|12|37|18|PM|first|afternoon|lunch| +45439|AAAAAAAAAIBLAAAA|45439|12|37|19|PM|first|afternoon|lunch| +45440|AAAAAAAABIBLAAAA|45440|12|37|20|PM|first|afternoon|lunch| +45441|AAAAAAAACIBLAAAA|45441|12|37|21|PM|first|afternoon|lunch| +45442|AAAAAAAADIBLAAAA|45442|12|37|22|PM|first|afternoon|lunch| +45443|AAAAAAAAEIBLAAAA|45443|12|37|23|PM|first|afternoon|lunch| +45444|AAAAAAAAFIBLAAAA|45444|12|37|24|PM|first|afternoon|lunch| +45445|AAAAAAAAGIBLAAAA|45445|12|37|25|PM|first|afternoon|lunch| +45446|AAAAAAAAHIBLAAAA|45446|12|37|26|PM|first|afternoon|lunch| +45447|AAAAAAAAIIBLAAAA|45447|12|37|27|PM|first|afternoon|lunch| +45448|AAAAAAAAJIBLAAAA|45448|12|37|28|PM|first|afternoon|lunch| +45449|AAAAAAAAKIBLAAAA|45449|12|37|29|PM|first|afternoon|lunch| +45450|AAAAAAAALIBLAAAA|45450|12|37|30|PM|first|afternoon|lunch| +45451|AAAAAAAAMIBLAAAA|45451|12|37|31|PM|first|afternoon|lunch| +45452|AAAAAAAANIBLAAAA|45452|12|37|32|PM|first|afternoon|lunch| +45453|AAAAAAAAOIBLAAAA|45453|12|37|33|PM|first|afternoon|lunch| +45454|AAAAAAAAPIBLAAAA|45454|12|37|34|PM|first|afternoon|lunch| +45455|AAAAAAAAAJBLAAAA|45455|12|37|35|PM|first|afternoon|lunch| +45456|AAAAAAAABJBLAAAA|45456|12|37|36|PM|first|afternoon|lunch| +45457|AAAAAAAACJBLAAAA|45457|12|37|37|PM|first|afternoon|lunch| +45458|AAAAAAAADJBLAAAA|45458|12|37|38|PM|first|afternoon|lunch| +45459|AAAAAAAAEJBLAAAA|45459|12|37|39|PM|first|afternoon|lunch| +45460|AAAAAAAAFJBLAAAA|45460|12|37|40|PM|first|afternoon|lunch| +45461|AAAAAAAAGJBLAAAA|45461|12|37|41|PM|first|afternoon|lunch| +45462|AAAAAAAAHJBLAAAA|45462|12|37|42|PM|first|afternoon|lunch| +45463|AAAAAAAAIJBLAAAA|45463|12|37|43|PM|first|afternoon|lunch| +45464|AAAAAAAAJJBLAAAA|45464|12|37|44|PM|first|afternoon|lunch| +45465|AAAAAAAAKJBLAAAA|45465|12|37|45|PM|first|afternoon|lunch| +45466|AAAAAAAALJBLAAAA|45466|12|37|46|PM|first|afternoon|lunch| +45467|AAAAAAAAMJBLAAAA|45467|12|37|47|PM|first|afternoon|lunch| +45468|AAAAAAAANJBLAAAA|45468|12|37|48|PM|first|afternoon|lunch| +45469|AAAAAAAAOJBLAAAA|45469|12|37|49|PM|first|afternoon|lunch| +45470|AAAAAAAAPJBLAAAA|45470|12|37|50|PM|first|afternoon|lunch| +45471|AAAAAAAAAKBLAAAA|45471|12|37|51|PM|first|afternoon|lunch| +45472|AAAAAAAABKBLAAAA|45472|12|37|52|PM|first|afternoon|lunch| +45473|AAAAAAAACKBLAAAA|45473|12|37|53|PM|first|afternoon|lunch| +45474|AAAAAAAADKBLAAAA|45474|12|37|54|PM|first|afternoon|lunch| +45475|AAAAAAAAEKBLAAAA|45475|12|37|55|PM|first|afternoon|lunch| +45476|AAAAAAAAFKBLAAAA|45476|12|37|56|PM|first|afternoon|lunch| +45477|AAAAAAAAGKBLAAAA|45477|12|37|57|PM|first|afternoon|lunch| +45478|AAAAAAAAHKBLAAAA|45478|12|37|58|PM|first|afternoon|lunch| +45479|AAAAAAAAIKBLAAAA|45479|12|37|59|PM|first|afternoon|lunch| +45480|AAAAAAAAJKBLAAAA|45480|12|38|0|PM|first|afternoon|lunch| +45481|AAAAAAAAKKBLAAAA|45481|12|38|1|PM|first|afternoon|lunch| +45482|AAAAAAAALKBLAAAA|45482|12|38|2|PM|first|afternoon|lunch| +45483|AAAAAAAAMKBLAAAA|45483|12|38|3|PM|first|afternoon|lunch| +45484|AAAAAAAANKBLAAAA|45484|12|38|4|PM|first|afternoon|lunch| +45485|AAAAAAAAOKBLAAAA|45485|12|38|5|PM|first|afternoon|lunch| +45486|AAAAAAAAPKBLAAAA|45486|12|38|6|PM|first|afternoon|lunch| +45487|AAAAAAAAALBLAAAA|45487|12|38|7|PM|first|afternoon|lunch| +45488|AAAAAAAABLBLAAAA|45488|12|38|8|PM|first|afternoon|lunch| +45489|AAAAAAAACLBLAAAA|45489|12|38|9|PM|first|afternoon|lunch| +45490|AAAAAAAADLBLAAAA|45490|12|38|10|PM|first|afternoon|lunch| +45491|AAAAAAAAELBLAAAA|45491|12|38|11|PM|first|afternoon|lunch| +45492|AAAAAAAAFLBLAAAA|45492|12|38|12|PM|first|afternoon|lunch| +45493|AAAAAAAAGLBLAAAA|45493|12|38|13|PM|first|afternoon|lunch| +45494|AAAAAAAAHLBLAAAA|45494|12|38|14|PM|first|afternoon|lunch| +45495|AAAAAAAAILBLAAAA|45495|12|38|15|PM|first|afternoon|lunch| +45496|AAAAAAAAJLBLAAAA|45496|12|38|16|PM|first|afternoon|lunch| +45497|AAAAAAAAKLBLAAAA|45497|12|38|17|PM|first|afternoon|lunch| +45498|AAAAAAAALLBLAAAA|45498|12|38|18|PM|first|afternoon|lunch| +45499|AAAAAAAAMLBLAAAA|45499|12|38|19|PM|first|afternoon|lunch| +45500|AAAAAAAANLBLAAAA|45500|12|38|20|PM|first|afternoon|lunch| +45501|AAAAAAAAOLBLAAAA|45501|12|38|21|PM|first|afternoon|lunch| +45502|AAAAAAAAPLBLAAAA|45502|12|38|22|PM|first|afternoon|lunch| +45503|AAAAAAAAAMBLAAAA|45503|12|38|23|PM|first|afternoon|lunch| +45504|AAAAAAAABMBLAAAA|45504|12|38|24|PM|first|afternoon|lunch| +45505|AAAAAAAACMBLAAAA|45505|12|38|25|PM|first|afternoon|lunch| +45506|AAAAAAAADMBLAAAA|45506|12|38|26|PM|first|afternoon|lunch| +45507|AAAAAAAAEMBLAAAA|45507|12|38|27|PM|first|afternoon|lunch| +45508|AAAAAAAAFMBLAAAA|45508|12|38|28|PM|first|afternoon|lunch| +45509|AAAAAAAAGMBLAAAA|45509|12|38|29|PM|first|afternoon|lunch| +45510|AAAAAAAAHMBLAAAA|45510|12|38|30|PM|first|afternoon|lunch| +45511|AAAAAAAAIMBLAAAA|45511|12|38|31|PM|first|afternoon|lunch| +45512|AAAAAAAAJMBLAAAA|45512|12|38|32|PM|first|afternoon|lunch| +45513|AAAAAAAAKMBLAAAA|45513|12|38|33|PM|first|afternoon|lunch| +45514|AAAAAAAALMBLAAAA|45514|12|38|34|PM|first|afternoon|lunch| +45515|AAAAAAAAMMBLAAAA|45515|12|38|35|PM|first|afternoon|lunch| +45516|AAAAAAAANMBLAAAA|45516|12|38|36|PM|first|afternoon|lunch| +45517|AAAAAAAAOMBLAAAA|45517|12|38|37|PM|first|afternoon|lunch| +45518|AAAAAAAAPMBLAAAA|45518|12|38|38|PM|first|afternoon|lunch| +45519|AAAAAAAAANBLAAAA|45519|12|38|39|PM|first|afternoon|lunch| +45520|AAAAAAAABNBLAAAA|45520|12|38|40|PM|first|afternoon|lunch| +45521|AAAAAAAACNBLAAAA|45521|12|38|41|PM|first|afternoon|lunch| +45522|AAAAAAAADNBLAAAA|45522|12|38|42|PM|first|afternoon|lunch| +45523|AAAAAAAAENBLAAAA|45523|12|38|43|PM|first|afternoon|lunch| +45524|AAAAAAAAFNBLAAAA|45524|12|38|44|PM|first|afternoon|lunch| +45525|AAAAAAAAGNBLAAAA|45525|12|38|45|PM|first|afternoon|lunch| +45526|AAAAAAAAHNBLAAAA|45526|12|38|46|PM|first|afternoon|lunch| +45527|AAAAAAAAINBLAAAA|45527|12|38|47|PM|first|afternoon|lunch| +45528|AAAAAAAAJNBLAAAA|45528|12|38|48|PM|first|afternoon|lunch| +45529|AAAAAAAAKNBLAAAA|45529|12|38|49|PM|first|afternoon|lunch| +45530|AAAAAAAALNBLAAAA|45530|12|38|50|PM|first|afternoon|lunch| +45531|AAAAAAAAMNBLAAAA|45531|12|38|51|PM|first|afternoon|lunch| +45532|AAAAAAAANNBLAAAA|45532|12|38|52|PM|first|afternoon|lunch| +45533|AAAAAAAAONBLAAAA|45533|12|38|53|PM|first|afternoon|lunch| +45534|AAAAAAAAPNBLAAAA|45534|12|38|54|PM|first|afternoon|lunch| +45535|AAAAAAAAAOBLAAAA|45535|12|38|55|PM|first|afternoon|lunch| +45536|AAAAAAAABOBLAAAA|45536|12|38|56|PM|first|afternoon|lunch| +45537|AAAAAAAACOBLAAAA|45537|12|38|57|PM|first|afternoon|lunch| +45538|AAAAAAAADOBLAAAA|45538|12|38|58|PM|first|afternoon|lunch| +45539|AAAAAAAAEOBLAAAA|45539|12|38|59|PM|first|afternoon|lunch| +45540|AAAAAAAAFOBLAAAA|45540|12|39|0|PM|first|afternoon|lunch| +45541|AAAAAAAAGOBLAAAA|45541|12|39|1|PM|first|afternoon|lunch| +45542|AAAAAAAAHOBLAAAA|45542|12|39|2|PM|first|afternoon|lunch| +45543|AAAAAAAAIOBLAAAA|45543|12|39|3|PM|first|afternoon|lunch| +45544|AAAAAAAAJOBLAAAA|45544|12|39|4|PM|first|afternoon|lunch| +45545|AAAAAAAAKOBLAAAA|45545|12|39|5|PM|first|afternoon|lunch| +45546|AAAAAAAALOBLAAAA|45546|12|39|6|PM|first|afternoon|lunch| +45547|AAAAAAAAMOBLAAAA|45547|12|39|7|PM|first|afternoon|lunch| +45548|AAAAAAAANOBLAAAA|45548|12|39|8|PM|first|afternoon|lunch| +45549|AAAAAAAAOOBLAAAA|45549|12|39|9|PM|first|afternoon|lunch| +45550|AAAAAAAAPOBLAAAA|45550|12|39|10|PM|first|afternoon|lunch| +45551|AAAAAAAAAPBLAAAA|45551|12|39|11|PM|first|afternoon|lunch| +45552|AAAAAAAABPBLAAAA|45552|12|39|12|PM|first|afternoon|lunch| +45553|AAAAAAAACPBLAAAA|45553|12|39|13|PM|first|afternoon|lunch| +45554|AAAAAAAADPBLAAAA|45554|12|39|14|PM|first|afternoon|lunch| +45555|AAAAAAAAEPBLAAAA|45555|12|39|15|PM|first|afternoon|lunch| +45556|AAAAAAAAFPBLAAAA|45556|12|39|16|PM|first|afternoon|lunch| +45557|AAAAAAAAGPBLAAAA|45557|12|39|17|PM|first|afternoon|lunch| +45558|AAAAAAAAHPBLAAAA|45558|12|39|18|PM|first|afternoon|lunch| +45559|AAAAAAAAIPBLAAAA|45559|12|39|19|PM|first|afternoon|lunch| +45560|AAAAAAAAJPBLAAAA|45560|12|39|20|PM|first|afternoon|lunch| +45561|AAAAAAAAKPBLAAAA|45561|12|39|21|PM|first|afternoon|lunch| +45562|AAAAAAAALPBLAAAA|45562|12|39|22|PM|first|afternoon|lunch| +45563|AAAAAAAAMPBLAAAA|45563|12|39|23|PM|first|afternoon|lunch| +45564|AAAAAAAANPBLAAAA|45564|12|39|24|PM|first|afternoon|lunch| +45565|AAAAAAAAOPBLAAAA|45565|12|39|25|PM|first|afternoon|lunch| +45566|AAAAAAAAPPBLAAAA|45566|12|39|26|PM|first|afternoon|lunch| +45567|AAAAAAAAAACLAAAA|45567|12|39|27|PM|first|afternoon|lunch| +45568|AAAAAAAABACLAAAA|45568|12|39|28|PM|first|afternoon|lunch| +45569|AAAAAAAACACLAAAA|45569|12|39|29|PM|first|afternoon|lunch| +45570|AAAAAAAADACLAAAA|45570|12|39|30|PM|first|afternoon|lunch| +45571|AAAAAAAAEACLAAAA|45571|12|39|31|PM|first|afternoon|lunch| +45572|AAAAAAAAFACLAAAA|45572|12|39|32|PM|first|afternoon|lunch| +45573|AAAAAAAAGACLAAAA|45573|12|39|33|PM|first|afternoon|lunch| +45574|AAAAAAAAHACLAAAA|45574|12|39|34|PM|first|afternoon|lunch| +45575|AAAAAAAAIACLAAAA|45575|12|39|35|PM|first|afternoon|lunch| +45576|AAAAAAAAJACLAAAA|45576|12|39|36|PM|first|afternoon|lunch| +45577|AAAAAAAAKACLAAAA|45577|12|39|37|PM|first|afternoon|lunch| +45578|AAAAAAAALACLAAAA|45578|12|39|38|PM|first|afternoon|lunch| +45579|AAAAAAAAMACLAAAA|45579|12|39|39|PM|first|afternoon|lunch| +45580|AAAAAAAANACLAAAA|45580|12|39|40|PM|first|afternoon|lunch| +45581|AAAAAAAAOACLAAAA|45581|12|39|41|PM|first|afternoon|lunch| +45582|AAAAAAAAPACLAAAA|45582|12|39|42|PM|first|afternoon|lunch| +45583|AAAAAAAAABCLAAAA|45583|12|39|43|PM|first|afternoon|lunch| +45584|AAAAAAAABBCLAAAA|45584|12|39|44|PM|first|afternoon|lunch| +45585|AAAAAAAACBCLAAAA|45585|12|39|45|PM|first|afternoon|lunch| +45586|AAAAAAAADBCLAAAA|45586|12|39|46|PM|first|afternoon|lunch| +45587|AAAAAAAAEBCLAAAA|45587|12|39|47|PM|first|afternoon|lunch| +45588|AAAAAAAAFBCLAAAA|45588|12|39|48|PM|first|afternoon|lunch| +45589|AAAAAAAAGBCLAAAA|45589|12|39|49|PM|first|afternoon|lunch| +45590|AAAAAAAAHBCLAAAA|45590|12|39|50|PM|first|afternoon|lunch| +45591|AAAAAAAAIBCLAAAA|45591|12|39|51|PM|first|afternoon|lunch| +45592|AAAAAAAAJBCLAAAA|45592|12|39|52|PM|first|afternoon|lunch| +45593|AAAAAAAAKBCLAAAA|45593|12|39|53|PM|first|afternoon|lunch| +45594|AAAAAAAALBCLAAAA|45594|12|39|54|PM|first|afternoon|lunch| +45595|AAAAAAAAMBCLAAAA|45595|12|39|55|PM|first|afternoon|lunch| +45596|AAAAAAAANBCLAAAA|45596|12|39|56|PM|first|afternoon|lunch| +45597|AAAAAAAAOBCLAAAA|45597|12|39|57|PM|first|afternoon|lunch| +45598|AAAAAAAAPBCLAAAA|45598|12|39|58|PM|first|afternoon|lunch| +45599|AAAAAAAAACCLAAAA|45599|12|39|59|PM|first|afternoon|lunch| +45600|AAAAAAAABCCLAAAA|45600|12|40|0|PM|first|afternoon|lunch| +45601|AAAAAAAACCCLAAAA|45601|12|40|1|PM|first|afternoon|lunch| +45602|AAAAAAAADCCLAAAA|45602|12|40|2|PM|first|afternoon|lunch| +45603|AAAAAAAAECCLAAAA|45603|12|40|3|PM|first|afternoon|lunch| +45604|AAAAAAAAFCCLAAAA|45604|12|40|4|PM|first|afternoon|lunch| +45605|AAAAAAAAGCCLAAAA|45605|12|40|5|PM|first|afternoon|lunch| +45606|AAAAAAAAHCCLAAAA|45606|12|40|6|PM|first|afternoon|lunch| +45607|AAAAAAAAICCLAAAA|45607|12|40|7|PM|first|afternoon|lunch| +45608|AAAAAAAAJCCLAAAA|45608|12|40|8|PM|first|afternoon|lunch| +45609|AAAAAAAAKCCLAAAA|45609|12|40|9|PM|first|afternoon|lunch| +45610|AAAAAAAALCCLAAAA|45610|12|40|10|PM|first|afternoon|lunch| +45611|AAAAAAAAMCCLAAAA|45611|12|40|11|PM|first|afternoon|lunch| +45612|AAAAAAAANCCLAAAA|45612|12|40|12|PM|first|afternoon|lunch| +45613|AAAAAAAAOCCLAAAA|45613|12|40|13|PM|first|afternoon|lunch| +45614|AAAAAAAAPCCLAAAA|45614|12|40|14|PM|first|afternoon|lunch| +45615|AAAAAAAAADCLAAAA|45615|12|40|15|PM|first|afternoon|lunch| +45616|AAAAAAAABDCLAAAA|45616|12|40|16|PM|first|afternoon|lunch| +45617|AAAAAAAACDCLAAAA|45617|12|40|17|PM|first|afternoon|lunch| +45618|AAAAAAAADDCLAAAA|45618|12|40|18|PM|first|afternoon|lunch| +45619|AAAAAAAAEDCLAAAA|45619|12|40|19|PM|first|afternoon|lunch| +45620|AAAAAAAAFDCLAAAA|45620|12|40|20|PM|first|afternoon|lunch| +45621|AAAAAAAAGDCLAAAA|45621|12|40|21|PM|first|afternoon|lunch| +45622|AAAAAAAAHDCLAAAA|45622|12|40|22|PM|first|afternoon|lunch| +45623|AAAAAAAAIDCLAAAA|45623|12|40|23|PM|first|afternoon|lunch| +45624|AAAAAAAAJDCLAAAA|45624|12|40|24|PM|first|afternoon|lunch| +45625|AAAAAAAAKDCLAAAA|45625|12|40|25|PM|first|afternoon|lunch| +45626|AAAAAAAALDCLAAAA|45626|12|40|26|PM|first|afternoon|lunch| +45627|AAAAAAAAMDCLAAAA|45627|12|40|27|PM|first|afternoon|lunch| +45628|AAAAAAAANDCLAAAA|45628|12|40|28|PM|first|afternoon|lunch| +45629|AAAAAAAAODCLAAAA|45629|12|40|29|PM|first|afternoon|lunch| +45630|AAAAAAAAPDCLAAAA|45630|12|40|30|PM|first|afternoon|lunch| +45631|AAAAAAAAAECLAAAA|45631|12|40|31|PM|first|afternoon|lunch| +45632|AAAAAAAABECLAAAA|45632|12|40|32|PM|first|afternoon|lunch| +45633|AAAAAAAACECLAAAA|45633|12|40|33|PM|first|afternoon|lunch| +45634|AAAAAAAADECLAAAA|45634|12|40|34|PM|first|afternoon|lunch| +45635|AAAAAAAAEECLAAAA|45635|12|40|35|PM|first|afternoon|lunch| +45636|AAAAAAAAFECLAAAA|45636|12|40|36|PM|first|afternoon|lunch| +45637|AAAAAAAAGECLAAAA|45637|12|40|37|PM|first|afternoon|lunch| +45638|AAAAAAAAHECLAAAA|45638|12|40|38|PM|first|afternoon|lunch| +45639|AAAAAAAAIECLAAAA|45639|12|40|39|PM|first|afternoon|lunch| +45640|AAAAAAAAJECLAAAA|45640|12|40|40|PM|first|afternoon|lunch| +45641|AAAAAAAAKECLAAAA|45641|12|40|41|PM|first|afternoon|lunch| +45642|AAAAAAAALECLAAAA|45642|12|40|42|PM|first|afternoon|lunch| +45643|AAAAAAAAMECLAAAA|45643|12|40|43|PM|first|afternoon|lunch| +45644|AAAAAAAANECLAAAA|45644|12|40|44|PM|first|afternoon|lunch| +45645|AAAAAAAAOECLAAAA|45645|12|40|45|PM|first|afternoon|lunch| +45646|AAAAAAAAPECLAAAA|45646|12|40|46|PM|first|afternoon|lunch| +45647|AAAAAAAAAFCLAAAA|45647|12|40|47|PM|first|afternoon|lunch| +45648|AAAAAAAABFCLAAAA|45648|12|40|48|PM|first|afternoon|lunch| +45649|AAAAAAAACFCLAAAA|45649|12|40|49|PM|first|afternoon|lunch| +45650|AAAAAAAADFCLAAAA|45650|12|40|50|PM|first|afternoon|lunch| +45651|AAAAAAAAEFCLAAAA|45651|12|40|51|PM|first|afternoon|lunch| +45652|AAAAAAAAFFCLAAAA|45652|12|40|52|PM|first|afternoon|lunch| +45653|AAAAAAAAGFCLAAAA|45653|12|40|53|PM|first|afternoon|lunch| +45654|AAAAAAAAHFCLAAAA|45654|12|40|54|PM|first|afternoon|lunch| +45655|AAAAAAAAIFCLAAAA|45655|12|40|55|PM|first|afternoon|lunch| +45656|AAAAAAAAJFCLAAAA|45656|12|40|56|PM|first|afternoon|lunch| +45657|AAAAAAAAKFCLAAAA|45657|12|40|57|PM|first|afternoon|lunch| +45658|AAAAAAAALFCLAAAA|45658|12|40|58|PM|first|afternoon|lunch| +45659|AAAAAAAAMFCLAAAA|45659|12|40|59|PM|first|afternoon|lunch| +45660|AAAAAAAANFCLAAAA|45660|12|41|0|PM|first|afternoon|lunch| +45661|AAAAAAAAOFCLAAAA|45661|12|41|1|PM|first|afternoon|lunch| +45662|AAAAAAAAPFCLAAAA|45662|12|41|2|PM|first|afternoon|lunch| +45663|AAAAAAAAAGCLAAAA|45663|12|41|3|PM|first|afternoon|lunch| +45664|AAAAAAAABGCLAAAA|45664|12|41|4|PM|first|afternoon|lunch| +45665|AAAAAAAACGCLAAAA|45665|12|41|5|PM|first|afternoon|lunch| +45666|AAAAAAAADGCLAAAA|45666|12|41|6|PM|first|afternoon|lunch| +45667|AAAAAAAAEGCLAAAA|45667|12|41|7|PM|first|afternoon|lunch| +45668|AAAAAAAAFGCLAAAA|45668|12|41|8|PM|first|afternoon|lunch| +45669|AAAAAAAAGGCLAAAA|45669|12|41|9|PM|first|afternoon|lunch| +45670|AAAAAAAAHGCLAAAA|45670|12|41|10|PM|first|afternoon|lunch| +45671|AAAAAAAAIGCLAAAA|45671|12|41|11|PM|first|afternoon|lunch| +45672|AAAAAAAAJGCLAAAA|45672|12|41|12|PM|first|afternoon|lunch| +45673|AAAAAAAAKGCLAAAA|45673|12|41|13|PM|first|afternoon|lunch| +45674|AAAAAAAALGCLAAAA|45674|12|41|14|PM|first|afternoon|lunch| +45675|AAAAAAAAMGCLAAAA|45675|12|41|15|PM|first|afternoon|lunch| +45676|AAAAAAAANGCLAAAA|45676|12|41|16|PM|first|afternoon|lunch| +45677|AAAAAAAAOGCLAAAA|45677|12|41|17|PM|first|afternoon|lunch| +45678|AAAAAAAAPGCLAAAA|45678|12|41|18|PM|first|afternoon|lunch| +45679|AAAAAAAAAHCLAAAA|45679|12|41|19|PM|first|afternoon|lunch| +45680|AAAAAAAABHCLAAAA|45680|12|41|20|PM|first|afternoon|lunch| +45681|AAAAAAAACHCLAAAA|45681|12|41|21|PM|first|afternoon|lunch| +45682|AAAAAAAADHCLAAAA|45682|12|41|22|PM|first|afternoon|lunch| +45683|AAAAAAAAEHCLAAAA|45683|12|41|23|PM|first|afternoon|lunch| +45684|AAAAAAAAFHCLAAAA|45684|12|41|24|PM|first|afternoon|lunch| +45685|AAAAAAAAGHCLAAAA|45685|12|41|25|PM|first|afternoon|lunch| +45686|AAAAAAAAHHCLAAAA|45686|12|41|26|PM|first|afternoon|lunch| +45687|AAAAAAAAIHCLAAAA|45687|12|41|27|PM|first|afternoon|lunch| +45688|AAAAAAAAJHCLAAAA|45688|12|41|28|PM|first|afternoon|lunch| +45689|AAAAAAAAKHCLAAAA|45689|12|41|29|PM|first|afternoon|lunch| +45690|AAAAAAAALHCLAAAA|45690|12|41|30|PM|first|afternoon|lunch| +45691|AAAAAAAAMHCLAAAA|45691|12|41|31|PM|first|afternoon|lunch| +45692|AAAAAAAANHCLAAAA|45692|12|41|32|PM|first|afternoon|lunch| +45693|AAAAAAAAOHCLAAAA|45693|12|41|33|PM|first|afternoon|lunch| +45694|AAAAAAAAPHCLAAAA|45694|12|41|34|PM|first|afternoon|lunch| +45695|AAAAAAAAAICLAAAA|45695|12|41|35|PM|first|afternoon|lunch| +45696|AAAAAAAABICLAAAA|45696|12|41|36|PM|first|afternoon|lunch| +45697|AAAAAAAACICLAAAA|45697|12|41|37|PM|first|afternoon|lunch| +45698|AAAAAAAADICLAAAA|45698|12|41|38|PM|first|afternoon|lunch| +45699|AAAAAAAAEICLAAAA|45699|12|41|39|PM|first|afternoon|lunch| +45700|AAAAAAAAFICLAAAA|45700|12|41|40|PM|first|afternoon|lunch| +45701|AAAAAAAAGICLAAAA|45701|12|41|41|PM|first|afternoon|lunch| +45702|AAAAAAAAHICLAAAA|45702|12|41|42|PM|first|afternoon|lunch| +45703|AAAAAAAAIICLAAAA|45703|12|41|43|PM|first|afternoon|lunch| +45704|AAAAAAAAJICLAAAA|45704|12|41|44|PM|first|afternoon|lunch| +45705|AAAAAAAAKICLAAAA|45705|12|41|45|PM|first|afternoon|lunch| +45706|AAAAAAAALICLAAAA|45706|12|41|46|PM|first|afternoon|lunch| +45707|AAAAAAAAMICLAAAA|45707|12|41|47|PM|first|afternoon|lunch| +45708|AAAAAAAANICLAAAA|45708|12|41|48|PM|first|afternoon|lunch| +45709|AAAAAAAAOICLAAAA|45709|12|41|49|PM|first|afternoon|lunch| +45710|AAAAAAAAPICLAAAA|45710|12|41|50|PM|first|afternoon|lunch| +45711|AAAAAAAAAJCLAAAA|45711|12|41|51|PM|first|afternoon|lunch| +45712|AAAAAAAABJCLAAAA|45712|12|41|52|PM|first|afternoon|lunch| +45713|AAAAAAAACJCLAAAA|45713|12|41|53|PM|first|afternoon|lunch| +45714|AAAAAAAADJCLAAAA|45714|12|41|54|PM|first|afternoon|lunch| +45715|AAAAAAAAEJCLAAAA|45715|12|41|55|PM|first|afternoon|lunch| +45716|AAAAAAAAFJCLAAAA|45716|12|41|56|PM|first|afternoon|lunch| +45717|AAAAAAAAGJCLAAAA|45717|12|41|57|PM|first|afternoon|lunch| +45718|AAAAAAAAHJCLAAAA|45718|12|41|58|PM|first|afternoon|lunch| +45719|AAAAAAAAIJCLAAAA|45719|12|41|59|PM|first|afternoon|lunch| +45720|AAAAAAAAJJCLAAAA|45720|12|42|0|PM|first|afternoon|lunch| +45721|AAAAAAAAKJCLAAAA|45721|12|42|1|PM|first|afternoon|lunch| +45722|AAAAAAAALJCLAAAA|45722|12|42|2|PM|first|afternoon|lunch| +45723|AAAAAAAAMJCLAAAA|45723|12|42|3|PM|first|afternoon|lunch| +45724|AAAAAAAANJCLAAAA|45724|12|42|4|PM|first|afternoon|lunch| +45725|AAAAAAAAOJCLAAAA|45725|12|42|5|PM|first|afternoon|lunch| +45726|AAAAAAAAPJCLAAAA|45726|12|42|6|PM|first|afternoon|lunch| +45727|AAAAAAAAAKCLAAAA|45727|12|42|7|PM|first|afternoon|lunch| +45728|AAAAAAAABKCLAAAA|45728|12|42|8|PM|first|afternoon|lunch| +45729|AAAAAAAACKCLAAAA|45729|12|42|9|PM|first|afternoon|lunch| +45730|AAAAAAAADKCLAAAA|45730|12|42|10|PM|first|afternoon|lunch| +45731|AAAAAAAAEKCLAAAA|45731|12|42|11|PM|first|afternoon|lunch| +45732|AAAAAAAAFKCLAAAA|45732|12|42|12|PM|first|afternoon|lunch| +45733|AAAAAAAAGKCLAAAA|45733|12|42|13|PM|first|afternoon|lunch| +45734|AAAAAAAAHKCLAAAA|45734|12|42|14|PM|first|afternoon|lunch| +45735|AAAAAAAAIKCLAAAA|45735|12|42|15|PM|first|afternoon|lunch| +45736|AAAAAAAAJKCLAAAA|45736|12|42|16|PM|first|afternoon|lunch| +45737|AAAAAAAAKKCLAAAA|45737|12|42|17|PM|first|afternoon|lunch| +45738|AAAAAAAALKCLAAAA|45738|12|42|18|PM|first|afternoon|lunch| +45739|AAAAAAAAMKCLAAAA|45739|12|42|19|PM|first|afternoon|lunch| +45740|AAAAAAAANKCLAAAA|45740|12|42|20|PM|first|afternoon|lunch| +45741|AAAAAAAAOKCLAAAA|45741|12|42|21|PM|first|afternoon|lunch| +45742|AAAAAAAAPKCLAAAA|45742|12|42|22|PM|first|afternoon|lunch| +45743|AAAAAAAAALCLAAAA|45743|12|42|23|PM|first|afternoon|lunch| +45744|AAAAAAAABLCLAAAA|45744|12|42|24|PM|first|afternoon|lunch| +45745|AAAAAAAACLCLAAAA|45745|12|42|25|PM|first|afternoon|lunch| +45746|AAAAAAAADLCLAAAA|45746|12|42|26|PM|first|afternoon|lunch| +45747|AAAAAAAAELCLAAAA|45747|12|42|27|PM|first|afternoon|lunch| +45748|AAAAAAAAFLCLAAAA|45748|12|42|28|PM|first|afternoon|lunch| +45749|AAAAAAAAGLCLAAAA|45749|12|42|29|PM|first|afternoon|lunch| +45750|AAAAAAAAHLCLAAAA|45750|12|42|30|PM|first|afternoon|lunch| +45751|AAAAAAAAILCLAAAA|45751|12|42|31|PM|first|afternoon|lunch| +45752|AAAAAAAAJLCLAAAA|45752|12|42|32|PM|first|afternoon|lunch| +45753|AAAAAAAAKLCLAAAA|45753|12|42|33|PM|first|afternoon|lunch| +45754|AAAAAAAALLCLAAAA|45754|12|42|34|PM|first|afternoon|lunch| +45755|AAAAAAAAMLCLAAAA|45755|12|42|35|PM|first|afternoon|lunch| +45756|AAAAAAAANLCLAAAA|45756|12|42|36|PM|first|afternoon|lunch| +45757|AAAAAAAAOLCLAAAA|45757|12|42|37|PM|first|afternoon|lunch| +45758|AAAAAAAAPLCLAAAA|45758|12|42|38|PM|first|afternoon|lunch| +45759|AAAAAAAAAMCLAAAA|45759|12|42|39|PM|first|afternoon|lunch| +45760|AAAAAAAABMCLAAAA|45760|12|42|40|PM|first|afternoon|lunch| +45761|AAAAAAAACMCLAAAA|45761|12|42|41|PM|first|afternoon|lunch| +45762|AAAAAAAADMCLAAAA|45762|12|42|42|PM|first|afternoon|lunch| +45763|AAAAAAAAEMCLAAAA|45763|12|42|43|PM|first|afternoon|lunch| +45764|AAAAAAAAFMCLAAAA|45764|12|42|44|PM|first|afternoon|lunch| +45765|AAAAAAAAGMCLAAAA|45765|12|42|45|PM|first|afternoon|lunch| +45766|AAAAAAAAHMCLAAAA|45766|12|42|46|PM|first|afternoon|lunch| +45767|AAAAAAAAIMCLAAAA|45767|12|42|47|PM|first|afternoon|lunch| +45768|AAAAAAAAJMCLAAAA|45768|12|42|48|PM|first|afternoon|lunch| +45769|AAAAAAAAKMCLAAAA|45769|12|42|49|PM|first|afternoon|lunch| +45770|AAAAAAAALMCLAAAA|45770|12|42|50|PM|first|afternoon|lunch| +45771|AAAAAAAAMMCLAAAA|45771|12|42|51|PM|first|afternoon|lunch| +45772|AAAAAAAANMCLAAAA|45772|12|42|52|PM|first|afternoon|lunch| +45773|AAAAAAAAOMCLAAAA|45773|12|42|53|PM|first|afternoon|lunch| +45774|AAAAAAAAPMCLAAAA|45774|12|42|54|PM|first|afternoon|lunch| +45775|AAAAAAAAANCLAAAA|45775|12|42|55|PM|first|afternoon|lunch| +45776|AAAAAAAABNCLAAAA|45776|12|42|56|PM|first|afternoon|lunch| +45777|AAAAAAAACNCLAAAA|45777|12|42|57|PM|first|afternoon|lunch| +45778|AAAAAAAADNCLAAAA|45778|12|42|58|PM|first|afternoon|lunch| +45779|AAAAAAAAENCLAAAA|45779|12|42|59|PM|first|afternoon|lunch| +45780|AAAAAAAAFNCLAAAA|45780|12|43|0|PM|first|afternoon|lunch| +45781|AAAAAAAAGNCLAAAA|45781|12|43|1|PM|first|afternoon|lunch| +45782|AAAAAAAAHNCLAAAA|45782|12|43|2|PM|first|afternoon|lunch| +45783|AAAAAAAAINCLAAAA|45783|12|43|3|PM|first|afternoon|lunch| +45784|AAAAAAAAJNCLAAAA|45784|12|43|4|PM|first|afternoon|lunch| +45785|AAAAAAAAKNCLAAAA|45785|12|43|5|PM|first|afternoon|lunch| +45786|AAAAAAAALNCLAAAA|45786|12|43|6|PM|first|afternoon|lunch| +45787|AAAAAAAAMNCLAAAA|45787|12|43|7|PM|first|afternoon|lunch| +45788|AAAAAAAANNCLAAAA|45788|12|43|8|PM|first|afternoon|lunch| +45789|AAAAAAAAONCLAAAA|45789|12|43|9|PM|first|afternoon|lunch| +45790|AAAAAAAAPNCLAAAA|45790|12|43|10|PM|first|afternoon|lunch| +45791|AAAAAAAAAOCLAAAA|45791|12|43|11|PM|first|afternoon|lunch| +45792|AAAAAAAABOCLAAAA|45792|12|43|12|PM|first|afternoon|lunch| +45793|AAAAAAAACOCLAAAA|45793|12|43|13|PM|first|afternoon|lunch| +45794|AAAAAAAADOCLAAAA|45794|12|43|14|PM|first|afternoon|lunch| +45795|AAAAAAAAEOCLAAAA|45795|12|43|15|PM|first|afternoon|lunch| +45796|AAAAAAAAFOCLAAAA|45796|12|43|16|PM|first|afternoon|lunch| +45797|AAAAAAAAGOCLAAAA|45797|12|43|17|PM|first|afternoon|lunch| +45798|AAAAAAAAHOCLAAAA|45798|12|43|18|PM|first|afternoon|lunch| +45799|AAAAAAAAIOCLAAAA|45799|12|43|19|PM|first|afternoon|lunch| +45800|AAAAAAAAJOCLAAAA|45800|12|43|20|PM|first|afternoon|lunch| +45801|AAAAAAAAKOCLAAAA|45801|12|43|21|PM|first|afternoon|lunch| +45802|AAAAAAAALOCLAAAA|45802|12|43|22|PM|first|afternoon|lunch| +45803|AAAAAAAAMOCLAAAA|45803|12|43|23|PM|first|afternoon|lunch| +45804|AAAAAAAANOCLAAAA|45804|12|43|24|PM|first|afternoon|lunch| +45805|AAAAAAAAOOCLAAAA|45805|12|43|25|PM|first|afternoon|lunch| +45806|AAAAAAAAPOCLAAAA|45806|12|43|26|PM|first|afternoon|lunch| +45807|AAAAAAAAAPCLAAAA|45807|12|43|27|PM|first|afternoon|lunch| +45808|AAAAAAAABPCLAAAA|45808|12|43|28|PM|first|afternoon|lunch| +45809|AAAAAAAACPCLAAAA|45809|12|43|29|PM|first|afternoon|lunch| +45810|AAAAAAAADPCLAAAA|45810|12|43|30|PM|first|afternoon|lunch| +45811|AAAAAAAAEPCLAAAA|45811|12|43|31|PM|first|afternoon|lunch| +45812|AAAAAAAAFPCLAAAA|45812|12|43|32|PM|first|afternoon|lunch| +45813|AAAAAAAAGPCLAAAA|45813|12|43|33|PM|first|afternoon|lunch| +45814|AAAAAAAAHPCLAAAA|45814|12|43|34|PM|first|afternoon|lunch| +45815|AAAAAAAAIPCLAAAA|45815|12|43|35|PM|first|afternoon|lunch| +45816|AAAAAAAAJPCLAAAA|45816|12|43|36|PM|first|afternoon|lunch| +45817|AAAAAAAAKPCLAAAA|45817|12|43|37|PM|first|afternoon|lunch| +45818|AAAAAAAALPCLAAAA|45818|12|43|38|PM|first|afternoon|lunch| +45819|AAAAAAAAMPCLAAAA|45819|12|43|39|PM|first|afternoon|lunch| +45820|AAAAAAAANPCLAAAA|45820|12|43|40|PM|first|afternoon|lunch| +45821|AAAAAAAAOPCLAAAA|45821|12|43|41|PM|first|afternoon|lunch| +45822|AAAAAAAAPPCLAAAA|45822|12|43|42|PM|first|afternoon|lunch| +45823|AAAAAAAAAADLAAAA|45823|12|43|43|PM|first|afternoon|lunch| +45824|AAAAAAAABADLAAAA|45824|12|43|44|PM|first|afternoon|lunch| +45825|AAAAAAAACADLAAAA|45825|12|43|45|PM|first|afternoon|lunch| +45826|AAAAAAAADADLAAAA|45826|12|43|46|PM|first|afternoon|lunch| +45827|AAAAAAAAEADLAAAA|45827|12|43|47|PM|first|afternoon|lunch| +45828|AAAAAAAAFADLAAAA|45828|12|43|48|PM|first|afternoon|lunch| +45829|AAAAAAAAGADLAAAA|45829|12|43|49|PM|first|afternoon|lunch| +45830|AAAAAAAAHADLAAAA|45830|12|43|50|PM|first|afternoon|lunch| +45831|AAAAAAAAIADLAAAA|45831|12|43|51|PM|first|afternoon|lunch| +45832|AAAAAAAAJADLAAAA|45832|12|43|52|PM|first|afternoon|lunch| +45833|AAAAAAAAKADLAAAA|45833|12|43|53|PM|first|afternoon|lunch| +45834|AAAAAAAALADLAAAA|45834|12|43|54|PM|first|afternoon|lunch| +45835|AAAAAAAAMADLAAAA|45835|12|43|55|PM|first|afternoon|lunch| +45836|AAAAAAAANADLAAAA|45836|12|43|56|PM|first|afternoon|lunch| +45837|AAAAAAAAOADLAAAA|45837|12|43|57|PM|first|afternoon|lunch| +45838|AAAAAAAAPADLAAAA|45838|12|43|58|PM|first|afternoon|lunch| +45839|AAAAAAAAABDLAAAA|45839|12|43|59|PM|first|afternoon|lunch| +45840|AAAAAAAABBDLAAAA|45840|12|44|0|PM|first|afternoon|lunch| +45841|AAAAAAAACBDLAAAA|45841|12|44|1|PM|first|afternoon|lunch| +45842|AAAAAAAADBDLAAAA|45842|12|44|2|PM|first|afternoon|lunch| +45843|AAAAAAAAEBDLAAAA|45843|12|44|3|PM|first|afternoon|lunch| +45844|AAAAAAAAFBDLAAAA|45844|12|44|4|PM|first|afternoon|lunch| +45845|AAAAAAAAGBDLAAAA|45845|12|44|5|PM|first|afternoon|lunch| +45846|AAAAAAAAHBDLAAAA|45846|12|44|6|PM|first|afternoon|lunch| +45847|AAAAAAAAIBDLAAAA|45847|12|44|7|PM|first|afternoon|lunch| +45848|AAAAAAAAJBDLAAAA|45848|12|44|8|PM|first|afternoon|lunch| +45849|AAAAAAAAKBDLAAAA|45849|12|44|9|PM|first|afternoon|lunch| +45850|AAAAAAAALBDLAAAA|45850|12|44|10|PM|first|afternoon|lunch| +45851|AAAAAAAAMBDLAAAA|45851|12|44|11|PM|first|afternoon|lunch| +45852|AAAAAAAANBDLAAAA|45852|12|44|12|PM|first|afternoon|lunch| +45853|AAAAAAAAOBDLAAAA|45853|12|44|13|PM|first|afternoon|lunch| +45854|AAAAAAAAPBDLAAAA|45854|12|44|14|PM|first|afternoon|lunch| +45855|AAAAAAAAACDLAAAA|45855|12|44|15|PM|first|afternoon|lunch| +45856|AAAAAAAABCDLAAAA|45856|12|44|16|PM|first|afternoon|lunch| +45857|AAAAAAAACCDLAAAA|45857|12|44|17|PM|first|afternoon|lunch| +45858|AAAAAAAADCDLAAAA|45858|12|44|18|PM|first|afternoon|lunch| +45859|AAAAAAAAECDLAAAA|45859|12|44|19|PM|first|afternoon|lunch| +45860|AAAAAAAAFCDLAAAA|45860|12|44|20|PM|first|afternoon|lunch| +45861|AAAAAAAAGCDLAAAA|45861|12|44|21|PM|first|afternoon|lunch| +45862|AAAAAAAAHCDLAAAA|45862|12|44|22|PM|first|afternoon|lunch| +45863|AAAAAAAAICDLAAAA|45863|12|44|23|PM|first|afternoon|lunch| +45864|AAAAAAAAJCDLAAAA|45864|12|44|24|PM|first|afternoon|lunch| +45865|AAAAAAAAKCDLAAAA|45865|12|44|25|PM|first|afternoon|lunch| +45866|AAAAAAAALCDLAAAA|45866|12|44|26|PM|first|afternoon|lunch| +45867|AAAAAAAAMCDLAAAA|45867|12|44|27|PM|first|afternoon|lunch| +45868|AAAAAAAANCDLAAAA|45868|12|44|28|PM|first|afternoon|lunch| +45869|AAAAAAAAOCDLAAAA|45869|12|44|29|PM|first|afternoon|lunch| +45870|AAAAAAAAPCDLAAAA|45870|12|44|30|PM|first|afternoon|lunch| +45871|AAAAAAAAADDLAAAA|45871|12|44|31|PM|first|afternoon|lunch| +45872|AAAAAAAABDDLAAAA|45872|12|44|32|PM|first|afternoon|lunch| +45873|AAAAAAAACDDLAAAA|45873|12|44|33|PM|first|afternoon|lunch| +45874|AAAAAAAADDDLAAAA|45874|12|44|34|PM|first|afternoon|lunch| +45875|AAAAAAAAEDDLAAAA|45875|12|44|35|PM|first|afternoon|lunch| +45876|AAAAAAAAFDDLAAAA|45876|12|44|36|PM|first|afternoon|lunch| +45877|AAAAAAAAGDDLAAAA|45877|12|44|37|PM|first|afternoon|lunch| +45878|AAAAAAAAHDDLAAAA|45878|12|44|38|PM|first|afternoon|lunch| +45879|AAAAAAAAIDDLAAAA|45879|12|44|39|PM|first|afternoon|lunch| +45880|AAAAAAAAJDDLAAAA|45880|12|44|40|PM|first|afternoon|lunch| +45881|AAAAAAAAKDDLAAAA|45881|12|44|41|PM|first|afternoon|lunch| +45882|AAAAAAAALDDLAAAA|45882|12|44|42|PM|first|afternoon|lunch| +45883|AAAAAAAAMDDLAAAA|45883|12|44|43|PM|first|afternoon|lunch| +45884|AAAAAAAANDDLAAAA|45884|12|44|44|PM|first|afternoon|lunch| +45885|AAAAAAAAODDLAAAA|45885|12|44|45|PM|first|afternoon|lunch| +45886|AAAAAAAAPDDLAAAA|45886|12|44|46|PM|first|afternoon|lunch| +45887|AAAAAAAAAEDLAAAA|45887|12|44|47|PM|first|afternoon|lunch| +45888|AAAAAAAABEDLAAAA|45888|12|44|48|PM|first|afternoon|lunch| +45889|AAAAAAAACEDLAAAA|45889|12|44|49|PM|first|afternoon|lunch| +45890|AAAAAAAADEDLAAAA|45890|12|44|50|PM|first|afternoon|lunch| +45891|AAAAAAAAEEDLAAAA|45891|12|44|51|PM|first|afternoon|lunch| +45892|AAAAAAAAFEDLAAAA|45892|12|44|52|PM|first|afternoon|lunch| +45893|AAAAAAAAGEDLAAAA|45893|12|44|53|PM|first|afternoon|lunch| +45894|AAAAAAAAHEDLAAAA|45894|12|44|54|PM|first|afternoon|lunch| +45895|AAAAAAAAIEDLAAAA|45895|12|44|55|PM|first|afternoon|lunch| +45896|AAAAAAAAJEDLAAAA|45896|12|44|56|PM|first|afternoon|lunch| +45897|AAAAAAAAKEDLAAAA|45897|12|44|57|PM|first|afternoon|lunch| +45898|AAAAAAAALEDLAAAA|45898|12|44|58|PM|first|afternoon|lunch| +45899|AAAAAAAAMEDLAAAA|45899|12|44|59|PM|first|afternoon|lunch| +45900|AAAAAAAANEDLAAAA|45900|12|45|0|PM|first|afternoon|lunch| +45901|AAAAAAAAOEDLAAAA|45901|12|45|1|PM|first|afternoon|lunch| +45902|AAAAAAAAPEDLAAAA|45902|12|45|2|PM|first|afternoon|lunch| +45903|AAAAAAAAAFDLAAAA|45903|12|45|3|PM|first|afternoon|lunch| +45904|AAAAAAAABFDLAAAA|45904|12|45|4|PM|first|afternoon|lunch| +45905|AAAAAAAACFDLAAAA|45905|12|45|5|PM|first|afternoon|lunch| +45906|AAAAAAAADFDLAAAA|45906|12|45|6|PM|first|afternoon|lunch| +45907|AAAAAAAAEFDLAAAA|45907|12|45|7|PM|first|afternoon|lunch| +45908|AAAAAAAAFFDLAAAA|45908|12|45|8|PM|first|afternoon|lunch| +45909|AAAAAAAAGFDLAAAA|45909|12|45|9|PM|first|afternoon|lunch| +45910|AAAAAAAAHFDLAAAA|45910|12|45|10|PM|first|afternoon|lunch| +45911|AAAAAAAAIFDLAAAA|45911|12|45|11|PM|first|afternoon|lunch| +45912|AAAAAAAAJFDLAAAA|45912|12|45|12|PM|first|afternoon|lunch| +45913|AAAAAAAAKFDLAAAA|45913|12|45|13|PM|first|afternoon|lunch| +45914|AAAAAAAALFDLAAAA|45914|12|45|14|PM|first|afternoon|lunch| +45915|AAAAAAAAMFDLAAAA|45915|12|45|15|PM|first|afternoon|lunch| +45916|AAAAAAAANFDLAAAA|45916|12|45|16|PM|first|afternoon|lunch| +45917|AAAAAAAAOFDLAAAA|45917|12|45|17|PM|first|afternoon|lunch| +45918|AAAAAAAAPFDLAAAA|45918|12|45|18|PM|first|afternoon|lunch| +45919|AAAAAAAAAGDLAAAA|45919|12|45|19|PM|first|afternoon|lunch| +45920|AAAAAAAABGDLAAAA|45920|12|45|20|PM|first|afternoon|lunch| +45921|AAAAAAAACGDLAAAA|45921|12|45|21|PM|first|afternoon|lunch| +45922|AAAAAAAADGDLAAAA|45922|12|45|22|PM|first|afternoon|lunch| +45923|AAAAAAAAEGDLAAAA|45923|12|45|23|PM|first|afternoon|lunch| +45924|AAAAAAAAFGDLAAAA|45924|12|45|24|PM|first|afternoon|lunch| +45925|AAAAAAAAGGDLAAAA|45925|12|45|25|PM|first|afternoon|lunch| +45926|AAAAAAAAHGDLAAAA|45926|12|45|26|PM|first|afternoon|lunch| +45927|AAAAAAAAIGDLAAAA|45927|12|45|27|PM|first|afternoon|lunch| +45928|AAAAAAAAJGDLAAAA|45928|12|45|28|PM|first|afternoon|lunch| +45929|AAAAAAAAKGDLAAAA|45929|12|45|29|PM|first|afternoon|lunch| +45930|AAAAAAAALGDLAAAA|45930|12|45|30|PM|first|afternoon|lunch| +45931|AAAAAAAAMGDLAAAA|45931|12|45|31|PM|first|afternoon|lunch| +45932|AAAAAAAANGDLAAAA|45932|12|45|32|PM|first|afternoon|lunch| +45933|AAAAAAAAOGDLAAAA|45933|12|45|33|PM|first|afternoon|lunch| +45934|AAAAAAAAPGDLAAAA|45934|12|45|34|PM|first|afternoon|lunch| +45935|AAAAAAAAAHDLAAAA|45935|12|45|35|PM|first|afternoon|lunch| +45936|AAAAAAAABHDLAAAA|45936|12|45|36|PM|first|afternoon|lunch| +45937|AAAAAAAACHDLAAAA|45937|12|45|37|PM|first|afternoon|lunch| +45938|AAAAAAAADHDLAAAA|45938|12|45|38|PM|first|afternoon|lunch| +45939|AAAAAAAAEHDLAAAA|45939|12|45|39|PM|first|afternoon|lunch| +45940|AAAAAAAAFHDLAAAA|45940|12|45|40|PM|first|afternoon|lunch| +45941|AAAAAAAAGHDLAAAA|45941|12|45|41|PM|first|afternoon|lunch| +45942|AAAAAAAAHHDLAAAA|45942|12|45|42|PM|first|afternoon|lunch| +45943|AAAAAAAAIHDLAAAA|45943|12|45|43|PM|first|afternoon|lunch| +45944|AAAAAAAAJHDLAAAA|45944|12|45|44|PM|first|afternoon|lunch| +45945|AAAAAAAAKHDLAAAA|45945|12|45|45|PM|first|afternoon|lunch| +45946|AAAAAAAALHDLAAAA|45946|12|45|46|PM|first|afternoon|lunch| +45947|AAAAAAAAMHDLAAAA|45947|12|45|47|PM|first|afternoon|lunch| +45948|AAAAAAAANHDLAAAA|45948|12|45|48|PM|first|afternoon|lunch| +45949|AAAAAAAAOHDLAAAA|45949|12|45|49|PM|first|afternoon|lunch| +45950|AAAAAAAAPHDLAAAA|45950|12|45|50|PM|first|afternoon|lunch| +45951|AAAAAAAAAIDLAAAA|45951|12|45|51|PM|first|afternoon|lunch| +45952|AAAAAAAABIDLAAAA|45952|12|45|52|PM|first|afternoon|lunch| +45953|AAAAAAAACIDLAAAA|45953|12|45|53|PM|first|afternoon|lunch| +45954|AAAAAAAADIDLAAAA|45954|12|45|54|PM|first|afternoon|lunch| +45955|AAAAAAAAEIDLAAAA|45955|12|45|55|PM|first|afternoon|lunch| +45956|AAAAAAAAFIDLAAAA|45956|12|45|56|PM|first|afternoon|lunch| +45957|AAAAAAAAGIDLAAAA|45957|12|45|57|PM|first|afternoon|lunch| +45958|AAAAAAAAHIDLAAAA|45958|12|45|58|PM|first|afternoon|lunch| +45959|AAAAAAAAIIDLAAAA|45959|12|45|59|PM|first|afternoon|lunch| +45960|AAAAAAAAJIDLAAAA|45960|12|46|0|PM|first|afternoon|lunch| +45961|AAAAAAAAKIDLAAAA|45961|12|46|1|PM|first|afternoon|lunch| +45962|AAAAAAAALIDLAAAA|45962|12|46|2|PM|first|afternoon|lunch| +45963|AAAAAAAAMIDLAAAA|45963|12|46|3|PM|first|afternoon|lunch| +45964|AAAAAAAANIDLAAAA|45964|12|46|4|PM|first|afternoon|lunch| +45965|AAAAAAAAOIDLAAAA|45965|12|46|5|PM|first|afternoon|lunch| +45966|AAAAAAAAPIDLAAAA|45966|12|46|6|PM|first|afternoon|lunch| +45967|AAAAAAAAAJDLAAAA|45967|12|46|7|PM|first|afternoon|lunch| +45968|AAAAAAAABJDLAAAA|45968|12|46|8|PM|first|afternoon|lunch| +45969|AAAAAAAACJDLAAAA|45969|12|46|9|PM|first|afternoon|lunch| +45970|AAAAAAAADJDLAAAA|45970|12|46|10|PM|first|afternoon|lunch| +45971|AAAAAAAAEJDLAAAA|45971|12|46|11|PM|first|afternoon|lunch| +45972|AAAAAAAAFJDLAAAA|45972|12|46|12|PM|first|afternoon|lunch| +45973|AAAAAAAAGJDLAAAA|45973|12|46|13|PM|first|afternoon|lunch| +45974|AAAAAAAAHJDLAAAA|45974|12|46|14|PM|first|afternoon|lunch| +45975|AAAAAAAAIJDLAAAA|45975|12|46|15|PM|first|afternoon|lunch| +45976|AAAAAAAAJJDLAAAA|45976|12|46|16|PM|first|afternoon|lunch| +45977|AAAAAAAAKJDLAAAA|45977|12|46|17|PM|first|afternoon|lunch| +45978|AAAAAAAALJDLAAAA|45978|12|46|18|PM|first|afternoon|lunch| +45979|AAAAAAAAMJDLAAAA|45979|12|46|19|PM|first|afternoon|lunch| +45980|AAAAAAAANJDLAAAA|45980|12|46|20|PM|first|afternoon|lunch| +45981|AAAAAAAAOJDLAAAA|45981|12|46|21|PM|first|afternoon|lunch| +45982|AAAAAAAAPJDLAAAA|45982|12|46|22|PM|first|afternoon|lunch| +45983|AAAAAAAAAKDLAAAA|45983|12|46|23|PM|first|afternoon|lunch| +45984|AAAAAAAABKDLAAAA|45984|12|46|24|PM|first|afternoon|lunch| +45985|AAAAAAAACKDLAAAA|45985|12|46|25|PM|first|afternoon|lunch| +45986|AAAAAAAADKDLAAAA|45986|12|46|26|PM|first|afternoon|lunch| +45987|AAAAAAAAEKDLAAAA|45987|12|46|27|PM|first|afternoon|lunch| +45988|AAAAAAAAFKDLAAAA|45988|12|46|28|PM|first|afternoon|lunch| +45989|AAAAAAAAGKDLAAAA|45989|12|46|29|PM|first|afternoon|lunch| +45990|AAAAAAAAHKDLAAAA|45990|12|46|30|PM|first|afternoon|lunch| +45991|AAAAAAAAIKDLAAAA|45991|12|46|31|PM|first|afternoon|lunch| +45992|AAAAAAAAJKDLAAAA|45992|12|46|32|PM|first|afternoon|lunch| +45993|AAAAAAAAKKDLAAAA|45993|12|46|33|PM|first|afternoon|lunch| +45994|AAAAAAAALKDLAAAA|45994|12|46|34|PM|first|afternoon|lunch| +45995|AAAAAAAAMKDLAAAA|45995|12|46|35|PM|first|afternoon|lunch| +45996|AAAAAAAANKDLAAAA|45996|12|46|36|PM|first|afternoon|lunch| +45997|AAAAAAAAOKDLAAAA|45997|12|46|37|PM|first|afternoon|lunch| +45998|AAAAAAAAPKDLAAAA|45998|12|46|38|PM|first|afternoon|lunch| +45999|AAAAAAAAALDLAAAA|45999|12|46|39|PM|first|afternoon|lunch| +46000|AAAAAAAABLDLAAAA|46000|12|46|40|PM|first|afternoon|lunch| +46001|AAAAAAAACLDLAAAA|46001|12|46|41|PM|first|afternoon|lunch| +46002|AAAAAAAADLDLAAAA|46002|12|46|42|PM|first|afternoon|lunch| +46003|AAAAAAAAELDLAAAA|46003|12|46|43|PM|first|afternoon|lunch| +46004|AAAAAAAAFLDLAAAA|46004|12|46|44|PM|first|afternoon|lunch| +46005|AAAAAAAAGLDLAAAA|46005|12|46|45|PM|first|afternoon|lunch| +46006|AAAAAAAAHLDLAAAA|46006|12|46|46|PM|first|afternoon|lunch| +46007|AAAAAAAAILDLAAAA|46007|12|46|47|PM|first|afternoon|lunch| +46008|AAAAAAAAJLDLAAAA|46008|12|46|48|PM|first|afternoon|lunch| +46009|AAAAAAAAKLDLAAAA|46009|12|46|49|PM|first|afternoon|lunch| +46010|AAAAAAAALLDLAAAA|46010|12|46|50|PM|first|afternoon|lunch| +46011|AAAAAAAAMLDLAAAA|46011|12|46|51|PM|first|afternoon|lunch| +46012|AAAAAAAANLDLAAAA|46012|12|46|52|PM|first|afternoon|lunch| +46013|AAAAAAAAOLDLAAAA|46013|12|46|53|PM|first|afternoon|lunch| +46014|AAAAAAAAPLDLAAAA|46014|12|46|54|PM|first|afternoon|lunch| +46015|AAAAAAAAAMDLAAAA|46015|12|46|55|PM|first|afternoon|lunch| +46016|AAAAAAAABMDLAAAA|46016|12|46|56|PM|first|afternoon|lunch| +46017|AAAAAAAACMDLAAAA|46017|12|46|57|PM|first|afternoon|lunch| +46018|AAAAAAAADMDLAAAA|46018|12|46|58|PM|first|afternoon|lunch| +46019|AAAAAAAAEMDLAAAA|46019|12|46|59|PM|first|afternoon|lunch| +46020|AAAAAAAAFMDLAAAA|46020|12|47|0|PM|first|afternoon|lunch| +46021|AAAAAAAAGMDLAAAA|46021|12|47|1|PM|first|afternoon|lunch| +46022|AAAAAAAAHMDLAAAA|46022|12|47|2|PM|first|afternoon|lunch| +46023|AAAAAAAAIMDLAAAA|46023|12|47|3|PM|first|afternoon|lunch| +46024|AAAAAAAAJMDLAAAA|46024|12|47|4|PM|first|afternoon|lunch| +46025|AAAAAAAAKMDLAAAA|46025|12|47|5|PM|first|afternoon|lunch| +46026|AAAAAAAALMDLAAAA|46026|12|47|6|PM|first|afternoon|lunch| +46027|AAAAAAAAMMDLAAAA|46027|12|47|7|PM|first|afternoon|lunch| +46028|AAAAAAAANMDLAAAA|46028|12|47|8|PM|first|afternoon|lunch| +46029|AAAAAAAAOMDLAAAA|46029|12|47|9|PM|first|afternoon|lunch| +46030|AAAAAAAAPMDLAAAA|46030|12|47|10|PM|first|afternoon|lunch| +46031|AAAAAAAAANDLAAAA|46031|12|47|11|PM|first|afternoon|lunch| +46032|AAAAAAAABNDLAAAA|46032|12|47|12|PM|first|afternoon|lunch| +46033|AAAAAAAACNDLAAAA|46033|12|47|13|PM|first|afternoon|lunch| +46034|AAAAAAAADNDLAAAA|46034|12|47|14|PM|first|afternoon|lunch| +46035|AAAAAAAAENDLAAAA|46035|12|47|15|PM|first|afternoon|lunch| +46036|AAAAAAAAFNDLAAAA|46036|12|47|16|PM|first|afternoon|lunch| +46037|AAAAAAAAGNDLAAAA|46037|12|47|17|PM|first|afternoon|lunch| +46038|AAAAAAAAHNDLAAAA|46038|12|47|18|PM|first|afternoon|lunch| +46039|AAAAAAAAINDLAAAA|46039|12|47|19|PM|first|afternoon|lunch| +46040|AAAAAAAAJNDLAAAA|46040|12|47|20|PM|first|afternoon|lunch| +46041|AAAAAAAAKNDLAAAA|46041|12|47|21|PM|first|afternoon|lunch| +46042|AAAAAAAALNDLAAAA|46042|12|47|22|PM|first|afternoon|lunch| +46043|AAAAAAAAMNDLAAAA|46043|12|47|23|PM|first|afternoon|lunch| +46044|AAAAAAAANNDLAAAA|46044|12|47|24|PM|first|afternoon|lunch| +46045|AAAAAAAAONDLAAAA|46045|12|47|25|PM|first|afternoon|lunch| +46046|AAAAAAAAPNDLAAAA|46046|12|47|26|PM|first|afternoon|lunch| +46047|AAAAAAAAAODLAAAA|46047|12|47|27|PM|first|afternoon|lunch| +46048|AAAAAAAABODLAAAA|46048|12|47|28|PM|first|afternoon|lunch| +46049|AAAAAAAACODLAAAA|46049|12|47|29|PM|first|afternoon|lunch| +46050|AAAAAAAADODLAAAA|46050|12|47|30|PM|first|afternoon|lunch| +46051|AAAAAAAAEODLAAAA|46051|12|47|31|PM|first|afternoon|lunch| +46052|AAAAAAAAFODLAAAA|46052|12|47|32|PM|first|afternoon|lunch| +46053|AAAAAAAAGODLAAAA|46053|12|47|33|PM|first|afternoon|lunch| +46054|AAAAAAAAHODLAAAA|46054|12|47|34|PM|first|afternoon|lunch| +46055|AAAAAAAAIODLAAAA|46055|12|47|35|PM|first|afternoon|lunch| +46056|AAAAAAAAJODLAAAA|46056|12|47|36|PM|first|afternoon|lunch| +46057|AAAAAAAAKODLAAAA|46057|12|47|37|PM|first|afternoon|lunch| +46058|AAAAAAAALODLAAAA|46058|12|47|38|PM|first|afternoon|lunch| +46059|AAAAAAAAMODLAAAA|46059|12|47|39|PM|first|afternoon|lunch| +46060|AAAAAAAANODLAAAA|46060|12|47|40|PM|first|afternoon|lunch| +46061|AAAAAAAAOODLAAAA|46061|12|47|41|PM|first|afternoon|lunch| +46062|AAAAAAAAPODLAAAA|46062|12|47|42|PM|first|afternoon|lunch| +46063|AAAAAAAAAPDLAAAA|46063|12|47|43|PM|first|afternoon|lunch| +46064|AAAAAAAABPDLAAAA|46064|12|47|44|PM|first|afternoon|lunch| +46065|AAAAAAAACPDLAAAA|46065|12|47|45|PM|first|afternoon|lunch| +46066|AAAAAAAADPDLAAAA|46066|12|47|46|PM|first|afternoon|lunch| +46067|AAAAAAAAEPDLAAAA|46067|12|47|47|PM|first|afternoon|lunch| +46068|AAAAAAAAFPDLAAAA|46068|12|47|48|PM|first|afternoon|lunch| +46069|AAAAAAAAGPDLAAAA|46069|12|47|49|PM|first|afternoon|lunch| +46070|AAAAAAAAHPDLAAAA|46070|12|47|50|PM|first|afternoon|lunch| +46071|AAAAAAAAIPDLAAAA|46071|12|47|51|PM|first|afternoon|lunch| +46072|AAAAAAAAJPDLAAAA|46072|12|47|52|PM|first|afternoon|lunch| +46073|AAAAAAAAKPDLAAAA|46073|12|47|53|PM|first|afternoon|lunch| +46074|AAAAAAAALPDLAAAA|46074|12|47|54|PM|first|afternoon|lunch| +46075|AAAAAAAAMPDLAAAA|46075|12|47|55|PM|first|afternoon|lunch| +46076|AAAAAAAANPDLAAAA|46076|12|47|56|PM|first|afternoon|lunch| +46077|AAAAAAAAOPDLAAAA|46077|12|47|57|PM|first|afternoon|lunch| +46078|AAAAAAAAPPDLAAAA|46078|12|47|58|PM|first|afternoon|lunch| +46079|AAAAAAAAAAELAAAA|46079|12|47|59|PM|first|afternoon|lunch| +46080|AAAAAAAABAELAAAA|46080|12|48|0|PM|first|afternoon|lunch| +46081|AAAAAAAACAELAAAA|46081|12|48|1|PM|first|afternoon|lunch| +46082|AAAAAAAADAELAAAA|46082|12|48|2|PM|first|afternoon|lunch| +46083|AAAAAAAAEAELAAAA|46083|12|48|3|PM|first|afternoon|lunch| +46084|AAAAAAAAFAELAAAA|46084|12|48|4|PM|first|afternoon|lunch| +46085|AAAAAAAAGAELAAAA|46085|12|48|5|PM|first|afternoon|lunch| +46086|AAAAAAAAHAELAAAA|46086|12|48|6|PM|first|afternoon|lunch| +46087|AAAAAAAAIAELAAAA|46087|12|48|7|PM|first|afternoon|lunch| +46088|AAAAAAAAJAELAAAA|46088|12|48|8|PM|first|afternoon|lunch| +46089|AAAAAAAAKAELAAAA|46089|12|48|9|PM|first|afternoon|lunch| +46090|AAAAAAAALAELAAAA|46090|12|48|10|PM|first|afternoon|lunch| +46091|AAAAAAAAMAELAAAA|46091|12|48|11|PM|first|afternoon|lunch| +46092|AAAAAAAANAELAAAA|46092|12|48|12|PM|first|afternoon|lunch| +46093|AAAAAAAAOAELAAAA|46093|12|48|13|PM|first|afternoon|lunch| +46094|AAAAAAAAPAELAAAA|46094|12|48|14|PM|first|afternoon|lunch| +46095|AAAAAAAAABELAAAA|46095|12|48|15|PM|first|afternoon|lunch| +46096|AAAAAAAABBELAAAA|46096|12|48|16|PM|first|afternoon|lunch| +46097|AAAAAAAACBELAAAA|46097|12|48|17|PM|first|afternoon|lunch| +46098|AAAAAAAADBELAAAA|46098|12|48|18|PM|first|afternoon|lunch| +46099|AAAAAAAAEBELAAAA|46099|12|48|19|PM|first|afternoon|lunch| +46100|AAAAAAAAFBELAAAA|46100|12|48|20|PM|first|afternoon|lunch| +46101|AAAAAAAAGBELAAAA|46101|12|48|21|PM|first|afternoon|lunch| +46102|AAAAAAAAHBELAAAA|46102|12|48|22|PM|first|afternoon|lunch| +46103|AAAAAAAAIBELAAAA|46103|12|48|23|PM|first|afternoon|lunch| +46104|AAAAAAAAJBELAAAA|46104|12|48|24|PM|first|afternoon|lunch| +46105|AAAAAAAAKBELAAAA|46105|12|48|25|PM|first|afternoon|lunch| +46106|AAAAAAAALBELAAAA|46106|12|48|26|PM|first|afternoon|lunch| +46107|AAAAAAAAMBELAAAA|46107|12|48|27|PM|first|afternoon|lunch| +46108|AAAAAAAANBELAAAA|46108|12|48|28|PM|first|afternoon|lunch| +46109|AAAAAAAAOBELAAAA|46109|12|48|29|PM|first|afternoon|lunch| +46110|AAAAAAAAPBELAAAA|46110|12|48|30|PM|first|afternoon|lunch| +46111|AAAAAAAAACELAAAA|46111|12|48|31|PM|first|afternoon|lunch| +46112|AAAAAAAABCELAAAA|46112|12|48|32|PM|first|afternoon|lunch| +46113|AAAAAAAACCELAAAA|46113|12|48|33|PM|first|afternoon|lunch| +46114|AAAAAAAADCELAAAA|46114|12|48|34|PM|first|afternoon|lunch| +46115|AAAAAAAAECELAAAA|46115|12|48|35|PM|first|afternoon|lunch| +46116|AAAAAAAAFCELAAAA|46116|12|48|36|PM|first|afternoon|lunch| +46117|AAAAAAAAGCELAAAA|46117|12|48|37|PM|first|afternoon|lunch| +46118|AAAAAAAAHCELAAAA|46118|12|48|38|PM|first|afternoon|lunch| +46119|AAAAAAAAICELAAAA|46119|12|48|39|PM|first|afternoon|lunch| +46120|AAAAAAAAJCELAAAA|46120|12|48|40|PM|first|afternoon|lunch| +46121|AAAAAAAAKCELAAAA|46121|12|48|41|PM|first|afternoon|lunch| +46122|AAAAAAAALCELAAAA|46122|12|48|42|PM|first|afternoon|lunch| +46123|AAAAAAAAMCELAAAA|46123|12|48|43|PM|first|afternoon|lunch| +46124|AAAAAAAANCELAAAA|46124|12|48|44|PM|first|afternoon|lunch| +46125|AAAAAAAAOCELAAAA|46125|12|48|45|PM|first|afternoon|lunch| +46126|AAAAAAAAPCELAAAA|46126|12|48|46|PM|first|afternoon|lunch| +46127|AAAAAAAAADELAAAA|46127|12|48|47|PM|first|afternoon|lunch| +46128|AAAAAAAABDELAAAA|46128|12|48|48|PM|first|afternoon|lunch| +46129|AAAAAAAACDELAAAA|46129|12|48|49|PM|first|afternoon|lunch| +46130|AAAAAAAADDELAAAA|46130|12|48|50|PM|first|afternoon|lunch| +46131|AAAAAAAAEDELAAAA|46131|12|48|51|PM|first|afternoon|lunch| +46132|AAAAAAAAFDELAAAA|46132|12|48|52|PM|first|afternoon|lunch| +46133|AAAAAAAAGDELAAAA|46133|12|48|53|PM|first|afternoon|lunch| +46134|AAAAAAAAHDELAAAA|46134|12|48|54|PM|first|afternoon|lunch| +46135|AAAAAAAAIDELAAAA|46135|12|48|55|PM|first|afternoon|lunch| +46136|AAAAAAAAJDELAAAA|46136|12|48|56|PM|first|afternoon|lunch| +46137|AAAAAAAAKDELAAAA|46137|12|48|57|PM|first|afternoon|lunch| +46138|AAAAAAAALDELAAAA|46138|12|48|58|PM|first|afternoon|lunch| +46139|AAAAAAAAMDELAAAA|46139|12|48|59|PM|first|afternoon|lunch| +46140|AAAAAAAANDELAAAA|46140|12|49|0|PM|first|afternoon|lunch| +46141|AAAAAAAAODELAAAA|46141|12|49|1|PM|first|afternoon|lunch| +46142|AAAAAAAAPDELAAAA|46142|12|49|2|PM|first|afternoon|lunch| +46143|AAAAAAAAAEELAAAA|46143|12|49|3|PM|first|afternoon|lunch| +46144|AAAAAAAABEELAAAA|46144|12|49|4|PM|first|afternoon|lunch| +46145|AAAAAAAACEELAAAA|46145|12|49|5|PM|first|afternoon|lunch| +46146|AAAAAAAADEELAAAA|46146|12|49|6|PM|first|afternoon|lunch| +46147|AAAAAAAAEEELAAAA|46147|12|49|7|PM|first|afternoon|lunch| +46148|AAAAAAAAFEELAAAA|46148|12|49|8|PM|first|afternoon|lunch| +46149|AAAAAAAAGEELAAAA|46149|12|49|9|PM|first|afternoon|lunch| +46150|AAAAAAAAHEELAAAA|46150|12|49|10|PM|first|afternoon|lunch| +46151|AAAAAAAAIEELAAAA|46151|12|49|11|PM|first|afternoon|lunch| +46152|AAAAAAAAJEELAAAA|46152|12|49|12|PM|first|afternoon|lunch| +46153|AAAAAAAAKEELAAAA|46153|12|49|13|PM|first|afternoon|lunch| +46154|AAAAAAAALEELAAAA|46154|12|49|14|PM|first|afternoon|lunch| +46155|AAAAAAAAMEELAAAA|46155|12|49|15|PM|first|afternoon|lunch| +46156|AAAAAAAANEELAAAA|46156|12|49|16|PM|first|afternoon|lunch| +46157|AAAAAAAAOEELAAAA|46157|12|49|17|PM|first|afternoon|lunch| +46158|AAAAAAAAPEELAAAA|46158|12|49|18|PM|first|afternoon|lunch| +46159|AAAAAAAAAFELAAAA|46159|12|49|19|PM|first|afternoon|lunch| +46160|AAAAAAAABFELAAAA|46160|12|49|20|PM|first|afternoon|lunch| +46161|AAAAAAAACFELAAAA|46161|12|49|21|PM|first|afternoon|lunch| +46162|AAAAAAAADFELAAAA|46162|12|49|22|PM|first|afternoon|lunch| +46163|AAAAAAAAEFELAAAA|46163|12|49|23|PM|first|afternoon|lunch| +46164|AAAAAAAAFFELAAAA|46164|12|49|24|PM|first|afternoon|lunch| +46165|AAAAAAAAGFELAAAA|46165|12|49|25|PM|first|afternoon|lunch| +46166|AAAAAAAAHFELAAAA|46166|12|49|26|PM|first|afternoon|lunch| +46167|AAAAAAAAIFELAAAA|46167|12|49|27|PM|first|afternoon|lunch| +46168|AAAAAAAAJFELAAAA|46168|12|49|28|PM|first|afternoon|lunch| +46169|AAAAAAAAKFELAAAA|46169|12|49|29|PM|first|afternoon|lunch| +46170|AAAAAAAALFELAAAA|46170|12|49|30|PM|first|afternoon|lunch| +46171|AAAAAAAAMFELAAAA|46171|12|49|31|PM|first|afternoon|lunch| +46172|AAAAAAAANFELAAAA|46172|12|49|32|PM|first|afternoon|lunch| +46173|AAAAAAAAOFELAAAA|46173|12|49|33|PM|first|afternoon|lunch| +46174|AAAAAAAAPFELAAAA|46174|12|49|34|PM|first|afternoon|lunch| +46175|AAAAAAAAAGELAAAA|46175|12|49|35|PM|first|afternoon|lunch| +46176|AAAAAAAABGELAAAA|46176|12|49|36|PM|first|afternoon|lunch| +46177|AAAAAAAACGELAAAA|46177|12|49|37|PM|first|afternoon|lunch| +46178|AAAAAAAADGELAAAA|46178|12|49|38|PM|first|afternoon|lunch| +46179|AAAAAAAAEGELAAAA|46179|12|49|39|PM|first|afternoon|lunch| +46180|AAAAAAAAFGELAAAA|46180|12|49|40|PM|first|afternoon|lunch| +46181|AAAAAAAAGGELAAAA|46181|12|49|41|PM|first|afternoon|lunch| +46182|AAAAAAAAHGELAAAA|46182|12|49|42|PM|first|afternoon|lunch| +46183|AAAAAAAAIGELAAAA|46183|12|49|43|PM|first|afternoon|lunch| +46184|AAAAAAAAJGELAAAA|46184|12|49|44|PM|first|afternoon|lunch| +46185|AAAAAAAAKGELAAAA|46185|12|49|45|PM|first|afternoon|lunch| +46186|AAAAAAAALGELAAAA|46186|12|49|46|PM|first|afternoon|lunch| +46187|AAAAAAAAMGELAAAA|46187|12|49|47|PM|first|afternoon|lunch| +46188|AAAAAAAANGELAAAA|46188|12|49|48|PM|first|afternoon|lunch| +46189|AAAAAAAAOGELAAAA|46189|12|49|49|PM|first|afternoon|lunch| +46190|AAAAAAAAPGELAAAA|46190|12|49|50|PM|first|afternoon|lunch| +46191|AAAAAAAAAHELAAAA|46191|12|49|51|PM|first|afternoon|lunch| +46192|AAAAAAAABHELAAAA|46192|12|49|52|PM|first|afternoon|lunch| +46193|AAAAAAAACHELAAAA|46193|12|49|53|PM|first|afternoon|lunch| +46194|AAAAAAAADHELAAAA|46194|12|49|54|PM|first|afternoon|lunch| +46195|AAAAAAAAEHELAAAA|46195|12|49|55|PM|first|afternoon|lunch| +46196|AAAAAAAAFHELAAAA|46196|12|49|56|PM|first|afternoon|lunch| +46197|AAAAAAAAGHELAAAA|46197|12|49|57|PM|first|afternoon|lunch| +46198|AAAAAAAAHHELAAAA|46198|12|49|58|PM|first|afternoon|lunch| +46199|AAAAAAAAIHELAAAA|46199|12|49|59|PM|first|afternoon|lunch| +46200|AAAAAAAAJHELAAAA|46200|12|50|0|PM|first|afternoon|lunch| +46201|AAAAAAAAKHELAAAA|46201|12|50|1|PM|first|afternoon|lunch| +46202|AAAAAAAALHELAAAA|46202|12|50|2|PM|first|afternoon|lunch| +46203|AAAAAAAAMHELAAAA|46203|12|50|3|PM|first|afternoon|lunch| +46204|AAAAAAAANHELAAAA|46204|12|50|4|PM|first|afternoon|lunch| +46205|AAAAAAAAOHELAAAA|46205|12|50|5|PM|first|afternoon|lunch| +46206|AAAAAAAAPHELAAAA|46206|12|50|6|PM|first|afternoon|lunch| +46207|AAAAAAAAAIELAAAA|46207|12|50|7|PM|first|afternoon|lunch| +46208|AAAAAAAABIELAAAA|46208|12|50|8|PM|first|afternoon|lunch| +46209|AAAAAAAACIELAAAA|46209|12|50|9|PM|first|afternoon|lunch| +46210|AAAAAAAADIELAAAA|46210|12|50|10|PM|first|afternoon|lunch| +46211|AAAAAAAAEIELAAAA|46211|12|50|11|PM|first|afternoon|lunch| +46212|AAAAAAAAFIELAAAA|46212|12|50|12|PM|first|afternoon|lunch| +46213|AAAAAAAAGIELAAAA|46213|12|50|13|PM|first|afternoon|lunch| +46214|AAAAAAAAHIELAAAA|46214|12|50|14|PM|first|afternoon|lunch| +46215|AAAAAAAAIIELAAAA|46215|12|50|15|PM|first|afternoon|lunch| +46216|AAAAAAAAJIELAAAA|46216|12|50|16|PM|first|afternoon|lunch| +46217|AAAAAAAAKIELAAAA|46217|12|50|17|PM|first|afternoon|lunch| +46218|AAAAAAAALIELAAAA|46218|12|50|18|PM|first|afternoon|lunch| +46219|AAAAAAAAMIELAAAA|46219|12|50|19|PM|first|afternoon|lunch| +46220|AAAAAAAANIELAAAA|46220|12|50|20|PM|first|afternoon|lunch| +46221|AAAAAAAAOIELAAAA|46221|12|50|21|PM|first|afternoon|lunch| +46222|AAAAAAAAPIELAAAA|46222|12|50|22|PM|first|afternoon|lunch| +46223|AAAAAAAAAJELAAAA|46223|12|50|23|PM|first|afternoon|lunch| +46224|AAAAAAAABJELAAAA|46224|12|50|24|PM|first|afternoon|lunch| +46225|AAAAAAAACJELAAAA|46225|12|50|25|PM|first|afternoon|lunch| +46226|AAAAAAAADJELAAAA|46226|12|50|26|PM|first|afternoon|lunch| +46227|AAAAAAAAEJELAAAA|46227|12|50|27|PM|first|afternoon|lunch| +46228|AAAAAAAAFJELAAAA|46228|12|50|28|PM|first|afternoon|lunch| +46229|AAAAAAAAGJELAAAA|46229|12|50|29|PM|first|afternoon|lunch| +46230|AAAAAAAAHJELAAAA|46230|12|50|30|PM|first|afternoon|lunch| +46231|AAAAAAAAIJELAAAA|46231|12|50|31|PM|first|afternoon|lunch| +46232|AAAAAAAAJJELAAAA|46232|12|50|32|PM|first|afternoon|lunch| +46233|AAAAAAAAKJELAAAA|46233|12|50|33|PM|first|afternoon|lunch| +46234|AAAAAAAALJELAAAA|46234|12|50|34|PM|first|afternoon|lunch| +46235|AAAAAAAAMJELAAAA|46235|12|50|35|PM|first|afternoon|lunch| +46236|AAAAAAAANJELAAAA|46236|12|50|36|PM|first|afternoon|lunch| +46237|AAAAAAAAOJELAAAA|46237|12|50|37|PM|first|afternoon|lunch| +46238|AAAAAAAAPJELAAAA|46238|12|50|38|PM|first|afternoon|lunch| +46239|AAAAAAAAAKELAAAA|46239|12|50|39|PM|first|afternoon|lunch| +46240|AAAAAAAABKELAAAA|46240|12|50|40|PM|first|afternoon|lunch| +46241|AAAAAAAACKELAAAA|46241|12|50|41|PM|first|afternoon|lunch| +46242|AAAAAAAADKELAAAA|46242|12|50|42|PM|first|afternoon|lunch| +46243|AAAAAAAAEKELAAAA|46243|12|50|43|PM|first|afternoon|lunch| +46244|AAAAAAAAFKELAAAA|46244|12|50|44|PM|first|afternoon|lunch| +46245|AAAAAAAAGKELAAAA|46245|12|50|45|PM|first|afternoon|lunch| +46246|AAAAAAAAHKELAAAA|46246|12|50|46|PM|first|afternoon|lunch| +46247|AAAAAAAAIKELAAAA|46247|12|50|47|PM|first|afternoon|lunch| +46248|AAAAAAAAJKELAAAA|46248|12|50|48|PM|first|afternoon|lunch| +46249|AAAAAAAAKKELAAAA|46249|12|50|49|PM|first|afternoon|lunch| +46250|AAAAAAAALKELAAAA|46250|12|50|50|PM|first|afternoon|lunch| +46251|AAAAAAAAMKELAAAA|46251|12|50|51|PM|first|afternoon|lunch| +46252|AAAAAAAANKELAAAA|46252|12|50|52|PM|first|afternoon|lunch| +46253|AAAAAAAAOKELAAAA|46253|12|50|53|PM|first|afternoon|lunch| +46254|AAAAAAAAPKELAAAA|46254|12|50|54|PM|first|afternoon|lunch| +46255|AAAAAAAAALELAAAA|46255|12|50|55|PM|first|afternoon|lunch| +46256|AAAAAAAABLELAAAA|46256|12|50|56|PM|first|afternoon|lunch| +46257|AAAAAAAACLELAAAA|46257|12|50|57|PM|first|afternoon|lunch| +46258|AAAAAAAADLELAAAA|46258|12|50|58|PM|first|afternoon|lunch| +46259|AAAAAAAAELELAAAA|46259|12|50|59|PM|first|afternoon|lunch| +46260|AAAAAAAAFLELAAAA|46260|12|51|0|PM|first|afternoon|lunch| +46261|AAAAAAAAGLELAAAA|46261|12|51|1|PM|first|afternoon|lunch| +46262|AAAAAAAAHLELAAAA|46262|12|51|2|PM|first|afternoon|lunch| +46263|AAAAAAAAILELAAAA|46263|12|51|3|PM|first|afternoon|lunch| +46264|AAAAAAAAJLELAAAA|46264|12|51|4|PM|first|afternoon|lunch| +46265|AAAAAAAAKLELAAAA|46265|12|51|5|PM|first|afternoon|lunch| +46266|AAAAAAAALLELAAAA|46266|12|51|6|PM|first|afternoon|lunch| +46267|AAAAAAAAMLELAAAA|46267|12|51|7|PM|first|afternoon|lunch| +46268|AAAAAAAANLELAAAA|46268|12|51|8|PM|first|afternoon|lunch| +46269|AAAAAAAAOLELAAAA|46269|12|51|9|PM|first|afternoon|lunch| +46270|AAAAAAAAPLELAAAA|46270|12|51|10|PM|first|afternoon|lunch| +46271|AAAAAAAAAMELAAAA|46271|12|51|11|PM|first|afternoon|lunch| +46272|AAAAAAAABMELAAAA|46272|12|51|12|PM|first|afternoon|lunch| +46273|AAAAAAAACMELAAAA|46273|12|51|13|PM|first|afternoon|lunch| +46274|AAAAAAAADMELAAAA|46274|12|51|14|PM|first|afternoon|lunch| +46275|AAAAAAAAEMELAAAA|46275|12|51|15|PM|first|afternoon|lunch| +46276|AAAAAAAAFMELAAAA|46276|12|51|16|PM|first|afternoon|lunch| +46277|AAAAAAAAGMELAAAA|46277|12|51|17|PM|first|afternoon|lunch| +46278|AAAAAAAAHMELAAAA|46278|12|51|18|PM|first|afternoon|lunch| +46279|AAAAAAAAIMELAAAA|46279|12|51|19|PM|first|afternoon|lunch| +46280|AAAAAAAAJMELAAAA|46280|12|51|20|PM|first|afternoon|lunch| +46281|AAAAAAAAKMELAAAA|46281|12|51|21|PM|first|afternoon|lunch| +46282|AAAAAAAALMELAAAA|46282|12|51|22|PM|first|afternoon|lunch| +46283|AAAAAAAAMMELAAAA|46283|12|51|23|PM|first|afternoon|lunch| +46284|AAAAAAAANMELAAAA|46284|12|51|24|PM|first|afternoon|lunch| +46285|AAAAAAAAOMELAAAA|46285|12|51|25|PM|first|afternoon|lunch| +46286|AAAAAAAAPMELAAAA|46286|12|51|26|PM|first|afternoon|lunch| +46287|AAAAAAAAANELAAAA|46287|12|51|27|PM|first|afternoon|lunch| +46288|AAAAAAAABNELAAAA|46288|12|51|28|PM|first|afternoon|lunch| +46289|AAAAAAAACNELAAAA|46289|12|51|29|PM|first|afternoon|lunch| +46290|AAAAAAAADNELAAAA|46290|12|51|30|PM|first|afternoon|lunch| +46291|AAAAAAAAENELAAAA|46291|12|51|31|PM|first|afternoon|lunch| +46292|AAAAAAAAFNELAAAA|46292|12|51|32|PM|first|afternoon|lunch| +46293|AAAAAAAAGNELAAAA|46293|12|51|33|PM|first|afternoon|lunch| +46294|AAAAAAAAHNELAAAA|46294|12|51|34|PM|first|afternoon|lunch| +46295|AAAAAAAAINELAAAA|46295|12|51|35|PM|first|afternoon|lunch| +46296|AAAAAAAAJNELAAAA|46296|12|51|36|PM|first|afternoon|lunch| +46297|AAAAAAAAKNELAAAA|46297|12|51|37|PM|first|afternoon|lunch| +46298|AAAAAAAALNELAAAA|46298|12|51|38|PM|first|afternoon|lunch| +46299|AAAAAAAAMNELAAAA|46299|12|51|39|PM|first|afternoon|lunch| +46300|AAAAAAAANNELAAAA|46300|12|51|40|PM|first|afternoon|lunch| +46301|AAAAAAAAONELAAAA|46301|12|51|41|PM|first|afternoon|lunch| +46302|AAAAAAAAPNELAAAA|46302|12|51|42|PM|first|afternoon|lunch| +46303|AAAAAAAAAOELAAAA|46303|12|51|43|PM|first|afternoon|lunch| +46304|AAAAAAAABOELAAAA|46304|12|51|44|PM|first|afternoon|lunch| +46305|AAAAAAAACOELAAAA|46305|12|51|45|PM|first|afternoon|lunch| +46306|AAAAAAAADOELAAAA|46306|12|51|46|PM|first|afternoon|lunch| +46307|AAAAAAAAEOELAAAA|46307|12|51|47|PM|first|afternoon|lunch| +46308|AAAAAAAAFOELAAAA|46308|12|51|48|PM|first|afternoon|lunch| +46309|AAAAAAAAGOELAAAA|46309|12|51|49|PM|first|afternoon|lunch| +46310|AAAAAAAAHOELAAAA|46310|12|51|50|PM|first|afternoon|lunch| +46311|AAAAAAAAIOELAAAA|46311|12|51|51|PM|first|afternoon|lunch| +46312|AAAAAAAAJOELAAAA|46312|12|51|52|PM|first|afternoon|lunch| +46313|AAAAAAAAKOELAAAA|46313|12|51|53|PM|first|afternoon|lunch| +46314|AAAAAAAALOELAAAA|46314|12|51|54|PM|first|afternoon|lunch| +46315|AAAAAAAAMOELAAAA|46315|12|51|55|PM|first|afternoon|lunch| +46316|AAAAAAAANOELAAAA|46316|12|51|56|PM|first|afternoon|lunch| +46317|AAAAAAAAOOELAAAA|46317|12|51|57|PM|first|afternoon|lunch| +46318|AAAAAAAAPOELAAAA|46318|12|51|58|PM|first|afternoon|lunch| +46319|AAAAAAAAAPELAAAA|46319|12|51|59|PM|first|afternoon|lunch| +46320|AAAAAAAABPELAAAA|46320|12|52|0|PM|first|afternoon|lunch| +46321|AAAAAAAACPELAAAA|46321|12|52|1|PM|first|afternoon|lunch| +46322|AAAAAAAADPELAAAA|46322|12|52|2|PM|first|afternoon|lunch| +46323|AAAAAAAAEPELAAAA|46323|12|52|3|PM|first|afternoon|lunch| +46324|AAAAAAAAFPELAAAA|46324|12|52|4|PM|first|afternoon|lunch| +46325|AAAAAAAAGPELAAAA|46325|12|52|5|PM|first|afternoon|lunch| +46326|AAAAAAAAHPELAAAA|46326|12|52|6|PM|first|afternoon|lunch| +46327|AAAAAAAAIPELAAAA|46327|12|52|7|PM|first|afternoon|lunch| +46328|AAAAAAAAJPELAAAA|46328|12|52|8|PM|first|afternoon|lunch| +46329|AAAAAAAAKPELAAAA|46329|12|52|9|PM|first|afternoon|lunch| +46330|AAAAAAAALPELAAAA|46330|12|52|10|PM|first|afternoon|lunch| +46331|AAAAAAAAMPELAAAA|46331|12|52|11|PM|first|afternoon|lunch| +46332|AAAAAAAANPELAAAA|46332|12|52|12|PM|first|afternoon|lunch| +46333|AAAAAAAAOPELAAAA|46333|12|52|13|PM|first|afternoon|lunch| +46334|AAAAAAAAPPELAAAA|46334|12|52|14|PM|first|afternoon|lunch| +46335|AAAAAAAAAAFLAAAA|46335|12|52|15|PM|first|afternoon|lunch| +46336|AAAAAAAABAFLAAAA|46336|12|52|16|PM|first|afternoon|lunch| +46337|AAAAAAAACAFLAAAA|46337|12|52|17|PM|first|afternoon|lunch| +46338|AAAAAAAADAFLAAAA|46338|12|52|18|PM|first|afternoon|lunch| +46339|AAAAAAAAEAFLAAAA|46339|12|52|19|PM|first|afternoon|lunch| +46340|AAAAAAAAFAFLAAAA|46340|12|52|20|PM|first|afternoon|lunch| +46341|AAAAAAAAGAFLAAAA|46341|12|52|21|PM|first|afternoon|lunch| +46342|AAAAAAAAHAFLAAAA|46342|12|52|22|PM|first|afternoon|lunch| +46343|AAAAAAAAIAFLAAAA|46343|12|52|23|PM|first|afternoon|lunch| +46344|AAAAAAAAJAFLAAAA|46344|12|52|24|PM|first|afternoon|lunch| +46345|AAAAAAAAKAFLAAAA|46345|12|52|25|PM|first|afternoon|lunch| +46346|AAAAAAAALAFLAAAA|46346|12|52|26|PM|first|afternoon|lunch| +46347|AAAAAAAAMAFLAAAA|46347|12|52|27|PM|first|afternoon|lunch| +46348|AAAAAAAANAFLAAAA|46348|12|52|28|PM|first|afternoon|lunch| +46349|AAAAAAAAOAFLAAAA|46349|12|52|29|PM|first|afternoon|lunch| +46350|AAAAAAAAPAFLAAAA|46350|12|52|30|PM|first|afternoon|lunch| +46351|AAAAAAAAABFLAAAA|46351|12|52|31|PM|first|afternoon|lunch| +46352|AAAAAAAABBFLAAAA|46352|12|52|32|PM|first|afternoon|lunch| +46353|AAAAAAAACBFLAAAA|46353|12|52|33|PM|first|afternoon|lunch| +46354|AAAAAAAADBFLAAAA|46354|12|52|34|PM|first|afternoon|lunch| +46355|AAAAAAAAEBFLAAAA|46355|12|52|35|PM|first|afternoon|lunch| +46356|AAAAAAAAFBFLAAAA|46356|12|52|36|PM|first|afternoon|lunch| +46357|AAAAAAAAGBFLAAAA|46357|12|52|37|PM|first|afternoon|lunch| +46358|AAAAAAAAHBFLAAAA|46358|12|52|38|PM|first|afternoon|lunch| +46359|AAAAAAAAIBFLAAAA|46359|12|52|39|PM|first|afternoon|lunch| +46360|AAAAAAAAJBFLAAAA|46360|12|52|40|PM|first|afternoon|lunch| +46361|AAAAAAAAKBFLAAAA|46361|12|52|41|PM|first|afternoon|lunch| +46362|AAAAAAAALBFLAAAA|46362|12|52|42|PM|first|afternoon|lunch| +46363|AAAAAAAAMBFLAAAA|46363|12|52|43|PM|first|afternoon|lunch| +46364|AAAAAAAANBFLAAAA|46364|12|52|44|PM|first|afternoon|lunch| +46365|AAAAAAAAOBFLAAAA|46365|12|52|45|PM|first|afternoon|lunch| +46366|AAAAAAAAPBFLAAAA|46366|12|52|46|PM|first|afternoon|lunch| +46367|AAAAAAAAACFLAAAA|46367|12|52|47|PM|first|afternoon|lunch| +46368|AAAAAAAABCFLAAAA|46368|12|52|48|PM|first|afternoon|lunch| +46369|AAAAAAAACCFLAAAA|46369|12|52|49|PM|first|afternoon|lunch| +46370|AAAAAAAADCFLAAAA|46370|12|52|50|PM|first|afternoon|lunch| +46371|AAAAAAAAECFLAAAA|46371|12|52|51|PM|first|afternoon|lunch| +46372|AAAAAAAAFCFLAAAA|46372|12|52|52|PM|first|afternoon|lunch| +46373|AAAAAAAAGCFLAAAA|46373|12|52|53|PM|first|afternoon|lunch| +46374|AAAAAAAAHCFLAAAA|46374|12|52|54|PM|first|afternoon|lunch| +46375|AAAAAAAAICFLAAAA|46375|12|52|55|PM|first|afternoon|lunch| +46376|AAAAAAAAJCFLAAAA|46376|12|52|56|PM|first|afternoon|lunch| +46377|AAAAAAAAKCFLAAAA|46377|12|52|57|PM|first|afternoon|lunch| +46378|AAAAAAAALCFLAAAA|46378|12|52|58|PM|first|afternoon|lunch| +46379|AAAAAAAAMCFLAAAA|46379|12|52|59|PM|first|afternoon|lunch| +46380|AAAAAAAANCFLAAAA|46380|12|53|0|PM|first|afternoon|lunch| +46381|AAAAAAAAOCFLAAAA|46381|12|53|1|PM|first|afternoon|lunch| +46382|AAAAAAAAPCFLAAAA|46382|12|53|2|PM|first|afternoon|lunch| +46383|AAAAAAAAADFLAAAA|46383|12|53|3|PM|first|afternoon|lunch| +46384|AAAAAAAABDFLAAAA|46384|12|53|4|PM|first|afternoon|lunch| +46385|AAAAAAAACDFLAAAA|46385|12|53|5|PM|first|afternoon|lunch| +46386|AAAAAAAADDFLAAAA|46386|12|53|6|PM|first|afternoon|lunch| +46387|AAAAAAAAEDFLAAAA|46387|12|53|7|PM|first|afternoon|lunch| +46388|AAAAAAAAFDFLAAAA|46388|12|53|8|PM|first|afternoon|lunch| +46389|AAAAAAAAGDFLAAAA|46389|12|53|9|PM|first|afternoon|lunch| +46390|AAAAAAAAHDFLAAAA|46390|12|53|10|PM|first|afternoon|lunch| +46391|AAAAAAAAIDFLAAAA|46391|12|53|11|PM|first|afternoon|lunch| +46392|AAAAAAAAJDFLAAAA|46392|12|53|12|PM|first|afternoon|lunch| +46393|AAAAAAAAKDFLAAAA|46393|12|53|13|PM|first|afternoon|lunch| +46394|AAAAAAAALDFLAAAA|46394|12|53|14|PM|first|afternoon|lunch| +46395|AAAAAAAAMDFLAAAA|46395|12|53|15|PM|first|afternoon|lunch| +46396|AAAAAAAANDFLAAAA|46396|12|53|16|PM|first|afternoon|lunch| +46397|AAAAAAAAODFLAAAA|46397|12|53|17|PM|first|afternoon|lunch| +46398|AAAAAAAAPDFLAAAA|46398|12|53|18|PM|first|afternoon|lunch| +46399|AAAAAAAAAEFLAAAA|46399|12|53|19|PM|first|afternoon|lunch| +46400|AAAAAAAABEFLAAAA|46400|12|53|20|PM|first|afternoon|lunch| +46401|AAAAAAAACEFLAAAA|46401|12|53|21|PM|first|afternoon|lunch| +46402|AAAAAAAADEFLAAAA|46402|12|53|22|PM|first|afternoon|lunch| +46403|AAAAAAAAEEFLAAAA|46403|12|53|23|PM|first|afternoon|lunch| +46404|AAAAAAAAFEFLAAAA|46404|12|53|24|PM|first|afternoon|lunch| +46405|AAAAAAAAGEFLAAAA|46405|12|53|25|PM|first|afternoon|lunch| +46406|AAAAAAAAHEFLAAAA|46406|12|53|26|PM|first|afternoon|lunch| +46407|AAAAAAAAIEFLAAAA|46407|12|53|27|PM|first|afternoon|lunch| +46408|AAAAAAAAJEFLAAAA|46408|12|53|28|PM|first|afternoon|lunch| +46409|AAAAAAAAKEFLAAAA|46409|12|53|29|PM|first|afternoon|lunch| +46410|AAAAAAAALEFLAAAA|46410|12|53|30|PM|first|afternoon|lunch| +46411|AAAAAAAAMEFLAAAA|46411|12|53|31|PM|first|afternoon|lunch| +46412|AAAAAAAANEFLAAAA|46412|12|53|32|PM|first|afternoon|lunch| +46413|AAAAAAAAOEFLAAAA|46413|12|53|33|PM|first|afternoon|lunch| +46414|AAAAAAAAPEFLAAAA|46414|12|53|34|PM|first|afternoon|lunch| +46415|AAAAAAAAAFFLAAAA|46415|12|53|35|PM|first|afternoon|lunch| +46416|AAAAAAAABFFLAAAA|46416|12|53|36|PM|first|afternoon|lunch| +46417|AAAAAAAACFFLAAAA|46417|12|53|37|PM|first|afternoon|lunch| +46418|AAAAAAAADFFLAAAA|46418|12|53|38|PM|first|afternoon|lunch| +46419|AAAAAAAAEFFLAAAA|46419|12|53|39|PM|first|afternoon|lunch| +46420|AAAAAAAAFFFLAAAA|46420|12|53|40|PM|first|afternoon|lunch| +46421|AAAAAAAAGFFLAAAA|46421|12|53|41|PM|first|afternoon|lunch| +46422|AAAAAAAAHFFLAAAA|46422|12|53|42|PM|first|afternoon|lunch| +46423|AAAAAAAAIFFLAAAA|46423|12|53|43|PM|first|afternoon|lunch| +46424|AAAAAAAAJFFLAAAA|46424|12|53|44|PM|first|afternoon|lunch| +46425|AAAAAAAAKFFLAAAA|46425|12|53|45|PM|first|afternoon|lunch| +46426|AAAAAAAALFFLAAAA|46426|12|53|46|PM|first|afternoon|lunch| +46427|AAAAAAAAMFFLAAAA|46427|12|53|47|PM|first|afternoon|lunch| +46428|AAAAAAAANFFLAAAA|46428|12|53|48|PM|first|afternoon|lunch| +46429|AAAAAAAAOFFLAAAA|46429|12|53|49|PM|first|afternoon|lunch| +46430|AAAAAAAAPFFLAAAA|46430|12|53|50|PM|first|afternoon|lunch| +46431|AAAAAAAAAGFLAAAA|46431|12|53|51|PM|first|afternoon|lunch| +46432|AAAAAAAABGFLAAAA|46432|12|53|52|PM|first|afternoon|lunch| +46433|AAAAAAAACGFLAAAA|46433|12|53|53|PM|first|afternoon|lunch| +46434|AAAAAAAADGFLAAAA|46434|12|53|54|PM|first|afternoon|lunch| +46435|AAAAAAAAEGFLAAAA|46435|12|53|55|PM|first|afternoon|lunch| +46436|AAAAAAAAFGFLAAAA|46436|12|53|56|PM|first|afternoon|lunch| +46437|AAAAAAAAGGFLAAAA|46437|12|53|57|PM|first|afternoon|lunch| +46438|AAAAAAAAHGFLAAAA|46438|12|53|58|PM|first|afternoon|lunch| +46439|AAAAAAAAIGFLAAAA|46439|12|53|59|PM|first|afternoon|lunch| +46440|AAAAAAAAJGFLAAAA|46440|12|54|0|PM|first|afternoon|lunch| +46441|AAAAAAAAKGFLAAAA|46441|12|54|1|PM|first|afternoon|lunch| +46442|AAAAAAAALGFLAAAA|46442|12|54|2|PM|first|afternoon|lunch| +46443|AAAAAAAAMGFLAAAA|46443|12|54|3|PM|first|afternoon|lunch| +46444|AAAAAAAANGFLAAAA|46444|12|54|4|PM|first|afternoon|lunch| +46445|AAAAAAAAOGFLAAAA|46445|12|54|5|PM|first|afternoon|lunch| +46446|AAAAAAAAPGFLAAAA|46446|12|54|6|PM|first|afternoon|lunch| +46447|AAAAAAAAAHFLAAAA|46447|12|54|7|PM|first|afternoon|lunch| +46448|AAAAAAAABHFLAAAA|46448|12|54|8|PM|first|afternoon|lunch| +46449|AAAAAAAACHFLAAAA|46449|12|54|9|PM|first|afternoon|lunch| +46450|AAAAAAAADHFLAAAA|46450|12|54|10|PM|first|afternoon|lunch| +46451|AAAAAAAAEHFLAAAA|46451|12|54|11|PM|first|afternoon|lunch| +46452|AAAAAAAAFHFLAAAA|46452|12|54|12|PM|first|afternoon|lunch| +46453|AAAAAAAAGHFLAAAA|46453|12|54|13|PM|first|afternoon|lunch| +46454|AAAAAAAAHHFLAAAA|46454|12|54|14|PM|first|afternoon|lunch| +46455|AAAAAAAAIHFLAAAA|46455|12|54|15|PM|first|afternoon|lunch| +46456|AAAAAAAAJHFLAAAA|46456|12|54|16|PM|first|afternoon|lunch| +46457|AAAAAAAAKHFLAAAA|46457|12|54|17|PM|first|afternoon|lunch| +46458|AAAAAAAALHFLAAAA|46458|12|54|18|PM|first|afternoon|lunch| +46459|AAAAAAAAMHFLAAAA|46459|12|54|19|PM|first|afternoon|lunch| +46460|AAAAAAAANHFLAAAA|46460|12|54|20|PM|first|afternoon|lunch| +46461|AAAAAAAAOHFLAAAA|46461|12|54|21|PM|first|afternoon|lunch| +46462|AAAAAAAAPHFLAAAA|46462|12|54|22|PM|first|afternoon|lunch| +46463|AAAAAAAAAIFLAAAA|46463|12|54|23|PM|first|afternoon|lunch| +46464|AAAAAAAABIFLAAAA|46464|12|54|24|PM|first|afternoon|lunch| +46465|AAAAAAAACIFLAAAA|46465|12|54|25|PM|first|afternoon|lunch| +46466|AAAAAAAADIFLAAAA|46466|12|54|26|PM|first|afternoon|lunch| +46467|AAAAAAAAEIFLAAAA|46467|12|54|27|PM|first|afternoon|lunch| +46468|AAAAAAAAFIFLAAAA|46468|12|54|28|PM|first|afternoon|lunch| +46469|AAAAAAAAGIFLAAAA|46469|12|54|29|PM|first|afternoon|lunch| +46470|AAAAAAAAHIFLAAAA|46470|12|54|30|PM|first|afternoon|lunch| +46471|AAAAAAAAIIFLAAAA|46471|12|54|31|PM|first|afternoon|lunch| +46472|AAAAAAAAJIFLAAAA|46472|12|54|32|PM|first|afternoon|lunch| +46473|AAAAAAAAKIFLAAAA|46473|12|54|33|PM|first|afternoon|lunch| +46474|AAAAAAAALIFLAAAA|46474|12|54|34|PM|first|afternoon|lunch| +46475|AAAAAAAAMIFLAAAA|46475|12|54|35|PM|first|afternoon|lunch| +46476|AAAAAAAANIFLAAAA|46476|12|54|36|PM|first|afternoon|lunch| +46477|AAAAAAAAOIFLAAAA|46477|12|54|37|PM|first|afternoon|lunch| +46478|AAAAAAAAPIFLAAAA|46478|12|54|38|PM|first|afternoon|lunch| +46479|AAAAAAAAAJFLAAAA|46479|12|54|39|PM|first|afternoon|lunch| +46480|AAAAAAAABJFLAAAA|46480|12|54|40|PM|first|afternoon|lunch| +46481|AAAAAAAACJFLAAAA|46481|12|54|41|PM|first|afternoon|lunch| +46482|AAAAAAAADJFLAAAA|46482|12|54|42|PM|first|afternoon|lunch| +46483|AAAAAAAAEJFLAAAA|46483|12|54|43|PM|first|afternoon|lunch| +46484|AAAAAAAAFJFLAAAA|46484|12|54|44|PM|first|afternoon|lunch| +46485|AAAAAAAAGJFLAAAA|46485|12|54|45|PM|first|afternoon|lunch| +46486|AAAAAAAAHJFLAAAA|46486|12|54|46|PM|first|afternoon|lunch| +46487|AAAAAAAAIJFLAAAA|46487|12|54|47|PM|first|afternoon|lunch| +46488|AAAAAAAAJJFLAAAA|46488|12|54|48|PM|first|afternoon|lunch| +46489|AAAAAAAAKJFLAAAA|46489|12|54|49|PM|first|afternoon|lunch| +46490|AAAAAAAALJFLAAAA|46490|12|54|50|PM|first|afternoon|lunch| +46491|AAAAAAAAMJFLAAAA|46491|12|54|51|PM|first|afternoon|lunch| +46492|AAAAAAAANJFLAAAA|46492|12|54|52|PM|first|afternoon|lunch| +46493|AAAAAAAAOJFLAAAA|46493|12|54|53|PM|first|afternoon|lunch| +46494|AAAAAAAAPJFLAAAA|46494|12|54|54|PM|first|afternoon|lunch| +46495|AAAAAAAAAKFLAAAA|46495|12|54|55|PM|first|afternoon|lunch| +46496|AAAAAAAABKFLAAAA|46496|12|54|56|PM|first|afternoon|lunch| +46497|AAAAAAAACKFLAAAA|46497|12|54|57|PM|first|afternoon|lunch| +46498|AAAAAAAADKFLAAAA|46498|12|54|58|PM|first|afternoon|lunch| +46499|AAAAAAAAEKFLAAAA|46499|12|54|59|PM|first|afternoon|lunch| +46500|AAAAAAAAFKFLAAAA|46500|12|55|0|PM|first|afternoon|lunch| +46501|AAAAAAAAGKFLAAAA|46501|12|55|1|PM|first|afternoon|lunch| +46502|AAAAAAAAHKFLAAAA|46502|12|55|2|PM|first|afternoon|lunch| +46503|AAAAAAAAIKFLAAAA|46503|12|55|3|PM|first|afternoon|lunch| +46504|AAAAAAAAJKFLAAAA|46504|12|55|4|PM|first|afternoon|lunch| +46505|AAAAAAAAKKFLAAAA|46505|12|55|5|PM|first|afternoon|lunch| +46506|AAAAAAAALKFLAAAA|46506|12|55|6|PM|first|afternoon|lunch| +46507|AAAAAAAAMKFLAAAA|46507|12|55|7|PM|first|afternoon|lunch| +46508|AAAAAAAANKFLAAAA|46508|12|55|8|PM|first|afternoon|lunch| +46509|AAAAAAAAOKFLAAAA|46509|12|55|9|PM|first|afternoon|lunch| +46510|AAAAAAAAPKFLAAAA|46510|12|55|10|PM|first|afternoon|lunch| +46511|AAAAAAAAALFLAAAA|46511|12|55|11|PM|first|afternoon|lunch| +46512|AAAAAAAABLFLAAAA|46512|12|55|12|PM|first|afternoon|lunch| +46513|AAAAAAAACLFLAAAA|46513|12|55|13|PM|first|afternoon|lunch| +46514|AAAAAAAADLFLAAAA|46514|12|55|14|PM|first|afternoon|lunch| +46515|AAAAAAAAELFLAAAA|46515|12|55|15|PM|first|afternoon|lunch| +46516|AAAAAAAAFLFLAAAA|46516|12|55|16|PM|first|afternoon|lunch| +46517|AAAAAAAAGLFLAAAA|46517|12|55|17|PM|first|afternoon|lunch| +46518|AAAAAAAAHLFLAAAA|46518|12|55|18|PM|first|afternoon|lunch| +46519|AAAAAAAAILFLAAAA|46519|12|55|19|PM|first|afternoon|lunch| +46520|AAAAAAAAJLFLAAAA|46520|12|55|20|PM|first|afternoon|lunch| +46521|AAAAAAAAKLFLAAAA|46521|12|55|21|PM|first|afternoon|lunch| +46522|AAAAAAAALLFLAAAA|46522|12|55|22|PM|first|afternoon|lunch| +46523|AAAAAAAAMLFLAAAA|46523|12|55|23|PM|first|afternoon|lunch| +46524|AAAAAAAANLFLAAAA|46524|12|55|24|PM|first|afternoon|lunch| +46525|AAAAAAAAOLFLAAAA|46525|12|55|25|PM|first|afternoon|lunch| +46526|AAAAAAAAPLFLAAAA|46526|12|55|26|PM|first|afternoon|lunch| +46527|AAAAAAAAAMFLAAAA|46527|12|55|27|PM|first|afternoon|lunch| +46528|AAAAAAAABMFLAAAA|46528|12|55|28|PM|first|afternoon|lunch| +46529|AAAAAAAACMFLAAAA|46529|12|55|29|PM|first|afternoon|lunch| +46530|AAAAAAAADMFLAAAA|46530|12|55|30|PM|first|afternoon|lunch| +46531|AAAAAAAAEMFLAAAA|46531|12|55|31|PM|first|afternoon|lunch| +46532|AAAAAAAAFMFLAAAA|46532|12|55|32|PM|first|afternoon|lunch| +46533|AAAAAAAAGMFLAAAA|46533|12|55|33|PM|first|afternoon|lunch| +46534|AAAAAAAAHMFLAAAA|46534|12|55|34|PM|first|afternoon|lunch| +46535|AAAAAAAAIMFLAAAA|46535|12|55|35|PM|first|afternoon|lunch| +46536|AAAAAAAAJMFLAAAA|46536|12|55|36|PM|first|afternoon|lunch| +46537|AAAAAAAAKMFLAAAA|46537|12|55|37|PM|first|afternoon|lunch| +46538|AAAAAAAALMFLAAAA|46538|12|55|38|PM|first|afternoon|lunch| +46539|AAAAAAAAMMFLAAAA|46539|12|55|39|PM|first|afternoon|lunch| +46540|AAAAAAAANMFLAAAA|46540|12|55|40|PM|first|afternoon|lunch| +46541|AAAAAAAAOMFLAAAA|46541|12|55|41|PM|first|afternoon|lunch| +46542|AAAAAAAAPMFLAAAA|46542|12|55|42|PM|first|afternoon|lunch| +46543|AAAAAAAAANFLAAAA|46543|12|55|43|PM|first|afternoon|lunch| +46544|AAAAAAAABNFLAAAA|46544|12|55|44|PM|first|afternoon|lunch| +46545|AAAAAAAACNFLAAAA|46545|12|55|45|PM|first|afternoon|lunch| +46546|AAAAAAAADNFLAAAA|46546|12|55|46|PM|first|afternoon|lunch| +46547|AAAAAAAAENFLAAAA|46547|12|55|47|PM|first|afternoon|lunch| +46548|AAAAAAAAFNFLAAAA|46548|12|55|48|PM|first|afternoon|lunch| +46549|AAAAAAAAGNFLAAAA|46549|12|55|49|PM|first|afternoon|lunch| +46550|AAAAAAAAHNFLAAAA|46550|12|55|50|PM|first|afternoon|lunch| +46551|AAAAAAAAINFLAAAA|46551|12|55|51|PM|first|afternoon|lunch| +46552|AAAAAAAAJNFLAAAA|46552|12|55|52|PM|first|afternoon|lunch| +46553|AAAAAAAAKNFLAAAA|46553|12|55|53|PM|first|afternoon|lunch| +46554|AAAAAAAALNFLAAAA|46554|12|55|54|PM|first|afternoon|lunch| +46555|AAAAAAAAMNFLAAAA|46555|12|55|55|PM|first|afternoon|lunch| +46556|AAAAAAAANNFLAAAA|46556|12|55|56|PM|first|afternoon|lunch| +46557|AAAAAAAAONFLAAAA|46557|12|55|57|PM|first|afternoon|lunch| +46558|AAAAAAAAPNFLAAAA|46558|12|55|58|PM|first|afternoon|lunch| +46559|AAAAAAAAAOFLAAAA|46559|12|55|59|PM|first|afternoon|lunch| +46560|AAAAAAAABOFLAAAA|46560|12|56|0|PM|first|afternoon|lunch| +46561|AAAAAAAACOFLAAAA|46561|12|56|1|PM|first|afternoon|lunch| +46562|AAAAAAAADOFLAAAA|46562|12|56|2|PM|first|afternoon|lunch| +46563|AAAAAAAAEOFLAAAA|46563|12|56|3|PM|first|afternoon|lunch| +46564|AAAAAAAAFOFLAAAA|46564|12|56|4|PM|first|afternoon|lunch| +46565|AAAAAAAAGOFLAAAA|46565|12|56|5|PM|first|afternoon|lunch| +46566|AAAAAAAAHOFLAAAA|46566|12|56|6|PM|first|afternoon|lunch| +46567|AAAAAAAAIOFLAAAA|46567|12|56|7|PM|first|afternoon|lunch| +46568|AAAAAAAAJOFLAAAA|46568|12|56|8|PM|first|afternoon|lunch| +46569|AAAAAAAAKOFLAAAA|46569|12|56|9|PM|first|afternoon|lunch| +46570|AAAAAAAALOFLAAAA|46570|12|56|10|PM|first|afternoon|lunch| +46571|AAAAAAAAMOFLAAAA|46571|12|56|11|PM|first|afternoon|lunch| +46572|AAAAAAAANOFLAAAA|46572|12|56|12|PM|first|afternoon|lunch| +46573|AAAAAAAAOOFLAAAA|46573|12|56|13|PM|first|afternoon|lunch| +46574|AAAAAAAAPOFLAAAA|46574|12|56|14|PM|first|afternoon|lunch| +46575|AAAAAAAAAPFLAAAA|46575|12|56|15|PM|first|afternoon|lunch| +46576|AAAAAAAABPFLAAAA|46576|12|56|16|PM|first|afternoon|lunch| +46577|AAAAAAAACPFLAAAA|46577|12|56|17|PM|first|afternoon|lunch| +46578|AAAAAAAADPFLAAAA|46578|12|56|18|PM|first|afternoon|lunch| +46579|AAAAAAAAEPFLAAAA|46579|12|56|19|PM|first|afternoon|lunch| +46580|AAAAAAAAFPFLAAAA|46580|12|56|20|PM|first|afternoon|lunch| +46581|AAAAAAAAGPFLAAAA|46581|12|56|21|PM|first|afternoon|lunch| +46582|AAAAAAAAHPFLAAAA|46582|12|56|22|PM|first|afternoon|lunch| +46583|AAAAAAAAIPFLAAAA|46583|12|56|23|PM|first|afternoon|lunch| +46584|AAAAAAAAJPFLAAAA|46584|12|56|24|PM|first|afternoon|lunch| +46585|AAAAAAAAKPFLAAAA|46585|12|56|25|PM|first|afternoon|lunch| +46586|AAAAAAAALPFLAAAA|46586|12|56|26|PM|first|afternoon|lunch| +46587|AAAAAAAAMPFLAAAA|46587|12|56|27|PM|first|afternoon|lunch| +46588|AAAAAAAANPFLAAAA|46588|12|56|28|PM|first|afternoon|lunch| +46589|AAAAAAAAOPFLAAAA|46589|12|56|29|PM|first|afternoon|lunch| +46590|AAAAAAAAPPFLAAAA|46590|12|56|30|PM|first|afternoon|lunch| +46591|AAAAAAAAAAGLAAAA|46591|12|56|31|PM|first|afternoon|lunch| +46592|AAAAAAAABAGLAAAA|46592|12|56|32|PM|first|afternoon|lunch| +46593|AAAAAAAACAGLAAAA|46593|12|56|33|PM|first|afternoon|lunch| +46594|AAAAAAAADAGLAAAA|46594|12|56|34|PM|first|afternoon|lunch| +46595|AAAAAAAAEAGLAAAA|46595|12|56|35|PM|first|afternoon|lunch| +46596|AAAAAAAAFAGLAAAA|46596|12|56|36|PM|first|afternoon|lunch| +46597|AAAAAAAAGAGLAAAA|46597|12|56|37|PM|first|afternoon|lunch| +46598|AAAAAAAAHAGLAAAA|46598|12|56|38|PM|first|afternoon|lunch| +46599|AAAAAAAAIAGLAAAA|46599|12|56|39|PM|first|afternoon|lunch| +46600|AAAAAAAAJAGLAAAA|46600|12|56|40|PM|first|afternoon|lunch| +46601|AAAAAAAAKAGLAAAA|46601|12|56|41|PM|first|afternoon|lunch| +46602|AAAAAAAALAGLAAAA|46602|12|56|42|PM|first|afternoon|lunch| +46603|AAAAAAAAMAGLAAAA|46603|12|56|43|PM|first|afternoon|lunch| +46604|AAAAAAAANAGLAAAA|46604|12|56|44|PM|first|afternoon|lunch| +46605|AAAAAAAAOAGLAAAA|46605|12|56|45|PM|first|afternoon|lunch| +46606|AAAAAAAAPAGLAAAA|46606|12|56|46|PM|first|afternoon|lunch| +46607|AAAAAAAAABGLAAAA|46607|12|56|47|PM|first|afternoon|lunch| +46608|AAAAAAAABBGLAAAA|46608|12|56|48|PM|first|afternoon|lunch| +46609|AAAAAAAACBGLAAAA|46609|12|56|49|PM|first|afternoon|lunch| +46610|AAAAAAAADBGLAAAA|46610|12|56|50|PM|first|afternoon|lunch| +46611|AAAAAAAAEBGLAAAA|46611|12|56|51|PM|first|afternoon|lunch| +46612|AAAAAAAAFBGLAAAA|46612|12|56|52|PM|first|afternoon|lunch| +46613|AAAAAAAAGBGLAAAA|46613|12|56|53|PM|first|afternoon|lunch| +46614|AAAAAAAAHBGLAAAA|46614|12|56|54|PM|first|afternoon|lunch| +46615|AAAAAAAAIBGLAAAA|46615|12|56|55|PM|first|afternoon|lunch| +46616|AAAAAAAAJBGLAAAA|46616|12|56|56|PM|first|afternoon|lunch| +46617|AAAAAAAAKBGLAAAA|46617|12|56|57|PM|first|afternoon|lunch| +46618|AAAAAAAALBGLAAAA|46618|12|56|58|PM|first|afternoon|lunch| +46619|AAAAAAAAMBGLAAAA|46619|12|56|59|PM|first|afternoon|lunch| +46620|AAAAAAAANBGLAAAA|46620|12|57|0|PM|first|afternoon|lunch| +46621|AAAAAAAAOBGLAAAA|46621|12|57|1|PM|first|afternoon|lunch| +46622|AAAAAAAAPBGLAAAA|46622|12|57|2|PM|first|afternoon|lunch| +46623|AAAAAAAAACGLAAAA|46623|12|57|3|PM|first|afternoon|lunch| +46624|AAAAAAAABCGLAAAA|46624|12|57|4|PM|first|afternoon|lunch| +46625|AAAAAAAACCGLAAAA|46625|12|57|5|PM|first|afternoon|lunch| +46626|AAAAAAAADCGLAAAA|46626|12|57|6|PM|first|afternoon|lunch| +46627|AAAAAAAAECGLAAAA|46627|12|57|7|PM|first|afternoon|lunch| +46628|AAAAAAAAFCGLAAAA|46628|12|57|8|PM|first|afternoon|lunch| +46629|AAAAAAAAGCGLAAAA|46629|12|57|9|PM|first|afternoon|lunch| +46630|AAAAAAAAHCGLAAAA|46630|12|57|10|PM|first|afternoon|lunch| +46631|AAAAAAAAICGLAAAA|46631|12|57|11|PM|first|afternoon|lunch| +46632|AAAAAAAAJCGLAAAA|46632|12|57|12|PM|first|afternoon|lunch| +46633|AAAAAAAAKCGLAAAA|46633|12|57|13|PM|first|afternoon|lunch| +46634|AAAAAAAALCGLAAAA|46634|12|57|14|PM|first|afternoon|lunch| +46635|AAAAAAAAMCGLAAAA|46635|12|57|15|PM|first|afternoon|lunch| +46636|AAAAAAAANCGLAAAA|46636|12|57|16|PM|first|afternoon|lunch| +46637|AAAAAAAAOCGLAAAA|46637|12|57|17|PM|first|afternoon|lunch| +46638|AAAAAAAAPCGLAAAA|46638|12|57|18|PM|first|afternoon|lunch| +46639|AAAAAAAAADGLAAAA|46639|12|57|19|PM|first|afternoon|lunch| +46640|AAAAAAAABDGLAAAA|46640|12|57|20|PM|first|afternoon|lunch| +46641|AAAAAAAACDGLAAAA|46641|12|57|21|PM|first|afternoon|lunch| +46642|AAAAAAAADDGLAAAA|46642|12|57|22|PM|first|afternoon|lunch| +46643|AAAAAAAAEDGLAAAA|46643|12|57|23|PM|first|afternoon|lunch| +46644|AAAAAAAAFDGLAAAA|46644|12|57|24|PM|first|afternoon|lunch| +46645|AAAAAAAAGDGLAAAA|46645|12|57|25|PM|first|afternoon|lunch| +46646|AAAAAAAAHDGLAAAA|46646|12|57|26|PM|first|afternoon|lunch| +46647|AAAAAAAAIDGLAAAA|46647|12|57|27|PM|first|afternoon|lunch| +46648|AAAAAAAAJDGLAAAA|46648|12|57|28|PM|first|afternoon|lunch| +46649|AAAAAAAAKDGLAAAA|46649|12|57|29|PM|first|afternoon|lunch| +46650|AAAAAAAALDGLAAAA|46650|12|57|30|PM|first|afternoon|lunch| +46651|AAAAAAAAMDGLAAAA|46651|12|57|31|PM|first|afternoon|lunch| +46652|AAAAAAAANDGLAAAA|46652|12|57|32|PM|first|afternoon|lunch| +46653|AAAAAAAAODGLAAAA|46653|12|57|33|PM|first|afternoon|lunch| +46654|AAAAAAAAPDGLAAAA|46654|12|57|34|PM|first|afternoon|lunch| +46655|AAAAAAAAAEGLAAAA|46655|12|57|35|PM|first|afternoon|lunch| +46656|AAAAAAAABEGLAAAA|46656|12|57|36|PM|first|afternoon|lunch| +46657|AAAAAAAACEGLAAAA|46657|12|57|37|PM|first|afternoon|lunch| +46658|AAAAAAAADEGLAAAA|46658|12|57|38|PM|first|afternoon|lunch| +46659|AAAAAAAAEEGLAAAA|46659|12|57|39|PM|first|afternoon|lunch| +46660|AAAAAAAAFEGLAAAA|46660|12|57|40|PM|first|afternoon|lunch| +46661|AAAAAAAAGEGLAAAA|46661|12|57|41|PM|first|afternoon|lunch| +46662|AAAAAAAAHEGLAAAA|46662|12|57|42|PM|first|afternoon|lunch| +46663|AAAAAAAAIEGLAAAA|46663|12|57|43|PM|first|afternoon|lunch| +46664|AAAAAAAAJEGLAAAA|46664|12|57|44|PM|first|afternoon|lunch| +46665|AAAAAAAAKEGLAAAA|46665|12|57|45|PM|first|afternoon|lunch| +46666|AAAAAAAALEGLAAAA|46666|12|57|46|PM|first|afternoon|lunch| +46667|AAAAAAAAMEGLAAAA|46667|12|57|47|PM|first|afternoon|lunch| +46668|AAAAAAAANEGLAAAA|46668|12|57|48|PM|first|afternoon|lunch| +46669|AAAAAAAAOEGLAAAA|46669|12|57|49|PM|first|afternoon|lunch| +46670|AAAAAAAAPEGLAAAA|46670|12|57|50|PM|first|afternoon|lunch| +46671|AAAAAAAAAFGLAAAA|46671|12|57|51|PM|first|afternoon|lunch| +46672|AAAAAAAABFGLAAAA|46672|12|57|52|PM|first|afternoon|lunch| +46673|AAAAAAAACFGLAAAA|46673|12|57|53|PM|first|afternoon|lunch| +46674|AAAAAAAADFGLAAAA|46674|12|57|54|PM|first|afternoon|lunch| +46675|AAAAAAAAEFGLAAAA|46675|12|57|55|PM|first|afternoon|lunch| +46676|AAAAAAAAFFGLAAAA|46676|12|57|56|PM|first|afternoon|lunch| +46677|AAAAAAAAGFGLAAAA|46677|12|57|57|PM|first|afternoon|lunch| +46678|AAAAAAAAHFGLAAAA|46678|12|57|58|PM|first|afternoon|lunch| +46679|AAAAAAAAIFGLAAAA|46679|12|57|59|PM|first|afternoon|lunch| +46680|AAAAAAAAJFGLAAAA|46680|12|58|0|PM|first|afternoon|lunch| +46681|AAAAAAAAKFGLAAAA|46681|12|58|1|PM|first|afternoon|lunch| +46682|AAAAAAAALFGLAAAA|46682|12|58|2|PM|first|afternoon|lunch| +46683|AAAAAAAAMFGLAAAA|46683|12|58|3|PM|first|afternoon|lunch| +46684|AAAAAAAANFGLAAAA|46684|12|58|4|PM|first|afternoon|lunch| +46685|AAAAAAAAOFGLAAAA|46685|12|58|5|PM|first|afternoon|lunch| +46686|AAAAAAAAPFGLAAAA|46686|12|58|6|PM|first|afternoon|lunch| +46687|AAAAAAAAAGGLAAAA|46687|12|58|7|PM|first|afternoon|lunch| +46688|AAAAAAAABGGLAAAA|46688|12|58|8|PM|first|afternoon|lunch| +46689|AAAAAAAACGGLAAAA|46689|12|58|9|PM|first|afternoon|lunch| +46690|AAAAAAAADGGLAAAA|46690|12|58|10|PM|first|afternoon|lunch| +46691|AAAAAAAAEGGLAAAA|46691|12|58|11|PM|first|afternoon|lunch| +46692|AAAAAAAAFGGLAAAA|46692|12|58|12|PM|first|afternoon|lunch| +46693|AAAAAAAAGGGLAAAA|46693|12|58|13|PM|first|afternoon|lunch| +46694|AAAAAAAAHGGLAAAA|46694|12|58|14|PM|first|afternoon|lunch| +46695|AAAAAAAAIGGLAAAA|46695|12|58|15|PM|first|afternoon|lunch| +46696|AAAAAAAAJGGLAAAA|46696|12|58|16|PM|first|afternoon|lunch| +46697|AAAAAAAAKGGLAAAA|46697|12|58|17|PM|first|afternoon|lunch| +46698|AAAAAAAALGGLAAAA|46698|12|58|18|PM|first|afternoon|lunch| +46699|AAAAAAAAMGGLAAAA|46699|12|58|19|PM|first|afternoon|lunch| +46700|AAAAAAAANGGLAAAA|46700|12|58|20|PM|first|afternoon|lunch| +46701|AAAAAAAAOGGLAAAA|46701|12|58|21|PM|first|afternoon|lunch| +46702|AAAAAAAAPGGLAAAA|46702|12|58|22|PM|first|afternoon|lunch| +46703|AAAAAAAAAHGLAAAA|46703|12|58|23|PM|first|afternoon|lunch| +46704|AAAAAAAABHGLAAAA|46704|12|58|24|PM|first|afternoon|lunch| +46705|AAAAAAAACHGLAAAA|46705|12|58|25|PM|first|afternoon|lunch| +46706|AAAAAAAADHGLAAAA|46706|12|58|26|PM|first|afternoon|lunch| +46707|AAAAAAAAEHGLAAAA|46707|12|58|27|PM|first|afternoon|lunch| +46708|AAAAAAAAFHGLAAAA|46708|12|58|28|PM|first|afternoon|lunch| +46709|AAAAAAAAGHGLAAAA|46709|12|58|29|PM|first|afternoon|lunch| +46710|AAAAAAAAHHGLAAAA|46710|12|58|30|PM|first|afternoon|lunch| +46711|AAAAAAAAIHGLAAAA|46711|12|58|31|PM|first|afternoon|lunch| +46712|AAAAAAAAJHGLAAAA|46712|12|58|32|PM|first|afternoon|lunch| +46713|AAAAAAAAKHGLAAAA|46713|12|58|33|PM|first|afternoon|lunch| +46714|AAAAAAAALHGLAAAA|46714|12|58|34|PM|first|afternoon|lunch| +46715|AAAAAAAAMHGLAAAA|46715|12|58|35|PM|first|afternoon|lunch| +46716|AAAAAAAANHGLAAAA|46716|12|58|36|PM|first|afternoon|lunch| +46717|AAAAAAAAOHGLAAAA|46717|12|58|37|PM|first|afternoon|lunch| +46718|AAAAAAAAPHGLAAAA|46718|12|58|38|PM|first|afternoon|lunch| +46719|AAAAAAAAAIGLAAAA|46719|12|58|39|PM|first|afternoon|lunch| +46720|AAAAAAAABIGLAAAA|46720|12|58|40|PM|first|afternoon|lunch| +46721|AAAAAAAACIGLAAAA|46721|12|58|41|PM|first|afternoon|lunch| +46722|AAAAAAAADIGLAAAA|46722|12|58|42|PM|first|afternoon|lunch| +46723|AAAAAAAAEIGLAAAA|46723|12|58|43|PM|first|afternoon|lunch| +46724|AAAAAAAAFIGLAAAA|46724|12|58|44|PM|first|afternoon|lunch| +46725|AAAAAAAAGIGLAAAA|46725|12|58|45|PM|first|afternoon|lunch| +46726|AAAAAAAAHIGLAAAA|46726|12|58|46|PM|first|afternoon|lunch| +46727|AAAAAAAAIIGLAAAA|46727|12|58|47|PM|first|afternoon|lunch| +46728|AAAAAAAAJIGLAAAA|46728|12|58|48|PM|first|afternoon|lunch| +46729|AAAAAAAAKIGLAAAA|46729|12|58|49|PM|first|afternoon|lunch| +46730|AAAAAAAALIGLAAAA|46730|12|58|50|PM|first|afternoon|lunch| +46731|AAAAAAAAMIGLAAAA|46731|12|58|51|PM|first|afternoon|lunch| +46732|AAAAAAAANIGLAAAA|46732|12|58|52|PM|first|afternoon|lunch| +46733|AAAAAAAAOIGLAAAA|46733|12|58|53|PM|first|afternoon|lunch| +46734|AAAAAAAAPIGLAAAA|46734|12|58|54|PM|first|afternoon|lunch| +46735|AAAAAAAAAJGLAAAA|46735|12|58|55|PM|first|afternoon|lunch| +46736|AAAAAAAABJGLAAAA|46736|12|58|56|PM|first|afternoon|lunch| +46737|AAAAAAAACJGLAAAA|46737|12|58|57|PM|first|afternoon|lunch| +46738|AAAAAAAADJGLAAAA|46738|12|58|58|PM|first|afternoon|lunch| +46739|AAAAAAAAEJGLAAAA|46739|12|58|59|PM|first|afternoon|lunch| +46740|AAAAAAAAFJGLAAAA|46740|12|59|0|PM|first|afternoon|lunch| +46741|AAAAAAAAGJGLAAAA|46741|12|59|1|PM|first|afternoon|lunch| +46742|AAAAAAAAHJGLAAAA|46742|12|59|2|PM|first|afternoon|lunch| +46743|AAAAAAAAIJGLAAAA|46743|12|59|3|PM|first|afternoon|lunch| +46744|AAAAAAAAJJGLAAAA|46744|12|59|4|PM|first|afternoon|lunch| +46745|AAAAAAAAKJGLAAAA|46745|12|59|5|PM|first|afternoon|lunch| +46746|AAAAAAAALJGLAAAA|46746|12|59|6|PM|first|afternoon|lunch| +46747|AAAAAAAAMJGLAAAA|46747|12|59|7|PM|first|afternoon|lunch| +46748|AAAAAAAANJGLAAAA|46748|12|59|8|PM|first|afternoon|lunch| +46749|AAAAAAAAOJGLAAAA|46749|12|59|9|PM|first|afternoon|lunch| +46750|AAAAAAAAPJGLAAAA|46750|12|59|10|PM|first|afternoon|lunch| +46751|AAAAAAAAAKGLAAAA|46751|12|59|11|PM|first|afternoon|lunch| +46752|AAAAAAAABKGLAAAA|46752|12|59|12|PM|first|afternoon|lunch| +46753|AAAAAAAACKGLAAAA|46753|12|59|13|PM|first|afternoon|lunch| +46754|AAAAAAAADKGLAAAA|46754|12|59|14|PM|first|afternoon|lunch| +46755|AAAAAAAAEKGLAAAA|46755|12|59|15|PM|first|afternoon|lunch| +46756|AAAAAAAAFKGLAAAA|46756|12|59|16|PM|first|afternoon|lunch| +46757|AAAAAAAAGKGLAAAA|46757|12|59|17|PM|first|afternoon|lunch| +46758|AAAAAAAAHKGLAAAA|46758|12|59|18|PM|first|afternoon|lunch| +46759|AAAAAAAAIKGLAAAA|46759|12|59|19|PM|first|afternoon|lunch| +46760|AAAAAAAAJKGLAAAA|46760|12|59|20|PM|first|afternoon|lunch| +46761|AAAAAAAAKKGLAAAA|46761|12|59|21|PM|first|afternoon|lunch| +46762|AAAAAAAALKGLAAAA|46762|12|59|22|PM|first|afternoon|lunch| +46763|AAAAAAAAMKGLAAAA|46763|12|59|23|PM|first|afternoon|lunch| +46764|AAAAAAAANKGLAAAA|46764|12|59|24|PM|first|afternoon|lunch| +46765|AAAAAAAAOKGLAAAA|46765|12|59|25|PM|first|afternoon|lunch| +46766|AAAAAAAAPKGLAAAA|46766|12|59|26|PM|first|afternoon|lunch| +46767|AAAAAAAAALGLAAAA|46767|12|59|27|PM|first|afternoon|lunch| +46768|AAAAAAAABLGLAAAA|46768|12|59|28|PM|first|afternoon|lunch| +46769|AAAAAAAACLGLAAAA|46769|12|59|29|PM|first|afternoon|lunch| +46770|AAAAAAAADLGLAAAA|46770|12|59|30|PM|first|afternoon|lunch| +46771|AAAAAAAAELGLAAAA|46771|12|59|31|PM|first|afternoon|lunch| +46772|AAAAAAAAFLGLAAAA|46772|12|59|32|PM|first|afternoon|lunch| +46773|AAAAAAAAGLGLAAAA|46773|12|59|33|PM|first|afternoon|lunch| +46774|AAAAAAAAHLGLAAAA|46774|12|59|34|PM|first|afternoon|lunch| +46775|AAAAAAAAILGLAAAA|46775|12|59|35|PM|first|afternoon|lunch| +46776|AAAAAAAAJLGLAAAA|46776|12|59|36|PM|first|afternoon|lunch| +46777|AAAAAAAAKLGLAAAA|46777|12|59|37|PM|first|afternoon|lunch| +46778|AAAAAAAALLGLAAAA|46778|12|59|38|PM|first|afternoon|lunch| +46779|AAAAAAAAMLGLAAAA|46779|12|59|39|PM|first|afternoon|lunch| +46780|AAAAAAAANLGLAAAA|46780|12|59|40|PM|first|afternoon|lunch| +46781|AAAAAAAAOLGLAAAA|46781|12|59|41|PM|first|afternoon|lunch| +46782|AAAAAAAAPLGLAAAA|46782|12|59|42|PM|first|afternoon|lunch| +46783|AAAAAAAAAMGLAAAA|46783|12|59|43|PM|first|afternoon|lunch| +46784|AAAAAAAABMGLAAAA|46784|12|59|44|PM|first|afternoon|lunch| +46785|AAAAAAAACMGLAAAA|46785|12|59|45|PM|first|afternoon|lunch| +46786|AAAAAAAADMGLAAAA|46786|12|59|46|PM|first|afternoon|lunch| +46787|AAAAAAAAEMGLAAAA|46787|12|59|47|PM|first|afternoon|lunch| +46788|AAAAAAAAFMGLAAAA|46788|12|59|48|PM|first|afternoon|lunch| +46789|AAAAAAAAGMGLAAAA|46789|12|59|49|PM|first|afternoon|lunch| +46790|AAAAAAAAHMGLAAAA|46790|12|59|50|PM|first|afternoon|lunch| +46791|AAAAAAAAIMGLAAAA|46791|12|59|51|PM|first|afternoon|lunch| +46792|AAAAAAAAJMGLAAAA|46792|12|59|52|PM|first|afternoon|lunch| +46793|AAAAAAAAKMGLAAAA|46793|12|59|53|PM|first|afternoon|lunch| +46794|AAAAAAAALMGLAAAA|46794|12|59|54|PM|first|afternoon|lunch| +46795|AAAAAAAAMMGLAAAA|46795|12|59|55|PM|first|afternoon|lunch| +46796|AAAAAAAANMGLAAAA|46796|12|59|56|PM|first|afternoon|lunch| +46797|AAAAAAAAOMGLAAAA|46797|12|59|57|PM|first|afternoon|lunch| +46798|AAAAAAAAPMGLAAAA|46798|12|59|58|PM|first|afternoon|lunch| +46799|AAAAAAAAANGLAAAA|46799|12|59|59|PM|first|afternoon|lunch| +46800|AAAAAAAABNGLAAAA|46800|13|0|0|PM|first|afternoon|lunch| +46801|AAAAAAAACNGLAAAA|46801|13|0|1|PM|first|afternoon|lunch| +46802|AAAAAAAADNGLAAAA|46802|13|0|2|PM|first|afternoon|lunch| +46803|AAAAAAAAENGLAAAA|46803|13|0|3|PM|first|afternoon|lunch| +46804|AAAAAAAAFNGLAAAA|46804|13|0|4|PM|first|afternoon|lunch| +46805|AAAAAAAAGNGLAAAA|46805|13|0|5|PM|first|afternoon|lunch| +46806|AAAAAAAAHNGLAAAA|46806|13|0|6|PM|first|afternoon|lunch| +46807|AAAAAAAAINGLAAAA|46807|13|0|7|PM|first|afternoon|lunch| +46808|AAAAAAAAJNGLAAAA|46808|13|0|8|PM|first|afternoon|lunch| +46809|AAAAAAAAKNGLAAAA|46809|13|0|9|PM|first|afternoon|lunch| +46810|AAAAAAAALNGLAAAA|46810|13|0|10|PM|first|afternoon|lunch| +46811|AAAAAAAAMNGLAAAA|46811|13|0|11|PM|first|afternoon|lunch| +46812|AAAAAAAANNGLAAAA|46812|13|0|12|PM|first|afternoon|lunch| +46813|AAAAAAAAONGLAAAA|46813|13|0|13|PM|first|afternoon|lunch| +46814|AAAAAAAAPNGLAAAA|46814|13|0|14|PM|first|afternoon|lunch| +46815|AAAAAAAAAOGLAAAA|46815|13|0|15|PM|first|afternoon|lunch| +46816|AAAAAAAABOGLAAAA|46816|13|0|16|PM|first|afternoon|lunch| +46817|AAAAAAAACOGLAAAA|46817|13|0|17|PM|first|afternoon|lunch| +46818|AAAAAAAADOGLAAAA|46818|13|0|18|PM|first|afternoon|lunch| +46819|AAAAAAAAEOGLAAAA|46819|13|0|19|PM|first|afternoon|lunch| +46820|AAAAAAAAFOGLAAAA|46820|13|0|20|PM|first|afternoon|lunch| +46821|AAAAAAAAGOGLAAAA|46821|13|0|21|PM|first|afternoon|lunch| +46822|AAAAAAAAHOGLAAAA|46822|13|0|22|PM|first|afternoon|lunch| +46823|AAAAAAAAIOGLAAAA|46823|13|0|23|PM|first|afternoon|lunch| +46824|AAAAAAAAJOGLAAAA|46824|13|0|24|PM|first|afternoon|lunch| +46825|AAAAAAAAKOGLAAAA|46825|13|0|25|PM|first|afternoon|lunch| +46826|AAAAAAAALOGLAAAA|46826|13|0|26|PM|first|afternoon|lunch| +46827|AAAAAAAAMOGLAAAA|46827|13|0|27|PM|first|afternoon|lunch| +46828|AAAAAAAANOGLAAAA|46828|13|0|28|PM|first|afternoon|lunch| +46829|AAAAAAAAOOGLAAAA|46829|13|0|29|PM|first|afternoon|lunch| +46830|AAAAAAAAPOGLAAAA|46830|13|0|30|PM|first|afternoon|lunch| +46831|AAAAAAAAAPGLAAAA|46831|13|0|31|PM|first|afternoon|lunch| +46832|AAAAAAAABPGLAAAA|46832|13|0|32|PM|first|afternoon|lunch| +46833|AAAAAAAACPGLAAAA|46833|13|0|33|PM|first|afternoon|lunch| +46834|AAAAAAAADPGLAAAA|46834|13|0|34|PM|first|afternoon|lunch| +46835|AAAAAAAAEPGLAAAA|46835|13|0|35|PM|first|afternoon|lunch| +46836|AAAAAAAAFPGLAAAA|46836|13|0|36|PM|first|afternoon|lunch| +46837|AAAAAAAAGPGLAAAA|46837|13|0|37|PM|first|afternoon|lunch| +46838|AAAAAAAAHPGLAAAA|46838|13|0|38|PM|first|afternoon|lunch| +46839|AAAAAAAAIPGLAAAA|46839|13|0|39|PM|first|afternoon|lunch| +46840|AAAAAAAAJPGLAAAA|46840|13|0|40|PM|first|afternoon|lunch| +46841|AAAAAAAAKPGLAAAA|46841|13|0|41|PM|first|afternoon|lunch| +46842|AAAAAAAALPGLAAAA|46842|13|0|42|PM|first|afternoon|lunch| +46843|AAAAAAAAMPGLAAAA|46843|13|0|43|PM|first|afternoon|lunch| +46844|AAAAAAAANPGLAAAA|46844|13|0|44|PM|first|afternoon|lunch| +46845|AAAAAAAAOPGLAAAA|46845|13|0|45|PM|first|afternoon|lunch| +46846|AAAAAAAAPPGLAAAA|46846|13|0|46|PM|first|afternoon|lunch| +46847|AAAAAAAAAAHLAAAA|46847|13|0|47|PM|first|afternoon|lunch| +46848|AAAAAAAABAHLAAAA|46848|13|0|48|PM|first|afternoon|lunch| +46849|AAAAAAAACAHLAAAA|46849|13|0|49|PM|first|afternoon|lunch| +46850|AAAAAAAADAHLAAAA|46850|13|0|50|PM|first|afternoon|lunch| +46851|AAAAAAAAEAHLAAAA|46851|13|0|51|PM|first|afternoon|lunch| +46852|AAAAAAAAFAHLAAAA|46852|13|0|52|PM|first|afternoon|lunch| +46853|AAAAAAAAGAHLAAAA|46853|13|0|53|PM|first|afternoon|lunch| +46854|AAAAAAAAHAHLAAAA|46854|13|0|54|PM|first|afternoon|lunch| +46855|AAAAAAAAIAHLAAAA|46855|13|0|55|PM|first|afternoon|lunch| +46856|AAAAAAAAJAHLAAAA|46856|13|0|56|PM|first|afternoon|lunch| +46857|AAAAAAAAKAHLAAAA|46857|13|0|57|PM|first|afternoon|lunch| +46858|AAAAAAAALAHLAAAA|46858|13|0|58|PM|first|afternoon|lunch| +46859|AAAAAAAAMAHLAAAA|46859|13|0|59|PM|first|afternoon|lunch| +46860|AAAAAAAANAHLAAAA|46860|13|1|0|PM|first|afternoon|lunch| +46861|AAAAAAAAOAHLAAAA|46861|13|1|1|PM|first|afternoon|lunch| +46862|AAAAAAAAPAHLAAAA|46862|13|1|2|PM|first|afternoon|lunch| +46863|AAAAAAAAABHLAAAA|46863|13|1|3|PM|first|afternoon|lunch| +46864|AAAAAAAABBHLAAAA|46864|13|1|4|PM|first|afternoon|lunch| +46865|AAAAAAAACBHLAAAA|46865|13|1|5|PM|first|afternoon|lunch| +46866|AAAAAAAADBHLAAAA|46866|13|1|6|PM|first|afternoon|lunch| +46867|AAAAAAAAEBHLAAAA|46867|13|1|7|PM|first|afternoon|lunch| +46868|AAAAAAAAFBHLAAAA|46868|13|1|8|PM|first|afternoon|lunch| +46869|AAAAAAAAGBHLAAAA|46869|13|1|9|PM|first|afternoon|lunch| +46870|AAAAAAAAHBHLAAAA|46870|13|1|10|PM|first|afternoon|lunch| +46871|AAAAAAAAIBHLAAAA|46871|13|1|11|PM|first|afternoon|lunch| +46872|AAAAAAAAJBHLAAAA|46872|13|1|12|PM|first|afternoon|lunch| +46873|AAAAAAAAKBHLAAAA|46873|13|1|13|PM|first|afternoon|lunch| +46874|AAAAAAAALBHLAAAA|46874|13|1|14|PM|first|afternoon|lunch| +46875|AAAAAAAAMBHLAAAA|46875|13|1|15|PM|first|afternoon|lunch| +46876|AAAAAAAANBHLAAAA|46876|13|1|16|PM|first|afternoon|lunch| +46877|AAAAAAAAOBHLAAAA|46877|13|1|17|PM|first|afternoon|lunch| +46878|AAAAAAAAPBHLAAAA|46878|13|1|18|PM|first|afternoon|lunch| +46879|AAAAAAAAACHLAAAA|46879|13|1|19|PM|first|afternoon|lunch| +46880|AAAAAAAABCHLAAAA|46880|13|1|20|PM|first|afternoon|lunch| +46881|AAAAAAAACCHLAAAA|46881|13|1|21|PM|first|afternoon|lunch| +46882|AAAAAAAADCHLAAAA|46882|13|1|22|PM|first|afternoon|lunch| +46883|AAAAAAAAECHLAAAA|46883|13|1|23|PM|first|afternoon|lunch| +46884|AAAAAAAAFCHLAAAA|46884|13|1|24|PM|first|afternoon|lunch| +46885|AAAAAAAAGCHLAAAA|46885|13|1|25|PM|first|afternoon|lunch| +46886|AAAAAAAAHCHLAAAA|46886|13|1|26|PM|first|afternoon|lunch| +46887|AAAAAAAAICHLAAAA|46887|13|1|27|PM|first|afternoon|lunch| +46888|AAAAAAAAJCHLAAAA|46888|13|1|28|PM|first|afternoon|lunch| +46889|AAAAAAAAKCHLAAAA|46889|13|1|29|PM|first|afternoon|lunch| +46890|AAAAAAAALCHLAAAA|46890|13|1|30|PM|first|afternoon|lunch| +46891|AAAAAAAAMCHLAAAA|46891|13|1|31|PM|first|afternoon|lunch| +46892|AAAAAAAANCHLAAAA|46892|13|1|32|PM|first|afternoon|lunch| +46893|AAAAAAAAOCHLAAAA|46893|13|1|33|PM|first|afternoon|lunch| +46894|AAAAAAAAPCHLAAAA|46894|13|1|34|PM|first|afternoon|lunch| +46895|AAAAAAAAADHLAAAA|46895|13|1|35|PM|first|afternoon|lunch| +46896|AAAAAAAABDHLAAAA|46896|13|1|36|PM|first|afternoon|lunch| +46897|AAAAAAAACDHLAAAA|46897|13|1|37|PM|first|afternoon|lunch| +46898|AAAAAAAADDHLAAAA|46898|13|1|38|PM|first|afternoon|lunch| +46899|AAAAAAAAEDHLAAAA|46899|13|1|39|PM|first|afternoon|lunch| +46900|AAAAAAAAFDHLAAAA|46900|13|1|40|PM|first|afternoon|lunch| +46901|AAAAAAAAGDHLAAAA|46901|13|1|41|PM|first|afternoon|lunch| +46902|AAAAAAAAHDHLAAAA|46902|13|1|42|PM|first|afternoon|lunch| +46903|AAAAAAAAIDHLAAAA|46903|13|1|43|PM|first|afternoon|lunch| +46904|AAAAAAAAJDHLAAAA|46904|13|1|44|PM|first|afternoon|lunch| +46905|AAAAAAAAKDHLAAAA|46905|13|1|45|PM|first|afternoon|lunch| +46906|AAAAAAAALDHLAAAA|46906|13|1|46|PM|first|afternoon|lunch| +46907|AAAAAAAAMDHLAAAA|46907|13|1|47|PM|first|afternoon|lunch| +46908|AAAAAAAANDHLAAAA|46908|13|1|48|PM|first|afternoon|lunch| +46909|AAAAAAAAODHLAAAA|46909|13|1|49|PM|first|afternoon|lunch| +46910|AAAAAAAAPDHLAAAA|46910|13|1|50|PM|first|afternoon|lunch| +46911|AAAAAAAAAEHLAAAA|46911|13|1|51|PM|first|afternoon|lunch| +46912|AAAAAAAABEHLAAAA|46912|13|1|52|PM|first|afternoon|lunch| +46913|AAAAAAAACEHLAAAA|46913|13|1|53|PM|first|afternoon|lunch| +46914|AAAAAAAADEHLAAAA|46914|13|1|54|PM|first|afternoon|lunch| +46915|AAAAAAAAEEHLAAAA|46915|13|1|55|PM|first|afternoon|lunch| +46916|AAAAAAAAFEHLAAAA|46916|13|1|56|PM|first|afternoon|lunch| +46917|AAAAAAAAGEHLAAAA|46917|13|1|57|PM|first|afternoon|lunch| +46918|AAAAAAAAHEHLAAAA|46918|13|1|58|PM|first|afternoon|lunch| +46919|AAAAAAAAIEHLAAAA|46919|13|1|59|PM|first|afternoon|lunch| +46920|AAAAAAAAJEHLAAAA|46920|13|2|0|PM|first|afternoon|lunch| +46921|AAAAAAAAKEHLAAAA|46921|13|2|1|PM|first|afternoon|lunch| +46922|AAAAAAAALEHLAAAA|46922|13|2|2|PM|first|afternoon|lunch| +46923|AAAAAAAAMEHLAAAA|46923|13|2|3|PM|first|afternoon|lunch| +46924|AAAAAAAANEHLAAAA|46924|13|2|4|PM|first|afternoon|lunch| +46925|AAAAAAAAOEHLAAAA|46925|13|2|5|PM|first|afternoon|lunch| +46926|AAAAAAAAPEHLAAAA|46926|13|2|6|PM|first|afternoon|lunch| +46927|AAAAAAAAAFHLAAAA|46927|13|2|7|PM|first|afternoon|lunch| +46928|AAAAAAAABFHLAAAA|46928|13|2|8|PM|first|afternoon|lunch| +46929|AAAAAAAACFHLAAAA|46929|13|2|9|PM|first|afternoon|lunch| +46930|AAAAAAAADFHLAAAA|46930|13|2|10|PM|first|afternoon|lunch| +46931|AAAAAAAAEFHLAAAA|46931|13|2|11|PM|first|afternoon|lunch| +46932|AAAAAAAAFFHLAAAA|46932|13|2|12|PM|first|afternoon|lunch| +46933|AAAAAAAAGFHLAAAA|46933|13|2|13|PM|first|afternoon|lunch| +46934|AAAAAAAAHFHLAAAA|46934|13|2|14|PM|first|afternoon|lunch| +46935|AAAAAAAAIFHLAAAA|46935|13|2|15|PM|first|afternoon|lunch| +46936|AAAAAAAAJFHLAAAA|46936|13|2|16|PM|first|afternoon|lunch| +46937|AAAAAAAAKFHLAAAA|46937|13|2|17|PM|first|afternoon|lunch| +46938|AAAAAAAALFHLAAAA|46938|13|2|18|PM|first|afternoon|lunch| +46939|AAAAAAAAMFHLAAAA|46939|13|2|19|PM|first|afternoon|lunch| +46940|AAAAAAAANFHLAAAA|46940|13|2|20|PM|first|afternoon|lunch| +46941|AAAAAAAAOFHLAAAA|46941|13|2|21|PM|first|afternoon|lunch| +46942|AAAAAAAAPFHLAAAA|46942|13|2|22|PM|first|afternoon|lunch| +46943|AAAAAAAAAGHLAAAA|46943|13|2|23|PM|first|afternoon|lunch| +46944|AAAAAAAABGHLAAAA|46944|13|2|24|PM|first|afternoon|lunch| +46945|AAAAAAAACGHLAAAA|46945|13|2|25|PM|first|afternoon|lunch| +46946|AAAAAAAADGHLAAAA|46946|13|2|26|PM|first|afternoon|lunch| +46947|AAAAAAAAEGHLAAAA|46947|13|2|27|PM|first|afternoon|lunch| +46948|AAAAAAAAFGHLAAAA|46948|13|2|28|PM|first|afternoon|lunch| +46949|AAAAAAAAGGHLAAAA|46949|13|2|29|PM|first|afternoon|lunch| +46950|AAAAAAAAHGHLAAAA|46950|13|2|30|PM|first|afternoon|lunch| +46951|AAAAAAAAIGHLAAAA|46951|13|2|31|PM|first|afternoon|lunch| +46952|AAAAAAAAJGHLAAAA|46952|13|2|32|PM|first|afternoon|lunch| +46953|AAAAAAAAKGHLAAAA|46953|13|2|33|PM|first|afternoon|lunch| +46954|AAAAAAAALGHLAAAA|46954|13|2|34|PM|first|afternoon|lunch| +46955|AAAAAAAAMGHLAAAA|46955|13|2|35|PM|first|afternoon|lunch| +46956|AAAAAAAANGHLAAAA|46956|13|2|36|PM|first|afternoon|lunch| +46957|AAAAAAAAOGHLAAAA|46957|13|2|37|PM|first|afternoon|lunch| +46958|AAAAAAAAPGHLAAAA|46958|13|2|38|PM|first|afternoon|lunch| +46959|AAAAAAAAAHHLAAAA|46959|13|2|39|PM|first|afternoon|lunch| +46960|AAAAAAAABHHLAAAA|46960|13|2|40|PM|first|afternoon|lunch| +46961|AAAAAAAACHHLAAAA|46961|13|2|41|PM|first|afternoon|lunch| +46962|AAAAAAAADHHLAAAA|46962|13|2|42|PM|first|afternoon|lunch| +46963|AAAAAAAAEHHLAAAA|46963|13|2|43|PM|first|afternoon|lunch| +46964|AAAAAAAAFHHLAAAA|46964|13|2|44|PM|first|afternoon|lunch| +46965|AAAAAAAAGHHLAAAA|46965|13|2|45|PM|first|afternoon|lunch| +46966|AAAAAAAAHHHLAAAA|46966|13|2|46|PM|first|afternoon|lunch| +46967|AAAAAAAAIHHLAAAA|46967|13|2|47|PM|first|afternoon|lunch| +46968|AAAAAAAAJHHLAAAA|46968|13|2|48|PM|first|afternoon|lunch| +46969|AAAAAAAAKHHLAAAA|46969|13|2|49|PM|first|afternoon|lunch| +46970|AAAAAAAALHHLAAAA|46970|13|2|50|PM|first|afternoon|lunch| +46971|AAAAAAAAMHHLAAAA|46971|13|2|51|PM|first|afternoon|lunch| +46972|AAAAAAAANHHLAAAA|46972|13|2|52|PM|first|afternoon|lunch| +46973|AAAAAAAAOHHLAAAA|46973|13|2|53|PM|first|afternoon|lunch| +46974|AAAAAAAAPHHLAAAA|46974|13|2|54|PM|first|afternoon|lunch| +46975|AAAAAAAAAIHLAAAA|46975|13|2|55|PM|first|afternoon|lunch| +46976|AAAAAAAABIHLAAAA|46976|13|2|56|PM|first|afternoon|lunch| +46977|AAAAAAAACIHLAAAA|46977|13|2|57|PM|first|afternoon|lunch| +46978|AAAAAAAADIHLAAAA|46978|13|2|58|PM|first|afternoon|lunch| +46979|AAAAAAAAEIHLAAAA|46979|13|2|59|PM|first|afternoon|lunch| +46980|AAAAAAAAFIHLAAAA|46980|13|3|0|PM|first|afternoon|lunch| +46981|AAAAAAAAGIHLAAAA|46981|13|3|1|PM|first|afternoon|lunch| +46982|AAAAAAAAHIHLAAAA|46982|13|3|2|PM|first|afternoon|lunch| +46983|AAAAAAAAIIHLAAAA|46983|13|3|3|PM|first|afternoon|lunch| +46984|AAAAAAAAJIHLAAAA|46984|13|3|4|PM|first|afternoon|lunch| +46985|AAAAAAAAKIHLAAAA|46985|13|3|5|PM|first|afternoon|lunch| +46986|AAAAAAAALIHLAAAA|46986|13|3|6|PM|first|afternoon|lunch| +46987|AAAAAAAAMIHLAAAA|46987|13|3|7|PM|first|afternoon|lunch| +46988|AAAAAAAANIHLAAAA|46988|13|3|8|PM|first|afternoon|lunch| +46989|AAAAAAAAOIHLAAAA|46989|13|3|9|PM|first|afternoon|lunch| +46990|AAAAAAAAPIHLAAAA|46990|13|3|10|PM|first|afternoon|lunch| +46991|AAAAAAAAAJHLAAAA|46991|13|3|11|PM|first|afternoon|lunch| +46992|AAAAAAAABJHLAAAA|46992|13|3|12|PM|first|afternoon|lunch| +46993|AAAAAAAACJHLAAAA|46993|13|3|13|PM|first|afternoon|lunch| +46994|AAAAAAAADJHLAAAA|46994|13|3|14|PM|first|afternoon|lunch| +46995|AAAAAAAAEJHLAAAA|46995|13|3|15|PM|first|afternoon|lunch| +46996|AAAAAAAAFJHLAAAA|46996|13|3|16|PM|first|afternoon|lunch| +46997|AAAAAAAAGJHLAAAA|46997|13|3|17|PM|first|afternoon|lunch| +46998|AAAAAAAAHJHLAAAA|46998|13|3|18|PM|first|afternoon|lunch| +46999|AAAAAAAAIJHLAAAA|46999|13|3|19|PM|first|afternoon|lunch| +47000|AAAAAAAAJJHLAAAA|47000|13|3|20|PM|first|afternoon|lunch| +47001|AAAAAAAAKJHLAAAA|47001|13|3|21|PM|first|afternoon|lunch| +47002|AAAAAAAALJHLAAAA|47002|13|3|22|PM|first|afternoon|lunch| +47003|AAAAAAAAMJHLAAAA|47003|13|3|23|PM|first|afternoon|lunch| +47004|AAAAAAAANJHLAAAA|47004|13|3|24|PM|first|afternoon|lunch| +47005|AAAAAAAAOJHLAAAA|47005|13|3|25|PM|first|afternoon|lunch| +47006|AAAAAAAAPJHLAAAA|47006|13|3|26|PM|first|afternoon|lunch| +47007|AAAAAAAAAKHLAAAA|47007|13|3|27|PM|first|afternoon|lunch| +47008|AAAAAAAABKHLAAAA|47008|13|3|28|PM|first|afternoon|lunch| +47009|AAAAAAAACKHLAAAA|47009|13|3|29|PM|first|afternoon|lunch| +47010|AAAAAAAADKHLAAAA|47010|13|3|30|PM|first|afternoon|lunch| +47011|AAAAAAAAEKHLAAAA|47011|13|3|31|PM|first|afternoon|lunch| +47012|AAAAAAAAFKHLAAAA|47012|13|3|32|PM|first|afternoon|lunch| +47013|AAAAAAAAGKHLAAAA|47013|13|3|33|PM|first|afternoon|lunch| +47014|AAAAAAAAHKHLAAAA|47014|13|3|34|PM|first|afternoon|lunch| +47015|AAAAAAAAIKHLAAAA|47015|13|3|35|PM|first|afternoon|lunch| +47016|AAAAAAAAJKHLAAAA|47016|13|3|36|PM|first|afternoon|lunch| +47017|AAAAAAAAKKHLAAAA|47017|13|3|37|PM|first|afternoon|lunch| +47018|AAAAAAAALKHLAAAA|47018|13|3|38|PM|first|afternoon|lunch| +47019|AAAAAAAAMKHLAAAA|47019|13|3|39|PM|first|afternoon|lunch| +47020|AAAAAAAANKHLAAAA|47020|13|3|40|PM|first|afternoon|lunch| +47021|AAAAAAAAOKHLAAAA|47021|13|3|41|PM|first|afternoon|lunch| +47022|AAAAAAAAPKHLAAAA|47022|13|3|42|PM|first|afternoon|lunch| +47023|AAAAAAAAALHLAAAA|47023|13|3|43|PM|first|afternoon|lunch| +47024|AAAAAAAABLHLAAAA|47024|13|3|44|PM|first|afternoon|lunch| +47025|AAAAAAAACLHLAAAA|47025|13|3|45|PM|first|afternoon|lunch| +47026|AAAAAAAADLHLAAAA|47026|13|3|46|PM|first|afternoon|lunch| +47027|AAAAAAAAELHLAAAA|47027|13|3|47|PM|first|afternoon|lunch| +47028|AAAAAAAAFLHLAAAA|47028|13|3|48|PM|first|afternoon|lunch| +47029|AAAAAAAAGLHLAAAA|47029|13|3|49|PM|first|afternoon|lunch| +47030|AAAAAAAAHLHLAAAA|47030|13|3|50|PM|first|afternoon|lunch| +47031|AAAAAAAAILHLAAAA|47031|13|3|51|PM|first|afternoon|lunch| +47032|AAAAAAAAJLHLAAAA|47032|13|3|52|PM|first|afternoon|lunch| +47033|AAAAAAAAKLHLAAAA|47033|13|3|53|PM|first|afternoon|lunch| +47034|AAAAAAAALLHLAAAA|47034|13|3|54|PM|first|afternoon|lunch| +47035|AAAAAAAAMLHLAAAA|47035|13|3|55|PM|first|afternoon|lunch| +47036|AAAAAAAANLHLAAAA|47036|13|3|56|PM|first|afternoon|lunch| +47037|AAAAAAAAOLHLAAAA|47037|13|3|57|PM|first|afternoon|lunch| +47038|AAAAAAAAPLHLAAAA|47038|13|3|58|PM|first|afternoon|lunch| +47039|AAAAAAAAAMHLAAAA|47039|13|3|59|PM|first|afternoon|lunch| +47040|AAAAAAAABMHLAAAA|47040|13|4|0|PM|first|afternoon|lunch| +47041|AAAAAAAACMHLAAAA|47041|13|4|1|PM|first|afternoon|lunch| +47042|AAAAAAAADMHLAAAA|47042|13|4|2|PM|first|afternoon|lunch| +47043|AAAAAAAAEMHLAAAA|47043|13|4|3|PM|first|afternoon|lunch| +47044|AAAAAAAAFMHLAAAA|47044|13|4|4|PM|first|afternoon|lunch| +47045|AAAAAAAAGMHLAAAA|47045|13|4|5|PM|first|afternoon|lunch| +47046|AAAAAAAAHMHLAAAA|47046|13|4|6|PM|first|afternoon|lunch| +47047|AAAAAAAAIMHLAAAA|47047|13|4|7|PM|first|afternoon|lunch| +47048|AAAAAAAAJMHLAAAA|47048|13|4|8|PM|first|afternoon|lunch| +47049|AAAAAAAAKMHLAAAA|47049|13|4|9|PM|first|afternoon|lunch| +47050|AAAAAAAALMHLAAAA|47050|13|4|10|PM|first|afternoon|lunch| +47051|AAAAAAAAMMHLAAAA|47051|13|4|11|PM|first|afternoon|lunch| +47052|AAAAAAAANMHLAAAA|47052|13|4|12|PM|first|afternoon|lunch| +47053|AAAAAAAAOMHLAAAA|47053|13|4|13|PM|first|afternoon|lunch| +47054|AAAAAAAAPMHLAAAA|47054|13|4|14|PM|first|afternoon|lunch| +47055|AAAAAAAAANHLAAAA|47055|13|4|15|PM|first|afternoon|lunch| +47056|AAAAAAAABNHLAAAA|47056|13|4|16|PM|first|afternoon|lunch| +47057|AAAAAAAACNHLAAAA|47057|13|4|17|PM|first|afternoon|lunch| +47058|AAAAAAAADNHLAAAA|47058|13|4|18|PM|first|afternoon|lunch| +47059|AAAAAAAAENHLAAAA|47059|13|4|19|PM|first|afternoon|lunch| +47060|AAAAAAAAFNHLAAAA|47060|13|4|20|PM|first|afternoon|lunch| +47061|AAAAAAAAGNHLAAAA|47061|13|4|21|PM|first|afternoon|lunch| +47062|AAAAAAAAHNHLAAAA|47062|13|4|22|PM|first|afternoon|lunch| +47063|AAAAAAAAINHLAAAA|47063|13|4|23|PM|first|afternoon|lunch| +47064|AAAAAAAAJNHLAAAA|47064|13|4|24|PM|first|afternoon|lunch| +47065|AAAAAAAAKNHLAAAA|47065|13|4|25|PM|first|afternoon|lunch| +47066|AAAAAAAALNHLAAAA|47066|13|4|26|PM|first|afternoon|lunch| +47067|AAAAAAAAMNHLAAAA|47067|13|4|27|PM|first|afternoon|lunch| +47068|AAAAAAAANNHLAAAA|47068|13|4|28|PM|first|afternoon|lunch| +47069|AAAAAAAAONHLAAAA|47069|13|4|29|PM|first|afternoon|lunch| +47070|AAAAAAAAPNHLAAAA|47070|13|4|30|PM|first|afternoon|lunch| +47071|AAAAAAAAAOHLAAAA|47071|13|4|31|PM|first|afternoon|lunch| +47072|AAAAAAAABOHLAAAA|47072|13|4|32|PM|first|afternoon|lunch| +47073|AAAAAAAACOHLAAAA|47073|13|4|33|PM|first|afternoon|lunch| +47074|AAAAAAAADOHLAAAA|47074|13|4|34|PM|first|afternoon|lunch| +47075|AAAAAAAAEOHLAAAA|47075|13|4|35|PM|first|afternoon|lunch| +47076|AAAAAAAAFOHLAAAA|47076|13|4|36|PM|first|afternoon|lunch| +47077|AAAAAAAAGOHLAAAA|47077|13|4|37|PM|first|afternoon|lunch| +47078|AAAAAAAAHOHLAAAA|47078|13|4|38|PM|first|afternoon|lunch| +47079|AAAAAAAAIOHLAAAA|47079|13|4|39|PM|first|afternoon|lunch| +47080|AAAAAAAAJOHLAAAA|47080|13|4|40|PM|first|afternoon|lunch| +47081|AAAAAAAAKOHLAAAA|47081|13|4|41|PM|first|afternoon|lunch| +47082|AAAAAAAALOHLAAAA|47082|13|4|42|PM|first|afternoon|lunch| +47083|AAAAAAAAMOHLAAAA|47083|13|4|43|PM|first|afternoon|lunch| +47084|AAAAAAAANOHLAAAA|47084|13|4|44|PM|first|afternoon|lunch| +47085|AAAAAAAAOOHLAAAA|47085|13|4|45|PM|first|afternoon|lunch| +47086|AAAAAAAAPOHLAAAA|47086|13|4|46|PM|first|afternoon|lunch| +47087|AAAAAAAAAPHLAAAA|47087|13|4|47|PM|first|afternoon|lunch| +47088|AAAAAAAABPHLAAAA|47088|13|4|48|PM|first|afternoon|lunch| +47089|AAAAAAAACPHLAAAA|47089|13|4|49|PM|first|afternoon|lunch| +47090|AAAAAAAADPHLAAAA|47090|13|4|50|PM|first|afternoon|lunch| +47091|AAAAAAAAEPHLAAAA|47091|13|4|51|PM|first|afternoon|lunch| +47092|AAAAAAAAFPHLAAAA|47092|13|4|52|PM|first|afternoon|lunch| +47093|AAAAAAAAGPHLAAAA|47093|13|4|53|PM|first|afternoon|lunch| +47094|AAAAAAAAHPHLAAAA|47094|13|4|54|PM|first|afternoon|lunch| +47095|AAAAAAAAIPHLAAAA|47095|13|4|55|PM|first|afternoon|lunch| +47096|AAAAAAAAJPHLAAAA|47096|13|4|56|PM|first|afternoon|lunch| +47097|AAAAAAAAKPHLAAAA|47097|13|4|57|PM|first|afternoon|lunch| +47098|AAAAAAAALPHLAAAA|47098|13|4|58|PM|first|afternoon|lunch| +47099|AAAAAAAAMPHLAAAA|47099|13|4|59|PM|first|afternoon|lunch| +47100|AAAAAAAANPHLAAAA|47100|13|5|0|PM|first|afternoon|lunch| +47101|AAAAAAAAOPHLAAAA|47101|13|5|1|PM|first|afternoon|lunch| +47102|AAAAAAAAPPHLAAAA|47102|13|5|2|PM|first|afternoon|lunch| +47103|AAAAAAAAAAILAAAA|47103|13|5|3|PM|first|afternoon|lunch| +47104|AAAAAAAABAILAAAA|47104|13|5|4|PM|first|afternoon|lunch| +47105|AAAAAAAACAILAAAA|47105|13|5|5|PM|first|afternoon|lunch| +47106|AAAAAAAADAILAAAA|47106|13|5|6|PM|first|afternoon|lunch| +47107|AAAAAAAAEAILAAAA|47107|13|5|7|PM|first|afternoon|lunch| +47108|AAAAAAAAFAILAAAA|47108|13|5|8|PM|first|afternoon|lunch| +47109|AAAAAAAAGAILAAAA|47109|13|5|9|PM|first|afternoon|lunch| +47110|AAAAAAAAHAILAAAA|47110|13|5|10|PM|first|afternoon|lunch| +47111|AAAAAAAAIAILAAAA|47111|13|5|11|PM|first|afternoon|lunch| +47112|AAAAAAAAJAILAAAA|47112|13|5|12|PM|first|afternoon|lunch| +47113|AAAAAAAAKAILAAAA|47113|13|5|13|PM|first|afternoon|lunch| +47114|AAAAAAAALAILAAAA|47114|13|5|14|PM|first|afternoon|lunch| +47115|AAAAAAAAMAILAAAA|47115|13|5|15|PM|first|afternoon|lunch| +47116|AAAAAAAANAILAAAA|47116|13|5|16|PM|first|afternoon|lunch| +47117|AAAAAAAAOAILAAAA|47117|13|5|17|PM|first|afternoon|lunch| +47118|AAAAAAAAPAILAAAA|47118|13|5|18|PM|first|afternoon|lunch| +47119|AAAAAAAAABILAAAA|47119|13|5|19|PM|first|afternoon|lunch| +47120|AAAAAAAABBILAAAA|47120|13|5|20|PM|first|afternoon|lunch| +47121|AAAAAAAACBILAAAA|47121|13|5|21|PM|first|afternoon|lunch| +47122|AAAAAAAADBILAAAA|47122|13|5|22|PM|first|afternoon|lunch| +47123|AAAAAAAAEBILAAAA|47123|13|5|23|PM|first|afternoon|lunch| +47124|AAAAAAAAFBILAAAA|47124|13|5|24|PM|first|afternoon|lunch| +47125|AAAAAAAAGBILAAAA|47125|13|5|25|PM|first|afternoon|lunch| +47126|AAAAAAAAHBILAAAA|47126|13|5|26|PM|first|afternoon|lunch| +47127|AAAAAAAAIBILAAAA|47127|13|5|27|PM|first|afternoon|lunch| +47128|AAAAAAAAJBILAAAA|47128|13|5|28|PM|first|afternoon|lunch| +47129|AAAAAAAAKBILAAAA|47129|13|5|29|PM|first|afternoon|lunch| +47130|AAAAAAAALBILAAAA|47130|13|5|30|PM|first|afternoon|lunch| +47131|AAAAAAAAMBILAAAA|47131|13|5|31|PM|first|afternoon|lunch| +47132|AAAAAAAANBILAAAA|47132|13|5|32|PM|first|afternoon|lunch| +47133|AAAAAAAAOBILAAAA|47133|13|5|33|PM|first|afternoon|lunch| +47134|AAAAAAAAPBILAAAA|47134|13|5|34|PM|first|afternoon|lunch| +47135|AAAAAAAAACILAAAA|47135|13|5|35|PM|first|afternoon|lunch| +47136|AAAAAAAABCILAAAA|47136|13|5|36|PM|first|afternoon|lunch| +47137|AAAAAAAACCILAAAA|47137|13|5|37|PM|first|afternoon|lunch| +47138|AAAAAAAADCILAAAA|47138|13|5|38|PM|first|afternoon|lunch| +47139|AAAAAAAAECILAAAA|47139|13|5|39|PM|first|afternoon|lunch| +47140|AAAAAAAAFCILAAAA|47140|13|5|40|PM|first|afternoon|lunch| +47141|AAAAAAAAGCILAAAA|47141|13|5|41|PM|first|afternoon|lunch| +47142|AAAAAAAAHCILAAAA|47142|13|5|42|PM|first|afternoon|lunch| +47143|AAAAAAAAICILAAAA|47143|13|5|43|PM|first|afternoon|lunch| +47144|AAAAAAAAJCILAAAA|47144|13|5|44|PM|first|afternoon|lunch| +47145|AAAAAAAAKCILAAAA|47145|13|5|45|PM|first|afternoon|lunch| +47146|AAAAAAAALCILAAAA|47146|13|5|46|PM|first|afternoon|lunch| +47147|AAAAAAAAMCILAAAA|47147|13|5|47|PM|first|afternoon|lunch| +47148|AAAAAAAANCILAAAA|47148|13|5|48|PM|first|afternoon|lunch| +47149|AAAAAAAAOCILAAAA|47149|13|5|49|PM|first|afternoon|lunch| +47150|AAAAAAAAPCILAAAA|47150|13|5|50|PM|first|afternoon|lunch| +47151|AAAAAAAAADILAAAA|47151|13|5|51|PM|first|afternoon|lunch| +47152|AAAAAAAABDILAAAA|47152|13|5|52|PM|first|afternoon|lunch| +47153|AAAAAAAACDILAAAA|47153|13|5|53|PM|first|afternoon|lunch| +47154|AAAAAAAADDILAAAA|47154|13|5|54|PM|first|afternoon|lunch| +47155|AAAAAAAAEDILAAAA|47155|13|5|55|PM|first|afternoon|lunch| +47156|AAAAAAAAFDILAAAA|47156|13|5|56|PM|first|afternoon|lunch| +47157|AAAAAAAAGDILAAAA|47157|13|5|57|PM|first|afternoon|lunch| +47158|AAAAAAAAHDILAAAA|47158|13|5|58|PM|first|afternoon|lunch| +47159|AAAAAAAAIDILAAAA|47159|13|5|59|PM|first|afternoon|lunch| +47160|AAAAAAAAJDILAAAA|47160|13|6|0|PM|first|afternoon|lunch| +47161|AAAAAAAAKDILAAAA|47161|13|6|1|PM|first|afternoon|lunch| +47162|AAAAAAAALDILAAAA|47162|13|6|2|PM|first|afternoon|lunch| +47163|AAAAAAAAMDILAAAA|47163|13|6|3|PM|first|afternoon|lunch| +47164|AAAAAAAANDILAAAA|47164|13|6|4|PM|first|afternoon|lunch| +47165|AAAAAAAAODILAAAA|47165|13|6|5|PM|first|afternoon|lunch| +47166|AAAAAAAAPDILAAAA|47166|13|6|6|PM|first|afternoon|lunch| +47167|AAAAAAAAAEILAAAA|47167|13|6|7|PM|first|afternoon|lunch| +47168|AAAAAAAABEILAAAA|47168|13|6|8|PM|first|afternoon|lunch| +47169|AAAAAAAACEILAAAA|47169|13|6|9|PM|first|afternoon|lunch| +47170|AAAAAAAADEILAAAA|47170|13|6|10|PM|first|afternoon|lunch| +47171|AAAAAAAAEEILAAAA|47171|13|6|11|PM|first|afternoon|lunch| +47172|AAAAAAAAFEILAAAA|47172|13|6|12|PM|first|afternoon|lunch| +47173|AAAAAAAAGEILAAAA|47173|13|6|13|PM|first|afternoon|lunch| +47174|AAAAAAAAHEILAAAA|47174|13|6|14|PM|first|afternoon|lunch| +47175|AAAAAAAAIEILAAAA|47175|13|6|15|PM|first|afternoon|lunch| +47176|AAAAAAAAJEILAAAA|47176|13|6|16|PM|first|afternoon|lunch| +47177|AAAAAAAAKEILAAAA|47177|13|6|17|PM|first|afternoon|lunch| +47178|AAAAAAAALEILAAAA|47178|13|6|18|PM|first|afternoon|lunch| +47179|AAAAAAAAMEILAAAA|47179|13|6|19|PM|first|afternoon|lunch| +47180|AAAAAAAANEILAAAA|47180|13|6|20|PM|first|afternoon|lunch| +47181|AAAAAAAAOEILAAAA|47181|13|6|21|PM|first|afternoon|lunch| +47182|AAAAAAAAPEILAAAA|47182|13|6|22|PM|first|afternoon|lunch| +47183|AAAAAAAAAFILAAAA|47183|13|6|23|PM|first|afternoon|lunch| +47184|AAAAAAAABFILAAAA|47184|13|6|24|PM|first|afternoon|lunch| +47185|AAAAAAAACFILAAAA|47185|13|6|25|PM|first|afternoon|lunch| +47186|AAAAAAAADFILAAAA|47186|13|6|26|PM|first|afternoon|lunch| +47187|AAAAAAAAEFILAAAA|47187|13|6|27|PM|first|afternoon|lunch| +47188|AAAAAAAAFFILAAAA|47188|13|6|28|PM|first|afternoon|lunch| +47189|AAAAAAAAGFILAAAA|47189|13|6|29|PM|first|afternoon|lunch| +47190|AAAAAAAAHFILAAAA|47190|13|6|30|PM|first|afternoon|lunch| +47191|AAAAAAAAIFILAAAA|47191|13|6|31|PM|first|afternoon|lunch| +47192|AAAAAAAAJFILAAAA|47192|13|6|32|PM|first|afternoon|lunch| +47193|AAAAAAAAKFILAAAA|47193|13|6|33|PM|first|afternoon|lunch| +47194|AAAAAAAALFILAAAA|47194|13|6|34|PM|first|afternoon|lunch| +47195|AAAAAAAAMFILAAAA|47195|13|6|35|PM|first|afternoon|lunch| +47196|AAAAAAAANFILAAAA|47196|13|6|36|PM|first|afternoon|lunch| +47197|AAAAAAAAOFILAAAA|47197|13|6|37|PM|first|afternoon|lunch| +47198|AAAAAAAAPFILAAAA|47198|13|6|38|PM|first|afternoon|lunch| +47199|AAAAAAAAAGILAAAA|47199|13|6|39|PM|first|afternoon|lunch| +47200|AAAAAAAABGILAAAA|47200|13|6|40|PM|first|afternoon|lunch| +47201|AAAAAAAACGILAAAA|47201|13|6|41|PM|first|afternoon|lunch| +47202|AAAAAAAADGILAAAA|47202|13|6|42|PM|first|afternoon|lunch| +47203|AAAAAAAAEGILAAAA|47203|13|6|43|PM|first|afternoon|lunch| +47204|AAAAAAAAFGILAAAA|47204|13|6|44|PM|first|afternoon|lunch| +47205|AAAAAAAAGGILAAAA|47205|13|6|45|PM|first|afternoon|lunch| +47206|AAAAAAAAHGILAAAA|47206|13|6|46|PM|first|afternoon|lunch| +47207|AAAAAAAAIGILAAAA|47207|13|6|47|PM|first|afternoon|lunch| +47208|AAAAAAAAJGILAAAA|47208|13|6|48|PM|first|afternoon|lunch| +47209|AAAAAAAAKGILAAAA|47209|13|6|49|PM|first|afternoon|lunch| +47210|AAAAAAAALGILAAAA|47210|13|6|50|PM|first|afternoon|lunch| +47211|AAAAAAAAMGILAAAA|47211|13|6|51|PM|first|afternoon|lunch| +47212|AAAAAAAANGILAAAA|47212|13|6|52|PM|first|afternoon|lunch| +47213|AAAAAAAAOGILAAAA|47213|13|6|53|PM|first|afternoon|lunch| +47214|AAAAAAAAPGILAAAA|47214|13|6|54|PM|first|afternoon|lunch| +47215|AAAAAAAAAHILAAAA|47215|13|6|55|PM|first|afternoon|lunch| +47216|AAAAAAAABHILAAAA|47216|13|6|56|PM|first|afternoon|lunch| +47217|AAAAAAAACHILAAAA|47217|13|6|57|PM|first|afternoon|lunch| +47218|AAAAAAAADHILAAAA|47218|13|6|58|PM|first|afternoon|lunch| +47219|AAAAAAAAEHILAAAA|47219|13|6|59|PM|first|afternoon|lunch| +47220|AAAAAAAAFHILAAAA|47220|13|7|0|PM|first|afternoon|lunch| +47221|AAAAAAAAGHILAAAA|47221|13|7|1|PM|first|afternoon|lunch| +47222|AAAAAAAAHHILAAAA|47222|13|7|2|PM|first|afternoon|lunch| +47223|AAAAAAAAIHILAAAA|47223|13|7|3|PM|first|afternoon|lunch| +47224|AAAAAAAAJHILAAAA|47224|13|7|4|PM|first|afternoon|lunch| +47225|AAAAAAAAKHILAAAA|47225|13|7|5|PM|first|afternoon|lunch| +47226|AAAAAAAALHILAAAA|47226|13|7|6|PM|first|afternoon|lunch| +47227|AAAAAAAAMHILAAAA|47227|13|7|7|PM|first|afternoon|lunch| +47228|AAAAAAAANHILAAAA|47228|13|7|8|PM|first|afternoon|lunch| +47229|AAAAAAAAOHILAAAA|47229|13|7|9|PM|first|afternoon|lunch| +47230|AAAAAAAAPHILAAAA|47230|13|7|10|PM|first|afternoon|lunch| +47231|AAAAAAAAAIILAAAA|47231|13|7|11|PM|first|afternoon|lunch| +47232|AAAAAAAABIILAAAA|47232|13|7|12|PM|first|afternoon|lunch| +47233|AAAAAAAACIILAAAA|47233|13|7|13|PM|first|afternoon|lunch| +47234|AAAAAAAADIILAAAA|47234|13|7|14|PM|first|afternoon|lunch| +47235|AAAAAAAAEIILAAAA|47235|13|7|15|PM|first|afternoon|lunch| +47236|AAAAAAAAFIILAAAA|47236|13|7|16|PM|first|afternoon|lunch| +47237|AAAAAAAAGIILAAAA|47237|13|7|17|PM|first|afternoon|lunch| +47238|AAAAAAAAHIILAAAA|47238|13|7|18|PM|first|afternoon|lunch| +47239|AAAAAAAAIIILAAAA|47239|13|7|19|PM|first|afternoon|lunch| +47240|AAAAAAAAJIILAAAA|47240|13|7|20|PM|first|afternoon|lunch| +47241|AAAAAAAAKIILAAAA|47241|13|7|21|PM|first|afternoon|lunch| +47242|AAAAAAAALIILAAAA|47242|13|7|22|PM|first|afternoon|lunch| +47243|AAAAAAAAMIILAAAA|47243|13|7|23|PM|first|afternoon|lunch| +47244|AAAAAAAANIILAAAA|47244|13|7|24|PM|first|afternoon|lunch| +47245|AAAAAAAAOIILAAAA|47245|13|7|25|PM|first|afternoon|lunch| +47246|AAAAAAAAPIILAAAA|47246|13|7|26|PM|first|afternoon|lunch| +47247|AAAAAAAAAJILAAAA|47247|13|7|27|PM|first|afternoon|lunch| +47248|AAAAAAAABJILAAAA|47248|13|7|28|PM|first|afternoon|lunch| +47249|AAAAAAAACJILAAAA|47249|13|7|29|PM|first|afternoon|lunch| +47250|AAAAAAAADJILAAAA|47250|13|7|30|PM|first|afternoon|lunch| +47251|AAAAAAAAEJILAAAA|47251|13|7|31|PM|first|afternoon|lunch| +47252|AAAAAAAAFJILAAAA|47252|13|7|32|PM|first|afternoon|lunch| +47253|AAAAAAAAGJILAAAA|47253|13|7|33|PM|first|afternoon|lunch| +47254|AAAAAAAAHJILAAAA|47254|13|7|34|PM|first|afternoon|lunch| +47255|AAAAAAAAIJILAAAA|47255|13|7|35|PM|first|afternoon|lunch| +47256|AAAAAAAAJJILAAAA|47256|13|7|36|PM|first|afternoon|lunch| +47257|AAAAAAAAKJILAAAA|47257|13|7|37|PM|first|afternoon|lunch| +47258|AAAAAAAALJILAAAA|47258|13|7|38|PM|first|afternoon|lunch| +47259|AAAAAAAAMJILAAAA|47259|13|7|39|PM|first|afternoon|lunch| +47260|AAAAAAAANJILAAAA|47260|13|7|40|PM|first|afternoon|lunch| +47261|AAAAAAAAOJILAAAA|47261|13|7|41|PM|first|afternoon|lunch| +47262|AAAAAAAAPJILAAAA|47262|13|7|42|PM|first|afternoon|lunch| +47263|AAAAAAAAAKILAAAA|47263|13|7|43|PM|first|afternoon|lunch| +47264|AAAAAAAABKILAAAA|47264|13|7|44|PM|first|afternoon|lunch| +47265|AAAAAAAACKILAAAA|47265|13|7|45|PM|first|afternoon|lunch| +47266|AAAAAAAADKILAAAA|47266|13|7|46|PM|first|afternoon|lunch| +47267|AAAAAAAAEKILAAAA|47267|13|7|47|PM|first|afternoon|lunch| +47268|AAAAAAAAFKILAAAA|47268|13|7|48|PM|first|afternoon|lunch| +47269|AAAAAAAAGKILAAAA|47269|13|7|49|PM|first|afternoon|lunch| +47270|AAAAAAAAHKILAAAA|47270|13|7|50|PM|first|afternoon|lunch| +47271|AAAAAAAAIKILAAAA|47271|13|7|51|PM|first|afternoon|lunch| +47272|AAAAAAAAJKILAAAA|47272|13|7|52|PM|first|afternoon|lunch| +47273|AAAAAAAAKKILAAAA|47273|13|7|53|PM|first|afternoon|lunch| +47274|AAAAAAAALKILAAAA|47274|13|7|54|PM|first|afternoon|lunch| +47275|AAAAAAAAMKILAAAA|47275|13|7|55|PM|first|afternoon|lunch| +47276|AAAAAAAANKILAAAA|47276|13|7|56|PM|first|afternoon|lunch| +47277|AAAAAAAAOKILAAAA|47277|13|7|57|PM|first|afternoon|lunch| +47278|AAAAAAAAPKILAAAA|47278|13|7|58|PM|first|afternoon|lunch| +47279|AAAAAAAAALILAAAA|47279|13|7|59|PM|first|afternoon|lunch| +47280|AAAAAAAABLILAAAA|47280|13|8|0|PM|first|afternoon|lunch| +47281|AAAAAAAACLILAAAA|47281|13|8|1|PM|first|afternoon|lunch| +47282|AAAAAAAADLILAAAA|47282|13|8|2|PM|first|afternoon|lunch| +47283|AAAAAAAAELILAAAA|47283|13|8|3|PM|first|afternoon|lunch| +47284|AAAAAAAAFLILAAAA|47284|13|8|4|PM|first|afternoon|lunch| +47285|AAAAAAAAGLILAAAA|47285|13|8|5|PM|first|afternoon|lunch| +47286|AAAAAAAAHLILAAAA|47286|13|8|6|PM|first|afternoon|lunch| +47287|AAAAAAAAILILAAAA|47287|13|8|7|PM|first|afternoon|lunch| +47288|AAAAAAAAJLILAAAA|47288|13|8|8|PM|first|afternoon|lunch| +47289|AAAAAAAAKLILAAAA|47289|13|8|9|PM|first|afternoon|lunch| +47290|AAAAAAAALLILAAAA|47290|13|8|10|PM|first|afternoon|lunch| +47291|AAAAAAAAMLILAAAA|47291|13|8|11|PM|first|afternoon|lunch| +47292|AAAAAAAANLILAAAA|47292|13|8|12|PM|first|afternoon|lunch| +47293|AAAAAAAAOLILAAAA|47293|13|8|13|PM|first|afternoon|lunch| +47294|AAAAAAAAPLILAAAA|47294|13|8|14|PM|first|afternoon|lunch| +47295|AAAAAAAAAMILAAAA|47295|13|8|15|PM|first|afternoon|lunch| +47296|AAAAAAAABMILAAAA|47296|13|8|16|PM|first|afternoon|lunch| +47297|AAAAAAAACMILAAAA|47297|13|8|17|PM|first|afternoon|lunch| +47298|AAAAAAAADMILAAAA|47298|13|8|18|PM|first|afternoon|lunch| +47299|AAAAAAAAEMILAAAA|47299|13|8|19|PM|first|afternoon|lunch| +47300|AAAAAAAAFMILAAAA|47300|13|8|20|PM|first|afternoon|lunch| +47301|AAAAAAAAGMILAAAA|47301|13|8|21|PM|first|afternoon|lunch| +47302|AAAAAAAAHMILAAAA|47302|13|8|22|PM|first|afternoon|lunch| +47303|AAAAAAAAIMILAAAA|47303|13|8|23|PM|first|afternoon|lunch| +47304|AAAAAAAAJMILAAAA|47304|13|8|24|PM|first|afternoon|lunch| +47305|AAAAAAAAKMILAAAA|47305|13|8|25|PM|first|afternoon|lunch| +47306|AAAAAAAALMILAAAA|47306|13|8|26|PM|first|afternoon|lunch| +47307|AAAAAAAAMMILAAAA|47307|13|8|27|PM|first|afternoon|lunch| +47308|AAAAAAAANMILAAAA|47308|13|8|28|PM|first|afternoon|lunch| +47309|AAAAAAAAOMILAAAA|47309|13|8|29|PM|first|afternoon|lunch| +47310|AAAAAAAAPMILAAAA|47310|13|8|30|PM|first|afternoon|lunch| +47311|AAAAAAAAANILAAAA|47311|13|8|31|PM|first|afternoon|lunch| +47312|AAAAAAAABNILAAAA|47312|13|8|32|PM|first|afternoon|lunch| +47313|AAAAAAAACNILAAAA|47313|13|8|33|PM|first|afternoon|lunch| +47314|AAAAAAAADNILAAAA|47314|13|8|34|PM|first|afternoon|lunch| +47315|AAAAAAAAENILAAAA|47315|13|8|35|PM|first|afternoon|lunch| +47316|AAAAAAAAFNILAAAA|47316|13|8|36|PM|first|afternoon|lunch| +47317|AAAAAAAAGNILAAAA|47317|13|8|37|PM|first|afternoon|lunch| +47318|AAAAAAAAHNILAAAA|47318|13|8|38|PM|first|afternoon|lunch| +47319|AAAAAAAAINILAAAA|47319|13|8|39|PM|first|afternoon|lunch| +47320|AAAAAAAAJNILAAAA|47320|13|8|40|PM|first|afternoon|lunch| +47321|AAAAAAAAKNILAAAA|47321|13|8|41|PM|first|afternoon|lunch| +47322|AAAAAAAALNILAAAA|47322|13|8|42|PM|first|afternoon|lunch| +47323|AAAAAAAAMNILAAAA|47323|13|8|43|PM|first|afternoon|lunch| +47324|AAAAAAAANNILAAAA|47324|13|8|44|PM|first|afternoon|lunch| +47325|AAAAAAAAONILAAAA|47325|13|8|45|PM|first|afternoon|lunch| +47326|AAAAAAAAPNILAAAA|47326|13|8|46|PM|first|afternoon|lunch| +47327|AAAAAAAAAOILAAAA|47327|13|8|47|PM|first|afternoon|lunch| +47328|AAAAAAAABOILAAAA|47328|13|8|48|PM|first|afternoon|lunch| +47329|AAAAAAAACOILAAAA|47329|13|8|49|PM|first|afternoon|lunch| +47330|AAAAAAAADOILAAAA|47330|13|8|50|PM|first|afternoon|lunch| +47331|AAAAAAAAEOILAAAA|47331|13|8|51|PM|first|afternoon|lunch| +47332|AAAAAAAAFOILAAAA|47332|13|8|52|PM|first|afternoon|lunch| +47333|AAAAAAAAGOILAAAA|47333|13|8|53|PM|first|afternoon|lunch| +47334|AAAAAAAAHOILAAAA|47334|13|8|54|PM|first|afternoon|lunch| +47335|AAAAAAAAIOILAAAA|47335|13|8|55|PM|first|afternoon|lunch| +47336|AAAAAAAAJOILAAAA|47336|13|8|56|PM|first|afternoon|lunch| +47337|AAAAAAAAKOILAAAA|47337|13|8|57|PM|first|afternoon|lunch| +47338|AAAAAAAALOILAAAA|47338|13|8|58|PM|first|afternoon|lunch| +47339|AAAAAAAAMOILAAAA|47339|13|8|59|PM|first|afternoon|lunch| +47340|AAAAAAAANOILAAAA|47340|13|9|0|PM|first|afternoon|lunch| +47341|AAAAAAAAOOILAAAA|47341|13|9|1|PM|first|afternoon|lunch| +47342|AAAAAAAAPOILAAAA|47342|13|9|2|PM|first|afternoon|lunch| +47343|AAAAAAAAAPILAAAA|47343|13|9|3|PM|first|afternoon|lunch| +47344|AAAAAAAABPILAAAA|47344|13|9|4|PM|first|afternoon|lunch| +47345|AAAAAAAACPILAAAA|47345|13|9|5|PM|first|afternoon|lunch| +47346|AAAAAAAADPILAAAA|47346|13|9|6|PM|first|afternoon|lunch| +47347|AAAAAAAAEPILAAAA|47347|13|9|7|PM|first|afternoon|lunch| +47348|AAAAAAAAFPILAAAA|47348|13|9|8|PM|first|afternoon|lunch| +47349|AAAAAAAAGPILAAAA|47349|13|9|9|PM|first|afternoon|lunch| +47350|AAAAAAAAHPILAAAA|47350|13|9|10|PM|first|afternoon|lunch| +47351|AAAAAAAAIPILAAAA|47351|13|9|11|PM|first|afternoon|lunch| +47352|AAAAAAAAJPILAAAA|47352|13|9|12|PM|first|afternoon|lunch| +47353|AAAAAAAAKPILAAAA|47353|13|9|13|PM|first|afternoon|lunch| +47354|AAAAAAAALPILAAAA|47354|13|9|14|PM|first|afternoon|lunch| +47355|AAAAAAAAMPILAAAA|47355|13|9|15|PM|first|afternoon|lunch| +47356|AAAAAAAANPILAAAA|47356|13|9|16|PM|first|afternoon|lunch| +47357|AAAAAAAAOPILAAAA|47357|13|9|17|PM|first|afternoon|lunch| +47358|AAAAAAAAPPILAAAA|47358|13|9|18|PM|first|afternoon|lunch| +47359|AAAAAAAAAAJLAAAA|47359|13|9|19|PM|first|afternoon|lunch| +47360|AAAAAAAABAJLAAAA|47360|13|9|20|PM|first|afternoon|lunch| +47361|AAAAAAAACAJLAAAA|47361|13|9|21|PM|first|afternoon|lunch| +47362|AAAAAAAADAJLAAAA|47362|13|9|22|PM|first|afternoon|lunch| +47363|AAAAAAAAEAJLAAAA|47363|13|9|23|PM|first|afternoon|lunch| +47364|AAAAAAAAFAJLAAAA|47364|13|9|24|PM|first|afternoon|lunch| +47365|AAAAAAAAGAJLAAAA|47365|13|9|25|PM|first|afternoon|lunch| +47366|AAAAAAAAHAJLAAAA|47366|13|9|26|PM|first|afternoon|lunch| +47367|AAAAAAAAIAJLAAAA|47367|13|9|27|PM|first|afternoon|lunch| +47368|AAAAAAAAJAJLAAAA|47368|13|9|28|PM|first|afternoon|lunch| +47369|AAAAAAAAKAJLAAAA|47369|13|9|29|PM|first|afternoon|lunch| +47370|AAAAAAAALAJLAAAA|47370|13|9|30|PM|first|afternoon|lunch| +47371|AAAAAAAAMAJLAAAA|47371|13|9|31|PM|first|afternoon|lunch| +47372|AAAAAAAANAJLAAAA|47372|13|9|32|PM|first|afternoon|lunch| +47373|AAAAAAAAOAJLAAAA|47373|13|9|33|PM|first|afternoon|lunch| +47374|AAAAAAAAPAJLAAAA|47374|13|9|34|PM|first|afternoon|lunch| +47375|AAAAAAAAABJLAAAA|47375|13|9|35|PM|first|afternoon|lunch| +47376|AAAAAAAABBJLAAAA|47376|13|9|36|PM|first|afternoon|lunch| +47377|AAAAAAAACBJLAAAA|47377|13|9|37|PM|first|afternoon|lunch| +47378|AAAAAAAADBJLAAAA|47378|13|9|38|PM|first|afternoon|lunch| +47379|AAAAAAAAEBJLAAAA|47379|13|9|39|PM|first|afternoon|lunch| +47380|AAAAAAAAFBJLAAAA|47380|13|9|40|PM|first|afternoon|lunch| +47381|AAAAAAAAGBJLAAAA|47381|13|9|41|PM|first|afternoon|lunch| +47382|AAAAAAAAHBJLAAAA|47382|13|9|42|PM|first|afternoon|lunch| +47383|AAAAAAAAIBJLAAAA|47383|13|9|43|PM|first|afternoon|lunch| +47384|AAAAAAAAJBJLAAAA|47384|13|9|44|PM|first|afternoon|lunch| +47385|AAAAAAAAKBJLAAAA|47385|13|9|45|PM|first|afternoon|lunch| +47386|AAAAAAAALBJLAAAA|47386|13|9|46|PM|first|afternoon|lunch| +47387|AAAAAAAAMBJLAAAA|47387|13|9|47|PM|first|afternoon|lunch| +47388|AAAAAAAANBJLAAAA|47388|13|9|48|PM|first|afternoon|lunch| +47389|AAAAAAAAOBJLAAAA|47389|13|9|49|PM|first|afternoon|lunch| +47390|AAAAAAAAPBJLAAAA|47390|13|9|50|PM|first|afternoon|lunch| +47391|AAAAAAAAACJLAAAA|47391|13|9|51|PM|first|afternoon|lunch| +47392|AAAAAAAABCJLAAAA|47392|13|9|52|PM|first|afternoon|lunch| +47393|AAAAAAAACCJLAAAA|47393|13|9|53|PM|first|afternoon|lunch| +47394|AAAAAAAADCJLAAAA|47394|13|9|54|PM|first|afternoon|lunch| +47395|AAAAAAAAECJLAAAA|47395|13|9|55|PM|first|afternoon|lunch| +47396|AAAAAAAAFCJLAAAA|47396|13|9|56|PM|first|afternoon|lunch| +47397|AAAAAAAAGCJLAAAA|47397|13|9|57|PM|first|afternoon|lunch| +47398|AAAAAAAAHCJLAAAA|47398|13|9|58|PM|first|afternoon|lunch| +47399|AAAAAAAAICJLAAAA|47399|13|9|59|PM|first|afternoon|lunch| +47400|AAAAAAAAJCJLAAAA|47400|13|10|0|PM|first|afternoon|lunch| +47401|AAAAAAAAKCJLAAAA|47401|13|10|1|PM|first|afternoon|lunch| +47402|AAAAAAAALCJLAAAA|47402|13|10|2|PM|first|afternoon|lunch| +47403|AAAAAAAAMCJLAAAA|47403|13|10|3|PM|first|afternoon|lunch| +47404|AAAAAAAANCJLAAAA|47404|13|10|4|PM|first|afternoon|lunch| +47405|AAAAAAAAOCJLAAAA|47405|13|10|5|PM|first|afternoon|lunch| +47406|AAAAAAAAPCJLAAAA|47406|13|10|6|PM|first|afternoon|lunch| +47407|AAAAAAAAADJLAAAA|47407|13|10|7|PM|first|afternoon|lunch| +47408|AAAAAAAABDJLAAAA|47408|13|10|8|PM|first|afternoon|lunch| +47409|AAAAAAAACDJLAAAA|47409|13|10|9|PM|first|afternoon|lunch| +47410|AAAAAAAADDJLAAAA|47410|13|10|10|PM|first|afternoon|lunch| +47411|AAAAAAAAEDJLAAAA|47411|13|10|11|PM|first|afternoon|lunch| +47412|AAAAAAAAFDJLAAAA|47412|13|10|12|PM|first|afternoon|lunch| +47413|AAAAAAAAGDJLAAAA|47413|13|10|13|PM|first|afternoon|lunch| +47414|AAAAAAAAHDJLAAAA|47414|13|10|14|PM|first|afternoon|lunch| +47415|AAAAAAAAIDJLAAAA|47415|13|10|15|PM|first|afternoon|lunch| +47416|AAAAAAAAJDJLAAAA|47416|13|10|16|PM|first|afternoon|lunch| +47417|AAAAAAAAKDJLAAAA|47417|13|10|17|PM|first|afternoon|lunch| +47418|AAAAAAAALDJLAAAA|47418|13|10|18|PM|first|afternoon|lunch| +47419|AAAAAAAAMDJLAAAA|47419|13|10|19|PM|first|afternoon|lunch| +47420|AAAAAAAANDJLAAAA|47420|13|10|20|PM|first|afternoon|lunch| +47421|AAAAAAAAODJLAAAA|47421|13|10|21|PM|first|afternoon|lunch| +47422|AAAAAAAAPDJLAAAA|47422|13|10|22|PM|first|afternoon|lunch| +47423|AAAAAAAAAEJLAAAA|47423|13|10|23|PM|first|afternoon|lunch| +47424|AAAAAAAABEJLAAAA|47424|13|10|24|PM|first|afternoon|lunch| +47425|AAAAAAAACEJLAAAA|47425|13|10|25|PM|first|afternoon|lunch| +47426|AAAAAAAADEJLAAAA|47426|13|10|26|PM|first|afternoon|lunch| +47427|AAAAAAAAEEJLAAAA|47427|13|10|27|PM|first|afternoon|lunch| +47428|AAAAAAAAFEJLAAAA|47428|13|10|28|PM|first|afternoon|lunch| +47429|AAAAAAAAGEJLAAAA|47429|13|10|29|PM|first|afternoon|lunch| +47430|AAAAAAAAHEJLAAAA|47430|13|10|30|PM|first|afternoon|lunch| +47431|AAAAAAAAIEJLAAAA|47431|13|10|31|PM|first|afternoon|lunch| +47432|AAAAAAAAJEJLAAAA|47432|13|10|32|PM|first|afternoon|lunch| +47433|AAAAAAAAKEJLAAAA|47433|13|10|33|PM|first|afternoon|lunch| +47434|AAAAAAAALEJLAAAA|47434|13|10|34|PM|first|afternoon|lunch| +47435|AAAAAAAAMEJLAAAA|47435|13|10|35|PM|first|afternoon|lunch| +47436|AAAAAAAANEJLAAAA|47436|13|10|36|PM|first|afternoon|lunch| +47437|AAAAAAAAOEJLAAAA|47437|13|10|37|PM|first|afternoon|lunch| +47438|AAAAAAAAPEJLAAAA|47438|13|10|38|PM|first|afternoon|lunch| +47439|AAAAAAAAAFJLAAAA|47439|13|10|39|PM|first|afternoon|lunch| +47440|AAAAAAAABFJLAAAA|47440|13|10|40|PM|first|afternoon|lunch| +47441|AAAAAAAACFJLAAAA|47441|13|10|41|PM|first|afternoon|lunch| +47442|AAAAAAAADFJLAAAA|47442|13|10|42|PM|first|afternoon|lunch| +47443|AAAAAAAAEFJLAAAA|47443|13|10|43|PM|first|afternoon|lunch| +47444|AAAAAAAAFFJLAAAA|47444|13|10|44|PM|first|afternoon|lunch| +47445|AAAAAAAAGFJLAAAA|47445|13|10|45|PM|first|afternoon|lunch| +47446|AAAAAAAAHFJLAAAA|47446|13|10|46|PM|first|afternoon|lunch| +47447|AAAAAAAAIFJLAAAA|47447|13|10|47|PM|first|afternoon|lunch| +47448|AAAAAAAAJFJLAAAA|47448|13|10|48|PM|first|afternoon|lunch| +47449|AAAAAAAAKFJLAAAA|47449|13|10|49|PM|first|afternoon|lunch| +47450|AAAAAAAALFJLAAAA|47450|13|10|50|PM|first|afternoon|lunch| +47451|AAAAAAAAMFJLAAAA|47451|13|10|51|PM|first|afternoon|lunch| +47452|AAAAAAAANFJLAAAA|47452|13|10|52|PM|first|afternoon|lunch| +47453|AAAAAAAAOFJLAAAA|47453|13|10|53|PM|first|afternoon|lunch| +47454|AAAAAAAAPFJLAAAA|47454|13|10|54|PM|first|afternoon|lunch| +47455|AAAAAAAAAGJLAAAA|47455|13|10|55|PM|first|afternoon|lunch| +47456|AAAAAAAABGJLAAAA|47456|13|10|56|PM|first|afternoon|lunch| +47457|AAAAAAAACGJLAAAA|47457|13|10|57|PM|first|afternoon|lunch| +47458|AAAAAAAADGJLAAAA|47458|13|10|58|PM|first|afternoon|lunch| +47459|AAAAAAAAEGJLAAAA|47459|13|10|59|PM|first|afternoon|lunch| +47460|AAAAAAAAFGJLAAAA|47460|13|11|0|PM|first|afternoon|lunch| +47461|AAAAAAAAGGJLAAAA|47461|13|11|1|PM|first|afternoon|lunch| +47462|AAAAAAAAHGJLAAAA|47462|13|11|2|PM|first|afternoon|lunch| +47463|AAAAAAAAIGJLAAAA|47463|13|11|3|PM|first|afternoon|lunch| +47464|AAAAAAAAJGJLAAAA|47464|13|11|4|PM|first|afternoon|lunch| +47465|AAAAAAAAKGJLAAAA|47465|13|11|5|PM|first|afternoon|lunch| +47466|AAAAAAAALGJLAAAA|47466|13|11|6|PM|first|afternoon|lunch| +47467|AAAAAAAAMGJLAAAA|47467|13|11|7|PM|first|afternoon|lunch| +47468|AAAAAAAANGJLAAAA|47468|13|11|8|PM|first|afternoon|lunch| +47469|AAAAAAAAOGJLAAAA|47469|13|11|9|PM|first|afternoon|lunch| +47470|AAAAAAAAPGJLAAAA|47470|13|11|10|PM|first|afternoon|lunch| +47471|AAAAAAAAAHJLAAAA|47471|13|11|11|PM|first|afternoon|lunch| +47472|AAAAAAAABHJLAAAA|47472|13|11|12|PM|first|afternoon|lunch| +47473|AAAAAAAACHJLAAAA|47473|13|11|13|PM|first|afternoon|lunch| +47474|AAAAAAAADHJLAAAA|47474|13|11|14|PM|first|afternoon|lunch| +47475|AAAAAAAAEHJLAAAA|47475|13|11|15|PM|first|afternoon|lunch| +47476|AAAAAAAAFHJLAAAA|47476|13|11|16|PM|first|afternoon|lunch| +47477|AAAAAAAAGHJLAAAA|47477|13|11|17|PM|first|afternoon|lunch| +47478|AAAAAAAAHHJLAAAA|47478|13|11|18|PM|first|afternoon|lunch| +47479|AAAAAAAAIHJLAAAA|47479|13|11|19|PM|first|afternoon|lunch| +47480|AAAAAAAAJHJLAAAA|47480|13|11|20|PM|first|afternoon|lunch| +47481|AAAAAAAAKHJLAAAA|47481|13|11|21|PM|first|afternoon|lunch| +47482|AAAAAAAALHJLAAAA|47482|13|11|22|PM|first|afternoon|lunch| +47483|AAAAAAAAMHJLAAAA|47483|13|11|23|PM|first|afternoon|lunch| +47484|AAAAAAAANHJLAAAA|47484|13|11|24|PM|first|afternoon|lunch| +47485|AAAAAAAAOHJLAAAA|47485|13|11|25|PM|first|afternoon|lunch| +47486|AAAAAAAAPHJLAAAA|47486|13|11|26|PM|first|afternoon|lunch| +47487|AAAAAAAAAIJLAAAA|47487|13|11|27|PM|first|afternoon|lunch| +47488|AAAAAAAABIJLAAAA|47488|13|11|28|PM|first|afternoon|lunch| +47489|AAAAAAAACIJLAAAA|47489|13|11|29|PM|first|afternoon|lunch| +47490|AAAAAAAADIJLAAAA|47490|13|11|30|PM|first|afternoon|lunch| +47491|AAAAAAAAEIJLAAAA|47491|13|11|31|PM|first|afternoon|lunch| +47492|AAAAAAAAFIJLAAAA|47492|13|11|32|PM|first|afternoon|lunch| +47493|AAAAAAAAGIJLAAAA|47493|13|11|33|PM|first|afternoon|lunch| +47494|AAAAAAAAHIJLAAAA|47494|13|11|34|PM|first|afternoon|lunch| +47495|AAAAAAAAIIJLAAAA|47495|13|11|35|PM|first|afternoon|lunch| +47496|AAAAAAAAJIJLAAAA|47496|13|11|36|PM|first|afternoon|lunch| +47497|AAAAAAAAKIJLAAAA|47497|13|11|37|PM|first|afternoon|lunch| +47498|AAAAAAAALIJLAAAA|47498|13|11|38|PM|first|afternoon|lunch| +47499|AAAAAAAAMIJLAAAA|47499|13|11|39|PM|first|afternoon|lunch| +47500|AAAAAAAANIJLAAAA|47500|13|11|40|PM|first|afternoon|lunch| +47501|AAAAAAAAOIJLAAAA|47501|13|11|41|PM|first|afternoon|lunch| +47502|AAAAAAAAPIJLAAAA|47502|13|11|42|PM|first|afternoon|lunch| +47503|AAAAAAAAAJJLAAAA|47503|13|11|43|PM|first|afternoon|lunch| +47504|AAAAAAAABJJLAAAA|47504|13|11|44|PM|first|afternoon|lunch| +47505|AAAAAAAACJJLAAAA|47505|13|11|45|PM|first|afternoon|lunch| +47506|AAAAAAAADJJLAAAA|47506|13|11|46|PM|first|afternoon|lunch| +47507|AAAAAAAAEJJLAAAA|47507|13|11|47|PM|first|afternoon|lunch| +47508|AAAAAAAAFJJLAAAA|47508|13|11|48|PM|first|afternoon|lunch| +47509|AAAAAAAAGJJLAAAA|47509|13|11|49|PM|first|afternoon|lunch| +47510|AAAAAAAAHJJLAAAA|47510|13|11|50|PM|first|afternoon|lunch| +47511|AAAAAAAAIJJLAAAA|47511|13|11|51|PM|first|afternoon|lunch| +47512|AAAAAAAAJJJLAAAA|47512|13|11|52|PM|first|afternoon|lunch| +47513|AAAAAAAAKJJLAAAA|47513|13|11|53|PM|first|afternoon|lunch| +47514|AAAAAAAALJJLAAAA|47514|13|11|54|PM|first|afternoon|lunch| +47515|AAAAAAAAMJJLAAAA|47515|13|11|55|PM|first|afternoon|lunch| +47516|AAAAAAAANJJLAAAA|47516|13|11|56|PM|first|afternoon|lunch| +47517|AAAAAAAAOJJLAAAA|47517|13|11|57|PM|first|afternoon|lunch| +47518|AAAAAAAAPJJLAAAA|47518|13|11|58|PM|first|afternoon|lunch| +47519|AAAAAAAAAKJLAAAA|47519|13|11|59|PM|first|afternoon|lunch| +47520|AAAAAAAABKJLAAAA|47520|13|12|0|PM|first|afternoon|lunch| +47521|AAAAAAAACKJLAAAA|47521|13|12|1|PM|first|afternoon|lunch| +47522|AAAAAAAADKJLAAAA|47522|13|12|2|PM|first|afternoon|lunch| +47523|AAAAAAAAEKJLAAAA|47523|13|12|3|PM|first|afternoon|lunch| +47524|AAAAAAAAFKJLAAAA|47524|13|12|4|PM|first|afternoon|lunch| +47525|AAAAAAAAGKJLAAAA|47525|13|12|5|PM|first|afternoon|lunch| +47526|AAAAAAAAHKJLAAAA|47526|13|12|6|PM|first|afternoon|lunch| +47527|AAAAAAAAIKJLAAAA|47527|13|12|7|PM|first|afternoon|lunch| +47528|AAAAAAAAJKJLAAAA|47528|13|12|8|PM|first|afternoon|lunch| +47529|AAAAAAAAKKJLAAAA|47529|13|12|9|PM|first|afternoon|lunch| +47530|AAAAAAAALKJLAAAA|47530|13|12|10|PM|first|afternoon|lunch| +47531|AAAAAAAAMKJLAAAA|47531|13|12|11|PM|first|afternoon|lunch| +47532|AAAAAAAANKJLAAAA|47532|13|12|12|PM|first|afternoon|lunch| +47533|AAAAAAAAOKJLAAAA|47533|13|12|13|PM|first|afternoon|lunch| +47534|AAAAAAAAPKJLAAAA|47534|13|12|14|PM|first|afternoon|lunch| +47535|AAAAAAAAALJLAAAA|47535|13|12|15|PM|first|afternoon|lunch| +47536|AAAAAAAABLJLAAAA|47536|13|12|16|PM|first|afternoon|lunch| +47537|AAAAAAAACLJLAAAA|47537|13|12|17|PM|first|afternoon|lunch| +47538|AAAAAAAADLJLAAAA|47538|13|12|18|PM|first|afternoon|lunch| +47539|AAAAAAAAELJLAAAA|47539|13|12|19|PM|first|afternoon|lunch| +47540|AAAAAAAAFLJLAAAA|47540|13|12|20|PM|first|afternoon|lunch| +47541|AAAAAAAAGLJLAAAA|47541|13|12|21|PM|first|afternoon|lunch| +47542|AAAAAAAAHLJLAAAA|47542|13|12|22|PM|first|afternoon|lunch| +47543|AAAAAAAAILJLAAAA|47543|13|12|23|PM|first|afternoon|lunch| +47544|AAAAAAAAJLJLAAAA|47544|13|12|24|PM|first|afternoon|lunch| +47545|AAAAAAAAKLJLAAAA|47545|13|12|25|PM|first|afternoon|lunch| +47546|AAAAAAAALLJLAAAA|47546|13|12|26|PM|first|afternoon|lunch| +47547|AAAAAAAAMLJLAAAA|47547|13|12|27|PM|first|afternoon|lunch| +47548|AAAAAAAANLJLAAAA|47548|13|12|28|PM|first|afternoon|lunch| +47549|AAAAAAAAOLJLAAAA|47549|13|12|29|PM|first|afternoon|lunch| +47550|AAAAAAAAPLJLAAAA|47550|13|12|30|PM|first|afternoon|lunch| +47551|AAAAAAAAAMJLAAAA|47551|13|12|31|PM|first|afternoon|lunch| +47552|AAAAAAAABMJLAAAA|47552|13|12|32|PM|first|afternoon|lunch| +47553|AAAAAAAACMJLAAAA|47553|13|12|33|PM|first|afternoon|lunch| +47554|AAAAAAAADMJLAAAA|47554|13|12|34|PM|first|afternoon|lunch| +47555|AAAAAAAAEMJLAAAA|47555|13|12|35|PM|first|afternoon|lunch| +47556|AAAAAAAAFMJLAAAA|47556|13|12|36|PM|first|afternoon|lunch| +47557|AAAAAAAAGMJLAAAA|47557|13|12|37|PM|first|afternoon|lunch| +47558|AAAAAAAAHMJLAAAA|47558|13|12|38|PM|first|afternoon|lunch| +47559|AAAAAAAAIMJLAAAA|47559|13|12|39|PM|first|afternoon|lunch| +47560|AAAAAAAAJMJLAAAA|47560|13|12|40|PM|first|afternoon|lunch| +47561|AAAAAAAAKMJLAAAA|47561|13|12|41|PM|first|afternoon|lunch| +47562|AAAAAAAALMJLAAAA|47562|13|12|42|PM|first|afternoon|lunch| +47563|AAAAAAAAMMJLAAAA|47563|13|12|43|PM|first|afternoon|lunch| +47564|AAAAAAAANMJLAAAA|47564|13|12|44|PM|first|afternoon|lunch| +47565|AAAAAAAAOMJLAAAA|47565|13|12|45|PM|first|afternoon|lunch| +47566|AAAAAAAAPMJLAAAA|47566|13|12|46|PM|first|afternoon|lunch| +47567|AAAAAAAAANJLAAAA|47567|13|12|47|PM|first|afternoon|lunch| +47568|AAAAAAAABNJLAAAA|47568|13|12|48|PM|first|afternoon|lunch| +47569|AAAAAAAACNJLAAAA|47569|13|12|49|PM|first|afternoon|lunch| +47570|AAAAAAAADNJLAAAA|47570|13|12|50|PM|first|afternoon|lunch| +47571|AAAAAAAAENJLAAAA|47571|13|12|51|PM|first|afternoon|lunch| +47572|AAAAAAAAFNJLAAAA|47572|13|12|52|PM|first|afternoon|lunch| +47573|AAAAAAAAGNJLAAAA|47573|13|12|53|PM|first|afternoon|lunch| +47574|AAAAAAAAHNJLAAAA|47574|13|12|54|PM|first|afternoon|lunch| +47575|AAAAAAAAINJLAAAA|47575|13|12|55|PM|first|afternoon|lunch| +47576|AAAAAAAAJNJLAAAA|47576|13|12|56|PM|first|afternoon|lunch| +47577|AAAAAAAAKNJLAAAA|47577|13|12|57|PM|first|afternoon|lunch| +47578|AAAAAAAALNJLAAAA|47578|13|12|58|PM|first|afternoon|lunch| +47579|AAAAAAAAMNJLAAAA|47579|13|12|59|PM|first|afternoon|lunch| +47580|AAAAAAAANNJLAAAA|47580|13|13|0|PM|first|afternoon|lunch| +47581|AAAAAAAAONJLAAAA|47581|13|13|1|PM|first|afternoon|lunch| +47582|AAAAAAAAPNJLAAAA|47582|13|13|2|PM|first|afternoon|lunch| +47583|AAAAAAAAAOJLAAAA|47583|13|13|3|PM|first|afternoon|lunch| +47584|AAAAAAAABOJLAAAA|47584|13|13|4|PM|first|afternoon|lunch| +47585|AAAAAAAACOJLAAAA|47585|13|13|5|PM|first|afternoon|lunch| +47586|AAAAAAAADOJLAAAA|47586|13|13|6|PM|first|afternoon|lunch| +47587|AAAAAAAAEOJLAAAA|47587|13|13|7|PM|first|afternoon|lunch| +47588|AAAAAAAAFOJLAAAA|47588|13|13|8|PM|first|afternoon|lunch| +47589|AAAAAAAAGOJLAAAA|47589|13|13|9|PM|first|afternoon|lunch| +47590|AAAAAAAAHOJLAAAA|47590|13|13|10|PM|first|afternoon|lunch| +47591|AAAAAAAAIOJLAAAA|47591|13|13|11|PM|first|afternoon|lunch| +47592|AAAAAAAAJOJLAAAA|47592|13|13|12|PM|first|afternoon|lunch| +47593|AAAAAAAAKOJLAAAA|47593|13|13|13|PM|first|afternoon|lunch| +47594|AAAAAAAALOJLAAAA|47594|13|13|14|PM|first|afternoon|lunch| +47595|AAAAAAAAMOJLAAAA|47595|13|13|15|PM|first|afternoon|lunch| +47596|AAAAAAAANOJLAAAA|47596|13|13|16|PM|first|afternoon|lunch| +47597|AAAAAAAAOOJLAAAA|47597|13|13|17|PM|first|afternoon|lunch| +47598|AAAAAAAAPOJLAAAA|47598|13|13|18|PM|first|afternoon|lunch| +47599|AAAAAAAAAPJLAAAA|47599|13|13|19|PM|first|afternoon|lunch| +47600|AAAAAAAABPJLAAAA|47600|13|13|20|PM|first|afternoon|lunch| +47601|AAAAAAAACPJLAAAA|47601|13|13|21|PM|first|afternoon|lunch| +47602|AAAAAAAADPJLAAAA|47602|13|13|22|PM|first|afternoon|lunch| +47603|AAAAAAAAEPJLAAAA|47603|13|13|23|PM|first|afternoon|lunch| +47604|AAAAAAAAFPJLAAAA|47604|13|13|24|PM|first|afternoon|lunch| +47605|AAAAAAAAGPJLAAAA|47605|13|13|25|PM|first|afternoon|lunch| +47606|AAAAAAAAHPJLAAAA|47606|13|13|26|PM|first|afternoon|lunch| +47607|AAAAAAAAIPJLAAAA|47607|13|13|27|PM|first|afternoon|lunch| +47608|AAAAAAAAJPJLAAAA|47608|13|13|28|PM|first|afternoon|lunch| +47609|AAAAAAAAKPJLAAAA|47609|13|13|29|PM|first|afternoon|lunch| +47610|AAAAAAAALPJLAAAA|47610|13|13|30|PM|first|afternoon|lunch| +47611|AAAAAAAAMPJLAAAA|47611|13|13|31|PM|first|afternoon|lunch| +47612|AAAAAAAANPJLAAAA|47612|13|13|32|PM|first|afternoon|lunch| +47613|AAAAAAAAOPJLAAAA|47613|13|13|33|PM|first|afternoon|lunch| +47614|AAAAAAAAPPJLAAAA|47614|13|13|34|PM|first|afternoon|lunch| +47615|AAAAAAAAAAKLAAAA|47615|13|13|35|PM|first|afternoon|lunch| +47616|AAAAAAAABAKLAAAA|47616|13|13|36|PM|first|afternoon|lunch| +47617|AAAAAAAACAKLAAAA|47617|13|13|37|PM|first|afternoon|lunch| +47618|AAAAAAAADAKLAAAA|47618|13|13|38|PM|first|afternoon|lunch| +47619|AAAAAAAAEAKLAAAA|47619|13|13|39|PM|first|afternoon|lunch| +47620|AAAAAAAAFAKLAAAA|47620|13|13|40|PM|first|afternoon|lunch| +47621|AAAAAAAAGAKLAAAA|47621|13|13|41|PM|first|afternoon|lunch| +47622|AAAAAAAAHAKLAAAA|47622|13|13|42|PM|first|afternoon|lunch| +47623|AAAAAAAAIAKLAAAA|47623|13|13|43|PM|first|afternoon|lunch| +47624|AAAAAAAAJAKLAAAA|47624|13|13|44|PM|first|afternoon|lunch| +47625|AAAAAAAAKAKLAAAA|47625|13|13|45|PM|first|afternoon|lunch| +47626|AAAAAAAALAKLAAAA|47626|13|13|46|PM|first|afternoon|lunch| +47627|AAAAAAAAMAKLAAAA|47627|13|13|47|PM|first|afternoon|lunch| +47628|AAAAAAAANAKLAAAA|47628|13|13|48|PM|first|afternoon|lunch| +47629|AAAAAAAAOAKLAAAA|47629|13|13|49|PM|first|afternoon|lunch| +47630|AAAAAAAAPAKLAAAA|47630|13|13|50|PM|first|afternoon|lunch| +47631|AAAAAAAAABKLAAAA|47631|13|13|51|PM|first|afternoon|lunch| +47632|AAAAAAAABBKLAAAA|47632|13|13|52|PM|first|afternoon|lunch| +47633|AAAAAAAACBKLAAAA|47633|13|13|53|PM|first|afternoon|lunch| +47634|AAAAAAAADBKLAAAA|47634|13|13|54|PM|first|afternoon|lunch| +47635|AAAAAAAAEBKLAAAA|47635|13|13|55|PM|first|afternoon|lunch| +47636|AAAAAAAAFBKLAAAA|47636|13|13|56|PM|first|afternoon|lunch| +47637|AAAAAAAAGBKLAAAA|47637|13|13|57|PM|first|afternoon|lunch| +47638|AAAAAAAAHBKLAAAA|47638|13|13|58|PM|first|afternoon|lunch| +47639|AAAAAAAAIBKLAAAA|47639|13|13|59|PM|first|afternoon|lunch| +47640|AAAAAAAAJBKLAAAA|47640|13|14|0|PM|first|afternoon|lunch| +47641|AAAAAAAAKBKLAAAA|47641|13|14|1|PM|first|afternoon|lunch| +47642|AAAAAAAALBKLAAAA|47642|13|14|2|PM|first|afternoon|lunch| +47643|AAAAAAAAMBKLAAAA|47643|13|14|3|PM|first|afternoon|lunch| +47644|AAAAAAAANBKLAAAA|47644|13|14|4|PM|first|afternoon|lunch| +47645|AAAAAAAAOBKLAAAA|47645|13|14|5|PM|first|afternoon|lunch| +47646|AAAAAAAAPBKLAAAA|47646|13|14|6|PM|first|afternoon|lunch| +47647|AAAAAAAAACKLAAAA|47647|13|14|7|PM|first|afternoon|lunch| +47648|AAAAAAAABCKLAAAA|47648|13|14|8|PM|first|afternoon|lunch| +47649|AAAAAAAACCKLAAAA|47649|13|14|9|PM|first|afternoon|lunch| +47650|AAAAAAAADCKLAAAA|47650|13|14|10|PM|first|afternoon|lunch| +47651|AAAAAAAAECKLAAAA|47651|13|14|11|PM|first|afternoon|lunch| +47652|AAAAAAAAFCKLAAAA|47652|13|14|12|PM|first|afternoon|lunch| +47653|AAAAAAAAGCKLAAAA|47653|13|14|13|PM|first|afternoon|lunch| +47654|AAAAAAAAHCKLAAAA|47654|13|14|14|PM|first|afternoon|lunch| +47655|AAAAAAAAICKLAAAA|47655|13|14|15|PM|first|afternoon|lunch| +47656|AAAAAAAAJCKLAAAA|47656|13|14|16|PM|first|afternoon|lunch| +47657|AAAAAAAAKCKLAAAA|47657|13|14|17|PM|first|afternoon|lunch| +47658|AAAAAAAALCKLAAAA|47658|13|14|18|PM|first|afternoon|lunch| +47659|AAAAAAAAMCKLAAAA|47659|13|14|19|PM|first|afternoon|lunch| +47660|AAAAAAAANCKLAAAA|47660|13|14|20|PM|first|afternoon|lunch| +47661|AAAAAAAAOCKLAAAA|47661|13|14|21|PM|first|afternoon|lunch| +47662|AAAAAAAAPCKLAAAA|47662|13|14|22|PM|first|afternoon|lunch| +47663|AAAAAAAAADKLAAAA|47663|13|14|23|PM|first|afternoon|lunch| +47664|AAAAAAAABDKLAAAA|47664|13|14|24|PM|first|afternoon|lunch| +47665|AAAAAAAACDKLAAAA|47665|13|14|25|PM|first|afternoon|lunch| +47666|AAAAAAAADDKLAAAA|47666|13|14|26|PM|first|afternoon|lunch| +47667|AAAAAAAAEDKLAAAA|47667|13|14|27|PM|first|afternoon|lunch| +47668|AAAAAAAAFDKLAAAA|47668|13|14|28|PM|first|afternoon|lunch| +47669|AAAAAAAAGDKLAAAA|47669|13|14|29|PM|first|afternoon|lunch| +47670|AAAAAAAAHDKLAAAA|47670|13|14|30|PM|first|afternoon|lunch| +47671|AAAAAAAAIDKLAAAA|47671|13|14|31|PM|first|afternoon|lunch| +47672|AAAAAAAAJDKLAAAA|47672|13|14|32|PM|first|afternoon|lunch| +47673|AAAAAAAAKDKLAAAA|47673|13|14|33|PM|first|afternoon|lunch| +47674|AAAAAAAALDKLAAAA|47674|13|14|34|PM|first|afternoon|lunch| +47675|AAAAAAAAMDKLAAAA|47675|13|14|35|PM|first|afternoon|lunch| +47676|AAAAAAAANDKLAAAA|47676|13|14|36|PM|first|afternoon|lunch| +47677|AAAAAAAAODKLAAAA|47677|13|14|37|PM|first|afternoon|lunch| +47678|AAAAAAAAPDKLAAAA|47678|13|14|38|PM|first|afternoon|lunch| +47679|AAAAAAAAAEKLAAAA|47679|13|14|39|PM|first|afternoon|lunch| +47680|AAAAAAAABEKLAAAA|47680|13|14|40|PM|first|afternoon|lunch| +47681|AAAAAAAACEKLAAAA|47681|13|14|41|PM|first|afternoon|lunch| +47682|AAAAAAAADEKLAAAA|47682|13|14|42|PM|first|afternoon|lunch| +47683|AAAAAAAAEEKLAAAA|47683|13|14|43|PM|first|afternoon|lunch| +47684|AAAAAAAAFEKLAAAA|47684|13|14|44|PM|first|afternoon|lunch| +47685|AAAAAAAAGEKLAAAA|47685|13|14|45|PM|first|afternoon|lunch| +47686|AAAAAAAAHEKLAAAA|47686|13|14|46|PM|first|afternoon|lunch| +47687|AAAAAAAAIEKLAAAA|47687|13|14|47|PM|first|afternoon|lunch| +47688|AAAAAAAAJEKLAAAA|47688|13|14|48|PM|first|afternoon|lunch| +47689|AAAAAAAAKEKLAAAA|47689|13|14|49|PM|first|afternoon|lunch| +47690|AAAAAAAALEKLAAAA|47690|13|14|50|PM|first|afternoon|lunch| +47691|AAAAAAAAMEKLAAAA|47691|13|14|51|PM|first|afternoon|lunch| +47692|AAAAAAAANEKLAAAA|47692|13|14|52|PM|first|afternoon|lunch| +47693|AAAAAAAAOEKLAAAA|47693|13|14|53|PM|first|afternoon|lunch| +47694|AAAAAAAAPEKLAAAA|47694|13|14|54|PM|first|afternoon|lunch| +47695|AAAAAAAAAFKLAAAA|47695|13|14|55|PM|first|afternoon|lunch| +47696|AAAAAAAABFKLAAAA|47696|13|14|56|PM|first|afternoon|lunch| +47697|AAAAAAAACFKLAAAA|47697|13|14|57|PM|first|afternoon|lunch| +47698|AAAAAAAADFKLAAAA|47698|13|14|58|PM|first|afternoon|lunch| +47699|AAAAAAAAEFKLAAAA|47699|13|14|59|PM|first|afternoon|lunch| +47700|AAAAAAAAFFKLAAAA|47700|13|15|0|PM|first|afternoon|lunch| +47701|AAAAAAAAGFKLAAAA|47701|13|15|1|PM|first|afternoon|lunch| +47702|AAAAAAAAHFKLAAAA|47702|13|15|2|PM|first|afternoon|lunch| +47703|AAAAAAAAIFKLAAAA|47703|13|15|3|PM|first|afternoon|lunch| +47704|AAAAAAAAJFKLAAAA|47704|13|15|4|PM|first|afternoon|lunch| +47705|AAAAAAAAKFKLAAAA|47705|13|15|5|PM|first|afternoon|lunch| +47706|AAAAAAAALFKLAAAA|47706|13|15|6|PM|first|afternoon|lunch| +47707|AAAAAAAAMFKLAAAA|47707|13|15|7|PM|first|afternoon|lunch| +47708|AAAAAAAANFKLAAAA|47708|13|15|8|PM|first|afternoon|lunch| +47709|AAAAAAAAOFKLAAAA|47709|13|15|9|PM|first|afternoon|lunch| +47710|AAAAAAAAPFKLAAAA|47710|13|15|10|PM|first|afternoon|lunch| +47711|AAAAAAAAAGKLAAAA|47711|13|15|11|PM|first|afternoon|lunch| +47712|AAAAAAAABGKLAAAA|47712|13|15|12|PM|first|afternoon|lunch| +47713|AAAAAAAACGKLAAAA|47713|13|15|13|PM|first|afternoon|lunch| +47714|AAAAAAAADGKLAAAA|47714|13|15|14|PM|first|afternoon|lunch| +47715|AAAAAAAAEGKLAAAA|47715|13|15|15|PM|first|afternoon|lunch| +47716|AAAAAAAAFGKLAAAA|47716|13|15|16|PM|first|afternoon|lunch| +47717|AAAAAAAAGGKLAAAA|47717|13|15|17|PM|first|afternoon|lunch| +47718|AAAAAAAAHGKLAAAA|47718|13|15|18|PM|first|afternoon|lunch| +47719|AAAAAAAAIGKLAAAA|47719|13|15|19|PM|first|afternoon|lunch| +47720|AAAAAAAAJGKLAAAA|47720|13|15|20|PM|first|afternoon|lunch| +47721|AAAAAAAAKGKLAAAA|47721|13|15|21|PM|first|afternoon|lunch| +47722|AAAAAAAALGKLAAAA|47722|13|15|22|PM|first|afternoon|lunch| +47723|AAAAAAAAMGKLAAAA|47723|13|15|23|PM|first|afternoon|lunch| +47724|AAAAAAAANGKLAAAA|47724|13|15|24|PM|first|afternoon|lunch| +47725|AAAAAAAAOGKLAAAA|47725|13|15|25|PM|first|afternoon|lunch| +47726|AAAAAAAAPGKLAAAA|47726|13|15|26|PM|first|afternoon|lunch| +47727|AAAAAAAAAHKLAAAA|47727|13|15|27|PM|first|afternoon|lunch| +47728|AAAAAAAABHKLAAAA|47728|13|15|28|PM|first|afternoon|lunch| +47729|AAAAAAAACHKLAAAA|47729|13|15|29|PM|first|afternoon|lunch| +47730|AAAAAAAADHKLAAAA|47730|13|15|30|PM|first|afternoon|lunch| +47731|AAAAAAAAEHKLAAAA|47731|13|15|31|PM|first|afternoon|lunch| +47732|AAAAAAAAFHKLAAAA|47732|13|15|32|PM|first|afternoon|lunch| +47733|AAAAAAAAGHKLAAAA|47733|13|15|33|PM|first|afternoon|lunch| +47734|AAAAAAAAHHKLAAAA|47734|13|15|34|PM|first|afternoon|lunch| +47735|AAAAAAAAIHKLAAAA|47735|13|15|35|PM|first|afternoon|lunch| +47736|AAAAAAAAJHKLAAAA|47736|13|15|36|PM|first|afternoon|lunch| +47737|AAAAAAAAKHKLAAAA|47737|13|15|37|PM|first|afternoon|lunch| +47738|AAAAAAAALHKLAAAA|47738|13|15|38|PM|first|afternoon|lunch| +47739|AAAAAAAAMHKLAAAA|47739|13|15|39|PM|first|afternoon|lunch| +47740|AAAAAAAANHKLAAAA|47740|13|15|40|PM|first|afternoon|lunch| +47741|AAAAAAAAOHKLAAAA|47741|13|15|41|PM|first|afternoon|lunch| +47742|AAAAAAAAPHKLAAAA|47742|13|15|42|PM|first|afternoon|lunch| +47743|AAAAAAAAAIKLAAAA|47743|13|15|43|PM|first|afternoon|lunch| +47744|AAAAAAAABIKLAAAA|47744|13|15|44|PM|first|afternoon|lunch| +47745|AAAAAAAACIKLAAAA|47745|13|15|45|PM|first|afternoon|lunch| +47746|AAAAAAAADIKLAAAA|47746|13|15|46|PM|first|afternoon|lunch| +47747|AAAAAAAAEIKLAAAA|47747|13|15|47|PM|first|afternoon|lunch| +47748|AAAAAAAAFIKLAAAA|47748|13|15|48|PM|first|afternoon|lunch| +47749|AAAAAAAAGIKLAAAA|47749|13|15|49|PM|first|afternoon|lunch| +47750|AAAAAAAAHIKLAAAA|47750|13|15|50|PM|first|afternoon|lunch| +47751|AAAAAAAAIIKLAAAA|47751|13|15|51|PM|first|afternoon|lunch| +47752|AAAAAAAAJIKLAAAA|47752|13|15|52|PM|first|afternoon|lunch| +47753|AAAAAAAAKIKLAAAA|47753|13|15|53|PM|first|afternoon|lunch| +47754|AAAAAAAALIKLAAAA|47754|13|15|54|PM|first|afternoon|lunch| +47755|AAAAAAAAMIKLAAAA|47755|13|15|55|PM|first|afternoon|lunch| +47756|AAAAAAAANIKLAAAA|47756|13|15|56|PM|first|afternoon|lunch| +47757|AAAAAAAAOIKLAAAA|47757|13|15|57|PM|first|afternoon|lunch| +47758|AAAAAAAAPIKLAAAA|47758|13|15|58|PM|first|afternoon|lunch| +47759|AAAAAAAAAJKLAAAA|47759|13|15|59|PM|first|afternoon|lunch| +47760|AAAAAAAABJKLAAAA|47760|13|16|0|PM|first|afternoon|lunch| +47761|AAAAAAAACJKLAAAA|47761|13|16|1|PM|first|afternoon|lunch| +47762|AAAAAAAADJKLAAAA|47762|13|16|2|PM|first|afternoon|lunch| +47763|AAAAAAAAEJKLAAAA|47763|13|16|3|PM|first|afternoon|lunch| +47764|AAAAAAAAFJKLAAAA|47764|13|16|4|PM|first|afternoon|lunch| +47765|AAAAAAAAGJKLAAAA|47765|13|16|5|PM|first|afternoon|lunch| +47766|AAAAAAAAHJKLAAAA|47766|13|16|6|PM|first|afternoon|lunch| +47767|AAAAAAAAIJKLAAAA|47767|13|16|7|PM|first|afternoon|lunch| +47768|AAAAAAAAJJKLAAAA|47768|13|16|8|PM|first|afternoon|lunch| +47769|AAAAAAAAKJKLAAAA|47769|13|16|9|PM|first|afternoon|lunch| +47770|AAAAAAAALJKLAAAA|47770|13|16|10|PM|first|afternoon|lunch| +47771|AAAAAAAAMJKLAAAA|47771|13|16|11|PM|first|afternoon|lunch| +47772|AAAAAAAANJKLAAAA|47772|13|16|12|PM|first|afternoon|lunch| +47773|AAAAAAAAOJKLAAAA|47773|13|16|13|PM|first|afternoon|lunch| +47774|AAAAAAAAPJKLAAAA|47774|13|16|14|PM|first|afternoon|lunch| +47775|AAAAAAAAAKKLAAAA|47775|13|16|15|PM|first|afternoon|lunch| +47776|AAAAAAAABKKLAAAA|47776|13|16|16|PM|first|afternoon|lunch| +47777|AAAAAAAACKKLAAAA|47777|13|16|17|PM|first|afternoon|lunch| +47778|AAAAAAAADKKLAAAA|47778|13|16|18|PM|first|afternoon|lunch| +47779|AAAAAAAAEKKLAAAA|47779|13|16|19|PM|first|afternoon|lunch| +47780|AAAAAAAAFKKLAAAA|47780|13|16|20|PM|first|afternoon|lunch| +47781|AAAAAAAAGKKLAAAA|47781|13|16|21|PM|first|afternoon|lunch| +47782|AAAAAAAAHKKLAAAA|47782|13|16|22|PM|first|afternoon|lunch| +47783|AAAAAAAAIKKLAAAA|47783|13|16|23|PM|first|afternoon|lunch| +47784|AAAAAAAAJKKLAAAA|47784|13|16|24|PM|first|afternoon|lunch| +47785|AAAAAAAAKKKLAAAA|47785|13|16|25|PM|first|afternoon|lunch| +47786|AAAAAAAALKKLAAAA|47786|13|16|26|PM|first|afternoon|lunch| +47787|AAAAAAAAMKKLAAAA|47787|13|16|27|PM|first|afternoon|lunch| +47788|AAAAAAAANKKLAAAA|47788|13|16|28|PM|first|afternoon|lunch| +47789|AAAAAAAAOKKLAAAA|47789|13|16|29|PM|first|afternoon|lunch| +47790|AAAAAAAAPKKLAAAA|47790|13|16|30|PM|first|afternoon|lunch| +47791|AAAAAAAAALKLAAAA|47791|13|16|31|PM|first|afternoon|lunch| +47792|AAAAAAAABLKLAAAA|47792|13|16|32|PM|first|afternoon|lunch| +47793|AAAAAAAACLKLAAAA|47793|13|16|33|PM|first|afternoon|lunch| +47794|AAAAAAAADLKLAAAA|47794|13|16|34|PM|first|afternoon|lunch| +47795|AAAAAAAAELKLAAAA|47795|13|16|35|PM|first|afternoon|lunch| +47796|AAAAAAAAFLKLAAAA|47796|13|16|36|PM|first|afternoon|lunch| +47797|AAAAAAAAGLKLAAAA|47797|13|16|37|PM|first|afternoon|lunch| +47798|AAAAAAAAHLKLAAAA|47798|13|16|38|PM|first|afternoon|lunch| +47799|AAAAAAAAILKLAAAA|47799|13|16|39|PM|first|afternoon|lunch| +47800|AAAAAAAAJLKLAAAA|47800|13|16|40|PM|first|afternoon|lunch| +47801|AAAAAAAAKLKLAAAA|47801|13|16|41|PM|first|afternoon|lunch| +47802|AAAAAAAALLKLAAAA|47802|13|16|42|PM|first|afternoon|lunch| +47803|AAAAAAAAMLKLAAAA|47803|13|16|43|PM|first|afternoon|lunch| +47804|AAAAAAAANLKLAAAA|47804|13|16|44|PM|first|afternoon|lunch| +47805|AAAAAAAAOLKLAAAA|47805|13|16|45|PM|first|afternoon|lunch| +47806|AAAAAAAAPLKLAAAA|47806|13|16|46|PM|first|afternoon|lunch| +47807|AAAAAAAAAMKLAAAA|47807|13|16|47|PM|first|afternoon|lunch| +47808|AAAAAAAABMKLAAAA|47808|13|16|48|PM|first|afternoon|lunch| +47809|AAAAAAAACMKLAAAA|47809|13|16|49|PM|first|afternoon|lunch| +47810|AAAAAAAADMKLAAAA|47810|13|16|50|PM|first|afternoon|lunch| +47811|AAAAAAAAEMKLAAAA|47811|13|16|51|PM|first|afternoon|lunch| +47812|AAAAAAAAFMKLAAAA|47812|13|16|52|PM|first|afternoon|lunch| +47813|AAAAAAAAGMKLAAAA|47813|13|16|53|PM|first|afternoon|lunch| +47814|AAAAAAAAHMKLAAAA|47814|13|16|54|PM|first|afternoon|lunch| +47815|AAAAAAAAIMKLAAAA|47815|13|16|55|PM|first|afternoon|lunch| +47816|AAAAAAAAJMKLAAAA|47816|13|16|56|PM|first|afternoon|lunch| +47817|AAAAAAAAKMKLAAAA|47817|13|16|57|PM|first|afternoon|lunch| +47818|AAAAAAAALMKLAAAA|47818|13|16|58|PM|first|afternoon|lunch| +47819|AAAAAAAAMMKLAAAA|47819|13|16|59|PM|first|afternoon|lunch| +47820|AAAAAAAANMKLAAAA|47820|13|17|0|PM|first|afternoon|lunch| +47821|AAAAAAAAOMKLAAAA|47821|13|17|1|PM|first|afternoon|lunch| +47822|AAAAAAAAPMKLAAAA|47822|13|17|2|PM|first|afternoon|lunch| +47823|AAAAAAAAANKLAAAA|47823|13|17|3|PM|first|afternoon|lunch| +47824|AAAAAAAABNKLAAAA|47824|13|17|4|PM|first|afternoon|lunch| +47825|AAAAAAAACNKLAAAA|47825|13|17|5|PM|first|afternoon|lunch| +47826|AAAAAAAADNKLAAAA|47826|13|17|6|PM|first|afternoon|lunch| +47827|AAAAAAAAENKLAAAA|47827|13|17|7|PM|first|afternoon|lunch| +47828|AAAAAAAAFNKLAAAA|47828|13|17|8|PM|first|afternoon|lunch| +47829|AAAAAAAAGNKLAAAA|47829|13|17|9|PM|first|afternoon|lunch| +47830|AAAAAAAAHNKLAAAA|47830|13|17|10|PM|first|afternoon|lunch| +47831|AAAAAAAAINKLAAAA|47831|13|17|11|PM|first|afternoon|lunch| +47832|AAAAAAAAJNKLAAAA|47832|13|17|12|PM|first|afternoon|lunch| +47833|AAAAAAAAKNKLAAAA|47833|13|17|13|PM|first|afternoon|lunch| +47834|AAAAAAAALNKLAAAA|47834|13|17|14|PM|first|afternoon|lunch| +47835|AAAAAAAAMNKLAAAA|47835|13|17|15|PM|first|afternoon|lunch| +47836|AAAAAAAANNKLAAAA|47836|13|17|16|PM|first|afternoon|lunch| +47837|AAAAAAAAONKLAAAA|47837|13|17|17|PM|first|afternoon|lunch| +47838|AAAAAAAAPNKLAAAA|47838|13|17|18|PM|first|afternoon|lunch| +47839|AAAAAAAAAOKLAAAA|47839|13|17|19|PM|first|afternoon|lunch| +47840|AAAAAAAABOKLAAAA|47840|13|17|20|PM|first|afternoon|lunch| +47841|AAAAAAAACOKLAAAA|47841|13|17|21|PM|first|afternoon|lunch| +47842|AAAAAAAADOKLAAAA|47842|13|17|22|PM|first|afternoon|lunch| +47843|AAAAAAAAEOKLAAAA|47843|13|17|23|PM|first|afternoon|lunch| +47844|AAAAAAAAFOKLAAAA|47844|13|17|24|PM|first|afternoon|lunch| +47845|AAAAAAAAGOKLAAAA|47845|13|17|25|PM|first|afternoon|lunch| +47846|AAAAAAAAHOKLAAAA|47846|13|17|26|PM|first|afternoon|lunch| +47847|AAAAAAAAIOKLAAAA|47847|13|17|27|PM|first|afternoon|lunch| +47848|AAAAAAAAJOKLAAAA|47848|13|17|28|PM|first|afternoon|lunch| +47849|AAAAAAAAKOKLAAAA|47849|13|17|29|PM|first|afternoon|lunch| +47850|AAAAAAAALOKLAAAA|47850|13|17|30|PM|first|afternoon|lunch| +47851|AAAAAAAAMOKLAAAA|47851|13|17|31|PM|first|afternoon|lunch| +47852|AAAAAAAANOKLAAAA|47852|13|17|32|PM|first|afternoon|lunch| +47853|AAAAAAAAOOKLAAAA|47853|13|17|33|PM|first|afternoon|lunch| +47854|AAAAAAAAPOKLAAAA|47854|13|17|34|PM|first|afternoon|lunch| +47855|AAAAAAAAAPKLAAAA|47855|13|17|35|PM|first|afternoon|lunch| +47856|AAAAAAAABPKLAAAA|47856|13|17|36|PM|first|afternoon|lunch| +47857|AAAAAAAACPKLAAAA|47857|13|17|37|PM|first|afternoon|lunch| +47858|AAAAAAAADPKLAAAA|47858|13|17|38|PM|first|afternoon|lunch| +47859|AAAAAAAAEPKLAAAA|47859|13|17|39|PM|first|afternoon|lunch| +47860|AAAAAAAAFPKLAAAA|47860|13|17|40|PM|first|afternoon|lunch| +47861|AAAAAAAAGPKLAAAA|47861|13|17|41|PM|first|afternoon|lunch| +47862|AAAAAAAAHPKLAAAA|47862|13|17|42|PM|first|afternoon|lunch| +47863|AAAAAAAAIPKLAAAA|47863|13|17|43|PM|first|afternoon|lunch| +47864|AAAAAAAAJPKLAAAA|47864|13|17|44|PM|first|afternoon|lunch| +47865|AAAAAAAAKPKLAAAA|47865|13|17|45|PM|first|afternoon|lunch| +47866|AAAAAAAALPKLAAAA|47866|13|17|46|PM|first|afternoon|lunch| +47867|AAAAAAAAMPKLAAAA|47867|13|17|47|PM|first|afternoon|lunch| +47868|AAAAAAAANPKLAAAA|47868|13|17|48|PM|first|afternoon|lunch| +47869|AAAAAAAAOPKLAAAA|47869|13|17|49|PM|first|afternoon|lunch| +47870|AAAAAAAAPPKLAAAA|47870|13|17|50|PM|first|afternoon|lunch| +47871|AAAAAAAAAALLAAAA|47871|13|17|51|PM|first|afternoon|lunch| +47872|AAAAAAAABALLAAAA|47872|13|17|52|PM|first|afternoon|lunch| +47873|AAAAAAAACALLAAAA|47873|13|17|53|PM|first|afternoon|lunch| +47874|AAAAAAAADALLAAAA|47874|13|17|54|PM|first|afternoon|lunch| +47875|AAAAAAAAEALLAAAA|47875|13|17|55|PM|first|afternoon|lunch| +47876|AAAAAAAAFALLAAAA|47876|13|17|56|PM|first|afternoon|lunch| +47877|AAAAAAAAGALLAAAA|47877|13|17|57|PM|first|afternoon|lunch| +47878|AAAAAAAAHALLAAAA|47878|13|17|58|PM|first|afternoon|lunch| +47879|AAAAAAAAIALLAAAA|47879|13|17|59|PM|first|afternoon|lunch| +47880|AAAAAAAAJALLAAAA|47880|13|18|0|PM|first|afternoon|lunch| +47881|AAAAAAAAKALLAAAA|47881|13|18|1|PM|first|afternoon|lunch| +47882|AAAAAAAALALLAAAA|47882|13|18|2|PM|first|afternoon|lunch| +47883|AAAAAAAAMALLAAAA|47883|13|18|3|PM|first|afternoon|lunch| +47884|AAAAAAAANALLAAAA|47884|13|18|4|PM|first|afternoon|lunch| +47885|AAAAAAAAOALLAAAA|47885|13|18|5|PM|first|afternoon|lunch| +47886|AAAAAAAAPALLAAAA|47886|13|18|6|PM|first|afternoon|lunch| +47887|AAAAAAAAABLLAAAA|47887|13|18|7|PM|first|afternoon|lunch| +47888|AAAAAAAABBLLAAAA|47888|13|18|8|PM|first|afternoon|lunch| +47889|AAAAAAAACBLLAAAA|47889|13|18|9|PM|first|afternoon|lunch| +47890|AAAAAAAADBLLAAAA|47890|13|18|10|PM|first|afternoon|lunch| +47891|AAAAAAAAEBLLAAAA|47891|13|18|11|PM|first|afternoon|lunch| +47892|AAAAAAAAFBLLAAAA|47892|13|18|12|PM|first|afternoon|lunch| +47893|AAAAAAAAGBLLAAAA|47893|13|18|13|PM|first|afternoon|lunch| +47894|AAAAAAAAHBLLAAAA|47894|13|18|14|PM|first|afternoon|lunch| +47895|AAAAAAAAIBLLAAAA|47895|13|18|15|PM|first|afternoon|lunch| +47896|AAAAAAAAJBLLAAAA|47896|13|18|16|PM|first|afternoon|lunch| +47897|AAAAAAAAKBLLAAAA|47897|13|18|17|PM|first|afternoon|lunch| +47898|AAAAAAAALBLLAAAA|47898|13|18|18|PM|first|afternoon|lunch| +47899|AAAAAAAAMBLLAAAA|47899|13|18|19|PM|first|afternoon|lunch| +47900|AAAAAAAANBLLAAAA|47900|13|18|20|PM|first|afternoon|lunch| +47901|AAAAAAAAOBLLAAAA|47901|13|18|21|PM|first|afternoon|lunch| +47902|AAAAAAAAPBLLAAAA|47902|13|18|22|PM|first|afternoon|lunch| +47903|AAAAAAAAACLLAAAA|47903|13|18|23|PM|first|afternoon|lunch| +47904|AAAAAAAABCLLAAAA|47904|13|18|24|PM|first|afternoon|lunch| +47905|AAAAAAAACCLLAAAA|47905|13|18|25|PM|first|afternoon|lunch| +47906|AAAAAAAADCLLAAAA|47906|13|18|26|PM|first|afternoon|lunch| +47907|AAAAAAAAECLLAAAA|47907|13|18|27|PM|first|afternoon|lunch| +47908|AAAAAAAAFCLLAAAA|47908|13|18|28|PM|first|afternoon|lunch| +47909|AAAAAAAAGCLLAAAA|47909|13|18|29|PM|first|afternoon|lunch| +47910|AAAAAAAAHCLLAAAA|47910|13|18|30|PM|first|afternoon|lunch| +47911|AAAAAAAAICLLAAAA|47911|13|18|31|PM|first|afternoon|lunch| +47912|AAAAAAAAJCLLAAAA|47912|13|18|32|PM|first|afternoon|lunch| +47913|AAAAAAAAKCLLAAAA|47913|13|18|33|PM|first|afternoon|lunch| +47914|AAAAAAAALCLLAAAA|47914|13|18|34|PM|first|afternoon|lunch| +47915|AAAAAAAAMCLLAAAA|47915|13|18|35|PM|first|afternoon|lunch| +47916|AAAAAAAANCLLAAAA|47916|13|18|36|PM|first|afternoon|lunch| +47917|AAAAAAAAOCLLAAAA|47917|13|18|37|PM|first|afternoon|lunch| +47918|AAAAAAAAPCLLAAAA|47918|13|18|38|PM|first|afternoon|lunch| +47919|AAAAAAAAADLLAAAA|47919|13|18|39|PM|first|afternoon|lunch| +47920|AAAAAAAABDLLAAAA|47920|13|18|40|PM|first|afternoon|lunch| +47921|AAAAAAAACDLLAAAA|47921|13|18|41|PM|first|afternoon|lunch| +47922|AAAAAAAADDLLAAAA|47922|13|18|42|PM|first|afternoon|lunch| +47923|AAAAAAAAEDLLAAAA|47923|13|18|43|PM|first|afternoon|lunch| +47924|AAAAAAAAFDLLAAAA|47924|13|18|44|PM|first|afternoon|lunch| +47925|AAAAAAAAGDLLAAAA|47925|13|18|45|PM|first|afternoon|lunch| +47926|AAAAAAAAHDLLAAAA|47926|13|18|46|PM|first|afternoon|lunch| +47927|AAAAAAAAIDLLAAAA|47927|13|18|47|PM|first|afternoon|lunch| +47928|AAAAAAAAJDLLAAAA|47928|13|18|48|PM|first|afternoon|lunch| +47929|AAAAAAAAKDLLAAAA|47929|13|18|49|PM|first|afternoon|lunch| +47930|AAAAAAAALDLLAAAA|47930|13|18|50|PM|first|afternoon|lunch| +47931|AAAAAAAAMDLLAAAA|47931|13|18|51|PM|first|afternoon|lunch| +47932|AAAAAAAANDLLAAAA|47932|13|18|52|PM|first|afternoon|lunch| +47933|AAAAAAAAODLLAAAA|47933|13|18|53|PM|first|afternoon|lunch| +47934|AAAAAAAAPDLLAAAA|47934|13|18|54|PM|first|afternoon|lunch| +47935|AAAAAAAAAELLAAAA|47935|13|18|55|PM|first|afternoon|lunch| +47936|AAAAAAAABELLAAAA|47936|13|18|56|PM|first|afternoon|lunch| +47937|AAAAAAAACELLAAAA|47937|13|18|57|PM|first|afternoon|lunch| +47938|AAAAAAAADELLAAAA|47938|13|18|58|PM|first|afternoon|lunch| +47939|AAAAAAAAEELLAAAA|47939|13|18|59|PM|first|afternoon|lunch| +47940|AAAAAAAAFELLAAAA|47940|13|19|0|PM|first|afternoon|lunch| +47941|AAAAAAAAGELLAAAA|47941|13|19|1|PM|first|afternoon|lunch| +47942|AAAAAAAAHELLAAAA|47942|13|19|2|PM|first|afternoon|lunch| +47943|AAAAAAAAIELLAAAA|47943|13|19|3|PM|first|afternoon|lunch| +47944|AAAAAAAAJELLAAAA|47944|13|19|4|PM|first|afternoon|lunch| +47945|AAAAAAAAKELLAAAA|47945|13|19|5|PM|first|afternoon|lunch| +47946|AAAAAAAALELLAAAA|47946|13|19|6|PM|first|afternoon|lunch| +47947|AAAAAAAAMELLAAAA|47947|13|19|7|PM|first|afternoon|lunch| +47948|AAAAAAAANELLAAAA|47948|13|19|8|PM|first|afternoon|lunch| +47949|AAAAAAAAOELLAAAA|47949|13|19|9|PM|first|afternoon|lunch| +47950|AAAAAAAAPELLAAAA|47950|13|19|10|PM|first|afternoon|lunch| +47951|AAAAAAAAAFLLAAAA|47951|13|19|11|PM|first|afternoon|lunch| +47952|AAAAAAAABFLLAAAA|47952|13|19|12|PM|first|afternoon|lunch| +47953|AAAAAAAACFLLAAAA|47953|13|19|13|PM|first|afternoon|lunch| +47954|AAAAAAAADFLLAAAA|47954|13|19|14|PM|first|afternoon|lunch| +47955|AAAAAAAAEFLLAAAA|47955|13|19|15|PM|first|afternoon|lunch| +47956|AAAAAAAAFFLLAAAA|47956|13|19|16|PM|first|afternoon|lunch| +47957|AAAAAAAAGFLLAAAA|47957|13|19|17|PM|first|afternoon|lunch| +47958|AAAAAAAAHFLLAAAA|47958|13|19|18|PM|first|afternoon|lunch| +47959|AAAAAAAAIFLLAAAA|47959|13|19|19|PM|first|afternoon|lunch| +47960|AAAAAAAAJFLLAAAA|47960|13|19|20|PM|first|afternoon|lunch| +47961|AAAAAAAAKFLLAAAA|47961|13|19|21|PM|first|afternoon|lunch| +47962|AAAAAAAALFLLAAAA|47962|13|19|22|PM|first|afternoon|lunch| +47963|AAAAAAAAMFLLAAAA|47963|13|19|23|PM|first|afternoon|lunch| +47964|AAAAAAAANFLLAAAA|47964|13|19|24|PM|first|afternoon|lunch| +47965|AAAAAAAAOFLLAAAA|47965|13|19|25|PM|first|afternoon|lunch| +47966|AAAAAAAAPFLLAAAA|47966|13|19|26|PM|first|afternoon|lunch| +47967|AAAAAAAAAGLLAAAA|47967|13|19|27|PM|first|afternoon|lunch| +47968|AAAAAAAABGLLAAAA|47968|13|19|28|PM|first|afternoon|lunch| +47969|AAAAAAAACGLLAAAA|47969|13|19|29|PM|first|afternoon|lunch| +47970|AAAAAAAADGLLAAAA|47970|13|19|30|PM|first|afternoon|lunch| +47971|AAAAAAAAEGLLAAAA|47971|13|19|31|PM|first|afternoon|lunch| +47972|AAAAAAAAFGLLAAAA|47972|13|19|32|PM|first|afternoon|lunch| +47973|AAAAAAAAGGLLAAAA|47973|13|19|33|PM|first|afternoon|lunch| +47974|AAAAAAAAHGLLAAAA|47974|13|19|34|PM|first|afternoon|lunch| +47975|AAAAAAAAIGLLAAAA|47975|13|19|35|PM|first|afternoon|lunch| +47976|AAAAAAAAJGLLAAAA|47976|13|19|36|PM|first|afternoon|lunch| +47977|AAAAAAAAKGLLAAAA|47977|13|19|37|PM|first|afternoon|lunch| +47978|AAAAAAAALGLLAAAA|47978|13|19|38|PM|first|afternoon|lunch| +47979|AAAAAAAAMGLLAAAA|47979|13|19|39|PM|first|afternoon|lunch| +47980|AAAAAAAANGLLAAAA|47980|13|19|40|PM|first|afternoon|lunch| +47981|AAAAAAAAOGLLAAAA|47981|13|19|41|PM|first|afternoon|lunch| +47982|AAAAAAAAPGLLAAAA|47982|13|19|42|PM|first|afternoon|lunch| +47983|AAAAAAAAAHLLAAAA|47983|13|19|43|PM|first|afternoon|lunch| +47984|AAAAAAAABHLLAAAA|47984|13|19|44|PM|first|afternoon|lunch| +47985|AAAAAAAACHLLAAAA|47985|13|19|45|PM|first|afternoon|lunch| +47986|AAAAAAAADHLLAAAA|47986|13|19|46|PM|first|afternoon|lunch| +47987|AAAAAAAAEHLLAAAA|47987|13|19|47|PM|first|afternoon|lunch| +47988|AAAAAAAAFHLLAAAA|47988|13|19|48|PM|first|afternoon|lunch| +47989|AAAAAAAAGHLLAAAA|47989|13|19|49|PM|first|afternoon|lunch| +47990|AAAAAAAAHHLLAAAA|47990|13|19|50|PM|first|afternoon|lunch| +47991|AAAAAAAAIHLLAAAA|47991|13|19|51|PM|first|afternoon|lunch| +47992|AAAAAAAAJHLLAAAA|47992|13|19|52|PM|first|afternoon|lunch| +47993|AAAAAAAAKHLLAAAA|47993|13|19|53|PM|first|afternoon|lunch| +47994|AAAAAAAALHLLAAAA|47994|13|19|54|PM|first|afternoon|lunch| +47995|AAAAAAAAMHLLAAAA|47995|13|19|55|PM|first|afternoon|lunch| +47996|AAAAAAAANHLLAAAA|47996|13|19|56|PM|first|afternoon|lunch| +47997|AAAAAAAAOHLLAAAA|47997|13|19|57|PM|first|afternoon|lunch| +47998|AAAAAAAAPHLLAAAA|47998|13|19|58|PM|first|afternoon|lunch| +47999|AAAAAAAAAILLAAAA|47999|13|19|59|PM|first|afternoon|lunch| +48000|AAAAAAAABILLAAAA|48000|13|20|0|PM|first|afternoon|lunch| +48001|AAAAAAAACILLAAAA|48001|13|20|1|PM|first|afternoon|lunch| +48002|AAAAAAAADILLAAAA|48002|13|20|2|PM|first|afternoon|lunch| +48003|AAAAAAAAEILLAAAA|48003|13|20|3|PM|first|afternoon|lunch| +48004|AAAAAAAAFILLAAAA|48004|13|20|4|PM|first|afternoon|lunch| +48005|AAAAAAAAGILLAAAA|48005|13|20|5|PM|first|afternoon|lunch| +48006|AAAAAAAAHILLAAAA|48006|13|20|6|PM|first|afternoon|lunch| +48007|AAAAAAAAIILLAAAA|48007|13|20|7|PM|first|afternoon|lunch| +48008|AAAAAAAAJILLAAAA|48008|13|20|8|PM|first|afternoon|lunch| +48009|AAAAAAAAKILLAAAA|48009|13|20|9|PM|first|afternoon|lunch| +48010|AAAAAAAALILLAAAA|48010|13|20|10|PM|first|afternoon|lunch| +48011|AAAAAAAAMILLAAAA|48011|13|20|11|PM|first|afternoon|lunch| +48012|AAAAAAAANILLAAAA|48012|13|20|12|PM|first|afternoon|lunch| +48013|AAAAAAAAOILLAAAA|48013|13|20|13|PM|first|afternoon|lunch| +48014|AAAAAAAAPILLAAAA|48014|13|20|14|PM|first|afternoon|lunch| +48015|AAAAAAAAAJLLAAAA|48015|13|20|15|PM|first|afternoon|lunch| +48016|AAAAAAAABJLLAAAA|48016|13|20|16|PM|first|afternoon|lunch| +48017|AAAAAAAACJLLAAAA|48017|13|20|17|PM|first|afternoon|lunch| +48018|AAAAAAAADJLLAAAA|48018|13|20|18|PM|first|afternoon|lunch| +48019|AAAAAAAAEJLLAAAA|48019|13|20|19|PM|first|afternoon|lunch| +48020|AAAAAAAAFJLLAAAA|48020|13|20|20|PM|first|afternoon|lunch| +48021|AAAAAAAAGJLLAAAA|48021|13|20|21|PM|first|afternoon|lunch| +48022|AAAAAAAAHJLLAAAA|48022|13|20|22|PM|first|afternoon|lunch| +48023|AAAAAAAAIJLLAAAA|48023|13|20|23|PM|first|afternoon|lunch| +48024|AAAAAAAAJJLLAAAA|48024|13|20|24|PM|first|afternoon|lunch| +48025|AAAAAAAAKJLLAAAA|48025|13|20|25|PM|first|afternoon|lunch| +48026|AAAAAAAALJLLAAAA|48026|13|20|26|PM|first|afternoon|lunch| +48027|AAAAAAAAMJLLAAAA|48027|13|20|27|PM|first|afternoon|lunch| +48028|AAAAAAAANJLLAAAA|48028|13|20|28|PM|first|afternoon|lunch| +48029|AAAAAAAAOJLLAAAA|48029|13|20|29|PM|first|afternoon|lunch| +48030|AAAAAAAAPJLLAAAA|48030|13|20|30|PM|first|afternoon|lunch| +48031|AAAAAAAAAKLLAAAA|48031|13|20|31|PM|first|afternoon|lunch| +48032|AAAAAAAABKLLAAAA|48032|13|20|32|PM|first|afternoon|lunch| +48033|AAAAAAAACKLLAAAA|48033|13|20|33|PM|first|afternoon|lunch| +48034|AAAAAAAADKLLAAAA|48034|13|20|34|PM|first|afternoon|lunch| +48035|AAAAAAAAEKLLAAAA|48035|13|20|35|PM|first|afternoon|lunch| +48036|AAAAAAAAFKLLAAAA|48036|13|20|36|PM|first|afternoon|lunch| +48037|AAAAAAAAGKLLAAAA|48037|13|20|37|PM|first|afternoon|lunch| +48038|AAAAAAAAHKLLAAAA|48038|13|20|38|PM|first|afternoon|lunch| +48039|AAAAAAAAIKLLAAAA|48039|13|20|39|PM|first|afternoon|lunch| +48040|AAAAAAAAJKLLAAAA|48040|13|20|40|PM|first|afternoon|lunch| +48041|AAAAAAAAKKLLAAAA|48041|13|20|41|PM|first|afternoon|lunch| +48042|AAAAAAAALKLLAAAA|48042|13|20|42|PM|first|afternoon|lunch| +48043|AAAAAAAAMKLLAAAA|48043|13|20|43|PM|first|afternoon|lunch| +48044|AAAAAAAANKLLAAAA|48044|13|20|44|PM|first|afternoon|lunch| +48045|AAAAAAAAOKLLAAAA|48045|13|20|45|PM|first|afternoon|lunch| +48046|AAAAAAAAPKLLAAAA|48046|13|20|46|PM|first|afternoon|lunch| +48047|AAAAAAAAALLLAAAA|48047|13|20|47|PM|first|afternoon|lunch| +48048|AAAAAAAABLLLAAAA|48048|13|20|48|PM|first|afternoon|lunch| +48049|AAAAAAAACLLLAAAA|48049|13|20|49|PM|first|afternoon|lunch| +48050|AAAAAAAADLLLAAAA|48050|13|20|50|PM|first|afternoon|lunch| +48051|AAAAAAAAELLLAAAA|48051|13|20|51|PM|first|afternoon|lunch| +48052|AAAAAAAAFLLLAAAA|48052|13|20|52|PM|first|afternoon|lunch| +48053|AAAAAAAAGLLLAAAA|48053|13|20|53|PM|first|afternoon|lunch| +48054|AAAAAAAAHLLLAAAA|48054|13|20|54|PM|first|afternoon|lunch| +48055|AAAAAAAAILLLAAAA|48055|13|20|55|PM|first|afternoon|lunch| +48056|AAAAAAAAJLLLAAAA|48056|13|20|56|PM|first|afternoon|lunch| +48057|AAAAAAAAKLLLAAAA|48057|13|20|57|PM|first|afternoon|lunch| +48058|AAAAAAAALLLLAAAA|48058|13|20|58|PM|first|afternoon|lunch| +48059|AAAAAAAAMLLLAAAA|48059|13|20|59|PM|first|afternoon|lunch| +48060|AAAAAAAANLLLAAAA|48060|13|21|0|PM|first|afternoon|lunch| +48061|AAAAAAAAOLLLAAAA|48061|13|21|1|PM|first|afternoon|lunch| +48062|AAAAAAAAPLLLAAAA|48062|13|21|2|PM|first|afternoon|lunch| +48063|AAAAAAAAAMLLAAAA|48063|13|21|3|PM|first|afternoon|lunch| +48064|AAAAAAAABMLLAAAA|48064|13|21|4|PM|first|afternoon|lunch| +48065|AAAAAAAACMLLAAAA|48065|13|21|5|PM|first|afternoon|lunch| +48066|AAAAAAAADMLLAAAA|48066|13|21|6|PM|first|afternoon|lunch| +48067|AAAAAAAAEMLLAAAA|48067|13|21|7|PM|first|afternoon|lunch| +48068|AAAAAAAAFMLLAAAA|48068|13|21|8|PM|first|afternoon|lunch| +48069|AAAAAAAAGMLLAAAA|48069|13|21|9|PM|first|afternoon|lunch| +48070|AAAAAAAAHMLLAAAA|48070|13|21|10|PM|first|afternoon|lunch| +48071|AAAAAAAAIMLLAAAA|48071|13|21|11|PM|first|afternoon|lunch| +48072|AAAAAAAAJMLLAAAA|48072|13|21|12|PM|first|afternoon|lunch| +48073|AAAAAAAAKMLLAAAA|48073|13|21|13|PM|first|afternoon|lunch| +48074|AAAAAAAALMLLAAAA|48074|13|21|14|PM|first|afternoon|lunch| +48075|AAAAAAAAMMLLAAAA|48075|13|21|15|PM|first|afternoon|lunch| +48076|AAAAAAAANMLLAAAA|48076|13|21|16|PM|first|afternoon|lunch| +48077|AAAAAAAAOMLLAAAA|48077|13|21|17|PM|first|afternoon|lunch| +48078|AAAAAAAAPMLLAAAA|48078|13|21|18|PM|first|afternoon|lunch| +48079|AAAAAAAAANLLAAAA|48079|13|21|19|PM|first|afternoon|lunch| +48080|AAAAAAAABNLLAAAA|48080|13|21|20|PM|first|afternoon|lunch| +48081|AAAAAAAACNLLAAAA|48081|13|21|21|PM|first|afternoon|lunch| +48082|AAAAAAAADNLLAAAA|48082|13|21|22|PM|first|afternoon|lunch| +48083|AAAAAAAAENLLAAAA|48083|13|21|23|PM|first|afternoon|lunch| +48084|AAAAAAAAFNLLAAAA|48084|13|21|24|PM|first|afternoon|lunch| +48085|AAAAAAAAGNLLAAAA|48085|13|21|25|PM|first|afternoon|lunch| +48086|AAAAAAAAHNLLAAAA|48086|13|21|26|PM|first|afternoon|lunch| +48087|AAAAAAAAINLLAAAA|48087|13|21|27|PM|first|afternoon|lunch| +48088|AAAAAAAAJNLLAAAA|48088|13|21|28|PM|first|afternoon|lunch| +48089|AAAAAAAAKNLLAAAA|48089|13|21|29|PM|first|afternoon|lunch| +48090|AAAAAAAALNLLAAAA|48090|13|21|30|PM|first|afternoon|lunch| +48091|AAAAAAAAMNLLAAAA|48091|13|21|31|PM|first|afternoon|lunch| +48092|AAAAAAAANNLLAAAA|48092|13|21|32|PM|first|afternoon|lunch| +48093|AAAAAAAAONLLAAAA|48093|13|21|33|PM|first|afternoon|lunch| +48094|AAAAAAAAPNLLAAAA|48094|13|21|34|PM|first|afternoon|lunch| +48095|AAAAAAAAAOLLAAAA|48095|13|21|35|PM|first|afternoon|lunch| +48096|AAAAAAAABOLLAAAA|48096|13|21|36|PM|first|afternoon|lunch| +48097|AAAAAAAACOLLAAAA|48097|13|21|37|PM|first|afternoon|lunch| +48098|AAAAAAAADOLLAAAA|48098|13|21|38|PM|first|afternoon|lunch| +48099|AAAAAAAAEOLLAAAA|48099|13|21|39|PM|first|afternoon|lunch| +48100|AAAAAAAAFOLLAAAA|48100|13|21|40|PM|first|afternoon|lunch| +48101|AAAAAAAAGOLLAAAA|48101|13|21|41|PM|first|afternoon|lunch| +48102|AAAAAAAAHOLLAAAA|48102|13|21|42|PM|first|afternoon|lunch| +48103|AAAAAAAAIOLLAAAA|48103|13|21|43|PM|first|afternoon|lunch| +48104|AAAAAAAAJOLLAAAA|48104|13|21|44|PM|first|afternoon|lunch| +48105|AAAAAAAAKOLLAAAA|48105|13|21|45|PM|first|afternoon|lunch| +48106|AAAAAAAALOLLAAAA|48106|13|21|46|PM|first|afternoon|lunch| +48107|AAAAAAAAMOLLAAAA|48107|13|21|47|PM|first|afternoon|lunch| +48108|AAAAAAAANOLLAAAA|48108|13|21|48|PM|first|afternoon|lunch| +48109|AAAAAAAAOOLLAAAA|48109|13|21|49|PM|first|afternoon|lunch| +48110|AAAAAAAAPOLLAAAA|48110|13|21|50|PM|first|afternoon|lunch| +48111|AAAAAAAAAPLLAAAA|48111|13|21|51|PM|first|afternoon|lunch| +48112|AAAAAAAABPLLAAAA|48112|13|21|52|PM|first|afternoon|lunch| +48113|AAAAAAAACPLLAAAA|48113|13|21|53|PM|first|afternoon|lunch| +48114|AAAAAAAADPLLAAAA|48114|13|21|54|PM|first|afternoon|lunch| +48115|AAAAAAAAEPLLAAAA|48115|13|21|55|PM|first|afternoon|lunch| +48116|AAAAAAAAFPLLAAAA|48116|13|21|56|PM|first|afternoon|lunch| +48117|AAAAAAAAGPLLAAAA|48117|13|21|57|PM|first|afternoon|lunch| +48118|AAAAAAAAHPLLAAAA|48118|13|21|58|PM|first|afternoon|lunch| +48119|AAAAAAAAIPLLAAAA|48119|13|21|59|PM|first|afternoon|lunch| +48120|AAAAAAAAJPLLAAAA|48120|13|22|0|PM|first|afternoon|lunch| +48121|AAAAAAAAKPLLAAAA|48121|13|22|1|PM|first|afternoon|lunch| +48122|AAAAAAAALPLLAAAA|48122|13|22|2|PM|first|afternoon|lunch| +48123|AAAAAAAAMPLLAAAA|48123|13|22|3|PM|first|afternoon|lunch| +48124|AAAAAAAANPLLAAAA|48124|13|22|4|PM|first|afternoon|lunch| +48125|AAAAAAAAOPLLAAAA|48125|13|22|5|PM|first|afternoon|lunch| +48126|AAAAAAAAPPLLAAAA|48126|13|22|6|PM|first|afternoon|lunch| +48127|AAAAAAAAAAMLAAAA|48127|13|22|7|PM|first|afternoon|lunch| +48128|AAAAAAAABAMLAAAA|48128|13|22|8|PM|first|afternoon|lunch| +48129|AAAAAAAACAMLAAAA|48129|13|22|9|PM|first|afternoon|lunch| +48130|AAAAAAAADAMLAAAA|48130|13|22|10|PM|first|afternoon|lunch| +48131|AAAAAAAAEAMLAAAA|48131|13|22|11|PM|first|afternoon|lunch| +48132|AAAAAAAAFAMLAAAA|48132|13|22|12|PM|first|afternoon|lunch| +48133|AAAAAAAAGAMLAAAA|48133|13|22|13|PM|first|afternoon|lunch| +48134|AAAAAAAAHAMLAAAA|48134|13|22|14|PM|first|afternoon|lunch| +48135|AAAAAAAAIAMLAAAA|48135|13|22|15|PM|first|afternoon|lunch| +48136|AAAAAAAAJAMLAAAA|48136|13|22|16|PM|first|afternoon|lunch| +48137|AAAAAAAAKAMLAAAA|48137|13|22|17|PM|first|afternoon|lunch| +48138|AAAAAAAALAMLAAAA|48138|13|22|18|PM|first|afternoon|lunch| +48139|AAAAAAAAMAMLAAAA|48139|13|22|19|PM|first|afternoon|lunch| +48140|AAAAAAAANAMLAAAA|48140|13|22|20|PM|first|afternoon|lunch| +48141|AAAAAAAAOAMLAAAA|48141|13|22|21|PM|first|afternoon|lunch| +48142|AAAAAAAAPAMLAAAA|48142|13|22|22|PM|first|afternoon|lunch| +48143|AAAAAAAAABMLAAAA|48143|13|22|23|PM|first|afternoon|lunch| +48144|AAAAAAAABBMLAAAA|48144|13|22|24|PM|first|afternoon|lunch| +48145|AAAAAAAACBMLAAAA|48145|13|22|25|PM|first|afternoon|lunch| +48146|AAAAAAAADBMLAAAA|48146|13|22|26|PM|first|afternoon|lunch| +48147|AAAAAAAAEBMLAAAA|48147|13|22|27|PM|first|afternoon|lunch| +48148|AAAAAAAAFBMLAAAA|48148|13|22|28|PM|first|afternoon|lunch| +48149|AAAAAAAAGBMLAAAA|48149|13|22|29|PM|first|afternoon|lunch| +48150|AAAAAAAAHBMLAAAA|48150|13|22|30|PM|first|afternoon|lunch| +48151|AAAAAAAAIBMLAAAA|48151|13|22|31|PM|first|afternoon|lunch| +48152|AAAAAAAAJBMLAAAA|48152|13|22|32|PM|first|afternoon|lunch| +48153|AAAAAAAAKBMLAAAA|48153|13|22|33|PM|first|afternoon|lunch| +48154|AAAAAAAALBMLAAAA|48154|13|22|34|PM|first|afternoon|lunch| +48155|AAAAAAAAMBMLAAAA|48155|13|22|35|PM|first|afternoon|lunch| +48156|AAAAAAAANBMLAAAA|48156|13|22|36|PM|first|afternoon|lunch| +48157|AAAAAAAAOBMLAAAA|48157|13|22|37|PM|first|afternoon|lunch| +48158|AAAAAAAAPBMLAAAA|48158|13|22|38|PM|first|afternoon|lunch| +48159|AAAAAAAAACMLAAAA|48159|13|22|39|PM|first|afternoon|lunch| +48160|AAAAAAAABCMLAAAA|48160|13|22|40|PM|first|afternoon|lunch| +48161|AAAAAAAACCMLAAAA|48161|13|22|41|PM|first|afternoon|lunch| +48162|AAAAAAAADCMLAAAA|48162|13|22|42|PM|first|afternoon|lunch| +48163|AAAAAAAAECMLAAAA|48163|13|22|43|PM|first|afternoon|lunch| +48164|AAAAAAAAFCMLAAAA|48164|13|22|44|PM|first|afternoon|lunch| +48165|AAAAAAAAGCMLAAAA|48165|13|22|45|PM|first|afternoon|lunch| +48166|AAAAAAAAHCMLAAAA|48166|13|22|46|PM|first|afternoon|lunch| +48167|AAAAAAAAICMLAAAA|48167|13|22|47|PM|first|afternoon|lunch| +48168|AAAAAAAAJCMLAAAA|48168|13|22|48|PM|first|afternoon|lunch| +48169|AAAAAAAAKCMLAAAA|48169|13|22|49|PM|first|afternoon|lunch| +48170|AAAAAAAALCMLAAAA|48170|13|22|50|PM|first|afternoon|lunch| +48171|AAAAAAAAMCMLAAAA|48171|13|22|51|PM|first|afternoon|lunch| +48172|AAAAAAAANCMLAAAA|48172|13|22|52|PM|first|afternoon|lunch| +48173|AAAAAAAAOCMLAAAA|48173|13|22|53|PM|first|afternoon|lunch| +48174|AAAAAAAAPCMLAAAA|48174|13|22|54|PM|first|afternoon|lunch| +48175|AAAAAAAAADMLAAAA|48175|13|22|55|PM|first|afternoon|lunch| +48176|AAAAAAAABDMLAAAA|48176|13|22|56|PM|first|afternoon|lunch| +48177|AAAAAAAACDMLAAAA|48177|13|22|57|PM|first|afternoon|lunch| +48178|AAAAAAAADDMLAAAA|48178|13|22|58|PM|first|afternoon|lunch| +48179|AAAAAAAAEDMLAAAA|48179|13|22|59|PM|first|afternoon|lunch| +48180|AAAAAAAAFDMLAAAA|48180|13|23|0|PM|first|afternoon|lunch| +48181|AAAAAAAAGDMLAAAA|48181|13|23|1|PM|first|afternoon|lunch| +48182|AAAAAAAAHDMLAAAA|48182|13|23|2|PM|first|afternoon|lunch| +48183|AAAAAAAAIDMLAAAA|48183|13|23|3|PM|first|afternoon|lunch| +48184|AAAAAAAAJDMLAAAA|48184|13|23|4|PM|first|afternoon|lunch| +48185|AAAAAAAAKDMLAAAA|48185|13|23|5|PM|first|afternoon|lunch| +48186|AAAAAAAALDMLAAAA|48186|13|23|6|PM|first|afternoon|lunch| +48187|AAAAAAAAMDMLAAAA|48187|13|23|7|PM|first|afternoon|lunch| +48188|AAAAAAAANDMLAAAA|48188|13|23|8|PM|first|afternoon|lunch| +48189|AAAAAAAAODMLAAAA|48189|13|23|9|PM|first|afternoon|lunch| +48190|AAAAAAAAPDMLAAAA|48190|13|23|10|PM|first|afternoon|lunch| +48191|AAAAAAAAAEMLAAAA|48191|13|23|11|PM|first|afternoon|lunch| +48192|AAAAAAAABEMLAAAA|48192|13|23|12|PM|first|afternoon|lunch| +48193|AAAAAAAACEMLAAAA|48193|13|23|13|PM|first|afternoon|lunch| +48194|AAAAAAAADEMLAAAA|48194|13|23|14|PM|first|afternoon|lunch| +48195|AAAAAAAAEEMLAAAA|48195|13|23|15|PM|first|afternoon|lunch| +48196|AAAAAAAAFEMLAAAA|48196|13|23|16|PM|first|afternoon|lunch| +48197|AAAAAAAAGEMLAAAA|48197|13|23|17|PM|first|afternoon|lunch| +48198|AAAAAAAAHEMLAAAA|48198|13|23|18|PM|first|afternoon|lunch| +48199|AAAAAAAAIEMLAAAA|48199|13|23|19|PM|first|afternoon|lunch| +48200|AAAAAAAAJEMLAAAA|48200|13|23|20|PM|first|afternoon|lunch| +48201|AAAAAAAAKEMLAAAA|48201|13|23|21|PM|first|afternoon|lunch| +48202|AAAAAAAALEMLAAAA|48202|13|23|22|PM|first|afternoon|lunch| +48203|AAAAAAAAMEMLAAAA|48203|13|23|23|PM|first|afternoon|lunch| +48204|AAAAAAAANEMLAAAA|48204|13|23|24|PM|first|afternoon|lunch| +48205|AAAAAAAAOEMLAAAA|48205|13|23|25|PM|first|afternoon|lunch| +48206|AAAAAAAAPEMLAAAA|48206|13|23|26|PM|first|afternoon|lunch| +48207|AAAAAAAAAFMLAAAA|48207|13|23|27|PM|first|afternoon|lunch| +48208|AAAAAAAABFMLAAAA|48208|13|23|28|PM|first|afternoon|lunch| +48209|AAAAAAAACFMLAAAA|48209|13|23|29|PM|first|afternoon|lunch| +48210|AAAAAAAADFMLAAAA|48210|13|23|30|PM|first|afternoon|lunch| +48211|AAAAAAAAEFMLAAAA|48211|13|23|31|PM|first|afternoon|lunch| +48212|AAAAAAAAFFMLAAAA|48212|13|23|32|PM|first|afternoon|lunch| +48213|AAAAAAAAGFMLAAAA|48213|13|23|33|PM|first|afternoon|lunch| +48214|AAAAAAAAHFMLAAAA|48214|13|23|34|PM|first|afternoon|lunch| +48215|AAAAAAAAIFMLAAAA|48215|13|23|35|PM|first|afternoon|lunch| +48216|AAAAAAAAJFMLAAAA|48216|13|23|36|PM|first|afternoon|lunch| +48217|AAAAAAAAKFMLAAAA|48217|13|23|37|PM|first|afternoon|lunch| +48218|AAAAAAAALFMLAAAA|48218|13|23|38|PM|first|afternoon|lunch| +48219|AAAAAAAAMFMLAAAA|48219|13|23|39|PM|first|afternoon|lunch| +48220|AAAAAAAANFMLAAAA|48220|13|23|40|PM|first|afternoon|lunch| +48221|AAAAAAAAOFMLAAAA|48221|13|23|41|PM|first|afternoon|lunch| +48222|AAAAAAAAPFMLAAAA|48222|13|23|42|PM|first|afternoon|lunch| +48223|AAAAAAAAAGMLAAAA|48223|13|23|43|PM|first|afternoon|lunch| +48224|AAAAAAAABGMLAAAA|48224|13|23|44|PM|first|afternoon|lunch| +48225|AAAAAAAACGMLAAAA|48225|13|23|45|PM|first|afternoon|lunch| +48226|AAAAAAAADGMLAAAA|48226|13|23|46|PM|first|afternoon|lunch| +48227|AAAAAAAAEGMLAAAA|48227|13|23|47|PM|first|afternoon|lunch| +48228|AAAAAAAAFGMLAAAA|48228|13|23|48|PM|first|afternoon|lunch| +48229|AAAAAAAAGGMLAAAA|48229|13|23|49|PM|first|afternoon|lunch| +48230|AAAAAAAAHGMLAAAA|48230|13|23|50|PM|first|afternoon|lunch| +48231|AAAAAAAAIGMLAAAA|48231|13|23|51|PM|first|afternoon|lunch| +48232|AAAAAAAAJGMLAAAA|48232|13|23|52|PM|first|afternoon|lunch| +48233|AAAAAAAAKGMLAAAA|48233|13|23|53|PM|first|afternoon|lunch| +48234|AAAAAAAALGMLAAAA|48234|13|23|54|PM|first|afternoon|lunch| +48235|AAAAAAAAMGMLAAAA|48235|13|23|55|PM|first|afternoon|lunch| +48236|AAAAAAAANGMLAAAA|48236|13|23|56|PM|first|afternoon|lunch| +48237|AAAAAAAAOGMLAAAA|48237|13|23|57|PM|first|afternoon|lunch| +48238|AAAAAAAAPGMLAAAA|48238|13|23|58|PM|first|afternoon|lunch| +48239|AAAAAAAAAHMLAAAA|48239|13|23|59|PM|first|afternoon|lunch| +48240|AAAAAAAABHMLAAAA|48240|13|24|0|PM|first|afternoon|lunch| +48241|AAAAAAAACHMLAAAA|48241|13|24|1|PM|first|afternoon|lunch| +48242|AAAAAAAADHMLAAAA|48242|13|24|2|PM|first|afternoon|lunch| +48243|AAAAAAAAEHMLAAAA|48243|13|24|3|PM|first|afternoon|lunch| +48244|AAAAAAAAFHMLAAAA|48244|13|24|4|PM|first|afternoon|lunch| +48245|AAAAAAAAGHMLAAAA|48245|13|24|5|PM|first|afternoon|lunch| +48246|AAAAAAAAHHMLAAAA|48246|13|24|6|PM|first|afternoon|lunch| +48247|AAAAAAAAIHMLAAAA|48247|13|24|7|PM|first|afternoon|lunch| +48248|AAAAAAAAJHMLAAAA|48248|13|24|8|PM|first|afternoon|lunch| +48249|AAAAAAAAKHMLAAAA|48249|13|24|9|PM|first|afternoon|lunch| +48250|AAAAAAAALHMLAAAA|48250|13|24|10|PM|first|afternoon|lunch| +48251|AAAAAAAAMHMLAAAA|48251|13|24|11|PM|first|afternoon|lunch| +48252|AAAAAAAANHMLAAAA|48252|13|24|12|PM|first|afternoon|lunch| +48253|AAAAAAAAOHMLAAAA|48253|13|24|13|PM|first|afternoon|lunch| +48254|AAAAAAAAPHMLAAAA|48254|13|24|14|PM|first|afternoon|lunch| +48255|AAAAAAAAAIMLAAAA|48255|13|24|15|PM|first|afternoon|lunch| +48256|AAAAAAAABIMLAAAA|48256|13|24|16|PM|first|afternoon|lunch| +48257|AAAAAAAACIMLAAAA|48257|13|24|17|PM|first|afternoon|lunch| +48258|AAAAAAAADIMLAAAA|48258|13|24|18|PM|first|afternoon|lunch| +48259|AAAAAAAAEIMLAAAA|48259|13|24|19|PM|first|afternoon|lunch| +48260|AAAAAAAAFIMLAAAA|48260|13|24|20|PM|first|afternoon|lunch| +48261|AAAAAAAAGIMLAAAA|48261|13|24|21|PM|first|afternoon|lunch| +48262|AAAAAAAAHIMLAAAA|48262|13|24|22|PM|first|afternoon|lunch| +48263|AAAAAAAAIIMLAAAA|48263|13|24|23|PM|first|afternoon|lunch| +48264|AAAAAAAAJIMLAAAA|48264|13|24|24|PM|first|afternoon|lunch| +48265|AAAAAAAAKIMLAAAA|48265|13|24|25|PM|first|afternoon|lunch| +48266|AAAAAAAALIMLAAAA|48266|13|24|26|PM|first|afternoon|lunch| +48267|AAAAAAAAMIMLAAAA|48267|13|24|27|PM|first|afternoon|lunch| +48268|AAAAAAAANIMLAAAA|48268|13|24|28|PM|first|afternoon|lunch| +48269|AAAAAAAAOIMLAAAA|48269|13|24|29|PM|first|afternoon|lunch| +48270|AAAAAAAAPIMLAAAA|48270|13|24|30|PM|first|afternoon|lunch| +48271|AAAAAAAAAJMLAAAA|48271|13|24|31|PM|first|afternoon|lunch| +48272|AAAAAAAABJMLAAAA|48272|13|24|32|PM|first|afternoon|lunch| +48273|AAAAAAAACJMLAAAA|48273|13|24|33|PM|first|afternoon|lunch| +48274|AAAAAAAADJMLAAAA|48274|13|24|34|PM|first|afternoon|lunch| +48275|AAAAAAAAEJMLAAAA|48275|13|24|35|PM|first|afternoon|lunch| +48276|AAAAAAAAFJMLAAAA|48276|13|24|36|PM|first|afternoon|lunch| +48277|AAAAAAAAGJMLAAAA|48277|13|24|37|PM|first|afternoon|lunch| +48278|AAAAAAAAHJMLAAAA|48278|13|24|38|PM|first|afternoon|lunch| +48279|AAAAAAAAIJMLAAAA|48279|13|24|39|PM|first|afternoon|lunch| +48280|AAAAAAAAJJMLAAAA|48280|13|24|40|PM|first|afternoon|lunch| +48281|AAAAAAAAKJMLAAAA|48281|13|24|41|PM|first|afternoon|lunch| +48282|AAAAAAAALJMLAAAA|48282|13|24|42|PM|first|afternoon|lunch| +48283|AAAAAAAAMJMLAAAA|48283|13|24|43|PM|first|afternoon|lunch| +48284|AAAAAAAANJMLAAAA|48284|13|24|44|PM|first|afternoon|lunch| +48285|AAAAAAAAOJMLAAAA|48285|13|24|45|PM|first|afternoon|lunch| +48286|AAAAAAAAPJMLAAAA|48286|13|24|46|PM|first|afternoon|lunch| +48287|AAAAAAAAAKMLAAAA|48287|13|24|47|PM|first|afternoon|lunch| +48288|AAAAAAAABKMLAAAA|48288|13|24|48|PM|first|afternoon|lunch| +48289|AAAAAAAACKMLAAAA|48289|13|24|49|PM|first|afternoon|lunch| +48290|AAAAAAAADKMLAAAA|48290|13|24|50|PM|first|afternoon|lunch| +48291|AAAAAAAAEKMLAAAA|48291|13|24|51|PM|first|afternoon|lunch| +48292|AAAAAAAAFKMLAAAA|48292|13|24|52|PM|first|afternoon|lunch| +48293|AAAAAAAAGKMLAAAA|48293|13|24|53|PM|first|afternoon|lunch| +48294|AAAAAAAAHKMLAAAA|48294|13|24|54|PM|first|afternoon|lunch| +48295|AAAAAAAAIKMLAAAA|48295|13|24|55|PM|first|afternoon|lunch| +48296|AAAAAAAAJKMLAAAA|48296|13|24|56|PM|first|afternoon|lunch| +48297|AAAAAAAAKKMLAAAA|48297|13|24|57|PM|first|afternoon|lunch| +48298|AAAAAAAALKMLAAAA|48298|13|24|58|PM|first|afternoon|lunch| +48299|AAAAAAAAMKMLAAAA|48299|13|24|59|PM|first|afternoon|lunch| +48300|AAAAAAAANKMLAAAA|48300|13|25|0|PM|first|afternoon|lunch| +48301|AAAAAAAAOKMLAAAA|48301|13|25|1|PM|first|afternoon|lunch| +48302|AAAAAAAAPKMLAAAA|48302|13|25|2|PM|first|afternoon|lunch| +48303|AAAAAAAAALMLAAAA|48303|13|25|3|PM|first|afternoon|lunch| +48304|AAAAAAAABLMLAAAA|48304|13|25|4|PM|first|afternoon|lunch| +48305|AAAAAAAACLMLAAAA|48305|13|25|5|PM|first|afternoon|lunch| +48306|AAAAAAAADLMLAAAA|48306|13|25|6|PM|first|afternoon|lunch| +48307|AAAAAAAAELMLAAAA|48307|13|25|7|PM|first|afternoon|lunch| +48308|AAAAAAAAFLMLAAAA|48308|13|25|8|PM|first|afternoon|lunch| +48309|AAAAAAAAGLMLAAAA|48309|13|25|9|PM|first|afternoon|lunch| +48310|AAAAAAAAHLMLAAAA|48310|13|25|10|PM|first|afternoon|lunch| +48311|AAAAAAAAILMLAAAA|48311|13|25|11|PM|first|afternoon|lunch| +48312|AAAAAAAAJLMLAAAA|48312|13|25|12|PM|first|afternoon|lunch| +48313|AAAAAAAAKLMLAAAA|48313|13|25|13|PM|first|afternoon|lunch| +48314|AAAAAAAALLMLAAAA|48314|13|25|14|PM|first|afternoon|lunch| +48315|AAAAAAAAMLMLAAAA|48315|13|25|15|PM|first|afternoon|lunch| +48316|AAAAAAAANLMLAAAA|48316|13|25|16|PM|first|afternoon|lunch| +48317|AAAAAAAAOLMLAAAA|48317|13|25|17|PM|first|afternoon|lunch| +48318|AAAAAAAAPLMLAAAA|48318|13|25|18|PM|first|afternoon|lunch| +48319|AAAAAAAAAMMLAAAA|48319|13|25|19|PM|first|afternoon|lunch| +48320|AAAAAAAABMMLAAAA|48320|13|25|20|PM|first|afternoon|lunch| +48321|AAAAAAAACMMLAAAA|48321|13|25|21|PM|first|afternoon|lunch| +48322|AAAAAAAADMMLAAAA|48322|13|25|22|PM|first|afternoon|lunch| +48323|AAAAAAAAEMMLAAAA|48323|13|25|23|PM|first|afternoon|lunch| +48324|AAAAAAAAFMMLAAAA|48324|13|25|24|PM|first|afternoon|lunch| +48325|AAAAAAAAGMMLAAAA|48325|13|25|25|PM|first|afternoon|lunch| +48326|AAAAAAAAHMMLAAAA|48326|13|25|26|PM|first|afternoon|lunch| +48327|AAAAAAAAIMMLAAAA|48327|13|25|27|PM|first|afternoon|lunch| +48328|AAAAAAAAJMMLAAAA|48328|13|25|28|PM|first|afternoon|lunch| +48329|AAAAAAAAKMMLAAAA|48329|13|25|29|PM|first|afternoon|lunch| +48330|AAAAAAAALMMLAAAA|48330|13|25|30|PM|first|afternoon|lunch| +48331|AAAAAAAAMMMLAAAA|48331|13|25|31|PM|first|afternoon|lunch| +48332|AAAAAAAANMMLAAAA|48332|13|25|32|PM|first|afternoon|lunch| +48333|AAAAAAAAOMMLAAAA|48333|13|25|33|PM|first|afternoon|lunch| +48334|AAAAAAAAPMMLAAAA|48334|13|25|34|PM|first|afternoon|lunch| +48335|AAAAAAAAANMLAAAA|48335|13|25|35|PM|first|afternoon|lunch| +48336|AAAAAAAABNMLAAAA|48336|13|25|36|PM|first|afternoon|lunch| +48337|AAAAAAAACNMLAAAA|48337|13|25|37|PM|first|afternoon|lunch| +48338|AAAAAAAADNMLAAAA|48338|13|25|38|PM|first|afternoon|lunch| +48339|AAAAAAAAENMLAAAA|48339|13|25|39|PM|first|afternoon|lunch| +48340|AAAAAAAAFNMLAAAA|48340|13|25|40|PM|first|afternoon|lunch| +48341|AAAAAAAAGNMLAAAA|48341|13|25|41|PM|first|afternoon|lunch| +48342|AAAAAAAAHNMLAAAA|48342|13|25|42|PM|first|afternoon|lunch| +48343|AAAAAAAAINMLAAAA|48343|13|25|43|PM|first|afternoon|lunch| +48344|AAAAAAAAJNMLAAAA|48344|13|25|44|PM|first|afternoon|lunch| +48345|AAAAAAAAKNMLAAAA|48345|13|25|45|PM|first|afternoon|lunch| +48346|AAAAAAAALNMLAAAA|48346|13|25|46|PM|first|afternoon|lunch| +48347|AAAAAAAAMNMLAAAA|48347|13|25|47|PM|first|afternoon|lunch| +48348|AAAAAAAANNMLAAAA|48348|13|25|48|PM|first|afternoon|lunch| +48349|AAAAAAAAONMLAAAA|48349|13|25|49|PM|first|afternoon|lunch| +48350|AAAAAAAAPNMLAAAA|48350|13|25|50|PM|first|afternoon|lunch| +48351|AAAAAAAAAOMLAAAA|48351|13|25|51|PM|first|afternoon|lunch| +48352|AAAAAAAABOMLAAAA|48352|13|25|52|PM|first|afternoon|lunch| +48353|AAAAAAAACOMLAAAA|48353|13|25|53|PM|first|afternoon|lunch| +48354|AAAAAAAADOMLAAAA|48354|13|25|54|PM|first|afternoon|lunch| +48355|AAAAAAAAEOMLAAAA|48355|13|25|55|PM|first|afternoon|lunch| +48356|AAAAAAAAFOMLAAAA|48356|13|25|56|PM|first|afternoon|lunch| +48357|AAAAAAAAGOMLAAAA|48357|13|25|57|PM|first|afternoon|lunch| +48358|AAAAAAAAHOMLAAAA|48358|13|25|58|PM|first|afternoon|lunch| +48359|AAAAAAAAIOMLAAAA|48359|13|25|59|PM|first|afternoon|lunch| +48360|AAAAAAAAJOMLAAAA|48360|13|26|0|PM|first|afternoon|lunch| +48361|AAAAAAAAKOMLAAAA|48361|13|26|1|PM|first|afternoon|lunch| +48362|AAAAAAAALOMLAAAA|48362|13|26|2|PM|first|afternoon|lunch| +48363|AAAAAAAAMOMLAAAA|48363|13|26|3|PM|first|afternoon|lunch| +48364|AAAAAAAANOMLAAAA|48364|13|26|4|PM|first|afternoon|lunch| +48365|AAAAAAAAOOMLAAAA|48365|13|26|5|PM|first|afternoon|lunch| +48366|AAAAAAAAPOMLAAAA|48366|13|26|6|PM|first|afternoon|lunch| +48367|AAAAAAAAAPMLAAAA|48367|13|26|7|PM|first|afternoon|lunch| +48368|AAAAAAAABPMLAAAA|48368|13|26|8|PM|first|afternoon|lunch| +48369|AAAAAAAACPMLAAAA|48369|13|26|9|PM|first|afternoon|lunch| +48370|AAAAAAAADPMLAAAA|48370|13|26|10|PM|first|afternoon|lunch| +48371|AAAAAAAAEPMLAAAA|48371|13|26|11|PM|first|afternoon|lunch| +48372|AAAAAAAAFPMLAAAA|48372|13|26|12|PM|first|afternoon|lunch| +48373|AAAAAAAAGPMLAAAA|48373|13|26|13|PM|first|afternoon|lunch| +48374|AAAAAAAAHPMLAAAA|48374|13|26|14|PM|first|afternoon|lunch| +48375|AAAAAAAAIPMLAAAA|48375|13|26|15|PM|first|afternoon|lunch| +48376|AAAAAAAAJPMLAAAA|48376|13|26|16|PM|first|afternoon|lunch| +48377|AAAAAAAAKPMLAAAA|48377|13|26|17|PM|first|afternoon|lunch| +48378|AAAAAAAALPMLAAAA|48378|13|26|18|PM|first|afternoon|lunch| +48379|AAAAAAAAMPMLAAAA|48379|13|26|19|PM|first|afternoon|lunch| +48380|AAAAAAAANPMLAAAA|48380|13|26|20|PM|first|afternoon|lunch| +48381|AAAAAAAAOPMLAAAA|48381|13|26|21|PM|first|afternoon|lunch| +48382|AAAAAAAAPPMLAAAA|48382|13|26|22|PM|first|afternoon|lunch| +48383|AAAAAAAAAANLAAAA|48383|13|26|23|PM|first|afternoon|lunch| +48384|AAAAAAAABANLAAAA|48384|13|26|24|PM|first|afternoon|lunch| +48385|AAAAAAAACANLAAAA|48385|13|26|25|PM|first|afternoon|lunch| +48386|AAAAAAAADANLAAAA|48386|13|26|26|PM|first|afternoon|lunch| +48387|AAAAAAAAEANLAAAA|48387|13|26|27|PM|first|afternoon|lunch| +48388|AAAAAAAAFANLAAAA|48388|13|26|28|PM|first|afternoon|lunch| +48389|AAAAAAAAGANLAAAA|48389|13|26|29|PM|first|afternoon|lunch| +48390|AAAAAAAAHANLAAAA|48390|13|26|30|PM|first|afternoon|lunch| +48391|AAAAAAAAIANLAAAA|48391|13|26|31|PM|first|afternoon|lunch| +48392|AAAAAAAAJANLAAAA|48392|13|26|32|PM|first|afternoon|lunch| +48393|AAAAAAAAKANLAAAA|48393|13|26|33|PM|first|afternoon|lunch| +48394|AAAAAAAALANLAAAA|48394|13|26|34|PM|first|afternoon|lunch| +48395|AAAAAAAAMANLAAAA|48395|13|26|35|PM|first|afternoon|lunch| +48396|AAAAAAAANANLAAAA|48396|13|26|36|PM|first|afternoon|lunch| +48397|AAAAAAAAOANLAAAA|48397|13|26|37|PM|first|afternoon|lunch| +48398|AAAAAAAAPANLAAAA|48398|13|26|38|PM|first|afternoon|lunch| +48399|AAAAAAAAABNLAAAA|48399|13|26|39|PM|first|afternoon|lunch| +48400|AAAAAAAABBNLAAAA|48400|13|26|40|PM|first|afternoon|lunch| +48401|AAAAAAAACBNLAAAA|48401|13|26|41|PM|first|afternoon|lunch| +48402|AAAAAAAADBNLAAAA|48402|13|26|42|PM|first|afternoon|lunch| +48403|AAAAAAAAEBNLAAAA|48403|13|26|43|PM|first|afternoon|lunch| +48404|AAAAAAAAFBNLAAAA|48404|13|26|44|PM|first|afternoon|lunch| +48405|AAAAAAAAGBNLAAAA|48405|13|26|45|PM|first|afternoon|lunch| +48406|AAAAAAAAHBNLAAAA|48406|13|26|46|PM|first|afternoon|lunch| +48407|AAAAAAAAIBNLAAAA|48407|13|26|47|PM|first|afternoon|lunch| +48408|AAAAAAAAJBNLAAAA|48408|13|26|48|PM|first|afternoon|lunch| +48409|AAAAAAAAKBNLAAAA|48409|13|26|49|PM|first|afternoon|lunch| +48410|AAAAAAAALBNLAAAA|48410|13|26|50|PM|first|afternoon|lunch| +48411|AAAAAAAAMBNLAAAA|48411|13|26|51|PM|first|afternoon|lunch| +48412|AAAAAAAANBNLAAAA|48412|13|26|52|PM|first|afternoon|lunch| +48413|AAAAAAAAOBNLAAAA|48413|13|26|53|PM|first|afternoon|lunch| +48414|AAAAAAAAPBNLAAAA|48414|13|26|54|PM|first|afternoon|lunch| +48415|AAAAAAAAACNLAAAA|48415|13|26|55|PM|first|afternoon|lunch| +48416|AAAAAAAABCNLAAAA|48416|13|26|56|PM|first|afternoon|lunch| +48417|AAAAAAAACCNLAAAA|48417|13|26|57|PM|first|afternoon|lunch| +48418|AAAAAAAADCNLAAAA|48418|13|26|58|PM|first|afternoon|lunch| +48419|AAAAAAAAECNLAAAA|48419|13|26|59|PM|first|afternoon|lunch| +48420|AAAAAAAAFCNLAAAA|48420|13|27|0|PM|first|afternoon|lunch| +48421|AAAAAAAAGCNLAAAA|48421|13|27|1|PM|first|afternoon|lunch| +48422|AAAAAAAAHCNLAAAA|48422|13|27|2|PM|first|afternoon|lunch| +48423|AAAAAAAAICNLAAAA|48423|13|27|3|PM|first|afternoon|lunch| +48424|AAAAAAAAJCNLAAAA|48424|13|27|4|PM|first|afternoon|lunch| +48425|AAAAAAAAKCNLAAAA|48425|13|27|5|PM|first|afternoon|lunch| +48426|AAAAAAAALCNLAAAA|48426|13|27|6|PM|first|afternoon|lunch| +48427|AAAAAAAAMCNLAAAA|48427|13|27|7|PM|first|afternoon|lunch| +48428|AAAAAAAANCNLAAAA|48428|13|27|8|PM|first|afternoon|lunch| +48429|AAAAAAAAOCNLAAAA|48429|13|27|9|PM|first|afternoon|lunch| +48430|AAAAAAAAPCNLAAAA|48430|13|27|10|PM|first|afternoon|lunch| +48431|AAAAAAAAADNLAAAA|48431|13|27|11|PM|first|afternoon|lunch| +48432|AAAAAAAABDNLAAAA|48432|13|27|12|PM|first|afternoon|lunch| +48433|AAAAAAAACDNLAAAA|48433|13|27|13|PM|first|afternoon|lunch| +48434|AAAAAAAADDNLAAAA|48434|13|27|14|PM|first|afternoon|lunch| +48435|AAAAAAAAEDNLAAAA|48435|13|27|15|PM|first|afternoon|lunch| +48436|AAAAAAAAFDNLAAAA|48436|13|27|16|PM|first|afternoon|lunch| +48437|AAAAAAAAGDNLAAAA|48437|13|27|17|PM|first|afternoon|lunch| +48438|AAAAAAAAHDNLAAAA|48438|13|27|18|PM|first|afternoon|lunch| +48439|AAAAAAAAIDNLAAAA|48439|13|27|19|PM|first|afternoon|lunch| +48440|AAAAAAAAJDNLAAAA|48440|13|27|20|PM|first|afternoon|lunch| +48441|AAAAAAAAKDNLAAAA|48441|13|27|21|PM|first|afternoon|lunch| +48442|AAAAAAAALDNLAAAA|48442|13|27|22|PM|first|afternoon|lunch| +48443|AAAAAAAAMDNLAAAA|48443|13|27|23|PM|first|afternoon|lunch| +48444|AAAAAAAANDNLAAAA|48444|13|27|24|PM|first|afternoon|lunch| +48445|AAAAAAAAODNLAAAA|48445|13|27|25|PM|first|afternoon|lunch| +48446|AAAAAAAAPDNLAAAA|48446|13|27|26|PM|first|afternoon|lunch| +48447|AAAAAAAAAENLAAAA|48447|13|27|27|PM|first|afternoon|lunch| +48448|AAAAAAAABENLAAAA|48448|13|27|28|PM|first|afternoon|lunch| +48449|AAAAAAAACENLAAAA|48449|13|27|29|PM|first|afternoon|lunch| +48450|AAAAAAAADENLAAAA|48450|13|27|30|PM|first|afternoon|lunch| +48451|AAAAAAAAEENLAAAA|48451|13|27|31|PM|first|afternoon|lunch| +48452|AAAAAAAAFENLAAAA|48452|13|27|32|PM|first|afternoon|lunch| +48453|AAAAAAAAGENLAAAA|48453|13|27|33|PM|first|afternoon|lunch| +48454|AAAAAAAAHENLAAAA|48454|13|27|34|PM|first|afternoon|lunch| +48455|AAAAAAAAIENLAAAA|48455|13|27|35|PM|first|afternoon|lunch| +48456|AAAAAAAAJENLAAAA|48456|13|27|36|PM|first|afternoon|lunch| +48457|AAAAAAAAKENLAAAA|48457|13|27|37|PM|first|afternoon|lunch| +48458|AAAAAAAALENLAAAA|48458|13|27|38|PM|first|afternoon|lunch| +48459|AAAAAAAAMENLAAAA|48459|13|27|39|PM|first|afternoon|lunch| +48460|AAAAAAAANENLAAAA|48460|13|27|40|PM|first|afternoon|lunch| +48461|AAAAAAAAOENLAAAA|48461|13|27|41|PM|first|afternoon|lunch| +48462|AAAAAAAAPENLAAAA|48462|13|27|42|PM|first|afternoon|lunch| +48463|AAAAAAAAAFNLAAAA|48463|13|27|43|PM|first|afternoon|lunch| +48464|AAAAAAAABFNLAAAA|48464|13|27|44|PM|first|afternoon|lunch| +48465|AAAAAAAACFNLAAAA|48465|13|27|45|PM|first|afternoon|lunch| +48466|AAAAAAAADFNLAAAA|48466|13|27|46|PM|first|afternoon|lunch| +48467|AAAAAAAAEFNLAAAA|48467|13|27|47|PM|first|afternoon|lunch| +48468|AAAAAAAAFFNLAAAA|48468|13|27|48|PM|first|afternoon|lunch| +48469|AAAAAAAAGFNLAAAA|48469|13|27|49|PM|first|afternoon|lunch| +48470|AAAAAAAAHFNLAAAA|48470|13|27|50|PM|first|afternoon|lunch| +48471|AAAAAAAAIFNLAAAA|48471|13|27|51|PM|first|afternoon|lunch| +48472|AAAAAAAAJFNLAAAA|48472|13|27|52|PM|first|afternoon|lunch| +48473|AAAAAAAAKFNLAAAA|48473|13|27|53|PM|first|afternoon|lunch| +48474|AAAAAAAALFNLAAAA|48474|13|27|54|PM|first|afternoon|lunch| +48475|AAAAAAAAMFNLAAAA|48475|13|27|55|PM|first|afternoon|lunch| +48476|AAAAAAAANFNLAAAA|48476|13|27|56|PM|first|afternoon|lunch| +48477|AAAAAAAAOFNLAAAA|48477|13|27|57|PM|first|afternoon|lunch| +48478|AAAAAAAAPFNLAAAA|48478|13|27|58|PM|first|afternoon|lunch| +48479|AAAAAAAAAGNLAAAA|48479|13|27|59|PM|first|afternoon|lunch| +48480|AAAAAAAABGNLAAAA|48480|13|28|0|PM|first|afternoon|lunch| +48481|AAAAAAAACGNLAAAA|48481|13|28|1|PM|first|afternoon|lunch| +48482|AAAAAAAADGNLAAAA|48482|13|28|2|PM|first|afternoon|lunch| +48483|AAAAAAAAEGNLAAAA|48483|13|28|3|PM|first|afternoon|lunch| +48484|AAAAAAAAFGNLAAAA|48484|13|28|4|PM|first|afternoon|lunch| +48485|AAAAAAAAGGNLAAAA|48485|13|28|5|PM|first|afternoon|lunch| +48486|AAAAAAAAHGNLAAAA|48486|13|28|6|PM|first|afternoon|lunch| +48487|AAAAAAAAIGNLAAAA|48487|13|28|7|PM|first|afternoon|lunch| +48488|AAAAAAAAJGNLAAAA|48488|13|28|8|PM|first|afternoon|lunch| +48489|AAAAAAAAKGNLAAAA|48489|13|28|9|PM|first|afternoon|lunch| +48490|AAAAAAAALGNLAAAA|48490|13|28|10|PM|first|afternoon|lunch| +48491|AAAAAAAAMGNLAAAA|48491|13|28|11|PM|first|afternoon|lunch| +48492|AAAAAAAANGNLAAAA|48492|13|28|12|PM|first|afternoon|lunch| +48493|AAAAAAAAOGNLAAAA|48493|13|28|13|PM|first|afternoon|lunch| +48494|AAAAAAAAPGNLAAAA|48494|13|28|14|PM|first|afternoon|lunch| +48495|AAAAAAAAAHNLAAAA|48495|13|28|15|PM|first|afternoon|lunch| +48496|AAAAAAAABHNLAAAA|48496|13|28|16|PM|first|afternoon|lunch| +48497|AAAAAAAACHNLAAAA|48497|13|28|17|PM|first|afternoon|lunch| +48498|AAAAAAAADHNLAAAA|48498|13|28|18|PM|first|afternoon|lunch| +48499|AAAAAAAAEHNLAAAA|48499|13|28|19|PM|first|afternoon|lunch| +48500|AAAAAAAAFHNLAAAA|48500|13|28|20|PM|first|afternoon|lunch| +48501|AAAAAAAAGHNLAAAA|48501|13|28|21|PM|first|afternoon|lunch| +48502|AAAAAAAAHHNLAAAA|48502|13|28|22|PM|first|afternoon|lunch| +48503|AAAAAAAAIHNLAAAA|48503|13|28|23|PM|first|afternoon|lunch| +48504|AAAAAAAAJHNLAAAA|48504|13|28|24|PM|first|afternoon|lunch| +48505|AAAAAAAAKHNLAAAA|48505|13|28|25|PM|first|afternoon|lunch| +48506|AAAAAAAALHNLAAAA|48506|13|28|26|PM|first|afternoon|lunch| +48507|AAAAAAAAMHNLAAAA|48507|13|28|27|PM|first|afternoon|lunch| +48508|AAAAAAAANHNLAAAA|48508|13|28|28|PM|first|afternoon|lunch| +48509|AAAAAAAAOHNLAAAA|48509|13|28|29|PM|first|afternoon|lunch| +48510|AAAAAAAAPHNLAAAA|48510|13|28|30|PM|first|afternoon|lunch| +48511|AAAAAAAAAINLAAAA|48511|13|28|31|PM|first|afternoon|lunch| +48512|AAAAAAAABINLAAAA|48512|13|28|32|PM|first|afternoon|lunch| +48513|AAAAAAAACINLAAAA|48513|13|28|33|PM|first|afternoon|lunch| +48514|AAAAAAAADINLAAAA|48514|13|28|34|PM|first|afternoon|lunch| +48515|AAAAAAAAEINLAAAA|48515|13|28|35|PM|first|afternoon|lunch| +48516|AAAAAAAAFINLAAAA|48516|13|28|36|PM|first|afternoon|lunch| +48517|AAAAAAAAGINLAAAA|48517|13|28|37|PM|first|afternoon|lunch| +48518|AAAAAAAAHINLAAAA|48518|13|28|38|PM|first|afternoon|lunch| +48519|AAAAAAAAIINLAAAA|48519|13|28|39|PM|first|afternoon|lunch| +48520|AAAAAAAAJINLAAAA|48520|13|28|40|PM|first|afternoon|lunch| +48521|AAAAAAAAKINLAAAA|48521|13|28|41|PM|first|afternoon|lunch| +48522|AAAAAAAALINLAAAA|48522|13|28|42|PM|first|afternoon|lunch| +48523|AAAAAAAAMINLAAAA|48523|13|28|43|PM|first|afternoon|lunch| +48524|AAAAAAAANINLAAAA|48524|13|28|44|PM|first|afternoon|lunch| +48525|AAAAAAAAOINLAAAA|48525|13|28|45|PM|first|afternoon|lunch| +48526|AAAAAAAAPINLAAAA|48526|13|28|46|PM|first|afternoon|lunch| +48527|AAAAAAAAAJNLAAAA|48527|13|28|47|PM|first|afternoon|lunch| +48528|AAAAAAAABJNLAAAA|48528|13|28|48|PM|first|afternoon|lunch| +48529|AAAAAAAACJNLAAAA|48529|13|28|49|PM|first|afternoon|lunch| +48530|AAAAAAAADJNLAAAA|48530|13|28|50|PM|first|afternoon|lunch| +48531|AAAAAAAAEJNLAAAA|48531|13|28|51|PM|first|afternoon|lunch| +48532|AAAAAAAAFJNLAAAA|48532|13|28|52|PM|first|afternoon|lunch| +48533|AAAAAAAAGJNLAAAA|48533|13|28|53|PM|first|afternoon|lunch| +48534|AAAAAAAAHJNLAAAA|48534|13|28|54|PM|first|afternoon|lunch| +48535|AAAAAAAAIJNLAAAA|48535|13|28|55|PM|first|afternoon|lunch| +48536|AAAAAAAAJJNLAAAA|48536|13|28|56|PM|first|afternoon|lunch| +48537|AAAAAAAAKJNLAAAA|48537|13|28|57|PM|first|afternoon|lunch| +48538|AAAAAAAALJNLAAAA|48538|13|28|58|PM|first|afternoon|lunch| +48539|AAAAAAAAMJNLAAAA|48539|13|28|59|PM|first|afternoon|lunch| +48540|AAAAAAAANJNLAAAA|48540|13|29|0|PM|first|afternoon|lunch| +48541|AAAAAAAAOJNLAAAA|48541|13|29|1|PM|first|afternoon|lunch| +48542|AAAAAAAAPJNLAAAA|48542|13|29|2|PM|first|afternoon|lunch| +48543|AAAAAAAAAKNLAAAA|48543|13|29|3|PM|first|afternoon|lunch| +48544|AAAAAAAABKNLAAAA|48544|13|29|4|PM|first|afternoon|lunch| +48545|AAAAAAAACKNLAAAA|48545|13|29|5|PM|first|afternoon|lunch| +48546|AAAAAAAADKNLAAAA|48546|13|29|6|PM|first|afternoon|lunch| +48547|AAAAAAAAEKNLAAAA|48547|13|29|7|PM|first|afternoon|lunch| +48548|AAAAAAAAFKNLAAAA|48548|13|29|8|PM|first|afternoon|lunch| +48549|AAAAAAAAGKNLAAAA|48549|13|29|9|PM|first|afternoon|lunch| +48550|AAAAAAAAHKNLAAAA|48550|13|29|10|PM|first|afternoon|lunch| +48551|AAAAAAAAIKNLAAAA|48551|13|29|11|PM|first|afternoon|lunch| +48552|AAAAAAAAJKNLAAAA|48552|13|29|12|PM|first|afternoon|lunch| +48553|AAAAAAAAKKNLAAAA|48553|13|29|13|PM|first|afternoon|lunch| +48554|AAAAAAAALKNLAAAA|48554|13|29|14|PM|first|afternoon|lunch| +48555|AAAAAAAAMKNLAAAA|48555|13|29|15|PM|first|afternoon|lunch| +48556|AAAAAAAANKNLAAAA|48556|13|29|16|PM|first|afternoon|lunch| +48557|AAAAAAAAOKNLAAAA|48557|13|29|17|PM|first|afternoon|lunch| +48558|AAAAAAAAPKNLAAAA|48558|13|29|18|PM|first|afternoon|lunch| +48559|AAAAAAAAALNLAAAA|48559|13|29|19|PM|first|afternoon|lunch| +48560|AAAAAAAABLNLAAAA|48560|13|29|20|PM|first|afternoon|lunch| +48561|AAAAAAAACLNLAAAA|48561|13|29|21|PM|first|afternoon|lunch| +48562|AAAAAAAADLNLAAAA|48562|13|29|22|PM|first|afternoon|lunch| +48563|AAAAAAAAELNLAAAA|48563|13|29|23|PM|first|afternoon|lunch| +48564|AAAAAAAAFLNLAAAA|48564|13|29|24|PM|first|afternoon|lunch| +48565|AAAAAAAAGLNLAAAA|48565|13|29|25|PM|first|afternoon|lunch| +48566|AAAAAAAAHLNLAAAA|48566|13|29|26|PM|first|afternoon|lunch| +48567|AAAAAAAAILNLAAAA|48567|13|29|27|PM|first|afternoon|lunch| +48568|AAAAAAAAJLNLAAAA|48568|13|29|28|PM|first|afternoon|lunch| +48569|AAAAAAAAKLNLAAAA|48569|13|29|29|PM|first|afternoon|lunch| +48570|AAAAAAAALLNLAAAA|48570|13|29|30|PM|first|afternoon|lunch| +48571|AAAAAAAAMLNLAAAA|48571|13|29|31|PM|first|afternoon|lunch| +48572|AAAAAAAANLNLAAAA|48572|13|29|32|PM|first|afternoon|lunch| +48573|AAAAAAAAOLNLAAAA|48573|13|29|33|PM|first|afternoon|lunch| +48574|AAAAAAAAPLNLAAAA|48574|13|29|34|PM|first|afternoon|lunch| +48575|AAAAAAAAAMNLAAAA|48575|13|29|35|PM|first|afternoon|lunch| +48576|AAAAAAAABMNLAAAA|48576|13|29|36|PM|first|afternoon|lunch| +48577|AAAAAAAACMNLAAAA|48577|13|29|37|PM|first|afternoon|lunch| +48578|AAAAAAAADMNLAAAA|48578|13|29|38|PM|first|afternoon|lunch| +48579|AAAAAAAAEMNLAAAA|48579|13|29|39|PM|first|afternoon|lunch| +48580|AAAAAAAAFMNLAAAA|48580|13|29|40|PM|first|afternoon|lunch| +48581|AAAAAAAAGMNLAAAA|48581|13|29|41|PM|first|afternoon|lunch| +48582|AAAAAAAAHMNLAAAA|48582|13|29|42|PM|first|afternoon|lunch| +48583|AAAAAAAAIMNLAAAA|48583|13|29|43|PM|first|afternoon|lunch| +48584|AAAAAAAAJMNLAAAA|48584|13|29|44|PM|first|afternoon|lunch| +48585|AAAAAAAAKMNLAAAA|48585|13|29|45|PM|first|afternoon|lunch| +48586|AAAAAAAALMNLAAAA|48586|13|29|46|PM|first|afternoon|lunch| +48587|AAAAAAAAMMNLAAAA|48587|13|29|47|PM|first|afternoon|lunch| +48588|AAAAAAAANMNLAAAA|48588|13|29|48|PM|first|afternoon|lunch| +48589|AAAAAAAAOMNLAAAA|48589|13|29|49|PM|first|afternoon|lunch| +48590|AAAAAAAAPMNLAAAA|48590|13|29|50|PM|first|afternoon|lunch| +48591|AAAAAAAAANNLAAAA|48591|13|29|51|PM|first|afternoon|lunch| +48592|AAAAAAAABNNLAAAA|48592|13|29|52|PM|first|afternoon|lunch| +48593|AAAAAAAACNNLAAAA|48593|13|29|53|PM|first|afternoon|lunch| +48594|AAAAAAAADNNLAAAA|48594|13|29|54|PM|first|afternoon|lunch| +48595|AAAAAAAAENNLAAAA|48595|13|29|55|PM|first|afternoon|lunch| +48596|AAAAAAAAFNNLAAAA|48596|13|29|56|PM|first|afternoon|lunch| +48597|AAAAAAAAGNNLAAAA|48597|13|29|57|PM|first|afternoon|lunch| +48598|AAAAAAAAHNNLAAAA|48598|13|29|58|PM|first|afternoon|lunch| +48599|AAAAAAAAINNLAAAA|48599|13|29|59|PM|first|afternoon|lunch| +48600|AAAAAAAAJNNLAAAA|48600|13|30|0|PM|first|afternoon|lunch| +48601|AAAAAAAAKNNLAAAA|48601|13|30|1|PM|first|afternoon|lunch| +48602|AAAAAAAALNNLAAAA|48602|13|30|2|PM|first|afternoon|lunch| +48603|AAAAAAAAMNNLAAAA|48603|13|30|3|PM|first|afternoon|lunch| +48604|AAAAAAAANNNLAAAA|48604|13|30|4|PM|first|afternoon|lunch| +48605|AAAAAAAAONNLAAAA|48605|13|30|5|PM|first|afternoon|lunch| +48606|AAAAAAAAPNNLAAAA|48606|13|30|6|PM|first|afternoon|lunch| +48607|AAAAAAAAAONLAAAA|48607|13|30|7|PM|first|afternoon|lunch| +48608|AAAAAAAABONLAAAA|48608|13|30|8|PM|first|afternoon|lunch| +48609|AAAAAAAACONLAAAA|48609|13|30|9|PM|first|afternoon|lunch| +48610|AAAAAAAADONLAAAA|48610|13|30|10|PM|first|afternoon|lunch| +48611|AAAAAAAAEONLAAAA|48611|13|30|11|PM|first|afternoon|lunch| +48612|AAAAAAAAFONLAAAA|48612|13|30|12|PM|first|afternoon|lunch| +48613|AAAAAAAAGONLAAAA|48613|13|30|13|PM|first|afternoon|lunch| +48614|AAAAAAAAHONLAAAA|48614|13|30|14|PM|first|afternoon|lunch| +48615|AAAAAAAAIONLAAAA|48615|13|30|15|PM|first|afternoon|lunch| +48616|AAAAAAAAJONLAAAA|48616|13|30|16|PM|first|afternoon|lunch| +48617|AAAAAAAAKONLAAAA|48617|13|30|17|PM|first|afternoon|lunch| +48618|AAAAAAAALONLAAAA|48618|13|30|18|PM|first|afternoon|lunch| +48619|AAAAAAAAMONLAAAA|48619|13|30|19|PM|first|afternoon|lunch| +48620|AAAAAAAANONLAAAA|48620|13|30|20|PM|first|afternoon|lunch| +48621|AAAAAAAAOONLAAAA|48621|13|30|21|PM|first|afternoon|lunch| +48622|AAAAAAAAPONLAAAA|48622|13|30|22|PM|first|afternoon|lunch| +48623|AAAAAAAAAPNLAAAA|48623|13|30|23|PM|first|afternoon|lunch| +48624|AAAAAAAABPNLAAAA|48624|13|30|24|PM|first|afternoon|lunch| +48625|AAAAAAAACPNLAAAA|48625|13|30|25|PM|first|afternoon|lunch| +48626|AAAAAAAADPNLAAAA|48626|13|30|26|PM|first|afternoon|lunch| +48627|AAAAAAAAEPNLAAAA|48627|13|30|27|PM|first|afternoon|lunch| +48628|AAAAAAAAFPNLAAAA|48628|13|30|28|PM|first|afternoon|lunch| +48629|AAAAAAAAGPNLAAAA|48629|13|30|29|PM|first|afternoon|lunch| +48630|AAAAAAAAHPNLAAAA|48630|13|30|30|PM|first|afternoon|lunch| +48631|AAAAAAAAIPNLAAAA|48631|13|30|31|PM|first|afternoon|lunch| +48632|AAAAAAAAJPNLAAAA|48632|13|30|32|PM|first|afternoon|lunch| +48633|AAAAAAAAKPNLAAAA|48633|13|30|33|PM|first|afternoon|lunch| +48634|AAAAAAAALPNLAAAA|48634|13|30|34|PM|first|afternoon|lunch| +48635|AAAAAAAAMPNLAAAA|48635|13|30|35|PM|first|afternoon|lunch| +48636|AAAAAAAANPNLAAAA|48636|13|30|36|PM|first|afternoon|lunch| +48637|AAAAAAAAOPNLAAAA|48637|13|30|37|PM|first|afternoon|lunch| +48638|AAAAAAAAPPNLAAAA|48638|13|30|38|PM|first|afternoon|lunch| +48639|AAAAAAAAAAOLAAAA|48639|13|30|39|PM|first|afternoon|lunch| +48640|AAAAAAAABAOLAAAA|48640|13|30|40|PM|first|afternoon|lunch| +48641|AAAAAAAACAOLAAAA|48641|13|30|41|PM|first|afternoon|lunch| +48642|AAAAAAAADAOLAAAA|48642|13|30|42|PM|first|afternoon|lunch| +48643|AAAAAAAAEAOLAAAA|48643|13|30|43|PM|first|afternoon|lunch| +48644|AAAAAAAAFAOLAAAA|48644|13|30|44|PM|first|afternoon|lunch| +48645|AAAAAAAAGAOLAAAA|48645|13|30|45|PM|first|afternoon|lunch| +48646|AAAAAAAAHAOLAAAA|48646|13|30|46|PM|first|afternoon|lunch| +48647|AAAAAAAAIAOLAAAA|48647|13|30|47|PM|first|afternoon|lunch| +48648|AAAAAAAAJAOLAAAA|48648|13|30|48|PM|first|afternoon|lunch| +48649|AAAAAAAAKAOLAAAA|48649|13|30|49|PM|first|afternoon|lunch| +48650|AAAAAAAALAOLAAAA|48650|13|30|50|PM|first|afternoon|lunch| +48651|AAAAAAAAMAOLAAAA|48651|13|30|51|PM|first|afternoon|lunch| +48652|AAAAAAAANAOLAAAA|48652|13|30|52|PM|first|afternoon|lunch| +48653|AAAAAAAAOAOLAAAA|48653|13|30|53|PM|first|afternoon|lunch| +48654|AAAAAAAAPAOLAAAA|48654|13|30|54|PM|first|afternoon|lunch| +48655|AAAAAAAAABOLAAAA|48655|13|30|55|PM|first|afternoon|lunch| +48656|AAAAAAAABBOLAAAA|48656|13|30|56|PM|first|afternoon|lunch| +48657|AAAAAAAACBOLAAAA|48657|13|30|57|PM|first|afternoon|lunch| +48658|AAAAAAAADBOLAAAA|48658|13|30|58|PM|first|afternoon|lunch| +48659|AAAAAAAAEBOLAAAA|48659|13|30|59|PM|first|afternoon|lunch| +48660|AAAAAAAAFBOLAAAA|48660|13|31|0|PM|first|afternoon|lunch| +48661|AAAAAAAAGBOLAAAA|48661|13|31|1|PM|first|afternoon|lunch| +48662|AAAAAAAAHBOLAAAA|48662|13|31|2|PM|first|afternoon|lunch| +48663|AAAAAAAAIBOLAAAA|48663|13|31|3|PM|first|afternoon|lunch| +48664|AAAAAAAAJBOLAAAA|48664|13|31|4|PM|first|afternoon|lunch| +48665|AAAAAAAAKBOLAAAA|48665|13|31|5|PM|first|afternoon|lunch| +48666|AAAAAAAALBOLAAAA|48666|13|31|6|PM|first|afternoon|lunch| +48667|AAAAAAAAMBOLAAAA|48667|13|31|7|PM|first|afternoon|lunch| +48668|AAAAAAAANBOLAAAA|48668|13|31|8|PM|first|afternoon|lunch| +48669|AAAAAAAAOBOLAAAA|48669|13|31|9|PM|first|afternoon|lunch| +48670|AAAAAAAAPBOLAAAA|48670|13|31|10|PM|first|afternoon|lunch| +48671|AAAAAAAAACOLAAAA|48671|13|31|11|PM|first|afternoon|lunch| +48672|AAAAAAAABCOLAAAA|48672|13|31|12|PM|first|afternoon|lunch| +48673|AAAAAAAACCOLAAAA|48673|13|31|13|PM|first|afternoon|lunch| +48674|AAAAAAAADCOLAAAA|48674|13|31|14|PM|first|afternoon|lunch| +48675|AAAAAAAAECOLAAAA|48675|13|31|15|PM|first|afternoon|lunch| +48676|AAAAAAAAFCOLAAAA|48676|13|31|16|PM|first|afternoon|lunch| +48677|AAAAAAAAGCOLAAAA|48677|13|31|17|PM|first|afternoon|lunch| +48678|AAAAAAAAHCOLAAAA|48678|13|31|18|PM|first|afternoon|lunch| +48679|AAAAAAAAICOLAAAA|48679|13|31|19|PM|first|afternoon|lunch| +48680|AAAAAAAAJCOLAAAA|48680|13|31|20|PM|first|afternoon|lunch| +48681|AAAAAAAAKCOLAAAA|48681|13|31|21|PM|first|afternoon|lunch| +48682|AAAAAAAALCOLAAAA|48682|13|31|22|PM|first|afternoon|lunch| +48683|AAAAAAAAMCOLAAAA|48683|13|31|23|PM|first|afternoon|lunch| +48684|AAAAAAAANCOLAAAA|48684|13|31|24|PM|first|afternoon|lunch| +48685|AAAAAAAAOCOLAAAA|48685|13|31|25|PM|first|afternoon|lunch| +48686|AAAAAAAAPCOLAAAA|48686|13|31|26|PM|first|afternoon|lunch| +48687|AAAAAAAAADOLAAAA|48687|13|31|27|PM|first|afternoon|lunch| +48688|AAAAAAAABDOLAAAA|48688|13|31|28|PM|first|afternoon|lunch| +48689|AAAAAAAACDOLAAAA|48689|13|31|29|PM|first|afternoon|lunch| +48690|AAAAAAAADDOLAAAA|48690|13|31|30|PM|first|afternoon|lunch| +48691|AAAAAAAAEDOLAAAA|48691|13|31|31|PM|first|afternoon|lunch| +48692|AAAAAAAAFDOLAAAA|48692|13|31|32|PM|first|afternoon|lunch| +48693|AAAAAAAAGDOLAAAA|48693|13|31|33|PM|first|afternoon|lunch| +48694|AAAAAAAAHDOLAAAA|48694|13|31|34|PM|first|afternoon|lunch| +48695|AAAAAAAAIDOLAAAA|48695|13|31|35|PM|first|afternoon|lunch| +48696|AAAAAAAAJDOLAAAA|48696|13|31|36|PM|first|afternoon|lunch| +48697|AAAAAAAAKDOLAAAA|48697|13|31|37|PM|first|afternoon|lunch| +48698|AAAAAAAALDOLAAAA|48698|13|31|38|PM|first|afternoon|lunch| +48699|AAAAAAAAMDOLAAAA|48699|13|31|39|PM|first|afternoon|lunch| +48700|AAAAAAAANDOLAAAA|48700|13|31|40|PM|first|afternoon|lunch| +48701|AAAAAAAAODOLAAAA|48701|13|31|41|PM|first|afternoon|lunch| +48702|AAAAAAAAPDOLAAAA|48702|13|31|42|PM|first|afternoon|lunch| +48703|AAAAAAAAAEOLAAAA|48703|13|31|43|PM|first|afternoon|lunch| +48704|AAAAAAAABEOLAAAA|48704|13|31|44|PM|first|afternoon|lunch| +48705|AAAAAAAACEOLAAAA|48705|13|31|45|PM|first|afternoon|lunch| +48706|AAAAAAAADEOLAAAA|48706|13|31|46|PM|first|afternoon|lunch| +48707|AAAAAAAAEEOLAAAA|48707|13|31|47|PM|first|afternoon|lunch| +48708|AAAAAAAAFEOLAAAA|48708|13|31|48|PM|first|afternoon|lunch| +48709|AAAAAAAAGEOLAAAA|48709|13|31|49|PM|first|afternoon|lunch| +48710|AAAAAAAAHEOLAAAA|48710|13|31|50|PM|first|afternoon|lunch| +48711|AAAAAAAAIEOLAAAA|48711|13|31|51|PM|first|afternoon|lunch| +48712|AAAAAAAAJEOLAAAA|48712|13|31|52|PM|first|afternoon|lunch| +48713|AAAAAAAAKEOLAAAA|48713|13|31|53|PM|first|afternoon|lunch| +48714|AAAAAAAALEOLAAAA|48714|13|31|54|PM|first|afternoon|lunch| +48715|AAAAAAAAMEOLAAAA|48715|13|31|55|PM|first|afternoon|lunch| +48716|AAAAAAAANEOLAAAA|48716|13|31|56|PM|first|afternoon|lunch| +48717|AAAAAAAAOEOLAAAA|48717|13|31|57|PM|first|afternoon|lunch| +48718|AAAAAAAAPEOLAAAA|48718|13|31|58|PM|first|afternoon|lunch| +48719|AAAAAAAAAFOLAAAA|48719|13|31|59|PM|first|afternoon|lunch| +48720|AAAAAAAABFOLAAAA|48720|13|32|0|PM|first|afternoon|lunch| +48721|AAAAAAAACFOLAAAA|48721|13|32|1|PM|first|afternoon|lunch| +48722|AAAAAAAADFOLAAAA|48722|13|32|2|PM|first|afternoon|lunch| +48723|AAAAAAAAEFOLAAAA|48723|13|32|3|PM|first|afternoon|lunch| +48724|AAAAAAAAFFOLAAAA|48724|13|32|4|PM|first|afternoon|lunch| +48725|AAAAAAAAGFOLAAAA|48725|13|32|5|PM|first|afternoon|lunch| +48726|AAAAAAAAHFOLAAAA|48726|13|32|6|PM|first|afternoon|lunch| +48727|AAAAAAAAIFOLAAAA|48727|13|32|7|PM|first|afternoon|lunch| +48728|AAAAAAAAJFOLAAAA|48728|13|32|8|PM|first|afternoon|lunch| +48729|AAAAAAAAKFOLAAAA|48729|13|32|9|PM|first|afternoon|lunch| +48730|AAAAAAAALFOLAAAA|48730|13|32|10|PM|first|afternoon|lunch| +48731|AAAAAAAAMFOLAAAA|48731|13|32|11|PM|first|afternoon|lunch| +48732|AAAAAAAANFOLAAAA|48732|13|32|12|PM|first|afternoon|lunch| +48733|AAAAAAAAOFOLAAAA|48733|13|32|13|PM|first|afternoon|lunch| +48734|AAAAAAAAPFOLAAAA|48734|13|32|14|PM|first|afternoon|lunch| +48735|AAAAAAAAAGOLAAAA|48735|13|32|15|PM|first|afternoon|lunch| +48736|AAAAAAAABGOLAAAA|48736|13|32|16|PM|first|afternoon|lunch| +48737|AAAAAAAACGOLAAAA|48737|13|32|17|PM|first|afternoon|lunch| +48738|AAAAAAAADGOLAAAA|48738|13|32|18|PM|first|afternoon|lunch| +48739|AAAAAAAAEGOLAAAA|48739|13|32|19|PM|first|afternoon|lunch| +48740|AAAAAAAAFGOLAAAA|48740|13|32|20|PM|first|afternoon|lunch| +48741|AAAAAAAAGGOLAAAA|48741|13|32|21|PM|first|afternoon|lunch| +48742|AAAAAAAAHGOLAAAA|48742|13|32|22|PM|first|afternoon|lunch| +48743|AAAAAAAAIGOLAAAA|48743|13|32|23|PM|first|afternoon|lunch| +48744|AAAAAAAAJGOLAAAA|48744|13|32|24|PM|first|afternoon|lunch| +48745|AAAAAAAAKGOLAAAA|48745|13|32|25|PM|first|afternoon|lunch| +48746|AAAAAAAALGOLAAAA|48746|13|32|26|PM|first|afternoon|lunch| +48747|AAAAAAAAMGOLAAAA|48747|13|32|27|PM|first|afternoon|lunch| +48748|AAAAAAAANGOLAAAA|48748|13|32|28|PM|first|afternoon|lunch| +48749|AAAAAAAAOGOLAAAA|48749|13|32|29|PM|first|afternoon|lunch| +48750|AAAAAAAAPGOLAAAA|48750|13|32|30|PM|first|afternoon|lunch| +48751|AAAAAAAAAHOLAAAA|48751|13|32|31|PM|first|afternoon|lunch| +48752|AAAAAAAABHOLAAAA|48752|13|32|32|PM|first|afternoon|lunch| +48753|AAAAAAAACHOLAAAA|48753|13|32|33|PM|first|afternoon|lunch| +48754|AAAAAAAADHOLAAAA|48754|13|32|34|PM|first|afternoon|lunch| +48755|AAAAAAAAEHOLAAAA|48755|13|32|35|PM|first|afternoon|lunch| +48756|AAAAAAAAFHOLAAAA|48756|13|32|36|PM|first|afternoon|lunch| +48757|AAAAAAAAGHOLAAAA|48757|13|32|37|PM|first|afternoon|lunch| +48758|AAAAAAAAHHOLAAAA|48758|13|32|38|PM|first|afternoon|lunch| +48759|AAAAAAAAIHOLAAAA|48759|13|32|39|PM|first|afternoon|lunch| +48760|AAAAAAAAJHOLAAAA|48760|13|32|40|PM|first|afternoon|lunch| +48761|AAAAAAAAKHOLAAAA|48761|13|32|41|PM|first|afternoon|lunch| +48762|AAAAAAAALHOLAAAA|48762|13|32|42|PM|first|afternoon|lunch| +48763|AAAAAAAAMHOLAAAA|48763|13|32|43|PM|first|afternoon|lunch| +48764|AAAAAAAANHOLAAAA|48764|13|32|44|PM|first|afternoon|lunch| +48765|AAAAAAAAOHOLAAAA|48765|13|32|45|PM|first|afternoon|lunch| +48766|AAAAAAAAPHOLAAAA|48766|13|32|46|PM|first|afternoon|lunch| +48767|AAAAAAAAAIOLAAAA|48767|13|32|47|PM|first|afternoon|lunch| +48768|AAAAAAAABIOLAAAA|48768|13|32|48|PM|first|afternoon|lunch| +48769|AAAAAAAACIOLAAAA|48769|13|32|49|PM|first|afternoon|lunch| +48770|AAAAAAAADIOLAAAA|48770|13|32|50|PM|first|afternoon|lunch| +48771|AAAAAAAAEIOLAAAA|48771|13|32|51|PM|first|afternoon|lunch| +48772|AAAAAAAAFIOLAAAA|48772|13|32|52|PM|first|afternoon|lunch| +48773|AAAAAAAAGIOLAAAA|48773|13|32|53|PM|first|afternoon|lunch| +48774|AAAAAAAAHIOLAAAA|48774|13|32|54|PM|first|afternoon|lunch| +48775|AAAAAAAAIIOLAAAA|48775|13|32|55|PM|first|afternoon|lunch| +48776|AAAAAAAAJIOLAAAA|48776|13|32|56|PM|first|afternoon|lunch| +48777|AAAAAAAAKIOLAAAA|48777|13|32|57|PM|first|afternoon|lunch| +48778|AAAAAAAALIOLAAAA|48778|13|32|58|PM|first|afternoon|lunch| +48779|AAAAAAAAMIOLAAAA|48779|13|32|59|PM|first|afternoon|lunch| +48780|AAAAAAAANIOLAAAA|48780|13|33|0|PM|first|afternoon|lunch| +48781|AAAAAAAAOIOLAAAA|48781|13|33|1|PM|first|afternoon|lunch| +48782|AAAAAAAAPIOLAAAA|48782|13|33|2|PM|first|afternoon|lunch| +48783|AAAAAAAAAJOLAAAA|48783|13|33|3|PM|first|afternoon|lunch| +48784|AAAAAAAABJOLAAAA|48784|13|33|4|PM|first|afternoon|lunch| +48785|AAAAAAAACJOLAAAA|48785|13|33|5|PM|first|afternoon|lunch| +48786|AAAAAAAADJOLAAAA|48786|13|33|6|PM|first|afternoon|lunch| +48787|AAAAAAAAEJOLAAAA|48787|13|33|7|PM|first|afternoon|lunch| +48788|AAAAAAAAFJOLAAAA|48788|13|33|8|PM|first|afternoon|lunch| +48789|AAAAAAAAGJOLAAAA|48789|13|33|9|PM|first|afternoon|lunch| +48790|AAAAAAAAHJOLAAAA|48790|13|33|10|PM|first|afternoon|lunch| +48791|AAAAAAAAIJOLAAAA|48791|13|33|11|PM|first|afternoon|lunch| +48792|AAAAAAAAJJOLAAAA|48792|13|33|12|PM|first|afternoon|lunch| +48793|AAAAAAAAKJOLAAAA|48793|13|33|13|PM|first|afternoon|lunch| +48794|AAAAAAAALJOLAAAA|48794|13|33|14|PM|first|afternoon|lunch| +48795|AAAAAAAAMJOLAAAA|48795|13|33|15|PM|first|afternoon|lunch| +48796|AAAAAAAANJOLAAAA|48796|13|33|16|PM|first|afternoon|lunch| +48797|AAAAAAAAOJOLAAAA|48797|13|33|17|PM|first|afternoon|lunch| +48798|AAAAAAAAPJOLAAAA|48798|13|33|18|PM|first|afternoon|lunch| +48799|AAAAAAAAAKOLAAAA|48799|13|33|19|PM|first|afternoon|lunch| +48800|AAAAAAAABKOLAAAA|48800|13|33|20|PM|first|afternoon|lunch| +48801|AAAAAAAACKOLAAAA|48801|13|33|21|PM|first|afternoon|lunch| +48802|AAAAAAAADKOLAAAA|48802|13|33|22|PM|first|afternoon|lunch| +48803|AAAAAAAAEKOLAAAA|48803|13|33|23|PM|first|afternoon|lunch| +48804|AAAAAAAAFKOLAAAA|48804|13|33|24|PM|first|afternoon|lunch| +48805|AAAAAAAAGKOLAAAA|48805|13|33|25|PM|first|afternoon|lunch| +48806|AAAAAAAAHKOLAAAA|48806|13|33|26|PM|first|afternoon|lunch| +48807|AAAAAAAAIKOLAAAA|48807|13|33|27|PM|first|afternoon|lunch| +48808|AAAAAAAAJKOLAAAA|48808|13|33|28|PM|first|afternoon|lunch| +48809|AAAAAAAAKKOLAAAA|48809|13|33|29|PM|first|afternoon|lunch| +48810|AAAAAAAALKOLAAAA|48810|13|33|30|PM|first|afternoon|lunch| +48811|AAAAAAAAMKOLAAAA|48811|13|33|31|PM|first|afternoon|lunch| +48812|AAAAAAAANKOLAAAA|48812|13|33|32|PM|first|afternoon|lunch| +48813|AAAAAAAAOKOLAAAA|48813|13|33|33|PM|first|afternoon|lunch| +48814|AAAAAAAAPKOLAAAA|48814|13|33|34|PM|first|afternoon|lunch| +48815|AAAAAAAAALOLAAAA|48815|13|33|35|PM|first|afternoon|lunch| +48816|AAAAAAAABLOLAAAA|48816|13|33|36|PM|first|afternoon|lunch| +48817|AAAAAAAACLOLAAAA|48817|13|33|37|PM|first|afternoon|lunch| +48818|AAAAAAAADLOLAAAA|48818|13|33|38|PM|first|afternoon|lunch| +48819|AAAAAAAAELOLAAAA|48819|13|33|39|PM|first|afternoon|lunch| +48820|AAAAAAAAFLOLAAAA|48820|13|33|40|PM|first|afternoon|lunch| +48821|AAAAAAAAGLOLAAAA|48821|13|33|41|PM|first|afternoon|lunch| +48822|AAAAAAAAHLOLAAAA|48822|13|33|42|PM|first|afternoon|lunch| +48823|AAAAAAAAILOLAAAA|48823|13|33|43|PM|first|afternoon|lunch| +48824|AAAAAAAAJLOLAAAA|48824|13|33|44|PM|first|afternoon|lunch| +48825|AAAAAAAAKLOLAAAA|48825|13|33|45|PM|first|afternoon|lunch| +48826|AAAAAAAALLOLAAAA|48826|13|33|46|PM|first|afternoon|lunch| +48827|AAAAAAAAMLOLAAAA|48827|13|33|47|PM|first|afternoon|lunch| +48828|AAAAAAAANLOLAAAA|48828|13|33|48|PM|first|afternoon|lunch| +48829|AAAAAAAAOLOLAAAA|48829|13|33|49|PM|first|afternoon|lunch| +48830|AAAAAAAAPLOLAAAA|48830|13|33|50|PM|first|afternoon|lunch| +48831|AAAAAAAAAMOLAAAA|48831|13|33|51|PM|first|afternoon|lunch| +48832|AAAAAAAABMOLAAAA|48832|13|33|52|PM|first|afternoon|lunch| +48833|AAAAAAAACMOLAAAA|48833|13|33|53|PM|first|afternoon|lunch| +48834|AAAAAAAADMOLAAAA|48834|13|33|54|PM|first|afternoon|lunch| +48835|AAAAAAAAEMOLAAAA|48835|13|33|55|PM|first|afternoon|lunch| +48836|AAAAAAAAFMOLAAAA|48836|13|33|56|PM|first|afternoon|lunch| +48837|AAAAAAAAGMOLAAAA|48837|13|33|57|PM|first|afternoon|lunch| +48838|AAAAAAAAHMOLAAAA|48838|13|33|58|PM|first|afternoon|lunch| +48839|AAAAAAAAIMOLAAAA|48839|13|33|59|PM|first|afternoon|lunch| +48840|AAAAAAAAJMOLAAAA|48840|13|34|0|PM|first|afternoon|lunch| +48841|AAAAAAAAKMOLAAAA|48841|13|34|1|PM|first|afternoon|lunch| +48842|AAAAAAAALMOLAAAA|48842|13|34|2|PM|first|afternoon|lunch| +48843|AAAAAAAAMMOLAAAA|48843|13|34|3|PM|first|afternoon|lunch| +48844|AAAAAAAANMOLAAAA|48844|13|34|4|PM|first|afternoon|lunch| +48845|AAAAAAAAOMOLAAAA|48845|13|34|5|PM|first|afternoon|lunch| +48846|AAAAAAAAPMOLAAAA|48846|13|34|6|PM|first|afternoon|lunch| +48847|AAAAAAAAANOLAAAA|48847|13|34|7|PM|first|afternoon|lunch| +48848|AAAAAAAABNOLAAAA|48848|13|34|8|PM|first|afternoon|lunch| +48849|AAAAAAAACNOLAAAA|48849|13|34|9|PM|first|afternoon|lunch| +48850|AAAAAAAADNOLAAAA|48850|13|34|10|PM|first|afternoon|lunch| +48851|AAAAAAAAENOLAAAA|48851|13|34|11|PM|first|afternoon|lunch| +48852|AAAAAAAAFNOLAAAA|48852|13|34|12|PM|first|afternoon|lunch| +48853|AAAAAAAAGNOLAAAA|48853|13|34|13|PM|first|afternoon|lunch| +48854|AAAAAAAAHNOLAAAA|48854|13|34|14|PM|first|afternoon|lunch| +48855|AAAAAAAAINOLAAAA|48855|13|34|15|PM|first|afternoon|lunch| +48856|AAAAAAAAJNOLAAAA|48856|13|34|16|PM|first|afternoon|lunch| +48857|AAAAAAAAKNOLAAAA|48857|13|34|17|PM|first|afternoon|lunch| +48858|AAAAAAAALNOLAAAA|48858|13|34|18|PM|first|afternoon|lunch| +48859|AAAAAAAAMNOLAAAA|48859|13|34|19|PM|first|afternoon|lunch| +48860|AAAAAAAANNOLAAAA|48860|13|34|20|PM|first|afternoon|lunch| +48861|AAAAAAAAONOLAAAA|48861|13|34|21|PM|first|afternoon|lunch| +48862|AAAAAAAAPNOLAAAA|48862|13|34|22|PM|first|afternoon|lunch| +48863|AAAAAAAAAOOLAAAA|48863|13|34|23|PM|first|afternoon|lunch| +48864|AAAAAAAABOOLAAAA|48864|13|34|24|PM|first|afternoon|lunch| +48865|AAAAAAAACOOLAAAA|48865|13|34|25|PM|first|afternoon|lunch| +48866|AAAAAAAADOOLAAAA|48866|13|34|26|PM|first|afternoon|lunch| +48867|AAAAAAAAEOOLAAAA|48867|13|34|27|PM|first|afternoon|lunch| +48868|AAAAAAAAFOOLAAAA|48868|13|34|28|PM|first|afternoon|lunch| +48869|AAAAAAAAGOOLAAAA|48869|13|34|29|PM|first|afternoon|lunch| +48870|AAAAAAAAHOOLAAAA|48870|13|34|30|PM|first|afternoon|lunch| +48871|AAAAAAAAIOOLAAAA|48871|13|34|31|PM|first|afternoon|lunch| +48872|AAAAAAAAJOOLAAAA|48872|13|34|32|PM|first|afternoon|lunch| +48873|AAAAAAAAKOOLAAAA|48873|13|34|33|PM|first|afternoon|lunch| +48874|AAAAAAAALOOLAAAA|48874|13|34|34|PM|first|afternoon|lunch| +48875|AAAAAAAAMOOLAAAA|48875|13|34|35|PM|first|afternoon|lunch| +48876|AAAAAAAANOOLAAAA|48876|13|34|36|PM|first|afternoon|lunch| +48877|AAAAAAAAOOOLAAAA|48877|13|34|37|PM|first|afternoon|lunch| +48878|AAAAAAAAPOOLAAAA|48878|13|34|38|PM|first|afternoon|lunch| +48879|AAAAAAAAAPOLAAAA|48879|13|34|39|PM|first|afternoon|lunch| +48880|AAAAAAAABPOLAAAA|48880|13|34|40|PM|first|afternoon|lunch| +48881|AAAAAAAACPOLAAAA|48881|13|34|41|PM|first|afternoon|lunch| +48882|AAAAAAAADPOLAAAA|48882|13|34|42|PM|first|afternoon|lunch| +48883|AAAAAAAAEPOLAAAA|48883|13|34|43|PM|first|afternoon|lunch| +48884|AAAAAAAAFPOLAAAA|48884|13|34|44|PM|first|afternoon|lunch| +48885|AAAAAAAAGPOLAAAA|48885|13|34|45|PM|first|afternoon|lunch| +48886|AAAAAAAAHPOLAAAA|48886|13|34|46|PM|first|afternoon|lunch| +48887|AAAAAAAAIPOLAAAA|48887|13|34|47|PM|first|afternoon|lunch| +48888|AAAAAAAAJPOLAAAA|48888|13|34|48|PM|first|afternoon|lunch| +48889|AAAAAAAAKPOLAAAA|48889|13|34|49|PM|first|afternoon|lunch| +48890|AAAAAAAALPOLAAAA|48890|13|34|50|PM|first|afternoon|lunch| +48891|AAAAAAAAMPOLAAAA|48891|13|34|51|PM|first|afternoon|lunch| +48892|AAAAAAAANPOLAAAA|48892|13|34|52|PM|first|afternoon|lunch| +48893|AAAAAAAAOPOLAAAA|48893|13|34|53|PM|first|afternoon|lunch| +48894|AAAAAAAAPPOLAAAA|48894|13|34|54|PM|first|afternoon|lunch| +48895|AAAAAAAAAAPLAAAA|48895|13|34|55|PM|first|afternoon|lunch| +48896|AAAAAAAABAPLAAAA|48896|13|34|56|PM|first|afternoon|lunch| +48897|AAAAAAAACAPLAAAA|48897|13|34|57|PM|first|afternoon|lunch| +48898|AAAAAAAADAPLAAAA|48898|13|34|58|PM|first|afternoon|lunch| +48899|AAAAAAAAEAPLAAAA|48899|13|34|59|PM|first|afternoon|lunch| +48900|AAAAAAAAFAPLAAAA|48900|13|35|0|PM|first|afternoon|lunch| +48901|AAAAAAAAGAPLAAAA|48901|13|35|1|PM|first|afternoon|lunch| +48902|AAAAAAAAHAPLAAAA|48902|13|35|2|PM|first|afternoon|lunch| +48903|AAAAAAAAIAPLAAAA|48903|13|35|3|PM|first|afternoon|lunch| +48904|AAAAAAAAJAPLAAAA|48904|13|35|4|PM|first|afternoon|lunch| +48905|AAAAAAAAKAPLAAAA|48905|13|35|5|PM|first|afternoon|lunch| +48906|AAAAAAAALAPLAAAA|48906|13|35|6|PM|first|afternoon|lunch| +48907|AAAAAAAAMAPLAAAA|48907|13|35|7|PM|first|afternoon|lunch| +48908|AAAAAAAANAPLAAAA|48908|13|35|8|PM|first|afternoon|lunch| +48909|AAAAAAAAOAPLAAAA|48909|13|35|9|PM|first|afternoon|lunch| +48910|AAAAAAAAPAPLAAAA|48910|13|35|10|PM|first|afternoon|lunch| +48911|AAAAAAAAABPLAAAA|48911|13|35|11|PM|first|afternoon|lunch| +48912|AAAAAAAABBPLAAAA|48912|13|35|12|PM|first|afternoon|lunch| +48913|AAAAAAAACBPLAAAA|48913|13|35|13|PM|first|afternoon|lunch| +48914|AAAAAAAADBPLAAAA|48914|13|35|14|PM|first|afternoon|lunch| +48915|AAAAAAAAEBPLAAAA|48915|13|35|15|PM|first|afternoon|lunch| +48916|AAAAAAAAFBPLAAAA|48916|13|35|16|PM|first|afternoon|lunch| +48917|AAAAAAAAGBPLAAAA|48917|13|35|17|PM|first|afternoon|lunch| +48918|AAAAAAAAHBPLAAAA|48918|13|35|18|PM|first|afternoon|lunch| +48919|AAAAAAAAIBPLAAAA|48919|13|35|19|PM|first|afternoon|lunch| +48920|AAAAAAAAJBPLAAAA|48920|13|35|20|PM|first|afternoon|lunch| +48921|AAAAAAAAKBPLAAAA|48921|13|35|21|PM|first|afternoon|lunch| +48922|AAAAAAAALBPLAAAA|48922|13|35|22|PM|first|afternoon|lunch| +48923|AAAAAAAAMBPLAAAA|48923|13|35|23|PM|first|afternoon|lunch| +48924|AAAAAAAANBPLAAAA|48924|13|35|24|PM|first|afternoon|lunch| +48925|AAAAAAAAOBPLAAAA|48925|13|35|25|PM|first|afternoon|lunch| +48926|AAAAAAAAPBPLAAAA|48926|13|35|26|PM|first|afternoon|lunch| +48927|AAAAAAAAACPLAAAA|48927|13|35|27|PM|first|afternoon|lunch| +48928|AAAAAAAABCPLAAAA|48928|13|35|28|PM|first|afternoon|lunch| +48929|AAAAAAAACCPLAAAA|48929|13|35|29|PM|first|afternoon|lunch| +48930|AAAAAAAADCPLAAAA|48930|13|35|30|PM|first|afternoon|lunch| +48931|AAAAAAAAECPLAAAA|48931|13|35|31|PM|first|afternoon|lunch| +48932|AAAAAAAAFCPLAAAA|48932|13|35|32|PM|first|afternoon|lunch| +48933|AAAAAAAAGCPLAAAA|48933|13|35|33|PM|first|afternoon|lunch| +48934|AAAAAAAAHCPLAAAA|48934|13|35|34|PM|first|afternoon|lunch| +48935|AAAAAAAAICPLAAAA|48935|13|35|35|PM|first|afternoon|lunch| +48936|AAAAAAAAJCPLAAAA|48936|13|35|36|PM|first|afternoon|lunch| +48937|AAAAAAAAKCPLAAAA|48937|13|35|37|PM|first|afternoon|lunch| +48938|AAAAAAAALCPLAAAA|48938|13|35|38|PM|first|afternoon|lunch| +48939|AAAAAAAAMCPLAAAA|48939|13|35|39|PM|first|afternoon|lunch| +48940|AAAAAAAANCPLAAAA|48940|13|35|40|PM|first|afternoon|lunch| +48941|AAAAAAAAOCPLAAAA|48941|13|35|41|PM|first|afternoon|lunch| +48942|AAAAAAAAPCPLAAAA|48942|13|35|42|PM|first|afternoon|lunch| +48943|AAAAAAAAADPLAAAA|48943|13|35|43|PM|first|afternoon|lunch| +48944|AAAAAAAABDPLAAAA|48944|13|35|44|PM|first|afternoon|lunch| +48945|AAAAAAAACDPLAAAA|48945|13|35|45|PM|first|afternoon|lunch| +48946|AAAAAAAADDPLAAAA|48946|13|35|46|PM|first|afternoon|lunch| +48947|AAAAAAAAEDPLAAAA|48947|13|35|47|PM|first|afternoon|lunch| +48948|AAAAAAAAFDPLAAAA|48948|13|35|48|PM|first|afternoon|lunch| +48949|AAAAAAAAGDPLAAAA|48949|13|35|49|PM|first|afternoon|lunch| +48950|AAAAAAAAHDPLAAAA|48950|13|35|50|PM|first|afternoon|lunch| +48951|AAAAAAAAIDPLAAAA|48951|13|35|51|PM|first|afternoon|lunch| +48952|AAAAAAAAJDPLAAAA|48952|13|35|52|PM|first|afternoon|lunch| +48953|AAAAAAAAKDPLAAAA|48953|13|35|53|PM|first|afternoon|lunch| +48954|AAAAAAAALDPLAAAA|48954|13|35|54|PM|first|afternoon|lunch| +48955|AAAAAAAAMDPLAAAA|48955|13|35|55|PM|first|afternoon|lunch| +48956|AAAAAAAANDPLAAAA|48956|13|35|56|PM|first|afternoon|lunch| +48957|AAAAAAAAODPLAAAA|48957|13|35|57|PM|first|afternoon|lunch| +48958|AAAAAAAAPDPLAAAA|48958|13|35|58|PM|first|afternoon|lunch| +48959|AAAAAAAAAEPLAAAA|48959|13|35|59|PM|first|afternoon|lunch| +48960|AAAAAAAABEPLAAAA|48960|13|36|0|PM|first|afternoon|lunch| +48961|AAAAAAAACEPLAAAA|48961|13|36|1|PM|first|afternoon|lunch| +48962|AAAAAAAADEPLAAAA|48962|13|36|2|PM|first|afternoon|lunch| +48963|AAAAAAAAEEPLAAAA|48963|13|36|3|PM|first|afternoon|lunch| +48964|AAAAAAAAFEPLAAAA|48964|13|36|4|PM|first|afternoon|lunch| +48965|AAAAAAAAGEPLAAAA|48965|13|36|5|PM|first|afternoon|lunch| +48966|AAAAAAAAHEPLAAAA|48966|13|36|6|PM|first|afternoon|lunch| +48967|AAAAAAAAIEPLAAAA|48967|13|36|7|PM|first|afternoon|lunch| +48968|AAAAAAAAJEPLAAAA|48968|13|36|8|PM|first|afternoon|lunch| +48969|AAAAAAAAKEPLAAAA|48969|13|36|9|PM|first|afternoon|lunch| +48970|AAAAAAAALEPLAAAA|48970|13|36|10|PM|first|afternoon|lunch| +48971|AAAAAAAAMEPLAAAA|48971|13|36|11|PM|first|afternoon|lunch| +48972|AAAAAAAANEPLAAAA|48972|13|36|12|PM|first|afternoon|lunch| +48973|AAAAAAAAOEPLAAAA|48973|13|36|13|PM|first|afternoon|lunch| +48974|AAAAAAAAPEPLAAAA|48974|13|36|14|PM|first|afternoon|lunch| +48975|AAAAAAAAAFPLAAAA|48975|13|36|15|PM|first|afternoon|lunch| +48976|AAAAAAAABFPLAAAA|48976|13|36|16|PM|first|afternoon|lunch| +48977|AAAAAAAACFPLAAAA|48977|13|36|17|PM|first|afternoon|lunch| +48978|AAAAAAAADFPLAAAA|48978|13|36|18|PM|first|afternoon|lunch| +48979|AAAAAAAAEFPLAAAA|48979|13|36|19|PM|first|afternoon|lunch| +48980|AAAAAAAAFFPLAAAA|48980|13|36|20|PM|first|afternoon|lunch| +48981|AAAAAAAAGFPLAAAA|48981|13|36|21|PM|first|afternoon|lunch| +48982|AAAAAAAAHFPLAAAA|48982|13|36|22|PM|first|afternoon|lunch| +48983|AAAAAAAAIFPLAAAA|48983|13|36|23|PM|first|afternoon|lunch| +48984|AAAAAAAAJFPLAAAA|48984|13|36|24|PM|first|afternoon|lunch| +48985|AAAAAAAAKFPLAAAA|48985|13|36|25|PM|first|afternoon|lunch| +48986|AAAAAAAALFPLAAAA|48986|13|36|26|PM|first|afternoon|lunch| +48987|AAAAAAAAMFPLAAAA|48987|13|36|27|PM|first|afternoon|lunch| +48988|AAAAAAAANFPLAAAA|48988|13|36|28|PM|first|afternoon|lunch| +48989|AAAAAAAAOFPLAAAA|48989|13|36|29|PM|first|afternoon|lunch| +48990|AAAAAAAAPFPLAAAA|48990|13|36|30|PM|first|afternoon|lunch| +48991|AAAAAAAAAGPLAAAA|48991|13|36|31|PM|first|afternoon|lunch| +48992|AAAAAAAABGPLAAAA|48992|13|36|32|PM|first|afternoon|lunch| +48993|AAAAAAAACGPLAAAA|48993|13|36|33|PM|first|afternoon|lunch| +48994|AAAAAAAADGPLAAAA|48994|13|36|34|PM|first|afternoon|lunch| +48995|AAAAAAAAEGPLAAAA|48995|13|36|35|PM|first|afternoon|lunch| +48996|AAAAAAAAFGPLAAAA|48996|13|36|36|PM|first|afternoon|lunch| +48997|AAAAAAAAGGPLAAAA|48997|13|36|37|PM|first|afternoon|lunch| +48998|AAAAAAAAHGPLAAAA|48998|13|36|38|PM|first|afternoon|lunch| +48999|AAAAAAAAIGPLAAAA|48999|13|36|39|PM|first|afternoon|lunch| +49000|AAAAAAAAJGPLAAAA|49000|13|36|40|PM|first|afternoon|lunch| +49001|AAAAAAAAKGPLAAAA|49001|13|36|41|PM|first|afternoon|lunch| +49002|AAAAAAAALGPLAAAA|49002|13|36|42|PM|first|afternoon|lunch| +49003|AAAAAAAAMGPLAAAA|49003|13|36|43|PM|first|afternoon|lunch| +49004|AAAAAAAANGPLAAAA|49004|13|36|44|PM|first|afternoon|lunch| +49005|AAAAAAAAOGPLAAAA|49005|13|36|45|PM|first|afternoon|lunch| +49006|AAAAAAAAPGPLAAAA|49006|13|36|46|PM|first|afternoon|lunch| +49007|AAAAAAAAAHPLAAAA|49007|13|36|47|PM|first|afternoon|lunch| +49008|AAAAAAAABHPLAAAA|49008|13|36|48|PM|first|afternoon|lunch| +49009|AAAAAAAACHPLAAAA|49009|13|36|49|PM|first|afternoon|lunch| +49010|AAAAAAAADHPLAAAA|49010|13|36|50|PM|first|afternoon|lunch| +49011|AAAAAAAAEHPLAAAA|49011|13|36|51|PM|first|afternoon|lunch| +49012|AAAAAAAAFHPLAAAA|49012|13|36|52|PM|first|afternoon|lunch| +49013|AAAAAAAAGHPLAAAA|49013|13|36|53|PM|first|afternoon|lunch| +49014|AAAAAAAAHHPLAAAA|49014|13|36|54|PM|first|afternoon|lunch| +49015|AAAAAAAAIHPLAAAA|49015|13|36|55|PM|first|afternoon|lunch| +49016|AAAAAAAAJHPLAAAA|49016|13|36|56|PM|first|afternoon|lunch| +49017|AAAAAAAAKHPLAAAA|49017|13|36|57|PM|first|afternoon|lunch| +49018|AAAAAAAALHPLAAAA|49018|13|36|58|PM|first|afternoon|lunch| +49019|AAAAAAAAMHPLAAAA|49019|13|36|59|PM|first|afternoon|lunch| +49020|AAAAAAAANHPLAAAA|49020|13|37|0|PM|first|afternoon|lunch| +49021|AAAAAAAAOHPLAAAA|49021|13|37|1|PM|first|afternoon|lunch| +49022|AAAAAAAAPHPLAAAA|49022|13|37|2|PM|first|afternoon|lunch| +49023|AAAAAAAAAIPLAAAA|49023|13|37|3|PM|first|afternoon|lunch| +49024|AAAAAAAABIPLAAAA|49024|13|37|4|PM|first|afternoon|lunch| +49025|AAAAAAAACIPLAAAA|49025|13|37|5|PM|first|afternoon|lunch| +49026|AAAAAAAADIPLAAAA|49026|13|37|6|PM|first|afternoon|lunch| +49027|AAAAAAAAEIPLAAAA|49027|13|37|7|PM|first|afternoon|lunch| +49028|AAAAAAAAFIPLAAAA|49028|13|37|8|PM|first|afternoon|lunch| +49029|AAAAAAAAGIPLAAAA|49029|13|37|9|PM|first|afternoon|lunch| +49030|AAAAAAAAHIPLAAAA|49030|13|37|10|PM|first|afternoon|lunch| +49031|AAAAAAAAIIPLAAAA|49031|13|37|11|PM|first|afternoon|lunch| +49032|AAAAAAAAJIPLAAAA|49032|13|37|12|PM|first|afternoon|lunch| +49033|AAAAAAAAKIPLAAAA|49033|13|37|13|PM|first|afternoon|lunch| +49034|AAAAAAAALIPLAAAA|49034|13|37|14|PM|first|afternoon|lunch| +49035|AAAAAAAAMIPLAAAA|49035|13|37|15|PM|first|afternoon|lunch| +49036|AAAAAAAANIPLAAAA|49036|13|37|16|PM|first|afternoon|lunch| +49037|AAAAAAAAOIPLAAAA|49037|13|37|17|PM|first|afternoon|lunch| +49038|AAAAAAAAPIPLAAAA|49038|13|37|18|PM|first|afternoon|lunch| +49039|AAAAAAAAAJPLAAAA|49039|13|37|19|PM|first|afternoon|lunch| +49040|AAAAAAAABJPLAAAA|49040|13|37|20|PM|first|afternoon|lunch| +49041|AAAAAAAACJPLAAAA|49041|13|37|21|PM|first|afternoon|lunch| +49042|AAAAAAAADJPLAAAA|49042|13|37|22|PM|first|afternoon|lunch| +49043|AAAAAAAAEJPLAAAA|49043|13|37|23|PM|first|afternoon|lunch| +49044|AAAAAAAAFJPLAAAA|49044|13|37|24|PM|first|afternoon|lunch| +49045|AAAAAAAAGJPLAAAA|49045|13|37|25|PM|first|afternoon|lunch| +49046|AAAAAAAAHJPLAAAA|49046|13|37|26|PM|first|afternoon|lunch| +49047|AAAAAAAAIJPLAAAA|49047|13|37|27|PM|first|afternoon|lunch| +49048|AAAAAAAAJJPLAAAA|49048|13|37|28|PM|first|afternoon|lunch| +49049|AAAAAAAAKJPLAAAA|49049|13|37|29|PM|first|afternoon|lunch| +49050|AAAAAAAALJPLAAAA|49050|13|37|30|PM|first|afternoon|lunch| +49051|AAAAAAAAMJPLAAAA|49051|13|37|31|PM|first|afternoon|lunch| +49052|AAAAAAAANJPLAAAA|49052|13|37|32|PM|first|afternoon|lunch| +49053|AAAAAAAAOJPLAAAA|49053|13|37|33|PM|first|afternoon|lunch| +49054|AAAAAAAAPJPLAAAA|49054|13|37|34|PM|first|afternoon|lunch| +49055|AAAAAAAAAKPLAAAA|49055|13|37|35|PM|first|afternoon|lunch| +49056|AAAAAAAABKPLAAAA|49056|13|37|36|PM|first|afternoon|lunch| +49057|AAAAAAAACKPLAAAA|49057|13|37|37|PM|first|afternoon|lunch| +49058|AAAAAAAADKPLAAAA|49058|13|37|38|PM|first|afternoon|lunch| +49059|AAAAAAAAEKPLAAAA|49059|13|37|39|PM|first|afternoon|lunch| +49060|AAAAAAAAFKPLAAAA|49060|13|37|40|PM|first|afternoon|lunch| +49061|AAAAAAAAGKPLAAAA|49061|13|37|41|PM|first|afternoon|lunch| +49062|AAAAAAAAHKPLAAAA|49062|13|37|42|PM|first|afternoon|lunch| +49063|AAAAAAAAIKPLAAAA|49063|13|37|43|PM|first|afternoon|lunch| +49064|AAAAAAAAJKPLAAAA|49064|13|37|44|PM|first|afternoon|lunch| +49065|AAAAAAAAKKPLAAAA|49065|13|37|45|PM|first|afternoon|lunch| +49066|AAAAAAAALKPLAAAA|49066|13|37|46|PM|first|afternoon|lunch| +49067|AAAAAAAAMKPLAAAA|49067|13|37|47|PM|first|afternoon|lunch| +49068|AAAAAAAANKPLAAAA|49068|13|37|48|PM|first|afternoon|lunch| +49069|AAAAAAAAOKPLAAAA|49069|13|37|49|PM|first|afternoon|lunch| +49070|AAAAAAAAPKPLAAAA|49070|13|37|50|PM|first|afternoon|lunch| +49071|AAAAAAAAALPLAAAA|49071|13|37|51|PM|first|afternoon|lunch| +49072|AAAAAAAABLPLAAAA|49072|13|37|52|PM|first|afternoon|lunch| +49073|AAAAAAAACLPLAAAA|49073|13|37|53|PM|first|afternoon|lunch| +49074|AAAAAAAADLPLAAAA|49074|13|37|54|PM|first|afternoon|lunch| +49075|AAAAAAAAELPLAAAA|49075|13|37|55|PM|first|afternoon|lunch| +49076|AAAAAAAAFLPLAAAA|49076|13|37|56|PM|first|afternoon|lunch| +49077|AAAAAAAAGLPLAAAA|49077|13|37|57|PM|first|afternoon|lunch| +49078|AAAAAAAAHLPLAAAA|49078|13|37|58|PM|first|afternoon|lunch| +49079|AAAAAAAAILPLAAAA|49079|13|37|59|PM|first|afternoon|lunch| +49080|AAAAAAAAJLPLAAAA|49080|13|38|0|PM|first|afternoon|lunch| +49081|AAAAAAAAKLPLAAAA|49081|13|38|1|PM|first|afternoon|lunch| +49082|AAAAAAAALLPLAAAA|49082|13|38|2|PM|first|afternoon|lunch| +49083|AAAAAAAAMLPLAAAA|49083|13|38|3|PM|first|afternoon|lunch| +49084|AAAAAAAANLPLAAAA|49084|13|38|4|PM|first|afternoon|lunch| +49085|AAAAAAAAOLPLAAAA|49085|13|38|5|PM|first|afternoon|lunch| +49086|AAAAAAAAPLPLAAAA|49086|13|38|6|PM|first|afternoon|lunch| +49087|AAAAAAAAAMPLAAAA|49087|13|38|7|PM|first|afternoon|lunch| +49088|AAAAAAAABMPLAAAA|49088|13|38|8|PM|first|afternoon|lunch| +49089|AAAAAAAACMPLAAAA|49089|13|38|9|PM|first|afternoon|lunch| +49090|AAAAAAAADMPLAAAA|49090|13|38|10|PM|first|afternoon|lunch| +49091|AAAAAAAAEMPLAAAA|49091|13|38|11|PM|first|afternoon|lunch| +49092|AAAAAAAAFMPLAAAA|49092|13|38|12|PM|first|afternoon|lunch| +49093|AAAAAAAAGMPLAAAA|49093|13|38|13|PM|first|afternoon|lunch| +49094|AAAAAAAAHMPLAAAA|49094|13|38|14|PM|first|afternoon|lunch| +49095|AAAAAAAAIMPLAAAA|49095|13|38|15|PM|first|afternoon|lunch| +49096|AAAAAAAAJMPLAAAA|49096|13|38|16|PM|first|afternoon|lunch| +49097|AAAAAAAAKMPLAAAA|49097|13|38|17|PM|first|afternoon|lunch| +49098|AAAAAAAALMPLAAAA|49098|13|38|18|PM|first|afternoon|lunch| +49099|AAAAAAAAMMPLAAAA|49099|13|38|19|PM|first|afternoon|lunch| +49100|AAAAAAAANMPLAAAA|49100|13|38|20|PM|first|afternoon|lunch| +49101|AAAAAAAAOMPLAAAA|49101|13|38|21|PM|first|afternoon|lunch| +49102|AAAAAAAAPMPLAAAA|49102|13|38|22|PM|first|afternoon|lunch| +49103|AAAAAAAAANPLAAAA|49103|13|38|23|PM|first|afternoon|lunch| +49104|AAAAAAAABNPLAAAA|49104|13|38|24|PM|first|afternoon|lunch| +49105|AAAAAAAACNPLAAAA|49105|13|38|25|PM|first|afternoon|lunch| +49106|AAAAAAAADNPLAAAA|49106|13|38|26|PM|first|afternoon|lunch| +49107|AAAAAAAAENPLAAAA|49107|13|38|27|PM|first|afternoon|lunch| +49108|AAAAAAAAFNPLAAAA|49108|13|38|28|PM|first|afternoon|lunch| +49109|AAAAAAAAGNPLAAAA|49109|13|38|29|PM|first|afternoon|lunch| +49110|AAAAAAAAHNPLAAAA|49110|13|38|30|PM|first|afternoon|lunch| +49111|AAAAAAAAINPLAAAA|49111|13|38|31|PM|first|afternoon|lunch| +49112|AAAAAAAAJNPLAAAA|49112|13|38|32|PM|first|afternoon|lunch| +49113|AAAAAAAAKNPLAAAA|49113|13|38|33|PM|first|afternoon|lunch| +49114|AAAAAAAALNPLAAAA|49114|13|38|34|PM|first|afternoon|lunch| +49115|AAAAAAAAMNPLAAAA|49115|13|38|35|PM|first|afternoon|lunch| +49116|AAAAAAAANNPLAAAA|49116|13|38|36|PM|first|afternoon|lunch| +49117|AAAAAAAAONPLAAAA|49117|13|38|37|PM|first|afternoon|lunch| +49118|AAAAAAAAPNPLAAAA|49118|13|38|38|PM|first|afternoon|lunch| +49119|AAAAAAAAAOPLAAAA|49119|13|38|39|PM|first|afternoon|lunch| +49120|AAAAAAAABOPLAAAA|49120|13|38|40|PM|first|afternoon|lunch| +49121|AAAAAAAACOPLAAAA|49121|13|38|41|PM|first|afternoon|lunch| +49122|AAAAAAAADOPLAAAA|49122|13|38|42|PM|first|afternoon|lunch| +49123|AAAAAAAAEOPLAAAA|49123|13|38|43|PM|first|afternoon|lunch| +49124|AAAAAAAAFOPLAAAA|49124|13|38|44|PM|first|afternoon|lunch| +49125|AAAAAAAAGOPLAAAA|49125|13|38|45|PM|first|afternoon|lunch| +49126|AAAAAAAAHOPLAAAA|49126|13|38|46|PM|first|afternoon|lunch| +49127|AAAAAAAAIOPLAAAA|49127|13|38|47|PM|first|afternoon|lunch| +49128|AAAAAAAAJOPLAAAA|49128|13|38|48|PM|first|afternoon|lunch| +49129|AAAAAAAAKOPLAAAA|49129|13|38|49|PM|first|afternoon|lunch| +49130|AAAAAAAALOPLAAAA|49130|13|38|50|PM|first|afternoon|lunch| +49131|AAAAAAAAMOPLAAAA|49131|13|38|51|PM|first|afternoon|lunch| +49132|AAAAAAAANOPLAAAA|49132|13|38|52|PM|first|afternoon|lunch| +49133|AAAAAAAAOOPLAAAA|49133|13|38|53|PM|first|afternoon|lunch| +49134|AAAAAAAAPOPLAAAA|49134|13|38|54|PM|first|afternoon|lunch| +49135|AAAAAAAAAPPLAAAA|49135|13|38|55|PM|first|afternoon|lunch| +49136|AAAAAAAABPPLAAAA|49136|13|38|56|PM|first|afternoon|lunch| +49137|AAAAAAAACPPLAAAA|49137|13|38|57|PM|first|afternoon|lunch| +49138|AAAAAAAADPPLAAAA|49138|13|38|58|PM|first|afternoon|lunch| +49139|AAAAAAAAEPPLAAAA|49139|13|38|59|PM|first|afternoon|lunch| +49140|AAAAAAAAFPPLAAAA|49140|13|39|0|PM|first|afternoon|lunch| +49141|AAAAAAAAGPPLAAAA|49141|13|39|1|PM|first|afternoon|lunch| +49142|AAAAAAAAHPPLAAAA|49142|13|39|2|PM|first|afternoon|lunch| +49143|AAAAAAAAIPPLAAAA|49143|13|39|3|PM|first|afternoon|lunch| +49144|AAAAAAAAJPPLAAAA|49144|13|39|4|PM|first|afternoon|lunch| +49145|AAAAAAAAKPPLAAAA|49145|13|39|5|PM|first|afternoon|lunch| +49146|AAAAAAAALPPLAAAA|49146|13|39|6|PM|first|afternoon|lunch| +49147|AAAAAAAAMPPLAAAA|49147|13|39|7|PM|first|afternoon|lunch| +49148|AAAAAAAANPPLAAAA|49148|13|39|8|PM|first|afternoon|lunch| +49149|AAAAAAAAOPPLAAAA|49149|13|39|9|PM|first|afternoon|lunch| +49150|AAAAAAAAPPPLAAAA|49150|13|39|10|PM|first|afternoon|lunch| +49151|AAAAAAAAAAAMAAAA|49151|13|39|11|PM|first|afternoon|lunch| +49152|AAAAAAAABAAMAAAA|49152|13|39|12|PM|first|afternoon|lunch| +49153|AAAAAAAACAAMAAAA|49153|13|39|13|PM|first|afternoon|lunch| +49154|AAAAAAAADAAMAAAA|49154|13|39|14|PM|first|afternoon|lunch| +49155|AAAAAAAAEAAMAAAA|49155|13|39|15|PM|first|afternoon|lunch| +49156|AAAAAAAAFAAMAAAA|49156|13|39|16|PM|first|afternoon|lunch| +49157|AAAAAAAAGAAMAAAA|49157|13|39|17|PM|first|afternoon|lunch| +49158|AAAAAAAAHAAMAAAA|49158|13|39|18|PM|first|afternoon|lunch| +49159|AAAAAAAAIAAMAAAA|49159|13|39|19|PM|first|afternoon|lunch| +49160|AAAAAAAAJAAMAAAA|49160|13|39|20|PM|first|afternoon|lunch| +49161|AAAAAAAAKAAMAAAA|49161|13|39|21|PM|first|afternoon|lunch| +49162|AAAAAAAALAAMAAAA|49162|13|39|22|PM|first|afternoon|lunch| +49163|AAAAAAAAMAAMAAAA|49163|13|39|23|PM|first|afternoon|lunch| +49164|AAAAAAAANAAMAAAA|49164|13|39|24|PM|first|afternoon|lunch| +49165|AAAAAAAAOAAMAAAA|49165|13|39|25|PM|first|afternoon|lunch| +49166|AAAAAAAAPAAMAAAA|49166|13|39|26|PM|first|afternoon|lunch| +49167|AAAAAAAAABAMAAAA|49167|13|39|27|PM|first|afternoon|lunch| +49168|AAAAAAAABBAMAAAA|49168|13|39|28|PM|first|afternoon|lunch| +49169|AAAAAAAACBAMAAAA|49169|13|39|29|PM|first|afternoon|lunch| +49170|AAAAAAAADBAMAAAA|49170|13|39|30|PM|first|afternoon|lunch| +49171|AAAAAAAAEBAMAAAA|49171|13|39|31|PM|first|afternoon|lunch| +49172|AAAAAAAAFBAMAAAA|49172|13|39|32|PM|first|afternoon|lunch| +49173|AAAAAAAAGBAMAAAA|49173|13|39|33|PM|first|afternoon|lunch| +49174|AAAAAAAAHBAMAAAA|49174|13|39|34|PM|first|afternoon|lunch| +49175|AAAAAAAAIBAMAAAA|49175|13|39|35|PM|first|afternoon|lunch| +49176|AAAAAAAAJBAMAAAA|49176|13|39|36|PM|first|afternoon|lunch| +49177|AAAAAAAAKBAMAAAA|49177|13|39|37|PM|first|afternoon|lunch| +49178|AAAAAAAALBAMAAAA|49178|13|39|38|PM|first|afternoon|lunch| +49179|AAAAAAAAMBAMAAAA|49179|13|39|39|PM|first|afternoon|lunch| +49180|AAAAAAAANBAMAAAA|49180|13|39|40|PM|first|afternoon|lunch| +49181|AAAAAAAAOBAMAAAA|49181|13|39|41|PM|first|afternoon|lunch| +49182|AAAAAAAAPBAMAAAA|49182|13|39|42|PM|first|afternoon|lunch| +49183|AAAAAAAAACAMAAAA|49183|13|39|43|PM|first|afternoon|lunch| +49184|AAAAAAAABCAMAAAA|49184|13|39|44|PM|first|afternoon|lunch| +49185|AAAAAAAACCAMAAAA|49185|13|39|45|PM|first|afternoon|lunch| +49186|AAAAAAAADCAMAAAA|49186|13|39|46|PM|first|afternoon|lunch| +49187|AAAAAAAAECAMAAAA|49187|13|39|47|PM|first|afternoon|lunch| +49188|AAAAAAAAFCAMAAAA|49188|13|39|48|PM|first|afternoon|lunch| +49189|AAAAAAAAGCAMAAAA|49189|13|39|49|PM|first|afternoon|lunch| +49190|AAAAAAAAHCAMAAAA|49190|13|39|50|PM|first|afternoon|lunch| +49191|AAAAAAAAICAMAAAA|49191|13|39|51|PM|first|afternoon|lunch| +49192|AAAAAAAAJCAMAAAA|49192|13|39|52|PM|first|afternoon|lunch| +49193|AAAAAAAAKCAMAAAA|49193|13|39|53|PM|first|afternoon|lunch| +49194|AAAAAAAALCAMAAAA|49194|13|39|54|PM|first|afternoon|lunch| +49195|AAAAAAAAMCAMAAAA|49195|13|39|55|PM|first|afternoon|lunch| +49196|AAAAAAAANCAMAAAA|49196|13|39|56|PM|first|afternoon|lunch| +49197|AAAAAAAAOCAMAAAA|49197|13|39|57|PM|first|afternoon|lunch| +49198|AAAAAAAAPCAMAAAA|49198|13|39|58|PM|first|afternoon|lunch| +49199|AAAAAAAAADAMAAAA|49199|13|39|59|PM|first|afternoon|lunch| +49200|AAAAAAAABDAMAAAA|49200|13|40|0|PM|first|afternoon|lunch| +49201|AAAAAAAACDAMAAAA|49201|13|40|1|PM|first|afternoon|lunch| +49202|AAAAAAAADDAMAAAA|49202|13|40|2|PM|first|afternoon|lunch| +49203|AAAAAAAAEDAMAAAA|49203|13|40|3|PM|first|afternoon|lunch| +49204|AAAAAAAAFDAMAAAA|49204|13|40|4|PM|first|afternoon|lunch| +49205|AAAAAAAAGDAMAAAA|49205|13|40|5|PM|first|afternoon|lunch| +49206|AAAAAAAAHDAMAAAA|49206|13|40|6|PM|first|afternoon|lunch| +49207|AAAAAAAAIDAMAAAA|49207|13|40|7|PM|first|afternoon|lunch| +49208|AAAAAAAAJDAMAAAA|49208|13|40|8|PM|first|afternoon|lunch| +49209|AAAAAAAAKDAMAAAA|49209|13|40|9|PM|first|afternoon|lunch| +49210|AAAAAAAALDAMAAAA|49210|13|40|10|PM|first|afternoon|lunch| +49211|AAAAAAAAMDAMAAAA|49211|13|40|11|PM|first|afternoon|lunch| +49212|AAAAAAAANDAMAAAA|49212|13|40|12|PM|first|afternoon|lunch| +49213|AAAAAAAAODAMAAAA|49213|13|40|13|PM|first|afternoon|lunch| +49214|AAAAAAAAPDAMAAAA|49214|13|40|14|PM|first|afternoon|lunch| +49215|AAAAAAAAAEAMAAAA|49215|13|40|15|PM|first|afternoon|lunch| +49216|AAAAAAAABEAMAAAA|49216|13|40|16|PM|first|afternoon|lunch| +49217|AAAAAAAACEAMAAAA|49217|13|40|17|PM|first|afternoon|lunch| +49218|AAAAAAAADEAMAAAA|49218|13|40|18|PM|first|afternoon|lunch| +49219|AAAAAAAAEEAMAAAA|49219|13|40|19|PM|first|afternoon|lunch| +49220|AAAAAAAAFEAMAAAA|49220|13|40|20|PM|first|afternoon|lunch| +49221|AAAAAAAAGEAMAAAA|49221|13|40|21|PM|first|afternoon|lunch| +49222|AAAAAAAAHEAMAAAA|49222|13|40|22|PM|first|afternoon|lunch| +49223|AAAAAAAAIEAMAAAA|49223|13|40|23|PM|first|afternoon|lunch| +49224|AAAAAAAAJEAMAAAA|49224|13|40|24|PM|first|afternoon|lunch| +49225|AAAAAAAAKEAMAAAA|49225|13|40|25|PM|first|afternoon|lunch| +49226|AAAAAAAALEAMAAAA|49226|13|40|26|PM|first|afternoon|lunch| +49227|AAAAAAAAMEAMAAAA|49227|13|40|27|PM|first|afternoon|lunch| +49228|AAAAAAAANEAMAAAA|49228|13|40|28|PM|first|afternoon|lunch| +49229|AAAAAAAAOEAMAAAA|49229|13|40|29|PM|first|afternoon|lunch| +49230|AAAAAAAAPEAMAAAA|49230|13|40|30|PM|first|afternoon|lunch| +49231|AAAAAAAAAFAMAAAA|49231|13|40|31|PM|first|afternoon|lunch| +49232|AAAAAAAABFAMAAAA|49232|13|40|32|PM|first|afternoon|lunch| +49233|AAAAAAAACFAMAAAA|49233|13|40|33|PM|first|afternoon|lunch| +49234|AAAAAAAADFAMAAAA|49234|13|40|34|PM|first|afternoon|lunch| +49235|AAAAAAAAEFAMAAAA|49235|13|40|35|PM|first|afternoon|lunch| +49236|AAAAAAAAFFAMAAAA|49236|13|40|36|PM|first|afternoon|lunch| +49237|AAAAAAAAGFAMAAAA|49237|13|40|37|PM|first|afternoon|lunch| +49238|AAAAAAAAHFAMAAAA|49238|13|40|38|PM|first|afternoon|lunch| +49239|AAAAAAAAIFAMAAAA|49239|13|40|39|PM|first|afternoon|lunch| +49240|AAAAAAAAJFAMAAAA|49240|13|40|40|PM|first|afternoon|lunch| +49241|AAAAAAAAKFAMAAAA|49241|13|40|41|PM|first|afternoon|lunch| +49242|AAAAAAAALFAMAAAA|49242|13|40|42|PM|first|afternoon|lunch| +49243|AAAAAAAAMFAMAAAA|49243|13|40|43|PM|first|afternoon|lunch| +49244|AAAAAAAANFAMAAAA|49244|13|40|44|PM|first|afternoon|lunch| +49245|AAAAAAAAOFAMAAAA|49245|13|40|45|PM|first|afternoon|lunch| +49246|AAAAAAAAPFAMAAAA|49246|13|40|46|PM|first|afternoon|lunch| +49247|AAAAAAAAAGAMAAAA|49247|13|40|47|PM|first|afternoon|lunch| +49248|AAAAAAAABGAMAAAA|49248|13|40|48|PM|first|afternoon|lunch| +49249|AAAAAAAACGAMAAAA|49249|13|40|49|PM|first|afternoon|lunch| +49250|AAAAAAAADGAMAAAA|49250|13|40|50|PM|first|afternoon|lunch| +49251|AAAAAAAAEGAMAAAA|49251|13|40|51|PM|first|afternoon|lunch| +49252|AAAAAAAAFGAMAAAA|49252|13|40|52|PM|first|afternoon|lunch| +49253|AAAAAAAAGGAMAAAA|49253|13|40|53|PM|first|afternoon|lunch| +49254|AAAAAAAAHGAMAAAA|49254|13|40|54|PM|first|afternoon|lunch| +49255|AAAAAAAAIGAMAAAA|49255|13|40|55|PM|first|afternoon|lunch| +49256|AAAAAAAAJGAMAAAA|49256|13|40|56|PM|first|afternoon|lunch| +49257|AAAAAAAAKGAMAAAA|49257|13|40|57|PM|first|afternoon|lunch| +49258|AAAAAAAALGAMAAAA|49258|13|40|58|PM|first|afternoon|lunch| +49259|AAAAAAAAMGAMAAAA|49259|13|40|59|PM|first|afternoon|lunch| +49260|AAAAAAAANGAMAAAA|49260|13|41|0|PM|first|afternoon|lunch| +49261|AAAAAAAAOGAMAAAA|49261|13|41|1|PM|first|afternoon|lunch| +49262|AAAAAAAAPGAMAAAA|49262|13|41|2|PM|first|afternoon|lunch| +49263|AAAAAAAAAHAMAAAA|49263|13|41|3|PM|first|afternoon|lunch| +49264|AAAAAAAABHAMAAAA|49264|13|41|4|PM|first|afternoon|lunch| +49265|AAAAAAAACHAMAAAA|49265|13|41|5|PM|first|afternoon|lunch| +49266|AAAAAAAADHAMAAAA|49266|13|41|6|PM|first|afternoon|lunch| +49267|AAAAAAAAEHAMAAAA|49267|13|41|7|PM|first|afternoon|lunch| +49268|AAAAAAAAFHAMAAAA|49268|13|41|8|PM|first|afternoon|lunch| +49269|AAAAAAAAGHAMAAAA|49269|13|41|9|PM|first|afternoon|lunch| +49270|AAAAAAAAHHAMAAAA|49270|13|41|10|PM|first|afternoon|lunch| +49271|AAAAAAAAIHAMAAAA|49271|13|41|11|PM|first|afternoon|lunch| +49272|AAAAAAAAJHAMAAAA|49272|13|41|12|PM|first|afternoon|lunch| +49273|AAAAAAAAKHAMAAAA|49273|13|41|13|PM|first|afternoon|lunch| +49274|AAAAAAAALHAMAAAA|49274|13|41|14|PM|first|afternoon|lunch| +49275|AAAAAAAAMHAMAAAA|49275|13|41|15|PM|first|afternoon|lunch| +49276|AAAAAAAANHAMAAAA|49276|13|41|16|PM|first|afternoon|lunch| +49277|AAAAAAAAOHAMAAAA|49277|13|41|17|PM|first|afternoon|lunch| +49278|AAAAAAAAPHAMAAAA|49278|13|41|18|PM|first|afternoon|lunch| +49279|AAAAAAAAAIAMAAAA|49279|13|41|19|PM|first|afternoon|lunch| +49280|AAAAAAAABIAMAAAA|49280|13|41|20|PM|first|afternoon|lunch| +49281|AAAAAAAACIAMAAAA|49281|13|41|21|PM|first|afternoon|lunch| +49282|AAAAAAAADIAMAAAA|49282|13|41|22|PM|first|afternoon|lunch| +49283|AAAAAAAAEIAMAAAA|49283|13|41|23|PM|first|afternoon|lunch| +49284|AAAAAAAAFIAMAAAA|49284|13|41|24|PM|first|afternoon|lunch| +49285|AAAAAAAAGIAMAAAA|49285|13|41|25|PM|first|afternoon|lunch| +49286|AAAAAAAAHIAMAAAA|49286|13|41|26|PM|first|afternoon|lunch| +49287|AAAAAAAAIIAMAAAA|49287|13|41|27|PM|first|afternoon|lunch| +49288|AAAAAAAAJIAMAAAA|49288|13|41|28|PM|first|afternoon|lunch| +49289|AAAAAAAAKIAMAAAA|49289|13|41|29|PM|first|afternoon|lunch| +49290|AAAAAAAALIAMAAAA|49290|13|41|30|PM|first|afternoon|lunch| +49291|AAAAAAAAMIAMAAAA|49291|13|41|31|PM|first|afternoon|lunch| +49292|AAAAAAAANIAMAAAA|49292|13|41|32|PM|first|afternoon|lunch| +49293|AAAAAAAAOIAMAAAA|49293|13|41|33|PM|first|afternoon|lunch| +49294|AAAAAAAAPIAMAAAA|49294|13|41|34|PM|first|afternoon|lunch| +49295|AAAAAAAAAJAMAAAA|49295|13|41|35|PM|first|afternoon|lunch| +49296|AAAAAAAABJAMAAAA|49296|13|41|36|PM|first|afternoon|lunch| +49297|AAAAAAAACJAMAAAA|49297|13|41|37|PM|first|afternoon|lunch| +49298|AAAAAAAADJAMAAAA|49298|13|41|38|PM|first|afternoon|lunch| +49299|AAAAAAAAEJAMAAAA|49299|13|41|39|PM|first|afternoon|lunch| +49300|AAAAAAAAFJAMAAAA|49300|13|41|40|PM|first|afternoon|lunch| +49301|AAAAAAAAGJAMAAAA|49301|13|41|41|PM|first|afternoon|lunch| +49302|AAAAAAAAHJAMAAAA|49302|13|41|42|PM|first|afternoon|lunch| +49303|AAAAAAAAIJAMAAAA|49303|13|41|43|PM|first|afternoon|lunch| +49304|AAAAAAAAJJAMAAAA|49304|13|41|44|PM|first|afternoon|lunch| +49305|AAAAAAAAKJAMAAAA|49305|13|41|45|PM|first|afternoon|lunch| +49306|AAAAAAAALJAMAAAA|49306|13|41|46|PM|first|afternoon|lunch| +49307|AAAAAAAAMJAMAAAA|49307|13|41|47|PM|first|afternoon|lunch| +49308|AAAAAAAANJAMAAAA|49308|13|41|48|PM|first|afternoon|lunch| +49309|AAAAAAAAOJAMAAAA|49309|13|41|49|PM|first|afternoon|lunch| +49310|AAAAAAAAPJAMAAAA|49310|13|41|50|PM|first|afternoon|lunch| +49311|AAAAAAAAAKAMAAAA|49311|13|41|51|PM|first|afternoon|lunch| +49312|AAAAAAAABKAMAAAA|49312|13|41|52|PM|first|afternoon|lunch| +49313|AAAAAAAACKAMAAAA|49313|13|41|53|PM|first|afternoon|lunch| +49314|AAAAAAAADKAMAAAA|49314|13|41|54|PM|first|afternoon|lunch| +49315|AAAAAAAAEKAMAAAA|49315|13|41|55|PM|first|afternoon|lunch| +49316|AAAAAAAAFKAMAAAA|49316|13|41|56|PM|first|afternoon|lunch| +49317|AAAAAAAAGKAMAAAA|49317|13|41|57|PM|first|afternoon|lunch| +49318|AAAAAAAAHKAMAAAA|49318|13|41|58|PM|first|afternoon|lunch| +49319|AAAAAAAAIKAMAAAA|49319|13|41|59|PM|first|afternoon|lunch| +49320|AAAAAAAAJKAMAAAA|49320|13|42|0|PM|first|afternoon|lunch| +49321|AAAAAAAAKKAMAAAA|49321|13|42|1|PM|first|afternoon|lunch| +49322|AAAAAAAALKAMAAAA|49322|13|42|2|PM|first|afternoon|lunch| +49323|AAAAAAAAMKAMAAAA|49323|13|42|3|PM|first|afternoon|lunch| +49324|AAAAAAAANKAMAAAA|49324|13|42|4|PM|first|afternoon|lunch| +49325|AAAAAAAAOKAMAAAA|49325|13|42|5|PM|first|afternoon|lunch| +49326|AAAAAAAAPKAMAAAA|49326|13|42|6|PM|first|afternoon|lunch| +49327|AAAAAAAAALAMAAAA|49327|13|42|7|PM|first|afternoon|lunch| +49328|AAAAAAAABLAMAAAA|49328|13|42|8|PM|first|afternoon|lunch| +49329|AAAAAAAACLAMAAAA|49329|13|42|9|PM|first|afternoon|lunch| +49330|AAAAAAAADLAMAAAA|49330|13|42|10|PM|first|afternoon|lunch| +49331|AAAAAAAAELAMAAAA|49331|13|42|11|PM|first|afternoon|lunch| +49332|AAAAAAAAFLAMAAAA|49332|13|42|12|PM|first|afternoon|lunch| +49333|AAAAAAAAGLAMAAAA|49333|13|42|13|PM|first|afternoon|lunch| +49334|AAAAAAAAHLAMAAAA|49334|13|42|14|PM|first|afternoon|lunch| +49335|AAAAAAAAILAMAAAA|49335|13|42|15|PM|first|afternoon|lunch| +49336|AAAAAAAAJLAMAAAA|49336|13|42|16|PM|first|afternoon|lunch| +49337|AAAAAAAAKLAMAAAA|49337|13|42|17|PM|first|afternoon|lunch| +49338|AAAAAAAALLAMAAAA|49338|13|42|18|PM|first|afternoon|lunch| +49339|AAAAAAAAMLAMAAAA|49339|13|42|19|PM|first|afternoon|lunch| +49340|AAAAAAAANLAMAAAA|49340|13|42|20|PM|first|afternoon|lunch| +49341|AAAAAAAAOLAMAAAA|49341|13|42|21|PM|first|afternoon|lunch| +49342|AAAAAAAAPLAMAAAA|49342|13|42|22|PM|first|afternoon|lunch| +49343|AAAAAAAAAMAMAAAA|49343|13|42|23|PM|first|afternoon|lunch| +49344|AAAAAAAABMAMAAAA|49344|13|42|24|PM|first|afternoon|lunch| +49345|AAAAAAAACMAMAAAA|49345|13|42|25|PM|first|afternoon|lunch| +49346|AAAAAAAADMAMAAAA|49346|13|42|26|PM|first|afternoon|lunch| +49347|AAAAAAAAEMAMAAAA|49347|13|42|27|PM|first|afternoon|lunch| +49348|AAAAAAAAFMAMAAAA|49348|13|42|28|PM|first|afternoon|lunch| +49349|AAAAAAAAGMAMAAAA|49349|13|42|29|PM|first|afternoon|lunch| +49350|AAAAAAAAHMAMAAAA|49350|13|42|30|PM|first|afternoon|lunch| +49351|AAAAAAAAIMAMAAAA|49351|13|42|31|PM|first|afternoon|lunch| +49352|AAAAAAAAJMAMAAAA|49352|13|42|32|PM|first|afternoon|lunch| +49353|AAAAAAAAKMAMAAAA|49353|13|42|33|PM|first|afternoon|lunch| +49354|AAAAAAAALMAMAAAA|49354|13|42|34|PM|first|afternoon|lunch| +49355|AAAAAAAAMMAMAAAA|49355|13|42|35|PM|first|afternoon|lunch| +49356|AAAAAAAANMAMAAAA|49356|13|42|36|PM|first|afternoon|lunch| +49357|AAAAAAAAOMAMAAAA|49357|13|42|37|PM|first|afternoon|lunch| +49358|AAAAAAAAPMAMAAAA|49358|13|42|38|PM|first|afternoon|lunch| +49359|AAAAAAAAANAMAAAA|49359|13|42|39|PM|first|afternoon|lunch| +49360|AAAAAAAABNAMAAAA|49360|13|42|40|PM|first|afternoon|lunch| +49361|AAAAAAAACNAMAAAA|49361|13|42|41|PM|first|afternoon|lunch| +49362|AAAAAAAADNAMAAAA|49362|13|42|42|PM|first|afternoon|lunch| +49363|AAAAAAAAENAMAAAA|49363|13|42|43|PM|first|afternoon|lunch| +49364|AAAAAAAAFNAMAAAA|49364|13|42|44|PM|first|afternoon|lunch| +49365|AAAAAAAAGNAMAAAA|49365|13|42|45|PM|first|afternoon|lunch| +49366|AAAAAAAAHNAMAAAA|49366|13|42|46|PM|first|afternoon|lunch| +49367|AAAAAAAAINAMAAAA|49367|13|42|47|PM|first|afternoon|lunch| +49368|AAAAAAAAJNAMAAAA|49368|13|42|48|PM|first|afternoon|lunch| +49369|AAAAAAAAKNAMAAAA|49369|13|42|49|PM|first|afternoon|lunch| +49370|AAAAAAAALNAMAAAA|49370|13|42|50|PM|first|afternoon|lunch| +49371|AAAAAAAAMNAMAAAA|49371|13|42|51|PM|first|afternoon|lunch| +49372|AAAAAAAANNAMAAAA|49372|13|42|52|PM|first|afternoon|lunch| +49373|AAAAAAAAONAMAAAA|49373|13|42|53|PM|first|afternoon|lunch| +49374|AAAAAAAAPNAMAAAA|49374|13|42|54|PM|first|afternoon|lunch| +49375|AAAAAAAAAOAMAAAA|49375|13|42|55|PM|first|afternoon|lunch| +49376|AAAAAAAABOAMAAAA|49376|13|42|56|PM|first|afternoon|lunch| +49377|AAAAAAAACOAMAAAA|49377|13|42|57|PM|first|afternoon|lunch| +49378|AAAAAAAADOAMAAAA|49378|13|42|58|PM|first|afternoon|lunch| +49379|AAAAAAAAEOAMAAAA|49379|13|42|59|PM|first|afternoon|lunch| +49380|AAAAAAAAFOAMAAAA|49380|13|43|0|PM|first|afternoon|lunch| +49381|AAAAAAAAGOAMAAAA|49381|13|43|1|PM|first|afternoon|lunch| +49382|AAAAAAAAHOAMAAAA|49382|13|43|2|PM|first|afternoon|lunch| +49383|AAAAAAAAIOAMAAAA|49383|13|43|3|PM|first|afternoon|lunch| +49384|AAAAAAAAJOAMAAAA|49384|13|43|4|PM|first|afternoon|lunch| +49385|AAAAAAAAKOAMAAAA|49385|13|43|5|PM|first|afternoon|lunch| +49386|AAAAAAAALOAMAAAA|49386|13|43|6|PM|first|afternoon|lunch| +49387|AAAAAAAAMOAMAAAA|49387|13|43|7|PM|first|afternoon|lunch| +49388|AAAAAAAANOAMAAAA|49388|13|43|8|PM|first|afternoon|lunch| +49389|AAAAAAAAOOAMAAAA|49389|13|43|9|PM|first|afternoon|lunch| +49390|AAAAAAAAPOAMAAAA|49390|13|43|10|PM|first|afternoon|lunch| +49391|AAAAAAAAAPAMAAAA|49391|13|43|11|PM|first|afternoon|lunch| +49392|AAAAAAAABPAMAAAA|49392|13|43|12|PM|first|afternoon|lunch| +49393|AAAAAAAACPAMAAAA|49393|13|43|13|PM|first|afternoon|lunch| +49394|AAAAAAAADPAMAAAA|49394|13|43|14|PM|first|afternoon|lunch| +49395|AAAAAAAAEPAMAAAA|49395|13|43|15|PM|first|afternoon|lunch| +49396|AAAAAAAAFPAMAAAA|49396|13|43|16|PM|first|afternoon|lunch| +49397|AAAAAAAAGPAMAAAA|49397|13|43|17|PM|first|afternoon|lunch| +49398|AAAAAAAAHPAMAAAA|49398|13|43|18|PM|first|afternoon|lunch| +49399|AAAAAAAAIPAMAAAA|49399|13|43|19|PM|first|afternoon|lunch| +49400|AAAAAAAAJPAMAAAA|49400|13|43|20|PM|first|afternoon|lunch| +49401|AAAAAAAAKPAMAAAA|49401|13|43|21|PM|first|afternoon|lunch| +49402|AAAAAAAALPAMAAAA|49402|13|43|22|PM|first|afternoon|lunch| +49403|AAAAAAAAMPAMAAAA|49403|13|43|23|PM|first|afternoon|lunch| +49404|AAAAAAAANPAMAAAA|49404|13|43|24|PM|first|afternoon|lunch| +49405|AAAAAAAAOPAMAAAA|49405|13|43|25|PM|first|afternoon|lunch| +49406|AAAAAAAAPPAMAAAA|49406|13|43|26|PM|first|afternoon|lunch| +49407|AAAAAAAAAABMAAAA|49407|13|43|27|PM|first|afternoon|lunch| +49408|AAAAAAAABABMAAAA|49408|13|43|28|PM|first|afternoon|lunch| +49409|AAAAAAAACABMAAAA|49409|13|43|29|PM|first|afternoon|lunch| +49410|AAAAAAAADABMAAAA|49410|13|43|30|PM|first|afternoon|lunch| +49411|AAAAAAAAEABMAAAA|49411|13|43|31|PM|first|afternoon|lunch| +49412|AAAAAAAAFABMAAAA|49412|13|43|32|PM|first|afternoon|lunch| +49413|AAAAAAAAGABMAAAA|49413|13|43|33|PM|first|afternoon|lunch| +49414|AAAAAAAAHABMAAAA|49414|13|43|34|PM|first|afternoon|lunch| +49415|AAAAAAAAIABMAAAA|49415|13|43|35|PM|first|afternoon|lunch| +49416|AAAAAAAAJABMAAAA|49416|13|43|36|PM|first|afternoon|lunch| +49417|AAAAAAAAKABMAAAA|49417|13|43|37|PM|first|afternoon|lunch| +49418|AAAAAAAALABMAAAA|49418|13|43|38|PM|first|afternoon|lunch| +49419|AAAAAAAAMABMAAAA|49419|13|43|39|PM|first|afternoon|lunch| +49420|AAAAAAAANABMAAAA|49420|13|43|40|PM|first|afternoon|lunch| +49421|AAAAAAAAOABMAAAA|49421|13|43|41|PM|first|afternoon|lunch| +49422|AAAAAAAAPABMAAAA|49422|13|43|42|PM|first|afternoon|lunch| +49423|AAAAAAAAABBMAAAA|49423|13|43|43|PM|first|afternoon|lunch| +49424|AAAAAAAABBBMAAAA|49424|13|43|44|PM|first|afternoon|lunch| +49425|AAAAAAAACBBMAAAA|49425|13|43|45|PM|first|afternoon|lunch| +49426|AAAAAAAADBBMAAAA|49426|13|43|46|PM|first|afternoon|lunch| +49427|AAAAAAAAEBBMAAAA|49427|13|43|47|PM|first|afternoon|lunch| +49428|AAAAAAAAFBBMAAAA|49428|13|43|48|PM|first|afternoon|lunch| +49429|AAAAAAAAGBBMAAAA|49429|13|43|49|PM|first|afternoon|lunch| +49430|AAAAAAAAHBBMAAAA|49430|13|43|50|PM|first|afternoon|lunch| +49431|AAAAAAAAIBBMAAAA|49431|13|43|51|PM|first|afternoon|lunch| +49432|AAAAAAAAJBBMAAAA|49432|13|43|52|PM|first|afternoon|lunch| +49433|AAAAAAAAKBBMAAAA|49433|13|43|53|PM|first|afternoon|lunch| +49434|AAAAAAAALBBMAAAA|49434|13|43|54|PM|first|afternoon|lunch| +49435|AAAAAAAAMBBMAAAA|49435|13|43|55|PM|first|afternoon|lunch| +49436|AAAAAAAANBBMAAAA|49436|13|43|56|PM|first|afternoon|lunch| +49437|AAAAAAAAOBBMAAAA|49437|13|43|57|PM|first|afternoon|lunch| +49438|AAAAAAAAPBBMAAAA|49438|13|43|58|PM|first|afternoon|lunch| +49439|AAAAAAAAACBMAAAA|49439|13|43|59|PM|first|afternoon|lunch| +49440|AAAAAAAABCBMAAAA|49440|13|44|0|PM|first|afternoon|lunch| +49441|AAAAAAAACCBMAAAA|49441|13|44|1|PM|first|afternoon|lunch| +49442|AAAAAAAADCBMAAAA|49442|13|44|2|PM|first|afternoon|lunch| +49443|AAAAAAAAECBMAAAA|49443|13|44|3|PM|first|afternoon|lunch| +49444|AAAAAAAAFCBMAAAA|49444|13|44|4|PM|first|afternoon|lunch| +49445|AAAAAAAAGCBMAAAA|49445|13|44|5|PM|first|afternoon|lunch| +49446|AAAAAAAAHCBMAAAA|49446|13|44|6|PM|first|afternoon|lunch| +49447|AAAAAAAAICBMAAAA|49447|13|44|7|PM|first|afternoon|lunch| +49448|AAAAAAAAJCBMAAAA|49448|13|44|8|PM|first|afternoon|lunch| +49449|AAAAAAAAKCBMAAAA|49449|13|44|9|PM|first|afternoon|lunch| +49450|AAAAAAAALCBMAAAA|49450|13|44|10|PM|first|afternoon|lunch| +49451|AAAAAAAAMCBMAAAA|49451|13|44|11|PM|first|afternoon|lunch| +49452|AAAAAAAANCBMAAAA|49452|13|44|12|PM|first|afternoon|lunch| +49453|AAAAAAAAOCBMAAAA|49453|13|44|13|PM|first|afternoon|lunch| +49454|AAAAAAAAPCBMAAAA|49454|13|44|14|PM|first|afternoon|lunch| +49455|AAAAAAAAADBMAAAA|49455|13|44|15|PM|first|afternoon|lunch| +49456|AAAAAAAABDBMAAAA|49456|13|44|16|PM|first|afternoon|lunch| +49457|AAAAAAAACDBMAAAA|49457|13|44|17|PM|first|afternoon|lunch| +49458|AAAAAAAADDBMAAAA|49458|13|44|18|PM|first|afternoon|lunch| +49459|AAAAAAAAEDBMAAAA|49459|13|44|19|PM|first|afternoon|lunch| +49460|AAAAAAAAFDBMAAAA|49460|13|44|20|PM|first|afternoon|lunch| +49461|AAAAAAAAGDBMAAAA|49461|13|44|21|PM|first|afternoon|lunch| +49462|AAAAAAAAHDBMAAAA|49462|13|44|22|PM|first|afternoon|lunch| +49463|AAAAAAAAIDBMAAAA|49463|13|44|23|PM|first|afternoon|lunch| +49464|AAAAAAAAJDBMAAAA|49464|13|44|24|PM|first|afternoon|lunch| +49465|AAAAAAAAKDBMAAAA|49465|13|44|25|PM|first|afternoon|lunch| +49466|AAAAAAAALDBMAAAA|49466|13|44|26|PM|first|afternoon|lunch| +49467|AAAAAAAAMDBMAAAA|49467|13|44|27|PM|first|afternoon|lunch| +49468|AAAAAAAANDBMAAAA|49468|13|44|28|PM|first|afternoon|lunch| +49469|AAAAAAAAODBMAAAA|49469|13|44|29|PM|first|afternoon|lunch| +49470|AAAAAAAAPDBMAAAA|49470|13|44|30|PM|first|afternoon|lunch| +49471|AAAAAAAAAEBMAAAA|49471|13|44|31|PM|first|afternoon|lunch| +49472|AAAAAAAABEBMAAAA|49472|13|44|32|PM|first|afternoon|lunch| +49473|AAAAAAAACEBMAAAA|49473|13|44|33|PM|first|afternoon|lunch| +49474|AAAAAAAADEBMAAAA|49474|13|44|34|PM|first|afternoon|lunch| +49475|AAAAAAAAEEBMAAAA|49475|13|44|35|PM|first|afternoon|lunch| +49476|AAAAAAAAFEBMAAAA|49476|13|44|36|PM|first|afternoon|lunch| +49477|AAAAAAAAGEBMAAAA|49477|13|44|37|PM|first|afternoon|lunch| +49478|AAAAAAAAHEBMAAAA|49478|13|44|38|PM|first|afternoon|lunch| +49479|AAAAAAAAIEBMAAAA|49479|13|44|39|PM|first|afternoon|lunch| +49480|AAAAAAAAJEBMAAAA|49480|13|44|40|PM|first|afternoon|lunch| +49481|AAAAAAAAKEBMAAAA|49481|13|44|41|PM|first|afternoon|lunch| +49482|AAAAAAAALEBMAAAA|49482|13|44|42|PM|first|afternoon|lunch| +49483|AAAAAAAAMEBMAAAA|49483|13|44|43|PM|first|afternoon|lunch| +49484|AAAAAAAANEBMAAAA|49484|13|44|44|PM|first|afternoon|lunch| +49485|AAAAAAAAOEBMAAAA|49485|13|44|45|PM|first|afternoon|lunch| +49486|AAAAAAAAPEBMAAAA|49486|13|44|46|PM|first|afternoon|lunch| +49487|AAAAAAAAAFBMAAAA|49487|13|44|47|PM|first|afternoon|lunch| +49488|AAAAAAAABFBMAAAA|49488|13|44|48|PM|first|afternoon|lunch| +49489|AAAAAAAACFBMAAAA|49489|13|44|49|PM|first|afternoon|lunch| +49490|AAAAAAAADFBMAAAA|49490|13|44|50|PM|first|afternoon|lunch| +49491|AAAAAAAAEFBMAAAA|49491|13|44|51|PM|first|afternoon|lunch| +49492|AAAAAAAAFFBMAAAA|49492|13|44|52|PM|first|afternoon|lunch| +49493|AAAAAAAAGFBMAAAA|49493|13|44|53|PM|first|afternoon|lunch| +49494|AAAAAAAAHFBMAAAA|49494|13|44|54|PM|first|afternoon|lunch| +49495|AAAAAAAAIFBMAAAA|49495|13|44|55|PM|first|afternoon|lunch| +49496|AAAAAAAAJFBMAAAA|49496|13|44|56|PM|first|afternoon|lunch| +49497|AAAAAAAAKFBMAAAA|49497|13|44|57|PM|first|afternoon|lunch| +49498|AAAAAAAALFBMAAAA|49498|13|44|58|PM|first|afternoon|lunch| +49499|AAAAAAAAMFBMAAAA|49499|13|44|59|PM|first|afternoon|lunch| +49500|AAAAAAAANFBMAAAA|49500|13|45|0|PM|first|afternoon|lunch| +49501|AAAAAAAAOFBMAAAA|49501|13|45|1|PM|first|afternoon|lunch| +49502|AAAAAAAAPFBMAAAA|49502|13|45|2|PM|first|afternoon|lunch| +49503|AAAAAAAAAGBMAAAA|49503|13|45|3|PM|first|afternoon|lunch| +49504|AAAAAAAABGBMAAAA|49504|13|45|4|PM|first|afternoon|lunch| +49505|AAAAAAAACGBMAAAA|49505|13|45|5|PM|first|afternoon|lunch| +49506|AAAAAAAADGBMAAAA|49506|13|45|6|PM|first|afternoon|lunch| +49507|AAAAAAAAEGBMAAAA|49507|13|45|7|PM|first|afternoon|lunch| +49508|AAAAAAAAFGBMAAAA|49508|13|45|8|PM|first|afternoon|lunch| +49509|AAAAAAAAGGBMAAAA|49509|13|45|9|PM|first|afternoon|lunch| +49510|AAAAAAAAHGBMAAAA|49510|13|45|10|PM|first|afternoon|lunch| +49511|AAAAAAAAIGBMAAAA|49511|13|45|11|PM|first|afternoon|lunch| +49512|AAAAAAAAJGBMAAAA|49512|13|45|12|PM|first|afternoon|lunch| +49513|AAAAAAAAKGBMAAAA|49513|13|45|13|PM|first|afternoon|lunch| +49514|AAAAAAAALGBMAAAA|49514|13|45|14|PM|first|afternoon|lunch| +49515|AAAAAAAAMGBMAAAA|49515|13|45|15|PM|first|afternoon|lunch| +49516|AAAAAAAANGBMAAAA|49516|13|45|16|PM|first|afternoon|lunch| +49517|AAAAAAAAOGBMAAAA|49517|13|45|17|PM|first|afternoon|lunch| +49518|AAAAAAAAPGBMAAAA|49518|13|45|18|PM|first|afternoon|lunch| +49519|AAAAAAAAAHBMAAAA|49519|13|45|19|PM|first|afternoon|lunch| +49520|AAAAAAAABHBMAAAA|49520|13|45|20|PM|first|afternoon|lunch| +49521|AAAAAAAACHBMAAAA|49521|13|45|21|PM|first|afternoon|lunch| +49522|AAAAAAAADHBMAAAA|49522|13|45|22|PM|first|afternoon|lunch| +49523|AAAAAAAAEHBMAAAA|49523|13|45|23|PM|first|afternoon|lunch| +49524|AAAAAAAAFHBMAAAA|49524|13|45|24|PM|first|afternoon|lunch| +49525|AAAAAAAAGHBMAAAA|49525|13|45|25|PM|first|afternoon|lunch| +49526|AAAAAAAAHHBMAAAA|49526|13|45|26|PM|first|afternoon|lunch| +49527|AAAAAAAAIHBMAAAA|49527|13|45|27|PM|first|afternoon|lunch| +49528|AAAAAAAAJHBMAAAA|49528|13|45|28|PM|first|afternoon|lunch| +49529|AAAAAAAAKHBMAAAA|49529|13|45|29|PM|first|afternoon|lunch| +49530|AAAAAAAALHBMAAAA|49530|13|45|30|PM|first|afternoon|lunch| +49531|AAAAAAAAMHBMAAAA|49531|13|45|31|PM|first|afternoon|lunch| +49532|AAAAAAAANHBMAAAA|49532|13|45|32|PM|first|afternoon|lunch| +49533|AAAAAAAAOHBMAAAA|49533|13|45|33|PM|first|afternoon|lunch| +49534|AAAAAAAAPHBMAAAA|49534|13|45|34|PM|first|afternoon|lunch| +49535|AAAAAAAAAIBMAAAA|49535|13|45|35|PM|first|afternoon|lunch| +49536|AAAAAAAABIBMAAAA|49536|13|45|36|PM|first|afternoon|lunch| +49537|AAAAAAAACIBMAAAA|49537|13|45|37|PM|first|afternoon|lunch| +49538|AAAAAAAADIBMAAAA|49538|13|45|38|PM|first|afternoon|lunch| +49539|AAAAAAAAEIBMAAAA|49539|13|45|39|PM|first|afternoon|lunch| +49540|AAAAAAAAFIBMAAAA|49540|13|45|40|PM|first|afternoon|lunch| +49541|AAAAAAAAGIBMAAAA|49541|13|45|41|PM|first|afternoon|lunch| +49542|AAAAAAAAHIBMAAAA|49542|13|45|42|PM|first|afternoon|lunch| +49543|AAAAAAAAIIBMAAAA|49543|13|45|43|PM|first|afternoon|lunch| +49544|AAAAAAAAJIBMAAAA|49544|13|45|44|PM|first|afternoon|lunch| +49545|AAAAAAAAKIBMAAAA|49545|13|45|45|PM|first|afternoon|lunch| +49546|AAAAAAAALIBMAAAA|49546|13|45|46|PM|first|afternoon|lunch| +49547|AAAAAAAAMIBMAAAA|49547|13|45|47|PM|first|afternoon|lunch| +49548|AAAAAAAANIBMAAAA|49548|13|45|48|PM|first|afternoon|lunch| +49549|AAAAAAAAOIBMAAAA|49549|13|45|49|PM|first|afternoon|lunch| +49550|AAAAAAAAPIBMAAAA|49550|13|45|50|PM|first|afternoon|lunch| +49551|AAAAAAAAAJBMAAAA|49551|13|45|51|PM|first|afternoon|lunch| +49552|AAAAAAAABJBMAAAA|49552|13|45|52|PM|first|afternoon|lunch| +49553|AAAAAAAACJBMAAAA|49553|13|45|53|PM|first|afternoon|lunch| +49554|AAAAAAAADJBMAAAA|49554|13|45|54|PM|first|afternoon|lunch| +49555|AAAAAAAAEJBMAAAA|49555|13|45|55|PM|first|afternoon|lunch| +49556|AAAAAAAAFJBMAAAA|49556|13|45|56|PM|first|afternoon|lunch| +49557|AAAAAAAAGJBMAAAA|49557|13|45|57|PM|first|afternoon|lunch| +49558|AAAAAAAAHJBMAAAA|49558|13|45|58|PM|first|afternoon|lunch| +49559|AAAAAAAAIJBMAAAA|49559|13|45|59|PM|first|afternoon|lunch| +49560|AAAAAAAAJJBMAAAA|49560|13|46|0|PM|first|afternoon|lunch| +49561|AAAAAAAAKJBMAAAA|49561|13|46|1|PM|first|afternoon|lunch| +49562|AAAAAAAALJBMAAAA|49562|13|46|2|PM|first|afternoon|lunch| +49563|AAAAAAAAMJBMAAAA|49563|13|46|3|PM|first|afternoon|lunch| +49564|AAAAAAAANJBMAAAA|49564|13|46|4|PM|first|afternoon|lunch| +49565|AAAAAAAAOJBMAAAA|49565|13|46|5|PM|first|afternoon|lunch| +49566|AAAAAAAAPJBMAAAA|49566|13|46|6|PM|first|afternoon|lunch| +49567|AAAAAAAAAKBMAAAA|49567|13|46|7|PM|first|afternoon|lunch| +49568|AAAAAAAABKBMAAAA|49568|13|46|8|PM|first|afternoon|lunch| +49569|AAAAAAAACKBMAAAA|49569|13|46|9|PM|first|afternoon|lunch| +49570|AAAAAAAADKBMAAAA|49570|13|46|10|PM|first|afternoon|lunch| +49571|AAAAAAAAEKBMAAAA|49571|13|46|11|PM|first|afternoon|lunch| +49572|AAAAAAAAFKBMAAAA|49572|13|46|12|PM|first|afternoon|lunch| +49573|AAAAAAAAGKBMAAAA|49573|13|46|13|PM|first|afternoon|lunch| +49574|AAAAAAAAHKBMAAAA|49574|13|46|14|PM|first|afternoon|lunch| +49575|AAAAAAAAIKBMAAAA|49575|13|46|15|PM|first|afternoon|lunch| +49576|AAAAAAAAJKBMAAAA|49576|13|46|16|PM|first|afternoon|lunch| +49577|AAAAAAAAKKBMAAAA|49577|13|46|17|PM|first|afternoon|lunch| +49578|AAAAAAAALKBMAAAA|49578|13|46|18|PM|first|afternoon|lunch| +49579|AAAAAAAAMKBMAAAA|49579|13|46|19|PM|first|afternoon|lunch| +49580|AAAAAAAANKBMAAAA|49580|13|46|20|PM|first|afternoon|lunch| +49581|AAAAAAAAOKBMAAAA|49581|13|46|21|PM|first|afternoon|lunch| +49582|AAAAAAAAPKBMAAAA|49582|13|46|22|PM|first|afternoon|lunch| +49583|AAAAAAAAALBMAAAA|49583|13|46|23|PM|first|afternoon|lunch| +49584|AAAAAAAABLBMAAAA|49584|13|46|24|PM|first|afternoon|lunch| +49585|AAAAAAAACLBMAAAA|49585|13|46|25|PM|first|afternoon|lunch| +49586|AAAAAAAADLBMAAAA|49586|13|46|26|PM|first|afternoon|lunch| +49587|AAAAAAAAELBMAAAA|49587|13|46|27|PM|first|afternoon|lunch| +49588|AAAAAAAAFLBMAAAA|49588|13|46|28|PM|first|afternoon|lunch| +49589|AAAAAAAAGLBMAAAA|49589|13|46|29|PM|first|afternoon|lunch| +49590|AAAAAAAAHLBMAAAA|49590|13|46|30|PM|first|afternoon|lunch| +49591|AAAAAAAAILBMAAAA|49591|13|46|31|PM|first|afternoon|lunch| +49592|AAAAAAAAJLBMAAAA|49592|13|46|32|PM|first|afternoon|lunch| +49593|AAAAAAAAKLBMAAAA|49593|13|46|33|PM|first|afternoon|lunch| +49594|AAAAAAAALLBMAAAA|49594|13|46|34|PM|first|afternoon|lunch| +49595|AAAAAAAAMLBMAAAA|49595|13|46|35|PM|first|afternoon|lunch| +49596|AAAAAAAANLBMAAAA|49596|13|46|36|PM|first|afternoon|lunch| +49597|AAAAAAAAOLBMAAAA|49597|13|46|37|PM|first|afternoon|lunch| +49598|AAAAAAAAPLBMAAAA|49598|13|46|38|PM|first|afternoon|lunch| +49599|AAAAAAAAAMBMAAAA|49599|13|46|39|PM|first|afternoon|lunch| +49600|AAAAAAAABMBMAAAA|49600|13|46|40|PM|first|afternoon|lunch| +49601|AAAAAAAACMBMAAAA|49601|13|46|41|PM|first|afternoon|lunch| +49602|AAAAAAAADMBMAAAA|49602|13|46|42|PM|first|afternoon|lunch| +49603|AAAAAAAAEMBMAAAA|49603|13|46|43|PM|first|afternoon|lunch| +49604|AAAAAAAAFMBMAAAA|49604|13|46|44|PM|first|afternoon|lunch| +49605|AAAAAAAAGMBMAAAA|49605|13|46|45|PM|first|afternoon|lunch| +49606|AAAAAAAAHMBMAAAA|49606|13|46|46|PM|first|afternoon|lunch| +49607|AAAAAAAAIMBMAAAA|49607|13|46|47|PM|first|afternoon|lunch| +49608|AAAAAAAAJMBMAAAA|49608|13|46|48|PM|first|afternoon|lunch| +49609|AAAAAAAAKMBMAAAA|49609|13|46|49|PM|first|afternoon|lunch| +49610|AAAAAAAALMBMAAAA|49610|13|46|50|PM|first|afternoon|lunch| +49611|AAAAAAAAMMBMAAAA|49611|13|46|51|PM|first|afternoon|lunch| +49612|AAAAAAAANMBMAAAA|49612|13|46|52|PM|first|afternoon|lunch| +49613|AAAAAAAAOMBMAAAA|49613|13|46|53|PM|first|afternoon|lunch| +49614|AAAAAAAAPMBMAAAA|49614|13|46|54|PM|first|afternoon|lunch| +49615|AAAAAAAAANBMAAAA|49615|13|46|55|PM|first|afternoon|lunch| +49616|AAAAAAAABNBMAAAA|49616|13|46|56|PM|first|afternoon|lunch| +49617|AAAAAAAACNBMAAAA|49617|13|46|57|PM|first|afternoon|lunch| +49618|AAAAAAAADNBMAAAA|49618|13|46|58|PM|first|afternoon|lunch| +49619|AAAAAAAAENBMAAAA|49619|13|46|59|PM|first|afternoon|lunch| +49620|AAAAAAAAFNBMAAAA|49620|13|47|0|PM|first|afternoon|lunch| +49621|AAAAAAAAGNBMAAAA|49621|13|47|1|PM|first|afternoon|lunch| +49622|AAAAAAAAHNBMAAAA|49622|13|47|2|PM|first|afternoon|lunch| +49623|AAAAAAAAINBMAAAA|49623|13|47|3|PM|first|afternoon|lunch| +49624|AAAAAAAAJNBMAAAA|49624|13|47|4|PM|first|afternoon|lunch| +49625|AAAAAAAAKNBMAAAA|49625|13|47|5|PM|first|afternoon|lunch| +49626|AAAAAAAALNBMAAAA|49626|13|47|6|PM|first|afternoon|lunch| +49627|AAAAAAAAMNBMAAAA|49627|13|47|7|PM|first|afternoon|lunch| +49628|AAAAAAAANNBMAAAA|49628|13|47|8|PM|first|afternoon|lunch| +49629|AAAAAAAAONBMAAAA|49629|13|47|9|PM|first|afternoon|lunch| +49630|AAAAAAAAPNBMAAAA|49630|13|47|10|PM|first|afternoon|lunch| +49631|AAAAAAAAAOBMAAAA|49631|13|47|11|PM|first|afternoon|lunch| +49632|AAAAAAAABOBMAAAA|49632|13|47|12|PM|first|afternoon|lunch| +49633|AAAAAAAACOBMAAAA|49633|13|47|13|PM|first|afternoon|lunch| +49634|AAAAAAAADOBMAAAA|49634|13|47|14|PM|first|afternoon|lunch| +49635|AAAAAAAAEOBMAAAA|49635|13|47|15|PM|first|afternoon|lunch| +49636|AAAAAAAAFOBMAAAA|49636|13|47|16|PM|first|afternoon|lunch| +49637|AAAAAAAAGOBMAAAA|49637|13|47|17|PM|first|afternoon|lunch| +49638|AAAAAAAAHOBMAAAA|49638|13|47|18|PM|first|afternoon|lunch| +49639|AAAAAAAAIOBMAAAA|49639|13|47|19|PM|first|afternoon|lunch| +49640|AAAAAAAAJOBMAAAA|49640|13|47|20|PM|first|afternoon|lunch| +49641|AAAAAAAAKOBMAAAA|49641|13|47|21|PM|first|afternoon|lunch| +49642|AAAAAAAALOBMAAAA|49642|13|47|22|PM|first|afternoon|lunch| +49643|AAAAAAAAMOBMAAAA|49643|13|47|23|PM|first|afternoon|lunch| +49644|AAAAAAAANOBMAAAA|49644|13|47|24|PM|first|afternoon|lunch| +49645|AAAAAAAAOOBMAAAA|49645|13|47|25|PM|first|afternoon|lunch| +49646|AAAAAAAAPOBMAAAA|49646|13|47|26|PM|first|afternoon|lunch| +49647|AAAAAAAAAPBMAAAA|49647|13|47|27|PM|first|afternoon|lunch| +49648|AAAAAAAABPBMAAAA|49648|13|47|28|PM|first|afternoon|lunch| +49649|AAAAAAAACPBMAAAA|49649|13|47|29|PM|first|afternoon|lunch| +49650|AAAAAAAADPBMAAAA|49650|13|47|30|PM|first|afternoon|lunch| +49651|AAAAAAAAEPBMAAAA|49651|13|47|31|PM|first|afternoon|lunch| +49652|AAAAAAAAFPBMAAAA|49652|13|47|32|PM|first|afternoon|lunch| +49653|AAAAAAAAGPBMAAAA|49653|13|47|33|PM|first|afternoon|lunch| +49654|AAAAAAAAHPBMAAAA|49654|13|47|34|PM|first|afternoon|lunch| +49655|AAAAAAAAIPBMAAAA|49655|13|47|35|PM|first|afternoon|lunch| +49656|AAAAAAAAJPBMAAAA|49656|13|47|36|PM|first|afternoon|lunch| +49657|AAAAAAAAKPBMAAAA|49657|13|47|37|PM|first|afternoon|lunch| +49658|AAAAAAAALPBMAAAA|49658|13|47|38|PM|first|afternoon|lunch| +49659|AAAAAAAAMPBMAAAA|49659|13|47|39|PM|first|afternoon|lunch| +49660|AAAAAAAANPBMAAAA|49660|13|47|40|PM|first|afternoon|lunch| +49661|AAAAAAAAOPBMAAAA|49661|13|47|41|PM|first|afternoon|lunch| +49662|AAAAAAAAPPBMAAAA|49662|13|47|42|PM|first|afternoon|lunch| +49663|AAAAAAAAAACMAAAA|49663|13|47|43|PM|first|afternoon|lunch| +49664|AAAAAAAABACMAAAA|49664|13|47|44|PM|first|afternoon|lunch| +49665|AAAAAAAACACMAAAA|49665|13|47|45|PM|first|afternoon|lunch| +49666|AAAAAAAADACMAAAA|49666|13|47|46|PM|first|afternoon|lunch| +49667|AAAAAAAAEACMAAAA|49667|13|47|47|PM|first|afternoon|lunch| +49668|AAAAAAAAFACMAAAA|49668|13|47|48|PM|first|afternoon|lunch| +49669|AAAAAAAAGACMAAAA|49669|13|47|49|PM|first|afternoon|lunch| +49670|AAAAAAAAHACMAAAA|49670|13|47|50|PM|first|afternoon|lunch| +49671|AAAAAAAAIACMAAAA|49671|13|47|51|PM|first|afternoon|lunch| +49672|AAAAAAAAJACMAAAA|49672|13|47|52|PM|first|afternoon|lunch| +49673|AAAAAAAAKACMAAAA|49673|13|47|53|PM|first|afternoon|lunch| +49674|AAAAAAAALACMAAAA|49674|13|47|54|PM|first|afternoon|lunch| +49675|AAAAAAAAMACMAAAA|49675|13|47|55|PM|first|afternoon|lunch| +49676|AAAAAAAANACMAAAA|49676|13|47|56|PM|first|afternoon|lunch| +49677|AAAAAAAAOACMAAAA|49677|13|47|57|PM|first|afternoon|lunch| +49678|AAAAAAAAPACMAAAA|49678|13|47|58|PM|first|afternoon|lunch| +49679|AAAAAAAAABCMAAAA|49679|13|47|59|PM|first|afternoon|lunch| +49680|AAAAAAAABBCMAAAA|49680|13|48|0|PM|first|afternoon|lunch| +49681|AAAAAAAACBCMAAAA|49681|13|48|1|PM|first|afternoon|lunch| +49682|AAAAAAAADBCMAAAA|49682|13|48|2|PM|first|afternoon|lunch| +49683|AAAAAAAAEBCMAAAA|49683|13|48|3|PM|first|afternoon|lunch| +49684|AAAAAAAAFBCMAAAA|49684|13|48|4|PM|first|afternoon|lunch| +49685|AAAAAAAAGBCMAAAA|49685|13|48|5|PM|first|afternoon|lunch| +49686|AAAAAAAAHBCMAAAA|49686|13|48|6|PM|first|afternoon|lunch| +49687|AAAAAAAAIBCMAAAA|49687|13|48|7|PM|first|afternoon|lunch| +49688|AAAAAAAAJBCMAAAA|49688|13|48|8|PM|first|afternoon|lunch| +49689|AAAAAAAAKBCMAAAA|49689|13|48|9|PM|first|afternoon|lunch| +49690|AAAAAAAALBCMAAAA|49690|13|48|10|PM|first|afternoon|lunch| +49691|AAAAAAAAMBCMAAAA|49691|13|48|11|PM|first|afternoon|lunch| +49692|AAAAAAAANBCMAAAA|49692|13|48|12|PM|first|afternoon|lunch| +49693|AAAAAAAAOBCMAAAA|49693|13|48|13|PM|first|afternoon|lunch| +49694|AAAAAAAAPBCMAAAA|49694|13|48|14|PM|first|afternoon|lunch| +49695|AAAAAAAAACCMAAAA|49695|13|48|15|PM|first|afternoon|lunch| +49696|AAAAAAAABCCMAAAA|49696|13|48|16|PM|first|afternoon|lunch| +49697|AAAAAAAACCCMAAAA|49697|13|48|17|PM|first|afternoon|lunch| +49698|AAAAAAAADCCMAAAA|49698|13|48|18|PM|first|afternoon|lunch| +49699|AAAAAAAAECCMAAAA|49699|13|48|19|PM|first|afternoon|lunch| +49700|AAAAAAAAFCCMAAAA|49700|13|48|20|PM|first|afternoon|lunch| +49701|AAAAAAAAGCCMAAAA|49701|13|48|21|PM|first|afternoon|lunch| +49702|AAAAAAAAHCCMAAAA|49702|13|48|22|PM|first|afternoon|lunch| +49703|AAAAAAAAICCMAAAA|49703|13|48|23|PM|first|afternoon|lunch| +49704|AAAAAAAAJCCMAAAA|49704|13|48|24|PM|first|afternoon|lunch| +49705|AAAAAAAAKCCMAAAA|49705|13|48|25|PM|first|afternoon|lunch| +49706|AAAAAAAALCCMAAAA|49706|13|48|26|PM|first|afternoon|lunch| +49707|AAAAAAAAMCCMAAAA|49707|13|48|27|PM|first|afternoon|lunch| +49708|AAAAAAAANCCMAAAA|49708|13|48|28|PM|first|afternoon|lunch| +49709|AAAAAAAAOCCMAAAA|49709|13|48|29|PM|first|afternoon|lunch| +49710|AAAAAAAAPCCMAAAA|49710|13|48|30|PM|first|afternoon|lunch| +49711|AAAAAAAAADCMAAAA|49711|13|48|31|PM|first|afternoon|lunch| +49712|AAAAAAAABDCMAAAA|49712|13|48|32|PM|first|afternoon|lunch| +49713|AAAAAAAACDCMAAAA|49713|13|48|33|PM|first|afternoon|lunch| +49714|AAAAAAAADDCMAAAA|49714|13|48|34|PM|first|afternoon|lunch| +49715|AAAAAAAAEDCMAAAA|49715|13|48|35|PM|first|afternoon|lunch| +49716|AAAAAAAAFDCMAAAA|49716|13|48|36|PM|first|afternoon|lunch| +49717|AAAAAAAAGDCMAAAA|49717|13|48|37|PM|first|afternoon|lunch| +49718|AAAAAAAAHDCMAAAA|49718|13|48|38|PM|first|afternoon|lunch| +49719|AAAAAAAAIDCMAAAA|49719|13|48|39|PM|first|afternoon|lunch| +49720|AAAAAAAAJDCMAAAA|49720|13|48|40|PM|first|afternoon|lunch| +49721|AAAAAAAAKDCMAAAA|49721|13|48|41|PM|first|afternoon|lunch| +49722|AAAAAAAALDCMAAAA|49722|13|48|42|PM|first|afternoon|lunch| +49723|AAAAAAAAMDCMAAAA|49723|13|48|43|PM|first|afternoon|lunch| +49724|AAAAAAAANDCMAAAA|49724|13|48|44|PM|first|afternoon|lunch| +49725|AAAAAAAAODCMAAAA|49725|13|48|45|PM|first|afternoon|lunch| +49726|AAAAAAAAPDCMAAAA|49726|13|48|46|PM|first|afternoon|lunch| +49727|AAAAAAAAAECMAAAA|49727|13|48|47|PM|first|afternoon|lunch| +49728|AAAAAAAABECMAAAA|49728|13|48|48|PM|first|afternoon|lunch| +49729|AAAAAAAACECMAAAA|49729|13|48|49|PM|first|afternoon|lunch| +49730|AAAAAAAADECMAAAA|49730|13|48|50|PM|first|afternoon|lunch| +49731|AAAAAAAAEECMAAAA|49731|13|48|51|PM|first|afternoon|lunch| +49732|AAAAAAAAFECMAAAA|49732|13|48|52|PM|first|afternoon|lunch| +49733|AAAAAAAAGECMAAAA|49733|13|48|53|PM|first|afternoon|lunch| +49734|AAAAAAAAHECMAAAA|49734|13|48|54|PM|first|afternoon|lunch| +49735|AAAAAAAAIECMAAAA|49735|13|48|55|PM|first|afternoon|lunch| +49736|AAAAAAAAJECMAAAA|49736|13|48|56|PM|first|afternoon|lunch| +49737|AAAAAAAAKECMAAAA|49737|13|48|57|PM|first|afternoon|lunch| +49738|AAAAAAAALECMAAAA|49738|13|48|58|PM|first|afternoon|lunch| +49739|AAAAAAAAMECMAAAA|49739|13|48|59|PM|first|afternoon|lunch| +49740|AAAAAAAANECMAAAA|49740|13|49|0|PM|first|afternoon|lunch| +49741|AAAAAAAAOECMAAAA|49741|13|49|1|PM|first|afternoon|lunch| +49742|AAAAAAAAPECMAAAA|49742|13|49|2|PM|first|afternoon|lunch| +49743|AAAAAAAAAFCMAAAA|49743|13|49|3|PM|first|afternoon|lunch| +49744|AAAAAAAABFCMAAAA|49744|13|49|4|PM|first|afternoon|lunch| +49745|AAAAAAAACFCMAAAA|49745|13|49|5|PM|first|afternoon|lunch| +49746|AAAAAAAADFCMAAAA|49746|13|49|6|PM|first|afternoon|lunch| +49747|AAAAAAAAEFCMAAAA|49747|13|49|7|PM|first|afternoon|lunch| +49748|AAAAAAAAFFCMAAAA|49748|13|49|8|PM|first|afternoon|lunch| +49749|AAAAAAAAGFCMAAAA|49749|13|49|9|PM|first|afternoon|lunch| +49750|AAAAAAAAHFCMAAAA|49750|13|49|10|PM|first|afternoon|lunch| +49751|AAAAAAAAIFCMAAAA|49751|13|49|11|PM|first|afternoon|lunch| +49752|AAAAAAAAJFCMAAAA|49752|13|49|12|PM|first|afternoon|lunch| +49753|AAAAAAAAKFCMAAAA|49753|13|49|13|PM|first|afternoon|lunch| +49754|AAAAAAAALFCMAAAA|49754|13|49|14|PM|first|afternoon|lunch| +49755|AAAAAAAAMFCMAAAA|49755|13|49|15|PM|first|afternoon|lunch| +49756|AAAAAAAANFCMAAAA|49756|13|49|16|PM|first|afternoon|lunch| +49757|AAAAAAAAOFCMAAAA|49757|13|49|17|PM|first|afternoon|lunch| +49758|AAAAAAAAPFCMAAAA|49758|13|49|18|PM|first|afternoon|lunch| +49759|AAAAAAAAAGCMAAAA|49759|13|49|19|PM|first|afternoon|lunch| +49760|AAAAAAAABGCMAAAA|49760|13|49|20|PM|first|afternoon|lunch| +49761|AAAAAAAACGCMAAAA|49761|13|49|21|PM|first|afternoon|lunch| +49762|AAAAAAAADGCMAAAA|49762|13|49|22|PM|first|afternoon|lunch| +49763|AAAAAAAAEGCMAAAA|49763|13|49|23|PM|first|afternoon|lunch| +49764|AAAAAAAAFGCMAAAA|49764|13|49|24|PM|first|afternoon|lunch| +49765|AAAAAAAAGGCMAAAA|49765|13|49|25|PM|first|afternoon|lunch| +49766|AAAAAAAAHGCMAAAA|49766|13|49|26|PM|first|afternoon|lunch| +49767|AAAAAAAAIGCMAAAA|49767|13|49|27|PM|first|afternoon|lunch| +49768|AAAAAAAAJGCMAAAA|49768|13|49|28|PM|first|afternoon|lunch| +49769|AAAAAAAAKGCMAAAA|49769|13|49|29|PM|first|afternoon|lunch| +49770|AAAAAAAALGCMAAAA|49770|13|49|30|PM|first|afternoon|lunch| +49771|AAAAAAAAMGCMAAAA|49771|13|49|31|PM|first|afternoon|lunch| +49772|AAAAAAAANGCMAAAA|49772|13|49|32|PM|first|afternoon|lunch| +49773|AAAAAAAAOGCMAAAA|49773|13|49|33|PM|first|afternoon|lunch| +49774|AAAAAAAAPGCMAAAA|49774|13|49|34|PM|first|afternoon|lunch| +49775|AAAAAAAAAHCMAAAA|49775|13|49|35|PM|first|afternoon|lunch| +49776|AAAAAAAABHCMAAAA|49776|13|49|36|PM|first|afternoon|lunch| +49777|AAAAAAAACHCMAAAA|49777|13|49|37|PM|first|afternoon|lunch| +49778|AAAAAAAADHCMAAAA|49778|13|49|38|PM|first|afternoon|lunch| +49779|AAAAAAAAEHCMAAAA|49779|13|49|39|PM|first|afternoon|lunch| +49780|AAAAAAAAFHCMAAAA|49780|13|49|40|PM|first|afternoon|lunch| +49781|AAAAAAAAGHCMAAAA|49781|13|49|41|PM|first|afternoon|lunch| +49782|AAAAAAAAHHCMAAAA|49782|13|49|42|PM|first|afternoon|lunch| +49783|AAAAAAAAIHCMAAAA|49783|13|49|43|PM|first|afternoon|lunch| +49784|AAAAAAAAJHCMAAAA|49784|13|49|44|PM|first|afternoon|lunch| +49785|AAAAAAAAKHCMAAAA|49785|13|49|45|PM|first|afternoon|lunch| +49786|AAAAAAAALHCMAAAA|49786|13|49|46|PM|first|afternoon|lunch| +49787|AAAAAAAAMHCMAAAA|49787|13|49|47|PM|first|afternoon|lunch| +49788|AAAAAAAANHCMAAAA|49788|13|49|48|PM|first|afternoon|lunch| +49789|AAAAAAAAOHCMAAAA|49789|13|49|49|PM|first|afternoon|lunch| +49790|AAAAAAAAPHCMAAAA|49790|13|49|50|PM|first|afternoon|lunch| +49791|AAAAAAAAAICMAAAA|49791|13|49|51|PM|first|afternoon|lunch| +49792|AAAAAAAABICMAAAA|49792|13|49|52|PM|first|afternoon|lunch| +49793|AAAAAAAACICMAAAA|49793|13|49|53|PM|first|afternoon|lunch| +49794|AAAAAAAADICMAAAA|49794|13|49|54|PM|first|afternoon|lunch| +49795|AAAAAAAAEICMAAAA|49795|13|49|55|PM|first|afternoon|lunch| +49796|AAAAAAAAFICMAAAA|49796|13|49|56|PM|first|afternoon|lunch| +49797|AAAAAAAAGICMAAAA|49797|13|49|57|PM|first|afternoon|lunch| +49798|AAAAAAAAHICMAAAA|49798|13|49|58|PM|first|afternoon|lunch| +49799|AAAAAAAAIICMAAAA|49799|13|49|59|PM|first|afternoon|lunch| +49800|AAAAAAAAJICMAAAA|49800|13|50|0|PM|first|afternoon|lunch| +49801|AAAAAAAAKICMAAAA|49801|13|50|1|PM|first|afternoon|lunch| +49802|AAAAAAAALICMAAAA|49802|13|50|2|PM|first|afternoon|lunch| +49803|AAAAAAAAMICMAAAA|49803|13|50|3|PM|first|afternoon|lunch| +49804|AAAAAAAANICMAAAA|49804|13|50|4|PM|first|afternoon|lunch| +49805|AAAAAAAAOICMAAAA|49805|13|50|5|PM|first|afternoon|lunch| +49806|AAAAAAAAPICMAAAA|49806|13|50|6|PM|first|afternoon|lunch| +49807|AAAAAAAAAJCMAAAA|49807|13|50|7|PM|first|afternoon|lunch| +49808|AAAAAAAABJCMAAAA|49808|13|50|8|PM|first|afternoon|lunch| +49809|AAAAAAAACJCMAAAA|49809|13|50|9|PM|first|afternoon|lunch| +49810|AAAAAAAADJCMAAAA|49810|13|50|10|PM|first|afternoon|lunch| +49811|AAAAAAAAEJCMAAAA|49811|13|50|11|PM|first|afternoon|lunch| +49812|AAAAAAAAFJCMAAAA|49812|13|50|12|PM|first|afternoon|lunch| +49813|AAAAAAAAGJCMAAAA|49813|13|50|13|PM|first|afternoon|lunch| +49814|AAAAAAAAHJCMAAAA|49814|13|50|14|PM|first|afternoon|lunch| +49815|AAAAAAAAIJCMAAAA|49815|13|50|15|PM|first|afternoon|lunch| +49816|AAAAAAAAJJCMAAAA|49816|13|50|16|PM|first|afternoon|lunch| +49817|AAAAAAAAKJCMAAAA|49817|13|50|17|PM|first|afternoon|lunch| +49818|AAAAAAAALJCMAAAA|49818|13|50|18|PM|first|afternoon|lunch| +49819|AAAAAAAAMJCMAAAA|49819|13|50|19|PM|first|afternoon|lunch| +49820|AAAAAAAANJCMAAAA|49820|13|50|20|PM|first|afternoon|lunch| +49821|AAAAAAAAOJCMAAAA|49821|13|50|21|PM|first|afternoon|lunch| +49822|AAAAAAAAPJCMAAAA|49822|13|50|22|PM|first|afternoon|lunch| +49823|AAAAAAAAAKCMAAAA|49823|13|50|23|PM|first|afternoon|lunch| +49824|AAAAAAAABKCMAAAA|49824|13|50|24|PM|first|afternoon|lunch| +49825|AAAAAAAACKCMAAAA|49825|13|50|25|PM|first|afternoon|lunch| +49826|AAAAAAAADKCMAAAA|49826|13|50|26|PM|first|afternoon|lunch| +49827|AAAAAAAAEKCMAAAA|49827|13|50|27|PM|first|afternoon|lunch| +49828|AAAAAAAAFKCMAAAA|49828|13|50|28|PM|first|afternoon|lunch| +49829|AAAAAAAAGKCMAAAA|49829|13|50|29|PM|first|afternoon|lunch| +49830|AAAAAAAAHKCMAAAA|49830|13|50|30|PM|first|afternoon|lunch| +49831|AAAAAAAAIKCMAAAA|49831|13|50|31|PM|first|afternoon|lunch| +49832|AAAAAAAAJKCMAAAA|49832|13|50|32|PM|first|afternoon|lunch| +49833|AAAAAAAAKKCMAAAA|49833|13|50|33|PM|first|afternoon|lunch| +49834|AAAAAAAALKCMAAAA|49834|13|50|34|PM|first|afternoon|lunch| +49835|AAAAAAAAMKCMAAAA|49835|13|50|35|PM|first|afternoon|lunch| +49836|AAAAAAAANKCMAAAA|49836|13|50|36|PM|first|afternoon|lunch| +49837|AAAAAAAAOKCMAAAA|49837|13|50|37|PM|first|afternoon|lunch| +49838|AAAAAAAAPKCMAAAA|49838|13|50|38|PM|first|afternoon|lunch| +49839|AAAAAAAAALCMAAAA|49839|13|50|39|PM|first|afternoon|lunch| +49840|AAAAAAAABLCMAAAA|49840|13|50|40|PM|first|afternoon|lunch| +49841|AAAAAAAACLCMAAAA|49841|13|50|41|PM|first|afternoon|lunch| +49842|AAAAAAAADLCMAAAA|49842|13|50|42|PM|first|afternoon|lunch| +49843|AAAAAAAAELCMAAAA|49843|13|50|43|PM|first|afternoon|lunch| +49844|AAAAAAAAFLCMAAAA|49844|13|50|44|PM|first|afternoon|lunch| +49845|AAAAAAAAGLCMAAAA|49845|13|50|45|PM|first|afternoon|lunch| +49846|AAAAAAAAHLCMAAAA|49846|13|50|46|PM|first|afternoon|lunch| +49847|AAAAAAAAILCMAAAA|49847|13|50|47|PM|first|afternoon|lunch| +49848|AAAAAAAAJLCMAAAA|49848|13|50|48|PM|first|afternoon|lunch| +49849|AAAAAAAAKLCMAAAA|49849|13|50|49|PM|first|afternoon|lunch| +49850|AAAAAAAALLCMAAAA|49850|13|50|50|PM|first|afternoon|lunch| +49851|AAAAAAAAMLCMAAAA|49851|13|50|51|PM|first|afternoon|lunch| +49852|AAAAAAAANLCMAAAA|49852|13|50|52|PM|first|afternoon|lunch| +49853|AAAAAAAAOLCMAAAA|49853|13|50|53|PM|first|afternoon|lunch| +49854|AAAAAAAAPLCMAAAA|49854|13|50|54|PM|first|afternoon|lunch| +49855|AAAAAAAAAMCMAAAA|49855|13|50|55|PM|first|afternoon|lunch| +49856|AAAAAAAABMCMAAAA|49856|13|50|56|PM|first|afternoon|lunch| +49857|AAAAAAAACMCMAAAA|49857|13|50|57|PM|first|afternoon|lunch| +49858|AAAAAAAADMCMAAAA|49858|13|50|58|PM|first|afternoon|lunch| +49859|AAAAAAAAEMCMAAAA|49859|13|50|59|PM|first|afternoon|lunch| +49860|AAAAAAAAFMCMAAAA|49860|13|51|0|PM|first|afternoon|lunch| +49861|AAAAAAAAGMCMAAAA|49861|13|51|1|PM|first|afternoon|lunch| +49862|AAAAAAAAHMCMAAAA|49862|13|51|2|PM|first|afternoon|lunch| +49863|AAAAAAAAIMCMAAAA|49863|13|51|3|PM|first|afternoon|lunch| +49864|AAAAAAAAJMCMAAAA|49864|13|51|4|PM|first|afternoon|lunch| +49865|AAAAAAAAKMCMAAAA|49865|13|51|5|PM|first|afternoon|lunch| +49866|AAAAAAAALMCMAAAA|49866|13|51|6|PM|first|afternoon|lunch| +49867|AAAAAAAAMMCMAAAA|49867|13|51|7|PM|first|afternoon|lunch| +49868|AAAAAAAANMCMAAAA|49868|13|51|8|PM|first|afternoon|lunch| +49869|AAAAAAAAOMCMAAAA|49869|13|51|9|PM|first|afternoon|lunch| +49870|AAAAAAAAPMCMAAAA|49870|13|51|10|PM|first|afternoon|lunch| +49871|AAAAAAAAANCMAAAA|49871|13|51|11|PM|first|afternoon|lunch| +49872|AAAAAAAABNCMAAAA|49872|13|51|12|PM|first|afternoon|lunch| +49873|AAAAAAAACNCMAAAA|49873|13|51|13|PM|first|afternoon|lunch| +49874|AAAAAAAADNCMAAAA|49874|13|51|14|PM|first|afternoon|lunch| +49875|AAAAAAAAENCMAAAA|49875|13|51|15|PM|first|afternoon|lunch| +49876|AAAAAAAAFNCMAAAA|49876|13|51|16|PM|first|afternoon|lunch| +49877|AAAAAAAAGNCMAAAA|49877|13|51|17|PM|first|afternoon|lunch| +49878|AAAAAAAAHNCMAAAA|49878|13|51|18|PM|first|afternoon|lunch| +49879|AAAAAAAAINCMAAAA|49879|13|51|19|PM|first|afternoon|lunch| +49880|AAAAAAAAJNCMAAAA|49880|13|51|20|PM|first|afternoon|lunch| +49881|AAAAAAAAKNCMAAAA|49881|13|51|21|PM|first|afternoon|lunch| +49882|AAAAAAAALNCMAAAA|49882|13|51|22|PM|first|afternoon|lunch| +49883|AAAAAAAAMNCMAAAA|49883|13|51|23|PM|first|afternoon|lunch| +49884|AAAAAAAANNCMAAAA|49884|13|51|24|PM|first|afternoon|lunch| +49885|AAAAAAAAONCMAAAA|49885|13|51|25|PM|first|afternoon|lunch| +49886|AAAAAAAAPNCMAAAA|49886|13|51|26|PM|first|afternoon|lunch| +49887|AAAAAAAAAOCMAAAA|49887|13|51|27|PM|first|afternoon|lunch| +49888|AAAAAAAABOCMAAAA|49888|13|51|28|PM|first|afternoon|lunch| +49889|AAAAAAAACOCMAAAA|49889|13|51|29|PM|first|afternoon|lunch| +49890|AAAAAAAADOCMAAAA|49890|13|51|30|PM|first|afternoon|lunch| +49891|AAAAAAAAEOCMAAAA|49891|13|51|31|PM|first|afternoon|lunch| +49892|AAAAAAAAFOCMAAAA|49892|13|51|32|PM|first|afternoon|lunch| +49893|AAAAAAAAGOCMAAAA|49893|13|51|33|PM|first|afternoon|lunch| +49894|AAAAAAAAHOCMAAAA|49894|13|51|34|PM|first|afternoon|lunch| +49895|AAAAAAAAIOCMAAAA|49895|13|51|35|PM|first|afternoon|lunch| +49896|AAAAAAAAJOCMAAAA|49896|13|51|36|PM|first|afternoon|lunch| +49897|AAAAAAAAKOCMAAAA|49897|13|51|37|PM|first|afternoon|lunch| +49898|AAAAAAAALOCMAAAA|49898|13|51|38|PM|first|afternoon|lunch| +49899|AAAAAAAAMOCMAAAA|49899|13|51|39|PM|first|afternoon|lunch| +49900|AAAAAAAANOCMAAAA|49900|13|51|40|PM|first|afternoon|lunch| +49901|AAAAAAAAOOCMAAAA|49901|13|51|41|PM|first|afternoon|lunch| +49902|AAAAAAAAPOCMAAAA|49902|13|51|42|PM|first|afternoon|lunch| +49903|AAAAAAAAAPCMAAAA|49903|13|51|43|PM|first|afternoon|lunch| +49904|AAAAAAAABPCMAAAA|49904|13|51|44|PM|first|afternoon|lunch| +49905|AAAAAAAACPCMAAAA|49905|13|51|45|PM|first|afternoon|lunch| +49906|AAAAAAAADPCMAAAA|49906|13|51|46|PM|first|afternoon|lunch| +49907|AAAAAAAAEPCMAAAA|49907|13|51|47|PM|first|afternoon|lunch| +49908|AAAAAAAAFPCMAAAA|49908|13|51|48|PM|first|afternoon|lunch| +49909|AAAAAAAAGPCMAAAA|49909|13|51|49|PM|first|afternoon|lunch| +49910|AAAAAAAAHPCMAAAA|49910|13|51|50|PM|first|afternoon|lunch| +49911|AAAAAAAAIPCMAAAA|49911|13|51|51|PM|first|afternoon|lunch| +49912|AAAAAAAAJPCMAAAA|49912|13|51|52|PM|first|afternoon|lunch| +49913|AAAAAAAAKPCMAAAA|49913|13|51|53|PM|first|afternoon|lunch| +49914|AAAAAAAALPCMAAAA|49914|13|51|54|PM|first|afternoon|lunch| +49915|AAAAAAAAMPCMAAAA|49915|13|51|55|PM|first|afternoon|lunch| +49916|AAAAAAAANPCMAAAA|49916|13|51|56|PM|first|afternoon|lunch| +49917|AAAAAAAAOPCMAAAA|49917|13|51|57|PM|first|afternoon|lunch| +49918|AAAAAAAAPPCMAAAA|49918|13|51|58|PM|first|afternoon|lunch| +49919|AAAAAAAAAADMAAAA|49919|13|51|59|PM|first|afternoon|lunch| +49920|AAAAAAAABADMAAAA|49920|13|52|0|PM|first|afternoon|lunch| +49921|AAAAAAAACADMAAAA|49921|13|52|1|PM|first|afternoon|lunch| +49922|AAAAAAAADADMAAAA|49922|13|52|2|PM|first|afternoon|lunch| +49923|AAAAAAAAEADMAAAA|49923|13|52|3|PM|first|afternoon|lunch| +49924|AAAAAAAAFADMAAAA|49924|13|52|4|PM|first|afternoon|lunch| +49925|AAAAAAAAGADMAAAA|49925|13|52|5|PM|first|afternoon|lunch| +49926|AAAAAAAAHADMAAAA|49926|13|52|6|PM|first|afternoon|lunch| +49927|AAAAAAAAIADMAAAA|49927|13|52|7|PM|first|afternoon|lunch| +49928|AAAAAAAAJADMAAAA|49928|13|52|8|PM|first|afternoon|lunch| +49929|AAAAAAAAKADMAAAA|49929|13|52|9|PM|first|afternoon|lunch| +49930|AAAAAAAALADMAAAA|49930|13|52|10|PM|first|afternoon|lunch| +49931|AAAAAAAAMADMAAAA|49931|13|52|11|PM|first|afternoon|lunch| +49932|AAAAAAAANADMAAAA|49932|13|52|12|PM|first|afternoon|lunch| +49933|AAAAAAAAOADMAAAA|49933|13|52|13|PM|first|afternoon|lunch| +49934|AAAAAAAAPADMAAAA|49934|13|52|14|PM|first|afternoon|lunch| +49935|AAAAAAAAABDMAAAA|49935|13|52|15|PM|first|afternoon|lunch| +49936|AAAAAAAABBDMAAAA|49936|13|52|16|PM|first|afternoon|lunch| +49937|AAAAAAAACBDMAAAA|49937|13|52|17|PM|first|afternoon|lunch| +49938|AAAAAAAADBDMAAAA|49938|13|52|18|PM|first|afternoon|lunch| +49939|AAAAAAAAEBDMAAAA|49939|13|52|19|PM|first|afternoon|lunch| +49940|AAAAAAAAFBDMAAAA|49940|13|52|20|PM|first|afternoon|lunch| +49941|AAAAAAAAGBDMAAAA|49941|13|52|21|PM|first|afternoon|lunch| +49942|AAAAAAAAHBDMAAAA|49942|13|52|22|PM|first|afternoon|lunch| +49943|AAAAAAAAIBDMAAAA|49943|13|52|23|PM|first|afternoon|lunch| +49944|AAAAAAAAJBDMAAAA|49944|13|52|24|PM|first|afternoon|lunch| +49945|AAAAAAAAKBDMAAAA|49945|13|52|25|PM|first|afternoon|lunch| +49946|AAAAAAAALBDMAAAA|49946|13|52|26|PM|first|afternoon|lunch| +49947|AAAAAAAAMBDMAAAA|49947|13|52|27|PM|first|afternoon|lunch| +49948|AAAAAAAANBDMAAAA|49948|13|52|28|PM|first|afternoon|lunch| +49949|AAAAAAAAOBDMAAAA|49949|13|52|29|PM|first|afternoon|lunch| +49950|AAAAAAAAPBDMAAAA|49950|13|52|30|PM|first|afternoon|lunch| +49951|AAAAAAAAACDMAAAA|49951|13|52|31|PM|first|afternoon|lunch| +49952|AAAAAAAABCDMAAAA|49952|13|52|32|PM|first|afternoon|lunch| +49953|AAAAAAAACCDMAAAA|49953|13|52|33|PM|first|afternoon|lunch| +49954|AAAAAAAADCDMAAAA|49954|13|52|34|PM|first|afternoon|lunch| +49955|AAAAAAAAECDMAAAA|49955|13|52|35|PM|first|afternoon|lunch| +49956|AAAAAAAAFCDMAAAA|49956|13|52|36|PM|first|afternoon|lunch| +49957|AAAAAAAAGCDMAAAA|49957|13|52|37|PM|first|afternoon|lunch| +49958|AAAAAAAAHCDMAAAA|49958|13|52|38|PM|first|afternoon|lunch| +49959|AAAAAAAAICDMAAAA|49959|13|52|39|PM|first|afternoon|lunch| +49960|AAAAAAAAJCDMAAAA|49960|13|52|40|PM|first|afternoon|lunch| +49961|AAAAAAAAKCDMAAAA|49961|13|52|41|PM|first|afternoon|lunch| +49962|AAAAAAAALCDMAAAA|49962|13|52|42|PM|first|afternoon|lunch| +49963|AAAAAAAAMCDMAAAA|49963|13|52|43|PM|first|afternoon|lunch| +49964|AAAAAAAANCDMAAAA|49964|13|52|44|PM|first|afternoon|lunch| +49965|AAAAAAAAOCDMAAAA|49965|13|52|45|PM|first|afternoon|lunch| +49966|AAAAAAAAPCDMAAAA|49966|13|52|46|PM|first|afternoon|lunch| +49967|AAAAAAAAADDMAAAA|49967|13|52|47|PM|first|afternoon|lunch| +49968|AAAAAAAABDDMAAAA|49968|13|52|48|PM|first|afternoon|lunch| +49969|AAAAAAAACDDMAAAA|49969|13|52|49|PM|first|afternoon|lunch| +49970|AAAAAAAADDDMAAAA|49970|13|52|50|PM|first|afternoon|lunch| +49971|AAAAAAAAEDDMAAAA|49971|13|52|51|PM|first|afternoon|lunch| +49972|AAAAAAAAFDDMAAAA|49972|13|52|52|PM|first|afternoon|lunch| +49973|AAAAAAAAGDDMAAAA|49973|13|52|53|PM|first|afternoon|lunch| +49974|AAAAAAAAHDDMAAAA|49974|13|52|54|PM|first|afternoon|lunch| +49975|AAAAAAAAIDDMAAAA|49975|13|52|55|PM|first|afternoon|lunch| +49976|AAAAAAAAJDDMAAAA|49976|13|52|56|PM|first|afternoon|lunch| +49977|AAAAAAAAKDDMAAAA|49977|13|52|57|PM|first|afternoon|lunch| +49978|AAAAAAAALDDMAAAA|49978|13|52|58|PM|first|afternoon|lunch| +49979|AAAAAAAAMDDMAAAA|49979|13|52|59|PM|first|afternoon|lunch| +49980|AAAAAAAANDDMAAAA|49980|13|53|0|PM|first|afternoon|lunch| +49981|AAAAAAAAODDMAAAA|49981|13|53|1|PM|first|afternoon|lunch| +49982|AAAAAAAAPDDMAAAA|49982|13|53|2|PM|first|afternoon|lunch| +49983|AAAAAAAAAEDMAAAA|49983|13|53|3|PM|first|afternoon|lunch| +49984|AAAAAAAABEDMAAAA|49984|13|53|4|PM|first|afternoon|lunch| +49985|AAAAAAAACEDMAAAA|49985|13|53|5|PM|first|afternoon|lunch| +49986|AAAAAAAADEDMAAAA|49986|13|53|6|PM|first|afternoon|lunch| +49987|AAAAAAAAEEDMAAAA|49987|13|53|7|PM|first|afternoon|lunch| +49988|AAAAAAAAFEDMAAAA|49988|13|53|8|PM|first|afternoon|lunch| +49989|AAAAAAAAGEDMAAAA|49989|13|53|9|PM|first|afternoon|lunch| +49990|AAAAAAAAHEDMAAAA|49990|13|53|10|PM|first|afternoon|lunch| +49991|AAAAAAAAIEDMAAAA|49991|13|53|11|PM|first|afternoon|lunch| +49992|AAAAAAAAJEDMAAAA|49992|13|53|12|PM|first|afternoon|lunch| +49993|AAAAAAAAKEDMAAAA|49993|13|53|13|PM|first|afternoon|lunch| +49994|AAAAAAAALEDMAAAA|49994|13|53|14|PM|first|afternoon|lunch| +49995|AAAAAAAAMEDMAAAA|49995|13|53|15|PM|first|afternoon|lunch| +49996|AAAAAAAANEDMAAAA|49996|13|53|16|PM|first|afternoon|lunch| +49997|AAAAAAAAOEDMAAAA|49997|13|53|17|PM|first|afternoon|lunch| +49998|AAAAAAAAPEDMAAAA|49998|13|53|18|PM|first|afternoon|lunch| +49999|AAAAAAAAAFDMAAAA|49999|13|53|19|PM|first|afternoon|lunch| +50000|AAAAAAAABFDMAAAA|50000|13|53|20|PM|first|afternoon|lunch| +50001|AAAAAAAACFDMAAAA|50001|13|53|21|PM|first|afternoon|lunch| +50002|AAAAAAAADFDMAAAA|50002|13|53|22|PM|first|afternoon|lunch| +50003|AAAAAAAAEFDMAAAA|50003|13|53|23|PM|first|afternoon|lunch| +50004|AAAAAAAAFFDMAAAA|50004|13|53|24|PM|first|afternoon|lunch| +50005|AAAAAAAAGFDMAAAA|50005|13|53|25|PM|first|afternoon|lunch| +50006|AAAAAAAAHFDMAAAA|50006|13|53|26|PM|first|afternoon|lunch| +50007|AAAAAAAAIFDMAAAA|50007|13|53|27|PM|first|afternoon|lunch| +50008|AAAAAAAAJFDMAAAA|50008|13|53|28|PM|first|afternoon|lunch| +50009|AAAAAAAAKFDMAAAA|50009|13|53|29|PM|first|afternoon|lunch| +50010|AAAAAAAALFDMAAAA|50010|13|53|30|PM|first|afternoon|lunch| +50011|AAAAAAAAMFDMAAAA|50011|13|53|31|PM|first|afternoon|lunch| +50012|AAAAAAAANFDMAAAA|50012|13|53|32|PM|first|afternoon|lunch| +50013|AAAAAAAAOFDMAAAA|50013|13|53|33|PM|first|afternoon|lunch| +50014|AAAAAAAAPFDMAAAA|50014|13|53|34|PM|first|afternoon|lunch| +50015|AAAAAAAAAGDMAAAA|50015|13|53|35|PM|first|afternoon|lunch| +50016|AAAAAAAABGDMAAAA|50016|13|53|36|PM|first|afternoon|lunch| +50017|AAAAAAAACGDMAAAA|50017|13|53|37|PM|first|afternoon|lunch| +50018|AAAAAAAADGDMAAAA|50018|13|53|38|PM|first|afternoon|lunch| +50019|AAAAAAAAEGDMAAAA|50019|13|53|39|PM|first|afternoon|lunch| +50020|AAAAAAAAFGDMAAAA|50020|13|53|40|PM|first|afternoon|lunch| +50021|AAAAAAAAGGDMAAAA|50021|13|53|41|PM|first|afternoon|lunch| +50022|AAAAAAAAHGDMAAAA|50022|13|53|42|PM|first|afternoon|lunch| +50023|AAAAAAAAIGDMAAAA|50023|13|53|43|PM|first|afternoon|lunch| +50024|AAAAAAAAJGDMAAAA|50024|13|53|44|PM|first|afternoon|lunch| +50025|AAAAAAAAKGDMAAAA|50025|13|53|45|PM|first|afternoon|lunch| +50026|AAAAAAAALGDMAAAA|50026|13|53|46|PM|first|afternoon|lunch| +50027|AAAAAAAAMGDMAAAA|50027|13|53|47|PM|first|afternoon|lunch| +50028|AAAAAAAANGDMAAAA|50028|13|53|48|PM|first|afternoon|lunch| +50029|AAAAAAAAOGDMAAAA|50029|13|53|49|PM|first|afternoon|lunch| +50030|AAAAAAAAPGDMAAAA|50030|13|53|50|PM|first|afternoon|lunch| +50031|AAAAAAAAAHDMAAAA|50031|13|53|51|PM|first|afternoon|lunch| +50032|AAAAAAAABHDMAAAA|50032|13|53|52|PM|first|afternoon|lunch| +50033|AAAAAAAACHDMAAAA|50033|13|53|53|PM|first|afternoon|lunch| +50034|AAAAAAAADHDMAAAA|50034|13|53|54|PM|first|afternoon|lunch| +50035|AAAAAAAAEHDMAAAA|50035|13|53|55|PM|first|afternoon|lunch| +50036|AAAAAAAAFHDMAAAA|50036|13|53|56|PM|first|afternoon|lunch| +50037|AAAAAAAAGHDMAAAA|50037|13|53|57|PM|first|afternoon|lunch| +50038|AAAAAAAAHHDMAAAA|50038|13|53|58|PM|first|afternoon|lunch| +50039|AAAAAAAAIHDMAAAA|50039|13|53|59|PM|first|afternoon|lunch| +50040|AAAAAAAAJHDMAAAA|50040|13|54|0|PM|first|afternoon|lunch| +50041|AAAAAAAAKHDMAAAA|50041|13|54|1|PM|first|afternoon|lunch| +50042|AAAAAAAALHDMAAAA|50042|13|54|2|PM|first|afternoon|lunch| +50043|AAAAAAAAMHDMAAAA|50043|13|54|3|PM|first|afternoon|lunch| +50044|AAAAAAAANHDMAAAA|50044|13|54|4|PM|first|afternoon|lunch| +50045|AAAAAAAAOHDMAAAA|50045|13|54|5|PM|first|afternoon|lunch| +50046|AAAAAAAAPHDMAAAA|50046|13|54|6|PM|first|afternoon|lunch| +50047|AAAAAAAAAIDMAAAA|50047|13|54|7|PM|first|afternoon|lunch| +50048|AAAAAAAABIDMAAAA|50048|13|54|8|PM|first|afternoon|lunch| +50049|AAAAAAAACIDMAAAA|50049|13|54|9|PM|first|afternoon|lunch| +50050|AAAAAAAADIDMAAAA|50050|13|54|10|PM|first|afternoon|lunch| +50051|AAAAAAAAEIDMAAAA|50051|13|54|11|PM|first|afternoon|lunch| +50052|AAAAAAAAFIDMAAAA|50052|13|54|12|PM|first|afternoon|lunch| +50053|AAAAAAAAGIDMAAAA|50053|13|54|13|PM|first|afternoon|lunch| +50054|AAAAAAAAHIDMAAAA|50054|13|54|14|PM|first|afternoon|lunch| +50055|AAAAAAAAIIDMAAAA|50055|13|54|15|PM|first|afternoon|lunch| +50056|AAAAAAAAJIDMAAAA|50056|13|54|16|PM|first|afternoon|lunch| +50057|AAAAAAAAKIDMAAAA|50057|13|54|17|PM|first|afternoon|lunch| +50058|AAAAAAAALIDMAAAA|50058|13|54|18|PM|first|afternoon|lunch| +50059|AAAAAAAAMIDMAAAA|50059|13|54|19|PM|first|afternoon|lunch| +50060|AAAAAAAANIDMAAAA|50060|13|54|20|PM|first|afternoon|lunch| +50061|AAAAAAAAOIDMAAAA|50061|13|54|21|PM|first|afternoon|lunch| +50062|AAAAAAAAPIDMAAAA|50062|13|54|22|PM|first|afternoon|lunch| +50063|AAAAAAAAAJDMAAAA|50063|13|54|23|PM|first|afternoon|lunch| +50064|AAAAAAAABJDMAAAA|50064|13|54|24|PM|first|afternoon|lunch| +50065|AAAAAAAACJDMAAAA|50065|13|54|25|PM|first|afternoon|lunch| +50066|AAAAAAAADJDMAAAA|50066|13|54|26|PM|first|afternoon|lunch| +50067|AAAAAAAAEJDMAAAA|50067|13|54|27|PM|first|afternoon|lunch| +50068|AAAAAAAAFJDMAAAA|50068|13|54|28|PM|first|afternoon|lunch| +50069|AAAAAAAAGJDMAAAA|50069|13|54|29|PM|first|afternoon|lunch| +50070|AAAAAAAAHJDMAAAA|50070|13|54|30|PM|first|afternoon|lunch| +50071|AAAAAAAAIJDMAAAA|50071|13|54|31|PM|first|afternoon|lunch| +50072|AAAAAAAAJJDMAAAA|50072|13|54|32|PM|first|afternoon|lunch| +50073|AAAAAAAAKJDMAAAA|50073|13|54|33|PM|first|afternoon|lunch| +50074|AAAAAAAALJDMAAAA|50074|13|54|34|PM|first|afternoon|lunch| +50075|AAAAAAAAMJDMAAAA|50075|13|54|35|PM|first|afternoon|lunch| +50076|AAAAAAAANJDMAAAA|50076|13|54|36|PM|first|afternoon|lunch| +50077|AAAAAAAAOJDMAAAA|50077|13|54|37|PM|first|afternoon|lunch| +50078|AAAAAAAAPJDMAAAA|50078|13|54|38|PM|first|afternoon|lunch| +50079|AAAAAAAAAKDMAAAA|50079|13|54|39|PM|first|afternoon|lunch| +50080|AAAAAAAABKDMAAAA|50080|13|54|40|PM|first|afternoon|lunch| +50081|AAAAAAAACKDMAAAA|50081|13|54|41|PM|first|afternoon|lunch| +50082|AAAAAAAADKDMAAAA|50082|13|54|42|PM|first|afternoon|lunch| +50083|AAAAAAAAEKDMAAAA|50083|13|54|43|PM|first|afternoon|lunch| +50084|AAAAAAAAFKDMAAAA|50084|13|54|44|PM|first|afternoon|lunch| +50085|AAAAAAAAGKDMAAAA|50085|13|54|45|PM|first|afternoon|lunch| +50086|AAAAAAAAHKDMAAAA|50086|13|54|46|PM|first|afternoon|lunch| +50087|AAAAAAAAIKDMAAAA|50087|13|54|47|PM|first|afternoon|lunch| +50088|AAAAAAAAJKDMAAAA|50088|13|54|48|PM|first|afternoon|lunch| +50089|AAAAAAAAKKDMAAAA|50089|13|54|49|PM|first|afternoon|lunch| +50090|AAAAAAAALKDMAAAA|50090|13|54|50|PM|first|afternoon|lunch| +50091|AAAAAAAAMKDMAAAA|50091|13|54|51|PM|first|afternoon|lunch| +50092|AAAAAAAANKDMAAAA|50092|13|54|52|PM|first|afternoon|lunch| +50093|AAAAAAAAOKDMAAAA|50093|13|54|53|PM|first|afternoon|lunch| +50094|AAAAAAAAPKDMAAAA|50094|13|54|54|PM|first|afternoon|lunch| +50095|AAAAAAAAALDMAAAA|50095|13|54|55|PM|first|afternoon|lunch| +50096|AAAAAAAABLDMAAAA|50096|13|54|56|PM|first|afternoon|lunch| +50097|AAAAAAAACLDMAAAA|50097|13|54|57|PM|first|afternoon|lunch| +50098|AAAAAAAADLDMAAAA|50098|13|54|58|PM|first|afternoon|lunch| +50099|AAAAAAAAELDMAAAA|50099|13|54|59|PM|first|afternoon|lunch| +50100|AAAAAAAAFLDMAAAA|50100|13|55|0|PM|first|afternoon|lunch| +50101|AAAAAAAAGLDMAAAA|50101|13|55|1|PM|first|afternoon|lunch| +50102|AAAAAAAAHLDMAAAA|50102|13|55|2|PM|first|afternoon|lunch| +50103|AAAAAAAAILDMAAAA|50103|13|55|3|PM|first|afternoon|lunch| +50104|AAAAAAAAJLDMAAAA|50104|13|55|4|PM|first|afternoon|lunch| +50105|AAAAAAAAKLDMAAAA|50105|13|55|5|PM|first|afternoon|lunch| +50106|AAAAAAAALLDMAAAA|50106|13|55|6|PM|first|afternoon|lunch| +50107|AAAAAAAAMLDMAAAA|50107|13|55|7|PM|first|afternoon|lunch| +50108|AAAAAAAANLDMAAAA|50108|13|55|8|PM|first|afternoon|lunch| +50109|AAAAAAAAOLDMAAAA|50109|13|55|9|PM|first|afternoon|lunch| +50110|AAAAAAAAPLDMAAAA|50110|13|55|10|PM|first|afternoon|lunch| +50111|AAAAAAAAAMDMAAAA|50111|13|55|11|PM|first|afternoon|lunch| +50112|AAAAAAAABMDMAAAA|50112|13|55|12|PM|first|afternoon|lunch| +50113|AAAAAAAACMDMAAAA|50113|13|55|13|PM|first|afternoon|lunch| +50114|AAAAAAAADMDMAAAA|50114|13|55|14|PM|first|afternoon|lunch| +50115|AAAAAAAAEMDMAAAA|50115|13|55|15|PM|first|afternoon|lunch| +50116|AAAAAAAAFMDMAAAA|50116|13|55|16|PM|first|afternoon|lunch| +50117|AAAAAAAAGMDMAAAA|50117|13|55|17|PM|first|afternoon|lunch| +50118|AAAAAAAAHMDMAAAA|50118|13|55|18|PM|first|afternoon|lunch| +50119|AAAAAAAAIMDMAAAA|50119|13|55|19|PM|first|afternoon|lunch| +50120|AAAAAAAAJMDMAAAA|50120|13|55|20|PM|first|afternoon|lunch| +50121|AAAAAAAAKMDMAAAA|50121|13|55|21|PM|first|afternoon|lunch| +50122|AAAAAAAALMDMAAAA|50122|13|55|22|PM|first|afternoon|lunch| +50123|AAAAAAAAMMDMAAAA|50123|13|55|23|PM|first|afternoon|lunch| +50124|AAAAAAAANMDMAAAA|50124|13|55|24|PM|first|afternoon|lunch| +50125|AAAAAAAAOMDMAAAA|50125|13|55|25|PM|first|afternoon|lunch| +50126|AAAAAAAAPMDMAAAA|50126|13|55|26|PM|first|afternoon|lunch| +50127|AAAAAAAAANDMAAAA|50127|13|55|27|PM|first|afternoon|lunch| +50128|AAAAAAAABNDMAAAA|50128|13|55|28|PM|first|afternoon|lunch| +50129|AAAAAAAACNDMAAAA|50129|13|55|29|PM|first|afternoon|lunch| +50130|AAAAAAAADNDMAAAA|50130|13|55|30|PM|first|afternoon|lunch| +50131|AAAAAAAAENDMAAAA|50131|13|55|31|PM|first|afternoon|lunch| +50132|AAAAAAAAFNDMAAAA|50132|13|55|32|PM|first|afternoon|lunch| +50133|AAAAAAAAGNDMAAAA|50133|13|55|33|PM|first|afternoon|lunch| +50134|AAAAAAAAHNDMAAAA|50134|13|55|34|PM|first|afternoon|lunch| +50135|AAAAAAAAINDMAAAA|50135|13|55|35|PM|first|afternoon|lunch| +50136|AAAAAAAAJNDMAAAA|50136|13|55|36|PM|first|afternoon|lunch| +50137|AAAAAAAAKNDMAAAA|50137|13|55|37|PM|first|afternoon|lunch| +50138|AAAAAAAALNDMAAAA|50138|13|55|38|PM|first|afternoon|lunch| +50139|AAAAAAAAMNDMAAAA|50139|13|55|39|PM|first|afternoon|lunch| +50140|AAAAAAAANNDMAAAA|50140|13|55|40|PM|first|afternoon|lunch| +50141|AAAAAAAAONDMAAAA|50141|13|55|41|PM|first|afternoon|lunch| +50142|AAAAAAAAPNDMAAAA|50142|13|55|42|PM|first|afternoon|lunch| +50143|AAAAAAAAAODMAAAA|50143|13|55|43|PM|first|afternoon|lunch| +50144|AAAAAAAABODMAAAA|50144|13|55|44|PM|first|afternoon|lunch| +50145|AAAAAAAACODMAAAA|50145|13|55|45|PM|first|afternoon|lunch| +50146|AAAAAAAADODMAAAA|50146|13|55|46|PM|first|afternoon|lunch| +50147|AAAAAAAAEODMAAAA|50147|13|55|47|PM|first|afternoon|lunch| +50148|AAAAAAAAFODMAAAA|50148|13|55|48|PM|first|afternoon|lunch| +50149|AAAAAAAAGODMAAAA|50149|13|55|49|PM|first|afternoon|lunch| +50150|AAAAAAAAHODMAAAA|50150|13|55|50|PM|first|afternoon|lunch| +50151|AAAAAAAAIODMAAAA|50151|13|55|51|PM|first|afternoon|lunch| +50152|AAAAAAAAJODMAAAA|50152|13|55|52|PM|first|afternoon|lunch| +50153|AAAAAAAAKODMAAAA|50153|13|55|53|PM|first|afternoon|lunch| +50154|AAAAAAAALODMAAAA|50154|13|55|54|PM|first|afternoon|lunch| +50155|AAAAAAAAMODMAAAA|50155|13|55|55|PM|first|afternoon|lunch| +50156|AAAAAAAANODMAAAA|50156|13|55|56|PM|first|afternoon|lunch| +50157|AAAAAAAAOODMAAAA|50157|13|55|57|PM|first|afternoon|lunch| +50158|AAAAAAAAPODMAAAA|50158|13|55|58|PM|first|afternoon|lunch| +50159|AAAAAAAAAPDMAAAA|50159|13|55|59|PM|first|afternoon|lunch| +50160|AAAAAAAABPDMAAAA|50160|13|56|0|PM|first|afternoon|lunch| +50161|AAAAAAAACPDMAAAA|50161|13|56|1|PM|first|afternoon|lunch| +50162|AAAAAAAADPDMAAAA|50162|13|56|2|PM|first|afternoon|lunch| +50163|AAAAAAAAEPDMAAAA|50163|13|56|3|PM|first|afternoon|lunch| +50164|AAAAAAAAFPDMAAAA|50164|13|56|4|PM|first|afternoon|lunch| +50165|AAAAAAAAGPDMAAAA|50165|13|56|5|PM|first|afternoon|lunch| +50166|AAAAAAAAHPDMAAAA|50166|13|56|6|PM|first|afternoon|lunch| +50167|AAAAAAAAIPDMAAAA|50167|13|56|7|PM|first|afternoon|lunch| +50168|AAAAAAAAJPDMAAAA|50168|13|56|8|PM|first|afternoon|lunch| +50169|AAAAAAAAKPDMAAAA|50169|13|56|9|PM|first|afternoon|lunch| +50170|AAAAAAAALPDMAAAA|50170|13|56|10|PM|first|afternoon|lunch| +50171|AAAAAAAAMPDMAAAA|50171|13|56|11|PM|first|afternoon|lunch| +50172|AAAAAAAANPDMAAAA|50172|13|56|12|PM|first|afternoon|lunch| +50173|AAAAAAAAOPDMAAAA|50173|13|56|13|PM|first|afternoon|lunch| +50174|AAAAAAAAPPDMAAAA|50174|13|56|14|PM|first|afternoon|lunch| +50175|AAAAAAAAAAEMAAAA|50175|13|56|15|PM|first|afternoon|lunch| +50176|AAAAAAAABAEMAAAA|50176|13|56|16|PM|first|afternoon|lunch| +50177|AAAAAAAACAEMAAAA|50177|13|56|17|PM|first|afternoon|lunch| +50178|AAAAAAAADAEMAAAA|50178|13|56|18|PM|first|afternoon|lunch| +50179|AAAAAAAAEAEMAAAA|50179|13|56|19|PM|first|afternoon|lunch| +50180|AAAAAAAAFAEMAAAA|50180|13|56|20|PM|first|afternoon|lunch| +50181|AAAAAAAAGAEMAAAA|50181|13|56|21|PM|first|afternoon|lunch| +50182|AAAAAAAAHAEMAAAA|50182|13|56|22|PM|first|afternoon|lunch| +50183|AAAAAAAAIAEMAAAA|50183|13|56|23|PM|first|afternoon|lunch| +50184|AAAAAAAAJAEMAAAA|50184|13|56|24|PM|first|afternoon|lunch| +50185|AAAAAAAAKAEMAAAA|50185|13|56|25|PM|first|afternoon|lunch| +50186|AAAAAAAALAEMAAAA|50186|13|56|26|PM|first|afternoon|lunch| +50187|AAAAAAAAMAEMAAAA|50187|13|56|27|PM|first|afternoon|lunch| +50188|AAAAAAAANAEMAAAA|50188|13|56|28|PM|first|afternoon|lunch| +50189|AAAAAAAAOAEMAAAA|50189|13|56|29|PM|first|afternoon|lunch| +50190|AAAAAAAAPAEMAAAA|50190|13|56|30|PM|first|afternoon|lunch| +50191|AAAAAAAAABEMAAAA|50191|13|56|31|PM|first|afternoon|lunch| +50192|AAAAAAAABBEMAAAA|50192|13|56|32|PM|first|afternoon|lunch| +50193|AAAAAAAACBEMAAAA|50193|13|56|33|PM|first|afternoon|lunch| +50194|AAAAAAAADBEMAAAA|50194|13|56|34|PM|first|afternoon|lunch| +50195|AAAAAAAAEBEMAAAA|50195|13|56|35|PM|first|afternoon|lunch| +50196|AAAAAAAAFBEMAAAA|50196|13|56|36|PM|first|afternoon|lunch| +50197|AAAAAAAAGBEMAAAA|50197|13|56|37|PM|first|afternoon|lunch| +50198|AAAAAAAAHBEMAAAA|50198|13|56|38|PM|first|afternoon|lunch| +50199|AAAAAAAAIBEMAAAA|50199|13|56|39|PM|first|afternoon|lunch| +50200|AAAAAAAAJBEMAAAA|50200|13|56|40|PM|first|afternoon|lunch| +50201|AAAAAAAAKBEMAAAA|50201|13|56|41|PM|first|afternoon|lunch| +50202|AAAAAAAALBEMAAAA|50202|13|56|42|PM|first|afternoon|lunch| +50203|AAAAAAAAMBEMAAAA|50203|13|56|43|PM|first|afternoon|lunch| +50204|AAAAAAAANBEMAAAA|50204|13|56|44|PM|first|afternoon|lunch| +50205|AAAAAAAAOBEMAAAA|50205|13|56|45|PM|first|afternoon|lunch| +50206|AAAAAAAAPBEMAAAA|50206|13|56|46|PM|first|afternoon|lunch| +50207|AAAAAAAAACEMAAAA|50207|13|56|47|PM|first|afternoon|lunch| +50208|AAAAAAAABCEMAAAA|50208|13|56|48|PM|first|afternoon|lunch| +50209|AAAAAAAACCEMAAAA|50209|13|56|49|PM|first|afternoon|lunch| +50210|AAAAAAAADCEMAAAA|50210|13|56|50|PM|first|afternoon|lunch| +50211|AAAAAAAAECEMAAAA|50211|13|56|51|PM|first|afternoon|lunch| +50212|AAAAAAAAFCEMAAAA|50212|13|56|52|PM|first|afternoon|lunch| +50213|AAAAAAAAGCEMAAAA|50213|13|56|53|PM|first|afternoon|lunch| +50214|AAAAAAAAHCEMAAAA|50214|13|56|54|PM|first|afternoon|lunch| +50215|AAAAAAAAICEMAAAA|50215|13|56|55|PM|first|afternoon|lunch| +50216|AAAAAAAAJCEMAAAA|50216|13|56|56|PM|first|afternoon|lunch| +50217|AAAAAAAAKCEMAAAA|50217|13|56|57|PM|first|afternoon|lunch| +50218|AAAAAAAALCEMAAAA|50218|13|56|58|PM|first|afternoon|lunch| +50219|AAAAAAAAMCEMAAAA|50219|13|56|59|PM|first|afternoon|lunch| +50220|AAAAAAAANCEMAAAA|50220|13|57|0|PM|first|afternoon|lunch| +50221|AAAAAAAAOCEMAAAA|50221|13|57|1|PM|first|afternoon|lunch| +50222|AAAAAAAAPCEMAAAA|50222|13|57|2|PM|first|afternoon|lunch| +50223|AAAAAAAAADEMAAAA|50223|13|57|3|PM|first|afternoon|lunch| +50224|AAAAAAAABDEMAAAA|50224|13|57|4|PM|first|afternoon|lunch| +50225|AAAAAAAACDEMAAAA|50225|13|57|5|PM|first|afternoon|lunch| +50226|AAAAAAAADDEMAAAA|50226|13|57|6|PM|first|afternoon|lunch| +50227|AAAAAAAAEDEMAAAA|50227|13|57|7|PM|first|afternoon|lunch| +50228|AAAAAAAAFDEMAAAA|50228|13|57|8|PM|first|afternoon|lunch| +50229|AAAAAAAAGDEMAAAA|50229|13|57|9|PM|first|afternoon|lunch| +50230|AAAAAAAAHDEMAAAA|50230|13|57|10|PM|first|afternoon|lunch| +50231|AAAAAAAAIDEMAAAA|50231|13|57|11|PM|first|afternoon|lunch| +50232|AAAAAAAAJDEMAAAA|50232|13|57|12|PM|first|afternoon|lunch| +50233|AAAAAAAAKDEMAAAA|50233|13|57|13|PM|first|afternoon|lunch| +50234|AAAAAAAALDEMAAAA|50234|13|57|14|PM|first|afternoon|lunch| +50235|AAAAAAAAMDEMAAAA|50235|13|57|15|PM|first|afternoon|lunch| +50236|AAAAAAAANDEMAAAA|50236|13|57|16|PM|first|afternoon|lunch| +50237|AAAAAAAAODEMAAAA|50237|13|57|17|PM|first|afternoon|lunch| +50238|AAAAAAAAPDEMAAAA|50238|13|57|18|PM|first|afternoon|lunch| +50239|AAAAAAAAAEEMAAAA|50239|13|57|19|PM|first|afternoon|lunch| +50240|AAAAAAAABEEMAAAA|50240|13|57|20|PM|first|afternoon|lunch| +50241|AAAAAAAACEEMAAAA|50241|13|57|21|PM|first|afternoon|lunch| +50242|AAAAAAAADEEMAAAA|50242|13|57|22|PM|first|afternoon|lunch| +50243|AAAAAAAAEEEMAAAA|50243|13|57|23|PM|first|afternoon|lunch| +50244|AAAAAAAAFEEMAAAA|50244|13|57|24|PM|first|afternoon|lunch| +50245|AAAAAAAAGEEMAAAA|50245|13|57|25|PM|first|afternoon|lunch| +50246|AAAAAAAAHEEMAAAA|50246|13|57|26|PM|first|afternoon|lunch| +50247|AAAAAAAAIEEMAAAA|50247|13|57|27|PM|first|afternoon|lunch| +50248|AAAAAAAAJEEMAAAA|50248|13|57|28|PM|first|afternoon|lunch| +50249|AAAAAAAAKEEMAAAA|50249|13|57|29|PM|first|afternoon|lunch| +50250|AAAAAAAALEEMAAAA|50250|13|57|30|PM|first|afternoon|lunch| +50251|AAAAAAAAMEEMAAAA|50251|13|57|31|PM|first|afternoon|lunch| +50252|AAAAAAAANEEMAAAA|50252|13|57|32|PM|first|afternoon|lunch| +50253|AAAAAAAAOEEMAAAA|50253|13|57|33|PM|first|afternoon|lunch| +50254|AAAAAAAAPEEMAAAA|50254|13|57|34|PM|first|afternoon|lunch| +50255|AAAAAAAAAFEMAAAA|50255|13|57|35|PM|first|afternoon|lunch| +50256|AAAAAAAABFEMAAAA|50256|13|57|36|PM|first|afternoon|lunch| +50257|AAAAAAAACFEMAAAA|50257|13|57|37|PM|first|afternoon|lunch| +50258|AAAAAAAADFEMAAAA|50258|13|57|38|PM|first|afternoon|lunch| +50259|AAAAAAAAEFEMAAAA|50259|13|57|39|PM|first|afternoon|lunch| +50260|AAAAAAAAFFEMAAAA|50260|13|57|40|PM|first|afternoon|lunch| +50261|AAAAAAAAGFEMAAAA|50261|13|57|41|PM|first|afternoon|lunch| +50262|AAAAAAAAHFEMAAAA|50262|13|57|42|PM|first|afternoon|lunch| +50263|AAAAAAAAIFEMAAAA|50263|13|57|43|PM|first|afternoon|lunch| +50264|AAAAAAAAJFEMAAAA|50264|13|57|44|PM|first|afternoon|lunch| +50265|AAAAAAAAKFEMAAAA|50265|13|57|45|PM|first|afternoon|lunch| +50266|AAAAAAAALFEMAAAA|50266|13|57|46|PM|first|afternoon|lunch| +50267|AAAAAAAAMFEMAAAA|50267|13|57|47|PM|first|afternoon|lunch| +50268|AAAAAAAANFEMAAAA|50268|13|57|48|PM|first|afternoon|lunch| +50269|AAAAAAAAOFEMAAAA|50269|13|57|49|PM|first|afternoon|lunch| +50270|AAAAAAAAPFEMAAAA|50270|13|57|50|PM|first|afternoon|lunch| +50271|AAAAAAAAAGEMAAAA|50271|13|57|51|PM|first|afternoon|lunch| +50272|AAAAAAAABGEMAAAA|50272|13|57|52|PM|first|afternoon|lunch| +50273|AAAAAAAACGEMAAAA|50273|13|57|53|PM|first|afternoon|lunch| +50274|AAAAAAAADGEMAAAA|50274|13|57|54|PM|first|afternoon|lunch| +50275|AAAAAAAAEGEMAAAA|50275|13|57|55|PM|first|afternoon|lunch| +50276|AAAAAAAAFGEMAAAA|50276|13|57|56|PM|first|afternoon|lunch| +50277|AAAAAAAAGGEMAAAA|50277|13|57|57|PM|first|afternoon|lunch| +50278|AAAAAAAAHGEMAAAA|50278|13|57|58|PM|first|afternoon|lunch| +50279|AAAAAAAAIGEMAAAA|50279|13|57|59|PM|first|afternoon|lunch| +50280|AAAAAAAAJGEMAAAA|50280|13|58|0|PM|first|afternoon|lunch| +50281|AAAAAAAAKGEMAAAA|50281|13|58|1|PM|first|afternoon|lunch| +50282|AAAAAAAALGEMAAAA|50282|13|58|2|PM|first|afternoon|lunch| +50283|AAAAAAAAMGEMAAAA|50283|13|58|3|PM|first|afternoon|lunch| +50284|AAAAAAAANGEMAAAA|50284|13|58|4|PM|first|afternoon|lunch| +50285|AAAAAAAAOGEMAAAA|50285|13|58|5|PM|first|afternoon|lunch| +50286|AAAAAAAAPGEMAAAA|50286|13|58|6|PM|first|afternoon|lunch| +50287|AAAAAAAAAHEMAAAA|50287|13|58|7|PM|first|afternoon|lunch| +50288|AAAAAAAABHEMAAAA|50288|13|58|8|PM|first|afternoon|lunch| +50289|AAAAAAAACHEMAAAA|50289|13|58|9|PM|first|afternoon|lunch| +50290|AAAAAAAADHEMAAAA|50290|13|58|10|PM|first|afternoon|lunch| +50291|AAAAAAAAEHEMAAAA|50291|13|58|11|PM|first|afternoon|lunch| +50292|AAAAAAAAFHEMAAAA|50292|13|58|12|PM|first|afternoon|lunch| +50293|AAAAAAAAGHEMAAAA|50293|13|58|13|PM|first|afternoon|lunch| +50294|AAAAAAAAHHEMAAAA|50294|13|58|14|PM|first|afternoon|lunch| +50295|AAAAAAAAIHEMAAAA|50295|13|58|15|PM|first|afternoon|lunch| +50296|AAAAAAAAJHEMAAAA|50296|13|58|16|PM|first|afternoon|lunch| +50297|AAAAAAAAKHEMAAAA|50297|13|58|17|PM|first|afternoon|lunch| +50298|AAAAAAAALHEMAAAA|50298|13|58|18|PM|first|afternoon|lunch| +50299|AAAAAAAAMHEMAAAA|50299|13|58|19|PM|first|afternoon|lunch| +50300|AAAAAAAANHEMAAAA|50300|13|58|20|PM|first|afternoon|lunch| +50301|AAAAAAAAOHEMAAAA|50301|13|58|21|PM|first|afternoon|lunch| +50302|AAAAAAAAPHEMAAAA|50302|13|58|22|PM|first|afternoon|lunch| +50303|AAAAAAAAAIEMAAAA|50303|13|58|23|PM|first|afternoon|lunch| +50304|AAAAAAAABIEMAAAA|50304|13|58|24|PM|first|afternoon|lunch| +50305|AAAAAAAACIEMAAAA|50305|13|58|25|PM|first|afternoon|lunch| +50306|AAAAAAAADIEMAAAA|50306|13|58|26|PM|first|afternoon|lunch| +50307|AAAAAAAAEIEMAAAA|50307|13|58|27|PM|first|afternoon|lunch| +50308|AAAAAAAAFIEMAAAA|50308|13|58|28|PM|first|afternoon|lunch| +50309|AAAAAAAAGIEMAAAA|50309|13|58|29|PM|first|afternoon|lunch| +50310|AAAAAAAAHIEMAAAA|50310|13|58|30|PM|first|afternoon|lunch| +50311|AAAAAAAAIIEMAAAA|50311|13|58|31|PM|first|afternoon|lunch| +50312|AAAAAAAAJIEMAAAA|50312|13|58|32|PM|first|afternoon|lunch| +50313|AAAAAAAAKIEMAAAA|50313|13|58|33|PM|first|afternoon|lunch| +50314|AAAAAAAALIEMAAAA|50314|13|58|34|PM|first|afternoon|lunch| +50315|AAAAAAAAMIEMAAAA|50315|13|58|35|PM|first|afternoon|lunch| +50316|AAAAAAAANIEMAAAA|50316|13|58|36|PM|first|afternoon|lunch| +50317|AAAAAAAAOIEMAAAA|50317|13|58|37|PM|first|afternoon|lunch| +50318|AAAAAAAAPIEMAAAA|50318|13|58|38|PM|first|afternoon|lunch| +50319|AAAAAAAAAJEMAAAA|50319|13|58|39|PM|first|afternoon|lunch| +50320|AAAAAAAABJEMAAAA|50320|13|58|40|PM|first|afternoon|lunch| +50321|AAAAAAAACJEMAAAA|50321|13|58|41|PM|first|afternoon|lunch| +50322|AAAAAAAADJEMAAAA|50322|13|58|42|PM|first|afternoon|lunch| +50323|AAAAAAAAEJEMAAAA|50323|13|58|43|PM|first|afternoon|lunch| +50324|AAAAAAAAFJEMAAAA|50324|13|58|44|PM|first|afternoon|lunch| +50325|AAAAAAAAGJEMAAAA|50325|13|58|45|PM|first|afternoon|lunch| +50326|AAAAAAAAHJEMAAAA|50326|13|58|46|PM|first|afternoon|lunch| +50327|AAAAAAAAIJEMAAAA|50327|13|58|47|PM|first|afternoon|lunch| +50328|AAAAAAAAJJEMAAAA|50328|13|58|48|PM|first|afternoon|lunch| +50329|AAAAAAAAKJEMAAAA|50329|13|58|49|PM|first|afternoon|lunch| +50330|AAAAAAAALJEMAAAA|50330|13|58|50|PM|first|afternoon|lunch| +50331|AAAAAAAAMJEMAAAA|50331|13|58|51|PM|first|afternoon|lunch| +50332|AAAAAAAANJEMAAAA|50332|13|58|52|PM|first|afternoon|lunch| +50333|AAAAAAAAOJEMAAAA|50333|13|58|53|PM|first|afternoon|lunch| +50334|AAAAAAAAPJEMAAAA|50334|13|58|54|PM|first|afternoon|lunch| +50335|AAAAAAAAAKEMAAAA|50335|13|58|55|PM|first|afternoon|lunch| +50336|AAAAAAAABKEMAAAA|50336|13|58|56|PM|first|afternoon|lunch| +50337|AAAAAAAACKEMAAAA|50337|13|58|57|PM|first|afternoon|lunch| +50338|AAAAAAAADKEMAAAA|50338|13|58|58|PM|first|afternoon|lunch| +50339|AAAAAAAAEKEMAAAA|50339|13|58|59|PM|first|afternoon|lunch| +50340|AAAAAAAAFKEMAAAA|50340|13|59|0|PM|first|afternoon|lunch| +50341|AAAAAAAAGKEMAAAA|50341|13|59|1|PM|first|afternoon|lunch| +50342|AAAAAAAAHKEMAAAA|50342|13|59|2|PM|first|afternoon|lunch| +50343|AAAAAAAAIKEMAAAA|50343|13|59|3|PM|first|afternoon|lunch| +50344|AAAAAAAAJKEMAAAA|50344|13|59|4|PM|first|afternoon|lunch| +50345|AAAAAAAAKKEMAAAA|50345|13|59|5|PM|first|afternoon|lunch| +50346|AAAAAAAALKEMAAAA|50346|13|59|6|PM|first|afternoon|lunch| +50347|AAAAAAAAMKEMAAAA|50347|13|59|7|PM|first|afternoon|lunch| +50348|AAAAAAAANKEMAAAA|50348|13|59|8|PM|first|afternoon|lunch| +50349|AAAAAAAAOKEMAAAA|50349|13|59|9|PM|first|afternoon|lunch| +50350|AAAAAAAAPKEMAAAA|50350|13|59|10|PM|first|afternoon|lunch| +50351|AAAAAAAAALEMAAAA|50351|13|59|11|PM|first|afternoon|lunch| +50352|AAAAAAAABLEMAAAA|50352|13|59|12|PM|first|afternoon|lunch| +50353|AAAAAAAACLEMAAAA|50353|13|59|13|PM|first|afternoon|lunch| +50354|AAAAAAAADLEMAAAA|50354|13|59|14|PM|first|afternoon|lunch| +50355|AAAAAAAAELEMAAAA|50355|13|59|15|PM|first|afternoon|lunch| +50356|AAAAAAAAFLEMAAAA|50356|13|59|16|PM|first|afternoon|lunch| +50357|AAAAAAAAGLEMAAAA|50357|13|59|17|PM|first|afternoon|lunch| +50358|AAAAAAAAHLEMAAAA|50358|13|59|18|PM|first|afternoon|lunch| +50359|AAAAAAAAILEMAAAA|50359|13|59|19|PM|first|afternoon|lunch| +50360|AAAAAAAAJLEMAAAA|50360|13|59|20|PM|first|afternoon|lunch| +50361|AAAAAAAAKLEMAAAA|50361|13|59|21|PM|first|afternoon|lunch| +50362|AAAAAAAALLEMAAAA|50362|13|59|22|PM|first|afternoon|lunch| +50363|AAAAAAAAMLEMAAAA|50363|13|59|23|PM|first|afternoon|lunch| +50364|AAAAAAAANLEMAAAA|50364|13|59|24|PM|first|afternoon|lunch| +50365|AAAAAAAAOLEMAAAA|50365|13|59|25|PM|first|afternoon|lunch| +50366|AAAAAAAAPLEMAAAA|50366|13|59|26|PM|first|afternoon|lunch| +50367|AAAAAAAAAMEMAAAA|50367|13|59|27|PM|first|afternoon|lunch| +50368|AAAAAAAABMEMAAAA|50368|13|59|28|PM|first|afternoon|lunch| +50369|AAAAAAAACMEMAAAA|50369|13|59|29|PM|first|afternoon|lunch| +50370|AAAAAAAADMEMAAAA|50370|13|59|30|PM|first|afternoon|lunch| +50371|AAAAAAAAEMEMAAAA|50371|13|59|31|PM|first|afternoon|lunch| +50372|AAAAAAAAFMEMAAAA|50372|13|59|32|PM|first|afternoon|lunch| +50373|AAAAAAAAGMEMAAAA|50373|13|59|33|PM|first|afternoon|lunch| +50374|AAAAAAAAHMEMAAAA|50374|13|59|34|PM|first|afternoon|lunch| +50375|AAAAAAAAIMEMAAAA|50375|13|59|35|PM|first|afternoon|lunch| +50376|AAAAAAAAJMEMAAAA|50376|13|59|36|PM|first|afternoon|lunch| +50377|AAAAAAAAKMEMAAAA|50377|13|59|37|PM|first|afternoon|lunch| +50378|AAAAAAAALMEMAAAA|50378|13|59|38|PM|first|afternoon|lunch| +50379|AAAAAAAAMMEMAAAA|50379|13|59|39|PM|first|afternoon|lunch| +50380|AAAAAAAANMEMAAAA|50380|13|59|40|PM|first|afternoon|lunch| +50381|AAAAAAAAOMEMAAAA|50381|13|59|41|PM|first|afternoon|lunch| +50382|AAAAAAAAPMEMAAAA|50382|13|59|42|PM|first|afternoon|lunch| +50383|AAAAAAAAANEMAAAA|50383|13|59|43|PM|first|afternoon|lunch| +50384|AAAAAAAABNEMAAAA|50384|13|59|44|PM|first|afternoon|lunch| +50385|AAAAAAAACNEMAAAA|50385|13|59|45|PM|first|afternoon|lunch| +50386|AAAAAAAADNEMAAAA|50386|13|59|46|PM|first|afternoon|lunch| +50387|AAAAAAAAENEMAAAA|50387|13|59|47|PM|first|afternoon|lunch| +50388|AAAAAAAAFNEMAAAA|50388|13|59|48|PM|first|afternoon|lunch| +50389|AAAAAAAAGNEMAAAA|50389|13|59|49|PM|first|afternoon|lunch| +50390|AAAAAAAAHNEMAAAA|50390|13|59|50|PM|first|afternoon|lunch| +50391|AAAAAAAAINEMAAAA|50391|13|59|51|PM|first|afternoon|lunch| +50392|AAAAAAAAJNEMAAAA|50392|13|59|52|PM|first|afternoon|lunch| +50393|AAAAAAAAKNEMAAAA|50393|13|59|53|PM|first|afternoon|lunch| +50394|AAAAAAAALNEMAAAA|50394|13|59|54|PM|first|afternoon|lunch| +50395|AAAAAAAAMNEMAAAA|50395|13|59|55|PM|first|afternoon|lunch| +50396|AAAAAAAANNEMAAAA|50396|13|59|56|PM|first|afternoon|lunch| +50397|AAAAAAAAONEMAAAA|50397|13|59|57|PM|first|afternoon|lunch| +50398|AAAAAAAAPNEMAAAA|50398|13|59|58|PM|first|afternoon|lunch| +50399|AAAAAAAAAOEMAAAA|50399|13|59|59|PM|first|afternoon|lunch| +50400|AAAAAAAABOEMAAAA|50400|14|0|0|PM|first|afternoon|lunch| +50401|AAAAAAAACOEMAAAA|50401|14|0|1|PM|first|afternoon|lunch| +50402|AAAAAAAADOEMAAAA|50402|14|0|2|PM|first|afternoon|lunch| +50403|AAAAAAAAEOEMAAAA|50403|14|0|3|PM|first|afternoon|lunch| +50404|AAAAAAAAFOEMAAAA|50404|14|0|4|PM|first|afternoon|lunch| +50405|AAAAAAAAGOEMAAAA|50405|14|0|5|PM|first|afternoon|lunch| +50406|AAAAAAAAHOEMAAAA|50406|14|0|6|PM|first|afternoon|lunch| +50407|AAAAAAAAIOEMAAAA|50407|14|0|7|PM|first|afternoon|lunch| +50408|AAAAAAAAJOEMAAAA|50408|14|0|8|PM|first|afternoon|lunch| +50409|AAAAAAAAKOEMAAAA|50409|14|0|9|PM|first|afternoon|lunch| +50410|AAAAAAAALOEMAAAA|50410|14|0|10|PM|first|afternoon|lunch| +50411|AAAAAAAAMOEMAAAA|50411|14|0|11|PM|first|afternoon|lunch| +50412|AAAAAAAANOEMAAAA|50412|14|0|12|PM|first|afternoon|lunch| +50413|AAAAAAAAOOEMAAAA|50413|14|0|13|PM|first|afternoon|lunch| +50414|AAAAAAAAPOEMAAAA|50414|14|0|14|PM|first|afternoon|lunch| +50415|AAAAAAAAAPEMAAAA|50415|14|0|15|PM|first|afternoon|lunch| +50416|AAAAAAAABPEMAAAA|50416|14|0|16|PM|first|afternoon|lunch| +50417|AAAAAAAACPEMAAAA|50417|14|0|17|PM|first|afternoon|lunch| +50418|AAAAAAAADPEMAAAA|50418|14|0|18|PM|first|afternoon|lunch| +50419|AAAAAAAAEPEMAAAA|50419|14|0|19|PM|first|afternoon|lunch| +50420|AAAAAAAAFPEMAAAA|50420|14|0|20|PM|first|afternoon|lunch| +50421|AAAAAAAAGPEMAAAA|50421|14|0|21|PM|first|afternoon|lunch| +50422|AAAAAAAAHPEMAAAA|50422|14|0|22|PM|first|afternoon|lunch| +50423|AAAAAAAAIPEMAAAA|50423|14|0|23|PM|first|afternoon|lunch| +50424|AAAAAAAAJPEMAAAA|50424|14|0|24|PM|first|afternoon|lunch| +50425|AAAAAAAAKPEMAAAA|50425|14|0|25|PM|first|afternoon|lunch| +50426|AAAAAAAALPEMAAAA|50426|14|0|26|PM|first|afternoon|lunch| +50427|AAAAAAAAMPEMAAAA|50427|14|0|27|PM|first|afternoon|lunch| +50428|AAAAAAAANPEMAAAA|50428|14|0|28|PM|first|afternoon|lunch| +50429|AAAAAAAAOPEMAAAA|50429|14|0|29|PM|first|afternoon|lunch| +50430|AAAAAAAAPPEMAAAA|50430|14|0|30|PM|first|afternoon|lunch| +50431|AAAAAAAAAAFMAAAA|50431|14|0|31|PM|first|afternoon|lunch| +50432|AAAAAAAABAFMAAAA|50432|14|0|32|PM|first|afternoon|lunch| +50433|AAAAAAAACAFMAAAA|50433|14|0|33|PM|first|afternoon|lunch| +50434|AAAAAAAADAFMAAAA|50434|14|0|34|PM|first|afternoon|lunch| +50435|AAAAAAAAEAFMAAAA|50435|14|0|35|PM|first|afternoon|lunch| +50436|AAAAAAAAFAFMAAAA|50436|14|0|36|PM|first|afternoon|lunch| +50437|AAAAAAAAGAFMAAAA|50437|14|0|37|PM|first|afternoon|lunch| +50438|AAAAAAAAHAFMAAAA|50438|14|0|38|PM|first|afternoon|lunch| +50439|AAAAAAAAIAFMAAAA|50439|14|0|39|PM|first|afternoon|lunch| +50440|AAAAAAAAJAFMAAAA|50440|14|0|40|PM|first|afternoon|lunch| +50441|AAAAAAAAKAFMAAAA|50441|14|0|41|PM|first|afternoon|lunch| +50442|AAAAAAAALAFMAAAA|50442|14|0|42|PM|first|afternoon|lunch| +50443|AAAAAAAAMAFMAAAA|50443|14|0|43|PM|first|afternoon|lunch| +50444|AAAAAAAANAFMAAAA|50444|14|0|44|PM|first|afternoon|lunch| +50445|AAAAAAAAOAFMAAAA|50445|14|0|45|PM|first|afternoon|lunch| +50446|AAAAAAAAPAFMAAAA|50446|14|0|46|PM|first|afternoon|lunch| +50447|AAAAAAAAABFMAAAA|50447|14|0|47|PM|first|afternoon|lunch| +50448|AAAAAAAABBFMAAAA|50448|14|0|48|PM|first|afternoon|lunch| +50449|AAAAAAAACBFMAAAA|50449|14|0|49|PM|first|afternoon|lunch| +50450|AAAAAAAADBFMAAAA|50450|14|0|50|PM|first|afternoon|lunch| +50451|AAAAAAAAEBFMAAAA|50451|14|0|51|PM|first|afternoon|lunch| +50452|AAAAAAAAFBFMAAAA|50452|14|0|52|PM|first|afternoon|lunch| +50453|AAAAAAAAGBFMAAAA|50453|14|0|53|PM|first|afternoon|lunch| +50454|AAAAAAAAHBFMAAAA|50454|14|0|54|PM|first|afternoon|lunch| +50455|AAAAAAAAIBFMAAAA|50455|14|0|55|PM|first|afternoon|lunch| +50456|AAAAAAAAJBFMAAAA|50456|14|0|56|PM|first|afternoon|lunch| +50457|AAAAAAAAKBFMAAAA|50457|14|0|57|PM|first|afternoon|lunch| +50458|AAAAAAAALBFMAAAA|50458|14|0|58|PM|first|afternoon|lunch| +50459|AAAAAAAAMBFMAAAA|50459|14|0|59|PM|first|afternoon|lunch| +50460|AAAAAAAANBFMAAAA|50460|14|1|0|PM|first|afternoon|lunch| +50461|AAAAAAAAOBFMAAAA|50461|14|1|1|PM|first|afternoon|lunch| +50462|AAAAAAAAPBFMAAAA|50462|14|1|2|PM|first|afternoon|lunch| +50463|AAAAAAAAACFMAAAA|50463|14|1|3|PM|first|afternoon|lunch| +50464|AAAAAAAABCFMAAAA|50464|14|1|4|PM|first|afternoon|lunch| +50465|AAAAAAAACCFMAAAA|50465|14|1|5|PM|first|afternoon|lunch| +50466|AAAAAAAADCFMAAAA|50466|14|1|6|PM|first|afternoon|lunch| +50467|AAAAAAAAECFMAAAA|50467|14|1|7|PM|first|afternoon|lunch| +50468|AAAAAAAAFCFMAAAA|50468|14|1|8|PM|first|afternoon|lunch| +50469|AAAAAAAAGCFMAAAA|50469|14|1|9|PM|first|afternoon|lunch| +50470|AAAAAAAAHCFMAAAA|50470|14|1|10|PM|first|afternoon|lunch| +50471|AAAAAAAAICFMAAAA|50471|14|1|11|PM|first|afternoon|lunch| +50472|AAAAAAAAJCFMAAAA|50472|14|1|12|PM|first|afternoon|lunch| +50473|AAAAAAAAKCFMAAAA|50473|14|1|13|PM|first|afternoon|lunch| +50474|AAAAAAAALCFMAAAA|50474|14|1|14|PM|first|afternoon|lunch| +50475|AAAAAAAAMCFMAAAA|50475|14|1|15|PM|first|afternoon|lunch| +50476|AAAAAAAANCFMAAAA|50476|14|1|16|PM|first|afternoon|lunch| +50477|AAAAAAAAOCFMAAAA|50477|14|1|17|PM|first|afternoon|lunch| +50478|AAAAAAAAPCFMAAAA|50478|14|1|18|PM|first|afternoon|lunch| +50479|AAAAAAAAADFMAAAA|50479|14|1|19|PM|first|afternoon|lunch| +50480|AAAAAAAABDFMAAAA|50480|14|1|20|PM|first|afternoon|lunch| +50481|AAAAAAAACDFMAAAA|50481|14|1|21|PM|first|afternoon|lunch| +50482|AAAAAAAADDFMAAAA|50482|14|1|22|PM|first|afternoon|lunch| +50483|AAAAAAAAEDFMAAAA|50483|14|1|23|PM|first|afternoon|lunch| +50484|AAAAAAAAFDFMAAAA|50484|14|1|24|PM|first|afternoon|lunch| +50485|AAAAAAAAGDFMAAAA|50485|14|1|25|PM|first|afternoon|lunch| +50486|AAAAAAAAHDFMAAAA|50486|14|1|26|PM|first|afternoon|lunch| +50487|AAAAAAAAIDFMAAAA|50487|14|1|27|PM|first|afternoon|lunch| +50488|AAAAAAAAJDFMAAAA|50488|14|1|28|PM|first|afternoon|lunch| +50489|AAAAAAAAKDFMAAAA|50489|14|1|29|PM|first|afternoon|lunch| +50490|AAAAAAAALDFMAAAA|50490|14|1|30|PM|first|afternoon|lunch| +50491|AAAAAAAAMDFMAAAA|50491|14|1|31|PM|first|afternoon|lunch| +50492|AAAAAAAANDFMAAAA|50492|14|1|32|PM|first|afternoon|lunch| +50493|AAAAAAAAODFMAAAA|50493|14|1|33|PM|first|afternoon|lunch| +50494|AAAAAAAAPDFMAAAA|50494|14|1|34|PM|first|afternoon|lunch| +50495|AAAAAAAAAEFMAAAA|50495|14|1|35|PM|first|afternoon|lunch| +50496|AAAAAAAABEFMAAAA|50496|14|1|36|PM|first|afternoon|lunch| +50497|AAAAAAAACEFMAAAA|50497|14|1|37|PM|first|afternoon|lunch| +50498|AAAAAAAADEFMAAAA|50498|14|1|38|PM|first|afternoon|lunch| +50499|AAAAAAAAEEFMAAAA|50499|14|1|39|PM|first|afternoon|lunch| +50500|AAAAAAAAFEFMAAAA|50500|14|1|40|PM|first|afternoon|lunch| +50501|AAAAAAAAGEFMAAAA|50501|14|1|41|PM|first|afternoon|lunch| +50502|AAAAAAAAHEFMAAAA|50502|14|1|42|PM|first|afternoon|lunch| +50503|AAAAAAAAIEFMAAAA|50503|14|1|43|PM|first|afternoon|lunch| +50504|AAAAAAAAJEFMAAAA|50504|14|1|44|PM|first|afternoon|lunch| +50505|AAAAAAAAKEFMAAAA|50505|14|1|45|PM|first|afternoon|lunch| +50506|AAAAAAAALEFMAAAA|50506|14|1|46|PM|first|afternoon|lunch| +50507|AAAAAAAAMEFMAAAA|50507|14|1|47|PM|first|afternoon|lunch| +50508|AAAAAAAANEFMAAAA|50508|14|1|48|PM|first|afternoon|lunch| +50509|AAAAAAAAOEFMAAAA|50509|14|1|49|PM|first|afternoon|lunch| +50510|AAAAAAAAPEFMAAAA|50510|14|1|50|PM|first|afternoon|lunch| +50511|AAAAAAAAAFFMAAAA|50511|14|1|51|PM|first|afternoon|lunch| +50512|AAAAAAAABFFMAAAA|50512|14|1|52|PM|first|afternoon|lunch| +50513|AAAAAAAACFFMAAAA|50513|14|1|53|PM|first|afternoon|lunch| +50514|AAAAAAAADFFMAAAA|50514|14|1|54|PM|first|afternoon|lunch| +50515|AAAAAAAAEFFMAAAA|50515|14|1|55|PM|first|afternoon|lunch| +50516|AAAAAAAAFFFMAAAA|50516|14|1|56|PM|first|afternoon|lunch| +50517|AAAAAAAAGFFMAAAA|50517|14|1|57|PM|first|afternoon|lunch| +50518|AAAAAAAAHFFMAAAA|50518|14|1|58|PM|first|afternoon|lunch| +50519|AAAAAAAAIFFMAAAA|50519|14|1|59|PM|first|afternoon|lunch| +50520|AAAAAAAAJFFMAAAA|50520|14|2|0|PM|first|afternoon|lunch| +50521|AAAAAAAAKFFMAAAA|50521|14|2|1|PM|first|afternoon|lunch| +50522|AAAAAAAALFFMAAAA|50522|14|2|2|PM|first|afternoon|lunch| +50523|AAAAAAAAMFFMAAAA|50523|14|2|3|PM|first|afternoon|lunch| +50524|AAAAAAAANFFMAAAA|50524|14|2|4|PM|first|afternoon|lunch| +50525|AAAAAAAAOFFMAAAA|50525|14|2|5|PM|first|afternoon|lunch| +50526|AAAAAAAAPFFMAAAA|50526|14|2|6|PM|first|afternoon|lunch| +50527|AAAAAAAAAGFMAAAA|50527|14|2|7|PM|first|afternoon|lunch| +50528|AAAAAAAABGFMAAAA|50528|14|2|8|PM|first|afternoon|lunch| +50529|AAAAAAAACGFMAAAA|50529|14|2|9|PM|first|afternoon|lunch| +50530|AAAAAAAADGFMAAAA|50530|14|2|10|PM|first|afternoon|lunch| +50531|AAAAAAAAEGFMAAAA|50531|14|2|11|PM|first|afternoon|lunch| +50532|AAAAAAAAFGFMAAAA|50532|14|2|12|PM|first|afternoon|lunch| +50533|AAAAAAAAGGFMAAAA|50533|14|2|13|PM|first|afternoon|lunch| +50534|AAAAAAAAHGFMAAAA|50534|14|2|14|PM|first|afternoon|lunch| +50535|AAAAAAAAIGFMAAAA|50535|14|2|15|PM|first|afternoon|lunch| +50536|AAAAAAAAJGFMAAAA|50536|14|2|16|PM|first|afternoon|lunch| +50537|AAAAAAAAKGFMAAAA|50537|14|2|17|PM|first|afternoon|lunch| +50538|AAAAAAAALGFMAAAA|50538|14|2|18|PM|first|afternoon|lunch| +50539|AAAAAAAAMGFMAAAA|50539|14|2|19|PM|first|afternoon|lunch| +50540|AAAAAAAANGFMAAAA|50540|14|2|20|PM|first|afternoon|lunch| +50541|AAAAAAAAOGFMAAAA|50541|14|2|21|PM|first|afternoon|lunch| +50542|AAAAAAAAPGFMAAAA|50542|14|2|22|PM|first|afternoon|lunch| +50543|AAAAAAAAAHFMAAAA|50543|14|2|23|PM|first|afternoon|lunch| +50544|AAAAAAAABHFMAAAA|50544|14|2|24|PM|first|afternoon|lunch| +50545|AAAAAAAACHFMAAAA|50545|14|2|25|PM|first|afternoon|lunch| +50546|AAAAAAAADHFMAAAA|50546|14|2|26|PM|first|afternoon|lunch| +50547|AAAAAAAAEHFMAAAA|50547|14|2|27|PM|first|afternoon|lunch| +50548|AAAAAAAAFHFMAAAA|50548|14|2|28|PM|first|afternoon|lunch| +50549|AAAAAAAAGHFMAAAA|50549|14|2|29|PM|first|afternoon|lunch| +50550|AAAAAAAAHHFMAAAA|50550|14|2|30|PM|first|afternoon|lunch| +50551|AAAAAAAAIHFMAAAA|50551|14|2|31|PM|first|afternoon|lunch| +50552|AAAAAAAAJHFMAAAA|50552|14|2|32|PM|first|afternoon|lunch| +50553|AAAAAAAAKHFMAAAA|50553|14|2|33|PM|first|afternoon|lunch| +50554|AAAAAAAALHFMAAAA|50554|14|2|34|PM|first|afternoon|lunch| +50555|AAAAAAAAMHFMAAAA|50555|14|2|35|PM|first|afternoon|lunch| +50556|AAAAAAAANHFMAAAA|50556|14|2|36|PM|first|afternoon|lunch| +50557|AAAAAAAAOHFMAAAA|50557|14|2|37|PM|first|afternoon|lunch| +50558|AAAAAAAAPHFMAAAA|50558|14|2|38|PM|first|afternoon|lunch| +50559|AAAAAAAAAIFMAAAA|50559|14|2|39|PM|first|afternoon|lunch| +50560|AAAAAAAABIFMAAAA|50560|14|2|40|PM|first|afternoon|lunch| +50561|AAAAAAAACIFMAAAA|50561|14|2|41|PM|first|afternoon|lunch| +50562|AAAAAAAADIFMAAAA|50562|14|2|42|PM|first|afternoon|lunch| +50563|AAAAAAAAEIFMAAAA|50563|14|2|43|PM|first|afternoon|lunch| +50564|AAAAAAAAFIFMAAAA|50564|14|2|44|PM|first|afternoon|lunch| +50565|AAAAAAAAGIFMAAAA|50565|14|2|45|PM|first|afternoon|lunch| +50566|AAAAAAAAHIFMAAAA|50566|14|2|46|PM|first|afternoon|lunch| +50567|AAAAAAAAIIFMAAAA|50567|14|2|47|PM|first|afternoon|lunch| +50568|AAAAAAAAJIFMAAAA|50568|14|2|48|PM|first|afternoon|lunch| +50569|AAAAAAAAKIFMAAAA|50569|14|2|49|PM|first|afternoon|lunch| +50570|AAAAAAAALIFMAAAA|50570|14|2|50|PM|first|afternoon|lunch| +50571|AAAAAAAAMIFMAAAA|50571|14|2|51|PM|first|afternoon|lunch| +50572|AAAAAAAANIFMAAAA|50572|14|2|52|PM|first|afternoon|lunch| +50573|AAAAAAAAOIFMAAAA|50573|14|2|53|PM|first|afternoon|lunch| +50574|AAAAAAAAPIFMAAAA|50574|14|2|54|PM|first|afternoon|lunch| +50575|AAAAAAAAAJFMAAAA|50575|14|2|55|PM|first|afternoon|lunch| +50576|AAAAAAAABJFMAAAA|50576|14|2|56|PM|first|afternoon|lunch| +50577|AAAAAAAACJFMAAAA|50577|14|2|57|PM|first|afternoon|lunch| +50578|AAAAAAAADJFMAAAA|50578|14|2|58|PM|first|afternoon|lunch| +50579|AAAAAAAAEJFMAAAA|50579|14|2|59|PM|first|afternoon|lunch| +50580|AAAAAAAAFJFMAAAA|50580|14|3|0|PM|first|afternoon|lunch| +50581|AAAAAAAAGJFMAAAA|50581|14|3|1|PM|first|afternoon|lunch| +50582|AAAAAAAAHJFMAAAA|50582|14|3|2|PM|first|afternoon|lunch| +50583|AAAAAAAAIJFMAAAA|50583|14|3|3|PM|first|afternoon|lunch| +50584|AAAAAAAAJJFMAAAA|50584|14|3|4|PM|first|afternoon|lunch| +50585|AAAAAAAAKJFMAAAA|50585|14|3|5|PM|first|afternoon|lunch| +50586|AAAAAAAALJFMAAAA|50586|14|3|6|PM|first|afternoon|lunch| +50587|AAAAAAAAMJFMAAAA|50587|14|3|7|PM|first|afternoon|lunch| +50588|AAAAAAAANJFMAAAA|50588|14|3|8|PM|first|afternoon|lunch| +50589|AAAAAAAAOJFMAAAA|50589|14|3|9|PM|first|afternoon|lunch| +50590|AAAAAAAAPJFMAAAA|50590|14|3|10|PM|first|afternoon|lunch| +50591|AAAAAAAAAKFMAAAA|50591|14|3|11|PM|first|afternoon|lunch| +50592|AAAAAAAABKFMAAAA|50592|14|3|12|PM|first|afternoon|lunch| +50593|AAAAAAAACKFMAAAA|50593|14|3|13|PM|first|afternoon|lunch| +50594|AAAAAAAADKFMAAAA|50594|14|3|14|PM|first|afternoon|lunch| +50595|AAAAAAAAEKFMAAAA|50595|14|3|15|PM|first|afternoon|lunch| +50596|AAAAAAAAFKFMAAAA|50596|14|3|16|PM|first|afternoon|lunch| +50597|AAAAAAAAGKFMAAAA|50597|14|3|17|PM|first|afternoon|lunch| +50598|AAAAAAAAHKFMAAAA|50598|14|3|18|PM|first|afternoon|lunch| +50599|AAAAAAAAIKFMAAAA|50599|14|3|19|PM|first|afternoon|lunch| +50600|AAAAAAAAJKFMAAAA|50600|14|3|20|PM|first|afternoon|lunch| +50601|AAAAAAAAKKFMAAAA|50601|14|3|21|PM|first|afternoon|lunch| +50602|AAAAAAAALKFMAAAA|50602|14|3|22|PM|first|afternoon|lunch| +50603|AAAAAAAAMKFMAAAA|50603|14|3|23|PM|first|afternoon|lunch| +50604|AAAAAAAANKFMAAAA|50604|14|3|24|PM|first|afternoon|lunch| +50605|AAAAAAAAOKFMAAAA|50605|14|3|25|PM|first|afternoon|lunch| +50606|AAAAAAAAPKFMAAAA|50606|14|3|26|PM|first|afternoon|lunch| +50607|AAAAAAAAALFMAAAA|50607|14|3|27|PM|first|afternoon|lunch| +50608|AAAAAAAABLFMAAAA|50608|14|3|28|PM|first|afternoon|lunch| +50609|AAAAAAAACLFMAAAA|50609|14|3|29|PM|first|afternoon|lunch| +50610|AAAAAAAADLFMAAAA|50610|14|3|30|PM|first|afternoon|lunch| +50611|AAAAAAAAELFMAAAA|50611|14|3|31|PM|first|afternoon|lunch| +50612|AAAAAAAAFLFMAAAA|50612|14|3|32|PM|first|afternoon|lunch| +50613|AAAAAAAAGLFMAAAA|50613|14|3|33|PM|first|afternoon|lunch| +50614|AAAAAAAAHLFMAAAA|50614|14|3|34|PM|first|afternoon|lunch| +50615|AAAAAAAAILFMAAAA|50615|14|3|35|PM|first|afternoon|lunch| +50616|AAAAAAAAJLFMAAAA|50616|14|3|36|PM|first|afternoon|lunch| +50617|AAAAAAAAKLFMAAAA|50617|14|3|37|PM|first|afternoon|lunch| +50618|AAAAAAAALLFMAAAA|50618|14|3|38|PM|first|afternoon|lunch| +50619|AAAAAAAAMLFMAAAA|50619|14|3|39|PM|first|afternoon|lunch| +50620|AAAAAAAANLFMAAAA|50620|14|3|40|PM|first|afternoon|lunch| +50621|AAAAAAAAOLFMAAAA|50621|14|3|41|PM|first|afternoon|lunch| +50622|AAAAAAAAPLFMAAAA|50622|14|3|42|PM|first|afternoon|lunch| +50623|AAAAAAAAAMFMAAAA|50623|14|3|43|PM|first|afternoon|lunch| +50624|AAAAAAAABMFMAAAA|50624|14|3|44|PM|first|afternoon|lunch| +50625|AAAAAAAACMFMAAAA|50625|14|3|45|PM|first|afternoon|lunch| +50626|AAAAAAAADMFMAAAA|50626|14|3|46|PM|first|afternoon|lunch| +50627|AAAAAAAAEMFMAAAA|50627|14|3|47|PM|first|afternoon|lunch| +50628|AAAAAAAAFMFMAAAA|50628|14|3|48|PM|first|afternoon|lunch| +50629|AAAAAAAAGMFMAAAA|50629|14|3|49|PM|first|afternoon|lunch| +50630|AAAAAAAAHMFMAAAA|50630|14|3|50|PM|first|afternoon|lunch| +50631|AAAAAAAAIMFMAAAA|50631|14|3|51|PM|first|afternoon|lunch| +50632|AAAAAAAAJMFMAAAA|50632|14|3|52|PM|first|afternoon|lunch| +50633|AAAAAAAAKMFMAAAA|50633|14|3|53|PM|first|afternoon|lunch| +50634|AAAAAAAALMFMAAAA|50634|14|3|54|PM|first|afternoon|lunch| +50635|AAAAAAAAMMFMAAAA|50635|14|3|55|PM|first|afternoon|lunch| +50636|AAAAAAAANMFMAAAA|50636|14|3|56|PM|first|afternoon|lunch| +50637|AAAAAAAAOMFMAAAA|50637|14|3|57|PM|first|afternoon|lunch| +50638|AAAAAAAAPMFMAAAA|50638|14|3|58|PM|first|afternoon|lunch| +50639|AAAAAAAAANFMAAAA|50639|14|3|59|PM|first|afternoon|lunch| +50640|AAAAAAAABNFMAAAA|50640|14|4|0|PM|first|afternoon|lunch| +50641|AAAAAAAACNFMAAAA|50641|14|4|1|PM|first|afternoon|lunch| +50642|AAAAAAAADNFMAAAA|50642|14|4|2|PM|first|afternoon|lunch| +50643|AAAAAAAAENFMAAAA|50643|14|4|3|PM|first|afternoon|lunch| +50644|AAAAAAAAFNFMAAAA|50644|14|4|4|PM|first|afternoon|lunch| +50645|AAAAAAAAGNFMAAAA|50645|14|4|5|PM|first|afternoon|lunch| +50646|AAAAAAAAHNFMAAAA|50646|14|4|6|PM|first|afternoon|lunch| +50647|AAAAAAAAINFMAAAA|50647|14|4|7|PM|first|afternoon|lunch| +50648|AAAAAAAAJNFMAAAA|50648|14|4|8|PM|first|afternoon|lunch| +50649|AAAAAAAAKNFMAAAA|50649|14|4|9|PM|first|afternoon|lunch| +50650|AAAAAAAALNFMAAAA|50650|14|4|10|PM|first|afternoon|lunch| +50651|AAAAAAAAMNFMAAAA|50651|14|4|11|PM|first|afternoon|lunch| +50652|AAAAAAAANNFMAAAA|50652|14|4|12|PM|first|afternoon|lunch| +50653|AAAAAAAAONFMAAAA|50653|14|4|13|PM|first|afternoon|lunch| +50654|AAAAAAAAPNFMAAAA|50654|14|4|14|PM|first|afternoon|lunch| +50655|AAAAAAAAAOFMAAAA|50655|14|4|15|PM|first|afternoon|lunch| +50656|AAAAAAAABOFMAAAA|50656|14|4|16|PM|first|afternoon|lunch| +50657|AAAAAAAACOFMAAAA|50657|14|4|17|PM|first|afternoon|lunch| +50658|AAAAAAAADOFMAAAA|50658|14|4|18|PM|first|afternoon|lunch| +50659|AAAAAAAAEOFMAAAA|50659|14|4|19|PM|first|afternoon|lunch| +50660|AAAAAAAAFOFMAAAA|50660|14|4|20|PM|first|afternoon|lunch| +50661|AAAAAAAAGOFMAAAA|50661|14|4|21|PM|first|afternoon|lunch| +50662|AAAAAAAAHOFMAAAA|50662|14|4|22|PM|first|afternoon|lunch| +50663|AAAAAAAAIOFMAAAA|50663|14|4|23|PM|first|afternoon|lunch| +50664|AAAAAAAAJOFMAAAA|50664|14|4|24|PM|first|afternoon|lunch| +50665|AAAAAAAAKOFMAAAA|50665|14|4|25|PM|first|afternoon|lunch| +50666|AAAAAAAALOFMAAAA|50666|14|4|26|PM|first|afternoon|lunch| +50667|AAAAAAAAMOFMAAAA|50667|14|4|27|PM|first|afternoon|lunch| +50668|AAAAAAAANOFMAAAA|50668|14|4|28|PM|first|afternoon|lunch| +50669|AAAAAAAAOOFMAAAA|50669|14|4|29|PM|first|afternoon|lunch| +50670|AAAAAAAAPOFMAAAA|50670|14|4|30|PM|first|afternoon|lunch| +50671|AAAAAAAAAPFMAAAA|50671|14|4|31|PM|first|afternoon|lunch| +50672|AAAAAAAABPFMAAAA|50672|14|4|32|PM|first|afternoon|lunch| +50673|AAAAAAAACPFMAAAA|50673|14|4|33|PM|first|afternoon|lunch| +50674|AAAAAAAADPFMAAAA|50674|14|4|34|PM|first|afternoon|lunch| +50675|AAAAAAAAEPFMAAAA|50675|14|4|35|PM|first|afternoon|lunch| +50676|AAAAAAAAFPFMAAAA|50676|14|4|36|PM|first|afternoon|lunch| +50677|AAAAAAAAGPFMAAAA|50677|14|4|37|PM|first|afternoon|lunch| +50678|AAAAAAAAHPFMAAAA|50678|14|4|38|PM|first|afternoon|lunch| +50679|AAAAAAAAIPFMAAAA|50679|14|4|39|PM|first|afternoon|lunch| +50680|AAAAAAAAJPFMAAAA|50680|14|4|40|PM|first|afternoon|lunch| +50681|AAAAAAAAKPFMAAAA|50681|14|4|41|PM|first|afternoon|lunch| +50682|AAAAAAAALPFMAAAA|50682|14|4|42|PM|first|afternoon|lunch| +50683|AAAAAAAAMPFMAAAA|50683|14|4|43|PM|first|afternoon|lunch| +50684|AAAAAAAANPFMAAAA|50684|14|4|44|PM|first|afternoon|lunch| +50685|AAAAAAAAOPFMAAAA|50685|14|4|45|PM|first|afternoon|lunch| +50686|AAAAAAAAPPFMAAAA|50686|14|4|46|PM|first|afternoon|lunch| +50687|AAAAAAAAAAGMAAAA|50687|14|4|47|PM|first|afternoon|lunch| +50688|AAAAAAAABAGMAAAA|50688|14|4|48|PM|first|afternoon|lunch| +50689|AAAAAAAACAGMAAAA|50689|14|4|49|PM|first|afternoon|lunch| +50690|AAAAAAAADAGMAAAA|50690|14|4|50|PM|first|afternoon|lunch| +50691|AAAAAAAAEAGMAAAA|50691|14|4|51|PM|first|afternoon|lunch| +50692|AAAAAAAAFAGMAAAA|50692|14|4|52|PM|first|afternoon|lunch| +50693|AAAAAAAAGAGMAAAA|50693|14|4|53|PM|first|afternoon|lunch| +50694|AAAAAAAAHAGMAAAA|50694|14|4|54|PM|first|afternoon|lunch| +50695|AAAAAAAAIAGMAAAA|50695|14|4|55|PM|first|afternoon|lunch| +50696|AAAAAAAAJAGMAAAA|50696|14|4|56|PM|first|afternoon|lunch| +50697|AAAAAAAAKAGMAAAA|50697|14|4|57|PM|first|afternoon|lunch| +50698|AAAAAAAALAGMAAAA|50698|14|4|58|PM|first|afternoon|lunch| +50699|AAAAAAAAMAGMAAAA|50699|14|4|59|PM|first|afternoon|lunch| +50700|AAAAAAAANAGMAAAA|50700|14|5|0|PM|first|afternoon|lunch| +50701|AAAAAAAAOAGMAAAA|50701|14|5|1|PM|first|afternoon|lunch| +50702|AAAAAAAAPAGMAAAA|50702|14|5|2|PM|first|afternoon|lunch| +50703|AAAAAAAAABGMAAAA|50703|14|5|3|PM|first|afternoon|lunch| +50704|AAAAAAAABBGMAAAA|50704|14|5|4|PM|first|afternoon|lunch| +50705|AAAAAAAACBGMAAAA|50705|14|5|5|PM|first|afternoon|lunch| +50706|AAAAAAAADBGMAAAA|50706|14|5|6|PM|first|afternoon|lunch| +50707|AAAAAAAAEBGMAAAA|50707|14|5|7|PM|first|afternoon|lunch| +50708|AAAAAAAAFBGMAAAA|50708|14|5|8|PM|first|afternoon|lunch| +50709|AAAAAAAAGBGMAAAA|50709|14|5|9|PM|first|afternoon|lunch| +50710|AAAAAAAAHBGMAAAA|50710|14|5|10|PM|first|afternoon|lunch| +50711|AAAAAAAAIBGMAAAA|50711|14|5|11|PM|first|afternoon|lunch| +50712|AAAAAAAAJBGMAAAA|50712|14|5|12|PM|first|afternoon|lunch| +50713|AAAAAAAAKBGMAAAA|50713|14|5|13|PM|first|afternoon|lunch| +50714|AAAAAAAALBGMAAAA|50714|14|5|14|PM|first|afternoon|lunch| +50715|AAAAAAAAMBGMAAAA|50715|14|5|15|PM|first|afternoon|lunch| +50716|AAAAAAAANBGMAAAA|50716|14|5|16|PM|first|afternoon|lunch| +50717|AAAAAAAAOBGMAAAA|50717|14|5|17|PM|first|afternoon|lunch| +50718|AAAAAAAAPBGMAAAA|50718|14|5|18|PM|first|afternoon|lunch| +50719|AAAAAAAAACGMAAAA|50719|14|5|19|PM|first|afternoon|lunch| +50720|AAAAAAAABCGMAAAA|50720|14|5|20|PM|first|afternoon|lunch| +50721|AAAAAAAACCGMAAAA|50721|14|5|21|PM|first|afternoon|lunch| +50722|AAAAAAAADCGMAAAA|50722|14|5|22|PM|first|afternoon|lunch| +50723|AAAAAAAAECGMAAAA|50723|14|5|23|PM|first|afternoon|lunch| +50724|AAAAAAAAFCGMAAAA|50724|14|5|24|PM|first|afternoon|lunch| +50725|AAAAAAAAGCGMAAAA|50725|14|5|25|PM|first|afternoon|lunch| +50726|AAAAAAAAHCGMAAAA|50726|14|5|26|PM|first|afternoon|lunch| +50727|AAAAAAAAICGMAAAA|50727|14|5|27|PM|first|afternoon|lunch| +50728|AAAAAAAAJCGMAAAA|50728|14|5|28|PM|first|afternoon|lunch| +50729|AAAAAAAAKCGMAAAA|50729|14|5|29|PM|first|afternoon|lunch| +50730|AAAAAAAALCGMAAAA|50730|14|5|30|PM|first|afternoon|lunch| +50731|AAAAAAAAMCGMAAAA|50731|14|5|31|PM|first|afternoon|lunch| +50732|AAAAAAAANCGMAAAA|50732|14|5|32|PM|first|afternoon|lunch| +50733|AAAAAAAAOCGMAAAA|50733|14|5|33|PM|first|afternoon|lunch| +50734|AAAAAAAAPCGMAAAA|50734|14|5|34|PM|first|afternoon|lunch| +50735|AAAAAAAAADGMAAAA|50735|14|5|35|PM|first|afternoon|lunch| +50736|AAAAAAAABDGMAAAA|50736|14|5|36|PM|first|afternoon|lunch| +50737|AAAAAAAACDGMAAAA|50737|14|5|37|PM|first|afternoon|lunch| +50738|AAAAAAAADDGMAAAA|50738|14|5|38|PM|first|afternoon|lunch| +50739|AAAAAAAAEDGMAAAA|50739|14|5|39|PM|first|afternoon|lunch| +50740|AAAAAAAAFDGMAAAA|50740|14|5|40|PM|first|afternoon|lunch| +50741|AAAAAAAAGDGMAAAA|50741|14|5|41|PM|first|afternoon|lunch| +50742|AAAAAAAAHDGMAAAA|50742|14|5|42|PM|first|afternoon|lunch| +50743|AAAAAAAAIDGMAAAA|50743|14|5|43|PM|first|afternoon|lunch| +50744|AAAAAAAAJDGMAAAA|50744|14|5|44|PM|first|afternoon|lunch| +50745|AAAAAAAAKDGMAAAA|50745|14|5|45|PM|first|afternoon|lunch| +50746|AAAAAAAALDGMAAAA|50746|14|5|46|PM|first|afternoon|lunch| +50747|AAAAAAAAMDGMAAAA|50747|14|5|47|PM|first|afternoon|lunch| +50748|AAAAAAAANDGMAAAA|50748|14|5|48|PM|first|afternoon|lunch| +50749|AAAAAAAAODGMAAAA|50749|14|5|49|PM|first|afternoon|lunch| +50750|AAAAAAAAPDGMAAAA|50750|14|5|50|PM|first|afternoon|lunch| +50751|AAAAAAAAAEGMAAAA|50751|14|5|51|PM|first|afternoon|lunch| +50752|AAAAAAAABEGMAAAA|50752|14|5|52|PM|first|afternoon|lunch| +50753|AAAAAAAACEGMAAAA|50753|14|5|53|PM|first|afternoon|lunch| +50754|AAAAAAAADEGMAAAA|50754|14|5|54|PM|first|afternoon|lunch| +50755|AAAAAAAAEEGMAAAA|50755|14|5|55|PM|first|afternoon|lunch| +50756|AAAAAAAAFEGMAAAA|50756|14|5|56|PM|first|afternoon|lunch| +50757|AAAAAAAAGEGMAAAA|50757|14|5|57|PM|first|afternoon|lunch| +50758|AAAAAAAAHEGMAAAA|50758|14|5|58|PM|first|afternoon|lunch| +50759|AAAAAAAAIEGMAAAA|50759|14|5|59|PM|first|afternoon|lunch| +50760|AAAAAAAAJEGMAAAA|50760|14|6|0|PM|first|afternoon|lunch| +50761|AAAAAAAAKEGMAAAA|50761|14|6|1|PM|first|afternoon|lunch| +50762|AAAAAAAALEGMAAAA|50762|14|6|2|PM|first|afternoon|lunch| +50763|AAAAAAAAMEGMAAAA|50763|14|6|3|PM|first|afternoon|lunch| +50764|AAAAAAAANEGMAAAA|50764|14|6|4|PM|first|afternoon|lunch| +50765|AAAAAAAAOEGMAAAA|50765|14|6|5|PM|first|afternoon|lunch| +50766|AAAAAAAAPEGMAAAA|50766|14|6|6|PM|first|afternoon|lunch| +50767|AAAAAAAAAFGMAAAA|50767|14|6|7|PM|first|afternoon|lunch| +50768|AAAAAAAABFGMAAAA|50768|14|6|8|PM|first|afternoon|lunch| +50769|AAAAAAAACFGMAAAA|50769|14|6|9|PM|first|afternoon|lunch| +50770|AAAAAAAADFGMAAAA|50770|14|6|10|PM|first|afternoon|lunch| +50771|AAAAAAAAEFGMAAAA|50771|14|6|11|PM|first|afternoon|lunch| +50772|AAAAAAAAFFGMAAAA|50772|14|6|12|PM|first|afternoon|lunch| +50773|AAAAAAAAGFGMAAAA|50773|14|6|13|PM|first|afternoon|lunch| +50774|AAAAAAAAHFGMAAAA|50774|14|6|14|PM|first|afternoon|lunch| +50775|AAAAAAAAIFGMAAAA|50775|14|6|15|PM|first|afternoon|lunch| +50776|AAAAAAAAJFGMAAAA|50776|14|6|16|PM|first|afternoon|lunch| +50777|AAAAAAAAKFGMAAAA|50777|14|6|17|PM|first|afternoon|lunch| +50778|AAAAAAAALFGMAAAA|50778|14|6|18|PM|first|afternoon|lunch| +50779|AAAAAAAAMFGMAAAA|50779|14|6|19|PM|first|afternoon|lunch| +50780|AAAAAAAANFGMAAAA|50780|14|6|20|PM|first|afternoon|lunch| +50781|AAAAAAAAOFGMAAAA|50781|14|6|21|PM|first|afternoon|lunch| +50782|AAAAAAAAPFGMAAAA|50782|14|6|22|PM|first|afternoon|lunch| +50783|AAAAAAAAAGGMAAAA|50783|14|6|23|PM|first|afternoon|lunch| +50784|AAAAAAAABGGMAAAA|50784|14|6|24|PM|first|afternoon|lunch| +50785|AAAAAAAACGGMAAAA|50785|14|6|25|PM|first|afternoon|lunch| +50786|AAAAAAAADGGMAAAA|50786|14|6|26|PM|first|afternoon|lunch| +50787|AAAAAAAAEGGMAAAA|50787|14|6|27|PM|first|afternoon|lunch| +50788|AAAAAAAAFGGMAAAA|50788|14|6|28|PM|first|afternoon|lunch| +50789|AAAAAAAAGGGMAAAA|50789|14|6|29|PM|first|afternoon|lunch| +50790|AAAAAAAAHGGMAAAA|50790|14|6|30|PM|first|afternoon|lunch| +50791|AAAAAAAAIGGMAAAA|50791|14|6|31|PM|first|afternoon|lunch| +50792|AAAAAAAAJGGMAAAA|50792|14|6|32|PM|first|afternoon|lunch| +50793|AAAAAAAAKGGMAAAA|50793|14|6|33|PM|first|afternoon|lunch| +50794|AAAAAAAALGGMAAAA|50794|14|6|34|PM|first|afternoon|lunch| +50795|AAAAAAAAMGGMAAAA|50795|14|6|35|PM|first|afternoon|lunch| +50796|AAAAAAAANGGMAAAA|50796|14|6|36|PM|first|afternoon|lunch| +50797|AAAAAAAAOGGMAAAA|50797|14|6|37|PM|first|afternoon|lunch| +50798|AAAAAAAAPGGMAAAA|50798|14|6|38|PM|first|afternoon|lunch| +50799|AAAAAAAAAHGMAAAA|50799|14|6|39|PM|first|afternoon|lunch| +50800|AAAAAAAABHGMAAAA|50800|14|6|40|PM|first|afternoon|lunch| +50801|AAAAAAAACHGMAAAA|50801|14|6|41|PM|first|afternoon|lunch| +50802|AAAAAAAADHGMAAAA|50802|14|6|42|PM|first|afternoon|lunch| +50803|AAAAAAAAEHGMAAAA|50803|14|6|43|PM|first|afternoon|lunch| +50804|AAAAAAAAFHGMAAAA|50804|14|6|44|PM|first|afternoon|lunch| +50805|AAAAAAAAGHGMAAAA|50805|14|6|45|PM|first|afternoon|lunch| +50806|AAAAAAAAHHGMAAAA|50806|14|6|46|PM|first|afternoon|lunch| +50807|AAAAAAAAIHGMAAAA|50807|14|6|47|PM|first|afternoon|lunch| +50808|AAAAAAAAJHGMAAAA|50808|14|6|48|PM|first|afternoon|lunch| +50809|AAAAAAAAKHGMAAAA|50809|14|6|49|PM|first|afternoon|lunch| +50810|AAAAAAAALHGMAAAA|50810|14|6|50|PM|first|afternoon|lunch| +50811|AAAAAAAAMHGMAAAA|50811|14|6|51|PM|first|afternoon|lunch| +50812|AAAAAAAANHGMAAAA|50812|14|6|52|PM|first|afternoon|lunch| +50813|AAAAAAAAOHGMAAAA|50813|14|6|53|PM|first|afternoon|lunch| +50814|AAAAAAAAPHGMAAAA|50814|14|6|54|PM|first|afternoon|lunch| +50815|AAAAAAAAAIGMAAAA|50815|14|6|55|PM|first|afternoon|lunch| +50816|AAAAAAAABIGMAAAA|50816|14|6|56|PM|first|afternoon|lunch| +50817|AAAAAAAACIGMAAAA|50817|14|6|57|PM|first|afternoon|lunch| +50818|AAAAAAAADIGMAAAA|50818|14|6|58|PM|first|afternoon|lunch| +50819|AAAAAAAAEIGMAAAA|50819|14|6|59|PM|first|afternoon|lunch| +50820|AAAAAAAAFIGMAAAA|50820|14|7|0|PM|first|afternoon|lunch| +50821|AAAAAAAAGIGMAAAA|50821|14|7|1|PM|first|afternoon|lunch| +50822|AAAAAAAAHIGMAAAA|50822|14|7|2|PM|first|afternoon|lunch| +50823|AAAAAAAAIIGMAAAA|50823|14|7|3|PM|first|afternoon|lunch| +50824|AAAAAAAAJIGMAAAA|50824|14|7|4|PM|first|afternoon|lunch| +50825|AAAAAAAAKIGMAAAA|50825|14|7|5|PM|first|afternoon|lunch| +50826|AAAAAAAALIGMAAAA|50826|14|7|6|PM|first|afternoon|lunch| +50827|AAAAAAAAMIGMAAAA|50827|14|7|7|PM|first|afternoon|lunch| +50828|AAAAAAAANIGMAAAA|50828|14|7|8|PM|first|afternoon|lunch| +50829|AAAAAAAAOIGMAAAA|50829|14|7|9|PM|first|afternoon|lunch| +50830|AAAAAAAAPIGMAAAA|50830|14|7|10|PM|first|afternoon|lunch| +50831|AAAAAAAAAJGMAAAA|50831|14|7|11|PM|first|afternoon|lunch| +50832|AAAAAAAABJGMAAAA|50832|14|7|12|PM|first|afternoon|lunch| +50833|AAAAAAAACJGMAAAA|50833|14|7|13|PM|first|afternoon|lunch| +50834|AAAAAAAADJGMAAAA|50834|14|7|14|PM|first|afternoon|lunch| +50835|AAAAAAAAEJGMAAAA|50835|14|7|15|PM|first|afternoon|lunch| +50836|AAAAAAAAFJGMAAAA|50836|14|7|16|PM|first|afternoon|lunch| +50837|AAAAAAAAGJGMAAAA|50837|14|7|17|PM|first|afternoon|lunch| +50838|AAAAAAAAHJGMAAAA|50838|14|7|18|PM|first|afternoon|lunch| +50839|AAAAAAAAIJGMAAAA|50839|14|7|19|PM|first|afternoon|lunch| +50840|AAAAAAAAJJGMAAAA|50840|14|7|20|PM|first|afternoon|lunch| +50841|AAAAAAAAKJGMAAAA|50841|14|7|21|PM|first|afternoon|lunch| +50842|AAAAAAAALJGMAAAA|50842|14|7|22|PM|first|afternoon|lunch| +50843|AAAAAAAAMJGMAAAA|50843|14|7|23|PM|first|afternoon|lunch| +50844|AAAAAAAANJGMAAAA|50844|14|7|24|PM|first|afternoon|lunch| +50845|AAAAAAAAOJGMAAAA|50845|14|7|25|PM|first|afternoon|lunch| +50846|AAAAAAAAPJGMAAAA|50846|14|7|26|PM|first|afternoon|lunch| +50847|AAAAAAAAAKGMAAAA|50847|14|7|27|PM|first|afternoon|lunch| +50848|AAAAAAAABKGMAAAA|50848|14|7|28|PM|first|afternoon|lunch| +50849|AAAAAAAACKGMAAAA|50849|14|7|29|PM|first|afternoon|lunch| +50850|AAAAAAAADKGMAAAA|50850|14|7|30|PM|first|afternoon|lunch| +50851|AAAAAAAAEKGMAAAA|50851|14|7|31|PM|first|afternoon|lunch| +50852|AAAAAAAAFKGMAAAA|50852|14|7|32|PM|first|afternoon|lunch| +50853|AAAAAAAAGKGMAAAA|50853|14|7|33|PM|first|afternoon|lunch| +50854|AAAAAAAAHKGMAAAA|50854|14|7|34|PM|first|afternoon|lunch| +50855|AAAAAAAAIKGMAAAA|50855|14|7|35|PM|first|afternoon|lunch| +50856|AAAAAAAAJKGMAAAA|50856|14|7|36|PM|first|afternoon|lunch| +50857|AAAAAAAAKKGMAAAA|50857|14|7|37|PM|first|afternoon|lunch| +50858|AAAAAAAALKGMAAAA|50858|14|7|38|PM|first|afternoon|lunch| +50859|AAAAAAAAMKGMAAAA|50859|14|7|39|PM|first|afternoon|lunch| +50860|AAAAAAAANKGMAAAA|50860|14|7|40|PM|first|afternoon|lunch| +50861|AAAAAAAAOKGMAAAA|50861|14|7|41|PM|first|afternoon|lunch| +50862|AAAAAAAAPKGMAAAA|50862|14|7|42|PM|first|afternoon|lunch| +50863|AAAAAAAAALGMAAAA|50863|14|7|43|PM|first|afternoon|lunch| +50864|AAAAAAAABLGMAAAA|50864|14|7|44|PM|first|afternoon|lunch| +50865|AAAAAAAACLGMAAAA|50865|14|7|45|PM|first|afternoon|lunch| +50866|AAAAAAAADLGMAAAA|50866|14|7|46|PM|first|afternoon|lunch| +50867|AAAAAAAAELGMAAAA|50867|14|7|47|PM|first|afternoon|lunch| +50868|AAAAAAAAFLGMAAAA|50868|14|7|48|PM|first|afternoon|lunch| +50869|AAAAAAAAGLGMAAAA|50869|14|7|49|PM|first|afternoon|lunch| +50870|AAAAAAAAHLGMAAAA|50870|14|7|50|PM|first|afternoon|lunch| +50871|AAAAAAAAILGMAAAA|50871|14|7|51|PM|first|afternoon|lunch| +50872|AAAAAAAAJLGMAAAA|50872|14|7|52|PM|first|afternoon|lunch| +50873|AAAAAAAAKLGMAAAA|50873|14|7|53|PM|first|afternoon|lunch| +50874|AAAAAAAALLGMAAAA|50874|14|7|54|PM|first|afternoon|lunch| +50875|AAAAAAAAMLGMAAAA|50875|14|7|55|PM|first|afternoon|lunch| +50876|AAAAAAAANLGMAAAA|50876|14|7|56|PM|first|afternoon|lunch| +50877|AAAAAAAAOLGMAAAA|50877|14|7|57|PM|first|afternoon|lunch| +50878|AAAAAAAAPLGMAAAA|50878|14|7|58|PM|first|afternoon|lunch| +50879|AAAAAAAAAMGMAAAA|50879|14|7|59|PM|first|afternoon|lunch| +50880|AAAAAAAABMGMAAAA|50880|14|8|0|PM|first|afternoon|lunch| +50881|AAAAAAAACMGMAAAA|50881|14|8|1|PM|first|afternoon|lunch| +50882|AAAAAAAADMGMAAAA|50882|14|8|2|PM|first|afternoon|lunch| +50883|AAAAAAAAEMGMAAAA|50883|14|8|3|PM|first|afternoon|lunch| +50884|AAAAAAAAFMGMAAAA|50884|14|8|4|PM|first|afternoon|lunch| +50885|AAAAAAAAGMGMAAAA|50885|14|8|5|PM|first|afternoon|lunch| +50886|AAAAAAAAHMGMAAAA|50886|14|8|6|PM|first|afternoon|lunch| +50887|AAAAAAAAIMGMAAAA|50887|14|8|7|PM|first|afternoon|lunch| +50888|AAAAAAAAJMGMAAAA|50888|14|8|8|PM|first|afternoon|lunch| +50889|AAAAAAAAKMGMAAAA|50889|14|8|9|PM|first|afternoon|lunch| +50890|AAAAAAAALMGMAAAA|50890|14|8|10|PM|first|afternoon|lunch| +50891|AAAAAAAAMMGMAAAA|50891|14|8|11|PM|first|afternoon|lunch| +50892|AAAAAAAANMGMAAAA|50892|14|8|12|PM|first|afternoon|lunch| +50893|AAAAAAAAOMGMAAAA|50893|14|8|13|PM|first|afternoon|lunch| +50894|AAAAAAAAPMGMAAAA|50894|14|8|14|PM|first|afternoon|lunch| +50895|AAAAAAAAANGMAAAA|50895|14|8|15|PM|first|afternoon|lunch| +50896|AAAAAAAABNGMAAAA|50896|14|8|16|PM|first|afternoon|lunch| +50897|AAAAAAAACNGMAAAA|50897|14|8|17|PM|first|afternoon|lunch| +50898|AAAAAAAADNGMAAAA|50898|14|8|18|PM|first|afternoon|lunch| +50899|AAAAAAAAENGMAAAA|50899|14|8|19|PM|first|afternoon|lunch| +50900|AAAAAAAAFNGMAAAA|50900|14|8|20|PM|first|afternoon|lunch| +50901|AAAAAAAAGNGMAAAA|50901|14|8|21|PM|first|afternoon|lunch| +50902|AAAAAAAAHNGMAAAA|50902|14|8|22|PM|first|afternoon|lunch| +50903|AAAAAAAAINGMAAAA|50903|14|8|23|PM|first|afternoon|lunch| +50904|AAAAAAAAJNGMAAAA|50904|14|8|24|PM|first|afternoon|lunch| +50905|AAAAAAAAKNGMAAAA|50905|14|8|25|PM|first|afternoon|lunch| +50906|AAAAAAAALNGMAAAA|50906|14|8|26|PM|first|afternoon|lunch| +50907|AAAAAAAAMNGMAAAA|50907|14|8|27|PM|first|afternoon|lunch| +50908|AAAAAAAANNGMAAAA|50908|14|8|28|PM|first|afternoon|lunch| +50909|AAAAAAAAONGMAAAA|50909|14|8|29|PM|first|afternoon|lunch| +50910|AAAAAAAAPNGMAAAA|50910|14|8|30|PM|first|afternoon|lunch| +50911|AAAAAAAAAOGMAAAA|50911|14|8|31|PM|first|afternoon|lunch| +50912|AAAAAAAABOGMAAAA|50912|14|8|32|PM|first|afternoon|lunch| +50913|AAAAAAAACOGMAAAA|50913|14|8|33|PM|first|afternoon|lunch| +50914|AAAAAAAADOGMAAAA|50914|14|8|34|PM|first|afternoon|lunch| +50915|AAAAAAAAEOGMAAAA|50915|14|8|35|PM|first|afternoon|lunch| +50916|AAAAAAAAFOGMAAAA|50916|14|8|36|PM|first|afternoon|lunch| +50917|AAAAAAAAGOGMAAAA|50917|14|8|37|PM|first|afternoon|lunch| +50918|AAAAAAAAHOGMAAAA|50918|14|8|38|PM|first|afternoon|lunch| +50919|AAAAAAAAIOGMAAAA|50919|14|8|39|PM|first|afternoon|lunch| +50920|AAAAAAAAJOGMAAAA|50920|14|8|40|PM|first|afternoon|lunch| +50921|AAAAAAAAKOGMAAAA|50921|14|8|41|PM|first|afternoon|lunch| +50922|AAAAAAAALOGMAAAA|50922|14|8|42|PM|first|afternoon|lunch| +50923|AAAAAAAAMOGMAAAA|50923|14|8|43|PM|first|afternoon|lunch| +50924|AAAAAAAANOGMAAAA|50924|14|8|44|PM|first|afternoon|lunch| +50925|AAAAAAAAOOGMAAAA|50925|14|8|45|PM|first|afternoon|lunch| +50926|AAAAAAAAPOGMAAAA|50926|14|8|46|PM|first|afternoon|lunch| +50927|AAAAAAAAAPGMAAAA|50927|14|8|47|PM|first|afternoon|lunch| +50928|AAAAAAAABPGMAAAA|50928|14|8|48|PM|first|afternoon|lunch| +50929|AAAAAAAACPGMAAAA|50929|14|8|49|PM|first|afternoon|lunch| +50930|AAAAAAAADPGMAAAA|50930|14|8|50|PM|first|afternoon|lunch| +50931|AAAAAAAAEPGMAAAA|50931|14|8|51|PM|first|afternoon|lunch| +50932|AAAAAAAAFPGMAAAA|50932|14|8|52|PM|first|afternoon|lunch| +50933|AAAAAAAAGPGMAAAA|50933|14|8|53|PM|first|afternoon|lunch| +50934|AAAAAAAAHPGMAAAA|50934|14|8|54|PM|first|afternoon|lunch| +50935|AAAAAAAAIPGMAAAA|50935|14|8|55|PM|first|afternoon|lunch| +50936|AAAAAAAAJPGMAAAA|50936|14|8|56|PM|first|afternoon|lunch| +50937|AAAAAAAAKPGMAAAA|50937|14|8|57|PM|first|afternoon|lunch| +50938|AAAAAAAALPGMAAAA|50938|14|8|58|PM|first|afternoon|lunch| +50939|AAAAAAAAMPGMAAAA|50939|14|8|59|PM|first|afternoon|lunch| +50940|AAAAAAAANPGMAAAA|50940|14|9|0|PM|first|afternoon|lunch| +50941|AAAAAAAAOPGMAAAA|50941|14|9|1|PM|first|afternoon|lunch| +50942|AAAAAAAAPPGMAAAA|50942|14|9|2|PM|first|afternoon|lunch| +50943|AAAAAAAAAAHMAAAA|50943|14|9|3|PM|first|afternoon|lunch| +50944|AAAAAAAABAHMAAAA|50944|14|9|4|PM|first|afternoon|lunch| +50945|AAAAAAAACAHMAAAA|50945|14|9|5|PM|first|afternoon|lunch| +50946|AAAAAAAADAHMAAAA|50946|14|9|6|PM|first|afternoon|lunch| +50947|AAAAAAAAEAHMAAAA|50947|14|9|7|PM|first|afternoon|lunch| +50948|AAAAAAAAFAHMAAAA|50948|14|9|8|PM|first|afternoon|lunch| +50949|AAAAAAAAGAHMAAAA|50949|14|9|9|PM|first|afternoon|lunch| +50950|AAAAAAAAHAHMAAAA|50950|14|9|10|PM|first|afternoon|lunch| +50951|AAAAAAAAIAHMAAAA|50951|14|9|11|PM|first|afternoon|lunch| +50952|AAAAAAAAJAHMAAAA|50952|14|9|12|PM|first|afternoon|lunch| +50953|AAAAAAAAKAHMAAAA|50953|14|9|13|PM|first|afternoon|lunch| +50954|AAAAAAAALAHMAAAA|50954|14|9|14|PM|first|afternoon|lunch| +50955|AAAAAAAAMAHMAAAA|50955|14|9|15|PM|first|afternoon|lunch| +50956|AAAAAAAANAHMAAAA|50956|14|9|16|PM|first|afternoon|lunch| +50957|AAAAAAAAOAHMAAAA|50957|14|9|17|PM|first|afternoon|lunch| +50958|AAAAAAAAPAHMAAAA|50958|14|9|18|PM|first|afternoon|lunch| +50959|AAAAAAAAABHMAAAA|50959|14|9|19|PM|first|afternoon|lunch| +50960|AAAAAAAABBHMAAAA|50960|14|9|20|PM|first|afternoon|lunch| +50961|AAAAAAAACBHMAAAA|50961|14|9|21|PM|first|afternoon|lunch| +50962|AAAAAAAADBHMAAAA|50962|14|9|22|PM|first|afternoon|lunch| +50963|AAAAAAAAEBHMAAAA|50963|14|9|23|PM|first|afternoon|lunch| +50964|AAAAAAAAFBHMAAAA|50964|14|9|24|PM|first|afternoon|lunch| +50965|AAAAAAAAGBHMAAAA|50965|14|9|25|PM|first|afternoon|lunch| +50966|AAAAAAAAHBHMAAAA|50966|14|9|26|PM|first|afternoon|lunch| +50967|AAAAAAAAIBHMAAAA|50967|14|9|27|PM|first|afternoon|lunch| +50968|AAAAAAAAJBHMAAAA|50968|14|9|28|PM|first|afternoon|lunch| +50969|AAAAAAAAKBHMAAAA|50969|14|9|29|PM|first|afternoon|lunch| +50970|AAAAAAAALBHMAAAA|50970|14|9|30|PM|first|afternoon|lunch| +50971|AAAAAAAAMBHMAAAA|50971|14|9|31|PM|first|afternoon|lunch| +50972|AAAAAAAANBHMAAAA|50972|14|9|32|PM|first|afternoon|lunch| +50973|AAAAAAAAOBHMAAAA|50973|14|9|33|PM|first|afternoon|lunch| +50974|AAAAAAAAPBHMAAAA|50974|14|9|34|PM|first|afternoon|lunch| +50975|AAAAAAAAACHMAAAA|50975|14|9|35|PM|first|afternoon|lunch| +50976|AAAAAAAABCHMAAAA|50976|14|9|36|PM|first|afternoon|lunch| +50977|AAAAAAAACCHMAAAA|50977|14|9|37|PM|first|afternoon|lunch| +50978|AAAAAAAADCHMAAAA|50978|14|9|38|PM|first|afternoon|lunch| +50979|AAAAAAAAECHMAAAA|50979|14|9|39|PM|first|afternoon|lunch| +50980|AAAAAAAAFCHMAAAA|50980|14|9|40|PM|first|afternoon|lunch| +50981|AAAAAAAAGCHMAAAA|50981|14|9|41|PM|first|afternoon|lunch| +50982|AAAAAAAAHCHMAAAA|50982|14|9|42|PM|first|afternoon|lunch| +50983|AAAAAAAAICHMAAAA|50983|14|9|43|PM|first|afternoon|lunch| +50984|AAAAAAAAJCHMAAAA|50984|14|9|44|PM|first|afternoon|lunch| +50985|AAAAAAAAKCHMAAAA|50985|14|9|45|PM|first|afternoon|lunch| +50986|AAAAAAAALCHMAAAA|50986|14|9|46|PM|first|afternoon|lunch| +50987|AAAAAAAAMCHMAAAA|50987|14|9|47|PM|first|afternoon|lunch| +50988|AAAAAAAANCHMAAAA|50988|14|9|48|PM|first|afternoon|lunch| +50989|AAAAAAAAOCHMAAAA|50989|14|9|49|PM|first|afternoon|lunch| +50990|AAAAAAAAPCHMAAAA|50990|14|9|50|PM|first|afternoon|lunch| +50991|AAAAAAAAADHMAAAA|50991|14|9|51|PM|first|afternoon|lunch| +50992|AAAAAAAABDHMAAAA|50992|14|9|52|PM|first|afternoon|lunch| +50993|AAAAAAAACDHMAAAA|50993|14|9|53|PM|first|afternoon|lunch| +50994|AAAAAAAADDHMAAAA|50994|14|9|54|PM|first|afternoon|lunch| +50995|AAAAAAAAEDHMAAAA|50995|14|9|55|PM|first|afternoon|lunch| +50996|AAAAAAAAFDHMAAAA|50996|14|9|56|PM|first|afternoon|lunch| +50997|AAAAAAAAGDHMAAAA|50997|14|9|57|PM|first|afternoon|lunch| +50998|AAAAAAAAHDHMAAAA|50998|14|9|58|PM|first|afternoon|lunch| +50999|AAAAAAAAIDHMAAAA|50999|14|9|59|PM|first|afternoon|lunch| +51000|AAAAAAAAJDHMAAAA|51000|14|10|0|PM|first|afternoon|lunch| +51001|AAAAAAAAKDHMAAAA|51001|14|10|1|PM|first|afternoon|lunch| +51002|AAAAAAAALDHMAAAA|51002|14|10|2|PM|first|afternoon|lunch| +51003|AAAAAAAAMDHMAAAA|51003|14|10|3|PM|first|afternoon|lunch| +51004|AAAAAAAANDHMAAAA|51004|14|10|4|PM|first|afternoon|lunch| +51005|AAAAAAAAODHMAAAA|51005|14|10|5|PM|first|afternoon|lunch| +51006|AAAAAAAAPDHMAAAA|51006|14|10|6|PM|first|afternoon|lunch| +51007|AAAAAAAAAEHMAAAA|51007|14|10|7|PM|first|afternoon|lunch| +51008|AAAAAAAABEHMAAAA|51008|14|10|8|PM|first|afternoon|lunch| +51009|AAAAAAAACEHMAAAA|51009|14|10|9|PM|first|afternoon|lunch| +51010|AAAAAAAADEHMAAAA|51010|14|10|10|PM|first|afternoon|lunch| +51011|AAAAAAAAEEHMAAAA|51011|14|10|11|PM|first|afternoon|lunch| +51012|AAAAAAAAFEHMAAAA|51012|14|10|12|PM|first|afternoon|lunch| +51013|AAAAAAAAGEHMAAAA|51013|14|10|13|PM|first|afternoon|lunch| +51014|AAAAAAAAHEHMAAAA|51014|14|10|14|PM|first|afternoon|lunch| +51015|AAAAAAAAIEHMAAAA|51015|14|10|15|PM|first|afternoon|lunch| +51016|AAAAAAAAJEHMAAAA|51016|14|10|16|PM|first|afternoon|lunch| +51017|AAAAAAAAKEHMAAAA|51017|14|10|17|PM|first|afternoon|lunch| +51018|AAAAAAAALEHMAAAA|51018|14|10|18|PM|first|afternoon|lunch| +51019|AAAAAAAAMEHMAAAA|51019|14|10|19|PM|first|afternoon|lunch| +51020|AAAAAAAANEHMAAAA|51020|14|10|20|PM|first|afternoon|lunch| +51021|AAAAAAAAOEHMAAAA|51021|14|10|21|PM|first|afternoon|lunch| +51022|AAAAAAAAPEHMAAAA|51022|14|10|22|PM|first|afternoon|lunch| +51023|AAAAAAAAAFHMAAAA|51023|14|10|23|PM|first|afternoon|lunch| +51024|AAAAAAAABFHMAAAA|51024|14|10|24|PM|first|afternoon|lunch| +51025|AAAAAAAACFHMAAAA|51025|14|10|25|PM|first|afternoon|lunch| +51026|AAAAAAAADFHMAAAA|51026|14|10|26|PM|first|afternoon|lunch| +51027|AAAAAAAAEFHMAAAA|51027|14|10|27|PM|first|afternoon|lunch| +51028|AAAAAAAAFFHMAAAA|51028|14|10|28|PM|first|afternoon|lunch| +51029|AAAAAAAAGFHMAAAA|51029|14|10|29|PM|first|afternoon|lunch| +51030|AAAAAAAAHFHMAAAA|51030|14|10|30|PM|first|afternoon|lunch| +51031|AAAAAAAAIFHMAAAA|51031|14|10|31|PM|first|afternoon|lunch| +51032|AAAAAAAAJFHMAAAA|51032|14|10|32|PM|first|afternoon|lunch| +51033|AAAAAAAAKFHMAAAA|51033|14|10|33|PM|first|afternoon|lunch| +51034|AAAAAAAALFHMAAAA|51034|14|10|34|PM|first|afternoon|lunch| +51035|AAAAAAAAMFHMAAAA|51035|14|10|35|PM|first|afternoon|lunch| +51036|AAAAAAAANFHMAAAA|51036|14|10|36|PM|first|afternoon|lunch| +51037|AAAAAAAAOFHMAAAA|51037|14|10|37|PM|first|afternoon|lunch| +51038|AAAAAAAAPFHMAAAA|51038|14|10|38|PM|first|afternoon|lunch| +51039|AAAAAAAAAGHMAAAA|51039|14|10|39|PM|first|afternoon|lunch| +51040|AAAAAAAABGHMAAAA|51040|14|10|40|PM|first|afternoon|lunch| +51041|AAAAAAAACGHMAAAA|51041|14|10|41|PM|first|afternoon|lunch| +51042|AAAAAAAADGHMAAAA|51042|14|10|42|PM|first|afternoon|lunch| +51043|AAAAAAAAEGHMAAAA|51043|14|10|43|PM|first|afternoon|lunch| +51044|AAAAAAAAFGHMAAAA|51044|14|10|44|PM|first|afternoon|lunch| +51045|AAAAAAAAGGHMAAAA|51045|14|10|45|PM|first|afternoon|lunch| +51046|AAAAAAAAHGHMAAAA|51046|14|10|46|PM|first|afternoon|lunch| +51047|AAAAAAAAIGHMAAAA|51047|14|10|47|PM|first|afternoon|lunch| +51048|AAAAAAAAJGHMAAAA|51048|14|10|48|PM|first|afternoon|lunch| +51049|AAAAAAAAKGHMAAAA|51049|14|10|49|PM|first|afternoon|lunch| +51050|AAAAAAAALGHMAAAA|51050|14|10|50|PM|first|afternoon|lunch| +51051|AAAAAAAAMGHMAAAA|51051|14|10|51|PM|first|afternoon|lunch| +51052|AAAAAAAANGHMAAAA|51052|14|10|52|PM|first|afternoon|lunch| +51053|AAAAAAAAOGHMAAAA|51053|14|10|53|PM|first|afternoon|lunch| +51054|AAAAAAAAPGHMAAAA|51054|14|10|54|PM|first|afternoon|lunch| +51055|AAAAAAAAAHHMAAAA|51055|14|10|55|PM|first|afternoon|lunch| +51056|AAAAAAAABHHMAAAA|51056|14|10|56|PM|first|afternoon|lunch| +51057|AAAAAAAACHHMAAAA|51057|14|10|57|PM|first|afternoon|lunch| +51058|AAAAAAAADHHMAAAA|51058|14|10|58|PM|first|afternoon|lunch| +51059|AAAAAAAAEHHMAAAA|51059|14|10|59|PM|first|afternoon|lunch| +51060|AAAAAAAAFHHMAAAA|51060|14|11|0|PM|first|afternoon|lunch| +51061|AAAAAAAAGHHMAAAA|51061|14|11|1|PM|first|afternoon|lunch| +51062|AAAAAAAAHHHMAAAA|51062|14|11|2|PM|first|afternoon|lunch| +51063|AAAAAAAAIHHMAAAA|51063|14|11|3|PM|first|afternoon|lunch| +51064|AAAAAAAAJHHMAAAA|51064|14|11|4|PM|first|afternoon|lunch| +51065|AAAAAAAAKHHMAAAA|51065|14|11|5|PM|first|afternoon|lunch| +51066|AAAAAAAALHHMAAAA|51066|14|11|6|PM|first|afternoon|lunch| +51067|AAAAAAAAMHHMAAAA|51067|14|11|7|PM|first|afternoon|lunch| +51068|AAAAAAAANHHMAAAA|51068|14|11|8|PM|first|afternoon|lunch| +51069|AAAAAAAAOHHMAAAA|51069|14|11|9|PM|first|afternoon|lunch| +51070|AAAAAAAAPHHMAAAA|51070|14|11|10|PM|first|afternoon|lunch| +51071|AAAAAAAAAIHMAAAA|51071|14|11|11|PM|first|afternoon|lunch| +51072|AAAAAAAABIHMAAAA|51072|14|11|12|PM|first|afternoon|lunch| +51073|AAAAAAAACIHMAAAA|51073|14|11|13|PM|first|afternoon|lunch| +51074|AAAAAAAADIHMAAAA|51074|14|11|14|PM|first|afternoon|lunch| +51075|AAAAAAAAEIHMAAAA|51075|14|11|15|PM|first|afternoon|lunch| +51076|AAAAAAAAFIHMAAAA|51076|14|11|16|PM|first|afternoon|lunch| +51077|AAAAAAAAGIHMAAAA|51077|14|11|17|PM|first|afternoon|lunch| +51078|AAAAAAAAHIHMAAAA|51078|14|11|18|PM|first|afternoon|lunch| +51079|AAAAAAAAIIHMAAAA|51079|14|11|19|PM|first|afternoon|lunch| +51080|AAAAAAAAJIHMAAAA|51080|14|11|20|PM|first|afternoon|lunch| +51081|AAAAAAAAKIHMAAAA|51081|14|11|21|PM|first|afternoon|lunch| +51082|AAAAAAAALIHMAAAA|51082|14|11|22|PM|first|afternoon|lunch| +51083|AAAAAAAAMIHMAAAA|51083|14|11|23|PM|first|afternoon|lunch| +51084|AAAAAAAANIHMAAAA|51084|14|11|24|PM|first|afternoon|lunch| +51085|AAAAAAAAOIHMAAAA|51085|14|11|25|PM|first|afternoon|lunch| +51086|AAAAAAAAPIHMAAAA|51086|14|11|26|PM|first|afternoon|lunch| +51087|AAAAAAAAAJHMAAAA|51087|14|11|27|PM|first|afternoon|lunch| +51088|AAAAAAAABJHMAAAA|51088|14|11|28|PM|first|afternoon|lunch| +51089|AAAAAAAACJHMAAAA|51089|14|11|29|PM|first|afternoon|lunch| +51090|AAAAAAAADJHMAAAA|51090|14|11|30|PM|first|afternoon|lunch| +51091|AAAAAAAAEJHMAAAA|51091|14|11|31|PM|first|afternoon|lunch| +51092|AAAAAAAAFJHMAAAA|51092|14|11|32|PM|first|afternoon|lunch| +51093|AAAAAAAAGJHMAAAA|51093|14|11|33|PM|first|afternoon|lunch| +51094|AAAAAAAAHJHMAAAA|51094|14|11|34|PM|first|afternoon|lunch| +51095|AAAAAAAAIJHMAAAA|51095|14|11|35|PM|first|afternoon|lunch| +51096|AAAAAAAAJJHMAAAA|51096|14|11|36|PM|first|afternoon|lunch| +51097|AAAAAAAAKJHMAAAA|51097|14|11|37|PM|first|afternoon|lunch| +51098|AAAAAAAALJHMAAAA|51098|14|11|38|PM|first|afternoon|lunch| +51099|AAAAAAAAMJHMAAAA|51099|14|11|39|PM|first|afternoon|lunch| +51100|AAAAAAAANJHMAAAA|51100|14|11|40|PM|first|afternoon|lunch| +51101|AAAAAAAAOJHMAAAA|51101|14|11|41|PM|first|afternoon|lunch| +51102|AAAAAAAAPJHMAAAA|51102|14|11|42|PM|first|afternoon|lunch| +51103|AAAAAAAAAKHMAAAA|51103|14|11|43|PM|first|afternoon|lunch| +51104|AAAAAAAABKHMAAAA|51104|14|11|44|PM|first|afternoon|lunch| +51105|AAAAAAAACKHMAAAA|51105|14|11|45|PM|first|afternoon|lunch| +51106|AAAAAAAADKHMAAAA|51106|14|11|46|PM|first|afternoon|lunch| +51107|AAAAAAAAEKHMAAAA|51107|14|11|47|PM|first|afternoon|lunch| +51108|AAAAAAAAFKHMAAAA|51108|14|11|48|PM|first|afternoon|lunch| +51109|AAAAAAAAGKHMAAAA|51109|14|11|49|PM|first|afternoon|lunch| +51110|AAAAAAAAHKHMAAAA|51110|14|11|50|PM|first|afternoon|lunch| +51111|AAAAAAAAIKHMAAAA|51111|14|11|51|PM|first|afternoon|lunch| +51112|AAAAAAAAJKHMAAAA|51112|14|11|52|PM|first|afternoon|lunch| +51113|AAAAAAAAKKHMAAAA|51113|14|11|53|PM|first|afternoon|lunch| +51114|AAAAAAAALKHMAAAA|51114|14|11|54|PM|first|afternoon|lunch| +51115|AAAAAAAAMKHMAAAA|51115|14|11|55|PM|first|afternoon|lunch| +51116|AAAAAAAANKHMAAAA|51116|14|11|56|PM|first|afternoon|lunch| +51117|AAAAAAAAOKHMAAAA|51117|14|11|57|PM|first|afternoon|lunch| +51118|AAAAAAAAPKHMAAAA|51118|14|11|58|PM|first|afternoon|lunch| +51119|AAAAAAAAALHMAAAA|51119|14|11|59|PM|first|afternoon|lunch| +51120|AAAAAAAABLHMAAAA|51120|14|12|0|PM|first|afternoon|lunch| +51121|AAAAAAAACLHMAAAA|51121|14|12|1|PM|first|afternoon|lunch| +51122|AAAAAAAADLHMAAAA|51122|14|12|2|PM|first|afternoon|lunch| +51123|AAAAAAAAELHMAAAA|51123|14|12|3|PM|first|afternoon|lunch| +51124|AAAAAAAAFLHMAAAA|51124|14|12|4|PM|first|afternoon|lunch| +51125|AAAAAAAAGLHMAAAA|51125|14|12|5|PM|first|afternoon|lunch| +51126|AAAAAAAAHLHMAAAA|51126|14|12|6|PM|first|afternoon|lunch| +51127|AAAAAAAAILHMAAAA|51127|14|12|7|PM|first|afternoon|lunch| +51128|AAAAAAAAJLHMAAAA|51128|14|12|8|PM|first|afternoon|lunch| +51129|AAAAAAAAKLHMAAAA|51129|14|12|9|PM|first|afternoon|lunch| +51130|AAAAAAAALLHMAAAA|51130|14|12|10|PM|first|afternoon|lunch| +51131|AAAAAAAAMLHMAAAA|51131|14|12|11|PM|first|afternoon|lunch| +51132|AAAAAAAANLHMAAAA|51132|14|12|12|PM|first|afternoon|lunch| +51133|AAAAAAAAOLHMAAAA|51133|14|12|13|PM|first|afternoon|lunch| +51134|AAAAAAAAPLHMAAAA|51134|14|12|14|PM|first|afternoon|lunch| +51135|AAAAAAAAAMHMAAAA|51135|14|12|15|PM|first|afternoon|lunch| +51136|AAAAAAAABMHMAAAA|51136|14|12|16|PM|first|afternoon|lunch| +51137|AAAAAAAACMHMAAAA|51137|14|12|17|PM|first|afternoon|lunch| +51138|AAAAAAAADMHMAAAA|51138|14|12|18|PM|first|afternoon|lunch| +51139|AAAAAAAAEMHMAAAA|51139|14|12|19|PM|first|afternoon|lunch| +51140|AAAAAAAAFMHMAAAA|51140|14|12|20|PM|first|afternoon|lunch| +51141|AAAAAAAAGMHMAAAA|51141|14|12|21|PM|first|afternoon|lunch| +51142|AAAAAAAAHMHMAAAA|51142|14|12|22|PM|first|afternoon|lunch| +51143|AAAAAAAAIMHMAAAA|51143|14|12|23|PM|first|afternoon|lunch| +51144|AAAAAAAAJMHMAAAA|51144|14|12|24|PM|first|afternoon|lunch| +51145|AAAAAAAAKMHMAAAA|51145|14|12|25|PM|first|afternoon|lunch| +51146|AAAAAAAALMHMAAAA|51146|14|12|26|PM|first|afternoon|lunch| +51147|AAAAAAAAMMHMAAAA|51147|14|12|27|PM|first|afternoon|lunch| +51148|AAAAAAAANMHMAAAA|51148|14|12|28|PM|first|afternoon|lunch| +51149|AAAAAAAAOMHMAAAA|51149|14|12|29|PM|first|afternoon|lunch| +51150|AAAAAAAAPMHMAAAA|51150|14|12|30|PM|first|afternoon|lunch| +51151|AAAAAAAAANHMAAAA|51151|14|12|31|PM|first|afternoon|lunch| +51152|AAAAAAAABNHMAAAA|51152|14|12|32|PM|first|afternoon|lunch| +51153|AAAAAAAACNHMAAAA|51153|14|12|33|PM|first|afternoon|lunch| +51154|AAAAAAAADNHMAAAA|51154|14|12|34|PM|first|afternoon|lunch| +51155|AAAAAAAAENHMAAAA|51155|14|12|35|PM|first|afternoon|lunch| +51156|AAAAAAAAFNHMAAAA|51156|14|12|36|PM|first|afternoon|lunch| +51157|AAAAAAAAGNHMAAAA|51157|14|12|37|PM|first|afternoon|lunch| +51158|AAAAAAAAHNHMAAAA|51158|14|12|38|PM|first|afternoon|lunch| +51159|AAAAAAAAINHMAAAA|51159|14|12|39|PM|first|afternoon|lunch| +51160|AAAAAAAAJNHMAAAA|51160|14|12|40|PM|first|afternoon|lunch| +51161|AAAAAAAAKNHMAAAA|51161|14|12|41|PM|first|afternoon|lunch| +51162|AAAAAAAALNHMAAAA|51162|14|12|42|PM|first|afternoon|lunch| +51163|AAAAAAAAMNHMAAAA|51163|14|12|43|PM|first|afternoon|lunch| +51164|AAAAAAAANNHMAAAA|51164|14|12|44|PM|first|afternoon|lunch| +51165|AAAAAAAAONHMAAAA|51165|14|12|45|PM|first|afternoon|lunch| +51166|AAAAAAAAPNHMAAAA|51166|14|12|46|PM|first|afternoon|lunch| +51167|AAAAAAAAAOHMAAAA|51167|14|12|47|PM|first|afternoon|lunch| +51168|AAAAAAAABOHMAAAA|51168|14|12|48|PM|first|afternoon|lunch| +51169|AAAAAAAACOHMAAAA|51169|14|12|49|PM|first|afternoon|lunch| +51170|AAAAAAAADOHMAAAA|51170|14|12|50|PM|first|afternoon|lunch| +51171|AAAAAAAAEOHMAAAA|51171|14|12|51|PM|first|afternoon|lunch| +51172|AAAAAAAAFOHMAAAA|51172|14|12|52|PM|first|afternoon|lunch| +51173|AAAAAAAAGOHMAAAA|51173|14|12|53|PM|first|afternoon|lunch| +51174|AAAAAAAAHOHMAAAA|51174|14|12|54|PM|first|afternoon|lunch| +51175|AAAAAAAAIOHMAAAA|51175|14|12|55|PM|first|afternoon|lunch| +51176|AAAAAAAAJOHMAAAA|51176|14|12|56|PM|first|afternoon|lunch| +51177|AAAAAAAAKOHMAAAA|51177|14|12|57|PM|first|afternoon|lunch| +51178|AAAAAAAALOHMAAAA|51178|14|12|58|PM|first|afternoon|lunch| +51179|AAAAAAAAMOHMAAAA|51179|14|12|59|PM|first|afternoon|lunch| +51180|AAAAAAAANOHMAAAA|51180|14|13|0|PM|first|afternoon|lunch| +51181|AAAAAAAAOOHMAAAA|51181|14|13|1|PM|first|afternoon|lunch| +51182|AAAAAAAAPOHMAAAA|51182|14|13|2|PM|first|afternoon|lunch| +51183|AAAAAAAAAPHMAAAA|51183|14|13|3|PM|first|afternoon|lunch| +51184|AAAAAAAABPHMAAAA|51184|14|13|4|PM|first|afternoon|lunch| +51185|AAAAAAAACPHMAAAA|51185|14|13|5|PM|first|afternoon|lunch| +51186|AAAAAAAADPHMAAAA|51186|14|13|6|PM|first|afternoon|lunch| +51187|AAAAAAAAEPHMAAAA|51187|14|13|7|PM|first|afternoon|lunch| +51188|AAAAAAAAFPHMAAAA|51188|14|13|8|PM|first|afternoon|lunch| +51189|AAAAAAAAGPHMAAAA|51189|14|13|9|PM|first|afternoon|lunch| +51190|AAAAAAAAHPHMAAAA|51190|14|13|10|PM|first|afternoon|lunch| +51191|AAAAAAAAIPHMAAAA|51191|14|13|11|PM|first|afternoon|lunch| +51192|AAAAAAAAJPHMAAAA|51192|14|13|12|PM|first|afternoon|lunch| +51193|AAAAAAAAKPHMAAAA|51193|14|13|13|PM|first|afternoon|lunch| +51194|AAAAAAAALPHMAAAA|51194|14|13|14|PM|first|afternoon|lunch| +51195|AAAAAAAAMPHMAAAA|51195|14|13|15|PM|first|afternoon|lunch| +51196|AAAAAAAANPHMAAAA|51196|14|13|16|PM|first|afternoon|lunch| +51197|AAAAAAAAOPHMAAAA|51197|14|13|17|PM|first|afternoon|lunch| +51198|AAAAAAAAPPHMAAAA|51198|14|13|18|PM|first|afternoon|lunch| +51199|AAAAAAAAAAIMAAAA|51199|14|13|19|PM|first|afternoon|lunch| +51200|AAAAAAAABAIMAAAA|51200|14|13|20|PM|first|afternoon|lunch| +51201|AAAAAAAACAIMAAAA|51201|14|13|21|PM|first|afternoon|lunch| +51202|AAAAAAAADAIMAAAA|51202|14|13|22|PM|first|afternoon|lunch| +51203|AAAAAAAAEAIMAAAA|51203|14|13|23|PM|first|afternoon|lunch| +51204|AAAAAAAAFAIMAAAA|51204|14|13|24|PM|first|afternoon|lunch| +51205|AAAAAAAAGAIMAAAA|51205|14|13|25|PM|first|afternoon|lunch| +51206|AAAAAAAAHAIMAAAA|51206|14|13|26|PM|first|afternoon|lunch| +51207|AAAAAAAAIAIMAAAA|51207|14|13|27|PM|first|afternoon|lunch| +51208|AAAAAAAAJAIMAAAA|51208|14|13|28|PM|first|afternoon|lunch| +51209|AAAAAAAAKAIMAAAA|51209|14|13|29|PM|first|afternoon|lunch| +51210|AAAAAAAALAIMAAAA|51210|14|13|30|PM|first|afternoon|lunch| +51211|AAAAAAAAMAIMAAAA|51211|14|13|31|PM|first|afternoon|lunch| +51212|AAAAAAAANAIMAAAA|51212|14|13|32|PM|first|afternoon|lunch| +51213|AAAAAAAAOAIMAAAA|51213|14|13|33|PM|first|afternoon|lunch| +51214|AAAAAAAAPAIMAAAA|51214|14|13|34|PM|first|afternoon|lunch| +51215|AAAAAAAAABIMAAAA|51215|14|13|35|PM|first|afternoon|lunch| +51216|AAAAAAAABBIMAAAA|51216|14|13|36|PM|first|afternoon|lunch| +51217|AAAAAAAACBIMAAAA|51217|14|13|37|PM|first|afternoon|lunch| +51218|AAAAAAAADBIMAAAA|51218|14|13|38|PM|first|afternoon|lunch| +51219|AAAAAAAAEBIMAAAA|51219|14|13|39|PM|first|afternoon|lunch| +51220|AAAAAAAAFBIMAAAA|51220|14|13|40|PM|first|afternoon|lunch| +51221|AAAAAAAAGBIMAAAA|51221|14|13|41|PM|first|afternoon|lunch| +51222|AAAAAAAAHBIMAAAA|51222|14|13|42|PM|first|afternoon|lunch| +51223|AAAAAAAAIBIMAAAA|51223|14|13|43|PM|first|afternoon|lunch| +51224|AAAAAAAAJBIMAAAA|51224|14|13|44|PM|first|afternoon|lunch| +51225|AAAAAAAAKBIMAAAA|51225|14|13|45|PM|first|afternoon|lunch| +51226|AAAAAAAALBIMAAAA|51226|14|13|46|PM|first|afternoon|lunch| +51227|AAAAAAAAMBIMAAAA|51227|14|13|47|PM|first|afternoon|lunch| +51228|AAAAAAAANBIMAAAA|51228|14|13|48|PM|first|afternoon|lunch| +51229|AAAAAAAAOBIMAAAA|51229|14|13|49|PM|first|afternoon|lunch| +51230|AAAAAAAAPBIMAAAA|51230|14|13|50|PM|first|afternoon|lunch| +51231|AAAAAAAAACIMAAAA|51231|14|13|51|PM|first|afternoon|lunch| +51232|AAAAAAAABCIMAAAA|51232|14|13|52|PM|first|afternoon|lunch| +51233|AAAAAAAACCIMAAAA|51233|14|13|53|PM|first|afternoon|lunch| +51234|AAAAAAAADCIMAAAA|51234|14|13|54|PM|first|afternoon|lunch| +51235|AAAAAAAAECIMAAAA|51235|14|13|55|PM|first|afternoon|lunch| +51236|AAAAAAAAFCIMAAAA|51236|14|13|56|PM|first|afternoon|lunch| +51237|AAAAAAAAGCIMAAAA|51237|14|13|57|PM|first|afternoon|lunch| +51238|AAAAAAAAHCIMAAAA|51238|14|13|58|PM|first|afternoon|lunch| +51239|AAAAAAAAICIMAAAA|51239|14|13|59|PM|first|afternoon|lunch| +51240|AAAAAAAAJCIMAAAA|51240|14|14|0|PM|first|afternoon|lunch| +51241|AAAAAAAAKCIMAAAA|51241|14|14|1|PM|first|afternoon|lunch| +51242|AAAAAAAALCIMAAAA|51242|14|14|2|PM|first|afternoon|lunch| +51243|AAAAAAAAMCIMAAAA|51243|14|14|3|PM|first|afternoon|lunch| +51244|AAAAAAAANCIMAAAA|51244|14|14|4|PM|first|afternoon|lunch| +51245|AAAAAAAAOCIMAAAA|51245|14|14|5|PM|first|afternoon|lunch| +51246|AAAAAAAAPCIMAAAA|51246|14|14|6|PM|first|afternoon|lunch| +51247|AAAAAAAAADIMAAAA|51247|14|14|7|PM|first|afternoon|lunch| +51248|AAAAAAAABDIMAAAA|51248|14|14|8|PM|first|afternoon|lunch| +51249|AAAAAAAACDIMAAAA|51249|14|14|9|PM|first|afternoon|lunch| +51250|AAAAAAAADDIMAAAA|51250|14|14|10|PM|first|afternoon|lunch| +51251|AAAAAAAAEDIMAAAA|51251|14|14|11|PM|first|afternoon|lunch| +51252|AAAAAAAAFDIMAAAA|51252|14|14|12|PM|first|afternoon|lunch| +51253|AAAAAAAAGDIMAAAA|51253|14|14|13|PM|first|afternoon|lunch| +51254|AAAAAAAAHDIMAAAA|51254|14|14|14|PM|first|afternoon|lunch| +51255|AAAAAAAAIDIMAAAA|51255|14|14|15|PM|first|afternoon|lunch| +51256|AAAAAAAAJDIMAAAA|51256|14|14|16|PM|first|afternoon|lunch| +51257|AAAAAAAAKDIMAAAA|51257|14|14|17|PM|first|afternoon|lunch| +51258|AAAAAAAALDIMAAAA|51258|14|14|18|PM|first|afternoon|lunch| +51259|AAAAAAAAMDIMAAAA|51259|14|14|19|PM|first|afternoon|lunch| +51260|AAAAAAAANDIMAAAA|51260|14|14|20|PM|first|afternoon|lunch| +51261|AAAAAAAAODIMAAAA|51261|14|14|21|PM|first|afternoon|lunch| +51262|AAAAAAAAPDIMAAAA|51262|14|14|22|PM|first|afternoon|lunch| +51263|AAAAAAAAAEIMAAAA|51263|14|14|23|PM|first|afternoon|lunch| +51264|AAAAAAAABEIMAAAA|51264|14|14|24|PM|first|afternoon|lunch| +51265|AAAAAAAACEIMAAAA|51265|14|14|25|PM|first|afternoon|lunch| +51266|AAAAAAAADEIMAAAA|51266|14|14|26|PM|first|afternoon|lunch| +51267|AAAAAAAAEEIMAAAA|51267|14|14|27|PM|first|afternoon|lunch| +51268|AAAAAAAAFEIMAAAA|51268|14|14|28|PM|first|afternoon|lunch| +51269|AAAAAAAAGEIMAAAA|51269|14|14|29|PM|first|afternoon|lunch| +51270|AAAAAAAAHEIMAAAA|51270|14|14|30|PM|first|afternoon|lunch| +51271|AAAAAAAAIEIMAAAA|51271|14|14|31|PM|first|afternoon|lunch| +51272|AAAAAAAAJEIMAAAA|51272|14|14|32|PM|first|afternoon|lunch| +51273|AAAAAAAAKEIMAAAA|51273|14|14|33|PM|first|afternoon|lunch| +51274|AAAAAAAALEIMAAAA|51274|14|14|34|PM|first|afternoon|lunch| +51275|AAAAAAAAMEIMAAAA|51275|14|14|35|PM|first|afternoon|lunch| +51276|AAAAAAAANEIMAAAA|51276|14|14|36|PM|first|afternoon|lunch| +51277|AAAAAAAAOEIMAAAA|51277|14|14|37|PM|first|afternoon|lunch| +51278|AAAAAAAAPEIMAAAA|51278|14|14|38|PM|first|afternoon|lunch| +51279|AAAAAAAAAFIMAAAA|51279|14|14|39|PM|first|afternoon|lunch| +51280|AAAAAAAABFIMAAAA|51280|14|14|40|PM|first|afternoon|lunch| +51281|AAAAAAAACFIMAAAA|51281|14|14|41|PM|first|afternoon|lunch| +51282|AAAAAAAADFIMAAAA|51282|14|14|42|PM|first|afternoon|lunch| +51283|AAAAAAAAEFIMAAAA|51283|14|14|43|PM|first|afternoon|lunch| +51284|AAAAAAAAFFIMAAAA|51284|14|14|44|PM|first|afternoon|lunch| +51285|AAAAAAAAGFIMAAAA|51285|14|14|45|PM|first|afternoon|lunch| +51286|AAAAAAAAHFIMAAAA|51286|14|14|46|PM|first|afternoon|lunch| +51287|AAAAAAAAIFIMAAAA|51287|14|14|47|PM|first|afternoon|lunch| +51288|AAAAAAAAJFIMAAAA|51288|14|14|48|PM|first|afternoon|lunch| +51289|AAAAAAAAKFIMAAAA|51289|14|14|49|PM|first|afternoon|lunch| +51290|AAAAAAAALFIMAAAA|51290|14|14|50|PM|first|afternoon|lunch| +51291|AAAAAAAAMFIMAAAA|51291|14|14|51|PM|first|afternoon|lunch| +51292|AAAAAAAANFIMAAAA|51292|14|14|52|PM|first|afternoon|lunch| +51293|AAAAAAAAOFIMAAAA|51293|14|14|53|PM|first|afternoon|lunch| +51294|AAAAAAAAPFIMAAAA|51294|14|14|54|PM|first|afternoon|lunch| +51295|AAAAAAAAAGIMAAAA|51295|14|14|55|PM|first|afternoon|lunch| +51296|AAAAAAAABGIMAAAA|51296|14|14|56|PM|first|afternoon|lunch| +51297|AAAAAAAACGIMAAAA|51297|14|14|57|PM|first|afternoon|lunch| +51298|AAAAAAAADGIMAAAA|51298|14|14|58|PM|first|afternoon|lunch| +51299|AAAAAAAAEGIMAAAA|51299|14|14|59|PM|first|afternoon|lunch| +51300|AAAAAAAAFGIMAAAA|51300|14|15|0|PM|first|afternoon|lunch| +51301|AAAAAAAAGGIMAAAA|51301|14|15|1|PM|first|afternoon|lunch| +51302|AAAAAAAAHGIMAAAA|51302|14|15|2|PM|first|afternoon|lunch| +51303|AAAAAAAAIGIMAAAA|51303|14|15|3|PM|first|afternoon|lunch| +51304|AAAAAAAAJGIMAAAA|51304|14|15|4|PM|first|afternoon|lunch| +51305|AAAAAAAAKGIMAAAA|51305|14|15|5|PM|first|afternoon|lunch| +51306|AAAAAAAALGIMAAAA|51306|14|15|6|PM|first|afternoon|lunch| +51307|AAAAAAAAMGIMAAAA|51307|14|15|7|PM|first|afternoon|lunch| +51308|AAAAAAAANGIMAAAA|51308|14|15|8|PM|first|afternoon|lunch| +51309|AAAAAAAAOGIMAAAA|51309|14|15|9|PM|first|afternoon|lunch| +51310|AAAAAAAAPGIMAAAA|51310|14|15|10|PM|first|afternoon|lunch| +51311|AAAAAAAAAHIMAAAA|51311|14|15|11|PM|first|afternoon|lunch| +51312|AAAAAAAABHIMAAAA|51312|14|15|12|PM|first|afternoon|lunch| +51313|AAAAAAAACHIMAAAA|51313|14|15|13|PM|first|afternoon|lunch| +51314|AAAAAAAADHIMAAAA|51314|14|15|14|PM|first|afternoon|lunch| +51315|AAAAAAAAEHIMAAAA|51315|14|15|15|PM|first|afternoon|lunch| +51316|AAAAAAAAFHIMAAAA|51316|14|15|16|PM|first|afternoon|lunch| +51317|AAAAAAAAGHIMAAAA|51317|14|15|17|PM|first|afternoon|lunch| +51318|AAAAAAAAHHIMAAAA|51318|14|15|18|PM|first|afternoon|lunch| +51319|AAAAAAAAIHIMAAAA|51319|14|15|19|PM|first|afternoon|lunch| +51320|AAAAAAAAJHIMAAAA|51320|14|15|20|PM|first|afternoon|lunch| +51321|AAAAAAAAKHIMAAAA|51321|14|15|21|PM|first|afternoon|lunch| +51322|AAAAAAAALHIMAAAA|51322|14|15|22|PM|first|afternoon|lunch| +51323|AAAAAAAAMHIMAAAA|51323|14|15|23|PM|first|afternoon|lunch| +51324|AAAAAAAANHIMAAAA|51324|14|15|24|PM|first|afternoon|lunch| +51325|AAAAAAAAOHIMAAAA|51325|14|15|25|PM|first|afternoon|lunch| +51326|AAAAAAAAPHIMAAAA|51326|14|15|26|PM|first|afternoon|lunch| +51327|AAAAAAAAAIIMAAAA|51327|14|15|27|PM|first|afternoon|lunch| +51328|AAAAAAAABIIMAAAA|51328|14|15|28|PM|first|afternoon|lunch| +51329|AAAAAAAACIIMAAAA|51329|14|15|29|PM|first|afternoon|lunch| +51330|AAAAAAAADIIMAAAA|51330|14|15|30|PM|first|afternoon|lunch| +51331|AAAAAAAAEIIMAAAA|51331|14|15|31|PM|first|afternoon|lunch| +51332|AAAAAAAAFIIMAAAA|51332|14|15|32|PM|first|afternoon|lunch| +51333|AAAAAAAAGIIMAAAA|51333|14|15|33|PM|first|afternoon|lunch| +51334|AAAAAAAAHIIMAAAA|51334|14|15|34|PM|first|afternoon|lunch| +51335|AAAAAAAAIIIMAAAA|51335|14|15|35|PM|first|afternoon|lunch| +51336|AAAAAAAAJIIMAAAA|51336|14|15|36|PM|first|afternoon|lunch| +51337|AAAAAAAAKIIMAAAA|51337|14|15|37|PM|first|afternoon|lunch| +51338|AAAAAAAALIIMAAAA|51338|14|15|38|PM|first|afternoon|lunch| +51339|AAAAAAAAMIIMAAAA|51339|14|15|39|PM|first|afternoon|lunch| +51340|AAAAAAAANIIMAAAA|51340|14|15|40|PM|first|afternoon|lunch| +51341|AAAAAAAAOIIMAAAA|51341|14|15|41|PM|first|afternoon|lunch| +51342|AAAAAAAAPIIMAAAA|51342|14|15|42|PM|first|afternoon|lunch| +51343|AAAAAAAAAJIMAAAA|51343|14|15|43|PM|first|afternoon|lunch| +51344|AAAAAAAABJIMAAAA|51344|14|15|44|PM|first|afternoon|lunch| +51345|AAAAAAAACJIMAAAA|51345|14|15|45|PM|first|afternoon|lunch| +51346|AAAAAAAADJIMAAAA|51346|14|15|46|PM|first|afternoon|lunch| +51347|AAAAAAAAEJIMAAAA|51347|14|15|47|PM|first|afternoon|lunch| +51348|AAAAAAAAFJIMAAAA|51348|14|15|48|PM|first|afternoon|lunch| +51349|AAAAAAAAGJIMAAAA|51349|14|15|49|PM|first|afternoon|lunch| +51350|AAAAAAAAHJIMAAAA|51350|14|15|50|PM|first|afternoon|lunch| +51351|AAAAAAAAIJIMAAAA|51351|14|15|51|PM|first|afternoon|lunch| +51352|AAAAAAAAJJIMAAAA|51352|14|15|52|PM|first|afternoon|lunch| +51353|AAAAAAAAKJIMAAAA|51353|14|15|53|PM|first|afternoon|lunch| +51354|AAAAAAAALJIMAAAA|51354|14|15|54|PM|first|afternoon|lunch| +51355|AAAAAAAAMJIMAAAA|51355|14|15|55|PM|first|afternoon|lunch| +51356|AAAAAAAANJIMAAAA|51356|14|15|56|PM|first|afternoon|lunch| +51357|AAAAAAAAOJIMAAAA|51357|14|15|57|PM|first|afternoon|lunch| +51358|AAAAAAAAPJIMAAAA|51358|14|15|58|PM|first|afternoon|lunch| +51359|AAAAAAAAAKIMAAAA|51359|14|15|59|PM|first|afternoon|lunch| +51360|AAAAAAAABKIMAAAA|51360|14|16|0|PM|first|afternoon|lunch| +51361|AAAAAAAACKIMAAAA|51361|14|16|1|PM|first|afternoon|lunch| +51362|AAAAAAAADKIMAAAA|51362|14|16|2|PM|first|afternoon|lunch| +51363|AAAAAAAAEKIMAAAA|51363|14|16|3|PM|first|afternoon|lunch| +51364|AAAAAAAAFKIMAAAA|51364|14|16|4|PM|first|afternoon|lunch| +51365|AAAAAAAAGKIMAAAA|51365|14|16|5|PM|first|afternoon|lunch| +51366|AAAAAAAAHKIMAAAA|51366|14|16|6|PM|first|afternoon|lunch| +51367|AAAAAAAAIKIMAAAA|51367|14|16|7|PM|first|afternoon|lunch| +51368|AAAAAAAAJKIMAAAA|51368|14|16|8|PM|first|afternoon|lunch| +51369|AAAAAAAAKKIMAAAA|51369|14|16|9|PM|first|afternoon|lunch| +51370|AAAAAAAALKIMAAAA|51370|14|16|10|PM|first|afternoon|lunch| +51371|AAAAAAAAMKIMAAAA|51371|14|16|11|PM|first|afternoon|lunch| +51372|AAAAAAAANKIMAAAA|51372|14|16|12|PM|first|afternoon|lunch| +51373|AAAAAAAAOKIMAAAA|51373|14|16|13|PM|first|afternoon|lunch| +51374|AAAAAAAAPKIMAAAA|51374|14|16|14|PM|first|afternoon|lunch| +51375|AAAAAAAAALIMAAAA|51375|14|16|15|PM|first|afternoon|lunch| +51376|AAAAAAAABLIMAAAA|51376|14|16|16|PM|first|afternoon|lunch| +51377|AAAAAAAACLIMAAAA|51377|14|16|17|PM|first|afternoon|lunch| +51378|AAAAAAAADLIMAAAA|51378|14|16|18|PM|first|afternoon|lunch| +51379|AAAAAAAAELIMAAAA|51379|14|16|19|PM|first|afternoon|lunch| +51380|AAAAAAAAFLIMAAAA|51380|14|16|20|PM|first|afternoon|lunch| +51381|AAAAAAAAGLIMAAAA|51381|14|16|21|PM|first|afternoon|lunch| +51382|AAAAAAAAHLIMAAAA|51382|14|16|22|PM|first|afternoon|lunch| +51383|AAAAAAAAILIMAAAA|51383|14|16|23|PM|first|afternoon|lunch| +51384|AAAAAAAAJLIMAAAA|51384|14|16|24|PM|first|afternoon|lunch| +51385|AAAAAAAAKLIMAAAA|51385|14|16|25|PM|first|afternoon|lunch| +51386|AAAAAAAALLIMAAAA|51386|14|16|26|PM|first|afternoon|lunch| +51387|AAAAAAAAMLIMAAAA|51387|14|16|27|PM|first|afternoon|lunch| +51388|AAAAAAAANLIMAAAA|51388|14|16|28|PM|first|afternoon|lunch| +51389|AAAAAAAAOLIMAAAA|51389|14|16|29|PM|first|afternoon|lunch| +51390|AAAAAAAAPLIMAAAA|51390|14|16|30|PM|first|afternoon|lunch| +51391|AAAAAAAAAMIMAAAA|51391|14|16|31|PM|first|afternoon|lunch| +51392|AAAAAAAABMIMAAAA|51392|14|16|32|PM|first|afternoon|lunch| +51393|AAAAAAAACMIMAAAA|51393|14|16|33|PM|first|afternoon|lunch| +51394|AAAAAAAADMIMAAAA|51394|14|16|34|PM|first|afternoon|lunch| +51395|AAAAAAAAEMIMAAAA|51395|14|16|35|PM|first|afternoon|lunch| +51396|AAAAAAAAFMIMAAAA|51396|14|16|36|PM|first|afternoon|lunch| +51397|AAAAAAAAGMIMAAAA|51397|14|16|37|PM|first|afternoon|lunch| +51398|AAAAAAAAHMIMAAAA|51398|14|16|38|PM|first|afternoon|lunch| +51399|AAAAAAAAIMIMAAAA|51399|14|16|39|PM|first|afternoon|lunch| +51400|AAAAAAAAJMIMAAAA|51400|14|16|40|PM|first|afternoon|lunch| +51401|AAAAAAAAKMIMAAAA|51401|14|16|41|PM|first|afternoon|lunch| +51402|AAAAAAAALMIMAAAA|51402|14|16|42|PM|first|afternoon|lunch| +51403|AAAAAAAAMMIMAAAA|51403|14|16|43|PM|first|afternoon|lunch| +51404|AAAAAAAANMIMAAAA|51404|14|16|44|PM|first|afternoon|lunch| +51405|AAAAAAAAOMIMAAAA|51405|14|16|45|PM|first|afternoon|lunch| +51406|AAAAAAAAPMIMAAAA|51406|14|16|46|PM|first|afternoon|lunch| +51407|AAAAAAAAANIMAAAA|51407|14|16|47|PM|first|afternoon|lunch| +51408|AAAAAAAABNIMAAAA|51408|14|16|48|PM|first|afternoon|lunch| +51409|AAAAAAAACNIMAAAA|51409|14|16|49|PM|first|afternoon|lunch| +51410|AAAAAAAADNIMAAAA|51410|14|16|50|PM|first|afternoon|lunch| +51411|AAAAAAAAENIMAAAA|51411|14|16|51|PM|first|afternoon|lunch| +51412|AAAAAAAAFNIMAAAA|51412|14|16|52|PM|first|afternoon|lunch| +51413|AAAAAAAAGNIMAAAA|51413|14|16|53|PM|first|afternoon|lunch| +51414|AAAAAAAAHNIMAAAA|51414|14|16|54|PM|first|afternoon|lunch| +51415|AAAAAAAAINIMAAAA|51415|14|16|55|PM|first|afternoon|lunch| +51416|AAAAAAAAJNIMAAAA|51416|14|16|56|PM|first|afternoon|lunch| +51417|AAAAAAAAKNIMAAAA|51417|14|16|57|PM|first|afternoon|lunch| +51418|AAAAAAAALNIMAAAA|51418|14|16|58|PM|first|afternoon|lunch| +51419|AAAAAAAAMNIMAAAA|51419|14|16|59|PM|first|afternoon|lunch| +51420|AAAAAAAANNIMAAAA|51420|14|17|0|PM|first|afternoon|lunch| +51421|AAAAAAAAONIMAAAA|51421|14|17|1|PM|first|afternoon|lunch| +51422|AAAAAAAAPNIMAAAA|51422|14|17|2|PM|first|afternoon|lunch| +51423|AAAAAAAAAOIMAAAA|51423|14|17|3|PM|first|afternoon|lunch| +51424|AAAAAAAABOIMAAAA|51424|14|17|4|PM|first|afternoon|lunch| +51425|AAAAAAAACOIMAAAA|51425|14|17|5|PM|first|afternoon|lunch| +51426|AAAAAAAADOIMAAAA|51426|14|17|6|PM|first|afternoon|lunch| +51427|AAAAAAAAEOIMAAAA|51427|14|17|7|PM|first|afternoon|lunch| +51428|AAAAAAAAFOIMAAAA|51428|14|17|8|PM|first|afternoon|lunch| +51429|AAAAAAAAGOIMAAAA|51429|14|17|9|PM|first|afternoon|lunch| +51430|AAAAAAAAHOIMAAAA|51430|14|17|10|PM|first|afternoon|lunch| +51431|AAAAAAAAIOIMAAAA|51431|14|17|11|PM|first|afternoon|lunch| +51432|AAAAAAAAJOIMAAAA|51432|14|17|12|PM|first|afternoon|lunch| +51433|AAAAAAAAKOIMAAAA|51433|14|17|13|PM|first|afternoon|lunch| +51434|AAAAAAAALOIMAAAA|51434|14|17|14|PM|first|afternoon|lunch| +51435|AAAAAAAAMOIMAAAA|51435|14|17|15|PM|first|afternoon|lunch| +51436|AAAAAAAANOIMAAAA|51436|14|17|16|PM|first|afternoon|lunch| +51437|AAAAAAAAOOIMAAAA|51437|14|17|17|PM|first|afternoon|lunch| +51438|AAAAAAAAPOIMAAAA|51438|14|17|18|PM|first|afternoon|lunch| +51439|AAAAAAAAAPIMAAAA|51439|14|17|19|PM|first|afternoon|lunch| +51440|AAAAAAAABPIMAAAA|51440|14|17|20|PM|first|afternoon|lunch| +51441|AAAAAAAACPIMAAAA|51441|14|17|21|PM|first|afternoon|lunch| +51442|AAAAAAAADPIMAAAA|51442|14|17|22|PM|first|afternoon|lunch| +51443|AAAAAAAAEPIMAAAA|51443|14|17|23|PM|first|afternoon|lunch| +51444|AAAAAAAAFPIMAAAA|51444|14|17|24|PM|first|afternoon|lunch| +51445|AAAAAAAAGPIMAAAA|51445|14|17|25|PM|first|afternoon|lunch| +51446|AAAAAAAAHPIMAAAA|51446|14|17|26|PM|first|afternoon|lunch| +51447|AAAAAAAAIPIMAAAA|51447|14|17|27|PM|first|afternoon|lunch| +51448|AAAAAAAAJPIMAAAA|51448|14|17|28|PM|first|afternoon|lunch| +51449|AAAAAAAAKPIMAAAA|51449|14|17|29|PM|first|afternoon|lunch| +51450|AAAAAAAALPIMAAAA|51450|14|17|30|PM|first|afternoon|lunch| +51451|AAAAAAAAMPIMAAAA|51451|14|17|31|PM|first|afternoon|lunch| +51452|AAAAAAAANPIMAAAA|51452|14|17|32|PM|first|afternoon|lunch| +51453|AAAAAAAAOPIMAAAA|51453|14|17|33|PM|first|afternoon|lunch| +51454|AAAAAAAAPPIMAAAA|51454|14|17|34|PM|first|afternoon|lunch| +51455|AAAAAAAAAAJMAAAA|51455|14|17|35|PM|first|afternoon|lunch| +51456|AAAAAAAABAJMAAAA|51456|14|17|36|PM|first|afternoon|lunch| +51457|AAAAAAAACAJMAAAA|51457|14|17|37|PM|first|afternoon|lunch| +51458|AAAAAAAADAJMAAAA|51458|14|17|38|PM|first|afternoon|lunch| +51459|AAAAAAAAEAJMAAAA|51459|14|17|39|PM|first|afternoon|lunch| +51460|AAAAAAAAFAJMAAAA|51460|14|17|40|PM|first|afternoon|lunch| +51461|AAAAAAAAGAJMAAAA|51461|14|17|41|PM|first|afternoon|lunch| +51462|AAAAAAAAHAJMAAAA|51462|14|17|42|PM|first|afternoon|lunch| +51463|AAAAAAAAIAJMAAAA|51463|14|17|43|PM|first|afternoon|lunch| +51464|AAAAAAAAJAJMAAAA|51464|14|17|44|PM|first|afternoon|lunch| +51465|AAAAAAAAKAJMAAAA|51465|14|17|45|PM|first|afternoon|lunch| +51466|AAAAAAAALAJMAAAA|51466|14|17|46|PM|first|afternoon|lunch| +51467|AAAAAAAAMAJMAAAA|51467|14|17|47|PM|first|afternoon|lunch| +51468|AAAAAAAANAJMAAAA|51468|14|17|48|PM|first|afternoon|lunch| +51469|AAAAAAAAOAJMAAAA|51469|14|17|49|PM|first|afternoon|lunch| +51470|AAAAAAAAPAJMAAAA|51470|14|17|50|PM|first|afternoon|lunch| +51471|AAAAAAAAABJMAAAA|51471|14|17|51|PM|first|afternoon|lunch| +51472|AAAAAAAABBJMAAAA|51472|14|17|52|PM|first|afternoon|lunch| +51473|AAAAAAAACBJMAAAA|51473|14|17|53|PM|first|afternoon|lunch| +51474|AAAAAAAADBJMAAAA|51474|14|17|54|PM|first|afternoon|lunch| +51475|AAAAAAAAEBJMAAAA|51475|14|17|55|PM|first|afternoon|lunch| +51476|AAAAAAAAFBJMAAAA|51476|14|17|56|PM|first|afternoon|lunch| +51477|AAAAAAAAGBJMAAAA|51477|14|17|57|PM|first|afternoon|lunch| +51478|AAAAAAAAHBJMAAAA|51478|14|17|58|PM|first|afternoon|lunch| +51479|AAAAAAAAIBJMAAAA|51479|14|17|59|PM|first|afternoon|lunch| +51480|AAAAAAAAJBJMAAAA|51480|14|18|0|PM|first|afternoon|lunch| +51481|AAAAAAAAKBJMAAAA|51481|14|18|1|PM|first|afternoon|lunch| +51482|AAAAAAAALBJMAAAA|51482|14|18|2|PM|first|afternoon|lunch| +51483|AAAAAAAAMBJMAAAA|51483|14|18|3|PM|first|afternoon|lunch| +51484|AAAAAAAANBJMAAAA|51484|14|18|4|PM|first|afternoon|lunch| +51485|AAAAAAAAOBJMAAAA|51485|14|18|5|PM|first|afternoon|lunch| +51486|AAAAAAAAPBJMAAAA|51486|14|18|6|PM|first|afternoon|lunch| +51487|AAAAAAAAACJMAAAA|51487|14|18|7|PM|first|afternoon|lunch| +51488|AAAAAAAABCJMAAAA|51488|14|18|8|PM|first|afternoon|lunch| +51489|AAAAAAAACCJMAAAA|51489|14|18|9|PM|first|afternoon|lunch| +51490|AAAAAAAADCJMAAAA|51490|14|18|10|PM|first|afternoon|lunch| +51491|AAAAAAAAECJMAAAA|51491|14|18|11|PM|first|afternoon|lunch| +51492|AAAAAAAAFCJMAAAA|51492|14|18|12|PM|first|afternoon|lunch| +51493|AAAAAAAAGCJMAAAA|51493|14|18|13|PM|first|afternoon|lunch| +51494|AAAAAAAAHCJMAAAA|51494|14|18|14|PM|first|afternoon|lunch| +51495|AAAAAAAAICJMAAAA|51495|14|18|15|PM|first|afternoon|lunch| +51496|AAAAAAAAJCJMAAAA|51496|14|18|16|PM|first|afternoon|lunch| +51497|AAAAAAAAKCJMAAAA|51497|14|18|17|PM|first|afternoon|lunch| +51498|AAAAAAAALCJMAAAA|51498|14|18|18|PM|first|afternoon|lunch| +51499|AAAAAAAAMCJMAAAA|51499|14|18|19|PM|first|afternoon|lunch| +51500|AAAAAAAANCJMAAAA|51500|14|18|20|PM|first|afternoon|lunch| +51501|AAAAAAAAOCJMAAAA|51501|14|18|21|PM|first|afternoon|lunch| +51502|AAAAAAAAPCJMAAAA|51502|14|18|22|PM|first|afternoon|lunch| +51503|AAAAAAAAADJMAAAA|51503|14|18|23|PM|first|afternoon|lunch| +51504|AAAAAAAABDJMAAAA|51504|14|18|24|PM|first|afternoon|lunch| +51505|AAAAAAAACDJMAAAA|51505|14|18|25|PM|first|afternoon|lunch| +51506|AAAAAAAADDJMAAAA|51506|14|18|26|PM|first|afternoon|lunch| +51507|AAAAAAAAEDJMAAAA|51507|14|18|27|PM|first|afternoon|lunch| +51508|AAAAAAAAFDJMAAAA|51508|14|18|28|PM|first|afternoon|lunch| +51509|AAAAAAAAGDJMAAAA|51509|14|18|29|PM|first|afternoon|lunch| +51510|AAAAAAAAHDJMAAAA|51510|14|18|30|PM|first|afternoon|lunch| +51511|AAAAAAAAIDJMAAAA|51511|14|18|31|PM|first|afternoon|lunch| +51512|AAAAAAAAJDJMAAAA|51512|14|18|32|PM|first|afternoon|lunch| +51513|AAAAAAAAKDJMAAAA|51513|14|18|33|PM|first|afternoon|lunch| +51514|AAAAAAAALDJMAAAA|51514|14|18|34|PM|first|afternoon|lunch| +51515|AAAAAAAAMDJMAAAA|51515|14|18|35|PM|first|afternoon|lunch| +51516|AAAAAAAANDJMAAAA|51516|14|18|36|PM|first|afternoon|lunch| +51517|AAAAAAAAODJMAAAA|51517|14|18|37|PM|first|afternoon|lunch| +51518|AAAAAAAAPDJMAAAA|51518|14|18|38|PM|first|afternoon|lunch| +51519|AAAAAAAAAEJMAAAA|51519|14|18|39|PM|first|afternoon|lunch| +51520|AAAAAAAABEJMAAAA|51520|14|18|40|PM|first|afternoon|lunch| +51521|AAAAAAAACEJMAAAA|51521|14|18|41|PM|first|afternoon|lunch| +51522|AAAAAAAADEJMAAAA|51522|14|18|42|PM|first|afternoon|lunch| +51523|AAAAAAAAEEJMAAAA|51523|14|18|43|PM|first|afternoon|lunch| +51524|AAAAAAAAFEJMAAAA|51524|14|18|44|PM|first|afternoon|lunch| +51525|AAAAAAAAGEJMAAAA|51525|14|18|45|PM|first|afternoon|lunch| +51526|AAAAAAAAHEJMAAAA|51526|14|18|46|PM|first|afternoon|lunch| +51527|AAAAAAAAIEJMAAAA|51527|14|18|47|PM|first|afternoon|lunch| +51528|AAAAAAAAJEJMAAAA|51528|14|18|48|PM|first|afternoon|lunch| +51529|AAAAAAAAKEJMAAAA|51529|14|18|49|PM|first|afternoon|lunch| +51530|AAAAAAAALEJMAAAA|51530|14|18|50|PM|first|afternoon|lunch| +51531|AAAAAAAAMEJMAAAA|51531|14|18|51|PM|first|afternoon|lunch| +51532|AAAAAAAANEJMAAAA|51532|14|18|52|PM|first|afternoon|lunch| +51533|AAAAAAAAOEJMAAAA|51533|14|18|53|PM|first|afternoon|lunch| +51534|AAAAAAAAPEJMAAAA|51534|14|18|54|PM|first|afternoon|lunch| +51535|AAAAAAAAAFJMAAAA|51535|14|18|55|PM|first|afternoon|lunch| +51536|AAAAAAAABFJMAAAA|51536|14|18|56|PM|first|afternoon|lunch| +51537|AAAAAAAACFJMAAAA|51537|14|18|57|PM|first|afternoon|lunch| +51538|AAAAAAAADFJMAAAA|51538|14|18|58|PM|first|afternoon|lunch| +51539|AAAAAAAAEFJMAAAA|51539|14|18|59|PM|first|afternoon|lunch| +51540|AAAAAAAAFFJMAAAA|51540|14|19|0|PM|first|afternoon|lunch| +51541|AAAAAAAAGFJMAAAA|51541|14|19|1|PM|first|afternoon|lunch| +51542|AAAAAAAAHFJMAAAA|51542|14|19|2|PM|first|afternoon|lunch| +51543|AAAAAAAAIFJMAAAA|51543|14|19|3|PM|first|afternoon|lunch| +51544|AAAAAAAAJFJMAAAA|51544|14|19|4|PM|first|afternoon|lunch| +51545|AAAAAAAAKFJMAAAA|51545|14|19|5|PM|first|afternoon|lunch| +51546|AAAAAAAALFJMAAAA|51546|14|19|6|PM|first|afternoon|lunch| +51547|AAAAAAAAMFJMAAAA|51547|14|19|7|PM|first|afternoon|lunch| +51548|AAAAAAAANFJMAAAA|51548|14|19|8|PM|first|afternoon|lunch| +51549|AAAAAAAAOFJMAAAA|51549|14|19|9|PM|first|afternoon|lunch| +51550|AAAAAAAAPFJMAAAA|51550|14|19|10|PM|first|afternoon|lunch| +51551|AAAAAAAAAGJMAAAA|51551|14|19|11|PM|first|afternoon|lunch| +51552|AAAAAAAABGJMAAAA|51552|14|19|12|PM|first|afternoon|lunch| +51553|AAAAAAAACGJMAAAA|51553|14|19|13|PM|first|afternoon|lunch| +51554|AAAAAAAADGJMAAAA|51554|14|19|14|PM|first|afternoon|lunch| +51555|AAAAAAAAEGJMAAAA|51555|14|19|15|PM|first|afternoon|lunch| +51556|AAAAAAAAFGJMAAAA|51556|14|19|16|PM|first|afternoon|lunch| +51557|AAAAAAAAGGJMAAAA|51557|14|19|17|PM|first|afternoon|lunch| +51558|AAAAAAAAHGJMAAAA|51558|14|19|18|PM|first|afternoon|lunch| +51559|AAAAAAAAIGJMAAAA|51559|14|19|19|PM|first|afternoon|lunch| +51560|AAAAAAAAJGJMAAAA|51560|14|19|20|PM|first|afternoon|lunch| +51561|AAAAAAAAKGJMAAAA|51561|14|19|21|PM|first|afternoon|lunch| +51562|AAAAAAAALGJMAAAA|51562|14|19|22|PM|first|afternoon|lunch| +51563|AAAAAAAAMGJMAAAA|51563|14|19|23|PM|first|afternoon|lunch| +51564|AAAAAAAANGJMAAAA|51564|14|19|24|PM|first|afternoon|lunch| +51565|AAAAAAAAOGJMAAAA|51565|14|19|25|PM|first|afternoon|lunch| +51566|AAAAAAAAPGJMAAAA|51566|14|19|26|PM|first|afternoon|lunch| +51567|AAAAAAAAAHJMAAAA|51567|14|19|27|PM|first|afternoon|lunch| +51568|AAAAAAAABHJMAAAA|51568|14|19|28|PM|first|afternoon|lunch| +51569|AAAAAAAACHJMAAAA|51569|14|19|29|PM|first|afternoon|lunch| +51570|AAAAAAAADHJMAAAA|51570|14|19|30|PM|first|afternoon|lunch| +51571|AAAAAAAAEHJMAAAA|51571|14|19|31|PM|first|afternoon|lunch| +51572|AAAAAAAAFHJMAAAA|51572|14|19|32|PM|first|afternoon|lunch| +51573|AAAAAAAAGHJMAAAA|51573|14|19|33|PM|first|afternoon|lunch| +51574|AAAAAAAAHHJMAAAA|51574|14|19|34|PM|first|afternoon|lunch| +51575|AAAAAAAAIHJMAAAA|51575|14|19|35|PM|first|afternoon|lunch| +51576|AAAAAAAAJHJMAAAA|51576|14|19|36|PM|first|afternoon|lunch| +51577|AAAAAAAAKHJMAAAA|51577|14|19|37|PM|first|afternoon|lunch| +51578|AAAAAAAALHJMAAAA|51578|14|19|38|PM|first|afternoon|lunch| +51579|AAAAAAAAMHJMAAAA|51579|14|19|39|PM|first|afternoon|lunch| +51580|AAAAAAAANHJMAAAA|51580|14|19|40|PM|first|afternoon|lunch| +51581|AAAAAAAAOHJMAAAA|51581|14|19|41|PM|first|afternoon|lunch| +51582|AAAAAAAAPHJMAAAA|51582|14|19|42|PM|first|afternoon|lunch| +51583|AAAAAAAAAIJMAAAA|51583|14|19|43|PM|first|afternoon|lunch| +51584|AAAAAAAABIJMAAAA|51584|14|19|44|PM|first|afternoon|lunch| +51585|AAAAAAAACIJMAAAA|51585|14|19|45|PM|first|afternoon|lunch| +51586|AAAAAAAADIJMAAAA|51586|14|19|46|PM|first|afternoon|lunch| +51587|AAAAAAAAEIJMAAAA|51587|14|19|47|PM|first|afternoon|lunch| +51588|AAAAAAAAFIJMAAAA|51588|14|19|48|PM|first|afternoon|lunch| +51589|AAAAAAAAGIJMAAAA|51589|14|19|49|PM|first|afternoon|lunch| +51590|AAAAAAAAHIJMAAAA|51590|14|19|50|PM|first|afternoon|lunch| +51591|AAAAAAAAIIJMAAAA|51591|14|19|51|PM|first|afternoon|lunch| +51592|AAAAAAAAJIJMAAAA|51592|14|19|52|PM|first|afternoon|lunch| +51593|AAAAAAAAKIJMAAAA|51593|14|19|53|PM|first|afternoon|lunch| +51594|AAAAAAAALIJMAAAA|51594|14|19|54|PM|first|afternoon|lunch| +51595|AAAAAAAAMIJMAAAA|51595|14|19|55|PM|first|afternoon|lunch| +51596|AAAAAAAANIJMAAAA|51596|14|19|56|PM|first|afternoon|lunch| +51597|AAAAAAAAOIJMAAAA|51597|14|19|57|PM|first|afternoon|lunch| +51598|AAAAAAAAPIJMAAAA|51598|14|19|58|PM|first|afternoon|lunch| +51599|AAAAAAAAAJJMAAAA|51599|14|19|59|PM|first|afternoon|lunch| +51600|AAAAAAAABJJMAAAA|51600|14|20|0|PM|first|afternoon|lunch| +51601|AAAAAAAACJJMAAAA|51601|14|20|1|PM|first|afternoon|lunch| +51602|AAAAAAAADJJMAAAA|51602|14|20|2|PM|first|afternoon|lunch| +51603|AAAAAAAAEJJMAAAA|51603|14|20|3|PM|first|afternoon|lunch| +51604|AAAAAAAAFJJMAAAA|51604|14|20|4|PM|first|afternoon|lunch| +51605|AAAAAAAAGJJMAAAA|51605|14|20|5|PM|first|afternoon|lunch| +51606|AAAAAAAAHJJMAAAA|51606|14|20|6|PM|first|afternoon|lunch| +51607|AAAAAAAAIJJMAAAA|51607|14|20|7|PM|first|afternoon|lunch| +51608|AAAAAAAAJJJMAAAA|51608|14|20|8|PM|first|afternoon|lunch| +51609|AAAAAAAAKJJMAAAA|51609|14|20|9|PM|first|afternoon|lunch| +51610|AAAAAAAALJJMAAAA|51610|14|20|10|PM|first|afternoon|lunch| +51611|AAAAAAAAMJJMAAAA|51611|14|20|11|PM|first|afternoon|lunch| +51612|AAAAAAAANJJMAAAA|51612|14|20|12|PM|first|afternoon|lunch| +51613|AAAAAAAAOJJMAAAA|51613|14|20|13|PM|first|afternoon|lunch| +51614|AAAAAAAAPJJMAAAA|51614|14|20|14|PM|first|afternoon|lunch| +51615|AAAAAAAAAKJMAAAA|51615|14|20|15|PM|first|afternoon|lunch| +51616|AAAAAAAABKJMAAAA|51616|14|20|16|PM|first|afternoon|lunch| +51617|AAAAAAAACKJMAAAA|51617|14|20|17|PM|first|afternoon|lunch| +51618|AAAAAAAADKJMAAAA|51618|14|20|18|PM|first|afternoon|lunch| +51619|AAAAAAAAEKJMAAAA|51619|14|20|19|PM|first|afternoon|lunch| +51620|AAAAAAAAFKJMAAAA|51620|14|20|20|PM|first|afternoon|lunch| +51621|AAAAAAAAGKJMAAAA|51621|14|20|21|PM|first|afternoon|lunch| +51622|AAAAAAAAHKJMAAAA|51622|14|20|22|PM|first|afternoon|lunch| +51623|AAAAAAAAIKJMAAAA|51623|14|20|23|PM|first|afternoon|lunch| +51624|AAAAAAAAJKJMAAAA|51624|14|20|24|PM|first|afternoon|lunch| +51625|AAAAAAAAKKJMAAAA|51625|14|20|25|PM|first|afternoon|lunch| +51626|AAAAAAAALKJMAAAA|51626|14|20|26|PM|first|afternoon|lunch| +51627|AAAAAAAAMKJMAAAA|51627|14|20|27|PM|first|afternoon|lunch| +51628|AAAAAAAANKJMAAAA|51628|14|20|28|PM|first|afternoon|lunch| +51629|AAAAAAAAOKJMAAAA|51629|14|20|29|PM|first|afternoon|lunch| +51630|AAAAAAAAPKJMAAAA|51630|14|20|30|PM|first|afternoon|lunch| +51631|AAAAAAAAALJMAAAA|51631|14|20|31|PM|first|afternoon|lunch| +51632|AAAAAAAABLJMAAAA|51632|14|20|32|PM|first|afternoon|lunch| +51633|AAAAAAAACLJMAAAA|51633|14|20|33|PM|first|afternoon|lunch| +51634|AAAAAAAADLJMAAAA|51634|14|20|34|PM|first|afternoon|lunch| +51635|AAAAAAAAELJMAAAA|51635|14|20|35|PM|first|afternoon|lunch| +51636|AAAAAAAAFLJMAAAA|51636|14|20|36|PM|first|afternoon|lunch| +51637|AAAAAAAAGLJMAAAA|51637|14|20|37|PM|first|afternoon|lunch| +51638|AAAAAAAAHLJMAAAA|51638|14|20|38|PM|first|afternoon|lunch| +51639|AAAAAAAAILJMAAAA|51639|14|20|39|PM|first|afternoon|lunch| +51640|AAAAAAAAJLJMAAAA|51640|14|20|40|PM|first|afternoon|lunch| +51641|AAAAAAAAKLJMAAAA|51641|14|20|41|PM|first|afternoon|lunch| +51642|AAAAAAAALLJMAAAA|51642|14|20|42|PM|first|afternoon|lunch| +51643|AAAAAAAAMLJMAAAA|51643|14|20|43|PM|first|afternoon|lunch| +51644|AAAAAAAANLJMAAAA|51644|14|20|44|PM|first|afternoon|lunch| +51645|AAAAAAAAOLJMAAAA|51645|14|20|45|PM|first|afternoon|lunch| +51646|AAAAAAAAPLJMAAAA|51646|14|20|46|PM|first|afternoon|lunch| +51647|AAAAAAAAAMJMAAAA|51647|14|20|47|PM|first|afternoon|lunch| +51648|AAAAAAAABMJMAAAA|51648|14|20|48|PM|first|afternoon|lunch| +51649|AAAAAAAACMJMAAAA|51649|14|20|49|PM|first|afternoon|lunch| +51650|AAAAAAAADMJMAAAA|51650|14|20|50|PM|first|afternoon|lunch| +51651|AAAAAAAAEMJMAAAA|51651|14|20|51|PM|first|afternoon|lunch| +51652|AAAAAAAAFMJMAAAA|51652|14|20|52|PM|first|afternoon|lunch| +51653|AAAAAAAAGMJMAAAA|51653|14|20|53|PM|first|afternoon|lunch| +51654|AAAAAAAAHMJMAAAA|51654|14|20|54|PM|first|afternoon|lunch| +51655|AAAAAAAAIMJMAAAA|51655|14|20|55|PM|first|afternoon|lunch| +51656|AAAAAAAAJMJMAAAA|51656|14|20|56|PM|first|afternoon|lunch| +51657|AAAAAAAAKMJMAAAA|51657|14|20|57|PM|first|afternoon|lunch| +51658|AAAAAAAALMJMAAAA|51658|14|20|58|PM|first|afternoon|lunch| +51659|AAAAAAAAMMJMAAAA|51659|14|20|59|PM|first|afternoon|lunch| +51660|AAAAAAAANMJMAAAA|51660|14|21|0|PM|first|afternoon|lunch| +51661|AAAAAAAAOMJMAAAA|51661|14|21|1|PM|first|afternoon|lunch| +51662|AAAAAAAAPMJMAAAA|51662|14|21|2|PM|first|afternoon|lunch| +51663|AAAAAAAAANJMAAAA|51663|14|21|3|PM|first|afternoon|lunch| +51664|AAAAAAAABNJMAAAA|51664|14|21|4|PM|first|afternoon|lunch| +51665|AAAAAAAACNJMAAAA|51665|14|21|5|PM|first|afternoon|lunch| +51666|AAAAAAAADNJMAAAA|51666|14|21|6|PM|first|afternoon|lunch| +51667|AAAAAAAAENJMAAAA|51667|14|21|7|PM|first|afternoon|lunch| +51668|AAAAAAAAFNJMAAAA|51668|14|21|8|PM|first|afternoon|lunch| +51669|AAAAAAAAGNJMAAAA|51669|14|21|9|PM|first|afternoon|lunch| +51670|AAAAAAAAHNJMAAAA|51670|14|21|10|PM|first|afternoon|lunch| +51671|AAAAAAAAINJMAAAA|51671|14|21|11|PM|first|afternoon|lunch| +51672|AAAAAAAAJNJMAAAA|51672|14|21|12|PM|first|afternoon|lunch| +51673|AAAAAAAAKNJMAAAA|51673|14|21|13|PM|first|afternoon|lunch| +51674|AAAAAAAALNJMAAAA|51674|14|21|14|PM|first|afternoon|lunch| +51675|AAAAAAAAMNJMAAAA|51675|14|21|15|PM|first|afternoon|lunch| +51676|AAAAAAAANNJMAAAA|51676|14|21|16|PM|first|afternoon|lunch| +51677|AAAAAAAAONJMAAAA|51677|14|21|17|PM|first|afternoon|lunch| +51678|AAAAAAAAPNJMAAAA|51678|14|21|18|PM|first|afternoon|lunch| +51679|AAAAAAAAAOJMAAAA|51679|14|21|19|PM|first|afternoon|lunch| +51680|AAAAAAAABOJMAAAA|51680|14|21|20|PM|first|afternoon|lunch| +51681|AAAAAAAACOJMAAAA|51681|14|21|21|PM|first|afternoon|lunch| +51682|AAAAAAAADOJMAAAA|51682|14|21|22|PM|first|afternoon|lunch| +51683|AAAAAAAAEOJMAAAA|51683|14|21|23|PM|first|afternoon|lunch| +51684|AAAAAAAAFOJMAAAA|51684|14|21|24|PM|first|afternoon|lunch| +51685|AAAAAAAAGOJMAAAA|51685|14|21|25|PM|first|afternoon|lunch| +51686|AAAAAAAAHOJMAAAA|51686|14|21|26|PM|first|afternoon|lunch| +51687|AAAAAAAAIOJMAAAA|51687|14|21|27|PM|first|afternoon|lunch| +51688|AAAAAAAAJOJMAAAA|51688|14|21|28|PM|first|afternoon|lunch| +51689|AAAAAAAAKOJMAAAA|51689|14|21|29|PM|first|afternoon|lunch| +51690|AAAAAAAALOJMAAAA|51690|14|21|30|PM|first|afternoon|lunch| +51691|AAAAAAAAMOJMAAAA|51691|14|21|31|PM|first|afternoon|lunch| +51692|AAAAAAAANOJMAAAA|51692|14|21|32|PM|first|afternoon|lunch| +51693|AAAAAAAAOOJMAAAA|51693|14|21|33|PM|first|afternoon|lunch| +51694|AAAAAAAAPOJMAAAA|51694|14|21|34|PM|first|afternoon|lunch| +51695|AAAAAAAAAPJMAAAA|51695|14|21|35|PM|first|afternoon|lunch| +51696|AAAAAAAABPJMAAAA|51696|14|21|36|PM|first|afternoon|lunch| +51697|AAAAAAAACPJMAAAA|51697|14|21|37|PM|first|afternoon|lunch| +51698|AAAAAAAADPJMAAAA|51698|14|21|38|PM|first|afternoon|lunch| +51699|AAAAAAAAEPJMAAAA|51699|14|21|39|PM|first|afternoon|lunch| +51700|AAAAAAAAFPJMAAAA|51700|14|21|40|PM|first|afternoon|lunch| +51701|AAAAAAAAGPJMAAAA|51701|14|21|41|PM|first|afternoon|lunch| +51702|AAAAAAAAHPJMAAAA|51702|14|21|42|PM|first|afternoon|lunch| +51703|AAAAAAAAIPJMAAAA|51703|14|21|43|PM|first|afternoon|lunch| +51704|AAAAAAAAJPJMAAAA|51704|14|21|44|PM|first|afternoon|lunch| +51705|AAAAAAAAKPJMAAAA|51705|14|21|45|PM|first|afternoon|lunch| +51706|AAAAAAAALPJMAAAA|51706|14|21|46|PM|first|afternoon|lunch| +51707|AAAAAAAAMPJMAAAA|51707|14|21|47|PM|first|afternoon|lunch| +51708|AAAAAAAANPJMAAAA|51708|14|21|48|PM|first|afternoon|lunch| +51709|AAAAAAAAOPJMAAAA|51709|14|21|49|PM|first|afternoon|lunch| +51710|AAAAAAAAPPJMAAAA|51710|14|21|50|PM|first|afternoon|lunch| +51711|AAAAAAAAAAKMAAAA|51711|14|21|51|PM|first|afternoon|lunch| +51712|AAAAAAAABAKMAAAA|51712|14|21|52|PM|first|afternoon|lunch| +51713|AAAAAAAACAKMAAAA|51713|14|21|53|PM|first|afternoon|lunch| +51714|AAAAAAAADAKMAAAA|51714|14|21|54|PM|first|afternoon|lunch| +51715|AAAAAAAAEAKMAAAA|51715|14|21|55|PM|first|afternoon|lunch| +51716|AAAAAAAAFAKMAAAA|51716|14|21|56|PM|first|afternoon|lunch| +51717|AAAAAAAAGAKMAAAA|51717|14|21|57|PM|first|afternoon|lunch| +51718|AAAAAAAAHAKMAAAA|51718|14|21|58|PM|first|afternoon|lunch| +51719|AAAAAAAAIAKMAAAA|51719|14|21|59|PM|first|afternoon|lunch| +51720|AAAAAAAAJAKMAAAA|51720|14|22|0|PM|first|afternoon|lunch| +51721|AAAAAAAAKAKMAAAA|51721|14|22|1|PM|first|afternoon|lunch| +51722|AAAAAAAALAKMAAAA|51722|14|22|2|PM|first|afternoon|lunch| +51723|AAAAAAAAMAKMAAAA|51723|14|22|3|PM|first|afternoon|lunch| +51724|AAAAAAAANAKMAAAA|51724|14|22|4|PM|first|afternoon|lunch| +51725|AAAAAAAAOAKMAAAA|51725|14|22|5|PM|first|afternoon|lunch| +51726|AAAAAAAAPAKMAAAA|51726|14|22|6|PM|first|afternoon|lunch| +51727|AAAAAAAAABKMAAAA|51727|14|22|7|PM|first|afternoon|lunch| +51728|AAAAAAAABBKMAAAA|51728|14|22|8|PM|first|afternoon|lunch| +51729|AAAAAAAACBKMAAAA|51729|14|22|9|PM|first|afternoon|lunch| +51730|AAAAAAAADBKMAAAA|51730|14|22|10|PM|first|afternoon|lunch| +51731|AAAAAAAAEBKMAAAA|51731|14|22|11|PM|first|afternoon|lunch| +51732|AAAAAAAAFBKMAAAA|51732|14|22|12|PM|first|afternoon|lunch| +51733|AAAAAAAAGBKMAAAA|51733|14|22|13|PM|first|afternoon|lunch| +51734|AAAAAAAAHBKMAAAA|51734|14|22|14|PM|first|afternoon|lunch| +51735|AAAAAAAAIBKMAAAA|51735|14|22|15|PM|first|afternoon|lunch| +51736|AAAAAAAAJBKMAAAA|51736|14|22|16|PM|first|afternoon|lunch| +51737|AAAAAAAAKBKMAAAA|51737|14|22|17|PM|first|afternoon|lunch| +51738|AAAAAAAALBKMAAAA|51738|14|22|18|PM|first|afternoon|lunch| +51739|AAAAAAAAMBKMAAAA|51739|14|22|19|PM|first|afternoon|lunch| +51740|AAAAAAAANBKMAAAA|51740|14|22|20|PM|first|afternoon|lunch| +51741|AAAAAAAAOBKMAAAA|51741|14|22|21|PM|first|afternoon|lunch| +51742|AAAAAAAAPBKMAAAA|51742|14|22|22|PM|first|afternoon|lunch| +51743|AAAAAAAAACKMAAAA|51743|14|22|23|PM|first|afternoon|lunch| +51744|AAAAAAAABCKMAAAA|51744|14|22|24|PM|first|afternoon|lunch| +51745|AAAAAAAACCKMAAAA|51745|14|22|25|PM|first|afternoon|lunch| +51746|AAAAAAAADCKMAAAA|51746|14|22|26|PM|first|afternoon|lunch| +51747|AAAAAAAAECKMAAAA|51747|14|22|27|PM|first|afternoon|lunch| +51748|AAAAAAAAFCKMAAAA|51748|14|22|28|PM|first|afternoon|lunch| +51749|AAAAAAAAGCKMAAAA|51749|14|22|29|PM|first|afternoon|lunch| +51750|AAAAAAAAHCKMAAAA|51750|14|22|30|PM|first|afternoon|lunch| +51751|AAAAAAAAICKMAAAA|51751|14|22|31|PM|first|afternoon|lunch| +51752|AAAAAAAAJCKMAAAA|51752|14|22|32|PM|first|afternoon|lunch| +51753|AAAAAAAAKCKMAAAA|51753|14|22|33|PM|first|afternoon|lunch| +51754|AAAAAAAALCKMAAAA|51754|14|22|34|PM|first|afternoon|lunch| +51755|AAAAAAAAMCKMAAAA|51755|14|22|35|PM|first|afternoon|lunch| +51756|AAAAAAAANCKMAAAA|51756|14|22|36|PM|first|afternoon|lunch| +51757|AAAAAAAAOCKMAAAA|51757|14|22|37|PM|first|afternoon|lunch| +51758|AAAAAAAAPCKMAAAA|51758|14|22|38|PM|first|afternoon|lunch| +51759|AAAAAAAAADKMAAAA|51759|14|22|39|PM|first|afternoon|lunch| +51760|AAAAAAAABDKMAAAA|51760|14|22|40|PM|first|afternoon|lunch| +51761|AAAAAAAACDKMAAAA|51761|14|22|41|PM|first|afternoon|lunch| +51762|AAAAAAAADDKMAAAA|51762|14|22|42|PM|first|afternoon|lunch| +51763|AAAAAAAAEDKMAAAA|51763|14|22|43|PM|first|afternoon|lunch| +51764|AAAAAAAAFDKMAAAA|51764|14|22|44|PM|first|afternoon|lunch| +51765|AAAAAAAAGDKMAAAA|51765|14|22|45|PM|first|afternoon|lunch| +51766|AAAAAAAAHDKMAAAA|51766|14|22|46|PM|first|afternoon|lunch| +51767|AAAAAAAAIDKMAAAA|51767|14|22|47|PM|first|afternoon|lunch| +51768|AAAAAAAAJDKMAAAA|51768|14|22|48|PM|first|afternoon|lunch| +51769|AAAAAAAAKDKMAAAA|51769|14|22|49|PM|first|afternoon|lunch| +51770|AAAAAAAALDKMAAAA|51770|14|22|50|PM|first|afternoon|lunch| +51771|AAAAAAAAMDKMAAAA|51771|14|22|51|PM|first|afternoon|lunch| +51772|AAAAAAAANDKMAAAA|51772|14|22|52|PM|first|afternoon|lunch| +51773|AAAAAAAAODKMAAAA|51773|14|22|53|PM|first|afternoon|lunch| +51774|AAAAAAAAPDKMAAAA|51774|14|22|54|PM|first|afternoon|lunch| +51775|AAAAAAAAAEKMAAAA|51775|14|22|55|PM|first|afternoon|lunch| +51776|AAAAAAAABEKMAAAA|51776|14|22|56|PM|first|afternoon|lunch| +51777|AAAAAAAACEKMAAAA|51777|14|22|57|PM|first|afternoon|lunch| +51778|AAAAAAAADEKMAAAA|51778|14|22|58|PM|first|afternoon|lunch| +51779|AAAAAAAAEEKMAAAA|51779|14|22|59|PM|first|afternoon|lunch| +51780|AAAAAAAAFEKMAAAA|51780|14|23|0|PM|first|afternoon|lunch| +51781|AAAAAAAAGEKMAAAA|51781|14|23|1|PM|first|afternoon|lunch| +51782|AAAAAAAAHEKMAAAA|51782|14|23|2|PM|first|afternoon|lunch| +51783|AAAAAAAAIEKMAAAA|51783|14|23|3|PM|first|afternoon|lunch| +51784|AAAAAAAAJEKMAAAA|51784|14|23|4|PM|first|afternoon|lunch| +51785|AAAAAAAAKEKMAAAA|51785|14|23|5|PM|first|afternoon|lunch| +51786|AAAAAAAALEKMAAAA|51786|14|23|6|PM|first|afternoon|lunch| +51787|AAAAAAAAMEKMAAAA|51787|14|23|7|PM|first|afternoon|lunch| +51788|AAAAAAAANEKMAAAA|51788|14|23|8|PM|first|afternoon|lunch| +51789|AAAAAAAAOEKMAAAA|51789|14|23|9|PM|first|afternoon|lunch| +51790|AAAAAAAAPEKMAAAA|51790|14|23|10|PM|first|afternoon|lunch| +51791|AAAAAAAAAFKMAAAA|51791|14|23|11|PM|first|afternoon|lunch| +51792|AAAAAAAABFKMAAAA|51792|14|23|12|PM|first|afternoon|lunch| +51793|AAAAAAAACFKMAAAA|51793|14|23|13|PM|first|afternoon|lunch| +51794|AAAAAAAADFKMAAAA|51794|14|23|14|PM|first|afternoon|lunch| +51795|AAAAAAAAEFKMAAAA|51795|14|23|15|PM|first|afternoon|lunch| +51796|AAAAAAAAFFKMAAAA|51796|14|23|16|PM|first|afternoon|lunch| +51797|AAAAAAAAGFKMAAAA|51797|14|23|17|PM|first|afternoon|lunch| +51798|AAAAAAAAHFKMAAAA|51798|14|23|18|PM|first|afternoon|lunch| +51799|AAAAAAAAIFKMAAAA|51799|14|23|19|PM|first|afternoon|lunch| +51800|AAAAAAAAJFKMAAAA|51800|14|23|20|PM|first|afternoon|lunch| +51801|AAAAAAAAKFKMAAAA|51801|14|23|21|PM|first|afternoon|lunch| +51802|AAAAAAAALFKMAAAA|51802|14|23|22|PM|first|afternoon|lunch| +51803|AAAAAAAAMFKMAAAA|51803|14|23|23|PM|first|afternoon|lunch| +51804|AAAAAAAANFKMAAAA|51804|14|23|24|PM|first|afternoon|lunch| +51805|AAAAAAAAOFKMAAAA|51805|14|23|25|PM|first|afternoon|lunch| +51806|AAAAAAAAPFKMAAAA|51806|14|23|26|PM|first|afternoon|lunch| +51807|AAAAAAAAAGKMAAAA|51807|14|23|27|PM|first|afternoon|lunch| +51808|AAAAAAAABGKMAAAA|51808|14|23|28|PM|first|afternoon|lunch| +51809|AAAAAAAACGKMAAAA|51809|14|23|29|PM|first|afternoon|lunch| +51810|AAAAAAAADGKMAAAA|51810|14|23|30|PM|first|afternoon|lunch| +51811|AAAAAAAAEGKMAAAA|51811|14|23|31|PM|first|afternoon|lunch| +51812|AAAAAAAAFGKMAAAA|51812|14|23|32|PM|first|afternoon|lunch| +51813|AAAAAAAAGGKMAAAA|51813|14|23|33|PM|first|afternoon|lunch| +51814|AAAAAAAAHGKMAAAA|51814|14|23|34|PM|first|afternoon|lunch| +51815|AAAAAAAAIGKMAAAA|51815|14|23|35|PM|first|afternoon|lunch| +51816|AAAAAAAAJGKMAAAA|51816|14|23|36|PM|first|afternoon|lunch| +51817|AAAAAAAAKGKMAAAA|51817|14|23|37|PM|first|afternoon|lunch| +51818|AAAAAAAALGKMAAAA|51818|14|23|38|PM|first|afternoon|lunch| +51819|AAAAAAAAMGKMAAAA|51819|14|23|39|PM|first|afternoon|lunch| +51820|AAAAAAAANGKMAAAA|51820|14|23|40|PM|first|afternoon|lunch| +51821|AAAAAAAAOGKMAAAA|51821|14|23|41|PM|first|afternoon|lunch| +51822|AAAAAAAAPGKMAAAA|51822|14|23|42|PM|first|afternoon|lunch| +51823|AAAAAAAAAHKMAAAA|51823|14|23|43|PM|first|afternoon|lunch| +51824|AAAAAAAABHKMAAAA|51824|14|23|44|PM|first|afternoon|lunch| +51825|AAAAAAAACHKMAAAA|51825|14|23|45|PM|first|afternoon|lunch| +51826|AAAAAAAADHKMAAAA|51826|14|23|46|PM|first|afternoon|lunch| +51827|AAAAAAAAEHKMAAAA|51827|14|23|47|PM|first|afternoon|lunch| +51828|AAAAAAAAFHKMAAAA|51828|14|23|48|PM|first|afternoon|lunch| +51829|AAAAAAAAGHKMAAAA|51829|14|23|49|PM|first|afternoon|lunch| +51830|AAAAAAAAHHKMAAAA|51830|14|23|50|PM|first|afternoon|lunch| +51831|AAAAAAAAIHKMAAAA|51831|14|23|51|PM|first|afternoon|lunch| +51832|AAAAAAAAJHKMAAAA|51832|14|23|52|PM|first|afternoon|lunch| +51833|AAAAAAAAKHKMAAAA|51833|14|23|53|PM|first|afternoon|lunch| +51834|AAAAAAAALHKMAAAA|51834|14|23|54|PM|first|afternoon|lunch| +51835|AAAAAAAAMHKMAAAA|51835|14|23|55|PM|first|afternoon|lunch| +51836|AAAAAAAANHKMAAAA|51836|14|23|56|PM|first|afternoon|lunch| +51837|AAAAAAAAOHKMAAAA|51837|14|23|57|PM|first|afternoon|lunch| +51838|AAAAAAAAPHKMAAAA|51838|14|23|58|PM|first|afternoon|lunch| +51839|AAAAAAAAAIKMAAAA|51839|14|23|59|PM|first|afternoon|lunch| +51840|AAAAAAAABIKMAAAA|51840|14|24|0|PM|first|afternoon|lunch| +51841|AAAAAAAACIKMAAAA|51841|14|24|1|PM|first|afternoon|lunch| +51842|AAAAAAAADIKMAAAA|51842|14|24|2|PM|first|afternoon|lunch| +51843|AAAAAAAAEIKMAAAA|51843|14|24|3|PM|first|afternoon|lunch| +51844|AAAAAAAAFIKMAAAA|51844|14|24|4|PM|first|afternoon|lunch| +51845|AAAAAAAAGIKMAAAA|51845|14|24|5|PM|first|afternoon|lunch| +51846|AAAAAAAAHIKMAAAA|51846|14|24|6|PM|first|afternoon|lunch| +51847|AAAAAAAAIIKMAAAA|51847|14|24|7|PM|first|afternoon|lunch| +51848|AAAAAAAAJIKMAAAA|51848|14|24|8|PM|first|afternoon|lunch| +51849|AAAAAAAAKIKMAAAA|51849|14|24|9|PM|first|afternoon|lunch| +51850|AAAAAAAALIKMAAAA|51850|14|24|10|PM|first|afternoon|lunch| +51851|AAAAAAAAMIKMAAAA|51851|14|24|11|PM|first|afternoon|lunch| +51852|AAAAAAAANIKMAAAA|51852|14|24|12|PM|first|afternoon|lunch| +51853|AAAAAAAAOIKMAAAA|51853|14|24|13|PM|first|afternoon|lunch| +51854|AAAAAAAAPIKMAAAA|51854|14|24|14|PM|first|afternoon|lunch| +51855|AAAAAAAAAJKMAAAA|51855|14|24|15|PM|first|afternoon|lunch| +51856|AAAAAAAABJKMAAAA|51856|14|24|16|PM|first|afternoon|lunch| +51857|AAAAAAAACJKMAAAA|51857|14|24|17|PM|first|afternoon|lunch| +51858|AAAAAAAADJKMAAAA|51858|14|24|18|PM|first|afternoon|lunch| +51859|AAAAAAAAEJKMAAAA|51859|14|24|19|PM|first|afternoon|lunch| +51860|AAAAAAAAFJKMAAAA|51860|14|24|20|PM|first|afternoon|lunch| +51861|AAAAAAAAGJKMAAAA|51861|14|24|21|PM|first|afternoon|lunch| +51862|AAAAAAAAHJKMAAAA|51862|14|24|22|PM|first|afternoon|lunch| +51863|AAAAAAAAIJKMAAAA|51863|14|24|23|PM|first|afternoon|lunch| +51864|AAAAAAAAJJKMAAAA|51864|14|24|24|PM|first|afternoon|lunch| +51865|AAAAAAAAKJKMAAAA|51865|14|24|25|PM|first|afternoon|lunch| +51866|AAAAAAAALJKMAAAA|51866|14|24|26|PM|first|afternoon|lunch| +51867|AAAAAAAAMJKMAAAA|51867|14|24|27|PM|first|afternoon|lunch| +51868|AAAAAAAANJKMAAAA|51868|14|24|28|PM|first|afternoon|lunch| +51869|AAAAAAAAOJKMAAAA|51869|14|24|29|PM|first|afternoon|lunch| +51870|AAAAAAAAPJKMAAAA|51870|14|24|30|PM|first|afternoon|lunch| +51871|AAAAAAAAAKKMAAAA|51871|14|24|31|PM|first|afternoon|lunch| +51872|AAAAAAAABKKMAAAA|51872|14|24|32|PM|first|afternoon|lunch| +51873|AAAAAAAACKKMAAAA|51873|14|24|33|PM|first|afternoon|lunch| +51874|AAAAAAAADKKMAAAA|51874|14|24|34|PM|first|afternoon|lunch| +51875|AAAAAAAAEKKMAAAA|51875|14|24|35|PM|first|afternoon|lunch| +51876|AAAAAAAAFKKMAAAA|51876|14|24|36|PM|first|afternoon|lunch| +51877|AAAAAAAAGKKMAAAA|51877|14|24|37|PM|first|afternoon|lunch| +51878|AAAAAAAAHKKMAAAA|51878|14|24|38|PM|first|afternoon|lunch| +51879|AAAAAAAAIKKMAAAA|51879|14|24|39|PM|first|afternoon|lunch| +51880|AAAAAAAAJKKMAAAA|51880|14|24|40|PM|first|afternoon|lunch| +51881|AAAAAAAAKKKMAAAA|51881|14|24|41|PM|first|afternoon|lunch| +51882|AAAAAAAALKKMAAAA|51882|14|24|42|PM|first|afternoon|lunch| +51883|AAAAAAAAMKKMAAAA|51883|14|24|43|PM|first|afternoon|lunch| +51884|AAAAAAAANKKMAAAA|51884|14|24|44|PM|first|afternoon|lunch| +51885|AAAAAAAAOKKMAAAA|51885|14|24|45|PM|first|afternoon|lunch| +51886|AAAAAAAAPKKMAAAA|51886|14|24|46|PM|first|afternoon|lunch| +51887|AAAAAAAAALKMAAAA|51887|14|24|47|PM|first|afternoon|lunch| +51888|AAAAAAAABLKMAAAA|51888|14|24|48|PM|first|afternoon|lunch| +51889|AAAAAAAACLKMAAAA|51889|14|24|49|PM|first|afternoon|lunch| +51890|AAAAAAAADLKMAAAA|51890|14|24|50|PM|first|afternoon|lunch| +51891|AAAAAAAAELKMAAAA|51891|14|24|51|PM|first|afternoon|lunch| +51892|AAAAAAAAFLKMAAAA|51892|14|24|52|PM|first|afternoon|lunch| +51893|AAAAAAAAGLKMAAAA|51893|14|24|53|PM|first|afternoon|lunch| +51894|AAAAAAAAHLKMAAAA|51894|14|24|54|PM|first|afternoon|lunch| +51895|AAAAAAAAILKMAAAA|51895|14|24|55|PM|first|afternoon|lunch| +51896|AAAAAAAAJLKMAAAA|51896|14|24|56|PM|first|afternoon|lunch| +51897|AAAAAAAAKLKMAAAA|51897|14|24|57|PM|first|afternoon|lunch| +51898|AAAAAAAALLKMAAAA|51898|14|24|58|PM|first|afternoon|lunch| +51899|AAAAAAAAMLKMAAAA|51899|14|24|59|PM|first|afternoon|lunch| +51900|AAAAAAAANLKMAAAA|51900|14|25|0|PM|first|afternoon|lunch| +51901|AAAAAAAAOLKMAAAA|51901|14|25|1|PM|first|afternoon|lunch| +51902|AAAAAAAAPLKMAAAA|51902|14|25|2|PM|first|afternoon|lunch| +51903|AAAAAAAAAMKMAAAA|51903|14|25|3|PM|first|afternoon|lunch| +51904|AAAAAAAABMKMAAAA|51904|14|25|4|PM|first|afternoon|lunch| +51905|AAAAAAAACMKMAAAA|51905|14|25|5|PM|first|afternoon|lunch| +51906|AAAAAAAADMKMAAAA|51906|14|25|6|PM|first|afternoon|lunch| +51907|AAAAAAAAEMKMAAAA|51907|14|25|7|PM|first|afternoon|lunch| +51908|AAAAAAAAFMKMAAAA|51908|14|25|8|PM|first|afternoon|lunch| +51909|AAAAAAAAGMKMAAAA|51909|14|25|9|PM|first|afternoon|lunch| +51910|AAAAAAAAHMKMAAAA|51910|14|25|10|PM|first|afternoon|lunch| +51911|AAAAAAAAIMKMAAAA|51911|14|25|11|PM|first|afternoon|lunch| +51912|AAAAAAAAJMKMAAAA|51912|14|25|12|PM|first|afternoon|lunch| +51913|AAAAAAAAKMKMAAAA|51913|14|25|13|PM|first|afternoon|lunch| +51914|AAAAAAAALMKMAAAA|51914|14|25|14|PM|first|afternoon|lunch| +51915|AAAAAAAAMMKMAAAA|51915|14|25|15|PM|first|afternoon|lunch| +51916|AAAAAAAANMKMAAAA|51916|14|25|16|PM|first|afternoon|lunch| +51917|AAAAAAAAOMKMAAAA|51917|14|25|17|PM|first|afternoon|lunch| +51918|AAAAAAAAPMKMAAAA|51918|14|25|18|PM|first|afternoon|lunch| +51919|AAAAAAAAANKMAAAA|51919|14|25|19|PM|first|afternoon|lunch| +51920|AAAAAAAABNKMAAAA|51920|14|25|20|PM|first|afternoon|lunch| +51921|AAAAAAAACNKMAAAA|51921|14|25|21|PM|first|afternoon|lunch| +51922|AAAAAAAADNKMAAAA|51922|14|25|22|PM|first|afternoon|lunch| +51923|AAAAAAAAENKMAAAA|51923|14|25|23|PM|first|afternoon|lunch| +51924|AAAAAAAAFNKMAAAA|51924|14|25|24|PM|first|afternoon|lunch| +51925|AAAAAAAAGNKMAAAA|51925|14|25|25|PM|first|afternoon|lunch| +51926|AAAAAAAAHNKMAAAA|51926|14|25|26|PM|first|afternoon|lunch| +51927|AAAAAAAAINKMAAAA|51927|14|25|27|PM|first|afternoon|lunch| +51928|AAAAAAAAJNKMAAAA|51928|14|25|28|PM|first|afternoon|lunch| +51929|AAAAAAAAKNKMAAAA|51929|14|25|29|PM|first|afternoon|lunch| +51930|AAAAAAAALNKMAAAA|51930|14|25|30|PM|first|afternoon|lunch| +51931|AAAAAAAAMNKMAAAA|51931|14|25|31|PM|first|afternoon|lunch| +51932|AAAAAAAANNKMAAAA|51932|14|25|32|PM|first|afternoon|lunch| +51933|AAAAAAAAONKMAAAA|51933|14|25|33|PM|first|afternoon|lunch| +51934|AAAAAAAAPNKMAAAA|51934|14|25|34|PM|first|afternoon|lunch| +51935|AAAAAAAAAOKMAAAA|51935|14|25|35|PM|first|afternoon|lunch| +51936|AAAAAAAABOKMAAAA|51936|14|25|36|PM|first|afternoon|lunch| +51937|AAAAAAAACOKMAAAA|51937|14|25|37|PM|first|afternoon|lunch| +51938|AAAAAAAADOKMAAAA|51938|14|25|38|PM|first|afternoon|lunch| +51939|AAAAAAAAEOKMAAAA|51939|14|25|39|PM|first|afternoon|lunch| +51940|AAAAAAAAFOKMAAAA|51940|14|25|40|PM|first|afternoon|lunch| +51941|AAAAAAAAGOKMAAAA|51941|14|25|41|PM|first|afternoon|lunch| +51942|AAAAAAAAHOKMAAAA|51942|14|25|42|PM|first|afternoon|lunch| +51943|AAAAAAAAIOKMAAAA|51943|14|25|43|PM|first|afternoon|lunch| +51944|AAAAAAAAJOKMAAAA|51944|14|25|44|PM|first|afternoon|lunch| +51945|AAAAAAAAKOKMAAAA|51945|14|25|45|PM|first|afternoon|lunch| +51946|AAAAAAAALOKMAAAA|51946|14|25|46|PM|first|afternoon|lunch| +51947|AAAAAAAAMOKMAAAA|51947|14|25|47|PM|first|afternoon|lunch| +51948|AAAAAAAANOKMAAAA|51948|14|25|48|PM|first|afternoon|lunch| +51949|AAAAAAAAOOKMAAAA|51949|14|25|49|PM|first|afternoon|lunch| +51950|AAAAAAAAPOKMAAAA|51950|14|25|50|PM|first|afternoon|lunch| +51951|AAAAAAAAAPKMAAAA|51951|14|25|51|PM|first|afternoon|lunch| +51952|AAAAAAAABPKMAAAA|51952|14|25|52|PM|first|afternoon|lunch| +51953|AAAAAAAACPKMAAAA|51953|14|25|53|PM|first|afternoon|lunch| +51954|AAAAAAAADPKMAAAA|51954|14|25|54|PM|first|afternoon|lunch| +51955|AAAAAAAAEPKMAAAA|51955|14|25|55|PM|first|afternoon|lunch| +51956|AAAAAAAAFPKMAAAA|51956|14|25|56|PM|first|afternoon|lunch| +51957|AAAAAAAAGPKMAAAA|51957|14|25|57|PM|first|afternoon|lunch| +51958|AAAAAAAAHPKMAAAA|51958|14|25|58|PM|first|afternoon|lunch| +51959|AAAAAAAAIPKMAAAA|51959|14|25|59|PM|first|afternoon|lunch| +51960|AAAAAAAAJPKMAAAA|51960|14|26|0|PM|first|afternoon|lunch| +51961|AAAAAAAAKPKMAAAA|51961|14|26|1|PM|first|afternoon|lunch| +51962|AAAAAAAALPKMAAAA|51962|14|26|2|PM|first|afternoon|lunch| +51963|AAAAAAAAMPKMAAAA|51963|14|26|3|PM|first|afternoon|lunch| +51964|AAAAAAAANPKMAAAA|51964|14|26|4|PM|first|afternoon|lunch| +51965|AAAAAAAAOPKMAAAA|51965|14|26|5|PM|first|afternoon|lunch| +51966|AAAAAAAAPPKMAAAA|51966|14|26|6|PM|first|afternoon|lunch| +51967|AAAAAAAAAALMAAAA|51967|14|26|7|PM|first|afternoon|lunch| +51968|AAAAAAAABALMAAAA|51968|14|26|8|PM|first|afternoon|lunch| +51969|AAAAAAAACALMAAAA|51969|14|26|9|PM|first|afternoon|lunch| +51970|AAAAAAAADALMAAAA|51970|14|26|10|PM|first|afternoon|lunch| +51971|AAAAAAAAEALMAAAA|51971|14|26|11|PM|first|afternoon|lunch| +51972|AAAAAAAAFALMAAAA|51972|14|26|12|PM|first|afternoon|lunch| +51973|AAAAAAAAGALMAAAA|51973|14|26|13|PM|first|afternoon|lunch| +51974|AAAAAAAAHALMAAAA|51974|14|26|14|PM|first|afternoon|lunch| +51975|AAAAAAAAIALMAAAA|51975|14|26|15|PM|first|afternoon|lunch| +51976|AAAAAAAAJALMAAAA|51976|14|26|16|PM|first|afternoon|lunch| +51977|AAAAAAAAKALMAAAA|51977|14|26|17|PM|first|afternoon|lunch| +51978|AAAAAAAALALMAAAA|51978|14|26|18|PM|first|afternoon|lunch| +51979|AAAAAAAAMALMAAAA|51979|14|26|19|PM|first|afternoon|lunch| +51980|AAAAAAAANALMAAAA|51980|14|26|20|PM|first|afternoon|lunch| +51981|AAAAAAAAOALMAAAA|51981|14|26|21|PM|first|afternoon|lunch| +51982|AAAAAAAAPALMAAAA|51982|14|26|22|PM|first|afternoon|lunch| +51983|AAAAAAAAABLMAAAA|51983|14|26|23|PM|first|afternoon|lunch| +51984|AAAAAAAABBLMAAAA|51984|14|26|24|PM|first|afternoon|lunch| +51985|AAAAAAAACBLMAAAA|51985|14|26|25|PM|first|afternoon|lunch| +51986|AAAAAAAADBLMAAAA|51986|14|26|26|PM|first|afternoon|lunch| +51987|AAAAAAAAEBLMAAAA|51987|14|26|27|PM|first|afternoon|lunch| +51988|AAAAAAAAFBLMAAAA|51988|14|26|28|PM|first|afternoon|lunch| +51989|AAAAAAAAGBLMAAAA|51989|14|26|29|PM|first|afternoon|lunch| +51990|AAAAAAAAHBLMAAAA|51990|14|26|30|PM|first|afternoon|lunch| +51991|AAAAAAAAIBLMAAAA|51991|14|26|31|PM|first|afternoon|lunch| +51992|AAAAAAAAJBLMAAAA|51992|14|26|32|PM|first|afternoon|lunch| +51993|AAAAAAAAKBLMAAAA|51993|14|26|33|PM|first|afternoon|lunch| +51994|AAAAAAAALBLMAAAA|51994|14|26|34|PM|first|afternoon|lunch| +51995|AAAAAAAAMBLMAAAA|51995|14|26|35|PM|first|afternoon|lunch| +51996|AAAAAAAANBLMAAAA|51996|14|26|36|PM|first|afternoon|lunch| +51997|AAAAAAAAOBLMAAAA|51997|14|26|37|PM|first|afternoon|lunch| +51998|AAAAAAAAPBLMAAAA|51998|14|26|38|PM|first|afternoon|lunch| +51999|AAAAAAAAACLMAAAA|51999|14|26|39|PM|first|afternoon|lunch| +52000|AAAAAAAABCLMAAAA|52000|14|26|40|PM|first|afternoon|lunch| +52001|AAAAAAAACCLMAAAA|52001|14|26|41|PM|first|afternoon|lunch| +52002|AAAAAAAADCLMAAAA|52002|14|26|42|PM|first|afternoon|lunch| +52003|AAAAAAAAECLMAAAA|52003|14|26|43|PM|first|afternoon|lunch| +52004|AAAAAAAAFCLMAAAA|52004|14|26|44|PM|first|afternoon|lunch| +52005|AAAAAAAAGCLMAAAA|52005|14|26|45|PM|first|afternoon|lunch| +52006|AAAAAAAAHCLMAAAA|52006|14|26|46|PM|first|afternoon|lunch| +52007|AAAAAAAAICLMAAAA|52007|14|26|47|PM|first|afternoon|lunch| +52008|AAAAAAAAJCLMAAAA|52008|14|26|48|PM|first|afternoon|lunch| +52009|AAAAAAAAKCLMAAAA|52009|14|26|49|PM|first|afternoon|lunch| +52010|AAAAAAAALCLMAAAA|52010|14|26|50|PM|first|afternoon|lunch| +52011|AAAAAAAAMCLMAAAA|52011|14|26|51|PM|first|afternoon|lunch| +52012|AAAAAAAANCLMAAAA|52012|14|26|52|PM|first|afternoon|lunch| +52013|AAAAAAAAOCLMAAAA|52013|14|26|53|PM|first|afternoon|lunch| +52014|AAAAAAAAPCLMAAAA|52014|14|26|54|PM|first|afternoon|lunch| +52015|AAAAAAAAADLMAAAA|52015|14|26|55|PM|first|afternoon|lunch| +52016|AAAAAAAABDLMAAAA|52016|14|26|56|PM|first|afternoon|lunch| +52017|AAAAAAAACDLMAAAA|52017|14|26|57|PM|first|afternoon|lunch| +52018|AAAAAAAADDLMAAAA|52018|14|26|58|PM|first|afternoon|lunch| +52019|AAAAAAAAEDLMAAAA|52019|14|26|59|PM|first|afternoon|lunch| +52020|AAAAAAAAFDLMAAAA|52020|14|27|0|PM|first|afternoon|lunch| +52021|AAAAAAAAGDLMAAAA|52021|14|27|1|PM|first|afternoon|lunch| +52022|AAAAAAAAHDLMAAAA|52022|14|27|2|PM|first|afternoon|lunch| +52023|AAAAAAAAIDLMAAAA|52023|14|27|3|PM|first|afternoon|lunch| +52024|AAAAAAAAJDLMAAAA|52024|14|27|4|PM|first|afternoon|lunch| +52025|AAAAAAAAKDLMAAAA|52025|14|27|5|PM|first|afternoon|lunch| +52026|AAAAAAAALDLMAAAA|52026|14|27|6|PM|first|afternoon|lunch| +52027|AAAAAAAAMDLMAAAA|52027|14|27|7|PM|first|afternoon|lunch| +52028|AAAAAAAANDLMAAAA|52028|14|27|8|PM|first|afternoon|lunch| +52029|AAAAAAAAODLMAAAA|52029|14|27|9|PM|first|afternoon|lunch| +52030|AAAAAAAAPDLMAAAA|52030|14|27|10|PM|first|afternoon|lunch| +52031|AAAAAAAAAELMAAAA|52031|14|27|11|PM|first|afternoon|lunch| +52032|AAAAAAAABELMAAAA|52032|14|27|12|PM|first|afternoon|lunch| +52033|AAAAAAAACELMAAAA|52033|14|27|13|PM|first|afternoon|lunch| +52034|AAAAAAAADELMAAAA|52034|14|27|14|PM|first|afternoon|lunch| +52035|AAAAAAAAEELMAAAA|52035|14|27|15|PM|first|afternoon|lunch| +52036|AAAAAAAAFELMAAAA|52036|14|27|16|PM|first|afternoon|lunch| +52037|AAAAAAAAGELMAAAA|52037|14|27|17|PM|first|afternoon|lunch| +52038|AAAAAAAAHELMAAAA|52038|14|27|18|PM|first|afternoon|lunch| +52039|AAAAAAAAIELMAAAA|52039|14|27|19|PM|first|afternoon|lunch| +52040|AAAAAAAAJELMAAAA|52040|14|27|20|PM|first|afternoon|lunch| +52041|AAAAAAAAKELMAAAA|52041|14|27|21|PM|first|afternoon|lunch| +52042|AAAAAAAALELMAAAA|52042|14|27|22|PM|first|afternoon|lunch| +52043|AAAAAAAAMELMAAAA|52043|14|27|23|PM|first|afternoon|lunch| +52044|AAAAAAAANELMAAAA|52044|14|27|24|PM|first|afternoon|lunch| +52045|AAAAAAAAOELMAAAA|52045|14|27|25|PM|first|afternoon|lunch| +52046|AAAAAAAAPELMAAAA|52046|14|27|26|PM|first|afternoon|lunch| +52047|AAAAAAAAAFLMAAAA|52047|14|27|27|PM|first|afternoon|lunch| +52048|AAAAAAAABFLMAAAA|52048|14|27|28|PM|first|afternoon|lunch| +52049|AAAAAAAACFLMAAAA|52049|14|27|29|PM|first|afternoon|lunch| +52050|AAAAAAAADFLMAAAA|52050|14|27|30|PM|first|afternoon|lunch| +52051|AAAAAAAAEFLMAAAA|52051|14|27|31|PM|first|afternoon|lunch| +52052|AAAAAAAAFFLMAAAA|52052|14|27|32|PM|first|afternoon|lunch| +52053|AAAAAAAAGFLMAAAA|52053|14|27|33|PM|first|afternoon|lunch| +52054|AAAAAAAAHFLMAAAA|52054|14|27|34|PM|first|afternoon|lunch| +52055|AAAAAAAAIFLMAAAA|52055|14|27|35|PM|first|afternoon|lunch| +52056|AAAAAAAAJFLMAAAA|52056|14|27|36|PM|first|afternoon|lunch| +52057|AAAAAAAAKFLMAAAA|52057|14|27|37|PM|first|afternoon|lunch| +52058|AAAAAAAALFLMAAAA|52058|14|27|38|PM|first|afternoon|lunch| +52059|AAAAAAAAMFLMAAAA|52059|14|27|39|PM|first|afternoon|lunch| +52060|AAAAAAAANFLMAAAA|52060|14|27|40|PM|first|afternoon|lunch| +52061|AAAAAAAAOFLMAAAA|52061|14|27|41|PM|first|afternoon|lunch| +52062|AAAAAAAAPFLMAAAA|52062|14|27|42|PM|first|afternoon|lunch| +52063|AAAAAAAAAGLMAAAA|52063|14|27|43|PM|first|afternoon|lunch| +52064|AAAAAAAABGLMAAAA|52064|14|27|44|PM|first|afternoon|lunch| +52065|AAAAAAAACGLMAAAA|52065|14|27|45|PM|first|afternoon|lunch| +52066|AAAAAAAADGLMAAAA|52066|14|27|46|PM|first|afternoon|lunch| +52067|AAAAAAAAEGLMAAAA|52067|14|27|47|PM|first|afternoon|lunch| +52068|AAAAAAAAFGLMAAAA|52068|14|27|48|PM|first|afternoon|lunch| +52069|AAAAAAAAGGLMAAAA|52069|14|27|49|PM|first|afternoon|lunch| +52070|AAAAAAAAHGLMAAAA|52070|14|27|50|PM|first|afternoon|lunch| +52071|AAAAAAAAIGLMAAAA|52071|14|27|51|PM|first|afternoon|lunch| +52072|AAAAAAAAJGLMAAAA|52072|14|27|52|PM|first|afternoon|lunch| +52073|AAAAAAAAKGLMAAAA|52073|14|27|53|PM|first|afternoon|lunch| +52074|AAAAAAAALGLMAAAA|52074|14|27|54|PM|first|afternoon|lunch| +52075|AAAAAAAAMGLMAAAA|52075|14|27|55|PM|first|afternoon|lunch| +52076|AAAAAAAANGLMAAAA|52076|14|27|56|PM|first|afternoon|lunch| +52077|AAAAAAAAOGLMAAAA|52077|14|27|57|PM|first|afternoon|lunch| +52078|AAAAAAAAPGLMAAAA|52078|14|27|58|PM|first|afternoon|lunch| +52079|AAAAAAAAAHLMAAAA|52079|14|27|59|PM|first|afternoon|lunch| +52080|AAAAAAAABHLMAAAA|52080|14|28|0|PM|first|afternoon|lunch| +52081|AAAAAAAACHLMAAAA|52081|14|28|1|PM|first|afternoon|lunch| +52082|AAAAAAAADHLMAAAA|52082|14|28|2|PM|first|afternoon|lunch| +52083|AAAAAAAAEHLMAAAA|52083|14|28|3|PM|first|afternoon|lunch| +52084|AAAAAAAAFHLMAAAA|52084|14|28|4|PM|first|afternoon|lunch| +52085|AAAAAAAAGHLMAAAA|52085|14|28|5|PM|first|afternoon|lunch| +52086|AAAAAAAAHHLMAAAA|52086|14|28|6|PM|first|afternoon|lunch| +52087|AAAAAAAAIHLMAAAA|52087|14|28|7|PM|first|afternoon|lunch| +52088|AAAAAAAAJHLMAAAA|52088|14|28|8|PM|first|afternoon|lunch| +52089|AAAAAAAAKHLMAAAA|52089|14|28|9|PM|first|afternoon|lunch| +52090|AAAAAAAALHLMAAAA|52090|14|28|10|PM|first|afternoon|lunch| +52091|AAAAAAAAMHLMAAAA|52091|14|28|11|PM|first|afternoon|lunch| +52092|AAAAAAAANHLMAAAA|52092|14|28|12|PM|first|afternoon|lunch| +52093|AAAAAAAAOHLMAAAA|52093|14|28|13|PM|first|afternoon|lunch| +52094|AAAAAAAAPHLMAAAA|52094|14|28|14|PM|first|afternoon|lunch| +52095|AAAAAAAAAILMAAAA|52095|14|28|15|PM|first|afternoon|lunch| +52096|AAAAAAAABILMAAAA|52096|14|28|16|PM|first|afternoon|lunch| +52097|AAAAAAAACILMAAAA|52097|14|28|17|PM|first|afternoon|lunch| +52098|AAAAAAAADILMAAAA|52098|14|28|18|PM|first|afternoon|lunch| +52099|AAAAAAAAEILMAAAA|52099|14|28|19|PM|first|afternoon|lunch| +52100|AAAAAAAAFILMAAAA|52100|14|28|20|PM|first|afternoon|lunch| +52101|AAAAAAAAGILMAAAA|52101|14|28|21|PM|first|afternoon|lunch| +52102|AAAAAAAAHILMAAAA|52102|14|28|22|PM|first|afternoon|lunch| +52103|AAAAAAAAIILMAAAA|52103|14|28|23|PM|first|afternoon|lunch| +52104|AAAAAAAAJILMAAAA|52104|14|28|24|PM|first|afternoon|lunch| +52105|AAAAAAAAKILMAAAA|52105|14|28|25|PM|first|afternoon|lunch| +52106|AAAAAAAALILMAAAA|52106|14|28|26|PM|first|afternoon|lunch| +52107|AAAAAAAAMILMAAAA|52107|14|28|27|PM|first|afternoon|lunch| +52108|AAAAAAAANILMAAAA|52108|14|28|28|PM|first|afternoon|lunch| +52109|AAAAAAAAOILMAAAA|52109|14|28|29|PM|first|afternoon|lunch| +52110|AAAAAAAAPILMAAAA|52110|14|28|30|PM|first|afternoon|lunch| +52111|AAAAAAAAAJLMAAAA|52111|14|28|31|PM|first|afternoon|lunch| +52112|AAAAAAAABJLMAAAA|52112|14|28|32|PM|first|afternoon|lunch| +52113|AAAAAAAACJLMAAAA|52113|14|28|33|PM|first|afternoon|lunch| +52114|AAAAAAAADJLMAAAA|52114|14|28|34|PM|first|afternoon|lunch| +52115|AAAAAAAAEJLMAAAA|52115|14|28|35|PM|first|afternoon|lunch| +52116|AAAAAAAAFJLMAAAA|52116|14|28|36|PM|first|afternoon|lunch| +52117|AAAAAAAAGJLMAAAA|52117|14|28|37|PM|first|afternoon|lunch| +52118|AAAAAAAAHJLMAAAA|52118|14|28|38|PM|first|afternoon|lunch| +52119|AAAAAAAAIJLMAAAA|52119|14|28|39|PM|first|afternoon|lunch| +52120|AAAAAAAAJJLMAAAA|52120|14|28|40|PM|first|afternoon|lunch| +52121|AAAAAAAAKJLMAAAA|52121|14|28|41|PM|first|afternoon|lunch| +52122|AAAAAAAALJLMAAAA|52122|14|28|42|PM|first|afternoon|lunch| +52123|AAAAAAAAMJLMAAAA|52123|14|28|43|PM|first|afternoon|lunch| +52124|AAAAAAAANJLMAAAA|52124|14|28|44|PM|first|afternoon|lunch| +52125|AAAAAAAAOJLMAAAA|52125|14|28|45|PM|first|afternoon|lunch| +52126|AAAAAAAAPJLMAAAA|52126|14|28|46|PM|first|afternoon|lunch| +52127|AAAAAAAAAKLMAAAA|52127|14|28|47|PM|first|afternoon|lunch| +52128|AAAAAAAABKLMAAAA|52128|14|28|48|PM|first|afternoon|lunch| +52129|AAAAAAAACKLMAAAA|52129|14|28|49|PM|first|afternoon|lunch| +52130|AAAAAAAADKLMAAAA|52130|14|28|50|PM|first|afternoon|lunch| +52131|AAAAAAAAEKLMAAAA|52131|14|28|51|PM|first|afternoon|lunch| +52132|AAAAAAAAFKLMAAAA|52132|14|28|52|PM|first|afternoon|lunch| +52133|AAAAAAAAGKLMAAAA|52133|14|28|53|PM|first|afternoon|lunch| +52134|AAAAAAAAHKLMAAAA|52134|14|28|54|PM|first|afternoon|lunch| +52135|AAAAAAAAIKLMAAAA|52135|14|28|55|PM|first|afternoon|lunch| +52136|AAAAAAAAJKLMAAAA|52136|14|28|56|PM|first|afternoon|lunch| +52137|AAAAAAAAKKLMAAAA|52137|14|28|57|PM|first|afternoon|lunch| +52138|AAAAAAAALKLMAAAA|52138|14|28|58|PM|first|afternoon|lunch| +52139|AAAAAAAAMKLMAAAA|52139|14|28|59|PM|first|afternoon|lunch| +52140|AAAAAAAANKLMAAAA|52140|14|29|0|PM|first|afternoon|lunch| +52141|AAAAAAAAOKLMAAAA|52141|14|29|1|PM|first|afternoon|lunch| +52142|AAAAAAAAPKLMAAAA|52142|14|29|2|PM|first|afternoon|lunch| +52143|AAAAAAAAALLMAAAA|52143|14|29|3|PM|first|afternoon|lunch| +52144|AAAAAAAABLLMAAAA|52144|14|29|4|PM|first|afternoon|lunch| +52145|AAAAAAAACLLMAAAA|52145|14|29|5|PM|first|afternoon|lunch| +52146|AAAAAAAADLLMAAAA|52146|14|29|6|PM|first|afternoon|lunch| +52147|AAAAAAAAELLMAAAA|52147|14|29|7|PM|first|afternoon|lunch| +52148|AAAAAAAAFLLMAAAA|52148|14|29|8|PM|first|afternoon|lunch| +52149|AAAAAAAAGLLMAAAA|52149|14|29|9|PM|first|afternoon|lunch| +52150|AAAAAAAAHLLMAAAA|52150|14|29|10|PM|first|afternoon|lunch| +52151|AAAAAAAAILLMAAAA|52151|14|29|11|PM|first|afternoon|lunch| +52152|AAAAAAAAJLLMAAAA|52152|14|29|12|PM|first|afternoon|lunch| +52153|AAAAAAAAKLLMAAAA|52153|14|29|13|PM|first|afternoon|lunch| +52154|AAAAAAAALLLMAAAA|52154|14|29|14|PM|first|afternoon|lunch| +52155|AAAAAAAAMLLMAAAA|52155|14|29|15|PM|first|afternoon|lunch| +52156|AAAAAAAANLLMAAAA|52156|14|29|16|PM|first|afternoon|lunch| +52157|AAAAAAAAOLLMAAAA|52157|14|29|17|PM|first|afternoon|lunch| +52158|AAAAAAAAPLLMAAAA|52158|14|29|18|PM|first|afternoon|lunch| +52159|AAAAAAAAAMLMAAAA|52159|14|29|19|PM|first|afternoon|lunch| +52160|AAAAAAAABMLMAAAA|52160|14|29|20|PM|first|afternoon|lunch| +52161|AAAAAAAACMLMAAAA|52161|14|29|21|PM|first|afternoon|lunch| +52162|AAAAAAAADMLMAAAA|52162|14|29|22|PM|first|afternoon|lunch| +52163|AAAAAAAAEMLMAAAA|52163|14|29|23|PM|first|afternoon|lunch| +52164|AAAAAAAAFMLMAAAA|52164|14|29|24|PM|first|afternoon|lunch| +52165|AAAAAAAAGMLMAAAA|52165|14|29|25|PM|first|afternoon|lunch| +52166|AAAAAAAAHMLMAAAA|52166|14|29|26|PM|first|afternoon|lunch| +52167|AAAAAAAAIMLMAAAA|52167|14|29|27|PM|first|afternoon|lunch| +52168|AAAAAAAAJMLMAAAA|52168|14|29|28|PM|first|afternoon|lunch| +52169|AAAAAAAAKMLMAAAA|52169|14|29|29|PM|first|afternoon|lunch| +52170|AAAAAAAALMLMAAAA|52170|14|29|30|PM|first|afternoon|lunch| +52171|AAAAAAAAMMLMAAAA|52171|14|29|31|PM|first|afternoon|lunch| +52172|AAAAAAAANMLMAAAA|52172|14|29|32|PM|first|afternoon|lunch| +52173|AAAAAAAAOMLMAAAA|52173|14|29|33|PM|first|afternoon|lunch| +52174|AAAAAAAAPMLMAAAA|52174|14|29|34|PM|first|afternoon|lunch| +52175|AAAAAAAAANLMAAAA|52175|14|29|35|PM|first|afternoon|lunch| +52176|AAAAAAAABNLMAAAA|52176|14|29|36|PM|first|afternoon|lunch| +52177|AAAAAAAACNLMAAAA|52177|14|29|37|PM|first|afternoon|lunch| +52178|AAAAAAAADNLMAAAA|52178|14|29|38|PM|first|afternoon|lunch| +52179|AAAAAAAAENLMAAAA|52179|14|29|39|PM|first|afternoon|lunch| +52180|AAAAAAAAFNLMAAAA|52180|14|29|40|PM|first|afternoon|lunch| +52181|AAAAAAAAGNLMAAAA|52181|14|29|41|PM|first|afternoon|lunch| +52182|AAAAAAAAHNLMAAAA|52182|14|29|42|PM|first|afternoon|lunch| +52183|AAAAAAAAINLMAAAA|52183|14|29|43|PM|first|afternoon|lunch| +52184|AAAAAAAAJNLMAAAA|52184|14|29|44|PM|first|afternoon|lunch| +52185|AAAAAAAAKNLMAAAA|52185|14|29|45|PM|first|afternoon|lunch| +52186|AAAAAAAALNLMAAAA|52186|14|29|46|PM|first|afternoon|lunch| +52187|AAAAAAAAMNLMAAAA|52187|14|29|47|PM|first|afternoon|lunch| +52188|AAAAAAAANNLMAAAA|52188|14|29|48|PM|first|afternoon|lunch| +52189|AAAAAAAAONLMAAAA|52189|14|29|49|PM|first|afternoon|lunch| +52190|AAAAAAAAPNLMAAAA|52190|14|29|50|PM|first|afternoon|lunch| +52191|AAAAAAAAAOLMAAAA|52191|14|29|51|PM|first|afternoon|lunch| +52192|AAAAAAAABOLMAAAA|52192|14|29|52|PM|first|afternoon|lunch| +52193|AAAAAAAACOLMAAAA|52193|14|29|53|PM|first|afternoon|lunch| +52194|AAAAAAAADOLMAAAA|52194|14|29|54|PM|first|afternoon|lunch| +52195|AAAAAAAAEOLMAAAA|52195|14|29|55|PM|first|afternoon|lunch| +52196|AAAAAAAAFOLMAAAA|52196|14|29|56|PM|first|afternoon|lunch| +52197|AAAAAAAAGOLMAAAA|52197|14|29|57|PM|first|afternoon|lunch| +52198|AAAAAAAAHOLMAAAA|52198|14|29|58|PM|first|afternoon|lunch| +52199|AAAAAAAAIOLMAAAA|52199|14|29|59|PM|first|afternoon|lunch| +52200|AAAAAAAAJOLMAAAA|52200|14|30|0|PM|first|afternoon|lunch| +52201|AAAAAAAAKOLMAAAA|52201|14|30|1|PM|first|afternoon|lunch| +52202|AAAAAAAALOLMAAAA|52202|14|30|2|PM|first|afternoon|lunch| +52203|AAAAAAAAMOLMAAAA|52203|14|30|3|PM|first|afternoon|lunch| +52204|AAAAAAAANOLMAAAA|52204|14|30|4|PM|first|afternoon|lunch| +52205|AAAAAAAAOOLMAAAA|52205|14|30|5|PM|first|afternoon|lunch| +52206|AAAAAAAAPOLMAAAA|52206|14|30|6|PM|first|afternoon|lunch| +52207|AAAAAAAAAPLMAAAA|52207|14|30|7|PM|first|afternoon|lunch| +52208|AAAAAAAABPLMAAAA|52208|14|30|8|PM|first|afternoon|lunch| +52209|AAAAAAAACPLMAAAA|52209|14|30|9|PM|first|afternoon|lunch| +52210|AAAAAAAADPLMAAAA|52210|14|30|10|PM|first|afternoon|lunch| +52211|AAAAAAAAEPLMAAAA|52211|14|30|11|PM|first|afternoon|lunch| +52212|AAAAAAAAFPLMAAAA|52212|14|30|12|PM|first|afternoon|lunch| +52213|AAAAAAAAGPLMAAAA|52213|14|30|13|PM|first|afternoon|lunch| +52214|AAAAAAAAHPLMAAAA|52214|14|30|14|PM|first|afternoon|lunch| +52215|AAAAAAAAIPLMAAAA|52215|14|30|15|PM|first|afternoon|lunch| +52216|AAAAAAAAJPLMAAAA|52216|14|30|16|PM|first|afternoon|lunch| +52217|AAAAAAAAKPLMAAAA|52217|14|30|17|PM|first|afternoon|lunch| +52218|AAAAAAAALPLMAAAA|52218|14|30|18|PM|first|afternoon|lunch| +52219|AAAAAAAAMPLMAAAA|52219|14|30|19|PM|first|afternoon|lunch| +52220|AAAAAAAANPLMAAAA|52220|14|30|20|PM|first|afternoon|lunch| +52221|AAAAAAAAOPLMAAAA|52221|14|30|21|PM|first|afternoon|lunch| +52222|AAAAAAAAPPLMAAAA|52222|14|30|22|PM|first|afternoon|lunch| +52223|AAAAAAAAAAMMAAAA|52223|14|30|23|PM|first|afternoon|lunch| +52224|AAAAAAAABAMMAAAA|52224|14|30|24|PM|first|afternoon|lunch| +52225|AAAAAAAACAMMAAAA|52225|14|30|25|PM|first|afternoon|lunch| +52226|AAAAAAAADAMMAAAA|52226|14|30|26|PM|first|afternoon|lunch| +52227|AAAAAAAAEAMMAAAA|52227|14|30|27|PM|first|afternoon|lunch| +52228|AAAAAAAAFAMMAAAA|52228|14|30|28|PM|first|afternoon|lunch| +52229|AAAAAAAAGAMMAAAA|52229|14|30|29|PM|first|afternoon|lunch| +52230|AAAAAAAAHAMMAAAA|52230|14|30|30|PM|first|afternoon|lunch| +52231|AAAAAAAAIAMMAAAA|52231|14|30|31|PM|first|afternoon|lunch| +52232|AAAAAAAAJAMMAAAA|52232|14|30|32|PM|first|afternoon|lunch| +52233|AAAAAAAAKAMMAAAA|52233|14|30|33|PM|first|afternoon|lunch| +52234|AAAAAAAALAMMAAAA|52234|14|30|34|PM|first|afternoon|lunch| +52235|AAAAAAAAMAMMAAAA|52235|14|30|35|PM|first|afternoon|lunch| +52236|AAAAAAAANAMMAAAA|52236|14|30|36|PM|first|afternoon|lunch| +52237|AAAAAAAAOAMMAAAA|52237|14|30|37|PM|first|afternoon|lunch| +52238|AAAAAAAAPAMMAAAA|52238|14|30|38|PM|first|afternoon|lunch| +52239|AAAAAAAAABMMAAAA|52239|14|30|39|PM|first|afternoon|lunch| +52240|AAAAAAAABBMMAAAA|52240|14|30|40|PM|first|afternoon|lunch| +52241|AAAAAAAACBMMAAAA|52241|14|30|41|PM|first|afternoon|lunch| +52242|AAAAAAAADBMMAAAA|52242|14|30|42|PM|first|afternoon|lunch| +52243|AAAAAAAAEBMMAAAA|52243|14|30|43|PM|first|afternoon|lunch| +52244|AAAAAAAAFBMMAAAA|52244|14|30|44|PM|first|afternoon|lunch| +52245|AAAAAAAAGBMMAAAA|52245|14|30|45|PM|first|afternoon|lunch| +52246|AAAAAAAAHBMMAAAA|52246|14|30|46|PM|first|afternoon|lunch| +52247|AAAAAAAAIBMMAAAA|52247|14|30|47|PM|first|afternoon|lunch| +52248|AAAAAAAAJBMMAAAA|52248|14|30|48|PM|first|afternoon|lunch| +52249|AAAAAAAAKBMMAAAA|52249|14|30|49|PM|first|afternoon|lunch| +52250|AAAAAAAALBMMAAAA|52250|14|30|50|PM|first|afternoon|lunch| +52251|AAAAAAAAMBMMAAAA|52251|14|30|51|PM|first|afternoon|lunch| +52252|AAAAAAAANBMMAAAA|52252|14|30|52|PM|first|afternoon|lunch| +52253|AAAAAAAAOBMMAAAA|52253|14|30|53|PM|first|afternoon|lunch| +52254|AAAAAAAAPBMMAAAA|52254|14|30|54|PM|first|afternoon|lunch| +52255|AAAAAAAAACMMAAAA|52255|14|30|55|PM|first|afternoon|lunch| +52256|AAAAAAAABCMMAAAA|52256|14|30|56|PM|first|afternoon|lunch| +52257|AAAAAAAACCMMAAAA|52257|14|30|57|PM|first|afternoon|lunch| +52258|AAAAAAAADCMMAAAA|52258|14|30|58|PM|first|afternoon|lunch| +52259|AAAAAAAAECMMAAAA|52259|14|30|59|PM|first|afternoon|lunch| +52260|AAAAAAAAFCMMAAAA|52260|14|31|0|PM|first|afternoon|lunch| +52261|AAAAAAAAGCMMAAAA|52261|14|31|1|PM|first|afternoon|lunch| +52262|AAAAAAAAHCMMAAAA|52262|14|31|2|PM|first|afternoon|lunch| +52263|AAAAAAAAICMMAAAA|52263|14|31|3|PM|first|afternoon|lunch| +52264|AAAAAAAAJCMMAAAA|52264|14|31|4|PM|first|afternoon|lunch| +52265|AAAAAAAAKCMMAAAA|52265|14|31|5|PM|first|afternoon|lunch| +52266|AAAAAAAALCMMAAAA|52266|14|31|6|PM|first|afternoon|lunch| +52267|AAAAAAAAMCMMAAAA|52267|14|31|7|PM|first|afternoon|lunch| +52268|AAAAAAAANCMMAAAA|52268|14|31|8|PM|first|afternoon|lunch| +52269|AAAAAAAAOCMMAAAA|52269|14|31|9|PM|first|afternoon|lunch| +52270|AAAAAAAAPCMMAAAA|52270|14|31|10|PM|first|afternoon|lunch| +52271|AAAAAAAAADMMAAAA|52271|14|31|11|PM|first|afternoon|lunch| +52272|AAAAAAAABDMMAAAA|52272|14|31|12|PM|first|afternoon|lunch| +52273|AAAAAAAACDMMAAAA|52273|14|31|13|PM|first|afternoon|lunch| +52274|AAAAAAAADDMMAAAA|52274|14|31|14|PM|first|afternoon|lunch| +52275|AAAAAAAAEDMMAAAA|52275|14|31|15|PM|first|afternoon|lunch| +52276|AAAAAAAAFDMMAAAA|52276|14|31|16|PM|first|afternoon|lunch| +52277|AAAAAAAAGDMMAAAA|52277|14|31|17|PM|first|afternoon|lunch| +52278|AAAAAAAAHDMMAAAA|52278|14|31|18|PM|first|afternoon|lunch| +52279|AAAAAAAAIDMMAAAA|52279|14|31|19|PM|first|afternoon|lunch| +52280|AAAAAAAAJDMMAAAA|52280|14|31|20|PM|first|afternoon|lunch| +52281|AAAAAAAAKDMMAAAA|52281|14|31|21|PM|first|afternoon|lunch| +52282|AAAAAAAALDMMAAAA|52282|14|31|22|PM|first|afternoon|lunch| +52283|AAAAAAAAMDMMAAAA|52283|14|31|23|PM|first|afternoon|lunch| +52284|AAAAAAAANDMMAAAA|52284|14|31|24|PM|first|afternoon|lunch| +52285|AAAAAAAAODMMAAAA|52285|14|31|25|PM|first|afternoon|lunch| +52286|AAAAAAAAPDMMAAAA|52286|14|31|26|PM|first|afternoon|lunch| +52287|AAAAAAAAAEMMAAAA|52287|14|31|27|PM|first|afternoon|lunch| +52288|AAAAAAAABEMMAAAA|52288|14|31|28|PM|first|afternoon|lunch| +52289|AAAAAAAACEMMAAAA|52289|14|31|29|PM|first|afternoon|lunch| +52290|AAAAAAAADEMMAAAA|52290|14|31|30|PM|first|afternoon|lunch| +52291|AAAAAAAAEEMMAAAA|52291|14|31|31|PM|first|afternoon|lunch| +52292|AAAAAAAAFEMMAAAA|52292|14|31|32|PM|first|afternoon|lunch| +52293|AAAAAAAAGEMMAAAA|52293|14|31|33|PM|first|afternoon|lunch| +52294|AAAAAAAAHEMMAAAA|52294|14|31|34|PM|first|afternoon|lunch| +52295|AAAAAAAAIEMMAAAA|52295|14|31|35|PM|first|afternoon|lunch| +52296|AAAAAAAAJEMMAAAA|52296|14|31|36|PM|first|afternoon|lunch| +52297|AAAAAAAAKEMMAAAA|52297|14|31|37|PM|first|afternoon|lunch| +52298|AAAAAAAALEMMAAAA|52298|14|31|38|PM|first|afternoon|lunch| +52299|AAAAAAAAMEMMAAAA|52299|14|31|39|PM|first|afternoon|lunch| +52300|AAAAAAAANEMMAAAA|52300|14|31|40|PM|first|afternoon|lunch| +52301|AAAAAAAAOEMMAAAA|52301|14|31|41|PM|first|afternoon|lunch| +52302|AAAAAAAAPEMMAAAA|52302|14|31|42|PM|first|afternoon|lunch| +52303|AAAAAAAAAFMMAAAA|52303|14|31|43|PM|first|afternoon|lunch| +52304|AAAAAAAABFMMAAAA|52304|14|31|44|PM|first|afternoon|lunch| +52305|AAAAAAAACFMMAAAA|52305|14|31|45|PM|first|afternoon|lunch| +52306|AAAAAAAADFMMAAAA|52306|14|31|46|PM|first|afternoon|lunch| +52307|AAAAAAAAEFMMAAAA|52307|14|31|47|PM|first|afternoon|lunch| +52308|AAAAAAAAFFMMAAAA|52308|14|31|48|PM|first|afternoon|lunch| +52309|AAAAAAAAGFMMAAAA|52309|14|31|49|PM|first|afternoon|lunch| +52310|AAAAAAAAHFMMAAAA|52310|14|31|50|PM|first|afternoon|lunch| +52311|AAAAAAAAIFMMAAAA|52311|14|31|51|PM|first|afternoon|lunch| +52312|AAAAAAAAJFMMAAAA|52312|14|31|52|PM|first|afternoon|lunch| +52313|AAAAAAAAKFMMAAAA|52313|14|31|53|PM|first|afternoon|lunch| +52314|AAAAAAAALFMMAAAA|52314|14|31|54|PM|first|afternoon|lunch| +52315|AAAAAAAAMFMMAAAA|52315|14|31|55|PM|first|afternoon|lunch| +52316|AAAAAAAANFMMAAAA|52316|14|31|56|PM|first|afternoon|lunch| +52317|AAAAAAAAOFMMAAAA|52317|14|31|57|PM|first|afternoon|lunch| +52318|AAAAAAAAPFMMAAAA|52318|14|31|58|PM|first|afternoon|lunch| +52319|AAAAAAAAAGMMAAAA|52319|14|31|59|PM|first|afternoon|lunch| +52320|AAAAAAAABGMMAAAA|52320|14|32|0|PM|first|afternoon|lunch| +52321|AAAAAAAACGMMAAAA|52321|14|32|1|PM|first|afternoon|lunch| +52322|AAAAAAAADGMMAAAA|52322|14|32|2|PM|first|afternoon|lunch| +52323|AAAAAAAAEGMMAAAA|52323|14|32|3|PM|first|afternoon|lunch| +52324|AAAAAAAAFGMMAAAA|52324|14|32|4|PM|first|afternoon|lunch| +52325|AAAAAAAAGGMMAAAA|52325|14|32|5|PM|first|afternoon|lunch| +52326|AAAAAAAAHGMMAAAA|52326|14|32|6|PM|first|afternoon|lunch| +52327|AAAAAAAAIGMMAAAA|52327|14|32|7|PM|first|afternoon|lunch| +52328|AAAAAAAAJGMMAAAA|52328|14|32|8|PM|first|afternoon|lunch| +52329|AAAAAAAAKGMMAAAA|52329|14|32|9|PM|first|afternoon|lunch| +52330|AAAAAAAALGMMAAAA|52330|14|32|10|PM|first|afternoon|lunch| +52331|AAAAAAAAMGMMAAAA|52331|14|32|11|PM|first|afternoon|lunch| +52332|AAAAAAAANGMMAAAA|52332|14|32|12|PM|first|afternoon|lunch| +52333|AAAAAAAAOGMMAAAA|52333|14|32|13|PM|first|afternoon|lunch| +52334|AAAAAAAAPGMMAAAA|52334|14|32|14|PM|first|afternoon|lunch| +52335|AAAAAAAAAHMMAAAA|52335|14|32|15|PM|first|afternoon|lunch| +52336|AAAAAAAABHMMAAAA|52336|14|32|16|PM|first|afternoon|lunch| +52337|AAAAAAAACHMMAAAA|52337|14|32|17|PM|first|afternoon|lunch| +52338|AAAAAAAADHMMAAAA|52338|14|32|18|PM|first|afternoon|lunch| +52339|AAAAAAAAEHMMAAAA|52339|14|32|19|PM|first|afternoon|lunch| +52340|AAAAAAAAFHMMAAAA|52340|14|32|20|PM|first|afternoon|lunch| +52341|AAAAAAAAGHMMAAAA|52341|14|32|21|PM|first|afternoon|lunch| +52342|AAAAAAAAHHMMAAAA|52342|14|32|22|PM|first|afternoon|lunch| +52343|AAAAAAAAIHMMAAAA|52343|14|32|23|PM|first|afternoon|lunch| +52344|AAAAAAAAJHMMAAAA|52344|14|32|24|PM|first|afternoon|lunch| +52345|AAAAAAAAKHMMAAAA|52345|14|32|25|PM|first|afternoon|lunch| +52346|AAAAAAAALHMMAAAA|52346|14|32|26|PM|first|afternoon|lunch| +52347|AAAAAAAAMHMMAAAA|52347|14|32|27|PM|first|afternoon|lunch| +52348|AAAAAAAANHMMAAAA|52348|14|32|28|PM|first|afternoon|lunch| +52349|AAAAAAAAOHMMAAAA|52349|14|32|29|PM|first|afternoon|lunch| +52350|AAAAAAAAPHMMAAAA|52350|14|32|30|PM|first|afternoon|lunch| +52351|AAAAAAAAAIMMAAAA|52351|14|32|31|PM|first|afternoon|lunch| +52352|AAAAAAAABIMMAAAA|52352|14|32|32|PM|first|afternoon|lunch| +52353|AAAAAAAACIMMAAAA|52353|14|32|33|PM|first|afternoon|lunch| +52354|AAAAAAAADIMMAAAA|52354|14|32|34|PM|first|afternoon|lunch| +52355|AAAAAAAAEIMMAAAA|52355|14|32|35|PM|first|afternoon|lunch| +52356|AAAAAAAAFIMMAAAA|52356|14|32|36|PM|first|afternoon|lunch| +52357|AAAAAAAAGIMMAAAA|52357|14|32|37|PM|first|afternoon|lunch| +52358|AAAAAAAAHIMMAAAA|52358|14|32|38|PM|first|afternoon|lunch| +52359|AAAAAAAAIIMMAAAA|52359|14|32|39|PM|first|afternoon|lunch| +52360|AAAAAAAAJIMMAAAA|52360|14|32|40|PM|first|afternoon|lunch| +52361|AAAAAAAAKIMMAAAA|52361|14|32|41|PM|first|afternoon|lunch| +52362|AAAAAAAALIMMAAAA|52362|14|32|42|PM|first|afternoon|lunch| +52363|AAAAAAAAMIMMAAAA|52363|14|32|43|PM|first|afternoon|lunch| +52364|AAAAAAAANIMMAAAA|52364|14|32|44|PM|first|afternoon|lunch| +52365|AAAAAAAAOIMMAAAA|52365|14|32|45|PM|first|afternoon|lunch| +52366|AAAAAAAAPIMMAAAA|52366|14|32|46|PM|first|afternoon|lunch| +52367|AAAAAAAAAJMMAAAA|52367|14|32|47|PM|first|afternoon|lunch| +52368|AAAAAAAABJMMAAAA|52368|14|32|48|PM|first|afternoon|lunch| +52369|AAAAAAAACJMMAAAA|52369|14|32|49|PM|first|afternoon|lunch| +52370|AAAAAAAADJMMAAAA|52370|14|32|50|PM|first|afternoon|lunch| +52371|AAAAAAAAEJMMAAAA|52371|14|32|51|PM|first|afternoon|lunch| +52372|AAAAAAAAFJMMAAAA|52372|14|32|52|PM|first|afternoon|lunch| +52373|AAAAAAAAGJMMAAAA|52373|14|32|53|PM|first|afternoon|lunch| +52374|AAAAAAAAHJMMAAAA|52374|14|32|54|PM|first|afternoon|lunch| +52375|AAAAAAAAIJMMAAAA|52375|14|32|55|PM|first|afternoon|lunch| +52376|AAAAAAAAJJMMAAAA|52376|14|32|56|PM|first|afternoon|lunch| +52377|AAAAAAAAKJMMAAAA|52377|14|32|57|PM|first|afternoon|lunch| +52378|AAAAAAAALJMMAAAA|52378|14|32|58|PM|first|afternoon|lunch| +52379|AAAAAAAAMJMMAAAA|52379|14|32|59|PM|first|afternoon|lunch| +52380|AAAAAAAANJMMAAAA|52380|14|33|0|PM|first|afternoon|lunch| +52381|AAAAAAAAOJMMAAAA|52381|14|33|1|PM|first|afternoon|lunch| +52382|AAAAAAAAPJMMAAAA|52382|14|33|2|PM|first|afternoon|lunch| +52383|AAAAAAAAAKMMAAAA|52383|14|33|3|PM|first|afternoon|lunch| +52384|AAAAAAAABKMMAAAA|52384|14|33|4|PM|first|afternoon|lunch| +52385|AAAAAAAACKMMAAAA|52385|14|33|5|PM|first|afternoon|lunch| +52386|AAAAAAAADKMMAAAA|52386|14|33|6|PM|first|afternoon|lunch| +52387|AAAAAAAAEKMMAAAA|52387|14|33|7|PM|first|afternoon|lunch| +52388|AAAAAAAAFKMMAAAA|52388|14|33|8|PM|first|afternoon|lunch| +52389|AAAAAAAAGKMMAAAA|52389|14|33|9|PM|first|afternoon|lunch| +52390|AAAAAAAAHKMMAAAA|52390|14|33|10|PM|first|afternoon|lunch| +52391|AAAAAAAAIKMMAAAA|52391|14|33|11|PM|first|afternoon|lunch| +52392|AAAAAAAAJKMMAAAA|52392|14|33|12|PM|first|afternoon|lunch| +52393|AAAAAAAAKKMMAAAA|52393|14|33|13|PM|first|afternoon|lunch| +52394|AAAAAAAALKMMAAAA|52394|14|33|14|PM|first|afternoon|lunch| +52395|AAAAAAAAMKMMAAAA|52395|14|33|15|PM|first|afternoon|lunch| +52396|AAAAAAAANKMMAAAA|52396|14|33|16|PM|first|afternoon|lunch| +52397|AAAAAAAAOKMMAAAA|52397|14|33|17|PM|first|afternoon|lunch| +52398|AAAAAAAAPKMMAAAA|52398|14|33|18|PM|first|afternoon|lunch| +52399|AAAAAAAAALMMAAAA|52399|14|33|19|PM|first|afternoon|lunch| +52400|AAAAAAAABLMMAAAA|52400|14|33|20|PM|first|afternoon|lunch| +52401|AAAAAAAACLMMAAAA|52401|14|33|21|PM|first|afternoon|lunch| +52402|AAAAAAAADLMMAAAA|52402|14|33|22|PM|first|afternoon|lunch| +52403|AAAAAAAAELMMAAAA|52403|14|33|23|PM|first|afternoon|lunch| +52404|AAAAAAAAFLMMAAAA|52404|14|33|24|PM|first|afternoon|lunch| +52405|AAAAAAAAGLMMAAAA|52405|14|33|25|PM|first|afternoon|lunch| +52406|AAAAAAAAHLMMAAAA|52406|14|33|26|PM|first|afternoon|lunch| +52407|AAAAAAAAILMMAAAA|52407|14|33|27|PM|first|afternoon|lunch| +52408|AAAAAAAAJLMMAAAA|52408|14|33|28|PM|first|afternoon|lunch| +52409|AAAAAAAAKLMMAAAA|52409|14|33|29|PM|first|afternoon|lunch| +52410|AAAAAAAALLMMAAAA|52410|14|33|30|PM|first|afternoon|lunch| +52411|AAAAAAAAMLMMAAAA|52411|14|33|31|PM|first|afternoon|lunch| +52412|AAAAAAAANLMMAAAA|52412|14|33|32|PM|first|afternoon|lunch| +52413|AAAAAAAAOLMMAAAA|52413|14|33|33|PM|first|afternoon|lunch| +52414|AAAAAAAAPLMMAAAA|52414|14|33|34|PM|first|afternoon|lunch| +52415|AAAAAAAAAMMMAAAA|52415|14|33|35|PM|first|afternoon|lunch| +52416|AAAAAAAABMMMAAAA|52416|14|33|36|PM|first|afternoon|lunch| +52417|AAAAAAAACMMMAAAA|52417|14|33|37|PM|first|afternoon|lunch| +52418|AAAAAAAADMMMAAAA|52418|14|33|38|PM|first|afternoon|lunch| +52419|AAAAAAAAEMMMAAAA|52419|14|33|39|PM|first|afternoon|lunch| +52420|AAAAAAAAFMMMAAAA|52420|14|33|40|PM|first|afternoon|lunch| +52421|AAAAAAAAGMMMAAAA|52421|14|33|41|PM|first|afternoon|lunch| +52422|AAAAAAAAHMMMAAAA|52422|14|33|42|PM|first|afternoon|lunch| +52423|AAAAAAAAIMMMAAAA|52423|14|33|43|PM|first|afternoon|lunch| +52424|AAAAAAAAJMMMAAAA|52424|14|33|44|PM|first|afternoon|lunch| +52425|AAAAAAAAKMMMAAAA|52425|14|33|45|PM|first|afternoon|lunch| +52426|AAAAAAAALMMMAAAA|52426|14|33|46|PM|first|afternoon|lunch| +52427|AAAAAAAAMMMMAAAA|52427|14|33|47|PM|first|afternoon|lunch| +52428|AAAAAAAANMMMAAAA|52428|14|33|48|PM|first|afternoon|lunch| +52429|AAAAAAAAOMMMAAAA|52429|14|33|49|PM|first|afternoon|lunch| +52430|AAAAAAAAPMMMAAAA|52430|14|33|50|PM|first|afternoon|lunch| +52431|AAAAAAAAANMMAAAA|52431|14|33|51|PM|first|afternoon|lunch| +52432|AAAAAAAABNMMAAAA|52432|14|33|52|PM|first|afternoon|lunch| +52433|AAAAAAAACNMMAAAA|52433|14|33|53|PM|first|afternoon|lunch| +52434|AAAAAAAADNMMAAAA|52434|14|33|54|PM|first|afternoon|lunch| +52435|AAAAAAAAENMMAAAA|52435|14|33|55|PM|first|afternoon|lunch| +52436|AAAAAAAAFNMMAAAA|52436|14|33|56|PM|first|afternoon|lunch| +52437|AAAAAAAAGNMMAAAA|52437|14|33|57|PM|first|afternoon|lunch| +52438|AAAAAAAAHNMMAAAA|52438|14|33|58|PM|first|afternoon|lunch| +52439|AAAAAAAAINMMAAAA|52439|14|33|59|PM|first|afternoon|lunch| +52440|AAAAAAAAJNMMAAAA|52440|14|34|0|PM|first|afternoon|lunch| +52441|AAAAAAAAKNMMAAAA|52441|14|34|1|PM|first|afternoon|lunch| +52442|AAAAAAAALNMMAAAA|52442|14|34|2|PM|first|afternoon|lunch| +52443|AAAAAAAAMNMMAAAA|52443|14|34|3|PM|first|afternoon|lunch| +52444|AAAAAAAANNMMAAAA|52444|14|34|4|PM|first|afternoon|lunch| +52445|AAAAAAAAONMMAAAA|52445|14|34|5|PM|first|afternoon|lunch| +52446|AAAAAAAAPNMMAAAA|52446|14|34|6|PM|first|afternoon|lunch| +52447|AAAAAAAAAOMMAAAA|52447|14|34|7|PM|first|afternoon|lunch| +52448|AAAAAAAABOMMAAAA|52448|14|34|8|PM|first|afternoon|lunch| +52449|AAAAAAAACOMMAAAA|52449|14|34|9|PM|first|afternoon|lunch| +52450|AAAAAAAADOMMAAAA|52450|14|34|10|PM|first|afternoon|lunch| +52451|AAAAAAAAEOMMAAAA|52451|14|34|11|PM|first|afternoon|lunch| +52452|AAAAAAAAFOMMAAAA|52452|14|34|12|PM|first|afternoon|lunch| +52453|AAAAAAAAGOMMAAAA|52453|14|34|13|PM|first|afternoon|lunch| +52454|AAAAAAAAHOMMAAAA|52454|14|34|14|PM|first|afternoon|lunch| +52455|AAAAAAAAIOMMAAAA|52455|14|34|15|PM|first|afternoon|lunch| +52456|AAAAAAAAJOMMAAAA|52456|14|34|16|PM|first|afternoon|lunch| +52457|AAAAAAAAKOMMAAAA|52457|14|34|17|PM|first|afternoon|lunch| +52458|AAAAAAAALOMMAAAA|52458|14|34|18|PM|first|afternoon|lunch| +52459|AAAAAAAAMOMMAAAA|52459|14|34|19|PM|first|afternoon|lunch| +52460|AAAAAAAANOMMAAAA|52460|14|34|20|PM|first|afternoon|lunch| +52461|AAAAAAAAOOMMAAAA|52461|14|34|21|PM|first|afternoon|lunch| +52462|AAAAAAAAPOMMAAAA|52462|14|34|22|PM|first|afternoon|lunch| +52463|AAAAAAAAAPMMAAAA|52463|14|34|23|PM|first|afternoon|lunch| +52464|AAAAAAAABPMMAAAA|52464|14|34|24|PM|first|afternoon|lunch| +52465|AAAAAAAACPMMAAAA|52465|14|34|25|PM|first|afternoon|lunch| +52466|AAAAAAAADPMMAAAA|52466|14|34|26|PM|first|afternoon|lunch| +52467|AAAAAAAAEPMMAAAA|52467|14|34|27|PM|first|afternoon|lunch| +52468|AAAAAAAAFPMMAAAA|52468|14|34|28|PM|first|afternoon|lunch| +52469|AAAAAAAAGPMMAAAA|52469|14|34|29|PM|first|afternoon|lunch| +52470|AAAAAAAAHPMMAAAA|52470|14|34|30|PM|first|afternoon|lunch| +52471|AAAAAAAAIPMMAAAA|52471|14|34|31|PM|first|afternoon|lunch| +52472|AAAAAAAAJPMMAAAA|52472|14|34|32|PM|first|afternoon|lunch| +52473|AAAAAAAAKPMMAAAA|52473|14|34|33|PM|first|afternoon|lunch| +52474|AAAAAAAALPMMAAAA|52474|14|34|34|PM|first|afternoon|lunch| +52475|AAAAAAAAMPMMAAAA|52475|14|34|35|PM|first|afternoon|lunch| +52476|AAAAAAAANPMMAAAA|52476|14|34|36|PM|first|afternoon|lunch| +52477|AAAAAAAAOPMMAAAA|52477|14|34|37|PM|first|afternoon|lunch| +52478|AAAAAAAAPPMMAAAA|52478|14|34|38|PM|first|afternoon|lunch| +52479|AAAAAAAAAANMAAAA|52479|14|34|39|PM|first|afternoon|lunch| +52480|AAAAAAAABANMAAAA|52480|14|34|40|PM|first|afternoon|lunch| +52481|AAAAAAAACANMAAAA|52481|14|34|41|PM|first|afternoon|lunch| +52482|AAAAAAAADANMAAAA|52482|14|34|42|PM|first|afternoon|lunch| +52483|AAAAAAAAEANMAAAA|52483|14|34|43|PM|first|afternoon|lunch| +52484|AAAAAAAAFANMAAAA|52484|14|34|44|PM|first|afternoon|lunch| +52485|AAAAAAAAGANMAAAA|52485|14|34|45|PM|first|afternoon|lunch| +52486|AAAAAAAAHANMAAAA|52486|14|34|46|PM|first|afternoon|lunch| +52487|AAAAAAAAIANMAAAA|52487|14|34|47|PM|first|afternoon|lunch| +52488|AAAAAAAAJANMAAAA|52488|14|34|48|PM|first|afternoon|lunch| +52489|AAAAAAAAKANMAAAA|52489|14|34|49|PM|first|afternoon|lunch| +52490|AAAAAAAALANMAAAA|52490|14|34|50|PM|first|afternoon|lunch| +52491|AAAAAAAAMANMAAAA|52491|14|34|51|PM|first|afternoon|lunch| +52492|AAAAAAAANANMAAAA|52492|14|34|52|PM|first|afternoon|lunch| +52493|AAAAAAAAOANMAAAA|52493|14|34|53|PM|first|afternoon|lunch| +52494|AAAAAAAAPANMAAAA|52494|14|34|54|PM|first|afternoon|lunch| +52495|AAAAAAAAABNMAAAA|52495|14|34|55|PM|first|afternoon|lunch| +52496|AAAAAAAABBNMAAAA|52496|14|34|56|PM|first|afternoon|lunch| +52497|AAAAAAAACBNMAAAA|52497|14|34|57|PM|first|afternoon|lunch| +52498|AAAAAAAADBNMAAAA|52498|14|34|58|PM|first|afternoon|lunch| +52499|AAAAAAAAEBNMAAAA|52499|14|34|59|PM|first|afternoon|lunch| +52500|AAAAAAAAFBNMAAAA|52500|14|35|0|PM|first|afternoon|lunch| +52501|AAAAAAAAGBNMAAAA|52501|14|35|1|PM|first|afternoon|lunch| +52502|AAAAAAAAHBNMAAAA|52502|14|35|2|PM|first|afternoon|lunch| +52503|AAAAAAAAIBNMAAAA|52503|14|35|3|PM|first|afternoon|lunch| +52504|AAAAAAAAJBNMAAAA|52504|14|35|4|PM|first|afternoon|lunch| +52505|AAAAAAAAKBNMAAAA|52505|14|35|5|PM|first|afternoon|lunch| +52506|AAAAAAAALBNMAAAA|52506|14|35|6|PM|first|afternoon|lunch| +52507|AAAAAAAAMBNMAAAA|52507|14|35|7|PM|first|afternoon|lunch| +52508|AAAAAAAANBNMAAAA|52508|14|35|8|PM|first|afternoon|lunch| +52509|AAAAAAAAOBNMAAAA|52509|14|35|9|PM|first|afternoon|lunch| +52510|AAAAAAAAPBNMAAAA|52510|14|35|10|PM|first|afternoon|lunch| +52511|AAAAAAAAACNMAAAA|52511|14|35|11|PM|first|afternoon|lunch| +52512|AAAAAAAABCNMAAAA|52512|14|35|12|PM|first|afternoon|lunch| +52513|AAAAAAAACCNMAAAA|52513|14|35|13|PM|first|afternoon|lunch| +52514|AAAAAAAADCNMAAAA|52514|14|35|14|PM|first|afternoon|lunch| +52515|AAAAAAAAECNMAAAA|52515|14|35|15|PM|first|afternoon|lunch| +52516|AAAAAAAAFCNMAAAA|52516|14|35|16|PM|first|afternoon|lunch| +52517|AAAAAAAAGCNMAAAA|52517|14|35|17|PM|first|afternoon|lunch| +52518|AAAAAAAAHCNMAAAA|52518|14|35|18|PM|first|afternoon|lunch| +52519|AAAAAAAAICNMAAAA|52519|14|35|19|PM|first|afternoon|lunch| +52520|AAAAAAAAJCNMAAAA|52520|14|35|20|PM|first|afternoon|lunch| +52521|AAAAAAAAKCNMAAAA|52521|14|35|21|PM|first|afternoon|lunch| +52522|AAAAAAAALCNMAAAA|52522|14|35|22|PM|first|afternoon|lunch| +52523|AAAAAAAAMCNMAAAA|52523|14|35|23|PM|first|afternoon|lunch| +52524|AAAAAAAANCNMAAAA|52524|14|35|24|PM|first|afternoon|lunch| +52525|AAAAAAAAOCNMAAAA|52525|14|35|25|PM|first|afternoon|lunch| +52526|AAAAAAAAPCNMAAAA|52526|14|35|26|PM|first|afternoon|lunch| +52527|AAAAAAAAADNMAAAA|52527|14|35|27|PM|first|afternoon|lunch| +52528|AAAAAAAABDNMAAAA|52528|14|35|28|PM|first|afternoon|lunch| +52529|AAAAAAAACDNMAAAA|52529|14|35|29|PM|first|afternoon|lunch| +52530|AAAAAAAADDNMAAAA|52530|14|35|30|PM|first|afternoon|lunch| +52531|AAAAAAAAEDNMAAAA|52531|14|35|31|PM|first|afternoon|lunch| +52532|AAAAAAAAFDNMAAAA|52532|14|35|32|PM|first|afternoon|lunch| +52533|AAAAAAAAGDNMAAAA|52533|14|35|33|PM|first|afternoon|lunch| +52534|AAAAAAAAHDNMAAAA|52534|14|35|34|PM|first|afternoon|lunch| +52535|AAAAAAAAIDNMAAAA|52535|14|35|35|PM|first|afternoon|lunch| +52536|AAAAAAAAJDNMAAAA|52536|14|35|36|PM|first|afternoon|lunch| +52537|AAAAAAAAKDNMAAAA|52537|14|35|37|PM|first|afternoon|lunch| +52538|AAAAAAAALDNMAAAA|52538|14|35|38|PM|first|afternoon|lunch| +52539|AAAAAAAAMDNMAAAA|52539|14|35|39|PM|first|afternoon|lunch| +52540|AAAAAAAANDNMAAAA|52540|14|35|40|PM|first|afternoon|lunch| +52541|AAAAAAAAODNMAAAA|52541|14|35|41|PM|first|afternoon|lunch| +52542|AAAAAAAAPDNMAAAA|52542|14|35|42|PM|first|afternoon|lunch| +52543|AAAAAAAAAENMAAAA|52543|14|35|43|PM|first|afternoon|lunch| +52544|AAAAAAAABENMAAAA|52544|14|35|44|PM|first|afternoon|lunch| +52545|AAAAAAAACENMAAAA|52545|14|35|45|PM|first|afternoon|lunch| +52546|AAAAAAAADENMAAAA|52546|14|35|46|PM|first|afternoon|lunch| +52547|AAAAAAAAEENMAAAA|52547|14|35|47|PM|first|afternoon|lunch| +52548|AAAAAAAAFENMAAAA|52548|14|35|48|PM|first|afternoon|lunch| +52549|AAAAAAAAGENMAAAA|52549|14|35|49|PM|first|afternoon|lunch| +52550|AAAAAAAAHENMAAAA|52550|14|35|50|PM|first|afternoon|lunch| +52551|AAAAAAAAIENMAAAA|52551|14|35|51|PM|first|afternoon|lunch| +52552|AAAAAAAAJENMAAAA|52552|14|35|52|PM|first|afternoon|lunch| +52553|AAAAAAAAKENMAAAA|52553|14|35|53|PM|first|afternoon|lunch| +52554|AAAAAAAALENMAAAA|52554|14|35|54|PM|first|afternoon|lunch| +52555|AAAAAAAAMENMAAAA|52555|14|35|55|PM|first|afternoon|lunch| +52556|AAAAAAAANENMAAAA|52556|14|35|56|PM|first|afternoon|lunch| +52557|AAAAAAAAOENMAAAA|52557|14|35|57|PM|first|afternoon|lunch| +52558|AAAAAAAAPENMAAAA|52558|14|35|58|PM|first|afternoon|lunch| +52559|AAAAAAAAAFNMAAAA|52559|14|35|59|PM|first|afternoon|lunch| +52560|AAAAAAAABFNMAAAA|52560|14|36|0|PM|first|afternoon|lunch| +52561|AAAAAAAACFNMAAAA|52561|14|36|1|PM|first|afternoon|lunch| +52562|AAAAAAAADFNMAAAA|52562|14|36|2|PM|first|afternoon|lunch| +52563|AAAAAAAAEFNMAAAA|52563|14|36|3|PM|first|afternoon|lunch| +52564|AAAAAAAAFFNMAAAA|52564|14|36|4|PM|first|afternoon|lunch| +52565|AAAAAAAAGFNMAAAA|52565|14|36|5|PM|first|afternoon|lunch| +52566|AAAAAAAAHFNMAAAA|52566|14|36|6|PM|first|afternoon|lunch| +52567|AAAAAAAAIFNMAAAA|52567|14|36|7|PM|first|afternoon|lunch| +52568|AAAAAAAAJFNMAAAA|52568|14|36|8|PM|first|afternoon|lunch| +52569|AAAAAAAAKFNMAAAA|52569|14|36|9|PM|first|afternoon|lunch| +52570|AAAAAAAALFNMAAAA|52570|14|36|10|PM|first|afternoon|lunch| +52571|AAAAAAAAMFNMAAAA|52571|14|36|11|PM|first|afternoon|lunch| +52572|AAAAAAAANFNMAAAA|52572|14|36|12|PM|first|afternoon|lunch| +52573|AAAAAAAAOFNMAAAA|52573|14|36|13|PM|first|afternoon|lunch| +52574|AAAAAAAAPFNMAAAA|52574|14|36|14|PM|first|afternoon|lunch| +52575|AAAAAAAAAGNMAAAA|52575|14|36|15|PM|first|afternoon|lunch| +52576|AAAAAAAABGNMAAAA|52576|14|36|16|PM|first|afternoon|lunch| +52577|AAAAAAAACGNMAAAA|52577|14|36|17|PM|first|afternoon|lunch| +52578|AAAAAAAADGNMAAAA|52578|14|36|18|PM|first|afternoon|lunch| +52579|AAAAAAAAEGNMAAAA|52579|14|36|19|PM|first|afternoon|lunch| +52580|AAAAAAAAFGNMAAAA|52580|14|36|20|PM|first|afternoon|lunch| +52581|AAAAAAAAGGNMAAAA|52581|14|36|21|PM|first|afternoon|lunch| +52582|AAAAAAAAHGNMAAAA|52582|14|36|22|PM|first|afternoon|lunch| +52583|AAAAAAAAIGNMAAAA|52583|14|36|23|PM|first|afternoon|lunch| +52584|AAAAAAAAJGNMAAAA|52584|14|36|24|PM|first|afternoon|lunch| +52585|AAAAAAAAKGNMAAAA|52585|14|36|25|PM|first|afternoon|lunch| +52586|AAAAAAAALGNMAAAA|52586|14|36|26|PM|first|afternoon|lunch| +52587|AAAAAAAAMGNMAAAA|52587|14|36|27|PM|first|afternoon|lunch| +52588|AAAAAAAANGNMAAAA|52588|14|36|28|PM|first|afternoon|lunch| +52589|AAAAAAAAOGNMAAAA|52589|14|36|29|PM|first|afternoon|lunch| +52590|AAAAAAAAPGNMAAAA|52590|14|36|30|PM|first|afternoon|lunch| +52591|AAAAAAAAAHNMAAAA|52591|14|36|31|PM|first|afternoon|lunch| +52592|AAAAAAAABHNMAAAA|52592|14|36|32|PM|first|afternoon|lunch| +52593|AAAAAAAACHNMAAAA|52593|14|36|33|PM|first|afternoon|lunch| +52594|AAAAAAAADHNMAAAA|52594|14|36|34|PM|first|afternoon|lunch| +52595|AAAAAAAAEHNMAAAA|52595|14|36|35|PM|first|afternoon|lunch| +52596|AAAAAAAAFHNMAAAA|52596|14|36|36|PM|first|afternoon|lunch| +52597|AAAAAAAAGHNMAAAA|52597|14|36|37|PM|first|afternoon|lunch| +52598|AAAAAAAAHHNMAAAA|52598|14|36|38|PM|first|afternoon|lunch| +52599|AAAAAAAAIHNMAAAA|52599|14|36|39|PM|first|afternoon|lunch| +52600|AAAAAAAAJHNMAAAA|52600|14|36|40|PM|first|afternoon|lunch| +52601|AAAAAAAAKHNMAAAA|52601|14|36|41|PM|first|afternoon|lunch| +52602|AAAAAAAALHNMAAAA|52602|14|36|42|PM|first|afternoon|lunch| +52603|AAAAAAAAMHNMAAAA|52603|14|36|43|PM|first|afternoon|lunch| +52604|AAAAAAAANHNMAAAA|52604|14|36|44|PM|first|afternoon|lunch| +52605|AAAAAAAAOHNMAAAA|52605|14|36|45|PM|first|afternoon|lunch| +52606|AAAAAAAAPHNMAAAA|52606|14|36|46|PM|first|afternoon|lunch| +52607|AAAAAAAAAINMAAAA|52607|14|36|47|PM|first|afternoon|lunch| +52608|AAAAAAAABINMAAAA|52608|14|36|48|PM|first|afternoon|lunch| +52609|AAAAAAAACINMAAAA|52609|14|36|49|PM|first|afternoon|lunch| +52610|AAAAAAAADINMAAAA|52610|14|36|50|PM|first|afternoon|lunch| +52611|AAAAAAAAEINMAAAA|52611|14|36|51|PM|first|afternoon|lunch| +52612|AAAAAAAAFINMAAAA|52612|14|36|52|PM|first|afternoon|lunch| +52613|AAAAAAAAGINMAAAA|52613|14|36|53|PM|first|afternoon|lunch| +52614|AAAAAAAAHINMAAAA|52614|14|36|54|PM|first|afternoon|lunch| +52615|AAAAAAAAIINMAAAA|52615|14|36|55|PM|first|afternoon|lunch| +52616|AAAAAAAAJINMAAAA|52616|14|36|56|PM|first|afternoon|lunch| +52617|AAAAAAAAKINMAAAA|52617|14|36|57|PM|first|afternoon|lunch| +52618|AAAAAAAALINMAAAA|52618|14|36|58|PM|first|afternoon|lunch| +52619|AAAAAAAAMINMAAAA|52619|14|36|59|PM|first|afternoon|lunch| +52620|AAAAAAAANINMAAAA|52620|14|37|0|PM|first|afternoon|lunch| +52621|AAAAAAAAOINMAAAA|52621|14|37|1|PM|first|afternoon|lunch| +52622|AAAAAAAAPINMAAAA|52622|14|37|2|PM|first|afternoon|lunch| +52623|AAAAAAAAAJNMAAAA|52623|14|37|3|PM|first|afternoon|lunch| +52624|AAAAAAAABJNMAAAA|52624|14|37|4|PM|first|afternoon|lunch| +52625|AAAAAAAACJNMAAAA|52625|14|37|5|PM|first|afternoon|lunch| +52626|AAAAAAAADJNMAAAA|52626|14|37|6|PM|first|afternoon|lunch| +52627|AAAAAAAAEJNMAAAA|52627|14|37|7|PM|first|afternoon|lunch| +52628|AAAAAAAAFJNMAAAA|52628|14|37|8|PM|first|afternoon|lunch| +52629|AAAAAAAAGJNMAAAA|52629|14|37|9|PM|first|afternoon|lunch| +52630|AAAAAAAAHJNMAAAA|52630|14|37|10|PM|first|afternoon|lunch| +52631|AAAAAAAAIJNMAAAA|52631|14|37|11|PM|first|afternoon|lunch| +52632|AAAAAAAAJJNMAAAA|52632|14|37|12|PM|first|afternoon|lunch| +52633|AAAAAAAAKJNMAAAA|52633|14|37|13|PM|first|afternoon|lunch| +52634|AAAAAAAALJNMAAAA|52634|14|37|14|PM|first|afternoon|lunch| +52635|AAAAAAAAMJNMAAAA|52635|14|37|15|PM|first|afternoon|lunch| +52636|AAAAAAAANJNMAAAA|52636|14|37|16|PM|first|afternoon|lunch| +52637|AAAAAAAAOJNMAAAA|52637|14|37|17|PM|first|afternoon|lunch| +52638|AAAAAAAAPJNMAAAA|52638|14|37|18|PM|first|afternoon|lunch| +52639|AAAAAAAAAKNMAAAA|52639|14|37|19|PM|first|afternoon|lunch| +52640|AAAAAAAABKNMAAAA|52640|14|37|20|PM|first|afternoon|lunch| +52641|AAAAAAAACKNMAAAA|52641|14|37|21|PM|first|afternoon|lunch| +52642|AAAAAAAADKNMAAAA|52642|14|37|22|PM|first|afternoon|lunch| +52643|AAAAAAAAEKNMAAAA|52643|14|37|23|PM|first|afternoon|lunch| +52644|AAAAAAAAFKNMAAAA|52644|14|37|24|PM|first|afternoon|lunch| +52645|AAAAAAAAGKNMAAAA|52645|14|37|25|PM|first|afternoon|lunch| +52646|AAAAAAAAHKNMAAAA|52646|14|37|26|PM|first|afternoon|lunch| +52647|AAAAAAAAIKNMAAAA|52647|14|37|27|PM|first|afternoon|lunch| +52648|AAAAAAAAJKNMAAAA|52648|14|37|28|PM|first|afternoon|lunch| +52649|AAAAAAAAKKNMAAAA|52649|14|37|29|PM|first|afternoon|lunch| +52650|AAAAAAAALKNMAAAA|52650|14|37|30|PM|first|afternoon|lunch| +52651|AAAAAAAAMKNMAAAA|52651|14|37|31|PM|first|afternoon|lunch| +52652|AAAAAAAANKNMAAAA|52652|14|37|32|PM|first|afternoon|lunch| +52653|AAAAAAAAOKNMAAAA|52653|14|37|33|PM|first|afternoon|lunch| +52654|AAAAAAAAPKNMAAAA|52654|14|37|34|PM|first|afternoon|lunch| +52655|AAAAAAAAALNMAAAA|52655|14|37|35|PM|first|afternoon|lunch| +52656|AAAAAAAABLNMAAAA|52656|14|37|36|PM|first|afternoon|lunch| +52657|AAAAAAAACLNMAAAA|52657|14|37|37|PM|first|afternoon|lunch| +52658|AAAAAAAADLNMAAAA|52658|14|37|38|PM|first|afternoon|lunch| +52659|AAAAAAAAELNMAAAA|52659|14|37|39|PM|first|afternoon|lunch| +52660|AAAAAAAAFLNMAAAA|52660|14|37|40|PM|first|afternoon|lunch| +52661|AAAAAAAAGLNMAAAA|52661|14|37|41|PM|first|afternoon|lunch| +52662|AAAAAAAAHLNMAAAA|52662|14|37|42|PM|first|afternoon|lunch| +52663|AAAAAAAAILNMAAAA|52663|14|37|43|PM|first|afternoon|lunch| +52664|AAAAAAAAJLNMAAAA|52664|14|37|44|PM|first|afternoon|lunch| +52665|AAAAAAAAKLNMAAAA|52665|14|37|45|PM|first|afternoon|lunch| +52666|AAAAAAAALLNMAAAA|52666|14|37|46|PM|first|afternoon|lunch| +52667|AAAAAAAAMLNMAAAA|52667|14|37|47|PM|first|afternoon|lunch| +52668|AAAAAAAANLNMAAAA|52668|14|37|48|PM|first|afternoon|lunch| +52669|AAAAAAAAOLNMAAAA|52669|14|37|49|PM|first|afternoon|lunch| +52670|AAAAAAAAPLNMAAAA|52670|14|37|50|PM|first|afternoon|lunch| +52671|AAAAAAAAAMNMAAAA|52671|14|37|51|PM|first|afternoon|lunch| +52672|AAAAAAAABMNMAAAA|52672|14|37|52|PM|first|afternoon|lunch| +52673|AAAAAAAACMNMAAAA|52673|14|37|53|PM|first|afternoon|lunch| +52674|AAAAAAAADMNMAAAA|52674|14|37|54|PM|first|afternoon|lunch| +52675|AAAAAAAAEMNMAAAA|52675|14|37|55|PM|first|afternoon|lunch| +52676|AAAAAAAAFMNMAAAA|52676|14|37|56|PM|first|afternoon|lunch| +52677|AAAAAAAAGMNMAAAA|52677|14|37|57|PM|first|afternoon|lunch| +52678|AAAAAAAAHMNMAAAA|52678|14|37|58|PM|first|afternoon|lunch| +52679|AAAAAAAAIMNMAAAA|52679|14|37|59|PM|first|afternoon|lunch| +52680|AAAAAAAAJMNMAAAA|52680|14|38|0|PM|first|afternoon|lunch| +52681|AAAAAAAAKMNMAAAA|52681|14|38|1|PM|first|afternoon|lunch| +52682|AAAAAAAALMNMAAAA|52682|14|38|2|PM|first|afternoon|lunch| +52683|AAAAAAAAMMNMAAAA|52683|14|38|3|PM|first|afternoon|lunch| +52684|AAAAAAAANMNMAAAA|52684|14|38|4|PM|first|afternoon|lunch| +52685|AAAAAAAAOMNMAAAA|52685|14|38|5|PM|first|afternoon|lunch| +52686|AAAAAAAAPMNMAAAA|52686|14|38|6|PM|first|afternoon|lunch| +52687|AAAAAAAAANNMAAAA|52687|14|38|7|PM|first|afternoon|lunch| +52688|AAAAAAAABNNMAAAA|52688|14|38|8|PM|first|afternoon|lunch| +52689|AAAAAAAACNNMAAAA|52689|14|38|9|PM|first|afternoon|lunch| +52690|AAAAAAAADNNMAAAA|52690|14|38|10|PM|first|afternoon|lunch| +52691|AAAAAAAAENNMAAAA|52691|14|38|11|PM|first|afternoon|lunch| +52692|AAAAAAAAFNNMAAAA|52692|14|38|12|PM|first|afternoon|lunch| +52693|AAAAAAAAGNNMAAAA|52693|14|38|13|PM|first|afternoon|lunch| +52694|AAAAAAAAHNNMAAAA|52694|14|38|14|PM|first|afternoon|lunch| +52695|AAAAAAAAINNMAAAA|52695|14|38|15|PM|first|afternoon|lunch| +52696|AAAAAAAAJNNMAAAA|52696|14|38|16|PM|first|afternoon|lunch| +52697|AAAAAAAAKNNMAAAA|52697|14|38|17|PM|first|afternoon|lunch| +52698|AAAAAAAALNNMAAAA|52698|14|38|18|PM|first|afternoon|lunch| +52699|AAAAAAAAMNNMAAAA|52699|14|38|19|PM|first|afternoon|lunch| +52700|AAAAAAAANNNMAAAA|52700|14|38|20|PM|first|afternoon|lunch| +52701|AAAAAAAAONNMAAAA|52701|14|38|21|PM|first|afternoon|lunch| +52702|AAAAAAAAPNNMAAAA|52702|14|38|22|PM|first|afternoon|lunch| +52703|AAAAAAAAAONMAAAA|52703|14|38|23|PM|first|afternoon|lunch| +52704|AAAAAAAABONMAAAA|52704|14|38|24|PM|first|afternoon|lunch| +52705|AAAAAAAACONMAAAA|52705|14|38|25|PM|first|afternoon|lunch| +52706|AAAAAAAADONMAAAA|52706|14|38|26|PM|first|afternoon|lunch| +52707|AAAAAAAAEONMAAAA|52707|14|38|27|PM|first|afternoon|lunch| +52708|AAAAAAAAFONMAAAA|52708|14|38|28|PM|first|afternoon|lunch| +52709|AAAAAAAAGONMAAAA|52709|14|38|29|PM|first|afternoon|lunch| +52710|AAAAAAAAHONMAAAA|52710|14|38|30|PM|first|afternoon|lunch| +52711|AAAAAAAAIONMAAAA|52711|14|38|31|PM|first|afternoon|lunch| +52712|AAAAAAAAJONMAAAA|52712|14|38|32|PM|first|afternoon|lunch| +52713|AAAAAAAAKONMAAAA|52713|14|38|33|PM|first|afternoon|lunch| +52714|AAAAAAAALONMAAAA|52714|14|38|34|PM|first|afternoon|lunch| +52715|AAAAAAAAMONMAAAA|52715|14|38|35|PM|first|afternoon|lunch| +52716|AAAAAAAANONMAAAA|52716|14|38|36|PM|first|afternoon|lunch| +52717|AAAAAAAAOONMAAAA|52717|14|38|37|PM|first|afternoon|lunch| +52718|AAAAAAAAPONMAAAA|52718|14|38|38|PM|first|afternoon|lunch| +52719|AAAAAAAAAPNMAAAA|52719|14|38|39|PM|first|afternoon|lunch| +52720|AAAAAAAABPNMAAAA|52720|14|38|40|PM|first|afternoon|lunch| +52721|AAAAAAAACPNMAAAA|52721|14|38|41|PM|first|afternoon|lunch| +52722|AAAAAAAADPNMAAAA|52722|14|38|42|PM|first|afternoon|lunch| +52723|AAAAAAAAEPNMAAAA|52723|14|38|43|PM|first|afternoon|lunch| +52724|AAAAAAAAFPNMAAAA|52724|14|38|44|PM|first|afternoon|lunch| +52725|AAAAAAAAGPNMAAAA|52725|14|38|45|PM|first|afternoon|lunch| +52726|AAAAAAAAHPNMAAAA|52726|14|38|46|PM|first|afternoon|lunch| +52727|AAAAAAAAIPNMAAAA|52727|14|38|47|PM|first|afternoon|lunch| +52728|AAAAAAAAJPNMAAAA|52728|14|38|48|PM|first|afternoon|lunch| +52729|AAAAAAAAKPNMAAAA|52729|14|38|49|PM|first|afternoon|lunch| +52730|AAAAAAAALPNMAAAA|52730|14|38|50|PM|first|afternoon|lunch| +52731|AAAAAAAAMPNMAAAA|52731|14|38|51|PM|first|afternoon|lunch| +52732|AAAAAAAANPNMAAAA|52732|14|38|52|PM|first|afternoon|lunch| +52733|AAAAAAAAOPNMAAAA|52733|14|38|53|PM|first|afternoon|lunch| +52734|AAAAAAAAPPNMAAAA|52734|14|38|54|PM|first|afternoon|lunch| +52735|AAAAAAAAAAOMAAAA|52735|14|38|55|PM|first|afternoon|lunch| +52736|AAAAAAAABAOMAAAA|52736|14|38|56|PM|first|afternoon|lunch| +52737|AAAAAAAACAOMAAAA|52737|14|38|57|PM|first|afternoon|lunch| +52738|AAAAAAAADAOMAAAA|52738|14|38|58|PM|first|afternoon|lunch| +52739|AAAAAAAAEAOMAAAA|52739|14|38|59|PM|first|afternoon|lunch| +52740|AAAAAAAAFAOMAAAA|52740|14|39|0|PM|first|afternoon|lunch| +52741|AAAAAAAAGAOMAAAA|52741|14|39|1|PM|first|afternoon|lunch| +52742|AAAAAAAAHAOMAAAA|52742|14|39|2|PM|first|afternoon|lunch| +52743|AAAAAAAAIAOMAAAA|52743|14|39|3|PM|first|afternoon|lunch| +52744|AAAAAAAAJAOMAAAA|52744|14|39|4|PM|first|afternoon|lunch| +52745|AAAAAAAAKAOMAAAA|52745|14|39|5|PM|first|afternoon|lunch| +52746|AAAAAAAALAOMAAAA|52746|14|39|6|PM|first|afternoon|lunch| +52747|AAAAAAAAMAOMAAAA|52747|14|39|7|PM|first|afternoon|lunch| +52748|AAAAAAAANAOMAAAA|52748|14|39|8|PM|first|afternoon|lunch| +52749|AAAAAAAAOAOMAAAA|52749|14|39|9|PM|first|afternoon|lunch| +52750|AAAAAAAAPAOMAAAA|52750|14|39|10|PM|first|afternoon|lunch| +52751|AAAAAAAAABOMAAAA|52751|14|39|11|PM|first|afternoon|lunch| +52752|AAAAAAAABBOMAAAA|52752|14|39|12|PM|first|afternoon|lunch| +52753|AAAAAAAACBOMAAAA|52753|14|39|13|PM|first|afternoon|lunch| +52754|AAAAAAAADBOMAAAA|52754|14|39|14|PM|first|afternoon|lunch| +52755|AAAAAAAAEBOMAAAA|52755|14|39|15|PM|first|afternoon|lunch| +52756|AAAAAAAAFBOMAAAA|52756|14|39|16|PM|first|afternoon|lunch| +52757|AAAAAAAAGBOMAAAA|52757|14|39|17|PM|first|afternoon|lunch| +52758|AAAAAAAAHBOMAAAA|52758|14|39|18|PM|first|afternoon|lunch| +52759|AAAAAAAAIBOMAAAA|52759|14|39|19|PM|first|afternoon|lunch| +52760|AAAAAAAAJBOMAAAA|52760|14|39|20|PM|first|afternoon|lunch| +52761|AAAAAAAAKBOMAAAA|52761|14|39|21|PM|first|afternoon|lunch| +52762|AAAAAAAALBOMAAAA|52762|14|39|22|PM|first|afternoon|lunch| +52763|AAAAAAAAMBOMAAAA|52763|14|39|23|PM|first|afternoon|lunch| +52764|AAAAAAAANBOMAAAA|52764|14|39|24|PM|first|afternoon|lunch| +52765|AAAAAAAAOBOMAAAA|52765|14|39|25|PM|first|afternoon|lunch| +52766|AAAAAAAAPBOMAAAA|52766|14|39|26|PM|first|afternoon|lunch| +52767|AAAAAAAAACOMAAAA|52767|14|39|27|PM|first|afternoon|lunch| +52768|AAAAAAAABCOMAAAA|52768|14|39|28|PM|first|afternoon|lunch| +52769|AAAAAAAACCOMAAAA|52769|14|39|29|PM|first|afternoon|lunch| +52770|AAAAAAAADCOMAAAA|52770|14|39|30|PM|first|afternoon|lunch| +52771|AAAAAAAAECOMAAAA|52771|14|39|31|PM|first|afternoon|lunch| +52772|AAAAAAAAFCOMAAAA|52772|14|39|32|PM|first|afternoon|lunch| +52773|AAAAAAAAGCOMAAAA|52773|14|39|33|PM|first|afternoon|lunch| +52774|AAAAAAAAHCOMAAAA|52774|14|39|34|PM|first|afternoon|lunch| +52775|AAAAAAAAICOMAAAA|52775|14|39|35|PM|first|afternoon|lunch| +52776|AAAAAAAAJCOMAAAA|52776|14|39|36|PM|first|afternoon|lunch| +52777|AAAAAAAAKCOMAAAA|52777|14|39|37|PM|first|afternoon|lunch| +52778|AAAAAAAALCOMAAAA|52778|14|39|38|PM|first|afternoon|lunch| +52779|AAAAAAAAMCOMAAAA|52779|14|39|39|PM|first|afternoon|lunch| +52780|AAAAAAAANCOMAAAA|52780|14|39|40|PM|first|afternoon|lunch| +52781|AAAAAAAAOCOMAAAA|52781|14|39|41|PM|first|afternoon|lunch| +52782|AAAAAAAAPCOMAAAA|52782|14|39|42|PM|first|afternoon|lunch| +52783|AAAAAAAAADOMAAAA|52783|14|39|43|PM|first|afternoon|lunch| +52784|AAAAAAAABDOMAAAA|52784|14|39|44|PM|first|afternoon|lunch| +52785|AAAAAAAACDOMAAAA|52785|14|39|45|PM|first|afternoon|lunch| +52786|AAAAAAAADDOMAAAA|52786|14|39|46|PM|first|afternoon|lunch| +52787|AAAAAAAAEDOMAAAA|52787|14|39|47|PM|first|afternoon|lunch| +52788|AAAAAAAAFDOMAAAA|52788|14|39|48|PM|first|afternoon|lunch| +52789|AAAAAAAAGDOMAAAA|52789|14|39|49|PM|first|afternoon|lunch| +52790|AAAAAAAAHDOMAAAA|52790|14|39|50|PM|first|afternoon|lunch| +52791|AAAAAAAAIDOMAAAA|52791|14|39|51|PM|first|afternoon|lunch| +52792|AAAAAAAAJDOMAAAA|52792|14|39|52|PM|first|afternoon|lunch| +52793|AAAAAAAAKDOMAAAA|52793|14|39|53|PM|first|afternoon|lunch| +52794|AAAAAAAALDOMAAAA|52794|14|39|54|PM|first|afternoon|lunch| +52795|AAAAAAAAMDOMAAAA|52795|14|39|55|PM|first|afternoon|lunch| +52796|AAAAAAAANDOMAAAA|52796|14|39|56|PM|first|afternoon|lunch| +52797|AAAAAAAAODOMAAAA|52797|14|39|57|PM|first|afternoon|lunch| +52798|AAAAAAAAPDOMAAAA|52798|14|39|58|PM|first|afternoon|lunch| +52799|AAAAAAAAAEOMAAAA|52799|14|39|59|PM|first|afternoon|lunch| +52800|AAAAAAAABEOMAAAA|52800|14|40|0|PM|first|afternoon|lunch| +52801|AAAAAAAACEOMAAAA|52801|14|40|1|PM|first|afternoon|lunch| +52802|AAAAAAAADEOMAAAA|52802|14|40|2|PM|first|afternoon|lunch| +52803|AAAAAAAAEEOMAAAA|52803|14|40|3|PM|first|afternoon|lunch| +52804|AAAAAAAAFEOMAAAA|52804|14|40|4|PM|first|afternoon|lunch| +52805|AAAAAAAAGEOMAAAA|52805|14|40|5|PM|first|afternoon|lunch| +52806|AAAAAAAAHEOMAAAA|52806|14|40|6|PM|first|afternoon|lunch| +52807|AAAAAAAAIEOMAAAA|52807|14|40|7|PM|first|afternoon|lunch| +52808|AAAAAAAAJEOMAAAA|52808|14|40|8|PM|first|afternoon|lunch| +52809|AAAAAAAAKEOMAAAA|52809|14|40|9|PM|first|afternoon|lunch| +52810|AAAAAAAALEOMAAAA|52810|14|40|10|PM|first|afternoon|lunch| +52811|AAAAAAAAMEOMAAAA|52811|14|40|11|PM|first|afternoon|lunch| +52812|AAAAAAAANEOMAAAA|52812|14|40|12|PM|first|afternoon|lunch| +52813|AAAAAAAAOEOMAAAA|52813|14|40|13|PM|first|afternoon|lunch| +52814|AAAAAAAAPEOMAAAA|52814|14|40|14|PM|first|afternoon|lunch| +52815|AAAAAAAAAFOMAAAA|52815|14|40|15|PM|first|afternoon|lunch| +52816|AAAAAAAABFOMAAAA|52816|14|40|16|PM|first|afternoon|lunch| +52817|AAAAAAAACFOMAAAA|52817|14|40|17|PM|first|afternoon|lunch| +52818|AAAAAAAADFOMAAAA|52818|14|40|18|PM|first|afternoon|lunch| +52819|AAAAAAAAEFOMAAAA|52819|14|40|19|PM|first|afternoon|lunch| +52820|AAAAAAAAFFOMAAAA|52820|14|40|20|PM|first|afternoon|lunch| +52821|AAAAAAAAGFOMAAAA|52821|14|40|21|PM|first|afternoon|lunch| +52822|AAAAAAAAHFOMAAAA|52822|14|40|22|PM|first|afternoon|lunch| +52823|AAAAAAAAIFOMAAAA|52823|14|40|23|PM|first|afternoon|lunch| +52824|AAAAAAAAJFOMAAAA|52824|14|40|24|PM|first|afternoon|lunch| +52825|AAAAAAAAKFOMAAAA|52825|14|40|25|PM|first|afternoon|lunch| +52826|AAAAAAAALFOMAAAA|52826|14|40|26|PM|first|afternoon|lunch| +52827|AAAAAAAAMFOMAAAA|52827|14|40|27|PM|first|afternoon|lunch| +52828|AAAAAAAANFOMAAAA|52828|14|40|28|PM|first|afternoon|lunch| +52829|AAAAAAAAOFOMAAAA|52829|14|40|29|PM|first|afternoon|lunch| +52830|AAAAAAAAPFOMAAAA|52830|14|40|30|PM|first|afternoon|lunch| +52831|AAAAAAAAAGOMAAAA|52831|14|40|31|PM|first|afternoon|lunch| +52832|AAAAAAAABGOMAAAA|52832|14|40|32|PM|first|afternoon|lunch| +52833|AAAAAAAACGOMAAAA|52833|14|40|33|PM|first|afternoon|lunch| +52834|AAAAAAAADGOMAAAA|52834|14|40|34|PM|first|afternoon|lunch| +52835|AAAAAAAAEGOMAAAA|52835|14|40|35|PM|first|afternoon|lunch| +52836|AAAAAAAAFGOMAAAA|52836|14|40|36|PM|first|afternoon|lunch| +52837|AAAAAAAAGGOMAAAA|52837|14|40|37|PM|first|afternoon|lunch| +52838|AAAAAAAAHGOMAAAA|52838|14|40|38|PM|first|afternoon|lunch| +52839|AAAAAAAAIGOMAAAA|52839|14|40|39|PM|first|afternoon|lunch| +52840|AAAAAAAAJGOMAAAA|52840|14|40|40|PM|first|afternoon|lunch| +52841|AAAAAAAAKGOMAAAA|52841|14|40|41|PM|first|afternoon|lunch| +52842|AAAAAAAALGOMAAAA|52842|14|40|42|PM|first|afternoon|lunch| +52843|AAAAAAAAMGOMAAAA|52843|14|40|43|PM|first|afternoon|lunch| +52844|AAAAAAAANGOMAAAA|52844|14|40|44|PM|first|afternoon|lunch| +52845|AAAAAAAAOGOMAAAA|52845|14|40|45|PM|first|afternoon|lunch| +52846|AAAAAAAAPGOMAAAA|52846|14|40|46|PM|first|afternoon|lunch| +52847|AAAAAAAAAHOMAAAA|52847|14|40|47|PM|first|afternoon|lunch| +52848|AAAAAAAABHOMAAAA|52848|14|40|48|PM|first|afternoon|lunch| +52849|AAAAAAAACHOMAAAA|52849|14|40|49|PM|first|afternoon|lunch| +52850|AAAAAAAADHOMAAAA|52850|14|40|50|PM|first|afternoon|lunch| +52851|AAAAAAAAEHOMAAAA|52851|14|40|51|PM|first|afternoon|lunch| +52852|AAAAAAAAFHOMAAAA|52852|14|40|52|PM|first|afternoon|lunch| +52853|AAAAAAAAGHOMAAAA|52853|14|40|53|PM|first|afternoon|lunch| +52854|AAAAAAAAHHOMAAAA|52854|14|40|54|PM|first|afternoon|lunch| +52855|AAAAAAAAIHOMAAAA|52855|14|40|55|PM|first|afternoon|lunch| +52856|AAAAAAAAJHOMAAAA|52856|14|40|56|PM|first|afternoon|lunch| +52857|AAAAAAAAKHOMAAAA|52857|14|40|57|PM|first|afternoon|lunch| +52858|AAAAAAAALHOMAAAA|52858|14|40|58|PM|first|afternoon|lunch| +52859|AAAAAAAAMHOMAAAA|52859|14|40|59|PM|first|afternoon|lunch| +52860|AAAAAAAANHOMAAAA|52860|14|41|0|PM|first|afternoon|lunch| +52861|AAAAAAAAOHOMAAAA|52861|14|41|1|PM|first|afternoon|lunch| +52862|AAAAAAAAPHOMAAAA|52862|14|41|2|PM|first|afternoon|lunch| +52863|AAAAAAAAAIOMAAAA|52863|14|41|3|PM|first|afternoon|lunch| +52864|AAAAAAAABIOMAAAA|52864|14|41|4|PM|first|afternoon|lunch| +52865|AAAAAAAACIOMAAAA|52865|14|41|5|PM|first|afternoon|lunch| +52866|AAAAAAAADIOMAAAA|52866|14|41|6|PM|first|afternoon|lunch| +52867|AAAAAAAAEIOMAAAA|52867|14|41|7|PM|first|afternoon|lunch| +52868|AAAAAAAAFIOMAAAA|52868|14|41|8|PM|first|afternoon|lunch| +52869|AAAAAAAAGIOMAAAA|52869|14|41|9|PM|first|afternoon|lunch| +52870|AAAAAAAAHIOMAAAA|52870|14|41|10|PM|first|afternoon|lunch| +52871|AAAAAAAAIIOMAAAA|52871|14|41|11|PM|first|afternoon|lunch| +52872|AAAAAAAAJIOMAAAA|52872|14|41|12|PM|first|afternoon|lunch| +52873|AAAAAAAAKIOMAAAA|52873|14|41|13|PM|first|afternoon|lunch| +52874|AAAAAAAALIOMAAAA|52874|14|41|14|PM|first|afternoon|lunch| +52875|AAAAAAAAMIOMAAAA|52875|14|41|15|PM|first|afternoon|lunch| +52876|AAAAAAAANIOMAAAA|52876|14|41|16|PM|first|afternoon|lunch| +52877|AAAAAAAAOIOMAAAA|52877|14|41|17|PM|first|afternoon|lunch| +52878|AAAAAAAAPIOMAAAA|52878|14|41|18|PM|first|afternoon|lunch| +52879|AAAAAAAAAJOMAAAA|52879|14|41|19|PM|first|afternoon|lunch| +52880|AAAAAAAABJOMAAAA|52880|14|41|20|PM|first|afternoon|lunch| +52881|AAAAAAAACJOMAAAA|52881|14|41|21|PM|first|afternoon|lunch| +52882|AAAAAAAADJOMAAAA|52882|14|41|22|PM|first|afternoon|lunch| +52883|AAAAAAAAEJOMAAAA|52883|14|41|23|PM|first|afternoon|lunch| +52884|AAAAAAAAFJOMAAAA|52884|14|41|24|PM|first|afternoon|lunch| +52885|AAAAAAAAGJOMAAAA|52885|14|41|25|PM|first|afternoon|lunch| +52886|AAAAAAAAHJOMAAAA|52886|14|41|26|PM|first|afternoon|lunch| +52887|AAAAAAAAIJOMAAAA|52887|14|41|27|PM|first|afternoon|lunch| +52888|AAAAAAAAJJOMAAAA|52888|14|41|28|PM|first|afternoon|lunch| +52889|AAAAAAAAKJOMAAAA|52889|14|41|29|PM|first|afternoon|lunch| +52890|AAAAAAAALJOMAAAA|52890|14|41|30|PM|first|afternoon|lunch| +52891|AAAAAAAAMJOMAAAA|52891|14|41|31|PM|first|afternoon|lunch| +52892|AAAAAAAANJOMAAAA|52892|14|41|32|PM|first|afternoon|lunch| +52893|AAAAAAAAOJOMAAAA|52893|14|41|33|PM|first|afternoon|lunch| +52894|AAAAAAAAPJOMAAAA|52894|14|41|34|PM|first|afternoon|lunch| +52895|AAAAAAAAAKOMAAAA|52895|14|41|35|PM|first|afternoon|lunch| +52896|AAAAAAAABKOMAAAA|52896|14|41|36|PM|first|afternoon|lunch| +52897|AAAAAAAACKOMAAAA|52897|14|41|37|PM|first|afternoon|lunch| +52898|AAAAAAAADKOMAAAA|52898|14|41|38|PM|first|afternoon|lunch| +52899|AAAAAAAAEKOMAAAA|52899|14|41|39|PM|first|afternoon|lunch| +52900|AAAAAAAAFKOMAAAA|52900|14|41|40|PM|first|afternoon|lunch| +52901|AAAAAAAAGKOMAAAA|52901|14|41|41|PM|first|afternoon|lunch| +52902|AAAAAAAAHKOMAAAA|52902|14|41|42|PM|first|afternoon|lunch| +52903|AAAAAAAAIKOMAAAA|52903|14|41|43|PM|first|afternoon|lunch| +52904|AAAAAAAAJKOMAAAA|52904|14|41|44|PM|first|afternoon|lunch| +52905|AAAAAAAAKKOMAAAA|52905|14|41|45|PM|first|afternoon|lunch| +52906|AAAAAAAALKOMAAAA|52906|14|41|46|PM|first|afternoon|lunch| +52907|AAAAAAAAMKOMAAAA|52907|14|41|47|PM|first|afternoon|lunch| +52908|AAAAAAAANKOMAAAA|52908|14|41|48|PM|first|afternoon|lunch| +52909|AAAAAAAAOKOMAAAA|52909|14|41|49|PM|first|afternoon|lunch| +52910|AAAAAAAAPKOMAAAA|52910|14|41|50|PM|first|afternoon|lunch| +52911|AAAAAAAAALOMAAAA|52911|14|41|51|PM|first|afternoon|lunch| +52912|AAAAAAAABLOMAAAA|52912|14|41|52|PM|first|afternoon|lunch| +52913|AAAAAAAACLOMAAAA|52913|14|41|53|PM|first|afternoon|lunch| +52914|AAAAAAAADLOMAAAA|52914|14|41|54|PM|first|afternoon|lunch| +52915|AAAAAAAAELOMAAAA|52915|14|41|55|PM|first|afternoon|lunch| +52916|AAAAAAAAFLOMAAAA|52916|14|41|56|PM|first|afternoon|lunch| +52917|AAAAAAAAGLOMAAAA|52917|14|41|57|PM|first|afternoon|lunch| +52918|AAAAAAAAHLOMAAAA|52918|14|41|58|PM|first|afternoon|lunch| +52919|AAAAAAAAILOMAAAA|52919|14|41|59|PM|first|afternoon|lunch| +52920|AAAAAAAAJLOMAAAA|52920|14|42|0|PM|first|afternoon|lunch| +52921|AAAAAAAAKLOMAAAA|52921|14|42|1|PM|first|afternoon|lunch| +52922|AAAAAAAALLOMAAAA|52922|14|42|2|PM|first|afternoon|lunch| +52923|AAAAAAAAMLOMAAAA|52923|14|42|3|PM|first|afternoon|lunch| +52924|AAAAAAAANLOMAAAA|52924|14|42|4|PM|first|afternoon|lunch| +52925|AAAAAAAAOLOMAAAA|52925|14|42|5|PM|first|afternoon|lunch| +52926|AAAAAAAAPLOMAAAA|52926|14|42|6|PM|first|afternoon|lunch| +52927|AAAAAAAAAMOMAAAA|52927|14|42|7|PM|first|afternoon|lunch| +52928|AAAAAAAABMOMAAAA|52928|14|42|8|PM|first|afternoon|lunch| +52929|AAAAAAAACMOMAAAA|52929|14|42|9|PM|first|afternoon|lunch| +52930|AAAAAAAADMOMAAAA|52930|14|42|10|PM|first|afternoon|lunch| +52931|AAAAAAAAEMOMAAAA|52931|14|42|11|PM|first|afternoon|lunch| +52932|AAAAAAAAFMOMAAAA|52932|14|42|12|PM|first|afternoon|lunch| +52933|AAAAAAAAGMOMAAAA|52933|14|42|13|PM|first|afternoon|lunch| +52934|AAAAAAAAHMOMAAAA|52934|14|42|14|PM|first|afternoon|lunch| +52935|AAAAAAAAIMOMAAAA|52935|14|42|15|PM|first|afternoon|lunch| +52936|AAAAAAAAJMOMAAAA|52936|14|42|16|PM|first|afternoon|lunch| +52937|AAAAAAAAKMOMAAAA|52937|14|42|17|PM|first|afternoon|lunch| +52938|AAAAAAAALMOMAAAA|52938|14|42|18|PM|first|afternoon|lunch| +52939|AAAAAAAAMMOMAAAA|52939|14|42|19|PM|first|afternoon|lunch| +52940|AAAAAAAANMOMAAAA|52940|14|42|20|PM|first|afternoon|lunch| +52941|AAAAAAAAOMOMAAAA|52941|14|42|21|PM|first|afternoon|lunch| +52942|AAAAAAAAPMOMAAAA|52942|14|42|22|PM|first|afternoon|lunch| +52943|AAAAAAAAANOMAAAA|52943|14|42|23|PM|first|afternoon|lunch| +52944|AAAAAAAABNOMAAAA|52944|14|42|24|PM|first|afternoon|lunch| +52945|AAAAAAAACNOMAAAA|52945|14|42|25|PM|first|afternoon|lunch| +52946|AAAAAAAADNOMAAAA|52946|14|42|26|PM|first|afternoon|lunch| +52947|AAAAAAAAENOMAAAA|52947|14|42|27|PM|first|afternoon|lunch| +52948|AAAAAAAAFNOMAAAA|52948|14|42|28|PM|first|afternoon|lunch| +52949|AAAAAAAAGNOMAAAA|52949|14|42|29|PM|first|afternoon|lunch| +52950|AAAAAAAAHNOMAAAA|52950|14|42|30|PM|first|afternoon|lunch| +52951|AAAAAAAAINOMAAAA|52951|14|42|31|PM|first|afternoon|lunch| +52952|AAAAAAAAJNOMAAAA|52952|14|42|32|PM|first|afternoon|lunch| +52953|AAAAAAAAKNOMAAAA|52953|14|42|33|PM|first|afternoon|lunch| +52954|AAAAAAAALNOMAAAA|52954|14|42|34|PM|first|afternoon|lunch| +52955|AAAAAAAAMNOMAAAA|52955|14|42|35|PM|first|afternoon|lunch| +52956|AAAAAAAANNOMAAAA|52956|14|42|36|PM|first|afternoon|lunch| +52957|AAAAAAAAONOMAAAA|52957|14|42|37|PM|first|afternoon|lunch| +52958|AAAAAAAAPNOMAAAA|52958|14|42|38|PM|first|afternoon|lunch| +52959|AAAAAAAAAOOMAAAA|52959|14|42|39|PM|first|afternoon|lunch| +52960|AAAAAAAABOOMAAAA|52960|14|42|40|PM|first|afternoon|lunch| +52961|AAAAAAAACOOMAAAA|52961|14|42|41|PM|first|afternoon|lunch| +52962|AAAAAAAADOOMAAAA|52962|14|42|42|PM|first|afternoon|lunch| +52963|AAAAAAAAEOOMAAAA|52963|14|42|43|PM|first|afternoon|lunch| +52964|AAAAAAAAFOOMAAAA|52964|14|42|44|PM|first|afternoon|lunch| +52965|AAAAAAAAGOOMAAAA|52965|14|42|45|PM|first|afternoon|lunch| +52966|AAAAAAAAHOOMAAAA|52966|14|42|46|PM|first|afternoon|lunch| +52967|AAAAAAAAIOOMAAAA|52967|14|42|47|PM|first|afternoon|lunch| +52968|AAAAAAAAJOOMAAAA|52968|14|42|48|PM|first|afternoon|lunch| +52969|AAAAAAAAKOOMAAAA|52969|14|42|49|PM|first|afternoon|lunch| +52970|AAAAAAAALOOMAAAA|52970|14|42|50|PM|first|afternoon|lunch| +52971|AAAAAAAAMOOMAAAA|52971|14|42|51|PM|first|afternoon|lunch| +52972|AAAAAAAANOOMAAAA|52972|14|42|52|PM|first|afternoon|lunch| +52973|AAAAAAAAOOOMAAAA|52973|14|42|53|PM|first|afternoon|lunch| +52974|AAAAAAAAPOOMAAAA|52974|14|42|54|PM|first|afternoon|lunch| +52975|AAAAAAAAAPOMAAAA|52975|14|42|55|PM|first|afternoon|lunch| +52976|AAAAAAAABPOMAAAA|52976|14|42|56|PM|first|afternoon|lunch| +52977|AAAAAAAACPOMAAAA|52977|14|42|57|PM|first|afternoon|lunch| +52978|AAAAAAAADPOMAAAA|52978|14|42|58|PM|first|afternoon|lunch| +52979|AAAAAAAAEPOMAAAA|52979|14|42|59|PM|first|afternoon|lunch| +52980|AAAAAAAAFPOMAAAA|52980|14|43|0|PM|first|afternoon|lunch| +52981|AAAAAAAAGPOMAAAA|52981|14|43|1|PM|first|afternoon|lunch| +52982|AAAAAAAAHPOMAAAA|52982|14|43|2|PM|first|afternoon|lunch| +52983|AAAAAAAAIPOMAAAA|52983|14|43|3|PM|first|afternoon|lunch| +52984|AAAAAAAAJPOMAAAA|52984|14|43|4|PM|first|afternoon|lunch| +52985|AAAAAAAAKPOMAAAA|52985|14|43|5|PM|first|afternoon|lunch| +52986|AAAAAAAALPOMAAAA|52986|14|43|6|PM|first|afternoon|lunch| +52987|AAAAAAAAMPOMAAAA|52987|14|43|7|PM|first|afternoon|lunch| +52988|AAAAAAAANPOMAAAA|52988|14|43|8|PM|first|afternoon|lunch| +52989|AAAAAAAAOPOMAAAA|52989|14|43|9|PM|first|afternoon|lunch| +52990|AAAAAAAAPPOMAAAA|52990|14|43|10|PM|first|afternoon|lunch| +52991|AAAAAAAAAAPMAAAA|52991|14|43|11|PM|first|afternoon|lunch| +52992|AAAAAAAABAPMAAAA|52992|14|43|12|PM|first|afternoon|lunch| +52993|AAAAAAAACAPMAAAA|52993|14|43|13|PM|first|afternoon|lunch| +52994|AAAAAAAADAPMAAAA|52994|14|43|14|PM|first|afternoon|lunch| +52995|AAAAAAAAEAPMAAAA|52995|14|43|15|PM|first|afternoon|lunch| +52996|AAAAAAAAFAPMAAAA|52996|14|43|16|PM|first|afternoon|lunch| +52997|AAAAAAAAGAPMAAAA|52997|14|43|17|PM|first|afternoon|lunch| +52998|AAAAAAAAHAPMAAAA|52998|14|43|18|PM|first|afternoon|lunch| +52999|AAAAAAAAIAPMAAAA|52999|14|43|19|PM|first|afternoon|lunch| +53000|AAAAAAAAJAPMAAAA|53000|14|43|20|PM|first|afternoon|lunch| +53001|AAAAAAAAKAPMAAAA|53001|14|43|21|PM|first|afternoon|lunch| +53002|AAAAAAAALAPMAAAA|53002|14|43|22|PM|first|afternoon|lunch| +53003|AAAAAAAAMAPMAAAA|53003|14|43|23|PM|first|afternoon|lunch| +53004|AAAAAAAANAPMAAAA|53004|14|43|24|PM|first|afternoon|lunch| +53005|AAAAAAAAOAPMAAAA|53005|14|43|25|PM|first|afternoon|lunch| +53006|AAAAAAAAPAPMAAAA|53006|14|43|26|PM|first|afternoon|lunch| +53007|AAAAAAAAABPMAAAA|53007|14|43|27|PM|first|afternoon|lunch| +53008|AAAAAAAABBPMAAAA|53008|14|43|28|PM|first|afternoon|lunch| +53009|AAAAAAAACBPMAAAA|53009|14|43|29|PM|first|afternoon|lunch| +53010|AAAAAAAADBPMAAAA|53010|14|43|30|PM|first|afternoon|lunch| +53011|AAAAAAAAEBPMAAAA|53011|14|43|31|PM|first|afternoon|lunch| +53012|AAAAAAAAFBPMAAAA|53012|14|43|32|PM|first|afternoon|lunch| +53013|AAAAAAAAGBPMAAAA|53013|14|43|33|PM|first|afternoon|lunch| +53014|AAAAAAAAHBPMAAAA|53014|14|43|34|PM|first|afternoon|lunch| +53015|AAAAAAAAIBPMAAAA|53015|14|43|35|PM|first|afternoon|lunch| +53016|AAAAAAAAJBPMAAAA|53016|14|43|36|PM|first|afternoon|lunch| +53017|AAAAAAAAKBPMAAAA|53017|14|43|37|PM|first|afternoon|lunch| +53018|AAAAAAAALBPMAAAA|53018|14|43|38|PM|first|afternoon|lunch| +53019|AAAAAAAAMBPMAAAA|53019|14|43|39|PM|first|afternoon|lunch| +53020|AAAAAAAANBPMAAAA|53020|14|43|40|PM|first|afternoon|lunch| +53021|AAAAAAAAOBPMAAAA|53021|14|43|41|PM|first|afternoon|lunch| +53022|AAAAAAAAPBPMAAAA|53022|14|43|42|PM|first|afternoon|lunch| +53023|AAAAAAAAACPMAAAA|53023|14|43|43|PM|first|afternoon|lunch| +53024|AAAAAAAABCPMAAAA|53024|14|43|44|PM|first|afternoon|lunch| +53025|AAAAAAAACCPMAAAA|53025|14|43|45|PM|first|afternoon|lunch| +53026|AAAAAAAADCPMAAAA|53026|14|43|46|PM|first|afternoon|lunch| +53027|AAAAAAAAECPMAAAA|53027|14|43|47|PM|first|afternoon|lunch| +53028|AAAAAAAAFCPMAAAA|53028|14|43|48|PM|first|afternoon|lunch| +53029|AAAAAAAAGCPMAAAA|53029|14|43|49|PM|first|afternoon|lunch| +53030|AAAAAAAAHCPMAAAA|53030|14|43|50|PM|first|afternoon|lunch| +53031|AAAAAAAAICPMAAAA|53031|14|43|51|PM|first|afternoon|lunch| +53032|AAAAAAAAJCPMAAAA|53032|14|43|52|PM|first|afternoon|lunch| +53033|AAAAAAAAKCPMAAAA|53033|14|43|53|PM|first|afternoon|lunch| +53034|AAAAAAAALCPMAAAA|53034|14|43|54|PM|first|afternoon|lunch| +53035|AAAAAAAAMCPMAAAA|53035|14|43|55|PM|first|afternoon|lunch| +53036|AAAAAAAANCPMAAAA|53036|14|43|56|PM|first|afternoon|lunch| +53037|AAAAAAAAOCPMAAAA|53037|14|43|57|PM|first|afternoon|lunch| +53038|AAAAAAAAPCPMAAAA|53038|14|43|58|PM|first|afternoon|lunch| +53039|AAAAAAAAADPMAAAA|53039|14|43|59|PM|first|afternoon|lunch| +53040|AAAAAAAABDPMAAAA|53040|14|44|0|PM|first|afternoon|lunch| +53041|AAAAAAAACDPMAAAA|53041|14|44|1|PM|first|afternoon|lunch| +53042|AAAAAAAADDPMAAAA|53042|14|44|2|PM|first|afternoon|lunch| +53043|AAAAAAAAEDPMAAAA|53043|14|44|3|PM|first|afternoon|lunch| +53044|AAAAAAAAFDPMAAAA|53044|14|44|4|PM|first|afternoon|lunch| +53045|AAAAAAAAGDPMAAAA|53045|14|44|5|PM|first|afternoon|lunch| +53046|AAAAAAAAHDPMAAAA|53046|14|44|6|PM|first|afternoon|lunch| +53047|AAAAAAAAIDPMAAAA|53047|14|44|7|PM|first|afternoon|lunch| +53048|AAAAAAAAJDPMAAAA|53048|14|44|8|PM|first|afternoon|lunch| +53049|AAAAAAAAKDPMAAAA|53049|14|44|9|PM|first|afternoon|lunch| +53050|AAAAAAAALDPMAAAA|53050|14|44|10|PM|first|afternoon|lunch| +53051|AAAAAAAAMDPMAAAA|53051|14|44|11|PM|first|afternoon|lunch| +53052|AAAAAAAANDPMAAAA|53052|14|44|12|PM|first|afternoon|lunch| +53053|AAAAAAAAODPMAAAA|53053|14|44|13|PM|first|afternoon|lunch| +53054|AAAAAAAAPDPMAAAA|53054|14|44|14|PM|first|afternoon|lunch| +53055|AAAAAAAAAEPMAAAA|53055|14|44|15|PM|first|afternoon|lunch| +53056|AAAAAAAABEPMAAAA|53056|14|44|16|PM|first|afternoon|lunch| +53057|AAAAAAAACEPMAAAA|53057|14|44|17|PM|first|afternoon|lunch| +53058|AAAAAAAADEPMAAAA|53058|14|44|18|PM|first|afternoon|lunch| +53059|AAAAAAAAEEPMAAAA|53059|14|44|19|PM|first|afternoon|lunch| +53060|AAAAAAAAFEPMAAAA|53060|14|44|20|PM|first|afternoon|lunch| +53061|AAAAAAAAGEPMAAAA|53061|14|44|21|PM|first|afternoon|lunch| +53062|AAAAAAAAHEPMAAAA|53062|14|44|22|PM|first|afternoon|lunch| +53063|AAAAAAAAIEPMAAAA|53063|14|44|23|PM|first|afternoon|lunch| +53064|AAAAAAAAJEPMAAAA|53064|14|44|24|PM|first|afternoon|lunch| +53065|AAAAAAAAKEPMAAAA|53065|14|44|25|PM|first|afternoon|lunch| +53066|AAAAAAAALEPMAAAA|53066|14|44|26|PM|first|afternoon|lunch| +53067|AAAAAAAAMEPMAAAA|53067|14|44|27|PM|first|afternoon|lunch| +53068|AAAAAAAANEPMAAAA|53068|14|44|28|PM|first|afternoon|lunch| +53069|AAAAAAAAOEPMAAAA|53069|14|44|29|PM|first|afternoon|lunch| +53070|AAAAAAAAPEPMAAAA|53070|14|44|30|PM|first|afternoon|lunch| +53071|AAAAAAAAAFPMAAAA|53071|14|44|31|PM|first|afternoon|lunch| +53072|AAAAAAAABFPMAAAA|53072|14|44|32|PM|first|afternoon|lunch| +53073|AAAAAAAACFPMAAAA|53073|14|44|33|PM|first|afternoon|lunch| +53074|AAAAAAAADFPMAAAA|53074|14|44|34|PM|first|afternoon|lunch| +53075|AAAAAAAAEFPMAAAA|53075|14|44|35|PM|first|afternoon|lunch| +53076|AAAAAAAAFFPMAAAA|53076|14|44|36|PM|first|afternoon|lunch| +53077|AAAAAAAAGFPMAAAA|53077|14|44|37|PM|first|afternoon|lunch| +53078|AAAAAAAAHFPMAAAA|53078|14|44|38|PM|first|afternoon|lunch| +53079|AAAAAAAAIFPMAAAA|53079|14|44|39|PM|first|afternoon|lunch| +53080|AAAAAAAAJFPMAAAA|53080|14|44|40|PM|first|afternoon|lunch| +53081|AAAAAAAAKFPMAAAA|53081|14|44|41|PM|first|afternoon|lunch| +53082|AAAAAAAALFPMAAAA|53082|14|44|42|PM|first|afternoon|lunch| +53083|AAAAAAAAMFPMAAAA|53083|14|44|43|PM|first|afternoon|lunch| +53084|AAAAAAAANFPMAAAA|53084|14|44|44|PM|first|afternoon|lunch| +53085|AAAAAAAAOFPMAAAA|53085|14|44|45|PM|first|afternoon|lunch| +53086|AAAAAAAAPFPMAAAA|53086|14|44|46|PM|first|afternoon|lunch| +53087|AAAAAAAAAGPMAAAA|53087|14|44|47|PM|first|afternoon|lunch| +53088|AAAAAAAABGPMAAAA|53088|14|44|48|PM|first|afternoon|lunch| +53089|AAAAAAAACGPMAAAA|53089|14|44|49|PM|first|afternoon|lunch| +53090|AAAAAAAADGPMAAAA|53090|14|44|50|PM|first|afternoon|lunch| +53091|AAAAAAAAEGPMAAAA|53091|14|44|51|PM|first|afternoon|lunch| +53092|AAAAAAAAFGPMAAAA|53092|14|44|52|PM|first|afternoon|lunch| +53093|AAAAAAAAGGPMAAAA|53093|14|44|53|PM|first|afternoon|lunch| +53094|AAAAAAAAHGPMAAAA|53094|14|44|54|PM|first|afternoon|lunch| +53095|AAAAAAAAIGPMAAAA|53095|14|44|55|PM|first|afternoon|lunch| +53096|AAAAAAAAJGPMAAAA|53096|14|44|56|PM|first|afternoon|lunch| +53097|AAAAAAAAKGPMAAAA|53097|14|44|57|PM|first|afternoon|lunch| +53098|AAAAAAAALGPMAAAA|53098|14|44|58|PM|first|afternoon|lunch| +53099|AAAAAAAAMGPMAAAA|53099|14|44|59|PM|first|afternoon|lunch| +53100|AAAAAAAANGPMAAAA|53100|14|45|0|PM|first|afternoon|lunch| +53101|AAAAAAAAOGPMAAAA|53101|14|45|1|PM|first|afternoon|lunch| +53102|AAAAAAAAPGPMAAAA|53102|14|45|2|PM|first|afternoon|lunch| +53103|AAAAAAAAAHPMAAAA|53103|14|45|3|PM|first|afternoon|lunch| +53104|AAAAAAAABHPMAAAA|53104|14|45|4|PM|first|afternoon|lunch| +53105|AAAAAAAACHPMAAAA|53105|14|45|5|PM|first|afternoon|lunch| +53106|AAAAAAAADHPMAAAA|53106|14|45|6|PM|first|afternoon|lunch| +53107|AAAAAAAAEHPMAAAA|53107|14|45|7|PM|first|afternoon|lunch| +53108|AAAAAAAAFHPMAAAA|53108|14|45|8|PM|first|afternoon|lunch| +53109|AAAAAAAAGHPMAAAA|53109|14|45|9|PM|first|afternoon|lunch| +53110|AAAAAAAAHHPMAAAA|53110|14|45|10|PM|first|afternoon|lunch| +53111|AAAAAAAAIHPMAAAA|53111|14|45|11|PM|first|afternoon|lunch| +53112|AAAAAAAAJHPMAAAA|53112|14|45|12|PM|first|afternoon|lunch| +53113|AAAAAAAAKHPMAAAA|53113|14|45|13|PM|first|afternoon|lunch| +53114|AAAAAAAALHPMAAAA|53114|14|45|14|PM|first|afternoon|lunch| +53115|AAAAAAAAMHPMAAAA|53115|14|45|15|PM|first|afternoon|lunch| +53116|AAAAAAAANHPMAAAA|53116|14|45|16|PM|first|afternoon|lunch| +53117|AAAAAAAAOHPMAAAA|53117|14|45|17|PM|first|afternoon|lunch| +53118|AAAAAAAAPHPMAAAA|53118|14|45|18|PM|first|afternoon|lunch| +53119|AAAAAAAAAIPMAAAA|53119|14|45|19|PM|first|afternoon|lunch| +53120|AAAAAAAABIPMAAAA|53120|14|45|20|PM|first|afternoon|lunch| +53121|AAAAAAAACIPMAAAA|53121|14|45|21|PM|first|afternoon|lunch| +53122|AAAAAAAADIPMAAAA|53122|14|45|22|PM|first|afternoon|lunch| +53123|AAAAAAAAEIPMAAAA|53123|14|45|23|PM|first|afternoon|lunch| +53124|AAAAAAAAFIPMAAAA|53124|14|45|24|PM|first|afternoon|lunch| +53125|AAAAAAAAGIPMAAAA|53125|14|45|25|PM|first|afternoon|lunch| +53126|AAAAAAAAHIPMAAAA|53126|14|45|26|PM|first|afternoon|lunch| +53127|AAAAAAAAIIPMAAAA|53127|14|45|27|PM|first|afternoon|lunch| +53128|AAAAAAAAJIPMAAAA|53128|14|45|28|PM|first|afternoon|lunch| +53129|AAAAAAAAKIPMAAAA|53129|14|45|29|PM|first|afternoon|lunch| +53130|AAAAAAAALIPMAAAA|53130|14|45|30|PM|first|afternoon|lunch| +53131|AAAAAAAAMIPMAAAA|53131|14|45|31|PM|first|afternoon|lunch| +53132|AAAAAAAANIPMAAAA|53132|14|45|32|PM|first|afternoon|lunch| +53133|AAAAAAAAOIPMAAAA|53133|14|45|33|PM|first|afternoon|lunch| +53134|AAAAAAAAPIPMAAAA|53134|14|45|34|PM|first|afternoon|lunch| +53135|AAAAAAAAAJPMAAAA|53135|14|45|35|PM|first|afternoon|lunch| +53136|AAAAAAAABJPMAAAA|53136|14|45|36|PM|first|afternoon|lunch| +53137|AAAAAAAACJPMAAAA|53137|14|45|37|PM|first|afternoon|lunch| +53138|AAAAAAAADJPMAAAA|53138|14|45|38|PM|first|afternoon|lunch| +53139|AAAAAAAAEJPMAAAA|53139|14|45|39|PM|first|afternoon|lunch| +53140|AAAAAAAAFJPMAAAA|53140|14|45|40|PM|first|afternoon|lunch| +53141|AAAAAAAAGJPMAAAA|53141|14|45|41|PM|first|afternoon|lunch| +53142|AAAAAAAAHJPMAAAA|53142|14|45|42|PM|first|afternoon|lunch| +53143|AAAAAAAAIJPMAAAA|53143|14|45|43|PM|first|afternoon|lunch| +53144|AAAAAAAAJJPMAAAA|53144|14|45|44|PM|first|afternoon|lunch| +53145|AAAAAAAAKJPMAAAA|53145|14|45|45|PM|first|afternoon|lunch| +53146|AAAAAAAALJPMAAAA|53146|14|45|46|PM|first|afternoon|lunch| +53147|AAAAAAAAMJPMAAAA|53147|14|45|47|PM|first|afternoon|lunch| +53148|AAAAAAAANJPMAAAA|53148|14|45|48|PM|first|afternoon|lunch| +53149|AAAAAAAAOJPMAAAA|53149|14|45|49|PM|first|afternoon|lunch| +53150|AAAAAAAAPJPMAAAA|53150|14|45|50|PM|first|afternoon|lunch| +53151|AAAAAAAAAKPMAAAA|53151|14|45|51|PM|first|afternoon|lunch| +53152|AAAAAAAABKPMAAAA|53152|14|45|52|PM|first|afternoon|lunch| +53153|AAAAAAAACKPMAAAA|53153|14|45|53|PM|first|afternoon|lunch| +53154|AAAAAAAADKPMAAAA|53154|14|45|54|PM|first|afternoon|lunch| +53155|AAAAAAAAEKPMAAAA|53155|14|45|55|PM|first|afternoon|lunch| +53156|AAAAAAAAFKPMAAAA|53156|14|45|56|PM|first|afternoon|lunch| +53157|AAAAAAAAGKPMAAAA|53157|14|45|57|PM|first|afternoon|lunch| +53158|AAAAAAAAHKPMAAAA|53158|14|45|58|PM|first|afternoon|lunch| +53159|AAAAAAAAIKPMAAAA|53159|14|45|59|PM|first|afternoon|lunch| +53160|AAAAAAAAJKPMAAAA|53160|14|46|0|PM|first|afternoon|lunch| +53161|AAAAAAAAKKPMAAAA|53161|14|46|1|PM|first|afternoon|lunch| +53162|AAAAAAAALKPMAAAA|53162|14|46|2|PM|first|afternoon|lunch| +53163|AAAAAAAAMKPMAAAA|53163|14|46|3|PM|first|afternoon|lunch| +53164|AAAAAAAANKPMAAAA|53164|14|46|4|PM|first|afternoon|lunch| +53165|AAAAAAAAOKPMAAAA|53165|14|46|5|PM|first|afternoon|lunch| +53166|AAAAAAAAPKPMAAAA|53166|14|46|6|PM|first|afternoon|lunch| +53167|AAAAAAAAALPMAAAA|53167|14|46|7|PM|first|afternoon|lunch| +53168|AAAAAAAABLPMAAAA|53168|14|46|8|PM|first|afternoon|lunch| +53169|AAAAAAAACLPMAAAA|53169|14|46|9|PM|first|afternoon|lunch| +53170|AAAAAAAADLPMAAAA|53170|14|46|10|PM|first|afternoon|lunch| +53171|AAAAAAAAELPMAAAA|53171|14|46|11|PM|first|afternoon|lunch| +53172|AAAAAAAAFLPMAAAA|53172|14|46|12|PM|first|afternoon|lunch| +53173|AAAAAAAAGLPMAAAA|53173|14|46|13|PM|first|afternoon|lunch| +53174|AAAAAAAAHLPMAAAA|53174|14|46|14|PM|first|afternoon|lunch| +53175|AAAAAAAAILPMAAAA|53175|14|46|15|PM|first|afternoon|lunch| +53176|AAAAAAAAJLPMAAAA|53176|14|46|16|PM|first|afternoon|lunch| +53177|AAAAAAAAKLPMAAAA|53177|14|46|17|PM|first|afternoon|lunch| +53178|AAAAAAAALLPMAAAA|53178|14|46|18|PM|first|afternoon|lunch| +53179|AAAAAAAAMLPMAAAA|53179|14|46|19|PM|first|afternoon|lunch| +53180|AAAAAAAANLPMAAAA|53180|14|46|20|PM|first|afternoon|lunch| +53181|AAAAAAAAOLPMAAAA|53181|14|46|21|PM|first|afternoon|lunch| +53182|AAAAAAAAPLPMAAAA|53182|14|46|22|PM|first|afternoon|lunch| +53183|AAAAAAAAAMPMAAAA|53183|14|46|23|PM|first|afternoon|lunch| +53184|AAAAAAAABMPMAAAA|53184|14|46|24|PM|first|afternoon|lunch| +53185|AAAAAAAACMPMAAAA|53185|14|46|25|PM|first|afternoon|lunch| +53186|AAAAAAAADMPMAAAA|53186|14|46|26|PM|first|afternoon|lunch| +53187|AAAAAAAAEMPMAAAA|53187|14|46|27|PM|first|afternoon|lunch| +53188|AAAAAAAAFMPMAAAA|53188|14|46|28|PM|first|afternoon|lunch| +53189|AAAAAAAAGMPMAAAA|53189|14|46|29|PM|first|afternoon|lunch| +53190|AAAAAAAAHMPMAAAA|53190|14|46|30|PM|first|afternoon|lunch| +53191|AAAAAAAAIMPMAAAA|53191|14|46|31|PM|first|afternoon|lunch| +53192|AAAAAAAAJMPMAAAA|53192|14|46|32|PM|first|afternoon|lunch| +53193|AAAAAAAAKMPMAAAA|53193|14|46|33|PM|first|afternoon|lunch| +53194|AAAAAAAALMPMAAAA|53194|14|46|34|PM|first|afternoon|lunch| +53195|AAAAAAAAMMPMAAAA|53195|14|46|35|PM|first|afternoon|lunch| +53196|AAAAAAAANMPMAAAA|53196|14|46|36|PM|first|afternoon|lunch| +53197|AAAAAAAAOMPMAAAA|53197|14|46|37|PM|first|afternoon|lunch| +53198|AAAAAAAAPMPMAAAA|53198|14|46|38|PM|first|afternoon|lunch| +53199|AAAAAAAAANPMAAAA|53199|14|46|39|PM|first|afternoon|lunch| +53200|AAAAAAAABNPMAAAA|53200|14|46|40|PM|first|afternoon|lunch| +53201|AAAAAAAACNPMAAAA|53201|14|46|41|PM|first|afternoon|lunch| +53202|AAAAAAAADNPMAAAA|53202|14|46|42|PM|first|afternoon|lunch| +53203|AAAAAAAAENPMAAAA|53203|14|46|43|PM|first|afternoon|lunch| +53204|AAAAAAAAFNPMAAAA|53204|14|46|44|PM|first|afternoon|lunch| +53205|AAAAAAAAGNPMAAAA|53205|14|46|45|PM|first|afternoon|lunch| +53206|AAAAAAAAHNPMAAAA|53206|14|46|46|PM|first|afternoon|lunch| +53207|AAAAAAAAINPMAAAA|53207|14|46|47|PM|first|afternoon|lunch| +53208|AAAAAAAAJNPMAAAA|53208|14|46|48|PM|first|afternoon|lunch| +53209|AAAAAAAAKNPMAAAA|53209|14|46|49|PM|first|afternoon|lunch| +53210|AAAAAAAALNPMAAAA|53210|14|46|50|PM|first|afternoon|lunch| +53211|AAAAAAAAMNPMAAAA|53211|14|46|51|PM|first|afternoon|lunch| +53212|AAAAAAAANNPMAAAA|53212|14|46|52|PM|first|afternoon|lunch| +53213|AAAAAAAAONPMAAAA|53213|14|46|53|PM|first|afternoon|lunch| +53214|AAAAAAAAPNPMAAAA|53214|14|46|54|PM|first|afternoon|lunch| +53215|AAAAAAAAAOPMAAAA|53215|14|46|55|PM|first|afternoon|lunch| +53216|AAAAAAAABOPMAAAA|53216|14|46|56|PM|first|afternoon|lunch| +53217|AAAAAAAACOPMAAAA|53217|14|46|57|PM|first|afternoon|lunch| +53218|AAAAAAAADOPMAAAA|53218|14|46|58|PM|first|afternoon|lunch| +53219|AAAAAAAAEOPMAAAA|53219|14|46|59|PM|first|afternoon|lunch| +53220|AAAAAAAAFOPMAAAA|53220|14|47|0|PM|first|afternoon|lunch| +53221|AAAAAAAAGOPMAAAA|53221|14|47|1|PM|first|afternoon|lunch| +53222|AAAAAAAAHOPMAAAA|53222|14|47|2|PM|first|afternoon|lunch| +53223|AAAAAAAAIOPMAAAA|53223|14|47|3|PM|first|afternoon|lunch| +53224|AAAAAAAAJOPMAAAA|53224|14|47|4|PM|first|afternoon|lunch| +53225|AAAAAAAAKOPMAAAA|53225|14|47|5|PM|first|afternoon|lunch| +53226|AAAAAAAALOPMAAAA|53226|14|47|6|PM|first|afternoon|lunch| +53227|AAAAAAAAMOPMAAAA|53227|14|47|7|PM|first|afternoon|lunch| +53228|AAAAAAAANOPMAAAA|53228|14|47|8|PM|first|afternoon|lunch| +53229|AAAAAAAAOOPMAAAA|53229|14|47|9|PM|first|afternoon|lunch| +53230|AAAAAAAAPOPMAAAA|53230|14|47|10|PM|first|afternoon|lunch| +53231|AAAAAAAAAPPMAAAA|53231|14|47|11|PM|first|afternoon|lunch| +53232|AAAAAAAABPPMAAAA|53232|14|47|12|PM|first|afternoon|lunch| +53233|AAAAAAAACPPMAAAA|53233|14|47|13|PM|first|afternoon|lunch| +53234|AAAAAAAADPPMAAAA|53234|14|47|14|PM|first|afternoon|lunch| +53235|AAAAAAAAEPPMAAAA|53235|14|47|15|PM|first|afternoon|lunch| +53236|AAAAAAAAFPPMAAAA|53236|14|47|16|PM|first|afternoon|lunch| +53237|AAAAAAAAGPPMAAAA|53237|14|47|17|PM|first|afternoon|lunch| +53238|AAAAAAAAHPPMAAAA|53238|14|47|18|PM|first|afternoon|lunch| +53239|AAAAAAAAIPPMAAAA|53239|14|47|19|PM|first|afternoon|lunch| +53240|AAAAAAAAJPPMAAAA|53240|14|47|20|PM|first|afternoon|lunch| +53241|AAAAAAAAKPPMAAAA|53241|14|47|21|PM|first|afternoon|lunch| +53242|AAAAAAAALPPMAAAA|53242|14|47|22|PM|first|afternoon|lunch| +53243|AAAAAAAAMPPMAAAA|53243|14|47|23|PM|first|afternoon|lunch| +53244|AAAAAAAANPPMAAAA|53244|14|47|24|PM|first|afternoon|lunch| +53245|AAAAAAAAOPPMAAAA|53245|14|47|25|PM|first|afternoon|lunch| +53246|AAAAAAAAPPPMAAAA|53246|14|47|26|PM|first|afternoon|lunch| +53247|AAAAAAAAAAANAAAA|53247|14|47|27|PM|first|afternoon|lunch| +53248|AAAAAAAABAANAAAA|53248|14|47|28|PM|first|afternoon|lunch| +53249|AAAAAAAACAANAAAA|53249|14|47|29|PM|first|afternoon|lunch| +53250|AAAAAAAADAANAAAA|53250|14|47|30|PM|first|afternoon|lunch| +53251|AAAAAAAAEAANAAAA|53251|14|47|31|PM|first|afternoon|lunch| +53252|AAAAAAAAFAANAAAA|53252|14|47|32|PM|first|afternoon|lunch| +53253|AAAAAAAAGAANAAAA|53253|14|47|33|PM|first|afternoon|lunch| +53254|AAAAAAAAHAANAAAA|53254|14|47|34|PM|first|afternoon|lunch| +53255|AAAAAAAAIAANAAAA|53255|14|47|35|PM|first|afternoon|lunch| +53256|AAAAAAAAJAANAAAA|53256|14|47|36|PM|first|afternoon|lunch| +53257|AAAAAAAAKAANAAAA|53257|14|47|37|PM|first|afternoon|lunch| +53258|AAAAAAAALAANAAAA|53258|14|47|38|PM|first|afternoon|lunch| +53259|AAAAAAAAMAANAAAA|53259|14|47|39|PM|first|afternoon|lunch| +53260|AAAAAAAANAANAAAA|53260|14|47|40|PM|first|afternoon|lunch| +53261|AAAAAAAAOAANAAAA|53261|14|47|41|PM|first|afternoon|lunch| +53262|AAAAAAAAPAANAAAA|53262|14|47|42|PM|first|afternoon|lunch| +53263|AAAAAAAAABANAAAA|53263|14|47|43|PM|first|afternoon|lunch| +53264|AAAAAAAABBANAAAA|53264|14|47|44|PM|first|afternoon|lunch| +53265|AAAAAAAACBANAAAA|53265|14|47|45|PM|first|afternoon|lunch| +53266|AAAAAAAADBANAAAA|53266|14|47|46|PM|first|afternoon|lunch| +53267|AAAAAAAAEBANAAAA|53267|14|47|47|PM|first|afternoon|lunch| +53268|AAAAAAAAFBANAAAA|53268|14|47|48|PM|first|afternoon|lunch| +53269|AAAAAAAAGBANAAAA|53269|14|47|49|PM|first|afternoon|lunch| +53270|AAAAAAAAHBANAAAA|53270|14|47|50|PM|first|afternoon|lunch| +53271|AAAAAAAAIBANAAAA|53271|14|47|51|PM|first|afternoon|lunch| +53272|AAAAAAAAJBANAAAA|53272|14|47|52|PM|first|afternoon|lunch| +53273|AAAAAAAAKBANAAAA|53273|14|47|53|PM|first|afternoon|lunch| +53274|AAAAAAAALBANAAAA|53274|14|47|54|PM|first|afternoon|lunch| +53275|AAAAAAAAMBANAAAA|53275|14|47|55|PM|first|afternoon|lunch| +53276|AAAAAAAANBANAAAA|53276|14|47|56|PM|first|afternoon|lunch| +53277|AAAAAAAAOBANAAAA|53277|14|47|57|PM|first|afternoon|lunch| +53278|AAAAAAAAPBANAAAA|53278|14|47|58|PM|first|afternoon|lunch| +53279|AAAAAAAAACANAAAA|53279|14|47|59|PM|first|afternoon|lunch| +53280|AAAAAAAABCANAAAA|53280|14|48|0|PM|first|afternoon|lunch| +53281|AAAAAAAACCANAAAA|53281|14|48|1|PM|first|afternoon|lunch| +53282|AAAAAAAADCANAAAA|53282|14|48|2|PM|first|afternoon|lunch| +53283|AAAAAAAAECANAAAA|53283|14|48|3|PM|first|afternoon|lunch| +53284|AAAAAAAAFCANAAAA|53284|14|48|4|PM|first|afternoon|lunch| +53285|AAAAAAAAGCANAAAA|53285|14|48|5|PM|first|afternoon|lunch| +53286|AAAAAAAAHCANAAAA|53286|14|48|6|PM|first|afternoon|lunch| +53287|AAAAAAAAICANAAAA|53287|14|48|7|PM|first|afternoon|lunch| +53288|AAAAAAAAJCANAAAA|53288|14|48|8|PM|first|afternoon|lunch| +53289|AAAAAAAAKCANAAAA|53289|14|48|9|PM|first|afternoon|lunch| +53290|AAAAAAAALCANAAAA|53290|14|48|10|PM|first|afternoon|lunch| +53291|AAAAAAAAMCANAAAA|53291|14|48|11|PM|first|afternoon|lunch| +53292|AAAAAAAANCANAAAA|53292|14|48|12|PM|first|afternoon|lunch| +53293|AAAAAAAAOCANAAAA|53293|14|48|13|PM|first|afternoon|lunch| +53294|AAAAAAAAPCANAAAA|53294|14|48|14|PM|first|afternoon|lunch| +53295|AAAAAAAAADANAAAA|53295|14|48|15|PM|first|afternoon|lunch| +53296|AAAAAAAABDANAAAA|53296|14|48|16|PM|first|afternoon|lunch| +53297|AAAAAAAACDANAAAA|53297|14|48|17|PM|first|afternoon|lunch| +53298|AAAAAAAADDANAAAA|53298|14|48|18|PM|first|afternoon|lunch| +53299|AAAAAAAAEDANAAAA|53299|14|48|19|PM|first|afternoon|lunch| +53300|AAAAAAAAFDANAAAA|53300|14|48|20|PM|first|afternoon|lunch| +53301|AAAAAAAAGDANAAAA|53301|14|48|21|PM|first|afternoon|lunch| +53302|AAAAAAAAHDANAAAA|53302|14|48|22|PM|first|afternoon|lunch| +53303|AAAAAAAAIDANAAAA|53303|14|48|23|PM|first|afternoon|lunch| +53304|AAAAAAAAJDANAAAA|53304|14|48|24|PM|first|afternoon|lunch| +53305|AAAAAAAAKDANAAAA|53305|14|48|25|PM|first|afternoon|lunch| +53306|AAAAAAAALDANAAAA|53306|14|48|26|PM|first|afternoon|lunch| +53307|AAAAAAAAMDANAAAA|53307|14|48|27|PM|first|afternoon|lunch| +53308|AAAAAAAANDANAAAA|53308|14|48|28|PM|first|afternoon|lunch| +53309|AAAAAAAAODANAAAA|53309|14|48|29|PM|first|afternoon|lunch| +53310|AAAAAAAAPDANAAAA|53310|14|48|30|PM|first|afternoon|lunch| +53311|AAAAAAAAAEANAAAA|53311|14|48|31|PM|first|afternoon|lunch| +53312|AAAAAAAABEANAAAA|53312|14|48|32|PM|first|afternoon|lunch| +53313|AAAAAAAACEANAAAA|53313|14|48|33|PM|first|afternoon|lunch| +53314|AAAAAAAADEANAAAA|53314|14|48|34|PM|first|afternoon|lunch| +53315|AAAAAAAAEEANAAAA|53315|14|48|35|PM|first|afternoon|lunch| +53316|AAAAAAAAFEANAAAA|53316|14|48|36|PM|first|afternoon|lunch| +53317|AAAAAAAAGEANAAAA|53317|14|48|37|PM|first|afternoon|lunch| +53318|AAAAAAAAHEANAAAA|53318|14|48|38|PM|first|afternoon|lunch| +53319|AAAAAAAAIEANAAAA|53319|14|48|39|PM|first|afternoon|lunch| +53320|AAAAAAAAJEANAAAA|53320|14|48|40|PM|first|afternoon|lunch| +53321|AAAAAAAAKEANAAAA|53321|14|48|41|PM|first|afternoon|lunch| +53322|AAAAAAAALEANAAAA|53322|14|48|42|PM|first|afternoon|lunch| +53323|AAAAAAAAMEANAAAA|53323|14|48|43|PM|first|afternoon|lunch| +53324|AAAAAAAANEANAAAA|53324|14|48|44|PM|first|afternoon|lunch| +53325|AAAAAAAAOEANAAAA|53325|14|48|45|PM|first|afternoon|lunch| +53326|AAAAAAAAPEANAAAA|53326|14|48|46|PM|first|afternoon|lunch| +53327|AAAAAAAAAFANAAAA|53327|14|48|47|PM|first|afternoon|lunch| +53328|AAAAAAAABFANAAAA|53328|14|48|48|PM|first|afternoon|lunch| +53329|AAAAAAAACFANAAAA|53329|14|48|49|PM|first|afternoon|lunch| +53330|AAAAAAAADFANAAAA|53330|14|48|50|PM|first|afternoon|lunch| +53331|AAAAAAAAEFANAAAA|53331|14|48|51|PM|first|afternoon|lunch| +53332|AAAAAAAAFFANAAAA|53332|14|48|52|PM|first|afternoon|lunch| +53333|AAAAAAAAGFANAAAA|53333|14|48|53|PM|first|afternoon|lunch| +53334|AAAAAAAAHFANAAAA|53334|14|48|54|PM|first|afternoon|lunch| +53335|AAAAAAAAIFANAAAA|53335|14|48|55|PM|first|afternoon|lunch| +53336|AAAAAAAAJFANAAAA|53336|14|48|56|PM|first|afternoon|lunch| +53337|AAAAAAAAKFANAAAA|53337|14|48|57|PM|first|afternoon|lunch| +53338|AAAAAAAALFANAAAA|53338|14|48|58|PM|first|afternoon|lunch| +53339|AAAAAAAAMFANAAAA|53339|14|48|59|PM|first|afternoon|lunch| +53340|AAAAAAAANFANAAAA|53340|14|49|0|PM|first|afternoon|lunch| +53341|AAAAAAAAOFANAAAA|53341|14|49|1|PM|first|afternoon|lunch| +53342|AAAAAAAAPFANAAAA|53342|14|49|2|PM|first|afternoon|lunch| +53343|AAAAAAAAAGANAAAA|53343|14|49|3|PM|first|afternoon|lunch| +53344|AAAAAAAABGANAAAA|53344|14|49|4|PM|first|afternoon|lunch| +53345|AAAAAAAACGANAAAA|53345|14|49|5|PM|first|afternoon|lunch| +53346|AAAAAAAADGANAAAA|53346|14|49|6|PM|first|afternoon|lunch| +53347|AAAAAAAAEGANAAAA|53347|14|49|7|PM|first|afternoon|lunch| +53348|AAAAAAAAFGANAAAA|53348|14|49|8|PM|first|afternoon|lunch| +53349|AAAAAAAAGGANAAAA|53349|14|49|9|PM|first|afternoon|lunch| +53350|AAAAAAAAHGANAAAA|53350|14|49|10|PM|first|afternoon|lunch| +53351|AAAAAAAAIGANAAAA|53351|14|49|11|PM|first|afternoon|lunch| +53352|AAAAAAAAJGANAAAA|53352|14|49|12|PM|first|afternoon|lunch| +53353|AAAAAAAAKGANAAAA|53353|14|49|13|PM|first|afternoon|lunch| +53354|AAAAAAAALGANAAAA|53354|14|49|14|PM|first|afternoon|lunch| +53355|AAAAAAAAMGANAAAA|53355|14|49|15|PM|first|afternoon|lunch| +53356|AAAAAAAANGANAAAA|53356|14|49|16|PM|first|afternoon|lunch| +53357|AAAAAAAAOGANAAAA|53357|14|49|17|PM|first|afternoon|lunch| +53358|AAAAAAAAPGANAAAA|53358|14|49|18|PM|first|afternoon|lunch| +53359|AAAAAAAAAHANAAAA|53359|14|49|19|PM|first|afternoon|lunch| +53360|AAAAAAAABHANAAAA|53360|14|49|20|PM|first|afternoon|lunch| +53361|AAAAAAAACHANAAAA|53361|14|49|21|PM|first|afternoon|lunch| +53362|AAAAAAAADHANAAAA|53362|14|49|22|PM|first|afternoon|lunch| +53363|AAAAAAAAEHANAAAA|53363|14|49|23|PM|first|afternoon|lunch| +53364|AAAAAAAAFHANAAAA|53364|14|49|24|PM|first|afternoon|lunch| +53365|AAAAAAAAGHANAAAA|53365|14|49|25|PM|first|afternoon|lunch| +53366|AAAAAAAAHHANAAAA|53366|14|49|26|PM|first|afternoon|lunch| +53367|AAAAAAAAIHANAAAA|53367|14|49|27|PM|first|afternoon|lunch| +53368|AAAAAAAAJHANAAAA|53368|14|49|28|PM|first|afternoon|lunch| +53369|AAAAAAAAKHANAAAA|53369|14|49|29|PM|first|afternoon|lunch| +53370|AAAAAAAALHANAAAA|53370|14|49|30|PM|first|afternoon|lunch| +53371|AAAAAAAAMHANAAAA|53371|14|49|31|PM|first|afternoon|lunch| +53372|AAAAAAAANHANAAAA|53372|14|49|32|PM|first|afternoon|lunch| +53373|AAAAAAAAOHANAAAA|53373|14|49|33|PM|first|afternoon|lunch| +53374|AAAAAAAAPHANAAAA|53374|14|49|34|PM|first|afternoon|lunch| +53375|AAAAAAAAAIANAAAA|53375|14|49|35|PM|first|afternoon|lunch| +53376|AAAAAAAABIANAAAA|53376|14|49|36|PM|first|afternoon|lunch| +53377|AAAAAAAACIANAAAA|53377|14|49|37|PM|first|afternoon|lunch| +53378|AAAAAAAADIANAAAA|53378|14|49|38|PM|first|afternoon|lunch| +53379|AAAAAAAAEIANAAAA|53379|14|49|39|PM|first|afternoon|lunch| +53380|AAAAAAAAFIANAAAA|53380|14|49|40|PM|first|afternoon|lunch| +53381|AAAAAAAAGIANAAAA|53381|14|49|41|PM|first|afternoon|lunch| +53382|AAAAAAAAHIANAAAA|53382|14|49|42|PM|first|afternoon|lunch| +53383|AAAAAAAAIIANAAAA|53383|14|49|43|PM|first|afternoon|lunch| +53384|AAAAAAAAJIANAAAA|53384|14|49|44|PM|first|afternoon|lunch| +53385|AAAAAAAAKIANAAAA|53385|14|49|45|PM|first|afternoon|lunch| +53386|AAAAAAAALIANAAAA|53386|14|49|46|PM|first|afternoon|lunch| +53387|AAAAAAAAMIANAAAA|53387|14|49|47|PM|first|afternoon|lunch| +53388|AAAAAAAANIANAAAA|53388|14|49|48|PM|first|afternoon|lunch| +53389|AAAAAAAAOIANAAAA|53389|14|49|49|PM|first|afternoon|lunch| +53390|AAAAAAAAPIANAAAA|53390|14|49|50|PM|first|afternoon|lunch| +53391|AAAAAAAAAJANAAAA|53391|14|49|51|PM|first|afternoon|lunch| +53392|AAAAAAAABJANAAAA|53392|14|49|52|PM|first|afternoon|lunch| +53393|AAAAAAAACJANAAAA|53393|14|49|53|PM|first|afternoon|lunch| +53394|AAAAAAAADJANAAAA|53394|14|49|54|PM|first|afternoon|lunch| +53395|AAAAAAAAEJANAAAA|53395|14|49|55|PM|first|afternoon|lunch| +53396|AAAAAAAAFJANAAAA|53396|14|49|56|PM|first|afternoon|lunch| +53397|AAAAAAAAGJANAAAA|53397|14|49|57|PM|first|afternoon|lunch| +53398|AAAAAAAAHJANAAAA|53398|14|49|58|PM|first|afternoon|lunch| +53399|AAAAAAAAIJANAAAA|53399|14|49|59|PM|first|afternoon|lunch| +53400|AAAAAAAAJJANAAAA|53400|14|50|0|PM|first|afternoon|lunch| +53401|AAAAAAAAKJANAAAA|53401|14|50|1|PM|first|afternoon|lunch| +53402|AAAAAAAALJANAAAA|53402|14|50|2|PM|first|afternoon|lunch| +53403|AAAAAAAAMJANAAAA|53403|14|50|3|PM|first|afternoon|lunch| +53404|AAAAAAAANJANAAAA|53404|14|50|4|PM|first|afternoon|lunch| +53405|AAAAAAAAOJANAAAA|53405|14|50|5|PM|first|afternoon|lunch| +53406|AAAAAAAAPJANAAAA|53406|14|50|6|PM|first|afternoon|lunch| +53407|AAAAAAAAAKANAAAA|53407|14|50|7|PM|first|afternoon|lunch| +53408|AAAAAAAABKANAAAA|53408|14|50|8|PM|first|afternoon|lunch| +53409|AAAAAAAACKANAAAA|53409|14|50|9|PM|first|afternoon|lunch| +53410|AAAAAAAADKANAAAA|53410|14|50|10|PM|first|afternoon|lunch| +53411|AAAAAAAAEKANAAAA|53411|14|50|11|PM|first|afternoon|lunch| +53412|AAAAAAAAFKANAAAA|53412|14|50|12|PM|first|afternoon|lunch| +53413|AAAAAAAAGKANAAAA|53413|14|50|13|PM|first|afternoon|lunch| +53414|AAAAAAAAHKANAAAA|53414|14|50|14|PM|first|afternoon|lunch| +53415|AAAAAAAAIKANAAAA|53415|14|50|15|PM|first|afternoon|lunch| +53416|AAAAAAAAJKANAAAA|53416|14|50|16|PM|first|afternoon|lunch| +53417|AAAAAAAAKKANAAAA|53417|14|50|17|PM|first|afternoon|lunch| +53418|AAAAAAAALKANAAAA|53418|14|50|18|PM|first|afternoon|lunch| +53419|AAAAAAAAMKANAAAA|53419|14|50|19|PM|first|afternoon|lunch| +53420|AAAAAAAANKANAAAA|53420|14|50|20|PM|first|afternoon|lunch| +53421|AAAAAAAAOKANAAAA|53421|14|50|21|PM|first|afternoon|lunch| +53422|AAAAAAAAPKANAAAA|53422|14|50|22|PM|first|afternoon|lunch| +53423|AAAAAAAAALANAAAA|53423|14|50|23|PM|first|afternoon|lunch| +53424|AAAAAAAABLANAAAA|53424|14|50|24|PM|first|afternoon|lunch| +53425|AAAAAAAACLANAAAA|53425|14|50|25|PM|first|afternoon|lunch| +53426|AAAAAAAADLANAAAA|53426|14|50|26|PM|first|afternoon|lunch| +53427|AAAAAAAAELANAAAA|53427|14|50|27|PM|first|afternoon|lunch| +53428|AAAAAAAAFLANAAAA|53428|14|50|28|PM|first|afternoon|lunch| +53429|AAAAAAAAGLANAAAA|53429|14|50|29|PM|first|afternoon|lunch| +53430|AAAAAAAAHLANAAAA|53430|14|50|30|PM|first|afternoon|lunch| +53431|AAAAAAAAILANAAAA|53431|14|50|31|PM|first|afternoon|lunch| +53432|AAAAAAAAJLANAAAA|53432|14|50|32|PM|first|afternoon|lunch| +53433|AAAAAAAAKLANAAAA|53433|14|50|33|PM|first|afternoon|lunch| +53434|AAAAAAAALLANAAAA|53434|14|50|34|PM|first|afternoon|lunch| +53435|AAAAAAAAMLANAAAA|53435|14|50|35|PM|first|afternoon|lunch| +53436|AAAAAAAANLANAAAA|53436|14|50|36|PM|first|afternoon|lunch| +53437|AAAAAAAAOLANAAAA|53437|14|50|37|PM|first|afternoon|lunch| +53438|AAAAAAAAPLANAAAA|53438|14|50|38|PM|first|afternoon|lunch| +53439|AAAAAAAAAMANAAAA|53439|14|50|39|PM|first|afternoon|lunch| +53440|AAAAAAAABMANAAAA|53440|14|50|40|PM|first|afternoon|lunch| +53441|AAAAAAAACMANAAAA|53441|14|50|41|PM|first|afternoon|lunch| +53442|AAAAAAAADMANAAAA|53442|14|50|42|PM|first|afternoon|lunch| +53443|AAAAAAAAEMANAAAA|53443|14|50|43|PM|first|afternoon|lunch| +53444|AAAAAAAAFMANAAAA|53444|14|50|44|PM|first|afternoon|lunch| +53445|AAAAAAAAGMANAAAA|53445|14|50|45|PM|first|afternoon|lunch| +53446|AAAAAAAAHMANAAAA|53446|14|50|46|PM|first|afternoon|lunch| +53447|AAAAAAAAIMANAAAA|53447|14|50|47|PM|first|afternoon|lunch| +53448|AAAAAAAAJMANAAAA|53448|14|50|48|PM|first|afternoon|lunch| +53449|AAAAAAAAKMANAAAA|53449|14|50|49|PM|first|afternoon|lunch| +53450|AAAAAAAALMANAAAA|53450|14|50|50|PM|first|afternoon|lunch| +53451|AAAAAAAAMMANAAAA|53451|14|50|51|PM|first|afternoon|lunch| +53452|AAAAAAAANMANAAAA|53452|14|50|52|PM|first|afternoon|lunch| +53453|AAAAAAAAOMANAAAA|53453|14|50|53|PM|first|afternoon|lunch| +53454|AAAAAAAAPMANAAAA|53454|14|50|54|PM|first|afternoon|lunch| +53455|AAAAAAAAANANAAAA|53455|14|50|55|PM|first|afternoon|lunch| +53456|AAAAAAAABNANAAAA|53456|14|50|56|PM|first|afternoon|lunch| +53457|AAAAAAAACNANAAAA|53457|14|50|57|PM|first|afternoon|lunch| +53458|AAAAAAAADNANAAAA|53458|14|50|58|PM|first|afternoon|lunch| +53459|AAAAAAAAENANAAAA|53459|14|50|59|PM|first|afternoon|lunch| +53460|AAAAAAAAFNANAAAA|53460|14|51|0|PM|first|afternoon|lunch| +53461|AAAAAAAAGNANAAAA|53461|14|51|1|PM|first|afternoon|lunch| +53462|AAAAAAAAHNANAAAA|53462|14|51|2|PM|first|afternoon|lunch| +53463|AAAAAAAAINANAAAA|53463|14|51|3|PM|first|afternoon|lunch| +53464|AAAAAAAAJNANAAAA|53464|14|51|4|PM|first|afternoon|lunch| +53465|AAAAAAAAKNANAAAA|53465|14|51|5|PM|first|afternoon|lunch| +53466|AAAAAAAALNANAAAA|53466|14|51|6|PM|first|afternoon|lunch| +53467|AAAAAAAAMNANAAAA|53467|14|51|7|PM|first|afternoon|lunch| +53468|AAAAAAAANNANAAAA|53468|14|51|8|PM|first|afternoon|lunch| +53469|AAAAAAAAONANAAAA|53469|14|51|9|PM|first|afternoon|lunch| +53470|AAAAAAAAPNANAAAA|53470|14|51|10|PM|first|afternoon|lunch| +53471|AAAAAAAAAOANAAAA|53471|14|51|11|PM|first|afternoon|lunch| +53472|AAAAAAAABOANAAAA|53472|14|51|12|PM|first|afternoon|lunch| +53473|AAAAAAAACOANAAAA|53473|14|51|13|PM|first|afternoon|lunch| +53474|AAAAAAAADOANAAAA|53474|14|51|14|PM|first|afternoon|lunch| +53475|AAAAAAAAEOANAAAA|53475|14|51|15|PM|first|afternoon|lunch| +53476|AAAAAAAAFOANAAAA|53476|14|51|16|PM|first|afternoon|lunch| +53477|AAAAAAAAGOANAAAA|53477|14|51|17|PM|first|afternoon|lunch| +53478|AAAAAAAAHOANAAAA|53478|14|51|18|PM|first|afternoon|lunch| +53479|AAAAAAAAIOANAAAA|53479|14|51|19|PM|first|afternoon|lunch| +53480|AAAAAAAAJOANAAAA|53480|14|51|20|PM|first|afternoon|lunch| +53481|AAAAAAAAKOANAAAA|53481|14|51|21|PM|first|afternoon|lunch| +53482|AAAAAAAALOANAAAA|53482|14|51|22|PM|first|afternoon|lunch| +53483|AAAAAAAAMOANAAAA|53483|14|51|23|PM|first|afternoon|lunch| +53484|AAAAAAAANOANAAAA|53484|14|51|24|PM|first|afternoon|lunch| +53485|AAAAAAAAOOANAAAA|53485|14|51|25|PM|first|afternoon|lunch| +53486|AAAAAAAAPOANAAAA|53486|14|51|26|PM|first|afternoon|lunch| +53487|AAAAAAAAAPANAAAA|53487|14|51|27|PM|first|afternoon|lunch| +53488|AAAAAAAABPANAAAA|53488|14|51|28|PM|first|afternoon|lunch| +53489|AAAAAAAACPANAAAA|53489|14|51|29|PM|first|afternoon|lunch| +53490|AAAAAAAADPANAAAA|53490|14|51|30|PM|first|afternoon|lunch| +53491|AAAAAAAAEPANAAAA|53491|14|51|31|PM|first|afternoon|lunch| +53492|AAAAAAAAFPANAAAA|53492|14|51|32|PM|first|afternoon|lunch| +53493|AAAAAAAAGPANAAAA|53493|14|51|33|PM|first|afternoon|lunch| +53494|AAAAAAAAHPANAAAA|53494|14|51|34|PM|first|afternoon|lunch| +53495|AAAAAAAAIPANAAAA|53495|14|51|35|PM|first|afternoon|lunch| +53496|AAAAAAAAJPANAAAA|53496|14|51|36|PM|first|afternoon|lunch| +53497|AAAAAAAAKPANAAAA|53497|14|51|37|PM|first|afternoon|lunch| +53498|AAAAAAAALPANAAAA|53498|14|51|38|PM|first|afternoon|lunch| +53499|AAAAAAAAMPANAAAA|53499|14|51|39|PM|first|afternoon|lunch| +53500|AAAAAAAANPANAAAA|53500|14|51|40|PM|first|afternoon|lunch| +53501|AAAAAAAAOPANAAAA|53501|14|51|41|PM|first|afternoon|lunch| +53502|AAAAAAAAPPANAAAA|53502|14|51|42|PM|first|afternoon|lunch| +53503|AAAAAAAAAABNAAAA|53503|14|51|43|PM|first|afternoon|lunch| +53504|AAAAAAAABABNAAAA|53504|14|51|44|PM|first|afternoon|lunch| +53505|AAAAAAAACABNAAAA|53505|14|51|45|PM|first|afternoon|lunch| +53506|AAAAAAAADABNAAAA|53506|14|51|46|PM|first|afternoon|lunch| +53507|AAAAAAAAEABNAAAA|53507|14|51|47|PM|first|afternoon|lunch| +53508|AAAAAAAAFABNAAAA|53508|14|51|48|PM|first|afternoon|lunch| +53509|AAAAAAAAGABNAAAA|53509|14|51|49|PM|first|afternoon|lunch| +53510|AAAAAAAAHABNAAAA|53510|14|51|50|PM|first|afternoon|lunch| +53511|AAAAAAAAIABNAAAA|53511|14|51|51|PM|first|afternoon|lunch| +53512|AAAAAAAAJABNAAAA|53512|14|51|52|PM|first|afternoon|lunch| +53513|AAAAAAAAKABNAAAA|53513|14|51|53|PM|first|afternoon|lunch| +53514|AAAAAAAALABNAAAA|53514|14|51|54|PM|first|afternoon|lunch| +53515|AAAAAAAAMABNAAAA|53515|14|51|55|PM|first|afternoon|lunch| +53516|AAAAAAAANABNAAAA|53516|14|51|56|PM|first|afternoon|lunch| +53517|AAAAAAAAOABNAAAA|53517|14|51|57|PM|first|afternoon|lunch| +53518|AAAAAAAAPABNAAAA|53518|14|51|58|PM|first|afternoon|lunch| +53519|AAAAAAAAABBNAAAA|53519|14|51|59|PM|first|afternoon|lunch| +53520|AAAAAAAABBBNAAAA|53520|14|52|0|PM|first|afternoon|lunch| +53521|AAAAAAAACBBNAAAA|53521|14|52|1|PM|first|afternoon|lunch| +53522|AAAAAAAADBBNAAAA|53522|14|52|2|PM|first|afternoon|lunch| +53523|AAAAAAAAEBBNAAAA|53523|14|52|3|PM|first|afternoon|lunch| +53524|AAAAAAAAFBBNAAAA|53524|14|52|4|PM|first|afternoon|lunch| +53525|AAAAAAAAGBBNAAAA|53525|14|52|5|PM|first|afternoon|lunch| +53526|AAAAAAAAHBBNAAAA|53526|14|52|6|PM|first|afternoon|lunch| +53527|AAAAAAAAIBBNAAAA|53527|14|52|7|PM|first|afternoon|lunch| +53528|AAAAAAAAJBBNAAAA|53528|14|52|8|PM|first|afternoon|lunch| +53529|AAAAAAAAKBBNAAAA|53529|14|52|9|PM|first|afternoon|lunch| +53530|AAAAAAAALBBNAAAA|53530|14|52|10|PM|first|afternoon|lunch| +53531|AAAAAAAAMBBNAAAA|53531|14|52|11|PM|first|afternoon|lunch| +53532|AAAAAAAANBBNAAAA|53532|14|52|12|PM|first|afternoon|lunch| +53533|AAAAAAAAOBBNAAAA|53533|14|52|13|PM|first|afternoon|lunch| +53534|AAAAAAAAPBBNAAAA|53534|14|52|14|PM|first|afternoon|lunch| +53535|AAAAAAAAACBNAAAA|53535|14|52|15|PM|first|afternoon|lunch| +53536|AAAAAAAABCBNAAAA|53536|14|52|16|PM|first|afternoon|lunch| +53537|AAAAAAAACCBNAAAA|53537|14|52|17|PM|first|afternoon|lunch| +53538|AAAAAAAADCBNAAAA|53538|14|52|18|PM|first|afternoon|lunch| +53539|AAAAAAAAECBNAAAA|53539|14|52|19|PM|first|afternoon|lunch| +53540|AAAAAAAAFCBNAAAA|53540|14|52|20|PM|first|afternoon|lunch| +53541|AAAAAAAAGCBNAAAA|53541|14|52|21|PM|first|afternoon|lunch| +53542|AAAAAAAAHCBNAAAA|53542|14|52|22|PM|first|afternoon|lunch| +53543|AAAAAAAAICBNAAAA|53543|14|52|23|PM|first|afternoon|lunch| +53544|AAAAAAAAJCBNAAAA|53544|14|52|24|PM|first|afternoon|lunch| +53545|AAAAAAAAKCBNAAAA|53545|14|52|25|PM|first|afternoon|lunch| +53546|AAAAAAAALCBNAAAA|53546|14|52|26|PM|first|afternoon|lunch| +53547|AAAAAAAAMCBNAAAA|53547|14|52|27|PM|first|afternoon|lunch| +53548|AAAAAAAANCBNAAAA|53548|14|52|28|PM|first|afternoon|lunch| +53549|AAAAAAAAOCBNAAAA|53549|14|52|29|PM|first|afternoon|lunch| +53550|AAAAAAAAPCBNAAAA|53550|14|52|30|PM|first|afternoon|lunch| +53551|AAAAAAAAADBNAAAA|53551|14|52|31|PM|first|afternoon|lunch| +53552|AAAAAAAABDBNAAAA|53552|14|52|32|PM|first|afternoon|lunch| +53553|AAAAAAAACDBNAAAA|53553|14|52|33|PM|first|afternoon|lunch| +53554|AAAAAAAADDBNAAAA|53554|14|52|34|PM|first|afternoon|lunch| +53555|AAAAAAAAEDBNAAAA|53555|14|52|35|PM|first|afternoon|lunch| +53556|AAAAAAAAFDBNAAAA|53556|14|52|36|PM|first|afternoon|lunch| +53557|AAAAAAAAGDBNAAAA|53557|14|52|37|PM|first|afternoon|lunch| +53558|AAAAAAAAHDBNAAAA|53558|14|52|38|PM|first|afternoon|lunch| +53559|AAAAAAAAIDBNAAAA|53559|14|52|39|PM|first|afternoon|lunch| +53560|AAAAAAAAJDBNAAAA|53560|14|52|40|PM|first|afternoon|lunch| +53561|AAAAAAAAKDBNAAAA|53561|14|52|41|PM|first|afternoon|lunch| +53562|AAAAAAAALDBNAAAA|53562|14|52|42|PM|first|afternoon|lunch| +53563|AAAAAAAAMDBNAAAA|53563|14|52|43|PM|first|afternoon|lunch| +53564|AAAAAAAANDBNAAAA|53564|14|52|44|PM|first|afternoon|lunch| +53565|AAAAAAAAODBNAAAA|53565|14|52|45|PM|first|afternoon|lunch| +53566|AAAAAAAAPDBNAAAA|53566|14|52|46|PM|first|afternoon|lunch| +53567|AAAAAAAAAEBNAAAA|53567|14|52|47|PM|first|afternoon|lunch| +53568|AAAAAAAABEBNAAAA|53568|14|52|48|PM|first|afternoon|lunch| +53569|AAAAAAAACEBNAAAA|53569|14|52|49|PM|first|afternoon|lunch| +53570|AAAAAAAADEBNAAAA|53570|14|52|50|PM|first|afternoon|lunch| +53571|AAAAAAAAEEBNAAAA|53571|14|52|51|PM|first|afternoon|lunch| +53572|AAAAAAAAFEBNAAAA|53572|14|52|52|PM|first|afternoon|lunch| +53573|AAAAAAAAGEBNAAAA|53573|14|52|53|PM|first|afternoon|lunch| +53574|AAAAAAAAHEBNAAAA|53574|14|52|54|PM|first|afternoon|lunch| +53575|AAAAAAAAIEBNAAAA|53575|14|52|55|PM|first|afternoon|lunch| +53576|AAAAAAAAJEBNAAAA|53576|14|52|56|PM|first|afternoon|lunch| +53577|AAAAAAAAKEBNAAAA|53577|14|52|57|PM|first|afternoon|lunch| +53578|AAAAAAAALEBNAAAA|53578|14|52|58|PM|first|afternoon|lunch| +53579|AAAAAAAAMEBNAAAA|53579|14|52|59|PM|first|afternoon|lunch| +53580|AAAAAAAANEBNAAAA|53580|14|53|0|PM|first|afternoon|lunch| +53581|AAAAAAAAOEBNAAAA|53581|14|53|1|PM|first|afternoon|lunch| +53582|AAAAAAAAPEBNAAAA|53582|14|53|2|PM|first|afternoon|lunch| +53583|AAAAAAAAAFBNAAAA|53583|14|53|3|PM|first|afternoon|lunch| +53584|AAAAAAAABFBNAAAA|53584|14|53|4|PM|first|afternoon|lunch| +53585|AAAAAAAACFBNAAAA|53585|14|53|5|PM|first|afternoon|lunch| +53586|AAAAAAAADFBNAAAA|53586|14|53|6|PM|first|afternoon|lunch| +53587|AAAAAAAAEFBNAAAA|53587|14|53|7|PM|first|afternoon|lunch| +53588|AAAAAAAAFFBNAAAA|53588|14|53|8|PM|first|afternoon|lunch| +53589|AAAAAAAAGFBNAAAA|53589|14|53|9|PM|first|afternoon|lunch| +53590|AAAAAAAAHFBNAAAA|53590|14|53|10|PM|first|afternoon|lunch| +53591|AAAAAAAAIFBNAAAA|53591|14|53|11|PM|first|afternoon|lunch| +53592|AAAAAAAAJFBNAAAA|53592|14|53|12|PM|first|afternoon|lunch| +53593|AAAAAAAAKFBNAAAA|53593|14|53|13|PM|first|afternoon|lunch| +53594|AAAAAAAALFBNAAAA|53594|14|53|14|PM|first|afternoon|lunch| +53595|AAAAAAAAMFBNAAAA|53595|14|53|15|PM|first|afternoon|lunch| +53596|AAAAAAAANFBNAAAA|53596|14|53|16|PM|first|afternoon|lunch| +53597|AAAAAAAAOFBNAAAA|53597|14|53|17|PM|first|afternoon|lunch| +53598|AAAAAAAAPFBNAAAA|53598|14|53|18|PM|first|afternoon|lunch| +53599|AAAAAAAAAGBNAAAA|53599|14|53|19|PM|first|afternoon|lunch| +53600|AAAAAAAABGBNAAAA|53600|14|53|20|PM|first|afternoon|lunch| +53601|AAAAAAAACGBNAAAA|53601|14|53|21|PM|first|afternoon|lunch| +53602|AAAAAAAADGBNAAAA|53602|14|53|22|PM|first|afternoon|lunch| +53603|AAAAAAAAEGBNAAAA|53603|14|53|23|PM|first|afternoon|lunch| +53604|AAAAAAAAFGBNAAAA|53604|14|53|24|PM|first|afternoon|lunch| +53605|AAAAAAAAGGBNAAAA|53605|14|53|25|PM|first|afternoon|lunch| +53606|AAAAAAAAHGBNAAAA|53606|14|53|26|PM|first|afternoon|lunch| +53607|AAAAAAAAIGBNAAAA|53607|14|53|27|PM|first|afternoon|lunch| +53608|AAAAAAAAJGBNAAAA|53608|14|53|28|PM|first|afternoon|lunch| +53609|AAAAAAAAKGBNAAAA|53609|14|53|29|PM|first|afternoon|lunch| +53610|AAAAAAAALGBNAAAA|53610|14|53|30|PM|first|afternoon|lunch| +53611|AAAAAAAAMGBNAAAA|53611|14|53|31|PM|first|afternoon|lunch| +53612|AAAAAAAANGBNAAAA|53612|14|53|32|PM|first|afternoon|lunch| +53613|AAAAAAAAOGBNAAAA|53613|14|53|33|PM|first|afternoon|lunch| +53614|AAAAAAAAPGBNAAAA|53614|14|53|34|PM|first|afternoon|lunch| +53615|AAAAAAAAAHBNAAAA|53615|14|53|35|PM|first|afternoon|lunch| +53616|AAAAAAAABHBNAAAA|53616|14|53|36|PM|first|afternoon|lunch| +53617|AAAAAAAACHBNAAAA|53617|14|53|37|PM|first|afternoon|lunch| +53618|AAAAAAAADHBNAAAA|53618|14|53|38|PM|first|afternoon|lunch| +53619|AAAAAAAAEHBNAAAA|53619|14|53|39|PM|first|afternoon|lunch| +53620|AAAAAAAAFHBNAAAA|53620|14|53|40|PM|first|afternoon|lunch| +53621|AAAAAAAAGHBNAAAA|53621|14|53|41|PM|first|afternoon|lunch| +53622|AAAAAAAAHHBNAAAA|53622|14|53|42|PM|first|afternoon|lunch| +53623|AAAAAAAAIHBNAAAA|53623|14|53|43|PM|first|afternoon|lunch| +53624|AAAAAAAAJHBNAAAA|53624|14|53|44|PM|first|afternoon|lunch| +53625|AAAAAAAAKHBNAAAA|53625|14|53|45|PM|first|afternoon|lunch| +53626|AAAAAAAALHBNAAAA|53626|14|53|46|PM|first|afternoon|lunch| +53627|AAAAAAAAMHBNAAAA|53627|14|53|47|PM|first|afternoon|lunch| +53628|AAAAAAAANHBNAAAA|53628|14|53|48|PM|first|afternoon|lunch| +53629|AAAAAAAAOHBNAAAA|53629|14|53|49|PM|first|afternoon|lunch| +53630|AAAAAAAAPHBNAAAA|53630|14|53|50|PM|first|afternoon|lunch| +53631|AAAAAAAAAIBNAAAA|53631|14|53|51|PM|first|afternoon|lunch| +53632|AAAAAAAABIBNAAAA|53632|14|53|52|PM|first|afternoon|lunch| +53633|AAAAAAAACIBNAAAA|53633|14|53|53|PM|first|afternoon|lunch| +53634|AAAAAAAADIBNAAAA|53634|14|53|54|PM|first|afternoon|lunch| +53635|AAAAAAAAEIBNAAAA|53635|14|53|55|PM|first|afternoon|lunch| +53636|AAAAAAAAFIBNAAAA|53636|14|53|56|PM|first|afternoon|lunch| +53637|AAAAAAAAGIBNAAAA|53637|14|53|57|PM|first|afternoon|lunch| +53638|AAAAAAAAHIBNAAAA|53638|14|53|58|PM|first|afternoon|lunch| +53639|AAAAAAAAIIBNAAAA|53639|14|53|59|PM|first|afternoon|lunch| +53640|AAAAAAAAJIBNAAAA|53640|14|54|0|PM|first|afternoon|lunch| +53641|AAAAAAAAKIBNAAAA|53641|14|54|1|PM|first|afternoon|lunch| +53642|AAAAAAAALIBNAAAA|53642|14|54|2|PM|first|afternoon|lunch| +53643|AAAAAAAAMIBNAAAA|53643|14|54|3|PM|first|afternoon|lunch| +53644|AAAAAAAANIBNAAAA|53644|14|54|4|PM|first|afternoon|lunch| +53645|AAAAAAAAOIBNAAAA|53645|14|54|5|PM|first|afternoon|lunch| +53646|AAAAAAAAPIBNAAAA|53646|14|54|6|PM|first|afternoon|lunch| +53647|AAAAAAAAAJBNAAAA|53647|14|54|7|PM|first|afternoon|lunch| +53648|AAAAAAAABJBNAAAA|53648|14|54|8|PM|first|afternoon|lunch| +53649|AAAAAAAACJBNAAAA|53649|14|54|9|PM|first|afternoon|lunch| +53650|AAAAAAAADJBNAAAA|53650|14|54|10|PM|first|afternoon|lunch| +53651|AAAAAAAAEJBNAAAA|53651|14|54|11|PM|first|afternoon|lunch| +53652|AAAAAAAAFJBNAAAA|53652|14|54|12|PM|first|afternoon|lunch| +53653|AAAAAAAAGJBNAAAA|53653|14|54|13|PM|first|afternoon|lunch| +53654|AAAAAAAAHJBNAAAA|53654|14|54|14|PM|first|afternoon|lunch| +53655|AAAAAAAAIJBNAAAA|53655|14|54|15|PM|first|afternoon|lunch| +53656|AAAAAAAAJJBNAAAA|53656|14|54|16|PM|first|afternoon|lunch| +53657|AAAAAAAAKJBNAAAA|53657|14|54|17|PM|first|afternoon|lunch| +53658|AAAAAAAALJBNAAAA|53658|14|54|18|PM|first|afternoon|lunch| +53659|AAAAAAAAMJBNAAAA|53659|14|54|19|PM|first|afternoon|lunch| +53660|AAAAAAAANJBNAAAA|53660|14|54|20|PM|first|afternoon|lunch| +53661|AAAAAAAAOJBNAAAA|53661|14|54|21|PM|first|afternoon|lunch| +53662|AAAAAAAAPJBNAAAA|53662|14|54|22|PM|first|afternoon|lunch| +53663|AAAAAAAAAKBNAAAA|53663|14|54|23|PM|first|afternoon|lunch| +53664|AAAAAAAABKBNAAAA|53664|14|54|24|PM|first|afternoon|lunch| +53665|AAAAAAAACKBNAAAA|53665|14|54|25|PM|first|afternoon|lunch| +53666|AAAAAAAADKBNAAAA|53666|14|54|26|PM|first|afternoon|lunch| +53667|AAAAAAAAEKBNAAAA|53667|14|54|27|PM|first|afternoon|lunch| +53668|AAAAAAAAFKBNAAAA|53668|14|54|28|PM|first|afternoon|lunch| +53669|AAAAAAAAGKBNAAAA|53669|14|54|29|PM|first|afternoon|lunch| +53670|AAAAAAAAHKBNAAAA|53670|14|54|30|PM|first|afternoon|lunch| +53671|AAAAAAAAIKBNAAAA|53671|14|54|31|PM|first|afternoon|lunch| +53672|AAAAAAAAJKBNAAAA|53672|14|54|32|PM|first|afternoon|lunch| +53673|AAAAAAAAKKBNAAAA|53673|14|54|33|PM|first|afternoon|lunch| +53674|AAAAAAAALKBNAAAA|53674|14|54|34|PM|first|afternoon|lunch| +53675|AAAAAAAAMKBNAAAA|53675|14|54|35|PM|first|afternoon|lunch| +53676|AAAAAAAANKBNAAAA|53676|14|54|36|PM|first|afternoon|lunch| +53677|AAAAAAAAOKBNAAAA|53677|14|54|37|PM|first|afternoon|lunch| +53678|AAAAAAAAPKBNAAAA|53678|14|54|38|PM|first|afternoon|lunch| +53679|AAAAAAAAALBNAAAA|53679|14|54|39|PM|first|afternoon|lunch| +53680|AAAAAAAABLBNAAAA|53680|14|54|40|PM|first|afternoon|lunch| +53681|AAAAAAAACLBNAAAA|53681|14|54|41|PM|first|afternoon|lunch| +53682|AAAAAAAADLBNAAAA|53682|14|54|42|PM|first|afternoon|lunch| +53683|AAAAAAAAELBNAAAA|53683|14|54|43|PM|first|afternoon|lunch| +53684|AAAAAAAAFLBNAAAA|53684|14|54|44|PM|first|afternoon|lunch| +53685|AAAAAAAAGLBNAAAA|53685|14|54|45|PM|first|afternoon|lunch| +53686|AAAAAAAAHLBNAAAA|53686|14|54|46|PM|first|afternoon|lunch| +53687|AAAAAAAAILBNAAAA|53687|14|54|47|PM|first|afternoon|lunch| +53688|AAAAAAAAJLBNAAAA|53688|14|54|48|PM|first|afternoon|lunch| +53689|AAAAAAAAKLBNAAAA|53689|14|54|49|PM|first|afternoon|lunch| +53690|AAAAAAAALLBNAAAA|53690|14|54|50|PM|first|afternoon|lunch| +53691|AAAAAAAAMLBNAAAA|53691|14|54|51|PM|first|afternoon|lunch| +53692|AAAAAAAANLBNAAAA|53692|14|54|52|PM|first|afternoon|lunch| +53693|AAAAAAAAOLBNAAAA|53693|14|54|53|PM|first|afternoon|lunch| +53694|AAAAAAAAPLBNAAAA|53694|14|54|54|PM|first|afternoon|lunch| +53695|AAAAAAAAAMBNAAAA|53695|14|54|55|PM|first|afternoon|lunch| +53696|AAAAAAAABMBNAAAA|53696|14|54|56|PM|first|afternoon|lunch| +53697|AAAAAAAACMBNAAAA|53697|14|54|57|PM|first|afternoon|lunch| +53698|AAAAAAAADMBNAAAA|53698|14|54|58|PM|first|afternoon|lunch| +53699|AAAAAAAAEMBNAAAA|53699|14|54|59|PM|first|afternoon|lunch| +53700|AAAAAAAAFMBNAAAA|53700|14|55|0|PM|first|afternoon|lunch| +53701|AAAAAAAAGMBNAAAA|53701|14|55|1|PM|first|afternoon|lunch| +53702|AAAAAAAAHMBNAAAA|53702|14|55|2|PM|first|afternoon|lunch| +53703|AAAAAAAAIMBNAAAA|53703|14|55|3|PM|first|afternoon|lunch| +53704|AAAAAAAAJMBNAAAA|53704|14|55|4|PM|first|afternoon|lunch| +53705|AAAAAAAAKMBNAAAA|53705|14|55|5|PM|first|afternoon|lunch| +53706|AAAAAAAALMBNAAAA|53706|14|55|6|PM|first|afternoon|lunch| +53707|AAAAAAAAMMBNAAAA|53707|14|55|7|PM|first|afternoon|lunch| +53708|AAAAAAAANMBNAAAA|53708|14|55|8|PM|first|afternoon|lunch| +53709|AAAAAAAAOMBNAAAA|53709|14|55|9|PM|first|afternoon|lunch| +53710|AAAAAAAAPMBNAAAA|53710|14|55|10|PM|first|afternoon|lunch| +53711|AAAAAAAAANBNAAAA|53711|14|55|11|PM|first|afternoon|lunch| +53712|AAAAAAAABNBNAAAA|53712|14|55|12|PM|first|afternoon|lunch| +53713|AAAAAAAACNBNAAAA|53713|14|55|13|PM|first|afternoon|lunch| +53714|AAAAAAAADNBNAAAA|53714|14|55|14|PM|first|afternoon|lunch| +53715|AAAAAAAAENBNAAAA|53715|14|55|15|PM|first|afternoon|lunch| +53716|AAAAAAAAFNBNAAAA|53716|14|55|16|PM|first|afternoon|lunch| +53717|AAAAAAAAGNBNAAAA|53717|14|55|17|PM|first|afternoon|lunch| +53718|AAAAAAAAHNBNAAAA|53718|14|55|18|PM|first|afternoon|lunch| +53719|AAAAAAAAINBNAAAA|53719|14|55|19|PM|first|afternoon|lunch| +53720|AAAAAAAAJNBNAAAA|53720|14|55|20|PM|first|afternoon|lunch| +53721|AAAAAAAAKNBNAAAA|53721|14|55|21|PM|first|afternoon|lunch| +53722|AAAAAAAALNBNAAAA|53722|14|55|22|PM|first|afternoon|lunch| +53723|AAAAAAAAMNBNAAAA|53723|14|55|23|PM|first|afternoon|lunch| +53724|AAAAAAAANNBNAAAA|53724|14|55|24|PM|first|afternoon|lunch| +53725|AAAAAAAAONBNAAAA|53725|14|55|25|PM|first|afternoon|lunch| +53726|AAAAAAAAPNBNAAAA|53726|14|55|26|PM|first|afternoon|lunch| +53727|AAAAAAAAAOBNAAAA|53727|14|55|27|PM|first|afternoon|lunch| +53728|AAAAAAAABOBNAAAA|53728|14|55|28|PM|first|afternoon|lunch| +53729|AAAAAAAACOBNAAAA|53729|14|55|29|PM|first|afternoon|lunch| +53730|AAAAAAAADOBNAAAA|53730|14|55|30|PM|first|afternoon|lunch| +53731|AAAAAAAAEOBNAAAA|53731|14|55|31|PM|first|afternoon|lunch| +53732|AAAAAAAAFOBNAAAA|53732|14|55|32|PM|first|afternoon|lunch| +53733|AAAAAAAAGOBNAAAA|53733|14|55|33|PM|first|afternoon|lunch| +53734|AAAAAAAAHOBNAAAA|53734|14|55|34|PM|first|afternoon|lunch| +53735|AAAAAAAAIOBNAAAA|53735|14|55|35|PM|first|afternoon|lunch| +53736|AAAAAAAAJOBNAAAA|53736|14|55|36|PM|first|afternoon|lunch| +53737|AAAAAAAAKOBNAAAA|53737|14|55|37|PM|first|afternoon|lunch| +53738|AAAAAAAALOBNAAAA|53738|14|55|38|PM|first|afternoon|lunch| +53739|AAAAAAAAMOBNAAAA|53739|14|55|39|PM|first|afternoon|lunch| +53740|AAAAAAAANOBNAAAA|53740|14|55|40|PM|first|afternoon|lunch| +53741|AAAAAAAAOOBNAAAA|53741|14|55|41|PM|first|afternoon|lunch| +53742|AAAAAAAAPOBNAAAA|53742|14|55|42|PM|first|afternoon|lunch| +53743|AAAAAAAAAPBNAAAA|53743|14|55|43|PM|first|afternoon|lunch| +53744|AAAAAAAABPBNAAAA|53744|14|55|44|PM|first|afternoon|lunch| +53745|AAAAAAAACPBNAAAA|53745|14|55|45|PM|first|afternoon|lunch| +53746|AAAAAAAADPBNAAAA|53746|14|55|46|PM|first|afternoon|lunch| +53747|AAAAAAAAEPBNAAAA|53747|14|55|47|PM|first|afternoon|lunch| +53748|AAAAAAAAFPBNAAAA|53748|14|55|48|PM|first|afternoon|lunch| +53749|AAAAAAAAGPBNAAAA|53749|14|55|49|PM|first|afternoon|lunch| +53750|AAAAAAAAHPBNAAAA|53750|14|55|50|PM|first|afternoon|lunch| +53751|AAAAAAAAIPBNAAAA|53751|14|55|51|PM|first|afternoon|lunch| +53752|AAAAAAAAJPBNAAAA|53752|14|55|52|PM|first|afternoon|lunch| +53753|AAAAAAAAKPBNAAAA|53753|14|55|53|PM|first|afternoon|lunch| +53754|AAAAAAAALPBNAAAA|53754|14|55|54|PM|first|afternoon|lunch| +53755|AAAAAAAAMPBNAAAA|53755|14|55|55|PM|first|afternoon|lunch| +53756|AAAAAAAANPBNAAAA|53756|14|55|56|PM|first|afternoon|lunch| +53757|AAAAAAAAOPBNAAAA|53757|14|55|57|PM|first|afternoon|lunch| +53758|AAAAAAAAPPBNAAAA|53758|14|55|58|PM|first|afternoon|lunch| +53759|AAAAAAAAAACNAAAA|53759|14|55|59|PM|first|afternoon|lunch| +53760|AAAAAAAABACNAAAA|53760|14|56|0|PM|first|afternoon|lunch| +53761|AAAAAAAACACNAAAA|53761|14|56|1|PM|first|afternoon|lunch| +53762|AAAAAAAADACNAAAA|53762|14|56|2|PM|first|afternoon|lunch| +53763|AAAAAAAAEACNAAAA|53763|14|56|3|PM|first|afternoon|lunch| +53764|AAAAAAAAFACNAAAA|53764|14|56|4|PM|first|afternoon|lunch| +53765|AAAAAAAAGACNAAAA|53765|14|56|5|PM|first|afternoon|lunch| +53766|AAAAAAAAHACNAAAA|53766|14|56|6|PM|first|afternoon|lunch| +53767|AAAAAAAAIACNAAAA|53767|14|56|7|PM|first|afternoon|lunch| +53768|AAAAAAAAJACNAAAA|53768|14|56|8|PM|first|afternoon|lunch| +53769|AAAAAAAAKACNAAAA|53769|14|56|9|PM|first|afternoon|lunch| +53770|AAAAAAAALACNAAAA|53770|14|56|10|PM|first|afternoon|lunch| +53771|AAAAAAAAMACNAAAA|53771|14|56|11|PM|first|afternoon|lunch| +53772|AAAAAAAANACNAAAA|53772|14|56|12|PM|first|afternoon|lunch| +53773|AAAAAAAAOACNAAAA|53773|14|56|13|PM|first|afternoon|lunch| +53774|AAAAAAAAPACNAAAA|53774|14|56|14|PM|first|afternoon|lunch| +53775|AAAAAAAAABCNAAAA|53775|14|56|15|PM|first|afternoon|lunch| +53776|AAAAAAAABBCNAAAA|53776|14|56|16|PM|first|afternoon|lunch| +53777|AAAAAAAACBCNAAAA|53777|14|56|17|PM|first|afternoon|lunch| +53778|AAAAAAAADBCNAAAA|53778|14|56|18|PM|first|afternoon|lunch| +53779|AAAAAAAAEBCNAAAA|53779|14|56|19|PM|first|afternoon|lunch| +53780|AAAAAAAAFBCNAAAA|53780|14|56|20|PM|first|afternoon|lunch| +53781|AAAAAAAAGBCNAAAA|53781|14|56|21|PM|first|afternoon|lunch| +53782|AAAAAAAAHBCNAAAA|53782|14|56|22|PM|first|afternoon|lunch| +53783|AAAAAAAAIBCNAAAA|53783|14|56|23|PM|first|afternoon|lunch| +53784|AAAAAAAAJBCNAAAA|53784|14|56|24|PM|first|afternoon|lunch| +53785|AAAAAAAAKBCNAAAA|53785|14|56|25|PM|first|afternoon|lunch| +53786|AAAAAAAALBCNAAAA|53786|14|56|26|PM|first|afternoon|lunch| +53787|AAAAAAAAMBCNAAAA|53787|14|56|27|PM|first|afternoon|lunch| +53788|AAAAAAAANBCNAAAA|53788|14|56|28|PM|first|afternoon|lunch| +53789|AAAAAAAAOBCNAAAA|53789|14|56|29|PM|first|afternoon|lunch| +53790|AAAAAAAAPBCNAAAA|53790|14|56|30|PM|first|afternoon|lunch| +53791|AAAAAAAAACCNAAAA|53791|14|56|31|PM|first|afternoon|lunch| +53792|AAAAAAAABCCNAAAA|53792|14|56|32|PM|first|afternoon|lunch| +53793|AAAAAAAACCCNAAAA|53793|14|56|33|PM|first|afternoon|lunch| +53794|AAAAAAAADCCNAAAA|53794|14|56|34|PM|first|afternoon|lunch| +53795|AAAAAAAAECCNAAAA|53795|14|56|35|PM|first|afternoon|lunch| +53796|AAAAAAAAFCCNAAAA|53796|14|56|36|PM|first|afternoon|lunch| +53797|AAAAAAAAGCCNAAAA|53797|14|56|37|PM|first|afternoon|lunch| +53798|AAAAAAAAHCCNAAAA|53798|14|56|38|PM|first|afternoon|lunch| +53799|AAAAAAAAICCNAAAA|53799|14|56|39|PM|first|afternoon|lunch| +53800|AAAAAAAAJCCNAAAA|53800|14|56|40|PM|first|afternoon|lunch| +53801|AAAAAAAAKCCNAAAA|53801|14|56|41|PM|first|afternoon|lunch| +53802|AAAAAAAALCCNAAAA|53802|14|56|42|PM|first|afternoon|lunch| +53803|AAAAAAAAMCCNAAAA|53803|14|56|43|PM|first|afternoon|lunch| +53804|AAAAAAAANCCNAAAA|53804|14|56|44|PM|first|afternoon|lunch| +53805|AAAAAAAAOCCNAAAA|53805|14|56|45|PM|first|afternoon|lunch| +53806|AAAAAAAAPCCNAAAA|53806|14|56|46|PM|first|afternoon|lunch| +53807|AAAAAAAAADCNAAAA|53807|14|56|47|PM|first|afternoon|lunch| +53808|AAAAAAAABDCNAAAA|53808|14|56|48|PM|first|afternoon|lunch| +53809|AAAAAAAACDCNAAAA|53809|14|56|49|PM|first|afternoon|lunch| +53810|AAAAAAAADDCNAAAA|53810|14|56|50|PM|first|afternoon|lunch| +53811|AAAAAAAAEDCNAAAA|53811|14|56|51|PM|first|afternoon|lunch| +53812|AAAAAAAAFDCNAAAA|53812|14|56|52|PM|first|afternoon|lunch| +53813|AAAAAAAAGDCNAAAA|53813|14|56|53|PM|first|afternoon|lunch| +53814|AAAAAAAAHDCNAAAA|53814|14|56|54|PM|first|afternoon|lunch| +53815|AAAAAAAAIDCNAAAA|53815|14|56|55|PM|first|afternoon|lunch| +53816|AAAAAAAAJDCNAAAA|53816|14|56|56|PM|first|afternoon|lunch| +53817|AAAAAAAAKDCNAAAA|53817|14|56|57|PM|first|afternoon|lunch| +53818|AAAAAAAALDCNAAAA|53818|14|56|58|PM|first|afternoon|lunch| +53819|AAAAAAAAMDCNAAAA|53819|14|56|59|PM|first|afternoon|lunch| +53820|AAAAAAAANDCNAAAA|53820|14|57|0|PM|first|afternoon|lunch| +53821|AAAAAAAAODCNAAAA|53821|14|57|1|PM|first|afternoon|lunch| +53822|AAAAAAAAPDCNAAAA|53822|14|57|2|PM|first|afternoon|lunch| +53823|AAAAAAAAAECNAAAA|53823|14|57|3|PM|first|afternoon|lunch| +53824|AAAAAAAABECNAAAA|53824|14|57|4|PM|first|afternoon|lunch| +53825|AAAAAAAACECNAAAA|53825|14|57|5|PM|first|afternoon|lunch| +53826|AAAAAAAADECNAAAA|53826|14|57|6|PM|first|afternoon|lunch| +53827|AAAAAAAAEECNAAAA|53827|14|57|7|PM|first|afternoon|lunch| +53828|AAAAAAAAFECNAAAA|53828|14|57|8|PM|first|afternoon|lunch| +53829|AAAAAAAAGECNAAAA|53829|14|57|9|PM|first|afternoon|lunch| +53830|AAAAAAAAHECNAAAA|53830|14|57|10|PM|first|afternoon|lunch| +53831|AAAAAAAAIECNAAAA|53831|14|57|11|PM|first|afternoon|lunch| +53832|AAAAAAAAJECNAAAA|53832|14|57|12|PM|first|afternoon|lunch| +53833|AAAAAAAAKECNAAAA|53833|14|57|13|PM|first|afternoon|lunch| +53834|AAAAAAAALECNAAAA|53834|14|57|14|PM|first|afternoon|lunch| +53835|AAAAAAAAMECNAAAA|53835|14|57|15|PM|first|afternoon|lunch| +53836|AAAAAAAANECNAAAA|53836|14|57|16|PM|first|afternoon|lunch| +53837|AAAAAAAAOECNAAAA|53837|14|57|17|PM|first|afternoon|lunch| +53838|AAAAAAAAPECNAAAA|53838|14|57|18|PM|first|afternoon|lunch| +53839|AAAAAAAAAFCNAAAA|53839|14|57|19|PM|first|afternoon|lunch| +53840|AAAAAAAABFCNAAAA|53840|14|57|20|PM|first|afternoon|lunch| +53841|AAAAAAAACFCNAAAA|53841|14|57|21|PM|first|afternoon|lunch| +53842|AAAAAAAADFCNAAAA|53842|14|57|22|PM|first|afternoon|lunch| +53843|AAAAAAAAEFCNAAAA|53843|14|57|23|PM|first|afternoon|lunch| +53844|AAAAAAAAFFCNAAAA|53844|14|57|24|PM|first|afternoon|lunch| +53845|AAAAAAAAGFCNAAAA|53845|14|57|25|PM|first|afternoon|lunch| +53846|AAAAAAAAHFCNAAAA|53846|14|57|26|PM|first|afternoon|lunch| +53847|AAAAAAAAIFCNAAAA|53847|14|57|27|PM|first|afternoon|lunch| +53848|AAAAAAAAJFCNAAAA|53848|14|57|28|PM|first|afternoon|lunch| +53849|AAAAAAAAKFCNAAAA|53849|14|57|29|PM|first|afternoon|lunch| +53850|AAAAAAAALFCNAAAA|53850|14|57|30|PM|first|afternoon|lunch| +53851|AAAAAAAAMFCNAAAA|53851|14|57|31|PM|first|afternoon|lunch| +53852|AAAAAAAANFCNAAAA|53852|14|57|32|PM|first|afternoon|lunch| +53853|AAAAAAAAOFCNAAAA|53853|14|57|33|PM|first|afternoon|lunch| +53854|AAAAAAAAPFCNAAAA|53854|14|57|34|PM|first|afternoon|lunch| +53855|AAAAAAAAAGCNAAAA|53855|14|57|35|PM|first|afternoon|lunch| +53856|AAAAAAAABGCNAAAA|53856|14|57|36|PM|first|afternoon|lunch| +53857|AAAAAAAACGCNAAAA|53857|14|57|37|PM|first|afternoon|lunch| +53858|AAAAAAAADGCNAAAA|53858|14|57|38|PM|first|afternoon|lunch| +53859|AAAAAAAAEGCNAAAA|53859|14|57|39|PM|first|afternoon|lunch| +53860|AAAAAAAAFGCNAAAA|53860|14|57|40|PM|first|afternoon|lunch| +53861|AAAAAAAAGGCNAAAA|53861|14|57|41|PM|first|afternoon|lunch| +53862|AAAAAAAAHGCNAAAA|53862|14|57|42|PM|first|afternoon|lunch| +53863|AAAAAAAAIGCNAAAA|53863|14|57|43|PM|first|afternoon|lunch| +53864|AAAAAAAAJGCNAAAA|53864|14|57|44|PM|first|afternoon|lunch| +53865|AAAAAAAAKGCNAAAA|53865|14|57|45|PM|first|afternoon|lunch| +53866|AAAAAAAALGCNAAAA|53866|14|57|46|PM|first|afternoon|lunch| +53867|AAAAAAAAMGCNAAAA|53867|14|57|47|PM|first|afternoon|lunch| +53868|AAAAAAAANGCNAAAA|53868|14|57|48|PM|first|afternoon|lunch| +53869|AAAAAAAAOGCNAAAA|53869|14|57|49|PM|first|afternoon|lunch| +53870|AAAAAAAAPGCNAAAA|53870|14|57|50|PM|first|afternoon|lunch| +53871|AAAAAAAAAHCNAAAA|53871|14|57|51|PM|first|afternoon|lunch| +53872|AAAAAAAABHCNAAAA|53872|14|57|52|PM|first|afternoon|lunch| +53873|AAAAAAAACHCNAAAA|53873|14|57|53|PM|first|afternoon|lunch| +53874|AAAAAAAADHCNAAAA|53874|14|57|54|PM|first|afternoon|lunch| +53875|AAAAAAAAEHCNAAAA|53875|14|57|55|PM|first|afternoon|lunch| +53876|AAAAAAAAFHCNAAAA|53876|14|57|56|PM|first|afternoon|lunch| +53877|AAAAAAAAGHCNAAAA|53877|14|57|57|PM|first|afternoon|lunch| +53878|AAAAAAAAHHCNAAAA|53878|14|57|58|PM|first|afternoon|lunch| +53879|AAAAAAAAIHCNAAAA|53879|14|57|59|PM|first|afternoon|lunch| +53880|AAAAAAAAJHCNAAAA|53880|14|58|0|PM|first|afternoon|lunch| +53881|AAAAAAAAKHCNAAAA|53881|14|58|1|PM|first|afternoon|lunch| +53882|AAAAAAAALHCNAAAA|53882|14|58|2|PM|first|afternoon|lunch| +53883|AAAAAAAAMHCNAAAA|53883|14|58|3|PM|first|afternoon|lunch| +53884|AAAAAAAANHCNAAAA|53884|14|58|4|PM|first|afternoon|lunch| +53885|AAAAAAAAOHCNAAAA|53885|14|58|5|PM|first|afternoon|lunch| +53886|AAAAAAAAPHCNAAAA|53886|14|58|6|PM|first|afternoon|lunch| +53887|AAAAAAAAAICNAAAA|53887|14|58|7|PM|first|afternoon|lunch| +53888|AAAAAAAABICNAAAA|53888|14|58|8|PM|first|afternoon|lunch| +53889|AAAAAAAACICNAAAA|53889|14|58|9|PM|first|afternoon|lunch| +53890|AAAAAAAADICNAAAA|53890|14|58|10|PM|first|afternoon|lunch| +53891|AAAAAAAAEICNAAAA|53891|14|58|11|PM|first|afternoon|lunch| +53892|AAAAAAAAFICNAAAA|53892|14|58|12|PM|first|afternoon|lunch| +53893|AAAAAAAAGICNAAAA|53893|14|58|13|PM|first|afternoon|lunch| +53894|AAAAAAAAHICNAAAA|53894|14|58|14|PM|first|afternoon|lunch| +53895|AAAAAAAAIICNAAAA|53895|14|58|15|PM|first|afternoon|lunch| +53896|AAAAAAAAJICNAAAA|53896|14|58|16|PM|first|afternoon|lunch| +53897|AAAAAAAAKICNAAAA|53897|14|58|17|PM|first|afternoon|lunch| +53898|AAAAAAAALICNAAAA|53898|14|58|18|PM|first|afternoon|lunch| +53899|AAAAAAAAMICNAAAA|53899|14|58|19|PM|first|afternoon|lunch| +53900|AAAAAAAANICNAAAA|53900|14|58|20|PM|first|afternoon|lunch| +53901|AAAAAAAAOICNAAAA|53901|14|58|21|PM|first|afternoon|lunch| +53902|AAAAAAAAPICNAAAA|53902|14|58|22|PM|first|afternoon|lunch| +53903|AAAAAAAAAJCNAAAA|53903|14|58|23|PM|first|afternoon|lunch| +53904|AAAAAAAABJCNAAAA|53904|14|58|24|PM|first|afternoon|lunch| +53905|AAAAAAAACJCNAAAA|53905|14|58|25|PM|first|afternoon|lunch| +53906|AAAAAAAADJCNAAAA|53906|14|58|26|PM|first|afternoon|lunch| +53907|AAAAAAAAEJCNAAAA|53907|14|58|27|PM|first|afternoon|lunch| +53908|AAAAAAAAFJCNAAAA|53908|14|58|28|PM|first|afternoon|lunch| +53909|AAAAAAAAGJCNAAAA|53909|14|58|29|PM|first|afternoon|lunch| +53910|AAAAAAAAHJCNAAAA|53910|14|58|30|PM|first|afternoon|lunch| +53911|AAAAAAAAIJCNAAAA|53911|14|58|31|PM|first|afternoon|lunch| +53912|AAAAAAAAJJCNAAAA|53912|14|58|32|PM|first|afternoon|lunch| +53913|AAAAAAAAKJCNAAAA|53913|14|58|33|PM|first|afternoon|lunch| +53914|AAAAAAAALJCNAAAA|53914|14|58|34|PM|first|afternoon|lunch| +53915|AAAAAAAAMJCNAAAA|53915|14|58|35|PM|first|afternoon|lunch| +53916|AAAAAAAANJCNAAAA|53916|14|58|36|PM|first|afternoon|lunch| +53917|AAAAAAAAOJCNAAAA|53917|14|58|37|PM|first|afternoon|lunch| +53918|AAAAAAAAPJCNAAAA|53918|14|58|38|PM|first|afternoon|lunch| +53919|AAAAAAAAAKCNAAAA|53919|14|58|39|PM|first|afternoon|lunch| +53920|AAAAAAAABKCNAAAA|53920|14|58|40|PM|first|afternoon|lunch| +53921|AAAAAAAACKCNAAAA|53921|14|58|41|PM|first|afternoon|lunch| +53922|AAAAAAAADKCNAAAA|53922|14|58|42|PM|first|afternoon|lunch| +53923|AAAAAAAAEKCNAAAA|53923|14|58|43|PM|first|afternoon|lunch| +53924|AAAAAAAAFKCNAAAA|53924|14|58|44|PM|first|afternoon|lunch| +53925|AAAAAAAAGKCNAAAA|53925|14|58|45|PM|first|afternoon|lunch| +53926|AAAAAAAAHKCNAAAA|53926|14|58|46|PM|first|afternoon|lunch| +53927|AAAAAAAAIKCNAAAA|53927|14|58|47|PM|first|afternoon|lunch| +53928|AAAAAAAAJKCNAAAA|53928|14|58|48|PM|first|afternoon|lunch| +53929|AAAAAAAAKKCNAAAA|53929|14|58|49|PM|first|afternoon|lunch| +53930|AAAAAAAALKCNAAAA|53930|14|58|50|PM|first|afternoon|lunch| +53931|AAAAAAAAMKCNAAAA|53931|14|58|51|PM|first|afternoon|lunch| +53932|AAAAAAAANKCNAAAA|53932|14|58|52|PM|first|afternoon|lunch| +53933|AAAAAAAAOKCNAAAA|53933|14|58|53|PM|first|afternoon|lunch| +53934|AAAAAAAAPKCNAAAA|53934|14|58|54|PM|first|afternoon|lunch| +53935|AAAAAAAAALCNAAAA|53935|14|58|55|PM|first|afternoon|lunch| +53936|AAAAAAAABLCNAAAA|53936|14|58|56|PM|first|afternoon|lunch| +53937|AAAAAAAACLCNAAAA|53937|14|58|57|PM|first|afternoon|lunch| +53938|AAAAAAAADLCNAAAA|53938|14|58|58|PM|first|afternoon|lunch| +53939|AAAAAAAAELCNAAAA|53939|14|58|59|PM|first|afternoon|lunch| +53940|AAAAAAAAFLCNAAAA|53940|14|59|0|PM|first|afternoon|lunch| +53941|AAAAAAAAGLCNAAAA|53941|14|59|1|PM|first|afternoon|lunch| +53942|AAAAAAAAHLCNAAAA|53942|14|59|2|PM|first|afternoon|lunch| +53943|AAAAAAAAILCNAAAA|53943|14|59|3|PM|first|afternoon|lunch| +53944|AAAAAAAAJLCNAAAA|53944|14|59|4|PM|first|afternoon|lunch| +53945|AAAAAAAAKLCNAAAA|53945|14|59|5|PM|first|afternoon|lunch| +53946|AAAAAAAALLCNAAAA|53946|14|59|6|PM|first|afternoon|lunch| +53947|AAAAAAAAMLCNAAAA|53947|14|59|7|PM|first|afternoon|lunch| +53948|AAAAAAAANLCNAAAA|53948|14|59|8|PM|first|afternoon|lunch| +53949|AAAAAAAAOLCNAAAA|53949|14|59|9|PM|first|afternoon|lunch| +53950|AAAAAAAAPLCNAAAA|53950|14|59|10|PM|first|afternoon|lunch| +53951|AAAAAAAAAMCNAAAA|53951|14|59|11|PM|first|afternoon|lunch| +53952|AAAAAAAABMCNAAAA|53952|14|59|12|PM|first|afternoon|lunch| +53953|AAAAAAAACMCNAAAA|53953|14|59|13|PM|first|afternoon|lunch| +53954|AAAAAAAADMCNAAAA|53954|14|59|14|PM|first|afternoon|lunch| +53955|AAAAAAAAEMCNAAAA|53955|14|59|15|PM|first|afternoon|lunch| +53956|AAAAAAAAFMCNAAAA|53956|14|59|16|PM|first|afternoon|lunch| +53957|AAAAAAAAGMCNAAAA|53957|14|59|17|PM|first|afternoon|lunch| +53958|AAAAAAAAHMCNAAAA|53958|14|59|18|PM|first|afternoon|lunch| +53959|AAAAAAAAIMCNAAAA|53959|14|59|19|PM|first|afternoon|lunch| +53960|AAAAAAAAJMCNAAAA|53960|14|59|20|PM|first|afternoon|lunch| +53961|AAAAAAAAKMCNAAAA|53961|14|59|21|PM|first|afternoon|lunch| +53962|AAAAAAAALMCNAAAA|53962|14|59|22|PM|first|afternoon|lunch| +53963|AAAAAAAAMMCNAAAA|53963|14|59|23|PM|first|afternoon|lunch| +53964|AAAAAAAANMCNAAAA|53964|14|59|24|PM|first|afternoon|lunch| +53965|AAAAAAAAOMCNAAAA|53965|14|59|25|PM|first|afternoon|lunch| +53966|AAAAAAAAPMCNAAAA|53966|14|59|26|PM|first|afternoon|lunch| +53967|AAAAAAAAANCNAAAA|53967|14|59|27|PM|first|afternoon|lunch| +53968|AAAAAAAABNCNAAAA|53968|14|59|28|PM|first|afternoon|lunch| +53969|AAAAAAAACNCNAAAA|53969|14|59|29|PM|first|afternoon|lunch| +53970|AAAAAAAADNCNAAAA|53970|14|59|30|PM|first|afternoon|lunch| +53971|AAAAAAAAENCNAAAA|53971|14|59|31|PM|first|afternoon|lunch| +53972|AAAAAAAAFNCNAAAA|53972|14|59|32|PM|first|afternoon|lunch| +53973|AAAAAAAAGNCNAAAA|53973|14|59|33|PM|first|afternoon|lunch| +53974|AAAAAAAAHNCNAAAA|53974|14|59|34|PM|first|afternoon|lunch| +53975|AAAAAAAAINCNAAAA|53975|14|59|35|PM|first|afternoon|lunch| +53976|AAAAAAAAJNCNAAAA|53976|14|59|36|PM|first|afternoon|lunch| +53977|AAAAAAAAKNCNAAAA|53977|14|59|37|PM|first|afternoon|lunch| +53978|AAAAAAAALNCNAAAA|53978|14|59|38|PM|first|afternoon|lunch| +53979|AAAAAAAAMNCNAAAA|53979|14|59|39|PM|first|afternoon|lunch| +53980|AAAAAAAANNCNAAAA|53980|14|59|40|PM|first|afternoon|lunch| +53981|AAAAAAAAONCNAAAA|53981|14|59|41|PM|first|afternoon|lunch| +53982|AAAAAAAAPNCNAAAA|53982|14|59|42|PM|first|afternoon|lunch| +53983|AAAAAAAAAOCNAAAA|53983|14|59|43|PM|first|afternoon|lunch| +53984|AAAAAAAABOCNAAAA|53984|14|59|44|PM|first|afternoon|lunch| +53985|AAAAAAAACOCNAAAA|53985|14|59|45|PM|first|afternoon|lunch| +53986|AAAAAAAADOCNAAAA|53986|14|59|46|PM|first|afternoon|lunch| +53987|AAAAAAAAEOCNAAAA|53987|14|59|47|PM|first|afternoon|lunch| +53988|AAAAAAAAFOCNAAAA|53988|14|59|48|PM|first|afternoon|lunch| +53989|AAAAAAAAGOCNAAAA|53989|14|59|49|PM|first|afternoon|lunch| +53990|AAAAAAAAHOCNAAAA|53990|14|59|50|PM|first|afternoon|lunch| +53991|AAAAAAAAIOCNAAAA|53991|14|59|51|PM|first|afternoon|lunch| +53992|AAAAAAAAJOCNAAAA|53992|14|59|52|PM|first|afternoon|lunch| +53993|AAAAAAAAKOCNAAAA|53993|14|59|53|PM|first|afternoon|lunch| +53994|AAAAAAAALOCNAAAA|53994|14|59|54|PM|first|afternoon|lunch| +53995|AAAAAAAAMOCNAAAA|53995|14|59|55|PM|first|afternoon|lunch| +53996|AAAAAAAANOCNAAAA|53996|14|59|56|PM|first|afternoon|lunch| +53997|AAAAAAAAOOCNAAAA|53997|14|59|57|PM|first|afternoon|lunch| +53998|AAAAAAAAPOCNAAAA|53998|14|59|58|PM|first|afternoon|lunch| +53999|AAAAAAAAAPCNAAAA|53999|14|59|59|PM|first|afternoon|lunch| +54000|AAAAAAAABPCNAAAA|54000|15|0|0|PM|second|afternoon|| +54001|AAAAAAAACPCNAAAA|54001|15|0|1|PM|second|afternoon|| +54002|AAAAAAAADPCNAAAA|54002|15|0|2|PM|second|afternoon|| +54003|AAAAAAAAEPCNAAAA|54003|15|0|3|PM|second|afternoon|| +54004|AAAAAAAAFPCNAAAA|54004|15|0|4|PM|second|afternoon|| +54005|AAAAAAAAGPCNAAAA|54005|15|0|5|PM|second|afternoon|| +54006|AAAAAAAAHPCNAAAA|54006|15|0|6|PM|second|afternoon|| +54007|AAAAAAAAIPCNAAAA|54007|15|0|7|PM|second|afternoon|| +54008|AAAAAAAAJPCNAAAA|54008|15|0|8|PM|second|afternoon|| +54009|AAAAAAAAKPCNAAAA|54009|15|0|9|PM|second|afternoon|| +54010|AAAAAAAALPCNAAAA|54010|15|0|10|PM|second|afternoon|| +54011|AAAAAAAAMPCNAAAA|54011|15|0|11|PM|second|afternoon|| +54012|AAAAAAAANPCNAAAA|54012|15|0|12|PM|second|afternoon|| +54013|AAAAAAAAOPCNAAAA|54013|15|0|13|PM|second|afternoon|| +54014|AAAAAAAAPPCNAAAA|54014|15|0|14|PM|second|afternoon|| +54015|AAAAAAAAAADNAAAA|54015|15|0|15|PM|second|afternoon|| +54016|AAAAAAAABADNAAAA|54016|15|0|16|PM|second|afternoon|| +54017|AAAAAAAACADNAAAA|54017|15|0|17|PM|second|afternoon|| +54018|AAAAAAAADADNAAAA|54018|15|0|18|PM|second|afternoon|| +54019|AAAAAAAAEADNAAAA|54019|15|0|19|PM|second|afternoon|| +54020|AAAAAAAAFADNAAAA|54020|15|0|20|PM|second|afternoon|| +54021|AAAAAAAAGADNAAAA|54021|15|0|21|PM|second|afternoon|| +54022|AAAAAAAAHADNAAAA|54022|15|0|22|PM|second|afternoon|| +54023|AAAAAAAAIADNAAAA|54023|15|0|23|PM|second|afternoon|| +54024|AAAAAAAAJADNAAAA|54024|15|0|24|PM|second|afternoon|| +54025|AAAAAAAAKADNAAAA|54025|15|0|25|PM|second|afternoon|| +54026|AAAAAAAALADNAAAA|54026|15|0|26|PM|second|afternoon|| +54027|AAAAAAAAMADNAAAA|54027|15|0|27|PM|second|afternoon|| +54028|AAAAAAAANADNAAAA|54028|15|0|28|PM|second|afternoon|| +54029|AAAAAAAAOADNAAAA|54029|15|0|29|PM|second|afternoon|| +54030|AAAAAAAAPADNAAAA|54030|15|0|30|PM|second|afternoon|| +54031|AAAAAAAAABDNAAAA|54031|15|0|31|PM|second|afternoon|| +54032|AAAAAAAABBDNAAAA|54032|15|0|32|PM|second|afternoon|| +54033|AAAAAAAACBDNAAAA|54033|15|0|33|PM|second|afternoon|| +54034|AAAAAAAADBDNAAAA|54034|15|0|34|PM|second|afternoon|| +54035|AAAAAAAAEBDNAAAA|54035|15|0|35|PM|second|afternoon|| +54036|AAAAAAAAFBDNAAAA|54036|15|0|36|PM|second|afternoon|| +54037|AAAAAAAAGBDNAAAA|54037|15|0|37|PM|second|afternoon|| +54038|AAAAAAAAHBDNAAAA|54038|15|0|38|PM|second|afternoon|| +54039|AAAAAAAAIBDNAAAA|54039|15|0|39|PM|second|afternoon|| +54040|AAAAAAAAJBDNAAAA|54040|15|0|40|PM|second|afternoon|| +54041|AAAAAAAAKBDNAAAA|54041|15|0|41|PM|second|afternoon|| +54042|AAAAAAAALBDNAAAA|54042|15|0|42|PM|second|afternoon|| +54043|AAAAAAAAMBDNAAAA|54043|15|0|43|PM|second|afternoon|| +54044|AAAAAAAANBDNAAAA|54044|15|0|44|PM|second|afternoon|| +54045|AAAAAAAAOBDNAAAA|54045|15|0|45|PM|second|afternoon|| +54046|AAAAAAAAPBDNAAAA|54046|15|0|46|PM|second|afternoon|| +54047|AAAAAAAAACDNAAAA|54047|15|0|47|PM|second|afternoon|| +54048|AAAAAAAABCDNAAAA|54048|15|0|48|PM|second|afternoon|| +54049|AAAAAAAACCDNAAAA|54049|15|0|49|PM|second|afternoon|| +54050|AAAAAAAADCDNAAAA|54050|15|0|50|PM|second|afternoon|| +54051|AAAAAAAAECDNAAAA|54051|15|0|51|PM|second|afternoon|| +54052|AAAAAAAAFCDNAAAA|54052|15|0|52|PM|second|afternoon|| +54053|AAAAAAAAGCDNAAAA|54053|15|0|53|PM|second|afternoon|| +54054|AAAAAAAAHCDNAAAA|54054|15|0|54|PM|second|afternoon|| +54055|AAAAAAAAICDNAAAA|54055|15|0|55|PM|second|afternoon|| +54056|AAAAAAAAJCDNAAAA|54056|15|0|56|PM|second|afternoon|| +54057|AAAAAAAAKCDNAAAA|54057|15|0|57|PM|second|afternoon|| +54058|AAAAAAAALCDNAAAA|54058|15|0|58|PM|second|afternoon|| +54059|AAAAAAAAMCDNAAAA|54059|15|0|59|PM|second|afternoon|| +54060|AAAAAAAANCDNAAAA|54060|15|1|0|PM|second|afternoon|| +54061|AAAAAAAAOCDNAAAA|54061|15|1|1|PM|second|afternoon|| +54062|AAAAAAAAPCDNAAAA|54062|15|1|2|PM|second|afternoon|| +54063|AAAAAAAAADDNAAAA|54063|15|1|3|PM|second|afternoon|| +54064|AAAAAAAABDDNAAAA|54064|15|1|4|PM|second|afternoon|| +54065|AAAAAAAACDDNAAAA|54065|15|1|5|PM|second|afternoon|| +54066|AAAAAAAADDDNAAAA|54066|15|1|6|PM|second|afternoon|| +54067|AAAAAAAAEDDNAAAA|54067|15|1|7|PM|second|afternoon|| +54068|AAAAAAAAFDDNAAAA|54068|15|1|8|PM|second|afternoon|| +54069|AAAAAAAAGDDNAAAA|54069|15|1|9|PM|second|afternoon|| +54070|AAAAAAAAHDDNAAAA|54070|15|1|10|PM|second|afternoon|| +54071|AAAAAAAAIDDNAAAA|54071|15|1|11|PM|second|afternoon|| +54072|AAAAAAAAJDDNAAAA|54072|15|1|12|PM|second|afternoon|| +54073|AAAAAAAAKDDNAAAA|54073|15|1|13|PM|second|afternoon|| +54074|AAAAAAAALDDNAAAA|54074|15|1|14|PM|second|afternoon|| +54075|AAAAAAAAMDDNAAAA|54075|15|1|15|PM|second|afternoon|| +54076|AAAAAAAANDDNAAAA|54076|15|1|16|PM|second|afternoon|| +54077|AAAAAAAAODDNAAAA|54077|15|1|17|PM|second|afternoon|| +54078|AAAAAAAAPDDNAAAA|54078|15|1|18|PM|second|afternoon|| +54079|AAAAAAAAAEDNAAAA|54079|15|1|19|PM|second|afternoon|| +54080|AAAAAAAABEDNAAAA|54080|15|1|20|PM|second|afternoon|| +54081|AAAAAAAACEDNAAAA|54081|15|1|21|PM|second|afternoon|| +54082|AAAAAAAADEDNAAAA|54082|15|1|22|PM|second|afternoon|| +54083|AAAAAAAAEEDNAAAA|54083|15|1|23|PM|second|afternoon|| +54084|AAAAAAAAFEDNAAAA|54084|15|1|24|PM|second|afternoon|| +54085|AAAAAAAAGEDNAAAA|54085|15|1|25|PM|second|afternoon|| +54086|AAAAAAAAHEDNAAAA|54086|15|1|26|PM|second|afternoon|| +54087|AAAAAAAAIEDNAAAA|54087|15|1|27|PM|second|afternoon|| +54088|AAAAAAAAJEDNAAAA|54088|15|1|28|PM|second|afternoon|| +54089|AAAAAAAAKEDNAAAA|54089|15|1|29|PM|second|afternoon|| +54090|AAAAAAAALEDNAAAA|54090|15|1|30|PM|second|afternoon|| +54091|AAAAAAAAMEDNAAAA|54091|15|1|31|PM|second|afternoon|| +54092|AAAAAAAANEDNAAAA|54092|15|1|32|PM|second|afternoon|| +54093|AAAAAAAAOEDNAAAA|54093|15|1|33|PM|second|afternoon|| +54094|AAAAAAAAPEDNAAAA|54094|15|1|34|PM|second|afternoon|| +54095|AAAAAAAAAFDNAAAA|54095|15|1|35|PM|second|afternoon|| +54096|AAAAAAAABFDNAAAA|54096|15|1|36|PM|second|afternoon|| +54097|AAAAAAAACFDNAAAA|54097|15|1|37|PM|second|afternoon|| +54098|AAAAAAAADFDNAAAA|54098|15|1|38|PM|second|afternoon|| +54099|AAAAAAAAEFDNAAAA|54099|15|1|39|PM|second|afternoon|| +54100|AAAAAAAAFFDNAAAA|54100|15|1|40|PM|second|afternoon|| +54101|AAAAAAAAGFDNAAAA|54101|15|1|41|PM|second|afternoon|| +54102|AAAAAAAAHFDNAAAA|54102|15|1|42|PM|second|afternoon|| +54103|AAAAAAAAIFDNAAAA|54103|15|1|43|PM|second|afternoon|| +54104|AAAAAAAAJFDNAAAA|54104|15|1|44|PM|second|afternoon|| +54105|AAAAAAAAKFDNAAAA|54105|15|1|45|PM|second|afternoon|| +54106|AAAAAAAALFDNAAAA|54106|15|1|46|PM|second|afternoon|| +54107|AAAAAAAAMFDNAAAA|54107|15|1|47|PM|second|afternoon|| +54108|AAAAAAAANFDNAAAA|54108|15|1|48|PM|second|afternoon|| +54109|AAAAAAAAOFDNAAAA|54109|15|1|49|PM|second|afternoon|| +54110|AAAAAAAAPFDNAAAA|54110|15|1|50|PM|second|afternoon|| +54111|AAAAAAAAAGDNAAAA|54111|15|1|51|PM|second|afternoon|| +54112|AAAAAAAABGDNAAAA|54112|15|1|52|PM|second|afternoon|| +54113|AAAAAAAACGDNAAAA|54113|15|1|53|PM|second|afternoon|| +54114|AAAAAAAADGDNAAAA|54114|15|1|54|PM|second|afternoon|| +54115|AAAAAAAAEGDNAAAA|54115|15|1|55|PM|second|afternoon|| +54116|AAAAAAAAFGDNAAAA|54116|15|1|56|PM|second|afternoon|| +54117|AAAAAAAAGGDNAAAA|54117|15|1|57|PM|second|afternoon|| +54118|AAAAAAAAHGDNAAAA|54118|15|1|58|PM|second|afternoon|| +54119|AAAAAAAAIGDNAAAA|54119|15|1|59|PM|second|afternoon|| +54120|AAAAAAAAJGDNAAAA|54120|15|2|0|PM|second|afternoon|| +54121|AAAAAAAAKGDNAAAA|54121|15|2|1|PM|second|afternoon|| +54122|AAAAAAAALGDNAAAA|54122|15|2|2|PM|second|afternoon|| +54123|AAAAAAAAMGDNAAAA|54123|15|2|3|PM|second|afternoon|| +54124|AAAAAAAANGDNAAAA|54124|15|2|4|PM|second|afternoon|| +54125|AAAAAAAAOGDNAAAA|54125|15|2|5|PM|second|afternoon|| +54126|AAAAAAAAPGDNAAAA|54126|15|2|6|PM|second|afternoon|| +54127|AAAAAAAAAHDNAAAA|54127|15|2|7|PM|second|afternoon|| +54128|AAAAAAAABHDNAAAA|54128|15|2|8|PM|second|afternoon|| +54129|AAAAAAAACHDNAAAA|54129|15|2|9|PM|second|afternoon|| +54130|AAAAAAAADHDNAAAA|54130|15|2|10|PM|second|afternoon|| +54131|AAAAAAAAEHDNAAAA|54131|15|2|11|PM|second|afternoon|| +54132|AAAAAAAAFHDNAAAA|54132|15|2|12|PM|second|afternoon|| +54133|AAAAAAAAGHDNAAAA|54133|15|2|13|PM|second|afternoon|| +54134|AAAAAAAAHHDNAAAA|54134|15|2|14|PM|second|afternoon|| +54135|AAAAAAAAIHDNAAAA|54135|15|2|15|PM|second|afternoon|| +54136|AAAAAAAAJHDNAAAA|54136|15|2|16|PM|second|afternoon|| +54137|AAAAAAAAKHDNAAAA|54137|15|2|17|PM|second|afternoon|| +54138|AAAAAAAALHDNAAAA|54138|15|2|18|PM|second|afternoon|| +54139|AAAAAAAAMHDNAAAA|54139|15|2|19|PM|second|afternoon|| +54140|AAAAAAAANHDNAAAA|54140|15|2|20|PM|second|afternoon|| +54141|AAAAAAAAOHDNAAAA|54141|15|2|21|PM|second|afternoon|| +54142|AAAAAAAAPHDNAAAA|54142|15|2|22|PM|second|afternoon|| +54143|AAAAAAAAAIDNAAAA|54143|15|2|23|PM|second|afternoon|| +54144|AAAAAAAABIDNAAAA|54144|15|2|24|PM|second|afternoon|| +54145|AAAAAAAACIDNAAAA|54145|15|2|25|PM|second|afternoon|| +54146|AAAAAAAADIDNAAAA|54146|15|2|26|PM|second|afternoon|| +54147|AAAAAAAAEIDNAAAA|54147|15|2|27|PM|second|afternoon|| +54148|AAAAAAAAFIDNAAAA|54148|15|2|28|PM|second|afternoon|| +54149|AAAAAAAAGIDNAAAA|54149|15|2|29|PM|second|afternoon|| +54150|AAAAAAAAHIDNAAAA|54150|15|2|30|PM|second|afternoon|| +54151|AAAAAAAAIIDNAAAA|54151|15|2|31|PM|second|afternoon|| +54152|AAAAAAAAJIDNAAAA|54152|15|2|32|PM|second|afternoon|| +54153|AAAAAAAAKIDNAAAA|54153|15|2|33|PM|second|afternoon|| +54154|AAAAAAAALIDNAAAA|54154|15|2|34|PM|second|afternoon|| +54155|AAAAAAAAMIDNAAAA|54155|15|2|35|PM|second|afternoon|| +54156|AAAAAAAANIDNAAAA|54156|15|2|36|PM|second|afternoon|| +54157|AAAAAAAAOIDNAAAA|54157|15|2|37|PM|second|afternoon|| +54158|AAAAAAAAPIDNAAAA|54158|15|2|38|PM|second|afternoon|| +54159|AAAAAAAAAJDNAAAA|54159|15|2|39|PM|second|afternoon|| +54160|AAAAAAAABJDNAAAA|54160|15|2|40|PM|second|afternoon|| +54161|AAAAAAAACJDNAAAA|54161|15|2|41|PM|second|afternoon|| +54162|AAAAAAAADJDNAAAA|54162|15|2|42|PM|second|afternoon|| +54163|AAAAAAAAEJDNAAAA|54163|15|2|43|PM|second|afternoon|| +54164|AAAAAAAAFJDNAAAA|54164|15|2|44|PM|second|afternoon|| +54165|AAAAAAAAGJDNAAAA|54165|15|2|45|PM|second|afternoon|| +54166|AAAAAAAAHJDNAAAA|54166|15|2|46|PM|second|afternoon|| +54167|AAAAAAAAIJDNAAAA|54167|15|2|47|PM|second|afternoon|| +54168|AAAAAAAAJJDNAAAA|54168|15|2|48|PM|second|afternoon|| +54169|AAAAAAAAKJDNAAAA|54169|15|2|49|PM|second|afternoon|| +54170|AAAAAAAALJDNAAAA|54170|15|2|50|PM|second|afternoon|| +54171|AAAAAAAAMJDNAAAA|54171|15|2|51|PM|second|afternoon|| +54172|AAAAAAAANJDNAAAA|54172|15|2|52|PM|second|afternoon|| +54173|AAAAAAAAOJDNAAAA|54173|15|2|53|PM|second|afternoon|| +54174|AAAAAAAAPJDNAAAA|54174|15|2|54|PM|second|afternoon|| +54175|AAAAAAAAAKDNAAAA|54175|15|2|55|PM|second|afternoon|| +54176|AAAAAAAABKDNAAAA|54176|15|2|56|PM|second|afternoon|| +54177|AAAAAAAACKDNAAAA|54177|15|2|57|PM|second|afternoon|| +54178|AAAAAAAADKDNAAAA|54178|15|2|58|PM|second|afternoon|| +54179|AAAAAAAAEKDNAAAA|54179|15|2|59|PM|second|afternoon|| +54180|AAAAAAAAFKDNAAAA|54180|15|3|0|PM|second|afternoon|| +54181|AAAAAAAAGKDNAAAA|54181|15|3|1|PM|second|afternoon|| +54182|AAAAAAAAHKDNAAAA|54182|15|3|2|PM|second|afternoon|| +54183|AAAAAAAAIKDNAAAA|54183|15|3|3|PM|second|afternoon|| +54184|AAAAAAAAJKDNAAAA|54184|15|3|4|PM|second|afternoon|| +54185|AAAAAAAAKKDNAAAA|54185|15|3|5|PM|second|afternoon|| +54186|AAAAAAAALKDNAAAA|54186|15|3|6|PM|second|afternoon|| +54187|AAAAAAAAMKDNAAAA|54187|15|3|7|PM|second|afternoon|| +54188|AAAAAAAANKDNAAAA|54188|15|3|8|PM|second|afternoon|| +54189|AAAAAAAAOKDNAAAA|54189|15|3|9|PM|second|afternoon|| +54190|AAAAAAAAPKDNAAAA|54190|15|3|10|PM|second|afternoon|| +54191|AAAAAAAAALDNAAAA|54191|15|3|11|PM|second|afternoon|| +54192|AAAAAAAABLDNAAAA|54192|15|3|12|PM|second|afternoon|| +54193|AAAAAAAACLDNAAAA|54193|15|3|13|PM|second|afternoon|| +54194|AAAAAAAADLDNAAAA|54194|15|3|14|PM|second|afternoon|| +54195|AAAAAAAAELDNAAAA|54195|15|3|15|PM|second|afternoon|| +54196|AAAAAAAAFLDNAAAA|54196|15|3|16|PM|second|afternoon|| +54197|AAAAAAAAGLDNAAAA|54197|15|3|17|PM|second|afternoon|| +54198|AAAAAAAAHLDNAAAA|54198|15|3|18|PM|second|afternoon|| +54199|AAAAAAAAILDNAAAA|54199|15|3|19|PM|second|afternoon|| +54200|AAAAAAAAJLDNAAAA|54200|15|3|20|PM|second|afternoon|| +54201|AAAAAAAAKLDNAAAA|54201|15|3|21|PM|second|afternoon|| +54202|AAAAAAAALLDNAAAA|54202|15|3|22|PM|second|afternoon|| +54203|AAAAAAAAMLDNAAAA|54203|15|3|23|PM|second|afternoon|| +54204|AAAAAAAANLDNAAAA|54204|15|3|24|PM|second|afternoon|| +54205|AAAAAAAAOLDNAAAA|54205|15|3|25|PM|second|afternoon|| +54206|AAAAAAAAPLDNAAAA|54206|15|3|26|PM|second|afternoon|| +54207|AAAAAAAAAMDNAAAA|54207|15|3|27|PM|second|afternoon|| +54208|AAAAAAAABMDNAAAA|54208|15|3|28|PM|second|afternoon|| +54209|AAAAAAAACMDNAAAA|54209|15|3|29|PM|second|afternoon|| +54210|AAAAAAAADMDNAAAA|54210|15|3|30|PM|second|afternoon|| +54211|AAAAAAAAEMDNAAAA|54211|15|3|31|PM|second|afternoon|| +54212|AAAAAAAAFMDNAAAA|54212|15|3|32|PM|second|afternoon|| +54213|AAAAAAAAGMDNAAAA|54213|15|3|33|PM|second|afternoon|| +54214|AAAAAAAAHMDNAAAA|54214|15|3|34|PM|second|afternoon|| +54215|AAAAAAAAIMDNAAAA|54215|15|3|35|PM|second|afternoon|| +54216|AAAAAAAAJMDNAAAA|54216|15|3|36|PM|second|afternoon|| +54217|AAAAAAAAKMDNAAAA|54217|15|3|37|PM|second|afternoon|| +54218|AAAAAAAALMDNAAAA|54218|15|3|38|PM|second|afternoon|| +54219|AAAAAAAAMMDNAAAA|54219|15|3|39|PM|second|afternoon|| +54220|AAAAAAAANMDNAAAA|54220|15|3|40|PM|second|afternoon|| +54221|AAAAAAAAOMDNAAAA|54221|15|3|41|PM|second|afternoon|| +54222|AAAAAAAAPMDNAAAA|54222|15|3|42|PM|second|afternoon|| +54223|AAAAAAAAANDNAAAA|54223|15|3|43|PM|second|afternoon|| +54224|AAAAAAAABNDNAAAA|54224|15|3|44|PM|second|afternoon|| +54225|AAAAAAAACNDNAAAA|54225|15|3|45|PM|second|afternoon|| +54226|AAAAAAAADNDNAAAA|54226|15|3|46|PM|second|afternoon|| +54227|AAAAAAAAENDNAAAA|54227|15|3|47|PM|second|afternoon|| +54228|AAAAAAAAFNDNAAAA|54228|15|3|48|PM|second|afternoon|| +54229|AAAAAAAAGNDNAAAA|54229|15|3|49|PM|second|afternoon|| +54230|AAAAAAAAHNDNAAAA|54230|15|3|50|PM|second|afternoon|| +54231|AAAAAAAAINDNAAAA|54231|15|3|51|PM|second|afternoon|| +54232|AAAAAAAAJNDNAAAA|54232|15|3|52|PM|second|afternoon|| +54233|AAAAAAAAKNDNAAAA|54233|15|3|53|PM|second|afternoon|| +54234|AAAAAAAALNDNAAAA|54234|15|3|54|PM|second|afternoon|| +54235|AAAAAAAAMNDNAAAA|54235|15|3|55|PM|second|afternoon|| +54236|AAAAAAAANNDNAAAA|54236|15|3|56|PM|second|afternoon|| +54237|AAAAAAAAONDNAAAA|54237|15|3|57|PM|second|afternoon|| +54238|AAAAAAAAPNDNAAAA|54238|15|3|58|PM|second|afternoon|| +54239|AAAAAAAAAODNAAAA|54239|15|3|59|PM|second|afternoon|| +54240|AAAAAAAABODNAAAA|54240|15|4|0|PM|second|afternoon|| +54241|AAAAAAAACODNAAAA|54241|15|4|1|PM|second|afternoon|| +54242|AAAAAAAADODNAAAA|54242|15|4|2|PM|second|afternoon|| +54243|AAAAAAAAEODNAAAA|54243|15|4|3|PM|second|afternoon|| +54244|AAAAAAAAFODNAAAA|54244|15|4|4|PM|second|afternoon|| +54245|AAAAAAAAGODNAAAA|54245|15|4|5|PM|second|afternoon|| +54246|AAAAAAAAHODNAAAA|54246|15|4|6|PM|second|afternoon|| +54247|AAAAAAAAIODNAAAA|54247|15|4|7|PM|second|afternoon|| +54248|AAAAAAAAJODNAAAA|54248|15|4|8|PM|second|afternoon|| +54249|AAAAAAAAKODNAAAA|54249|15|4|9|PM|second|afternoon|| +54250|AAAAAAAALODNAAAA|54250|15|4|10|PM|second|afternoon|| +54251|AAAAAAAAMODNAAAA|54251|15|4|11|PM|second|afternoon|| +54252|AAAAAAAANODNAAAA|54252|15|4|12|PM|second|afternoon|| +54253|AAAAAAAAOODNAAAA|54253|15|4|13|PM|second|afternoon|| +54254|AAAAAAAAPODNAAAA|54254|15|4|14|PM|second|afternoon|| +54255|AAAAAAAAAPDNAAAA|54255|15|4|15|PM|second|afternoon|| +54256|AAAAAAAABPDNAAAA|54256|15|4|16|PM|second|afternoon|| +54257|AAAAAAAACPDNAAAA|54257|15|4|17|PM|second|afternoon|| +54258|AAAAAAAADPDNAAAA|54258|15|4|18|PM|second|afternoon|| +54259|AAAAAAAAEPDNAAAA|54259|15|4|19|PM|second|afternoon|| +54260|AAAAAAAAFPDNAAAA|54260|15|4|20|PM|second|afternoon|| +54261|AAAAAAAAGPDNAAAA|54261|15|4|21|PM|second|afternoon|| +54262|AAAAAAAAHPDNAAAA|54262|15|4|22|PM|second|afternoon|| +54263|AAAAAAAAIPDNAAAA|54263|15|4|23|PM|second|afternoon|| +54264|AAAAAAAAJPDNAAAA|54264|15|4|24|PM|second|afternoon|| +54265|AAAAAAAAKPDNAAAA|54265|15|4|25|PM|second|afternoon|| +54266|AAAAAAAALPDNAAAA|54266|15|4|26|PM|second|afternoon|| +54267|AAAAAAAAMPDNAAAA|54267|15|4|27|PM|second|afternoon|| +54268|AAAAAAAANPDNAAAA|54268|15|4|28|PM|second|afternoon|| +54269|AAAAAAAAOPDNAAAA|54269|15|4|29|PM|second|afternoon|| +54270|AAAAAAAAPPDNAAAA|54270|15|4|30|PM|second|afternoon|| +54271|AAAAAAAAAAENAAAA|54271|15|4|31|PM|second|afternoon|| +54272|AAAAAAAABAENAAAA|54272|15|4|32|PM|second|afternoon|| +54273|AAAAAAAACAENAAAA|54273|15|4|33|PM|second|afternoon|| +54274|AAAAAAAADAENAAAA|54274|15|4|34|PM|second|afternoon|| +54275|AAAAAAAAEAENAAAA|54275|15|4|35|PM|second|afternoon|| +54276|AAAAAAAAFAENAAAA|54276|15|4|36|PM|second|afternoon|| +54277|AAAAAAAAGAENAAAA|54277|15|4|37|PM|second|afternoon|| +54278|AAAAAAAAHAENAAAA|54278|15|4|38|PM|second|afternoon|| +54279|AAAAAAAAIAENAAAA|54279|15|4|39|PM|second|afternoon|| +54280|AAAAAAAAJAENAAAA|54280|15|4|40|PM|second|afternoon|| +54281|AAAAAAAAKAENAAAA|54281|15|4|41|PM|second|afternoon|| +54282|AAAAAAAALAENAAAA|54282|15|4|42|PM|second|afternoon|| +54283|AAAAAAAAMAENAAAA|54283|15|4|43|PM|second|afternoon|| +54284|AAAAAAAANAENAAAA|54284|15|4|44|PM|second|afternoon|| +54285|AAAAAAAAOAENAAAA|54285|15|4|45|PM|second|afternoon|| +54286|AAAAAAAAPAENAAAA|54286|15|4|46|PM|second|afternoon|| +54287|AAAAAAAAABENAAAA|54287|15|4|47|PM|second|afternoon|| +54288|AAAAAAAABBENAAAA|54288|15|4|48|PM|second|afternoon|| +54289|AAAAAAAACBENAAAA|54289|15|4|49|PM|second|afternoon|| +54290|AAAAAAAADBENAAAA|54290|15|4|50|PM|second|afternoon|| +54291|AAAAAAAAEBENAAAA|54291|15|4|51|PM|second|afternoon|| +54292|AAAAAAAAFBENAAAA|54292|15|4|52|PM|second|afternoon|| +54293|AAAAAAAAGBENAAAA|54293|15|4|53|PM|second|afternoon|| +54294|AAAAAAAAHBENAAAA|54294|15|4|54|PM|second|afternoon|| +54295|AAAAAAAAIBENAAAA|54295|15|4|55|PM|second|afternoon|| +54296|AAAAAAAAJBENAAAA|54296|15|4|56|PM|second|afternoon|| +54297|AAAAAAAAKBENAAAA|54297|15|4|57|PM|second|afternoon|| +54298|AAAAAAAALBENAAAA|54298|15|4|58|PM|second|afternoon|| +54299|AAAAAAAAMBENAAAA|54299|15|4|59|PM|second|afternoon|| +54300|AAAAAAAANBENAAAA|54300|15|5|0|PM|second|afternoon|| +54301|AAAAAAAAOBENAAAA|54301|15|5|1|PM|second|afternoon|| +54302|AAAAAAAAPBENAAAA|54302|15|5|2|PM|second|afternoon|| +54303|AAAAAAAAACENAAAA|54303|15|5|3|PM|second|afternoon|| +54304|AAAAAAAABCENAAAA|54304|15|5|4|PM|second|afternoon|| +54305|AAAAAAAACCENAAAA|54305|15|5|5|PM|second|afternoon|| +54306|AAAAAAAADCENAAAA|54306|15|5|6|PM|second|afternoon|| +54307|AAAAAAAAECENAAAA|54307|15|5|7|PM|second|afternoon|| +54308|AAAAAAAAFCENAAAA|54308|15|5|8|PM|second|afternoon|| +54309|AAAAAAAAGCENAAAA|54309|15|5|9|PM|second|afternoon|| +54310|AAAAAAAAHCENAAAA|54310|15|5|10|PM|second|afternoon|| +54311|AAAAAAAAICENAAAA|54311|15|5|11|PM|second|afternoon|| +54312|AAAAAAAAJCENAAAA|54312|15|5|12|PM|second|afternoon|| +54313|AAAAAAAAKCENAAAA|54313|15|5|13|PM|second|afternoon|| +54314|AAAAAAAALCENAAAA|54314|15|5|14|PM|second|afternoon|| +54315|AAAAAAAAMCENAAAA|54315|15|5|15|PM|second|afternoon|| +54316|AAAAAAAANCENAAAA|54316|15|5|16|PM|second|afternoon|| +54317|AAAAAAAAOCENAAAA|54317|15|5|17|PM|second|afternoon|| +54318|AAAAAAAAPCENAAAA|54318|15|5|18|PM|second|afternoon|| +54319|AAAAAAAAADENAAAA|54319|15|5|19|PM|second|afternoon|| +54320|AAAAAAAABDENAAAA|54320|15|5|20|PM|second|afternoon|| +54321|AAAAAAAACDENAAAA|54321|15|5|21|PM|second|afternoon|| +54322|AAAAAAAADDENAAAA|54322|15|5|22|PM|second|afternoon|| +54323|AAAAAAAAEDENAAAA|54323|15|5|23|PM|second|afternoon|| +54324|AAAAAAAAFDENAAAA|54324|15|5|24|PM|second|afternoon|| +54325|AAAAAAAAGDENAAAA|54325|15|5|25|PM|second|afternoon|| +54326|AAAAAAAAHDENAAAA|54326|15|5|26|PM|second|afternoon|| +54327|AAAAAAAAIDENAAAA|54327|15|5|27|PM|second|afternoon|| +54328|AAAAAAAAJDENAAAA|54328|15|5|28|PM|second|afternoon|| +54329|AAAAAAAAKDENAAAA|54329|15|5|29|PM|second|afternoon|| +54330|AAAAAAAALDENAAAA|54330|15|5|30|PM|second|afternoon|| +54331|AAAAAAAAMDENAAAA|54331|15|5|31|PM|second|afternoon|| +54332|AAAAAAAANDENAAAA|54332|15|5|32|PM|second|afternoon|| +54333|AAAAAAAAODENAAAA|54333|15|5|33|PM|second|afternoon|| +54334|AAAAAAAAPDENAAAA|54334|15|5|34|PM|second|afternoon|| +54335|AAAAAAAAAEENAAAA|54335|15|5|35|PM|second|afternoon|| +54336|AAAAAAAABEENAAAA|54336|15|5|36|PM|second|afternoon|| +54337|AAAAAAAACEENAAAA|54337|15|5|37|PM|second|afternoon|| +54338|AAAAAAAADEENAAAA|54338|15|5|38|PM|second|afternoon|| +54339|AAAAAAAAEEENAAAA|54339|15|5|39|PM|second|afternoon|| +54340|AAAAAAAAFEENAAAA|54340|15|5|40|PM|second|afternoon|| +54341|AAAAAAAAGEENAAAA|54341|15|5|41|PM|second|afternoon|| +54342|AAAAAAAAHEENAAAA|54342|15|5|42|PM|second|afternoon|| +54343|AAAAAAAAIEENAAAA|54343|15|5|43|PM|second|afternoon|| +54344|AAAAAAAAJEENAAAA|54344|15|5|44|PM|second|afternoon|| +54345|AAAAAAAAKEENAAAA|54345|15|5|45|PM|second|afternoon|| +54346|AAAAAAAALEENAAAA|54346|15|5|46|PM|second|afternoon|| +54347|AAAAAAAAMEENAAAA|54347|15|5|47|PM|second|afternoon|| +54348|AAAAAAAANEENAAAA|54348|15|5|48|PM|second|afternoon|| +54349|AAAAAAAAOEENAAAA|54349|15|5|49|PM|second|afternoon|| +54350|AAAAAAAAPEENAAAA|54350|15|5|50|PM|second|afternoon|| +54351|AAAAAAAAAFENAAAA|54351|15|5|51|PM|second|afternoon|| +54352|AAAAAAAABFENAAAA|54352|15|5|52|PM|second|afternoon|| +54353|AAAAAAAACFENAAAA|54353|15|5|53|PM|second|afternoon|| +54354|AAAAAAAADFENAAAA|54354|15|5|54|PM|second|afternoon|| +54355|AAAAAAAAEFENAAAA|54355|15|5|55|PM|second|afternoon|| +54356|AAAAAAAAFFENAAAA|54356|15|5|56|PM|second|afternoon|| +54357|AAAAAAAAGFENAAAA|54357|15|5|57|PM|second|afternoon|| +54358|AAAAAAAAHFENAAAA|54358|15|5|58|PM|second|afternoon|| +54359|AAAAAAAAIFENAAAA|54359|15|5|59|PM|second|afternoon|| +54360|AAAAAAAAJFENAAAA|54360|15|6|0|PM|second|afternoon|| +54361|AAAAAAAAKFENAAAA|54361|15|6|1|PM|second|afternoon|| +54362|AAAAAAAALFENAAAA|54362|15|6|2|PM|second|afternoon|| +54363|AAAAAAAAMFENAAAA|54363|15|6|3|PM|second|afternoon|| +54364|AAAAAAAANFENAAAA|54364|15|6|4|PM|second|afternoon|| +54365|AAAAAAAAOFENAAAA|54365|15|6|5|PM|second|afternoon|| +54366|AAAAAAAAPFENAAAA|54366|15|6|6|PM|second|afternoon|| +54367|AAAAAAAAAGENAAAA|54367|15|6|7|PM|second|afternoon|| +54368|AAAAAAAABGENAAAA|54368|15|6|8|PM|second|afternoon|| +54369|AAAAAAAACGENAAAA|54369|15|6|9|PM|second|afternoon|| +54370|AAAAAAAADGENAAAA|54370|15|6|10|PM|second|afternoon|| +54371|AAAAAAAAEGENAAAA|54371|15|6|11|PM|second|afternoon|| +54372|AAAAAAAAFGENAAAA|54372|15|6|12|PM|second|afternoon|| +54373|AAAAAAAAGGENAAAA|54373|15|6|13|PM|second|afternoon|| +54374|AAAAAAAAHGENAAAA|54374|15|6|14|PM|second|afternoon|| +54375|AAAAAAAAIGENAAAA|54375|15|6|15|PM|second|afternoon|| +54376|AAAAAAAAJGENAAAA|54376|15|6|16|PM|second|afternoon|| +54377|AAAAAAAAKGENAAAA|54377|15|6|17|PM|second|afternoon|| +54378|AAAAAAAALGENAAAA|54378|15|6|18|PM|second|afternoon|| +54379|AAAAAAAAMGENAAAA|54379|15|6|19|PM|second|afternoon|| +54380|AAAAAAAANGENAAAA|54380|15|6|20|PM|second|afternoon|| +54381|AAAAAAAAOGENAAAA|54381|15|6|21|PM|second|afternoon|| +54382|AAAAAAAAPGENAAAA|54382|15|6|22|PM|second|afternoon|| +54383|AAAAAAAAAHENAAAA|54383|15|6|23|PM|second|afternoon|| +54384|AAAAAAAABHENAAAA|54384|15|6|24|PM|second|afternoon|| +54385|AAAAAAAACHENAAAA|54385|15|6|25|PM|second|afternoon|| +54386|AAAAAAAADHENAAAA|54386|15|6|26|PM|second|afternoon|| +54387|AAAAAAAAEHENAAAA|54387|15|6|27|PM|second|afternoon|| +54388|AAAAAAAAFHENAAAA|54388|15|6|28|PM|second|afternoon|| +54389|AAAAAAAAGHENAAAA|54389|15|6|29|PM|second|afternoon|| +54390|AAAAAAAAHHENAAAA|54390|15|6|30|PM|second|afternoon|| +54391|AAAAAAAAIHENAAAA|54391|15|6|31|PM|second|afternoon|| +54392|AAAAAAAAJHENAAAA|54392|15|6|32|PM|second|afternoon|| +54393|AAAAAAAAKHENAAAA|54393|15|6|33|PM|second|afternoon|| +54394|AAAAAAAALHENAAAA|54394|15|6|34|PM|second|afternoon|| +54395|AAAAAAAAMHENAAAA|54395|15|6|35|PM|second|afternoon|| +54396|AAAAAAAANHENAAAA|54396|15|6|36|PM|second|afternoon|| +54397|AAAAAAAAOHENAAAA|54397|15|6|37|PM|second|afternoon|| +54398|AAAAAAAAPHENAAAA|54398|15|6|38|PM|second|afternoon|| +54399|AAAAAAAAAIENAAAA|54399|15|6|39|PM|second|afternoon|| +54400|AAAAAAAABIENAAAA|54400|15|6|40|PM|second|afternoon|| +54401|AAAAAAAACIENAAAA|54401|15|6|41|PM|second|afternoon|| +54402|AAAAAAAADIENAAAA|54402|15|6|42|PM|second|afternoon|| +54403|AAAAAAAAEIENAAAA|54403|15|6|43|PM|second|afternoon|| +54404|AAAAAAAAFIENAAAA|54404|15|6|44|PM|second|afternoon|| +54405|AAAAAAAAGIENAAAA|54405|15|6|45|PM|second|afternoon|| +54406|AAAAAAAAHIENAAAA|54406|15|6|46|PM|second|afternoon|| +54407|AAAAAAAAIIENAAAA|54407|15|6|47|PM|second|afternoon|| +54408|AAAAAAAAJIENAAAA|54408|15|6|48|PM|second|afternoon|| +54409|AAAAAAAAKIENAAAA|54409|15|6|49|PM|second|afternoon|| +54410|AAAAAAAALIENAAAA|54410|15|6|50|PM|second|afternoon|| +54411|AAAAAAAAMIENAAAA|54411|15|6|51|PM|second|afternoon|| +54412|AAAAAAAANIENAAAA|54412|15|6|52|PM|second|afternoon|| +54413|AAAAAAAAOIENAAAA|54413|15|6|53|PM|second|afternoon|| +54414|AAAAAAAAPIENAAAA|54414|15|6|54|PM|second|afternoon|| +54415|AAAAAAAAAJENAAAA|54415|15|6|55|PM|second|afternoon|| +54416|AAAAAAAABJENAAAA|54416|15|6|56|PM|second|afternoon|| +54417|AAAAAAAACJENAAAA|54417|15|6|57|PM|second|afternoon|| +54418|AAAAAAAADJENAAAA|54418|15|6|58|PM|second|afternoon|| +54419|AAAAAAAAEJENAAAA|54419|15|6|59|PM|second|afternoon|| +54420|AAAAAAAAFJENAAAA|54420|15|7|0|PM|second|afternoon|| +54421|AAAAAAAAGJENAAAA|54421|15|7|1|PM|second|afternoon|| +54422|AAAAAAAAHJENAAAA|54422|15|7|2|PM|second|afternoon|| +54423|AAAAAAAAIJENAAAA|54423|15|7|3|PM|second|afternoon|| +54424|AAAAAAAAJJENAAAA|54424|15|7|4|PM|second|afternoon|| +54425|AAAAAAAAKJENAAAA|54425|15|7|5|PM|second|afternoon|| +54426|AAAAAAAALJENAAAA|54426|15|7|6|PM|second|afternoon|| +54427|AAAAAAAAMJENAAAA|54427|15|7|7|PM|second|afternoon|| +54428|AAAAAAAANJENAAAA|54428|15|7|8|PM|second|afternoon|| +54429|AAAAAAAAOJENAAAA|54429|15|7|9|PM|second|afternoon|| +54430|AAAAAAAAPJENAAAA|54430|15|7|10|PM|second|afternoon|| +54431|AAAAAAAAAKENAAAA|54431|15|7|11|PM|second|afternoon|| +54432|AAAAAAAABKENAAAA|54432|15|7|12|PM|second|afternoon|| +54433|AAAAAAAACKENAAAA|54433|15|7|13|PM|second|afternoon|| +54434|AAAAAAAADKENAAAA|54434|15|7|14|PM|second|afternoon|| +54435|AAAAAAAAEKENAAAA|54435|15|7|15|PM|second|afternoon|| +54436|AAAAAAAAFKENAAAA|54436|15|7|16|PM|second|afternoon|| +54437|AAAAAAAAGKENAAAA|54437|15|7|17|PM|second|afternoon|| +54438|AAAAAAAAHKENAAAA|54438|15|7|18|PM|second|afternoon|| +54439|AAAAAAAAIKENAAAA|54439|15|7|19|PM|second|afternoon|| +54440|AAAAAAAAJKENAAAA|54440|15|7|20|PM|second|afternoon|| +54441|AAAAAAAAKKENAAAA|54441|15|7|21|PM|second|afternoon|| +54442|AAAAAAAALKENAAAA|54442|15|7|22|PM|second|afternoon|| +54443|AAAAAAAAMKENAAAA|54443|15|7|23|PM|second|afternoon|| +54444|AAAAAAAANKENAAAA|54444|15|7|24|PM|second|afternoon|| +54445|AAAAAAAAOKENAAAA|54445|15|7|25|PM|second|afternoon|| +54446|AAAAAAAAPKENAAAA|54446|15|7|26|PM|second|afternoon|| +54447|AAAAAAAAALENAAAA|54447|15|7|27|PM|second|afternoon|| +54448|AAAAAAAABLENAAAA|54448|15|7|28|PM|second|afternoon|| +54449|AAAAAAAACLENAAAA|54449|15|7|29|PM|second|afternoon|| +54450|AAAAAAAADLENAAAA|54450|15|7|30|PM|second|afternoon|| +54451|AAAAAAAAELENAAAA|54451|15|7|31|PM|second|afternoon|| +54452|AAAAAAAAFLENAAAA|54452|15|7|32|PM|second|afternoon|| +54453|AAAAAAAAGLENAAAA|54453|15|7|33|PM|second|afternoon|| +54454|AAAAAAAAHLENAAAA|54454|15|7|34|PM|second|afternoon|| +54455|AAAAAAAAILENAAAA|54455|15|7|35|PM|second|afternoon|| +54456|AAAAAAAAJLENAAAA|54456|15|7|36|PM|second|afternoon|| +54457|AAAAAAAAKLENAAAA|54457|15|7|37|PM|second|afternoon|| +54458|AAAAAAAALLENAAAA|54458|15|7|38|PM|second|afternoon|| +54459|AAAAAAAAMLENAAAA|54459|15|7|39|PM|second|afternoon|| +54460|AAAAAAAANLENAAAA|54460|15|7|40|PM|second|afternoon|| +54461|AAAAAAAAOLENAAAA|54461|15|7|41|PM|second|afternoon|| +54462|AAAAAAAAPLENAAAA|54462|15|7|42|PM|second|afternoon|| +54463|AAAAAAAAAMENAAAA|54463|15|7|43|PM|second|afternoon|| +54464|AAAAAAAABMENAAAA|54464|15|7|44|PM|second|afternoon|| +54465|AAAAAAAACMENAAAA|54465|15|7|45|PM|second|afternoon|| +54466|AAAAAAAADMENAAAA|54466|15|7|46|PM|second|afternoon|| +54467|AAAAAAAAEMENAAAA|54467|15|7|47|PM|second|afternoon|| +54468|AAAAAAAAFMENAAAA|54468|15|7|48|PM|second|afternoon|| +54469|AAAAAAAAGMENAAAA|54469|15|7|49|PM|second|afternoon|| +54470|AAAAAAAAHMENAAAA|54470|15|7|50|PM|second|afternoon|| +54471|AAAAAAAAIMENAAAA|54471|15|7|51|PM|second|afternoon|| +54472|AAAAAAAAJMENAAAA|54472|15|7|52|PM|second|afternoon|| +54473|AAAAAAAAKMENAAAA|54473|15|7|53|PM|second|afternoon|| +54474|AAAAAAAALMENAAAA|54474|15|7|54|PM|second|afternoon|| +54475|AAAAAAAAMMENAAAA|54475|15|7|55|PM|second|afternoon|| +54476|AAAAAAAANMENAAAA|54476|15|7|56|PM|second|afternoon|| +54477|AAAAAAAAOMENAAAA|54477|15|7|57|PM|second|afternoon|| +54478|AAAAAAAAPMENAAAA|54478|15|7|58|PM|second|afternoon|| +54479|AAAAAAAAANENAAAA|54479|15|7|59|PM|second|afternoon|| +54480|AAAAAAAABNENAAAA|54480|15|8|0|PM|second|afternoon|| +54481|AAAAAAAACNENAAAA|54481|15|8|1|PM|second|afternoon|| +54482|AAAAAAAADNENAAAA|54482|15|8|2|PM|second|afternoon|| +54483|AAAAAAAAENENAAAA|54483|15|8|3|PM|second|afternoon|| +54484|AAAAAAAAFNENAAAA|54484|15|8|4|PM|second|afternoon|| +54485|AAAAAAAAGNENAAAA|54485|15|8|5|PM|second|afternoon|| +54486|AAAAAAAAHNENAAAA|54486|15|8|6|PM|second|afternoon|| +54487|AAAAAAAAINENAAAA|54487|15|8|7|PM|second|afternoon|| +54488|AAAAAAAAJNENAAAA|54488|15|8|8|PM|second|afternoon|| +54489|AAAAAAAAKNENAAAA|54489|15|8|9|PM|second|afternoon|| +54490|AAAAAAAALNENAAAA|54490|15|8|10|PM|second|afternoon|| +54491|AAAAAAAAMNENAAAA|54491|15|8|11|PM|second|afternoon|| +54492|AAAAAAAANNENAAAA|54492|15|8|12|PM|second|afternoon|| +54493|AAAAAAAAONENAAAA|54493|15|8|13|PM|second|afternoon|| +54494|AAAAAAAAPNENAAAA|54494|15|8|14|PM|second|afternoon|| +54495|AAAAAAAAAOENAAAA|54495|15|8|15|PM|second|afternoon|| +54496|AAAAAAAABOENAAAA|54496|15|8|16|PM|second|afternoon|| +54497|AAAAAAAACOENAAAA|54497|15|8|17|PM|second|afternoon|| +54498|AAAAAAAADOENAAAA|54498|15|8|18|PM|second|afternoon|| +54499|AAAAAAAAEOENAAAA|54499|15|8|19|PM|second|afternoon|| +54500|AAAAAAAAFOENAAAA|54500|15|8|20|PM|second|afternoon|| +54501|AAAAAAAAGOENAAAA|54501|15|8|21|PM|second|afternoon|| +54502|AAAAAAAAHOENAAAA|54502|15|8|22|PM|second|afternoon|| +54503|AAAAAAAAIOENAAAA|54503|15|8|23|PM|second|afternoon|| +54504|AAAAAAAAJOENAAAA|54504|15|8|24|PM|second|afternoon|| +54505|AAAAAAAAKOENAAAA|54505|15|8|25|PM|second|afternoon|| +54506|AAAAAAAALOENAAAA|54506|15|8|26|PM|second|afternoon|| +54507|AAAAAAAAMOENAAAA|54507|15|8|27|PM|second|afternoon|| +54508|AAAAAAAANOENAAAA|54508|15|8|28|PM|second|afternoon|| +54509|AAAAAAAAOOENAAAA|54509|15|8|29|PM|second|afternoon|| +54510|AAAAAAAAPOENAAAA|54510|15|8|30|PM|second|afternoon|| +54511|AAAAAAAAAPENAAAA|54511|15|8|31|PM|second|afternoon|| +54512|AAAAAAAABPENAAAA|54512|15|8|32|PM|second|afternoon|| +54513|AAAAAAAACPENAAAA|54513|15|8|33|PM|second|afternoon|| +54514|AAAAAAAADPENAAAA|54514|15|8|34|PM|second|afternoon|| +54515|AAAAAAAAEPENAAAA|54515|15|8|35|PM|second|afternoon|| +54516|AAAAAAAAFPENAAAA|54516|15|8|36|PM|second|afternoon|| +54517|AAAAAAAAGPENAAAA|54517|15|8|37|PM|second|afternoon|| +54518|AAAAAAAAHPENAAAA|54518|15|8|38|PM|second|afternoon|| +54519|AAAAAAAAIPENAAAA|54519|15|8|39|PM|second|afternoon|| +54520|AAAAAAAAJPENAAAA|54520|15|8|40|PM|second|afternoon|| +54521|AAAAAAAAKPENAAAA|54521|15|8|41|PM|second|afternoon|| +54522|AAAAAAAALPENAAAA|54522|15|8|42|PM|second|afternoon|| +54523|AAAAAAAAMPENAAAA|54523|15|8|43|PM|second|afternoon|| +54524|AAAAAAAANPENAAAA|54524|15|8|44|PM|second|afternoon|| +54525|AAAAAAAAOPENAAAA|54525|15|8|45|PM|second|afternoon|| +54526|AAAAAAAAPPENAAAA|54526|15|8|46|PM|second|afternoon|| +54527|AAAAAAAAAAFNAAAA|54527|15|8|47|PM|second|afternoon|| +54528|AAAAAAAABAFNAAAA|54528|15|8|48|PM|second|afternoon|| +54529|AAAAAAAACAFNAAAA|54529|15|8|49|PM|second|afternoon|| +54530|AAAAAAAADAFNAAAA|54530|15|8|50|PM|second|afternoon|| +54531|AAAAAAAAEAFNAAAA|54531|15|8|51|PM|second|afternoon|| +54532|AAAAAAAAFAFNAAAA|54532|15|8|52|PM|second|afternoon|| +54533|AAAAAAAAGAFNAAAA|54533|15|8|53|PM|second|afternoon|| +54534|AAAAAAAAHAFNAAAA|54534|15|8|54|PM|second|afternoon|| +54535|AAAAAAAAIAFNAAAA|54535|15|8|55|PM|second|afternoon|| +54536|AAAAAAAAJAFNAAAA|54536|15|8|56|PM|second|afternoon|| +54537|AAAAAAAAKAFNAAAA|54537|15|8|57|PM|second|afternoon|| +54538|AAAAAAAALAFNAAAA|54538|15|8|58|PM|second|afternoon|| +54539|AAAAAAAAMAFNAAAA|54539|15|8|59|PM|second|afternoon|| +54540|AAAAAAAANAFNAAAA|54540|15|9|0|PM|second|afternoon|| +54541|AAAAAAAAOAFNAAAA|54541|15|9|1|PM|second|afternoon|| +54542|AAAAAAAAPAFNAAAA|54542|15|9|2|PM|second|afternoon|| +54543|AAAAAAAAABFNAAAA|54543|15|9|3|PM|second|afternoon|| +54544|AAAAAAAABBFNAAAA|54544|15|9|4|PM|second|afternoon|| +54545|AAAAAAAACBFNAAAA|54545|15|9|5|PM|second|afternoon|| +54546|AAAAAAAADBFNAAAA|54546|15|9|6|PM|second|afternoon|| +54547|AAAAAAAAEBFNAAAA|54547|15|9|7|PM|second|afternoon|| +54548|AAAAAAAAFBFNAAAA|54548|15|9|8|PM|second|afternoon|| +54549|AAAAAAAAGBFNAAAA|54549|15|9|9|PM|second|afternoon|| +54550|AAAAAAAAHBFNAAAA|54550|15|9|10|PM|second|afternoon|| +54551|AAAAAAAAIBFNAAAA|54551|15|9|11|PM|second|afternoon|| +54552|AAAAAAAAJBFNAAAA|54552|15|9|12|PM|second|afternoon|| +54553|AAAAAAAAKBFNAAAA|54553|15|9|13|PM|second|afternoon|| +54554|AAAAAAAALBFNAAAA|54554|15|9|14|PM|second|afternoon|| +54555|AAAAAAAAMBFNAAAA|54555|15|9|15|PM|second|afternoon|| +54556|AAAAAAAANBFNAAAA|54556|15|9|16|PM|second|afternoon|| +54557|AAAAAAAAOBFNAAAA|54557|15|9|17|PM|second|afternoon|| +54558|AAAAAAAAPBFNAAAA|54558|15|9|18|PM|second|afternoon|| +54559|AAAAAAAAACFNAAAA|54559|15|9|19|PM|second|afternoon|| +54560|AAAAAAAABCFNAAAA|54560|15|9|20|PM|second|afternoon|| +54561|AAAAAAAACCFNAAAA|54561|15|9|21|PM|second|afternoon|| +54562|AAAAAAAADCFNAAAA|54562|15|9|22|PM|second|afternoon|| +54563|AAAAAAAAECFNAAAA|54563|15|9|23|PM|second|afternoon|| +54564|AAAAAAAAFCFNAAAA|54564|15|9|24|PM|second|afternoon|| +54565|AAAAAAAAGCFNAAAA|54565|15|9|25|PM|second|afternoon|| +54566|AAAAAAAAHCFNAAAA|54566|15|9|26|PM|second|afternoon|| +54567|AAAAAAAAICFNAAAA|54567|15|9|27|PM|second|afternoon|| +54568|AAAAAAAAJCFNAAAA|54568|15|9|28|PM|second|afternoon|| +54569|AAAAAAAAKCFNAAAA|54569|15|9|29|PM|second|afternoon|| +54570|AAAAAAAALCFNAAAA|54570|15|9|30|PM|second|afternoon|| +54571|AAAAAAAAMCFNAAAA|54571|15|9|31|PM|second|afternoon|| +54572|AAAAAAAANCFNAAAA|54572|15|9|32|PM|second|afternoon|| +54573|AAAAAAAAOCFNAAAA|54573|15|9|33|PM|second|afternoon|| +54574|AAAAAAAAPCFNAAAA|54574|15|9|34|PM|second|afternoon|| +54575|AAAAAAAAADFNAAAA|54575|15|9|35|PM|second|afternoon|| +54576|AAAAAAAABDFNAAAA|54576|15|9|36|PM|second|afternoon|| +54577|AAAAAAAACDFNAAAA|54577|15|9|37|PM|second|afternoon|| +54578|AAAAAAAADDFNAAAA|54578|15|9|38|PM|second|afternoon|| +54579|AAAAAAAAEDFNAAAA|54579|15|9|39|PM|second|afternoon|| +54580|AAAAAAAAFDFNAAAA|54580|15|9|40|PM|second|afternoon|| +54581|AAAAAAAAGDFNAAAA|54581|15|9|41|PM|second|afternoon|| +54582|AAAAAAAAHDFNAAAA|54582|15|9|42|PM|second|afternoon|| +54583|AAAAAAAAIDFNAAAA|54583|15|9|43|PM|second|afternoon|| +54584|AAAAAAAAJDFNAAAA|54584|15|9|44|PM|second|afternoon|| +54585|AAAAAAAAKDFNAAAA|54585|15|9|45|PM|second|afternoon|| +54586|AAAAAAAALDFNAAAA|54586|15|9|46|PM|second|afternoon|| +54587|AAAAAAAAMDFNAAAA|54587|15|9|47|PM|second|afternoon|| +54588|AAAAAAAANDFNAAAA|54588|15|9|48|PM|second|afternoon|| +54589|AAAAAAAAODFNAAAA|54589|15|9|49|PM|second|afternoon|| +54590|AAAAAAAAPDFNAAAA|54590|15|9|50|PM|second|afternoon|| +54591|AAAAAAAAAEFNAAAA|54591|15|9|51|PM|second|afternoon|| +54592|AAAAAAAABEFNAAAA|54592|15|9|52|PM|second|afternoon|| +54593|AAAAAAAACEFNAAAA|54593|15|9|53|PM|second|afternoon|| +54594|AAAAAAAADEFNAAAA|54594|15|9|54|PM|second|afternoon|| +54595|AAAAAAAAEEFNAAAA|54595|15|9|55|PM|second|afternoon|| +54596|AAAAAAAAFEFNAAAA|54596|15|9|56|PM|second|afternoon|| +54597|AAAAAAAAGEFNAAAA|54597|15|9|57|PM|second|afternoon|| +54598|AAAAAAAAHEFNAAAA|54598|15|9|58|PM|second|afternoon|| +54599|AAAAAAAAIEFNAAAA|54599|15|9|59|PM|second|afternoon|| +54600|AAAAAAAAJEFNAAAA|54600|15|10|0|PM|second|afternoon|| +54601|AAAAAAAAKEFNAAAA|54601|15|10|1|PM|second|afternoon|| +54602|AAAAAAAALEFNAAAA|54602|15|10|2|PM|second|afternoon|| +54603|AAAAAAAAMEFNAAAA|54603|15|10|3|PM|second|afternoon|| +54604|AAAAAAAANEFNAAAA|54604|15|10|4|PM|second|afternoon|| +54605|AAAAAAAAOEFNAAAA|54605|15|10|5|PM|second|afternoon|| +54606|AAAAAAAAPEFNAAAA|54606|15|10|6|PM|second|afternoon|| +54607|AAAAAAAAAFFNAAAA|54607|15|10|7|PM|second|afternoon|| +54608|AAAAAAAABFFNAAAA|54608|15|10|8|PM|second|afternoon|| +54609|AAAAAAAACFFNAAAA|54609|15|10|9|PM|second|afternoon|| +54610|AAAAAAAADFFNAAAA|54610|15|10|10|PM|second|afternoon|| +54611|AAAAAAAAEFFNAAAA|54611|15|10|11|PM|second|afternoon|| +54612|AAAAAAAAFFFNAAAA|54612|15|10|12|PM|second|afternoon|| +54613|AAAAAAAAGFFNAAAA|54613|15|10|13|PM|second|afternoon|| +54614|AAAAAAAAHFFNAAAA|54614|15|10|14|PM|second|afternoon|| +54615|AAAAAAAAIFFNAAAA|54615|15|10|15|PM|second|afternoon|| +54616|AAAAAAAAJFFNAAAA|54616|15|10|16|PM|second|afternoon|| +54617|AAAAAAAAKFFNAAAA|54617|15|10|17|PM|second|afternoon|| +54618|AAAAAAAALFFNAAAA|54618|15|10|18|PM|second|afternoon|| +54619|AAAAAAAAMFFNAAAA|54619|15|10|19|PM|second|afternoon|| +54620|AAAAAAAANFFNAAAA|54620|15|10|20|PM|second|afternoon|| +54621|AAAAAAAAOFFNAAAA|54621|15|10|21|PM|second|afternoon|| +54622|AAAAAAAAPFFNAAAA|54622|15|10|22|PM|second|afternoon|| +54623|AAAAAAAAAGFNAAAA|54623|15|10|23|PM|second|afternoon|| +54624|AAAAAAAABGFNAAAA|54624|15|10|24|PM|second|afternoon|| +54625|AAAAAAAACGFNAAAA|54625|15|10|25|PM|second|afternoon|| +54626|AAAAAAAADGFNAAAA|54626|15|10|26|PM|second|afternoon|| +54627|AAAAAAAAEGFNAAAA|54627|15|10|27|PM|second|afternoon|| +54628|AAAAAAAAFGFNAAAA|54628|15|10|28|PM|second|afternoon|| +54629|AAAAAAAAGGFNAAAA|54629|15|10|29|PM|second|afternoon|| +54630|AAAAAAAAHGFNAAAA|54630|15|10|30|PM|second|afternoon|| +54631|AAAAAAAAIGFNAAAA|54631|15|10|31|PM|second|afternoon|| +54632|AAAAAAAAJGFNAAAA|54632|15|10|32|PM|second|afternoon|| +54633|AAAAAAAAKGFNAAAA|54633|15|10|33|PM|second|afternoon|| +54634|AAAAAAAALGFNAAAA|54634|15|10|34|PM|second|afternoon|| +54635|AAAAAAAAMGFNAAAA|54635|15|10|35|PM|second|afternoon|| +54636|AAAAAAAANGFNAAAA|54636|15|10|36|PM|second|afternoon|| +54637|AAAAAAAAOGFNAAAA|54637|15|10|37|PM|second|afternoon|| +54638|AAAAAAAAPGFNAAAA|54638|15|10|38|PM|second|afternoon|| +54639|AAAAAAAAAHFNAAAA|54639|15|10|39|PM|second|afternoon|| +54640|AAAAAAAABHFNAAAA|54640|15|10|40|PM|second|afternoon|| +54641|AAAAAAAACHFNAAAA|54641|15|10|41|PM|second|afternoon|| +54642|AAAAAAAADHFNAAAA|54642|15|10|42|PM|second|afternoon|| +54643|AAAAAAAAEHFNAAAA|54643|15|10|43|PM|second|afternoon|| +54644|AAAAAAAAFHFNAAAA|54644|15|10|44|PM|second|afternoon|| +54645|AAAAAAAAGHFNAAAA|54645|15|10|45|PM|second|afternoon|| +54646|AAAAAAAAHHFNAAAA|54646|15|10|46|PM|second|afternoon|| +54647|AAAAAAAAIHFNAAAA|54647|15|10|47|PM|second|afternoon|| +54648|AAAAAAAAJHFNAAAA|54648|15|10|48|PM|second|afternoon|| +54649|AAAAAAAAKHFNAAAA|54649|15|10|49|PM|second|afternoon|| +54650|AAAAAAAALHFNAAAA|54650|15|10|50|PM|second|afternoon|| +54651|AAAAAAAAMHFNAAAA|54651|15|10|51|PM|second|afternoon|| +54652|AAAAAAAANHFNAAAA|54652|15|10|52|PM|second|afternoon|| +54653|AAAAAAAAOHFNAAAA|54653|15|10|53|PM|second|afternoon|| +54654|AAAAAAAAPHFNAAAA|54654|15|10|54|PM|second|afternoon|| +54655|AAAAAAAAAIFNAAAA|54655|15|10|55|PM|second|afternoon|| +54656|AAAAAAAABIFNAAAA|54656|15|10|56|PM|second|afternoon|| +54657|AAAAAAAACIFNAAAA|54657|15|10|57|PM|second|afternoon|| +54658|AAAAAAAADIFNAAAA|54658|15|10|58|PM|second|afternoon|| +54659|AAAAAAAAEIFNAAAA|54659|15|10|59|PM|second|afternoon|| +54660|AAAAAAAAFIFNAAAA|54660|15|11|0|PM|second|afternoon|| +54661|AAAAAAAAGIFNAAAA|54661|15|11|1|PM|second|afternoon|| +54662|AAAAAAAAHIFNAAAA|54662|15|11|2|PM|second|afternoon|| +54663|AAAAAAAAIIFNAAAA|54663|15|11|3|PM|second|afternoon|| +54664|AAAAAAAAJIFNAAAA|54664|15|11|4|PM|second|afternoon|| +54665|AAAAAAAAKIFNAAAA|54665|15|11|5|PM|second|afternoon|| +54666|AAAAAAAALIFNAAAA|54666|15|11|6|PM|second|afternoon|| +54667|AAAAAAAAMIFNAAAA|54667|15|11|7|PM|second|afternoon|| +54668|AAAAAAAANIFNAAAA|54668|15|11|8|PM|second|afternoon|| +54669|AAAAAAAAOIFNAAAA|54669|15|11|9|PM|second|afternoon|| +54670|AAAAAAAAPIFNAAAA|54670|15|11|10|PM|second|afternoon|| +54671|AAAAAAAAAJFNAAAA|54671|15|11|11|PM|second|afternoon|| +54672|AAAAAAAABJFNAAAA|54672|15|11|12|PM|second|afternoon|| +54673|AAAAAAAACJFNAAAA|54673|15|11|13|PM|second|afternoon|| +54674|AAAAAAAADJFNAAAA|54674|15|11|14|PM|second|afternoon|| +54675|AAAAAAAAEJFNAAAA|54675|15|11|15|PM|second|afternoon|| +54676|AAAAAAAAFJFNAAAA|54676|15|11|16|PM|second|afternoon|| +54677|AAAAAAAAGJFNAAAA|54677|15|11|17|PM|second|afternoon|| +54678|AAAAAAAAHJFNAAAA|54678|15|11|18|PM|second|afternoon|| +54679|AAAAAAAAIJFNAAAA|54679|15|11|19|PM|second|afternoon|| +54680|AAAAAAAAJJFNAAAA|54680|15|11|20|PM|second|afternoon|| +54681|AAAAAAAAKJFNAAAA|54681|15|11|21|PM|second|afternoon|| +54682|AAAAAAAALJFNAAAA|54682|15|11|22|PM|second|afternoon|| +54683|AAAAAAAAMJFNAAAA|54683|15|11|23|PM|second|afternoon|| +54684|AAAAAAAANJFNAAAA|54684|15|11|24|PM|second|afternoon|| +54685|AAAAAAAAOJFNAAAA|54685|15|11|25|PM|second|afternoon|| +54686|AAAAAAAAPJFNAAAA|54686|15|11|26|PM|second|afternoon|| +54687|AAAAAAAAAKFNAAAA|54687|15|11|27|PM|second|afternoon|| +54688|AAAAAAAABKFNAAAA|54688|15|11|28|PM|second|afternoon|| +54689|AAAAAAAACKFNAAAA|54689|15|11|29|PM|second|afternoon|| +54690|AAAAAAAADKFNAAAA|54690|15|11|30|PM|second|afternoon|| +54691|AAAAAAAAEKFNAAAA|54691|15|11|31|PM|second|afternoon|| +54692|AAAAAAAAFKFNAAAA|54692|15|11|32|PM|second|afternoon|| +54693|AAAAAAAAGKFNAAAA|54693|15|11|33|PM|second|afternoon|| +54694|AAAAAAAAHKFNAAAA|54694|15|11|34|PM|second|afternoon|| +54695|AAAAAAAAIKFNAAAA|54695|15|11|35|PM|second|afternoon|| +54696|AAAAAAAAJKFNAAAA|54696|15|11|36|PM|second|afternoon|| +54697|AAAAAAAAKKFNAAAA|54697|15|11|37|PM|second|afternoon|| +54698|AAAAAAAALKFNAAAA|54698|15|11|38|PM|second|afternoon|| +54699|AAAAAAAAMKFNAAAA|54699|15|11|39|PM|second|afternoon|| +54700|AAAAAAAANKFNAAAA|54700|15|11|40|PM|second|afternoon|| +54701|AAAAAAAAOKFNAAAA|54701|15|11|41|PM|second|afternoon|| +54702|AAAAAAAAPKFNAAAA|54702|15|11|42|PM|second|afternoon|| +54703|AAAAAAAAALFNAAAA|54703|15|11|43|PM|second|afternoon|| +54704|AAAAAAAABLFNAAAA|54704|15|11|44|PM|second|afternoon|| +54705|AAAAAAAACLFNAAAA|54705|15|11|45|PM|second|afternoon|| +54706|AAAAAAAADLFNAAAA|54706|15|11|46|PM|second|afternoon|| +54707|AAAAAAAAELFNAAAA|54707|15|11|47|PM|second|afternoon|| +54708|AAAAAAAAFLFNAAAA|54708|15|11|48|PM|second|afternoon|| +54709|AAAAAAAAGLFNAAAA|54709|15|11|49|PM|second|afternoon|| +54710|AAAAAAAAHLFNAAAA|54710|15|11|50|PM|second|afternoon|| +54711|AAAAAAAAILFNAAAA|54711|15|11|51|PM|second|afternoon|| +54712|AAAAAAAAJLFNAAAA|54712|15|11|52|PM|second|afternoon|| +54713|AAAAAAAAKLFNAAAA|54713|15|11|53|PM|second|afternoon|| +54714|AAAAAAAALLFNAAAA|54714|15|11|54|PM|second|afternoon|| +54715|AAAAAAAAMLFNAAAA|54715|15|11|55|PM|second|afternoon|| +54716|AAAAAAAANLFNAAAA|54716|15|11|56|PM|second|afternoon|| +54717|AAAAAAAAOLFNAAAA|54717|15|11|57|PM|second|afternoon|| +54718|AAAAAAAAPLFNAAAA|54718|15|11|58|PM|second|afternoon|| +54719|AAAAAAAAAMFNAAAA|54719|15|11|59|PM|second|afternoon|| +54720|AAAAAAAABMFNAAAA|54720|15|12|0|PM|second|afternoon|| +54721|AAAAAAAACMFNAAAA|54721|15|12|1|PM|second|afternoon|| +54722|AAAAAAAADMFNAAAA|54722|15|12|2|PM|second|afternoon|| +54723|AAAAAAAAEMFNAAAA|54723|15|12|3|PM|second|afternoon|| +54724|AAAAAAAAFMFNAAAA|54724|15|12|4|PM|second|afternoon|| +54725|AAAAAAAAGMFNAAAA|54725|15|12|5|PM|second|afternoon|| +54726|AAAAAAAAHMFNAAAA|54726|15|12|6|PM|second|afternoon|| +54727|AAAAAAAAIMFNAAAA|54727|15|12|7|PM|second|afternoon|| +54728|AAAAAAAAJMFNAAAA|54728|15|12|8|PM|second|afternoon|| +54729|AAAAAAAAKMFNAAAA|54729|15|12|9|PM|second|afternoon|| +54730|AAAAAAAALMFNAAAA|54730|15|12|10|PM|second|afternoon|| +54731|AAAAAAAAMMFNAAAA|54731|15|12|11|PM|second|afternoon|| +54732|AAAAAAAANMFNAAAA|54732|15|12|12|PM|second|afternoon|| +54733|AAAAAAAAOMFNAAAA|54733|15|12|13|PM|second|afternoon|| +54734|AAAAAAAAPMFNAAAA|54734|15|12|14|PM|second|afternoon|| +54735|AAAAAAAAANFNAAAA|54735|15|12|15|PM|second|afternoon|| +54736|AAAAAAAABNFNAAAA|54736|15|12|16|PM|second|afternoon|| +54737|AAAAAAAACNFNAAAA|54737|15|12|17|PM|second|afternoon|| +54738|AAAAAAAADNFNAAAA|54738|15|12|18|PM|second|afternoon|| +54739|AAAAAAAAENFNAAAA|54739|15|12|19|PM|second|afternoon|| +54740|AAAAAAAAFNFNAAAA|54740|15|12|20|PM|second|afternoon|| +54741|AAAAAAAAGNFNAAAA|54741|15|12|21|PM|second|afternoon|| +54742|AAAAAAAAHNFNAAAA|54742|15|12|22|PM|second|afternoon|| +54743|AAAAAAAAINFNAAAA|54743|15|12|23|PM|second|afternoon|| +54744|AAAAAAAAJNFNAAAA|54744|15|12|24|PM|second|afternoon|| +54745|AAAAAAAAKNFNAAAA|54745|15|12|25|PM|second|afternoon|| +54746|AAAAAAAALNFNAAAA|54746|15|12|26|PM|second|afternoon|| +54747|AAAAAAAAMNFNAAAA|54747|15|12|27|PM|second|afternoon|| +54748|AAAAAAAANNFNAAAA|54748|15|12|28|PM|second|afternoon|| +54749|AAAAAAAAONFNAAAA|54749|15|12|29|PM|second|afternoon|| +54750|AAAAAAAAPNFNAAAA|54750|15|12|30|PM|second|afternoon|| +54751|AAAAAAAAAOFNAAAA|54751|15|12|31|PM|second|afternoon|| +54752|AAAAAAAABOFNAAAA|54752|15|12|32|PM|second|afternoon|| +54753|AAAAAAAACOFNAAAA|54753|15|12|33|PM|second|afternoon|| +54754|AAAAAAAADOFNAAAA|54754|15|12|34|PM|second|afternoon|| +54755|AAAAAAAAEOFNAAAA|54755|15|12|35|PM|second|afternoon|| +54756|AAAAAAAAFOFNAAAA|54756|15|12|36|PM|second|afternoon|| +54757|AAAAAAAAGOFNAAAA|54757|15|12|37|PM|second|afternoon|| +54758|AAAAAAAAHOFNAAAA|54758|15|12|38|PM|second|afternoon|| +54759|AAAAAAAAIOFNAAAA|54759|15|12|39|PM|second|afternoon|| +54760|AAAAAAAAJOFNAAAA|54760|15|12|40|PM|second|afternoon|| +54761|AAAAAAAAKOFNAAAA|54761|15|12|41|PM|second|afternoon|| +54762|AAAAAAAALOFNAAAA|54762|15|12|42|PM|second|afternoon|| +54763|AAAAAAAAMOFNAAAA|54763|15|12|43|PM|second|afternoon|| +54764|AAAAAAAANOFNAAAA|54764|15|12|44|PM|second|afternoon|| +54765|AAAAAAAAOOFNAAAA|54765|15|12|45|PM|second|afternoon|| +54766|AAAAAAAAPOFNAAAA|54766|15|12|46|PM|second|afternoon|| +54767|AAAAAAAAAPFNAAAA|54767|15|12|47|PM|second|afternoon|| +54768|AAAAAAAABPFNAAAA|54768|15|12|48|PM|second|afternoon|| +54769|AAAAAAAACPFNAAAA|54769|15|12|49|PM|second|afternoon|| +54770|AAAAAAAADPFNAAAA|54770|15|12|50|PM|second|afternoon|| +54771|AAAAAAAAEPFNAAAA|54771|15|12|51|PM|second|afternoon|| +54772|AAAAAAAAFPFNAAAA|54772|15|12|52|PM|second|afternoon|| +54773|AAAAAAAAGPFNAAAA|54773|15|12|53|PM|second|afternoon|| +54774|AAAAAAAAHPFNAAAA|54774|15|12|54|PM|second|afternoon|| +54775|AAAAAAAAIPFNAAAA|54775|15|12|55|PM|second|afternoon|| +54776|AAAAAAAAJPFNAAAA|54776|15|12|56|PM|second|afternoon|| +54777|AAAAAAAAKPFNAAAA|54777|15|12|57|PM|second|afternoon|| +54778|AAAAAAAALPFNAAAA|54778|15|12|58|PM|second|afternoon|| +54779|AAAAAAAAMPFNAAAA|54779|15|12|59|PM|second|afternoon|| +54780|AAAAAAAANPFNAAAA|54780|15|13|0|PM|second|afternoon|| +54781|AAAAAAAAOPFNAAAA|54781|15|13|1|PM|second|afternoon|| +54782|AAAAAAAAPPFNAAAA|54782|15|13|2|PM|second|afternoon|| +54783|AAAAAAAAAAGNAAAA|54783|15|13|3|PM|second|afternoon|| +54784|AAAAAAAABAGNAAAA|54784|15|13|4|PM|second|afternoon|| +54785|AAAAAAAACAGNAAAA|54785|15|13|5|PM|second|afternoon|| +54786|AAAAAAAADAGNAAAA|54786|15|13|6|PM|second|afternoon|| +54787|AAAAAAAAEAGNAAAA|54787|15|13|7|PM|second|afternoon|| +54788|AAAAAAAAFAGNAAAA|54788|15|13|8|PM|second|afternoon|| +54789|AAAAAAAAGAGNAAAA|54789|15|13|9|PM|second|afternoon|| +54790|AAAAAAAAHAGNAAAA|54790|15|13|10|PM|second|afternoon|| +54791|AAAAAAAAIAGNAAAA|54791|15|13|11|PM|second|afternoon|| +54792|AAAAAAAAJAGNAAAA|54792|15|13|12|PM|second|afternoon|| +54793|AAAAAAAAKAGNAAAA|54793|15|13|13|PM|second|afternoon|| +54794|AAAAAAAALAGNAAAA|54794|15|13|14|PM|second|afternoon|| +54795|AAAAAAAAMAGNAAAA|54795|15|13|15|PM|second|afternoon|| +54796|AAAAAAAANAGNAAAA|54796|15|13|16|PM|second|afternoon|| +54797|AAAAAAAAOAGNAAAA|54797|15|13|17|PM|second|afternoon|| +54798|AAAAAAAAPAGNAAAA|54798|15|13|18|PM|second|afternoon|| +54799|AAAAAAAAABGNAAAA|54799|15|13|19|PM|second|afternoon|| +54800|AAAAAAAABBGNAAAA|54800|15|13|20|PM|second|afternoon|| +54801|AAAAAAAACBGNAAAA|54801|15|13|21|PM|second|afternoon|| +54802|AAAAAAAADBGNAAAA|54802|15|13|22|PM|second|afternoon|| +54803|AAAAAAAAEBGNAAAA|54803|15|13|23|PM|second|afternoon|| +54804|AAAAAAAAFBGNAAAA|54804|15|13|24|PM|second|afternoon|| +54805|AAAAAAAAGBGNAAAA|54805|15|13|25|PM|second|afternoon|| +54806|AAAAAAAAHBGNAAAA|54806|15|13|26|PM|second|afternoon|| +54807|AAAAAAAAIBGNAAAA|54807|15|13|27|PM|second|afternoon|| +54808|AAAAAAAAJBGNAAAA|54808|15|13|28|PM|second|afternoon|| +54809|AAAAAAAAKBGNAAAA|54809|15|13|29|PM|second|afternoon|| +54810|AAAAAAAALBGNAAAA|54810|15|13|30|PM|second|afternoon|| +54811|AAAAAAAAMBGNAAAA|54811|15|13|31|PM|second|afternoon|| +54812|AAAAAAAANBGNAAAA|54812|15|13|32|PM|second|afternoon|| +54813|AAAAAAAAOBGNAAAA|54813|15|13|33|PM|second|afternoon|| +54814|AAAAAAAAPBGNAAAA|54814|15|13|34|PM|second|afternoon|| +54815|AAAAAAAAACGNAAAA|54815|15|13|35|PM|second|afternoon|| +54816|AAAAAAAABCGNAAAA|54816|15|13|36|PM|second|afternoon|| +54817|AAAAAAAACCGNAAAA|54817|15|13|37|PM|second|afternoon|| +54818|AAAAAAAADCGNAAAA|54818|15|13|38|PM|second|afternoon|| +54819|AAAAAAAAECGNAAAA|54819|15|13|39|PM|second|afternoon|| +54820|AAAAAAAAFCGNAAAA|54820|15|13|40|PM|second|afternoon|| +54821|AAAAAAAAGCGNAAAA|54821|15|13|41|PM|second|afternoon|| +54822|AAAAAAAAHCGNAAAA|54822|15|13|42|PM|second|afternoon|| +54823|AAAAAAAAICGNAAAA|54823|15|13|43|PM|second|afternoon|| +54824|AAAAAAAAJCGNAAAA|54824|15|13|44|PM|second|afternoon|| +54825|AAAAAAAAKCGNAAAA|54825|15|13|45|PM|second|afternoon|| +54826|AAAAAAAALCGNAAAA|54826|15|13|46|PM|second|afternoon|| +54827|AAAAAAAAMCGNAAAA|54827|15|13|47|PM|second|afternoon|| +54828|AAAAAAAANCGNAAAA|54828|15|13|48|PM|second|afternoon|| +54829|AAAAAAAAOCGNAAAA|54829|15|13|49|PM|second|afternoon|| +54830|AAAAAAAAPCGNAAAA|54830|15|13|50|PM|second|afternoon|| +54831|AAAAAAAAADGNAAAA|54831|15|13|51|PM|second|afternoon|| +54832|AAAAAAAABDGNAAAA|54832|15|13|52|PM|second|afternoon|| +54833|AAAAAAAACDGNAAAA|54833|15|13|53|PM|second|afternoon|| +54834|AAAAAAAADDGNAAAA|54834|15|13|54|PM|second|afternoon|| +54835|AAAAAAAAEDGNAAAA|54835|15|13|55|PM|second|afternoon|| +54836|AAAAAAAAFDGNAAAA|54836|15|13|56|PM|second|afternoon|| +54837|AAAAAAAAGDGNAAAA|54837|15|13|57|PM|second|afternoon|| +54838|AAAAAAAAHDGNAAAA|54838|15|13|58|PM|second|afternoon|| +54839|AAAAAAAAIDGNAAAA|54839|15|13|59|PM|second|afternoon|| +54840|AAAAAAAAJDGNAAAA|54840|15|14|0|PM|second|afternoon|| +54841|AAAAAAAAKDGNAAAA|54841|15|14|1|PM|second|afternoon|| +54842|AAAAAAAALDGNAAAA|54842|15|14|2|PM|second|afternoon|| +54843|AAAAAAAAMDGNAAAA|54843|15|14|3|PM|second|afternoon|| +54844|AAAAAAAANDGNAAAA|54844|15|14|4|PM|second|afternoon|| +54845|AAAAAAAAODGNAAAA|54845|15|14|5|PM|second|afternoon|| +54846|AAAAAAAAPDGNAAAA|54846|15|14|6|PM|second|afternoon|| +54847|AAAAAAAAAEGNAAAA|54847|15|14|7|PM|second|afternoon|| +54848|AAAAAAAABEGNAAAA|54848|15|14|8|PM|second|afternoon|| +54849|AAAAAAAACEGNAAAA|54849|15|14|9|PM|second|afternoon|| +54850|AAAAAAAADEGNAAAA|54850|15|14|10|PM|second|afternoon|| +54851|AAAAAAAAEEGNAAAA|54851|15|14|11|PM|second|afternoon|| +54852|AAAAAAAAFEGNAAAA|54852|15|14|12|PM|second|afternoon|| +54853|AAAAAAAAGEGNAAAA|54853|15|14|13|PM|second|afternoon|| +54854|AAAAAAAAHEGNAAAA|54854|15|14|14|PM|second|afternoon|| +54855|AAAAAAAAIEGNAAAA|54855|15|14|15|PM|second|afternoon|| +54856|AAAAAAAAJEGNAAAA|54856|15|14|16|PM|second|afternoon|| +54857|AAAAAAAAKEGNAAAA|54857|15|14|17|PM|second|afternoon|| +54858|AAAAAAAALEGNAAAA|54858|15|14|18|PM|second|afternoon|| +54859|AAAAAAAAMEGNAAAA|54859|15|14|19|PM|second|afternoon|| +54860|AAAAAAAANEGNAAAA|54860|15|14|20|PM|second|afternoon|| +54861|AAAAAAAAOEGNAAAA|54861|15|14|21|PM|second|afternoon|| +54862|AAAAAAAAPEGNAAAA|54862|15|14|22|PM|second|afternoon|| +54863|AAAAAAAAAFGNAAAA|54863|15|14|23|PM|second|afternoon|| +54864|AAAAAAAABFGNAAAA|54864|15|14|24|PM|second|afternoon|| +54865|AAAAAAAACFGNAAAA|54865|15|14|25|PM|second|afternoon|| +54866|AAAAAAAADFGNAAAA|54866|15|14|26|PM|second|afternoon|| +54867|AAAAAAAAEFGNAAAA|54867|15|14|27|PM|second|afternoon|| +54868|AAAAAAAAFFGNAAAA|54868|15|14|28|PM|second|afternoon|| +54869|AAAAAAAAGFGNAAAA|54869|15|14|29|PM|second|afternoon|| +54870|AAAAAAAAHFGNAAAA|54870|15|14|30|PM|second|afternoon|| +54871|AAAAAAAAIFGNAAAA|54871|15|14|31|PM|second|afternoon|| +54872|AAAAAAAAJFGNAAAA|54872|15|14|32|PM|second|afternoon|| +54873|AAAAAAAAKFGNAAAA|54873|15|14|33|PM|second|afternoon|| +54874|AAAAAAAALFGNAAAA|54874|15|14|34|PM|second|afternoon|| +54875|AAAAAAAAMFGNAAAA|54875|15|14|35|PM|second|afternoon|| +54876|AAAAAAAANFGNAAAA|54876|15|14|36|PM|second|afternoon|| +54877|AAAAAAAAOFGNAAAA|54877|15|14|37|PM|second|afternoon|| +54878|AAAAAAAAPFGNAAAA|54878|15|14|38|PM|second|afternoon|| +54879|AAAAAAAAAGGNAAAA|54879|15|14|39|PM|second|afternoon|| +54880|AAAAAAAABGGNAAAA|54880|15|14|40|PM|second|afternoon|| +54881|AAAAAAAACGGNAAAA|54881|15|14|41|PM|second|afternoon|| +54882|AAAAAAAADGGNAAAA|54882|15|14|42|PM|second|afternoon|| +54883|AAAAAAAAEGGNAAAA|54883|15|14|43|PM|second|afternoon|| +54884|AAAAAAAAFGGNAAAA|54884|15|14|44|PM|second|afternoon|| +54885|AAAAAAAAGGGNAAAA|54885|15|14|45|PM|second|afternoon|| +54886|AAAAAAAAHGGNAAAA|54886|15|14|46|PM|second|afternoon|| +54887|AAAAAAAAIGGNAAAA|54887|15|14|47|PM|second|afternoon|| +54888|AAAAAAAAJGGNAAAA|54888|15|14|48|PM|second|afternoon|| +54889|AAAAAAAAKGGNAAAA|54889|15|14|49|PM|second|afternoon|| +54890|AAAAAAAALGGNAAAA|54890|15|14|50|PM|second|afternoon|| +54891|AAAAAAAAMGGNAAAA|54891|15|14|51|PM|second|afternoon|| +54892|AAAAAAAANGGNAAAA|54892|15|14|52|PM|second|afternoon|| +54893|AAAAAAAAOGGNAAAA|54893|15|14|53|PM|second|afternoon|| +54894|AAAAAAAAPGGNAAAA|54894|15|14|54|PM|second|afternoon|| +54895|AAAAAAAAAHGNAAAA|54895|15|14|55|PM|second|afternoon|| +54896|AAAAAAAABHGNAAAA|54896|15|14|56|PM|second|afternoon|| +54897|AAAAAAAACHGNAAAA|54897|15|14|57|PM|second|afternoon|| +54898|AAAAAAAADHGNAAAA|54898|15|14|58|PM|second|afternoon|| +54899|AAAAAAAAEHGNAAAA|54899|15|14|59|PM|second|afternoon|| +54900|AAAAAAAAFHGNAAAA|54900|15|15|0|PM|second|afternoon|| +54901|AAAAAAAAGHGNAAAA|54901|15|15|1|PM|second|afternoon|| +54902|AAAAAAAAHHGNAAAA|54902|15|15|2|PM|second|afternoon|| +54903|AAAAAAAAIHGNAAAA|54903|15|15|3|PM|second|afternoon|| +54904|AAAAAAAAJHGNAAAA|54904|15|15|4|PM|second|afternoon|| +54905|AAAAAAAAKHGNAAAA|54905|15|15|5|PM|second|afternoon|| +54906|AAAAAAAALHGNAAAA|54906|15|15|6|PM|second|afternoon|| +54907|AAAAAAAAMHGNAAAA|54907|15|15|7|PM|second|afternoon|| +54908|AAAAAAAANHGNAAAA|54908|15|15|8|PM|second|afternoon|| +54909|AAAAAAAAOHGNAAAA|54909|15|15|9|PM|second|afternoon|| +54910|AAAAAAAAPHGNAAAA|54910|15|15|10|PM|second|afternoon|| +54911|AAAAAAAAAIGNAAAA|54911|15|15|11|PM|second|afternoon|| +54912|AAAAAAAABIGNAAAA|54912|15|15|12|PM|second|afternoon|| +54913|AAAAAAAACIGNAAAA|54913|15|15|13|PM|second|afternoon|| +54914|AAAAAAAADIGNAAAA|54914|15|15|14|PM|second|afternoon|| +54915|AAAAAAAAEIGNAAAA|54915|15|15|15|PM|second|afternoon|| +54916|AAAAAAAAFIGNAAAA|54916|15|15|16|PM|second|afternoon|| +54917|AAAAAAAAGIGNAAAA|54917|15|15|17|PM|second|afternoon|| +54918|AAAAAAAAHIGNAAAA|54918|15|15|18|PM|second|afternoon|| +54919|AAAAAAAAIIGNAAAA|54919|15|15|19|PM|second|afternoon|| +54920|AAAAAAAAJIGNAAAA|54920|15|15|20|PM|second|afternoon|| +54921|AAAAAAAAKIGNAAAA|54921|15|15|21|PM|second|afternoon|| +54922|AAAAAAAALIGNAAAA|54922|15|15|22|PM|second|afternoon|| +54923|AAAAAAAAMIGNAAAA|54923|15|15|23|PM|second|afternoon|| +54924|AAAAAAAANIGNAAAA|54924|15|15|24|PM|second|afternoon|| +54925|AAAAAAAAOIGNAAAA|54925|15|15|25|PM|second|afternoon|| +54926|AAAAAAAAPIGNAAAA|54926|15|15|26|PM|second|afternoon|| +54927|AAAAAAAAAJGNAAAA|54927|15|15|27|PM|second|afternoon|| +54928|AAAAAAAABJGNAAAA|54928|15|15|28|PM|second|afternoon|| +54929|AAAAAAAACJGNAAAA|54929|15|15|29|PM|second|afternoon|| +54930|AAAAAAAADJGNAAAA|54930|15|15|30|PM|second|afternoon|| +54931|AAAAAAAAEJGNAAAA|54931|15|15|31|PM|second|afternoon|| +54932|AAAAAAAAFJGNAAAA|54932|15|15|32|PM|second|afternoon|| +54933|AAAAAAAAGJGNAAAA|54933|15|15|33|PM|second|afternoon|| +54934|AAAAAAAAHJGNAAAA|54934|15|15|34|PM|second|afternoon|| +54935|AAAAAAAAIJGNAAAA|54935|15|15|35|PM|second|afternoon|| +54936|AAAAAAAAJJGNAAAA|54936|15|15|36|PM|second|afternoon|| +54937|AAAAAAAAKJGNAAAA|54937|15|15|37|PM|second|afternoon|| +54938|AAAAAAAALJGNAAAA|54938|15|15|38|PM|second|afternoon|| +54939|AAAAAAAAMJGNAAAA|54939|15|15|39|PM|second|afternoon|| +54940|AAAAAAAANJGNAAAA|54940|15|15|40|PM|second|afternoon|| +54941|AAAAAAAAOJGNAAAA|54941|15|15|41|PM|second|afternoon|| +54942|AAAAAAAAPJGNAAAA|54942|15|15|42|PM|second|afternoon|| +54943|AAAAAAAAAKGNAAAA|54943|15|15|43|PM|second|afternoon|| +54944|AAAAAAAABKGNAAAA|54944|15|15|44|PM|second|afternoon|| +54945|AAAAAAAACKGNAAAA|54945|15|15|45|PM|second|afternoon|| +54946|AAAAAAAADKGNAAAA|54946|15|15|46|PM|second|afternoon|| +54947|AAAAAAAAEKGNAAAA|54947|15|15|47|PM|second|afternoon|| +54948|AAAAAAAAFKGNAAAA|54948|15|15|48|PM|second|afternoon|| +54949|AAAAAAAAGKGNAAAA|54949|15|15|49|PM|second|afternoon|| +54950|AAAAAAAAHKGNAAAA|54950|15|15|50|PM|second|afternoon|| +54951|AAAAAAAAIKGNAAAA|54951|15|15|51|PM|second|afternoon|| +54952|AAAAAAAAJKGNAAAA|54952|15|15|52|PM|second|afternoon|| +54953|AAAAAAAAKKGNAAAA|54953|15|15|53|PM|second|afternoon|| +54954|AAAAAAAALKGNAAAA|54954|15|15|54|PM|second|afternoon|| +54955|AAAAAAAAMKGNAAAA|54955|15|15|55|PM|second|afternoon|| +54956|AAAAAAAANKGNAAAA|54956|15|15|56|PM|second|afternoon|| +54957|AAAAAAAAOKGNAAAA|54957|15|15|57|PM|second|afternoon|| +54958|AAAAAAAAPKGNAAAA|54958|15|15|58|PM|second|afternoon|| +54959|AAAAAAAAALGNAAAA|54959|15|15|59|PM|second|afternoon|| +54960|AAAAAAAABLGNAAAA|54960|15|16|0|PM|second|afternoon|| +54961|AAAAAAAACLGNAAAA|54961|15|16|1|PM|second|afternoon|| +54962|AAAAAAAADLGNAAAA|54962|15|16|2|PM|second|afternoon|| +54963|AAAAAAAAELGNAAAA|54963|15|16|3|PM|second|afternoon|| +54964|AAAAAAAAFLGNAAAA|54964|15|16|4|PM|second|afternoon|| +54965|AAAAAAAAGLGNAAAA|54965|15|16|5|PM|second|afternoon|| +54966|AAAAAAAAHLGNAAAA|54966|15|16|6|PM|second|afternoon|| +54967|AAAAAAAAILGNAAAA|54967|15|16|7|PM|second|afternoon|| +54968|AAAAAAAAJLGNAAAA|54968|15|16|8|PM|second|afternoon|| +54969|AAAAAAAAKLGNAAAA|54969|15|16|9|PM|second|afternoon|| +54970|AAAAAAAALLGNAAAA|54970|15|16|10|PM|second|afternoon|| +54971|AAAAAAAAMLGNAAAA|54971|15|16|11|PM|second|afternoon|| +54972|AAAAAAAANLGNAAAA|54972|15|16|12|PM|second|afternoon|| +54973|AAAAAAAAOLGNAAAA|54973|15|16|13|PM|second|afternoon|| +54974|AAAAAAAAPLGNAAAA|54974|15|16|14|PM|second|afternoon|| +54975|AAAAAAAAAMGNAAAA|54975|15|16|15|PM|second|afternoon|| +54976|AAAAAAAABMGNAAAA|54976|15|16|16|PM|second|afternoon|| +54977|AAAAAAAACMGNAAAA|54977|15|16|17|PM|second|afternoon|| +54978|AAAAAAAADMGNAAAA|54978|15|16|18|PM|second|afternoon|| +54979|AAAAAAAAEMGNAAAA|54979|15|16|19|PM|second|afternoon|| +54980|AAAAAAAAFMGNAAAA|54980|15|16|20|PM|second|afternoon|| +54981|AAAAAAAAGMGNAAAA|54981|15|16|21|PM|second|afternoon|| +54982|AAAAAAAAHMGNAAAA|54982|15|16|22|PM|second|afternoon|| +54983|AAAAAAAAIMGNAAAA|54983|15|16|23|PM|second|afternoon|| +54984|AAAAAAAAJMGNAAAA|54984|15|16|24|PM|second|afternoon|| +54985|AAAAAAAAKMGNAAAA|54985|15|16|25|PM|second|afternoon|| +54986|AAAAAAAALMGNAAAA|54986|15|16|26|PM|second|afternoon|| +54987|AAAAAAAAMMGNAAAA|54987|15|16|27|PM|second|afternoon|| +54988|AAAAAAAANMGNAAAA|54988|15|16|28|PM|second|afternoon|| +54989|AAAAAAAAOMGNAAAA|54989|15|16|29|PM|second|afternoon|| +54990|AAAAAAAAPMGNAAAA|54990|15|16|30|PM|second|afternoon|| +54991|AAAAAAAAANGNAAAA|54991|15|16|31|PM|second|afternoon|| +54992|AAAAAAAABNGNAAAA|54992|15|16|32|PM|second|afternoon|| +54993|AAAAAAAACNGNAAAA|54993|15|16|33|PM|second|afternoon|| +54994|AAAAAAAADNGNAAAA|54994|15|16|34|PM|second|afternoon|| +54995|AAAAAAAAENGNAAAA|54995|15|16|35|PM|second|afternoon|| +54996|AAAAAAAAFNGNAAAA|54996|15|16|36|PM|second|afternoon|| +54997|AAAAAAAAGNGNAAAA|54997|15|16|37|PM|second|afternoon|| +54998|AAAAAAAAHNGNAAAA|54998|15|16|38|PM|second|afternoon|| +54999|AAAAAAAAINGNAAAA|54999|15|16|39|PM|second|afternoon|| +55000|AAAAAAAAJNGNAAAA|55000|15|16|40|PM|second|afternoon|| +55001|AAAAAAAAKNGNAAAA|55001|15|16|41|PM|second|afternoon|| +55002|AAAAAAAALNGNAAAA|55002|15|16|42|PM|second|afternoon|| +55003|AAAAAAAAMNGNAAAA|55003|15|16|43|PM|second|afternoon|| +55004|AAAAAAAANNGNAAAA|55004|15|16|44|PM|second|afternoon|| +55005|AAAAAAAAONGNAAAA|55005|15|16|45|PM|second|afternoon|| +55006|AAAAAAAAPNGNAAAA|55006|15|16|46|PM|second|afternoon|| +55007|AAAAAAAAAOGNAAAA|55007|15|16|47|PM|second|afternoon|| +55008|AAAAAAAABOGNAAAA|55008|15|16|48|PM|second|afternoon|| +55009|AAAAAAAACOGNAAAA|55009|15|16|49|PM|second|afternoon|| +55010|AAAAAAAADOGNAAAA|55010|15|16|50|PM|second|afternoon|| +55011|AAAAAAAAEOGNAAAA|55011|15|16|51|PM|second|afternoon|| +55012|AAAAAAAAFOGNAAAA|55012|15|16|52|PM|second|afternoon|| +55013|AAAAAAAAGOGNAAAA|55013|15|16|53|PM|second|afternoon|| +55014|AAAAAAAAHOGNAAAA|55014|15|16|54|PM|second|afternoon|| +55015|AAAAAAAAIOGNAAAA|55015|15|16|55|PM|second|afternoon|| +55016|AAAAAAAAJOGNAAAA|55016|15|16|56|PM|second|afternoon|| +55017|AAAAAAAAKOGNAAAA|55017|15|16|57|PM|second|afternoon|| +55018|AAAAAAAALOGNAAAA|55018|15|16|58|PM|second|afternoon|| +55019|AAAAAAAAMOGNAAAA|55019|15|16|59|PM|second|afternoon|| +55020|AAAAAAAANOGNAAAA|55020|15|17|0|PM|second|afternoon|| +55021|AAAAAAAAOOGNAAAA|55021|15|17|1|PM|second|afternoon|| +55022|AAAAAAAAPOGNAAAA|55022|15|17|2|PM|second|afternoon|| +55023|AAAAAAAAAPGNAAAA|55023|15|17|3|PM|second|afternoon|| +55024|AAAAAAAABPGNAAAA|55024|15|17|4|PM|second|afternoon|| +55025|AAAAAAAACPGNAAAA|55025|15|17|5|PM|second|afternoon|| +55026|AAAAAAAADPGNAAAA|55026|15|17|6|PM|second|afternoon|| +55027|AAAAAAAAEPGNAAAA|55027|15|17|7|PM|second|afternoon|| +55028|AAAAAAAAFPGNAAAA|55028|15|17|8|PM|second|afternoon|| +55029|AAAAAAAAGPGNAAAA|55029|15|17|9|PM|second|afternoon|| +55030|AAAAAAAAHPGNAAAA|55030|15|17|10|PM|second|afternoon|| +55031|AAAAAAAAIPGNAAAA|55031|15|17|11|PM|second|afternoon|| +55032|AAAAAAAAJPGNAAAA|55032|15|17|12|PM|second|afternoon|| +55033|AAAAAAAAKPGNAAAA|55033|15|17|13|PM|second|afternoon|| +55034|AAAAAAAALPGNAAAA|55034|15|17|14|PM|second|afternoon|| +55035|AAAAAAAAMPGNAAAA|55035|15|17|15|PM|second|afternoon|| +55036|AAAAAAAANPGNAAAA|55036|15|17|16|PM|second|afternoon|| +55037|AAAAAAAAOPGNAAAA|55037|15|17|17|PM|second|afternoon|| +55038|AAAAAAAAPPGNAAAA|55038|15|17|18|PM|second|afternoon|| +55039|AAAAAAAAAAHNAAAA|55039|15|17|19|PM|second|afternoon|| +55040|AAAAAAAABAHNAAAA|55040|15|17|20|PM|second|afternoon|| +55041|AAAAAAAACAHNAAAA|55041|15|17|21|PM|second|afternoon|| +55042|AAAAAAAADAHNAAAA|55042|15|17|22|PM|second|afternoon|| +55043|AAAAAAAAEAHNAAAA|55043|15|17|23|PM|second|afternoon|| +55044|AAAAAAAAFAHNAAAA|55044|15|17|24|PM|second|afternoon|| +55045|AAAAAAAAGAHNAAAA|55045|15|17|25|PM|second|afternoon|| +55046|AAAAAAAAHAHNAAAA|55046|15|17|26|PM|second|afternoon|| +55047|AAAAAAAAIAHNAAAA|55047|15|17|27|PM|second|afternoon|| +55048|AAAAAAAAJAHNAAAA|55048|15|17|28|PM|second|afternoon|| +55049|AAAAAAAAKAHNAAAA|55049|15|17|29|PM|second|afternoon|| +55050|AAAAAAAALAHNAAAA|55050|15|17|30|PM|second|afternoon|| +55051|AAAAAAAAMAHNAAAA|55051|15|17|31|PM|second|afternoon|| +55052|AAAAAAAANAHNAAAA|55052|15|17|32|PM|second|afternoon|| +55053|AAAAAAAAOAHNAAAA|55053|15|17|33|PM|second|afternoon|| +55054|AAAAAAAAPAHNAAAA|55054|15|17|34|PM|second|afternoon|| +55055|AAAAAAAAABHNAAAA|55055|15|17|35|PM|second|afternoon|| +55056|AAAAAAAABBHNAAAA|55056|15|17|36|PM|second|afternoon|| +55057|AAAAAAAACBHNAAAA|55057|15|17|37|PM|second|afternoon|| +55058|AAAAAAAADBHNAAAA|55058|15|17|38|PM|second|afternoon|| +55059|AAAAAAAAEBHNAAAA|55059|15|17|39|PM|second|afternoon|| +55060|AAAAAAAAFBHNAAAA|55060|15|17|40|PM|second|afternoon|| +55061|AAAAAAAAGBHNAAAA|55061|15|17|41|PM|second|afternoon|| +55062|AAAAAAAAHBHNAAAA|55062|15|17|42|PM|second|afternoon|| +55063|AAAAAAAAIBHNAAAA|55063|15|17|43|PM|second|afternoon|| +55064|AAAAAAAAJBHNAAAA|55064|15|17|44|PM|second|afternoon|| +55065|AAAAAAAAKBHNAAAA|55065|15|17|45|PM|second|afternoon|| +55066|AAAAAAAALBHNAAAA|55066|15|17|46|PM|second|afternoon|| +55067|AAAAAAAAMBHNAAAA|55067|15|17|47|PM|second|afternoon|| +55068|AAAAAAAANBHNAAAA|55068|15|17|48|PM|second|afternoon|| +55069|AAAAAAAAOBHNAAAA|55069|15|17|49|PM|second|afternoon|| +55070|AAAAAAAAPBHNAAAA|55070|15|17|50|PM|second|afternoon|| +55071|AAAAAAAAACHNAAAA|55071|15|17|51|PM|second|afternoon|| +55072|AAAAAAAABCHNAAAA|55072|15|17|52|PM|second|afternoon|| +55073|AAAAAAAACCHNAAAA|55073|15|17|53|PM|second|afternoon|| +55074|AAAAAAAADCHNAAAA|55074|15|17|54|PM|second|afternoon|| +55075|AAAAAAAAECHNAAAA|55075|15|17|55|PM|second|afternoon|| +55076|AAAAAAAAFCHNAAAA|55076|15|17|56|PM|second|afternoon|| +55077|AAAAAAAAGCHNAAAA|55077|15|17|57|PM|second|afternoon|| +55078|AAAAAAAAHCHNAAAA|55078|15|17|58|PM|second|afternoon|| +55079|AAAAAAAAICHNAAAA|55079|15|17|59|PM|second|afternoon|| +55080|AAAAAAAAJCHNAAAA|55080|15|18|0|PM|second|afternoon|| +55081|AAAAAAAAKCHNAAAA|55081|15|18|1|PM|second|afternoon|| +55082|AAAAAAAALCHNAAAA|55082|15|18|2|PM|second|afternoon|| +55083|AAAAAAAAMCHNAAAA|55083|15|18|3|PM|second|afternoon|| +55084|AAAAAAAANCHNAAAA|55084|15|18|4|PM|second|afternoon|| +55085|AAAAAAAAOCHNAAAA|55085|15|18|5|PM|second|afternoon|| +55086|AAAAAAAAPCHNAAAA|55086|15|18|6|PM|second|afternoon|| +55087|AAAAAAAAADHNAAAA|55087|15|18|7|PM|second|afternoon|| +55088|AAAAAAAABDHNAAAA|55088|15|18|8|PM|second|afternoon|| +55089|AAAAAAAACDHNAAAA|55089|15|18|9|PM|second|afternoon|| +55090|AAAAAAAADDHNAAAA|55090|15|18|10|PM|second|afternoon|| +55091|AAAAAAAAEDHNAAAA|55091|15|18|11|PM|second|afternoon|| +55092|AAAAAAAAFDHNAAAA|55092|15|18|12|PM|second|afternoon|| +55093|AAAAAAAAGDHNAAAA|55093|15|18|13|PM|second|afternoon|| +55094|AAAAAAAAHDHNAAAA|55094|15|18|14|PM|second|afternoon|| +55095|AAAAAAAAIDHNAAAA|55095|15|18|15|PM|second|afternoon|| +55096|AAAAAAAAJDHNAAAA|55096|15|18|16|PM|second|afternoon|| +55097|AAAAAAAAKDHNAAAA|55097|15|18|17|PM|second|afternoon|| +55098|AAAAAAAALDHNAAAA|55098|15|18|18|PM|second|afternoon|| +55099|AAAAAAAAMDHNAAAA|55099|15|18|19|PM|second|afternoon|| +55100|AAAAAAAANDHNAAAA|55100|15|18|20|PM|second|afternoon|| +55101|AAAAAAAAODHNAAAA|55101|15|18|21|PM|second|afternoon|| +55102|AAAAAAAAPDHNAAAA|55102|15|18|22|PM|second|afternoon|| +55103|AAAAAAAAAEHNAAAA|55103|15|18|23|PM|second|afternoon|| +55104|AAAAAAAABEHNAAAA|55104|15|18|24|PM|second|afternoon|| +55105|AAAAAAAACEHNAAAA|55105|15|18|25|PM|second|afternoon|| +55106|AAAAAAAADEHNAAAA|55106|15|18|26|PM|second|afternoon|| +55107|AAAAAAAAEEHNAAAA|55107|15|18|27|PM|second|afternoon|| +55108|AAAAAAAAFEHNAAAA|55108|15|18|28|PM|second|afternoon|| +55109|AAAAAAAAGEHNAAAA|55109|15|18|29|PM|second|afternoon|| +55110|AAAAAAAAHEHNAAAA|55110|15|18|30|PM|second|afternoon|| +55111|AAAAAAAAIEHNAAAA|55111|15|18|31|PM|second|afternoon|| +55112|AAAAAAAAJEHNAAAA|55112|15|18|32|PM|second|afternoon|| +55113|AAAAAAAAKEHNAAAA|55113|15|18|33|PM|second|afternoon|| +55114|AAAAAAAALEHNAAAA|55114|15|18|34|PM|second|afternoon|| +55115|AAAAAAAAMEHNAAAA|55115|15|18|35|PM|second|afternoon|| +55116|AAAAAAAANEHNAAAA|55116|15|18|36|PM|second|afternoon|| +55117|AAAAAAAAOEHNAAAA|55117|15|18|37|PM|second|afternoon|| +55118|AAAAAAAAPEHNAAAA|55118|15|18|38|PM|second|afternoon|| +55119|AAAAAAAAAFHNAAAA|55119|15|18|39|PM|second|afternoon|| +55120|AAAAAAAABFHNAAAA|55120|15|18|40|PM|second|afternoon|| +55121|AAAAAAAACFHNAAAA|55121|15|18|41|PM|second|afternoon|| +55122|AAAAAAAADFHNAAAA|55122|15|18|42|PM|second|afternoon|| +55123|AAAAAAAAEFHNAAAA|55123|15|18|43|PM|second|afternoon|| +55124|AAAAAAAAFFHNAAAA|55124|15|18|44|PM|second|afternoon|| +55125|AAAAAAAAGFHNAAAA|55125|15|18|45|PM|second|afternoon|| +55126|AAAAAAAAHFHNAAAA|55126|15|18|46|PM|second|afternoon|| +55127|AAAAAAAAIFHNAAAA|55127|15|18|47|PM|second|afternoon|| +55128|AAAAAAAAJFHNAAAA|55128|15|18|48|PM|second|afternoon|| +55129|AAAAAAAAKFHNAAAA|55129|15|18|49|PM|second|afternoon|| +55130|AAAAAAAALFHNAAAA|55130|15|18|50|PM|second|afternoon|| +55131|AAAAAAAAMFHNAAAA|55131|15|18|51|PM|second|afternoon|| +55132|AAAAAAAANFHNAAAA|55132|15|18|52|PM|second|afternoon|| +55133|AAAAAAAAOFHNAAAA|55133|15|18|53|PM|second|afternoon|| +55134|AAAAAAAAPFHNAAAA|55134|15|18|54|PM|second|afternoon|| +55135|AAAAAAAAAGHNAAAA|55135|15|18|55|PM|second|afternoon|| +55136|AAAAAAAABGHNAAAA|55136|15|18|56|PM|second|afternoon|| +55137|AAAAAAAACGHNAAAA|55137|15|18|57|PM|second|afternoon|| +55138|AAAAAAAADGHNAAAA|55138|15|18|58|PM|second|afternoon|| +55139|AAAAAAAAEGHNAAAA|55139|15|18|59|PM|second|afternoon|| +55140|AAAAAAAAFGHNAAAA|55140|15|19|0|PM|second|afternoon|| +55141|AAAAAAAAGGHNAAAA|55141|15|19|1|PM|second|afternoon|| +55142|AAAAAAAAHGHNAAAA|55142|15|19|2|PM|second|afternoon|| +55143|AAAAAAAAIGHNAAAA|55143|15|19|3|PM|second|afternoon|| +55144|AAAAAAAAJGHNAAAA|55144|15|19|4|PM|second|afternoon|| +55145|AAAAAAAAKGHNAAAA|55145|15|19|5|PM|second|afternoon|| +55146|AAAAAAAALGHNAAAA|55146|15|19|6|PM|second|afternoon|| +55147|AAAAAAAAMGHNAAAA|55147|15|19|7|PM|second|afternoon|| +55148|AAAAAAAANGHNAAAA|55148|15|19|8|PM|second|afternoon|| +55149|AAAAAAAAOGHNAAAA|55149|15|19|9|PM|second|afternoon|| +55150|AAAAAAAAPGHNAAAA|55150|15|19|10|PM|second|afternoon|| +55151|AAAAAAAAAHHNAAAA|55151|15|19|11|PM|second|afternoon|| +55152|AAAAAAAABHHNAAAA|55152|15|19|12|PM|second|afternoon|| +55153|AAAAAAAACHHNAAAA|55153|15|19|13|PM|second|afternoon|| +55154|AAAAAAAADHHNAAAA|55154|15|19|14|PM|second|afternoon|| +55155|AAAAAAAAEHHNAAAA|55155|15|19|15|PM|second|afternoon|| +55156|AAAAAAAAFHHNAAAA|55156|15|19|16|PM|second|afternoon|| +55157|AAAAAAAAGHHNAAAA|55157|15|19|17|PM|second|afternoon|| +55158|AAAAAAAAHHHNAAAA|55158|15|19|18|PM|second|afternoon|| +55159|AAAAAAAAIHHNAAAA|55159|15|19|19|PM|second|afternoon|| +55160|AAAAAAAAJHHNAAAA|55160|15|19|20|PM|second|afternoon|| +55161|AAAAAAAAKHHNAAAA|55161|15|19|21|PM|second|afternoon|| +55162|AAAAAAAALHHNAAAA|55162|15|19|22|PM|second|afternoon|| +55163|AAAAAAAAMHHNAAAA|55163|15|19|23|PM|second|afternoon|| +55164|AAAAAAAANHHNAAAA|55164|15|19|24|PM|second|afternoon|| +55165|AAAAAAAAOHHNAAAA|55165|15|19|25|PM|second|afternoon|| +55166|AAAAAAAAPHHNAAAA|55166|15|19|26|PM|second|afternoon|| +55167|AAAAAAAAAIHNAAAA|55167|15|19|27|PM|second|afternoon|| +55168|AAAAAAAABIHNAAAA|55168|15|19|28|PM|second|afternoon|| +55169|AAAAAAAACIHNAAAA|55169|15|19|29|PM|second|afternoon|| +55170|AAAAAAAADIHNAAAA|55170|15|19|30|PM|second|afternoon|| +55171|AAAAAAAAEIHNAAAA|55171|15|19|31|PM|second|afternoon|| +55172|AAAAAAAAFIHNAAAA|55172|15|19|32|PM|second|afternoon|| +55173|AAAAAAAAGIHNAAAA|55173|15|19|33|PM|second|afternoon|| +55174|AAAAAAAAHIHNAAAA|55174|15|19|34|PM|second|afternoon|| +55175|AAAAAAAAIIHNAAAA|55175|15|19|35|PM|second|afternoon|| +55176|AAAAAAAAJIHNAAAA|55176|15|19|36|PM|second|afternoon|| +55177|AAAAAAAAKIHNAAAA|55177|15|19|37|PM|second|afternoon|| +55178|AAAAAAAALIHNAAAA|55178|15|19|38|PM|second|afternoon|| +55179|AAAAAAAAMIHNAAAA|55179|15|19|39|PM|second|afternoon|| +55180|AAAAAAAANIHNAAAA|55180|15|19|40|PM|second|afternoon|| +55181|AAAAAAAAOIHNAAAA|55181|15|19|41|PM|second|afternoon|| +55182|AAAAAAAAPIHNAAAA|55182|15|19|42|PM|second|afternoon|| +55183|AAAAAAAAAJHNAAAA|55183|15|19|43|PM|second|afternoon|| +55184|AAAAAAAABJHNAAAA|55184|15|19|44|PM|second|afternoon|| +55185|AAAAAAAACJHNAAAA|55185|15|19|45|PM|second|afternoon|| +55186|AAAAAAAADJHNAAAA|55186|15|19|46|PM|second|afternoon|| +55187|AAAAAAAAEJHNAAAA|55187|15|19|47|PM|second|afternoon|| +55188|AAAAAAAAFJHNAAAA|55188|15|19|48|PM|second|afternoon|| +55189|AAAAAAAAGJHNAAAA|55189|15|19|49|PM|second|afternoon|| +55190|AAAAAAAAHJHNAAAA|55190|15|19|50|PM|second|afternoon|| +55191|AAAAAAAAIJHNAAAA|55191|15|19|51|PM|second|afternoon|| +55192|AAAAAAAAJJHNAAAA|55192|15|19|52|PM|second|afternoon|| +55193|AAAAAAAAKJHNAAAA|55193|15|19|53|PM|second|afternoon|| +55194|AAAAAAAALJHNAAAA|55194|15|19|54|PM|second|afternoon|| +55195|AAAAAAAAMJHNAAAA|55195|15|19|55|PM|second|afternoon|| +55196|AAAAAAAANJHNAAAA|55196|15|19|56|PM|second|afternoon|| +55197|AAAAAAAAOJHNAAAA|55197|15|19|57|PM|second|afternoon|| +55198|AAAAAAAAPJHNAAAA|55198|15|19|58|PM|second|afternoon|| +55199|AAAAAAAAAKHNAAAA|55199|15|19|59|PM|second|afternoon|| +55200|AAAAAAAABKHNAAAA|55200|15|20|0|PM|second|afternoon|| +55201|AAAAAAAACKHNAAAA|55201|15|20|1|PM|second|afternoon|| +55202|AAAAAAAADKHNAAAA|55202|15|20|2|PM|second|afternoon|| +55203|AAAAAAAAEKHNAAAA|55203|15|20|3|PM|second|afternoon|| +55204|AAAAAAAAFKHNAAAA|55204|15|20|4|PM|second|afternoon|| +55205|AAAAAAAAGKHNAAAA|55205|15|20|5|PM|second|afternoon|| +55206|AAAAAAAAHKHNAAAA|55206|15|20|6|PM|second|afternoon|| +55207|AAAAAAAAIKHNAAAA|55207|15|20|7|PM|second|afternoon|| +55208|AAAAAAAAJKHNAAAA|55208|15|20|8|PM|second|afternoon|| +55209|AAAAAAAAKKHNAAAA|55209|15|20|9|PM|second|afternoon|| +55210|AAAAAAAALKHNAAAA|55210|15|20|10|PM|second|afternoon|| +55211|AAAAAAAAMKHNAAAA|55211|15|20|11|PM|second|afternoon|| +55212|AAAAAAAANKHNAAAA|55212|15|20|12|PM|second|afternoon|| +55213|AAAAAAAAOKHNAAAA|55213|15|20|13|PM|second|afternoon|| +55214|AAAAAAAAPKHNAAAA|55214|15|20|14|PM|second|afternoon|| +55215|AAAAAAAAALHNAAAA|55215|15|20|15|PM|second|afternoon|| +55216|AAAAAAAABLHNAAAA|55216|15|20|16|PM|second|afternoon|| +55217|AAAAAAAACLHNAAAA|55217|15|20|17|PM|second|afternoon|| +55218|AAAAAAAADLHNAAAA|55218|15|20|18|PM|second|afternoon|| +55219|AAAAAAAAELHNAAAA|55219|15|20|19|PM|second|afternoon|| +55220|AAAAAAAAFLHNAAAA|55220|15|20|20|PM|second|afternoon|| +55221|AAAAAAAAGLHNAAAA|55221|15|20|21|PM|second|afternoon|| +55222|AAAAAAAAHLHNAAAA|55222|15|20|22|PM|second|afternoon|| +55223|AAAAAAAAILHNAAAA|55223|15|20|23|PM|second|afternoon|| +55224|AAAAAAAAJLHNAAAA|55224|15|20|24|PM|second|afternoon|| +55225|AAAAAAAAKLHNAAAA|55225|15|20|25|PM|second|afternoon|| +55226|AAAAAAAALLHNAAAA|55226|15|20|26|PM|second|afternoon|| +55227|AAAAAAAAMLHNAAAA|55227|15|20|27|PM|second|afternoon|| +55228|AAAAAAAANLHNAAAA|55228|15|20|28|PM|second|afternoon|| +55229|AAAAAAAAOLHNAAAA|55229|15|20|29|PM|second|afternoon|| +55230|AAAAAAAAPLHNAAAA|55230|15|20|30|PM|second|afternoon|| +55231|AAAAAAAAAMHNAAAA|55231|15|20|31|PM|second|afternoon|| +55232|AAAAAAAABMHNAAAA|55232|15|20|32|PM|second|afternoon|| +55233|AAAAAAAACMHNAAAA|55233|15|20|33|PM|second|afternoon|| +55234|AAAAAAAADMHNAAAA|55234|15|20|34|PM|second|afternoon|| +55235|AAAAAAAAEMHNAAAA|55235|15|20|35|PM|second|afternoon|| +55236|AAAAAAAAFMHNAAAA|55236|15|20|36|PM|second|afternoon|| +55237|AAAAAAAAGMHNAAAA|55237|15|20|37|PM|second|afternoon|| +55238|AAAAAAAAHMHNAAAA|55238|15|20|38|PM|second|afternoon|| +55239|AAAAAAAAIMHNAAAA|55239|15|20|39|PM|second|afternoon|| +55240|AAAAAAAAJMHNAAAA|55240|15|20|40|PM|second|afternoon|| +55241|AAAAAAAAKMHNAAAA|55241|15|20|41|PM|second|afternoon|| +55242|AAAAAAAALMHNAAAA|55242|15|20|42|PM|second|afternoon|| +55243|AAAAAAAAMMHNAAAA|55243|15|20|43|PM|second|afternoon|| +55244|AAAAAAAANMHNAAAA|55244|15|20|44|PM|second|afternoon|| +55245|AAAAAAAAOMHNAAAA|55245|15|20|45|PM|second|afternoon|| +55246|AAAAAAAAPMHNAAAA|55246|15|20|46|PM|second|afternoon|| +55247|AAAAAAAAANHNAAAA|55247|15|20|47|PM|second|afternoon|| +55248|AAAAAAAABNHNAAAA|55248|15|20|48|PM|second|afternoon|| +55249|AAAAAAAACNHNAAAA|55249|15|20|49|PM|second|afternoon|| +55250|AAAAAAAADNHNAAAA|55250|15|20|50|PM|second|afternoon|| +55251|AAAAAAAAENHNAAAA|55251|15|20|51|PM|second|afternoon|| +55252|AAAAAAAAFNHNAAAA|55252|15|20|52|PM|second|afternoon|| +55253|AAAAAAAAGNHNAAAA|55253|15|20|53|PM|second|afternoon|| +55254|AAAAAAAAHNHNAAAA|55254|15|20|54|PM|second|afternoon|| +55255|AAAAAAAAINHNAAAA|55255|15|20|55|PM|second|afternoon|| +55256|AAAAAAAAJNHNAAAA|55256|15|20|56|PM|second|afternoon|| +55257|AAAAAAAAKNHNAAAA|55257|15|20|57|PM|second|afternoon|| +55258|AAAAAAAALNHNAAAA|55258|15|20|58|PM|second|afternoon|| +55259|AAAAAAAAMNHNAAAA|55259|15|20|59|PM|second|afternoon|| +55260|AAAAAAAANNHNAAAA|55260|15|21|0|PM|second|afternoon|| +55261|AAAAAAAAONHNAAAA|55261|15|21|1|PM|second|afternoon|| +55262|AAAAAAAAPNHNAAAA|55262|15|21|2|PM|second|afternoon|| +55263|AAAAAAAAAOHNAAAA|55263|15|21|3|PM|second|afternoon|| +55264|AAAAAAAABOHNAAAA|55264|15|21|4|PM|second|afternoon|| +55265|AAAAAAAACOHNAAAA|55265|15|21|5|PM|second|afternoon|| +55266|AAAAAAAADOHNAAAA|55266|15|21|6|PM|second|afternoon|| +55267|AAAAAAAAEOHNAAAA|55267|15|21|7|PM|second|afternoon|| +55268|AAAAAAAAFOHNAAAA|55268|15|21|8|PM|second|afternoon|| +55269|AAAAAAAAGOHNAAAA|55269|15|21|9|PM|second|afternoon|| +55270|AAAAAAAAHOHNAAAA|55270|15|21|10|PM|second|afternoon|| +55271|AAAAAAAAIOHNAAAA|55271|15|21|11|PM|second|afternoon|| +55272|AAAAAAAAJOHNAAAA|55272|15|21|12|PM|second|afternoon|| +55273|AAAAAAAAKOHNAAAA|55273|15|21|13|PM|second|afternoon|| +55274|AAAAAAAALOHNAAAA|55274|15|21|14|PM|second|afternoon|| +55275|AAAAAAAAMOHNAAAA|55275|15|21|15|PM|second|afternoon|| +55276|AAAAAAAANOHNAAAA|55276|15|21|16|PM|second|afternoon|| +55277|AAAAAAAAOOHNAAAA|55277|15|21|17|PM|second|afternoon|| +55278|AAAAAAAAPOHNAAAA|55278|15|21|18|PM|second|afternoon|| +55279|AAAAAAAAAPHNAAAA|55279|15|21|19|PM|second|afternoon|| +55280|AAAAAAAABPHNAAAA|55280|15|21|20|PM|second|afternoon|| +55281|AAAAAAAACPHNAAAA|55281|15|21|21|PM|second|afternoon|| +55282|AAAAAAAADPHNAAAA|55282|15|21|22|PM|second|afternoon|| +55283|AAAAAAAAEPHNAAAA|55283|15|21|23|PM|second|afternoon|| +55284|AAAAAAAAFPHNAAAA|55284|15|21|24|PM|second|afternoon|| +55285|AAAAAAAAGPHNAAAA|55285|15|21|25|PM|second|afternoon|| +55286|AAAAAAAAHPHNAAAA|55286|15|21|26|PM|second|afternoon|| +55287|AAAAAAAAIPHNAAAA|55287|15|21|27|PM|second|afternoon|| +55288|AAAAAAAAJPHNAAAA|55288|15|21|28|PM|second|afternoon|| +55289|AAAAAAAAKPHNAAAA|55289|15|21|29|PM|second|afternoon|| +55290|AAAAAAAALPHNAAAA|55290|15|21|30|PM|second|afternoon|| +55291|AAAAAAAAMPHNAAAA|55291|15|21|31|PM|second|afternoon|| +55292|AAAAAAAANPHNAAAA|55292|15|21|32|PM|second|afternoon|| +55293|AAAAAAAAOPHNAAAA|55293|15|21|33|PM|second|afternoon|| +55294|AAAAAAAAPPHNAAAA|55294|15|21|34|PM|second|afternoon|| +55295|AAAAAAAAAAINAAAA|55295|15|21|35|PM|second|afternoon|| +55296|AAAAAAAABAINAAAA|55296|15|21|36|PM|second|afternoon|| +55297|AAAAAAAACAINAAAA|55297|15|21|37|PM|second|afternoon|| +55298|AAAAAAAADAINAAAA|55298|15|21|38|PM|second|afternoon|| +55299|AAAAAAAAEAINAAAA|55299|15|21|39|PM|second|afternoon|| +55300|AAAAAAAAFAINAAAA|55300|15|21|40|PM|second|afternoon|| +55301|AAAAAAAAGAINAAAA|55301|15|21|41|PM|second|afternoon|| +55302|AAAAAAAAHAINAAAA|55302|15|21|42|PM|second|afternoon|| +55303|AAAAAAAAIAINAAAA|55303|15|21|43|PM|second|afternoon|| +55304|AAAAAAAAJAINAAAA|55304|15|21|44|PM|second|afternoon|| +55305|AAAAAAAAKAINAAAA|55305|15|21|45|PM|second|afternoon|| +55306|AAAAAAAALAINAAAA|55306|15|21|46|PM|second|afternoon|| +55307|AAAAAAAAMAINAAAA|55307|15|21|47|PM|second|afternoon|| +55308|AAAAAAAANAINAAAA|55308|15|21|48|PM|second|afternoon|| +55309|AAAAAAAAOAINAAAA|55309|15|21|49|PM|second|afternoon|| +55310|AAAAAAAAPAINAAAA|55310|15|21|50|PM|second|afternoon|| +55311|AAAAAAAAABINAAAA|55311|15|21|51|PM|second|afternoon|| +55312|AAAAAAAABBINAAAA|55312|15|21|52|PM|second|afternoon|| +55313|AAAAAAAACBINAAAA|55313|15|21|53|PM|second|afternoon|| +55314|AAAAAAAADBINAAAA|55314|15|21|54|PM|second|afternoon|| +55315|AAAAAAAAEBINAAAA|55315|15|21|55|PM|second|afternoon|| +55316|AAAAAAAAFBINAAAA|55316|15|21|56|PM|second|afternoon|| +55317|AAAAAAAAGBINAAAA|55317|15|21|57|PM|second|afternoon|| +55318|AAAAAAAAHBINAAAA|55318|15|21|58|PM|second|afternoon|| +55319|AAAAAAAAIBINAAAA|55319|15|21|59|PM|second|afternoon|| +55320|AAAAAAAAJBINAAAA|55320|15|22|0|PM|second|afternoon|| +55321|AAAAAAAAKBINAAAA|55321|15|22|1|PM|second|afternoon|| +55322|AAAAAAAALBINAAAA|55322|15|22|2|PM|second|afternoon|| +55323|AAAAAAAAMBINAAAA|55323|15|22|3|PM|second|afternoon|| +55324|AAAAAAAANBINAAAA|55324|15|22|4|PM|second|afternoon|| +55325|AAAAAAAAOBINAAAA|55325|15|22|5|PM|second|afternoon|| +55326|AAAAAAAAPBINAAAA|55326|15|22|6|PM|second|afternoon|| +55327|AAAAAAAAACINAAAA|55327|15|22|7|PM|second|afternoon|| +55328|AAAAAAAABCINAAAA|55328|15|22|8|PM|second|afternoon|| +55329|AAAAAAAACCINAAAA|55329|15|22|9|PM|second|afternoon|| +55330|AAAAAAAADCINAAAA|55330|15|22|10|PM|second|afternoon|| +55331|AAAAAAAAECINAAAA|55331|15|22|11|PM|second|afternoon|| +55332|AAAAAAAAFCINAAAA|55332|15|22|12|PM|second|afternoon|| +55333|AAAAAAAAGCINAAAA|55333|15|22|13|PM|second|afternoon|| +55334|AAAAAAAAHCINAAAA|55334|15|22|14|PM|second|afternoon|| +55335|AAAAAAAAICINAAAA|55335|15|22|15|PM|second|afternoon|| +55336|AAAAAAAAJCINAAAA|55336|15|22|16|PM|second|afternoon|| +55337|AAAAAAAAKCINAAAA|55337|15|22|17|PM|second|afternoon|| +55338|AAAAAAAALCINAAAA|55338|15|22|18|PM|second|afternoon|| +55339|AAAAAAAAMCINAAAA|55339|15|22|19|PM|second|afternoon|| +55340|AAAAAAAANCINAAAA|55340|15|22|20|PM|second|afternoon|| +55341|AAAAAAAAOCINAAAA|55341|15|22|21|PM|second|afternoon|| +55342|AAAAAAAAPCINAAAA|55342|15|22|22|PM|second|afternoon|| +55343|AAAAAAAAADINAAAA|55343|15|22|23|PM|second|afternoon|| +55344|AAAAAAAABDINAAAA|55344|15|22|24|PM|second|afternoon|| +55345|AAAAAAAACDINAAAA|55345|15|22|25|PM|second|afternoon|| +55346|AAAAAAAADDINAAAA|55346|15|22|26|PM|second|afternoon|| +55347|AAAAAAAAEDINAAAA|55347|15|22|27|PM|second|afternoon|| +55348|AAAAAAAAFDINAAAA|55348|15|22|28|PM|second|afternoon|| +55349|AAAAAAAAGDINAAAA|55349|15|22|29|PM|second|afternoon|| +55350|AAAAAAAAHDINAAAA|55350|15|22|30|PM|second|afternoon|| +55351|AAAAAAAAIDINAAAA|55351|15|22|31|PM|second|afternoon|| +55352|AAAAAAAAJDINAAAA|55352|15|22|32|PM|second|afternoon|| +55353|AAAAAAAAKDINAAAA|55353|15|22|33|PM|second|afternoon|| +55354|AAAAAAAALDINAAAA|55354|15|22|34|PM|second|afternoon|| +55355|AAAAAAAAMDINAAAA|55355|15|22|35|PM|second|afternoon|| +55356|AAAAAAAANDINAAAA|55356|15|22|36|PM|second|afternoon|| +55357|AAAAAAAAODINAAAA|55357|15|22|37|PM|second|afternoon|| +55358|AAAAAAAAPDINAAAA|55358|15|22|38|PM|second|afternoon|| +55359|AAAAAAAAAEINAAAA|55359|15|22|39|PM|second|afternoon|| +55360|AAAAAAAABEINAAAA|55360|15|22|40|PM|second|afternoon|| +55361|AAAAAAAACEINAAAA|55361|15|22|41|PM|second|afternoon|| +55362|AAAAAAAADEINAAAA|55362|15|22|42|PM|second|afternoon|| +55363|AAAAAAAAEEINAAAA|55363|15|22|43|PM|second|afternoon|| +55364|AAAAAAAAFEINAAAA|55364|15|22|44|PM|second|afternoon|| +55365|AAAAAAAAGEINAAAA|55365|15|22|45|PM|second|afternoon|| +55366|AAAAAAAAHEINAAAA|55366|15|22|46|PM|second|afternoon|| +55367|AAAAAAAAIEINAAAA|55367|15|22|47|PM|second|afternoon|| +55368|AAAAAAAAJEINAAAA|55368|15|22|48|PM|second|afternoon|| +55369|AAAAAAAAKEINAAAA|55369|15|22|49|PM|second|afternoon|| +55370|AAAAAAAALEINAAAA|55370|15|22|50|PM|second|afternoon|| +55371|AAAAAAAAMEINAAAA|55371|15|22|51|PM|second|afternoon|| +55372|AAAAAAAANEINAAAA|55372|15|22|52|PM|second|afternoon|| +55373|AAAAAAAAOEINAAAA|55373|15|22|53|PM|second|afternoon|| +55374|AAAAAAAAPEINAAAA|55374|15|22|54|PM|second|afternoon|| +55375|AAAAAAAAAFINAAAA|55375|15|22|55|PM|second|afternoon|| +55376|AAAAAAAABFINAAAA|55376|15|22|56|PM|second|afternoon|| +55377|AAAAAAAACFINAAAA|55377|15|22|57|PM|second|afternoon|| +55378|AAAAAAAADFINAAAA|55378|15|22|58|PM|second|afternoon|| +55379|AAAAAAAAEFINAAAA|55379|15|22|59|PM|second|afternoon|| +55380|AAAAAAAAFFINAAAA|55380|15|23|0|PM|second|afternoon|| +55381|AAAAAAAAGFINAAAA|55381|15|23|1|PM|second|afternoon|| +55382|AAAAAAAAHFINAAAA|55382|15|23|2|PM|second|afternoon|| +55383|AAAAAAAAIFINAAAA|55383|15|23|3|PM|second|afternoon|| +55384|AAAAAAAAJFINAAAA|55384|15|23|4|PM|second|afternoon|| +55385|AAAAAAAAKFINAAAA|55385|15|23|5|PM|second|afternoon|| +55386|AAAAAAAALFINAAAA|55386|15|23|6|PM|second|afternoon|| +55387|AAAAAAAAMFINAAAA|55387|15|23|7|PM|second|afternoon|| +55388|AAAAAAAANFINAAAA|55388|15|23|8|PM|second|afternoon|| +55389|AAAAAAAAOFINAAAA|55389|15|23|9|PM|second|afternoon|| +55390|AAAAAAAAPFINAAAA|55390|15|23|10|PM|second|afternoon|| +55391|AAAAAAAAAGINAAAA|55391|15|23|11|PM|second|afternoon|| +55392|AAAAAAAABGINAAAA|55392|15|23|12|PM|second|afternoon|| +55393|AAAAAAAACGINAAAA|55393|15|23|13|PM|second|afternoon|| +55394|AAAAAAAADGINAAAA|55394|15|23|14|PM|second|afternoon|| +55395|AAAAAAAAEGINAAAA|55395|15|23|15|PM|second|afternoon|| +55396|AAAAAAAAFGINAAAA|55396|15|23|16|PM|second|afternoon|| +55397|AAAAAAAAGGINAAAA|55397|15|23|17|PM|second|afternoon|| +55398|AAAAAAAAHGINAAAA|55398|15|23|18|PM|second|afternoon|| +55399|AAAAAAAAIGINAAAA|55399|15|23|19|PM|second|afternoon|| +55400|AAAAAAAAJGINAAAA|55400|15|23|20|PM|second|afternoon|| +55401|AAAAAAAAKGINAAAA|55401|15|23|21|PM|second|afternoon|| +55402|AAAAAAAALGINAAAA|55402|15|23|22|PM|second|afternoon|| +55403|AAAAAAAAMGINAAAA|55403|15|23|23|PM|second|afternoon|| +55404|AAAAAAAANGINAAAA|55404|15|23|24|PM|second|afternoon|| +55405|AAAAAAAAOGINAAAA|55405|15|23|25|PM|second|afternoon|| +55406|AAAAAAAAPGINAAAA|55406|15|23|26|PM|second|afternoon|| +55407|AAAAAAAAAHINAAAA|55407|15|23|27|PM|second|afternoon|| +55408|AAAAAAAABHINAAAA|55408|15|23|28|PM|second|afternoon|| +55409|AAAAAAAACHINAAAA|55409|15|23|29|PM|second|afternoon|| +55410|AAAAAAAADHINAAAA|55410|15|23|30|PM|second|afternoon|| +55411|AAAAAAAAEHINAAAA|55411|15|23|31|PM|second|afternoon|| +55412|AAAAAAAAFHINAAAA|55412|15|23|32|PM|second|afternoon|| +55413|AAAAAAAAGHINAAAA|55413|15|23|33|PM|second|afternoon|| +55414|AAAAAAAAHHINAAAA|55414|15|23|34|PM|second|afternoon|| +55415|AAAAAAAAIHINAAAA|55415|15|23|35|PM|second|afternoon|| +55416|AAAAAAAAJHINAAAA|55416|15|23|36|PM|second|afternoon|| +55417|AAAAAAAAKHINAAAA|55417|15|23|37|PM|second|afternoon|| +55418|AAAAAAAALHINAAAA|55418|15|23|38|PM|second|afternoon|| +55419|AAAAAAAAMHINAAAA|55419|15|23|39|PM|second|afternoon|| +55420|AAAAAAAANHINAAAA|55420|15|23|40|PM|second|afternoon|| +55421|AAAAAAAAOHINAAAA|55421|15|23|41|PM|second|afternoon|| +55422|AAAAAAAAPHINAAAA|55422|15|23|42|PM|second|afternoon|| +55423|AAAAAAAAAIINAAAA|55423|15|23|43|PM|second|afternoon|| +55424|AAAAAAAABIINAAAA|55424|15|23|44|PM|second|afternoon|| +55425|AAAAAAAACIINAAAA|55425|15|23|45|PM|second|afternoon|| +55426|AAAAAAAADIINAAAA|55426|15|23|46|PM|second|afternoon|| +55427|AAAAAAAAEIINAAAA|55427|15|23|47|PM|second|afternoon|| +55428|AAAAAAAAFIINAAAA|55428|15|23|48|PM|second|afternoon|| +55429|AAAAAAAAGIINAAAA|55429|15|23|49|PM|second|afternoon|| +55430|AAAAAAAAHIINAAAA|55430|15|23|50|PM|second|afternoon|| +55431|AAAAAAAAIIINAAAA|55431|15|23|51|PM|second|afternoon|| +55432|AAAAAAAAJIINAAAA|55432|15|23|52|PM|second|afternoon|| +55433|AAAAAAAAKIINAAAA|55433|15|23|53|PM|second|afternoon|| +55434|AAAAAAAALIINAAAA|55434|15|23|54|PM|second|afternoon|| +55435|AAAAAAAAMIINAAAA|55435|15|23|55|PM|second|afternoon|| +55436|AAAAAAAANIINAAAA|55436|15|23|56|PM|second|afternoon|| +55437|AAAAAAAAOIINAAAA|55437|15|23|57|PM|second|afternoon|| +55438|AAAAAAAAPIINAAAA|55438|15|23|58|PM|second|afternoon|| +55439|AAAAAAAAAJINAAAA|55439|15|23|59|PM|second|afternoon|| +55440|AAAAAAAABJINAAAA|55440|15|24|0|PM|second|afternoon|| +55441|AAAAAAAACJINAAAA|55441|15|24|1|PM|second|afternoon|| +55442|AAAAAAAADJINAAAA|55442|15|24|2|PM|second|afternoon|| +55443|AAAAAAAAEJINAAAA|55443|15|24|3|PM|second|afternoon|| +55444|AAAAAAAAFJINAAAA|55444|15|24|4|PM|second|afternoon|| +55445|AAAAAAAAGJINAAAA|55445|15|24|5|PM|second|afternoon|| +55446|AAAAAAAAHJINAAAA|55446|15|24|6|PM|second|afternoon|| +55447|AAAAAAAAIJINAAAA|55447|15|24|7|PM|second|afternoon|| +55448|AAAAAAAAJJINAAAA|55448|15|24|8|PM|second|afternoon|| +55449|AAAAAAAAKJINAAAA|55449|15|24|9|PM|second|afternoon|| +55450|AAAAAAAALJINAAAA|55450|15|24|10|PM|second|afternoon|| +55451|AAAAAAAAMJINAAAA|55451|15|24|11|PM|second|afternoon|| +55452|AAAAAAAANJINAAAA|55452|15|24|12|PM|second|afternoon|| +55453|AAAAAAAAOJINAAAA|55453|15|24|13|PM|second|afternoon|| +55454|AAAAAAAAPJINAAAA|55454|15|24|14|PM|second|afternoon|| +55455|AAAAAAAAAKINAAAA|55455|15|24|15|PM|second|afternoon|| +55456|AAAAAAAABKINAAAA|55456|15|24|16|PM|second|afternoon|| +55457|AAAAAAAACKINAAAA|55457|15|24|17|PM|second|afternoon|| +55458|AAAAAAAADKINAAAA|55458|15|24|18|PM|second|afternoon|| +55459|AAAAAAAAEKINAAAA|55459|15|24|19|PM|second|afternoon|| +55460|AAAAAAAAFKINAAAA|55460|15|24|20|PM|second|afternoon|| +55461|AAAAAAAAGKINAAAA|55461|15|24|21|PM|second|afternoon|| +55462|AAAAAAAAHKINAAAA|55462|15|24|22|PM|second|afternoon|| +55463|AAAAAAAAIKINAAAA|55463|15|24|23|PM|second|afternoon|| +55464|AAAAAAAAJKINAAAA|55464|15|24|24|PM|second|afternoon|| +55465|AAAAAAAAKKINAAAA|55465|15|24|25|PM|second|afternoon|| +55466|AAAAAAAALKINAAAA|55466|15|24|26|PM|second|afternoon|| +55467|AAAAAAAAMKINAAAA|55467|15|24|27|PM|second|afternoon|| +55468|AAAAAAAANKINAAAA|55468|15|24|28|PM|second|afternoon|| +55469|AAAAAAAAOKINAAAA|55469|15|24|29|PM|second|afternoon|| +55470|AAAAAAAAPKINAAAA|55470|15|24|30|PM|second|afternoon|| +55471|AAAAAAAAALINAAAA|55471|15|24|31|PM|second|afternoon|| +55472|AAAAAAAABLINAAAA|55472|15|24|32|PM|second|afternoon|| +55473|AAAAAAAACLINAAAA|55473|15|24|33|PM|second|afternoon|| +55474|AAAAAAAADLINAAAA|55474|15|24|34|PM|second|afternoon|| +55475|AAAAAAAAELINAAAA|55475|15|24|35|PM|second|afternoon|| +55476|AAAAAAAAFLINAAAA|55476|15|24|36|PM|second|afternoon|| +55477|AAAAAAAAGLINAAAA|55477|15|24|37|PM|second|afternoon|| +55478|AAAAAAAAHLINAAAA|55478|15|24|38|PM|second|afternoon|| +55479|AAAAAAAAILINAAAA|55479|15|24|39|PM|second|afternoon|| +55480|AAAAAAAAJLINAAAA|55480|15|24|40|PM|second|afternoon|| +55481|AAAAAAAAKLINAAAA|55481|15|24|41|PM|second|afternoon|| +55482|AAAAAAAALLINAAAA|55482|15|24|42|PM|second|afternoon|| +55483|AAAAAAAAMLINAAAA|55483|15|24|43|PM|second|afternoon|| +55484|AAAAAAAANLINAAAA|55484|15|24|44|PM|second|afternoon|| +55485|AAAAAAAAOLINAAAA|55485|15|24|45|PM|second|afternoon|| +55486|AAAAAAAAPLINAAAA|55486|15|24|46|PM|second|afternoon|| +55487|AAAAAAAAAMINAAAA|55487|15|24|47|PM|second|afternoon|| +55488|AAAAAAAABMINAAAA|55488|15|24|48|PM|second|afternoon|| +55489|AAAAAAAACMINAAAA|55489|15|24|49|PM|second|afternoon|| +55490|AAAAAAAADMINAAAA|55490|15|24|50|PM|second|afternoon|| +55491|AAAAAAAAEMINAAAA|55491|15|24|51|PM|second|afternoon|| +55492|AAAAAAAAFMINAAAA|55492|15|24|52|PM|second|afternoon|| +55493|AAAAAAAAGMINAAAA|55493|15|24|53|PM|second|afternoon|| +55494|AAAAAAAAHMINAAAA|55494|15|24|54|PM|second|afternoon|| +55495|AAAAAAAAIMINAAAA|55495|15|24|55|PM|second|afternoon|| +55496|AAAAAAAAJMINAAAA|55496|15|24|56|PM|second|afternoon|| +55497|AAAAAAAAKMINAAAA|55497|15|24|57|PM|second|afternoon|| +55498|AAAAAAAALMINAAAA|55498|15|24|58|PM|second|afternoon|| +55499|AAAAAAAAMMINAAAA|55499|15|24|59|PM|second|afternoon|| +55500|AAAAAAAANMINAAAA|55500|15|25|0|PM|second|afternoon|| +55501|AAAAAAAAOMINAAAA|55501|15|25|1|PM|second|afternoon|| +55502|AAAAAAAAPMINAAAA|55502|15|25|2|PM|second|afternoon|| +55503|AAAAAAAAANINAAAA|55503|15|25|3|PM|second|afternoon|| +55504|AAAAAAAABNINAAAA|55504|15|25|4|PM|second|afternoon|| +55505|AAAAAAAACNINAAAA|55505|15|25|5|PM|second|afternoon|| +55506|AAAAAAAADNINAAAA|55506|15|25|6|PM|second|afternoon|| +55507|AAAAAAAAENINAAAA|55507|15|25|7|PM|second|afternoon|| +55508|AAAAAAAAFNINAAAA|55508|15|25|8|PM|second|afternoon|| +55509|AAAAAAAAGNINAAAA|55509|15|25|9|PM|second|afternoon|| +55510|AAAAAAAAHNINAAAA|55510|15|25|10|PM|second|afternoon|| +55511|AAAAAAAAININAAAA|55511|15|25|11|PM|second|afternoon|| +55512|AAAAAAAAJNINAAAA|55512|15|25|12|PM|second|afternoon|| +55513|AAAAAAAAKNINAAAA|55513|15|25|13|PM|second|afternoon|| +55514|AAAAAAAALNINAAAA|55514|15|25|14|PM|second|afternoon|| +55515|AAAAAAAAMNINAAAA|55515|15|25|15|PM|second|afternoon|| +55516|AAAAAAAANNINAAAA|55516|15|25|16|PM|second|afternoon|| +55517|AAAAAAAAONINAAAA|55517|15|25|17|PM|second|afternoon|| +55518|AAAAAAAAPNINAAAA|55518|15|25|18|PM|second|afternoon|| +55519|AAAAAAAAAOINAAAA|55519|15|25|19|PM|second|afternoon|| +55520|AAAAAAAABOINAAAA|55520|15|25|20|PM|second|afternoon|| +55521|AAAAAAAACOINAAAA|55521|15|25|21|PM|second|afternoon|| +55522|AAAAAAAADOINAAAA|55522|15|25|22|PM|second|afternoon|| +55523|AAAAAAAAEOINAAAA|55523|15|25|23|PM|second|afternoon|| +55524|AAAAAAAAFOINAAAA|55524|15|25|24|PM|second|afternoon|| +55525|AAAAAAAAGOINAAAA|55525|15|25|25|PM|second|afternoon|| +55526|AAAAAAAAHOINAAAA|55526|15|25|26|PM|second|afternoon|| +55527|AAAAAAAAIOINAAAA|55527|15|25|27|PM|second|afternoon|| +55528|AAAAAAAAJOINAAAA|55528|15|25|28|PM|second|afternoon|| +55529|AAAAAAAAKOINAAAA|55529|15|25|29|PM|second|afternoon|| +55530|AAAAAAAALOINAAAA|55530|15|25|30|PM|second|afternoon|| +55531|AAAAAAAAMOINAAAA|55531|15|25|31|PM|second|afternoon|| +55532|AAAAAAAANOINAAAA|55532|15|25|32|PM|second|afternoon|| +55533|AAAAAAAAOOINAAAA|55533|15|25|33|PM|second|afternoon|| +55534|AAAAAAAAPOINAAAA|55534|15|25|34|PM|second|afternoon|| +55535|AAAAAAAAAPINAAAA|55535|15|25|35|PM|second|afternoon|| +55536|AAAAAAAABPINAAAA|55536|15|25|36|PM|second|afternoon|| +55537|AAAAAAAACPINAAAA|55537|15|25|37|PM|second|afternoon|| +55538|AAAAAAAADPINAAAA|55538|15|25|38|PM|second|afternoon|| +55539|AAAAAAAAEPINAAAA|55539|15|25|39|PM|second|afternoon|| +55540|AAAAAAAAFPINAAAA|55540|15|25|40|PM|second|afternoon|| +55541|AAAAAAAAGPINAAAA|55541|15|25|41|PM|second|afternoon|| +55542|AAAAAAAAHPINAAAA|55542|15|25|42|PM|second|afternoon|| +55543|AAAAAAAAIPINAAAA|55543|15|25|43|PM|second|afternoon|| +55544|AAAAAAAAJPINAAAA|55544|15|25|44|PM|second|afternoon|| +55545|AAAAAAAAKPINAAAA|55545|15|25|45|PM|second|afternoon|| +55546|AAAAAAAALPINAAAA|55546|15|25|46|PM|second|afternoon|| +55547|AAAAAAAAMPINAAAA|55547|15|25|47|PM|second|afternoon|| +55548|AAAAAAAANPINAAAA|55548|15|25|48|PM|second|afternoon|| +55549|AAAAAAAAOPINAAAA|55549|15|25|49|PM|second|afternoon|| +55550|AAAAAAAAPPINAAAA|55550|15|25|50|PM|second|afternoon|| +55551|AAAAAAAAAAJNAAAA|55551|15|25|51|PM|second|afternoon|| +55552|AAAAAAAABAJNAAAA|55552|15|25|52|PM|second|afternoon|| +55553|AAAAAAAACAJNAAAA|55553|15|25|53|PM|second|afternoon|| +55554|AAAAAAAADAJNAAAA|55554|15|25|54|PM|second|afternoon|| +55555|AAAAAAAAEAJNAAAA|55555|15|25|55|PM|second|afternoon|| +55556|AAAAAAAAFAJNAAAA|55556|15|25|56|PM|second|afternoon|| +55557|AAAAAAAAGAJNAAAA|55557|15|25|57|PM|second|afternoon|| +55558|AAAAAAAAHAJNAAAA|55558|15|25|58|PM|second|afternoon|| +55559|AAAAAAAAIAJNAAAA|55559|15|25|59|PM|second|afternoon|| +55560|AAAAAAAAJAJNAAAA|55560|15|26|0|PM|second|afternoon|| +55561|AAAAAAAAKAJNAAAA|55561|15|26|1|PM|second|afternoon|| +55562|AAAAAAAALAJNAAAA|55562|15|26|2|PM|second|afternoon|| +55563|AAAAAAAAMAJNAAAA|55563|15|26|3|PM|second|afternoon|| +55564|AAAAAAAANAJNAAAA|55564|15|26|4|PM|second|afternoon|| +55565|AAAAAAAAOAJNAAAA|55565|15|26|5|PM|second|afternoon|| +55566|AAAAAAAAPAJNAAAA|55566|15|26|6|PM|second|afternoon|| +55567|AAAAAAAAABJNAAAA|55567|15|26|7|PM|second|afternoon|| +55568|AAAAAAAABBJNAAAA|55568|15|26|8|PM|second|afternoon|| +55569|AAAAAAAACBJNAAAA|55569|15|26|9|PM|second|afternoon|| +55570|AAAAAAAADBJNAAAA|55570|15|26|10|PM|second|afternoon|| +55571|AAAAAAAAEBJNAAAA|55571|15|26|11|PM|second|afternoon|| +55572|AAAAAAAAFBJNAAAA|55572|15|26|12|PM|second|afternoon|| +55573|AAAAAAAAGBJNAAAA|55573|15|26|13|PM|second|afternoon|| +55574|AAAAAAAAHBJNAAAA|55574|15|26|14|PM|second|afternoon|| +55575|AAAAAAAAIBJNAAAA|55575|15|26|15|PM|second|afternoon|| +55576|AAAAAAAAJBJNAAAA|55576|15|26|16|PM|second|afternoon|| +55577|AAAAAAAAKBJNAAAA|55577|15|26|17|PM|second|afternoon|| +55578|AAAAAAAALBJNAAAA|55578|15|26|18|PM|second|afternoon|| +55579|AAAAAAAAMBJNAAAA|55579|15|26|19|PM|second|afternoon|| +55580|AAAAAAAANBJNAAAA|55580|15|26|20|PM|second|afternoon|| +55581|AAAAAAAAOBJNAAAA|55581|15|26|21|PM|second|afternoon|| +55582|AAAAAAAAPBJNAAAA|55582|15|26|22|PM|second|afternoon|| +55583|AAAAAAAAACJNAAAA|55583|15|26|23|PM|second|afternoon|| +55584|AAAAAAAABCJNAAAA|55584|15|26|24|PM|second|afternoon|| +55585|AAAAAAAACCJNAAAA|55585|15|26|25|PM|second|afternoon|| +55586|AAAAAAAADCJNAAAA|55586|15|26|26|PM|second|afternoon|| +55587|AAAAAAAAECJNAAAA|55587|15|26|27|PM|second|afternoon|| +55588|AAAAAAAAFCJNAAAA|55588|15|26|28|PM|second|afternoon|| +55589|AAAAAAAAGCJNAAAA|55589|15|26|29|PM|second|afternoon|| +55590|AAAAAAAAHCJNAAAA|55590|15|26|30|PM|second|afternoon|| +55591|AAAAAAAAICJNAAAA|55591|15|26|31|PM|second|afternoon|| +55592|AAAAAAAAJCJNAAAA|55592|15|26|32|PM|second|afternoon|| +55593|AAAAAAAAKCJNAAAA|55593|15|26|33|PM|second|afternoon|| +55594|AAAAAAAALCJNAAAA|55594|15|26|34|PM|second|afternoon|| +55595|AAAAAAAAMCJNAAAA|55595|15|26|35|PM|second|afternoon|| +55596|AAAAAAAANCJNAAAA|55596|15|26|36|PM|second|afternoon|| +55597|AAAAAAAAOCJNAAAA|55597|15|26|37|PM|second|afternoon|| +55598|AAAAAAAAPCJNAAAA|55598|15|26|38|PM|second|afternoon|| +55599|AAAAAAAAADJNAAAA|55599|15|26|39|PM|second|afternoon|| +55600|AAAAAAAABDJNAAAA|55600|15|26|40|PM|second|afternoon|| +55601|AAAAAAAACDJNAAAA|55601|15|26|41|PM|second|afternoon|| +55602|AAAAAAAADDJNAAAA|55602|15|26|42|PM|second|afternoon|| +55603|AAAAAAAAEDJNAAAA|55603|15|26|43|PM|second|afternoon|| +55604|AAAAAAAAFDJNAAAA|55604|15|26|44|PM|second|afternoon|| +55605|AAAAAAAAGDJNAAAA|55605|15|26|45|PM|second|afternoon|| +55606|AAAAAAAAHDJNAAAA|55606|15|26|46|PM|second|afternoon|| +55607|AAAAAAAAIDJNAAAA|55607|15|26|47|PM|second|afternoon|| +55608|AAAAAAAAJDJNAAAA|55608|15|26|48|PM|second|afternoon|| +55609|AAAAAAAAKDJNAAAA|55609|15|26|49|PM|second|afternoon|| +55610|AAAAAAAALDJNAAAA|55610|15|26|50|PM|second|afternoon|| +55611|AAAAAAAAMDJNAAAA|55611|15|26|51|PM|second|afternoon|| +55612|AAAAAAAANDJNAAAA|55612|15|26|52|PM|second|afternoon|| +55613|AAAAAAAAODJNAAAA|55613|15|26|53|PM|second|afternoon|| +55614|AAAAAAAAPDJNAAAA|55614|15|26|54|PM|second|afternoon|| +55615|AAAAAAAAAEJNAAAA|55615|15|26|55|PM|second|afternoon|| +55616|AAAAAAAABEJNAAAA|55616|15|26|56|PM|second|afternoon|| +55617|AAAAAAAACEJNAAAA|55617|15|26|57|PM|second|afternoon|| +55618|AAAAAAAADEJNAAAA|55618|15|26|58|PM|second|afternoon|| +55619|AAAAAAAAEEJNAAAA|55619|15|26|59|PM|second|afternoon|| +55620|AAAAAAAAFEJNAAAA|55620|15|27|0|PM|second|afternoon|| +55621|AAAAAAAAGEJNAAAA|55621|15|27|1|PM|second|afternoon|| +55622|AAAAAAAAHEJNAAAA|55622|15|27|2|PM|second|afternoon|| +55623|AAAAAAAAIEJNAAAA|55623|15|27|3|PM|second|afternoon|| +55624|AAAAAAAAJEJNAAAA|55624|15|27|4|PM|second|afternoon|| +55625|AAAAAAAAKEJNAAAA|55625|15|27|5|PM|second|afternoon|| +55626|AAAAAAAALEJNAAAA|55626|15|27|6|PM|second|afternoon|| +55627|AAAAAAAAMEJNAAAA|55627|15|27|7|PM|second|afternoon|| +55628|AAAAAAAANEJNAAAA|55628|15|27|8|PM|second|afternoon|| +55629|AAAAAAAAOEJNAAAA|55629|15|27|9|PM|second|afternoon|| +55630|AAAAAAAAPEJNAAAA|55630|15|27|10|PM|second|afternoon|| +55631|AAAAAAAAAFJNAAAA|55631|15|27|11|PM|second|afternoon|| +55632|AAAAAAAABFJNAAAA|55632|15|27|12|PM|second|afternoon|| +55633|AAAAAAAACFJNAAAA|55633|15|27|13|PM|second|afternoon|| +55634|AAAAAAAADFJNAAAA|55634|15|27|14|PM|second|afternoon|| +55635|AAAAAAAAEFJNAAAA|55635|15|27|15|PM|second|afternoon|| +55636|AAAAAAAAFFJNAAAA|55636|15|27|16|PM|second|afternoon|| +55637|AAAAAAAAGFJNAAAA|55637|15|27|17|PM|second|afternoon|| +55638|AAAAAAAAHFJNAAAA|55638|15|27|18|PM|second|afternoon|| +55639|AAAAAAAAIFJNAAAA|55639|15|27|19|PM|second|afternoon|| +55640|AAAAAAAAJFJNAAAA|55640|15|27|20|PM|second|afternoon|| +55641|AAAAAAAAKFJNAAAA|55641|15|27|21|PM|second|afternoon|| +55642|AAAAAAAALFJNAAAA|55642|15|27|22|PM|second|afternoon|| +55643|AAAAAAAAMFJNAAAA|55643|15|27|23|PM|second|afternoon|| +55644|AAAAAAAANFJNAAAA|55644|15|27|24|PM|second|afternoon|| +55645|AAAAAAAAOFJNAAAA|55645|15|27|25|PM|second|afternoon|| +55646|AAAAAAAAPFJNAAAA|55646|15|27|26|PM|second|afternoon|| +55647|AAAAAAAAAGJNAAAA|55647|15|27|27|PM|second|afternoon|| +55648|AAAAAAAABGJNAAAA|55648|15|27|28|PM|second|afternoon|| +55649|AAAAAAAACGJNAAAA|55649|15|27|29|PM|second|afternoon|| +55650|AAAAAAAADGJNAAAA|55650|15|27|30|PM|second|afternoon|| +55651|AAAAAAAAEGJNAAAA|55651|15|27|31|PM|second|afternoon|| +55652|AAAAAAAAFGJNAAAA|55652|15|27|32|PM|second|afternoon|| +55653|AAAAAAAAGGJNAAAA|55653|15|27|33|PM|second|afternoon|| +55654|AAAAAAAAHGJNAAAA|55654|15|27|34|PM|second|afternoon|| +55655|AAAAAAAAIGJNAAAA|55655|15|27|35|PM|second|afternoon|| +55656|AAAAAAAAJGJNAAAA|55656|15|27|36|PM|second|afternoon|| +55657|AAAAAAAAKGJNAAAA|55657|15|27|37|PM|second|afternoon|| +55658|AAAAAAAALGJNAAAA|55658|15|27|38|PM|second|afternoon|| +55659|AAAAAAAAMGJNAAAA|55659|15|27|39|PM|second|afternoon|| +55660|AAAAAAAANGJNAAAA|55660|15|27|40|PM|second|afternoon|| +55661|AAAAAAAAOGJNAAAA|55661|15|27|41|PM|second|afternoon|| +55662|AAAAAAAAPGJNAAAA|55662|15|27|42|PM|second|afternoon|| +55663|AAAAAAAAAHJNAAAA|55663|15|27|43|PM|second|afternoon|| +55664|AAAAAAAABHJNAAAA|55664|15|27|44|PM|second|afternoon|| +55665|AAAAAAAACHJNAAAA|55665|15|27|45|PM|second|afternoon|| +55666|AAAAAAAADHJNAAAA|55666|15|27|46|PM|second|afternoon|| +55667|AAAAAAAAEHJNAAAA|55667|15|27|47|PM|second|afternoon|| +55668|AAAAAAAAFHJNAAAA|55668|15|27|48|PM|second|afternoon|| +55669|AAAAAAAAGHJNAAAA|55669|15|27|49|PM|second|afternoon|| +55670|AAAAAAAAHHJNAAAA|55670|15|27|50|PM|second|afternoon|| +55671|AAAAAAAAIHJNAAAA|55671|15|27|51|PM|second|afternoon|| +55672|AAAAAAAAJHJNAAAA|55672|15|27|52|PM|second|afternoon|| +55673|AAAAAAAAKHJNAAAA|55673|15|27|53|PM|second|afternoon|| +55674|AAAAAAAALHJNAAAA|55674|15|27|54|PM|second|afternoon|| +55675|AAAAAAAAMHJNAAAA|55675|15|27|55|PM|second|afternoon|| +55676|AAAAAAAANHJNAAAA|55676|15|27|56|PM|second|afternoon|| +55677|AAAAAAAAOHJNAAAA|55677|15|27|57|PM|second|afternoon|| +55678|AAAAAAAAPHJNAAAA|55678|15|27|58|PM|second|afternoon|| +55679|AAAAAAAAAIJNAAAA|55679|15|27|59|PM|second|afternoon|| +55680|AAAAAAAABIJNAAAA|55680|15|28|0|PM|second|afternoon|| +55681|AAAAAAAACIJNAAAA|55681|15|28|1|PM|second|afternoon|| +55682|AAAAAAAADIJNAAAA|55682|15|28|2|PM|second|afternoon|| +55683|AAAAAAAAEIJNAAAA|55683|15|28|3|PM|second|afternoon|| +55684|AAAAAAAAFIJNAAAA|55684|15|28|4|PM|second|afternoon|| +55685|AAAAAAAAGIJNAAAA|55685|15|28|5|PM|second|afternoon|| +55686|AAAAAAAAHIJNAAAA|55686|15|28|6|PM|second|afternoon|| +55687|AAAAAAAAIIJNAAAA|55687|15|28|7|PM|second|afternoon|| +55688|AAAAAAAAJIJNAAAA|55688|15|28|8|PM|second|afternoon|| +55689|AAAAAAAAKIJNAAAA|55689|15|28|9|PM|second|afternoon|| +55690|AAAAAAAALIJNAAAA|55690|15|28|10|PM|second|afternoon|| +55691|AAAAAAAAMIJNAAAA|55691|15|28|11|PM|second|afternoon|| +55692|AAAAAAAANIJNAAAA|55692|15|28|12|PM|second|afternoon|| +55693|AAAAAAAAOIJNAAAA|55693|15|28|13|PM|second|afternoon|| +55694|AAAAAAAAPIJNAAAA|55694|15|28|14|PM|second|afternoon|| +55695|AAAAAAAAAJJNAAAA|55695|15|28|15|PM|second|afternoon|| +55696|AAAAAAAABJJNAAAA|55696|15|28|16|PM|second|afternoon|| +55697|AAAAAAAACJJNAAAA|55697|15|28|17|PM|second|afternoon|| +55698|AAAAAAAADJJNAAAA|55698|15|28|18|PM|second|afternoon|| +55699|AAAAAAAAEJJNAAAA|55699|15|28|19|PM|second|afternoon|| +55700|AAAAAAAAFJJNAAAA|55700|15|28|20|PM|second|afternoon|| +55701|AAAAAAAAGJJNAAAA|55701|15|28|21|PM|second|afternoon|| +55702|AAAAAAAAHJJNAAAA|55702|15|28|22|PM|second|afternoon|| +55703|AAAAAAAAIJJNAAAA|55703|15|28|23|PM|second|afternoon|| +55704|AAAAAAAAJJJNAAAA|55704|15|28|24|PM|second|afternoon|| +55705|AAAAAAAAKJJNAAAA|55705|15|28|25|PM|second|afternoon|| +55706|AAAAAAAALJJNAAAA|55706|15|28|26|PM|second|afternoon|| +55707|AAAAAAAAMJJNAAAA|55707|15|28|27|PM|second|afternoon|| +55708|AAAAAAAANJJNAAAA|55708|15|28|28|PM|second|afternoon|| +55709|AAAAAAAAOJJNAAAA|55709|15|28|29|PM|second|afternoon|| +55710|AAAAAAAAPJJNAAAA|55710|15|28|30|PM|second|afternoon|| +55711|AAAAAAAAAKJNAAAA|55711|15|28|31|PM|second|afternoon|| +55712|AAAAAAAABKJNAAAA|55712|15|28|32|PM|second|afternoon|| +55713|AAAAAAAACKJNAAAA|55713|15|28|33|PM|second|afternoon|| +55714|AAAAAAAADKJNAAAA|55714|15|28|34|PM|second|afternoon|| +55715|AAAAAAAAEKJNAAAA|55715|15|28|35|PM|second|afternoon|| +55716|AAAAAAAAFKJNAAAA|55716|15|28|36|PM|second|afternoon|| +55717|AAAAAAAAGKJNAAAA|55717|15|28|37|PM|second|afternoon|| +55718|AAAAAAAAHKJNAAAA|55718|15|28|38|PM|second|afternoon|| +55719|AAAAAAAAIKJNAAAA|55719|15|28|39|PM|second|afternoon|| +55720|AAAAAAAAJKJNAAAA|55720|15|28|40|PM|second|afternoon|| +55721|AAAAAAAAKKJNAAAA|55721|15|28|41|PM|second|afternoon|| +55722|AAAAAAAALKJNAAAA|55722|15|28|42|PM|second|afternoon|| +55723|AAAAAAAAMKJNAAAA|55723|15|28|43|PM|second|afternoon|| +55724|AAAAAAAANKJNAAAA|55724|15|28|44|PM|second|afternoon|| +55725|AAAAAAAAOKJNAAAA|55725|15|28|45|PM|second|afternoon|| +55726|AAAAAAAAPKJNAAAA|55726|15|28|46|PM|second|afternoon|| +55727|AAAAAAAAALJNAAAA|55727|15|28|47|PM|second|afternoon|| +55728|AAAAAAAABLJNAAAA|55728|15|28|48|PM|second|afternoon|| +55729|AAAAAAAACLJNAAAA|55729|15|28|49|PM|second|afternoon|| +55730|AAAAAAAADLJNAAAA|55730|15|28|50|PM|second|afternoon|| +55731|AAAAAAAAELJNAAAA|55731|15|28|51|PM|second|afternoon|| +55732|AAAAAAAAFLJNAAAA|55732|15|28|52|PM|second|afternoon|| +55733|AAAAAAAAGLJNAAAA|55733|15|28|53|PM|second|afternoon|| +55734|AAAAAAAAHLJNAAAA|55734|15|28|54|PM|second|afternoon|| +55735|AAAAAAAAILJNAAAA|55735|15|28|55|PM|second|afternoon|| +55736|AAAAAAAAJLJNAAAA|55736|15|28|56|PM|second|afternoon|| +55737|AAAAAAAAKLJNAAAA|55737|15|28|57|PM|second|afternoon|| +55738|AAAAAAAALLJNAAAA|55738|15|28|58|PM|second|afternoon|| +55739|AAAAAAAAMLJNAAAA|55739|15|28|59|PM|second|afternoon|| +55740|AAAAAAAANLJNAAAA|55740|15|29|0|PM|second|afternoon|| +55741|AAAAAAAAOLJNAAAA|55741|15|29|1|PM|second|afternoon|| +55742|AAAAAAAAPLJNAAAA|55742|15|29|2|PM|second|afternoon|| +55743|AAAAAAAAAMJNAAAA|55743|15|29|3|PM|second|afternoon|| +55744|AAAAAAAABMJNAAAA|55744|15|29|4|PM|second|afternoon|| +55745|AAAAAAAACMJNAAAA|55745|15|29|5|PM|second|afternoon|| +55746|AAAAAAAADMJNAAAA|55746|15|29|6|PM|second|afternoon|| +55747|AAAAAAAAEMJNAAAA|55747|15|29|7|PM|second|afternoon|| +55748|AAAAAAAAFMJNAAAA|55748|15|29|8|PM|second|afternoon|| +55749|AAAAAAAAGMJNAAAA|55749|15|29|9|PM|second|afternoon|| +55750|AAAAAAAAHMJNAAAA|55750|15|29|10|PM|second|afternoon|| +55751|AAAAAAAAIMJNAAAA|55751|15|29|11|PM|second|afternoon|| +55752|AAAAAAAAJMJNAAAA|55752|15|29|12|PM|second|afternoon|| +55753|AAAAAAAAKMJNAAAA|55753|15|29|13|PM|second|afternoon|| +55754|AAAAAAAALMJNAAAA|55754|15|29|14|PM|second|afternoon|| +55755|AAAAAAAAMMJNAAAA|55755|15|29|15|PM|second|afternoon|| +55756|AAAAAAAANMJNAAAA|55756|15|29|16|PM|second|afternoon|| +55757|AAAAAAAAOMJNAAAA|55757|15|29|17|PM|second|afternoon|| +55758|AAAAAAAAPMJNAAAA|55758|15|29|18|PM|second|afternoon|| +55759|AAAAAAAAANJNAAAA|55759|15|29|19|PM|second|afternoon|| +55760|AAAAAAAABNJNAAAA|55760|15|29|20|PM|second|afternoon|| +55761|AAAAAAAACNJNAAAA|55761|15|29|21|PM|second|afternoon|| +55762|AAAAAAAADNJNAAAA|55762|15|29|22|PM|second|afternoon|| +55763|AAAAAAAAENJNAAAA|55763|15|29|23|PM|second|afternoon|| +55764|AAAAAAAAFNJNAAAA|55764|15|29|24|PM|second|afternoon|| +55765|AAAAAAAAGNJNAAAA|55765|15|29|25|PM|second|afternoon|| +55766|AAAAAAAAHNJNAAAA|55766|15|29|26|PM|second|afternoon|| +55767|AAAAAAAAINJNAAAA|55767|15|29|27|PM|second|afternoon|| +55768|AAAAAAAAJNJNAAAA|55768|15|29|28|PM|second|afternoon|| +55769|AAAAAAAAKNJNAAAA|55769|15|29|29|PM|second|afternoon|| +55770|AAAAAAAALNJNAAAA|55770|15|29|30|PM|second|afternoon|| +55771|AAAAAAAAMNJNAAAA|55771|15|29|31|PM|second|afternoon|| +55772|AAAAAAAANNJNAAAA|55772|15|29|32|PM|second|afternoon|| +55773|AAAAAAAAONJNAAAA|55773|15|29|33|PM|second|afternoon|| +55774|AAAAAAAAPNJNAAAA|55774|15|29|34|PM|second|afternoon|| +55775|AAAAAAAAAOJNAAAA|55775|15|29|35|PM|second|afternoon|| +55776|AAAAAAAABOJNAAAA|55776|15|29|36|PM|second|afternoon|| +55777|AAAAAAAACOJNAAAA|55777|15|29|37|PM|second|afternoon|| +55778|AAAAAAAADOJNAAAA|55778|15|29|38|PM|second|afternoon|| +55779|AAAAAAAAEOJNAAAA|55779|15|29|39|PM|second|afternoon|| +55780|AAAAAAAAFOJNAAAA|55780|15|29|40|PM|second|afternoon|| +55781|AAAAAAAAGOJNAAAA|55781|15|29|41|PM|second|afternoon|| +55782|AAAAAAAAHOJNAAAA|55782|15|29|42|PM|second|afternoon|| +55783|AAAAAAAAIOJNAAAA|55783|15|29|43|PM|second|afternoon|| +55784|AAAAAAAAJOJNAAAA|55784|15|29|44|PM|second|afternoon|| +55785|AAAAAAAAKOJNAAAA|55785|15|29|45|PM|second|afternoon|| +55786|AAAAAAAALOJNAAAA|55786|15|29|46|PM|second|afternoon|| +55787|AAAAAAAAMOJNAAAA|55787|15|29|47|PM|second|afternoon|| +55788|AAAAAAAANOJNAAAA|55788|15|29|48|PM|second|afternoon|| +55789|AAAAAAAAOOJNAAAA|55789|15|29|49|PM|second|afternoon|| +55790|AAAAAAAAPOJNAAAA|55790|15|29|50|PM|second|afternoon|| +55791|AAAAAAAAAPJNAAAA|55791|15|29|51|PM|second|afternoon|| +55792|AAAAAAAABPJNAAAA|55792|15|29|52|PM|second|afternoon|| +55793|AAAAAAAACPJNAAAA|55793|15|29|53|PM|second|afternoon|| +55794|AAAAAAAADPJNAAAA|55794|15|29|54|PM|second|afternoon|| +55795|AAAAAAAAEPJNAAAA|55795|15|29|55|PM|second|afternoon|| +55796|AAAAAAAAFPJNAAAA|55796|15|29|56|PM|second|afternoon|| +55797|AAAAAAAAGPJNAAAA|55797|15|29|57|PM|second|afternoon|| +55798|AAAAAAAAHPJNAAAA|55798|15|29|58|PM|second|afternoon|| +55799|AAAAAAAAIPJNAAAA|55799|15|29|59|PM|second|afternoon|| +55800|AAAAAAAAJPJNAAAA|55800|15|30|0|PM|second|afternoon|| +55801|AAAAAAAAKPJNAAAA|55801|15|30|1|PM|second|afternoon|| +55802|AAAAAAAALPJNAAAA|55802|15|30|2|PM|second|afternoon|| +55803|AAAAAAAAMPJNAAAA|55803|15|30|3|PM|second|afternoon|| +55804|AAAAAAAANPJNAAAA|55804|15|30|4|PM|second|afternoon|| +55805|AAAAAAAAOPJNAAAA|55805|15|30|5|PM|second|afternoon|| +55806|AAAAAAAAPPJNAAAA|55806|15|30|6|PM|second|afternoon|| +55807|AAAAAAAAAAKNAAAA|55807|15|30|7|PM|second|afternoon|| +55808|AAAAAAAABAKNAAAA|55808|15|30|8|PM|second|afternoon|| +55809|AAAAAAAACAKNAAAA|55809|15|30|9|PM|second|afternoon|| +55810|AAAAAAAADAKNAAAA|55810|15|30|10|PM|second|afternoon|| +55811|AAAAAAAAEAKNAAAA|55811|15|30|11|PM|second|afternoon|| +55812|AAAAAAAAFAKNAAAA|55812|15|30|12|PM|second|afternoon|| +55813|AAAAAAAAGAKNAAAA|55813|15|30|13|PM|second|afternoon|| +55814|AAAAAAAAHAKNAAAA|55814|15|30|14|PM|second|afternoon|| +55815|AAAAAAAAIAKNAAAA|55815|15|30|15|PM|second|afternoon|| +55816|AAAAAAAAJAKNAAAA|55816|15|30|16|PM|second|afternoon|| +55817|AAAAAAAAKAKNAAAA|55817|15|30|17|PM|second|afternoon|| +55818|AAAAAAAALAKNAAAA|55818|15|30|18|PM|second|afternoon|| +55819|AAAAAAAAMAKNAAAA|55819|15|30|19|PM|second|afternoon|| +55820|AAAAAAAANAKNAAAA|55820|15|30|20|PM|second|afternoon|| +55821|AAAAAAAAOAKNAAAA|55821|15|30|21|PM|second|afternoon|| +55822|AAAAAAAAPAKNAAAA|55822|15|30|22|PM|second|afternoon|| +55823|AAAAAAAAABKNAAAA|55823|15|30|23|PM|second|afternoon|| +55824|AAAAAAAABBKNAAAA|55824|15|30|24|PM|second|afternoon|| +55825|AAAAAAAACBKNAAAA|55825|15|30|25|PM|second|afternoon|| +55826|AAAAAAAADBKNAAAA|55826|15|30|26|PM|second|afternoon|| +55827|AAAAAAAAEBKNAAAA|55827|15|30|27|PM|second|afternoon|| +55828|AAAAAAAAFBKNAAAA|55828|15|30|28|PM|second|afternoon|| +55829|AAAAAAAAGBKNAAAA|55829|15|30|29|PM|second|afternoon|| +55830|AAAAAAAAHBKNAAAA|55830|15|30|30|PM|second|afternoon|| +55831|AAAAAAAAIBKNAAAA|55831|15|30|31|PM|second|afternoon|| +55832|AAAAAAAAJBKNAAAA|55832|15|30|32|PM|second|afternoon|| +55833|AAAAAAAAKBKNAAAA|55833|15|30|33|PM|second|afternoon|| +55834|AAAAAAAALBKNAAAA|55834|15|30|34|PM|second|afternoon|| +55835|AAAAAAAAMBKNAAAA|55835|15|30|35|PM|second|afternoon|| +55836|AAAAAAAANBKNAAAA|55836|15|30|36|PM|second|afternoon|| +55837|AAAAAAAAOBKNAAAA|55837|15|30|37|PM|second|afternoon|| +55838|AAAAAAAAPBKNAAAA|55838|15|30|38|PM|second|afternoon|| +55839|AAAAAAAAACKNAAAA|55839|15|30|39|PM|second|afternoon|| +55840|AAAAAAAABCKNAAAA|55840|15|30|40|PM|second|afternoon|| +55841|AAAAAAAACCKNAAAA|55841|15|30|41|PM|second|afternoon|| +55842|AAAAAAAADCKNAAAA|55842|15|30|42|PM|second|afternoon|| +55843|AAAAAAAAECKNAAAA|55843|15|30|43|PM|second|afternoon|| +55844|AAAAAAAAFCKNAAAA|55844|15|30|44|PM|second|afternoon|| +55845|AAAAAAAAGCKNAAAA|55845|15|30|45|PM|second|afternoon|| +55846|AAAAAAAAHCKNAAAA|55846|15|30|46|PM|second|afternoon|| +55847|AAAAAAAAICKNAAAA|55847|15|30|47|PM|second|afternoon|| +55848|AAAAAAAAJCKNAAAA|55848|15|30|48|PM|second|afternoon|| +55849|AAAAAAAAKCKNAAAA|55849|15|30|49|PM|second|afternoon|| +55850|AAAAAAAALCKNAAAA|55850|15|30|50|PM|second|afternoon|| +55851|AAAAAAAAMCKNAAAA|55851|15|30|51|PM|second|afternoon|| +55852|AAAAAAAANCKNAAAA|55852|15|30|52|PM|second|afternoon|| +55853|AAAAAAAAOCKNAAAA|55853|15|30|53|PM|second|afternoon|| +55854|AAAAAAAAPCKNAAAA|55854|15|30|54|PM|second|afternoon|| +55855|AAAAAAAAADKNAAAA|55855|15|30|55|PM|second|afternoon|| +55856|AAAAAAAABDKNAAAA|55856|15|30|56|PM|second|afternoon|| +55857|AAAAAAAACDKNAAAA|55857|15|30|57|PM|second|afternoon|| +55858|AAAAAAAADDKNAAAA|55858|15|30|58|PM|second|afternoon|| +55859|AAAAAAAAEDKNAAAA|55859|15|30|59|PM|second|afternoon|| +55860|AAAAAAAAFDKNAAAA|55860|15|31|0|PM|second|afternoon|| +55861|AAAAAAAAGDKNAAAA|55861|15|31|1|PM|second|afternoon|| +55862|AAAAAAAAHDKNAAAA|55862|15|31|2|PM|second|afternoon|| +55863|AAAAAAAAIDKNAAAA|55863|15|31|3|PM|second|afternoon|| +55864|AAAAAAAAJDKNAAAA|55864|15|31|4|PM|second|afternoon|| +55865|AAAAAAAAKDKNAAAA|55865|15|31|5|PM|second|afternoon|| +55866|AAAAAAAALDKNAAAA|55866|15|31|6|PM|second|afternoon|| +55867|AAAAAAAAMDKNAAAA|55867|15|31|7|PM|second|afternoon|| +55868|AAAAAAAANDKNAAAA|55868|15|31|8|PM|second|afternoon|| +55869|AAAAAAAAODKNAAAA|55869|15|31|9|PM|second|afternoon|| +55870|AAAAAAAAPDKNAAAA|55870|15|31|10|PM|second|afternoon|| +55871|AAAAAAAAAEKNAAAA|55871|15|31|11|PM|second|afternoon|| +55872|AAAAAAAABEKNAAAA|55872|15|31|12|PM|second|afternoon|| +55873|AAAAAAAACEKNAAAA|55873|15|31|13|PM|second|afternoon|| +55874|AAAAAAAADEKNAAAA|55874|15|31|14|PM|second|afternoon|| +55875|AAAAAAAAEEKNAAAA|55875|15|31|15|PM|second|afternoon|| +55876|AAAAAAAAFEKNAAAA|55876|15|31|16|PM|second|afternoon|| +55877|AAAAAAAAGEKNAAAA|55877|15|31|17|PM|second|afternoon|| +55878|AAAAAAAAHEKNAAAA|55878|15|31|18|PM|second|afternoon|| +55879|AAAAAAAAIEKNAAAA|55879|15|31|19|PM|second|afternoon|| +55880|AAAAAAAAJEKNAAAA|55880|15|31|20|PM|second|afternoon|| +55881|AAAAAAAAKEKNAAAA|55881|15|31|21|PM|second|afternoon|| +55882|AAAAAAAALEKNAAAA|55882|15|31|22|PM|second|afternoon|| +55883|AAAAAAAAMEKNAAAA|55883|15|31|23|PM|second|afternoon|| +55884|AAAAAAAANEKNAAAA|55884|15|31|24|PM|second|afternoon|| +55885|AAAAAAAAOEKNAAAA|55885|15|31|25|PM|second|afternoon|| +55886|AAAAAAAAPEKNAAAA|55886|15|31|26|PM|second|afternoon|| +55887|AAAAAAAAAFKNAAAA|55887|15|31|27|PM|second|afternoon|| +55888|AAAAAAAABFKNAAAA|55888|15|31|28|PM|second|afternoon|| +55889|AAAAAAAACFKNAAAA|55889|15|31|29|PM|second|afternoon|| +55890|AAAAAAAADFKNAAAA|55890|15|31|30|PM|second|afternoon|| +55891|AAAAAAAAEFKNAAAA|55891|15|31|31|PM|second|afternoon|| +55892|AAAAAAAAFFKNAAAA|55892|15|31|32|PM|second|afternoon|| +55893|AAAAAAAAGFKNAAAA|55893|15|31|33|PM|second|afternoon|| +55894|AAAAAAAAHFKNAAAA|55894|15|31|34|PM|second|afternoon|| +55895|AAAAAAAAIFKNAAAA|55895|15|31|35|PM|second|afternoon|| +55896|AAAAAAAAJFKNAAAA|55896|15|31|36|PM|second|afternoon|| +55897|AAAAAAAAKFKNAAAA|55897|15|31|37|PM|second|afternoon|| +55898|AAAAAAAALFKNAAAA|55898|15|31|38|PM|second|afternoon|| +55899|AAAAAAAAMFKNAAAA|55899|15|31|39|PM|second|afternoon|| +55900|AAAAAAAANFKNAAAA|55900|15|31|40|PM|second|afternoon|| +55901|AAAAAAAAOFKNAAAA|55901|15|31|41|PM|second|afternoon|| +55902|AAAAAAAAPFKNAAAA|55902|15|31|42|PM|second|afternoon|| +55903|AAAAAAAAAGKNAAAA|55903|15|31|43|PM|second|afternoon|| +55904|AAAAAAAABGKNAAAA|55904|15|31|44|PM|second|afternoon|| +55905|AAAAAAAACGKNAAAA|55905|15|31|45|PM|second|afternoon|| +55906|AAAAAAAADGKNAAAA|55906|15|31|46|PM|second|afternoon|| +55907|AAAAAAAAEGKNAAAA|55907|15|31|47|PM|second|afternoon|| +55908|AAAAAAAAFGKNAAAA|55908|15|31|48|PM|second|afternoon|| +55909|AAAAAAAAGGKNAAAA|55909|15|31|49|PM|second|afternoon|| +55910|AAAAAAAAHGKNAAAA|55910|15|31|50|PM|second|afternoon|| +55911|AAAAAAAAIGKNAAAA|55911|15|31|51|PM|second|afternoon|| +55912|AAAAAAAAJGKNAAAA|55912|15|31|52|PM|second|afternoon|| +55913|AAAAAAAAKGKNAAAA|55913|15|31|53|PM|second|afternoon|| +55914|AAAAAAAALGKNAAAA|55914|15|31|54|PM|second|afternoon|| +55915|AAAAAAAAMGKNAAAA|55915|15|31|55|PM|second|afternoon|| +55916|AAAAAAAANGKNAAAA|55916|15|31|56|PM|second|afternoon|| +55917|AAAAAAAAOGKNAAAA|55917|15|31|57|PM|second|afternoon|| +55918|AAAAAAAAPGKNAAAA|55918|15|31|58|PM|second|afternoon|| +55919|AAAAAAAAAHKNAAAA|55919|15|31|59|PM|second|afternoon|| +55920|AAAAAAAABHKNAAAA|55920|15|32|0|PM|second|afternoon|| +55921|AAAAAAAACHKNAAAA|55921|15|32|1|PM|second|afternoon|| +55922|AAAAAAAADHKNAAAA|55922|15|32|2|PM|second|afternoon|| +55923|AAAAAAAAEHKNAAAA|55923|15|32|3|PM|second|afternoon|| +55924|AAAAAAAAFHKNAAAA|55924|15|32|4|PM|second|afternoon|| +55925|AAAAAAAAGHKNAAAA|55925|15|32|5|PM|second|afternoon|| +55926|AAAAAAAAHHKNAAAA|55926|15|32|6|PM|second|afternoon|| +55927|AAAAAAAAIHKNAAAA|55927|15|32|7|PM|second|afternoon|| +55928|AAAAAAAAJHKNAAAA|55928|15|32|8|PM|second|afternoon|| +55929|AAAAAAAAKHKNAAAA|55929|15|32|9|PM|second|afternoon|| +55930|AAAAAAAALHKNAAAA|55930|15|32|10|PM|second|afternoon|| +55931|AAAAAAAAMHKNAAAA|55931|15|32|11|PM|second|afternoon|| +55932|AAAAAAAANHKNAAAA|55932|15|32|12|PM|second|afternoon|| +55933|AAAAAAAAOHKNAAAA|55933|15|32|13|PM|second|afternoon|| +55934|AAAAAAAAPHKNAAAA|55934|15|32|14|PM|second|afternoon|| +55935|AAAAAAAAAIKNAAAA|55935|15|32|15|PM|second|afternoon|| +55936|AAAAAAAABIKNAAAA|55936|15|32|16|PM|second|afternoon|| +55937|AAAAAAAACIKNAAAA|55937|15|32|17|PM|second|afternoon|| +55938|AAAAAAAADIKNAAAA|55938|15|32|18|PM|second|afternoon|| +55939|AAAAAAAAEIKNAAAA|55939|15|32|19|PM|second|afternoon|| +55940|AAAAAAAAFIKNAAAA|55940|15|32|20|PM|second|afternoon|| +55941|AAAAAAAAGIKNAAAA|55941|15|32|21|PM|second|afternoon|| +55942|AAAAAAAAHIKNAAAA|55942|15|32|22|PM|second|afternoon|| +55943|AAAAAAAAIIKNAAAA|55943|15|32|23|PM|second|afternoon|| +55944|AAAAAAAAJIKNAAAA|55944|15|32|24|PM|second|afternoon|| +55945|AAAAAAAAKIKNAAAA|55945|15|32|25|PM|second|afternoon|| +55946|AAAAAAAALIKNAAAA|55946|15|32|26|PM|second|afternoon|| +55947|AAAAAAAAMIKNAAAA|55947|15|32|27|PM|second|afternoon|| +55948|AAAAAAAANIKNAAAA|55948|15|32|28|PM|second|afternoon|| +55949|AAAAAAAAOIKNAAAA|55949|15|32|29|PM|second|afternoon|| +55950|AAAAAAAAPIKNAAAA|55950|15|32|30|PM|second|afternoon|| +55951|AAAAAAAAAJKNAAAA|55951|15|32|31|PM|second|afternoon|| +55952|AAAAAAAABJKNAAAA|55952|15|32|32|PM|second|afternoon|| +55953|AAAAAAAACJKNAAAA|55953|15|32|33|PM|second|afternoon|| +55954|AAAAAAAADJKNAAAA|55954|15|32|34|PM|second|afternoon|| +55955|AAAAAAAAEJKNAAAA|55955|15|32|35|PM|second|afternoon|| +55956|AAAAAAAAFJKNAAAA|55956|15|32|36|PM|second|afternoon|| +55957|AAAAAAAAGJKNAAAA|55957|15|32|37|PM|second|afternoon|| +55958|AAAAAAAAHJKNAAAA|55958|15|32|38|PM|second|afternoon|| +55959|AAAAAAAAIJKNAAAA|55959|15|32|39|PM|second|afternoon|| +55960|AAAAAAAAJJKNAAAA|55960|15|32|40|PM|second|afternoon|| +55961|AAAAAAAAKJKNAAAA|55961|15|32|41|PM|second|afternoon|| +55962|AAAAAAAALJKNAAAA|55962|15|32|42|PM|second|afternoon|| +55963|AAAAAAAAMJKNAAAA|55963|15|32|43|PM|second|afternoon|| +55964|AAAAAAAANJKNAAAA|55964|15|32|44|PM|second|afternoon|| +55965|AAAAAAAAOJKNAAAA|55965|15|32|45|PM|second|afternoon|| +55966|AAAAAAAAPJKNAAAA|55966|15|32|46|PM|second|afternoon|| +55967|AAAAAAAAAKKNAAAA|55967|15|32|47|PM|second|afternoon|| +55968|AAAAAAAABKKNAAAA|55968|15|32|48|PM|second|afternoon|| +55969|AAAAAAAACKKNAAAA|55969|15|32|49|PM|second|afternoon|| +55970|AAAAAAAADKKNAAAA|55970|15|32|50|PM|second|afternoon|| +55971|AAAAAAAAEKKNAAAA|55971|15|32|51|PM|second|afternoon|| +55972|AAAAAAAAFKKNAAAA|55972|15|32|52|PM|second|afternoon|| +55973|AAAAAAAAGKKNAAAA|55973|15|32|53|PM|second|afternoon|| +55974|AAAAAAAAHKKNAAAA|55974|15|32|54|PM|second|afternoon|| +55975|AAAAAAAAIKKNAAAA|55975|15|32|55|PM|second|afternoon|| +55976|AAAAAAAAJKKNAAAA|55976|15|32|56|PM|second|afternoon|| +55977|AAAAAAAAKKKNAAAA|55977|15|32|57|PM|second|afternoon|| +55978|AAAAAAAALKKNAAAA|55978|15|32|58|PM|second|afternoon|| +55979|AAAAAAAAMKKNAAAA|55979|15|32|59|PM|second|afternoon|| +55980|AAAAAAAANKKNAAAA|55980|15|33|0|PM|second|afternoon|| +55981|AAAAAAAAOKKNAAAA|55981|15|33|1|PM|second|afternoon|| +55982|AAAAAAAAPKKNAAAA|55982|15|33|2|PM|second|afternoon|| +55983|AAAAAAAAALKNAAAA|55983|15|33|3|PM|second|afternoon|| +55984|AAAAAAAABLKNAAAA|55984|15|33|4|PM|second|afternoon|| +55985|AAAAAAAACLKNAAAA|55985|15|33|5|PM|second|afternoon|| +55986|AAAAAAAADLKNAAAA|55986|15|33|6|PM|second|afternoon|| +55987|AAAAAAAAELKNAAAA|55987|15|33|7|PM|second|afternoon|| +55988|AAAAAAAAFLKNAAAA|55988|15|33|8|PM|second|afternoon|| +55989|AAAAAAAAGLKNAAAA|55989|15|33|9|PM|second|afternoon|| +55990|AAAAAAAAHLKNAAAA|55990|15|33|10|PM|second|afternoon|| +55991|AAAAAAAAILKNAAAA|55991|15|33|11|PM|second|afternoon|| +55992|AAAAAAAAJLKNAAAA|55992|15|33|12|PM|second|afternoon|| +55993|AAAAAAAAKLKNAAAA|55993|15|33|13|PM|second|afternoon|| +55994|AAAAAAAALLKNAAAA|55994|15|33|14|PM|second|afternoon|| +55995|AAAAAAAAMLKNAAAA|55995|15|33|15|PM|second|afternoon|| +55996|AAAAAAAANLKNAAAA|55996|15|33|16|PM|second|afternoon|| +55997|AAAAAAAAOLKNAAAA|55997|15|33|17|PM|second|afternoon|| +55998|AAAAAAAAPLKNAAAA|55998|15|33|18|PM|second|afternoon|| +55999|AAAAAAAAAMKNAAAA|55999|15|33|19|PM|second|afternoon|| +56000|AAAAAAAABMKNAAAA|56000|15|33|20|PM|second|afternoon|| +56001|AAAAAAAACMKNAAAA|56001|15|33|21|PM|second|afternoon|| +56002|AAAAAAAADMKNAAAA|56002|15|33|22|PM|second|afternoon|| +56003|AAAAAAAAEMKNAAAA|56003|15|33|23|PM|second|afternoon|| +56004|AAAAAAAAFMKNAAAA|56004|15|33|24|PM|second|afternoon|| +56005|AAAAAAAAGMKNAAAA|56005|15|33|25|PM|second|afternoon|| +56006|AAAAAAAAHMKNAAAA|56006|15|33|26|PM|second|afternoon|| +56007|AAAAAAAAIMKNAAAA|56007|15|33|27|PM|second|afternoon|| +56008|AAAAAAAAJMKNAAAA|56008|15|33|28|PM|second|afternoon|| +56009|AAAAAAAAKMKNAAAA|56009|15|33|29|PM|second|afternoon|| +56010|AAAAAAAALMKNAAAA|56010|15|33|30|PM|second|afternoon|| +56011|AAAAAAAAMMKNAAAA|56011|15|33|31|PM|second|afternoon|| +56012|AAAAAAAANMKNAAAA|56012|15|33|32|PM|second|afternoon|| +56013|AAAAAAAAOMKNAAAA|56013|15|33|33|PM|second|afternoon|| +56014|AAAAAAAAPMKNAAAA|56014|15|33|34|PM|second|afternoon|| +56015|AAAAAAAAANKNAAAA|56015|15|33|35|PM|second|afternoon|| +56016|AAAAAAAABNKNAAAA|56016|15|33|36|PM|second|afternoon|| +56017|AAAAAAAACNKNAAAA|56017|15|33|37|PM|second|afternoon|| +56018|AAAAAAAADNKNAAAA|56018|15|33|38|PM|second|afternoon|| +56019|AAAAAAAAENKNAAAA|56019|15|33|39|PM|second|afternoon|| +56020|AAAAAAAAFNKNAAAA|56020|15|33|40|PM|second|afternoon|| +56021|AAAAAAAAGNKNAAAA|56021|15|33|41|PM|second|afternoon|| +56022|AAAAAAAAHNKNAAAA|56022|15|33|42|PM|second|afternoon|| +56023|AAAAAAAAINKNAAAA|56023|15|33|43|PM|second|afternoon|| +56024|AAAAAAAAJNKNAAAA|56024|15|33|44|PM|second|afternoon|| +56025|AAAAAAAAKNKNAAAA|56025|15|33|45|PM|second|afternoon|| +56026|AAAAAAAALNKNAAAA|56026|15|33|46|PM|second|afternoon|| +56027|AAAAAAAAMNKNAAAA|56027|15|33|47|PM|second|afternoon|| +56028|AAAAAAAANNKNAAAA|56028|15|33|48|PM|second|afternoon|| +56029|AAAAAAAAONKNAAAA|56029|15|33|49|PM|second|afternoon|| +56030|AAAAAAAAPNKNAAAA|56030|15|33|50|PM|second|afternoon|| +56031|AAAAAAAAAOKNAAAA|56031|15|33|51|PM|second|afternoon|| +56032|AAAAAAAABOKNAAAA|56032|15|33|52|PM|second|afternoon|| +56033|AAAAAAAACOKNAAAA|56033|15|33|53|PM|second|afternoon|| +56034|AAAAAAAADOKNAAAA|56034|15|33|54|PM|second|afternoon|| +56035|AAAAAAAAEOKNAAAA|56035|15|33|55|PM|second|afternoon|| +56036|AAAAAAAAFOKNAAAA|56036|15|33|56|PM|second|afternoon|| +56037|AAAAAAAAGOKNAAAA|56037|15|33|57|PM|second|afternoon|| +56038|AAAAAAAAHOKNAAAA|56038|15|33|58|PM|second|afternoon|| +56039|AAAAAAAAIOKNAAAA|56039|15|33|59|PM|second|afternoon|| +56040|AAAAAAAAJOKNAAAA|56040|15|34|0|PM|second|afternoon|| +56041|AAAAAAAAKOKNAAAA|56041|15|34|1|PM|second|afternoon|| +56042|AAAAAAAALOKNAAAA|56042|15|34|2|PM|second|afternoon|| +56043|AAAAAAAAMOKNAAAA|56043|15|34|3|PM|second|afternoon|| +56044|AAAAAAAANOKNAAAA|56044|15|34|4|PM|second|afternoon|| +56045|AAAAAAAAOOKNAAAA|56045|15|34|5|PM|second|afternoon|| +56046|AAAAAAAAPOKNAAAA|56046|15|34|6|PM|second|afternoon|| +56047|AAAAAAAAAPKNAAAA|56047|15|34|7|PM|second|afternoon|| +56048|AAAAAAAABPKNAAAA|56048|15|34|8|PM|second|afternoon|| +56049|AAAAAAAACPKNAAAA|56049|15|34|9|PM|second|afternoon|| +56050|AAAAAAAADPKNAAAA|56050|15|34|10|PM|second|afternoon|| +56051|AAAAAAAAEPKNAAAA|56051|15|34|11|PM|second|afternoon|| +56052|AAAAAAAAFPKNAAAA|56052|15|34|12|PM|second|afternoon|| +56053|AAAAAAAAGPKNAAAA|56053|15|34|13|PM|second|afternoon|| +56054|AAAAAAAAHPKNAAAA|56054|15|34|14|PM|second|afternoon|| +56055|AAAAAAAAIPKNAAAA|56055|15|34|15|PM|second|afternoon|| +56056|AAAAAAAAJPKNAAAA|56056|15|34|16|PM|second|afternoon|| +56057|AAAAAAAAKPKNAAAA|56057|15|34|17|PM|second|afternoon|| +56058|AAAAAAAALPKNAAAA|56058|15|34|18|PM|second|afternoon|| +56059|AAAAAAAAMPKNAAAA|56059|15|34|19|PM|second|afternoon|| +56060|AAAAAAAANPKNAAAA|56060|15|34|20|PM|second|afternoon|| +56061|AAAAAAAAOPKNAAAA|56061|15|34|21|PM|second|afternoon|| +56062|AAAAAAAAPPKNAAAA|56062|15|34|22|PM|second|afternoon|| +56063|AAAAAAAAAALNAAAA|56063|15|34|23|PM|second|afternoon|| +56064|AAAAAAAABALNAAAA|56064|15|34|24|PM|second|afternoon|| +56065|AAAAAAAACALNAAAA|56065|15|34|25|PM|second|afternoon|| +56066|AAAAAAAADALNAAAA|56066|15|34|26|PM|second|afternoon|| +56067|AAAAAAAAEALNAAAA|56067|15|34|27|PM|second|afternoon|| +56068|AAAAAAAAFALNAAAA|56068|15|34|28|PM|second|afternoon|| +56069|AAAAAAAAGALNAAAA|56069|15|34|29|PM|second|afternoon|| +56070|AAAAAAAAHALNAAAA|56070|15|34|30|PM|second|afternoon|| +56071|AAAAAAAAIALNAAAA|56071|15|34|31|PM|second|afternoon|| +56072|AAAAAAAAJALNAAAA|56072|15|34|32|PM|second|afternoon|| +56073|AAAAAAAAKALNAAAA|56073|15|34|33|PM|second|afternoon|| +56074|AAAAAAAALALNAAAA|56074|15|34|34|PM|second|afternoon|| +56075|AAAAAAAAMALNAAAA|56075|15|34|35|PM|second|afternoon|| +56076|AAAAAAAANALNAAAA|56076|15|34|36|PM|second|afternoon|| +56077|AAAAAAAAOALNAAAA|56077|15|34|37|PM|second|afternoon|| +56078|AAAAAAAAPALNAAAA|56078|15|34|38|PM|second|afternoon|| +56079|AAAAAAAAABLNAAAA|56079|15|34|39|PM|second|afternoon|| +56080|AAAAAAAABBLNAAAA|56080|15|34|40|PM|second|afternoon|| +56081|AAAAAAAACBLNAAAA|56081|15|34|41|PM|second|afternoon|| +56082|AAAAAAAADBLNAAAA|56082|15|34|42|PM|second|afternoon|| +56083|AAAAAAAAEBLNAAAA|56083|15|34|43|PM|second|afternoon|| +56084|AAAAAAAAFBLNAAAA|56084|15|34|44|PM|second|afternoon|| +56085|AAAAAAAAGBLNAAAA|56085|15|34|45|PM|second|afternoon|| +56086|AAAAAAAAHBLNAAAA|56086|15|34|46|PM|second|afternoon|| +56087|AAAAAAAAIBLNAAAA|56087|15|34|47|PM|second|afternoon|| +56088|AAAAAAAAJBLNAAAA|56088|15|34|48|PM|second|afternoon|| +56089|AAAAAAAAKBLNAAAA|56089|15|34|49|PM|second|afternoon|| +56090|AAAAAAAALBLNAAAA|56090|15|34|50|PM|second|afternoon|| +56091|AAAAAAAAMBLNAAAA|56091|15|34|51|PM|second|afternoon|| +56092|AAAAAAAANBLNAAAA|56092|15|34|52|PM|second|afternoon|| +56093|AAAAAAAAOBLNAAAA|56093|15|34|53|PM|second|afternoon|| +56094|AAAAAAAAPBLNAAAA|56094|15|34|54|PM|second|afternoon|| +56095|AAAAAAAAACLNAAAA|56095|15|34|55|PM|second|afternoon|| +56096|AAAAAAAABCLNAAAA|56096|15|34|56|PM|second|afternoon|| +56097|AAAAAAAACCLNAAAA|56097|15|34|57|PM|second|afternoon|| +56098|AAAAAAAADCLNAAAA|56098|15|34|58|PM|second|afternoon|| +56099|AAAAAAAAECLNAAAA|56099|15|34|59|PM|second|afternoon|| +56100|AAAAAAAAFCLNAAAA|56100|15|35|0|PM|second|afternoon|| +56101|AAAAAAAAGCLNAAAA|56101|15|35|1|PM|second|afternoon|| +56102|AAAAAAAAHCLNAAAA|56102|15|35|2|PM|second|afternoon|| +56103|AAAAAAAAICLNAAAA|56103|15|35|3|PM|second|afternoon|| +56104|AAAAAAAAJCLNAAAA|56104|15|35|4|PM|second|afternoon|| +56105|AAAAAAAAKCLNAAAA|56105|15|35|5|PM|second|afternoon|| +56106|AAAAAAAALCLNAAAA|56106|15|35|6|PM|second|afternoon|| +56107|AAAAAAAAMCLNAAAA|56107|15|35|7|PM|second|afternoon|| +56108|AAAAAAAANCLNAAAA|56108|15|35|8|PM|second|afternoon|| +56109|AAAAAAAAOCLNAAAA|56109|15|35|9|PM|second|afternoon|| +56110|AAAAAAAAPCLNAAAA|56110|15|35|10|PM|second|afternoon|| +56111|AAAAAAAAADLNAAAA|56111|15|35|11|PM|second|afternoon|| +56112|AAAAAAAABDLNAAAA|56112|15|35|12|PM|second|afternoon|| +56113|AAAAAAAACDLNAAAA|56113|15|35|13|PM|second|afternoon|| +56114|AAAAAAAADDLNAAAA|56114|15|35|14|PM|second|afternoon|| +56115|AAAAAAAAEDLNAAAA|56115|15|35|15|PM|second|afternoon|| +56116|AAAAAAAAFDLNAAAA|56116|15|35|16|PM|second|afternoon|| +56117|AAAAAAAAGDLNAAAA|56117|15|35|17|PM|second|afternoon|| +56118|AAAAAAAAHDLNAAAA|56118|15|35|18|PM|second|afternoon|| +56119|AAAAAAAAIDLNAAAA|56119|15|35|19|PM|second|afternoon|| +56120|AAAAAAAAJDLNAAAA|56120|15|35|20|PM|second|afternoon|| +56121|AAAAAAAAKDLNAAAA|56121|15|35|21|PM|second|afternoon|| +56122|AAAAAAAALDLNAAAA|56122|15|35|22|PM|second|afternoon|| +56123|AAAAAAAAMDLNAAAA|56123|15|35|23|PM|second|afternoon|| +56124|AAAAAAAANDLNAAAA|56124|15|35|24|PM|second|afternoon|| +56125|AAAAAAAAODLNAAAA|56125|15|35|25|PM|second|afternoon|| +56126|AAAAAAAAPDLNAAAA|56126|15|35|26|PM|second|afternoon|| +56127|AAAAAAAAAELNAAAA|56127|15|35|27|PM|second|afternoon|| +56128|AAAAAAAABELNAAAA|56128|15|35|28|PM|second|afternoon|| +56129|AAAAAAAACELNAAAA|56129|15|35|29|PM|second|afternoon|| +56130|AAAAAAAADELNAAAA|56130|15|35|30|PM|second|afternoon|| +56131|AAAAAAAAEELNAAAA|56131|15|35|31|PM|second|afternoon|| +56132|AAAAAAAAFELNAAAA|56132|15|35|32|PM|second|afternoon|| +56133|AAAAAAAAGELNAAAA|56133|15|35|33|PM|second|afternoon|| +56134|AAAAAAAAHELNAAAA|56134|15|35|34|PM|second|afternoon|| +56135|AAAAAAAAIELNAAAA|56135|15|35|35|PM|second|afternoon|| +56136|AAAAAAAAJELNAAAA|56136|15|35|36|PM|second|afternoon|| +56137|AAAAAAAAKELNAAAA|56137|15|35|37|PM|second|afternoon|| +56138|AAAAAAAALELNAAAA|56138|15|35|38|PM|second|afternoon|| +56139|AAAAAAAAMELNAAAA|56139|15|35|39|PM|second|afternoon|| +56140|AAAAAAAANELNAAAA|56140|15|35|40|PM|second|afternoon|| +56141|AAAAAAAAOELNAAAA|56141|15|35|41|PM|second|afternoon|| +56142|AAAAAAAAPELNAAAA|56142|15|35|42|PM|second|afternoon|| +56143|AAAAAAAAAFLNAAAA|56143|15|35|43|PM|second|afternoon|| +56144|AAAAAAAABFLNAAAA|56144|15|35|44|PM|second|afternoon|| +56145|AAAAAAAACFLNAAAA|56145|15|35|45|PM|second|afternoon|| +56146|AAAAAAAADFLNAAAA|56146|15|35|46|PM|second|afternoon|| +56147|AAAAAAAAEFLNAAAA|56147|15|35|47|PM|second|afternoon|| +56148|AAAAAAAAFFLNAAAA|56148|15|35|48|PM|second|afternoon|| +56149|AAAAAAAAGFLNAAAA|56149|15|35|49|PM|second|afternoon|| +56150|AAAAAAAAHFLNAAAA|56150|15|35|50|PM|second|afternoon|| +56151|AAAAAAAAIFLNAAAA|56151|15|35|51|PM|second|afternoon|| +56152|AAAAAAAAJFLNAAAA|56152|15|35|52|PM|second|afternoon|| +56153|AAAAAAAAKFLNAAAA|56153|15|35|53|PM|second|afternoon|| +56154|AAAAAAAALFLNAAAA|56154|15|35|54|PM|second|afternoon|| +56155|AAAAAAAAMFLNAAAA|56155|15|35|55|PM|second|afternoon|| +56156|AAAAAAAANFLNAAAA|56156|15|35|56|PM|second|afternoon|| +56157|AAAAAAAAOFLNAAAA|56157|15|35|57|PM|second|afternoon|| +56158|AAAAAAAAPFLNAAAA|56158|15|35|58|PM|second|afternoon|| +56159|AAAAAAAAAGLNAAAA|56159|15|35|59|PM|second|afternoon|| +56160|AAAAAAAABGLNAAAA|56160|15|36|0|PM|second|afternoon|| +56161|AAAAAAAACGLNAAAA|56161|15|36|1|PM|second|afternoon|| +56162|AAAAAAAADGLNAAAA|56162|15|36|2|PM|second|afternoon|| +56163|AAAAAAAAEGLNAAAA|56163|15|36|3|PM|second|afternoon|| +56164|AAAAAAAAFGLNAAAA|56164|15|36|4|PM|second|afternoon|| +56165|AAAAAAAAGGLNAAAA|56165|15|36|5|PM|second|afternoon|| +56166|AAAAAAAAHGLNAAAA|56166|15|36|6|PM|second|afternoon|| +56167|AAAAAAAAIGLNAAAA|56167|15|36|7|PM|second|afternoon|| +56168|AAAAAAAAJGLNAAAA|56168|15|36|8|PM|second|afternoon|| +56169|AAAAAAAAKGLNAAAA|56169|15|36|9|PM|second|afternoon|| +56170|AAAAAAAALGLNAAAA|56170|15|36|10|PM|second|afternoon|| +56171|AAAAAAAAMGLNAAAA|56171|15|36|11|PM|second|afternoon|| +56172|AAAAAAAANGLNAAAA|56172|15|36|12|PM|second|afternoon|| +56173|AAAAAAAAOGLNAAAA|56173|15|36|13|PM|second|afternoon|| +56174|AAAAAAAAPGLNAAAA|56174|15|36|14|PM|second|afternoon|| +56175|AAAAAAAAAHLNAAAA|56175|15|36|15|PM|second|afternoon|| +56176|AAAAAAAABHLNAAAA|56176|15|36|16|PM|second|afternoon|| +56177|AAAAAAAACHLNAAAA|56177|15|36|17|PM|second|afternoon|| +56178|AAAAAAAADHLNAAAA|56178|15|36|18|PM|second|afternoon|| +56179|AAAAAAAAEHLNAAAA|56179|15|36|19|PM|second|afternoon|| +56180|AAAAAAAAFHLNAAAA|56180|15|36|20|PM|second|afternoon|| +56181|AAAAAAAAGHLNAAAA|56181|15|36|21|PM|second|afternoon|| +56182|AAAAAAAAHHLNAAAA|56182|15|36|22|PM|second|afternoon|| +56183|AAAAAAAAIHLNAAAA|56183|15|36|23|PM|second|afternoon|| +56184|AAAAAAAAJHLNAAAA|56184|15|36|24|PM|second|afternoon|| +56185|AAAAAAAAKHLNAAAA|56185|15|36|25|PM|second|afternoon|| +56186|AAAAAAAALHLNAAAA|56186|15|36|26|PM|second|afternoon|| +56187|AAAAAAAAMHLNAAAA|56187|15|36|27|PM|second|afternoon|| +56188|AAAAAAAANHLNAAAA|56188|15|36|28|PM|second|afternoon|| +56189|AAAAAAAAOHLNAAAA|56189|15|36|29|PM|second|afternoon|| +56190|AAAAAAAAPHLNAAAA|56190|15|36|30|PM|second|afternoon|| +56191|AAAAAAAAAILNAAAA|56191|15|36|31|PM|second|afternoon|| +56192|AAAAAAAABILNAAAA|56192|15|36|32|PM|second|afternoon|| +56193|AAAAAAAACILNAAAA|56193|15|36|33|PM|second|afternoon|| +56194|AAAAAAAADILNAAAA|56194|15|36|34|PM|second|afternoon|| +56195|AAAAAAAAEILNAAAA|56195|15|36|35|PM|second|afternoon|| +56196|AAAAAAAAFILNAAAA|56196|15|36|36|PM|second|afternoon|| +56197|AAAAAAAAGILNAAAA|56197|15|36|37|PM|second|afternoon|| +56198|AAAAAAAAHILNAAAA|56198|15|36|38|PM|second|afternoon|| +56199|AAAAAAAAIILNAAAA|56199|15|36|39|PM|second|afternoon|| +56200|AAAAAAAAJILNAAAA|56200|15|36|40|PM|second|afternoon|| +56201|AAAAAAAAKILNAAAA|56201|15|36|41|PM|second|afternoon|| +56202|AAAAAAAALILNAAAA|56202|15|36|42|PM|second|afternoon|| +56203|AAAAAAAAMILNAAAA|56203|15|36|43|PM|second|afternoon|| +56204|AAAAAAAANILNAAAA|56204|15|36|44|PM|second|afternoon|| +56205|AAAAAAAAOILNAAAA|56205|15|36|45|PM|second|afternoon|| +56206|AAAAAAAAPILNAAAA|56206|15|36|46|PM|second|afternoon|| +56207|AAAAAAAAAJLNAAAA|56207|15|36|47|PM|second|afternoon|| +56208|AAAAAAAABJLNAAAA|56208|15|36|48|PM|second|afternoon|| +56209|AAAAAAAACJLNAAAA|56209|15|36|49|PM|second|afternoon|| +56210|AAAAAAAADJLNAAAA|56210|15|36|50|PM|second|afternoon|| +56211|AAAAAAAAEJLNAAAA|56211|15|36|51|PM|second|afternoon|| +56212|AAAAAAAAFJLNAAAA|56212|15|36|52|PM|second|afternoon|| +56213|AAAAAAAAGJLNAAAA|56213|15|36|53|PM|second|afternoon|| +56214|AAAAAAAAHJLNAAAA|56214|15|36|54|PM|second|afternoon|| +56215|AAAAAAAAIJLNAAAA|56215|15|36|55|PM|second|afternoon|| +56216|AAAAAAAAJJLNAAAA|56216|15|36|56|PM|second|afternoon|| +56217|AAAAAAAAKJLNAAAA|56217|15|36|57|PM|second|afternoon|| +56218|AAAAAAAALJLNAAAA|56218|15|36|58|PM|second|afternoon|| +56219|AAAAAAAAMJLNAAAA|56219|15|36|59|PM|second|afternoon|| +56220|AAAAAAAANJLNAAAA|56220|15|37|0|PM|second|afternoon|| +56221|AAAAAAAAOJLNAAAA|56221|15|37|1|PM|second|afternoon|| +56222|AAAAAAAAPJLNAAAA|56222|15|37|2|PM|second|afternoon|| +56223|AAAAAAAAAKLNAAAA|56223|15|37|3|PM|second|afternoon|| +56224|AAAAAAAABKLNAAAA|56224|15|37|4|PM|second|afternoon|| +56225|AAAAAAAACKLNAAAA|56225|15|37|5|PM|second|afternoon|| +56226|AAAAAAAADKLNAAAA|56226|15|37|6|PM|second|afternoon|| +56227|AAAAAAAAEKLNAAAA|56227|15|37|7|PM|second|afternoon|| +56228|AAAAAAAAFKLNAAAA|56228|15|37|8|PM|second|afternoon|| +56229|AAAAAAAAGKLNAAAA|56229|15|37|9|PM|second|afternoon|| +56230|AAAAAAAAHKLNAAAA|56230|15|37|10|PM|second|afternoon|| +56231|AAAAAAAAIKLNAAAA|56231|15|37|11|PM|second|afternoon|| +56232|AAAAAAAAJKLNAAAA|56232|15|37|12|PM|second|afternoon|| +56233|AAAAAAAAKKLNAAAA|56233|15|37|13|PM|second|afternoon|| +56234|AAAAAAAALKLNAAAA|56234|15|37|14|PM|second|afternoon|| +56235|AAAAAAAAMKLNAAAA|56235|15|37|15|PM|second|afternoon|| +56236|AAAAAAAANKLNAAAA|56236|15|37|16|PM|second|afternoon|| +56237|AAAAAAAAOKLNAAAA|56237|15|37|17|PM|second|afternoon|| +56238|AAAAAAAAPKLNAAAA|56238|15|37|18|PM|second|afternoon|| +56239|AAAAAAAAALLNAAAA|56239|15|37|19|PM|second|afternoon|| +56240|AAAAAAAABLLNAAAA|56240|15|37|20|PM|second|afternoon|| +56241|AAAAAAAACLLNAAAA|56241|15|37|21|PM|second|afternoon|| +56242|AAAAAAAADLLNAAAA|56242|15|37|22|PM|second|afternoon|| +56243|AAAAAAAAELLNAAAA|56243|15|37|23|PM|second|afternoon|| +56244|AAAAAAAAFLLNAAAA|56244|15|37|24|PM|second|afternoon|| +56245|AAAAAAAAGLLNAAAA|56245|15|37|25|PM|second|afternoon|| +56246|AAAAAAAAHLLNAAAA|56246|15|37|26|PM|second|afternoon|| +56247|AAAAAAAAILLNAAAA|56247|15|37|27|PM|second|afternoon|| +56248|AAAAAAAAJLLNAAAA|56248|15|37|28|PM|second|afternoon|| +56249|AAAAAAAAKLLNAAAA|56249|15|37|29|PM|second|afternoon|| +56250|AAAAAAAALLLNAAAA|56250|15|37|30|PM|second|afternoon|| +56251|AAAAAAAAMLLNAAAA|56251|15|37|31|PM|second|afternoon|| +56252|AAAAAAAANLLNAAAA|56252|15|37|32|PM|second|afternoon|| +56253|AAAAAAAAOLLNAAAA|56253|15|37|33|PM|second|afternoon|| +56254|AAAAAAAAPLLNAAAA|56254|15|37|34|PM|second|afternoon|| +56255|AAAAAAAAAMLNAAAA|56255|15|37|35|PM|second|afternoon|| +56256|AAAAAAAABMLNAAAA|56256|15|37|36|PM|second|afternoon|| +56257|AAAAAAAACMLNAAAA|56257|15|37|37|PM|second|afternoon|| +56258|AAAAAAAADMLNAAAA|56258|15|37|38|PM|second|afternoon|| +56259|AAAAAAAAEMLNAAAA|56259|15|37|39|PM|second|afternoon|| +56260|AAAAAAAAFMLNAAAA|56260|15|37|40|PM|second|afternoon|| +56261|AAAAAAAAGMLNAAAA|56261|15|37|41|PM|second|afternoon|| +56262|AAAAAAAAHMLNAAAA|56262|15|37|42|PM|second|afternoon|| +56263|AAAAAAAAIMLNAAAA|56263|15|37|43|PM|second|afternoon|| +56264|AAAAAAAAJMLNAAAA|56264|15|37|44|PM|second|afternoon|| +56265|AAAAAAAAKMLNAAAA|56265|15|37|45|PM|second|afternoon|| +56266|AAAAAAAALMLNAAAA|56266|15|37|46|PM|second|afternoon|| +56267|AAAAAAAAMMLNAAAA|56267|15|37|47|PM|second|afternoon|| +56268|AAAAAAAANMLNAAAA|56268|15|37|48|PM|second|afternoon|| +56269|AAAAAAAAOMLNAAAA|56269|15|37|49|PM|second|afternoon|| +56270|AAAAAAAAPMLNAAAA|56270|15|37|50|PM|second|afternoon|| +56271|AAAAAAAAANLNAAAA|56271|15|37|51|PM|second|afternoon|| +56272|AAAAAAAABNLNAAAA|56272|15|37|52|PM|second|afternoon|| +56273|AAAAAAAACNLNAAAA|56273|15|37|53|PM|second|afternoon|| +56274|AAAAAAAADNLNAAAA|56274|15|37|54|PM|second|afternoon|| +56275|AAAAAAAAENLNAAAA|56275|15|37|55|PM|second|afternoon|| +56276|AAAAAAAAFNLNAAAA|56276|15|37|56|PM|second|afternoon|| +56277|AAAAAAAAGNLNAAAA|56277|15|37|57|PM|second|afternoon|| +56278|AAAAAAAAHNLNAAAA|56278|15|37|58|PM|second|afternoon|| +56279|AAAAAAAAINLNAAAA|56279|15|37|59|PM|second|afternoon|| +56280|AAAAAAAAJNLNAAAA|56280|15|38|0|PM|second|afternoon|| +56281|AAAAAAAAKNLNAAAA|56281|15|38|1|PM|second|afternoon|| +56282|AAAAAAAALNLNAAAA|56282|15|38|2|PM|second|afternoon|| +56283|AAAAAAAAMNLNAAAA|56283|15|38|3|PM|second|afternoon|| +56284|AAAAAAAANNLNAAAA|56284|15|38|4|PM|second|afternoon|| +56285|AAAAAAAAONLNAAAA|56285|15|38|5|PM|second|afternoon|| +56286|AAAAAAAAPNLNAAAA|56286|15|38|6|PM|second|afternoon|| +56287|AAAAAAAAAOLNAAAA|56287|15|38|7|PM|second|afternoon|| +56288|AAAAAAAABOLNAAAA|56288|15|38|8|PM|second|afternoon|| +56289|AAAAAAAACOLNAAAA|56289|15|38|9|PM|second|afternoon|| +56290|AAAAAAAADOLNAAAA|56290|15|38|10|PM|second|afternoon|| +56291|AAAAAAAAEOLNAAAA|56291|15|38|11|PM|second|afternoon|| +56292|AAAAAAAAFOLNAAAA|56292|15|38|12|PM|second|afternoon|| +56293|AAAAAAAAGOLNAAAA|56293|15|38|13|PM|second|afternoon|| +56294|AAAAAAAAHOLNAAAA|56294|15|38|14|PM|second|afternoon|| +56295|AAAAAAAAIOLNAAAA|56295|15|38|15|PM|second|afternoon|| +56296|AAAAAAAAJOLNAAAA|56296|15|38|16|PM|second|afternoon|| +56297|AAAAAAAAKOLNAAAA|56297|15|38|17|PM|second|afternoon|| +56298|AAAAAAAALOLNAAAA|56298|15|38|18|PM|second|afternoon|| +56299|AAAAAAAAMOLNAAAA|56299|15|38|19|PM|second|afternoon|| +56300|AAAAAAAANOLNAAAA|56300|15|38|20|PM|second|afternoon|| +56301|AAAAAAAAOOLNAAAA|56301|15|38|21|PM|second|afternoon|| +56302|AAAAAAAAPOLNAAAA|56302|15|38|22|PM|second|afternoon|| +56303|AAAAAAAAAPLNAAAA|56303|15|38|23|PM|second|afternoon|| +56304|AAAAAAAABPLNAAAA|56304|15|38|24|PM|second|afternoon|| +56305|AAAAAAAACPLNAAAA|56305|15|38|25|PM|second|afternoon|| +56306|AAAAAAAADPLNAAAA|56306|15|38|26|PM|second|afternoon|| +56307|AAAAAAAAEPLNAAAA|56307|15|38|27|PM|second|afternoon|| +56308|AAAAAAAAFPLNAAAA|56308|15|38|28|PM|second|afternoon|| +56309|AAAAAAAAGPLNAAAA|56309|15|38|29|PM|second|afternoon|| +56310|AAAAAAAAHPLNAAAA|56310|15|38|30|PM|second|afternoon|| +56311|AAAAAAAAIPLNAAAA|56311|15|38|31|PM|second|afternoon|| +56312|AAAAAAAAJPLNAAAA|56312|15|38|32|PM|second|afternoon|| +56313|AAAAAAAAKPLNAAAA|56313|15|38|33|PM|second|afternoon|| +56314|AAAAAAAALPLNAAAA|56314|15|38|34|PM|second|afternoon|| +56315|AAAAAAAAMPLNAAAA|56315|15|38|35|PM|second|afternoon|| +56316|AAAAAAAANPLNAAAA|56316|15|38|36|PM|second|afternoon|| +56317|AAAAAAAAOPLNAAAA|56317|15|38|37|PM|second|afternoon|| +56318|AAAAAAAAPPLNAAAA|56318|15|38|38|PM|second|afternoon|| +56319|AAAAAAAAAAMNAAAA|56319|15|38|39|PM|second|afternoon|| +56320|AAAAAAAABAMNAAAA|56320|15|38|40|PM|second|afternoon|| +56321|AAAAAAAACAMNAAAA|56321|15|38|41|PM|second|afternoon|| +56322|AAAAAAAADAMNAAAA|56322|15|38|42|PM|second|afternoon|| +56323|AAAAAAAAEAMNAAAA|56323|15|38|43|PM|second|afternoon|| +56324|AAAAAAAAFAMNAAAA|56324|15|38|44|PM|second|afternoon|| +56325|AAAAAAAAGAMNAAAA|56325|15|38|45|PM|second|afternoon|| +56326|AAAAAAAAHAMNAAAA|56326|15|38|46|PM|second|afternoon|| +56327|AAAAAAAAIAMNAAAA|56327|15|38|47|PM|second|afternoon|| +56328|AAAAAAAAJAMNAAAA|56328|15|38|48|PM|second|afternoon|| +56329|AAAAAAAAKAMNAAAA|56329|15|38|49|PM|second|afternoon|| +56330|AAAAAAAALAMNAAAA|56330|15|38|50|PM|second|afternoon|| +56331|AAAAAAAAMAMNAAAA|56331|15|38|51|PM|second|afternoon|| +56332|AAAAAAAANAMNAAAA|56332|15|38|52|PM|second|afternoon|| +56333|AAAAAAAAOAMNAAAA|56333|15|38|53|PM|second|afternoon|| +56334|AAAAAAAAPAMNAAAA|56334|15|38|54|PM|second|afternoon|| +56335|AAAAAAAAABMNAAAA|56335|15|38|55|PM|second|afternoon|| +56336|AAAAAAAABBMNAAAA|56336|15|38|56|PM|second|afternoon|| +56337|AAAAAAAACBMNAAAA|56337|15|38|57|PM|second|afternoon|| +56338|AAAAAAAADBMNAAAA|56338|15|38|58|PM|second|afternoon|| +56339|AAAAAAAAEBMNAAAA|56339|15|38|59|PM|second|afternoon|| +56340|AAAAAAAAFBMNAAAA|56340|15|39|0|PM|second|afternoon|| +56341|AAAAAAAAGBMNAAAA|56341|15|39|1|PM|second|afternoon|| +56342|AAAAAAAAHBMNAAAA|56342|15|39|2|PM|second|afternoon|| +56343|AAAAAAAAIBMNAAAA|56343|15|39|3|PM|second|afternoon|| +56344|AAAAAAAAJBMNAAAA|56344|15|39|4|PM|second|afternoon|| +56345|AAAAAAAAKBMNAAAA|56345|15|39|5|PM|second|afternoon|| +56346|AAAAAAAALBMNAAAA|56346|15|39|6|PM|second|afternoon|| +56347|AAAAAAAAMBMNAAAA|56347|15|39|7|PM|second|afternoon|| +56348|AAAAAAAANBMNAAAA|56348|15|39|8|PM|second|afternoon|| +56349|AAAAAAAAOBMNAAAA|56349|15|39|9|PM|second|afternoon|| +56350|AAAAAAAAPBMNAAAA|56350|15|39|10|PM|second|afternoon|| +56351|AAAAAAAAACMNAAAA|56351|15|39|11|PM|second|afternoon|| +56352|AAAAAAAABCMNAAAA|56352|15|39|12|PM|second|afternoon|| +56353|AAAAAAAACCMNAAAA|56353|15|39|13|PM|second|afternoon|| +56354|AAAAAAAADCMNAAAA|56354|15|39|14|PM|second|afternoon|| +56355|AAAAAAAAECMNAAAA|56355|15|39|15|PM|second|afternoon|| +56356|AAAAAAAAFCMNAAAA|56356|15|39|16|PM|second|afternoon|| +56357|AAAAAAAAGCMNAAAA|56357|15|39|17|PM|second|afternoon|| +56358|AAAAAAAAHCMNAAAA|56358|15|39|18|PM|second|afternoon|| +56359|AAAAAAAAICMNAAAA|56359|15|39|19|PM|second|afternoon|| +56360|AAAAAAAAJCMNAAAA|56360|15|39|20|PM|second|afternoon|| +56361|AAAAAAAAKCMNAAAA|56361|15|39|21|PM|second|afternoon|| +56362|AAAAAAAALCMNAAAA|56362|15|39|22|PM|second|afternoon|| +56363|AAAAAAAAMCMNAAAA|56363|15|39|23|PM|second|afternoon|| +56364|AAAAAAAANCMNAAAA|56364|15|39|24|PM|second|afternoon|| +56365|AAAAAAAAOCMNAAAA|56365|15|39|25|PM|second|afternoon|| +56366|AAAAAAAAPCMNAAAA|56366|15|39|26|PM|second|afternoon|| +56367|AAAAAAAAADMNAAAA|56367|15|39|27|PM|second|afternoon|| +56368|AAAAAAAABDMNAAAA|56368|15|39|28|PM|second|afternoon|| +56369|AAAAAAAACDMNAAAA|56369|15|39|29|PM|second|afternoon|| +56370|AAAAAAAADDMNAAAA|56370|15|39|30|PM|second|afternoon|| +56371|AAAAAAAAEDMNAAAA|56371|15|39|31|PM|second|afternoon|| +56372|AAAAAAAAFDMNAAAA|56372|15|39|32|PM|second|afternoon|| +56373|AAAAAAAAGDMNAAAA|56373|15|39|33|PM|second|afternoon|| +56374|AAAAAAAAHDMNAAAA|56374|15|39|34|PM|second|afternoon|| +56375|AAAAAAAAIDMNAAAA|56375|15|39|35|PM|second|afternoon|| +56376|AAAAAAAAJDMNAAAA|56376|15|39|36|PM|second|afternoon|| +56377|AAAAAAAAKDMNAAAA|56377|15|39|37|PM|second|afternoon|| +56378|AAAAAAAALDMNAAAA|56378|15|39|38|PM|second|afternoon|| +56379|AAAAAAAAMDMNAAAA|56379|15|39|39|PM|second|afternoon|| +56380|AAAAAAAANDMNAAAA|56380|15|39|40|PM|second|afternoon|| +56381|AAAAAAAAODMNAAAA|56381|15|39|41|PM|second|afternoon|| +56382|AAAAAAAAPDMNAAAA|56382|15|39|42|PM|second|afternoon|| +56383|AAAAAAAAAEMNAAAA|56383|15|39|43|PM|second|afternoon|| +56384|AAAAAAAABEMNAAAA|56384|15|39|44|PM|second|afternoon|| +56385|AAAAAAAACEMNAAAA|56385|15|39|45|PM|second|afternoon|| +56386|AAAAAAAADEMNAAAA|56386|15|39|46|PM|second|afternoon|| +56387|AAAAAAAAEEMNAAAA|56387|15|39|47|PM|second|afternoon|| +56388|AAAAAAAAFEMNAAAA|56388|15|39|48|PM|second|afternoon|| +56389|AAAAAAAAGEMNAAAA|56389|15|39|49|PM|second|afternoon|| +56390|AAAAAAAAHEMNAAAA|56390|15|39|50|PM|second|afternoon|| +56391|AAAAAAAAIEMNAAAA|56391|15|39|51|PM|second|afternoon|| +56392|AAAAAAAAJEMNAAAA|56392|15|39|52|PM|second|afternoon|| +56393|AAAAAAAAKEMNAAAA|56393|15|39|53|PM|second|afternoon|| +56394|AAAAAAAALEMNAAAA|56394|15|39|54|PM|second|afternoon|| +56395|AAAAAAAAMEMNAAAA|56395|15|39|55|PM|second|afternoon|| +56396|AAAAAAAANEMNAAAA|56396|15|39|56|PM|second|afternoon|| +56397|AAAAAAAAOEMNAAAA|56397|15|39|57|PM|second|afternoon|| +56398|AAAAAAAAPEMNAAAA|56398|15|39|58|PM|second|afternoon|| +56399|AAAAAAAAAFMNAAAA|56399|15|39|59|PM|second|afternoon|| +56400|AAAAAAAABFMNAAAA|56400|15|40|0|PM|second|afternoon|| +56401|AAAAAAAACFMNAAAA|56401|15|40|1|PM|second|afternoon|| +56402|AAAAAAAADFMNAAAA|56402|15|40|2|PM|second|afternoon|| +56403|AAAAAAAAEFMNAAAA|56403|15|40|3|PM|second|afternoon|| +56404|AAAAAAAAFFMNAAAA|56404|15|40|4|PM|second|afternoon|| +56405|AAAAAAAAGFMNAAAA|56405|15|40|5|PM|second|afternoon|| +56406|AAAAAAAAHFMNAAAA|56406|15|40|6|PM|second|afternoon|| +56407|AAAAAAAAIFMNAAAA|56407|15|40|7|PM|second|afternoon|| +56408|AAAAAAAAJFMNAAAA|56408|15|40|8|PM|second|afternoon|| +56409|AAAAAAAAKFMNAAAA|56409|15|40|9|PM|second|afternoon|| +56410|AAAAAAAALFMNAAAA|56410|15|40|10|PM|second|afternoon|| +56411|AAAAAAAAMFMNAAAA|56411|15|40|11|PM|second|afternoon|| +56412|AAAAAAAANFMNAAAA|56412|15|40|12|PM|second|afternoon|| +56413|AAAAAAAAOFMNAAAA|56413|15|40|13|PM|second|afternoon|| +56414|AAAAAAAAPFMNAAAA|56414|15|40|14|PM|second|afternoon|| +56415|AAAAAAAAAGMNAAAA|56415|15|40|15|PM|second|afternoon|| +56416|AAAAAAAABGMNAAAA|56416|15|40|16|PM|second|afternoon|| +56417|AAAAAAAACGMNAAAA|56417|15|40|17|PM|second|afternoon|| +56418|AAAAAAAADGMNAAAA|56418|15|40|18|PM|second|afternoon|| +56419|AAAAAAAAEGMNAAAA|56419|15|40|19|PM|second|afternoon|| +56420|AAAAAAAAFGMNAAAA|56420|15|40|20|PM|second|afternoon|| +56421|AAAAAAAAGGMNAAAA|56421|15|40|21|PM|second|afternoon|| +56422|AAAAAAAAHGMNAAAA|56422|15|40|22|PM|second|afternoon|| +56423|AAAAAAAAIGMNAAAA|56423|15|40|23|PM|second|afternoon|| +56424|AAAAAAAAJGMNAAAA|56424|15|40|24|PM|second|afternoon|| +56425|AAAAAAAAKGMNAAAA|56425|15|40|25|PM|second|afternoon|| +56426|AAAAAAAALGMNAAAA|56426|15|40|26|PM|second|afternoon|| +56427|AAAAAAAAMGMNAAAA|56427|15|40|27|PM|second|afternoon|| +56428|AAAAAAAANGMNAAAA|56428|15|40|28|PM|second|afternoon|| +56429|AAAAAAAAOGMNAAAA|56429|15|40|29|PM|second|afternoon|| +56430|AAAAAAAAPGMNAAAA|56430|15|40|30|PM|second|afternoon|| +56431|AAAAAAAAAHMNAAAA|56431|15|40|31|PM|second|afternoon|| +56432|AAAAAAAABHMNAAAA|56432|15|40|32|PM|second|afternoon|| +56433|AAAAAAAACHMNAAAA|56433|15|40|33|PM|second|afternoon|| +56434|AAAAAAAADHMNAAAA|56434|15|40|34|PM|second|afternoon|| +56435|AAAAAAAAEHMNAAAA|56435|15|40|35|PM|second|afternoon|| +56436|AAAAAAAAFHMNAAAA|56436|15|40|36|PM|second|afternoon|| +56437|AAAAAAAAGHMNAAAA|56437|15|40|37|PM|second|afternoon|| +56438|AAAAAAAAHHMNAAAA|56438|15|40|38|PM|second|afternoon|| +56439|AAAAAAAAIHMNAAAA|56439|15|40|39|PM|second|afternoon|| +56440|AAAAAAAAJHMNAAAA|56440|15|40|40|PM|second|afternoon|| +56441|AAAAAAAAKHMNAAAA|56441|15|40|41|PM|second|afternoon|| +56442|AAAAAAAALHMNAAAA|56442|15|40|42|PM|second|afternoon|| +56443|AAAAAAAAMHMNAAAA|56443|15|40|43|PM|second|afternoon|| +56444|AAAAAAAANHMNAAAA|56444|15|40|44|PM|second|afternoon|| +56445|AAAAAAAAOHMNAAAA|56445|15|40|45|PM|second|afternoon|| +56446|AAAAAAAAPHMNAAAA|56446|15|40|46|PM|second|afternoon|| +56447|AAAAAAAAAIMNAAAA|56447|15|40|47|PM|second|afternoon|| +56448|AAAAAAAABIMNAAAA|56448|15|40|48|PM|second|afternoon|| +56449|AAAAAAAACIMNAAAA|56449|15|40|49|PM|second|afternoon|| +56450|AAAAAAAADIMNAAAA|56450|15|40|50|PM|second|afternoon|| +56451|AAAAAAAAEIMNAAAA|56451|15|40|51|PM|second|afternoon|| +56452|AAAAAAAAFIMNAAAA|56452|15|40|52|PM|second|afternoon|| +56453|AAAAAAAAGIMNAAAA|56453|15|40|53|PM|second|afternoon|| +56454|AAAAAAAAHIMNAAAA|56454|15|40|54|PM|second|afternoon|| +56455|AAAAAAAAIIMNAAAA|56455|15|40|55|PM|second|afternoon|| +56456|AAAAAAAAJIMNAAAA|56456|15|40|56|PM|second|afternoon|| +56457|AAAAAAAAKIMNAAAA|56457|15|40|57|PM|second|afternoon|| +56458|AAAAAAAALIMNAAAA|56458|15|40|58|PM|second|afternoon|| +56459|AAAAAAAAMIMNAAAA|56459|15|40|59|PM|second|afternoon|| +56460|AAAAAAAANIMNAAAA|56460|15|41|0|PM|second|afternoon|| +56461|AAAAAAAAOIMNAAAA|56461|15|41|1|PM|second|afternoon|| +56462|AAAAAAAAPIMNAAAA|56462|15|41|2|PM|second|afternoon|| +56463|AAAAAAAAAJMNAAAA|56463|15|41|3|PM|second|afternoon|| +56464|AAAAAAAABJMNAAAA|56464|15|41|4|PM|second|afternoon|| +56465|AAAAAAAACJMNAAAA|56465|15|41|5|PM|second|afternoon|| +56466|AAAAAAAADJMNAAAA|56466|15|41|6|PM|second|afternoon|| +56467|AAAAAAAAEJMNAAAA|56467|15|41|7|PM|second|afternoon|| +56468|AAAAAAAAFJMNAAAA|56468|15|41|8|PM|second|afternoon|| +56469|AAAAAAAAGJMNAAAA|56469|15|41|9|PM|second|afternoon|| +56470|AAAAAAAAHJMNAAAA|56470|15|41|10|PM|second|afternoon|| +56471|AAAAAAAAIJMNAAAA|56471|15|41|11|PM|second|afternoon|| +56472|AAAAAAAAJJMNAAAA|56472|15|41|12|PM|second|afternoon|| +56473|AAAAAAAAKJMNAAAA|56473|15|41|13|PM|second|afternoon|| +56474|AAAAAAAALJMNAAAA|56474|15|41|14|PM|second|afternoon|| +56475|AAAAAAAAMJMNAAAA|56475|15|41|15|PM|second|afternoon|| +56476|AAAAAAAANJMNAAAA|56476|15|41|16|PM|second|afternoon|| +56477|AAAAAAAAOJMNAAAA|56477|15|41|17|PM|second|afternoon|| +56478|AAAAAAAAPJMNAAAA|56478|15|41|18|PM|second|afternoon|| +56479|AAAAAAAAAKMNAAAA|56479|15|41|19|PM|second|afternoon|| +56480|AAAAAAAABKMNAAAA|56480|15|41|20|PM|second|afternoon|| +56481|AAAAAAAACKMNAAAA|56481|15|41|21|PM|second|afternoon|| +56482|AAAAAAAADKMNAAAA|56482|15|41|22|PM|second|afternoon|| +56483|AAAAAAAAEKMNAAAA|56483|15|41|23|PM|second|afternoon|| +56484|AAAAAAAAFKMNAAAA|56484|15|41|24|PM|second|afternoon|| +56485|AAAAAAAAGKMNAAAA|56485|15|41|25|PM|second|afternoon|| +56486|AAAAAAAAHKMNAAAA|56486|15|41|26|PM|second|afternoon|| +56487|AAAAAAAAIKMNAAAA|56487|15|41|27|PM|second|afternoon|| +56488|AAAAAAAAJKMNAAAA|56488|15|41|28|PM|second|afternoon|| +56489|AAAAAAAAKKMNAAAA|56489|15|41|29|PM|second|afternoon|| +56490|AAAAAAAALKMNAAAA|56490|15|41|30|PM|second|afternoon|| +56491|AAAAAAAAMKMNAAAA|56491|15|41|31|PM|second|afternoon|| +56492|AAAAAAAANKMNAAAA|56492|15|41|32|PM|second|afternoon|| +56493|AAAAAAAAOKMNAAAA|56493|15|41|33|PM|second|afternoon|| +56494|AAAAAAAAPKMNAAAA|56494|15|41|34|PM|second|afternoon|| +56495|AAAAAAAAALMNAAAA|56495|15|41|35|PM|second|afternoon|| +56496|AAAAAAAABLMNAAAA|56496|15|41|36|PM|second|afternoon|| +56497|AAAAAAAACLMNAAAA|56497|15|41|37|PM|second|afternoon|| +56498|AAAAAAAADLMNAAAA|56498|15|41|38|PM|second|afternoon|| +56499|AAAAAAAAELMNAAAA|56499|15|41|39|PM|second|afternoon|| +56500|AAAAAAAAFLMNAAAA|56500|15|41|40|PM|second|afternoon|| +56501|AAAAAAAAGLMNAAAA|56501|15|41|41|PM|second|afternoon|| +56502|AAAAAAAAHLMNAAAA|56502|15|41|42|PM|second|afternoon|| +56503|AAAAAAAAILMNAAAA|56503|15|41|43|PM|second|afternoon|| +56504|AAAAAAAAJLMNAAAA|56504|15|41|44|PM|second|afternoon|| +56505|AAAAAAAAKLMNAAAA|56505|15|41|45|PM|second|afternoon|| +56506|AAAAAAAALLMNAAAA|56506|15|41|46|PM|second|afternoon|| +56507|AAAAAAAAMLMNAAAA|56507|15|41|47|PM|second|afternoon|| +56508|AAAAAAAANLMNAAAA|56508|15|41|48|PM|second|afternoon|| +56509|AAAAAAAAOLMNAAAA|56509|15|41|49|PM|second|afternoon|| +56510|AAAAAAAAPLMNAAAA|56510|15|41|50|PM|second|afternoon|| +56511|AAAAAAAAAMMNAAAA|56511|15|41|51|PM|second|afternoon|| +56512|AAAAAAAABMMNAAAA|56512|15|41|52|PM|second|afternoon|| +56513|AAAAAAAACMMNAAAA|56513|15|41|53|PM|second|afternoon|| +56514|AAAAAAAADMMNAAAA|56514|15|41|54|PM|second|afternoon|| +56515|AAAAAAAAEMMNAAAA|56515|15|41|55|PM|second|afternoon|| +56516|AAAAAAAAFMMNAAAA|56516|15|41|56|PM|second|afternoon|| +56517|AAAAAAAAGMMNAAAA|56517|15|41|57|PM|second|afternoon|| +56518|AAAAAAAAHMMNAAAA|56518|15|41|58|PM|second|afternoon|| +56519|AAAAAAAAIMMNAAAA|56519|15|41|59|PM|second|afternoon|| +56520|AAAAAAAAJMMNAAAA|56520|15|42|0|PM|second|afternoon|| +56521|AAAAAAAAKMMNAAAA|56521|15|42|1|PM|second|afternoon|| +56522|AAAAAAAALMMNAAAA|56522|15|42|2|PM|second|afternoon|| +56523|AAAAAAAAMMMNAAAA|56523|15|42|3|PM|second|afternoon|| +56524|AAAAAAAANMMNAAAA|56524|15|42|4|PM|second|afternoon|| +56525|AAAAAAAAOMMNAAAA|56525|15|42|5|PM|second|afternoon|| +56526|AAAAAAAAPMMNAAAA|56526|15|42|6|PM|second|afternoon|| +56527|AAAAAAAAANMNAAAA|56527|15|42|7|PM|second|afternoon|| +56528|AAAAAAAABNMNAAAA|56528|15|42|8|PM|second|afternoon|| +56529|AAAAAAAACNMNAAAA|56529|15|42|9|PM|second|afternoon|| +56530|AAAAAAAADNMNAAAA|56530|15|42|10|PM|second|afternoon|| +56531|AAAAAAAAENMNAAAA|56531|15|42|11|PM|second|afternoon|| +56532|AAAAAAAAFNMNAAAA|56532|15|42|12|PM|second|afternoon|| +56533|AAAAAAAAGNMNAAAA|56533|15|42|13|PM|second|afternoon|| +56534|AAAAAAAAHNMNAAAA|56534|15|42|14|PM|second|afternoon|| +56535|AAAAAAAAINMNAAAA|56535|15|42|15|PM|second|afternoon|| +56536|AAAAAAAAJNMNAAAA|56536|15|42|16|PM|second|afternoon|| +56537|AAAAAAAAKNMNAAAA|56537|15|42|17|PM|second|afternoon|| +56538|AAAAAAAALNMNAAAA|56538|15|42|18|PM|second|afternoon|| +56539|AAAAAAAAMNMNAAAA|56539|15|42|19|PM|second|afternoon|| +56540|AAAAAAAANNMNAAAA|56540|15|42|20|PM|second|afternoon|| +56541|AAAAAAAAONMNAAAA|56541|15|42|21|PM|second|afternoon|| +56542|AAAAAAAAPNMNAAAA|56542|15|42|22|PM|second|afternoon|| +56543|AAAAAAAAAOMNAAAA|56543|15|42|23|PM|second|afternoon|| +56544|AAAAAAAABOMNAAAA|56544|15|42|24|PM|second|afternoon|| +56545|AAAAAAAACOMNAAAA|56545|15|42|25|PM|second|afternoon|| +56546|AAAAAAAADOMNAAAA|56546|15|42|26|PM|second|afternoon|| +56547|AAAAAAAAEOMNAAAA|56547|15|42|27|PM|second|afternoon|| +56548|AAAAAAAAFOMNAAAA|56548|15|42|28|PM|second|afternoon|| +56549|AAAAAAAAGOMNAAAA|56549|15|42|29|PM|second|afternoon|| +56550|AAAAAAAAHOMNAAAA|56550|15|42|30|PM|second|afternoon|| +56551|AAAAAAAAIOMNAAAA|56551|15|42|31|PM|second|afternoon|| +56552|AAAAAAAAJOMNAAAA|56552|15|42|32|PM|second|afternoon|| +56553|AAAAAAAAKOMNAAAA|56553|15|42|33|PM|second|afternoon|| +56554|AAAAAAAALOMNAAAA|56554|15|42|34|PM|second|afternoon|| +56555|AAAAAAAAMOMNAAAA|56555|15|42|35|PM|second|afternoon|| +56556|AAAAAAAANOMNAAAA|56556|15|42|36|PM|second|afternoon|| +56557|AAAAAAAAOOMNAAAA|56557|15|42|37|PM|second|afternoon|| +56558|AAAAAAAAPOMNAAAA|56558|15|42|38|PM|second|afternoon|| +56559|AAAAAAAAAPMNAAAA|56559|15|42|39|PM|second|afternoon|| +56560|AAAAAAAABPMNAAAA|56560|15|42|40|PM|second|afternoon|| +56561|AAAAAAAACPMNAAAA|56561|15|42|41|PM|second|afternoon|| +56562|AAAAAAAADPMNAAAA|56562|15|42|42|PM|second|afternoon|| +56563|AAAAAAAAEPMNAAAA|56563|15|42|43|PM|second|afternoon|| +56564|AAAAAAAAFPMNAAAA|56564|15|42|44|PM|second|afternoon|| +56565|AAAAAAAAGPMNAAAA|56565|15|42|45|PM|second|afternoon|| +56566|AAAAAAAAHPMNAAAA|56566|15|42|46|PM|second|afternoon|| +56567|AAAAAAAAIPMNAAAA|56567|15|42|47|PM|second|afternoon|| +56568|AAAAAAAAJPMNAAAA|56568|15|42|48|PM|second|afternoon|| +56569|AAAAAAAAKPMNAAAA|56569|15|42|49|PM|second|afternoon|| +56570|AAAAAAAALPMNAAAA|56570|15|42|50|PM|second|afternoon|| +56571|AAAAAAAAMPMNAAAA|56571|15|42|51|PM|second|afternoon|| +56572|AAAAAAAANPMNAAAA|56572|15|42|52|PM|second|afternoon|| +56573|AAAAAAAAOPMNAAAA|56573|15|42|53|PM|second|afternoon|| +56574|AAAAAAAAPPMNAAAA|56574|15|42|54|PM|second|afternoon|| +56575|AAAAAAAAAANNAAAA|56575|15|42|55|PM|second|afternoon|| +56576|AAAAAAAABANNAAAA|56576|15|42|56|PM|second|afternoon|| +56577|AAAAAAAACANNAAAA|56577|15|42|57|PM|second|afternoon|| +56578|AAAAAAAADANNAAAA|56578|15|42|58|PM|second|afternoon|| +56579|AAAAAAAAEANNAAAA|56579|15|42|59|PM|second|afternoon|| +56580|AAAAAAAAFANNAAAA|56580|15|43|0|PM|second|afternoon|| +56581|AAAAAAAAGANNAAAA|56581|15|43|1|PM|second|afternoon|| +56582|AAAAAAAAHANNAAAA|56582|15|43|2|PM|second|afternoon|| +56583|AAAAAAAAIANNAAAA|56583|15|43|3|PM|second|afternoon|| +56584|AAAAAAAAJANNAAAA|56584|15|43|4|PM|second|afternoon|| +56585|AAAAAAAAKANNAAAA|56585|15|43|5|PM|second|afternoon|| +56586|AAAAAAAALANNAAAA|56586|15|43|6|PM|second|afternoon|| +56587|AAAAAAAAMANNAAAA|56587|15|43|7|PM|second|afternoon|| +56588|AAAAAAAANANNAAAA|56588|15|43|8|PM|second|afternoon|| +56589|AAAAAAAAOANNAAAA|56589|15|43|9|PM|second|afternoon|| +56590|AAAAAAAAPANNAAAA|56590|15|43|10|PM|second|afternoon|| +56591|AAAAAAAAABNNAAAA|56591|15|43|11|PM|second|afternoon|| +56592|AAAAAAAABBNNAAAA|56592|15|43|12|PM|second|afternoon|| +56593|AAAAAAAACBNNAAAA|56593|15|43|13|PM|second|afternoon|| +56594|AAAAAAAADBNNAAAA|56594|15|43|14|PM|second|afternoon|| +56595|AAAAAAAAEBNNAAAA|56595|15|43|15|PM|second|afternoon|| +56596|AAAAAAAAFBNNAAAA|56596|15|43|16|PM|second|afternoon|| +56597|AAAAAAAAGBNNAAAA|56597|15|43|17|PM|second|afternoon|| +56598|AAAAAAAAHBNNAAAA|56598|15|43|18|PM|second|afternoon|| +56599|AAAAAAAAIBNNAAAA|56599|15|43|19|PM|second|afternoon|| +56600|AAAAAAAAJBNNAAAA|56600|15|43|20|PM|second|afternoon|| +56601|AAAAAAAAKBNNAAAA|56601|15|43|21|PM|second|afternoon|| +56602|AAAAAAAALBNNAAAA|56602|15|43|22|PM|second|afternoon|| +56603|AAAAAAAAMBNNAAAA|56603|15|43|23|PM|second|afternoon|| +56604|AAAAAAAANBNNAAAA|56604|15|43|24|PM|second|afternoon|| +56605|AAAAAAAAOBNNAAAA|56605|15|43|25|PM|second|afternoon|| +56606|AAAAAAAAPBNNAAAA|56606|15|43|26|PM|second|afternoon|| +56607|AAAAAAAAACNNAAAA|56607|15|43|27|PM|second|afternoon|| +56608|AAAAAAAABCNNAAAA|56608|15|43|28|PM|second|afternoon|| +56609|AAAAAAAACCNNAAAA|56609|15|43|29|PM|second|afternoon|| +56610|AAAAAAAADCNNAAAA|56610|15|43|30|PM|second|afternoon|| +56611|AAAAAAAAECNNAAAA|56611|15|43|31|PM|second|afternoon|| +56612|AAAAAAAAFCNNAAAA|56612|15|43|32|PM|second|afternoon|| +56613|AAAAAAAAGCNNAAAA|56613|15|43|33|PM|second|afternoon|| +56614|AAAAAAAAHCNNAAAA|56614|15|43|34|PM|second|afternoon|| +56615|AAAAAAAAICNNAAAA|56615|15|43|35|PM|second|afternoon|| +56616|AAAAAAAAJCNNAAAA|56616|15|43|36|PM|second|afternoon|| +56617|AAAAAAAAKCNNAAAA|56617|15|43|37|PM|second|afternoon|| +56618|AAAAAAAALCNNAAAA|56618|15|43|38|PM|second|afternoon|| +56619|AAAAAAAAMCNNAAAA|56619|15|43|39|PM|second|afternoon|| +56620|AAAAAAAANCNNAAAA|56620|15|43|40|PM|second|afternoon|| +56621|AAAAAAAAOCNNAAAA|56621|15|43|41|PM|second|afternoon|| +56622|AAAAAAAAPCNNAAAA|56622|15|43|42|PM|second|afternoon|| +56623|AAAAAAAAADNNAAAA|56623|15|43|43|PM|second|afternoon|| +56624|AAAAAAAABDNNAAAA|56624|15|43|44|PM|second|afternoon|| +56625|AAAAAAAACDNNAAAA|56625|15|43|45|PM|second|afternoon|| +56626|AAAAAAAADDNNAAAA|56626|15|43|46|PM|second|afternoon|| +56627|AAAAAAAAEDNNAAAA|56627|15|43|47|PM|second|afternoon|| +56628|AAAAAAAAFDNNAAAA|56628|15|43|48|PM|second|afternoon|| +56629|AAAAAAAAGDNNAAAA|56629|15|43|49|PM|second|afternoon|| +56630|AAAAAAAAHDNNAAAA|56630|15|43|50|PM|second|afternoon|| +56631|AAAAAAAAIDNNAAAA|56631|15|43|51|PM|second|afternoon|| +56632|AAAAAAAAJDNNAAAA|56632|15|43|52|PM|second|afternoon|| +56633|AAAAAAAAKDNNAAAA|56633|15|43|53|PM|second|afternoon|| +56634|AAAAAAAALDNNAAAA|56634|15|43|54|PM|second|afternoon|| +56635|AAAAAAAAMDNNAAAA|56635|15|43|55|PM|second|afternoon|| +56636|AAAAAAAANDNNAAAA|56636|15|43|56|PM|second|afternoon|| +56637|AAAAAAAAODNNAAAA|56637|15|43|57|PM|second|afternoon|| +56638|AAAAAAAAPDNNAAAA|56638|15|43|58|PM|second|afternoon|| +56639|AAAAAAAAAENNAAAA|56639|15|43|59|PM|second|afternoon|| +56640|AAAAAAAABENNAAAA|56640|15|44|0|PM|second|afternoon|| +56641|AAAAAAAACENNAAAA|56641|15|44|1|PM|second|afternoon|| +56642|AAAAAAAADENNAAAA|56642|15|44|2|PM|second|afternoon|| +56643|AAAAAAAAEENNAAAA|56643|15|44|3|PM|second|afternoon|| +56644|AAAAAAAAFENNAAAA|56644|15|44|4|PM|second|afternoon|| +56645|AAAAAAAAGENNAAAA|56645|15|44|5|PM|second|afternoon|| +56646|AAAAAAAAHENNAAAA|56646|15|44|6|PM|second|afternoon|| +56647|AAAAAAAAIENNAAAA|56647|15|44|7|PM|second|afternoon|| +56648|AAAAAAAAJENNAAAA|56648|15|44|8|PM|second|afternoon|| +56649|AAAAAAAAKENNAAAA|56649|15|44|9|PM|second|afternoon|| +56650|AAAAAAAALENNAAAA|56650|15|44|10|PM|second|afternoon|| +56651|AAAAAAAAMENNAAAA|56651|15|44|11|PM|second|afternoon|| +56652|AAAAAAAANENNAAAA|56652|15|44|12|PM|second|afternoon|| +56653|AAAAAAAAOENNAAAA|56653|15|44|13|PM|second|afternoon|| +56654|AAAAAAAAPENNAAAA|56654|15|44|14|PM|second|afternoon|| +56655|AAAAAAAAAFNNAAAA|56655|15|44|15|PM|second|afternoon|| +56656|AAAAAAAABFNNAAAA|56656|15|44|16|PM|second|afternoon|| +56657|AAAAAAAACFNNAAAA|56657|15|44|17|PM|second|afternoon|| +56658|AAAAAAAADFNNAAAA|56658|15|44|18|PM|second|afternoon|| +56659|AAAAAAAAEFNNAAAA|56659|15|44|19|PM|second|afternoon|| +56660|AAAAAAAAFFNNAAAA|56660|15|44|20|PM|second|afternoon|| +56661|AAAAAAAAGFNNAAAA|56661|15|44|21|PM|second|afternoon|| +56662|AAAAAAAAHFNNAAAA|56662|15|44|22|PM|second|afternoon|| +56663|AAAAAAAAIFNNAAAA|56663|15|44|23|PM|second|afternoon|| +56664|AAAAAAAAJFNNAAAA|56664|15|44|24|PM|second|afternoon|| +56665|AAAAAAAAKFNNAAAA|56665|15|44|25|PM|second|afternoon|| +56666|AAAAAAAALFNNAAAA|56666|15|44|26|PM|second|afternoon|| +56667|AAAAAAAAMFNNAAAA|56667|15|44|27|PM|second|afternoon|| +56668|AAAAAAAANFNNAAAA|56668|15|44|28|PM|second|afternoon|| +56669|AAAAAAAAOFNNAAAA|56669|15|44|29|PM|second|afternoon|| +56670|AAAAAAAAPFNNAAAA|56670|15|44|30|PM|second|afternoon|| +56671|AAAAAAAAAGNNAAAA|56671|15|44|31|PM|second|afternoon|| +56672|AAAAAAAABGNNAAAA|56672|15|44|32|PM|second|afternoon|| +56673|AAAAAAAACGNNAAAA|56673|15|44|33|PM|second|afternoon|| +56674|AAAAAAAADGNNAAAA|56674|15|44|34|PM|second|afternoon|| +56675|AAAAAAAAEGNNAAAA|56675|15|44|35|PM|second|afternoon|| +56676|AAAAAAAAFGNNAAAA|56676|15|44|36|PM|second|afternoon|| +56677|AAAAAAAAGGNNAAAA|56677|15|44|37|PM|second|afternoon|| +56678|AAAAAAAAHGNNAAAA|56678|15|44|38|PM|second|afternoon|| +56679|AAAAAAAAIGNNAAAA|56679|15|44|39|PM|second|afternoon|| +56680|AAAAAAAAJGNNAAAA|56680|15|44|40|PM|second|afternoon|| +56681|AAAAAAAAKGNNAAAA|56681|15|44|41|PM|second|afternoon|| +56682|AAAAAAAALGNNAAAA|56682|15|44|42|PM|second|afternoon|| +56683|AAAAAAAAMGNNAAAA|56683|15|44|43|PM|second|afternoon|| +56684|AAAAAAAANGNNAAAA|56684|15|44|44|PM|second|afternoon|| +56685|AAAAAAAAOGNNAAAA|56685|15|44|45|PM|second|afternoon|| +56686|AAAAAAAAPGNNAAAA|56686|15|44|46|PM|second|afternoon|| +56687|AAAAAAAAAHNNAAAA|56687|15|44|47|PM|second|afternoon|| +56688|AAAAAAAABHNNAAAA|56688|15|44|48|PM|second|afternoon|| +56689|AAAAAAAACHNNAAAA|56689|15|44|49|PM|second|afternoon|| +56690|AAAAAAAADHNNAAAA|56690|15|44|50|PM|second|afternoon|| +56691|AAAAAAAAEHNNAAAA|56691|15|44|51|PM|second|afternoon|| +56692|AAAAAAAAFHNNAAAA|56692|15|44|52|PM|second|afternoon|| +56693|AAAAAAAAGHNNAAAA|56693|15|44|53|PM|second|afternoon|| +56694|AAAAAAAAHHNNAAAA|56694|15|44|54|PM|second|afternoon|| +56695|AAAAAAAAIHNNAAAA|56695|15|44|55|PM|second|afternoon|| +56696|AAAAAAAAJHNNAAAA|56696|15|44|56|PM|second|afternoon|| +56697|AAAAAAAAKHNNAAAA|56697|15|44|57|PM|second|afternoon|| +56698|AAAAAAAALHNNAAAA|56698|15|44|58|PM|second|afternoon|| +56699|AAAAAAAAMHNNAAAA|56699|15|44|59|PM|second|afternoon|| +56700|AAAAAAAANHNNAAAA|56700|15|45|0|PM|second|afternoon|| +56701|AAAAAAAAOHNNAAAA|56701|15|45|1|PM|second|afternoon|| +56702|AAAAAAAAPHNNAAAA|56702|15|45|2|PM|second|afternoon|| +56703|AAAAAAAAAINNAAAA|56703|15|45|3|PM|second|afternoon|| +56704|AAAAAAAABINNAAAA|56704|15|45|4|PM|second|afternoon|| +56705|AAAAAAAACINNAAAA|56705|15|45|5|PM|second|afternoon|| +56706|AAAAAAAADINNAAAA|56706|15|45|6|PM|second|afternoon|| +56707|AAAAAAAAEINNAAAA|56707|15|45|7|PM|second|afternoon|| +56708|AAAAAAAAFINNAAAA|56708|15|45|8|PM|second|afternoon|| +56709|AAAAAAAAGINNAAAA|56709|15|45|9|PM|second|afternoon|| +56710|AAAAAAAAHINNAAAA|56710|15|45|10|PM|second|afternoon|| +56711|AAAAAAAAIINNAAAA|56711|15|45|11|PM|second|afternoon|| +56712|AAAAAAAAJINNAAAA|56712|15|45|12|PM|second|afternoon|| +56713|AAAAAAAAKINNAAAA|56713|15|45|13|PM|second|afternoon|| +56714|AAAAAAAALINNAAAA|56714|15|45|14|PM|second|afternoon|| +56715|AAAAAAAAMINNAAAA|56715|15|45|15|PM|second|afternoon|| +56716|AAAAAAAANINNAAAA|56716|15|45|16|PM|second|afternoon|| +56717|AAAAAAAAOINNAAAA|56717|15|45|17|PM|second|afternoon|| +56718|AAAAAAAAPINNAAAA|56718|15|45|18|PM|second|afternoon|| +56719|AAAAAAAAAJNNAAAA|56719|15|45|19|PM|second|afternoon|| +56720|AAAAAAAABJNNAAAA|56720|15|45|20|PM|second|afternoon|| +56721|AAAAAAAACJNNAAAA|56721|15|45|21|PM|second|afternoon|| +56722|AAAAAAAADJNNAAAA|56722|15|45|22|PM|second|afternoon|| +56723|AAAAAAAAEJNNAAAA|56723|15|45|23|PM|second|afternoon|| +56724|AAAAAAAAFJNNAAAA|56724|15|45|24|PM|second|afternoon|| +56725|AAAAAAAAGJNNAAAA|56725|15|45|25|PM|second|afternoon|| +56726|AAAAAAAAHJNNAAAA|56726|15|45|26|PM|second|afternoon|| +56727|AAAAAAAAIJNNAAAA|56727|15|45|27|PM|second|afternoon|| +56728|AAAAAAAAJJNNAAAA|56728|15|45|28|PM|second|afternoon|| +56729|AAAAAAAAKJNNAAAA|56729|15|45|29|PM|second|afternoon|| +56730|AAAAAAAALJNNAAAA|56730|15|45|30|PM|second|afternoon|| +56731|AAAAAAAAMJNNAAAA|56731|15|45|31|PM|second|afternoon|| +56732|AAAAAAAANJNNAAAA|56732|15|45|32|PM|second|afternoon|| +56733|AAAAAAAAOJNNAAAA|56733|15|45|33|PM|second|afternoon|| +56734|AAAAAAAAPJNNAAAA|56734|15|45|34|PM|second|afternoon|| +56735|AAAAAAAAAKNNAAAA|56735|15|45|35|PM|second|afternoon|| +56736|AAAAAAAABKNNAAAA|56736|15|45|36|PM|second|afternoon|| +56737|AAAAAAAACKNNAAAA|56737|15|45|37|PM|second|afternoon|| +56738|AAAAAAAADKNNAAAA|56738|15|45|38|PM|second|afternoon|| +56739|AAAAAAAAEKNNAAAA|56739|15|45|39|PM|second|afternoon|| +56740|AAAAAAAAFKNNAAAA|56740|15|45|40|PM|second|afternoon|| +56741|AAAAAAAAGKNNAAAA|56741|15|45|41|PM|second|afternoon|| +56742|AAAAAAAAHKNNAAAA|56742|15|45|42|PM|second|afternoon|| +56743|AAAAAAAAIKNNAAAA|56743|15|45|43|PM|second|afternoon|| +56744|AAAAAAAAJKNNAAAA|56744|15|45|44|PM|second|afternoon|| +56745|AAAAAAAAKKNNAAAA|56745|15|45|45|PM|second|afternoon|| +56746|AAAAAAAALKNNAAAA|56746|15|45|46|PM|second|afternoon|| +56747|AAAAAAAAMKNNAAAA|56747|15|45|47|PM|second|afternoon|| +56748|AAAAAAAANKNNAAAA|56748|15|45|48|PM|second|afternoon|| +56749|AAAAAAAAOKNNAAAA|56749|15|45|49|PM|second|afternoon|| +56750|AAAAAAAAPKNNAAAA|56750|15|45|50|PM|second|afternoon|| +56751|AAAAAAAAALNNAAAA|56751|15|45|51|PM|second|afternoon|| +56752|AAAAAAAABLNNAAAA|56752|15|45|52|PM|second|afternoon|| +56753|AAAAAAAACLNNAAAA|56753|15|45|53|PM|second|afternoon|| +56754|AAAAAAAADLNNAAAA|56754|15|45|54|PM|second|afternoon|| +56755|AAAAAAAAELNNAAAA|56755|15|45|55|PM|second|afternoon|| +56756|AAAAAAAAFLNNAAAA|56756|15|45|56|PM|second|afternoon|| +56757|AAAAAAAAGLNNAAAA|56757|15|45|57|PM|second|afternoon|| +56758|AAAAAAAAHLNNAAAA|56758|15|45|58|PM|second|afternoon|| +56759|AAAAAAAAILNNAAAA|56759|15|45|59|PM|second|afternoon|| +56760|AAAAAAAAJLNNAAAA|56760|15|46|0|PM|second|afternoon|| +56761|AAAAAAAAKLNNAAAA|56761|15|46|1|PM|second|afternoon|| +56762|AAAAAAAALLNNAAAA|56762|15|46|2|PM|second|afternoon|| +56763|AAAAAAAAMLNNAAAA|56763|15|46|3|PM|second|afternoon|| +56764|AAAAAAAANLNNAAAA|56764|15|46|4|PM|second|afternoon|| +56765|AAAAAAAAOLNNAAAA|56765|15|46|5|PM|second|afternoon|| +56766|AAAAAAAAPLNNAAAA|56766|15|46|6|PM|second|afternoon|| +56767|AAAAAAAAAMNNAAAA|56767|15|46|7|PM|second|afternoon|| +56768|AAAAAAAABMNNAAAA|56768|15|46|8|PM|second|afternoon|| +56769|AAAAAAAACMNNAAAA|56769|15|46|9|PM|second|afternoon|| +56770|AAAAAAAADMNNAAAA|56770|15|46|10|PM|second|afternoon|| +56771|AAAAAAAAEMNNAAAA|56771|15|46|11|PM|second|afternoon|| +56772|AAAAAAAAFMNNAAAA|56772|15|46|12|PM|second|afternoon|| +56773|AAAAAAAAGMNNAAAA|56773|15|46|13|PM|second|afternoon|| +56774|AAAAAAAAHMNNAAAA|56774|15|46|14|PM|second|afternoon|| +56775|AAAAAAAAIMNNAAAA|56775|15|46|15|PM|second|afternoon|| +56776|AAAAAAAAJMNNAAAA|56776|15|46|16|PM|second|afternoon|| +56777|AAAAAAAAKMNNAAAA|56777|15|46|17|PM|second|afternoon|| +56778|AAAAAAAALMNNAAAA|56778|15|46|18|PM|second|afternoon|| +56779|AAAAAAAAMMNNAAAA|56779|15|46|19|PM|second|afternoon|| +56780|AAAAAAAANMNNAAAA|56780|15|46|20|PM|second|afternoon|| +56781|AAAAAAAAOMNNAAAA|56781|15|46|21|PM|second|afternoon|| +56782|AAAAAAAAPMNNAAAA|56782|15|46|22|PM|second|afternoon|| +56783|AAAAAAAAANNNAAAA|56783|15|46|23|PM|second|afternoon|| +56784|AAAAAAAABNNNAAAA|56784|15|46|24|PM|second|afternoon|| +56785|AAAAAAAACNNNAAAA|56785|15|46|25|PM|second|afternoon|| +56786|AAAAAAAADNNNAAAA|56786|15|46|26|PM|second|afternoon|| +56787|AAAAAAAAENNNAAAA|56787|15|46|27|PM|second|afternoon|| +56788|AAAAAAAAFNNNAAAA|56788|15|46|28|PM|second|afternoon|| +56789|AAAAAAAAGNNNAAAA|56789|15|46|29|PM|second|afternoon|| +56790|AAAAAAAAHNNNAAAA|56790|15|46|30|PM|second|afternoon|| +56791|AAAAAAAAINNNAAAA|56791|15|46|31|PM|second|afternoon|| +56792|AAAAAAAAJNNNAAAA|56792|15|46|32|PM|second|afternoon|| +56793|AAAAAAAAKNNNAAAA|56793|15|46|33|PM|second|afternoon|| +56794|AAAAAAAALNNNAAAA|56794|15|46|34|PM|second|afternoon|| +56795|AAAAAAAAMNNNAAAA|56795|15|46|35|PM|second|afternoon|| +56796|AAAAAAAANNNNAAAA|56796|15|46|36|PM|second|afternoon|| +56797|AAAAAAAAONNNAAAA|56797|15|46|37|PM|second|afternoon|| +56798|AAAAAAAAPNNNAAAA|56798|15|46|38|PM|second|afternoon|| +56799|AAAAAAAAAONNAAAA|56799|15|46|39|PM|second|afternoon|| +56800|AAAAAAAABONNAAAA|56800|15|46|40|PM|second|afternoon|| +56801|AAAAAAAACONNAAAA|56801|15|46|41|PM|second|afternoon|| +56802|AAAAAAAADONNAAAA|56802|15|46|42|PM|second|afternoon|| +56803|AAAAAAAAEONNAAAA|56803|15|46|43|PM|second|afternoon|| +56804|AAAAAAAAFONNAAAA|56804|15|46|44|PM|second|afternoon|| +56805|AAAAAAAAGONNAAAA|56805|15|46|45|PM|second|afternoon|| +56806|AAAAAAAAHONNAAAA|56806|15|46|46|PM|second|afternoon|| +56807|AAAAAAAAIONNAAAA|56807|15|46|47|PM|second|afternoon|| +56808|AAAAAAAAJONNAAAA|56808|15|46|48|PM|second|afternoon|| +56809|AAAAAAAAKONNAAAA|56809|15|46|49|PM|second|afternoon|| +56810|AAAAAAAALONNAAAA|56810|15|46|50|PM|second|afternoon|| +56811|AAAAAAAAMONNAAAA|56811|15|46|51|PM|second|afternoon|| +56812|AAAAAAAANONNAAAA|56812|15|46|52|PM|second|afternoon|| +56813|AAAAAAAAOONNAAAA|56813|15|46|53|PM|second|afternoon|| +56814|AAAAAAAAPONNAAAA|56814|15|46|54|PM|second|afternoon|| +56815|AAAAAAAAAPNNAAAA|56815|15|46|55|PM|second|afternoon|| +56816|AAAAAAAABPNNAAAA|56816|15|46|56|PM|second|afternoon|| +56817|AAAAAAAACPNNAAAA|56817|15|46|57|PM|second|afternoon|| +56818|AAAAAAAADPNNAAAA|56818|15|46|58|PM|second|afternoon|| +56819|AAAAAAAAEPNNAAAA|56819|15|46|59|PM|second|afternoon|| +56820|AAAAAAAAFPNNAAAA|56820|15|47|0|PM|second|afternoon|| +56821|AAAAAAAAGPNNAAAA|56821|15|47|1|PM|second|afternoon|| +56822|AAAAAAAAHPNNAAAA|56822|15|47|2|PM|second|afternoon|| +56823|AAAAAAAAIPNNAAAA|56823|15|47|3|PM|second|afternoon|| +56824|AAAAAAAAJPNNAAAA|56824|15|47|4|PM|second|afternoon|| +56825|AAAAAAAAKPNNAAAA|56825|15|47|5|PM|second|afternoon|| +56826|AAAAAAAALPNNAAAA|56826|15|47|6|PM|second|afternoon|| +56827|AAAAAAAAMPNNAAAA|56827|15|47|7|PM|second|afternoon|| +56828|AAAAAAAANPNNAAAA|56828|15|47|8|PM|second|afternoon|| +56829|AAAAAAAAOPNNAAAA|56829|15|47|9|PM|second|afternoon|| +56830|AAAAAAAAPPNNAAAA|56830|15|47|10|PM|second|afternoon|| +56831|AAAAAAAAAAONAAAA|56831|15|47|11|PM|second|afternoon|| +56832|AAAAAAAABAONAAAA|56832|15|47|12|PM|second|afternoon|| +56833|AAAAAAAACAONAAAA|56833|15|47|13|PM|second|afternoon|| +56834|AAAAAAAADAONAAAA|56834|15|47|14|PM|second|afternoon|| +56835|AAAAAAAAEAONAAAA|56835|15|47|15|PM|second|afternoon|| +56836|AAAAAAAAFAONAAAA|56836|15|47|16|PM|second|afternoon|| +56837|AAAAAAAAGAONAAAA|56837|15|47|17|PM|second|afternoon|| +56838|AAAAAAAAHAONAAAA|56838|15|47|18|PM|second|afternoon|| +56839|AAAAAAAAIAONAAAA|56839|15|47|19|PM|second|afternoon|| +56840|AAAAAAAAJAONAAAA|56840|15|47|20|PM|second|afternoon|| +56841|AAAAAAAAKAONAAAA|56841|15|47|21|PM|second|afternoon|| +56842|AAAAAAAALAONAAAA|56842|15|47|22|PM|second|afternoon|| +56843|AAAAAAAAMAONAAAA|56843|15|47|23|PM|second|afternoon|| +56844|AAAAAAAANAONAAAA|56844|15|47|24|PM|second|afternoon|| +56845|AAAAAAAAOAONAAAA|56845|15|47|25|PM|second|afternoon|| +56846|AAAAAAAAPAONAAAA|56846|15|47|26|PM|second|afternoon|| +56847|AAAAAAAAABONAAAA|56847|15|47|27|PM|second|afternoon|| +56848|AAAAAAAABBONAAAA|56848|15|47|28|PM|second|afternoon|| +56849|AAAAAAAACBONAAAA|56849|15|47|29|PM|second|afternoon|| +56850|AAAAAAAADBONAAAA|56850|15|47|30|PM|second|afternoon|| +56851|AAAAAAAAEBONAAAA|56851|15|47|31|PM|second|afternoon|| +56852|AAAAAAAAFBONAAAA|56852|15|47|32|PM|second|afternoon|| +56853|AAAAAAAAGBONAAAA|56853|15|47|33|PM|second|afternoon|| +56854|AAAAAAAAHBONAAAA|56854|15|47|34|PM|second|afternoon|| +56855|AAAAAAAAIBONAAAA|56855|15|47|35|PM|second|afternoon|| +56856|AAAAAAAAJBONAAAA|56856|15|47|36|PM|second|afternoon|| +56857|AAAAAAAAKBONAAAA|56857|15|47|37|PM|second|afternoon|| +56858|AAAAAAAALBONAAAA|56858|15|47|38|PM|second|afternoon|| +56859|AAAAAAAAMBONAAAA|56859|15|47|39|PM|second|afternoon|| +56860|AAAAAAAANBONAAAA|56860|15|47|40|PM|second|afternoon|| +56861|AAAAAAAAOBONAAAA|56861|15|47|41|PM|second|afternoon|| +56862|AAAAAAAAPBONAAAA|56862|15|47|42|PM|second|afternoon|| +56863|AAAAAAAAACONAAAA|56863|15|47|43|PM|second|afternoon|| +56864|AAAAAAAABCONAAAA|56864|15|47|44|PM|second|afternoon|| +56865|AAAAAAAACCONAAAA|56865|15|47|45|PM|second|afternoon|| +56866|AAAAAAAADCONAAAA|56866|15|47|46|PM|second|afternoon|| +56867|AAAAAAAAECONAAAA|56867|15|47|47|PM|second|afternoon|| +56868|AAAAAAAAFCONAAAA|56868|15|47|48|PM|second|afternoon|| +56869|AAAAAAAAGCONAAAA|56869|15|47|49|PM|second|afternoon|| +56870|AAAAAAAAHCONAAAA|56870|15|47|50|PM|second|afternoon|| +56871|AAAAAAAAICONAAAA|56871|15|47|51|PM|second|afternoon|| +56872|AAAAAAAAJCONAAAA|56872|15|47|52|PM|second|afternoon|| +56873|AAAAAAAAKCONAAAA|56873|15|47|53|PM|second|afternoon|| +56874|AAAAAAAALCONAAAA|56874|15|47|54|PM|second|afternoon|| +56875|AAAAAAAAMCONAAAA|56875|15|47|55|PM|second|afternoon|| +56876|AAAAAAAANCONAAAA|56876|15|47|56|PM|second|afternoon|| +56877|AAAAAAAAOCONAAAA|56877|15|47|57|PM|second|afternoon|| +56878|AAAAAAAAPCONAAAA|56878|15|47|58|PM|second|afternoon|| +56879|AAAAAAAAADONAAAA|56879|15|47|59|PM|second|afternoon|| +56880|AAAAAAAABDONAAAA|56880|15|48|0|PM|second|afternoon|| +56881|AAAAAAAACDONAAAA|56881|15|48|1|PM|second|afternoon|| +56882|AAAAAAAADDONAAAA|56882|15|48|2|PM|second|afternoon|| +56883|AAAAAAAAEDONAAAA|56883|15|48|3|PM|second|afternoon|| +56884|AAAAAAAAFDONAAAA|56884|15|48|4|PM|second|afternoon|| +56885|AAAAAAAAGDONAAAA|56885|15|48|5|PM|second|afternoon|| +56886|AAAAAAAAHDONAAAA|56886|15|48|6|PM|second|afternoon|| +56887|AAAAAAAAIDONAAAA|56887|15|48|7|PM|second|afternoon|| +56888|AAAAAAAAJDONAAAA|56888|15|48|8|PM|second|afternoon|| +56889|AAAAAAAAKDONAAAA|56889|15|48|9|PM|second|afternoon|| +56890|AAAAAAAALDONAAAA|56890|15|48|10|PM|second|afternoon|| +56891|AAAAAAAAMDONAAAA|56891|15|48|11|PM|second|afternoon|| +56892|AAAAAAAANDONAAAA|56892|15|48|12|PM|second|afternoon|| +56893|AAAAAAAAODONAAAA|56893|15|48|13|PM|second|afternoon|| +56894|AAAAAAAAPDONAAAA|56894|15|48|14|PM|second|afternoon|| +56895|AAAAAAAAAEONAAAA|56895|15|48|15|PM|second|afternoon|| +56896|AAAAAAAABEONAAAA|56896|15|48|16|PM|second|afternoon|| +56897|AAAAAAAACEONAAAA|56897|15|48|17|PM|second|afternoon|| +56898|AAAAAAAADEONAAAA|56898|15|48|18|PM|second|afternoon|| +56899|AAAAAAAAEEONAAAA|56899|15|48|19|PM|second|afternoon|| +56900|AAAAAAAAFEONAAAA|56900|15|48|20|PM|second|afternoon|| +56901|AAAAAAAAGEONAAAA|56901|15|48|21|PM|second|afternoon|| +56902|AAAAAAAAHEONAAAA|56902|15|48|22|PM|second|afternoon|| +56903|AAAAAAAAIEONAAAA|56903|15|48|23|PM|second|afternoon|| +56904|AAAAAAAAJEONAAAA|56904|15|48|24|PM|second|afternoon|| +56905|AAAAAAAAKEONAAAA|56905|15|48|25|PM|second|afternoon|| +56906|AAAAAAAALEONAAAA|56906|15|48|26|PM|second|afternoon|| +56907|AAAAAAAAMEONAAAA|56907|15|48|27|PM|second|afternoon|| +56908|AAAAAAAANEONAAAA|56908|15|48|28|PM|second|afternoon|| +56909|AAAAAAAAOEONAAAA|56909|15|48|29|PM|second|afternoon|| +56910|AAAAAAAAPEONAAAA|56910|15|48|30|PM|second|afternoon|| +56911|AAAAAAAAAFONAAAA|56911|15|48|31|PM|second|afternoon|| +56912|AAAAAAAABFONAAAA|56912|15|48|32|PM|second|afternoon|| +56913|AAAAAAAACFONAAAA|56913|15|48|33|PM|second|afternoon|| +56914|AAAAAAAADFONAAAA|56914|15|48|34|PM|second|afternoon|| +56915|AAAAAAAAEFONAAAA|56915|15|48|35|PM|second|afternoon|| +56916|AAAAAAAAFFONAAAA|56916|15|48|36|PM|second|afternoon|| +56917|AAAAAAAAGFONAAAA|56917|15|48|37|PM|second|afternoon|| +56918|AAAAAAAAHFONAAAA|56918|15|48|38|PM|second|afternoon|| +56919|AAAAAAAAIFONAAAA|56919|15|48|39|PM|second|afternoon|| +56920|AAAAAAAAJFONAAAA|56920|15|48|40|PM|second|afternoon|| +56921|AAAAAAAAKFONAAAA|56921|15|48|41|PM|second|afternoon|| +56922|AAAAAAAALFONAAAA|56922|15|48|42|PM|second|afternoon|| +56923|AAAAAAAAMFONAAAA|56923|15|48|43|PM|second|afternoon|| +56924|AAAAAAAANFONAAAA|56924|15|48|44|PM|second|afternoon|| +56925|AAAAAAAAOFONAAAA|56925|15|48|45|PM|second|afternoon|| +56926|AAAAAAAAPFONAAAA|56926|15|48|46|PM|second|afternoon|| +56927|AAAAAAAAAGONAAAA|56927|15|48|47|PM|second|afternoon|| +56928|AAAAAAAABGONAAAA|56928|15|48|48|PM|second|afternoon|| +56929|AAAAAAAACGONAAAA|56929|15|48|49|PM|second|afternoon|| +56930|AAAAAAAADGONAAAA|56930|15|48|50|PM|second|afternoon|| +56931|AAAAAAAAEGONAAAA|56931|15|48|51|PM|second|afternoon|| +56932|AAAAAAAAFGONAAAA|56932|15|48|52|PM|second|afternoon|| +56933|AAAAAAAAGGONAAAA|56933|15|48|53|PM|second|afternoon|| +56934|AAAAAAAAHGONAAAA|56934|15|48|54|PM|second|afternoon|| +56935|AAAAAAAAIGONAAAA|56935|15|48|55|PM|second|afternoon|| +56936|AAAAAAAAJGONAAAA|56936|15|48|56|PM|second|afternoon|| +56937|AAAAAAAAKGONAAAA|56937|15|48|57|PM|second|afternoon|| +56938|AAAAAAAALGONAAAA|56938|15|48|58|PM|second|afternoon|| +56939|AAAAAAAAMGONAAAA|56939|15|48|59|PM|second|afternoon|| +56940|AAAAAAAANGONAAAA|56940|15|49|0|PM|second|afternoon|| +56941|AAAAAAAAOGONAAAA|56941|15|49|1|PM|second|afternoon|| +56942|AAAAAAAAPGONAAAA|56942|15|49|2|PM|second|afternoon|| +56943|AAAAAAAAAHONAAAA|56943|15|49|3|PM|second|afternoon|| +56944|AAAAAAAABHONAAAA|56944|15|49|4|PM|second|afternoon|| +56945|AAAAAAAACHONAAAA|56945|15|49|5|PM|second|afternoon|| +56946|AAAAAAAADHONAAAA|56946|15|49|6|PM|second|afternoon|| +56947|AAAAAAAAEHONAAAA|56947|15|49|7|PM|second|afternoon|| +56948|AAAAAAAAFHONAAAA|56948|15|49|8|PM|second|afternoon|| +56949|AAAAAAAAGHONAAAA|56949|15|49|9|PM|second|afternoon|| +56950|AAAAAAAAHHONAAAA|56950|15|49|10|PM|second|afternoon|| +56951|AAAAAAAAIHONAAAA|56951|15|49|11|PM|second|afternoon|| +56952|AAAAAAAAJHONAAAA|56952|15|49|12|PM|second|afternoon|| +56953|AAAAAAAAKHONAAAA|56953|15|49|13|PM|second|afternoon|| +56954|AAAAAAAALHONAAAA|56954|15|49|14|PM|second|afternoon|| +56955|AAAAAAAAMHONAAAA|56955|15|49|15|PM|second|afternoon|| +56956|AAAAAAAANHONAAAA|56956|15|49|16|PM|second|afternoon|| +56957|AAAAAAAAOHONAAAA|56957|15|49|17|PM|second|afternoon|| +56958|AAAAAAAAPHONAAAA|56958|15|49|18|PM|second|afternoon|| +56959|AAAAAAAAAIONAAAA|56959|15|49|19|PM|second|afternoon|| +56960|AAAAAAAABIONAAAA|56960|15|49|20|PM|second|afternoon|| +56961|AAAAAAAACIONAAAA|56961|15|49|21|PM|second|afternoon|| +56962|AAAAAAAADIONAAAA|56962|15|49|22|PM|second|afternoon|| +56963|AAAAAAAAEIONAAAA|56963|15|49|23|PM|second|afternoon|| +56964|AAAAAAAAFIONAAAA|56964|15|49|24|PM|second|afternoon|| +56965|AAAAAAAAGIONAAAA|56965|15|49|25|PM|second|afternoon|| +56966|AAAAAAAAHIONAAAA|56966|15|49|26|PM|second|afternoon|| +56967|AAAAAAAAIIONAAAA|56967|15|49|27|PM|second|afternoon|| +56968|AAAAAAAAJIONAAAA|56968|15|49|28|PM|second|afternoon|| +56969|AAAAAAAAKIONAAAA|56969|15|49|29|PM|second|afternoon|| +56970|AAAAAAAALIONAAAA|56970|15|49|30|PM|second|afternoon|| +56971|AAAAAAAAMIONAAAA|56971|15|49|31|PM|second|afternoon|| +56972|AAAAAAAANIONAAAA|56972|15|49|32|PM|second|afternoon|| +56973|AAAAAAAAOIONAAAA|56973|15|49|33|PM|second|afternoon|| +56974|AAAAAAAAPIONAAAA|56974|15|49|34|PM|second|afternoon|| +56975|AAAAAAAAAJONAAAA|56975|15|49|35|PM|second|afternoon|| +56976|AAAAAAAABJONAAAA|56976|15|49|36|PM|second|afternoon|| +56977|AAAAAAAACJONAAAA|56977|15|49|37|PM|second|afternoon|| +56978|AAAAAAAADJONAAAA|56978|15|49|38|PM|second|afternoon|| +56979|AAAAAAAAEJONAAAA|56979|15|49|39|PM|second|afternoon|| +56980|AAAAAAAAFJONAAAA|56980|15|49|40|PM|second|afternoon|| +56981|AAAAAAAAGJONAAAA|56981|15|49|41|PM|second|afternoon|| +56982|AAAAAAAAHJONAAAA|56982|15|49|42|PM|second|afternoon|| +56983|AAAAAAAAIJONAAAA|56983|15|49|43|PM|second|afternoon|| +56984|AAAAAAAAJJONAAAA|56984|15|49|44|PM|second|afternoon|| +56985|AAAAAAAAKJONAAAA|56985|15|49|45|PM|second|afternoon|| +56986|AAAAAAAALJONAAAA|56986|15|49|46|PM|second|afternoon|| +56987|AAAAAAAAMJONAAAA|56987|15|49|47|PM|second|afternoon|| +56988|AAAAAAAANJONAAAA|56988|15|49|48|PM|second|afternoon|| +56989|AAAAAAAAOJONAAAA|56989|15|49|49|PM|second|afternoon|| +56990|AAAAAAAAPJONAAAA|56990|15|49|50|PM|second|afternoon|| +56991|AAAAAAAAAKONAAAA|56991|15|49|51|PM|second|afternoon|| +56992|AAAAAAAABKONAAAA|56992|15|49|52|PM|second|afternoon|| +56993|AAAAAAAACKONAAAA|56993|15|49|53|PM|second|afternoon|| +56994|AAAAAAAADKONAAAA|56994|15|49|54|PM|second|afternoon|| +56995|AAAAAAAAEKONAAAA|56995|15|49|55|PM|second|afternoon|| +56996|AAAAAAAAFKONAAAA|56996|15|49|56|PM|second|afternoon|| +56997|AAAAAAAAGKONAAAA|56997|15|49|57|PM|second|afternoon|| +56998|AAAAAAAAHKONAAAA|56998|15|49|58|PM|second|afternoon|| +56999|AAAAAAAAIKONAAAA|56999|15|49|59|PM|second|afternoon|| +57000|AAAAAAAAJKONAAAA|57000|15|50|0|PM|second|afternoon|| +57001|AAAAAAAAKKONAAAA|57001|15|50|1|PM|second|afternoon|| +57002|AAAAAAAALKONAAAA|57002|15|50|2|PM|second|afternoon|| +57003|AAAAAAAAMKONAAAA|57003|15|50|3|PM|second|afternoon|| +57004|AAAAAAAANKONAAAA|57004|15|50|4|PM|second|afternoon|| +57005|AAAAAAAAOKONAAAA|57005|15|50|5|PM|second|afternoon|| +57006|AAAAAAAAPKONAAAA|57006|15|50|6|PM|second|afternoon|| +57007|AAAAAAAAALONAAAA|57007|15|50|7|PM|second|afternoon|| +57008|AAAAAAAABLONAAAA|57008|15|50|8|PM|second|afternoon|| +57009|AAAAAAAACLONAAAA|57009|15|50|9|PM|second|afternoon|| +57010|AAAAAAAADLONAAAA|57010|15|50|10|PM|second|afternoon|| +57011|AAAAAAAAELONAAAA|57011|15|50|11|PM|second|afternoon|| +57012|AAAAAAAAFLONAAAA|57012|15|50|12|PM|second|afternoon|| +57013|AAAAAAAAGLONAAAA|57013|15|50|13|PM|second|afternoon|| +57014|AAAAAAAAHLONAAAA|57014|15|50|14|PM|second|afternoon|| +57015|AAAAAAAAILONAAAA|57015|15|50|15|PM|second|afternoon|| +57016|AAAAAAAAJLONAAAA|57016|15|50|16|PM|second|afternoon|| +57017|AAAAAAAAKLONAAAA|57017|15|50|17|PM|second|afternoon|| +57018|AAAAAAAALLONAAAA|57018|15|50|18|PM|second|afternoon|| +57019|AAAAAAAAMLONAAAA|57019|15|50|19|PM|second|afternoon|| +57020|AAAAAAAANLONAAAA|57020|15|50|20|PM|second|afternoon|| +57021|AAAAAAAAOLONAAAA|57021|15|50|21|PM|second|afternoon|| +57022|AAAAAAAAPLONAAAA|57022|15|50|22|PM|second|afternoon|| +57023|AAAAAAAAAMONAAAA|57023|15|50|23|PM|second|afternoon|| +57024|AAAAAAAABMONAAAA|57024|15|50|24|PM|second|afternoon|| +57025|AAAAAAAACMONAAAA|57025|15|50|25|PM|second|afternoon|| +57026|AAAAAAAADMONAAAA|57026|15|50|26|PM|second|afternoon|| +57027|AAAAAAAAEMONAAAA|57027|15|50|27|PM|second|afternoon|| +57028|AAAAAAAAFMONAAAA|57028|15|50|28|PM|second|afternoon|| +57029|AAAAAAAAGMONAAAA|57029|15|50|29|PM|second|afternoon|| +57030|AAAAAAAAHMONAAAA|57030|15|50|30|PM|second|afternoon|| +57031|AAAAAAAAIMONAAAA|57031|15|50|31|PM|second|afternoon|| +57032|AAAAAAAAJMONAAAA|57032|15|50|32|PM|second|afternoon|| +57033|AAAAAAAAKMONAAAA|57033|15|50|33|PM|second|afternoon|| +57034|AAAAAAAALMONAAAA|57034|15|50|34|PM|second|afternoon|| +57035|AAAAAAAAMMONAAAA|57035|15|50|35|PM|second|afternoon|| +57036|AAAAAAAANMONAAAA|57036|15|50|36|PM|second|afternoon|| +57037|AAAAAAAAOMONAAAA|57037|15|50|37|PM|second|afternoon|| +57038|AAAAAAAAPMONAAAA|57038|15|50|38|PM|second|afternoon|| +57039|AAAAAAAAANONAAAA|57039|15|50|39|PM|second|afternoon|| +57040|AAAAAAAABNONAAAA|57040|15|50|40|PM|second|afternoon|| +57041|AAAAAAAACNONAAAA|57041|15|50|41|PM|second|afternoon|| +57042|AAAAAAAADNONAAAA|57042|15|50|42|PM|second|afternoon|| +57043|AAAAAAAAENONAAAA|57043|15|50|43|PM|second|afternoon|| +57044|AAAAAAAAFNONAAAA|57044|15|50|44|PM|second|afternoon|| +57045|AAAAAAAAGNONAAAA|57045|15|50|45|PM|second|afternoon|| +57046|AAAAAAAAHNONAAAA|57046|15|50|46|PM|second|afternoon|| +57047|AAAAAAAAINONAAAA|57047|15|50|47|PM|second|afternoon|| +57048|AAAAAAAAJNONAAAA|57048|15|50|48|PM|second|afternoon|| +57049|AAAAAAAAKNONAAAA|57049|15|50|49|PM|second|afternoon|| +57050|AAAAAAAALNONAAAA|57050|15|50|50|PM|second|afternoon|| +57051|AAAAAAAAMNONAAAA|57051|15|50|51|PM|second|afternoon|| +57052|AAAAAAAANNONAAAA|57052|15|50|52|PM|second|afternoon|| +57053|AAAAAAAAONONAAAA|57053|15|50|53|PM|second|afternoon|| +57054|AAAAAAAAPNONAAAA|57054|15|50|54|PM|second|afternoon|| +57055|AAAAAAAAAOONAAAA|57055|15|50|55|PM|second|afternoon|| +57056|AAAAAAAABOONAAAA|57056|15|50|56|PM|second|afternoon|| +57057|AAAAAAAACOONAAAA|57057|15|50|57|PM|second|afternoon|| +57058|AAAAAAAADOONAAAA|57058|15|50|58|PM|second|afternoon|| +57059|AAAAAAAAEOONAAAA|57059|15|50|59|PM|second|afternoon|| +57060|AAAAAAAAFOONAAAA|57060|15|51|0|PM|second|afternoon|| +57061|AAAAAAAAGOONAAAA|57061|15|51|1|PM|second|afternoon|| +57062|AAAAAAAAHOONAAAA|57062|15|51|2|PM|second|afternoon|| +57063|AAAAAAAAIOONAAAA|57063|15|51|3|PM|second|afternoon|| +57064|AAAAAAAAJOONAAAA|57064|15|51|4|PM|second|afternoon|| +57065|AAAAAAAAKOONAAAA|57065|15|51|5|PM|second|afternoon|| +57066|AAAAAAAALOONAAAA|57066|15|51|6|PM|second|afternoon|| +57067|AAAAAAAAMOONAAAA|57067|15|51|7|PM|second|afternoon|| +57068|AAAAAAAANOONAAAA|57068|15|51|8|PM|second|afternoon|| +57069|AAAAAAAAOOONAAAA|57069|15|51|9|PM|second|afternoon|| +57070|AAAAAAAAPOONAAAA|57070|15|51|10|PM|second|afternoon|| +57071|AAAAAAAAAPONAAAA|57071|15|51|11|PM|second|afternoon|| +57072|AAAAAAAABPONAAAA|57072|15|51|12|PM|second|afternoon|| +57073|AAAAAAAACPONAAAA|57073|15|51|13|PM|second|afternoon|| +57074|AAAAAAAADPONAAAA|57074|15|51|14|PM|second|afternoon|| +57075|AAAAAAAAEPONAAAA|57075|15|51|15|PM|second|afternoon|| +57076|AAAAAAAAFPONAAAA|57076|15|51|16|PM|second|afternoon|| +57077|AAAAAAAAGPONAAAA|57077|15|51|17|PM|second|afternoon|| +57078|AAAAAAAAHPONAAAA|57078|15|51|18|PM|second|afternoon|| +57079|AAAAAAAAIPONAAAA|57079|15|51|19|PM|second|afternoon|| +57080|AAAAAAAAJPONAAAA|57080|15|51|20|PM|second|afternoon|| +57081|AAAAAAAAKPONAAAA|57081|15|51|21|PM|second|afternoon|| +57082|AAAAAAAALPONAAAA|57082|15|51|22|PM|second|afternoon|| +57083|AAAAAAAAMPONAAAA|57083|15|51|23|PM|second|afternoon|| +57084|AAAAAAAANPONAAAA|57084|15|51|24|PM|second|afternoon|| +57085|AAAAAAAAOPONAAAA|57085|15|51|25|PM|second|afternoon|| +57086|AAAAAAAAPPONAAAA|57086|15|51|26|PM|second|afternoon|| +57087|AAAAAAAAAAPNAAAA|57087|15|51|27|PM|second|afternoon|| +57088|AAAAAAAABAPNAAAA|57088|15|51|28|PM|second|afternoon|| +57089|AAAAAAAACAPNAAAA|57089|15|51|29|PM|second|afternoon|| +57090|AAAAAAAADAPNAAAA|57090|15|51|30|PM|second|afternoon|| +57091|AAAAAAAAEAPNAAAA|57091|15|51|31|PM|second|afternoon|| +57092|AAAAAAAAFAPNAAAA|57092|15|51|32|PM|second|afternoon|| +57093|AAAAAAAAGAPNAAAA|57093|15|51|33|PM|second|afternoon|| +57094|AAAAAAAAHAPNAAAA|57094|15|51|34|PM|second|afternoon|| +57095|AAAAAAAAIAPNAAAA|57095|15|51|35|PM|second|afternoon|| +57096|AAAAAAAAJAPNAAAA|57096|15|51|36|PM|second|afternoon|| +57097|AAAAAAAAKAPNAAAA|57097|15|51|37|PM|second|afternoon|| +57098|AAAAAAAALAPNAAAA|57098|15|51|38|PM|second|afternoon|| +57099|AAAAAAAAMAPNAAAA|57099|15|51|39|PM|second|afternoon|| +57100|AAAAAAAANAPNAAAA|57100|15|51|40|PM|second|afternoon|| +57101|AAAAAAAAOAPNAAAA|57101|15|51|41|PM|second|afternoon|| +57102|AAAAAAAAPAPNAAAA|57102|15|51|42|PM|second|afternoon|| +57103|AAAAAAAAABPNAAAA|57103|15|51|43|PM|second|afternoon|| +57104|AAAAAAAABBPNAAAA|57104|15|51|44|PM|second|afternoon|| +57105|AAAAAAAACBPNAAAA|57105|15|51|45|PM|second|afternoon|| +57106|AAAAAAAADBPNAAAA|57106|15|51|46|PM|second|afternoon|| +57107|AAAAAAAAEBPNAAAA|57107|15|51|47|PM|second|afternoon|| +57108|AAAAAAAAFBPNAAAA|57108|15|51|48|PM|second|afternoon|| +57109|AAAAAAAAGBPNAAAA|57109|15|51|49|PM|second|afternoon|| +57110|AAAAAAAAHBPNAAAA|57110|15|51|50|PM|second|afternoon|| +57111|AAAAAAAAIBPNAAAA|57111|15|51|51|PM|second|afternoon|| +57112|AAAAAAAAJBPNAAAA|57112|15|51|52|PM|second|afternoon|| +57113|AAAAAAAAKBPNAAAA|57113|15|51|53|PM|second|afternoon|| +57114|AAAAAAAALBPNAAAA|57114|15|51|54|PM|second|afternoon|| +57115|AAAAAAAAMBPNAAAA|57115|15|51|55|PM|second|afternoon|| +57116|AAAAAAAANBPNAAAA|57116|15|51|56|PM|second|afternoon|| +57117|AAAAAAAAOBPNAAAA|57117|15|51|57|PM|second|afternoon|| +57118|AAAAAAAAPBPNAAAA|57118|15|51|58|PM|second|afternoon|| +57119|AAAAAAAAACPNAAAA|57119|15|51|59|PM|second|afternoon|| +57120|AAAAAAAABCPNAAAA|57120|15|52|0|PM|second|afternoon|| +57121|AAAAAAAACCPNAAAA|57121|15|52|1|PM|second|afternoon|| +57122|AAAAAAAADCPNAAAA|57122|15|52|2|PM|second|afternoon|| +57123|AAAAAAAAECPNAAAA|57123|15|52|3|PM|second|afternoon|| +57124|AAAAAAAAFCPNAAAA|57124|15|52|4|PM|second|afternoon|| +57125|AAAAAAAAGCPNAAAA|57125|15|52|5|PM|second|afternoon|| +57126|AAAAAAAAHCPNAAAA|57126|15|52|6|PM|second|afternoon|| +57127|AAAAAAAAICPNAAAA|57127|15|52|7|PM|second|afternoon|| +57128|AAAAAAAAJCPNAAAA|57128|15|52|8|PM|second|afternoon|| +57129|AAAAAAAAKCPNAAAA|57129|15|52|9|PM|second|afternoon|| +57130|AAAAAAAALCPNAAAA|57130|15|52|10|PM|second|afternoon|| +57131|AAAAAAAAMCPNAAAA|57131|15|52|11|PM|second|afternoon|| +57132|AAAAAAAANCPNAAAA|57132|15|52|12|PM|second|afternoon|| +57133|AAAAAAAAOCPNAAAA|57133|15|52|13|PM|second|afternoon|| +57134|AAAAAAAAPCPNAAAA|57134|15|52|14|PM|second|afternoon|| +57135|AAAAAAAAADPNAAAA|57135|15|52|15|PM|second|afternoon|| +57136|AAAAAAAABDPNAAAA|57136|15|52|16|PM|second|afternoon|| +57137|AAAAAAAACDPNAAAA|57137|15|52|17|PM|second|afternoon|| +57138|AAAAAAAADDPNAAAA|57138|15|52|18|PM|second|afternoon|| +57139|AAAAAAAAEDPNAAAA|57139|15|52|19|PM|second|afternoon|| +57140|AAAAAAAAFDPNAAAA|57140|15|52|20|PM|second|afternoon|| +57141|AAAAAAAAGDPNAAAA|57141|15|52|21|PM|second|afternoon|| +57142|AAAAAAAAHDPNAAAA|57142|15|52|22|PM|second|afternoon|| +57143|AAAAAAAAIDPNAAAA|57143|15|52|23|PM|second|afternoon|| +57144|AAAAAAAAJDPNAAAA|57144|15|52|24|PM|second|afternoon|| +57145|AAAAAAAAKDPNAAAA|57145|15|52|25|PM|second|afternoon|| +57146|AAAAAAAALDPNAAAA|57146|15|52|26|PM|second|afternoon|| +57147|AAAAAAAAMDPNAAAA|57147|15|52|27|PM|second|afternoon|| +57148|AAAAAAAANDPNAAAA|57148|15|52|28|PM|second|afternoon|| +57149|AAAAAAAAODPNAAAA|57149|15|52|29|PM|second|afternoon|| +57150|AAAAAAAAPDPNAAAA|57150|15|52|30|PM|second|afternoon|| +57151|AAAAAAAAAEPNAAAA|57151|15|52|31|PM|second|afternoon|| +57152|AAAAAAAABEPNAAAA|57152|15|52|32|PM|second|afternoon|| +57153|AAAAAAAACEPNAAAA|57153|15|52|33|PM|second|afternoon|| +57154|AAAAAAAADEPNAAAA|57154|15|52|34|PM|second|afternoon|| +57155|AAAAAAAAEEPNAAAA|57155|15|52|35|PM|second|afternoon|| +57156|AAAAAAAAFEPNAAAA|57156|15|52|36|PM|second|afternoon|| +57157|AAAAAAAAGEPNAAAA|57157|15|52|37|PM|second|afternoon|| +57158|AAAAAAAAHEPNAAAA|57158|15|52|38|PM|second|afternoon|| +57159|AAAAAAAAIEPNAAAA|57159|15|52|39|PM|second|afternoon|| +57160|AAAAAAAAJEPNAAAA|57160|15|52|40|PM|second|afternoon|| +57161|AAAAAAAAKEPNAAAA|57161|15|52|41|PM|second|afternoon|| +57162|AAAAAAAALEPNAAAA|57162|15|52|42|PM|second|afternoon|| +57163|AAAAAAAAMEPNAAAA|57163|15|52|43|PM|second|afternoon|| +57164|AAAAAAAANEPNAAAA|57164|15|52|44|PM|second|afternoon|| +57165|AAAAAAAAOEPNAAAA|57165|15|52|45|PM|second|afternoon|| +57166|AAAAAAAAPEPNAAAA|57166|15|52|46|PM|second|afternoon|| +57167|AAAAAAAAAFPNAAAA|57167|15|52|47|PM|second|afternoon|| +57168|AAAAAAAABFPNAAAA|57168|15|52|48|PM|second|afternoon|| +57169|AAAAAAAACFPNAAAA|57169|15|52|49|PM|second|afternoon|| +57170|AAAAAAAADFPNAAAA|57170|15|52|50|PM|second|afternoon|| +57171|AAAAAAAAEFPNAAAA|57171|15|52|51|PM|second|afternoon|| +57172|AAAAAAAAFFPNAAAA|57172|15|52|52|PM|second|afternoon|| +57173|AAAAAAAAGFPNAAAA|57173|15|52|53|PM|second|afternoon|| +57174|AAAAAAAAHFPNAAAA|57174|15|52|54|PM|second|afternoon|| +57175|AAAAAAAAIFPNAAAA|57175|15|52|55|PM|second|afternoon|| +57176|AAAAAAAAJFPNAAAA|57176|15|52|56|PM|second|afternoon|| +57177|AAAAAAAAKFPNAAAA|57177|15|52|57|PM|second|afternoon|| +57178|AAAAAAAALFPNAAAA|57178|15|52|58|PM|second|afternoon|| +57179|AAAAAAAAMFPNAAAA|57179|15|52|59|PM|second|afternoon|| +57180|AAAAAAAANFPNAAAA|57180|15|53|0|PM|second|afternoon|| +57181|AAAAAAAAOFPNAAAA|57181|15|53|1|PM|second|afternoon|| +57182|AAAAAAAAPFPNAAAA|57182|15|53|2|PM|second|afternoon|| +57183|AAAAAAAAAGPNAAAA|57183|15|53|3|PM|second|afternoon|| +57184|AAAAAAAABGPNAAAA|57184|15|53|4|PM|second|afternoon|| +57185|AAAAAAAACGPNAAAA|57185|15|53|5|PM|second|afternoon|| +57186|AAAAAAAADGPNAAAA|57186|15|53|6|PM|second|afternoon|| +57187|AAAAAAAAEGPNAAAA|57187|15|53|7|PM|second|afternoon|| +57188|AAAAAAAAFGPNAAAA|57188|15|53|8|PM|second|afternoon|| +57189|AAAAAAAAGGPNAAAA|57189|15|53|9|PM|second|afternoon|| +57190|AAAAAAAAHGPNAAAA|57190|15|53|10|PM|second|afternoon|| +57191|AAAAAAAAIGPNAAAA|57191|15|53|11|PM|second|afternoon|| +57192|AAAAAAAAJGPNAAAA|57192|15|53|12|PM|second|afternoon|| +57193|AAAAAAAAKGPNAAAA|57193|15|53|13|PM|second|afternoon|| +57194|AAAAAAAALGPNAAAA|57194|15|53|14|PM|second|afternoon|| +57195|AAAAAAAAMGPNAAAA|57195|15|53|15|PM|second|afternoon|| +57196|AAAAAAAANGPNAAAA|57196|15|53|16|PM|second|afternoon|| +57197|AAAAAAAAOGPNAAAA|57197|15|53|17|PM|second|afternoon|| +57198|AAAAAAAAPGPNAAAA|57198|15|53|18|PM|second|afternoon|| +57199|AAAAAAAAAHPNAAAA|57199|15|53|19|PM|second|afternoon|| +57200|AAAAAAAABHPNAAAA|57200|15|53|20|PM|second|afternoon|| +57201|AAAAAAAACHPNAAAA|57201|15|53|21|PM|second|afternoon|| +57202|AAAAAAAADHPNAAAA|57202|15|53|22|PM|second|afternoon|| +57203|AAAAAAAAEHPNAAAA|57203|15|53|23|PM|second|afternoon|| +57204|AAAAAAAAFHPNAAAA|57204|15|53|24|PM|second|afternoon|| +57205|AAAAAAAAGHPNAAAA|57205|15|53|25|PM|second|afternoon|| +57206|AAAAAAAAHHPNAAAA|57206|15|53|26|PM|second|afternoon|| +57207|AAAAAAAAIHPNAAAA|57207|15|53|27|PM|second|afternoon|| +57208|AAAAAAAAJHPNAAAA|57208|15|53|28|PM|second|afternoon|| +57209|AAAAAAAAKHPNAAAA|57209|15|53|29|PM|second|afternoon|| +57210|AAAAAAAALHPNAAAA|57210|15|53|30|PM|second|afternoon|| +57211|AAAAAAAAMHPNAAAA|57211|15|53|31|PM|second|afternoon|| +57212|AAAAAAAANHPNAAAA|57212|15|53|32|PM|second|afternoon|| +57213|AAAAAAAAOHPNAAAA|57213|15|53|33|PM|second|afternoon|| +57214|AAAAAAAAPHPNAAAA|57214|15|53|34|PM|second|afternoon|| +57215|AAAAAAAAAIPNAAAA|57215|15|53|35|PM|second|afternoon|| +57216|AAAAAAAABIPNAAAA|57216|15|53|36|PM|second|afternoon|| +57217|AAAAAAAACIPNAAAA|57217|15|53|37|PM|second|afternoon|| +57218|AAAAAAAADIPNAAAA|57218|15|53|38|PM|second|afternoon|| +57219|AAAAAAAAEIPNAAAA|57219|15|53|39|PM|second|afternoon|| +57220|AAAAAAAAFIPNAAAA|57220|15|53|40|PM|second|afternoon|| +57221|AAAAAAAAGIPNAAAA|57221|15|53|41|PM|second|afternoon|| +57222|AAAAAAAAHIPNAAAA|57222|15|53|42|PM|second|afternoon|| +57223|AAAAAAAAIIPNAAAA|57223|15|53|43|PM|second|afternoon|| +57224|AAAAAAAAJIPNAAAA|57224|15|53|44|PM|second|afternoon|| +57225|AAAAAAAAKIPNAAAA|57225|15|53|45|PM|second|afternoon|| +57226|AAAAAAAALIPNAAAA|57226|15|53|46|PM|second|afternoon|| +57227|AAAAAAAAMIPNAAAA|57227|15|53|47|PM|second|afternoon|| +57228|AAAAAAAANIPNAAAA|57228|15|53|48|PM|second|afternoon|| +57229|AAAAAAAAOIPNAAAA|57229|15|53|49|PM|second|afternoon|| +57230|AAAAAAAAPIPNAAAA|57230|15|53|50|PM|second|afternoon|| +57231|AAAAAAAAAJPNAAAA|57231|15|53|51|PM|second|afternoon|| +57232|AAAAAAAABJPNAAAA|57232|15|53|52|PM|second|afternoon|| +57233|AAAAAAAACJPNAAAA|57233|15|53|53|PM|second|afternoon|| +57234|AAAAAAAADJPNAAAA|57234|15|53|54|PM|second|afternoon|| +57235|AAAAAAAAEJPNAAAA|57235|15|53|55|PM|second|afternoon|| +57236|AAAAAAAAFJPNAAAA|57236|15|53|56|PM|second|afternoon|| +57237|AAAAAAAAGJPNAAAA|57237|15|53|57|PM|second|afternoon|| +57238|AAAAAAAAHJPNAAAA|57238|15|53|58|PM|second|afternoon|| +57239|AAAAAAAAIJPNAAAA|57239|15|53|59|PM|second|afternoon|| +57240|AAAAAAAAJJPNAAAA|57240|15|54|0|PM|second|afternoon|| +57241|AAAAAAAAKJPNAAAA|57241|15|54|1|PM|second|afternoon|| +57242|AAAAAAAALJPNAAAA|57242|15|54|2|PM|second|afternoon|| +57243|AAAAAAAAMJPNAAAA|57243|15|54|3|PM|second|afternoon|| +57244|AAAAAAAANJPNAAAA|57244|15|54|4|PM|second|afternoon|| +57245|AAAAAAAAOJPNAAAA|57245|15|54|5|PM|second|afternoon|| +57246|AAAAAAAAPJPNAAAA|57246|15|54|6|PM|second|afternoon|| +57247|AAAAAAAAAKPNAAAA|57247|15|54|7|PM|second|afternoon|| +57248|AAAAAAAABKPNAAAA|57248|15|54|8|PM|second|afternoon|| +57249|AAAAAAAACKPNAAAA|57249|15|54|9|PM|second|afternoon|| +57250|AAAAAAAADKPNAAAA|57250|15|54|10|PM|second|afternoon|| +57251|AAAAAAAAEKPNAAAA|57251|15|54|11|PM|second|afternoon|| +57252|AAAAAAAAFKPNAAAA|57252|15|54|12|PM|second|afternoon|| +57253|AAAAAAAAGKPNAAAA|57253|15|54|13|PM|second|afternoon|| +57254|AAAAAAAAHKPNAAAA|57254|15|54|14|PM|second|afternoon|| +57255|AAAAAAAAIKPNAAAA|57255|15|54|15|PM|second|afternoon|| +57256|AAAAAAAAJKPNAAAA|57256|15|54|16|PM|second|afternoon|| +57257|AAAAAAAAKKPNAAAA|57257|15|54|17|PM|second|afternoon|| +57258|AAAAAAAALKPNAAAA|57258|15|54|18|PM|second|afternoon|| +57259|AAAAAAAAMKPNAAAA|57259|15|54|19|PM|second|afternoon|| +57260|AAAAAAAANKPNAAAA|57260|15|54|20|PM|second|afternoon|| +57261|AAAAAAAAOKPNAAAA|57261|15|54|21|PM|second|afternoon|| +57262|AAAAAAAAPKPNAAAA|57262|15|54|22|PM|second|afternoon|| +57263|AAAAAAAAALPNAAAA|57263|15|54|23|PM|second|afternoon|| +57264|AAAAAAAABLPNAAAA|57264|15|54|24|PM|second|afternoon|| +57265|AAAAAAAACLPNAAAA|57265|15|54|25|PM|second|afternoon|| +57266|AAAAAAAADLPNAAAA|57266|15|54|26|PM|second|afternoon|| +57267|AAAAAAAAELPNAAAA|57267|15|54|27|PM|second|afternoon|| +57268|AAAAAAAAFLPNAAAA|57268|15|54|28|PM|second|afternoon|| +57269|AAAAAAAAGLPNAAAA|57269|15|54|29|PM|second|afternoon|| +57270|AAAAAAAAHLPNAAAA|57270|15|54|30|PM|second|afternoon|| +57271|AAAAAAAAILPNAAAA|57271|15|54|31|PM|second|afternoon|| +57272|AAAAAAAAJLPNAAAA|57272|15|54|32|PM|second|afternoon|| +57273|AAAAAAAAKLPNAAAA|57273|15|54|33|PM|second|afternoon|| +57274|AAAAAAAALLPNAAAA|57274|15|54|34|PM|second|afternoon|| +57275|AAAAAAAAMLPNAAAA|57275|15|54|35|PM|second|afternoon|| +57276|AAAAAAAANLPNAAAA|57276|15|54|36|PM|second|afternoon|| +57277|AAAAAAAAOLPNAAAA|57277|15|54|37|PM|second|afternoon|| +57278|AAAAAAAAPLPNAAAA|57278|15|54|38|PM|second|afternoon|| +57279|AAAAAAAAAMPNAAAA|57279|15|54|39|PM|second|afternoon|| +57280|AAAAAAAABMPNAAAA|57280|15|54|40|PM|second|afternoon|| +57281|AAAAAAAACMPNAAAA|57281|15|54|41|PM|second|afternoon|| +57282|AAAAAAAADMPNAAAA|57282|15|54|42|PM|second|afternoon|| +57283|AAAAAAAAEMPNAAAA|57283|15|54|43|PM|second|afternoon|| +57284|AAAAAAAAFMPNAAAA|57284|15|54|44|PM|second|afternoon|| +57285|AAAAAAAAGMPNAAAA|57285|15|54|45|PM|second|afternoon|| +57286|AAAAAAAAHMPNAAAA|57286|15|54|46|PM|second|afternoon|| +57287|AAAAAAAAIMPNAAAA|57287|15|54|47|PM|second|afternoon|| +57288|AAAAAAAAJMPNAAAA|57288|15|54|48|PM|second|afternoon|| +57289|AAAAAAAAKMPNAAAA|57289|15|54|49|PM|second|afternoon|| +57290|AAAAAAAALMPNAAAA|57290|15|54|50|PM|second|afternoon|| +57291|AAAAAAAAMMPNAAAA|57291|15|54|51|PM|second|afternoon|| +57292|AAAAAAAANMPNAAAA|57292|15|54|52|PM|second|afternoon|| +57293|AAAAAAAAOMPNAAAA|57293|15|54|53|PM|second|afternoon|| +57294|AAAAAAAAPMPNAAAA|57294|15|54|54|PM|second|afternoon|| +57295|AAAAAAAAANPNAAAA|57295|15|54|55|PM|second|afternoon|| +57296|AAAAAAAABNPNAAAA|57296|15|54|56|PM|second|afternoon|| +57297|AAAAAAAACNPNAAAA|57297|15|54|57|PM|second|afternoon|| +57298|AAAAAAAADNPNAAAA|57298|15|54|58|PM|second|afternoon|| +57299|AAAAAAAAENPNAAAA|57299|15|54|59|PM|second|afternoon|| +57300|AAAAAAAAFNPNAAAA|57300|15|55|0|PM|second|afternoon|| +57301|AAAAAAAAGNPNAAAA|57301|15|55|1|PM|second|afternoon|| +57302|AAAAAAAAHNPNAAAA|57302|15|55|2|PM|second|afternoon|| +57303|AAAAAAAAINPNAAAA|57303|15|55|3|PM|second|afternoon|| +57304|AAAAAAAAJNPNAAAA|57304|15|55|4|PM|second|afternoon|| +57305|AAAAAAAAKNPNAAAA|57305|15|55|5|PM|second|afternoon|| +57306|AAAAAAAALNPNAAAA|57306|15|55|6|PM|second|afternoon|| +57307|AAAAAAAAMNPNAAAA|57307|15|55|7|PM|second|afternoon|| +57308|AAAAAAAANNPNAAAA|57308|15|55|8|PM|second|afternoon|| +57309|AAAAAAAAONPNAAAA|57309|15|55|9|PM|second|afternoon|| +57310|AAAAAAAAPNPNAAAA|57310|15|55|10|PM|second|afternoon|| +57311|AAAAAAAAAOPNAAAA|57311|15|55|11|PM|second|afternoon|| +57312|AAAAAAAABOPNAAAA|57312|15|55|12|PM|second|afternoon|| +57313|AAAAAAAACOPNAAAA|57313|15|55|13|PM|second|afternoon|| +57314|AAAAAAAADOPNAAAA|57314|15|55|14|PM|second|afternoon|| +57315|AAAAAAAAEOPNAAAA|57315|15|55|15|PM|second|afternoon|| +57316|AAAAAAAAFOPNAAAA|57316|15|55|16|PM|second|afternoon|| +57317|AAAAAAAAGOPNAAAA|57317|15|55|17|PM|second|afternoon|| +57318|AAAAAAAAHOPNAAAA|57318|15|55|18|PM|second|afternoon|| +57319|AAAAAAAAIOPNAAAA|57319|15|55|19|PM|second|afternoon|| +57320|AAAAAAAAJOPNAAAA|57320|15|55|20|PM|second|afternoon|| +57321|AAAAAAAAKOPNAAAA|57321|15|55|21|PM|second|afternoon|| +57322|AAAAAAAALOPNAAAA|57322|15|55|22|PM|second|afternoon|| +57323|AAAAAAAAMOPNAAAA|57323|15|55|23|PM|second|afternoon|| +57324|AAAAAAAANOPNAAAA|57324|15|55|24|PM|second|afternoon|| +57325|AAAAAAAAOOPNAAAA|57325|15|55|25|PM|second|afternoon|| +57326|AAAAAAAAPOPNAAAA|57326|15|55|26|PM|second|afternoon|| +57327|AAAAAAAAAPPNAAAA|57327|15|55|27|PM|second|afternoon|| +57328|AAAAAAAABPPNAAAA|57328|15|55|28|PM|second|afternoon|| +57329|AAAAAAAACPPNAAAA|57329|15|55|29|PM|second|afternoon|| +57330|AAAAAAAADPPNAAAA|57330|15|55|30|PM|second|afternoon|| +57331|AAAAAAAAEPPNAAAA|57331|15|55|31|PM|second|afternoon|| +57332|AAAAAAAAFPPNAAAA|57332|15|55|32|PM|second|afternoon|| +57333|AAAAAAAAGPPNAAAA|57333|15|55|33|PM|second|afternoon|| +57334|AAAAAAAAHPPNAAAA|57334|15|55|34|PM|second|afternoon|| +57335|AAAAAAAAIPPNAAAA|57335|15|55|35|PM|second|afternoon|| +57336|AAAAAAAAJPPNAAAA|57336|15|55|36|PM|second|afternoon|| +57337|AAAAAAAAKPPNAAAA|57337|15|55|37|PM|second|afternoon|| +57338|AAAAAAAALPPNAAAA|57338|15|55|38|PM|second|afternoon|| +57339|AAAAAAAAMPPNAAAA|57339|15|55|39|PM|second|afternoon|| +57340|AAAAAAAANPPNAAAA|57340|15|55|40|PM|second|afternoon|| +57341|AAAAAAAAOPPNAAAA|57341|15|55|41|PM|second|afternoon|| +57342|AAAAAAAAPPPNAAAA|57342|15|55|42|PM|second|afternoon|| +57343|AAAAAAAAAAAOAAAA|57343|15|55|43|PM|second|afternoon|| +57344|AAAAAAAABAAOAAAA|57344|15|55|44|PM|second|afternoon|| +57345|AAAAAAAACAAOAAAA|57345|15|55|45|PM|second|afternoon|| +57346|AAAAAAAADAAOAAAA|57346|15|55|46|PM|second|afternoon|| +57347|AAAAAAAAEAAOAAAA|57347|15|55|47|PM|second|afternoon|| +57348|AAAAAAAAFAAOAAAA|57348|15|55|48|PM|second|afternoon|| +57349|AAAAAAAAGAAOAAAA|57349|15|55|49|PM|second|afternoon|| +57350|AAAAAAAAHAAOAAAA|57350|15|55|50|PM|second|afternoon|| +57351|AAAAAAAAIAAOAAAA|57351|15|55|51|PM|second|afternoon|| +57352|AAAAAAAAJAAOAAAA|57352|15|55|52|PM|second|afternoon|| +57353|AAAAAAAAKAAOAAAA|57353|15|55|53|PM|second|afternoon|| +57354|AAAAAAAALAAOAAAA|57354|15|55|54|PM|second|afternoon|| +57355|AAAAAAAAMAAOAAAA|57355|15|55|55|PM|second|afternoon|| +57356|AAAAAAAANAAOAAAA|57356|15|55|56|PM|second|afternoon|| +57357|AAAAAAAAOAAOAAAA|57357|15|55|57|PM|second|afternoon|| +57358|AAAAAAAAPAAOAAAA|57358|15|55|58|PM|second|afternoon|| +57359|AAAAAAAAABAOAAAA|57359|15|55|59|PM|second|afternoon|| +57360|AAAAAAAABBAOAAAA|57360|15|56|0|PM|second|afternoon|| +57361|AAAAAAAACBAOAAAA|57361|15|56|1|PM|second|afternoon|| +57362|AAAAAAAADBAOAAAA|57362|15|56|2|PM|second|afternoon|| +57363|AAAAAAAAEBAOAAAA|57363|15|56|3|PM|second|afternoon|| +57364|AAAAAAAAFBAOAAAA|57364|15|56|4|PM|second|afternoon|| +57365|AAAAAAAAGBAOAAAA|57365|15|56|5|PM|second|afternoon|| +57366|AAAAAAAAHBAOAAAA|57366|15|56|6|PM|second|afternoon|| +57367|AAAAAAAAIBAOAAAA|57367|15|56|7|PM|second|afternoon|| +57368|AAAAAAAAJBAOAAAA|57368|15|56|8|PM|second|afternoon|| +57369|AAAAAAAAKBAOAAAA|57369|15|56|9|PM|second|afternoon|| +57370|AAAAAAAALBAOAAAA|57370|15|56|10|PM|second|afternoon|| +57371|AAAAAAAAMBAOAAAA|57371|15|56|11|PM|second|afternoon|| +57372|AAAAAAAANBAOAAAA|57372|15|56|12|PM|second|afternoon|| +57373|AAAAAAAAOBAOAAAA|57373|15|56|13|PM|second|afternoon|| +57374|AAAAAAAAPBAOAAAA|57374|15|56|14|PM|second|afternoon|| +57375|AAAAAAAAACAOAAAA|57375|15|56|15|PM|second|afternoon|| +57376|AAAAAAAABCAOAAAA|57376|15|56|16|PM|second|afternoon|| +57377|AAAAAAAACCAOAAAA|57377|15|56|17|PM|second|afternoon|| +57378|AAAAAAAADCAOAAAA|57378|15|56|18|PM|second|afternoon|| +57379|AAAAAAAAECAOAAAA|57379|15|56|19|PM|second|afternoon|| +57380|AAAAAAAAFCAOAAAA|57380|15|56|20|PM|second|afternoon|| +57381|AAAAAAAAGCAOAAAA|57381|15|56|21|PM|second|afternoon|| +57382|AAAAAAAAHCAOAAAA|57382|15|56|22|PM|second|afternoon|| +57383|AAAAAAAAICAOAAAA|57383|15|56|23|PM|second|afternoon|| +57384|AAAAAAAAJCAOAAAA|57384|15|56|24|PM|second|afternoon|| +57385|AAAAAAAAKCAOAAAA|57385|15|56|25|PM|second|afternoon|| +57386|AAAAAAAALCAOAAAA|57386|15|56|26|PM|second|afternoon|| +57387|AAAAAAAAMCAOAAAA|57387|15|56|27|PM|second|afternoon|| +57388|AAAAAAAANCAOAAAA|57388|15|56|28|PM|second|afternoon|| +57389|AAAAAAAAOCAOAAAA|57389|15|56|29|PM|second|afternoon|| +57390|AAAAAAAAPCAOAAAA|57390|15|56|30|PM|second|afternoon|| +57391|AAAAAAAAADAOAAAA|57391|15|56|31|PM|second|afternoon|| +57392|AAAAAAAABDAOAAAA|57392|15|56|32|PM|second|afternoon|| +57393|AAAAAAAACDAOAAAA|57393|15|56|33|PM|second|afternoon|| +57394|AAAAAAAADDAOAAAA|57394|15|56|34|PM|second|afternoon|| +57395|AAAAAAAAEDAOAAAA|57395|15|56|35|PM|second|afternoon|| +57396|AAAAAAAAFDAOAAAA|57396|15|56|36|PM|second|afternoon|| +57397|AAAAAAAAGDAOAAAA|57397|15|56|37|PM|second|afternoon|| +57398|AAAAAAAAHDAOAAAA|57398|15|56|38|PM|second|afternoon|| +57399|AAAAAAAAIDAOAAAA|57399|15|56|39|PM|second|afternoon|| +57400|AAAAAAAAJDAOAAAA|57400|15|56|40|PM|second|afternoon|| +57401|AAAAAAAAKDAOAAAA|57401|15|56|41|PM|second|afternoon|| +57402|AAAAAAAALDAOAAAA|57402|15|56|42|PM|second|afternoon|| +57403|AAAAAAAAMDAOAAAA|57403|15|56|43|PM|second|afternoon|| +57404|AAAAAAAANDAOAAAA|57404|15|56|44|PM|second|afternoon|| +57405|AAAAAAAAODAOAAAA|57405|15|56|45|PM|second|afternoon|| +57406|AAAAAAAAPDAOAAAA|57406|15|56|46|PM|second|afternoon|| +57407|AAAAAAAAAEAOAAAA|57407|15|56|47|PM|second|afternoon|| +57408|AAAAAAAABEAOAAAA|57408|15|56|48|PM|second|afternoon|| +57409|AAAAAAAACEAOAAAA|57409|15|56|49|PM|second|afternoon|| +57410|AAAAAAAADEAOAAAA|57410|15|56|50|PM|second|afternoon|| +57411|AAAAAAAAEEAOAAAA|57411|15|56|51|PM|second|afternoon|| +57412|AAAAAAAAFEAOAAAA|57412|15|56|52|PM|second|afternoon|| +57413|AAAAAAAAGEAOAAAA|57413|15|56|53|PM|second|afternoon|| +57414|AAAAAAAAHEAOAAAA|57414|15|56|54|PM|second|afternoon|| +57415|AAAAAAAAIEAOAAAA|57415|15|56|55|PM|second|afternoon|| +57416|AAAAAAAAJEAOAAAA|57416|15|56|56|PM|second|afternoon|| +57417|AAAAAAAAKEAOAAAA|57417|15|56|57|PM|second|afternoon|| +57418|AAAAAAAALEAOAAAA|57418|15|56|58|PM|second|afternoon|| +57419|AAAAAAAAMEAOAAAA|57419|15|56|59|PM|second|afternoon|| +57420|AAAAAAAANEAOAAAA|57420|15|57|0|PM|second|afternoon|| +57421|AAAAAAAAOEAOAAAA|57421|15|57|1|PM|second|afternoon|| +57422|AAAAAAAAPEAOAAAA|57422|15|57|2|PM|second|afternoon|| +57423|AAAAAAAAAFAOAAAA|57423|15|57|3|PM|second|afternoon|| +57424|AAAAAAAABFAOAAAA|57424|15|57|4|PM|second|afternoon|| +57425|AAAAAAAACFAOAAAA|57425|15|57|5|PM|second|afternoon|| +57426|AAAAAAAADFAOAAAA|57426|15|57|6|PM|second|afternoon|| +57427|AAAAAAAAEFAOAAAA|57427|15|57|7|PM|second|afternoon|| +57428|AAAAAAAAFFAOAAAA|57428|15|57|8|PM|second|afternoon|| +57429|AAAAAAAAGFAOAAAA|57429|15|57|9|PM|second|afternoon|| +57430|AAAAAAAAHFAOAAAA|57430|15|57|10|PM|second|afternoon|| +57431|AAAAAAAAIFAOAAAA|57431|15|57|11|PM|second|afternoon|| +57432|AAAAAAAAJFAOAAAA|57432|15|57|12|PM|second|afternoon|| +57433|AAAAAAAAKFAOAAAA|57433|15|57|13|PM|second|afternoon|| +57434|AAAAAAAALFAOAAAA|57434|15|57|14|PM|second|afternoon|| +57435|AAAAAAAAMFAOAAAA|57435|15|57|15|PM|second|afternoon|| +57436|AAAAAAAANFAOAAAA|57436|15|57|16|PM|second|afternoon|| +57437|AAAAAAAAOFAOAAAA|57437|15|57|17|PM|second|afternoon|| +57438|AAAAAAAAPFAOAAAA|57438|15|57|18|PM|second|afternoon|| +57439|AAAAAAAAAGAOAAAA|57439|15|57|19|PM|second|afternoon|| +57440|AAAAAAAABGAOAAAA|57440|15|57|20|PM|second|afternoon|| +57441|AAAAAAAACGAOAAAA|57441|15|57|21|PM|second|afternoon|| +57442|AAAAAAAADGAOAAAA|57442|15|57|22|PM|second|afternoon|| +57443|AAAAAAAAEGAOAAAA|57443|15|57|23|PM|second|afternoon|| +57444|AAAAAAAAFGAOAAAA|57444|15|57|24|PM|second|afternoon|| +57445|AAAAAAAAGGAOAAAA|57445|15|57|25|PM|second|afternoon|| +57446|AAAAAAAAHGAOAAAA|57446|15|57|26|PM|second|afternoon|| +57447|AAAAAAAAIGAOAAAA|57447|15|57|27|PM|second|afternoon|| +57448|AAAAAAAAJGAOAAAA|57448|15|57|28|PM|second|afternoon|| +57449|AAAAAAAAKGAOAAAA|57449|15|57|29|PM|second|afternoon|| +57450|AAAAAAAALGAOAAAA|57450|15|57|30|PM|second|afternoon|| +57451|AAAAAAAAMGAOAAAA|57451|15|57|31|PM|second|afternoon|| +57452|AAAAAAAANGAOAAAA|57452|15|57|32|PM|second|afternoon|| +57453|AAAAAAAAOGAOAAAA|57453|15|57|33|PM|second|afternoon|| +57454|AAAAAAAAPGAOAAAA|57454|15|57|34|PM|second|afternoon|| +57455|AAAAAAAAAHAOAAAA|57455|15|57|35|PM|second|afternoon|| +57456|AAAAAAAABHAOAAAA|57456|15|57|36|PM|second|afternoon|| +57457|AAAAAAAACHAOAAAA|57457|15|57|37|PM|second|afternoon|| +57458|AAAAAAAADHAOAAAA|57458|15|57|38|PM|second|afternoon|| +57459|AAAAAAAAEHAOAAAA|57459|15|57|39|PM|second|afternoon|| +57460|AAAAAAAAFHAOAAAA|57460|15|57|40|PM|second|afternoon|| +57461|AAAAAAAAGHAOAAAA|57461|15|57|41|PM|second|afternoon|| +57462|AAAAAAAAHHAOAAAA|57462|15|57|42|PM|second|afternoon|| +57463|AAAAAAAAIHAOAAAA|57463|15|57|43|PM|second|afternoon|| +57464|AAAAAAAAJHAOAAAA|57464|15|57|44|PM|second|afternoon|| +57465|AAAAAAAAKHAOAAAA|57465|15|57|45|PM|second|afternoon|| +57466|AAAAAAAALHAOAAAA|57466|15|57|46|PM|second|afternoon|| +57467|AAAAAAAAMHAOAAAA|57467|15|57|47|PM|second|afternoon|| +57468|AAAAAAAANHAOAAAA|57468|15|57|48|PM|second|afternoon|| +57469|AAAAAAAAOHAOAAAA|57469|15|57|49|PM|second|afternoon|| +57470|AAAAAAAAPHAOAAAA|57470|15|57|50|PM|second|afternoon|| +57471|AAAAAAAAAIAOAAAA|57471|15|57|51|PM|second|afternoon|| +57472|AAAAAAAABIAOAAAA|57472|15|57|52|PM|second|afternoon|| +57473|AAAAAAAACIAOAAAA|57473|15|57|53|PM|second|afternoon|| +57474|AAAAAAAADIAOAAAA|57474|15|57|54|PM|second|afternoon|| +57475|AAAAAAAAEIAOAAAA|57475|15|57|55|PM|second|afternoon|| +57476|AAAAAAAAFIAOAAAA|57476|15|57|56|PM|second|afternoon|| +57477|AAAAAAAAGIAOAAAA|57477|15|57|57|PM|second|afternoon|| +57478|AAAAAAAAHIAOAAAA|57478|15|57|58|PM|second|afternoon|| +57479|AAAAAAAAIIAOAAAA|57479|15|57|59|PM|second|afternoon|| +57480|AAAAAAAAJIAOAAAA|57480|15|58|0|PM|second|afternoon|| +57481|AAAAAAAAKIAOAAAA|57481|15|58|1|PM|second|afternoon|| +57482|AAAAAAAALIAOAAAA|57482|15|58|2|PM|second|afternoon|| +57483|AAAAAAAAMIAOAAAA|57483|15|58|3|PM|second|afternoon|| +57484|AAAAAAAANIAOAAAA|57484|15|58|4|PM|second|afternoon|| +57485|AAAAAAAAOIAOAAAA|57485|15|58|5|PM|second|afternoon|| +57486|AAAAAAAAPIAOAAAA|57486|15|58|6|PM|second|afternoon|| +57487|AAAAAAAAAJAOAAAA|57487|15|58|7|PM|second|afternoon|| +57488|AAAAAAAABJAOAAAA|57488|15|58|8|PM|second|afternoon|| +57489|AAAAAAAACJAOAAAA|57489|15|58|9|PM|second|afternoon|| +57490|AAAAAAAADJAOAAAA|57490|15|58|10|PM|second|afternoon|| +57491|AAAAAAAAEJAOAAAA|57491|15|58|11|PM|second|afternoon|| +57492|AAAAAAAAFJAOAAAA|57492|15|58|12|PM|second|afternoon|| +57493|AAAAAAAAGJAOAAAA|57493|15|58|13|PM|second|afternoon|| +57494|AAAAAAAAHJAOAAAA|57494|15|58|14|PM|second|afternoon|| +57495|AAAAAAAAIJAOAAAA|57495|15|58|15|PM|second|afternoon|| +57496|AAAAAAAAJJAOAAAA|57496|15|58|16|PM|second|afternoon|| +57497|AAAAAAAAKJAOAAAA|57497|15|58|17|PM|second|afternoon|| +57498|AAAAAAAALJAOAAAA|57498|15|58|18|PM|second|afternoon|| +57499|AAAAAAAAMJAOAAAA|57499|15|58|19|PM|second|afternoon|| +57500|AAAAAAAANJAOAAAA|57500|15|58|20|PM|second|afternoon|| +57501|AAAAAAAAOJAOAAAA|57501|15|58|21|PM|second|afternoon|| +57502|AAAAAAAAPJAOAAAA|57502|15|58|22|PM|second|afternoon|| +57503|AAAAAAAAAKAOAAAA|57503|15|58|23|PM|second|afternoon|| +57504|AAAAAAAABKAOAAAA|57504|15|58|24|PM|second|afternoon|| +57505|AAAAAAAACKAOAAAA|57505|15|58|25|PM|second|afternoon|| +57506|AAAAAAAADKAOAAAA|57506|15|58|26|PM|second|afternoon|| +57507|AAAAAAAAEKAOAAAA|57507|15|58|27|PM|second|afternoon|| +57508|AAAAAAAAFKAOAAAA|57508|15|58|28|PM|second|afternoon|| +57509|AAAAAAAAGKAOAAAA|57509|15|58|29|PM|second|afternoon|| +57510|AAAAAAAAHKAOAAAA|57510|15|58|30|PM|second|afternoon|| +57511|AAAAAAAAIKAOAAAA|57511|15|58|31|PM|second|afternoon|| +57512|AAAAAAAAJKAOAAAA|57512|15|58|32|PM|second|afternoon|| +57513|AAAAAAAAKKAOAAAA|57513|15|58|33|PM|second|afternoon|| +57514|AAAAAAAALKAOAAAA|57514|15|58|34|PM|second|afternoon|| +57515|AAAAAAAAMKAOAAAA|57515|15|58|35|PM|second|afternoon|| +57516|AAAAAAAANKAOAAAA|57516|15|58|36|PM|second|afternoon|| +57517|AAAAAAAAOKAOAAAA|57517|15|58|37|PM|second|afternoon|| +57518|AAAAAAAAPKAOAAAA|57518|15|58|38|PM|second|afternoon|| +57519|AAAAAAAAALAOAAAA|57519|15|58|39|PM|second|afternoon|| +57520|AAAAAAAABLAOAAAA|57520|15|58|40|PM|second|afternoon|| +57521|AAAAAAAACLAOAAAA|57521|15|58|41|PM|second|afternoon|| +57522|AAAAAAAADLAOAAAA|57522|15|58|42|PM|second|afternoon|| +57523|AAAAAAAAELAOAAAA|57523|15|58|43|PM|second|afternoon|| +57524|AAAAAAAAFLAOAAAA|57524|15|58|44|PM|second|afternoon|| +57525|AAAAAAAAGLAOAAAA|57525|15|58|45|PM|second|afternoon|| +57526|AAAAAAAAHLAOAAAA|57526|15|58|46|PM|second|afternoon|| +57527|AAAAAAAAILAOAAAA|57527|15|58|47|PM|second|afternoon|| +57528|AAAAAAAAJLAOAAAA|57528|15|58|48|PM|second|afternoon|| +57529|AAAAAAAAKLAOAAAA|57529|15|58|49|PM|second|afternoon|| +57530|AAAAAAAALLAOAAAA|57530|15|58|50|PM|second|afternoon|| +57531|AAAAAAAAMLAOAAAA|57531|15|58|51|PM|second|afternoon|| +57532|AAAAAAAANLAOAAAA|57532|15|58|52|PM|second|afternoon|| +57533|AAAAAAAAOLAOAAAA|57533|15|58|53|PM|second|afternoon|| +57534|AAAAAAAAPLAOAAAA|57534|15|58|54|PM|second|afternoon|| +57535|AAAAAAAAAMAOAAAA|57535|15|58|55|PM|second|afternoon|| +57536|AAAAAAAABMAOAAAA|57536|15|58|56|PM|second|afternoon|| +57537|AAAAAAAACMAOAAAA|57537|15|58|57|PM|second|afternoon|| +57538|AAAAAAAADMAOAAAA|57538|15|58|58|PM|second|afternoon|| +57539|AAAAAAAAEMAOAAAA|57539|15|58|59|PM|second|afternoon|| +57540|AAAAAAAAFMAOAAAA|57540|15|59|0|PM|second|afternoon|| +57541|AAAAAAAAGMAOAAAA|57541|15|59|1|PM|second|afternoon|| +57542|AAAAAAAAHMAOAAAA|57542|15|59|2|PM|second|afternoon|| +57543|AAAAAAAAIMAOAAAA|57543|15|59|3|PM|second|afternoon|| +57544|AAAAAAAAJMAOAAAA|57544|15|59|4|PM|second|afternoon|| +57545|AAAAAAAAKMAOAAAA|57545|15|59|5|PM|second|afternoon|| +57546|AAAAAAAALMAOAAAA|57546|15|59|6|PM|second|afternoon|| +57547|AAAAAAAAMMAOAAAA|57547|15|59|7|PM|second|afternoon|| +57548|AAAAAAAANMAOAAAA|57548|15|59|8|PM|second|afternoon|| +57549|AAAAAAAAOMAOAAAA|57549|15|59|9|PM|second|afternoon|| +57550|AAAAAAAAPMAOAAAA|57550|15|59|10|PM|second|afternoon|| +57551|AAAAAAAAANAOAAAA|57551|15|59|11|PM|second|afternoon|| +57552|AAAAAAAABNAOAAAA|57552|15|59|12|PM|second|afternoon|| +57553|AAAAAAAACNAOAAAA|57553|15|59|13|PM|second|afternoon|| +57554|AAAAAAAADNAOAAAA|57554|15|59|14|PM|second|afternoon|| +57555|AAAAAAAAENAOAAAA|57555|15|59|15|PM|second|afternoon|| +57556|AAAAAAAAFNAOAAAA|57556|15|59|16|PM|second|afternoon|| +57557|AAAAAAAAGNAOAAAA|57557|15|59|17|PM|second|afternoon|| +57558|AAAAAAAAHNAOAAAA|57558|15|59|18|PM|second|afternoon|| +57559|AAAAAAAAINAOAAAA|57559|15|59|19|PM|second|afternoon|| +57560|AAAAAAAAJNAOAAAA|57560|15|59|20|PM|second|afternoon|| +57561|AAAAAAAAKNAOAAAA|57561|15|59|21|PM|second|afternoon|| +57562|AAAAAAAALNAOAAAA|57562|15|59|22|PM|second|afternoon|| +57563|AAAAAAAAMNAOAAAA|57563|15|59|23|PM|second|afternoon|| +57564|AAAAAAAANNAOAAAA|57564|15|59|24|PM|second|afternoon|| +57565|AAAAAAAAONAOAAAA|57565|15|59|25|PM|second|afternoon|| +57566|AAAAAAAAPNAOAAAA|57566|15|59|26|PM|second|afternoon|| +57567|AAAAAAAAAOAOAAAA|57567|15|59|27|PM|second|afternoon|| +57568|AAAAAAAABOAOAAAA|57568|15|59|28|PM|second|afternoon|| +57569|AAAAAAAACOAOAAAA|57569|15|59|29|PM|second|afternoon|| +57570|AAAAAAAADOAOAAAA|57570|15|59|30|PM|second|afternoon|| +57571|AAAAAAAAEOAOAAAA|57571|15|59|31|PM|second|afternoon|| +57572|AAAAAAAAFOAOAAAA|57572|15|59|32|PM|second|afternoon|| +57573|AAAAAAAAGOAOAAAA|57573|15|59|33|PM|second|afternoon|| +57574|AAAAAAAAHOAOAAAA|57574|15|59|34|PM|second|afternoon|| +57575|AAAAAAAAIOAOAAAA|57575|15|59|35|PM|second|afternoon|| +57576|AAAAAAAAJOAOAAAA|57576|15|59|36|PM|second|afternoon|| +57577|AAAAAAAAKOAOAAAA|57577|15|59|37|PM|second|afternoon|| +57578|AAAAAAAALOAOAAAA|57578|15|59|38|PM|second|afternoon|| +57579|AAAAAAAAMOAOAAAA|57579|15|59|39|PM|second|afternoon|| +57580|AAAAAAAANOAOAAAA|57580|15|59|40|PM|second|afternoon|| +57581|AAAAAAAAOOAOAAAA|57581|15|59|41|PM|second|afternoon|| +57582|AAAAAAAAPOAOAAAA|57582|15|59|42|PM|second|afternoon|| +57583|AAAAAAAAAPAOAAAA|57583|15|59|43|PM|second|afternoon|| +57584|AAAAAAAABPAOAAAA|57584|15|59|44|PM|second|afternoon|| +57585|AAAAAAAACPAOAAAA|57585|15|59|45|PM|second|afternoon|| +57586|AAAAAAAADPAOAAAA|57586|15|59|46|PM|second|afternoon|| +57587|AAAAAAAAEPAOAAAA|57587|15|59|47|PM|second|afternoon|| +57588|AAAAAAAAFPAOAAAA|57588|15|59|48|PM|second|afternoon|| +57589|AAAAAAAAGPAOAAAA|57589|15|59|49|PM|second|afternoon|| +57590|AAAAAAAAHPAOAAAA|57590|15|59|50|PM|second|afternoon|| +57591|AAAAAAAAIPAOAAAA|57591|15|59|51|PM|second|afternoon|| +57592|AAAAAAAAJPAOAAAA|57592|15|59|52|PM|second|afternoon|| +57593|AAAAAAAAKPAOAAAA|57593|15|59|53|PM|second|afternoon|| +57594|AAAAAAAALPAOAAAA|57594|15|59|54|PM|second|afternoon|| +57595|AAAAAAAAMPAOAAAA|57595|15|59|55|PM|second|afternoon|| +57596|AAAAAAAANPAOAAAA|57596|15|59|56|PM|second|afternoon|| +57597|AAAAAAAAOPAOAAAA|57597|15|59|57|PM|second|afternoon|| +57598|AAAAAAAAPPAOAAAA|57598|15|59|58|PM|second|afternoon|| +57599|AAAAAAAAAABOAAAA|57599|15|59|59|PM|second|afternoon|| +57600|AAAAAAAABABOAAAA|57600|16|0|0|PM|second|afternoon|| +57601|AAAAAAAACABOAAAA|57601|16|0|1|PM|second|afternoon|| +57602|AAAAAAAADABOAAAA|57602|16|0|2|PM|second|afternoon|| +57603|AAAAAAAAEABOAAAA|57603|16|0|3|PM|second|afternoon|| +57604|AAAAAAAAFABOAAAA|57604|16|0|4|PM|second|afternoon|| +57605|AAAAAAAAGABOAAAA|57605|16|0|5|PM|second|afternoon|| +57606|AAAAAAAAHABOAAAA|57606|16|0|6|PM|second|afternoon|| +57607|AAAAAAAAIABOAAAA|57607|16|0|7|PM|second|afternoon|| +57608|AAAAAAAAJABOAAAA|57608|16|0|8|PM|second|afternoon|| +57609|AAAAAAAAKABOAAAA|57609|16|0|9|PM|second|afternoon|| +57610|AAAAAAAALABOAAAA|57610|16|0|10|PM|second|afternoon|| +57611|AAAAAAAAMABOAAAA|57611|16|0|11|PM|second|afternoon|| +57612|AAAAAAAANABOAAAA|57612|16|0|12|PM|second|afternoon|| +57613|AAAAAAAAOABOAAAA|57613|16|0|13|PM|second|afternoon|| +57614|AAAAAAAAPABOAAAA|57614|16|0|14|PM|second|afternoon|| +57615|AAAAAAAAABBOAAAA|57615|16|0|15|PM|second|afternoon|| +57616|AAAAAAAABBBOAAAA|57616|16|0|16|PM|second|afternoon|| +57617|AAAAAAAACBBOAAAA|57617|16|0|17|PM|second|afternoon|| +57618|AAAAAAAADBBOAAAA|57618|16|0|18|PM|second|afternoon|| +57619|AAAAAAAAEBBOAAAA|57619|16|0|19|PM|second|afternoon|| +57620|AAAAAAAAFBBOAAAA|57620|16|0|20|PM|second|afternoon|| +57621|AAAAAAAAGBBOAAAA|57621|16|0|21|PM|second|afternoon|| +57622|AAAAAAAAHBBOAAAA|57622|16|0|22|PM|second|afternoon|| +57623|AAAAAAAAIBBOAAAA|57623|16|0|23|PM|second|afternoon|| +57624|AAAAAAAAJBBOAAAA|57624|16|0|24|PM|second|afternoon|| +57625|AAAAAAAAKBBOAAAA|57625|16|0|25|PM|second|afternoon|| +57626|AAAAAAAALBBOAAAA|57626|16|0|26|PM|second|afternoon|| +57627|AAAAAAAAMBBOAAAA|57627|16|0|27|PM|second|afternoon|| +57628|AAAAAAAANBBOAAAA|57628|16|0|28|PM|second|afternoon|| +57629|AAAAAAAAOBBOAAAA|57629|16|0|29|PM|second|afternoon|| +57630|AAAAAAAAPBBOAAAA|57630|16|0|30|PM|second|afternoon|| +57631|AAAAAAAAACBOAAAA|57631|16|0|31|PM|second|afternoon|| +57632|AAAAAAAABCBOAAAA|57632|16|0|32|PM|second|afternoon|| +57633|AAAAAAAACCBOAAAA|57633|16|0|33|PM|second|afternoon|| +57634|AAAAAAAADCBOAAAA|57634|16|0|34|PM|second|afternoon|| +57635|AAAAAAAAECBOAAAA|57635|16|0|35|PM|second|afternoon|| +57636|AAAAAAAAFCBOAAAA|57636|16|0|36|PM|second|afternoon|| +57637|AAAAAAAAGCBOAAAA|57637|16|0|37|PM|second|afternoon|| +57638|AAAAAAAAHCBOAAAA|57638|16|0|38|PM|second|afternoon|| +57639|AAAAAAAAICBOAAAA|57639|16|0|39|PM|second|afternoon|| +57640|AAAAAAAAJCBOAAAA|57640|16|0|40|PM|second|afternoon|| +57641|AAAAAAAAKCBOAAAA|57641|16|0|41|PM|second|afternoon|| +57642|AAAAAAAALCBOAAAA|57642|16|0|42|PM|second|afternoon|| +57643|AAAAAAAAMCBOAAAA|57643|16|0|43|PM|second|afternoon|| +57644|AAAAAAAANCBOAAAA|57644|16|0|44|PM|second|afternoon|| +57645|AAAAAAAAOCBOAAAA|57645|16|0|45|PM|second|afternoon|| +57646|AAAAAAAAPCBOAAAA|57646|16|0|46|PM|second|afternoon|| +57647|AAAAAAAAADBOAAAA|57647|16|0|47|PM|second|afternoon|| +57648|AAAAAAAABDBOAAAA|57648|16|0|48|PM|second|afternoon|| +57649|AAAAAAAACDBOAAAA|57649|16|0|49|PM|second|afternoon|| +57650|AAAAAAAADDBOAAAA|57650|16|0|50|PM|second|afternoon|| +57651|AAAAAAAAEDBOAAAA|57651|16|0|51|PM|second|afternoon|| +57652|AAAAAAAAFDBOAAAA|57652|16|0|52|PM|second|afternoon|| +57653|AAAAAAAAGDBOAAAA|57653|16|0|53|PM|second|afternoon|| +57654|AAAAAAAAHDBOAAAA|57654|16|0|54|PM|second|afternoon|| +57655|AAAAAAAAIDBOAAAA|57655|16|0|55|PM|second|afternoon|| +57656|AAAAAAAAJDBOAAAA|57656|16|0|56|PM|second|afternoon|| +57657|AAAAAAAAKDBOAAAA|57657|16|0|57|PM|second|afternoon|| +57658|AAAAAAAALDBOAAAA|57658|16|0|58|PM|second|afternoon|| +57659|AAAAAAAAMDBOAAAA|57659|16|0|59|PM|second|afternoon|| +57660|AAAAAAAANDBOAAAA|57660|16|1|0|PM|second|afternoon|| +57661|AAAAAAAAODBOAAAA|57661|16|1|1|PM|second|afternoon|| +57662|AAAAAAAAPDBOAAAA|57662|16|1|2|PM|second|afternoon|| +57663|AAAAAAAAAEBOAAAA|57663|16|1|3|PM|second|afternoon|| +57664|AAAAAAAABEBOAAAA|57664|16|1|4|PM|second|afternoon|| +57665|AAAAAAAACEBOAAAA|57665|16|1|5|PM|second|afternoon|| +57666|AAAAAAAADEBOAAAA|57666|16|1|6|PM|second|afternoon|| +57667|AAAAAAAAEEBOAAAA|57667|16|1|7|PM|second|afternoon|| +57668|AAAAAAAAFEBOAAAA|57668|16|1|8|PM|second|afternoon|| +57669|AAAAAAAAGEBOAAAA|57669|16|1|9|PM|second|afternoon|| +57670|AAAAAAAAHEBOAAAA|57670|16|1|10|PM|second|afternoon|| +57671|AAAAAAAAIEBOAAAA|57671|16|1|11|PM|second|afternoon|| +57672|AAAAAAAAJEBOAAAA|57672|16|1|12|PM|second|afternoon|| +57673|AAAAAAAAKEBOAAAA|57673|16|1|13|PM|second|afternoon|| +57674|AAAAAAAALEBOAAAA|57674|16|1|14|PM|second|afternoon|| +57675|AAAAAAAAMEBOAAAA|57675|16|1|15|PM|second|afternoon|| +57676|AAAAAAAANEBOAAAA|57676|16|1|16|PM|second|afternoon|| +57677|AAAAAAAAOEBOAAAA|57677|16|1|17|PM|second|afternoon|| +57678|AAAAAAAAPEBOAAAA|57678|16|1|18|PM|second|afternoon|| +57679|AAAAAAAAAFBOAAAA|57679|16|1|19|PM|second|afternoon|| +57680|AAAAAAAABFBOAAAA|57680|16|1|20|PM|second|afternoon|| +57681|AAAAAAAACFBOAAAA|57681|16|1|21|PM|second|afternoon|| +57682|AAAAAAAADFBOAAAA|57682|16|1|22|PM|second|afternoon|| +57683|AAAAAAAAEFBOAAAA|57683|16|1|23|PM|second|afternoon|| +57684|AAAAAAAAFFBOAAAA|57684|16|1|24|PM|second|afternoon|| +57685|AAAAAAAAGFBOAAAA|57685|16|1|25|PM|second|afternoon|| +57686|AAAAAAAAHFBOAAAA|57686|16|1|26|PM|second|afternoon|| +57687|AAAAAAAAIFBOAAAA|57687|16|1|27|PM|second|afternoon|| +57688|AAAAAAAAJFBOAAAA|57688|16|1|28|PM|second|afternoon|| +57689|AAAAAAAAKFBOAAAA|57689|16|1|29|PM|second|afternoon|| +57690|AAAAAAAALFBOAAAA|57690|16|1|30|PM|second|afternoon|| +57691|AAAAAAAAMFBOAAAA|57691|16|1|31|PM|second|afternoon|| +57692|AAAAAAAANFBOAAAA|57692|16|1|32|PM|second|afternoon|| +57693|AAAAAAAAOFBOAAAA|57693|16|1|33|PM|second|afternoon|| +57694|AAAAAAAAPFBOAAAA|57694|16|1|34|PM|second|afternoon|| +57695|AAAAAAAAAGBOAAAA|57695|16|1|35|PM|second|afternoon|| +57696|AAAAAAAABGBOAAAA|57696|16|1|36|PM|second|afternoon|| +57697|AAAAAAAACGBOAAAA|57697|16|1|37|PM|second|afternoon|| +57698|AAAAAAAADGBOAAAA|57698|16|1|38|PM|second|afternoon|| +57699|AAAAAAAAEGBOAAAA|57699|16|1|39|PM|second|afternoon|| +57700|AAAAAAAAFGBOAAAA|57700|16|1|40|PM|second|afternoon|| +57701|AAAAAAAAGGBOAAAA|57701|16|1|41|PM|second|afternoon|| +57702|AAAAAAAAHGBOAAAA|57702|16|1|42|PM|second|afternoon|| +57703|AAAAAAAAIGBOAAAA|57703|16|1|43|PM|second|afternoon|| +57704|AAAAAAAAJGBOAAAA|57704|16|1|44|PM|second|afternoon|| +57705|AAAAAAAAKGBOAAAA|57705|16|1|45|PM|second|afternoon|| +57706|AAAAAAAALGBOAAAA|57706|16|1|46|PM|second|afternoon|| +57707|AAAAAAAAMGBOAAAA|57707|16|1|47|PM|second|afternoon|| +57708|AAAAAAAANGBOAAAA|57708|16|1|48|PM|second|afternoon|| +57709|AAAAAAAAOGBOAAAA|57709|16|1|49|PM|second|afternoon|| +57710|AAAAAAAAPGBOAAAA|57710|16|1|50|PM|second|afternoon|| +57711|AAAAAAAAAHBOAAAA|57711|16|1|51|PM|second|afternoon|| +57712|AAAAAAAABHBOAAAA|57712|16|1|52|PM|second|afternoon|| +57713|AAAAAAAACHBOAAAA|57713|16|1|53|PM|second|afternoon|| +57714|AAAAAAAADHBOAAAA|57714|16|1|54|PM|second|afternoon|| +57715|AAAAAAAAEHBOAAAA|57715|16|1|55|PM|second|afternoon|| +57716|AAAAAAAAFHBOAAAA|57716|16|1|56|PM|second|afternoon|| +57717|AAAAAAAAGHBOAAAA|57717|16|1|57|PM|second|afternoon|| +57718|AAAAAAAAHHBOAAAA|57718|16|1|58|PM|second|afternoon|| +57719|AAAAAAAAIHBOAAAA|57719|16|1|59|PM|second|afternoon|| +57720|AAAAAAAAJHBOAAAA|57720|16|2|0|PM|second|afternoon|| +57721|AAAAAAAAKHBOAAAA|57721|16|2|1|PM|second|afternoon|| +57722|AAAAAAAALHBOAAAA|57722|16|2|2|PM|second|afternoon|| +57723|AAAAAAAAMHBOAAAA|57723|16|2|3|PM|second|afternoon|| +57724|AAAAAAAANHBOAAAA|57724|16|2|4|PM|second|afternoon|| +57725|AAAAAAAAOHBOAAAA|57725|16|2|5|PM|second|afternoon|| +57726|AAAAAAAAPHBOAAAA|57726|16|2|6|PM|second|afternoon|| +57727|AAAAAAAAAIBOAAAA|57727|16|2|7|PM|second|afternoon|| +57728|AAAAAAAABIBOAAAA|57728|16|2|8|PM|second|afternoon|| +57729|AAAAAAAACIBOAAAA|57729|16|2|9|PM|second|afternoon|| +57730|AAAAAAAADIBOAAAA|57730|16|2|10|PM|second|afternoon|| +57731|AAAAAAAAEIBOAAAA|57731|16|2|11|PM|second|afternoon|| +57732|AAAAAAAAFIBOAAAA|57732|16|2|12|PM|second|afternoon|| +57733|AAAAAAAAGIBOAAAA|57733|16|2|13|PM|second|afternoon|| +57734|AAAAAAAAHIBOAAAA|57734|16|2|14|PM|second|afternoon|| +57735|AAAAAAAAIIBOAAAA|57735|16|2|15|PM|second|afternoon|| +57736|AAAAAAAAJIBOAAAA|57736|16|2|16|PM|second|afternoon|| +57737|AAAAAAAAKIBOAAAA|57737|16|2|17|PM|second|afternoon|| +57738|AAAAAAAALIBOAAAA|57738|16|2|18|PM|second|afternoon|| +57739|AAAAAAAAMIBOAAAA|57739|16|2|19|PM|second|afternoon|| +57740|AAAAAAAANIBOAAAA|57740|16|2|20|PM|second|afternoon|| +57741|AAAAAAAAOIBOAAAA|57741|16|2|21|PM|second|afternoon|| +57742|AAAAAAAAPIBOAAAA|57742|16|2|22|PM|second|afternoon|| +57743|AAAAAAAAAJBOAAAA|57743|16|2|23|PM|second|afternoon|| +57744|AAAAAAAABJBOAAAA|57744|16|2|24|PM|second|afternoon|| +57745|AAAAAAAACJBOAAAA|57745|16|2|25|PM|second|afternoon|| +57746|AAAAAAAADJBOAAAA|57746|16|2|26|PM|second|afternoon|| +57747|AAAAAAAAEJBOAAAA|57747|16|2|27|PM|second|afternoon|| +57748|AAAAAAAAFJBOAAAA|57748|16|2|28|PM|second|afternoon|| +57749|AAAAAAAAGJBOAAAA|57749|16|2|29|PM|second|afternoon|| +57750|AAAAAAAAHJBOAAAA|57750|16|2|30|PM|second|afternoon|| +57751|AAAAAAAAIJBOAAAA|57751|16|2|31|PM|second|afternoon|| +57752|AAAAAAAAJJBOAAAA|57752|16|2|32|PM|second|afternoon|| +57753|AAAAAAAAKJBOAAAA|57753|16|2|33|PM|second|afternoon|| +57754|AAAAAAAALJBOAAAA|57754|16|2|34|PM|second|afternoon|| +57755|AAAAAAAAMJBOAAAA|57755|16|2|35|PM|second|afternoon|| +57756|AAAAAAAANJBOAAAA|57756|16|2|36|PM|second|afternoon|| +57757|AAAAAAAAOJBOAAAA|57757|16|2|37|PM|second|afternoon|| +57758|AAAAAAAAPJBOAAAA|57758|16|2|38|PM|second|afternoon|| +57759|AAAAAAAAAKBOAAAA|57759|16|2|39|PM|second|afternoon|| +57760|AAAAAAAABKBOAAAA|57760|16|2|40|PM|second|afternoon|| +57761|AAAAAAAACKBOAAAA|57761|16|2|41|PM|second|afternoon|| +57762|AAAAAAAADKBOAAAA|57762|16|2|42|PM|second|afternoon|| +57763|AAAAAAAAEKBOAAAA|57763|16|2|43|PM|second|afternoon|| +57764|AAAAAAAAFKBOAAAA|57764|16|2|44|PM|second|afternoon|| +57765|AAAAAAAAGKBOAAAA|57765|16|2|45|PM|second|afternoon|| +57766|AAAAAAAAHKBOAAAA|57766|16|2|46|PM|second|afternoon|| +57767|AAAAAAAAIKBOAAAA|57767|16|2|47|PM|second|afternoon|| +57768|AAAAAAAAJKBOAAAA|57768|16|2|48|PM|second|afternoon|| +57769|AAAAAAAAKKBOAAAA|57769|16|2|49|PM|second|afternoon|| +57770|AAAAAAAALKBOAAAA|57770|16|2|50|PM|second|afternoon|| +57771|AAAAAAAAMKBOAAAA|57771|16|2|51|PM|second|afternoon|| +57772|AAAAAAAANKBOAAAA|57772|16|2|52|PM|second|afternoon|| +57773|AAAAAAAAOKBOAAAA|57773|16|2|53|PM|second|afternoon|| +57774|AAAAAAAAPKBOAAAA|57774|16|2|54|PM|second|afternoon|| +57775|AAAAAAAAALBOAAAA|57775|16|2|55|PM|second|afternoon|| +57776|AAAAAAAABLBOAAAA|57776|16|2|56|PM|second|afternoon|| +57777|AAAAAAAACLBOAAAA|57777|16|2|57|PM|second|afternoon|| +57778|AAAAAAAADLBOAAAA|57778|16|2|58|PM|second|afternoon|| +57779|AAAAAAAAELBOAAAA|57779|16|2|59|PM|second|afternoon|| +57780|AAAAAAAAFLBOAAAA|57780|16|3|0|PM|second|afternoon|| +57781|AAAAAAAAGLBOAAAA|57781|16|3|1|PM|second|afternoon|| +57782|AAAAAAAAHLBOAAAA|57782|16|3|2|PM|second|afternoon|| +57783|AAAAAAAAILBOAAAA|57783|16|3|3|PM|second|afternoon|| +57784|AAAAAAAAJLBOAAAA|57784|16|3|4|PM|second|afternoon|| +57785|AAAAAAAAKLBOAAAA|57785|16|3|5|PM|second|afternoon|| +57786|AAAAAAAALLBOAAAA|57786|16|3|6|PM|second|afternoon|| +57787|AAAAAAAAMLBOAAAA|57787|16|3|7|PM|second|afternoon|| +57788|AAAAAAAANLBOAAAA|57788|16|3|8|PM|second|afternoon|| +57789|AAAAAAAAOLBOAAAA|57789|16|3|9|PM|second|afternoon|| +57790|AAAAAAAAPLBOAAAA|57790|16|3|10|PM|second|afternoon|| +57791|AAAAAAAAAMBOAAAA|57791|16|3|11|PM|second|afternoon|| +57792|AAAAAAAABMBOAAAA|57792|16|3|12|PM|second|afternoon|| +57793|AAAAAAAACMBOAAAA|57793|16|3|13|PM|second|afternoon|| +57794|AAAAAAAADMBOAAAA|57794|16|3|14|PM|second|afternoon|| +57795|AAAAAAAAEMBOAAAA|57795|16|3|15|PM|second|afternoon|| +57796|AAAAAAAAFMBOAAAA|57796|16|3|16|PM|second|afternoon|| +57797|AAAAAAAAGMBOAAAA|57797|16|3|17|PM|second|afternoon|| +57798|AAAAAAAAHMBOAAAA|57798|16|3|18|PM|second|afternoon|| +57799|AAAAAAAAIMBOAAAA|57799|16|3|19|PM|second|afternoon|| +57800|AAAAAAAAJMBOAAAA|57800|16|3|20|PM|second|afternoon|| +57801|AAAAAAAAKMBOAAAA|57801|16|3|21|PM|second|afternoon|| +57802|AAAAAAAALMBOAAAA|57802|16|3|22|PM|second|afternoon|| +57803|AAAAAAAAMMBOAAAA|57803|16|3|23|PM|second|afternoon|| +57804|AAAAAAAANMBOAAAA|57804|16|3|24|PM|second|afternoon|| +57805|AAAAAAAAOMBOAAAA|57805|16|3|25|PM|second|afternoon|| +57806|AAAAAAAAPMBOAAAA|57806|16|3|26|PM|second|afternoon|| +57807|AAAAAAAAANBOAAAA|57807|16|3|27|PM|second|afternoon|| +57808|AAAAAAAABNBOAAAA|57808|16|3|28|PM|second|afternoon|| +57809|AAAAAAAACNBOAAAA|57809|16|3|29|PM|second|afternoon|| +57810|AAAAAAAADNBOAAAA|57810|16|3|30|PM|second|afternoon|| +57811|AAAAAAAAENBOAAAA|57811|16|3|31|PM|second|afternoon|| +57812|AAAAAAAAFNBOAAAA|57812|16|3|32|PM|second|afternoon|| +57813|AAAAAAAAGNBOAAAA|57813|16|3|33|PM|second|afternoon|| +57814|AAAAAAAAHNBOAAAA|57814|16|3|34|PM|second|afternoon|| +57815|AAAAAAAAINBOAAAA|57815|16|3|35|PM|second|afternoon|| +57816|AAAAAAAAJNBOAAAA|57816|16|3|36|PM|second|afternoon|| +57817|AAAAAAAAKNBOAAAA|57817|16|3|37|PM|second|afternoon|| +57818|AAAAAAAALNBOAAAA|57818|16|3|38|PM|second|afternoon|| +57819|AAAAAAAAMNBOAAAA|57819|16|3|39|PM|second|afternoon|| +57820|AAAAAAAANNBOAAAA|57820|16|3|40|PM|second|afternoon|| +57821|AAAAAAAAONBOAAAA|57821|16|3|41|PM|second|afternoon|| +57822|AAAAAAAAPNBOAAAA|57822|16|3|42|PM|second|afternoon|| +57823|AAAAAAAAAOBOAAAA|57823|16|3|43|PM|second|afternoon|| +57824|AAAAAAAABOBOAAAA|57824|16|3|44|PM|second|afternoon|| +57825|AAAAAAAACOBOAAAA|57825|16|3|45|PM|second|afternoon|| +57826|AAAAAAAADOBOAAAA|57826|16|3|46|PM|second|afternoon|| +57827|AAAAAAAAEOBOAAAA|57827|16|3|47|PM|second|afternoon|| +57828|AAAAAAAAFOBOAAAA|57828|16|3|48|PM|second|afternoon|| +57829|AAAAAAAAGOBOAAAA|57829|16|3|49|PM|second|afternoon|| +57830|AAAAAAAAHOBOAAAA|57830|16|3|50|PM|second|afternoon|| +57831|AAAAAAAAIOBOAAAA|57831|16|3|51|PM|second|afternoon|| +57832|AAAAAAAAJOBOAAAA|57832|16|3|52|PM|second|afternoon|| +57833|AAAAAAAAKOBOAAAA|57833|16|3|53|PM|second|afternoon|| +57834|AAAAAAAALOBOAAAA|57834|16|3|54|PM|second|afternoon|| +57835|AAAAAAAAMOBOAAAA|57835|16|3|55|PM|second|afternoon|| +57836|AAAAAAAANOBOAAAA|57836|16|3|56|PM|second|afternoon|| +57837|AAAAAAAAOOBOAAAA|57837|16|3|57|PM|second|afternoon|| +57838|AAAAAAAAPOBOAAAA|57838|16|3|58|PM|second|afternoon|| +57839|AAAAAAAAAPBOAAAA|57839|16|3|59|PM|second|afternoon|| +57840|AAAAAAAABPBOAAAA|57840|16|4|0|PM|second|afternoon|| +57841|AAAAAAAACPBOAAAA|57841|16|4|1|PM|second|afternoon|| +57842|AAAAAAAADPBOAAAA|57842|16|4|2|PM|second|afternoon|| +57843|AAAAAAAAEPBOAAAA|57843|16|4|3|PM|second|afternoon|| +57844|AAAAAAAAFPBOAAAA|57844|16|4|4|PM|second|afternoon|| +57845|AAAAAAAAGPBOAAAA|57845|16|4|5|PM|second|afternoon|| +57846|AAAAAAAAHPBOAAAA|57846|16|4|6|PM|second|afternoon|| +57847|AAAAAAAAIPBOAAAA|57847|16|4|7|PM|second|afternoon|| +57848|AAAAAAAAJPBOAAAA|57848|16|4|8|PM|second|afternoon|| +57849|AAAAAAAAKPBOAAAA|57849|16|4|9|PM|second|afternoon|| +57850|AAAAAAAALPBOAAAA|57850|16|4|10|PM|second|afternoon|| +57851|AAAAAAAAMPBOAAAA|57851|16|4|11|PM|second|afternoon|| +57852|AAAAAAAANPBOAAAA|57852|16|4|12|PM|second|afternoon|| +57853|AAAAAAAAOPBOAAAA|57853|16|4|13|PM|second|afternoon|| +57854|AAAAAAAAPPBOAAAA|57854|16|4|14|PM|second|afternoon|| +57855|AAAAAAAAAACOAAAA|57855|16|4|15|PM|second|afternoon|| +57856|AAAAAAAABACOAAAA|57856|16|4|16|PM|second|afternoon|| +57857|AAAAAAAACACOAAAA|57857|16|4|17|PM|second|afternoon|| +57858|AAAAAAAADACOAAAA|57858|16|4|18|PM|second|afternoon|| +57859|AAAAAAAAEACOAAAA|57859|16|4|19|PM|second|afternoon|| +57860|AAAAAAAAFACOAAAA|57860|16|4|20|PM|second|afternoon|| +57861|AAAAAAAAGACOAAAA|57861|16|4|21|PM|second|afternoon|| +57862|AAAAAAAAHACOAAAA|57862|16|4|22|PM|second|afternoon|| +57863|AAAAAAAAIACOAAAA|57863|16|4|23|PM|second|afternoon|| +57864|AAAAAAAAJACOAAAA|57864|16|4|24|PM|second|afternoon|| +57865|AAAAAAAAKACOAAAA|57865|16|4|25|PM|second|afternoon|| +57866|AAAAAAAALACOAAAA|57866|16|4|26|PM|second|afternoon|| +57867|AAAAAAAAMACOAAAA|57867|16|4|27|PM|second|afternoon|| +57868|AAAAAAAANACOAAAA|57868|16|4|28|PM|second|afternoon|| +57869|AAAAAAAAOACOAAAA|57869|16|4|29|PM|second|afternoon|| +57870|AAAAAAAAPACOAAAA|57870|16|4|30|PM|second|afternoon|| +57871|AAAAAAAAABCOAAAA|57871|16|4|31|PM|second|afternoon|| +57872|AAAAAAAABBCOAAAA|57872|16|4|32|PM|second|afternoon|| +57873|AAAAAAAACBCOAAAA|57873|16|4|33|PM|second|afternoon|| +57874|AAAAAAAADBCOAAAA|57874|16|4|34|PM|second|afternoon|| +57875|AAAAAAAAEBCOAAAA|57875|16|4|35|PM|second|afternoon|| +57876|AAAAAAAAFBCOAAAA|57876|16|4|36|PM|second|afternoon|| +57877|AAAAAAAAGBCOAAAA|57877|16|4|37|PM|second|afternoon|| +57878|AAAAAAAAHBCOAAAA|57878|16|4|38|PM|second|afternoon|| +57879|AAAAAAAAIBCOAAAA|57879|16|4|39|PM|second|afternoon|| +57880|AAAAAAAAJBCOAAAA|57880|16|4|40|PM|second|afternoon|| +57881|AAAAAAAAKBCOAAAA|57881|16|4|41|PM|second|afternoon|| +57882|AAAAAAAALBCOAAAA|57882|16|4|42|PM|second|afternoon|| +57883|AAAAAAAAMBCOAAAA|57883|16|4|43|PM|second|afternoon|| +57884|AAAAAAAANBCOAAAA|57884|16|4|44|PM|second|afternoon|| +57885|AAAAAAAAOBCOAAAA|57885|16|4|45|PM|second|afternoon|| +57886|AAAAAAAAPBCOAAAA|57886|16|4|46|PM|second|afternoon|| +57887|AAAAAAAAACCOAAAA|57887|16|4|47|PM|second|afternoon|| +57888|AAAAAAAABCCOAAAA|57888|16|4|48|PM|second|afternoon|| +57889|AAAAAAAACCCOAAAA|57889|16|4|49|PM|second|afternoon|| +57890|AAAAAAAADCCOAAAA|57890|16|4|50|PM|second|afternoon|| +57891|AAAAAAAAECCOAAAA|57891|16|4|51|PM|second|afternoon|| +57892|AAAAAAAAFCCOAAAA|57892|16|4|52|PM|second|afternoon|| +57893|AAAAAAAAGCCOAAAA|57893|16|4|53|PM|second|afternoon|| +57894|AAAAAAAAHCCOAAAA|57894|16|4|54|PM|second|afternoon|| +57895|AAAAAAAAICCOAAAA|57895|16|4|55|PM|second|afternoon|| +57896|AAAAAAAAJCCOAAAA|57896|16|4|56|PM|second|afternoon|| +57897|AAAAAAAAKCCOAAAA|57897|16|4|57|PM|second|afternoon|| +57898|AAAAAAAALCCOAAAA|57898|16|4|58|PM|second|afternoon|| +57899|AAAAAAAAMCCOAAAA|57899|16|4|59|PM|second|afternoon|| +57900|AAAAAAAANCCOAAAA|57900|16|5|0|PM|second|afternoon|| +57901|AAAAAAAAOCCOAAAA|57901|16|5|1|PM|second|afternoon|| +57902|AAAAAAAAPCCOAAAA|57902|16|5|2|PM|second|afternoon|| +57903|AAAAAAAAADCOAAAA|57903|16|5|3|PM|second|afternoon|| +57904|AAAAAAAABDCOAAAA|57904|16|5|4|PM|second|afternoon|| +57905|AAAAAAAACDCOAAAA|57905|16|5|5|PM|second|afternoon|| +57906|AAAAAAAADDCOAAAA|57906|16|5|6|PM|second|afternoon|| +57907|AAAAAAAAEDCOAAAA|57907|16|5|7|PM|second|afternoon|| +57908|AAAAAAAAFDCOAAAA|57908|16|5|8|PM|second|afternoon|| +57909|AAAAAAAAGDCOAAAA|57909|16|5|9|PM|second|afternoon|| +57910|AAAAAAAAHDCOAAAA|57910|16|5|10|PM|second|afternoon|| +57911|AAAAAAAAIDCOAAAA|57911|16|5|11|PM|second|afternoon|| +57912|AAAAAAAAJDCOAAAA|57912|16|5|12|PM|second|afternoon|| +57913|AAAAAAAAKDCOAAAA|57913|16|5|13|PM|second|afternoon|| +57914|AAAAAAAALDCOAAAA|57914|16|5|14|PM|second|afternoon|| +57915|AAAAAAAAMDCOAAAA|57915|16|5|15|PM|second|afternoon|| +57916|AAAAAAAANDCOAAAA|57916|16|5|16|PM|second|afternoon|| +57917|AAAAAAAAODCOAAAA|57917|16|5|17|PM|second|afternoon|| +57918|AAAAAAAAPDCOAAAA|57918|16|5|18|PM|second|afternoon|| +57919|AAAAAAAAAECOAAAA|57919|16|5|19|PM|second|afternoon|| +57920|AAAAAAAABECOAAAA|57920|16|5|20|PM|second|afternoon|| +57921|AAAAAAAACECOAAAA|57921|16|5|21|PM|second|afternoon|| +57922|AAAAAAAADECOAAAA|57922|16|5|22|PM|second|afternoon|| +57923|AAAAAAAAEECOAAAA|57923|16|5|23|PM|second|afternoon|| +57924|AAAAAAAAFECOAAAA|57924|16|5|24|PM|second|afternoon|| +57925|AAAAAAAAGECOAAAA|57925|16|5|25|PM|second|afternoon|| +57926|AAAAAAAAHECOAAAA|57926|16|5|26|PM|second|afternoon|| +57927|AAAAAAAAIECOAAAA|57927|16|5|27|PM|second|afternoon|| +57928|AAAAAAAAJECOAAAA|57928|16|5|28|PM|second|afternoon|| +57929|AAAAAAAAKECOAAAA|57929|16|5|29|PM|second|afternoon|| +57930|AAAAAAAALECOAAAA|57930|16|5|30|PM|second|afternoon|| +57931|AAAAAAAAMECOAAAA|57931|16|5|31|PM|second|afternoon|| +57932|AAAAAAAANECOAAAA|57932|16|5|32|PM|second|afternoon|| +57933|AAAAAAAAOECOAAAA|57933|16|5|33|PM|second|afternoon|| +57934|AAAAAAAAPECOAAAA|57934|16|5|34|PM|second|afternoon|| +57935|AAAAAAAAAFCOAAAA|57935|16|5|35|PM|second|afternoon|| +57936|AAAAAAAABFCOAAAA|57936|16|5|36|PM|second|afternoon|| +57937|AAAAAAAACFCOAAAA|57937|16|5|37|PM|second|afternoon|| +57938|AAAAAAAADFCOAAAA|57938|16|5|38|PM|second|afternoon|| +57939|AAAAAAAAEFCOAAAA|57939|16|5|39|PM|second|afternoon|| +57940|AAAAAAAAFFCOAAAA|57940|16|5|40|PM|second|afternoon|| +57941|AAAAAAAAGFCOAAAA|57941|16|5|41|PM|second|afternoon|| +57942|AAAAAAAAHFCOAAAA|57942|16|5|42|PM|second|afternoon|| +57943|AAAAAAAAIFCOAAAA|57943|16|5|43|PM|second|afternoon|| +57944|AAAAAAAAJFCOAAAA|57944|16|5|44|PM|second|afternoon|| +57945|AAAAAAAAKFCOAAAA|57945|16|5|45|PM|second|afternoon|| +57946|AAAAAAAALFCOAAAA|57946|16|5|46|PM|second|afternoon|| +57947|AAAAAAAAMFCOAAAA|57947|16|5|47|PM|second|afternoon|| +57948|AAAAAAAANFCOAAAA|57948|16|5|48|PM|second|afternoon|| +57949|AAAAAAAAOFCOAAAA|57949|16|5|49|PM|second|afternoon|| +57950|AAAAAAAAPFCOAAAA|57950|16|5|50|PM|second|afternoon|| +57951|AAAAAAAAAGCOAAAA|57951|16|5|51|PM|second|afternoon|| +57952|AAAAAAAABGCOAAAA|57952|16|5|52|PM|second|afternoon|| +57953|AAAAAAAACGCOAAAA|57953|16|5|53|PM|second|afternoon|| +57954|AAAAAAAADGCOAAAA|57954|16|5|54|PM|second|afternoon|| +57955|AAAAAAAAEGCOAAAA|57955|16|5|55|PM|second|afternoon|| +57956|AAAAAAAAFGCOAAAA|57956|16|5|56|PM|second|afternoon|| +57957|AAAAAAAAGGCOAAAA|57957|16|5|57|PM|second|afternoon|| +57958|AAAAAAAAHGCOAAAA|57958|16|5|58|PM|second|afternoon|| +57959|AAAAAAAAIGCOAAAA|57959|16|5|59|PM|second|afternoon|| +57960|AAAAAAAAJGCOAAAA|57960|16|6|0|PM|second|afternoon|| +57961|AAAAAAAAKGCOAAAA|57961|16|6|1|PM|second|afternoon|| +57962|AAAAAAAALGCOAAAA|57962|16|6|2|PM|second|afternoon|| +57963|AAAAAAAAMGCOAAAA|57963|16|6|3|PM|second|afternoon|| +57964|AAAAAAAANGCOAAAA|57964|16|6|4|PM|second|afternoon|| +57965|AAAAAAAAOGCOAAAA|57965|16|6|5|PM|second|afternoon|| +57966|AAAAAAAAPGCOAAAA|57966|16|6|6|PM|second|afternoon|| +57967|AAAAAAAAAHCOAAAA|57967|16|6|7|PM|second|afternoon|| +57968|AAAAAAAABHCOAAAA|57968|16|6|8|PM|second|afternoon|| +57969|AAAAAAAACHCOAAAA|57969|16|6|9|PM|second|afternoon|| +57970|AAAAAAAADHCOAAAA|57970|16|6|10|PM|second|afternoon|| +57971|AAAAAAAAEHCOAAAA|57971|16|6|11|PM|second|afternoon|| +57972|AAAAAAAAFHCOAAAA|57972|16|6|12|PM|second|afternoon|| +57973|AAAAAAAAGHCOAAAA|57973|16|6|13|PM|second|afternoon|| +57974|AAAAAAAAHHCOAAAA|57974|16|6|14|PM|second|afternoon|| +57975|AAAAAAAAIHCOAAAA|57975|16|6|15|PM|second|afternoon|| +57976|AAAAAAAAJHCOAAAA|57976|16|6|16|PM|second|afternoon|| +57977|AAAAAAAAKHCOAAAA|57977|16|6|17|PM|second|afternoon|| +57978|AAAAAAAALHCOAAAA|57978|16|6|18|PM|second|afternoon|| +57979|AAAAAAAAMHCOAAAA|57979|16|6|19|PM|second|afternoon|| +57980|AAAAAAAANHCOAAAA|57980|16|6|20|PM|second|afternoon|| +57981|AAAAAAAAOHCOAAAA|57981|16|6|21|PM|second|afternoon|| +57982|AAAAAAAAPHCOAAAA|57982|16|6|22|PM|second|afternoon|| +57983|AAAAAAAAAICOAAAA|57983|16|6|23|PM|second|afternoon|| +57984|AAAAAAAABICOAAAA|57984|16|6|24|PM|second|afternoon|| +57985|AAAAAAAACICOAAAA|57985|16|6|25|PM|second|afternoon|| +57986|AAAAAAAADICOAAAA|57986|16|6|26|PM|second|afternoon|| +57987|AAAAAAAAEICOAAAA|57987|16|6|27|PM|second|afternoon|| +57988|AAAAAAAAFICOAAAA|57988|16|6|28|PM|second|afternoon|| +57989|AAAAAAAAGICOAAAA|57989|16|6|29|PM|second|afternoon|| +57990|AAAAAAAAHICOAAAA|57990|16|6|30|PM|second|afternoon|| +57991|AAAAAAAAIICOAAAA|57991|16|6|31|PM|second|afternoon|| +57992|AAAAAAAAJICOAAAA|57992|16|6|32|PM|second|afternoon|| +57993|AAAAAAAAKICOAAAA|57993|16|6|33|PM|second|afternoon|| +57994|AAAAAAAALICOAAAA|57994|16|6|34|PM|second|afternoon|| +57995|AAAAAAAAMICOAAAA|57995|16|6|35|PM|second|afternoon|| +57996|AAAAAAAANICOAAAA|57996|16|6|36|PM|second|afternoon|| +57997|AAAAAAAAOICOAAAA|57997|16|6|37|PM|second|afternoon|| +57998|AAAAAAAAPICOAAAA|57998|16|6|38|PM|second|afternoon|| +57999|AAAAAAAAAJCOAAAA|57999|16|6|39|PM|second|afternoon|| +58000|AAAAAAAABJCOAAAA|58000|16|6|40|PM|second|afternoon|| +58001|AAAAAAAACJCOAAAA|58001|16|6|41|PM|second|afternoon|| +58002|AAAAAAAADJCOAAAA|58002|16|6|42|PM|second|afternoon|| +58003|AAAAAAAAEJCOAAAA|58003|16|6|43|PM|second|afternoon|| +58004|AAAAAAAAFJCOAAAA|58004|16|6|44|PM|second|afternoon|| +58005|AAAAAAAAGJCOAAAA|58005|16|6|45|PM|second|afternoon|| +58006|AAAAAAAAHJCOAAAA|58006|16|6|46|PM|second|afternoon|| +58007|AAAAAAAAIJCOAAAA|58007|16|6|47|PM|second|afternoon|| +58008|AAAAAAAAJJCOAAAA|58008|16|6|48|PM|second|afternoon|| +58009|AAAAAAAAKJCOAAAA|58009|16|6|49|PM|second|afternoon|| +58010|AAAAAAAALJCOAAAA|58010|16|6|50|PM|second|afternoon|| +58011|AAAAAAAAMJCOAAAA|58011|16|6|51|PM|second|afternoon|| +58012|AAAAAAAANJCOAAAA|58012|16|6|52|PM|second|afternoon|| +58013|AAAAAAAAOJCOAAAA|58013|16|6|53|PM|second|afternoon|| +58014|AAAAAAAAPJCOAAAA|58014|16|6|54|PM|second|afternoon|| +58015|AAAAAAAAAKCOAAAA|58015|16|6|55|PM|second|afternoon|| +58016|AAAAAAAABKCOAAAA|58016|16|6|56|PM|second|afternoon|| +58017|AAAAAAAACKCOAAAA|58017|16|6|57|PM|second|afternoon|| +58018|AAAAAAAADKCOAAAA|58018|16|6|58|PM|second|afternoon|| +58019|AAAAAAAAEKCOAAAA|58019|16|6|59|PM|second|afternoon|| +58020|AAAAAAAAFKCOAAAA|58020|16|7|0|PM|second|afternoon|| +58021|AAAAAAAAGKCOAAAA|58021|16|7|1|PM|second|afternoon|| +58022|AAAAAAAAHKCOAAAA|58022|16|7|2|PM|second|afternoon|| +58023|AAAAAAAAIKCOAAAA|58023|16|7|3|PM|second|afternoon|| +58024|AAAAAAAAJKCOAAAA|58024|16|7|4|PM|second|afternoon|| +58025|AAAAAAAAKKCOAAAA|58025|16|7|5|PM|second|afternoon|| +58026|AAAAAAAALKCOAAAA|58026|16|7|6|PM|second|afternoon|| +58027|AAAAAAAAMKCOAAAA|58027|16|7|7|PM|second|afternoon|| +58028|AAAAAAAANKCOAAAA|58028|16|7|8|PM|second|afternoon|| +58029|AAAAAAAAOKCOAAAA|58029|16|7|9|PM|second|afternoon|| +58030|AAAAAAAAPKCOAAAA|58030|16|7|10|PM|second|afternoon|| +58031|AAAAAAAAALCOAAAA|58031|16|7|11|PM|second|afternoon|| +58032|AAAAAAAABLCOAAAA|58032|16|7|12|PM|second|afternoon|| +58033|AAAAAAAACLCOAAAA|58033|16|7|13|PM|second|afternoon|| +58034|AAAAAAAADLCOAAAA|58034|16|7|14|PM|second|afternoon|| +58035|AAAAAAAAELCOAAAA|58035|16|7|15|PM|second|afternoon|| +58036|AAAAAAAAFLCOAAAA|58036|16|7|16|PM|second|afternoon|| +58037|AAAAAAAAGLCOAAAA|58037|16|7|17|PM|second|afternoon|| +58038|AAAAAAAAHLCOAAAA|58038|16|7|18|PM|second|afternoon|| +58039|AAAAAAAAILCOAAAA|58039|16|7|19|PM|second|afternoon|| +58040|AAAAAAAAJLCOAAAA|58040|16|7|20|PM|second|afternoon|| +58041|AAAAAAAAKLCOAAAA|58041|16|7|21|PM|second|afternoon|| +58042|AAAAAAAALLCOAAAA|58042|16|7|22|PM|second|afternoon|| +58043|AAAAAAAAMLCOAAAA|58043|16|7|23|PM|second|afternoon|| +58044|AAAAAAAANLCOAAAA|58044|16|7|24|PM|second|afternoon|| +58045|AAAAAAAAOLCOAAAA|58045|16|7|25|PM|second|afternoon|| +58046|AAAAAAAAPLCOAAAA|58046|16|7|26|PM|second|afternoon|| +58047|AAAAAAAAAMCOAAAA|58047|16|7|27|PM|second|afternoon|| +58048|AAAAAAAABMCOAAAA|58048|16|7|28|PM|second|afternoon|| +58049|AAAAAAAACMCOAAAA|58049|16|7|29|PM|second|afternoon|| +58050|AAAAAAAADMCOAAAA|58050|16|7|30|PM|second|afternoon|| +58051|AAAAAAAAEMCOAAAA|58051|16|7|31|PM|second|afternoon|| +58052|AAAAAAAAFMCOAAAA|58052|16|7|32|PM|second|afternoon|| +58053|AAAAAAAAGMCOAAAA|58053|16|7|33|PM|second|afternoon|| +58054|AAAAAAAAHMCOAAAA|58054|16|7|34|PM|second|afternoon|| +58055|AAAAAAAAIMCOAAAA|58055|16|7|35|PM|second|afternoon|| +58056|AAAAAAAAJMCOAAAA|58056|16|7|36|PM|second|afternoon|| +58057|AAAAAAAAKMCOAAAA|58057|16|7|37|PM|second|afternoon|| +58058|AAAAAAAALMCOAAAA|58058|16|7|38|PM|second|afternoon|| +58059|AAAAAAAAMMCOAAAA|58059|16|7|39|PM|second|afternoon|| +58060|AAAAAAAANMCOAAAA|58060|16|7|40|PM|second|afternoon|| +58061|AAAAAAAAOMCOAAAA|58061|16|7|41|PM|second|afternoon|| +58062|AAAAAAAAPMCOAAAA|58062|16|7|42|PM|second|afternoon|| +58063|AAAAAAAAANCOAAAA|58063|16|7|43|PM|second|afternoon|| +58064|AAAAAAAABNCOAAAA|58064|16|7|44|PM|second|afternoon|| +58065|AAAAAAAACNCOAAAA|58065|16|7|45|PM|second|afternoon|| +58066|AAAAAAAADNCOAAAA|58066|16|7|46|PM|second|afternoon|| +58067|AAAAAAAAENCOAAAA|58067|16|7|47|PM|second|afternoon|| +58068|AAAAAAAAFNCOAAAA|58068|16|7|48|PM|second|afternoon|| +58069|AAAAAAAAGNCOAAAA|58069|16|7|49|PM|second|afternoon|| +58070|AAAAAAAAHNCOAAAA|58070|16|7|50|PM|second|afternoon|| +58071|AAAAAAAAINCOAAAA|58071|16|7|51|PM|second|afternoon|| +58072|AAAAAAAAJNCOAAAA|58072|16|7|52|PM|second|afternoon|| +58073|AAAAAAAAKNCOAAAA|58073|16|7|53|PM|second|afternoon|| +58074|AAAAAAAALNCOAAAA|58074|16|7|54|PM|second|afternoon|| +58075|AAAAAAAAMNCOAAAA|58075|16|7|55|PM|second|afternoon|| +58076|AAAAAAAANNCOAAAA|58076|16|7|56|PM|second|afternoon|| +58077|AAAAAAAAONCOAAAA|58077|16|7|57|PM|second|afternoon|| +58078|AAAAAAAAPNCOAAAA|58078|16|7|58|PM|second|afternoon|| +58079|AAAAAAAAAOCOAAAA|58079|16|7|59|PM|second|afternoon|| +58080|AAAAAAAABOCOAAAA|58080|16|8|0|PM|second|afternoon|| +58081|AAAAAAAACOCOAAAA|58081|16|8|1|PM|second|afternoon|| +58082|AAAAAAAADOCOAAAA|58082|16|8|2|PM|second|afternoon|| +58083|AAAAAAAAEOCOAAAA|58083|16|8|3|PM|second|afternoon|| +58084|AAAAAAAAFOCOAAAA|58084|16|8|4|PM|second|afternoon|| +58085|AAAAAAAAGOCOAAAA|58085|16|8|5|PM|second|afternoon|| +58086|AAAAAAAAHOCOAAAA|58086|16|8|6|PM|second|afternoon|| +58087|AAAAAAAAIOCOAAAA|58087|16|8|7|PM|second|afternoon|| +58088|AAAAAAAAJOCOAAAA|58088|16|8|8|PM|second|afternoon|| +58089|AAAAAAAAKOCOAAAA|58089|16|8|9|PM|second|afternoon|| +58090|AAAAAAAALOCOAAAA|58090|16|8|10|PM|second|afternoon|| +58091|AAAAAAAAMOCOAAAA|58091|16|8|11|PM|second|afternoon|| +58092|AAAAAAAANOCOAAAA|58092|16|8|12|PM|second|afternoon|| +58093|AAAAAAAAOOCOAAAA|58093|16|8|13|PM|second|afternoon|| +58094|AAAAAAAAPOCOAAAA|58094|16|8|14|PM|second|afternoon|| +58095|AAAAAAAAAPCOAAAA|58095|16|8|15|PM|second|afternoon|| +58096|AAAAAAAABPCOAAAA|58096|16|8|16|PM|second|afternoon|| +58097|AAAAAAAACPCOAAAA|58097|16|8|17|PM|second|afternoon|| +58098|AAAAAAAADPCOAAAA|58098|16|8|18|PM|second|afternoon|| +58099|AAAAAAAAEPCOAAAA|58099|16|8|19|PM|second|afternoon|| +58100|AAAAAAAAFPCOAAAA|58100|16|8|20|PM|second|afternoon|| +58101|AAAAAAAAGPCOAAAA|58101|16|8|21|PM|second|afternoon|| +58102|AAAAAAAAHPCOAAAA|58102|16|8|22|PM|second|afternoon|| +58103|AAAAAAAAIPCOAAAA|58103|16|8|23|PM|second|afternoon|| +58104|AAAAAAAAJPCOAAAA|58104|16|8|24|PM|second|afternoon|| +58105|AAAAAAAAKPCOAAAA|58105|16|8|25|PM|second|afternoon|| +58106|AAAAAAAALPCOAAAA|58106|16|8|26|PM|second|afternoon|| +58107|AAAAAAAAMPCOAAAA|58107|16|8|27|PM|second|afternoon|| +58108|AAAAAAAANPCOAAAA|58108|16|8|28|PM|second|afternoon|| +58109|AAAAAAAAOPCOAAAA|58109|16|8|29|PM|second|afternoon|| +58110|AAAAAAAAPPCOAAAA|58110|16|8|30|PM|second|afternoon|| +58111|AAAAAAAAAADOAAAA|58111|16|8|31|PM|second|afternoon|| +58112|AAAAAAAABADOAAAA|58112|16|8|32|PM|second|afternoon|| +58113|AAAAAAAACADOAAAA|58113|16|8|33|PM|second|afternoon|| +58114|AAAAAAAADADOAAAA|58114|16|8|34|PM|second|afternoon|| +58115|AAAAAAAAEADOAAAA|58115|16|8|35|PM|second|afternoon|| +58116|AAAAAAAAFADOAAAA|58116|16|8|36|PM|second|afternoon|| +58117|AAAAAAAAGADOAAAA|58117|16|8|37|PM|second|afternoon|| +58118|AAAAAAAAHADOAAAA|58118|16|8|38|PM|second|afternoon|| +58119|AAAAAAAAIADOAAAA|58119|16|8|39|PM|second|afternoon|| +58120|AAAAAAAAJADOAAAA|58120|16|8|40|PM|second|afternoon|| +58121|AAAAAAAAKADOAAAA|58121|16|8|41|PM|second|afternoon|| +58122|AAAAAAAALADOAAAA|58122|16|8|42|PM|second|afternoon|| +58123|AAAAAAAAMADOAAAA|58123|16|8|43|PM|second|afternoon|| +58124|AAAAAAAANADOAAAA|58124|16|8|44|PM|second|afternoon|| +58125|AAAAAAAAOADOAAAA|58125|16|8|45|PM|second|afternoon|| +58126|AAAAAAAAPADOAAAA|58126|16|8|46|PM|second|afternoon|| +58127|AAAAAAAAABDOAAAA|58127|16|8|47|PM|second|afternoon|| +58128|AAAAAAAABBDOAAAA|58128|16|8|48|PM|second|afternoon|| +58129|AAAAAAAACBDOAAAA|58129|16|8|49|PM|second|afternoon|| +58130|AAAAAAAADBDOAAAA|58130|16|8|50|PM|second|afternoon|| +58131|AAAAAAAAEBDOAAAA|58131|16|8|51|PM|second|afternoon|| +58132|AAAAAAAAFBDOAAAA|58132|16|8|52|PM|second|afternoon|| +58133|AAAAAAAAGBDOAAAA|58133|16|8|53|PM|second|afternoon|| +58134|AAAAAAAAHBDOAAAA|58134|16|8|54|PM|second|afternoon|| +58135|AAAAAAAAIBDOAAAA|58135|16|8|55|PM|second|afternoon|| +58136|AAAAAAAAJBDOAAAA|58136|16|8|56|PM|second|afternoon|| +58137|AAAAAAAAKBDOAAAA|58137|16|8|57|PM|second|afternoon|| +58138|AAAAAAAALBDOAAAA|58138|16|8|58|PM|second|afternoon|| +58139|AAAAAAAAMBDOAAAA|58139|16|8|59|PM|second|afternoon|| +58140|AAAAAAAANBDOAAAA|58140|16|9|0|PM|second|afternoon|| +58141|AAAAAAAAOBDOAAAA|58141|16|9|1|PM|second|afternoon|| +58142|AAAAAAAAPBDOAAAA|58142|16|9|2|PM|second|afternoon|| +58143|AAAAAAAAACDOAAAA|58143|16|9|3|PM|second|afternoon|| +58144|AAAAAAAABCDOAAAA|58144|16|9|4|PM|second|afternoon|| +58145|AAAAAAAACCDOAAAA|58145|16|9|5|PM|second|afternoon|| +58146|AAAAAAAADCDOAAAA|58146|16|9|6|PM|second|afternoon|| +58147|AAAAAAAAECDOAAAA|58147|16|9|7|PM|second|afternoon|| +58148|AAAAAAAAFCDOAAAA|58148|16|9|8|PM|second|afternoon|| +58149|AAAAAAAAGCDOAAAA|58149|16|9|9|PM|second|afternoon|| +58150|AAAAAAAAHCDOAAAA|58150|16|9|10|PM|second|afternoon|| +58151|AAAAAAAAICDOAAAA|58151|16|9|11|PM|second|afternoon|| +58152|AAAAAAAAJCDOAAAA|58152|16|9|12|PM|second|afternoon|| +58153|AAAAAAAAKCDOAAAA|58153|16|9|13|PM|second|afternoon|| +58154|AAAAAAAALCDOAAAA|58154|16|9|14|PM|second|afternoon|| +58155|AAAAAAAAMCDOAAAA|58155|16|9|15|PM|second|afternoon|| +58156|AAAAAAAANCDOAAAA|58156|16|9|16|PM|second|afternoon|| +58157|AAAAAAAAOCDOAAAA|58157|16|9|17|PM|second|afternoon|| +58158|AAAAAAAAPCDOAAAA|58158|16|9|18|PM|second|afternoon|| +58159|AAAAAAAAADDOAAAA|58159|16|9|19|PM|second|afternoon|| +58160|AAAAAAAABDDOAAAA|58160|16|9|20|PM|second|afternoon|| +58161|AAAAAAAACDDOAAAA|58161|16|9|21|PM|second|afternoon|| +58162|AAAAAAAADDDOAAAA|58162|16|9|22|PM|second|afternoon|| +58163|AAAAAAAAEDDOAAAA|58163|16|9|23|PM|second|afternoon|| +58164|AAAAAAAAFDDOAAAA|58164|16|9|24|PM|second|afternoon|| +58165|AAAAAAAAGDDOAAAA|58165|16|9|25|PM|second|afternoon|| +58166|AAAAAAAAHDDOAAAA|58166|16|9|26|PM|second|afternoon|| +58167|AAAAAAAAIDDOAAAA|58167|16|9|27|PM|second|afternoon|| +58168|AAAAAAAAJDDOAAAA|58168|16|9|28|PM|second|afternoon|| +58169|AAAAAAAAKDDOAAAA|58169|16|9|29|PM|second|afternoon|| +58170|AAAAAAAALDDOAAAA|58170|16|9|30|PM|second|afternoon|| +58171|AAAAAAAAMDDOAAAA|58171|16|9|31|PM|second|afternoon|| +58172|AAAAAAAANDDOAAAA|58172|16|9|32|PM|second|afternoon|| +58173|AAAAAAAAODDOAAAA|58173|16|9|33|PM|second|afternoon|| +58174|AAAAAAAAPDDOAAAA|58174|16|9|34|PM|second|afternoon|| +58175|AAAAAAAAAEDOAAAA|58175|16|9|35|PM|second|afternoon|| +58176|AAAAAAAABEDOAAAA|58176|16|9|36|PM|second|afternoon|| +58177|AAAAAAAACEDOAAAA|58177|16|9|37|PM|second|afternoon|| +58178|AAAAAAAADEDOAAAA|58178|16|9|38|PM|second|afternoon|| +58179|AAAAAAAAEEDOAAAA|58179|16|9|39|PM|second|afternoon|| +58180|AAAAAAAAFEDOAAAA|58180|16|9|40|PM|second|afternoon|| +58181|AAAAAAAAGEDOAAAA|58181|16|9|41|PM|second|afternoon|| +58182|AAAAAAAAHEDOAAAA|58182|16|9|42|PM|second|afternoon|| +58183|AAAAAAAAIEDOAAAA|58183|16|9|43|PM|second|afternoon|| +58184|AAAAAAAAJEDOAAAA|58184|16|9|44|PM|second|afternoon|| +58185|AAAAAAAAKEDOAAAA|58185|16|9|45|PM|second|afternoon|| +58186|AAAAAAAALEDOAAAA|58186|16|9|46|PM|second|afternoon|| +58187|AAAAAAAAMEDOAAAA|58187|16|9|47|PM|second|afternoon|| +58188|AAAAAAAANEDOAAAA|58188|16|9|48|PM|second|afternoon|| +58189|AAAAAAAAOEDOAAAA|58189|16|9|49|PM|second|afternoon|| +58190|AAAAAAAAPEDOAAAA|58190|16|9|50|PM|second|afternoon|| +58191|AAAAAAAAAFDOAAAA|58191|16|9|51|PM|second|afternoon|| +58192|AAAAAAAABFDOAAAA|58192|16|9|52|PM|second|afternoon|| +58193|AAAAAAAACFDOAAAA|58193|16|9|53|PM|second|afternoon|| +58194|AAAAAAAADFDOAAAA|58194|16|9|54|PM|second|afternoon|| +58195|AAAAAAAAEFDOAAAA|58195|16|9|55|PM|second|afternoon|| +58196|AAAAAAAAFFDOAAAA|58196|16|9|56|PM|second|afternoon|| +58197|AAAAAAAAGFDOAAAA|58197|16|9|57|PM|second|afternoon|| +58198|AAAAAAAAHFDOAAAA|58198|16|9|58|PM|second|afternoon|| +58199|AAAAAAAAIFDOAAAA|58199|16|9|59|PM|second|afternoon|| +58200|AAAAAAAAJFDOAAAA|58200|16|10|0|PM|second|afternoon|| +58201|AAAAAAAAKFDOAAAA|58201|16|10|1|PM|second|afternoon|| +58202|AAAAAAAALFDOAAAA|58202|16|10|2|PM|second|afternoon|| +58203|AAAAAAAAMFDOAAAA|58203|16|10|3|PM|second|afternoon|| +58204|AAAAAAAANFDOAAAA|58204|16|10|4|PM|second|afternoon|| +58205|AAAAAAAAOFDOAAAA|58205|16|10|5|PM|second|afternoon|| +58206|AAAAAAAAPFDOAAAA|58206|16|10|6|PM|second|afternoon|| +58207|AAAAAAAAAGDOAAAA|58207|16|10|7|PM|second|afternoon|| +58208|AAAAAAAABGDOAAAA|58208|16|10|8|PM|second|afternoon|| +58209|AAAAAAAACGDOAAAA|58209|16|10|9|PM|second|afternoon|| +58210|AAAAAAAADGDOAAAA|58210|16|10|10|PM|second|afternoon|| +58211|AAAAAAAAEGDOAAAA|58211|16|10|11|PM|second|afternoon|| +58212|AAAAAAAAFGDOAAAA|58212|16|10|12|PM|second|afternoon|| +58213|AAAAAAAAGGDOAAAA|58213|16|10|13|PM|second|afternoon|| +58214|AAAAAAAAHGDOAAAA|58214|16|10|14|PM|second|afternoon|| +58215|AAAAAAAAIGDOAAAA|58215|16|10|15|PM|second|afternoon|| +58216|AAAAAAAAJGDOAAAA|58216|16|10|16|PM|second|afternoon|| +58217|AAAAAAAAKGDOAAAA|58217|16|10|17|PM|second|afternoon|| +58218|AAAAAAAALGDOAAAA|58218|16|10|18|PM|second|afternoon|| +58219|AAAAAAAAMGDOAAAA|58219|16|10|19|PM|second|afternoon|| +58220|AAAAAAAANGDOAAAA|58220|16|10|20|PM|second|afternoon|| +58221|AAAAAAAAOGDOAAAA|58221|16|10|21|PM|second|afternoon|| +58222|AAAAAAAAPGDOAAAA|58222|16|10|22|PM|second|afternoon|| +58223|AAAAAAAAAHDOAAAA|58223|16|10|23|PM|second|afternoon|| +58224|AAAAAAAABHDOAAAA|58224|16|10|24|PM|second|afternoon|| +58225|AAAAAAAACHDOAAAA|58225|16|10|25|PM|second|afternoon|| +58226|AAAAAAAADHDOAAAA|58226|16|10|26|PM|second|afternoon|| +58227|AAAAAAAAEHDOAAAA|58227|16|10|27|PM|second|afternoon|| +58228|AAAAAAAAFHDOAAAA|58228|16|10|28|PM|second|afternoon|| +58229|AAAAAAAAGHDOAAAA|58229|16|10|29|PM|second|afternoon|| +58230|AAAAAAAAHHDOAAAA|58230|16|10|30|PM|second|afternoon|| +58231|AAAAAAAAIHDOAAAA|58231|16|10|31|PM|second|afternoon|| +58232|AAAAAAAAJHDOAAAA|58232|16|10|32|PM|second|afternoon|| +58233|AAAAAAAAKHDOAAAA|58233|16|10|33|PM|second|afternoon|| +58234|AAAAAAAALHDOAAAA|58234|16|10|34|PM|second|afternoon|| +58235|AAAAAAAAMHDOAAAA|58235|16|10|35|PM|second|afternoon|| +58236|AAAAAAAANHDOAAAA|58236|16|10|36|PM|second|afternoon|| +58237|AAAAAAAAOHDOAAAA|58237|16|10|37|PM|second|afternoon|| +58238|AAAAAAAAPHDOAAAA|58238|16|10|38|PM|second|afternoon|| +58239|AAAAAAAAAIDOAAAA|58239|16|10|39|PM|second|afternoon|| +58240|AAAAAAAABIDOAAAA|58240|16|10|40|PM|second|afternoon|| +58241|AAAAAAAACIDOAAAA|58241|16|10|41|PM|second|afternoon|| +58242|AAAAAAAADIDOAAAA|58242|16|10|42|PM|second|afternoon|| +58243|AAAAAAAAEIDOAAAA|58243|16|10|43|PM|second|afternoon|| +58244|AAAAAAAAFIDOAAAA|58244|16|10|44|PM|second|afternoon|| +58245|AAAAAAAAGIDOAAAA|58245|16|10|45|PM|second|afternoon|| +58246|AAAAAAAAHIDOAAAA|58246|16|10|46|PM|second|afternoon|| +58247|AAAAAAAAIIDOAAAA|58247|16|10|47|PM|second|afternoon|| +58248|AAAAAAAAJIDOAAAA|58248|16|10|48|PM|second|afternoon|| +58249|AAAAAAAAKIDOAAAA|58249|16|10|49|PM|second|afternoon|| +58250|AAAAAAAALIDOAAAA|58250|16|10|50|PM|second|afternoon|| +58251|AAAAAAAAMIDOAAAA|58251|16|10|51|PM|second|afternoon|| +58252|AAAAAAAANIDOAAAA|58252|16|10|52|PM|second|afternoon|| +58253|AAAAAAAAOIDOAAAA|58253|16|10|53|PM|second|afternoon|| +58254|AAAAAAAAPIDOAAAA|58254|16|10|54|PM|second|afternoon|| +58255|AAAAAAAAAJDOAAAA|58255|16|10|55|PM|second|afternoon|| +58256|AAAAAAAABJDOAAAA|58256|16|10|56|PM|second|afternoon|| +58257|AAAAAAAACJDOAAAA|58257|16|10|57|PM|second|afternoon|| +58258|AAAAAAAADJDOAAAA|58258|16|10|58|PM|second|afternoon|| +58259|AAAAAAAAEJDOAAAA|58259|16|10|59|PM|second|afternoon|| +58260|AAAAAAAAFJDOAAAA|58260|16|11|0|PM|second|afternoon|| +58261|AAAAAAAAGJDOAAAA|58261|16|11|1|PM|second|afternoon|| +58262|AAAAAAAAHJDOAAAA|58262|16|11|2|PM|second|afternoon|| +58263|AAAAAAAAIJDOAAAA|58263|16|11|3|PM|second|afternoon|| +58264|AAAAAAAAJJDOAAAA|58264|16|11|4|PM|second|afternoon|| +58265|AAAAAAAAKJDOAAAA|58265|16|11|5|PM|second|afternoon|| +58266|AAAAAAAALJDOAAAA|58266|16|11|6|PM|second|afternoon|| +58267|AAAAAAAAMJDOAAAA|58267|16|11|7|PM|second|afternoon|| +58268|AAAAAAAANJDOAAAA|58268|16|11|8|PM|second|afternoon|| +58269|AAAAAAAAOJDOAAAA|58269|16|11|9|PM|second|afternoon|| +58270|AAAAAAAAPJDOAAAA|58270|16|11|10|PM|second|afternoon|| +58271|AAAAAAAAAKDOAAAA|58271|16|11|11|PM|second|afternoon|| +58272|AAAAAAAABKDOAAAA|58272|16|11|12|PM|second|afternoon|| +58273|AAAAAAAACKDOAAAA|58273|16|11|13|PM|second|afternoon|| +58274|AAAAAAAADKDOAAAA|58274|16|11|14|PM|second|afternoon|| +58275|AAAAAAAAEKDOAAAA|58275|16|11|15|PM|second|afternoon|| +58276|AAAAAAAAFKDOAAAA|58276|16|11|16|PM|second|afternoon|| +58277|AAAAAAAAGKDOAAAA|58277|16|11|17|PM|second|afternoon|| +58278|AAAAAAAAHKDOAAAA|58278|16|11|18|PM|second|afternoon|| +58279|AAAAAAAAIKDOAAAA|58279|16|11|19|PM|second|afternoon|| +58280|AAAAAAAAJKDOAAAA|58280|16|11|20|PM|second|afternoon|| +58281|AAAAAAAAKKDOAAAA|58281|16|11|21|PM|second|afternoon|| +58282|AAAAAAAALKDOAAAA|58282|16|11|22|PM|second|afternoon|| +58283|AAAAAAAAMKDOAAAA|58283|16|11|23|PM|second|afternoon|| +58284|AAAAAAAANKDOAAAA|58284|16|11|24|PM|second|afternoon|| +58285|AAAAAAAAOKDOAAAA|58285|16|11|25|PM|second|afternoon|| +58286|AAAAAAAAPKDOAAAA|58286|16|11|26|PM|second|afternoon|| +58287|AAAAAAAAALDOAAAA|58287|16|11|27|PM|second|afternoon|| +58288|AAAAAAAABLDOAAAA|58288|16|11|28|PM|second|afternoon|| +58289|AAAAAAAACLDOAAAA|58289|16|11|29|PM|second|afternoon|| +58290|AAAAAAAADLDOAAAA|58290|16|11|30|PM|second|afternoon|| +58291|AAAAAAAAELDOAAAA|58291|16|11|31|PM|second|afternoon|| +58292|AAAAAAAAFLDOAAAA|58292|16|11|32|PM|second|afternoon|| +58293|AAAAAAAAGLDOAAAA|58293|16|11|33|PM|second|afternoon|| +58294|AAAAAAAAHLDOAAAA|58294|16|11|34|PM|second|afternoon|| +58295|AAAAAAAAILDOAAAA|58295|16|11|35|PM|second|afternoon|| +58296|AAAAAAAAJLDOAAAA|58296|16|11|36|PM|second|afternoon|| +58297|AAAAAAAAKLDOAAAA|58297|16|11|37|PM|second|afternoon|| +58298|AAAAAAAALLDOAAAA|58298|16|11|38|PM|second|afternoon|| +58299|AAAAAAAAMLDOAAAA|58299|16|11|39|PM|second|afternoon|| +58300|AAAAAAAANLDOAAAA|58300|16|11|40|PM|second|afternoon|| +58301|AAAAAAAAOLDOAAAA|58301|16|11|41|PM|second|afternoon|| +58302|AAAAAAAAPLDOAAAA|58302|16|11|42|PM|second|afternoon|| +58303|AAAAAAAAAMDOAAAA|58303|16|11|43|PM|second|afternoon|| +58304|AAAAAAAABMDOAAAA|58304|16|11|44|PM|second|afternoon|| +58305|AAAAAAAACMDOAAAA|58305|16|11|45|PM|second|afternoon|| +58306|AAAAAAAADMDOAAAA|58306|16|11|46|PM|second|afternoon|| +58307|AAAAAAAAEMDOAAAA|58307|16|11|47|PM|second|afternoon|| +58308|AAAAAAAAFMDOAAAA|58308|16|11|48|PM|second|afternoon|| +58309|AAAAAAAAGMDOAAAA|58309|16|11|49|PM|second|afternoon|| +58310|AAAAAAAAHMDOAAAA|58310|16|11|50|PM|second|afternoon|| +58311|AAAAAAAAIMDOAAAA|58311|16|11|51|PM|second|afternoon|| +58312|AAAAAAAAJMDOAAAA|58312|16|11|52|PM|second|afternoon|| +58313|AAAAAAAAKMDOAAAA|58313|16|11|53|PM|second|afternoon|| +58314|AAAAAAAALMDOAAAA|58314|16|11|54|PM|second|afternoon|| +58315|AAAAAAAAMMDOAAAA|58315|16|11|55|PM|second|afternoon|| +58316|AAAAAAAANMDOAAAA|58316|16|11|56|PM|second|afternoon|| +58317|AAAAAAAAOMDOAAAA|58317|16|11|57|PM|second|afternoon|| +58318|AAAAAAAAPMDOAAAA|58318|16|11|58|PM|second|afternoon|| +58319|AAAAAAAAANDOAAAA|58319|16|11|59|PM|second|afternoon|| +58320|AAAAAAAABNDOAAAA|58320|16|12|0|PM|second|afternoon|| +58321|AAAAAAAACNDOAAAA|58321|16|12|1|PM|second|afternoon|| +58322|AAAAAAAADNDOAAAA|58322|16|12|2|PM|second|afternoon|| +58323|AAAAAAAAENDOAAAA|58323|16|12|3|PM|second|afternoon|| +58324|AAAAAAAAFNDOAAAA|58324|16|12|4|PM|second|afternoon|| +58325|AAAAAAAAGNDOAAAA|58325|16|12|5|PM|second|afternoon|| +58326|AAAAAAAAHNDOAAAA|58326|16|12|6|PM|second|afternoon|| +58327|AAAAAAAAINDOAAAA|58327|16|12|7|PM|second|afternoon|| +58328|AAAAAAAAJNDOAAAA|58328|16|12|8|PM|second|afternoon|| +58329|AAAAAAAAKNDOAAAA|58329|16|12|9|PM|second|afternoon|| +58330|AAAAAAAALNDOAAAA|58330|16|12|10|PM|second|afternoon|| +58331|AAAAAAAAMNDOAAAA|58331|16|12|11|PM|second|afternoon|| +58332|AAAAAAAANNDOAAAA|58332|16|12|12|PM|second|afternoon|| +58333|AAAAAAAAONDOAAAA|58333|16|12|13|PM|second|afternoon|| +58334|AAAAAAAAPNDOAAAA|58334|16|12|14|PM|second|afternoon|| +58335|AAAAAAAAAODOAAAA|58335|16|12|15|PM|second|afternoon|| +58336|AAAAAAAABODOAAAA|58336|16|12|16|PM|second|afternoon|| +58337|AAAAAAAACODOAAAA|58337|16|12|17|PM|second|afternoon|| +58338|AAAAAAAADODOAAAA|58338|16|12|18|PM|second|afternoon|| +58339|AAAAAAAAEODOAAAA|58339|16|12|19|PM|second|afternoon|| +58340|AAAAAAAAFODOAAAA|58340|16|12|20|PM|second|afternoon|| +58341|AAAAAAAAGODOAAAA|58341|16|12|21|PM|second|afternoon|| +58342|AAAAAAAAHODOAAAA|58342|16|12|22|PM|second|afternoon|| +58343|AAAAAAAAIODOAAAA|58343|16|12|23|PM|second|afternoon|| +58344|AAAAAAAAJODOAAAA|58344|16|12|24|PM|second|afternoon|| +58345|AAAAAAAAKODOAAAA|58345|16|12|25|PM|second|afternoon|| +58346|AAAAAAAALODOAAAA|58346|16|12|26|PM|second|afternoon|| +58347|AAAAAAAAMODOAAAA|58347|16|12|27|PM|second|afternoon|| +58348|AAAAAAAANODOAAAA|58348|16|12|28|PM|second|afternoon|| +58349|AAAAAAAAOODOAAAA|58349|16|12|29|PM|second|afternoon|| +58350|AAAAAAAAPODOAAAA|58350|16|12|30|PM|second|afternoon|| +58351|AAAAAAAAAPDOAAAA|58351|16|12|31|PM|second|afternoon|| +58352|AAAAAAAABPDOAAAA|58352|16|12|32|PM|second|afternoon|| +58353|AAAAAAAACPDOAAAA|58353|16|12|33|PM|second|afternoon|| +58354|AAAAAAAADPDOAAAA|58354|16|12|34|PM|second|afternoon|| +58355|AAAAAAAAEPDOAAAA|58355|16|12|35|PM|second|afternoon|| +58356|AAAAAAAAFPDOAAAA|58356|16|12|36|PM|second|afternoon|| +58357|AAAAAAAAGPDOAAAA|58357|16|12|37|PM|second|afternoon|| +58358|AAAAAAAAHPDOAAAA|58358|16|12|38|PM|second|afternoon|| +58359|AAAAAAAAIPDOAAAA|58359|16|12|39|PM|second|afternoon|| +58360|AAAAAAAAJPDOAAAA|58360|16|12|40|PM|second|afternoon|| +58361|AAAAAAAAKPDOAAAA|58361|16|12|41|PM|second|afternoon|| +58362|AAAAAAAALPDOAAAA|58362|16|12|42|PM|second|afternoon|| +58363|AAAAAAAAMPDOAAAA|58363|16|12|43|PM|second|afternoon|| +58364|AAAAAAAANPDOAAAA|58364|16|12|44|PM|second|afternoon|| +58365|AAAAAAAAOPDOAAAA|58365|16|12|45|PM|second|afternoon|| +58366|AAAAAAAAPPDOAAAA|58366|16|12|46|PM|second|afternoon|| +58367|AAAAAAAAAAEOAAAA|58367|16|12|47|PM|second|afternoon|| +58368|AAAAAAAABAEOAAAA|58368|16|12|48|PM|second|afternoon|| +58369|AAAAAAAACAEOAAAA|58369|16|12|49|PM|second|afternoon|| +58370|AAAAAAAADAEOAAAA|58370|16|12|50|PM|second|afternoon|| +58371|AAAAAAAAEAEOAAAA|58371|16|12|51|PM|second|afternoon|| +58372|AAAAAAAAFAEOAAAA|58372|16|12|52|PM|second|afternoon|| +58373|AAAAAAAAGAEOAAAA|58373|16|12|53|PM|second|afternoon|| +58374|AAAAAAAAHAEOAAAA|58374|16|12|54|PM|second|afternoon|| +58375|AAAAAAAAIAEOAAAA|58375|16|12|55|PM|second|afternoon|| +58376|AAAAAAAAJAEOAAAA|58376|16|12|56|PM|second|afternoon|| +58377|AAAAAAAAKAEOAAAA|58377|16|12|57|PM|second|afternoon|| +58378|AAAAAAAALAEOAAAA|58378|16|12|58|PM|second|afternoon|| +58379|AAAAAAAAMAEOAAAA|58379|16|12|59|PM|second|afternoon|| +58380|AAAAAAAANAEOAAAA|58380|16|13|0|PM|second|afternoon|| +58381|AAAAAAAAOAEOAAAA|58381|16|13|1|PM|second|afternoon|| +58382|AAAAAAAAPAEOAAAA|58382|16|13|2|PM|second|afternoon|| +58383|AAAAAAAAABEOAAAA|58383|16|13|3|PM|second|afternoon|| +58384|AAAAAAAABBEOAAAA|58384|16|13|4|PM|second|afternoon|| +58385|AAAAAAAACBEOAAAA|58385|16|13|5|PM|second|afternoon|| +58386|AAAAAAAADBEOAAAA|58386|16|13|6|PM|second|afternoon|| +58387|AAAAAAAAEBEOAAAA|58387|16|13|7|PM|second|afternoon|| +58388|AAAAAAAAFBEOAAAA|58388|16|13|8|PM|second|afternoon|| +58389|AAAAAAAAGBEOAAAA|58389|16|13|9|PM|second|afternoon|| +58390|AAAAAAAAHBEOAAAA|58390|16|13|10|PM|second|afternoon|| +58391|AAAAAAAAIBEOAAAA|58391|16|13|11|PM|second|afternoon|| +58392|AAAAAAAAJBEOAAAA|58392|16|13|12|PM|second|afternoon|| +58393|AAAAAAAAKBEOAAAA|58393|16|13|13|PM|second|afternoon|| +58394|AAAAAAAALBEOAAAA|58394|16|13|14|PM|second|afternoon|| +58395|AAAAAAAAMBEOAAAA|58395|16|13|15|PM|second|afternoon|| +58396|AAAAAAAANBEOAAAA|58396|16|13|16|PM|second|afternoon|| +58397|AAAAAAAAOBEOAAAA|58397|16|13|17|PM|second|afternoon|| +58398|AAAAAAAAPBEOAAAA|58398|16|13|18|PM|second|afternoon|| +58399|AAAAAAAAACEOAAAA|58399|16|13|19|PM|second|afternoon|| +58400|AAAAAAAABCEOAAAA|58400|16|13|20|PM|second|afternoon|| +58401|AAAAAAAACCEOAAAA|58401|16|13|21|PM|second|afternoon|| +58402|AAAAAAAADCEOAAAA|58402|16|13|22|PM|second|afternoon|| +58403|AAAAAAAAECEOAAAA|58403|16|13|23|PM|second|afternoon|| +58404|AAAAAAAAFCEOAAAA|58404|16|13|24|PM|second|afternoon|| +58405|AAAAAAAAGCEOAAAA|58405|16|13|25|PM|second|afternoon|| +58406|AAAAAAAAHCEOAAAA|58406|16|13|26|PM|second|afternoon|| +58407|AAAAAAAAICEOAAAA|58407|16|13|27|PM|second|afternoon|| +58408|AAAAAAAAJCEOAAAA|58408|16|13|28|PM|second|afternoon|| +58409|AAAAAAAAKCEOAAAA|58409|16|13|29|PM|second|afternoon|| +58410|AAAAAAAALCEOAAAA|58410|16|13|30|PM|second|afternoon|| +58411|AAAAAAAAMCEOAAAA|58411|16|13|31|PM|second|afternoon|| +58412|AAAAAAAANCEOAAAA|58412|16|13|32|PM|second|afternoon|| +58413|AAAAAAAAOCEOAAAA|58413|16|13|33|PM|second|afternoon|| +58414|AAAAAAAAPCEOAAAA|58414|16|13|34|PM|second|afternoon|| +58415|AAAAAAAAADEOAAAA|58415|16|13|35|PM|second|afternoon|| +58416|AAAAAAAABDEOAAAA|58416|16|13|36|PM|second|afternoon|| +58417|AAAAAAAACDEOAAAA|58417|16|13|37|PM|second|afternoon|| +58418|AAAAAAAADDEOAAAA|58418|16|13|38|PM|second|afternoon|| +58419|AAAAAAAAEDEOAAAA|58419|16|13|39|PM|second|afternoon|| +58420|AAAAAAAAFDEOAAAA|58420|16|13|40|PM|second|afternoon|| +58421|AAAAAAAAGDEOAAAA|58421|16|13|41|PM|second|afternoon|| +58422|AAAAAAAAHDEOAAAA|58422|16|13|42|PM|second|afternoon|| +58423|AAAAAAAAIDEOAAAA|58423|16|13|43|PM|second|afternoon|| +58424|AAAAAAAAJDEOAAAA|58424|16|13|44|PM|second|afternoon|| +58425|AAAAAAAAKDEOAAAA|58425|16|13|45|PM|second|afternoon|| +58426|AAAAAAAALDEOAAAA|58426|16|13|46|PM|second|afternoon|| +58427|AAAAAAAAMDEOAAAA|58427|16|13|47|PM|second|afternoon|| +58428|AAAAAAAANDEOAAAA|58428|16|13|48|PM|second|afternoon|| +58429|AAAAAAAAODEOAAAA|58429|16|13|49|PM|second|afternoon|| +58430|AAAAAAAAPDEOAAAA|58430|16|13|50|PM|second|afternoon|| +58431|AAAAAAAAAEEOAAAA|58431|16|13|51|PM|second|afternoon|| +58432|AAAAAAAABEEOAAAA|58432|16|13|52|PM|second|afternoon|| +58433|AAAAAAAACEEOAAAA|58433|16|13|53|PM|second|afternoon|| +58434|AAAAAAAADEEOAAAA|58434|16|13|54|PM|second|afternoon|| +58435|AAAAAAAAEEEOAAAA|58435|16|13|55|PM|second|afternoon|| +58436|AAAAAAAAFEEOAAAA|58436|16|13|56|PM|second|afternoon|| +58437|AAAAAAAAGEEOAAAA|58437|16|13|57|PM|second|afternoon|| +58438|AAAAAAAAHEEOAAAA|58438|16|13|58|PM|second|afternoon|| +58439|AAAAAAAAIEEOAAAA|58439|16|13|59|PM|second|afternoon|| +58440|AAAAAAAAJEEOAAAA|58440|16|14|0|PM|second|afternoon|| +58441|AAAAAAAAKEEOAAAA|58441|16|14|1|PM|second|afternoon|| +58442|AAAAAAAALEEOAAAA|58442|16|14|2|PM|second|afternoon|| +58443|AAAAAAAAMEEOAAAA|58443|16|14|3|PM|second|afternoon|| +58444|AAAAAAAANEEOAAAA|58444|16|14|4|PM|second|afternoon|| +58445|AAAAAAAAOEEOAAAA|58445|16|14|5|PM|second|afternoon|| +58446|AAAAAAAAPEEOAAAA|58446|16|14|6|PM|second|afternoon|| +58447|AAAAAAAAAFEOAAAA|58447|16|14|7|PM|second|afternoon|| +58448|AAAAAAAABFEOAAAA|58448|16|14|8|PM|second|afternoon|| +58449|AAAAAAAACFEOAAAA|58449|16|14|9|PM|second|afternoon|| +58450|AAAAAAAADFEOAAAA|58450|16|14|10|PM|second|afternoon|| +58451|AAAAAAAAEFEOAAAA|58451|16|14|11|PM|second|afternoon|| +58452|AAAAAAAAFFEOAAAA|58452|16|14|12|PM|second|afternoon|| +58453|AAAAAAAAGFEOAAAA|58453|16|14|13|PM|second|afternoon|| +58454|AAAAAAAAHFEOAAAA|58454|16|14|14|PM|second|afternoon|| +58455|AAAAAAAAIFEOAAAA|58455|16|14|15|PM|second|afternoon|| +58456|AAAAAAAAJFEOAAAA|58456|16|14|16|PM|second|afternoon|| +58457|AAAAAAAAKFEOAAAA|58457|16|14|17|PM|second|afternoon|| +58458|AAAAAAAALFEOAAAA|58458|16|14|18|PM|second|afternoon|| +58459|AAAAAAAAMFEOAAAA|58459|16|14|19|PM|second|afternoon|| +58460|AAAAAAAANFEOAAAA|58460|16|14|20|PM|second|afternoon|| +58461|AAAAAAAAOFEOAAAA|58461|16|14|21|PM|second|afternoon|| +58462|AAAAAAAAPFEOAAAA|58462|16|14|22|PM|second|afternoon|| +58463|AAAAAAAAAGEOAAAA|58463|16|14|23|PM|second|afternoon|| +58464|AAAAAAAABGEOAAAA|58464|16|14|24|PM|second|afternoon|| +58465|AAAAAAAACGEOAAAA|58465|16|14|25|PM|second|afternoon|| +58466|AAAAAAAADGEOAAAA|58466|16|14|26|PM|second|afternoon|| +58467|AAAAAAAAEGEOAAAA|58467|16|14|27|PM|second|afternoon|| +58468|AAAAAAAAFGEOAAAA|58468|16|14|28|PM|second|afternoon|| +58469|AAAAAAAAGGEOAAAA|58469|16|14|29|PM|second|afternoon|| +58470|AAAAAAAAHGEOAAAA|58470|16|14|30|PM|second|afternoon|| +58471|AAAAAAAAIGEOAAAA|58471|16|14|31|PM|second|afternoon|| +58472|AAAAAAAAJGEOAAAA|58472|16|14|32|PM|second|afternoon|| +58473|AAAAAAAAKGEOAAAA|58473|16|14|33|PM|second|afternoon|| +58474|AAAAAAAALGEOAAAA|58474|16|14|34|PM|second|afternoon|| +58475|AAAAAAAAMGEOAAAA|58475|16|14|35|PM|second|afternoon|| +58476|AAAAAAAANGEOAAAA|58476|16|14|36|PM|second|afternoon|| +58477|AAAAAAAAOGEOAAAA|58477|16|14|37|PM|second|afternoon|| +58478|AAAAAAAAPGEOAAAA|58478|16|14|38|PM|second|afternoon|| +58479|AAAAAAAAAHEOAAAA|58479|16|14|39|PM|second|afternoon|| +58480|AAAAAAAABHEOAAAA|58480|16|14|40|PM|second|afternoon|| +58481|AAAAAAAACHEOAAAA|58481|16|14|41|PM|second|afternoon|| +58482|AAAAAAAADHEOAAAA|58482|16|14|42|PM|second|afternoon|| +58483|AAAAAAAAEHEOAAAA|58483|16|14|43|PM|second|afternoon|| +58484|AAAAAAAAFHEOAAAA|58484|16|14|44|PM|second|afternoon|| +58485|AAAAAAAAGHEOAAAA|58485|16|14|45|PM|second|afternoon|| +58486|AAAAAAAAHHEOAAAA|58486|16|14|46|PM|second|afternoon|| +58487|AAAAAAAAIHEOAAAA|58487|16|14|47|PM|second|afternoon|| +58488|AAAAAAAAJHEOAAAA|58488|16|14|48|PM|second|afternoon|| +58489|AAAAAAAAKHEOAAAA|58489|16|14|49|PM|second|afternoon|| +58490|AAAAAAAALHEOAAAA|58490|16|14|50|PM|second|afternoon|| +58491|AAAAAAAAMHEOAAAA|58491|16|14|51|PM|second|afternoon|| +58492|AAAAAAAANHEOAAAA|58492|16|14|52|PM|second|afternoon|| +58493|AAAAAAAAOHEOAAAA|58493|16|14|53|PM|second|afternoon|| +58494|AAAAAAAAPHEOAAAA|58494|16|14|54|PM|second|afternoon|| +58495|AAAAAAAAAIEOAAAA|58495|16|14|55|PM|second|afternoon|| +58496|AAAAAAAABIEOAAAA|58496|16|14|56|PM|second|afternoon|| +58497|AAAAAAAACIEOAAAA|58497|16|14|57|PM|second|afternoon|| +58498|AAAAAAAADIEOAAAA|58498|16|14|58|PM|second|afternoon|| +58499|AAAAAAAAEIEOAAAA|58499|16|14|59|PM|second|afternoon|| +58500|AAAAAAAAFIEOAAAA|58500|16|15|0|PM|second|afternoon|| +58501|AAAAAAAAGIEOAAAA|58501|16|15|1|PM|second|afternoon|| +58502|AAAAAAAAHIEOAAAA|58502|16|15|2|PM|second|afternoon|| +58503|AAAAAAAAIIEOAAAA|58503|16|15|3|PM|second|afternoon|| +58504|AAAAAAAAJIEOAAAA|58504|16|15|4|PM|second|afternoon|| +58505|AAAAAAAAKIEOAAAA|58505|16|15|5|PM|second|afternoon|| +58506|AAAAAAAALIEOAAAA|58506|16|15|6|PM|second|afternoon|| +58507|AAAAAAAAMIEOAAAA|58507|16|15|7|PM|second|afternoon|| +58508|AAAAAAAANIEOAAAA|58508|16|15|8|PM|second|afternoon|| +58509|AAAAAAAAOIEOAAAA|58509|16|15|9|PM|second|afternoon|| +58510|AAAAAAAAPIEOAAAA|58510|16|15|10|PM|second|afternoon|| +58511|AAAAAAAAAJEOAAAA|58511|16|15|11|PM|second|afternoon|| +58512|AAAAAAAABJEOAAAA|58512|16|15|12|PM|second|afternoon|| +58513|AAAAAAAACJEOAAAA|58513|16|15|13|PM|second|afternoon|| +58514|AAAAAAAADJEOAAAA|58514|16|15|14|PM|second|afternoon|| +58515|AAAAAAAAEJEOAAAA|58515|16|15|15|PM|second|afternoon|| +58516|AAAAAAAAFJEOAAAA|58516|16|15|16|PM|second|afternoon|| +58517|AAAAAAAAGJEOAAAA|58517|16|15|17|PM|second|afternoon|| +58518|AAAAAAAAHJEOAAAA|58518|16|15|18|PM|second|afternoon|| +58519|AAAAAAAAIJEOAAAA|58519|16|15|19|PM|second|afternoon|| +58520|AAAAAAAAJJEOAAAA|58520|16|15|20|PM|second|afternoon|| +58521|AAAAAAAAKJEOAAAA|58521|16|15|21|PM|second|afternoon|| +58522|AAAAAAAALJEOAAAA|58522|16|15|22|PM|second|afternoon|| +58523|AAAAAAAAMJEOAAAA|58523|16|15|23|PM|second|afternoon|| +58524|AAAAAAAANJEOAAAA|58524|16|15|24|PM|second|afternoon|| +58525|AAAAAAAAOJEOAAAA|58525|16|15|25|PM|second|afternoon|| +58526|AAAAAAAAPJEOAAAA|58526|16|15|26|PM|second|afternoon|| +58527|AAAAAAAAAKEOAAAA|58527|16|15|27|PM|second|afternoon|| +58528|AAAAAAAABKEOAAAA|58528|16|15|28|PM|second|afternoon|| +58529|AAAAAAAACKEOAAAA|58529|16|15|29|PM|second|afternoon|| +58530|AAAAAAAADKEOAAAA|58530|16|15|30|PM|second|afternoon|| +58531|AAAAAAAAEKEOAAAA|58531|16|15|31|PM|second|afternoon|| +58532|AAAAAAAAFKEOAAAA|58532|16|15|32|PM|second|afternoon|| +58533|AAAAAAAAGKEOAAAA|58533|16|15|33|PM|second|afternoon|| +58534|AAAAAAAAHKEOAAAA|58534|16|15|34|PM|second|afternoon|| +58535|AAAAAAAAIKEOAAAA|58535|16|15|35|PM|second|afternoon|| +58536|AAAAAAAAJKEOAAAA|58536|16|15|36|PM|second|afternoon|| +58537|AAAAAAAAKKEOAAAA|58537|16|15|37|PM|second|afternoon|| +58538|AAAAAAAALKEOAAAA|58538|16|15|38|PM|second|afternoon|| +58539|AAAAAAAAMKEOAAAA|58539|16|15|39|PM|second|afternoon|| +58540|AAAAAAAANKEOAAAA|58540|16|15|40|PM|second|afternoon|| +58541|AAAAAAAAOKEOAAAA|58541|16|15|41|PM|second|afternoon|| +58542|AAAAAAAAPKEOAAAA|58542|16|15|42|PM|second|afternoon|| +58543|AAAAAAAAALEOAAAA|58543|16|15|43|PM|second|afternoon|| +58544|AAAAAAAABLEOAAAA|58544|16|15|44|PM|second|afternoon|| +58545|AAAAAAAACLEOAAAA|58545|16|15|45|PM|second|afternoon|| +58546|AAAAAAAADLEOAAAA|58546|16|15|46|PM|second|afternoon|| +58547|AAAAAAAAELEOAAAA|58547|16|15|47|PM|second|afternoon|| +58548|AAAAAAAAFLEOAAAA|58548|16|15|48|PM|second|afternoon|| +58549|AAAAAAAAGLEOAAAA|58549|16|15|49|PM|second|afternoon|| +58550|AAAAAAAAHLEOAAAA|58550|16|15|50|PM|second|afternoon|| +58551|AAAAAAAAILEOAAAA|58551|16|15|51|PM|second|afternoon|| +58552|AAAAAAAAJLEOAAAA|58552|16|15|52|PM|second|afternoon|| +58553|AAAAAAAAKLEOAAAA|58553|16|15|53|PM|second|afternoon|| +58554|AAAAAAAALLEOAAAA|58554|16|15|54|PM|second|afternoon|| +58555|AAAAAAAAMLEOAAAA|58555|16|15|55|PM|second|afternoon|| +58556|AAAAAAAANLEOAAAA|58556|16|15|56|PM|second|afternoon|| +58557|AAAAAAAAOLEOAAAA|58557|16|15|57|PM|second|afternoon|| +58558|AAAAAAAAPLEOAAAA|58558|16|15|58|PM|second|afternoon|| +58559|AAAAAAAAAMEOAAAA|58559|16|15|59|PM|second|afternoon|| +58560|AAAAAAAABMEOAAAA|58560|16|16|0|PM|second|afternoon|| +58561|AAAAAAAACMEOAAAA|58561|16|16|1|PM|second|afternoon|| +58562|AAAAAAAADMEOAAAA|58562|16|16|2|PM|second|afternoon|| +58563|AAAAAAAAEMEOAAAA|58563|16|16|3|PM|second|afternoon|| +58564|AAAAAAAAFMEOAAAA|58564|16|16|4|PM|second|afternoon|| +58565|AAAAAAAAGMEOAAAA|58565|16|16|5|PM|second|afternoon|| +58566|AAAAAAAAHMEOAAAA|58566|16|16|6|PM|second|afternoon|| +58567|AAAAAAAAIMEOAAAA|58567|16|16|7|PM|second|afternoon|| +58568|AAAAAAAAJMEOAAAA|58568|16|16|8|PM|second|afternoon|| +58569|AAAAAAAAKMEOAAAA|58569|16|16|9|PM|second|afternoon|| +58570|AAAAAAAALMEOAAAA|58570|16|16|10|PM|second|afternoon|| +58571|AAAAAAAAMMEOAAAA|58571|16|16|11|PM|second|afternoon|| +58572|AAAAAAAANMEOAAAA|58572|16|16|12|PM|second|afternoon|| +58573|AAAAAAAAOMEOAAAA|58573|16|16|13|PM|second|afternoon|| +58574|AAAAAAAAPMEOAAAA|58574|16|16|14|PM|second|afternoon|| +58575|AAAAAAAAANEOAAAA|58575|16|16|15|PM|second|afternoon|| +58576|AAAAAAAABNEOAAAA|58576|16|16|16|PM|second|afternoon|| +58577|AAAAAAAACNEOAAAA|58577|16|16|17|PM|second|afternoon|| +58578|AAAAAAAADNEOAAAA|58578|16|16|18|PM|second|afternoon|| +58579|AAAAAAAAENEOAAAA|58579|16|16|19|PM|second|afternoon|| +58580|AAAAAAAAFNEOAAAA|58580|16|16|20|PM|second|afternoon|| +58581|AAAAAAAAGNEOAAAA|58581|16|16|21|PM|second|afternoon|| +58582|AAAAAAAAHNEOAAAA|58582|16|16|22|PM|second|afternoon|| +58583|AAAAAAAAINEOAAAA|58583|16|16|23|PM|second|afternoon|| +58584|AAAAAAAAJNEOAAAA|58584|16|16|24|PM|second|afternoon|| +58585|AAAAAAAAKNEOAAAA|58585|16|16|25|PM|second|afternoon|| +58586|AAAAAAAALNEOAAAA|58586|16|16|26|PM|second|afternoon|| +58587|AAAAAAAAMNEOAAAA|58587|16|16|27|PM|second|afternoon|| +58588|AAAAAAAANNEOAAAA|58588|16|16|28|PM|second|afternoon|| +58589|AAAAAAAAONEOAAAA|58589|16|16|29|PM|second|afternoon|| +58590|AAAAAAAAPNEOAAAA|58590|16|16|30|PM|second|afternoon|| +58591|AAAAAAAAAOEOAAAA|58591|16|16|31|PM|second|afternoon|| +58592|AAAAAAAABOEOAAAA|58592|16|16|32|PM|second|afternoon|| +58593|AAAAAAAACOEOAAAA|58593|16|16|33|PM|second|afternoon|| +58594|AAAAAAAADOEOAAAA|58594|16|16|34|PM|second|afternoon|| +58595|AAAAAAAAEOEOAAAA|58595|16|16|35|PM|second|afternoon|| +58596|AAAAAAAAFOEOAAAA|58596|16|16|36|PM|second|afternoon|| +58597|AAAAAAAAGOEOAAAA|58597|16|16|37|PM|second|afternoon|| +58598|AAAAAAAAHOEOAAAA|58598|16|16|38|PM|second|afternoon|| +58599|AAAAAAAAIOEOAAAA|58599|16|16|39|PM|second|afternoon|| +58600|AAAAAAAAJOEOAAAA|58600|16|16|40|PM|second|afternoon|| +58601|AAAAAAAAKOEOAAAA|58601|16|16|41|PM|second|afternoon|| +58602|AAAAAAAALOEOAAAA|58602|16|16|42|PM|second|afternoon|| +58603|AAAAAAAAMOEOAAAA|58603|16|16|43|PM|second|afternoon|| +58604|AAAAAAAANOEOAAAA|58604|16|16|44|PM|second|afternoon|| +58605|AAAAAAAAOOEOAAAA|58605|16|16|45|PM|second|afternoon|| +58606|AAAAAAAAPOEOAAAA|58606|16|16|46|PM|second|afternoon|| +58607|AAAAAAAAAPEOAAAA|58607|16|16|47|PM|second|afternoon|| +58608|AAAAAAAABPEOAAAA|58608|16|16|48|PM|second|afternoon|| +58609|AAAAAAAACPEOAAAA|58609|16|16|49|PM|second|afternoon|| +58610|AAAAAAAADPEOAAAA|58610|16|16|50|PM|second|afternoon|| +58611|AAAAAAAAEPEOAAAA|58611|16|16|51|PM|second|afternoon|| +58612|AAAAAAAAFPEOAAAA|58612|16|16|52|PM|second|afternoon|| +58613|AAAAAAAAGPEOAAAA|58613|16|16|53|PM|second|afternoon|| +58614|AAAAAAAAHPEOAAAA|58614|16|16|54|PM|second|afternoon|| +58615|AAAAAAAAIPEOAAAA|58615|16|16|55|PM|second|afternoon|| +58616|AAAAAAAAJPEOAAAA|58616|16|16|56|PM|second|afternoon|| +58617|AAAAAAAAKPEOAAAA|58617|16|16|57|PM|second|afternoon|| +58618|AAAAAAAALPEOAAAA|58618|16|16|58|PM|second|afternoon|| +58619|AAAAAAAAMPEOAAAA|58619|16|16|59|PM|second|afternoon|| +58620|AAAAAAAANPEOAAAA|58620|16|17|0|PM|second|afternoon|| +58621|AAAAAAAAOPEOAAAA|58621|16|17|1|PM|second|afternoon|| +58622|AAAAAAAAPPEOAAAA|58622|16|17|2|PM|second|afternoon|| +58623|AAAAAAAAAAFOAAAA|58623|16|17|3|PM|second|afternoon|| +58624|AAAAAAAABAFOAAAA|58624|16|17|4|PM|second|afternoon|| +58625|AAAAAAAACAFOAAAA|58625|16|17|5|PM|second|afternoon|| +58626|AAAAAAAADAFOAAAA|58626|16|17|6|PM|second|afternoon|| +58627|AAAAAAAAEAFOAAAA|58627|16|17|7|PM|second|afternoon|| +58628|AAAAAAAAFAFOAAAA|58628|16|17|8|PM|second|afternoon|| +58629|AAAAAAAAGAFOAAAA|58629|16|17|9|PM|second|afternoon|| +58630|AAAAAAAAHAFOAAAA|58630|16|17|10|PM|second|afternoon|| +58631|AAAAAAAAIAFOAAAA|58631|16|17|11|PM|second|afternoon|| +58632|AAAAAAAAJAFOAAAA|58632|16|17|12|PM|second|afternoon|| +58633|AAAAAAAAKAFOAAAA|58633|16|17|13|PM|second|afternoon|| +58634|AAAAAAAALAFOAAAA|58634|16|17|14|PM|second|afternoon|| +58635|AAAAAAAAMAFOAAAA|58635|16|17|15|PM|second|afternoon|| +58636|AAAAAAAANAFOAAAA|58636|16|17|16|PM|second|afternoon|| +58637|AAAAAAAAOAFOAAAA|58637|16|17|17|PM|second|afternoon|| +58638|AAAAAAAAPAFOAAAA|58638|16|17|18|PM|second|afternoon|| +58639|AAAAAAAAABFOAAAA|58639|16|17|19|PM|second|afternoon|| +58640|AAAAAAAABBFOAAAA|58640|16|17|20|PM|second|afternoon|| +58641|AAAAAAAACBFOAAAA|58641|16|17|21|PM|second|afternoon|| +58642|AAAAAAAADBFOAAAA|58642|16|17|22|PM|second|afternoon|| +58643|AAAAAAAAEBFOAAAA|58643|16|17|23|PM|second|afternoon|| +58644|AAAAAAAAFBFOAAAA|58644|16|17|24|PM|second|afternoon|| +58645|AAAAAAAAGBFOAAAA|58645|16|17|25|PM|second|afternoon|| +58646|AAAAAAAAHBFOAAAA|58646|16|17|26|PM|second|afternoon|| +58647|AAAAAAAAIBFOAAAA|58647|16|17|27|PM|second|afternoon|| +58648|AAAAAAAAJBFOAAAA|58648|16|17|28|PM|second|afternoon|| +58649|AAAAAAAAKBFOAAAA|58649|16|17|29|PM|second|afternoon|| +58650|AAAAAAAALBFOAAAA|58650|16|17|30|PM|second|afternoon|| +58651|AAAAAAAAMBFOAAAA|58651|16|17|31|PM|second|afternoon|| +58652|AAAAAAAANBFOAAAA|58652|16|17|32|PM|second|afternoon|| +58653|AAAAAAAAOBFOAAAA|58653|16|17|33|PM|second|afternoon|| +58654|AAAAAAAAPBFOAAAA|58654|16|17|34|PM|second|afternoon|| +58655|AAAAAAAAACFOAAAA|58655|16|17|35|PM|second|afternoon|| +58656|AAAAAAAABCFOAAAA|58656|16|17|36|PM|second|afternoon|| +58657|AAAAAAAACCFOAAAA|58657|16|17|37|PM|second|afternoon|| +58658|AAAAAAAADCFOAAAA|58658|16|17|38|PM|second|afternoon|| +58659|AAAAAAAAECFOAAAA|58659|16|17|39|PM|second|afternoon|| +58660|AAAAAAAAFCFOAAAA|58660|16|17|40|PM|second|afternoon|| +58661|AAAAAAAAGCFOAAAA|58661|16|17|41|PM|second|afternoon|| +58662|AAAAAAAAHCFOAAAA|58662|16|17|42|PM|second|afternoon|| +58663|AAAAAAAAICFOAAAA|58663|16|17|43|PM|second|afternoon|| +58664|AAAAAAAAJCFOAAAA|58664|16|17|44|PM|second|afternoon|| +58665|AAAAAAAAKCFOAAAA|58665|16|17|45|PM|second|afternoon|| +58666|AAAAAAAALCFOAAAA|58666|16|17|46|PM|second|afternoon|| +58667|AAAAAAAAMCFOAAAA|58667|16|17|47|PM|second|afternoon|| +58668|AAAAAAAANCFOAAAA|58668|16|17|48|PM|second|afternoon|| +58669|AAAAAAAAOCFOAAAA|58669|16|17|49|PM|second|afternoon|| +58670|AAAAAAAAPCFOAAAA|58670|16|17|50|PM|second|afternoon|| +58671|AAAAAAAAADFOAAAA|58671|16|17|51|PM|second|afternoon|| +58672|AAAAAAAABDFOAAAA|58672|16|17|52|PM|second|afternoon|| +58673|AAAAAAAACDFOAAAA|58673|16|17|53|PM|second|afternoon|| +58674|AAAAAAAADDFOAAAA|58674|16|17|54|PM|second|afternoon|| +58675|AAAAAAAAEDFOAAAA|58675|16|17|55|PM|second|afternoon|| +58676|AAAAAAAAFDFOAAAA|58676|16|17|56|PM|second|afternoon|| +58677|AAAAAAAAGDFOAAAA|58677|16|17|57|PM|second|afternoon|| +58678|AAAAAAAAHDFOAAAA|58678|16|17|58|PM|second|afternoon|| +58679|AAAAAAAAIDFOAAAA|58679|16|17|59|PM|second|afternoon|| +58680|AAAAAAAAJDFOAAAA|58680|16|18|0|PM|second|afternoon|| +58681|AAAAAAAAKDFOAAAA|58681|16|18|1|PM|second|afternoon|| +58682|AAAAAAAALDFOAAAA|58682|16|18|2|PM|second|afternoon|| +58683|AAAAAAAAMDFOAAAA|58683|16|18|3|PM|second|afternoon|| +58684|AAAAAAAANDFOAAAA|58684|16|18|4|PM|second|afternoon|| +58685|AAAAAAAAODFOAAAA|58685|16|18|5|PM|second|afternoon|| +58686|AAAAAAAAPDFOAAAA|58686|16|18|6|PM|second|afternoon|| +58687|AAAAAAAAAEFOAAAA|58687|16|18|7|PM|second|afternoon|| +58688|AAAAAAAABEFOAAAA|58688|16|18|8|PM|second|afternoon|| +58689|AAAAAAAACEFOAAAA|58689|16|18|9|PM|second|afternoon|| +58690|AAAAAAAADEFOAAAA|58690|16|18|10|PM|second|afternoon|| +58691|AAAAAAAAEEFOAAAA|58691|16|18|11|PM|second|afternoon|| +58692|AAAAAAAAFEFOAAAA|58692|16|18|12|PM|second|afternoon|| +58693|AAAAAAAAGEFOAAAA|58693|16|18|13|PM|second|afternoon|| +58694|AAAAAAAAHEFOAAAA|58694|16|18|14|PM|second|afternoon|| +58695|AAAAAAAAIEFOAAAA|58695|16|18|15|PM|second|afternoon|| +58696|AAAAAAAAJEFOAAAA|58696|16|18|16|PM|second|afternoon|| +58697|AAAAAAAAKEFOAAAA|58697|16|18|17|PM|second|afternoon|| +58698|AAAAAAAALEFOAAAA|58698|16|18|18|PM|second|afternoon|| +58699|AAAAAAAAMEFOAAAA|58699|16|18|19|PM|second|afternoon|| +58700|AAAAAAAANEFOAAAA|58700|16|18|20|PM|second|afternoon|| +58701|AAAAAAAAOEFOAAAA|58701|16|18|21|PM|second|afternoon|| +58702|AAAAAAAAPEFOAAAA|58702|16|18|22|PM|second|afternoon|| +58703|AAAAAAAAAFFOAAAA|58703|16|18|23|PM|second|afternoon|| +58704|AAAAAAAABFFOAAAA|58704|16|18|24|PM|second|afternoon|| +58705|AAAAAAAACFFOAAAA|58705|16|18|25|PM|second|afternoon|| +58706|AAAAAAAADFFOAAAA|58706|16|18|26|PM|second|afternoon|| +58707|AAAAAAAAEFFOAAAA|58707|16|18|27|PM|second|afternoon|| +58708|AAAAAAAAFFFOAAAA|58708|16|18|28|PM|second|afternoon|| +58709|AAAAAAAAGFFOAAAA|58709|16|18|29|PM|second|afternoon|| +58710|AAAAAAAAHFFOAAAA|58710|16|18|30|PM|second|afternoon|| +58711|AAAAAAAAIFFOAAAA|58711|16|18|31|PM|second|afternoon|| +58712|AAAAAAAAJFFOAAAA|58712|16|18|32|PM|second|afternoon|| +58713|AAAAAAAAKFFOAAAA|58713|16|18|33|PM|second|afternoon|| +58714|AAAAAAAALFFOAAAA|58714|16|18|34|PM|second|afternoon|| +58715|AAAAAAAAMFFOAAAA|58715|16|18|35|PM|second|afternoon|| +58716|AAAAAAAANFFOAAAA|58716|16|18|36|PM|second|afternoon|| +58717|AAAAAAAAOFFOAAAA|58717|16|18|37|PM|second|afternoon|| +58718|AAAAAAAAPFFOAAAA|58718|16|18|38|PM|second|afternoon|| +58719|AAAAAAAAAGFOAAAA|58719|16|18|39|PM|second|afternoon|| +58720|AAAAAAAABGFOAAAA|58720|16|18|40|PM|second|afternoon|| +58721|AAAAAAAACGFOAAAA|58721|16|18|41|PM|second|afternoon|| +58722|AAAAAAAADGFOAAAA|58722|16|18|42|PM|second|afternoon|| +58723|AAAAAAAAEGFOAAAA|58723|16|18|43|PM|second|afternoon|| +58724|AAAAAAAAFGFOAAAA|58724|16|18|44|PM|second|afternoon|| +58725|AAAAAAAAGGFOAAAA|58725|16|18|45|PM|second|afternoon|| +58726|AAAAAAAAHGFOAAAA|58726|16|18|46|PM|second|afternoon|| +58727|AAAAAAAAIGFOAAAA|58727|16|18|47|PM|second|afternoon|| +58728|AAAAAAAAJGFOAAAA|58728|16|18|48|PM|second|afternoon|| +58729|AAAAAAAAKGFOAAAA|58729|16|18|49|PM|second|afternoon|| +58730|AAAAAAAALGFOAAAA|58730|16|18|50|PM|second|afternoon|| +58731|AAAAAAAAMGFOAAAA|58731|16|18|51|PM|second|afternoon|| +58732|AAAAAAAANGFOAAAA|58732|16|18|52|PM|second|afternoon|| +58733|AAAAAAAAOGFOAAAA|58733|16|18|53|PM|second|afternoon|| +58734|AAAAAAAAPGFOAAAA|58734|16|18|54|PM|second|afternoon|| +58735|AAAAAAAAAHFOAAAA|58735|16|18|55|PM|second|afternoon|| +58736|AAAAAAAABHFOAAAA|58736|16|18|56|PM|second|afternoon|| +58737|AAAAAAAACHFOAAAA|58737|16|18|57|PM|second|afternoon|| +58738|AAAAAAAADHFOAAAA|58738|16|18|58|PM|second|afternoon|| +58739|AAAAAAAAEHFOAAAA|58739|16|18|59|PM|second|afternoon|| +58740|AAAAAAAAFHFOAAAA|58740|16|19|0|PM|second|afternoon|| +58741|AAAAAAAAGHFOAAAA|58741|16|19|1|PM|second|afternoon|| +58742|AAAAAAAAHHFOAAAA|58742|16|19|2|PM|second|afternoon|| +58743|AAAAAAAAIHFOAAAA|58743|16|19|3|PM|second|afternoon|| +58744|AAAAAAAAJHFOAAAA|58744|16|19|4|PM|second|afternoon|| +58745|AAAAAAAAKHFOAAAA|58745|16|19|5|PM|second|afternoon|| +58746|AAAAAAAALHFOAAAA|58746|16|19|6|PM|second|afternoon|| +58747|AAAAAAAAMHFOAAAA|58747|16|19|7|PM|second|afternoon|| +58748|AAAAAAAANHFOAAAA|58748|16|19|8|PM|second|afternoon|| +58749|AAAAAAAAOHFOAAAA|58749|16|19|9|PM|second|afternoon|| +58750|AAAAAAAAPHFOAAAA|58750|16|19|10|PM|second|afternoon|| +58751|AAAAAAAAAIFOAAAA|58751|16|19|11|PM|second|afternoon|| +58752|AAAAAAAABIFOAAAA|58752|16|19|12|PM|second|afternoon|| +58753|AAAAAAAACIFOAAAA|58753|16|19|13|PM|second|afternoon|| +58754|AAAAAAAADIFOAAAA|58754|16|19|14|PM|second|afternoon|| +58755|AAAAAAAAEIFOAAAA|58755|16|19|15|PM|second|afternoon|| +58756|AAAAAAAAFIFOAAAA|58756|16|19|16|PM|second|afternoon|| +58757|AAAAAAAAGIFOAAAA|58757|16|19|17|PM|second|afternoon|| +58758|AAAAAAAAHIFOAAAA|58758|16|19|18|PM|second|afternoon|| +58759|AAAAAAAAIIFOAAAA|58759|16|19|19|PM|second|afternoon|| +58760|AAAAAAAAJIFOAAAA|58760|16|19|20|PM|second|afternoon|| +58761|AAAAAAAAKIFOAAAA|58761|16|19|21|PM|second|afternoon|| +58762|AAAAAAAALIFOAAAA|58762|16|19|22|PM|second|afternoon|| +58763|AAAAAAAAMIFOAAAA|58763|16|19|23|PM|second|afternoon|| +58764|AAAAAAAANIFOAAAA|58764|16|19|24|PM|second|afternoon|| +58765|AAAAAAAAOIFOAAAA|58765|16|19|25|PM|second|afternoon|| +58766|AAAAAAAAPIFOAAAA|58766|16|19|26|PM|second|afternoon|| +58767|AAAAAAAAAJFOAAAA|58767|16|19|27|PM|second|afternoon|| +58768|AAAAAAAABJFOAAAA|58768|16|19|28|PM|second|afternoon|| +58769|AAAAAAAACJFOAAAA|58769|16|19|29|PM|second|afternoon|| +58770|AAAAAAAADJFOAAAA|58770|16|19|30|PM|second|afternoon|| +58771|AAAAAAAAEJFOAAAA|58771|16|19|31|PM|second|afternoon|| +58772|AAAAAAAAFJFOAAAA|58772|16|19|32|PM|second|afternoon|| +58773|AAAAAAAAGJFOAAAA|58773|16|19|33|PM|second|afternoon|| +58774|AAAAAAAAHJFOAAAA|58774|16|19|34|PM|second|afternoon|| +58775|AAAAAAAAIJFOAAAA|58775|16|19|35|PM|second|afternoon|| +58776|AAAAAAAAJJFOAAAA|58776|16|19|36|PM|second|afternoon|| +58777|AAAAAAAAKJFOAAAA|58777|16|19|37|PM|second|afternoon|| +58778|AAAAAAAALJFOAAAA|58778|16|19|38|PM|second|afternoon|| +58779|AAAAAAAAMJFOAAAA|58779|16|19|39|PM|second|afternoon|| +58780|AAAAAAAANJFOAAAA|58780|16|19|40|PM|second|afternoon|| +58781|AAAAAAAAOJFOAAAA|58781|16|19|41|PM|second|afternoon|| +58782|AAAAAAAAPJFOAAAA|58782|16|19|42|PM|second|afternoon|| +58783|AAAAAAAAAKFOAAAA|58783|16|19|43|PM|second|afternoon|| +58784|AAAAAAAABKFOAAAA|58784|16|19|44|PM|second|afternoon|| +58785|AAAAAAAACKFOAAAA|58785|16|19|45|PM|second|afternoon|| +58786|AAAAAAAADKFOAAAA|58786|16|19|46|PM|second|afternoon|| +58787|AAAAAAAAEKFOAAAA|58787|16|19|47|PM|second|afternoon|| +58788|AAAAAAAAFKFOAAAA|58788|16|19|48|PM|second|afternoon|| +58789|AAAAAAAAGKFOAAAA|58789|16|19|49|PM|second|afternoon|| +58790|AAAAAAAAHKFOAAAA|58790|16|19|50|PM|second|afternoon|| +58791|AAAAAAAAIKFOAAAA|58791|16|19|51|PM|second|afternoon|| +58792|AAAAAAAAJKFOAAAA|58792|16|19|52|PM|second|afternoon|| +58793|AAAAAAAAKKFOAAAA|58793|16|19|53|PM|second|afternoon|| +58794|AAAAAAAALKFOAAAA|58794|16|19|54|PM|second|afternoon|| +58795|AAAAAAAAMKFOAAAA|58795|16|19|55|PM|second|afternoon|| +58796|AAAAAAAANKFOAAAA|58796|16|19|56|PM|second|afternoon|| +58797|AAAAAAAAOKFOAAAA|58797|16|19|57|PM|second|afternoon|| +58798|AAAAAAAAPKFOAAAA|58798|16|19|58|PM|second|afternoon|| +58799|AAAAAAAAALFOAAAA|58799|16|19|59|PM|second|afternoon|| +58800|AAAAAAAABLFOAAAA|58800|16|20|0|PM|second|afternoon|| +58801|AAAAAAAACLFOAAAA|58801|16|20|1|PM|second|afternoon|| +58802|AAAAAAAADLFOAAAA|58802|16|20|2|PM|second|afternoon|| +58803|AAAAAAAAELFOAAAA|58803|16|20|3|PM|second|afternoon|| +58804|AAAAAAAAFLFOAAAA|58804|16|20|4|PM|second|afternoon|| +58805|AAAAAAAAGLFOAAAA|58805|16|20|5|PM|second|afternoon|| +58806|AAAAAAAAHLFOAAAA|58806|16|20|6|PM|second|afternoon|| +58807|AAAAAAAAILFOAAAA|58807|16|20|7|PM|second|afternoon|| +58808|AAAAAAAAJLFOAAAA|58808|16|20|8|PM|second|afternoon|| +58809|AAAAAAAAKLFOAAAA|58809|16|20|9|PM|second|afternoon|| +58810|AAAAAAAALLFOAAAA|58810|16|20|10|PM|second|afternoon|| +58811|AAAAAAAAMLFOAAAA|58811|16|20|11|PM|second|afternoon|| +58812|AAAAAAAANLFOAAAA|58812|16|20|12|PM|second|afternoon|| +58813|AAAAAAAAOLFOAAAA|58813|16|20|13|PM|second|afternoon|| +58814|AAAAAAAAPLFOAAAA|58814|16|20|14|PM|second|afternoon|| +58815|AAAAAAAAAMFOAAAA|58815|16|20|15|PM|second|afternoon|| +58816|AAAAAAAABMFOAAAA|58816|16|20|16|PM|second|afternoon|| +58817|AAAAAAAACMFOAAAA|58817|16|20|17|PM|second|afternoon|| +58818|AAAAAAAADMFOAAAA|58818|16|20|18|PM|second|afternoon|| +58819|AAAAAAAAEMFOAAAA|58819|16|20|19|PM|second|afternoon|| +58820|AAAAAAAAFMFOAAAA|58820|16|20|20|PM|second|afternoon|| +58821|AAAAAAAAGMFOAAAA|58821|16|20|21|PM|second|afternoon|| +58822|AAAAAAAAHMFOAAAA|58822|16|20|22|PM|second|afternoon|| +58823|AAAAAAAAIMFOAAAA|58823|16|20|23|PM|second|afternoon|| +58824|AAAAAAAAJMFOAAAA|58824|16|20|24|PM|second|afternoon|| +58825|AAAAAAAAKMFOAAAA|58825|16|20|25|PM|second|afternoon|| +58826|AAAAAAAALMFOAAAA|58826|16|20|26|PM|second|afternoon|| +58827|AAAAAAAAMMFOAAAA|58827|16|20|27|PM|second|afternoon|| +58828|AAAAAAAANMFOAAAA|58828|16|20|28|PM|second|afternoon|| +58829|AAAAAAAAOMFOAAAA|58829|16|20|29|PM|second|afternoon|| +58830|AAAAAAAAPMFOAAAA|58830|16|20|30|PM|second|afternoon|| +58831|AAAAAAAAANFOAAAA|58831|16|20|31|PM|second|afternoon|| +58832|AAAAAAAABNFOAAAA|58832|16|20|32|PM|second|afternoon|| +58833|AAAAAAAACNFOAAAA|58833|16|20|33|PM|second|afternoon|| +58834|AAAAAAAADNFOAAAA|58834|16|20|34|PM|second|afternoon|| +58835|AAAAAAAAENFOAAAA|58835|16|20|35|PM|second|afternoon|| +58836|AAAAAAAAFNFOAAAA|58836|16|20|36|PM|second|afternoon|| +58837|AAAAAAAAGNFOAAAA|58837|16|20|37|PM|second|afternoon|| +58838|AAAAAAAAHNFOAAAA|58838|16|20|38|PM|second|afternoon|| +58839|AAAAAAAAINFOAAAA|58839|16|20|39|PM|second|afternoon|| +58840|AAAAAAAAJNFOAAAA|58840|16|20|40|PM|second|afternoon|| +58841|AAAAAAAAKNFOAAAA|58841|16|20|41|PM|second|afternoon|| +58842|AAAAAAAALNFOAAAA|58842|16|20|42|PM|second|afternoon|| +58843|AAAAAAAAMNFOAAAA|58843|16|20|43|PM|second|afternoon|| +58844|AAAAAAAANNFOAAAA|58844|16|20|44|PM|second|afternoon|| +58845|AAAAAAAAONFOAAAA|58845|16|20|45|PM|second|afternoon|| +58846|AAAAAAAAPNFOAAAA|58846|16|20|46|PM|second|afternoon|| +58847|AAAAAAAAAOFOAAAA|58847|16|20|47|PM|second|afternoon|| +58848|AAAAAAAABOFOAAAA|58848|16|20|48|PM|second|afternoon|| +58849|AAAAAAAACOFOAAAA|58849|16|20|49|PM|second|afternoon|| +58850|AAAAAAAADOFOAAAA|58850|16|20|50|PM|second|afternoon|| +58851|AAAAAAAAEOFOAAAA|58851|16|20|51|PM|second|afternoon|| +58852|AAAAAAAAFOFOAAAA|58852|16|20|52|PM|second|afternoon|| +58853|AAAAAAAAGOFOAAAA|58853|16|20|53|PM|second|afternoon|| +58854|AAAAAAAAHOFOAAAA|58854|16|20|54|PM|second|afternoon|| +58855|AAAAAAAAIOFOAAAA|58855|16|20|55|PM|second|afternoon|| +58856|AAAAAAAAJOFOAAAA|58856|16|20|56|PM|second|afternoon|| +58857|AAAAAAAAKOFOAAAA|58857|16|20|57|PM|second|afternoon|| +58858|AAAAAAAALOFOAAAA|58858|16|20|58|PM|second|afternoon|| +58859|AAAAAAAAMOFOAAAA|58859|16|20|59|PM|second|afternoon|| +58860|AAAAAAAANOFOAAAA|58860|16|21|0|PM|second|afternoon|| +58861|AAAAAAAAOOFOAAAA|58861|16|21|1|PM|second|afternoon|| +58862|AAAAAAAAPOFOAAAA|58862|16|21|2|PM|second|afternoon|| +58863|AAAAAAAAAPFOAAAA|58863|16|21|3|PM|second|afternoon|| +58864|AAAAAAAABPFOAAAA|58864|16|21|4|PM|second|afternoon|| +58865|AAAAAAAACPFOAAAA|58865|16|21|5|PM|second|afternoon|| +58866|AAAAAAAADPFOAAAA|58866|16|21|6|PM|second|afternoon|| +58867|AAAAAAAAEPFOAAAA|58867|16|21|7|PM|second|afternoon|| +58868|AAAAAAAAFPFOAAAA|58868|16|21|8|PM|second|afternoon|| +58869|AAAAAAAAGPFOAAAA|58869|16|21|9|PM|second|afternoon|| +58870|AAAAAAAAHPFOAAAA|58870|16|21|10|PM|second|afternoon|| +58871|AAAAAAAAIPFOAAAA|58871|16|21|11|PM|second|afternoon|| +58872|AAAAAAAAJPFOAAAA|58872|16|21|12|PM|second|afternoon|| +58873|AAAAAAAAKPFOAAAA|58873|16|21|13|PM|second|afternoon|| +58874|AAAAAAAALPFOAAAA|58874|16|21|14|PM|second|afternoon|| +58875|AAAAAAAAMPFOAAAA|58875|16|21|15|PM|second|afternoon|| +58876|AAAAAAAANPFOAAAA|58876|16|21|16|PM|second|afternoon|| +58877|AAAAAAAAOPFOAAAA|58877|16|21|17|PM|second|afternoon|| +58878|AAAAAAAAPPFOAAAA|58878|16|21|18|PM|second|afternoon|| +58879|AAAAAAAAAAGOAAAA|58879|16|21|19|PM|second|afternoon|| +58880|AAAAAAAABAGOAAAA|58880|16|21|20|PM|second|afternoon|| +58881|AAAAAAAACAGOAAAA|58881|16|21|21|PM|second|afternoon|| +58882|AAAAAAAADAGOAAAA|58882|16|21|22|PM|second|afternoon|| +58883|AAAAAAAAEAGOAAAA|58883|16|21|23|PM|second|afternoon|| +58884|AAAAAAAAFAGOAAAA|58884|16|21|24|PM|second|afternoon|| +58885|AAAAAAAAGAGOAAAA|58885|16|21|25|PM|second|afternoon|| +58886|AAAAAAAAHAGOAAAA|58886|16|21|26|PM|second|afternoon|| +58887|AAAAAAAAIAGOAAAA|58887|16|21|27|PM|second|afternoon|| +58888|AAAAAAAAJAGOAAAA|58888|16|21|28|PM|second|afternoon|| +58889|AAAAAAAAKAGOAAAA|58889|16|21|29|PM|second|afternoon|| +58890|AAAAAAAALAGOAAAA|58890|16|21|30|PM|second|afternoon|| +58891|AAAAAAAAMAGOAAAA|58891|16|21|31|PM|second|afternoon|| +58892|AAAAAAAANAGOAAAA|58892|16|21|32|PM|second|afternoon|| +58893|AAAAAAAAOAGOAAAA|58893|16|21|33|PM|second|afternoon|| +58894|AAAAAAAAPAGOAAAA|58894|16|21|34|PM|second|afternoon|| +58895|AAAAAAAAABGOAAAA|58895|16|21|35|PM|second|afternoon|| +58896|AAAAAAAABBGOAAAA|58896|16|21|36|PM|second|afternoon|| +58897|AAAAAAAACBGOAAAA|58897|16|21|37|PM|second|afternoon|| +58898|AAAAAAAADBGOAAAA|58898|16|21|38|PM|second|afternoon|| +58899|AAAAAAAAEBGOAAAA|58899|16|21|39|PM|second|afternoon|| +58900|AAAAAAAAFBGOAAAA|58900|16|21|40|PM|second|afternoon|| +58901|AAAAAAAAGBGOAAAA|58901|16|21|41|PM|second|afternoon|| +58902|AAAAAAAAHBGOAAAA|58902|16|21|42|PM|second|afternoon|| +58903|AAAAAAAAIBGOAAAA|58903|16|21|43|PM|second|afternoon|| +58904|AAAAAAAAJBGOAAAA|58904|16|21|44|PM|second|afternoon|| +58905|AAAAAAAAKBGOAAAA|58905|16|21|45|PM|second|afternoon|| +58906|AAAAAAAALBGOAAAA|58906|16|21|46|PM|second|afternoon|| +58907|AAAAAAAAMBGOAAAA|58907|16|21|47|PM|second|afternoon|| +58908|AAAAAAAANBGOAAAA|58908|16|21|48|PM|second|afternoon|| +58909|AAAAAAAAOBGOAAAA|58909|16|21|49|PM|second|afternoon|| +58910|AAAAAAAAPBGOAAAA|58910|16|21|50|PM|second|afternoon|| +58911|AAAAAAAAACGOAAAA|58911|16|21|51|PM|second|afternoon|| +58912|AAAAAAAABCGOAAAA|58912|16|21|52|PM|second|afternoon|| +58913|AAAAAAAACCGOAAAA|58913|16|21|53|PM|second|afternoon|| +58914|AAAAAAAADCGOAAAA|58914|16|21|54|PM|second|afternoon|| +58915|AAAAAAAAECGOAAAA|58915|16|21|55|PM|second|afternoon|| +58916|AAAAAAAAFCGOAAAA|58916|16|21|56|PM|second|afternoon|| +58917|AAAAAAAAGCGOAAAA|58917|16|21|57|PM|second|afternoon|| +58918|AAAAAAAAHCGOAAAA|58918|16|21|58|PM|second|afternoon|| +58919|AAAAAAAAICGOAAAA|58919|16|21|59|PM|second|afternoon|| +58920|AAAAAAAAJCGOAAAA|58920|16|22|0|PM|second|afternoon|| +58921|AAAAAAAAKCGOAAAA|58921|16|22|1|PM|second|afternoon|| +58922|AAAAAAAALCGOAAAA|58922|16|22|2|PM|second|afternoon|| +58923|AAAAAAAAMCGOAAAA|58923|16|22|3|PM|second|afternoon|| +58924|AAAAAAAANCGOAAAA|58924|16|22|4|PM|second|afternoon|| +58925|AAAAAAAAOCGOAAAA|58925|16|22|5|PM|second|afternoon|| +58926|AAAAAAAAPCGOAAAA|58926|16|22|6|PM|second|afternoon|| +58927|AAAAAAAAADGOAAAA|58927|16|22|7|PM|second|afternoon|| +58928|AAAAAAAABDGOAAAA|58928|16|22|8|PM|second|afternoon|| +58929|AAAAAAAACDGOAAAA|58929|16|22|9|PM|second|afternoon|| +58930|AAAAAAAADDGOAAAA|58930|16|22|10|PM|second|afternoon|| +58931|AAAAAAAAEDGOAAAA|58931|16|22|11|PM|second|afternoon|| +58932|AAAAAAAAFDGOAAAA|58932|16|22|12|PM|second|afternoon|| +58933|AAAAAAAAGDGOAAAA|58933|16|22|13|PM|second|afternoon|| +58934|AAAAAAAAHDGOAAAA|58934|16|22|14|PM|second|afternoon|| +58935|AAAAAAAAIDGOAAAA|58935|16|22|15|PM|second|afternoon|| +58936|AAAAAAAAJDGOAAAA|58936|16|22|16|PM|second|afternoon|| +58937|AAAAAAAAKDGOAAAA|58937|16|22|17|PM|second|afternoon|| +58938|AAAAAAAALDGOAAAA|58938|16|22|18|PM|second|afternoon|| +58939|AAAAAAAAMDGOAAAA|58939|16|22|19|PM|second|afternoon|| +58940|AAAAAAAANDGOAAAA|58940|16|22|20|PM|second|afternoon|| +58941|AAAAAAAAODGOAAAA|58941|16|22|21|PM|second|afternoon|| +58942|AAAAAAAAPDGOAAAA|58942|16|22|22|PM|second|afternoon|| +58943|AAAAAAAAAEGOAAAA|58943|16|22|23|PM|second|afternoon|| +58944|AAAAAAAABEGOAAAA|58944|16|22|24|PM|second|afternoon|| +58945|AAAAAAAACEGOAAAA|58945|16|22|25|PM|second|afternoon|| +58946|AAAAAAAADEGOAAAA|58946|16|22|26|PM|second|afternoon|| +58947|AAAAAAAAEEGOAAAA|58947|16|22|27|PM|second|afternoon|| +58948|AAAAAAAAFEGOAAAA|58948|16|22|28|PM|second|afternoon|| +58949|AAAAAAAAGEGOAAAA|58949|16|22|29|PM|second|afternoon|| +58950|AAAAAAAAHEGOAAAA|58950|16|22|30|PM|second|afternoon|| +58951|AAAAAAAAIEGOAAAA|58951|16|22|31|PM|second|afternoon|| +58952|AAAAAAAAJEGOAAAA|58952|16|22|32|PM|second|afternoon|| +58953|AAAAAAAAKEGOAAAA|58953|16|22|33|PM|second|afternoon|| +58954|AAAAAAAALEGOAAAA|58954|16|22|34|PM|second|afternoon|| +58955|AAAAAAAAMEGOAAAA|58955|16|22|35|PM|second|afternoon|| +58956|AAAAAAAANEGOAAAA|58956|16|22|36|PM|second|afternoon|| +58957|AAAAAAAAOEGOAAAA|58957|16|22|37|PM|second|afternoon|| +58958|AAAAAAAAPEGOAAAA|58958|16|22|38|PM|second|afternoon|| +58959|AAAAAAAAAFGOAAAA|58959|16|22|39|PM|second|afternoon|| +58960|AAAAAAAABFGOAAAA|58960|16|22|40|PM|second|afternoon|| +58961|AAAAAAAACFGOAAAA|58961|16|22|41|PM|second|afternoon|| +58962|AAAAAAAADFGOAAAA|58962|16|22|42|PM|second|afternoon|| +58963|AAAAAAAAEFGOAAAA|58963|16|22|43|PM|second|afternoon|| +58964|AAAAAAAAFFGOAAAA|58964|16|22|44|PM|second|afternoon|| +58965|AAAAAAAAGFGOAAAA|58965|16|22|45|PM|second|afternoon|| +58966|AAAAAAAAHFGOAAAA|58966|16|22|46|PM|second|afternoon|| +58967|AAAAAAAAIFGOAAAA|58967|16|22|47|PM|second|afternoon|| +58968|AAAAAAAAJFGOAAAA|58968|16|22|48|PM|second|afternoon|| +58969|AAAAAAAAKFGOAAAA|58969|16|22|49|PM|second|afternoon|| +58970|AAAAAAAALFGOAAAA|58970|16|22|50|PM|second|afternoon|| +58971|AAAAAAAAMFGOAAAA|58971|16|22|51|PM|second|afternoon|| +58972|AAAAAAAANFGOAAAA|58972|16|22|52|PM|second|afternoon|| +58973|AAAAAAAAOFGOAAAA|58973|16|22|53|PM|second|afternoon|| +58974|AAAAAAAAPFGOAAAA|58974|16|22|54|PM|second|afternoon|| +58975|AAAAAAAAAGGOAAAA|58975|16|22|55|PM|second|afternoon|| +58976|AAAAAAAABGGOAAAA|58976|16|22|56|PM|second|afternoon|| +58977|AAAAAAAACGGOAAAA|58977|16|22|57|PM|second|afternoon|| +58978|AAAAAAAADGGOAAAA|58978|16|22|58|PM|second|afternoon|| +58979|AAAAAAAAEGGOAAAA|58979|16|22|59|PM|second|afternoon|| +58980|AAAAAAAAFGGOAAAA|58980|16|23|0|PM|second|afternoon|| +58981|AAAAAAAAGGGOAAAA|58981|16|23|1|PM|second|afternoon|| +58982|AAAAAAAAHGGOAAAA|58982|16|23|2|PM|second|afternoon|| +58983|AAAAAAAAIGGOAAAA|58983|16|23|3|PM|second|afternoon|| +58984|AAAAAAAAJGGOAAAA|58984|16|23|4|PM|second|afternoon|| +58985|AAAAAAAAKGGOAAAA|58985|16|23|5|PM|second|afternoon|| +58986|AAAAAAAALGGOAAAA|58986|16|23|6|PM|second|afternoon|| +58987|AAAAAAAAMGGOAAAA|58987|16|23|7|PM|second|afternoon|| +58988|AAAAAAAANGGOAAAA|58988|16|23|8|PM|second|afternoon|| +58989|AAAAAAAAOGGOAAAA|58989|16|23|9|PM|second|afternoon|| +58990|AAAAAAAAPGGOAAAA|58990|16|23|10|PM|second|afternoon|| +58991|AAAAAAAAAHGOAAAA|58991|16|23|11|PM|second|afternoon|| +58992|AAAAAAAABHGOAAAA|58992|16|23|12|PM|second|afternoon|| +58993|AAAAAAAACHGOAAAA|58993|16|23|13|PM|second|afternoon|| +58994|AAAAAAAADHGOAAAA|58994|16|23|14|PM|second|afternoon|| +58995|AAAAAAAAEHGOAAAA|58995|16|23|15|PM|second|afternoon|| +58996|AAAAAAAAFHGOAAAA|58996|16|23|16|PM|second|afternoon|| +58997|AAAAAAAAGHGOAAAA|58997|16|23|17|PM|second|afternoon|| +58998|AAAAAAAAHHGOAAAA|58998|16|23|18|PM|second|afternoon|| +58999|AAAAAAAAIHGOAAAA|58999|16|23|19|PM|second|afternoon|| +59000|AAAAAAAAJHGOAAAA|59000|16|23|20|PM|second|afternoon|| +59001|AAAAAAAAKHGOAAAA|59001|16|23|21|PM|second|afternoon|| +59002|AAAAAAAALHGOAAAA|59002|16|23|22|PM|second|afternoon|| +59003|AAAAAAAAMHGOAAAA|59003|16|23|23|PM|second|afternoon|| +59004|AAAAAAAANHGOAAAA|59004|16|23|24|PM|second|afternoon|| +59005|AAAAAAAAOHGOAAAA|59005|16|23|25|PM|second|afternoon|| +59006|AAAAAAAAPHGOAAAA|59006|16|23|26|PM|second|afternoon|| +59007|AAAAAAAAAIGOAAAA|59007|16|23|27|PM|second|afternoon|| +59008|AAAAAAAABIGOAAAA|59008|16|23|28|PM|second|afternoon|| +59009|AAAAAAAACIGOAAAA|59009|16|23|29|PM|second|afternoon|| +59010|AAAAAAAADIGOAAAA|59010|16|23|30|PM|second|afternoon|| +59011|AAAAAAAAEIGOAAAA|59011|16|23|31|PM|second|afternoon|| +59012|AAAAAAAAFIGOAAAA|59012|16|23|32|PM|second|afternoon|| +59013|AAAAAAAAGIGOAAAA|59013|16|23|33|PM|second|afternoon|| +59014|AAAAAAAAHIGOAAAA|59014|16|23|34|PM|second|afternoon|| +59015|AAAAAAAAIIGOAAAA|59015|16|23|35|PM|second|afternoon|| +59016|AAAAAAAAJIGOAAAA|59016|16|23|36|PM|second|afternoon|| +59017|AAAAAAAAKIGOAAAA|59017|16|23|37|PM|second|afternoon|| +59018|AAAAAAAALIGOAAAA|59018|16|23|38|PM|second|afternoon|| +59019|AAAAAAAAMIGOAAAA|59019|16|23|39|PM|second|afternoon|| +59020|AAAAAAAANIGOAAAA|59020|16|23|40|PM|second|afternoon|| +59021|AAAAAAAAOIGOAAAA|59021|16|23|41|PM|second|afternoon|| +59022|AAAAAAAAPIGOAAAA|59022|16|23|42|PM|second|afternoon|| +59023|AAAAAAAAAJGOAAAA|59023|16|23|43|PM|second|afternoon|| +59024|AAAAAAAABJGOAAAA|59024|16|23|44|PM|second|afternoon|| +59025|AAAAAAAACJGOAAAA|59025|16|23|45|PM|second|afternoon|| +59026|AAAAAAAADJGOAAAA|59026|16|23|46|PM|second|afternoon|| +59027|AAAAAAAAEJGOAAAA|59027|16|23|47|PM|second|afternoon|| +59028|AAAAAAAAFJGOAAAA|59028|16|23|48|PM|second|afternoon|| +59029|AAAAAAAAGJGOAAAA|59029|16|23|49|PM|second|afternoon|| +59030|AAAAAAAAHJGOAAAA|59030|16|23|50|PM|second|afternoon|| +59031|AAAAAAAAIJGOAAAA|59031|16|23|51|PM|second|afternoon|| +59032|AAAAAAAAJJGOAAAA|59032|16|23|52|PM|second|afternoon|| +59033|AAAAAAAAKJGOAAAA|59033|16|23|53|PM|second|afternoon|| +59034|AAAAAAAALJGOAAAA|59034|16|23|54|PM|second|afternoon|| +59035|AAAAAAAAMJGOAAAA|59035|16|23|55|PM|second|afternoon|| +59036|AAAAAAAANJGOAAAA|59036|16|23|56|PM|second|afternoon|| +59037|AAAAAAAAOJGOAAAA|59037|16|23|57|PM|second|afternoon|| +59038|AAAAAAAAPJGOAAAA|59038|16|23|58|PM|second|afternoon|| +59039|AAAAAAAAAKGOAAAA|59039|16|23|59|PM|second|afternoon|| +59040|AAAAAAAABKGOAAAA|59040|16|24|0|PM|second|afternoon|| +59041|AAAAAAAACKGOAAAA|59041|16|24|1|PM|second|afternoon|| +59042|AAAAAAAADKGOAAAA|59042|16|24|2|PM|second|afternoon|| +59043|AAAAAAAAEKGOAAAA|59043|16|24|3|PM|second|afternoon|| +59044|AAAAAAAAFKGOAAAA|59044|16|24|4|PM|second|afternoon|| +59045|AAAAAAAAGKGOAAAA|59045|16|24|5|PM|second|afternoon|| +59046|AAAAAAAAHKGOAAAA|59046|16|24|6|PM|second|afternoon|| +59047|AAAAAAAAIKGOAAAA|59047|16|24|7|PM|second|afternoon|| +59048|AAAAAAAAJKGOAAAA|59048|16|24|8|PM|second|afternoon|| +59049|AAAAAAAAKKGOAAAA|59049|16|24|9|PM|second|afternoon|| +59050|AAAAAAAALKGOAAAA|59050|16|24|10|PM|second|afternoon|| +59051|AAAAAAAAMKGOAAAA|59051|16|24|11|PM|second|afternoon|| +59052|AAAAAAAANKGOAAAA|59052|16|24|12|PM|second|afternoon|| +59053|AAAAAAAAOKGOAAAA|59053|16|24|13|PM|second|afternoon|| +59054|AAAAAAAAPKGOAAAA|59054|16|24|14|PM|second|afternoon|| +59055|AAAAAAAAALGOAAAA|59055|16|24|15|PM|second|afternoon|| +59056|AAAAAAAABLGOAAAA|59056|16|24|16|PM|second|afternoon|| +59057|AAAAAAAACLGOAAAA|59057|16|24|17|PM|second|afternoon|| +59058|AAAAAAAADLGOAAAA|59058|16|24|18|PM|second|afternoon|| +59059|AAAAAAAAELGOAAAA|59059|16|24|19|PM|second|afternoon|| +59060|AAAAAAAAFLGOAAAA|59060|16|24|20|PM|second|afternoon|| +59061|AAAAAAAAGLGOAAAA|59061|16|24|21|PM|second|afternoon|| +59062|AAAAAAAAHLGOAAAA|59062|16|24|22|PM|second|afternoon|| +59063|AAAAAAAAILGOAAAA|59063|16|24|23|PM|second|afternoon|| +59064|AAAAAAAAJLGOAAAA|59064|16|24|24|PM|second|afternoon|| +59065|AAAAAAAAKLGOAAAA|59065|16|24|25|PM|second|afternoon|| +59066|AAAAAAAALLGOAAAA|59066|16|24|26|PM|second|afternoon|| +59067|AAAAAAAAMLGOAAAA|59067|16|24|27|PM|second|afternoon|| +59068|AAAAAAAANLGOAAAA|59068|16|24|28|PM|second|afternoon|| +59069|AAAAAAAAOLGOAAAA|59069|16|24|29|PM|second|afternoon|| +59070|AAAAAAAAPLGOAAAA|59070|16|24|30|PM|second|afternoon|| +59071|AAAAAAAAAMGOAAAA|59071|16|24|31|PM|second|afternoon|| +59072|AAAAAAAABMGOAAAA|59072|16|24|32|PM|second|afternoon|| +59073|AAAAAAAACMGOAAAA|59073|16|24|33|PM|second|afternoon|| +59074|AAAAAAAADMGOAAAA|59074|16|24|34|PM|second|afternoon|| +59075|AAAAAAAAEMGOAAAA|59075|16|24|35|PM|second|afternoon|| +59076|AAAAAAAAFMGOAAAA|59076|16|24|36|PM|second|afternoon|| +59077|AAAAAAAAGMGOAAAA|59077|16|24|37|PM|second|afternoon|| +59078|AAAAAAAAHMGOAAAA|59078|16|24|38|PM|second|afternoon|| +59079|AAAAAAAAIMGOAAAA|59079|16|24|39|PM|second|afternoon|| +59080|AAAAAAAAJMGOAAAA|59080|16|24|40|PM|second|afternoon|| +59081|AAAAAAAAKMGOAAAA|59081|16|24|41|PM|second|afternoon|| +59082|AAAAAAAALMGOAAAA|59082|16|24|42|PM|second|afternoon|| +59083|AAAAAAAAMMGOAAAA|59083|16|24|43|PM|second|afternoon|| +59084|AAAAAAAANMGOAAAA|59084|16|24|44|PM|second|afternoon|| +59085|AAAAAAAAOMGOAAAA|59085|16|24|45|PM|second|afternoon|| +59086|AAAAAAAAPMGOAAAA|59086|16|24|46|PM|second|afternoon|| +59087|AAAAAAAAANGOAAAA|59087|16|24|47|PM|second|afternoon|| +59088|AAAAAAAABNGOAAAA|59088|16|24|48|PM|second|afternoon|| +59089|AAAAAAAACNGOAAAA|59089|16|24|49|PM|second|afternoon|| +59090|AAAAAAAADNGOAAAA|59090|16|24|50|PM|second|afternoon|| +59091|AAAAAAAAENGOAAAA|59091|16|24|51|PM|second|afternoon|| +59092|AAAAAAAAFNGOAAAA|59092|16|24|52|PM|second|afternoon|| +59093|AAAAAAAAGNGOAAAA|59093|16|24|53|PM|second|afternoon|| +59094|AAAAAAAAHNGOAAAA|59094|16|24|54|PM|second|afternoon|| +59095|AAAAAAAAINGOAAAA|59095|16|24|55|PM|second|afternoon|| +59096|AAAAAAAAJNGOAAAA|59096|16|24|56|PM|second|afternoon|| +59097|AAAAAAAAKNGOAAAA|59097|16|24|57|PM|second|afternoon|| +59098|AAAAAAAALNGOAAAA|59098|16|24|58|PM|second|afternoon|| +59099|AAAAAAAAMNGOAAAA|59099|16|24|59|PM|second|afternoon|| +59100|AAAAAAAANNGOAAAA|59100|16|25|0|PM|second|afternoon|| +59101|AAAAAAAAONGOAAAA|59101|16|25|1|PM|second|afternoon|| +59102|AAAAAAAAPNGOAAAA|59102|16|25|2|PM|second|afternoon|| +59103|AAAAAAAAAOGOAAAA|59103|16|25|3|PM|second|afternoon|| +59104|AAAAAAAABOGOAAAA|59104|16|25|4|PM|second|afternoon|| +59105|AAAAAAAACOGOAAAA|59105|16|25|5|PM|second|afternoon|| +59106|AAAAAAAADOGOAAAA|59106|16|25|6|PM|second|afternoon|| +59107|AAAAAAAAEOGOAAAA|59107|16|25|7|PM|second|afternoon|| +59108|AAAAAAAAFOGOAAAA|59108|16|25|8|PM|second|afternoon|| +59109|AAAAAAAAGOGOAAAA|59109|16|25|9|PM|second|afternoon|| +59110|AAAAAAAAHOGOAAAA|59110|16|25|10|PM|second|afternoon|| +59111|AAAAAAAAIOGOAAAA|59111|16|25|11|PM|second|afternoon|| +59112|AAAAAAAAJOGOAAAA|59112|16|25|12|PM|second|afternoon|| +59113|AAAAAAAAKOGOAAAA|59113|16|25|13|PM|second|afternoon|| +59114|AAAAAAAALOGOAAAA|59114|16|25|14|PM|second|afternoon|| +59115|AAAAAAAAMOGOAAAA|59115|16|25|15|PM|second|afternoon|| +59116|AAAAAAAANOGOAAAA|59116|16|25|16|PM|second|afternoon|| +59117|AAAAAAAAOOGOAAAA|59117|16|25|17|PM|second|afternoon|| +59118|AAAAAAAAPOGOAAAA|59118|16|25|18|PM|second|afternoon|| +59119|AAAAAAAAAPGOAAAA|59119|16|25|19|PM|second|afternoon|| +59120|AAAAAAAABPGOAAAA|59120|16|25|20|PM|second|afternoon|| +59121|AAAAAAAACPGOAAAA|59121|16|25|21|PM|second|afternoon|| +59122|AAAAAAAADPGOAAAA|59122|16|25|22|PM|second|afternoon|| +59123|AAAAAAAAEPGOAAAA|59123|16|25|23|PM|second|afternoon|| +59124|AAAAAAAAFPGOAAAA|59124|16|25|24|PM|second|afternoon|| +59125|AAAAAAAAGPGOAAAA|59125|16|25|25|PM|second|afternoon|| +59126|AAAAAAAAHPGOAAAA|59126|16|25|26|PM|second|afternoon|| +59127|AAAAAAAAIPGOAAAA|59127|16|25|27|PM|second|afternoon|| +59128|AAAAAAAAJPGOAAAA|59128|16|25|28|PM|second|afternoon|| +59129|AAAAAAAAKPGOAAAA|59129|16|25|29|PM|second|afternoon|| +59130|AAAAAAAALPGOAAAA|59130|16|25|30|PM|second|afternoon|| +59131|AAAAAAAAMPGOAAAA|59131|16|25|31|PM|second|afternoon|| +59132|AAAAAAAANPGOAAAA|59132|16|25|32|PM|second|afternoon|| +59133|AAAAAAAAOPGOAAAA|59133|16|25|33|PM|second|afternoon|| +59134|AAAAAAAAPPGOAAAA|59134|16|25|34|PM|second|afternoon|| +59135|AAAAAAAAAAHOAAAA|59135|16|25|35|PM|second|afternoon|| +59136|AAAAAAAABAHOAAAA|59136|16|25|36|PM|second|afternoon|| +59137|AAAAAAAACAHOAAAA|59137|16|25|37|PM|second|afternoon|| +59138|AAAAAAAADAHOAAAA|59138|16|25|38|PM|second|afternoon|| +59139|AAAAAAAAEAHOAAAA|59139|16|25|39|PM|second|afternoon|| +59140|AAAAAAAAFAHOAAAA|59140|16|25|40|PM|second|afternoon|| +59141|AAAAAAAAGAHOAAAA|59141|16|25|41|PM|second|afternoon|| +59142|AAAAAAAAHAHOAAAA|59142|16|25|42|PM|second|afternoon|| +59143|AAAAAAAAIAHOAAAA|59143|16|25|43|PM|second|afternoon|| +59144|AAAAAAAAJAHOAAAA|59144|16|25|44|PM|second|afternoon|| +59145|AAAAAAAAKAHOAAAA|59145|16|25|45|PM|second|afternoon|| +59146|AAAAAAAALAHOAAAA|59146|16|25|46|PM|second|afternoon|| +59147|AAAAAAAAMAHOAAAA|59147|16|25|47|PM|second|afternoon|| +59148|AAAAAAAANAHOAAAA|59148|16|25|48|PM|second|afternoon|| +59149|AAAAAAAAOAHOAAAA|59149|16|25|49|PM|second|afternoon|| +59150|AAAAAAAAPAHOAAAA|59150|16|25|50|PM|second|afternoon|| +59151|AAAAAAAAABHOAAAA|59151|16|25|51|PM|second|afternoon|| +59152|AAAAAAAABBHOAAAA|59152|16|25|52|PM|second|afternoon|| +59153|AAAAAAAACBHOAAAA|59153|16|25|53|PM|second|afternoon|| +59154|AAAAAAAADBHOAAAA|59154|16|25|54|PM|second|afternoon|| +59155|AAAAAAAAEBHOAAAA|59155|16|25|55|PM|second|afternoon|| +59156|AAAAAAAAFBHOAAAA|59156|16|25|56|PM|second|afternoon|| +59157|AAAAAAAAGBHOAAAA|59157|16|25|57|PM|second|afternoon|| +59158|AAAAAAAAHBHOAAAA|59158|16|25|58|PM|second|afternoon|| +59159|AAAAAAAAIBHOAAAA|59159|16|25|59|PM|second|afternoon|| +59160|AAAAAAAAJBHOAAAA|59160|16|26|0|PM|second|afternoon|| +59161|AAAAAAAAKBHOAAAA|59161|16|26|1|PM|second|afternoon|| +59162|AAAAAAAALBHOAAAA|59162|16|26|2|PM|second|afternoon|| +59163|AAAAAAAAMBHOAAAA|59163|16|26|3|PM|second|afternoon|| +59164|AAAAAAAANBHOAAAA|59164|16|26|4|PM|second|afternoon|| +59165|AAAAAAAAOBHOAAAA|59165|16|26|5|PM|second|afternoon|| +59166|AAAAAAAAPBHOAAAA|59166|16|26|6|PM|second|afternoon|| +59167|AAAAAAAAACHOAAAA|59167|16|26|7|PM|second|afternoon|| +59168|AAAAAAAABCHOAAAA|59168|16|26|8|PM|second|afternoon|| +59169|AAAAAAAACCHOAAAA|59169|16|26|9|PM|second|afternoon|| +59170|AAAAAAAADCHOAAAA|59170|16|26|10|PM|second|afternoon|| +59171|AAAAAAAAECHOAAAA|59171|16|26|11|PM|second|afternoon|| +59172|AAAAAAAAFCHOAAAA|59172|16|26|12|PM|second|afternoon|| +59173|AAAAAAAAGCHOAAAA|59173|16|26|13|PM|second|afternoon|| +59174|AAAAAAAAHCHOAAAA|59174|16|26|14|PM|second|afternoon|| +59175|AAAAAAAAICHOAAAA|59175|16|26|15|PM|second|afternoon|| +59176|AAAAAAAAJCHOAAAA|59176|16|26|16|PM|second|afternoon|| +59177|AAAAAAAAKCHOAAAA|59177|16|26|17|PM|second|afternoon|| +59178|AAAAAAAALCHOAAAA|59178|16|26|18|PM|second|afternoon|| +59179|AAAAAAAAMCHOAAAA|59179|16|26|19|PM|second|afternoon|| +59180|AAAAAAAANCHOAAAA|59180|16|26|20|PM|second|afternoon|| +59181|AAAAAAAAOCHOAAAA|59181|16|26|21|PM|second|afternoon|| +59182|AAAAAAAAPCHOAAAA|59182|16|26|22|PM|second|afternoon|| +59183|AAAAAAAAADHOAAAA|59183|16|26|23|PM|second|afternoon|| +59184|AAAAAAAABDHOAAAA|59184|16|26|24|PM|second|afternoon|| +59185|AAAAAAAACDHOAAAA|59185|16|26|25|PM|second|afternoon|| +59186|AAAAAAAADDHOAAAA|59186|16|26|26|PM|second|afternoon|| +59187|AAAAAAAAEDHOAAAA|59187|16|26|27|PM|second|afternoon|| +59188|AAAAAAAAFDHOAAAA|59188|16|26|28|PM|second|afternoon|| +59189|AAAAAAAAGDHOAAAA|59189|16|26|29|PM|second|afternoon|| +59190|AAAAAAAAHDHOAAAA|59190|16|26|30|PM|second|afternoon|| +59191|AAAAAAAAIDHOAAAA|59191|16|26|31|PM|second|afternoon|| +59192|AAAAAAAAJDHOAAAA|59192|16|26|32|PM|second|afternoon|| +59193|AAAAAAAAKDHOAAAA|59193|16|26|33|PM|second|afternoon|| +59194|AAAAAAAALDHOAAAA|59194|16|26|34|PM|second|afternoon|| +59195|AAAAAAAAMDHOAAAA|59195|16|26|35|PM|second|afternoon|| +59196|AAAAAAAANDHOAAAA|59196|16|26|36|PM|second|afternoon|| +59197|AAAAAAAAODHOAAAA|59197|16|26|37|PM|second|afternoon|| +59198|AAAAAAAAPDHOAAAA|59198|16|26|38|PM|second|afternoon|| +59199|AAAAAAAAAEHOAAAA|59199|16|26|39|PM|second|afternoon|| +59200|AAAAAAAABEHOAAAA|59200|16|26|40|PM|second|afternoon|| +59201|AAAAAAAACEHOAAAA|59201|16|26|41|PM|second|afternoon|| +59202|AAAAAAAADEHOAAAA|59202|16|26|42|PM|second|afternoon|| +59203|AAAAAAAAEEHOAAAA|59203|16|26|43|PM|second|afternoon|| +59204|AAAAAAAAFEHOAAAA|59204|16|26|44|PM|second|afternoon|| +59205|AAAAAAAAGEHOAAAA|59205|16|26|45|PM|second|afternoon|| +59206|AAAAAAAAHEHOAAAA|59206|16|26|46|PM|second|afternoon|| +59207|AAAAAAAAIEHOAAAA|59207|16|26|47|PM|second|afternoon|| +59208|AAAAAAAAJEHOAAAA|59208|16|26|48|PM|second|afternoon|| +59209|AAAAAAAAKEHOAAAA|59209|16|26|49|PM|second|afternoon|| +59210|AAAAAAAALEHOAAAA|59210|16|26|50|PM|second|afternoon|| +59211|AAAAAAAAMEHOAAAA|59211|16|26|51|PM|second|afternoon|| +59212|AAAAAAAANEHOAAAA|59212|16|26|52|PM|second|afternoon|| +59213|AAAAAAAAOEHOAAAA|59213|16|26|53|PM|second|afternoon|| +59214|AAAAAAAAPEHOAAAA|59214|16|26|54|PM|second|afternoon|| +59215|AAAAAAAAAFHOAAAA|59215|16|26|55|PM|second|afternoon|| +59216|AAAAAAAABFHOAAAA|59216|16|26|56|PM|second|afternoon|| +59217|AAAAAAAACFHOAAAA|59217|16|26|57|PM|second|afternoon|| +59218|AAAAAAAADFHOAAAA|59218|16|26|58|PM|second|afternoon|| +59219|AAAAAAAAEFHOAAAA|59219|16|26|59|PM|second|afternoon|| +59220|AAAAAAAAFFHOAAAA|59220|16|27|0|PM|second|afternoon|| +59221|AAAAAAAAGFHOAAAA|59221|16|27|1|PM|second|afternoon|| +59222|AAAAAAAAHFHOAAAA|59222|16|27|2|PM|second|afternoon|| +59223|AAAAAAAAIFHOAAAA|59223|16|27|3|PM|second|afternoon|| +59224|AAAAAAAAJFHOAAAA|59224|16|27|4|PM|second|afternoon|| +59225|AAAAAAAAKFHOAAAA|59225|16|27|5|PM|second|afternoon|| +59226|AAAAAAAALFHOAAAA|59226|16|27|6|PM|second|afternoon|| +59227|AAAAAAAAMFHOAAAA|59227|16|27|7|PM|second|afternoon|| +59228|AAAAAAAANFHOAAAA|59228|16|27|8|PM|second|afternoon|| +59229|AAAAAAAAOFHOAAAA|59229|16|27|9|PM|second|afternoon|| +59230|AAAAAAAAPFHOAAAA|59230|16|27|10|PM|second|afternoon|| +59231|AAAAAAAAAGHOAAAA|59231|16|27|11|PM|second|afternoon|| +59232|AAAAAAAABGHOAAAA|59232|16|27|12|PM|second|afternoon|| +59233|AAAAAAAACGHOAAAA|59233|16|27|13|PM|second|afternoon|| +59234|AAAAAAAADGHOAAAA|59234|16|27|14|PM|second|afternoon|| +59235|AAAAAAAAEGHOAAAA|59235|16|27|15|PM|second|afternoon|| +59236|AAAAAAAAFGHOAAAA|59236|16|27|16|PM|second|afternoon|| +59237|AAAAAAAAGGHOAAAA|59237|16|27|17|PM|second|afternoon|| +59238|AAAAAAAAHGHOAAAA|59238|16|27|18|PM|second|afternoon|| +59239|AAAAAAAAIGHOAAAA|59239|16|27|19|PM|second|afternoon|| +59240|AAAAAAAAJGHOAAAA|59240|16|27|20|PM|second|afternoon|| +59241|AAAAAAAAKGHOAAAA|59241|16|27|21|PM|second|afternoon|| +59242|AAAAAAAALGHOAAAA|59242|16|27|22|PM|second|afternoon|| +59243|AAAAAAAAMGHOAAAA|59243|16|27|23|PM|second|afternoon|| +59244|AAAAAAAANGHOAAAA|59244|16|27|24|PM|second|afternoon|| +59245|AAAAAAAAOGHOAAAA|59245|16|27|25|PM|second|afternoon|| +59246|AAAAAAAAPGHOAAAA|59246|16|27|26|PM|second|afternoon|| +59247|AAAAAAAAAHHOAAAA|59247|16|27|27|PM|second|afternoon|| +59248|AAAAAAAABHHOAAAA|59248|16|27|28|PM|second|afternoon|| +59249|AAAAAAAACHHOAAAA|59249|16|27|29|PM|second|afternoon|| +59250|AAAAAAAADHHOAAAA|59250|16|27|30|PM|second|afternoon|| +59251|AAAAAAAAEHHOAAAA|59251|16|27|31|PM|second|afternoon|| +59252|AAAAAAAAFHHOAAAA|59252|16|27|32|PM|second|afternoon|| +59253|AAAAAAAAGHHOAAAA|59253|16|27|33|PM|second|afternoon|| +59254|AAAAAAAAHHHOAAAA|59254|16|27|34|PM|second|afternoon|| +59255|AAAAAAAAIHHOAAAA|59255|16|27|35|PM|second|afternoon|| +59256|AAAAAAAAJHHOAAAA|59256|16|27|36|PM|second|afternoon|| +59257|AAAAAAAAKHHOAAAA|59257|16|27|37|PM|second|afternoon|| +59258|AAAAAAAALHHOAAAA|59258|16|27|38|PM|second|afternoon|| +59259|AAAAAAAAMHHOAAAA|59259|16|27|39|PM|second|afternoon|| +59260|AAAAAAAANHHOAAAA|59260|16|27|40|PM|second|afternoon|| +59261|AAAAAAAAOHHOAAAA|59261|16|27|41|PM|second|afternoon|| +59262|AAAAAAAAPHHOAAAA|59262|16|27|42|PM|second|afternoon|| +59263|AAAAAAAAAIHOAAAA|59263|16|27|43|PM|second|afternoon|| +59264|AAAAAAAABIHOAAAA|59264|16|27|44|PM|second|afternoon|| +59265|AAAAAAAACIHOAAAA|59265|16|27|45|PM|second|afternoon|| +59266|AAAAAAAADIHOAAAA|59266|16|27|46|PM|second|afternoon|| +59267|AAAAAAAAEIHOAAAA|59267|16|27|47|PM|second|afternoon|| +59268|AAAAAAAAFIHOAAAA|59268|16|27|48|PM|second|afternoon|| +59269|AAAAAAAAGIHOAAAA|59269|16|27|49|PM|second|afternoon|| +59270|AAAAAAAAHIHOAAAA|59270|16|27|50|PM|second|afternoon|| +59271|AAAAAAAAIIHOAAAA|59271|16|27|51|PM|second|afternoon|| +59272|AAAAAAAAJIHOAAAA|59272|16|27|52|PM|second|afternoon|| +59273|AAAAAAAAKIHOAAAA|59273|16|27|53|PM|second|afternoon|| +59274|AAAAAAAALIHOAAAA|59274|16|27|54|PM|second|afternoon|| +59275|AAAAAAAAMIHOAAAA|59275|16|27|55|PM|second|afternoon|| +59276|AAAAAAAANIHOAAAA|59276|16|27|56|PM|second|afternoon|| +59277|AAAAAAAAOIHOAAAA|59277|16|27|57|PM|second|afternoon|| +59278|AAAAAAAAPIHOAAAA|59278|16|27|58|PM|second|afternoon|| +59279|AAAAAAAAAJHOAAAA|59279|16|27|59|PM|second|afternoon|| +59280|AAAAAAAABJHOAAAA|59280|16|28|0|PM|second|afternoon|| +59281|AAAAAAAACJHOAAAA|59281|16|28|1|PM|second|afternoon|| +59282|AAAAAAAADJHOAAAA|59282|16|28|2|PM|second|afternoon|| +59283|AAAAAAAAEJHOAAAA|59283|16|28|3|PM|second|afternoon|| +59284|AAAAAAAAFJHOAAAA|59284|16|28|4|PM|second|afternoon|| +59285|AAAAAAAAGJHOAAAA|59285|16|28|5|PM|second|afternoon|| +59286|AAAAAAAAHJHOAAAA|59286|16|28|6|PM|second|afternoon|| +59287|AAAAAAAAIJHOAAAA|59287|16|28|7|PM|second|afternoon|| +59288|AAAAAAAAJJHOAAAA|59288|16|28|8|PM|second|afternoon|| +59289|AAAAAAAAKJHOAAAA|59289|16|28|9|PM|second|afternoon|| +59290|AAAAAAAALJHOAAAA|59290|16|28|10|PM|second|afternoon|| +59291|AAAAAAAAMJHOAAAA|59291|16|28|11|PM|second|afternoon|| +59292|AAAAAAAANJHOAAAA|59292|16|28|12|PM|second|afternoon|| +59293|AAAAAAAAOJHOAAAA|59293|16|28|13|PM|second|afternoon|| +59294|AAAAAAAAPJHOAAAA|59294|16|28|14|PM|second|afternoon|| +59295|AAAAAAAAAKHOAAAA|59295|16|28|15|PM|second|afternoon|| +59296|AAAAAAAABKHOAAAA|59296|16|28|16|PM|second|afternoon|| +59297|AAAAAAAACKHOAAAA|59297|16|28|17|PM|second|afternoon|| +59298|AAAAAAAADKHOAAAA|59298|16|28|18|PM|second|afternoon|| +59299|AAAAAAAAEKHOAAAA|59299|16|28|19|PM|second|afternoon|| +59300|AAAAAAAAFKHOAAAA|59300|16|28|20|PM|second|afternoon|| +59301|AAAAAAAAGKHOAAAA|59301|16|28|21|PM|second|afternoon|| +59302|AAAAAAAAHKHOAAAA|59302|16|28|22|PM|second|afternoon|| +59303|AAAAAAAAIKHOAAAA|59303|16|28|23|PM|second|afternoon|| +59304|AAAAAAAAJKHOAAAA|59304|16|28|24|PM|second|afternoon|| +59305|AAAAAAAAKKHOAAAA|59305|16|28|25|PM|second|afternoon|| +59306|AAAAAAAALKHOAAAA|59306|16|28|26|PM|second|afternoon|| +59307|AAAAAAAAMKHOAAAA|59307|16|28|27|PM|second|afternoon|| +59308|AAAAAAAANKHOAAAA|59308|16|28|28|PM|second|afternoon|| +59309|AAAAAAAAOKHOAAAA|59309|16|28|29|PM|second|afternoon|| +59310|AAAAAAAAPKHOAAAA|59310|16|28|30|PM|second|afternoon|| +59311|AAAAAAAAALHOAAAA|59311|16|28|31|PM|second|afternoon|| +59312|AAAAAAAABLHOAAAA|59312|16|28|32|PM|second|afternoon|| +59313|AAAAAAAACLHOAAAA|59313|16|28|33|PM|second|afternoon|| +59314|AAAAAAAADLHOAAAA|59314|16|28|34|PM|second|afternoon|| +59315|AAAAAAAAELHOAAAA|59315|16|28|35|PM|second|afternoon|| +59316|AAAAAAAAFLHOAAAA|59316|16|28|36|PM|second|afternoon|| +59317|AAAAAAAAGLHOAAAA|59317|16|28|37|PM|second|afternoon|| +59318|AAAAAAAAHLHOAAAA|59318|16|28|38|PM|second|afternoon|| +59319|AAAAAAAAILHOAAAA|59319|16|28|39|PM|second|afternoon|| +59320|AAAAAAAAJLHOAAAA|59320|16|28|40|PM|second|afternoon|| +59321|AAAAAAAAKLHOAAAA|59321|16|28|41|PM|second|afternoon|| +59322|AAAAAAAALLHOAAAA|59322|16|28|42|PM|second|afternoon|| +59323|AAAAAAAAMLHOAAAA|59323|16|28|43|PM|second|afternoon|| +59324|AAAAAAAANLHOAAAA|59324|16|28|44|PM|second|afternoon|| +59325|AAAAAAAAOLHOAAAA|59325|16|28|45|PM|second|afternoon|| +59326|AAAAAAAAPLHOAAAA|59326|16|28|46|PM|second|afternoon|| +59327|AAAAAAAAAMHOAAAA|59327|16|28|47|PM|second|afternoon|| +59328|AAAAAAAABMHOAAAA|59328|16|28|48|PM|second|afternoon|| +59329|AAAAAAAACMHOAAAA|59329|16|28|49|PM|second|afternoon|| +59330|AAAAAAAADMHOAAAA|59330|16|28|50|PM|second|afternoon|| +59331|AAAAAAAAEMHOAAAA|59331|16|28|51|PM|second|afternoon|| +59332|AAAAAAAAFMHOAAAA|59332|16|28|52|PM|second|afternoon|| +59333|AAAAAAAAGMHOAAAA|59333|16|28|53|PM|second|afternoon|| +59334|AAAAAAAAHMHOAAAA|59334|16|28|54|PM|second|afternoon|| +59335|AAAAAAAAIMHOAAAA|59335|16|28|55|PM|second|afternoon|| +59336|AAAAAAAAJMHOAAAA|59336|16|28|56|PM|second|afternoon|| +59337|AAAAAAAAKMHOAAAA|59337|16|28|57|PM|second|afternoon|| +59338|AAAAAAAALMHOAAAA|59338|16|28|58|PM|second|afternoon|| +59339|AAAAAAAAMMHOAAAA|59339|16|28|59|PM|second|afternoon|| +59340|AAAAAAAANMHOAAAA|59340|16|29|0|PM|second|afternoon|| +59341|AAAAAAAAOMHOAAAA|59341|16|29|1|PM|second|afternoon|| +59342|AAAAAAAAPMHOAAAA|59342|16|29|2|PM|second|afternoon|| +59343|AAAAAAAAANHOAAAA|59343|16|29|3|PM|second|afternoon|| +59344|AAAAAAAABNHOAAAA|59344|16|29|4|PM|second|afternoon|| +59345|AAAAAAAACNHOAAAA|59345|16|29|5|PM|second|afternoon|| +59346|AAAAAAAADNHOAAAA|59346|16|29|6|PM|second|afternoon|| +59347|AAAAAAAAENHOAAAA|59347|16|29|7|PM|second|afternoon|| +59348|AAAAAAAAFNHOAAAA|59348|16|29|8|PM|second|afternoon|| +59349|AAAAAAAAGNHOAAAA|59349|16|29|9|PM|second|afternoon|| +59350|AAAAAAAAHNHOAAAA|59350|16|29|10|PM|second|afternoon|| +59351|AAAAAAAAINHOAAAA|59351|16|29|11|PM|second|afternoon|| +59352|AAAAAAAAJNHOAAAA|59352|16|29|12|PM|second|afternoon|| +59353|AAAAAAAAKNHOAAAA|59353|16|29|13|PM|second|afternoon|| +59354|AAAAAAAALNHOAAAA|59354|16|29|14|PM|second|afternoon|| +59355|AAAAAAAAMNHOAAAA|59355|16|29|15|PM|second|afternoon|| +59356|AAAAAAAANNHOAAAA|59356|16|29|16|PM|second|afternoon|| +59357|AAAAAAAAONHOAAAA|59357|16|29|17|PM|second|afternoon|| +59358|AAAAAAAAPNHOAAAA|59358|16|29|18|PM|second|afternoon|| +59359|AAAAAAAAAOHOAAAA|59359|16|29|19|PM|second|afternoon|| +59360|AAAAAAAABOHOAAAA|59360|16|29|20|PM|second|afternoon|| +59361|AAAAAAAACOHOAAAA|59361|16|29|21|PM|second|afternoon|| +59362|AAAAAAAADOHOAAAA|59362|16|29|22|PM|second|afternoon|| +59363|AAAAAAAAEOHOAAAA|59363|16|29|23|PM|second|afternoon|| +59364|AAAAAAAAFOHOAAAA|59364|16|29|24|PM|second|afternoon|| +59365|AAAAAAAAGOHOAAAA|59365|16|29|25|PM|second|afternoon|| +59366|AAAAAAAAHOHOAAAA|59366|16|29|26|PM|second|afternoon|| +59367|AAAAAAAAIOHOAAAA|59367|16|29|27|PM|second|afternoon|| +59368|AAAAAAAAJOHOAAAA|59368|16|29|28|PM|second|afternoon|| +59369|AAAAAAAAKOHOAAAA|59369|16|29|29|PM|second|afternoon|| +59370|AAAAAAAALOHOAAAA|59370|16|29|30|PM|second|afternoon|| +59371|AAAAAAAAMOHOAAAA|59371|16|29|31|PM|second|afternoon|| +59372|AAAAAAAANOHOAAAA|59372|16|29|32|PM|second|afternoon|| +59373|AAAAAAAAOOHOAAAA|59373|16|29|33|PM|second|afternoon|| +59374|AAAAAAAAPOHOAAAA|59374|16|29|34|PM|second|afternoon|| +59375|AAAAAAAAAPHOAAAA|59375|16|29|35|PM|second|afternoon|| +59376|AAAAAAAABPHOAAAA|59376|16|29|36|PM|second|afternoon|| +59377|AAAAAAAACPHOAAAA|59377|16|29|37|PM|second|afternoon|| +59378|AAAAAAAADPHOAAAA|59378|16|29|38|PM|second|afternoon|| +59379|AAAAAAAAEPHOAAAA|59379|16|29|39|PM|second|afternoon|| +59380|AAAAAAAAFPHOAAAA|59380|16|29|40|PM|second|afternoon|| +59381|AAAAAAAAGPHOAAAA|59381|16|29|41|PM|second|afternoon|| +59382|AAAAAAAAHPHOAAAA|59382|16|29|42|PM|second|afternoon|| +59383|AAAAAAAAIPHOAAAA|59383|16|29|43|PM|second|afternoon|| +59384|AAAAAAAAJPHOAAAA|59384|16|29|44|PM|second|afternoon|| +59385|AAAAAAAAKPHOAAAA|59385|16|29|45|PM|second|afternoon|| +59386|AAAAAAAALPHOAAAA|59386|16|29|46|PM|second|afternoon|| +59387|AAAAAAAAMPHOAAAA|59387|16|29|47|PM|second|afternoon|| +59388|AAAAAAAANPHOAAAA|59388|16|29|48|PM|second|afternoon|| +59389|AAAAAAAAOPHOAAAA|59389|16|29|49|PM|second|afternoon|| +59390|AAAAAAAAPPHOAAAA|59390|16|29|50|PM|second|afternoon|| +59391|AAAAAAAAAAIOAAAA|59391|16|29|51|PM|second|afternoon|| +59392|AAAAAAAABAIOAAAA|59392|16|29|52|PM|second|afternoon|| +59393|AAAAAAAACAIOAAAA|59393|16|29|53|PM|second|afternoon|| +59394|AAAAAAAADAIOAAAA|59394|16|29|54|PM|second|afternoon|| +59395|AAAAAAAAEAIOAAAA|59395|16|29|55|PM|second|afternoon|| +59396|AAAAAAAAFAIOAAAA|59396|16|29|56|PM|second|afternoon|| +59397|AAAAAAAAGAIOAAAA|59397|16|29|57|PM|second|afternoon|| +59398|AAAAAAAAHAIOAAAA|59398|16|29|58|PM|second|afternoon|| +59399|AAAAAAAAIAIOAAAA|59399|16|29|59|PM|second|afternoon|| +59400|AAAAAAAAJAIOAAAA|59400|16|30|0|PM|second|afternoon|| +59401|AAAAAAAAKAIOAAAA|59401|16|30|1|PM|second|afternoon|| +59402|AAAAAAAALAIOAAAA|59402|16|30|2|PM|second|afternoon|| +59403|AAAAAAAAMAIOAAAA|59403|16|30|3|PM|second|afternoon|| +59404|AAAAAAAANAIOAAAA|59404|16|30|4|PM|second|afternoon|| +59405|AAAAAAAAOAIOAAAA|59405|16|30|5|PM|second|afternoon|| +59406|AAAAAAAAPAIOAAAA|59406|16|30|6|PM|second|afternoon|| +59407|AAAAAAAAABIOAAAA|59407|16|30|7|PM|second|afternoon|| +59408|AAAAAAAABBIOAAAA|59408|16|30|8|PM|second|afternoon|| +59409|AAAAAAAACBIOAAAA|59409|16|30|9|PM|second|afternoon|| +59410|AAAAAAAADBIOAAAA|59410|16|30|10|PM|second|afternoon|| +59411|AAAAAAAAEBIOAAAA|59411|16|30|11|PM|second|afternoon|| +59412|AAAAAAAAFBIOAAAA|59412|16|30|12|PM|second|afternoon|| +59413|AAAAAAAAGBIOAAAA|59413|16|30|13|PM|second|afternoon|| +59414|AAAAAAAAHBIOAAAA|59414|16|30|14|PM|second|afternoon|| +59415|AAAAAAAAIBIOAAAA|59415|16|30|15|PM|second|afternoon|| +59416|AAAAAAAAJBIOAAAA|59416|16|30|16|PM|second|afternoon|| +59417|AAAAAAAAKBIOAAAA|59417|16|30|17|PM|second|afternoon|| +59418|AAAAAAAALBIOAAAA|59418|16|30|18|PM|second|afternoon|| +59419|AAAAAAAAMBIOAAAA|59419|16|30|19|PM|second|afternoon|| +59420|AAAAAAAANBIOAAAA|59420|16|30|20|PM|second|afternoon|| +59421|AAAAAAAAOBIOAAAA|59421|16|30|21|PM|second|afternoon|| +59422|AAAAAAAAPBIOAAAA|59422|16|30|22|PM|second|afternoon|| +59423|AAAAAAAAACIOAAAA|59423|16|30|23|PM|second|afternoon|| +59424|AAAAAAAABCIOAAAA|59424|16|30|24|PM|second|afternoon|| +59425|AAAAAAAACCIOAAAA|59425|16|30|25|PM|second|afternoon|| +59426|AAAAAAAADCIOAAAA|59426|16|30|26|PM|second|afternoon|| +59427|AAAAAAAAECIOAAAA|59427|16|30|27|PM|second|afternoon|| +59428|AAAAAAAAFCIOAAAA|59428|16|30|28|PM|second|afternoon|| +59429|AAAAAAAAGCIOAAAA|59429|16|30|29|PM|second|afternoon|| +59430|AAAAAAAAHCIOAAAA|59430|16|30|30|PM|second|afternoon|| +59431|AAAAAAAAICIOAAAA|59431|16|30|31|PM|second|afternoon|| +59432|AAAAAAAAJCIOAAAA|59432|16|30|32|PM|second|afternoon|| +59433|AAAAAAAAKCIOAAAA|59433|16|30|33|PM|second|afternoon|| +59434|AAAAAAAALCIOAAAA|59434|16|30|34|PM|second|afternoon|| +59435|AAAAAAAAMCIOAAAA|59435|16|30|35|PM|second|afternoon|| +59436|AAAAAAAANCIOAAAA|59436|16|30|36|PM|second|afternoon|| +59437|AAAAAAAAOCIOAAAA|59437|16|30|37|PM|second|afternoon|| +59438|AAAAAAAAPCIOAAAA|59438|16|30|38|PM|second|afternoon|| +59439|AAAAAAAAADIOAAAA|59439|16|30|39|PM|second|afternoon|| +59440|AAAAAAAABDIOAAAA|59440|16|30|40|PM|second|afternoon|| +59441|AAAAAAAACDIOAAAA|59441|16|30|41|PM|second|afternoon|| +59442|AAAAAAAADDIOAAAA|59442|16|30|42|PM|second|afternoon|| +59443|AAAAAAAAEDIOAAAA|59443|16|30|43|PM|second|afternoon|| +59444|AAAAAAAAFDIOAAAA|59444|16|30|44|PM|second|afternoon|| +59445|AAAAAAAAGDIOAAAA|59445|16|30|45|PM|second|afternoon|| +59446|AAAAAAAAHDIOAAAA|59446|16|30|46|PM|second|afternoon|| +59447|AAAAAAAAIDIOAAAA|59447|16|30|47|PM|second|afternoon|| +59448|AAAAAAAAJDIOAAAA|59448|16|30|48|PM|second|afternoon|| +59449|AAAAAAAAKDIOAAAA|59449|16|30|49|PM|second|afternoon|| +59450|AAAAAAAALDIOAAAA|59450|16|30|50|PM|second|afternoon|| +59451|AAAAAAAAMDIOAAAA|59451|16|30|51|PM|second|afternoon|| +59452|AAAAAAAANDIOAAAA|59452|16|30|52|PM|second|afternoon|| +59453|AAAAAAAAODIOAAAA|59453|16|30|53|PM|second|afternoon|| +59454|AAAAAAAAPDIOAAAA|59454|16|30|54|PM|second|afternoon|| +59455|AAAAAAAAAEIOAAAA|59455|16|30|55|PM|second|afternoon|| +59456|AAAAAAAABEIOAAAA|59456|16|30|56|PM|second|afternoon|| +59457|AAAAAAAACEIOAAAA|59457|16|30|57|PM|second|afternoon|| +59458|AAAAAAAADEIOAAAA|59458|16|30|58|PM|second|afternoon|| +59459|AAAAAAAAEEIOAAAA|59459|16|30|59|PM|second|afternoon|| +59460|AAAAAAAAFEIOAAAA|59460|16|31|0|PM|second|afternoon|| +59461|AAAAAAAAGEIOAAAA|59461|16|31|1|PM|second|afternoon|| +59462|AAAAAAAAHEIOAAAA|59462|16|31|2|PM|second|afternoon|| +59463|AAAAAAAAIEIOAAAA|59463|16|31|3|PM|second|afternoon|| +59464|AAAAAAAAJEIOAAAA|59464|16|31|4|PM|second|afternoon|| +59465|AAAAAAAAKEIOAAAA|59465|16|31|5|PM|second|afternoon|| +59466|AAAAAAAALEIOAAAA|59466|16|31|6|PM|second|afternoon|| +59467|AAAAAAAAMEIOAAAA|59467|16|31|7|PM|second|afternoon|| +59468|AAAAAAAANEIOAAAA|59468|16|31|8|PM|second|afternoon|| +59469|AAAAAAAAOEIOAAAA|59469|16|31|9|PM|second|afternoon|| +59470|AAAAAAAAPEIOAAAA|59470|16|31|10|PM|second|afternoon|| +59471|AAAAAAAAAFIOAAAA|59471|16|31|11|PM|second|afternoon|| +59472|AAAAAAAABFIOAAAA|59472|16|31|12|PM|second|afternoon|| +59473|AAAAAAAACFIOAAAA|59473|16|31|13|PM|second|afternoon|| +59474|AAAAAAAADFIOAAAA|59474|16|31|14|PM|second|afternoon|| +59475|AAAAAAAAEFIOAAAA|59475|16|31|15|PM|second|afternoon|| +59476|AAAAAAAAFFIOAAAA|59476|16|31|16|PM|second|afternoon|| +59477|AAAAAAAAGFIOAAAA|59477|16|31|17|PM|second|afternoon|| +59478|AAAAAAAAHFIOAAAA|59478|16|31|18|PM|second|afternoon|| +59479|AAAAAAAAIFIOAAAA|59479|16|31|19|PM|second|afternoon|| +59480|AAAAAAAAJFIOAAAA|59480|16|31|20|PM|second|afternoon|| +59481|AAAAAAAAKFIOAAAA|59481|16|31|21|PM|second|afternoon|| +59482|AAAAAAAALFIOAAAA|59482|16|31|22|PM|second|afternoon|| +59483|AAAAAAAAMFIOAAAA|59483|16|31|23|PM|second|afternoon|| +59484|AAAAAAAANFIOAAAA|59484|16|31|24|PM|second|afternoon|| +59485|AAAAAAAAOFIOAAAA|59485|16|31|25|PM|second|afternoon|| +59486|AAAAAAAAPFIOAAAA|59486|16|31|26|PM|second|afternoon|| +59487|AAAAAAAAAGIOAAAA|59487|16|31|27|PM|second|afternoon|| +59488|AAAAAAAABGIOAAAA|59488|16|31|28|PM|second|afternoon|| +59489|AAAAAAAACGIOAAAA|59489|16|31|29|PM|second|afternoon|| +59490|AAAAAAAADGIOAAAA|59490|16|31|30|PM|second|afternoon|| +59491|AAAAAAAAEGIOAAAA|59491|16|31|31|PM|second|afternoon|| +59492|AAAAAAAAFGIOAAAA|59492|16|31|32|PM|second|afternoon|| +59493|AAAAAAAAGGIOAAAA|59493|16|31|33|PM|second|afternoon|| +59494|AAAAAAAAHGIOAAAA|59494|16|31|34|PM|second|afternoon|| +59495|AAAAAAAAIGIOAAAA|59495|16|31|35|PM|second|afternoon|| +59496|AAAAAAAAJGIOAAAA|59496|16|31|36|PM|second|afternoon|| +59497|AAAAAAAAKGIOAAAA|59497|16|31|37|PM|second|afternoon|| +59498|AAAAAAAALGIOAAAA|59498|16|31|38|PM|second|afternoon|| +59499|AAAAAAAAMGIOAAAA|59499|16|31|39|PM|second|afternoon|| +59500|AAAAAAAANGIOAAAA|59500|16|31|40|PM|second|afternoon|| +59501|AAAAAAAAOGIOAAAA|59501|16|31|41|PM|second|afternoon|| +59502|AAAAAAAAPGIOAAAA|59502|16|31|42|PM|second|afternoon|| +59503|AAAAAAAAAHIOAAAA|59503|16|31|43|PM|second|afternoon|| +59504|AAAAAAAABHIOAAAA|59504|16|31|44|PM|second|afternoon|| +59505|AAAAAAAACHIOAAAA|59505|16|31|45|PM|second|afternoon|| +59506|AAAAAAAADHIOAAAA|59506|16|31|46|PM|second|afternoon|| +59507|AAAAAAAAEHIOAAAA|59507|16|31|47|PM|second|afternoon|| +59508|AAAAAAAAFHIOAAAA|59508|16|31|48|PM|second|afternoon|| +59509|AAAAAAAAGHIOAAAA|59509|16|31|49|PM|second|afternoon|| +59510|AAAAAAAAHHIOAAAA|59510|16|31|50|PM|second|afternoon|| +59511|AAAAAAAAIHIOAAAA|59511|16|31|51|PM|second|afternoon|| +59512|AAAAAAAAJHIOAAAA|59512|16|31|52|PM|second|afternoon|| +59513|AAAAAAAAKHIOAAAA|59513|16|31|53|PM|second|afternoon|| +59514|AAAAAAAALHIOAAAA|59514|16|31|54|PM|second|afternoon|| +59515|AAAAAAAAMHIOAAAA|59515|16|31|55|PM|second|afternoon|| +59516|AAAAAAAANHIOAAAA|59516|16|31|56|PM|second|afternoon|| +59517|AAAAAAAAOHIOAAAA|59517|16|31|57|PM|second|afternoon|| +59518|AAAAAAAAPHIOAAAA|59518|16|31|58|PM|second|afternoon|| +59519|AAAAAAAAAIIOAAAA|59519|16|31|59|PM|second|afternoon|| +59520|AAAAAAAABIIOAAAA|59520|16|32|0|PM|second|afternoon|| +59521|AAAAAAAACIIOAAAA|59521|16|32|1|PM|second|afternoon|| +59522|AAAAAAAADIIOAAAA|59522|16|32|2|PM|second|afternoon|| +59523|AAAAAAAAEIIOAAAA|59523|16|32|3|PM|second|afternoon|| +59524|AAAAAAAAFIIOAAAA|59524|16|32|4|PM|second|afternoon|| +59525|AAAAAAAAGIIOAAAA|59525|16|32|5|PM|second|afternoon|| +59526|AAAAAAAAHIIOAAAA|59526|16|32|6|PM|second|afternoon|| +59527|AAAAAAAAIIIOAAAA|59527|16|32|7|PM|second|afternoon|| +59528|AAAAAAAAJIIOAAAA|59528|16|32|8|PM|second|afternoon|| +59529|AAAAAAAAKIIOAAAA|59529|16|32|9|PM|second|afternoon|| +59530|AAAAAAAALIIOAAAA|59530|16|32|10|PM|second|afternoon|| +59531|AAAAAAAAMIIOAAAA|59531|16|32|11|PM|second|afternoon|| +59532|AAAAAAAANIIOAAAA|59532|16|32|12|PM|second|afternoon|| +59533|AAAAAAAAOIIOAAAA|59533|16|32|13|PM|second|afternoon|| +59534|AAAAAAAAPIIOAAAA|59534|16|32|14|PM|second|afternoon|| +59535|AAAAAAAAAJIOAAAA|59535|16|32|15|PM|second|afternoon|| +59536|AAAAAAAABJIOAAAA|59536|16|32|16|PM|second|afternoon|| +59537|AAAAAAAACJIOAAAA|59537|16|32|17|PM|second|afternoon|| +59538|AAAAAAAADJIOAAAA|59538|16|32|18|PM|second|afternoon|| +59539|AAAAAAAAEJIOAAAA|59539|16|32|19|PM|second|afternoon|| +59540|AAAAAAAAFJIOAAAA|59540|16|32|20|PM|second|afternoon|| +59541|AAAAAAAAGJIOAAAA|59541|16|32|21|PM|second|afternoon|| +59542|AAAAAAAAHJIOAAAA|59542|16|32|22|PM|second|afternoon|| +59543|AAAAAAAAIJIOAAAA|59543|16|32|23|PM|second|afternoon|| +59544|AAAAAAAAJJIOAAAA|59544|16|32|24|PM|second|afternoon|| +59545|AAAAAAAAKJIOAAAA|59545|16|32|25|PM|second|afternoon|| +59546|AAAAAAAALJIOAAAA|59546|16|32|26|PM|second|afternoon|| +59547|AAAAAAAAMJIOAAAA|59547|16|32|27|PM|second|afternoon|| +59548|AAAAAAAANJIOAAAA|59548|16|32|28|PM|second|afternoon|| +59549|AAAAAAAAOJIOAAAA|59549|16|32|29|PM|second|afternoon|| +59550|AAAAAAAAPJIOAAAA|59550|16|32|30|PM|second|afternoon|| +59551|AAAAAAAAAKIOAAAA|59551|16|32|31|PM|second|afternoon|| +59552|AAAAAAAABKIOAAAA|59552|16|32|32|PM|second|afternoon|| +59553|AAAAAAAACKIOAAAA|59553|16|32|33|PM|second|afternoon|| +59554|AAAAAAAADKIOAAAA|59554|16|32|34|PM|second|afternoon|| +59555|AAAAAAAAEKIOAAAA|59555|16|32|35|PM|second|afternoon|| +59556|AAAAAAAAFKIOAAAA|59556|16|32|36|PM|second|afternoon|| +59557|AAAAAAAAGKIOAAAA|59557|16|32|37|PM|second|afternoon|| +59558|AAAAAAAAHKIOAAAA|59558|16|32|38|PM|second|afternoon|| +59559|AAAAAAAAIKIOAAAA|59559|16|32|39|PM|second|afternoon|| +59560|AAAAAAAAJKIOAAAA|59560|16|32|40|PM|second|afternoon|| +59561|AAAAAAAAKKIOAAAA|59561|16|32|41|PM|second|afternoon|| +59562|AAAAAAAALKIOAAAA|59562|16|32|42|PM|second|afternoon|| +59563|AAAAAAAAMKIOAAAA|59563|16|32|43|PM|second|afternoon|| +59564|AAAAAAAANKIOAAAA|59564|16|32|44|PM|second|afternoon|| +59565|AAAAAAAAOKIOAAAA|59565|16|32|45|PM|second|afternoon|| +59566|AAAAAAAAPKIOAAAA|59566|16|32|46|PM|second|afternoon|| +59567|AAAAAAAAALIOAAAA|59567|16|32|47|PM|second|afternoon|| +59568|AAAAAAAABLIOAAAA|59568|16|32|48|PM|second|afternoon|| +59569|AAAAAAAACLIOAAAA|59569|16|32|49|PM|second|afternoon|| +59570|AAAAAAAADLIOAAAA|59570|16|32|50|PM|second|afternoon|| +59571|AAAAAAAAELIOAAAA|59571|16|32|51|PM|second|afternoon|| +59572|AAAAAAAAFLIOAAAA|59572|16|32|52|PM|second|afternoon|| +59573|AAAAAAAAGLIOAAAA|59573|16|32|53|PM|second|afternoon|| +59574|AAAAAAAAHLIOAAAA|59574|16|32|54|PM|second|afternoon|| +59575|AAAAAAAAILIOAAAA|59575|16|32|55|PM|second|afternoon|| +59576|AAAAAAAAJLIOAAAA|59576|16|32|56|PM|second|afternoon|| +59577|AAAAAAAAKLIOAAAA|59577|16|32|57|PM|second|afternoon|| +59578|AAAAAAAALLIOAAAA|59578|16|32|58|PM|second|afternoon|| +59579|AAAAAAAAMLIOAAAA|59579|16|32|59|PM|second|afternoon|| +59580|AAAAAAAANLIOAAAA|59580|16|33|0|PM|second|afternoon|| +59581|AAAAAAAAOLIOAAAA|59581|16|33|1|PM|second|afternoon|| +59582|AAAAAAAAPLIOAAAA|59582|16|33|2|PM|second|afternoon|| +59583|AAAAAAAAAMIOAAAA|59583|16|33|3|PM|second|afternoon|| +59584|AAAAAAAABMIOAAAA|59584|16|33|4|PM|second|afternoon|| +59585|AAAAAAAACMIOAAAA|59585|16|33|5|PM|second|afternoon|| +59586|AAAAAAAADMIOAAAA|59586|16|33|6|PM|second|afternoon|| +59587|AAAAAAAAEMIOAAAA|59587|16|33|7|PM|second|afternoon|| +59588|AAAAAAAAFMIOAAAA|59588|16|33|8|PM|second|afternoon|| +59589|AAAAAAAAGMIOAAAA|59589|16|33|9|PM|second|afternoon|| +59590|AAAAAAAAHMIOAAAA|59590|16|33|10|PM|second|afternoon|| +59591|AAAAAAAAIMIOAAAA|59591|16|33|11|PM|second|afternoon|| +59592|AAAAAAAAJMIOAAAA|59592|16|33|12|PM|second|afternoon|| +59593|AAAAAAAAKMIOAAAA|59593|16|33|13|PM|second|afternoon|| +59594|AAAAAAAALMIOAAAA|59594|16|33|14|PM|second|afternoon|| +59595|AAAAAAAAMMIOAAAA|59595|16|33|15|PM|second|afternoon|| +59596|AAAAAAAANMIOAAAA|59596|16|33|16|PM|second|afternoon|| +59597|AAAAAAAAOMIOAAAA|59597|16|33|17|PM|second|afternoon|| +59598|AAAAAAAAPMIOAAAA|59598|16|33|18|PM|second|afternoon|| +59599|AAAAAAAAANIOAAAA|59599|16|33|19|PM|second|afternoon|| +59600|AAAAAAAABNIOAAAA|59600|16|33|20|PM|second|afternoon|| +59601|AAAAAAAACNIOAAAA|59601|16|33|21|PM|second|afternoon|| +59602|AAAAAAAADNIOAAAA|59602|16|33|22|PM|second|afternoon|| +59603|AAAAAAAAENIOAAAA|59603|16|33|23|PM|second|afternoon|| +59604|AAAAAAAAFNIOAAAA|59604|16|33|24|PM|second|afternoon|| +59605|AAAAAAAAGNIOAAAA|59605|16|33|25|PM|second|afternoon|| +59606|AAAAAAAAHNIOAAAA|59606|16|33|26|PM|second|afternoon|| +59607|AAAAAAAAINIOAAAA|59607|16|33|27|PM|second|afternoon|| +59608|AAAAAAAAJNIOAAAA|59608|16|33|28|PM|second|afternoon|| +59609|AAAAAAAAKNIOAAAA|59609|16|33|29|PM|second|afternoon|| +59610|AAAAAAAALNIOAAAA|59610|16|33|30|PM|second|afternoon|| +59611|AAAAAAAAMNIOAAAA|59611|16|33|31|PM|second|afternoon|| +59612|AAAAAAAANNIOAAAA|59612|16|33|32|PM|second|afternoon|| +59613|AAAAAAAAONIOAAAA|59613|16|33|33|PM|second|afternoon|| +59614|AAAAAAAAPNIOAAAA|59614|16|33|34|PM|second|afternoon|| +59615|AAAAAAAAAOIOAAAA|59615|16|33|35|PM|second|afternoon|| +59616|AAAAAAAABOIOAAAA|59616|16|33|36|PM|second|afternoon|| +59617|AAAAAAAACOIOAAAA|59617|16|33|37|PM|second|afternoon|| +59618|AAAAAAAADOIOAAAA|59618|16|33|38|PM|second|afternoon|| +59619|AAAAAAAAEOIOAAAA|59619|16|33|39|PM|second|afternoon|| +59620|AAAAAAAAFOIOAAAA|59620|16|33|40|PM|second|afternoon|| +59621|AAAAAAAAGOIOAAAA|59621|16|33|41|PM|second|afternoon|| +59622|AAAAAAAAHOIOAAAA|59622|16|33|42|PM|second|afternoon|| +59623|AAAAAAAAIOIOAAAA|59623|16|33|43|PM|second|afternoon|| +59624|AAAAAAAAJOIOAAAA|59624|16|33|44|PM|second|afternoon|| +59625|AAAAAAAAKOIOAAAA|59625|16|33|45|PM|second|afternoon|| +59626|AAAAAAAALOIOAAAA|59626|16|33|46|PM|second|afternoon|| +59627|AAAAAAAAMOIOAAAA|59627|16|33|47|PM|second|afternoon|| +59628|AAAAAAAANOIOAAAA|59628|16|33|48|PM|second|afternoon|| +59629|AAAAAAAAOOIOAAAA|59629|16|33|49|PM|second|afternoon|| +59630|AAAAAAAAPOIOAAAA|59630|16|33|50|PM|second|afternoon|| +59631|AAAAAAAAAPIOAAAA|59631|16|33|51|PM|second|afternoon|| +59632|AAAAAAAABPIOAAAA|59632|16|33|52|PM|second|afternoon|| +59633|AAAAAAAACPIOAAAA|59633|16|33|53|PM|second|afternoon|| +59634|AAAAAAAADPIOAAAA|59634|16|33|54|PM|second|afternoon|| +59635|AAAAAAAAEPIOAAAA|59635|16|33|55|PM|second|afternoon|| +59636|AAAAAAAAFPIOAAAA|59636|16|33|56|PM|second|afternoon|| +59637|AAAAAAAAGPIOAAAA|59637|16|33|57|PM|second|afternoon|| +59638|AAAAAAAAHPIOAAAA|59638|16|33|58|PM|second|afternoon|| +59639|AAAAAAAAIPIOAAAA|59639|16|33|59|PM|second|afternoon|| +59640|AAAAAAAAJPIOAAAA|59640|16|34|0|PM|second|afternoon|| +59641|AAAAAAAAKPIOAAAA|59641|16|34|1|PM|second|afternoon|| +59642|AAAAAAAALPIOAAAA|59642|16|34|2|PM|second|afternoon|| +59643|AAAAAAAAMPIOAAAA|59643|16|34|3|PM|second|afternoon|| +59644|AAAAAAAANPIOAAAA|59644|16|34|4|PM|second|afternoon|| +59645|AAAAAAAAOPIOAAAA|59645|16|34|5|PM|second|afternoon|| +59646|AAAAAAAAPPIOAAAA|59646|16|34|6|PM|second|afternoon|| +59647|AAAAAAAAAAJOAAAA|59647|16|34|7|PM|second|afternoon|| +59648|AAAAAAAABAJOAAAA|59648|16|34|8|PM|second|afternoon|| +59649|AAAAAAAACAJOAAAA|59649|16|34|9|PM|second|afternoon|| +59650|AAAAAAAADAJOAAAA|59650|16|34|10|PM|second|afternoon|| +59651|AAAAAAAAEAJOAAAA|59651|16|34|11|PM|second|afternoon|| +59652|AAAAAAAAFAJOAAAA|59652|16|34|12|PM|second|afternoon|| +59653|AAAAAAAAGAJOAAAA|59653|16|34|13|PM|second|afternoon|| +59654|AAAAAAAAHAJOAAAA|59654|16|34|14|PM|second|afternoon|| +59655|AAAAAAAAIAJOAAAA|59655|16|34|15|PM|second|afternoon|| +59656|AAAAAAAAJAJOAAAA|59656|16|34|16|PM|second|afternoon|| +59657|AAAAAAAAKAJOAAAA|59657|16|34|17|PM|second|afternoon|| +59658|AAAAAAAALAJOAAAA|59658|16|34|18|PM|second|afternoon|| +59659|AAAAAAAAMAJOAAAA|59659|16|34|19|PM|second|afternoon|| +59660|AAAAAAAANAJOAAAA|59660|16|34|20|PM|second|afternoon|| +59661|AAAAAAAAOAJOAAAA|59661|16|34|21|PM|second|afternoon|| +59662|AAAAAAAAPAJOAAAA|59662|16|34|22|PM|second|afternoon|| +59663|AAAAAAAAABJOAAAA|59663|16|34|23|PM|second|afternoon|| +59664|AAAAAAAABBJOAAAA|59664|16|34|24|PM|second|afternoon|| +59665|AAAAAAAACBJOAAAA|59665|16|34|25|PM|second|afternoon|| +59666|AAAAAAAADBJOAAAA|59666|16|34|26|PM|second|afternoon|| +59667|AAAAAAAAEBJOAAAA|59667|16|34|27|PM|second|afternoon|| +59668|AAAAAAAAFBJOAAAA|59668|16|34|28|PM|second|afternoon|| +59669|AAAAAAAAGBJOAAAA|59669|16|34|29|PM|second|afternoon|| +59670|AAAAAAAAHBJOAAAA|59670|16|34|30|PM|second|afternoon|| +59671|AAAAAAAAIBJOAAAA|59671|16|34|31|PM|second|afternoon|| +59672|AAAAAAAAJBJOAAAA|59672|16|34|32|PM|second|afternoon|| +59673|AAAAAAAAKBJOAAAA|59673|16|34|33|PM|second|afternoon|| +59674|AAAAAAAALBJOAAAA|59674|16|34|34|PM|second|afternoon|| +59675|AAAAAAAAMBJOAAAA|59675|16|34|35|PM|second|afternoon|| +59676|AAAAAAAANBJOAAAA|59676|16|34|36|PM|second|afternoon|| +59677|AAAAAAAAOBJOAAAA|59677|16|34|37|PM|second|afternoon|| +59678|AAAAAAAAPBJOAAAA|59678|16|34|38|PM|second|afternoon|| +59679|AAAAAAAAACJOAAAA|59679|16|34|39|PM|second|afternoon|| +59680|AAAAAAAABCJOAAAA|59680|16|34|40|PM|second|afternoon|| +59681|AAAAAAAACCJOAAAA|59681|16|34|41|PM|second|afternoon|| +59682|AAAAAAAADCJOAAAA|59682|16|34|42|PM|second|afternoon|| +59683|AAAAAAAAECJOAAAA|59683|16|34|43|PM|second|afternoon|| +59684|AAAAAAAAFCJOAAAA|59684|16|34|44|PM|second|afternoon|| +59685|AAAAAAAAGCJOAAAA|59685|16|34|45|PM|second|afternoon|| +59686|AAAAAAAAHCJOAAAA|59686|16|34|46|PM|second|afternoon|| +59687|AAAAAAAAICJOAAAA|59687|16|34|47|PM|second|afternoon|| +59688|AAAAAAAAJCJOAAAA|59688|16|34|48|PM|second|afternoon|| +59689|AAAAAAAAKCJOAAAA|59689|16|34|49|PM|second|afternoon|| +59690|AAAAAAAALCJOAAAA|59690|16|34|50|PM|second|afternoon|| +59691|AAAAAAAAMCJOAAAA|59691|16|34|51|PM|second|afternoon|| +59692|AAAAAAAANCJOAAAA|59692|16|34|52|PM|second|afternoon|| +59693|AAAAAAAAOCJOAAAA|59693|16|34|53|PM|second|afternoon|| +59694|AAAAAAAAPCJOAAAA|59694|16|34|54|PM|second|afternoon|| +59695|AAAAAAAAADJOAAAA|59695|16|34|55|PM|second|afternoon|| +59696|AAAAAAAABDJOAAAA|59696|16|34|56|PM|second|afternoon|| +59697|AAAAAAAACDJOAAAA|59697|16|34|57|PM|second|afternoon|| +59698|AAAAAAAADDJOAAAA|59698|16|34|58|PM|second|afternoon|| +59699|AAAAAAAAEDJOAAAA|59699|16|34|59|PM|second|afternoon|| +59700|AAAAAAAAFDJOAAAA|59700|16|35|0|PM|second|afternoon|| +59701|AAAAAAAAGDJOAAAA|59701|16|35|1|PM|second|afternoon|| +59702|AAAAAAAAHDJOAAAA|59702|16|35|2|PM|second|afternoon|| +59703|AAAAAAAAIDJOAAAA|59703|16|35|3|PM|second|afternoon|| +59704|AAAAAAAAJDJOAAAA|59704|16|35|4|PM|second|afternoon|| +59705|AAAAAAAAKDJOAAAA|59705|16|35|5|PM|second|afternoon|| +59706|AAAAAAAALDJOAAAA|59706|16|35|6|PM|second|afternoon|| +59707|AAAAAAAAMDJOAAAA|59707|16|35|7|PM|second|afternoon|| +59708|AAAAAAAANDJOAAAA|59708|16|35|8|PM|second|afternoon|| +59709|AAAAAAAAODJOAAAA|59709|16|35|9|PM|second|afternoon|| +59710|AAAAAAAAPDJOAAAA|59710|16|35|10|PM|second|afternoon|| +59711|AAAAAAAAAEJOAAAA|59711|16|35|11|PM|second|afternoon|| +59712|AAAAAAAABEJOAAAA|59712|16|35|12|PM|second|afternoon|| +59713|AAAAAAAACEJOAAAA|59713|16|35|13|PM|second|afternoon|| +59714|AAAAAAAADEJOAAAA|59714|16|35|14|PM|second|afternoon|| +59715|AAAAAAAAEEJOAAAA|59715|16|35|15|PM|second|afternoon|| +59716|AAAAAAAAFEJOAAAA|59716|16|35|16|PM|second|afternoon|| +59717|AAAAAAAAGEJOAAAA|59717|16|35|17|PM|second|afternoon|| +59718|AAAAAAAAHEJOAAAA|59718|16|35|18|PM|second|afternoon|| +59719|AAAAAAAAIEJOAAAA|59719|16|35|19|PM|second|afternoon|| +59720|AAAAAAAAJEJOAAAA|59720|16|35|20|PM|second|afternoon|| +59721|AAAAAAAAKEJOAAAA|59721|16|35|21|PM|second|afternoon|| +59722|AAAAAAAALEJOAAAA|59722|16|35|22|PM|second|afternoon|| +59723|AAAAAAAAMEJOAAAA|59723|16|35|23|PM|second|afternoon|| +59724|AAAAAAAANEJOAAAA|59724|16|35|24|PM|second|afternoon|| +59725|AAAAAAAAOEJOAAAA|59725|16|35|25|PM|second|afternoon|| +59726|AAAAAAAAPEJOAAAA|59726|16|35|26|PM|second|afternoon|| +59727|AAAAAAAAAFJOAAAA|59727|16|35|27|PM|second|afternoon|| +59728|AAAAAAAABFJOAAAA|59728|16|35|28|PM|second|afternoon|| +59729|AAAAAAAACFJOAAAA|59729|16|35|29|PM|second|afternoon|| +59730|AAAAAAAADFJOAAAA|59730|16|35|30|PM|second|afternoon|| +59731|AAAAAAAAEFJOAAAA|59731|16|35|31|PM|second|afternoon|| +59732|AAAAAAAAFFJOAAAA|59732|16|35|32|PM|second|afternoon|| +59733|AAAAAAAAGFJOAAAA|59733|16|35|33|PM|second|afternoon|| +59734|AAAAAAAAHFJOAAAA|59734|16|35|34|PM|second|afternoon|| +59735|AAAAAAAAIFJOAAAA|59735|16|35|35|PM|second|afternoon|| +59736|AAAAAAAAJFJOAAAA|59736|16|35|36|PM|second|afternoon|| +59737|AAAAAAAAKFJOAAAA|59737|16|35|37|PM|second|afternoon|| +59738|AAAAAAAALFJOAAAA|59738|16|35|38|PM|second|afternoon|| +59739|AAAAAAAAMFJOAAAA|59739|16|35|39|PM|second|afternoon|| +59740|AAAAAAAANFJOAAAA|59740|16|35|40|PM|second|afternoon|| +59741|AAAAAAAAOFJOAAAA|59741|16|35|41|PM|second|afternoon|| +59742|AAAAAAAAPFJOAAAA|59742|16|35|42|PM|second|afternoon|| +59743|AAAAAAAAAGJOAAAA|59743|16|35|43|PM|second|afternoon|| +59744|AAAAAAAABGJOAAAA|59744|16|35|44|PM|second|afternoon|| +59745|AAAAAAAACGJOAAAA|59745|16|35|45|PM|second|afternoon|| +59746|AAAAAAAADGJOAAAA|59746|16|35|46|PM|second|afternoon|| +59747|AAAAAAAAEGJOAAAA|59747|16|35|47|PM|second|afternoon|| +59748|AAAAAAAAFGJOAAAA|59748|16|35|48|PM|second|afternoon|| +59749|AAAAAAAAGGJOAAAA|59749|16|35|49|PM|second|afternoon|| +59750|AAAAAAAAHGJOAAAA|59750|16|35|50|PM|second|afternoon|| +59751|AAAAAAAAIGJOAAAA|59751|16|35|51|PM|second|afternoon|| +59752|AAAAAAAAJGJOAAAA|59752|16|35|52|PM|second|afternoon|| +59753|AAAAAAAAKGJOAAAA|59753|16|35|53|PM|second|afternoon|| +59754|AAAAAAAALGJOAAAA|59754|16|35|54|PM|second|afternoon|| +59755|AAAAAAAAMGJOAAAA|59755|16|35|55|PM|second|afternoon|| +59756|AAAAAAAANGJOAAAA|59756|16|35|56|PM|second|afternoon|| +59757|AAAAAAAAOGJOAAAA|59757|16|35|57|PM|second|afternoon|| +59758|AAAAAAAAPGJOAAAA|59758|16|35|58|PM|second|afternoon|| +59759|AAAAAAAAAHJOAAAA|59759|16|35|59|PM|second|afternoon|| +59760|AAAAAAAABHJOAAAA|59760|16|36|0|PM|second|afternoon|| +59761|AAAAAAAACHJOAAAA|59761|16|36|1|PM|second|afternoon|| +59762|AAAAAAAADHJOAAAA|59762|16|36|2|PM|second|afternoon|| +59763|AAAAAAAAEHJOAAAA|59763|16|36|3|PM|second|afternoon|| +59764|AAAAAAAAFHJOAAAA|59764|16|36|4|PM|second|afternoon|| +59765|AAAAAAAAGHJOAAAA|59765|16|36|5|PM|second|afternoon|| +59766|AAAAAAAAHHJOAAAA|59766|16|36|6|PM|second|afternoon|| +59767|AAAAAAAAIHJOAAAA|59767|16|36|7|PM|second|afternoon|| +59768|AAAAAAAAJHJOAAAA|59768|16|36|8|PM|second|afternoon|| +59769|AAAAAAAAKHJOAAAA|59769|16|36|9|PM|second|afternoon|| +59770|AAAAAAAALHJOAAAA|59770|16|36|10|PM|second|afternoon|| +59771|AAAAAAAAMHJOAAAA|59771|16|36|11|PM|second|afternoon|| +59772|AAAAAAAANHJOAAAA|59772|16|36|12|PM|second|afternoon|| +59773|AAAAAAAAOHJOAAAA|59773|16|36|13|PM|second|afternoon|| +59774|AAAAAAAAPHJOAAAA|59774|16|36|14|PM|second|afternoon|| +59775|AAAAAAAAAIJOAAAA|59775|16|36|15|PM|second|afternoon|| +59776|AAAAAAAABIJOAAAA|59776|16|36|16|PM|second|afternoon|| +59777|AAAAAAAACIJOAAAA|59777|16|36|17|PM|second|afternoon|| +59778|AAAAAAAADIJOAAAA|59778|16|36|18|PM|second|afternoon|| +59779|AAAAAAAAEIJOAAAA|59779|16|36|19|PM|second|afternoon|| +59780|AAAAAAAAFIJOAAAA|59780|16|36|20|PM|second|afternoon|| +59781|AAAAAAAAGIJOAAAA|59781|16|36|21|PM|second|afternoon|| +59782|AAAAAAAAHIJOAAAA|59782|16|36|22|PM|second|afternoon|| +59783|AAAAAAAAIIJOAAAA|59783|16|36|23|PM|second|afternoon|| +59784|AAAAAAAAJIJOAAAA|59784|16|36|24|PM|second|afternoon|| +59785|AAAAAAAAKIJOAAAA|59785|16|36|25|PM|second|afternoon|| +59786|AAAAAAAALIJOAAAA|59786|16|36|26|PM|second|afternoon|| +59787|AAAAAAAAMIJOAAAA|59787|16|36|27|PM|second|afternoon|| +59788|AAAAAAAANIJOAAAA|59788|16|36|28|PM|second|afternoon|| +59789|AAAAAAAAOIJOAAAA|59789|16|36|29|PM|second|afternoon|| +59790|AAAAAAAAPIJOAAAA|59790|16|36|30|PM|second|afternoon|| +59791|AAAAAAAAAJJOAAAA|59791|16|36|31|PM|second|afternoon|| +59792|AAAAAAAABJJOAAAA|59792|16|36|32|PM|second|afternoon|| +59793|AAAAAAAACJJOAAAA|59793|16|36|33|PM|second|afternoon|| +59794|AAAAAAAADJJOAAAA|59794|16|36|34|PM|second|afternoon|| +59795|AAAAAAAAEJJOAAAA|59795|16|36|35|PM|second|afternoon|| +59796|AAAAAAAAFJJOAAAA|59796|16|36|36|PM|second|afternoon|| +59797|AAAAAAAAGJJOAAAA|59797|16|36|37|PM|second|afternoon|| +59798|AAAAAAAAHJJOAAAA|59798|16|36|38|PM|second|afternoon|| +59799|AAAAAAAAIJJOAAAA|59799|16|36|39|PM|second|afternoon|| +59800|AAAAAAAAJJJOAAAA|59800|16|36|40|PM|second|afternoon|| +59801|AAAAAAAAKJJOAAAA|59801|16|36|41|PM|second|afternoon|| +59802|AAAAAAAALJJOAAAA|59802|16|36|42|PM|second|afternoon|| +59803|AAAAAAAAMJJOAAAA|59803|16|36|43|PM|second|afternoon|| +59804|AAAAAAAANJJOAAAA|59804|16|36|44|PM|second|afternoon|| +59805|AAAAAAAAOJJOAAAA|59805|16|36|45|PM|second|afternoon|| +59806|AAAAAAAAPJJOAAAA|59806|16|36|46|PM|second|afternoon|| +59807|AAAAAAAAAKJOAAAA|59807|16|36|47|PM|second|afternoon|| +59808|AAAAAAAABKJOAAAA|59808|16|36|48|PM|second|afternoon|| +59809|AAAAAAAACKJOAAAA|59809|16|36|49|PM|second|afternoon|| +59810|AAAAAAAADKJOAAAA|59810|16|36|50|PM|second|afternoon|| +59811|AAAAAAAAEKJOAAAA|59811|16|36|51|PM|second|afternoon|| +59812|AAAAAAAAFKJOAAAA|59812|16|36|52|PM|second|afternoon|| +59813|AAAAAAAAGKJOAAAA|59813|16|36|53|PM|second|afternoon|| +59814|AAAAAAAAHKJOAAAA|59814|16|36|54|PM|second|afternoon|| +59815|AAAAAAAAIKJOAAAA|59815|16|36|55|PM|second|afternoon|| +59816|AAAAAAAAJKJOAAAA|59816|16|36|56|PM|second|afternoon|| +59817|AAAAAAAAKKJOAAAA|59817|16|36|57|PM|second|afternoon|| +59818|AAAAAAAALKJOAAAA|59818|16|36|58|PM|second|afternoon|| +59819|AAAAAAAAMKJOAAAA|59819|16|36|59|PM|second|afternoon|| +59820|AAAAAAAANKJOAAAA|59820|16|37|0|PM|second|afternoon|| +59821|AAAAAAAAOKJOAAAA|59821|16|37|1|PM|second|afternoon|| +59822|AAAAAAAAPKJOAAAA|59822|16|37|2|PM|second|afternoon|| +59823|AAAAAAAAALJOAAAA|59823|16|37|3|PM|second|afternoon|| +59824|AAAAAAAABLJOAAAA|59824|16|37|4|PM|second|afternoon|| +59825|AAAAAAAACLJOAAAA|59825|16|37|5|PM|second|afternoon|| +59826|AAAAAAAADLJOAAAA|59826|16|37|6|PM|second|afternoon|| +59827|AAAAAAAAELJOAAAA|59827|16|37|7|PM|second|afternoon|| +59828|AAAAAAAAFLJOAAAA|59828|16|37|8|PM|second|afternoon|| +59829|AAAAAAAAGLJOAAAA|59829|16|37|9|PM|second|afternoon|| +59830|AAAAAAAAHLJOAAAA|59830|16|37|10|PM|second|afternoon|| +59831|AAAAAAAAILJOAAAA|59831|16|37|11|PM|second|afternoon|| +59832|AAAAAAAAJLJOAAAA|59832|16|37|12|PM|second|afternoon|| +59833|AAAAAAAAKLJOAAAA|59833|16|37|13|PM|second|afternoon|| +59834|AAAAAAAALLJOAAAA|59834|16|37|14|PM|second|afternoon|| +59835|AAAAAAAAMLJOAAAA|59835|16|37|15|PM|second|afternoon|| +59836|AAAAAAAANLJOAAAA|59836|16|37|16|PM|second|afternoon|| +59837|AAAAAAAAOLJOAAAA|59837|16|37|17|PM|second|afternoon|| +59838|AAAAAAAAPLJOAAAA|59838|16|37|18|PM|second|afternoon|| +59839|AAAAAAAAAMJOAAAA|59839|16|37|19|PM|second|afternoon|| +59840|AAAAAAAABMJOAAAA|59840|16|37|20|PM|second|afternoon|| +59841|AAAAAAAACMJOAAAA|59841|16|37|21|PM|second|afternoon|| +59842|AAAAAAAADMJOAAAA|59842|16|37|22|PM|second|afternoon|| +59843|AAAAAAAAEMJOAAAA|59843|16|37|23|PM|second|afternoon|| +59844|AAAAAAAAFMJOAAAA|59844|16|37|24|PM|second|afternoon|| +59845|AAAAAAAAGMJOAAAA|59845|16|37|25|PM|second|afternoon|| +59846|AAAAAAAAHMJOAAAA|59846|16|37|26|PM|second|afternoon|| +59847|AAAAAAAAIMJOAAAA|59847|16|37|27|PM|second|afternoon|| +59848|AAAAAAAAJMJOAAAA|59848|16|37|28|PM|second|afternoon|| +59849|AAAAAAAAKMJOAAAA|59849|16|37|29|PM|second|afternoon|| +59850|AAAAAAAALMJOAAAA|59850|16|37|30|PM|second|afternoon|| +59851|AAAAAAAAMMJOAAAA|59851|16|37|31|PM|second|afternoon|| +59852|AAAAAAAANMJOAAAA|59852|16|37|32|PM|second|afternoon|| +59853|AAAAAAAAOMJOAAAA|59853|16|37|33|PM|second|afternoon|| +59854|AAAAAAAAPMJOAAAA|59854|16|37|34|PM|second|afternoon|| +59855|AAAAAAAAANJOAAAA|59855|16|37|35|PM|second|afternoon|| +59856|AAAAAAAABNJOAAAA|59856|16|37|36|PM|second|afternoon|| +59857|AAAAAAAACNJOAAAA|59857|16|37|37|PM|second|afternoon|| +59858|AAAAAAAADNJOAAAA|59858|16|37|38|PM|second|afternoon|| +59859|AAAAAAAAENJOAAAA|59859|16|37|39|PM|second|afternoon|| +59860|AAAAAAAAFNJOAAAA|59860|16|37|40|PM|second|afternoon|| +59861|AAAAAAAAGNJOAAAA|59861|16|37|41|PM|second|afternoon|| +59862|AAAAAAAAHNJOAAAA|59862|16|37|42|PM|second|afternoon|| +59863|AAAAAAAAINJOAAAA|59863|16|37|43|PM|second|afternoon|| +59864|AAAAAAAAJNJOAAAA|59864|16|37|44|PM|second|afternoon|| +59865|AAAAAAAAKNJOAAAA|59865|16|37|45|PM|second|afternoon|| +59866|AAAAAAAALNJOAAAA|59866|16|37|46|PM|second|afternoon|| +59867|AAAAAAAAMNJOAAAA|59867|16|37|47|PM|second|afternoon|| +59868|AAAAAAAANNJOAAAA|59868|16|37|48|PM|second|afternoon|| +59869|AAAAAAAAONJOAAAA|59869|16|37|49|PM|second|afternoon|| +59870|AAAAAAAAPNJOAAAA|59870|16|37|50|PM|second|afternoon|| +59871|AAAAAAAAAOJOAAAA|59871|16|37|51|PM|second|afternoon|| +59872|AAAAAAAABOJOAAAA|59872|16|37|52|PM|second|afternoon|| +59873|AAAAAAAACOJOAAAA|59873|16|37|53|PM|second|afternoon|| +59874|AAAAAAAADOJOAAAA|59874|16|37|54|PM|second|afternoon|| +59875|AAAAAAAAEOJOAAAA|59875|16|37|55|PM|second|afternoon|| +59876|AAAAAAAAFOJOAAAA|59876|16|37|56|PM|second|afternoon|| +59877|AAAAAAAAGOJOAAAA|59877|16|37|57|PM|second|afternoon|| +59878|AAAAAAAAHOJOAAAA|59878|16|37|58|PM|second|afternoon|| +59879|AAAAAAAAIOJOAAAA|59879|16|37|59|PM|second|afternoon|| +59880|AAAAAAAAJOJOAAAA|59880|16|38|0|PM|second|afternoon|| +59881|AAAAAAAAKOJOAAAA|59881|16|38|1|PM|second|afternoon|| +59882|AAAAAAAALOJOAAAA|59882|16|38|2|PM|second|afternoon|| +59883|AAAAAAAAMOJOAAAA|59883|16|38|3|PM|second|afternoon|| +59884|AAAAAAAANOJOAAAA|59884|16|38|4|PM|second|afternoon|| +59885|AAAAAAAAOOJOAAAA|59885|16|38|5|PM|second|afternoon|| +59886|AAAAAAAAPOJOAAAA|59886|16|38|6|PM|second|afternoon|| +59887|AAAAAAAAAPJOAAAA|59887|16|38|7|PM|second|afternoon|| +59888|AAAAAAAABPJOAAAA|59888|16|38|8|PM|second|afternoon|| +59889|AAAAAAAACPJOAAAA|59889|16|38|9|PM|second|afternoon|| +59890|AAAAAAAADPJOAAAA|59890|16|38|10|PM|second|afternoon|| +59891|AAAAAAAAEPJOAAAA|59891|16|38|11|PM|second|afternoon|| +59892|AAAAAAAAFPJOAAAA|59892|16|38|12|PM|second|afternoon|| +59893|AAAAAAAAGPJOAAAA|59893|16|38|13|PM|second|afternoon|| +59894|AAAAAAAAHPJOAAAA|59894|16|38|14|PM|second|afternoon|| +59895|AAAAAAAAIPJOAAAA|59895|16|38|15|PM|second|afternoon|| +59896|AAAAAAAAJPJOAAAA|59896|16|38|16|PM|second|afternoon|| +59897|AAAAAAAAKPJOAAAA|59897|16|38|17|PM|second|afternoon|| +59898|AAAAAAAALPJOAAAA|59898|16|38|18|PM|second|afternoon|| +59899|AAAAAAAAMPJOAAAA|59899|16|38|19|PM|second|afternoon|| +59900|AAAAAAAANPJOAAAA|59900|16|38|20|PM|second|afternoon|| +59901|AAAAAAAAOPJOAAAA|59901|16|38|21|PM|second|afternoon|| +59902|AAAAAAAAPPJOAAAA|59902|16|38|22|PM|second|afternoon|| +59903|AAAAAAAAAAKOAAAA|59903|16|38|23|PM|second|afternoon|| +59904|AAAAAAAABAKOAAAA|59904|16|38|24|PM|second|afternoon|| +59905|AAAAAAAACAKOAAAA|59905|16|38|25|PM|second|afternoon|| +59906|AAAAAAAADAKOAAAA|59906|16|38|26|PM|second|afternoon|| +59907|AAAAAAAAEAKOAAAA|59907|16|38|27|PM|second|afternoon|| +59908|AAAAAAAAFAKOAAAA|59908|16|38|28|PM|second|afternoon|| +59909|AAAAAAAAGAKOAAAA|59909|16|38|29|PM|second|afternoon|| +59910|AAAAAAAAHAKOAAAA|59910|16|38|30|PM|second|afternoon|| +59911|AAAAAAAAIAKOAAAA|59911|16|38|31|PM|second|afternoon|| +59912|AAAAAAAAJAKOAAAA|59912|16|38|32|PM|second|afternoon|| +59913|AAAAAAAAKAKOAAAA|59913|16|38|33|PM|second|afternoon|| +59914|AAAAAAAALAKOAAAA|59914|16|38|34|PM|second|afternoon|| +59915|AAAAAAAAMAKOAAAA|59915|16|38|35|PM|second|afternoon|| +59916|AAAAAAAANAKOAAAA|59916|16|38|36|PM|second|afternoon|| +59917|AAAAAAAAOAKOAAAA|59917|16|38|37|PM|second|afternoon|| +59918|AAAAAAAAPAKOAAAA|59918|16|38|38|PM|second|afternoon|| +59919|AAAAAAAAABKOAAAA|59919|16|38|39|PM|second|afternoon|| +59920|AAAAAAAABBKOAAAA|59920|16|38|40|PM|second|afternoon|| +59921|AAAAAAAACBKOAAAA|59921|16|38|41|PM|second|afternoon|| +59922|AAAAAAAADBKOAAAA|59922|16|38|42|PM|second|afternoon|| +59923|AAAAAAAAEBKOAAAA|59923|16|38|43|PM|second|afternoon|| +59924|AAAAAAAAFBKOAAAA|59924|16|38|44|PM|second|afternoon|| +59925|AAAAAAAAGBKOAAAA|59925|16|38|45|PM|second|afternoon|| +59926|AAAAAAAAHBKOAAAA|59926|16|38|46|PM|second|afternoon|| +59927|AAAAAAAAIBKOAAAA|59927|16|38|47|PM|second|afternoon|| +59928|AAAAAAAAJBKOAAAA|59928|16|38|48|PM|second|afternoon|| +59929|AAAAAAAAKBKOAAAA|59929|16|38|49|PM|second|afternoon|| +59930|AAAAAAAALBKOAAAA|59930|16|38|50|PM|second|afternoon|| +59931|AAAAAAAAMBKOAAAA|59931|16|38|51|PM|second|afternoon|| +59932|AAAAAAAANBKOAAAA|59932|16|38|52|PM|second|afternoon|| +59933|AAAAAAAAOBKOAAAA|59933|16|38|53|PM|second|afternoon|| +59934|AAAAAAAAPBKOAAAA|59934|16|38|54|PM|second|afternoon|| +59935|AAAAAAAAACKOAAAA|59935|16|38|55|PM|second|afternoon|| +59936|AAAAAAAABCKOAAAA|59936|16|38|56|PM|second|afternoon|| +59937|AAAAAAAACCKOAAAA|59937|16|38|57|PM|second|afternoon|| +59938|AAAAAAAADCKOAAAA|59938|16|38|58|PM|second|afternoon|| +59939|AAAAAAAAECKOAAAA|59939|16|38|59|PM|second|afternoon|| +59940|AAAAAAAAFCKOAAAA|59940|16|39|0|PM|second|afternoon|| +59941|AAAAAAAAGCKOAAAA|59941|16|39|1|PM|second|afternoon|| +59942|AAAAAAAAHCKOAAAA|59942|16|39|2|PM|second|afternoon|| +59943|AAAAAAAAICKOAAAA|59943|16|39|3|PM|second|afternoon|| +59944|AAAAAAAAJCKOAAAA|59944|16|39|4|PM|second|afternoon|| +59945|AAAAAAAAKCKOAAAA|59945|16|39|5|PM|second|afternoon|| +59946|AAAAAAAALCKOAAAA|59946|16|39|6|PM|second|afternoon|| +59947|AAAAAAAAMCKOAAAA|59947|16|39|7|PM|second|afternoon|| +59948|AAAAAAAANCKOAAAA|59948|16|39|8|PM|second|afternoon|| +59949|AAAAAAAAOCKOAAAA|59949|16|39|9|PM|second|afternoon|| +59950|AAAAAAAAPCKOAAAA|59950|16|39|10|PM|second|afternoon|| +59951|AAAAAAAAADKOAAAA|59951|16|39|11|PM|second|afternoon|| +59952|AAAAAAAABDKOAAAA|59952|16|39|12|PM|second|afternoon|| +59953|AAAAAAAACDKOAAAA|59953|16|39|13|PM|second|afternoon|| +59954|AAAAAAAADDKOAAAA|59954|16|39|14|PM|second|afternoon|| +59955|AAAAAAAAEDKOAAAA|59955|16|39|15|PM|second|afternoon|| +59956|AAAAAAAAFDKOAAAA|59956|16|39|16|PM|second|afternoon|| +59957|AAAAAAAAGDKOAAAA|59957|16|39|17|PM|second|afternoon|| +59958|AAAAAAAAHDKOAAAA|59958|16|39|18|PM|second|afternoon|| +59959|AAAAAAAAIDKOAAAA|59959|16|39|19|PM|second|afternoon|| +59960|AAAAAAAAJDKOAAAA|59960|16|39|20|PM|second|afternoon|| +59961|AAAAAAAAKDKOAAAA|59961|16|39|21|PM|second|afternoon|| +59962|AAAAAAAALDKOAAAA|59962|16|39|22|PM|second|afternoon|| +59963|AAAAAAAAMDKOAAAA|59963|16|39|23|PM|second|afternoon|| +59964|AAAAAAAANDKOAAAA|59964|16|39|24|PM|second|afternoon|| +59965|AAAAAAAAODKOAAAA|59965|16|39|25|PM|second|afternoon|| +59966|AAAAAAAAPDKOAAAA|59966|16|39|26|PM|second|afternoon|| +59967|AAAAAAAAAEKOAAAA|59967|16|39|27|PM|second|afternoon|| +59968|AAAAAAAABEKOAAAA|59968|16|39|28|PM|second|afternoon|| +59969|AAAAAAAACEKOAAAA|59969|16|39|29|PM|second|afternoon|| +59970|AAAAAAAADEKOAAAA|59970|16|39|30|PM|second|afternoon|| +59971|AAAAAAAAEEKOAAAA|59971|16|39|31|PM|second|afternoon|| +59972|AAAAAAAAFEKOAAAA|59972|16|39|32|PM|second|afternoon|| +59973|AAAAAAAAGEKOAAAA|59973|16|39|33|PM|second|afternoon|| +59974|AAAAAAAAHEKOAAAA|59974|16|39|34|PM|second|afternoon|| +59975|AAAAAAAAIEKOAAAA|59975|16|39|35|PM|second|afternoon|| +59976|AAAAAAAAJEKOAAAA|59976|16|39|36|PM|second|afternoon|| +59977|AAAAAAAAKEKOAAAA|59977|16|39|37|PM|second|afternoon|| +59978|AAAAAAAALEKOAAAA|59978|16|39|38|PM|second|afternoon|| +59979|AAAAAAAAMEKOAAAA|59979|16|39|39|PM|second|afternoon|| +59980|AAAAAAAANEKOAAAA|59980|16|39|40|PM|second|afternoon|| +59981|AAAAAAAAOEKOAAAA|59981|16|39|41|PM|second|afternoon|| +59982|AAAAAAAAPEKOAAAA|59982|16|39|42|PM|second|afternoon|| +59983|AAAAAAAAAFKOAAAA|59983|16|39|43|PM|second|afternoon|| +59984|AAAAAAAABFKOAAAA|59984|16|39|44|PM|second|afternoon|| +59985|AAAAAAAACFKOAAAA|59985|16|39|45|PM|second|afternoon|| +59986|AAAAAAAADFKOAAAA|59986|16|39|46|PM|second|afternoon|| +59987|AAAAAAAAEFKOAAAA|59987|16|39|47|PM|second|afternoon|| +59988|AAAAAAAAFFKOAAAA|59988|16|39|48|PM|second|afternoon|| +59989|AAAAAAAAGFKOAAAA|59989|16|39|49|PM|second|afternoon|| +59990|AAAAAAAAHFKOAAAA|59990|16|39|50|PM|second|afternoon|| +59991|AAAAAAAAIFKOAAAA|59991|16|39|51|PM|second|afternoon|| +59992|AAAAAAAAJFKOAAAA|59992|16|39|52|PM|second|afternoon|| +59993|AAAAAAAAKFKOAAAA|59993|16|39|53|PM|second|afternoon|| +59994|AAAAAAAALFKOAAAA|59994|16|39|54|PM|second|afternoon|| +59995|AAAAAAAAMFKOAAAA|59995|16|39|55|PM|second|afternoon|| +59996|AAAAAAAANFKOAAAA|59996|16|39|56|PM|second|afternoon|| +59997|AAAAAAAAOFKOAAAA|59997|16|39|57|PM|second|afternoon|| +59998|AAAAAAAAPFKOAAAA|59998|16|39|58|PM|second|afternoon|| +59999|AAAAAAAAAGKOAAAA|59999|16|39|59|PM|second|afternoon|| +60000|AAAAAAAABGKOAAAA|60000|16|40|0|PM|second|afternoon|| +60001|AAAAAAAACGKOAAAA|60001|16|40|1|PM|second|afternoon|| +60002|AAAAAAAADGKOAAAA|60002|16|40|2|PM|second|afternoon|| +60003|AAAAAAAAEGKOAAAA|60003|16|40|3|PM|second|afternoon|| +60004|AAAAAAAAFGKOAAAA|60004|16|40|4|PM|second|afternoon|| +60005|AAAAAAAAGGKOAAAA|60005|16|40|5|PM|second|afternoon|| +60006|AAAAAAAAHGKOAAAA|60006|16|40|6|PM|second|afternoon|| +60007|AAAAAAAAIGKOAAAA|60007|16|40|7|PM|second|afternoon|| +60008|AAAAAAAAJGKOAAAA|60008|16|40|8|PM|second|afternoon|| +60009|AAAAAAAAKGKOAAAA|60009|16|40|9|PM|second|afternoon|| +60010|AAAAAAAALGKOAAAA|60010|16|40|10|PM|second|afternoon|| +60011|AAAAAAAAMGKOAAAA|60011|16|40|11|PM|second|afternoon|| +60012|AAAAAAAANGKOAAAA|60012|16|40|12|PM|second|afternoon|| +60013|AAAAAAAAOGKOAAAA|60013|16|40|13|PM|second|afternoon|| +60014|AAAAAAAAPGKOAAAA|60014|16|40|14|PM|second|afternoon|| +60015|AAAAAAAAAHKOAAAA|60015|16|40|15|PM|second|afternoon|| +60016|AAAAAAAABHKOAAAA|60016|16|40|16|PM|second|afternoon|| +60017|AAAAAAAACHKOAAAA|60017|16|40|17|PM|second|afternoon|| +60018|AAAAAAAADHKOAAAA|60018|16|40|18|PM|second|afternoon|| +60019|AAAAAAAAEHKOAAAA|60019|16|40|19|PM|second|afternoon|| +60020|AAAAAAAAFHKOAAAA|60020|16|40|20|PM|second|afternoon|| +60021|AAAAAAAAGHKOAAAA|60021|16|40|21|PM|second|afternoon|| +60022|AAAAAAAAHHKOAAAA|60022|16|40|22|PM|second|afternoon|| +60023|AAAAAAAAIHKOAAAA|60023|16|40|23|PM|second|afternoon|| +60024|AAAAAAAAJHKOAAAA|60024|16|40|24|PM|second|afternoon|| +60025|AAAAAAAAKHKOAAAA|60025|16|40|25|PM|second|afternoon|| +60026|AAAAAAAALHKOAAAA|60026|16|40|26|PM|second|afternoon|| +60027|AAAAAAAAMHKOAAAA|60027|16|40|27|PM|second|afternoon|| +60028|AAAAAAAANHKOAAAA|60028|16|40|28|PM|second|afternoon|| +60029|AAAAAAAAOHKOAAAA|60029|16|40|29|PM|second|afternoon|| +60030|AAAAAAAAPHKOAAAA|60030|16|40|30|PM|second|afternoon|| +60031|AAAAAAAAAIKOAAAA|60031|16|40|31|PM|second|afternoon|| +60032|AAAAAAAABIKOAAAA|60032|16|40|32|PM|second|afternoon|| +60033|AAAAAAAACIKOAAAA|60033|16|40|33|PM|second|afternoon|| +60034|AAAAAAAADIKOAAAA|60034|16|40|34|PM|second|afternoon|| +60035|AAAAAAAAEIKOAAAA|60035|16|40|35|PM|second|afternoon|| +60036|AAAAAAAAFIKOAAAA|60036|16|40|36|PM|second|afternoon|| +60037|AAAAAAAAGIKOAAAA|60037|16|40|37|PM|second|afternoon|| +60038|AAAAAAAAHIKOAAAA|60038|16|40|38|PM|second|afternoon|| +60039|AAAAAAAAIIKOAAAA|60039|16|40|39|PM|second|afternoon|| +60040|AAAAAAAAJIKOAAAA|60040|16|40|40|PM|second|afternoon|| +60041|AAAAAAAAKIKOAAAA|60041|16|40|41|PM|second|afternoon|| +60042|AAAAAAAALIKOAAAA|60042|16|40|42|PM|second|afternoon|| +60043|AAAAAAAAMIKOAAAA|60043|16|40|43|PM|second|afternoon|| +60044|AAAAAAAANIKOAAAA|60044|16|40|44|PM|second|afternoon|| +60045|AAAAAAAAOIKOAAAA|60045|16|40|45|PM|second|afternoon|| +60046|AAAAAAAAPIKOAAAA|60046|16|40|46|PM|second|afternoon|| +60047|AAAAAAAAAJKOAAAA|60047|16|40|47|PM|second|afternoon|| +60048|AAAAAAAABJKOAAAA|60048|16|40|48|PM|second|afternoon|| +60049|AAAAAAAACJKOAAAA|60049|16|40|49|PM|second|afternoon|| +60050|AAAAAAAADJKOAAAA|60050|16|40|50|PM|second|afternoon|| +60051|AAAAAAAAEJKOAAAA|60051|16|40|51|PM|second|afternoon|| +60052|AAAAAAAAFJKOAAAA|60052|16|40|52|PM|second|afternoon|| +60053|AAAAAAAAGJKOAAAA|60053|16|40|53|PM|second|afternoon|| +60054|AAAAAAAAHJKOAAAA|60054|16|40|54|PM|second|afternoon|| +60055|AAAAAAAAIJKOAAAA|60055|16|40|55|PM|second|afternoon|| +60056|AAAAAAAAJJKOAAAA|60056|16|40|56|PM|second|afternoon|| +60057|AAAAAAAAKJKOAAAA|60057|16|40|57|PM|second|afternoon|| +60058|AAAAAAAALJKOAAAA|60058|16|40|58|PM|second|afternoon|| +60059|AAAAAAAAMJKOAAAA|60059|16|40|59|PM|second|afternoon|| +60060|AAAAAAAANJKOAAAA|60060|16|41|0|PM|second|afternoon|| +60061|AAAAAAAAOJKOAAAA|60061|16|41|1|PM|second|afternoon|| +60062|AAAAAAAAPJKOAAAA|60062|16|41|2|PM|second|afternoon|| +60063|AAAAAAAAAKKOAAAA|60063|16|41|3|PM|second|afternoon|| +60064|AAAAAAAABKKOAAAA|60064|16|41|4|PM|second|afternoon|| +60065|AAAAAAAACKKOAAAA|60065|16|41|5|PM|second|afternoon|| +60066|AAAAAAAADKKOAAAA|60066|16|41|6|PM|second|afternoon|| +60067|AAAAAAAAEKKOAAAA|60067|16|41|7|PM|second|afternoon|| +60068|AAAAAAAAFKKOAAAA|60068|16|41|8|PM|second|afternoon|| +60069|AAAAAAAAGKKOAAAA|60069|16|41|9|PM|second|afternoon|| +60070|AAAAAAAAHKKOAAAA|60070|16|41|10|PM|second|afternoon|| +60071|AAAAAAAAIKKOAAAA|60071|16|41|11|PM|second|afternoon|| +60072|AAAAAAAAJKKOAAAA|60072|16|41|12|PM|second|afternoon|| +60073|AAAAAAAAKKKOAAAA|60073|16|41|13|PM|second|afternoon|| +60074|AAAAAAAALKKOAAAA|60074|16|41|14|PM|second|afternoon|| +60075|AAAAAAAAMKKOAAAA|60075|16|41|15|PM|second|afternoon|| +60076|AAAAAAAANKKOAAAA|60076|16|41|16|PM|second|afternoon|| +60077|AAAAAAAAOKKOAAAA|60077|16|41|17|PM|second|afternoon|| +60078|AAAAAAAAPKKOAAAA|60078|16|41|18|PM|second|afternoon|| +60079|AAAAAAAAALKOAAAA|60079|16|41|19|PM|second|afternoon|| +60080|AAAAAAAABLKOAAAA|60080|16|41|20|PM|second|afternoon|| +60081|AAAAAAAACLKOAAAA|60081|16|41|21|PM|second|afternoon|| +60082|AAAAAAAADLKOAAAA|60082|16|41|22|PM|second|afternoon|| +60083|AAAAAAAAELKOAAAA|60083|16|41|23|PM|second|afternoon|| +60084|AAAAAAAAFLKOAAAA|60084|16|41|24|PM|second|afternoon|| +60085|AAAAAAAAGLKOAAAA|60085|16|41|25|PM|second|afternoon|| +60086|AAAAAAAAHLKOAAAA|60086|16|41|26|PM|second|afternoon|| +60087|AAAAAAAAILKOAAAA|60087|16|41|27|PM|second|afternoon|| +60088|AAAAAAAAJLKOAAAA|60088|16|41|28|PM|second|afternoon|| +60089|AAAAAAAAKLKOAAAA|60089|16|41|29|PM|second|afternoon|| +60090|AAAAAAAALLKOAAAA|60090|16|41|30|PM|second|afternoon|| +60091|AAAAAAAAMLKOAAAA|60091|16|41|31|PM|second|afternoon|| +60092|AAAAAAAANLKOAAAA|60092|16|41|32|PM|second|afternoon|| +60093|AAAAAAAAOLKOAAAA|60093|16|41|33|PM|second|afternoon|| +60094|AAAAAAAAPLKOAAAA|60094|16|41|34|PM|second|afternoon|| +60095|AAAAAAAAAMKOAAAA|60095|16|41|35|PM|second|afternoon|| +60096|AAAAAAAABMKOAAAA|60096|16|41|36|PM|second|afternoon|| +60097|AAAAAAAACMKOAAAA|60097|16|41|37|PM|second|afternoon|| +60098|AAAAAAAADMKOAAAA|60098|16|41|38|PM|second|afternoon|| +60099|AAAAAAAAEMKOAAAA|60099|16|41|39|PM|second|afternoon|| +60100|AAAAAAAAFMKOAAAA|60100|16|41|40|PM|second|afternoon|| +60101|AAAAAAAAGMKOAAAA|60101|16|41|41|PM|second|afternoon|| +60102|AAAAAAAAHMKOAAAA|60102|16|41|42|PM|second|afternoon|| +60103|AAAAAAAAIMKOAAAA|60103|16|41|43|PM|second|afternoon|| +60104|AAAAAAAAJMKOAAAA|60104|16|41|44|PM|second|afternoon|| +60105|AAAAAAAAKMKOAAAA|60105|16|41|45|PM|second|afternoon|| +60106|AAAAAAAALMKOAAAA|60106|16|41|46|PM|second|afternoon|| +60107|AAAAAAAAMMKOAAAA|60107|16|41|47|PM|second|afternoon|| +60108|AAAAAAAANMKOAAAA|60108|16|41|48|PM|second|afternoon|| +60109|AAAAAAAAOMKOAAAA|60109|16|41|49|PM|second|afternoon|| +60110|AAAAAAAAPMKOAAAA|60110|16|41|50|PM|second|afternoon|| +60111|AAAAAAAAANKOAAAA|60111|16|41|51|PM|second|afternoon|| +60112|AAAAAAAABNKOAAAA|60112|16|41|52|PM|second|afternoon|| +60113|AAAAAAAACNKOAAAA|60113|16|41|53|PM|second|afternoon|| +60114|AAAAAAAADNKOAAAA|60114|16|41|54|PM|second|afternoon|| +60115|AAAAAAAAENKOAAAA|60115|16|41|55|PM|second|afternoon|| +60116|AAAAAAAAFNKOAAAA|60116|16|41|56|PM|second|afternoon|| +60117|AAAAAAAAGNKOAAAA|60117|16|41|57|PM|second|afternoon|| +60118|AAAAAAAAHNKOAAAA|60118|16|41|58|PM|second|afternoon|| +60119|AAAAAAAAINKOAAAA|60119|16|41|59|PM|second|afternoon|| +60120|AAAAAAAAJNKOAAAA|60120|16|42|0|PM|second|afternoon|| +60121|AAAAAAAAKNKOAAAA|60121|16|42|1|PM|second|afternoon|| +60122|AAAAAAAALNKOAAAA|60122|16|42|2|PM|second|afternoon|| +60123|AAAAAAAAMNKOAAAA|60123|16|42|3|PM|second|afternoon|| +60124|AAAAAAAANNKOAAAA|60124|16|42|4|PM|second|afternoon|| +60125|AAAAAAAAONKOAAAA|60125|16|42|5|PM|second|afternoon|| +60126|AAAAAAAAPNKOAAAA|60126|16|42|6|PM|second|afternoon|| +60127|AAAAAAAAAOKOAAAA|60127|16|42|7|PM|second|afternoon|| +60128|AAAAAAAABOKOAAAA|60128|16|42|8|PM|second|afternoon|| +60129|AAAAAAAACOKOAAAA|60129|16|42|9|PM|second|afternoon|| +60130|AAAAAAAADOKOAAAA|60130|16|42|10|PM|second|afternoon|| +60131|AAAAAAAAEOKOAAAA|60131|16|42|11|PM|second|afternoon|| +60132|AAAAAAAAFOKOAAAA|60132|16|42|12|PM|second|afternoon|| +60133|AAAAAAAAGOKOAAAA|60133|16|42|13|PM|second|afternoon|| +60134|AAAAAAAAHOKOAAAA|60134|16|42|14|PM|second|afternoon|| +60135|AAAAAAAAIOKOAAAA|60135|16|42|15|PM|second|afternoon|| +60136|AAAAAAAAJOKOAAAA|60136|16|42|16|PM|second|afternoon|| +60137|AAAAAAAAKOKOAAAA|60137|16|42|17|PM|second|afternoon|| +60138|AAAAAAAALOKOAAAA|60138|16|42|18|PM|second|afternoon|| +60139|AAAAAAAAMOKOAAAA|60139|16|42|19|PM|second|afternoon|| +60140|AAAAAAAANOKOAAAA|60140|16|42|20|PM|second|afternoon|| +60141|AAAAAAAAOOKOAAAA|60141|16|42|21|PM|second|afternoon|| +60142|AAAAAAAAPOKOAAAA|60142|16|42|22|PM|second|afternoon|| +60143|AAAAAAAAAPKOAAAA|60143|16|42|23|PM|second|afternoon|| +60144|AAAAAAAABPKOAAAA|60144|16|42|24|PM|second|afternoon|| +60145|AAAAAAAACPKOAAAA|60145|16|42|25|PM|second|afternoon|| +60146|AAAAAAAADPKOAAAA|60146|16|42|26|PM|second|afternoon|| +60147|AAAAAAAAEPKOAAAA|60147|16|42|27|PM|second|afternoon|| +60148|AAAAAAAAFPKOAAAA|60148|16|42|28|PM|second|afternoon|| +60149|AAAAAAAAGPKOAAAA|60149|16|42|29|PM|second|afternoon|| +60150|AAAAAAAAHPKOAAAA|60150|16|42|30|PM|second|afternoon|| +60151|AAAAAAAAIPKOAAAA|60151|16|42|31|PM|second|afternoon|| +60152|AAAAAAAAJPKOAAAA|60152|16|42|32|PM|second|afternoon|| +60153|AAAAAAAAKPKOAAAA|60153|16|42|33|PM|second|afternoon|| +60154|AAAAAAAALPKOAAAA|60154|16|42|34|PM|second|afternoon|| +60155|AAAAAAAAMPKOAAAA|60155|16|42|35|PM|second|afternoon|| +60156|AAAAAAAANPKOAAAA|60156|16|42|36|PM|second|afternoon|| +60157|AAAAAAAAOPKOAAAA|60157|16|42|37|PM|second|afternoon|| +60158|AAAAAAAAPPKOAAAA|60158|16|42|38|PM|second|afternoon|| +60159|AAAAAAAAAALOAAAA|60159|16|42|39|PM|second|afternoon|| +60160|AAAAAAAABALOAAAA|60160|16|42|40|PM|second|afternoon|| +60161|AAAAAAAACALOAAAA|60161|16|42|41|PM|second|afternoon|| +60162|AAAAAAAADALOAAAA|60162|16|42|42|PM|second|afternoon|| +60163|AAAAAAAAEALOAAAA|60163|16|42|43|PM|second|afternoon|| +60164|AAAAAAAAFALOAAAA|60164|16|42|44|PM|second|afternoon|| +60165|AAAAAAAAGALOAAAA|60165|16|42|45|PM|second|afternoon|| +60166|AAAAAAAAHALOAAAA|60166|16|42|46|PM|second|afternoon|| +60167|AAAAAAAAIALOAAAA|60167|16|42|47|PM|second|afternoon|| +60168|AAAAAAAAJALOAAAA|60168|16|42|48|PM|second|afternoon|| +60169|AAAAAAAAKALOAAAA|60169|16|42|49|PM|second|afternoon|| +60170|AAAAAAAALALOAAAA|60170|16|42|50|PM|second|afternoon|| +60171|AAAAAAAAMALOAAAA|60171|16|42|51|PM|second|afternoon|| +60172|AAAAAAAANALOAAAA|60172|16|42|52|PM|second|afternoon|| +60173|AAAAAAAAOALOAAAA|60173|16|42|53|PM|second|afternoon|| +60174|AAAAAAAAPALOAAAA|60174|16|42|54|PM|second|afternoon|| +60175|AAAAAAAAABLOAAAA|60175|16|42|55|PM|second|afternoon|| +60176|AAAAAAAABBLOAAAA|60176|16|42|56|PM|second|afternoon|| +60177|AAAAAAAACBLOAAAA|60177|16|42|57|PM|second|afternoon|| +60178|AAAAAAAADBLOAAAA|60178|16|42|58|PM|second|afternoon|| +60179|AAAAAAAAEBLOAAAA|60179|16|42|59|PM|second|afternoon|| +60180|AAAAAAAAFBLOAAAA|60180|16|43|0|PM|second|afternoon|| +60181|AAAAAAAAGBLOAAAA|60181|16|43|1|PM|second|afternoon|| +60182|AAAAAAAAHBLOAAAA|60182|16|43|2|PM|second|afternoon|| +60183|AAAAAAAAIBLOAAAA|60183|16|43|3|PM|second|afternoon|| +60184|AAAAAAAAJBLOAAAA|60184|16|43|4|PM|second|afternoon|| +60185|AAAAAAAAKBLOAAAA|60185|16|43|5|PM|second|afternoon|| +60186|AAAAAAAALBLOAAAA|60186|16|43|6|PM|second|afternoon|| +60187|AAAAAAAAMBLOAAAA|60187|16|43|7|PM|second|afternoon|| +60188|AAAAAAAANBLOAAAA|60188|16|43|8|PM|second|afternoon|| +60189|AAAAAAAAOBLOAAAA|60189|16|43|9|PM|second|afternoon|| +60190|AAAAAAAAPBLOAAAA|60190|16|43|10|PM|second|afternoon|| +60191|AAAAAAAAACLOAAAA|60191|16|43|11|PM|second|afternoon|| +60192|AAAAAAAABCLOAAAA|60192|16|43|12|PM|second|afternoon|| +60193|AAAAAAAACCLOAAAA|60193|16|43|13|PM|second|afternoon|| +60194|AAAAAAAADCLOAAAA|60194|16|43|14|PM|second|afternoon|| +60195|AAAAAAAAECLOAAAA|60195|16|43|15|PM|second|afternoon|| +60196|AAAAAAAAFCLOAAAA|60196|16|43|16|PM|second|afternoon|| +60197|AAAAAAAAGCLOAAAA|60197|16|43|17|PM|second|afternoon|| +60198|AAAAAAAAHCLOAAAA|60198|16|43|18|PM|second|afternoon|| +60199|AAAAAAAAICLOAAAA|60199|16|43|19|PM|second|afternoon|| +60200|AAAAAAAAJCLOAAAA|60200|16|43|20|PM|second|afternoon|| +60201|AAAAAAAAKCLOAAAA|60201|16|43|21|PM|second|afternoon|| +60202|AAAAAAAALCLOAAAA|60202|16|43|22|PM|second|afternoon|| +60203|AAAAAAAAMCLOAAAA|60203|16|43|23|PM|second|afternoon|| +60204|AAAAAAAANCLOAAAA|60204|16|43|24|PM|second|afternoon|| +60205|AAAAAAAAOCLOAAAA|60205|16|43|25|PM|second|afternoon|| +60206|AAAAAAAAPCLOAAAA|60206|16|43|26|PM|second|afternoon|| +60207|AAAAAAAAADLOAAAA|60207|16|43|27|PM|second|afternoon|| +60208|AAAAAAAABDLOAAAA|60208|16|43|28|PM|second|afternoon|| +60209|AAAAAAAACDLOAAAA|60209|16|43|29|PM|second|afternoon|| +60210|AAAAAAAADDLOAAAA|60210|16|43|30|PM|second|afternoon|| +60211|AAAAAAAAEDLOAAAA|60211|16|43|31|PM|second|afternoon|| +60212|AAAAAAAAFDLOAAAA|60212|16|43|32|PM|second|afternoon|| +60213|AAAAAAAAGDLOAAAA|60213|16|43|33|PM|second|afternoon|| +60214|AAAAAAAAHDLOAAAA|60214|16|43|34|PM|second|afternoon|| +60215|AAAAAAAAIDLOAAAA|60215|16|43|35|PM|second|afternoon|| +60216|AAAAAAAAJDLOAAAA|60216|16|43|36|PM|second|afternoon|| +60217|AAAAAAAAKDLOAAAA|60217|16|43|37|PM|second|afternoon|| +60218|AAAAAAAALDLOAAAA|60218|16|43|38|PM|second|afternoon|| +60219|AAAAAAAAMDLOAAAA|60219|16|43|39|PM|second|afternoon|| +60220|AAAAAAAANDLOAAAA|60220|16|43|40|PM|second|afternoon|| +60221|AAAAAAAAODLOAAAA|60221|16|43|41|PM|second|afternoon|| +60222|AAAAAAAAPDLOAAAA|60222|16|43|42|PM|second|afternoon|| +60223|AAAAAAAAAELOAAAA|60223|16|43|43|PM|second|afternoon|| +60224|AAAAAAAABELOAAAA|60224|16|43|44|PM|second|afternoon|| +60225|AAAAAAAACELOAAAA|60225|16|43|45|PM|second|afternoon|| +60226|AAAAAAAADELOAAAA|60226|16|43|46|PM|second|afternoon|| +60227|AAAAAAAAEELOAAAA|60227|16|43|47|PM|second|afternoon|| +60228|AAAAAAAAFELOAAAA|60228|16|43|48|PM|second|afternoon|| +60229|AAAAAAAAGELOAAAA|60229|16|43|49|PM|second|afternoon|| +60230|AAAAAAAAHELOAAAA|60230|16|43|50|PM|second|afternoon|| +60231|AAAAAAAAIELOAAAA|60231|16|43|51|PM|second|afternoon|| +60232|AAAAAAAAJELOAAAA|60232|16|43|52|PM|second|afternoon|| +60233|AAAAAAAAKELOAAAA|60233|16|43|53|PM|second|afternoon|| +60234|AAAAAAAALELOAAAA|60234|16|43|54|PM|second|afternoon|| +60235|AAAAAAAAMELOAAAA|60235|16|43|55|PM|second|afternoon|| +60236|AAAAAAAANELOAAAA|60236|16|43|56|PM|second|afternoon|| +60237|AAAAAAAAOELOAAAA|60237|16|43|57|PM|second|afternoon|| +60238|AAAAAAAAPELOAAAA|60238|16|43|58|PM|second|afternoon|| +60239|AAAAAAAAAFLOAAAA|60239|16|43|59|PM|second|afternoon|| +60240|AAAAAAAABFLOAAAA|60240|16|44|0|PM|second|afternoon|| +60241|AAAAAAAACFLOAAAA|60241|16|44|1|PM|second|afternoon|| +60242|AAAAAAAADFLOAAAA|60242|16|44|2|PM|second|afternoon|| +60243|AAAAAAAAEFLOAAAA|60243|16|44|3|PM|second|afternoon|| +60244|AAAAAAAAFFLOAAAA|60244|16|44|4|PM|second|afternoon|| +60245|AAAAAAAAGFLOAAAA|60245|16|44|5|PM|second|afternoon|| +60246|AAAAAAAAHFLOAAAA|60246|16|44|6|PM|second|afternoon|| +60247|AAAAAAAAIFLOAAAA|60247|16|44|7|PM|second|afternoon|| +60248|AAAAAAAAJFLOAAAA|60248|16|44|8|PM|second|afternoon|| +60249|AAAAAAAAKFLOAAAA|60249|16|44|9|PM|second|afternoon|| +60250|AAAAAAAALFLOAAAA|60250|16|44|10|PM|second|afternoon|| +60251|AAAAAAAAMFLOAAAA|60251|16|44|11|PM|second|afternoon|| +60252|AAAAAAAANFLOAAAA|60252|16|44|12|PM|second|afternoon|| +60253|AAAAAAAAOFLOAAAA|60253|16|44|13|PM|second|afternoon|| +60254|AAAAAAAAPFLOAAAA|60254|16|44|14|PM|second|afternoon|| +60255|AAAAAAAAAGLOAAAA|60255|16|44|15|PM|second|afternoon|| +60256|AAAAAAAABGLOAAAA|60256|16|44|16|PM|second|afternoon|| +60257|AAAAAAAACGLOAAAA|60257|16|44|17|PM|second|afternoon|| +60258|AAAAAAAADGLOAAAA|60258|16|44|18|PM|second|afternoon|| +60259|AAAAAAAAEGLOAAAA|60259|16|44|19|PM|second|afternoon|| +60260|AAAAAAAAFGLOAAAA|60260|16|44|20|PM|second|afternoon|| +60261|AAAAAAAAGGLOAAAA|60261|16|44|21|PM|second|afternoon|| +60262|AAAAAAAAHGLOAAAA|60262|16|44|22|PM|second|afternoon|| +60263|AAAAAAAAIGLOAAAA|60263|16|44|23|PM|second|afternoon|| +60264|AAAAAAAAJGLOAAAA|60264|16|44|24|PM|second|afternoon|| +60265|AAAAAAAAKGLOAAAA|60265|16|44|25|PM|second|afternoon|| +60266|AAAAAAAALGLOAAAA|60266|16|44|26|PM|second|afternoon|| +60267|AAAAAAAAMGLOAAAA|60267|16|44|27|PM|second|afternoon|| +60268|AAAAAAAANGLOAAAA|60268|16|44|28|PM|second|afternoon|| +60269|AAAAAAAAOGLOAAAA|60269|16|44|29|PM|second|afternoon|| +60270|AAAAAAAAPGLOAAAA|60270|16|44|30|PM|second|afternoon|| +60271|AAAAAAAAAHLOAAAA|60271|16|44|31|PM|second|afternoon|| +60272|AAAAAAAABHLOAAAA|60272|16|44|32|PM|second|afternoon|| +60273|AAAAAAAACHLOAAAA|60273|16|44|33|PM|second|afternoon|| +60274|AAAAAAAADHLOAAAA|60274|16|44|34|PM|second|afternoon|| +60275|AAAAAAAAEHLOAAAA|60275|16|44|35|PM|second|afternoon|| +60276|AAAAAAAAFHLOAAAA|60276|16|44|36|PM|second|afternoon|| +60277|AAAAAAAAGHLOAAAA|60277|16|44|37|PM|second|afternoon|| +60278|AAAAAAAAHHLOAAAA|60278|16|44|38|PM|second|afternoon|| +60279|AAAAAAAAIHLOAAAA|60279|16|44|39|PM|second|afternoon|| +60280|AAAAAAAAJHLOAAAA|60280|16|44|40|PM|second|afternoon|| +60281|AAAAAAAAKHLOAAAA|60281|16|44|41|PM|second|afternoon|| +60282|AAAAAAAALHLOAAAA|60282|16|44|42|PM|second|afternoon|| +60283|AAAAAAAAMHLOAAAA|60283|16|44|43|PM|second|afternoon|| +60284|AAAAAAAANHLOAAAA|60284|16|44|44|PM|second|afternoon|| +60285|AAAAAAAAOHLOAAAA|60285|16|44|45|PM|second|afternoon|| +60286|AAAAAAAAPHLOAAAA|60286|16|44|46|PM|second|afternoon|| +60287|AAAAAAAAAILOAAAA|60287|16|44|47|PM|second|afternoon|| +60288|AAAAAAAABILOAAAA|60288|16|44|48|PM|second|afternoon|| +60289|AAAAAAAACILOAAAA|60289|16|44|49|PM|second|afternoon|| +60290|AAAAAAAADILOAAAA|60290|16|44|50|PM|second|afternoon|| +60291|AAAAAAAAEILOAAAA|60291|16|44|51|PM|second|afternoon|| +60292|AAAAAAAAFILOAAAA|60292|16|44|52|PM|second|afternoon|| +60293|AAAAAAAAGILOAAAA|60293|16|44|53|PM|second|afternoon|| +60294|AAAAAAAAHILOAAAA|60294|16|44|54|PM|second|afternoon|| +60295|AAAAAAAAIILOAAAA|60295|16|44|55|PM|second|afternoon|| +60296|AAAAAAAAJILOAAAA|60296|16|44|56|PM|second|afternoon|| +60297|AAAAAAAAKILOAAAA|60297|16|44|57|PM|second|afternoon|| +60298|AAAAAAAALILOAAAA|60298|16|44|58|PM|second|afternoon|| +60299|AAAAAAAAMILOAAAA|60299|16|44|59|PM|second|afternoon|| +60300|AAAAAAAANILOAAAA|60300|16|45|0|PM|second|afternoon|| +60301|AAAAAAAAOILOAAAA|60301|16|45|1|PM|second|afternoon|| +60302|AAAAAAAAPILOAAAA|60302|16|45|2|PM|second|afternoon|| +60303|AAAAAAAAAJLOAAAA|60303|16|45|3|PM|second|afternoon|| +60304|AAAAAAAABJLOAAAA|60304|16|45|4|PM|second|afternoon|| +60305|AAAAAAAACJLOAAAA|60305|16|45|5|PM|second|afternoon|| +60306|AAAAAAAADJLOAAAA|60306|16|45|6|PM|second|afternoon|| +60307|AAAAAAAAEJLOAAAA|60307|16|45|7|PM|second|afternoon|| +60308|AAAAAAAAFJLOAAAA|60308|16|45|8|PM|second|afternoon|| +60309|AAAAAAAAGJLOAAAA|60309|16|45|9|PM|second|afternoon|| +60310|AAAAAAAAHJLOAAAA|60310|16|45|10|PM|second|afternoon|| +60311|AAAAAAAAIJLOAAAA|60311|16|45|11|PM|second|afternoon|| +60312|AAAAAAAAJJLOAAAA|60312|16|45|12|PM|second|afternoon|| +60313|AAAAAAAAKJLOAAAA|60313|16|45|13|PM|second|afternoon|| +60314|AAAAAAAALJLOAAAA|60314|16|45|14|PM|second|afternoon|| +60315|AAAAAAAAMJLOAAAA|60315|16|45|15|PM|second|afternoon|| +60316|AAAAAAAANJLOAAAA|60316|16|45|16|PM|second|afternoon|| +60317|AAAAAAAAOJLOAAAA|60317|16|45|17|PM|second|afternoon|| +60318|AAAAAAAAPJLOAAAA|60318|16|45|18|PM|second|afternoon|| +60319|AAAAAAAAAKLOAAAA|60319|16|45|19|PM|second|afternoon|| +60320|AAAAAAAABKLOAAAA|60320|16|45|20|PM|second|afternoon|| +60321|AAAAAAAACKLOAAAA|60321|16|45|21|PM|second|afternoon|| +60322|AAAAAAAADKLOAAAA|60322|16|45|22|PM|second|afternoon|| +60323|AAAAAAAAEKLOAAAA|60323|16|45|23|PM|second|afternoon|| +60324|AAAAAAAAFKLOAAAA|60324|16|45|24|PM|second|afternoon|| +60325|AAAAAAAAGKLOAAAA|60325|16|45|25|PM|second|afternoon|| +60326|AAAAAAAAHKLOAAAA|60326|16|45|26|PM|second|afternoon|| +60327|AAAAAAAAIKLOAAAA|60327|16|45|27|PM|second|afternoon|| +60328|AAAAAAAAJKLOAAAA|60328|16|45|28|PM|second|afternoon|| +60329|AAAAAAAAKKLOAAAA|60329|16|45|29|PM|second|afternoon|| +60330|AAAAAAAALKLOAAAA|60330|16|45|30|PM|second|afternoon|| +60331|AAAAAAAAMKLOAAAA|60331|16|45|31|PM|second|afternoon|| +60332|AAAAAAAANKLOAAAA|60332|16|45|32|PM|second|afternoon|| +60333|AAAAAAAAOKLOAAAA|60333|16|45|33|PM|second|afternoon|| +60334|AAAAAAAAPKLOAAAA|60334|16|45|34|PM|second|afternoon|| +60335|AAAAAAAAALLOAAAA|60335|16|45|35|PM|second|afternoon|| +60336|AAAAAAAABLLOAAAA|60336|16|45|36|PM|second|afternoon|| +60337|AAAAAAAACLLOAAAA|60337|16|45|37|PM|second|afternoon|| +60338|AAAAAAAADLLOAAAA|60338|16|45|38|PM|second|afternoon|| +60339|AAAAAAAAELLOAAAA|60339|16|45|39|PM|second|afternoon|| +60340|AAAAAAAAFLLOAAAA|60340|16|45|40|PM|second|afternoon|| +60341|AAAAAAAAGLLOAAAA|60341|16|45|41|PM|second|afternoon|| +60342|AAAAAAAAHLLOAAAA|60342|16|45|42|PM|second|afternoon|| +60343|AAAAAAAAILLOAAAA|60343|16|45|43|PM|second|afternoon|| +60344|AAAAAAAAJLLOAAAA|60344|16|45|44|PM|second|afternoon|| +60345|AAAAAAAAKLLOAAAA|60345|16|45|45|PM|second|afternoon|| +60346|AAAAAAAALLLOAAAA|60346|16|45|46|PM|second|afternoon|| +60347|AAAAAAAAMLLOAAAA|60347|16|45|47|PM|second|afternoon|| +60348|AAAAAAAANLLOAAAA|60348|16|45|48|PM|second|afternoon|| +60349|AAAAAAAAOLLOAAAA|60349|16|45|49|PM|second|afternoon|| +60350|AAAAAAAAPLLOAAAA|60350|16|45|50|PM|second|afternoon|| +60351|AAAAAAAAAMLOAAAA|60351|16|45|51|PM|second|afternoon|| +60352|AAAAAAAABMLOAAAA|60352|16|45|52|PM|second|afternoon|| +60353|AAAAAAAACMLOAAAA|60353|16|45|53|PM|second|afternoon|| +60354|AAAAAAAADMLOAAAA|60354|16|45|54|PM|second|afternoon|| +60355|AAAAAAAAEMLOAAAA|60355|16|45|55|PM|second|afternoon|| +60356|AAAAAAAAFMLOAAAA|60356|16|45|56|PM|second|afternoon|| +60357|AAAAAAAAGMLOAAAA|60357|16|45|57|PM|second|afternoon|| +60358|AAAAAAAAHMLOAAAA|60358|16|45|58|PM|second|afternoon|| +60359|AAAAAAAAIMLOAAAA|60359|16|45|59|PM|second|afternoon|| +60360|AAAAAAAAJMLOAAAA|60360|16|46|0|PM|second|afternoon|| +60361|AAAAAAAAKMLOAAAA|60361|16|46|1|PM|second|afternoon|| +60362|AAAAAAAALMLOAAAA|60362|16|46|2|PM|second|afternoon|| +60363|AAAAAAAAMMLOAAAA|60363|16|46|3|PM|second|afternoon|| +60364|AAAAAAAANMLOAAAA|60364|16|46|4|PM|second|afternoon|| +60365|AAAAAAAAOMLOAAAA|60365|16|46|5|PM|second|afternoon|| +60366|AAAAAAAAPMLOAAAA|60366|16|46|6|PM|second|afternoon|| +60367|AAAAAAAAANLOAAAA|60367|16|46|7|PM|second|afternoon|| +60368|AAAAAAAABNLOAAAA|60368|16|46|8|PM|second|afternoon|| +60369|AAAAAAAACNLOAAAA|60369|16|46|9|PM|second|afternoon|| +60370|AAAAAAAADNLOAAAA|60370|16|46|10|PM|second|afternoon|| +60371|AAAAAAAAENLOAAAA|60371|16|46|11|PM|second|afternoon|| +60372|AAAAAAAAFNLOAAAA|60372|16|46|12|PM|second|afternoon|| +60373|AAAAAAAAGNLOAAAA|60373|16|46|13|PM|second|afternoon|| +60374|AAAAAAAAHNLOAAAA|60374|16|46|14|PM|second|afternoon|| +60375|AAAAAAAAINLOAAAA|60375|16|46|15|PM|second|afternoon|| +60376|AAAAAAAAJNLOAAAA|60376|16|46|16|PM|second|afternoon|| +60377|AAAAAAAAKNLOAAAA|60377|16|46|17|PM|second|afternoon|| +60378|AAAAAAAALNLOAAAA|60378|16|46|18|PM|second|afternoon|| +60379|AAAAAAAAMNLOAAAA|60379|16|46|19|PM|second|afternoon|| +60380|AAAAAAAANNLOAAAA|60380|16|46|20|PM|second|afternoon|| +60381|AAAAAAAAONLOAAAA|60381|16|46|21|PM|second|afternoon|| +60382|AAAAAAAAPNLOAAAA|60382|16|46|22|PM|second|afternoon|| +60383|AAAAAAAAAOLOAAAA|60383|16|46|23|PM|second|afternoon|| +60384|AAAAAAAABOLOAAAA|60384|16|46|24|PM|second|afternoon|| +60385|AAAAAAAACOLOAAAA|60385|16|46|25|PM|second|afternoon|| +60386|AAAAAAAADOLOAAAA|60386|16|46|26|PM|second|afternoon|| +60387|AAAAAAAAEOLOAAAA|60387|16|46|27|PM|second|afternoon|| +60388|AAAAAAAAFOLOAAAA|60388|16|46|28|PM|second|afternoon|| +60389|AAAAAAAAGOLOAAAA|60389|16|46|29|PM|second|afternoon|| +60390|AAAAAAAAHOLOAAAA|60390|16|46|30|PM|second|afternoon|| +60391|AAAAAAAAIOLOAAAA|60391|16|46|31|PM|second|afternoon|| +60392|AAAAAAAAJOLOAAAA|60392|16|46|32|PM|second|afternoon|| +60393|AAAAAAAAKOLOAAAA|60393|16|46|33|PM|second|afternoon|| +60394|AAAAAAAALOLOAAAA|60394|16|46|34|PM|second|afternoon|| +60395|AAAAAAAAMOLOAAAA|60395|16|46|35|PM|second|afternoon|| +60396|AAAAAAAANOLOAAAA|60396|16|46|36|PM|second|afternoon|| +60397|AAAAAAAAOOLOAAAA|60397|16|46|37|PM|second|afternoon|| +60398|AAAAAAAAPOLOAAAA|60398|16|46|38|PM|second|afternoon|| +60399|AAAAAAAAAPLOAAAA|60399|16|46|39|PM|second|afternoon|| +60400|AAAAAAAABPLOAAAA|60400|16|46|40|PM|second|afternoon|| +60401|AAAAAAAACPLOAAAA|60401|16|46|41|PM|second|afternoon|| +60402|AAAAAAAADPLOAAAA|60402|16|46|42|PM|second|afternoon|| +60403|AAAAAAAAEPLOAAAA|60403|16|46|43|PM|second|afternoon|| +60404|AAAAAAAAFPLOAAAA|60404|16|46|44|PM|second|afternoon|| +60405|AAAAAAAAGPLOAAAA|60405|16|46|45|PM|second|afternoon|| +60406|AAAAAAAAHPLOAAAA|60406|16|46|46|PM|second|afternoon|| +60407|AAAAAAAAIPLOAAAA|60407|16|46|47|PM|second|afternoon|| +60408|AAAAAAAAJPLOAAAA|60408|16|46|48|PM|second|afternoon|| +60409|AAAAAAAAKPLOAAAA|60409|16|46|49|PM|second|afternoon|| +60410|AAAAAAAALPLOAAAA|60410|16|46|50|PM|second|afternoon|| +60411|AAAAAAAAMPLOAAAA|60411|16|46|51|PM|second|afternoon|| +60412|AAAAAAAANPLOAAAA|60412|16|46|52|PM|second|afternoon|| +60413|AAAAAAAAOPLOAAAA|60413|16|46|53|PM|second|afternoon|| +60414|AAAAAAAAPPLOAAAA|60414|16|46|54|PM|second|afternoon|| +60415|AAAAAAAAAAMOAAAA|60415|16|46|55|PM|second|afternoon|| +60416|AAAAAAAABAMOAAAA|60416|16|46|56|PM|second|afternoon|| +60417|AAAAAAAACAMOAAAA|60417|16|46|57|PM|second|afternoon|| +60418|AAAAAAAADAMOAAAA|60418|16|46|58|PM|second|afternoon|| +60419|AAAAAAAAEAMOAAAA|60419|16|46|59|PM|second|afternoon|| +60420|AAAAAAAAFAMOAAAA|60420|16|47|0|PM|second|afternoon|| +60421|AAAAAAAAGAMOAAAA|60421|16|47|1|PM|second|afternoon|| +60422|AAAAAAAAHAMOAAAA|60422|16|47|2|PM|second|afternoon|| +60423|AAAAAAAAIAMOAAAA|60423|16|47|3|PM|second|afternoon|| +60424|AAAAAAAAJAMOAAAA|60424|16|47|4|PM|second|afternoon|| +60425|AAAAAAAAKAMOAAAA|60425|16|47|5|PM|second|afternoon|| +60426|AAAAAAAALAMOAAAA|60426|16|47|6|PM|second|afternoon|| +60427|AAAAAAAAMAMOAAAA|60427|16|47|7|PM|second|afternoon|| +60428|AAAAAAAANAMOAAAA|60428|16|47|8|PM|second|afternoon|| +60429|AAAAAAAAOAMOAAAA|60429|16|47|9|PM|second|afternoon|| +60430|AAAAAAAAPAMOAAAA|60430|16|47|10|PM|second|afternoon|| +60431|AAAAAAAAABMOAAAA|60431|16|47|11|PM|second|afternoon|| +60432|AAAAAAAABBMOAAAA|60432|16|47|12|PM|second|afternoon|| +60433|AAAAAAAACBMOAAAA|60433|16|47|13|PM|second|afternoon|| +60434|AAAAAAAADBMOAAAA|60434|16|47|14|PM|second|afternoon|| +60435|AAAAAAAAEBMOAAAA|60435|16|47|15|PM|second|afternoon|| +60436|AAAAAAAAFBMOAAAA|60436|16|47|16|PM|second|afternoon|| +60437|AAAAAAAAGBMOAAAA|60437|16|47|17|PM|second|afternoon|| +60438|AAAAAAAAHBMOAAAA|60438|16|47|18|PM|second|afternoon|| +60439|AAAAAAAAIBMOAAAA|60439|16|47|19|PM|second|afternoon|| +60440|AAAAAAAAJBMOAAAA|60440|16|47|20|PM|second|afternoon|| +60441|AAAAAAAAKBMOAAAA|60441|16|47|21|PM|second|afternoon|| +60442|AAAAAAAALBMOAAAA|60442|16|47|22|PM|second|afternoon|| +60443|AAAAAAAAMBMOAAAA|60443|16|47|23|PM|second|afternoon|| +60444|AAAAAAAANBMOAAAA|60444|16|47|24|PM|second|afternoon|| +60445|AAAAAAAAOBMOAAAA|60445|16|47|25|PM|second|afternoon|| +60446|AAAAAAAAPBMOAAAA|60446|16|47|26|PM|second|afternoon|| +60447|AAAAAAAAACMOAAAA|60447|16|47|27|PM|second|afternoon|| +60448|AAAAAAAABCMOAAAA|60448|16|47|28|PM|second|afternoon|| +60449|AAAAAAAACCMOAAAA|60449|16|47|29|PM|second|afternoon|| +60450|AAAAAAAADCMOAAAA|60450|16|47|30|PM|second|afternoon|| +60451|AAAAAAAAECMOAAAA|60451|16|47|31|PM|second|afternoon|| +60452|AAAAAAAAFCMOAAAA|60452|16|47|32|PM|second|afternoon|| +60453|AAAAAAAAGCMOAAAA|60453|16|47|33|PM|second|afternoon|| +60454|AAAAAAAAHCMOAAAA|60454|16|47|34|PM|second|afternoon|| +60455|AAAAAAAAICMOAAAA|60455|16|47|35|PM|second|afternoon|| +60456|AAAAAAAAJCMOAAAA|60456|16|47|36|PM|second|afternoon|| +60457|AAAAAAAAKCMOAAAA|60457|16|47|37|PM|second|afternoon|| +60458|AAAAAAAALCMOAAAA|60458|16|47|38|PM|second|afternoon|| +60459|AAAAAAAAMCMOAAAA|60459|16|47|39|PM|second|afternoon|| +60460|AAAAAAAANCMOAAAA|60460|16|47|40|PM|second|afternoon|| +60461|AAAAAAAAOCMOAAAA|60461|16|47|41|PM|second|afternoon|| +60462|AAAAAAAAPCMOAAAA|60462|16|47|42|PM|second|afternoon|| +60463|AAAAAAAAADMOAAAA|60463|16|47|43|PM|second|afternoon|| +60464|AAAAAAAABDMOAAAA|60464|16|47|44|PM|second|afternoon|| +60465|AAAAAAAACDMOAAAA|60465|16|47|45|PM|second|afternoon|| +60466|AAAAAAAADDMOAAAA|60466|16|47|46|PM|second|afternoon|| +60467|AAAAAAAAEDMOAAAA|60467|16|47|47|PM|second|afternoon|| +60468|AAAAAAAAFDMOAAAA|60468|16|47|48|PM|second|afternoon|| +60469|AAAAAAAAGDMOAAAA|60469|16|47|49|PM|second|afternoon|| +60470|AAAAAAAAHDMOAAAA|60470|16|47|50|PM|second|afternoon|| +60471|AAAAAAAAIDMOAAAA|60471|16|47|51|PM|second|afternoon|| +60472|AAAAAAAAJDMOAAAA|60472|16|47|52|PM|second|afternoon|| +60473|AAAAAAAAKDMOAAAA|60473|16|47|53|PM|second|afternoon|| +60474|AAAAAAAALDMOAAAA|60474|16|47|54|PM|second|afternoon|| +60475|AAAAAAAAMDMOAAAA|60475|16|47|55|PM|second|afternoon|| +60476|AAAAAAAANDMOAAAA|60476|16|47|56|PM|second|afternoon|| +60477|AAAAAAAAODMOAAAA|60477|16|47|57|PM|second|afternoon|| +60478|AAAAAAAAPDMOAAAA|60478|16|47|58|PM|second|afternoon|| +60479|AAAAAAAAAEMOAAAA|60479|16|47|59|PM|second|afternoon|| +60480|AAAAAAAABEMOAAAA|60480|16|48|0|PM|second|afternoon|| +60481|AAAAAAAACEMOAAAA|60481|16|48|1|PM|second|afternoon|| +60482|AAAAAAAADEMOAAAA|60482|16|48|2|PM|second|afternoon|| +60483|AAAAAAAAEEMOAAAA|60483|16|48|3|PM|second|afternoon|| +60484|AAAAAAAAFEMOAAAA|60484|16|48|4|PM|second|afternoon|| +60485|AAAAAAAAGEMOAAAA|60485|16|48|5|PM|second|afternoon|| +60486|AAAAAAAAHEMOAAAA|60486|16|48|6|PM|second|afternoon|| +60487|AAAAAAAAIEMOAAAA|60487|16|48|7|PM|second|afternoon|| +60488|AAAAAAAAJEMOAAAA|60488|16|48|8|PM|second|afternoon|| +60489|AAAAAAAAKEMOAAAA|60489|16|48|9|PM|second|afternoon|| +60490|AAAAAAAALEMOAAAA|60490|16|48|10|PM|second|afternoon|| +60491|AAAAAAAAMEMOAAAA|60491|16|48|11|PM|second|afternoon|| +60492|AAAAAAAANEMOAAAA|60492|16|48|12|PM|second|afternoon|| +60493|AAAAAAAAOEMOAAAA|60493|16|48|13|PM|second|afternoon|| +60494|AAAAAAAAPEMOAAAA|60494|16|48|14|PM|second|afternoon|| +60495|AAAAAAAAAFMOAAAA|60495|16|48|15|PM|second|afternoon|| +60496|AAAAAAAABFMOAAAA|60496|16|48|16|PM|second|afternoon|| +60497|AAAAAAAACFMOAAAA|60497|16|48|17|PM|second|afternoon|| +60498|AAAAAAAADFMOAAAA|60498|16|48|18|PM|second|afternoon|| +60499|AAAAAAAAEFMOAAAA|60499|16|48|19|PM|second|afternoon|| +60500|AAAAAAAAFFMOAAAA|60500|16|48|20|PM|second|afternoon|| +60501|AAAAAAAAGFMOAAAA|60501|16|48|21|PM|second|afternoon|| +60502|AAAAAAAAHFMOAAAA|60502|16|48|22|PM|second|afternoon|| +60503|AAAAAAAAIFMOAAAA|60503|16|48|23|PM|second|afternoon|| +60504|AAAAAAAAJFMOAAAA|60504|16|48|24|PM|second|afternoon|| +60505|AAAAAAAAKFMOAAAA|60505|16|48|25|PM|second|afternoon|| +60506|AAAAAAAALFMOAAAA|60506|16|48|26|PM|second|afternoon|| +60507|AAAAAAAAMFMOAAAA|60507|16|48|27|PM|second|afternoon|| +60508|AAAAAAAANFMOAAAA|60508|16|48|28|PM|second|afternoon|| +60509|AAAAAAAAOFMOAAAA|60509|16|48|29|PM|second|afternoon|| +60510|AAAAAAAAPFMOAAAA|60510|16|48|30|PM|second|afternoon|| +60511|AAAAAAAAAGMOAAAA|60511|16|48|31|PM|second|afternoon|| +60512|AAAAAAAABGMOAAAA|60512|16|48|32|PM|second|afternoon|| +60513|AAAAAAAACGMOAAAA|60513|16|48|33|PM|second|afternoon|| +60514|AAAAAAAADGMOAAAA|60514|16|48|34|PM|second|afternoon|| +60515|AAAAAAAAEGMOAAAA|60515|16|48|35|PM|second|afternoon|| +60516|AAAAAAAAFGMOAAAA|60516|16|48|36|PM|second|afternoon|| +60517|AAAAAAAAGGMOAAAA|60517|16|48|37|PM|second|afternoon|| +60518|AAAAAAAAHGMOAAAA|60518|16|48|38|PM|second|afternoon|| +60519|AAAAAAAAIGMOAAAA|60519|16|48|39|PM|second|afternoon|| +60520|AAAAAAAAJGMOAAAA|60520|16|48|40|PM|second|afternoon|| +60521|AAAAAAAAKGMOAAAA|60521|16|48|41|PM|second|afternoon|| +60522|AAAAAAAALGMOAAAA|60522|16|48|42|PM|second|afternoon|| +60523|AAAAAAAAMGMOAAAA|60523|16|48|43|PM|second|afternoon|| +60524|AAAAAAAANGMOAAAA|60524|16|48|44|PM|second|afternoon|| +60525|AAAAAAAAOGMOAAAA|60525|16|48|45|PM|second|afternoon|| +60526|AAAAAAAAPGMOAAAA|60526|16|48|46|PM|second|afternoon|| +60527|AAAAAAAAAHMOAAAA|60527|16|48|47|PM|second|afternoon|| +60528|AAAAAAAABHMOAAAA|60528|16|48|48|PM|second|afternoon|| +60529|AAAAAAAACHMOAAAA|60529|16|48|49|PM|second|afternoon|| +60530|AAAAAAAADHMOAAAA|60530|16|48|50|PM|second|afternoon|| +60531|AAAAAAAAEHMOAAAA|60531|16|48|51|PM|second|afternoon|| +60532|AAAAAAAAFHMOAAAA|60532|16|48|52|PM|second|afternoon|| +60533|AAAAAAAAGHMOAAAA|60533|16|48|53|PM|second|afternoon|| +60534|AAAAAAAAHHMOAAAA|60534|16|48|54|PM|second|afternoon|| +60535|AAAAAAAAIHMOAAAA|60535|16|48|55|PM|second|afternoon|| +60536|AAAAAAAAJHMOAAAA|60536|16|48|56|PM|second|afternoon|| +60537|AAAAAAAAKHMOAAAA|60537|16|48|57|PM|second|afternoon|| +60538|AAAAAAAALHMOAAAA|60538|16|48|58|PM|second|afternoon|| +60539|AAAAAAAAMHMOAAAA|60539|16|48|59|PM|second|afternoon|| +60540|AAAAAAAANHMOAAAA|60540|16|49|0|PM|second|afternoon|| +60541|AAAAAAAAOHMOAAAA|60541|16|49|1|PM|second|afternoon|| +60542|AAAAAAAAPHMOAAAA|60542|16|49|2|PM|second|afternoon|| +60543|AAAAAAAAAIMOAAAA|60543|16|49|3|PM|second|afternoon|| +60544|AAAAAAAABIMOAAAA|60544|16|49|4|PM|second|afternoon|| +60545|AAAAAAAACIMOAAAA|60545|16|49|5|PM|second|afternoon|| +60546|AAAAAAAADIMOAAAA|60546|16|49|6|PM|second|afternoon|| +60547|AAAAAAAAEIMOAAAA|60547|16|49|7|PM|second|afternoon|| +60548|AAAAAAAAFIMOAAAA|60548|16|49|8|PM|second|afternoon|| +60549|AAAAAAAAGIMOAAAA|60549|16|49|9|PM|second|afternoon|| +60550|AAAAAAAAHIMOAAAA|60550|16|49|10|PM|second|afternoon|| +60551|AAAAAAAAIIMOAAAA|60551|16|49|11|PM|second|afternoon|| +60552|AAAAAAAAJIMOAAAA|60552|16|49|12|PM|second|afternoon|| +60553|AAAAAAAAKIMOAAAA|60553|16|49|13|PM|second|afternoon|| +60554|AAAAAAAALIMOAAAA|60554|16|49|14|PM|second|afternoon|| +60555|AAAAAAAAMIMOAAAA|60555|16|49|15|PM|second|afternoon|| +60556|AAAAAAAANIMOAAAA|60556|16|49|16|PM|second|afternoon|| +60557|AAAAAAAAOIMOAAAA|60557|16|49|17|PM|second|afternoon|| +60558|AAAAAAAAPIMOAAAA|60558|16|49|18|PM|second|afternoon|| +60559|AAAAAAAAAJMOAAAA|60559|16|49|19|PM|second|afternoon|| +60560|AAAAAAAABJMOAAAA|60560|16|49|20|PM|second|afternoon|| +60561|AAAAAAAACJMOAAAA|60561|16|49|21|PM|second|afternoon|| +60562|AAAAAAAADJMOAAAA|60562|16|49|22|PM|second|afternoon|| +60563|AAAAAAAAEJMOAAAA|60563|16|49|23|PM|second|afternoon|| +60564|AAAAAAAAFJMOAAAA|60564|16|49|24|PM|second|afternoon|| +60565|AAAAAAAAGJMOAAAA|60565|16|49|25|PM|second|afternoon|| +60566|AAAAAAAAHJMOAAAA|60566|16|49|26|PM|second|afternoon|| +60567|AAAAAAAAIJMOAAAA|60567|16|49|27|PM|second|afternoon|| +60568|AAAAAAAAJJMOAAAA|60568|16|49|28|PM|second|afternoon|| +60569|AAAAAAAAKJMOAAAA|60569|16|49|29|PM|second|afternoon|| +60570|AAAAAAAALJMOAAAA|60570|16|49|30|PM|second|afternoon|| +60571|AAAAAAAAMJMOAAAA|60571|16|49|31|PM|second|afternoon|| +60572|AAAAAAAANJMOAAAA|60572|16|49|32|PM|second|afternoon|| +60573|AAAAAAAAOJMOAAAA|60573|16|49|33|PM|second|afternoon|| +60574|AAAAAAAAPJMOAAAA|60574|16|49|34|PM|second|afternoon|| +60575|AAAAAAAAAKMOAAAA|60575|16|49|35|PM|second|afternoon|| +60576|AAAAAAAABKMOAAAA|60576|16|49|36|PM|second|afternoon|| +60577|AAAAAAAACKMOAAAA|60577|16|49|37|PM|second|afternoon|| +60578|AAAAAAAADKMOAAAA|60578|16|49|38|PM|second|afternoon|| +60579|AAAAAAAAEKMOAAAA|60579|16|49|39|PM|second|afternoon|| +60580|AAAAAAAAFKMOAAAA|60580|16|49|40|PM|second|afternoon|| +60581|AAAAAAAAGKMOAAAA|60581|16|49|41|PM|second|afternoon|| +60582|AAAAAAAAHKMOAAAA|60582|16|49|42|PM|second|afternoon|| +60583|AAAAAAAAIKMOAAAA|60583|16|49|43|PM|second|afternoon|| +60584|AAAAAAAAJKMOAAAA|60584|16|49|44|PM|second|afternoon|| +60585|AAAAAAAAKKMOAAAA|60585|16|49|45|PM|second|afternoon|| +60586|AAAAAAAALKMOAAAA|60586|16|49|46|PM|second|afternoon|| +60587|AAAAAAAAMKMOAAAA|60587|16|49|47|PM|second|afternoon|| +60588|AAAAAAAANKMOAAAA|60588|16|49|48|PM|second|afternoon|| +60589|AAAAAAAAOKMOAAAA|60589|16|49|49|PM|second|afternoon|| +60590|AAAAAAAAPKMOAAAA|60590|16|49|50|PM|second|afternoon|| +60591|AAAAAAAAALMOAAAA|60591|16|49|51|PM|second|afternoon|| +60592|AAAAAAAABLMOAAAA|60592|16|49|52|PM|second|afternoon|| +60593|AAAAAAAACLMOAAAA|60593|16|49|53|PM|second|afternoon|| +60594|AAAAAAAADLMOAAAA|60594|16|49|54|PM|second|afternoon|| +60595|AAAAAAAAELMOAAAA|60595|16|49|55|PM|second|afternoon|| +60596|AAAAAAAAFLMOAAAA|60596|16|49|56|PM|second|afternoon|| +60597|AAAAAAAAGLMOAAAA|60597|16|49|57|PM|second|afternoon|| +60598|AAAAAAAAHLMOAAAA|60598|16|49|58|PM|second|afternoon|| +60599|AAAAAAAAILMOAAAA|60599|16|49|59|PM|second|afternoon|| +60600|AAAAAAAAJLMOAAAA|60600|16|50|0|PM|second|afternoon|| +60601|AAAAAAAAKLMOAAAA|60601|16|50|1|PM|second|afternoon|| +60602|AAAAAAAALLMOAAAA|60602|16|50|2|PM|second|afternoon|| +60603|AAAAAAAAMLMOAAAA|60603|16|50|3|PM|second|afternoon|| +60604|AAAAAAAANLMOAAAA|60604|16|50|4|PM|second|afternoon|| +60605|AAAAAAAAOLMOAAAA|60605|16|50|5|PM|second|afternoon|| +60606|AAAAAAAAPLMOAAAA|60606|16|50|6|PM|second|afternoon|| +60607|AAAAAAAAAMMOAAAA|60607|16|50|7|PM|second|afternoon|| +60608|AAAAAAAABMMOAAAA|60608|16|50|8|PM|second|afternoon|| +60609|AAAAAAAACMMOAAAA|60609|16|50|9|PM|second|afternoon|| +60610|AAAAAAAADMMOAAAA|60610|16|50|10|PM|second|afternoon|| +60611|AAAAAAAAEMMOAAAA|60611|16|50|11|PM|second|afternoon|| +60612|AAAAAAAAFMMOAAAA|60612|16|50|12|PM|second|afternoon|| +60613|AAAAAAAAGMMOAAAA|60613|16|50|13|PM|second|afternoon|| +60614|AAAAAAAAHMMOAAAA|60614|16|50|14|PM|second|afternoon|| +60615|AAAAAAAAIMMOAAAA|60615|16|50|15|PM|second|afternoon|| +60616|AAAAAAAAJMMOAAAA|60616|16|50|16|PM|second|afternoon|| +60617|AAAAAAAAKMMOAAAA|60617|16|50|17|PM|second|afternoon|| +60618|AAAAAAAALMMOAAAA|60618|16|50|18|PM|second|afternoon|| +60619|AAAAAAAAMMMOAAAA|60619|16|50|19|PM|second|afternoon|| +60620|AAAAAAAANMMOAAAA|60620|16|50|20|PM|second|afternoon|| +60621|AAAAAAAAOMMOAAAA|60621|16|50|21|PM|second|afternoon|| +60622|AAAAAAAAPMMOAAAA|60622|16|50|22|PM|second|afternoon|| +60623|AAAAAAAAANMOAAAA|60623|16|50|23|PM|second|afternoon|| +60624|AAAAAAAABNMOAAAA|60624|16|50|24|PM|second|afternoon|| +60625|AAAAAAAACNMOAAAA|60625|16|50|25|PM|second|afternoon|| +60626|AAAAAAAADNMOAAAA|60626|16|50|26|PM|second|afternoon|| +60627|AAAAAAAAENMOAAAA|60627|16|50|27|PM|second|afternoon|| +60628|AAAAAAAAFNMOAAAA|60628|16|50|28|PM|second|afternoon|| +60629|AAAAAAAAGNMOAAAA|60629|16|50|29|PM|second|afternoon|| +60630|AAAAAAAAHNMOAAAA|60630|16|50|30|PM|second|afternoon|| +60631|AAAAAAAAINMOAAAA|60631|16|50|31|PM|second|afternoon|| +60632|AAAAAAAAJNMOAAAA|60632|16|50|32|PM|second|afternoon|| +60633|AAAAAAAAKNMOAAAA|60633|16|50|33|PM|second|afternoon|| +60634|AAAAAAAALNMOAAAA|60634|16|50|34|PM|second|afternoon|| +60635|AAAAAAAAMNMOAAAA|60635|16|50|35|PM|second|afternoon|| +60636|AAAAAAAANNMOAAAA|60636|16|50|36|PM|second|afternoon|| +60637|AAAAAAAAONMOAAAA|60637|16|50|37|PM|second|afternoon|| +60638|AAAAAAAAPNMOAAAA|60638|16|50|38|PM|second|afternoon|| +60639|AAAAAAAAAOMOAAAA|60639|16|50|39|PM|second|afternoon|| +60640|AAAAAAAABOMOAAAA|60640|16|50|40|PM|second|afternoon|| +60641|AAAAAAAACOMOAAAA|60641|16|50|41|PM|second|afternoon|| +60642|AAAAAAAADOMOAAAA|60642|16|50|42|PM|second|afternoon|| +60643|AAAAAAAAEOMOAAAA|60643|16|50|43|PM|second|afternoon|| +60644|AAAAAAAAFOMOAAAA|60644|16|50|44|PM|second|afternoon|| +60645|AAAAAAAAGOMOAAAA|60645|16|50|45|PM|second|afternoon|| +60646|AAAAAAAAHOMOAAAA|60646|16|50|46|PM|second|afternoon|| +60647|AAAAAAAAIOMOAAAA|60647|16|50|47|PM|second|afternoon|| +60648|AAAAAAAAJOMOAAAA|60648|16|50|48|PM|second|afternoon|| +60649|AAAAAAAAKOMOAAAA|60649|16|50|49|PM|second|afternoon|| +60650|AAAAAAAALOMOAAAA|60650|16|50|50|PM|second|afternoon|| +60651|AAAAAAAAMOMOAAAA|60651|16|50|51|PM|second|afternoon|| +60652|AAAAAAAANOMOAAAA|60652|16|50|52|PM|second|afternoon|| +60653|AAAAAAAAOOMOAAAA|60653|16|50|53|PM|second|afternoon|| +60654|AAAAAAAAPOMOAAAA|60654|16|50|54|PM|second|afternoon|| +60655|AAAAAAAAAPMOAAAA|60655|16|50|55|PM|second|afternoon|| +60656|AAAAAAAABPMOAAAA|60656|16|50|56|PM|second|afternoon|| +60657|AAAAAAAACPMOAAAA|60657|16|50|57|PM|second|afternoon|| +60658|AAAAAAAADPMOAAAA|60658|16|50|58|PM|second|afternoon|| +60659|AAAAAAAAEPMOAAAA|60659|16|50|59|PM|second|afternoon|| +60660|AAAAAAAAFPMOAAAA|60660|16|51|0|PM|second|afternoon|| +60661|AAAAAAAAGPMOAAAA|60661|16|51|1|PM|second|afternoon|| +60662|AAAAAAAAHPMOAAAA|60662|16|51|2|PM|second|afternoon|| +60663|AAAAAAAAIPMOAAAA|60663|16|51|3|PM|second|afternoon|| +60664|AAAAAAAAJPMOAAAA|60664|16|51|4|PM|second|afternoon|| +60665|AAAAAAAAKPMOAAAA|60665|16|51|5|PM|second|afternoon|| +60666|AAAAAAAALPMOAAAA|60666|16|51|6|PM|second|afternoon|| +60667|AAAAAAAAMPMOAAAA|60667|16|51|7|PM|second|afternoon|| +60668|AAAAAAAANPMOAAAA|60668|16|51|8|PM|second|afternoon|| +60669|AAAAAAAAOPMOAAAA|60669|16|51|9|PM|second|afternoon|| +60670|AAAAAAAAPPMOAAAA|60670|16|51|10|PM|second|afternoon|| +60671|AAAAAAAAAANOAAAA|60671|16|51|11|PM|second|afternoon|| +60672|AAAAAAAABANOAAAA|60672|16|51|12|PM|second|afternoon|| +60673|AAAAAAAACANOAAAA|60673|16|51|13|PM|second|afternoon|| +60674|AAAAAAAADANOAAAA|60674|16|51|14|PM|second|afternoon|| +60675|AAAAAAAAEANOAAAA|60675|16|51|15|PM|second|afternoon|| +60676|AAAAAAAAFANOAAAA|60676|16|51|16|PM|second|afternoon|| +60677|AAAAAAAAGANOAAAA|60677|16|51|17|PM|second|afternoon|| +60678|AAAAAAAAHANOAAAA|60678|16|51|18|PM|second|afternoon|| +60679|AAAAAAAAIANOAAAA|60679|16|51|19|PM|second|afternoon|| +60680|AAAAAAAAJANOAAAA|60680|16|51|20|PM|second|afternoon|| +60681|AAAAAAAAKANOAAAA|60681|16|51|21|PM|second|afternoon|| +60682|AAAAAAAALANOAAAA|60682|16|51|22|PM|second|afternoon|| +60683|AAAAAAAAMANOAAAA|60683|16|51|23|PM|second|afternoon|| +60684|AAAAAAAANANOAAAA|60684|16|51|24|PM|second|afternoon|| +60685|AAAAAAAAOANOAAAA|60685|16|51|25|PM|second|afternoon|| +60686|AAAAAAAAPANOAAAA|60686|16|51|26|PM|second|afternoon|| +60687|AAAAAAAAABNOAAAA|60687|16|51|27|PM|second|afternoon|| +60688|AAAAAAAABBNOAAAA|60688|16|51|28|PM|second|afternoon|| +60689|AAAAAAAACBNOAAAA|60689|16|51|29|PM|second|afternoon|| +60690|AAAAAAAADBNOAAAA|60690|16|51|30|PM|second|afternoon|| +60691|AAAAAAAAEBNOAAAA|60691|16|51|31|PM|second|afternoon|| +60692|AAAAAAAAFBNOAAAA|60692|16|51|32|PM|second|afternoon|| +60693|AAAAAAAAGBNOAAAA|60693|16|51|33|PM|second|afternoon|| +60694|AAAAAAAAHBNOAAAA|60694|16|51|34|PM|second|afternoon|| +60695|AAAAAAAAIBNOAAAA|60695|16|51|35|PM|second|afternoon|| +60696|AAAAAAAAJBNOAAAA|60696|16|51|36|PM|second|afternoon|| +60697|AAAAAAAAKBNOAAAA|60697|16|51|37|PM|second|afternoon|| +60698|AAAAAAAALBNOAAAA|60698|16|51|38|PM|second|afternoon|| +60699|AAAAAAAAMBNOAAAA|60699|16|51|39|PM|second|afternoon|| +60700|AAAAAAAANBNOAAAA|60700|16|51|40|PM|second|afternoon|| +60701|AAAAAAAAOBNOAAAA|60701|16|51|41|PM|second|afternoon|| +60702|AAAAAAAAPBNOAAAA|60702|16|51|42|PM|second|afternoon|| +60703|AAAAAAAAACNOAAAA|60703|16|51|43|PM|second|afternoon|| +60704|AAAAAAAABCNOAAAA|60704|16|51|44|PM|second|afternoon|| +60705|AAAAAAAACCNOAAAA|60705|16|51|45|PM|second|afternoon|| +60706|AAAAAAAADCNOAAAA|60706|16|51|46|PM|second|afternoon|| +60707|AAAAAAAAECNOAAAA|60707|16|51|47|PM|second|afternoon|| +60708|AAAAAAAAFCNOAAAA|60708|16|51|48|PM|second|afternoon|| +60709|AAAAAAAAGCNOAAAA|60709|16|51|49|PM|second|afternoon|| +60710|AAAAAAAAHCNOAAAA|60710|16|51|50|PM|second|afternoon|| +60711|AAAAAAAAICNOAAAA|60711|16|51|51|PM|second|afternoon|| +60712|AAAAAAAAJCNOAAAA|60712|16|51|52|PM|second|afternoon|| +60713|AAAAAAAAKCNOAAAA|60713|16|51|53|PM|second|afternoon|| +60714|AAAAAAAALCNOAAAA|60714|16|51|54|PM|second|afternoon|| +60715|AAAAAAAAMCNOAAAA|60715|16|51|55|PM|second|afternoon|| +60716|AAAAAAAANCNOAAAA|60716|16|51|56|PM|second|afternoon|| +60717|AAAAAAAAOCNOAAAA|60717|16|51|57|PM|second|afternoon|| +60718|AAAAAAAAPCNOAAAA|60718|16|51|58|PM|second|afternoon|| +60719|AAAAAAAAADNOAAAA|60719|16|51|59|PM|second|afternoon|| +60720|AAAAAAAABDNOAAAA|60720|16|52|0|PM|second|afternoon|| +60721|AAAAAAAACDNOAAAA|60721|16|52|1|PM|second|afternoon|| +60722|AAAAAAAADDNOAAAA|60722|16|52|2|PM|second|afternoon|| +60723|AAAAAAAAEDNOAAAA|60723|16|52|3|PM|second|afternoon|| +60724|AAAAAAAAFDNOAAAA|60724|16|52|4|PM|second|afternoon|| +60725|AAAAAAAAGDNOAAAA|60725|16|52|5|PM|second|afternoon|| +60726|AAAAAAAAHDNOAAAA|60726|16|52|6|PM|second|afternoon|| +60727|AAAAAAAAIDNOAAAA|60727|16|52|7|PM|second|afternoon|| +60728|AAAAAAAAJDNOAAAA|60728|16|52|8|PM|second|afternoon|| +60729|AAAAAAAAKDNOAAAA|60729|16|52|9|PM|second|afternoon|| +60730|AAAAAAAALDNOAAAA|60730|16|52|10|PM|second|afternoon|| +60731|AAAAAAAAMDNOAAAA|60731|16|52|11|PM|second|afternoon|| +60732|AAAAAAAANDNOAAAA|60732|16|52|12|PM|second|afternoon|| +60733|AAAAAAAAODNOAAAA|60733|16|52|13|PM|second|afternoon|| +60734|AAAAAAAAPDNOAAAA|60734|16|52|14|PM|second|afternoon|| +60735|AAAAAAAAAENOAAAA|60735|16|52|15|PM|second|afternoon|| +60736|AAAAAAAABENOAAAA|60736|16|52|16|PM|second|afternoon|| +60737|AAAAAAAACENOAAAA|60737|16|52|17|PM|second|afternoon|| +60738|AAAAAAAADENOAAAA|60738|16|52|18|PM|second|afternoon|| +60739|AAAAAAAAEENOAAAA|60739|16|52|19|PM|second|afternoon|| +60740|AAAAAAAAFENOAAAA|60740|16|52|20|PM|second|afternoon|| +60741|AAAAAAAAGENOAAAA|60741|16|52|21|PM|second|afternoon|| +60742|AAAAAAAAHENOAAAA|60742|16|52|22|PM|second|afternoon|| +60743|AAAAAAAAIENOAAAA|60743|16|52|23|PM|second|afternoon|| +60744|AAAAAAAAJENOAAAA|60744|16|52|24|PM|second|afternoon|| +60745|AAAAAAAAKENOAAAA|60745|16|52|25|PM|second|afternoon|| +60746|AAAAAAAALENOAAAA|60746|16|52|26|PM|second|afternoon|| +60747|AAAAAAAAMENOAAAA|60747|16|52|27|PM|second|afternoon|| +60748|AAAAAAAANENOAAAA|60748|16|52|28|PM|second|afternoon|| +60749|AAAAAAAAOENOAAAA|60749|16|52|29|PM|second|afternoon|| +60750|AAAAAAAAPENOAAAA|60750|16|52|30|PM|second|afternoon|| +60751|AAAAAAAAAFNOAAAA|60751|16|52|31|PM|second|afternoon|| +60752|AAAAAAAABFNOAAAA|60752|16|52|32|PM|second|afternoon|| +60753|AAAAAAAACFNOAAAA|60753|16|52|33|PM|second|afternoon|| +60754|AAAAAAAADFNOAAAA|60754|16|52|34|PM|second|afternoon|| +60755|AAAAAAAAEFNOAAAA|60755|16|52|35|PM|second|afternoon|| +60756|AAAAAAAAFFNOAAAA|60756|16|52|36|PM|second|afternoon|| +60757|AAAAAAAAGFNOAAAA|60757|16|52|37|PM|second|afternoon|| +60758|AAAAAAAAHFNOAAAA|60758|16|52|38|PM|second|afternoon|| +60759|AAAAAAAAIFNOAAAA|60759|16|52|39|PM|second|afternoon|| +60760|AAAAAAAAJFNOAAAA|60760|16|52|40|PM|second|afternoon|| +60761|AAAAAAAAKFNOAAAA|60761|16|52|41|PM|second|afternoon|| +60762|AAAAAAAALFNOAAAA|60762|16|52|42|PM|second|afternoon|| +60763|AAAAAAAAMFNOAAAA|60763|16|52|43|PM|second|afternoon|| +60764|AAAAAAAANFNOAAAA|60764|16|52|44|PM|second|afternoon|| +60765|AAAAAAAAOFNOAAAA|60765|16|52|45|PM|second|afternoon|| +60766|AAAAAAAAPFNOAAAA|60766|16|52|46|PM|second|afternoon|| +60767|AAAAAAAAAGNOAAAA|60767|16|52|47|PM|second|afternoon|| +60768|AAAAAAAABGNOAAAA|60768|16|52|48|PM|second|afternoon|| +60769|AAAAAAAACGNOAAAA|60769|16|52|49|PM|second|afternoon|| +60770|AAAAAAAADGNOAAAA|60770|16|52|50|PM|second|afternoon|| +60771|AAAAAAAAEGNOAAAA|60771|16|52|51|PM|second|afternoon|| +60772|AAAAAAAAFGNOAAAA|60772|16|52|52|PM|second|afternoon|| +60773|AAAAAAAAGGNOAAAA|60773|16|52|53|PM|second|afternoon|| +60774|AAAAAAAAHGNOAAAA|60774|16|52|54|PM|second|afternoon|| +60775|AAAAAAAAIGNOAAAA|60775|16|52|55|PM|second|afternoon|| +60776|AAAAAAAAJGNOAAAA|60776|16|52|56|PM|second|afternoon|| +60777|AAAAAAAAKGNOAAAA|60777|16|52|57|PM|second|afternoon|| +60778|AAAAAAAALGNOAAAA|60778|16|52|58|PM|second|afternoon|| +60779|AAAAAAAAMGNOAAAA|60779|16|52|59|PM|second|afternoon|| +60780|AAAAAAAANGNOAAAA|60780|16|53|0|PM|second|afternoon|| +60781|AAAAAAAAOGNOAAAA|60781|16|53|1|PM|second|afternoon|| +60782|AAAAAAAAPGNOAAAA|60782|16|53|2|PM|second|afternoon|| +60783|AAAAAAAAAHNOAAAA|60783|16|53|3|PM|second|afternoon|| +60784|AAAAAAAABHNOAAAA|60784|16|53|4|PM|second|afternoon|| +60785|AAAAAAAACHNOAAAA|60785|16|53|5|PM|second|afternoon|| +60786|AAAAAAAADHNOAAAA|60786|16|53|6|PM|second|afternoon|| +60787|AAAAAAAAEHNOAAAA|60787|16|53|7|PM|second|afternoon|| +60788|AAAAAAAAFHNOAAAA|60788|16|53|8|PM|second|afternoon|| +60789|AAAAAAAAGHNOAAAA|60789|16|53|9|PM|second|afternoon|| +60790|AAAAAAAAHHNOAAAA|60790|16|53|10|PM|second|afternoon|| +60791|AAAAAAAAIHNOAAAA|60791|16|53|11|PM|second|afternoon|| +60792|AAAAAAAAJHNOAAAA|60792|16|53|12|PM|second|afternoon|| +60793|AAAAAAAAKHNOAAAA|60793|16|53|13|PM|second|afternoon|| +60794|AAAAAAAALHNOAAAA|60794|16|53|14|PM|second|afternoon|| +60795|AAAAAAAAMHNOAAAA|60795|16|53|15|PM|second|afternoon|| +60796|AAAAAAAANHNOAAAA|60796|16|53|16|PM|second|afternoon|| +60797|AAAAAAAAOHNOAAAA|60797|16|53|17|PM|second|afternoon|| +60798|AAAAAAAAPHNOAAAA|60798|16|53|18|PM|second|afternoon|| +60799|AAAAAAAAAINOAAAA|60799|16|53|19|PM|second|afternoon|| +60800|AAAAAAAABINOAAAA|60800|16|53|20|PM|second|afternoon|| +60801|AAAAAAAACINOAAAA|60801|16|53|21|PM|second|afternoon|| +60802|AAAAAAAADINOAAAA|60802|16|53|22|PM|second|afternoon|| +60803|AAAAAAAAEINOAAAA|60803|16|53|23|PM|second|afternoon|| +60804|AAAAAAAAFINOAAAA|60804|16|53|24|PM|second|afternoon|| +60805|AAAAAAAAGINOAAAA|60805|16|53|25|PM|second|afternoon|| +60806|AAAAAAAAHINOAAAA|60806|16|53|26|PM|second|afternoon|| +60807|AAAAAAAAIINOAAAA|60807|16|53|27|PM|second|afternoon|| +60808|AAAAAAAAJINOAAAA|60808|16|53|28|PM|second|afternoon|| +60809|AAAAAAAAKINOAAAA|60809|16|53|29|PM|second|afternoon|| +60810|AAAAAAAALINOAAAA|60810|16|53|30|PM|second|afternoon|| +60811|AAAAAAAAMINOAAAA|60811|16|53|31|PM|second|afternoon|| +60812|AAAAAAAANINOAAAA|60812|16|53|32|PM|second|afternoon|| +60813|AAAAAAAAOINOAAAA|60813|16|53|33|PM|second|afternoon|| +60814|AAAAAAAAPINOAAAA|60814|16|53|34|PM|second|afternoon|| +60815|AAAAAAAAAJNOAAAA|60815|16|53|35|PM|second|afternoon|| +60816|AAAAAAAABJNOAAAA|60816|16|53|36|PM|second|afternoon|| +60817|AAAAAAAACJNOAAAA|60817|16|53|37|PM|second|afternoon|| +60818|AAAAAAAADJNOAAAA|60818|16|53|38|PM|second|afternoon|| +60819|AAAAAAAAEJNOAAAA|60819|16|53|39|PM|second|afternoon|| +60820|AAAAAAAAFJNOAAAA|60820|16|53|40|PM|second|afternoon|| +60821|AAAAAAAAGJNOAAAA|60821|16|53|41|PM|second|afternoon|| +60822|AAAAAAAAHJNOAAAA|60822|16|53|42|PM|second|afternoon|| +60823|AAAAAAAAIJNOAAAA|60823|16|53|43|PM|second|afternoon|| +60824|AAAAAAAAJJNOAAAA|60824|16|53|44|PM|second|afternoon|| +60825|AAAAAAAAKJNOAAAA|60825|16|53|45|PM|second|afternoon|| +60826|AAAAAAAALJNOAAAA|60826|16|53|46|PM|second|afternoon|| +60827|AAAAAAAAMJNOAAAA|60827|16|53|47|PM|second|afternoon|| +60828|AAAAAAAANJNOAAAA|60828|16|53|48|PM|second|afternoon|| +60829|AAAAAAAAOJNOAAAA|60829|16|53|49|PM|second|afternoon|| +60830|AAAAAAAAPJNOAAAA|60830|16|53|50|PM|second|afternoon|| +60831|AAAAAAAAAKNOAAAA|60831|16|53|51|PM|second|afternoon|| +60832|AAAAAAAABKNOAAAA|60832|16|53|52|PM|second|afternoon|| +60833|AAAAAAAACKNOAAAA|60833|16|53|53|PM|second|afternoon|| +60834|AAAAAAAADKNOAAAA|60834|16|53|54|PM|second|afternoon|| +60835|AAAAAAAAEKNOAAAA|60835|16|53|55|PM|second|afternoon|| +60836|AAAAAAAAFKNOAAAA|60836|16|53|56|PM|second|afternoon|| +60837|AAAAAAAAGKNOAAAA|60837|16|53|57|PM|second|afternoon|| +60838|AAAAAAAAHKNOAAAA|60838|16|53|58|PM|second|afternoon|| +60839|AAAAAAAAIKNOAAAA|60839|16|53|59|PM|second|afternoon|| +60840|AAAAAAAAJKNOAAAA|60840|16|54|0|PM|second|afternoon|| +60841|AAAAAAAAKKNOAAAA|60841|16|54|1|PM|second|afternoon|| +60842|AAAAAAAALKNOAAAA|60842|16|54|2|PM|second|afternoon|| +60843|AAAAAAAAMKNOAAAA|60843|16|54|3|PM|second|afternoon|| +60844|AAAAAAAANKNOAAAA|60844|16|54|4|PM|second|afternoon|| +60845|AAAAAAAAOKNOAAAA|60845|16|54|5|PM|second|afternoon|| +60846|AAAAAAAAPKNOAAAA|60846|16|54|6|PM|second|afternoon|| +60847|AAAAAAAAALNOAAAA|60847|16|54|7|PM|second|afternoon|| +60848|AAAAAAAABLNOAAAA|60848|16|54|8|PM|second|afternoon|| +60849|AAAAAAAACLNOAAAA|60849|16|54|9|PM|second|afternoon|| +60850|AAAAAAAADLNOAAAA|60850|16|54|10|PM|second|afternoon|| +60851|AAAAAAAAELNOAAAA|60851|16|54|11|PM|second|afternoon|| +60852|AAAAAAAAFLNOAAAA|60852|16|54|12|PM|second|afternoon|| +60853|AAAAAAAAGLNOAAAA|60853|16|54|13|PM|second|afternoon|| +60854|AAAAAAAAHLNOAAAA|60854|16|54|14|PM|second|afternoon|| +60855|AAAAAAAAILNOAAAA|60855|16|54|15|PM|second|afternoon|| +60856|AAAAAAAAJLNOAAAA|60856|16|54|16|PM|second|afternoon|| +60857|AAAAAAAAKLNOAAAA|60857|16|54|17|PM|second|afternoon|| +60858|AAAAAAAALLNOAAAA|60858|16|54|18|PM|second|afternoon|| +60859|AAAAAAAAMLNOAAAA|60859|16|54|19|PM|second|afternoon|| +60860|AAAAAAAANLNOAAAA|60860|16|54|20|PM|second|afternoon|| +60861|AAAAAAAAOLNOAAAA|60861|16|54|21|PM|second|afternoon|| +60862|AAAAAAAAPLNOAAAA|60862|16|54|22|PM|second|afternoon|| +60863|AAAAAAAAAMNOAAAA|60863|16|54|23|PM|second|afternoon|| +60864|AAAAAAAABMNOAAAA|60864|16|54|24|PM|second|afternoon|| +60865|AAAAAAAACMNOAAAA|60865|16|54|25|PM|second|afternoon|| +60866|AAAAAAAADMNOAAAA|60866|16|54|26|PM|second|afternoon|| +60867|AAAAAAAAEMNOAAAA|60867|16|54|27|PM|second|afternoon|| +60868|AAAAAAAAFMNOAAAA|60868|16|54|28|PM|second|afternoon|| +60869|AAAAAAAAGMNOAAAA|60869|16|54|29|PM|second|afternoon|| +60870|AAAAAAAAHMNOAAAA|60870|16|54|30|PM|second|afternoon|| +60871|AAAAAAAAIMNOAAAA|60871|16|54|31|PM|second|afternoon|| +60872|AAAAAAAAJMNOAAAA|60872|16|54|32|PM|second|afternoon|| +60873|AAAAAAAAKMNOAAAA|60873|16|54|33|PM|second|afternoon|| +60874|AAAAAAAALMNOAAAA|60874|16|54|34|PM|second|afternoon|| +60875|AAAAAAAAMMNOAAAA|60875|16|54|35|PM|second|afternoon|| +60876|AAAAAAAANMNOAAAA|60876|16|54|36|PM|second|afternoon|| +60877|AAAAAAAAOMNOAAAA|60877|16|54|37|PM|second|afternoon|| +60878|AAAAAAAAPMNOAAAA|60878|16|54|38|PM|second|afternoon|| +60879|AAAAAAAAANNOAAAA|60879|16|54|39|PM|second|afternoon|| +60880|AAAAAAAABNNOAAAA|60880|16|54|40|PM|second|afternoon|| +60881|AAAAAAAACNNOAAAA|60881|16|54|41|PM|second|afternoon|| +60882|AAAAAAAADNNOAAAA|60882|16|54|42|PM|second|afternoon|| +60883|AAAAAAAAENNOAAAA|60883|16|54|43|PM|second|afternoon|| +60884|AAAAAAAAFNNOAAAA|60884|16|54|44|PM|second|afternoon|| +60885|AAAAAAAAGNNOAAAA|60885|16|54|45|PM|second|afternoon|| +60886|AAAAAAAAHNNOAAAA|60886|16|54|46|PM|second|afternoon|| +60887|AAAAAAAAINNOAAAA|60887|16|54|47|PM|second|afternoon|| +60888|AAAAAAAAJNNOAAAA|60888|16|54|48|PM|second|afternoon|| +60889|AAAAAAAAKNNOAAAA|60889|16|54|49|PM|second|afternoon|| +60890|AAAAAAAALNNOAAAA|60890|16|54|50|PM|second|afternoon|| +60891|AAAAAAAAMNNOAAAA|60891|16|54|51|PM|second|afternoon|| +60892|AAAAAAAANNNOAAAA|60892|16|54|52|PM|second|afternoon|| +60893|AAAAAAAAONNOAAAA|60893|16|54|53|PM|second|afternoon|| +60894|AAAAAAAAPNNOAAAA|60894|16|54|54|PM|second|afternoon|| +60895|AAAAAAAAAONOAAAA|60895|16|54|55|PM|second|afternoon|| +60896|AAAAAAAABONOAAAA|60896|16|54|56|PM|second|afternoon|| +60897|AAAAAAAACONOAAAA|60897|16|54|57|PM|second|afternoon|| +60898|AAAAAAAADONOAAAA|60898|16|54|58|PM|second|afternoon|| +60899|AAAAAAAAEONOAAAA|60899|16|54|59|PM|second|afternoon|| +60900|AAAAAAAAFONOAAAA|60900|16|55|0|PM|second|afternoon|| +60901|AAAAAAAAGONOAAAA|60901|16|55|1|PM|second|afternoon|| +60902|AAAAAAAAHONOAAAA|60902|16|55|2|PM|second|afternoon|| +60903|AAAAAAAAIONOAAAA|60903|16|55|3|PM|second|afternoon|| +60904|AAAAAAAAJONOAAAA|60904|16|55|4|PM|second|afternoon|| +60905|AAAAAAAAKONOAAAA|60905|16|55|5|PM|second|afternoon|| +60906|AAAAAAAALONOAAAA|60906|16|55|6|PM|second|afternoon|| +60907|AAAAAAAAMONOAAAA|60907|16|55|7|PM|second|afternoon|| +60908|AAAAAAAANONOAAAA|60908|16|55|8|PM|second|afternoon|| +60909|AAAAAAAAOONOAAAA|60909|16|55|9|PM|second|afternoon|| +60910|AAAAAAAAPONOAAAA|60910|16|55|10|PM|second|afternoon|| +60911|AAAAAAAAAPNOAAAA|60911|16|55|11|PM|second|afternoon|| +60912|AAAAAAAABPNOAAAA|60912|16|55|12|PM|second|afternoon|| +60913|AAAAAAAACPNOAAAA|60913|16|55|13|PM|second|afternoon|| +60914|AAAAAAAADPNOAAAA|60914|16|55|14|PM|second|afternoon|| +60915|AAAAAAAAEPNOAAAA|60915|16|55|15|PM|second|afternoon|| +60916|AAAAAAAAFPNOAAAA|60916|16|55|16|PM|second|afternoon|| +60917|AAAAAAAAGPNOAAAA|60917|16|55|17|PM|second|afternoon|| +60918|AAAAAAAAHPNOAAAA|60918|16|55|18|PM|second|afternoon|| +60919|AAAAAAAAIPNOAAAA|60919|16|55|19|PM|second|afternoon|| +60920|AAAAAAAAJPNOAAAA|60920|16|55|20|PM|second|afternoon|| +60921|AAAAAAAAKPNOAAAA|60921|16|55|21|PM|second|afternoon|| +60922|AAAAAAAALPNOAAAA|60922|16|55|22|PM|second|afternoon|| +60923|AAAAAAAAMPNOAAAA|60923|16|55|23|PM|second|afternoon|| +60924|AAAAAAAANPNOAAAA|60924|16|55|24|PM|second|afternoon|| +60925|AAAAAAAAOPNOAAAA|60925|16|55|25|PM|second|afternoon|| +60926|AAAAAAAAPPNOAAAA|60926|16|55|26|PM|second|afternoon|| +60927|AAAAAAAAAAOOAAAA|60927|16|55|27|PM|second|afternoon|| +60928|AAAAAAAABAOOAAAA|60928|16|55|28|PM|second|afternoon|| +60929|AAAAAAAACAOOAAAA|60929|16|55|29|PM|second|afternoon|| +60930|AAAAAAAADAOOAAAA|60930|16|55|30|PM|second|afternoon|| +60931|AAAAAAAAEAOOAAAA|60931|16|55|31|PM|second|afternoon|| +60932|AAAAAAAAFAOOAAAA|60932|16|55|32|PM|second|afternoon|| +60933|AAAAAAAAGAOOAAAA|60933|16|55|33|PM|second|afternoon|| +60934|AAAAAAAAHAOOAAAA|60934|16|55|34|PM|second|afternoon|| +60935|AAAAAAAAIAOOAAAA|60935|16|55|35|PM|second|afternoon|| +60936|AAAAAAAAJAOOAAAA|60936|16|55|36|PM|second|afternoon|| +60937|AAAAAAAAKAOOAAAA|60937|16|55|37|PM|second|afternoon|| +60938|AAAAAAAALAOOAAAA|60938|16|55|38|PM|second|afternoon|| +60939|AAAAAAAAMAOOAAAA|60939|16|55|39|PM|second|afternoon|| +60940|AAAAAAAANAOOAAAA|60940|16|55|40|PM|second|afternoon|| +60941|AAAAAAAAOAOOAAAA|60941|16|55|41|PM|second|afternoon|| +60942|AAAAAAAAPAOOAAAA|60942|16|55|42|PM|second|afternoon|| +60943|AAAAAAAAABOOAAAA|60943|16|55|43|PM|second|afternoon|| +60944|AAAAAAAABBOOAAAA|60944|16|55|44|PM|second|afternoon|| +60945|AAAAAAAACBOOAAAA|60945|16|55|45|PM|second|afternoon|| +60946|AAAAAAAADBOOAAAA|60946|16|55|46|PM|second|afternoon|| +60947|AAAAAAAAEBOOAAAA|60947|16|55|47|PM|second|afternoon|| +60948|AAAAAAAAFBOOAAAA|60948|16|55|48|PM|second|afternoon|| +60949|AAAAAAAAGBOOAAAA|60949|16|55|49|PM|second|afternoon|| +60950|AAAAAAAAHBOOAAAA|60950|16|55|50|PM|second|afternoon|| +60951|AAAAAAAAIBOOAAAA|60951|16|55|51|PM|second|afternoon|| +60952|AAAAAAAAJBOOAAAA|60952|16|55|52|PM|second|afternoon|| +60953|AAAAAAAAKBOOAAAA|60953|16|55|53|PM|second|afternoon|| +60954|AAAAAAAALBOOAAAA|60954|16|55|54|PM|second|afternoon|| +60955|AAAAAAAAMBOOAAAA|60955|16|55|55|PM|second|afternoon|| +60956|AAAAAAAANBOOAAAA|60956|16|55|56|PM|second|afternoon|| +60957|AAAAAAAAOBOOAAAA|60957|16|55|57|PM|second|afternoon|| +60958|AAAAAAAAPBOOAAAA|60958|16|55|58|PM|second|afternoon|| +60959|AAAAAAAAACOOAAAA|60959|16|55|59|PM|second|afternoon|| +60960|AAAAAAAABCOOAAAA|60960|16|56|0|PM|second|afternoon|| +60961|AAAAAAAACCOOAAAA|60961|16|56|1|PM|second|afternoon|| +60962|AAAAAAAADCOOAAAA|60962|16|56|2|PM|second|afternoon|| +60963|AAAAAAAAECOOAAAA|60963|16|56|3|PM|second|afternoon|| +60964|AAAAAAAAFCOOAAAA|60964|16|56|4|PM|second|afternoon|| +60965|AAAAAAAAGCOOAAAA|60965|16|56|5|PM|second|afternoon|| +60966|AAAAAAAAHCOOAAAA|60966|16|56|6|PM|second|afternoon|| +60967|AAAAAAAAICOOAAAA|60967|16|56|7|PM|second|afternoon|| +60968|AAAAAAAAJCOOAAAA|60968|16|56|8|PM|second|afternoon|| +60969|AAAAAAAAKCOOAAAA|60969|16|56|9|PM|second|afternoon|| +60970|AAAAAAAALCOOAAAA|60970|16|56|10|PM|second|afternoon|| +60971|AAAAAAAAMCOOAAAA|60971|16|56|11|PM|second|afternoon|| +60972|AAAAAAAANCOOAAAA|60972|16|56|12|PM|second|afternoon|| +60973|AAAAAAAAOCOOAAAA|60973|16|56|13|PM|second|afternoon|| +60974|AAAAAAAAPCOOAAAA|60974|16|56|14|PM|second|afternoon|| +60975|AAAAAAAAADOOAAAA|60975|16|56|15|PM|second|afternoon|| +60976|AAAAAAAABDOOAAAA|60976|16|56|16|PM|second|afternoon|| +60977|AAAAAAAACDOOAAAA|60977|16|56|17|PM|second|afternoon|| +60978|AAAAAAAADDOOAAAA|60978|16|56|18|PM|second|afternoon|| +60979|AAAAAAAAEDOOAAAA|60979|16|56|19|PM|second|afternoon|| +60980|AAAAAAAAFDOOAAAA|60980|16|56|20|PM|second|afternoon|| +60981|AAAAAAAAGDOOAAAA|60981|16|56|21|PM|second|afternoon|| +60982|AAAAAAAAHDOOAAAA|60982|16|56|22|PM|second|afternoon|| +60983|AAAAAAAAIDOOAAAA|60983|16|56|23|PM|second|afternoon|| +60984|AAAAAAAAJDOOAAAA|60984|16|56|24|PM|second|afternoon|| +60985|AAAAAAAAKDOOAAAA|60985|16|56|25|PM|second|afternoon|| +60986|AAAAAAAALDOOAAAA|60986|16|56|26|PM|second|afternoon|| +60987|AAAAAAAAMDOOAAAA|60987|16|56|27|PM|second|afternoon|| +60988|AAAAAAAANDOOAAAA|60988|16|56|28|PM|second|afternoon|| +60989|AAAAAAAAODOOAAAA|60989|16|56|29|PM|second|afternoon|| +60990|AAAAAAAAPDOOAAAA|60990|16|56|30|PM|second|afternoon|| +60991|AAAAAAAAAEOOAAAA|60991|16|56|31|PM|second|afternoon|| +60992|AAAAAAAABEOOAAAA|60992|16|56|32|PM|second|afternoon|| +60993|AAAAAAAACEOOAAAA|60993|16|56|33|PM|second|afternoon|| +60994|AAAAAAAADEOOAAAA|60994|16|56|34|PM|second|afternoon|| +60995|AAAAAAAAEEOOAAAA|60995|16|56|35|PM|second|afternoon|| +60996|AAAAAAAAFEOOAAAA|60996|16|56|36|PM|second|afternoon|| +60997|AAAAAAAAGEOOAAAA|60997|16|56|37|PM|second|afternoon|| +60998|AAAAAAAAHEOOAAAA|60998|16|56|38|PM|second|afternoon|| +60999|AAAAAAAAIEOOAAAA|60999|16|56|39|PM|second|afternoon|| +61000|AAAAAAAAJEOOAAAA|61000|16|56|40|PM|second|afternoon|| +61001|AAAAAAAAKEOOAAAA|61001|16|56|41|PM|second|afternoon|| +61002|AAAAAAAALEOOAAAA|61002|16|56|42|PM|second|afternoon|| +61003|AAAAAAAAMEOOAAAA|61003|16|56|43|PM|second|afternoon|| +61004|AAAAAAAANEOOAAAA|61004|16|56|44|PM|second|afternoon|| +61005|AAAAAAAAOEOOAAAA|61005|16|56|45|PM|second|afternoon|| +61006|AAAAAAAAPEOOAAAA|61006|16|56|46|PM|second|afternoon|| +61007|AAAAAAAAAFOOAAAA|61007|16|56|47|PM|second|afternoon|| +61008|AAAAAAAABFOOAAAA|61008|16|56|48|PM|second|afternoon|| +61009|AAAAAAAACFOOAAAA|61009|16|56|49|PM|second|afternoon|| +61010|AAAAAAAADFOOAAAA|61010|16|56|50|PM|second|afternoon|| +61011|AAAAAAAAEFOOAAAA|61011|16|56|51|PM|second|afternoon|| +61012|AAAAAAAAFFOOAAAA|61012|16|56|52|PM|second|afternoon|| +61013|AAAAAAAAGFOOAAAA|61013|16|56|53|PM|second|afternoon|| +61014|AAAAAAAAHFOOAAAA|61014|16|56|54|PM|second|afternoon|| +61015|AAAAAAAAIFOOAAAA|61015|16|56|55|PM|second|afternoon|| +61016|AAAAAAAAJFOOAAAA|61016|16|56|56|PM|second|afternoon|| +61017|AAAAAAAAKFOOAAAA|61017|16|56|57|PM|second|afternoon|| +61018|AAAAAAAALFOOAAAA|61018|16|56|58|PM|second|afternoon|| +61019|AAAAAAAAMFOOAAAA|61019|16|56|59|PM|second|afternoon|| +61020|AAAAAAAANFOOAAAA|61020|16|57|0|PM|second|afternoon|| +61021|AAAAAAAAOFOOAAAA|61021|16|57|1|PM|second|afternoon|| +61022|AAAAAAAAPFOOAAAA|61022|16|57|2|PM|second|afternoon|| +61023|AAAAAAAAAGOOAAAA|61023|16|57|3|PM|second|afternoon|| +61024|AAAAAAAABGOOAAAA|61024|16|57|4|PM|second|afternoon|| +61025|AAAAAAAACGOOAAAA|61025|16|57|5|PM|second|afternoon|| +61026|AAAAAAAADGOOAAAA|61026|16|57|6|PM|second|afternoon|| +61027|AAAAAAAAEGOOAAAA|61027|16|57|7|PM|second|afternoon|| +61028|AAAAAAAAFGOOAAAA|61028|16|57|8|PM|second|afternoon|| +61029|AAAAAAAAGGOOAAAA|61029|16|57|9|PM|second|afternoon|| +61030|AAAAAAAAHGOOAAAA|61030|16|57|10|PM|second|afternoon|| +61031|AAAAAAAAIGOOAAAA|61031|16|57|11|PM|second|afternoon|| +61032|AAAAAAAAJGOOAAAA|61032|16|57|12|PM|second|afternoon|| +61033|AAAAAAAAKGOOAAAA|61033|16|57|13|PM|second|afternoon|| +61034|AAAAAAAALGOOAAAA|61034|16|57|14|PM|second|afternoon|| +61035|AAAAAAAAMGOOAAAA|61035|16|57|15|PM|second|afternoon|| +61036|AAAAAAAANGOOAAAA|61036|16|57|16|PM|second|afternoon|| +61037|AAAAAAAAOGOOAAAA|61037|16|57|17|PM|second|afternoon|| +61038|AAAAAAAAPGOOAAAA|61038|16|57|18|PM|second|afternoon|| +61039|AAAAAAAAAHOOAAAA|61039|16|57|19|PM|second|afternoon|| +61040|AAAAAAAABHOOAAAA|61040|16|57|20|PM|second|afternoon|| +61041|AAAAAAAACHOOAAAA|61041|16|57|21|PM|second|afternoon|| +61042|AAAAAAAADHOOAAAA|61042|16|57|22|PM|second|afternoon|| +61043|AAAAAAAAEHOOAAAA|61043|16|57|23|PM|second|afternoon|| +61044|AAAAAAAAFHOOAAAA|61044|16|57|24|PM|second|afternoon|| +61045|AAAAAAAAGHOOAAAA|61045|16|57|25|PM|second|afternoon|| +61046|AAAAAAAAHHOOAAAA|61046|16|57|26|PM|second|afternoon|| +61047|AAAAAAAAIHOOAAAA|61047|16|57|27|PM|second|afternoon|| +61048|AAAAAAAAJHOOAAAA|61048|16|57|28|PM|second|afternoon|| +61049|AAAAAAAAKHOOAAAA|61049|16|57|29|PM|second|afternoon|| +61050|AAAAAAAALHOOAAAA|61050|16|57|30|PM|second|afternoon|| +61051|AAAAAAAAMHOOAAAA|61051|16|57|31|PM|second|afternoon|| +61052|AAAAAAAANHOOAAAA|61052|16|57|32|PM|second|afternoon|| +61053|AAAAAAAAOHOOAAAA|61053|16|57|33|PM|second|afternoon|| +61054|AAAAAAAAPHOOAAAA|61054|16|57|34|PM|second|afternoon|| +61055|AAAAAAAAAIOOAAAA|61055|16|57|35|PM|second|afternoon|| +61056|AAAAAAAABIOOAAAA|61056|16|57|36|PM|second|afternoon|| +61057|AAAAAAAACIOOAAAA|61057|16|57|37|PM|second|afternoon|| +61058|AAAAAAAADIOOAAAA|61058|16|57|38|PM|second|afternoon|| +61059|AAAAAAAAEIOOAAAA|61059|16|57|39|PM|second|afternoon|| +61060|AAAAAAAAFIOOAAAA|61060|16|57|40|PM|second|afternoon|| +61061|AAAAAAAAGIOOAAAA|61061|16|57|41|PM|second|afternoon|| +61062|AAAAAAAAHIOOAAAA|61062|16|57|42|PM|second|afternoon|| +61063|AAAAAAAAIIOOAAAA|61063|16|57|43|PM|second|afternoon|| +61064|AAAAAAAAJIOOAAAA|61064|16|57|44|PM|second|afternoon|| +61065|AAAAAAAAKIOOAAAA|61065|16|57|45|PM|second|afternoon|| +61066|AAAAAAAALIOOAAAA|61066|16|57|46|PM|second|afternoon|| +61067|AAAAAAAAMIOOAAAA|61067|16|57|47|PM|second|afternoon|| +61068|AAAAAAAANIOOAAAA|61068|16|57|48|PM|second|afternoon|| +61069|AAAAAAAAOIOOAAAA|61069|16|57|49|PM|second|afternoon|| +61070|AAAAAAAAPIOOAAAA|61070|16|57|50|PM|second|afternoon|| +61071|AAAAAAAAAJOOAAAA|61071|16|57|51|PM|second|afternoon|| +61072|AAAAAAAABJOOAAAA|61072|16|57|52|PM|second|afternoon|| +61073|AAAAAAAACJOOAAAA|61073|16|57|53|PM|second|afternoon|| +61074|AAAAAAAADJOOAAAA|61074|16|57|54|PM|second|afternoon|| +61075|AAAAAAAAEJOOAAAA|61075|16|57|55|PM|second|afternoon|| +61076|AAAAAAAAFJOOAAAA|61076|16|57|56|PM|second|afternoon|| +61077|AAAAAAAAGJOOAAAA|61077|16|57|57|PM|second|afternoon|| +61078|AAAAAAAAHJOOAAAA|61078|16|57|58|PM|second|afternoon|| +61079|AAAAAAAAIJOOAAAA|61079|16|57|59|PM|second|afternoon|| +61080|AAAAAAAAJJOOAAAA|61080|16|58|0|PM|second|afternoon|| +61081|AAAAAAAAKJOOAAAA|61081|16|58|1|PM|second|afternoon|| +61082|AAAAAAAALJOOAAAA|61082|16|58|2|PM|second|afternoon|| +61083|AAAAAAAAMJOOAAAA|61083|16|58|3|PM|second|afternoon|| +61084|AAAAAAAANJOOAAAA|61084|16|58|4|PM|second|afternoon|| +61085|AAAAAAAAOJOOAAAA|61085|16|58|5|PM|second|afternoon|| +61086|AAAAAAAAPJOOAAAA|61086|16|58|6|PM|second|afternoon|| +61087|AAAAAAAAAKOOAAAA|61087|16|58|7|PM|second|afternoon|| +61088|AAAAAAAABKOOAAAA|61088|16|58|8|PM|second|afternoon|| +61089|AAAAAAAACKOOAAAA|61089|16|58|9|PM|second|afternoon|| +61090|AAAAAAAADKOOAAAA|61090|16|58|10|PM|second|afternoon|| +61091|AAAAAAAAEKOOAAAA|61091|16|58|11|PM|second|afternoon|| +61092|AAAAAAAAFKOOAAAA|61092|16|58|12|PM|second|afternoon|| +61093|AAAAAAAAGKOOAAAA|61093|16|58|13|PM|second|afternoon|| +61094|AAAAAAAAHKOOAAAA|61094|16|58|14|PM|second|afternoon|| +61095|AAAAAAAAIKOOAAAA|61095|16|58|15|PM|second|afternoon|| +61096|AAAAAAAAJKOOAAAA|61096|16|58|16|PM|second|afternoon|| +61097|AAAAAAAAKKOOAAAA|61097|16|58|17|PM|second|afternoon|| +61098|AAAAAAAALKOOAAAA|61098|16|58|18|PM|second|afternoon|| +61099|AAAAAAAAMKOOAAAA|61099|16|58|19|PM|second|afternoon|| +61100|AAAAAAAANKOOAAAA|61100|16|58|20|PM|second|afternoon|| +61101|AAAAAAAAOKOOAAAA|61101|16|58|21|PM|second|afternoon|| +61102|AAAAAAAAPKOOAAAA|61102|16|58|22|PM|second|afternoon|| +61103|AAAAAAAAALOOAAAA|61103|16|58|23|PM|second|afternoon|| +61104|AAAAAAAABLOOAAAA|61104|16|58|24|PM|second|afternoon|| +61105|AAAAAAAACLOOAAAA|61105|16|58|25|PM|second|afternoon|| +61106|AAAAAAAADLOOAAAA|61106|16|58|26|PM|second|afternoon|| +61107|AAAAAAAAELOOAAAA|61107|16|58|27|PM|second|afternoon|| +61108|AAAAAAAAFLOOAAAA|61108|16|58|28|PM|second|afternoon|| +61109|AAAAAAAAGLOOAAAA|61109|16|58|29|PM|second|afternoon|| +61110|AAAAAAAAHLOOAAAA|61110|16|58|30|PM|second|afternoon|| +61111|AAAAAAAAILOOAAAA|61111|16|58|31|PM|second|afternoon|| +61112|AAAAAAAAJLOOAAAA|61112|16|58|32|PM|second|afternoon|| +61113|AAAAAAAAKLOOAAAA|61113|16|58|33|PM|second|afternoon|| +61114|AAAAAAAALLOOAAAA|61114|16|58|34|PM|second|afternoon|| +61115|AAAAAAAAMLOOAAAA|61115|16|58|35|PM|second|afternoon|| +61116|AAAAAAAANLOOAAAA|61116|16|58|36|PM|second|afternoon|| +61117|AAAAAAAAOLOOAAAA|61117|16|58|37|PM|second|afternoon|| +61118|AAAAAAAAPLOOAAAA|61118|16|58|38|PM|second|afternoon|| +61119|AAAAAAAAAMOOAAAA|61119|16|58|39|PM|second|afternoon|| +61120|AAAAAAAABMOOAAAA|61120|16|58|40|PM|second|afternoon|| +61121|AAAAAAAACMOOAAAA|61121|16|58|41|PM|second|afternoon|| +61122|AAAAAAAADMOOAAAA|61122|16|58|42|PM|second|afternoon|| +61123|AAAAAAAAEMOOAAAA|61123|16|58|43|PM|second|afternoon|| +61124|AAAAAAAAFMOOAAAA|61124|16|58|44|PM|second|afternoon|| +61125|AAAAAAAAGMOOAAAA|61125|16|58|45|PM|second|afternoon|| +61126|AAAAAAAAHMOOAAAA|61126|16|58|46|PM|second|afternoon|| +61127|AAAAAAAAIMOOAAAA|61127|16|58|47|PM|second|afternoon|| +61128|AAAAAAAAJMOOAAAA|61128|16|58|48|PM|second|afternoon|| +61129|AAAAAAAAKMOOAAAA|61129|16|58|49|PM|second|afternoon|| +61130|AAAAAAAALMOOAAAA|61130|16|58|50|PM|second|afternoon|| +61131|AAAAAAAAMMOOAAAA|61131|16|58|51|PM|second|afternoon|| +61132|AAAAAAAANMOOAAAA|61132|16|58|52|PM|second|afternoon|| +61133|AAAAAAAAOMOOAAAA|61133|16|58|53|PM|second|afternoon|| +61134|AAAAAAAAPMOOAAAA|61134|16|58|54|PM|second|afternoon|| +61135|AAAAAAAAANOOAAAA|61135|16|58|55|PM|second|afternoon|| +61136|AAAAAAAABNOOAAAA|61136|16|58|56|PM|second|afternoon|| +61137|AAAAAAAACNOOAAAA|61137|16|58|57|PM|second|afternoon|| +61138|AAAAAAAADNOOAAAA|61138|16|58|58|PM|second|afternoon|| +61139|AAAAAAAAENOOAAAA|61139|16|58|59|PM|second|afternoon|| +61140|AAAAAAAAFNOOAAAA|61140|16|59|0|PM|second|afternoon|| +61141|AAAAAAAAGNOOAAAA|61141|16|59|1|PM|second|afternoon|| +61142|AAAAAAAAHNOOAAAA|61142|16|59|2|PM|second|afternoon|| +61143|AAAAAAAAINOOAAAA|61143|16|59|3|PM|second|afternoon|| +61144|AAAAAAAAJNOOAAAA|61144|16|59|4|PM|second|afternoon|| +61145|AAAAAAAAKNOOAAAA|61145|16|59|5|PM|second|afternoon|| +61146|AAAAAAAALNOOAAAA|61146|16|59|6|PM|second|afternoon|| +61147|AAAAAAAAMNOOAAAA|61147|16|59|7|PM|second|afternoon|| +61148|AAAAAAAANNOOAAAA|61148|16|59|8|PM|second|afternoon|| +61149|AAAAAAAAONOOAAAA|61149|16|59|9|PM|second|afternoon|| +61150|AAAAAAAAPNOOAAAA|61150|16|59|10|PM|second|afternoon|| +61151|AAAAAAAAAOOOAAAA|61151|16|59|11|PM|second|afternoon|| +61152|AAAAAAAABOOOAAAA|61152|16|59|12|PM|second|afternoon|| +61153|AAAAAAAACOOOAAAA|61153|16|59|13|PM|second|afternoon|| +61154|AAAAAAAADOOOAAAA|61154|16|59|14|PM|second|afternoon|| +61155|AAAAAAAAEOOOAAAA|61155|16|59|15|PM|second|afternoon|| +61156|AAAAAAAAFOOOAAAA|61156|16|59|16|PM|second|afternoon|| +61157|AAAAAAAAGOOOAAAA|61157|16|59|17|PM|second|afternoon|| +61158|AAAAAAAAHOOOAAAA|61158|16|59|18|PM|second|afternoon|| +61159|AAAAAAAAIOOOAAAA|61159|16|59|19|PM|second|afternoon|| +61160|AAAAAAAAJOOOAAAA|61160|16|59|20|PM|second|afternoon|| +61161|AAAAAAAAKOOOAAAA|61161|16|59|21|PM|second|afternoon|| +61162|AAAAAAAALOOOAAAA|61162|16|59|22|PM|second|afternoon|| +61163|AAAAAAAAMOOOAAAA|61163|16|59|23|PM|second|afternoon|| +61164|AAAAAAAANOOOAAAA|61164|16|59|24|PM|second|afternoon|| +61165|AAAAAAAAOOOOAAAA|61165|16|59|25|PM|second|afternoon|| +61166|AAAAAAAAPOOOAAAA|61166|16|59|26|PM|second|afternoon|| +61167|AAAAAAAAAPOOAAAA|61167|16|59|27|PM|second|afternoon|| +61168|AAAAAAAABPOOAAAA|61168|16|59|28|PM|second|afternoon|| +61169|AAAAAAAACPOOAAAA|61169|16|59|29|PM|second|afternoon|| +61170|AAAAAAAADPOOAAAA|61170|16|59|30|PM|second|afternoon|| +61171|AAAAAAAAEPOOAAAA|61171|16|59|31|PM|second|afternoon|| +61172|AAAAAAAAFPOOAAAA|61172|16|59|32|PM|second|afternoon|| +61173|AAAAAAAAGPOOAAAA|61173|16|59|33|PM|second|afternoon|| +61174|AAAAAAAAHPOOAAAA|61174|16|59|34|PM|second|afternoon|| +61175|AAAAAAAAIPOOAAAA|61175|16|59|35|PM|second|afternoon|| +61176|AAAAAAAAJPOOAAAA|61176|16|59|36|PM|second|afternoon|| +61177|AAAAAAAAKPOOAAAA|61177|16|59|37|PM|second|afternoon|| +61178|AAAAAAAALPOOAAAA|61178|16|59|38|PM|second|afternoon|| +61179|AAAAAAAAMPOOAAAA|61179|16|59|39|PM|second|afternoon|| +61180|AAAAAAAANPOOAAAA|61180|16|59|40|PM|second|afternoon|| +61181|AAAAAAAAOPOOAAAA|61181|16|59|41|PM|second|afternoon|| +61182|AAAAAAAAPPOOAAAA|61182|16|59|42|PM|second|afternoon|| +61183|AAAAAAAAAAPOAAAA|61183|16|59|43|PM|second|afternoon|| +61184|AAAAAAAABAPOAAAA|61184|16|59|44|PM|second|afternoon|| +61185|AAAAAAAACAPOAAAA|61185|16|59|45|PM|second|afternoon|| +61186|AAAAAAAADAPOAAAA|61186|16|59|46|PM|second|afternoon|| +61187|AAAAAAAAEAPOAAAA|61187|16|59|47|PM|second|afternoon|| +61188|AAAAAAAAFAPOAAAA|61188|16|59|48|PM|second|afternoon|| +61189|AAAAAAAAGAPOAAAA|61189|16|59|49|PM|second|afternoon|| +61190|AAAAAAAAHAPOAAAA|61190|16|59|50|PM|second|afternoon|| +61191|AAAAAAAAIAPOAAAA|61191|16|59|51|PM|second|afternoon|| +61192|AAAAAAAAJAPOAAAA|61192|16|59|52|PM|second|afternoon|| +61193|AAAAAAAAKAPOAAAA|61193|16|59|53|PM|second|afternoon|| +61194|AAAAAAAALAPOAAAA|61194|16|59|54|PM|second|afternoon|| +61195|AAAAAAAAMAPOAAAA|61195|16|59|55|PM|second|afternoon|| +61196|AAAAAAAANAPOAAAA|61196|16|59|56|PM|second|afternoon|| +61197|AAAAAAAAOAPOAAAA|61197|16|59|57|PM|second|afternoon|| +61198|AAAAAAAAPAPOAAAA|61198|16|59|58|PM|second|afternoon|| +61199|AAAAAAAAABPOAAAA|61199|16|59|59|PM|second|afternoon|| +61200|AAAAAAAABBPOAAAA|61200|17|0|0|PM|second|evening|dinner| +61201|AAAAAAAACBPOAAAA|61201|17|0|1|PM|second|evening|dinner| +61202|AAAAAAAADBPOAAAA|61202|17|0|2|PM|second|evening|dinner| +61203|AAAAAAAAEBPOAAAA|61203|17|0|3|PM|second|evening|dinner| +61204|AAAAAAAAFBPOAAAA|61204|17|0|4|PM|second|evening|dinner| +61205|AAAAAAAAGBPOAAAA|61205|17|0|5|PM|second|evening|dinner| +61206|AAAAAAAAHBPOAAAA|61206|17|0|6|PM|second|evening|dinner| +61207|AAAAAAAAIBPOAAAA|61207|17|0|7|PM|second|evening|dinner| +61208|AAAAAAAAJBPOAAAA|61208|17|0|8|PM|second|evening|dinner| +61209|AAAAAAAAKBPOAAAA|61209|17|0|9|PM|second|evening|dinner| +61210|AAAAAAAALBPOAAAA|61210|17|0|10|PM|second|evening|dinner| +61211|AAAAAAAAMBPOAAAA|61211|17|0|11|PM|second|evening|dinner| +61212|AAAAAAAANBPOAAAA|61212|17|0|12|PM|second|evening|dinner| +61213|AAAAAAAAOBPOAAAA|61213|17|0|13|PM|second|evening|dinner| +61214|AAAAAAAAPBPOAAAA|61214|17|0|14|PM|second|evening|dinner| +61215|AAAAAAAAACPOAAAA|61215|17|0|15|PM|second|evening|dinner| +61216|AAAAAAAABCPOAAAA|61216|17|0|16|PM|second|evening|dinner| +61217|AAAAAAAACCPOAAAA|61217|17|0|17|PM|second|evening|dinner| +61218|AAAAAAAADCPOAAAA|61218|17|0|18|PM|second|evening|dinner| +61219|AAAAAAAAECPOAAAA|61219|17|0|19|PM|second|evening|dinner| +61220|AAAAAAAAFCPOAAAA|61220|17|0|20|PM|second|evening|dinner| +61221|AAAAAAAAGCPOAAAA|61221|17|0|21|PM|second|evening|dinner| +61222|AAAAAAAAHCPOAAAA|61222|17|0|22|PM|second|evening|dinner| +61223|AAAAAAAAICPOAAAA|61223|17|0|23|PM|second|evening|dinner| +61224|AAAAAAAAJCPOAAAA|61224|17|0|24|PM|second|evening|dinner| +61225|AAAAAAAAKCPOAAAA|61225|17|0|25|PM|second|evening|dinner| +61226|AAAAAAAALCPOAAAA|61226|17|0|26|PM|second|evening|dinner| +61227|AAAAAAAAMCPOAAAA|61227|17|0|27|PM|second|evening|dinner| +61228|AAAAAAAANCPOAAAA|61228|17|0|28|PM|second|evening|dinner| +61229|AAAAAAAAOCPOAAAA|61229|17|0|29|PM|second|evening|dinner| +61230|AAAAAAAAPCPOAAAA|61230|17|0|30|PM|second|evening|dinner| +61231|AAAAAAAAADPOAAAA|61231|17|0|31|PM|second|evening|dinner| +61232|AAAAAAAABDPOAAAA|61232|17|0|32|PM|second|evening|dinner| +61233|AAAAAAAACDPOAAAA|61233|17|0|33|PM|second|evening|dinner| +61234|AAAAAAAADDPOAAAA|61234|17|0|34|PM|second|evening|dinner| +61235|AAAAAAAAEDPOAAAA|61235|17|0|35|PM|second|evening|dinner| +61236|AAAAAAAAFDPOAAAA|61236|17|0|36|PM|second|evening|dinner| +61237|AAAAAAAAGDPOAAAA|61237|17|0|37|PM|second|evening|dinner| +61238|AAAAAAAAHDPOAAAA|61238|17|0|38|PM|second|evening|dinner| +61239|AAAAAAAAIDPOAAAA|61239|17|0|39|PM|second|evening|dinner| +61240|AAAAAAAAJDPOAAAA|61240|17|0|40|PM|second|evening|dinner| +61241|AAAAAAAAKDPOAAAA|61241|17|0|41|PM|second|evening|dinner| +61242|AAAAAAAALDPOAAAA|61242|17|0|42|PM|second|evening|dinner| +61243|AAAAAAAAMDPOAAAA|61243|17|0|43|PM|second|evening|dinner| +61244|AAAAAAAANDPOAAAA|61244|17|0|44|PM|second|evening|dinner| +61245|AAAAAAAAODPOAAAA|61245|17|0|45|PM|second|evening|dinner| +61246|AAAAAAAAPDPOAAAA|61246|17|0|46|PM|second|evening|dinner| +61247|AAAAAAAAAEPOAAAA|61247|17|0|47|PM|second|evening|dinner| +61248|AAAAAAAABEPOAAAA|61248|17|0|48|PM|second|evening|dinner| +61249|AAAAAAAACEPOAAAA|61249|17|0|49|PM|second|evening|dinner| +61250|AAAAAAAADEPOAAAA|61250|17|0|50|PM|second|evening|dinner| +61251|AAAAAAAAEEPOAAAA|61251|17|0|51|PM|second|evening|dinner| +61252|AAAAAAAAFEPOAAAA|61252|17|0|52|PM|second|evening|dinner| +61253|AAAAAAAAGEPOAAAA|61253|17|0|53|PM|second|evening|dinner| +61254|AAAAAAAAHEPOAAAA|61254|17|0|54|PM|second|evening|dinner| +61255|AAAAAAAAIEPOAAAA|61255|17|0|55|PM|second|evening|dinner| +61256|AAAAAAAAJEPOAAAA|61256|17|0|56|PM|second|evening|dinner| +61257|AAAAAAAAKEPOAAAA|61257|17|0|57|PM|second|evening|dinner| +61258|AAAAAAAALEPOAAAA|61258|17|0|58|PM|second|evening|dinner| +61259|AAAAAAAAMEPOAAAA|61259|17|0|59|PM|second|evening|dinner| +61260|AAAAAAAANEPOAAAA|61260|17|1|0|PM|second|evening|dinner| +61261|AAAAAAAAOEPOAAAA|61261|17|1|1|PM|second|evening|dinner| +61262|AAAAAAAAPEPOAAAA|61262|17|1|2|PM|second|evening|dinner| +61263|AAAAAAAAAFPOAAAA|61263|17|1|3|PM|second|evening|dinner| +61264|AAAAAAAABFPOAAAA|61264|17|1|4|PM|second|evening|dinner| +61265|AAAAAAAACFPOAAAA|61265|17|1|5|PM|second|evening|dinner| +61266|AAAAAAAADFPOAAAA|61266|17|1|6|PM|second|evening|dinner| +61267|AAAAAAAAEFPOAAAA|61267|17|1|7|PM|second|evening|dinner| +61268|AAAAAAAAFFPOAAAA|61268|17|1|8|PM|second|evening|dinner| +61269|AAAAAAAAGFPOAAAA|61269|17|1|9|PM|second|evening|dinner| +61270|AAAAAAAAHFPOAAAA|61270|17|1|10|PM|second|evening|dinner| +61271|AAAAAAAAIFPOAAAA|61271|17|1|11|PM|second|evening|dinner| +61272|AAAAAAAAJFPOAAAA|61272|17|1|12|PM|second|evening|dinner| +61273|AAAAAAAAKFPOAAAA|61273|17|1|13|PM|second|evening|dinner| +61274|AAAAAAAALFPOAAAA|61274|17|1|14|PM|second|evening|dinner| +61275|AAAAAAAAMFPOAAAA|61275|17|1|15|PM|second|evening|dinner| +61276|AAAAAAAANFPOAAAA|61276|17|1|16|PM|second|evening|dinner| +61277|AAAAAAAAOFPOAAAA|61277|17|1|17|PM|second|evening|dinner| +61278|AAAAAAAAPFPOAAAA|61278|17|1|18|PM|second|evening|dinner| +61279|AAAAAAAAAGPOAAAA|61279|17|1|19|PM|second|evening|dinner| +61280|AAAAAAAABGPOAAAA|61280|17|1|20|PM|second|evening|dinner| +61281|AAAAAAAACGPOAAAA|61281|17|1|21|PM|second|evening|dinner| +61282|AAAAAAAADGPOAAAA|61282|17|1|22|PM|second|evening|dinner| +61283|AAAAAAAAEGPOAAAA|61283|17|1|23|PM|second|evening|dinner| +61284|AAAAAAAAFGPOAAAA|61284|17|1|24|PM|second|evening|dinner| +61285|AAAAAAAAGGPOAAAA|61285|17|1|25|PM|second|evening|dinner| +61286|AAAAAAAAHGPOAAAA|61286|17|1|26|PM|second|evening|dinner| +61287|AAAAAAAAIGPOAAAA|61287|17|1|27|PM|second|evening|dinner| +61288|AAAAAAAAJGPOAAAA|61288|17|1|28|PM|second|evening|dinner| +61289|AAAAAAAAKGPOAAAA|61289|17|1|29|PM|second|evening|dinner| +61290|AAAAAAAALGPOAAAA|61290|17|1|30|PM|second|evening|dinner| +61291|AAAAAAAAMGPOAAAA|61291|17|1|31|PM|second|evening|dinner| +61292|AAAAAAAANGPOAAAA|61292|17|1|32|PM|second|evening|dinner| +61293|AAAAAAAAOGPOAAAA|61293|17|1|33|PM|second|evening|dinner| +61294|AAAAAAAAPGPOAAAA|61294|17|1|34|PM|second|evening|dinner| +61295|AAAAAAAAAHPOAAAA|61295|17|1|35|PM|second|evening|dinner| +61296|AAAAAAAABHPOAAAA|61296|17|1|36|PM|second|evening|dinner| +61297|AAAAAAAACHPOAAAA|61297|17|1|37|PM|second|evening|dinner| +61298|AAAAAAAADHPOAAAA|61298|17|1|38|PM|second|evening|dinner| +61299|AAAAAAAAEHPOAAAA|61299|17|1|39|PM|second|evening|dinner| +61300|AAAAAAAAFHPOAAAA|61300|17|1|40|PM|second|evening|dinner| +61301|AAAAAAAAGHPOAAAA|61301|17|1|41|PM|second|evening|dinner| +61302|AAAAAAAAHHPOAAAA|61302|17|1|42|PM|second|evening|dinner| +61303|AAAAAAAAIHPOAAAA|61303|17|1|43|PM|second|evening|dinner| +61304|AAAAAAAAJHPOAAAA|61304|17|1|44|PM|second|evening|dinner| +61305|AAAAAAAAKHPOAAAA|61305|17|1|45|PM|second|evening|dinner| +61306|AAAAAAAALHPOAAAA|61306|17|1|46|PM|second|evening|dinner| +61307|AAAAAAAAMHPOAAAA|61307|17|1|47|PM|second|evening|dinner| +61308|AAAAAAAANHPOAAAA|61308|17|1|48|PM|second|evening|dinner| +61309|AAAAAAAAOHPOAAAA|61309|17|1|49|PM|second|evening|dinner| +61310|AAAAAAAAPHPOAAAA|61310|17|1|50|PM|second|evening|dinner| +61311|AAAAAAAAAIPOAAAA|61311|17|1|51|PM|second|evening|dinner| +61312|AAAAAAAABIPOAAAA|61312|17|1|52|PM|second|evening|dinner| +61313|AAAAAAAACIPOAAAA|61313|17|1|53|PM|second|evening|dinner| +61314|AAAAAAAADIPOAAAA|61314|17|1|54|PM|second|evening|dinner| +61315|AAAAAAAAEIPOAAAA|61315|17|1|55|PM|second|evening|dinner| +61316|AAAAAAAAFIPOAAAA|61316|17|1|56|PM|second|evening|dinner| +61317|AAAAAAAAGIPOAAAA|61317|17|1|57|PM|second|evening|dinner| +61318|AAAAAAAAHIPOAAAA|61318|17|1|58|PM|second|evening|dinner| +61319|AAAAAAAAIIPOAAAA|61319|17|1|59|PM|second|evening|dinner| +61320|AAAAAAAAJIPOAAAA|61320|17|2|0|PM|second|evening|dinner| +61321|AAAAAAAAKIPOAAAA|61321|17|2|1|PM|second|evening|dinner| +61322|AAAAAAAALIPOAAAA|61322|17|2|2|PM|second|evening|dinner| +61323|AAAAAAAAMIPOAAAA|61323|17|2|3|PM|second|evening|dinner| +61324|AAAAAAAANIPOAAAA|61324|17|2|4|PM|second|evening|dinner| +61325|AAAAAAAAOIPOAAAA|61325|17|2|5|PM|second|evening|dinner| +61326|AAAAAAAAPIPOAAAA|61326|17|2|6|PM|second|evening|dinner| +61327|AAAAAAAAAJPOAAAA|61327|17|2|7|PM|second|evening|dinner| +61328|AAAAAAAABJPOAAAA|61328|17|2|8|PM|second|evening|dinner| +61329|AAAAAAAACJPOAAAA|61329|17|2|9|PM|second|evening|dinner| +61330|AAAAAAAADJPOAAAA|61330|17|2|10|PM|second|evening|dinner| +61331|AAAAAAAAEJPOAAAA|61331|17|2|11|PM|second|evening|dinner| +61332|AAAAAAAAFJPOAAAA|61332|17|2|12|PM|second|evening|dinner| +61333|AAAAAAAAGJPOAAAA|61333|17|2|13|PM|second|evening|dinner| +61334|AAAAAAAAHJPOAAAA|61334|17|2|14|PM|second|evening|dinner| +61335|AAAAAAAAIJPOAAAA|61335|17|2|15|PM|second|evening|dinner| +61336|AAAAAAAAJJPOAAAA|61336|17|2|16|PM|second|evening|dinner| +61337|AAAAAAAAKJPOAAAA|61337|17|2|17|PM|second|evening|dinner| +61338|AAAAAAAALJPOAAAA|61338|17|2|18|PM|second|evening|dinner| +61339|AAAAAAAAMJPOAAAA|61339|17|2|19|PM|second|evening|dinner| +61340|AAAAAAAANJPOAAAA|61340|17|2|20|PM|second|evening|dinner| +61341|AAAAAAAAOJPOAAAA|61341|17|2|21|PM|second|evening|dinner| +61342|AAAAAAAAPJPOAAAA|61342|17|2|22|PM|second|evening|dinner| +61343|AAAAAAAAAKPOAAAA|61343|17|2|23|PM|second|evening|dinner| +61344|AAAAAAAABKPOAAAA|61344|17|2|24|PM|second|evening|dinner| +61345|AAAAAAAACKPOAAAA|61345|17|2|25|PM|second|evening|dinner| +61346|AAAAAAAADKPOAAAA|61346|17|2|26|PM|second|evening|dinner| +61347|AAAAAAAAEKPOAAAA|61347|17|2|27|PM|second|evening|dinner| +61348|AAAAAAAAFKPOAAAA|61348|17|2|28|PM|second|evening|dinner| +61349|AAAAAAAAGKPOAAAA|61349|17|2|29|PM|second|evening|dinner| +61350|AAAAAAAAHKPOAAAA|61350|17|2|30|PM|second|evening|dinner| +61351|AAAAAAAAIKPOAAAA|61351|17|2|31|PM|second|evening|dinner| +61352|AAAAAAAAJKPOAAAA|61352|17|2|32|PM|second|evening|dinner| +61353|AAAAAAAAKKPOAAAA|61353|17|2|33|PM|second|evening|dinner| +61354|AAAAAAAALKPOAAAA|61354|17|2|34|PM|second|evening|dinner| +61355|AAAAAAAAMKPOAAAA|61355|17|2|35|PM|second|evening|dinner| +61356|AAAAAAAANKPOAAAA|61356|17|2|36|PM|second|evening|dinner| +61357|AAAAAAAAOKPOAAAA|61357|17|2|37|PM|second|evening|dinner| +61358|AAAAAAAAPKPOAAAA|61358|17|2|38|PM|second|evening|dinner| +61359|AAAAAAAAALPOAAAA|61359|17|2|39|PM|second|evening|dinner| +61360|AAAAAAAABLPOAAAA|61360|17|2|40|PM|second|evening|dinner| +61361|AAAAAAAACLPOAAAA|61361|17|2|41|PM|second|evening|dinner| +61362|AAAAAAAADLPOAAAA|61362|17|2|42|PM|second|evening|dinner| +61363|AAAAAAAAELPOAAAA|61363|17|2|43|PM|second|evening|dinner| +61364|AAAAAAAAFLPOAAAA|61364|17|2|44|PM|second|evening|dinner| +61365|AAAAAAAAGLPOAAAA|61365|17|2|45|PM|second|evening|dinner| +61366|AAAAAAAAHLPOAAAA|61366|17|2|46|PM|second|evening|dinner| +61367|AAAAAAAAILPOAAAA|61367|17|2|47|PM|second|evening|dinner| +61368|AAAAAAAAJLPOAAAA|61368|17|2|48|PM|second|evening|dinner| +61369|AAAAAAAAKLPOAAAA|61369|17|2|49|PM|second|evening|dinner| +61370|AAAAAAAALLPOAAAA|61370|17|2|50|PM|second|evening|dinner| +61371|AAAAAAAAMLPOAAAA|61371|17|2|51|PM|second|evening|dinner| +61372|AAAAAAAANLPOAAAA|61372|17|2|52|PM|second|evening|dinner| +61373|AAAAAAAAOLPOAAAA|61373|17|2|53|PM|second|evening|dinner| +61374|AAAAAAAAPLPOAAAA|61374|17|2|54|PM|second|evening|dinner| +61375|AAAAAAAAAMPOAAAA|61375|17|2|55|PM|second|evening|dinner| +61376|AAAAAAAABMPOAAAA|61376|17|2|56|PM|second|evening|dinner| +61377|AAAAAAAACMPOAAAA|61377|17|2|57|PM|second|evening|dinner| +61378|AAAAAAAADMPOAAAA|61378|17|2|58|PM|second|evening|dinner| +61379|AAAAAAAAEMPOAAAA|61379|17|2|59|PM|second|evening|dinner| +61380|AAAAAAAAFMPOAAAA|61380|17|3|0|PM|second|evening|dinner| +61381|AAAAAAAAGMPOAAAA|61381|17|3|1|PM|second|evening|dinner| +61382|AAAAAAAAHMPOAAAA|61382|17|3|2|PM|second|evening|dinner| +61383|AAAAAAAAIMPOAAAA|61383|17|3|3|PM|second|evening|dinner| +61384|AAAAAAAAJMPOAAAA|61384|17|3|4|PM|second|evening|dinner| +61385|AAAAAAAAKMPOAAAA|61385|17|3|5|PM|second|evening|dinner| +61386|AAAAAAAALMPOAAAA|61386|17|3|6|PM|second|evening|dinner| +61387|AAAAAAAAMMPOAAAA|61387|17|3|7|PM|second|evening|dinner| +61388|AAAAAAAANMPOAAAA|61388|17|3|8|PM|second|evening|dinner| +61389|AAAAAAAAOMPOAAAA|61389|17|3|9|PM|second|evening|dinner| +61390|AAAAAAAAPMPOAAAA|61390|17|3|10|PM|second|evening|dinner| +61391|AAAAAAAAANPOAAAA|61391|17|3|11|PM|second|evening|dinner| +61392|AAAAAAAABNPOAAAA|61392|17|3|12|PM|second|evening|dinner| +61393|AAAAAAAACNPOAAAA|61393|17|3|13|PM|second|evening|dinner| +61394|AAAAAAAADNPOAAAA|61394|17|3|14|PM|second|evening|dinner| +61395|AAAAAAAAENPOAAAA|61395|17|3|15|PM|second|evening|dinner| +61396|AAAAAAAAFNPOAAAA|61396|17|3|16|PM|second|evening|dinner| +61397|AAAAAAAAGNPOAAAA|61397|17|3|17|PM|second|evening|dinner| +61398|AAAAAAAAHNPOAAAA|61398|17|3|18|PM|second|evening|dinner| +61399|AAAAAAAAINPOAAAA|61399|17|3|19|PM|second|evening|dinner| +61400|AAAAAAAAJNPOAAAA|61400|17|3|20|PM|second|evening|dinner| +61401|AAAAAAAAKNPOAAAA|61401|17|3|21|PM|second|evening|dinner| +61402|AAAAAAAALNPOAAAA|61402|17|3|22|PM|second|evening|dinner| +61403|AAAAAAAAMNPOAAAA|61403|17|3|23|PM|second|evening|dinner| +61404|AAAAAAAANNPOAAAA|61404|17|3|24|PM|second|evening|dinner| +61405|AAAAAAAAONPOAAAA|61405|17|3|25|PM|second|evening|dinner| +61406|AAAAAAAAPNPOAAAA|61406|17|3|26|PM|second|evening|dinner| +61407|AAAAAAAAAOPOAAAA|61407|17|3|27|PM|second|evening|dinner| +61408|AAAAAAAABOPOAAAA|61408|17|3|28|PM|second|evening|dinner| +61409|AAAAAAAACOPOAAAA|61409|17|3|29|PM|second|evening|dinner| +61410|AAAAAAAADOPOAAAA|61410|17|3|30|PM|second|evening|dinner| +61411|AAAAAAAAEOPOAAAA|61411|17|3|31|PM|second|evening|dinner| +61412|AAAAAAAAFOPOAAAA|61412|17|3|32|PM|second|evening|dinner| +61413|AAAAAAAAGOPOAAAA|61413|17|3|33|PM|second|evening|dinner| +61414|AAAAAAAAHOPOAAAA|61414|17|3|34|PM|second|evening|dinner| +61415|AAAAAAAAIOPOAAAA|61415|17|3|35|PM|second|evening|dinner| +61416|AAAAAAAAJOPOAAAA|61416|17|3|36|PM|second|evening|dinner| +61417|AAAAAAAAKOPOAAAA|61417|17|3|37|PM|second|evening|dinner| +61418|AAAAAAAALOPOAAAA|61418|17|3|38|PM|second|evening|dinner| +61419|AAAAAAAAMOPOAAAA|61419|17|3|39|PM|second|evening|dinner| +61420|AAAAAAAANOPOAAAA|61420|17|3|40|PM|second|evening|dinner| +61421|AAAAAAAAOOPOAAAA|61421|17|3|41|PM|second|evening|dinner| +61422|AAAAAAAAPOPOAAAA|61422|17|3|42|PM|second|evening|dinner| +61423|AAAAAAAAAPPOAAAA|61423|17|3|43|PM|second|evening|dinner| +61424|AAAAAAAABPPOAAAA|61424|17|3|44|PM|second|evening|dinner| +61425|AAAAAAAACPPOAAAA|61425|17|3|45|PM|second|evening|dinner| +61426|AAAAAAAADPPOAAAA|61426|17|3|46|PM|second|evening|dinner| +61427|AAAAAAAAEPPOAAAA|61427|17|3|47|PM|second|evening|dinner| +61428|AAAAAAAAFPPOAAAA|61428|17|3|48|PM|second|evening|dinner| +61429|AAAAAAAAGPPOAAAA|61429|17|3|49|PM|second|evening|dinner| +61430|AAAAAAAAHPPOAAAA|61430|17|3|50|PM|second|evening|dinner| +61431|AAAAAAAAIPPOAAAA|61431|17|3|51|PM|second|evening|dinner| +61432|AAAAAAAAJPPOAAAA|61432|17|3|52|PM|second|evening|dinner| +61433|AAAAAAAAKPPOAAAA|61433|17|3|53|PM|second|evening|dinner| +61434|AAAAAAAALPPOAAAA|61434|17|3|54|PM|second|evening|dinner| +61435|AAAAAAAAMPPOAAAA|61435|17|3|55|PM|second|evening|dinner| +61436|AAAAAAAANPPOAAAA|61436|17|3|56|PM|second|evening|dinner| +61437|AAAAAAAAOPPOAAAA|61437|17|3|57|PM|second|evening|dinner| +61438|AAAAAAAAPPPOAAAA|61438|17|3|58|PM|second|evening|dinner| +61439|AAAAAAAAAAAPAAAA|61439|17|3|59|PM|second|evening|dinner| +61440|AAAAAAAABAAPAAAA|61440|17|4|0|PM|second|evening|dinner| +61441|AAAAAAAACAAPAAAA|61441|17|4|1|PM|second|evening|dinner| +61442|AAAAAAAADAAPAAAA|61442|17|4|2|PM|second|evening|dinner| +61443|AAAAAAAAEAAPAAAA|61443|17|4|3|PM|second|evening|dinner| +61444|AAAAAAAAFAAPAAAA|61444|17|4|4|PM|second|evening|dinner| +61445|AAAAAAAAGAAPAAAA|61445|17|4|5|PM|second|evening|dinner| +61446|AAAAAAAAHAAPAAAA|61446|17|4|6|PM|second|evening|dinner| +61447|AAAAAAAAIAAPAAAA|61447|17|4|7|PM|second|evening|dinner| +61448|AAAAAAAAJAAPAAAA|61448|17|4|8|PM|second|evening|dinner| +61449|AAAAAAAAKAAPAAAA|61449|17|4|9|PM|second|evening|dinner| +61450|AAAAAAAALAAPAAAA|61450|17|4|10|PM|second|evening|dinner| +61451|AAAAAAAAMAAPAAAA|61451|17|4|11|PM|second|evening|dinner| +61452|AAAAAAAANAAPAAAA|61452|17|4|12|PM|second|evening|dinner| +61453|AAAAAAAAOAAPAAAA|61453|17|4|13|PM|second|evening|dinner| +61454|AAAAAAAAPAAPAAAA|61454|17|4|14|PM|second|evening|dinner| +61455|AAAAAAAAABAPAAAA|61455|17|4|15|PM|second|evening|dinner| +61456|AAAAAAAABBAPAAAA|61456|17|4|16|PM|second|evening|dinner| +61457|AAAAAAAACBAPAAAA|61457|17|4|17|PM|second|evening|dinner| +61458|AAAAAAAADBAPAAAA|61458|17|4|18|PM|second|evening|dinner| +61459|AAAAAAAAEBAPAAAA|61459|17|4|19|PM|second|evening|dinner| +61460|AAAAAAAAFBAPAAAA|61460|17|4|20|PM|second|evening|dinner| +61461|AAAAAAAAGBAPAAAA|61461|17|4|21|PM|second|evening|dinner| +61462|AAAAAAAAHBAPAAAA|61462|17|4|22|PM|second|evening|dinner| +61463|AAAAAAAAIBAPAAAA|61463|17|4|23|PM|second|evening|dinner| +61464|AAAAAAAAJBAPAAAA|61464|17|4|24|PM|second|evening|dinner| +61465|AAAAAAAAKBAPAAAA|61465|17|4|25|PM|second|evening|dinner| +61466|AAAAAAAALBAPAAAA|61466|17|4|26|PM|second|evening|dinner| +61467|AAAAAAAAMBAPAAAA|61467|17|4|27|PM|second|evening|dinner| +61468|AAAAAAAANBAPAAAA|61468|17|4|28|PM|second|evening|dinner| +61469|AAAAAAAAOBAPAAAA|61469|17|4|29|PM|second|evening|dinner| +61470|AAAAAAAAPBAPAAAA|61470|17|4|30|PM|second|evening|dinner| +61471|AAAAAAAAACAPAAAA|61471|17|4|31|PM|second|evening|dinner| +61472|AAAAAAAABCAPAAAA|61472|17|4|32|PM|second|evening|dinner| +61473|AAAAAAAACCAPAAAA|61473|17|4|33|PM|second|evening|dinner| +61474|AAAAAAAADCAPAAAA|61474|17|4|34|PM|second|evening|dinner| +61475|AAAAAAAAECAPAAAA|61475|17|4|35|PM|second|evening|dinner| +61476|AAAAAAAAFCAPAAAA|61476|17|4|36|PM|second|evening|dinner| +61477|AAAAAAAAGCAPAAAA|61477|17|4|37|PM|second|evening|dinner| +61478|AAAAAAAAHCAPAAAA|61478|17|4|38|PM|second|evening|dinner| +61479|AAAAAAAAICAPAAAA|61479|17|4|39|PM|second|evening|dinner| +61480|AAAAAAAAJCAPAAAA|61480|17|4|40|PM|second|evening|dinner| +61481|AAAAAAAAKCAPAAAA|61481|17|4|41|PM|second|evening|dinner| +61482|AAAAAAAALCAPAAAA|61482|17|4|42|PM|second|evening|dinner| +61483|AAAAAAAAMCAPAAAA|61483|17|4|43|PM|second|evening|dinner| +61484|AAAAAAAANCAPAAAA|61484|17|4|44|PM|second|evening|dinner| +61485|AAAAAAAAOCAPAAAA|61485|17|4|45|PM|second|evening|dinner| +61486|AAAAAAAAPCAPAAAA|61486|17|4|46|PM|second|evening|dinner| +61487|AAAAAAAAADAPAAAA|61487|17|4|47|PM|second|evening|dinner| +61488|AAAAAAAABDAPAAAA|61488|17|4|48|PM|second|evening|dinner| +61489|AAAAAAAACDAPAAAA|61489|17|4|49|PM|second|evening|dinner| +61490|AAAAAAAADDAPAAAA|61490|17|4|50|PM|second|evening|dinner| +61491|AAAAAAAAEDAPAAAA|61491|17|4|51|PM|second|evening|dinner| +61492|AAAAAAAAFDAPAAAA|61492|17|4|52|PM|second|evening|dinner| +61493|AAAAAAAAGDAPAAAA|61493|17|4|53|PM|second|evening|dinner| +61494|AAAAAAAAHDAPAAAA|61494|17|4|54|PM|second|evening|dinner| +61495|AAAAAAAAIDAPAAAA|61495|17|4|55|PM|second|evening|dinner| +61496|AAAAAAAAJDAPAAAA|61496|17|4|56|PM|second|evening|dinner| +61497|AAAAAAAAKDAPAAAA|61497|17|4|57|PM|second|evening|dinner| +61498|AAAAAAAALDAPAAAA|61498|17|4|58|PM|second|evening|dinner| +61499|AAAAAAAAMDAPAAAA|61499|17|4|59|PM|second|evening|dinner| +61500|AAAAAAAANDAPAAAA|61500|17|5|0|PM|second|evening|dinner| +61501|AAAAAAAAODAPAAAA|61501|17|5|1|PM|second|evening|dinner| +61502|AAAAAAAAPDAPAAAA|61502|17|5|2|PM|second|evening|dinner| +61503|AAAAAAAAAEAPAAAA|61503|17|5|3|PM|second|evening|dinner| +61504|AAAAAAAABEAPAAAA|61504|17|5|4|PM|second|evening|dinner| +61505|AAAAAAAACEAPAAAA|61505|17|5|5|PM|second|evening|dinner| +61506|AAAAAAAADEAPAAAA|61506|17|5|6|PM|second|evening|dinner| +61507|AAAAAAAAEEAPAAAA|61507|17|5|7|PM|second|evening|dinner| +61508|AAAAAAAAFEAPAAAA|61508|17|5|8|PM|second|evening|dinner| +61509|AAAAAAAAGEAPAAAA|61509|17|5|9|PM|second|evening|dinner| +61510|AAAAAAAAHEAPAAAA|61510|17|5|10|PM|second|evening|dinner| +61511|AAAAAAAAIEAPAAAA|61511|17|5|11|PM|second|evening|dinner| +61512|AAAAAAAAJEAPAAAA|61512|17|5|12|PM|second|evening|dinner| +61513|AAAAAAAAKEAPAAAA|61513|17|5|13|PM|second|evening|dinner| +61514|AAAAAAAALEAPAAAA|61514|17|5|14|PM|second|evening|dinner| +61515|AAAAAAAAMEAPAAAA|61515|17|5|15|PM|second|evening|dinner| +61516|AAAAAAAANEAPAAAA|61516|17|5|16|PM|second|evening|dinner| +61517|AAAAAAAAOEAPAAAA|61517|17|5|17|PM|second|evening|dinner| +61518|AAAAAAAAPEAPAAAA|61518|17|5|18|PM|second|evening|dinner| +61519|AAAAAAAAAFAPAAAA|61519|17|5|19|PM|second|evening|dinner| +61520|AAAAAAAABFAPAAAA|61520|17|5|20|PM|second|evening|dinner| +61521|AAAAAAAACFAPAAAA|61521|17|5|21|PM|second|evening|dinner| +61522|AAAAAAAADFAPAAAA|61522|17|5|22|PM|second|evening|dinner| +61523|AAAAAAAAEFAPAAAA|61523|17|5|23|PM|second|evening|dinner| +61524|AAAAAAAAFFAPAAAA|61524|17|5|24|PM|second|evening|dinner| +61525|AAAAAAAAGFAPAAAA|61525|17|5|25|PM|second|evening|dinner| +61526|AAAAAAAAHFAPAAAA|61526|17|5|26|PM|second|evening|dinner| +61527|AAAAAAAAIFAPAAAA|61527|17|5|27|PM|second|evening|dinner| +61528|AAAAAAAAJFAPAAAA|61528|17|5|28|PM|second|evening|dinner| +61529|AAAAAAAAKFAPAAAA|61529|17|5|29|PM|second|evening|dinner| +61530|AAAAAAAALFAPAAAA|61530|17|5|30|PM|second|evening|dinner| +61531|AAAAAAAAMFAPAAAA|61531|17|5|31|PM|second|evening|dinner| +61532|AAAAAAAANFAPAAAA|61532|17|5|32|PM|second|evening|dinner| +61533|AAAAAAAAOFAPAAAA|61533|17|5|33|PM|second|evening|dinner| +61534|AAAAAAAAPFAPAAAA|61534|17|5|34|PM|second|evening|dinner| +61535|AAAAAAAAAGAPAAAA|61535|17|5|35|PM|second|evening|dinner| +61536|AAAAAAAABGAPAAAA|61536|17|5|36|PM|second|evening|dinner| +61537|AAAAAAAACGAPAAAA|61537|17|5|37|PM|second|evening|dinner| +61538|AAAAAAAADGAPAAAA|61538|17|5|38|PM|second|evening|dinner| +61539|AAAAAAAAEGAPAAAA|61539|17|5|39|PM|second|evening|dinner| +61540|AAAAAAAAFGAPAAAA|61540|17|5|40|PM|second|evening|dinner| +61541|AAAAAAAAGGAPAAAA|61541|17|5|41|PM|second|evening|dinner| +61542|AAAAAAAAHGAPAAAA|61542|17|5|42|PM|second|evening|dinner| +61543|AAAAAAAAIGAPAAAA|61543|17|5|43|PM|second|evening|dinner| +61544|AAAAAAAAJGAPAAAA|61544|17|5|44|PM|second|evening|dinner| +61545|AAAAAAAAKGAPAAAA|61545|17|5|45|PM|second|evening|dinner| +61546|AAAAAAAALGAPAAAA|61546|17|5|46|PM|second|evening|dinner| +61547|AAAAAAAAMGAPAAAA|61547|17|5|47|PM|second|evening|dinner| +61548|AAAAAAAANGAPAAAA|61548|17|5|48|PM|second|evening|dinner| +61549|AAAAAAAAOGAPAAAA|61549|17|5|49|PM|second|evening|dinner| +61550|AAAAAAAAPGAPAAAA|61550|17|5|50|PM|second|evening|dinner| +61551|AAAAAAAAAHAPAAAA|61551|17|5|51|PM|second|evening|dinner| +61552|AAAAAAAABHAPAAAA|61552|17|5|52|PM|second|evening|dinner| +61553|AAAAAAAACHAPAAAA|61553|17|5|53|PM|second|evening|dinner| +61554|AAAAAAAADHAPAAAA|61554|17|5|54|PM|second|evening|dinner| +61555|AAAAAAAAEHAPAAAA|61555|17|5|55|PM|second|evening|dinner| +61556|AAAAAAAAFHAPAAAA|61556|17|5|56|PM|second|evening|dinner| +61557|AAAAAAAAGHAPAAAA|61557|17|5|57|PM|second|evening|dinner| +61558|AAAAAAAAHHAPAAAA|61558|17|5|58|PM|second|evening|dinner| +61559|AAAAAAAAIHAPAAAA|61559|17|5|59|PM|second|evening|dinner| +61560|AAAAAAAAJHAPAAAA|61560|17|6|0|PM|second|evening|dinner| +61561|AAAAAAAAKHAPAAAA|61561|17|6|1|PM|second|evening|dinner| +61562|AAAAAAAALHAPAAAA|61562|17|6|2|PM|second|evening|dinner| +61563|AAAAAAAAMHAPAAAA|61563|17|6|3|PM|second|evening|dinner| +61564|AAAAAAAANHAPAAAA|61564|17|6|4|PM|second|evening|dinner| +61565|AAAAAAAAOHAPAAAA|61565|17|6|5|PM|second|evening|dinner| +61566|AAAAAAAAPHAPAAAA|61566|17|6|6|PM|second|evening|dinner| +61567|AAAAAAAAAIAPAAAA|61567|17|6|7|PM|second|evening|dinner| +61568|AAAAAAAABIAPAAAA|61568|17|6|8|PM|second|evening|dinner| +61569|AAAAAAAACIAPAAAA|61569|17|6|9|PM|second|evening|dinner| +61570|AAAAAAAADIAPAAAA|61570|17|6|10|PM|second|evening|dinner| +61571|AAAAAAAAEIAPAAAA|61571|17|6|11|PM|second|evening|dinner| +61572|AAAAAAAAFIAPAAAA|61572|17|6|12|PM|second|evening|dinner| +61573|AAAAAAAAGIAPAAAA|61573|17|6|13|PM|second|evening|dinner| +61574|AAAAAAAAHIAPAAAA|61574|17|6|14|PM|second|evening|dinner| +61575|AAAAAAAAIIAPAAAA|61575|17|6|15|PM|second|evening|dinner| +61576|AAAAAAAAJIAPAAAA|61576|17|6|16|PM|second|evening|dinner| +61577|AAAAAAAAKIAPAAAA|61577|17|6|17|PM|second|evening|dinner| +61578|AAAAAAAALIAPAAAA|61578|17|6|18|PM|second|evening|dinner| +61579|AAAAAAAAMIAPAAAA|61579|17|6|19|PM|second|evening|dinner| +61580|AAAAAAAANIAPAAAA|61580|17|6|20|PM|second|evening|dinner| +61581|AAAAAAAAOIAPAAAA|61581|17|6|21|PM|second|evening|dinner| +61582|AAAAAAAAPIAPAAAA|61582|17|6|22|PM|second|evening|dinner| +61583|AAAAAAAAAJAPAAAA|61583|17|6|23|PM|second|evening|dinner| +61584|AAAAAAAABJAPAAAA|61584|17|6|24|PM|second|evening|dinner| +61585|AAAAAAAACJAPAAAA|61585|17|6|25|PM|second|evening|dinner| +61586|AAAAAAAADJAPAAAA|61586|17|6|26|PM|second|evening|dinner| +61587|AAAAAAAAEJAPAAAA|61587|17|6|27|PM|second|evening|dinner| +61588|AAAAAAAAFJAPAAAA|61588|17|6|28|PM|second|evening|dinner| +61589|AAAAAAAAGJAPAAAA|61589|17|6|29|PM|second|evening|dinner| +61590|AAAAAAAAHJAPAAAA|61590|17|6|30|PM|second|evening|dinner| +61591|AAAAAAAAIJAPAAAA|61591|17|6|31|PM|second|evening|dinner| +61592|AAAAAAAAJJAPAAAA|61592|17|6|32|PM|second|evening|dinner| +61593|AAAAAAAAKJAPAAAA|61593|17|6|33|PM|second|evening|dinner| +61594|AAAAAAAALJAPAAAA|61594|17|6|34|PM|second|evening|dinner| +61595|AAAAAAAAMJAPAAAA|61595|17|6|35|PM|second|evening|dinner| +61596|AAAAAAAANJAPAAAA|61596|17|6|36|PM|second|evening|dinner| +61597|AAAAAAAAOJAPAAAA|61597|17|6|37|PM|second|evening|dinner| +61598|AAAAAAAAPJAPAAAA|61598|17|6|38|PM|second|evening|dinner| +61599|AAAAAAAAAKAPAAAA|61599|17|6|39|PM|second|evening|dinner| +61600|AAAAAAAABKAPAAAA|61600|17|6|40|PM|second|evening|dinner| +61601|AAAAAAAACKAPAAAA|61601|17|6|41|PM|second|evening|dinner| +61602|AAAAAAAADKAPAAAA|61602|17|6|42|PM|second|evening|dinner| +61603|AAAAAAAAEKAPAAAA|61603|17|6|43|PM|second|evening|dinner| +61604|AAAAAAAAFKAPAAAA|61604|17|6|44|PM|second|evening|dinner| +61605|AAAAAAAAGKAPAAAA|61605|17|6|45|PM|second|evening|dinner| +61606|AAAAAAAAHKAPAAAA|61606|17|6|46|PM|second|evening|dinner| +61607|AAAAAAAAIKAPAAAA|61607|17|6|47|PM|second|evening|dinner| +61608|AAAAAAAAJKAPAAAA|61608|17|6|48|PM|second|evening|dinner| +61609|AAAAAAAAKKAPAAAA|61609|17|6|49|PM|second|evening|dinner| +61610|AAAAAAAALKAPAAAA|61610|17|6|50|PM|second|evening|dinner| +61611|AAAAAAAAMKAPAAAA|61611|17|6|51|PM|second|evening|dinner| +61612|AAAAAAAANKAPAAAA|61612|17|6|52|PM|second|evening|dinner| +61613|AAAAAAAAOKAPAAAA|61613|17|6|53|PM|second|evening|dinner| +61614|AAAAAAAAPKAPAAAA|61614|17|6|54|PM|second|evening|dinner| +61615|AAAAAAAAALAPAAAA|61615|17|6|55|PM|second|evening|dinner| +61616|AAAAAAAABLAPAAAA|61616|17|6|56|PM|second|evening|dinner| +61617|AAAAAAAACLAPAAAA|61617|17|6|57|PM|second|evening|dinner| +61618|AAAAAAAADLAPAAAA|61618|17|6|58|PM|second|evening|dinner| +61619|AAAAAAAAELAPAAAA|61619|17|6|59|PM|second|evening|dinner| +61620|AAAAAAAAFLAPAAAA|61620|17|7|0|PM|second|evening|dinner| +61621|AAAAAAAAGLAPAAAA|61621|17|7|1|PM|second|evening|dinner| +61622|AAAAAAAAHLAPAAAA|61622|17|7|2|PM|second|evening|dinner| +61623|AAAAAAAAILAPAAAA|61623|17|7|3|PM|second|evening|dinner| +61624|AAAAAAAAJLAPAAAA|61624|17|7|4|PM|second|evening|dinner| +61625|AAAAAAAAKLAPAAAA|61625|17|7|5|PM|second|evening|dinner| +61626|AAAAAAAALLAPAAAA|61626|17|7|6|PM|second|evening|dinner| +61627|AAAAAAAAMLAPAAAA|61627|17|7|7|PM|second|evening|dinner| +61628|AAAAAAAANLAPAAAA|61628|17|7|8|PM|second|evening|dinner| +61629|AAAAAAAAOLAPAAAA|61629|17|7|9|PM|second|evening|dinner| +61630|AAAAAAAAPLAPAAAA|61630|17|7|10|PM|second|evening|dinner| +61631|AAAAAAAAAMAPAAAA|61631|17|7|11|PM|second|evening|dinner| +61632|AAAAAAAABMAPAAAA|61632|17|7|12|PM|second|evening|dinner| +61633|AAAAAAAACMAPAAAA|61633|17|7|13|PM|second|evening|dinner| +61634|AAAAAAAADMAPAAAA|61634|17|7|14|PM|second|evening|dinner| +61635|AAAAAAAAEMAPAAAA|61635|17|7|15|PM|second|evening|dinner| +61636|AAAAAAAAFMAPAAAA|61636|17|7|16|PM|second|evening|dinner| +61637|AAAAAAAAGMAPAAAA|61637|17|7|17|PM|second|evening|dinner| +61638|AAAAAAAAHMAPAAAA|61638|17|7|18|PM|second|evening|dinner| +61639|AAAAAAAAIMAPAAAA|61639|17|7|19|PM|second|evening|dinner| +61640|AAAAAAAAJMAPAAAA|61640|17|7|20|PM|second|evening|dinner| +61641|AAAAAAAAKMAPAAAA|61641|17|7|21|PM|second|evening|dinner| +61642|AAAAAAAALMAPAAAA|61642|17|7|22|PM|second|evening|dinner| +61643|AAAAAAAAMMAPAAAA|61643|17|7|23|PM|second|evening|dinner| +61644|AAAAAAAANMAPAAAA|61644|17|7|24|PM|second|evening|dinner| +61645|AAAAAAAAOMAPAAAA|61645|17|7|25|PM|second|evening|dinner| +61646|AAAAAAAAPMAPAAAA|61646|17|7|26|PM|second|evening|dinner| +61647|AAAAAAAAANAPAAAA|61647|17|7|27|PM|second|evening|dinner| +61648|AAAAAAAABNAPAAAA|61648|17|7|28|PM|second|evening|dinner| +61649|AAAAAAAACNAPAAAA|61649|17|7|29|PM|second|evening|dinner| +61650|AAAAAAAADNAPAAAA|61650|17|7|30|PM|second|evening|dinner| +61651|AAAAAAAAENAPAAAA|61651|17|7|31|PM|second|evening|dinner| +61652|AAAAAAAAFNAPAAAA|61652|17|7|32|PM|second|evening|dinner| +61653|AAAAAAAAGNAPAAAA|61653|17|7|33|PM|second|evening|dinner| +61654|AAAAAAAAHNAPAAAA|61654|17|7|34|PM|second|evening|dinner| +61655|AAAAAAAAINAPAAAA|61655|17|7|35|PM|second|evening|dinner| +61656|AAAAAAAAJNAPAAAA|61656|17|7|36|PM|second|evening|dinner| +61657|AAAAAAAAKNAPAAAA|61657|17|7|37|PM|second|evening|dinner| +61658|AAAAAAAALNAPAAAA|61658|17|7|38|PM|second|evening|dinner| +61659|AAAAAAAAMNAPAAAA|61659|17|7|39|PM|second|evening|dinner| +61660|AAAAAAAANNAPAAAA|61660|17|7|40|PM|second|evening|dinner| +61661|AAAAAAAAONAPAAAA|61661|17|7|41|PM|second|evening|dinner| +61662|AAAAAAAAPNAPAAAA|61662|17|7|42|PM|second|evening|dinner| +61663|AAAAAAAAAOAPAAAA|61663|17|7|43|PM|second|evening|dinner| +61664|AAAAAAAABOAPAAAA|61664|17|7|44|PM|second|evening|dinner| +61665|AAAAAAAACOAPAAAA|61665|17|7|45|PM|second|evening|dinner| +61666|AAAAAAAADOAPAAAA|61666|17|7|46|PM|second|evening|dinner| +61667|AAAAAAAAEOAPAAAA|61667|17|7|47|PM|second|evening|dinner| +61668|AAAAAAAAFOAPAAAA|61668|17|7|48|PM|second|evening|dinner| +61669|AAAAAAAAGOAPAAAA|61669|17|7|49|PM|second|evening|dinner| +61670|AAAAAAAAHOAPAAAA|61670|17|7|50|PM|second|evening|dinner| +61671|AAAAAAAAIOAPAAAA|61671|17|7|51|PM|second|evening|dinner| +61672|AAAAAAAAJOAPAAAA|61672|17|7|52|PM|second|evening|dinner| +61673|AAAAAAAAKOAPAAAA|61673|17|7|53|PM|second|evening|dinner| +61674|AAAAAAAALOAPAAAA|61674|17|7|54|PM|second|evening|dinner| +61675|AAAAAAAAMOAPAAAA|61675|17|7|55|PM|second|evening|dinner| +61676|AAAAAAAANOAPAAAA|61676|17|7|56|PM|second|evening|dinner| +61677|AAAAAAAAOOAPAAAA|61677|17|7|57|PM|second|evening|dinner| +61678|AAAAAAAAPOAPAAAA|61678|17|7|58|PM|second|evening|dinner| +61679|AAAAAAAAAPAPAAAA|61679|17|7|59|PM|second|evening|dinner| +61680|AAAAAAAABPAPAAAA|61680|17|8|0|PM|second|evening|dinner| +61681|AAAAAAAACPAPAAAA|61681|17|8|1|PM|second|evening|dinner| +61682|AAAAAAAADPAPAAAA|61682|17|8|2|PM|second|evening|dinner| +61683|AAAAAAAAEPAPAAAA|61683|17|8|3|PM|second|evening|dinner| +61684|AAAAAAAAFPAPAAAA|61684|17|8|4|PM|second|evening|dinner| +61685|AAAAAAAAGPAPAAAA|61685|17|8|5|PM|second|evening|dinner| +61686|AAAAAAAAHPAPAAAA|61686|17|8|6|PM|second|evening|dinner| +61687|AAAAAAAAIPAPAAAA|61687|17|8|7|PM|second|evening|dinner| +61688|AAAAAAAAJPAPAAAA|61688|17|8|8|PM|second|evening|dinner| +61689|AAAAAAAAKPAPAAAA|61689|17|8|9|PM|second|evening|dinner| +61690|AAAAAAAALPAPAAAA|61690|17|8|10|PM|second|evening|dinner| +61691|AAAAAAAAMPAPAAAA|61691|17|8|11|PM|second|evening|dinner| +61692|AAAAAAAANPAPAAAA|61692|17|8|12|PM|second|evening|dinner| +61693|AAAAAAAAOPAPAAAA|61693|17|8|13|PM|second|evening|dinner| +61694|AAAAAAAAPPAPAAAA|61694|17|8|14|PM|second|evening|dinner| +61695|AAAAAAAAAABPAAAA|61695|17|8|15|PM|second|evening|dinner| +61696|AAAAAAAABABPAAAA|61696|17|8|16|PM|second|evening|dinner| +61697|AAAAAAAACABPAAAA|61697|17|8|17|PM|second|evening|dinner| +61698|AAAAAAAADABPAAAA|61698|17|8|18|PM|second|evening|dinner| +61699|AAAAAAAAEABPAAAA|61699|17|8|19|PM|second|evening|dinner| +61700|AAAAAAAAFABPAAAA|61700|17|8|20|PM|second|evening|dinner| +61701|AAAAAAAAGABPAAAA|61701|17|8|21|PM|second|evening|dinner| +61702|AAAAAAAAHABPAAAA|61702|17|8|22|PM|second|evening|dinner| +61703|AAAAAAAAIABPAAAA|61703|17|8|23|PM|second|evening|dinner| +61704|AAAAAAAAJABPAAAA|61704|17|8|24|PM|second|evening|dinner| +61705|AAAAAAAAKABPAAAA|61705|17|8|25|PM|second|evening|dinner| +61706|AAAAAAAALABPAAAA|61706|17|8|26|PM|second|evening|dinner| +61707|AAAAAAAAMABPAAAA|61707|17|8|27|PM|second|evening|dinner| +61708|AAAAAAAANABPAAAA|61708|17|8|28|PM|second|evening|dinner| +61709|AAAAAAAAOABPAAAA|61709|17|8|29|PM|second|evening|dinner| +61710|AAAAAAAAPABPAAAA|61710|17|8|30|PM|second|evening|dinner| +61711|AAAAAAAAABBPAAAA|61711|17|8|31|PM|second|evening|dinner| +61712|AAAAAAAABBBPAAAA|61712|17|8|32|PM|second|evening|dinner| +61713|AAAAAAAACBBPAAAA|61713|17|8|33|PM|second|evening|dinner| +61714|AAAAAAAADBBPAAAA|61714|17|8|34|PM|second|evening|dinner| +61715|AAAAAAAAEBBPAAAA|61715|17|8|35|PM|second|evening|dinner| +61716|AAAAAAAAFBBPAAAA|61716|17|8|36|PM|second|evening|dinner| +61717|AAAAAAAAGBBPAAAA|61717|17|8|37|PM|second|evening|dinner| +61718|AAAAAAAAHBBPAAAA|61718|17|8|38|PM|second|evening|dinner| +61719|AAAAAAAAIBBPAAAA|61719|17|8|39|PM|second|evening|dinner| +61720|AAAAAAAAJBBPAAAA|61720|17|8|40|PM|second|evening|dinner| +61721|AAAAAAAAKBBPAAAA|61721|17|8|41|PM|second|evening|dinner| +61722|AAAAAAAALBBPAAAA|61722|17|8|42|PM|second|evening|dinner| +61723|AAAAAAAAMBBPAAAA|61723|17|8|43|PM|second|evening|dinner| +61724|AAAAAAAANBBPAAAA|61724|17|8|44|PM|second|evening|dinner| +61725|AAAAAAAAOBBPAAAA|61725|17|8|45|PM|second|evening|dinner| +61726|AAAAAAAAPBBPAAAA|61726|17|8|46|PM|second|evening|dinner| +61727|AAAAAAAAACBPAAAA|61727|17|8|47|PM|second|evening|dinner| +61728|AAAAAAAABCBPAAAA|61728|17|8|48|PM|second|evening|dinner| +61729|AAAAAAAACCBPAAAA|61729|17|8|49|PM|second|evening|dinner| +61730|AAAAAAAADCBPAAAA|61730|17|8|50|PM|second|evening|dinner| +61731|AAAAAAAAECBPAAAA|61731|17|8|51|PM|second|evening|dinner| +61732|AAAAAAAAFCBPAAAA|61732|17|8|52|PM|second|evening|dinner| +61733|AAAAAAAAGCBPAAAA|61733|17|8|53|PM|second|evening|dinner| +61734|AAAAAAAAHCBPAAAA|61734|17|8|54|PM|second|evening|dinner| +61735|AAAAAAAAICBPAAAA|61735|17|8|55|PM|second|evening|dinner| +61736|AAAAAAAAJCBPAAAA|61736|17|8|56|PM|second|evening|dinner| +61737|AAAAAAAAKCBPAAAA|61737|17|8|57|PM|second|evening|dinner| +61738|AAAAAAAALCBPAAAA|61738|17|8|58|PM|second|evening|dinner| +61739|AAAAAAAAMCBPAAAA|61739|17|8|59|PM|second|evening|dinner| +61740|AAAAAAAANCBPAAAA|61740|17|9|0|PM|second|evening|dinner| +61741|AAAAAAAAOCBPAAAA|61741|17|9|1|PM|second|evening|dinner| +61742|AAAAAAAAPCBPAAAA|61742|17|9|2|PM|second|evening|dinner| +61743|AAAAAAAAADBPAAAA|61743|17|9|3|PM|second|evening|dinner| +61744|AAAAAAAABDBPAAAA|61744|17|9|4|PM|second|evening|dinner| +61745|AAAAAAAACDBPAAAA|61745|17|9|5|PM|second|evening|dinner| +61746|AAAAAAAADDBPAAAA|61746|17|9|6|PM|second|evening|dinner| +61747|AAAAAAAAEDBPAAAA|61747|17|9|7|PM|second|evening|dinner| +61748|AAAAAAAAFDBPAAAA|61748|17|9|8|PM|second|evening|dinner| +61749|AAAAAAAAGDBPAAAA|61749|17|9|9|PM|second|evening|dinner| +61750|AAAAAAAAHDBPAAAA|61750|17|9|10|PM|second|evening|dinner| +61751|AAAAAAAAIDBPAAAA|61751|17|9|11|PM|second|evening|dinner| +61752|AAAAAAAAJDBPAAAA|61752|17|9|12|PM|second|evening|dinner| +61753|AAAAAAAAKDBPAAAA|61753|17|9|13|PM|second|evening|dinner| +61754|AAAAAAAALDBPAAAA|61754|17|9|14|PM|second|evening|dinner| +61755|AAAAAAAAMDBPAAAA|61755|17|9|15|PM|second|evening|dinner| +61756|AAAAAAAANDBPAAAA|61756|17|9|16|PM|second|evening|dinner| +61757|AAAAAAAAODBPAAAA|61757|17|9|17|PM|second|evening|dinner| +61758|AAAAAAAAPDBPAAAA|61758|17|9|18|PM|second|evening|dinner| +61759|AAAAAAAAAEBPAAAA|61759|17|9|19|PM|second|evening|dinner| +61760|AAAAAAAABEBPAAAA|61760|17|9|20|PM|second|evening|dinner| +61761|AAAAAAAACEBPAAAA|61761|17|9|21|PM|second|evening|dinner| +61762|AAAAAAAADEBPAAAA|61762|17|9|22|PM|second|evening|dinner| +61763|AAAAAAAAEEBPAAAA|61763|17|9|23|PM|second|evening|dinner| +61764|AAAAAAAAFEBPAAAA|61764|17|9|24|PM|second|evening|dinner| +61765|AAAAAAAAGEBPAAAA|61765|17|9|25|PM|second|evening|dinner| +61766|AAAAAAAAHEBPAAAA|61766|17|9|26|PM|second|evening|dinner| +61767|AAAAAAAAIEBPAAAA|61767|17|9|27|PM|second|evening|dinner| +61768|AAAAAAAAJEBPAAAA|61768|17|9|28|PM|second|evening|dinner| +61769|AAAAAAAAKEBPAAAA|61769|17|9|29|PM|second|evening|dinner| +61770|AAAAAAAALEBPAAAA|61770|17|9|30|PM|second|evening|dinner| +61771|AAAAAAAAMEBPAAAA|61771|17|9|31|PM|second|evening|dinner| +61772|AAAAAAAANEBPAAAA|61772|17|9|32|PM|second|evening|dinner| +61773|AAAAAAAAOEBPAAAA|61773|17|9|33|PM|second|evening|dinner| +61774|AAAAAAAAPEBPAAAA|61774|17|9|34|PM|second|evening|dinner| +61775|AAAAAAAAAFBPAAAA|61775|17|9|35|PM|second|evening|dinner| +61776|AAAAAAAABFBPAAAA|61776|17|9|36|PM|second|evening|dinner| +61777|AAAAAAAACFBPAAAA|61777|17|9|37|PM|second|evening|dinner| +61778|AAAAAAAADFBPAAAA|61778|17|9|38|PM|second|evening|dinner| +61779|AAAAAAAAEFBPAAAA|61779|17|9|39|PM|second|evening|dinner| +61780|AAAAAAAAFFBPAAAA|61780|17|9|40|PM|second|evening|dinner| +61781|AAAAAAAAGFBPAAAA|61781|17|9|41|PM|second|evening|dinner| +61782|AAAAAAAAHFBPAAAA|61782|17|9|42|PM|second|evening|dinner| +61783|AAAAAAAAIFBPAAAA|61783|17|9|43|PM|second|evening|dinner| +61784|AAAAAAAAJFBPAAAA|61784|17|9|44|PM|second|evening|dinner| +61785|AAAAAAAAKFBPAAAA|61785|17|9|45|PM|second|evening|dinner| +61786|AAAAAAAALFBPAAAA|61786|17|9|46|PM|second|evening|dinner| +61787|AAAAAAAAMFBPAAAA|61787|17|9|47|PM|second|evening|dinner| +61788|AAAAAAAANFBPAAAA|61788|17|9|48|PM|second|evening|dinner| +61789|AAAAAAAAOFBPAAAA|61789|17|9|49|PM|second|evening|dinner| +61790|AAAAAAAAPFBPAAAA|61790|17|9|50|PM|second|evening|dinner| +61791|AAAAAAAAAGBPAAAA|61791|17|9|51|PM|second|evening|dinner| +61792|AAAAAAAABGBPAAAA|61792|17|9|52|PM|second|evening|dinner| +61793|AAAAAAAACGBPAAAA|61793|17|9|53|PM|second|evening|dinner| +61794|AAAAAAAADGBPAAAA|61794|17|9|54|PM|second|evening|dinner| +61795|AAAAAAAAEGBPAAAA|61795|17|9|55|PM|second|evening|dinner| +61796|AAAAAAAAFGBPAAAA|61796|17|9|56|PM|second|evening|dinner| +61797|AAAAAAAAGGBPAAAA|61797|17|9|57|PM|second|evening|dinner| +61798|AAAAAAAAHGBPAAAA|61798|17|9|58|PM|second|evening|dinner| +61799|AAAAAAAAIGBPAAAA|61799|17|9|59|PM|second|evening|dinner| +61800|AAAAAAAAJGBPAAAA|61800|17|10|0|PM|second|evening|dinner| +61801|AAAAAAAAKGBPAAAA|61801|17|10|1|PM|second|evening|dinner| +61802|AAAAAAAALGBPAAAA|61802|17|10|2|PM|second|evening|dinner| +61803|AAAAAAAAMGBPAAAA|61803|17|10|3|PM|second|evening|dinner| +61804|AAAAAAAANGBPAAAA|61804|17|10|4|PM|second|evening|dinner| +61805|AAAAAAAAOGBPAAAA|61805|17|10|5|PM|second|evening|dinner| +61806|AAAAAAAAPGBPAAAA|61806|17|10|6|PM|second|evening|dinner| +61807|AAAAAAAAAHBPAAAA|61807|17|10|7|PM|second|evening|dinner| +61808|AAAAAAAABHBPAAAA|61808|17|10|8|PM|second|evening|dinner| +61809|AAAAAAAACHBPAAAA|61809|17|10|9|PM|second|evening|dinner| +61810|AAAAAAAADHBPAAAA|61810|17|10|10|PM|second|evening|dinner| +61811|AAAAAAAAEHBPAAAA|61811|17|10|11|PM|second|evening|dinner| +61812|AAAAAAAAFHBPAAAA|61812|17|10|12|PM|second|evening|dinner| +61813|AAAAAAAAGHBPAAAA|61813|17|10|13|PM|second|evening|dinner| +61814|AAAAAAAAHHBPAAAA|61814|17|10|14|PM|second|evening|dinner| +61815|AAAAAAAAIHBPAAAA|61815|17|10|15|PM|second|evening|dinner| +61816|AAAAAAAAJHBPAAAA|61816|17|10|16|PM|second|evening|dinner| +61817|AAAAAAAAKHBPAAAA|61817|17|10|17|PM|second|evening|dinner| +61818|AAAAAAAALHBPAAAA|61818|17|10|18|PM|second|evening|dinner| +61819|AAAAAAAAMHBPAAAA|61819|17|10|19|PM|second|evening|dinner| +61820|AAAAAAAANHBPAAAA|61820|17|10|20|PM|second|evening|dinner| +61821|AAAAAAAAOHBPAAAA|61821|17|10|21|PM|second|evening|dinner| +61822|AAAAAAAAPHBPAAAA|61822|17|10|22|PM|second|evening|dinner| +61823|AAAAAAAAAIBPAAAA|61823|17|10|23|PM|second|evening|dinner| +61824|AAAAAAAABIBPAAAA|61824|17|10|24|PM|second|evening|dinner| +61825|AAAAAAAACIBPAAAA|61825|17|10|25|PM|second|evening|dinner| +61826|AAAAAAAADIBPAAAA|61826|17|10|26|PM|second|evening|dinner| +61827|AAAAAAAAEIBPAAAA|61827|17|10|27|PM|second|evening|dinner| +61828|AAAAAAAAFIBPAAAA|61828|17|10|28|PM|second|evening|dinner| +61829|AAAAAAAAGIBPAAAA|61829|17|10|29|PM|second|evening|dinner| +61830|AAAAAAAAHIBPAAAA|61830|17|10|30|PM|second|evening|dinner| +61831|AAAAAAAAIIBPAAAA|61831|17|10|31|PM|second|evening|dinner| +61832|AAAAAAAAJIBPAAAA|61832|17|10|32|PM|second|evening|dinner| +61833|AAAAAAAAKIBPAAAA|61833|17|10|33|PM|second|evening|dinner| +61834|AAAAAAAALIBPAAAA|61834|17|10|34|PM|second|evening|dinner| +61835|AAAAAAAAMIBPAAAA|61835|17|10|35|PM|second|evening|dinner| +61836|AAAAAAAANIBPAAAA|61836|17|10|36|PM|second|evening|dinner| +61837|AAAAAAAAOIBPAAAA|61837|17|10|37|PM|second|evening|dinner| +61838|AAAAAAAAPIBPAAAA|61838|17|10|38|PM|second|evening|dinner| +61839|AAAAAAAAAJBPAAAA|61839|17|10|39|PM|second|evening|dinner| +61840|AAAAAAAABJBPAAAA|61840|17|10|40|PM|second|evening|dinner| +61841|AAAAAAAACJBPAAAA|61841|17|10|41|PM|second|evening|dinner| +61842|AAAAAAAADJBPAAAA|61842|17|10|42|PM|second|evening|dinner| +61843|AAAAAAAAEJBPAAAA|61843|17|10|43|PM|second|evening|dinner| +61844|AAAAAAAAFJBPAAAA|61844|17|10|44|PM|second|evening|dinner| +61845|AAAAAAAAGJBPAAAA|61845|17|10|45|PM|second|evening|dinner| +61846|AAAAAAAAHJBPAAAA|61846|17|10|46|PM|second|evening|dinner| +61847|AAAAAAAAIJBPAAAA|61847|17|10|47|PM|second|evening|dinner| +61848|AAAAAAAAJJBPAAAA|61848|17|10|48|PM|second|evening|dinner| +61849|AAAAAAAAKJBPAAAA|61849|17|10|49|PM|second|evening|dinner| +61850|AAAAAAAALJBPAAAA|61850|17|10|50|PM|second|evening|dinner| +61851|AAAAAAAAMJBPAAAA|61851|17|10|51|PM|second|evening|dinner| +61852|AAAAAAAANJBPAAAA|61852|17|10|52|PM|second|evening|dinner| +61853|AAAAAAAAOJBPAAAA|61853|17|10|53|PM|second|evening|dinner| +61854|AAAAAAAAPJBPAAAA|61854|17|10|54|PM|second|evening|dinner| +61855|AAAAAAAAAKBPAAAA|61855|17|10|55|PM|second|evening|dinner| +61856|AAAAAAAABKBPAAAA|61856|17|10|56|PM|second|evening|dinner| +61857|AAAAAAAACKBPAAAA|61857|17|10|57|PM|second|evening|dinner| +61858|AAAAAAAADKBPAAAA|61858|17|10|58|PM|second|evening|dinner| +61859|AAAAAAAAEKBPAAAA|61859|17|10|59|PM|second|evening|dinner| +61860|AAAAAAAAFKBPAAAA|61860|17|11|0|PM|second|evening|dinner| +61861|AAAAAAAAGKBPAAAA|61861|17|11|1|PM|second|evening|dinner| +61862|AAAAAAAAHKBPAAAA|61862|17|11|2|PM|second|evening|dinner| +61863|AAAAAAAAIKBPAAAA|61863|17|11|3|PM|second|evening|dinner| +61864|AAAAAAAAJKBPAAAA|61864|17|11|4|PM|second|evening|dinner| +61865|AAAAAAAAKKBPAAAA|61865|17|11|5|PM|second|evening|dinner| +61866|AAAAAAAALKBPAAAA|61866|17|11|6|PM|second|evening|dinner| +61867|AAAAAAAAMKBPAAAA|61867|17|11|7|PM|second|evening|dinner| +61868|AAAAAAAANKBPAAAA|61868|17|11|8|PM|second|evening|dinner| +61869|AAAAAAAAOKBPAAAA|61869|17|11|9|PM|second|evening|dinner| +61870|AAAAAAAAPKBPAAAA|61870|17|11|10|PM|second|evening|dinner| +61871|AAAAAAAAALBPAAAA|61871|17|11|11|PM|second|evening|dinner| +61872|AAAAAAAABLBPAAAA|61872|17|11|12|PM|second|evening|dinner| +61873|AAAAAAAACLBPAAAA|61873|17|11|13|PM|second|evening|dinner| +61874|AAAAAAAADLBPAAAA|61874|17|11|14|PM|second|evening|dinner| +61875|AAAAAAAAELBPAAAA|61875|17|11|15|PM|second|evening|dinner| +61876|AAAAAAAAFLBPAAAA|61876|17|11|16|PM|second|evening|dinner| +61877|AAAAAAAAGLBPAAAA|61877|17|11|17|PM|second|evening|dinner| +61878|AAAAAAAAHLBPAAAA|61878|17|11|18|PM|second|evening|dinner| +61879|AAAAAAAAILBPAAAA|61879|17|11|19|PM|second|evening|dinner| +61880|AAAAAAAAJLBPAAAA|61880|17|11|20|PM|second|evening|dinner| +61881|AAAAAAAAKLBPAAAA|61881|17|11|21|PM|second|evening|dinner| +61882|AAAAAAAALLBPAAAA|61882|17|11|22|PM|second|evening|dinner| +61883|AAAAAAAAMLBPAAAA|61883|17|11|23|PM|second|evening|dinner| +61884|AAAAAAAANLBPAAAA|61884|17|11|24|PM|second|evening|dinner| +61885|AAAAAAAAOLBPAAAA|61885|17|11|25|PM|second|evening|dinner| +61886|AAAAAAAAPLBPAAAA|61886|17|11|26|PM|second|evening|dinner| +61887|AAAAAAAAAMBPAAAA|61887|17|11|27|PM|second|evening|dinner| +61888|AAAAAAAABMBPAAAA|61888|17|11|28|PM|second|evening|dinner| +61889|AAAAAAAACMBPAAAA|61889|17|11|29|PM|second|evening|dinner| +61890|AAAAAAAADMBPAAAA|61890|17|11|30|PM|second|evening|dinner| +61891|AAAAAAAAEMBPAAAA|61891|17|11|31|PM|second|evening|dinner| +61892|AAAAAAAAFMBPAAAA|61892|17|11|32|PM|second|evening|dinner| +61893|AAAAAAAAGMBPAAAA|61893|17|11|33|PM|second|evening|dinner| +61894|AAAAAAAAHMBPAAAA|61894|17|11|34|PM|second|evening|dinner| +61895|AAAAAAAAIMBPAAAA|61895|17|11|35|PM|second|evening|dinner| +61896|AAAAAAAAJMBPAAAA|61896|17|11|36|PM|second|evening|dinner| +61897|AAAAAAAAKMBPAAAA|61897|17|11|37|PM|second|evening|dinner| +61898|AAAAAAAALMBPAAAA|61898|17|11|38|PM|second|evening|dinner| +61899|AAAAAAAAMMBPAAAA|61899|17|11|39|PM|second|evening|dinner| +61900|AAAAAAAANMBPAAAA|61900|17|11|40|PM|second|evening|dinner| +61901|AAAAAAAAOMBPAAAA|61901|17|11|41|PM|second|evening|dinner| +61902|AAAAAAAAPMBPAAAA|61902|17|11|42|PM|second|evening|dinner| +61903|AAAAAAAAANBPAAAA|61903|17|11|43|PM|second|evening|dinner| +61904|AAAAAAAABNBPAAAA|61904|17|11|44|PM|second|evening|dinner| +61905|AAAAAAAACNBPAAAA|61905|17|11|45|PM|second|evening|dinner| +61906|AAAAAAAADNBPAAAA|61906|17|11|46|PM|second|evening|dinner| +61907|AAAAAAAAENBPAAAA|61907|17|11|47|PM|second|evening|dinner| +61908|AAAAAAAAFNBPAAAA|61908|17|11|48|PM|second|evening|dinner| +61909|AAAAAAAAGNBPAAAA|61909|17|11|49|PM|second|evening|dinner| +61910|AAAAAAAAHNBPAAAA|61910|17|11|50|PM|second|evening|dinner| +61911|AAAAAAAAINBPAAAA|61911|17|11|51|PM|second|evening|dinner| +61912|AAAAAAAAJNBPAAAA|61912|17|11|52|PM|second|evening|dinner| +61913|AAAAAAAAKNBPAAAA|61913|17|11|53|PM|second|evening|dinner| +61914|AAAAAAAALNBPAAAA|61914|17|11|54|PM|second|evening|dinner| +61915|AAAAAAAAMNBPAAAA|61915|17|11|55|PM|second|evening|dinner| +61916|AAAAAAAANNBPAAAA|61916|17|11|56|PM|second|evening|dinner| +61917|AAAAAAAAONBPAAAA|61917|17|11|57|PM|second|evening|dinner| +61918|AAAAAAAAPNBPAAAA|61918|17|11|58|PM|second|evening|dinner| +61919|AAAAAAAAAOBPAAAA|61919|17|11|59|PM|second|evening|dinner| +61920|AAAAAAAABOBPAAAA|61920|17|12|0|PM|second|evening|dinner| +61921|AAAAAAAACOBPAAAA|61921|17|12|1|PM|second|evening|dinner| +61922|AAAAAAAADOBPAAAA|61922|17|12|2|PM|second|evening|dinner| +61923|AAAAAAAAEOBPAAAA|61923|17|12|3|PM|second|evening|dinner| +61924|AAAAAAAAFOBPAAAA|61924|17|12|4|PM|second|evening|dinner| +61925|AAAAAAAAGOBPAAAA|61925|17|12|5|PM|second|evening|dinner| +61926|AAAAAAAAHOBPAAAA|61926|17|12|6|PM|second|evening|dinner| +61927|AAAAAAAAIOBPAAAA|61927|17|12|7|PM|second|evening|dinner| +61928|AAAAAAAAJOBPAAAA|61928|17|12|8|PM|second|evening|dinner| +61929|AAAAAAAAKOBPAAAA|61929|17|12|9|PM|second|evening|dinner| +61930|AAAAAAAALOBPAAAA|61930|17|12|10|PM|second|evening|dinner| +61931|AAAAAAAAMOBPAAAA|61931|17|12|11|PM|second|evening|dinner| +61932|AAAAAAAANOBPAAAA|61932|17|12|12|PM|second|evening|dinner| +61933|AAAAAAAAOOBPAAAA|61933|17|12|13|PM|second|evening|dinner| +61934|AAAAAAAAPOBPAAAA|61934|17|12|14|PM|second|evening|dinner| +61935|AAAAAAAAAPBPAAAA|61935|17|12|15|PM|second|evening|dinner| +61936|AAAAAAAABPBPAAAA|61936|17|12|16|PM|second|evening|dinner| +61937|AAAAAAAACPBPAAAA|61937|17|12|17|PM|second|evening|dinner| +61938|AAAAAAAADPBPAAAA|61938|17|12|18|PM|second|evening|dinner| +61939|AAAAAAAAEPBPAAAA|61939|17|12|19|PM|second|evening|dinner| +61940|AAAAAAAAFPBPAAAA|61940|17|12|20|PM|second|evening|dinner| +61941|AAAAAAAAGPBPAAAA|61941|17|12|21|PM|second|evening|dinner| +61942|AAAAAAAAHPBPAAAA|61942|17|12|22|PM|second|evening|dinner| +61943|AAAAAAAAIPBPAAAA|61943|17|12|23|PM|second|evening|dinner| +61944|AAAAAAAAJPBPAAAA|61944|17|12|24|PM|second|evening|dinner| +61945|AAAAAAAAKPBPAAAA|61945|17|12|25|PM|second|evening|dinner| +61946|AAAAAAAALPBPAAAA|61946|17|12|26|PM|second|evening|dinner| +61947|AAAAAAAAMPBPAAAA|61947|17|12|27|PM|second|evening|dinner| +61948|AAAAAAAANPBPAAAA|61948|17|12|28|PM|second|evening|dinner| +61949|AAAAAAAAOPBPAAAA|61949|17|12|29|PM|second|evening|dinner| +61950|AAAAAAAAPPBPAAAA|61950|17|12|30|PM|second|evening|dinner| +61951|AAAAAAAAAACPAAAA|61951|17|12|31|PM|second|evening|dinner| +61952|AAAAAAAABACPAAAA|61952|17|12|32|PM|second|evening|dinner| +61953|AAAAAAAACACPAAAA|61953|17|12|33|PM|second|evening|dinner| +61954|AAAAAAAADACPAAAA|61954|17|12|34|PM|second|evening|dinner| +61955|AAAAAAAAEACPAAAA|61955|17|12|35|PM|second|evening|dinner| +61956|AAAAAAAAFACPAAAA|61956|17|12|36|PM|second|evening|dinner| +61957|AAAAAAAAGACPAAAA|61957|17|12|37|PM|second|evening|dinner| +61958|AAAAAAAAHACPAAAA|61958|17|12|38|PM|second|evening|dinner| +61959|AAAAAAAAIACPAAAA|61959|17|12|39|PM|second|evening|dinner| +61960|AAAAAAAAJACPAAAA|61960|17|12|40|PM|second|evening|dinner| +61961|AAAAAAAAKACPAAAA|61961|17|12|41|PM|second|evening|dinner| +61962|AAAAAAAALACPAAAA|61962|17|12|42|PM|second|evening|dinner| +61963|AAAAAAAAMACPAAAA|61963|17|12|43|PM|second|evening|dinner| +61964|AAAAAAAANACPAAAA|61964|17|12|44|PM|second|evening|dinner| +61965|AAAAAAAAOACPAAAA|61965|17|12|45|PM|second|evening|dinner| +61966|AAAAAAAAPACPAAAA|61966|17|12|46|PM|second|evening|dinner| +61967|AAAAAAAAABCPAAAA|61967|17|12|47|PM|second|evening|dinner| +61968|AAAAAAAABBCPAAAA|61968|17|12|48|PM|second|evening|dinner| +61969|AAAAAAAACBCPAAAA|61969|17|12|49|PM|second|evening|dinner| +61970|AAAAAAAADBCPAAAA|61970|17|12|50|PM|second|evening|dinner| +61971|AAAAAAAAEBCPAAAA|61971|17|12|51|PM|second|evening|dinner| +61972|AAAAAAAAFBCPAAAA|61972|17|12|52|PM|second|evening|dinner| +61973|AAAAAAAAGBCPAAAA|61973|17|12|53|PM|second|evening|dinner| +61974|AAAAAAAAHBCPAAAA|61974|17|12|54|PM|second|evening|dinner| +61975|AAAAAAAAIBCPAAAA|61975|17|12|55|PM|second|evening|dinner| +61976|AAAAAAAAJBCPAAAA|61976|17|12|56|PM|second|evening|dinner| +61977|AAAAAAAAKBCPAAAA|61977|17|12|57|PM|second|evening|dinner| +61978|AAAAAAAALBCPAAAA|61978|17|12|58|PM|second|evening|dinner| +61979|AAAAAAAAMBCPAAAA|61979|17|12|59|PM|second|evening|dinner| +61980|AAAAAAAANBCPAAAA|61980|17|13|0|PM|second|evening|dinner| +61981|AAAAAAAAOBCPAAAA|61981|17|13|1|PM|second|evening|dinner| +61982|AAAAAAAAPBCPAAAA|61982|17|13|2|PM|second|evening|dinner| +61983|AAAAAAAAACCPAAAA|61983|17|13|3|PM|second|evening|dinner| +61984|AAAAAAAABCCPAAAA|61984|17|13|4|PM|second|evening|dinner| +61985|AAAAAAAACCCPAAAA|61985|17|13|5|PM|second|evening|dinner| +61986|AAAAAAAADCCPAAAA|61986|17|13|6|PM|second|evening|dinner| +61987|AAAAAAAAECCPAAAA|61987|17|13|7|PM|second|evening|dinner| +61988|AAAAAAAAFCCPAAAA|61988|17|13|8|PM|second|evening|dinner| +61989|AAAAAAAAGCCPAAAA|61989|17|13|9|PM|second|evening|dinner| +61990|AAAAAAAAHCCPAAAA|61990|17|13|10|PM|second|evening|dinner| +61991|AAAAAAAAICCPAAAA|61991|17|13|11|PM|second|evening|dinner| +61992|AAAAAAAAJCCPAAAA|61992|17|13|12|PM|second|evening|dinner| +61993|AAAAAAAAKCCPAAAA|61993|17|13|13|PM|second|evening|dinner| +61994|AAAAAAAALCCPAAAA|61994|17|13|14|PM|second|evening|dinner| +61995|AAAAAAAAMCCPAAAA|61995|17|13|15|PM|second|evening|dinner| +61996|AAAAAAAANCCPAAAA|61996|17|13|16|PM|second|evening|dinner| +61997|AAAAAAAAOCCPAAAA|61997|17|13|17|PM|second|evening|dinner| +61998|AAAAAAAAPCCPAAAA|61998|17|13|18|PM|second|evening|dinner| +61999|AAAAAAAAADCPAAAA|61999|17|13|19|PM|second|evening|dinner| +62000|AAAAAAAABDCPAAAA|62000|17|13|20|PM|second|evening|dinner| +62001|AAAAAAAACDCPAAAA|62001|17|13|21|PM|second|evening|dinner| +62002|AAAAAAAADDCPAAAA|62002|17|13|22|PM|second|evening|dinner| +62003|AAAAAAAAEDCPAAAA|62003|17|13|23|PM|second|evening|dinner| +62004|AAAAAAAAFDCPAAAA|62004|17|13|24|PM|second|evening|dinner| +62005|AAAAAAAAGDCPAAAA|62005|17|13|25|PM|second|evening|dinner| +62006|AAAAAAAAHDCPAAAA|62006|17|13|26|PM|second|evening|dinner| +62007|AAAAAAAAIDCPAAAA|62007|17|13|27|PM|second|evening|dinner| +62008|AAAAAAAAJDCPAAAA|62008|17|13|28|PM|second|evening|dinner| +62009|AAAAAAAAKDCPAAAA|62009|17|13|29|PM|second|evening|dinner| +62010|AAAAAAAALDCPAAAA|62010|17|13|30|PM|second|evening|dinner| +62011|AAAAAAAAMDCPAAAA|62011|17|13|31|PM|second|evening|dinner| +62012|AAAAAAAANDCPAAAA|62012|17|13|32|PM|second|evening|dinner| +62013|AAAAAAAAODCPAAAA|62013|17|13|33|PM|second|evening|dinner| +62014|AAAAAAAAPDCPAAAA|62014|17|13|34|PM|second|evening|dinner| +62015|AAAAAAAAAECPAAAA|62015|17|13|35|PM|second|evening|dinner| +62016|AAAAAAAABECPAAAA|62016|17|13|36|PM|second|evening|dinner| +62017|AAAAAAAACECPAAAA|62017|17|13|37|PM|second|evening|dinner| +62018|AAAAAAAADECPAAAA|62018|17|13|38|PM|second|evening|dinner| +62019|AAAAAAAAEECPAAAA|62019|17|13|39|PM|second|evening|dinner| +62020|AAAAAAAAFECPAAAA|62020|17|13|40|PM|second|evening|dinner| +62021|AAAAAAAAGECPAAAA|62021|17|13|41|PM|second|evening|dinner| +62022|AAAAAAAAHECPAAAA|62022|17|13|42|PM|second|evening|dinner| +62023|AAAAAAAAIECPAAAA|62023|17|13|43|PM|second|evening|dinner| +62024|AAAAAAAAJECPAAAA|62024|17|13|44|PM|second|evening|dinner| +62025|AAAAAAAAKECPAAAA|62025|17|13|45|PM|second|evening|dinner| +62026|AAAAAAAALECPAAAA|62026|17|13|46|PM|second|evening|dinner| +62027|AAAAAAAAMECPAAAA|62027|17|13|47|PM|second|evening|dinner| +62028|AAAAAAAANECPAAAA|62028|17|13|48|PM|second|evening|dinner| +62029|AAAAAAAAOECPAAAA|62029|17|13|49|PM|second|evening|dinner| +62030|AAAAAAAAPECPAAAA|62030|17|13|50|PM|second|evening|dinner| +62031|AAAAAAAAAFCPAAAA|62031|17|13|51|PM|second|evening|dinner| +62032|AAAAAAAABFCPAAAA|62032|17|13|52|PM|second|evening|dinner| +62033|AAAAAAAACFCPAAAA|62033|17|13|53|PM|second|evening|dinner| +62034|AAAAAAAADFCPAAAA|62034|17|13|54|PM|second|evening|dinner| +62035|AAAAAAAAEFCPAAAA|62035|17|13|55|PM|second|evening|dinner| +62036|AAAAAAAAFFCPAAAA|62036|17|13|56|PM|second|evening|dinner| +62037|AAAAAAAAGFCPAAAA|62037|17|13|57|PM|second|evening|dinner| +62038|AAAAAAAAHFCPAAAA|62038|17|13|58|PM|second|evening|dinner| +62039|AAAAAAAAIFCPAAAA|62039|17|13|59|PM|second|evening|dinner| +62040|AAAAAAAAJFCPAAAA|62040|17|14|0|PM|second|evening|dinner| +62041|AAAAAAAAKFCPAAAA|62041|17|14|1|PM|second|evening|dinner| +62042|AAAAAAAALFCPAAAA|62042|17|14|2|PM|second|evening|dinner| +62043|AAAAAAAAMFCPAAAA|62043|17|14|3|PM|second|evening|dinner| +62044|AAAAAAAANFCPAAAA|62044|17|14|4|PM|second|evening|dinner| +62045|AAAAAAAAOFCPAAAA|62045|17|14|5|PM|second|evening|dinner| +62046|AAAAAAAAPFCPAAAA|62046|17|14|6|PM|second|evening|dinner| +62047|AAAAAAAAAGCPAAAA|62047|17|14|7|PM|second|evening|dinner| +62048|AAAAAAAABGCPAAAA|62048|17|14|8|PM|second|evening|dinner| +62049|AAAAAAAACGCPAAAA|62049|17|14|9|PM|second|evening|dinner| +62050|AAAAAAAADGCPAAAA|62050|17|14|10|PM|second|evening|dinner| +62051|AAAAAAAAEGCPAAAA|62051|17|14|11|PM|second|evening|dinner| +62052|AAAAAAAAFGCPAAAA|62052|17|14|12|PM|second|evening|dinner| +62053|AAAAAAAAGGCPAAAA|62053|17|14|13|PM|second|evening|dinner| +62054|AAAAAAAAHGCPAAAA|62054|17|14|14|PM|second|evening|dinner| +62055|AAAAAAAAIGCPAAAA|62055|17|14|15|PM|second|evening|dinner| +62056|AAAAAAAAJGCPAAAA|62056|17|14|16|PM|second|evening|dinner| +62057|AAAAAAAAKGCPAAAA|62057|17|14|17|PM|second|evening|dinner| +62058|AAAAAAAALGCPAAAA|62058|17|14|18|PM|second|evening|dinner| +62059|AAAAAAAAMGCPAAAA|62059|17|14|19|PM|second|evening|dinner| +62060|AAAAAAAANGCPAAAA|62060|17|14|20|PM|second|evening|dinner| +62061|AAAAAAAAOGCPAAAA|62061|17|14|21|PM|second|evening|dinner| +62062|AAAAAAAAPGCPAAAA|62062|17|14|22|PM|second|evening|dinner| +62063|AAAAAAAAAHCPAAAA|62063|17|14|23|PM|second|evening|dinner| +62064|AAAAAAAABHCPAAAA|62064|17|14|24|PM|second|evening|dinner| +62065|AAAAAAAACHCPAAAA|62065|17|14|25|PM|second|evening|dinner| +62066|AAAAAAAADHCPAAAA|62066|17|14|26|PM|second|evening|dinner| +62067|AAAAAAAAEHCPAAAA|62067|17|14|27|PM|second|evening|dinner| +62068|AAAAAAAAFHCPAAAA|62068|17|14|28|PM|second|evening|dinner| +62069|AAAAAAAAGHCPAAAA|62069|17|14|29|PM|second|evening|dinner| +62070|AAAAAAAAHHCPAAAA|62070|17|14|30|PM|second|evening|dinner| +62071|AAAAAAAAIHCPAAAA|62071|17|14|31|PM|second|evening|dinner| +62072|AAAAAAAAJHCPAAAA|62072|17|14|32|PM|second|evening|dinner| +62073|AAAAAAAAKHCPAAAA|62073|17|14|33|PM|second|evening|dinner| +62074|AAAAAAAALHCPAAAA|62074|17|14|34|PM|second|evening|dinner| +62075|AAAAAAAAMHCPAAAA|62075|17|14|35|PM|second|evening|dinner| +62076|AAAAAAAANHCPAAAA|62076|17|14|36|PM|second|evening|dinner| +62077|AAAAAAAAOHCPAAAA|62077|17|14|37|PM|second|evening|dinner| +62078|AAAAAAAAPHCPAAAA|62078|17|14|38|PM|second|evening|dinner| +62079|AAAAAAAAAICPAAAA|62079|17|14|39|PM|second|evening|dinner| +62080|AAAAAAAABICPAAAA|62080|17|14|40|PM|second|evening|dinner| +62081|AAAAAAAACICPAAAA|62081|17|14|41|PM|second|evening|dinner| +62082|AAAAAAAADICPAAAA|62082|17|14|42|PM|second|evening|dinner| +62083|AAAAAAAAEICPAAAA|62083|17|14|43|PM|second|evening|dinner| +62084|AAAAAAAAFICPAAAA|62084|17|14|44|PM|second|evening|dinner| +62085|AAAAAAAAGICPAAAA|62085|17|14|45|PM|second|evening|dinner| +62086|AAAAAAAAHICPAAAA|62086|17|14|46|PM|second|evening|dinner| +62087|AAAAAAAAIICPAAAA|62087|17|14|47|PM|second|evening|dinner| +62088|AAAAAAAAJICPAAAA|62088|17|14|48|PM|second|evening|dinner| +62089|AAAAAAAAKICPAAAA|62089|17|14|49|PM|second|evening|dinner| +62090|AAAAAAAALICPAAAA|62090|17|14|50|PM|second|evening|dinner| +62091|AAAAAAAAMICPAAAA|62091|17|14|51|PM|second|evening|dinner| +62092|AAAAAAAANICPAAAA|62092|17|14|52|PM|second|evening|dinner| +62093|AAAAAAAAOICPAAAA|62093|17|14|53|PM|second|evening|dinner| +62094|AAAAAAAAPICPAAAA|62094|17|14|54|PM|second|evening|dinner| +62095|AAAAAAAAAJCPAAAA|62095|17|14|55|PM|second|evening|dinner| +62096|AAAAAAAABJCPAAAA|62096|17|14|56|PM|second|evening|dinner| +62097|AAAAAAAACJCPAAAA|62097|17|14|57|PM|second|evening|dinner| +62098|AAAAAAAADJCPAAAA|62098|17|14|58|PM|second|evening|dinner| +62099|AAAAAAAAEJCPAAAA|62099|17|14|59|PM|second|evening|dinner| +62100|AAAAAAAAFJCPAAAA|62100|17|15|0|PM|second|evening|dinner| +62101|AAAAAAAAGJCPAAAA|62101|17|15|1|PM|second|evening|dinner| +62102|AAAAAAAAHJCPAAAA|62102|17|15|2|PM|second|evening|dinner| +62103|AAAAAAAAIJCPAAAA|62103|17|15|3|PM|second|evening|dinner| +62104|AAAAAAAAJJCPAAAA|62104|17|15|4|PM|second|evening|dinner| +62105|AAAAAAAAKJCPAAAA|62105|17|15|5|PM|second|evening|dinner| +62106|AAAAAAAALJCPAAAA|62106|17|15|6|PM|second|evening|dinner| +62107|AAAAAAAAMJCPAAAA|62107|17|15|7|PM|second|evening|dinner| +62108|AAAAAAAANJCPAAAA|62108|17|15|8|PM|second|evening|dinner| +62109|AAAAAAAAOJCPAAAA|62109|17|15|9|PM|second|evening|dinner| +62110|AAAAAAAAPJCPAAAA|62110|17|15|10|PM|second|evening|dinner| +62111|AAAAAAAAAKCPAAAA|62111|17|15|11|PM|second|evening|dinner| +62112|AAAAAAAABKCPAAAA|62112|17|15|12|PM|second|evening|dinner| +62113|AAAAAAAACKCPAAAA|62113|17|15|13|PM|second|evening|dinner| +62114|AAAAAAAADKCPAAAA|62114|17|15|14|PM|second|evening|dinner| +62115|AAAAAAAAEKCPAAAA|62115|17|15|15|PM|second|evening|dinner| +62116|AAAAAAAAFKCPAAAA|62116|17|15|16|PM|second|evening|dinner| +62117|AAAAAAAAGKCPAAAA|62117|17|15|17|PM|second|evening|dinner| +62118|AAAAAAAAHKCPAAAA|62118|17|15|18|PM|second|evening|dinner| +62119|AAAAAAAAIKCPAAAA|62119|17|15|19|PM|second|evening|dinner| +62120|AAAAAAAAJKCPAAAA|62120|17|15|20|PM|second|evening|dinner| +62121|AAAAAAAAKKCPAAAA|62121|17|15|21|PM|second|evening|dinner| +62122|AAAAAAAALKCPAAAA|62122|17|15|22|PM|second|evening|dinner| +62123|AAAAAAAAMKCPAAAA|62123|17|15|23|PM|second|evening|dinner| +62124|AAAAAAAANKCPAAAA|62124|17|15|24|PM|second|evening|dinner| +62125|AAAAAAAAOKCPAAAA|62125|17|15|25|PM|second|evening|dinner| +62126|AAAAAAAAPKCPAAAA|62126|17|15|26|PM|second|evening|dinner| +62127|AAAAAAAAALCPAAAA|62127|17|15|27|PM|second|evening|dinner| +62128|AAAAAAAABLCPAAAA|62128|17|15|28|PM|second|evening|dinner| +62129|AAAAAAAACLCPAAAA|62129|17|15|29|PM|second|evening|dinner| +62130|AAAAAAAADLCPAAAA|62130|17|15|30|PM|second|evening|dinner| +62131|AAAAAAAAELCPAAAA|62131|17|15|31|PM|second|evening|dinner| +62132|AAAAAAAAFLCPAAAA|62132|17|15|32|PM|second|evening|dinner| +62133|AAAAAAAAGLCPAAAA|62133|17|15|33|PM|second|evening|dinner| +62134|AAAAAAAAHLCPAAAA|62134|17|15|34|PM|second|evening|dinner| +62135|AAAAAAAAILCPAAAA|62135|17|15|35|PM|second|evening|dinner| +62136|AAAAAAAAJLCPAAAA|62136|17|15|36|PM|second|evening|dinner| +62137|AAAAAAAAKLCPAAAA|62137|17|15|37|PM|second|evening|dinner| +62138|AAAAAAAALLCPAAAA|62138|17|15|38|PM|second|evening|dinner| +62139|AAAAAAAAMLCPAAAA|62139|17|15|39|PM|second|evening|dinner| +62140|AAAAAAAANLCPAAAA|62140|17|15|40|PM|second|evening|dinner| +62141|AAAAAAAAOLCPAAAA|62141|17|15|41|PM|second|evening|dinner| +62142|AAAAAAAAPLCPAAAA|62142|17|15|42|PM|second|evening|dinner| +62143|AAAAAAAAAMCPAAAA|62143|17|15|43|PM|second|evening|dinner| +62144|AAAAAAAABMCPAAAA|62144|17|15|44|PM|second|evening|dinner| +62145|AAAAAAAACMCPAAAA|62145|17|15|45|PM|second|evening|dinner| +62146|AAAAAAAADMCPAAAA|62146|17|15|46|PM|second|evening|dinner| +62147|AAAAAAAAEMCPAAAA|62147|17|15|47|PM|second|evening|dinner| +62148|AAAAAAAAFMCPAAAA|62148|17|15|48|PM|second|evening|dinner| +62149|AAAAAAAAGMCPAAAA|62149|17|15|49|PM|second|evening|dinner| +62150|AAAAAAAAHMCPAAAA|62150|17|15|50|PM|second|evening|dinner| +62151|AAAAAAAAIMCPAAAA|62151|17|15|51|PM|second|evening|dinner| +62152|AAAAAAAAJMCPAAAA|62152|17|15|52|PM|second|evening|dinner| +62153|AAAAAAAAKMCPAAAA|62153|17|15|53|PM|second|evening|dinner| +62154|AAAAAAAALMCPAAAA|62154|17|15|54|PM|second|evening|dinner| +62155|AAAAAAAAMMCPAAAA|62155|17|15|55|PM|second|evening|dinner| +62156|AAAAAAAANMCPAAAA|62156|17|15|56|PM|second|evening|dinner| +62157|AAAAAAAAOMCPAAAA|62157|17|15|57|PM|second|evening|dinner| +62158|AAAAAAAAPMCPAAAA|62158|17|15|58|PM|second|evening|dinner| +62159|AAAAAAAAANCPAAAA|62159|17|15|59|PM|second|evening|dinner| +62160|AAAAAAAABNCPAAAA|62160|17|16|0|PM|second|evening|dinner| +62161|AAAAAAAACNCPAAAA|62161|17|16|1|PM|second|evening|dinner| +62162|AAAAAAAADNCPAAAA|62162|17|16|2|PM|second|evening|dinner| +62163|AAAAAAAAENCPAAAA|62163|17|16|3|PM|second|evening|dinner| +62164|AAAAAAAAFNCPAAAA|62164|17|16|4|PM|second|evening|dinner| +62165|AAAAAAAAGNCPAAAA|62165|17|16|5|PM|second|evening|dinner| +62166|AAAAAAAAHNCPAAAA|62166|17|16|6|PM|second|evening|dinner| +62167|AAAAAAAAINCPAAAA|62167|17|16|7|PM|second|evening|dinner| +62168|AAAAAAAAJNCPAAAA|62168|17|16|8|PM|second|evening|dinner| +62169|AAAAAAAAKNCPAAAA|62169|17|16|9|PM|second|evening|dinner| +62170|AAAAAAAALNCPAAAA|62170|17|16|10|PM|second|evening|dinner| +62171|AAAAAAAAMNCPAAAA|62171|17|16|11|PM|second|evening|dinner| +62172|AAAAAAAANNCPAAAA|62172|17|16|12|PM|second|evening|dinner| +62173|AAAAAAAAONCPAAAA|62173|17|16|13|PM|second|evening|dinner| +62174|AAAAAAAAPNCPAAAA|62174|17|16|14|PM|second|evening|dinner| +62175|AAAAAAAAAOCPAAAA|62175|17|16|15|PM|second|evening|dinner| +62176|AAAAAAAABOCPAAAA|62176|17|16|16|PM|second|evening|dinner| +62177|AAAAAAAACOCPAAAA|62177|17|16|17|PM|second|evening|dinner| +62178|AAAAAAAADOCPAAAA|62178|17|16|18|PM|second|evening|dinner| +62179|AAAAAAAAEOCPAAAA|62179|17|16|19|PM|second|evening|dinner| +62180|AAAAAAAAFOCPAAAA|62180|17|16|20|PM|second|evening|dinner| +62181|AAAAAAAAGOCPAAAA|62181|17|16|21|PM|second|evening|dinner| +62182|AAAAAAAAHOCPAAAA|62182|17|16|22|PM|second|evening|dinner| +62183|AAAAAAAAIOCPAAAA|62183|17|16|23|PM|second|evening|dinner| +62184|AAAAAAAAJOCPAAAA|62184|17|16|24|PM|second|evening|dinner| +62185|AAAAAAAAKOCPAAAA|62185|17|16|25|PM|second|evening|dinner| +62186|AAAAAAAALOCPAAAA|62186|17|16|26|PM|second|evening|dinner| +62187|AAAAAAAAMOCPAAAA|62187|17|16|27|PM|second|evening|dinner| +62188|AAAAAAAANOCPAAAA|62188|17|16|28|PM|second|evening|dinner| +62189|AAAAAAAAOOCPAAAA|62189|17|16|29|PM|second|evening|dinner| +62190|AAAAAAAAPOCPAAAA|62190|17|16|30|PM|second|evening|dinner| +62191|AAAAAAAAAPCPAAAA|62191|17|16|31|PM|second|evening|dinner| +62192|AAAAAAAABPCPAAAA|62192|17|16|32|PM|second|evening|dinner| +62193|AAAAAAAACPCPAAAA|62193|17|16|33|PM|second|evening|dinner| +62194|AAAAAAAADPCPAAAA|62194|17|16|34|PM|second|evening|dinner| +62195|AAAAAAAAEPCPAAAA|62195|17|16|35|PM|second|evening|dinner| +62196|AAAAAAAAFPCPAAAA|62196|17|16|36|PM|second|evening|dinner| +62197|AAAAAAAAGPCPAAAA|62197|17|16|37|PM|second|evening|dinner| +62198|AAAAAAAAHPCPAAAA|62198|17|16|38|PM|second|evening|dinner| +62199|AAAAAAAAIPCPAAAA|62199|17|16|39|PM|second|evening|dinner| +62200|AAAAAAAAJPCPAAAA|62200|17|16|40|PM|second|evening|dinner| +62201|AAAAAAAAKPCPAAAA|62201|17|16|41|PM|second|evening|dinner| +62202|AAAAAAAALPCPAAAA|62202|17|16|42|PM|second|evening|dinner| +62203|AAAAAAAAMPCPAAAA|62203|17|16|43|PM|second|evening|dinner| +62204|AAAAAAAANPCPAAAA|62204|17|16|44|PM|second|evening|dinner| +62205|AAAAAAAAOPCPAAAA|62205|17|16|45|PM|second|evening|dinner| +62206|AAAAAAAAPPCPAAAA|62206|17|16|46|PM|second|evening|dinner| +62207|AAAAAAAAAADPAAAA|62207|17|16|47|PM|second|evening|dinner| +62208|AAAAAAAABADPAAAA|62208|17|16|48|PM|second|evening|dinner| +62209|AAAAAAAACADPAAAA|62209|17|16|49|PM|second|evening|dinner| +62210|AAAAAAAADADPAAAA|62210|17|16|50|PM|second|evening|dinner| +62211|AAAAAAAAEADPAAAA|62211|17|16|51|PM|second|evening|dinner| +62212|AAAAAAAAFADPAAAA|62212|17|16|52|PM|second|evening|dinner| +62213|AAAAAAAAGADPAAAA|62213|17|16|53|PM|second|evening|dinner| +62214|AAAAAAAAHADPAAAA|62214|17|16|54|PM|second|evening|dinner| +62215|AAAAAAAAIADPAAAA|62215|17|16|55|PM|second|evening|dinner| +62216|AAAAAAAAJADPAAAA|62216|17|16|56|PM|second|evening|dinner| +62217|AAAAAAAAKADPAAAA|62217|17|16|57|PM|second|evening|dinner| +62218|AAAAAAAALADPAAAA|62218|17|16|58|PM|second|evening|dinner| +62219|AAAAAAAAMADPAAAA|62219|17|16|59|PM|second|evening|dinner| +62220|AAAAAAAANADPAAAA|62220|17|17|0|PM|second|evening|dinner| +62221|AAAAAAAAOADPAAAA|62221|17|17|1|PM|second|evening|dinner| +62222|AAAAAAAAPADPAAAA|62222|17|17|2|PM|second|evening|dinner| +62223|AAAAAAAAABDPAAAA|62223|17|17|3|PM|second|evening|dinner| +62224|AAAAAAAABBDPAAAA|62224|17|17|4|PM|second|evening|dinner| +62225|AAAAAAAACBDPAAAA|62225|17|17|5|PM|second|evening|dinner| +62226|AAAAAAAADBDPAAAA|62226|17|17|6|PM|second|evening|dinner| +62227|AAAAAAAAEBDPAAAA|62227|17|17|7|PM|second|evening|dinner| +62228|AAAAAAAAFBDPAAAA|62228|17|17|8|PM|second|evening|dinner| +62229|AAAAAAAAGBDPAAAA|62229|17|17|9|PM|second|evening|dinner| +62230|AAAAAAAAHBDPAAAA|62230|17|17|10|PM|second|evening|dinner| +62231|AAAAAAAAIBDPAAAA|62231|17|17|11|PM|second|evening|dinner| +62232|AAAAAAAAJBDPAAAA|62232|17|17|12|PM|second|evening|dinner| +62233|AAAAAAAAKBDPAAAA|62233|17|17|13|PM|second|evening|dinner| +62234|AAAAAAAALBDPAAAA|62234|17|17|14|PM|second|evening|dinner| +62235|AAAAAAAAMBDPAAAA|62235|17|17|15|PM|second|evening|dinner| +62236|AAAAAAAANBDPAAAA|62236|17|17|16|PM|second|evening|dinner| +62237|AAAAAAAAOBDPAAAA|62237|17|17|17|PM|second|evening|dinner| +62238|AAAAAAAAPBDPAAAA|62238|17|17|18|PM|second|evening|dinner| +62239|AAAAAAAAACDPAAAA|62239|17|17|19|PM|second|evening|dinner| +62240|AAAAAAAABCDPAAAA|62240|17|17|20|PM|second|evening|dinner| +62241|AAAAAAAACCDPAAAA|62241|17|17|21|PM|second|evening|dinner| +62242|AAAAAAAADCDPAAAA|62242|17|17|22|PM|second|evening|dinner| +62243|AAAAAAAAECDPAAAA|62243|17|17|23|PM|second|evening|dinner| +62244|AAAAAAAAFCDPAAAA|62244|17|17|24|PM|second|evening|dinner| +62245|AAAAAAAAGCDPAAAA|62245|17|17|25|PM|second|evening|dinner| +62246|AAAAAAAAHCDPAAAA|62246|17|17|26|PM|second|evening|dinner| +62247|AAAAAAAAICDPAAAA|62247|17|17|27|PM|second|evening|dinner| +62248|AAAAAAAAJCDPAAAA|62248|17|17|28|PM|second|evening|dinner| +62249|AAAAAAAAKCDPAAAA|62249|17|17|29|PM|second|evening|dinner| +62250|AAAAAAAALCDPAAAA|62250|17|17|30|PM|second|evening|dinner| +62251|AAAAAAAAMCDPAAAA|62251|17|17|31|PM|second|evening|dinner| +62252|AAAAAAAANCDPAAAA|62252|17|17|32|PM|second|evening|dinner| +62253|AAAAAAAAOCDPAAAA|62253|17|17|33|PM|second|evening|dinner| +62254|AAAAAAAAPCDPAAAA|62254|17|17|34|PM|second|evening|dinner| +62255|AAAAAAAAADDPAAAA|62255|17|17|35|PM|second|evening|dinner| +62256|AAAAAAAABDDPAAAA|62256|17|17|36|PM|second|evening|dinner| +62257|AAAAAAAACDDPAAAA|62257|17|17|37|PM|second|evening|dinner| +62258|AAAAAAAADDDPAAAA|62258|17|17|38|PM|second|evening|dinner| +62259|AAAAAAAAEDDPAAAA|62259|17|17|39|PM|second|evening|dinner| +62260|AAAAAAAAFDDPAAAA|62260|17|17|40|PM|second|evening|dinner| +62261|AAAAAAAAGDDPAAAA|62261|17|17|41|PM|second|evening|dinner| +62262|AAAAAAAAHDDPAAAA|62262|17|17|42|PM|second|evening|dinner| +62263|AAAAAAAAIDDPAAAA|62263|17|17|43|PM|second|evening|dinner| +62264|AAAAAAAAJDDPAAAA|62264|17|17|44|PM|second|evening|dinner| +62265|AAAAAAAAKDDPAAAA|62265|17|17|45|PM|second|evening|dinner| +62266|AAAAAAAALDDPAAAA|62266|17|17|46|PM|second|evening|dinner| +62267|AAAAAAAAMDDPAAAA|62267|17|17|47|PM|second|evening|dinner| +62268|AAAAAAAANDDPAAAA|62268|17|17|48|PM|second|evening|dinner| +62269|AAAAAAAAODDPAAAA|62269|17|17|49|PM|second|evening|dinner| +62270|AAAAAAAAPDDPAAAA|62270|17|17|50|PM|second|evening|dinner| +62271|AAAAAAAAAEDPAAAA|62271|17|17|51|PM|second|evening|dinner| +62272|AAAAAAAABEDPAAAA|62272|17|17|52|PM|second|evening|dinner| +62273|AAAAAAAACEDPAAAA|62273|17|17|53|PM|second|evening|dinner| +62274|AAAAAAAADEDPAAAA|62274|17|17|54|PM|second|evening|dinner| +62275|AAAAAAAAEEDPAAAA|62275|17|17|55|PM|second|evening|dinner| +62276|AAAAAAAAFEDPAAAA|62276|17|17|56|PM|second|evening|dinner| +62277|AAAAAAAAGEDPAAAA|62277|17|17|57|PM|second|evening|dinner| +62278|AAAAAAAAHEDPAAAA|62278|17|17|58|PM|second|evening|dinner| +62279|AAAAAAAAIEDPAAAA|62279|17|17|59|PM|second|evening|dinner| +62280|AAAAAAAAJEDPAAAA|62280|17|18|0|PM|second|evening|dinner| +62281|AAAAAAAAKEDPAAAA|62281|17|18|1|PM|second|evening|dinner| +62282|AAAAAAAALEDPAAAA|62282|17|18|2|PM|second|evening|dinner| +62283|AAAAAAAAMEDPAAAA|62283|17|18|3|PM|second|evening|dinner| +62284|AAAAAAAANEDPAAAA|62284|17|18|4|PM|second|evening|dinner| +62285|AAAAAAAAOEDPAAAA|62285|17|18|5|PM|second|evening|dinner| +62286|AAAAAAAAPEDPAAAA|62286|17|18|6|PM|second|evening|dinner| +62287|AAAAAAAAAFDPAAAA|62287|17|18|7|PM|second|evening|dinner| +62288|AAAAAAAABFDPAAAA|62288|17|18|8|PM|second|evening|dinner| +62289|AAAAAAAACFDPAAAA|62289|17|18|9|PM|second|evening|dinner| +62290|AAAAAAAADFDPAAAA|62290|17|18|10|PM|second|evening|dinner| +62291|AAAAAAAAEFDPAAAA|62291|17|18|11|PM|second|evening|dinner| +62292|AAAAAAAAFFDPAAAA|62292|17|18|12|PM|second|evening|dinner| +62293|AAAAAAAAGFDPAAAA|62293|17|18|13|PM|second|evening|dinner| +62294|AAAAAAAAHFDPAAAA|62294|17|18|14|PM|second|evening|dinner| +62295|AAAAAAAAIFDPAAAA|62295|17|18|15|PM|second|evening|dinner| +62296|AAAAAAAAJFDPAAAA|62296|17|18|16|PM|second|evening|dinner| +62297|AAAAAAAAKFDPAAAA|62297|17|18|17|PM|second|evening|dinner| +62298|AAAAAAAALFDPAAAA|62298|17|18|18|PM|second|evening|dinner| +62299|AAAAAAAAMFDPAAAA|62299|17|18|19|PM|second|evening|dinner| +62300|AAAAAAAANFDPAAAA|62300|17|18|20|PM|second|evening|dinner| +62301|AAAAAAAAOFDPAAAA|62301|17|18|21|PM|second|evening|dinner| +62302|AAAAAAAAPFDPAAAA|62302|17|18|22|PM|second|evening|dinner| +62303|AAAAAAAAAGDPAAAA|62303|17|18|23|PM|second|evening|dinner| +62304|AAAAAAAABGDPAAAA|62304|17|18|24|PM|second|evening|dinner| +62305|AAAAAAAACGDPAAAA|62305|17|18|25|PM|second|evening|dinner| +62306|AAAAAAAADGDPAAAA|62306|17|18|26|PM|second|evening|dinner| +62307|AAAAAAAAEGDPAAAA|62307|17|18|27|PM|second|evening|dinner| +62308|AAAAAAAAFGDPAAAA|62308|17|18|28|PM|second|evening|dinner| +62309|AAAAAAAAGGDPAAAA|62309|17|18|29|PM|second|evening|dinner| +62310|AAAAAAAAHGDPAAAA|62310|17|18|30|PM|second|evening|dinner| +62311|AAAAAAAAIGDPAAAA|62311|17|18|31|PM|second|evening|dinner| +62312|AAAAAAAAJGDPAAAA|62312|17|18|32|PM|second|evening|dinner| +62313|AAAAAAAAKGDPAAAA|62313|17|18|33|PM|second|evening|dinner| +62314|AAAAAAAALGDPAAAA|62314|17|18|34|PM|second|evening|dinner| +62315|AAAAAAAAMGDPAAAA|62315|17|18|35|PM|second|evening|dinner| +62316|AAAAAAAANGDPAAAA|62316|17|18|36|PM|second|evening|dinner| +62317|AAAAAAAAOGDPAAAA|62317|17|18|37|PM|second|evening|dinner| +62318|AAAAAAAAPGDPAAAA|62318|17|18|38|PM|second|evening|dinner| +62319|AAAAAAAAAHDPAAAA|62319|17|18|39|PM|second|evening|dinner| +62320|AAAAAAAABHDPAAAA|62320|17|18|40|PM|second|evening|dinner| +62321|AAAAAAAACHDPAAAA|62321|17|18|41|PM|second|evening|dinner| +62322|AAAAAAAADHDPAAAA|62322|17|18|42|PM|second|evening|dinner| +62323|AAAAAAAAEHDPAAAA|62323|17|18|43|PM|second|evening|dinner| +62324|AAAAAAAAFHDPAAAA|62324|17|18|44|PM|second|evening|dinner| +62325|AAAAAAAAGHDPAAAA|62325|17|18|45|PM|second|evening|dinner| +62326|AAAAAAAAHHDPAAAA|62326|17|18|46|PM|second|evening|dinner| +62327|AAAAAAAAIHDPAAAA|62327|17|18|47|PM|second|evening|dinner| +62328|AAAAAAAAJHDPAAAA|62328|17|18|48|PM|second|evening|dinner| +62329|AAAAAAAAKHDPAAAA|62329|17|18|49|PM|second|evening|dinner| +62330|AAAAAAAALHDPAAAA|62330|17|18|50|PM|second|evening|dinner| +62331|AAAAAAAAMHDPAAAA|62331|17|18|51|PM|second|evening|dinner| +62332|AAAAAAAANHDPAAAA|62332|17|18|52|PM|second|evening|dinner| +62333|AAAAAAAAOHDPAAAA|62333|17|18|53|PM|second|evening|dinner| +62334|AAAAAAAAPHDPAAAA|62334|17|18|54|PM|second|evening|dinner| +62335|AAAAAAAAAIDPAAAA|62335|17|18|55|PM|second|evening|dinner| +62336|AAAAAAAABIDPAAAA|62336|17|18|56|PM|second|evening|dinner| +62337|AAAAAAAACIDPAAAA|62337|17|18|57|PM|second|evening|dinner| +62338|AAAAAAAADIDPAAAA|62338|17|18|58|PM|second|evening|dinner| +62339|AAAAAAAAEIDPAAAA|62339|17|18|59|PM|second|evening|dinner| +62340|AAAAAAAAFIDPAAAA|62340|17|19|0|PM|second|evening|dinner| +62341|AAAAAAAAGIDPAAAA|62341|17|19|1|PM|second|evening|dinner| +62342|AAAAAAAAHIDPAAAA|62342|17|19|2|PM|second|evening|dinner| +62343|AAAAAAAAIIDPAAAA|62343|17|19|3|PM|second|evening|dinner| +62344|AAAAAAAAJIDPAAAA|62344|17|19|4|PM|second|evening|dinner| +62345|AAAAAAAAKIDPAAAA|62345|17|19|5|PM|second|evening|dinner| +62346|AAAAAAAALIDPAAAA|62346|17|19|6|PM|second|evening|dinner| +62347|AAAAAAAAMIDPAAAA|62347|17|19|7|PM|second|evening|dinner| +62348|AAAAAAAANIDPAAAA|62348|17|19|8|PM|second|evening|dinner| +62349|AAAAAAAAOIDPAAAA|62349|17|19|9|PM|second|evening|dinner| +62350|AAAAAAAAPIDPAAAA|62350|17|19|10|PM|second|evening|dinner| +62351|AAAAAAAAAJDPAAAA|62351|17|19|11|PM|second|evening|dinner| +62352|AAAAAAAABJDPAAAA|62352|17|19|12|PM|second|evening|dinner| +62353|AAAAAAAACJDPAAAA|62353|17|19|13|PM|second|evening|dinner| +62354|AAAAAAAADJDPAAAA|62354|17|19|14|PM|second|evening|dinner| +62355|AAAAAAAAEJDPAAAA|62355|17|19|15|PM|second|evening|dinner| +62356|AAAAAAAAFJDPAAAA|62356|17|19|16|PM|second|evening|dinner| +62357|AAAAAAAAGJDPAAAA|62357|17|19|17|PM|second|evening|dinner| +62358|AAAAAAAAHJDPAAAA|62358|17|19|18|PM|second|evening|dinner| +62359|AAAAAAAAIJDPAAAA|62359|17|19|19|PM|second|evening|dinner| +62360|AAAAAAAAJJDPAAAA|62360|17|19|20|PM|second|evening|dinner| +62361|AAAAAAAAKJDPAAAA|62361|17|19|21|PM|second|evening|dinner| +62362|AAAAAAAALJDPAAAA|62362|17|19|22|PM|second|evening|dinner| +62363|AAAAAAAAMJDPAAAA|62363|17|19|23|PM|second|evening|dinner| +62364|AAAAAAAANJDPAAAA|62364|17|19|24|PM|second|evening|dinner| +62365|AAAAAAAAOJDPAAAA|62365|17|19|25|PM|second|evening|dinner| +62366|AAAAAAAAPJDPAAAA|62366|17|19|26|PM|second|evening|dinner| +62367|AAAAAAAAAKDPAAAA|62367|17|19|27|PM|second|evening|dinner| +62368|AAAAAAAABKDPAAAA|62368|17|19|28|PM|second|evening|dinner| +62369|AAAAAAAACKDPAAAA|62369|17|19|29|PM|second|evening|dinner| +62370|AAAAAAAADKDPAAAA|62370|17|19|30|PM|second|evening|dinner| +62371|AAAAAAAAEKDPAAAA|62371|17|19|31|PM|second|evening|dinner| +62372|AAAAAAAAFKDPAAAA|62372|17|19|32|PM|second|evening|dinner| +62373|AAAAAAAAGKDPAAAA|62373|17|19|33|PM|second|evening|dinner| +62374|AAAAAAAAHKDPAAAA|62374|17|19|34|PM|second|evening|dinner| +62375|AAAAAAAAIKDPAAAA|62375|17|19|35|PM|second|evening|dinner| +62376|AAAAAAAAJKDPAAAA|62376|17|19|36|PM|second|evening|dinner| +62377|AAAAAAAAKKDPAAAA|62377|17|19|37|PM|second|evening|dinner| +62378|AAAAAAAALKDPAAAA|62378|17|19|38|PM|second|evening|dinner| +62379|AAAAAAAAMKDPAAAA|62379|17|19|39|PM|second|evening|dinner| +62380|AAAAAAAANKDPAAAA|62380|17|19|40|PM|second|evening|dinner| +62381|AAAAAAAAOKDPAAAA|62381|17|19|41|PM|second|evening|dinner| +62382|AAAAAAAAPKDPAAAA|62382|17|19|42|PM|second|evening|dinner| +62383|AAAAAAAAALDPAAAA|62383|17|19|43|PM|second|evening|dinner| +62384|AAAAAAAABLDPAAAA|62384|17|19|44|PM|second|evening|dinner| +62385|AAAAAAAACLDPAAAA|62385|17|19|45|PM|second|evening|dinner| +62386|AAAAAAAADLDPAAAA|62386|17|19|46|PM|second|evening|dinner| +62387|AAAAAAAAELDPAAAA|62387|17|19|47|PM|second|evening|dinner| +62388|AAAAAAAAFLDPAAAA|62388|17|19|48|PM|second|evening|dinner| +62389|AAAAAAAAGLDPAAAA|62389|17|19|49|PM|second|evening|dinner| +62390|AAAAAAAAHLDPAAAA|62390|17|19|50|PM|second|evening|dinner| +62391|AAAAAAAAILDPAAAA|62391|17|19|51|PM|second|evening|dinner| +62392|AAAAAAAAJLDPAAAA|62392|17|19|52|PM|second|evening|dinner| +62393|AAAAAAAAKLDPAAAA|62393|17|19|53|PM|second|evening|dinner| +62394|AAAAAAAALLDPAAAA|62394|17|19|54|PM|second|evening|dinner| +62395|AAAAAAAAMLDPAAAA|62395|17|19|55|PM|second|evening|dinner| +62396|AAAAAAAANLDPAAAA|62396|17|19|56|PM|second|evening|dinner| +62397|AAAAAAAAOLDPAAAA|62397|17|19|57|PM|second|evening|dinner| +62398|AAAAAAAAPLDPAAAA|62398|17|19|58|PM|second|evening|dinner| +62399|AAAAAAAAAMDPAAAA|62399|17|19|59|PM|second|evening|dinner| +62400|AAAAAAAABMDPAAAA|62400|17|20|0|PM|second|evening|dinner| +62401|AAAAAAAACMDPAAAA|62401|17|20|1|PM|second|evening|dinner| +62402|AAAAAAAADMDPAAAA|62402|17|20|2|PM|second|evening|dinner| +62403|AAAAAAAAEMDPAAAA|62403|17|20|3|PM|second|evening|dinner| +62404|AAAAAAAAFMDPAAAA|62404|17|20|4|PM|second|evening|dinner| +62405|AAAAAAAAGMDPAAAA|62405|17|20|5|PM|second|evening|dinner| +62406|AAAAAAAAHMDPAAAA|62406|17|20|6|PM|second|evening|dinner| +62407|AAAAAAAAIMDPAAAA|62407|17|20|7|PM|second|evening|dinner| +62408|AAAAAAAAJMDPAAAA|62408|17|20|8|PM|second|evening|dinner| +62409|AAAAAAAAKMDPAAAA|62409|17|20|9|PM|second|evening|dinner| +62410|AAAAAAAALMDPAAAA|62410|17|20|10|PM|second|evening|dinner| +62411|AAAAAAAAMMDPAAAA|62411|17|20|11|PM|second|evening|dinner| +62412|AAAAAAAANMDPAAAA|62412|17|20|12|PM|second|evening|dinner| +62413|AAAAAAAAOMDPAAAA|62413|17|20|13|PM|second|evening|dinner| +62414|AAAAAAAAPMDPAAAA|62414|17|20|14|PM|second|evening|dinner| +62415|AAAAAAAAANDPAAAA|62415|17|20|15|PM|second|evening|dinner| +62416|AAAAAAAABNDPAAAA|62416|17|20|16|PM|second|evening|dinner| +62417|AAAAAAAACNDPAAAA|62417|17|20|17|PM|second|evening|dinner| +62418|AAAAAAAADNDPAAAA|62418|17|20|18|PM|second|evening|dinner| +62419|AAAAAAAAENDPAAAA|62419|17|20|19|PM|second|evening|dinner| +62420|AAAAAAAAFNDPAAAA|62420|17|20|20|PM|second|evening|dinner| +62421|AAAAAAAAGNDPAAAA|62421|17|20|21|PM|second|evening|dinner| +62422|AAAAAAAAHNDPAAAA|62422|17|20|22|PM|second|evening|dinner| +62423|AAAAAAAAINDPAAAA|62423|17|20|23|PM|second|evening|dinner| +62424|AAAAAAAAJNDPAAAA|62424|17|20|24|PM|second|evening|dinner| +62425|AAAAAAAAKNDPAAAA|62425|17|20|25|PM|second|evening|dinner| +62426|AAAAAAAALNDPAAAA|62426|17|20|26|PM|second|evening|dinner| +62427|AAAAAAAAMNDPAAAA|62427|17|20|27|PM|second|evening|dinner| +62428|AAAAAAAANNDPAAAA|62428|17|20|28|PM|second|evening|dinner| +62429|AAAAAAAAONDPAAAA|62429|17|20|29|PM|second|evening|dinner| +62430|AAAAAAAAPNDPAAAA|62430|17|20|30|PM|second|evening|dinner| +62431|AAAAAAAAAODPAAAA|62431|17|20|31|PM|second|evening|dinner| +62432|AAAAAAAABODPAAAA|62432|17|20|32|PM|second|evening|dinner| +62433|AAAAAAAACODPAAAA|62433|17|20|33|PM|second|evening|dinner| +62434|AAAAAAAADODPAAAA|62434|17|20|34|PM|second|evening|dinner| +62435|AAAAAAAAEODPAAAA|62435|17|20|35|PM|second|evening|dinner| +62436|AAAAAAAAFODPAAAA|62436|17|20|36|PM|second|evening|dinner| +62437|AAAAAAAAGODPAAAA|62437|17|20|37|PM|second|evening|dinner| +62438|AAAAAAAAHODPAAAA|62438|17|20|38|PM|second|evening|dinner| +62439|AAAAAAAAIODPAAAA|62439|17|20|39|PM|second|evening|dinner| +62440|AAAAAAAAJODPAAAA|62440|17|20|40|PM|second|evening|dinner| +62441|AAAAAAAAKODPAAAA|62441|17|20|41|PM|second|evening|dinner| +62442|AAAAAAAALODPAAAA|62442|17|20|42|PM|second|evening|dinner| +62443|AAAAAAAAMODPAAAA|62443|17|20|43|PM|second|evening|dinner| +62444|AAAAAAAANODPAAAA|62444|17|20|44|PM|second|evening|dinner| +62445|AAAAAAAAOODPAAAA|62445|17|20|45|PM|second|evening|dinner| +62446|AAAAAAAAPODPAAAA|62446|17|20|46|PM|second|evening|dinner| +62447|AAAAAAAAAPDPAAAA|62447|17|20|47|PM|second|evening|dinner| +62448|AAAAAAAABPDPAAAA|62448|17|20|48|PM|second|evening|dinner| +62449|AAAAAAAACPDPAAAA|62449|17|20|49|PM|second|evening|dinner| +62450|AAAAAAAADPDPAAAA|62450|17|20|50|PM|second|evening|dinner| +62451|AAAAAAAAEPDPAAAA|62451|17|20|51|PM|second|evening|dinner| +62452|AAAAAAAAFPDPAAAA|62452|17|20|52|PM|second|evening|dinner| +62453|AAAAAAAAGPDPAAAA|62453|17|20|53|PM|second|evening|dinner| +62454|AAAAAAAAHPDPAAAA|62454|17|20|54|PM|second|evening|dinner| +62455|AAAAAAAAIPDPAAAA|62455|17|20|55|PM|second|evening|dinner| +62456|AAAAAAAAJPDPAAAA|62456|17|20|56|PM|second|evening|dinner| +62457|AAAAAAAAKPDPAAAA|62457|17|20|57|PM|second|evening|dinner| +62458|AAAAAAAALPDPAAAA|62458|17|20|58|PM|second|evening|dinner| +62459|AAAAAAAAMPDPAAAA|62459|17|20|59|PM|second|evening|dinner| +62460|AAAAAAAANPDPAAAA|62460|17|21|0|PM|second|evening|dinner| +62461|AAAAAAAAOPDPAAAA|62461|17|21|1|PM|second|evening|dinner| +62462|AAAAAAAAPPDPAAAA|62462|17|21|2|PM|second|evening|dinner| +62463|AAAAAAAAAAEPAAAA|62463|17|21|3|PM|second|evening|dinner| +62464|AAAAAAAABAEPAAAA|62464|17|21|4|PM|second|evening|dinner| +62465|AAAAAAAACAEPAAAA|62465|17|21|5|PM|second|evening|dinner| +62466|AAAAAAAADAEPAAAA|62466|17|21|6|PM|second|evening|dinner| +62467|AAAAAAAAEAEPAAAA|62467|17|21|7|PM|second|evening|dinner| +62468|AAAAAAAAFAEPAAAA|62468|17|21|8|PM|second|evening|dinner| +62469|AAAAAAAAGAEPAAAA|62469|17|21|9|PM|second|evening|dinner| +62470|AAAAAAAAHAEPAAAA|62470|17|21|10|PM|second|evening|dinner| +62471|AAAAAAAAIAEPAAAA|62471|17|21|11|PM|second|evening|dinner| +62472|AAAAAAAAJAEPAAAA|62472|17|21|12|PM|second|evening|dinner| +62473|AAAAAAAAKAEPAAAA|62473|17|21|13|PM|second|evening|dinner| +62474|AAAAAAAALAEPAAAA|62474|17|21|14|PM|second|evening|dinner| +62475|AAAAAAAAMAEPAAAA|62475|17|21|15|PM|second|evening|dinner| +62476|AAAAAAAANAEPAAAA|62476|17|21|16|PM|second|evening|dinner| +62477|AAAAAAAAOAEPAAAA|62477|17|21|17|PM|second|evening|dinner| +62478|AAAAAAAAPAEPAAAA|62478|17|21|18|PM|second|evening|dinner| +62479|AAAAAAAAABEPAAAA|62479|17|21|19|PM|second|evening|dinner| +62480|AAAAAAAABBEPAAAA|62480|17|21|20|PM|second|evening|dinner| +62481|AAAAAAAACBEPAAAA|62481|17|21|21|PM|second|evening|dinner| +62482|AAAAAAAADBEPAAAA|62482|17|21|22|PM|second|evening|dinner| +62483|AAAAAAAAEBEPAAAA|62483|17|21|23|PM|second|evening|dinner| +62484|AAAAAAAAFBEPAAAA|62484|17|21|24|PM|second|evening|dinner| +62485|AAAAAAAAGBEPAAAA|62485|17|21|25|PM|second|evening|dinner| +62486|AAAAAAAAHBEPAAAA|62486|17|21|26|PM|second|evening|dinner| +62487|AAAAAAAAIBEPAAAA|62487|17|21|27|PM|second|evening|dinner| +62488|AAAAAAAAJBEPAAAA|62488|17|21|28|PM|second|evening|dinner| +62489|AAAAAAAAKBEPAAAA|62489|17|21|29|PM|second|evening|dinner| +62490|AAAAAAAALBEPAAAA|62490|17|21|30|PM|second|evening|dinner| +62491|AAAAAAAAMBEPAAAA|62491|17|21|31|PM|second|evening|dinner| +62492|AAAAAAAANBEPAAAA|62492|17|21|32|PM|second|evening|dinner| +62493|AAAAAAAAOBEPAAAA|62493|17|21|33|PM|second|evening|dinner| +62494|AAAAAAAAPBEPAAAA|62494|17|21|34|PM|second|evening|dinner| +62495|AAAAAAAAACEPAAAA|62495|17|21|35|PM|second|evening|dinner| +62496|AAAAAAAABCEPAAAA|62496|17|21|36|PM|second|evening|dinner| +62497|AAAAAAAACCEPAAAA|62497|17|21|37|PM|second|evening|dinner| +62498|AAAAAAAADCEPAAAA|62498|17|21|38|PM|second|evening|dinner| +62499|AAAAAAAAECEPAAAA|62499|17|21|39|PM|second|evening|dinner| +62500|AAAAAAAAFCEPAAAA|62500|17|21|40|PM|second|evening|dinner| +62501|AAAAAAAAGCEPAAAA|62501|17|21|41|PM|second|evening|dinner| +62502|AAAAAAAAHCEPAAAA|62502|17|21|42|PM|second|evening|dinner| +62503|AAAAAAAAICEPAAAA|62503|17|21|43|PM|second|evening|dinner| +62504|AAAAAAAAJCEPAAAA|62504|17|21|44|PM|second|evening|dinner| +62505|AAAAAAAAKCEPAAAA|62505|17|21|45|PM|second|evening|dinner| +62506|AAAAAAAALCEPAAAA|62506|17|21|46|PM|second|evening|dinner| +62507|AAAAAAAAMCEPAAAA|62507|17|21|47|PM|second|evening|dinner| +62508|AAAAAAAANCEPAAAA|62508|17|21|48|PM|second|evening|dinner| +62509|AAAAAAAAOCEPAAAA|62509|17|21|49|PM|second|evening|dinner| +62510|AAAAAAAAPCEPAAAA|62510|17|21|50|PM|second|evening|dinner| +62511|AAAAAAAAADEPAAAA|62511|17|21|51|PM|second|evening|dinner| +62512|AAAAAAAABDEPAAAA|62512|17|21|52|PM|second|evening|dinner| +62513|AAAAAAAACDEPAAAA|62513|17|21|53|PM|second|evening|dinner| +62514|AAAAAAAADDEPAAAA|62514|17|21|54|PM|second|evening|dinner| +62515|AAAAAAAAEDEPAAAA|62515|17|21|55|PM|second|evening|dinner| +62516|AAAAAAAAFDEPAAAA|62516|17|21|56|PM|second|evening|dinner| +62517|AAAAAAAAGDEPAAAA|62517|17|21|57|PM|second|evening|dinner| +62518|AAAAAAAAHDEPAAAA|62518|17|21|58|PM|second|evening|dinner| +62519|AAAAAAAAIDEPAAAA|62519|17|21|59|PM|second|evening|dinner| +62520|AAAAAAAAJDEPAAAA|62520|17|22|0|PM|second|evening|dinner| +62521|AAAAAAAAKDEPAAAA|62521|17|22|1|PM|second|evening|dinner| +62522|AAAAAAAALDEPAAAA|62522|17|22|2|PM|second|evening|dinner| +62523|AAAAAAAAMDEPAAAA|62523|17|22|3|PM|second|evening|dinner| +62524|AAAAAAAANDEPAAAA|62524|17|22|4|PM|second|evening|dinner| +62525|AAAAAAAAODEPAAAA|62525|17|22|5|PM|second|evening|dinner| +62526|AAAAAAAAPDEPAAAA|62526|17|22|6|PM|second|evening|dinner| +62527|AAAAAAAAAEEPAAAA|62527|17|22|7|PM|second|evening|dinner| +62528|AAAAAAAABEEPAAAA|62528|17|22|8|PM|second|evening|dinner| +62529|AAAAAAAACEEPAAAA|62529|17|22|9|PM|second|evening|dinner| +62530|AAAAAAAADEEPAAAA|62530|17|22|10|PM|second|evening|dinner| +62531|AAAAAAAAEEEPAAAA|62531|17|22|11|PM|second|evening|dinner| +62532|AAAAAAAAFEEPAAAA|62532|17|22|12|PM|second|evening|dinner| +62533|AAAAAAAAGEEPAAAA|62533|17|22|13|PM|second|evening|dinner| +62534|AAAAAAAAHEEPAAAA|62534|17|22|14|PM|second|evening|dinner| +62535|AAAAAAAAIEEPAAAA|62535|17|22|15|PM|second|evening|dinner| +62536|AAAAAAAAJEEPAAAA|62536|17|22|16|PM|second|evening|dinner| +62537|AAAAAAAAKEEPAAAA|62537|17|22|17|PM|second|evening|dinner| +62538|AAAAAAAALEEPAAAA|62538|17|22|18|PM|second|evening|dinner| +62539|AAAAAAAAMEEPAAAA|62539|17|22|19|PM|second|evening|dinner| +62540|AAAAAAAANEEPAAAA|62540|17|22|20|PM|second|evening|dinner| +62541|AAAAAAAAOEEPAAAA|62541|17|22|21|PM|second|evening|dinner| +62542|AAAAAAAAPEEPAAAA|62542|17|22|22|PM|second|evening|dinner| +62543|AAAAAAAAAFEPAAAA|62543|17|22|23|PM|second|evening|dinner| +62544|AAAAAAAABFEPAAAA|62544|17|22|24|PM|second|evening|dinner| +62545|AAAAAAAACFEPAAAA|62545|17|22|25|PM|second|evening|dinner| +62546|AAAAAAAADFEPAAAA|62546|17|22|26|PM|second|evening|dinner| +62547|AAAAAAAAEFEPAAAA|62547|17|22|27|PM|second|evening|dinner| +62548|AAAAAAAAFFEPAAAA|62548|17|22|28|PM|second|evening|dinner| +62549|AAAAAAAAGFEPAAAA|62549|17|22|29|PM|second|evening|dinner| +62550|AAAAAAAAHFEPAAAA|62550|17|22|30|PM|second|evening|dinner| +62551|AAAAAAAAIFEPAAAA|62551|17|22|31|PM|second|evening|dinner| +62552|AAAAAAAAJFEPAAAA|62552|17|22|32|PM|second|evening|dinner| +62553|AAAAAAAAKFEPAAAA|62553|17|22|33|PM|second|evening|dinner| +62554|AAAAAAAALFEPAAAA|62554|17|22|34|PM|second|evening|dinner| +62555|AAAAAAAAMFEPAAAA|62555|17|22|35|PM|second|evening|dinner| +62556|AAAAAAAANFEPAAAA|62556|17|22|36|PM|second|evening|dinner| +62557|AAAAAAAAOFEPAAAA|62557|17|22|37|PM|second|evening|dinner| +62558|AAAAAAAAPFEPAAAA|62558|17|22|38|PM|second|evening|dinner| +62559|AAAAAAAAAGEPAAAA|62559|17|22|39|PM|second|evening|dinner| +62560|AAAAAAAABGEPAAAA|62560|17|22|40|PM|second|evening|dinner| +62561|AAAAAAAACGEPAAAA|62561|17|22|41|PM|second|evening|dinner| +62562|AAAAAAAADGEPAAAA|62562|17|22|42|PM|second|evening|dinner| +62563|AAAAAAAAEGEPAAAA|62563|17|22|43|PM|second|evening|dinner| +62564|AAAAAAAAFGEPAAAA|62564|17|22|44|PM|second|evening|dinner| +62565|AAAAAAAAGGEPAAAA|62565|17|22|45|PM|second|evening|dinner| +62566|AAAAAAAAHGEPAAAA|62566|17|22|46|PM|second|evening|dinner| +62567|AAAAAAAAIGEPAAAA|62567|17|22|47|PM|second|evening|dinner| +62568|AAAAAAAAJGEPAAAA|62568|17|22|48|PM|second|evening|dinner| +62569|AAAAAAAAKGEPAAAA|62569|17|22|49|PM|second|evening|dinner| +62570|AAAAAAAALGEPAAAA|62570|17|22|50|PM|second|evening|dinner| +62571|AAAAAAAAMGEPAAAA|62571|17|22|51|PM|second|evening|dinner| +62572|AAAAAAAANGEPAAAA|62572|17|22|52|PM|second|evening|dinner| +62573|AAAAAAAAOGEPAAAA|62573|17|22|53|PM|second|evening|dinner| +62574|AAAAAAAAPGEPAAAA|62574|17|22|54|PM|second|evening|dinner| +62575|AAAAAAAAAHEPAAAA|62575|17|22|55|PM|second|evening|dinner| +62576|AAAAAAAABHEPAAAA|62576|17|22|56|PM|second|evening|dinner| +62577|AAAAAAAACHEPAAAA|62577|17|22|57|PM|second|evening|dinner| +62578|AAAAAAAADHEPAAAA|62578|17|22|58|PM|second|evening|dinner| +62579|AAAAAAAAEHEPAAAA|62579|17|22|59|PM|second|evening|dinner| +62580|AAAAAAAAFHEPAAAA|62580|17|23|0|PM|second|evening|dinner| +62581|AAAAAAAAGHEPAAAA|62581|17|23|1|PM|second|evening|dinner| +62582|AAAAAAAAHHEPAAAA|62582|17|23|2|PM|second|evening|dinner| +62583|AAAAAAAAIHEPAAAA|62583|17|23|3|PM|second|evening|dinner| +62584|AAAAAAAAJHEPAAAA|62584|17|23|4|PM|second|evening|dinner| +62585|AAAAAAAAKHEPAAAA|62585|17|23|5|PM|second|evening|dinner| +62586|AAAAAAAALHEPAAAA|62586|17|23|6|PM|second|evening|dinner| +62587|AAAAAAAAMHEPAAAA|62587|17|23|7|PM|second|evening|dinner| +62588|AAAAAAAANHEPAAAA|62588|17|23|8|PM|second|evening|dinner| +62589|AAAAAAAAOHEPAAAA|62589|17|23|9|PM|second|evening|dinner| +62590|AAAAAAAAPHEPAAAA|62590|17|23|10|PM|second|evening|dinner| +62591|AAAAAAAAAIEPAAAA|62591|17|23|11|PM|second|evening|dinner| +62592|AAAAAAAABIEPAAAA|62592|17|23|12|PM|second|evening|dinner| +62593|AAAAAAAACIEPAAAA|62593|17|23|13|PM|second|evening|dinner| +62594|AAAAAAAADIEPAAAA|62594|17|23|14|PM|second|evening|dinner| +62595|AAAAAAAAEIEPAAAA|62595|17|23|15|PM|second|evening|dinner| +62596|AAAAAAAAFIEPAAAA|62596|17|23|16|PM|second|evening|dinner| +62597|AAAAAAAAGIEPAAAA|62597|17|23|17|PM|second|evening|dinner| +62598|AAAAAAAAHIEPAAAA|62598|17|23|18|PM|second|evening|dinner| +62599|AAAAAAAAIIEPAAAA|62599|17|23|19|PM|second|evening|dinner| +62600|AAAAAAAAJIEPAAAA|62600|17|23|20|PM|second|evening|dinner| +62601|AAAAAAAAKIEPAAAA|62601|17|23|21|PM|second|evening|dinner| +62602|AAAAAAAALIEPAAAA|62602|17|23|22|PM|second|evening|dinner| +62603|AAAAAAAAMIEPAAAA|62603|17|23|23|PM|second|evening|dinner| +62604|AAAAAAAANIEPAAAA|62604|17|23|24|PM|second|evening|dinner| +62605|AAAAAAAAOIEPAAAA|62605|17|23|25|PM|second|evening|dinner| +62606|AAAAAAAAPIEPAAAA|62606|17|23|26|PM|second|evening|dinner| +62607|AAAAAAAAAJEPAAAA|62607|17|23|27|PM|second|evening|dinner| +62608|AAAAAAAABJEPAAAA|62608|17|23|28|PM|second|evening|dinner| +62609|AAAAAAAACJEPAAAA|62609|17|23|29|PM|second|evening|dinner| +62610|AAAAAAAADJEPAAAA|62610|17|23|30|PM|second|evening|dinner| +62611|AAAAAAAAEJEPAAAA|62611|17|23|31|PM|second|evening|dinner| +62612|AAAAAAAAFJEPAAAA|62612|17|23|32|PM|second|evening|dinner| +62613|AAAAAAAAGJEPAAAA|62613|17|23|33|PM|second|evening|dinner| +62614|AAAAAAAAHJEPAAAA|62614|17|23|34|PM|second|evening|dinner| +62615|AAAAAAAAIJEPAAAA|62615|17|23|35|PM|second|evening|dinner| +62616|AAAAAAAAJJEPAAAA|62616|17|23|36|PM|second|evening|dinner| +62617|AAAAAAAAKJEPAAAA|62617|17|23|37|PM|second|evening|dinner| +62618|AAAAAAAALJEPAAAA|62618|17|23|38|PM|second|evening|dinner| +62619|AAAAAAAAMJEPAAAA|62619|17|23|39|PM|second|evening|dinner| +62620|AAAAAAAANJEPAAAA|62620|17|23|40|PM|second|evening|dinner| +62621|AAAAAAAAOJEPAAAA|62621|17|23|41|PM|second|evening|dinner| +62622|AAAAAAAAPJEPAAAA|62622|17|23|42|PM|second|evening|dinner| +62623|AAAAAAAAAKEPAAAA|62623|17|23|43|PM|second|evening|dinner| +62624|AAAAAAAABKEPAAAA|62624|17|23|44|PM|second|evening|dinner| +62625|AAAAAAAACKEPAAAA|62625|17|23|45|PM|second|evening|dinner| +62626|AAAAAAAADKEPAAAA|62626|17|23|46|PM|second|evening|dinner| +62627|AAAAAAAAEKEPAAAA|62627|17|23|47|PM|second|evening|dinner| +62628|AAAAAAAAFKEPAAAA|62628|17|23|48|PM|second|evening|dinner| +62629|AAAAAAAAGKEPAAAA|62629|17|23|49|PM|second|evening|dinner| +62630|AAAAAAAAHKEPAAAA|62630|17|23|50|PM|second|evening|dinner| +62631|AAAAAAAAIKEPAAAA|62631|17|23|51|PM|second|evening|dinner| +62632|AAAAAAAAJKEPAAAA|62632|17|23|52|PM|second|evening|dinner| +62633|AAAAAAAAKKEPAAAA|62633|17|23|53|PM|second|evening|dinner| +62634|AAAAAAAALKEPAAAA|62634|17|23|54|PM|second|evening|dinner| +62635|AAAAAAAAMKEPAAAA|62635|17|23|55|PM|second|evening|dinner| +62636|AAAAAAAANKEPAAAA|62636|17|23|56|PM|second|evening|dinner| +62637|AAAAAAAAOKEPAAAA|62637|17|23|57|PM|second|evening|dinner| +62638|AAAAAAAAPKEPAAAA|62638|17|23|58|PM|second|evening|dinner| +62639|AAAAAAAAALEPAAAA|62639|17|23|59|PM|second|evening|dinner| +62640|AAAAAAAABLEPAAAA|62640|17|24|0|PM|second|evening|dinner| +62641|AAAAAAAACLEPAAAA|62641|17|24|1|PM|second|evening|dinner| +62642|AAAAAAAADLEPAAAA|62642|17|24|2|PM|second|evening|dinner| +62643|AAAAAAAAELEPAAAA|62643|17|24|3|PM|second|evening|dinner| +62644|AAAAAAAAFLEPAAAA|62644|17|24|4|PM|second|evening|dinner| +62645|AAAAAAAAGLEPAAAA|62645|17|24|5|PM|second|evening|dinner| +62646|AAAAAAAAHLEPAAAA|62646|17|24|6|PM|second|evening|dinner| +62647|AAAAAAAAILEPAAAA|62647|17|24|7|PM|second|evening|dinner| +62648|AAAAAAAAJLEPAAAA|62648|17|24|8|PM|second|evening|dinner| +62649|AAAAAAAAKLEPAAAA|62649|17|24|9|PM|second|evening|dinner| +62650|AAAAAAAALLEPAAAA|62650|17|24|10|PM|second|evening|dinner| +62651|AAAAAAAAMLEPAAAA|62651|17|24|11|PM|second|evening|dinner| +62652|AAAAAAAANLEPAAAA|62652|17|24|12|PM|second|evening|dinner| +62653|AAAAAAAAOLEPAAAA|62653|17|24|13|PM|second|evening|dinner| +62654|AAAAAAAAPLEPAAAA|62654|17|24|14|PM|second|evening|dinner| +62655|AAAAAAAAAMEPAAAA|62655|17|24|15|PM|second|evening|dinner| +62656|AAAAAAAABMEPAAAA|62656|17|24|16|PM|second|evening|dinner| +62657|AAAAAAAACMEPAAAA|62657|17|24|17|PM|second|evening|dinner| +62658|AAAAAAAADMEPAAAA|62658|17|24|18|PM|second|evening|dinner| +62659|AAAAAAAAEMEPAAAA|62659|17|24|19|PM|second|evening|dinner| +62660|AAAAAAAAFMEPAAAA|62660|17|24|20|PM|second|evening|dinner| +62661|AAAAAAAAGMEPAAAA|62661|17|24|21|PM|second|evening|dinner| +62662|AAAAAAAAHMEPAAAA|62662|17|24|22|PM|second|evening|dinner| +62663|AAAAAAAAIMEPAAAA|62663|17|24|23|PM|second|evening|dinner| +62664|AAAAAAAAJMEPAAAA|62664|17|24|24|PM|second|evening|dinner| +62665|AAAAAAAAKMEPAAAA|62665|17|24|25|PM|second|evening|dinner| +62666|AAAAAAAALMEPAAAA|62666|17|24|26|PM|second|evening|dinner| +62667|AAAAAAAAMMEPAAAA|62667|17|24|27|PM|second|evening|dinner| +62668|AAAAAAAANMEPAAAA|62668|17|24|28|PM|second|evening|dinner| +62669|AAAAAAAAOMEPAAAA|62669|17|24|29|PM|second|evening|dinner| +62670|AAAAAAAAPMEPAAAA|62670|17|24|30|PM|second|evening|dinner| +62671|AAAAAAAAANEPAAAA|62671|17|24|31|PM|second|evening|dinner| +62672|AAAAAAAABNEPAAAA|62672|17|24|32|PM|second|evening|dinner| +62673|AAAAAAAACNEPAAAA|62673|17|24|33|PM|second|evening|dinner| +62674|AAAAAAAADNEPAAAA|62674|17|24|34|PM|second|evening|dinner| +62675|AAAAAAAAENEPAAAA|62675|17|24|35|PM|second|evening|dinner| +62676|AAAAAAAAFNEPAAAA|62676|17|24|36|PM|second|evening|dinner| +62677|AAAAAAAAGNEPAAAA|62677|17|24|37|PM|second|evening|dinner| +62678|AAAAAAAAHNEPAAAA|62678|17|24|38|PM|second|evening|dinner| +62679|AAAAAAAAINEPAAAA|62679|17|24|39|PM|second|evening|dinner| +62680|AAAAAAAAJNEPAAAA|62680|17|24|40|PM|second|evening|dinner| +62681|AAAAAAAAKNEPAAAA|62681|17|24|41|PM|second|evening|dinner| +62682|AAAAAAAALNEPAAAA|62682|17|24|42|PM|second|evening|dinner| +62683|AAAAAAAAMNEPAAAA|62683|17|24|43|PM|second|evening|dinner| +62684|AAAAAAAANNEPAAAA|62684|17|24|44|PM|second|evening|dinner| +62685|AAAAAAAAONEPAAAA|62685|17|24|45|PM|second|evening|dinner| +62686|AAAAAAAAPNEPAAAA|62686|17|24|46|PM|second|evening|dinner| +62687|AAAAAAAAAOEPAAAA|62687|17|24|47|PM|second|evening|dinner| +62688|AAAAAAAABOEPAAAA|62688|17|24|48|PM|second|evening|dinner| +62689|AAAAAAAACOEPAAAA|62689|17|24|49|PM|second|evening|dinner| +62690|AAAAAAAADOEPAAAA|62690|17|24|50|PM|second|evening|dinner| +62691|AAAAAAAAEOEPAAAA|62691|17|24|51|PM|second|evening|dinner| +62692|AAAAAAAAFOEPAAAA|62692|17|24|52|PM|second|evening|dinner| +62693|AAAAAAAAGOEPAAAA|62693|17|24|53|PM|second|evening|dinner| +62694|AAAAAAAAHOEPAAAA|62694|17|24|54|PM|second|evening|dinner| +62695|AAAAAAAAIOEPAAAA|62695|17|24|55|PM|second|evening|dinner| +62696|AAAAAAAAJOEPAAAA|62696|17|24|56|PM|second|evening|dinner| +62697|AAAAAAAAKOEPAAAA|62697|17|24|57|PM|second|evening|dinner| +62698|AAAAAAAALOEPAAAA|62698|17|24|58|PM|second|evening|dinner| +62699|AAAAAAAAMOEPAAAA|62699|17|24|59|PM|second|evening|dinner| +62700|AAAAAAAANOEPAAAA|62700|17|25|0|PM|second|evening|dinner| +62701|AAAAAAAAOOEPAAAA|62701|17|25|1|PM|second|evening|dinner| +62702|AAAAAAAAPOEPAAAA|62702|17|25|2|PM|second|evening|dinner| +62703|AAAAAAAAAPEPAAAA|62703|17|25|3|PM|second|evening|dinner| +62704|AAAAAAAABPEPAAAA|62704|17|25|4|PM|second|evening|dinner| +62705|AAAAAAAACPEPAAAA|62705|17|25|5|PM|second|evening|dinner| +62706|AAAAAAAADPEPAAAA|62706|17|25|6|PM|second|evening|dinner| +62707|AAAAAAAAEPEPAAAA|62707|17|25|7|PM|second|evening|dinner| +62708|AAAAAAAAFPEPAAAA|62708|17|25|8|PM|second|evening|dinner| +62709|AAAAAAAAGPEPAAAA|62709|17|25|9|PM|second|evening|dinner| +62710|AAAAAAAAHPEPAAAA|62710|17|25|10|PM|second|evening|dinner| +62711|AAAAAAAAIPEPAAAA|62711|17|25|11|PM|second|evening|dinner| +62712|AAAAAAAAJPEPAAAA|62712|17|25|12|PM|second|evening|dinner| +62713|AAAAAAAAKPEPAAAA|62713|17|25|13|PM|second|evening|dinner| +62714|AAAAAAAALPEPAAAA|62714|17|25|14|PM|second|evening|dinner| +62715|AAAAAAAAMPEPAAAA|62715|17|25|15|PM|second|evening|dinner| +62716|AAAAAAAANPEPAAAA|62716|17|25|16|PM|second|evening|dinner| +62717|AAAAAAAAOPEPAAAA|62717|17|25|17|PM|second|evening|dinner| +62718|AAAAAAAAPPEPAAAA|62718|17|25|18|PM|second|evening|dinner| +62719|AAAAAAAAAAFPAAAA|62719|17|25|19|PM|second|evening|dinner| +62720|AAAAAAAABAFPAAAA|62720|17|25|20|PM|second|evening|dinner| +62721|AAAAAAAACAFPAAAA|62721|17|25|21|PM|second|evening|dinner| +62722|AAAAAAAADAFPAAAA|62722|17|25|22|PM|second|evening|dinner| +62723|AAAAAAAAEAFPAAAA|62723|17|25|23|PM|second|evening|dinner| +62724|AAAAAAAAFAFPAAAA|62724|17|25|24|PM|second|evening|dinner| +62725|AAAAAAAAGAFPAAAA|62725|17|25|25|PM|second|evening|dinner| +62726|AAAAAAAAHAFPAAAA|62726|17|25|26|PM|second|evening|dinner| +62727|AAAAAAAAIAFPAAAA|62727|17|25|27|PM|second|evening|dinner| +62728|AAAAAAAAJAFPAAAA|62728|17|25|28|PM|second|evening|dinner| +62729|AAAAAAAAKAFPAAAA|62729|17|25|29|PM|second|evening|dinner| +62730|AAAAAAAALAFPAAAA|62730|17|25|30|PM|second|evening|dinner| +62731|AAAAAAAAMAFPAAAA|62731|17|25|31|PM|second|evening|dinner| +62732|AAAAAAAANAFPAAAA|62732|17|25|32|PM|second|evening|dinner| +62733|AAAAAAAAOAFPAAAA|62733|17|25|33|PM|second|evening|dinner| +62734|AAAAAAAAPAFPAAAA|62734|17|25|34|PM|second|evening|dinner| +62735|AAAAAAAAABFPAAAA|62735|17|25|35|PM|second|evening|dinner| +62736|AAAAAAAABBFPAAAA|62736|17|25|36|PM|second|evening|dinner| +62737|AAAAAAAACBFPAAAA|62737|17|25|37|PM|second|evening|dinner| +62738|AAAAAAAADBFPAAAA|62738|17|25|38|PM|second|evening|dinner| +62739|AAAAAAAAEBFPAAAA|62739|17|25|39|PM|second|evening|dinner| +62740|AAAAAAAAFBFPAAAA|62740|17|25|40|PM|second|evening|dinner| +62741|AAAAAAAAGBFPAAAA|62741|17|25|41|PM|second|evening|dinner| +62742|AAAAAAAAHBFPAAAA|62742|17|25|42|PM|second|evening|dinner| +62743|AAAAAAAAIBFPAAAA|62743|17|25|43|PM|second|evening|dinner| +62744|AAAAAAAAJBFPAAAA|62744|17|25|44|PM|second|evening|dinner| +62745|AAAAAAAAKBFPAAAA|62745|17|25|45|PM|second|evening|dinner| +62746|AAAAAAAALBFPAAAA|62746|17|25|46|PM|second|evening|dinner| +62747|AAAAAAAAMBFPAAAA|62747|17|25|47|PM|second|evening|dinner| +62748|AAAAAAAANBFPAAAA|62748|17|25|48|PM|second|evening|dinner| +62749|AAAAAAAAOBFPAAAA|62749|17|25|49|PM|second|evening|dinner| +62750|AAAAAAAAPBFPAAAA|62750|17|25|50|PM|second|evening|dinner| +62751|AAAAAAAAACFPAAAA|62751|17|25|51|PM|second|evening|dinner| +62752|AAAAAAAABCFPAAAA|62752|17|25|52|PM|second|evening|dinner| +62753|AAAAAAAACCFPAAAA|62753|17|25|53|PM|second|evening|dinner| +62754|AAAAAAAADCFPAAAA|62754|17|25|54|PM|second|evening|dinner| +62755|AAAAAAAAECFPAAAA|62755|17|25|55|PM|second|evening|dinner| +62756|AAAAAAAAFCFPAAAA|62756|17|25|56|PM|second|evening|dinner| +62757|AAAAAAAAGCFPAAAA|62757|17|25|57|PM|second|evening|dinner| +62758|AAAAAAAAHCFPAAAA|62758|17|25|58|PM|second|evening|dinner| +62759|AAAAAAAAICFPAAAA|62759|17|25|59|PM|second|evening|dinner| +62760|AAAAAAAAJCFPAAAA|62760|17|26|0|PM|second|evening|dinner| +62761|AAAAAAAAKCFPAAAA|62761|17|26|1|PM|second|evening|dinner| +62762|AAAAAAAALCFPAAAA|62762|17|26|2|PM|second|evening|dinner| +62763|AAAAAAAAMCFPAAAA|62763|17|26|3|PM|second|evening|dinner| +62764|AAAAAAAANCFPAAAA|62764|17|26|4|PM|second|evening|dinner| +62765|AAAAAAAAOCFPAAAA|62765|17|26|5|PM|second|evening|dinner| +62766|AAAAAAAAPCFPAAAA|62766|17|26|6|PM|second|evening|dinner| +62767|AAAAAAAAADFPAAAA|62767|17|26|7|PM|second|evening|dinner| +62768|AAAAAAAABDFPAAAA|62768|17|26|8|PM|second|evening|dinner| +62769|AAAAAAAACDFPAAAA|62769|17|26|9|PM|second|evening|dinner| +62770|AAAAAAAADDFPAAAA|62770|17|26|10|PM|second|evening|dinner| +62771|AAAAAAAAEDFPAAAA|62771|17|26|11|PM|second|evening|dinner| +62772|AAAAAAAAFDFPAAAA|62772|17|26|12|PM|second|evening|dinner| +62773|AAAAAAAAGDFPAAAA|62773|17|26|13|PM|second|evening|dinner| +62774|AAAAAAAAHDFPAAAA|62774|17|26|14|PM|second|evening|dinner| +62775|AAAAAAAAIDFPAAAA|62775|17|26|15|PM|second|evening|dinner| +62776|AAAAAAAAJDFPAAAA|62776|17|26|16|PM|second|evening|dinner| +62777|AAAAAAAAKDFPAAAA|62777|17|26|17|PM|second|evening|dinner| +62778|AAAAAAAALDFPAAAA|62778|17|26|18|PM|second|evening|dinner| +62779|AAAAAAAAMDFPAAAA|62779|17|26|19|PM|second|evening|dinner| +62780|AAAAAAAANDFPAAAA|62780|17|26|20|PM|second|evening|dinner| +62781|AAAAAAAAODFPAAAA|62781|17|26|21|PM|second|evening|dinner| +62782|AAAAAAAAPDFPAAAA|62782|17|26|22|PM|second|evening|dinner| +62783|AAAAAAAAAEFPAAAA|62783|17|26|23|PM|second|evening|dinner| +62784|AAAAAAAABEFPAAAA|62784|17|26|24|PM|second|evening|dinner| +62785|AAAAAAAACEFPAAAA|62785|17|26|25|PM|second|evening|dinner| +62786|AAAAAAAADEFPAAAA|62786|17|26|26|PM|second|evening|dinner| +62787|AAAAAAAAEEFPAAAA|62787|17|26|27|PM|second|evening|dinner| +62788|AAAAAAAAFEFPAAAA|62788|17|26|28|PM|second|evening|dinner| +62789|AAAAAAAAGEFPAAAA|62789|17|26|29|PM|second|evening|dinner| +62790|AAAAAAAAHEFPAAAA|62790|17|26|30|PM|second|evening|dinner| +62791|AAAAAAAAIEFPAAAA|62791|17|26|31|PM|second|evening|dinner| +62792|AAAAAAAAJEFPAAAA|62792|17|26|32|PM|second|evening|dinner| +62793|AAAAAAAAKEFPAAAA|62793|17|26|33|PM|second|evening|dinner| +62794|AAAAAAAALEFPAAAA|62794|17|26|34|PM|second|evening|dinner| +62795|AAAAAAAAMEFPAAAA|62795|17|26|35|PM|second|evening|dinner| +62796|AAAAAAAANEFPAAAA|62796|17|26|36|PM|second|evening|dinner| +62797|AAAAAAAAOEFPAAAA|62797|17|26|37|PM|second|evening|dinner| +62798|AAAAAAAAPEFPAAAA|62798|17|26|38|PM|second|evening|dinner| +62799|AAAAAAAAAFFPAAAA|62799|17|26|39|PM|second|evening|dinner| +62800|AAAAAAAABFFPAAAA|62800|17|26|40|PM|second|evening|dinner| +62801|AAAAAAAACFFPAAAA|62801|17|26|41|PM|second|evening|dinner| +62802|AAAAAAAADFFPAAAA|62802|17|26|42|PM|second|evening|dinner| +62803|AAAAAAAAEFFPAAAA|62803|17|26|43|PM|second|evening|dinner| +62804|AAAAAAAAFFFPAAAA|62804|17|26|44|PM|second|evening|dinner| +62805|AAAAAAAAGFFPAAAA|62805|17|26|45|PM|second|evening|dinner| +62806|AAAAAAAAHFFPAAAA|62806|17|26|46|PM|second|evening|dinner| +62807|AAAAAAAAIFFPAAAA|62807|17|26|47|PM|second|evening|dinner| +62808|AAAAAAAAJFFPAAAA|62808|17|26|48|PM|second|evening|dinner| +62809|AAAAAAAAKFFPAAAA|62809|17|26|49|PM|second|evening|dinner| +62810|AAAAAAAALFFPAAAA|62810|17|26|50|PM|second|evening|dinner| +62811|AAAAAAAAMFFPAAAA|62811|17|26|51|PM|second|evening|dinner| +62812|AAAAAAAANFFPAAAA|62812|17|26|52|PM|second|evening|dinner| +62813|AAAAAAAAOFFPAAAA|62813|17|26|53|PM|second|evening|dinner| +62814|AAAAAAAAPFFPAAAA|62814|17|26|54|PM|second|evening|dinner| +62815|AAAAAAAAAGFPAAAA|62815|17|26|55|PM|second|evening|dinner| +62816|AAAAAAAABGFPAAAA|62816|17|26|56|PM|second|evening|dinner| +62817|AAAAAAAACGFPAAAA|62817|17|26|57|PM|second|evening|dinner| +62818|AAAAAAAADGFPAAAA|62818|17|26|58|PM|second|evening|dinner| +62819|AAAAAAAAEGFPAAAA|62819|17|26|59|PM|second|evening|dinner| +62820|AAAAAAAAFGFPAAAA|62820|17|27|0|PM|second|evening|dinner| +62821|AAAAAAAAGGFPAAAA|62821|17|27|1|PM|second|evening|dinner| +62822|AAAAAAAAHGFPAAAA|62822|17|27|2|PM|second|evening|dinner| +62823|AAAAAAAAIGFPAAAA|62823|17|27|3|PM|second|evening|dinner| +62824|AAAAAAAAJGFPAAAA|62824|17|27|4|PM|second|evening|dinner| +62825|AAAAAAAAKGFPAAAA|62825|17|27|5|PM|second|evening|dinner| +62826|AAAAAAAALGFPAAAA|62826|17|27|6|PM|second|evening|dinner| +62827|AAAAAAAAMGFPAAAA|62827|17|27|7|PM|second|evening|dinner| +62828|AAAAAAAANGFPAAAA|62828|17|27|8|PM|second|evening|dinner| +62829|AAAAAAAAOGFPAAAA|62829|17|27|9|PM|second|evening|dinner| +62830|AAAAAAAAPGFPAAAA|62830|17|27|10|PM|second|evening|dinner| +62831|AAAAAAAAAHFPAAAA|62831|17|27|11|PM|second|evening|dinner| +62832|AAAAAAAABHFPAAAA|62832|17|27|12|PM|second|evening|dinner| +62833|AAAAAAAACHFPAAAA|62833|17|27|13|PM|second|evening|dinner| +62834|AAAAAAAADHFPAAAA|62834|17|27|14|PM|second|evening|dinner| +62835|AAAAAAAAEHFPAAAA|62835|17|27|15|PM|second|evening|dinner| +62836|AAAAAAAAFHFPAAAA|62836|17|27|16|PM|second|evening|dinner| +62837|AAAAAAAAGHFPAAAA|62837|17|27|17|PM|second|evening|dinner| +62838|AAAAAAAAHHFPAAAA|62838|17|27|18|PM|second|evening|dinner| +62839|AAAAAAAAIHFPAAAA|62839|17|27|19|PM|second|evening|dinner| +62840|AAAAAAAAJHFPAAAA|62840|17|27|20|PM|second|evening|dinner| +62841|AAAAAAAAKHFPAAAA|62841|17|27|21|PM|second|evening|dinner| +62842|AAAAAAAALHFPAAAA|62842|17|27|22|PM|second|evening|dinner| +62843|AAAAAAAAMHFPAAAA|62843|17|27|23|PM|second|evening|dinner| +62844|AAAAAAAANHFPAAAA|62844|17|27|24|PM|second|evening|dinner| +62845|AAAAAAAAOHFPAAAA|62845|17|27|25|PM|second|evening|dinner| +62846|AAAAAAAAPHFPAAAA|62846|17|27|26|PM|second|evening|dinner| +62847|AAAAAAAAAIFPAAAA|62847|17|27|27|PM|second|evening|dinner| +62848|AAAAAAAABIFPAAAA|62848|17|27|28|PM|second|evening|dinner| +62849|AAAAAAAACIFPAAAA|62849|17|27|29|PM|second|evening|dinner| +62850|AAAAAAAADIFPAAAA|62850|17|27|30|PM|second|evening|dinner| +62851|AAAAAAAAEIFPAAAA|62851|17|27|31|PM|second|evening|dinner| +62852|AAAAAAAAFIFPAAAA|62852|17|27|32|PM|second|evening|dinner| +62853|AAAAAAAAGIFPAAAA|62853|17|27|33|PM|second|evening|dinner| +62854|AAAAAAAAHIFPAAAA|62854|17|27|34|PM|second|evening|dinner| +62855|AAAAAAAAIIFPAAAA|62855|17|27|35|PM|second|evening|dinner| +62856|AAAAAAAAJIFPAAAA|62856|17|27|36|PM|second|evening|dinner| +62857|AAAAAAAAKIFPAAAA|62857|17|27|37|PM|second|evening|dinner| +62858|AAAAAAAALIFPAAAA|62858|17|27|38|PM|second|evening|dinner| +62859|AAAAAAAAMIFPAAAA|62859|17|27|39|PM|second|evening|dinner| +62860|AAAAAAAANIFPAAAA|62860|17|27|40|PM|second|evening|dinner| +62861|AAAAAAAAOIFPAAAA|62861|17|27|41|PM|second|evening|dinner| +62862|AAAAAAAAPIFPAAAA|62862|17|27|42|PM|second|evening|dinner| +62863|AAAAAAAAAJFPAAAA|62863|17|27|43|PM|second|evening|dinner| +62864|AAAAAAAABJFPAAAA|62864|17|27|44|PM|second|evening|dinner| +62865|AAAAAAAACJFPAAAA|62865|17|27|45|PM|second|evening|dinner| +62866|AAAAAAAADJFPAAAA|62866|17|27|46|PM|second|evening|dinner| +62867|AAAAAAAAEJFPAAAA|62867|17|27|47|PM|second|evening|dinner| +62868|AAAAAAAAFJFPAAAA|62868|17|27|48|PM|second|evening|dinner| +62869|AAAAAAAAGJFPAAAA|62869|17|27|49|PM|second|evening|dinner| +62870|AAAAAAAAHJFPAAAA|62870|17|27|50|PM|second|evening|dinner| +62871|AAAAAAAAIJFPAAAA|62871|17|27|51|PM|second|evening|dinner| +62872|AAAAAAAAJJFPAAAA|62872|17|27|52|PM|second|evening|dinner| +62873|AAAAAAAAKJFPAAAA|62873|17|27|53|PM|second|evening|dinner| +62874|AAAAAAAALJFPAAAA|62874|17|27|54|PM|second|evening|dinner| +62875|AAAAAAAAMJFPAAAA|62875|17|27|55|PM|second|evening|dinner| +62876|AAAAAAAANJFPAAAA|62876|17|27|56|PM|second|evening|dinner| +62877|AAAAAAAAOJFPAAAA|62877|17|27|57|PM|second|evening|dinner| +62878|AAAAAAAAPJFPAAAA|62878|17|27|58|PM|second|evening|dinner| +62879|AAAAAAAAAKFPAAAA|62879|17|27|59|PM|second|evening|dinner| +62880|AAAAAAAABKFPAAAA|62880|17|28|0|PM|second|evening|dinner| +62881|AAAAAAAACKFPAAAA|62881|17|28|1|PM|second|evening|dinner| +62882|AAAAAAAADKFPAAAA|62882|17|28|2|PM|second|evening|dinner| +62883|AAAAAAAAEKFPAAAA|62883|17|28|3|PM|second|evening|dinner| +62884|AAAAAAAAFKFPAAAA|62884|17|28|4|PM|second|evening|dinner| +62885|AAAAAAAAGKFPAAAA|62885|17|28|5|PM|second|evening|dinner| +62886|AAAAAAAAHKFPAAAA|62886|17|28|6|PM|second|evening|dinner| +62887|AAAAAAAAIKFPAAAA|62887|17|28|7|PM|second|evening|dinner| +62888|AAAAAAAAJKFPAAAA|62888|17|28|8|PM|second|evening|dinner| +62889|AAAAAAAAKKFPAAAA|62889|17|28|9|PM|second|evening|dinner| +62890|AAAAAAAALKFPAAAA|62890|17|28|10|PM|second|evening|dinner| +62891|AAAAAAAAMKFPAAAA|62891|17|28|11|PM|second|evening|dinner| +62892|AAAAAAAANKFPAAAA|62892|17|28|12|PM|second|evening|dinner| +62893|AAAAAAAAOKFPAAAA|62893|17|28|13|PM|second|evening|dinner| +62894|AAAAAAAAPKFPAAAA|62894|17|28|14|PM|second|evening|dinner| +62895|AAAAAAAAALFPAAAA|62895|17|28|15|PM|second|evening|dinner| +62896|AAAAAAAABLFPAAAA|62896|17|28|16|PM|second|evening|dinner| +62897|AAAAAAAACLFPAAAA|62897|17|28|17|PM|second|evening|dinner| +62898|AAAAAAAADLFPAAAA|62898|17|28|18|PM|second|evening|dinner| +62899|AAAAAAAAELFPAAAA|62899|17|28|19|PM|second|evening|dinner| +62900|AAAAAAAAFLFPAAAA|62900|17|28|20|PM|second|evening|dinner| +62901|AAAAAAAAGLFPAAAA|62901|17|28|21|PM|second|evening|dinner| +62902|AAAAAAAAHLFPAAAA|62902|17|28|22|PM|second|evening|dinner| +62903|AAAAAAAAILFPAAAA|62903|17|28|23|PM|second|evening|dinner| +62904|AAAAAAAAJLFPAAAA|62904|17|28|24|PM|second|evening|dinner| +62905|AAAAAAAAKLFPAAAA|62905|17|28|25|PM|second|evening|dinner| +62906|AAAAAAAALLFPAAAA|62906|17|28|26|PM|second|evening|dinner| +62907|AAAAAAAAMLFPAAAA|62907|17|28|27|PM|second|evening|dinner| +62908|AAAAAAAANLFPAAAA|62908|17|28|28|PM|second|evening|dinner| +62909|AAAAAAAAOLFPAAAA|62909|17|28|29|PM|second|evening|dinner| +62910|AAAAAAAAPLFPAAAA|62910|17|28|30|PM|second|evening|dinner| +62911|AAAAAAAAAMFPAAAA|62911|17|28|31|PM|second|evening|dinner| +62912|AAAAAAAABMFPAAAA|62912|17|28|32|PM|second|evening|dinner| +62913|AAAAAAAACMFPAAAA|62913|17|28|33|PM|second|evening|dinner| +62914|AAAAAAAADMFPAAAA|62914|17|28|34|PM|second|evening|dinner| +62915|AAAAAAAAEMFPAAAA|62915|17|28|35|PM|second|evening|dinner| +62916|AAAAAAAAFMFPAAAA|62916|17|28|36|PM|second|evening|dinner| +62917|AAAAAAAAGMFPAAAA|62917|17|28|37|PM|second|evening|dinner| +62918|AAAAAAAAHMFPAAAA|62918|17|28|38|PM|second|evening|dinner| +62919|AAAAAAAAIMFPAAAA|62919|17|28|39|PM|second|evening|dinner| +62920|AAAAAAAAJMFPAAAA|62920|17|28|40|PM|second|evening|dinner| +62921|AAAAAAAAKMFPAAAA|62921|17|28|41|PM|second|evening|dinner| +62922|AAAAAAAALMFPAAAA|62922|17|28|42|PM|second|evening|dinner| +62923|AAAAAAAAMMFPAAAA|62923|17|28|43|PM|second|evening|dinner| +62924|AAAAAAAANMFPAAAA|62924|17|28|44|PM|second|evening|dinner| +62925|AAAAAAAAOMFPAAAA|62925|17|28|45|PM|second|evening|dinner| +62926|AAAAAAAAPMFPAAAA|62926|17|28|46|PM|second|evening|dinner| +62927|AAAAAAAAANFPAAAA|62927|17|28|47|PM|second|evening|dinner| +62928|AAAAAAAABNFPAAAA|62928|17|28|48|PM|second|evening|dinner| +62929|AAAAAAAACNFPAAAA|62929|17|28|49|PM|second|evening|dinner| +62930|AAAAAAAADNFPAAAA|62930|17|28|50|PM|second|evening|dinner| +62931|AAAAAAAAENFPAAAA|62931|17|28|51|PM|second|evening|dinner| +62932|AAAAAAAAFNFPAAAA|62932|17|28|52|PM|second|evening|dinner| +62933|AAAAAAAAGNFPAAAA|62933|17|28|53|PM|second|evening|dinner| +62934|AAAAAAAAHNFPAAAA|62934|17|28|54|PM|second|evening|dinner| +62935|AAAAAAAAINFPAAAA|62935|17|28|55|PM|second|evening|dinner| +62936|AAAAAAAAJNFPAAAA|62936|17|28|56|PM|second|evening|dinner| +62937|AAAAAAAAKNFPAAAA|62937|17|28|57|PM|second|evening|dinner| +62938|AAAAAAAALNFPAAAA|62938|17|28|58|PM|second|evening|dinner| +62939|AAAAAAAAMNFPAAAA|62939|17|28|59|PM|second|evening|dinner| +62940|AAAAAAAANNFPAAAA|62940|17|29|0|PM|second|evening|dinner| +62941|AAAAAAAAONFPAAAA|62941|17|29|1|PM|second|evening|dinner| +62942|AAAAAAAAPNFPAAAA|62942|17|29|2|PM|second|evening|dinner| +62943|AAAAAAAAAOFPAAAA|62943|17|29|3|PM|second|evening|dinner| +62944|AAAAAAAABOFPAAAA|62944|17|29|4|PM|second|evening|dinner| +62945|AAAAAAAACOFPAAAA|62945|17|29|5|PM|second|evening|dinner| +62946|AAAAAAAADOFPAAAA|62946|17|29|6|PM|second|evening|dinner| +62947|AAAAAAAAEOFPAAAA|62947|17|29|7|PM|second|evening|dinner| +62948|AAAAAAAAFOFPAAAA|62948|17|29|8|PM|second|evening|dinner| +62949|AAAAAAAAGOFPAAAA|62949|17|29|9|PM|second|evening|dinner| +62950|AAAAAAAAHOFPAAAA|62950|17|29|10|PM|second|evening|dinner| +62951|AAAAAAAAIOFPAAAA|62951|17|29|11|PM|second|evening|dinner| +62952|AAAAAAAAJOFPAAAA|62952|17|29|12|PM|second|evening|dinner| +62953|AAAAAAAAKOFPAAAA|62953|17|29|13|PM|second|evening|dinner| +62954|AAAAAAAALOFPAAAA|62954|17|29|14|PM|second|evening|dinner| +62955|AAAAAAAAMOFPAAAA|62955|17|29|15|PM|second|evening|dinner| +62956|AAAAAAAANOFPAAAA|62956|17|29|16|PM|second|evening|dinner| +62957|AAAAAAAAOOFPAAAA|62957|17|29|17|PM|second|evening|dinner| +62958|AAAAAAAAPOFPAAAA|62958|17|29|18|PM|second|evening|dinner| +62959|AAAAAAAAAPFPAAAA|62959|17|29|19|PM|second|evening|dinner| +62960|AAAAAAAABPFPAAAA|62960|17|29|20|PM|second|evening|dinner| +62961|AAAAAAAACPFPAAAA|62961|17|29|21|PM|second|evening|dinner| +62962|AAAAAAAADPFPAAAA|62962|17|29|22|PM|second|evening|dinner| +62963|AAAAAAAAEPFPAAAA|62963|17|29|23|PM|second|evening|dinner| +62964|AAAAAAAAFPFPAAAA|62964|17|29|24|PM|second|evening|dinner| +62965|AAAAAAAAGPFPAAAA|62965|17|29|25|PM|second|evening|dinner| +62966|AAAAAAAAHPFPAAAA|62966|17|29|26|PM|second|evening|dinner| +62967|AAAAAAAAIPFPAAAA|62967|17|29|27|PM|second|evening|dinner| +62968|AAAAAAAAJPFPAAAA|62968|17|29|28|PM|second|evening|dinner| +62969|AAAAAAAAKPFPAAAA|62969|17|29|29|PM|second|evening|dinner| +62970|AAAAAAAALPFPAAAA|62970|17|29|30|PM|second|evening|dinner| +62971|AAAAAAAAMPFPAAAA|62971|17|29|31|PM|second|evening|dinner| +62972|AAAAAAAANPFPAAAA|62972|17|29|32|PM|second|evening|dinner| +62973|AAAAAAAAOPFPAAAA|62973|17|29|33|PM|second|evening|dinner| +62974|AAAAAAAAPPFPAAAA|62974|17|29|34|PM|second|evening|dinner| +62975|AAAAAAAAAAGPAAAA|62975|17|29|35|PM|second|evening|dinner| +62976|AAAAAAAABAGPAAAA|62976|17|29|36|PM|second|evening|dinner| +62977|AAAAAAAACAGPAAAA|62977|17|29|37|PM|second|evening|dinner| +62978|AAAAAAAADAGPAAAA|62978|17|29|38|PM|second|evening|dinner| +62979|AAAAAAAAEAGPAAAA|62979|17|29|39|PM|second|evening|dinner| +62980|AAAAAAAAFAGPAAAA|62980|17|29|40|PM|second|evening|dinner| +62981|AAAAAAAAGAGPAAAA|62981|17|29|41|PM|second|evening|dinner| +62982|AAAAAAAAHAGPAAAA|62982|17|29|42|PM|second|evening|dinner| +62983|AAAAAAAAIAGPAAAA|62983|17|29|43|PM|second|evening|dinner| +62984|AAAAAAAAJAGPAAAA|62984|17|29|44|PM|second|evening|dinner| +62985|AAAAAAAAKAGPAAAA|62985|17|29|45|PM|second|evening|dinner| +62986|AAAAAAAALAGPAAAA|62986|17|29|46|PM|second|evening|dinner| +62987|AAAAAAAAMAGPAAAA|62987|17|29|47|PM|second|evening|dinner| +62988|AAAAAAAANAGPAAAA|62988|17|29|48|PM|second|evening|dinner| +62989|AAAAAAAAOAGPAAAA|62989|17|29|49|PM|second|evening|dinner| +62990|AAAAAAAAPAGPAAAA|62990|17|29|50|PM|second|evening|dinner| +62991|AAAAAAAAABGPAAAA|62991|17|29|51|PM|second|evening|dinner| +62992|AAAAAAAABBGPAAAA|62992|17|29|52|PM|second|evening|dinner| +62993|AAAAAAAACBGPAAAA|62993|17|29|53|PM|second|evening|dinner| +62994|AAAAAAAADBGPAAAA|62994|17|29|54|PM|second|evening|dinner| +62995|AAAAAAAAEBGPAAAA|62995|17|29|55|PM|second|evening|dinner| +62996|AAAAAAAAFBGPAAAA|62996|17|29|56|PM|second|evening|dinner| +62997|AAAAAAAAGBGPAAAA|62997|17|29|57|PM|second|evening|dinner| +62998|AAAAAAAAHBGPAAAA|62998|17|29|58|PM|second|evening|dinner| +62999|AAAAAAAAIBGPAAAA|62999|17|29|59|PM|second|evening|dinner| +63000|AAAAAAAAJBGPAAAA|63000|17|30|0|PM|second|evening|dinner| +63001|AAAAAAAAKBGPAAAA|63001|17|30|1|PM|second|evening|dinner| +63002|AAAAAAAALBGPAAAA|63002|17|30|2|PM|second|evening|dinner| +63003|AAAAAAAAMBGPAAAA|63003|17|30|3|PM|second|evening|dinner| +63004|AAAAAAAANBGPAAAA|63004|17|30|4|PM|second|evening|dinner| +63005|AAAAAAAAOBGPAAAA|63005|17|30|5|PM|second|evening|dinner| +63006|AAAAAAAAPBGPAAAA|63006|17|30|6|PM|second|evening|dinner| +63007|AAAAAAAAACGPAAAA|63007|17|30|7|PM|second|evening|dinner| +63008|AAAAAAAABCGPAAAA|63008|17|30|8|PM|second|evening|dinner| +63009|AAAAAAAACCGPAAAA|63009|17|30|9|PM|second|evening|dinner| +63010|AAAAAAAADCGPAAAA|63010|17|30|10|PM|second|evening|dinner| +63011|AAAAAAAAECGPAAAA|63011|17|30|11|PM|second|evening|dinner| +63012|AAAAAAAAFCGPAAAA|63012|17|30|12|PM|second|evening|dinner| +63013|AAAAAAAAGCGPAAAA|63013|17|30|13|PM|second|evening|dinner| +63014|AAAAAAAAHCGPAAAA|63014|17|30|14|PM|second|evening|dinner| +63015|AAAAAAAAICGPAAAA|63015|17|30|15|PM|second|evening|dinner| +63016|AAAAAAAAJCGPAAAA|63016|17|30|16|PM|second|evening|dinner| +63017|AAAAAAAAKCGPAAAA|63017|17|30|17|PM|second|evening|dinner| +63018|AAAAAAAALCGPAAAA|63018|17|30|18|PM|second|evening|dinner| +63019|AAAAAAAAMCGPAAAA|63019|17|30|19|PM|second|evening|dinner| +63020|AAAAAAAANCGPAAAA|63020|17|30|20|PM|second|evening|dinner| +63021|AAAAAAAAOCGPAAAA|63021|17|30|21|PM|second|evening|dinner| +63022|AAAAAAAAPCGPAAAA|63022|17|30|22|PM|second|evening|dinner| +63023|AAAAAAAAADGPAAAA|63023|17|30|23|PM|second|evening|dinner| +63024|AAAAAAAABDGPAAAA|63024|17|30|24|PM|second|evening|dinner| +63025|AAAAAAAACDGPAAAA|63025|17|30|25|PM|second|evening|dinner| +63026|AAAAAAAADDGPAAAA|63026|17|30|26|PM|second|evening|dinner| +63027|AAAAAAAAEDGPAAAA|63027|17|30|27|PM|second|evening|dinner| +63028|AAAAAAAAFDGPAAAA|63028|17|30|28|PM|second|evening|dinner| +63029|AAAAAAAAGDGPAAAA|63029|17|30|29|PM|second|evening|dinner| +63030|AAAAAAAAHDGPAAAA|63030|17|30|30|PM|second|evening|dinner| +63031|AAAAAAAAIDGPAAAA|63031|17|30|31|PM|second|evening|dinner| +63032|AAAAAAAAJDGPAAAA|63032|17|30|32|PM|second|evening|dinner| +63033|AAAAAAAAKDGPAAAA|63033|17|30|33|PM|second|evening|dinner| +63034|AAAAAAAALDGPAAAA|63034|17|30|34|PM|second|evening|dinner| +63035|AAAAAAAAMDGPAAAA|63035|17|30|35|PM|second|evening|dinner| +63036|AAAAAAAANDGPAAAA|63036|17|30|36|PM|second|evening|dinner| +63037|AAAAAAAAODGPAAAA|63037|17|30|37|PM|second|evening|dinner| +63038|AAAAAAAAPDGPAAAA|63038|17|30|38|PM|second|evening|dinner| +63039|AAAAAAAAAEGPAAAA|63039|17|30|39|PM|second|evening|dinner| +63040|AAAAAAAABEGPAAAA|63040|17|30|40|PM|second|evening|dinner| +63041|AAAAAAAACEGPAAAA|63041|17|30|41|PM|second|evening|dinner| +63042|AAAAAAAADEGPAAAA|63042|17|30|42|PM|second|evening|dinner| +63043|AAAAAAAAEEGPAAAA|63043|17|30|43|PM|second|evening|dinner| +63044|AAAAAAAAFEGPAAAA|63044|17|30|44|PM|second|evening|dinner| +63045|AAAAAAAAGEGPAAAA|63045|17|30|45|PM|second|evening|dinner| +63046|AAAAAAAAHEGPAAAA|63046|17|30|46|PM|second|evening|dinner| +63047|AAAAAAAAIEGPAAAA|63047|17|30|47|PM|second|evening|dinner| +63048|AAAAAAAAJEGPAAAA|63048|17|30|48|PM|second|evening|dinner| +63049|AAAAAAAAKEGPAAAA|63049|17|30|49|PM|second|evening|dinner| +63050|AAAAAAAALEGPAAAA|63050|17|30|50|PM|second|evening|dinner| +63051|AAAAAAAAMEGPAAAA|63051|17|30|51|PM|second|evening|dinner| +63052|AAAAAAAANEGPAAAA|63052|17|30|52|PM|second|evening|dinner| +63053|AAAAAAAAOEGPAAAA|63053|17|30|53|PM|second|evening|dinner| +63054|AAAAAAAAPEGPAAAA|63054|17|30|54|PM|second|evening|dinner| +63055|AAAAAAAAAFGPAAAA|63055|17|30|55|PM|second|evening|dinner| +63056|AAAAAAAABFGPAAAA|63056|17|30|56|PM|second|evening|dinner| +63057|AAAAAAAACFGPAAAA|63057|17|30|57|PM|second|evening|dinner| +63058|AAAAAAAADFGPAAAA|63058|17|30|58|PM|second|evening|dinner| +63059|AAAAAAAAEFGPAAAA|63059|17|30|59|PM|second|evening|dinner| +63060|AAAAAAAAFFGPAAAA|63060|17|31|0|PM|second|evening|dinner| +63061|AAAAAAAAGFGPAAAA|63061|17|31|1|PM|second|evening|dinner| +63062|AAAAAAAAHFGPAAAA|63062|17|31|2|PM|second|evening|dinner| +63063|AAAAAAAAIFGPAAAA|63063|17|31|3|PM|second|evening|dinner| +63064|AAAAAAAAJFGPAAAA|63064|17|31|4|PM|second|evening|dinner| +63065|AAAAAAAAKFGPAAAA|63065|17|31|5|PM|second|evening|dinner| +63066|AAAAAAAALFGPAAAA|63066|17|31|6|PM|second|evening|dinner| +63067|AAAAAAAAMFGPAAAA|63067|17|31|7|PM|second|evening|dinner| +63068|AAAAAAAANFGPAAAA|63068|17|31|8|PM|second|evening|dinner| +63069|AAAAAAAAOFGPAAAA|63069|17|31|9|PM|second|evening|dinner| +63070|AAAAAAAAPFGPAAAA|63070|17|31|10|PM|second|evening|dinner| +63071|AAAAAAAAAGGPAAAA|63071|17|31|11|PM|second|evening|dinner| +63072|AAAAAAAABGGPAAAA|63072|17|31|12|PM|second|evening|dinner| +63073|AAAAAAAACGGPAAAA|63073|17|31|13|PM|second|evening|dinner| +63074|AAAAAAAADGGPAAAA|63074|17|31|14|PM|second|evening|dinner| +63075|AAAAAAAAEGGPAAAA|63075|17|31|15|PM|second|evening|dinner| +63076|AAAAAAAAFGGPAAAA|63076|17|31|16|PM|second|evening|dinner| +63077|AAAAAAAAGGGPAAAA|63077|17|31|17|PM|second|evening|dinner| +63078|AAAAAAAAHGGPAAAA|63078|17|31|18|PM|second|evening|dinner| +63079|AAAAAAAAIGGPAAAA|63079|17|31|19|PM|second|evening|dinner| +63080|AAAAAAAAJGGPAAAA|63080|17|31|20|PM|second|evening|dinner| +63081|AAAAAAAAKGGPAAAA|63081|17|31|21|PM|second|evening|dinner| +63082|AAAAAAAALGGPAAAA|63082|17|31|22|PM|second|evening|dinner| +63083|AAAAAAAAMGGPAAAA|63083|17|31|23|PM|second|evening|dinner| +63084|AAAAAAAANGGPAAAA|63084|17|31|24|PM|second|evening|dinner| +63085|AAAAAAAAOGGPAAAA|63085|17|31|25|PM|second|evening|dinner| +63086|AAAAAAAAPGGPAAAA|63086|17|31|26|PM|second|evening|dinner| +63087|AAAAAAAAAHGPAAAA|63087|17|31|27|PM|second|evening|dinner| +63088|AAAAAAAABHGPAAAA|63088|17|31|28|PM|second|evening|dinner| +63089|AAAAAAAACHGPAAAA|63089|17|31|29|PM|second|evening|dinner| +63090|AAAAAAAADHGPAAAA|63090|17|31|30|PM|second|evening|dinner| +63091|AAAAAAAAEHGPAAAA|63091|17|31|31|PM|second|evening|dinner| +63092|AAAAAAAAFHGPAAAA|63092|17|31|32|PM|second|evening|dinner| +63093|AAAAAAAAGHGPAAAA|63093|17|31|33|PM|second|evening|dinner| +63094|AAAAAAAAHHGPAAAA|63094|17|31|34|PM|second|evening|dinner| +63095|AAAAAAAAIHGPAAAA|63095|17|31|35|PM|second|evening|dinner| +63096|AAAAAAAAJHGPAAAA|63096|17|31|36|PM|second|evening|dinner| +63097|AAAAAAAAKHGPAAAA|63097|17|31|37|PM|second|evening|dinner| +63098|AAAAAAAALHGPAAAA|63098|17|31|38|PM|second|evening|dinner| +63099|AAAAAAAAMHGPAAAA|63099|17|31|39|PM|second|evening|dinner| +63100|AAAAAAAANHGPAAAA|63100|17|31|40|PM|second|evening|dinner| +63101|AAAAAAAAOHGPAAAA|63101|17|31|41|PM|second|evening|dinner| +63102|AAAAAAAAPHGPAAAA|63102|17|31|42|PM|second|evening|dinner| +63103|AAAAAAAAAIGPAAAA|63103|17|31|43|PM|second|evening|dinner| +63104|AAAAAAAABIGPAAAA|63104|17|31|44|PM|second|evening|dinner| +63105|AAAAAAAACIGPAAAA|63105|17|31|45|PM|second|evening|dinner| +63106|AAAAAAAADIGPAAAA|63106|17|31|46|PM|second|evening|dinner| +63107|AAAAAAAAEIGPAAAA|63107|17|31|47|PM|second|evening|dinner| +63108|AAAAAAAAFIGPAAAA|63108|17|31|48|PM|second|evening|dinner| +63109|AAAAAAAAGIGPAAAA|63109|17|31|49|PM|second|evening|dinner| +63110|AAAAAAAAHIGPAAAA|63110|17|31|50|PM|second|evening|dinner| +63111|AAAAAAAAIIGPAAAA|63111|17|31|51|PM|second|evening|dinner| +63112|AAAAAAAAJIGPAAAA|63112|17|31|52|PM|second|evening|dinner| +63113|AAAAAAAAKIGPAAAA|63113|17|31|53|PM|second|evening|dinner| +63114|AAAAAAAALIGPAAAA|63114|17|31|54|PM|second|evening|dinner| +63115|AAAAAAAAMIGPAAAA|63115|17|31|55|PM|second|evening|dinner| +63116|AAAAAAAANIGPAAAA|63116|17|31|56|PM|second|evening|dinner| +63117|AAAAAAAAOIGPAAAA|63117|17|31|57|PM|second|evening|dinner| +63118|AAAAAAAAPIGPAAAA|63118|17|31|58|PM|second|evening|dinner| +63119|AAAAAAAAAJGPAAAA|63119|17|31|59|PM|second|evening|dinner| +63120|AAAAAAAABJGPAAAA|63120|17|32|0|PM|second|evening|dinner| +63121|AAAAAAAACJGPAAAA|63121|17|32|1|PM|second|evening|dinner| +63122|AAAAAAAADJGPAAAA|63122|17|32|2|PM|second|evening|dinner| +63123|AAAAAAAAEJGPAAAA|63123|17|32|3|PM|second|evening|dinner| +63124|AAAAAAAAFJGPAAAA|63124|17|32|4|PM|second|evening|dinner| +63125|AAAAAAAAGJGPAAAA|63125|17|32|5|PM|second|evening|dinner| +63126|AAAAAAAAHJGPAAAA|63126|17|32|6|PM|second|evening|dinner| +63127|AAAAAAAAIJGPAAAA|63127|17|32|7|PM|second|evening|dinner| +63128|AAAAAAAAJJGPAAAA|63128|17|32|8|PM|second|evening|dinner| +63129|AAAAAAAAKJGPAAAA|63129|17|32|9|PM|second|evening|dinner| +63130|AAAAAAAALJGPAAAA|63130|17|32|10|PM|second|evening|dinner| +63131|AAAAAAAAMJGPAAAA|63131|17|32|11|PM|second|evening|dinner| +63132|AAAAAAAANJGPAAAA|63132|17|32|12|PM|second|evening|dinner| +63133|AAAAAAAAOJGPAAAA|63133|17|32|13|PM|second|evening|dinner| +63134|AAAAAAAAPJGPAAAA|63134|17|32|14|PM|second|evening|dinner| +63135|AAAAAAAAAKGPAAAA|63135|17|32|15|PM|second|evening|dinner| +63136|AAAAAAAABKGPAAAA|63136|17|32|16|PM|second|evening|dinner| +63137|AAAAAAAACKGPAAAA|63137|17|32|17|PM|second|evening|dinner| +63138|AAAAAAAADKGPAAAA|63138|17|32|18|PM|second|evening|dinner| +63139|AAAAAAAAEKGPAAAA|63139|17|32|19|PM|second|evening|dinner| +63140|AAAAAAAAFKGPAAAA|63140|17|32|20|PM|second|evening|dinner| +63141|AAAAAAAAGKGPAAAA|63141|17|32|21|PM|second|evening|dinner| +63142|AAAAAAAAHKGPAAAA|63142|17|32|22|PM|second|evening|dinner| +63143|AAAAAAAAIKGPAAAA|63143|17|32|23|PM|second|evening|dinner| +63144|AAAAAAAAJKGPAAAA|63144|17|32|24|PM|second|evening|dinner| +63145|AAAAAAAAKKGPAAAA|63145|17|32|25|PM|second|evening|dinner| +63146|AAAAAAAALKGPAAAA|63146|17|32|26|PM|second|evening|dinner| +63147|AAAAAAAAMKGPAAAA|63147|17|32|27|PM|second|evening|dinner| +63148|AAAAAAAANKGPAAAA|63148|17|32|28|PM|second|evening|dinner| +63149|AAAAAAAAOKGPAAAA|63149|17|32|29|PM|second|evening|dinner| +63150|AAAAAAAAPKGPAAAA|63150|17|32|30|PM|second|evening|dinner| +63151|AAAAAAAAALGPAAAA|63151|17|32|31|PM|second|evening|dinner| +63152|AAAAAAAABLGPAAAA|63152|17|32|32|PM|second|evening|dinner| +63153|AAAAAAAACLGPAAAA|63153|17|32|33|PM|second|evening|dinner| +63154|AAAAAAAADLGPAAAA|63154|17|32|34|PM|second|evening|dinner| +63155|AAAAAAAAELGPAAAA|63155|17|32|35|PM|second|evening|dinner| +63156|AAAAAAAAFLGPAAAA|63156|17|32|36|PM|second|evening|dinner| +63157|AAAAAAAAGLGPAAAA|63157|17|32|37|PM|second|evening|dinner| +63158|AAAAAAAAHLGPAAAA|63158|17|32|38|PM|second|evening|dinner| +63159|AAAAAAAAILGPAAAA|63159|17|32|39|PM|second|evening|dinner| +63160|AAAAAAAAJLGPAAAA|63160|17|32|40|PM|second|evening|dinner| +63161|AAAAAAAAKLGPAAAA|63161|17|32|41|PM|second|evening|dinner| +63162|AAAAAAAALLGPAAAA|63162|17|32|42|PM|second|evening|dinner| +63163|AAAAAAAAMLGPAAAA|63163|17|32|43|PM|second|evening|dinner| +63164|AAAAAAAANLGPAAAA|63164|17|32|44|PM|second|evening|dinner| +63165|AAAAAAAAOLGPAAAA|63165|17|32|45|PM|second|evening|dinner| +63166|AAAAAAAAPLGPAAAA|63166|17|32|46|PM|second|evening|dinner| +63167|AAAAAAAAAMGPAAAA|63167|17|32|47|PM|second|evening|dinner| +63168|AAAAAAAABMGPAAAA|63168|17|32|48|PM|second|evening|dinner| +63169|AAAAAAAACMGPAAAA|63169|17|32|49|PM|second|evening|dinner| +63170|AAAAAAAADMGPAAAA|63170|17|32|50|PM|second|evening|dinner| +63171|AAAAAAAAEMGPAAAA|63171|17|32|51|PM|second|evening|dinner| +63172|AAAAAAAAFMGPAAAA|63172|17|32|52|PM|second|evening|dinner| +63173|AAAAAAAAGMGPAAAA|63173|17|32|53|PM|second|evening|dinner| +63174|AAAAAAAAHMGPAAAA|63174|17|32|54|PM|second|evening|dinner| +63175|AAAAAAAAIMGPAAAA|63175|17|32|55|PM|second|evening|dinner| +63176|AAAAAAAAJMGPAAAA|63176|17|32|56|PM|second|evening|dinner| +63177|AAAAAAAAKMGPAAAA|63177|17|32|57|PM|second|evening|dinner| +63178|AAAAAAAALMGPAAAA|63178|17|32|58|PM|second|evening|dinner| +63179|AAAAAAAAMMGPAAAA|63179|17|32|59|PM|second|evening|dinner| +63180|AAAAAAAANMGPAAAA|63180|17|33|0|PM|second|evening|dinner| +63181|AAAAAAAAOMGPAAAA|63181|17|33|1|PM|second|evening|dinner| +63182|AAAAAAAAPMGPAAAA|63182|17|33|2|PM|second|evening|dinner| +63183|AAAAAAAAANGPAAAA|63183|17|33|3|PM|second|evening|dinner| +63184|AAAAAAAABNGPAAAA|63184|17|33|4|PM|second|evening|dinner| +63185|AAAAAAAACNGPAAAA|63185|17|33|5|PM|second|evening|dinner| +63186|AAAAAAAADNGPAAAA|63186|17|33|6|PM|second|evening|dinner| +63187|AAAAAAAAENGPAAAA|63187|17|33|7|PM|second|evening|dinner| +63188|AAAAAAAAFNGPAAAA|63188|17|33|8|PM|second|evening|dinner| +63189|AAAAAAAAGNGPAAAA|63189|17|33|9|PM|second|evening|dinner| +63190|AAAAAAAAHNGPAAAA|63190|17|33|10|PM|second|evening|dinner| +63191|AAAAAAAAINGPAAAA|63191|17|33|11|PM|second|evening|dinner| +63192|AAAAAAAAJNGPAAAA|63192|17|33|12|PM|second|evening|dinner| +63193|AAAAAAAAKNGPAAAA|63193|17|33|13|PM|second|evening|dinner| +63194|AAAAAAAALNGPAAAA|63194|17|33|14|PM|second|evening|dinner| +63195|AAAAAAAAMNGPAAAA|63195|17|33|15|PM|second|evening|dinner| +63196|AAAAAAAANNGPAAAA|63196|17|33|16|PM|second|evening|dinner| +63197|AAAAAAAAONGPAAAA|63197|17|33|17|PM|second|evening|dinner| +63198|AAAAAAAAPNGPAAAA|63198|17|33|18|PM|second|evening|dinner| +63199|AAAAAAAAAOGPAAAA|63199|17|33|19|PM|second|evening|dinner| +63200|AAAAAAAABOGPAAAA|63200|17|33|20|PM|second|evening|dinner| +63201|AAAAAAAACOGPAAAA|63201|17|33|21|PM|second|evening|dinner| +63202|AAAAAAAADOGPAAAA|63202|17|33|22|PM|second|evening|dinner| +63203|AAAAAAAAEOGPAAAA|63203|17|33|23|PM|second|evening|dinner| +63204|AAAAAAAAFOGPAAAA|63204|17|33|24|PM|second|evening|dinner| +63205|AAAAAAAAGOGPAAAA|63205|17|33|25|PM|second|evening|dinner| +63206|AAAAAAAAHOGPAAAA|63206|17|33|26|PM|second|evening|dinner| +63207|AAAAAAAAIOGPAAAA|63207|17|33|27|PM|second|evening|dinner| +63208|AAAAAAAAJOGPAAAA|63208|17|33|28|PM|second|evening|dinner| +63209|AAAAAAAAKOGPAAAA|63209|17|33|29|PM|second|evening|dinner| +63210|AAAAAAAALOGPAAAA|63210|17|33|30|PM|second|evening|dinner| +63211|AAAAAAAAMOGPAAAA|63211|17|33|31|PM|second|evening|dinner| +63212|AAAAAAAANOGPAAAA|63212|17|33|32|PM|second|evening|dinner| +63213|AAAAAAAAOOGPAAAA|63213|17|33|33|PM|second|evening|dinner| +63214|AAAAAAAAPOGPAAAA|63214|17|33|34|PM|second|evening|dinner| +63215|AAAAAAAAAPGPAAAA|63215|17|33|35|PM|second|evening|dinner| +63216|AAAAAAAABPGPAAAA|63216|17|33|36|PM|second|evening|dinner| +63217|AAAAAAAACPGPAAAA|63217|17|33|37|PM|second|evening|dinner| +63218|AAAAAAAADPGPAAAA|63218|17|33|38|PM|second|evening|dinner| +63219|AAAAAAAAEPGPAAAA|63219|17|33|39|PM|second|evening|dinner| +63220|AAAAAAAAFPGPAAAA|63220|17|33|40|PM|second|evening|dinner| +63221|AAAAAAAAGPGPAAAA|63221|17|33|41|PM|second|evening|dinner| +63222|AAAAAAAAHPGPAAAA|63222|17|33|42|PM|second|evening|dinner| +63223|AAAAAAAAIPGPAAAA|63223|17|33|43|PM|second|evening|dinner| +63224|AAAAAAAAJPGPAAAA|63224|17|33|44|PM|second|evening|dinner| +63225|AAAAAAAAKPGPAAAA|63225|17|33|45|PM|second|evening|dinner| +63226|AAAAAAAALPGPAAAA|63226|17|33|46|PM|second|evening|dinner| +63227|AAAAAAAAMPGPAAAA|63227|17|33|47|PM|second|evening|dinner| +63228|AAAAAAAANPGPAAAA|63228|17|33|48|PM|second|evening|dinner| +63229|AAAAAAAAOPGPAAAA|63229|17|33|49|PM|second|evening|dinner| +63230|AAAAAAAAPPGPAAAA|63230|17|33|50|PM|second|evening|dinner| +63231|AAAAAAAAAAHPAAAA|63231|17|33|51|PM|second|evening|dinner| +63232|AAAAAAAABAHPAAAA|63232|17|33|52|PM|second|evening|dinner| +63233|AAAAAAAACAHPAAAA|63233|17|33|53|PM|second|evening|dinner| +63234|AAAAAAAADAHPAAAA|63234|17|33|54|PM|second|evening|dinner| +63235|AAAAAAAAEAHPAAAA|63235|17|33|55|PM|second|evening|dinner| +63236|AAAAAAAAFAHPAAAA|63236|17|33|56|PM|second|evening|dinner| +63237|AAAAAAAAGAHPAAAA|63237|17|33|57|PM|second|evening|dinner| +63238|AAAAAAAAHAHPAAAA|63238|17|33|58|PM|second|evening|dinner| +63239|AAAAAAAAIAHPAAAA|63239|17|33|59|PM|second|evening|dinner| +63240|AAAAAAAAJAHPAAAA|63240|17|34|0|PM|second|evening|dinner| +63241|AAAAAAAAKAHPAAAA|63241|17|34|1|PM|second|evening|dinner| +63242|AAAAAAAALAHPAAAA|63242|17|34|2|PM|second|evening|dinner| +63243|AAAAAAAAMAHPAAAA|63243|17|34|3|PM|second|evening|dinner| +63244|AAAAAAAANAHPAAAA|63244|17|34|4|PM|second|evening|dinner| +63245|AAAAAAAAOAHPAAAA|63245|17|34|5|PM|second|evening|dinner| +63246|AAAAAAAAPAHPAAAA|63246|17|34|6|PM|second|evening|dinner| +63247|AAAAAAAAABHPAAAA|63247|17|34|7|PM|second|evening|dinner| +63248|AAAAAAAABBHPAAAA|63248|17|34|8|PM|second|evening|dinner| +63249|AAAAAAAACBHPAAAA|63249|17|34|9|PM|second|evening|dinner| +63250|AAAAAAAADBHPAAAA|63250|17|34|10|PM|second|evening|dinner| +63251|AAAAAAAAEBHPAAAA|63251|17|34|11|PM|second|evening|dinner| +63252|AAAAAAAAFBHPAAAA|63252|17|34|12|PM|second|evening|dinner| +63253|AAAAAAAAGBHPAAAA|63253|17|34|13|PM|second|evening|dinner| +63254|AAAAAAAAHBHPAAAA|63254|17|34|14|PM|second|evening|dinner| +63255|AAAAAAAAIBHPAAAA|63255|17|34|15|PM|second|evening|dinner| +63256|AAAAAAAAJBHPAAAA|63256|17|34|16|PM|second|evening|dinner| +63257|AAAAAAAAKBHPAAAA|63257|17|34|17|PM|second|evening|dinner| +63258|AAAAAAAALBHPAAAA|63258|17|34|18|PM|second|evening|dinner| +63259|AAAAAAAAMBHPAAAA|63259|17|34|19|PM|second|evening|dinner| +63260|AAAAAAAANBHPAAAA|63260|17|34|20|PM|second|evening|dinner| +63261|AAAAAAAAOBHPAAAA|63261|17|34|21|PM|second|evening|dinner| +63262|AAAAAAAAPBHPAAAA|63262|17|34|22|PM|second|evening|dinner| +63263|AAAAAAAAACHPAAAA|63263|17|34|23|PM|second|evening|dinner| +63264|AAAAAAAABCHPAAAA|63264|17|34|24|PM|second|evening|dinner| +63265|AAAAAAAACCHPAAAA|63265|17|34|25|PM|second|evening|dinner| +63266|AAAAAAAADCHPAAAA|63266|17|34|26|PM|second|evening|dinner| +63267|AAAAAAAAECHPAAAA|63267|17|34|27|PM|second|evening|dinner| +63268|AAAAAAAAFCHPAAAA|63268|17|34|28|PM|second|evening|dinner| +63269|AAAAAAAAGCHPAAAA|63269|17|34|29|PM|second|evening|dinner| +63270|AAAAAAAAHCHPAAAA|63270|17|34|30|PM|second|evening|dinner| +63271|AAAAAAAAICHPAAAA|63271|17|34|31|PM|second|evening|dinner| +63272|AAAAAAAAJCHPAAAA|63272|17|34|32|PM|second|evening|dinner| +63273|AAAAAAAAKCHPAAAA|63273|17|34|33|PM|second|evening|dinner| +63274|AAAAAAAALCHPAAAA|63274|17|34|34|PM|second|evening|dinner| +63275|AAAAAAAAMCHPAAAA|63275|17|34|35|PM|second|evening|dinner| +63276|AAAAAAAANCHPAAAA|63276|17|34|36|PM|second|evening|dinner| +63277|AAAAAAAAOCHPAAAA|63277|17|34|37|PM|second|evening|dinner| +63278|AAAAAAAAPCHPAAAA|63278|17|34|38|PM|second|evening|dinner| +63279|AAAAAAAAADHPAAAA|63279|17|34|39|PM|second|evening|dinner| +63280|AAAAAAAABDHPAAAA|63280|17|34|40|PM|second|evening|dinner| +63281|AAAAAAAACDHPAAAA|63281|17|34|41|PM|second|evening|dinner| +63282|AAAAAAAADDHPAAAA|63282|17|34|42|PM|second|evening|dinner| +63283|AAAAAAAAEDHPAAAA|63283|17|34|43|PM|second|evening|dinner| +63284|AAAAAAAAFDHPAAAA|63284|17|34|44|PM|second|evening|dinner| +63285|AAAAAAAAGDHPAAAA|63285|17|34|45|PM|second|evening|dinner| +63286|AAAAAAAAHDHPAAAA|63286|17|34|46|PM|second|evening|dinner| +63287|AAAAAAAAIDHPAAAA|63287|17|34|47|PM|second|evening|dinner| +63288|AAAAAAAAJDHPAAAA|63288|17|34|48|PM|second|evening|dinner| +63289|AAAAAAAAKDHPAAAA|63289|17|34|49|PM|second|evening|dinner| +63290|AAAAAAAALDHPAAAA|63290|17|34|50|PM|second|evening|dinner| +63291|AAAAAAAAMDHPAAAA|63291|17|34|51|PM|second|evening|dinner| +63292|AAAAAAAANDHPAAAA|63292|17|34|52|PM|second|evening|dinner| +63293|AAAAAAAAODHPAAAA|63293|17|34|53|PM|second|evening|dinner| +63294|AAAAAAAAPDHPAAAA|63294|17|34|54|PM|second|evening|dinner| +63295|AAAAAAAAAEHPAAAA|63295|17|34|55|PM|second|evening|dinner| +63296|AAAAAAAABEHPAAAA|63296|17|34|56|PM|second|evening|dinner| +63297|AAAAAAAACEHPAAAA|63297|17|34|57|PM|second|evening|dinner| +63298|AAAAAAAADEHPAAAA|63298|17|34|58|PM|second|evening|dinner| +63299|AAAAAAAAEEHPAAAA|63299|17|34|59|PM|second|evening|dinner| +63300|AAAAAAAAFEHPAAAA|63300|17|35|0|PM|second|evening|dinner| +63301|AAAAAAAAGEHPAAAA|63301|17|35|1|PM|second|evening|dinner| +63302|AAAAAAAAHEHPAAAA|63302|17|35|2|PM|second|evening|dinner| +63303|AAAAAAAAIEHPAAAA|63303|17|35|3|PM|second|evening|dinner| +63304|AAAAAAAAJEHPAAAA|63304|17|35|4|PM|second|evening|dinner| +63305|AAAAAAAAKEHPAAAA|63305|17|35|5|PM|second|evening|dinner| +63306|AAAAAAAALEHPAAAA|63306|17|35|6|PM|second|evening|dinner| +63307|AAAAAAAAMEHPAAAA|63307|17|35|7|PM|second|evening|dinner| +63308|AAAAAAAANEHPAAAA|63308|17|35|8|PM|second|evening|dinner| +63309|AAAAAAAAOEHPAAAA|63309|17|35|9|PM|second|evening|dinner| +63310|AAAAAAAAPEHPAAAA|63310|17|35|10|PM|second|evening|dinner| +63311|AAAAAAAAAFHPAAAA|63311|17|35|11|PM|second|evening|dinner| +63312|AAAAAAAABFHPAAAA|63312|17|35|12|PM|second|evening|dinner| +63313|AAAAAAAACFHPAAAA|63313|17|35|13|PM|second|evening|dinner| +63314|AAAAAAAADFHPAAAA|63314|17|35|14|PM|second|evening|dinner| +63315|AAAAAAAAEFHPAAAA|63315|17|35|15|PM|second|evening|dinner| +63316|AAAAAAAAFFHPAAAA|63316|17|35|16|PM|second|evening|dinner| +63317|AAAAAAAAGFHPAAAA|63317|17|35|17|PM|second|evening|dinner| +63318|AAAAAAAAHFHPAAAA|63318|17|35|18|PM|second|evening|dinner| +63319|AAAAAAAAIFHPAAAA|63319|17|35|19|PM|second|evening|dinner| +63320|AAAAAAAAJFHPAAAA|63320|17|35|20|PM|second|evening|dinner| +63321|AAAAAAAAKFHPAAAA|63321|17|35|21|PM|second|evening|dinner| +63322|AAAAAAAALFHPAAAA|63322|17|35|22|PM|second|evening|dinner| +63323|AAAAAAAAMFHPAAAA|63323|17|35|23|PM|second|evening|dinner| +63324|AAAAAAAANFHPAAAA|63324|17|35|24|PM|second|evening|dinner| +63325|AAAAAAAAOFHPAAAA|63325|17|35|25|PM|second|evening|dinner| +63326|AAAAAAAAPFHPAAAA|63326|17|35|26|PM|second|evening|dinner| +63327|AAAAAAAAAGHPAAAA|63327|17|35|27|PM|second|evening|dinner| +63328|AAAAAAAABGHPAAAA|63328|17|35|28|PM|second|evening|dinner| +63329|AAAAAAAACGHPAAAA|63329|17|35|29|PM|second|evening|dinner| +63330|AAAAAAAADGHPAAAA|63330|17|35|30|PM|second|evening|dinner| +63331|AAAAAAAAEGHPAAAA|63331|17|35|31|PM|second|evening|dinner| +63332|AAAAAAAAFGHPAAAA|63332|17|35|32|PM|second|evening|dinner| +63333|AAAAAAAAGGHPAAAA|63333|17|35|33|PM|second|evening|dinner| +63334|AAAAAAAAHGHPAAAA|63334|17|35|34|PM|second|evening|dinner| +63335|AAAAAAAAIGHPAAAA|63335|17|35|35|PM|second|evening|dinner| +63336|AAAAAAAAJGHPAAAA|63336|17|35|36|PM|second|evening|dinner| +63337|AAAAAAAAKGHPAAAA|63337|17|35|37|PM|second|evening|dinner| +63338|AAAAAAAALGHPAAAA|63338|17|35|38|PM|second|evening|dinner| +63339|AAAAAAAAMGHPAAAA|63339|17|35|39|PM|second|evening|dinner| +63340|AAAAAAAANGHPAAAA|63340|17|35|40|PM|second|evening|dinner| +63341|AAAAAAAAOGHPAAAA|63341|17|35|41|PM|second|evening|dinner| +63342|AAAAAAAAPGHPAAAA|63342|17|35|42|PM|second|evening|dinner| +63343|AAAAAAAAAHHPAAAA|63343|17|35|43|PM|second|evening|dinner| +63344|AAAAAAAABHHPAAAA|63344|17|35|44|PM|second|evening|dinner| +63345|AAAAAAAACHHPAAAA|63345|17|35|45|PM|second|evening|dinner| +63346|AAAAAAAADHHPAAAA|63346|17|35|46|PM|second|evening|dinner| +63347|AAAAAAAAEHHPAAAA|63347|17|35|47|PM|second|evening|dinner| +63348|AAAAAAAAFHHPAAAA|63348|17|35|48|PM|second|evening|dinner| +63349|AAAAAAAAGHHPAAAA|63349|17|35|49|PM|second|evening|dinner| +63350|AAAAAAAAHHHPAAAA|63350|17|35|50|PM|second|evening|dinner| +63351|AAAAAAAAIHHPAAAA|63351|17|35|51|PM|second|evening|dinner| +63352|AAAAAAAAJHHPAAAA|63352|17|35|52|PM|second|evening|dinner| +63353|AAAAAAAAKHHPAAAA|63353|17|35|53|PM|second|evening|dinner| +63354|AAAAAAAALHHPAAAA|63354|17|35|54|PM|second|evening|dinner| +63355|AAAAAAAAMHHPAAAA|63355|17|35|55|PM|second|evening|dinner| +63356|AAAAAAAANHHPAAAA|63356|17|35|56|PM|second|evening|dinner| +63357|AAAAAAAAOHHPAAAA|63357|17|35|57|PM|second|evening|dinner| +63358|AAAAAAAAPHHPAAAA|63358|17|35|58|PM|second|evening|dinner| +63359|AAAAAAAAAIHPAAAA|63359|17|35|59|PM|second|evening|dinner| +63360|AAAAAAAABIHPAAAA|63360|17|36|0|PM|second|evening|dinner| +63361|AAAAAAAACIHPAAAA|63361|17|36|1|PM|second|evening|dinner| +63362|AAAAAAAADIHPAAAA|63362|17|36|2|PM|second|evening|dinner| +63363|AAAAAAAAEIHPAAAA|63363|17|36|3|PM|second|evening|dinner| +63364|AAAAAAAAFIHPAAAA|63364|17|36|4|PM|second|evening|dinner| +63365|AAAAAAAAGIHPAAAA|63365|17|36|5|PM|second|evening|dinner| +63366|AAAAAAAAHIHPAAAA|63366|17|36|6|PM|second|evening|dinner| +63367|AAAAAAAAIIHPAAAA|63367|17|36|7|PM|second|evening|dinner| +63368|AAAAAAAAJIHPAAAA|63368|17|36|8|PM|second|evening|dinner| +63369|AAAAAAAAKIHPAAAA|63369|17|36|9|PM|second|evening|dinner| +63370|AAAAAAAALIHPAAAA|63370|17|36|10|PM|second|evening|dinner| +63371|AAAAAAAAMIHPAAAA|63371|17|36|11|PM|second|evening|dinner| +63372|AAAAAAAANIHPAAAA|63372|17|36|12|PM|second|evening|dinner| +63373|AAAAAAAAOIHPAAAA|63373|17|36|13|PM|second|evening|dinner| +63374|AAAAAAAAPIHPAAAA|63374|17|36|14|PM|second|evening|dinner| +63375|AAAAAAAAAJHPAAAA|63375|17|36|15|PM|second|evening|dinner| +63376|AAAAAAAABJHPAAAA|63376|17|36|16|PM|second|evening|dinner| +63377|AAAAAAAACJHPAAAA|63377|17|36|17|PM|second|evening|dinner| +63378|AAAAAAAADJHPAAAA|63378|17|36|18|PM|second|evening|dinner| +63379|AAAAAAAAEJHPAAAA|63379|17|36|19|PM|second|evening|dinner| +63380|AAAAAAAAFJHPAAAA|63380|17|36|20|PM|second|evening|dinner| +63381|AAAAAAAAGJHPAAAA|63381|17|36|21|PM|second|evening|dinner| +63382|AAAAAAAAHJHPAAAA|63382|17|36|22|PM|second|evening|dinner| +63383|AAAAAAAAIJHPAAAA|63383|17|36|23|PM|second|evening|dinner| +63384|AAAAAAAAJJHPAAAA|63384|17|36|24|PM|second|evening|dinner| +63385|AAAAAAAAKJHPAAAA|63385|17|36|25|PM|second|evening|dinner| +63386|AAAAAAAALJHPAAAA|63386|17|36|26|PM|second|evening|dinner| +63387|AAAAAAAAMJHPAAAA|63387|17|36|27|PM|second|evening|dinner| +63388|AAAAAAAANJHPAAAA|63388|17|36|28|PM|second|evening|dinner| +63389|AAAAAAAAOJHPAAAA|63389|17|36|29|PM|second|evening|dinner| +63390|AAAAAAAAPJHPAAAA|63390|17|36|30|PM|second|evening|dinner| +63391|AAAAAAAAAKHPAAAA|63391|17|36|31|PM|second|evening|dinner| +63392|AAAAAAAABKHPAAAA|63392|17|36|32|PM|second|evening|dinner| +63393|AAAAAAAACKHPAAAA|63393|17|36|33|PM|second|evening|dinner| +63394|AAAAAAAADKHPAAAA|63394|17|36|34|PM|second|evening|dinner| +63395|AAAAAAAAEKHPAAAA|63395|17|36|35|PM|second|evening|dinner| +63396|AAAAAAAAFKHPAAAA|63396|17|36|36|PM|second|evening|dinner| +63397|AAAAAAAAGKHPAAAA|63397|17|36|37|PM|second|evening|dinner| +63398|AAAAAAAAHKHPAAAA|63398|17|36|38|PM|second|evening|dinner| +63399|AAAAAAAAIKHPAAAA|63399|17|36|39|PM|second|evening|dinner| +63400|AAAAAAAAJKHPAAAA|63400|17|36|40|PM|second|evening|dinner| +63401|AAAAAAAAKKHPAAAA|63401|17|36|41|PM|second|evening|dinner| +63402|AAAAAAAALKHPAAAA|63402|17|36|42|PM|second|evening|dinner| +63403|AAAAAAAAMKHPAAAA|63403|17|36|43|PM|second|evening|dinner| +63404|AAAAAAAANKHPAAAA|63404|17|36|44|PM|second|evening|dinner| +63405|AAAAAAAAOKHPAAAA|63405|17|36|45|PM|second|evening|dinner| +63406|AAAAAAAAPKHPAAAA|63406|17|36|46|PM|second|evening|dinner| +63407|AAAAAAAAALHPAAAA|63407|17|36|47|PM|second|evening|dinner| +63408|AAAAAAAABLHPAAAA|63408|17|36|48|PM|second|evening|dinner| +63409|AAAAAAAACLHPAAAA|63409|17|36|49|PM|second|evening|dinner| +63410|AAAAAAAADLHPAAAA|63410|17|36|50|PM|second|evening|dinner| +63411|AAAAAAAAELHPAAAA|63411|17|36|51|PM|second|evening|dinner| +63412|AAAAAAAAFLHPAAAA|63412|17|36|52|PM|second|evening|dinner| +63413|AAAAAAAAGLHPAAAA|63413|17|36|53|PM|second|evening|dinner| +63414|AAAAAAAAHLHPAAAA|63414|17|36|54|PM|second|evening|dinner| +63415|AAAAAAAAILHPAAAA|63415|17|36|55|PM|second|evening|dinner| +63416|AAAAAAAAJLHPAAAA|63416|17|36|56|PM|second|evening|dinner| +63417|AAAAAAAAKLHPAAAA|63417|17|36|57|PM|second|evening|dinner| +63418|AAAAAAAALLHPAAAA|63418|17|36|58|PM|second|evening|dinner| +63419|AAAAAAAAMLHPAAAA|63419|17|36|59|PM|second|evening|dinner| +63420|AAAAAAAANLHPAAAA|63420|17|37|0|PM|second|evening|dinner| +63421|AAAAAAAAOLHPAAAA|63421|17|37|1|PM|second|evening|dinner| +63422|AAAAAAAAPLHPAAAA|63422|17|37|2|PM|second|evening|dinner| +63423|AAAAAAAAAMHPAAAA|63423|17|37|3|PM|second|evening|dinner| +63424|AAAAAAAABMHPAAAA|63424|17|37|4|PM|second|evening|dinner| +63425|AAAAAAAACMHPAAAA|63425|17|37|5|PM|second|evening|dinner| +63426|AAAAAAAADMHPAAAA|63426|17|37|6|PM|second|evening|dinner| +63427|AAAAAAAAEMHPAAAA|63427|17|37|7|PM|second|evening|dinner| +63428|AAAAAAAAFMHPAAAA|63428|17|37|8|PM|second|evening|dinner| +63429|AAAAAAAAGMHPAAAA|63429|17|37|9|PM|second|evening|dinner| +63430|AAAAAAAAHMHPAAAA|63430|17|37|10|PM|second|evening|dinner| +63431|AAAAAAAAIMHPAAAA|63431|17|37|11|PM|second|evening|dinner| +63432|AAAAAAAAJMHPAAAA|63432|17|37|12|PM|second|evening|dinner| +63433|AAAAAAAAKMHPAAAA|63433|17|37|13|PM|second|evening|dinner| +63434|AAAAAAAALMHPAAAA|63434|17|37|14|PM|second|evening|dinner| +63435|AAAAAAAAMMHPAAAA|63435|17|37|15|PM|second|evening|dinner| +63436|AAAAAAAANMHPAAAA|63436|17|37|16|PM|second|evening|dinner| +63437|AAAAAAAAOMHPAAAA|63437|17|37|17|PM|second|evening|dinner| +63438|AAAAAAAAPMHPAAAA|63438|17|37|18|PM|second|evening|dinner| +63439|AAAAAAAAANHPAAAA|63439|17|37|19|PM|second|evening|dinner| +63440|AAAAAAAABNHPAAAA|63440|17|37|20|PM|second|evening|dinner| +63441|AAAAAAAACNHPAAAA|63441|17|37|21|PM|second|evening|dinner| +63442|AAAAAAAADNHPAAAA|63442|17|37|22|PM|second|evening|dinner| +63443|AAAAAAAAENHPAAAA|63443|17|37|23|PM|second|evening|dinner| +63444|AAAAAAAAFNHPAAAA|63444|17|37|24|PM|second|evening|dinner| +63445|AAAAAAAAGNHPAAAA|63445|17|37|25|PM|second|evening|dinner| +63446|AAAAAAAAHNHPAAAA|63446|17|37|26|PM|second|evening|dinner| +63447|AAAAAAAAINHPAAAA|63447|17|37|27|PM|second|evening|dinner| +63448|AAAAAAAAJNHPAAAA|63448|17|37|28|PM|second|evening|dinner| +63449|AAAAAAAAKNHPAAAA|63449|17|37|29|PM|second|evening|dinner| +63450|AAAAAAAALNHPAAAA|63450|17|37|30|PM|second|evening|dinner| +63451|AAAAAAAAMNHPAAAA|63451|17|37|31|PM|second|evening|dinner| +63452|AAAAAAAANNHPAAAA|63452|17|37|32|PM|second|evening|dinner| +63453|AAAAAAAAONHPAAAA|63453|17|37|33|PM|second|evening|dinner| +63454|AAAAAAAAPNHPAAAA|63454|17|37|34|PM|second|evening|dinner| +63455|AAAAAAAAAOHPAAAA|63455|17|37|35|PM|second|evening|dinner| +63456|AAAAAAAABOHPAAAA|63456|17|37|36|PM|second|evening|dinner| +63457|AAAAAAAACOHPAAAA|63457|17|37|37|PM|second|evening|dinner| +63458|AAAAAAAADOHPAAAA|63458|17|37|38|PM|second|evening|dinner| +63459|AAAAAAAAEOHPAAAA|63459|17|37|39|PM|second|evening|dinner| +63460|AAAAAAAAFOHPAAAA|63460|17|37|40|PM|second|evening|dinner| +63461|AAAAAAAAGOHPAAAA|63461|17|37|41|PM|second|evening|dinner| +63462|AAAAAAAAHOHPAAAA|63462|17|37|42|PM|second|evening|dinner| +63463|AAAAAAAAIOHPAAAA|63463|17|37|43|PM|second|evening|dinner| +63464|AAAAAAAAJOHPAAAA|63464|17|37|44|PM|second|evening|dinner| +63465|AAAAAAAAKOHPAAAA|63465|17|37|45|PM|second|evening|dinner| +63466|AAAAAAAALOHPAAAA|63466|17|37|46|PM|second|evening|dinner| +63467|AAAAAAAAMOHPAAAA|63467|17|37|47|PM|second|evening|dinner| +63468|AAAAAAAANOHPAAAA|63468|17|37|48|PM|second|evening|dinner| +63469|AAAAAAAAOOHPAAAA|63469|17|37|49|PM|second|evening|dinner| +63470|AAAAAAAAPOHPAAAA|63470|17|37|50|PM|second|evening|dinner| +63471|AAAAAAAAAPHPAAAA|63471|17|37|51|PM|second|evening|dinner| +63472|AAAAAAAABPHPAAAA|63472|17|37|52|PM|second|evening|dinner| +63473|AAAAAAAACPHPAAAA|63473|17|37|53|PM|second|evening|dinner| +63474|AAAAAAAADPHPAAAA|63474|17|37|54|PM|second|evening|dinner| +63475|AAAAAAAAEPHPAAAA|63475|17|37|55|PM|second|evening|dinner| +63476|AAAAAAAAFPHPAAAA|63476|17|37|56|PM|second|evening|dinner| +63477|AAAAAAAAGPHPAAAA|63477|17|37|57|PM|second|evening|dinner| +63478|AAAAAAAAHPHPAAAA|63478|17|37|58|PM|second|evening|dinner| +63479|AAAAAAAAIPHPAAAA|63479|17|37|59|PM|second|evening|dinner| +63480|AAAAAAAAJPHPAAAA|63480|17|38|0|PM|second|evening|dinner| +63481|AAAAAAAAKPHPAAAA|63481|17|38|1|PM|second|evening|dinner| +63482|AAAAAAAALPHPAAAA|63482|17|38|2|PM|second|evening|dinner| +63483|AAAAAAAAMPHPAAAA|63483|17|38|3|PM|second|evening|dinner| +63484|AAAAAAAANPHPAAAA|63484|17|38|4|PM|second|evening|dinner| +63485|AAAAAAAAOPHPAAAA|63485|17|38|5|PM|second|evening|dinner| +63486|AAAAAAAAPPHPAAAA|63486|17|38|6|PM|second|evening|dinner| +63487|AAAAAAAAAAIPAAAA|63487|17|38|7|PM|second|evening|dinner| +63488|AAAAAAAABAIPAAAA|63488|17|38|8|PM|second|evening|dinner| +63489|AAAAAAAACAIPAAAA|63489|17|38|9|PM|second|evening|dinner| +63490|AAAAAAAADAIPAAAA|63490|17|38|10|PM|second|evening|dinner| +63491|AAAAAAAAEAIPAAAA|63491|17|38|11|PM|second|evening|dinner| +63492|AAAAAAAAFAIPAAAA|63492|17|38|12|PM|second|evening|dinner| +63493|AAAAAAAAGAIPAAAA|63493|17|38|13|PM|second|evening|dinner| +63494|AAAAAAAAHAIPAAAA|63494|17|38|14|PM|second|evening|dinner| +63495|AAAAAAAAIAIPAAAA|63495|17|38|15|PM|second|evening|dinner| +63496|AAAAAAAAJAIPAAAA|63496|17|38|16|PM|second|evening|dinner| +63497|AAAAAAAAKAIPAAAA|63497|17|38|17|PM|second|evening|dinner| +63498|AAAAAAAALAIPAAAA|63498|17|38|18|PM|second|evening|dinner| +63499|AAAAAAAAMAIPAAAA|63499|17|38|19|PM|second|evening|dinner| +63500|AAAAAAAANAIPAAAA|63500|17|38|20|PM|second|evening|dinner| +63501|AAAAAAAAOAIPAAAA|63501|17|38|21|PM|second|evening|dinner| +63502|AAAAAAAAPAIPAAAA|63502|17|38|22|PM|second|evening|dinner| +63503|AAAAAAAAABIPAAAA|63503|17|38|23|PM|second|evening|dinner| +63504|AAAAAAAABBIPAAAA|63504|17|38|24|PM|second|evening|dinner| +63505|AAAAAAAACBIPAAAA|63505|17|38|25|PM|second|evening|dinner| +63506|AAAAAAAADBIPAAAA|63506|17|38|26|PM|second|evening|dinner| +63507|AAAAAAAAEBIPAAAA|63507|17|38|27|PM|second|evening|dinner| +63508|AAAAAAAAFBIPAAAA|63508|17|38|28|PM|second|evening|dinner| +63509|AAAAAAAAGBIPAAAA|63509|17|38|29|PM|second|evening|dinner| +63510|AAAAAAAAHBIPAAAA|63510|17|38|30|PM|second|evening|dinner| +63511|AAAAAAAAIBIPAAAA|63511|17|38|31|PM|second|evening|dinner| +63512|AAAAAAAAJBIPAAAA|63512|17|38|32|PM|second|evening|dinner| +63513|AAAAAAAAKBIPAAAA|63513|17|38|33|PM|second|evening|dinner| +63514|AAAAAAAALBIPAAAA|63514|17|38|34|PM|second|evening|dinner| +63515|AAAAAAAAMBIPAAAA|63515|17|38|35|PM|second|evening|dinner| +63516|AAAAAAAANBIPAAAA|63516|17|38|36|PM|second|evening|dinner| +63517|AAAAAAAAOBIPAAAA|63517|17|38|37|PM|second|evening|dinner| +63518|AAAAAAAAPBIPAAAA|63518|17|38|38|PM|second|evening|dinner| +63519|AAAAAAAAACIPAAAA|63519|17|38|39|PM|second|evening|dinner| +63520|AAAAAAAABCIPAAAA|63520|17|38|40|PM|second|evening|dinner| +63521|AAAAAAAACCIPAAAA|63521|17|38|41|PM|second|evening|dinner| +63522|AAAAAAAADCIPAAAA|63522|17|38|42|PM|second|evening|dinner| +63523|AAAAAAAAECIPAAAA|63523|17|38|43|PM|second|evening|dinner| +63524|AAAAAAAAFCIPAAAA|63524|17|38|44|PM|second|evening|dinner| +63525|AAAAAAAAGCIPAAAA|63525|17|38|45|PM|second|evening|dinner| +63526|AAAAAAAAHCIPAAAA|63526|17|38|46|PM|second|evening|dinner| +63527|AAAAAAAAICIPAAAA|63527|17|38|47|PM|second|evening|dinner| +63528|AAAAAAAAJCIPAAAA|63528|17|38|48|PM|second|evening|dinner| +63529|AAAAAAAAKCIPAAAA|63529|17|38|49|PM|second|evening|dinner| +63530|AAAAAAAALCIPAAAA|63530|17|38|50|PM|second|evening|dinner| +63531|AAAAAAAAMCIPAAAA|63531|17|38|51|PM|second|evening|dinner| +63532|AAAAAAAANCIPAAAA|63532|17|38|52|PM|second|evening|dinner| +63533|AAAAAAAAOCIPAAAA|63533|17|38|53|PM|second|evening|dinner| +63534|AAAAAAAAPCIPAAAA|63534|17|38|54|PM|second|evening|dinner| +63535|AAAAAAAAADIPAAAA|63535|17|38|55|PM|second|evening|dinner| +63536|AAAAAAAABDIPAAAA|63536|17|38|56|PM|second|evening|dinner| +63537|AAAAAAAACDIPAAAA|63537|17|38|57|PM|second|evening|dinner| +63538|AAAAAAAADDIPAAAA|63538|17|38|58|PM|second|evening|dinner| +63539|AAAAAAAAEDIPAAAA|63539|17|38|59|PM|second|evening|dinner| +63540|AAAAAAAAFDIPAAAA|63540|17|39|0|PM|second|evening|dinner| +63541|AAAAAAAAGDIPAAAA|63541|17|39|1|PM|second|evening|dinner| +63542|AAAAAAAAHDIPAAAA|63542|17|39|2|PM|second|evening|dinner| +63543|AAAAAAAAIDIPAAAA|63543|17|39|3|PM|second|evening|dinner| +63544|AAAAAAAAJDIPAAAA|63544|17|39|4|PM|second|evening|dinner| +63545|AAAAAAAAKDIPAAAA|63545|17|39|5|PM|second|evening|dinner| +63546|AAAAAAAALDIPAAAA|63546|17|39|6|PM|second|evening|dinner| +63547|AAAAAAAAMDIPAAAA|63547|17|39|7|PM|second|evening|dinner| +63548|AAAAAAAANDIPAAAA|63548|17|39|8|PM|second|evening|dinner| +63549|AAAAAAAAODIPAAAA|63549|17|39|9|PM|second|evening|dinner| +63550|AAAAAAAAPDIPAAAA|63550|17|39|10|PM|second|evening|dinner| +63551|AAAAAAAAAEIPAAAA|63551|17|39|11|PM|second|evening|dinner| +63552|AAAAAAAABEIPAAAA|63552|17|39|12|PM|second|evening|dinner| +63553|AAAAAAAACEIPAAAA|63553|17|39|13|PM|second|evening|dinner| +63554|AAAAAAAADEIPAAAA|63554|17|39|14|PM|second|evening|dinner| +63555|AAAAAAAAEEIPAAAA|63555|17|39|15|PM|second|evening|dinner| +63556|AAAAAAAAFEIPAAAA|63556|17|39|16|PM|second|evening|dinner| +63557|AAAAAAAAGEIPAAAA|63557|17|39|17|PM|second|evening|dinner| +63558|AAAAAAAAHEIPAAAA|63558|17|39|18|PM|second|evening|dinner| +63559|AAAAAAAAIEIPAAAA|63559|17|39|19|PM|second|evening|dinner| +63560|AAAAAAAAJEIPAAAA|63560|17|39|20|PM|second|evening|dinner| +63561|AAAAAAAAKEIPAAAA|63561|17|39|21|PM|second|evening|dinner| +63562|AAAAAAAALEIPAAAA|63562|17|39|22|PM|second|evening|dinner| +63563|AAAAAAAAMEIPAAAA|63563|17|39|23|PM|second|evening|dinner| +63564|AAAAAAAANEIPAAAA|63564|17|39|24|PM|second|evening|dinner| +63565|AAAAAAAAOEIPAAAA|63565|17|39|25|PM|second|evening|dinner| +63566|AAAAAAAAPEIPAAAA|63566|17|39|26|PM|second|evening|dinner| +63567|AAAAAAAAAFIPAAAA|63567|17|39|27|PM|second|evening|dinner| +63568|AAAAAAAABFIPAAAA|63568|17|39|28|PM|second|evening|dinner| +63569|AAAAAAAACFIPAAAA|63569|17|39|29|PM|second|evening|dinner| +63570|AAAAAAAADFIPAAAA|63570|17|39|30|PM|second|evening|dinner| +63571|AAAAAAAAEFIPAAAA|63571|17|39|31|PM|second|evening|dinner| +63572|AAAAAAAAFFIPAAAA|63572|17|39|32|PM|second|evening|dinner| +63573|AAAAAAAAGFIPAAAA|63573|17|39|33|PM|second|evening|dinner| +63574|AAAAAAAAHFIPAAAA|63574|17|39|34|PM|second|evening|dinner| +63575|AAAAAAAAIFIPAAAA|63575|17|39|35|PM|second|evening|dinner| +63576|AAAAAAAAJFIPAAAA|63576|17|39|36|PM|second|evening|dinner| +63577|AAAAAAAAKFIPAAAA|63577|17|39|37|PM|second|evening|dinner| +63578|AAAAAAAALFIPAAAA|63578|17|39|38|PM|second|evening|dinner| +63579|AAAAAAAAMFIPAAAA|63579|17|39|39|PM|second|evening|dinner| +63580|AAAAAAAANFIPAAAA|63580|17|39|40|PM|second|evening|dinner| +63581|AAAAAAAAOFIPAAAA|63581|17|39|41|PM|second|evening|dinner| +63582|AAAAAAAAPFIPAAAA|63582|17|39|42|PM|second|evening|dinner| +63583|AAAAAAAAAGIPAAAA|63583|17|39|43|PM|second|evening|dinner| +63584|AAAAAAAABGIPAAAA|63584|17|39|44|PM|second|evening|dinner| +63585|AAAAAAAACGIPAAAA|63585|17|39|45|PM|second|evening|dinner| +63586|AAAAAAAADGIPAAAA|63586|17|39|46|PM|second|evening|dinner| +63587|AAAAAAAAEGIPAAAA|63587|17|39|47|PM|second|evening|dinner| +63588|AAAAAAAAFGIPAAAA|63588|17|39|48|PM|second|evening|dinner| +63589|AAAAAAAAGGIPAAAA|63589|17|39|49|PM|second|evening|dinner| +63590|AAAAAAAAHGIPAAAA|63590|17|39|50|PM|second|evening|dinner| +63591|AAAAAAAAIGIPAAAA|63591|17|39|51|PM|second|evening|dinner| +63592|AAAAAAAAJGIPAAAA|63592|17|39|52|PM|second|evening|dinner| +63593|AAAAAAAAKGIPAAAA|63593|17|39|53|PM|second|evening|dinner| +63594|AAAAAAAALGIPAAAA|63594|17|39|54|PM|second|evening|dinner| +63595|AAAAAAAAMGIPAAAA|63595|17|39|55|PM|second|evening|dinner| +63596|AAAAAAAANGIPAAAA|63596|17|39|56|PM|second|evening|dinner| +63597|AAAAAAAAOGIPAAAA|63597|17|39|57|PM|second|evening|dinner| +63598|AAAAAAAAPGIPAAAA|63598|17|39|58|PM|second|evening|dinner| +63599|AAAAAAAAAHIPAAAA|63599|17|39|59|PM|second|evening|dinner| +63600|AAAAAAAABHIPAAAA|63600|17|40|0|PM|second|evening|dinner| +63601|AAAAAAAACHIPAAAA|63601|17|40|1|PM|second|evening|dinner| +63602|AAAAAAAADHIPAAAA|63602|17|40|2|PM|second|evening|dinner| +63603|AAAAAAAAEHIPAAAA|63603|17|40|3|PM|second|evening|dinner| +63604|AAAAAAAAFHIPAAAA|63604|17|40|4|PM|second|evening|dinner| +63605|AAAAAAAAGHIPAAAA|63605|17|40|5|PM|second|evening|dinner| +63606|AAAAAAAAHHIPAAAA|63606|17|40|6|PM|second|evening|dinner| +63607|AAAAAAAAIHIPAAAA|63607|17|40|7|PM|second|evening|dinner| +63608|AAAAAAAAJHIPAAAA|63608|17|40|8|PM|second|evening|dinner| +63609|AAAAAAAAKHIPAAAA|63609|17|40|9|PM|second|evening|dinner| +63610|AAAAAAAALHIPAAAA|63610|17|40|10|PM|second|evening|dinner| +63611|AAAAAAAAMHIPAAAA|63611|17|40|11|PM|second|evening|dinner| +63612|AAAAAAAANHIPAAAA|63612|17|40|12|PM|second|evening|dinner| +63613|AAAAAAAAOHIPAAAA|63613|17|40|13|PM|second|evening|dinner| +63614|AAAAAAAAPHIPAAAA|63614|17|40|14|PM|second|evening|dinner| +63615|AAAAAAAAAIIPAAAA|63615|17|40|15|PM|second|evening|dinner| +63616|AAAAAAAABIIPAAAA|63616|17|40|16|PM|second|evening|dinner| +63617|AAAAAAAACIIPAAAA|63617|17|40|17|PM|second|evening|dinner| +63618|AAAAAAAADIIPAAAA|63618|17|40|18|PM|second|evening|dinner| +63619|AAAAAAAAEIIPAAAA|63619|17|40|19|PM|second|evening|dinner| +63620|AAAAAAAAFIIPAAAA|63620|17|40|20|PM|second|evening|dinner| +63621|AAAAAAAAGIIPAAAA|63621|17|40|21|PM|second|evening|dinner| +63622|AAAAAAAAHIIPAAAA|63622|17|40|22|PM|second|evening|dinner| +63623|AAAAAAAAIIIPAAAA|63623|17|40|23|PM|second|evening|dinner| +63624|AAAAAAAAJIIPAAAA|63624|17|40|24|PM|second|evening|dinner| +63625|AAAAAAAAKIIPAAAA|63625|17|40|25|PM|second|evening|dinner| +63626|AAAAAAAALIIPAAAA|63626|17|40|26|PM|second|evening|dinner| +63627|AAAAAAAAMIIPAAAA|63627|17|40|27|PM|second|evening|dinner| +63628|AAAAAAAANIIPAAAA|63628|17|40|28|PM|second|evening|dinner| +63629|AAAAAAAAOIIPAAAA|63629|17|40|29|PM|second|evening|dinner| +63630|AAAAAAAAPIIPAAAA|63630|17|40|30|PM|second|evening|dinner| +63631|AAAAAAAAAJIPAAAA|63631|17|40|31|PM|second|evening|dinner| +63632|AAAAAAAABJIPAAAA|63632|17|40|32|PM|second|evening|dinner| +63633|AAAAAAAACJIPAAAA|63633|17|40|33|PM|second|evening|dinner| +63634|AAAAAAAADJIPAAAA|63634|17|40|34|PM|second|evening|dinner| +63635|AAAAAAAAEJIPAAAA|63635|17|40|35|PM|second|evening|dinner| +63636|AAAAAAAAFJIPAAAA|63636|17|40|36|PM|second|evening|dinner| +63637|AAAAAAAAGJIPAAAA|63637|17|40|37|PM|second|evening|dinner| +63638|AAAAAAAAHJIPAAAA|63638|17|40|38|PM|second|evening|dinner| +63639|AAAAAAAAIJIPAAAA|63639|17|40|39|PM|second|evening|dinner| +63640|AAAAAAAAJJIPAAAA|63640|17|40|40|PM|second|evening|dinner| +63641|AAAAAAAAKJIPAAAA|63641|17|40|41|PM|second|evening|dinner| +63642|AAAAAAAALJIPAAAA|63642|17|40|42|PM|second|evening|dinner| +63643|AAAAAAAAMJIPAAAA|63643|17|40|43|PM|second|evening|dinner| +63644|AAAAAAAANJIPAAAA|63644|17|40|44|PM|second|evening|dinner| +63645|AAAAAAAAOJIPAAAA|63645|17|40|45|PM|second|evening|dinner| +63646|AAAAAAAAPJIPAAAA|63646|17|40|46|PM|second|evening|dinner| +63647|AAAAAAAAAKIPAAAA|63647|17|40|47|PM|second|evening|dinner| +63648|AAAAAAAABKIPAAAA|63648|17|40|48|PM|second|evening|dinner| +63649|AAAAAAAACKIPAAAA|63649|17|40|49|PM|second|evening|dinner| +63650|AAAAAAAADKIPAAAA|63650|17|40|50|PM|second|evening|dinner| +63651|AAAAAAAAEKIPAAAA|63651|17|40|51|PM|second|evening|dinner| +63652|AAAAAAAAFKIPAAAA|63652|17|40|52|PM|second|evening|dinner| +63653|AAAAAAAAGKIPAAAA|63653|17|40|53|PM|second|evening|dinner| +63654|AAAAAAAAHKIPAAAA|63654|17|40|54|PM|second|evening|dinner| +63655|AAAAAAAAIKIPAAAA|63655|17|40|55|PM|second|evening|dinner| +63656|AAAAAAAAJKIPAAAA|63656|17|40|56|PM|second|evening|dinner| +63657|AAAAAAAAKKIPAAAA|63657|17|40|57|PM|second|evening|dinner| +63658|AAAAAAAALKIPAAAA|63658|17|40|58|PM|second|evening|dinner| +63659|AAAAAAAAMKIPAAAA|63659|17|40|59|PM|second|evening|dinner| +63660|AAAAAAAANKIPAAAA|63660|17|41|0|PM|second|evening|dinner| +63661|AAAAAAAAOKIPAAAA|63661|17|41|1|PM|second|evening|dinner| +63662|AAAAAAAAPKIPAAAA|63662|17|41|2|PM|second|evening|dinner| +63663|AAAAAAAAALIPAAAA|63663|17|41|3|PM|second|evening|dinner| +63664|AAAAAAAABLIPAAAA|63664|17|41|4|PM|second|evening|dinner| +63665|AAAAAAAACLIPAAAA|63665|17|41|5|PM|second|evening|dinner| +63666|AAAAAAAADLIPAAAA|63666|17|41|6|PM|second|evening|dinner| +63667|AAAAAAAAELIPAAAA|63667|17|41|7|PM|second|evening|dinner| +63668|AAAAAAAAFLIPAAAA|63668|17|41|8|PM|second|evening|dinner| +63669|AAAAAAAAGLIPAAAA|63669|17|41|9|PM|second|evening|dinner| +63670|AAAAAAAAHLIPAAAA|63670|17|41|10|PM|second|evening|dinner| +63671|AAAAAAAAILIPAAAA|63671|17|41|11|PM|second|evening|dinner| +63672|AAAAAAAAJLIPAAAA|63672|17|41|12|PM|second|evening|dinner| +63673|AAAAAAAAKLIPAAAA|63673|17|41|13|PM|second|evening|dinner| +63674|AAAAAAAALLIPAAAA|63674|17|41|14|PM|second|evening|dinner| +63675|AAAAAAAAMLIPAAAA|63675|17|41|15|PM|second|evening|dinner| +63676|AAAAAAAANLIPAAAA|63676|17|41|16|PM|second|evening|dinner| +63677|AAAAAAAAOLIPAAAA|63677|17|41|17|PM|second|evening|dinner| +63678|AAAAAAAAPLIPAAAA|63678|17|41|18|PM|second|evening|dinner| +63679|AAAAAAAAAMIPAAAA|63679|17|41|19|PM|second|evening|dinner| +63680|AAAAAAAABMIPAAAA|63680|17|41|20|PM|second|evening|dinner| +63681|AAAAAAAACMIPAAAA|63681|17|41|21|PM|second|evening|dinner| +63682|AAAAAAAADMIPAAAA|63682|17|41|22|PM|second|evening|dinner| +63683|AAAAAAAAEMIPAAAA|63683|17|41|23|PM|second|evening|dinner| +63684|AAAAAAAAFMIPAAAA|63684|17|41|24|PM|second|evening|dinner| +63685|AAAAAAAAGMIPAAAA|63685|17|41|25|PM|second|evening|dinner| +63686|AAAAAAAAHMIPAAAA|63686|17|41|26|PM|second|evening|dinner| +63687|AAAAAAAAIMIPAAAA|63687|17|41|27|PM|second|evening|dinner| +63688|AAAAAAAAJMIPAAAA|63688|17|41|28|PM|second|evening|dinner| +63689|AAAAAAAAKMIPAAAA|63689|17|41|29|PM|second|evening|dinner| +63690|AAAAAAAALMIPAAAA|63690|17|41|30|PM|second|evening|dinner| +63691|AAAAAAAAMMIPAAAA|63691|17|41|31|PM|second|evening|dinner| +63692|AAAAAAAANMIPAAAA|63692|17|41|32|PM|second|evening|dinner| +63693|AAAAAAAAOMIPAAAA|63693|17|41|33|PM|second|evening|dinner| +63694|AAAAAAAAPMIPAAAA|63694|17|41|34|PM|second|evening|dinner| +63695|AAAAAAAAANIPAAAA|63695|17|41|35|PM|second|evening|dinner| +63696|AAAAAAAABNIPAAAA|63696|17|41|36|PM|second|evening|dinner| +63697|AAAAAAAACNIPAAAA|63697|17|41|37|PM|second|evening|dinner| +63698|AAAAAAAADNIPAAAA|63698|17|41|38|PM|second|evening|dinner| +63699|AAAAAAAAENIPAAAA|63699|17|41|39|PM|second|evening|dinner| +63700|AAAAAAAAFNIPAAAA|63700|17|41|40|PM|second|evening|dinner| +63701|AAAAAAAAGNIPAAAA|63701|17|41|41|PM|second|evening|dinner| +63702|AAAAAAAAHNIPAAAA|63702|17|41|42|PM|second|evening|dinner| +63703|AAAAAAAAINIPAAAA|63703|17|41|43|PM|second|evening|dinner| +63704|AAAAAAAAJNIPAAAA|63704|17|41|44|PM|second|evening|dinner| +63705|AAAAAAAAKNIPAAAA|63705|17|41|45|PM|second|evening|dinner| +63706|AAAAAAAALNIPAAAA|63706|17|41|46|PM|second|evening|dinner| +63707|AAAAAAAAMNIPAAAA|63707|17|41|47|PM|second|evening|dinner| +63708|AAAAAAAANNIPAAAA|63708|17|41|48|PM|second|evening|dinner| +63709|AAAAAAAAONIPAAAA|63709|17|41|49|PM|second|evening|dinner| +63710|AAAAAAAAPNIPAAAA|63710|17|41|50|PM|second|evening|dinner| +63711|AAAAAAAAAOIPAAAA|63711|17|41|51|PM|second|evening|dinner| +63712|AAAAAAAABOIPAAAA|63712|17|41|52|PM|second|evening|dinner| +63713|AAAAAAAACOIPAAAA|63713|17|41|53|PM|second|evening|dinner| +63714|AAAAAAAADOIPAAAA|63714|17|41|54|PM|second|evening|dinner| +63715|AAAAAAAAEOIPAAAA|63715|17|41|55|PM|second|evening|dinner| +63716|AAAAAAAAFOIPAAAA|63716|17|41|56|PM|second|evening|dinner| +63717|AAAAAAAAGOIPAAAA|63717|17|41|57|PM|second|evening|dinner| +63718|AAAAAAAAHOIPAAAA|63718|17|41|58|PM|second|evening|dinner| +63719|AAAAAAAAIOIPAAAA|63719|17|41|59|PM|second|evening|dinner| +63720|AAAAAAAAJOIPAAAA|63720|17|42|0|PM|second|evening|dinner| +63721|AAAAAAAAKOIPAAAA|63721|17|42|1|PM|second|evening|dinner| +63722|AAAAAAAALOIPAAAA|63722|17|42|2|PM|second|evening|dinner| +63723|AAAAAAAAMOIPAAAA|63723|17|42|3|PM|second|evening|dinner| +63724|AAAAAAAANOIPAAAA|63724|17|42|4|PM|second|evening|dinner| +63725|AAAAAAAAOOIPAAAA|63725|17|42|5|PM|second|evening|dinner| +63726|AAAAAAAAPOIPAAAA|63726|17|42|6|PM|second|evening|dinner| +63727|AAAAAAAAAPIPAAAA|63727|17|42|7|PM|second|evening|dinner| +63728|AAAAAAAABPIPAAAA|63728|17|42|8|PM|second|evening|dinner| +63729|AAAAAAAACPIPAAAA|63729|17|42|9|PM|second|evening|dinner| +63730|AAAAAAAADPIPAAAA|63730|17|42|10|PM|second|evening|dinner| +63731|AAAAAAAAEPIPAAAA|63731|17|42|11|PM|second|evening|dinner| +63732|AAAAAAAAFPIPAAAA|63732|17|42|12|PM|second|evening|dinner| +63733|AAAAAAAAGPIPAAAA|63733|17|42|13|PM|second|evening|dinner| +63734|AAAAAAAAHPIPAAAA|63734|17|42|14|PM|second|evening|dinner| +63735|AAAAAAAAIPIPAAAA|63735|17|42|15|PM|second|evening|dinner| +63736|AAAAAAAAJPIPAAAA|63736|17|42|16|PM|second|evening|dinner| +63737|AAAAAAAAKPIPAAAA|63737|17|42|17|PM|second|evening|dinner| +63738|AAAAAAAALPIPAAAA|63738|17|42|18|PM|second|evening|dinner| +63739|AAAAAAAAMPIPAAAA|63739|17|42|19|PM|second|evening|dinner| +63740|AAAAAAAANPIPAAAA|63740|17|42|20|PM|second|evening|dinner| +63741|AAAAAAAAOPIPAAAA|63741|17|42|21|PM|second|evening|dinner| +63742|AAAAAAAAPPIPAAAA|63742|17|42|22|PM|second|evening|dinner| +63743|AAAAAAAAAAJPAAAA|63743|17|42|23|PM|second|evening|dinner| +63744|AAAAAAAABAJPAAAA|63744|17|42|24|PM|second|evening|dinner| +63745|AAAAAAAACAJPAAAA|63745|17|42|25|PM|second|evening|dinner| +63746|AAAAAAAADAJPAAAA|63746|17|42|26|PM|second|evening|dinner| +63747|AAAAAAAAEAJPAAAA|63747|17|42|27|PM|second|evening|dinner| +63748|AAAAAAAAFAJPAAAA|63748|17|42|28|PM|second|evening|dinner| +63749|AAAAAAAAGAJPAAAA|63749|17|42|29|PM|second|evening|dinner| +63750|AAAAAAAAHAJPAAAA|63750|17|42|30|PM|second|evening|dinner| +63751|AAAAAAAAIAJPAAAA|63751|17|42|31|PM|second|evening|dinner| +63752|AAAAAAAAJAJPAAAA|63752|17|42|32|PM|second|evening|dinner| +63753|AAAAAAAAKAJPAAAA|63753|17|42|33|PM|second|evening|dinner| +63754|AAAAAAAALAJPAAAA|63754|17|42|34|PM|second|evening|dinner| +63755|AAAAAAAAMAJPAAAA|63755|17|42|35|PM|second|evening|dinner| +63756|AAAAAAAANAJPAAAA|63756|17|42|36|PM|second|evening|dinner| +63757|AAAAAAAAOAJPAAAA|63757|17|42|37|PM|second|evening|dinner| +63758|AAAAAAAAPAJPAAAA|63758|17|42|38|PM|second|evening|dinner| +63759|AAAAAAAAABJPAAAA|63759|17|42|39|PM|second|evening|dinner| +63760|AAAAAAAABBJPAAAA|63760|17|42|40|PM|second|evening|dinner| +63761|AAAAAAAACBJPAAAA|63761|17|42|41|PM|second|evening|dinner| +63762|AAAAAAAADBJPAAAA|63762|17|42|42|PM|second|evening|dinner| +63763|AAAAAAAAEBJPAAAA|63763|17|42|43|PM|second|evening|dinner| +63764|AAAAAAAAFBJPAAAA|63764|17|42|44|PM|second|evening|dinner| +63765|AAAAAAAAGBJPAAAA|63765|17|42|45|PM|second|evening|dinner| +63766|AAAAAAAAHBJPAAAA|63766|17|42|46|PM|second|evening|dinner| +63767|AAAAAAAAIBJPAAAA|63767|17|42|47|PM|second|evening|dinner| +63768|AAAAAAAAJBJPAAAA|63768|17|42|48|PM|second|evening|dinner| +63769|AAAAAAAAKBJPAAAA|63769|17|42|49|PM|second|evening|dinner| +63770|AAAAAAAALBJPAAAA|63770|17|42|50|PM|second|evening|dinner| +63771|AAAAAAAAMBJPAAAA|63771|17|42|51|PM|second|evening|dinner| +63772|AAAAAAAANBJPAAAA|63772|17|42|52|PM|second|evening|dinner| +63773|AAAAAAAAOBJPAAAA|63773|17|42|53|PM|second|evening|dinner| +63774|AAAAAAAAPBJPAAAA|63774|17|42|54|PM|second|evening|dinner| +63775|AAAAAAAAACJPAAAA|63775|17|42|55|PM|second|evening|dinner| +63776|AAAAAAAABCJPAAAA|63776|17|42|56|PM|second|evening|dinner| +63777|AAAAAAAACCJPAAAA|63777|17|42|57|PM|second|evening|dinner| +63778|AAAAAAAADCJPAAAA|63778|17|42|58|PM|second|evening|dinner| +63779|AAAAAAAAECJPAAAA|63779|17|42|59|PM|second|evening|dinner| +63780|AAAAAAAAFCJPAAAA|63780|17|43|0|PM|second|evening|dinner| +63781|AAAAAAAAGCJPAAAA|63781|17|43|1|PM|second|evening|dinner| +63782|AAAAAAAAHCJPAAAA|63782|17|43|2|PM|second|evening|dinner| +63783|AAAAAAAAICJPAAAA|63783|17|43|3|PM|second|evening|dinner| +63784|AAAAAAAAJCJPAAAA|63784|17|43|4|PM|second|evening|dinner| +63785|AAAAAAAAKCJPAAAA|63785|17|43|5|PM|second|evening|dinner| +63786|AAAAAAAALCJPAAAA|63786|17|43|6|PM|second|evening|dinner| +63787|AAAAAAAAMCJPAAAA|63787|17|43|7|PM|second|evening|dinner| +63788|AAAAAAAANCJPAAAA|63788|17|43|8|PM|second|evening|dinner| +63789|AAAAAAAAOCJPAAAA|63789|17|43|9|PM|second|evening|dinner| +63790|AAAAAAAAPCJPAAAA|63790|17|43|10|PM|second|evening|dinner| +63791|AAAAAAAAADJPAAAA|63791|17|43|11|PM|second|evening|dinner| +63792|AAAAAAAABDJPAAAA|63792|17|43|12|PM|second|evening|dinner| +63793|AAAAAAAACDJPAAAA|63793|17|43|13|PM|second|evening|dinner| +63794|AAAAAAAADDJPAAAA|63794|17|43|14|PM|second|evening|dinner| +63795|AAAAAAAAEDJPAAAA|63795|17|43|15|PM|second|evening|dinner| +63796|AAAAAAAAFDJPAAAA|63796|17|43|16|PM|second|evening|dinner| +63797|AAAAAAAAGDJPAAAA|63797|17|43|17|PM|second|evening|dinner| +63798|AAAAAAAAHDJPAAAA|63798|17|43|18|PM|second|evening|dinner| +63799|AAAAAAAAIDJPAAAA|63799|17|43|19|PM|second|evening|dinner| +63800|AAAAAAAAJDJPAAAA|63800|17|43|20|PM|second|evening|dinner| +63801|AAAAAAAAKDJPAAAA|63801|17|43|21|PM|second|evening|dinner| +63802|AAAAAAAALDJPAAAA|63802|17|43|22|PM|second|evening|dinner| +63803|AAAAAAAAMDJPAAAA|63803|17|43|23|PM|second|evening|dinner| +63804|AAAAAAAANDJPAAAA|63804|17|43|24|PM|second|evening|dinner| +63805|AAAAAAAAODJPAAAA|63805|17|43|25|PM|second|evening|dinner| +63806|AAAAAAAAPDJPAAAA|63806|17|43|26|PM|second|evening|dinner| +63807|AAAAAAAAAEJPAAAA|63807|17|43|27|PM|second|evening|dinner| +63808|AAAAAAAABEJPAAAA|63808|17|43|28|PM|second|evening|dinner| +63809|AAAAAAAACEJPAAAA|63809|17|43|29|PM|second|evening|dinner| +63810|AAAAAAAADEJPAAAA|63810|17|43|30|PM|second|evening|dinner| +63811|AAAAAAAAEEJPAAAA|63811|17|43|31|PM|second|evening|dinner| +63812|AAAAAAAAFEJPAAAA|63812|17|43|32|PM|second|evening|dinner| +63813|AAAAAAAAGEJPAAAA|63813|17|43|33|PM|second|evening|dinner| +63814|AAAAAAAAHEJPAAAA|63814|17|43|34|PM|second|evening|dinner| +63815|AAAAAAAAIEJPAAAA|63815|17|43|35|PM|second|evening|dinner| +63816|AAAAAAAAJEJPAAAA|63816|17|43|36|PM|second|evening|dinner| +63817|AAAAAAAAKEJPAAAA|63817|17|43|37|PM|second|evening|dinner| +63818|AAAAAAAALEJPAAAA|63818|17|43|38|PM|second|evening|dinner| +63819|AAAAAAAAMEJPAAAA|63819|17|43|39|PM|second|evening|dinner| +63820|AAAAAAAANEJPAAAA|63820|17|43|40|PM|second|evening|dinner| +63821|AAAAAAAAOEJPAAAA|63821|17|43|41|PM|second|evening|dinner| +63822|AAAAAAAAPEJPAAAA|63822|17|43|42|PM|second|evening|dinner| +63823|AAAAAAAAAFJPAAAA|63823|17|43|43|PM|second|evening|dinner| +63824|AAAAAAAABFJPAAAA|63824|17|43|44|PM|second|evening|dinner| +63825|AAAAAAAACFJPAAAA|63825|17|43|45|PM|second|evening|dinner| +63826|AAAAAAAADFJPAAAA|63826|17|43|46|PM|second|evening|dinner| +63827|AAAAAAAAEFJPAAAA|63827|17|43|47|PM|second|evening|dinner| +63828|AAAAAAAAFFJPAAAA|63828|17|43|48|PM|second|evening|dinner| +63829|AAAAAAAAGFJPAAAA|63829|17|43|49|PM|second|evening|dinner| +63830|AAAAAAAAHFJPAAAA|63830|17|43|50|PM|second|evening|dinner| +63831|AAAAAAAAIFJPAAAA|63831|17|43|51|PM|second|evening|dinner| +63832|AAAAAAAAJFJPAAAA|63832|17|43|52|PM|second|evening|dinner| +63833|AAAAAAAAKFJPAAAA|63833|17|43|53|PM|second|evening|dinner| +63834|AAAAAAAALFJPAAAA|63834|17|43|54|PM|second|evening|dinner| +63835|AAAAAAAAMFJPAAAA|63835|17|43|55|PM|second|evening|dinner| +63836|AAAAAAAANFJPAAAA|63836|17|43|56|PM|second|evening|dinner| +63837|AAAAAAAAOFJPAAAA|63837|17|43|57|PM|second|evening|dinner| +63838|AAAAAAAAPFJPAAAA|63838|17|43|58|PM|second|evening|dinner| +63839|AAAAAAAAAGJPAAAA|63839|17|43|59|PM|second|evening|dinner| +63840|AAAAAAAABGJPAAAA|63840|17|44|0|PM|second|evening|dinner| +63841|AAAAAAAACGJPAAAA|63841|17|44|1|PM|second|evening|dinner| +63842|AAAAAAAADGJPAAAA|63842|17|44|2|PM|second|evening|dinner| +63843|AAAAAAAAEGJPAAAA|63843|17|44|3|PM|second|evening|dinner| +63844|AAAAAAAAFGJPAAAA|63844|17|44|4|PM|second|evening|dinner| +63845|AAAAAAAAGGJPAAAA|63845|17|44|5|PM|second|evening|dinner| +63846|AAAAAAAAHGJPAAAA|63846|17|44|6|PM|second|evening|dinner| +63847|AAAAAAAAIGJPAAAA|63847|17|44|7|PM|second|evening|dinner| +63848|AAAAAAAAJGJPAAAA|63848|17|44|8|PM|second|evening|dinner| +63849|AAAAAAAAKGJPAAAA|63849|17|44|9|PM|second|evening|dinner| +63850|AAAAAAAALGJPAAAA|63850|17|44|10|PM|second|evening|dinner| +63851|AAAAAAAAMGJPAAAA|63851|17|44|11|PM|second|evening|dinner| +63852|AAAAAAAANGJPAAAA|63852|17|44|12|PM|second|evening|dinner| +63853|AAAAAAAAOGJPAAAA|63853|17|44|13|PM|second|evening|dinner| +63854|AAAAAAAAPGJPAAAA|63854|17|44|14|PM|second|evening|dinner| +63855|AAAAAAAAAHJPAAAA|63855|17|44|15|PM|second|evening|dinner| +63856|AAAAAAAABHJPAAAA|63856|17|44|16|PM|second|evening|dinner| +63857|AAAAAAAACHJPAAAA|63857|17|44|17|PM|second|evening|dinner| +63858|AAAAAAAADHJPAAAA|63858|17|44|18|PM|second|evening|dinner| +63859|AAAAAAAAEHJPAAAA|63859|17|44|19|PM|second|evening|dinner| +63860|AAAAAAAAFHJPAAAA|63860|17|44|20|PM|second|evening|dinner| +63861|AAAAAAAAGHJPAAAA|63861|17|44|21|PM|second|evening|dinner| +63862|AAAAAAAAHHJPAAAA|63862|17|44|22|PM|second|evening|dinner| +63863|AAAAAAAAIHJPAAAA|63863|17|44|23|PM|second|evening|dinner| +63864|AAAAAAAAJHJPAAAA|63864|17|44|24|PM|second|evening|dinner| +63865|AAAAAAAAKHJPAAAA|63865|17|44|25|PM|second|evening|dinner| +63866|AAAAAAAALHJPAAAA|63866|17|44|26|PM|second|evening|dinner| +63867|AAAAAAAAMHJPAAAA|63867|17|44|27|PM|second|evening|dinner| +63868|AAAAAAAANHJPAAAA|63868|17|44|28|PM|second|evening|dinner| +63869|AAAAAAAAOHJPAAAA|63869|17|44|29|PM|second|evening|dinner| +63870|AAAAAAAAPHJPAAAA|63870|17|44|30|PM|second|evening|dinner| +63871|AAAAAAAAAIJPAAAA|63871|17|44|31|PM|second|evening|dinner| +63872|AAAAAAAABIJPAAAA|63872|17|44|32|PM|second|evening|dinner| +63873|AAAAAAAACIJPAAAA|63873|17|44|33|PM|second|evening|dinner| +63874|AAAAAAAADIJPAAAA|63874|17|44|34|PM|second|evening|dinner| +63875|AAAAAAAAEIJPAAAA|63875|17|44|35|PM|second|evening|dinner| +63876|AAAAAAAAFIJPAAAA|63876|17|44|36|PM|second|evening|dinner| +63877|AAAAAAAAGIJPAAAA|63877|17|44|37|PM|second|evening|dinner| +63878|AAAAAAAAHIJPAAAA|63878|17|44|38|PM|second|evening|dinner| +63879|AAAAAAAAIIJPAAAA|63879|17|44|39|PM|second|evening|dinner| +63880|AAAAAAAAJIJPAAAA|63880|17|44|40|PM|second|evening|dinner| +63881|AAAAAAAAKIJPAAAA|63881|17|44|41|PM|second|evening|dinner| +63882|AAAAAAAALIJPAAAA|63882|17|44|42|PM|second|evening|dinner| +63883|AAAAAAAAMIJPAAAA|63883|17|44|43|PM|second|evening|dinner| +63884|AAAAAAAANIJPAAAA|63884|17|44|44|PM|second|evening|dinner| +63885|AAAAAAAAOIJPAAAA|63885|17|44|45|PM|second|evening|dinner| +63886|AAAAAAAAPIJPAAAA|63886|17|44|46|PM|second|evening|dinner| +63887|AAAAAAAAAJJPAAAA|63887|17|44|47|PM|second|evening|dinner| +63888|AAAAAAAABJJPAAAA|63888|17|44|48|PM|second|evening|dinner| +63889|AAAAAAAACJJPAAAA|63889|17|44|49|PM|second|evening|dinner| +63890|AAAAAAAADJJPAAAA|63890|17|44|50|PM|second|evening|dinner| +63891|AAAAAAAAEJJPAAAA|63891|17|44|51|PM|second|evening|dinner| +63892|AAAAAAAAFJJPAAAA|63892|17|44|52|PM|second|evening|dinner| +63893|AAAAAAAAGJJPAAAA|63893|17|44|53|PM|second|evening|dinner| +63894|AAAAAAAAHJJPAAAA|63894|17|44|54|PM|second|evening|dinner| +63895|AAAAAAAAIJJPAAAA|63895|17|44|55|PM|second|evening|dinner| +63896|AAAAAAAAJJJPAAAA|63896|17|44|56|PM|second|evening|dinner| +63897|AAAAAAAAKJJPAAAA|63897|17|44|57|PM|second|evening|dinner| +63898|AAAAAAAALJJPAAAA|63898|17|44|58|PM|second|evening|dinner| +63899|AAAAAAAAMJJPAAAA|63899|17|44|59|PM|second|evening|dinner| +63900|AAAAAAAANJJPAAAA|63900|17|45|0|PM|second|evening|dinner| +63901|AAAAAAAAOJJPAAAA|63901|17|45|1|PM|second|evening|dinner| +63902|AAAAAAAAPJJPAAAA|63902|17|45|2|PM|second|evening|dinner| +63903|AAAAAAAAAKJPAAAA|63903|17|45|3|PM|second|evening|dinner| +63904|AAAAAAAABKJPAAAA|63904|17|45|4|PM|second|evening|dinner| +63905|AAAAAAAACKJPAAAA|63905|17|45|5|PM|second|evening|dinner| +63906|AAAAAAAADKJPAAAA|63906|17|45|6|PM|second|evening|dinner| +63907|AAAAAAAAEKJPAAAA|63907|17|45|7|PM|second|evening|dinner| +63908|AAAAAAAAFKJPAAAA|63908|17|45|8|PM|second|evening|dinner| +63909|AAAAAAAAGKJPAAAA|63909|17|45|9|PM|second|evening|dinner| +63910|AAAAAAAAHKJPAAAA|63910|17|45|10|PM|second|evening|dinner| +63911|AAAAAAAAIKJPAAAA|63911|17|45|11|PM|second|evening|dinner| +63912|AAAAAAAAJKJPAAAA|63912|17|45|12|PM|second|evening|dinner| +63913|AAAAAAAAKKJPAAAA|63913|17|45|13|PM|second|evening|dinner| +63914|AAAAAAAALKJPAAAA|63914|17|45|14|PM|second|evening|dinner| +63915|AAAAAAAAMKJPAAAA|63915|17|45|15|PM|second|evening|dinner| +63916|AAAAAAAANKJPAAAA|63916|17|45|16|PM|second|evening|dinner| +63917|AAAAAAAAOKJPAAAA|63917|17|45|17|PM|second|evening|dinner| +63918|AAAAAAAAPKJPAAAA|63918|17|45|18|PM|second|evening|dinner| +63919|AAAAAAAAALJPAAAA|63919|17|45|19|PM|second|evening|dinner| +63920|AAAAAAAABLJPAAAA|63920|17|45|20|PM|second|evening|dinner| +63921|AAAAAAAACLJPAAAA|63921|17|45|21|PM|second|evening|dinner| +63922|AAAAAAAADLJPAAAA|63922|17|45|22|PM|second|evening|dinner| +63923|AAAAAAAAELJPAAAA|63923|17|45|23|PM|second|evening|dinner| +63924|AAAAAAAAFLJPAAAA|63924|17|45|24|PM|second|evening|dinner| +63925|AAAAAAAAGLJPAAAA|63925|17|45|25|PM|second|evening|dinner| +63926|AAAAAAAAHLJPAAAA|63926|17|45|26|PM|second|evening|dinner| +63927|AAAAAAAAILJPAAAA|63927|17|45|27|PM|second|evening|dinner| +63928|AAAAAAAAJLJPAAAA|63928|17|45|28|PM|second|evening|dinner| +63929|AAAAAAAAKLJPAAAA|63929|17|45|29|PM|second|evening|dinner| +63930|AAAAAAAALLJPAAAA|63930|17|45|30|PM|second|evening|dinner| +63931|AAAAAAAAMLJPAAAA|63931|17|45|31|PM|second|evening|dinner| +63932|AAAAAAAANLJPAAAA|63932|17|45|32|PM|second|evening|dinner| +63933|AAAAAAAAOLJPAAAA|63933|17|45|33|PM|second|evening|dinner| +63934|AAAAAAAAPLJPAAAA|63934|17|45|34|PM|second|evening|dinner| +63935|AAAAAAAAAMJPAAAA|63935|17|45|35|PM|second|evening|dinner| +63936|AAAAAAAABMJPAAAA|63936|17|45|36|PM|second|evening|dinner| +63937|AAAAAAAACMJPAAAA|63937|17|45|37|PM|second|evening|dinner| +63938|AAAAAAAADMJPAAAA|63938|17|45|38|PM|second|evening|dinner| +63939|AAAAAAAAEMJPAAAA|63939|17|45|39|PM|second|evening|dinner| +63940|AAAAAAAAFMJPAAAA|63940|17|45|40|PM|second|evening|dinner| +63941|AAAAAAAAGMJPAAAA|63941|17|45|41|PM|second|evening|dinner| +63942|AAAAAAAAHMJPAAAA|63942|17|45|42|PM|second|evening|dinner| +63943|AAAAAAAAIMJPAAAA|63943|17|45|43|PM|second|evening|dinner| +63944|AAAAAAAAJMJPAAAA|63944|17|45|44|PM|second|evening|dinner| +63945|AAAAAAAAKMJPAAAA|63945|17|45|45|PM|second|evening|dinner| +63946|AAAAAAAALMJPAAAA|63946|17|45|46|PM|second|evening|dinner| +63947|AAAAAAAAMMJPAAAA|63947|17|45|47|PM|second|evening|dinner| +63948|AAAAAAAANMJPAAAA|63948|17|45|48|PM|second|evening|dinner| +63949|AAAAAAAAOMJPAAAA|63949|17|45|49|PM|second|evening|dinner| +63950|AAAAAAAAPMJPAAAA|63950|17|45|50|PM|second|evening|dinner| +63951|AAAAAAAAANJPAAAA|63951|17|45|51|PM|second|evening|dinner| +63952|AAAAAAAABNJPAAAA|63952|17|45|52|PM|second|evening|dinner| +63953|AAAAAAAACNJPAAAA|63953|17|45|53|PM|second|evening|dinner| +63954|AAAAAAAADNJPAAAA|63954|17|45|54|PM|second|evening|dinner| +63955|AAAAAAAAENJPAAAA|63955|17|45|55|PM|second|evening|dinner| +63956|AAAAAAAAFNJPAAAA|63956|17|45|56|PM|second|evening|dinner| +63957|AAAAAAAAGNJPAAAA|63957|17|45|57|PM|second|evening|dinner| +63958|AAAAAAAAHNJPAAAA|63958|17|45|58|PM|second|evening|dinner| +63959|AAAAAAAAINJPAAAA|63959|17|45|59|PM|second|evening|dinner| +63960|AAAAAAAAJNJPAAAA|63960|17|46|0|PM|second|evening|dinner| +63961|AAAAAAAAKNJPAAAA|63961|17|46|1|PM|second|evening|dinner| +63962|AAAAAAAALNJPAAAA|63962|17|46|2|PM|second|evening|dinner| +63963|AAAAAAAAMNJPAAAA|63963|17|46|3|PM|second|evening|dinner| +63964|AAAAAAAANNJPAAAA|63964|17|46|4|PM|second|evening|dinner| +63965|AAAAAAAAONJPAAAA|63965|17|46|5|PM|second|evening|dinner| +63966|AAAAAAAAPNJPAAAA|63966|17|46|6|PM|second|evening|dinner| +63967|AAAAAAAAAOJPAAAA|63967|17|46|7|PM|second|evening|dinner| +63968|AAAAAAAABOJPAAAA|63968|17|46|8|PM|second|evening|dinner| +63969|AAAAAAAACOJPAAAA|63969|17|46|9|PM|second|evening|dinner| +63970|AAAAAAAADOJPAAAA|63970|17|46|10|PM|second|evening|dinner| +63971|AAAAAAAAEOJPAAAA|63971|17|46|11|PM|second|evening|dinner| +63972|AAAAAAAAFOJPAAAA|63972|17|46|12|PM|second|evening|dinner| +63973|AAAAAAAAGOJPAAAA|63973|17|46|13|PM|second|evening|dinner| +63974|AAAAAAAAHOJPAAAA|63974|17|46|14|PM|second|evening|dinner| +63975|AAAAAAAAIOJPAAAA|63975|17|46|15|PM|second|evening|dinner| +63976|AAAAAAAAJOJPAAAA|63976|17|46|16|PM|second|evening|dinner| +63977|AAAAAAAAKOJPAAAA|63977|17|46|17|PM|second|evening|dinner| +63978|AAAAAAAALOJPAAAA|63978|17|46|18|PM|second|evening|dinner| +63979|AAAAAAAAMOJPAAAA|63979|17|46|19|PM|second|evening|dinner| +63980|AAAAAAAANOJPAAAA|63980|17|46|20|PM|second|evening|dinner| +63981|AAAAAAAAOOJPAAAA|63981|17|46|21|PM|second|evening|dinner| +63982|AAAAAAAAPOJPAAAA|63982|17|46|22|PM|second|evening|dinner| +63983|AAAAAAAAAPJPAAAA|63983|17|46|23|PM|second|evening|dinner| +63984|AAAAAAAABPJPAAAA|63984|17|46|24|PM|second|evening|dinner| +63985|AAAAAAAACPJPAAAA|63985|17|46|25|PM|second|evening|dinner| +63986|AAAAAAAADPJPAAAA|63986|17|46|26|PM|second|evening|dinner| +63987|AAAAAAAAEPJPAAAA|63987|17|46|27|PM|second|evening|dinner| +63988|AAAAAAAAFPJPAAAA|63988|17|46|28|PM|second|evening|dinner| +63989|AAAAAAAAGPJPAAAA|63989|17|46|29|PM|second|evening|dinner| +63990|AAAAAAAAHPJPAAAA|63990|17|46|30|PM|second|evening|dinner| +63991|AAAAAAAAIPJPAAAA|63991|17|46|31|PM|second|evening|dinner| +63992|AAAAAAAAJPJPAAAA|63992|17|46|32|PM|second|evening|dinner| +63993|AAAAAAAAKPJPAAAA|63993|17|46|33|PM|second|evening|dinner| +63994|AAAAAAAALPJPAAAA|63994|17|46|34|PM|second|evening|dinner| +63995|AAAAAAAAMPJPAAAA|63995|17|46|35|PM|second|evening|dinner| +63996|AAAAAAAANPJPAAAA|63996|17|46|36|PM|second|evening|dinner| +63997|AAAAAAAAOPJPAAAA|63997|17|46|37|PM|second|evening|dinner| +63998|AAAAAAAAPPJPAAAA|63998|17|46|38|PM|second|evening|dinner| +63999|AAAAAAAAAAKPAAAA|63999|17|46|39|PM|second|evening|dinner| +64000|AAAAAAAABAKPAAAA|64000|17|46|40|PM|second|evening|dinner| +64001|AAAAAAAACAKPAAAA|64001|17|46|41|PM|second|evening|dinner| +64002|AAAAAAAADAKPAAAA|64002|17|46|42|PM|second|evening|dinner| +64003|AAAAAAAAEAKPAAAA|64003|17|46|43|PM|second|evening|dinner| +64004|AAAAAAAAFAKPAAAA|64004|17|46|44|PM|second|evening|dinner| +64005|AAAAAAAAGAKPAAAA|64005|17|46|45|PM|second|evening|dinner| +64006|AAAAAAAAHAKPAAAA|64006|17|46|46|PM|second|evening|dinner| +64007|AAAAAAAAIAKPAAAA|64007|17|46|47|PM|second|evening|dinner| +64008|AAAAAAAAJAKPAAAA|64008|17|46|48|PM|second|evening|dinner| +64009|AAAAAAAAKAKPAAAA|64009|17|46|49|PM|second|evening|dinner| +64010|AAAAAAAALAKPAAAA|64010|17|46|50|PM|second|evening|dinner| +64011|AAAAAAAAMAKPAAAA|64011|17|46|51|PM|second|evening|dinner| +64012|AAAAAAAANAKPAAAA|64012|17|46|52|PM|second|evening|dinner| +64013|AAAAAAAAOAKPAAAA|64013|17|46|53|PM|second|evening|dinner| +64014|AAAAAAAAPAKPAAAA|64014|17|46|54|PM|second|evening|dinner| +64015|AAAAAAAAABKPAAAA|64015|17|46|55|PM|second|evening|dinner| +64016|AAAAAAAABBKPAAAA|64016|17|46|56|PM|second|evening|dinner| +64017|AAAAAAAACBKPAAAA|64017|17|46|57|PM|second|evening|dinner| +64018|AAAAAAAADBKPAAAA|64018|17|46|58|PM|second|evening|dinner| +64019|AAAAAAAAEBKPAAAA|64019|17|46|59|PM|second|evening|dinner| +64020|AAAAAAAAFBKPAAAA|64020|17|47|0|PM|second|evening|dinner| +64021|AAAAAAAAGBKPAAAA|64021|17|47|1|PM|second|evening|dinner| +64022|AAAAAAAAHBKPAAAA|64022|17|47|2|PM|second|evening|dinner| +64023|AAAAAAAAIBKPAAAA|64023|17|47|3|PM|second|evening|dinner| +64024|AAAAAAAAJBKPAAAA|64024|17|47|4|PM|second|evening|dinner| +64025|AAAAAAAAKBKPAAAA|64025|17|47|5|PM|second|evening|dinner| +64026|AAAAAAAALBKPAAAA|64026|17|47|6|PM|second|evening|dinner| +64027|AAAAAAAAMBKPAAAA|64027|17|47|7|PM|second|evening|dinner| +64028|AAAAAAAANBKPAAAA|64028|17|47|8|PM|second|evening|dinner| +64029|AAAAAAAAOBKPAAAA|64029|17|47|9|PM|second|evening|dinner| +64030|AAAAAAAAPBKPAAAA|64030|17|47|10|PM|second|evening|dinner| +64031|AAAAAAAAACKPAAAA|64031|17|47|11|PM|second|evening|dinner| +64032|AAAAAAAABCKPAAAA|64032|17|47|12|PM|second|evening|dinner| +64033|AAAAAAAACCKPAAAA|64033|17|47|13|PM|second|evening|dinner| +64034|AAAAAAAADCKPAAAA|64034|17|47|14|PM|second|evening|dinner| +64035|AAAAAAAAECKPAAAA|64035|17|47|15|PM|second|evening|dinner| +64036|AAAAAAAAFCKPAAAA|64036|17|47|16|PM|second|evening|dinner| +64037|AAAAAAAAGCKPAAAA|64037|17|47|17|PM|second|evening|dinner| +64038|AAAAAAAAHCKPAAAA|64038|17|47|18|PM|second|evening|dinner| +64039|AAAAAAAAICKPAAAA|64039|17|47|19|PM|second|evening|dinner| +64040|AAAAAAAAJCKPAAAA|64040|17|47|20|PM|second|evening|dinner| +64041|AAAAAAAAKCKPAAAA|64041|17|47|21|PM|second|evening|dinner| +64042|AAAAAAAALCKPAAAA|64042|17|47|22|PM|second|evening|dinner| +64043|AAAAAAAAMCKPAAAA|64043|17|47|23|PM|second|evening|dinner| +64044|AAAAAAAANCKPAAAA|64044|17|47|24|PM|second|evening|dinner| +64045|AAAAAAAAOCKPAAAA|64045|17|47|25|PM|second|evening|dinner| +64046|AAAAAAAAPCKPAAAA|64046|17|47|26|PM|second|evening|dinner| +64047|AAAAAAAAADKPAAAA|64047|17|47|27|PM|second|evening|dinner| +64048|AAAAAAAABDKPAAAA|64048|17|47|28|PM|second|evening|dinner| +64049|AAAAAAAACDKPAAAA|64049|17|47|29|PM|second|evening|dinner| +64050|AAAAAAAADDKPAAAA|64050|17|47|30|PM|second|evening|dinner| +64051|AAAAAAAAEDKPAAAA|64051|17|47|31|PM|second|evening|dinner| +64052|AAAAAAAAFDKPAAAA|64052|17|47|32|PM|second|evening|dinner| +64053|AAAAAAAAGDKPAAAA|64053|17|47|33|PM|second|evening|dinner| +64054|AAAAAAAAHDKPAAAA|64054|17|47|34|PM|second|evening|dinner| +64055|AAAAAAAAIDKPAAAA|64055|17|47|35|PM|second|evening|dinner| +64056|AAAAAAAAJDKPAAAA|64056|17|47|36|PM|second|evening|dinner| +64057|AAAAAAAAKDKPAAAA|64057|17|47|37|PM|second|evening|dinner| +64058|AAAAAAAALDKPAAAA|64058|17|47|38|PM|second|evening|dinner| +64059|AAAAAAAAMDKPAAAA|64059|17|47|39|PM|second|evening|dinner| +64060|AAAAAAAANDKPAAAA|64060|17|47|40|PM|second|evening|dinner| +64061|AAAAAAAAODKPAAAA|64061|17|47|41|PM|second|evening|dinner| +64062|AAAAAAAAPDKPAAAA|64062|17|47|42|PM|second|evening|dinner| +64063|AAAAAAAAAEKPAAAA|64063|17|47|43|PM|second|evening|dinner| +64064|AAAAAAAABEKPAAAA|64064|17|47|44|PM|second|evening|dinner| +64065|AAAAAAAACEKPAAAA|64065|17|47|45|PM|second|evening|dinner| +64066|AAAAAAAADEKPAAAA|64066|17|47|46|PM|second|evening|dinner| +64067|AAAAAAAAEEKPAAAA|64067|17|47|47|PM|second|evening|dinner| +64068|AAAAAAAAFEKPAAAA|64068|17|47|48|PM|second|evening|dinner| +64069|AAAAAAAAGEKPAAAA|64069|17|47|49|PM|second|evening|dinner| +64070|AAAAAAAAHEKPAAAA|64070|17|47|50|PM|second|evening|dinner| +64071|AAAAAAAAIEKPAAAA|64071|17|47|51|PM|second|evening|dinner| +64072|AAAAAAAAJEKPAAAA|64072|17|47|52|PM|second|evening|dinner| +64073|AAAAAAAAKEKPAAAA|64073|17|47|53|PM|second|evening|dinner| +64074|AAAAAAAALEKPAAAA|64074|17|47|54|PM|second|evening|dinner| +64075|AAAAAAAAMEKPAAAA|64075|17|47|55|PM|second|evening|dinner| +64076|AAAAAAAANEKPAAAA|64076|17|47|56|PM|second|evening|dinner| +64077|AAAAAAAAOEKPAAAA|64077|17|47|57|PM|second|evening|dinner| +64078|AAAAAAAAPEKPAAAA|64078|17|47|58|PM|second|evening|dinner| +64079|AAAAAAAAAFKPAAAA|64079|17|47|59|PM|second|evening|dinner| +64080|AAAAAAAABFKPAAAA|64080|17|48|0|PM|second|evening|dinner| +64081|AAAAAAAACFKPAAAA|64081|17|48|1|PM|second|evening|dinner| +64082|AAAAAAAADFKPAAAA|64082|17|48|2|PM|second|evening|dinner| +64083|AAAAAAAAEFKPAAAA|64083|17|48|3|PM|second|evening|dinner| +64084|AAAAAAAAFFKPAAAA|64084|17|48|4|PM|second|evening|dinner| +64085|AAAAAAAAGFKPAAAA|64085|17|48|5|PM|second|evening|dinner| +64086|AAAAAAAAHFKPAAAA|64086|17|48|6|PM|second|evening|dinner| +64087|AAAAAAAAIFKPAAAA|64087|17|48|7|PM|second|evening|dinner| +64088|AAAAAAAAJFKPAAAA|64088|17|48|8|PM|second|evening|dinner| +64089|AAAAAAAAKFKPAAAA|64089|17|48|9|PM|second|evening|dinner| +64090|AAAAAAAALFKPAAAA|64090|17|48|10|PM|second|evening|dinner| +64091|AAAAAAAAMFKPAAAA|64091|17|48|11|PM|second|evening|dinner| +64092|AAAAAAAANFKPAAAA|64092|17|48|12|PM|second|evening|dinner| +64093|AAAAAAAAOFKPAAAA|64093|17|48|13|PM|second|evening|dinner| +64094|AAAAAAAAPFKPAAAA|64094|17|48|14|PM|second|evening|dinner| +64095|AAAAAAAAAGKPAAAA|64095|17|48|15|PM|second|evening|dinner| +64096|AAAAAAAABGKPAAAA|64096|17|48|16|PM|second|evening|dinner| +64097|AAAAAAAACGKPAAAA|64097|17|48|17|PM|second|evening|dinner| +64098|AAAAAAAADGKPAAAA|64098|17|48|18|PM|second|evening|dinner| +64099|AAAAAAAAEGKPAAAA|64099|17|48|19|PM|second|evening|dinner| +64100|AAAAAAAAFGKPAAAA|64100|17|48|20|PM|second|evening|dinner| +64101|AAAAAAAAGGKPAAAA|64101|17|48|21|PM|second|evening|dinner| +64102|AAAAAAAAHGKPAAAA|64102|17|48|22|PM|second|evening|dinner| +64103|AAAAAAAAIGKPAAAA|64103|17|48|23|PM|second|evening|dinner| +64104|AAAAAAAAJGKPAAAA|64104|17|48|24|PM|second|evening|dinner| +64105|AAAAAAAAKGKPAAAA|64105|17|48|25|PM|second|evening|dinner| +64106|AAAAAAAALGKPAAAA|64106|17|48|26|PM|second|evening|dinner| +64107|AAAAAAAAMGKPAAAA|64107|17|48|27|PM|second|evening|dinner| +64108|AAAAAAAANGKPAAAA|64108|17|48|28|PM|second|evening|dinner| +64109|AAAAAAAAOGKPAAAA|64109|17|48|29|PM|second|evening|dinner| +64110|AAAAAAAAPGKPAAAA|64110|17|48|30|PM|second|evening|dinner| +64111|AAAAAAAAAHKPAAAA|64111|17|48|31|PM|second|evening|dinner| +64112|AAAAAAAABHKPAAAA|64112|17|48|32|PM|second|evening|dinner| +64113|AAAAAAAACHKPAAAA|64113|17|48|33|PM|second|evening|dinner| +64114|AAAAAAAADHKPAAAA|64114|17|48|34|PM|second|evening|dinner| +64115|AAAAAAAAEHKPAAAA|64115|17|48|35|PM|second|evening|dinner| +64116|AAAAAAAAFHKPAAAA|64116|17|48|36|PM|second|evening|dinner| +64117|AAAAAAAAGHKPAAAA|64117|17|48|37|PM|second|evening|dinner| +64118|AAAAAAAAHHKPAAAA|64118|17|48|38|PM|second|evening|dinner| +64119|AAAAAAAAIHKPAAAA|64119|17|48|39|PM|second|evening|dinner| +64120|AAAAAAAAJHKPAAAA|64120|17|48|40|PM|second|evening|dinner| +64121|AAAAAAAAKHKPAAAA|64121|17|48|41|PM|second|evening|dinner| +64122|AAAAAAAALHKPAAAA|64122|17|48|42|PM|second|evening|dinner| +64123|AAAAAAAAMHKPAAAA|64123|17|48|43|PM|second|evening|dinner| +64124|AAAAAAAANHKPAAAA|64124|17|48|44|PM|second|evening|dinner| +64125|AAAAAAAAOHKPAAAA|64125|17|48|45|PM|second|evening|dinner| +64126|AAAAAAAAPHKPAAAA|64126|17|48|46|PM|second|evening|dinner| +64127|AAAAAAAAAIKPAAAA|64127|17|48|47|PM|second|evening|dinner| +64128|AAAAAAAABIKPAAAA|64128|17|48|48|PM|second|evening|dinner| +64129|AAAAAAAACIKPAAAA|64129|17|48|49|PM|second|evening|dinner| +64130|AAAAAAAADIKPAAAA|64130|17|48|50|PM|second|evening|dinner| +64131|AAAAAAAAEIKPAAAA|64131|17|48|51|PM|second|evening|dinner| +64132|AAAAAAAAFIKPAAAA|64132|17|48|52|PM|second|evening|dinner| +64133|AAAAAAAAGIKPAAAA|64133|17|48|53|PM|second|evening|dinner| +64134|AAAAAAAAHIKPAAAA|64134|17|48|54|PM|second|evening|dinner| +64135|AAAAAAAAIIKPAAAA|64135|17|48|55|PM|second|evening|dinner| +64136|AAAAAAAAJIKPAAAA|64136|17|48|56|PM|second|evening|dinner| +64137|AAAAAAAAKIKPAAAA|64137|17|48|57|PM|second|evening|dinner| +64138|AAAAAAAALIKPAAAA|64138|17|48|58|PM|second|evening|dinner| +64139|AAAAAAAAMIKPAAAA|64139|17|48|59|PM|second|evening|dinner| +64140|AAAAAAAANIKPAAAA|64140|17|49|0|PM|second|evening|dinner| +64141|AAAAAAAAOIKPAAAA|64141|17|49|1|PM|second|evening|dinner| +64142|AAAAAAAAPIKPAAAA|64142|17|49|2|PM|second|evening|dinner| +64143|AAAAAAAAAJKPAAAA|64143|17|49|3|PM|second|evening|dinner| +64144|AAAAAAAABJKPAAAA|64144|17|49|4|PM|second|evening|dinner| +64145|AAAAAAAACJKPAAAA|64145|17|49|5|PM|second|evening|dinner| +64146|AAAAAAAADJKPAAAA|64146|17|49|6|PM|second|evening|dinner| +64147|AAAAAAAAEJKPAAAA|64147|17|49|7|PM|second|evening|dinner| +64148|AAAAAAAAFJKPAAAA|64148|17|49|8|PM|second|evening|dinner| +64149|AAAAAAAAGJKPAAAA|64149|17|49|9|PM|second|evening|dinner| +64150|AAAAAAAAHJKPAAAA|64150|17|49|10|PM|second|evening|dinner| +64151|AAAAAAAAIJKPAAAA|64151|17|49|11|PM|second|evening|dinner| +64152|AAAAAAAAJJKPAAAA|64152|17|49|12|PM|second|evening|dinner| +64153|AAAAAAAAKJKPAAAA|64153|17|49|13|PM|second|evening|dinner| +64154|AAAAAAAALJKPAAAA|64154|17|49|14|PM|second|evening|dinner| +64155|AAAAAAAAMJKPAAAA|64155|17|49|15|PM|second|evening|dinner| +64156|AAAAAAAANJKPAAAA|64156|17|49|16|PM|second|evening|dinner| +64157|AAAAAAAAOJKPAAAA|64157|17|49|17|PM|second|evening|dinner| +64158|AAAAAAAAPJKPAAAA|64158|17|49|18|PM|second|evening|dinner| +64159|AAAAAAAAAKKPAAAA|64159|17|49|19|PM|second|evening|dinner| +64160|AAAAAAAABKKPAAAA|64160|17|49|20|PM|second|evening|dinner| +64161|AAAAAAAACKKPAAAA|64161|17|49|21|PM|second|evening|dinner| +64162|AAAAAAAADKKPAAAA|64162|17|49|22|PM|second|evening|dinner| +64163|AAAAAAAAEKKPAAAA|64163|17|49|23|PM|second|evening|dinner| +64164|AAAAAAAAFKKPAAAA|64164|17|49|24|PM|second|evening|dinner| +64165|AAAAAAAAGKKPAAAA|64165|17|49|25|PM|second|evening|dinner| +64166|AAAAAAAAHKKPAAAA|64166|17|49|26|PM|second|evening|dinner| +64167|AAAAAAAAIKKPAAAA|64167|17|49|27|PM|second|evening|dinner| +64168|AAAAAAAAJKKPAAAA|64168|17|49|28|PM|second|evening|dinner| +64169|AAAAAAAAKKKPAAAA|64169|17|49|29|PM|second|evening|dinner| +64170|AAAAAAAALKKPAAAA|64170|17|49|30|PM|second|evening|dinner| +64171|AAAAAAAAMKKPAAAA|64171|17|49|31|PM|second|evening|dinner| +64172|AAAAAAAANKKPAAAA|64172|17|49|32|PM|second|evening|dinner| +64173|AAAAAAAAOKKPAAAA|64173|17|49|33|PM|second|evening|dinner| +64174|AAAAAAAAPKKPAAAA|64174|17|49|34|PM|second|evening|dinner| +64175|AAAAAAAAALKPAAAA|64175|17|49|35|PM|second|evening|dinner| +64176|AAAAAAAABLKPAAAA|64176|17|49|36|PM|second|evening|dinner| +64177|AAAAAAAACLKPAAAA|64177|17|49|37|PM|second|evening|dinner| +64178|AAAAAAAADLKPAAAA|64178|17|49|38|PM|second|evening|dinner| +64179|AAAAAAAAELKPAAAA|64179|17|49|39|PM|second|evening|dinner| +64180|AAAAAAAAFLKPAAAA|64180|17|49|40|PM|second|evening|dinner| +64181|AAAAAAAAGLKPAAAA|64181|17|49|41|PM|second|evening|dinner| +64182|AAAAAAAAHLKPAAAA|64182|17|49|42|PM|second|evening|dinner| +64183|AAAAAAAAILKPAAAA|64183|17|49|43|PM|second|evening|dinner| +64184|AAAAAAAAJLKPAAAA|64184|17|49|44|PM|second|evening|dinner| +64185|AAAAAAAAKLKPAAAA|64185|17|49|45|PM|second|evening|dinner| +64186|AAAAAAAALLKPAAAA|64186|17|49|46|PM|second|evening|dinner| +64187|AAAAAAAAMLKPAAAA|64187|17|49|47|PM|second|evening|dinner| +64188|AAAAAAAANLKPAAAA|64188|17|49|48|PM|second|evening|dinner| +64189|AAAAAAAAOLKPAAAA|64189|17|49|49|PM|second|evening|dinner| +64190|AAAAAAAAPLKPAAAA|64190|17|49|50|PM|second|evening|dinner| +64191|AAAAAAAAAMKPAAAA|64191|17|49|51|PM|second|evening|dinner| +64192|AAAAAAAABMKPAAAA|64192|17|49|52|PM|second|evening|dinner| +64193|AAAAAAAACMKPAAAA|64193|17|49|53|PM|second|evening|dinner| +64194|AAAAAAAADMKPAAAA|64194|17|49|54|PM|second|evening|dinner| +64195|AAAAAAAAEMKPAAAA|64195|17|49|55|PM|second|evening|dinner| +64196|AAAAAAAAFMKPAAAA|64196|17|49|56|PM|second|evening|dinner| +64197|AAAAAAAAGMKPAAAA|64197|17|49|57|PM|second|evening|dinner| +64198|AAAAAAAAHMKPAAAA|64198|17|49|58|PM|second|evening|dinner| +64199|AAAAAAAAIMKPAAAA|64199|17|49|59|PM|second|evening|dinner| +64200|AAAAAAAAJMKPAAAA|64200|17|50|0|PM|second|evening|dinner| +64201|AAAAAAAAKMKPAAAA|64201|17|50|1|PM|second|evening|dinner| +64202|AAAAAAAALMKPAAAA|64202|17|50|2|PM|second|evening|dinner| +64203|AAAAAAAAMMKPAAAA|64203|17|50|3|PM|second|evening|dinner| +64204|AAAAAAAANMKPAAAA|64204|17|50|4|PM|second|evening|dinner| +64205|AAAAAAAAOMKPAAAA|64205|17|50|5|PM|second|evening|dinner| +64206|AAAAAAAAPMKPAAAA|64206|17|50|6|PM|second|evening|dinner| +64207|AAAAAAAAANKPAAAA|64207|17|50|7|PM|second|evening|dinner| +64208|AAAAAAAABNKPAAAA|64208|17|50|8|PM|second|evening|dinner| +64209|AAAAAAAACNKPAAAA|64209|17|50|9|PM|second|evening|dinner| +64210|AAAAAAAADNKPAAAA|64210|17|50|10|PM|second|evening|dinner| +64211|AAAAAAAAENKPAAAA|64211|17|50|11|PM|second|evening|dinner| +64212|AAAAAAAAFNKPAAAA|64212|17|50|12|PM|second|evening|dinner| +64213|AAAAAAAAGNKPAAAA|64213|17|50|13|PM|second|evening|dinner| +64214|AAAAAAAAHNKPAAAA|64214|17|50|14|PM|second|evening|dinner| +64215|AAAAAAAAINKPAAAA|64215|17|50|15|PM|second|evening|dinner| +64216|AAAAAAAAJNKPAAAA|64216|17|50|16|PM|second|evening|dinner| +64217|AAAAAAAAKNKPAAAA|64217|17|50|17|PM|second|evening|dinner| +64218|AAAAAAAALNKPAAAA|64218|17|50|18|PM|second|evening|dinner| +64219|AAAAAAAAMNKPAAAA|64219|17|50|19|PM|second|evening|dinner| +64220|AAAAAAAANNKPAAAA|64220|17|50|20|PM|second|evening|dinner| +64221|AAAAAAAAONKPAAAA|64221|17|50|21|PM|second|evening|dinner| +64222|AAAAAAAAPNKPAAAA|64222|17|50|22|PM|second|evening|dinner| +64223|AAAAAAAAAOKPAAAA|64223|17|50|23|PM|second|evening|dinner| +64224|AAAAAAAABOKPAAAA|64224|17|50|24|PM|second|evening|dinner| +64225|AAAAAAAACOKPAAAA|64225|17|50|25|PM|second|evening|dinner| +64226|AAAAAAAADOKPAAAA|64226|17|50|26|PM|second|evening|dinner| +64227|AAAAAAAAEOKPAAAA|64227|17|50|27|PM|second|evening|dinner| +64228|AAAAAAAAFOKPAAAA|64228|17|50|28|PM|second|evening|dinner| +64229|AAAAAAAAGOKPAAAA|64229|17|50|29|PM|second|evening|dinner| +64230|AAAAAAAAHOKPAAAA|64230|17|50|30|PM|second|evening|dinner| +64231|AAAAAAAAIOKPAAAA|64231|17|50|31|PM|second|evening|dinner| +64232|AAAAAAAAJOKPAAAA|64232|17|50|32|PM|second|evening|dinner| +64233|AAAAAAAAKOKPAAAA|64233|17|50|33|PM|second|evening|dinner| +64234|AAAAAAAALOKPAAAA|64234|17|50|34|PM|second|evening|dinner| +64235|AAAAAAAAMOKPAAAA|64235|17|50|35|PM|second|evening|dinner| +64236|AAAAAAAANOKPAAAA|64236|17|50|36|PM|second|evening|dinner| +64237|AAAAAAAAOOKPAAAA|64237|17|50|37|PM|second|evening|dinner| +64238|AAAAAAAAPOKPAAAA|64238|17|50|38|PM|second|evening|dinner| +64239|AAAAAAAAAPKPAAAA|64239|17|50|39|PM|second|evening|dinner| +64240|AAAAAAAABPKPAAAA|64240|17|50|40|PM|second|evening|dinner| +64241|AAAAAAAACPKPAAAA|64241|17|50|41|PM|second|evening|dinner| +64242|AAAAAAAADPKPAAAA|64242|17|50|42|PM|second|evening|dinner| +64243|AAAAAAAAEPKPAAAA|64243|17|50|43|PM|second|evening|dinner| +64244|AAAAAAAAFPKPAAAA|64244|17|50|44|PM|second|evening|dinner| +64245|AAAAAAAAGPKPAAAA|64245|17|50|45|PM|second|evening|dinner| +64246|AAAAAAAAHPKPAAAA|64246|17|50|46|PM|second|evening|dinner| +64247|AAAAAAAAIPKPAAAA|64247|17|50|47|PM|second|evening|dinner| +64248|AAAAAAAAJPKPAAAA|64248|17|50|48|PM|second|evening|dinner| +64249|AAAAAAAAKPKPAAAA|64249|17|50|49|PM|second|evening|dinner| +64250|AAAAAAAALPKPAAAA|64250|17|50|50|PM|second|evening|dinner| +64251|AAAAAAAAMPKPAAAA|64251|17|50|51|PM|second|evening|dinner| +64252|AAAAAAAANPKPAAAA|64252|17|50|52|PM|second|evening|dinner| +64253|AAAAAAAAOPKPAAAA|64253|17|50|53|PM|second|evening|dinner| +64254|AAAAAAAAPPKPAAAA|64254|17|50|54|PM|second|evening|dinner| +64255|AAAAAAAAAALPAAAA|64255|17|50|55|PM|second|evening|dinner| +64256|AAAAAAAABALPAAAA|64256|17|50|56|PM|second|evening|dinner| +64257|AAAAAAAACALPAAAA|64257|17|50|57|PM|second|evening|dinner| +64258|AAAAAAAADALPAAAA|64258|17|50|58|PM|second|evening|dinner| +64259|AAAAAAAAEALPAAAA|64259|17|50|59|PM|second|evening|dinner| +64260|AAAAAAAAFALPAAAA|64260|17|51|0|PM|second|evening|dinner| +64261|AAAAAAAAGALPAAAA|64261|17|51|1|PM|second|evening|dinner| +64262|AAAAAAAAHALPAAAA|64262|17|51|2|PM|second|evening|dinner| +64263|AAAAAAAAIALPAAAA|64263|17|51|3|PM|second|evening|dinner| +64264|AAAAAAAAJALPAAAA|64264|17|51|4|PM|second|evening|dinner| +64265|AAAAAAAAKALPAAAA|64265|17|51|5|PM|second|evening|dinner| +64266|AAAAAAAALALPAAAA|64266|17|51|6|PM|second|evening|dinner| +64267|AAAAAAAAMALPAAAA|64267|17|51|7|PM|second|evening|dinner| +64268|AAAAAAAANALPAAAA|64268|17|51|8|PM|second|evening|dinner| +64269|AAAAAAAAOALPAAAA|64269|17|51|9|PM|second|evening|dinner| +64270|AAAAAAAAPALPAAAA|64270|17|51|10|PM|second|evening|dinner| +64271|AAAAAAAAABLPAAAA|64271|17|51|11|PM|second|evening|dinner| +64272|AAAAAAAABBLPAAAA|64272|17|51|12|PM|second|evening|dinner| +64273|AAAAAAAACBLPAAAA|64273|17|51|13|PM|second|evening|dinner| +64274|AAAAAAAADBLPAAAA|64274|17|51|14|PM|second|evening|dinner| +64275|AAAAAAAAEBLPAAAA|64275|17|51|15|PM|second|evening|dinner| +64276|AAAAAAAAFBLPAAAA|64276|17|51|16|PM|second|evening|dinner| +64277|AAAAAAAAGBLPAAAA|64277|17|51|17|PM|second|evening|dinner| +64278|AAAAAAAAHBLPAAAA|64278|17|51|18|PM|second|evening|dinner| +64279|AAAAAAAAIBLPAAAA|64279|17|51|19|PM|second|evening|dinner| +64280|AAAAAAAAJBLPAAAA|64280|17|51|20|PM|second|evening|dinner| +64281|AAAAAAAAKBLPAAAA|64281|17|51|21|PM|second|evening|dinner| +64282|AAAAAAAALBLPAAAA|64282|17|51|22|PM|second|evening|dinner| +64283|AAAAAAAAMBLPAAAA|64283|17|51|23|PM|second|evening|dinner| +64284|AAAAAAAANBLPAAAA|64284|17|51|24|PM|second|evening|dinner| +64285|AAAAAAAAOBLPAAAA|64285|17|51|25|PM|second|evening|dinner| +64286|AAAAAAAAPBLPAAAA|64286|17|51|26|PM|second|evening|dinner| +64287|AAAAAAAAACLPAAAA|64287|17|51|27|PM|second|evening|dinner| +64288|AAAAAAAABCLPAAAA|64288|17|51|28|PM|second|evening|dinner| +64289|AAAAAAAACCLPAAAA|64289|17|51|29|PM|second|evening|dinner| +64290|AAAAAAAADCLPAAAA|64290|17|51|30|PM|second|evening|dinner| +64291|AAAAAAAAECLPAAAA|64291|17|51|31|PM|second|evening|dinner| +64292|AAAAAAAAFCLPAAAA|64292|17|51|32|PM|second|evening|dinner| +64293|AAAAAAAAGCLPAAAA|64293|17|51|33|PM|second|evening|dinner| +64294|AAAAAAAAHCLPAAAA|64294|17|51|34|PM|second|evening|dinner| +64295|AAAAAAAAICLPAAAA|64295|17|51|35|PM|second|evening|dinner| +64296|AAAAAAAAJCLPAAAA|64296|17|51|36|PM|second|evening|dinner| +64297|AAAAAAAAKCLPAAAA|64297|17|51|37|PM|second|evening|dinner| +64298|AAAAAAAALCLPAAAA|64298|17|51|38|PM|second|evening|dinner| +64299|AAAAAAAAMCLPAAAA|64299|17|51|39|PM|second|evening|dinner| +64300|AAAAAAAANCLPAAAA|64300|17|51|40|PM|second|evening|dinner| +64301|AAAAAAAAOCLPAAAA|64301|17|51|41|PM|second|evening|dinner| +64302|AAAAAAAAPCLPAAAA|64302|17|51|42|PM|second|evening|dinner| +64303|AAAAAAAAADLPAAAA|64303|17|51|43|PM|second|evening|dinner| +64304|AAAAAAAABDLPAAAA|64304|17|51|44|PM|second|evening|dinner| +64305|AAAAAAAACDLPAAAA|64305|17|51|45|PM|second|evening|dinner| +64306|AAAAAAAADDLPAAAA|64306|17|51|46|PM|second|evening|dinner| +64307|AAAAAAAAEDLPAAAA|64307|17|51|47|PM|second|evening|dinner| +64308|AAAAAAAAFDLPAAAA|64308|17|51|48|PM|second|evening|dinner| +64309|AAAAAAAAGDLPAAAA|64309|17|51|49|PM|second|evening|dinner| +64310|AAAAAAAAHDLPAAAA|64310|17|51|50|PM|second|evening|dinner| +64311|AAAAAAAAIDLPAAAA|64311|17|51|51|PM|second|evening|dinner| +64312|AAAAAAAAJDLPAAAA|64312|17|51|52|PM|second|evening|dinner| +64313|AAAAAAAAKDLPAAAA|64313|17|51|53|PM|second|evening|dinner| +64314|AAAAAAAALDLPAAAA|64314|17|51|54|PM|second|evening|dinner| +64315|AAAAAAAAMDLPAAAA|64315|17|51|55|PM|second|evening|dinner| +64316|AAAAAAAANDLPAAAA|64316|17|51|56|PM|second|evening|dinner| +64317|AAAAAAAAODLPAAAA|64317|17|51|57|PM|second|evening|dinner| +64318|AAAAAAAAPDLPAAAA|64318|17|51|58|PM|second|evening|dinner| +64319|AAAAAAAAAELPAAAA|64319|17|51|59|PM|second|evening|dinner| +64320|AAAAAAAABELPAAAA|64320|17|52|0|PM|second|evening|dinner| +64321|AAAAAAAACELPAAAA|64321|17|52|1|PM|second|evening|dinner| +64322|AAAAAAAADELPAAAA|64322|17|52|2|PM|second|evening|dinner| +64323|AAAAAAAAEELPAAAA|64323|17|52|3|PM|second|evening|dinner| +64324|AAAAAAAAFELPAAAA|64324|17|52|4|PM|second|evening|dinner| +64325|AAAAAAAAGELPAAAA|64325|17|52|5|PM|second|evening|dinner| +64326|AAAAAAAAHELPAAAA|64326|17|52|6|PM|second|evening|dinner| +64327|AAAAAAAAIELPAAAA|64327|17|52|7|PM|second|evening|dinner| +64328|AAAAAAAAJELPAAAA|64328|17|52|8|PM|second|evening|dinner| +64329|AAAAAAAAKELPAAAA|64329|17|52|9|PM|second|evening|dinner| +64330|AAAAAAAALELPAAAA|64330|17|52|10|PM|second|evening|dinner| +64331|AAAAAAAAMELPAAAA|64331|17|52|11|PM|second|evening|dinner| +64332|AAAAAAAANELPAAAA|64332|17|52|12|PM|second|evening|dinner| +64333|AAAAAAAAOELPAAAA|64333|17|52|13|PM|second|evening|dinner| +64334|AAAAAAAAPELPAAAA|64334|17|52|14|PM|second|evening|dinner| +64335|AAAAAAAAAFLPAAAA|64335|17|52|15|PM|second|evening|dinner| +64336|AAAAAAAABFLPAAAA|64336|17|52|16|PM|second|evening|dinner| +64337|AAAAAAAACFLPAAAA|64337|17|52|17|PM|second|evening|dinner| +64338|AAAAAAAADFLPAAAA|64338|17|52|18|PM|second|evening|dinner| +64339|AAAAAAAAEFLPAAAA|64339|17|52|19|PM|second|evening|dinner| +64340|AAAAAAAAFFLPAAAA|64340|17|52|20|PM|second|evening|dinner| +64341|AAAAAAAAGFLPAAAA|64341|17|52|21|PM|second|evening|dinner| +64342|AAAAAAAAHFLPAAAA|64342|17|52|22|PM|second|evening|dinner| +64343|AAAAAAAAIFLPAAAA|64343|17|52|23|PM|second|evening|dinner| +64344|AAAAAAAAJFLPAAAA|64344|17|52|24|PM|second|evening|dinner| +64345|AAAAAAAAKFLPAAAA|64345|17|52|25|PM|second|evening|dinner| +64346|AAAAAAAALFLPAAAA|64346|17|52|26|PM|second|evening|dinner| +64347|AAAAAAAAMFLPAAAA|64347|17|52|27|PM|second|evening|dinner| +64348|AAAAAAAANFLPAAAA|64348|17|52|28|PM|second|evening|dinner| +64349|AAAAAAAAOFLPAAAA|64349|17|52|29|PM|second|evening|dinner| +64350|AAAAAAAAPFLPAAAA|64350|17|52|30|PM|second|evening|dinner| +64351|AAAAAAAAAGLPAAAA|64351|17|52|31|PM|second|evening|dinner| +64352|AAAAAAAABGLPAAAA|64352|17|52|32|PM|second|evening|dinner| +64353|AAAAAAAACGLPAAAA|64353|17|52|33|PM|second|evening|dinner| +64354|AAAAAAAADGLPAAAA|64354|17|52|34|PM|second|evening|dinner| +64355|AAAAAAAAEGLPAAAA|64355|17|52|35|PM|second|evening|dinner| +64356|AAAAAAAAFGLPAAAA|64356|17|52|36|PM|second|evening|dinner| +64357|AAAAAAAAGGLPAAAA|64357|17|52|37|PM|second|evening|dinner| +64358|AAAAAAAAHGLPAAAA|64358|17|52|38|PM|second|evening|dinner| +64359|AAAAAAAAIGLPAAAA|64359|17|52|39|PM|second|evening|dinner| +64360|AAAAAAAAJGLPAAAA|64360|17|52|40|PM|second|evening|dinner| +64361|AAAAAAAAKGLPAAAA|64361|17|52|41|PM|second|evening|dinner| +64362|AAAAAAAALGLPAAAA|64362|17|52|42|PM|second|evening|dinner| +64363|AAAAAAAAMGLPAAAA|64363|17|52|43|PM|second|evening|dinner| +64364|AAAAAAAANGLPAAAA|64364|17|52|44|PM|second|evening|dinner| +64365|AAAAAAAAOGLPAAAA|64365|17|52|45|PM|second|evening|dinner| +64366|AAAAAAAAPGLPAAAA|64366|17|52|46|PM|second|evening|dinner| +64367|AAAAAAAAAHLPAAAA|64367|17|52|47|PM|second|evening|dinner| +64368|AAAAAAAABHLPAAAA|64368|17|52|48|PM|second|evening|dinner| +64369|AAAAAAAACHLPAAAA|64369|17|52|49|PM|second|evening|dinner| +64370|AAAAAAAADHLPAAAA|64370|17|52|50|PM|second|evening|dinner| +64371|AAAAAAAAEHLPAAAA|64371|17|52|51|PM|second|evening|dinner| +64372|AAAAAAAAFHLPAAAA|64372|17|52|52|PM|second|evening|dinner| +64373|AAAAAAAAGHLPAAAA|64373|17|52|53|PM|second|evening|dinner| +64374|AAAAAAAAHHLPAAAA|64374|17|52|54|PM|second|evening|dinner| +64375|AAAAAAAAIHLPAAAA|64375|17|52|55|PM|second|evening|dinner| +64376|AAAAAAAAJHLPAAAA|64376|17|52|56|PM|second|evening|dinner| +64377|AAAAAAAAKHLPAAAA|64377|17|52|57|PM|second|evening|dinner| +64378|AAAAAAAALHLPAAAA|64378|17|52|58|PM|second|evening|dinner| +64379|AAAAAAAAMHLPAAAA|64379|17|52|59|PM|second|evening|dinner| +64380|AAAAAAAANHLPAAAA|64380|17|53|0|PM|second|evening|dinner| +64381|AAAAAAAAOHLPAAAA|64381|17|53|1|PM|second|evening|dinner| +64382|AAAAAAAAPHLPAAAA|64382|17|53|2|PM|second|evening|dinner| +64383|AAAAAAAAAILPAAAA|64383|17|53|3|PM|second|evening|dinner| +64384|AAAAAAAABILPAAAA|64384|17|53|4|PM|second|evening|dinner| +64385|AAAAAAAACILPAAAA|64385|17|53|5|PM|second|evening|dinner| +64386|AAAAAAAADILPAAAA|64386|17|53|6|PM|second|evening|dinner| +64387|AAAAAAAAEILPAAAA|64387|17|53|7|PM|second|evening|dinner| +64388|AAAAAAAAFILPAAAA|64388|17|53|8|PM|second|evening|dinner| +64389|AAAAAAAAGILPAAAA|64389|17|53|9|PM|second|evening|dinner| +64390|AAAAAAAAHILPAAAA|64390|17|53|10|PM|second|evening|dinner| +64391|AAAAAAAAIILPAAAA|64391|17|53|11|PM|second|evening|dinner| +64392|AAAAAAAAJILPAAAA|64392|17|53|12|PM|second|evening|dinner| +64393|AAAAAAAAKILPAAAA|64393|17|53|13|PM|second|evening|dinner| +64394|AAAAAAAALILPAAAA|64394|17|53|14|PM|second|evening|dinner| +64395|AAAAAAAAMILPAAAA|64395|17|53|15|PM|second|evening|dinner| +64396|AAAAAAAANILPAAAA|64396|17|53|16|PM|second|evening|dinner| +64397|AAAAAAAAOILPAAAA|64397|17|53|17|PM|second|evening|dinner| +64398|AAAAAAAAPILPAAAA|64398|17|53|18|PM|second|evening|dinner| +64399|AAAAAAAAAJLPAAAA|64399|17|53|19|PM|second|evening|dinner| +64400|AAAAAAAABJLPAAAA|64400|17|53|20|PM|second|evening|dinner| +64401|AAAAAAAACJLPAAAA|64401|17|53|21|PM|second|evening|dinner| +64402|AAAAAAAADJLPAAAA|64402|17|53|22|PM|second|evening|dinner| +64403|AAAAAAAAEJLPAAAA|64403|17|53|23|PM|second|evening|dinner| +64404|AAAAAAAAFJLPAAAA|64404|17|53|24|PM|second|evening|dinner| +64405|AAAAAAAAGJLPAAAA|64405|17|53|25|PM|second|evening|dinner| +64406|AAAAAAAAHJLPAAAA|64406|17|53|26|PM|second|evening|dinner| +64407|AAAAAAAAIJLPAAAA|64407|17|53|27|PM|second|evening|dinner| +64408|AAAAAAAAJJLPAAAA|64408|17|53|28|PM|second|evening|dinner| +64409|AAAAAAAAKJLPAAAA|64409|17|53|29|PM|second|evening|dinner| +64410|AAAAAAAALJLPAAAA|64410|17|53|30|PM|second|evening|dinner| +64411|AAAAAAAAMJLPAAAA|64411|17|53|31|PM|second|evening|dinner| +64412|AAAAAAAANJLPAAAA|64412|17|53|32|PM|second|evening|dinner| +64413|AAAAAAAAOJLPAAAA|64413|17|53|33|PM|second|evening|dinner| +64414|AAAAAAAAPJLPAAAA|64414|17|53|34|PM|second|evening|dinner| +64415|AAAAAAAAAKLPAAAA|64415|17|53|35|PM|second|evening|dinner| +64416|AAAAAAAABKLPAAAA|64416|17|53|36|PM|second|evening|dinner| +64417|AAAAAAAACKLPAAAA|64417|17|53|37|PM|second|evening|dinner| +64418|AAAAAAAADKLPAAAA|64418|17|53|38|PM|second|evening|dinner| +64419|AAAAAAAAEKLPAAAA|64419|17|53|39|PM|second|evening|dinner| +64420|AAAAAAAAFKLPAAAA|64420|17|53|40|PM|second|evening|dinner| +64421|AAAAAAAAGKLPAAAA|64421|17|53|41|PM|second|evening|dinner| +64422|AAAAAAAAHKLPAAAA|64422|17|53|42|PM|second|evening|dinner| +64423|AAAAAAAAIKLPAAAA|64423|17|53|43|PM|second|evening|dinner| +64424|AAAAAAAAJKLPAAAA|64424|17|53|44|PM|second|evening|dinner| +64425|AAAAAAAAKKLPAAAA|64425|17|53|45|PM|second|evening|dinner| +64426|AAAAAAAALKLPAAAA|64426|17|53|46|PM|second|evening|dinner| +64427|AAAAAAAAMKLPAAAA|64427|17|53|47|PM|second|evening|dinner| +64428|AAAAAAAANKLPAAAA|64428|17|53|48|PM|second|evening|dinner| +64429|AAAAAAAAOKLPAAAA|64429|17|53|49|PM|second|evening|dinner| +64430|AAAAAAAAPKLPAAAA|64430|17|53|50|PM|second|evening|dinner| +64431|AAAAAAAAALLPAAAA|64431|17|53|51|PM|second|evening|dinner| +64432|AAAAAAAABLLPAAAA|64432|17|53|52|PM|second|evening|dinner| +64433|AAAAAAAACLLPAAAA|64433|17|53|53|PM|second|evening|dinner| +64434|AAAAAAAADLLPAAAA|64434|17|53|54|PM|second|evening|dinner| +64435|AAAAAAAAELLPAAAA|64435|17|53|55|PM|second|evening|dinner| +64436|AAAAAAAAFLLPAAAA|64436|17|53|56|PM|second|evening|dinner| +64437|AAAAAAAAGLLPAAAA|64437|17|53|57|PM|second|evening|dinner| +64438|AAAAAAAAHLLPAAAA|64438|17|53|58|PM|second|evening|dinner| +64439|AAAAAAAAILLPAAAA|64439|17|53|59|PM|second|evening|dinner| +64440|AAAAAAAAJLLPAAAA|64440|17|54|0|PM|second|evening|dinner| +64441|AAAAAAAAKLLPAAAA|64441|17|54|1|PM|second|evening|dinner| +64442|AAAAAAAALLLPAAAA|64442|17|54|2|PM|second|evening|dinner| +64443|AAAAAAAAMLLPAAAA|64443|17|54|3|PM|second|evening|dinner| +64444|AAAAAAAANLLPAAAA|64444|17|54|4|PM|second|evening|dinner| +64445|AAAAAAAAOLLPAAAA|64445|17|54|5|PM|second|evening|dinner| +64446|AAAAAAAAPLLPAAAA|64446|17|54|6|PM|second|evening|dinner| +64447|AAAAAAAAAMLPAAAA|64447|17|54|7|PM|second|evening|dinner| +64448|AAAAAAAABMLPAAAA|64448|17|54|8|PM|second|evening|dinner| +64449|AAAAAAAACMLPAAAA|64449|17|54|9|PM|second|evening|dinner| +64450|AAAAAAAADMLPAAAA|64450|17|54|10|PM|second|evening|dinner| +64451|AAAAAAAAEMLPAAAA|64451|17|54|11|PM|second|evening|dinner| +64452|AAAAAAAAFMLPAAAA|64452|17|54|12|PM|second|evening|dinner| +64453|AAAAAAAAGMLPAAAA|64453|17|54|13|PM|second|evening|dinner| +64454|AAAAAAAAHMLPAAAA|64454|17|54|14|PM|second|evening|dinner| +64455|AAAAAAAAIMLPAAAA|64455|17|54|15|PM|second|evening|dinner| +64456|AAAAAAAAJMLPAAAA|64456|17|54|16|PM|second|evening|dinner| +64457|AAAAAAAAKMLPAAAA|64457|17|54|17|PM|second|evening|dinner| +64458|AAAAAAAALMLPAAAA|64458|17|54|18|PM|second|evening|dinner| +64459|AAAAAAAAMMLPAAAA|64459|17|54|19|PM|second|evening|dinner| +64460|AAAAAAAANMLPAAAA|64460|17|54|20|PM|second|evening|dinner| +64461|AAAAAAAAOMLPAAAA|64461|17|54|21|PM|second|evening|dinner| +64462|AAAAAAAAPMLPAAAA|64462|17|54|22|PM|second|evening|dinner| +64463|AAAAAAAAANLPAAAA|64463|17|54|23|PM|second|evening|dinner| +64464|AAAAAAAABNLPAAAA|64464|17|54|24|PM|second|evening|dinner| +64465|AAAAAAAACNLPAAAA|64465|17|54|25|PM|second|evening|dinner| +64466|AAAAAAAADNLPAAAA|64466|17|54|26|PM|second|evening|dinner| +64467|AAAAAAAAENLPAAAA|64467|17|54|27|PM|second|evening|dinner| +64468|AAAAAAAAFNLPAAAA|64468|17|54|28|PM|second|evening|dinner| +64469|AAAAAAAAGNLPAAAA|64469|17|54|29|PM|second|evening|dinner| +64470|AAAAAAAAHNLPAAAA|64470|17|54|30|PM|second|evening|dinner| +64471|AAAAAAAAINLPAAAA|64471|17|54|31|PM|second|evening|dinner| +64472|AAAAAAAAJNLPAAAA|64472|17|54|32|PM|second|evening|dinner| +64473|AAAAAAAAKNLPAAAA|64473|17|54|33|PM|second|evening|dinner| +64474|AAAAAAAALNLPAAAA|64474|17|54|34|PM|second|evening|dinner| +64475|AAAAAAAAMNLPAAAA|64475|17|54|35|PM|second|evening|dinner| +64476|AAAAAAAANNLPAAAA|64476|17|54|36|PM|second|evening|dinner| +64477|AAAAAAAAONLPAAAA|64477|17|54|37|PM|second|evening|dinner| +64478|AAAAAAAAPNLPAAAA|64478|17|54|38|PM|second|evening|dinner| +64479|AAAAAAAAAOLPAAAA|64479|17|54|39|PM|second|evening|dinner| +64480|AAAAAAAABOLPAAAA|64480|17|54|40|PM|second|evening|dinner| +64481|AAAAAAAACOLPAAAA|64481|17|54|41|PM|second|evening|dinner| +64482|AAAAAAAADOLPAAAA|64482|17|54|42|PM|second|evening|dinner| +64483|AAAAAAAAEOLPAAAA|64483|17|54|43|PM|second|evening|dinner| +64484|AAAAAAAAFOLPAAAA|64484|17|54|44|PM|second|evening|dinner| +64485|AAAAAAAAGOLPAAAA|64485|17|54|45|PM|second|evening|dinner| +64486|AAAAAAAAHOLPAAAA|64486|17|54|46|PM|second|evening|dinner| +64487|AAAAAAAAIOLPAAAA|64487|17|54|47|PM|second|evening|dinner| +64488|AAAAAAAAJOLPAAAA|64488|17|54|48|PM|second|evening|dinner| +64489|AAAAAAAAKOLPAAAA|64489|17|54|49|PM|second|evening|dinner| +64490|AAAAAAAALOLPAAAA|64490|17|54|50|PM|second|evening|dinner| +64491|AAAAAAAAMOLPAAAA|64491|17|54|51|PM|second|evening|dinner| +64492|AAAAAAAANOLPAAAA|64492|17|54|52|PM|second|evening|dinner| +64493|AAAAAAAAOOLPAAAA|64493|17|54|53|PM|second|evening|dinner| +64494|AAAAAAAAPOLPAAAA|64494|17|54|54|PM|second|evening|dinner| +64495|AAAAAAAAAPLPAAAA|64495|17|54|55|PM|second|evening|dinner| +64496|AAAAAAAABPLPAAAA|64496|17|54|56|PM|second|evening|dinner| +64497|AAAAAAAACPLPAAAA|64497|17|54|57|PM|second|evening|dinner| +64498|AAAAAAAADPLPAAAA|64498|17|54|58|PM|second|evening|dinner| +64499|AAAAAAAAEPLPAAAA|64499|17|54|59|PM|second|evening|dinner| +64500|AAAAAAAAFPLPAAAA|64500|17|55|0|PM|second|evening|dinner| +64501|AAAAAAAAGPLPAAAA|64501|17|55|1|PM|second|evening|dinner| +64502|AAAAAAAAHPLPAAAA|64502|17|55|2|PM|second|evening|dinner| +64503|AAAAAAAAIPLPAAAA|64503|17|55|3|PM|second|evening|dinner| +64504|AAAAAAAAJPLPAAAA|64504|17|55|4|PM|second|evening|dinner| +64505|AAAAAAAAKPLPAAAA|64505|17|55|5|PM|second|evening|dinner| +64506|AAAAAAAALPLPAAAA|64506|17|55|6|PM|second|evening|dinner| +64507|AAAAAAAAMPLPAAAA|64507|17|55|7|PM|second|evening|dinner| +64508|AAAAAAAANPLPAAAA|64508|17|55|8|PM|second|evening|dinner| +64509|AAAAAAAAOPLPAAAA|64509|17|55|9|PM|second|evening|dinner| +64510|AAAAAAAAPPLPAAAA|64510|17|55|10|PM|second|evening|dinner| +64511|AAAAAAAAAAMPAAAA|64511|17|55|11|PM|second|evening|dinner| +64512|AAAAAAAABAMPAAAA|64512|17|55|12|PM|second|evening|dinner| +64513|AAAAAAAACAMPAAAA|64513|17|55|13|PM|second|evening|dinner| +64514|AAAAAAAADAMPAAAA|64514|17|55|14|PM|second|evening|dinner| +64515|AAAAAAAAEAMPAAAA|64515|17|55|15|PM|second|evening|dinner| +64516|AAAAAAAAFAMPAAAA|64516|17|55|16|PM|second|evening|dinner| +64517|AAAAAAAAGAMPAAAA|64517|17|55|17|PM|second|evening|dinner| +64518|AAAAAAAAHAMPAAAA|64518|17|55|18|PM|second|evening|dinner| +64519|AAAAAAAAIAMPAAAA|64519|17|55|19|PM|second|evening|dinner| +64520|AAAAAAAAJAMPAAAA|64520|17|55|20|PM|second|evening|dinner| +64521|AAAAAAAAKAMPAAAA|64521|17|55|21|PM|second|evening|dinner| +64522|AAAAAAAALAMPAAAA|64522|17|55|22|PM|second|evening|dinner| +64523|AAAAAAAAMAMPAAAA|64523|17|55|23|PM|second|evening|dinner| +64524|AAAAAAAANAMPAAAA|64524|17|55|24|PM|second|evening|dinner| +64525|AAAAAAAAOAMPAAAA|64525|17|55|25|PM|second|evening|dinner| +64526|AAAAAAAAPAMPAAAA|64526|17|55|26|PM|second|evening|dinner| +64527|AAAAAAAAABMPAAAA|64527|17|55|27|PM|second|evening|dinner| +64528|AAAAAAAABBMPAAAA|64528|17|55|28|PM|second|evening|dinner| +64529|AAAAAAAACBMPAAAA|64529|17|55|29|PM|second|evening|dinner| +64530|AAAAAAAADBMPAAAA|64530|17|55|30|PM|second|evening|dinner| +64531|AAAAAAAAEBMPAAAA|64531|17|55|31|PM|second|evening|dinner| +64532|AAAAAAAAFBMPAAAA|64532|17|55|32|PM|second|evening|dinner| +64533|AAAAAAAAGBMPAAAA|64533|17|55|33|PM|second|evening|dinner| +64534|AAAAAAAAHBMPAAAA|64534|17|55|34|PM|second|evening|dinner| +64535|AAAAAAAAIBMPAAAA|64535|17|55|35|PM|second|evening|dinner| +64536|AAAAAAAAJBMPAAAA|64536|17|55|36|PM|second|evening|dinner| +64537|AAAAAAAAKBMPAAAA|64537|17|55|37|PM|second|evening|dinner| +64538|AAAAAAAALBMPAAAA|64538|17|55|38|PM|second|evening|dinner| +64539|AAAAAAAAMBMPAAAA|64539|17|55|39|PM|second|evening|dinner| +64540|AAAAAAAANBMPAAAA|64540|17|55|40|PM|second|evening|dinner| +64541|AAAAAAAAOBMPAAAA|64541|17|55|41|PM|second|evening|dinner| +64542|AAAAAAAAPBMPAAAA|64542|17|55|42|PM|second|evening|dinner| +64543|AAAAAAAAACMPAAAA|64543|17|55|43|PM|second|evening|dinner| +64544|AAAAAAAABCMPAAAA|64544|17|55|44|PM|second|evening|dinner| +64545|AAAAAAAACCMPAAAA|64545|17|55|45|PM|second|evening|dinner| +64546|AAAAAAAADCMPAAAA|64546|17|55|46|PM|second|evening|dinner| +64547|AAAAAAAAECMPAAAA|64547|17|55|47|PM|second|evening|dinner| +64548|AAAAAAAAFCMPAAAA|64548|17|55|48|PM|second|evening|dinner| +64549|AAAAAAAAGCMPAAAA|64549|17|55|49|PM|second|evening|dinner| +64550|AAAAAAAAHCMPAAAA|64550|17|55|50|PM|second|evening|dinner| +64551|AAAAAAAAICMPAAAA|64551|17|55|51|PM|second|evening|dinner| +64552|AAAAAAAAJCMPAAAA|64552|17|55|52|PM|second|evening|dinner| +64553|AAAAAAAAKCMPAAAA|64553|17|55|53|PM|second|evening|dinner| +64554|AAAAAAAALCMPAAAA|64554|17|55|54|PM|second|evening|dinner| +64555|AAAAAAAAMCMPAAAA|64555|17|55|55|PM|second|evening|dinner| +64556|AAAAAAAANCMPAAAA|64556|17|55|56|PM|second|evening|dinner| +64557|AAAAAAAAOCMPAAAA|64557|17|55|57|PM|second|evening|dinner| +64558|AAAAAAAAPCMPAAAA|64558|17|55|58|PM|second|evening|dinner| +64559|AAAAAAAAADMPAAAA|64559|17|55|59|PM|second|evening|dinner| +64560|AAAAAAAABDMPAAAA|64560|17|56|0|PM|second|evening|dinner| +64561|AAAAAAAACDMPAAAA|64561|17|56|1|PM|second|evening|dinner| +64562|AAAAAAAADDMPAAAA|64562|17|56|2|PM|second|evening|dinner| +64563|AAAAAAAAEDMPAAAA|64563|17|56|3|PM|second|evening|dinner| +64564|AAAAAAAAFDMPAAAA|64564|17|56|4|PM|second|evening|dinner| +64565|AAAAAAAAGDMPAAAA|64565|17|56|5|PM|second|evening|dinner| +64566|AAAAAAAAHDMPAAAA|64566|17|56|6|PM|second|evening|dinner| +64567|AAAAAAAAIDMPAAAA|64567|17|56|7|PM|second|evening|dinner| +64568|AAAAAAAAJDMPAAAA|64568|17|56|8|PM|second|evening|dinner| +64569|AAAAAAAAKDMPAAAA|64569|17|56|9|PM|second|evening|dinner| +64570|AAAAAAAALDMPAAAA|64570|17|56|10|PM|second|evening|dinner| +64571|AAAAAAAAMDMPAAAA|64571|17|56|11|PM|second|evening|dinner| +64572|AAAAAAAANDMPAAAA|64572|17|56|12|PM|second|evening|dinner| +64573|AAAAAAAAODMPAAAA|64573|17|56|13|PM|second|evening|dinner| +64574|AAAAAAAAPDMPAAAA|64574|17|56|14|PM|second|evening|dinner| +64575|AAAAAAAAAEMPAAAA|64575|17|56|15|PM|second|evening|dinner| +64576|AAAAAAAABEMPAAAA|64576|17|56|16|PM|second|evening|dinner| +64577|AAAAAAAACEMPAAAA|64577|17|56|17|PM|second|evening|dinner| +64578|AAAAAAAADEMPAAAA|64578|17|56|18|PM|second|evening|dinner| +64579|AAAAAAAAEEMPAAAA|64579|17|56|19|PM|second|evening|dinner| +64580|AAAAAAAAFEMPAAAA|64580|17|56|20|PM|second|evening|dinner| +64581|AAAAAAAAGEMPAAAA|64581|17|56|21|PM|second|evening|dinner| +64582|AAAAAAAAHEMPAAAA|64582|17|56|22|PM|second|evening|dinner| +64583|AAAAAAAAIEMPAAAA|64583|17|56|23|PM|second|evening|dinner| +64584|AAAAAAAAJEMPAAAA|64584|17|56|24|PM|second|evening|dinner| +64585|AAAAAAAAKEMPAAAA|64585|17|56|25|PM|second|evening|dinner| +64586|AAAAAAAALEMPAAAA|64586|17|56|26|PM|second|evening|dinner| +64587|AAAAAAAAMEMPAAAA|64587|17|56|27|PM|second|evening|dinner| +64588|AAAAAAAANEMPAAAA|64588|17|56|28|PM|second|evening|dinner| +64589|AAAAAAAAOEMPAAAA|64589|17|56|29|PM|second|evening|dinner| +64590|AAAAAAAAPEMPAAAA|64590|17|56|30|PM|second|evening|dinner| +64591|AAAAAAAAAFMPAAAA|64591|17|56|31|PM|second|evening|dinner| +64592|AAAAAAAABFMPAAAA|64592|17|56|32|PM|second|evening|dinner| +64593|AAAAAAAACFMPAAAA|64593|17|56|33|PM|second|evening|dinner| +64594|AAAAAAAADFMPAAAA|64594|17|56|34|PM|second|evening|dinner| +64595|AAAAAAAAEFMPAAAA|64595|17|56|35|PM|second|evening|dinner| +64596|AAAAAAAAFFMPAAAA|64596|17|56|36|PM|second|evening|dinner| +64597|AAAAAAAAGFMPAAAA|64597|17|56|37|PM|second|evening|dinner| +64598|AAAAAAAAHFMPAAAA|64598|17|56|38|PM|second|evening|dinner| +64599|AAAAAAAAIFMPAAAA|64599|17|56|39|PM|second|evening|dinner| +64600|AAAAAAAAJFMPAAAA|64600|17|56|40|PM|second|evening|dinner| +64601|AAAAAAAAKFMPAAAA|64601|17|56|41|PM|second|evening|dinner| +64602|AAAAAAAALFMPAAAA|64602|17|56|42|PM|second|evening|dinner| +64603|AAAAAAAAMFMPAAAA|64603|17|56|43|PM|second|evening|dinner| +64604|AAAAAAAANFMPAAAA|64604|17|56|44|PM|second|evening|dinner| +64605|AAAAAAAAOFMPAAAA|64605|17|56|45|PM|second|evening|dinner| +64606|AAAAAAAAPFMPAAAA|64606|17|56|46|PM|second|evening|dinner| +64607|AAAAAAAAAGMPAAAA|64607|17|56|47|PM|second|evening|dinner| +64608|AAAAAAAABGMPAAAA|64608|17|56|48|PM|second|evening|dinner| +64609|AAAAAAAACGMPAAAA|64609|17|56|49|PM|second|evening|dinner| +64610|AAAAAAAADGMPAAAA|64610|17|56|50|PM|second|evening|dinner| +64611|AAAAAAAAEGMPAAAA|64611|17|56|51|PM|second|evening|dinner| +64612|AAAAAAAAFGMPAAAA|64612|17|56|52|PM|second|evening|dinner| +64613|AAAAAAAAGGMPAAAA|64613|17|56|53|PM|second|evening|dinner| +64614|AAAAAAAAHGMPAAAA|64614|17|56|54|PM|second|evening|dinner| +64615|AAAAAAAAIGMPAAAA|64615|17|56|55|PM|second|evening|dinner| +64616|AAAAAAAAJGMPAAAA|64616|17|56|56|PM|second|evening|dinner| +64617|AAAAAAAAKGMPAAAA|64617|17|56|57|PM|second|evening|dinner| +64618|AAAAAAAALGMPAAAA|64618|17|56|58|PM|second|evening|dinner| +64619|AAAAAAAAMGMPAAAA|64619|17|56|59|PM|second|evening|dinner| +64620|AAAAAAAANGMPAAAA|64620|17|57|0|PM|second|evening|dinner| +64621|AAAAAAAAOGMPAAAA|64621|17|57|1|PM|second|evening|dinner| +64622|AAAAAAAAPGMPAAAA|64622|17|57|2|PM|second|evening|dinner| +64623|AAAAAAAAAHMPAAAA|64623|17|57|3|PM|second|evening|dinner| +64624|AAAAAAAABHMPAAAA|64624|17|57|4|PM|second|evening|dinner| +64625|AAAAAAAACHMPAAAA|64625|17|57|5|PM|second|evening|dinner| +64626|AAAAAAAADHMPAAAA|64626|17|57|6|PM|second|evening|dinner| +64627|AAAAAAAAEHMPAAAA|64627|17|57|7|PM|second|evening|dinner| +64628|AAAAAAAAFHMPAAAA|64628|17|57|8|PM|second|evening|dinner| +64629|AAAAAAAAGHMPAAAA|64629|17|57|9|PM|second|evening|dinner| +64630|AAAAAAAAHHMPAAAA|64630|17|57|10|PM|second|evening|dinner| +64631|AAAAAAAAIHMPAAAA|64631|17|57|11|PM|second|evening|dinner| +64632|AAAAAAAAJHMPAAAA|64632|17|57|12|PM|second|evening|dinner| +64633|AAAAAAAAKHMPAAAA|64633|17|57|13|PM|second|evening|dinner| +64634|AAAAAAAALHMPAAAA|64634|17|57|14|PM|second|evening|dinner| +64635|AAAAAAAAMHMPAAAA|64635|17|57|15|PM|second|evening|dinner| +64636|AAAAAAAANHMPAAAA|64636|17|57|16|PM|second|evening|dinner| +64637|AAAAAAAAOHMPAAAA|64637|17|57|17|PM|second|evening|dinner| +64638|AAAAAAAAPHMPAAAA|64638|17|57|18|PM|second|evening|dinner| +64639|AAAAAAAAAIMPAAAA|64639|17|57|19|PM|second|evening|dinner| +64640|AAAAAAAABIMPAAAA|64640|17|57|20|PM|second|evening|dinner| +64641|AAAAAAAACIMPAAAA|64641|17|57|21|PM|second|evening|dinner| +64642|AAAAAAAADIMPAAAA|64642|17|57|22|PM|second|evening|dinner| +64643|AAAAAAAAEIMPAAAA|64643|17|57|23|PM|second|evening|dinner| +64644|AAAAAAAAFIMPAAAA|64644|17|57|24|PM|second|evening|dinner| +64645|AAAAAAAAGIMPAAAA|64645|17|57|25|PM|second|evening|dinner| +64646|AAAAAAAAHIMPAAAA|64646|17|57|26|PM|second|evening|dinner| +64647|AAAAAAAAIIMPAAAA|64647|17|57|27|PM|second|evening|dinner| +64648|AAAAAAAAJIMPAAAA|64648|17|57|28|PM|second|evening|dinner| +64649|AAAAAAAAKIMPAAAA|64649|17|57|29|PM|second|evening|dinner| +64650|AAAAAAAALIMPAAAA|64650|17|57|30|PM|second|evening|dinner| +64651|AAAAAAAAMIMPAAAA|64651|17|57|31|PM|second|evening|dinner| +64652|AAAAAAAANIMPAAAA|64652|17|57|32|PM|second|evening|dinner| +64653|AAAAAAAAOIMPAAAA|64653|17|57|33|PM|second|evening|dinner| +64654|AAAAAAAAPIMPAAAA|64654|17|57|34|PM|second|evening|dinner| +64655|AAAAAAAAAJMPAAAA|64655|17|57|35|PM|second|evening|dinner| +64656|AAAAAAAABJMPAAAA|64656|17|57|36|PM|second|evening|dinner| +64657|AAAAAAAACJMPAAAA|64657|17|57|37|PM|second|evening|dinner| +64658|AAAAAAAADJMPAAAA|64658|17|57|38|PM|second|evening|dinner| +64659|AAAAAAAAEJMPAAAA|64659|17|57|39|PM|second|evening|dinner| +64660|AAAAAAAAFJMPAAAA|64660|17|57|40|PM|second|evening|dinner| +64661|AAAAAAAAGJMPAAAA|64661|17|57|41|PM|second|evening|dinner| +64662|AAAAAAAAHJMPAAAA|64662|17|57|42|PM|second|evening|dinner| +64663|AAAAAAAAIJMPAAAA|64663|17|57|43|PM|second|evening|dinner| +64664|AAAAAAAAJJMPAAAA|64664|17|57|44|PM|second|evening|dinner| +64665|AAAAAAAAKJMPAAAA|64665|17|57|45|PM|second|evening|dinner| +64666|AAAAAAAALJMPAAAA|64666|17|57|46|PM|second|evening|dinner| +64667|AAAAAAAAMJMPAAAA|64667|17|57|47|PM|second|evening|dinner| +64668|AAAAAAAANJMPAAAA|64668|17|57|48|PM|second|evening|dinner| +64669|AAAAAAAAOJMPAAAA|64669|17|57|49|PM|second|evening|dinner| +64670|AAAAAAAAPJMPAAAA|64670|17|57|50|PM|second|evening|dinner| +64671|AAAAAAAAAKMPAAAA|64671|17|57|51|PM|second|evening|dinner| +64672|AAAAAAAABKMPAAAA|64672|17|57|52|PM|second|evening|dinner| +64673|AAAAAAAACKMPAAAA|64673|17|57|53|PM|second|evening|dinner| +64674|AAAAAAAADKMPAAAA|64674|17|57|54|PM|second|evening|dinner| +64675|AAAAAAAAEKMPAAAA|64675|17|57|55|PM|second|evening|dinner| +64676|AAAAAAAAFKMPAAAA|64676|17|57|56|PM|second|evening|dinner| +64677|AAAAAAAAGKMPAAAA|64677|17|57|57|PM|second|evening|dinner| +64678|AAAAAAAAHKMPAAAA|64678|17|57|58|PM|second|evening|dinner| +64679|AAAAAAAAIKMPAAAA|64679|17|57|59|PM|second|evening|dinner| +64680|AAAAAAAAJKMPAAAA|64680|17|58|0|PM|second|evening|dinner| +64681|AAAAAAAAKKMPAAAA|64681|17|58|1|PM|second|evening|dinner| +64682|AAAAAAAALKMPAAAA|64682|17|58|2|PM|second|evening|dinner| +64683|AAAAAAAAMKMPAAAA|64683|17|58|3|PM|second|evening|dinner| +64684|AAAAAAAANKMPAAAA|64684|17|58|4|PM|second|evening|dinner| +64685|AAAAAAAAOKMPAAAA|64685|17|58|5|PM|second|evening|dinner| +64686|AAAAAAAAPKMPAAAA|64686|17|58|6|PM|second|evening|dinner| +64687|AAAAAAAAALMPAAAA|64687|17|58|7|PM|second|evening|dinner| +64688|AAAAAAAABLMPAAAA|64688|17|58|8|PM|second|evening|dinner| +64689|AAAAAAAACLMPAAAA|64689|17|58|9|PM|second|evening|dinner| +64690|AAAAAAAADLMPAAAA|64690|17|58|10|PM|second|evening|dinner| +64691|AAAAAAAAELMPAAAA|64691|17|58|11|PM|second|evening|dinner| +64692|AAAAAAAAFLMPAAAA|64692|17|58|12|PM|second|evening|dinner| +64693|AAAAAAAAGLMPAAAA|64693|17|58|13|PM|second|evening|dinner| +64694|AAAAAAAAHLMPAAAA|64694|17|58|14|PM|second|evening|dinner| +64695|AAAAAAAAILMPAAAA|64695|17|58|15|PM|second|evening|dinner| +64696|AAAAAAAAJLMPAAAA|64696|17|58|16|PM|second|evening|dinner| +64697|AAAAAAAAKLMPAAAA|64697|17|58|17|PM|second|evening|dinner| +64698|AAAAAAAALLMPAAAA|64698|17|58|18|PM|second|evening|dinner| +64699|AAAAAAAAMLMPAAAA|64699|17|58|19|PM|second|evening|dinner| +64700|AAAAAAAANLMPAAAA|64700|17|58|20|PM|second|evening|dinner| +64701|AAAAAAAAOLMPAAAA|64701|17|58|21|PM|second|evening|dinner| +64702|AAAAAAAAPLMPAAAA|64702|17|58|22|PM|second|evening|dinner| +64703|AAAAAAAAAMMPAAAA|64703|17|58|23|PM|second|evening|dinner| +64704|AAAAAAAABMMPAAAA|64704|17|58|24|PM|second|evening|dinner| +64705|AAAAAAAACMMPAAAA|64705|17|58|25|PM|second|evening|dinner| +64706|AAAAAAAADMMPAAAA|64706|17|58|26|PM|second|evening|dinner| +64707|AAAAAAAAEMMPAAAA|64707|17|58|27|PM|second|evening|dinner| +64708|AAAAAAAAFMMPAAAA|64708|17|58|28|PM|second|evening|dinner| +64709|AAAAAAAAGMMPAAAA|64709|17|58|29|PM|second|evening|dinner| +64710|AAAAAAAAHMMPAAAA|64710|17|58|30|PM|second|evening|dinner| +64711|AAAAAAAAIMMPAAAA|64711|17|58|31|PM|second|evening|dinner| +64712|AAAAAAAAJMMPAAAA|64712|17|58|32|PM|second|evening|dinner| +64713|AAAAAAAAKMMPAAAA|64713|17|58|33|PM|second|evening|dinner| +64714|AAAAAAAALMMPAAAA|64714|17|58|34|PM|second|evening|dinner| +64715|AAAAAAAAMMMPAAAA|64715|17|58|35|PM|second|evening|dinner| +64716|AAAAAAAANMMPAAAA|64716|17|58|36|PM|second|evening|dinner| +64717|AAAAAAAAOMMPAAAA|64717|17|58|37|PM|second|evening|dinner| +64718|AAAAAAAAPMMPAAAA|64718|17|58|38|PM|second|evening|dinner| +64719|AAAAAAAAANMPAAAA|64719|17|58|39|PM|second|evening|dinner| +64720|AAAAAAAABNMPAAAA|64720|17|58|40|PM|second|evening|dinner| +64721|AAAAAAAACNMPAAAA|64721|17|58|41|PM|second|evening|dinner| +64722|AAAAAAAADNMPAAAA|64722|17|58|42|PM|second|evening|dinner| +64723|AAAAAAAAENMPAAAA|64723|17|58|43|PM|second|evening|dinner| +64724|AAAAAAAAFNMPAAAA|64724|17|58|44|PM|second|evening|dinner| +64725|AAAAAAAAGNMPAAAA|64725|17|58|45|PM|second|evening|dinner| +64726|AAAAAAAAHNMPAAAA|64726|17|58|46|PM|second|evening|dinner| +64727|AAAAAAAAINMPAAAA|64727|17|58|47|PM|second|evening|dinner| +64728|AAAAAAAAJNMPAAAA|64728|17|58|48|PM|second|evening|dinner| +64729|AAAAAAAAKNMPAAAA|64729|17|58|49|PM|second|evening|dinner| +64730|AAAAAAAALNMPAAAA|64730|17|58|50|PM|second|evening|dinner| +64731|AAAAAAAAMNMPAAAA|64731|17|58|51|PM|second|evening|dinner| +64732|AAAAAAAANNMPAAAA|64732|17|58|52|PM|second|evening|dinner| +64733|AAAAAAAAONMPAAAA|64733|17|58|53|PM|second|evening|dinner| +64734|AAAAAAAAPNMPAAAA|64734|17|58|54|PM|second|evening|dinner| +64735|AAAAAAAAAOMPAAAA|64735|17|58|55|PM|second|evening|dinner| +64736|AAAAAAAABOMPAAAA|64736|17|58|56|PM|second|evening|dinner| +64737|AAAAAAAACOMPAAAA|64737|17|58|57|PM|second|evening|dinner| +64738|AAAAAAAADOMPAAAA|64738|17|58|58|PM|second|evening|dinner| +64739|AAAAAAAAEOMPAAAA|64739|17|58|59|PM|second|evening|dinner| +64740|AAAAAAAAFOMPAAAA|64740|17|59|0|PM|second|evening|dinner| +64741|AAAAAAAAGOMPAAAA|64741|17|59|1|PM|second|evening|dinner| +64742|AAAAAAAAHOMPAAAA|64742|17|59|2|PM|second|evening|dinner| +64743|AAAAAAAAIOMPAAAA|64743|17|59|3|PM|second|evening|dinner| +64744|AAAAAAAAJOMPAAAA|64744|17|59|4|PM|second|evening|dinner| +64745|AAAAAAAAKOMPAAAA|64745|17|59|5|PM|second|evening|dinner| +64746|AAAAAAAALOMPAAAA|64746|17|59|6|PM|second|evening|dinner| +64747|AAAAAAAAMOMPAAAA|64747|17|59|7|PM|second|evening|dinner| +64748|AAAAAAAANOMPAAAA|64748|17|59|8|PM|second|evening|dinner| +64749|AAAAAAAAOOMPAAAA|64749|17|59|9|PM|second|evening|dinner| +64750|AAAAAAAAPOMPAAAA|64750|17|59|10|PM|second|evening|dinner| +64751|AAAAAAAAAPMPAAAA|64751|17|59|11|PM|second|evening|dinner| +64752|AAAAAAAABPMPAAAA|64752|17|59|12|PM|second|evening|dinner| +64753|AAAAAAAACPMPAAAA|64753|17|59|13|PM|second|evening|dinner| +64754|AAAAAAAADPMPAAAA|64754|17|59|14|PM|second|evening|dinner| +64755|AAAAAAAAEPMPAAAA|64755|17|59|15|PM|second|evening|dinner| +64756|AAAAAAAAFPMPAAAA|64756|17|59|16|PM|second|evening|dinner| +64757|AAAAAAAAGPMPAAAA|64757|17|59|17|PM|second|evening|dinner| +64758|AAAAAAAAHPMPAAAA|64758|17|59|18|PM|second|evening|dinner| +64759|AAAAAAAAIPMPAAAA|64759|17|59|19|PM|second|evening|dinner| +64760|AAAAAAAAJPMPAAAA|64760|17|59|20|PM|second|evening|dinner| +64761|AAAAAAAAKPMPAAAA|64761|17|59|21|PM|second|evening|dinner| +64762|AAAAAAAALPMPAAAA|64762|17|59|22|PM|second|evening|dinner| +64763|AAAAAAAAMPMPAAAA|64763|17|59|23|PM|second|evening|dinner| +64764|AAAAAAAANPMPAAAA|64764|17|59|24|PM|second|evening|dinner| +64765|AAAAAAAAOPMPAAAA|64765|17|59|25|PM|second|evening|dinner| +64766|AAAAAAAAPPMPAAAA|64766|17|59|26|PM|second|evening|dinner| +64767|AAAAAAAAAANPAAAA|64767|17|59|27|PM|second|evening|dinner| +64768|AAAAAAAABANPAAAA|64768|17|59|28|PM|second|evening|dinner| +64769|AAAAAAAACANPAAAA|64769|17|59|29|PM|second|evening|dinner| +64770|AAAAAAAADANPAAAA|64770|17|59|30|PM|second|evening|dinner| +64771|AAAAAAAAEANPAAAA|64771|17|59|31|PM|second|evening|dinner| +64772|AAAAAAAAFANPAAAA|64772|17|59|32|PM|second|evening|dinner| +64773|AAAAAAAAGANPAAAA|64773|17|59|33|PM|second|evening|dinner| +64774|AAAAAAAAHANPAAAA|64774|17|59|34|PM|second|evening|dinner| +64775|AAAAAAAAIANPAAAA|64775|17|59|35|PM|second|evening|dinner| +64776|AAAAAAAAJANPAAAA|64776|17|59|36|PM|second|evening|dinner| +64777|AAAAAAAAKANPAAAA|64777|17|59|37|PM|second|evening|dinner| +64778|AAAAAAAALANPAAAA|64778|17|59|38|PM|second|evening|dinner| +64779|AAAAAAAAMANPAAAA|64779|17|59|39|PM|second|evening|dinner| +64780|AAAAAAAANANPAAAA|64780|17|59|40|PM|second|evening|dinner| +64781|AAAAAAAAOANPAAAA|64781|17|59|41|PM|second|evening|dinner| +64782|AAAAAAAAPANPAAAA|64782|17|59|42|PM|second|evening|dinner| +64783|AAAAAAAAABNPAAAA|64783|17|59|43|PM|second|evening|dinner| +64784|AAAAAAAABBNPAAAA|64784|17|59|44|PM|second|evening|dinner| +64785|AAAAAAAACBNPAAAA|64785|17|59|45|PM|second|evening|dinner| +64786|AAAAAAAADBNPAAAA|64786|17|59|46|PM|second|evening|dinner| +64787|AAAAAAAAEBNPAAAA|64787|17|59|47|PM|second|evening|dinner| +64788|AAAAAAAAFBNPAAAA|64788|17|59|48|PM|second|evening|dinner| +64789|AAAAAAAAGBNPAAAA|64789|17|59|49|PM|second|evening|dinner| +64790|AAAAAAAAHBNPAAAA|64790|17|59|50|PM|second|evening|dinner| +64791|AAAAAAAAIBNPAAAA|64791|17|59|51|PM|second|evening|dinner| +64792|AAAAAAAAJBNPAAAA|64792|17|59|52|PM|second|evening|dinner| +64793|AAAAAAAAKBNPAAAA|64793|17|59|53|PM|second|evening|dinner| +64794|AAAAAAAALBNPAAAA|64794|17|59|54|PM|second|evening|dinner| +64795|AAAAAAAAMBNPAAAA|64795|17|59|55|PM|second|evening|dinner| +64796|AAAAAAAANBNPAAAA|64796|17|59|56|PM|second|evening|dinner| +64797|AAAAAAAAOBNPAAAA|64797|17|59|57|PM|second|evening|dinner| +64798|AAAAAAAAPBNPAAAA|64798|17|59|58|PM|second|evening|dinner| +64799|AAAAAAAAACNPAAAA|64799|17|59|59|PM|second|evening|dinner| +64800|AAAAAAAABCNPAAAA|64800|18|0|0|PM|second|evening|dinner| +64801|AAAAAAAACCNPAAAA|64801|18|0|1|PM|second|evening|dinner| +64802|AAAAAAAADCNPAAAA|64802|18|0|2|PM|second|evening|dinner| +64803|AAAAAAAAECNPAAAA|64803|18|0|3|PM|second|evening|dinner| +64804|AAAAAAAAFCNPAAAA|64804|18|0|4|PM|second|evening|dinner| +64805|AAAAAAAAGCNPAAAA|64805|18|0|5|PM|second|evening|dinner| +64806|AAAAAAAAHCNPAAAA|64806|18|0|6|PM|second|evening|dinner| +64807|AAAAAAAAICNPAAAA|64807|18|0|7|PM|second|evening|dinner| +64808|AAAAAAAAJCNPAAAA|64808|18|0|8|PM|second|evening|dinner| +64809|AAAAAAAAKCNPAAAA|64809|18|0|9|PM|second|evening|dinner| +64810|AAAAAAAALCNPAAAA|64810|18|0|10|PM|second|evening|dinner| +64811|AAAAAAAAMCNPAAAA|64811|18|0|11|PM|second|evening|dinner| +64812|AAAAAAAANCNPAAAA|64812|18|0|12|PM|second|evening|dinner| +64813|AAAAAAAAOCNPAAAA|64813|18|0|13|PM|second|evening|dinner| +64814|AAAAAAAAPCNPAAAA|64814|18|0|14|PM|second|evening|dinner| +64815|AAAAAAAAADNPAAAA|64815|18|0|15|PM|second|evening|dinner| +64816|AAAAAAAABDNPAAAA|64816|18|0|16|PM|second|evening|dinner| +64817|AAAAAAAACDNPAAAA|64817|18|0|17|PM|second|evening|dinner| +64818|AAAAAAAADDNPAAAA|64818|18|0|18|PM|second|evening|dinner| +64819|AAAAAAAAEDNPAAAA|64819|18|0|19|PM|second|evening|dinner| +64820|AAAAAAAAFDNPAAAA|64820|18|0|20|PM|second|evening|dinner| +64821|AAAAAAAAGDNPAAAA|64821|18|0|21|PM|second|evening|dinner| +64822|AAAAAAAAHDNPAAAA|64822|18|0|22|PM|second|evening|dinner| +64823|AAAAAAAAIDNPAAAA|64823|18|0|23|PM|second|evening|dinner| +64824|AAAAAAAAJDNPAAAA|64824|18|0|24|PM|second|evening|dinner| +64825|AAAAAAAAKDNPAAAA|64825|18|0|25|PM|second|evening|dinner| +64826|AAAAAAAALDNPAAAA|64826|18|0|26|PM|second|evening|dinner| +64827|AAAAAAAAMDNPAAAA|64827|18|0|27|PM|second|evening|dinner| +64828|AAAAAAAANDNPAAAA|64828|18|0|28|PM|second|evening|dinner| +64829|AAAAAAAAODNPAAAA|64829|18|0|29|PM|second|evening|dinner| +64830|AAAAAAAAPDNPAAAA|64830|18|0|30|PM|second|evening|dinner| +64831|AAAAAAAAAENPAAAA|64831|18|0|31|PM|second|evening|dinner| +64832|AAAAAAAABENPAAAA|64832|18|0|32|PM|second|evening|dinner| +64833|AAAAAAAACENPAAAA|64833|18|0|33|PM|second|evening|dinner| +64834|AAAAAAAADENPAAAA|64834|18|0|34|PM|second|evening|dinner| +64835|AAAAAAAAEENPAAAA|64835|18|0|35|PM|second|evening|dinner| +64836|AAAAAAAAFENPAAAA|64836|18|0|36|PM|second|evening|dinner| +64837|AAAAAAAAGENPAAAA|64837|18|0|37|PM|second|evening|dinner| +64838|AAAAAAAAHENPAAAA|64838|18|0|38|PM|second|evening|dinner| +64839|AAAAAAAAIENPAAAA|64839|18|0|39|PM|second|evening|dinner| +64840|AAAAAAAAJENPAAAA|64840|18|0|40|PM|second|evening|dinner| +64841|AAAAAAAAKENPAAAA|64841|18|0|41|PM|second|evening|dinner| +64842|AAAAAAAALENPAAAA|64842|18|0|42|PM|second|evening|dinner| +64843|AAAAAAAAMENPAAAA|64843|18|0|43|PM|second|evening|dinner| +64844|AAAAAAAANENPAAAA|64844|18|0|44|PM|second|evening|dinner| +64845|AAAAAAAAOENPAAAA|64845|18|0|45|PM|second|evening|dinner| +64846|AAAAAAAAPENPAAAA|64846|18|0|46|PM|second|evening|dinner| +64847|AAAAAAAAAFNPAAAA|64847|18|0|47|PM|second|evening|dinner| +64848|AAAAAAAABFNPAAAA|64848|18|0|48|PM|second|evening|dinner| +64849|AAAAAAAACFNPAAAA|64849|18|0|49|PM|second|evening|dinner| +64850|AAAAAAAADFNPAAAA|64850|18|0|50|PM|second|evening|dinner| +64851|AAAAAAAAEFNPAAAA|64851|18|0|51|PM|second|evening|dinner| +64852|AAAAAAAAFFNPAAAA|64852|18|0|52|PM|second|evening|dinner| +64853|AAAAAAAAGFNPAAAA|64853|18|0|53|PM|second|evening|dinner| +64854|AAAAAAAAHFNPAAAA|64854|18|0|54|PM|second|evening|dinner| +64855|AAAAAAAAIFNPAAAA|64855|18|0|55|PM|second|evening|dinner| +64856|AAAAAAAAJFNPAAAA|64856|18|0|56|PM|second|evening|dinner| +64857|AAAAAAAAKFNPAAAA|64857|18|0|57|PM|second|evening|dinner| +64858|AAAAAAAALFNPAAAA|64858|18|0|58|PM|second|evening|dinner| +64859|AAAAAAAAMFNPAAAA|64859|18|0|59|PM|second|evening|dinner| +64860|AAAAAAAANFNPAAAA|64860|18|1|0|PM|second|evening|dinner| +64861|AAAAAAAAOFNPAAAA|64861|18|1|1|PM|second|evening|dinner| +64862|AAAAAAAAPFNPAAAA|64862|18|1|2|PM|second|evening|dinner| +64863|AAAAAAAAAGNPAAAA|64863|18|1|3|PM|second|evening|dinner| +64864|AAAAAAAABGNPAAAA|64864|18|1|4|PM|second|evening|dinner| +64865|AAAAAAAACGNPAAAA|64865|18|1|5|PM|second|evening|dinner| +64866|AAAAAAAADGNPAAAA|64866|18|1|6|PM|second|evening|dinner| +64867|AAAAAAAAEGNPAAAA|64867|18|1|7|PM|second|evening|dinner| +64868|AAAAAAAAFGNPAAAA|64868|18|1|8|PM|second|evening|dinner| +64869|AAAAAAAAGGNPAAAA|64869|18|1|9|PM|second|evening|dinner| +64870|AAAAAAAAHGNPAAAA|64870|18|1|10|PM|second|evening|dinner| +64871|AAAAAAAAIGNPAAAA|64871|18|1|11|PM|second|evening|dinner| +64872|AAAAAAAAJGNPAAAA|64872|18|1|12|PM|second|evening|dinner| +64873|AAAAAAAAKGNPAAAA|64873|18|1|13|PM|second|evening|dinner| +64874|AAAAAAAALGNPAAAA|64874|18|1|14|PM|second|evening|dinner| +64875|AAAAAAAAMGNPAAAA|64875|18|1|15|PM|second|evening|dinner| +64876|AAAAAAAANGNPAAAA|64876|18|1|16|PM|second|evening|dinner| +64877|AAAAAAAAOGNPAAAA|64877|18|1|17|PM|second|evening|dinner| +64878|AAAAAAAAPGNPAAAA|64878|18|1|18|PM|second|evening|dinner| +64879|AAAAAAAAAHNPAAAA|64879|18|1|19|PM|second|evening|dinner| +64880|AAAAAAAABHNPAAAA|64880|18|1|20|PM|second|evening|dinner| +64881|AAAAAAAACHNPAAAA|64881|18|1|21|PM|second|evening|dinner| +64882|AAAAAAAADHNPAAAA|64882|18|1|22|PM|second|evening|dinner| +64883|AAAAAAAAEHNPAAAA|64883|18|1|23|PM|second|evening|dinner| +64884|AAAAAAAAFHNPAAAA|64884|18|1|24|PM|second|evening|dinner| +64885|AAAAAAAAGHNPAAAA|64885|18|1|25|PM|second|evening|dinner| +64886|AAAAAAAAHHNPAAAA|64886|18|1|26|PM|second|evening|dinner| +64887|AAAAAAAAIHNPAAAA|64887|18|1|27|PM|second|evening|dinner| +64888|AAAAAAAAJHNPAAAA|64888|18|1|28|PM|second|evening|dinner| +64889|AAAAAAAAKHNPAAAA|64889|18|1|29|PM|second|evening|dinner| +64890|AAAAAAAALHNPAAAA|64890|18|1|30|PM|second|evening|dinner| +64891|AAAAAAAAMHNPAAAA|64891|18|1|31|PM|second|evening|dinner| +64892|AAAAAAAANHNPAAAA|64892|18|1|32|PM|second|evening|dinner| +64893|AAAAAAAAOHNPAAAA|64893|18|1|33|PM|second|evening|dinner| +64894|AAAAAAAAPHNPAAAA|64894|18|1|34|PM|second|evening|dinner| +64895|AAAAAAAAAINPAAAA|64895|18|1|35|PM|second|evening|dinner| +64896|AAAAAAAABINPAAAA|64896|18|1|36|PM|second|evening|dinner| +64897|AAAAAAAACINPAAAA|64897|18|1|37|PM|second|evening|dinner| +64898|AAAAAAAADINPAAAA|64898|18|1|38|PM|second|evening|dinner| +64899|AAAAAAAAEINPAAAA|64899|18|1|39|PM|second|evening|dinner| +64900|AAAAAAAAFINPAAAA|64900|18|1|40|PM|second|evening|dinner| +64901|AAAAAAAAGINPAAAA|64901|18|1|41|PM|second|evening|dinner| +64902|AAAAAAAAHINPAAAA|64902|18|1|42|PM|second|evening|dinner| +64903|AAAAAAAAIINPAAAA|64903|18|1|43|PM|second|evening|dinner| +64904|AAAAAAAAJINPAAAA|64904|18|1|44|PM|second|evening|dinner| +64905|AAAAAAAAKINPAAAA|64905|18|1|45|PM|second|evening|dinner| +64906|AAAAAAAALINPAAAA|64906|18|1|46|PM|second|evening|dinner| +64907|AAAAAAAAMINPAAAA|64907|18|1|47|PM|second|evening|dinner| +64908|AAAAAAAANINPAAAA|64908|18|1|48|PM|second|evening|dinner| +64909|AAAAAAAAOINPAAAA|64909|18|1|49|PM|second|evening|dinner| +64910|AAAAAAAAPINPAAAA|64910|18|1|50|PM|second|evening|dinner| +64911|AAAAAAAAAJNPAAAA|64911|18|1|51|PM|second|evening|dinner| +64912|AAAAAAAABJNPAAAA|64912|18|1|52|PM|second|evening|dinner| +64913|AAAAAAAACJNPAAAA|64913|18|1|53|PM|second|evening|dinner| +64914|AAAAAAAADJNPAAAA|64914|18|1|54|PM|second|evening|dinner| +64915|AAAAAAAAEJNPAAAA|64915|18|1|55|PM|second|evening|dinner| +64916|AAAAAAAAFJNPAAAA|64916|18|1|56|PM|second|evening|dinner| +64917|AAAAAAAAGJNPAAAA|64917|18|1|57|PM|second|evening|dinner| +64918|AAAAAAAAHJNPAAAA|64918|18|1|58|PM|second|evening|dinner| +64919|AAAAAAAAIJNPAAAA|64919|18|1|59|PM|second|evening|dinner| +64920|AAAAAAAAJJNPAAAA|64920|18|2|0|PM|second|evening|dinner| +64921|AAAAAAAAKJNPAAAA|64921|18|2|1|PM|second|evening|dinner| +64922|AAAAAAAALJNPAAAA|64922|18|2|2|PM|second|evening|dinner| +64923|AAAAAAAAMJNPAAAA|64923|18|2|3|PM|second|evening|dinner| +64924|AAAAAAAANJNPAAAA|64924|18|2|4|PM|second|evening|dinner| +64925|AAAAAAAAOJNPAAAA|64925|18|2|5|PM|second|evening|dinner| +64926|AAAAAAAAPJNPAAAA|64926|18|2|6|PM|second|evening|dinner| +64927|AAAAAAAAAKNPAAAA|64927|18|2|7|PM|second|evening|dinner| +64928|AAAAAAAABKNPAAAA|64928|18|2|8|PM|second|evening|dinner| +64929|AAAAAAAACKNPAAAA|64929|18|2|9|PM|second|evening|dinner| +64930|AAAAAAAADKNPAAAA|64930|18|2|10|PM|second|evening|dinner| +64931|AAAAAAAAEKNPAAAA|64931|18|2|11|PM|second|evening|dinner| +64932|AAAAAAAAFKNPAAAA|64932|18|2|12|PM|second|evening|dinner| +64933|AAAAAAAAGKNPAAAA|64933|18|2|13|PM|second|evening|dinner| +64934|AAAAAAAAHKNPAAAA|64934|18|2|14|PM|second|evening|dinner| +64935|AAAAAAAAIKNPAAAA|64935|18|2|15|PM|second|evening|dinner| +64936|AAAAAAAAJKNPAAAA|64936|18|2|16|PM|second|evening|dinner| +64937|AAAAAAAAKKNPAAAA|64937|18|2|17|PM|second|evening|dinner| +64938|AAAAAAAALKNPAAAA|64938|18|2|18|PM|second|evening|dinner| +64939|AAAAAAAAMKNPAAAA|64939|18|2|19|PM|second|evening|dinner| +64940|AAAAAAAANKNPAAAA|64940|18|2|20|PM|second|evening|dinner| +64941|AAAAAAAAOKNPAAAA|64941|18|2|21|PM|second|evening|dinner| +64942|AAAAAAAAPKNPAAAA|64942|18|2|22|PM|second|evening|dinner| +64943|AAAAAAAAALNPAAAA|64943|18|2|23|PM|second|evening|dinner| +64944|AAAAAAAABLNPAAAA|64944|18|2|24|PM|second|evening|dinner| +64945|AAAAAAAACLNPAAAA|64945|18|2|25|PM|second|evening|dinner| +64946|AAAAAAAADLNPAAAA|64946|18|2|26|PM|second|evening|dinner| +64947|AAAAAAAAELNPAAAA|64947|18|2|27|PM|second|evening|dinner| +64948|AAAAAAAAFLNPAAAA|64948|18|2|28|PM|second|evening|dinner| +64949|AAAAAAAAGLNPAAAA|64949|18|2|29|PM|second|evening|dinner| +64950|AAAAAAAAHLNPAAAA|64950|18|2|30|PM|second|evening|dinner| +64951|AAAAAAAAILNPAAAA|64951|18|2|31|PM|second|evening|dinner| +64952|AAAAAAAAJLNPAAAA|64952|18|2|32|PM|second|evening|dinner| +64953|AAAAAAAAKLNPAAAA|64953|18|2|33|PM|second|evening|dinner| +64954|AAAAAAAALLNPAAAA|64954|18|2|34|PM|second|evening|dinner| +64955|AAAAAAAAMLNPAAAA|64955|18|2|35|PM|second|evening|dinner| +64956|AAAAAAAANLNPAAAA|64956|18|2|36|PM|second|evening|dinner| +64957|AAAAAAAAOLNPAAAA|64957|18|2|37|PM|second|evening|dinner| +64958|AAAAAAAAPLNPAAAA|64958|18|2|38|PM|second|evening|dinner| +64959|AAAAAAAAAMNPAAAA|64959|18|2|39|PM|second|evening|dinner| +64960|AAAAAAAABMNPAAAA|64960|18|2|40|PM|second|evening|dinner| +64961|AAAAAAAACMNPAAAA|64961|18|2|41|PM|second|evening|dinner| +64962|AAAAAAAADMNPAAAA|64962|18|2|42|PM|second|evening|dinner| +64963|AAAAAAAAEMNPAAAA|64963|18|2|43|PM|second|evening|dinner| +64964|AAAAAAAAFMNPAAAA|64964|18|2|44|PM|second|evening|dinner| +64965|AAAAAAAAGMNPAAAA|64965|18|2|45|PM|second|evening|dinner| +64966|AAAAAAAAHMNPAAAA|64966|18|2|46|PM|second|evening|dinner| +64967|AAAAAAAAIMNPAAAA|64967|18|2|47|PM|second|evening|dinner| +64968|AAAAAAAAJMNPAAAA|64968|18|2|48|PM|second|evening|dinner| +64969|AAAAAAAAKMNPAAAA|64969|18|2|49|PM|second|evening|dinner| +64970|AAAAAAAALMNPAAAA|64970|18|2|50|PM|second|evening|dinner| +64971|AAAAAAAAMMNPAAAA|64971|18|2|51|PM|second|evening|dinner| +64972|AAAAAAAANMNPAAAA|64972|18|2|52|PM|second|evening|dinner| +64973|AAAAAAAAOMNPAAAA|64973|18|2|53|PM|second|evening|dinner| +64974|AAAAAAAAPMNPAAAA|64974|18|2|54|PM|second|evening|dinner| +64975|AAAAAAAAANNPAAAA|64975|18|2|55|PM|second|evening|dinner| +64976|AAAAAAAABNNPAAAA|64976|18|2|56|PM|second|evening|dinner| +64977|AAAAAAAACNNPAAAA|64977|18|2|57|PM|second|evening|dinner| +64978|AAAAAAAADNNPAAAA|64978|18|2|58|PM|second|evening|dinner| +64979|AAAAAAAAENNPAAAA|64979|18|2|59|PM|second|evening|dinner| +64980|AAAAAAAAFNNPAAAA|64980|18|3|0|PM|second|evening|dinner| +64981|AAAAAAAAGNNPAAAA|64981|18|3|1|PM|second|evening|dinner| +64982|AAAAAAAAHNNPAAAA|64982|18|3|2|PM|second|evening|dinner| +64983|AAAAAAAAINNPAAAA|64983|18|3|3|PM|second|evening|dinner| +64984|AAAAAAAAJNNPAAAA|64984|18|3|4|PM|second|evening|dinner| +64985|AAAAAAAAKNNPAAAA|64985|18|3|5|PM|second|evening|dinner| +64986|AAAAAAAALNNPAAAA|64986|18|3|6|PM|second|evening|dinner| +64987|AAAAAAAAMNNPAAAA|64987|18|3|7|PM|second|evening|dinner| +64988|AAAAAAAANNNPAAAA|64988|18|3|8|PM|second|evening|dinner| +64989|AAAAAAAAONNPAAAA|64989|18|3|9|PM|second|evening|dinner| +64990|AAAAAAAAPNNPAAAA|64990|18|3|10|PM|second|evening|dinner| +64991|AAAAAAAAAONPAAAA|64991|18|3|11|PM|second|evening|dinner| +64992|AAAAAAAABONPAAAA|64992|18|3|12|PM|second|evening|dinner| +64993|AAAAAAAACONPAAAA|64993|18|3|13|PM|second|evening|dinner| +64994|AAAAAAAADONPAAAA|64994|18|3|14|PM|second|evening|dinner| +64995|AAAAAAAAEONPAAAA|64995|18|3|15|PM|second|evening|dinner| +64996|AAAAAAAAFONPAAAA|64996|18|3|16|PM|second|evening|dinner| +64997|AAAAAAAAGONPAAAA|64997|18|3|17|PM|second|evening|dinner| +64998|AAAAAAAAHONPAAAA|64998|18|3|18|PM|second|evening|dinner| +64999|AAAAAAAAIONPAAAA|64999|18|3|19|PM|second|evening|dinner| +65000|AAAAAAAAJONPAAAA|65000|18|3|20|PM|second|evening|dinner| +65001|AAAAAAAAKONPAAAA|65001|18|3|21|PM|second|evening|dinner| +65002|AAAAAAAALONPAAAA|65002|18|3|22|PM|second|evening|dinner| +65003|AAAAAAAAMONPAAAA|65003|18|3|23|PM|second|evening|dinner| +65004|AAAAAAAANONPAAAA|65004|18|3|24|PM|second|evening|dinner| +65005|AAAAAAAAOONPAAAA|65005|18|3|25|PM|second|evening|dinner| +65006|AAAAAAAAPONPAAAA|65006|18|3|26|PM|second|evening|dinner| +65007|AAAAAAAAAPNPAAAA|65007|18|3|27|PM|second|evening|dinner| +65008|AAAAAAAABPNPAAAA|65008|18|3|28|PM|second|evening|dinner| +65009|AAAAAAAACPNPAAAA|65009|18|3|29|PM|second|evening|dinner| +65010|AAAAAAAADPNPAAAA|65010|18|3|30|PM|second|evening|dinner| +65011|AAAAAAAAEPNPAAAA|65011|18|3|31|PM|second|evening|dinner| +65012|AAAAAAAAFPNPAAAA|65012|18|3|32|PM|second|evening|dinner| +65013|AAAAAAAAGPNPAAAA|65013|18|3|33|PM|second|evening|dinner| +65014|AAAAAAAAHPNPAAAA|65014|18|3|34|PM|second|evening|dinner| +65015|AAAAAAAAIPNPAAAA|65015|18|3|35|PM|second|evening|dinner| +65016|AAAAAAAAJPNPAAAA|65016|18|3|36|PM|second|evening|dinner| +65017|AAAAAAAAKPNPAAAA|65017|18|3|37|PM|second|evening|dinner| +65018|AAAAAAAALPNPAAAA|65018|18|3|38|PM|second|evening|dinner| +65019|AAAAAAAAMPNPAAAA|65019|18|3|39|PM|second|evening|dinner| +65020|AAAAAAAANPNPAAAA|65020|18|3|40|PM|second|evening|dinner| +65021|AAAAAAAAOPNPAAAA|65021|18|3|41|PM|second|evening|dinner| +65022|AAAAAAAAPPNPAAAA|65022|18|3|42|PM|second|evening|dinner| +65023|AAAAAAAAAAOPAAAA|65023|18|3|43|PM|second|evening|dinner| +65024|AAAAAAAABAOPAAAA|65024|18|3|44|PM|second|evening|dinner| +65025|AAAAAAAACAOPAAAA|65025|18|3|45|PM|second|evening|dinner| +65026|AAAAAAAADAOPAAAA|65026|18|3|46|PM|second|evening|dinner| +65027|AAAAAAAAEAOPAAAA|65027|18|3|47|PM|second|evening|dinner| +65028|AAAAAAAAFAOPAAAA|65028|18|3|48|PM|second|evening|dinner| +65029|AAAAAAAAGAOPAAAA|65029|18|3|49|PM|second|evening|dinner| +65030|AAAAAAAAHAOPAAAA|65030|18|3|50|PM|second|evening|dinner| +65031|AAAAAAAAIAOPAAAA|65031|18|3|51|PM|second|evening|dinner| +65032|AAAAAAAAJAOPAAAA|65032|18|3|52|PM|second|evening|dinner| +65033|AAAAAAAAKAOPAAAA|65033|18|3|53|PM|second|evening|dinner| +65034|AAAAAAAALAOPAAAA|65034|18|3|54|PM|second|evening|dinner| +65035|AAAAAAAAMAOPAAAA|65035|18|3|55|PM|second|evening|dinner| +65036|AAAAAAAANAOPAAAA|65036|18|3|56|PM|second|evening|dinner| +65037|AAAAAAAAOAOPAAAA|65037|18|3|57|PM|second|evening|dinner| +65038|AAAAAAAAPAOPAAAA|65038|18|3|58|PM|second|evening|dinner| +65039|AAAAAAAAABOPAAAA|65039|18|3|59|PM|second|evening|dinner| +65040|AAAAAAAABBOPAAAA|65040|18|4|0|PM|second|evening|dinner| +65041|AAAAAAAACBOPAAAA|65041|18|4|1|PM|second|evening|dinner| +65042|AAAAAAAADBOPAAAA|65042|18|4|2|PM|second|evening|dinner| +65043|AAAAAAAAEBOPAAAA|65043|18|4|3|PM|second|evening|dinner| +65044|AAAAAAAAFBOPAAAA|65044|18|4|4|PM|second|evening|dinner| +65045|AAAAAAAAGBOPAAAA|65045|18|4|5|PM|second|evening|dinner| +65046|AAAAAAAAHBOPAAAA|65046|18|4|6|PM|second|evening|dinner| +65047|AAAAAAAAIBOPAAAA|65047|18|4|7|PM|second|evening|dinner| +65048|AAAAAAAAJBOPAAAA|65048|18|4|8|PM|second|evening|dinner| +65049|AAAAAAAAKBOPAAAA|65049|18|4|9|PM|second|evening|dinner| +65050|AAAAAAAALBOPAAAA|65050|18|4|10|PM|second|evening|dinner| +65051|AAAAAAAAMBOPAAAA|65051|18|4|11|PM|second|evening|dinner| +65052|AAAAAAAANBOPAAAA|65052|18|4|12|PM|second|evening|dinner| +65053|AAAAAAAAOBOPAAAA|65053|18|4|13|PM|second|evening|dinner| +65054|AAAAAAAAPBOPAAAA|65054|18|4|14|PM|second|evening|dinner| +65055|AAAAAAAAACOPAAAA|65055|18|4|15|PM|second|evening|dinner| +65056|AAAAAAAABCOPAAAA|65056|18|4|16|PM|second|evening|dinner| +65057|AAAAAAAACCOPAAAA|65057|18|4|17|PM|second|evening|dinner| +65058|AAAAAAAADCOPAAAA|65058|18|4|18|PM|second|evening|dinner| +65059|AAAAAAAAECOPAAAA|65059|18|4|19|PM|second|evening|dinner| +65060|AAAAAAAAFCOPAAAA|65060|18|4|20|PM|second|evening|dinner| +65061|AAAAAAAAGCOPAAAA|65061|18|4|21|PM|second|evening|dinner| +65062|AAAAAAAAHCOPAAAA|65062|18|4|22|PM|second|evening|dinner| +65063|AAAAAAAAICOPAAAA|65063|18|4|23|PM|second|evening|dinner| +65064|AAAAAAAAJCOPAAAA|65064|18|4|24|PM|second|evening|dinner| +65065|AAAAAAAAKCOPAAAA|65065|18|4|25|PM|second|evening|dinner| +65066|AAAAAAAALCOPAAAA|65066|18|4|26|PM|second|evening|dinner| +65067|AAAAAAAAMCOPAAAA|65067|18|4|27|PM|second|evening|dinner| +65068|AAAAAAAANCOPAAAA|65068|18|4|28|PM|second|evening|dinner| +65069|AAAAAAAAOCOPAAAA|65069|18|4|29|PM|second|evening|dinner| +65070|AAAAAAAAPCOPAAAA|65070|18|4|30|PM|second|evening|dinner| +65071|AAAAAAAAADOPAAAA|65071|18|4|31|PM|second|evening|dinner| +65072|AAAAAAAABDOPAAAA|65072|18|4|32|PM|second|evening|dinner| +65073|AAAAAAAACDOPAAAA|65073|18|4|33|PM|second|evening|dinner| +65074|AAAAAAAADDOPAAAA|65074|18|4|34|PM|second|evening|dinner| +65075|AAAAAAAAEDOPAAAA|65075|18|4|35|PM|second|evening|dinner| +65076|AAAAAAAAFDOPAAAA|65076|18|4|36|PM|second|evening|dinner| +65077|AAAAAAAAGDOPAAAA|65077|18|4|37|PM|second|evening|dinner| +65078|AAAAAAAAHDOPAAAA|65078|18|4|38|PM|second|evening|dinner| +65079|AAAAAAAAIDOPAAAA|65079|18|4|39|PM|second|evening|dinner| +65080|AAAAAAAAJDOPAAAA|65080|18|4|40|PM|second|evening|dinner| +65081|AAAAAAAAKDOPAAAA|65081|18|4|41|PM|second|evening|dinner| +65082|AAAAAAAALDOPAAAA|65082|18|4|42|PM|second|evening|dinner| +65083|AAAAAAAAMDOPAAAA|65083|18|4|43|PM|second|evening|dinner| +65084|AAAAAAAANDOPAAAA|65084|18|4|44|PM|second|evening|dinner| +65085|AAAAAAAAODOPAAAA|65085|18|4|45|PM|second|evening|dinner| +65086|AAAAAAAAPDOPAAAA|65086|18|4|46|PM|second|evening|dinner| +65087|AAAAAAAAAEOPAAAA|65087|18|4|47|PM|second|evening|dinner| +65088|AAAAAAAABEOPAAAA|65088|18|4|48|PM|second|evening|dinner| +65089|AAAAAAAACEOPAAAA|65089|18|4|49|PM|second|evening|dinner| +65090|AAAAAAAADEOPAAAA|65090|18|4|50|PM|second|evening|dinner| +65091|AAAAAAAAEEOPAAAA|65091|18|4|51|PM|second|evening|dinner| +65092|AAAAAAAAFEOPAAAA|65092|18|4|52|PM|second|evening|dinner| +65093|AAAAAAAAGEOPAAAA|65093|18|4|53|PM|second|evening|dinner| +65094|AAAAAAAAHEOPAAAA|65094|18|4|54|PM|second|evening|dinner| +65095|AAAAAAAAIEOPAAAA|65095|18|4|55|PM|second|evening|dinner| +65096|AAAAAAAAJEOPAAAA|65096|18|4|56|PM|second|evening|dinner| +65097|AAAAAAAAKEOPAAAA|65097|18|4|57|PM|second|evening|dinner| +65098|AAAAAAAALEOPAAAA|65098|18|4|58|PM|second|evening|dinner| +65099|AAAAAAAAMEOPAAAA|65099|18|4|59|PM|second|evening|dinner| +65100|AAAAAAAANEOPAAAA|65100|18|5|0|PM|second|evening|dinner| +65101|AAAAAAAAOEOPAAAA|65101|18|5|1|PM|second|evening|dinner| +65102|AAAAAAAAPEOPAAAA|65102|18|5|2|PM|second|evening|dinner| +65103|AAAAAAAAAFOPAAAA|65103|18|5|3|PM|second|evening|dinner| +65104|AAAAAAAABFOPAAAA|65104|18|5|4|PM|second|evening|dinner| +65105|AAAAAAAACFOPAAAA|65105|18|5|5|PM|second|evening|dinner| +65106|AAAAAAAADFOPAAAA|65106|18|5|6|PM|second|evening|dinner| +65107|AAAAAAAAEFOPAAAA|65107|18|5|7|PM|second|evening|dinner| +65108|AAAAAAAAFFOPAAAA|65108|18|5|8|PM|second|evening|dinner| +65109|AAAAAAAAGFOPAAAA|65109|18|5|9|PM|second|evening|dinner| +65110|AAAAAAAAHFOPAAAA|65110|18|5|10|PM|second|evening|dinner| +65111|AAAAAAAAIFOPAAAA|65111|18|5|11|PM|second|evening|dinner| +65112|AAAAAAAAJFOPAAAA|65112|18|5|12|PM|second|evening|dinner| +65113|AAAAAAAAKFOPAAAA|65113|18|5|13|PM|second|evening|dinner| +65114|AAAAAAAALFOPAAAA|65114|18|5|14|PM|second|evening|dinner| +65115|AAAAAAAAMFOPAAAA|65115|18|5|15|PM|second|evening|dinner| +65116|AAAAAAAANFOPAAAA|65116|18|5|16|PM|second|evening|dinner| +65117|AAAAAAAAOFOPAAAA|65117|18|5|17|PM|second|evening|dinner| +65118|AAAAAAAAPFOPAAAA|65118|18|5|18|PM|second|evening|dinner| +65119|AAAAAAAAAGOPAAAA|65119|18|5|19|PM|second|evening|dinner| +65120|AAAAAAAABGOPAAAA|65120|18|5|20|PM|second|evening|dinner| +65121|AAAAAAAACGOPAAAA|65121|18|5|21|PM|second|evening|dinner| +65122|AAAAAAAADGOPAAAA|65122|18|5|22|PM|second|evening|dinner| +65123|AAAAAAAAEGOPAAAA|65123|18|5|23|PM|second|evening|dinner| +65124|AAAAAAAAFGOPAAAA|65124|18|5|24|PM|second|evening|dinner| +65125|AAAAAAAAGGOPAAAA|65125|18|5|25|PM|second|evening|dinner| +65126|AAAAAAAAHGOPAAAA|65126|18|5|26|PM|second|evening|dinner| +65127|AAAAAAAAIGOPAAAA|65127|18|5|27|PM|second|evening|dinner| +65128|AAAAAAAAJGOPAAAA|65128|18|5|28|PM|second|evening|dinner| +65129|AAAAAAAAKGOPAAAA|65129|18|5|29|PM|second|evening|dinner| +65130|AAAAAAAALGOPAAAA|65130|18|5|30|PM|second|evening|dinner| +65131|AAAAAAAAMGOPAAAA|65131|18|5|31|PM|second|evening|dinner| +65132|AAAAAAAANGOPAAAA|65132|18|5|32|PM|second|evening|dinner| +65133|AAAAAAAAOGOPAAAA|65133|18|5|33|PM|second|evening|dinner| +65134|AAAAAAAAPGOPAAAA|65134|18|5|34|PM|second|evening|dinner| +65135|AAAAAAAAAHOPAAAA|65135|18|5|35|PM|second|evening|dinner| +65136|AAAAAAAABHOPAAAA|65136|18|5|36|PM|second|evening|dinner| +65137|AAAAAAAACHOPAAAA|65137|18|5|37|PM|second|evening|dinner| +65138|AAAAAAAADHOPAAAA|65138|18|5|38|PM|second|evening|dinner| +65139|AAAAAAAAEHOPAAAA|65139|18|5|39|PM|second|evening|dinner| +65140|AAAAAAAAFHOPAAAA|65140|18|5|40|PM|second|evening|dinner| +65141|AAAAAAAAGHOPAAAA|65141|18|5|41|PM|second|evening|dinner| +65142|AAAAAAAAHHOPAAAA|65142|18|5|42|PM|second|evening|dinner| +65143|AAAAAAAAIHOPAAAA|65143|18|5|43|PM|second|evening|dinner| +65144|AAAAAAAAJHOPAAAA|65144|18|5|44|PM|second|evening|dinner| +65145|AAAAAAAAKHOPAAAA|65145|18|5|45|PM|second|evening|dinner| +65146|AAAAAAAALHOPAAAA|65146|18|5|46|PM|second|evening|dinner| +65147|AAAAAAAAMHOPAAAA|65147|18|5|47|PM|second|evening|dinner| +65148|AAAAAAAANHOPAAAA|65148|18|5|48|PM|second|evening|dinner| +65149|AAAAAAAAOHOPAAAA|65149|18|5|49|PM|second|evening|dinner| +65150|AAAAAAAAPHOPAAAA|65150|18|5|50|PM|second|evening|dinner| +65151|AAAAAAAAAIOPAAAA|65151|18|5|51|PM|second|evening|dinner| +65152|AAAAAAAABIOPAAAA|65152|18|5|52|PM|second|evening|dinner| +65153|AAAAAAAACIOPAAAA|65153|18|5|53|PM|second|evening|dinner| +65154|AAAAAAAADIOPAAAA|65154|18|5|54|PM|second|evening|dinner| +65155|AAAAAAAAEIOPAAAA|65155|18|5|55|PM|second|evening|dinner| +65156|AAAAAAAAFIOPAAAA|65156|18|5|56|PM|second|evening|dinner| +65157|AAAAAAAAGIOPAAAA|65157|18|5|57|PM|second|evening|dinner| +65158|AAAAAAAAHIOPAAAA|65158|18|5|58|PM|second|evening|dinner| +65159|AAAAAAAAIIOPAAAA|65159|18|5|59|PM|second|evening|dinner| +65160|AAAAAAAAJIOPAAAA|65160|18|6|0|PM|second|evening|dinner| +65161|AAAAAAAAKIOPAAAA|65161|18|6|1|PM|second|evening|dinner| +65162|AAAAAAAALIOPAAAA|65162|18|6|2|PM|second|evening|dinner| +65163|AAAAAAAAMIOPAAAA|65163|18|6|3|PM|second|evening|dinner| +65164|AAAAAAAANIOPAAAA|65164|18|6|4|PM|second|evening|dinner| +65165|AAAAAAAAOIOPAAAA|65165|18|6|5|PM|second|evening|dinner| +65166|AAAAAAAAPIOPAAAA|65166|18|6|6|PM|second|evening|dinner| +65167|AAAAAAAAAJOPAAAA|65167|18|6|7|PM|second|evening|dinner| +65168|AAAAAAAABJOPAAAA|65168|18|6|8|PM|second|evening|dinner| +65169|AAAAAAAACJOPAAAA|65169|18|6|9|PM|second|evening|dinner| +65170|AAAAAAAADJOPAAAA|65170|18|6|10|PM|second|evening|dinner| +65171|AAAAAAAAEJOPAAAA|65171|18|6|11|PM|second|evening|dinner| +65172|AAAAAAAAFJOPAAAA|65172|18|6|12|PM|second|evening|dinner| +65173|AAAAAAAAGJOPAAAA|65173|18|6|13|PM|second|evening|dinner| +65174|AAAAAAAAHJOPAAAA|65174|18|6|14|PM|second|evening|dinner| +65175|AAAAAAAAIJOPAAAA|65175|18|6|15|PM|second|evening|dinner| +65176|AAAAAAAAJJOPAAAA|65176|18|6|16|PM|second|evening|dinner| +65177|AAAAAAAAKJOPAAAA|65177|18|6|17|PM|second|evening|dinner| +65178|AAAAAAAALJOPAAAA|65178|18|6|18|PM|second|evening|dinner| +65179|AAAAAAAAMJOPAAAA|65179|18|6|19|PM|second|evening|dinner| +65180|AAAAAAAANJOPAAAA|65180|18|6|20|PM|second|evening|dinner| +65181|AAAAAAAAOJOPAAAA|65181|18|6|21|PM|second|evening|dinner| +65182|AAAAAAAAPJOPAAAA|65182|18|6|22|PM|second|evening|dinner| +65183|AAAAAAAAAKOPAAAA|65183|18|6|23|PM|second|evening|dinner| +65184|AAAAAAAABKOPAAAA|65184|18|6|24|PM|second|evening|dinner| +65185|AAAAAAAACKOPAAAA|65185|18|6|25|PM|second|evening|dinner| +65186|AAAAAAAADKOPAAAA|65186|18|6|26|PM|second|evening|dinner| +65187|AAAAAAAAEKOPAAAA|65187|18|6|27|PM|second|evening|dinner| +65188|AAAAAAAAFKOPAAAA|65188|18|6|28|PM|second|evening|dinner| +65189|AAAAAAAAGKOPAAAA|65189|18|6|29|PM|second|evening|dinner| +65190|AAAAAAAAHKOPAAAA|65190|18|6|30|PM|second|evening|dinner| +65191|AAAAAAAAIKOPAAAA|65191|18|6|31|PM|second|evening|dinner| +65192|AAAAAAAAJKOPAAAA|65192|18|6|32|PM|second|evening|dinner| +65193|AAAAAAAAKKOPAAAA|65193|18|6|33|PM|second|evening|dinner| +65194|AAAAAAAALKOPAAAA|65194|18|6|34|PM|second|evening|dinner| +65195|AAAAAAAAMKOPAAAA|65195|18|6|35|PM|second|evening|dinner| +65196|AAAAAAAANKOPAAAA|65196|18|6|36|PM|second|evening|dinner| +65197|AAAAAAAAOKOPAAAA|65197|18|6|37|PM|second|evening|dinner| +65198|AAAAAAAAPKOPAAAA|65198|18|6|38|PM|second|evening|dinner| +65199|AAAAAAAAALOPAAAA|65199|18|6|39|PM|second|evening|dinner| +65200|AAAAAAAABLOPAAAA|65200|18|6|40|PM|second|evening|dinner| +65201|AAAAAAAACLOPAAAA|65201|18|6|41|PM|second|evening|dinner| +65202|AAAAAAAADLOPAAAA|65202|18|6|42|PM|second|evening|dinner| +65203|AAAAAAAAELOPAAAA|65203|18|6|43|PM|second|evening|dinner| +65204|AAAAAAAAFLOPAAAA|65204|18|6|44|PM|second|evening|dinner| +65205|AAAAAAAAGLOPAAAA|65205|18|6|45|PM|second|evening|dinner| +65206|AAAAAAAAHLOPAAAA|65206|18|6|46|PM|second|evening|dinner| +65207|AAAAAAAAILOPAAAA|65207|18|6|47|PM|second|evening|dinner| +65208|AAAAAAAAJLOPAAAA|65208|18|6|48|PM|second|evening|dinner| +65209|AAAAAAAAKLOPAAAA|65209|18|6|49|PM|second|evening|dinner| +65210|AAAAAAAALLOPAAAA|65210|18|6|50|PM|second|evening|dinner| +65211|AAAAAAAAMLOPAAAA|65211|18|6|51|PM|second|evening|dinner| +65212|AAAAAAAANLOPAAAA|65212|18|6|52|PM|second|evening|dinner| +65213|AAAAAAAAOLOPAAAA|65213|18|6|53|PM|second|evening|dinner| +65214|AAAAAAAAPLOPAAAA|65214|18|6|54|PM|second|evening|dinner| +65215|AAAAAAAAAMOPAAAA|65215|18|6|55|PM|second|evening|dinner| +65216|AAAAAAAABMOPAAAA|65216|18|6|56|PM|second|evening|dinner| +65217|AAAAAAAACMOPAAAA|65217|18|6|57|PM|second|evening|dinner| +65218|AAAAAAAADMOPAAAA|65218|18|6|58|PM|second|evening|dinner| +65219|AAAAAAAAEMOPAAAA|65219|18|6|59|PM|second|evening|dinner| +65220|AAAAAAAAFMOPAAAA|65220|18|7|0|PM|second|evening|dinner| +65221|AAAAAAAAGMOPAAAA|65221|18|7|1|PM|second|evening|dinner| +65222|AAAAAAAAHMOPAAAA|65222|18|7|2|PM|second|evening|dinner| +65223|AAAAAAAAIMOPAAAA|65223|18|7|3|PM|second|evening|dinner| +65224|AAAAAAAAJMOPAAAA|65224|18|7|4|PM|second|evening|dinner| +65225|AAAAAAAAKMOPAAAA|65225|18|7|5|PM|second|evening|dinner| +65226|AAAAAAAALMOPAAAA|65226|18|7|6|PM|second|evening|dinner| +65227|AAAAAAAAMMOPAAAA|65227|18|7|7|PM|second|evening|dinner| +65228|AAAAAAAANMOPAAAA|65228|18|7|8|PM|second|evening|dinner| +65229|AAAAAAAAOMOPAAAA|65229|18|7|9|PM|second|evening|dinner| +65230|AAAAAAAAPMOPAAAA|65230|18|7|10|PM|second|evening|dinner| +65231|AAAAAAAAANOPAAAA|65231|18|7|11|PM|second|evening|dinner| +65232|AAAAAAAABNOPAAAA|65232|18|7|12|PM|second|evening|dinner| +65233|AAAAAAAACNOPAAAA|65233|18|7|13|PM|second|evening|dinner| +65234|AAAAAAAADNOPAAAA|65234|18|7|14|PM|second|evening|dinner| +65235|AAAAAAAAENOPAAAA|65235|18|7|15|PM|second|evening|dinner| +65236|AAAAAAAAFNOPAAAA|65236|18|7|16|PM|second|evening|dinner| +65237|AAAAAAAAGNOPAAAA|65237|18|7|17|PM|second|evening|dinner| +65238|AAAAAAAAHNOPAAAA|65238|18|7|18|PM|second|evening|dinner| +65239|AAAAAAAAINOPAAAA|65239|18|7|19|PM|second|evening|dinner| +65240|AAAAAAAAJNOPAAAA|65240|18|7|20|PM|second|evening|dinner| +65241|AAAAAAAAKNOPAAAA|65241|18|7|21|PM|second|evening|dinner| +65242|AAAAAAAALNOPAAAA|65242|18|7|22|PM|second|evening|dinner| +65243|AAAAAAAAMNOPAAAA|65243|18|7|23|PM|second|evening|dinner| +65244|AAAAAAAANNOPAAAA|65244|18|7|24|PM|second|evening|dinner| +65245|AAAAAAAAONOPAAAA|65245|18|7|25|PM|second|evening|dinner| +65246|AAAAAAAAPNOPAAAA|65246|18|7|26|PM|second|evening|dinner| +65247|AAAAAAAAAOOPAAAA|65247|18|7|27|PM|second|evening|dinner| +65248|AAAAAAAABOOPAAAA|65248|18|7|28|PM|second|evening|dinner| +65249|AAAAAAAACOOPAAAA|65249|18|7|29|PM|second|evening|dinner| +65250|AAAAAAAADOOPAAAA|65250|18|7|30|PM|second|evening|dinner| +65251|AAAAAAAAEOOPAAAA|65251|18|7|31|PM|second|evening|dinner| +65252|AAAAAAAAFOOPAAAA|65252|18|7|32|PM|second|evening|dinner| +65253|AAAAAAAAGOOPAAAA|65253|18|7|33|PM|second|evening|dinner| +65254|AAAAAAAAHOOPAAAA|65254|18|7|34|PM|second|evening|dinner| +65255|AAAAAAAAIOOPAAAA|65255|18|7|35|PM|second|evening|dinner| +65256|AAAAAAAAJOOPAAAA|65256|18|7|36|PM|second|evening|dinner| +65257|AAAAAAAAKOOPAAAA|65257|18|7|37|PM|second|evening|dinner| +65258|AAAAAAAALOOPAAAA|65258|18|7|38|PM|second|evening|dinner| +65259|AAAAAAAAMOOPAAAA|65259|18|7|39|PM|second|evening|dinner| +65260|AAAAAAAANOOPAAAA|65260|18|7|40|PM|second|evening|dinner| +65261|AAAAAAAAOOOPAAAA|65261|18|7|41|PM|second|evening|dinner| +65262|AAAAAAAAPOOPAAAA|65262|18|7|42|PM|second|evening|dinner| +65263|AAAAAAAAAPOPAAAA|65263|18|7|43|PM|second|evening|dinner| +65264|AAAAAAAABPOPAAAA|65264|18|7|44|PM|second|evening|dinner| +65265|AAAAAAAACPOPAAAA|65265|18|7|45|PM|second|evening|dinner| +65266|AAAAAAAADPOPAAAA|65266|18|7|46|PM|second|evening|dinner| +65267|AAAAAAAAEPOPAAAA|65267|18|7|47|PM|second|evening|dinner| +65268|AAAAAAAAFPOPAAAA|65268|18|7|48|PM|second|evening|dinner| +65269|AAAAAAAAGPOPAAAA|65269|18|7|49|PM|second|evening|dinner| +65270|AAAAAAAAHPOPAAAA|65270|18|7|50|PM|second|evening|dinner| +65271|AAAAAAAAIPOPAAAA|65271|18|7|51|PM|second|evening|dinner| +65272|AAAAAAAAJPOPAAAA|65272|18|7|52|PM|second|evening|dinner| +65273|AAAAAAAAKPOPAAAA|65273|18|7|53|PM|second|evening|dinner| +65274|AAAAAAAALPOPAAAA|65274|18|7|54|PM|second|evening|dinner| +65275|AAAAAAAAMPOPAAAA|65275|18|7|55|PM|second|evening|dinner| +65276|AAAAAAAANPOPAAAA|65276|18|7|56|PM|second|evening|dinner| +65277|AAAAAAAAOPOPAAAA|65277|18|7|57|PM|second|evening|dinner| +65278|AAAAAAAAPPOPAAAA|65278|18|7|58|PM|second|evening|dinner| +65279|AAAAAAAAAAPPAAAA|65279|18|7|59|PM|second|evening|dinner| +65280|AAAAAAAABAPPAAAA|65280|18|8|0|PM|second|evening|dinner| +65281|AAAAAAAACAPPAAAA|65281|18|8|1|PM|second|evening|dinner| +65282|AAAAAAAADAPPAAAA|65282|18|8|2|PM|second|evening|dinner| +65283|AAAAAAAAEAPPAAAA|65283|18|8|3|PM|second|evening|dinner| +65284|AAAAAAAAFAPPAAAA|65284|18|8|4|PM|second|evening|dinner| +65285|AAAAAAAAGAPPAAAA|65285|18|8|5|PM|second|evening|dinner| +65286|AAAAAAAAHAPPAAAA|65286|18|8|6|PM|second|evening|dinner| +65287|AAAAAAAAIAPPAAAA|65287|18|8|7|PM|second|evening|dinner| +65288|AAAAAAAAJAPPAAAA|65288|18|8|8|PM|second|evening|dinner| +65289|AAAAAAAAKAPPAAAA|65289|18|8|9|PM|second|evening|dinner| +65290|AAAAAAAALAPPAAAA|65290|18|8|10|PM|second|evening|dinner| +65291|AAAAAAAAMAPPAAAA|65291|18|8|11|PM|second|evening|dinner| +65292|AAAAAAAANAPPAAAA|65292|18|8|12|PM|second|evening|dinner| +65293|AAAAAAAAOAPPAAAA|65293|18|8|13|PM|second|evening|dinner| +65294|AAAAAAAAPAPPAAAA|65294|18|8|14|PM|second|evening|dinner| +65295|AAAAAAAAABPPAAAA|65295|18|8|15|PM|second|evening|dinner| +65296|AAAAAAAABBPPAAAA|65296|18|8|16|PM|second|evening|dinner| +65297|AAAAAAAACBPPAAAA|65297|18|8|17|PM|second|evening|dinner| +65298|AAAAAAAADBPPAAAA|65298|18|8|18|PM|second|evening|dinner| +65299|AAAAAAAAEBPPAAAA|65299|18|8|19|PM|second|evening|dinner| +65300|AAAAAAAAFBPPAAAA|65300|18|8|20|PM|second|evening|dinner| +65301|AAAAAAAAGBPPAAAA|65301|18|8|21|PM|second|evening|dinner| +65302|AAAAAAAAHBPPAAAA|65302|18|8|22|PM|second|evening|dinner| +65303|AAAAAAAAIBPPAAAA|65303|18|8|23|PM|second|evening|dinner| +65304|AAAAAAAAJBPPAAAA|65304|18|8|24|PM|second|evening|dinner| +65305|AAAAAAAAKBPPAAAA|65305|18|8|25|PM|second|evening|dinner| +65306|AAAAAAAALBPPAAAA|65306|18|8|26|PM|second|evening|dinner| +65307|AAAAAAAAMBPPAAAA|65307|18|8|27|PM|second|evening|dinner| +65308|AAAAAAAANBPPAAAA|65308|18|8|28|PM|second|evening|dinner| +65309|AAAAAAAAOBPPAAAA|65309|18|8|29|PM|second|evening|dinner| +65310|AAAAAAAAPBPPAAAA|65310|18|8|30|PM|second|evening|dinner| +65311|AAAAAAAAACPPAAAA|65311|18|8|31|PM|second|evening|dinner| +65312|AAAAAAAABCPPAAAA|65312|18|8|32|PM|second|evening|dinner| +65313|AAAAAAAACCPPAAAA|65313|18|8|33|PM|second|evening|dinner| +65314|AAAAAAAADCPPAAAA|65314|18|8|34|PM|second|evening|dinner| +65315|AAAAAAAAECPPAAAA|65315|18|8|35|PM|second|evening|dinner| +65316|AAAAAAAAFCPPAAAA|65316|18|8|36|PM|second|evening|dinner| +65317|AAAAAAAAGCPPAAAA|65317|18|8|37|PM|second|evening|dinner| +65318|AAAAAAAAHCPPAAAA|65318|18|8|38|PM|second|evening|dinner| +65319|AAAAAAAAICPPAAAA|65319|18|8|39|PM|second|evening|dinner| +65320|AAAAAAAAJCPPAAAA|65320|18|8|40|PM|second|evening|dinner| +65321|AAAAAAAAKCPPAAAA|65321|18|8|41|PM|second|evening|dinner| +65322|AAAAAAAALCPPAAAA|65322|18|8|42|PM|second|evening|dinner| +65323|AAAAAAAAMCPPAAAA|65323|18|8|43|PM|second|evening|dinner| +65324|AAAAAAAANCPPAAAA|65324|18|8|44|PM|second|evening|dinner| +65325|AAAAAAAAOCPPAAAA|65325|18|8|45|PM|second|evening|dinner| +65326|AAAAAAAAPCPPAAAA|65326|18|8|46|PM|second|evening|dinner| +65327|AAAAAAAAADPPAAAA|65327|18|8|47|PM|second|evening|dinner| +65328|AAAAAAAABDPPAAAA|65328|18|8|48|PM|second|evening|dinner| +65329|AAAAAAAACDPPAAAA|65329|18|8|49|PM|second|evening|dinner| +65330|AAAAAAAADDPPAAAA|65330|18|8|50|PM|second|evening|dinner| +65331|AAAAAAAAEDPPAAAA|65331|18|8|51|PM|second|evening|dinner| +65332|AAAAAAAAFDPPAAAA|65332|18|8|52|PM|second|evening|dinner| +65333|AAAAAAAAGDPPAAAA|65333|18|8|53|PM|second|evening|dinner| +65334|AAAAAAAAHDPPAAAA|65334|18|8|54|PM|second|evening|dinner| +65335|AAAAAAAAIDPPAAAA|65335|18|8|55|PM|second|evening|dinner| +65336|AAAAAAAAJDPPAAAA|65336|18|8|56|PM|second|evening|dinner| +65337|AAAAAAAAKDPPAAAA|65337|18|8|57|PM|second|evening|dinner| +65338|AAAAAAAALDPPAAAA|65338|18|8|58|PM|second|evening|dinner| +65339|AAAAAAAAMDPPAAAA|65339|18|8|59|PM|second|evening|dinner| +65340|AAAAAAAANDPPAAAA|65340|18|9|0|PM|second|evening|dinner| +65341|AAAAAAAAODPPAAAA|65341|18|9|1|PM|second|evening|dinner| +65342|AAAAAAAAPDPPAAAA|65342|18|9|2|PM|second|evening|dinner| +65343|AAAAAAAAAEPPAAAA|65343|18|9|3|PM|second|evening|dinner| +65344|AAAAAAAABEPPAAAA|65344|18|9|4|PM|second|evening|dinner| +65345|AAAAAAAACEPPAAAA|65345|18|9|5|PM|second|evening|dinner| +65346|AAAAAAAADEPPAAAA|65346|18|9|6|PM|second|evening|dinner| +65347|AAAAAAAAEEPPAAAA|65347|18|9|7|PM|second|evening|dinner| +65348|AAAAAAAAFEPPAAAA|65348|18|9|8|PM|second|evening|dinner| +65349|AAAAAAAAGEPPAAAA|65349|18|9|9|PM|second|evening|dinner| +65350|AAAAAAAAHEPPAAAA|65350|18|9|10|PM|second|evening|dinner| +65351|AAAAAAAAIEPPAAAA|65351|18|9|11|PM|second|evening|dinner| +65352|AAAAAAAAJEPPAAAA|65352|18|9|12|PM|second|evening|dinner| +65353|AAAAAAAAKEPPAAAA|65353|18|9|13|PM|second|evening|dinner| +65354|AAAAAAAALEPPAAAA|65354|18|9|14|PM|second|evening|dinner| +65355|AAAAAAAAMEPPAAAA|65355|18|9|15|PM|second|evening|dinner| +65356|AAAAAAAANEPPAAAA|65356|18|9|16|PM|second|evening|dinner| +65357|AAAAAAAAOEPPAAAA|65357|18|9|17|PM|second|evening|dinner| +65358|AAAAAAAAPEPPAAAA|65358|18|9|18|PM|second|evening|dinner| +65359|AAAAAAAAAFPPAAAA|65359|18|9|19|PM|second|evening|dinner| +65360|AAAAAAAABFPPAAAA|65360|18|9|20|PM|second|evening|dinner| +65361|AAAAAAAACFPPAAAA|65361|18|9|21|PM|second|evening|dinner| +65362|AAAAAAAADFPPAAAA|65362|18|9|22|PM|second|evening|dinner| +65363|AAAAAAAAEFPPAAAA|65363|18|9|23|PM|second|evening|dinner| +65364|AAAAAAAAFFPPAAAA|65364|18|9|24|PM|second|evening|dinner| +65365|AAAAAAAAGFPPAAAA|65365|18|9|25|PM|second|evening|dinner| +65366|AAAAAAAAHFPPAAAA|65366|18|9|26|PM|second|evening|dinner| +65367|AAAAAAAAIFPPAAAA|65367|18|9|27|PM|second|evening|dinner| +65368|AAAAAAAAJFPPAAAA|65368|18|9|28|PM|second|evening|dinner| +65369|AAAAAAAAKFPPAAAA|65369|18|9|29|PM|second|evening|dinner| +65370|AAAAAAAALFPPAAAA|65370|18|9|30|PM|second|evening|dinner| +65371|AAAAAAAAMFPPAAAA|65371|18|9|31|PM|second|evening|dinner| +65372|AAAAAAAANFPPAAAA|65372|18|9|32|PM|second|evening|dinner| +65373|AAAAAAAAOFPPAAAA|65373|18|9|33|PM|second|evening|dinner| +65374|AAAAAAAAPFPPAAAA|65374|18|9|34|PM|second|evening|dinner| +65375|AAAAAAAAAGPPAAAA|65375|18|9|35|PM|second|evening|dinner| +65376|AAAAAAAABGPPAAAA|65376|18|9|36|PM|second|evening|dinner| +65377|AAAAAAAACGPPAAAA|65377|18|9|37|PM|second|evening|dinner| +65378|AAAAAAAADGPPAAAA|65378|18|9|38|PM|second|evening|dinner| +65379|AAAAAAAAEGPPAAAA|65379|18|9|39|PM|second|evening|dinner| +65380|AAAAAAAAFGPPAAAA|65380|18|9|40|PM|second|evening|dinner| +65381|AAAAAAAAGGPPAAAA|65381|18|9|41|PM|second|evening|dinner| +65382|AAAAAAAAHGPPAAAA|65382|18|9|42|PM|second|evening|dinner| +65383|AAAAAAAAIGPPAAAA|65383|18|9|43|PM|second|evening|dinner| +65384|AAAAAAAAJGPPAAAA|65384|18|9|44|PM|second|evening|dinner| +65385|AAAAAAAAKGPPAAAA|65385|18|9|45|PM|second|evening|dinner| +65386|AAAAAAAALGPPAAAA|65386|18|9|46|PM|second|evening|dinner| +65387|AAAAAAAAMGPPAAAA|65387|18|9|47|PM|second|evening|dinner| +65388|AAAAAAAANGPPAAAA|65388|18|9|48|PM|second|evening|dinner| +65389|AAAAAAAAOGPPAAAA|65389|18|9|49|PM|second|evening|dinner| +65390|AAAAAAAAPGPPAAAA|65390|18|9|50|PM|second|evening|dinner| +65391|AAAAAAAAAHPPAAAA|65391|18|9|51|PM|second|evening|dinner| +65392|AAAAAAAABHPPAAAA|65392|18|9|52|PM|second|evening|dinner| +65393|AAAAAAAACHPPAAAA|65393|18|9|53|PM|second|evening|dinner| +65394|AAAAAAAADHPPAAAA|65394|18|9|54|PM|second|evening|dinner| +65395|AAAAAAAAEHPPAAAA|65395|18|9|55|PM|second|evening|dinner| +65396|AAAAAAAAFHPPAAAA|65396|18|9|56|PM|second|evening|dinner| +65397|AAAAAAAAGHPPAAAA|65397|18|9|57|PM|second|evening|dinner| +65398|AAAAAAAAHHPPAAAA|65398|18|9|58|PM|second|evening|dinner| +65399|AAAAAAAAIHPPAAAA|65399|18|9|59|PM|second|evening|dinner| +65400|AAAAAAAAJHPPAAAA|65400|18|10|0|PM|second|evening|dinner| +65401|AAAAAAAAKHPPAAAA|65401|18|10|1|PM|second|evening|dinner| +65402|AAAAAAAALHPPAAAA|65402|18|10|2|PM|second|evening|dinner| +65403|AAAAAAAAMHPPAAAA|65403|18|10|3|PM|second|evening|dinner| +65404|AAAAAAAANHPPAAAA|65404|18|10|4|PM|second|evening|dinner| +65405|AAAAAAAAOHPPAAAA|65405|18|10|5|PM|second|evening|dinner| +65406|AAAAAAAAPHPPAAAA|65406|18|10|6|PM|second|evening|dinner| +65407|AAAAAAAAAIPPAAAA|65407|18|10|7|PM|second|evening|dinner| +65408|AAAAAAAABIPPAAAA|65408|18|10|8|PM|second|evening|dinner| +65409|AAAAAAAACIPPAAAA|65409|18|10|9|PM|second|evening|dinner| +65410|AAAAAAAADIPPAAAA|65410|18|10|10|PM|second|evening|dinner| +65411|AAAAAAAAEIPPAAAA|65411|18|10|11|PM|second|evening|dinner| +65412|AAAAAAAAFIPPAAAA|65412|18|10|12|PM|second|evening|dinner| +65413|AAAAAAAAGIPPAAAA|65413|18|10|13|PM|second|evening|dinner| +65414|AAAAAAAAHIPPAAAA|65414|18|10|14|PM|second|evening|dinner| +65415|AAAAAAAAIIPPAAAA|65415|18|10|15|PM|second|evening|dinner| +65416|AAAAAAAAJIPPAAAA|65416|18|10|16|PM|second|evening|dinner| +65417|AAAAAAAAKIPPAAAA|65417|18|10|17|PM|second|evening|dinner| +65418|AAAAAAAALIPPAAAA|65418|18|10|18|PM|second|evening|dinner| +65419|AAAAAAAAMIPPAAAA|65419|18|10|19|PM|second|evening|dinner| +65420|AAAAAAAANIPPAAAA|65420|18|10|20|PM|second|evening|dinner| +65421|AAAAAAAAOIPPAAAA|65421|18|10|21|PM|second|evening|dinner| +65422|AAAAAAAAPIPPAAAA|65422|18|10|22|PM|second|evening|dinner| +65423|AAAAAAAAAJPPAAAA|65423|18|10|23|PM|second|evening|dinner| +65424|AAAAAAAABJPPAAAA|65424|18|10|24|PM|second|evening|dinner| +65425|AAAAAAAACJPPAAAA|65425|18|10|25|PM|second|evening|dinner| +65426|AAAAAAAADJPPAAAA|65426|18|10|26|PM|second|evening|dinner| +65427|AAAAAAAAEJPPAAAA|65427|18|10|27|PM|second|evening|dinner| +65428|AAAAAAAAFJPPAAAA|65428|18|10|28|PM|second|evening|dinner| +65429|AAAAAAAAGJPPAAAA|65429|18|10|29|PM|second|evening|dinner| +65430|AAAAAAAAHJPPAAAA|65430|18|10|30|PM|second|evening|dinner| +65431|AAAAAAAAIJPPAAAA|65431|18|10|31|PM|second|evening|dinner| +65432|AAAAAAAAJJPPAAAA|65432|18|10|32|PM|second|evening|dinner| +65433|AAAAAAAAKJPPAAAA|65433|18|10|33|PM|second|evening|dinner| +65434|AAAAAAAALJPPAAAA|65434|18|10|34|PM|second|evening|dinner| +65435|AAAAAAAAMJPPAAAA|65435|18|10|35|PM|second|evening|dinner| +65436|AAAAAAAANJPPAAAA|65436|18|10|36|PM|second|evening|dinner| +65437|AAAAAAAAOJPPAAAA|65437|18|10|37|PM|second|evening|dinner| +65438|AAAAAAAAPJPPAAAA|65438|18|10|38|PM|second|evening|dinner| +65439|AAAAAAAAAKPPAAAA|65439|18|10|39|PM|second|evening|dinner| +65440|AAAAAAAABKPPAAAA|65440|18|10|40|PM|second|evening|dinner| +65441|AAAAAAAACKPPAAAA|65441|18|10|41|PM|second|evening|dinner| +65442|AAAAAAAADKPPAAAA|65442|18|10|42|PM|second|evening|dinner| +65443|AAAAAAAAEKPPAAAA|65443|18|10|43|PM|second|evening|dinner| +65444|AAAAAAAAFKPPAAAA|65444|18|10|44|PM|second|evening|dinner| +65445|AAAAAAAAGKPPAAAA|65445|18|10|45|PM|second|evening|dinner| +65446|AAAAAAAAHKPPAAAA|65446|18|10|46|PM|second|evening|dinner| +65447|AAAAAAAAIKPPAAAA|65447|18|10|47|PM|second|evening|dinner| +65448|AAAAAAAAJKPPAAAA|65448|18|10|48|PM|second|evening|dinner| +65449|AAAAAAAAKKPPAAAA|65449|18|10|49|PM|second|evening|dinner| +65450|AAAAAAAALKPPAAAA|65450|18|10|50|PM|second|evening|dinner| +65451|AAAAAAAAMKPPAAAA|65451|18|10|51|PM|second|evening|dinner| +65452|AAAAAAAANKPPAAAA|65452|18|10|52|PM|second|evening|dinner| +65453|AAAAAAAAOKPPAAAA|65453|18|10|53|PM|second|evening|dinner| +65454|AAAAAAAAPKPPAAAA|65454|18|10|54|PM|second|evening|dinner| +65455|AAAAAAAAALPPAAAA|65455|18|10|55|PM|second|evening|dinner| +65456|AAAAAAAABLPPAAAA|65456|18|10|56|PM|second|evening|dinner| +65457|AAAAAAAACLPPAAAA|65457|18|10|57|PM|second|evening|dinner| +65458|AAAAAAAADLPPAAAA|65458|18|10|58|PM|second|evening|dinner| +65459|AAAAAAAAELPPAAAA|65459|18|10|59|PM|second|evening|dinner| +65460|AAAAAAAAFLPPAAAA|65460|18|11|0|PM|second|evening|dinner| +65461|AAAAAAAAGLPPAAAA|65461|18|11|1|PM|second|evening|dinner| +65462|AAAAAAAAHLPPAAAA|65462|18|11|2|PM|second|evening|dinner| +65463|AAAAAAAAILPPAAAA|65463|18|11|3|PM|second|evening|dinner| +65464|AAAAAAAAJLPPAAAA|65464|18|11|4|PM|second|evening|dinner| +65465|AAAAAAAAKLPPAAAA|65465|18|11|5|PM|second|evening|dinner| +65466|AAAAAAAALLPPAAAA|65466|18|11|6|PM|second|evening|dinner| +65467|AAAAAAAAMLPPAAAA|65467|18|11|7|PM|second|evening|dinner| +65468|AAAAAAAANLPPAAAA|65468|18|11|8|PM|second|evening|dinner| +65469|AAAAAAAAOLPPAAAA|65469|18|11|9|PM|second|evening|dinner| +65470|AAAAAAAAPLPPAAAA|65470|18|11|10|PM|second|evening|dinner| +65471|AAAAAAAAAMPPAAAA|65471|18|11|11|PM|second|evening|dinner| +65472|AAAAAAAABMPPAAAA|65472|18|11|12|PM|second|evening|dinner| +65473|AAAAAAAACMPPAAAA|65473|18|11|13|PM|second|evening|dinner| +65474|AAAAAAAADMPPAAAA|65474|18|11|14|PM|second|evening|dinner| +65475|AAAAAAAAEMPPAAAA|65475|18|11|15|PM|second|evening|dinner| +65476|AAAAAAAAFMPPAAAA|65476|18|11|16|PM|second|evening|dinner| +65477|AAAAAAAAGMPPAAAA|65477|18|11|17|PM|second|evening|dinner| +65478|AAAAAAAAHMPPAAAA|65478|18|11|18|PM|second|evening|dinner| +65479|AAAAAAAAIMPPAAAA|65479|18|11|19|PM|second|evening|dinner| +65480|AAAAAAAAJMPPAAAA|65480|18|11|20|PM|second|evening|dinner| +65481|AAAAAAAAKMPPAAAA|65481|18|11|21|PM|second|evening|dinner| +65482|AAAAAAAALMPPAAAA|65482|18|11|22|PM|second|evening|dinner| +65483|AAAAAAAAMMPPAAAA|65483|18|11|23|PM|second|evening|dinner| +65484|AAAAAAAANMPPAAAA|65484|18|11|24|PM|second|evening|dinner| +65485|AAAAAAAAOMPPAAAA|65485|18|11|25|PM|second|evening|dinner| +65486|AAAAAAAAPMPPAAAA|65486|18|11|26|PM|second|evening|dinner| +65487|AAAAAAAAANPPAAAA|65487|18|11|27|PM|second|evening|dinner| +65488|AAAAAAAABNPPAAAA|65488|18|11|28|PM|second|evening|dinner| +65489|AAAAAAAACNPPAAAA|65489|18|11|29|PM|second|evening|dinner| +65490|AAAAAAAADNPPAAAA|65490|18|11|30|PM|second|evening|dinner| +65491|AAAAAAAAENPPAAAA|65491|18|11|31|PM|second|evening|dinner| +65492|AAAAAAAAFNPPAAAA|65492|18|11|32|PM|second|evening|dinner| +65493|AAAAAAAAGNPPAAAA|65493|18|11|33|PM|second|evening|dinner| +65494|AAAAAAAAHNPPAAAA|65494|18|11|34|PM|second|evening|dinner| +65495|AAAAAAAAINPPAAAA|65495|18|11|35|PM|second|evening|dinner| +65496|AAAAAAAAJNPPAAAA|65496|18|11|36|PM|second|evening|dinner| +65497|AAAAAAAAKNPPAAAA|65497|18|11|37|PM|second|evening|dinner| +65498|AAAAAAAALNPPAAAA|65498|18|11|38|PM|second|evening|dinner| +65499|AAAAAAAAMNPPAAAA|65499|18|11|39|PM|second|evening|dinner| +65500|AAAAAAAANNPPAAAA|65500|18|11|40|PM|second|evening|dinner| +65501|AAAAAAAAONPPAAAA|65501|18|11|41|PM|second|evening|dinner| +65502|AAAAAAAAPNPPAAAA|65502|18|11|42|PM|second|evening|dinner| +65503|AAAAAAAAAOPPAAAA|65503|18|11|43|PM|second|evening|dinner| +65504|AAAAAAAABOPPAAAA|65504|18|11|44|PM|second|evening|dinner| +65505|AAAAAAAACOPPAAAA|65505|18|11|45|PM|second|evening|dinner| +65506|AAAAAAAADOPPAAAA|65506|18|11|46|PM|second|evening|dinner| +65507|AAAAAAAAEOPPAAAA|65507|18|11|47|PM|second|evening|dinner| +65508|AAAAAAAAFOPPAAAA|65508|18|11|48|PM|second|evening|dinner| +65509|AAAAAAAAGOPPAAAA|65509|18|11|49|PM|second|evening|dinner| +65510|AAAAAAAAHOPPAAAA|65510|18|11|50|PM|second|evening|dinner| +65511|AAAAAAAAIOPPAAAA|65511|18|11|51|PM|second|evening|dinner| +65512|AAAAAAAAJOPPAAAA|65512|18|11|52|PM|second|evening|dinner| +65513|AAAAAAAAKOPPAAAA|65513|18|11|53|PM|second|evening|dinner| +65514|AAAAAAAALOPPAAAA|65514|18|11|54|PM|second|evening|dinner| +65515|AAAAAAAAMOPPAAAA|65515|18|11|55|PM|second|evening|dinner| +65516|AAAAAAAANOPPAAAA|65516|18|11|56|PM|second|evening|dinner| +65517|AAAAAAAAOOPPAAAA|65517|18|11|57|PM|second|evening|dinner| +65518|AAAAAAAAPOPPAAAA|65518|18|11|58|PM|second|evening|dinner| +65519|AAAAAAAAAPPPAAAA|65519|18|11|59|PM|second|evening|dinner| +65520|AAAAAAAABPPPAAAA|65520|18|12|0|PM|second|evening|dinner| +65521|AAAAAAAACPPPAAAA|65521|18|12|1|PM|second|evening|dinner| +65522|AAAAAAAADPPPAAAA|65522|18|12|2|PM|second|evening|dinner| +65523|AAAAAAAAEPPPAAAA|65523|18|12|3|PM|second|evening|dinner| +65524|AAAAAAAAFPPPAAAA|65524|18|12|4|PM|second|evening|dinner| +65525|AAAAAAAAGPPPAAAA|65525|18|12|5|PM|second|evening|dinner| +65526|AAAAAAAAHPPPAAAA|65526|18|12|6|PM|second|evening|dinner| +65527|AAAAAAAAIPPPAAAA|65527|18|12|7|PM|second|evening|dinner| +65528|AAAAAAAAJPPPAAAA|65528|18|12|8|PM|second|evening|dinner| +65529|AAAAAAAAKPPPAAAA|65529|18|12|9|PM|second|evening|dinner| +65530|AAAAAAAALPPPAAAA|65530|18|12|10|PM|second|evening|dinner| +65531|AAAAAAAAMPPPAAAA|65531|18|12|11|PM|second|evening|dinner| +65532|AAAAAAAANPPPAAAA|65532|18|12|12|PM|second|evening|dinner| +65533|AAAAAAAAOPPPAAAA|65533|18|12|13|PM|second|evening|dinner| +65534|AAAAAAAAPPPPAAAA|65534|18|12|14|PM|second|evening|dinner| +65535|AAAAAAAAAAAABAAA|65535|18|12|15|PM|second|evening|dinner| +65536|AAAAAAAABAAABAAA|65536|18|12|16|PM|second|evening|dinner| +65537|AAAAAAAACAAABAAA|65537|18|12|17|PM|second|evening|dinner| +65538|AAAAAAAADAAABAAA|65538|18|12|18|PM|second|evening|dinner| +65539|AAAAAAAAEAAABAAA|65539|18|12|19|PM|second|evening|dinner| +65540|AAAAAAAAFAAABAAA|65540|18|12|20|PM|second|evening|dinner| +65541|AAAAAAAAGAAABAAA|65541|18|12|21|PM|second|evening|dinner| +65542|AAAAAAAAHAAABAAA|65542|18|12|22|PM|second|evening|dinner| +65543|AAAAAAAAIAAABAAA|65543|18|12|23|PM|second|evening|dinner| +65544|AAAAAAAAJAAABAAA|65544|18|12|24|PM|second|evening|dinner| +65545|AAAAAAAAKAAABAAA|65545|18|12|25|PM|second|evening|dinner| +65546|AAAAAAAALAAABAAA|65546|18|12|26|PM|second|evening|dinner| +65547|AAAAAAAAMAAABAAA|65547|18|12|27|PM|second|evening|dinner| +65548|AAAAAAAANAAABAAA|65548|18|12|28|PM|second|evening|dinner| +65549|AAAAAAAAOAAABAAA|65549|18|12|29|PM|second|evening|dinner| +65550|AAAAAAAAPAAABAAA|65550|18|12|30|PM|second|evening|dinner| +65551|AAAAAAAAABAABAAA|65551|18|12|31|PM|second|evening|dinner| +65552|AAAAAAAABBAABAAA|65552|18|12|32|PM|second|evening|dinner| +65553|AAAAAAAACBAABAAA|65553|18|12|33|PM|second|evening|dinner| +65554|AAAAAAAADBAABAAA|65554|18|12|34|PM|second|evening|dinner| +65555|AAAAAAAAEBAABAAA|65555|18|12|35|PM|second|evening|dinner| +65556|AAAAAAAAFBAABAAA|65556|18|12|36|PM|second|evening|dinner| +65557|AAAAAAAAGBAABAAA|65557|18|12|37|PM|second|evening|dinner| +65558|AAAAAAAAHBAABAAA|65558|18|12|38|PM|second|evening|dinner| +65559|AAAAAAAAIBAABAAA|65559|18|12|39|PM|second|evening|dinner| +65560|AAAAAAAAJBAABAAA|65560|18|12|40|PM|second|evening|dinner| +65561|AAAAAAAAKBAABAAA|65561|18|12|41|PM|second|evening|dinner| +65562|AAAAAAAALBAABAAA|65562|18|12|42|PM|second|evening|dinner| +65563|AAAAAAAAMBAABAAA|65563|18|12|43|PM|second|evening|dinner| +65564|AAAAAAAANBAABAAA|65564|18|12|44|PM|second|evening|dinner| +65565|AAAAAAAAOBAABAAA|65565|18|12|45|PM|second|evening|dinner| +65566|AAAAAAAAPBAABAAA|65566|18|12|46|PM|second|evening|dinner| +65567|AAAAAAAAACAABAAA|65567|18|12|47|PM|second|evening|dinner| +65568|AAAAAAAABCAABAAA|65568|18|12|48|PM|second|evening|dinner| +65569|AAAAAAAACCAABAAA|65569|18|12|49|PM|second|evening|dinner| +65570|AAAAAAAADCAABAAA|65570|18|12|50|PM|second|evening|dinner| +65571|AAAAAAAAECAABAAA|65571|18|12|51|PM|second|evening|dinner| +65572|AAAAAAAAFCAABAAA|65572|18|12|52|PM|second|evening|dinner| +65573|AAAAAAAAGCAABAAA|65573|18|12|53|PM|second|evening|dinner| +65574|AAAAAAAAHCAABAAA|65574|18|12|54|PM|second|evening|dinner| +65575|AAAAAAAAICAABAAA|65575|18|12|55|PM|second|evening|dinner| +65576|AAAAAAAAJCAABAAA|65576|18|12|56|PM|second|evening|dinner| +65577|AAAAAAAAKCAABAAA|65577|18|12|57|PM|second|evening|dinner| +65578|AAAAAAAALCAABAAA|65578|18|12|58|PM|second|evening|dinner| +65579|AAAAAAAAMCAABAAA|65579|18|12|59|PM|second|evening|dinner| +65580|AAAAAAAANCAABAAA|65580|18|13|0|PM|second|evening|dinner| +65581|AAAAAAAAOCAABAAA|65581|18|13|1|PM|second|evening|dinner| +65582|AAAAAAAAPCAABAAA|65582|18|13|2|PM|second|evening|dinner| +65583|AAAAAAAAADAABAAA|65583|18|13|3|PM|second|evening|dinner| +65584|AAAAAAAABDAABAAA|65584|18|13|4|PM|second|evening|dinner| +65585|AAAAAAAACDAABAAA|65585|18|13|5|PM|second|evening|dinner| +65586|AAAAAAAADDAABAAA|65586|18|13|6|PM|second|evening|dinner| +65587|AAAAAAAAEDAABAAA|65587|18|13|7|PM|second|evening|dinner| +65588|AAAAAAAAFDAABAAA|65588|18|13|8|PM|second|evening|dinner| +65589|AAAAAAAAGDAABAAA|65589|18|13|9|PM|second|evening|dinner| +65590|AAAAAAAAHDAABAAA|65590|18|13|10|PM|second|evening|dinner| +65591|AAAAAAAAIDAABAAA|65591|18|13|11|PM|second|evening|dinner| +65592|AAAAAAAAJDAABAAA|65592|18|13|12|PM|second|evening|dinner| +65593|AAAAAAAAKDAABAAA|65593|18|13|13|PM|second|evening|dinner| +65594|AAAAAAAALDAABAAA|65594|18|13|14|PM|second|evening|dinner| +65595|AAAAAAAAMDAABAAA|65595|18|13|15|PM|second|evening|dinner| +65596|AAAAAAAANDAABAAA|65596|18|13|16|PM|second|evening|dinner| +65597|AAAAAAAAODAABAAA|65597|18|13|17|PM|second|evening|dinner| +65598|AAAAAAAAPDAABAAA|65598|18|13|18|PM|second|evening|dinner| +65599|AAAAAAAAAEAABAAA|65599|18|13|19|PM|second|evening|dinner| +65600|AAAAAAAABEAABAAA|65600|18|13|20|PM|second|evening|dinner| +65601|AAAAAAAACEAABAAA|65601|18|13|21|PM|second|evening|dinner| +65602|AAAAAAAADEAABAAA|65602|18|13|22|PM|second|evening|dinner| +65603|AAAAAAAAEEAABAAA|65603|18|13|23|PM|second|evening|dinner| +65604|AAAAAAAAFEAABAAA|65604|18|13|24|PM|second|evening|dinner| +65605|AAAAAAAAGEAABAAA|65605|18|13|25|PM|second|evening|dinner| +65606|AAAAAAAAHEAABAAA|65606|18|13|26|PM|second|evening|dinner| +65607|AAAAAAAAIEAABAAA|65607|18|13|27|PM|second|evening|dinner| +65608|AAAAAAAAJEAABAAA|65608|18|13|28|PM|second|evening|dinner| +65609|AAAAAAAAKEAABAAA|65609|18|13|29|PM|second|evening|dinner| +65610|AAAAAAAALEAABAAA|65610|18|13|30|PM|second|evening|dinner| +65611|AAAAAAAAMEAABAAA|65611|18|13|31|PM|second|evening|dinner| +65612|AAAAAAAANEAABAAA|65612|18|13|32|PM|second|evening|dinner| +65613|AAAAAAAAOEAABAAA|65613|18|13|33|PM|second|evening|dinner| +65614|AAAAAAAAPEAABAAA|65614|18|13|34|PM|second|evening|dinner| +65615|AAAAAAAAAFAABAAA|65615|18|13|35|PM|second|evening|dinner| +65616|AAAAAAAABFAABAAA|65616|18|13|36|PM|second|evening|dinner| +65617|AAAAAAAACFAABAAA|65617|18|13|37|PM|second|evening|dinner| +65618|AAAAAAAADFAABAAA|65618|18|13|38|PM|second|evening|dinner| +65619|AAAAAAAAEFAABAAA|65619|18|13|39|PM|second|evening|dinner| +65620|AAAAAAAAFFAABAAA|65620|18|13|40|PM|second|evening|dinner| +65621|AAAAAAAAGFAABAAA|65621|18|13|41|PM|second|evening|dinner| +65622|AAAAAAAAHFAABAAA|65622|18|13|42|PM|second|evening|dinner| +65623|AAAAAAAAIFAABAAA|65623|18|13|43|PM|second|evening|dinner| +65624|AAAAAAAAJFAABAAA|65624|18|13|44|PM|second|evening|dinner| +65625|AAAAAAAAKFAABAAA|65625|18|13|45|PM|second|evening|dinner| +65626|AAAAAAAALFAABAAA|65626|18|13|46|PM|second|evening|dinner| +65627|AAAAAAAAMFAABAAA|65627|18|13|47|PM|second|evening|dinner| +65628|AAAAAAAANFAABAAA|65628|18|13|48|PM|second|evening|dinner| +65629|AAAAAAAAOFAABAAA|65629|18|13|49|PM|second|evening|dinner| +65630|AAAAAAAAPFAABAAA|65630|18|13|50|PM|second|evening|dinner| +65631|AAAAAAAAAGAABAAA|65631|18|13|51|PM|second|evening|dinner| +65632|AAAAAAAABGAABAAA|65632|18|13|52|PM|second|evening|dinner| +65633|AAAAAAAACGAABAAA|65633|18|13|53|PM|second|evening|dinner| +65634|AAAAAAAADGAABAAA|65634|18|13|54|PM|second|evening|dinner| +65635|AAAAAAAAEGAABAAA|65635|18|13|55|PM|second|evening|dinner| +65636|AAAAAAAAFGAABAAA|65636|18|13|56|PM|second|evening|dinner| +65637|AAAAAAAAGGAABAAA|65637|18|13|57|PM|second|evening|dinner| +65638|AAAAAAAAHGAABAAA|65638|18|13|58|PM|second|evening|dinner| +65639|AAAAAAAAIGAABAAA|65639|18|13|59|PM|second|evening|dinner| +65640|AAAAAAAAJGAABAAA|65640|18|14|0|PM|second|evening|dinner| +65641|AAAAAAAAKGAABAAA|65641|18|14|1|PM|second|evening|dinner| +65642|AAAAAAAALGAABAAA|65642|18|14|2|PM|second|evening|dinner| +65643|AAAAAAAAMGAABAAA|65643|18|14|3|PM|second|evening|dinner| +65644|AAAAAAAANGAABAAA|65644|18|14|4|PM|second|evening|dinner| +65645|AAAAAAAAOGAABAAA|65645|18|14|5|PM|second|evening|dinner| +65646|AAAAAAAAPGAABAAA|65646|18|14|6|PM|second|evening|dinner| +65647|AAAAAAAAAHAABAAA|65647|18|14|7|PM|second|evening|dinner| +65648|AAAAAAAABHAABAAA|65648|18|14|8|PM|second|evening|dinner| +65649|AAAAAAAACHAABAAA|65649|18|14|9|PM|second|evening|dinner| +65650|AAAAAAAADHAABAAA|65650|18|14|10|PM|second|evening|dinner| +65651|AAAAAAAAEHAABAAA|65651|18|14|11|PM|second|evening|dinner| +65652|AAAAAAAAFHAABAAA|65652|18|14|12|PM|second|evening|dinner| +65653|AAAAAAAAGHAABAAA|65653|18|14|13|PM|second|evening|dinner| +65654|AAAAAAAAHHAABAAA|65654|18|14|14|PM|second|evening|dinner| +65655|AAAAAAAAIHAABAAA|65655|18|14|15|PM|second|evening|dinner| +65656|AAAAAAAAJHAABAAA|65656|18|14|16|PM|second|evening|dinner| +65657|AAAAAAAAKHAABAAA|65657|18|14|17|PM|second|evening|dinner| +65658|AAAAAAAALHAABAAA|65658|18|14|18|PM|second|evening|dinner| +65659|AAAAAAAAMHAABAAA|65659|18|14|19|PM|second|evening|dinner| +65660|AAAAAAAANHAABAAA|65660|18|14|20|PM|second|evening|dinner| +65661|AAAAAAAAOHAABAAA|65661|18|14|21|PM|second|evening|dinner| +65662|AAAAAAAAPHAABAAA|65662|18|14|22|PM|second|evening|dinner| +65663|AAAAAAAAAIAABAAA|65663|18|14|23|PM|second|evening|dinner| +65664|AAAAAAAABIAABAAA|65664|18|14|24|PM|second|evening|dinner| +65665|AAAAAAAACIAABAAA|65665|18|14|25|PM|second|evening|dinner| +65666|AAAAAAAADIAABAAA|65666|18|14|26|PM|second|evening|dinner| +65667|AAAAAAAAEIAABAAA|65667|18|14|27|PM|second|evening|dinner| +65668|AAAAAAAAFIAABAAA|65668|18|14|28|PM|second|evening|dinner| +65669|AAAAAAAAGIAABAAA|65669|18|14|29|PM|second|evening|dinner| +65670|AAAAAAAAHIAABAAA|65670|18|14|30|PM|second|evening|dinner| +65671|AAAAAAAAIIAABAAA|65671|18|14|31|PM|second|evening|dinner| +65672|AAAAAAAAJIAABAAA|65672|18|14|32|PM|second|evening|dinner| +65673|AAAAAAAAKIAABAAA|65673|18|14|33|PM|second|evening|dinner| +65674|AAAAAAAALIAABAAA|65674|18|14|34|PM|second|evening|dinner| +65675|AAAAAAAAMIAABAAA|65675|18|14|35|PM|second|evening|dinner| +65676|AAAAAAAANIAABAAA|65676|18|14|36|PM|second|evening|dinner| +65677|AAAAAAAAOIAABAAA|65677|18|14|37|PM|second|evening|dinner| +65678|AAAAAAAAPIAABAAA|65678|18|14|38|PM|second|evening|dinner| +65679|AAAAAAAAAJAABAAA|65679|18|14|39|PM|second|evening|dinner| +65680|AAAAAAAABJAABAAA|65680|18|14|40|PM|second|evening|dinner| +65681|AAAAAAAACJAABAAA|65681|18|14|41|PM|second|evening|dinner| +65682|AAAAAAAADJAABAAA|65682|18|14|42|PM|second|evening|dinner| +65683|AAAAAAAAEJAABAAA|65683|18|14|43|PM|second|evening|dinner| +65684|AAAAAAAAFJAABAAA|65684|18|14|44|PM|second|evening|dinner| +65685|AAAAAAAAGJAABAAA|65685|18|14|45|PM|second|evening|dinner| +65686|AAAAAAAAHJAABAAA|65686|18|14|46|PM|second|evening|dinner| +65687|AAAAAAAAIJAABAAA|65687|18|14|47|PM|second|evening|dinner| +65688|AAAAAAAAJJAABAAA|65688|18|14|48|PM|second|evening|dinner| +65689|AAAAAAAAKJAABAAA|65689|18|14|49|PM|second|evening|dinner| +65690|AAAAAAAALJAABAAA|65690|18|14|50|PM|second|evening|dinner| +65691|AAAAAAAAMJAABAAA|65691|18|14|51|PM|second|evening|dinner| +65692|AAAAAAAANJAABAAA|65692|18|14|52|PM|second|evening|dinner| +65693|AAAAAAAAOJAABAAA|65693|18|14|53|PM|second|evening|dinner| +65694|AAAAAAAAPJAABAAA|65694|18|14|54|PM|second|evening|dinner| +65695|AAAAAAAAAKAABAAA|65695|18|14|55|PM|second|evening|dinner| +65696|AAAAAAAABKAABAAA|65696|18|14|56|PM|second|evening|dinner| +65697|AAAAAAAACKAABAAA|65697|18|14|57|PM|second|evening|dinner| +65698|AAAAAAAADKAABAAA|65698|18|14|58|PM|second|evening|dinner| +65699|AAAAAAAAEKAABAAA|65699|18|14|59|PM|second|evening|dinner| +65700|AAAAAAAAFKAABAAA|65700|18|15|0|PM|second|evening|dinner| +65701|AAAAAAAAGKAABAAA|65701|18|15|1|PM|second|evening|dinner| +65702|AAAAAAAAHKAABAAA|65702|18|15|2|PM|second|evening|dinner| +65703|AAAAAAAAIKAABAAA|65703|18|15|3|PM|second|evening|dinner| +65704|AAAAAAAAJKAABAAA|65704|18|15|4|PM|second|evening|dinner| +65705|AAAAAAAAKKAABAAA|65705|18|15|5|PM|second|evening|dinner| +65706|AAAAAAAALKAABAAA|65706|18|15|6|PM|second|evening|dinner| +65707|AAAAAAAAMKAABAAA|65707|18|15|7|PM|second|evening|dinner| +65708|AAAAAAAANKAABAAA|65708|18|15|8|PM|second|evening|dinner| +65709|AAAAAAAAOKAABAAA|65709|18|15|9|PM|second|evening|dinner| +65710|AAAAAAAAPKAABAAA|65710|18|15|10|PM|second|evening|dinner| +65711|AAAAAAAAALAABAAA|65711|18|15|11|PM|second|evening|dinner| +65712|AAAAAAAABLAABAAA|65712|18|15|12|PM|second|evening|dinner| +65713|AAAAAAAACLAABAAA|65713|18|15|13|PM|second|evening|dinner| +65714|AAAAAAAADLAABAAA|65714|18|15|14|PM|second|evening|dinner| +65715|AAAAAAAAELAABAAA|65715|18|15|15|PM|second|evening|dinner| +65716|AAAAAAAAFLAABAAA|65716|18|15|16|PM|second|evening|dinner| +65717|AAAAAAAAGLAABAAA|65717|18|15|17|PM|second|evening|dinner| +65718|AAAAAAAAHLAABAAA|65718|18|15|18|PM|second|evening|dinner| +65719|AAAAAAAAILAABAAA|65719|18|15|19|PM|second|evening|dinner| +65720|AAAAAAAAJLAABAAA|65720|18|15|20|PM|second|evening|dinner| +65721|AAAAAAAAKLAABAAA|65721|18|15|21|PM|second|evening|dinner| +65722|AAAAAAAALLAABAAA|65722|18|15|22|PM|second|evening|dinner| +65723|AAAAAAAAMLAABAAA|65723|18|15|23|PM|second|evening|dinner| +65724|AAAAAAAANLAABAAA|65724|18|15|24|PM|second|evening|dinner| +65725|AAAAAAAAOLAABAAA|65725|18|15|25|PM|second|evening|dinner| +65726|AAAAAAAAPLAABAAA|65726|18|15|26|PM|second|evening|dinner| +65727|AAAAAAAAAMAABAAA|65727|18|15|27|PM|second|evening|dinner| +65728|AAAAAAAABMAABAAA|65728|18|15|28|PM|second|evening|dinner| +65729|AAAAAAAACMAABAAA|65729|18|15|29|PM|second|evening|dinner| +65730|AAAAAAAADMAABAAA|65730|18|15|30|PM|second|evening|dinner| +65731|AAAAAAAAEMAABAAA|65731|18|15|31|PM|second|evening|dinner| +65732|AAAAAAAAFMAABAAA|65732|18|15|32|PM|second|evening|dinner| +65733|AAAAAAAAGMAABAAA|65733|18|15|33|PM|second|evening|dinner| +65734|AAAAAAAAHMAABAAA|65734|18|15|34|PM|second|evening|dinner| +65735|AAAAAAAAIMAABAAA|65735|18|15|35|PM|second|evening|dinner| +65736|AAAAAAAAJMAABAAA|65736|18|15|36|PM|second|evening|dinner| +65737|AAAAAAAAKMAABAAA|65737|18|15|37|PM|second|evening|dinner| +65738|AAAAAAAALMAABAAA|65738|18|15|38|PM|second|evening|dinner| +65739|AAAAAAAAMMAABAAA|65739|18|15|39|PM|second|evening|dinner| +65740|AAAAAAAANMAABAAA|65740|18|15|40|PM|second|evening|dinner| +65741|AAAAAAAAOMAABAAA|65741|18|15|41|PM|second|evening|dinner| +65742|AAAAAAAAPMAABAAA|65742|18|15|42|PM|second|evening|dinner| +65743|AAAAAAAAANAABAAA|65743|18|15|43|PM|second|evening|dinner| +65744|AAAAAAAABNAABAAA|65744|18|15|44|PM|second|evening|dinner| +65745|AAAAAAAACNAABAAA|65745|18|15|45|PM|second|evening|dinner| +65746|AAAAAAAADNAABAAA|65746|18|15|46|PM|second|evening|dinner| +65747|AAAAAAAAENAABAAA|65747|18|15|47|PM|second|evening|dinner| +65748|AAAAAAAAFNAABAAA|65748|18|15|48|PM|second|evening|dinner| +65749|AAAAAAAAGNAABAAA|65749|18|15|49|PM|second|evening|dinner| +65750|AAAAAAAAHNAABAAA|65750|18|15|50|PM|second|evening|dinner| +65751|AAAAAAAAINAABAAA|65751|18|15|51|PM|second|evening|dinner| +65752|AAAAAAAAJNAABAAA|65752|18|15|52|PM|second|evening|dinner| +65753|AAAAAAAAKNAABAAA|65753|18|15|53|PM|second|evening|dinner| +65754|AAAAAAAALNAABAAA|65754|18|15|54|PM|second|evening|dinner| +65755|AAAAAAAAMNAABAAA|65755|18|15|55|PM|second|evening|dinner| +65756|AAAAAAAANNAABAAA|65756|18|15|56|PM|second|evening|dinner| +65757|AAAAAAAAONAABAAA|65757|18|15|57|PM|second|evening|dinner| +65758|AAAAAAAAPNAABAAA|65758|18|15|58|PM|second|evening|dinner| +65759|AAAAAAAAAOAABAAA|65759|18|15|59|PM|second|evening|dinner| +65760|AAAAAAAABOAABAAA|65760|18|16|0|PM|second|evening|dinner| +65761|AAAAAAAACOAABAAA|65761|18|16|1|PM|second|evening|dinner| +65762|AAAAAAAADOAABAAA|65762|18|16|2|PM|second|evening|dinner| +65763|AAAAAAAAEOAABAAA|65763|18|16|3|PM|second|evening|dinner| +65764|AAAAAAAAFOAABAAA|65764|18|16|4|PM|second|evening|dinner| +65765|AAAAAAAAGOAABAAA|65765|18|16|5|PM|second|evening|dinner| +65766|AAAAAAAAHOAABAAA|65766|18|16|6|PM|second|evening|dinner| +65767|AAAAAAAAIOAABAAA|65767|18|16|7|PM|second|evening|dinner| +65768|AAAAAAAAJOAABAAA|65768|18|16|8|PM|second|evening|dinner| +65769|AAAAAAAAKOAABAAA|65769|18|16|9|PM|second|evening|dinner| +65770|AAAAAAAALOAABAAA|65770|18|16|10|PM|second|evening|dinner| +65771|AAAAAAAAMOAABAAA|65771|18|16|11|PM|second|evening|dinner| +65772|AAAAAAAANOAABAAA|65772|18|16|12|PM|second|evening|dinner| +65773|AAAAAAAAOOAABAAA|65773|18|16|13|PM|second|evening|dinner| +65774|AAAAAAAAPOAABAAA|65774|18|16|14|PM|second|evening|dinner| +65775|AAAAAAAAAPAABAAA|65775|18|16|15|PM|second|evening|dinner| +65776|AAAAAAAABPAABAAA|65776|18|16|16|PM|second|evening|dinner| +65777|AAAAAAAACPAABAAA|65777|18|16|17|PM|second|evening|dinner| +65778|AAAAAAAADPAABAAA|65778|18|16|18|PM|second|evening|dinner| +65779|AAAAAAAAEPAABAAA|65779|18|16|19|PM|second|evening|dinner| +65780|AAAAAAAAFPAABAAA|65780|18|16|20|PM|second|evening|dinner| +65781|AAAAAAAAGPAABAAA|65781|18|16|21|PM|second|evening|dinner| +65782|AAAAAAAAHPAABAAA|65782|18|16|22|PM|second|evening|dinner| +65783|AAAAAAAAIPAABAAA|65783|18|16|23|PM|second|evening|dinner| +65784|AAAAAAAAJPAABAAA|65784|18|16|24|PM|second|evening|dinner| +65785|AAAAAAAAKPAABAAA|65785|18|16|25|PM|second|evening|dinner| +65786|AAAAAAAALPAABAAA|65786|18|16|26|PM|second|evening|dinner| +65787|AAAAAAAAMPAABAAA|65787|18|16|27|PM|second|evening|dinner| +65788|AAAAAAAANPAABAAA|65788|18|16|28|PM|second|evening|dinner| +65789|AAAAAAAAOPAABAAA|65789|18|16|29|PM|second|evening|dinner| +65790|AAAAAAAAPPAABAAA|65790|18|16|30|PM|second|evening|dinner| +65791|AAAAAAAAAABABAAA|65791|18|16|31|PM|second|evening|dinner| +65792|AAAAAAAABABABAAA|65792|18|16|32|PM|second|evening|dinner| +65793|AAAAAAAACABABAAA|65793|18|16|33|PM|second|evening|dinner| +65794|AAAAAAAADABABAAA|65794|18|16|34|PM|second|evening|dinner| +65795|AAAAAAAAEABABAAA|65795|18|16|35|PM|second|evening|dinner| +65796|AAAAAAAAFABABAAA|65796|18|16|36|PM|second|evening|dinner| +65797|AAAAAAAAGABABAAA|65797|18|16|37|PM|second|evening|dinner| +65798|AAAAAAAAHABABAAA|65798|18|16|38|PM|second|evening|dinner| +65799|AAAAAAAAIABABAAA|65799|18|16|39|PM|second|evening|dinner| +65800|AAAAAAAAJABABAAA|65800|18|16|40|PM|second|evening|dinner| +65801|AAAAAAAAKABABAAA|65801|18|16|41|PM|second|evening|dinner| +65802|AAAAAAAALABABAAA|65802|18|16|42|PM|second|evening|dinner| +65803|AAAAAAAAMABABAAA|65803|18|16|43|PM|second|evening|dinner| +65804|AAAAAAAANABABAAA|65804|18|16|44|PM|second|evening|dinner| +65805|AAAAAAAAOABABAAA|65805|18|16|45|PM|second|evening|dinner| +65806|AAAAAAAAPABABAAA|65806|18|16|46|PM|second|evening|dinner| +65807|AAAAAAAAABBABAAA|65807|18|16|47|PM|second|evening|dinner| +65808|AAAAAAAABBBABAAA|65808|18|16|48|PM|second|evening|dinner| +65809|AAAAAAAACBBABAAA|65809|18|16|49|PM|second|evening|dinner| +65810|AAAAAAAADBBABAAA|65810|18|16|50|PM|second|evening|dinner| +65811|AAAAAAAAEBBABAAA|65811|18|16|51|PM|second|evening|dinner| +65812|AAAAAAAAFBBABAAA|65812|18|16|52|PM|second|evening|dinner| +65813|AAAAAAAAGBBABAAA|65813|18|16|53|PM|second|evening|dinner| +65814|AAAAAAAAHBBABAAA|65814|18|16|54|PM|second|evening|dinner| +65815|AAAAAAAAIBBABAAA|65815|18|16|55|PM|second|evening|dinner| +65816|AAAAAAAAJBBABAAA|65816|18|16|56|PM|second|evening|dinner| +65817|AAAAAAAAKBBABAAA|65817|18|16|57|PM|second|evening|dinner| +65818|AAAAAAAALBBABAAA|65818|18|16|58|PM|second|evening|dinner| +65819|AAAAAAAAMBBABAAA|65819|18|16|59|PM|second|evening|dinner| +65820|AAAAAAAANBBABAAA|65820|18|17|0|PM|second|evening|dinner| +65821|AAAAAAAAOBBABAAA|65821|18|17|1|PM|second|evening|dinner| +65822|AAAAAAAAPBBABAAA|65822|18|17|2|PM|second|evening|dinner| +65823|AAAAAAAAACBABAAA|65823|18|17|3|PM|second|evening|dinner| +65824|AAAAAAAABCBABAAA|65824|18|17|4|PM|second|evening|dinner| +65825|AAAAAAAACCBABAAA|65825|18|17|5|PM|second|evening|dinner| +65826|AAAAAAAADCBABAAA|65826|18|17|6|PM|second|evening|dinner| +65827|AAAAAAAAECBABAAA|65827|18|17|7|PM|second|evening|dinner| +65828|AAAAAAAAFCBABAAA|65828|18|17|8|PM|second|evening|dinner| +65829|AAAAAAAAGCBABAAA|65829|18|17|9|PM|second|evening|dinner| +65830|AAAAAAAAHCBABAAA|65830|18|17|10|PM|second|evening|dinner| +65831|AAAAAAAAICBABAAA|65831|18|17|11|PM|second|evening|dinner| +65832|AAAAAAAAJCBABAAA|65832|18|17|12|PM|second|evening|dinner| +65833|AAAAAAAAKCBABAAA|65833|18|17|13|PM|second|evening|dinner| +65834|AAAAAAAALCBABAAA|65834|18|17|14|PM|second|evening|dinner| +65835|AAAAAAAAMCBABAAA|65835|18|17|15|PM|second|evening|dinner| +65836|AAAAAAAANCBABAAA|65836|18|17|16|PM|second|evening|dinner| +65837|AAAAAAAAOCBABAAA|65837|18|17|17|PM|second|evening|dinner| +65838|AAAAAAAAPCBABAAA|65838|18|17|18|PM|second|evening|dinner| +65839|AAAAAAAAADBABAAA|65839|18|17|19|PM|second|evening|dinner| +65840|AAAAAAAABDBABAAA|65840|18|17|20|PM|second|evening|dinner| +65841|AAAAAAAACDBABAAA|65841|18|17|21|PM|second|evening|dinner| +65842|AAAAAAAADDBABAAA|65842|18|17|22|PM|second|evening|dinner| +65843|AAAAAAAAEDBABAAA|65843|18|17|23|PM|second|evening|dinner| +65844|AAAAAAAAFDBABAAA|65844|18|17|24|PM|second|evening|dinner| +65845|AAAAAAAAGDBABAAA|65845|18|17|25|PM|second|evening|dinner| +65846|AAAAAAAAHDBABAAA|65846|18|17|26|PM|second|evening|dinner| +65847|AAAAAAAAIDBABAAA|65847|18|17|27|PM|second|evening|dinner| +65848|AAAAAAAAJDBABAAA|65848|18|17|28|PM|second|evening|dinner| +65849|AAAAAAAAKDBABAAA|65849|18|17|29|PM|second|evening|dinner| +65850|AAAAAAAALDBABAAA|65850|18|17|30|PM|second|evening|dinner| +65851|AAAAAAAAMDBABAAA|65851|18|17|31|PM|second|evening|dinner| +65852|AAAAAAAANDBABAAA|65852|18|17|32|PM|second|evening|dinner| +65853|AAAAAAAAODBABAAA|65853|18|17|33|PM|second|evening|dinner| +65854|AAAAAAAAPDBABAAA|65854|18|17|34|PM|second|evening|dinner| +65855|AAAAAAAAAEBABAAA|65855|18|17|35|PM|second|evening|dinner| +65856|AAAAAAAABEBABAAA|65856|18|17|36|PM|second|evening|dinner| +65857|AAAAAAAACEBABAAA|65857|18|17|37|PM|second|evening|dinner| +65858|AAAAAAAADEBABAAA|65858|18|17|38|PM|second|evening|dinner| +65859|AAAAAAAAEEBABAAA|65859|18|17|39|PM|second|evening|dinner| +65860|AAAAAAAAFEBABAAA|65860|18|17|40|PM|second|evening|dinner| +65861|AAAAAAAAGEBABAAA|65861|18|17|41|PM|second|evening|dinner| +65862|AAAAAAAAHEBABAAA|65862|18|17|42|PM|second|evening|dinner| +65863|AAAAAAAAIEBABAAA|65863|18|17|43|PM|second|evening|dinner| +65864|AAAAAAAAJEBABAAA|65864|18|17|44|PM|second|evening|dinner| +65865|AAAAAAAAKEBABAAA|65865|18|17|45|PM|second|evening|dinner| +65866|AAAAAAAALEBABAAA|65866|18|17|46|PM|second|evening|dinner| +65867|AAAAAAAAMEBABAAA|65867|18|17|47|PM|second|evening|dinner| +65868|AAAAAAAANEBABAAA|65868|18|17|48|PM|second|evening|dinner| +65869|AAAAAAAAOEBABAAA|65869|18|17|49|PM|second|evening|dinner| +65870|AAAAAAAAPEBABAAA|65870|18|17|50|PM|second|evening|dinner| +65871|AAAAAAAAAFBABAAA|65871|18|17|51|PM|second|evening|dinner| +65872|AAAAAAAABFBABAAA|65872|18|17|52|PM|second|evening|dinner| +65873|AAAAAAAACFBABAAA|65873|18|17|53|PM|second|evening|dinner| +65874|AAAAAAAADFBABAAA|65874|18|17|54|PM|second|evening|dinner| +65875|AAAAAAAAEFBABAAA|65875|18|17|55|PM|second|evening|dinner| +65876|AAAAAAAAFFBABAAA|65876|18|17|56|PM|second|evening|dinner| +65877|AAAAAAAAGFBABAAA|65877|18|17|57|PM|second|evening|dinner| +65878|AAAAAAAAHFBABAAA|65878|18|17|58|PM|second|evening|dinner| +65879|AAAAAAAAIFBABAAA|65879|18|17|59|PM|second|evening|dinner| +65880|AAAAAAAAJFBABAAA|65880|18|18|0|PM|second|evening|dinner| +65881|AAAAAAAAKFBABAAA|65881|18|18|1|PM|second|evening|dinner| +65882|AAAAAAAALFBABAAA|65882|18|18|2|PM|second|evening|dinner| +65883|AAAAAAAAMFBABAAA|65883|18|18|3|PM|second|evening|dinner| +65884|AAAAAAAANFBABAAA|65884|18|18|4|PM|second|evening|dinner| +65885|AAAAAAAAOFBABAAA|65885|18|18|5|PM|second|evening|dinner| +65886|AAAAAAAAPFBABAAA|65886|18|18|6|PM|second|evening|dinner| +65887|AAAAAAAAAGBABAAA|65887|18|18|7|PM|second|evening|dinner| +65888|AAAAAAAABGBABAAA|65888|18|18|8|PM|second|evening|dinner| +65889|AAAAAAAACGBABAAA|65889|18|18|9|PM|second|evening|dinner| +65890|AAAAAAAADGBABAAA|65890|18|18|10|PM|second|evening|dinner| +65891|AAAAAAAAEGBABAAA|65891|18|18|11|PM|second|evening|dinner| +65892|AAAAAAAAFGBABAAA|65892|18|18|12|PM|second|evening|dinner| +65893|AAAAAAAAGGBABAAA|65893|18|18|13|PM|second|evening|dinner| +65894|AAAAAAAAHGBABAAA|65894|18|18|14|PM|second|evening|dinner| +65895|AAAAAAAAIGBABAAA|65895|18|18|15|PM|second|evening|dinner| +65896|AAAAAAAAJGBABAAA|65896|18|18|16|PM|second|evening|dinner| +65897|AAAAAAAAKGBABAAA|65897|18|18|17|PM|second|evening|dinner| +65898|AAAAAAAALGBABAAA|65898|18|18|18|PM|second|evening|dinner| +65899|AAAAAAAAMGBABAAA|65899|18|18|19|PM|second|evening|dinner| +65900|AAAAAAAANGBABAAA|65900|18|18|20|PM|second|evening|dinner| +65901|AAAAAAAAOGBABAAA|65901|18|18|21|PM|second|evening|dinner| +65902|AAAAAAAAPGBABAAA|65902|18|18|22|PM|second|evening|dinner| +65903|AAAAAAAAAHBABAAA|65903|18|18|23|PM|second|evening|dinner| +65904|AAAAAAAABHBABAAA|65904|18|18|24|PM|second|evening|dinner| +65905|AAAAAAAACHBABAAA|65905|18|18|25|PM|second|evening|dinner| +65906|AAAAAAAADHBABAAA|65906|18|18|26|PM|second|evening|dinner| +65907|AAAAAAAAEHBABAAA|65907|18|18|27|PM|second|evening|dinner| +65908|AAAAAAAAFHBABAAA|65908|18|18|28|PM|second|evening|dinner| +65909|AAAAAAAAGHBABAAA|65909|18|18|29|PM|second|evening|dinner| +65910|AAAAAAAAHHBABAAA|65910|18|18|30|PM|second|evening|dinner| +65911|AAAAAAAAIHBABAAA|65911|18|18|31|PM|second|evening|dinner| +65912|AAAAAAAAJHBABAAA|65912|18|18|32|PM|second|evening|dinner| +65913|AAAAAAAAKHBABAAA|65913|18|18|33|PM|second|evening|dinner| +65914|AAAAAAAALHBABAAA|65914|18|18|34|PM|second|evening|dinner| +65915|AAAAAAAAMHBABAAA|65915|18|18|35|PM|second|evening|dinner| +65916|AAAAAAAANHBABAAA|65916|18|18|36|PM|second|evening|dinner| +65917|AAAAAAAAOHBABAAA|65917|18|18|37|PM|second|evening|dinner| +65918|AAAAAAAAPHBABAAA|65918|18|18|38|PM|second|evening|dinner| +65919|AAAAAAAAAIBABAAA|65919|18|18|39|PM|second|evening|dinner| +65920|AAAAAAAABIBABAAA|65920|18|18|40|PM|second|evening|dinner| +65921|AAAAAAAACIBABAAA|65921|18|18|41|PM|second|evening|dinner| +65922|AAAAAAAADIBABAAA|65922|18|18|42|PM|second|evening|dinner| +65923|AAAAAAAAEIBABAAA|65923|18|18|43|PM|second|evening|dinner| +65924|AAAAAAAAFIBABAAA|65924|18|18|44|PM|second|evening|dinner| +65925|AAAAAAAAGIBABAAA|65925|18|18|45|PM|second|evening|dinner| +65926|AAAAAAAAHIBABAAA|65926|18|18|46|PM|second|evening|dinner| +65927|AAAAAAAAIIBABAAA|65927|18|18|47|PM|second|evening|dinner| +65928|AAAAAAAAJIBABAAA|65928|18|18|48|PM|second|evening|dinner| +65929|AAAAAAAAKIBABAAA|65929|18|18|49|PM|second|evening|dinner| +65930|AAAAAAAALIBABAAA|65930|18|18|50|PM|second|evening|dinner| +65931|AAAAAAAAMIBABAAA|65931|18|18|51|PM|second|evening|dinner| +65932|AAAAAAAANIBABAAA|65932|18|18|52|PM|second|evening|dinner| +65933|AAAAAAAAOIBABAAA|65933|18|18|53|PM|second|evening|dinner| +65934|AAAAAAAAPIBABAAA|65934|18|18|54|PM|second|evening|dinner| +65935|AAAAAAAAAJBABAAA|65935|18|18|55|PM|second|evening|dinner| +65936|AAAAAAAABJBABAAA|65936|18|18|56|PM|second|evening|dinner| +65937|AAAAAAAACJBABAAA|65937|18|18|57|PM|second|evening|dinner| +65938|AAAAAAAADJBABAAA|65938|18|18|58|PM|second|evening|dinner| +65939|AAAAAAAAEJBABAAA|65939|18|18|59|PM|second|evening|dinner| +65940|AAAAAAAAFJBABAAA|65940|18|19|0|PM|second|evening|dinner| +65941|AAAAAAAAGJBABAAA|65941|18|19|1|PM|second|evening|dinner| +65942|AAAAAAAAHJBABAAA|65942|18|19|2|PM|second|evening|dinner| +65943|AAAAAAAAIJBABAAA|65943|18|19|3|PM|second|evening|dinner| +65944|AAAAAAAAJJBABAAA|65944|18|19|4|PM|second|evening|dinner| +65945|AAAAAAAAKJBABAAA|65945|18|19|5|PM|second|evening|dinner| +65946|AAAAAAAALJBABAAA|65946|18|19|6|PM|second|evening|dinner| +65947|AAAAAAAAMJBABAAA|65947|18|19|7|PM|second|evening|dinner| +65948|AAAAAAAANJBABAAA|65948|18|19|8|PM|second|evening|dinner| +65949|AAAAAAAAOJBABAAA|65949|18|19|9|PM|second|evening|dinner| +65950|AAAAAAAAPJBABAAA|65950|18|19|10|PM|second|evening|dinner| +65951|AAAAAAAAAKBABAAA|65951|18|19|11|PM|second|evening|dinner| +65952|AAAAAAAABKBABAAA|65952|18|19|12|PM|second|evening|dinner| +65953|AAAAAAAACKBABAAA|65953|18|19|13|PM|second|evening|dinner| +65954|AAAAAAAADKBABAAA|65954|18|19|14|PM|second|evening|dinner| +65955|AAAAAAAAEKBABAAA|65955|18|19|15|PM|second|evening|dinner| +65956|AAAAAAAAFKBABAAA|65956|18|19|16|PM|second|evening|dinner| +65957|AAAAAAAAGKBABAAA|65957|18|19|17|PM|second|evening|dinner| +65958|AAAAAAAAHKBABAAA|65958|18|19|18|PM|second|evening|dinner| +65959|AAAAAAAAIKBABAAA|65959|18|19|19|PM|second|evening|dinner| +65960|AAAAAAAAJKBABAAA|65960|18|19|20|PM|second|evening|dinner| +65961|AAAAAAAAKKBABAAA|65961|18|19|21|PM|second|evening|dinner| +65962|AAAAAAAALKBABAAA|65962|18|19|22|PM|second|evening|dinner| +65963|AAAAAAAAMKBABAAA|65963|18|19|23|PM|second|evening|dinner| +65964|AAAAAAAANKBABAAA|65964|18|19|24|PM|second|evening|dinner| +65965|AAAAAAAAOKBABAAA|65965|18|19|25|PM|second|evening|dinner| +65966|AAAAAAAAPKBABAAA|65966|18|19|26|PM|second|evening|dinner| +65967|AAAAAAAAALBABAAA|65967|18|19|27|PM|second|evening|dinner| +65968|AAAAAAAABLBABAAA|65968|18|19|28|PM|second|evening|dinner| +65969|AAAAAAAACLBABAAA|65969|18|19|29|PM|second|evening|dinner| +65970|AAAAAAAADLBABAAA|65970|18|19|30|PM|second|evening|dinner| +65971|AAAAAAAAELBABAAA|65971|18|19|31|PM|second|evening|dinner| +65972|AAAAAAAAFLBABAAA|65972|18|19|32|PM|second|evening|dinner| +65973|AAAAAAAAGLBABAAA|65973|18|19|33|PM|second|evening|dinner| +65974|AAAAAAAAHLBABAAA|65974|18|19|34|PM|second|evening|dinner| +65975|AAAAAAAAILBABAAA|65975|18|19|35|PM|second|evening|dinner| +65976|AAAAAAAAJLBABAAA|65976|18|19|36|PM|second|evening|dinner| +65977|AAAAAAAAKLBABAAA|65977|18|19|37|PM|second|evening|dinner| +65978|AAAAAAAALLBABAAA|65978|18|19|38|PM|second|evening|dinner| +65979|AAAAAAAAMLBABAAA|65979|18|19|39|PM|second|evening|dinner| +65980|AAAAAAAANLBABAAA|65980|18|19|40|PM|second|evening|dinner| +65981|AAAAAAAAOLBABAAA|65981|18|19|41|PM|second|evening|dinner| +65982|AAAAAAAAPLBABAAA|65982|18|19|42|PM|second|evening|dinner| +65983|AAAAAAAAAMBABAAA|65983|18|19|43|PM|second|evening|dinner| +65984|AAAAAAAABMBABAAA|65984|18|19|44|PM|second|evening|dinner| +65985|AAAAAAAACMBABAAA|65985|18|19|45|PM|second|evening|dinner| +65986|AAAAAAAADMBABAAA|65986|18|19|46|PM|second|evening|dinner| +65987|AAAAAAAAEMBABAAA|65987|18|19|47|PM|second|evening|dinner| +65988|AAAAAAAAFMBABAAA|65988|18|19|48|PM|second|evening|dinner| +65989|AAAAAAAAGMBABAAA|65989|18|19|49|PM|second|evening|dinner| +65990|AAAAAAAAHMBABAAA|65990|18|19|50|PM|second|evening|dinner| +65991|AAAAAAAAIMBABAAA|65991|18|19|51|PM|second|evening|dinner| +65992|AAAAAAAAJMBABAAA|65992|18|19|52|PM|second|evening|dinner| +65993|AAAAAAAAKMBABAAA|65993|18|19|53|PM|second|evening|dinner| +65994|AAAAAAAALMBABAAA|65994|18|19|54|PM|second|evening|dinner| +65995|AAAAAAAAMMBABAAA|65995|18|19|55|PM|second|evening|dinner| +65996|AAAAAAAANMBABAAA|65996|18|19|56|PM|second|evening|dinner| +65997|AAAAAAAAOMBABAAA|65997|18|19|57|PM|second|evening|dinner| +65998|AAAAAAAAPMBABAAA|65998|18|19|58|PM|second|evening|dinner| +65999|AAAAAAAAANBABAAA|65999|18|19|59|PM|second|evening|dinner| +66000|AAAAAAAABNBABAAA|66000|18|20|0|PM|second|evening|dinner| +66001|AAAAAAAACNBABAAA|66001|18|20|1|PM|second|evening|dinner| +66002|AAAAAAAADNBABAAA|66002|18|20|2|PM|second|evening|dinner| +66003|AAAAAAAAENBABAAA|66003|18|20|3|PM|second|evening|dinner| +66004|AAAAAAAAFNBABAAA|66004|18|20|4|PM|second|evening|dinner| +66005|AAAAAAAAGNBABAAA|66005|18|20|5|PM|second|evening|dinner| +66006|AAAAAAAAHNBABAAA|66006|18|20|6|PM|second|evening|dinner| +66007|AAAAAAAAINBABAAA|66007|18|20|7|PM|second|evening|dinner| +66008|AAAAAAAAJNBABAAA|66008|18|20|8|PM|second|evening|dinner| +66009|AAAAAAAAKNBABAAA|66009|18|20|9|PM|second|evening|dinner| +66010|AAAAAAAALNBABAAA|66010|18|20|10|PM|second|evening|dinner| +66011|AAAAAAAAMNBABAAA|66011|18|20|11|PM|second|evening|dinner| +66012|AAAAAAAANNBABAAA|66012|18|20|12|PM|second|evening|dinner| +66013|AAAAAAAAONBABAAA|66013|18|20|13|PM|second|evening|dinner| +66014|AAAAAAAAPNBABAAA|66014|18|20|14|PM|second|evening|dinner| +66015|AAAAAAAAAOBABAAA|66015|18|20|15|PM|second|evening|dinner| +66016|AAAAAAAABOBABAAA|66016|18|20|16|PM|second|evening|dinner| +66017|AAAAAAAACOBABAAA|66017|18|20|17|PM|second|evening|dinner| +66018|AAAAAAAADOBABAAA|66018|18|20|18|PM|second|evening|dinner| +66019|AAAAAAAAEOBABAAA|66019|18|20|19|PM|second|evening|dinner| +66020|AAAAAAAAFOBABAAA|66020|18|20|20|PM|second|evening|dinner| +66021|AAAAAAAAGOBABAAA|66021|18|20|21|PM|second|evening|dinner| +66022|AAAAAAAAHOBABAAA|66022|18|20|22|PM|second|evening|dinner| +66023|AAAAAAAAIOBABAAA|66023|18|20|23|PM|second|evening|dinner| +66024|AAAAAAAAJOBABAAA|66024|18|20|24|PM|second|evening|dinner| +66025|AAAAAAAAKOBABAAA|66025|18|20|25|PM|second|evening|dinner| +66026|AAAAAAAALOBABAAA|66026|18|20|26|PM|second|evening|dinner| +66027|AAAAAAAAMOBABAAA|66027|18|20|27|PM|second|evening|dinner| +66028|AAAAAAAANOBABAAA|66028|18|20|28|PM|second|evening|dinner| +66029|AAAAAAAAOOBABAAA|66029|18|20|29|PM|second|evening|dinner| +66030|AAAAAAAAPOBABAAA|66030|18|20|30|PM|second|evening|dinner| +66031|AAAAAAAAAPBABAAA|66031|18|20|31|PM|second|evening|dinner| +66032|AAAAAAAABPBABAAA|66032|18|20|32|PM|second|evening|dinner| +66033|AAAAAAAACPBABAAA|66033|18|20|33|PM|second|evening|dinner| +66034|AAAAAAAADPBABAAA|66034|18|20|34|PM|second|evening|dinner| +66035|AAAAAAAAEPBABAAA|66035|18|20|35|PM|second|evening|dinner| +66036|AAAAAAAAFPBABAAA|66036|18|20|36|PM|second|evening|dinner| +66037|AAAAAAAAGPBABAAA|66037|18|20|37|PM|second|evening|dinner| +66038|AAAAAAAAHPBABAAA|66038|18|20|38|PM|second|evening|dinner| +66039|AAAAAAAAIPBABAAA|66039|18|20|39|PM|second|evening|dinner| +66040|AAAAAAAAJPBABAAA|66040|18|20|40|PM|second|evening|dinner| +66041|AAAAAAAAKPBABAAA|66041|18|20|41|PM|second|evening|dinner| +66042|AAAAAAAALPBABAAA|66042|18|20|42|PM|second|evening|dinner| +66043|AAAAAAAAMPBABAAA|66043|18|20|43|PM|second|evening|dinner| +66044|AAAAAAAANPBABAAA|66044|18|20|44|PM|second|evening|dinner| +66045|AAAAAAAAOPBABAAA|66045|18|20|45|PM|second|evening|dinner| +66046|AAAAAAAAPPBABAAA|66046|18|20|46|PM|second|evening|dinner| +66047|AAAAAAAAAACABAAA|66047|18|20|47|PM|second|evening|dinner| +66048|AAAAAAAABACABAAA|66048|18|20|48|PM|second|evening|dinner| +66049|AAAAAAAACACABAAA|66049|18|20|49|PM|second|evening|dinner| +66050|AAAAAAAADACABAAA|66050|18|20|50|PM|second|evening|dinner| +66051|AAAAAAAAEACABAAA|66051|18|20|51|PM|second|evening|dinner| +66052|AAAAAAAAFACABAAA|66052|18|20|52|PM|second|evening|dinner| +66053|AAAAAAAAGACABAAA|66053|18|20|53|PM|second|evening|dinner| +66054|AAAAAAAAHACABAAA|66054|18|20|54|PM|second|evening|dinner| +66055|AAAAAAAAIACABAAA|66055|18|20|55|PM|second|evening|dinner| +66056|AAAAAAAAJACABAAA|66056|18|20|56|PM|second|evening|dinner| +66057|AAAAAAAAKACABAAA|66057|18|20|57|PM|second|evening|dinner| +66058|AAAAAAAALACABAAA|66058|18|20|58|PM|second|evening|dinner| +66059|AAAAAAAAMACABAAA|66059|18|20|59|PM|second|evening|dinner| +66060|AAAAAAAANACABAAA|66060|18|21|0|PM|second|evening|dinner| +66061|AAAAAAAAOACABAAA|66061|18|21|1|PM|second|evening|dinner| +66062|AAAAAAAAPACABAAA|66062|18|21|2|PM|second|evening|dinner| +66063|AAAAAAAAABCABAAA|66063|18|21|3|PM|second|evening|dinner| +66064|AAAAAAAABBCABAAA|66064|18|21|4|PM|second|evening|dinner| +66065|AAAAAAAACBCABAAA|66065|18|21|5|PM|second|evening|dinner| +66066|AAAAAAAADBCABAAA|66066|18|21|6|PM|second|evening|dinner| +66067|AAAAAAAAEBCABAAA|66067|18|21|7|PM|second|evening|dinner| +66068|AAAAAAAAFBCABAAA|66068|18|21|8|PM|second|evening|dinner| +66069|AAAAAAAAGBCABAAA|66069|18|21|9|PM|second|evening|dinner| +66070|AAAAAAAAHBCABAAA|66070|18|21|10|PM|second|evening|dinner| +66071|AAAAAAAAIBCABAAA|66071|18|21|11|PM|second|evening|dinner| +66072|AAAAAAAAJBCABAAA|66072|18|21|12|PM|second|evening|dinner| +66073|AAAAAAAAKBCABAAA|66073|18|21|13|PM|second|evening|dinner| +66074|AAAAAAAALBCABAAA|66074|18|21|14|PM|second|evening|dinner| +66075|AAAAAAAAMBCABAAA|66075|18|21|15|PM|second|evening|dinner| +66076|AAAAAAAANBCABAAA|66076|18|21|16|PM|second|evening|dinner| +66077|AAAAAAAAOBCABAAA|66077|18|21|17|PM|second|evening|dinner| +66078|AAAAAAAAPBCABAAA|66078|18|21|18|PM|second|evening|dinner| +66079|AAAAAAAAACCABAAA|66079|18|21|19|PM|second|evening|dinner| +66080|AAAAAAAABCCABAAA|66080|18|21|20|PM|second|evening|dinner| +66081|AAAAAAAACCCABAAA|66081|18|21|21|PM|second|evening|dinner| +66082|AAAAAAAADCCABAAA|66082|18|21|22|PM|second|evening|dinner| +66083|AAAAAAAAECCABAAA|66083|18|21|23|PM|second|evening|dinner| +66084|AAAAAAAAFCCABAAA|66084|18|21|24|PM|second|evening|dinner| +66085|AAAAAAAAGCCABAAA|66085|18|21|25|PM|second|evening|dinner| +66086|AAAAAAAAHCCABAAA|66086|18|21|26|PM|second|evening|dinner| +66087|AAAAAAAAICCABAAA|66087|18|21|27|PM|second|evening|dinner| +66088|AAAAAAAAJCCABAAA|66088|18|21|28|PM|second|evening|dinner| +66089|AAAAAAAAKCCABAAA|66089|18|21|29|PM|second|evening|dinner| +66090|AAAAAAAALCCABAAA|66090|18|21|30|PM|second|evening|dinner| +66091|AAAAAAAAMCCABAAA|66091|18|21|31|PM|second|evening|dinner| +66092|AAAAAAAANCCABAAA|66092|18|21|32|PM|second|evening|dinner| +66093|AAAAAAAAOCCABAAA|66093|18|21|33|PM|second|evening|dinner| +66094|AAAAAAAAPCCABAAA|66094|18|21|34|PM|second|evening|dinner| +66095|AAAAAAAAADCABAAA|66095|18|21|35|PM|second|evening|dinner| +66096|AAAAAAAABDCABAAA|66096|18|21|36|PM|second|evening|dinner| +66097|AAAAAAAACDCABAAA|66097|18|21|37|PM|second|evening|dinner| +66098|AAAAAAAADDCABAAA|66098|18|21|38|PM|second|evening|dinner| +66099|AAAAAAAAEDCABAAA|66099|18|21|39|PM|second|evening|dinner| +66100|AAAAAAAAFDCABAAA|66100|18|21|40|PM|second|evening|dinner| +66101|AAAAAAAAGDCABAAA|66101|18|21|41|PM|second|evening|dinner| +66102|AAAAAAAAHDCABAAA|66102|18|21|42|PM|second|evening|dinner| +66103|AAAAAAAAIDCABAAA|66103|18|21|43|PM|second|evening|dinner| +66104|AAAAAAAAJDCABAAA|66104|18|21|44|PM|second|evening|dinner| +66105|AAAAAAAAKDCABAAA|66105|18|21|45|PM|second|evening|dinner| +66106|AAAAAAAALDCABAAA|66106|18|21|46|PM|second|evening|dinner| +66107|AAAAAAAAMDCABAAA|66107|18|21|47|PM|second|evening|dinner| +66108|AAAAAAAANDCABAAA|66108|18|21|48|PM|second|evening|dinner| +66109|AAAAAAAAODCABAAA|66109|18|21|49|PM|second|evening|dinner| +66110|AAAAAAAAPDCABAAA|66110|18|21|50|PM|second|evening|dinner| +66111|AAAAAAAAAECABAAA|66111|18|21|51|PM|second|evening|dinner| +66112|AAAAAAAABECABAAA|66112|18|21|52|PM|second|evening|dinner| +66113|AAAAAAAACECABAAA|66113|18|21|53|PM|second|evening|dinner| +66114|AAAAAAAADECABAAA|66114|18|21|54|PM|second|evening|dinner| +66115|AAAAAAAAEECABAAA|66115|18|21|55|PM|second|evening|dinner| +66116|AAAAAAAAFECABAAA|66116|18|21|56|PM|second|evening|dinner| +66117|AAAAAAAAGECABAAA|66117|18|21|57|PM|second|evening|dinner| +66118|AAAAAAAAHECABAAA|66118|18|21|58|PM|second|evening|dinner| +66119|AAAAAAAAIECABAAA|66119|18|21|59|PM|second|evening|dinner| +66120|AAAAAAAAJECABAAA|66120|18|22|0|PM|second|evening|dinner| +66121|AAAAAAAAKECABAAA|66121|18|22|1|PM|second|evening|dinner| +66122|AAAAAAAALECABAAA|66122|18|22|2|PM|second|evening|dinner| +66123|AAAAAAAAMECABAAA|66123|18|22|3|PM|second|evening|dinner| +66124|AAAAAAAANECABAAA|66124|18|22|4|PM|second|evening|dinner| +66125|AAAAAAAAOECABAAA|66125|18|22|5|PM|second|evening|dinner| +66126|AAAAAAAAPECABAAA|66126|18|22|6|PM|second|evening|dinner| +66127|AAAAAAAAAFCABAAA|66127|18|22|7|PM|second|evening|dinner| +66128|AAAAAAAABFCABAAA|66128|18|22|8|PM|second|evening|dinner| +66129|AAAAAAAACFCABAAA|66129|18|22|9|PM|second|evening|dinner| +66130|AAAAAAAADFCABAAA|66130|18|22|10|PM|second|evening|dinner| +66131|AAAAAAAAEFCABAAA|66131|18|22|11|PM|second|evening|dinner| +66132|AAAAAAAAFFCABAAA|66132|18|22|12|PM|second|evening|dinner| +66133|AAAAAAAAGFCABAAA|66133|18|22|13|PM|second|evening|dinner| +66134|AAAAAAAAHFCABAAA|66134|18|22|14|PM|second|evening|dinner| +66135|AAAAAAAAIFCABAAA|66135|18|22|15|PM|second|evening|dinner| +66136|AAAAAAAAJFCABAAA|66136|18|22|16|PM|second|evening|dinner| +66137|AAAAAAAAKFCABAAA|66137|18|22|17|PM|second|evening|dinner| +66138|AAAAAAAALFCABAAA|66138|18|22|18|PM|second|evening|dinner| +66139|AAAAAAAAMFCABAAA|66139|18|22|19|PM|second|evening|dinner| +66140|AAAAAAAANFCABAAA|66140|18|22|20|PM|second|evening|dinner| +66141|AAAAAAAAOFCABAAA|66141|18|22|21|PM|second|evening|dinner| +66142|AAAAAAAAPFCABAAA|66142|18|22|22|PM|second|evening|dinner| +66143|AAAAAAAAAGCABAAA|66143|18|22|23|PM|second|evening|dinner| +66144|AAAAAAAABGCABAAA|66144|18|22|24|PM|second|evening|dinner| +66145|AAAAAAAACGCABAAA|66145|18|22|25|PM|second|evening|dinner| +66146|AAAAAAAADGCABAAA|66146|18|22|26|PM|second|evening|dinner| +66147|AAAAAAAAEGCABAAA|66147|18|22|27|PM|second|evening|dinner| +66148|AAAAAAAAFGCABAAA|66148|18|22|28|PM|second|evening|dinner| +66149|AAAAAAAAGGCABAAA|66149|18|22|29|PM|second|evening|dinner| +66150|AAAAAAAAHGCABAAA|66150|18|22|30|PM|second|evening|dinner| +66151|AAAAAAAAIGCABAAA|66151|18|22|31|PM|second|evening|dinner| +66152|AAAAAAAAJGCABAAA|66152|18|22|32|PM|second|evening|dinner| +66153|AAAAAAAAKGCABAAA|66153|18|22|33|PM|second|evening|dinner| +66154|AAAAAAAALGCABAAA|66154|18|22|34|PM|second|evening|dinner| +66155|AAAAAAAAMGCABAAA|66155|18|22|35|PM|second|evening|dinner| +66156|AAAAAAAANGCABAAA|66156|18|22|36|PM|second|evening|dinner| +66157|AAAAAAAAOGCABAAA|66157|18|22|37|PM|second|evening|dinner| +66158|AAAAAAAAPGCABAAA|66158|18|22|38|PM|second|evening|dinner| +66159|AAAAAAAAAHCABAAA|66159|18|22|39|PM|second|evening|dinner| +66160|AAAAAAAABHCABAAA|66160|18|22|40|PM|second|evening|dinner| +66161|AAAAAAAACHCABAAA|66161|18|22|41|PM|second|evening|dinner| +66162|AAAAAAAADHCABAAA|66162|18|22|42|PM|second|evening|dinner| +66163|AAAAAAAAEHCABAAA|66163|18|22|43|PM|second|evening|dinner| +66164|AAAAAAAAFHCABAAA|66164|18|22|44|PM|second|evening|dinner| +66165|AAAAAAAAGHCABAAA|66165|18|22|45|PM|second|evening|dinner| +66166|AAAAAAAAHHCABAAA|66166|18|22|46|PM|second|evening|dinner| +66167|AAAAAAAAIHCABAAA|66167|18|22|47|PM|second|evening|dinner| +66168|AAAAAAAAJHCABAAA|66168|18|22|48|PM|second|evening|dinner| +66169|AAAAAAAAKHCABAAA|66169|18|22|49|PM|second|evening|dinner| +66170|AAAAAAAALHCABAAA|66170|18|22|50|PM|second|evening|dinner| +66171|AAAAAAAAMHCABAAA|66171|18|22|51|PM|second|evening|dinner| +66172|AAAAAAAANHCABAAA|66172|18|22|52|PM|second|evening|dinner| +66173|AAAAAAAAOHCABAAA|66173|18|22|53|PM|second|evening|dinner| +66174|AAAAAAAAPHCABAAA|66174|18|22|54|PM|second|evening|dinner| +66175|AAAAAAAAAICABAAA|66175|18|22|55|PM|second|evening|dinner| +66176|AAAAAAAABICABAAA|66176|18|22|56|PM|second|evening|dinner| +66177|AAAAAAAACICABAAA|66177|18|22|57|PM|second|evening|dinner| +66178|AAAAAAAADICABAAA|66178|18|22|58|PM|second|evening|dinner| +66179|AAAAAAAAEICABAAA|66179|18|22|59|PM|second|evening|dinner| +66180|AAAAAAAAFICABAAA|66180|18|23|0|PM|second|evening|dinner| +66181|AAAAAAAAGICABAAA|66181|18|23|1|PM|second|evening|dinner| +66182|AAAAAAAAHICABAAA|66182|18|23|2|PM|second|evening|dinner| +66183|AAAAAAAAIICABAAA|66183|18|23|3|PM|second|evening|dinner| +66184|AAAAAAAAJICABAAA|66184|18|23|4|PM|second|evening|dinner| +66185|AAAAAAAAKICABAAA|66185|18|23|5|PM|second|evening|dinner| +66186|AAAAAAAALICABAAA|66186|18|23|6|PM|second|evening|dinner| +66187|AAAAAAAAMICABAAA|66187|18|23|7|PM|second|evening|dinner| +66188|AAAAAAAANICABAAA|66188|18|23|8|PM|second|evening|dinner| +66189|AAAAAAAAOICABAAA|66189|18|23|9|PM|second|evening|dinner| +66190|AAAAAAAAPICABAAA|66190|18|23|10|PM|second|evening|dinner| +66191|AAAAAAAAAJCABAAA|66191|18|23|11|PM|second|evening|dinner| +66192|AAAAAAAABJCABAAA|66192|18|23|12|PM|second|evening|dinner| +66193|AAAAAAAACJCABAAA|66193|18|23|13|PM|second|evening|dinner| +66194|AAAAAAAADJCABAAA|66194|18|23|14|PM|second|evening|dinner| +66195|AAAAAAAAEJCABAAA|66195|18|23|15|PM|second|evening|dinner| +66196|AAAAAAAAFJCABAAA|66196|18|23|16|PM|second|evening|dinner| +66197|AAAAAAAAGJCABAAA|66197|18|23|17|PM|second|evening|dinner| +66198|AAAAAAAAHJCABAAA|66198|18|23|18|PM|second|evening|dinner| +66199|AAAAAAAAIJCABAAA|66199|18|23|19|PM|second|evening|dinner| +66200|AAAAAAAAJJCABAAA|66200|18|23|20|PM|second|evening|dinner| +66201|AAAAAAAAKJCABAAA|66201|18|23|21|PM|second|evening|dinner| +66202|AAAAAAAALJCABAAA|66202|18|23|22|PM|second|evening|dinner| +66203|AAAAAAAAMJCABAAA|66203|18|23|23|PM|second|evening|dinner| +66204|AAAAAAAANJCABAAA|66204|18|23|24|PM|second|evening|dinner| +66205|AAAAAAAAOJCABAAA|66205|18|23|25|PM|second|evening|dinner| +66206|AAAAAAAAPJCABAAA|66206|18|23|26|PM|second|evening|dinner| +66207|AAAAAAAAAKCABAAA|66207|18|23|27|PM|second|evening|dinner| +66208|AAAAAAAABKCABAAA|66208|18|23|28|PM|second|evening|dinner| +66209|AAAAAAAACKCABAAA|66209|18|23|29|PM|second|evening|dinner| +66210|AAAAAAAADKCABAAA|66210|18|23|30|PM|second|evening|dinner| +66211|AAAAAAAAEKCABAAA|66211|18|23|31|PM|second|evening|dinner| +66212|AAAAAAAAFKCABAAA|66212|18|23|32|PM|second|evening|dinner| +66213|AAAAAAAAGKCABAAA|66213|18|23|33|PM|second|evening|dinner| +66214|AAAAAAAAHKCABAAA|66214|18|23|34|PM|second|evening|dinner| +66215|AAAAAAAAIKCABAAA|66215|18|23|35|PM|second|evening|dinner| +66216|AAAAAAAAJKCABAAA|66216|18|23|36|PM|second|evening|dinner| +66217|AAAAAAAAKKCABAAA|66217|18|23|37|PM|second|evening|dinner| +66218|AAAAAAAALKCABAAA|66218|18|23|38|PM|second|evening|dinner| +66219|AAAAAAAAMKCABAAA|66219|18|23|39|PM|second|evening|dinner| +66220|AAAAAAAANKCABAAA|66220|18|23|40|PM|second|evening|dinner| +66221|AAAAAAAAOKCABAAA|66221|18|23|41|PM|second|evening|dinner| +66222|AAAAAAAAPKCABAAA|66222|18|23|42|PM|second|evening|dinner| +66223|AAAAAAAAALCABAAA|66223|18|23|43|PM|second|evening|dinner| +66224|AAAAAAAABLCABAAA|66224|18|23|44|PM|second|evening|dinner| +66225|AAAAAAAACLCABAAA|66225|18|23|45|PM|second|evening|dinner| +66226|AAAAAAAADLCABAAA|66226|18|23|46|PM|second|evening|dinner| +66227|AAAAAAAAELCABAAA|66227|18|23|47|PM|second|evening|dinner| +66228|AAAAAAAAFLCABAAA|66228|18|23|48|PM|second|evening|dinner| +66229|AAAAAAAAGLCABAAA|66229|18|23|49|PM|second|evening|dinner| +66230|AAAAAAAAHLCABAAA|66230|18|23|50|PM|second|evening|dinner| +66231|AAAAAAAAILCABAAA|66231|18|23|51|PM|second|evening|dinner| +66232|AAAAAAAAJLCABAAA|66232|18|23|52|PM|second|evening|dinner| +66233|AAAAAAAAKLCABAAA|66233|18|23|53|PM|second|evening|dinner| +66234|AAAAAAAALLCABAAA|66234|18|23|54|PM|second|evening|dinner| +66235|AAAAAAAAMLCABAAA|66235|18|23|55|PM|second|evening|dinner| +66236|AAAAAAAANLCABAAA|66236|18|23|56|PM|second|evening|dinner| +66237|AAAAAAAAOLCABAAA|66237|18|23|57|PM|second|evening|dinner| +66238|AAAAAAAAPLCABAAA|66238|18|23|58|PM|second|evening|dinner| +66239|AAAAAAAAAMCABAAA|66239|18|23|59|PM|second|evening|dinner| +66240|AAAAAAAABMCABAAA|66240|18|24|0|PM|second|evening|dinner| +66241|AAAAAAAACMCABAAA|66241|18|24|1|PM|second|evening|dinner| +66242|AAAAAAAADMCABAAA|66242|18|24|2|PM|second|evening|dinner| +66243|AAAAAAAAEMCABAAA|66243|18|24|3|PM|second|evening|dinner| +66244|AAAAAAAAFMCABAAA|66244|18|24|4|PM|second|evening|dinner| +66245|AAAAAAAAGMCABAAA|66245|18|24|5|PM|second|evening|dinner| +66246|AAAAAAAAHMCABAAA|66246|18|24|6|PM|second|evening|dinner| +66247|AAAAAAAAIMCABAAA|66247|18|24|7|PM|second|evening|dinner| +66248|AAAAAAAAJMCABAAA|66248|18|24|8|PM|second|evening|dinner| +66249|AAAAAAAAKMCABAAA|66249|18|24|9|PM|second|evening|dinner| +66250|AAAAAAAALMCABAAA|66250|18|24|10|PM|second|evening|dinner| +66251|AAAAAAAAMMCABAAA|66251|18|24|11|PM|second|evening|dinner| +66252|AAAAAAAANMCABAAA|66252|18|24|12|PM|second|evening|dinner| +66253|AAAAAAAAOMCABAAA|66253|18|24|13|PM|second|evening|dinner| +66254|AAAAAAAAPMCABAAA|66254|18|24|14|PM|second|evening|dinner| +66255|AAAAAAAAANCABAAA|66255|18|24|15|PM|second|evening|dinner| +66256|AAAAAAAABNCABAAA|66256|18|24|16|PM|second|evening|dinner| +66257|AAAAAAAACNCABAAA|66257|18|24|17|PM|second|evening|dinner| +66258|AAAAAAAADNCABAAA|66258|18|24|18|PM|second|evening|dinner| +66259|AAAAAAAAENCABAAA|66259|18|24|19|PM|second|evening|dinner| +66260|AAAAAAAAFNCABAAA|66260|18|24|20|PM|second|evening|dinner| +66261|AAAAAAAAGNCABAAA|66261|18|24|21|PM|second|evening|dinner| +66262|AAAAAAAAHNCABAAA|66262|18|24|22|PM|second|evening|dinner| +66263|AAAAAAAAINCABAAA|66263|18|24|23|PM|second|evening|dinner| +66264|AAAAAAAAJNCABAAA|66264|18|24|24|PM|second|evening|dinner| +66265|AAAAAAAAKNCABAAA|66265|18|24|25|PM|second|evening|dinner| +66266|AAAAAAAALNCABAAA|66266|18|24|26|PM|second|evening|dinner| +66267|AAAAAAAAMNCABAAA|66267|18|24|27|PM|second|evening|dinner| +66268|AAAAAAAANNCABAAA|66268|18|24|28|PM|second|evening|dinner| +66269|AAAAAAAAONCABAAA|66269|18|24|29|PM|second|evening|dinner| +66270|AAAAAAAAPNCABAAA|66270|18|24|30|PM|second|evening|dinner| +66271|AAAAAAAAAOCABAAA|66271|18|24|31|PM|second|evening|dinner| +66272|AAAAAAAABOCABAAA|66272|18|24|32|PM|second|evening|dinner| +66273|AAAAAAAACOCABAAA|66273|18|24|33|PM|second|evening|dinner| +66274|AAAAAAAADOCABAAA|66274|18|24|34|PM|second|evening|dinner| +66275|AAAAAAAAEOCABAAA|66275|18|24|35|PM|second|evening|dinner| +66276|AAAAAAAAFOCABAAA|66276|18|24|36|PM|second|evening|dinner| +66277|AAAAAAAAGOCABAAA|66277|18|24|37|PM|second|evening|dinner| +66278|AAAAAAAAHOCABAAA|66278|18|24|38|PM|second|evening|dinner| +66279|AAAAAAAAIOCABAAA|66279|18|24|39|PM|second|evening|dinner| +66280|AAAAAAAAJOCABAAA|66280|18|24|40|PM|second|evening|dinner| +66281|AAAAAAAAKOCABAAA|66281|18|24|41|PM|second|evening|dinner| +66282|AAAAAAAALOCABAAA|66282|18|24|42|PM|second|evening|dinner| +66283|AAAAAAAAMOCABAAA|66283|18|24|43|PM|second|evening|dinner| +66284|AAAAAAAANOCABAAA|66284|18|24|44|PM|second|evening|dinner| +66285|AAAAAAAAOOCABAAA|66285|18|24|45|PM|second|evening|dinner| +66286|AAAAAAAAPOCABAAA|66286|18|24|46|PM|second|evening|dinner| +66287|AAAAAAAAAPCABAAA|66287|18|24|47|PM|second|evening|dinner| +66288|AAAAAAAABPCABAAA|66288|18|24|48|PM|second|evening|dinner| +66289|AAAAAAAACPCABAAA|66289|18|24|49|PM|second|evening|dinner| +66290|AAAAAAAADPCABAAA|66290|18|24|50|PM|second|evening|dinner| +66291|AAAAAAAAEPCABAAA|66291|18|24|51|PM|second|evening|dinner| +66292|AAAAAAAAFPCABAAA|66292|18|24|52|PM|second|evening|dinner| +66293|AAAAAAAAGPCABAAA|66293|18|24|53|PM|second|evening|dinner| +66294|AAAAAAAAHPCABAAA|66294|18|24|54|PM|second|evening|dinner| +66295|AAAAAAAAIPCABAAA|66295|18|24|55|PM|second|evening|dinner| +66296|AAAAAAAAJPCABAAA|66296|18|24|56|PM|second|evening|dinner| +66297|AAAAAAAAKPCABAAA|66297|18|24|57|PM|second|evening|dinner| +66298|AAAAAAAALPCABAAA|66298|18|24|58|PM|second|evening|dinner| +66299|AAAAAAAAMPCABAAA|66299|18|24|59|PM|second|evening|dinner| +66300|AAAAAAAANPCABAAA|66300|18|25|0|PM|second|evening|dinner| +66301|AAAAAAAAOPCABAAA|66301|18|25|1|PM|second|evening|dinner| +66302|AAAAAAAAPPCABAAA|66302|18|25|2|PM|second|evening|dinner| +66303|AAAAAAAAAADABAAA|66303|18|25|3|PM|second|evening|dinner| +66304|AAAAAAAABADABAAA|66304|18|25|4|PM|second|evening|dinner| +66305|AAAAAAAACADABAAA|66305|18|25|5|PM|second|evening|dinner| +66306|AAAAAAAADADABAAA|66306|18|25|6|PM|second|evening|dinner| +66307|AAAAAAAAEADABAAA|66307|18|25|7|PM|second|evening|dinner| +66308|AAAAAAAAFADABAAA|66308|18|25|8|PM|second|evening|dinner| +66309|AAAAAAAAGADABAAA|66309|18|25|9|PM|second|evening|dinner| +66310|AAAAAAAAHADABAAA|66310|18|25|10|PM|second|evening|dinner| +66311|AAAAAAAAIADABAAA|66311|18|25|11|PM|second|evening|dinner| +66312|AAAAAAAAJADABAAA|66312|18|25|12|PM|second|evening|dinner| +66313|AAAAAAAAKADABAAA|66313|18|25|13|PM|second|evening|dinner| +66314|AAAAAAAALADABAAA|66314|18|25|14|PM|second|evening|dinner| +66315|AAAAAAAAMADABAAA|66315|18|25|15|PM|second|evening|dinner| +66316|AAAAAAAANADABAAA|66316|18|25|16|PM|second|evening|dinner| +66317|AAAAAAAAOADABAAA|66317|18|25|17|PM|second|evening|dinner| +66318|AAAAAAAAPADABAAA|66318|18|25|18|PM|second|evening|dinner| +66319|AAAAAAAAABDABAAA|66319|18|25|19|PM|second|evening|dinner| +66320|AAAAAAAABBDABAAA|66320|18|25|20|PM|second|evening|dinner| +66321|AAAAAAAACBDABAAA|66321|18|25|21|PM|second|evening|dinner| +66322|AAAAAAAADBDABAAA|66322|18|25|22|PM|second|evening|dinner| +66323|AAAAAAAAEBDABAAA|66323|18|25|23|PM|second|evening|dinner| +66324|AAAAAAAAFBDABAAA|66324|18|25|24|PM|second|evening|dinner| +66325|AAAAAAAAGBDABAAA|66325|18|25|25|PM|second|evening|dinner| +66326|AAAAAAAAHBDABAAA|66326|18|25|26|PM|second|evening|dinner| +66327|AAAAAAAAIBDABAAA|66327|18|25|27|PM|second|evening|dinner| +66328|AAAAAAAAJBDABAAA|66328|18|25|28|PM|second|evening|dinner| +66329|AAAAAAAAKBDABAAA|66329|18|25|29|PM|second|evening|dinner| +66330|AAAAAAAALBDABAAA|66330|18|25|30|PM|second|evening|dinner| +66331|AAAAAAAAMBDABAAA|66331|18|25|31|PM|second|evening|dinner| +66332|AAAAAAAANBDABAAA|66332|18|25|32|PM|second|evening|dinner| +66333|AAAAAAAAOBDABAAA|66333|18|25|33|PM|second|evening|dinner| +66334|AAAAAAAAPBDABAAA|66334|18|25|34|PM|second|evening|dinner| +66335|AAAAAAAAACDABAAA|66335|18|25|35|PM|second|evening|dinner| +66336|AAAAAAAABCDABAAA|66336|18|25|36|PM|second|evening|dinner| +66337|AAAAAAAACCDABAAA|66337|18|25|37|PM|second|evening|dinner| +66338|AAAAAAAADCDABAAA|66338|18|25|38|PM|second|evening|dinner| +66339|AAAAAAAAECDABAAA|66339|18|25|39|PM|second|evening|dinner| +66340|AAAAAAAAFCDABAAA|66340|18|25|40|PM|second|evening|dinner| +66341|AAAAAAAAGCDABAAA|66341|18|25|41|PM|second|evening|dinner| +66342|AAAAAAAAHCDABAAA|66342|18|25|42|PM|second|evening|dinner| +66343|AAAAAAAAICDABAAA|66343|18|25|43|PM|second|evening|dinner| +66344|AAAAAAAAJCDABAAA|66344|18|25|44|PM|second|evening|dinner| +66345|AAAAAAAAKCDABAAA|66345|18|25|45|PM|second|evening|dinner| +66346|AAAAAAAALCDABAAA|66346|18|25|46|PM|second|evening|dinner| +66347|AAAAAAAAMCDABAAA|66347|18|25|47|PM|second|evening|dinner| +66348|AAAAAAAANCDABAAA|66348|18|25|48|PM|second|evening|dinner| +66349|AAAAAAAAOCDABAAA|66349|18|25|49|PM|second|evening|dinner| +66350|AAAAAAAAPCDABAAA|66350|18|25|50|PM|second|evening|dinner| +66351|AAAAAAAAADDABAAA|66351|18|25|51|PM|second|evening|dinner| +66352|AAAAAAAABDDABAAA|66352|18|25|52|PM|second|evening|dinner| +66353|AAAAAAAACDDABAAA|66353|18|25|53|PM|second|evening|dinner| +66354|AAAAAAAADDDABAAA|66354|18|25|54|PM|second|evening|dinner| +66355|AAAAAAAAEDDABAAA|66355|18|25|55|PM|second|evening|dinner| +66356|AAAAAAAAFDDABAAA|66356|18|25|56|PM|second|evening|dinner| +66357|AAAAAAAAGDDABAAA|66357|18|25|57|PM|second|evening|dinner| +66358|AAAAAAAAHDDABAAA|66358|18|25|58|PM|second|evening|dinner| +66359|AAAAAAAAIDDABAAA|66359|18|25|59|PM|second|evening|dinner| +66360|AAAAAAAAJDDABAAA|66360|18|26|0|PM|second|evening|dinner| +66361|AAAAAAAAKDDABAAA|66361|18|26|1|PM|second|evening|dinner| +66362|AAAAAAAALDDABAAA|66362|18|26|2|PM|second|evening|dinner| +66363|AAAAAAAAMDDABAAA|66363|18|26|3|PM|second|evening|dinner| +66364|AAAAAAAANDDABAAA|66364|18|26|4|PM|second|evening|dinner| +66365|AAAAAAAAODDABAAA|66365|18|26|5|PM|second|evening|dinner| +66366|AAAAAAAAPDDABAAA|66366|18|26|6|PM|second|evening|dinner| +66367|AAAAAAAAAEDABAAA|66367|18|26|7|PM|second|evening|dinner| +66368|AAAAAAAABEDABAAA|66368|18|26|8|PM|second|evening|dinner| +66369|AAAAAAAACEDABAAA|66369|18|26|9|PM|second|evening|dinner| +66370|AAAAAAAADEDABAAA|66370|18|26|10|PM|second|evening|dinner| +66371|AAAAAAAAEEDABAAA|66371|18|26|11|PM|second|evening|dinner| +66372|AAAAAAAAFEDABAAA|66372|18|26|12|PM|second|evening|dinner| +66373|AAAAAAAAGEDABAAA|66373|18|26|13|PM|second|evening|dinner| +66374|AAAAAAAAHEDABAAA|66374|18|26|14|PM|second|evening|dinner| +66375|AAAAAAAAIEDABAAA|66375|18|26|15|PM|second|evening|dinner| +66376|AAAAAAAAJEDABAAA|66376|18|26|16|PM|second|evening|dinner| +66377|AAAAAAAAKEDABAAA|66377|18|26|17|PM|second|evening|dinner| +66378|AAAAAAAALEDABAAA|66378|18|26|18|PM|second|evening|dinner| +66379|AAAAAAAAMEDABAAA|66379|18|26|19|PM|second|evening|dinner| +66380|AAAAAAAANEDABAAA|66380|18|26|20|PM|second|evening|dinner| +66381|AAAAAAAAOEDABAAA|66381|18|26|21|PM|second|evening|dinner| +66382|AAAAAAAAPEDABAAA|66382|18|26|22|PM|second|evening|dinner| +66383|AAAAAAAAAFDABAAA|66383|18|26|23|PM|second|evening|dinner| +66384|AAAAAAAABFDABAAA|66384|18|26|24|PM|second|evening|dinner| +66385|AAAAAAAACFDABAAA|66385|18|26|25|PM|second|evening|dinner| +66386|AAAAAAAADFDABAAA|66386|18|26|26|PM|second|evening|dinner| +66387|AAAAAAAAEFDABAAA|66387|18|26|27|PM|second|evening|dinner| +66388|AAAAAAAAFFDABAAA|66388|18|26|28|PM|second|evening|dinner| +66389|AAAAAAAAGFDABAAA|66389|18|26|29|PM|second|evening|dinner| +66390|AAAAAAAAHFDABAAA|66390|18|26|30|PM|second|evening|dinner| +66391|AAAAAAAAIFDABAAA|66391|18|26|31|PM|second|evening|dinner| +66392|AAAAAAAAJFDABAAA|66392|18|26|32|PM|second|evening|dinner| +66393|AAAAAAAAKFDABAAA|66393|18|26|33|PM|second|evening|dinner| +66394|AAAAAAAALFDABAAA|66394|18|26|34|PM|second|evening|dinner| +66395|AAAAAAAAMFDABAAA|66395|18|26|35|PM|second|evening|dinner| +66396|AAAAAAAANFDABAAA|66396|18|26|36|PM|second|evening|dinner| +66397|AAAAAAAAOFDABAAA|66397|18|26|37|PM|second|evening|dinner| +66398|AAAAAAAAPFDABAAA|66398|18|26|38|PM|second|evening|dinner| +66399|AAAAAAAAAGDABAAA|66399|18|26|39|PM|second|evening|dinner| +66400|AAAAAAAABGDABAAA|66400|18|26|40|PM|second|evening|dinner| +66401|AAAAAAAACGDABAAA|66401|18|26|41|PM|second|evening|dinner| +66402|AAAAAAAADGDABAAA|66402|18|26|42|PM|second|evening|dinner| +66403|AAAAAAAAEGDABAAA|66403|18|26|43|PM|second|evening|dinner| +66404|AAAAAAAAFGDABAAA|66404|18|26|44|PM|second|evening|dinner| +66405|AAAAAAAAGGDABAAA|66405|18|26|45|PM|second|evening|dinner| +66406|AAAAAAAAHGDABAAA|66406|18|26|46|PM|second|evening|dinner| +66407|AAAAAAAAIGDABAAA|66407|18|26|47|PM|second|evening|dinner| +66408|AAAAAAAAJGDABAAA|66408|18|26|48|PM|second|evening|dinner| +66409|AAAAAAAAKGDABAAA|66409|18|26|49|PM|second|evening|dinner| +66410|AAAAAAAALGDABAAA|66410|18|26|50|PM|second|evening|dinner| +66411|AAAAAAAAMGDABAAA|66411|18|26|51|PM|second|evening|dinner| +66412|AAAAAAAANGDABAAA|66412|18|26|52|PM|second|evening|dinner| +66413|AAAAAAAAOGDABAAA|66413|18|26|53|PM|second|evening|dinner| +66414|AAAAAAAAPGDABAAA|66414|18|26|54|PM|second|evening|dinner| +66415|AAAAAAAAAHDABAAA|66415|18|26|55|PM|second|evening|dinner| +66416|AAAAAAAABHDABAAA|66416|18|26|56|PM|second|evening|dinner| +66417|AAAAAAAACHDABAAA|66417|18|26|57|PM|second|evening|dinner| +66418|AAAAAAAADHDABAAA|66418|18|26|58|PM|second|evening|dinner| +66419|AAAAAAAAEHDABAAA|66419|18|26|59|PM|second|evening|dinner| +66420|AAAAAAAAFHDABAAA|66420|18|27|0|PM|second|evening|dinner| +66421|AAAAAAAAGHDABAAA|66421|18|27|1|PM|second|evening|dinner| +66422|AAAAAAAAHHDABAAA|66422|18|27|2|PM|second|evening|dinner| +66423|AAAAAAAAIHDABAAA|66423|18|27|3|PM|second|evening|dinner| +66424|AAAAAAAAJHDABAAA|66424|18|27|4|PM|second|evening|dinner| +66425|AAAAAAAAKHDABAAA|66425|18|27|5|PM|second|evening|dinner| +66426|AAAAAAAALHDABAAA|66426|18|27|6|PM|second|evening|dinner| +66427|AAAAAAAAMHDABAAA|66427|18|27|7|PM|second|evening|dinner| +66428|AAAAAAAANHDABAAA|66428|18|27|8|PM|second|evening|dinner| +66429|AAAAAAAAOHDABAAA|66429|18|27|9|PM|second|evening|dinner| +66430|AAAAAAAAPHDABAAA|66430|18|27|10|PM|second|evening|dinner| +66431|AAAAAAAAAIDABAAA|66431|18|27|11|PM|second|evening|dinner| +66432|AAAAAAAABIDABAAA|66432|18|27|12|PM|second|evening|dinner| +66433|AAAAAAAACIDABAAA|66433|18|27|13|PM|second|evening|dinner| +66434|AAAAAAAADIDABAAA|66434|18|27|14|PM|second|evening|dinner| +66435|AAAAAAAAEIDABAAA|66435|18|27|15|PM|second|evening|dinner| +66436|AAAAAAAAFIDABAAA|66436|18|27|16|PM|second|evening|dinner| +66437|AAAAAAAAGIDABAAA|66437|18|27|17|PM|second|evening|dinner| +66438|AAAAAAAAHIDABAAA|66438|18|27|18|PM|second|evening|dinner| +66439|AAAAAAAAIIDABAAA|66439|18|27|19|PM|second|evening|dinner| +66440|AAAAAAAAJIDABAAA|66440|18|27|20|PM|second|evening|dinner| +66441|AAAAAAAAKIDABAAA|66441|18|27|21|PM|second|evening|dinner| +66442|AAAAAAAALIDABAAA|66442|18|27|22|PM|second|evening|dinner| +66443|AAAAAAAAMIDABAAA|66443|18|27|23|PM|second|evening|dinner| +66444|AAAAAAAANIDABAAA|66444|18|27|24|PM|second|evening|dinner| +66445|AAAAAAAAOIDABAAA|66445|18|27|25|PM|second|evening|dinner| +66446|AAAAAAAAPIDABAAA|66446|18|27|26|PM|second|evening|dinner| +66447|AAAAAAAAAJDABAAA|66447|18|27|27|PM|second|evening|dinner| +66448|AAAAAAAABJDABAAA|66448|18|27|28|PM|second|evening|dinner| +66449|AAAAAAAACJDABAAA|66449|18|27|29|PM|second|evening|dinner| +66450|AAAAAAAADJDABAAA|66450|18|27|30|PM|second|evening|dinner| +66451|AAAAAAAAEJDABAAA|66451|18|27|31|PM|second|evening|dinner| +66452|AAAAAAAAFJDABAAA|66452|18|27|32|PM|second|evening|dinner| +66453|AAAAAAAAGJDABAAA|66453|18|27|33|PM|second|evening|dinner| +66454|AAAAAAAAHJDABAAA|66454|18|27|34|PM|second|evening|dinner| +66455|AAAAAAAAIJDABAAA|66455|18|27|35|PM|second|evening|dinner| +66456|AAAAAAAAJJDABAAA|66456|18|27|36|PM|second|evening|dinner| +66457|AAAAAAAAKJDABAAA|66457|18|27|37|PM|second|evening|dinner| +66458|AAAAAAAALJDABAAA|66458|18|27|38|PM|second|evening|dinner| +66459|AAAAAAAAMJDABAAA|66459|18|27|39|PM|second|evening|dinner| +66460|AAAAAAAANJDABAAA|66460|18|27|40|PM|second|evening|dinner| +66461|AAAAAAAAOJDABAAA|66461|18|27|41|PM|second|evening|dinner| +66462|AAAAAAAAPJDABAAA|66462|18|27|42|PM|second|evening|dinner| +66463|AAAAAAAAAKDABAAA|66463|18|27|43|PM|second|evening|dinner| +66464|AAAAAAAABKDABAAA|66464|18|27|44|PM|second|evening|dinner| +66465|AAAAAAAACKDABAAA|66465|18|27|45|PM|second|evening|dinner| +66466|AAAAAAAADKDABAAA|66466|18|27|46|PM|second|evening|dinner| +66467|AAAAAAAAEKDABAAA|66467|18|27|47|PM|second|evening|dinner| +66468|AAAAAAAAFKDABAAA|66468|18|27|48|PM|second|evening|dinner| +66469|AAAAAAAAGKDABAAA|66469|18|27|49|PM|second|evening|dinner| +66470|AAAAAAAAHKDABAAA|66470|18|27|50|PM|second|evening|dinner| +66471|AAAAAAAAIKDABAAA|66471|18|27|51|PM|second|evening|dinner| +66472|AAAAAAAAJKDABAAA|66472|18|27|52|PM|second|evening|dinner| +66473|AAAAAAAAKKDABAAA|66473|18|27|53|PM|second|evening|dinner| +66474|AAAAAAAALKDABAAA|66474|18|27|54|PM|second|evening|dinner| +66475|AAAAAAAAMKDABAAA|66475|18|27|55|PM|second|evening|dinner| +66476|AAAAAAAANKDABAAA|66476|18|27|56|PM|second|evening|dinner| +66477|AAAAAAAAOKDABAAA|66477|18|27|57|PM|second|evening|dinner| +66478|AAAAAAAAPKDABAAA|66478|18|27|58|PM|second|evening|dinner| +66479|AAAAAAAAALDABAAA|66479|18|27|59|PM|second|evening|dinner| +66480|AAAAAAAABLDABAAA|66480|18|28|0|PM|second|evening|dinner| +66481|AAAAAAAACLDABAAA|66481|18|28|1|PM|second|evening|dinner| +66482|AAAAAAAADLDABAAA|66482|18|28|2|PM|second|evening|dinner| +66483|AAAAAAAAELDABAAA|66483|18|28|3|PM|second|evening|dinner| +66484|AAAAAAAAFLDABAAA|66484|18|28|4|PM|second|evening|dinner| +66485|AAAAAAAAGLDABAAA|66485|18|28|5|PM|second|evening|dinner| +66486|AAAAAAAAHLDABAAA|66486|18|28|6|PM|second|evening|dinner| +66487|AAAAAAAAILDABAAA|66487|18|28|7|PM|second|evening|dinner| +66488|AAAAAAAAJLDABAAA|66488|18|28|8|PM|second|evening|dinner| +66489|AAAAAAAAKLDABAAA|66489|18|28|9|PM|second|evening|dinner| +66490|AAAAAAAALLDABAAA|66490|18|28|10|PM|second|evening|dinner| +66491|AAAAAAAAMLDABAAA|66491|18|28|11|PM|second|evening|dinner| +66492|AAAAAAAANLDABAAA|66492|18|28|12|PM|second|evening|dinner| +66493|AAAAAAAAOLDABAAA|66493|18|28|13|PM|second|evening|dinner| +66494|AAAAAAAAPLDABAAA|66494|18|28|14|PM|second|evening|dinner| +66495|AAAAAAAAAMDABAAA|66495|18|28|15|PM|second|evening|dinner| +66496|AAAAAAAABMDABAAA|66496|18|28|16|PM|second|evening|dinner| +66497|AAAAAAAACMDABAAA|66497|18|28|17|PM|second|evening|dinner| +66498|AAAAAAAADMDABAAA|66498|18|28|18|PM|second|evening|dinner| +66499|AAAAAAAAEMDABAAA|66499|18|28|19|PM|second|evening|dinner| +66500|AAAAAAAAFMDABAAA|66500|18|28|20|PM|second|evening|dinner| +66501|AAAAAAAAGMDABAAA|66501|18|28|21|PM|second|evening|dinner| +66502|AAAAAAAAHMDABAAA|66502|18|28|22|PM|second|evening|dinner| +66503|AAAAAAAAIMDABAAA|66503|18|28|23|PM|second|evening|dinner| +66504|AAAAAAAAJMDABAAA|66504|18|28|24|PM|second|evening|dinner| +66505|AAAAAAAAKMDABAAA|66505|18|28|25|PM|second|evening|dinner| +66506|AAAAAAAALMDABAAA|66506|18|28|26|PM|second|evening|dinner| +66507|AAAAAAAAMMDABAAA|66507|18|28|27|PM|second|evening|dinner| +66508|AAAAAAAANMDABAAA|66508|18|28|28|PM|second|evening|dinner| +66509|AAAAAAAAOMDABAAA|66509|18|28|29|PM|second|evening|dinner| +66510|AAAAAAAAPMDABAAA|66510|18|28|30|PM|second|evening|dinner| +66511|AAAAAAAAANDABAAA|66511|18|28|31|PM|second|evening|dinner| +66512|AAAAAAAABNDABAAA|66512|18|28|32|PM|second|evening|dinner| +66513|AAAAAAAACNDABAAA|66513|18|28|33|PM|second|evening|dinner| +66514|AAAAAAAADNDABAAA|66514|18|28|34|PM|second|evening|dinner| +66515|AAAAAAAAENDABAAA|66515|18|28|35|PM|second|evening|dinner| +66516|AAAAAAAAFNDABAAA|66516|18|28|36|PM|second|evening|dinner| +66517|AAAAAAAAGNDABAAA|66517|18|28|37|PM|second|evening|dinner| +66518|AAAAAAAAHNDABAAA|66518|18|28|38|PM|second|evening|dinner| +66519|AAAAAAAAINDABAAA|66519|18|28|39|PM|second|evening|dinner| +66520|AAAAAAAAJNDABAAA|66520|18|28|40|PM|second|evening|dinner| +66521|AAAAAAAAKNDABAAA|66521|18|28|41|PM|second|evening|dinner| +66522|AAAAAAAALNDABAAA|66522|18|28|42|PM|second|evening|dinner| +66523|AAAAAAAAMNDABAAA|66523|18|28|43|PM|second|evening|dinner| +66524|AAAAAAAANNDABAAA|66524|18|28|44|PM|second|evening|dinner| +66525|AAAAAAAAONDABAAA|66525|18|28|45|PM|second|evening|dinner| +66526|AAAAAAAAPNDABAAA|66526|18|28|46|PM|second|evening|dinner| +66527|AAAAAAAAAODABAAA|66527|18|28|47|PM|second|evening|dinner| +66528|AAAAAAAABODABAAA|66528|18|28|48|PM|second|evening|dinner| +66529|AAAAAAAACODABAAA|66529|18|28|49|PM|second|evening|dinner| +66530|AAAAAAAADODABAAA|66530|18|28|50|PM|second|evening|dinner| +66531|AAAAAAAAEODABAAA|66531|18|28|51|PM|second|evening|dinner| +66532|AAAAAAAAFODABAAA|66532|18|28|52|PM|second|evening|dinner| +66533|AAAAAAAAGODABAAA|66533|18|28|53|PM|second|evening|dinner| +66534|AAAAAAAAHODABAAA|66534|18|28|54|PM|second|evening|dinner| +66535|AAAAAAAAIODABAAA|66535|18|28|55|PM|second|evening|dinner| +66536|AAAAAAAAJODABAAA|66536|18|28|56|PM|second|evening|dinner| +66537|AAAAAAAAKODABAAA|66537|18|28|57|PM|second|evening|dinner| +66538|AAAAAAAALODABAAA|66538|18|28|58|PM|second|evening|dinner| +66539|AAAAAAAAMODABAAA|66539|18|28|59|PM|second|evening|dinner| +66540|AAAAAAAANODABAAA|66540|18|29|0|PM|second|evening|dinner| +66541|AAAAAAAAOODABAAA|66541|18|29|1|PM|second|evening|dinner| +66542|AAAAAAAAPODABAAA|66542|18|29|2|PM|second|evening|dinner| +66543|AAAAAAAAAPDABAAA|66543|18|29|3|PM|second|evening|dinner| +66544|AAAAAAAABPDABAAA|66544|18|29|4|PM|second|evening|dinner| +66545|AAAAAAAACPDABAAA|66545|18|29|5|PM|second|evening|dinner| +66546|AAAAAAAADPDABAAA|66546|18|29|6|PM|second|evening|dinner| +66547|AAAAAAAAEPDABAAA|66547|18|29|7|PM|second|evening|dinner| +66548|AAAAAAAAFPDABAAA|66548|18|29|8|PM|second|evening|dinner| +66549|AAAAAAAAGPDABAAA|66549|18|29|9|PM|second|evening|dinner| +66550|AAAAAAAAHPDABAAA|66550|18|29|10|PM|second|evening|dinner| +66551|AAAAAAAAIPDABAAA|66551|18|29|11|PM|second|evening|dinner| +66552|AAAAAAAAJPDABAAA|66552|18|29|12|PM|second|evening|dinner| +66553|AAAAAAAAKPDABAAA|66553|18|29|13|PM|second|evening|dinner| +66554|AAAAAAAALPDABAAA|66554|18|29|14|PM|second|evening|dinner| +66555|AAAAAAAAMPDABAAA|66555|18|29|15|PM|second|evening|dinner| +66556|AAAAAAAANPDABAAA|66556|18|29|16|PM|second|evening|dinner| +66557|AAAAAAAAOPDABAAA|66557|18|29|17|PM|second|evening|dinner| +66558|AAAAAAAAPPDABAAA|66558|18|29|18|PM|second|evening|dinner| +66559|AAAAAAAAAAEABAAA|66559|18|29|19|PM|second|evening|dinner| +66560|AAAAAAAABAEABAAA|66560|18|29|20|PM|second|evening|dinner| +66561|AAAAAAAACAEABAAA|66561|18|29|21|PM|second|evening|dinner| +66562|AAAAAAAADAEABAAA|66562|18|29|22|PM|second|evening|dinner| +66563|AAAAAAAAEAEABAAA|66563|18|29|23|PM|second|evening|dinner| +66564|AAAAAAAAFAEABAAA|66564|18|29|24|PM|second|evening|dinner| +66565|AAAAAAAAGAEABAAA|66565|18|29|25|PM|second|evening|dinner| +66566|AAAAAAAAHAEABAAA|66566|18|29|26|PM|second|evening|dinner| +66567|AAAAAAAAIAEABAAA|66567|18|29|27|PM|second|evening|dinner| +66568|AAAAAAAAJAEABAAA|66568|18|29|28|PM|second|evening|dinner| +66569|AAAAAAAAKAEABAAA|66569|18|29|29|PM|second|evening|dinner| +66570|AAAAAAAALAEABAAA|66570|18|29|30|PM|second|evening|dinner| +66571|AAAAAAAAMAEABAAA|66571|18|29|31|PM|second|evening|dinner| +66572|AAAAAAAANAEABAAA|66572|18|29|32|PM|second|evening|dinner| +66573|AAAAAAAAOAEABAAA|66573|18|29|33|PM|second|evening|dinner| +66574|AAAAAAAAPAEABAAA|66574|18|29|34|PM|second|evening|dinner| +66575|AAAAAAAAABEABAAA|66575|18|29|35|PM|second|evening|dinner| +66576|AAAAAAAABBEABAAA|66576|18|29|36|PM|second|evening|dinner| +66577|AAAAAAAACBEABAAA|66577|18|29|37|PM|second|evening|dinner| +66578|AAAAAAAADBEABAAA|66578|18|29|38|PM|second|evening|dinner| +66579|AAAAAAAAEBEABAAA|66579|18|29|39|PM|second|evening|dinner| +66580|AAAAAAAAFBEABAAA|66580|18|29|40|PM|second|evening|dinner| +66581|AAAAAAAAGBEABAAA|66581|18|29|41|PM|second|evening|dinner| +66582|AAAAAAAAHBEABAAA|66582|18|29|42|PM|second|evening|dinner| +66583|AAAAAAAAIBEABAAA|66583|18|29|43|PM|second|evening|dinner| +66584|AAAAAAAAJBEABAAA|66584|18|29|44|PM|second|evening|dinner| +66585|AAAAAAAAKBEABAAA|66585|18|29|45|PM|second|evening|dinner| +66586|AAAAAAAALBEABAAA|66586|18|29|46|PM|second|evening|dinner| +66587|AAAAAAAAMBEABAAA|66587|18|29|47|PM|second|evening|dinner| +66588|AAAAAAAANBEABAAA|66588|18|29|48|PM|second|evening|dinner| +66589|AAAAAAAAOBEABAAA|66589|18|29|49|PM|second|evening|dinner| +66590|AAAAAAAAPBEABAAA|66590|18|29|50|PM|second|evening|dinner| +66591|AAAAAAAAACEABAAA|66591|18|29|51|PM|second|evening|dinner| +66592|AAAAAAAABCEABAAA|66592|18|29|52|PM|second|evening|dinner| +66593|AAAAAAAACCEABAAA|66593|18|29|53|PM|second|evening|dinner| +66594|AAAAAAAADCEABAAA|66594|18|29|54|PM|second|evening|dinner| +66595|AAAAAAAAECEABAAA|66595|18|29|55|PM|second|evening|dinner| +66596|AAAAAAAAFCEABAAA|66596|18|29|56|PM|second|evening|dinner| +66597|AAAAAAAAGCEABAAA|66597|18|29|57|PM|second|evening|dinner| +66598|AAAAAAAAHCEABAAA|66598|18|29|58|PM|second|evening|dinner| +66599|AAAAAAAAICEABAAA|66599|18|29|59|PM|second|evening|dinner| +66600|AAAAAAAAJCEABAAA|66600|18|30|0|PM|second|evening|dinner| +66601|AAAAAAAAKCEABAAA|66601|18|30|1|PM|second|evening|dinner| +66602|AAAAAAAALCEABAAA|66602|18|30|2|PM|second|evening|dinner| +66603|AAAAAAAAMCEABAAA|66603|18|30|3|PM|second|evening|dinner| +66604|AAAAAAAANCEABAAA|66604|18|30|4|PM|second|evening|dinner| +66605|AAAAAAAAOCEABAAA|66605|18|30|5|PM|second|evening|dinner| +66606|AAAAAAAAPCEABAAA|66606|18|30|6|PM|second|evening|dinner| +66607|AAAAAAAAADEABAAA|66607|18|30|7|PM|second|evening|dinner| +66608|AAAAAAAABDEABAAA|66608|18|30|8|PM|second|evening|dinner| +66609|AAAAAAAACDEABAAA|66609|18|30|9|PM|second|evening|dinner| +66610|AAAAAAAADDEABAAA|66610|18|30|10|PM|second|evening|dinner| +66611|AAAAAAAAEDEABAAA|66611|18|30|11|PM|second|evening|dinner| +66612|AAAAAAAAFDEABAAA|66612|18|30|12|PM|second|evening|dinner| +66613|AAAAAAAAGDEABAAA|66613|18|30|13|PM|second|evening|dinner| +66614|AAAAAAAAHDEABAAA|66614|18|30|14|PM|second|evening|dinner| +66615|AAAAAAAAIDEABAAA|66615|18|30|15|PM|second|evening|dinner| +66616|AAAAAAAAJDEABAAA|66616|18|30|16|PM|second|evening|dinner| +66617|AAAAAAAAKDEABAAA|66617|18|30|17|PM|second|evening|dinner| +66618|AAAAAAAALDEABAAA|66618|18|30|18|PM|second|evening|dinner| +66619|AAAAAAAAMDEABAAA|66619|18|30|19|PM|second|evening|dinner| +66620|AAAAAAAANDEABAAA|66620|18|30|20|PM|second|evening|dinner| +66621|AAAAAAAAODEABAAA|66621|18|30|21|PM|second|evening|dinner| +66622|AAAAAAAAPDEABAAA|66622|18|30|22|PM|second|evening|dinner| +66623|AAAAAAAAAEEABAAA|66623|18|30|23|PM|second|evening|dinner| +66624|AAAAAAAABEEABAAA|66624|18|30|24|PM|second|evening|dinner| +66625|AAAAAAAACEEABAAA|66625|18|30|25|PM|second|evening|dinner| +66626|AAAAAAAADEEABAAA|66626|18|30|26|PM|second|evening|dinner| +66627|AAAAAAAAEEEABAAA|66627|18|30|27|PM|second|evening|dinner| +66628|AAAAAAAAFEEABAAA|66628|18|30|28|PM|second|evening|dinner| +66629|AAAAAAAAGEEABAAA|66629|18|30|29|PM|second|evening|dinner| +66630|AAAAAAAAHEEABAAA|66630|18|30|30|PM|second|evening|dinner| +66631|AAAAAAAAIEEABAAA|66631|18|30|31|PM|second|evening|dinner| +66632|AAAAAAAAJEEABAAA|66632|18|30|32|PM|second|evening|dinner| +66633|AAAAAAAAKEEABAAA|66633|18|30|33|PM|second|evening|dinner| +66634|AAAAAAAALEEABAAA|66634|18|30|34|PM|second|evening|dinner| +66635|AAAAAAAAMEEABAAA|66635|18|30|35|PM|second|evening|dinner| +66636|AAAAAAAANEEABAAA|66636|18|30|36|PM|second|evening|dinner| +66637|AAAAAAAAOEEABAAA|66637|18|30|37|PM|second|evening|dinner| +66638|AAAAAAAAPEEABAAA|66638|18|30|38|PM|second|evening|dinner| +66639|AAAAAAAAAFEABAAA|66639|18|30|39|PM|second|evening|dinner| +66640|AAAAAAAABFEABAAA|66640|18|30|40|PM|second|evening|dinner| +66641|AAAAAAAACFEABAAA|66641|18|30|41|PM|second|evening|dinner| +66642|AAAAAAAADFEABAAA|66642|18|30|42|PM|second|evening|dinner| +66643|AAAAAAAAEFEABAAA|66643|18|30|43|PM|second|evening|dinner| +66644|AAAAAAAAFFEABAAA|66644|18|30|44|PM|second|evening|dinner| +66645|AAAAAAAAGFEABAAA|66645|18|30|45|PM|second|evening|dinner| +66646|AAAAAAAAHFEABAAA|66646|18|30|46|PM|second|evening|dinner| +66647|AAAAAAAAIFEABAAA|66647|18|30|47|PM|second|evening|dinner| +66648|AAAAAAAAJFEABAAA|66648|18|30|48|PM|second|evening|dinner| +66649|AAAAAAAAKFEABAAA|66649|18|30|49|PM|second|evening|dinner| +66650|AAAAAAAALFEABAAA|66650|18|30|50|PM|second|evening|dinner| +66651|AAAAAAAAMFEABAAA|66651|18|30|51|PM|second|evening|dinner| +66652|AAAAAAAANFEABAAA|66652|18|30|52|PM|second|evening|dinner| +66653|AAAAAAAAOFEABAAA|66653|18|30|53|PM|second|evening|dinner| +66654|AAAAAAAAPFEABAAA|66654|18|30|54|PM|second|evening|dinner| +66655|AAAAAAAAAGEABAAA|66655|18|30|55|PM|second|evening|dinner| +66656|AAAAAAAABGEABAAA|66656|18|30|56|PM|second|evening|dinner| +66657|AAAAAAAACGEABAAA|66657|18|30|57|PM|second|evening|dinner| +66658|AAAAAAAADGEABAAA|66658|18|30|58|PM|second|evening|dinner| +66659|AAAAAAAAEGEABAAA|66659|18|30|59|PM|second|evening|dinner| +66660|AAAAAAAAFGEABAAA|66660|18|31|0|PM|second|evening|dinner| +66661|AAAAAAAAGGEABAAA|66661|18|31|1|PM|second|evening|dinner| +66662|AAAAAAAAHGEABAAA|66662|18|31|2|PM|second|evening|dinner| +66663|AAAAAAAAIGEABAAA|66663|18|31|3|PM|second|evening|dinner| +66664|AAAAAAAAJGEABAAA|66664|18|31|4|PM|second|evening|dinner| +66665|AAAAAAAAKGEABAAA|66665|18|31|5|PM|second|evening|dinner| +66666|AAAAAAAALGEABAAA|66666|18|31|6|PM|second|evening|dinner| +66667|AAAAAAAAMGEABAAA|66667|18|31|7|PM|second|evening|dinner| +66668|AAAAAAAANGEABAAA|66668|18|31|8|PM|second|evening|dinner| +66669|AAAAAAAAOGEABAAA|66669|18|31|9|PM|second|evening|dinner| +66670|AAAAAAAAPGEABAAA|66670|18|31|10|PM|second|evening|dinner| +66671|AAAAAAAAAHEABAAA|66671|18|31|11|PM|second|evening|dinner| +66672|AAAAAAAABHEABAAA|66672|18|31|12|PM|second|evening|dinner| +66673|AAAAAAAACHEABAAA|66673|18|31|13|PM|second|evening|dinner| +66674|AAAAAAAADHEABAAA|66674|18|31|14|PM|second|evening|dinner| +66675|AAAAAAAAEHEABAAA|66675|18|31|15|PM|second|evening|dinner| +66676|AAAAAAAAFHEABAAA|66676|18|31|16|PM|second|evening|dinner| +66677|AAAAAAAAGHEABAAA|66677|18|31|17|PM|second|evening|dinner| +66678|AAAAAAAAHHEABAAA|66678|18|31|18|PM|second|evening|dinner| +66679|AAAAAAAAIHEABAAA|66679|18|31|19|PM|second|evening|dinner| +66680|AAAAAAAAJHEABAAA|66680|18|31|20|PM|second|evening|dinner| +66681|AAAAAAAAKHEABAAA|66681|18|31|21|PM|second|evening|dinner| +66682|AAAAAAAALHEABAAA|66682|18|31|22|PM|second|evening|dinner| +66683|AAAAAAAAMHEABAAA|66683|18|31|23|PM|second|evening|dinner| +66684|AAAAAAAANHEABAAA|66684|18|31|24|PM|second|evening|dinner| +66685|AAAAAAAAOHEABAAA|66685|18|31|25|PM|second|evening|dinner| +66686|AAAAAAAAPHEABAAA|66686|18|31|26|PM|second|evening|dinner| +66687|AAAAAAAAAIEABAAA|66687|18|31|27|PM|second|evening|dinner| +66688|AAAAAAAABIEABAAA|66688|18|31|28|PM|second|evening|dinner| +66689|AAAAAAAACIEABAAA|66689|18|31|29|PM|second|evening|dinner| +66690|AAAAAAAADIEABAAA|66690|18|31|30|PM|second|evening|dinner| +66691|AAAAAAAAEIEABAAA|66691|18|31|31|PM|second|evening|dinner| +66692|AAAAAAAAFIEABAAA|66692|18|31|32|PM|second|evening|dinner| +66693|AAAAAAAAGIEABAAA|66693|18|31|33|PM|second|evening|dinner| +66694|AAAAAAAAHIEABAAA|66694|18|31|34|PM|second|evening|dinner| +66695|AAAAAAAAIIEABAAA|66695|18|31|35|PM|second|evening|dinner| +66696|AAAAAAAAJIEABAAA|66696|18|31|36|PM|second|evening|dinner| +66697|AAAAAAAAKIEABAAA|66697|18|31|37|PM|second|evening|dinner| +66698|AAAAAAAALIEABAAA|66698|18|31|38|PM|second|evening|dinner| +66699|AAAAAAAAMIEABAAA|66699|18|31|39|PM|second|evening|dinner| +66700|AAAAAAAANIEABAAA|66700|18|31|40|PM|second|evening|dinner| +66701|AAAAAAAAOIEABAAA|66701|18|31|41|PM|second|evening|dinner| +66702|AAAAAAAAPIEABAAA|66702|18|31|42|PM|second|evening|dinner| +66703|AAAAAAAAAJEABAAA|66703|18|31|43|PM|second|evening|dinner| +66704|AAAAAAAABJEABAAA|66704|18|31|44|PM|second|evening|dinner| +66705|AAAAAAAACJEABAAA|66705|18|31|45|PM|second|evening|dinner| +66706|AAAAAAAADJEABAAA|66706|18|31|46|PM|second|evening|dinner| +66707|AAAAAAAAEJEABAAA|66707|18|31|47|PM|second|evening|dinner| +66708|AAAAAAAAFJEABAAA|66708|18|31|48|PM|second|evening|dinner| +66709|AAAAAAAAGJEABAAA|66709|18|31|49|PM|second|evening|dinner| +66710|AAAAAAAAHJEABAAA|66710|18|31|50|PM|second|evening|dinner| +66711|AAAAAAAAIJEABAAA|66711|18|31|51|PM|second|evening|dinner| +66712|AAAAAAAAJJEABAAA|66712|18|31|52|PM|second|evening|dinner| +66713|AAAAAAAAKJEABAAA|66713|18|31|53|PM|second|evening|dinner| +66714|AAAAAAAALJEABAAA|66714|18|31|54|PM|second|evening|dinner| +66715|AAAAAAAAMJEABAAA|66715|18|31|55|PM|second|evening|dinner| +66716|AAAAAAAANJEABAAA|66716|18|31|56|PM|second|evening|dinner| +66717|AAAAAAAAOJEABAAA|66717|18|31|57|PM|second|evening|dinner| +66718|AAAAAAAAPJEABAAA|66718|18|31|58|PM|second|evening|dinner| +66719|AAAAAAAAAKEABAAA|66719|18|31|59|PM|second|evening|dinner| +66720|AAAAAAAABKEABAAA|66720|18|32|0|PM|second|evening|dinner| +66721|AAAAAAAACKEABAAA|66721|18|32|1|PM|second|evening|dinner| +66722|AAAAAAAADKEABAAA|66722|18|32|2|PM|second|evening|dinner| +66723|AAAAAAAAEKEABAAA|66723|18|32|3|PM|second|evening|dinner| +66724|AAAAAAAAFKEABAAA|66724|18|32|4|PM|second|evening|dinner| +66725|AAAAAAAAGKEABAAA|66725|18|32|5|PM|second|evening|dinner| +66726|AAAAAAAAHKEABAAA|66726|18|32|6|PM|second|evening|dinner| +66727|AAAAAAAAIKEABAAA|66727|18|32|7|PM|second|evening|dinner| +66728|AAAAAAAAJKEABAAA|66728|18|32|8|PM|second|evening|dinner| +66729|AAAAAAAAKKEABAAA|66729|18|32|9|PM|second|evening|dinner| +66730|AAAAAAAALKEABAAA|66730|18|32|10|PM|second|evening|dinner| +66731|AAAAAAAAMKEABAAA|66731|18|32|11|PM|second|evening|dinner| +66732|AAAAAAAANKEABAAA|66732|18|32|12|PM|second|evening|dinner| +66733|AAAAAAAAOKEABAAA|66733|18|32|13|PM|second|evening|dinner| +66734|AAAAAAAAPKEABAAA|66734|18|32|14|PM|second|evening|dinner| +66735|AAAAAAAAALEABAAA|66735|18|32|15|PM|second|evening|dinner| +66736|AAAAAAAABLEABAAA|66736|18|32|16|PM|second|evening|dinner| +66737|AAAAAAAACLEABAAA|66737|18|32|17|PM|second|evening|dinner| +66738|AAAAAAAADLEABAAA|66738|18|32|18|PM|second|evening|dinner| +66739|AAAAAAAAELEABAAA|66739|18|32|19|PM|second|evening|dinner| +66740|AAAAAAAAFLEABAAA|66740|18|32|20|PM|second|evening|dinner| +66741|AAAAAAAAGLEABAAA|66741|18|32|21|PM|second|evening|dinner| +66742|AAAAAAAAHLEABAAA|66742|18|32|22|PM|second|evening|dinner| +66743|AAAAAAAAILEABAAA|66743|18|32|23|PM|second|evening|dinner| +66744|AAAAAAAAJLEABAAA|66744|18|32|24|PM|second|evening|dinner| +66745|AAAAAAAAKLEABAAA|66745|18|32|25|PM|second|evening|dinner| +66746|AAAAAAAALLEABAAA|66746|18|32|26|PM|second|evening|dinner| +66747|AAAAAAAAMLEABAAA|66747|18|32|27|PM|second|evening|dinner| +66748|AAAAAAAANLEABAAA|66748|18|32|28|PM|second|evening|dinner| +66749|AAAAAAAAOLEABAAA|66749|18|32|29|PM|second|evening|dinner| +66750|AAAAAAAAPLEABAAA|66750|18|32|30|PM|second|evening|dinner| +66751|AAAAAAAAAMEABAAA|66751|18|32|31|PM|second|evening|dinner| +66752|AAAAAAAABMEABAAA|66752|18|32|32|PM|second|evening|dinner| +66753|AAAAAAAACMEABAAA|66753|18|32|33|PM|second|evening|dinner| +66754|AAAAAAAADMEABAAA|66754|18|32|34|PM|second|evening|dinner| +66755|AAAAAAAAEMEABAAA|66755|18|32|35|PM|second|evening|dinner| +66756|AAAAAAAAFMEABAAA|66756|18|32|36|PM|second|evening|dinner| +66757|AAAAAAAAGMEABAAA|66757|18|32|37|PM|second|evening|dinner| +66758|AAAAAAAAHMEABAAA|66758|18|32|38|PM|second|evening|dinner| +66759|AAAAAAAAIMEABAAA|66759|18|32|39|PM|second|evening|dinner| +66760|AAAAAAAAJMEABAAA|66760|18|32|40|PM|second|evening|dinner| +66761|AAAAAAAAKMEABAAA|66761|18|32|41|PM|second|evening|dinner| +66762|AAAAAAAALMEABAAA|66762|18|32|42|PM|second|evening|dinner| +66763|AAAAAAAAMMEABAAA|66763|18|32|43|PM|second|evening|dinner| +66764|AAAAAAAANMEABAAA|66764|18|32|44|PM|second|evening|dinner| +66765|AAAAAAAAOMEABAAA|66765|18|32|45|PM|second|evening|dinner| +66766|AAAAAAAAPMEABAAA|66766|18|32|46|PM|second|evening|dinner| +66767|AAAAAAAAANEABAAA|66767|18|32|47|PM|second|evening|dinner| +66768|AAAAAAAABNEABAAA|66768|18|32|48|PM|second|evening|dinner| +66769|AAAAAAAACNEABAAA|66769|18|32|49|PM|second|evening|dinner| +66770|AAAAAAAADNEABAAA|66770|18|32|50|PM|second|evening|dinner| +66771|AAAAAAAAENEABAAA|66771|18|32|51|PM|second|evening|dinner| +66772|AAAAAAAAFNEABAAA|66772|18|32|52|PM|second|evening|dinner| +66773|AAAAAAAAGNEABAAA|66773|18|32|53|PM|second|evening|dinner| +66774|AAAAAAAAHNEABAAA|66774|18|32|54|PM|second|evening|dinner| +66775|AAAAAAAAINEABAAA|66775|18|32|55|PM|second|evening|dinner| +66776|AAAAAAAAJNEABAAA|66776|18|32|56|PM|second|evening|dinner| +66777|AAAAAAAAKNEABAAA|66777|18|32|57|PM|second|evening|dinner| +66778|AAAAAAAALNEABAAA|66778|18|32|58|PM|second|evening|dinner| +66779|AAAAAAAAMNEABAAA|66779|18|32|59|PM|second|evening|dinner| +66780|AAAAAAAANNEABAAA|66780|18|33|0|PM|second|evening|dinner| +66781|AAAAAAAAONEABAAA|66781|18|33|1|PM|second|evening|dinner| +66782|AAAAAAAAPNEABAAA|66782|18|33|2|PM|second|evening|dinner| +66783|AAAAAAAAAOEABAAA|66783|18|33|3|PM|second|evening|dinner| +66784|AAAAAAAABOEABAAA|66784|18|33|4|PM|second|evening|dinner| +66785|AAAAAAAACOEABAAA|66785|18|33|5|PM|second|evening|dinner| +66786|AAAAAAAADOEABAAA|66786|18|33|6|PM|second|evening|dinner| +66787|AAAAAAAAEOEABAAA|66787|18|33|7|PM|second|evening|dinner| +66788|AAAAAAAAFOEABAAA|66788|18|33|8|PM|second|evening|dinner| +66789|AAAAAAAAGOEABAAA|66789|18|33|9|PM|second|evening|dinner| +66790|AAAAAAAAHOEABAAA|66790|18|33|10|PM|second|evening|dinner| +66791|AAAAAAAAIOEABAAA|66791|18|33|11|PM|second|evening|dinner| +66792|AAAAAAAAJOEABAAA|66792|18|33|12|PM|second|evening|dinner| +66793|AAAAAAAAKOEABAAA|66793|18|33|13|PM|second|evening|dinner| +66794|AAAAAAAALOEABAAA|66794|18|33|14|PM|second|evening|dinner| +66795|AAAAAAAAMOEABAAA|66795|18|33|15|PM|second|evening|dinner| +66796|AAAAAAAANOEABAAA|66796|18|33|16|PM|second|evening|dinner| +66797|AAAAAAAAOOEABAAA|66797|18|33|17|PM|second|evening|dinner| +66798|AAAAAAAAPOEABAAA|66798|18|33|18|PM|second|evening|dinner| +66799|AAAAAAAAAPEABAAA|66799|18|33|19|PM|second|evening|dinner| +66800|AAAAAAAABPEABAAA|66800|18|33|20|PM|second|evening|dinner| +66801|AAAAAAAACPEABAAA|66801|18|33|21|PM|second|evening|dinner| +66802|AAAAAAAADPEABAAA|66802|18|33|22|PM|second|evening|dinner| +66803|AAAAAAAAEPEABAAA|66803|18|33|23|PM|second|evening|dinner| +66804|AAAAAAAAFPEABAAA|66804|18|33|24|PM|second|evening|dinner| +66805|AAAAAAAAGPEABAAA|66805|18|33|25|PM|second|evening|dinner| +66806|AAAAAAAAHPEABAAA|66806|18|33|26|PM|second|evening|dinner| +66807|AAAAAAAAIPEABAAA|66807|18|33|27|PM|second|evening|dinner| +66808|AAAAAAAAJPEABAAA|66808|18|33|28|PM|second|evening|dinner| +66809|AAAAAAAAKPEABAAA|66809|18|33|29|PM|second|evening|dinner| +66810|AAAAAAAALPEABAAA|66810|18|33|30|PM|second|evening|dinner| +66811|AAAAAAAAMPEABAAA|66811|18|33|31|PM|second|evening|dinner| +66812|AAAAAAAANPEABAAA|66812|18|33|32|PM|second|evening|dinner| +66813|AAAAAAAAOPEABAAA|66813|18|33|33|PM|second|evening|dinner| +66814|AAAAAAAAPPEABAAA|66814|18|33|34|PM|second|evening|dinner| +66815|AAAAAAAAAAFABAAA|66815|18|33|35|PM|second|evening|dinner| +66816|AAAAAAAABAFABAAA|66816|18|33|36|PM|second|evening|dinner| +66817|AAAAAAAACAFABAAA|66817|18|33|37|PM|second|evening|dinner| +66818|AAAAAAAADAFABAAA|66818|18|33|38|PM|second|evening|dinner| +66819|AAAAAAAAEAFABAAA|66819|18|33|39|PM|second|evening|dinner| +66820|AAAAAAAAFAFABAAA|66820|18|33|40|PM|second|evening|dinner| +66821|AAAAAAAAGAFABAAA|66821|18|33|41|PM|second|evening|dinner| +66822|AAAAAAAAHAFABAAA|66822|18|33|42|PM|second|evening|dinner| +66823|AAAAAAAAIAFABAAA|66823|18|33|43|PM|second|evening|dinner| +66824|AAAAAAAAJAFABAAA|66824|18|33|44|PM|second|evening|dinner| +66825|AAAAAAAAKAFABAAA|66825|18|33|45|PM|second|evening|dinner| +66826|AAAAAAAALAFABAAA|66826|18|33|46|PM|second|evening|dinner| +66827|AAAAAAAAMAFABAAA|66827|18|33|47|PM|second|evening|dinner| +66828|AAAAAAAANAFABAAA|66828|18|33|48|PM|second|evening|dinner| +66829|AAAAAAAAOAFABAAA|66829|18|33|49|PM|second|evening|dinner| +66830|AAAAAAAAPAFABAAA|66830|18|33|50|PM|second|evening|dinner| +66831|AAAAAAAAABFABAAA|66831|18|33|51|PM|second|evening|dinner| +66832|AAAAAAAABBFABAAA|66832|18|33|52|PM|second|evening|dinner| +66833|AAAAAAAACBFABAAA|66833|18|33|53|PM|second|evening|dinner| +66834|AAAAAAAADBFABAAA|66834|18|33|54|PM|second|evening|dinner| +66835|AAAAAAAAEBFABAAA|66835|18|33|55|PM|second|evening|dinner| +66836|AAAAAAAAFBFABAAA|66836|18|33|56|PM|second|evening|dinner| +66837|AAAAAAAAGBFABAAA|66837|18|33|57|PM|second|evening|dinner| +66838|AAAAAAAAHBFABAAA|66838|18|33|58|PM|second|evening|dinner| +66839|AAAAAAAAIBFABAAA|66839|18|33|59|PM|second|evening|dinner| +66840|AAAAAAAAJBFABAAA|66840|18|34|0|PM|second|evening|dinner| +66841|AAAAAAAAKBFABAAA|66841|18|34|1|PM|second|evening|dinner| +66842|AAAAAAAALBFABAAA|66842|18|34|2|PM|second|evening|dinner| +66843|AAAAAAAAMBFABAAA|66843|18|34|3|PM|second|evening|dinner| +66844|AAAAAAAANBFABAAA|66844|18|34|4|PM|second|evening|dinner| +66845|AAAAAAAAOBFABAAA|66845|18|34|5|PM|second|evening|dinner| +66846|AAAAAAAAPBFABAAA|66846|18|34|6|PM|second|evening|dinner| +66847|AAAAAAAAACFABAAA|66847|18|34|7|PM|second|evening|dinner| +66848|AAAAAAAABCFABAAA|66848|18|34|8|PM|second|evening|dinner| +66849|AAAAAAAACCFABAAA|66849|18|34|9|PM|second|evening|dinner| +66850|AAAAAAAADCFABAAA|66850|18|34|10|PM|second|evening|dinner| +66851|AAAAAAAAECFABAAA|66851|18|34|11|PM|second|evening|dinner| +66852|AAAAAAAAFCFABAAA|66852|18|34|12|PM|second|evening|dinner| +66853|AAAAAAAAGCFABAAA|66853|18|34|13|PM|second|evening|dinner| +66854|AAAAAAAAHCFABAAA|66854|18|34|14|PM|second|evening|dinner| +66855|AAAAAAAAICFABAAA|66855|18|34|15|PM|second|evening|dinner| +66856|AAAAAAAAJCFABAAA|66856|18|34|16|PM|second|evening|dinner| +66857|AAAAAAAAKCFABAAA|66857|18|34|17|PM|second|evening|dinner| +66858|AAAAAAAALCFABAAA|66858|18|34|18|PM|second|evening|dinner| +66859|AAAAAAAAMCFABAAA|66859|18|34|19|PM|second|evening|dinner| +66860|AAAAAAAANCFABAAA|66860|18|34|20|PM|second|evening|dinner| +66861|AAAAAAAAOCFABAAA|66861|18|34|21|PM|second|evening|dinner| +66862|AAAAAAAAPCFABAAA|66862|18|34|22|PM|second|evening|dinner| +66863|AAAAAAAAADFABAAA|66863|18|34|23|PM|second|evening|dinner| +66864|AAAAAAAABDFABAAA|66864|18|34|24|PM|second|evening|dinner| +66865|AAAAAAAACDFABAAA|66865|18|34|25|PM|second|evening|dinner| +66866|AAAAAAAADDFABAAA|66866|18|34|26|PM|second|evening|dinner| +66867|AAAAAAAAEDFABAAA|66867|18|34|27|PM|second|evening|dinner| +66868|AAAAAAAAFDFABAAA|66868|18|34|28|PM|second|evening|dinner| +66869|AAAAAAAAGDFABAAA|66869|18|34|29|PM|second|evening|dinner| +66870|AAAAAAAAHDFABAAA|66870|18|34|30|PM|second|evening|dinner| +66871|AAAAAAAAIDFABAAA|66871|18|34|31|PM|second|evening|dinner| +66872|AAAAAAAAJDFABAAA|66872|18|34|32|PM|second|evening|dinner| +66873|AAAAAAAAKDFABAAA|66873|18|34|33|PM|second|evening|dinner| +66874|AAAAAAAALDFABAAA|66874|18|34|34|PM|second|evening|dinner| +66875|AAAAAAAAMDFABAAA|66875|18|34|35|PM|second|evening|dinner| +66876|AAAAAAAANDFABAAA|66876|18|34|36|PM|second|evening|dinner| +66877|AAAAAAAAODFABAAA|66877|18|34|37|PM|second|evening|dinner| +66878|AAAAAAAAPDFABAAA|66878|18|34|38|PM|second|evening|dinner| +66879|AAAAAAAAAEFABAAA|66879|18|34|39|PM|second|evening|dinner| +66880|AAAAAAAABEFABAAA|66880|18|34|40|PM|second|evening|dinner| +66881|AAAAAAAACEFABAAA|66881|18|34|41|PM|second|evening|dinner| +66882|AAAAAAAADEFABAAA|66882|18|34|42|PM|second|evening|dinner| +66883|AAAAAAAAEEFABAAA|66883|18|34|43|PM|second|evening|dinner| +66884|AAAAAAAAFEFABAAA|66884|18|34|44|PM|second|evening|dinner| +66885|AAAAAAAAGEFABAAA|66885|18|34|45|PM|second|evening|dinner| +66886|AAAAAAAAHEFABAAA|66886|18|34|46|PM|second|evening|dinner| +66887|AAAAAAAAIEFABAAA|66887|18|34|47|PM|second|evening|dinner| +66888|AAAAAAAAJEFABAAA|66888|18|34|48|PM|second|evening|dinner| +66889|AAAAAAAAKEFABAAA|66889|18|34|49|PM|second|evening|dinner| +66890|AAAAAAAALEFABAAA|66890|18|34|50|PM|second|evening|dinner| +66891|AAAAAAAAMEFABAAA|66891|18|34|51|PM|second|evening|dinner| +66892|AAAAAAAANEFABAAA|66892|18|34|52|PM|second|evening|dinner| +66893|AAAAAAAAOEFABAAA|66893|18|34|53|PM|second|evening|dinner| +66894|AAAAAAAAPEFABAAA|66894|18|34|54|PM|second|evening|dinner| +66895|AAAAAAAAAFFABAAA|66895|18|34|55|PM|second|evening|dinner| +66896|AAAAAAAABFFABAAA|66896|18|34|56|PM|second|evening|dinner| +66897|AAAAAAAACFFABAAA|66897|18|34|57|PM|second|evening|dinner| +66898|AAAAAAAADFFABAAA|66898|18|34|58|PM|second|evening|dinner| +66899|AAAAAAAAEFFABAAA|66899|18|34|59|PM|second|evening|dinner| +66900|AAAAAAAAFFFABAAA|66900|18|35|0|PM|second|evening|dinner| +66901|AAAAAAAAGFFABAAA|66901|18|35|1|PM|second|evening|dinner| +66902|AAAAAAAAHFFABAAA|66902|18|35|2|PM|second|evening|dinner| +66903|AAAAAAAAIFFABAAA|66903|18|35|3|PM|second|evening|dinner| +66904|AAAAAAAAJFFABAAA|66904|18|35|4|PM|second|evening|dinner| +66905|AAAAAAAAKFFABAAA|66905|18|35|5|PM|second|evening|dinner| +66906|AAAAAAAALFFABAAA|66906|18|35|6|PM|second|evening|dinner| +66907|AAAAAAAAMFFABAAA|66907|18|35|7|PM|second|evening|dinner| +66908|AAAAAAAANFFABAAA|66908|18|35|8|PM|second|evening|dinner| +66909|AAAAAAAAOFFABAAA|66909|18|35|9|PM|second|evening|dinner| +66910|AAAAAAAAPFFABAAA|66910|18|35|10|PM|second|evening|dinner| +66911|AAAAAAAAAGFABAAA|66911|18|35|11|PM|second|evening|dinner| +66912|AAAAAAAABGFABAAA|66912|18|35|12|PM|second|evening|dinner| +66913|AAAAAAAACGFABAAA|66913|18|35|13|PM|second|evening|dinner| +66914|AAAAAAAADGFABAAA|66914|18|35|14|PM|second|evening|dinner| +66915|AAAAAAAAEGFABAAA|66915|18|35|15|PM|second|evening|dinner| +66916|AAAAAAAAFGFABAAA|66916|18|35|16|PM|second|evening|dinner| +66917|AAAAAAAAGGFABAAA|66917|18|35|17|PM|second|evening|dinner| +66918|AAAAAAAAHGFABAAA|66918|18|35|18|PM|second|evening|dinner| +66919|AAAAAAAAIGFABAAA|66919|18|35|19|PM|second|evening|dinner| +66920|AAAAAAAAJGFABAAA|66920|18|35|20|PM|second|evening|dinner| +66921|AAAAAAAAKGFABAAA|66921|18|35|21|PM|second|evening|dinner| +66922|AAAAAAAALGFABAAA|66922|18|35|22|PM|second|evening|dinner| +66923|AAAAAAAAMGFABAAA|66923|18|35|23|PM|second|evening|dinner| +66924|AAAAAAAANGFABAAA|66924|18|35|24|PM|second|evening|dinner| +66925|AAAAAAAAOGFABAAA|66925|18|35|25|PM|second|evening|dinner| +66926|AAAAAAAAPGFABAAA|66926|18|35|26|PM|second|evening|dinner| +66927|AAAAAAAAAHFABAAA|66927|18|35|27|PM|second|evening|dinner| +66928|AAAAAAAABHFABAAA|66928|18|35|28|PM|second|evening|dinner| +66929|AAAAAAAACHFABAAA|66929|18|35|29|PM|second|evening|dinner| +66930|AAAAAAAADHFABAAA|66930|18|35|30|PM|second|evening|dinner| +66931|AAAAAAAAEHFABAAA|66931|18|35|31|PM|second|evening|dinner| +66932|AAAAAAAAFHFABAAA|66932|18|35|32|PM|second|evening|dinner| +66933|AAAAAAAAGHFABAAA|66933|18|35|33|PM|second|evening|dinner| +66934|AAAAAAAAHHFABAAA|66934|18|35|34|PM|second|evening|dinner| +66935|AAAAAAAAIHFABAAA|66935|18|35|35|PM|second|evening|dinner| +66936|AAAAAAAAJHFABAAA|66936|18|35|36|PM|second|evening|dinner| +66937|AAAAAAAAKHFABAAA|66937|18|35|37|PM|second|evening|dinner| +66938|AAAAAAAALHFABAAA|66938|18|35|38|PM|second|evening|dinner| +66939|AAAAAAAAMHFABAAA|66939|18|35|39|PM|second|evening|dinner| +66940|AAAAAAAANHFABAAA|66940|18|35|40|PM|second|evening|dinner| +66941|AAAAAAAAOHFABAAA|66941|18|35|41|PM|second|evening|dinner| +66942|AAAAAAAAPHFABAAA|66942|18|35|42|PM|second|evening|dinner| +66943|AAAAAAAAAIFABAAA|66943|18|35|43|PM|second|evening|dinner| +66944|AAAAAAAABIFABAAA|66944|18|35|44|PM|second|evening|dinner| +66945|AAAAAAAACIFABAAA|66945|18|35|45|PM|second|evening|dinner| +66946|AAAAAAAADIFABAAA|66946|18|35|46|PM|second|evening|dinner| +66947|AAAAAAAAEIFABAAA|66947|18|35|47|PM|second|evening|dinner| +66948|AAAAAAAAFIFABAAA|66948|18|35|48|PM|second|evening|dinner| +66949|AAAAAAAAGIFABAAA|66949|18|35|49|PM|second|evening|dinner| +66950|AAAAAAAAHIFABAAA|66950|18|35|50|PM|second|evening|dinner| +66951|AAAAAAAAIIFABAAA|66951|18|35|51|PM|second|evening|dinner| +66952|AAAAAAAAJIFABAAA|66952|18|35|52|PM|second|evening|dinner| +66953|AAAAAAAAKIFABAAA|66953|18|35|53|PM|second|evening|dinner| +66954|AAAAAAAALIFABAAA|66954|18|35|54|PM|second|evening|dinner| +66955|AAAAAAAAMIFABAAA|66955|18|35|55|PM|second|evening|dinner| +66956|AAAAAAAANIFABAAA|66956|18|35|56|PM|second|evening|dinner| +66957|AAAAAAAAOIFABAAA|66957|18|35|57|PM|second|evening|dinner| +66958|AAAAAAAAPIFABAAA|66958|18|35|58|PM|second|evening|dinner| +66959|AAAAAAAAAJFABAAA|66959|18|35|59|PM|second|evening|dinner| +66960|AAAAAAAABJFABAAA|66960|18|36|0|PM|second|evening|dinner| +66961|AAAAAAAACJFABAAA|66961|18|36|1|PM|second|evening|dinner| +66962|AAAAAAAADJFABAAA|66962|18|36|2|PM|second|evening|dinner| +66963|AAAAAAAAEJFABAAA|66963|18|36|3|PM|second|evening|dinner| +66964|AAAAAAAAFJFABAAA|66964|18|36|4|PM|second|evening|dinner| +66965|AAAAAAAAGJFABAAA|66965|18|36|5|PM|second|evening|dinner| +66966|AAAAAAAAHJFABAAA|66966|18|36|6|PM|second|evening|dinner| +66967|AAAAAAAAIJFABAAA|66967|18|36|7|PM|second|evening|dinner| +66968|AAAAAAAAJJFABAAA|66968|18|36|8|PM|second|evening|dinner| +66969|AAAAAAAAKJFABAAA|66969|18|36|9|PM|second|evening|dinner| +66970|AAAAAAAALJFABAAA|66970|18|36|10|PM|second|evening|dinner| +66971|AAAAAAAAMJFABAAA|66971|18|36|11|PM|second|evening|dinner| +66972|AAAAAAAANJFABAAA|66972|18|36|12|PM|second|evening|dinner| +66973|AAAAAAAAOJFABAAA|66973|18|36|13|PM|second|evening|dinner| +66974|AAAAAAAAPJFABAAA|66974|18|36|14|PM|second|evening|dinner| +66975|AAAAAAAAAKFABAAA|66975|18|36|15|PM|second|evening|dinner| +66976|AAAAAAAABKFABAAA|66976|18|36|16|PM|second|evening|dinner| +66977|AAAAAAAACKFABAAA|66977|18|36|17|PM|second|evening|dinner| +66978|AAAAAAAADKFABAAA|66978|18|36|18|PM|second|evening|dinner| +66979|AAAAAAAAEKFABAAA|66979|18|36|19|PM|second|evening|dinner| +66980|AAAAAAAAFKFABAAA|66980|18|36|20|PM|second|evening|dinner| +66981|AAAAAAAAGKFABAAA|66981|18|36|21|PM|second|evening|dinner| +66982|AAAAAAAAHKFABAAA|66982|18|36|22|PM|second|evening|dinner| +66983|AAAAAAAAIKFABAAA|66983|18|36|23|PM|second|evening|dinner| +66984|AAAAAAAAJKFABAAA|66984|18|36|24|PM|second|evening|dinner| +66985|AAAAAAAAKKFABAAA|66985|18|36|25|PM|second|evening|dinner| +66986|AAAAAAAALKFABAAA|66986|18|36|26|PM|second|evening|dinner| +66987|AAAAAAAAMKFABAAA|66987|18|36|27|PM|second|evening|dinner| +66988|AAAAAAAANKFABAAA|66988|18|36|28|PM|second|evening|dinner| +66989|AAAAAAAAOKFABAAA|66989|18|36|29|PM|second|evening|dinner| +66990|AAAAAAAAPKFABAAA|66990|18|36|30|PM|second|evening|dinner| +66991|AAAAAAAAALFABAAA|66991|18|36|31|PM|second|evening|dinner| +66992|AAAAAAAABLFABAAA|66992|18|36|32|PM|second|evening|dinner| +66993|AAAAAAAACLFABAAA|66993|18|36|33|PM|second|evening|dinner| +66994|AAAAAAAADLFABAAA|66994|18|36|34|PM|second|evening|dinner| +66995|AAAAAAAAELFABAAA|66995|18|36|35|PM|second|evening|dinner| +66996|AAAAAAAAFLFABAAA|66996|18|36|36|PM|second|evening|dinner| +66997|AAAAAAAAGLFABAAA|66997|18|36|37|PM|second|evening|dinner| +66998|AAAAAAAAHLFABAAA|66998|18|36|38|PM|second|evening|dinner| +66999|AAAAAAAAILFABAAA|66999|18|36|39|PM|second|evening|dinner| +67000|AAAAAAAAJLFABAAA|67000|18|36|40|PM|second|evening|dinner| +67001|AAAAAAAAKLFABAAA|67001|18|36|41|PM|second|evening|dinner| +67002|AAAAAAAALLFABAAA|67002|18|36|42|PM|second|evening|dinner| +67003|AAAAAAAAMLFABAAA|67003|18|36|43|PM|second|evening|dinner| +67004|AAAAAAAANLFABAAA|67004|18|36|44|PM|second|evening|dinner| +67005|AAAAAAAAOLFABAAA|67005|18|36|45|PM|second|evening|dinner| +67006|AAAAAAAAPLFABAAA|67006|18|36|46|PM|second|evening|dinner| +67007|AAAAAAAAAMFABAAA|67007|18|36|47|PM|second|evening|dinner| +67008|AAAAAAAABMFABAAA|67008|18|36|48|PM|second|evening|dinner| +67009|AAAAAAAACMFABAAA|67009|18|36|49|PM|second|evening|dinner| +67010|AAAAAAAADMFABAAA|67010|18|36|50|PM|second|evening|dinner| +67011|AAAAAAAAEMFABAAA|67011|18|36|51|PM|second|evening|dinner| +67012|AAAAAAAAFMFABAAA|67012|18|36|52|PM|second|evening|dinner| +67013|AAAAAAAAGMFABAAA|67013|18|36|53|PM|second|evening|dinner| +67014|AAAAAAAAHMFABAAA|67014|18|36|54|PM|second|evening|dinner| +67015|AAAAAAAAIMFABAAA|67015|18|36|55|PM|second|evening|dinner| +67016|AAAAAAAAJMFABAAA|67016|18|36|56|PM|second|evening|dinner| +67017|AAAAAAAAKMFABAAA|67017|18|36|57|PM|second|evening|dinner| +67018|AAAAAAAALMFABAAA|67018|18|36|58|PM|second|evening|dinner| +67019|AAAAAAAAMMFABAAA|67019|18|36|59|PM|second|evening|dinner| +67020|AAAAAAAANMFABAAA|67020|18|37|0|PM|second|evening|dinner| +67021|AAAAAAAAOMFABAAA|67021|18|37|1|PM|second|evening|dinner| +67022|AAAAAAAAPMFABAAA|67022|18|37|2|PM|second|evening|dinner| +67023|AAAAAAAAANFABAAA|67023|18|37|3|PM|second|evening|dinner| +67024|AAAAAAAABNFABAAA|67024|18|37|4|PM|second|evening|dinner| +67025|AAAAAAAACNFABAAA|67025|18|37|5|PM|second|evening|dinner| +67026|AAAAAAAADNFABAAA|67026|18|37|6|PM|second|evening|dinner| +67027|AAAAAAAAENFABAAA|67027|18|37|7|PM|second|evening|dinner| +67028|AAAAAAAAFNFABAAA|67028|18|37|8|PM|second|evening|dinner| +67029|AAAAAAAAGNFABAAA|67029|18|37|9|PM|second|evening|dinner| +67030|AAAAAAAAHNFABAAA|67030|18|37|10|PM|second|evening|dinner| +67031|AAAAAAAAINFABAAA|67031|18|37|11|PM|second|evening|dinner| +67032|AAAAAAAAJNFABAAA|67032|18|37|12|PM|second|evening|dinner| +67033|AAAAAAAAKNFABAAA|67033|18|37|13|PM|second|evening|dinner| +67034|AAAAAAAALNFABAAA|67034|18|37|14|PM|second|evening|dinner| +67035|AAAAAAAAMNFABAAA|67035|18|37|15|PM|second|evening|dinner| +67036|AAAAAAAANNFABAAA|67036|18|37|16|PM|second|evening|dinner| +67037|AAAAAAAAONFABAAA|67037|18|37|17|PM|second|evening|dinner| +67038|AAAAAAAAPNFABAAA|67038|18|37|18|PM|second|evening|dinner| +67039|AAAAAAAAAOFABAAA|67039|18|37|19|PM|second|evening|dinner| +67040|AAAAAAAABOFABAAA|67040|18|37|20|PM|second|evening|dinner| +67041|AAAAAAAACOFABAAA|67041|18|37|21|PM|second|evening|dinner| +67042|AAAAAAAADOFABAAA|67042|18|37|22|PM|second|evening|dinner| +67043|AAAAAAAAEOFABAAA|67043|18|37|23|PM|second|evening|dinner| +67044|AAAAAAAAFOFABAAA|67044|18|37|24|PM|second|evening|dinner| +67045|AAAAAAAAGOFABAAA|67045|18|37|25|PM|second|evening|dinner| +67046|AAAAAAAAHOFABAAA|67046|18|37|26|PM|second|evening|dinner| +67047|AAAAAAAAIOFABAAA|67047|18|37|27|PM|second|evening|dinner| +67048|AAAAAAAAJOFABAAA|67048|18|37|28|PM|second|evening|dinner| +67049|AAAAAAAAKOFABAAA|67049|18|37|29|PM|second|evening|dinner| +67050|AAAAAAAALOFABAAA|67050|18|37|30|PM|second|evening|dinner| +67051|AAAAAAAAMOFABAAA|67051|18|37|31|PM|second|evening|dinner| +67052|AAAAAAAANOFABAAA|67052|18|37|32|PM|second|evening|dinner| +67053|AAAAAAAAOOFABAAA|67053|18|37|33|PM|second|evening|dinner| +67054|AAAAAAAAPOFABAAA|67054|18|37|34|PM|second|evening|dinner| +67055|AAAAAAAAAPFABAAA|67055|18|37|35|PM|second|evening|dinner| +67056|AAAAAAAABPFABAAA|67056|18|37|36|PM|second|evening|dinner| +67057|AAAAAAAACPFABAAA|67057|18|37|37|PM|second|evening|dinner| +67058|AAAAAAAADPFABAAA|67058|18|37|38|PM|second|evening|dinner| +67059|AAAAAAAAEPFABAAA|67059|18|37|39|PM|second|evening|dinner| +67060|AAAAAAAAFPFABAAA|67060|18|37|40|PM|second|evening|dinner| +67061|AAAAAAAAGPFABAAA|67061|18|37|41|PM|second|evening|dinner| +67062|AAAAAAAAHPFABAAA|67062|18|37|42|PM|second|evening|dinner| +67063|AAAAAAAAIPFABAAA|67063|18|37|43|PM|second|evening|dinner| +67064|AAAAAAAAJPFABAAA|67064|18|37|44|PM|second|evening|dinner| +67065|AAAAAAAAKPFABAAA|67065|18|37|45|PM|second|evening|dinner| +67066|AAAAAAAALPFABAAA|67066|18|37|46|PM|second|evening|dinner| +67067|AAAAAAAAMPFABAAA|67067|18|37|47|PM|second|evening|dinner| +67068|AAAAAAAANPFABAAA|67068|18|37|48|PM|second|evening|dinner| +67069|AAAAAAAAOPFABAAA|67069|18|37|49|PM|second|evening|dinner| +67070|AAAAAAAAPPFABAAA|67070|18|37|50|PM|second|evening|dinner| +67071|AAAAAAAAAAGABAAA|67071|18|37|51|PM|second|evening|dinner| +67072|AAAAAAAABAGABAAA|67072|18|37|52|PM|second|evening|dinner| +67073|AAAAAAAACAGABAAA|67073|18|37|53|PM|second|evening|dinner| +67074|AAAAAAAADAGABAAA|67074|18|37|54|PM|second|evening|dinner| +67075|AAAAAAAAEAGABAAA|67075|18|37|55|PM|second|evening|dinner| +67076|AAAAAAAAFAGABAAA|67076|18|37|56|PM|second|evening|dinner| +67077|AAAAAAAAGAGABAAA|67077|18|37|57|PM|second|evening|dinner| +67078|AAAAAAAAHAGABAAA|67078|18|37|58|PM|second|evening|dinner| +67079|AAAAAAAAIAGABAAA|67079|18|37|59|PM|second|evening|dinner| +67080|AAAAAAAAJAGABAAA|67080|18|38|0|PM|second|evening|dinner| +67081|AAAAAAAAKAGABAAA|67081|18|38|1|PM|second|evening|dinner| +67082|AAAAAAAALAGABAAA|67082|18|38|2|PM|second|evening|dinner| +67083|AAAAAAAAMAGABAAA|67083|18|38|3|PM|second|evening|dinner| +67084|AAAAAAAANAGABAAA|67084|18|38|4|PM|second|evening|dinner| +67085|AAAAAAAAOAGABAAA|67085|18|38|5|PM|second|evening|dinner| +67086|AAAAAAAAPAGABAAA|67086|18|38|6|PM|second|evening|dinner| +67087|AAAAAAAAABGABAAA|67087|18|38|7|PM|second|evening|dinner| +67088|AAAAAAAABBGABAAA|67088|18|38|8|PM|second|evening|dinner| +67089|AAAAAAAACBGABAAA|67089|18|38|9|PM|second|evening|dinner| +67090|AAAAAAAADBGABAAA|67090|18|38|10|PM|second|evening|dinner| +67091|AAAAAAAAEBGABAAA|67091|18|38|11|PM|second|evening|dinner| +67092|AAAAAAAAFBGABAAA|67092|18|38|12|PM|second|evening|dinner| +67093|AAAAAAAAGBGABAAA|67093|18|38|13|PM|second|evening|dinner| +67094|AAAAAAAAHBGABAAA|67094|18|38|14|PM|second|evening|dinner| +67095|AAAAAAAAIBGABAAA|67095|18|38|15|PM|second|evening|dinner| +67096|AAAAAAAAJBGABAAA|67096|18|38|16|PM|second|evening|dinner| +67097|AAAAAAAAKBGABAAA|67097|18|38|17|PM|second|evening|dinner| +67098|AAAAAAAALBGABAAA|67098|18|38|18|PM|second|evening|dinner| +67099|AAAAAAAAMBGABAAA|67099|18|38|19|PM|second|evening|dinner| +67100|AAAAAAAANBGABAAA|67100|18|38|20|PM|second|evening|dinner| +67101|AAAAAAAAOBGABAAA|67101|18|38|21|PM|second|evening|dinner| +67102|AAAAAAAAPBGABAAA|67102|18|38|22|PM|second|evening|dinner| +67103|AAAAAAAAACGABAAA|67103|18|38|23|PM|second|evening|dinner| +67104|AAAAAAAABCGABAAA|67104|18|38|24|PM|second|evening|dinner| +67105|AAAAAAAACCGABAAA|67105|18|38|25|PM|second|evening|dinner| +67106|AAAAAAAADCGABAAA|67106|18|38|26|PM|second|evening|dinner| +67107|AAAAAAAAECGABAAA|67107|18|38|27|PM|second|evening|dinner| +67108|AAAAAAAAFCGABAAA|67108|18|38|28|PM|second|evening|dinner| +67109|AAAAAAAAGCGABAAA|67109|18|38|29|PM|second|evening|dinner| +67110|AAAAAAAAHCGABAAA|67110|18|38|30|PM|second|evening|dinner| +67111|AAAAAAAAICGABAAA|67111|18|38|31|PM|second|evening|dinner| +67112|AAAAAAAAJCGABAAA|67112|18|38|32|PM|second|evening|dinner| +67113|AAAAAAAAKCGABAAA|67113|18|38|33|PM|second|evening|dinner| +67114|AAAAAAAALCGABAAA|67114|18|38|34|PM|second|evening|dinner| +67115|AAAAAAAAMCGABAAA|67115|18|38|35|PM|second|evening|dinner| +67116|AAAAAAAANCGABAAA|67116|18|38|36|PM|second|evening|dinner| +67117|AAAAAAAAOCGABAAA|67117|18|38|37|PM|second|evening|dinner| +67118|AAAAAAAAPCGABAAA|67118|18|38|38|PM|second|evening|dinner| +67119|AAAAAAAAADGABAAA|67119|18|38|39|PM|second|evening|dinner| +67120|AAAAAAAABDGABAAA|67120|18|38|40|PM|second|evening|dinner| +67121|AAAAAAAACDGABAAA|67121|18|38|41|PM|second|evening|dinner| +67122|AAAAAAAADDGABAAA|67122|18|38|42|PM|second|evening|dinner| +67123|AAAAAAAAEDGABAAA|67123|18|38|43|PM|second|evening|dinner| +67124|AAAAAAAAFDGABAAA|67124|18|38|44|PM|second|evening|dinner| +67125|AAAAAAAAGDGABAAA|67125|18|38|45|PM|second|evening|dinner| +67126|AAAAAAAAHDGABAAA|67126|18|38|46|PM|second|evening|dinner| +67127|AAAAAAAAIDGABAAA|67127|18|38|47|PM|second|evening|dinner| +67128|AAAAAAAAJDGABAAA|67128|18|38|48|PM|second|evening|dinner| +67129|AAAAAAAAKDGABAAA|67129|18|38|49|PM|second|evening|dinner| +67130|AAAAAAAALDGABAAA|67130|18|38|50|PM|second|evening|dinner| +67131|AAAAAAAAMDGABAAA|67131|18|38|51|PM|second|evening|dinner| +67132|AAAAAAAANDGABAAA|67132|18|38|52|PM|second|evening|dinner| +67133|AAAAAAAAODGABAAA|67133|18|38|53|PM|second|evening|dinner| +67134|AAAAAAAAPDGABAAA|67134|18|38|54|PM|second|evening|dinner| +67135|AAAAAAAAAEGABAAA|67135|18|38|55|PM|second|evening|dinner| +67136|AAAAAAAABEGABAAA|67136|18|38|56|PM|second|evening|dinner| +67137|AAAAAAAACEGABAAA|67137|18|38|57|PM|second|evening|dinner| +67138|AAAAAAAADEGABAAA|67138|18|38|58|PM|second|evening|dinner| +67139|AAAAAAAAEEGABAAA|67139|18|38|59|PM|second|evening|dinner| +67140|AAAAAAAAFEGABAAA|67140|18|39|0|PM|second|evening|dinner| +67141|AAAAAAAAGEGABAAA|67141|18|39|1|PM|second|evening|dinner| +67142|AAAAAAAAHEGABAAA|67142|18|39|2|PM|second|evening|dinner| +67143|AAAAAAAAIEGABAAA|67143|18|39|3|PM|second|evening|dinner| +67144|AAAAAAAAJEGABAAA|67144|18|39|4|PM|second|evening|dinner| +67145|AAAAAAAAKEGABAAA|67145|18|39|5|PM|second|evening|dinner| +67146|AAAAAAAALEGABAAA|67146|18|39|6|PM|second|evening|dinner| +67147|AAAAAAAAMEGABAAA|67147|18|39|7|PM|second|evening|dinner| +67148|AAAAAAAANEGABAAA|67148|18|39|8|PM|second|evening|dinner| +67149|AAAAAAAAOEGABAAA|67149|18|39|9|PM|second|evening|dinner| +67150|AAAAAAAAPEGABAAA|67150|18|39|10|PM|second|evening|dinner| +67151|AAAAAAAAAFGABAAA|67151|18|39|11|PM|second|evening|dinner| +67152|AAAAAAAABFGABAAA|67152|18|39|12|PM|second|evening|dinner| +67153|AAAAAAAACFGABAAA|67153|18|39|13|PM|second|evening|dinner| +67154|AAAAAAAADFGABAAA|67154|18|39|14|PM|second|evening|dinner| +67155|AAAAAAAAEFGABAAA|67155|18|39|15|PM|second|evening|dinner| +67156|AAAAAAAAFFGABAAA|67156|18|39|16|PM|second|evening|dinner| +67157|AAAAAAAAGFGABAAA|67157|18|39|17|PM|second|evening|dinner| +67158|AAAAAAAAHFGABAAA|67158|18|39|18|PM|second|evening|dinner| +67159|AAAAAAAAIFGABAAA|67159|18|39|19|PM|second|evening|dinner| +67160|AAAAAAAAJFGABAAA|67160|18|39|20|PM|second|evening|dinner| +67161|AAAAAAAAKFGABAAA|67161|18|39|21|PM|second|evening|dinner| +67162|AAAAAAAALFGABAAA|67162|18|39|22|PM|second|evening|dinner| +67163|AAAAAAAAMFGABAAA|67163|18|39|23|PM|second|evening|dinner| +67164|AAAAAAAANFGABAAA|67164|18|39|24|PM|second|evening|dinner| +67165|AAAAAAAAOFGABAAA|67165|18|39|25|PM|second|evening|dinner| +67166|AAAAAAAAPFGABAAA|67166|18|39|26|PM|second|evening|dinner| +67167|AAAAAAAAAGGABAAA|67167|18|39|27|PM|second|evening|dinner| +67168|AAAAAAAABGGABAAA|67168|18|39|28|PM|second|evening|dinner| +67169|AAAAAAAACGGABAAA|67169|18|39|29|PM|second|evening|dinner| +67170|AAAAAAAADGGABAAA|67170|18|39|30|PM|second|evening|dinner| +67171|AAAAAAAAEGGABAAA|67171|18|39|31|PM|second|evening|dinner| +67172|AAAAAAAAFGGABAAA|67172|18|39|32|PM|second|evening|dinner| +67173|AAAAAAAAGGGABAAA|67173|18|39|33|PM|second|evening|dinner| +67174|AAAAAAAAHGGABAAA|67174|18|39|34|PM|second|evening|dinner| +67175|AAAAAAAAIGGABAAA|67175|18|39|35|PM|second|evening|dinner| +67176|AAAAAAAAJGGABAAA|67176|18|39|36|PM|second|evening|dinner| +67177|AAAAAAAAKGGABAAA|67177|18|39|37|PM|second|evening|dinner| +67178|AAAAAAAALGGABAAA|67178|18|39|38|PM|second|evening|dinner| +67179|AAAAAAAAMGGABAAA|67179|18|39|39|PM|second|evening|dinner| +67180|AAAAAAAANGGABAAA|67180|18|39|40|PM|second|evening|dinner| +67181|AAAAAAAAOGGABAAA|67181|18|39|41|PM|second|evening|dinner| +67182|AAAAAAAAPGGABAAA|67182|18|39|42|PM|second|evening|dinner| +67183|AAAAAAAAAHGABAAA|67183|18|39|43|PM|second|evening|dinner| +67184|AAAAAAAABHGABAAA|67184|18|39|44|PM|second|evening|dinner| +67185|AAAAAAAACHGABAAA|67185|18|39|45|PM|second|evening|dinner| +67186|AAAAAAAADHGABAAA|67186|18|39|46|PM|second|evening|dinner| +67187|AAAAAAAAEHGABAAA|67187|18|39|47|PM|second|evening|dinner| +67188|AAAAAAAAFHGABAAA|67188|18|39|48|PM|second|evening|dinner| +67189|AAAAAAAAGHGABAAA|67189|18|39|49|PM|second|evening|dinner| +67190|AAAAAAAAHHGABAAA|67190|18|39|50|PM|second|evening|dinner| +67191|AAAAAAAAIHGABAAA|67191|18|39|51|PM|second|evening|dinner| +67192|AAAAAAAAJHGABAAA|67192|18|39|52|PM|second|evening|dinner| +67193|AAAAAAAAKHGABAAA|67193|18|39|53|PM|second|evening|dinner| +67194|AAAAAAAALHGABAAA|67194|18|39|54|PM|second|evening|dinner| +67195|AAAAAAAAMHGABAAA|67195|18|39|55|PM|second|evening|dinner| +67196|AAAAAAAANHGABAAA|67196|18|39|56|PM|second|evening|dinner| +67197|AAAAAAAAOHGABAAA|67197|18|39|57|PM|second|evening|dinner| +67198|AAAAAAAAPHGABAAA|67198|18|39|58|PM|second|evening|dinner| +67199|AAAAAAAAAIGABAAA|67199|18|39|59|PM|second|evening|dinner| +67200|AAAAAAAABIGABAAA|67200|18|40|0|PM|second|evening|dinner| +67201|AAAAAAAACIGABAAA|67201|18|40|1|PM|second|evening|dinner| +67202|AAAAAAAADIGABAAA|67202|18|40|2|PM|second|evening|dinner| +67203|AAAAAAAAEIGABAAA|67203|18|40|3|PM|second|evening|dinner| +67204|AAAAAAAAFIGABAAA|67204|18|40|4|PM|second|evening|dinner| +67205|AAAAAAAAGIGABAAA|67205|18|40|5|PM|second|evening|dinner| +67206|AAAAAAAAHIGABAAA|67206|18|40|6|PM|second|evening|dinner| +67207|AAAAAAAAIIGABAAA|67207|18|40|7|PM|second|evening|dinner| +67208|AAAAAAAAJIGABAAA|67208|18|40|8|PM|second|evening|dinner| +67209|AAAAAAAAKIGABAAA|67209|18|40|9|PM|second|evening|dinner| +67210|AAAAAAAALIGABAAA|67210|18|40|10|PM|second|evening|dinner| +67211|AAAAAAAAMIGABAAA|67211|18|40|11|PM|second|evening|dinner| +67212|AAAAAAAANIGABAAA|67212|18|40|12|PM|second|evening|dinner| +67213|AAAAAAAAOIGABAAA|67213|18|40|13|PM|second|evening|dinner| +67214|AAAAAAAAPIGABAAA|67214|18|40|14|PM|second|evening|dinner| +67215|AAAAAAAAAJGABAAA|67215|18|40|15|PM|second|evening|dinner| +67216|AAAAAAAABJGABAAA|67216|18|40|16|PM|second|evening|dinner| +67217|AAAAAAAACJGABAAA|67217|18|40|17|PM|second|evening|dinner| +67218|AAAAAAAADJGABAAA|67218|18|40|18|PM|second|evening|dinner| +67219|AAAAAAAAEJGABAAA|67219|18|40|19|PM|second|evening|dinner| +67220|AAAAAAAAFJGABAAA|67220|18|40|20|PM|second|evening|dinner| +67221|AAAAAAAAGJGABAAA|67221|18|40|21|PM|second|evening|dinner| +67222|AAAAAAAAHJGABAAA|67222|18|40|22|PM|second|evening|dinner| +67223|AAAAAAAAIJGABAAA|67223|18|40|23|PM|second|evening|dinner| +67224|AAAAAAAAJJGABAAA|67224|18|40|24|PM|second|evening|dinner| +67225|AAAAAAAAKJGABAAA|67225|18|40|25|PM|second|evening|dinner| +67226|AAAAAAAALJGABAAA|67226|18|40|26|PM|second|evening|dinner| +67227|AAAAAAAAMJGABAAA|67227|18|40|27|PM|second|evening|dinner| +67228|AAAAAAAANJGABAAA|67228|18|40|28|PM|second|evening|dinner| +67229|AAAAAAAAOJGABAAA|67229|18|40|29|PM|second|evening|dinner| +67230|AAAAAAAAPJGABAAA|67230|18|40|30|PM|second|evening|dinner| +67231|AAAAAAAAAKGABAAA|67231|18|40|31|PM|second|evening|dinner| +67232|AAAAAAAABKGABAAA|67232|18|40|32|PM|second|evening|dinner| +67233|AAAAAAAACKGABAAA|67233|18|40|33|PM|second|evening|dinner| +67234|AAAAAAAADKGABAAA|67234|18|40|34|PM|second|evening|dinner| +67235|AAAAAAAAEKGABAAA|67235|18|40|35|PM|second|evening|dinner| +67236|AAAAAAAAFKGABAAA|67236|18|40|36|PM|second|evening|dinner| +67237|AAAAAAAAGKGABAAA|67237|18|40|37|PM|second|evening|dinner| +67238|AAAAAAAAHKGABAAA|67238|18|40|38|PM|second|evening|dinner| +67239|AAAAAAAAIKGABAAA|67239|18|40|39|PM|second|evening|dinner| +67240|AAAAAAAAJKGABAAA|67240|18|40|40|PM|second|evening|dinner| +67241|AAAAAAAAKKGABAAA|67241|18|40|41|PM|second|evening|dinner| +67242|AAAAAAAALKGABAAA|67242|18|40|42|PM|second|evening|dinner| +67243|AAAAAAAAMKGABAAA|67243|18|40|43|PM|second|evening|dinner| +67244|AAAAAAAANKGABAAA|67244|18|40|44|PM|second|evening|dinner| +67245|AAAAAAAAOKGABAAA|67245|18|40|45|PM|second|evening|dinner| +67246|AAAAAAAAPKGABAAA|67246|18|40|46|PM|second|evening|dinner| +67247|AAAAAAAAALGABAAA|67247|18|40|47|PM|second|evening|dinner| +67248|AAAAAAAABLGABAAA|67248|18|40|48|PM|second|evening|dinner| +67249|AAAAAAAACLGABAAA|67249|18|40|49|PM|second|evening|dinner| +67250|AAAAAAAADLGABAAA|67250|18|40|50|PM|second|evening|dinner| +67251|AAAAAAAAELGABAAA|67251|18|40|51|PM|second|evening|dinner| +67252|AAAAAAAAFLGABAAA|67252|18|40|52|PM|second|evening|dinner| +67253|AAAAAAAAGLGABAAA|67253|18|40|53|PM|second|evening|dinner| +67254|AAAAAAAAHLGABAAA|67254|18|40|54|PM|second|evening|dinner| +67255|AAAAAAAAILGABAAA|67255|18|40|55|PM|second|evening|dinner| +67256|AAAAAAAAJLGABAAA|67256|18|40|56|PM|second|evening|dinner| +67257|AAAAAAAAKLGABAAA|67257|18|40|57|PM|second|evening|dinner| +67258|AAAAAAAALLGABAAA|67258|18|40|58|PM|second|evening|dinner| +67259|AAAAAAAAMLGABAAA|67259|18|40|59|PM|second|evening|dinner| +67260|AAAAAAAANLGABAAA|67260|18|41|0|PM|second|evening|dinner| +67261|AAAAAAAAOLGABAAA|67261|18|41|1|PM|second|evening|dinner| +67262|AAAAAAAAPLGABAAA|67262|18|41|2|PM|second|evening|dinner| +67263|AAAAAAAAAMGABAAA|67263|18|41|3|PM|second|evening|dinner| +67264|AAAAAAAABMGABAAA|67264|18|41|4|PM|second|evening|dinner| +67265|AAAAAAAACMGABAAA|67265|18|41|5|PM|second|evening|dinner| +67266|AAAAAAAADMGABAAA|67266|18|41|6|PM|second|evening|dinner| +67267|AAAAAAAAEMGABAAA|67267|18|41|7|PM|second|evening|dinner| +67268|AAAAAAAAFMGABAAA|67268|18|41|8|PM|second|evening|dinner| +67269|AAAAAAAAGMGABAAA|67269|18|41|9|PM|second|evening|dinner| +67270|AAAAAAAAHMGABAAA|67270|18|41|10|PM|second|evening|dinner| +67271|AAAAAAAAIMGABAAA|67271|18|41|11|PM|second|evening|dinner| +67272|AAAAAAAAJMGABAAA|67272|18|41|12|PM|second|evening|dinner| +67273|AAAAAAAAKMGABAAA|67273|18|41|13|PM|second|evening|dinner| +67274|AAAAAAAALMGABAAA|67274|18|41|14|PM|second|evening|dinner| +67275|AAAAAAAAMMGABAAA|67275|18|41|15|PM|second|evening|dinner| +67276|AAAAAAAANMGABAAA|67276|18|41|16|PM|second|evening|dinner| +67277|AAAAAAAAOMGABAAA|67277|18|41|17|PM|second|evening|dinner| +67278|AAAAAAAAPMGABAAA|67278|18|41|18|PM|second|evening|dinner| +67279|AAAAAAAAANGABAAA|67279|18|41|19|PM|second|evening|dinner| +67280|AAAAAAAABNGABAAA|67280|18|41|20|PM|second|evening|dinner| +67281|AAAAAAAACNGABAAA|67281|18|41|21|PM|second|evening|dinner| +67282|AAAAAAAADNGABAAA|67282|18|41|22|PM|second|evening|dinner| +67283|AAAAAAAAENGABAAA|67283|18|41|23|PM|second|evening|dinner| +67284|AAAAAAAAFNGABAAA|67284|18|41|24|PM|second|evening|dinner| +67285|AAAAAAAAGNGABAAA|67285|18|41|25|PM|second|evening|dinner| +67286|AAAAAAAAHNGABAAA|67286|18|41|26|PM|second|evening|dinner| +67287|AAAAAAAAINGABAAA|67287|18|41|27|PM|second|evening|dinner| +67288|AAAAAAAAJNGABAAA|67288|18|41|28|PM|second|evening|dinner| +67289|AAAAAAAAKNGABAAA|67289|18|41|29|PM|second|evening|dinner| +67290|AAAAAAAALNGABAAA|67290|18|41|30|PM|second|evening|dinner| +67291|AAAAAAAAMNGABAAA|67291|18|41|31|PM|second|evening|dinner| +67292|AAAAAAAANNGABAAA|67292|18|41|32|PM|second|evening|dinner| +67293|AAAAAAAAONGABAAA|67293|18|41|33|PM|second|evening|dinner| +67294|AAAAAAAAPNGABAAA|67294|18|41|34|PM|second|evening|dinner| +67295|AAAAAAAAAOGABAAA|67295|18|41|35|PM|second|evening|dinner| +67296|AAAAAAAABOGABAAA|67296|18|41|36|PM|second|evening|dinner| +67297|AAAAAAAACOGABAAA|67297|18|41|37|PM|second|evening|dinner| +67298|AAAAAAAADOGABAAA|67298|18|41|38|PM|second|evening|dinner| +67299|AAAAAAAAEOGABAAA|67299|18|41|39|PM|second|evening|dinner| +67300|AAAAAAAAFOGABAAA|67300|18|41|40|PM|second|evening|dinner| +67301|AAAAAAAAGOGABAAA|67301|18|41|41|PM|second|evening|dinner| +67302|AAAAAAAAHOGABAAA|67302|18|41|42|PM|second|evening|dinner| +67303|AAAAAAAAIOGABAAA|67303|18|41|43|PM|second|evening|dinner| +67304|AAAAAAAAJOGABAAA|67304|18|41|44|PM|second|evening|dinner| +67305|AAAAAAAAKOGABAAA|67305|18|41|45|PM|second|evening|dinner| +67306|AAAAAAAALOGABAAA|67306|18|41|46|PM|second|evening|dinner| +67307|AAAAAAAAMOGABAAA|67307|18|41|47|PM|second|evening|dinner| +67308|AAAAAAAANOGABAAA|67308|18|41|48|PM|second|evening|dinner| +67309|AAAAAAAAOOGABAAA|67309|18|41|49|PM|second|evening|dinner| +67310|AAAAAAAAPOGABAAA|67310|18|41|50|PM|second|evening|dinner| +67311|AAAAAAAAAPGABAAA|67311|18|41|51|PM|second|evening|dinner| +67312|AAAAAAAABPGABAAA|67312|18|41|52|PM|second|evening|dinner| +67313|AAAAAAAACPGABAAA|67313|18|41|53|PM|second|evening|dinner| +67314|AAAAAAAADPGABAAA|67314|18|41|54|PM|second|evening|dinner| +67315|AAAAAAAAEPGABAAA|67315|18|41|55|PM|second|evening|dinner| +67316|AAAAAAAAFPGABAAA|67316|18|41|56|PM|second|evening|dinner| +67317|AAAAAAAAGPGABAAA|67317|18|41|57|PM|second|evening|dinner| +67318|AAAAAAAAHPGABAAA|67318|18|41|58|PM|second|evening|dinner| +67319|AAAAAAAAIPGABAAA|67319|18|41|59|PM|second|evening|dinner| +67320|AAAAAAAAJPGABAAA|67320|18|42|0|PM|second|evening|dinner| +67321|AAAAAAAAKPGABAAA|67321|18|42|1|PM|second|evening|dinner| +67322|AAAAAAAALPGABAAA|67322|18|42|2|PM|second|evening|dinner| +67323|AAAAAAAAMPGABAAA|67323|18|42|3|PM|second|evening|dinner| +67324|AAAAAAAANPGABAAA|67324|18|42|4|PM|second|evening|dinner| +67325|AAAAAAAAOPGABAAA|67325|18|42|5|PM|second|evening|dinner| +67326|AAAAAAAAPPGABAAA|67326|18|42|6|PM|second|evening|dinner| +67327|AAAAAAAAAAHABAAA|67327|18|42|7|PM|second|evening|dinner| +67328|AAAAAAAABAHABAAA|67328|18|42|8|PM|second|evening|dinner| +67329|AAAAAAAACAHABAAA|67329|18|42|9|PM|second|evening|dinner| +67330|AAAAAAAADAHABAAA|67330|18|42|10|PM|second|evening|dinner| +67331|AAAAAAAAEAHABAAA|67331|18|42|11|PM|second|evening|dinner| +67332|AAAAAAAAFAHABAAA|67332|18|42|12|PM|second|evening|dinner| +67333|AAAAAAAAGAHABAAA|67333|18|42|13|PM|second|evening|dinner| +67334|AAAAAAAAHAHABAAA|67334|18|42|14|PM|second|evening|dinner| +67335|AAAAAAAAIAHABAAA|67335|18|42|15|PM|second|evening|dinner| +67336|AAAAAAAAJAHABAAA|67336|18|42|16|PM|second|evening|dinner| +67337|AAAAAAAAKAHABAAA|67337|18|42|17|PM|second|evening|dinner| +67338|AAAAAAAALAHABAAA|67338|18|42|18|PM|second|evening|dinner| +67339|AAAAAAAAMAHABAAA|67339|18|42|19|PM|second|evening|dinner| +67340|AAAAAAAANAHABAAA|67340|18|42|20|PM|second|evening|dinner| +67341|AAAAAAAAOAHABAAA|67341|18|42|21|PM|second|evening|dinner| +67342|AAAAAAAAPAHABAAA|67342|18|42|22|PM|second|evening|dinner| +67343|AAAAAAAAABHABAAA|67343|18|42|23|PM|second|evening|dinner| +67344|AAAAAAAABBHABAAA|67344|18|42|24|PM|second|evening|dinner| +67345|AAAAAAAACBHABAAA|67345|18|42|25|PM|second|evening|dinner| +67346|AAAAAAAADBHABAAA|67346|18|42|26|PM|second|evening|dinner| +67347|AAAAAAAAEBHABAAA|67347|18|42|27|PM|second|evening|dinner| +67348|AAAAAAAAFBHABAAA|67348|18|42|28|PM|second|evening|dinner| +67349|AAAAAAAAGBHABAAA|67349|18|42|29|PM|second|evening|dinner| +67350|AAAAAAAAHBHABAAA|67350|18|42|30|PM|second|evening|dinner| +67351|AAAAAAAAIBHABAAA|67351|18|42|31|PM|second|evening|dinner| +67352|AAAAAAAAJBHABAAA|67352|18|42|32|PM|second|evening|dinner| +67353|AAAAAAAAKBHABAAA|67353|18|42|33|PM|second|evening|dinner| +67354|AAAAAAAALBHABAAA|67354|18|42|34|PM|second|evening|dinner| +67355|AAAAAAAAMBHABAAA|67355|18|42|35|PM|second|evening|dinner| +67356|AAAAAAAANBHABAAA|67356|18|42|36|PM|second|evening|dinner| +67357|AAAAAAAAOBHABAAA|67357|18|42|37|PM|second|evening|dinner| +67358|AAAAAAAAPBHABAAA|67358|18|42|38|PM|second|evening|dinner| +67359|AAAAAAAAACHABAAA|67359|18|42|39|PM|second|evening|dinner| +67360|AAAAAAAABCHABAAA|67360|18|42|40|PM|second|evening|dinner| +67361|AAAAAAAACCHABAAA|67361|18|42|41|PM|second|evening|dinner| +67362|AAAAAAAADCHABAAA|67362|18|42|42|PM|second|evening|dinner| +67363|AAAAAAAAECHABAAA|67363|18|42|43|PM|second|evening|dinner| +67364|AAAAAAAAFCHABAAA|67364|18|42|44|PM|second|evening|dinner| +67365|AAAAAAAAGCHABAAA|67365|18|42|45|PM|second|evening|dinner| +67366|AAAAAAAAHCHABAAA|67366|18|42|46|PM|second|evening|dinner| +67367|AAAAAAAAICHABAAA|67367|18|42|47|PM|second|evening|dinner| +67368|AAAAAAAAJCHABAAA|67368|18|42|48|PM|second|evening|dinner| +67369|AAAAAAAAKCHABAAA|67369|18|42|49|PM|second|evening|dinner| +67370|AAAAAAAALCHABAAA|67370|18|42|50|PM|second|evening|dinner| +67371|AAAAAAAAMCHABAAA|67371|18|42|51|PM|second|evening|dinner| +67372|AAAAAAAANCHABAAA|67372|18|42|52|PM|second|evening|dinner| +67373|AAAAAAAAOCHABAAA|67373|18|42|53|PM|second|evening|dinner| +67374|AAAAAAAAPCHABAAA|67374|18|42|54|PM|second|evening|dinner| +67375|AAAAAAAAADHABAAA|67375|18|42|55|PM|second|evening|dinner| +67376|AAAAAAAABDHABAAA|67376|18|42|56|PM|second|evening|dinner| +67377|AAAAAAAACDHABAAA|67377|18|42|57|PM|second|evening|dinner| +67378|AAAAAAAADDHABAAA|67378|18|42|58|PM|second|evening|dinner| +67379|AAAAAAAAEDHABAAA|67379|18|42|59|PM|second|evening|dinner| +67380|AAAAAAAAFDHABAAA|67380|18|43|0|PM|second|evening|dinner| +67381|AAAAAAAAGDHABAAA|67381|18|43|1|PM|second|evening|dinner| +67382|AAAAAAAAHDHABAAA|67382|18|43|2|PM|second|evening|dinner| +67383|AAAAAAAAIDHABAAA|67383|18|43|3|PM|second|evening|dinner| +67384|AAAAAAAAJDHABAAA|67384|18|43|4|PM|second|evening|dinner| +67385|AAAAAAAAKDHABAAA|67385|18|43|5|PM|second|evening|dinner| +67386|AAAAAAAALDHABAAA|67386|18|43|6|PM|second|evening|dinner| +67387|AAAAAAAAMDHABAAA|67387|18|43|7|PM|second|evening|dinner| +67388|AAAAAAAANDHABAAA|67388|18|43|8|PM|second|evening|dinner| +67389|AAAAAAAAODHABAAA|67389|18|43|9|PM|second|evening|dinner| +67390|AAAAAAAAPDHABAAA|67390|18|43|10|PM|second|evening|dinner| +67391|AAAAAAAAAEHABAAA|67391|18|43|11|PM|second|evening|dinner| +67392|AAAAAAAABEHABAAA|67392|18|43|12|PM|second|evening|dinner| +67393|AAAAAAAACEHABAAA|67393|18|43|13|PM|second|evening|dinner| +67394|AAAAAAAADEHABAAA|67394|18|43|14|PM|second|evening|dinner| +67395|AAAAAAAAEEHABAAA|67395|18|43|15|PM|second|evening|dinner| +67396|AAAAAAAAFEHABAAA|67396|18|43|16|PM|second|evening|dinner| +67397|AAAAAAAAGEHABAAA|67397|18|43|17|PM|second|evening|dinner| +67398|AAAAAAAAHEHABAAA|67398|18|43|18|PM|second|evening|dinner| +67399|AAAAAAAAIEHABAAA|67399|18|43|19|PM|second|evening|dinner| +67400|AAAAAAAAJEHABAAA|67400|18|43|20|PM|second|evening|dinner| +67401|AAAAAAAAKEHABAAA|67401|18|43|21|PM|second|evening|dinner| +67402|AAAAAAAALEHABAAA|67402|18|43|22|PM|second|evening|dinner| +67403|AAAAAAAAMEHABAAA|67403|18|43|23|PM|second|evening|dinner| +67404|AAAAAAAANEHABAAA|67404|18|43|24|PM|second|evening|dinner| +67405|AAAAAAAAOEHABAAA|67405|18|43|25|PM|second|evening|dinner| +67406|AAAAAAAAPEHABAAA|67406|18|43|26|PM|second|evening|dinner| +67407|AAAAAAAAAFHABAAA|67407|18|43|27|PM|second|evening|dinner| +67408|AAAAAAAABFHABAAA|67408|18|43|28|PM|second|evening|dinner| +67409|AAAAAAAACFHABAAA|67409|18|43|29|PM|second|evening|dinner| +67410|AAAAAAAADFHABAAA|67410|18|43|30|PM|second|evening|dinner| +67411|AAAAAAAAEFHABAAA|67411|18|43|31|PM|second|evening|dinner| +67412|AAAAAAAAFFHABAAA|67412|18|43|32|PM|second|evening|dinner| +67413|AAAAAAAAGFHABAAA|67413|18|43|33|PM|second|evening|dinner| +67414|AAAAAAAAHFHABAAA|67414|18|43|34|PM|second|evening|dinner| +67415|AAAAAAAAIFHABAAA|67415|18|43|35|PM|second|evening|dinner| +67416|AAAAAAAAJFHABAAA|67416|18|43|36|PM|second|evening|dinner| +67417|AAAAAAAAKFHABAAA|67417|18|43|37|PM|second|evening|dinner| +67418|AAAAAAAALFHABAAA|67418|18|43|38|PM|second|evening|dinner| +67419|AAAAAAAAMFHABAAA|67419|18|43|39|PM|second|evening|dinner| +67420|AAAAAAAANFHABAAA|67420|18|43|40|PM|second|evening|dinner| +67421|AAAAAAAAOFHABAAA|67421|18|43|41|PM|second|evening|dinner| +67422|AAAAAAAAPFHABAAA|67422|18|43|42|PM|second|evening|dinner| +67423|AAAAAAAAAGHABAAA|67423|18|43|43|PM|second|evening|dinner| +67424|AAAAAAAABGHABAAA|67424|18|43|44|PM|second|evening|dinner| +67425|AAAAAAAACGHABAAA|67425|18|43|45|PM|second|evening|dinner| +67426|AAAAAAAADGHABAAA|67426|18|43|46|PM|second|evening|dinner| +67427|AAAAAAAAEGHABAAA|67427|18|43|47|PM|second|evening|dinner| +67428|AAAAAAAAFGHABAAA|67428|18|43|48|PM|second|evening|dinner| +67429|AAAAAAAAGGHABAAA|67429|18|43|49|PM|second|evening|dinner| +67430|AAAAAAAAHGHABAAA|67430|18|43|50|PM|second|evening|dinner| +67431|AAAAAAAAIGHABAAA|67431|18|43|51|PM|second|evening|dinner| +67432|AAAAAAAAJGHABAAA|67432|18|43|52|PM|second|evening|dinner| +67433|AAAAAAAAKGHABAAA|67433|18|43|53|PM|second|evening|dinner| +67434|AAAAAAAALGHABAAA|67434|18|43|54|PM|second|evening|dinner| +67435|AAAAAAAAMGHABAAA|67435|18|43|55|PM|second|evening|dinner| +67436|AAAAAAAANGHABAAA|67436|18|43|56|PM|second|evening|dinner| +67437|AAAAAAAAOGHABAAA|67437|18|43|57|PM|second|evening|dinner| +67438|AAAAAAAAPGHABAAA|67438|18|43|58|PM|second|evening|dinner| +67439|AAAAAAAAAHHABAAA|67439|18|43|59|PM|second|evening|dinner| +67440|AAAAAAAABHHABAAA|67440|18|44|0|PM|second|evening|dinner| +67441|AAAAAAAACHHABAAA|67441|18|44|1|PM|second|evening|dinner| +67442|AAAAAAAADHHABAAA|67442|18|44|2|PM|second|evening|dinner| +67443|AAAAAAAAEHHABAAA|67443|18|44|3|PM|second|evening|dinner| +67444|AAAAAAAAFHHABAAA|67444|18|44|4|PM|second|evening|dinner| +67445|AAAAAAAAGHHABAAA|67445|18|44|5|PM|second|evening|dinner| +67446|AAAAAAAAHHHABAAA|67446|18|44|6|PM|second|evening|dinner| +67447|AAAAAAAAIHHABAAA|67447|18|44|7|PM|second|evening|dinner| +67448|AAAAAAAAJHHABAAA|67448|18|44|8|PM|second|evening|dinner| +67449|AAAAAAAAKHHABAAA|67449|18|44|9|PM|second|evening|dinner| +67450|AAAAAAAALHHABAAA|67450|18|44|10|PM|second|evening|dinner| +67451|AAAAAAAAMHHABAAA|67451|18|44|11|PM|second|evening|dinner| +67452|AAAAAAAANHHABAAA|67452|18|44|12|PM|second|evening|dinner| +67453|AAAAAAAAOHHABAAA|67453|18|44|13|PM|second|evening|dinner| +67454|AAAAAAAAPHHABAAA|67454|18|44|14|PM|second|evening|dinner| +67455|AAAAAAAAAIHABAAA|67455|18|44|15|PM|second|evening|dinner| +67456|AAAAAAAABIHABAAA|67456|18|44|16|PM|second|evening|dinner| +67457|AAAAAAAACIHABAAA|67457|18|44|17|PM|second|evening|dinner| +67458|AAAAAAAADIHABAAA|67458|18|44|18|PM|second|evening|dinner| +67459|AAAAAAAAEIHABAAA|67459|18|44|19|PM|second|evening|dinner| +67460|AAAAAAAAFIHABAAA|67460|18|44|20|PM|second|evening|dinner| +67461|AAAAAAAAGIHABAAA|67461|18|44|21|PM|second|evening|dinner| +67462|AAAAAAAAHIHABAAA|67462|18|44|22|PM|second|evening|dinner| +67463|AAAAAAAAIIHABAAA|67463|18|44|23|PM|second|evening|dinner| +67464|AAAAAAAAJIHABAAA|67464|18|44|24|PM|second|evening|dinner| +67465|AAAAAAAAKIHABAAA|67465|18|44|25|PM|second|evening|dinner| +67466|AAAAAAAALIHABAAA|67466|18|44|26|PM|second|evening|dinner| +67467|AAAAAAAAMIHABAAA|67467|18|44|27|PM|second|evening|dinner| +67468|AAAAAAAANIHABAAA|67468|18|44|28|PM|second|evening|dinner| +67469|AAAAAAAAOIHABAAA|67469|18|44|29|PM|second|evening|dinner| +67470|AAAAAAAAPIHABAAA|67470|18|44|30|PM|second|evening|dinner| +67471|AAAAAAAAAJHABAAA|67471|18|44|31|PM|second|evening|dinner| +67472|AAAAAAAABJHABAAA|67472|18|44|32|PM|second|evening|dinner| +67473|AAAAAAAACJHABAAA|67473|18|44|33|PM|second|evening|dinner| +67474|AAAAAAAADJHABAAA|67474|18|44|34|PM|second|evening|dinner| +67475|AAAAAAAAEJHABAAA|67475|18|44|35|PM|second|evening|dinner| +67476|AAAAAAAAFJHABAAA|67476|18|44|36|PM|second|evening|dinner| +67477|AAAAAAAAGJHABAAA|67477|18|44|37|PM|second|evening|dinner| +67478|AAAAAAAAHJHABAAA|67478|18|44|38|PM|second|evening|dinner| +67479|AAAAAAAAIJHABAAA|67479|18|44|39|PM|second|evening|dinner| +67480|AAAAAAAAJJHABAAA|67480|18|44|40|PM|second|evening|dinner| +67481|AAAAAAAAKJHABAAA|67481|18|44|41|PM|second|evening|dinner| +67482|AAAAAAAALJHABAAA|67482|18|44|42|PM|second|evening|dinner| +67483|AAAAAAAAMJHABAAA|67483|18|44|43|PM|second|evening|dinner| +67484|AAAAAAAANJHABAAA|67484|18|44|44|PM|second|evening|dinner| +67485|AAAAAAAAOJHABAAA|67485|18|44|45|PM|second|evening|dinner| +67486|AAAAAAAAPJHABAAA|67486|18|44|46|PM|second|evening|dinner| +67487|AAAAAAAAAKHABAAA|67487|18|44|47|PM|second|evening|dinner| +67488|AAAAAAAABKHABAAA|67488|18|44|48|PM|second|evening|dinner| +67489|AAAAAAAACKHABAAA|67489|18|44|49|PM|second|evening|dinner| +67490|AAAAAAAADKHABAAA|67490|18|44|50|PM|second|evening|dinner| +67491|AAAAAAAAEKHABAAA|67491|18|44|51|PM|second|evening|dinner| +67492|AAAAAAAAFKHABAAA|67492|18|44|52|PM|second|evening|dinner| +67493|AAAAAAAAGKHABAAA|67493|18|44|53|PM|second|evening|dinner| +67494|AAAAAAAAHKHABAAA|67494|18|44|54|PM|second|evening|dinner| +67495|AAAAAAAAIKHABAAA|67495|18|44|55|PM|second|evening|dinner| +67496|AAAAAAAAJKHABAAA|67496|18|44|56|PM|second|evening|dinner| +67497|AAAAAAAAKKHABAAA|67497|18|44|57|PM|second|evening|dinner| +67498|AAAAAAAALKHABAAA|67498|18|44|58|PM|second|evening|dinner| +67499|AAAAAAAAMKHABAAA|67499|18|44|59|PM|second|evening|dinner| +67500|AAAAAAAANKHABAAA|67500|18|45|0|PM|second|evening|dinner| +67501|AAAAAAAAOKHABAAA|67501|18|45|1|PM|second|evening|dinner| +67502|AAAAAAAAPKHABAAA|67502|18|45|2|PM|second|evening|dinner| +67503|AAAAAAAAALHABAAA|67503|18|45|3|PM|second|evening|dinner| +67504|AAAAAAAABLHABAAA|67504|18|45|4|PM|second|evening|dinner| +67505|AAAAAAAACLHABAAA|67505|18|45|5|PM|second|evening|dinner| +67506|AAAAAAAADLHABAAA|67506|18|45|6|PM|second|evening|dinner| +67507|AAAAAAAAELHABAAA|67507|18|45|7|PM|second|evening|dinner| +67508|AAAAAAAAFLHABAAA|67508|18|45|8|PM|second|evening|dinner| +67509|AAAAAAAAGLHABAAA|67509|18|45|9|PM|second|evening|dinner| +67510|AAAAAAAAHLHABAAA|67510|18|45|10|PM|second|evening|dinner| +67511|AAAAAAAAILHABAAA|67511|18|45|11|PM|second|evening|dinner| +67512|AAAAAAAAJLHABAAA|67512|18|45|12|PM|second|evening|dinner| +67513|AAAAAAAAKLHABAAA|67513|18|45|13|PM|second|evening|dinner| +67514|AAAAAAAALLHABAAA|67514|18|45|14|PM|second|evening|dinner| +67515|AAAAAAAAMLHABAAA|67515|18|45|15|PM|second|evening|dinner| +67516|AAAAAAAANLHABAAA|67516|18|45|16|PM|second|evening|dinner| +67517|AAAAAAAAOLHABAAA|67517|18|45|17|PM|second|evening|dinner| +67518|AAAAAAAAPLHABAAA|67518|18|45|18|PM|second|evening|dinner| +67519|AAAAAAAAAMHABAAA|67519|18|45|19|PM|second|evening|dinner| +67520|AAAAAAAABMHABAAA|67520|18|45|20|PM|second|evening|dinner| +67521|AAAAAAAACMHABAAA|67521|18|45|21|PM|second|evening|dinner| +67522|AAAAAAAADMHABAAA|67522|18|45|22|PM|second|evening|dinner| +67523|AAAAAAAAEMHABAAA|67523|18|45|23|PM|second|evening|dinner| +67524|AAAAAAAAFMHABAAA|67524|18|45|24|PM|second|evening|dinner| +67525|AAAAAAAAGMHABAAA|67525|18|45|25|PM|second|evening|dinner| +67526|AAAAAAAAHMHABAAA|67526|18|45|26|PM|second|evening|dinner| +67527|AAAAAAAAIMHABAAA|67527|18|45|27|PM|second|evening|dinner| +67528|AAAAAAAAJMHABAAA|67528|18|45|28|PM|second|evening|dinner| +67529|AAAAAAAAKMHABAAA|67529|18|45|29|PM|second|evening|dinner| +67530|AAAAAAAALMHABAAA|67530|18|45|30|PM|second|evening|dinner| +67531|AAAAAAAAMMHABAAA|67531|18|45|31|PM|second|evening|dinner| +67532|AAAAAAAANMHABAAA|67532|18|45|32|PM|second|evening|dinner| +67533|AAAAAAAAOMHABAAA|67533|18|45|33|PM|second|evening|dinner| +67534|AAAAAAAAPMHABAAA|67534|18|45|34|PM|second|evening|dinner| +67535|AAAAAAAAANHABAAA|67535|18|45|35|PM|second|evening|dinner| +67536|AAAAAAAABNHABAAA|67536|18|45|36|PM|second|evening|dinner| +67537|AAAAAAAACNHABAAA|67537|18|45|37|PM|second|evening|dinner| +67538|AAAAAAAADNHABAAA|67538|18|45|38|PM|second|evening|dinner| +67539|AAAAAAAAENHABAAA|67539|18|45|39|PM|second|evening|dinner| +67540|AAAAAAAAFNHABAAA|67540|18|45|40|PM|second|evening|dinner| +67541|AAAAAAAAGNHABAAA|67541|18|45|41|PM|second|evening|dinner| +67542|AAAAAAAAHNHABAAA|67542|18|45|42|PM|second|evening|dinner| +67543|AAAAAAAAINHABAAA|67543|18|45|43|PM|second|evening|dinner| +67544|AAAAAAAAJNHABAAA|67544|18|45|44|PM|second|evening|dinner| +67545|AAAAAAAAKNHABAAA|67545|18|45|45|PM|second|evening|dinner| +67546|AAAAAAAALNHABAAA|67546|18|45|46|PM|second|evening|dinner| +67547|AAAAAAAAMNHABAAA|67547|18|45|47|PM|second|evening|dinner| +67548|AAAAAAAANNHABAAA|67548|18|45|48|PM|second|evening|dinner| +67549|AAAAAAAAONHABAAA|67549|18|45|49|PM|second|evening|dinner| +67550|AAAAAAAAPNHABAAA|67550|18|45|50|PM|second|evening|dinner| +67551|AAAAAAAAAOHABAAA|67551|18|45|51|PM|second|evening|dinner| +67552|AAAAAAAABOHABAAA|67552|18|45|52|PM|second|evening|dinner| +67553|AAAAAAAACOHABAAA|67553|18|45|53|PM|second|evening|dinner| +67554|AAAAAAAADOHABAAA|67554|18|45|54|PM|second|evening|dinner| +67555|AAAAAAAAEOHABAAA|67555|18|45|55|PM|second|evening|dinner| +67556|AAAAAAAAFOHABAAA|67556|18|45|56|PM|second|evening|dinner| +67557|AAAAAAAAGOHABAAA|67557|18|45|57|PM|second|evening|dinner| +67558|AAAAAAAAHOHABAAA|67558|18|45|58|PM|second|evening|dinner| +67559|AAAAAAAAIOHABAAA|67559|18|45|59|PM|second|evening|dinner| +67560|AAAAAAAAJOHABAAA|67560|18|46|0|PM|second|evening|dinner| +67561|AAAAAAAAKOHABAAA|67561|18|46|1|PM|second|evening|dinner| +67562|AAAAAAAALOHABAAA|67562|18|46|2|PM|second|evening|dinner| +67563|AAAAAAAAMOHABAAA|67563|18|46|3|PM|second|evening|dinner| +67564|AAAAAAAANOHABAAA|67564|18|46|4|PM|second|evening|dinner| +67565|AAAAAAAAOOHABAAA|67565|18|46|5|PM|second|evening|dinner| +67566|AAAAAAAAPOHABAAA|67566|18|46|6|PM|second|evening|dinner| +67567|AAAAAAAAAPHABAAA|67567|18|46|7|PM|second|evening|dinner| +67568|AAAAAAAABPHABAAA|67568|18|46|8|PM|second|evening|dinner| +67569|AAAAAAAACPHABAAA|67569|18|46|9|PM|second|evening|dinner| +67570|AAAAAAAADPHABAAA|67570|18|46|10|PM|second|evening|dinner| +67571|AAAAAAAAEPHABAAA|67571|18|46|11|PM|second|evening|dinner| +67572|AAAAAAAAFPHABAAA|67572|18|46|12|PM|second|evening|dinner| +67573|AAAAAAAAGPHABAAA|67573|18|46|13|PM|second|evening|dinner| +67574|AAAAAAAAHPHABAAA|67574|18|46|14|PM|second|evening|dinner| +67575|AAAAAAAAIPHABAAA|67575|18|46|15|PM|second|evening|dinner| +67576|AAAAAAAAJPHABAAA|67576|18|46|16|PM|second|evening|dinner| +67577|AAAAAAAAKPHABAAA|67577|18|46|17|PM|second|evening|dinner| +67578|AAAAAAAALPHABAAA|67578|18|46|18|PM|second|evening|dinner| +67579|AAAAAAAAMPHABAAA|67579|18|46|19|PM|second|evening|dinner| +67580|AAAAAAAANPHABAAA|67580|18|46|20|PM|second|evening|dinner| +67581|AAAAAAAAOPHABAAA|67581|18|46|21|PM|second|evening|dinner| +67582|AAAAAAAAPPHABAAA|67582|18|46|22|PM|second|evening|dinner| +67583|AAAAAAAAAAIABAAA|67583|18|46|23|PM|second|evening|dinner| +67584|AAAAAAAABAIABAAA|67584|18|46|24|PM|second|evening|dinner| +67585|AAAAAAAACAIABAAA|67585|18|46|25|PM|second|evening|dinner| +67586|AAAAAAAADAIABAAA|67586|18|46|26|PM|second|evening|dinner| +67587|AAAAAAAAEAIABAAA|67587|18|46|27|PM|second|evening|dinner| +67588|AAAAAAAAFAIABAAA|67588|18|46|28|PM|second|evening|dinner| +67589|AAAAAAAAGAIABAAA|67589|18|46|29|PM|second|evening|dinner| +67590|AAAAAAAAHAIABAAA|67590|18|46|30|PM|second|evening|dinner| +67591|AAAAAAAAIAIABAAA|67591|18|46|31|PM|second|evening|dinner| +67592|AAAAAAAAJAIABAAA|67592|18|46|32|PM|second|evening|dinner| +67593|AAAAAAAAKAIABAAA|67593|18|46|33|PM|second|evening|dinner| +67594|AAAAAAAALAIABAAA|67594|18|46|34|PM|second|evening|dinner| +67595|AAAAAAAAMAIABAAA|67595|18|46|35|PM|second|evening|dinner| +67596|AAAAAAAANAIABAAA|67596|18|46|36|PM|second|evening|dinner| +67597|AAAAAAAAOAIABAAA|67597|18|46|37|PM|second|evening|dinner| +67598|AAAAAAAAPAIABAAA|67598|18|46|38|PM|second|evening|dinner| +67599|AAAAAAAAABIABAAA|67599|18|46|39|PM|second|evening|dinner| +67600|AAAAAAAABBIABAAA|67600|18|46|40|PM|second|evening|dinner| +67601|AAAAAAAACBIABAAA|67601|18|46|41|PM|second|evening|dinner| +67602|AAAAAAAADBIABAAA|67602|18|46|42|PM|second|evening|dinner| +67603|AAAAAAAAEBIABAAA|67603|18|46|43|PM|second|evening|dinner| +67604|AAAAAAAAFBIABAAA|67604|18|46|44|PM|second|evening|dinner| +67605|AAAAAAAAGBIABAAA|67605|18|46|45|PM|second|evening|dinner| +67606|AAAAAAAAHBIABAAA|67606|18|46|46|PM|second|evening|dinner| +67607|AAAAAAAAIBIABAAA|67607|18|46|47|PM|second|evening|dinner| +67608|AAAAAAAAJBIABAAA|67608|18|46|48|PM|second|evening|dinner| +67609|AAAAAAAAKBIABAAA|67609|18|46|49|PM|second|evening|dinner| +67610|AAAAAAAALBIABAAA|67610|18|46|50|PM|second|evening|dinner| +67611|AAAAAAAAMBIABAAA|67611|18|46|51|PM|second|evening|dinner| +67612|AAAAAAAANBIABAAA|67612|18|46|52|PM|second|evening|dinner| +67613|AAAAAAAAOBIABAAA|67613|18|46|53|PM|second|evening|dinner| +67614|AAAAAAAAPBIABAAA|67614|18|46|54|PM|second|evening|dinner| +67615|AAAAAAAAACIABAAA|67615|18|46|55|PM|second|evening|dinner| +67616|AAAAAAAABCIABAAA|67616|18|46|56|PM|second|evening|dinner| +67617|AAAAAAAACCIABAAA|67617|18|46|57|PM|second|evening|dinner| +67618|AAAAAAAADCIABAAA|67618|18|46|58|PM|second|evening|dinner| +67619|AAAAAAAAECIABAAA|67619|18|46|59|PM|second|evening|dinner| +67620|AAAAAAAAFCIABAAA|67620|18|47|0|PM|second|evening|dinner| +67621|AAAAAAAAGCIABAAA|67621|18|47|1|PM|second|evening|dinner| +67622|AAAAAAAAHCIABAAA|67622|18|47|2|PM|second|evening|dinner| +67623|AAAAAAAAICIABAAA|67623|18|47|3|PM|second|evening|dinner| +67624|AAAAAAAAJCIABAAA|67624|18|47|4|PM|second|evening|dinner| +67625|AAAAAAAAKCIABAAA|67625|18|47|5|PM|second|evening|dinner| +67626|AAAAAAAALCIABAAA|67626|18|47|6|PM|second|evening|dinner| +67627|AAAAAAAAMCIABAAA|67627|18|47|7|PM|second|evening|dinner| +67628|AAAAAAAANCIABAAA|67628|18|47|8|PM|second|evening|dinner| +67629|AAAAAAAAOCIABAAA|67629|18|47|9|PM|second|evening|dinner| +67630|AAAAAAAAPCIABAAA|67630|18|47|10|PM|second|evening|dinner| +67631|AAAAAAAAADIABAAA|67631|18|47|11|PM|second|evening|dinner| +67632|AAAAAAAABDIABAAA|67632|18|47|12|PM|second|evening|dinner| +67633|AAAAAAAACDIABAAA|67633|18|47|13|PM|second|evening|dinner| +67634|AAAAAAAADDIABAAA|67634|18|47|14|PM|second|evening|dinner| +67635|AAAAAAAAEDIABAAA|67635|18|47|15|PM|second|evening|dinner| +67636|AAAAAAAAFDIABAAA|67636|18|47|16|PM|second|evening|dinner| +67637|AAAAAAAAGDIABAAA|67637|18|47|17|PM|second|evening|dinner| +67638|AAAAAAAAHDIABAAA|67638|18|47|18|PM|second|evening|dinner| +67639|AAAAAAAAIDIABAAA|67639|18|47|19|PM|second|evening|dinner| +67640|AAAAAAAAJDIABAAA|67640|18|47|20|PM|second|evening|dinner| +67641|AAAAAAAAKDIABAAA|67641|18|47|21|PM|second|evening|dinner| +67642|AAAAAAAALDIABAAA|67642|18|47|22|PM|second|evening|dinner| +67643|AAAAAAAAMDIABAAA|67643|18|47|23|PM|second|evening|dinner| +67644|AAAAAAAANDIABAAA|67644|18|47|24|PM|second|evening|dinner| +67645|AAAAAAAAODIABAAA|67645|18|47|25|PM|second|evening|dinner| +67646|AAAAAAAAPDIABAAA|67646|18|47|26|PM|second|evening|dinner| +67647|AAAAAAAAAEIABAAA|67647|18|47|27|PM|second|evening|dinner| +67648|AAAAAAAABEIABAAA|67648|18|47|28|PM|second|evening|dinner| +67649|AAAAAAAACEIABAAA|67649|18|47|29|PM|second|evening|dinner| +67650|AAAAAAAADEIABAAA|67650|18|47|30|PM|second|evening|dinner| +67651|AAAAAAAAEEIABAAA|67651|18|47|31|PM|second|evening|dinner| +67652|AAAAAAAAFEIABAAA|67652|18|47|32|PM|second|evening|dinner| +67653|AAAAAAAAGEIABAAA|67653|18|47|33|PM|second|evening|dinner| +67654|AAAAAAAAHEIABAAA|67654|18|47|34|PM|second|evening|dinner| +67655|AAAAAAAAIEIABAAA|67655|18|47|35|PM|second|evening|dinner| +67656|AAAAAAAAJEIABAAA|67656|18|47|36|PM|second|evening|dinner| +67657|AAAAAAAAKEIABAAA|67657|18|47|37|PM|second|evening|dinner| +67658|AAAAAAAALEIABAAA|67658|18|47|38|PM|second|evening|dinner| +67659|AAAAAAAAMEIABAAA|67659|18|47|39|PM|second|evening|dinner| +67660|AAAAAAAANEIABAAA|67660|18|47|40|PM|second|evening|dinner| +67661|AAAAAAAAOEIABAAA|67661|18|47|41|PM|second|evening|dinner| +67662|AAAAAAAAPEIABAAA|67662|18|47|42|PM|second|evening|dinner| +67663|AAAAAAAAAFIABAAA|67663|18|47|43|PM|second|evening|dinner| +67664|AAAAAAAABFIABAAA|67664|18|47|44|PM|second|evening|dinner| +67665|AAAAAAAACFIABAAA|67665|18|47|45|PM|second|evening|dinner| +67666|AAAAAAAADFIABAAA|67666|18|47|46|PM|second|evening|dinner| +67667|AAAAAAAAEFIABAAA|67667|18|47|47|PM|second|evening|dinner| +67668|AAAAAAAAFFIABAAA|67668|18|47|48|PM|second|evening|dinner| +67669|AAAAAAAAGFIABAAA|67669|18|47|49|PM|second|evening|dinner| +67670|AAAAAAAAHFIABAAA|67670|18|47|50|PM|second|evening|dinner| +67671|AAAAAAAAIFIABAAA|67671|18|47|51|PM|second|evening|dinner| +67672|AAAAAAAAJFIABAAA|67672|18|47|52|PM|second|evening|dinner| +67673|AAAAAAAAKFIABAAA|67673|18|47|53|PM|second|evening|dinner| +67674|AAAAAAAALFIABAAA|67674|18|47|54|PM|second|evening|dinner| +67675|AAAAAAAAMFIABAAA|67675|18|47|55|PM|second|evening|dinner| +67676|AAAAAAAANFIABAAA|67676|18|47|56|PM|second|evening|dinner| +67677|AAAAAAAAOFIABAAA|67677|18|47|57|PM|second|evening|dinner| +67678|AAAAAAAAPFIABAAA|67678|18|47|58|PM|second|evening|dinner| +67679|AAAAAAAAAGIABAAA|67679|18|47|59|PM|second|evening|dinner| +67680|AAAAAAAABGIABAAA|67680|18|48|0|PM|second|evening|dinner| +67681|AAAAAAAACGIABAAA|67681|18|48|1|PM|second|evening|dinner| +67682|AAAAAAAADGIABAAA|67682|18|48|2|PM|second|evening|dinner| +67683|AAAAAAAAEGIABAAA|67683|18|48|3|PM|second|evening|dinner| +67684|AAAAAAAAFGIABAAA|67684|18|48|4|PM|second|evening|dinner| +67685|AAAAAAAAGGIABAAA|67685|18|48|5|PM|second|evening|dinner| +67686|AAAAAAAAHGIABAAA|67686|18|48|6|PM|second|evening|dinner| +67687|AAAAAAAAIGIABAAA|67687|18|48|7|PM|second|evening|dinner| +67688|AAAAAAAAJGIABAAA|67688|18|48|8|PM|second|evening|dinner| +67689|AAAAAAAAKGIABAAA|67689|18|48|9|PM|second|evening|dinner| +67690|AAAAAAAALGIABAAA|67690|18|48|10|PM|second|evening|dinner| +67691|AAAAAAAAMGIABAAA|67691|18|48|11|PM|second|evening|dinner| +67692|AAAAAAAANGIABAAA|67692|18|48|12|PM|second|evening|dinner| +67693|AAAAAAAAOGIABAAA|67693|18|48|13|PM|second|evening|dinner| +67694|AAAAAAAAPGIABAAA|67694|18|48|14|PM|second|evening|dinner| +67695|AAAAAAAAAHIABAAA|67695|18|48|15|PM|second|evening|dinner| +67696|AAAAAAAABHIABAAA|67696|18|48|16|PM|second|evening|dinner| +67697|AAAAAAAACHIABAAA|67697|18|48|17|PM|second|evening|dinner| +67698|AAAAAAAADHIABAAA|67698|18|48|18|PM|second|evening|dinner| +67699|AAAAAAAAEHIABAAA|67699|18|48|19|PM|second|evening|dinner| +67700|AAAAAAAAFHIABAAA|67700|18|48|20|PM|second|evening|dinner| +67701|AAAAAAAAGHIABAAA|67701|18|48|21|PM|second|evening|dinner| +67702|AAAAAAAAHHIABAAA|67702|18|48|22|PM|second|evening|dinner| +67703|AAAAAAAAIHIABAAA|67703|18|48|23|PM|second|evening|dinner| +67704|AAAAAAAAJHIABAAA|67704|18|48|24|PM|second|evening|dinner| +67705|AAAAAAAAKHIABAAA|67705|18|48|25|PM|second|evening|dinner| +67706|AAAAAAAALHIABAAA|67706|18|48|26|PM|second|evening|dinner| +67707|AAAAAAAAMHIABAAA|67707|18|48|27|PM|second|evening|dinner| +67708|AAAAAAAANHIABAAA|67708|18|48|28|PM|second|evening|dinner| +67709|AAAAAAAAOHIABAAA|67709|18|48|29|PM|second|evening|dinner| +67710|AAAAAAAAPHIABAAA|67710|18|48|30|PM|second|evening|dinner| +67711|AAAAAAAAAIIABAAA|67711|18|48|31|PM|second|evening|dinner| +67712|AAAAAAAABIIABAAA|67712|18|48|32|PM|second|evening|dinner| +67713|AAAAAAAACIIABAAA|67713|18|48|33|PM|second|evening|dinner| +67714|AAAAAAAADIIABAAA|67714|18|48|34|PM|second|evening|dinner| +67715|AAAAAAAAEIIABAAA|67715|18|48|35|PM|second|evening|dinner| +67716|AAAAAAAAFIIABAAA|67716|18|48|36|PM|second|evening|dinner| +67717|AAAAAAAAGIIABAAA|67717|18|48|37|PM|second|evening|dinner| +67718|AAAAAAAAHIIABAAA|67718|18|48|38|PM|second|evening|dinner| +67719|AAAAAAAAIIIABAAA|67719|18|48|39|PM|second|evening|dinner| +67720|AAAAAAAAJIIABAAA|67720|18|48|40|PM|second|evening|dinner| +67721|AAAAAAAAKIIABAAA|67721|18|48|41|PM|second|evening|dinner| +67722|AAAAAAAALIIABAAA|67722|18|48|42|PM|second|evening|dinner| +67723|AAAAAAAAMIIABAAA|67723|18|48|43|PM|second|evening|dinner| +67724|AAAAAAAANIIABAAA|67724|18|48|44|PM|second|evening|dinner| +67725|AAAAAAAAOIIABAAA|67725|18|48|45|PM|second|evening|dinner| +67726|AAAAAAAAPIIABAAA|67726|18|48|46|PM|second|evening|dinner| +67727|AAAAAAAAAJIABAAA|67727|18|48|47|PM|second|evening|dinner| +67728|AAAAAAAABJIABAAA|67728|18|48|48|PM|second|evening|dinner| +67729|AAAAAAAACJIABAAA|67729|18|48|49|PM|second|evening|dinner| +67730|AAAAAAAADJIABAAA|67730|18|48|50|PM|second|evening|dinner| +67731|AAAAAAAAEJIABAAA|67731|18|48|51|PM|second|evening|dinner| +67732|AAAAAAAAFJIABAAA|67732|18|48|52|PM|second|evening|dinner| +67733|AAAAAAAAGJIABAAA|67733|18|48|53|PM|second|evening|dinner| +67734|AAAAAAAAHJIABAAA|67734|18|48|54|PM|second|evening|dinner| +67735|AAAAAAAAIJIABAAA|67735|18|48|55|PM|second|evening|dinner| +67736|AAAAAAAAJJIABAAA|67736|18|48|56|PM|second|evening|dinner| +67737|AAAAAAAAKJIABAAA|67737|18|48|57|PM|second|evening|dinner| +67738|AAAAAAAALJIABAAA|67738|18|48|58|PM|second|evening|dinner| +67739|AAAAAAAAMJIABAAA|67739|18|48|59|PM|second|evening|dinner| +67740|AAAAAAAANJIABAAA|67740|18|49|0|PM|second|evening|dinner| +67741|AAAAAAAAOJIABAAA|67741|18|49|1|PM|second|evening|dinner| +67742|AAAAAAAAPJIABAAA|67742|18|49|2|PM|second|evening|dinner| +67743|AAAAAAAAAKIABAAA|67743|18|49|3|PM|second|evening|dinner| +67744|AAAAAAAABKIABAAA|67744|18|49|4|PM|second|evening|dinner| +67745|AAAAAAAACKIABAAA|67745|18|49|5|PM|second|evening|dinner| +67746|AAAAAAAADKIABAAA|67746|18|49|6|PM|second|evening|dinner| +67747|AAAAAAAAEKIABAAA|67747|18|49|7|PM|second|evening|dinner| +67748|AAAAAAAAFKIABAAA|67748|18|49|8|PM|second|evening|dinner| +67749|AAAAAAAAGKIABAAA|67749|18|49|9|PM|second|evening|dinner| +67750|AAAAAAAAHKIABAAA|67750|18|49|10|PM|second|evening|dinner| +67751|AAAAAAAAIKIABAAA|67751|18|49|11|PM|second|evening|dinner| +67752|AAAAAAAAJKIABAAA|67752|18|49|12|PM|second|evening|dinner| +67753|AAAAAAAAKKIABAAA|67753|18|49|13|PM|second|evening|dinner| +67754|AAAAAAAALKIABAAA|67754|18|49|14|PM|second|evening|dinner| +67755|AAAAAAAAMKIABAAA|67755|18|49|15|PM|second|evening|dinner| +67756|AAAAAAAANKIABAAA|67756|18|49|16|PM|second|evening|dinner| +67757|AAAAAAAAOKIABAAA|67757|18|49|17|PM|second|evening|dinner| +67758|AAAAAAAAPKIABAAA|67758|18|49|18|PM|second|evening|dinner| +67759|AAAAAAAAALIABAAA|67759|18|49|19|PM|second|evening|dinner| +67760|AAAAAAAABLIABAAA|67760|18|49|20|PM|second|evening|dinner| +67761|AAAAAAAACLIABAAA|67761|18|49|21|PM|second|evening|dinner| +67762|AAAAAAAADLIABAAA|67762|18|49|22|PM|second|evening|dinner| +67763|AAAAAAAAELIABAAA|67763|18|49|23|PM|second|evening|dinner| +67764|AAAAAAAAFLIABAAA|67764|18|49|24|PM|second|evening|dinner| +67765|AAAAAAAAGLIABAAA|67765|18|49|25|PM|second|evening|dinner| +67766|AAAAAAAAHLIABAAA|67766|18|49|26|PM|second|evening|dinner| +67767|AAAAAAAAILIABAAA|67767|18|49|27|PM|second|evening|dinner| +67768|AAAAAAAAJLIABAAA|67768|18|49|28|PM|second|evening|dinner| +67769|AAAAAAAAKLIABAAA|67769|18|49|29|PM|second|evening|dinner| +67770|AAAAAAAALLIABAAA|67770|18|49|30|PM|second|evening|dinner| +67771|AAAAAAAAMLIABAAA|67771|18|49|31|PM|second|evening|dinner| +67772|AAAAAAAANLIABAAA|67772|18|49|32|PM|second|evening|dinner| +67773|AAAAAAAAOLIABAAA|67773|18|49|33|PM|second|evening|dinner| +67774|AAAAAAAAPLIABAAA|67774|18|49|34|PM|second|evening|dinner| +67775|AAAAAAAAAMIABAAA|67775|18|49|35|PM|second|evening|dinner| +67776|AAAAAAAABMIABAAA|67776|18|49|36|PM|second|evening|dinner| +67777|AAAAAAAACMIABAAA|67777|18|49|37|PM|second|evening|dinner| +67778|AAAAAAAADMIABAAA|67778|18|49|38|PM|second|evening|dinner| +67779|AAAAAAAAEMIABAAA|67779|18|49|39|PM|second|evening|dinner| +67780|AAAAAAAAFMIABAAA|67780|18|49|40|PM|second|evening|dinner| +67781|AAAAAAAAGMIABAAA|67781|18|49|41|PM|second|evening|dinner| +67782|AAAAAAAAHMIABAAA|67782|18|49|42|PM|second|evening|dinner| +67783|AAAAAAAAIMIABAAA|67783|18|49|43|PM|second|evening|dinner| +67784|AAAAAAAAJMIABAAA|67784|18|49|44|PM|second|evening|dinner| +67785|AAAAAAAAKMIABAAA|67785|18|49|45|PM|second|evening|dinner| +67786|AAAAAAAALMIABAAA|67786|18|49|46|PM|second|evening|dinner| +67787|AAAAAAAAMMIABAAA|67787|18|49|47|PM|second|evening|dinner| +67788|AAAAAAAANMIABAAA|67788|18|49|48|PM|second|evening|dinner| +67789|AAAAAAAAOMIABAAA|67789|18|49|49|PM|second|evening|dinner| +67790|AAAAAAAAPMIABAAA|67790|18|49|50|PM|second|evening|dinner| +67791|AAAAAAAAANIABAAA|67791|18|49|51|PM|second|evening|dinner| +67792|AAAAAAAABNIABAAA|67792|18|49|52|PM|second|evening|dinner| +67793|AAAAAAAACNIABAAA|67793|18|49|53|PM|second|evening|dinner| +67794|AAAAAAAADNIABAAA|67794|18|49|54|PM|second|evening|dinner| +67795|AAAAAAAAENIABAAA|67795|18|49|55|PM|second|evening|dinner| +67796|AAAAAAAAFNIABAAA|67796|18|49|56|PM|second|evening|dinner| +67797|AAAAAAAAGNIABAAA|67797|18|49|57|PM|second|evening|dinner| +67798|AAAAAAAAHNIABAAA|67798|18|49|58|PM|second|evening|dinner| +67799|AAAAAAAAINIABAAA|67799|18|49|59|PM|second|evening|dinner| +67800|AAAAAAAAJNIABAAA|67800|18|50|0|PM|second|evening|dinner| +67801|AAAAAAAAKNIABAAA|67801|18|50|1|PM|second|evening|dinner| +67802|AAAAAAAALNIABAAA|67802|18|50|2|PM|second|evening|dinner| +67803|AAAAAAAAMNIABAAA|67803|18|50|3|PM|second|evening|dinner| +67804|AAAAAAAANNIABAAA|67804|18|50|4|PM|second|evening|dinner| +67805|AAAAAAAAONIABAAA|67805|18|50|5|PM|second|evening|dinner| +67806|AAAAAAAAPNIABAAA|67806|18|50|6|PM|second|evening|dinner| +67807|AAAAAAAAAOIABAAA|67807|18|50|7|PM|second|evening|dinner| +67808|AAAAAAAABOIABAAA|67808|18|50|8|PM|second|evening|dinner| +67809|AAAAAAAACOIABAAA|67809|18|50|9|PM|second|evening|dinner| +67810|AAAAAAAADOIABAAA|67810|18|50|10|PM|second|evening|dinner| +67811|AAAAAAAAEOIABAAA|67811|18|50|11|PM|second|evening|dinner| +67812|AAAAAAAAFOIABAAA|67812|18|50|12|PM|second|evening|dinner| +67813|AAAAAAAAGOIABAAA|67813|18|50|13|PM|second|evening|dinner| +67814|AAAAAAAAHOIABAAA|67814|18|50|14|PM|second|evening|dinner| +67815|AAAAAAAAIOIABAAA|67815|18|50|15|PM|second|evening|dinner| +67816|AAAAAAAAJOIABAAA|67816|18|50|16|PM|second|evening|dinner| +67817|AAAAAAAAKOIABAAA|67817|18|50|17|PM|second|evening|dinner| +67818|AAAAAAAALOIABAAA|67818|18|50|18|PM|second|evening|dinner| +67819|AAAAAAAAMOIABAAA|67819|18|50|19|PM|second|evening|dinner| +67820|AAAAAAAANOIABAAA|67820|18|50|20|PM|second|evening|dinner| +67821|AAAAAAAAOOIABAAA|67821|18|50|21|PM|second|evening|dinner| +67822|AAAAAAAAPOIABAAA|67822|18|50|22|PM|second|evening|dinner| +67823|AAAAAAAAAPIABAAA|67823|18|50|23|PM|second|evening|dinner| +67824|AAAAAAAABPIABAAA|67824|18|50|24|PM|second|evening|dinner| +67825|AAAAAAAACPIABAAA|67825|18|50|25|PM|second|evening|dinner| +67826|AAAAAAAADPIABAAA|67826|18|50|26|PM|second|evening|dinner| +67827|AAAAAAAAEPIABAAA|67827|18|50|27|PM|second|evening|dinner| +67828|AAAAAAAAFPIABAAA|67828|18|50|28|PM|second|evening|dinner| +67829|AAAAAAAAGPIABAAA|67829|18|50|29|PM|second|evening|dinner| +67830|AAAAAAAAHPIABAAA|67830|18|50|30|PM|second|evening|dinner| +67831|AAAAAAAAIPIABAAA|67831|18|50|31|PM|second|evening|dinner| +67832|AAAAAAAAJPIABAAA|67832|18|50|32|PM|second|evening|dinner| +67833|AAAAAAAAKPIABAAA|67833|18|50|33|PM|second|evening|dinner| +67834|AAAAAAAALPIABAAA|67834|18|50|34|PM|second|evening|dinner| +67835|AAAAAAAAMPIABAAA|67835|18|50|35|PM|second|evening|dinner| +67836|AAAAAAAANPIABAAA|67836|18|50|36|PM|second|evening|dinner| +67837|AAAAAAAAOPIABAAA|67837|18|50|37|PM|second|evening|dinner| +67838|AAAAAAAAPPIABAAA|67838|18|50|38|PM|second|evening|dinner| +67839|AAAAAAAAAAJABAAA|67839|18|50|39|PM|second|evening|dinner| +67840|AAAAAAAABAJABAAA|67840|18|50|40|PM|second|evening|dinner| +67841|AAAAAAAACAJABAAA|67841|18|50|41|PM|second|evening|dinner| +67842|AAAAAAAADAJABAAA|67842|18|50|42|PM|second|evening|dinner| +67843|AAAAAAAAEAJABAAA|67843|18|50|43|PM|second|evening|dinner| +67844|AAAAAAAAFAJABAAA|67844|18|50|44|PM|second|evening|dinner| +67845|AAAAAAAAGAJABAAA|67845|18|50|45|PM|second|evening|dinner| +67846|AAAAAAAAHAJABAAA|67846|18|50|46|PM|second|evening|dinner| +67847|AAAAAAAAIAJABAAA|67847|18|50|47|PM|second|evening|dinner| +67848|AAAAAAAAJAJABAAA|67848|18|50|48|PM|second|evening|dinner| +67849|AAAAAAAAKAJABAAA|67849|18|50|49|PM|second|evening|dinner| +67850|AAAAAAAALAJABAAA|67850|18|50|50|PM|second|evening|dinner| +67851|AAAAAAAAMAJABAAA|67851|18|50|51|PM|second|evening|dinner| +67852|AAAAAAAANAJABAAA|67852|18|50|52|PM|second|evening|dinner| +67853|AAAAAAAAOAJABAAA|67853|18|50|53|PM|second|evening|dinner| +67854|AAAAAAAAPAJABAAA|67854|18|50|54|PM|second|evening|dinner| +67855|AAAAAAAAABJABAAA|67855|18|50|55|PM|second|evening|dinner| +67856|AAAAAAAABBJABAAA|67856|18|50|56|PM|second|evening|dinner| +67857|AAAAAAAACBJABAAA|67857|18|50|57|PM|second|evening|dinner| +67858|AAAAAAAADBJABAAA|67858|18|50|58|PM|second|evening|dinner| +67859|AAAAAAAAEBJABAAA|67859|18|50|59|PM|second|evening|dinner| +67860|AAAAAAAAFBJABAAA|67860|18|51|0|PM|second|evening|dinner| +67861|AAAAAAAAGBJABAAA|67861|18|51|1|PM|second|evening|dinner| +67862|AAAAAAAAHBJABAAA|67862|18|51|2|PM|second|evening|dinner| +67863|AAAAAAAAIBJABAAA|67863|18|51|3|PM|second|evening|dinner| +67864|AAAAAAAAJBJABAAA|67864|18|51|4|PM|second|evening|dinner| +67865|AAAAAAAAKBJABAAA|67865|18|51|5|PM|second|evening|dinner| +67866|AAAAAAAALBJABAAA|67866|18|51|6|PM|second|evening|dinner| +67867|AAAAAAAAMBJABAAA|67867|18|51|7|PM|second|evening|dinner| +67868|AAAAAAAANBJABAAA|67868|18|51|8|PM|second|evening|dinner| +67869|AAAAAAAAOBJABAAA|67869|18|51|9|PM|second|evening|dinner| +67870|AAAAAAAAPBJABAAA|67870|18|51|10|PM|second|evening|dinner| +67871|AAAAAAAAACJABAAA|67871|18|51|11|PM|second|evening|dinner| +67872|AAAAAAAABCJABAAA|67872|18|51|12|PM|second|evening|dinner| +67873|AAAAAAAACCJABAAA|67873|18|51|13|PM|second|evening|dinner| +67874|AAAAAAAADCJABAAA|67874|18|51|14|PM|second|evening|dinner| +67875|AAAAAAAAECJABAAA|67875|18|51|15|PM|second|evening|dinner| +67876|AAAAAAAAFCJABAAA|67876|18|51|16|PM|second|evening|dinner| +67877|AAAAAAAAGCJABAAA|67877|18|51|17|PM|second|evening|dinner| +67878|AAAAAAAAHCJABAAA|67878|18|51|18|PM|second|evening|dinner| +67879|AAAAAAAAICJABAAA|67879|18|51|19|PM|second|evening|dinner| +67880|AAAAAAAAJCJABAAA|67880|18|51|20|PM|second|evening|dinner| +67881|AAAAAAAAKCJABAAA|67881|18|51|21|PM|second|evening|dinner| +67882|AAAAAAAALCJABAAA|67882|18|51|22|PM|second|evening|dinner| +67883|AAAAAAAAMCJABAAA|67883|18|51|23|PM|second|evening|dinner| +67884|AAAAAAAANCJABAAA|67884|18|51|24|PM|second|evening|dinner| +67885|AAAAAAAAOCJABAAA|67885|18|51|25|PM|second|evening|dinner| +67886|AAAAAAAAPCJABAAA|67886|18|51|26|PM|second|evening|dinner| +67887|AAAAAAAAADJABAAA|67887|18|51|27|PM|second|evening|dinner| +67888|AAAAAAAABDJABAAA|67888|18|51|28|PM|second|evening|dinner| +67889|AAAAAAAACDJABAAA|67889|18|51|29|PM|second|evening|dinner| +67890|AAAAAAAADDJABAAA|67890|18|51|30|PM|second|evening|dinner| +67891|AAAAAAAAEDJABAAA|67891|18|51|31|PM|second|evening|dinner| +67892|AAAAAAAAFDJABAAA|67892|18|51|32|PM|second|evening|dinner| +67893|AAAAAAAAGDJABAAA|67893|18|51|33|PM|second|evening|dinner| +67894|AAAAAAAAHDJABAAA|67894|18|51|34|PM|second|evening|dinner| +67895|AAAAAAAAIDJABAAA|67895|18|51|35|PM|second|evening|dinner| +67896|AAAAAAAAJDJABAAA|67896|18|51|36|PM|second|evening|dinner| +67897|AAAAAAAAKDJABAAA|67897|18|51|37|PM|second|evening|dinner| +67898|AAAAAAAALDJABAAA|67898|18|51|38|PM|second|evening|dinner| +67899|AAAAAAAAMDJABAAA|67899|18|51|39|PM|second|evening|dinner| +67900|AAAAAAAANDJABAAA|67900|18|51|40|PM|second|evening|dinner| +67901|AAAAAAAAODJABAAA|67901|18|51|41|PM|second|evening|dinner| +67902|AAAAAAAAPDJABAAA|67902|18|51|42|PM|second|evening|dinner| +67903|AAAAAAAAAEJABAAA|67903|18|51|43|PM|second|evening|dinner| +67904|AAAAAAAABEJABAAA|67904|18|51|44|PM|second|evening|dinner| +67905|AAAAAAAACEJABAAA|67905|18|51|45|PM|second|evening|dinner| +67906|AAAAAAAADEJABAAA|67906|18|51|46|PM|second|evening|dinner| +67907|AAAAAAAAEEJABAAA|67907|18|51|47|PM|second|evening|dinner| +67908|AAAAAAAAFEJABAAA|67908|18|51|48|PM|second|evening|dinner| +67909|AAAAAAAAGEJABAAA|67909|18|51|49|PM|second|evening|dinner| +67910|AAAAAAAAHEJABAAA|67910|18|51|50|PM|second|evening|dinner| +67911|AAAAAAAAIEJABAAA|67911|18|51|51|PM|second|evening|dinner| +67912|AAAAAAAAJEJABAAA|67912|18|51|52|PM|second|evening|dinner| +67913|AAAAAAAAKEJABAAA|67913|18|51|53|PM|second|evening|dinner| +67914|AAAAAAAALEJABAAA|67914|18|51|54|PM|second|evening|dinner| +67915|AAAAAAAAMEJABAAA|67915|18|51|55|PM|second|evening|dinner| +67916|AAAAAAAANEJABAAA|67916|18|51|56|PM|second|evening|dinner| +67917|AAAAAAAAOEJABAAA|67917|18|51|57|PM|second|evening|dinner| +67918|AAAAAAAAPEJABAAA|67918|18|51|58|PM|second|evening|dinner| +67919|AAAAAAAAAFJABAAA|67919|18|51|59|PM|second|evening|dinner| +67920|AAAAAAAABFJABAAA|67920|18|52|0|PM|second|evening|dinner| +67921|AAAAAAAACFJABAAA|67921|18|52|1|PM|second|evening|dinner| +67922|AAAAAAAADFJABAAA|67922|18|52|2|PM|second|evening|dinner| +67923|AAAAAAAAEFJABAAA|67923|18|52|3|PM|second|evening|dinner| +67924|AAAAAAAAFFJABAAA|67924|18|52|4|PM|second|evening|dinner| +67925|AAAAAAAAGFJABAAA|67925|18|52|5|PM|second|evening|dinner| +67926|AAAAAAAAHFJABAAA|67926|18|52|6|PM|second|evening|dinner| +67927|AAAAAAAAIFJABAAA|67927|18|52|7|PM|second|evening|dinner| +67928|AAAAAAAAJFJABAAA|67928|18|52|8|PM|second|evening|dinner| +67929|AAAAAAAAKFJABAAA|67929|18|52|9|PM|second|evening|dinner| +67930|AAAAAAAALFJABAAA|67930|18|52|10|PM|second|evening|dinner| +67931|AAAAAAAAMFJABAAA|67931|18|52|11|PM|second|evening|dinner| +67932|AAAAAAAANFJABAAA|67932|18|52|12|PM|second|evening|dinner| +67933|AAAAAAAAOFJABAAA|67933|18|52|13|PM|second|evening|dinner| +67934|AAAAAAAAPFJABAAA|67934|18|52|14|PM|second|evening|dinner| +67935|AAAAAAAAAGJABAAA|67935|18|52|15|PM|second|evening|dinner| +67936|AAAAAAAABGJABAAA|67936|18|52|16|PM|second|evening|dinner| +67937|AAAAAAAACGJABAAA|67937|18|52|17|PM|second|evening|dinner| +67938|AAAAAAAADGJABAAA|67938|18|52|18|PM|second|evening|dinner| +67939|AAAAAAAAEGJABAAA|67939|18|52|19|PM|second|evening|dinner| +67940|AAAAAAAAFGJABAAA|67940|18|52|20|PM|second|evening|dinner| +67941|AAAAAAAAGGJABAAA|67941|18|52|21|PM|second|evening|dinner| +67942|AAAAAAAAHGJABAAA|67942|18|52|22|PM|second|evening|dinner| +67943|AAAAAAAAIGJABAAA|67943|18|52|23|PM|second|evening|dinner| +67944|AAAAAAAAJGJABAAA|67944|18|52|24|PM|second|evening|dinner| +67945|AAAAAAAAKGJABAAA|67945|18|52|25|PM|second|evening|dinner| +67946|AAAAAAAALGJABAAA|67946|18|52|26|PM|second|evening|dinner| +67947|AAAAAAAAMGJABAAA|67947|18|52|27|PM|second|evening|dinner| +67948|AAAAAAAANGJABAAA|67948|18|52|28|PM|second|evening|dinner| +67949|AAAAAAAAOGJABAAA|67949|18|52|29|PM|second|evening|dinner| +67950|AAAAAAAAPGJABAAA|67950|18|52|30|PM|second|evening|dinner| +67951|AAAAAAAAAHJABAAA|67951|18|52|31|PM|second|evening|dinner| +67952|AAAAAAAABHJABAAA|67952|18|52|32|PM|second|evening|dinner| +67953|AAAAAAAACHJABAAA|67953|18|52|33|PM|second|evening|dinner| +67954|AAAAAAAADHJABAAA|67954|18|52|34|PM|second|evening|dinner| +67955|AAAAAAAAEHJABAAA|67955|18|52|35|PM|second|evening|dinner| +67956|AAAAAAAAFHJABAAA|67956|18|52|36|PM|second|evening|dinner| +67957|AAAAAAAAGHJABAAA|67957|18|52|37|PM|second|evening|dinner| +67958|AAAAAAAAHHJABAAA|67958|18|52|38|PM|second|evening|dinner| +67959|AAAAAAAAIHJABAAA|67959|18|52|39|PM|second|evening|dinner| +67960|AAAAAAAAJHJABAAA|67960|18|52|40|PM|second|evening|dinner| +67961|AAAAAAAAKHJABAAA|67961|18|52|41|PM|second|evening|dinner| +67962|AAAAAAAALHJABAAA|67962|18|52|42|PM|second|evening|dinner| +67963|AAAAAAAAMHJABAAA|67963|18|52|43|PM|second|evening|dinner| +67964|AAAAAAAANHJABAAA|67964|18|52|44|PM|second|evening|dinner| +67965|AAAAAAAAOHJABAAA|67965|18|52|45|PM|second|evening|dinner| +67966|AAAAAAAAPHJABAAA|67966|18|52|46|PM|second|evening|dinner| +67967|AAAAAAAAAIJABAAA|67967|18|52|47|PM|second|evening|dinner| +67968|AAAAAAAABIJABAAA|67968|18|52|48|PM|second|evening|dinner| +67969|AAAAAAAACIJABAAA|67969|18|52|49|PM|second|evening|dinner| +67970|AAAAAAAADIJABAAA|67970|18|52|50|PM|second|evening|dinner| +67971|AAAAAAAAEIJABAAA|67971|18|52|51|PM|second|evening|dinner| +67972|AAAAAAAAFIJABAAA|67972|18|52|52|PM|second|evening|dinner| +67973|AAAAAAAAGIJABAAA|67973|18|52|53|PM|second|evening|dinner| +67974|AAAAAAAAHIJABAAA|67974|18|52|54|PM|second|evening|dinner| +67975|AAAAAAAAIIJABAAA|67975|18|52|55|PM|second|evening|dinner| +67976|AAAAAAAAJIJABAAA|67976|18|52|56|PM|second|evening|dinner| +67977|AAAAAAAAKIJABAAA|67977|18|52|57|PM|second|evening|dinner| +67978|AAAAAAAALIJABAAA|67978|18|52|58|PM|second|evening|dinner| +67979|AAAAAAAAMIJABAAA|67979|18|52|59|PM|second|evening|dinner| +67980|AAAAAAAANIJABAAA|67980|18|53|0|PM|second|evening|dinner| +67981|AAAAAAAAOIJABAAA|67981|18|53|1|PM|second|evening|dinner| +67982|AAAAAAAAPIJABAAA|67982|18|53|2|PM|second|evening|dinner| +67983|AAAAAAAAAJJABAAA|67983|18|53|3|PM|second|evening|dinner| +67984|AAAAAAAABJJABAAA|67984|18|53|4|PM|second|evening|dinner| +67985|AAAAAAAACJJABAAA|67985|18|53|5|PM|second|evening|dinner| +67986|AAAAAAAADJJABAAA|67986|18|53|6|PM|second|evening|dinner| +67987|AAAAAAAAEJJABAAA|67987|18|53|7|PM|second|evening|dinner| +67988|AAAAAAAAFJJABAAA|67988|18|53|8|PM|second|evening|dinner| +67989|AAAAAAAAGJJABAAA|67989|18|53|9|PM|second|evening|dinner| +67990|AAAAAAAAHJJABAAA|67990|18|53|10|PM|second|evening|dinner| +67991|AAAAAAAAIJJABAAA|67991|18|53|11|PM|second|evening|dinner| +67992|AAAAAAAAJJJABAAA|67992|18|53|12|PM|second|evening|dinner| +67993|AAAAAAAAKJJABAAA|67993|18|53|13|PM|second|evening|dinner| +67994|AAAAAAAALJJABAAA|67994|18|53|14|PM|second|evening|dinner| +67995|AAAAAAAAMJJABAAA|67995|18|53|15|PM|second|evening|dinner| +67996|AAAAAAAANJJABAAA|67996|18|53|16|PM|second|evening|dinner| +67997|AAAAAAAAOJJABAAA|67997|18|53|17|PM|second|evening|dinner| +67998|AAAAAAAAPJJABAAA|67998|18|53|18|PM|second|evening|dinner| +67999|AAAAAAAAAKJABAAA|67999|18|53|19|PM|second|evening|dinner| +68000|AAAAAAAABKJABAAA|68000|18|53|20|PM|second|evening|dinner| +68001|AAAAAAAACKJABAAA|68001|18|53|21|PM|second|evening|dinner| +68002|AAAAAAAADKJABAAA|68002|18|53|22|PM|second|evening|dinner| +68003|AAAAAAAAEKJABAAA|68003|18|53|23|PM|second|evening|dinner| +68004|AAAAAAAAFKJABAAA|68004|18|53|24|PM|second|evening|dinner| +68005|AAAAAAAAGKJABAAA|68005|18|53|25|PM|second|evening|dinner| +68006|AAAAAAAAHKJABAAA|68006|18|53|26|PM|second|evening|dinner| +68007|AAAAAAAAIKJABAAA|68007|18|53|27|PM|second|evening|dinner| +68008|AAAAAAAAJKJABAAA|68008|18|53|28|PM|second|evening|dinner| +68009|AAAAAAAAKKJABAAA|68009|18|53|29|PM|second|evening|dinner| +68010|AAAAAAAALKJABAAA|68010|18|53|30|PM|second|evening|dinner| +68011|AAAAAAAAMKJABAAA|68011|18|53|31|PM|second|evening|dinner| +68012|AAAAAAAANKJABAAA|68012|18|53|32|PM|second|evening|dinner| +68013|AAAAAAAAOKJABAAA|68013|18|53|33|PM|second|evening|dinner| +68014|AAAAAAAAPKJABAAA|68014|18|53|34|PM|second|evening|dinner| +68015|AAAAAAAAALJABAAA|68015|18|53|35|PM|second|evening|dinner| +68016|AAAAAAAABLJABAAA|68016|18|53|36|PM|second|evening|dinner| +68017|AAAAAAAACLJABAAA|68017|18|53|37|PM|second|evening|dinner| +68018|AAAAAAAADLJABAAA|68018|18|53|38|PM|second|evening|dinner| +68019|AAAAAAAAELJABAAA|68019|18|53|39|PM|second|evening|dinner| +68020|AAAAAAAAFLJABAAA|68020|18|53|40|PM|second|evening|dinner| +68021|AAAAAAAAGLJABAAA|68021|18|53|41|PM|second|evening|dinner| +68022|AAAAAAAAHLJABAAA|68022|18|53|42|PM|second|evening|dinner| +68023|AAAAAAAAILJABAAA|68023|18|53|43|PM|second|evening|dinner| +68024|AAAAAAAAJLJABAAA|68024|18|53|44|PM|second|evening|dinner| +68025|AAAAAAAAKLJABAAA|68025|18|53|45|PM|second|evening|dinner| +68026|AAAAAAAALLJABAAA|68026|18|53|46|PM|second|evening|dinner| +68027|AAAAAAAAMLJABAAA|68027|18|53|47|PM|second|evening|dinner| +68028|AAAAAAAANLJABAAA|68028|18|53|48|PM|second|evening|dinner| +68029|AAAAAAAAOLJABAAA|68029|18|53|49|PM|second|evening|dinner| +68030|AAAAAAAAPLJABAAA|68030|18|53|50|PM|second|evening|dinner| +68031|AAAAAAAAAMJABAAA|68031|18|53|51|PM|second|evening|dinner| +68032|AAAAAAAABMJABAAA|68032|18|53|52|PM|second|evening|dinner| +68033|AAAAAAAACMJABAAA|68033|18|53|53|PM|second|evening|dinner| +68034|AAAAAAAADMJABAAA|68034|18|53|54|PM|second|evening|dinner| +68035|AAAAAAAAEMJABAAA|68035|18|53|55|PM|second|evening|dinner| +68036|AAAAAAAAFMJABAAA|68036|18|53|56|PM|second|evening|dinner| +68037|AAAAAAAAGMJABAAA|68037|18|53|57|PM|second|evening|dinner| +68038|AAAAAAAAHMJABAAA|68038|18|53|58|PM|second|evening|dinner| +68039|AAAAAAAAIMJABAAA|68039|18|53|59|PM|second|evening|dinner| +68040|AAAAAAAAJMJABAAA|68040|18|54|0|PM|second|evening|dinner| +68041|AAAAAAAAKMJABAAA|68041|18|54|1|PM|second|evening|dinner| +68042|AAAAAAAALMJABAAA|68042|18|54|2|PM|second|evening|dinner| +68043|AAAAAAAAMMJABAAA|68043|18|54|3|PM|second|evening|dinner| +68044|AAAAAAAANMJABAAA|68044|18|54|4|PM|second|evening|dinner| +68045|AAAAAAAAOMJABAAA|68045|18|54|5|PM|second|evening|dinner| +68046|AAAAAAAAPMJABAAA|68046|18|54|6|PM|second|evening|dinner| +68047|AAAAAAAAANJABAAA|68047|18|54|7|PM|second|evening|dinner| +68048|AAAAAAAABNJABAAA|68048|18|54|8|PM|second|evening|dinner| +68049|AAAAAAAACNJABAAA|68049|18|54|9|PM|second|evening|dinner| +68050|AAAAAAAADNJABAAA|68050|18|54|10|PM|second|evening|dinner| +68051|AAAAAAAAENJABAAA|68051|18|54|11|PM|second|evening|dinner| +68052|AAAAAAAAFNJABAAA|68052|18|54|12|PM|second|evening|dinner| +68053|AAAAAAAAGNJABAAA|68053|18|54|13|PM|second|evening|dinner| +68054|AAAAAAAAHNJABAAA|68054|18|54|14|PM|second|evening|dinner| +68055|AAAAAAAAINJABAAA|68055|18|54|15|PM|second|evening|dinner| +68056|AAAAAAAAJNJABAAA|68056|18|54|16|PM|second|evening|dinner| +68057|AAAAAAAAKNJABAAA|68057|18|54|17|PM|second|evening|dinner| +68058|AAAAAAAALNJABAAA|68058|18|54|18|PM|second|evening|dinner| +68059|AAAAAAAAMNJABAAA|68059|18|54|19|PM|second|evening|dinner| +68060|AAAAAAAANNJABAAA|68060|18|54|20|PM|second|evening|dinner| +68061|AAAAAAAAONJABAAA|68061|18|54|21|PM|second|evening|dinner| +68062|AAAAAAAAPNJABAAA|68062|18|54|22|PM|second|evening|dinner| +68063|AAAAAAAAAOJABAAA|68063|18|54|23|PM|second|evening|dinner| +68064|AAAAAAAABOJABAAA|68064|18|54|24|PM|second|evening|dinner| +68065|AAAAAAAACOJABAAA|68065|18|54|25|PM|second|evening|dinner| +68066|AAAAAAAADOJABAAA|68066|18|54|26|PM|second|evening|dinner| +68067|AAAAAAAAEOJABAAA|68067|18|54|27|PM|second|evening|dinner| +68068|AAAAAAAAFOJABAAA|68068|18|54|28|PM|second|evening|dinner| +68069|AAAAAAAAGOJABAAA|68069|18|54|29|PM|second|evening|dinner| +68070|AAAAAAAAHOJABAAA|68070|18|54|30|PM|second|evening|dinner| +68071|AAAAAAAAIOJABAAA|68071|18|54|31|PM|second|evening|dinner| +68072|AAAAAAAAJOJABAAA|68072|18|54|32|PM|second|evening|dinner| +68073|AAAAAAAAKOJABAAA|68073|18|54|33|PM|second|evening|dinner| +68074|AAAAAAAALOJABAAA|68074|18|54|34|PM|second|evening|dinner| +68075|AAAAAAAAMOJABAAA|68075|18|54|35|PM|second|evening|dinner| +68076|AAAAAAAANOJABAAA|68076|18|54|36|PM|second|evening|dinner| +68077|AAAAAAAAOOJABAAA|68077|18|54|37|PM|second|evening|dinner| +68078|AAAAAAAAPOJABAAA|68078|18|54|38|PM|second|evening|dinner| +68079|AAAAAAAAAPJABAAA|68079|18|54|39|PM|second|evening|dinner| +68080|AAAAAAAABPJABAAA|68080|18|54|40|PM|second|evening|dinner| +68081|AAAAAAAACPJABAAA|68081|18|54|41|PM|second|evening|dinner| +68082|AAAAAAAADPJABAAA|68082|18|54|42|PM|second|evening|dinner| +68083|AAAAAAAAEPJABAAA|68083|18|54|43|PM|second|evening|dinner| +68084|AAAAAAAAFPJABAAA|68084|18|54|44|PM|second|evening|dinner| +68085|AAAAAAAAGPJABAAA|68085|18|54|45|PM|second|evening|dinner| +68086|AAAAAAAAHPJABAAA|68086|18|54|46|PM|second|evening|dinner| +68087|AAAAAAAAIPJABAAA|68087|18|54|47|PM|second|evening|dinner| +68088|AAAAAAAAJPJABAAA|68088|18|54|48|PM|second|evening|dinner| +68089|AAAAAAAAKPJABAAA|68089|18|54|49|PM|second|evening|dinner| +68090|AAAAAAAALPJABAAA|68090|18|54|50|PM|second|evening|dinner| +68091|AAAAAAAAMPJABAAA|68091|18|54|51|PM|second|evening|dinner| +68092|AAAAAAAANPJABAAA|68092|18|54|52|PM|second|evening|dinner| +68093|AAAAAAAAOPJABAAA|68093|18|54|53|PM|second|evening|dinner| +68094|AAAAAAAAPPJABAAA|68094|18|54|54|PM|second|evening|dinner| +68095|AAAAAAAAAAKABAAA|68095|18|54|55|PM|second|evening|dinner| +68096|AAAAAAAABAKABAAA|68096|18|54|56|PM|second|evening|dinner| +68097|AAAAAAAACAKABAAA|68097|18|54|57|PM|second|evening|dinner| +68098|AAAAAAAADAKABAAA|68098|18|54|58|PM|second|evening|dinner| +68099|AAAAAAAAEAKABAAA|68099|18|54|59|PM|second|evening|dinner| +68100|AAAAAAAAFAKABAAA|68100|18|55|0|PM|second|evening|dinner| +68101|AAAAAAAAGAKABAAA|68101|18|55|1|PM|second|evening|dinner| +68102|AAAAAAAAHAKABAAA|68102|18|55|2|PM|second|evening|dinner| +68103|AAAAAAAAIAKABAAA|68103|18|55|3|PM|second|evening|dinner| +68104|AAAAAAAAJAKABAAA|68104|18|55|4|PM|second|evening|dinner| +68105|AAAAAAAAKAKABAAA|68105|18|55|5|PM|second|evening|dinner| +68106|AAAAAAAALAKABAAA|68106|18|55|6|PM|second|evening|dinner| +68107|AAAAAAAAMAKABAAA|68107|18|55|7|PM|second|evening|dinner| +68108|AAAAAAAANAKABAAA|68108|18|55|8|PM|second|evening|dinner| +68109|AAAAAAAAOAKABAAA|68109|18|55|9|PM|second|evening|dinner| +68110|AAAAAAAAPAKABAAA|68110|18|55|10|PM|second|evening|dinner| +68111|AAAAAAAAABKABAAA|68111|18|55|11|PM|second|evening|dinner| +68112|AAAAAAAABBKABAAA|68112|18|55|12|PM|second|evening|dinner| +68113|AAAAAAAACBKABAAA|68113|18|55|13|PM|second|evening|dinner| +68114|AAAAAAAADBKABAAA|68114|18|55|14|PM|second|evening|dinner| +68115|AAAAAAAAEBKABAAA|68115|18|55|15|PM|second|evening|dinner| +68116|AAAAAAAAFBKABAAA|68116|18|55|16|PM|second|evening|dinner| +68117|AAAAAAAAGBKABAAA|68117|18|55|17|PM|second|evening|dinner| +68118|AAAAAAAAHBKABAAA|68118|18|55|18|PM|second|evening|dinner| +68119|AAAAAAAAIBKABAAA|68119|18|55|19|PM|second|evening|dinner| +68120|AAAAAAAAJBKABAAA|68120|18|55|20|PM|second|evening|dinner| +68121|AAAAAAAAKBKABAAA|68121|18|55|21|PM|second|evening|dinner| +68122|AAAAAAAALBKABAAA|68122|18|55|22|PM|second|evening|dinner| +68123|AAAAAAAAMBKABAAA|68123|18|55|23|PM|second|evening|dinner| +68124|AAAAAAAANBKABAAA|68124|18|55|24|PM|second|evening|dinner| +68125|AAAAAAAAOBKABAAA|68125|18|55|25|PM|second|evening|dinner| +68126|AAAAAAAAPBKABAAA|68126|18|55|26|PM|second|evening|dinner| +68127|AAAAAAAAACKABAAA|68127|18|55|27|PM|second|evening|dinner| +68128|AAAAAAAABCKABAAA|68128|18|55|28|PM|second|evening|dinner| +68129|AAAAAAAACCKABAAA|68129|18|55|29|PM|second|evening|dinner| +68130|AAAAAAAADCKABAAA|68130|18|55|30|PM|second|evening|dinner| +68131|AAAAAAAAECKABAAA|68131|18|55|31|PM|second|evening|dinner| +68132|AAAAAAAAFCKABAAA|68132|18|55|32|PM|second|evening|dinner| +68133|AAAAAAAAGCKABAAA|68133|18|55|33|PM|second|evening|dinner| +68134|AAAAAAAAHCKABAAA|68134|18|55|34|PM|second|evening|dinner| +68135|AAAAAAAAICKABAAA|68135|18|55|35|PM|second|evening|dinner| +68136|AAAAAAAAJCKABAAA|68136|18|55|36|PM|second|evening|dinner| +68137|AAAAAAAAKCKABAAA|68137|18|55|37|PM|second|evening|dinner| +68138|AAAAAAAALCKABAAA|68138|18|55|38|PM|second|evening|dinner| +68139|AAAAAAAAMCKABAAA|68139|18|55|39|PM|second|evening|dinner| +68140|AAAAAAAANCKABAAA|68140|18|55|40|PM|second|evening|dinner| +68141|AAAAAAAAOCKABAAA|68141|18|55|41|PM|second|evening|dinner| +68142|AAAAAAAAPCKABAAA|68142|18|55|42|PM|second|evening|dinner| +68143|AAAAAAAAADKABAAA|68143|18|55|43|PM|second|evening|dinner| +68144|AAAAAAAABDKABAAA|68144|18|55|44|PM|second|evening|dinner| +68145|AAAAAAAACDKABAAA|68145|18|55|45|PM|second|evening|dinner| +68146|AAAAAAAADDKABAAA|68146|18|55|46|PM|second|evening|dinner| +68147|AAAAAAAAEDKABAAA|68147|18|55|47|PM|second|evening|dinner| +68148|AAAAAAAAFDKABAAA|68148|18|55|48|PM|second|evening|dinner| +68149|AAAAAAAAGDKABAAA|68149|18|55|49|PM|second|evening|dinner| +68150|AAAAAAAAHDKABAAA|68150|18|55|50|PM|second|evening|dinner| +68151|AAAAAAAAIDKABAAA|68151|18|55|51|PM|second|evening|dinner| +68152|AAAAAAAAJDKABAAA|68152|18|55|52|PM|second|evening|dinner| +68153|AAAAAAAAKDKABAAA|68153|18|55|53|PM|second|evening|dinner| +68154|AAAAAAAALDKABAAA|68154|18|55|54|PM|second|evening|dinner| +68155|AAAAAAAAMDKABAAA|68155|18|55|55|PM|second|evening|dinner| +68156|AAAAAAAANDKABAAA|68156|18|55|56|PM|second|evening|dinner| +68157|AAAAAAAAODKABAAA|68157|18|55|57|PM|second|evening|dinner| +68158|AAAAAAAAPDKABAAA|68158|18|55|58|PM|second|evening|dinner| +68159|AAAAAAAAAEKABAAA|68159|18|55|59|PM|second|evening|dinner| +68160|AAAAAAAABEKABAAA|68160|18|56|0|PM|second|evening|dinner| +68161|AAAAAAAACEKABAAA|68161|18|56|1|PM|second|evening|dinner| +68162|AAAAAAAADEKABAAA|68162|18|56|2|PM|second|evening|dinner| +68163|AAAAAAAAEEKABAAA|68163|18|56|3|PM|second|evening|dinner| +68164|AAAAAAAAFEKABAAA|68164|18|56|4|PM|second|evening|dinner| +68165|AAAAAAAAGEKABAAA|68165|18|56|5|PM|second|evening|dinner| +68166|AAAAAAAAHEKABAAA|68166|18|56|6|PM|second|evening|dinner| +68167|AAAAAAAAIEKABAAA|68167|18|56|7|PM|second|evening|dinner| +68168|AAAAAAAAJEKABAAA|68168|18|56|8|PM|second|evening|dinner| +68169|AAAAAAAAKEKABAAA|68169|18|56|9|PM|second|evening|dinner| +68170|AAAAAAAALEKABAAA|68170|18|56|10|PM|second|evening|dinner| +68171|AAAAAAAAMEKABAAA|68171|18|56|11|PM|second|evening|dinner| +68172|AAAAAAAANEKABAAA|68172|18|56|12|PM|second|evening|dinner| +68173|AAAAAAAAOEKABAAA|68173|18|56|13|PM|second|evening|dinner| +68174|AAAAAAAAPEKABAAA|68174|18|56|14|PM|second|evening|dinner| +68175|AAAAAAAAAFKABAAA|68175|18|56|15|PM|second|evening|dinner| +68176|AAAAAAAABFKABAAA|68176|18|56|16|PM|second|evening|dinner| +68177|AAAAAAAACFKABAAA|68177|18|56|17|PM|second|evening|dinner| +68178|AAAAAAAADFKABAAA|68178|18|56|18|PM|second|evening|dinner| +68179|AAAAAAAAEFKABAAA|68179|18|56|19|PM|second|evening|dinner| +68180|AAAAAAAAFFKABAAA|68180|18|56|20|PM|second|evening|dinner| +68181|AAAAAAAAGFKABAAA|68181|18|56|21|PM|second|evening|dinner| +68182|AAAAAAAAHFKABAAA|68182|18|56|22|PM|second|evening|dinner| +68183|AAAAAAAAIFKABAAA|68183|18|56|23|PM|second|evening|dinner| +68184|AAAAAAAAJFKABAAA|68184|18|56|24|PM|second|evening|dinner| +68185|AAAAAAAAKFKABAAA|68185|18|56|25|PM|second|evening|dinner| +68186|AAAAAAAALFKABAAA|68186|18|56|26|PM|second|evening|dinner| +68187|AAAAAAAAMFKABAAA|68187|18|56|27|PM|second|evening|dinner| +68188|AAAAAAAANFKABAAA|68188|18|56|28|PM|second|evening|dinner| +68189|AAAAAAAAOFKABAAA|68189|18|56|29|PM|second|evening|dinner| +68190|AAAAAAAAPFKABAAA|68190|18|56|30|PM|second|evening|dinner| +68191|AAAAAAAAAGKABAAA|68191|18|56|31|PM|second|evening|dinner| +68192|AAAAAAAABGKABAAA|68192|18|56|32|PM|second|evening|dinner| +68193|AAAAAAAACGKABAAA|68193|18|56|33|PM|second|evening|dinner| +68194|AAAAAAAADGKABAAA|68194|18|56|34|PM|second|evening|dinner| +68195|AAAAAAAAEGKABAAA|68195|18|56|35|PM|second|evening|dinner| +68196|AAAAAAAAFGKABAAA|68196|18|56|36|PM|second|evening|dinner| +68197|AAAAAAAAGGKABAAA|68197|18|56|37|PM|second|evening|dinner| +68198|AAAAAAAAHGKABAAA|68198|18|56|38|PM|second|evening|dinner| +68199|AAAAAAAAIGKABAAA|68199|18|56|39|PM|second|evening|dinner| +68200|AAAAAAAAJGKABAAA|68200|18|56|40|PM|second|evening|dinner| +68201|AAAAAAAAKGKABAAA|68201|18|56|41|PM|second|evening|dinner| +68202|AAAAAAAALGKABAAA|68202|18|56|42|PM|second|evening|dinner| +68203|AAAAAAAAMGKABAAA|68203|18|56|43|PM|second|evening|dinner| +68204|AAAAAAAANGKABAAA|68204|18|56|44|PM|second|evening|dinner| +68205|AAAAAAAAOGKABAAA|68205|18|56|45|PM|second|evening|dinner| +68206|AAAAAAAAPGKABAAA|68206|18|56|46|PM|second|evening|dinner| +68207|AAAAAAAAAHKABAAA|68207|18|56|47|PM|second|evening|dinner| +68208|AAAAAAAABHKABAAA|68208|18|56|48|PM|second|evening|dinner| +68209|AAAAAAAACHKABAAA|68209|18|56|49|PM|second|evening|dinner| +68210|AAAAAAAADHKABAAA|68210|18|56|50|PM|second|evening|dinner| +68211|AAAAAAAAEHKABAAA|68211|18|56|51|PM|second|evening|dinner| +68212|AAAAAAAAFHKABAAA|68212|18|56|52|PM|second|evening|dinner| +68213|AAAAAAAAGHKABAAA|68213|18|56|53|PM|second|evening|dinner| +68214|AAAAAAAAHHKABAAA|68214|18|56|54|PM|second|evening|dinner| +68215|AAAAAAAAIHKABAAA|68215|18|56|55|PM|second|evening|dinner| +68216|AAAAAAAAJHKABAAA|68216|18|56|56|PM|second|evening|dinner| +68217|AAAAAAAAKHKABAAA|68217|18|56|57|PM|second|evening|dinner| +68218|AAAAAAAALHKABAAA|68218|18|56|58|PM|second|evening|dinner| +68219|AAAAAAAAMHKABAAA|68219|18|56|59|PM|second|evening|dinner| +68220|AAAAAAAANHKABAAA|68220|18|57|0|PM|second|evening|dinner| +68221|AAAAAAAAOHKABAAA|68221|18|57|1|PM|second|evening|dinner| +68222|AAAAAAAAPHKABAAA|68222|18|57|2|PM|second|evening|dinner| +68223|AAAAAAAAAIKABAAA|68223|18|57|3|PM|second|evening|dinner| +68224|AAAAAAAABIKABAAA|68224|18|57|4|PM|second|evening|dinner| +68225|AAAAAAAACIKABAAA|68225|18|57|5|PM|second|evening|dinner| +68226|AAAAAAAADIKABAAA|68226|18|57|6|PM|second|evening|dinner| +68227|AAAAAAAAEIKABAAA|68227|18|57|7|PM|second|evening|dinner| +68228|AAAAAAAAFIKABAAA|68228|18|57|8|PM|second|evening|dinner| +68229|AAAAAAAAGIKABAAA|68229|18|57|9|PM|second|evening|dinner| +68230|AAAAAAAAHIKABAAA|68230|18|57|10|PM|second|evening|dinner| +68231|AAAAAAAAIIKABAAA|68231|18|57|11|PM|second|evening|dinner| +68232|AAAAAAAAJIKABAAA|68232|18|57|12|PM|second|evening|dinner| +68233|AAAAAAAAKIKABAAA|68233|18|57|13|PM|second|evening|dinner| +68234|AAAAAAAALIKABAAA|68234|18|57|14|PM|second|evening|dinner| +68235|AAAAAAAAMIKABAAA|68235|18|57|15|PM|second|evening|dinner| +68236|AAAAAAAANIKABAAA|68236|18|57|16|PM|second|evening|dinner| +68237|AAAAAAAAOIKABAAA|68237|18|57|17|PM|second|evening|dinner| +68238|AAAAAAAAPIKABAAA|68238|18|57|18|PM|second|evening|dinner| +68239|AAAAAAAAAJKABAAA|68239|18|57|19|PM|second|evening|dinner| +68240|AAAAAAAABJKABAAA|68240|18|57|20|PM|second|evening|dinner| +68241|AAAAAAAACJKABAAA|68241|18|57|21|PM|second|evening|dinner| +68242|AAAAAAAADJKABAAA|68242|18|57|22|PM|second|evening|dinner| +68243|AAAAAAAAEJKABAAA|68243|18|57|23|PM|second|evening|dinner| +68244|AAAAAAAAFJKABAAA|68244|18|57|24|PM|second|evening|dinner| +68245|AAAAAAAAGJKABAAA|68245|18|57|25|PM|second|evening|dinner| +68246|AAAAAAAAHJKABAAA|68246|18|57|26|PM|second|evening|dinner| +68247|AAAAAAAAIJKABAAA|68247|18|57|27|PM|second|evening|dinner| +68248|AAAAAAAAJJKABAAA|68248|18|57|28|PM|second|evening|dinner| +68249|AAAAAAAAKJKABAAA|68249|18|57|29|PM|second|evening|dinner| +68250|AAAAAAAALJKABAAA|68250|18|57|30|PM|second|evening|dinner| +68251|AAAAAAAAMJKABAAA|68251|18|57|31|PM|second|evening|dinner| +68252|AAAAAAAANJKABAAA|68252|18|57|32|PM|second|evening|dinner| +68253|AAAAAAAAOJKABAAA|68253|18|57|33|PM|second|evening|dinner| +68254|AAAAAAAAPJKABAAA|68254|18|57|34|PM|second|evening|dinner| +68255|AAAAAAAAAKKABAAA|68255|18|57|35|PM|second|evening|dinner| +68256|AAAAAAAABKKABAAA|68256|18|57|36|PM|second|evening|dinner| +68257|AAAAAAAACKKABAAA|68257|18|57|37|PM|second|evening|dinner| +68258|AAAAAAAADKKABAAA|68258|18|57|38|PM|second|evening|dinner| +68259|AAAAAAAAEKKABAAA|68259|18|57|39|PM|second|evening|dinner| +68260|AAAAAAAAFKKABAAA|68260|18|57|40|PM|second|evening|dinner| +68261|AAAAAAAAGKKABAAA|68261|18|57|41|PM|second|evening|dinner| +68262|AAAAAAAAHKKABAAA|68262|18|57|42|PM|second|evening|dinner| +68263|AAAAAAAAIKKABAAA|68263|18|57|43|PM|second|evening|dinner| +68264|AAAAAAAAJKKABAAA|68264|18|57|44|PM|second|evening|dinner| +68265|AAAAAAAAKKKABAAA|68265|18|57|45|PM|second|evening|dinner| +68266|AAAAAAAALKKABAAA|68266|18|57|46|PM|second|evening|dinner| +68267|AAAAAAAAMKKABAAA|68267|18|57|47|PM|second|evening|dinner| +68268|AAAAAAAANKKABAAA|68268|18|57|48|PM|second|evening|dinner| +68269|AAAAAAAAOKKABAAA|68269|18|57|49|PM|second|evening|dinner| +68270|AAAAAAAAPKKABAAA|68270|18|57|50|PM|second|evening|dinner| +68271|AAAAAAAAALKABAAA|68271|18|57|51|PM|second|evening|dinner| +68272|AAAAAAAABLKABAAA|68272|18|57|52|PM|second|evening|dinner| +68273|AAAAAAAACLKABAAA|68273|18|57|53|PM|second|evening|dinner| +68274|AAAAAAAADLKABAAA|68274|18|57|54|PM|second|evening|dinner| +68275|AAAAAAAAELKABAAA|68275|18|57|55|PM|second|evening|dinner| +68276|AAAAAAAAFLKABAAA|68276|18|57|56|PM|second|evening|dinner| +68277|AAAAAAAAGLKABAAA|68277|18|57|57|PM|second|evening|dinner| +68278|AAAAAAAAHLKABAAA|68278|18|57|58|PM|second|evening|dinner| +68279|AAAAAAAAILKABAAA|68279|18|57|59|PM|second|evening|dinner| +68280|AAAAAAAAJLKABAAA|68280|18|58|0|PM|second|evening|dinner| +68281|AAAAAAAAKLKABAAA|68281|18|58|1|PM|second|evening|dinner| +68282|AAAAAAAALLKABAAA|68282|18|58|2|PM|second|evening|dinner| +68283|AAAAAAAAMLKABAAA|68283|18|58|3|PM|second|evening|dinner| +68284|AAAAAAAANLKABAAA|68284|18|58|4|PM|second|evening|dinner| +68285|AAAAAAAAOLKABAAA|68285|18|58|5|PM|second|evening|dinner| +68286|AAAAAAAAPLKABAAA|68286|18|58|6|PM|second|evening|dinner| +68287|AAAAAAAAAMKABAAA|68287|18|58|7|PM|second|evening|dinner| +68288|AAAAAAAABMKABAAA|68288|18|58|8|PM|second|evening|dinner| +68289|AAAAAAAACMKABAAA|68289|18|58|9|PM|second|evening|dinner| +68290|AAAAAAAADMKABAAA|68290|18|58|10|PM|second|evening|dinner| +68291|AAAAAAAAEMKABAAA|68291|18|58|11|PM|second|evening|dinner| +68292|AAAAAAAAFMKABAAA|68292|18|58|12|PM|second|evening|dinner| +68293|AAAAAAAAGMKABAAA|68293|18|58|13|PM|second|evening|dinner| +68294|AAAAAAAAHMKABAAA|68294|18|58|14|PM|second|evening|dinner| +68295|AAAAAAAAIMKABAAA|68295|18|58|15|PM|second|evening|dinner| +68296|AAAAAAAAJMKABAAA|68296|18|58|16|PM|second|evening|dinner| +68297|AAAAAAAAKMKABAAA|68297|18|58|17|PM|second|evening|dinner| +68298|AAAAAAAALMKABAAA|68298|18|58|18|PM|second|evening|dinner| +68299|AAAAAAAAMMKABAAA|68299|18|58|19|PM|second|evening|dinner| +68300|AAAAAAAANMKABAAA|68300|18|58|20|PM|second|evening|dinner| +68301|AAAAAAAAOMKABAAA|68301|18|58|21|PM|second|evening|dinner| +68302|AAAAAAAAPMKABAAA|68302|18|58|22|PM|second|evening|dinner| +68303|AAAAAAAAANKABAAA|68303|18|58|23|PM|second|evening|dinner| +68304|AAAAAAAABNKABAAA|68304|18|58|24|PM|second|evening|dinner| +68305|AAAAAAAACNKABAAA|68305|18|58|25|PM|second|evening|dinner| +68306|AAAAAAAADNKABAAA|68306|18|58|26|PM|second|evening|dinner| +68307|AAAAAAAAENKABAAA|68307|18|58|27|PM|second|evening|dinner| +68308|AAAAAAAAFNKABAAA|68308|18|58|28|PM|second|evening|dinner| +68309|AAAAAAAAGNKABAAA|68309|18|58|29|PM|second|evening|dinner| +68310|AAAAAAAAHNKABAAA|68310|18|58|30|PM|second|evening|dinner| +68311|AAAAAAAAINKABAAA|68311|18|58|31|PM|second|evening|dinner| +68312|AAAAAAAAJNKABAAA|68312|18|58|32|PM|second|evening|dinner| +68313|AAAAAAAAKNKABAAA|68313|18|58|33|PM|second|evening|dinner| +68314|AAAAAAAALNKABAAA|68314|18|58|34|PM|second|evening|dinner| +68315|AAAAAAAAMNKABAAA|68315|18|58|35|PM|second|evening|dinner| +68316|AAAAAAAANNKABAAA|68316|18|58|36|PM|second|evening|dinner| +68317|AAAAAAAAONKABAAA|68317|18|58|37|PM|second|evening|dinner| +68318|AAAAAAAAPNKABAAA|68318|18|58|38|PM|second|evening|dinner| +68319|AAAAAAAAAOKABAAA|68319|18|58|39|PM|second|evening|dinner| +68320|AAAAAAAABOKABAAA|68320|18|58|40|PM|second|evening|dinner| +68321|AAAAAAAACOKABAAA|68321|18|58|41|PM|second|evening|dinner| +68322|AAAAAAAADOKABAAA|68322|18|58|42|PM|second|evening|dinner| +68323|AAAAAAAAEOKABAAA|68323|18|58|43|PM|second|evening|dinner| +68324|AAAAAAAAFOKABAAA|68324|18|58|44|PM|second|evening|dinner| +68325|AAAAAAAAGOKABAAA|68325|18|58|45|PM|second|evening|dinner| +68326|AAAAAAAAHOKABAAA|68326|18|58|46|PM|second|evening|dinner| +68327|AAAAAAAAIOKABAAA|68327|18|58|47|PM|second|evening|dinner| +68328|AAAAAAAAJOKABAAA|68328|18|58|48|PM|second|evening|dinner| +68329|AAAAAAAAKOKABAAA|68329|18|58|49|PM|second|evening|dinner| +68330|AAAAAAAALOKABAAA|68330|18|58|50|PM|second|evening|dinner| +68331|AAAAAAAAMOKABAAA|68331|18|58|51|PM|second|evening|dinner| +68332|AAAAAAAANOKABAAA|68332|18|58|52|PM|second|evening|dinner| +68333|AAAAAAAAOOKABAAA|68333|18|58|53|PM|second|evening|dinner| +68334|AAAAAAAAPOKABAAA|68334|18|58|54|PM|second|evening|dinner| +68335|AAAAAAAAAPKABAAA|68335|18|58|55|PM|second|evening|dinner| +68336|AAAAAAAABPKABAAA|68336|18|58|56|PM|second|evening|dinner| +68337|AAAAAAAACPKABAAA|68337|18|58|57|PM|second|evening|dinner| +68338|AAAAAAAADPKABAAA|68338|18|58|58|PM|second|evening|dinner| +68339|AAAAAAAAEPKABAAA|68339|18|58|59|PM|second|evening|dinner| +68340|AAAAAAAAFPKABAAA|68340|18|59|0|PM|second|evening|dinner| +68341|AAAAAAAAGPKABAAA|68341|18|59|1|PM|second|evening|dinner| +68342|AAAAAAAAHPKABAAA|68342|18|59|2|PM|second|evening|dinner| +68343|AAAAAAAAIPKABAAA|68343|18|59|3|PM|second|evening|dinner| +68344|AAAAAAAAJPKABAAA|68344|18|59|4|PM|second|evening|dinner| +68345|AAAAAAAAKPKABAAA|68345|18|59|5|PM|second|evening|dinner| +68346|AAAAAAAALPKABAAA|68346|18|59|6|PM|second|evening|dinner| +68347|AAAAAAAAMPKABAAA|68347|18|59|7|PM|second|evening|dinner| +68348|AAAAAAAANPKABAAA|68348|18|59|8|PM|second|evening|dinner| +68349|AAAAAAAAOPKABAAA|68349|18|59|9|PM|second|evening|dinner| +68350|AAAAAAAAPPKABAAA|68350|18|59|10|PM|second|evening|dinner| +68351|AAAAAAAAAALABAAA|68351|18|59|11|PM|second|evening|dinner| +68352|AAAAAAAABALABAAA|68352|18|59|12|PM|second|evening|dinner| +68353|AAAAAAAACALABAAA|68353|18|59|13|PM|second|evening|dinner| +68354|AAAAAAAADALABAAA|68354|18|59|14|PM|second|evening|dinner| +68355|AAAAAAAAEALABAAA|68355|18|59|15|PM|second|evening|dinner| +68356|AAAAAAAAFALABAAA|68356|18|59|16|PM|second|evening|dinner| +68357|AAAAAAAAGALABAAA|68357|18|59|17|PM|second|evening|dinner| +68358|AAAAAAAAHALABAAA|68358|18|59|18|PM|second|evening|dinner| +68359|AAAAAAAAIALABAAA|68359|18|59|19|PM|second|evening|dinner| +68360|AAAAAAAAJALABAAA|68360|18|59|20|PM|second|evening|dinner| +68361|AAAAAAAAKALABAAA|68361|18|59|21|PM|second|evening|dinner| +68362|AAAAAAAALALABAAA|68362|18|59|22|PM|second|evening|dinner| +68363|AAAAAAAAMALABAAA|68363|18|59|23|PM|second|evening|dinner| +68364|AAAAAAAANALABAAA|68364|18|59|24|PM|second|evening|dinner| +68365|AAAAAAAAOALABAAA|68365|18|59|25|PM|second|evening|dinner| +68366|AAAAAAAAPALABAAA|68366|18|59|26|PM|second|evening|dinner| +68367|AAAAAAAAABLABAAA|68367|18|59|27|PM|second|evening|dinner| +68368|AAAAAAAABBLABAAA|68368|18|59|28|PM|second|evening|dinner| +68369|AAAAAAAACBLABAAA|68369|18|59|29|PM|second|evening|dinner| +68370|AAAAAAAADBLABAAA|68370|18|59|30|PM|second|evening|dinner| +68371|AAAAAAAAEBLABAAA|68371|18|59|31|PM|second|evening|dinner| +68372|AAAAAAAAFBLABAAA|68372|18|59|32|PM|second|evening|dinner| +68373|AAAAAAAAGBLABAAA|68373|18|59|33|PM|second|evening|dinner| +68374|AAAAAAAAHBLABAAA|68374|18|59|34|PM|second|evening|dinner| +68375|AAAAAAAAIBLABAAA|68375|18|59|35|PM|second|evening|dinner| +68376|AAAAAAAAJBLABAAA|68376|18|59|36|PM|second|evening|dinner| +68377|AAAAAAAAKBLABAAA|68377|18|59|37|PM|second|evening|dinner| +68378|AAAAAAAALBLABAAA|68378|18|59|38|PM|second|evening|dinner| +68379|AAAAAAAAMBLABAAA|68379|18|59|39|PM|second|evening|dinner| +68380|AAAAAAAANBLABAAA|68380|18|59|40|PM|second|evening|dinner| +68381|AAAAAAAAOBLABAAA|68381|18|59|41|PM|second|evening|dinner| +68382|AAAAAAAAPBLABAAA|68382|18|59|42|PM|second|evening|dinner| +68383|AAAAAAAAACLABAAA|68383|18|59|43|PM|second|evening|dinner| +68384|AAAAAAAABCLABAAA|68384|18|59|44|PM|second|evening|dinner| +68385|AAAAAAAACCLABAAA|68385|18|59|45|PM|second|evening|dinner| +68386|AAAAAAAADCLABAAA|68386|18|59|46|PM|second|evening|dinner| +68387|AAAAAAAAECLABAAA|68387|18|59|47|PM|second|evening|dinner| +68388|AAAAAAAAFCLABAAA|68388|18|59|48|PM|second|evening|dinner| +68389|AAAAAAAAGCLABAAA|68389|18|59|49|PM|second|evening|dinner| +68390|AAAAAAAAHCLABAAA|68390|18|59|50|PM|second|evening|dinner| +68391|AAAAAAAAICLABAAA|68391|18|59|51|PM|second|evening|dinner| +68392|AAAAAAAAJCLABAAA|68392|18|59|52|PM|second|evening|dinner| +68393|AAAAAAAAKCLABAAA|68393|18|59|53|PM|second|evening|dinner| +68394|AAAAAAAALCLABAAA|68394|18|59|54|PM|second|evening|dinner| +68395|AAAAAAAAMCLABAAA|68395|18|59|55|PM|second|evening|dinner| +68396|AAAAAAAANCLABAAA|68396|18|59|56|PM|second|evening|dinner| +68397|AAAAAAAAOCLABAAA|68397|18|59|57|PM|second|evening|dinner| +68398|AAAAAAAAPCLABAAA|68398|18|59|58|PM|second|evening|dinner| +68399|AAAAAAAAADLABAAA|68399|18|59|59|PM|second|evening|dinner| +68400|AAAAAAAABDLABAAA|68400|19|0|0|PM|second|evening|dinner| +68401|AAAAAAAACDLABAAA|68401|19|0|1|PM|second|evening|dinner| +68402|AAAAAAAADDLABAAA|68402|19|0|2|PM|second|evening|dinner| +68403|AAAAAAAAEDLABAAA|68403|19|0|3|PM|second|evening|dinner| +68404|AAAAAAAAFDLABAAA|68404|19|0|4|PM|second|evening|dinner| +68405|AAAAAAAAGDLABAAA|68405|19|0|5|PM|second|evening|dinner| +68406|AAAAAAAAHDLABAAA|68406|19|0|6|PM|second|evening|dinner| +68407|AAAAAAAAIDLABAAA|68407|19|0|7|PM|second|evening|dinner| +68408|AAAAAAAAJDLABAAA|68408|19|0|8|PM|second|evening|dinner| +68409|AAAAAAAAKDLABAAA|68409|19|0|9|PM|second|evening|dinner| +68410|AAAAAAAALDLABAAA|68410|19|0|10|PM|second|evening|dinner| +68411|AAAAAAAAMDLABAAA|68411|19|0|11|PM|second|evening|dinner| +68412|AAAAAAAANDLABAAA|68412|19|0|12|PM|second|evening|dinner| +68413|AAAAAAAAODLABAAA|68413|19|0|13|PM|second|evening|dinner| +68414|AAAAAAAAPDLABAAA|68414|19|0|14|PM|second|evening|dinner| +68415|AAAAAAAAAELABAAA|68415|19|0|15|PM|second|evening|dinner| +68416|AAAAAAAABELABAAA|68416|19|0|16|PM|second|evening|dinner| +68417|AAAAAAAACELABAAA|68417|19|0|17|PM|second|evening|dinner| +68418|AAAAAAAADELABAAA|68418|19|0|18|PM|second|evening|dinner| +68419|AAAAAAAAEELABAAA|68419|19|0|19|PM|second|evening|dinner| +68420|AAAAAAAAFELABAAA|68420|19|0|20|PM|second|evening|dinner| +68421|AAAAAAAAGELABAAA|68421|19|0|21|PM|second|evening|dinner| +68422|AAAAAAAAHELABAAA|68422|19|0|22|PM|second|evening|dinner| +68423|AAAAAAAAIELABAAA|68423|19|0|23|PM|second|evening|dinner| +68424|AAAAAAAAJELABAAA|68424|19|0|24|PM|second|evening|dinner| +68425|AAAAAAAAKELABAAA|68425|19|0|25|PM|second|evening|dinner| +68426|AAAAAAAALELABAAA|68426|19|0|26|PM|second|evening|dinner| +68427|AAAAAAAAMELABAAA|68427|19|0|27|PM|second|evening|dinner| +68428|AAAAAAAANELABAAA|68428|19|0|28|PM|second|evening|dinner| +68429|AAAAAAAAOELABAAA|68429|19|0|29|PM|second|evening|dinner| +68430|AAAAAAAAPELABAAA|68430|19|0|30|PM|second|evening|dinner| +68431|AAAAAAAAAFLABAAA|68431|19|0|31|PM|second|evening|dinner| +68432|AAAAAAAABFLABAAA|68432|19|0|32|PM|second|evening|dinner| +68433|AAAAAAAACFLABAAA|68433|19|0|33|PM|second|evening|dinner| +68434|AAAAAAAADFLABAAA|68434|19|0|34|PM|second|evening|dinner| +68435|AAAAAAAAEFLABAAA|68435|19|0|35|PM|second|evening|dinner| +68436|AAAAAAAAFFLABAAA|68436|19|0|36|PM|second|evening|dinner| +68437|AAAAAAAAGFLABAAA|68437|19|0|37|PM|second|evening|dinner| +68438|AAAAAAAAHFLABAAA|68438|19|0|38|PM|second|evening|dinner| +68439|AAAAAAAAIFLABAAA|68439|19|0|39|PM|second|evening|dinner| +68440|AAAAAAAAJFLABAAA|68440|19|0|40|PM|second|evening|dinner| +68441|AAAAAAAAKFLABAAA|68441|19|0|41|PM|second|evening|dinner| +68442|AAAAAAAALFLABAAA|68442|19|0|42|PM|second|evening|dinner| +68443|AAAAAAAAMFLABAAA|68443|19|0|43|PM|second|evening|dinner| +68444|AAAAAAAANFLABAAA|68444|19|0|44|PM|second|evening|dinner| +68445|AAAAAAAAOFLABAAA|68445|19|0|45|PM|second|evening|dinner| +68446|AAAAAAAAPFLABAAA|68446|19|0|46|PM|second|evening|dinner| +68447|AAAAAAAAAGLABAAA|68447|19|0|47|PM|second|evening|dinner| +68448|AAAAAAAABGLABAAA|68448|19|0|48|PM|second|evening|dinner| +68449|AAAAAAAACGLABAAA|68449|19|0|49|PM|second|evening|dinner| +68450|AAAAAAAADGLABAAA|68450|19|0|50|PM|second|evening|dinner| +68451|AAAAAAAAEGLABAAA|68451|19|0|51|PM|second|evening|dinner| +68452|AAAAAAAAFGLABAAA|68452|19|0|52|PM|second|evening|dinner| +68453|AAAAAAAAGGLABAAA|68453|19|0|53|PM|second|evening|dinner| +68454|AAAAAAAAHGLABAAA|68454|19|0|54|PM|second|evening|dinner| +68455|AAAAAAAAIGLABAAA|68455|19|0|55|PM|second|evening|dinner| +68456|AAAAAAAAJGLABAAA|68456|19|0|56|PM|second|evening|dinner| +68457|AAAAAAAAKGLABAAA|68457|19|0|57|PM|second|evening|dinner| +68458|AAAAAAAALGLABAAA|68458|19|0|58|PM|second|evening|dinner| +68459|AAAAAAAAMGLABAAA|68459|19|0|59|PM|second|evening|dinner| +68460|AAAAAAAANGLABAAA|68460|19|1|0|PM|second|evening|dinner| +68461|AAAAAAAAOGLABAAA|68461|19|1|1|PM|second|evening|dinner| +68462|AAAAAAAAPGLABAAA|68462|19|1|2|PM|second|evening|dinner| +68463|AAAAAAAAAHLABAAA|68463|19|1|3|PM|second|evening|dinner| +68464|AAAAAAAABHLABAAA|68464|19|1|4|PM|second|evening|dinner| +68465|AAAAAAAACHLABAAA|68465|19|1|5|PM|second|evening|dinner| +68466|AAAAAAAADHLABAAA|68466|19|1|6|PM|second|evening|dinner| +68467|AAAAAAAAEHLABAAA|68467|19|1|7|PM|second|evening|dinner| +68468|AAAAAAAAFHLABAAA|68468|19|1|8|PM|second|evening|dinner| +68469|AAAAAAAAGHLABAAA|68469|19|1|9|PM|second|evening|dinner| +68470|AAAAAAAAHHLABAAA|68470|19|1|10|PM|second|evening|dinner| +68471|AAAAAAAAIHLABAAA|68471|19|1|11|PM|second|evening|dinner| +68472|AAAAAAAAJHLABAAA|68472|19|1|12|PM|second|evening|dinner| +68473|AAAAAAAAKHLABAAA|68473|19|1|13|PM|second|evening|dinner| +68474|AAAAAAAALHLABAAA|68474|19|1|14|PM|second|evening|dinner| +68475|AAAAAAAAMHLABAAA|68475|19|1|15|PM|second|evening|dinner| +68476|AAAAAAAANHLABAAA|68476|19|1|16|PM|second|evening|dinner| +68477|AAAAAAAAOHLABAAA|68477|19|1|17|PM|second|evening|dinner| +68478|AAAAAAAAPHLABAAA|68478|19|1|18|PM|second|evening|dinner| +68479|AAAAAAAAAILABAAA|68479|19|1|19|PM|second|evening|dinner| +68480|AAAAAAAABILABAAA|68480|19|1|20|PM|second|evening|dinner| +68481|AAAAAAAACILABAAA|68481|19|1|21|PM|second|evening|dinner| +68482|AAAAAAAADILABAAA|68482|19|1|22|PM|second|evening|dinner| +68483|AAAAAAAAEILABAAA|68483|19|1|23|PM|second|evening|dinner| +68484|AAAAAAAAFILABAAA|68484|19|1|24|PM|second|evening|dinner| +68485|AAAAAAAAGILABAAA|68485|19|1|25|PM|second|evening|dinner| +68486|AAAAAAAAHILABAAA|68486|19|1|26|PM|second|evening|dinner| +68487|AAAAAAAAIILABAAA|68487|19|1|27|PM|second|evening|dinner| +68488|AAAAAAAAJILABAAA|68488|19|1|28|PM|second|evening|dinner| +68489|AAAAAAAAKILABAAA|68489|19|1|29|PM|second|evening|dinner| +68490|AAAAAAAALILABAAA|68490|19|1|30|PM|second|evening|dinner| +68491|AAAAAAAAMILABAAA|68491|19|1|31|PM|second|evening|dinner| +68492|AAAAAAAANILABAAA|68492|19|1|32|PM|second|evening|dinner| +68493|AAAAAAAAOILABAAA|68493|19|1|33|PM|second|evening|dinner| +68494|AAAAAAAAPILABAAA|68494|19|1|34|PM|second|evening|dinner| +68495|AAAAAAAAAJLABAAA|68495|19|1|35|PM|second|evening|dinner| +68496|AAAAAAAABJLABAAA|68496|19|1|36|PM|second|evening|dinner| +68497|AAAAAAAACJLABAAA|68497|19|1|37|PM|second|evening|dinner| +68498|AAAAAAAADJLABAAA|68498|19|1|38|PM|second|evening|dinner| +68499|AAAAAAAAEJLABAAA|68499|19|1|39|PM|second|evening|dinner| +68500|AAAAAAAAFJLABAAA|68500|19|1|40|PM|second|evening|dinner| +68501|AAAAAAAAGJLABAAA|68501|19|1|41|PM|second|evening|dinner| +68502|AAAAAAAAHJLABAAA|68502|19|1|42|PM|second|evening|dinner| +68503|AAAAAAAAIJLABAAA|68503|19|1|43|PM|second|evening|dinner| +68504|AAAAAAAAJJLABAAA|68504|19|1|44|PM|second|evening|dinner| +68505|AAAAAAAAKJLABAAA|68505|19|1|45|PM|second|evening|dinner| +68506|AAAAAAAALJLABAAA|68506|19|1|46|PM|second|evening|dinner| +68507|AAAAAAAAMJLABAAA|68507|19|1|47|PM|second|evening|dinner| +68508|AAAAAAAANJLABAAA|68508|19|1|48|PM|second|evening|dinner| +68509|AAAAAAAAOJLABAAA|68509|19|1|49|PM|second|evening|dinner| +68510|AAAAAAAAPJLABAAA|68510|19|1|50|PM|second|evening|dinner| +68511|AAAAAAAAAKLABAAA|68511|19|1|51|PM|second|evening|dinner| +68512|AAAAAAAABKLABAAA|68512|19|1|52|PM|second|evening|dinner| +68513|AAAAAAAACKLABAAA|68513|19|1|53|PM|second|evening|dinner| +68514|AAAAAAAADKLABAAA|68514|19|1|54|PM|second|evening|dinner| +68515|AAAAAAAAEKLABAAA|68515|19|1|55|PM|second|evening|dinner| +68516|AAAAAAAAFKLABAAA|68516|19|1|56|PM|second|evening|dinner| +68517|AAAAAAAAGKLABAAA|68517|19|1|57|PM|second|evening|dinner| +68518|AAAAAAAAHKLABAAA|68518|19|1|58|PM|second|evening|dinner| +68519|AAAAAAAAIKLABAAA|68519|19|1|59|PM|second|evening|dinner| +68520|AAAAAAAAJKLABAAA|68520|19|2|0|PM|second|evening|dinner| +68521|AAAAAAAAKKLABAAA|68521|19|2|1|PM|second|evening|dinner| +68522|AAAAAAAALKLABAAA|68522|19|2|2|PM|second|evening|dinner| +68523|AAAAAAAAMKLABAAA|68523|19|2|3|PM|second|evening|dinner| +68524|AAAAAAAANKLABAAA|68524|19|2|4|PM|second|evening|dinner| +68525|AAAAAAAAOKLABAAA|68525|19|2|5|PM|second|evening|dinner| +68526|AAAAAAAAPKLABAAA|68526|19|2|6|PM|second|evening|dinner| +68527|AAAAAAAAALLABAAA|68527|19|2|7|PM|second|evening|dinner| +68528|AAAAAAAABLLABAAA|68528|19|2|8|PM|second|evening|dinner| +68529|AAAAAAAACLLABAAA|68529|19|2|9|PM|second|evening|dinner| +68530|AAAAAAAADLLABAAA|68530|19|2|10|PM|second|evening|dinner| +68531|AAAAAAAAELLABAAA|68531|19|2|11|PM|second|evening|dinner| +68532|AAAAAAAAFLLABAAA|68532|19|2|12|PM|second|evening|dinner| +68533|AAAAAAAAGLLABAAA|68533|19|2|13|PM|second|evening|dinner| +68534|AAAAAAAAHLLABAAA|68534|19|2|14|PM|second|evening|dinner| +68535|AAAAAAAAILLABAAA|68535|19|2|15|PM|second|evening|dinner| +68536|AAAAAAAAJLLABAAA|68536|19|2|16|PM|second|evening|dinner| +68537|AAAAAAAAKLLABAAA|68537|19|2|17|PM|second|evening|dinner| +68538|AAAAAAAALLLABAAA|68538|19|2|18|PM|second|evening|dinner| +68539|AAAAAAAAMLLABAAA|68539|19|2|19|PM|second|evening|dinner| +68540|AAAAAAAANLLABAAA|68540|19|2|20|PM|second|evening|dinner| +68541|AAAAAAAAOLLABAAA|68541|19|2|21|PM|second|evening|dinner| +68542|AAAAAAAAPLLABAAA|68542|19|2|22|PM|second|evening|dinner| +68543|AAAAAAAAAMLABAAA|68543|19|2|23|PM|second|evening|dinner| +68544|AAAAAAAABMLABAAA|68544|19|2|24|PM|second|evening|dinner| +68545|AAAAAAAACMLABAAA|68545|19|2|25|PM|second|evening|dinner| +68546|AAAAAAAADMLABAAA|68546|19|2|26|PM|second|evening|dinner| +68547|AAAAAAAAEMLABAAA|68547|19|2|27|PM|second|evening|dinner| +68548|AAAAAAAAFMLABAAA|68548|19|2|28|PM|second|evening|dinner| +68549|AAAAAAAAGMLABAAA|68549|19|2|29|PM|second|evening|dinner| +68550|AAAAAAAAHMLABAAA|68550|19|2|30|PM|second|evening|dinner| +68551|AAAAAAAAIMLABAAA|68551|19|2|31|PM|second|evening|dinner| +68552|AAAAAAAAJMLABAAA|68552|19|2|32|PM|second|evening|dinner| +68553|AAAAAAAAKMLABAAA|68553|19|2|33|PM|second|evening|dinner| +68554|AAAAAAAALMLABAAA|68554|19|2|34|PM|second|evening|dinner| +68555|AAAAAAAAMMLABAAA|68555|19|2|35|PM|second|evening|dinner| +68556|AAAAAAAANMLABAAA|68556|19|2|36|PM|second|evening|dinner| +68557|AAAAAAAAOMLABAAA|68557|19|2|37|PM|second|evening|dinner| +68558|AAAAAAAAPMLABAAA|68558|19|2|38|PM|second|evening|dinner| +68559|AAAAAAAAANLABAAA|68559|19|2|39|PM|second|evening|dinner| +68560|AAAAAAAABNLABAAA|68560|19|2|40|PM|second|evening|dinner| +68561|AAAAAAAACNLABAAA|68561|19|2|41|PM|second|evening|dinner| +68562|AAAAAAAADNLABAAA|68562|19|2|42|PM|second|evening|dinner| +68563|AAAAAAAAENLABAAA|68563|19|2|43|PM|second|evening|dinner| +68564|AAAAAAAAFNLABAAA|68564|19|2|44|PM|second|evening|dinner| +68565|AAAAAAAAGNLABAAA|68565|19|2|45|PM|second|evening|dinner| +68566|AAAAAAAAHNLABAAA|68566|19|2|46|PM|second|evening|dinner| +68567|AAAAAAAAINLABAAA|68567|19|2|47|PM|second|evening|dinner| +68568|AAAAAAAAJNLABAAA|68568|19|2|48|PM|second|evening|dinner| +68569|AAAAAAAAKNLABAAA|68569|19|2|49|PM|second|evening|dinner| +68570|AAAAAAAALNLABAAA|68570|19|2|50|PM|second|evening|dinner| +68571|AAAAAAAAMNLABAAA|68571|19|2|51|PM|second|evening|dinner| +68572|AAAAAAAANNLABAAA|68572|19|2|52|PM|second|evening|dinner| +68573|AAAAAAAAONLABAAA|68573|19|2|53|PM|second|evening|dinner| +68574|AAAAAAAAPNLABAAA|68574|19|2|54|PM|second|evening|dinner| +68575|AAAAAAAAAOLABAAA|68575|19|2|55|PM|second|evening|dinner| +68576|AAAAAAAABOLABAAA|68576|19|2|56|PM|second|evening|dinner| +68577|AAAAAAAACOLABAAA|68577|19|2|57|PM|second|evening|dinner| +68578|AAAAAAAADOLABAAA|68578|19|2|58|PM|second|evening|dinner| +68579|AAAAAAAAEOLABAAA|68579|19|2|59|PM|second|evening|dinner| +68580|AAAAAAAAFOLABAAA|68580|19|3|0|PM|second|evening|dinner| +68581|AAAAAAAAGOLABAAA|68581|19|3|1|PM|second|evening|dinner| +68582|AAAAAAAAHOLABAAA|68582|19|3|2|PM|second|evening|dinner| +68583|AAAAAAAAIOLABAAA|68583|19|3|3|PM|second|evening|dinner| +68584|AAAAAAAAJOLABAAA|68584|19|3|4|PM|second|evening|dinner| +68585|AAAAAAAAKOLABAAA|68585|19|3|5|PM|second|evening|dinner| +68586|AAAAAAAALOLABAAA|68586|19|3|6|PM|second|evening|dinner| +68587|AAAAAAAAMOLABAAA|68587|19|3|7|PM|second|evening|dinner| +68588|AAAAAAAANOLABAAA|68588|19|3|8|PM|second|evening|dinner| +68589|AAAAAAAAOOLABAAA|68589|19|3|9|PM|second|evening|dinner| +68590|AAAAAAAAPOLABAAA|68590|19|3|10|PM|second|evening|dinner| +68591|AAAAAAAAAPLABAAA|68591|19|3|11|PM|second|evening|dinner| +68592|AAAAAAAABPLABAAA|68592|19|3|12|PM|second|evening|dinner| +68593|AAAAAAAACPLABAAA|68593|19|3|13|PM|second|evening|dinner| +68594|AAAAAAAADPLABAAA|68594|19|3|14|PM|second|evening|dinner| +68595|AAAAAAAAEPLABAAA|68595|19|3|15|PM|second|evening|dinner| +68596|AAAAAAAAFPLABAAA|68596|19|3|16|PM|second|evening|dinner| +68597|AAAAAAAAGPLABAAA|68597|19|3|17|PM|second|evening|dinner| +68598|AAAAAAAAHPLABAAA|68598|19|3|18|PM|second|evening|dinner| +68599|AAAAAAAAIPLABAAA|68599|19|3|19|PM|second|evening|dinner| +68600|AAAAAAAAJPLABAAA|68600|19|3|20|PM|second|evening|dinner| +68601|AAAAAAAAKPLABAAA|68601|19|3|21|PM|second|evening|dinner| +68602|AAAAAAAALPLABAAA|68602|19|3|22|PM|second|evening|dinner| +68603|AAAAAAAAMPLABAAA|68603|19|3|23|PM|second|evening|dinner| +68604|AAAAAAAANPLABAAA|68604|19|3|24|PM|second|evening|dinner| +68605|AAAAAAAAOPLABAAA|68605|19|3|25|PM|second|evening|dinner| +68606|AAAAAAAAPPLABAAA|68606|19|3|26|PM|second|evening|dinner| +68607|AAAAAAAAAAMABAAA|68607|19|3|27|PM|second|evening|dinner| +68608|AAAAAAAABAMABAAA|68608|19|3|28|PM|second|evening|dinner| +68609|AAAAAAAACAMABAAA|68609|19|3|29|PM|second|evening|dinner| +68610|AAAAAAAADAMABAAA|68610|19|3|30|PM|second|evening|dinner| +68611|AAAAAAAAEAMABAAA|68611|19|3|31|PM|second|evening|dinner| +68612|AAAAAAAAFAMABAAA|68612|19|3|32|PM|second|evening|dinner| +68613|AAAAAAAAGAMABAAA|68613|19|3|33|PM|second|evening|dinner| +68614|AAAAAAAAHAMABAAA|68614|19|3|34|PM|second|evening|dinner| +68615|AAAAAAAAIAMABAAA|68615|19|3|35|PM|second|evening|dinner| +68616|AAAAAAAAJAMABAAA|68616|19|3|36|PM|second|evening|dinner| +68617|AAAAAAAAKAMABAAA|68617|19|3|37|PM|second|evening|dinner| +68618|AAAAAAAALAMABAAA|68618|19|3|38|PM|second|evening|dinner| +68619|AAAAAAAAMAMABAAA|68619|19|3|39|PM|second|evening|dinner| +68620|AAAAAAAANAMABAAA|68620|19|3|40|PM|second|evening|dinner| +68621|AAAAAAAAOAMABAAA|68621|19|3|41|PM|second|evening|dinner| +68622|AAAAAAAAPAMABAAA|68622|19|3|42|PM|second|evening|dinner| +68623|AAAAAAAAABMABAAA|68623|19|3|43|PM|second|evening|dinner| +68624|AAAAAAAABBMABAAA|68624|19|3|44|PM|second|evening|dinner| +68625|AAAAAAAACBMABAAA|68625|19|3|45|PM|second|evening|dinner| +68626|AAAAAAAADBMABAAA|68626|19|3|46|PM|second|evening|dinner| +68627|AAAAAAAAEBMABAAA|68627|19|3|47|PM|second|evening|dinner| +68628|AAAAAAAAFBMABAAA|68628|19|3|48|PM|second|evening|dinner| +68629|AAAAAAAAGBMABAAA|68629|19|3|49|PM|second|evening|dinner| +68630|AAAAAAAAHBMABAAA|68630|19|3|50|PM|second|evening|dinner| +68631|AAAAAAAAIBMABAAA|68631|19|3|51|PM|second|evening|dinner| +68632|AAAAAAAAJBMABAAA|68632|19|3|52|PM|second|evening|dinner| +68633|AAAAAAAAKBMABAAA|68633|19|3|53|PM|second|evening|dinner| +68634|AAAAAAAALBMABAAA|68634|19|3|54|PM|second|evening|dinner| +68635|AAAAAAAAMBMABAAA|68635|19|3|55|PM|second|evening|dinner| +68636|AAAAAAAANBMABAAA|68636|19|3|56|PM|second|evening|dinner| +68637|AAAAAAAAOBMABAAA|68637|19|3|57|PM|second|evening|dinner| +68638|AAAAAAAAPBMABAAA|68638|19|3|58|PM|second|evening|dinner| +68639|AAAAAAAAACMABAAA|68639|19|3|59|PM|second|evening|dinner| +68640|AAAAAAAABCMABAAA|68640|19|4|0|PM|second|evening|dinner| +68641|AAAAAAAACCMABAAA|68641|19|4|1|PM|second|evening|dinner| +68642|AAAAAAAADCMABAAA|68642|19|4|2|PM|second|evening|dinner| +68643|AAAAAAAAECMABAAA|68643|19|4|3|PM|second|evening|dinner| +68644|AAAAAAAAFCMABAAA|68644|19|4|4|PM|second|evening|dinner| +68645|AAAAAAAAGCMABAAA|68645|19|4|5|PM|second|evening|dinner| +68646|AAAAAAAAHCMABAAA|68646|19|4|6|PM|second|evening|dinner| +68647|AAAAAAAAICMABAAA|68647|19|4|7|PM|second|evening|dinner| +68648|AAAAAAAAJCMABAAA|68648|19|4|8|PM|second|evening|dinner| +68649|AAAAAAAAKCMABAAA|68649|19|4|9|PM|second|evening|dinner| +68650|AAAAAAAALCMABAAA|68650|19|4|10|PM|second|evening|dinner| +68651|AAAAAAAAMCMABAAA|68651|19|4|11|PM|second|evening|dinner| +68652|AAAAAAAANCMABAAA|68652|19|4|12|PM|second|evening|dinner| +68653|AAAAAAAAOCMABAAA|68653|19|4|13|PM|second|evening|dinner| +68654|AAAAAAAAPCMABAAA|68654|19|4|14|PM|second|evening|dinner| +68655|AAAAAAAAADMABAAA|68655|19|4|15|PM|second|evening|dinner| +68656|AAAAAAAABDMABAAA|68656|19|4|16|PM|second|evening|dinner| +68657|AAAAAAAACDMABAAA|68657|19|4|17|PM|second|evening|dinner| +68658|AAAAAAAADDMABAAA|68658|19|4|18|PM|second|evening|dinner| +68659|AAAAAAAAEDMABAAA|68659|19|4|19|PM|second|evening|dinner| +68660|AAAAAAAAFDMABAAA|68660|19|4|20|PM|second|evening|dinner| +68661|AAAAAAAAGDMABAAA|68661|19|4|21|PM|second|evening|dinner| +68662|AAAAAAAAHDMABAAA|68662|19|4|22|PM|second|evening|dinner| +68663|AAAAAAAAIDMABAAA|68663|19|4|23|PM|second|evening|dinner| +68664|AAAAAAAAJDMABAAA|68664|19|4|24|PM|second|evening|dinner| +68665|AAAAAAAAKDMABAAA|68665|19|4|25|PM|second|evening|dinner| +68666|AAAAAAAALDMABAAA|68666|19|4|26|PM|second|evening|dinner| +68667|AAAAAAAAMDMABAAA|68667|19|4|27|PM|second|evening|dinner| +68668|AAAAAAAANDMABAAA|68668|19|4|28|PM|second|evening|dinner| +68669|AAAAAAAAODMABAAA|68669|19|4|29|PM|second|evening|dinner| +68670|AAAAAAAAPDMABAAA|68670|19|4|30|PM|second|evening|dinner| +68671|AAAAAAAAAEMABAAA|68671|19|4|31|PM|second|evening|dinner| +68672|AAAAAAAABEMABAAA|68672|19|4|32|PM|second|evening|dinner| +68673|AAAAAAAACEMABAAA|68673|19|4|33|PM|second|evening|dinner| +68674|AAAAAAAADEMABAAA|68674|19|4|34|PM|second|evening|dinner| +68675|AAAAAAAAEEMABAAA|68675|19|4|35|PM|second|evening|dinner| +68676|AAAAAAAAFEMABAAA|68676|19|4|36|PM|second|evening|dinner| +68677|AAAAAAAAGEMABAAA|68677|19|4|37|PM|second|evening|dinner| +68678|AAAAAAAAHEMABAAA|68678|19|4|38|PM|second|evening|dinner| +68679|AAAAAAAAIEMABAAA|68679|19|4|39|PM|second|evening|dinner| +68680|AAAAAAAAJEMABAAA|68680|19|4|40|PM|second|evening|dinner| +68681|AAAAAAAAKEMABAAA|68681|19|4|41|PM|second|evening|dinner| +68682|AAAAAAAALEMABAAA|68682|19|4|42|PM|second|evening|dinner| +68683|AAAAAAAAMEMABAAA|68683|19|4|43|PM|second|evening|dinner| +68684|AAAAAAAANEMABAAA|68684|19|4|44|PM|second|evening|dinner| +68685|AAAAAAAAOEMABAAA|68685|19|4|45|PM|second|evening|dinner| +68686|AAAAAAAAPEMABAAA|68686|19|4|46|PM|second|evening|dinner| +68687|AAAAAAAAAFMABAAA|68687|19|4|47|PM|second|evening|dinner| +68688|AAAAAAAABFMABAAA|68688|19|4|48|PM|second|evening|dinner| +68689|AAAAAAAACFMABAAA|68689|19|4|49|PM|second|evening|dinner| +68690|AAAAAAAADFMABAAA|68690|19|4|50|PM|second|evening|dinner| +68691|AAAAAAAAEFMABAAA|68691|19|4|51|PM|second|evening|dinner| +68692|AAAAAAAAFFMABAAA|68692|19|4|52|PM|second|evening|dinner| +68693|AAAAAAAAGFMABAAA|68693|19|4|53|PM|second|evening|dinner| +68694|AAAAAAAAHFMABAAA|68694|19|4|54|PM|second|evening|dinner| +68695|AAAAAAAAIFMABAAA|68695|19|4|55|PM|second|evening|dinner| +68696|AAAAAAAAJFMABAAA|68696|19|4|56|PM|second|evening|dinner| +68697|AAAAAAAAKFMABAAA|68697|19|4|57|PM|second|evening|dinner| +68698|AAAAAAAALFMABAAA|68698|19|4|58|PM|second|evening|dinner| +68699|AAAAAAAAMFMABAAA|68699|19|4|59|PM|second|evening|dinner| +68700|AAAAAAAANFMABAAA|68700|19|5|0|PM|second|evening|dinner| +68701|AAAAAAAAOFMABAAA|68701|19|5|1|PM|second|evening|dinner| +68702|AAAAAAAAPFMABAAA|68702|19|5|2|PM|second|evening|dinner| +68703|AAAAAAAAAGMABAAA|68703|19|5|3|PM|second|evening|dinner| +68704|AAAAAAAABGMABAAA|68704|19|5|4|PM|second|evening|dinner| +68705|AAAAAAAACGMABAAA|68705|19|5|5|PM|second|evening|dinner| +68706|AAAAAAAADGMABAAA|68706|19|5|6|PM|second|evening|dinner| +68707|AAAAAAAAEGMABAAA|68707|19|5|7|PM|second|evening|dinner| +68708|AAAAAAAAFGMABAAA|68708|19|5|8|PM|second|evening|dinner| +68709|AAAAAAAAGGMABAAA|68709|19|5|9|PM|second|evening|dinner| +68710|AAAAAAAAHGMABAAA|68710|19|5|10|PM|second|evening|dinner| +68711|AAAAAAAAIGMABAAA|68711|19|5|11|PM|second|evening|dinner| +68712|AAAAAAAAJGMABAAA|68712|19|5|12|PM|second|evening|dinner| +68713|AAAAAAAAKGMABAAA|68713|19|5|13|PM|second|evening|dinner| +68714|AAAAAAAALGMABAAA|68714|19|5|14|PM|second|evening|dinner| +68715|AAAAAAAAMGMABAAA|68715|19|5|15|PM|second|evening|dinner| +68716|AAAAAAAANGMABAAA|68716|19|5|16|PM|second|evening|dinner| +68717|AAAAAAAAOGMABAAA|68717|19|5|17|PM|second|evening|dinner| +68718|AAAAAAAAPGMABAAA|68718|19|5|18|PM|second|evening|dinner| +68719|AAAAAAAAAHMABAAA|68719|19|5|19|PM|second|evening|dinner| +68720|AAAAAAAABHMABAAA|68720|19|5|20|PM|second|evening|dinner| +68721|AAAAAAAACHMABAAA|68721|19|5|21|PM|second|evening|dinner| +68722|AAAAAAAADHMABAAA|68722|19|5|22|PM|second|evening|dinner| +68723|AAAAAAAAEHMABAAA|68723|19|5|23|PM|second|evening|dinner| +68724|AAAAAAAAFHMABAAA|68724|19|5|24|PM|second|evening|dinner| +68725|AAAAAAAAGHMABAAA|68725|19|5|25|PM|second|evening|dinner| +68726|AAAAAAAAHHMABAAA|68726|19|5|26|PM|second|evening|dinner| +68727|AAAAAAAAIHMABAAA|68727|19|5|27|PM|second|evening|dinner| +68728|AAAAAAAAJHMABAAA|68728|19|5|28|PM|second|evening|dinner| +68729|AAAAAAAAKHMABAAA|68729|19|5|29|PM|second|evening|dinner| +68730|AAAAAAAALHMABAAA|68730|19|5|30|PM|second|evening|dinner| +68731|AAAAAAAAMHMABAAA|68731|19|5|31|PM|second|evening|dinner| +68732|AAAAAAAANHMABAAA|68732|19|5|32|PM|second|evening|dinner| +68733|AAAAAAAAOHMABAAA|68733|19|5|33|PM|second|evening|dinner| +68734|AAAAAAAAPHMABAAA|68734|19|5|34|PM|second|evening|dinner| +68735|AAAAAAAAAIMABAAA|68735|19|5|35|PM|second|evening|dinner| +68736|AAAAAAAABIMABAAA|68736|19|5|36|PM|second|evening|dinner| +68737|AAAAAAAACIMABAAA|68737|19|5|37|PM|second|evening|dinner| +68738|AAAAAAAADIMABAAA|68738|19|5|38|PM|second|evening|dinner| +68739|AAAAAAAAEIMABAAA|68739|19|5|39|PM|second|evening|dinner| +68740|AAAAAAAAFIMABAAA|68740|19|5|40|PM|second|evening|dinner| +68741|AAAAAAAAGIMABAAA|68741|19|5|41|PM|second|evening|dinner| +68742|AAAAAAAAHIMABAAA|68742|19|5|42|PM|second|evening|dinner| +68743|AAAAAAAAIIMABAAA|68743|19|5|43|PM|second|evening|dinner| +68744|AAAAAAAAJIMABAAA|68744|19|5|44|PM|second|evening|dinner| +68745|AAAAAAAAKIMABAAA|68745|19|5|45|PM|second|evening|dinner| +68746|AAAAAAAALIMABAAA|68746|19|5|46|PM|second|evening|dinner| +68747|AAAAAAAAMIMABAAA|68747|19|5|47|PM|second|evening|dinner| +68748|AAAAAAAANIMABAAA|68748|19|5|48|PM|second|evening|dinner| +68749|AAAAAAAAOIMABAAA|68749|19|5|49|PM|second|evening|dinner| +68750|AAAAAAAAPIMABAAA|68750|19|5|50|PM|second|evening|dinner| +68751|AAAAAAAAAJMABAAA|68751|19|5|51|PM|second|evening|dinner| +68752|AAAAAAAABJMABAAA|68752|19|5|52|PM|second|evening|dinner| +68753|AAAAAAAACJMABAAA|68753|19|5|53|PM|second|evening|dinner| +68754|AAAAAAAADJMABAAA|68754|19|5|54|PM|second|evening|dinner| +68755|AAAAAAAAEJMABAAA|68755|19|5|55|PM|second|evening|dinner| +68756|AAAAAAAAFJMABAAA|68756|19|5|56|PM|second|evening|dinner| +68757|AAAAAAAAGJMABAAA|68757|19|5|57|PM|second|evening|dinner| +68758|AAAAAAAAHJMABAAA|68758|19|5|58|PM|second|evening|dinner| +68759|AAAAAAAAIJMABAAA|68759|19|5|59|PM|second|evening|dinner| +68760|AAAAAAAAJJMABAAA|68760|19|6|0|PM|second|evening|dinner| +68761|AAAAAAAAKJMABAAA|68761|19|6|1|PM|second|evening|dinner| +68762|AAAAAAAALJMABAAA|68762|19|6|2|PM|second|evening|dinner| +68763|AAAAAAAAMJMABAAA|68763|19|6|3|PM|second|evening|dinner| +68764|AAAAAAAANJMABAAA|68764|19|6|4|PM|second|evening|dinner| +68765|AAAAAAAAOJMABAAA|68765|19|6|5|PM|second|evening|dinner| +68766|AAAAAAAAPJMABAAA|68766|19|6|6|PM|second|evening|dinner| +68767|AAAAAAAAAKMABAAA|68767|19|6|7|PM|second|evening|dinner| +68768|AAAAAAAABKMABAAA|68768|19|6|8|PM|second|evening|dinner| +68769|AAAAAAAACKMABAAA|68769|19|6|9|PM|second|evening|dinner| +68770|AAAAAAAADKMABAAA|68770|19|6|10|PM|second|evening|dinner| +68771|AAAAAAAAEKMABAAA|68771|19|6|11|PM|second|evening|dinner| +68772|AAAAAAAAFKMABAAA|68772|19|6|12|PM|second|evening|dinner| +68773|AAAAAAAAGKMABAAA|68773|19|6|13|PM|second|evening|dinner| +68774|AAAAAAAAHKMABAAA|68774|19|6|14|PM|second|evening|dinner| +68775|AAAAAAAAIKMABAAA|68775|19|6|15|PM|second|evening|dinner| +68776|AAAAAAAAJKMABAAA|68776|19|6|16|PM|second|evening|dinner| +68777|AAAAAAAAKKMABAAA|68777|19|6|17|PM|second|evening|dinner| +68778|AAAAAAAALKMABAAA|68778|19|6|18|PM|second|evening|dinner| +68779|AAAAAAAAMKMABAAA|68779|19|6|19|PM|second|evening|dinner| +68780|AAAAAAAANKMABAAA|68780|19|6|20|PM|second|evening|dinner| +68781|AAAAAAAAOKMABAAA|68781|19|6|21|PM|second|evening|dinner| +68782|AAAAAAAAPKMABAAA|68782|19|6|22|PM|second|evening|dinner| +68783|AAAAAAAAALMABAAA|68783|19|6|23|PM|second|evening|dinner| +68784|AAAAAAAABLMABAAA|68784|19|6|24|PM|second|evening|dinner| +68785|AAAAAAAACLMABAAA|68785|19|6|25|PM|second|evening|dinner| +68786|AAAAAAAADLMABAAA|68786|19|6|26|PM|second|evening|dinner| +68787|AAAAAAAAELMABAAA|68787|19|6|27|PM|second|evening|dinner| +68788|AAAAAAAAFLMABAAA|68788|19|6|28|PM|second|evening|dinner| +68789|AAAAAAAAGLMABAAA|68789|19|6|29|PM|second|evening|dinner| +68790|AAAAAAAAHLMABAAA|68790|19|6|30|PM|second|evening|dinner| +68791|AAAAAAAAILMABAAA|68791|19|6|31|PM|second|evening|dinner| +68792|AAAAAAAAJLMABAAA|68792|19|6|32|PM|second|evening|dinner| +68793|AAAAAAAAKLMABAAA|68793|19|6|33|PM|second|evening|dinner| +68794|AAAAAAAALLMABAAA|68794|19|6|34|PM|second|evening|dinner| +68795|AAAAAAAAMLMABAAA|68795|19|6|35|PM|second|evening|dinner| +68796|AAAAAAAANLMABAAA|68796|19|6|36|PM|second|evening|dinner| +68797|AAAAAAAAOLMABAAA|68797|19|6|37|PM|second|evening|dinner| +68798|AAAAAAAAPLMABAAA|68798|19|6|38|PM|second|evening|dinner| +68799|AAAAAAAAAMMABAAA|68799|19|6|39|PM|second|evening|dinner| +68800|AAAAAAAABMMABAAA|68800|19|6|40|PM|second|evening|dinner| +68801|AAAAAAAACMMABAAA|68801|19|6|41|PM|second|evening|dinner| +68802|AAAAAAAADMMABAAA|68802|19|6|42|PM|second|evening|dinner| +68803|AAAAAAAAEMMABAAA|68803|19|6|43|PM|second|evening|dinner| +68804|AAAAAAAAFMMABAAA|68804|19|6|44|PM|second|evening|dinner| +68805|AAAAAAAAGMMABAAA|68805|19|6|45|PM|second|evening|dinner| +68806|AAAAAAAAHMMABAAA|68806|19|6|46|PM|second|evening|dinner| +68807|AAAAAAAAIMMABAAA|68807|19|6|47|PM|second|evening|dinner| +68808|AAAAAAAAJMMABAAA|68808|19|6|48|PM|second|evening|dinner| +68809|AAAAAAAAKMMABAAA|68809|19|6|49|PM|second|evening|dinner| +68810|AAAAAAAALMMABAAA|68810|19|6|50|PM|second|evening|dinner| +68811|AAAAAAAAMMMABAAA|68811|19|6|51|PM|second|evening|dinner| +68812|AAAAAAAANMMABAAA|68812|19|6|52|PM|second|evening|dinner| +68813|AAAAAAAAOMMABAAA|68813|19|6|53|PM|second|evening|dinner| +68814|AAAAAAAAPMMABAAA|68814|19|6|54|PM|second|evening|dinner| +68815|AAAAAAAAANMABAAA|68815|19|6|55|PM|second|evening|dinner| +68816|AAAAAAAABNMABAAA|68816|19|6|56|PM|second|evening|dinner| +68817|AAAAAAAACNMABAAA|68817|19|6|57|PM|second|evening|dinner| +68818|AAAAAAAADNMABAAA|68818|19|6|58|PM|second|evening|dinner| +68819|AAAAAAAAENMABAAA|68819|19|6|59|PM|second|evening|dinner| +68820|AAAAAAAAFNMABAAA|68820|19|7|0|PM|second|evening|dinner| +68821|AAAAAAAAGNMABAAA|68821|19|7|1|PM|second|evening|dinner| +68822|AAAAAAAAHNMABAAA|68822|19|7|2|PM|second|evening|dinner| +68823|AAAAAAAAINMABAAA|68823|19|7|3|PM|second|evening|dinner| +68824|AAAAAAAAJNMABAAA|68824|19|7|4|PM|second|evening|dinner| +68825|AAAAAAAAKNMABAAA|68825|19|7|5|PM|second|evening|dinner| +68826|AAAAAAAALNMABAAA|68826|19|7|6|PM|second|evening|dinner| +68827|AAAAAAAAMNMABAAA|68827|19|7|7|PM|second|evening|dinner| +68828|AAAAAAAANNMABAAA|68828|19|7|8|PM|second|evening|dinner| +68829|AAAAAAAAONMABAAA|68829|19|7|9|PM|second|evening|dinner| +68830|AAAAAAAAPNMABAAA|68830|19|7|10|PM|second|evening|dinner| +68831|AAAAAAAAAOMABAAA|68831|19|7|11|PM|second|evening|dinner| +68832|AAAAAAAABOMABAAA|68832|19|7|12|PM|second|evening|dinner| +68833|AAAAAAAACOMABAAA|68833|19|7|13|PM|second|evening|dinner| +68834|AAAAAAAADOMABAAA|68834|19|7|14|PM|second|evening|dinner| +68835|AAAAAAAAEOMABAAA|68835|19|7|15|PM|second|evening|dinner| +68836|AAAAAAAAFOMABAAA|68836|19|7|16|PM|second|evening|dinner| +68837|AAAAAAAAGOMABAAA|68837|19|7|17|PM|second|evening|dinner| +68838|AAAAAAAAHOMABAAA|68838|19|7|18|PM|second|evening|dinner| +68839|AAAAAAAAIOMABAAA|68839|19|7|19|PM|second|evening|dinner| +68840|AAAAAAAAJOMABAAA|68840|19|7|20|PM|second|evening|dinner| +68841|AAAAAAAAKOMABAAA|68841|19|7|21|PM|second|evening|dinner| +68842|AAAAAAAALOMABAAA|68842|19|7|22|PM|second|evening|dinner| +68843|AAAAAAAAMOMABAAA|68843|19|7|23|PM|second|evening|dinner| +68844|AAAAAAAANOMABAAA|68844|19|7|24|PM|second|evening|dinner| +68845|AAAAAAAAOOMABAAA|68845|19|7|25|PM|second|evening|dinner| +68846|AAAAAAAAPOMABAAA|68846|19|7|26|PM|second|evening|dinner| +68847|AAAAAAAAAPMABAAA|68847|19|7|27|PM|second|evening|dinner| +68848|AAAAAAAABPMABAAA|68848|19|7|28|PM|second|evening|dinner| +68849|AAAAAAAACPMABAAA|68849|19|7|29|PM|second|evening|dinner| +68850|AAAAAAAADPMABAAA|68850|19|7|30|PM|second|evening|dinner| +68851|AAAAAAAAEPMABAAA|68851|19|7|31|PM|second|evening|dinner| +68852|AAAAAAAAFPMABAAA|68852|19|7|32|PM|second|evening|dinner| +68853|AAAAAAAAGPMABAAA|68853|19|7|33|PM|second|evening|dinner| +68854|AAAAAAAAHPMABAAA|68854|19|7|34|PM|second|evening|dinner| +68855|AAAAAAAAIPMABAAA|68855|19|7|35|PM|second|evening|dinner| +68856|AAAAAAAAJPMABAAA|68856|19|7|36|PM|second|evening|dinner| +68857|AAAAAAAAKPMABAAA|68857|19|7|37|PM|second|evening|dinner| +68858|AAAAAAAALPMABAAA|68858|19|7|38|PM|second|evening|dinner| +68859|AAAAAAAAMPMABAAA|68859|19|7|39|PM|second|evening|dinner| +68860|AAAAAAAANPMABAAA|68860|19|7|40|PM|second|evening|dinner| +68861|AAAAAAAAOPMABAAA|68861|19|7|41|PM|second|evening|dinner| +68862|AAAAAAAAPPMABAAA|68862|19|7|42|PM|second|evening|dinner| +68863|AAAAAAAAAANABAAA|68863|19|7|43|PM|second|evening|dinner| +68864|AAAAAAAABANABAAA|68864|19|7|44|PM|second|evening|dinner| +68865|AAAAAAAACANABAAA|68865|19|7|45|PM|second|evening|dinner| +68866|AAAAAAAADANABAAA|68866|19|7|46|PM|second|evening|dinner| +68867|AAAAAAAAEANABAAA|68867|19|7|47|PM|second|evening|dinner| +68868|AAAAAAAAFANABAAA|68868|19|7|48|PM|second|evening|dinner| +68869|AAAAAAAAGANABAAA|68869|19|7|49|PM|second|evening|dinner| +68870|AAAAAAAAHANABAAA|68870|19|7|50|PM|second|evening|dinner| +68871|AAAAAAAAIANABAAA|68871|19|7|51|PM|second|evening|dinner| +68872|AAAAAAAAJANABAAA|68872|19|7|52|PM|second|evening|dinner| +68873|AAAAAAAAKANABAAA|68873|19|7|53|PM|second|evening|dinner| +68874|AAAAAAAALANABAAA|68874|19|7|54|PM|second|evening|dinner| +68875|AAAAAAAAMANABAAA|68875|19|7|55|PM|second|evening|dinner| +68876|AAAAAAAANANABAAA|68876|19|7|56|PM|second|evening|dinner| +68877|AAAAAAAAOANABAAA|68877|19|7|57|PM|second|evening|dinner| +68878|AAAAAAAAPANABAAA|68878|19|7|58|PM|second|evening|dinner| +68879|AAAAAAAAABNABAAA|68879|19|7|59|PM|second|evening|dinner| +68880|AAAAAAAABBNABAAA|68880|19|8|0|PM|second|evening|dinner| +68881|AAAAAAAACBNABAAA|68881|19|8|1|PM|second|evening|dinner| +68882|AAAAAAAADBNABAAA|68882|19|8|2|PM|second|evening|dinner| +68883|AAAAAAAAEBNABAAA|68883|19|8|3|PM|second|evening|dinner| +68884|AAAAAAAAFBNABAAA|68884|19|8|4|PM|second|evening|dinner| +68885|AAAAAAAAGBNABAAA|68885|19|8|5|PM|second|evening|dinner| +68886|AAAAAAAAHBNABAAA|68886|19|8|6|PM|second|evening|dinner| +68887|AAAAAAAAIBNABAAA|68887|19|8|7|PM|second|evening|dinner| +68888|AAAAAAAAJBNABAAA|68888|19|8|8|PM|second|evening|dinner| +68889|AAAAAAAAKBNABAAA|68889|19|8|9|PM|second|evening|dinner| +68890|AAAAAAAALBNABAAA|68890|19|8|10|PM|second|evening|dinner| +68891|AAAAAAAAMBNABAAA|68891|19|8|11|PM|second|evening|dinner| +68892|AAAAAAAANBNABAAA|68892|19|8|12|PM|second|evening|dinner| +68893|AAAAAAAAOBNABAAA|68893|19|8|13|PM|second|evening|dinner| +68894|AAAAAAAAPBNABAAA|68894|19|8|14|PM|second|evening|dinner| +68895|AAAAAAAAACNABAAA|68895|19|8|15|PM|second|evening|dinner| +68896|AAAAAAAABCNABAAA|68896|19|8|16|PM|second|evening|dinner| +68897|AAAAAAAACCNABAAA|68897|19|8|17|PM|second|evening|dinner| +68898|AAAAAAAADCNABAAA|68898|19|8|18|PM|second|evening|dinner| +68899|AAAAAAAAECNABAAA|68899|19|8|19|PM|second|evening|dinner| +68900|AAAAAAAAFCNABAAA|68900|19|8|20|PM|second|evening|dinner| +68901|AAAAAAAAGCNABAAA|68901|19|8|21|PM|second|evening|dinner| +68902|AAAAAAAAHCNABAAA|68902|19|8|22|PM|second|evening|dinner| +68903|AAAAAAAAICNABAAA|68903|19|8|23|PM|second|evening|dinner| +68904|AAAAAAAAJCNABAAA|68904|19|8|24|PM|second|evening|dinner| +68905|AAAAAAAAKCNABAAA|68905|19|8|25|PM|second|evening|dinner| +68906|AAAAAAAALCNABAAA|68906|19|8|26|PM|second|evening|dinner| +68907|AAAAAAAAMCNABAAA|68907|19|8|27|PM|second|evening|dinner| +68908|AAAAAAAANCNABAAA|68908|19|8|28|PM|second|evening|dinner| +68909|AAAAAAAAOCNABAAA|68909|19|8|29|PM|second|evening|dinner| +68910|AAAAAAAAPCNABAAA|68910|19|8|30|PM|second|evening|dinner| +68911|AAAAAAAAADNABAAA|68911|19|8|31|PM|second|evening|dinner| +68912|AAAAAAAABDNABAAA|68912|19|8|32|PM|second|evening|dinner| +68913|AAAAAAAACDNABAAA|68913|19|8|33|PM|second|evening|dinner| +68914|AAAAAAAADDNABAAA|68914|19|8|34|PM|second|evening|dinner| +68915|AAAAAAAAEDNABAAA|68915|19|8|35|PM|second|evening|dinner| +68916|AAAAAAAAFDNABAAA|68916|19|8|36|PM|second|evening|dinner| +68917|AAAAAAAAGDNABAAA|68917|19|8|37|PM|second|evening|dinner| +68918|AAAAAAAAHDNABAAA|68918|19|8|38|PM|second|evening|dinner| +68919|AAAAAAAAIDNABAAA|68919|19|8|39|PM|second|evening|dinner| +68920|AAAAAAAAJDNABAAA|68920|19|8|40|PM|second|evening|dinner| +68921|AAAAAAAAKDNABAAA|68921|19|8|41|PM|second|evening|dinner| +68922|AAAAAAAALDNABAAA|68922|19|8|42|PM|second|evening|dinner| +68923|AAAAAAAAMDNABAAA|68923|19|8|43|PM|second|evening|dinner| +68924|AAAAAAAANDNABAAA|68924|19|8|44|PM|second|evening|dinner| +68925|AAAAAAAAODNABAAA|68925|19|8|45|PM|second|evening|dinner| +68926|AAAAAAAAPDNABAAA|68926|19|8|46|PM|second|evening|dinner| +68927|AAAAAAAAAENABAAA|68927|19|8|47|PM|second|evening|dinner| +68928|AAAAAAAABENABAAA|68928|19|8|48|PM|second|evening|dinner| +68929|AAAAAAAACENABAAA|68929|19|8|49|PM|second|evening|dinner| +68930|AAAAAAAADENABAAA|68930|19|8|50|PM|second|evening|dinner| +68931|AAAAAAAAEENABAAA|68931|19|8|51|PM|second|evening|dinner| +68932|AAAAAAAAFENABAAA|68932|19|8|52|PM|second|evening|dinner| +68933|AAAAAAAAGENABAAA|68933|19|8|53|PM|second|evening|dinner| +68934|AAAAAAAAHENABAAA|68934|19|8|54|PM|second|evening|dinner| +68935|AAAAAAAAIENABAAA|68935|19|8|55|PM|second|evening|dinner| +68936|AAAAAAAAJENABAAA|68936|19|8|56|PM|second|evening|dinner| +68937|AAAAAAAAKENABAAA|68937|19|8|57|PM|second|evening|dinner| +68938|AAAAAAAALENABAAA|68938|19|8|58|PM|second|evening|dinner| +68939|AAAAAAAAMENABAAA|68939|19|8|59|PM|second|evening|dinner| +68940|AAAAAAAANENABAAA|68940|19|9|0|PM|second|evening|dinner| +68941|AAAAAAAAOENABAAA|68941|19|9|1|PM|second|evening|dinner| +68942|AAAAAAAAPENABAAA|68942|19|9|2|PM|second|evening|dinner| +68943|AAAAAAAAAFNABAAA|68943|19|9|3|PM|second|evening|dinner| +68944|AAAAAAAABFNABAAA|68944|19|9|4|PM|second|evening|dinner| +68945|AAAAAAAACFNABAAA|68945|19|9|5|PM|second|evening|dinner| +68946|AAAAAAAADFNABAAA|68946|19|9|6|PM|second|evening|dinner| +68947|AAAAAAAAEFNABAAA|68947|19|9|7|PM|second|evening|dinner| +68948|AAAAAAAAFFNABAAA|68948|19|9|8|PM|second|evening|dinner| +68949|AAAAAAAAGFNABAAA|68949|19|9|9|PM|second|evening|dinner| +68950|AAAAAAAAHFNABAAA|68950|19|9|10|PM|second|evening|dinner| +68951|AAAAAAAAIFNABAAA|68951|19|9|11|PM|second|evening|dinner| +68952|AAAAAAAAJFNABAAA|68952|19|9|12|PM|second|evening|dinner| +68953|AAAAAAAAKFNABAAA|68953|19|9|13|PM|second|evening|dinner| +68954|AAAAAAAALFNABAAA|68954|19|9|14|PM|second|evening|dinner| +68955|AAAAAAAAMFNABAAA|68955|19|9|15|PM|second|evening|dinner| +68956|AAAAAAAANFNABAAA|68956|19|9|16|PM|second|evening|dinner| +68957|AAAAAAAAOFNABAAA|68957|19|9|17|PM|second|evening|dinner| +68958|AAAAAAAAPFNABAAA|68958|19|9|18|PM|second|evening|dinner| +68959|AAAAAAAAAGNABAAA|68959|19|9|19|PM|second|evening|dinner| +68960|AAAAAAAABGNABAAA|68960|19|9|20|PM|second|evening|dinner| +68961|AAAAAAAACGNABAAA|68961|19|9|21|PM|second|evening|dinner| +68962|AAAAAAAADGNABAAA|68962|19|9|22|PM|second|evening|dinner| +68963|AAAAAAAAEGNABAAA|68963|19|9|23|PM|second|evening|dinner| +68964|AAAAAAAAFGNABAAA|68964|19|9|24|PM|second|evening|dinner| +68965|AAAAAAAAGGNABAAA|68965|19|9|25|PM|second|evening|dinner| +68966|AAAAAAAAHGNABAAA|68966|19|9|26|PM|second|evening|dinner| +68967|AAAAAAAAIGNABAAA|68967|19|9|27|PM|second|evening|dinner| +68968|AAAAAAAAJGNABAAA|68968|19|9|28|PM|second|evening|dinner| +68969|AAAAAAAAKGNABAAA|68969|19|9|29|PM|second|evening|dinner| +68970|AAAAAAAALGNABAAA|68970|19|9|30|PM|second|evening|dinner| +68971|AAAAAAAAMGNABAAA|68971|19|9|31|PM|second|evening|dinner| +68972|AAAAAAAANGNABAAA|68972|19|9|32|PM|second|evening|dinner| +68973|AAAAAAAAOGNABAAA|68973|19|9|33|PM|second|evening|dinner| +68974|AAAAAAAAPGNABAAA|68974|19|9|34|PM|second|evening|dinner| +68975|AAAAAAAAAHNABAAA|68975|19|9|35|PM|second|evening|dinner| +68976|AAAAAAAABHNABAAA|68976|19|9|36|PM|second|evening|dinner| +68977|AAAAAAAACHNABAAA|68977|19|9|37|PM|second|evening|dinner| +68978|AAAAAAAADHNABAAA|68978|19|9|38|PM|second|evening|dinner| +68979|AAAAAAAAEHNABAAA|68979|19|9|39|PM|second|evening|dinner| +68980|AAAAAAAAFHNABAAA|68980|19|9|40|PM|second|evening|dinner| +68981|AAAAAAAAGHNABAAA|68981|19|9|41|PM|second|evening|dinner| +68982|AAAAAAAAHHNABAAA|68982|19|9|42|PM|second|evening|dinner| +68983|AAAAAAAAIHNABAAA|68983|19|9|43|PM|second|evening|dinner| +68984|AAAAAAAAJHNABAAA|68984|19|9|44|PM|second|evening|dinner| +68985|AAAAAAAAKHNABAAA|68985|19|9|45|PM|second|evening|dinner| +68986|AAAAAAAALHNABAAA|68986|19|9|46|PM|second|evening|dinner| +68987|AAAAAAAAMHNABAAA|68987|19|9|47|PM|second|evening|dinner| +68988|AAAAAAAANHNABAAA|68988|19|9|48|PM|second|evening|dinner| +68989|AAAAAAAAOHNABAAA|68989|19|9|49|PM|second|evening|dinner| +68990|AAAAAAAAPHNABAAA|68990|19|9|50|PM|second|evening|dinner| +68991|AAAAAAAAAINABAAA|68991|19|9|51|PM|second|evening|dinner| +68992|AAAAAAAABINABAAA|68992|19|9|52|PM|second|evening|dinner| +68993|AAAAAAAACINABAAA|68993|19|9|53|PM|second|evening|dinner| +68994|AAAAAAAADINABAAA|68994|19|9|54|PM|second|evening|dinner| +68995|AAAAAAAAEINABAAA|68995|19|9|55|PM|second|evening|dinner| +68996|AAAAAAAAFINABAAA|68996|19|9|56|PM|second|evening|dinner| +68997|AAAAAAAAGINABAAA|68997|19|9|57|PM|second|evening|dinner| +68998|AAAAAAAAHINABAAA|68998|19|9|58|PM|second|evening|dinner| +68999|AAAAAAAAIINABAAA|68999|19|9|59|PM|second|evening|dinner| +69000|AAAAAAAAJINABAAA|69000|19|10|0|PM|second|evening|dinner| +69001|AAAAAAAAKINABAAA|69001|19|10|1|PM|second|evening|dinner| +69002|AAAAAAAALINABAAA|69002|19|10|2|PM|second|evening|dinner| +69003|AAAAAAAAMINABAAA|69003|19|10|3|PM|second|evening|dinner| +69004|AAAAAAAANINABAAA|69004|19|10|4|PM|second|evening|dinner| +69005|AAAAAAAAOINABAAA|69005|19|10|5|PM|second|evening|dinner| +69006|AAAAAAAAPINABAAA|69006|19|10|6|PM|second|evening|dinner| +69007|AAAAAAAAAJNABAAA|69007|19|10|7|PM|second|evening|dinner| +69008|AAAAAAAABJNABAAA|69008|19|10|8|PM|second|evening|dinner| +69009|AAAAAAAACJNABAAA|69009|19|10|9|PM|second|evening|dinner| +69010|AAAAAAAADJNABAAA|69010|19|10|10|PM|second|evening|dinner| +69011|AAAAAAAAEJNABAAA|69011|19|10|11|PM|second|evening|dinner| +69012|AAAAAAAAFJNABAAA|69012|19|10|12|PM|second|evening|dinner| +69013|AAAAAAAAGJNABAAA|69013|19|10|13|PM|second|evening|dinner| +69014|AAAAAAAAHJNABAAA|69014|19|10|14|PM|second|evening|dinner| +69015|AAAAAAAAIJNABAAA|69015|19|10|15|PM|second|evening|dinner| +69016|AAAAAAAAJJNABAAA|69016|19|10|16|PM|second|evening|dinner| +69017|AAAAAAAAKJNABAAA|69017|19|10|17|PM|second|evening|dinner| +69018|AAAAAAAALJNABAAA|69018|19|10|18|PM|second|evening|dinner| +69019|AAAAAAAAMJNABAAA|69019|19|10|19|PM|second|evening|dinner| +69020|AAAAAAAANJNABAAA|69020|19|10|20|PM|second|evening|dinner| +69021|AAAAAAAAOJNABAAA|69021|19|10|21|PM|second|evening|dinner| +69022|AAAAAAAAPJNABAAA|69022|19|10|22|PM|second|evening|dinner| +69023|AAAAAAAAAKNABAAA|69023|19|10|23|PM|second|evening|dinner| +69024|AAAAAAAABKNABAAA|69024|19|10|24|PM|second|evening|dinner| +69025|AAAAAAAACKNABAAA|69025|19|10|25|PM|second|evening|dinner| +69026|AAAAAAAADKNABAAA|69026|19|10|26|PM|second|evening|dinner| +69027|AAAAAAAAEKNABAAA|69027|19|10|27|PM|second|evening|dinner| +69028|AAAAAAAAFKNABAAA|69028|19|10|28|PM|second|evening|dinner| +69029|AAAAAAAAGKNABAAA|69029|19|10|29|PM|second|evening|dinner| +69030|AAAAAAAAHKNABAAA|69030|19|10|30|PM|second|evening|dinner| +69031|AAAAAAAAIKNABAAA|69031|19|10|31|PM|second|evening|dinner| +69032|AAAAAAAAJKNABAAA|69032|19|10|32|PM|second|evening|dinner| +69033|AAAAAAAAKKNABAAA|69033|19|10|33|PM|second|evening|dinner| +69034|AAAAAAAALKNABAAA|69034|19|10|34|PM|second|evening|dinner| +69035|AAAAAAAAMKNABAAA|69035|19|10|35|PM|second|evening|dinner| +69036|AAAAAAAANKNABAAA|69036|19|10|36|PM|second|evening|dinner| +69037|AAAAAAAAOKNABAAA|69037|19|10|37|PM|second|evening|dinner| +69038|AAAAAAAAPKNABAAA|69038|19|10|38|PM|second|evening|dinner| +69039|AAAAAAAAALNABAAA|69039|19|10|39|PM|second|evening|dinner| +69040|AAAAAAAABLNABAAA|69040|19|10|40|PM|second|evening|dinner| +69041|AAAAAAAACLNABAAA|69041|19|10|41|PM|second|evening|dinner| +69042|AAAAAAAADLNABAAA|69042|19|10|42|PM|second|evening|dinner| +69043|AAAAAAAAELNABAAA|69043|19|10|43|PM|second|evening|dinner| +69044|AAAAAAAAFLNABAAA|69044|19|10|44|PM|second|evening|dinner| +69045|AAAAAAAAGLNABAAA|69045|19|10|45|PM|second|evening|dinner| +69046|AAAAAAAAHLNABAAA|69046|19|10|46|PM|second|evening|dinner| +69047|AAAAAAAAILNABAAA|69047|19|10|47|PM|second|evening|dinner| +69048|AAAAAAAAJLNABAAA|69048|19|10|48|PM|second|evening|dinner| +69049|AAAAAAAAKLNABAAA|69049|19|10|49|PM|second|evening|dinner| +69050|AAAAAAAALLNABAAA|69050|19|10|50|PM|second|evening|dinner| +69051|AAAAAAAAMLNABAAA|69051|19|10|51|PM|second|evening|dinner| +69052|AAAAAAAANLNABAAA|69052|19|10|52|PM|second|evening|dinner| +69053|AAAAAAAAOLNABAAA|69053|19|10|53|PM|second|evening|dinner| +69054|AAAAAAAAPLNABAAA|69054|19|10|54|PM|second|evening|dinner| +69055|AAAAAAAAAMNABAAA|69055|19|10|55|PM|second|evening|dinner| +69056|AAAAAAAABMNABAAA|69056|19|10|56|PM|second|evening|dinner| +69057|AAAAAAAACMNABAAA|69057|19|10|57|PM|second|evening|dinner| +69058|AAAAAAAADMNABAAA|69058|19|10|58|PM|second|evening|dinner| +69059|AAAAAAAAEMNABAAA|69059|19|10|59|PM|second|evening|dinner| +69060|AAAAAAAAFMNABAAA|69060|19|11|0|PM|second|evening|dinner| +69061|AAAAAAAAGMNABAAA|69061|19|11|1|PM|second|evening|dinner| +69062|AAAAAAAAHMNABAAA|69062|19|11|2|PM|second|evening|dinner| +69063|AAAAAAAAIMNABAAA|69063|19|11|3|PM|second|evening|dinner| +69064|AAAAAAAAJMNABAAA|69064|19|11|4|PM|second|evening|dinner| +69065|AAAAAAAAKMNABAAA|69065|19|11|5|PM|second|evening|dinner| +69066|AAAAAAAALMNABAAA|69066|19|11|6|PM|second|evening|dinner| +69067|AAAAAAAAMMNABAAA|69067|19|11|7|PM|second|evening|dinner| +69068|AAAAAAAANMNABAAA|69068|19|11|8|PM|second|evening|dinner| +69069|AAAAAAAAOMNABAAA|69069|19|11|9|PM|second|evening|dinner| +69070|AAAAAAAAPMNABAAA|69070|19|11|10|PM|second|evening|dinner| +69071|AAAAAAAAANNABAAA|69071|19|11|11|PM|second|evening|dinner| +69072|AAAAAAAABNNABAAA|69072|19|11|12|PM|second|evening|dinner| +69073|AAAAAAAACNNABAAA|69073|19|11|13|PM|second|evening|dinner| +69074|AAAAAAAADNNABAAA|69074|19|11|14|PM|second|evening|dinner| +69075|AAAAAAAAENNABAAA|69075|19|11|15|PM|second|evening|dinner| +69076|AAAAAAAAFNNABAAA|69076|19|11|16|PM|second|evening|dinner| +69077|AAAAAAAAGNNABAAA|69077|19|11|17|PM|second|evening|dinner| +69078|AAAAAAAAHNNABAAA|69078|19|11|18|PM|second|evening|dinner| +69079|AAAAAAAAINNABAAA|69079|19|11|19|PM|second|evening|dinner| +69080|AAAAAAAAJNNABAAA|69080|19|11|20|PM|second|evening|dinner| +69081|AAAAAAAAKNNABAAA|69081|19|11|21|PM|second|evening|dinner| +69082|AAAAAAAALNNABAAA|69082|19|11|22|PM|second|evening|dinner| +69083|AAAAAAAAMNNABAAA|69083|19|11|23|PM|second|evening|dinner| +69084|AAAAAAAANNNABAAA|69084|19|11|24|PM|second|evening|dinner| +69085|AAAAAAAAONNABAAA|69085|19|11|25|PM|second|evening|dinner| +69086|AAAAAAAAPNNABAAA|69086|19|11|26|PM|second|evening|dinner| +69087|AAAAAAAAAONABAAA|69087|19|11|27|PM|second|evening|dinner| +69088|AAAAAAAABONABAAA|69088|19|11|28|PM|second|evening|dinner| +69089|AAAAAAAACONABAAA|69089|19|11|29|PM|second|evening|dinner| +69090|AAAAAAAADONABAAA|69090|19|11|30|PM|second|evening|dinner| +69091|AAAAAAAAEONABAAA|69091|19|11|31|PM|second|evening|dinner| +69092|AAAAAAAAFONABAAA|69092|19|11|32|PM|second|evening|dinner| +69093|AAAAAAAAGONABAAA|69093|19|11|33|PM|second|evening|dinner| +69094|AAAAAAAAHONABAAA|69094|19|11|34|PM|second|evening|dinner| +69095|AAAAAAAAIONABAAA|69095|19|11|35|PM|second|evening|dinner| +69096|AAAAAAAAJONABAAA|69096|19|11|36|PM|second|evening|dinner| +69097|AAAAAAAAKONABAAA|69097|19|11|37|PM|second|evening|dinner| +69098|AAAAAAAALONABAAA|69098|19|11|38|PM|second|evening|dinner| +69099|AAAAAAAAMONABAAA|69099|19|11|39|PM|second|evening|dinner| +69100|AAAAAAAANONABAAA|69100|19|11|40|PM|second|evening|dinner| +69101|AAAAAAAAOONABAAA|69101|19|11|41|PM|second|evening|dinner| +69102|AAAAAAAAPONABAAA|69102|19|11|42|PM|second|evening|dinner| +69103|AAAAAAAAAPNABAAA|69103|19|11|43|PM|second|evening|dinner| +69104|AAAAAAAABPNABAAA|69104|19|11|44|PM|second|evening|dinner| +69105|AAAAAAAACPNABAAA|69105|19|11|45|PM|second|evening|dinner| +69106|AAAAAAAADPNABAAA|69106|19|11|46|PM|second|evening|dinner| +69107|AAAAAAAAEPNABAAA|69107|19|11|47|PM|second|evening|dinner| +69108|AAAAAAAAFPNABAAA|69108|19|11|48|PM|second|evening|dinner| +69109|AAAAAAAAGPNABAAA|69109|19|11|49|PM|second|evening|dinner| +69110|AAAAAAAAHPNABAAA|69110|19|11|50|PM|second|evening|dinner| +69111|AAAAAAAAIPNABAAA|69111|19|11|51|PM|second|evening|dinner| +69112|AAAAAAAAJPNABAAA|69112|19|11|52|PM|second|evening|dinner| +69113|AAAAAAAAKPNABAAA|69113|19|11|53|PM|second|evening|dinner| +69114|AAAAAAAALPNABAAA|69114|19|11|54|PM|second|evening|dinner| +69115|AAAAAAAAMPNABAAA|69115|19|11|55|PM|second|evening|dinner| +69116|AAAAAAAANPNABAAA|69116|19|11|56|PM|second|evening|dinner| +69117|AAAAAAAAOPNABAAA|69117|19|11|57|PM|second|evening|dinner| +69118|AAAAAAAAPPNABAAA|69118|19|11|58|PM|second|evening|dinner| +69119|AAAAAAAAAAOABAAA|69119|19|11|59|PM|second|evening|dinner| +69120|AAAAAAAABAOABAAA|69120|19|12|0|PM|second|evening|dinner| +69121|AAAAAAAACAOABAAA|69121|19|12|1|PM|second|evening|dinner| +69122|AAAAAAAADAOABAAA|69122|19|12|2|PM|second|evening|dinner| +69123|AAAAAAAAEAOABAAA|69123|19|12|3|PM|second|evening|dinner| +69124|AAAAAAAAFAOABAAA|69124|19|12|4|PM|second|evening|dinner| +69125|AAAAAAAAGAOABAAA|69125|19|12|5|PM|second|evening|dinner| +69126|AAAAAAAAHAOABAAA|69126|19|12|6|PM|second|evening|dinner| +69127|AAAAAAAAIAOABAAA|69127|19|12|7|PM|second|evening|dinner| +69128|AAAAAAAAJAOABAAA|69128|19|12|8|PM|second|evening|dinner| +69129|AAAAAAAAKAOABAAA|69129|19|12|9|PM|second|evening|dinner| +69130|AAAAAAAALAOABAAA|69130|19|12|10|PM|second|evening|dinner| +69131|AAAAAAAAMAOABAAA|69131|19|12|11|PM|second|evening|dinner| +69132|AAAAAAAANAOABAAA|69132|19|12|12|PM|second|evening|dinner| +69133|AAAAAAAAOAOABAAA|69133|19|12|13|PM|second|evening|dinner| +69134|AAAAAAAAPAOABAAA|69134|19|12|14|PM|second|evening|dinner| +69135|AAAAAAAAABOABAAA|69135|19|12|15|PM|second|evening|dinner| +69136|AAAAAAAABBOABAAA|69136|19|12|16|PM|second|evening|dinner| +69137|AAAAAAAACBOABAAA|69137|19|12|17|PM|second|evening|dinner| +69138|AAAAAAAADBOABAAA|69138|19|12|18|PM|second|evening|dinner| +69139|AAAAAAAAEBOABAAA|69139|19|12|19|PM|second|evening|dinner| +69140|AAAAAAAAFBOABAAA|69140|19|12|20|PM|second|evening|dinner| +69141|AAAAAAAAGBOABAAA|69141|19|12|21|PM|second|evening|dinner| +69142|AAAAAAAAHBOABAAA|69142|19|12|22|PM|second|evening|dinner| +69143|AAAAAAAAIBOABAAA|69143|19|12|23|PM|second|evening|dinner| +69144|AAAAAAAAJBOABAAA|69144|19|12|24|PM|second|evening|dinner| +69145|AAAAAAAAKBOABAAA|69145|19|12|25|PM|second|evening|dinner| +69146|AAAAAAAALBOABAAA|69146|19|12|26|PM|second|evening|dinner| +69147|AAAAAAAAMBOABAAA|69147|19|12|27|PM|second|evening|dinner| +69148|AAAAAAAANBOABAAA|69148|19|12|28|PM|second|evening|dinner| +69149|AAAAAAAAOBOABAAA|69149|19|12|29|PM|second|evening|dinner| +69150|AAAAAAAAPBOABAAA|69150|19|12|30|PM|second|evening|dinner| +69151|AAAAAAAAACOABAAA|69151|19|12|31|PM|second|evening|dinner| +69152|AAAAAAAABCOABAAA|69152|19|12|32|PM|second|evening|dinner| +69153|AAAAAAAACCOABAAA|69153|19|12|33|PM|second|evening|dinner| +69154|AAAAAAAADCOABAAA|69154|19|12|34|PM|second|evening|dinner| +69155|AAAAAAAAECOABAAA|69155|19|12|35|PM|second|evening|dinner| +69156|AAAAAAAAFCOABAAA|69156|19|12|36|PM|second|evening|dinner| +69157|AAAAAAAAGCOABAAA|69157|19|12|37|PM|second|evening|dinner| +69158|AAAAAAAAHCOABAAA|69158|19|12|38|PM|second|evening|dinner| +69159|AAAAAAAAICOABAAA|69159|19|12|39|PM|second|evening|dinner| +69160|AAAAAAAAJCOABAAA|69160|19|12|40|PM|second|evening|dinner| +69161|AAAAAAAAKCOABAAA|69161|19|12|41|PM|second|evening|dinner| +69162|AAAAAAAALCOABAAA|69162|19|12|42|PM|second|evening|dinner| +69163|AAAAAAAAMCOABAAA|69163|19|12|43|PM|second|evening|dinner| +69164|AAAAAAAANCOABAAA|69164|19|12|44|PM|second|evening|dinner| +69165|AAAAAAAAOCOABAAA|69165|19|12|45|PM|second|evening|dinner| +69166|AAAAAAAAPCOABAAA|69166|19|12|46|PM|second|evening|dinner| +69167|AAAAAAAAADOABAAA|69167|19|12|47|PM|second|evening|dinner| +69168|AAAAAAAABDOABAAA|69168|19|12|48|PM|second|evening|dinner| +69169|AAAAAAAACDOABAAA|69169|19|12|49|PM|second|evening|dinner| +69170|AAAAAAAADDOABAAA|69170|19|12|50|PM|second|evening|dinner| +69171|AAAAAAAAEDOABAAA|69171|19|12|51|PM|second|evening|dinner| +69172|AAAAAAAAFDOABAAA|69172|19|12|52|PM|second|evening|dinner| +69173|AAAAAAAAGDOABAAA|69173|19|12|53|PM|second|evening|dinner| +69174|AAAAAAAAHDOABAAA|69174|19|12|54|PM|second|evening|dinner| +69175|AAAAAAAAIDOABAAA|69175|19|12|55|PM|second|evening|dinner| +69176|AAAAAAAAJDOABAAA|69176|19|12|56|PM|second|evening|dinner| +69177|AAAAAAAAKDOABAAA|69177|19|12|57|PM|second|evening|dinner| +69178|AAAAAAAALDOABAAA|69178|19|12|58|PM|second|evening|dinner| +69179|AAAAAAAAMDOABAAA|69179|19|12|59|PM|second|evening|dinner| +69180|AAAAAAAANDOABAAA|69180|19|13|0|PM|second|evening|dinner| +69181|AAAAAAAAODOABAAA|69181|19|13|1|PM|second|evening|dinner| +69182|AAAAAAAAPDOABAAA|69182|19|13|2|PM|second|evening|dinner| +69183|AAAAAAAAAEOABAAA|69183|19|13|3|PM|second|evening|dinner| +69184|AAAAAAAABEOABAAA|69184|19|13|4|PM|second|evening|dinner| +69185|AAAAAAAACEOABAAA|69185|19|13|5|PM|second|evening|dinner| +69186|AAAAAAAADEOABAAA|69186|19|13|6|PM|second|evening|dinner| +69187|AAAAAAAAEEOABAAA|69187|19|13|7|PM|second|evening|dinner| +69188|AAAAAAAAFEOABAAA|69188|19|13|8|PM|second|evening|dinner| +69189|AAAAAAAAGEOABAAA|69189|19|13|9|PM|second|evening|dinner| +69190|AAAAAAAAHEOABAAA|69190|19|13|10|PM|second|evening|dinner| +69191|AAAAAAAAIEOABAAA|69191|19|13|11|PM|second|evening|dinner| +69192|AAAAAAAAJEOABAAA|69192|19|13|12|PM|second|evening|dinner| +69193|AAAAAAAAKEOABAAA|69193|19|13|13|PM|second|evening|dinner| +69194|AAAAAAAALEOABAAA|69194|19|13|14|PM|second|evening|dinner| +69195|AAAAAAAAMEOABAAA|69195|19|13|15|PM|second|evening|dinner| +69196|AAAAAAAANEOABAAA|69196|19|13|16|PM|second|evening|dinner| +69197|AAAAAAAAOEOABAAA|69197|19|13|17|PM|second|evening|dinner| +69198|AAAAAAAAPEOABAAA|69198|19|13|18|PM|second|evening|dinner| +69199|AAAAAAAAAFOABAAA|69199|19|13|19|PM|second|evening|dinner| +69200|AAAAAAAABFOABAAA|69200|19|13|20|PM|second|evening|dinner| +69201|AAAAAAAACFOABAAA|69201|19|13|21|PM|second|evening|dinner| +69202|AAAAAAAADFOABAAA|69202|19|13|22|PM|second|evening|dinner| +69203|AAAAAAAAEFOABAAA|69203|19|13|23|PM|second|evening|dinner| +69204|AAAAAAAAFFOABAAA|69204|19|13|24|PM|second|evening|dinner| +69205|AAAAAAAAGFOABAAA|69205|19|13|25|PM|second|evening|dinner| +69206|AAAAAAAAHFOABAAA|69206|19|13|26|PM|second|evening|dinner| +69207|AAAAAAAAIFOABAAA|69207|19|13|27|PM|second|evening|dinner| +69208|AAAAAAAAJFOABAAA|69208|19|13|28|PM|second|evening|dinner| +69209|AAAAAAAAKFOABAAA|69209|19|13|29|PM|second|evening|dinner| +69210|AAAAAAAALFOABAAA|69210|19|13|30|PM|second|evening|dinner| +69211|AAAAAAAAMFOABAAA|69211|19|13|31|PM|second|evening|dinner| +69212|AAAAAAAANFOABAAA|69212|19|13|32|PM|second|evening|dinner| +69213|AAAAAAAAOFOABAAA|69213|19|13|33|PM|second|evening|dinner| +69214|AAAAAAAAPFOABAAA|69214|19|13|34|PM|second|evening|dinner| +69215|AAAAAAAAAGOABAAA|69215|19|13|35|PM|second|evening|dinner| +69216|AAAAAAAABGOABAAA|69216|19|13|36|PM|second|evening|dinner| +69217|AAAAAAAACGOABAAA|69217|19|13|37|PM|second|evening|dinner| +69218|AAAAAAAADGOABAAA|69218|19|13|38|PM|second|evening|dinner| +69219|AAAAAAAAEGOABAAA|69219|19|13|39|PM|second|evening|dinner| +69220|AAAAAAAAFGOABAAA|69220|19|13|40|PM|second|evening|dinner| +69221|AAAAAAAAGGOABAAA|69221|19|13|41|PM|second|evening|dinner| +69222|AAAAAAAAHGOABAAA|69222|19|13|42|PM|second|evening|dinner| +69223|AAAAAAAAIGOABAAA|69223|19|13|43|PM|second|evening|dinner| +69224|AAAAAAAAJGOABAAA|69224|19|13|44|PM|second|evening|dinner| +69225|AAAAAAAAKGOABAAA|69225|19|13|45|PM|second|evening|dinner| +69226|AAAAAAAALGOABAAA|69226|19|13|46|PM|second|evening|dinner| +69227|AAAAAAAAMGOABAAA|69227|19|13|47|PM|second|evening|dinner| +69228|AAAAAAAANGOABAAA|69228|19|13|48|PM|second|evening|dinner| +69229|AAAAAAAAOGOABAAA|69229|19|13|49|PM|second|evening|dinner| +69230|AAAAAAAAPGOABAAA|69230|19|13|50|PM|second|evening|dinner| +69231|AAAAAAAAAHOABAAA|69231|19|13|51|PM|second|evening|dinner| +69232|AAAAAAAABHOABAAA|69232|19|13|52|PM|second|evening|dinner| +69233|AAAAAAAACHOABAAA|69233|19|13|53|PM|second|evening|dinner| +69234|AAAAAAAADHOABAAA|69234|19|13|54|PM|second|evening|dinner| +69235|AAAAAAAAEHOABAAA|69235|19|13|55|PM|second|evening|dinner| +69236|AAAAAAAAFHOABAAA|69236|19|13|56|PM|second|evening|dinner| +69237|AAAAAAAAGHOABAAA|69237|19|13|57|PM|second|evening|dinner| +69238|AAAAAAAAHHOABAAA|69238|19|13|58|PM|second|evening|dinner| +69239|AAAAAAAAIHOABAAA|69239|19|13|59|PM|second|evening|dinner| +69240|AAAAAAAAJHOABAAA|69240|19|14|0|PM|second|evening|dinner| +69241|AAAAAAAAKHOABAAA|69241|19|14|1|PM|second|evening|dinner| +69242|AAAAAAAALHOABAAA|69242|19|14|2|PM|second|evening|dinner| +69243|AAAAAAAAMHOABAAA|69243|19|14|3|PM|second|evening|dinner| +69244|AAAAAAAANHOABAAA|69244|19|14|4|PM|second|evening|dinner| +69245|AAAAAAAAOHOABAAA|69245|19|14|5|PM|second|evening|dinner| +69246|AAAAAAAAPHOABAAA|69246|19|14|6|PM|second|evening|dinner| +69247|AAAAAAAAAIOABAAA|69247|19|14|7|PM|second|evening|dinner| +69248|AAAAAAAABIOABAAA|69248|19|14|8|PM|second|evening|dinner| +69249|AAAAAAAACIOABAAA|69249|19|14|9|PM|second|evening|dinner| +69250|AAAAAAAADIOABAAA|69250|19|14|10|PM|second|evening|dinner| +69251|AAAAAAAAEIOABAAA|69251|19|14|11|PM|second|evening|dinner| +69252|AAAAAAAAFIOABAAA|69252|19|14|12|PM|second|evening|dinner| +69253|AAAAAAAAGIOABAAA|69253|19|14|13|PM|second|evening|dinner| +69254|AAAAAAAAHIOABAAA|69254|19|14|14|PM|second|evening|dinner| +69255|AAAAAAAAIIOABAAA|69255|19|14|15|PM|second|evening|dinner| +69256|AAAAAAAAJIOABAAA|69256|19|14|16|PM|second|evening|dinner| +69257|AAAAAAAAKIOABAAA|69257|19|14|17|PM|second|evening|dinner| +69258|AAAAAAAALIOABAAA|69258|19|14|18|PM|second|evening|dinner| +69259|AAAAAAAAMIOABAAA|69259|19|14|19|PM|second|evening|dinner| +69260|AAAAAAAANIOABAAA|69260|19|14|20|PM|second|evening|dinner| +69261|AAAAAAAAOIOABAAA|69261|19|14|21|PM|second|evening|dinner| +69262|AAAAAAAAPIOABAAA|69262|19|14|22|PM|second|evening|dinner| +69263|AAAAAAAAAJOABAAA|69263|19|14|23|PM|second|evening|dinner| +69264|AAAAAAAABJOABAAA|69264|19|14|24|PM|second|evening|dinner| +69265|AAAAAAAACJOABAAA|69265|19|14|25|PM|second|evening|dinner| +69266|AAAAAAAADJOABAAA|69266|19|14|26|PM|second|evening|dinner| +69267|AAAAAAAAEJOABAAA|69267|19|14|27|PM|second|evening|dinner| +69268|AAAAAAAAFJOABAAA|69268|19|14|28|PM|second|evening|dinner| +69269|AAAAAAAAGJOABAAA|69269|19|14|29|PM|second|evening|dinner| +69270|AAAAAAAAHJOABAAA|69270|19|14|30|PM|second|evening|dinner| +69271|AAAAAAAAIJOABAAA|69271|19|14|31|PM|second|evening|dinner| +69272|AAAAAAAAJJOABAAA|69272|19|14|32|PM|second|evening|dinner| +69273|AAAAAAAAKJOABAAA|69273|19|14|33|PM|second|evening|dinner| +69274|AAAAAAAALJOABAAA|69274|19|14|34|PM|second|evening|dinner| +69275|AAAAAAAAMJOABAAA|69275|19|14|35|PM|second|evening|dinner| +69276|AAAAAAAANJOABAAA|69276|19|14|36|PM|second|evening|dinner| +69277|AAAAAAAAOJOABAAA|69277|19|14|37|PM|second|evening|dinner| +69278|AAAAAAAAPJOABAAA|69278|19|14|38|PM|second|evening|dinner| +69279|AAAAAAAAAKOABAAA|69279|19|14|39|PM|second|evening|dinner| +69280|AAAAAAAABKOABAAA|69280|19|14|40|PM|second|evening|dinner| +69281|AAAAAAAACKOABAAA|69281|19|14|41|PM|second|evening|dinner| +69282|AAAAAAAADKOABAAA|69282|19|14|42|PM|second|evening|dinner| +69283|AAAAAAAAEKOABAAA|69283|19|14|43|PM|second|evening|dinner| +69284|AAAAAAAAFKOABAAA|69284|19|14|44|PM|second|evening|dinner| +69285|AAAAAAAAGKOABAAA|69285|19|14|45|PM|second|evening|dinner| +69286|AAAAAAAAHKOABAAA|69286|19|14|46|PM|second|evening|dinner| +69287|AAAAAAAAIKOABAAA|69287|19|14|47|PM|second|evening|dinner| +69288|AAAAAAAAJKOABAAA|69288|19|14|48|PM|second|evening|dinner| +69289|AAAAAAAAKKOABAAA|69289|19|14|49|PM|second|evening|dinner| +69290|AAAAAAAALKOABAAA|69290|19|14|50|PM|second|evening|dinner| +69291|AAAAAAAAMKOABAAA|69291|19|14|51|PM|second|evening|dinner| +69292|AAAAAAAANKOABAAA|69292|19|14|52|PM|second|evening|dinner| +69293|AAAAAAAAOKOABAAA|69293|19|14|53|PM|second|evening|dinner| +69294|AAAAAAAAPKOABAAA|69294|19|14|54|PM|second|evening|dinner| +69295|AAAAAAAAALOABAAA|69295|19|14|55|PM|second|evening|dinner| +69296|AAAAAAAABLOABAAA|69296|19|14|56|PM|second|evening|dinner| +69297|AAAAAAAACLOABAAA|69297|19|14|57|PM|second|evening|dinner| +69298|AAAAAAAADLOABAAA|69298|19|14|58|PM|second|evening|dinner| +69299|AAAAAAAAELOABAAA|69299|19|14|59|PM|second|evening|dinner| +69300|AAAAAAAAFLOABAAA|69300|19|15|0|PM|second|evening|dinner| +69301|AAAAAAAAGLOABAAA|69301|19|15|1|PM|second|evening|dinner| +69302|AAAAAAAAHLOABAAA|69302|19|15|2|PM|second|evening|dinner| +69303|AAAAAAAAILOABAAA|69303|19|15|3|PM|second|evening|dinner| +69304|AAAAAAAAJLOABAAA|69304|19|15|4|PM|second|evening|dinner| +69305|AAAAAAAAKLOABAAA|69305|19|15|5|PM|second|evening|dinner| +69306|AAAAAAAALLOABAAA|69306|19|15|6|PM|second|evening|dinner| +69307|AAAAAAAAMLOABAAA|69307|19|15|7|PM|second|evening|dinner| +69308|AAAAAAAANLOABAAA|69308|19|15|8|PM|second|evening|dinner| +69309|AAAAAAAAOLOABAAA|69309|19|15|9|PM|second|evening|dinner| +69310|AAAAAAAAPLOABAAA|69310|19|15|10|PM|second|evening|dinner| +69311|AAAAAAAAAMOABAAA|69311|19|15|11|PM|second|evening|dinner| +69312|AAAAAAAABMOABAAA|69312|19|15|12|PM|second|evening|dinner| +69313|AAAAAAAACMOABAAA|69313|19|15|13|PM|second|evening|dinner| +69314|AAAAAAAADMOABAAA|69314|19|15|14|PM|second|evening|dinner| +69315|AAAAAAAAEMOABAAA|69315|19|15|15|PM|second|evening|dinner| +69316|AAAAAAAAFMOABAAA|69316|19|15|16|PM|second|evening|dinner| +69317|AAAAAAAAGMOABAAA|69317|19|15|17|PM|second|evening|dinner| +69318|AAAAAAAAHMOABAAA|69318|19|15|18|PM|second|evening|dinner| +69319|AAAAAAAAIMOABAAA|69319|19|15|19|PM|second|evening|dinner| +69320|AAAAAAAAJMOABAAA|69320|19|15|20|PM|second|evening|dinner| +69321|AAAAAAAAKMOABAAA|69321|19|15|21|PM|second|evening|dinner| +69322|AAAAAAAALMOABAAA|69322|19|15|22|PM|second|evening|dinner| +69323|AAAAAAAAMMOABAAA|69323|19|15|23|PM|second|evening|dinner| +69324|AAAAAAAANMOABAAA|69324|19|15|24|PM|second|evening|dinner| +69325|AAAAAAAAOMOABAAA|69325|19|15|25|PM|second|evening|dinner| +69326|AAAAAAAAPMOABAAA|69326|19|15|26|PM|second|evening|dinner| +69327|AAAAAAAAANOABAAA|69327|19|15|27|PM|second|evening|dinner| +69328|AAAAAAAABNOABAAA|69328|19|15|28|PM|second|evening|dinner| +69329|AAAAAAAACNOABAAA|69329|19|15|29|PM|second|evening|dinner| +69330|AAAAAAAADNOABAAA|69330|19|15|30|PM|second|evening|dinner| +69331|AAAAAAAAENOABAAA|69331|19|15|31|PM|second|evening|dinner| +69332|AAAAAAAAFNOABAAA|69332|19|15|32|PM|second|evening|dinner| +69333|AAAAAAAAGNOABAAA|69333|19|15|33|PM|second|evening|dinner| +69334|AAAAAAAAHNOABAAA|69334|19|15|34|PM|second|evening|dinner| +69335|AAAAAAAAINOABAAA|69335|19|15|35|PM|second|evening|dinner| +69336|AAAAAAAAJNOABAAA|69336|19|15|36|PM|second|evening|dinner| +69337|AAAAAAAAKNOABAAA|69337|19|15|37|PM|second|evening|dinner| +69338|AAAAAAAALNOABAAA|69338|19|15|38|PM|second|evening|dinner| +69339|AAAAAAAAMNOABAAA|69339|19|15|39|PM|second|evening|dinner| +69340|AAAAAAAANNOABAAA|69340|19|15|40|PM|second|evening|dinner| +69341|AAAAAAAAONOABAAA|69341|19|15|41|PM|second|evening|dinner| +69342|AAAAAAAAPNOABAAA|69342|19|15|42|PM|second|evening|dinner| +69343|AAAAAAAAAOOABAAA|69343|19|15|43|PM|second|evening|dinner| +69344|AAAAAAAABOOABAAA|69344|19|15|44|PM|second|evening|dinner| +69345|AAAAAAAACOOABAAA|69345|19|15|45|PM|second|evening|dinner| +69346|AAAAAAAADOOABAAA|69346|19|15|46|PM|second|evening|dinner| +69347|AAAAAAAAEOOABAAA|69347|19|15|47|PM|second|evening|dinner| +69348|AAAAAAAAFOOABAAA|69348|19|15|48|PM|second|evening|dinner| +69349|AAAAAAAAGOOABAAA|69349|19|15|49|PM|second|evening|dinner| +69350|AAAAAAAAHOOABAAA|69350|19|15|50|PM|second|evening|dinner| +69351|AAAAAAAAIOOABAAA|69351|19|15|51|PM|second|evening|dinner| +69352|AAAAAAAAJOOABAAA|69352|19|15|52|PM|second|evening|dinner| +69353|AAAAAAAAKOOABAAA|69353|19|15|53|PM|second|evening|dinner| +69354|AAAAAAAALOOABAAA|69354|19|15|54|PM|second|evening|dinner| +69355|AAAAAAAAMOOABAAA|69355|19|15|55|PM|second|evening|dinner| +69356|AAAAAAAANOOABAAA|69356|19|15|56|PM|second|evening|dinner| +69357|AAAAAAAAOOOABAAA|69357|19|15|57|PM|second|evening|dinner| +69358|AAAAAAAAPOOABAAA|69358|19|15|58|PM|second|evening|dinner| +69359|AAAAAAAAAPOABAAA|69359|19|15|59|PM|second|evening|dinner| +69360|AAAAAAAABPOABAAA|69360|19|16|0|PM|second|evening|dinner| +69361|AAAAAAAACPOABAAA|69361|19|16|1|PM|second|evening|dinner| +69362|AAAAAAAADPOABAAA|69362|19|16|2|PM|second|evening|dinner| +69363|AAAAAAAAEPOABAAA|69363|19|16|3|PM|second|evening|dinner| +69364|AAAAAAAAFPOABAAA|69364|19|16|4|PM|second|evening|dinner| +69365|AAAAAAAAGPOABAAA|69365|19|16|5|PM|second|evening|dinner| +69366|AAAAAAAAHPOABAAA|69366|19|16|6|PM|second|evening|dinner| +69367|AAAAAAAAIPOABAAA|69367|19|16|7|PM|second|evening|dinner| +69368|AAAAAAAAJPOABAAA|69368|19|16|8|PM|second|evening|dinner| +69369|AAAAAAAAKPOABAAA|69369|19|16|9|PM|second|evening|dinner| +69370|AAAAAAAALPOABAAA|69370|19|16|10|PM|second|evening|dinner| +69371|AAAAAAAAMPOABAAA|69371|19|16|11|PM|second|evening|dinner| +69372|AAAAAAAANPOABAAA|69372|19|16|12|PM|second|evening|dinner| +69373|AAAAAAAAOPOABAAA|69373|19|16|13|PM|second|evening|dinner| +69374|AAAAAAAAPPOABAAA|69374|19|16|14|PM|second|evening|dinner| +69375|AAAAAAAAAAPABAAA|69375|19|16|15|PM|second|evening|dinner| +69376|AAAAAAAABAPABAAA|69376|19|16|16|PM|second|evening|dinner| +69377|AAAAAAAACAPABAAA|69377|19|16|17|PM|second|evening|dinner| +69378|AAAAAAAADAPABAAA|69378|19|16|18|PM|second|evening|dinner| +69379|AAAAAAAAEAPABAAA|69379|19|16|19|PM|second|evening|dinner| +69380|AAAAAAAAFAPABAAA|69380|19|16|20|PM|second|evening|dinner| +69381|AAAAAAAAGAPABAAA|69381|19|16|21|PM|second|evening|dinner| +69382|AAAAAAAAHAPABAAA|69382|19|16|22|PM|second|evening|dinner| +69383|AAAAAAAAIAPABAAA|69383|19|16|23|PM|second|evening|dinner| +69384|AAAAAAAAJAPABAAA|69384|19|16|24|PM|second|evening|dinner| +69385|AAAAAAAAKAPABAAA|69385|19|16|25|PM|second|evening|dinner| +69386|AAAAAAAALAPABAAA|69386|19|16|26|PM|second|evening|dinner| +69387|AAAAAAAAMAPABAAA|69387|19|16|27|PM|second|evening|dinner| +69388|AAAAAAAANAPABAAA|69388|19|16|28|PM|second|evening|dinner| +69389|AAAAAAAAOAPABAAA|69389|19|16|29|PM|second|evening|dinner| +69390|AAAAAAAAPAPABAAA|69390|19|16|30|PM|second|evening|dinner| +69391|AAAAAAAAABPABAAA|69391|19|16|31|PM|second|evening|dinner| +69392|AAAAAAAABBPABAAA|69392|19|16|32|PM|second|evening|dinner| +69393|AAAAAAAACBPABAAA|69393|19|16|33|PM|second|evening|dinner| +69394|AAAAAAAADBPABAAA|69394|19|16|34|PM|second|evening|dinner| +69395|AAAAAAAAEBPABAAA|69395|19|16|35|PM|second|evening|dinner| +69396|AAAAAAAAFBPABAAA|69396|19|16|36|PM|second|evening|dinner| +69397|AAAAAAAAGBPABAAA|69397|19|16|37|PM|second|evening|dinner| +69398|AAAAAAAAHBPABAAA|69398|19|16|38|PM|second|evening|dinner| +69399|AAAAAAAAIBPABAAA|69399|19|16|39|PM|second|evening|dinner| +69400|AAAAAAAAJBPABAAA|69400|19|16|40|PM|second|evening|dinner| +69401|AAAAAAAAKBPABAAA|69401|19|16|41|PM|second|evening|dinner| +69402|AAAAAAAALBPABAAA|69402|19|16|42|PM|second|evening|dinner| +69403|AAAAAAAAMBPABAAA|69403|19|16|43|PM|second|evening|dinner| +69404|AAAAAAAANBPABAAA|69404|19|16|44|PM|second|evening|dinner| +69405|AAAAAAAAOBPABAAA|69405|19|16|45|PM|second|evening|dinner| +69406|AAAAAAAAPBPABAAA|69406|19|16|46|PM|second|evening|dinner| +69407|AAAAAAAAACPABAAA|69407|19|16|47|PM|second|evening|dinner| +69408|AAAAAAAABCPABAAA|69408|19|16|48|PM|second|evening|dinner| +69409|AAAAAAAACCPABAAA|69409|19|16|49|PM|second|evening|dinner| +69410|AAAAAAAADCPABAAA|69410|19|16|50|PM|second|evening|dinner| +69411|AAAAAAAAECPABAAA|69411|19|16|51|PM|second|evening|dinner| +69412|AAAAAAAAFCPABAAA|69412|19|16|52|PM|second|evening|dinner| +69413|AAAAAAAAGCPABAAA|69413|19|16|53|PM|second|evening|dinner| +69414|AAAAAAAAHCPABAAA|69414|19|16|54|PM|second|evening|dinner| +69415|AAAAAAAAICPABAAA|69415|19|16|55|PM|second|evening|dinner| +69416|AAAAAAAAJCPABAAA|69416|19|16|56|PM|second|evening|dinner| +69417|AAAAAAAAKCPABAAA|69417|19|16|57|PM|second|evening|dinner| +69418|AAAAAAAALCPABAAA|69418|19|16|58|PM|second|evening|dinner| +69419|AAAAAAAAMCPABAAA|69419|19|16|59|PM|second|evening|dinner| +69420|AAAAAAAANCPABAAA|69420|19|17|0|PM|second|evening|dinner| +69421|AAAAAAAAOCPABAAA|69421|19|17|1|PM|second|evening|dinner| +69422|AAAAAAAAPCPABAAA|69422|19|17|2|PM|second|evening|dinner| +69423|AAAAAAAAADPABAAA|69423|19|17|3|PM|second|evening|dinner| +69424|AAAAAAAABDPABAAA|69424|19|17|4|PM|second|evening|dinner| +69425|AAAAAAAACDPABAAA|69425|19|17|5|PM|second|evening|dinner| +69426|AAAAAAAADDPABAAA|69426|19|17|6|PM|second|evening|dinner| +69427|AAAAAAAAEDPABAAA|69427|19|17|7|PM|second|evening|dinner| +69428|AAAAAAAAFDPABAAA|69428|19|17|8|PM|second|evening|dinner| +69429|AAAAAAAAGDPABAAA|69429|19|17|9|PM|second|evening|dinner| +69430|AAAAAAAAHDPABAAA|69430|19|17|10|PM|second|evening|dinner| +69431|AAAAAAAAIDPABAAA|69431|19|17|11|PM|second|evening|dinner| +69432|AAAAAAAAJDPABAAA|69432|19|17|12|PM|second|evening|dinner| +69433|AAAAAAAAKDPABAAA|69433|19|17|13|PM|second|evening|dinner| +69434|AAAAAAAALDPABAAA|69434|19|17|14|PM|second|evening|dinner| +69435|AAAAAAAAMDPABAAA|69435|19|17|15|PM|second|evening|dinner| +69436|AAAAAAAANDPABAAA|69436|19|17|16|PM|second|evening|dinner| +69437|AAAAAAAAODPABAAA|69437|19|17|17|PM|second|evening|dinner| +69438|AAAAAAAAPDPABAAA|69438|19|17|18|PM|second|evening|dinner| +69439|AAAAAAAAAEPABAAA|69439|19|17|19|PM|second|evening|dinner| +69440|AAAAAAAABEPABAAA|69440|19|17|20|PM|second|evening|dinner| +69441|AAAAAAAACEPABAAA|69441|19|17|21|PM|second|evening|dinner| +69442|AAAAAAAADEPABAAA|69442|19|17|22|PM|second|evening|dinner| +69443|AAAAAAAAEEPABAAA|69443|19|17|23|PM|second|evening|dinner| +69444|AAAAAAAAFEPABAAA|69444|19|17|24|PM|second|evening|dinner| +69445|AAAAAAAAGEPABAAA|69445|19|17|25|PM|second|evening|dinner| +69446|AAAAAAAAHEPABAAA|69446|19|17|26|PM|second|evening|dinner| +69447|AAAAAAAAIEPABAAA|69447|19|17|27|PM|second|evening|dinner| +69448|AAAAAAAAJEPABAAA|69448|19|17|28|PM|second|evening|dinner| +69449|AAAAAAAAKEPABAAA|69449|19|17|29|PM|second|evening|dinner| +69450|AAAAAAAALEPABAAA|69450|19|17|30|PM|second|evening|dinner| +69451|AAAAAAAAMEPABAAA|69451|19|17|31|PM|second|evening|dinner| +69452|AAAAAAAANEPABAAA|69452|19|17|32|PM|second|evening|dinner| +69453|AAAAAAAAOEPABAAA|69453|19|17|33|PM|second|evening|dinner| +69454|AAAAAAAAPEPABAAA|69454|19|17|34|PM|second|evening|dinner| +69455|AAAAAAAAAFPABAAA|69455|19|17|35|PM|second|evening|dinner| +69456|AAAAAAAABFPABAAA|69456|19|17|36|PM|second|evening|dinner| +69457|AAAAAAAACFPABAAA|69457|19|17|37|PM|second|evening|dinner| +69458|AAAAAAAADFPABAAA|69458|19|17|38|PM|second|evening|dinner| +69459|AAAAAAAAEFPABAAA|69459|19|17|39|PM|second|evening|dinner| +69460|AAAAAAAAFFPABAAA|69460|19|17|40|PM|second|evening|dinner| +69461|AAAAAAAAGFPABAAA|69461|19|17|41|PM|second|evening|dinner| +69462|AAAAAAAAHFPABAAA|69462|19|17|42|PM|second|evening|dinner| +69463|AAAAAAAAIFPABAAA|69463|19|17|43|PM|second|evening|dinner| +69464|AAAAAAAAJFPABAAA|69464|19|17|44|PM|second|evening|dinner| +69465|AAAAAAAAKFPABAAA|69465|19|17|45|PM|second|evening|dinner| +69466|AAAAAAAALFPABAAA|69466|19|17|46|PM|second|evening|dinner| +69467|AAAAAAAAMFPABAAA|69467|19|17|47|PM|second|evening|dinner| +69468|AAAAAAAANFPABAAA|69468|19|17|48|PM|second|evening|dinner| +69469|AAAAAAAAOFPABAAA|69469|19|17|49|PM|second|evening|dinner| +69470|AAAAAAAAPFPABAAA|69470|19|17|50|PM|second|evening|dinner| +69471|AAAAAAAAAGPABAAA|69471|19|17|51|PM|second|evening|dinner| +69472|AAAAAAAABGPABAAA|69472|19|17|52|PM|second|evening|dinner| +69473|AAAAAAAACGPABAAA|69473|19|17|53|PM|second|evening|dinner| +69474|AAAAAAAADGPABAAA|69474|19|17|54|PM|second|evening|dinner| +69475|AAAAAAAAEGPABAAA|69475|19|17|55|PM|second|evening|dinner| +69476|AAAAAAAAFGPABAAA|69476|19|17|56|PM|second|evening|dinner| +69477|AAAAAAAAGGPABAAA|69477|19|17|57|PM|second|evening|dinner| +69478|AAAAAAAAHGPABAAA|69478|19|17|58|PM|second|evening|dinner| +69479|AAAAAAAAIGPABAAA|69479|19|17|59|PM|second|evening|dinner| +69480|AAAAAAAAJGPABAAA|69480|19|18|0|PM|second|evening|dinner| +69481|AAAAAAAAKGPABAAA|69481|19|18|1|PM|second|evening|dinner| +69482|AAAAAAAALGPABAAA|69482|19|18|2|PM|second|evening|dinner| +69483|AAAAAAAAMGPABAAA|69483|19|18|3|PM|second|evening|dinner| +69484|AAAAAAAANGPABAAA|69484|19|18|4|PM|second|evening|dinner| +69485|AAAAAAAAOGPABAAA|69485|19|18|5|PM|second|evening|dinner| +69486|AAAAAAAAPGPABAAA|69486|19|18|6|PM|second|evening|dinner| +69487|AAAAAAAAAHPABAAA|69487|19|18|7|PM|second|evening|dinner| +69488|AAAAAAAABHPABAAA|69488|19|18|8|PM|second|evening|dinner| +69489|AAAAAAAACHPABAAA|69489|19|18|9|PM|second|evening|dinner| +69490|AAAAAAAADHPABAAA|69490|19|18|10|PM|second|evening|dinner| +69491|AAAAAAAAEHPABAAA|69491|19|18|11|PM|second|evening|dinner| +69492|AAAAAAAAFHPABAAA|69492|19|18|12|PM|second|evening|dinner| +69493|AAAAAAAAGHPABAAA|69493|19|18|13|PM|second|evening|dinner| +69494|AAAAAAAAHHPABAAA|69494|19|18|14|PM|second|evening|dinner| +69495|AAAAAAAAIHPABAAA|69495|19|18|15|PM|second|evening|dinner| +69496|AAAAAAAAJHPABAAA|69496|19|18|16|PM|second|evening|dinner| +69497|AAAAAAAAKHPABAAA|69497|19|18|17|PM|second|evening|dinner| +69498|AAAAAAAALHPABAAA|69498|19|18|18|PM|second|evening|dinner| +69499|AAAAAAAAMHPABAAA|69499|19|18|19|PM|second|evening|dinner| +69500|AAAAAAAANHPABAAA|69500|19|18|20|PM|second|evening|dinner| +69501|AAAAAAAAOHPABAAA|69501|19|18|21|PM|second|evening|dinner| +69502|AAAAAAAAPHPABAAA|69502|19|18|22|PM|second|evening|dinner| +69503|AAAAAAAAAIPABAAA|69503|19|18|23|PM|second|evening|dinner| +69504|AAAAAAAABIPABAAA|69504|19|18|24|PM|second|evening|dinner| +69505|AAAAAAAACIPABAAA|69505|19|18|25|PM|second|evening|dinner| +69506|AAAAAAAADIPABAAA|69506|19|18|26|PM|second|evening|dinner| +69507|AAAAAAAAEIPABAAA|69507|19|18|27|PM|second|evening|dinner| +69508|AAAAAAAAFIPABAAA|69508|19|18|28|PM|second|evening|dinner| +69509|AAAAAAAAGIPABAAA|69509|19|18|29|PM|second|evening|dinner| +69510|AAAAAAAAHIPABAAA|69510|19|18|30|PM|second|evening|dinner| +69511|AAAAAAAAIIPABAAA|69511|19|18|31|PM|second|evening|dinner| +69512|AAAAAAAAJIPABAAA|69512|19|18|32|PM|second|evening|dinner| +69513|AAAAAAAAKIPABAAA|69513|19|18|33|PM|second|evening|dinner| +69514|AAAAAAAALIPABAAA|69514|19|18|34|PM|second|evening|dinner| +69515|AAAAAAAAMIPABAAA|69515|19|18|35|PM|second|evening|dinner| +69516|AAAAAAAANIPABAAA|69516|19|18|36|PM|second|evening|dinner| +69517|AAAAAAAAOIPABAAA|69517|19|18|37|PM|second|evening|dinner| +69518|AAAAAAAAPIPABAAA|69518|19|18|38|PM|second|evening|dinner| +69519|AAAAAAAAAJPABAAA|69519|19|18|39|PM|second|evening|dinner| +69520|AAAAAAAABJPABAAA|69520|19|18|40|PM|second|evening|dinner| +69521|AAAAAAAACJPABAAA|69521|19|18|41|PM|second|evening|dinner| +69522|AAAAAAAADJPABAAA|69522|19|18|42|PM|second|evening|dinner| +69523|AAAAAAAAEJPABAAA|69523|19|18|43|PM|second|evening|dinner| +69524|AAAAAAAAFJPABAAA|69524|19|18|44|PM|second|evening|dinner| +69525|AAAAAAAAGJPABAAA|69525|19|18|45|PM|second|evening|dinner| +69526|AAAAAAAAHJPABAAA|69526|19|18|46|PM|second|evening|dinner| +69527|AAAAAAAAIJPABAAA|69527|19|18|47|PM|second|evening|dinner| +69528|AAAAAAAAJJPABAAA|69528|19|18|48|PM|second|evening|dinner| +69529|AAAAAAAAKJPABAAA|69529|19|18|49|PM|second|evening|dinner| +69530|AAAAAAAALJPABAAA|69530|19|18|50|PM|second|evening|dinner| +69531|AAAAAAAAMJPABAAA|69531|19|18|51|PM|second|evening|dinner| +69532|AAAAAAAANJPABAAA|69532|19|18|52|PM|second|evening|dinner| +69533|AAAAAAAAOJPABAAA|69533|19|18|53|PM|second|evening|dinner| +69534|AAAAAAAAPJPABAAA|69534|19|18|54|PM|second|evening|dinner| +69535|AAAAAAAAAKPABAAA|69535|19|18|55|PM|second|evening|dinner| +69536|AAAAAAAABKPABAAA|69536|19|18|56|PM|second|evening|dinner| +69537|AAAAAAAACKPABAAA|69537|19|18|57|PM|second|evening|dinner| +69538|AAAAAAAADKPABAAA|69538|19|18|58|PM|second|evening|dinner| +69539|AAAAAAAAEKPABAAA|69539|19|18|59|PM|second|evening|dinner| +69540|AAAAAAAAFKPABAAA|69540|19|19|0|PM|second|evening|dinner| +69541|AAAAAAAAGKPABAAA|69541|19|19|1|PM|second|evening|dinner| +69542|AAAAAAAAHKPABAAA|69542|19|19|2|PM|second|evening|dinner| +69543|AAAAAAAAIKPABAAA|69543|19|19|3|PM|second|evening|dinner| +69544|AAAAAAAAJKPABAAA|69544|19|19|4|PM|second|evening|dinner| +69545|AAAAAAAAKKPABAAA|69545|19|19|5|PM|second|evening|dinner| +69546|AAAAAAAALKPABAAA|69546|19|19|6|PM|second|evening|dinner| +69547|AAAAAAAAMKPABAAA|69547|19|19|7|PM|second|evening|dinner| +69548|AAAAAAAANKPABAAA|69548|19|19|8|PM|second|evening|dinner| +69549|AAAAAAAAOKPABAAA|69549|19|19|9|PM|second|evening|dinner| +69550|AAAAAAAAPKPABAAA|69550|19|19|10|PM|second|evening|dinner| +69551|AAAAAAAAALPABAAA|69551|19|19|11|PM|second|evening|dinner| +69552|AAAAAAAABLPABAAA|69552|19|19|12|PM|second|evening|dinner| +69553|AAAAAAAACLPABAAA|69553|19|19|13|PM|second|evening|dinner| +69554|AAAAAAAADLPABAAA|69554|19|19|14|PM|second|evening|dinner| +69555|AAAAAAAAELPABAAA|69555|19|19|15|PM|second|evening|dinner| +69556|AAAAAAAAFLPABAAA|69556|19|19|16|PM|second|evening|dinner| +69557|AAAAAAAAGLPABAAA|69557|19|19|17|PM|second|evening|dinner| +69558|AAAAAAAAHLPABAAA|69558|19|19|18|PM|second|evening|dinner| +69559|AAAAAAAAILPABAAA|69559|19|19|19|PM|second|evening|dinner| +69560|AAAAAAAAJLPABAAA|69560|19|19|20|PM|second|evening|dinner| +69561|AAAAAAAAKLPABAAA|69561|19|19|21|PM|second|evening|dinner| +69562|AAAAAAAALLPABAAA|69562|19|19|22|PM|second|evening|dinner| +69563|AAAAAAAAMLPABAAA|69563|19|19|23|PM|second|evening|dinner| +69564|AAAAAAAANLPABAAA|69564|19|19|24|PM|second|evening|dinner| +69565|AAAAAAAAOLPABAAA|69565|19|19|25|PM|second|evening|dinner| +69566|AAAAAAAAPLPABAAA|69566|19|19|26|PM|second|evening|dinner| +69567|AAAAAAAAAMPABAAA|69567|19|19|27|PM|second|evening|dinner| +69568|AAAAAAAABMPABAAA|69568|19|19|28|PM|second|evening|dinner| +69569|AAAAAAAACMPABAAA|69569|19|19|29|PM|second|evening|dinner| +69570|AAAAAAAADMPABAAA|69570|19|19|30|PM|second|evening|dinner| +69571|AAAAAAAAEMPABAAA|69571|19|19|31|PM|second|evening|dinner| +69572|AAAAAAAAFMPABAAA|69572|19|19|32|PM|second|evening|dinner| +69573|AAAAAAAAGMPABAAA|69573|19|19|33|PM|second|evening|dinner| +69574|AAAAAAAAHMPABAAA|69574|19|19|34|PM|second|evening|dinner| +69575|AAAAAAAAIMPABAAA|69575|19|19|35|PM|second|evening|dinner| +69576|AAAAAAAAJMPABAAA|69576|19|19|36|PM|second|evening|dinner| +69577|AAAAAAAAKMPABAAA|69577|19|19|37|PM|second|evening|dinner| +69578|AAAAAAAALMPABAAA|69578|19|19|38|PM|second|evening|dinner| +69579|AAAAAAAAMMPABAAA|69579|19|19|39|PM|second|evening|dinner| +69580|AAAAAAAANMPABAAA|69580|19|19|40|PM|second|evening|dinner| +69581|AAAAAAAAOMPABAAA|69581|19|19|41|PM|second|evening|dinner| +69582|AAAAAAAAPMPABAAA|69582|19|19|42|PM|second|evening|dinner| +69583|AAAAAAAAANPABAAA|69583|19|19|43|PM|second|evening|dinner| +69584|AAAAAAAABNPABAAA|69584|19|19|44|PM|second|evening|dinner| +69585|AAAAAAAACNPABAAA|69585|19|19|45|PM|second|evening|dinner| +69586|AAAAAAAADNPABAAA|69586|19|19|46|PM|second|evening|dinner| +69587|AAAAAAAAENPABAAA|69587|19|19|47|PM|second|evening|dinner| +69588|AAAAAAAAFNPABAAA|69588|19|19|48|PM|second|evening|dinner| +69589|AAAAAAAAGNPABAAA|69589|19|19|49|PM|second|evening|dinner| +69590|AAAAAAAAHNPABAAA|69590|19|19|50|PM|second|evening|dinner| +69591|AAAAAAAAINPABAAA|69591|19|19|51|PM|second|evening|dinner| +69592|AAAAAAAAJNPABAAA|69592|19|19|52|PM|second|evening|dinner| +69593|AAAAAAAAKNPABAAA|69593|19|19|53|PM|second|evening|dinner| +69594|AAAAAAAALNPABAAA|69594|19|19|54|PM|second|evening|dinner| +69595|AAAAAAAAMNPABAAA|69595|19|19|55|PM|second|evening|dinner| +69596|AAAAAAAANNPABAAA|69596|19|19|56|PM|second|evening|dinner| +69597|AAAAAAAAONPABAAA|69597|19|19|57|PM|second|evening|dinner| +69598|AAAAAAAAPNPABAAA|69598|19|19|58|PM|second|evening|dinner| +69599|AAAAAAAAAOPABAAA|69599|19|19|59|PM|second|evening|dinner| +69600|AAAAAAAABOPABAAA|69600|19|20|0|PM|second|evening|dinner| +69601|AAAAAAAACOPABAAA|69601|19|20|1|PM|second|evening|dinner| +69602|AAAAAAAADOPABAAA|69602|19|20|2|PM|second|evening|dinner| +69603|AAAAAAAAEOPABAAA|69603|19|20|3|PM|second|evening|dinner| +69604|AAAAAAAAFOPABAAA|69604|19|20|4|PM|second|evening|dinner| +69605|AAAAAAAAGOPABAAA|69605|19|20|5|PM|second|evening|dinner| +69606|AAAAAAAAHOPABAAA|69606|19|20|6|PM|second|evening|dinner| +69607|AAAAAAAAIOPABAAA|69607|19|20|7|PM|second|evening|dinner| +69608|AAAAAAAAJOPABAAA|69608|19|20|8|PM|second|evening|dinner| +69609|AAAAAAAAKOPABAAA|69609|19|20|9|PM|second|evening|dinner| +69610|AAAAAAAALOPABAAA|69610|19|20|10|PM|second|evening|dinner| +69611|AAAAAAAAMOPABAAA|69611|19|20|11|PM|second|evening|dinner| +69612|AAAAAAAANOPABAAA|69612|19|20|12|PM|second|evening|dinner| +69613|AAAAAAAAOOPABAAA|69613|19|20|13|PM|second|evening|dinner| +69614|AAAAAAAAPOPABAAA|69614|19|20|14|PM|second|evening|dinner| +69615|AAAAAAAAAPPABAAA|69615|19|20|15|PM|second|evening|dinner| +69616|AAAAAAAABPPABAAA|69616|19|20|16|PM|second|evening|dinner| +69617|AAAAAAAACPPABAAA|69617|19|20|17|PM|second|evening|dinner| +69618|AAAAAAAADPPABAAA|69618|19|20|18|PM|second|evening|dinner| +69619|AAAAAAAAEPPABAAA|69619|19|20|19|PM|second|evening|dinner| +69620|AAAAAAAAFPPABAAA|69620|19|20|20|PM|second|evening|dinner| +69621|AAAAAAAAGPPABAAA|69621|19|20|21|PM|second|evening|dinner| +69622|AAAAAAAAHPPABAAA|69622|19|20|22|PM|second|evening|dinner| +69623|AAAAAAAAIPPABAAA|69623|19|20|23|PM|second|evening|dinner| +69624|AAAAAAAAJPPABAAA|69624|19|20|24|PM|second|evening|dinner| +69625|AAAAAAAAKPPABAAA|69625|19|20|25|PM|second|evening|dinner| +69626|AAAAAAAALPPABAAA|69626|19|20|26|PM|second|evening|dinner| +69627|AAAAAAAAMPPABAAA|69627|19|20|27|PM|second|evening|dinner| +69628|AAAAAAAANPPABAAA|69628|19|20|28|PM|second|evening|dinner| +69629|AAAAAAAAOPPABAAA|69629|19|20|29|PM|second|evening|dinner| +69630|AAAAAAAAPPPABAAA|69630|19|20|30|PM|second|evening|dinner| +69631|AAAAAAAAAAABBAAA|69631|19|20|31|PM|second|evening|dinner| +69632|AAAAAAAABAABBAAA|69632|19|20|32|PM|second|evening|dinner| +69633|AAAAAAAACAABBAAA|69633|19|20|33|PM|second|evening|dinner| +69634|AAAAAAAADAABBAAA|69634|19|20|34|PM|second|evening|dinner| +69635|AAAAAAAAEAABBAAA|69635|19|20|35|PM|second|evening|dinner| +69636|AAAAAAAAFAABBAAA|69636|19|20|36|PM|second|evening|dinner| +69637|AAAAAAAAGAABBAAA|69637|19|20|37|PM|second|evening|dinner| +69638|AAAAAAAAHAABBAAA|69638|19|20|38|PM|second|evening|dinner| +69639|AAAAAAAAIAABBAAA|69639|19|20|39|PM|second|evening|dinner| +69640|AAAAAAAAJAABBAAA|69640|19|20|40|PM|second|evening|dinner| +69641|AAAAAAAAKAABBAAA|69641|19|20|41|PM|second|evening|dinner| +69642|AAAAAAAALAABBAAA|69642|19|20|42|PM|second|evening|dinner| +69643|AAAAAAAAMAABBAAA|69643|19|20|43|PM|second|evening|dinner| +69644|AAAAAAAANAABBAAA|69644|19|20|44|PM|second|evening|dinner| +69645|AAAAAAAAOAABBAAA|69645|19|20|45|PM|second|evening|dinner| +69646|AAAAAAAAPAABBAAA|69646|19|20|46|PM|second|evening|dinner| +69647|AAAAAAAAABABBAAA|69647|19|20|47|PM|second|evening|dinner| +69648|AAAAAAAABBABBAAA|69648|19|20|48|PM|second|evening|dinner| +69649|AAAAAAAACBABBAAA|69649|19|20|49|PM|second|evening|dinner| +69650|AAAAAAAADBABBAAA|69650|19|20|50|PM|second|evening|dinner| +69651|AAAAAAAAEBABBAAA|69651|19|20|51|PM|second|evening|dinner| +69652|AAAAAAAAFBABBAAA|69652|19|20|52|PM|second|evening|dinner| +69653|AAAAAAAAGBABBAAA|69653|19|20|53|PM|second|evening|dinner| +69654|AAAAAAAAHBABBAAA|69654|19|20|54|PM|second|evening|dinner| +69655|AAAAAAAAIBABBAAA|69655|19|20|55|PM|second|evening|dinner| +69656|AAAAAAAAJBABBAAA|69656|19|20|56|PM|second|evening|dinner| +69657|AAAAAAAAKBABBAAA|69657|19|20|57|PM|second|evening|dinner| +69658|AAAAAAAALBABBAAA|69658|19|20|58|PM|second|evening|dinner| +69659|AAAAAAAAMBABBAAA|69659|19|20|59|PM|second|evening|dinner| +69660|AAAAAAAANBABBAAA|69660|19|21|0|PM|second|evening|dinner| +69661|AAAAAAAAOBABBAAA|69661|19|21|1|PM|second|evening|dinner| +69662|AAAAAAAAPBABBAAA|69662|19|21|2|PM|second|evening|dinner| +69663|AAAAAAAAACABBAAA|69663|19|21|3|PM|second|evening|dinner| +69664|AAAAAAAABCABBAAA|69664|19|21|4|PM|second|evening|dinner| +69665|AAAAAAAACCABBAAA|69665|19|21|5|PM|second|evening|dinner| +69666|AAAAAAAADCABBAAA|69666|19|21|6|PM|second|evening|dinner| +69667|AAAAAAAAECABBAAA|69667|19|21|7|PM|second|evening|dinner| +69668|AAAAAAAAFCABBAAA|69668|19|21|8|PM|second|evening|dinner| +69669|AAAAAAAAGCABBAAA|69669|19|21|9|PM|second|evening|dinner| +69670|AAAAAAAAHCABBAAA|69670|19|21|10|PM|second|evening|dinner| +69671|AAAAAAAAICABBAAA|69671|19|21|11|PM|second|evening|dinner| +69672|AAAAAAAAJCABBAAA|69672|19|21|12|PM|second|evening|dinner| +69673|AAAAAAAAKCABBAAA|69673|19|21|13|PM|second|evening|dinner| +69674|AAAAAAAALCABBAAA|69674|19|21|14|PM|second|evening|dinner| +69675|AAAAAAAAMCABBAAA|69675|19|21|15|PM|second|evening|dinner| +69676|AAAAAAAANCABBAAA|69676|19|21|16|PM|second|evening|dinner| +69677|AAAAAAAAOCABBAAA|69677|19|21|17|PM|second|evening|dinner| +69678|AAAAAAAAPCABBAAA|69678|19|21|18|PM|second|evening|dinner| +69679|AAAAAAAAADABBAAA|69679|19|21|19|PM|second|evening|dinner| +69680|AAAAAAAABDABBAAA|69680|19|21|20|PM|second|evening|dinner| +69681|AAAAAAAACDABBAAA|69681|19|21|21|PM|second|evening|dinner| +69682|AAAAAAAADDABBAAA|69682|19|21|22|PM|second|evening|dinner| +69683|AAAAAAAAEDABBAAA|69683|19|21|23|PM|second|evening|dinner| +69684|AAAAAAAAFDABBAAA|69684|19|21|24|PM|second|evening|dinner| +69685|AAAAAAAAGDABBAAA|69685|19|21|25|PM|second|evening|dinner| +69686|AAAAAAAAHDABBAAA|69686|19|21|26|PM|second|evening|dinner| +69687|AAAAAAAAIDABBAAA|69687|19|21|27|PM|second|evening|dinner| +69688|AAAAAAAAJDABBAAA|69688|19|21|28|PM|second|evening|dinner| +69689|AAAAAAAAKDABBAAA|69689|19|21|29|PM|second|evening|dinner| +69690|AAAAAAAALDABBAAA|69690|19|21|30|PM|second|evening|dinner| +69691|AAAAAAAAMDABBAAA|69691|19|21|31|PM|second|evening|dinner| +69692|AAAAAAAANDABBAAA|69692|19|21|32|PM|second|evening|dinner| +69693|AAAAAAAAODABBAAA|69693|19|21|33|PM|second|evening|dinner| +69694|AAAAAAAAPDABBAAA|69694|19|21|34|PM|second|evening|dinner| +69695|AAAAAAAAAEABBAAA|69695|19|21|35|PM|second|evening|dinner| +69696|AAAAAAAABEABBAAA|69696|19|21|36|PM|second|evening|dinner| +69697|AAAAAAAACEABBAAA|69697|19|21|37|PM|second|evening|dinner| +69698|AAAAAAAADEABBAAA|69698|19|21|38|PM|second|evening|dinner| +69699|AAAAAAAAEEABBAAA|69699|19|21|39|PM|second|evening|dinner| +69700|AAAAAAAAFEABBAAA|69700|19|21|40|PM|second|evening|dinner| +69701|AAAAAAAAGEABBAAA|69701|19|21|41|PM|second|evening|dinner| +69702|AAAAAAAAHEABBAAA|69702|19|21|42|PM|second|evening|dinner| +69703|AAAAAAAAIEABBAAA|69703|19|21|43|PM|second|evening|dinner| +69704|AAAAAAAAJEABBAAA|69704|19|21|44|PM|second|evening|dinner| +69705|AAAAAAAAKEABBAAA|69705|19|21|45|PM|second|evening|dinner| +69706|AAAAAAAALEABBAAA|69706|19|21|46|PM|second|evening|dinner| +69707|AAAAAAAAMEABBAAA|69707|19|21|47|PM|second|evening|dinner| +69708|AAAAAAAANEABBAAA|69708|19|21|48|PM|second|evening|dinner| +69709|AAAAAAAAOEABBAAA|69709|19|21|49|PM|second|evening|dinner| +69710|AAAAAAAAPEABBAAA|69710|19|21|50|PM|second|evening|dinner| +69711|AAAAAAAAAFABBAAA|69711|19|21|51|PM|second|evening|dinner| +69712|AAAAAAAABFABBAAA|69712|19|21|52|PM|second|evening|dinner| +69713|AAAAAAAACFABBAAA|69713|19|21|53|PM|second|evening|dinner| +69714|AAAAAAAADFABBAAA|69714|19|21|54|PM|second|evening|dinner| +69715|AAAAAAAAEFABBAAA|69715|19|21|55|PM|second|evening|dinner| +69716|AAAAAAAAFFABBAAA|69716|19|21|56|PM|second|evening|dinner| +69717|AAAAAAAAGFABBAAA|69717|19|21|57|PM|second|evening|dinner| +69718|AAAAAAAAHFABBAAA|69718|19|21|58|PM|second|evening|dinner| +69719|AAAAAAAAIFABBAAA|69719|19|21|59|PM|second|evening|dinner| +69720|AAAAAAAAJFABBAAA|69720|19|22|0|PM|second|evening|dinner| +69721|AAAAAAAAKFABBAAA|69721|19|22|1|PM|second|evening|dinner| +69722|AAAAAAAALFABBAAA|69722|19|22|2|PM|second|evening|dinner| +69723|AAAAAAAAMFABBAAA|69723|19|22|3|PM|second|evening|dinner| +69724|AAAAAAAANFABBAAA|69724|19|22|4|PM|second|evening|dinner| +69725|AAAAAAAAOFABBAAA|69725|19|22|5|PM|second|evening|dinner| +69726|AAAAAAAAPFABBAAA|69726|19|22|6|PM|second|evening|dinner| +69727|AAAAAAAAAGABBAAA|69727|19|22|7|PM|second|evening|dinner| +69728|AAAAAAAABGABBAAA|69728|19|22|8|PM|second|evening|dinner| +69729|AAAAAAAACGABBAAA|69729|19|22|9|PM|second|evening|dinner| +69730|AAAAAAAADGABBAAA|69730|19|22|10|PM|second|evening|dinner| +69731|AAAAAAAAEGABBAAA|69731|19|22|11|PM|second|evening|dinner| +69732|AAAAAAAAFGABBAAA|69732|19|22|12|PM|second|evening|dinner| +69733|AAAAAAAAGGABBAAA|69733|19|22|13|PM|second|evening|dinner| +69734|AAAAAAAAHGABBAAA|69734|19|22|14|PM|second|evening|dinner| +69735|AAAAAAAAIGABBAAA|69735|19|22|15|PM|second|evening|dinner| +69736|AAAAAAAAJGABBAAA|69736|19|22|16|PM|second|evening|dinner| +69737|AAAAAAAAKGABBAAA|69737|19|22|17|PM|second|evening|dinner| +69738|AAAAAAAALGABBAAA|69738|19|22|18|PM|second|evening|dinner| +69739|AAAAAAAAMGABBAAA|69739|19|22|19|PM|second|evening|dinner| +69740|AAAAAAAANGABBAAA|69740|19|22|20|PM|second|evening|dinner| +69741|AAAAAAAAOGABBAAA|69741|19|22|21|PM|second|evening|dinner| +69742|AAAAAAAAPGABBAAA|69742|19|22|22|PM|second|evening|dinner| +69743|AAAAAAAAAHABBAAA|69743|19|22|23|PM|second|evening|dinner| +69744|AAAAAAAABHABBAAA|69744|19|22|24|PM|second|evening|dinner| +69745|AAAAAAAACHABBAAA|69745|19|22|25|PM|second|evening|dinner| +69746|AAAAAAAADHABBAAA|69746|19|22|26|PM|second|evening|dinner| +69747|AAAAAAAAEHABBAAA|69747|19|22|27|PM|second|evening|dinner| +69748|AAAAAAAAFHABBAAA|69748|19|22|28|PM|second|evening|dinner| +69749|AAAAAAAAGHABBAAA|69749|19|22|29|PM|second|evening|dinner| +69750|AAAAAAAAHHABBAAA|69750|19|22|30|PM|second|evening|dinner| +69751|AAAAAAAAIHABBAAA|69751|19|22|31|PM|second|evening|dinner| +69752|AAAAAAAAJHABBAAA|69752|19|22|32|PM|second|evening|dinner| +69753|AAAAAAAAKHABBAAA|69753|19|22|33|PM|second|evening|dinner| +69754|AAAAAAAALHABBAAA|69754|19|22|34|PM|second|evening|dinner| +69755|AAAAAAAAMHABBAAA|69755|19|22|35|PM|second|evening|dinner| +69756|AAAAAAAANHABBAAA|69756|19|22|36|PM|second|evening|dinner| +69757|AAAAAAAAOHABBAAA|69757|19|22|37|PM|second|evening|dinner| +69758|AAAAAAAAPHABBAAA|69758|19|22|38|PM|second|evening|dinner| +69759|AAAAAAAAAIABBAAA|69759|19|22|39|PM|second|evening|dinner| +69760|AAAAAAAABIABBAAA|69760|19|22|40|PM|second|evening|dinner| +69761|AAAAAAAACIABBAAA|69761|19|22|41|PM|second|evening|dinner| +69762|AAAAAAAADIABBAAA|69762|19|22|42|PM|second|evening|dinner| +69763|AAAAAAAAEIABBAAA|69763|19|22|43|PM|second|evening|dinner| +69764|AAAAAAAAFIABBAAA|69764|19|22|44|PM|second|evening|dinner| +69765|AAAAAAAAGIABBAAA|69765|19|22|45|PM|second|evening|dinner| +69766|AAAAAAAAHIABBAAA|69766|19|22|46|PM|second|evening|dinner| +69767|AAAAAAAAIIABBAAA|69767|19|22|47|PM|second|evening|dinner| +69768|AAAAAAAAJIABBAAA|69768|19|22|48|PM|second|evening|dinner| +69769|AAAAAAAAKIABBAAA|69769|19|22|49|PM|second|evening|dinner| +69770|AAAAAAAALIABBAAA|69770|19|22|50|PM|second|evening|dinner| +69771|AAAAAAAAMIABBAAA|69771|19|22|51|PM|second|evening|dinner| +69772|AAAAAAAANIABBAAA|69772|19|22|52|PM|second|evening|dinner| +69773|AAAAAAAAOIABBAAA|69773|19|22|53|PM|second|evening|dinner| +69774|AAAAAAAAPIABBAAA|69774|19|22|54|PM|second|evening|dinner| +69775|AAAAAAAAAJABBAAA|69775|19|22|55|PM|second|evening|dinner| +69776|AAAAAAAABJABBAAA|69776|19|22|56|PM|second|evening|dinner| +69777|AAAAAAAACJABBAAA|69777|19|22|57|PM|second|evening|dinner| +69778|AAAAAAAADJABBAAA|69778|19|22|58|PM|second|evening|dinner| +69779|AAAAAAAAEJABBAAA|69779|19|22|59|PM|second|evening|dinner| +69780|AAAAAAAAFJABBAAA|69780|19|23|0|PM|second|evening|dinner| +69781|AAAAAAAAGJABBAAA|69781|19|23|1|PM|second|evening|dinner| +69782|AAAAAAAAHJABBAAA|69782|19|23|2|PM|second|evening|dinner| +69783|AAAAAAAAIJABBAAA|69783|19|23|3|PM|second|evening|dinner| +69784|AAAAAAAAJJABBAAA|69784|19|23|4|PM|second|evening|dinner| +69785|AAAAAAAAKJABBAAA|69785|19|23|5|PM|second|evening|dinner| +69786|AAAAAAAALJABBAAA|69786|19|23|6|PM|second|evening|dinner| +69787|AAAAAAAAMJABBAAA|69787|19|23|7|PM|second|evening|dinner| +69788|AAAAAAAANJABBAAA|69788|19|23|8|PM|second|evening|dinner| +69789|AAAAAAAAOJABBAAA|69789|19|23|9|PM|second|evening|dinner| +69790|AAAAAAAAPJABBAAA|69790|19|23|10|PM|second|evening|dinner| +69791|AAAAAAAAAKABBAAA|69791|19|23|11|PM|second|evening|dinner| +69792|AAAAAAAABKABBAAA|69792|19|23|12|PM|second|evening|dinner| +69793|AAAAAAAACKABBAAA|69793|19|23|13|PM|second|evening|dinner| +69794|AAAAAAAADKABBAAA|69794|19|23|14|PM|second|evening|dinner| +69795|AAAAAAAAEKABBAAA|69795|19|23|15|PM|second|evening|dinner| +69796|AAAAAAAAFKABBAAA|69796|19|23|16|PM|second|evening|dinner| +69797|AAAAAAAAGKABBAAA|69797|19|23|17|PM|second|evening|dinner| +69798|AAAAAAAAHKABBAAA|69798|19|23|18|PM|second|evening|dinner| +69799|AAAAAAAAIKABBAAA|69799|19|23|19|PM|second|evening|dinner| +69800|AAAAAAAAJKABBAAA|69800|19|23|20|PM|second|evening|dinner| +69801|AAAAAAAAKKABBAAA|69801|19|23|21|PM|second|evening|dinner| +69802|AAAAAAAALKABBAAA|69802|19|23|22|PM|second|evening|dinner| +69803|AAAAAAAAMKABBAAA|69803|19|23|23|PM|second|evening|dinner| +69804|AAAAAAAANKABBAAA|69804|19|23|24|PM|second|evening|dinner| +69805|AAAAAAAAOKABBAAA|69805|19|23|25|PM|second|evening|dinner| +69806|AAAAAAAAPKABBAAA|69806|19|23|26|PM|second|evening|dinner| +69807|AAAAAAAAALABBAAA|69807|19|23|27|PM|second|evening|dinner| +69808|AAAAAAAABLABBAAA|69808|19|23|28|PM|second|evening|dinner| +69809|AAAAAAAACLABBAAA|69809|19|23|29|PM|second|evening|dinner| +69810|AAAAAAAADLABBAAA|69810|19|23|30|PM|second|evening|dinner| +69811|AAAAAAAAELABBAAA|69811|19|23|31|PM|second|evening|dinner| +69812|AAAAAAAAFLABBAAA|69812|19|23|32|PM|second|evening|dinner| +69813|AAAAAAAAGLABBAAA|69813|19|23|33|PM|second|evening|dinner| +69814|AAAAAAAAHLABBAAA|69814|19|23|34|PM|second|evening|dinner| +69815|AAAAAAAAILABBAAA|69815|19|23|35|PM|second|evening|dinner| +69816|AAAAAAAAJLABBAAA|69816|19|23|36|PM|second|evening|dinner| +69817|AAAAAAAAKLABBAAA|69817|19|23|37|PM|second|evening|dinner| +69818|AAAAAAAALLABBAAA|69818|19|23|38|PM|second|evening|dinner| +69819|AAAAAAAAMLABBAAA|69819|19|23|39|PM|second|evening|dinner| +69820|AAAAAAAANLABBAAA|69820|19|23|40|PM|second|evening|dinner| +69821|AAAAAAAAOLABBAAA|69821|19|23|41|PM|second|evening|dinner| +69822|AAAAAAAAPLABBAAA|69822|19|23|42|PM|second|evening|dinner| +69823|AAAAAAAAAMABBAAA|69823|19|23|43|PM|second|evening|dinner| +69824|AAAAAAAABMABBAAA|69824|19|23|44|PM|second|evening|dinner| +69825|AAAAAAAACMABBAAA|69825|19|23|45|PM|second|evening|dinner| +69826|AAAAAAAADMABBAAA|69826|19|23|46|PM|second|evening|dinner| +69827|AAAAAAAAEMABBAAA|69827|19|23|47|PM|second|evening|dinner| +69828|AAAAAAAAFMABBAAA|69828|19|23|48|PM|second|evening|dinner| +69829|AAAAAAAAGMABBAAA|69829|19|23|49|PM|second|evening|dinner| +69830|AAAAAAAAHMABBAAA|69830|19|23|50|PM|second|evening|dinner| +69831|AAAAAAAAIMABBAAA|69831|19|23|51|PM|second|evening|dinner| +69832|AAAAAAAAJMABBAAA|69832|19|23|52|PM|second|evening|dinner| +69833|AAAAAAAAKMABBAAA|69833|19|23|53|PM|second|evening|dinner| +69834|AAAAAAAALMABBAAA|69834|19|23|54|PM|second|evening|dinner| +69835|AAAAAAAAMMABBAAA|69835|19|23|55|PM|second|evening|dinner| +69836|AAAAAAAANMABBAAA|69836|19|23|56|PM|second|evening|dinner| +69837|AAAAAAAAOMABBAAA|69837|19|23|57|PM|second|evening|dinner| +69838|AAAAAAAAPMABBAAA|69838|19|23|58|PM|second|evening|dinner| +69839|AAAAAAAAANABBAAA|69839|19|23|59|PM|second|evening|dinner| +69840|AAAAAAAABNABBAAA|69840|19|24|0|PM|second|evening|dinner| +69841|AAAAAAAACNABBAAA|69841|19|24|1|PM|second|evening|dinner| +69842|AAAAAAAADNABBAAA|69842|19|24|2|PM|second|evening|dinner| +69843|AAAAAAAAENABBAAA|69843|19|24|3|PM|second|evening|dinner| +69844|AAAAAAAAFNABBAAA|69844|19|24|4|PM|second|evening|dinner| +69845|AAAAAAAAGNABBAAA|69845|19|24|5|PM|second|evening|dinner| +69846|AAAAAAAAHNABBAAA|69846|19|24|6|PM|second|evening|dinner| +69847|AAAAAAAAINABBAAA|69847|19|24|7|PM|second|evening|dinner| +69848|AAAAAAAAJNABBAAA|69848|19|24|8|PM|second|evening|dinner| +69849|AAAAAAAAKNABBAAA|69849|19|24|9|PM|second|evening|dinner| +69850|AAAAAAAALNABBAAA|69850|19|24|10|PM|second|evening|dinner| +69851|AAAAAAAAMNABBAAA|69851|19|24|11|PM|second|evening|dinner| +69852|AAAAAAAANNABBAAA|69852|19|24|12|PM|second|evening|dinner| +69853|AAAAAAAAONABBAAA|69853|19|24|13|PM|second|evening|dinner| +69854|AAAAAAAAPNABBAAA|69854|19|24|14|PM|second|evening|dinner| +69855|AAAAAAAAAOABBAAA|69855|19|24|15|PM|second|evening|dinner| +69856|AAAAAAAABOABBAAA|69856|19|24|16|PM|second|evening|dinner| +69857|AAAAAAAACOABBAAA|69857|19|24|17|PM|second|evening|dinner| +69858|AAAAAAAADOABBAAA|69858|19|24|18|PM|second|evening|dinner| +69859|AAAAAAAAEOABBAAA|69859|19|24|19|PM|second|evening|dinner| +69860|AAAAAAAAFOABBAAA|69860|19|24|20|PM|second|evening|dinner| +69861|AAAAAAAAGOABBAAA|69861|19|24|21|PM|second|evening|dinner| +69862|AAAAAAAAHOABBAAA|69862|19|24|22|PM|second|evening|dinner| +69863|AAAAAAAAIOABBAAA|69863|19|24|23|PM|second|evening|dinner| +69864|AAAAAAAAJOABBAAA|69864|19|24|24|PM|second|evening|dinner| +69865|AAAAAAAAKOABBAAA|69865|19|24|25|PM|second|evening|dinner| +69866|AAAAAAAALOABBAAA|69866|19|24|26|PM|second|evening|dinner| +69867|AAAAAAAAMOABBAAA|69867|19|24|27|PM|second|evening|dinner| +69868|AAAAAAAANOABBAAA|69868|19|24|28|PM|second|evening|dinner| +69869|AAAAAAAAOOABBAAA|69869|19|24|29|PM|second|evening|dinner| +69870|AAAAAAAAPOABBAAA|69870|19|24|30|PM|second|evening|dinner| +69871|AAAAAAAAAPABBAAA|69871|19|24|31|PM|second|evening|dinner| +69872|AAAAAAAABPABBAAA|69872|19|24|32|PM|second|evening|dinner| +69873|AAAAAAAACPABBAAA|69873|19|24|33|PM|second|evening|dinner| +69874|AAAAAAAADPABBAAA|69874|19|24|34|PM|second|evening|dinner| +69875|AAAAAAAAEPABBAAA|69875|19|24|35|PM|second|evening|dinner| +69876|AAAAAAAAFPABBAAA|69876|19|24|36|PM|second|evening|dinner| +69877|AAAAAAAAGPABBAAA|69877|19|24|37|PM|second|evening|dinner| +69878|AAAAAAAAHPABBAAA|69878|19|24|38|PM|second|evening|dinner| +69879|AAAAAAAAIPABBAAA|69879|19|24|39|PM|second|evening|dinner| +69880|AAAAAAAAJPABBAAA|69880|19|24|40|PM|second|evening|dinner| +69881|AAAAAAAAKPABBAAA|69881|19|24|41|PM|second|evening|dinner| +69882|AAAAAAAALPABBAAA|69882|19|24|42|PM|second|evening|dinner| +69883|AAAAAAAAMPABBAAA|69883|19|24|43|PM|second|evening|dinner| +69884|AAAAAAAANPABBAAA|69884|19|24|44|PM|second|evening|dinner| +69885|AAAAAAAAOPABBAAA|69885|19|24|45|PM|second|evening|dinner| +69886|AAAAAAAAPPABBAAA|69886|19|24|46|PM|second|evening|dinner| +69887|AAAAAAAAAABBBAAA|69887|19|24|47|PM|second|evening|dinner| +69888|AAAAAAAABABBBAAA|69888|19|24|48|PM|second|evening|dinner| +69889|AAAAAAAACABBBAAA|69889|19|24|49|PM|second|evening|dinner| +69890|AAAAAAAADABBBAAA|69890|19|24|50|PM|second|evening|dinner| +69891|AAAAAAAAEABBBAAA|69891|19|24|51|PM|second|evening|dinner| +69892|AAAAAAAAFABBBAAA|69892|19|24|52|PM|second|evening|dinner| +69893|AAAAAAAAGABBBAAA|69893|19|24|53|PM|second|evening|dinner| +69894|AAAAAAAAHABBBAAA|69894|19|24|54|PM|second|evening|dinner| +69895|AAAAAAAAIABBBAAA|69895|19|24|55|PM|second|evening|dinner| +69896|AAAAAAAAJABBBAAA|69896|19|24|56|PM|second|evening|dinner| +69897|AAAAAAAAKABBBAAA|69897|19|24|57|PM|second|evening|dinner| +69898|AAAAAAAALABBBAAA|69898|19|24|58|PM|second|evening|dinner| +69899|AAAAAAAAMABBBAAA|69899|19|24|59|PM|second|evening|dinner| +69900|AAAAAAAANABBBAAA|69900|19|25|0|PM|second|evening|dinner| +69901|AAAAAAAAOABBBAAA|69901|19|25|1|PM|second|evening|dinner| +69902|AAAAAAAAPABBBAAA|69902|19|25|2|PM|second|evening|dinner| +69903|AAAAAAAAABBBBAAA|69903|19|25|3|PM|second|evening|dinner| +69904|AAAAAAAABBBBBAAA|69904|19|25|4|PM|second|evening|dinner| +69905|AAAAAAAACBBBBAAA|69905|19|25|5|PM|second|evening|dinner| +69906|AAAAAAAADBBBBAAA|69906|19|25|6|PM|second|evening|dinner| +69907|AAAAAAAAEBBBBAAA|69907|19|25|7|PM|second|evening|dinner| +69908|AAAAAAAAFBBBBAAA|69908|19|25|8|PM|second|evening|dinner| +69909|AAAAAAAAGBBBBAAA|69909|19|25|9|PM|second|evening|dinner| +69910|AAAAAAAAHBBBBAAA|69910|19|25|10|PM|second|evening|dinner| +69911|AAAAAAAAIBBBBAAA|69911|19|25|11|PM|second|evening|dinner| +69912|AAAAAAAAJBBBBAAA|69912|19|25|12|PM|second|evening|dinner| +69913|AAAAAAAAKBBBBAAA|69913|19|25|13|PM|second|evening|dinner| +69914|AAAAAAAALBBBBAAA|69914|19|25|14|PM|second|evening|dinner| +69915|AAAAAAAAMBBBBAAA|69915|19|25|15|PM|second|evening|dinner| +69916|AAAAAAAANBBBBAAA|69916|19|25|16|PM|second|evening|dinner| +69917|AAAAAAAAOBBBBAAA|69917|19|25|17|PM|second|evening|dinner| +69918|AAAAAAAAPBBBBAAA|69918|19|25|18|PM|second|evening|dinner| +69919|AAAAAAAAACBBBAAA|69919|19|25|19|PM|second|evening|dinner| +69920|AAAAAAAABCBBBAAA|69920|19|25|20|PM|second|evening|dinner| +69921|AAAAAAAACCBBBAAA|69921|19|25|21|PM|second|evening|dinner| +69922|AAAAAAAADCBBBAAA|69922|19|25|22|PM|second|evening|dinner| +69923|AAAAAAAAECBBBAAA|69923|19|25|23|PM|second|evening|dinner| +69924|AAAAAAAAFCBBBAAA|69924|19|25|24|PM|second|evening|dinner| +69925|AAAAAAAAGCBBBAAA|69925|19|25|25|PM|second|evening|dinner| +69926|AAAAAAAAHCBBBAAA|69926|19|25|26|PM|second|evening|dinner| +69927|AAAAAAAAICBBBAAA|69927|19|25|27|PM|second|evening|dinner| +69928|AAAAAAAAJCBBBAAA|69928|19|25|28|PM|second|evening|dinner| +69929|AAAAAAAAKCBBBAAA|69929|19|25|29|PM|second|evening|dinner| +69930|AAAAAAAALCBBBAAA|69930|19|25|30|PM|second|evening|dinner| +69931|AAAAAAAAMCBBBAAA|69931|19|25|31|PM|second|evening|dinner| +69932|AAAAAAAANCBBBAAA|69932|19|25|32|PM|second|evening|dinner| +69933|AAAAAAAAOCBBBAAA|69933|19|25|33|PM|second|evening|dinner| +69934|AAAAAAAAPCBBBAAA|69934|19|25|34|PM|second|evening|dinner| +69935|AAAAAAAAADBBBAAA|69935|19|25|35|PM|second|evening|dinner| +69936|AAAAAAAABDBBBAAA|69936|19|25|36|PM|second|evening|dinner| +69937|AAAAAAAACDBBBAAA|69937|19|25|37|PM|second|evening|dinner| +69938|AAAAAAAADDBBBAAA|69938|19|25|38|PM|second|evening|dinner| +69939|AAAAAAAAEDBBBAAA|69939|19|25|39|PM|second|evening|dinner| +69940|AAAAAAAAFDBBBAAA|69940|19|25|40|PM|second|evening|dinner| +69941|AAAAAAAAGDBBBAAA|69941|19|25|41|PM|second|evening|dinner| +69942|AAAAAAAAHDBBBAAA|69942|19|25|42|PM|second|evening|dinner| +69943|AAAAAAAAIDBBBAAA|69943|19|25|43|PM|second|evening|dinner| +69944|AAAAAAAAJDBBBAAA|69944|19|25|44|PM|second|evening|dinner| +69945|AAAAAAAAKDBBBAAA|69945|19|25|45|PM|second|evening|dinner| +69946|AAAAAAAALDBBBAAA|69946|19|25|46|PM|second|evening|dinner| +69947|AAAAAAAAMDBBBAAA|69947|19|25|47|PM|second|evening|dinner| +69948|AAAAAAAANDBBBAAA|69948|19|25|48|PM|second|evening|dinner| +69949|AAAAAAAAODBBBAAA|69949|19|25|49|PM|second|evening|dinner| +69950|AAAAAAAAPDBBBAAA|69950|19|25|50|PM|second|evening|dinner| +69951|AAAAAAAAAEBBBAAA|69951|19|25|51|PM|second|evening|dinner| +69952|AAAAAAAABEBBBAAA|69952|19|25|52|PM|second|evening|dinner| +69953|AAAAAAAACEBBBAAA|69953|19|25|53|PM|second|evening|dinner| +69954|AAAAAAAADEBBBAAA|69954|19|25|54|PM|second|evening|dinner| +69955|AAAAAAAAEEBBBAAA|69955|19|25|55|PM|second|evening|dinner| +69956|AAAAAAAAFEBBBAAA|69956|19|25|56|PM|second|evening|dinner| +69957|AAAAAAAAGEBBBAAA|69957|19|25|57|PM|second|evening|dinner| +69958|AAAAAAAAHEBBBAAA|69958|19|25|58|PM|second|evening|dinner| +69959|AAAAAAAAIEBBBAAA|69959|19|25|59|PM|second|evening|dinner| +69960|AAAAAAAAJEBBBAAA|69960|19|26|0|PM|second|evening|dinner| +69961|AAAAAAAAKEBBBAAA|69961|19|26|1|PM|second|evening|dinner| +69962|AAAAAAAALEBBBAAA|69962|19|26|2|PM|second|evening|dinner| +69963|AAAAAAAAMEBBBAAA|69963|19|26|3|PM|second|evening|dinner| +69964|AAAAAAAANEBBBAAA|69964|19|26|4|PM|second|evening|dinner| +69965|AAAAAAAAOEBBBAAA|69965|19|26|5|PM|second|evening|dinner| +69966|AAAAAAAAPEBBBAAA|69966|19|26|6|PM|second|evening|dinner| +69967|AAAAAAAAAFBBBAAA|69967|19|26|7|PM|second|evening|dinner| +69968|AAAAAAAABFBBBAAA|69968|19|26|8|PM|second|evening|dinner| +69969|AAAAAAAACFBBBAAA|69969|19|26|9|PM|second|evening|dinner| +69970|AAAAAAAADFBBBAAA|69970|19|26|10|PM|second|evening|dinner| +69971|AAAAAAAAEFBBBAAA|69971|19|26|11|PM|second|evening|dinner| +69972|AAAAAAAAFFBBBAAA|69972|19|26|12|PM|second|evening|dinner| +69973|AAAAAAAAGFBBBAAA|69973|19|26|13|PM|second|evening|dinner| +69974|AAAAAAAAHFBBBAAA|69974|19|26|14|PM|second|evening|dinner| +69975|AAAAAAAAIFBBBAAA|69975|19|26|15|PM|second|evening|dinner| +69976|AAAAAAAAJFBBBAAA|69976|19|26|16|PM|second|evening|dinner| +69977|AAAAAAAAKFBBBAAA|69977|19|26|17|PM|second|evening|dinner| +69978|AAAAAAAALFBBBAAA|69978|19|26|18|PM|second|evening|dinner| +69979|AAAAAAAAMFBBBAAA|69979|19|26|19|PM|second|evening|dinner| +69980|AAAAAAAANFBBBAAA|69980|19|26|20|PM|second|evening|dinner| +69981|AAAAAAAAOFBBBAAA|69981|19|26|21|PM|second|evening|dinner| +69982|AAAAAAAAPFBBBAAA|69982|19|26|22|PM|second|evening|dinner| +69983|AAAAAAAAAGBBBAAA|69983|19|26|23|PM|second|evening|dinner| +69984|AAAAAAAABGBBBAAA|69984|19|26|24|PM|second|evening|dinner| +69985|AAAAAAAACGBBBAAA|69985|19|26|25|PM|second|evening|dinner| +69986|AAAAAAAADGBBBAAA|69986|19|26|26|PM|second|evening|dinner| +69987|AAAAAAAAEGBBBAAA|69987|19|26|27|PM|second|evening|dinner| +69988|AAAAAAAAFGBBBAAA|69988|19|26|28|PM|second|evening|dinner| +69989|AAAAAAAAGGBBBAAA|69989|19|26|29|PM|second|evening|dinner| +69990|AAAAAAAAHGBBBAAA|69990|19|26|30|PM|second|evening|dinner| +69991|AAAAAAAAIGBBBAAA|69991|19|26|31|PM|second|evening|dinner| +69992|AAAAAAAAJGBBBAAA|69992|19|26|32|PM|second|evening|dinner| +69993|AAAAAAAAKGBBBAAA|69993|19|26|33|PM|second|evening|dinner| +69994|AAAAAAAALGBBBAAA|69994|19|26|34|PM|second|evening|dinner| +69995|AAAAAAAAMGBBBAAA|69995|19|26|35|PM|second|evening|dinner| +69996|AAAAAAAANGBBBAAA|69996|19|26|36|PM|second|evening|dinner| +69997|AAAAAAAAOGBBBAAA|69997|19|26|37|PM|second|evening|dinner| +69998|AAAAAAAAPGBBBAAA|69998|19|26|38|PM|second|evening|dinner| +69999|AAAAAAAAAHBBBAAA|69999|19|26|39|PM|second|evening|dinner| +70000|AAAAAAAABHBBBAAA|70000|19|26|40|PM|second|evening|dinner| +70001|AAAAAAAACHBBBAAA|70001|19|26|41|PM|second|evening|dinner| +70002|AAAAAAAADHBBBAAA|70002|19|26|42|PM|second|evening|dinner| +70003|AAAAAAAAEHBBBAAA|70003|19|26|43|PM|second|evening|dinner| +70004|AAAAAAAAFHBBBAAA|70004|19|26|44|PM|second|evening|dinner| +70005|AAAAAAAAGHBBBAAA|70005|19|26|45|PM|second|evening|dinner| +70006|AAAAAAAAHHBBBAAA|70006|19|26|46|PM|second|evening|dinner| +70007|AAAAAAAAIHBBBAAA|70007|19|26|47|PM|second|evening|dinner| +70008|AAAAAAAAJHBBBAAA|70008|19|26|48|PM|second|evening|dinner| +70009|AAAAAAAAKHBBBAAA|70009|19|26|49|PM|second|evening|dinner| +70010|AAAAAAAALHBBBAAA|70010|19|26|50|PM|second|evening|dinner| +70011|AAAAAAAAMHBBBAAA|70011|19|26|51|PM|second|evening|dinner| +70012|AAAAAAAANHBBBAAA|70012|19|26|52|PM|second|evening|dinner| +70013|AAAAAAAAOHBBBAAA|70013|19|26|53|PM|second|evening|dinner| +70014|AAAAAAAAPHBBBAAA|70014|19|26|54|PM|second|evening|dinner| +70015|AAAAAAAAAIBBBAAA|70015|19|26|55|PM|second|evening|dinner| +70016|AAAAAAAABIBBBAAA|70016|19|26|56|PM|second|evening|dinner| +70017|AAAAAAAACIBBBAAA|70017|19|26|57|PM|second|evening|dinner| +70018|AAAAAAAADIBBBAAA|70018|19|26|58|PM|second|evening|dinner| +70019|AAAAAAAAEIBBBAAA|70019|19|26|59|PM|second|evening|dinner| +70020|AAAAAAAAFIBBBAAA|70020|19|27|0|PM|second|evening|dinner| +70021|AAAAAAAAGIBBBAAA|70021|19|27|1|PM|second|evening|dinner| +70022|AAAAAAAAHIBBBAAA|70022|19|27|2|PM|second|evening|dinner| +70023|AAAAAAAAIIBBBAAA|70023|19|27|3|PM|second|evening|dinner| +70024|AAAAAAAAJIBBBAAA|70024|19|27|4|PM|second|evening|dinner| +70025|AAAAAAAAKIBBBAAA|70025|19|27|5|PM|second|evening|dinner| +70026|AAAAAAAALIBBBAAA|70026|19|27|6|PM|second|evening|dinner| +70027|AAAAAAAAMIBBBAAA|70027|19|27|7|PM|second|evening|dinner| +70028|AAAAAAAANIBBBAAA|70028|19|27|8|PM|second|evening|dinner| +70029|AAAAAAAAOIBBBAAA|70029|19|27|9|PM|second|evening|dinner| +70030|AAAAAAAAPIBBBAAA|70030|19|27|10|PM|second|evening|dinner| +70031|AAAAAAAAAJBBBAAA|70031|19|27|11|PM|second|evening|dinner| +70032|AAAAAAAABJBBBAAA|70032|19|27|12|PM|second|evening|dinner| +70033|AAAAAAAACJBBBAAA|70033|19|27|13|PM|second|evening|dinner| +70034|AAAAAAAADJBBBAAA|70034|19|27|14|PM|second|evening|dinner| +70035|AAAAAAAAEJBBBAAA|70035|19|27|15|PM|second|evening|dinner| +70036|AAAAAAAAFJBBBAAA|70036|19|27|16|PM|second|evening|dinner| +70037|AAAAAAAAGJBBBAAA|70037|19|27|17|PM|second|evening|dinner| +70038|AAAAAAAAHJBBBAAA|70038|19|27|18|PM|second|evening|dinner| +70039|AAAAAAAAIJBBBAAA|70039|19|27|19|PM|second|evening|dinner| +70040|AAAAAAAAJJBBBAAA|70040|19|27|20|PM|second|evening|dinner| +70041|AAAAAAAAKJBBBAAA|70041|19|27|21|PM|second|evening|dinner| +70042|AAAAAAAALJBBBAAA|70042|19|27|22|PM|second|evening|dinner| +70043|AAAAAAAAMJBBBAAA|70043|19|27|23|PM|second|evening|dinner| +70044|AAAAAAAANJBBBAAA|70044|19|27|24|PM|second|evening|dinner| +70045|AAAAAAAAOJBBBAAA|70045|19|27|25|PM|second|evening|dinner| +70046|AAAAAAAAPJBBBAAA|70046|19|27|26|PM|second|evening|dinner| +70047|AAAAAAAAAKBBBAAA|70047|19|27|27|PM|second|evening|dinner| +70048|AAAAAAAABKBBBAAA|70048|19|27|28|PM|second|evening|dinner| +70049|AAAAAAAACKBBBAAA|70049|19|27|29|PM|second|evening|dinner| +70050|AAAAAAAADKBBBAAA|70050|19|27|30|PM|second|evening|dinner| +70051|AAAAAAAAEKBBBAAA|70051|19|27|31|PM|second|evening|dinner| +70052|AAAAAAAAFKBBBAAA|70052|19|27|32|PM|second|evening|dinner| +70053|AAAAAAAAGKBBBAAA|70053|19|27|33|PM|second|evening|dinner| +70054|AAAAAAAAHKBBBAAA|70054|19|27|34|PM|second|evening|dinner| +70055|AAAAAAAAIKBBBAAA|70055|19|27|35|PM|second|evening|dinner| +70056|AAAAAAAAJKBBBAAA|70056|19|27|36|PM|second|evening|dinner| +70057|AAAAAAAAKKBBBAAA|70057|19|27|37|PM|second|evening|dinner| +70058|AAAAAAAALKBBBAAA|70058|19|27|38|PM|second|evening|dinner| +70059|AAAAAAAAMKBBBAAA|70059|19|27|39|PM|second|evening|dinner| +70060|AAAAAAAANKBBBAAA|70060|19|27|40|PM|second|evening|dinner| +70061|AAAAAAAAOKBBBAAA|70061|19|27|41|PM|second|evening|dinner| +70062|AAAAAAAAPKBBBAAA|70062|19|27|42|PM|second|evening|dinner| +70063|AAAAAAAAALBBBAAA|70063|19|27|43|PM|second|evening|dinner| +70064|AAAAAAAABLBBBAAA|70064|19|27|44|PM|second|evening|dinner| +70065|AAAAAAAACLBBBAAA|70065|19|27|45|PM|second|evening|dinner| +70066|AAAAAAAADLBBBAAA|70066|19|27|46|PM|second|evening|dinner| +70067|AAAAAAAAELBBBAAA|70067|19|27|47|PM|second|evening|dinner| +70068|AAAAAAAAFLBBBAAA|70068|19|27|48|PM|second|evening|dinner| +70069|AAAAAAAAGLBBBAAA|70069|19|27|49|PM|second|evening|dinner| +70070|AAAAAAAAHLBBBAAA|70070|19|27|50|PM|second|evening|dinner| +70071|AAAAAAAAILBBBAAA|70071|19|27|51|PM|second|evening|dinner| +70072|AAAAAAAAJLBBBAAA|70072|19|27|52|PM|second|evening|dinner| +70073|AAAAAAAAKLBBBAAA|70073|19|27|53|PM|second|evening|dinner| +70074|AAAAAAAALLBBBAAA|70074|19|27|54|PM|second|evening|dinner| +70075|AAAAAAAAMLBBBAAA|70075|19|27|55|PM|second|evening|dinner| +70076|AAAAAAAANLBBBAAA|70076|19|27|56|PM|second|evening|dinner| +70077|AAAAAAAAOLBBBAAA|70077|19|27|57|PM|second|evening|dinner| +70078|AAAAAAAAPLBBBAAA|70078|19|27|58|PM|second|evening|dinner| +70079|AAAAAAAAAMBBBAAA|70079|19|27|59|PM|second|evening|dinner| +70080|AAAAAAAABMBBBAAA|70080|19|28|0|PM|second|evening|dinner| +70081|AAAAAAAACMBBBAAA|70081|19|28|1|PM|second|evening|dinner| +70082|AAAAAAAADMBBBAAA|70082|19|28|2|PM|second|evening|dinner| +70083|AAAAAAAAEMBBBAAA|70083|19|28|3|PM|second|evening|dinner| +70084|AAAAAAAAFMBBBAAA|70084|19|28|4|PM|second|evening|dinner| +70085|AAAAAAAAGMBBBAAA|70085|19|28|5|PM|second|evening|dinner| +70086|AAAAAAAAHMBBBAAA|70086|19|28|6|PM|second|evening|dinner| +70087|AAAAAAAAIMBBBAAA|70087|19|28|7|PM|second|evening|dinner| +70088|AAAAAAAAJMBBBAAA|70088|19|28|8|PM|second|evening|dinner| +70089|AAAAAAAAKMBBBAAA|70089|19|28|9|PM|second|evening|dinner| +70090|AAAAAAAALMBBBAAA|70090|19|28|10|PM|second|evening|dinner| +70091|AAAAAAAAMMBBBAAA|70091|19|28|11|PM|second|evening|dinner| +70092|AAAAAAAANMBBBAAA|70092|19|28|12|PM|second|evening|dinner| +70093|AAAAAAAAOMBBBAAA|70093|19|28|13|PM|second|evening|dinner| +70094|AAAAAAAAPMBBBAAA|70094|19|28|14|PM|second|evening|dinner| +70095|AAAAAAAAANBBBAAA|70095|19|28|15|PM|second|evening|dinner| +70096|AAAAAAAABNBBBAAA|70096|19|28|16|PM|second|evening|dinner| +70097|AAAAAAAACNBBBAAA|70097|19|28|17|PM|second|evening|dinner| +70098|AAAAAAAADNBBBAAA|70098|19|28|18|PM|second|evening|dinner| +70099|AAAAAAAAENBBBAAA|70099|19|28|19|PM|second|evening|dinner| +70100|AAAAAAAAFNBBBAAA|70100|19|28|20|PM|second|evening|dinner| +70101|AAAAAAAAGNBBBAAA|70101|19|28|21|PM|second|evening|dinner| +70102|AAAAAAAAHNBBBAAA|70102|19|28|22|PM|second|evening|dinner| +70103|AAAAAAAAINBBBAAA|70103|19|28|23|PM|second|evening|dinner| +70104|AAAAAAAAJNBBBAAA|70104|19|28|24|PM|second|evening|dinner| +70105|AAAAAAAAKNBBBAAA|70105|19|28|25|PM|second|evening|dinner| +70106|AAAAAAAALNBBBAAA|70106|19|28|26|PM|second|evening|dinner| +70107|AAAAAAAAMNBBBAAA|70107|19|28|27|PM|second|evening|dinner| +70108|AAAAAAAANNBBBAAA|70108|19|28|28|PM|second|evening|dinner| +70109|AAAAAAAAONBBBAAA|70109|19|28|29|PM|second|evening|dinner| +70110|AAAAAAAAPNBBBAAA|70110|19|28|30|PM|second|evening|dinner| +70111|AAAAAAAAAOBBBAAA|70111|19|28|31|PM|second|evening|dinner| +70112|AAAAAAAABOBBBAAA|70112|19|28|32|PM|second|evening|dinner| +70113|AAAAAAAACOBBBAAA|70113|19|28|33|PM|second|evening|dinner| +70114|AAAAAAAADOBBBAAA|70114|19|28|34|PM|second|evening|dinner| +70115|AAAAAAAAEOBBBAAA|70115|19|28|35|PM|second|evening|dinner| +70116|AAAAAAAAFOBBBAAA|70116|19|28|36|PM|second|evening|dinner| +70117|AAAAAAAAGOBBBAAA|70117|19|28|37|PM|second|evening|dinner| +70118|AAAAAAAAHOBBBAAA|70118|19|28|38|PM|second|evening|dinner| +70119|AAAAAAAAIOBBBAAA|70119|19|28|39|PM|second|evening|dinner| +70120|AAAAAAAAJOBBBAAA|70120|19|28|40|PM|second|evening|dinner| +70121|AAAAAAAAKOBBBAAA|70121|19|28|41|PM|second|evening|dinner| +70122|AAAAAAAALOBBBAAA|70122|19|28|42|PM|second|evening|dinner| +70123|AAAAAAAAMOBBBAAA|70123|19|28|43|PM|second|evening|dinner| +70124|AAAAAAAANOBBBAAA|70124|19|28|44|PM|second|evening|dinner| +70125|AAAAAAAAOOBBBAAA|70125|19|28|45|PM|second|evening|dinner| +70126|AAAAAAAAPOBBBAAA|70126|19|28|46|PM|second|evening|dinner| +70127|AAAAAAAAAPBBBAAA|70127|19|28|47|PM|second|evening|dinner| +70128|AAAAAAAABPBBBAAA|70128|19|28|48|PM|second|evening|dinner| +70129|AAAAAAAACPBBBAAA|70129|19|28|49|PM|second|evening|dinner| +70130|AAAAAAAADPBBBAAA|70130|19|28|50|PM|second|evening|dinner| +70131|AAAAAAAAEPBBBAAA|70131|19|28|51|PM|second|evening|dinner| +70132|AAAAAAAAFPBBBAAA|70132|19|28|52|PM|second|evening|dinner| +70133|AAAAAAAAGPBBBAAA|70133|19|28|53|PM|second|evening|dinner| +70134|AAAAAAAAHPBBBAAA|70134|19|28|54|PM|second|evening|dinner| +70135|AAAAAAAAIPBBBAAA|70135|19|28|55|PM|second|evening|dinner| +70136|AAAAAAAAJPBBBAAA|70136|19|28|56|PM|second|evening|dinner| +70137|AAAAAAAAKPBBBAAA|70137|19|28|57|PM|second|evening|dinner| +70138|AAAAAAAALPBBBAAA|70138|19|28|58|PM|second|evening|dinner| +70139|AAAAAAAAMPBBBAAA|70139|19|28|59|PM|second|evening|dinner| +70140|AAAAAAAANPBBBAAA|70140|19|29|0|PM|second|evening|dinner| +70141|AAAAAAAAOPBBBAAA|70141|19|29|1|PM|second|evening|dinner| +70142|AAAAAAAAPPBBBAAA|70142|19|29|2|PM|second|evening|dinner| +70143|AAAAAAAAAACBBAAA|70143|19|29|3|PM|second|evening|dinner| +70144|AAAAAAAABACBBAAA|70144|19|29|4|PM|second|evening|dinner| +70145|AAAAAAAACACBBAAA|70145|19|29|5|PM|second|evening|dinner| +70146|AAAAAAAADACBBAAA|70146|19|29|6|PM|second|evening|dinner| +70147|AAAAAAAAEACBBAAA|70147|19|29|7|PM|second|evening|dinner| +70148|AAAAAAAAFACBBAAA|70148|19|29|8|PM|second|evening|dinner| +70149|AAAAAAAAGACBBAAA|70149|19|29|9|PM|second|evening|dinner| +70150|AAAAAAAAHACBBAAA|70150|19|29|10|PM|second|evening|dinner| +70151|AAAAAAAAIACBBAAA|70151|19|29|11|PM|second|evening|dinner| +70152|AAAAAAAAJACBBAAA|70152|19|29|12|PM|second|evening|dinner| +70153|AAAAAAAAKACBBAAA|70153|19|29|13|PM|second|evening|dinner| +70154|AAAAAAAALACBBAAA|70154|19|29|14|PM|second|evening|dinner| +70155|AAAAAAAAMACBBAAA|70155|19|29|15|PM|second|evening|dinner| +70156|AAAAAAAANACBBAAA|70156|19|29|16|PM|second|evening|dinner| +70157|AAAAAAAAOACBBAAA|70157|19|29|17|PM|second|evening|dinner| +70158|AAAAAAAAPACBBAAA|70158|19|29|18|PM|second|evening|dinner| +70159|AAAAAAAAABCBBAAA|70159|19|29|19|PM|second|evening|dinner| +70160|AAAAAAAABBCBBAAA|70160|19|29|20|PM|second|evening|dinner| +70161|AAAAAAAACBCBBAAA|70161|19|29|21|PM|second|evening|dinner| +70162|AAAAAAAADBCBBAAA|70162|19|29|22|PM|second|evening|dinner| +70163|AAAAAAAAEBCBBAAA|70163|19|29|23|PM|second|evening|dinner| +70164|AAAAAAAAFBCBBAAA|70164|19|29|24|PM|second|evening|dinner| +70165|AAAAAAAAGBCBBAAA|70165|19|29|25|PM|second|evening|dinner| +70166|AAAAAAAAHBCBBAAA|70166|19|29|26|PM|second|evening|dinner| +70167|AAAAAAAAIBCBBAAA|70167|19|29|27|PM|second|evening|dinner| +70168|AAAAAAAAJBCBBAAA|70168|19|29|28|PM|second|evening|dinner| +70169|AAAAAAAAKBCBBAAA|70169|19|29|29|PM|second|evening|dinner| +70170|AAAAAAAALBCBBAAA|70170|19|29|30|PM|second|evening|dinner| +70171|AAAAAAAAMBCBBAAA|70171|19|29|31|PM|second|evening|dinner| +70172|AAAAAAAANBCBBAAA|70172|19|29|32|PM|second|evening|dinner| +70173|AAAAAAAAOBCBBAAA|70173|19|29|33|PM|second|evening|dinner| +70174|AAAAAAAAPBCBBAAA|70174|19|29|34|PM|second|evening|dinner| +70175|AAAAAAAAACCBBAAA|70175|19|29|35|PM|second|evening|dinner| +70176|AAAAAAAABCCBBAAA|70176|19|29|36|PM|second|evening|dinner| +70177|AAAAAAAACCCBBAAA|70177|19|29|37|PM|second|evening|dinner| +70178|AAAAAAAADCCBBAAA|70178|19|29|38|PM|second|evening|dinner| +70179|AAAAAAAAECCBBAAA|70179|19|29|39|PM|second|evening|dinner| +70180|AAAAAAAAFCCBBAAA|70180|19|29|40|PM|second|evening|dinner| +70181|AAAAAAAAGCCBBAAA|70181|19|29|41|PM|second|evening|dinner| +70182|AAAAAAAAHCCBBAAA|70182|19|29|42|PM|second|evening|dinner| +70183|AAAAAAAAICCBBAAA|70183|19|29|43|PM|second|evening|dinner| +70184|AAAAAAAAJCCBBAAA|70184|19|29|44|PM|second|evening|dinner| +70185|AAAAAAAAKCCBBAAA|70185|19|29|45|PM|second|evening|dinner| +70186|AAAAAAAALCCBBAAA|70186|19|29|46|PM|second|evening|dinner| +70187|AAAAAAAAMCCBBAAA|70187|19|29|47|PM|second|evening|dinner| +70188|AAAAAAAANCCBBAAA|70188|19|29|48|PM|second|evening|dinner| +70189|AAAAAAAAOCCBBAAA|70189|19|29|49|PM|second|evening|dinner| +70190|AAAAAAAAPCCBBAAA|70190|19|29|50|PM|second|evening|dinner| +70191|AAAAAAAAADCBBAAA|70191|19|29|51|PM|second|evening|dinner| +70192|AAAAAAAABDCBBAAA|70192|19|29|52|PM|second|evening|dinner| +70193|AAAAAAAACDCBBAAA|70193|19|29|53|PM|second|evening|dinner| +70194|AAAAAAAADDCBBAAA|70194|19|29|54|PM|second|evening|dinner| +70195|AAAAAAAAEDCBBAAA|70195|19|29|55|PM|second|evening|dinner| +70196|AAAAAAAAFDCBBAAA|70196|19|29|56|PM|second|evening|dinner| +70197|AAAAAAAAGDCBBAAA|70197|19|29|57|PM|second|evening|dinner| +70198|AAAAAAAAHDCBBAAA|70198|19|29|58|PM|second|evening|dinner| +70199|AAAAAAAAIDCBBAAA|70199|19|29|59|PM|second|evening|dinner| +70200|AAAAAAAAJDCBBAAA|70200|19|30|0|PM|second|evening|dinner| +70201|AAAAAAAAKDCBBAAA|70201|19|30|1|PM|second|evening|dinner| +70202|AAAAAAAALDCBBAAA|70202|19|30|2|PM|second|evening|dinner| +70203|AAAAAAAAMDCBBAAA|70203|19|30|3|PM|second|evening|dinner| +70204|AAAAAAAANDCBBAAA|70204|19|30|4|PM|second|evening|dinner| +70205|AAAAAAAAODCBBAAA|70205|19|30|5|PM|second|evening|dinner| +70206|AAAAAAAAPDCBBAAA|70206|19|30|6|PM|second|evening|dinner| +70207|AAAAAAAAAECBBAAA|70207|19|30|7|PM|second|evening|dinner| +70208|AAAAAAAABECBBAAA|70208|19|30|8|PM|second|evening|dinner| +70209|AAAAAAAACECBBAAA|70209|19|30|9|PM|second|evening|dinner| +70210|AAAAAAAADECBBAAA|70210|19|30|10|PM|second|evening|dinner| +70211|AAAAAAAAEECBBAAA|70211|19|30|11|PM|second|evening|dinner| +70212|AAAAAAAAFECBBAAA|70212|19|30|12|PM|second|evening|dinner| +70213|AAAAAAAAGECBBAAA|70213|19|30|13|PM|second|evening|dinner| +70214|AAAAAAAAHECBBAAA|70214|19|30|14|PM|second|evening|dinner| +70215|AAAAAAAAIECBBAAA|70215|19|30|15|PM|second|evening|dinner| +70216|AAAAAAAAJECBBAAA|70216|19|30|16|PM|second|evening|dinner| +70217|AAAAAAAAKECBBAAA|70217|19|30|17|PM|second|evening|dinner| +70218|AAAAAAAALECBBAAA|70218|19|30|18|PM|second|evening|dinner| +70219|AAAAAAAAMECBBAAA|70219|19|30|19|PM|second|evening|dinner| +70220|AAAAAAAANECBBAAA|70220|19|30|20|PM|second|evening|dinner| +70221|AAAAAAAAOECBBAAA|70221|19|30|21|PM|second|evening|dinner| +70222|AAAAAAAAPECBBAAA|70222|19|30|22|PM|second|evening|dinner| +70223|AAAAAAAAAFCBBAAA|70223|19|30|23|PM|second|evening|dinner| +70224|AAAAAAAABFCBBAAA|70224|19|30|24|PM|second|evening|dinner| +70225|AAAAAAAACFCBBAAA|70225|19|30|25|PM|second|evening|dinner| +70226|AAAAAAAADFCBBAAA|70226|19|30|26|PM|second|evening|dinner| +70227|AAAAAAAAEFCBBAAA|70227|19|30|27|PM|second|evening|dinner| +70228|AAAAAAAAFFCBBAAA|70228|19|30|28|PM|second|evening|dinner| +70229|AAAAAAAAGFCBBAAA|70229|19|30|29|PM|second|evening|dinner| +70230|AAAAAAAAHFCBBAAA|70230|19|30|30|PM|second|evening|dinner| +70231|AAAAAAAAIFCBBAAA|70231|19|30|31|PM|second|evening|dinner| +70232|AAAAAAAAJFCBBAAA|70232|19|30|32|PM|second|evening|dinner| +70233|AAAAAAAAKFCBBAAA|70233|19|30|33|PM|second|evening|dinner| +70234|AAAAAAAALFCBBAAA|70234|19|30|34|PM|second|evening|dinner| +70235|AAAAAAAAMFCBBAAA|70235|19|30|35|PM|second|evening|dinner| +70236|AAAAAAAANFCBBAAA|70236|19|30|36|PM|second|evening|dinner| +70237|AAAAAAAAOFCBBAAA|70237|19|30|37|PM|second|evening|dinner| +70238|AAAAAAAAPFCBBAAA|70238|19|30|38|PM|second|evening|dinner| +70239|AAAAAAAAAGCBBAAA|70239|19|30|39|PM|second|evening|dinner| +70240|AAAAAAAABGCBBAAA|70240|19|30|40|PM|second|evening|dinner| +70241|AAAAAAAACGCBBAAA|70241|19|30|41|PM|second|evening|dinner| +70242|AAAAAAAADGCBBAAA|70242|19|30|42|PM|second|evening|dinner| +70243|AAAAAAAAEGCBBAAA|70243|19|30|43|PM|second|evening|dinner| +70244|AAAAAAAAFGCBBAAA|70244|19|30|44|PM|second|evening|dinner| +70245|AAAAAAAAGGCBBAAA|70245|19|30|45|PM|second|evening|dinner| +70246|AAAAAAAAHGCBBAAA|70246|19|30|46|PM|second|evening|dinner| +70247|AAAAAAAAIGCBBAAA|70247|19|30|47|PM|second|evening|dinner| +70248|AAAAAAAAJGCBBAAA|70248|19|30|48|PM|second|evening|dinner| +70249|AAAAAAAAKGCBBAAA|70249|19|30|49|PM|second|evening|dinner| +70250|AAAAAAAALGCBBAAA|70250|19|30|50|PM|second|evening|dinner| +70251|AAAAAAAAMGCBBAAA|70251|19|30|51|PM|second|evening|dinner| +70252|AAAAAAAANGCBBAAA|70252|19|30|52|PM|second|evening|dinner| +70253|AAAAAAAAOGCBBAAA|70253|19|30|53|PM|second|evening|dinner| +70254|AAAAAAAAPGCBBAAA|70254|19|30|54|PM|second|evening|dinner| +70255|AAAAAAAAAHCBBAAA|70255|19|30|55|PM|second|evening|dinner| +70256|AAAAAAAABHCBBAAA|70256|19|30|56|PM|second|evening|dinner| +70257|AAAAAAAACHCBBAAA|70257|19|30|57|PM|second|evening|dinner| +70258|AAAAAAAADHCBBAAA|70258|19|30|58|PM|second|evening|dinner| +70259|AAAAAAAAEHCBBAAA|70259|19|30|59|PM|second|evening|dinner| +70260|AAAAAAAAFHCBBAAA|70260|19|31|0|PM|second|evening|dinner| +70261|AAAAAAAAGHCBBAAA|70261|19|31|1|PM|second|evening|dinner| +70262|AAAAAAAAHHCBBAAA|70262|19|31|2|PM|second|evening|dinner| +70263|AAAAAAAAIHCBBAAA|70263|19|31|3|PM|second|evening|dinner| +70264|AAAAAAAAJHCBBAAA|70264|19|31|4|PM|second|evening|dinner| +70265|AAAAAAAAKHCBBAAA|70265|19|31|5|PM|second|evening|dinner| +70266|AAAAAAAALHCBBAAA|70266|19|31|6|PM|second|evening|dinner| +70267|AAAAAAAAMHCBBAAA|70267|19|31|7|PM|second|evening|dinner| +70268|AAAAAAAANHCBBAAA|70268|19|31|8|PM|second|evening|dinner| +70269|AAAAAAAAOHCBBAAA|70269|19|31|9|PM|second|evening|dinner| +70270|AAAAAAAAPHCBBAAA|70270|19|31|10|PM|second|evening|dinner| +70271|AAAAAAAAAICBBAAA|70271|19|31|11|PM|second|evening|dinner| +70272|AAAAAAAABICBBAAA|70272|19|31|12|PM|second|evening|dinner| +70273|AAAAAAAACICBBAAA|70273|19|31|13|PM|second|evening|dinner| +70274|AAAAAAAADICBBAAA|70274|19|31|14|PM|second|evening|dinner| +70275|AAAAAAAAEICBBAAA|70275|19|31|15|PM|second|evening|dinner| +70276|AAAAAAAAFICBBAAA|70276|19|31|16|PM|second|evening|dinner| +70277|AAAAAAAAGICBBAAA|70277|19|31|17|PM|second|evening|dinner| +70278|AAAAAAAAHICBBAAA|70278|19|31|18|PM|second|evening|dinner| +70279|AAAAAAAAIICBBAAA|70279|19|31|19|PM|second|evening|dinner| +70280|AAAAAAAAJICBBAAA|70280|19|31|20|PM|second|evening|dinner| +70281|AAAAAAAAKICBBAAA|70281|19|31|21|PM|second|evening|dinner| +70282|AAAAAAAALICBBAAA|70282|19|31|22|PM|second|evening|dinner| +70283|AAAAAAAAMICBBAAA|70283|19|31|23|PM|second|evening|dinner| +70284|AAAAAAAANICBBAAA|70284|19|31|24|PM|second|evening|dinner| +70285|AAAAAAAAOICBBAAA|70285|19|31|25|PM|second|evening|dinner| +70286|AAAAAAAAPICBBAAA|70286|19|31|26|PM|second|evening|dinner| +70287|AAAAAAAAAJCBBAAA|70287|19|31|27|PM|second|evening|dinner| +70288|AAAAAAAABJCBBAAA|70288|19|31|28|PM|second|evening|dinner| +70289|AAAAAAAACJCBBAAA|70289|19|31|29|PM|second|evening|dinner| +70290|AAAAAAAADJCBBAAA|70290|19|31|30|PM|second|evening|dinner| +70291|AAAAAAAAEJCBBAAA|70291|19|31|31|PM|second|evening|dinner| +70292|AAAAAAAAFJCBBAAA|70292|19|31|32|PM|second|evening|dinner| +70293|AAAAAAAAGJCBBAAA|70293|19|31|33|PM|second|evening|dinner| +70294|AAAAAAAAHJCBBAAA|70294|19|31|34|PM|second|evening|dinner| +70295|AAAAAAAAIJCBBAAA|70295|19|31|35|PM|second|evening|dinner| +70296|AAAAAAAAJJCBBAAA|70296|19|31|36|PM|second|evening|dinner| +70297|AAAAAAAAKJCBBAAA|70297|19|31|37|PM|second|evening|dinner| +70298|AAAAAAAALJCBBAAA|70298|19|31|38|PM|second|evening|dinner| +70299|AAAAAAAAMJCBBAAA|70299|19|31|39|PM|second|evening|dinner| +70300|AAAAAAAANJCBBAAA|70300|19|31|40|PM|second|evening|dinner| +70301|AAAAAAAAOJCBBAAA|70301|19|31|41|PM|second|evening|dinner| +70302|AAAAAAAAPJCBBAAA|70302|19|31|42|PM|second|evening|dinner| +70303|AAAAAAAAAKCBBAAA|70303|19|31|43|PM|second|evening|dinner| +70304|AAAAAAAABKCBBAAA|70304|19|31|44|PM|second|evening|dinner| +70305|AAAAAAAACKCBBAAA|70305|19|31|45|PM|second|evening|dinner| +70306|AAAAAAAADKCBBAAA|70306|19|31|46|PM|second|evening|dinner| +70307|AAAAAAAAEKCBBAAA|70307|19|31|47|PM|second|evening|dinner| +70308|AAAAAAAAFKCBBAAA|70308|19|31|48|PM|second|evening|dinner| +70309|AAAAAAAAGKCBBAAA|70309|19|31|49|PM|second|evening|dinner| +70310|AAAAAAAAHKCBBAAA|70310|19|31|50|PM|second|evening|dinner| +70311|AAAAAAAAIKCBBAAA|70311|19|31|51|PM|second|evening|dinner| +70312|AAAAAAAAJKCBBAAA|70312|19|31|52|PM|second|evening|dinner| +70313|AAAAAAAAKKCBBAAA|70313|19|31|53|PM|second|evening|dinner| +70314|AAAAAAAALKCBBAAA|70314|19|31|54|PM|second|evening|dinner| +70315|AAAAAAAAMKCBBAAA|70315|19|31|55|PM|second|evening|dinner| +70316|AAAAAAAANKCBBAAA|70316|19|31|56|PM|second|evening|dinner| +70317|AAAAAAAAOKCBBAAA|70317|19|31|57|PM|second|evening|dinner| +70318|AAAAAAAAPKCBBAAA|70318|19|31|58|PM|second|evening|dinner| +70319|AAAAAAAAALCBBAAA|70319|19|31|59|PM|second|evening|dinner| +70320|AAAAAAAABLCBBAAA|70320|19|32|0|PM|second|evening|dinner| +70321|AAAAAAAACLCBBAAA|70321|19|32|1|PM|second|evening|dinner| +70322|AAAAAAAADLCBBAAA|70322|19|32|2|PM|second|evening|dinner| +70323|AAAAAAAAELCBBAAA|70323|19|32|3|PM|second|evening|dinner| +70324|AAAAAAAAFLCBBAAA|70324|19|32|4|PM|second|evening|dinner| +70325|AAAAAAAAGLCBBAAA|70325|19|32|5|PM|second|evening|dinner| +70326|AAAAAAAAHLCBBAAA|70326|19|32|6|PM|second|evening|dinner| +70327|AAAAAAAAILCBBAAA|70327|19|32|7|PM|second|evening|dinner| +70328|AAAAAAAAJLCBBAAA|70328|19|32|8|PM|second|evening|dinner| +70329|AAAAAAAAKLCBBAAA|70329|19|32|9|PM|second|evening|dinner| +70330|AAAAAAAALLCBBAAA|70330|19|32|10|PM|second|evening|dinner| +70331|AAAAAAAAMLCBBAAA|70331|19|32|11|PM|second|evening|dinner| +70332|AAAAAAAANLCBBAAA|70332|19|32|12|PM|second|evening|dinner| +70333|AAAAAAAAOLCBBAAA|70333|19|32|13|PM|second|evening|dinner| +70334|AAAAAAAAPLCBBAAA|70334|19|32|14|PM|second|evening|dinner| +70335|AAAAAAAAAMCBBAAA|70335|19|32|15|PM|second|evening|dinner| +70336|AAAAAAAABMCBBAAA|70336|19|32|16|PM|second|evening|dinner| +70337|AAAAAAAACMCBBAAA|70337|19|32|17|PM|second|evening|dinner| +70338|AAAAAAAADMCBBAAA|70338|19|32|18|PM|second|evening|dinner| +70339|AAAAAAAAEMCBBAAA|70339|19|32|19|PM|second|evening|dinner| +70340|AAAAAAAAFMCBBAAA|70340|19|32|20|PM|second|evening|dinner| +70341|AAAAAAAAGMCBBAAA|70341|19|32|21|PM|second|evening|dinner| +70342|AAAAAAAAHMCBBAAA|70342|19|32|22|PM|second|evening|dinner| +70343|AAAAAAAAIMCBBAAA|70343|19|32|23|PM|second|evening|dinner| +70344|AAAAAAAAJMCBBAAA|70344|19|32|24|PM|second|evening|dinner| +70345|AAAAAAAAKMCBBAAA|70345|19|32|25|PM|second|evening|dinner| +70346|AAAAAAAALMCBBAAA|70346|19|32|26|PM|second|evening|dinner| +70347|AAAAAAAAMMCBBAAA|70347|19|32|27|PM|second|evening|dinner| +70348|AAAAAAAANMCBBAAA|70348|19|32|28|PM|second|evening|dinner| +70349|AAAAAAAAOMCBBAAA|70349|19|32|29|PM|second|evening|dinner| +70350|AAAAAAAAPMCBBAAA|70350|19|32|30|PM|second|evening|dinner| +70351|AAAAAAAAANCBBAAA|70351|19|32|31|PM|second|evening|dinner| +70352|AAAAAAAABNCBBAAA|70352|19|32|32|PM|second|evening|dinner| +70353|AAAAAAAACNCBBAAA|70353|19|32|33|PM|second|evening|dinner| +70354|AAAAAAAADNCBBAAA|70354|19|32|34|PM|second|evening|dinner| +70355|AAAAAAAAENCBBAAA|70355|19|32|35|PM|second|evening|dinner| +70356|AAAAAAAAFNCBBAAA|70356|19|32|36|PM|second|evening|dinner| +70357|AAAAAAAAGNCBBAAA|70357|19|32|37|PM|second|evening|dinner| +70358|AAAAAAAAHNCBBAAA|70358|19|32|38|PM|second|evening|dinner| +70359|AAAAAAAAINCBBAAA|70359|19|32|39|PM|second|evening|dinner| +70360|AAAAAAAAJNCBBAAA|70360|19|32|40|PM|second|evening|dinner| +70361|AAAAAAAAKNCBBAAA|70361|19|32|41|PM|second|evening|dinner| +70362|AAAAAAAALNCBBAAA|70362|19|32|42|PM|second|evening|dinner| +70363|AAAAAAAAMNCBBAAA|70363|19|32|43|PM|second|evening|dinner| +70364|AAAAAAAANNCBBAAA|70364|19|32|44|PM|second|evening|dinner| +70365|AAAAAAAAONCBBAAA|70365|19|32|45|PM|second|evening|dinner| +70366|AAAAAAAAPNCBBAAA|70366|19|32|46|PM|second|evening|dinner| +70367|AAAAAAAAAOCBBAAA|70367|19|32|47|PM|second|evening|dinner| +70368|AAAAAAAABOCBBAAA|70368|19|32|48|PM|second|evening|dinner| +70369|AAAAAAAACOCBBAAA|70369|19|32|49|PM|second|evening|dinner| +70370|AAAAAAAADOCBBAAA|70370|19|32|50|PM|second|evening|dinner| +70371|AAAAAAAAEOCBBAAA|70371|19|32|51|PM|second|evening|dinner| +70372|AAAAAAAAFOCBBAAA|70372|19|32|52|PM|second|evening|dinner| +70373|AAAAAAAAGOCBBAAA|70373|19|32|53|PM|second|evening|dinner| +70374|AAAAAAAAHOCBBAAA|70374|19|32|54|PM|second|evening|dinner| +70375|AAAAAAAAIOCBBAAA|70375|19|32|55|PM|second|evening|dinner| +70376|AAAAAAAAJOCBBAAA|70376|19|32|56|PM|second|evening|dinner| +70377|AAAAAAAAKOCBBAAA|70377|19|32|57|PM|second|evening|dinner| +70378|AAAAAAAALOCBBAAA|70378|19|32|58|PM|second|evening|dinner| +70379|AAAAAAAAMOCBBAAA|70379|19|32|59|PM|second|evening|dinner| +70380|AAAAAAAANOCBBAAA|70380|19|33|0|PM|second|evening|dinner| +70381|AAAAAAAAOOCBBAAA|70381|19|33|1|PM|second|evening|dinner| +70382|AAAAAAAAPOCBBAAA|70382|19|33|2|PM|second|evening|dinner| +70383|AAAAAAAAAPCBBAAA|70383|19|33|3|PM|second|evening|dinner| +70384|AAAAAAAABPCBBAAA|70384|19|33|4|PM|second|evening|dinner| +70385|AAAAAAAACPCBBAAA|70385|19|33|5|PM|second|evening|dinner| +70386|AAAAAAAADPCBBAAA|70386|19|33|6|PM|second|evening|dinner| +70387|AAAAAAAAEPCBBAAA|70387|19|33|7|PM|second|evening|dinner| +70388|AAAAAAAAFPCBBAAA|70388|19|33|8|PM|second|evening|dinner| +70389|AAAAAAAAGPCBBAAA|70389|19|33|9|PM|second|evening|dinner| +70390|AAAAAAAAHPCBBAAA|70390|19|33|10|PM|second|evening|dinner| +70391|AAAAAAAAIPCBBAAA|70391|19|33|11|PM|second|evening|dinner| +70392|AAAAAAAAJPCBBAAA|70392|19|33|12|PM|second|evening|dinner| +70393|AAAAAAAAKPCBBAAA|70393|19|33|13|PM|second|evening|dinner| +70394|AAAAAAAALPCBBAAA|70394|19|33|14|PM|second|evening|dinner| +70395|AAAAAAAAMPCBBAAA|70395|19|33|15|PM|second|evening|dinner| +70396|AAAAAAAANPCBBAAA|70396|19|33|16|PM|second|evening|dinner| +70397|AAAAAAAAOPCBBAAA|70397|19|33|17|PM|second|evening|dinner| +70398|AAAAAAAAPPCBBAAA|70398|19|33|18|PM|second|evening|dinner| +70399|AAAAAAAAAADBBAAA|70399|19|33|19|PM|second|evening|dinner| +70400|AAAAAAAABADBBAAA|70400|19|33|20|PM|second|evening|dinner| +70401|AAAAAAAACADBBAAA|70401|19|33|21|PM|second|evening|dinner| +70402|AAAAAAAADADBBAAA|70402|19|33|22|PM|second|evening|dinner| +70403|AAAAAAAAEADBBAAA|70403|19|33|23|PM|second|evening|dinner| +70404|AAAAAAAAFADBBAAA|70404|19|33|24|PM|second|evening|dinner| +70405|AAAAAAAAGADBBAAA|70405|19|33|25|PM|second|evening|dinner| +70406|AAAAAAAAHADBBAAA|70406|19|33|26|PM|second|evening|dinner| +70407|AAAAAAAAIADBBAAA|70407|19|33|27|PM|second|evening|dinner| +70408|AAAAAAAAJADBBAAA|70408|19|33|28|PM|second|evening|dinner| +70409|AAAAAAAAKADBBAAA|70409|19|33|29|PM|second|evening|dinner| +70410|AAAAAAAALADBBAAA|70410|19|33|30|PM|second|evening|dinner| +70411|AAAAAAAAMADBBAAA|70411|19|33|31|PM|second|evening|dinner| +70412|AAAAAAAANADBBAAA|70412|19|33|32|PM|second|evening|dinner| +70413|AAAAAAAAOADBBAAA|70413|19|33|33|PM|second|evening|dinner| +70414|AAAAAAAAPADBBAAA|70414|19|33|34|PM|second|evening|dinner| +70415|AAAAAAAAABDBBAAA|70415|19|33|35|PM|second|evening|dinner| +70416|AAAAAAAABBDBBAAA|70416|19|33|36|PM|second|evening|dinner| +70417|AAAAAAAACBDBBAAA|70417|19|33|37|PM|second|evening|dinner| +70418|AAAAAAAADBDBBAAA|70418|19|33|38|PM|second|evening|dinner| +70419|AAAAAAAAEBDBBAAA|70419|19|33|39|PM|second|evening|dinner| +70420|AAAAAAAAFBDBBAAA|70420|19|33|40|PM|second|evening|dinner| +70421|AAAAAAAAGBDBBAAA|70421|19|33|41|PM|second|evening|dinner| +70422|AAAAAAAAHBDBBAAA|70422|19|33|42|PM|second|evening|dinner| +70423|AAAAAAAAIBDBBAAA|70423|19|33|43|PM|second|evening|dinner| +70424|AAAAAAAAJBDBBAAA|70424|19|33|44|PM|second|evening|dinner| +70425|AAAAAAAAKBDBBAAA|70425|19|33|45|PM|second|evening|dinner| +70426|AAAAAAAALBDBBAAA|70426|19|33|46|PM|second|evening|dinner| +70427|AAAAAAAAMBDBBAAA|70427|19|33|47|PM|second|evening|dinner| +70428|AAAAAAAANBDBBAAA|70428|19|33|48|PM|second|evening|dinner| +70429|AAAAAAAAOBDBBAAA|70429|19|33|49|PM|second|evening|dinner| +70430|AAAAAAAAPBDBBAAA|70430|19|33|50|PM|second|evening|dinner| +70431|AAAAAAAAACDBBAAA|70431|19|33|51|PM|second|evening|dinner| +70432|AAAAAAAABCDBBAAA|70432|19|33|52|PM|second|evening|dinner| +70433|AAAAAAAACCDBBAAA|70433|19|33|53|PM|second|evening|dinner| +70434|AAAAAAAADCDBBAAA|70434|19|33|54|PM|second|evening|dinner| +70435|AAAAAAAAECDBBAAA|70435|19|33|55|PM|second|evening|dinner| +70436|AAAAAAAAFCDBBAAA|70436|19|33|56|PM|second|evening|dinner| +70437|AAAAAAAAGCDBBAAA|70437|19|33|57|PM|second|evening|dinner| +70438|AAAAAAAAHCDBBAAA|70438|19|33|58|PM|second|evening|dinner| +70439|AAAAAAAAICDBBAAA|70439|19|33|59|PM|second|evening|dinner| +70440|AAAAAAAAJCDBBAAA|70440|19|34|0|PM|second|evening|dinner| +70441|AAAAAAAAKCDBBAAA|70441|19|34|1|PM|second|evening|dinner| +70442|AAAAAAAALCDBBAAA|70442|19|34|2|PM|second|evening|dinner| +70443|AAAAAAAAMCDBBAAA|70443|19|34|3|PM|second|evening|dinner| +70444|AAAAAAAANCDBBAAA|70444|19|34|4|PM|second|evening|dinner| +70445|AAAAAAAAOCDBBAAA|70445|19|34|5|PM|second|evening|dinner| +70446|AAAAAAAAPCDBBAAA|70446|19|34|6|PM|second|evening|dinner| +70447|AAAAAAAAADDBBAAA|70447|19|34|7|PM|second|evening|dinner| +70448|AAAAAAAABDDBBAAA|70448|19|34|8|PM|second|evening|dinner| +70449|AAAAAAAACDDBBAAA|70449|19|34|9|PM|second|evening|dinner| +70450|AAAAAAAADDDBBAAA|70450|19|34|10|PM|second|evening|dinner| +70451|AAAAAAAAEDDBBAAA|70451|19|34|11|PM|second|evening|dinner| +70452|AAAAAAAAFDDBBAAA|70452|19|34|12|PM|second|evening|dinner| +70453|AAAAAAAAGDDBBAAA|70453|19|34|13|PM|second|evening|dinner| +70454|AAAAAAAAHDDBBAAA|70454|19|34|14|PM|second|evening|dinner| +70455|AAAAAAAAIDDBBAAA|70455|19|34|15|PM|second|evening|dinner| +70456|AAAAAAAAJDDBBAAA|70456|19|34|16|PM|second|evening|dinner| +70457|AAAAAAAAKDDBBAAA|70457|19|34|17|PM|second|evening|dinner| +70458|AAAAAAAALDDBBAAA|70458|19|34|18|PM|second|evening|dinner| +70459|AAAAAAAAMDDBBAAA|70459|19|34|19|PM|second|evening|dinner| +70460|AAAAAAAANDDBBAAA|70460|19|34|20|PM|second|evening|dinner| +70461|AAAAAAAAODDBBAAA|70461|19|34|21|PM|second|evening|dinner| +70462|AAAAAAAAPDDBBAAA|70462|19|34|22|PM|second|evening|dinner| +70463|AAAAAAAAAEDBBAAA|70463|19|34|23|PM|second|evening|dinner| +70464|AAAAAAAABEDBBAAA|70464|19|34|24|PM|second|evening|dinner| +70465|AAAAAAAACEDBBAAA|70465|19|34|25|PM|second|evening|dinner| +70466|AAAAAAAADEDBBAAA|70466|19|34|26|PM|second|evening|dinner| +70467|AAAAAAAAEEDBBAAA|70467|19|34|27|PM|second|evening|dinner| +70468|AAAAAAAAFEDBBAAA|70468|19|34|28|PM|second|evening|dinner| +70469|AAAAAAAAGEDBBAAA|70469|19|34|29|PM|second|evening|dinner| +70470|AAAAAAAAHEDBBAAA|70470|19|34|30|PM|second|evening|dinner| +70471|AAAAAAAAIEDBBAAA|70471|19|34|31|PM|second|evening|dinner| +70472|AAAAAAAAJEDBBAAA|70472|19|34|32|PM|second|evening|dinner| +70473|AAAAAAAAKEDBBAAA|70473|19|34|33|PM|second|evening|dinner| +70474|AAAAAAAALEDBBAAA|70474|19|34|34|PM|second|evening|dinner| +70475|AAAAAAAAMEDBBAAA|70475|19|34|35|PM|second|evening|dinner| +70476|AAAAAAAANEDBBAAA|70476|19|34|36|PM|second|evening|dinner| +70477|AAAAAAAAOEDBBAAA|70477|19|34|37|PM|second|evening|dinner| +70478|AAAAAAAAPEDBBAAA|70478|19|34|38|PM|second|evening|dinner| +70479|AAAAAAAAAFDBBAAA|70479|19|34|39|PM|second|evening|dinner| +70480|AAAAAAAABFDBBAAA|70480|19|34|40|PM|second|evening|dinner| +70481|AAAAAAAACFDBBAAA|70481|19|34|41|PM|second|evening|dinner| +70482|AAAAAAAADFDBBAAA|70482|19|34|42|PM|second|evening|dinner| +70483|AAAAAAAAEFDBBAAA|70483|19|34|43|PM|second|evening|dinner| +70484|AAAAAAAAFFDBBAAA|70484|19|34|44|PM|second|evening|dinner| +70485|AAAAAAAAGFDBBAAA|70485|19|34|45|PM|second|evening|dinner| +70486|AAAAAAAAHFDBBAAA|70486|19|34|46|PM|second|evening|dinner| +70487|AAAAAAAAIFDBBAAA|70487|19|34|47|PM|second|evening|dinner| +70488|AAAAAAAAJFDBBAAA|70488|19|34|48|PM|second|evening|dinner| +70489|AAAAAAAAKFDBBAAA|70489|19|34|49|PM|second|evening|dinner| +70490|AAAAAAAALFDBBAAA|70490|19|34|50|PM|second|evening|dinner| +70491|AAAAAAAAMFDBBAAA|70491|19|34|51|PM|second|evening|dinner| +70492|AAAAAAAANFDBBAAA|70492|19|34|52|PM|second|evening|dinner| +70493|AAAAAAAAOFDBBAAA|70493|19|34|53|PM|second|evening|dinner| +70494|AAAAAAAAPFDBBAAA|70494|19|34|54|PM|second|evening|dinner| +70495|AAAAAAAAAGDBBAAA|70495|19|34|55|PM|second|evening|dinner| +70496|AAAAAAAABGDBBAAA|70496|19|34|56|PM|second|evening|dinner| +70497|AAAAAAAACGDBBAAA|70497|19|34|57|PM|second|evening|dinner| +70498|AAAAAAAADGDBBAAA|70498|19|34|58|PM|second|evening|dinner| +70499|AAAAAAAAEGDBBAAA|70499|19|34|59|PM|second|evening|dinner| +70500|AAAAAAAAFGDBBAAA|70500|19|35|0|PM|second|evening|dinner| +70501|AAAAAAAAGGDBBAAA|70501|19|35|1|PM|second|evening|dinner| +70502|AAAAAAAAHGDBBAAA|70502|19|35|2|PM|second|evening|dinner| +70503|AAAAAAAAIGDBBAAA|70503|19|35|3|PM|second|evening|dinner| +70504|AAAAAAAAJGDBBAAA|70504|19|35|4|PM|second|evening|dinner| +70505|AAAAAAAAKGDBBAAA|70505|19|35|5|PM|second|evening|dinner| +70506|AAAAAAAALGDBBAAA|70506|19|35|6|PM|second|evening|dinner| +70507|AAAAAAAAMGDBBAAA|70507|19|35|7|PM|second|evening|dinner| +70508|AAAAAAAANGDBBAAA|70508|19|35|8|PM|second|evening|dinner| +70509|AAAAAAAAOGDBBAAA|70509|19|35|9|PM|second|evening|dinner| +70510|AAAAAAAAPGDBBAAA|70510|19|35|10|PM|second|evening|dinner| +70511|AAAAAAAAAHDBBAAA|70511|19|35|11|PM|second|evening|dinner| +70512|AAAAAAAABHDBBAAA|70512|19|35|12|PM|second|evening|dinner| +70513|AAAAAAAACHDBBAAA|70513|19|35|13|PM|second|evening|dinner| +70514|AAAAAAAADHDBBAAA|70514|19|35|14|PM|second|evening|dinner| +70515|AAAAAAAAEHDBBAAA|70515|19|35|15|PM|second|evening|dinner| +70516|AAAAAAAAFHDBBAAA|70516|19|35|16|PM|second|evening|dinner| +70517|AAAAAAAAGHDBBAAA|70517|19|35|17|PM|second|evening|dinner| +70518|AAAAAAAAHHDBBAAA|70518|19|35|18|PM|second|evening|dinner| +70519|AAAAAAAAIHDBBAAA|70519|19|35|19|PM|second|evening|dinner| +70520|AAAAAAAAJHDBBAAA|70520|19|35|20|PM|second|evening|dinner| +70521|AAAAAAAAKHDBBAAA|70521|19|35|21|PM|second|evening|dinner| +70522|AAAAAAAALHDBBAAA|70522|19|35|22|PM|second|evening|dinner| +70523|AAAAAAAAMHDBBAAA|70523|19|35|23|PM|second|evening|dinner| +70524|AAAAAAAANHDBBAAA|70524|19|35|24|PM|second|evening|dinner| +70525|AAAAAAAAOHDBBAAA|70525|19|35|25|PM|second|evening|dinner| +70526|AAAAAAAAPHDBBAAA|70526|19|35|26|PM|second|evening|dinner| +70527|AAAAAAAAAIDBBAAA|70527|19|35|27|PM|second|evening|dinner| +70528|AAAAAAAABIDBBAAA|70528|19|35|28|PM|second|evening|dinner| +70529|AAAAAAAACIDBBAAA|70529|19|35|29|PM|second|evening|dinner| +70530|AAAAAAAADIDBBAAA|70530|19|35|30|PM|second|evening|dinner| +70531|AAAAAAAAEIDBBAAA|70531|19|35|31|PM|second|evening|dinner| +70532|AAAAAAAAFIDBBAAA|70532|19|35|32|PM|second|evening|dinner| +70533|AAAAAAAAGIDBBAAA|70533|19|35|33|PM|second|evening|dinner| +70534|AAAAAAAAHIDBBAAA|70534|19|35|34|PM|second|evening|dinner| +70535|AAAAAAAAIIDBBAAA|70535|19|35|35|PM|second|evening|dinner| +70536|AAAAAAAAJIDBBAAA|70536|19|35|36|PM|second|evening|dinner| +70537|AAAAAAAAKIDBBAAA|70537|19|35|37|PM|second|evening|dinner| +70538|AAAAAAAALIDBBAAA|70538|19|35|38|PM|second|evening|dinner| +70539|AAAAAAAAMIDBBAAA|70539|19|35|39|PM|second|evening|dinner| +70540|AAAAAAAANIDBBAAA|70540|19|35|40|PM|second|evening|dinner| +70541|AAAAAAAAOIDBBAAA|70541|19|35|41|PM|second|evening|dinner| +70542|AAAAAAAAPIDBBAAA|70542|19|35|42|PM|second|evening|dinner| +70543|AAAAAAAAAJDBBAAA|70543|19|35|43|PM|second|evening|dinner| +70544|AAAAAAAABJDBBAAA|70544|19|35|44|PM|second|evening|dinner| +70545|AAAAAAAACJDBBAAA|70545|19|35|45|PM|second|evening|dinner| +70546|AAAAAAAADJDBBAAA|70546|19|35|46|PM|second|evening|dinner| +70547|AAAAAAAAEJDBBAAA|70547|19|35|47|PM|second|evening|dinner| +70548|AAAAAAAAFJDBBAAA|70548|19|35|48|PM|second|evening|dinner| +70549|AAAAAAAAGJDBBAAA|70549|19|35|49|PM|second|evening|dinner| +70550|AAAAAAAAHJDBBAAA|70550|19|35|50|PM|second|evening|dinner| +70551|AAAAAAAAIJDBBAAA|70551|19|35|51|PM|second|evening|dinner| +70552|AAAAAAAAJJDBBAAA|70552|19|35|52|PM|second|evening|dinner| +70553|AAAAAAAAKJDBBAAA|70553|19|35|53|PM|second|evening|dinner| +70554|AAAAAAAALJDBBAAA|70554|19|35|54|PM|second|evening|dinner| +70555|AAAAAAAAMJDBBAAA|70555|19|35|55|PM|second|evening|dinner| +70556|AAAAAAAANJDBBAAA|70556|19|35|56|PM|second|evening|dinner| +70557|AAAAAAAAOJDBBAAA|70557|19|35|57|PM|second|evening|dinner| +70558|AAAAAAAAPJDBBAAA|70558|19|35|58|PM|second|evening|dinner| +70559|AAAAAAAAAKDBBAAA|70559|19|35|59|PM|second|evening|dinner| +70560|AAAAAAAABKDBBAAA|70560|19|36|0|PM|second|evening|dinner| +70561|AAAAAAAACKDBBAAA|70561|19|36|1|PM|second|evening|dinner| +70562|AAAAAAAADKDBBAAA|70562|19|36|2|PM|second|evening|dinner| +70563|AAAAAAAAEKDBBAAA|70563|19|36|3|PM|second|evening|dinner| +70564|AAAAAAAAFKDBBAAA|70564|19|36|4|PM|second|evening|dinner| +70565|AAAAAAAAGKDBBAAA|70565|19|36|5|PM|second|evening|dinner| +70566|AAAAAAAAHKDBBAAA|70566|19|36|6|PM|second|evening|dinner| +70567|AAAAAAAAIKDBBAAA|70567|19|36|7|PM|second|evening|dinner| +70568|AAAAAAAAJKDBBAAA|70568|19|36|8|PM|second|evening|dinner| +70569|AAAAAAAAKKDBBAAA|70569|19|36|9|PM|second|evening|dinner| +70570|AAAAAAAALKDBBAAA|70570|19|36|10|PM|second|evening|dinner| +70571|AAAAAAAAMKDBBAAA|70571|19|36|11|PM|second|evening|dinner| +70572|AAAAAAAANKDBBAAA|70572|19|36|12|PM|second|evening|dinner| +70573|AAAAAAAAOKDBBAAA|70573|19|36|13|PM|second|evening|dinner| +70574|AAAAAAAAPKDBBAAA|70574|19|36|14|PM|second|evening|dinner| +70575|AAAAAAAAALDBBAAA|70575|19|36|15|PM|second|evening|dinner| +70576|AAAAAAAABLDBBAAA|70576|19|36|16|PM|second|evening|dinner| +70577|AAAAAAAACLDBBAAA|70577|19|36|17|PM|second|evening|dinner| +70578|AAAAAAAADLDBBAAA|70578|19|36|18|PM|second|evening|dinner| +70579|AAAAAAAAELDBBAAA|70579|19|36|19|PM|second|evening|dinner| +70580|AAAAAAAAFLDBBAAA|70580|19|36|20|PM|second|evening|dinner| +70581|AAAAAAAAGLDBBAAA|70581|19|36|21|PM|second|evening|dinner| +70582|AAAAAAAAHLDBBAAA|70582|19|36|22|PM|second|evening|dinner| +70583|AAAAAAAAILDBBAAA|70583|19|36|23|PM|second|evening|dinner| +70584|AAAAAAAAJLDBBAAA|70584|19|36|24|PM|second|evening|dinner| +70585|AAAAAAAAKLDBBAAA|70585|19|36|25|PM|second|evening|dinner| +70586|AAAAAAAALLDBBAAA|70586|19|36|26|PM|second|evening|dinner| +70587|AAAAAAAAMLDBBAAA|70587|19|36|27|PM|second|evening|dinner| +70588|AAAAAAAANLDBBAAA|70588|19|36|28|PM|second|evening|dinner| +70589|AAAAAAAAOLDBBAAA|70589|19|36|29|PM|second|evening|dinner| +70590|AAAAAAAAPLDBBAAA|70590|19|36|30|PM|second|evening|dinner| +70591|AAAAAAAAAMDBBAAA|70591|19|36|31|PM|second|evening|dinner| +70592|AAAAAAAABMDBBAAA|70592|19|36|32|PM|second|evening|dinner| +70593|AAAAAAAACMDBBAAA|70593|19|36|33|PM|second|evening|dinner| +70594|AAAAAAAADMDBBAAA|70594|19|36|34|PM|second|evening|dinner| +70595|AAAAAAAAEMDBBAAA|70595|19|36|35|PM|second|evening|dinner| +70596|AAAAAAAAFMDBBAAA|70596|19|36|36|PM|second|evening|dinner| +70597|AAAAAAAAGMDBBAAA|70597|19|36|37|PM|second|evening|dinner| +70598|AAAAAAAAHMDBBAAA|70598|19|36|38|PM|second|evening|dinner| +70599|AAAAAAAAIMDBBAAA|70599|19|36|39|PM|second|evening|dinner| +70600|AAAAAAAAJMDBBAAA|70600|19|36|40|PM|second|evening|dinner| +70601|AAAAAAAAKMDBBAAA|70601|19|36|41|PM|second|evening|dinner| +70602|AAAAAAAALMDBBAAA|70602|19|36|42|PM|second|evening|dinner| +70603|AAAAAAAAMMDBBAAA|70603|19|36|43|PM|second|evening|dinner| +70604|AAAAAAAANMDBBAAA|70604|19|36|44|PM|second|evening|dinner| +70605|AAAAAAAAOMDBBAAA|70605|19|36|45|PM|second|evening|dinner| +70606|AAAAAAAAPMDBBAAA|70606|19|36|46|PM|second|evening|dinner| +70607|AAAAAAAAANDBBAAA|70607|19|36|47|PM|second|evening|dinner| +70608|AAAAAAAABNDBBAAA|70608|19|36|48|PM|second|evening|dinner| +70609|AAAAAAAACNDBBAAA|70609|19|36|49|PM|second|evening|dinner| +70610|AAAAAAAADNDBBAAA|70610|19|36|50|PM|second|evening|dinner| +70611|AAAAAAAAENDBBAAA|70611|19|36|51|PM|second|evening|dinner| +70612|AAAAAAAAFNDBBAAA|70612|19|36|52|PM|second|evening|dinner| +70613|AAAAAAAAGNDBBAAA|70613|19|36|53|PM|second|evening|dinner| +70614|AAAAAAAAHNDBBAAA|70614|19|36|54|PM|second|evening|dinner| +70615|AAAAAAAAINDBBAAA|70615|19|36|55|PM|second|evening|dinner| +70616|AAAAAAAAJNDBBAAA|70616|19|36|56|PM|second|evening|dinner| +70617|AAAAAAAAKNDBBAAA|70617|19|36|57|PM|second|evening|dinner| +70618|AAAAAAAALNDBBAAA|70618|19|36|58|PM|second|evening|dinner| +70619|AAAAAAAAMNDBBAAA|70619|19|36|59|PM|second|evening|dinner| +70620|AAAAAAAANNDBBAAA|70620|19|37|0|PM|second|evening|dinner| +70621|AAAAAAAAONDBBAAA|70621|19|37|1|PM|second|evening|dinner| +70622|AAAAAAAAPNDBBAAA|70622|19|37|2|PM|second|evening|dinner| +70623|AAAAAAAAAODBBAAA|70623|19|37|3|PM|second|evening|dinner| +70624|AAAAAAAABODBBAAA|70624|19|37|4|PM|second|evening|dinner| +70625|AAAAAAAACODBBAAA|70625|19|37|5|PM|second|evening|dinner| +70626|AAAAAAAADODBBAAA|70626|19|37|6|PM|second|evening|dinner| +70627|AAAAAAAAEODBBAAA|70627|19|37|7|PM|second|evening|dinner| +70628|AAAAAAAAFODBBAAA|70628|19|37|8|PM|second|evening|dinner| +70629|AAAAAAAAGODBBAAA|70629|19|37|9|PM|second|evening|dinner| +70630|AAAAAAAAHODBBAAA|70630|19|37|10|PM|second|evening|dinner| +70631|AAAAAAAAIODBBAAA|70631|19|37|11|PM|second|evening|dinner| +70632|AAAAAAAAJODBBAAA|70632|19|37|12|PM|second|evening|dinner| +70633|AAAAAAAAKODBBAAA|70633|19|37|13|PM|second|evening|dinner| +70634|AAAAAAAALODBBAAA|70634|19|37|14|PM|second|evening|dinner| +70635|AAAAAAAAMODBBAAA|70635|19|37|15|PM|second|evening|dinner| +70636|AAAAAAAANODBBAAA|70636|19|37|16|PM|second|evening|dinner| +70637|AAAAAAAAOODBBAAA|70637|19|37|17|PM|second|evening|dinner| +70638|AAAAAAAAPODBBAAA|70638|19|37|18|PM|second|evening|dinner| +70639|AAAAAAAAAPDBBAAA|70639|19|37|19|PM|second|evening|dinner| +70640|AAAAAAAABPDBBAAA|70640|19|37|20|PM|second|evening|dinner| +70641|AAAAAAAACPDBBAAA|70641|19|37|21|PM|second|evening|dinner| +70642|AAAAAAAADPDBBAAA|70642|19|37|22|PM|second|evening|dinner| +70643|AAAAAAAAEPDBBAAA|70643|19|37|23|PM|second|evening|dinner| +70644|AAAAAAAAFPDBBAAA|70644|19|37|24|PM|second|evening|dinner| +70645|AAAAAAAAGPDBBAAA|70645|19|37|25|PM|second|evening|dinner| +70646|AAAAAAAAHPDBBAAA|70646|19|37|26|PM|second|evening|dinner| +70647|AAAAAAAAIPDBBAAA|70647|19|37|27|PM|second|evening|dinner| +70648|AAAAAAAAJPDBBAAA|70648|19|37|28|PM|second|evening|dinner| +70649|AAAAAAAAKPDBBAAA|70649|19|37|29|PM|second|evening|dinner| +70650|AAAAAAAALPDBBAAA|70650|19|37|30|PM|second|evening|dinner| +70651|AAAAAAAAMPDBBAAA|70651|19|37|31|PM|second|evening|dinner| +70652|AAAAAAAANPDBBAAA|70652|19|37|32|PM|second|evening|dinner| +70653|AAAAAAAAOPDBBAAA|70653|19|37|33|PM|second|evening|dinner| +70654|AAAAAAAAPPDBBAAA|70654|19|37|34|PM|second|evening|dinner| +70655|AAAAAAAAAAEBBAAA|70655|19|37|35|PM|second|evening|dinner| +70656|AAAAAAAABAEBBAAA|70656|19|37|36|PM|second|evening|dinner| +70657|AAAAAAAACAEBBAAA|70657|19|37|37|PM|second|evening|dinner| +70658|AAAAAAAADAEBBAAA|70658|19|37|38|PM|second|evening|dinner| +70659|AAAAAAAAEAEBBAAA|70659|19|37|39|PM|second|evening|dinner| +70660|AAAAAAAAFAEBBAAA|70660|19|37|40|PM|second|evening|dinner| +70661|AAAAAAAAGAEBBAAA|70661|19|37|41|PM|second|evening|dinner| +70662|AAAAAAAAHAEBBAAA|70662|19|37|42|PM|second|evening|dinner| +70663|AAAAAAAAIAEBBAAA|70663|19|37|43|PM|second|evening|dinner| +70664|AAAAAAAAJAEBBAAA|70664|19|37|44|PM|second|evening|dinner| +70665|AAAAAAAAKAEBBAAA|70665|19|37|45|PM|second|evening|dinner| +70666|AAAAAAAALAEBBAAA|70666|19|37|46|PM|second|evening|dinner| +70667|AAAAAAAAMAEBBAAA|70667|19|37|47|PM|second|evening|dinner| +70668|AAAAAAAANAEBBAAA|70668|19|37|48|PM|second|evening|dinner| +70669|AAAAAAAAOAEBBAAA|70669|19|37|49|PM|second|evening|dinner| +70670|AAAAAAAAPAEBBAAA|70670|19|37|50|PM|second|evening|dinner| +70671|AAAAAAAAABEBBAAA|70671|19|37|51|PM|second|evening|dinner| +70672|AAAAAAAABBEBBAAA|70672|19|37|52|PM|second|evening|dinner| +70673|AAAAAAAACBEBBAAA|70673|19|37|53|PM|second|evening|dinner| +70674|AAAAAAAADBEBBAAA|70674|19|37|54|PM|second|evening|dinner| +70675|AAAAAAAAEBEBBAAA|70675|19|37|55|PM|second|evening|dinner| +70676|AAAAAAAAFBEBBAAA|70676|19|37|56|PM|second|evening|dinner| +70677|AAAAAAAAGBEBBAAA|70677|19|37|57|PM|second|evening|dinner| +70678|AAAAAAAAHBEBBAAA|70678|19|37|58|PM|second|evening|dinner| +70679|AAAAAAAAIBEBBAAA|70679|19|37|59|PM|second|evening|dinner| +70680|AAAAAAAAJBEBBAAA|70680|19|38|0|PM|second|evening|dinner| +70681|AAAAAAAAKBEBBAAA|70681|19|38|1|PM|second|evening|dinner| +70682|AAAAAAAALBEBBAAA|70682|19|38|2|PM|second|evening|dinner| +70683|AAAAAAAAMBEBBAAA|70683|19|38|3|PM|second|evening|dinner| +70684|AAAAAAAANBEBBAAA|70684|19|38|4|PM|second|evening|dinner| +70685|AAAAAAAAOBEBBAAA|70685|19|38|5|PM|second|evening|dinner| +70686|AAAAAAAAPBEBBAAA|70686|19|38|6|PM|second|evening|dinner| +70687|AAAAAAAAACEBBAAA|70687|19|38|7|PM|second|evening|dinner| +70688|AAAAAAAABCEBBAAA|70688|19|38|8|PM|second|evening|dinner| +70689|AAAAAAAACCEBBAAA|70689|19|38|9|PM|second|evening|dinner| +70690|AAAAAAAADCEBBAAA|70690|19|38|10|PM|second|evening|dinner| +70691|AAAAAAAAECEBBAAA|70691|19|38|11|PM|second|evening|dinner| +70692|AAAAAAAAFCEBBAAA|70692|19|38|12|PM|second|evening|dinner| +70693|AAAAAAAAGCEBBAAA|70693|19|38|13|PM|second|evening|dinner| +70694|AAAAAAAAHCEBBAAA|70694|19|38|14|PM|second|evening|dinner| +70695|AAAAAAAAICEBBAAA|70695|19|38|15|PM|second|evening|dinner| +70696|AAAAAAAAJCEBBAAA|70696|19|38|16|PM|second|evening|dinner| +70697|AAAAAAAAKCEBBAAA|70697|19|38|17|PM|second|evening|dinner| +70698|AAAAAAAALCEBBAAA|70698|19|38|18|PM|second|evening|dinner| +70699|AAAAAAAAMCEBBAAA|70699|19|38|19|PM|second|evening|dinner| +70700|AAAAAAAANCEBBAAA|70700|19|38|20|PM|second|evening|dinner| +70701|AAAAAAAAOCEBBAAA|70701|19|38|21|PM|second|evening|dinner| +70702|AAAAAAAAPCEBBAAA|70702|19|38|22|PM|second|evening|dinner| +70703|AAAAAAAAADEBBAAA|70703|19|38|23|PM|second|evening|dinner| +70704|AAAAAAAABDEBBAAA|70704|19|38|24|PM|second|evening|dinner| +70705|AAAAAAAACDEBBAAA|70705|19|38|25|PM|second|evening|dinner| +70706|AAAAAAAADDEBBAAA|70706|19|38|26|PM|second|evening|dinner| +70707|AAAAAAAAEDEBBAAA|70707|19|38|27|PM|second|evening|dinner| +70708|AAAAAAAAFDEBBAAA|70708|19|38|28|PM|second|evening|dinner| +70709|AAAAAAAAGDEBBAAA|70709|19|38|29|PM|second|evening|dinner| +70710|AAAAAAAAHDEBBAAA|70710|19|38|30|PM|second|evening|dinner| +70711|AAAAAAAAIDEBBAAA|70711|19|38|31|PM|second|evening|dinner| +70712|AAAAAAAAJDEBBAAA|70712|19|38|32|PM|second|evening|dinner| +70713|AAAAAAAAKDEBBAAA|70713|19|38|33|PM|second|evening|dinner| +70714|AAAAAAAALDEBBAAA|70714|19|38|34|PM|second|evening|dinner| +70715|AAAAAAAAMDEBBAAA|70715|19|38|35|PM|second|evening|dinner| +70716|AAAAAAAANDEBBAAA|70716|19|38|36|PM|second|evening|dinner| +70717|AAAAAAAAODEBBAAA|70717|19|38|37|PM|second|evening|dinner| +70718|AAAAAAAAPDEBBAAA|70718|19|38|38|PM|second|evening|dinner| +70719|AAAAAAAAAEEBBAAA|70719|19|38|39|PM|second|evening|dinner| +70720|AAAAAAAABEEBBAAA|70720|19|38|40|PM|second|evening|dinner| +70721|AAAAAAAACEEBBAAA|70721|19|38|41|PM|second|evening|dinner| +70722|AAAAAAAADEEBBAAA|70722|19|38|42|PM|second|evening|dinner| +70723|AAAAAAAAEEEBBAAA|70723|19|38|43|PM|second|evening|dinner| +70724|AAAAAAAAFEEBBAAA|70724|19|38|44|PM|second|evening|dinner| +70725|AAAAAAAAGEEBBAAA|70725|19|38|45|PM|second|evening|dinner| +70726|AAAAAAAAHEEBBAAA|70726|19|38|46|PM|second|evening|dinner| +70727|AAAAAAAAIEEBBAAA|70727|19|38|47|PM|second|evening|dinner| +70728|AAAAAAAAJEEBBAAA|70728|19|38|48|PM|second|evening|dinner| +70729|AAAAAAAAKEEBBAAA|70729|19|38|49|PM|second|evening|dinner| +70730|AAAAAAAALEEBBAAA|70730|19|38|50|PM|second|evening|dinner| +70731|AAAAAAAAMEEBBAAA|70731|19|38|51|PM|second|evening|dinner| +70732|AAAAAAAANEEBBAAA|70732|19|38|52|PM|second|evening|dinner| +70733|AAAAAAAAOEEBBAAA|70733|19|38|53|PM|second|evening|dinner| +70734|AAAAAAAAPEEBBAAA|70734|19|38|54|PM|second|evening|dinner| +70735|AAAAAAAAAFEBBAAA|70735|19|38|55|PM|second|evening|dinner| +70736|AAAAAAAABFEBBAAA|70736|19|38|56|PM|second|evening|dinner| +70737|AAAAAAAACFEBBAAA|70737|19|38|57|PM|second|evening|dinner| +70738|AAAAAAAADFEBBAAA|70738|19|38|58|PM|second|evening|dinner| +70739|AAAAAAAAEFEBBAAA|70739|19|38|59|PM|second|evening|dinner| +70740|AAAAAAAAFFEBBAAA|70740|19|39|0|PM|second|evening|dinner| +70741|AAAAAAAAGFEBBAAA|70741|19|39|1|PM|second|evening|dinner| +70742|AAAAAAAAHFEBBAAA|70742|19|39|2|PM|second|evening|dinner| +70743|AAAAAAAAIFEBBAAA|70743|19|39|3|PM|second|evening|dinner| +70744|AAAAAAAAJFEBBAAA|70744|19|39|4|PM|second|evening|dinner| +70745|AAAAAAAAKFEBBAAA|70745|19|39|5|PM|second|evening|dinner| +70746|AAAAAAAALFEBBAAA|70746|19|39|6|PM|second|evening|dinner| +70747|AAAAAAAAMFEBBAAA|70747|19|39|7|PM|second|evening|dinner| +70748|AAAAAAAANFEBBAAA|70748|19|39|8|PM|second|evening|dinner| +70749|AAAAAAAAOFEBBAAA|70749|19|39|9|PM|second|evening|dinner| +70750|AAAAAAAAPFEBBAAA|70750|19|39|10|PM|second|evening|dinner| +70751|AAAAAAAAAGEBBAAA|70751|19|39|11|PM|second|evening|dinner| +70752|AAAAAAAABGEBBAAA|70752|19|39|12|PM|second|evening|dinner| +70753|AAAAAAAACGEBBAAA|70753|19|39|13|PM|second|evening|dinner| +70754|AAAAAAAADGEBBAAA|70754|19|39|14|PM|second|evening|dinner| +70755|AAAAAAAAEGEBBAAA|70755|19|39|15|PM|second|evening|dinner| +70756|AAAAAAAAFGEBBAAA|70756|19|39|16|PM|second|evening|dinner| +70757|AAAAAAAAGGEBBAAA|70757|19|39|17|PM|second|evening|dinner| +70758|AAAAAAAAHGEBBAAA|70758|19|39|18|PM|second|evening|dinner| +70759|AAAAAAAAIGEBBAAA|70759|19|39|19|PM|second|evening|dinner| +70760|AAAAAAAAJGEBBAAA|70760|19|39|20|PM|second|evening|dinner| +70761|AAAAAAAAKGEBBAAA|70761|19|39|21|PM|second|evening|dinner| +70762|AAAAAAAALGEBBAAA|70762|19|39|22|PM|second|evening|dinner| +70763|AAAAAAAAMGEBBAAA|70763|19|39|23|PM|second|evening|dinner| +70764|AAAAAAAANGEBBAAA|70764|19|39|24|PM|second|evening|dinner| +70765|AAAAAAAAOGEBBAAA|70765|19|39|25|PM|second|evening|dinner| +70766|AAAAAAAAPGEBBAAA|70766|19|39|26|PM|second|evening|dinner| +70767|AAAAAAAAAHEBBAAA|70767|19|39|27|PM|second|evening|dinner| +70768|AAAAAAAABHEBBAAA|70768|19|39|28|PM|second|evening|dinner| +70769|AAAAAAAACHEBBAAA|70769|19|39|29|PM|second|evening|dinner| +70770|AAAAAAAADHEBBAAA|70770|19|39|30|PM|second|evening|dinner| +70771|AAAAAAAAEHEBBAAA|70771|19|39|31|PM|second|evening|dinner| +70772|AAAAAAAAFHEBBAAA|70772|19|39|32|PM|second|evening|dinner| +70773|AAAAAAAAGHEBBAAA|70773|19|39|33|PM|second|evening|dinner| +70774|AAAAAAAAHHEBBAAA|70774|19|39|34|PM|second|evening|dinner| +70775|AAAAAAAAIHEBBAAA|70775|19|39|35|PM|second|evening|dinner| +70776|AAAAAAAAJHEBBAAA|70776|19|39|36|PM|second|evening|dinner| +70777|AAAAAAAAKHEBBAAA|70777|19|39|37|PM|second|evening|dinner| +70778|AAAAAAAALHEBBAAA|70778|19|39|38|PM|second|evening|dinner| +70779|AAAAAAAAMHEBBAAA|70779|19|39|39|PM|second|evening|dinner| +70780|AAAAAAAANHEBBAAA|70780|19|39|40|PM|second|evening|dinner| +70781|AAAAAAAAOHEBBAAA|70781|19|39|41|PM|second|evening|dinner| +70782|AAAAAAAAPHEBBAAA|70782|19|39|42|PM|second|evening|dinner| +70783|AAAAAAAAAIEBBAAA|70783|19|39|43|PM|second|evening|dinner| +70784|AAAAAAAABIEBBAAA|70784|19|39|44|PM|second|evening|dinner| +70785|AAAAAAAACIEBBAAA|70785|19|39|45|PM|second|evening|dinner| +70786|AAAAAAAADIEBBAAA|70786|19|39|46|PM|second|evening|dinner| +70787|AAAAAAAAEIEBBAAA|70787|19|39|47|PM|second|evening|dinner| +70788|AAAAAAAAFIEBBAAA|70788|19|39|48|PM|second|evening|dinner| +70789|AAAAAAAAGIEBBAAA|70789|19|39|49|PM|second|evening|dinner| +70790|AAAAAAAAHIEBBAAA|70790|19|39|50|PM|second|evening|dinner| +70791|AAAAAAAAIIEBBAAA|70791|19|39|51|PM|second|evening|dinner| +70792|AAAAAAAAJIEBBAAA|70792|19|39|52|PM|second|evening|dinner| +70793|AAAAAAAAKIEBBAAA|70793|19|39|53|PM|second|evening|dinner| +70794|AAAAAAAALIEBBAAA|70794|19|39|54|PM|second|evening|dinner| +70795|AAAAAAAAMIEBBAAA|70795|19|39|55|PM|second|evening|dinner| +70796|AAAAAAAANIEBBAAA|70796|19|39|56|PM|second|evening|dinner| +70797|AAAAAAAAOIEBBAAA|70797|19|39|57|PM|second|evening|dinner| +70798|AAAAAAAAPIEBBAAA|70798|19|39|58|PM|second|evening|dinner| +70799|AAAAAAAAAJEBBAAA|70799|19|39|59|PM|second|evening|dinner| +70800|AAAAAAAABJEBBAAA|70800|19|40|0|PM|second|evening|dinner| +70801|AAAAAAAACJEBBAAA|70801|19|40|1|PM|second|evening|dinner| +70802|AAAAAAAADJEBBAAA|70802|19|40|2|PM|second|evening|dinner| +70803|AAAAAAAAEJEBBAAA|70803|19|40|3|PM|second|evening|dinner| +70804|AAAAAAAAFJEBBAAA|70804|19|40|4|PM|second|evening|dinner| +70805|AAAAAAAAGJEBBAAA|70805|19|40|5|PM|second|evening|dinner| +70806|AAAAAAAAHJEBBAAA|70806|19|40|6|PM|second|evening|dinner| +70807|AAAAAAAAIJEBBAAA|70807|19|40|7|PM|second|evening|dinner| +70808|AAAAAAAAJJEBBAAA|70808|19|40|8|PM|second|evening|dinner| +70809|AAAAAAAAKJEBBAAA|70809|19|40|9|PM|second|evening|dinner| +70810|AAAAAAAALJEBBAAA|70810|19|40|10|PM|second|evening|dinner| +70811|AAAAAAAAMJEBBAAA|70811|19|40|11|PM|second|evening|dinner| +70812|AAAAAAAANJEBBAAA|70812|19|40|12|PM|second|evening|dinner| +70813|AAAAAAAAOJEBBAAA|70813|19|40|13|PM|second|evening|dinner| +70814|AAAAAAAAPJEBBAAA|70814|19|40|14|PM|second|evening|dinner| +70815|AAAAAAAAAKEBBAAA|70815|19|40|15|PM|second|evening|dinner| +70816|AAAAAAAABKEBBAAA|70816|19|40|16|PM|second|evening|dinner| +70817|AAAAAAAACKEBBAAA|70817|19|40|17|PM|second|evening|dinner| +70818|AAAAAAAADKEBBAAA|70818|19|40|18|PM|second|evening|dinner| +70819|AAAAAAAAEKEBBAAA|70819|19|40|19|PM|second|evening|dinner| +70820|AAAAAAAAFKEBBAAA|70820|19|40|20|PM|second|evening|dinner| +70821|AAAAAAAAGKEBBAAA|70821|19|40|21|PM|second|evening|dinner| +70822|AAAAAAAAHKEBBAAA|70822|19|40|22|PM|second|evening|dinner| +70823|AAAAAAAAIKEBBAAA|70823|19|40|23|PM|second|evening|dinner| +70824|AAAAAAAAJKEBBAAA|70824|19|40|24|PM|second|evening|dinner| +70825|AAAAAAAAKKEBBAAA|70825|19|40|25|PM|second|evening|dinner| +70826|AAAAAAAALKEBBAAA|70826|19|40|26|PM|second|evening|dinner| +70827|AAAAAAAAMKEBBAAA|70827|19|40|27|PM|second|evening|dinner| +70828|AAAAAAAANKEBBAAA|70828|19|40|28|PM|second|evening|dinner| +70829|AAAAAAAAOKEBBAAA|70829|19|40|29|PM|second|evening|dinner| +70830|AAAAAAAAPKEBBAAA|70830|19|40|30|PM|second|evening|dinner| +70831|AAAAAAAAALEBBAAA|70831|19|40|31|PM|second|evening|dinner| +70832|AAAAAAAABLEBBAAA|70832|19|40|32|PM|second|evening|dinner| +70833|AAAAAAAACLEBBAAA|70833|19|40|33|PM|second|evening|dinner| +70834|AAAAAAAADLEBBAAA|70834|19|40|34|PM|second|evening|dinner| +70835|AAAAAAAAELEBBAAA|70835|19|40|35|PM|second|evening|dinner| +70836|AAAAAAAAFLEBBAAA|70836|19|40|36|PM|second|evening|dinner| +70837|AAAAAAAAGLEBBAAA|70837|19|40|37|PM|second|evening|dinner| +70838|AAAAAAAAHLEBBAAA|70838|19|40|38|PM|second|evening|dinner| +70839|AAAAAAAAILEBBAAA|70839|19|40|39|PM|second|evening|dinner| +70840|AAAAAAAAJLEBBAAA|70840|19|40|40|PM|second|evening|dinner| +70841|AAAAAAAAKLEBBAAA|70841|19|40|41|PM|second|evening|dinner| +70842|AAAAAAAALLEBBAAA|70842|19|40|42|PM|second|evening|dinner| +70843|AAAAAAAAMLEBBAAA|70843|19|40|43|PM|second|evening|dinner| +70844|AAAAAAAANLEBBAAA|70844|19|40|44|PM|second|evening|dinner| +70845|AAAAAAAAOLEBBAAA|70845|19|40|45|PM|second|evening|dinner| +70846|AAAAAAAAPLEBBAAA|70846|19|40|46|PM|second|evening|dinner| +70847|AAAAAAAAAMEBBAAA|70847|19|40|47|PM|second|evening|dinner| +70848|AAAAAAAABMEBBAAA|70848|19|40|48|PM|second|evening|dinner| +70849|AAAAAAAACMEBBAAA|70849|19|40|49|PM|second|evening|dinner| +70850|AAAAAAAADMEBBAAA|70850|19|40|50|PM|second|evening|dinner| +70851|AAAAAAAAEMEBBAAA|70851|19|40|51|PM|second|evening|dinner| +70852|AAAAAAAAFMEBBAAA|70852|19|40|52|PM|second|evening|dinner| +70853|AAAAAAAAGMEBBAAA|70853|19|40|53|PM|second|evening|dinner| +70854|AAAAAAAAHMEBBAAA|70854|19|40|54|PM|second|evening|dinner| +70855|AAAAAAAAIMEBBAAA|70855|19|40|55|PM|second|evening|dinner| +70856|AAAAAAAAJMEBBAAA|70856|19|40|56|PM|second|evening|dinner| +70857|AAAAAAAAKMEBBAAA|70857|19|40|57|PM|second|evening|dinner| +70858|AAAAAAAALMEBBAAA|70858|19|40|58|PM|second|evening|dinner| +70859|AAAAAAAAMMEBBAAA|70859|19|40|59|PM|second|evening|dinner| +70860|AAAAAAAANMEBBAAA|70860|19|41|0|PM|second|evening|dinner| +70861|AAAAAAAAOMEBBAAA|70861|19|41|1|PM|second|evening|dinner| +70862|AAAAAAAAPMEBBAAA|70862|19|41|2|PM|second|evening|dinner| +70863|AAAAAAAAANEBBAAA|70863|19|41|3|PM|second|evening|dinner| +70864|AAAAAAAABNEBBAAA|70864|19|41|4|PM|second|evening|dinner| +70865|AAAAAAAACNEBBAAA|70865|19|41|5|PM|second|evening|dinner| +70866|AAAAAAAADNEBBAAA|70866|19|41|6|PM|second|evening|dinner| +70867|AAAAAAAAENEBBAAA|70867|19|41|7|PM|second|evening|dinner| +70868|AAAAAAAAFNEBBAAA|70868|19|41|8|PM|second|evening|dinner| +70869|AAAAAAAAGNEBBAAA|70869|19|41|9|PM|second|evening|dinner| +70870|AAAAAAAAHNEBBAAA|70870|19|41|10|PM|second|evening|dinner| +70871|AAAAAAAAINEBBAAA|70871|19|41|11|PM|second|evening|dinner| +70872|AAAAAAAAJNEBBAAA|70872|19|41|12|PM|second|evening|dinner| +70873|AAAAAAAAKNEBBAAA|70873|19|41|13|PM|second|evening|dinner| +70874|AAAAAAAALNEBBAAA|70874|19|41|14|PM|second|evening|dinner| +70875|AAAAAAAAMNEBBAAA|70875|19|41|15|PM|second|evening|dinner| +70876|AAAAAAAANNEBBAAA|70876|19|41|16|PM|second|evening|dinner| +70877|AAAAAAAAONEBBAAA|70877|19|41|17|PM|second|evening|dinner| +70878|AAAAAAAAPNEBBAAA|70878|19|41|18|PM|second|evening|dinner| +70879|AAAAAAAAAOEBBAAA|70879|19|41|19|PM|second|evening|dinner| +70880|AAAAAAAABOEBBAAA|70880|19|41|20|PM|second|evening|dinner| +70881|AAAAAAAACOEBBAAA|70881|19|41|21|PM|second|evening|dinner| +70882|AAAAAAAADOEBBAAA|70882|19|41|22|PM|second|evening|dinner| +70883|AAAAAAAAEOEBBAAA|70883|19|41|23|PM|second|evening|dinner| +70884|AAAAAAAAFOEBBAAA|70884|19|41|24|PM|second|evening|dinner| +70885|AAAAAAAAGOEBBAAA|70885|19|41|25|PM|second|evening|dinner| +70886|AAAAAAAAHOEBBAAA|70886|19|41|26|PM|second|evening|dinner| +70887|AAAAAAAAIOEBBAAA|70887|19|41|27|PM|second|evening|dinner| +70888|AAAAAAAAJOEBBAAA|70888|19|41|28|PM|second|evening|dinner| +70889|AAAAAAAAKOEBBAAA|70889|19|41|29|PM|second|evening|dinner| +70890|AAAAAAAALOEBBAAA|70890|19|41|30|PM|second|evening|dinner| +70891|AAAAAAAAMOEBBAAA|70891|19|41|31|PM|second|evening|dinner| +70892|AAAAAAAANOEBBAAA|70892|19|41|32|PM|second|evening|dinner| +70893|AAAAAAAAOOEBBAAA|70893|19|41|33|PM|second|evening|dinner| +70894|AAAAAAAAPOEBBAAA|70894|19|41|34|PM|second|evening|dinner| +70895|AAAAAAAAAPEBBAAA|70895|19|41|35|PM|second|evening|dinner| +70896|AAAAAAAABPEBBAAA|70896|19|41|36|PM|second|evening|dinner| +70897|AAAAAAAACPEBBAAA|70897|19|41|37|PM|second|evening|dinner| +70898|AAAAAAAADPEBBAAA|70898|19|41|38|PM|second|evening|dinner| +70899|AAAAAAAAEPEBBAAA|70899|19|41|39|PM|second|evening|dinner| +70900|AAAAAAAAFPEBBAAA|70900|19|41|40|PM|second|evening|dinner| +70901|AAAAAAAAGPEBBAAA|70901|19|41|41|PM|second|evening|dinner| +70902|AAAAAAAAHPEBBAAA|70902|19|41|42|PM|second|evening|dinner| +70903|AAAAAAAAIPEBBAAA|70903|19|41|43|PM|second|evening|dinner| +70904|AAAAAAAAJPEBBAAA|70904|19|41|44|PM|second|evening|dinner| +70905|AAAAAAAAKPEBBAAA|70905|19|41|45|PM|second|evening|dinner| +70906|AAAAAAAALPEBBAAA|70906|19|41|46|PM|second|evening|dinner| +70907|AAAAAAAAMPEBBAAA|70907|19|41|47|PM|second|evening|dinner| +70908|AAAAAAAANPEBBAAA|70908|19|41|48|PM|second|evening|dinner| +70909|AAAAAAAAOPEBBAAA|70909|19|41|49|PM|second|evening|dinner| +70910|AAAAAAAAPPEBBAAA|70910|19|41|50|PM|second|evening|dinner| +70911|AAAAAAAAAAFBBAAA|70911|19|41|51|PM|second|evening|dinner| +70912|AAAAAAAABAFBBAAA|70912|19|41|52|PM|second|evening|dinner| +70913|AAAAAAAACAFBBAAA|70913|19|41|53|PM|second|evening|dinner| +70914|AAAAAAAADAFBBAAA|70914|19|41|54|PM|second|evening|dinner| +70915|AAAAAAAAEAFBBAAA|70915|19|41|55|PM|second|evening|dinner| +70916|AAAAAAAAFAFBBAAA|70916|19|41|56|PM|second|evening|dinner| +70917|AAAAAAAAGAFBBAAA|70917|19|41|57|PM|second|evening|dinner| +70918|AAAAAAAAHAFBBAAA|70918|19|41|58|PM|second|evening|dinner| +70919|AAAAAAAAIAFBBAAA|70919|19|41|59|PM|second|evening|dinner| +70920|AAAAAAAAJAFBBAAA|70920|19|42|0|PM|second|evening|dinner| +70921|AAAAAAAAKAFBBAAA|70921|19|42|1|PM|second|evening|dinner| +70922|AAAAAAAALAFBBAAA|70922|19|42|2|PM|second|evening|dinner| +70923|AAAAAAAAMAFBBAAA|70923|19|42|3|PM|second|evening|dinner| +70924|AAAAAAAANAFBBAAA|70924|19|42|4|PM|second|evening|dinner| +70925|AAAAAAAAOAFBBAAA|70925|19|42|5|PM|second|evening|dinner| +70926|AAAAAAAAPAFBBAAA|70926|19|42|6|PM|second|evening|dinner| +70927|AAAAAAAAABFBBAAA|70927|19|42|7|PM|second|evening|dinner| +70928|AAAAAAAABBFBBAAA|70928|19|42|8|PM|second|evening|dinner| +70929|AAAAAAAACBFBBAAA|70929|19|42|9|PM|second|evening|dinner| +70930|AAAAAAAADBFBBAAA|70930|19|42|10|PM|second|evening|dinner| +70931|AAAAAAAAEBFBBAAA|70931|19|42|11|PM|second|evening|dinner| +70932|AAAAAAAAFBFBBAAA|70932|19|42|12|PM|second|evening|dinner| +70933|AAAAAAAAGBFBBAAA|70933|19|42|13|PM|second|evening|dinner| +70934|AAAAAAAAHBFBBAAA|70934|19|42|14|PM|second|evening|dinner| +70935|AAAAAAAAIBFBBAAA|70935|19|42|15|PM|second|evening|dinner| +70936|AAAAAAAAJBFBBAAA|70936|19|42|16|PM|second|evening|dinner| +70937|AAAAAAAAKBFBBAAA|70937|19|42|17|PM|second|evening|dinner| +70938|AAAAAAAALBFBBAAA|70938|19|42|18|PM|second|evening|dinner| +70939|AAAAAAAAMBFBBAAA|70939|19|42|19|PM|second|evening|dinner| +70940|AAAAAAAANBFBBAAA|70940|19|42|20|PM|second|evening|dinner| +70941|AAAAAAAAOBFBBAAA|70941|19|42|21|PM|second|evening|dinner| +70942|AAAAAAAAPBFBBAAA|70942|19|42|22|PM|second|evening|dinner| +70943|AAAAAAAAACFBBAAA|70943|19|42|23|PM|second|evening|dinner| +70944|AAAAAAAABCFBBAAA|70944|19|42|24|PM|second|evening|dinner| +70945|AAAAAAAACCFBBAAA|70945|19|42|25|PM|second|evening|dinner| +70946|AAAAAAAADCFBBAAA|70946|19|42|26|PM|second|evening|dinner| +70947|AAAAAAAAECFBBAAA|70947|19|42|27|PM|second|evening|dinner| +70948|AAAAAAAAFCFBBAAA|70948|19|42|28|PM|second|evening|dinner| +70949|AAAAAAAAGCFBBAAA|70949|19|42|29|PM|second|evening|dinner| +70950|AAAAAAAAHCFBBAAA|70950|19|42|30|PM|second|evening|dinner| +70951|AAAAAAAAICFBBAAA|70951|19|42|31|PM|second|evening|dinner| +70952|AAAAAAAAJCFBBAAA|70952|19|42|32|PM|second|evening|dinner| +70953|AAAAAAAAKCFBBAAA|70953|19|42|33|PM|second|evening|dinner| +70954|AAAAAAAALCFBBAAA|70954|19|42|34|PM|second|evening|dinner| +70955|AAAAAAAAMCFBBAAA|70955|19|42|35|PM|second|evening|dinner| +70956|AAAAAAAANCFBBAAA|70956|19|42|36|PM|second|evening|dinner| +70957|AAAAAAAAOCFBBAAA|70957|19|42|37|PM|second|evening|dinner| +70958|AAAAAAAAPCFBBAAA|70958|19|42|38|PM|second|evening|dinner| +70959|AAAAAAAAADFBBAAA|70959|19|42|39|PM|second|evening|dinner| +70960|AAAAAAAABDFBBAAA|70960|19|42|40|PM|second|evening|dinner| +70961|AAAAAAAACDFBBAAA|70961|19|42|41|PM|second|evening|dinner| +70962|AAAAAAAADDFBBAAA|70962|19|42|42|PM|second|evening|dinner| +70963|AAAAAAAAEDFBBAAA|70963|19|42|43|PM|second|evening|dinner| +70964|AAAAAAAAFDFBBAAA|70964|19|42|44|PM|second|evening|dinner| +70965|AAAAAAAAGDFBBAAA|70965|19|42|45|PM|second|evening|dinner| +70966|AAAAAAAAHDFBBAAA|70966|19|42|46|PM|second|evening|dinner| +70967|AAAAAAAAIDFBBAAA|70967|19|42|47|PM|second|evening|dinner| +70968|AAAAAAAAJDFBBAAA|70968|19|42|48|PM|second|evening|dinner| +70969|AAAAAAAAKDFBBAAA|70969|19|42|49|PM|second|evening|dinner| +70970|AAAAAAAALDFBBAAA|70970|19|42|50|PM|second|evening|dinner| +70971|AAAAAAAAMDFBBAAA|70971|19|42|51|PM|second|evening|dinner| +70972|AAAAAAAANDFBBAAA|70972|19|42|52|PM|second|evening|dinner| +70973|AAAAAAAAODFBBAAA|70973|19|42|53|PM|second|evening|dinner| +70974|AAAAAAAAPDFBBAAA|70974|19|42|54|PM|second|evening|dinner| +70975|AAAAAAAAAEFBBAAA|70975|19|42|55|PM|second|evening|dinner| +70976|AAAAAAAABEFBBAAA|70976|19|42|56|PM|second|evening|dinner| +70977|AAAAAAAACEFBBAAA|70977|19|42|57|PM|second|evening|dinner| +70978|AAAAAAAADEFBBAAA|70978|19|42|58|PM|second|evening|dinner| +70979|AAAAAAAAEEFBBAAA|70979|19|42|59|PM|second|evening|dinner| +70980|AAAAAAAAFEFBBAAA|70980|19|43|0|PM|second|evening|dinner| +70981|AAAAAAAAGEFBBAAA|70981|19|43|1|PM|second|evening|dinner| +70982|AAAAAAAAHEFBBAAA|70982|19|43|2|PM|second|evening|dinner| +70983|AAAAAAAAIEFBBAAA|70983|19|43|3|PM|second|evening|dinner| +70984|AAAAAAAAJEFBBAAA|70984|19|43|4|PM|second|evening|dinner| +70985|AAAAAAAAKEFBBAAA|70985|19|43|5|PM|second|evening|dinner| +70986|AAAAAAAALEFBBAAA|70986|19|43|6|PM|second|evening|dinner| +70987|AAAAAAAAMEFBBAAA|70987|19|43|7|PM|second|evening|dinner| +70988|AAAAAAAANEFBBAAA|70988|19|43|8|PM|second|evening|dinner| +70989|AAAAAAAAOEFBBAAA|70989|19|43|9|PM|second|evening|dinner| +70990|AAAAAAAAPEFBBAAA|70990|19|43|10|PM|second|evening|dinner| +70991|AAAAAAAAAFFBBAAA|70991|19|43|11|PM|second|evening|dinner| +70992|AAAAAAAABFFBBAAA|70992|19|43|12|PM|second|evening|dinner| +70993|AAAAAAAACFFBBAAA|70993|19|43|13|PM|second|evening|dinner| +70994|AAAAAAAADFFBBAAA|70994|19|43|14|PM|second|evening|dinner| +70995|AAAAAAAAEFFBBAAA|70995|19|43|15|PM|second|evening|dinner| +70996|AAAAAAAAFFFBBAAA|70996|19|43|16|PM|second|evening|dinner| +70997|AAAAAAAAGFFBBAAA|70997|19|43|17|PM|second|evening|dinner| +70998|AAAAAAAAHFFBBAAA|70998|19|43|18|PM|second|evening|dinner| +70999|AAAAAAAAIFFBBAAA|70999|19|43|19|PM|second|evening|dinner| +71000|AAAAAAAAJFFBBAAA|71000|19|43|20|PM|second|evening|dinner| +71001|AAAAAAAAKFFBBAAA|71001|19|43|21|PM|second|evening|dinner| +71002|AAAAAAAALFFBBAAA|71002|19|43|22|PM|second|evening|dinner| +71003|AAAAAAAAMFFBBAAA|71003|19|43|23|PM|second|evening|dinner| +71004|AAAAAAAANFFBBAAA|71004|19|43|24|PM|second|evening|dinner| +71005|AAAAAAAAOFFBBAAA|71005|19|43|25|PM|second|evening|dinner| +71006|AAAAAAAAPFFBBAAA|71006|19|43|26|PM|second|evening|dinner| +71007|AAAAAAAAAGFBBAAA|71007|19|43|27|PM|second|evening|dinner| +71008|AAAAAAAABGFBBAAA|71008|19|43|28|PM|second|evening|dinner| +71009|AAAAAAAACGFBBAAA|71009|19|43|29|PM|second|evening|dinner| +71010|AAAAAAAADGFBBAAA|71010|19|43|30|PM|second|evening|dinner| +71011|AAAAAAAAEGFBBAAA|71011|19|43|31|PM|second|evening|dinner| +71012|AAAAAAAAFGFBBAAA|71012|19|43|32|PM|second|evening|dinner| +71013|AAAAAAAAGGFBBAAA|71013|19|43|33|PM|second|evening|dinner| +71014|AAAAAAAAHGFBBAAA|71014|19|43|34|PM|second|evening|dinner| +71015|AAAAAAAAIGFBBAAA|71015|19|43|35|PM|second|evening|dinner| +71016|AAAAAAAAJGFBBAAA|71016|19|43|36|PM|second|evening|dinner| +71017|AAAAAAAAKGFBBAAA|71017|19|43|37|PM|second|evening|dinner| +71018|AAAAAAAALGFBBAAA|71018|19|43|38|PM|second|evening|dinner| +71019|AAAAAAAAMGFBBAAA|71019|19|43|39|PM|second|evening|dinner| +71020|AAAAAAAANGFBBAAA|71020|19|43|40|PM|second|evening|dinner| +71021|AAAAAAAAOGFBBAAA|71021|19|43|41|PM|second|evening|dinner| +71022|AAAAAAAAPGFBBAAA|71022|19|43|42|PM|second|evening|dinner| +71023|AAAAAAAAAHFBBAAA|71023|19|43|43|PM|second|evening|dinner| +71024|AAAAAAAABHFBBAAA|71024|19|43|44|PM|second|evening|dinner| +71025|AAAAAAAACHFBBAAA|71025|19|43|45|PM|second|evening|dinner| +71026|AAAAAAAADHFBBAAA|71026|19|43|46|PM|second|evening|dinner| +71027|AAAAAAAAEHFBBAAA|71027|19|43|47|PM|second|evening|dinner| +71028|AAAAAAAAFHFBBAAA|71028|19|43|48|PM|second|evening|dinner| +71029|AAAAAAAAGHFBBAAA|71029|19|43|49|PM|second|evening|dinner| +71030|AAAAAAAAHHFBBAAA|71030|19|43|50|PM|second|evening|dinner| +71031|AAAAAAAAIHFBBAAA|71031|19|43|51|PM|second|evening|dinner| +71032|AAAAAAAAJHFBBAAA|71032|19|43|52|PM|second|evening|dinner| +71033|AAAAAAAAKHFBBAAA|71033|19|43|53|PM|second|evening|dinner| +71034|AAAAAAAALHFBBAAA|71034|19|43|54|PM|second|evening|dinner| +71035|AAAAAAAAMHFBBAAA|71035|19|43|55|PM|second|evening|dinner| +71036|AAAAAAAANHFBBAAA|71036|19|43|56|PM|second|evening|dinner| +71037|AAAAAAAAOHFBBAAA|71037|19|43|57|PM|second|evening|dinner| +71038|AAAAAAAAPHFBBAAA|71038|19|43|58|PM|second|evening|dinner| +71039|AAAAAAAAAIFBBAAA|71039|19|43|59|PM|second|evening|dinner| +71040|AAAAAAAABIFBBAAA|71040|19|44|0|PM|second|evening|dinner| +71041|AAAAAAAACIFBBAAA|71041|19|44|1|PM|second|evening|dinner| +71042|AAAAAAAADIFBBAAA|71042|19|44|2|PM|second|evening|dinner| +71043|AAAAAAAAEIFBBAAA|71043|19|44|3|PM|second|evening|dinner| +71044|AAAAAAAAFIFBBAAA|71044|19|44|4|PM|second|evening|dinner| +71045|AAAAAAAAGIFBBAAA|71045|19|44|5|PM|second|evening|dinner| +71046|AAAAAAAAHIFBBAAA|71046|19|44|6|PM|second|evening|dinner| +71047|AAAAAAAAIIFBBAAA|71047|19|44|7|PM|second|evening|dinner| +71048|AAAAAAAAJIFBBAAA|71048|19|44|8|PM|second|evening|dinner| +71049|AAAAAAAAKIFBBAAA|71049|19|44|9|PM|second|evening|dinner| +71050|AAAAAAAALIFBBAAA|71050|19|44|10|PM|second|evening|dinner| +71051|AAAAAAAAMIFBBAAA|71051|19|44|11|PM|second|evening|dinner| +71052|AAAAAAAANIFBBAAA|71052|19|44|12|PM|second|evening|dinner| +71053|AAAAAAAAOIFBBAAA|71053|19|44|13|PM|second|evening|dinner| +71054|AAAAAAAAPIFBBAAA|71054|19|44|14|PM|second|evening|dinner| +71055|AAAAAAAAAJFBBAAA|71055|19|44|15|PM|second|evening|dinner| +71056|AAAAAAAABJFBBAAA|71056|19|44|16|PM|second|evening|dinner| +71057|AAAAAAAACJFBBAAA|71057|19|44|17|PM|second|evening|dinner| +71058|AAAAAAAADJFBBAAA|71058|19|44|18|PM|second|evening|dinner| +71059|AAAAAAAAEJFBBAAA|71059|19|44|19|PM|second|evening|dinner| +71060|AAAAAAAAFJFBBAAA|71060|19|44|20|PM|second|evening|dinner| +71061|AAAAAAAAGJFBBAAA|71061|19|44|21|PM|second|evening|dinner| +71062|AAAAAAAAHJFBBAAA|71062|19|44|22|PM|second|evening|dinner| +71063|AAAAAAAAIJFBBAAA|71063|19|44|23|PM|second|evening|dinner| +71064|AAAAAAAAJJFBBAAA|71064|19|44|24|PM|second|evening|dinner| +71065|AAAAAAAAKJFBBAAA|71065|19|44|25|PM|second|evening|dinner| +71066|AAAAAAAALJFBBAAA|71066|19|44|26|PM|second|evening|dinner| +71067|AAAAAAAAMJFBBAAA|71067|19|44|27|PM|second|evening|dinner| +71068|AAAAAAAANJFBBAAA|71068|19|44|28|PM|second|evening|dinner| +71069|AAAAAAAAOJFBBAAA|71069|19|44|29|PM|second|evening|dinner| +71070|AAAAAAAAPJFBBAAA|71070|19|44|30|PM|second|evening|dinner| +71071|AAAAAAAAAKFBBAAA|71071|19|44|31|PM|second|evening|dinner| +71072|AAAAAAAABKFBBAAA|71072|19|44|32|PM|second|evening|dinner| +71073|AAAAAAAACKFBBAAA|71073|19|44|33|PM|second|evening|dinner| +71074|AAAAAAAADKFBBAAA|71074|19|44|34|PM|second|evening|dinner| +71075|AAAAAAAAEKFBBAAA|71075|19|44|35|PM|second|evening|dinner| +71076|AAAAAAAAFKFBBAAA|71076|19|44|36|PM|second|evening|dinner| +71077|AAAAAAAAGKFBBAAA|71077|19|44|37|PM|second|evening|dinner| +71078|AAAAAAAAHKFBBAAA|71078|19|44|38|PM|second|evening|dinner| +71079|AAAAAAAAIKFBBAAA|71079|19|44|39|PM|second|evening|dinner| +71080|AAAAAAAAJKFBBAAA|71080|19|44|40|PM|second|evening|dinner| +71081|AAAAAAAAKKFBBAAA|71081|19|44|41|PM|second|evening|dinner| +71082|AAAAAAAALKFBBAAA|71082|19|44|42|PM|second|evening|dinner| +71083|AAAAAAAAMKFBBAAA|71083|19|44|43|PM|second|evening|dinner| +71084|AAAAAAAANKFBBAAA|71084|19|44|44|PM|second|evening|dinner| +71085|AAAAAAAAOKFBBAAA|71085|19|44|45|PM|second|evening|dinner| +71086|AAAAAAAAPKFBBAAA|71086|19|44|46|PM|second|evening|dinner| +71087|AAAAAAAAALFBBAAA|71087|19|44|47|PM|second|evening|dinner| +71088|AAAAAAAABLFBBAAA|71088|19|44|48|PM|second|evening|dinner| +71089|AAAAAAAACLFBBAAA|71089|19|44|49|PM|second|evening|dinner| +71090|AAAAAAAADLFBBAAA|71090|19|44|50|PM|second|evening|dinner| +71091|AAAAAAAAELFBBAAA|71091|19|44|51|PM|second|evening|dinner| +71092|AAAAAAAAFLFBBAAA|71092|19|44|52|PM|second|evening|dinner| +71093|AAAAAAAAGLFBBAAA|71093|19|44|53|PM|second|evening|dinner| +71094|AAAAAAAAHLFBBAAA|71094|19|44|54|PM|second|evening|dinner| +71095|AAAAAAAAILFBBAAA|71095|19|44|55|PM|second|evening|dinner| +71096|AAAAAAAAJLFBBAAA|71096|19|44|56|PM|second|evening|dinner| +71097|AAAAAAAAKLFBBAAA|71097|19|44|57|PM|second|evening|dinner| +71098|AAAAAAAALLFBBAAA|71098|19|44|58|PM|second|evening|dinner| +71099|AAAAAAAAMLFBBAAA|71099|19|44|59|PM|second|evening|dinner| +71100|AAAAAAAANLFBBAAA|71100|19|45|0|PM|second|evening|dinner| +71101|AAAAAAAAOLFBBAAA|71101|19|45|1|PM|second|evening|dinner| +71102|AAAAAAAAPLFBBAAA|71102|19|45|2|PM|second|evening|dinner| +71103|AAAAAAAAAMFBBAAA|71103|19|45|3|PM|second|evening|dinner| +71104|AAAAAAAABMFBBAAA|71104|19|45|4|PM|second|evening|dinner| +71105|AAAAAAAACMFBBAAA|71105|19|45|5|PM|second|evening|dinner| +71106|AAAAAAAADMFBBAAA|71106|19|45|6|PM|second|evening|dinner| +71107|AAAAAAAAEMFBBAAA|71107|19|45|7|PM|second|evening|dinner| +71108|AAAAAAAAFMFBBAAA|71108|19|45|8|PM|second|evening|dinner| +71109|AAAAAAAAGMFBBAAA|71109|19|45|9|PM|second|evening|dinner| +71110|AAAAAAAAHMFBBAAA|71110|19|45|10|PM|second|evening|dinner| +71111|AAAAAAAAIMFBBAAA|71111|19|45|11|PM|second|evening|dinner| +71112|AAAAAAAAJMFBBAAA|71112|19|45|12|PM|second|evening|dinner| +71113|AAAAAAAAKMFBBAAA|71113|19|45|13|PM|second|evening|dinner| +71114|AAAAAAAALMFBBAAA|71114|19|45|14|PM|second|evening|dinner| +71115|AAAAAAAAMMFBBAAA|71115|19|45|15|PM|second|evening|dinner| +71116|AAAAAAAANMFBBAAA|71116|19|45|16|PM|second|evening|dinner| +71117|AAAAAAAAOMFBBAAA|71117|19|45|17|PM|second|evening|dinner| +71118|AAAAAAAAPMFBBAAA|71118|19|45|18|PM|second|evening|dinner| +71119|AAAAAAAAANFBBAAA|71119|19|45|19|PM|second|evening|dinner| +71120|AAAAAAAABNFBBAAA|71120|19|45|20|PM|second|evening|dinner| +71121|AAAAAAAACNFBBAAA|71121|19|45|21|PM|second|evening|dinner| +71122|AAAAAAAADNFBBAAA|71122|19|45|22|PM|second|evening|dinner| +71123|AAAAAAAAENFBBAAA|71123|19|45|23|PM|second|evening|dinner| +71124|AAAAAAAAFNFBBAAA|71124|19|45|24|PM|second|evening|dinner| +71125|AAAAAAAAGNFBBAAA|71125|19|45|25|PM|second|evening|dinner| +71126|AAAAAAAAHNFBBAAA|71126|19|45|26|PM|second|evening|dinner| +71127|AAAAAAAAINFBBAAA|71127|19|45|27|PM|second|evening|dinner| +71128|AAAAAAAAJNFBBAAA|71128|19|45|28|PM|second|evening|dinner| +71129|AAAAAAAAKNFBBAAA|71129|19|45|29|PM|second|evening|dinner| +71130|AAAAAAAALNFBBAAA|71130|19|45|30|PM|second|evening|dinner| +71131|AAAAAAAAMNFBBAAA|71131|19|45|31|PM|second|evening|dinner| +71132|AAAAAAAANNFBBAAA|71132|19|45|32|PM|second|evening|dinner| +71133|AAAAAAAAONFBBAAA|71133|19|45|33|PM|second|evening|dinner| +71134|AAAAAAAAPNFBBAAA|71134|19|45|34|PM|second|evening|dinner| +71135|AAAAAAAAAOFBBAAA|71135|19|45|35|PM|second|evening|dinner| +71136|AAAAAAAABOFBBAAA|71136|19|45|36|PM|second|evening|dinner| +71137|AAAAAAAACOFBBAAA|71137|19|45|37|PM|second|evening|dinner| +71138|AAAAAAAADOFBBAAA|71138|19|45|38|PM|second|evening|dinner| +71139|AAAAAAAAEOFBBAAA|71139|19|45|39|PM|second|evening|dinner| +71140|AAAAAAAAFOFBBAAA|71140|19|45|40|PM|second|evening|dinner| +71141|AAAAAAAAGOFBBAAA|71141|19|45|41|PM|second|evening|dinner| +71142|AAAAAAAAHOFBBAAA|71142|19|45|42|PM|second|evening|dinner| +71143|AAAAAAAAIOFBBAAA|71143|19|45|43|PM|second|evening|dinner| +71144|AAAAAAAAJOFBBAAA|71144|19|45|44|PM|second|evening|dinner| +71145|AAAAAAAAKOFBBAAA|71145|19|45|45|PM|second|evening|dinner| +71146|AAAAAAAALOFBBAAA|71146|19|45|46|PM|second|evening|dinner| +71147|AAAAAAAAMOFBBAAA|71147|19|45|47|PM|second|evening|dinner| +71148|AAAAAAAANOFBBAAA|71148|19|45|48|PM|second|evening|dinner| +71149|AAAAAAAAOOFBBAAA|71149|19|45|49|PM|second|evening|dinner| +71150|AAAAAAAAPOFBBAAA|71150|19|45|50|PM|second|evening|dinner| +71151|AAAAAAAAAPFBBAAA|71151|19|45|51|PM|second|evening|dinner| +71152|AAAAAAAABPFBBAAA|71152|19|45|52|PM|second|evening|dinner| +71153|AAAAAAAACPFBBAAA|71153|19|45|53|PM|second|evening|dinner| +71154|AAAAAAAADPFBBAAA|71154|19|45|54|PM|second|evening|dinner| +71155|AAAAAAAAEPFBBAAA|71155|19|45|55|PM|second|evening|dinner| +71156|AAAAAAAAFPFBBAAA|71156|19|45|56|PM|second|evening|dinner| +71157|AAAAAAAAGPFBBAAA|71157|19|45|57|PM|second|evening|dinner| +71158|AAAAAAAAHPFBBAAA|71158|19|45|58|PM|second|evening|dinner| +71159|AAAAAAAAIPFBBAAA|71159|19|45|59|PM|second|evening|dinner| +71160|AAAAAAAAJPFBBAAA|71160|19|46|0|PM|second|evening|dinner| +71161|AAAAAAAAKPFBBAAA|71161|19|46|1|PM|second|evening|dinner| +71162|AAAAAAAALPFBBAAA|71162|19|46|2|PM|second|evening|dinner| +71163|AAAAAAAAMPFBBAAA|71163|19|46|3|PM|second|evening|dinner| +71164|AAAAAAAANPFBBAAA|71164|19|46|4|PM|second|evening|dinner| +71165|AAAAAAAAOPFBBAAA|71165|19|46|5|PM|second|evening|dinner| +71166|AAAAAAAAPPFBBAAA|71166|19|46|6|PM|second|evening|dinner| +71167|AAAAAAAAAAGBBAAA|71167|19|46|7|PM|second|evening|dinner| +71168|AAAAAAAABAGBBAAA|71168|19|46|8|PM|second|evening|dinner| +71169|AAAAAAAACAGBBAAA|71169|19|46|9|PM|second|evening|dinner| +71170|AAAAAAAADAGBBAAA|71170|19|46|10|PM|second|evening|dinner| +71171|AAAAAAAAEAGBBAAA|71171|19|46|11|PM|second|evening|dinner| +71172|AAAAAAAAFAGBBAAA|71172|19|46|12|PM|second|evening|dinner| +71173|AAAAAAAAGAGBBAAA|71173|19|46|13|PM|second|evening|dinner| +71174|AAAAAAAAHAGBBAAA|71174|19|46|14|PM|second|evening|dinner| +71175|AAAAAAAAIAGBBAAA|71175|19|46|15|PM|second|evening|dinner| +71176|AAAAAAAAJAGBBAAA|71176|19|46|16|PM|second|evening|dinner| +71177|AAAAAAAAKAGBBAAA|71177|19|46|17|PM|second|evening|dinner| +71178|AAAAAAAALAGBBAAA|71178|19|46|18|PM|second|evening|dinner| +71179|AAAAAAAAMAGBBAAA|71179|19|46|19|PM|second|evening|dinner| +71180|AAAAAAAANAGBBAAA|71180|19|46|20|PM|second|evening|dinner| +71181|AAAAAAAAOAGBBAAA|71181|19|46|21|PM|second|evening|dinner| +71182|AAAAAAAAPAGBBAAA|71182|19|46|22|PM|second|evening|dinner| +71183|AAAAAAAAABGBBAAA|71183|19|46|23|PM|second|evening|dinner| +71184|AAAAAAAABBGBBAAA|71184|19|46|24|PM|second|evening|dinner| +71185|AAAAAAAACBGBBAAA|71185|19|46|25|PM|second|evening|dinner| +71186|AAAAAAAADBGBBAAA|71186|19|46|26|PM|second|evening|dinner| +71187|AAAAAAAAEBGBBAAA|71187|19|46|27|PM|second|evening|dinner| +71188|AAAAAAAAFBGBBAAA|71188|19|46|28|PM|second|evening|dinner| +71189|AAAAAAAAGBGBBAAA|71189|19|46|29|PM|second|evening|dinner| +71190|AAAAAAAAHBGBBAAA|71190|19|46|30|PM|second|evening|dinner| +71191|AAAAAAAAIBGBBAAA|71191|19|46|31|PM|second|evening|dinner| +71192|AAAAAAAAJBGBBAAA|71192|19|46|32|PM|second|evening|dinner| +71193|AAAAAAAAKBGBBAAA|71193|19|46|33|PM|second|evening|dinner| +71194|AAAAAAAALBGBBAAA|71194|19|46|34|PM|second|evening|dinner| +71195|AAAAAAAAMBGBBAAA|71195|19|46|35|PM|second|evening|dinner| +71196|AAAAAAAANBGBBAAA|71196|19|46|36|PM|second|evening|dinner| +71197|AAAAAAAAOBGBBAAA|71197|19|46|37|PM|second|evening|dinner| +71198|AAAAAAAAPBGBBAAA|71198|19|46|38|PM|second|evening|dinner| +71199|AAAAAAAAACGBBAAA|71199|19|46|39|PM|second|evening|dinner| +71200|AAAAAAAABCGBBAAA|71200|19|46|40|PM|second|evening|dinner| +71201|AAAAAAAACCGBBAAA|71201|19|46|41|PM|second|evening|dinner| +71202|AAAAAAAADCGBBAAA|71202|19|46|42|PM|second|evening|dinner| +71203|AAAAAAAAECGBBAAA|71203|19|46|43|PM|second|evening|dinner| +71204|AAAAAAAAFCGBBAAA|71204|19|46|44|PM|second|evening|dinner| +71205|AAAAAAAAGCGBBAAA|71205|19|46|45|PM|second|evening|dinner| +71206|AAAAAAAAHCGBBAAA|71206|19|46|46|PM|second|evening|dinner| +71207|AAAAAAAAICGBBAAA|71207|19|46|47|PM|second|evening|dinner| +71208|AAAAAAAAJCGBBAAA|71208|19|46|48|PM|second|evening|dinner| +71209|AAAAAAAAKCGBBAAA|71209|19|46|49|PM|second|evening|dinner| +71210|AAAAAAAALCGBBAAA|71210|19|46|50|PM|second|evening|dinner| +71211|AAAAAAAAMCGBBAAA|71211|19|46|51|PM|second|evening|dinner| +71212|AAAAAAAANCGBBAAA|71212|19|46|52|PM|second|evening|dinner| +71213|AAAAAAAAOCGBBAAA|71213|19|46|53|PM|second|evening|dinner| +71214|AAAAAAAAPCGBBAAA|71214|19|46|54|PM|second|evening|dinner| +71215|AAAAAAAAADGBBAAA|71215|19|46|55|PM|second|evening|dinner| +71216|AAAAAAAABDGBBAAA|71216|19|46|56|PM|second|evening|dinner| +71217|AAAAAAAACDGBBAAA|71217|19|46|57|PM|second|evening|dinner| +71218|AAAAAAAADDGBBAAA|71218|19|46|58|PM|second|evening|dinner| +71219|AAAAAAAAEDGBBAAA|71219|19|46|59|PM|second|evening|dinner| +71220|AAAAAAAAFDGBBAAA|71220|19|47|0|PM|second|evening|dinner| +71221|AAAAAAAAGDGBBAAA|71221|19|47|1|PM|second|evening|dinner| +71222|AAAAAAAAHDGBBAAA|71222|19|47|2|PM|second|evening|dinner| +71223|AAAAAAAAIDGBBAAA|71223|19|47|3|PM|second|evening|dinner| +71224|AAAAAAAAJDGBBAAA|71224|19|47|4|PM|second|evening|dinner| +71225|AAAAAAAAKDGBBAAA|71225|19|47|5|PM|second|evening|dinner| +71226|AAAAAAAALDGBBAAA|71226|19|47|6|PM|second|evening|dinner| +71227|AAAAAAAAMDGBBAAA|71227|19|47|7|PM|second|evening|dinner| +71228|AAAAAAAANDGBBAAA|71228|19|47|8|PM|second|evening|dinner| +71229|AAAAAAAAODGBBAAA|71229|19|47|9|PM|second|evening|dinner| +71230|AAAAAAAAPDGBBAAA|71230|19|47|10|PM|second|evening|dinner| +71231|AAAAAAAAAEGBBAAA|71231|19|47|11|PM|second|evening|dinner| +71232|AAAAAAAABEGBBAAA|71232|19|47|12|PM|second|evening|dinner| +71233|AAAAAAAACEGBBAAA|71233|19|47|13|PM|second|evening|dinner| +71234|AAAAAAAADEGBBAAA|71234|19|47|14|PM|second|evening|dinner| +71235|AAAAAAAAEEGBBAAA|71235|19|47|15|PM|second|evening|dinner| +71236|AAAAAAAAFEGBBAAA|71236|19|47|16|PM|second|evening|dinner| +71237|AAAAAAAAGEGBBAAA|71237|19|47|17|PM|second|evening|dinner| +71238|AAAAAAAAHEGBBAAA|71238|19|47|18|PM|second|evening|dinner| +71239|AAAAAAAAIEGBBAAA|71239|19|47|19|PM|second|evening|dinner| +71240|AAAAAAAAJEGBBAAA|71240|19|47|20|PM|second|evening|dinner| +71241|AAAAAAAAKEGBBAAA|71241|19|47|21|PM|second|evening|dinner| +71242|AAAAAAAALEGBBAAA|71242|19|47|22|PM|second|evening|dinner| +71243|AAAAAAAAMEGBBAAA|71243|19|47|23|PM|second|evening|dinner| +71244|AAAAAAAANEGBBAAA|71244|19|47|24|PM|second|evening|dinner| +71245|AAAAAAAAOEGBBAAA|71245|19|47|25|PM|second|evening|dinner| +71246|AAAAAAAAPEGBBAAA|71246|19|47|26|PM|second|evening|dinner| +71247|AAAAAAAAAFGBBAAA|71247|19|47|27|PM|second|evening|dinner| +71248|AAAAAAAABFGBBAAA|71248|19|47|28|PM|second|evening|dinner| +71249|AAAAAAAACFGBBAAA|71249|19|47|29|PM|second|evening|dinner| +71250|AAAAAAAADFGBBAAA|71250|19|47|30|PM|second|evening|dinner| +71251|AAAAAAAAEFGBBAAA|71251|19|47|31|PM|second|evening|dinner| +71252|AAAAAAAAFFGBBAAA|71252|19|47|32|PM|second|evening|dinner| +71253|AAAAAAAAGFGBBAAA|71253|19|47|33|PM|second|evening|dinner| +71254|AAAAAAAAHFGBBAAA|71254|19|47|34|PM|second|evening|dinner| +71255|AAAAAAAAIFGBBAAA|71255|19|47|35|PM|second|evening|dinner| +71256|AAAAAAAAJFGBBAAA|71256|19|47|36|PM|second|evening|dinner| +71257|AAAAAAAAKFGBBAAA|71257|19|47|37|PM|second|evening|dinner| +71258|AAAAAAAALFGBBAAA|71258|19|47|38|PM|second|evening|dinner| +71259|AAAAAAAAMFGBBAAA|71259|19|47|39|PM|second|evening|dinner| +71260|AAAAAAAANFGBBAAA|71260|19|47|40|PM|second|evening|dinner| +71261|AAAAAAAAOFGBBAAA|71261|19|47|41|PM|second|evening|dinner| +71262|AAAAAAAAPFGBBAAA|71262|19|47|42|PM|second|evening|dinner| +71263|AAAAAAAAAGGBBAAA|71263|19|47|43|PM|second|evening|dinner| +71264|AAAAAAAABGGBBAAA|71264|19|47|44|PM|second|evening|dinner| +71265|AAAAAAAACGGBBAAA|71265|19|47|45|PM|second|evening|dinner| +71266|AAAAAAAADGGBBAAA|71266|19|47|46|PM|second|evening|dinner| +71267|AAAAAAAAEGGBBAAA|71267|19|47|47|PM|second|evening|dinner| +71268|AAAAAAAAFGGBBAAA|71268|19|47|48|PM|second|evening|dinner| +71269|AAAAAAAAGGGBBAAA|71269|19|47|49|PM|second|evening|dinner| +71270|AAAAAAAAHGGBBAAA|71270|19|47|50|PM|second|evening|dinner| +71271|AAAAAAAAIGGBBAAA|71271|19|47|51|PM|second|evening|dinner| +71272|AAAAAAAAJGGBBAAA|71272|19|47|52|PM|second|evening|dinner| +71273|AAAAAAAAKGGBBAAA|71273|19|47|53|PM|second|evening|dinner| +71274|AAAAAAAALGGBBAAA|71274|19|47|54|PM|second|evening|dinner| +71275|AAAAAAAAMGGBBAAA|71275|19|47|55|PM|second|evening|dinner| +71276|AAAAAAAANGGBBAAA|71276|19|47|56|PM|second|evening|dinner| +71277|AAAAAAAAOGGBBAAA|71277|19|47|57|PM|second|evening|dinner| +71278|AAAAAAAAPGGBBAAA|71278|19|47|58|PM|second|evening|dinner| +71279|AAAAAAAAAHGBBAAA|71279|19|47|59|PM|second|evening|dinner| +71280|AAAAAAAABHGBBAAA|71280|19|48|0|PM|second|evening|dinner| +71281|AAAAAAAACHGBBAAA|71281|19|48|1|PM|second|evening|dinner| +71282|AAAAAAAADHGBBAAA|71282|19|48|2|PM|second|evening|dinner| +71283|AAAAAAAAEHGBBAAA|71283|19|48|3|PM|second|evening|dinner| +71284|AAAAAAAAFHGBBAAA|71284|19|48|4|PM|second|evening|dinner| +71285|AAAAAAAAGHGBBAAA|71285|19|48|5|PM|second|evening|dinner| +71286|AAAAAAAAHHGBBAAA|71286|19|48|6|PM|second|evening|dinner| +71287|AAAAAAAAIHGBBAAA|71287|19|48|7|PM|second|evening|dinner| +71288|AAAAAAAAJHGBBAAA|71288|19|48|8|PM|second|evening|dinner| +71289|AAAAAAAAKHGBBAAA|71289|19|48|9|PM|second|evening|dinner| +71290|AAAAAAAALHGBBAAA|71290|19|48|10|PM|second|evening|dinner| +71291|AAAAAAAAMHGBBAAA|71291|19|48|11|PM|second|evening|dinner| +71292|AAAAAAAANHGBBAAA|71292|19|48|12|PM|second|evening|dinner| +71293|AAAAAAAAOHGBBAAA|71293|19|48|13|PM|second|evening|dinner| +71294|AAAAAAAAPHGBBAAA|71294|19|48|14|PM|second|evening|dinner| +71295|AAAAAAAAAIGBBAAA|71295|19|48|15|PM|second|evening|dinner| +71296|AAAAAAAABIGBBAAA|71296|19|48|16|PM|second|evening|dinner| +71297|AAAAAAAACIGBBAAA|71297|19|48|17|PM|second|evening|dinner| +71298|AAAAAAAADIGBBAAA|71298|19|48|18|PM|second|evening|dinner| +71299|AAAAAAAAEIGBBAAA|71299|19|48|19|PM|second|evening|dinner| +71300|AAAAAAAAFIGBBAAA|71300|19|48|20|PM|second|evening|dinner| +71301|AAAAAAAAGIGBBAAA|71301|19|48|21|PM|second|evening|dinner| +71302|AAAAAAAAHIGBBAAA|71302|19|48|22|PM|second|evening|dinner| +71303|AAAAAAAAIIGBBAAA|71303|19|48|23|PM|second|evening|dinner| +71304|AAAAAAAAJIGBBAAA|71304|19|48|24|PM|second|evening|dinner| +71305|AAAAAAAAKIGBBAAA|71305|19|48|25|PM|second|evening|dinner| +71306|AAAAAAAALIGBBAAA|71306|19|48|26|PM|second|evening|dinner| +71307|AAAAAAAAMIGBBAAA|71307|19|48|27|PM|second|evening|dinner| +71308|AAAAAAAANIGBBAAA|71308|19|48|28|PM|second|evening|dinner| +71309|AAAAAAAAOIGBBAAA|71309|19|48|29|PM|second|evening|dinner| +71310|AAAAAAAAPIGBBAAA|71310|19|48|30|PM|second|evening|dinner| +71311|AAAAAAAAAJGBBAAA|71311|19|48|31|PM|second|evening|dinner| +71312|AAAAAAAABJGBBAAA|71312|19|48|32|PM|second|evening|dinner| +71313|AAAAAAAACJGBBAAA|71313|19|48|33|PM|second|evening|dinner| +71314|AAAAAAAADJGBBAAA|71314|19|48|34|PM|second|evening|dinner| +71315|AAAAAAAAEJGBBAAA|71315|19|48|35|PM|second|evening|dinner| +71316|AAAAAAAAFJGBBAAA|71316|19|48|36|PM|second|evening|dinner| +71317|AAAAAAAAGJGBBAAA|71317|19|48|37|PM|second|evening|dinner| +71318|AAAAAAAAHJGBBAAA|71318|19|48|38|PM|second|evening|dinner| +71319|AAAAAAAAIJGBBAAA|71319|19|48|39|PM|second|evening|dinner| +71320|AAAAAAAAJJGBBAAA|71320|19|48|40|PM|second|evening|dinner| +71321|AAAAAAAAKJGBBAAA|71321|19|48|41|PM|second|evening|dinner| +71322|AAAAAAAALJGBBAAA|71322|19|48|42|PM|second|evening|dinner| +71323|AAAAAAAAMJGBBAAA|71323|19|48|43|PM|second|evening|dinner| +71324|AAAAAAAANJGBBAAA|71324|19|48|44|PM|second|evening|dinner| +71325|AAAAAAAAOJGBBAAA|71325|19|48|45|PM|second|evening|dinner| +71326|AAAAAAAAPJGBBAAA|71326|19|48|46|PM|second|evening|dinner| +71327|AAAAAAAAAKGBBAAA|71327|19|48|47|PM|second|evening|dinner| +71328|AAAAAAAABKGBBAAA|71328|19|48|48|PM|second|evening|dinner| +71329|AAAAAAAACKGBBAAA|71329|19|48|49|PM|second|evening|dinner| +71330|AAAAAAAADKGBBAAA|71330|19|48|50|PM|second|evening|dinner| +71331|AAAAAAAAEKGBBAAA|71331|19|48|51|PM|second|evening|dinner| +71332|AAAAAAAAFKGBBAAA|71332|19|48|52|PM|second|evening|dinner| +71333|AAAAAAAAGKGBBAAA|71333|19|48|53|PM|second|evening|dinner| +71334|AAAAAAAAHKGBBAAA|71334|19|48|54|PM|second|evening|dinner| +71335|AAAAAAAAIKGBBAAA|71335|19|48|55|PM|second|evening|dinner| +71336|AAAAAAAAJKGBBAAA|71336|19|48|56|PM|second|evening|dinner| +71337|AAAAAAAAKKGBBAAA|71337|19|48|57|PM|second|evening|dinner| +71338|AAAAAAAALKGBBAAA|71338|19|48|58|PM|second|evening|dinner| +71339|AAAAAAAAMKGBBAAA|71339|19|48|59|PM|second|evening|dinner| +71340|AAAAAAAANKGBBAAA|71340|19|49|0|PM|second|evening|dinner| +71341|AAAAAAAAOKGBBAAA|71341|19|49|1|PM|second|evening|dinner| +71342|AAAAAAAAPKGBBAAA|71342|19|49|2|PM|second|evening|dinner| +71343|AAAAAAAAALGBBAAA|71343|19|49|3|PM|second|evening|dinner| +71344|AAAAAAAABLGBBAAA|71344|19|49|4|PM|second|evening|dinner| +71345|AAAAAAAACLGBBAAA|71345|19|49|5|PM|second|evening|dinner| +71346|AAAAAAAADLGBBAAA|71346|19|49|6|PM|second|evening|dinner| +71347|AAAAAAAAELGBBAAA|71347|19|49|7|PM|second|evening|dinner| +71348|AAAAAAAAFLGBBAAA|71348|19|49|8|PM|second|evening|dinner| +71349|AAAAAAAAGLGBBAAA|71349|19|49|9|PM|second|evening|dinner| +71350|AAAAAAAAHLGBBAAA|71350|19|49|10|PM|second|evening|dinner| +71351|AAAAAAAAILGBBAAA|71351|19|49|11|PM|second|evening|dinner| +71352|AAAAAAAAJLGBBAAA|71352|19|49|12|PM|second|evening|dinner| +71353|AAAAAAAAKLGBBAAA|71353|19|49|13|PM|second|evening|dinner| +71354|AAAAAAAALLGBBAAA|71354|19|49|14|PM|second|evening|dinner| +71355|AAAAAAAAMLGBBAAA|71355|19|49|15|PM|second|evening|dinner| +71356|AAAAAAAANLGBBAAA|71356|19|49|16|PM|second|evening|dinner| +71357|AAAAAAAAOLGBBAAA|71357|19|49|17|PM|second|evening|dinner| +71358|AAAAAAAAPLGBBAAA|71358|19|49|18|PM|second|evening|dinner| +71359|AAAAAAAAAMGBBAAA|71359|19|49|19|PM|second|evening|dinner| +71360|AAAAAAAABMGBBAAA|71360|19|49|20|PM|second|evening|dinner| +71361|AAAAAAAACMGBBAAA|71361|19|49|21|PM|second|evening|dinner| +71362|AAAAAAAADMGBBAAA|71362|19|49|22|PM|second|evening|dinner| +71363|AAAAAAAAEMGBBAAA|71363|19|49|23|PM|second|evening|dinner| +71364|AAAAAAAAFMGBBAAA|71364|19|49|24|PM|second|evening|dinner| +71365|AAAAAAAAGMGBBAAA|71365|19|49|25|PM|second|evening|dinner| +71366|AAAAAAAAHMGBBAAA|71366|19|49|26|PM|second|evening|dinner| +71367|AAAAAAAAIMGBBAAA|71367|19|49|27|PM|second|evening|dinner| +71368|AAAAAAAAJMGBBAAA|71368|19|49|28|PM|second|evening|dinner| +71369|AAAAAAAAKMGBBAAA|71369|19|49|29|PM|second|evening|dinner| +71370|AAAAAAAALMGBBAAA|71370|19|49|30|PM|second|evening|dinner| +71371|AAAAAAAAMMGBBAAA|71371|19|49|31|PM|second|evening|dinner| +71372|AAAAAAAANMGBBAAA|71372|19|49|32|PM|second|evening|dinner| +71373|AAAAAAAAOMGBBAAA|71373|19|49|33|PM|second|evening|dinner| +71374|AAAAAAAAPMGBBAAA|71374|19|49|34|PM|second|evening|dinner| +71375|AAAAAAAAANGBBAAA|71375|19|49|35|PM|second|evening|dinner| +71376|AAAAAAAABNGBBAAA|71376|19|49|36|PM|second|evening|dinner| +71377|AAAAAAAACNGBBAAA|71377|19|49|37|PM|second|evening|dinner| +71378|AAAAAAAADNGBBAAA|71378|19|49|38|PM|second|evening|dinner| +71379|AAAAAAAAENGBBAAA|71379|19|49|39|PM|second|evening|dinner| +71380|AAAAAAAAFNGBBAAA|71380|19|49|40|PM|second|evening|dinner| +71381|AAAAAAAAGNGBBAAA|71381|19|49|41|PM|second|evening|dinner| +71382|AAAAAAAAHNGBBAAA|71382|19|49|42|PM|second|evening|dinner| +71383|AAAAAAAAINGBBAAA|71383|19|49|43|PM|second|evening|dinner| +71384|AAAAAAAAJNGBBAAA|71384|19|49|44|PM|second|evening|dinner| +71385|AAAAAAAAKNGBBAAA|71385|19|49|45|PM|second|evening|dinner| +71386|AAAAAAAALNGBBAAA|71386|19|49|46|PM|second|evening|dinner| +71387|AAAAAAAAMNGBBAAA|71387|19|49|47|PM|second|evening|dinner| +71388|AAAAAAAANNGBBAAA|71388|19|49|48|PM|second|evening|dinner| +71389|AAAAAAAAONGBBAAA|71389|19|49|49|PM|second|evening|dinner| +71390|AAAAAAAAPNGBBAAA|71390|19|49|50|PM|second|evening|dinner| +71391|AAAAAAAAAOGBBAAA|71391|19|49|51|PM|second|evening|dinner| +71392|AAAAAAAABOGBBAAA|71392|19|49|52|PM|second|evening|dinner| +71393|AAAAAAAACOGBBAAA|71393|19|49|53|PM|second|evening|dinner| +71394|AAAAAAAADOGBBAAA|71394|19|49|54|PM|second|evening|dinner| +71395|AAAAAAAAEOGBBAAA|71395|19|49|55|PM|second|evening|dinner| +71396|AAAAAAAAFOGBBAAA|71396|19|49|56|PM|second|evening|dinner| +71397|AAAAAAAAGOGBBAAA|71397|19|49|57|PM|second|evening|dinner| +71398|AAAAAAAAHOGBBAAA|71398|19|49|58|PM|second|evening|dinner| +71399|AAAAAAAAIOGBBAAA|71399|19|49|59|PM|second|evening|dinner| +71400|AAAAAAAAJOGBBAAA|71400|19|50|0|PM|second|evening|dinner| +71401|AAAAAAAAKOGBBAAA|71401|19|50|1|PM|second|evening|dinner| +71402|AAAAAAAALOGBBAAA|71402|19|50|2|PM|second|evening|dinner| +71403|AAAAAAAAMOGBBAAA|71403|19|50|3|PM|second|evening|dinner| +71404|AAAAAAAANOGBBAAA|71404|19|50|4|PM|second|evening|dinner| +71405|AAAAAAAAOOGBBAAA|71405|19|50|5|PM|second|evening|dinner| +71406|AAAAAAAAPOGBBAAA|71406|19|50|6|PM|second|evening|dinner| +71407|AAAAAAAAAPGBBAAA|71407|19|50|7|PM|second|evening|dinner| +71408|AAAAAAAABPGBBAAA|71408|19|50|8|PM|second|evening|dinner| +71409|AAAAAAAACPGBBAAA|71409|19|50|9|PM|second|evening|dinner| +71410|AAAAAAAADPGBBAAA|71410|19|50|10|PM|second|evening|dinner| +71411|AAAAAAAAEPGBBAAA|71411|19|50|11|PM|second|evening|dinner| +71412|AAAAAAAAFPGBBAAA|71412|19|50|12|PM|second|evening|dinner| +71413|AAAAAAAAGPGBBAAA|71413|19|50|13|PM|second|evening|dinner| +71414|AAAAAAAAHPGBBAAA|71414|19|50|14|PM|second|evening|dinner| +71415|AAAAAAAAIPGBBAAA|71415|19|50|15|PM|second|evening|dinner| +71416|AAAAAAAAJPGBBAAA|71416|19|50|16|PM|second|evening|dinner| +71417|AAAAAAAAKPGBBAAA|71417|19|50|17|PM|second|evening|dinner| +71418|AAAAAAAALPGBBAAA|71418|19|50|18|PM|second|evening|dinner| +71419|AAAAAAAAMPGBBAAA|71419|19|50|19|PM|second|evening|dinner| +71420|AAAAAAAANPGBBAAA|71420|19|50|20|PM|second|evening|dinner| +71421|AAAAAAAAOPGBBAAA|71421|19|50|21|PM|second|evening|dinner| +71422|AAAAAAAAPPGBBAAA|71422|19|50|22|PM|second|evening|dinner| +71423|AAAAAAAAAAHBBAAA|71423|19|50|23|PM|second|evening|dinner| +71424|AAAAAAAABAHBBAAA|71424|19|50|24|PM|second|evening|dinner| +71425|AAAAAAAACAHBBAAA|71425|19|50|25|PM|second|evening|dinner| +71426|AAAAAAAADAHBBAAA|71426|19|50|26|PM|second|evening|dinner| +71427|AAAAAAAAEAHBBAAA|71427|19|50|27|PM|second|evening|dinner| +71428|AAAAAAAAFAHBBAAA|71428|19|50|28|PM|second|evening|dinner| +71429|AAAAAAAAGAHBBAAA|71429|19|50|29|PM|second|evening|dinner| +71430|AAAAAAAAHAHBBAAA|71430|19|50|30|PM|second|evening|dinner| +71431|AAAAAAAAIAHBBAAA|71431|19|50|31|PM|second|evening|dinner| +71432|AAAAAAAAJAHBBAAA|71432|19|50|32|PM|second|evening|dinner| +71433|AAAAAAAAKAHBBAAA|71433|19|50|33|PM|second|evening|dinner| +71434|AAAAAAAALAHBBAAA|71434|19|50|34|PM|second|evening|dinner| +71435|AAAAAAAAMAHBBAAA|71435|19|50|35|PM|second|evening|dinner| +71436|AAAAAAAANAHBBAAA|71436|19|50|36|PM|second|evening|dinner| +71437|AAAAAAAAOAHBBAAA|71437|19|50|37|PM|second|evening|dinner| +71438|AAAAAAAAPAHBBAAA|71438|19|50|38|PM|second|evening|dinner| +71439|AAAAAAAAABHBBAAA|71439|19|50|39|PM|second|evening|dinner| +71440|AAAAAAAABBHBBAAA|71440|19|50|40|PM|second|evening|dinner| +71441|AAAAAAAACBHBBAAA|71441|19|50|41|PM|second|evening|dinner| +71442|AAAAAAAADBHBBAAA|71442|19|50|42|PM|second|evening|dinner| +71443|AAAAAAAAEBHBBAAA|71443|19|50|43|PM|second|evening|dinner| +71444|AAAAAAAAFBHBBAAA|71444|19|50|44|PM|second|evening|dinner| +71445|AAAAAAAAGBHBBAAA|71445|19|50|45|PM|second|evening|dinner| +71446|AAAAAAAAHBHBBAAA|71446|19|50|46|PM|second|evening|dinner| +71447|AAAAAAAAIBHBBAAA|71447|19|50|47|PM|second|evening|dinner| +71448|AAAAAAAAJBHBBAAA|71448|19|50|48|PM|second|evening|dinner| +71449|AAAAAAAAKBHBBAAA|71449|19|50|49|PM|second|evening|dinner| +71450|AAAAAAAALBHBBAAA|71450|19|50|50|PM|second|evening|dinner| +71451|AAAAAAAAMBHBBAAA|71451|19|50|51|PM|second|evening|dinner| +71452|AAAAAAAANBHBBAAA|71452|19|50|52|PM|second|evening|dinner| +71453|AAAAAAAAOBHBBAAA|71453|19|50|53|PM|second|evening|dinner| +71454|AAAAAAAAPBHBBAAA|71454|19|50|54|PM|second|evening|dinner| +71455|AAAAAAAAACHBBAAA|71455|19|50|55|PM|second|evening|dinner| +71456|AAAAAAAABCHBBAAA|71456|19|50|56|PM|second|evening|dinner| +71457|AAAAAAAACCHBBAAA|71457|19|50|57|PM|second|evening|dinner| +71458|AAAAAAAADCHBBAAA|71458|19|50|58|PM|second|evening|dinner| +71459|AAAAAAAAECHBBAAA|71459|19|50|59|PM|second|evening|dinner| +71460|AAAAAAAAFCHBBAAA|71460|19|51|0|PM|second|evening|dinner| +71461|AAAAAAAAGCHBBAAA|71461|19|51|1|PM|second|evening|dinner| +71462|AAAAAAAAHCHBBAAA|71462|19|51|2|PM|second|evening|dinner| +71463|AAAAAAAAICHBBAAA|71463|19|51|3|PM|second|evening|dinner| +71464|AAAAAAAAJCHBBAAA|71464|19|51|4|PM|second|evening|dinner| +71465|AAAAAAAAKCHBBAAA|71465|19|51|5|PM|second|evening|dinner| +71466|AAAAAAAALCHBBAAA|71466|19|51|6|PM|second|evening|dinner| +71467|AAAAAAAAMCHBBAAA|71467|19|51|7|PM|second|evening|dinner| +71468|AAAAAAAANCHBBAAA|71468|19|51|8|PM|second|evening|dinner| +71469|AAAAAAAAOCHBBAAA|71469|19|51|9|PM|second|evening|dinner| +71470|AAAAAAAAPCHBBAAA|71470|19|51|10|PM|second|evening|dinner| +71471|AAAAAAAAADHBBAAA|71471|19|51|11|PM|second|evening|dinner| +71472|AAAAAAAABDHBBAAA|71472|19|51|12|PM|second|evening|dinner| +71473|AAAAAAAACDHBBAAA|71473|19|51|13|PM|second|evening|dinner| +71474|AAAAAAAADDHBBAAA|71474|19|51|14|PM|second|evening|dinner| +71475|AAAAAAAAEDHBBAAA|71475|19|51|15|PM|second|evening|dinner| +71476|AAAAAAAAFDHBBAAA|71476|19|51|16|PM|second|evening|dinner| +71477|AAAAAAAAGDHBBAAA|71477|19|51|17|PM|second|evening|dinner| +71478|AAAAAAAAHDHBBAAA|71478|19|51|18|PM|second|evening|dinner| +71479|AAAAAAAAIDHBBAAA|71479|19|51|19|PM|second|evening|dinner| +71480|AAAAAAAAJDHBBAAA|71480|19|51|20|PM|second|evening|dinner| +71481|AAAAAAAAKDHBBAAA|71481|19|51|21|PM|second|evening|dinner| +71482|AAAAAAAALDHBBAAA|71482|19|51|22|PM|second|evening|dinner| +71483|AAAAAAAAMDHBBAAA|71483|19|51|23|PM|second|evening|dinner| +71484|AAAAAAAANDHBBAAA|71484|19|51|24|PM|second|evening|dinner| +71485|AAAAAAAAODHBBAAA|71485|19|51|25|PM|second|evening|dinner| +71486|AAAAAAAAPDHBBAAA|71486|19|51|26|PM|second|evening|dinner| +71487|AAAAAAAAAEHBBAAA|71487|19|51|27|PM|second|evening|dinner| +71488|AAAAAAAABEHBBAAA|71488|19|51|28|PM|second|evening|dinner| +71489|AAAAAAAACEHBBAAA|71489|19|51|29|PM|second|evening|dinner| +71490|AAAAAAAADEHBBAAA|71490|19|51|30|PM|second|evening|dinner| +71491|AAAAAAAAEEHBBAAA|71491|19|51|31|PM|second|evening|dinner| +71492|AAAAAAAAFEHBBAAA|71492|19|51|32|PM|second|evening|dinner| +71493|AAAAAAAAGEHBBAAA|71493|19|51|33|PM|second|evening|dinner| +71494|AAAAAAAAHEHBBAAA|71494|19|51|34|PM|second|evening|dinner| +71495|AAAAAAAAIEHBBAAA|71495|19|51|35|PM|second|evening|dinner| +71496|AAAAAAAAJEHBBAAA|71496|19|51|36|PM|second|evening|dinner| +71497|AAAAAAAAKEHBBAAA|71497|19|51|37|PM|second|evening|dinner| +71498|AAAAAAAALEHBBAAA|71498|19|51|38|PM|second|evening|dinner| +71499|AAAAAAAAMEHBBAAA|71499|19|51|39|PM|second|evening|dinner| +71500|AAAAAAAANEHBBAAA|71500|19|51|40|PM|second|evening|dinner| +71501|AAAAAAAAOEHBBAAA|71501|19|51|41|PM|second|evening|dinner| +71502|AAAAAAAAPEHBBAAA|71502|19|51|42|PM|second|evening|dinner| +71503|AAAAAAAAAFHBBAAA|71503|19|51|43|PM|second|evening|dinner| +71504|AAAAAAAABFHBBAAA|71504|19|51|44|PM|second|evening|dinner| +71505|AAAAAAAACFHBBAAA|71505|19|51|45|PM|second|evening|dinner| +71506|AAAAAAAADFHBBAAA|71506|19|51|46|PM|second|evening|dinner| +71507|AAAAAAAAEFHBBAAA|71507|19|51|47|PM|second|evening|dinner| +71508|AAAAAAAAFFHBBAAA|71508|19|51|48|PM|second|evening|dinner| +71509|AAAAAAAAGFHBBAAA|71509|19|51|49|PM|second|evening|dinner| +71510|AAAAAAAAHFHBBAAA|71510|19|51|50|PM|second|evening|dinner| +71511|AAAAAAAAIFHBBAAA|71511|19|51|51|PM|second|evening|dinner| +71512|AAAAAAAAJFHBBAAA|71512|19|51|52|PM|second|evening|dinner| +71513|AAAAAAAAKFHBBAAA|71513|19|51|53|PM|second|evening|dinner| +71514|AAAAAAAALFHBBAAA|71514|19|51|54|PM|second|evening|dinner| +71515|AAAAAAAAMFHBBAAA|71515|19|51|55|PM|second|evening|dinner| +71516|AAAAAAAANFHBBAAA|71516|19|51|56|PM|second|evening|dinner| +71517|AAAAAAAAOFHBBAAA|71517|19|51|57|PM|second|evening|dinner| +71518|AAAAAAAAPFHBBAAA|71518|19|51|58|PM|second|evening|dinner| +71519|AAAAAAAAAGHBBAAA|71519|19|51|59|PM|second|evening|dinner| +71520|AAAAAAAABGHBBAAA|71520|19|52|0|PM|second|evening|dinner| +71521|AAAAAAAACGHBBAAA|71521|19|52|1|PM|second|evening|dinner| +71522|AAAAAAAADGHBBAAA|71522|19|52|2|PM|second|evening|dinner| +71523|AAAAAAAAEGHBBAAA|71523|19|52|3|PM|second|evening|dinner| +71524|AAAAAAAAFGHBBAAA|71524|19|52|4|PM|second|evening|dinner| +71525|AAAAAAAAGGHBBAAA|71525|19|52|5|PM|second|evening|dinner| +71526|AAAAAAAAHGHBBAAA|71526|19|52|6|PM|second|evening|dinner| +71527|AAAAAAAAIGHBBAAA|71527|19|52|7|PM|second|evening|dinner| +71528|AAAAAAAAJGHBBAAA|71528|19|52|8|PM|second|evening|dinner| +71529|AAAAAAAAKGHBBAAA|71529|19|52|9|PM|second|evening|dinner| +71530|AAAAAAAALGHBBAAA|71530|19|52|10|PM|second|evening|dinner| +71531|AAAAAAAAMGHBBAAA|71531|19|52|11|PM|second|evening|dinner| +71532|AAAAAAAANGHBBAAA|71532|19|52|12|PM|second|evening|dinner| +71533|AAAAAAAAOGHBBAAA|71533|19|52|13|PM|second|evening|dinner| +71534|AAAAAAAAPGHBBAAA|71534|19|52|14|PM|second|evening|dinner| +71535|AAAAAAAAAHHBBAAA|71535|19|52|15|PM|second|evening|dinner| +71536|AAAAAAAABHHBBAAA|71536|19|52|16|PM|second|evening|dinner| +71537|AAAAAAAACHHBBAAA|71537|19|52|17|PM|second|evening|dinner| +71538|AAAAAAAADHHBBAAA|71538|19|52|18|PM|second|evening|dinner| +71539|AAAAAAAAEHHBBAAA|71539|19|52|19|PM|second|evening|dinner| +71540|AAAAAAAAFHHBBAAA|71540|19|52|20|PM|second|evening|dinner| +71541|AAAAAAAAGHHBBAAA|71541|19|52|21|PM|second|evening|dinner| +71542|AAAAAAAAHHHBBAAA|71542|19|52|22|PM|second|evening|dinner| +71543|AAAAAAAAIHHBBAAA|71543|19|52|23|PM|second|evening|dinner| +71544|AAAAAAAAJHHBBAAA|71544|19|52|24|PM|second|evening|dinner| +71545|AAAAAAAAKHHBBAAA|71545|19|52|25|PM|second|evening|dinner| +71546|AAAAAAAALHHBBAAA|71546|19|52|26|PM|second|evening|dinner| +71547|AAAAAAAAMHHBBAAA|71547|19|52|27|PM|second|evening|dinner| +71548|AAAAAAAANHHBBAAA|71548|19|52|28|PM|second|evening|dinner| +71549|AAAAAAAAOHHBBAAA|71549|19|52|29|PM|second|evening|dinner| +71550|AAAAAAAAPHHBBAAA|71550|19|52|30|PM|second|evening|dinner| +71551|AAAAAAAAAIHBBAAA|71551|19|52|31|PM|second|evening|dinner| +71552|AAAAAAAABIHBBAAA|71552|19|52|32|PM|second|evening|dinner| +71553|AAAAAAAACIHBBAAA|71553|19|52|33|PM|second|evening|dinner| +71554|AAAAAAAADIHBBAAA|71554|19|52|34|PM|second|evening|dinner| +71555|AAAAAAAAEIHBBAAA|71555|19|52|35|PM|second|evening|dinner| +71556|AAAAAAAAFIHBBAAA|71556|19|52|36|PM|second|evening|dinner| +71557|AAAAAAAAGIHBBAAA|71557|19|52|37|PM|second|evening|dinner| +71558|AAAAAAAAHIHBBAAA|71558|19|52|38|PM|second|evening|dinner| +71559|AAAAAAAAIIHBBAAA|71559|19|52|39|PM|second|evening|dinner| +71560|AAAAAAAAJIHBBAAA|71560|19|52|40|PM|second|evening|dinner| +71561|AAAAAAAAKIHBBAAA|71561|19|52|41|PM|second|evening|dinner| +71562|AAAAAAAALIHBBAAA|71562|19|52|42|PM|second|evening|dinner| +71563|AAAAAAAAMIHBBAAA|71563|19|52|43|PM|second|evening|dinner| +71564|AAAAAAAANIHBBAAA|71564|19|52|44|PM|second|evening|dinner| +71565|AAAAAAAAOIHBBAAA|71565|19|52|45|PM|second|evening|dinner| +71566|AAAAAAAAPIHBBAAA|71566|19|52|46|PM|second|evening|dinner| +71567|AAAAAAAAAJHBBAAA|71567|19|52|47|PM|second|evening|dinner| +71568|AAAAAAAABJHBBAAA|71568|19|52|48|PM|second|evening|dinner| +71569|AAAAAAAACJHBBAAA|71569|19|52|49|PM|second|evening|dinner| +71570|AAAAAAAADJHBBAAA|71570|19|52|50|PM|second|evening|dinner| +71571|AAAAAAAAEJHBBAAA|71571|19|52|51|PM|second|evening|dinner| +71572|AAAAAAAAFJHBBAAA|71572|19|52|52|PM|second|evening|dinner| +71573|AAAAAAAAGJHBBAAA|71573|19|52|53|PM|second|evening|dinner| +71574|AAAAAAAAHJHBBAAA|71574|19|52|54|PM|second|evening|dinner| +71575|AAAAAAAAIJHBBAAA|71575|19|52|55|PM|second|evening|dinner| +71576|AAAAAAAAJJHBBAAA|71576|19|52|56|PM|second|evening|dinner| +71577|AAAAAAAAKJHBBAAA|71577|19|52|57|PM|second|evening|dinner| +71578|AAAAAAAALJHBBAAA|71578|19|52|58|PM|second|evening|dinner| +71579|AAAAAAAAMJHBBAAA|71579|19|52|59|PM|second|evening|dinner| +71580|AAAAAAAANJHBBAAA|71580|19|53|0|PM|second|evening|dinner| +71581|AAAAAAAAOJHBBAAA|71581|19|53|1|PM|second|evening|dinner| +71582|AAAAAAAAPJHBBAAA|71582|19|53|2|PM|second|evening|dinner| +71583|AAAAAAAAAKHBBAAA|71583|19|53|3|PM|second|evening|dinner| +71584|AAAAAAAABKHBBAAA|71584|19|53|4|PM|second|evening|dinner| +71585|AAAAAAAACKHBBAAA|71585|19|53|5|PM|second|evening|dinner| +71586|AAAAAAAADKHBBAAA|71586|19|53|6|PM|second|evening|dinner| +71587|AAAAAAAAEKHBBAAA|71587|19|53|7|PM|second|evening|dinner| +71588|AAAAAAAAFKHBBAAA|71588|19|53|8|PM|second|evening|dinner| +71589|AAAAAAAAGKHBBAAA|71589|19|53|9|PM|second|evening|dinner| +71590|AAAAAAAAHKHBBAAA|71590|19|53|10|PM|second|evening|dinner| +71591|AAAAAAAAIKHBBAAA|71591|19|53|11|PM|second|evening|dinner| +71592|AAAAAAAAJKHBBAAA|71592|19|53|12|PM|second|evening|dinner| +71593|AAAAAAAAKKHBBAAA|71593|19|53|13|PM|second|evening|dinner| +71594|AAAAAAAALKHBBAAA|71594|19|53|14|PM|second|evening|dinner| +71595|AAAAAAAAMKHBBAAA|71595|19|53|15|PM|second|evening|dinner| +71596|AAAAAAAANKHBBAAA|71596|19|53|16|PM|second|evening|dinner| +71597|AAAAAAAAOKHBBAAA|71597|19|53|17|PM|second|evening|dinner| +71598|AAAAAAAAPKHBBAAA|71598|19|53|18|PM|second|evening|dinner| +71599|AAAAAAAAALHBBAAA|71599|19|53|19|PM|second|evening|dinner| +71600|AAAAAAAABLHBBAAA|71600|19|53|20|PM|second|evening|dinner| +71601|AAAAAAAACLHBBAAA|71601|19|53|21|PM|second|evening|dinner| +71602|AAAAAAAADLHBBAAA|71602|19|53|22|PM|second|evening|dinner| +71603|AAAAAAAAELHBBAAA|71603|19|53|23|PM|second|evening|dinner| +71604|AAAAAAAAFLHBBAAA|71604|19|53|24|PM|second|evening|dinner| +71605|AAAAAAAAGLHBBAAA|71605|19|53|25|PM|second|evening|dinner| +71606|AAAAAAAAHLHBBAAA|71606|19|53|26|PM|second|evening|dinner| +71607|AAAAAAAAILHBBAAA|71607|19|53|27|PM|second|evening|dinner| +71608|AAAAAAAAJLHBBAAA|71608|19|53|28|PM|second|evening|dinner| +71609|AAAAAAAAKLHBBAAA|71609|19|53|29|PM|second|evening|dinner| +71610|AAAAAAAALLHBBAAA|71610|19|53|30|PM|second|evening|dinner| +71611|AAAAAAAAMLHBBAAA|71611|19|53|31|PM|second|evening|dinner| +71612|AAAAAAAANLHBBAAA|71612|19|53|32|PM|second|evening|dinner| +71613|AAAAAAAAOLHBBAAA|71613|19|53|33|PM|second|evening|dinner| +71614|AAAAAAAAPLHBBAAA|71614|19|53|34|PM|second|evening|dinner| +71615|AAAAAAAAAMHBBAAA|71615|19|53|35|PM|second|evening|dinner| +71616|AAAAAAAABMHBBAAA|71616|19|53|36|PM|second|evening|dinner| +71617|AAAAAAAACMHBBAAA|71617|19|53|37|PM|second|evening|dinner| +71618|AAAAAAAADMHBBAAA|71618|19|53|38|PM|second|evening|dinner| +71619|AAAAAAAAEMHBBAAA|71619|19|53|39|PM|second|evening|dinner| +71620|AAAAAAAAFMHBBAAA|71620|19|53|40|PM|second|evening|dinner| +71621|AAAAAAAAGMHBBAAA|71621|19|53|41|PM|second|evening|dinner| +71622|AAAAAAAAHMHBBAAA|71622|19|53|42|PM|second|evening|dinner| +71623|AAAAAAAAIMHBBAAA|71623|19|53|43|PM|second|evening|dinner| +71624|AAAAAAAAJMHBBAAA|71624|19|53|44|PM|second|evening|dinner| +71625|AAAAAAAAKMHBBAAA|71625|19|53|45|PM|second|evening|dinner| +71626|AAAAAAAALMHBBAAA|71626|19|53|46|PM|second|evening|dinner| +71627|AAAAAAAAMMHBBAAA|71627|19|53|47|PM|second|evening|dinner| +71628|AAAAAAAANMHBBAAA|71628|19|53|48|PM|second|evening|dinner| +71629|AAAAAAAAOMHBBAAA|71629|19|53|49|PM|second|evening|dinner| +71630|AAAAAAAAPMHBBAAA|71630|19|53|50|PM|second|evening|dinner| +71631|AAAAAAAAANHBBAAA|71631|19|53|51|PM|second|evening|dinner| +71632|AAAAAAAABNHBBAAA|71632|19|53|52|PM|second|evening|dinner| +71633|AAAAAAAACNHBBAAA|71633|19|53|53|PM|second|evening|dinner| +71634|AAAAAAAADNHBBAAA|71634|19|53|54|PM|second|evening|dinner| +71635|AAAAAAAAENHBBAAA|71635|19|53|55|PM|second|evening|dinner| +71636|AAAAAAAAFNHBBAAA|71636|19|53|56|PM|second|evening|dinner| +71637|AAAAAAAAGNHBBAAA|71637|19|53|57|PM|second|evening|dinner| +71638|AAAAAAAAHNHBBAAA|71638|19|53|58|PM|second|evening|dinner| +71639|AAAAAAAAINHBBAAA|71639|19|53|59|PM|second|evening|dinner| +71640|AAAAAAAAJNHBBAAA|71640|19|54|0|PM|second|evening|dinner| +71641|AAAAAAAAKNHBBAAA|71641|19|54|1|PM|second|evening|dinner| +71642|AAAAAAAALNHBBAAA|71642|19|54|2|PM|second|evening|dinner| +71643|AAAAAAAAMNHBBAAA|71643|19|54|3|PM|second|evening|dinner| +71644|AAAAAAAANNHBBAAA|71644|19|54|4|PM|second|evening|dinner| +71645|AAAAAAAAONHBBAAA|71645|19|54|5|PM|second|evening|dinner| +71646|AAAAAAAAPNHBBAAA|71646|19|54|6|PM|second|evening|dinner| +71647|AAAAAAAAAOHBBAAA|71647|19|54|7|PM|second|evening|dinner| +71648|AAAAAAAABOHBBAAA|71648|19|54|8|PM|second|evening|dinner| +71649|AAAAAAAACOHBBAAA|71649|19|54|9|PM|second|evening|dinner| +71650|AAAAAAAADOHBBAAA|71650|19|54|10|PM|second|evening|dinner| +71651|AAAAAAAAEOHBBAAA|71651|19|54|11|PM|second|evening|dinner| +71652|AAAAAAAAFOHBBAAA|71652|19|54|12|PM|second|evening|dinner| +71653|AAAAAAAAGOHBBAAA|71653|19|54|13|PM|second|evening|dinner| +71654|AAAAAAAAHOHBBAAA|71654|19|54|14|PM|second|evening|dinner| +71655|AAAAAAAAIOHBBAAA|71655|19|54|15|PM|second|evening|dinner| +71656|AAAAAAAAJOHBBAAA|71656|19|54|16|PM|second|evening|dinner| +71657|AAAAAAAAKOHBBAAA|71657|19|54|17|PM|second|evening|dinner| +71658|AAAAAAAALOHBBAAA|71658|19|54|18|PM|second|evening|dinner| +71659|AAAAAAAAMOHBBAAA|71659|19|54|19|PM|second|evening|dinner| +71660|AAAAAAAANOHBBAAA|71660|19|54|20|PM|second|evening|dinner| +71661|AAAAAAAAOOHBBAAA|71661|19|54|21|PM|second|evening|dinner| +71662|AAAAAAAAPOHBBAAA|71662|19|54|22|PM|second|evening|dinner| +71663|AAAAAAAAAPHBBAAA|71663|19|54|23|PM|second|evening|dinner| +71664|AAAAAAAABPHBBAAA|71664|19|54|24|PM|second|evening|dinner| +71665|AAAAAAAACPHBBAAA|71665|19|54|25|PM|second|evening|dinner| +71666|AAAAAAAADPHBBAAA|71666|19|54|26|PM|second|evening|dinner| +71667|AAAAAAAAEPHBBAAA|71667|19|54|27|PM|second|evening|dinner| +71668|AAAAAAAAFPHBBAAA|71668|19|54|28|PM|second|evening|dinner| +71669|AAAAAAAAGPHBBAAA|71669|19|54|29|PM|second|evening|dinner| +71670|AAAAAAAAHPHBBAAA|71670|19|54|30|PM|second|evening|dinner| +71671|AAAAAAAAIPHBBAAA|71671|19|54|31|PM|second|evening|dinner| +71672|AAAAAAAAJPHBBAAA|71672|19|54|32|PM|second|evening|dinner| +71673|AAAAAAAAKPHBBAAA|71673|19|54|33|PM|second|evening|dinner| +71674|AAAAAAAALPHBBAAA|71674|19|54|34|PM|second|evening|dinner| +71675|AAAAAAAAMPHBBAAA|71675|19|54|35|PM|second|evening|dinner| +71676|AAAAAAAANPHBBAAA|71676|19|54|36|PM|second|evening|dinner| +71677|AAAAAAAAOPHBBAAA|71677|19|54|37|PM|second|evening|dinner| +71678|AAAAAAAAPPHBBAAA|71678|19|54|38|PM|second|evening|dinner| +71679|AAAAAAAAAAIBBAAA|71679|19|54|39|PM|second|evening|dinner| +71680|AAAAAAAABAIBBAAA|71680|19|54|40|PM|second|evening|dinner| +71681|AAAAAAAACAIBBAAA|71681|19|54|41|PM|second|evening|dinner| +71682|AAAAAAAADAIBBAAA|71682|19|54|42|PM|second|evening|dinner| +71683|AAAAAAAAEAIBBAAA|71683|19|54|43|PM|second|evening|dinner| +71684|AAAAAAAAFAIBBAAA|71684|19|54|44|PM|second|evening|dinner| +71685|AAAAAAAAGAIBBAAA|71685|19|54|45|PM|second|evening|dinner| +71686|AAAAAAAAHAIBBAAA|71686|19|54|46|PM|second|evening|dinner| +71687|AAAAAAAAIAIBBAAA|71687|19|54|47|PM|second|evening|dinner| +71688|AAAAAAAAJAIBBAAA|71688|19|54|48|PM|second|evening|dinner| +71689|AAAAAAAAKAIBBAAA|71689|19|54|49|PM|second|evening|dinner| +71690|AAAAAAAALAIBBAAA|71690|19|54|50|PM|second|evening|dinner| +71691|AAAAAAAAMAIBBAAA|71691|19|54|51|PM|second|evening|dinner| +71692|AAAAAAAANAIBBAAA|71692|19|54|52|PM|second|evening|dinner| +71693|AAAAAAAAOAIBBAAA|71693|19|54|53|PM|second|evening|dinner| +71694|AAAAAAAAPAIBBAAA|71694|19|54|54|PM|second|evening|dinner| +71695|AAAAAAAAABIBBAAA|71695|19|54|55|PM|second|evening|dinner| +71696|AAAAAAAABBIBBAAA|71696|19|54|56|PM|second|evening|dinner| +71697|AAAAAAAACBIBBAAA|71697|19|54|57|PM|second|evening|dinner| +71698|AAAAAAAADBIBBAAA|71698|19|54|58|PM|second|evening|dinner| +71699|AAAAAAAAEBIBBAAA|71699|19|54|59|PM|second|evening|dinner| +71700|AAAAAAAAFBIBBAAA|71700|19|55|0|PM|second|evening|dinner| +71701|AAAAAAAAGBIBBAAA|71701|19|55|1|PM|second|evening|dinner| +71702|AAAAAAAAHBIBBAAA|71702|19|55|2|PM|second|evening|dinner| +71703|AAAAAAAAIBIBBAAA|71703|19|55|3|PM|second|evening|dinner| +71704|AAAAAAAAJBIBBAAA|71704|19|55|4|PM|second|evening|dinner| +71705|AAAAAAAAKBIBBAAA|71705|19|55|5|PM|second|evening|dinner| +71706|AAAAAAAALBIBBAAA|71706|19|55|6|PM|second|evening|dinner| +71707|AAAAAAAAMBIBBAAA|71707|19|55|7|PM|second|evening|dinner| +71708|AAAAAAAANBIBBAAA|71708|19|55|8|PM|second|evening|dinner| +71709|AAAAAAAAOBIBBAAA|71709|19|55|9|PM|second|evening|dinner| +71710|AAAAAAAAPBIBBAAA|71710|19|55|10|PM|second|evening|dinner| +71711|AAAAAAAAACIBBAAA|71711|19|55|11|PM|second|evening|dinner| +71712|AAAAAAAABCIBBAAA|71712|19|55|12|PM|second|evening|dinner| +71713|AAAAAAAACCIBBAAA|71713|19|55|13|PM|second|evening|dinner| +71714|AAAAAAAADCIBBAAA|71714|19|55|14|PM|second|evening|dinner| +71715|AAAAAAAAECIBBAAA|71715|19|55|15|PM|second|evening|dinner| +71716|AAAAAAAAFCIBBAAA|71716|19|55|16|PM|second|evening|dinner| +71717|AAAAAAAAGCIBBAAA|71717|19|55|17|PM|second|evening|dinner| +71718|AAAAAAAAHCIBBAAA|71718|19|55|18|PM|second|evening|dinner| +71719|AAAAAAAAICIBBAAA|71719|19|55|19|PM|second|evening|dinner| +71720|AAAAAAAAJCIBBAAA|71720|19|55|20|PM|second|evening|dinner| +71721|AAAAAAAAKCIBBAAA|71721|19|55|21|PM|second|evening|dinner| +71722|AAAAAAAALCIBBAAA|71722|19|55|22|PM|second|evening|dinner| +71723|AAAAAAAAMCIBBAAA|71723|19|55|23|PM|second|evening|dinner| +71724|AAAAAAAANCIBBAAA|71724|19|55|24|PM|second|evening|dinner| +71725|AAAAAAAAOCIBBAAA|71725|19|55|25|PM|second|evening|dinner| +71726|AAAAAAAAPCIBBAAA|71726|19|55|26|PM|second|evening|dinner| +71727|AAAAAAAAADIBBAAA|71727|19|55|27|PM|second|evening|dinner| +71728|AAAAAAAABDIBBAAA|71728|19|55|28|PM|second|evening|dinner| +71729|AAAAAAAACDIBBAAA|71729|19|55|29|PM|second|evening|dinner| +71730|AAAAAAAADDIBBAAA|71730|19|55|30|PM|second|evening|dinner| +71731|AAAAAAAAEDIBBAAA|71731|19|55|31|PM|second|evening|dinner| +71732|AAAAAAAAFDIBBAAA|71732|19|55|32|PM|second|evening|dinner| +71733|AAAAAAAAGDIBBAAA|71733|19|55|33|PM|second|evening|dinner| +71734|AAAAAAAAHDIBBAAA|71734|19|55|34|PM|second|evening|dinner| +71735|AAAAAAAAIDIBBAAA|71735|19|55|35|PM|second|evening|dinner| +71736|AAAAAAAAJDIBBAAA|71736|19|55|36|PM|second|evening|dinner| +71737|AAAAAAAAKDIBBAAA|71737|19|55|37|PM|second|evening|dinner| +71738|AAAAAAAALDIBBAAA|71738|19|55|38|PM|second|evening|dinner| +71739|AAAAAAAAMDIBBAAA|71739|19|55|39|PM|second|evening|dinner| +71740|AAAAAAAANDIBBAAA|71740|19|55|40|PM|second|evening|dinner| +71741|AAAAAAAAODIBBAAA|71741|19|55|41|PM|second|evening|dinner| +71742|AAAAAAAAPDIBBAAA|71742|19|55|42|PM|second|evening|dinner| +71743|AAAAAAAAAEIBBAAA|71743|19|55|43|PM|second|evening|dinner| +71744|AAAAAAAABEIBBAAA|71744|19|55|44|PM|second|evening|dinner| +71745|AAAAAAAACEIBBAAA|71745|19|55|45|PM|second|evening|dinner| +71746|AAAAAAAADEIBBAAA|71746|19|55|46|PM|second|evening|dinner| +71747|AAAAAAAAEEIBBAAA|71747|19|55|47|PM|second|evening|dinner| +71748|AAAAAAAAFEIBBAAA|71748|19|55|48|PM|second|evening|dinner| +71749|AAAAAAAAGEIBBAAA|71749|19|55|49|PM|second|evening|dinner| +71750|AAAAAAAAHEIBBAAA|71750|19|55|50|PM|second|evening|dinner| +71751|AAAAAAAAIEIBBAAA|71751|19|55|51|PM|second|evening|dinner| +71752|AAAAAAAAJEIBBAAA|71752|19|55|52|PM|second|evening|dinner| +71753|AAAAAAAAKEIBBAAA|71753|19|55|53|PM|second|evening|dinner| +71754|AAAAAAAALEIBBAAA|71754|19|55|54|PM|second|evening|dinner| +71755|AAAAAAAAMEIBBAAA|71755|19|55|55|PM|second|evening|dinner| +71756|AAAAAAAANEIBBAAA|71756|19|55|56|PM|second|evening|dinner| +71757|AAAAAAAAOEIBBAAA|71757|19|55|57|PM|second|evening|dinner| +71758|AAAAAAAAPEIBBAAA|71758|19|55|58|PM|second|evening|dinner| +71759|AAAAAAAAAFIBBAAA|71759|19|55|59|PM|second|evening|dinner| +71760|AAAAAAAABFIBBAAA|71760|19|56|0|PM|second|evening|dinner| +71761|AAAAAAAACFIBBAAA|71761|19|56|1|PM|second|evening|dinner| +71762|AAAAAAAADFIBBAAA|71762|19|56|2|PM|second|evening|dinner| +71763|AAAAAAAAEFIBBAAA|71763|19|56|3|PM|second|evening|dinner| +71764|AAAAAAAAFFIBBAAA|71764|19|56|4|PM|second|evening|dinner| +71765|AAAAAAAAGFIBBAAA|71765|19|56|5|PM|second|evening|dinner| +71766|AAAAAAAAHFIBBAAA|71766|19|56|6|PM|second|evening|dinner| +71767|AAAAAAAAIFIBBAAA|71767|19|56|7|PM|second|evening|dinner| +71768|AAAAAAAAJFIBBAAA|71768|19|56|8|PM|second|evening|dinner| +71769|AAAAAAAAKFIBBAAA|71769|19|56|9|PM|second|evening|dinner| +71770|AAAAAAAALFIBBAAA|71770|19|56|10|PM|second|evening|dinner| +71771|AAAAAAAAMFIBBAAA|71771|19|56|11|PM|second|evening|dinner| +71772|AAAAAAAANFIBBAAA|71772|19|56|12|PM|second|evening|dinner| +71773|AAAAAAAAOFIBBAAA|71773|19|56|13|PM|second|evening|dinner| +71774|AAAAAAAAPFIBBAAA|71774|19|56|14|PM|second|evening|dinner| +71775|AAAAAAAAAGIBBAAA|71775|19|56|15|PM|second|evening|dinner| +71776|AAAAAAAABGIBBAAA|71776|19|56|16|PM|second|evening|dinner| +71777|AAAAAAAACGIBBAAA|71777|19|56|17|PM|second|evening|dinner| +71778|AAAAAAAADGIBBAAA|71778|19|56|18|PM|second|evening|dinner| +71779|AAAAAAAAEGIBBAAA|71779|19|56|19|PM|second|evening|dinner| +71780|AAAAAAAAFGIBBAAA|71780|19|56|20|PM|second|evening|dinner| +71781|AAAAAAAAGGIBBAAA|71781|19|56|21|PM|second|evening|dinner| +71782|AAAAAAAAHGIBBAAA|71782|19|56|22|PM|second|evening|dinner| +71783|AAAAAAAAIGIBBAAA|71783|19|56|23|PM|second|evening|dinner| +71784|AAAAAAAAJGIBBAAA|71784|19|56|24|PM|second|evening|dinner| +71785|AAAAAAAAKGIBBAAA|71785|19|56|25|PM|second|evening|dinner| +71786|AAAAAAAALGIBBAAA|71786|19|56|26|PM|second|evening|dinner| +71787|AAAAAAAAMGIBBAAA|71787|19|56|27|PM|second|evening|dinner| +71788|AAAAAAAANGIBBAAA|71788|19|56|28|PM|second|evening|dinner| +71789|AAAAAAAAOGIBBAAA|71789|19|56|29|PM|second|evening|dinner| +71790|AAAAAAAAPGIBBAAA|71790|19|56|30|PM|second|evening|dinner| +71791|AAAAAAAAAHIBBAAA|71791|19|56|31|PM|second|evening|dinner| +71792|AAAAAAAABHIBBAAA|71792|19|56|32|PM|second|evening|dinner| +71793|AAAAAAAACHIBBAAA|71793|19|56|33|PM|second|evening|dinner| +71794|AAAAAAAADHIBBAAA|71794|19|56|34|PM|second|evening|dinner| +71795|AAAAAAAAEHIBBAAA|71795|19|56|35|PM|second|evening|dinner| +71796|AAAAAAAAFHIBBAAA|71796|19|56|36|PM|second|evening|dinner| +71797|AAAAAAAAGHIBBAAA|71797|19|56|37|PM|second|evening|dinner| +71798|AAAAAAAAHHIBBAAA|71798|19|56|38|PM|second|evening|dinner| +71799|AAAAAAAAIHIBBAAA|71799|19|56|39|PM|second|evening|dinner| +71800|AAAAAAAAJHIBBAAA|71800|19|56|40|PM|second|evening|dinner| +71801|AAAAAAAAKHIBBAAA|71801|19|56|41|PM|second|evening|dinner| +71802|AAAAAAAALHIBBAAA|71802|19|56|42|PM|second|evening|dinner| +71803|AAAAAAAAMHIBBAAA|71803|19|56|43|PM|second|evening|dinner| +71804|AAAAAAAANHIBBAAA|71804|19|56|44|PM|second|evening|dinner| +71805|AAAAAAAAOHIBBAAA|71805|19|56|45|PM|second|evening|dinner| +71806|AAAAAAAAPHIBBAAA|71806|19|56|46|PM|second|evening|dinner| +71807|AAAAAAAAAIIBBAAA|71807|19|56|47|PM|second|evening|dinner| +71808|AAAAAAAABIIBBAAA|71808|19|56|48|PM|second|evening|dinner| +71809|AAAAAAAACIIBBAAA|71809|19|56|49|PM|second|evening|dinner| +71810|AAAAAAAADIIBBAAA|71810|19|56|50|PM|second|evening|dinner| +71811|AAAAAAAAEIIBBAAA|71811|19|56|51|PM|second|evening|dinner| +71812|AAAAAAAAFIIBBAAA|71812|19|56|52|PM|second|evening|dinner| +71813|AAAAAAAAGIIBBAAA|71813|19|56|53|PM|second|evening|dinner| +71814|AAAAAAAAHIIBBAAA|71814|19|56|54|PM|second|evening|dinner| +71815|AAAAAAAAIIIBBAAA|71815|19|56|55|PM|second|evening|dinner| +71816|AAAAAAAAJIIBBAAA|71816|19|56|56|PM|second|evening|dinner| +71817|AAAAAAAAKIIBBAAA|71817|19|56|57|PM|second|evening|dinner| +71818|AAAAAAAALIIBBAAA|71818|19|56|58|PM|second|evening|dinner| +71819|AAAAAAAAMIIBBAAA|71819|19|56|59|PM|second|evening|dinner| +71820|AAAAAAAANIIBBAAA|71820|19|57|0|PM|second|evening|dinner| +71821|AAAAAAAAOIIBBAAA|71821|19|57|1|PM|second|evening|dinner| +71822|AAAAAAAAPIIBBAAA|71822|19|57|2|PM|second|evening|dinner| +71823|AAAAAAAAAJIBBAAA|71823|19|57|3|PM|second|evening|dinner| +71824|AAAAAAAABJIBBAAA|71824|19|57|4|PM|second|evening|dinner| +71825|AAAAAAAACJIBBAAA|71825|19|57|5|PM|second|evening|dinner| +71826|AAAAAAAADJIBBAAA|71826|19|57|6|PM|second|evening|dinner| +71827|AAAAAAAAEJIBBAAA|71827|19|57|7|PM|second|evening|dinner| +71828|AAAAAAAAFJIBBAAA|71828|19|57|8|PM|second|evening|dinner| +71829|AAAAAAAAGJIBBAAA|71829|19|57|9|PM|second|evening|dinner| +71830|AAAAAAAAHJIBBAAA|71830|19|57|10|PM|second|evening|dinner| +71831|AAAAAAAAIJIBBAAA|71831|19|57|11|PM|second|evening|dinner| +71832|AAAAAAAAJJIBBAAA|71832|19|57|12|PM|second|evening|dinner| +71833|AAAAAAAAKJIBBAAA|71833|19|57|13|PM|second|evening|dinner| +71834|AAAAAAAALJIBBAAA|71834|19|57|14|PM|second|evening|dinner| +71835|AAAAAAAAMJIBBAAA|71835|19|57|15|PM|second|evening|dinner| +71836|AAAAAAAANJIBBAAA|71836|19|57|16|PM|second|evening|dinner| +71837|AAAAAAAAOJIBBAAA|71837|19|57|17|PM|second|evening|dinner| +71838|AAAAAAAAPJIBBAAA|71838|19|57|18|PM|second|evening|dinner| +71839|AAAAAAAAAKIBBAAA|71839|19|57|19|PM|second|evening|dinner| +71840|AAAAAAAABKIBBAAA|71840|19|57|20|PM|second|evening|dinner| +71841|AAAAAAAACKIBBAAA|71841|19|57|21|PM|second|evening|dinner| +71842|AAAAAAAADKIBBAAA|71842|19|57|22|PM|second|evening|dinner| +71843|AAAAAAAAEKIBBAAA|71843|19|57|23|PM|second|evening|dinner| +71844|AAAAAAAAFKIBBAAA|71844|19|57|24|PM|second|evening|dinner| +71845|AAAAAAAAGKIBBAAA|71845|19|57|25|PM|second|evening|dinner| +71846|AAAAAAAAHKIBBAAA|71846|19|57|26|PM|second|evening|dinner| +71847|AAAAAAAAIKIBBAAA|71847|19|57|27|PM|second|evening|dinner| +71848|AAAAAAAAJKIBBAAA|71848|19|57|28|PM|second|evening|dinner| +71849|AAAAAAAAKKIBBAAA|71849|19|57|29|PM|second|evening|dinner| +71850|AAAAAAAALKIBBAAA|71850|19|57|30|PM|second|evening|dinner| +71851|AAAAAAAAMKIBBAAA|71851|19|57|31|PM|second|evening|dinner| +71852|AAAAAAAANKIBBAAA|71852|19|57|32|PM|second|evening|dinner| +71853|AAAAAAAAOKIBBAAA|71853|19|57|33|PM|second|evening|dinner| +71854|AAAAAAAAPKIBBAAA|71854|19|57|34|PM|second|evening|dinner| +71855|AAAAAAAAALIBBAAA|71855|19|57|35|PM|second|evening|dinner| +71856|AAAAAAAABLIBBAAA|71856|19|57|36|PM|second|evening|dinner| +71857|AAAAAAAACLIBBAAA|71857|19|57|37|PM|second|evening|dinner| +71858|AAAAAAAADLIBBAAA|71858|19|57|38|PM|second|evening|dinner| +71859|AAAAAAAAELIBBAAA|71859|19|57|39|PM|second|evening|dinner| +71860|AAAAAAAAFLIBBAAA|71860|19|57|40|PM|second|evening|dinner| +71861|AAAAAAAAGLIBBAAA|71861|19|57|41|PM|second|evening|dinner| +71862|AAAAAAAAHLIBBAAA|71862|19|57|42|PM|second|evening|dinner| +71863|AAAAAAAAILIBBAAA|71863|19|57|43|PM|second|evening|dinner| +71864|AAAAAAAAJLIBBAAA|71864|19|57|44|PM|second|evening|dinner| +71865|AAAAAAAAKLIBBAAA|71865|19|57|45|PM|second|evening|dinner| +71866|AAAAAAAALLIBBAAA|71866|19|57|46|PM|second|evening|dinner| +71867|AAAAAAAAMLIBBAAA|71867|19|57|47|PM|second|evening|dinner| +71868|AAAAAAAANLIBBAAA|71868|19|57|48|PM|second|evening|dinner| +71869|AAAAAAAAOLIBBAAA|71869|19|57|49|PM|second|evening|dinner| +71870|AAAAAAAAPLIBBAAA|71870|19|57|50|PM|second|evening|dinner| +71871|AAAAAAAAAMIBBAAA|71871|19|57|51|PM|second|evening|dinner| +71872|AAAAAAAABMIBBAAA|71872|19|57|52|PM|second|evening|dinner| +71873|AAAAAAAACMIBBAAA|71873|19|57|53|PM|second|evening|dinner| +71874|AAAAAAAADMIBBAAA|71874|19|57|54|PM|second|evening|dinner| +71875|AAAAAAAAEMIBBAAA|71875|19|57|55|PM|second|evening|dinner| +71876|AAAAAAAAFMIBBAAA|71876|19|57|56|PM|second|evening|dinner| +71877|AAAAAAAAGMIBBAAA|71877|19|57|57|PM|second|evening|dinner| +71878|AAAAAAAAHMIBBAAA|71878|19|57|58|PM|second|evening|dinner| +71879|AAAAAAAAIMIBBAAA|71879|19|57|59|PM|second|evening|dinner| +71880|AAAAAAAAJMIBBAAA|71880|19|58|0|PM|second|evening|dinner| +71881|AAAAAAAAKMIBBAAA|71881|19|58|1|PM|second|evening|dinner| +71882|AAAAAAAALMIBBAAA|71882|19|58|2|PM|second|evening|dinner| +71883|AAAAAAAAMMIBBAAA|71883|19|58|3|PM|second|evening|dinner| +71884|AAAAAAAANMIBBAAA|71884|19|58|4|PM|second|evening|dinner| +71885|AAAAAAAAOMIBBAAA|71885|19|58|5|PM|second|evening|dinner| +71886|AAAAAAAAPMIBBAAA|71886|19|58|6|PM|second|evening|dinner| +71887|AAAAAAAAANIBBAAA|71887|19|58|7|PM|second|evening|dinner| +71888|AAAAAAAABNIBBAAA|71888|19|58|8|PM|second|evening|dinner| +71889|AAAAAAAACNIBBAAA|71889|19|58|9|PM|second|evening|dinner| +71890|AAAAAAAADNIBBAAA|71890|19|58|10|PM|second|evening|dinner| +71891|AAAAAAAAENIBBAAA|71891|19|58|11|PM|second|evening|dinner| +71892|AAAAAAAAFNIBBAAA|71892|19|58|12|PM|second|evening|dinner| +71893|AAAAAAAAGNIBBAAA|71893|19|58|13|PM|second|evening|dinner| +71894|AAAAAAAAHNIBBAAA|71894|19|58|14|PM|second|evening|dinner| +71895|AAAAAAAAINIBBAAA|71895|19|58|15|PM|second|evening|dinner| +71896|AAAAAAAAJNIBBAAA|71896|19|58|16|PM|second|evening|dinner| +71897|AAAAAAAAKNIBBAAA|71897|19|58|17|PM|second|evening|dinner| +71898|AAAAAAAALNIBBAAA|71898|19|58|18|PM|second|evening|dinner| +71899|AAAAAAAAMNIBBAAA|71899|19|58|19|PM|second|evening|dinner| +71900|AAAAAAAANNIBBAAA|71900|19|58|20|PM|second|evening|dinner| +71901|AAAAAAAAONIBBAAA|71901|19|58|21|PM|second|evening|dinner| +71902|AAAAAAAAPNIBBAAA|71902|19|58|22|PM|second|evening|dinner| +71903|AAAAAAAAAOIBBAAA|71903|19|58|23|PM|second|evening|dinner| +71904|AAAAAAAABOIBBAAA|71904|19|58|24|PM|second|evening|dinner| +71905|AAAAAAAACOIBBAAA|71905|19|58|25|PM|second|evening|dinner| +71906|AAAAAAAADOIBBAAA|71906|19|58|26|PM|second|evening|dinner| +71907|AAAAAAAAEOIBBAAA|71907|19|58|27|PM|second|evening|dinner| +71908|AAAAAAAAFOIBBAAA|71908|19|58|28|PM|second|evening|dinner| +71909|AAAAAAAAGOIBBAAA|71909|19|58|29|PM|second|evening|dinner| +71910|AAAAAAAAHOIBBAAA|71910|19|58|30|PM|second|evening|dinner| +71911|AAAAAAAAIOIBBAAA|71911|19|58|31|PM|second|evening|dinner| +71912|AAAAAAAAJOIBBAAA|71912|19|58|32|PM|second|evening|dinner| +71913|AAAAAAAAKOIBBAAA|71913|19|58|33|PM|second|evening|dinner| +71914|AAAAAAAALOIBBAAA|71914|19|58|34|PM|second|evening|dinner| +71915|AAAAAAAAMOIBBAAA|71915|19|58|35|PM|second|evening|dinner| +71916|AAAAAAAANOIBBAAA|71916|19|58|36|PM|second|evening|dinner| +71917|AAAAAAAAOOIBBAAA|71917|19|58|37|PM|second|evening|dinner| +71918|AAAAAAAAPOIBBAAA|71918|19|58|38|PM|second|evening|dinner| +71919|AAAAAAAAAPIBBAAA|71919|19|58|39|PM|second|evening|dinner| +71920|AAAAAAAABPIBBAAA|71920|19|58|40|PM|second|evening|dinner| +71921|AAAAAAAACPIBBAAA|71921|19|58|41|PM|second|evening|dinner| +71922|AAAAAAAADPIBBAAA|71922|19|58|42|PM|second|evening|dinner| +71923|AAAAAAAAEPIBBAAA|71923|19|58|43|PM|second|evening|dinner| +71924|AAAAAAAAFPIBBAAA|71924|19|58|44|PM|second|evening|dinner| +71925|AAAAAAAAGPIBBAAA|71925|19|58|45|PM|second|evening|dinner| +71926|AAAAAAAAHPIBBAAA|71926|19|58|46|PM|second|evening|dinner| +71927|AAAAAAAAIPIBBAAA|71927|19|58|47|PM|second|evening|dinner| +71928|AAAAAAAAJPIBBAAA|71928|19|58|48|PM|second|evening|dinner| +71929|AAAAAAAAKPIBBAAA|71929|19|58|49|PM|second|evening|dinner| +71930|AAAAAAAALPIBBAAA|71930|19|58|50|PM|second|evening|dinner| +71931|AAAAAAAAMPIBBAAA|71931|19|58|51|PM|second|evening|dinner| +71932|AAAAAAAANPIBBAAA|71932|19|58|52|PM|second|evening|dinner| +71933|AAAAAAAAOPIBBAAA|71933|19|58|53|PM|second|evening|dinner| +71934|AAAAAAAAPPIBBAAA|71934|19|58|54|PM|second|evening|dinner| +71935|AAAAAAAAAAJBBAAA|71935|19|58|55|PM|second|evening|dinner| +71936|AAAAAAAABAJBBAAA|71936|19|58|56|PM|second|evening|dinner| +71937|AAAAAAAACAJBBAAA|71937|19|58|57|PM|second|evening|dinner| +71938|AAAAAAAADAJBBAAA|71938|19|58|58|PM|second|evening|dinner| +71939|AAAAAAAAEAJBBAAA|71939|19|58|59|PM|second|evening|dinner| +71940|AAAAAAAAFAJBBAAA|71940|19|59|0|PM|second|evening|dinner| +71941|AAAAAAAAGAJBBAAA|71941|19|59|1|PM|second|evening|dinner| +71942|AAAAAAAAHAJBBAAA|71942|19|59|2|PM|second|evening|dinner| +71943|AAAAAAAAIAJBBAAA|71943|19|59|3|PM|second|evening|dinner| +71944|AAAAAAAAJAJBBAAA|71944|19|59|4|PM|second|evening|dinner| +71945|AAAAAAAAKAJBBAAA|71945|19|59|5|PM|second|evening|dinner| +71946|AAAAAAAALAJBBAAA|71946|19|59|6|PM|second|evening|dinner| +71947|AAAAAAAAMAJBBAAA|71947|19|59|7|PM|second|evening|dinner| +71948|AAAAAAAANAJBBAAA|71948|19|59|8|PM|second|evening|dinner| +71949|AAAAAAAAOAJBBAAA|71949|19|59|9|PM|second|evening|dinner| +71950|AAAAAAAAPAJBBAAA|71950|19|59|10|PM|second|evening|dinner| +71951|AAAAAAAAABJBBAAA|71951|19|59|11|PM|second|evening|dinner| +71952|AAAAAAAABBJBBAAA|71952|19|59|12|PM|second|evening|dinner| +71953|AAAAAAAACBJBBAAA|71953|19|59|13|PM|second|evening|dinner| +71954|AAAAAAAADBJBBAAA|71954|19|59|14|PM|second|evening|dinner| +71955|AAAAAAAAEBJBBAAA|71955|19|59|15|PM|second|evening|dinner| +71956|AAAAAAAAFBJBBAAA|71956|19|59|16|PM|second|evening|dinner| +71957|AAAAAAAAGBJBBAAA|71957|19|59|17|PM|second|evening|dinner| +71958|AAAAAAAAHBJBBAAA|71958|19|59|18|PM|second|evening|dinner| +71959|AAAAAAAAIBJBBAAA|71959|19|59|19|PM|second|evening|dinner| +71960|AAAAAAAAJBJBBAAA|71960|19|59|20|PM|second|evening|dinner| +71961|AAAAAAAAKBJBBAAA|71961|19|59|21|PM|second|evening|dinner| +71962|AAAAAAAALBJBBAAA|71962|19|59|22|PM|second|evening|dinner| +71963|AAAAAAAAMBJBBAAA|71963|19|59|23|PM|second|evening|dinner| +71964|AAAAAAAANBJBBAAA|71964|19|59|24|PM|second|evening|dinner| +71965|AAAAAAAAOBJBBAAA|71965|19|59|25|PM|second|evening|dinner| +71966|AAAAAAAAPBJBBAAA|71966|19|59|26|PM|second|evening|dinner| +71967|AAAAAAAAACJBBAAA|71967|19|59|27|PM|second|evening|dinner| +71968|AAAAAAAABCJBBAAA|71968|19|59|28|PM|second|evening|dinner| +71969|AAAAAAAACCJBBAAA|71969|19|59|29|PM|second|evening|dinner| +71970|AAAAAAAADCJBBAAA|71970|19|59|30|PM|second|evening|dinner| +71971|AAAAAAAAECJBBAAA|71971|19|59|31|PM|second|evening|dinner| +71972|AAAAAAAAFCJBBAAA|71972|19|59|32|PM|second|evening|dinner| +71973|AAAAAAAAGCJBBAAA|71973|19|59|33|PM|second|evening|dinner| +71974|AAAAAAAAHCJBBAAA|71974|19|59|34|PM|second|evening|dinner| +71975|AAAAAAAAICJBBAAA|71975|19|59|35|PM|second|evening|dinner| +71976|AAAAAAAAJCJBBAAA|71976|19|59|36|PM|second|evening|dinner| +71977|AAAAAAAAKCJBBAAA|71977|19|59|37|PM|second|evening|dinner| +71978|AAAAAAAALCJBBAAA|71978|19|59|38|PM|second|evening|dinner| +71979|AAAAAAAAMCJBBAAA|71979|19|59|39|PM|second|evening|dinner| +71980|AAAAAAAANCJBBAAA|71980|19|59|40|PM|second|evening|dinner| +71981|AAAAAAAAOCJBBAAA|71981|19|59|41|PM|second|evening|dinner| +71982|AAAAAAAAPCJBBAAA|71982|19|59|42|PM|second|evening|dinner| +71983|AAAAAAAAADJBBAAA|71983|19|59|43|PM|second|evening|dinner| +71984|AAAAAAAABDJBBAAA|71984|19|59|44|PM|second|evening|dinner| +71985|AAAAAAAACDJBBAAA|71985|19|59|45|PM|second|evening|dinner| +71986|AAAAAAAADDJBBAAA|71986|19|59|46|PM|second|evening|dinner| +71987|AAAAAAAAEDJBBAAA|71987|19|59|47|PM|second|evening|dinner| +71988|AAAAAAAAFDJBBAAA|71988|19|59|48|PM|second|evening|dinner| +71989|AAAAAAAAGDJBBAAA|71989|19|59|49|PM|second|evening|dinner| +71990|AAAAAAAAHDJBBAAA|71990|19|59|50|PM|second|evening|dinner| +71991|AAAAAAAAIDJBBAAA|71991|19|59|51|PM|second|evening|dinner| +71992|AAAAAAAAJDJBBAAA|71992|19|59|52|PM|second|evening|dinner| +71993|AAAAAAAAKDJBBAAA|71993|19|59|53|PM|second|evening|dinner| +71994|AAAAAAAALDJBBAAA|71994|19|59|54|PM|second|evening|dinner| +71995|AAAAAAAAMDJBBAAA|71995|19|59|55|PM|second|evening|dinner| +71996|AAAAAAAANDJBBAAA|71996|19|59|56|PM|second|evening|dinner| +71997|AAAAAAAAODJBBAAA|71997|19|59|57|PM|second|evening|dinner| +71998|AAAAAAAAPDJBBAAA|71998|19|59|58|PM|second|evening|dinner| +71999|AAAAAAAAAEJBBAAA|71999|19|59|59|PM|second|evening|dinner| +72000|AAAAAAAABEJBBAAA|72000|20|0|0|PM|second|evening|| +72001|AAAAAAAACEJBBAAA|72001|20|0|1|PM|second|evening|| +72002|AAAAAAAADEJBBAAA|72002|20|0|2|PM|second|evening|| +72003|AAAAAAAAEEJBBAAA|72003|20|0|3|PM|second|evening|| +72004|AAAAAAAAFEJBBAAA|72004|20|0|4|PM|second|evening|| +72005|AAAAAAAAGEJBBAAA|72005|20|0|5|PM|second|evening|| +72006|AAAAAAAAHEJBBAAA|72006|20|0|6|PM|second|evening|| +72007|AAAAAAAAIEJBBAAA|72007|20|0|7|PM|second|evening|| +72008|AAAAAAAAJEJBBAAA|72008|20|0|8|PM|second|evening|| +72009|AAAAAAAAKEJBBAAA|72009|20|0|9|PM|second|evening|| +72010|AAAAAAAALEJBBAAA|72010|20|0|10|PM|second|evening|| +72011|AAAAAAAAMEJBBAAA|72011|20|0|11|PM|second|evening|| +72012|AAAAAAAANEJBBAAA|72012|20|0|12|PM|second|evening|| +72013|AAAAAAAAOEJBBAAA|72013|20|0|13|PM|second|evening|| +72014|AAAAAAAAPEJBBAAA|72014|20|0|14|PM|second|evening|| +72015|AAAAAAAAAFJBBAAA|72015|20|0|15|PM|second|evening|| +72016|AAAAAAAABFJBBAAA|72016|20|0|16|PM|second|evening|| +72017|AAAAAAAACFJBBAAA|72017|20|0|17|PM|second|evening|| +72018|AAAAAAAADFJBBAAA|72018|20|0|18|PM|second|evening|| +72019|AAAAAAAAEFJBBAAA|72019|20|0|19|PM|second|evening|| +72020|AAAAAAAAFFJBBAAA|72020|20|0|20|PM|second|evening|| +72021|AAAAAAAAGFJBBAAA|72021|20|0|21|PM|second|evening|| +72022|AAAAAAAAHFJBBAAA|72022|20|0|22|PM|second|evening|| +72023|AAAAAAAAIFJBBAAA|72023|20|0|23|PM|second|evening|| +72024|AAAAAAAAJFJBBAAA|72024|20|0|24|PM|second|evening|| +72025|AAAAAAAAKFJBBAAA|72025|20|0|25|PM|second|evening|| +72026|AAAAAAAALFJBBAAA|72026|20|0|26|PM|second|evening|| +72027|AAAAAAAAMFJBBAAA|72027|20|0|27|PM|second|evening|| +72028|AAAAAAAANFJBBAAA|72028|20|0|28|PM|second|evening|| +72029|AAAAAAAAOFJBBAAA|72029|20|0|29|PM|second|evening|| +72030|AAAAAAAAPFJBBAAA|72030|20|0|30|PM|second|evening|| +72031|AAAAAAAAAGJBBAAA|72031|20|0|31|PM|second|evening|| +72032|AAAAAAAABGJBBAAA|72032|20|0|32|PM|second|evening|| +72033|AAAAAAAACGJBBAAA|72033|20|0|33|PM|second|evening|| +72034|AAAAAAAADGJBBAAA|72034|20|0|34|PM|second|evening|| +72035|AAAAAAAAEGJBBAAA|72035|20|0|35|PM|second|evening|| +72036|AAAAAAAAFGJBBAAA|72036|20|0|36|PM|second|evening|| +72037|AAAAAAAAGGJBBAAA|72037|20|0|37|PM|second|evening|| +72038|AAAAAAAAHGJBBAAA|72038|20|0|38|PM|second|evening|| +72039|AAAAAAAAIGJBBAAA|72039|20|0|39|PM|second|evening|| +72040|AAAAAAAAJGJBBAAA|72040|20|0|40|PM|second|evening|| +72041|AAAAAAAAKGJBBAAA|72041|20|0|41|PM|second|evening|| +72042|AAAAAAAALGJBBAAA|72042|20|0|42|PM|second|evening|| +72043|AAAAAAAAMGJBBAAA|72043|20|0|43|PM|second|evening|| +72044|AAAAAAAANGJBBAAA|72044|20|0|44|PM|second|evening|| +72045|AAAAAAAAOGJBBAAA|72045|20|0|45|PM|second|evening|| +72046|AAAAAAAAPGJBBAAA|72046|20|0|46|PM|second|evening|| +72047|AAAAAAAAAHJBBAAA|72047|20|0|47|PM|second|evening|| +72048|AAAAAAAABHJBBAAA|72048|20|0|48|PM|second|evening|| +72049|AAAAAAAACHJBBAAA|72049|20|0|49|PM|second|evening|| +72050|AAAAAAAADHJBBAAA|72050|20|0|50|PM|second|evening|| +72051|AAAAAAAAEHJBBAAA|72051|20|0|51|PM|second|evening|| +72052|AAAAAAAAFHJBBAAA|72052|20|0|52|PM|second|evening|| +72053|AAAAAAAAGHJBBAAA|72053|20|0|53|PM|second|evening|| +72054|AAAAAAAAHHJBBAAA|72054|20|0|54|PM|second|evening|| +72055|AAAAAAAAIHJBBAAA|72055|20|0|55|PM|second|evening|| +72056|AAAAAAAAJHJBBAAA|72056|20|0|56|PM|second|evening|| +72057|AAAAAAAAKHJBBAAA|72057|20|0|57|PM|second|evening|| +72058|AAAAAAAALHJBBAAA|72058|20|0|58|PM|second|evening|| +72059|AAAAAAAAMHJBBAAA|72059|20|0|59|PM|second|evening|| +72060|AAAAAAAANHJBBAAA|72060|20|1|0|PM|second|evening|| +72061|AAAAAAAAOHJBBAAA|72061|20|1|1|PM|second|evening|| +72062|AAAAAAAAPHJBBAAA|72062|20|1|2|PM|second|evening|| +72063|AAAAAAAAAIJBBAAA|72063|20|1|3|PM|second|evening|| +72064|AAAAAAAABIJBBAAA|72064|20|1|4|PM|second|evening|| +72065|AAAAAAAACIJBBAAA|72065|20|1|5|PM|second|evening|| +72066|AAAAAAAADIJBBAAA|72066|20|1|6|PM|second|evening|| +72067|AAAAAAAAEIJBBAAA|72067|20|1|7|PM|second|evening|| +72068|AAAAAAAAFIJBBAAA|72068|20|1|8|PM|second|evening|| +72069|AAAAAAAAGIJBBAAA|72069|20|1|9|PM|second|evening|| +72070|AAAAAAAAHIJBBAAA|72070|20|1|10|PM|second|evening|| +72071|AAAAAAAAIIJBBAAA|72071|20|1|11|PM|second|evening|| +72072|AAAAAAAAJIJBBAAA|72072|20|1|12|PM|second|evening|| +72073|AAAAAAAAKIJBBAAA|72073|20|1|13|PM|second|evening|| +72074|AAAAAAAALIJBBAAA|72074|20|1|14|PM|second|evening|| +72075|AAAAAAAAMIJBBAAA|72075|20|1|15|PM|second|evening|| +72076|AAAAAAAANIJBBAAA|72076|20|1|16|PM|second|evening|| +72077|AAAAAAAAOIJBBAAA|72077|20|1|17|PM|second|evening|| +72078|AAAAAAAAPIJBBAAA|72078|20|1|18|PM|second|evening|| +72079|AAAAAAAAAJJBBAAA|72079|20|1|19|PM|second|evening|| +72080|AAAAAAAABJJBBAAA|72080|20|1|20|PM|second|evening|| +72081|AAAAAAAACJJBBAAA|72081|20|1|21|PM|second|evening|| +72082|AAAAAAAADJJBBAAA|72082|20|1|22|PM|second|evening|| +72083|AAAAAAAAEJJBBAAA|72083|20|1|23|PM|second|evening|| +72084|AAAAAAAAFJJBBAAA|72084|20|1|24|PM|second|evening|| +72085|AAAAAAAAGJJBBAAA|72085|20|1|25|PM|second|evening|| +72086|AAAAAAAAHJJBBAAA|72086|20|1|26|PM|second|evening|| +72087|AAAAAAAAIJJBBAAA|72087|20|1|27|PM|second|evening|| +72088|AAAAAAAAJJJBBAAA|72088|20|1|28|PM|second|evening|| +72089|AAAAAAAAKJJBBAAA|72089|20|1|29|PM|second|evening|| +72090|AAAAAAAALJJBBAAA|72090|20|1|30|PM|second|evening|| +72091|AAAAAAAAMJJBBAAA|72091|20|1|31|PM|second|evening|| +72092|AAAAAAAANJJBBAAA|72092|20|1|32|PM|second|evening|| +72093|AAAAAAAAOJJBBAAA|72093|20|1|33|PM|second|evening|| +72094|AAAAAAAAPJJBBAAA|72094|20|1|34|PM|second|evening|| +72095|AAAAAAAAAKJBBAAA|72095|20|1|35|PM|second|evening|| +72096|AAAAAAAABKJBBAAA|72096|20|1|36|PM|second|evening|| +72097|AAAAAAAACKJBBAAA|72097|20|1|37|PM|second|evening|| +72098|AAAAAAAADKJBBAAA|72098|20|1|38|PM|second|evening|| +72099|AAAAAAAAEKJBBAAA|72099|20|1|39|PM|second|evening|| +72100|AAAAAAAAFKJBBAAA|72100|20|1|40|PM|second|evening|| +72101|AAAAAAAAGKJBBAAA|72101|20|1|41|PM|second|evening|| +72102|AAAAAAAAHKJBBAAA|72102|20|1|42|PM|second|evening|| +72103|AAAAAAAAIKJBBAAA|72103|20|1|43|PM|second|evening|| +72104|AAAAAAAAJKJBBAAA|72104|20|1|44|PM|second|evening|| +72105|AAAAAAAAKKJBBAAA|72105|20|1|45|PM|second|evening|| +72106|AAAAAAAALKJBBAAA|72106|20|1|46|PM|second|evening|| +72107|AAAAAAAAMKJBBAAA|72107|20|1|47|PM|second|evening|| +72108|AAAAAAAANKJBBAAA|72108|20|1|48|PM|second|evening|| +72109|AAAAAAAAOKJBBAAA|72109|20|1|49|PM|second|evening|| +72110|AAAAAAAAPKJBBAAA|72110|20|1|50|PM|second|evening|| +72111|AAAAAAAAALJBBAAA|72111|20|1|51|PM|second|evening|| +72112|AAAAAAAABLJBBAAA|72112|20|1|52|PM|second|evening|| +72113|AAAAAAAACLJBBAAA|72113|20|1|53|PM|second|evening|| +72114|AAAAAAAADLJBBAAA|72114|20|1|54|PM|second|evening|| +72115|AAAAAAAAELJBBAAA|72115|20|1|55|PM|second|evening|| +72116|AAAAAAAAFLJBBAAA|72116|20|1|56|PM|second|evening|| +72117|AAAAAAAAGLJBBAAA|72117|20|1|57|PM|second|evening|| +72118|AAAAAAAAHLJBBAAA|72118|20|1|58|PM|second|evening|| +72119|AAAAAAAAILJBBAAA|72119|20|1|59|PM|second|evening|| +72120|AAAAAAAAJLJBBAAA|72120|20|2|0|PM|second|evening|| +72121|AAAAAAAAKLJBBAAA|72121|20|2|1|PM|second|evening|| +72122|AAAAAAAALLJBBAAA|72122|20|2|2|PM|second|evening|| +72123|AAAAAAAAMLJBBAAA|72123|20|2|3|PM|second|evening|| +72124|AAAAAAAANLJBBAAA|72124|20|2|4|PM|second|evening|| +72125|AAAAAAAAOLJBBAAA|72125|20|2|5|PM|second|evening|| +72126|AAAAAAAAPLJBBAAA|72126|20|2|6|PM|second|evening|| +72127|AAAAAAAAAMJBBAAA|72127|20|2|7|PM|second|evening|| +72128|AAAAAAAABMJBBAAA|72128|20|2|8|PM|second|evening|| +72129|AAAAAAAACMJBBAAA|72129|20|2|9|PM|second|evening|| +72130|AAAAAAAADMJBBAAA|72130|20|2|10|PM|second|evening|| +72131|AAAAAAAAEMJBBAAA|72131|20|2|11|PM|second|evening|| +72132|AAAAAAAAFMJBBAAA|72132|20|2|12|PM|second|evening|| +72133|AAAAAAAAGMJBBAAA|72133|20|2|13|PM|second|evening|| +72134|AAAAAAAAHMJBBAAA|72134|20|2|14|PM|second|evening|| +72135|AAAAAAAAIMJBBAAA|72135|20|2|15|PM|second|evening|| +72136|AAAAAAAAJMJBBAAA|72136|20|2|16|PM|second|evening|| +72137|AAAAAAAAKMJBBAAA|72137|20|2|17|PM|second|evening|| +72138|AAAAAAAALMJBBAAA|72138|20|2|18|PM|second|evening|| +72139|AAAAAAAAMMJBBAAA|72139|20|2|19|PM|second|evening|| +72140|AAAAAAAANMJBBAAA|72140|20|2|20|PM|second|evening|| +72141|AAAAAAAAOMJBBAAA|72141|20|2|21|PM|second|evening|| +72142|AAAAAAAAPMJBBAAA|72142|20|2|22|PM|second|evening|| +72143|AAAAAAAAANJBBAAA|72143|20|2|23|PM|second|evening|| +72144|AAAAAAAABNJBBAAA|72144|20|2|24|PM|second|evening|| +72145|AAAAAAAACNJBBAAA|72145|20|2|25|PM|second|evening|| +72146|AAAAAAAADNJBBAAA|72146|20|2|26|PM|second|evening|| +72147|AAAAAAAAENJBBAAA|72147|20|2|27|PM|second|evening|| +72148|AAAAAAAAFNJBBAAA|72148|20|2|28|PM|second|evening|| +72149|AAAAAAAAGNJBBAAA|72149|20|2|29|PM|second|evening|| +72150|AAAAAAAAHNJBBAAA|72150|20|2|30|PM|second|evening|| +72151|AAAAAAAAINJBBAAA|72151|20|2|31|PM|second|evening|| +72152|AAAAAAAAJNJBBAAA|72152|20|2|32|PM|second|evening|| +72153|AAAAAAAAKNJBBAAA|72153|20|2|33|PM|second|evening|| +72154|AAAAAAAALNJBBAAA|72154|20|2|34|PM|second|evening|| +72155|AAAAAAAAMNJBBAAA|72155|20|2|35|PM|second|evening|| +72156|AAAAAAAANNJBBAAA|72156|20|2|36|PM|second|evening|| +72157|AAAAAAAAONJBBAAA|72157|20|2|37|PM|second|evening|| +72158|AAAAAAAAPNJBBAAA|72158|20|2|38|PM|second|evening|| +72159|AAAAAAAAAOJBBAAA|72159|20|2|39|PM|second|evening|| +72160|AAAAAAAABOJBBAAA|72160|20|2|40|PM|second|evening|| +72161|AAAAAAAACOJBBAAA|72161|20|2|41|PM|second|evening|| +72162|AAAAAAAADOJBBAAA|72162|20|2|42|PM|second|evening|| +72163|AAAAAAAAEOJBBAAA|72163|20|2|43|PM|second|evening|| +72164|AAAAAAAAFOJBBAAA|72164|20|2|44|PM|second|evening|| +72165|AAAAAAAAGOJBBAAA|72165|20|2|45|PM|second|evening|| +72166|AAAAAAAAHOJBBAAA|72166|20|2|46|PM|second|evening|| +72167|AAAAAAAAIOJBBAAA|72167|20|2|47|PM|second|evening|| +72168|AAAAAAAAJOJBBAAA|72168|20|2|48|PM|second|evening|| +72169|AAAAAAAAKOJBBAAA|72169|20|2|49|PM|second|evening|| +72170|AAAAAAAALOJBBAAA|72170|20|2|50|PM|second|evening|| +72171|AAAAAAAAMOJBBAAA|72171|20|2|51|PM|second|evening|| +72172|AAAAAAAANOJBBAAA|72172|20|2|52|PM|second|evening|| +72173|AAAAAAAAOOJBBAAA|72173|20|2|53|PM|second|evening|| +72174|AAAAAAAAPOJBBAAA|72174|20|2|54|PM|second|evening|| +72175|AAAAAAAAAPJBBAAA|72175|20|2|55|PM|second|evening|| +72176|AAAAAAAABPJBBAAA|72176|20|2|56|PM|second|evening|| +72177|AAAAAAAACPJBBAAA|72177|20|2|57|PM|second|evening|| +72178|AAAAAAAADPJBBAAA|72178|20|2|58|PM|second|evening|| +72179|AAAAAAAAEPJBBAAA|72179|20|2|59|PM|second|evening|| +72180|AAAAAAAAFPJBBAAA|72180|20|3|0|PM|second|evening|| +72181|AAAAAAAAGPJBBAAA|72181|20|3|1|PM|second|evening|| +72182|AAAAAAAAHPJBBAAA|72182|20|3|2|PM|second|evening|| +72183|AAAAAAAAIPJBBAAA|72183|20|3|3|PM|second|evening|| +72184|AAAAAAAAJPJBBAAA|72184|20|3|4|PM|second|evening|| +72185|AAAAAAAAKPJBBAAA|72185|20|3|5|PM|second|evening|| +72186|AAAAAAAALPJBBAAA|72186|20|3|6|PM|second|evening|| +72187|AAAAAAAAMPJBBAAA|72187|20|3|7|PM|second|evening|| +72188|AAAAAAAANPJBBAAA|72188|20|3|8|PM|second|evening|| +72189|AAAAAAAAOPJBBAAA|72189|20|3|9|PM|second|evening|| +72190|AAAAAAAAPPJBBAAA|72190|20|3|10|PM|second|evening|| +72191|AAAAAAAAAAKBBAAA|72191|20|3|11|PM|second|evening|| +72192|AAAAAAAABAKBBAAA|72192|20|3|12|PM|second|evening|| +72193|AAAAAAAACAKBBAAA|72193|20|3|13|PM|second|evening|| +72194|AAAAAAAADAKBBAAA|72194|20|3|14|PM|second|evening|| +72195|AAAAAAAAEAKBBAAA|72195|20|3|15|PM|second|evening|| +72196|AAAAAAAAFAKBBAAA|72196|20|3|16|PM|second|evening|| +72197|AAAAAAAAGAKBBAAA|72197|20|3|17|PM|second|evening|| +72198|AAAAAAAAHAKBBAAA|72198|20|3|18|PM|second|evening|| +72199|AAAAAAAAIAKBBAAA|72199|20|3|19|PM|second|evening|| +72200|AAAAAAAAJAKBBAAA|72200|20|3|20|PM|second|evening|| +72201|AAAAAAAAKAKBBAAA|72201|20|3|21|PM|second|evening|| +72202|AAAAAAAALAKBBAAA|72202|20|3|22|PM|second|evening|| +72203|AAAAAAAAMAKBBAAA|72203|20|3|23|PM|second|evening|| +72204|AAAAAAAANAKBBAAA|72204|20|3|24|PM|second|evening|| +72205|AAAAAAAAOAKBBAAA|72205|20|3|25|PM|second|evening|| +72206|AAAAAAAAPAKBBAAA|72206|20|3|26|PM|second|evening|| +72207|AAAAAAAAABKBBAAA|72207|20|3|27|PM|second|evening|| +72208|AAAAAAAABBKBBAAA|72208|20|3|28|PM|second|evening|| +72209|AAAAAAAACBKBBAAA|72209|20|3|29|PM|second|evening|| +72210|AAAAAAAADBKBBAAA|72210|20|3|30|PM|second|evening|| +72211|AAAAAAAAEBKBBAAA|72211|20|3|31|PM|second|evening|| +72212|AAAAAAAAFBKBBAAA|72212|20|3|32|PM|second|evening|| +72213|AAAAAAAAGBKBBAAA|72213|20|3|33|PM|second|evening|| +72214|AAAAAAAAHBKBBAAA|72214|20|3|34|PM|second|evening|| +72215|AAAAAAAAIBKBBAAA|72215|20|3|35|PM|second|evening|| +72216|AAAAAAAAJBKBBAAA|72216|20|3|36|PM|second|evening|| +72217|AAAAAAAAKBKBBAAA|72217|20|3|37|PM|second|evening|| +72218|AAAAAAAALBKBBAAA|72218|20|3|38|PM|second|evening|| +72219|AAAAAAAAMBKBBAAA|72219|20|3|39|PM|second|evening|| +72220|AAAAAAAANBKBBAAA|72220|20|3|40|PM|second|evening|| +72221|AAAAAAAAOBKBBAAA|72221|20|3|41|PM|second|evening|| +72222|AAAAAAAAPBKBBAAA|72222|20|3|42|PM|second|evening|| +72223|AAAAAAAAACKBBAAA|72223|20|3|43|PM|second|evening|| +72224|AAAAAAAABCKBBAAA|72224|20|3|44|PM|second|evening|| +72225|AAAAAAAACCKBBAAA|72225|20|3|45|PM|second|evening|| +72226|AAAAAAAADCKBBAAA|72226|20|3|46|PM|second|evening|| +72227|AAAAAAAAECKBBAAA|72227|20|3|47|PM|second|evening|| +72228|AAAAAAAAFCKBBAAA|72228|20|3|48|PM|second|evening|| +72229|AAAAAAAAGCKBBAAA|72229|20|3|49|PM|second|evening|| +72230|AAAAAAAAHCKBBAAA|72230|20|3|50|PM|second|evening|| +72231|AAAAAAAAICKBBAAA|72231|20|3|51|PM|second|evening|| +72232|AAAAAAAAJCKBBAAA|72232|20|3|52|PM|second|evening|| +72233|AAAAAAAAKCKBBAAA|72233|20|3|53|PM|second|evening|| +72234|AAAAAAAALCKBBAAA|72234|20|3|54|PM|second|evening|| +72235|AAAAAAAAMCKBBAAA|72235|20|3|55|PM|second|evening|| +72236|AAAAAAAANCKBBAAA|72236|20|3|56|PM|second|evening|| +72237|AAAAAAAAOCKBBAAA|72237|20|3|57|PM|second|evening|| +72238|AAAAAAAAPCKBBAAA|72238|20|3|58|PM|second|evening|| +72239|AAAAAAAAADKBBAAA|72239|20|3|59|PM|second|evening|| +72240|AAAAAAAABDKBBAAA|72240|20|4|0|PM|second|evening|| +72241|AAAAAAAACDKBBAAA|72241|20|4|1|PM|second|evening|| +72242|AAAAAAAADDKBBAAA|72242|20|4|2|PM|second|evening|| +72243|AAAAAAAAEDKBBAAA|72243|20|4|3|PM|second|evening|| +72244|AAAAAAAAFDKBBAAA|72244|20|4|4|PM|second|evening|| +72245|AAAAAAAAGDKBBAAA|72245|20|4|5|PM|second|evening|| +72246|AAAAAAAAHDKBBAAA|72246|20|4|6|PM|second|evening|| +72247|AAAAAAAAIDKBBAAA|72247|20|4|7|PM|second|evening|| +72248|AAAAAAAAJDKBBAAA|72248|20|4|8|PM|second|evening|| +72249|AAAAAAAAKDKBBAAA|72249|20|4|9|PM|second|evening|| +72250|AAAAAAAALDKBBAAA|72250|20|4|10|PM|second|evening|| +72251|AAAAAAAAMDKBBAAA|72251|20|4|11|PM|second|evening|| +72252|AAAAAAAANDKBBAAA|72252|20|4|12|PM|second|evening|| +72253|AAAAAAAAODKBBAAA|72253|20|4|13|PM|second|evening|| +72254|AAAAAAAAPDKBBAAA|72254|20|4|14|PM|second|evening|| +72255|AAAAAAAAAEKBBAAA|72255|20|4|15|PM|second|evening|| +72256|AAAAAAAABEKBBAAA|72256|20|4|16|PM|second|evening|| +72257|AAAAAAAACEKBBAAA|72257|20|4|17|PM|second|evening|| +72258|AAAAAAAADEKBBAAA|72258|20|4|18|PM|second|evening|| +72259|AAAAAAAAEEKBBAAA|72259|20|4|19|PM|second|evening|| +72260|AAAAAAAAFEKBBAAA|72260|20|4|20|PM|second|evening|| +72261|AAAAAAAAGEKBBAAA|72261|20|4|21|PM|second|evening|| +72262|AAAAAAAAHEKBBAAA|72262|20|4|22|PM|second|evening|| +72263|AAAAAAAAIEKBBAAA|72263|20|4|23|PM|second|evening|| +72264|AAAAAAAAJEKBBAAA|72264|20|4|24|PM|second|evening|| +72265|AAAAAAAAKEKBBAAA|72265|20|4|25|PM|second|evening|| +72266|AAAAAAAALEKBBAAA|72266|20|4|26|PM|second|evening|| +72267|AAAAAAAAMEKBBAAA|72267|20|4|27|PM|second|evening|| +72268|AAAAAAAANEKBBAAA|72268|20|4|28|PM|second|evening|| +72269|AAAAAAAAOEKBBAAA|72269|20|4|29|PM|second|evening|| +72270|AAAAAAAAPEKBBAAA|72270|20|4|30|PM|second|evening|| +72271|AAAAAAAAAFKBBAAA|72271|20|4|31|PM|second|evening|| +72272|AAAAAAAABFKBBAAA|72272|20|4|32|PM|second|evening|| +72273|AAAAAAAACFKBBAAA|72273|20|4|33|PM|second|evening|| +72274|AAAAAAAADFKBBAAA|72274|20|4|34|PM|second|evening|| +72275|AAAAAAAAEFKBBAAA|72275|20|4|35|PM|second|evening|| +72276|AAAAAAAAFFKBBAAA|72276|20|4|36|PM|second|evening|| +72277|AAAAAAAAGFKBBAAA|72277|20|4|37|PM|second|evening|| +72278|AAAAAAAAHFKBBAAA|72278|20|4|38|PM|second|evening|| +72279|AAAAAAAAIFKBBAAA|72279|20|4|39|PM|second|evening|| +72280|AAAAAAAAJFKBBAAA|72280|20|4|40|PM|second|evening|| +72281|AAAAAAAAKFKBBAAA|72281|20|4|41|PM|second|evening|| +72282|AAAAAAAALFKBBAAA|72282|20|4|42|PM|second|evening|| +72283|AAAAAAAAMFKBBAAA|72283|20|4|43|PM|second|evening|| +72284|AAAAAAAANFKBBAAA|72284|20|4|44|PM|second|evening|| +72285|AAAAAAAAOFKBBAAA|72285|20|4|45|PM|second|evening|| +72286|AAAAAAAAPFKBBAAA|72286|20|4|46|PM|second|evening|| +72287|AAAAAAAAAGKBBAAA|72287|20|4|47|PM|second|evening|| +72288|AAAAAAAABGKBBAAA|72288|20|4|48|PM|second|evening|| +72289|AAAAAAAACGKBBAAA|72289|20|4|49|PM|second|evening|| +72290|AAAAAAAADGKBBAAA|72290|20|4|50|PM|second|evening|| +72291|AAAAAAAAEGKBBAAA|72291|20|4|51|PM|second|evening|| +72292|AAAAAAAAFGKBBAAA|72292|20|4|52|PM|second|evening|| +72293|AAAAAAAAGGKBBAAA|72293|20|4|53|PM|second|evening|| +72294|AAAAAAAAHGKBBAAA|72294|20|4|54|PM|second|evening|| +72295|AAAAAAAAIGKBBAAA|72295|20|4|55|PM|second|evening|| +72296|AAAAAAAAJGKBBAAA|72296|20|4|56|PM|second|evening|| +72297|AAAAAAAAKGKBBAAA|72297|20|4|57|PM|second|evening|| +72298|AAAAAAAALGKBBAAA|72298|20|4|58|PM|second|evening|| +72299|AAAAAAAAMGKBBAAA|72299|20|4|59|PM|second|evening|| +72300|AAAAAAAANGKBBAAA|72300|20|5|0|PM|second|evening|| +72301|AAAAAAAAOGKBBAAA|72301|20|5|1|PM|second|evening|| +72302|AAAAAAAAPGKBBAAA|72302|20|5|2|PM|second|evening|| +72303|AAAAAAAAAHKBBAAA|72303|20|5|3|PM|second|evening|| +72304|AAAAAAAABHKBBAAA|72304|20|5|4|PM|second|evening|| +72305|AAAAAAAACHKBBAAA|72305|20|5|5|PM|second|evening|| +72306|AAAAAAAADHKBBAAA|72306|20|5|6|PM|second|evening|| +72307|AAAAAAAAEHKBBAAA|72307|20|5|7|PM|second|evening|| +72308|AAAAAAAAFHKBBAAA|72308|20|5|8|PM|second|evening|| +72309|AAAAAAAAGHKBBAAA|72309|20|5|9|PM|second|evening|| +72310|AAAAAAAAHHKBBAAA|72310|20|5|10|PM|second|evening|| +72311|AAAAAAAAIHKBBAAA|72311|20|5|11|PM|second|evening|| +72312|AAAAAAAAJHKBBAAA|72312|20|5|12|PM|second|evening|| +72313|AAAAAAAAKHKBBAAA|72313|20|5|13|PM|second|evening|| +72314|AAAAAAAALHKBBAAA|72314|20|5|14|PM|second|evening|| +72315|AAAAAAAAMHKBBAAA|72315|20|5|15|PM|second|evening|| +72316|AAAAAAAANHKBBAAA|72316|20|5|16|PM|second|evening|| +72317|AAAAAAAAOHKBBAAA|72317|20|5|17|PM|second|evening|| +72318|AAAAAAAAPHKBBAAA|72318|20|5|18|PM|second|evening|| +72319|AAAAAAAAAIKBBAAA|72319|20|5|19|PM|second|evening|| +72320|AAAAAAAABIKBBAAA|72320|20|5|20|PM|second|evening|| +72321|AAAAAAAACIKBBAAA|72321|20|5|21|PM|second|evening|| +72322|AAAAAAAADIKBBAAA|72322|20|5|22|PM|second|evening|| +72323|AAAAAAAAEIKBBAAA|72323|20|5|23|PM|second|evening|| +72324|AAAAAAAAFIKBBAAA|72324|20|5|24|PM|second|evening|| +72325|AAAAAAAAGIKBBAAA|72325|20|5|25|PM|second|evening|| +72326|AAAAAAAAHIKBBAAA|72326|20|5|26|PM|second|evening|| +72327|AAAAAAAAIIKBBAAA|72327|20|5|27|PM|second|evening|| +72328|AAAAAAAAJIKBBAAA|72328|20|5|28|PM|second|evening|| +72329|AAAAAAAAKIKBBAAA|72329|20|5|29|PM|second|evening|| +72330|AAAAAAAALIKBBAAA|72330|20|5|30|PM|second|evening|| +72331|AAAAAAAAMIKBBAAA|72331|20|5|31|PM|second|evening|| +72332|AAAAAAAANIKBBAAA|72332|20|5|32|PM|second|evening|| +72333|AAAAAAAAOIKBBAAA|72333|20|5|33|PM|second|evening|| +72334|AAAAAAAAPIKBBAAA|72334|20|5|34|PM|second|evening|| +72335|AAAAAAAAAJKBBAAA|72335|20|5|35|PM|second|evening|| +72336|AAAAAAAABJKBBAAA|72336|20|5|36|PM|second|evening|| +72337|AAAAAAAACJKBBAAA|72337|20|5|37|PM|second|evening|| +72338|AAAAAAAADJKBBAAA|72338|20|5|38|PM|second|evening|| +72339|AAAAAAAAEJKBBAAA|72339|20|5|39|PM|second|evening|| +72340|AAAAAAAAFJKBBAAA|72340|20|5|40|PM|second|evening|| +72341|AAAAAAAAGJKBBAAA|72341|20|5|41|PM|second|evening|| +72342|AAAAAAAAHJKBBAAA|72342|20|5|42|PM|second|evening|| +72343|AAAAAAAAIJKBBAAA|72343|20|5|43|PM|second|evening|| +72344|AAAAAAAAJJKBBAAA|72344|20|5|44|PM|second|evening|| +72345|AAAAAAAAKJKBBAAA|72345|20|5|45|PM|second|evening|| +72346|AAAAAAAALJKBBAAA|72346|20|5|46|PM|second|evening|| +72347|AAAAAAAAMJKBBAAA|72347|20|5|47|PM|second|evening|| +72348|AAAAAAAANJKBBAAA|72348|20|5|48|PM|second|evening|| +72349|AAAAAAAAOJKBBAAA|72349|20|5|49|PM|second|evening|| +72350|AAAAAAAAPJKBBAAA|72350|20|5|50|PM|second|evening|| +72351|AAAAAAAAAKKBBAAA|72351|20|5|51|PM|second|evening|| +72352|AAAAAAAABKKBBAAA|72352|20|5|52|PM|second|evening|| +72353|AAAAAAAACKKBBAAA|72353|20|5|53|PM|second|evening|| +72354|AAAAAAAADKKBBAAA|72354|20|5|54|PM|second|evening|| +72355|AAAAAAAAEKKBBAAA|72355|20|5|55|PM|second|evening|| +72356|AAAAAAAAFKKBBAAA|72356|20|5|56|PM|second|evening|| +72357|AAAAAAAAGKKBBAAA|72357|20|5|57|PM|second|evening|| +72358|AAAAAAAAHKKBBAAA|72358|20|5|58|PM|second|evening|| +72359|AAAAAAAAIKKBBAAA|72359|20|5|59|PM|second|evening|| +72360|AAAAAAAAJKKBBAAA|72360|20|6|0|PM|second|evening|| +72361|AAAAAAAAKKKBBAAA|72361|20|6|1|PM|second|evening|| +72362|AAAAAAAALKKBBAAA|72362|20|6|2|PM|second|evening|| +72363|AAAAAAAAMKKBBAAA|72363|20|6|3|PM|second|evening|| +72364|AAAAAAAANKKBBAAA|72364|20|6|4|PM|second|evening|| +72365|AAAAAAAAOKKBBAAA|72365|20|6|5|PM|second|evening|| +72366|AAAAAAAAPKKBBAAA|72366|20|6|6|PM|second|evening|| +72367|AAAAAAAAALKBBAAA|72367|20|6|7|PM|second|evening|| +72368|AAAAAAAABLKBBAAA|72368|20|6|8|PM|second|evening|| +72369|AAAAAAAACLKBBAAA|72369|20|6|9|PM|second|evening|| +72370|AAAAAAAADLKBBAAA|72370|20|6|10|PM|second|evening|| +72371|AAAAAAAAELKBBAAA|72371|20|6|11|PM|second|evening|| +72372|AAAAAAAAFLKBBAAA|72372|20|6|12|PM|second|evening|| +72373|AAAAAAAAGLKBBAAA|72373|20|6|13|PM|second|evening|| +72374|AAAAAAAAHLKBBAAA|72374|20|6|14|PM|second|evening|| +72375|AAAAAAAAILKBBAAA|72375|20|6|15|PM|second|evening|| +72376|AAAAAAAAJLKBBAAA|72376|20|6|16|PM|second|evening|| +72377|AAAAAAAAKLKBBAAA|72377|20|6|17|PM|second|evening|| +72378|AAAAAAAALLKBBAAA|72378|20|6|18|PM|second|evening|| +72379|AAAAAAAAMLKBBAAA|72379|20|6|19|PM|second|evening|| +72380|AAAAAAAANLKBBAAA|72380|20|6|20|PM|second|evening|| +72381|AAAAAAAAOLKBBAAA|72381|20|6|21|PM|second|evening|| +72382|AAAAAAAAPLKBBAAA|72382|20|6|22|PM|second|evening|| +72383|AAAAAAAAAMKBBAAA|72383|20|6|23|PM|second|evening|| +72384|AAAAAAAABMKBBAAA|72384|20|6|24|PM|second|evening|| +72385|AAAAAAAACMKBBAAA|72385|20|6|25|PM|second|evening|| +72386|AAAAAAAADMKBBAAA|72386|20|6|26|PM|second|evening|| +72387|AAAAAAAAEMKBBAAA|72387|20|6|27|PM|second|evening|| +72388|AAAAAAAAFMKBBAAA|72388|20|6|28|PM|second|evening|| +72389|AAAAAAAAGMKBBAAA|72389|20|6|29|PM|second|evening|| +72390|AAAAAAAAHMKBBAAA|72390|20|6|30|PM|second|evening|| +72391|AAAAAAAAIMKBBAAA|72391|20|6|31|PM|second|evening|| +72392|AAAAAAAAJMKBBAAA|72392|20|6|32|PM|second|evening|| +72393|AAAAAAAAKMKBBAAA|72393|20|6|33|PM|second|evening|| +72394|AAAAAAAALMKBBAAA|72394|20|6|34|PM|second|evening|| +72395|AAAAAAAAMMKBBAAA|72395|20|6|35|PM|second|evening|| +72396|AAAAAAAANMKBBAAA|72396|20|6|36|PM|second|evening|| +72397|AAAAAAAAOMKBBAAA|72397|20|6|37|PM|second|evening|| +72398|AAAAAAAAPMKBBAAA|72398|20|6|38|PM|second|evening|| +72399|AAAAAAAAANKBBAAA|72399|20|6|39|PM|second|evening|| +72400|AAAAAAAABNKBBAAA|72400|20|6|40|PM|second|evening|| +72401|AAAAAAAACNKBBAAA|72401|20|6|41|PM|second|evening|| +72402|AAAAAAAADNKBBAAA|72402|20|6|42|PM|second|evening|| +72403|AAAAAAAAENKBBAAA|72403|20|6|43|PM|second|evening|| +72404|AAAAAAAAFNKBBAAA|72404|20|6|44|PM|second|evening|| +72405|AAAAAAAAGNKBBAAA|72405|20|6|45|PM|second|evening|| +72406|AAAAAAAAHNKBBAAA|72406|20|6|46|PM|second|evening|| +72407|AAAAAAAAINKBBAAA|72407|20|6|47|PM|second|evening|| +72408|AAAAAAAAJNKBBAAA|72408|20|6|48|PM|second|evening|| +72409|AAAAAAAAKNKBBAAA|72409|20|6|49|PM|second|evening|| +72410|AAAAAAAALNKBBAAA|72410|20|6|50|PM|second|evening|| +72411|AAAAAAAAMNKBBAAA|72411|20|6|51|PM|second|evening|| +72412|AAAAAAAANNKBBAAA|72412|20|6|52|PM|second|evening|| +72413|AAAAAAAAONKBBAAA|72413|20|6|53|PM|second|evening|| +72414|AAAAAAAAPNKBBAAA|72414|20|6|54|PM|second|evening|| +72415|AAAAAAAAAOKBBAAA|72415|20|6|55|PM|second|evening|| +72416|AAAAAAAABOKBBAAA|72416|20|6|56|PM|second|evening|| +72417|AAAAAAAACOKBBAAA|72417|20|6|57|PM|second|evening|| +72418|AAAAAAAADOKBBAAA|72418|20|6|58|PM|second|evening|| +72419|AAAAAAAAEOKBBAAA|72419|20|6|59|PM|second|evening|| +72420|AAAAAAAAFOKBBAAA|72420|20|7|0|PM|second|evening|| +72421|AAAAAAAAGOKBBAAA|72421|20|7|1|PM|second|evening|| +72422|AAAAAAAAHOKBBAAA|72422|20|7|2|PM|second|evening|| +72423|AAAAAAAAIOKBBAAA|72423|20|7|3|PM|second|evening|| +72424|AAAAAAAAJOKBBAAA|72424|20|7|4|PM|second|evening|| +72425|AAAAAAAAKOKBBAAA|72425|20|7|5|PM|second|evening|| +72426|AAAAAAAALOKBBAAA|72426|20|7|6|PM|second|evening|| +72427|AAAAAAAAMOKBBAAA|72427|20|7|7|PM|second|evening|| +72428|AAAAAAAANOKBBAAA|72428|20|7|8|PM|second|evening|| +72429|AAAAAAAAOOKBBAAA|72429|20|7|9|PM|second|evening|| +72430|AAAAAAAAPOKBBAAA|72430|20|7|10|PM|second|evening|| +72431|AAAAAAAAAPKBBAAA|72431|20|7|11|PM|second|evening|| +72432|AAAAAAAABPKBBAAA|72432|20|7|12|PM|second|evening|| +72433|AAAAAAAACPKBBAAA|72433|20|7|13|PM|second|evening|| +72434|AAAAAAAADPKBBAAA|72434|20|7|14|PM|second|evening|| +72435|AAAAAAAAEPKBBAAA|72435|20|7|15|PM|second|evening|| +72436|AAAAAAAAFPKBBAAA|72436|20|7|16|PM|second|evening|| +72437|AAAAAAAAGPKBBAAA|72437|20|7|17|PM|second|evening|| +72438|AAAAAAAAHPKBBAAA|72438|20|7|18|PM|second|evening|| +72439|AAAAAAAAIPKBBAAA|72439|20|7|19|PM|second|evening|| +72440|AAAAAAAAJPKBBAAA|72440|20|7|20|PM|second|evening|| +72441|AAAAAAAAKPKBBAAA|72441|20|7|21|PM|second|evening|| +72442|AAAAAAAALPKBBAAA|72442|20|7|22|PM|second|evening|| +72443|AAAAAAAAMPKBBAAA|72443|20|7|23|PM|second|evening|| +72444|AAAAAAAANPKBBAAA|72444|20|7|24|PM|second|evening|| +72445|AAAAAAAAOPKBBAAA|72445|20|7|25|PM|second|evening|| +72446|AAAAAAAAPPKBBAAA|72446|20|7|26|PM|second|evening|| +72447|AAAAAAAAAALBBAAA|72447|20|7|27|PM|second|evening|| +72448|AAAAAAAABALBBAAA|72448|20|7|28|PM|second|evening|| +72449|AAAAAAAACALBBAAA|72449|20|7|29|PM|second|evening|| +72450|AAAAAAAADALBBAAA|72450|20|7|30|PM|second|evening|| +72451|AAAAAAAAEALBBAAA|72451|20|7|31|PM|second|evening|| +72452|AAAAAAAAFALBBAAA|72452|20|7|32|PM|second|evening|| +72453|AAAAAAAAGALBBAAA|72453|20|7|33|PM|second|evening|| +72454|AAAAAAAAHALBBAAA|72454|20|7|34|PM|second|evening|| +72455|AAAAAAAAIALBBAAA|72455|20|7|35|PM|second|evening|| +72456|AAAAAAAAJALBBAAA|72456|20|7|36|PM|second|evening|| +72457|AAAAAAAAKALBBAAA|72457|20|7|37|PM|second|evening|| +72458|AAAAAAAALALBBAAA|72458|20|7|38|PM|second|evening|| +72459|AAAAAAAAMALBBAAA|72459|20|7|39|PM|second|evening|| +72460|AAAAAAAANALBBAAA|72460|20|7|40|PM|second|evening|| +72461|AAAAAAAAOALBBAAA|72461|20|7|41|PM|second|evening|| +72462|AAAAAAAAPALBBAAA|72462|20|7|42|PM|second|evening|| +72463|AAAAAAAAABLBBAAA|72463|20|7|43|PM|second|evening|| +72464|AAAAAAAABBLBBAAA|72464|20|7|44|PM|second|evening|| +72465|AAAAAAAACBLBBAAA|72465|20|7|45|PM|second|evening|| +72466|AAAAAAAADBLBBAAA|72466|20|7|46|PM|second|evening|| +72467|AAAAAAAAEBLBBAAA|72467|20|7|47|PM|second|evening|| +72468|AAAAAAAAFBLBBAAA|72468|20|7|48|PM|second|evening|| +72469|AAAAAAAAGBLBBAAA|72469|20|7|49|PM|second|evening|| +72470|AAAAAAAAHBLBBAAA|72470|20|7|50|PM|second|evening|| +72471|AAAAAAAAIBLBBAAA|72471|20|7|51|PM|second|evening|| +72472|AAAAAAAAJBLBBAAA|72472|20|7|52|PM|second|evening|| +72473|AAAAAAAAKBLBBAAA|72473|20|7|53|PM|second|evening|| +72474|AAAAAAAALBLBBAAA|72474|20|7|54|PM|second|evening|| +72475|AAAAAAAAMBLBBAAA|72475|20|7|55|PM|second|evening|| +72476|AAAAAAAANBLBBAAA|72476|20|7|56|PM|second|evening|| +72477|AAAAAAAAOBLBBAAA|72477|20|7|57|PM|second|evening|| +72478|AAAAAAAAPBLBBAAA|72478|20|7|58|PM|second|evening|| +72479|AAAAAAAAACLBBAAA|72479|20|7|59|PM|second|evening|| +72480|AAAAAAAABCLBBAAA|72480|20|8|0|PM|second|evening|| +72481|AAAAAAAACCLBBAAA|72481|20|8|1|PM|second|evening|| +72482|AAAAAAAADCLBBAAA|72482|20|8|2|PM|second|evening|| +72483|AAAAAAAAECLBBAAA|72483|20|8|3|PM|second|evening|| +72484|AAAAAAAAFCLBBAAA|72484|20|8|4|PM|second|evening|| +72485|AAAAAAAAGCLBBAAA|72485|20|8|5|PM|second|evening|| +72486|AAAAAAAAHCLBBAAA|72486|20|8|6|PM|second|evening|| +72487|AAAAAAAAICLBBAAA|72487|20|8|7|PM|second|evening|| +72488|AAAAAAAAJCLBBAAA|72488|20|8|8|PM|second|evening|| +72489|AAAAAAAAKCLBBAAA|72489|20|8|9|PM|second|evening|| +72490|AAAAAAAALCLBBAAA|72490|20|8|10|PM|second|evening|| +72491|AAAAAAAAMCLBBAAA|72491|20|8|11|PM|second|evening|| +72492|AAAAAAAANCLBBAAA|72492|20|8|12|PM|second|evening|| +72493|AAAAAAAAOCLBBAAA|72493|20|8|13|PM|second|evening|| +72494|AAAAAAAAPCLBBAAA|72494|20|8|14|PM|second|evening|| +72495|AAAAAAAAADLBBAAA|72495|20|8|15|PM|second|evening|| +72496|AAAAAAAABDLBBAAA|72496|20|8|16|PM|second|evening|| +72497|AAAAAAAACDLBBAAA|72497|20|8|17|PM|second|evening|| +72498|AAAAAAAADDLBBAAA|72498|20|8|18|PM|second|evening|| +72499|AAAAAAAAEDLBBAAA|72499|20|8|19|PM|second|evening|| +72500|AAAAAAAAFDLBBAAA|72500|20|8|20|PM|second|evening|| +72501|AAAAAAAAGDLBBAAA|72501|20|8|21|PM|second|evening|| +72502|AAAAAAAAHDLBBAAA|72502|20|8|22|PM|second|evening|| +72503|AAAAAAAAIDLBBAAA|72503|20|8|23|PM|second|evening|| +72504|AAAAAAAAJDLBBAAA|72504|20|8|24|PM|second|evening|| +72505|AAAAAAAAKDLBBAAA|72505|20|8|25|PM|second|evening|| +72506|AAAAAAAALDLBBAAA|72506|20|8|26|PM|second|evening|| +72507|AAAAAAAAMDLBBAAA|72507|20|8|27|PM|second|evening|| +72508|AAAAAAAANDLBBAAA|72508|20|8|28|PM|second|evening|| +72509|AAAAAAAAODLBBAAA|72509|20|8|29|PM|second|evening|| +72510|AAAAAAAAPDLBBAAA|72510|20|8|30|PM|second|evening|| +72511|AAAAAAAAAELBBAAA|72511|20|8|31|PM|second|evening|| +72512|AAAAAAAABELBBAAA|72512|20|8|32|PM|second|evening|| +72513|AAAAAAAACELBBAAA|72513|20|8|33|PM|second|evening|| +72514|AAAAAAAADELBBAAA|72514|20|8|34|PM|second|evening|| +72515|AAAAAAAAEELBBAAA|72515|20|8|35|PM|second|evening|| +72516|AAAAAAAAFELBBAAA|72516|20|8|36|PM|second|evening|| +72517|AAAAAAAAGELBBAAA|72517|20|8|37|PM|second|evening|| +72518|AAAAAAAAHELBBAAA|72518|20|8|38|PM|second|evening|| +72519|AAAAAAAAIELBBAAA|72519|20|8|39|PM|second|evening|| +72520|AAAAAAAAJELBBAAA|72520|20|8|40|PM|second|evening|| +72521|AAAAAAAAKELBBAAA|72521|20|8|41|PM|second|evening|| +72522|AAAAAAAALELBBAAA|72522|20|8|42|PM|second|evening|| +72523|AAAAAAAAMELBBAAA|72523|20|8|43|PM|second|evening|| +72524|AAAAAAAANELBBAAA|72524|20|8|44|PM|second|evening|| +72525|AAAAAAAAOELBBAAA|72525|20|8|45|PM|second|evening|| +72526|AAAAAAAAPELBBAAA|72526|20|8|46|PM|second|evening|| +72527|AAAAAAAAAFLBBAAA|72527|20|8|47|PM|second|evening|| +72528|AAAAAAAABFLBBAAA|72528|20|8|48|PM|second|evening|| +72529|AAAAAAAACFLBBAAA|72529|20|8|49|PM|second|evening|| +72530|AAAAAAAADFLBBAAA|72530|20|8|50|PM|second|evening|| +72531|AAAAAAAAEFLBBAAA|72531|20|8|51|PM|second|evening|| +72532|AAAAAAAAFFLBBAAA|72532|20|8|52|PM|second|evening|| +72533|AAAAAAAAGFLBBAAA|72533|20|8|53|PM|second|evening|| +72534|AAAAAAAAHFLBBAAA|72534|20|8|54|PM|second|evening|| +72535|AAAAAAAAIFLBBAAA|72535|20|8|55|PM|second|evening|| +72536|AAAAAAAAJFLBBAAA|72536|20|8|56|PM|second|evening|| +72537|AAAAAAAAKFLBBAAA|72537|20|8|57|PM|second|evening|| +72538|AAAAAAAALFLBBAAA|72538|20|8|58|PM|second|evening|| +72539|AAAAAAAAMFLBBAAA|72539|20|8|59|PM|second|evening|| +72540|AAAAAAAANFLBBAAA|72540|20|9|0|PM|second|evening|| +72541|AAAAAAAAOFLBBAAA|72541|20|9|1|PM|second|evening|| +72542|AAAAAAAAPFLBBAAA|72542|20|9|2|PM|second|evening|| +72543|AAAAAAAAAGLBBAAA|72543|20|9|3|PM|second|evening|| +72544|AAAAAAAABGLBBAAA|72544|20|9|4|PM|second|evening|| +72545|AAAAAAAACGLBBAAA|72545|20|9|5|PM|second|evening|| +72546|AAAAAAAADGLBBAAA|72546|20|9|6|PM|second|evening|| +72547|AAAAAAAAEGLBBAAA|72547|20|9|7|PM|second|evening|| +72548|AAAAAAAAFGLBBAAA|72548|20|9|8|PM|second|evening|| +72549|AAAAAAAAGGLBBAAA|72549|20|9|9|PM|second|evening|| +72550|AAAAAAAAHGLBBAAA|72550|20|9|10|PM|second|evening|| +72551|AAAAAAAAIGLBBAAA|72551|20|9|11|PM|second|evening|| +72552|AAAAAAAAJGLBBAAA|72552|20|9|12|PM|second|evening|| +72553|AAAAAAAAKGLBBAAA|72553|20|9|13|PM|second|evening|| +72554|AAAAAAAALGLBBAAA|72554|20|9|14|PM|second|evening|| +72555|AAAAAAAAMGLBBAAA|72555|20|9|15|PM|second|evening|| +72556|AAAAAAAANGLBBAAA|72556|20|9|16|PM|second|evening|| +72557|AAAAAAAAOGLBBAAA|72557|20|9|17|PM|second|evening|| +72558|AAAAAAAAPGLBBAAA|72558|20|9|18|PM|second|evening|| +72559|AAAAAAAAAHLBBAAA|72559|20|9|19|PM|second|evening|| +72560|AAAAAAAABHLBBAAA|72560|20|9|20|PM|second|evening|| +72561|AAAAAAAACHLBBAAA|72561|20|9|21|PM|second|evening|| +72562|AAAAAAAADHLBBAAA|72562|20|9|22|PM|second|evening|| +72563|AAAAAAAAEHLBBAAA|72563|20|9|23|PM|second|evening|| +72564|AAAAAAAAFHLBBAAA|72564|20|9|24|PM|second|evening|| +72565|AAAAAAAAGHLBBAAA|72565|20|9|25|PM|second|evening|| +72566|AAAAAAAAHHLBBAAA|72566|20|9|26|PM|second|evening|| +72567|AAAAAAAAIHLBBAAA|72567|20|9|27|PM|second|evening|| +72568|AAAAAAAAJHLBBAAA|72568|20|9|28|PM|second|evening|| +72569|AAAAAAAAKHLBBAAA|72569|20|9|29|PM|second|evening|| +72570|AAAAAAAALHLBBAAA|72570|20|9|30|PM|second|evening|| +72571|AAAAAAAAMHLBBAAA|72571|20|9|31|PM|second|evening|| +72572|AAAAAAAANHLBBAAA|72572|20|9|32|PM|second|evening|| +72573|AAAAAAAAOHLBBAAA|72573|20|9|33|PM|second|evening|| +72574|AAAAAAAAPHLBBAAA|72574|20|9|34|PM|second|evening|| +72575|AAAAAAAAAILBBAAA|72575|20|9|35|PM|second|evening|| +72576|AAAAAAAABILBBAAA|72576|20|9|36|PM|second|evening|| +72577|AAAAAAAACILBBAAA|72577|20|9|37|PM|second|evening|| +72578|AAAAAAAADILBBAAA|72578|20|9|38|PM|second|evening|| +72579|AAAAAAAAEILBBAAA|72579|20|9|39|PM|second|evening|| +72580|AAAAAAAAFILBBAAA|72580|20|9|40|PM|second|evening|| +72581|AAAAAAAAGILBBAAA|72581|20|9|41|PM|second|evening|| +72582|AAAAAAAAHILBBAAA|72582|20|9|42|PM|second|evening|| +72583|AAAAAAAAIILBBAAA|72583|20|9|43|PM|second|evening|| +72584|AAAAAAAAJILBBAAA|72584|20|9|44|PM|second|evening|| +72585|AAAAAAAAKILBBAAA|72585|20|9|45|PM|second|evening|| +72586|AAAAAAAALILBBAAA|72586|20|9|46|PM|second|evening|| +72587|AAAAAAAAMILBBAAA|72587|20|9|47|PM|second|evening|| +72588|AAAAAAAANILBBAAA|72588|20|9|48|PM|second|evening|| +72589|AAAAAAAAOILBBAAA|72589|20|9|49|PM|second|evening|| +72590|AAAAAAAAPILBBAAA|72590|20|9|50|PM|second|evening|| +72591|AAAAAAAAAJLBBAAA|72591|20|9|51|PM|second|evening|| +72592|AAAAAAAABJLBBAAA|72592|20|9|52|PM|second|evening|| +72593|AAAAAAAACJLBBAAA|72593|20|9|53|PM|second|evening|| +72594|AAAAAAAADJLBBAAA|72594|20|9|54|PM|second|evening|| +72595|AAAAAAAAEJLBBAAA|72595|20|9|55|PM|second|evening|| +72596|AAAAAAAAFJLBBAAA|72596|20|9|56|PM|second|evening|| +72597|AAAAAAAAGJLBBAAA|72597|20|9|57|PM|second|evening|| +72598|AAAAAAAAHJLBBAAA|72598|20|9|58|PM|second|evening|| +72599|AAAAAAAAIJLBBAAA|72599|20|9|59|PM|second|evening|| +72600|AAAAAAAAJJLBBAAA|72600|20|10|0|PM|second|evening|| +72601|AAAAAAAAKJLBBAAA|72601|20|10|1|PM|second|evening|| +72602|AAAAAAAALJLBBAAA|72602|20|10|2|PM|second|evening|| +72603|AAAAAAAAMJLBBAAA|72603|20|10|3|PM|second|evening|| +72604|AAAAAAAANJLBBAAA|72604|20|10|4|PM|second|evening|| +72605|AAAAAAAAOJLBBAAA|72605|20|10|5|PM|second|evening|| +72606|AAAAAAAAPJLBBAAA|72606|20|10|6|PM|second|evening|| +72607|AAAAAAAAAKLBBAAA|72607|20|10|7|PM|second|evening|| +72608|AAAAAAAABKLBBAAA|72608|20|10|8|PM|second|evening|| +72609|AAAAAAAACKLBBAAA|72609|20|10|9|PM|second|evening|| +72610|AAAAAAAADKLBBAAA|72610|20|10|10|PM|second|evening|| +72611|AAAAAAAAEKLBBAAA|72611|20|10|11|PM|second|evening|| +72612|AAAAAAAAFKLBBAAA|72612|20|10|12|PM|second|evening|| +72613|AAAAAAAAGKLBBAAA|72613|20|10|13|PM|second|evening|| +72614|AAAAAAAAHKLBBAAA|72614|20|10|14|PM|second|evening|| +72615|AAAAAAAAIKLBBAAA|72615|20|10|15|PM|second|evening|| +72616|AAAAAAAAJKLBBAAA|72616|20|10|16|PM|second|evening|| +72617|AAAAAAAAKKLBBAAA|72617|20|10|17|PM|second|evening|| +72618|AAAAAAAALKLBBAAA|72618|20|10|18|PM|second|evening|| +72619|AAAAAAAAMKLBBAAA|72619|20|10|19|PM|second|evening|| +72620|AAAAAAAANKLBBAAA|72620|20|10|20|PM|second|evening|| +72621|AAAAAAAAOKLBBAAA|72621|20|10|21|PM|second|evening|| +72622|AAAAAAAAPKLBBAAA|72622|20|10|22|PM|second|evening|| +72623|AAAAAAAAALLBBAAA|72623|20|10|23|PM|second|evening|| +72624|AAAAAAAABLLBBAAA|72624|20|10|24|PM|second|evening|| +72625|AAAAAAAACLLBBAAA|72625|20|10|25|PM|second|evening|| +72626|AAAAAAAADLLBBAAA|72626|20|10|26|PM|second|evening|| +72627|AAAAAAAAELLBBAAA|72627|20|10|27|PM|second|evening|| +72628|AAAAAAAAFLLBBAAA|72628|20|10|28|PM|second|evening|| +72629|AAAAAAAAGLLBBAAA|72629|20|10|29|PM|second|evening|| +72630|AAAAAAAAHLLBBAAA|72630|20|10|30|PM|second|evening|| +72631|AAAAAAAAILLBBAAA|72631|20|10|31|PM|second|evening|| +72632|AAAAAAAAJLLBBAAA|72632|20|10|32|PM|second|evening|| +72633|AAAAAAAAKLLBBAAA|72633|20|10|33|PM|second|evening|| +72634|AAAAAAAALLLBBAAA|72634|20|10|34|PM|second|evening|| +72635|AAAAAAAAMLLBBAAA|72635|20|10|35|PM|second|evening|| +72636|AAAAAAAANLLBBAAA|72636|20|10|36|PM|second|evening|| +72637|AAAAAAAAOLLBBAAA|72637|20|10|37|PM|second|evening|| +72638|AAAAAAAAPLLBBAAA|72638|20|10|38|PM|second|evening|| +72639|AAAAAAAAAMLBBAAA|72639|20|10|39|PM|second|evening|| +72640|AAAAAAAABMLBBAAA|72640|20|10|40|PM|second|evening|| +72641|AAAAAAAACMLBBAAA|72641|20|10|41|PM|second|evening|| +72642|AAAAAAAADMLBBAAA|72642|20|10|42|PM|second|evening|| +72643|AAAAAAAAEMLBBAAA|72643|20|10|43|PM|second|evening|| +72644|AAAAAAAAFMLBBAAA|72644|20|10|44|PM|second|evening|| +72645|AAAAAAAAGMLBBAAA|72645|20|10|45|PM|second|evening|| +72646|AAAAAAAAHMLBBAAA|72646|20|10|46|PM|second|evening|| +72647|AAAAAAAAIMLBBAAA|72647|20|10|47|PM|second|evening|| +72648|AAAAAAAAJMLBBAAA|72648|20|10|48|PM|second|evening|| +72649|AAAAAAAAKMLBBAAA|72649|20|10|49|PM|second|evening|| +72650|AAAAAAAALMLBBAAA|72650|20|10|50|PM|second|evening|| +72651|AAAAAAAAMMLBBAAA|72651|20|10|51|PM|second|evening|| +72652|AAAAAAAANMLBBAAA|72652|20|10|52|PM|second|evening|| +72653|AAAAAAAAOMLBBAAA|72653|20|10|53|PM|second|evening|| +72654|AAAAAAAAPMLBBAAA|72654|20|10|54|PM|second|evening|| +72655|AAAAAAAAANLBBAAA|72655|20|10|55|PM|second|evening|| +72656|AAAAAAAABNLBBAAA|72656|20|10|56|PM|second|evening|| +72657|AAAAAAAACNLBBAAA|72657|20|10|57|PM|second|evening|| +72658|AAAAAAAADNLBBAAA|72658|20|10|58|PM|second|evening|| +72659|AAAAAAAAENLBBAAA|72659|20|10|59|PM|second|evening|| +72660|AAAAAAAAFNLBBAAA|72660|20|11|0|PM|second|evening|| +72661|AAAAAAAAGNLBBAAA|72661|20|11|1|PM|second|evening|| +72662|AAAAAAAAHNLBBAAA|72662|20|11|2|PM|second|evening|| +72663|AAAAAAAAINLBBAAA|72663|20|11|3|PM|second|evening|| +72664|AAAAAAAAJNLBBAAA|72664|20|11|4|PM|second|evening|| +72665|AAAAAAAAKNLBBAAA|72665|20|11|5|PM|second|evening|| +72666|AAAAAAAALNLBBAAA|72666|20|11|6|PM|second|evening|| +72667|AAAAAAAAMNLBBAAA|72667|20|11|7|PM|second|evening|| +72668|AAAAAAAANNLBBAAA|72668|20|11|8|PM|second|evening|| +72669|AAAAAAAAONLBBAAA|72669|20|11|9|PM|second|evening|| +72670|AAAAAAAAPNLBBAAA|72670|20|11|10|PM|second|evening|| +72671|AAAAAAAAAOLBBAAA|72671|20|11|11|PM|second|evening|| +72672|AAAAAAAABOLBBAAA|72672|20|11|12|PM|second|evening|| +72673|AAAAAAAACOLBBAAA|72673|20|11|13|PM|second|evening|| +72674|AAAAAAAADOLBBAAA|72674|20|11|14|PM|second|evening|| +72675|AAAAAAAAEOLBBAAA|72675|20|11|15|PM|second|evening|| +72676|AAAAAAAAFOLBBAAA|72676|20|11|16|PM|second|evening|| +72677|AAAAAAAAGOLBBAAA|72677|20|11|17|PM|second|evening|| +72678|AAAAAAAAHOLBBAAA|72678|20|11|18|PM|second|evening|| +72679|AAAAAAAAIOLBBAAA|72679|20|11|19|PM|second|evening|| +72680|AAAAAAAAJOLBBAAA|72680|20|11|20|PM|second|evening|| +72681|AAAAAAAAKOLBBAAA|72681|20|11|21|PM|second|evening|| +72682|AAAAAAAALOLBBAAA|72682|20|11|22|PM|second|evening|| +72683|AAAAAAAAMOLBBAAA|72683|20|11|23|PM|second|evening|| +72684|AAAAAAAANOLBBAAA|72684|20|11|24|PM|second|evening|| +72685|AAAAAAAAOOLBBAAA|72685|20|11|25|PM|second|evening|| +72686|AAAAAAAAPOLBBAAA|72686|20|11|26|PM|second|evening|| +72687|AAAAAAAAAPLBBAAA|72687|20|11|27|PM|second|evening|| +72688|AAAAAAAABPLBBAAA|72688|20|11|28|PM|second|evening|| +72689|AAAAAAAACPLBBAAA|72689|20|11|29|PM|second|evening|| +72690|AAAAAAAADPLBBAAA|72690|20|11|30|PM|second|evening|| +72691|AAAAAAAAEPLBBAAA|72691|20|11|31|PM|second|evening|| +72692|AAAAAAAAFPLBBAAA|72692|20|11|32|PM|second|evening|| +72693|AAAAAAAAGPLBBAAA|72693|20|11|33|PM|second|evening|| +72694|AAAAAAAAHPLBBAAA|72694|20|11|34|PM|second|evening|| +72695|AAAAAAAAIPLBBAAA|72695|20|11|35|PM|second|evening|| +72696|AAAAAAAAJPLBBAAA|72696|20|11|36|PM|second|evening|| +72697|AAAAAAAAKPLBBAAA|72697|20|11|37|PM|second|evening|| +72698|AAAAAAAALPLBBAAA|72698|20|11|38|PM|second|evening|| +72699|AAAAAAAAMPLBBAAA|72699|20|11|39|PM|second|evening|| +72700|AAAAAAAANPLBBAAA|72700|20|11|40|PM|second|evening|| +72701|AAAAAAAAOPLBBAAA|72701|20|11|41|PM|second|evening|| +72702|AAAAAAAAPPLBBAAA|72702|20|11|42|PM|second|evening|| +72703|AAAAAAAAAAMBBAAA|72703|20|11|43|PM|second|evening|| +72704|AAAAAAAABAMBBAAA|72704|20|11|44|PM|second|evening|| +72705|AAAAAAAACAMBBAAA|72705|20|11|45|PM|second|evening|| +72706|AAAAAAAADAMBBAAA|72706|20|11|46|PM|second|evening|| +72707|AAAAAAAAEAMBBAAA|72707|20|11|47|PM|second|evening|| +72708|AAAAAAAAFAMBBAAA|72708|20|11|48|PM|second|evening|| +72709|AAAAAAAAGAMBBAAA|72709|20|11|49|PM|second|evening|| +72710|AAAAAAAAHAMBBAAA|72710|20|11|50|PM|second|evening|| +72711|AAAAAAAAIAMBBAAA|72711|20|11|51|PM|second|evening|| +72712|AAAAAAAAJAMBBAAA|72712|20|11|52|PM|second|evening|| +72713|AAAAAAAAKAMBBAAA|72713|20|11|53|PM|second|evening|| +72714|AAAAAAAALAMBBAAA|72714|20|11|54|PM|second|evening|| +72715|AAAAAAAAMAMBBAAA|72715|20|11|55|PM|second|evening|| +72716|AAAAAAAANAMBBAAA|72716|20|11|56|PM|second|evening|| +72717|AAAAAAAAOAMBBAAA|72717|20|11|57|PM|second|evening|| +72718|AAAAAAAAPAMBBAAA|72718|20|11|58|PM|second|evening|| +72719|AAAAAAAAABMBBAAA|72719|20|11|59|PM|second|evening|| +72720|AAAAAAAABBMBBAAA|72720|20|12|0|PM|second|evening|| +72721|AAAAAAAACBMBBAAA|72721|20|12|1|PM|second|evening|| +72722|AAAAAAAADBMBBAAA|72722|20|12|2|PM|second|evening|| +72723|AAAAAAAAEBMBBAAA|72723|20|12|3|PM|second|evening|| +72724|AAAAAAAAFBMBBAAA|72724|20|12|4|PM|second|evening|| +72725|AAAAAAAAGBMBBAAA|72725|20|12|5|PM|second|evening|| +72726|AAAAAAAAHBMBBAAA|72726|20|12|6|PM|second|evening|| +72727|AAAAAAAAIBMBBAAA|72727|20|12|7|PM|second|evening|| +72728|AAAAAAAAJBMBBAAA|72728|20|12|8|PM|second|evening|| +72729|AAAAAAAAKBMBBAAA|72729|20|12|9|PM|second|evening|| +72730|AAAAAAAALBMBBAAA|72730|20|12|10|PM|second|evening|| +72731|AAAAAAAAMBMBBAAA|72731|20|12|11|PM|second|evening|| +72732|AAAAAAAANBMBBAAA|72732|20|12|12|PM|second|evening|| +72733|AAAAAAAAOBMBBAAA|72733|20|12|13|PM|second|evening|| +72734|AAAAAAAAPBMBBAAA|72734|20|12|14|PM|second|evening|| +72735|AAAAAAAAACMBBAAA|72735|20|12|15|PM|second|evening|| +72736|AAAAAAAABCMBBAAA|72736|20|12|16|PM|second|evening|| +72737|AAAAAAAACCMBBAAA|72737|20|12|17|PM|second|evening|| +72738|AAAAAAAADCMBBAAA|72738|20|12|18|PM|second|evening|| +72739|AAAAAAAAECMBBAAA|72739|20|12|19|PM|second|evening|| +72740|AAAAAAAAFCMBBAAA|72740|20|12|20|PM|second|evening|| +72741|AAAAAAAAGCMBBAAA|72741|20|12|21|PM|second|evening|| +72742|AAAAAAAAHCMBBAAA|72742|20|12|22|PM|second|evening|| +72743|AAAAAAAAICMBBAAA|72743|20|12|23|PM|second|evening|| +72744|AAAAAAAAJCMBBAAA|72744|20|12|24|PM|second|evening|| +72745|AAAAAAAAKCMBBAAA|72745|20|12|25|PM|second|evening|| +72746|AAAAAAAALCMBBAAA|72746|20|12|26|PM|second|evening|| +72747|AAAAAAAAMCMBBAAA|72747|20|12|27|PM|second|evening|| +72748|AAAAAAAANCMBBAAA|72748|20|12|28|PM|second|evening|| +72749|AAAAAAAAOCMBBAAA|72749|20|12|29|PM|second|evening|| +72750|AAAAAAAAPCMBBAAA|72750|20|12|30|PM|second|evening|| +72751|AAAAAAAAADMBBAAA|72751|20|12|31|PM|second|evening|| +72752|AAAAAAAABDMBBAAA|72752|20|12|32|PM|second|evening|| +72753|AAAAAAAACDMBBAAA|72753|20|12|33|PM|second|evening|| +72754|AAAAAAAADDMBBAAA|72754|20|12|34|PM|second|evening|| +72755|AAAAAAAAEDMBBAAA|72755|20|12|35|PM|second|evening|| +72756|AAAAAAAAFDMBBAAA|72756|20|12|36|PM|second|evening|| +72757|AAAAAAAAGDMBBAAA|72757|20|12|37|PM|second|evening|| +72758|AAAAAAAAHDMBBAAA|72758|20|12|38|PM|second|evening|| +72759|AAAAAAAAIDMBBAAA|72759|20|12|39|PM|second|evening|| +72760|AAAAAAAAJDMBBAAA|72760|20|12|40|PM|second|evening|| +72761|AAAAAAAAKDMBBAAA|72761|20|12|41|PM|second|evening|| +72762|AAAAAAAALDMBBAAA|72762|20|12|42|PM|second|evening|| +72763|AAAAAAAAMDMBBAAA|72763|20|12|43|PM|second|evening|| +72764|AAAAAAAANDMBBAAA|72764|20|12|44|PM|second|evening|| +72765|AAAAAAAAODMBBAAA|72765|20|12|45|PM|second|evening|| +72766|AAAAAAAAPDMBBAAA|72766|20|12|46|PM|second|evening|| +72767|AAAAAAAAAEMBBAAA|72767|20|12|47|PM|second|evening|| +72768|AAAAAAAABEMBBAAA|72768|20|12|48|PM|second|evening|| +72769|AAAAAAAACEMBBAAA|72769|20|12|49|PM|second|evening|| +72770|AAAAAAAADEMBBAAA|72770|20|12|50|PM|second|evening|| +72771|AAAAAAAAEEMBBAAA|72771|20|12|51|PM|second|evening|| +72772|AAAAAAAAFEMBBAAA|72772|20|12|52|PM|second|evening|| +72773|AAAAAAAAGEMBBAAA|72773|20|12|53|PM|second|evening|| +72774|AAAAAAAAHEMBBAAA|72774|20|12|54|PM|second|evening|| +72775|AAAAAAAAIEMBBAAA|72775|20|12|55|PM|second|evening|| +72776|AAAAAAAAJEMBBAAA|72776|20|12|56|PM|second|evening|| +72777|AAAAAAAAKEMBBAAA|72777|20|12|57|PM|second|evening|| +72778|AAAAAAAALEMBBAAA|72778|20|12|58|PM|second|evening|| +72779|AAAAAAAAMEMBBAAA|72779|20|12|59|PM|second|evening|| +72780|AAAAAAAANEMBBAAA|72780|20|13|0|PM|second|evening|| +72781|AAAAAAAAOEMBBAAA|72781|20|13|1|PM|second|evening|| +72782|AAAAAAAAPEMBBAAA|72782|20|13|2|PM|second|evening|| +72783|AAAAAAAAAFMBBAAA|72783|20|13|3|PM|second|evening|| +72784|AAAAAAAABFMBBAAA|72784|20|13|4|PM|second|evening|| +72785|AAAAAAAACFMBBAAA|72785|20|13|5|PM|second|evening|| +72786|AAAAAAAADFMBBAAA|72786|20|13|6|PM|second|evening|| +72787|AAAAAAAAEFMBBAAA|72787|20|13|7|PM|second|evening|| +72788|AAAAAAAAFFMBBAAA|72788|20|13|8|PM|second|evening|| +72789|AAAAAAAAGFMBBAAA|72789|20|13|9|PM|second|evening|| +72790|AAAAAAAAHFMBBAAA|72790|20|13|10|PM|second|evening|| +72791|AAAAAAAAIFMBBAAA|72791|20|13|11|PM|second|evening|| +72792|AAAAAAAAJFMBBAAA|72792|20|13|12|PM|second|evening|| +72793|AAAAAAAAKFMBBAAA|72793|20|13|13|PM|second|evening|| +72794|AAAAAAAALFMBBAAA|72794|20|13|14|PM|second|evening|| +72795|AAAAAAAAMFMBBAAA|72795|20|13|15|PM|second|evening|| +72796|AAAAAAAANFMBBAAA|72796|20|13|16|PM|second|evening|| +72797|AAAAAAAAOFMBBAAA|72797|20|13|17|PM|second|evening|| +72798|AAAAAAAAPFMBBAAA|72798|20|13|18|PM|second|evening|| +72799|AAAAAAAAAGMBBAAA|72799|20|13|19|PM|second|evening|| +72800|AAAAAAAABGMBBAAA|72800|20|13|20|PM|second|evening|| +72801|AAAAAAAACGMBBAAA|72801|20|13|21|PM|second|evening|| +72802|AAAAAAAADGMBBAAA|72802|20|13|22|PM|second|evening|| +72803|AAAAAAAAEGMBBAAA|72803|20|13|23|PM|second|evening|| +72804|AAAAAAAAFGMBBAAA|72804|20|13|24|PM|second|evening|| +72805|AAAAAAAAGGMBBAAA|72805|20|13|25|PM|second|evening|| +72806|AAAAAAAAHGMBBAAA|72806|20|13|26|PM|second|evening|| +72807|AAAAAAAAIGMBBAAA|72807|20|13|27|PM|second|evening|| +72808|AAAAAAAAJGMBBAAA|72808|20|13|28|PM|second|evening|| +72809|AAAAAAAAKGMBBAAA|72809|20|13|29|PM|second|evening|| +72810|AAAAAAAALGMBBAAA|72810|20|13|30|PM|second|evening|| +72811|AAAAAAAAMGMBBAAA|72811|20|13|31|PM|second|evening|| +72812|AAAAAAAANGMBBAAA|72812|20|13|32|PM|second|evening|| +72813|AAAAAAAAOGMBBAAA|72813|20|13|33|PM|second|evening|| +72814|AAAAAAAAPGMBBAAA|72814|20|13|34|PM|second|evening|| +72815|AAAAAAAAAHMBBAAA|72815|20|13|35|PM|second|evening|| +72816|AAAAAAAABHMBBAAA|72816|20|13|36|PM|second|evening|| +72817|AAAAAAAACHMBBAAA|72817|20|13|37|PM|second|evening|| +72818|AAAAAAAADHMBBAAA|72818|20|13|38|PM|second|evening|| +72819|AAAAAAAAEHMBBAAA|72819|20|13|39|PM|second|evening|| +72820|AAAAAAAAFHMBBAAA|72820|20|13|40|PM|second|evening|| +72821|AAAAAAAAGHMBBAAA|72821|20|13|41|PM|second|evening|| +72822|AAAAAAAAHHMBBAAA|72822|20|13|42|PM|second|evening|| +72823|AAAAAAAAIHMBBAAA|72823|20|13|43|PM|second|evening|| +72824|AAAAAAAAJHMBBAAA|72824|20|13|44|PM|second|evening|| +72825|AAAAAAAAKHMBBAAA|72825|20|13|45|PM|second|evening|| +72826|AAAAAAAALHMBBAAA|72826|20|13|46|PM|second|evening|| +72827|AAAAAAAAMHMBBAAA|72827|20|13|47|PM|second|evening|| +72828|AAAAAAAANHMBBAAA|72828|20|13|48|PM|second|evening|| +72829|AAAAAAAAOHMBBAAA|72829|20|13|49|PM|second|evening|| +72830|AAAAAAAAPHMBBAAA|72830|20|13|50|PM|second|evening|| +72831|AAAAAAAAAIMBBAAA|72831|20|13|51|PM|second|evening|| +72832|AAAAAAAABIMBBAAA|72832|20|13|52|PM|second|evening|| +72833|AAAAAAAACIMBBAAA|72833|20|13|53|PM|second|evening|| +72834|AAAAAAAADIMBBAAA|72834|20|13|54|PM|second|evening|| +72835|AAAAAAAAEIMBBAAA|72835|20|13|55|PM|second|evening|| +72836|AAAAAAAAFIMBBAAA|72836|20|13|56|PM|second|evening|| +72837|AAAAAAAAGIMBBAAA|72837|20|13|57|PM|second|evening|| +72838|AAAAAAAAHIMBBAAA|72838|20|13|58|PM|second|evening|| +72839|AAAAAAAAIIMBBAAA|72839|20|13|59|PM|second|evening|| +72840|AAAAAAAAJIMBBAAA|72840|20|14|0|PM|second|evening|| +72841|AAAAAAAAKIMBBAAA|72841|20|14|1|PM|second|evening|| +72842|AAAAAAAALIMBBAAA|72842|20|14|2|PM|second|evening|| +72843|AAAAAAAAMIMBBAAA|72843|20|14|3|PM|second|evening|| +72844|AAAAAAAANIMBBAAA|72844|20|14|4|PM|second|evening|| +72845|AAAAAAAAOIMBBAAA|72845|20|14|5|PM|second|evening|| +72846|AAAAAAAAPIMBBAAA|72846|20|14|6|PM|second|evening|| +72847|AAAAAAAAAJMBBAAA|72847|20|14|7|PM|second|evening|| +72848|AAAAAAAABJMBBAAA|72848|20|14|8|PM|second|evening|| +72849|AAAAAAAACJMBBAAA|72849|20|14|9|PM|second|evening|| +72850|AAAAAAAADJMBBAAA|72850|20|14|10|PM|second|evening|| +72851|AAAAAAAAEJMBBAAA|72851|20|14|11|PM|second|evening|| +72852|AAAAAAAAFJMBBAAA|72852|20|14|12|PM|second|evening|| +72853|AAAAAAAAGJMBBAAA|72853|20|14|13|PM|second|evening|| +72854|AAAAAAAAHJMBBAAA|72854|20|14|14|PM|second|evening|| +72855|AAAAAAAAIJMBBAAA|72855|20|14|15|PM|second|evening|| +72856|AAAAAAAAJJMBBAAA|72856|20|14|16|PM|second|evening|| +72857|AAAAAAAAKJMBBAAA|72857|20|14|17|PM|second|evening|| +72858|AAAAAAAALJMBBAAA|72858|20|14|18|PM|second|evening|| +72859|AAAAAAAAMJMBBAAA|72859|20|14|19|PM|second|evening|| +72860|AAAAAAAANJMBBAAA|72860|20|14|20|PM|second|evening|| +72861|AAAAAAAAOJMBBAAA|72861|20|14|21|PM|second|evening|| +72862|AAAAAAAAPJMBBAAA|72862|20|14|22|PM|second|evening|| +72863|AAAAAAAAAKMBBAAA|72863|20|14|23|PM|second|evening|| +72864|AAAAAAAABKMBBAAA|72864|20|14|24|PM|second|evening|| +72865|AAAAAAAACKMBBAAA|72865|20|14|25|PM|second|evening|| +72866|AAAAAAAADKMBBAAA|72866|20|14|26|PM|second|evening|| +72867|AAAAAAAAEKMBBAAA|72867|20|14|27|PM|second|evening|| +72868|AAAAAAAAFKMBBAAA|72868|20|14|28|PM|second|evening|| +72869|AAAAAAAAGKMBBAAA|72869|20|14|29|PM|second|evening|| +72870|AAAAAAAAHKMBBAAA|72870|20|14|30|PM|second|evening|| +72871|AAAAAAAAIKMBBAAA|72871|20|14|31|PM|second|evening|| +72872|AAAAAAAAJKMBBAAA|72872|20|14|32|PM|second|evening|| +72873|AAAAAAAAKKMBBAAA|72873|20|14|33|PM|second|evening|| +72874|AAAAAAAALKMBBAAA|72874|20|14|34|PM|second|evening|| +72875|AAAAAAAAMKMBBAAA|72875|20|14|35|PM|second|evening|| +72876|AAAAAAAANKMBBAAA|72876|20|14|36|PM|second|evening|| +72877|AAAAAAAAOKMBBAAA|72877|20|14|37|PM|second|evening|| +72878|AAAAAAAAPKMBBAAA|72878|20|14|38|PM|second|evening|| +72879|AAAAAAAAALMBBAAA|72879|20|14|39|PM|second|evening|| +72880|AAAAAAAABLMBBAAA|72880|20|14|40|PM|second|evening|| +72881|AAAAAAAACLMBBAAA|72881|20|14|41|PM|second|evening|| +72882|AAAAAAAADLMBBAAA|72882|20|14|42|PM|second|evening|| +72883|AAAAAAAAELMBBAAA|72883|20|14|43|PM|second|evening|| +72884|AAAAAAAAFLMBBAAA|72884|20|14|44|PM|second|evening|| +72885|AAAAAAAAGLMBBAAA|72885|20|14|45|PM|second|evening|| +72886|AAAAAAAAHLMBBAAA|72886|20|14|46|PM|second|evening|| +72887|AAAAAAAAILMBBAAA|72887|20|14|47|PM|second|evening|| +72888|AAAAAAAAJLMBBAAA|72888|20|14|48|PM|second|evening|| +72889|AAAAAAAAKLMBBAAA|72889|20|14|49|PM|second|evening|| +72890|AAAAAAAALLMBBAAA|72890|20|14|50|PM|second|evening|| +72891|AAAAAAAAMLMBBAAA|72891|20|14|51|PM|second|evening|| +72892|AAAAAAAANLMBBAAA|72892|20|14|52|PM|second|evening|| +72893|AAAAAAAAOLMBBAAA|72893|20|14|53|PM|second|evening|| +72894|AAAAAAAAPLMBBAAA|72894|20|14|54|PM|second|evening|| +72895|AAAAAAAAAMMBBAAA|72895|20|14|55|PM|second|evening|| +72896|AAAAAAAABMMBBAAA|72896|20|14|56|PM|second|evening|| +72897|AAAAAAAACMMBBAAA|72897|20|14|57|PM|second|evening|| +72898|AAAAAAAADMMBBAAA|72898|20|14|58|PM|second|evening|| +72899|AAAAAAAAEMMBBAAA|72899|20|14|59|PM|second|evening|| +72900|AAAAAAAAFMMBBAAA|72900|20|15|0|PM|second|evening|| +72901|AAAAAAAAGMMBBAAA|72901|20|15|1|PM|second|evening|| +72902|AAAAAAAAHMMBBAAA|72902|20|15|2|PM|second|evening|| +72903|AAAAAAAAIMMBBAAA|72903|20|15|3|PM|second|evening|| +72904|AAAAAAAAJMMBBAAA|72904|20|15|4|PM|second|evening|| +72905|AAAAAAAAKMMBBAAA|72905|20|15|5|PM|second|evening|| +72906|AAAAAAAALMMBBAAA|72906|20|15|6|PM|second|evening|| +72907|AAAAAAAAMMMBBAAA|72907|20|15|7|PM|second|evening|| +72908|AAAAAAAANMMBBAAA|72908|20|15|8|PM|second|evening|| +72909|AAAAAAAAOMMBBAAA|72909|20|15|9|PM|second|evening|| +72910|AAAAAAAAPMMBBAAA|72910|20|15|10|PM|second|evening|| +72911|AAAAAAAAANMBBAAA|72911|20|15|11|PM|second|evening|| +72912|AAAAAAAABNMBBAAA|72912|20|15|12|PM|second|evening|| +72913|AAAAAAAACNMBBAAA|72913|20|15|13|PM|second|evening|| +72914|AAAAAAAADNMBBAAA|72914|20|15|14|PM|second|evening|| +72915|AAAAAAAAENMBBAAA|72915|20|15|15|PM|second|evening|| +72916|AAAAAAAAFNMBBAAA|72916|20|15|16|PM|second|evening|| +72917|AAAAAAAAGNMBBAAA|72917|20|15|17|PM|second|evening|| +72918|AAAAAAAAHNMBBAAA|72918|20|15|18|PM|second|evening|| +72919|AAAAAAAAINMBBAAA|72919|20|15|19|PM|second|evening|| +72920|AAAAAAAAJNMBBAAA|72920|20|15|20|PM|second|evening|| +72921|AAAAAAAAKNMBBAAA|72921|20|15|21|PM|second|evening|| +72922|AAAAAAAALNMBBAAA|72922|20|15|22|PM|second|evening|| +72923|AAAAAAAAMNMBBAAA|72923|20|15|23|PM|second|evening|| +72924|AAAAAAAANNMBBAAA|72924|20|15|24|PM|second|evening|| +72925|AAAAAAAAONMBBAAA|72925|20|15|25|PM|second|evening|| +72926|AAAAAAAAPNMBBAAA|72926|20|15|26|PM|second|evening|| +72927|AAAAAAAAAOMBBAAA|72927|20|15|27|PM|second|evening|| +72928|AAAAAAAABOMBBAAA|72928|20|15|28|PM|second|evening|| +72929|AAAAAAAACOMBBAAA|72929|20|15|29|PM|second|evening|| +72930|AAAAAAAADOMBBAAA|72930|20|15|30|PM|second|evening|| +72931|AAAAAAAAEOMBBAAA|72931|20|15|31|PM|second|evening|| +72932|AAAAAAAAFOMBBAAA|72932|20|15|32|PM|second|evening|| +72933|AAAAAAAAGOMBBAAA|72933|20|15|33|PM|second|evening|| +72934|AAAAAAAAHOMBBAAA|72934|20|15|34|PM|second|evening|| +72935|AAAAAAAAIOMBBAAA|72935|20|15|35|PM|second|evening|| +72936|AAAAAAAAJOMBBAAA|72936|20|15|36|PM|second|evening|| +72937|AAAAAAAAKOMBBAAA|72937|20|15|37|PM|second|evening|| +72938|AAAAAAAALOMBBAAA|72938|20|15|38|PM|second|evening|| +72939|AAAAAAAAMOMBBAAA|72939|20|15|39|PM|second|evening|| +72940|AAAAAAAANOMBBAAA|72940|20|15|40|PM|second|evening|| +72941|AAAAAAAAOOMBBAAA|72941|20|15|41|PM|second|evening|| +72942|AAAAAAAAPOMBBAAA|72942|20|15|42|PM|second|evening|| +72943|AAAAAAAAAPMBBAAA|72943|20|15|43|PM|second|evening|| +72944|AAAAAAAABPMBBAAA|72944|20|15|44|PM|second|evening|| +72945|AAAAAAAACPMBBAAA|72945|20|15|45|PM|second|evening|| +72946|AAAAAAAADPMBBAAA|72946|20|15|46|PM|second|evening|| +72947|AAAAAAAAEPMBBAAA|72947|20|15|47|PM|second|evening|| +72948|AAAAAAAAFPMBBAAA|72948|20|15|48|PM|second|evening|| +72949|AAAAAAAAGPMBBAAA|72949|20|15|49|PM|second|evening|| +72950|AAAAAAAAHPMBBAAA|72950|20|15|50|PM|second|evening|| +72951|AAAAAAAAIPMBBAAA|72951|20|15|51|PM|second|evening|| +72952|AAAAAAAAJPMBBAAA|72952|20|15|52|PM|second|evening|| +72953|AAAAAAAAKPMBBAAA|72953|20|15|53|PM|second|evening|| +72954|AAAAAAAALPMBBAAA|72954|20|15|54|PM|second|evening|| +72955|AAAAAAAAMPMBBAAA|72955|20|15|55|PM|second|evening|| +72956|AAAAAAAANPMBBAAA|72956|20|15|56|PM|second|evening|| +72957|AAAAAAAAOPMBBAAA|72957|20|15|57|PM|second|evening|| +72958|AAAAAAAAPPMBBAAA|72958|20|15|58|PM|second|evening|| +72959|AAAAAAAAAANBBAAA|72959|20|15|59|PM|second|evening|| +72960|AAAAAAAABANBBAAA|72960|20|16|0|PM|second|evening|| +72961|AAAAAAAACANBBAAA|72961|20|16|1|PM|second|evening|| +72962|AAAAAAAADANBBAAA|72962|20|16|2|PM|second|evening|| +72963|AAAAAAAAEANBBAAA|72963|20|16|3|PM|second|evening|| +72964|AAAAAAAAFANBBAAA|72964|20|16|4|PM|second|evening|| +72965|AAAAAAAAGANBBAAA|72965|20|16|5|PM|second|evening|| +72966|AAAAAAAAHANBBAAA|72966|20|16|6|PM|second|evening|| +72967|AAAAAAAAIANBBAAA|72967|20|16|7|PM|second|evening|| +72968|AAAAAAAAJANBBAAA|72968|20|16|8|PM|second|evening|| +72969|AAAAAAAAKANBBAAA|72969|20|16|9|PM|second|evening|| +72970|AAAAAAAALANBBAAA|72970|20|16|10|PM|second|evening|| +72971|AAAAAAAAMANBBAAA|72971|20|16|11|PM|second|evening|| +72972|AAAAAAAANANBBAAA|72972|20|16|12|PM|second|evening|| +72973|AAAAAAAAOANBBAAA|72973|20|16|13|PM|second|evening|| +72974|AAAAAAAAPANBBAAA|72974|20|16|14|PM|second|evening|| +72975|AAAAAAAAABNBBAAA|72975|20|16|15|PM|second|evening|| +72976|AAAAAAAABBNBBAAA|72976|20|16|16|PM|second|evening|| +72977|AAAAAAAACBNBBAAA|72977|20|16|17|PM|second|evening|| +72978|AAAAAAAADBNBBAAA|72978|20|16|18|PM|second|evening|| +72979|AAAAAAAAEBNBBAAA|72979|20|16|19|PM|second|evening|| +72980|AAAAAAAAFBNBBAAA|72980|20|16|20|PM|second|evening|| +72981|AAAAAAAAGBNBBAAA|72981|20|16|21|PM|second|evening|| +72982|AAAAAAAAHBNBBAAA|72982|20|16|22|PM|second|evening|| +72983|AAAAAAAAIBNBBAAA|72983|20|16|23|PM|second|evening|| +72984|AAAAAAAAJBNBBAAA|72984|20|16|24|PM|second|evening|| +72985|AAAAAAAAKBNBBAAA|72985|20|16|25|PM|second|evening|| +72986|AAAAAAAALBNBBAAA|72986|20|16|26|PM|second|evening|| +72987|AAAAAAAAMBNBBAAA|72987|20|16|27|PM|second|evening|| +72988|AAAAAAAANBNBBAAA|72988|20|16|28|PM|second|evening|| +72989|AAAAAAAAOBNBBAAA|72989|20|16|29|PM|second|evening|| +72990|AAAAAAAAPBNBBAAA|72990|20|16|30|PM|second|evening|| +72991|AAAAAAAAACNBBAAA|72991|20|16|31|PM|second|evening|| +72992|AAAAAAAABCNBBAAA|72992|20|16|32|PM|second|evening|| +72993|AAAAAAAACCNBBAAA|72993|20|16|33|PM|second|evening|| +72994|AAAAAAAADCNBBAAA|72994|20|16|34|PM|second|evening|| +72995|AAAAAAAAECNBBAAA|72995|20|16|35|PM|second|evening|| +72996|AAAAAAAAFCNBBAAA|72996|20|16|36|PM|second|evening|| +72997|AAAAAAAAGCNBBAAA|72997|20|16|37|PM|second|evening|| +72998|AAAAAAAAHCNBBAAA|72998|20|16|38|PM|second|evening|| +72999|AAAAAAAAICNBBAAA|72999|20|16|39|PM|second|evening|| +73000|AAAAAAAAJCNBBAAA|73000|20|16|40|PM|second|evening|| +73001|AAAAAAAAKCNBBAAA|73001|20|16|41|PM|second|evening|| +73002|AAAAAAAALCNBBAAA|73002|20|16|42|PM|second|evening|| +73003|AAAAAAAAMCNBBAAA|73003|20|16|43|PM|second|evening|| +73004|AAAAAAAANCNBBAAA|73004|20|16|44|PM|second|evening|| +73005|AAAAAAAAOCNBBAAA|73005|20|16|45|PM|second|evening|| +73006|AAAAAAAAPCNBBAAA|73006|20|16|46|PM|second|evening|| +73007|AAAAAAAAADNBBAAA|73007|20|16|47|PM|second|evening|| +73008|AAAAAAAABDNBBAAA|73008|20|16|48|PM|second|evening|| +73009|AAAAAAAACDNBBAAA|73009|20|16|49|PM|second|evening|| +73010|AAAAAAAADDNBBAAA|73010|20|16|50|PM|second|evening|| +73011|AAAAAAAAEDNBBAAA|73011|20|16|51|PM|second|evening|| +73012|AAAAAAAAFDNBBAAA|73012|20|16|52|PM|second|evening|| +73013|AAAAAAAAGDNBBAAA|73013|20|16|53|PM|second|evening|| +73014|AAAAAAAAHDNBBAAA|73014|20|16|54|PM|second|evening|| +73015|AAAAAAAAIDNBBAAA|73015|20|16|55|PM|second|evening|| +73016|AAAAAAAAJDNBBAAA|73016|20|16|56|PM|second|evening|| +73017|AAAAAAAAKDNBBAAA|73017|20|16|57|PM|second|evening|| +73018|AAAAAAAALDNBBAAA|73018|20|16|58|PM|second|evening|| +73019|AAAAAAAAMDNBBAAA|73019|20|16|59|PM|second|evening|| +73020|AAAAAAAANDNBBAAA|73020|20|17|0|PM|second|evening|| +73021|AAAAAAAAODNBBAAA|73021|20|17|1|PM|second|evening|| +73022|AAAAAAAAPDNBBAAA|73022|20|17|2|PM|second|evening|| +73023|AAAAAAAAAENBBAAA|73023|20|17|3|PM|second|evening|| +73024|AAAAAAAABENBBAAA|73024|20|17|4|PM|second|evening|| +73025|AAAAAAAACENBBAAA|73025|20|17|5|PM|second|evening|| +73026|AAAAAAAADENBBAAA|73026|20|17|6|PM|second|evening|| +73027|AAAAAAAAEENBBAAA|73027|20|17|7|PM|second|evening|| +73028|AAAAAAAAFENBBAAA|73028|20|17|8|PM|second|evening|| +73029|AAAAAAAAGENBBAAA|73029|20|17|9|PM|second|evening|| +73030|AAAAAAAAHENBBAAA|73030|20|17|10|PM|second|evening|| +73031|AAAAAAAAIENBBAAA|73031|20|17|11|PM|second|evening|| +73032|AAAAAAAAJENBBAAA|73032|20|17|12|PM|second|evening|| +73033|AAAAAAAAKENBBAAA|73033|20|17|13|PM|second|evening|| +73034|AAAAAAAALENBBAAA|73034|20|17|14|PM|second|evening|| +73035|AAAAAAAAMENBBAAA|73035|20|17|15|PM|second|evening|| +73036|AAAAAAAANENBBAAA|73036|20|17|16|PM|second|evening|| +73037|AAAAAAAAOENBBAAA|73037|20|17|17|PM|second|evening|| +73038|AAAAAAAAPENBBAAA|73038|20|17|18|PM|second|evening|| +73039|AAAAAAAAAFNBBAAA|73039|20|17|19|PM|second|evening|| +73040|AAAAAAAABFNBBAAA|73040|20|17|20|PM|second|evening|| +73041|AAAAAAAACFNBBAAA|73041|20|17|21|PM|second|evening|| +73042|AAAAAAAADFNBBAAA|73042|20|17|22|PM|second|evening|| +73043|AAAAAAAAEFNBBAAA|73043|20|17|23|PM|second|evening|| +73044|AAAAAAAAFFNBBAAA|73044|20|17|24|PM|second|evening|| +73045|AAAAAAAAGFNBBAAA|73045|20|17|25|PM|second|evening|| +73046|AAAAAAAAHFNBBAAA|73046|20|17|26|PM|second|evening|| +73047|AAAAAAAAIFNBBAAA|73047|20|17|27|PM|second|evening|| +73048|AAAAAAAAJFNBBAAA|73048|20|17|28|PM|second|evening|| +73049|AAAAAAAAKFNBBAAA|73049|20|17|29|PM|second|evening|| +73050|AAAAAAAALFNBBAAA|73050|20|17|30|PM|second|evening|| +73051|AAAAAAAAMFNBBAAA|73051|20|17|31|PM|second|evening|| +73052|AAAAAAAANFNBBAAA|73052|20|17|32|PM|second|evening|| +73053|AAAAAAAAOFNBBAAA|73053|20|17|33|PM|second|evening|| +73054|AAAAAAAAPFNBBAAA|73054|20|17|34|PM|second|evening|| +73055|AAAAAAAAAGNBBAAA|73055|20|17|35|PM|second|evening|| +73056|AAAAAAAABGNBBAAA|73056|20|17|36|PM|second|evening|| +73057|AAAAAAAACGNBBAAA|73057|20|17|37|PM|second|evening|| +73058|AAAAAAAADGNBBAAA|73058|20|17|38|PM|second|evening|| +73059|AAAAAAAAEGNBBAAA|73059|20|17|39|PM|second|evening|| +73060|AAAAAAAAFGNBBAAA|73060|20|17|40|PM|second|evening|| +73061|AAAAAAAAGGNBBAAA|73061|20|17|41|PM|second|evening|| +73062|AAAAAAAAHGNBBAAA|73062|20|17|42|PM|second|evening|| +73063|AAAAAAAAIGNBBAAA|73063|20|17|43|PM|second|evening|| +73064|AAAAAAAAJGNBBAAA|73064|20|17|44|PM|second|evening|| +73065|AAAAAAAAKGNBBAAA|73065|20|17|45|PM|second|evening|| +73066|AAAAAAAALGNBBAAA|73066|20|17|46|PM|second|evening|| +73067|AAAAAAAAMGNBBAAA|73067|20|17|47|PM|second|evening|| +73068|AAAAAAAANGNBBAAA|73068|20|17|48|PM|second|evening|| +73069|AAAAAAAAOGNBBAAA|73069|20|17|49|PM|second|evening|| +73070|AAAAAAAAPGNBBAAA|73070|20|17|50|PM|second|evening|| +73071|AAAAAAAAAHNBBAAA|73071|20|17|51|PM|second|evening|| +73072|AAAAAAAABHNBBAAA|73072|20|17|52|PM|second|evening|| +73073|AAAAAAAACHNBBAAA|73073|20|17|53|PM|second|evening|| +73074|AAAAAAAADHNBBAAA|73074|20|17|54|PM|second|evening|| +73075|AAAAAAAAEHNBBAAA|73075|20|17|55|PM|second|evening|| +73076|AAAAAAAAFHNBBAAA|73076|20|17|56|PM|second|evening|| +73077|AAAAAAAAGHNBBAAA|73077|20|17|57|PM|second|evening|| +73078|AAAAAAAAHHNBBAAA|73078|20|17|58|PM|second|evening|| +73079|AAAAAAAAIHNBBAAA|73079|20|17|59|PM|second|evening|| +73080|AAAAAAAAJHNBBAAA|73080|20|18|0|PM|second|evening|| +73081|AAAAAAAAKHNBBAAA|73081|20|18|1|PM|second|evening|| +73082|AAAAAAAALHNBBAAA|73082|20|18|2|PM|second|evening|| +73083|AAAAAAAAMHNBBAAA|73083|20|18|3|PM|second|evening|| +73084|AAAAAAAANHNBBAAA|73084|20|18|4|PM|second|evening|| +73085|AAAAAAAAOHNBBAAA|73085|20|18|5|PM|second|evening|| +73086|AAAAAAAAPHNBBAAA|73086|20|18|6|PM|second|evening|| +73087|AAAAAAAAAINBBAAA|73087|20|18|7|PM|second|evening|| +73088|AAAAAAAABINBBAAA|73088|20|18|8|PM|second|evening|| +73089|AAAAAAAACINBBAAA|73089|20|18|9|PM|second|evening|| +73090|AAAAAAAADINBBAAA|73090|20|18|10|PM|second|evening|| +73091|AAAAAAAAEINBBAAA|73091|20|18|11|PM|second|evening|| +73092|AAAAAAAAFINBBAAA|73092|20|18|12|PM|second|evening|| +73093|AAAAAAAAGINBBAAA|73093|20|18|13|PM|second|evening|| +73094|AAAAAAAAHINBBAAA|73094|20|18|14|PM|second|evening|| +73095|AAAAAAAAIINBBAAA|73095|20|18|15|PM|second|evening|| +73096|AAAAAAAAJINBBAAA|73096|20|18|16|PM|second|evening|| +73097|AAAAAAAAKINBBAAA|73097|20|18|17|PM|second|evening|| +73098|AAAAAAAALINBBAAA|73098|20|18|18|PM|second|evening|| +73099|AAAAAAAAMINBBAAA|73099|20|18|19|PM|second|evening|| +73100|AAAAAAAANINBBAAA|73100|20|18|20|PM|second|evening|| +73101|AAAAAAAAOINBBAAA|73101|20|18|21|PM|second|evening|| +73102|AAAAAAAAPINBBAAA|73102|20|18|22|PM|second|evening|| +73103|AAAAAAAAAJNBBAAA|73103|20|18|23|PM|second|evening|| +73104|AAAAAAAABJNBBAAA|73104|20|18|24|PM|second|evening|| +73105|AAAAAAAACJNBBAAA|73105|20|18|25|PM|second|evening|| +73106|AAAAAAAADJNBBAAA|73106|20|18|26|PM|second|evening|| +73107|AAAAAAAAEJNBBAAA|73107|20|18|27|PM|second|evening|| +73108|AAAAAAAAFJNBBAAA|73108|20|18|28|PM|second|evening|| +73109|AAAAAAAAGJNBBAAA|73109|20|18|29|PM|second|evening|| +73110|AAAAAAAAHJNBBAAA|73110|20|18|30|PM|second|evening|| +73111|AAAAAAAAIJNBBAAA|73111|20|18|31|PM|second|evening|| +73112|AAAAAAAAJJNBBAAA|73112|20|18|32|PM|second|evening|| +73113|AAAAAAAAKJNBBAAA|73113|20|18|33|PM|second|evening|| +73114|AAAAAAAALJNBBAAA|73114|20|18|34|PM|second|evening|| +73115|AAAAAAAAMJNBBAAA|73115|20|18|35|PM|second|evening|| +73116|AAAAAAAANJNBBAAA|73116|20|18|36|PM|second|evening|| +73117|AAAAAAAAOJNBBAAA|73117|20|18|37|PM|second|evening|| +73118|AAAAAAAAPJNBBAAA|73118|20|18|38|PM|second|evening|| +73119|AAAAAAAAAKNBBAAA|73119|20|18|39|PM|second|evening|| +73120|AAAAAAAABKNBBAAA|73120|20|18|40|PM|second|evening|| +73121|AAAAAAAACKNBBAAA|73121|20|18|41|PM|second|evening|| +73122|AAAAAAAADKNBBAAA|73122|20|18|42|PM|second|evening|| +73123|AAAAAAAAEKNBBAAA|73123|20|18|43|PM|second|evening|| +73124|AAAAAAAAFKNBBAAA|73124|20|18|44|PM|second|evening|| +73125|AAAAAAAAGKNBBAAA|73125|20|18|45|PM|second|evening|| +73126|AAAAAAAAHKNBBAAA|73126|20|18|46|PM|second|evening|| +73127|AAAAAAAAIKNBBAAA|73127|20|18|47|PM|second|evening|| +73128|AAAAAAAAJKNBBAAA|73128|20|18|48|PM|second|evening|| +73129|AAAAAAAAKKNBBAAA|73129|20|18|49|PM|second|evening|| +73130|AAAAAAAALKNBBAAA|73130|20|18|50|PM|second|evening|| +73131|AAAAAAAAMKNBBAAA|73131|20|18|51|PM|second|evening|| +73132|AAAAAAAANKNBBAAA|73132|20|18|52|PM|second|evening|| +73133|AAAAAAAAOKNBBAAA|73133|20|18|53|PM|second|evening|| +73134|AAAAAAAAPKNBBAAA|73134|20|18|54|PM|second|evening|| +73135|AAAAAAAAALNBBAAA|73135|20|18|55|PM|second|evening|| +73136|AAAAAAAABLNBBAAA|73136|20|18|56|PM|second|evening|| +73137|AAAAAAAACLNBBAAA|73137|20|18|57|PM|second|evening|| +73138|AAAAAAAADLNBBAAA|73138|20|18|58|PM|second|evening|| +73139|AAAAAAAAELNBBAAA|73139|20|18|59|PM|second|evening|| +73140|AAAAAAAAFLNBBAAA|73140|20|19|0|PM|second|evening|| +73141|AAAAAAAAGLNBBAAA|73141|20|19|1|PM|second|evening|| +73142|AAAAAAAAHLNBBAAA|73142|20|19|2|PM|second|evening|| +73143|AAAAAAAAILNBBAAA|73143|20|19|3|PM|second|evening|| +73144|AAAAAAAAJLNBBAAA|73144|20|19|4|PM|second|evening|| +73145|AAAAAAAAKLNBBAAA|73145|20|19|5|PM|second|evening|| +73146|AAAAAAAALLNBBAAA|73146|20|19|6|PM|second|evening|| +73147|AAAAAAAAMLNBBAAA|73147|20|19|7|PM|second|evening|| +73148|AAAAAAAANLNBBAAA|73148|20|19|8|PM|second|evening|| +73149|AAAAAAAAOLNBBAAA|73149|20|19|9|PM|second|evening|| +73150|AAAAAAAAPLNBBAAA|73150|20|19|10|PM|second|evening|| +73151|AAAAAAAAAMNBBAAA|73151|20|19|11|PM|second|evening|| +73152|AAAAAAAABMNBBAAA|73152|20|19|12|PM|second|evening|| +73153|AAAAAAAACMNBBAAA|73153|20|19|13|PM|second|evening|| +73154|AAAAAAAADMNBBAAA|73154|20|19|14|PM|second|evening|| +73155|AAAAAAAAEMNBBAAA|73155|20|19|15|PM|second|evening|| +73156|AAAAAAAAFMNBBAAA|73156|20|19|16|PM|second|evening|| +73157|AAAAAAAAGMNBBAAA|73157|20|19|17|PM|second|evening|| +73158|AAAAAAAAHMNBBAAA|73158|20|19|18|PM|second|evening|| +73159|AAAAAAAAIMNBBAAA|73159|20|19|19|PM|second|evening|| +73160|AAAAAAAAJMNBBAAA|73160|20|19|20|PM|second|evening|| +73161|AAAAAAAAKMNBBAAA|73161|20|19|21|PM|second|evening|| +73162|AAAAAAAALMNBBAAA|73162|20|19|22|PM|second|evening|| +73163|AAAAAAAAMMNBBAAA|73163|20|19|23|PM|second|evening|| +73164|AAAAAAAANMNBBAAA|73164|20|19|24|PM|second|evening|| +73165|AAAAAAAAOMNBBAAA|73165|20|19|25|PM|second|evening|| +73166|AAAAAAAAPMNBBAAA|73166|20|19|26|PM|second|evening|| +73167|AAAAAAAAANNBBAAA|73167|20|19|27|PM|second|evening|| +73168|AAAAAAAABNNBBAAA|73168|20|19|28|PM|second|evening|| +73169|AAAAAAAACNNBBAAA|73169|20|19|29|PM|second|evening|| +73170|AAAAAAAADNNBBAAA|73170|20|19|30|PM|second|evening|| +73171|AAAAAAAAENNBBAAA|73171|20|19|31|PM|second|evening|| +73172|AAAAAAAAFNNBBAAA|73172|20|19|32|PM|second|evening|| +73173|AAAAAAAAGNNBBAAA|73173|20|19|33|PM|second|evening|| +73174|AAAAAAAAHNNBBAAA|73174|20|19|34|PM|second|evening|| +73175|AAAAAAAAINNBBAAA|73175|20|19|35|PM|second|evening|| +73176|AAAAAAAAJNNBBAAA|73176|20|19|36|PM|second|evening|| +73177|AAAAAAAAKNNBBAAA|73177|20|19|37|PM|second|evening|| +73178|AAAAAAAALNNBBAAA|73178|20|19|38|PM|second|evening|| +73179|AAAAAAAAMNNBBAAA|73179|20|19|39|PM|second|evening|| +73180|AAAAAAAANNNBBAAA|73180|20|19|40|PM|second|evening|| +73181|AAAAAAAAONNBBAAA|73181|20|19|41|PM|second|evening|| +73182|AAAAAAAAPNNBBAAA|73182|20|19|42|PM|second|evening|| +73183|AAAAAAAAAONBBAAA|73183|20|19|43|PM|second|evening|| +73184|AAAAAAAABONBBAAA|73184|20|19|44|PM|second|evening|| +73185|AAAAAAAACONBBAAA|73185|20|19|45|PM|second|evening|| +73186|AAAAAAAADONBBAAA|73186|20|19|46|PM|second|evening|| +73187|AAAAAAAAEONBBAAA|73187|20|19|47|PM|second|evening|| +73188|AAAAAAAAFONBBAAA|73188|20|19|48|PM|second|evening|| +73189|AAAAAAAAGONBBAAA|73189|20|19|49|PM|second|evening|| +73190|AAAAAAAAHONBBAAA|73190|20|19|50|PM|second|evening|| +73191|AAAAAAAAIONBBAAA|73191|20|19|51|PM|second|evening|| +73192|AAAAAAAAJONBBAAA|73192|20|19|52|PM|second|evening|| +73193|AAAAAAAAKONBBAAA|73193|20|19|53|PM|second|evening|| +73194|AAAAAAAALONBBAAA|73194|20|19|54|PM|second|evening|| +73195|AAAAAAAAMONBBAAA|73195|20|19|55|PM|second|evening|| +73196|AAAAAAAANONBBAAA|73196|20|19|56|PM|second|evening|| +73197|AAAAAAAAOONBBAAA|73197|20|19|57|PM|second|evening|| +73198|AAAAAAAAPONBBAAA|73198|20|19|58|PM|second|evening|| +73199|AAAAAAAAAPNBBAAA|73199|20|19|59|PM|second|evening|| +73200|AAAAAAAABPNBBAAA|73200|20|20|0|PM|second|evening|| +73201|AAAAAAAACPNBBAAA|73201|20|20|1|PM|second|evening|| +73202|AAAAAAAADPNBBAAA|73202|20|20|2|PM|second|evening|| +73203|AAAAAAAAEPNBBAAA|73203|20|20|3|PM|second|evening|| +73204|AAAAAAAAFPNBBAAA|73204|20|20|4|PM|second|evening|| +73205|AAAAAAAAGPNBBAAA|73205|20|20|5|PM|second|evening|| +73206|AAAAAAAAHPNBBAAA|73206|20|20|6|PM|second|evening|| +73207|AAAAAAAAIPNBBAAA|73207|20|20|7|PM|second|evening|| +73208|AAAAAAAAJPNBBAAA|73208|20|20|8|PM|second|evening|| +73209|AAAAAAAAKPNBBAAA|73209|20|20|9|PM|second|evening|| +73210|AAAAAAAALPNBBAAA|73210|20|20|10|PM|second|evening|| +73211|AAAAAAAAMPNBBAAA|73211|20|20|11|PM|second|evening|| +73212|AAAAAAAANPNBBAAA|73212|20|20|12|PM|second|evening|| +73213|AAAAAAAAOPNBBAAA|73213|20|20|13|PM|second|evening|| +73214|AAAAAAAAPPNBBAAA|73214|20|20|14|PM|second|evening|| +73215|AAAAAAAAAAOBBAAA|73215|20|20|15|PM|second|evening|| +73216|AAAAAAAABAOBBAAA|73216|20|20|16|PM|second|evening|| +73217|AAAAAAAACAOBBAAA|73217|20|20|17|PM|second|evening|| +73218|AAAAAAAADAOBBAAA|73218|20|20|18|PM|second|evening|| +73219|AAAAAAAAEAOBBAAA|73219|20|20|19|PM|second|evening|| +73220|AAAAAAAAFAOBBAAA|73220|20|20|20|PM|second|evening|| +73221|AAAAAAAAGAOBBAAA|73221|20|20|21|PM|second|evening|| +73222|AAAAAAAAHAOBBAAA|73222|20|20|22|PM|second|evening|| +73223|AAAAAAAAIAOBBAAA|73223|20|20|23|PM|second|evening|| +73224|AAAAAAAAJAOBBAAA|73224|20|20|24|PM|second|evening|| +73225|AAAAAAAAKAOBBAAA|73225|20|20|25|PM|second|evening|| +73226|AAAAAAAALAOBBAAA|73226|20|20|26|PM|second|evening|| +73227|AAAAAAAAMAOBBAAA|73227|20|20|27|PM|second|evening|| +73228|AAAAAAAANAOBBAAA|73228|20|20|28|PM|second|evening|| +73229|AAAAAAAAOAOBBAAA|73229|20|20|29|PM|second|evening|| +73230|AAAAAAAAPAOBBAAA|73230|20|20|30|PM|second|evening|| +73231|AAAAAAAAABOBBAAA|73231|20|20|31|PM|second|evening|| +73232|AAAAAAAABBOBBAAA|73232|20|20|32|PM|second|evening|| +73233|AAAAAAAACBOBBAAA|73233|20|20|33|PM|second|evening|| +73234|AAAAAAAADBOBBAAA|73234|20|20|34|PM|second|evening|| +73235|AAAAAAAAEBOBBAAA|73235|20|20|35|PM|second|evening|| +73236|AAAAAAAAFBOBBAAA|73236|20|20|36|PM|second|evening|| +73237|AAAAAAAAGBOBBAAA|73237|20|20|37|PM|second|evening|| +73238|AAAAAAAAHBOBBAAA|73238|20|20|38|PM|second|evening|| +73239|AAAAAAAAIBOBBAAA|73239|20|20|39|PM|second|evening|| +73240|AAAAAAAAJBOBBAAA|73240|20|20|40|PM|second|evening|| +73241|AAAAAAAAKBOBBAAA|73241|20|20|41|PM|second|evening|| +73242|AAAAAAAALBOBBAAA|73242|20|20|42|PM|second|evening|| +73243|AAAAAAAAMBOBBAAA|73243|20|20|43|PM|second|evening|| +73244|AAAAAAAANBOBBAAA|73244|20|20|44|PM|second|evening|| +73245|AAAAAAAAOBOBBAAA|73245|20|20|45|PM|second|evening|| +73246|AAAAAAAAPBOBBAAA|73246|20|20|46|PM|second|evening|| +73247|AAAAAAAAACOBBAAA|73247|20|20|47|PM|second|evening|| +73248|AAAAAAAABCOBBAAA|73248|20|20|48|PM|second|evening|| +73249|AAAAAAAACCOBBAAA|73249|20|20|49|PM|second|evening|| +73250|AAAAAAAADCOBBAAA|73250|20|20|50|PM|second|evening|| +73251|AAAAAAAAECOBBAAA|73251|20|20|51|PM|second|evening|| +73252|AAAAAAAAFCOBBAAA|73252|20|20|52|PM|second|evening|| +73253|AAAAAAAAGCOBBAAA|73253|20|20|53|PM|second|evening|| +73254|AAAAAAAAHCOBBAAA|73254|20|20|54|PM|second|evening|| +73255|AAAAAAAAICOBBAAA|73255|20|20|55|PM|second|evening|| +73256|AAAAAAAAJCOBBAAA|73256|20|20|56|PM|second|evening|| +73257|AAAAAAAAKCOBBAAA|73257|20|20|57|PM|second|evening|| +73258|AAAAAAAALCOBBAAA|73258|20|20|58|PM|second|evening|| +73259|AAAAAAAAMCOBBAAA|73259|20|20|59|PM|second|evening|| +73260|AAAAAAAANCOBBAAA|73260|20|21|0|PM|second|evening|| +73261|AAAAAAAAOCOBBAAA|73261|20|21|1|PM|second|evening|| +73262|AAAAAAAAPCOBBAAA|73262|20|21|2|PM|second|evening|| +73263|AAAAAAAAADOBBAAA|73263|20|21|3|PM|second|evening|| +73264|AAAAAAAABDOBBAAA|73264|20|21|4|PM|second|evening|| +73265|AAAAAAAACDOBBAAA|73265|20|21|5|PM|second|evening|| +73266|AAAAAAAADDOBBAAA|73266|20|21|6|PM|second|evening|| +73267|AAAAAAAAEDOBBAAA|73267|20|21|7|PM|second|evening|| +73268|AAAAAAAAFDOBBAAA|73268|20|21|8|PM|second|evening|| +73269|AAAAAAAAGDOBBAAA|73269|20|21|9|PM|second|evening|| +73270|AAAAAAAAHDOBBAAA|73270|20|21|10|PM|second|evening|| +73271|AAAAAAAAIDOBBAAA|73271|20|21|11|PM|second|evening|| +73272|AAAAAAAAJDOBBAAA|73272|20|21|12|PM|second|evening|| +73273|AAAAAAAAKDOBBAAA|73273|20|21|13|PM|second|evening|| +73274|AAAAAAAALDOBBAAA|73274|20|21|14|PM|second|evening|| +73275|AAAAAAAAMDOBBAAA|73275|20|21|15|PM|second|evening|| +73276|AAAAAAAANDOBBAAA|73276|20|21|16|PM|second|evening|| +73277|AAAAAAAAODOBBAAA|73277|20|21|17|PM|second|evening|| +73278|AAAAAAAAPDOBBAAA|73278|20|21|18|PM|second|evening|| +73279|AAAAAAAAAEOBBAAA|73279|20|21|19|PM|second|evening|| +73280|AAAAAAAABEOBBAAA|73280|20|21|20|PM|second|evening|| +73281|AAAAAAAACEOBBAAA|73281|20|21|21|PM|second|evening|| +73282|AAAAAAAADEOBBAAA|73282|20|21|22|PM|second|evening|| +73283|AAAAAAAAEEOBBAAA|73283|20|21|23|PM|second|evening|| +73284|AAAAAAAAFEOBBAAA|73284|20|21|24|PM|second|evening|| +73285|AAAAAAAAGEOBBAAA|73285|20|21|25|PM|second|evening|| +73286|AAAAAAAAHEOBBAAA|73286|20|21|26|PM|second|evening|| +73287|AAAAAAAAIEOBBAAA|73287|20|21|27|PM|second|evening|| +73288|AAAAAAAAJEOBBAAA|73288|20|21|28|PM|second|evening|| +73289|AAAAAAAAKEOBBAAA|73289|20|21|29|PM|second|evening|| +73290|AAAAAAAALEOBBAAA|73290|20|21|30|PM|second|evening|| +73291|AAAAAAAAMEOBBAAA|73291|20|21|31|PM|second|evening|| +73292|AAAAAAAANEOBBAAA|73292|20|21|32|PM|second|evening|| +73293|AAAAAAAAOEOBBAAA|73293|20|21|33|PM|second|evening|| +73294|AAAAAAAAPEOBBAAA|73294|20|21|34|PM|second|evening|| +73295|AAAAAAAAAFOBBAAA|73295|20|21|35|PM|second|evening|| +73296|AAAAAAAABFOBBAAA|73296|20|21|36|PM|second|evening|| +73297|AAAAAAAACFOBBAAA|73297|20|21|37|PM|second|evening|| +73298|AAAAAAAADFOBBAAA|73298|20|21|38|PM|second|evening|| +73299|AAAAAAAAEFOBBAAA|73299|20|21|39|PM|second|evening|| +73300|AAAAAAAAFFOBBAAA|73300|20|21|40|PM|second|evening|| +73301|AAAAAAAAGFOBBAAA|73301|20|21|41|PM|second|evening|| +73302|AAAAAAAAHFOBBAAA|73302|20|21|42|PM|second|evening|| +73303|AAAAAAAAIFOBBAAA|73303|20|21|43|PM|second|evening|| +73304|AAAAAAAAJFOBBAAA|73304|20|21|44|PM|second|evening|| +73305|AAAAAAAAKFOBBAAA|73305|20|21|45|PM|second|evening|| +73306|AAAAAAAALFOBBAAA|73306|20|21|46|PM|second|evening|| +73307|AAAAAAAAMFOBBAAA|73307|20|21|47|PM|second|evening|| +73308|AAAAAAAANFOBBAAA|73308|20|21|48|PM|second|evening|| +73309|AAAAAAAAOFOBBAAA|73309|20|21|49|PM|second|evening|| +73310|AAAAAAAAPFOBBAAA|73310|20|21|50|PM|second|evening|| +73311|AAAAAAAAAGOBBAAA|73311|20|21|51|PM|second|evening|| +73312|AAAAAAAABGOBBAAA|73312|20|21|52|PM|second|evening|| +73313|AAAAAAAACGOBBAAA|73313|20|21|53|PM|second|evening|| +73314|AAAAAAAADGOBBAAA|73314|20|21|54|PM|second|evening|| +73315|AAAAAAAAEGOBBAAA|73315|20|21|55|PM|second|evening|| +73316|AAAAAAAAFGOBBAAA|73316|20|21|56|PM|second|evening|| +73317|AAAAAAAAGGOBBAAA|73317|20|21|57|PM|second|evening|| +73318|AAAAAAAAHGOBBAAA|73318|20|21|58|PM|second|evening|| +73319|AAAAAAAAIGOBBAAA|73319|20|21|59|PM|second|evening|| +73320|AAAAAAAAJGOBBAAA|73320|20|22|0|PM|second|evening|| +73321|AAAAAAAAKGOBBAAA|73321|20|22|1|PM|second|evening|| +73322|AAAAAAAALGOBBAAA|73322|20|22|2|PM|second|evening|| +73323|AAAAAAAAMGOBBAAA|73323|20|22|3|PM|second|evening|| +73324|AAAAAAAANGOBBAAA|73324|20|22|4|PM|second|evening|| +73325|AAAAAAAAOGOBBAAA|73325|20|22|5|PM|second|evening|| +73326|AAAAAAAAPGOBBAAA|73326|20|22|6|PM|second|evening|| +73327|AAAAAAAAAHOBBAAA|73327|20|22|7|PM|second|evening|| +73328|AAAAAAAABHOBBAAA|73328|20|22|8|PM|second|evening|| +73329|AAAAAAAACHOBBAAA|73329|20|22|9|PM|second|evening|| +73330|AAAAAAAADHOBBAAA|73330|20|22|10|PM|second|evening|| +73331|AAAAAAAAEHOBBAAA|73331|20|22|11|PM|second|evening|| +73332|AAAAAAAAFHOBBAAA|73332|20|22|12|PM|second|evening|| +73333|AAAAAAAAGHOBBAAA|73333|20|22|13|PM|second|evening|| +73334|AAAAAAAAHHOBBAAA|73334|20|22|14|PM|second|evening|| +73335|AAAAAAAAIHOBBAAA|73335|20|22|15|PM|second|evening|| +73336|AAAAAAAAJHOBBAAA|73336|20|22|16|PM|second|evening|| +73337|AAAAAAAAKHOBBAAA|73337|20|22|17|PM|second|evening|| +73338|AAAAAAAALHOBBAAA|73338|20|22|18|PM|second|evening|| +73339|AAAAAAAAMHOBBAAA|73339|20|22|19|PM|second|evening|| +73340|AAAAAAAANHOBBAAA|73340|20|22|20|PM|second|evening|| +73341|AAAAAAAAOHOBBAAA|73341|20|22|21|PM|second|evening|| +73342|AAAAAAAAPHOBBAAA|73342|20|22|22|PM|second|evening|| +73343|AAAAAAAAAIOBBAAA|73343|20|22|23|PM|second|evening|| +73344|AAAAAAAABIOBBAAA|73344|20|22|24|PM|second|evening|| +73345|AAAAAAAACIOBBAAA|73345|20|22|25|PM|second|evening|| +73346|AAAAAAAADIOBBAAA|73346|20|22|26|PM|second|evening|| +73347|AAAAAAAAEIOBBAAA|73347|20|22|27|PM|second|evening|| +73348|AAAAAAAAFIOBBAAA|73348|20|22|28|PM|second|evening|| +73349|AAAAAAAAGIOBBAAA|73349|20|22|29|PM|second|evening|| +73350|AAAAAAAAHIOBBAAA|73350|20|22|30|PM|second|evening|| +73351|AAAAAAAAIIOBBAAA|73351|20|22|31|PM|second|evening|| +73352|AAAAAAAAJIOBBAAA|73352|20|22|32|PM|second|evening|| +73353|AAAAAAAAKIOBBAAA|73353|20|22|33|PM|second|evening|| +73354|AAAAAAAALIOBBAAA|73354|20|22|34|PM|second|evening|| +73355|AAAAAAAAMIOBBAAA|73355|20|22|35|PM|second|evening|| +73356|AAAAAAAANIOBBAAA|73356|20|22|36|PM|second|evening|| +73357|AAAAAAAAOIOBBAAA|73357|20|22|37|PM|second|evening|| +73358|AAAAAAAAPIOBBAAA|73358|20|22|38|PM|second|evening|| +73359|AAAAAAAAAJOBBAAA|73359|20|22|39|PM|second|evening|| +73360|AAAAAAAABJOBBAAA|73360|20|22|40|PM|second|evening|| +73361|AAAAAAAACJOBBAAA|73361|20|22|41|PM|second|evening|| +73362|AAAAAAAADJOBBAAA|73362|20|22|42|PM|second|evening|| +73363|AAAAAAAAEJOBBAAA|73363|20|22|43|PM|second|evening|| +73364|AAAAAAAAFJOBBAAA|73364|20|22|44|PM|second|evening|| +73365|AAAAAAAAGJOBBAAA|73365|20|22|45|PM|second|evening|| +73366|AAAAAAAAHJOBBAAA|73366|20|22|46|PM|second|evening|| +73367|AAAAAAAAIJOBBAAA|73367|20|22|47|PM|second|evening|| +73368|AAAAAAAAJJOBBAAA|73368|20|22|48|PM|second|evening|| +73369|AAAAAAAAKJOBBAAA|73369|20|22|49|PM|second|evening|| +73370|AAAAAAAALJOBBAAA|73370|20|22|50|PM|second|evening|| +73371|AAAAAAAAMJOBBAAA|73371|20|22|51|PM|second|evening|| +73372|AAAAAAAANJOBBAAA|73372|20|22|52|PM|second|evening|| +73373|AAAAAAAAOJOBBAAA|73373|20|22|53|PM|second|evening|| +73374|AAAAAAAAPJOBBAAA|73374|20|22|54|PM|second|evening|| +73375|AAAAAAAAAKOBBAAA|73375|20|22|55|PM|second|evening|| +73376|AAAAAAAABKOBBAAA|73376|20|22|56|PM|second|evening|| +73377|AAAAAAAACKOBBAAA|73377|20|22|57|PM|second|evening|| +73378|AAAAAAAADKOBBAAA|73378|20|22|58|PM|second|evening|| +73379|AAAAAAAAEKOBBAAA|73379|20|22|59|PM|second|evening|| +73380|AAAAAAAAFKOBBAAA|73380|20|23|0|PM|second|evening|| +73381|AAAAAAAAGKOBBAAA|73381|20|23|1|PM|second|evening|| +73382|AAAAAAAAHKOBBAAA|73382|20|23|2|PM|second|evening|| +73383|AAAAAAAAIKOBBAAA|73383|20|23|3|PM|second|evening|| +73384|AAAAAAAAJKOBBAAA|73384|20|23|4|PM|second|evening|| +73385|AAAAAAAAKKOBBAAA|73385|20|23|5|PM|second|evening|| +73386|AAAAAAAALKOBBAAA|73386|20|23|6|PM|second|evening|| +73387|AAAAAAAAMKOBBAAA|73387|20|23|7|PM|second|evening|| +73388|AAAAAAAANKOBBAAA|73388|20|23|8|PM|second|evening|| +73389|AAAAAAAAOKOBBAAA|73389|20|23|9|PM|second|evening|| +73390|AAAAAAAAPKOBBAAA|73390|20|23|10|PM|second|evening|| +73391|AAAAAAAAALOBBAAA|73391|20|23|11|PM|second|evening|| +73392|AAAAAAAABLOBBAAA|73392|20|23|12|PM|second|evening|| +73393|AAAAAAAACLOBBAAA|73393|20|23|13|PM|second|evening|| +73394|AAAAAAAADLOBBAAA|73394|20|23|14|PM|second|evening|| +73395|AAAAAAAAELOBBAAA|73395|20|23|15|PM|second|evening|| +73396|AAAAAAAAFLOBBAAA|73396|20|23|16|PM|second|evening|| +73397|AAAAAAAAGLOBBAAA|73397|20|23|17|PM|second|evening|| +73398|AAAAAAAAHLOBBAAA|73398|20|23|18|PM|second|evening|| +73399|AAAAAAAAILOBBAAA|73399|20|23|19|PM|second|evening|| +73400|AAAAAAAAJLOBBAAA|73400|20|23|20|PM|second|evening|| +73401|AAAAAAAAKLOBBAAA|73401|20|23|21|PM|second|evening|| +73402|AAAAAAAALLOBBAAA|73402|20|23|22|PM|second|evening|| +73403|AAAAAAAAMLOBBAAA|73403|20|23|23|PM|second|evening|| +73404|AAAAAAAANLOBBAAA|73404|20|23|24|PM|second|evening|| +73405|AAAAAAAAOLOBBAAA|73405|20|23|25|PM|second|evening|| +73406|AAAAAAAAPLOBBAAA|73406|20|23|26|PM|second|evening|| +73407|AAAAAAAAAMOBBAAA|73407|20|23|27|PM|second|evening|| +73408|AAAAAAAABMOBBAAA|73408|20|23|28|PM|second|evening|| +73409|AAAAAAAACMOBBAAA|73409|20|23|29|PM|second|evening|| +73410|AAAAAAAADMOBBAAA|73410|20|23|30|PM|second|evening|| +73411|AAAAAAAAEMOBBAAA|73411|20|23|31|PM|second|evening|| +73412|AAAAAAAAFMOBBAAA|73412|20|23|32|PM|second|evening|| +73413|AAAAAAAAGMOBBAAA|73413|20|23|33|PM|second|evening|| +73414|AAAAAAAAHMOBBAAA|73414|20|23|34|PM|second|evening|| +73415|AAAAAAAAIMOBBAAA|73415|20|23|35|PM|second|evening|| +73416|AAAAAAAAJMOBBAAA|73416|20|23|36|PM|second|evening|| +73417|AAAAAAAAKMOBBAAA|73417|20|23|37|PM|second|evening|| +73418|AAAAAAAALMOBBAAA|73418|20|23|38|PM|second|evening|| +73419|AAAAAAAAMMOBBAAA|73419|20|23|39|PM|second|evening|| +73420|AAAAAAAANMOBBAAA|73420|20|23|40|PM|second|evening|| +73421|AAAAAAAAOMOBBAAA|73421|20|23|41|PM|second|evening|| +73422|AAAAAAAAPMOBBAAA|73422|20|23|42|PM|second|evening|| +73423|AAAAAAAAANOBBAAA|73423|20|23|43|PM|second|evening|| +73424|AAAAAAAABNOBBAAA|73424|20|23|44|PM|second|evening|| +73425|AAAAAAAACNOBBAAA|73425|20|23|45|PM|second|evening|| +73426|AAAAAAAADNOBBAAA|73426|20|23|46|PM|second|evening|| +73427|AAAAAAAAENOBBAAA|73427|20|23|47|PM|second|evening|| +73428|AAAAAAAAFNOBBAAA|73428|20|23|48|PM|second|evening|| +73429|AAAAAAAAGNOBBAAA|73429|20|23|49|PM|second|evening|| +73430|AAAAAAAAHNOBBAAA|73430|20|23|50|PM|second|evening|| +73431|AAAAAAAAINOBBAAA|73431|20|23|51|PM|second|evening|| +73432|AAAAAAAAJNOBBAAA|73432|20|23|52|PM|second|evening|| +73433|AAAAAAAAKNOBBAAA|73433|20|23|53|PM|second|evening|| +73434|AAAAAAAALNOBBAAA|73434|20|23|54|PM|second|evening|| +73435|AAAAAAAAMNOBBAAA|73435|20|23|55|PM|second|evening|| +73436|AAAAAAAANNOBBAAA|73436|20|23|56|PM|second|evening|| +73437|AAAAAAAAONOBBAAA|73437|20|23|57|PM|second|evening|| +73438|AAAAAAAAPNOBBAAA|73438|20|23|58|PM|second|evening|| +73439|AAAAAAAAAOOBBAAA|73439|20|23|59|PM|second|evening|| +73440|AAAAAAAABOOBBAAA|73440|20|24|0|PM|second|evening|| +73441|AAAAAAAACOOBBAAA|73441|20|24|1|PM|second|evening|| +73442|AAAAAAAADOOBBAAA|73442|20|24|2|PM|second|evening|| +73443|AAAAAAAAEOOBBAAA|73443|20|24|3|PM|second|evening|| +73444|AAAAAAAAFOOBBAAA|73444|20|24|4|PM|second|evening|| +73445|AAAAAAAAGOOBBAAA|73445|20|24|5|PM|second|evening|| +73446|AAAAAAAAHOOBBAAA|73446|20|24|6|PM|second|evening|| +73447|AAAAAAAAIOOBBAAA|73447|20|24|7|PM|second|evening|| +73448|AAAAAAAAJOOBBAAA|73448|20|24|8|PM|second|evening|| +73449|AAAAAAAAKOOBBAAA|73449|20|24|9|PM|second|evening|| +73450|AAAAAAAALOOBBAAA|73450|20|24|10|PM|second|evening|| +73451|AAAAAAAAMOOBBAAA|73451|20|24|11|PM|second|evening|| +73452|AAAAAAAANOOBBAAA|73452|20|24|12|PM|second|evening|| +73453|AAAAAAAAOOOBBAAA|73453|20|24|13|PM|second|evening|| +73454|AAAAAAAAPOOBBAAA|73454|20|24|14|PM|second|evening|| +73455|AAAAAAAAAPOBBAAA|73455|20|24|15|PM|second|evening|| +73456|AAAAAAAABPOBBAAA|73456|20|24|16|PM|second|evening|| +73457|AAAAAAAACPOBBAAA|73457|20|24|17|PM|second|evening|| +73458|AAAAAAAADPOBBAAA|73458|20|24|18|PM|second|evening|| +73459|AAAAAAAAEPOBBAAA|73459|20|24|19|PM|second|evening|| +73460|AAAAAAAAFPOBBAAA|73460|20|24|20|PM|second|evening|| +73461|AAAAAAAAGPOBBAAA|73461|20|24|21|PM|second|evening|| +73462|AAAAAAAAHPOBBAAA|73462|20|24|22|PM|second|evening|| +73463|AAAAAAAAIPOBBAAA|73463|20|24|23|PM|second|evening|| +73464|AAAAAAAAJPOBBAAA|73464|20|24|24|PM|second|evening|| +73465|AAAAAAAAKPOBBAAA|73465|20|24|25|PM|second|evening|| +73466|AAAAAAAALPOBBAAA|73466|20|24|26|PM|second|evening|| +73467|AAAAAAAAMPOBBAAA|73467|20|24|27|PM|second|evening|| +73468|AAAAAAAANPOBBAAA|73468|20|24|28|PM|second|evening|| +73469|AAAAAAAAOPOBBAAA|73469|20|24|29|PM|second|evening|| +73470|AAAAAAAAPPOBBAAA|73470|20|24|30|PM|second|evening|| +73471|AAAAAAAAAAPBBAAA|73471|20|24|31|PM|second|evening|| +73472|AAAAAAAABAPBBAAA|73472|20|24|32|PM|second|evening|| +73473|AAAAAAAACAPBBAAA|73473|20|24|33|PM|second|evening|| +73474|AAAAAAAADAPBBAAA|73474|20|24|34|PM|second|evening|| +73475|AAAAAAAAEAPBBAAA|73475|20|24|35|PM|second|evening|| +73476|AAAAAAAAFAPBBAAA|73476|20|24|36|PM|second|evening|| +73477|AAAAAAAAGAPBBAAA|73477|20|24|37|PM|second|evening|| +73478|AAAAAAAAHAPBBAAA|73478|20|24|38|PM|second|evening|| +73479|AAAAAAAAIAPBBAAA|73479|20|24|39|PM|second|evening|| +73480|AAAAAAAAJAPBBAAA|73480|20|24|40|PM|second|evening|| +73481|AAAAAAAAKAPBBAAA|73481|20|24|41|PM|second|evening|| +73482|AAAAAAAALAPBBAAA|73482|20|24|42|PM|second|evening|| +73483|AAAAAAAAMAPBBAAA|73483|20|24|43|PM|second|evening|| +73484|AAAAAAAANAPBBAAA|73484|20|24|44|PM|second|evening|| +73485|AAAAAAAAOAPBBAAA|73485|20|24|45|PM|second|evening|| +73486|AAAAAAAAPAPBBAAA|73486|20|24|46|PM|second|evening|| +73487|AAAAAAAAABPBBAAA|73487|20|24|47|PM|second|evening|| +73488|AAAAAAAABBPBBAAA|73488|20|24|48|PM|second|evening|| +73489|AAAAAAAACBPBBAAA|73489|20|24|49|PM|second|evening|| +73490|AAAAAAAADBPBBAAA|73490|20|24|50|PM|second|evening|| +73491|AAAAAAAAEBPBBAAA|73491|20|24|51|PM|second|evening|| +73492|AAAAAAAAFBPBBAAA|73492|20|24|52|PM|second|evening|| +73493|AAAAAAAAGBPBBAAA|73493|20|24|53|PM|second|evening|| +73494|AAAAAAAAHBPBBAAA|73494|20|24|54|PM|second|evening|| +73495|AAAAAAAAIBPBBAAA|73495|20|24|55|PM|second|evening|| +73496|AAAAAAAAJBPBBAAA|73496|20|24|56|PM|second|evening|| +73497|AAAAAAAAKBPBBAAA|73497|20|24|57|PM|second|evening|| +73498|AAAAAAAALBPBBAAA|73498|20|24|58|PM|second|evening|| +73499|AAAAAAAAMBPBBAAA|73499|20|24|59|PM|second|evening|| +73500|AAAAAAAANBPBBAAA|73500|20|25|0|PM|second|evening|| +73501|AAAAAAAAOBPBBAAA|73501|20|25|1|PM|second|evening|| +73502|AAAAAAAAPBPBBAAA|73502|20|25|2|PM|second|evening|| +73503|AAAAAAAAACPBBAAA|73503|20|25|3|PM|second|evening|| +73504|AAAAAAAABCPBBAAA|73504|20|25|4|PM|second|evening|| +73505|AAAAAAAACCPBBAAA|73505|20|25|5|PM|second|evening|| +73506|AAAAAAAADCPBBAAA|73506|20|25|6|PM|second|evening|| +73507|AAAAAAAAECPBBAAA|73507|20|25|7|PM|second|evening|| +73508|AAAAAAAAFCPBBAAA|73508|20|25|8|PM|second|evening|| +73509|AAAAAAAAGCPBBAAA|73509|20|25|9|PM|second|evening|| +73510|AAAAAAAAHCPBBAAA|73510|20|25|10|PM|second|evening|| +73511|AAAAAAAAICPBBAAA|73511|20|25|11|PM|second|evening|| +73512|AAAAAAAAJCPBBAAA|73512|20|25|12|PM|second|evening|| +73513|AAAAAAAAKCPBBAAA|73513|20|25|13|PM|second|evening|| +73514|AAAAAAAALCPBBAAA|73514|20|25|14|PM|second|evening|| +73515|AAAAAAAAMCPBBAAA|73515|20|25|15|PM|second|evening|| +73516|AAAAAAAANCPBBAAA|73516|20|25|16|PM|second|evening|| +73517|AAAAAAAAOCPBBAAA|73517|20|25|17|PM|second|evening|| +73518|AAAAAAAAPCPBBAAA|73518|20|25|18|PM|second|evening|| +73519|AAAAAAAAADPBBAAA|73519|20|25|19|PM|second|evening|| +73520|AAAAAAAABDPBBAAA|73520|20|25|20|PM|second|evening|| +73521|AAAAAAAACDPBBAAA|73521|20|25|21|PM|second|evening|| +73522|AAAAAAAADDPBBAAA|73522|20|25|22|PM|second|evening|| +73523|AAAAAAAAEDPBBAAA|73523|20|25|23|PM|second|evening|| +73524|AAAAAAAAFDPBBAAA|73524|20|25|24|PM|second|evening|| +73525|AAAAAAAAGDPBBAAA|73525|20|25|25|PM|second|evening|| +73526|AAAAAAAAHDPBBAAA|73526|20|25|26|PM|second|evening|| +73527|AAAAAAAAIDPBBAAA|73527|20|25|27|PM|second|evening|| +73528|AAAAAAAAJDPBBAAA|73528|20|25|28|PM|second|evening|| +73529|AAAAAAAAKDPBBAAA|73529|20|25|29|PM|second|evening|| +73530|AAAAAAAALDPBBAAA|73530|20|25|30|PM|second|evening|| +73531|AAAAAAAAMDPBBAAA|73531|20|25|31|PM|second|evening|| +73532|AAAAAAAANDPBBAAA|73532|20|25|32|PM|second|evening|| +73533|AAAAAAAAODPBBAAA|73533|20|25|33|PM|second|evening|| +73534|AAAAAAAAPDPBBAAA|73534|20|25|34|PM|second|evening|| +73535|AAAAAAAAAEPBBAAA|73535|20|25|35|PM|second|evening|| +73536|AAAAAAAABEPBBAAA|73536|20|25|36|PM|second|evening|| +73537|AAAAAAAACEPBBAAA|73537|20|25|37|PM|second|evening|| +73538|AAAAAAAADEPBBAAA|73538|20|25|38|PM|second|evening|| +73539|AAAAAAAAEEPBBAAA|73539|20|25|39|PM|second|evening|| +73540|AAAAAAAAFEPBBAAA|73540|20|25|40|PM|second|evening|| +73541|AAAAAAAAGEPBBAAA|73541|20|25|41|PM|second|evening|| +73542|AAAAAAAAHEPBBAAA|73542|20|25|42|PM|second|evening|| +73543|AAAAAAAAIEPBBAAA|73543|20|25|43|PM|second|evening|| +73544|AAAAAAAAJEPBBAAA|73544|20|25|44|PM|second|evening|| +73545|AAAAAAAAKEPBBAAA|73545|20|25|45|PM|second|evening|| +73546|AAAAAAAALEPBBAAA|73546|20|25|46|PM|second|evening|| +73547|AAAAAAAAMEPBBAAA|73547|20|25|47|PM|second|evening|| +73548|AAAAAAAANEPBBAAA|73548|20|25|48|PM|second|evening|| +73549|AAAAAAAAOEPBBAAA|73549|20|25|49|PM|second|evening|| +73550|AAAAAAAAPEPBBAAA|73550|20|25|50|PM|second|evening|| +73551|AAAAAAAAAFPBBAAA|73551|20|25|51|PM|second|evening|| +73552|AAAAAAAABFPBBAAA|73552|20|25|52|PM|second|evening|| +73553|AAAAAAAACFPBBAAA|73553|20|25|53|PM|second|evening|| +73554|AAAAAAAADFPBBAAA|73554|20|25|54|PM|second|evening|| +73555|AAAAAAAAEFPBBAAA|73555|20|25|55|PM|second|evening|| +73556|AAAAAAAAFFPBBAAA|73556|20|25|56|PM|second|evening|| +73557|AAAAAAAAGFPBBAAA|73557|20|25|57|PM|second|evening|| +73558|AAAAAAAAHFPBBAAA|73558|20|25|58|PM|second|evening|| +73559|AAAAAAAAIFPBBAAA|73559|20|25|59|PM|second|evening|| +73560|AAAAAAAAJFPBBAAA|73560|20|26|0|PM|second|evening|| +73561|AAAAAAAAKFPBBAAA|73561|20|26|1|PM|second|evening|| +73562|AAAAAAAALFPBBAAA|73562|20|26|2|PM|second|evening|| +73563|AAAAAAAAMFPBBAAA|73563|20|26|3|PM|second|evening|| +73564|AAAAAAAANFPBBAAA|73564|20|26|4|PM|second|evening|| +73565|AAAAAAAAOFPBBAAA|73565|20|26|5|PM|second|evening|| +73566|AAAAAAAAPFPBBAAA|73566|20|26|6|PM|second|evening|| +73567|AAAAAAAAAGPBBAAA|73567|20|26|7|PM|second|evening|| +73568|AAAAAAAABGPBBAAA|73568|20|26|8|PM|second|evening|| +73569|AAAAAAAACGPBBAAA|73569|20|26|9|PM|second|evening|| +73570|AAAAAAAADGPBBAAA|73570|20|26|10|PM|second|evening|| +73571|AAAAAAAAEGPBBAAA|73571|20|26|11|PM|second|evening|| +73572|AAAAAAAAFGPBBAAA|73572|20|26|12|PM|second|evening|| +73573|AAAAAAAAGGPBBAAA|73573|20|26|13|PM|second|evening|| +73574|AAAAAAAAHGPBBAAA|73574|20|26|14|PM|second|evening|| +73575|AAAAAAAAIGPBBAAA|73575|20|26|15|PM|second|evening|| +73576|AAAAAAAAJGPBBAAA|73576|20|26|16|PM|second|evening|| +73577|AAAAAAAAKGPBBAAA|73577|20|26|17|PM|second|evening|| +73578|AAAAAAAALGPBBAAA|73578|20|26|18|PM|second|evening|| +73579|AAAAAAAAMGPBBAAA|73579|20|26|19|PM|second|evening|| +73580|AAAAAAAANGPBBAAA|73580|20|26|20|PM|second|evening|| +73581|AAAAAAAAOGPBBAAA|73581|20|26|21|PM|second|evening|| +73582|AAAAAAAAPGPBBAAA|73582|20|26|22|PM|second|evening|| +73583|AAAAAAAAAHPBBAAA|73583|20|26|23|PM|second|evening|| +73584|AAAAAAAABHPBBAAA|73584|20|26|24|PM|second|evening|| +73585|AAAAAAAACHPBBAAA|73585|20|26|25|PM|second|evening|| +73586|AAAAAAAADHPBBAAA|73586|20|26|26|PM|second|evening|| +73587|AAAAAAAAEHPBBAAA|73587|20|26|27|PM|second|evening|| +73588|AAAAAAAAFHPBBAAA|73588|20|26|28|PM|second|evening|| +73589|AAAAAAAAGHPBBAAA|73589|20|26|29|PM|second|evening|| +73590|AAAAAAAAHHPBBAAA|73590|20|26|30|PM|second|evening|| +73591|AAAAAAAAIHPBBAAA|73591|20|26|31|PM|second|evening|| +73592|AAAAAAAAJHPBBAAA|73592|20|26|32|PM|second|evening|| +73593|AAAAAAAAKHPBBAAA|73593|20|26|33|PM|second|evening|| +73594|AAAAAAAALHPBBAAA|73594|20|26|34|PM|second|evening|| +73595|AAAAAAAAMHPBBAAA|73595|20|26|35|PM|second|evening|| +73596|AAAAAAAANHPBBAAA|73596|20|26|36|PM|second|evening|| +73597|AAAAAAAAOHPBBAAA|73597|20|26|37|PM|second|evening|| +73598|AAAAAAAAPHPBBAAA|73598|20|26|38|PM|second|evening|| +73599|AAAAAAAAAIPBBAAA|73599|20|26|39|PM|second|evening|| +73600|AAAAAAAABIPBBAAA|73600|20|26|40|PM|second|evening|| +73601|AAAAAAAACIPBBAAA|73601|20|26|41|PM|second|evening|| +73602|AAAAAAAADIPBBAAA|73602|20|26|42|PM|second|evening|| +73603|AAAAAAAAEIPBBAAA|73603|20|26|43|PM|second|evening|| +73604|AAAAAAAAFIPBBAAA|73604|20|26|44|PM|second|evening|| +73605|AAAAAAAAGIPBBAAA|73605|20|26|45|PM|second|evening|| +73606|AAAAAAAAHIPBBAAA|73606|20|26|46|PM|second|evening|| +73607|AAAAAAAAIIPBBAAA|73607|20|26|47|PM|second|evening|| +73608|AAAAAAAAJIPBBAAA|73608|20|26|48|PM|second|evening|| +73609|AAAAAAAAKIPBBAAA|73609|20|26|49|PM|second|evening|| +73610|AAAAAAAALIPBBAAA|73610|20|26|50|PM|second|evening|| +73611|AAAAAAAAMIPBBAAA|73611|20|26|51|PM|second|evening|| +73612|AAAAAAAANIPBBAAA|73612|20|26|52|PM|second|evening|| +73613|AAAAAAAAOIPBBAAA|73613|20|26|53|PM|second|evening|| +73614|AAAAAAAAPIPBBAAA|73614|20|26|54|PM|second|evening|| +73615|AAAAAAAAAJPBBAAA|73615|20|26|55|PM|second|evening|| +73616|AAAAAAAABJPBBAAA|73616|20|26|56|PM|second|evening|| +73617|AAAAAAAACJPBBAAA|73617|20|26|57|PM|second|evening|| +73618|AAAAAAAADJPBBAAA|73618|20|26|58|PM|second|evening|| +73619|AAAAAAAAEJPBBAAA|73619|20|26|59|PM|second|evening|| +73620|AAAAAAAAFJPBBAAA|73620|20|27|0|PM|second|evening|| +73621|AAAAAAAAGJPBBAAA|73621|20|27|1|PM|second|evening|| +73622|AAAAAAAAHJPBBAAA|73622|20|27|2|PM|second|evening|| +73623|AAAAAAAAIJPBBAAA|73623|20|27|3|PM|second|evening|| +73624|AAAAAAAAJJPBBAAA|73624|20|27|4|PM|second|evening|| +73625|AAAAAAAAKJPBBAAA|73625|20|27|5|PM|second|evening|| +73626|AAAAAAAALJPBBAAA|73626|20|27|6|PM|second|evening|| +73627|AAAAAAAAMJPBBAAA|73627|20|27|7|PM|second|evening|| +73628|AAAAAAAANJPBBAAA|73628|20|27|8|PM|second|evening|| +73629|AAAAAAAAOJPBBAAA|73629|20|27|9|PM|second|evening|| +73630|AAAAAAAAPJPBBAAA|73630|20|27|10|PM|second|evening|| +73631|AAAAAAAAAKPBBAAA|73631|20|27|11|PM|second|evening|| +73632|AAAAAAAABKPBBAAA|73632|20|27|12|PM|second|evening|| +73633|AAAAAAAACKPBBAAA|73633|20|27|13|PM|second|evening|| +73634|AAAAAAAADKPBBAAA|73634|20|27|14|PM|second|evening|| +73635|AAAAAAAAEKPBBAAA|73635|20|27|15|PM|second|evening|| +73636|AAAAAAAAFKPBBAAA|73636|20|27|16|PM|second|evening|| +73637|AAAAAAAAGKPBBAAA|73637|20|27|17|PM|second|evening|| +73638|AAAAAAAAHKPBBAAA|73638|20|27|18|PM|second|evening|| +73639|AAAAAAAAIKPBBAAA|73639|20|27|19|PM|second|evening|| +73640|AAAAAAAAJKPBBAAA|73640|20|27|20|PM|second|evening|| +73641|AAAAAAAAKKPBBAAA|73641|20|27|21|PM|second|evening|| +73642|AAAAAAAALKPBBAAA|73642|20|27|22|PM|second|evening|| +73643|AAAAAAAAMKPBBAAA|73643|20|27|23|PM|second|evening|| +73644|AAAAAAAANKPBBAAA|73644|20|27|24|PM|second|evening|| +73645|AAAAAAAAOKPBBAAA|73645|20|27|25|PM|second|evening|| +73646|AAAAAAAAPKPBBAAA|73646|20|27|26|PM|second|evening|| +73647|AAAAAAAAALPBBAAA|73647|20|27|27|PM|second|evening|| +73648|AAAAAAAABLPBBAAA|73648|20|27|28|PM|second|evening|| +73649|AAAAAAAACLPBBAAA|73649|20|27|29|PM|second|evening|| +73650|AAAAAAAADLPBBAAA|73650|20|27|30|PM|second|evening|| +73651|AAAAAAAAELPBBAAA|73651|20|27|31|PM|second|evening|| +73652|AAAAAAAAFLPBBAAA|73652|20|27|32|PM|second|evening|| +73653|AAAAAAAAGLPBBAAA|73653|20|27|33|PM|second|evening|| +73654|AAAAAAAAHLPBBAAA|73654|20|27|34|PM|second|evening|| +73655|AAAAAAAAILPBBAAA|73655|20|27|35|PM|second|evening|| +73656|AAAAAAAAJLPBBAAA|73656|20|27|36|PM|second|evening|| +73657|AAAAAAAAKLPBBAAA|73657|20|27|37|PM|second|evening|| +73658|AAAAAAAALLPBBAAA|73658|20|27|38|PM|second|evening|| +73659|AAAAAAAAMLPBBAAA|73659|20|27|39|PM|second|evening|| +73660|AAAAAAAANLPBBAAA|73660|20|27|40|PM|second|evening|| +73661|AAAAAAAAOLPBBAAA|73661|20|27|41|PM|second|evening|| +73662|AAAAAAAAPLPBBAAA|73662|20|27|42|PM|second|evening|| +73663|AAAAAAAAAMPBBAAA|73663|20|27|43|PM|second|evening|| +73664|AAAAAAAABMPBBAAA|73664|20|27|44|PM|second|evening|| +73665|AAAAAAAACMPBBAAA|73665|20|27|45|PM|second|evening|| +73666|AAAAAAAADMPBBAAA|73666|20|27|46|PM|second|evening|| +73667|AAAAAAAAEMPBBAAA|73667|20|27|47|PM|second|evening|| +73668|AAAAAAAAFMPBBAAA|73668|20|27|48|PM|second|evening|| +73669|AAAAAAAAGMPBBAAA|73669|20|27|49|PM|second|evening|| +73670|AAAAAAAAHMPBBAAA|73670|20|27|50|PM|second|evening|| +73671|AAAAAAAAIMPBBAAA|73671|20|27|51|PM|second|evening|| +73672|AAAAAAAAJMPBBAAA|73672|20|27|52|PM|second|evening|| +73673|AAAAAAAAKMPBBAAA|73673|20|27|53|PM|second|evening|| +73674|AAAAAAAALMPBBAAA|73674|20|27|54|PM|second|evening|| +73675|AAAAAAAAMMPBBAAA|73675|20|27|55|PM|second|evening|| +73676|AAAAAAAANMPBBAAA|73676|20|27|56|PM|second|evening|| +73677|AAAAAAAAOMPBBAAA|73677|20|27|57|PM|second|evening|| +73678|AAAAAAAAPMPBBAAA|73678|20|27|58|PM|second|evening|| +73679|AAAAAAAAANPBBAAA|73679|20|27|59|PM|second|evening|| +73680|AAAAAAAABNPBBAAA|73680|20|28|0|PM|second|evening|| +73681|AAAAAAAACNPBBAAA|73681|20|28|1|PM|second|evening|| +73682|AAAAAAAADNPBBAAA|73682|20|28|2|PM|second|evening|| +73683|AAAAAAAAENPBBAAA|73683|20|28|3|PM|second|evening|| +73684|AAAAAAAAFNPBBAAA|73684|20|28|4|PM|second|evening|| +73685|AAAAAAAAGNPBBAAA|73685|20|28|5|PM|second|evening|| +73686|AAAAAAAAHNPBBAAA|73686|20|28|6|PM|second|evening|| +73687|AAAAAAAAINPBBAAA|73687|20|28|7|PM|second|evening|| +73688|AAAAAAAAJNPBBAAA|73688|20|28|8|PM|second|evening|| +73689|AAAAAAAAKNPBBAAA|73689|20|28|9|PM|second|evening|| +73690|AAAAAAAALNPBBAAA|73690|20|28|10|PM|second|evening|| +73691|AAAAAAAAMNPBBAAA|73691|20|28|11|PM|second|evening|| +73692|AAAAAAAANNPBBAAA|73692|20|28|12|PM|second|evening|| +73693|AAAAAAAAONPBBAAA|73693|20|28|13|PM|second|evening|| +73694|AAAAAAAAPNPBBAAA|73694|20|28|14|PM|second|evening|| +73695|AAAAAAAAAOPBBAAA|73695|20|28|15|PM|second|evening|| +73696|AAAAAAAABOPBBAAA|73696|20|28|16|PM|second|evening|| +73697|AAAAAAAACOPBBAAA|73697|20|28|17|PM|second|evening|| +73698|AAAAAAAADOPBBAAA|73698|20|28|18|PM|second|evening|| +73699|AAAAAAAAEOPBBAAA|73699|20|28|19|PM|second|evening|| +73700|AAAAAAAAFOPBBAAA|73700|20|28|20|PM|second|evening|| +73701|AAAAAAAAGOPBBAAA|73701|20|28|21|PM|second|evening|| +73702|AAAAAAAAHOPBBAAA|73702|20|28|22|PM|second|evening|| +73703|AAAAAAAAIOPBBAAA|73703|20|28|23|PM|second|evening|| +73704|AAAAAAAAJOPBBAAA|73704|20|28|24|PM|second|evening|| +73705|AAAAAAAAKOPBBAAA|73705|20|28|25|PM|second|evening|| +73706|AAAAAAAALOPBBAAA|73706|20|28|26|PM|second|evening|| +73707|AAAAAAAAMOPBBAAA|73707|20|28|27|PM|second|evening|| +73708|AAAAAAAANOPBBAAA|73708|20|28|28|PM|second|evening|| +73709|AAAAAAAAOOPBBAAA|73709|20|28|29|PM|second|evening|| +73710|AAAAAAAAPOPBBAAA|73710|20|28|30|PM|second|evening|| +73711|AAAAAAAAAPPBBAAA|73711|20|28|31|PM|second|evening|| +73712|AAAAAAAABPPBBAAA|73712|20|28|32|PM|second|evening|| +73713|AAAAAAAACPPBBAAA|73713|20|28|33|PM|second|evening|| +73714|AAAAAAAADPPBBAAA|73714|20|28|34|PM|second|evening|| +73715|AAAAAAAAEPPBBAAA|73715|20|28|35|PM|second|evening|| +73716|AAAAAAAAFPPBBAAA|73716|20|28|36|PM|second|evening|| +73717|AAAAAAAAGPPBBAAA|73717|20|28|37|PM|second|evening|| +73718|AAAAAAAAHPPBBAAA|73718|20|28|38|PM|second|evening|| +73719|AAAAAAAAIPPBBAAA|73719|20|28|39|PM|second|evening|| +73720|AAAAAAAAJPPBBAAA|73720|20|28|40|PM|second|evening|| +73721|AAAAAAAAKPPBBAAA|73721|20|28|41|PM|second|evening|| +73722|AAAAAAAALPPBBAAA|73722|20|28|42|PM|second|evening|| +73723|AAAAAAAAMPPBBAAA|73723|20|28|43|PM|second|evening|| +73724|AAAAAAAANPPBBAAA|73724|20|28|44|PM|second|evening|| +73725|AAAAAAAAOPPBBAAA|73725|20|28|45|PM|second|evening|| +73726|AAAAAAAAPPPBBAAA|73726|20|28|46|PM|second|evening|| +73727|AAAAAAAAAAACBAAA|73727|20|28|47|PM|second|evening|| +73728|AAAAAAAABAACBAAA|73728|20|28|48|PM|second|evening|| +73729|AAAAAAAACAACBAAA|73729|20|28|49|PM|second|evening|| +73730|AAAAAAAADAACBAAA|73730|20|28|50|PM|second|evening|| +73731|AAAAAAAAEAACBAAA|73731|20|28|51|PM|second|evening|| +73732|AAAAAAAAFAACBAAA|73732|20|28|52|PM|second|evening|| +73733|AAAAAAAAGAACBAAA|73733|20|28|53|PM|second|evening|| +73734|AAAAAAAAHAACBAAA|73734|20|28|54|PM|second|evening|| +73735|AAAAAAAAIAACBAAA|73735|20|28|55|PM|second|evening|| +73736|AAAAAAAAJAACBAAA|73736|20|28|56|PM|second|evening|| +73737|AAAAAAAAKAACBAAA|73737|20|28|57|PM|second|evening|| +73738|AAAAAAAALAACBAAA|73738|20|28|58|PM|second|evening|| +73739|AAAAAAAAMAACBAAA|73739|20|28|59|PM|second|evening|| +73740|AAAAAAAANAACBAAA|73740|20|29|0|PM|second|evening|| +73741|AAAAAAAAOAACBAAA|73741|20|29|1|PM|second|evening|| +73742|AAAAAAAAPAACBAAA|73742|20|29|2|PM|second|evening|| +73743|AAAAAAAAABACBAAA|73743|20|29|3|PM|second|evening|| +73744|AAAAAAAABBACBAAA|73744|20|29|4|PM|second|evening|| +73745|AAAAAAAACBACBAAA|73745|20|29|5|PM|second|evening|| +73746|AAAAAAAADBACBAAA|73746|20|29|6|PM|second|evening|| +73747|AAAAAAAAEBACBAAA|73747|20|29|7|PM|second|evening|| +73748|AAAAAAAAFBACBAAA|73748|20|29|8|PM|second|evening|| +73749|AAAAAAAAGBACBAAA|73749|20|29|9|PM|second|evening|| +73750|AAAAAAAAHBACBAAA|73750|20|29|10|PM|second|evening|| +73751|AAAAAAAAIBACBAAA|73751|20|29|11|PM|second|evening|| +73752|AAAAAAAAJBACBAAA|73752|20|29|12|PM|second|evening|| +73753|AAAAAAAAKBACBAAA|73753|20|29|13|PM|second|evening|| +73754|AAAAAAAALBACBAAA|73754|20|29|14|PM|second|evening|| +73755|AAAAAAAAMBACBAAA|73755|20|29|15|PM|second|evening|| +73756|AAAAAAAANBACBAAA|73756|20|29|16|PM|second|evening|| +73757|AAAAAAAAOBACBAAA|73757|20|29|17|PM|second|evening|| +73758|AAAAAAAAPBACBAAA|73758|20|29|18|PM|second|evening|| +73759|AAAAAAAAACACBAAA|73759|20|29|19|PM|second|evening|| +73760|AAAAAAAABCACBAAA|73760|20|29|20|PM|second|evening|| +73761|AAAAAAAACCACBAAA|73761|20|29|21|PM|second|evening|| +73762|AAAAAAAADCACBAAA|73762|20|29|22|PM|second|evening|| +73763|AAAAAAAAECACBAAA|73763|20|29|23|PM|second|evening|| +73764|AAAAAAAAFCACBAAA|73764|20|29|24|PM|second|evening|| +73765|AAAAAAAAGCACBAAA|73765|20|29|25|PM|second|evening|| +73766|AAAAAAAAHCACBAAA|73766|20|29|26|PM|second|evening|| +73767|AAAAAAAAICACBAAA|73767|20|29|27|PM|second|evening|| +73768|AAAAAAAAJCACBAAA|73768|20|29|28|PM|second|evening|| +73769|AAAAAAAAKCACBAAA|73769|20|29|29|PM|second|evening|| +73770|AAAAAAAALCACBAAA|73770|20|29|30|PM|second|evening|| +73771|AAAAAAAAMCACBAAA|73771|20|29|31|PM|second|evening|| +73772|AAAAAAAANCACBAAA|73772|20|29|32|PM|second|evening|| +73773|AAAAAAAAOCACBAAA|73773|20|29|33|PM|second|evening|| +73774|AAAAAAAAPCACBAAA|73774|20|29|34|PM|second|evening|| +73775|AAAAAAAAADACBAAA|73775|20|29|35|PM|second|evening|| +73776|AAAAAAAABDACBAAA|73776|20|29|36|PM|second|evening|| +73777|AAAAAAAACDACBAAA|73777|20|29|37|PM|second|evening|| +73778|AAAAAAAADDACBAAA|73778|20|29|38|PM|second|evening|| +73779|AAAAAAAAEDACBAAA|73779|20|29|39|PM|second|evening|| +73780|AAAAAAAAFDACBAAA|73780|20|29|40|PM|second|evening|| +73781|AAAAAAAAGDACBAAA|73781|20|29|41|PM|second|evening|| +73782|AAAAAAAAHDACBAAA|73782|20|29|42|PM|second|evening|| +73783|AAAAAAAAIDACBAAA|73783|20|29|43|PM|second|evening|| +73784|AAAAAAAAJDACBAAA|73784|20|29|44|PM|second|evening|| +73785|AAAAAAAAKDACBAAA|73785|20|29|45|PM|second|evening|| +73786|AAAAAAAALDACBAAA|73786|20|29|46|PM|second|evening|| +73787|AAAAAAAAMDACBAAA|73787|20|29|47|PM|second|evening|| +73788|AAAAAAAANDACBAAA|73788|20|29|48|PM|second|evening|| +73789|AAAAAAAAODACBAAA|73789|20|29|49|PM|second|evening|| +73790|AAAAAAAAPDACBAAA|73790|20|29|50|PM|second|evening|| +73791|AAAAAAAAAEACBAAA|73791|20|29|51|PM|second|evening|| +73792|AAAAAAAABEACBAAA|73792|20|29|52|PM|second|evening|| +73793|AAAAAAAACEACBAAA|73793|20|29|53|PM|second|evening|| +73794|AAAAAAAADEACBAAA|73794|20|29|54|PM|second|evening|| +73795|AAAAAAAAEEACBAAA|73795|20|29|55|PM|second|evening|| +73796|AAAAAAAAFEACBAAA|73796|20|29|56|PM|second|evening|| +73797|AAAAAAAAGEACBAAA|73797|20|29|57|PM|second|evening|| +73798|AAAAAAAAHEACBAAA|73798|20|29|58|PM|second|evening|| +73799|AAAAAAAAIEACBAAA|73799|20|29|59|PM|second|evening|| +73800|AAAAAAAAJEACBAAA|73800|20|30|0|PM|second|evening|| +73801|AAAAAAAAKEACBAAA|73801|20|30|1|PM|second|evening|| +73802|AAAAAAAALEACBAAA|73802|20|30|2|PM|second|evening|| +73803|AAAAAAAAMEACBAAA|73803|20|30|3|PM|second|evening|| +73804|AAAAAAAANEACBAAA|73804|20|30|4|PM|second|evening|| +73805|AAAAAAAAOEACBAAA|73805|20|30|5|PM|second|evening|| +73806|AAAAAAAAPEACBAAA|73806|20|30|6|PM|second|evening|| +73807|AAAAAAAAAFACBAAA|73807|20|30|7|PM|second|evening|| +73808|AAAAAAAABFACBAAA|73808|20|30|8|PM|second|evening|| +73809|AAAAAAAACFACBAAA|73809|20|30|9|PM|second|evening|| +73810|AAAAAAAADFACBAAA|73810|20|30|10|PM|second|evening|| +73811|AAAAAAAAEFACBAAA|73811|20|30|11|PM|second|evening|| +73812|AAAAAAAAFFACBAAA|73812|20|30|12|PM|second|evening|| +73813|AAAAAAAAGFACBAAA|73813|20|30|13|PM|second|evening|| +73814|AAAAAAAAHFACBAAA|73814|20|30|14|PM|second|evening|| +73815|AAAAAAAAIFACBAAA|73815|20|30|15|PM|second|evening|| +73816|AAAAAAAAJFACBAAA|73816|20|30|16|PM|second|evening|| +73817|AAAAAAAAKFACBAAA|73817|20|30|17|PM|second|evening|| +73818|AAAAAAAALFACBAAA|73818|20|30|18|PM|second|evening|| +73819|AAAAAAAAMFACBAAA|73819|20|30|19|PM|second|evening|| +73820|AAAAAAAANFACBAAA|73820|20|30|20|PM|second|evening|| +73821|AAAAAAAAOFACBAAA|73821|20|30|21|PM|second|evening|| +73822|AAAAAAAAPFACBAAA|73822|20|30|22|PM|second|evening|| +73823|AAAAAAAAAGACBAAA|73823|20|30|23|PM|second|evening|| +73824|AAAAAAAABGACBAAA|73824|20|30|24|PM|second|evening|| +73825|AAAAAAAACGACBAAA|73825|20|30|25|PM|second|evening|| +73826|AAAAAAAADGACBAAA|73826|20|30|26|PM|second|evening|| +73827|AAAAAAAAEGACBAAA|73827|20|30|27|PM|second|evening|| +73828|AAAAAAAAFGACBAAA|73828|20|30|28|PM|second|evening|| +73829|AAAAAAAAGGACBAAA|73829|20|30|29|PM|second|evening|| +73830|AAAAAAAAHGACBAAA|73830|20|30|30|PM|second|evening|| +73831|AAAAAAAAIGACBAAA|73831|20|30|31|PM|second|evening|| +73832|AAAAAAAAJGACBAAA|73832|20|30|32|PM|second|evening|| +73833|AAAAAAAAKGACBAAA|73833|20|30|33|PM|second|evening|| +73834|AAAAAAAALGACBAAA|73834|20|30|34|PM|second|evening|| +73835|AAAAAAAAMGACBAAA|73835|20|30|35|PM|second|evening|| +73836|AAAAAAAANGACBAAA|73836|20|30|36|PM|second|evening|| +73837|AAAAAAAAOGACBAAA|73837|20|30|37|PM|second|evening|| +73838|AAAAAAAAPGACBAAA|73838|20|30|38|PM|second|evening|| +73839|AAAAAAAAAHACBAAA|73839|20|30|39|PM|second|evening|| +73840|AAAAAAAABHACBAAA|73840|20|30|40|PM|second|evening|| +73841|AAAAAAAACHACBAAA|73841|20|30|41|PM|second|evening|| +73842|AAAAAAAADHACBAAA|73842|20|30|42|PM|second|evening|| +73843|AAAAAAAAEHACBAAA|73843|20|30|43|PM|second|evening|| +73844|AAAAAAAAFHACBAAA|73844|20|30|44|PM|second|evening|| +73845|AAAAAAAAGHACBAAA|73845|20|30|45|PM|second|evening|| +73846|AAAAAAAAHHACBAAA|73846|20|30|46|PM|second|evening|| +73847|AAAAAAAAIHACBAAA|73847|20|30|47|PM|second|evening|| +73848|AAAAAAAAJHACBAAA|73848|20|30|48|PM|second|evening|| +73849|AAAAAAAAKHACBAAA|73849|20|30|49|PM|second|evening|| +73850|AAAAAAAALHACBAAA|73850|20|30|50|PM|second|evening|| +73851|AAAAAAAAMHACBAAA|73851|20|30|51|PM|second|evening|| +73852|AAAAAAAANHACBAAA|73852|20|30|52|PM|second|evening|| +73853|AAAAAAAAOHACBAAA|73853|20|30|53|PM|second|evening|| +73854|AAAAAAAAPHACBAAA|73854|20|30|54|PM|second|evening|| +73855|AAAAAAAAAIACBAAA|73855|20|30|55|PM|second|evening|| +73856|AAAAAAAABIACBAAA|73856|20|30|56|PM|second|evening|| +73857|AAAAAAAACIACBAAA|73857|20|30|57|PM|second|evening|| +73858|AAAAAAAADIACBAAA|73858|20|30|58|PM|second|evening|| +73859|AAAAAAAAEIACBAAA|73859|20|30|59|PM|second|evening|| +73860|AAAAAAAAFIACBAAA|73860|20|31|0|PM|second|evening|| +73861|AAAAAAAAGIACBAAA|73861|20|31|1|PM|second|evening|| +73862|AAAAAAAAHIACBAAA|73862|20|31|2|PM|second|evening|| +73863|AAAAAAAAIIACBAAA|73863|20|31|3|PM|second|evening|| +73864|AAAAAAAAJIACBAAA|73864|20|31|4|PM|second|evening|| +73865|AAAAAAAAKIACBAAA|73865|20|31|5|PM|second|evening|| +73866|AAAAAAAALIACBAAA|73866|20|31|6|PM|second|evening|| +73867|AAAAAAAAMIACBAAA|73867|20|31|7|PM|second|evening|| +73868|AAAAAAAANIACBAAA|73868|20|31|8|PM|second|evening|| +73869|AAAAAAAAOIACBAAA|73869|20|31|9|PM|second|evening|| +73870|AAAAAAAAPIACBAAA|73870|20|31|10|PM|second|evening|| +73871|AAAAAAAAAJACBAAA|73871|20|31|11|PM|second|evening|| +73872|AAAAAAAABJACBAAA|73872|20|31|12|PM|second|evening|| +73873|AAAAAAAACJACBAAA|73873|20|31|13|PM|second|evening|| +73874|AAAAAAAADJACBAAA|73874|20|31|14|PM|second|evening|| +73875|AAAAAAAAEJACBAAA|73875|20|31|15|PM|second|evening|| +73876|AAAAAAAAFJACBAAA|73876|20|31|16|PM|second|evening|| +73877|AAAAAAAAGJACBAAA|73877|20|31|17|PM|second|evening|| +73878|AAAAAAAAHJACBAAA|73878|20|31|18|PM|second|evening|| +73879|AAAAAAAAIJACBAAA|73879|20|31|19|PM|second|evening|| +73880|AAAAAAAAJJACBAAA|73880|20|31|20|PM|second|evening|| +73881|AAAAAAAAKJACBAAA|73881|20|31|21|PM|second|evening|| +73882|AAAAAAAALJACBAAA|73882|20|31|22|PM|second|evening|| +73883|AAAAAAAAMJACBAAA|73883|20|31|23|PM|second|evening|| +73884|AAAAAAAANJACBAAA|73884|20|31|24|PM|second|evening|| +73885|AAAAAAAAOJACBAAA|73885|20|31|25|PM|second|evening|| +73886|AAAAAAAAPJACBAAA|73886|20|31|26|PM|second|evening|| +73887|AAAAAAAAAKACBAAA|73887|20|31|27|PM|second|evening|| +73888|AAAAAAAABKACBAAA|73888|20|31|28|PM|second|evening|| +73889|AAAAAAAACKACBAAA|73889|20|31|29|PM|second|evening|| +73890|AAAAAAAADKACBAAA|73890|20|31|30|PM|second|evening|| +73891|AAAAAAAAEKACBAAA|73891|20|31|31|PM|second|evening|| +73892|AAAAAAAAFKACBAAA|73892|20|31|32|PM|second|evening|| +73893|AAAAAAAAGKACBAAA|73893|20|31|33|PM|second|evening|| +73894|AAAAAAAAHKACBAAA|73894|20|31|34|PM|second|evening|| +73895|AAAAAAAAIKACBAAA|73895|20|31|35|PM|second|evening|| +73896|AAAAAAAAJKACBAAA|73896|20|31|36|PM|second|evening|| +73897|AAAAAAAAKKACBAAA|73897|20|31|37|PM|second|evening|| +73898|AAAAAAAALKACBAAA|73898|20|31|38|PM|second|evening|| +73899|AAAAAAAAMKACBAAA|73899|20|31|39|PM|second|evening|| +73900|AAAAAAAANKACBAAA|73900|20|31|40|PM|second|evening|| +73901|AAAAAAAAOKACBAAA|73901|20|31|41|PM|second|evening|| +73902|AAAAAAAAPKACBAAA|73902|20|31|42|PM|second|evening|| +73903|AAAAAAAAALACBAAA|73903|20|31|43|PM|second|evening|| +73904|AAAAAAAABLACBAAA|73904|20|31|44|PM|second|evening|| +73905|AAAAAAAACLACBAAA|73905|20|31|45|PM|second|evening|| +73906|AAAAAAAADLACBAAA|73906|20|31|46|PM|second|evening|| +73907|AAAAAAAAELACBAAA|73907|20|31|47|PM|second|evening|| +73908|AAAAAAAAFLACBAAA|73908|20|31|48|PM|second|evening|| +73909|AAAAAAAAGLACBAAA|73909|20|31|49|PM|second|evening|| +73910|AAAAAAAAHLACBAAA|73910|20|31|50|PM|second|evening|| +73911|AAAAAAAAILACBAAA|73911|20|31|51|PM|second|evening|| +73912|AAAAAAAAJLACBAAA|73912|20|31|52|PM|second|evening|| +73913|AAAAAAAAKLACBAAA|73913|20|31|53|PM|second|evening|| +73914|AAAAAAAALLACBAAA|73914|20|31|54|PM|second|evening|| +73915|AAAAAAAAMLACBAAA|73915|20|31|55|PM|second|evening|| +73916|AAAAAAAANLACBAAA|73916|20|31|56|PM|second|evening|| +73917|AAAAAAAAOLACBAAA|73917|20|31|57|PM|second|evening|| +73918|AAAAAAAAPLACBAAA|73918|20|31|58|PM|second|evening|| +73919|AAAAAAAAAMACBAAA|73919|20|31|59|PM|second|evening|| +73920|AAAAAAAABMACBAAA|73920|20|32|0|PM|second|evening|| +73921|AAAAAAAACMACBAAA|73921|20|32|1|PM|second|evening|| +73922|AAAAAAAADMACBAAA|73922|20|32|2|PM|second|evening|| +73923|AAAAAAAAEMACBAAA|73923|20|32|3|PM|second|evening|| +73924|AAAAAAAAFMACBAAA|73924|20|32|4|PM|second|evening|| +73925|AAAAAAAAGMACBAAA|73925|20|32|5|PM|second|evening|| +73926|AAAAAAAAHMACBAAA|73926|20|32|6|PM|second|evening|| +73927|AAAAAAAAIMACBAAA|73927|20|32|7|PM|second|evening|| +73928|AAAAAAAAJMACBAAA|73928|20|32|8|PM|second|evening|| +73929|AAAAAAAAKMACBAAA|73929|20|32|9|PM|second|evening|| +73930|AAAAAAAALMACBAAA|73930|20|32|10|PM|second|evening|| +73931|AAAAAAAAMMACBAAA|73931|20|32|11|PM|second|evening|| +73932|AAAAAAAANMACBAAA|73932|20|32|12|PM|second|evening|| +73933|AAAAAAAAOMACBAAA|73933|20|32|13|PM|second|evening|| +73934|AAAAAAAAPMACBAAA|73934|20|32|14|PM|second|evening|| +73935|AAAAAAAAANACBAAA|73935|20|32|15|PM|second|evening|| +73936|AAAAAAAABNACBAAA|73936|20|32|16|PM|second|evening|| +73937|AAAAAAAACNACBAAA|73937|20|32|17|PM|second|evening|| +73938|AAAAAAAADNACBAAA|73938|20|32|18|PM|second|evening|| +73939|AAAAAAAAENACBAAA|73939|20|32|19|PM|second|evening|| +73940|AAAAAAAAFNACBAAA|73940|20|32|20|PM|second|evening|| +73941|AAAAAAAAGNACBAAA|73941|20|32|21|PM|second|evening|| +73942|AAAAAAAAHNACBAAA|73942|20|32|22|PM|second|evening|| +73943|AAAAAAAAINACBAAA|73943|20|32|23|PM|second|evening|| +73944|AAAAAAAAJNACBAAA|73944|20|32|24|PM|second|evening|| +73945|AAAAAAAAKNACBAAA|73945|20|32|25|PM|second|evening|| +73946|AAAAAAAALNACBAAA|73946|20|32|26|PM|second|evening|| +73947|AAAAAAAAMNACBAAA|73947|20|32|27|PM|second|evening|| +73948|AAAAAAAANNACBAAA|73948|20|32|28|PM|second|evening|| +73949|AAAAAAAAONACBAAA|73949|20|32|29|PM|second|evening|| +73950|AAAAAAAAPNACBAAA|73950|20|32|30|PM|second|evening|| +73951|AAAAAAAAAOACBAAA|73951|20|32|31|PM|second|evening|| +73952|AAAAAAAABOACBAAA|73952|20|32|32|PM|second|evening|| +73953|AAAAAAAACOACBAAA|73953|20|32|33|PM|second|evening|| +73954|AAAAAAAADOACBAAA|73954|20|32|34|PM|second|evening|| +73955|AAAAAAAAEOACBAAA|73955|20|32|35|PM|second|evening|| +73956|AAAAAAAAFOACBAAA|73956|20|32|36|PM|second|evening|| +73957|AAAAAAAAGOACBAAA|73957|20|32|37|PM|second|evening|| +73958|AAAAAAAAHOACBAAA|73958|20|32|38|PM|second|evening|| +73959|AAAAAAAAIOACBAAA|73959|20|32|39|PM|second|evening|| +73960|AAAAAAAAJOACBAAA|73960|20|32|40|PM|second|evening|| +73961|AAAAAAAAKOACBAAA|73961|20|32|41|PM|second|evening|| +73962|AAAAAAAALOACBAAA|73962|20|32|42|PM|second|evening|| +73963|AAAAAAAAMOACBAAA|73963|20|32|43|PM|second|evening|| +73964|AAAAAAAANOACBAAA|73964|20|32|44|PM|second|evening|| +73965|AAAAAAAAOOACBAAA|73965|20|32|45|PM|second|evening|| +73966|AAAAAAAAPOACBAAA|73966|20|32|46|PM|second|evening|| +73967|AAAAAAAAAPACBAAA|73967|20|32|47|PM|second|evening|| +73968|AAAAAAAABPACBAAA|73968|20|32|48|PM|second|evening|| +73969|AAAAAAAACPACBAAA|73969|20|32|49|PM|second|evening|| +73970|AAAAAAAADPACBAAA|73970|20|32|50|PM|second|evening|| +73971|AAAAAAAAEPACBAAA|73971|20|32|51|PM|second|evening|| +73972|AAAAAAAAFPACBAAA|73972|20|32|52|PM|second|evening|| +73973|AAAAAAAAGPACBAAA|73973|20|32|53|PM|second|evening|| +73974|AAAAAAAAHPACBAAA|73974|20|32|54|PM|second|evening|| +73975|AAAAAAAAIPACBAAA|73975|20|32|55|PM|second|evening|| +73976|AAAAAAAAJPACBAAA|73976|20|32|56|PM|second|evening|| +73977|AAAAAAAAKPACBAAA|73977|20|32|57|PM|second|evening|| +73978|AAAAAAAALPACBAAA|73978|20|32|58|PM|second|evening|| +73979|AAAAAAAAMPACBAAA|73979|20|32|59|PM|second|evening|| +73980|AAAAAAAANPACBAAA|73980|20|33|0|PM|second|evening|| +73981|AAAAAAAAOPACBAAA|73981|20|33|1|PM|second|evening|| +73982|AAAAAAAAPPACBAAA|73982|20|33|2|PM|second|evening|| +73983|AAAAAAAAAABCBAAA|73983|20|33|3|PM|second|evening|| +73984|AAAAAAAABABCBAAA|73984|20|33|4|PM|second|evening|| +73985|AAAAAAAACABCBAAA|73985|20|33|5|PM|second|evening|| +73986|AAAAAAAADABCBAAA|73986|20|33|6|PM|second|evening|| +73987|AAAAAAAAEABCBAAA|73987|20|33|7|PM|second|evening|| +73988|AAAAAAAAFABCBAAA|73988|20|33|8|PM|second|evening|| +73989|AAAAAAAAGABCBAAA|73989|20|33|9|PM|second|evening|| +73990|AAAAAAAAHABCBAAA|73990|20|33|10|PM|second|evening|| +73991|AAAAAAAAIABCBAAA|73991|20|33|11|PM|second|evening|| +73992|AAAAAAAAJABCBAAA|73992|20|33|12|PM|second|evening|| +73993|AAAAAAAAKABCBAAA|73993|20|33|13|PM|second|evening|| +73994|AAAAAAAALABCBAAA|73994|20|33|14|PM|second|evening|| +73995|AAAAAAAAMABCBAAA|73995|20|33|15|PM|second|evening|| +73996|AAAAAAAANABCBAAA|73996|20|33|16|PM|second|evening|| +73997|AAAAAAAAOABCBAAA|73997|20|33|17|PM|second|evening|| +73998|AAAAAAAAPABCBAAA|73998|20|33|18|PM|second|evening|| +73999|AAAAAAAAABBCBAAA|73999|20|33|19|PM|second|evening|| +74000|AAAAAAAABBBCBAAA|74000|20|33|20|PM|second|evening|| +74001|AAAAAAAACBBCBAAA|74001|20|33|21|PM|second|evening|| +74002|AAAAAAAADBBCBAAA|74002|20|33|22|PM|second|evening|| +74003|AAAAAAAAEBBCBAAA|74003|20|33|23|PM|second|evening|| +74004|AAAAAAAAFBBCBAAA|74004|20|33|24|PM|second|evening|| +74005|AAAAAAAAGBBCBAAA|74005|20|33|25|PM|second|evening|| +74006|AAAAAAAAHBBCBAAA|74006|20|33|26|PM|second|evening|| +74007|AAAAAAAAIBBCBAAA|74007|20|33|27|PM|second|evening|| +74008|AAAAAAAAJBBCBAAA|74008|20|33|28|PM|second|evening|| +74009|AAAAAAAAKBBCBAAA|74009|20|33|29|PM|second|evening|| +74010|AAAAAAAALBBCBAAA|74010|20|33|30|PM|second|evening|| +74011|AAAAAAAAMBBCBAAA|74011|20|33|31|PM|second|evening|| +74012|AAAAAAAANBBCBAAA|74012|20|33|32|PM|second|evening|| +74013|AAAAAAAAOBBCBAAA|74013|20|33|33|PM|second|evening|| +74014|AAAAAAAAPBBCBAAA|74014|20|33|34|PM|second|evening|| +74015|AAAAAAAAACBCBAAA|74015|20|33|35|PM|second|evening|| +74016|AAAAAAAABCBCBAAA|74016|20|33|36|PM|second|evening|| +74017|AAAAAAAACCBCBAAA|74017|20|33|37|PM|second|evening|| +74018|AAAAAAAADCBCBAAA|74018|20|33|38|PM|second|evening|| +74019|AAAAAAAAECBCBAAA|74019|20|33|39|PM|second|evening|| +74020|AAAAAAAAFCBCBAAA|74020|20|33|40|PM|second|evening|| +74021|AAAAAAAAGCBCBAAA|74021|20|33|41|PM|second|evening|| +74022|AAAAAAAAHCBCBAAA|74022|20|33|42|PM|second|evening|| +74023|AAAAAAAAICBCBAAA|74023|20|33|43|PM|second|evening|| +74024|AAAAAAAAJCBCBAAA|74024|20|33|44|PM|second|evening|| +74025|AAAAAAAAKCBCBAAA|74025|20|33|45|PM|second|evening|| +74026|AAAAAAAALCBCBAAA|74026|20|33|46|PM|second|evening|| +74027|AAAAAAAAMCBCBAAA|74027|20|33|47|PM|second|evening|| +74028|AAAAAAAANCBCBAAA|74028|20|33|48|PM|second|evening|| +74029|AAAAAAAAOCBCBAAA|74029|20|33|49|PM|second|evening|| +74030|AAAAAAAAPCBCBAAA|74030|20|33|50|PM|second|evening|| +74031|AAAAAAAAADBCBAAA|74031|20|33|51|PM|second|evening|| +74032|AAAAAAAABDBCBAAA|74032|20|33|52|PM|second|evening|| +74033|AAAAAAAACDBCBAAA|74033|20|33|53|PM|second|evening|| +74034|AAAAAAAADDBCBAAA|74034|20|33|54|PM|second|evening|| +74035|AAAAAAAAEDBCBAAA|74035|20|33|55|PM|second|evening|| +74036|AAAAAAAAFDBCBAAA|74036|20|33|56|PM|second|evening|| +74037|AAAAAAAAGDBCBAAA|74037|20|33|57|PM|second|evening|| +74038|AAAAAAAAHDBCBAAA|74038|20|33|58|PM|second|evening|| +74039|AAAAAAAAIDBCBAAA|74039|20|33|59|PM|second|evening|| +74040|AAAAAAAAJDBCBAAA|74040|20|34|0|PM|second|evening|| +74041|AAAAAAAAKDBCBAAA|74041|20|34|1|PM|second|evening|| +74042|AAAAAAAALDBCBAAA|74042|20|34|2|PM|second|evening|| +74043|AAAAAAAAMDBCBAAA|74043|20|34|3|PM|second|evening|| +74044|AAAAAAAANDBCBAAA|74044|20|34|4|PM|second|evening|| +74045|AAAAAAAAODBCBAAA|74045|20|34|5|PM|second|evening|| +74046|AAAAAAAAPDBCBAAA|74046|20|34|6|PM|second|evening|| +74047|AAAAAAAAAEBCBAAA|74047|20|34|7|PM|second|evening|| +74048|AAAAAAAABEBCBAAA|74048|20|34|8|PM|second|evening|| +74049|AAAAAAAACEBCBAAA|74049|20|34|9|PM|second|evening|| +74050|AAAAAAAADEBCBAAA|74050|20|34|10|PM|second|evening|| +74051|AAAAAAAAEEBCBAAA|74051|20|34|11|PM|second|evening|| +74052|AAAAAAAAFEBCBAAA|74052|20|34|12|PM|second|evening|| +74053|AAAAAAAAGEBCBAAA|74053|20|34|13|PM|second|evening|| +74054|AAAAAAAAHEBCBAAA|74054|20|34|14|PM|second|evening|| +74055|AAAAAAAAIEBCBAAA|74055|20|34|15|PM|second|evening|| +74056|AAAAAAAAJEBCBAAA|74056|20|34|16|PM|second|evening|| +74057|AAAAAAAAKEBCBAAA|74057|20|34|17|PM|second|evening|| +74058|AAAAAAAALEBCBAAA|74058|20|34|18|PM|second|evening|| +74059|AAAAAAAAMEBCBAAA|74059|20|34|19|PM|second|evening|| +74060|AAAAAAAANEBCBAAA|74060|20|34|20|PM|second|evening|| +74061|AAAAAAAAOEBCBAAA|74061|20|34|21|PM|second|evening|| +74062|AAAAAAAAPEBCBAAA|74062|20|34|22|PM|second|evening|| +74063|AAAAAAAAAFBCBAAA|74063|20|34|23|PM|second|evening|| +74064|AAAAAAAABFBCBAAA|74064|20|34|24|PM|second|evening|| +74065|AAAAAAAACFBCBAAA|74065|20|34|25|PM|second|evening|| +74066|AAAAAAAADFBCBAAA|74066|20|34|26|PM|second|evening|| +74067|AAAAAAAAEFBCBAAA|74067|20|34|27|PM|second|evening|| +74068|AAAAAAAAFFBCBAAA|74068|20|34|28|PM|second|evening|| +74069|AAAAAAAAGFBCBAAA|74069|20|34|29|PM|second|evening|| +74070|AAAAAAAAHFBCBAAA|74070|20|34|30|PM|second|evening|| +74071|AAAAAAAAIFBCBAAA|74071|20|34|31|PM|second|evening|| +74072|AAAAAAAAJFBCBAAA|74072|20|34|32|PM|second|evening|| +74073|AAAAAAAAKFBCBAAA|74073|20|34|33|PM|second|evening|| +74074|AAAAAAAALFBCBAAA|74074|20|34|34|PM|second|evening|| +74075|AAAAAAAAMFBCBAAA|74075|20|34|35|PM|second|evening|| +74076|AAAAAAAANFBCBAAA|74076|20|34|36|PM|second|evening|| +74077|AAAAAAAAOFBCBAAA|74077|20|34|37|PM|second|evening|| +74078|AAAAAAAAPFBCBAAA|74078|20|34|38|PM|second|evening|| +74079|AAAAAAAAAGBCBAAA|74079|20|34|39|PM|second|evening|| +74080|AAAAAAAABGBCBAAA|74080|20|34|40|PM|second|evening|| +74081|AAAAAAAACGBCBAAA|74081|20|34|41|PM|second|evening|| +74082|AAAAAAAADGBCBAAA|74082|20|34|42|PM|second|evening|| +74083|AAAAAAAAEGBCBAAA|74083|20|34|43|PM|second|evening|| +74084|AAAAAAAAFGBCBAAA|74084|20|34|44|PM|second|evening|| +74085|AAAAAAAAGGBCBAAA|74085|20|34|45|PM|second|evening|| +74086|AAAAAAAAHGBCBAAA|74086|20|34|46|PM|second|evening|| +74087|AAAAAAAAIGBCBAAA|74087|20|34|47|PM|second|evening|| +74088|AAAAAAAAJGBCBAAA|74088|20|34|48|PM|second|evening|| +74089|AAAAAAAAKGBCBAAA|74089|20|34|49|PM|second|evening|| +74090|AAAAAAAALGBCBAAA|74090|20|34|50|PM|second|evening|| +74091|AAAAAAAAMGBCBAAA|74091|20|34|51|PM|second|evening|| +74092|AAAAAAAANGBCBAAA|74092|20|34|52|PM|second|evening|| +74093|AAAAAAAAOGBCBAAA|74093|20|34|53|PM|second|evening|| +74094|AAAAAAAAPGBCBAAA|74094|20|34|54|PM|second|evening|| +74095|AAAAAAAAAHBCBAAA|74095|20|34|55|PM|second|evening|| +74096|AAAAAAAABHBCBAAA|74096|20|34|56|PM|second|evening|| +74097|AAAAAAAACHBCBAAA|74097|20|34|57|PM|second|evening|| +74098|AAAAAAAADHBCBAAA|74098|20|34|58|PM|second|evening|| +74099|AAAAAAAAEHBCBAAA|74099|20|34|59|PM|second|evening|| +74100|AAAAAAAAFHBCBAAA|74100|20|35|0|PM|second|evening|| +74101|AAAAAAAAGHBCBAAA|74101|20|35|1|PM|second|evening|| +74102|AAAAAAAAHHBCBAAA|74102|20|35|2|PM|second|evening|| +74103|AAAAAAAAIHBCBAAA|74103|20|35|3|PM|second|evening|| +74104|AAAAAAAAJHBCBAAA|74104|20|35|4|PM|second|evening|| +74105|AAAAAAAAKHBCBAAA|74105|20|35|5|PM|second|evening|| +74106|AAAAAAAALHBCBAAA|74106|20|35|6|PM|second|evening|| +74107|AAAAAAAAMHBCBAAA|74107|20|35|7|PM|second|evening|| +74108|AAAAAAAANHBCBAAA|74108|20|35|8|PM|second|evening|| +74109|AAAAAAAAOHBCBAAA|74109|20|35|9|PM|second|evening|| +74110|AAAAAAAAPHBCBAAA|74110|20|35|10|PM|second|evening|| +74111|AAAAAAAAAIBCBAAA|74111|20|35|11|PM|second|evening|| +74112|AAAAAAAABIBCBAAA|74112|20|35|12|PM|second|evening|| +74113|AAAAAAAACIBCBAAA|74113|20|35|13|PM|second|evening|| +74114|AAAAAAAADIBCBAAA|74114|20|35|14|PM|second|evening|| +74115|AAAAAAAAEIBCBAAA|74115|20|35|15|PM|second|evening|| +74116|AAAAAAAAFIBCBAAA|74116|20|35|16|PM|second|evening|| +74117|AAAAAAAAGIBCBAAA|74117|20|35|17|PM|second|evening|| +74118|AAAAAAAAHIBCBAAA|74118|20|35|18|PM|second|evening|| +74119|AAAAAAAAIIBCBAAA|74119|20|35|19|PM|second|evening|| +74120|AAAAAAAAJIBCBAAA|74120|20|35|20|PM|second|evening|| +74121|AAAAAAAAKIBCBAAA|74121|20|35|21|PM|second|evening|| +74122|AAAAAAAALIBCBAAA|74122|20|35|22|PM|second|evening|| +74123|AAAAAAAAMIBCBAAA|74123|20|35|23|PM|second|evening|| +74124|AAAAAAAANIBCBAAA|74124|20|35|24|PM|second|evening|| +74125|AAAAAAAAOIBCBAAA|74125|20|35|25|PM|second|evening|| +74126|AAAAAAAAPIBCBAAA|74126|20|35|26|PM|second|evening|| +74127|AAAAAAAAAJBCBAAA|74127|20|35|27|PM|second|evening|| +74128|AAAAAAAABJBCBAAA|74128|20|35|28|PM|second|evening|| +74129|AAAAAAAACJBCBAAA|74129|20|35|29|PM|second|evening|| +74130|AAAAAAAADJBCBAAA|74130|20|35|30|PM|second|evening|| +74131|AAAAAAAAEJBCBAAA|74131|20|35|31|PM|second|evening|| +74132|AAAAAAAAFJBCBAAA|74132|20|35|32|PM|second|evening|| +74133|AAAAAAAAGJBCBAAA|74133|20|35|33|PM|second|evening|| +74134|AAAAAAAAHJBCBAAA|74134|20|35|34|PM|second|evening|| +74135|AAAAAAAAIJBCBAAA|74135|20|35|35|PM|second|evening|| +74136|AAAAAAAAJJBCBAAA|74136|20|35|36|PM|second|evening|| +74137|AAAAAAAAKJBCBAAA|74137|20|35|37|PM|second|evening|| +74138|AAAAAAAALJBCBAAA|74138|20|35|38|PM|second|evening|| +74139|AAAAAAAAMJBCBAAA|74139|20|35|39|PM|second|evening|| +74140|AAAAAAAANJBCBAAA|74140|20|35|40|PM|second|evening|| +74141|AAAAAAAAOJBCBAAA|74141|20|35|41|PM|second|evening|| +74142|AAAAAAAAPJBCBAAA|74142|20|35|42|PM|second|evening|| +74143|AAAAAAAAAKBCBAAA|74143|20|35|43|PM|second|evening|| +74144|AAAAAAAABKBCBAAA|74144|20|35|44|PM|second|evening|| +74145|AAAAAAAACKBCBAAA|74145|20|35|45|PM|second|evening|| +74146|AAAAAAAADKBCBAAA|74146|20|35|46|PM|second|evening|| +74147|AAAAAAAAEKBCBAAA|74147|20|35|47|PM|second|evening|| +74148|AAAAAAAAFKBCBAAA|74148|20|35|48|PM|second|evening|| +74149|AAAAAAAAGKBCBAAA|74149|20|35|49|PM|second|evening|| +74150|AAAAAAAAHKBCBAAA|74150|20|35|50|PM|second|evening|| +74151|AAAAAAAAIKBCBAAA|74151|20|35|51|PM|second|evening|| +74152|AAAAAAAAJKBCBAAA|74152|20|35|52|PM|second|evening|| +74153|AAAAAAAAKKBCBAAA|74153|20|35|53|PM|second|evening|| +74154|AAAAAAAALKBCBAAA|74154|20|35|54|PM|second|evening|| +74155|AAAAAAAAMKBCBAAA|74155|20|35|55|PM|second|evening|| +74156|AAAAAAAANKBCBAAA|74156|20|35|56|PM|second|evening|| +74157|AAAAAAAAOKBCBAAA|74157|20|35|57|PM|second|evening|| +74158|AAAAAAAAPKBCBAAA|74158|20|35|58|PM|second|evening|| +74159|AAAAAAAAALBCBAAA|74159|20|35|59|PM|second|evening|| +74160|AAAAAAAABLBCBAAA|74160|20|36|0|PM|second|evening|| +74161|AAAAAAAACLBCBAAA|74161|20|36|1|PM|second|evening|| +74162|AAAAAAAADLBCBAAA|74162|20|36|2|PM|second|evening|| +74163|AAAAAAAAELBCBAAA|74163|20|36|3|PM|second|evening|| +74164|AAAAAAAAFLBCBAAA|74164|20|36|4|PM|second|evening|| +74165|AAAAAAAAGLBCBAAA|74165|20|36|5|PM|second|evening|| +74166|AAAAAAAAHLBCBAAA|74166|20|36|6|PM|second|evening|| +74167|AAAAAAAAILBCBAAA|74167|20|36|7|PM|second|evening|| +74168|AAAAAAAAJLBCBAAA|74168|20|36|8|PM|second|evening|| +74169|AAAAAAAAKLBCBAAA|74169|20|36|9|PM|second|evening|| +74170|AAAAAAAALLBCBAAA|74170|20|36|10|PM|second|evening|| +74171|AAAAAAAAMLBCBAAA|74171|20|36|11|PM|second|evening|| +74172|AAAAAAAANLBCBAAA|74172|20|36|12|PM|second|evening|| +74173|AAAAAAAAOLBCBAAA|74173|20|36|13|PM|second|evening|| +74174|AAAAAAAAPLBCBAAA|74174|20|36|14|PM|second|evening|| +74175|AAAAAAAAAMBCBAAA|74175|20|36|15|PM|second|evening|| +74176|AAAAAAAABMBCBAAA|74176|20|36|16|PM|second|evening|| +74177|AAAAAAAACMBCBAAA|74177|20|36|17|PM|second|evening|| +74178|AAAAAAAADMBCBAAA|74178|20|36|18|PM|second|evening|| +74179|AAAAAAAAEMBCBAAA|74179|20|36|19|PM|second|evening|| +74180|AAAAAAAAFMBCBAAA|74180|20|36|20|PM|second|evening|| +74181|AAAAAAAAGMBCBAAA|74181|20|36|21|PM|second|evening|| +74182|AAAAAAAAHMBCBAAA|74182|20|36|22|PM|second|evening|| +74183|AAAAAAAAIMBCBAAA|74183|20|36|23|PM|second|evening|| +74184|AAAAAAAAJMBCBAAA|74184|20|36|24|PM|second|evening|| +74185|AAAAAAAAKMBCBAAA|74185|20|36|25|PM|second|evening|| +74186|AAAAAAAALMBCBAAA|74186|20|36|26|PM|second|evening|| +74187|AAAAAAAAMMBCBAAA|74187|20|36|27|PM|second|evening|| +74188|AAAAAAAANMBCBAAA|74188|20|36|28|PM|second|evening|| +74189|AAAAAAAAOMBCBAAA|74189|20|36|29|PM|second|evening|| +74190|AAAAAAAAPMBCBAAA|74190|20|36|30|PM|second|evening|| +74191|AAAAAAAAANBCBAAA|74191|20|36|31|PM|second|evening|| +74192|AAAAAAAABNBCBAAA|74192|20|36|32|PM|second|evening|| +74193|AAAAAAAACNBCBAAA|74193|20|36|33|PM|second|evening|| +74194|AAAAAAAADNBCBAAA|74194|20|36|34|PM|second|evening|| +74195|AAAAAAAAENBCBAAA|74195|20|36|35|PM|second|evening|| +74196|AAAAAAAAFNBCBAAA|74196|20|36|36|PM|second|evening|| +74197|AAAAAAAAGNBCBAAA|74197|20|36|37|PM|second|evening|| +74198|AAAAAAAAHNBCBAAA|74198|20|36|38|PM|second|evening|| +74199|AAAAAAAAINBCBAAA|74199|20|36|39|PM|second|evening|| +74200|AAAAAAAAJNBCBAAA|74200|20|36|40|PM|second|evening|| +74201|AAAAAAAAKNBCBAAA|74201|20|36|41|PM|second|evening|| +74202|AAAAAAAALNBCBAAA|74202|20|36|42|PM|second|evening|| +74203|AAAAAAAAMNBCBAAA|74203|20|36|43|PM|second|evening|| +74204|AAAAAAAANNBCBAAA|74204|20|36|44|PM|second|evening|| +74205|AAAAAAAAONBCBAAA|74205|20|36|45|PM|second|evening|| +74206|AAAAAAAAPNBCBAAA|74206|20|36|46|PM|second|evening|| +74207|AAAAAAAAAOBCBAAA|74207|20|36|47|PM|second|evening|| +74208|AAAAAAAABOBCBAAA|74208|20|36|48|PM|second|evening|| +74209|AAAAAAAACOBCBAAA|74209|20|36|49|PM|second|evening|| +74210|AAAAAAAADOBCBAAA|74210|20|36|50|PM|second|evening|| +74211|AAAAAAAAEOBCBAAA|74211|20|36|51|PM|second|evening|| +74212|AAAAAAAAFOBCBAAA|74212|20|36|52|PM|second|evening|| +74213|AAAAAAAAGOBCBAAA|74213|20|36|53|PM|second|evening|| +74214|AAAAAAAAHOBCBAAA|74214|20|36|54|PM|second|evening|| +74215|AAAAAAAAIOBCBAAA|74215|20|36|55|PM|second|evening|| +74216|AAAAAAAAJOBCBAAA|74216|20|36|56|PM|second|evening|| +74217|AAAAAAAAKOBCBAAA|74217|20|36|57|PM|second|evening|| +74218|AAAAAAAALOBCBAAA|74218|20|36|58|PM|second|evening|| +74219|AAAAAAAAMOBCBAAA|74219|20|36|59|PM|second|evening|| +74220|AAAAAAAANOBCBAAA|74220|20|37|0|PM|second|evening|| +74221|AAAAAAAAOOBCBAAA|74221|20|37|1|PM|second|evening|| +74222|AAAAAAAAPOBCBAAA|74222|20|37|2|PM|second|evening|| +74223|AAAAAAAAAPBCBAAA|74223|20|37|3|PM|second|evening|| +74224|AAAAAAAABPBCBAAA|74224|20|37|4|PM|second|evening|| +74225|AAAAAAAACPBCBAAA|74225|20|37|5|PM|second|evening|| +74226|AAAAAAAADPBCBAAA|74226|20|37|6|PM|second|evening|| +74227|AAAAAAAAEPBCBAAA|74227|20|37|7|PM|second|evening|| +74228|AAAAAAAAFPBCBAAA|74228|20|37|8|PM|second|evening|| +74229|AAAAAAAAGPBCBAAA|74229|20|37|9|PM|second|evening|| +74230|AAAAAAAAHPBCBAAA|74230|20|37|10|PM|second|evening|| +74231|AAAAAAAAIPBCBAAA|74231|20|37|11|PM|second|evening|| +74232|AAAAAAAAJPBCBAAA|74232|20|37|12|PM|second|evening|| +74233|AAAAAAAAKPBCBAAA|74233|20|37|13|PM|second|evening|| +74234|AAAAAAAALPBCBAAA|74234|20|37|14|PM|second|evening|| +74235|AAAAAAAAMPBCBAAA|74235|20|37|15|PM|second|evening|| +74236|AAAAAAAANPBCBAAA|74236|20|37|16|PM|second|evening|| +74237|AAAAAAAAOPBCBAAA|74237|20|37|17|PM|second|evening|| +74238|AAAAAAAAPPBCBAAA|74238|20|37|18|PM|second|evening|| +74239|AAAAAAAAAACCBAAA|74239|20|37|19|PM|second|evening|| +74240|AAAAAAAABACCBAAA|74240|20|37|20|PM|second|evening|| +74241|AAAAAAAACACCBAAA|74241|20|37|21|PM|second|evening|| +74242|AAAAAAAADACCBAAA|74242|20|37|22|PM|second|evening|| +74243|AAAAAAAAEACCBAAA|74243|20|37|23|PM|second|evening|| +74244|AAAAAAAAFACCBAAA|74244|20|37|24|PM|second|evening|| +74245|AAAAAAAAGACCBAAA|74245|20|37|25|PM|second|evening|| +74246|AAAAAAAAHACCBAAA|74246|20|37|26|PM|second|evening|| +74247|AAAAAAAAIACCBAAA|74247|20|37|27|PM|second|evening|| +74248|AAAAAAAAJACCBAAA|74248|20|37|28|PM|second|evening|| +74249|AAAAAAAAKACCBAAA|74249|20|37|29|PM|second|evening|| +74250|AAAAAAAALACCBAAA|74250|20|37|30|PM|second|evening|| +74251|AAAAAAAAMACCBAAA|74251|20|37|31|PM|second|evening|| +74252|AAAAAAAANACCBAAA|74252|20|37|32|PM|second|evening|| +74253|AAAAAAAAOACCBAAA|74253|20|37|33|PM|second|evening|| +74254|AAAAAAAAPACCBAAA|74254|20|37|34|PM|second|evening|| +74255|AAAAAAAAABCCBAAA|74255|20|37|35|PM|second|evening|| +74256|AAAAAAAABBCCBAAA|74256|20|37|36|PM|second|evening|| +74257|AAAAAAAACBCCBAAA|74257|20|37|37|PM|second|evening|| +74258|AAAAAAAADBCCBAAA|74258|20|37|38|PM|second|evening|| +74259|AAAAAAAAEBCCBAAA|74259|20|37|39|PM|second|evening|| +74260|AAAAAAAAFBCCBAAA|74260|20|37|40|PM|second|evening|| +74261|AAAAAAAAGBCCBAAA|74261|20|37|41|PM|second|evening|| +74262|AAAAAAAAHBCCBAAA|74262|20|37|42|PM|second|evening|| +74263|AAAAAAAAIBCCBAAA|74263|20|37|43|PM|second|evening|| +74264|AAAAAAAAJBCCBAAA|74264|20|37|44|PM|second|evening|| +74265|AAAAAAAAKBCCBAAA|74265|20|37|45|PM|second|evening|| +74266|AAAAAAAALBCCBAAA|74266|20|37|46|PM|second|evening|| +74267|AAAAAAAAMBCCBAAA|74267|20|37|47|PM|second|evening|| +74268|AAAAAAAANBCCBAAA|74268|20|37|48|PM|second|evening|| +74269|AAAAAAAAOBCCBAAA|74269|20|37|49|PM|second|evening|| +74270|AAAAAAAAPBCCBAAA|74270|20|37|50|PM|second|evening|| +74271|AAAAAAAAACCCBAAA|74271|20|37|51|PM|second|evening|| +74272|AAAAAAAABCCCBAAA|74272|20|37|52|PM|second|evening|| +74273|AAAAAAAACCCCBAAA|74273|20|37|53|PM|second|evening|| +74274|AAAAAAAADCCCBAAA|74274|20|37|54|PM|second|evening|| +74275|AAAAAAAAECCCBAAA|74275|20|37|55|PM|second|evening|| +74276|AAAAAAAAFCCCBAAA|74276|20|37|56|PM|second|evening|| +74277|AAAAAAAAGCCCBAAA|74277|20|37|57|PM|second|evening|| +74278|AAAAAAAAHCCCBAAA|74278|20|37|58|PM|second|evening|| +74279|AAAAAAAAICCCBAAA|74279|20|37|59|PM|second|evening|| +74280|AAAAAAAAJCCCBAAA|74280|20|38|0|PM|second|evening|| +74281|AAAAAAAAKCCCBAAA|74281|20|38|1|PM|second|evening|| +74282|AAAAAAAALCCCBAAA|74282|20|38|2|PM|second|evening|| +74283|AAAAAAAAMCCCBAAA|74283|20|38|3|PM|second|evening|| +74284|AAAAAAAANCCCBAAA|74284|20|38|4|PM|second|evening|| +74285|AAAAAAAAOCCCBAAA|74285|20|38|5|PM|second|evening|| +74286|AAAAAAAAPCCCBAAA|74286|20|38|6|PM|second|evening|| +74287|AAAAAAAAADCCBAAA|74287|20|38|7|PM|second|evening|| +74288|AAAAAAAABDCCBAAA|74288|20|38|8|PM|second|evening|| +74289|AAAAAAAACDCCBAAA|74289|20|38|9|PM|second|evening|| +74290|AAAAAAAADDCCBAAA|74290|20|38|10|PM|second|evening|| +74291|AAAAAAAAEDCCBAAA|74291|20|38|11|PM|second|evening|| +74292|AAAAAAAAFDCCBAAA|74292|20|38|12|PM|second|evening|| +74293|AAAAAAAAGDCCBAAA|74293|20|38|13|PM|second|evening|| +74294|AAAAAAAAHDCCBAAA|74294|20|38|14|PM|second|evening|| +74295|AAAAAAAAIDCCBAAA|74295|20|38|15|PM|second|evening|| +74296|AAAAAAAAJDCCBAAA|74296|20|38|16|PM|second|evening|| +74297|AAAAAAAAKDCCBAAA|74297|20|38|17|PM|second|evening|| +74298|AAAAAAAALDCCBAAA|74298|20|38|18|PM|second|evening|| +74299|AAAAAAAAMDCCBAAA|74299|20|38|19|PM|second|evening|| +74300|AAAAAAAANDCCBAAA|74300|20|38|20|PM|second|evening|| +74301|AAAAAAAAODCCBAAA|74301|20|38|21|PM|second|evening|| +74302|AAAAAAAAPDCCBAAA|74302|20|38|22|PM|second|evening|| +74303|AAAAAAAAAECCBAAA|74303|20|38|23|PM|second|evening|| +74304|AAAAAAAABECCBAAA|74304|20|38|24|PM|second|evening|| +74305|AAAAAAAACECCBAAA|74305|20|38|25|PM|second|evening|| +74306|AAAAAAAADECCBAAA|74306|20|38|26|PM|second|evening|| +74307|AAAAAAAAEECCBAAA|74307|20|38|27|PM|second|evening|| +74308|AAAAAAAAFECCBAAA|74308|20|38|28|PM|second|evening|| +74309|AAAAAAAAGECCBAAA|74309|20|38|29|PM|second|evening|| +74310|AAAAAAAAHECCBAAA|74310|20|38|30|PM|second|evening|| +74311|AAAAAAAAIECCBAAA|74311|20|38|31|PM|second|evening|| +74312|AAAAAAAAJECCBAAA|74312|20|38|32|PM|second|evening|| +74313|AAAAAAAAKECCBAAA|74313|20|38|33|PM|second|evening|| +74314|AAAAAAAALECCBAAA|74314|20|38|34|PM|second|evening|| +74315|AAAAAAAAMECCBAAA|74315|20|38|35|PM|second|evening|| +74316|AAAAAAAANECCBAAA|74316|20|38|36|PM|second|evening|| +74317|AAAAAAAAOECCBAAA|74317|20|38|37|PM|second|evening|| +74318|AAAAAAAAPECCBAAA|74318|20|38|38|PM|second|evening|| +74319|AAAAAAAAAFCCBAAA|74319|20|38|39|PM|second|evening|| +74320|AAAAAAAABFCCBAAA|74320|20|38|40|PM|second|evening|| +74321|AAAAAAAACFCCBAAA|74321|20|38|41|PM|second|evening|| +74322|AAAAAAAADFCCBAAA|74322|20|38|42|PM|second|evening|| +74323|AAAAAAAAEFCCBAAA|74323|20|38|43|PM|second|evening|| +74324|AAAAAAAAFFCCBAAA|74324|20|38|44|PM|second|evening|| +74325|AAAAAAAAGFCCBAAA|74325|20|38|45|PM|second|evening|| +74326|AAAAAAAAHFCCBAAA|74326|20|38|46|PM|second|evening|| +74327|AAAAAAAAIFCCBAAA|74327|20|38|47|PM|second|evening|| +74328|AAAAAAAAJFCCBAAA|74328|20|38|48|PM|second|evening|| +74329|AAAAAAAAKFCCBAAA|74329|20|38|49|PM|second|evening|| +74330|AAAAAAAALFCCBAAA|74330|20|38|50|PM|second|evening|| +74331|AAAAAAAAMFCCBAAA|74331|20|38|51|PM|second|evening|| +74332|AAAAAAAANFCCBAAA|74332|20|38|52|PM|second|evening|| +74333|AAAAAAAAOFCCBAAA|74333|20|38|53|PM|second|evening|| +74334|AAAAAAAAPFCCBAAA|74334|20|38|54|PM|second|evening|| +74335|AAAAAAAAAGCCBAAA|74335|20|38|55|PM|second|evening|| +74336|AAAAAAAABGCCBAAA|74336|20|38|56|PM|second|evening|| +74337|AAAAAAAACGCCBAAA|74337|20|38|57|PM|second|evening|| +74338|AAAAAAAADGCCBAAA|74338|20|38|58|PM|second|evening|| +74339|AAAAAAAAEGCCBAAA|74339|20|38|59|PM|second|evening|| +74340|AAAAAAAAFGCCBAAA|74340|20|39|0|PM|second|evening|| +74341|AAAAAAAAGGCCBAAA|74341|20|39|1|PM|second|evening|| +74342|AAAAAAAAHGCCBAAA|74342|20|39|2|PM|second|evening|| +74343|AAAAAAAAIGCCBAAA|74343|20|39|3|PM|second|evening|| +74344|AAAAAAAAJGCCBAAA|74344|20|39|4|PM|second|evening|| +74345|AAAAAAAAKGCCBAAA|74345|20|39|5|PM|second|evening|| +74346|AAAAAAAALGCCBAAA|74346|20|39|6|PM|second|evening|| +74347|AAAAAAAAMGCCBAAA|74347|20|39|7|PM|second|evening|| +74348|AAAAAAAANGCCBAAA|74348|20|39|8|PM|second|evening|| +74349|AAAAAAAAOGCCBAAA|74349|20|39|9|PM|second|evening|| +74350|AAAAAAAAPGCCBAAA|74350|20|39|10|PM|second|evening|| +74351|AAAAAAAAAHCCBAAA|74351|20|39|11|PM|second|evening|| +74352|AAAAAAAABHCCBAAA|74352|20|39|12|PM|second|evening|| +74353|AAAAAAAACHCCBAAA|74353|20|39|13|PM|second|evening|| +74354|AAAAAAAADHCCBAAA|74354|20|39|14|PM|second|evening|| +74355|AAAAAAAAEHCCBAAA|74355|20|39|15|PM|second|evening|| +74356|AAAAAAAAFHCCBAAA|74356|20|39|16|PM|second|evening|| +74357|AAAAAAAAGHCCBAAA|74357|20|39|17|PM|second|evening|| +74358|AAAAAAAAHHCCBAAA|74358|20|39|18|PM|second|evening|| +74359|AAAAAAAAIHCCBAAA|74359|20|39|19|PM|second|evening|| +74360|AAAAAAAAJHCCBAAA|74360|20|39|20|PM|second|evening|| +74361|AAAAAAAAKHCCBAAA|74361|20|39|21|PM|second|evening|| +74362|AAAAAAAALHCCBAAA|74362|20|39|22|PM|second|evening|| +74363|AAAAAAAAMHCCBAAA|74363|20|39|23|PM|second|evening|| +74364|AAAAAAAANHCCBAAA|74364|20|39|24|PM|second|evening|| +74365|AAAAAAAAOHCCBAAA|74365|20|39|25|PM|second|evening|| +74366|AAAAAAAAPHCCBAAA|74366|20|39|26|PM|second|evening|| +74367|AAAAAAAAAICCBAAA|74367|20|39|27|PM|second|evening|| +74368|AAAAAAAABICCBAAA|74368|20|39|28|PM|second|evening|| +74369|AAAAAAAACICCBAAA|74369|20|39|29|PM|second|evening|| +74370|AAAAAAAADICCBAAA|74370|20|39|30|PM|second|evening|| +74371|AAAAAAAAEICCBAAA|74371|20|39|31|PM|second|evening|| +74372|AAAAAAAAFICCBAAA|74372|20|39|32|PM|second|evening|| +74373|AAAAAAAAGICCBAAA|74373|20|39|33|PM|second|evening|| +74374|AAAAAAAAHICCBAAA|74374|20|39|34|PM|second|evening|| +74375|AAAAAAAAIICCBAAA|74375|20|39|35|PM|second|evening|| +74376|AAAAAAAAJICCBAAA|74376|20|39|36|PM|second|evening|| +74377|AAAAAAAAKICCBAAA|74377|20|39|37|PM|second|evening|| +74378|AAAAAAAALICCBAAA|74378|20|39|38|PM|second|evening|| +74379|AAAAAAAAMICCBAAA|74379|20|39|39|PM|second|evening|| +74380|AAAAAAAANICCBAAA|74380|20|39|40|PM|second|evening|| +74381|AAAAAAAAOICCBAAA|74381|20|39|41|PM|second|evening|| +74382|AAAAAAAAPICCBAAA|74382|20|39|42|PM|second|evening|| +74383|AAAAAAAAAJCCBAAA|74383|20|39|43|PM|second|evening|| +74384|AAAAAAAABJCCBAAA|74384|20|39|44|PM|second|evening|| +74385|AAAAAAAACJCCBAAA|74385|20|39|45|PM|second|evening|| +74386|AAAAAAAADJCCBAAA|74386|20|39|46|PM|second|evening|| +74387|AAAAAAAAEJCCBAAA|74387|20|39|47|PM|second|evening|| +74388|AAAAAAAAFJCCBAAA|74388|20|39|48|PM|second|evening|| +74389|AAAAAAAAGJCCBAAA|74389|20|39|49|PM|second|evening|| +74390|AAAAAAAAHJCCBAAA|74390|20|39|50|PM|second|evening|| +74391|AAAAAAAAIJCCBAAA|74391|20|39|51|PM|second|evening|| +74392|AAAAAAAAJJCCBAAA|74392|20|39|52|PM|second|evening|| +74393|AAAAAAAAKJCCBAAA|74393|20|39|53|PM|second|evening|| +74394|AAAAAAAALJCCBAAA|74394|20|39|54|PM|second|evening|| +74395|AAAAAAAAMJCCBAAA|74395|20|39|55|PM|second|evening|| +74396|AAAAAAAANJCCBAAA|74396|20|39|56|PM|second|evening|| +74397|AAAAAAAAOJCCBAAA|74397|20|39|57|PM|second|evening|| +74398|AAAAAAAAPJCCBAAA|74398|20|39|58|PM|second|evening|| +74399|AAAAAAAAAKCCBAAA|74399|20|39|59|PM|second|evening|| +74400|AAAAAAAABKCCBAAA|74400|20|40|0|PM|second|evening|| +74401|AAAAAAAACKCCBAAA|74401|20|40|1|PM|second|evening|| +74402|AAAAAAAADKCCBAAA|74402|20|40|2|PM|second|evening|| +74403|AAAAAAAAEKCCBAAA|74403|20|40|3|PM|second|evening|| +74404|AAAAAAAAFKCCBAAA|74404|20|40|4|PM|second|evening|| +74405|AAAAAAAAGKCCBAAA|74405|20|40|5|PM|second|evening|| +74406|AAAAAAAAHKCCBAAA|74406|20|40|6|PM|second|evening|| +74407|AAAAAAAAIKCCBAAA|74407|20|40|7|PM|second|evening|| +74408|AAAAAAAAJKCCBAAA|74408|20|40|8|PM|second|evening|| +74409|AAAAAAAAKKCCBAAA|74409|20|40|9|PM|second|evening|| +74410|AAAAAAAALKCCBAAA|74410|20|40|10|PM|second|evening|| +74411|AAAAAAAAMKCCBAAA|74411|20|40|11|PM|second|evening|| +74412|AAAAAAAANKCCBAAA|74412|20|40|12|PM|second|evening|| +74413|AAAAAAAAOKCCBAAA|74413|20|40|13|PM|second|evening|| +74414|AAAAAAAAPKCCBAAA|74414|20|40|14|PM|second|evening|| +74415|AAAAAAAAALCCBAAA|74415|20|40|15|PM|second|evening|| +74416|AAAAAAAABLCCBAAA|74416|20|40|16|PM|second|evening|| +74417|AAAAAAAACLCCBAAA|74417|20|40|17|PM|second|evening|| +74418|AAAAAAAADLCCBAAA|74418|20|40|18|PM|second|evening|| +74419|AAAAAAAAELCCBAAA|74419|20|40|19|PM|second|evening|| +74420|AAAAAAAAFLCCBAAA|74420|20|40|20|PM|second|evening|| +74421|AAAAAAAAGLCCBAAA|74421|20|40|21|PM|second|evening|| +74422|AAAAAAAAHLCCBAAA|74422|20|40|22|PM|second|evening|| +74423|AAAAAAAAILCCBAAA|74423|20|40|23|PM|second|evening|| +74424|AAAAAAAAJLCCBAAA|74424|20|40|24|PM|second|evening|| +74425|AAAAAAAAKLCCBAAA|74425|20|40|25|PM|second|evening|| +74426|AAAAAAAALLCCBAAA|74426|20|40|26|PM|second|evening|| +74427|AAAAAAAAMLCCBAAA|74427|20|40|27|PM|second|evening|| +74428|AAAAAAAANLCCBAAA|74428|20|40|28|PM|second|evening|| +74429|AAAAAAAAOLCCBAAA|74429|20|40|29|PM|second|evening|| +74430|AAAAAAAAPLCCBAAA|74430|20|40|30|PM|second|evening|| +74431|AAAAAAAAAMCCBAAA|74431|20|40|31|PM|second|evening|| +74432|AAAAAAAABMCCBAAA|74432|20|40|32|PM|second|evening|| +74433|AAAAAAAACMCCBAAA|74433|20|40|33|PM|second|evening|| +74434|AAAAAAAADMCCBAAA|74434|20|40|34|PM|second|evening|| +74435|AAAAAAAAEMCCBAAA|74435|20|40|35|PM|second|evening|| +74436|AAAAAAAAFMCCBAAA|74436|20|40|36|PM|second|evening|| +74437|AAAAAAAAGMCCBAAA|74437|20|40|37|PM|second|evening|| +74438|AAAAAAAAHMCCBAAA|74438|20|40|38|PM|second|evening|| +74439|AAAAAAAAIMCCBAAA|74439|20|40|39|PM|second|evening|| +74440|AAAAAAAAJMCCBAAA|74440|20|40|40|PM|second|evening|| +74441|AAAAAAAAKMCCBAAA|74441|20|40|41|PM|second|evening|| +74442|AAAAAAAALMCCBAAA|74442|20|40|42|PM|second|evening|| +74443|AAAAAAAAMMCCBAAA|74443|20|40|43|PM|second|evening|| +74444|AAAAAAAANMCCBAAA|74444|20|40|44|PM|second|evening|| +74445|AAAAAAAAOMCCBAAA|74445|20|40|45|PM|second|evening|| +74446|AAAAAAAAPMCCBAAA|74446|20|40|46|PM|second|evening|| +74447|AAAAAAAAANCCBAAA|74447|20|40|47|PM|second|evening|| +74448|AAAAAAAABNCCBAAA|74448|20|40|48|PM|second|evening|| +74449|AAAAAAAACNCCBAAA|74449|20|40|49|PM|second|evening|| +74450|AAAAAAAADNCCBAAA|74450|20|40|50|PM|second|evening|| +74451|AAAAAAAAENCCBAAA|74451|20|40|51|PM|second|evening|| +74452|AAAAAAAAFNCCBAAA|74452|20|40|52|PM|second|evening|| +74453|AAAAAAAAGNCCBAAA|74453|20|40|53|PM|second|evening|| +74454|AAAAAAAAHNCCBAAA|74454|20|40|54|PM|second|evening|| +74455|AAAAAAAAINCCBAAA|74455|20|40|55|PM|second|evening|| +74456|AAAAAAAAJNCCBAAA|74456|20|40|56|PM|second|evening|| +74457|AAAAAAAAKNCCBAAA|74457|20|40|57|PM|second|evening|| +74458|AAAAAAAALNCCBAAA|74458|20|40|58|PM|second|evening|| +74459|AAAAAAAAMNCCBAAA|74459|20|40|59|PM|second|evening|| +74460|AAAAAAAANNCCBAAA|74460|20|41|0|PM|second|evening|| +74461|AAAAAAAAONCCBAAA|74461|20|41|1|PM|second|evening|| +74462|AAAAAAAAPNCCBAAA|74462|20|41|2|PM|second|evening|| +74463|AAAAAAAAAOCCBAAA|74463|20|41|3|PM|second|evening|| +74464|AAAAAAAABOCCBAAA|74464|20|41|4|PM|second|evening|| +74465|AAAAAAAACOCCBAAA|74465|20|41|5|PM|second|evening|| +74466|AAAAAAAADOCCBAAA|74466|20|41|6|PM|second|evening|| +74467|AAAAAAAAEOCCBAAA|74467|20|41|7|PM|second|evening|| +74468|AAAAAAAAFOCCBAAA|74468|20|41|8|PM|second|evening|| +74469|AAAAAAAAGOCCBAAA|74469|20|41|9|PM|second|evening|| +74470|AAAAAAAAHOCCBAAA|74470|20|41|10|PM|second|evening|| +74471|AAAAAAAAIOCCBAAA|74471|20|41|11|PM|second|evening|| +74472|AAAAAAAAJOCCBAAA|74472|20|41|12|PM|second|evening|| +74473|AAAAAAAAKOCCBAAA|74473|20|41|13|PM|second|evening|| +74474|AAAAAAAALOCCBAAA|74474|20|41|14|PM|second|evening|| +74475|AAAAAAAAMOCCBAAA|74475|20|41|15|PM|second|evening|| +74476|AAAAAAAANOCCBAAA|74476|20|41|16|PM|second|evening|| +74477|AAAAAAAAOOCCBAAA|74477|20|41|17|PM|second|evening|| +74478|AAAAAAAAPOCCBAAA|74478|20|41|18|PM|second|evening|| +74479|AAAAAAAAAPCCBAAA|74479|20|41|19|PM|second|evening|| +74480|AAAAAAAABPCCBAAA|74480|20|41|20|PM|second|evening|| +74481|AAAAAAAACPCCBAAA|74481|20|41|21|PM|second|evening|| +74482|AAAAAAAADPCCBAAA|74482|20|41|22|PM|second|evening|| +74483|AAAAAAAAEPCCBAAA|74483|20|41|23|PM|second|evening|| +74484|AAAAAAAAFPCCBAAA|74484|20|41|24|PM|second|evening|| +74485|AAAAAAAAGPCCBAAA|74485|20|41|25|PM|second|evening|| +74486|AAAAAAAAHPCCBAAA|74486|20|41|26|PM|second|evening|| +74487|AAAAAAAAIPCCBAAA|74487|20|41|27|PM|second|evening|| +74488|AAAAAAAAJPCCBAAA|74488|20|41|28|PM|second|evening|| +74489|AAAAAAAAKPCCBAAA|74489|20|41|29|PM|second|evening|| +74490|AAAAAAAALPCCBAAA|74490|20|41|30|PM|second|evening|| +74491|AAAAAAAAMPCCBAAA|74491|20|41|31|PM|second|evening|| +74492|AAAAAAAANPCCBAAA|74492|20|41|32|PM|second|evening|| +74493|AAAAAAAAOPCCBAAA|74493|20|41|33|PM|second|evening|| +74494|AAAAAAAAPPCCBAAA|74494|20|41|34|PM|second|evening|| +74495|AAAAAAAAAADCBAAA|74495|20|41|35|PM|second|evening|| +74496|AAAAAAAABADCBAAA|74496|20|41|36|PM|second|evening|| +74497|AAAAAAAACADCBAAA|74497|20|41|37|PM|second|evening|| +74498|AAAAAAAADADCBAAA|74498|20|41|38|PM|second|evening|| +74499|AAAAAAAAEADCBAAA|74499|20|41|39|PM|second|evening|| +74500|AAAAAAAAFADCBAAA|74500|20|41|40|PM|second|evening|| +74501|AAAAAAAAGADCBAAA|74501|20|41|41|PM|second|evening|| +74502|AAAAAAAAHADCBAAA|74502|20|41|42|PM|second|evening|| +74503|AAAAAAAAIADCBAAA|74503|20|41|43|PM|second|evening|| +74504|AAAAAAAAJADCBAAA|74504|20|41|44|PM|second|evening|| +74505|AAAAAAAAKADCBAAA|74505|20|41|45|PM|second|evening|| +74506|AAAAAAAALADCBAAA|74506|20|41|46|PM|second|evening|| +74507|AAAAAAAAMADCBAAA|74507|20|41|47|PM|second|evening|| +74508|AAAAAAAANADCBAAA|74508|20|41|48|PM|second|evening|| +74509|AAAAAAAAOADCBAAA|74509|20|41|49|PM|second|evening|| +74510|AAAAAAAAPADCBAAA|74510|20|41|50|PM|second|evening|| +74511|AAAAAAAAABDCBAAA|74511|20|41|51|PM|second|evening|| +74512|AAAAAAAABBDCBAAA|74512|20|41|52|PM|second|evening|| +74513|AAAAAAAACBDCBAAA|74513|20|41|53|PM|second|evening|| +74514|AAAAAAAADBDCBAAA|74514|20|41|54|PM|second|evening|| +74515|AAAAAAAAEBDCBAAA|74515|20|41|55|PM|second|evening|| +74516|AAAAAAAAFBDCBAAA|74516|20|41|56|PM|second|evening|| +74517|AAAAAAAAGBDCBAAA|74517|20|41|57|PM|second|evening|| +74518|AAAAAAAAHBDCBAAA|74518|20|41|58|PM|second|evening|| +74519|AAAAAAAAIBDCBAAA|74519|20|41|59|PM|second|evening|| +74520|AAAAAAAAJBDCBAAA|74520|20|42|0|PM|second|evening|| +74521|AAAAAAAAKBDCBAAA|74521|20|42|1|PM|second|evening|| +74522|AAAAAAAALBDCBAAA|74522|20|42|2|PM|second|evening|| +74523|AAAAAAAAMBDCBAAA|74523|20|42|3|PM|second|evening|| +74524|AAAAAAAANBDCBAAA|74524|20|42|4|PM|second|evening|| +74525|AAAAAAAAOBDCBAAA|74525|20|42|5|PM|second|evening|| +74526|AAAAAAAAPBDCBAAA|74526|20|42|6|PM|second|evening|| +74527|AAAAAAAAACDCBAAA|74527|20|42|7|PM|second|evening|| +74528|AAAAAAAABCDCBAAA|74528|20|42|8|PM|second|evening|| +74529|AAAAAAAACCDCBAAA|74529|20|42|9|PM|second|evening|| +74530|AAAAAAAADCDCBAAA|74530|20|42|10|PM|second|evening|| +74531|AAAAAAAAECDCBAAA|74531|20|42|11|PM|second|evening|| +74532|AAAAAAAAFCDCBAAA|74532|20|42|12|PM|second|evening|| +74533|AAAAAAAAGCDCBAAA|74533|20|42|13|PM|second|evening|| +74534|AAAAAAAAHCDCBAAA|74534|20|42|14|PM|second|evening|| +74535|AAAAAAAAICDCBAAA|74535|20|42|15|PM|second|evening|| +74536|AAAAAAAAJCDCBAAA|74536|20|42|16|PM|second|evening|| +74537|AAAAAAAAKCDCBAAA|74537|20|42|17|PM|second|evening|| +74538|AAAAAAAALCDCBAAA|74538|20|42|18|PM|second|evening|| +74539|AAAAAAAAMCDCBAAA|74539|20|42|19|PM|second|evening|| +74540|AAAAAAAANCDCBAAA|74540|20|42|20|PM|second|evening|| +74541|AAAAAAAAOCDCBAAA|74541|20|42|21|PM|second|evening|| +74542|AAAAAAAAPCDCBAAA|74542|20|42|22|PM|second|evening|| +74543|AAAAAAAAADDCBAAA|74543|20|42|23|PM|second|evening|| +74544|AAAAAAAABDDCBAAA|74544|20|42|24|PM|second|evening|| +74545|AAAAAAAACDDCBAAA|74545|20|42|25|PM|second|evening|| +74546|AAAAAAAADDDCBAAA|74546|20|42|26|PM|second|evening|| +74547|AAAAAAAAEDDCBAAA|74547|20|42|27|PM|second|evening|| +74548|AAAAAAAAFDDCBAAA|74548|20|42|28|PM|second|evening|| +74549|AAAAAAAAGDDCBAAA|74549|20|42|29|PM|second|evening|| +74550|AAAAAAAAHDDCBAAA|74550|20|42|30|PM|second|evening|| +74551|AAAAAAAAIDDCBAAA|74551|20|42|31|PM|second|evening|| +74552|AAAAAAAAJDDCBAAA|74552|20|42|32|PM|second|evening|| +74553|AAAAAAAAKDDCBAAA|74553|20|42|33|PM|second|evening|| +74554|AAAAAAAALDDCBAAA|74554|20|42|34|PM|second|evening|| +74555|AAAAAAAAMDDCBAAA|74555|20|42|35|PM|second|evening|| +74556|AAAAAAAANDDCBAAA|74556|20|42|36|PM|second|evening|| +74557|AAAAAAAAODDCBAAA|74557|20|42|37|PM|second|evening|| +74558|AAAAAAAAPDDCBAAA|74558|20|42|38|PM|second|evening|| +74559|AAAAAAAAAEDCBAAA|74559|20|42|39|PM|second|evening|| +74560|AAAAAAAABEDCBAAA|74560|20|42|40|PM|second|evening|| +74561|AAAAAAAACEDCBAAA|74561|20|42|41|PM|second|evening|| +74562|AAAAAAAADEDCBAAA|74562|20|42|42|PM|second|evening|| +74563|AAAAAAAAEEDCBAAA|74563|20|42|43|PM|second|evening|| +74564|AAAAAAAAFEDCBAAA|74564|20|42|44|PM|second|evening|| +74565|AAAAAAAAGEDCBAAA|74565|20|42|45|PM|second|evening|| +74566|AAAAAAAAHEDCBAAA|74566|20|42|46|PM|second|evening|| +74567|AAAAAAAAIEDCBAAA|74567|20|42|47|PM|second|evening|| +74568|AAAAAAAAJEDCBAAA|74568|20|42|48|PM|second|evening|| +74569|AAAAAAAAKEDCBAAA|74569|20|42|49|PM|second|evening|| +74570|AAAAAAAALEDCBAAA|74570|20|42|50|PM|second|evening|| +74571|AAAAAAAAMEDCBAAA|74571|20|42|51|PM|second|evening|| +74572|AAAAAAAANEDCBAAA|74572|20|42|52|PM|second|evening|| +74573|AAAAAAAAOEDCBAAA|74573|20|42|53|PM|second|evening|| +74574|AAAAAAAAPEDCBAAA|74574|20|42|54|PM|second|evening|| +74575|AAAAAAAAAFDCBAAA|74575|20|42|55|PM|second|evening|| +74576|AAAAAAAABFDCBAAA|74576|20|42|56|PM|second|evening|| +74577|AAAAAAAACFDCBAAA|74577|20|42|57|PM|second|evening|| +74578|AAAAAAAADFDCBAAA|74578|20|42|58|PM|second|evening|| +74579|AAAAAAAAEFDCBAAA|74579|20|42|59|PM|second|evening|| +74580|AAAAAAAAFFDCBAAA|74580|20|43|0|PM|second|evening|| +74581|AAAAAAAAGFDCBAAA|74581|20|43|1|PM|second|evening|| +74582|AAAAAAAAHFDCBAAA|74582|20|43|2|PM|second|evening|| +74583|AAAAAAAAIFDCBAAA|74583|20|43|3|PM|second|evening|| +74584|AAAAAAAAJFDCBAAA|74584|20|43|4|PM|second|evening|| +74585|AAAAAAAAKFDCBAAA|74585|20|43|5|PM|second|evening|| +74586|AAAAAAAALFDCBAAA|74586|20|43|6|PM|second|evening|| +74587|AAAAAAAAMFDCBAAA|74587|20|43|7|PM|second|evening|| +74588|AAAAAAAANFDCBAAA|74588|20|43|8|PM|second|evening|| +74589|AAAAAAAAOFDCBAAA|74589|20|43|9|PM|second|evening|| +74590|AAAAAAAAPFDCBAAA|74590|20|43|10|PM|second|evening|| +74591|AAAAAAAAAGDCBAAA|74591|20|43|11|PM|second|evening|| +74592|AAAAAAAABGDCBAAA|74592|20|43|12|PM|second|evening|| +74593|AAAAAAAACGDCBAAA|74593|20|43|13|PM|second|evening|| +74594|AAAAAAAADGDCBAAA|74594|20|43|14|PM|second|evening|| +74595|AAAAAAAAEGDCBAAA|74595|20|43|15|PM|second|evening|| +74596|AAAAAAAAFGDCBAAA|74596|20|43|16|PM|second|evening|| +74597|AAAAAAAAGGDCBAAA|74597|20|43|17|PM|second|evening|| +74598|AAAAAAAAHGDCBAAA|74598|20|43|18|PM|second|evening|| +74599|AAAAAAAAIGDCBAAA|74599|20|43|19|PM|second|evening|| +74600|AAAAAAAAJGDCBAAA|74600|20|43|20|PM|second|evening|| +74601|AAAAAAAAKGDCBAAA|74601|20|43|21|PM|second|evening|| +74602|AAAAAAAALGDCBAAA|74602|20|43|22|PM|second|evening|| +74603|AAAAAAAAMGDCBAAA|74603|20|43|23|PM|second|evening|| +74604|AAAAAAAANGDCBAAA|74604|20|43|24|PM|second|evening|| +74605|AAAAAAAAOGDCBAAA|74605|20|43|25|PM|second|evening|| +74606|AAAAAAAAPGDCBAAA|74606|20|43|26|PM|second|evening|| +74607|AAAAAAAAAHDCBAAA|74607|20|43|27|PM|second|evening|| +74608|AAAAAAAABHDCBAAA|74608|20|43|28|PM|second|evening|| +74609|AAAAAAAACHDCBAAA|74609|20|43|29|PM|second|evening|| +74610|AAAAAAAADHDCBAAA|74610|20|43|30|PM|second|evening|| +74611|AAAAAAAAEHDCBAAA|74611|20|43|31|PM|second|evening|| +74612|AAAAAAAAFHDCBAAA|74612|20|43|32|PM|second|evening|| +74613|AAAAAAAAGHDCBAAA|74613|20|43|33|PM|second|evening|| +74614|AAAAAAAAHHDCBAAA|74614|20|43|34|PM|second|evening|| +74615|AAAAAAAAIHDCBAAA|74615|20|43|35|PM|second|evening|| +74616|AAAAAAAAJHDCBAAA|74616|20|43|36|PM|second|evening|| +74617|AAAAAAAAKHDCBAAA|74617|20|43|37|PM|second|evening|| +74618|AAAAAAAALHDCBAAA|74618|20|43|38|PM|second|evening|| +74619|AAAAAAAAMHDCBAAA|74619|20|43|39|PM|second|evening|| +74620|AAAAAAAANHDCBAAA|74620|20|43|40|PM|second|evening|| +74621|AAAAAAAAOHDCBAAA|74621|20|43|41|PM|second|evening|| +74622|AAAAAAAAPHDCBAAA|74622|20|43|42|PM|second|evening|| +74623|AAAAAAAAAIDCBAAA|74623|20|43|43|PM|second|evening|| +74624|AAAAAAAABIDCBAAA|74624|20|43|44|PM|second|evening|| +74625|AAAAAAAACIDCBAAA|74625|20|43|45|PM|second|evening|| +74626|AAAAAAAADIDCBAAA|74626|20|43|46|PM|second|evening|| +74627|AAAAAAAAEIDCBAAA|74627|20|43|47|PM|second|evening|| +74628|AAAAAAAAFIDCBAAA|74628|20|43|48|PM|second|evening|| +74629|AAAAAAAAGIDCBAAA|74629|20|43|49|PM|second|evening|| +74630|AAAAAAAAHIDCBAAA|74630|20|43|50|PM|second|evening|| +74631|AAAAAAAAIIDCBAAA|74631|20|43|51|PM|second|evening|| +74632|AAAAAAAAJIDCBAAA|74632|20|43|52|PM|second|evening|| +74633|AAAAAAAAKIDCBAAA|74633|20|43|53|PM|second|evening|| +74634|AAAAAAAALIDCBAAA|74634|20|43|54|PM|second|evening|| +74635|AAAAAAAAMIDCBAAA|74635|20|43|55|PM|second|evening|| +74636|AAAAAAAANIDCBAAA|74636|20|43|56|PM|second|evening|| +74637|AAAAAAAAOIDCBAAA|74637|20|43|57|PM|second|evening|| +74638|AAAAAAAAPIDCBAAA|74638|20|43|58|PM|second|evening|| +74639|AAAAAAAAAJDCBAAA|74639|20|43|59|PM|second|evening|| +74640|AAAAAAAABJDCBAAA|74640|20|44|0|PM|second|evening|| +74641|AAAAAAAACJDCBAAA|74641|20|44|1|PM|second|evening|| +74642|AAAAAAAADJDCBAAA|74642|20|44|2|PM|second|evening|| +74643|AAAAAAAAEJDCBAAA|74643|20|44|3|PM|second|evening|| +74644|AAAAAAAAFJDCBAAA|74644|20|44|4|PM|second|evening|| +74645|AAAAAAAAGJDCBAAA|74645|20|44|5|PM|second|evening|| +74646|AAAAAAAAHJDCBAAA|74646|20|44|6|PM|second|evening|| +74647|AAAAAAAAIJDCBAAA|74647|20|44|7|PM|second|evening|| +74648|AAAAAAAAJJDCBAAA|74648|20|44|8|PM|second|evening|| +74649|AAAAAAAAKJDCBAAA|74649|20|44|9|PM|second|evening|| +74650|AAAAAAAALJDCBAAA|74650|20|44|10|PM|second|evening|| +74651|AAAAAAAAMJDCBAAA|74651|20|44|11|PM|second|evening|| +74652|AAAAAAAANJDCBAAA|74652|20|44|12|PM|second|evening|| +74653|AAAAAAAAOJDCBAAA|74653|20|44|13|PM|second|evening|| +74654|AAAAAAAAPJDCBAAA|74654|20|44|14|PM|second|evening|| +74655|AAAAAAAAAKDCBAAA|74655|20|44|15|PM|second|evening|| +74656|AAAAAAAABKDCBAAA|74656|20|44|16|PM|second|evening|| +74657|AAAAAAAACKDCBAAA|74657|20|44|17|PM|second|evening|| +74658|AAAAAAAADKDCBAAA|74658|20|44|18|PM|second|evening|| +74659|AAAAAAAAEKDCBAAA|74659|20|44|19|PM|second|evening|| +74660|AAAAAAAAFKDCBAAA|74660|20|44|20|PM|second|evening|| +74661|AAAAAAAAGKDCBAAA|74661|20|44|21|PM|second|evening|| +74662|AAAAAAAAHKDCBAAA|74662|20|44|22|PM|second|evening|| +74663|AAAAAAAAIKDCBAAA|74663|20|44|23|PM|second|evening|| +74664|AAAAAAAAJKDCBAAA|74664|20|44|24|PM|second|evening|| +74665|AAAAAAAAKKDCBAAA|74665|20|44|25|PM|second|evening|| +74666|AAAAAAAALKDCBAAA|74666|20|44|26|PM|second|evening|| +74667|AAAAAAAAMKDCBAAA|74667|20|44|27|PM|second|evening|| +74668|AAAAAAAANKDCBAAA|74668|20|44|28|PM|second|evening|| +74669|AAAAAAAAOKDCBAAA|74669|20|44|29|PM|second|evening|| +74670|AAAAAAAAPKDCBAAA|74670|20|44|30|PM|second|evening|| +74671|AAAAAAAAALDCBAAA|74671|20|44|31|PM|second|evening|| +74672|AAAAAAAABLDCBAAA|74672|20|44|32|PM|second|evening|| +74673|AAAAAAAACLDCBAAA|74673|20|44|33|PM|second|evening|| +74674|AAAAAAAADLDCBAAA|74674|20|44|34|PM|second|evening|| +74675|AAAAAAAAELDCBAAA|74675|20|44|35|PM|second|evening|| +74676|AAAAAAAAFLDCBAAA|74676|20|44|36|PM|second|evening|| +74677|AAAAAAAAGLDCBAAA|74677|20|44|37|PM|second|evening|| +74678|AAAAAAAAHLDCBAAA|74678|20|44|38|PM|second|evening|| +74679|AAAAAAAAILDCBAAA|74679|20|44|39|PM|second|evening|| +74680|AAAAAAAAJLDCBAAA|74680|20|44|40|PM|second|evening|| +74681|AAAAAAAAKLDCBAAA|74681|20|44|41|PM|second|evening|| +74682|AAAAAAAALLDCBAAA|74682|20|44|42|PM|second|evening|| +74683|AAAAAAAAMLDCBAAA|74683|20|44|43|PM|second|evening|| +74684|AAAAAAAANLDCBAAA|74684|20|44|44|PM|second|evening|| +74685|AAAAAAAAOLDCBAAA|74685|20|44|45|PM|second|evening|| +74686|AAAAAAAAPLDCBAAA|74686|20|44|46|PM|second|evening|| +74687|AAAAAAAAAMDCBAAA|74687|20|44|47|PM|second|evening|| +74688|AAAAAAAABMDCBAAA|74688|20|44|48|PM|second|evening|| +74689|AAAAAAAACMDCBAAA|74689|20|44|49|PM|second|evening|| +74690|AAAAAAAADMDCBAAA|74690|20|44|50|PM|second|evening|| +74691|AAAAAAAAEMDCBAAA|74691|20|44|51|PM|second|evening|| +74692|AAAAAAAAFMDCBAAA|74692|20|44|52|PM|second|evening|| +74693|AAAAAAAAGMDCBAAA|74693|20|44|53|PM|second|evening|| +74694|AAAAAAAAHMDCBAAA|74694|20|44|54|PM|second|evening|| +74695|AAAAAAAAIMDCBAAA|74695|20|44|55|PM|second|evening|| +74696|AAAAAAAAJMDCBAAA|74696|20|44|56|PM|second|evening|| +74697|AAAAAAAAKMDCBAAA|74697|20|44|57|PM|second|evening|| +74698|AAAAAAAALMDCBAAA|74698|20|44|58|PM|second|evening|| +74699|AAAAAAAAMMDCBAAA|74699|20|44|59|PM|second|evening|| +74700|AAAAAAAANMDCBAAA|74700|20|45|0|PM|second|evening|| +74701|AAAAAAAAOMDCBAAA|74701|20|45|1|PM|second|evening|| +74702|AAAAAAAAPMDCBAAA|74702|20|45|2|PM|second|evening|| +74703|AAAAAAAAANDCBAAA|74703|20|45|3|PM|second|evening|| +74704|AAAAAAAABNDCBAAA|74704|20|45|4|PM|second|evening|| +74705|AAAAAAAACNDCBAAA|74705|20|45|5|PM|second|evening|| +74706|AAAAAAAADNDCBAAA|74706|20|45|6|PM|second|evening|| +74707|AAAAAAAAENDCBAAA|74707|20|45|7|PM|second|evening|| +74708|AAAAAAAAFNDCBAAA|74708|20|45|8|PM|second|evening|| +74709|AAAAAAAAGNDCBAAA|74709|20|45|9|PM|second|evening|| +74710|AAAAAAAAHNDCBAAA|74710|20|45|10|PM|second|evening|| +74711|AAAAAAAAINDCBAAA|74711|20|45|11|PM|second|evening|| +74712|AAAAAAAAJNDCBAAA|74712|20|45|12|PM|second|evening|| +74713|AAAAAAAAKNDCBAAA|74713|20|45|13|PM|second|evening|| +74714|AAAAAAAALNDCBAAA|74714|20|45|14|PM|second|evening|| +74715|AAAAAAAAMNDCBAAA|74715|20|45|15|PM|second|evening|| +74716|AAAAAAAANNDCBAAA|74716|20|45|16|PM|second|evening|| +74717|AAAAAAAAONDCBAAA|74717|20|45|17|PM|second|evening|| +74718|AAAAAAAAPNDCBAAA|74718|20|45|18|PM|second|evening|| +74719|AAAAAAAAAODCBAAA|74719|20|45|19|PM|second|evening|| +74720|AAAAAAAABODCBAAA|74720|20|45|20|PM|second|evening|| +74721|AAAAAAAACODCBAAA|74721|20|45|21|PM|second|evening|| +74722|AAAAAAAADODCBAAA|74722|20|45|22|PM|second|evening|| +74723|AAAAAAAAEODCBAAA|74723|20|45|23|PM|second|evening|| +74724|AAAAAAAAFODCBAAA|74724|20|45|24|PM|second|evening|| +74725|AAAAAAAAGODCBAAA|74725|20|45|25|PM|second|evening|| +74726|AAAAAAAAHODCBAAA|74726|20|45|26|PM|second|evening|| +74727|AAAAAAAAIODCBAAA|74727|20|45|27|PM|second|evening|| +74728|AAAAAAAAJODCBAAA|74728|20|45|28|PM|second|evening|| +74729|AAAAAAAAKODCBAAA|74729|20|45|29|PM|second|evening|| +74730|AAAAAAAALODCBAAA|74730|20|45|30|PM|second|evening|| +74731|AAAAAAAAMODCBAAA|74731|20|45|31|PM|second|evening|| +74732|AAAAAAAANODCBAAA|74732|20|45|32|PM|second|evening|| +74733|AAAAAAAAOODCBAAA|74733|20|45|33|PM|second|evening|| +74734|AAAAAAAAPODCBAAA|74734|20|45|34|PM|second|evening|| +74735|AAAAAAAAAPDCBAAA|74735|20|45|35|PM|second|evening|| +74736|AAAAAAAABPDCBAAA|74736|20|45|36|PM|second|evening|| +74737|AAAAAAAACPDCBAAA|74737|20|45|37|PM|second|evening|| +74738|AAAAAAAADPDCBAAA|74738|20|45|38|PM|second|evening|| +74739|AAAAAAAAEPDCBAAA|74739|20|45|39|PM|second|evening|| +74740|AAAAAAAAFPDCBAAA|74740|20|45|40|PM|second|evening|| +74741|AAAAAAAAGPDCBAAA|74741|20|45|41|PM|second|evening|| +74742|AAAAAAAAHPDCBAAA|74742|20|45|42|PM|second|evening|| +74743|AAAAAAAAIPDCBAAA|74743|20|45|43|PM|second|evening|| +74744|AAAAAAAAJPDCBAAA|74744|20|45|44|PM|second|evening|| +74745|AAAAAAAAKPDCBAAA|74745|20|45|45|PM|second|evening|| +74746|AAAAAAAALPDCBAAA|74746|20|45|46|PM|second|evening|| +74747|AAAAAAAAMPDCBAAA|74747|20|45|47|PM|second|evening|| +74748|AAAAAAAANPDCBAAA|74748|20|45|48|PM|second|evening|| +74749|AAAAAAAAOPDCBAAA|74749|20|45|49|PM|second|evening|| +74750|AAAAAAAAPPDCBAAA|74750|20|45|50|PM|second|evening|| +74751|AAAAAAAAAAECBAAA|74751|20|45|51|PM|second|evening|| +74752|AAAAAAAABAECBAAA|74752|20|45|52|PM|second|evening|| +74753|AAAAAAAACAECBAAA|74753|20|45|53|PM|second|evening|| +74754|AAAAAAAADAECBAAA|74754|20|45|54|PM|second|evening|| +74755|AAAAAAAAEAECBAAA|74755|20|45|55|PM|second|evening|| +74756|AAAAAAAAFAECBAAA|74756|20|45|56|PM|second|evening|| +74757|AAAAAAAAGAECBAAA|74757|20|45|57|PM|second|evening|| +74758|AAAAAAAAHAECBAAA|74758|20|45|58|PM|second|evening|| +74759|AAAAAAAAIAECBAAA|74759|20|45|59|PM|second|evening|| +74760|AAAAAAAAJAECBAAA|74760|20|46|0|PM|second|evening|| +74761|AAAAAAAAKAECBAAA|74761|20|46|1|PM|second|evening|| +74762|AAAAAAAALAECBAAA|74762|20|46|2|PM|second|evening|| +74763|AAAAAAAAMAECBAAA|74763|20|46|3|PM|second|evening|| +74764|AAAAAAAANAECBAAA|74764|20|46|4|PM|second|evening|| +74765|AAAAAAAAOAECBAAA|74765|20|46|5|PM|second|evening|| +74766|AAAAAAAAPAECBAAA|74766|20|46|6|PM|second|evening|| +74767|AAAAAAAAABECBAAA|74767|20|46|7|PM|second|evening|| +74768|AAAAAAAABBECBAAA|74768|20|46|8|PM|second|evening|| +74769|AAAAAAAACBECBAAA|74769|20|46|9|PM|second|evening|| +74770|AAAAAAAADBECBAAA|74770|20|46|10|PM|second|evening|| +74771|AAAAAAAAEBECBAAA|74771|20|46|11|PM|second|evening|| +74772|AAAAAAAAFBECBAAA|74772|20|46|12|PM|second|evening|| +74773|AAAAAAAAGBECBAAA|74773|20|46|13|PM|second|evening|| +74774|AAAAAAAAHBECBAAA|74774|20|46|14|PM|second|evening|| +74775|AAAAAAAAIBECBAAA|74775|20|46|15|PM|second|evening|| +74776|AAAAAAAAJBECBAAA|74776|20|46|16|PM|second|evening|| +74777|AAAAAAAAKBECBAAA|74777|20|46|17|PM|second|evening|| +74778|AAAAAAAALBECBAAA|74778|20|46|18|PM|second|evening|| +74779|AAAAAAAAMBECBAAA|74779|20|46|19|PM|second|evening|| +74780|AAAAAAAANBECBAAA|74780|20|46|20|PM|second|evening|| +74781|AAAAAAAAOBECBAAA|74781|20|46|21|PM|second|evening|| +74782|AAAAAAAAPBECBAAA|74782|20|46|22|PM|second|evening|| +74783|AAAAAAAAACECBAAA|74783|20|46|23|PM|second|evening|| +74784|AAAAAAAABCECBAAA|74784|20|46|24|PM|second|evening|| +74785|AAAAAAAACCECBAAA|74785|20|46|25|PM|second|evening|| +74786|AAAAAAAADCECBAAA|74786|20|46|26|PM|second|evening|| +74787|AAAAAAAAECECBAAA|74787|20|46|27|PM|second|evening|| +74788|AAAAAAAAFCECBAAA|74788|20|46|28|PM|second|evening|| +74789|AAAAAAAAGCECBAAA|74789|20|46|29|PM|second|evening|| +74790|AAAAAAAAHCECBAAA|74790|20|46|30|PM|second|evening|| +74791|AAAAAAAAICECBAAA|74791|20|46|31|PM|second|evening|| +74792|AAAAAAAAJCECBAAA|74792|20|46|32|PM|second|evening|| +74793|AAAAAAAAKCECBAAA|74793|20|46|33|PM|second|evening|| +74794|AAAAAAAALCECBAAA|74794|20|46|34|PM|second|evening|| +74795|AAAAAAAAMCECBAAA|74795|20|46|35|PM|second|evening|| +74796|AAAAAAAANCECBAAA|74796|20|46|36|PM|second|evening|| +74797|AAAAAAAAOCECBAAA|74797|20|46|37|PM|second|evening|| +74798|AAAAAAAAPCECBAAA|74798|20|46|38|PM|second|evening|| +74799|AAAAAAAAADECBAAA|74799|20|46|39|PM|second|evening|| +74800|AAAAAAAABDECBAAA|74800|20|46|40|PM|second|evening|| +74801|AAAAAAAACDECBAAA|74801|20|46|41|PM|second|evening|| +74802|AAAAAAAADDECBAAA|74802|20|46|42|PM|second|evening|| +74803|AAAAAAAAEDECBAAA|74803|20|46|43|PM|second|evening|| +74804|AAAAAAAAFDECBAAA|74804|20|46|44|PM|second|evening|| +74805|AAAAAAAAGDECBAAA|74805|20|46|45|PM|second|evening|| +74806|AAAAAAAAHDECBAAA|74806|20|46|46|PM|second|evening|| +74807|AAAAAAAAIDECBAAA|74807|20|46|47|PM|second|evening|| +74808|AAAAAAAAJDECBAAA|74808|20|46|48|PM|second|evening|| +74809|AAAAAAAAKDECBAAA|74809|20|46|49|PM|second|evening|| +74810|AAAAAAAALDECBAAA|74810|20|46|50|PM|second|evening|| +74811|AAAAAAAAMDECBAAA|74811|20|46|51|PM|second|evening|| +74812|AAAAAAAANDECBAAA|74812|20|46|52|PM|second|evening|| +74813|AAAAAAAAODECBAAA|74813|20|46|53|PM|second|evening|| +74814|AAAAAAAAPDECBAAA|74814|20|46|54|PM|second|evening|| +74815|AAAAAAAAAEECBAAA|74815|20|46|55|PM|second|evening|| +74816|AAAAAAAABEECBAAA|74816|20|46|56|PM|second|evening|| +74817|AAAAAAAACEECBAAA|74817|20|46|57|PM|second|evening|| +74818|AAAAAAAADEECBAAA|74818|20|46|58|PM|second|evening|| +74819|AAAAAAAAEEECBAAA|74819|20|46|59|PM|second|evening|| +74820|AAAAAAAAFEECBAAA|74820|20|47|0|PM|second|evening|| +74821|AAAAAAAAGEECBAAA|74821|20|47|1|PM|second|evening|| +74822|AAAAAAAAHEECBAAA|74822|20|47|2|PM|second|evening|| +74823|AAAAAAAAIEECBAAA|74823|20|47|3|PM|second|evening|| +74824|AAAAAAAAJEECBAAA|74824|20|47|4|PM|second|evening|| +74825|AAAAAAAAKEECBAAA|74825|20|47|5|PM|second|evening|| +74826|AAAAAAAALEECBAAA|74826|20|47|6|PM|second|evening|| +74827|AAAAAAAAMEECBAAA|74827|20|47|7|PM|second|evening|| +74828|AAAAAAAANEECBAAA|74828|20|47|8|PM|second|evening|| +74829|AAAAAAAAOEECBAAA|74829|20|47|9|PM|second|evening|| +74830|AAAAAAAAPEECBAAA|74830|20|47|10|PM|second|evening|| +74831|AAAAAAAAAFECBAAA|74831|20|47|11|PM|second|evening|| +74832|AAAAAAAABFECBAAA|74832|20|47|12|PM|second|evening|| +74833|AAAAAAAACFECBAAA|74833|20|47|13|PM|second|evening|| +74834|AAAAAAAADFECBAAA|74834|20|47|14|PM|second|evening|| +74835|AAAAAAAAEFECBAAA|74835|20|47|15|PM|second|evening|| +74836|AAAAAAAAFFECBAAA|74836|20|47|16|PM|second|evening|| +74837|AAAAAAAAGFECBAAA|74837|20|47|17|PM|second|evening|| +74838|AAAAAAAAHFECBAAA|74838|20|47|18|PM|second|evening|| +74839|AAAAAAAAIFECBAAA|74839|20|47|19|PM|second|evening|| +74840|AAAAAAAAJFECBAAA|74840|20|47|20|PM|second|evening|| +74841|AAAAAAAAKFECBAAA|74841|20|47|21|PM|second|evening|| +74842|AAAAAAAALFECBAAA|74842|20|47|22|PM|second|evening|| +74843|AAAAAAAAMFECBAAA|74843|20|47|23|PM|second|evening|| +74844|AAAAAAAANFECBAAA|74844|20|47|24|PM|second|evening|| +74845|AAAAAAAAOFECBAAA|74845|20|47|25|PM|second|evening|| +74846|AAAAAAAAPFECBAAA|74846|20|47|26|PM|second|evening|| +74847|AAAAAAAAAGECBAAA|74847|20|47|27|PM|second|evening|| +74848|AAAAAAAABGECBAAA|74848|20|47|28|PM|second|evening|| +74849|AAAAAAAACGECBAAA|74849|20|47|29|PM|second|evening|| +74850|AAAAAAAADGECBAAA|74850|20|47|30|PM|second|evening|| +74851|AAAAAAAAEGECBAAA|74851|20|47|31|PM|second|evening|| +74852|AAAAAAAAFGECBAAA|74852|20|47|32|PM|second|evening|| +74853|AAAAAAAAGGECBAAA|74853|20|47|33|PM|second|evening|| +74854|AAAAAAAAHGECBAAA|74854|20|47|34|PM|second|evening|| +74855|AAAAAAAAIGECBAAA|74855|20|47|35|PM|second|evening|| +74856|AAAAAAAAJGECBAAA|74856|20|47|36|PM|second|evening|| +74857|AAAAAAAAKGECBAAA|74857|20|47|37|PM|second|evening|| +74858|AAAAAAAALGECBAAA|74858|20|47|38|PM|second|evening|| +74859|AAAAAAAAMGECBAAA|74859|20|47|39|PM|second|evening|| +74860|AAAAAAAANGECBAAA|74860|20|47|40|PM|second|evening|| +74861|AAAAAAAAOGECBAAA|74861|20|47|41|PM|second|evening|| +74862|AAAAAAAAPGECBAAA|74862|20|47|42|PM|second|evening|| +74863|AAAAAAAAAHECBAAA|74863|20|47|43|PM|second|evening|| +74864|AAAAAAAABHECBAAA|74864|20|47|44|PM|second|evening|| +74865|AAAAAAAACHECBAAA|74865|20|47|45|PM|second|evening|| +74866|AAAAAAAADHECBAAA|74866|20|47|46|PM|second|evening|| +74867|AAAAAAAAEHECBAAA|74867|20|47|47|PM|second|evening|| +74868|AAAAAAAAFHECBAAA|74868|20|47|48|PM|second|evening|| +74869|AAAAAAAAGHECBAAA|74869|20|47|49|PM|second|evening|| +74870|AAAAAAAAHHECBAAA|74870|20|47|50|PM|second|evening|| +74871|AAAAAAAAIHECBAAA|74871|20|47|51|PM|second|evening|| +74872|AAAAAAAAJHECBAAA|74872|20|47|52|PM|second|evening|| +74873|AAAAAAAAKHECBAAA|74873|20|47|53|PM|second|evening|| +74874|AAAAAAAALHECBAAA|74874|20|47|54|PM|second|evening|| +74875|AAAAAAAAMHECBAAA|74875|20|47|55|PM|second|evening|| +74876|AAAAAAAANHECBAAA|74876|20|47|56|PM|second|evening|| +74877|AAAAAAAAOHECBAAA|74877|20|47|57|PM|second|evening|| +74878|AAAAAAAAPHECBAAA|74878|20|47|58|PM|second|evening|| +74879|AAAAAAAAAIECBAAA|74879|20|47|59|PM|second|evening|| +74880|AAAAAAAABIECBAAA|74880|20|48|0|PM|second|evening|| +74881|AAAAAAAACIECBAAA|74881|20|48|1|PM|second|evening|| +74882|AAAAAAAADIECBAAA|74882|20|48|2|PM|second|evening|| +74883|AAAAAAAAEIECBAAA|74883|20|48|3|PM|second|evening|| +74884|AAAAAAAAFIECBAAA|74884|20|48|4|PM|second|evening|| +74885|AAAAAAAAGIECBAAA|74885|20|48|5|PM|second|evening|| +74886|AAAAAAAAHIECBAAA|74886|20|48|6|PM|second|evening|| +74887|AAAAAAAAIIECBAAA|74887|20|48|7|PM|second|evening|| +74888|AAAAAAAAJIECBAAA|74888|20|48|8|PM|second|evening|| +74889|AAAAAAAAKIECBAAA|74889|20|48|9|PM|second|evening|| +74890|AAAAAAAALIECBAAA|74890|20|48|10|PM|second|evening|| +74891|AAAAAAAAMIECBAAA|74891|20|48|11|PM|second|evening|| +74892|AAAAAAAANIECBAAA|74892|20|48|12|PM|second|evening|| +74893|AAAAAAAAOIECBAAA|74893|20|48|13|PM|second|evening|| +74894|AAAAAAAAPIECBAAA|74894|20|48|14|PM|second|evening|| +74895|AAAAAAAAAJECBAAA|74895|20|48|15|PM|second|evening|| +74896|AAAAAAAABJECBAAA|74896|20|48|16|PM|second|evening|| +74897|AAAAAAAACJECBAAA|74897|20|48|17|PM|second|evening|| +74898|AAAAAAAADJECBAAA|74898|20|48|18|PM|second|evening|| +74899|AAAAAAAAEJECBAAA|74899|20|48|19|PM|second|evening|| +74900|AAAAAAAAFJECBAAA|74900|20|48|20|PM|second|evening|| +74901|AAAAAAAAGJECBAAA|74901|20|48|21|PM|second|evening|| +74902|AAAAAAAAHJECBAAA|74902|20|48|22|PM|second|evening|| +74903|AAAAAAAAIJECBAAA|74903|20|48|23|PM|second|evening|| +74904|AAAAAAAAJJECBAAA|74904|20|48|24|PM|second|evening|| +74905|AAAAAAAAKJECBAAA|74905|20|48|25|PM|second|evening|| +74906|AAAAAAAALJECBAAA|74906|20|48|26|PM|second|evening|| +74907|AAAAAAAAMJECBAAA|74907|20|48|27|PM|second|evening|| +74908|AAAAAAAANJECBAAA|74908|20|48|28|PM|second|evening|| +74909|AAAAAAAAOJECBAAA|74909|20|48|29|PM|second|evening|| +74910|AAAAAAAAPJECBAAA|74910|20|48|30|PM|second|evening|| +74911|AAAAAAAAAKECBAAA|74911|20|48|31|PM|second|evening|| +74912|AAAAAAAABKECBAAA|74912|20|48|32|PM|second|evening|| +74913|AAAAAAAACKECBAAA|74913|20|48|33|PM|second|evening|| +74914|AAAAAAAADKECBAAA|74914|20|48|34|PM|second|evening|| +74915|AAAAAAAAEKECBAAA|74915|20|48|35|PM|second|evening|| +74916|AAAAAAAAFKECBAAA|74916|20|48|36|PM|second|evening|| +74917|AAAAAAAAGKECBAAA|74917|20|48|37|PM|second|evening|| +74918|AAAAAAAAHKECBAAA|74918|20|48|38|PM|second|evening|| +74919|AAAAAAAAIKECBAAA|74919|20|48|39|PM|second|evening|| +74920|AAAAAAAAJKECBAAA|74920|20|48|40|PM|second|evening|| +74921|AAAAAAAAKKECBAAA|74921|20|48|41|PM|second|evening|| +74922|AAAAAAAALKECBAAA|74922|20|48|42|PM|second|evening|| +74923|AAAAAAAAMKECBAAA|74923|20|48|43|PM|second|evening|| +74924|AAAAAAAANKECBAAA|74924|20|48|44|PM|second|evening|| +74925|AAAAAAAAOKECBAAA|74925|20|48|45|PM|second|evening|| +74926|AAAAAAAAPKECBAAA|74926|20|48|46|PM|second|evening|| +74927|AAAAAAAAALECBAAA|74927|20|48|47|PM|second|evening|| +74928|AAAAAAAABLECBAAA|74928|20|48|48|PM|second|evening|| +74929|AAAAAAAACLECBAAA|74929|20|48|49|PM|second|evening|| +74930|AAAAAAAADLECBAAA|74930|20|48|50|PM|second|evening|| +74931|AAAAAAAAELECBAAA|74931|20|48|51|PM|second|evening|| +74932|AAAAAAAAFLECBAAA|74932|20|48|52|PM|second|evening|| +74933|AAAAAAAAGLECBAAA|74933|20|48|53|PM|second|evening|| +74934|AAAAAAAAHLECBAAA|74934|20|48|54|PM|second|evening|| +74935|AAAAAAAAILECBAAA|74935|20|48|55|PM|second|evening|| +74936|AAAAAAAAJLECBAAA|74936|20|48|56|PM|second|evening|| +74937|AAAAAAAAKLECBAAA|74937|20|48|57|PM|second|evening|| +74938|AAAAAAAALLECBAAA|74938|20|48|58|PM|second|evening|| +74939|AAAAAAAAMLECBAAA|74939|20|48|59|PM|second|evening|| +74940|AAAAAAAANLECBAAA|74940|20|49|0|PM|second|evening|| +74941|AAAAAAAAOLECBAAA|74941|20|49|1|PM|second|evening|| +74942|AAAAAAAAPLECBAAA|74942|20|49|2|PM|second|evening|| +74943|AAAAAAAAAMECBAAA|74943|20|49|3|PM|second|evening|| +74944|AAAAAAAABMECBAAA|74944|20|49|4|PM|second|evening|| +74945|AAAAAAAACMECBAAA|74945|20|49|5|PM|second|evening|| +74946|AAAAAAAADMECBAAA|74946|20|49|6|PM|second|evening|| +74947|AAAAAAAAEMECBAAA|74947|20|49|7|PM|second|evening|| +74948|AAAAAAAAFMECBAAA|74948|20|49|8|PM|second|evening|| +74949|AAAAAAAAGMECBAAA|74949|20|49|9|PM|second|evening|| +74950|AAAAAAAAHMECBAAA|74950|20|49|10|PM|second|evening|| +74951|AAAAAAAAIMECBAAA|74951|20|49|11|PM|second|evening|| +74952|AAAAAAAAJMECBAAA|74952|20|49|12|PM|second|evening|| +74953|AAAAAAAAKMECBAAA|74953|20|49|13|PM|second|evening|| +74954|AAAAAAAALMECBAAA|74954|20|49|14|PM|second|evening|| +74955|AAAAAAAAMMECBAAA|74955|20|49|15|PM|second|evening|| +74956|AAAAAAAANMECBAAA|74956|20|49|16|PM|second|evening|| +74957|AAAAAAAAOMECBAAA|74957|20|49|17|PM|second|evening|| +74958|AAAAAAAAPMECBAAA|74958|20|49|18|PM|second|evening|| +74959|AAAAAAAAANECBAAA|74959|20|49|19|PM|second|evening|| +74960|AAAAAAAABNECBAAA|74960|20|49|20|PM|second|evening|| +74961|AAAAAAAACNECBAAA|74961|20|49|21|PM|second|evening|| +74962|AAAAAAAADNECBAAA|74962|20|49|22|PM|second|evening|| +74963|AAAAAAAAENECBAAA|74963|20|49|23|PM|second|evening|| +74964|AAAAAAAAFNECBAAA|74964|20|49|24|PM|second|evening|| +74965|AAAAAAAAGNECBAAA|74965|20|49|25|PM|second|evening|| +74966|AAAAAAAAHNECBAAA|74966|20|49|26|PM|second|evening|| +74967|AAAAAAAAINECBAAA|74967|20|49|27|PM|second|evening|| +74968|AAAAAAAAJNECBAAA|74968|20|49|28|PM|second|evening|| +74969|AAAAAAAAKNECBAAA|74969|20|49|29|PM|second|evening|| +74970|AAAAAAAALNECBAAA|74970|20|49|30|PM|second|evening|| +74971|AAAAAAAAMNECBAAA|74971|20|49|31|PM|second|evening|| +74972|AAAAAAAANNECBAAA|74972|20|49|32|PM|second|evening|| +74973|AAAAAAAAONECBAAA|74973|20|49|33|PM|second|evening|| +74974|AAAAAAAAPNECBAAA|74974|20|49|34|PM|second|evening|| +74975|AAAAAAAAAOECBAAA|74975|20|49|35|PM|second|evening|| +74976|AAAAAAAABOECBAAA|74976|20|49|36|PM|second|evening|| +74977|AAAAAAAACOECBAAA|74977|20|49|37|PM|second|evening|| +74978|AAAAAAAADOECBAAA|74978|20|49|38|PM|second|evening|| +74979|AAAAAAAAEOECBAAA|74979|20|49|39|PM|second|evening|| +74980|AAAAAAAAFOECBAAA|74980|20|49|40|PM|second|evening|| +74981|AAAAAAAAGOECBAAA|74981|20|49|41|PM|second|evening|| +74982|AAAAAAAAHOECBAAA|74982|20|49|42|PM|second|evening|| +74983|AAAAAAAAIOECBAAA|74983|20|49|43|PM|second|evening|| +74984|AAAAAAAAJOECBAAA|74984|20|49|44|PM|second|evening|| +74985|AAAAAAAAKOECBAAA|74985|20|49|45|PM|second|evening|| +74986|AAAAAAAALOECBAAA|74986|20|49|46|PM|second|evening|| +74987|AAAAAAAAMOECBAAA|74987|20|49|47|PM|second|evening|| +74988|AAAAAAAANOECBAAA|74988|20|49|48|PM|second|evening|| +74989|AAAAAAAAOOECBAAA|74989|20|49|49|PM|second|evening|| +74990|AAAAAAAAPOECBAAA|74990|20|49|50|PM|second|evening|| +74991|AAAAAAAAAPECBAAA|74991|20|49|51|PM|second|evening|| +74992|AAAAAAAABPECBAAA|74992|20|49|52|PM|second|evening|| +74993|AAAAAAAACPECBAAA|74993|20|49|53|PM|second|evening|| +74994|AAAAAAAADPECBAAA|74994|20|49|54|PM|second|evening|| +74995|AAAAAAAAEPECBAAA|74995|20|49|55|PM|second|evening|| +74996|AAAAAAAAFPECBAAA|74996|20|49|56|PM|second|evening|| +74997|AAAAAAAAGPECBAAA|74997|20|49|57|PM|second|evening|| +74998|AAAAAAAAHPECBAAA|74998|20|49|58|PM|second|evening|| +74999|AAAAAAAAIPECBAAA|74999|20|49|59|PM|second|evening|| +75000|AAAAAAAAJPECBAAA|75000|20|50|0|PM|second|evening|| +75001|AAAAAAAAKPECBAAA|75001|20|50|1|PM|second|evening|| +75002|AAAAAAAALPECBAAA|75002|20|50|2|PM|second|evening|| +75003|AAAAAAAAMPECBAAA|75003|20|50|3|PM|second|evening|| +75004|AAAAAAAANPECBAAA|75004|20|50|4|PM|second|evening|| +75005|AAAAAAAAOPECBAAA|75005|20|50|5|PM|second|evening|| +75006|AAAAAAAAPPECBAAA|75006|20|50|6|PM|second|evening|| +75007|AAAAAAAAAAFCBAAA|75007|20|50|7|PM|second|evening|| +75008|AAAAAAAABAFCBAAA|75008|20|50|8|PM|second|evening|| +75009|AAAAAAAACAFCBAAA|75009|20|50|9|PM|second|evening|| +75010|AAAAAAAADAFCBAAA|75010|20|50|10|PM|second|evening|| +75011|AAAAAAAAEAFCBAAA|75011|20|50|11|PM|second|evening|| +75012|AAAAAAAAFAFCBAAA|75012|20|50|12|PM|second|evening|| +75013|AAAAAAAAGAFCBAAA|75013|20|50|13|PM|second|evening|| +75014|AAAAAAAAHAFCBAAA|75014|20|50|14|PM|second|evening|| +75015|AAAAAAAAIAFCBAAA|75015|20|50|15|PM|second|evening|| +75016|AAAAAAAAJAFCBAAA|75016|20|50|16|PM|second|evening|| +75017|AAAAAAAAKAFCBAAA|75017|20|50|17|PM|second|evening|| +75018|AAAAAAAALAFCBAAA|75018|20|50|18|PM|second|evening|| +75019|AAAAAAAAMAFCBAAA|75019|20|50|19|PM|second|evening|| +75020|AAAAAAAANAFCBAAA|75020|20|50|20|PM|second|evening|| +75021|AAAAAAAAOAFCBAAA|75021|20|50|21|PM|second|evening|| +75022|AAAAAAAAPAFCBAAA|75022|20|50|22|PM|second|evening|| +75023|AAAAAAAAABFCBAAA|75023|20|50|23|PM|second|evening|| +75024|AAAAAAAABBFCBAAA|75024|20|50|24|PM|second|evening|| +75025|AAAAAAAACBFCBAAA|75025|20|50|25|PM|second|evening|| +75026|AAAAAAAADBFCBAAA|75026|20|50|26|PM|second|evening|| +75027|AAAAAAAAEBFCBAAA|75027|20|50|27|PM|second|evening|| +75028|AAAAAAAAFBFCBAAA|75028|20|50|28|PM|second|evening|| +75029|AAAAAAAAGBFCBAAA|75029|20|50|29|PM|second|evening|| +75030|AAAAAAAAHBFCBAAA|75030|20|50|30|PM|second|evening|| +75031|AAAAAAAAIBFCBAAA|75031|20|50|31|PM|second|evening|| +75032|AAAAAAAAJBFCBAAA|75032|20|50|32|PM|second|evening|| +75033|AAAAAAAAKBFCBAAA|75033|20|50|33|PM|second|evening|| +75034|AAAAAAAALBFCBAAA|75034|20|50|34|PM|second|evening|| +75035|AAAAAAAAMBFCBAAA|75035|20|50|35|PM|second|evening|| +75036|AAAAAAAANBFCBAAA|75036|20|50|36|PM|second|evening|| +75037|AAAAAAAAOBFCBAAA|75037|20|50|37|PM|second|evening|| +75038|AAAAAAAAPBFCBAAA|75038|20|50|38|PM|second|evening|| +75039|AAAAAAAAACFCBAAA|75039|20|50|39|PM|second|evening|| +75040|AAAAAAAABCFCBAAA|75040|20|50|40|PM|second|evening|| +75041|AAAAAAAACCFCBAAA|75041|20|50|41|PM|second|evening|| +75042|AAAAAAAADCFCBAAA|75042|20|50|42|PM|second|evening|| +75043|AAAAAAAAECFCBAAA|75043|20|50|43|PM|second|evening|| +75044|AAAAAAAAFCFCBAAA|75044|20|50|44|PM|second|evening|| +75045|AAAAAAAAGCFCBAAA|75045|20|50|45|PM|second|evening|| +75046|AAAAAAAAHCFCBAAA|75046|20|50|46|PM|second|evening|| +75047|AAAAAAAAICFCBAAA|75047|20|50|47|PM|second|evening|| +75048|AAAAAAAAJCFCBAAA|75048|20|50|48|PM|second|evening|| +75049|AAAAAAAAKCFCBAAA|75049|20|50|49|PM|second|evening|| +75050|AAAAAAAALCFCBAAA|75050|20|50|50|PM|second|evening|| +75051|AAAAAAAAMCFCBAAA|75051|20|50|51|PM|second|evening|| +75052|AAAAAAAANCFCBAAA|75052|20|50|52|PM|second|evening|| +75053|AAAAAAAAOCFCBAAA|75053|20|50|53|PM|second|evening|| +75054|AAAAAAAAPCFCBAAA|75054|20|50|54|PM|second|evening|| +75055|AAAAAAAAADFCBAAA|75055|20|50|55|PM|second|evening|| +75056|AAAAAAAABDFCBAAA|75056|20|50|56|PM|second|evening|| +75057|AAAAAAAACDFCBAAA|75057|20|50|57|PM|second|evening|| +75058|AAAAAAAADDFCBAAA|75058|20|50|58|PM|second|evening|| +75059|AAAAAAAAEDFCBAAA|75059|20|50|59|PM|second|evening|| +75060|AAAAAAAAFDFCBAAA|75060|20|51|0|PM|second|evening|| +75061|AAAAAAAAGDFCBAAA|75061|20|51|1|PM|second|evening|| +75062|AAAAAAAAHDFCBAAA|75062|20|51|2|PM|second|evening|| +75063|AAAAAAAAIDFCBAAA|75063|20|51|3|PM|second|evening|| +75064|AAAAAAAAJDFCBAAA|75064|20|51|4|PM|second|evening|| +75065|AAAAAAAAKDFCBAAA|75065|20|51|5|PM|second|evening|| +75066|AAAAAAAALDFCBAAA|75066|20|51|6|PM|second|evening|| +75067|AAAAAAAAMDFCBAAA|75067|20|51|7|PM|second|evening|| +75068|AAAAAAAANDFCBAAA|75068|20|51|8|PM|second|evening|| +75069|AAAAAAAAODFCBAAA|75069|20|51|9|PM|second|evening|| +75070|AAAAAAAAPDFCBAAA|75070|20|51|10|PM|second|evening|| +75071|AAAAAAAAAEFCBAAA|75071|20|51|11|PM|second|evening|| +75072|AAAAAAAABEFCBAAA|75072|20|51|12|PM|second|evening|| +75073|AAAAAAAACEFCBAAA|75073|20|51|13|PM|second|evening|| +75074|AAAAAAAADEFCBAAA|75074|20|51|14|PM|second|evening|| +75075|AAAAAAAAEEFCBAAA|75075|20|51|15|PM|second|evening|| +75076|AAAAAAAAFEFCBAAA|75076|20|51|16|PM|second|evening|| +75077|AAAAAAAAGEFCBAAA|75077|20|51|17|PM|second|evening|| +75078|AAAAAAAAHEFCBAAA|75078|20|51|18|PM|second|evening|| +75079|AAAAAAAAIEFCBAAA|75079|20|51|19|PM|second|evening|| +75080|AAAAAAAAJEFCBAAA|75080|20|51|20|PM|second|evening|| +75081|AAAAAAAAKEFCBAAA|75081|20|51|21|PM|second|evening|| +75082|AAAAAAAALEFCBAAA|75082|20|51|22|PM|second|evening|| +75083|AAAAAAAAMEFCBAAA|75083|20|51|23|PM|second|evening|| +75084|AAAAAAAANEFCBAAA|75084|20|51|24|PM|second|evening|| +75085|AAAAAAAAOEFCBAAA|75085|20|51|25|PM|second|evening|| +75086|AAAAAAAAPEFCBAAA|75086|20|51|26|PM|second|evening|| +75087|AAAAAAAAAFFCBAAA|75087|20|51|27|PM|second|evening|| +75088|AAAAAAAABFFCBAAA|75088|20|51|28|PM|second|evening|| +75089|AAAAAAAACFFCBAAA|75089|20|51|29|PM|second|evening|| +75090|AAAAAAAADFFCBAAA|75090|20|51|30|PM|second|evening|| +75091|AAAAAAAAEFFCBAAA|75091|20|51|31|PM|second|evening|| +75092|AAAAAAAAFFFCBAAA|75092|20|51|32|PM|second|evening|| +75093|AAAAAAAAGFFCBAAA|75093|20|51|33|PM|second|evening|| +75094|AAAAAAAAHFFCBAAA|75094|20|51|34|PM|second|evening|| +75095|AAAAAAAAIFFCBAAA|75095|20|51|35|PM|second|evening|| +75096|AAAAAAAAJFFCBAAA|75096|20|51|36|PM|second|evening|| +75097|AAAAAAAAKFFCBAAA|75097|20|51|37|PM|second|evening|| +75098|AAAAAAAALFFCBAAA|75098|20|51|38|PM|second|evening|| +75099|AAAAAAAAMFFCBAAA|75099|20|51|39|PM|second|evening|| +75100|AAAAAAAANFFCBAAA|75100|20|51|40|PM|second|evening|| +75101|AAAAAAAAOFFCBAAA|75101|20|51|41|PM|second|evening|| +75102|AAAAAAAAPFFCBAAA|75102|20|51|42|PM|second|evening|| +75103|AAAAAAAAAGFCBAAA|75103|20|51|43|PM|second|evening|| +75104|AAAAAAAABGFCBAAA|75104|20|51|44|PM|second|evening|| +75105|AAAAAAAACGFCBAAA|75105|20|51|45|PM|second|evening|| +75106|AAAAAAAADGFCBAAA|75106|20|51|46|PM|second|evening|| +75107|AAAAAAAAEGFCBAAA|75107|20|51|47|PM|second|evening|| +75108|AAAAAAAAFGFCBAAA|75108|20|51|48|PM|second|evening|| +75109|AAAAAAAAGGFCBAAA|75109|20|51|49|PM|second|evening|| +75110|AAAAAAAAHGFCBAAA|75110|20|51|50|PM|second|evening|| +75111|AAAAAAAAIGFCBAAA|75111|20|51|51|PM|second|evening|| +75112|AAAAAAAAJGFCBAAA|75112|20|51|52|PM|second|evening|| +75113|AAAAAAAAKGFCBAAA|75113|20|51|53|PM|second|evening|| +75114|AAAAAAAALGFCBAAA|75114|20|51|54|PM|second|evening|| +75115|AAAAAAAAMGFCBAAA|75115|20|51|55|PM|second|evening|| +75116|AAAAAAAANGFCBAAA|75116|20|51|56|PM|second|evening|| +75117|AAAAAAAAOGFCBAAA|75117|20|51|57|PM|second|evening|| +75118|AAAAAAAAPGFCBAAA|75118|20|51|58|PM|second|evening|| +75119|AAAAAAAAAHFCBAAA|75119|20|51|59|PM|second|evening|| +75120|AAAAAAAABHFCBAAA|75120|20|52|0|PM|second|evening|| +75121|AAAAAAAACHFCBAAA|75121|20|52|1|PM|second|evening|| +75122|AAAAAAAADHFCBAAA|75122|20|52|2|PM|second|evening|| +75123|AAAAAAAAEHFCBAAA|75123|20|52|3|PM|second|evening|| +75124|AAAAAAAAFHFCBAAA|75124|20|52|4|PM|second|evening|| +75125|AAAAAAAAGHFCBAAA|75125|20|52|5|PM|second|evening|| +75126|AAAAAAAAHHFCBAAA|75126|20|52|6|PM|second|evening|| +75127|AAAAAAAAIHFCBAAA|75127|20|52|7|PM|second|evening|| +75128|AAAAAAAAJHFCBAAA|75128|20|52|8|PM|second|evening|| +75129|AAAAAAAAKHFCBAAA|75129|20|52|9|PM|second|evening|| +75130|AAAAAAAALHFCBAAA|75130|20|52|10|PM|second|evening|| +75131|AAAAAAAAMHFCBAAA|75131|20|52|11|PM|second|evening|| +75132|AAAAAAAANHFCBAAA|75132|20|52|12|PM|second|evening|| +75133|AAAAAAAAOHFCBAAA|75133|20|52|13|PM|second|evening|| +75134|AAAAAAAAPHFCBAAA|75134|20|52|14|PM|second|evening|| +75135|AAAAAAAAAIFCBAAA|75135|20|52|15|PM|second|evening|| +75136|AAAAAAAABIFCBAAA|75136|20|52|16|PM|second|evening|| +75137|AAAAAAAACIFCBAAA|75137|20|52|17|PM|second|evening|| +75138|AAAAAAAADIFCBAAA|75138|20|52|18|PM|second|evening|| +75139|AAAAAAAAEIFCBAAA|75139|20|52|19|PM|second|evening|| +75140|AAAAAAAAFIFCBAAA|75140|20|52|20|PM|second|evening|| +75141|AAAAAAAAGIFCBAAA|75141|20|52|21|PM|second|evening|| +75142|AAAAAAAAHIFCBAAA|75142|20|52|22|PM|second|evening|| +75143|AAAAAAAAIIFCBAAA|75143|20|52|23|PM|second|evening|| +75144|AAAAAAAAJIFCBAAA|75144|20|52|24|PM|second|evening|| +75145|AAAAAAAAKIFCBAAA|75145|20|52|25|PM|second|evening|| +75146|AAAAAAAALIFCBAAA|75146|20|52|26|PM|second|evening|| +75147|AAAAAAAAMIFCBAAA|75147|20|52|27|PM|second|evening|| +75148|AAAAAAAANIFCBAAA|75148|20|52|28|PM|second|evening|| +75149|AAAAAAAAOIFCBAAA|75149|20|52|29|PM|second|evening|| +75150|AAAAAAAAPIFCBAAA|75150|20|52|30|PM|second|evening|| +75151|AAAAAAAAAJFCBAAA|75151|20|52|31|PM|second|evening|| +75152|AAAAAAAABJFCBAAA|75152|20|52|32|PM|second|evening|| +75153|AAAAAAAACJFCBAAA|75153|20|52|33|PM|second|evening|| +75154|AAAAAAAADJFCBAAA|75154|20|52|34|PM|second|evening|| +75155|AAAAAAAAEJFCBAAA|75155|20|52|35|PM|second|evening|| +75156|AAAAAAAAFJFCBAAA|75156|20|52|36|PM|second|evening|| +75157|AAAAAAAAGJFCBAAA|75157|20|52|37|PM|second|evening|| +75158|AAAAAAAAHJFCBAAA|75158|20|52|38|PM|second|evening|| +75159|AAAAAAAAIJFCBAAA|75159|20|52|39|PM|second|evening|| +75160|AAAAAAAAJJFCBAAA|75160|20|52|40|PM|second|evening|| +75161|AAAAAAAAKJFCBAAA|75161|20|52|41|PM|second|evening|| +75162|AAAAAAAALJFCBAAA|75162|20|52|42|PM|second|evening|| +75163|AAAAAAAAMJFCBAAA|75163|20|52|43|PM|second|evening|| +75164|AAAAAAAANJFCBAAA|75164|20|52|44|PM|second|evening|| +75165|AAAAAAAAOJFCBAAA|75165|20|52|45|PM|second|evening|| +75166|AAAAAAAAPJFCBAAA|75166|20|52|46|PM|second|evening|| +75167|AAAAAAAAAKFCBAAA|75167|20|52|47|PM|second|evening|| +75168|AAAAAAAABKFCBAAA|75168|20|52|48|PM|second|evening|| +75169|AAAAAAAACKFCBAAA|75169|20|52|49|PM|second|evening|| +75170|AAAAAAAADKFCBAAA|75170|20|52|50|PM|second|evening|| +75171|AAAAAAAAEKFCBAAA|75171|20|52|51|PM|second|evening|| +75172|AAAAAAAAFKFCBAAA|75172|20|52|52|PM|second|evening|| +75173|AAAAAAAAGKFCBAAA|75173|20|52|53|PM|second|evening|| +75174|AAAAAAAAHKFCBAAA|75174|20|52|54|PM|second|evening|| +75175|AAAAAAAAIKFCBAAA|75175|20|52|55|PM|second|evening|| +75176|AAAAAAAAJKFCBAAA|75176|20|52|56|PM|second|evening|| +75177|AAAAAAAAKKFCBAAA|75177|20|52|57|PM|second|evening|| +75178|AAAAAAAALKFCBAAA|75178|20|52|58|PM|second|evening|| +75179|AAAAAAAAMKFCBAAA|75179|20|52|59|PM|second|evening|| +75180|AAAAAAAANKFCBAAA|75180|20|53|0|PM|second|evening|| +75181|AAAAAAAAOKFCBAAA|75181|20|53|1|PM|second|evening|| +75182|AAAAAAAAPKFCBAAA|75182|20|53|2|PM|second|evening|| +75183|AAAAAAAAALFCBAAA|75183|20|53|3|PM|second|evening|| +75184|AAAAAAAABLFCBAAA|75184|20|53|4|PM|second|evening|| +75185|AAAAAAAACLFCBAAA|75185|20|53|5|PM|second|evening|| +75186|AAAAAAAADLFCBAAA|75186|20|53|6|PM|second|evening|| +75187|AAAAAAAAELFCBAAA|75187|20|53|7|PM|second|evening|| +75188|AAAAAAAAFLFCBAAA|75188|20|53|8|PM|second|evening|| +75189|AAAAAAAAGLFCBAAA|75189|20|53|9|PM|second|evening|| +75190|AAAAAAAAHLFCBAAA|75190|20|53|10|PM|second|evening|| +75191|AAAAAAAAILFCBAAA|75191|20|53|11|PM|second|evening|| +75192|AAAAAAAAJLFCBAAA|75192|20|53|12|PM|second|evening|| +75193|AAAAAAAAKLFCBAAA|75193|20|53|13|PM|second|evening|| +75194|AAAAAAAALLFCBAAA|75194|20|53|14|PM|second|evening|| +75195|AAAAAAAAMLFCBAAA|75195|20|53|15|PM|second|evening|| +75196|AAAAAAAANLFCBAAA|75196|20|53|16|PM|second|evening|| +75197|AAAAAAAAOLFCBAAA|75197|20|53|17|PM|second|evening|| +75198|AAAAAAAAPLFCBAAA|75198|20|53|18|PM|second|evening|| +75199|AAAAAAAAAMFCBAAA|75199|20|53|19|PM|second|evening|| +75200|AAAAAAAABMFCBAAA|75200|20|53|20|PM|second|evening|| +75201|AAAAAAAACMFCBAAA|75201|20|53|21|PM|second|evening|| +75202|AAAAAAAADMFCBAAA|75202|20|53|22|PM|second|evening|| +75203|AAAAAAAAEMFCBAAA|75203|20|53|23|PM|second|evening|| +75204|AAAAAAAAFMFCBAAA|75204|20|53|24|PM|second|evening|| +75205|AAAAAAAAGMFCBAAA|75205|20|53|25|PM|second|evening|| +75206|AAAAAAAAHMFCBAAA|75206|20|53|26|PM|second|evening|| +75207|AAAAAAAAIMFCBAAA|75207|20|53|27|PM|second|evening|| +75208|AAAAAAAAJMFCBAAA|75208|20|53|28|PM|second|evening|| +75209|AAAAAAAAKMFCBAAA|75209|20|53|29|PM|second|evening|| +75210|AAAAAAAALMFCBAAA|75210|20|53|30|PM|second|evening|| +75211|AAAAAAAAMMFCBAAA|75211|20|53|31|PM|second|evening|| +75212|AAAAAAAANMFCBAAA|75212|20|53|32|PM|second|evening|| +75213|AAAAAAAAOMFCBAAA|75213|20|53|33|PM|second|evening|| +75214|AAAAAAAAPMFCBAAA|75214|20|53|34|PM|second|evening|| +75215|AAAAAAAAANFCBAAA|75215|20|53|35|PM|second|evening|| +75216|AAAAAAAABNFCBAAA|75216|20|53|36|PM|second|evening|| +75217|AAAAAAAACNFCBAAA|75217|20|53|37|PM|second|evening|| +75218|AAAAAAAADNFCBAAA|75218|20|53|38|PM|second|evening|| +75219|AAAAAAAAENFCBAAA|75219|20|53|39|PM|second|evening|| +75220|AAAAAAAAFNFCBAAA|75220|20|53|40|PM|second|evening|| +75221|AAAAAAAAGNFCBAAA|75221|20|53|41|PM|second|evening|| +75222|AAAAAAAAHNFCBAAA|75222|20|53|42|PM|second|evening|| +75223|AAAAAAAAINFCBAAA|75223|20|53|43|PM|second|evening|| +75224|AAAAAAAAJNFCBAAA|75224|20|53|44|PM|second|evening|| +75225|AAAAAAAAKNFCBAAA|75225|20|53|45|PM|second|evening|| +75226|AAAAAAAALNFCBAAA|75226|20|53|46|PM|second|evening|| +75227|AAAAAAAAMNFCBAAA|75227|20|53|47|PM|second|evening|| +75228|AAAAAAAANNFCBAAA|75228|20|53|48|PM|second|evening|| +75229|AAAAAAAAONFCBAAA|75229|20|53|49|PM|second|evening|| +75230|AAAAAAAAPNFCBAAA|75230|20|53|50|PM|second|evening|| +75231|AAAAAAAAAOFCBAAA|75231|20|53|51|PM|second|evening|| +75232|AAAAAAAABOFCBAAA|75232|20|53|52|PM|second|evening|| +75233|AAAAAAAACOFCBAAA|75233|20|53|53|PM|second|evening|| +75234|AAAAAAAADOFCBAAA|75234|20|53|54|PM|second|evening|| +75235|AAAAAAAAEOFCBAAA|75235|20|53|55|PM|second|evening|| +75236|AAAAAAAAFOFCBAAA|75236|20|53|56|PM|second|evening|| +75237|AAAAAAAAGOFCBAAA|75237|20|53|57|PM|second|evening|| +75238|AAAAAAAAHOFCBAAA|75238|20|53|58|PM|second|evening|| +75239|AAAAAAAAIOFCBAAA|75239|20|53|59|PM|second|evening|| +75240|AAAAAAAAJOFCBAAA|75240|20|54|0|PM|second|evening|| +75241|AAAAAAAAKOFCBAAA|75241|20|54|1|PM|second|evening|| +75242|AAAAAAAALOFCBAAA|75242|20|54|2|PM|second|evening|| +75243|AAAAAAAAMOFCBAAA|75243|20|54|3|PM|second|evening|| +75244|AAAAAAAANOFCBAAA|75244|20|54|4|PM|second|evening|| +75245|AAAAAAAAOOFCBAAA|75245|20|54|5|PM|second|evening|| +75246|AAAAAAAAPOFCBAAA|75246|20|54|6|PM|second|evening|| +75247|AAAAAAAAAPFCBAAA|75247|20|54|7|PM|second|evening|| +75248|AAAAAAAABPFCBAAA|75248|20|54|8|PM|second|evening|| +75249|AAAAAAAACPFCBAAA|75249|20|54|9|PM|second|evening|| +75250|AAAAAAAADPFCBAAA|75250|20|54|10|PM|second|evening|| +75251|AAAAAAAAEPFCBAAA|75251|20|54|11|PM|second|evening|| +75252|AAAAAAAAFPFCBAAA|75252|20|54|12|PM|second|evening|| +75253|AAAAAAAAGPFCBAAA|75253|20|54|13|PM|second|evening|| +75254|AAAAAAAAHPFCBAAA|75254|20|54|14|PM|second|evening|| +75255|AAAAAAAAIPFCBAAA|75255|20|54|15|PM|second|evening|| +75256|AAAAAAAAJPFCBAAA|75256|20|54|16|PM|second|evening|| +75257|AAAAAAAAKPFCBAAA|75257|20|54|17|PM|second|evening|| +75258|AAAAAAAALPFCBAAA|75258|20|54|18|PM|second|evening|| +75259|AAAAAAAAMPFCBAAA|75259|20|54|19|PM|second|evening|| +75260|AAAAAAAANPFCBAAA|75260|20|54|20|PM|second|evening|| +75261|AAAAAAAAOPFCBAAA|75261|20|54|21|PM|second|evening|| +75262|AAAAAAAAPPFCBAAA|75262|20|54|22|PM|second|evening|| +75263|AAAAAAAAAAGCBAAA|75263|20|54|23|PM|second|evening|| +75264|AAAAAAAABAGCBAAA|75264|20|54|24|PM|second|evening|| +75265|AAAAAAAACAGCBAAA|75265|20|54|25|PM|second|evening|| +75266|AAAAAAAADAGCBAAA|75266|20|54|26|PM|second|evening|| +75267|AAAAAAAAEAGCBAAA|75267|20|54|27|PM|second|evening|| +75268|AAAAAAAAFAGCBAAA|75268|20|54|28|PM|second|evening|| +75269|AAAAAAAAGAGCBAAA|75269|20|54|29|PM|second|evening|| +75270|AAAAAAAAHAGCBAAA|75270|20|54|30|PM|second|evening|| +75271|AAAAAAAAIAGCBAAA|75271|20|54|31|PM|second|evening|| +75272|AAAAAAAAJAGCBAAA|75272|20|54|32|PM|second|evening|| +75273|AAAAAAAAKAGCBAAA|75273|20|54|33|PM|second|evening|| +75274|AAAAAAAALAGCBAAA|75274|20|54|34|PM|second|evening|| +75275|AAAAAAAAMAGCBAAA|75275|20|54|35|PM|second|evening|| +75276|AAAAAAAANAGCBAAA|75276|20|54|36|PM|second|evening|| +75277|AAAAAAAAOAGCBAAA|75277|20|54|37|PM|second|evening|| +75278|AAAAAAAAPAGCBAAA|75278|20|54|38|PM|second|evening|| +75279|AAAAAAAAABGCBAAA|75279|20|54|39|PM|second|evening|| +75280|AAAAAAAABBGCBAAA|75280|20|54|40|PM|second|evening|| +75281|AAAAAAAACBGCBAAA|75281|20|54|41|PM|second|evening|| +75282|AAAAAAAADBGCBAAA|75282|20|54|42|PM|second|evening|| +75283|AAAAAAAAEBGCBAAA|75283|20|54|43|PM|second|evening|| +75284|AAAAAAAAFBGCBAAA|75284|20|54|44|PM|second|evening|| +75285|AAAAAAAAGBGCBAAA|75285|20|54|45|PM|second|evening|| +75286|AAAAAAAAHBGCBAAA|75286|20|54|46|PM|second|evening|| +75287|AAAAAAAAIBGCBAAA|75287|20|54|47|PM|second|evening|| +75288|AAAAAAAAJBGCBAAA|75288|20|54|48|PM|second|evening|| +75289|AAAAAAAAKBGCBAAA|75289|20|54|49|PM|second|evening|| +75290|AAAAAAAALBGCBAAA|75290|20|54|50|PM|second|evening|| +75291|AAAAAAAAMBGCBAAA|75291|20|54|51|PM|second|evening|| +75292|AAAAAAAANBGCBAAA|75292|20|54|52|PM|second|evening|| +75293|AAAAAAAAOBGCBAAA|75293|20|54|53|PM|second|evening|| +75294|AAAAAAAAPBGCBAAA|75294|20|54|54|PM|second|evening|| +75295|AAAAAAAAACGCBAAA|75295|20|54|55|PM|second|evening|| +75296|AAAAAAAABCGCBAAA|75296|20|54|56|PM|second|evening|| +75297|AAAAAAAACCGCBAAA|75297|20|54|57|PM|second|evening|| +75298|AAAAAAAADCGCBAAA|75298|20|54|58|PM|second|evening|| +75299|AAAAAAAAECGCBAAA|75299|20|54|59|PM|second|evening|| +75300|AAAAAAAAFCGCBAAA|75300|20|55|0|PM|second|evening|| +75301|AAAAAAAAGCGCBAAA|75301|20|55|1|PM|second|evening|| +75302|AAAAAAAAHCGCBAAA|75302|20|55|2|PM|second|evening|| +75303|AAAAAAAAICGCBAAA|75303|20|55|3|PM|second|evening|| +75304|AAAAAAAAJCGCBAAA|75304|20|55|4|PM|second|evening|| +75305|AAAAAAAAKCGCBAAA|75305|20|55|5|PM|second|evening|| +75306|AAAAAAAALCGCBAAA|75306|20|55|6|PM|second|evening|| +75307|AAAAAAAAMCGCBAAA|75307|20|55|7|PM|second|evening|| +75308|AAAAAAAANCGCBAAA|75308|20|55|8|PM|second|evening|| +75309|AAAAAAAAOCGCBAAA|75309|20|55|9|PM|second|evening|| +75310|AAAAAAAAPCGCBAAA|75310|20|55|10|PM|second|evening|| +75311|AAAAAAAAADGCBAAA|75311|20|55|11|PM|second|evening|| +75312|AAAAAAAABDGCBAAA|75312|20|55|12|PM|second|evening|| +75313|AAAAAAAACDGCBAAA|75313|20|55|13|PM|second|evening|| +75314|AAAAAAAADDGCBAAA|75314|20|55|14|PM|second|evening|| +75315|AAAAAAAAEDGCBAAA|75315|20|55|15|PM|second|evening|| +75316|AAAAAAAAFDGCBAAA|75316|20|55|16|PM|second|evening|| +75317|AAAAAAAAGDGCBAAA|75317|20|55|17|PM|second|evening|| +75318|AAAAAAAAHDGCBAAA|75318|20|55|18|PM|second|evening|| +75319|AAAAAAAAIDGCBAAA|75319|20|55|19|PM|second|evening|| +75320|AAAAAAAAJDGCBAAA|75320|20|55|20|PM|second|evening|| +75321|AAAAAAAAKDGCBAAA|75321|20|55|21|PM|second|evening|| +75322|AAAAAAAALDGCBAAA|75322|20|55|22|PM|second|evening|| +75323|AAAAAAAAMDGCBAAA|75323|20|55|23|PM|second|evening|| +75324|AAAAAAAANDGCBAAA|75324|20|55|24|PM|second|evening|| +75325|AAAAAAAAODGCBAAA|75325|20|55|25|PM|second|evening|| +75326|AAAAAAAAPDGCBAAA|75326|20|55|26|PM|second|evening|| +75327|AAAAAAAAAEGCBAAA|75327|20|55|27|PM|second|evening|| +75328|AAAAAAAABEGCBAAA|75328|20|55|28|PM|second|evening|| +75329|AAAAAAAACEGCBAAA|75329|20|55|29|PM|second|evening|| +75330|AAAAAAAADEGCBAAA|75330|20|55|30|PM|second|evening|| +75331|AAAAAAAAEEGCBAAA|75331|20|55|31|PM|second|evening|| +75332|AAAAAAAAFEGCBAAA|75332|20|55|32|PM|second|evening|| +75333|AAAAAAAAGEGCBAAA|75333|20|55|33|PM|second|evening|| +75334|AAAAAAAAHEGCBAAA|75334|20|55|34|PM|second|evening|| +75335|AAAAAAAAIEGCBAAA|75335|20|55|35|PM|second|evening|| +75336|AAAAAAAAJEGCBAAA|75336|20|55|36|PM|second|evening|| +75337|AAAAAAAAKEGCBAAA|75337|20|55|37|PM|second|evening|| +75338|AAAAAAAALEGCBAAA|75338|20|55|38|PM|second|evening|| +75339|AAAAAAAAMEGCBAAA|75339|20|55|39|PM|second|evening|| +75340|AAAAAAAANEGCBAAA|75340|20|55|40|PM|second|evening|| +75341|AAAAAAAAOEGCBAAA|75341|20|55|41|PM|second|evening|| +75342|AAAAAAAAPEGCBAAA|75342|20|55|42|PM|second|evening|| +75343|AAAAAAAAAFGCBAAA|75343|20|55|43|PM|second|evening|| +75344|AAAAAAAABFGCBAAA|75344|20|55|44|PM|second|evening|| +75345|AAAAAAAACFGCBAAA|75345|20|55|45|PM|second|evening|| +75346|AAAAAAAADFGCBAAA|75346|20|55|46|PM|second|evening|| +75347|AAAAAAAAEFGCBAAA|75347|20|55|47|PM|second|evening|| +75348|AAAAAAAAFFGCBAAA|75348|20|55|48|PM|second|evening|| +75349|AAAAAAAAGFGCBAAA|75349|20|55|49|PM|second|evening|| +75350|AAAAAAAAHFGCBAAA|75350|20|55|50|PM|second|evening|| +75351|AAAAAAAAIFGCBAAA|75351|20|55|51|PM|second|evening|| +75352|AAAAAAAAJFGCBAAA|75352|20|55|52|PM|second|evening|| +75353|AAAAAAAAKFGCBAAA|75353|20|55|53|PM|second|evening|| +75354|AAAAAAAALFGCBAAA|75354|20|55|54|PM|second|evening|| +75355|AAAAAAAAMFGCBAAA|75355|20|55|55|PM|second|evening|| +75356|AAAAAAAANFGCBAAA|75356|20|55|56|PM|second|evening|| +75357|AAAAAAAAOFGCBAAA|75357|20|55|57|PM|second|evening|| +75358|AAAAAAAAPFGCBAAA|75358|20|55|58|PM|second|evening|| +75359|AAAAAAAAAGGCBAAA|75359|20|55|59|PM|second|evening|| +75360|AAAAAAAABGGCBAAA|75360|20|56|0|PM|second|evening|| +75361|AAAAAAAACGGCBAAA|75361|20|56|1|PM|second|evening|| +75362|AAAAAAAADGGCBAAA|75362|20|56|2|PM|second|evening|| +75363|AAAAAAAAEGGCBAAA|75363|20|56|3|PM|second|evening|| +75364|AAAAAAAAFGGCBAAA|75364|20|56|4|PM|second|evening|| +75365|AAAAAAAAGGGCBAAA|75365|20|56|5|PM|second|evening|| +75366|AAAAAAAAHGGCBAAA|75366|20|56|6|PM|second|evening|| +75367|AAAAAAAAIGGCBAAA|75367|20|56|7|PM|second|evening|| +75368|AAAAAAAAJGGCBAAA|75368|20|56|8|PM|second|evening|| +75369|AAAAAAAAKGGCBAAA|75369|20|56|9|PM|second|evening|| +75370|AAAAAAAALGGCBAAA|75370|20|56|10|PM|second|evening|| +75371|AAAAAAAAMGGCBAAA|75371|20|56|11|PM|second|evening|| +75372|AAAAAAAANGGCBAAA|75372|20|56|12|PM|second|evening|| +75373|AAAAAAAAOGGCBAAA|75373|20|56|13|PM|second|evening|| +75374|AAAAAAAAPGGCBAAA|75374|20|56|14|PM|second|evening|| +75375|AAAAAAAAAHGCBAAA|75375|20|56|15|PM|second|evening|| +75376|AAAAAAAABHGCBAAA|75376|20|56|16|PM|second|evening|| +75377|AAAAAAAACHGCBAAA|75377|20|56|17|PM|second|evening|| +75378|AAAAAAAADHGCBAAA|75378|20|56|18|PM|second|evening|| +75379|AAAAAAAAEHGCBAAA|75379|20|56|19|PM|second|evening|| +75380|AAAAAAAAFHGCBAAA|75380|20|56|20|PM|second|evening|| +75381|AAAAAAAAGHGCBAAA|75381|20|56|21|PM|second|evening|| +75382|AAAAAAAAHHGCBAAA|75382|20|56|22|PM|second|evening|| +75383|AAAAAAAAIHGCBAAA|75383|20|56|23|PM|second|evening|| +75384|AAAAAAAAJHGCBAAA|75384|20|56|24|PM|second|evening|| +75385|AAAAAAAAKHGCBAAA|75385|20|56|25|PM|second|evening|| +75386|AAAAAAAALHGCBAAA|75386|20|56|26|PM|second|evening|| +75387|AAAAAAAAMHGCBAAA|75387|20|56|27|PM|second|evening|| +75388|AAAAAAAANHGCBAAA|75388|20|56|28|PM|second|evening|| +75389|AAAAAAAAOHGCBAAA|75389|20|56|29|PM|second|evening|| +75390|AAAAAAAAPHGCBAAA|75390|20|56|30|PM|second|evening|| +75391|AAAAAAAAAIGCBAAA|75391|20|56|31|PM|second|evening|| +75392|AAAAAAAABIGCBAAA|75392|20|56|32|PM|second|evening|| +75393|AAAAAAAACIGCBAAA|75393|20|56|33|PM|second|evening|| +75394|AAAAAAAADIGCBAAA|75394|20|56|34|PM|second|evening|| +75395|AAAAAAAAEIGCBAAA|75395|20|56|35|PM|second|evening|| +75396|AAAAAAAAFIGCBAAA|75396|20|56|36|PM|second|evening|| +75397|AAAAAAAAGIGCBAAA|75397|20|56|37|PM|second|evening|| +75398|AAAAAAAAHIGCBAAA|75398|20|56|38|PM|second|evening|| +75399|AAAAAAAAIIGCBAAA|75399|20|56|39|PM|second|evening|| +75400|AAAAAAAAJIGCBAAA|75400|20|56|40|PM|second|evening|| +75401|AAAAAAAAKIGCBAAA|75401|20|56|41|PM|second|evening|| +75402|AAAAAAAALIGCBAAA|75402|20|56|42|PM|second|evening|| +75403|AAAAAAAAMIGCBAAA|75403|20|56|43|PM|second|evening|| +75404|AAAAAAAANIGCBAAA|75404|20|56|44|PM|second|evening|| +75405|AAAAAAAAOIGCBAAA|75405|20|56|45|PM|second|evening|| +75406|AAAAAAAAPIGCBAAA|75406|20|56|46|PM|second|evening|| +75407|AAAAAAAAAJGCBAAA|75407|20|56|47|PM|second|evening|| +75408|AAAAAAAABJGCBAAA|75408|20|56|48|PM|second|evening|| +75409|AAAAAAAACJGCBAAA|75409|20|56|49|PM|second|evening|| +75410|AAAAAAAADJGCBAAA|75410|20|56|50|PM|second|evening|| +75411|AAAAAAAAEJGCBAAA|75411|20|56|51|PM|second|evening|| +75412|AAAAAAAAFJGCBAAA|75412|20|56|52|PM|second|evening|| +75413|AAAAAAAAGJGCBAAA|75413|20|56|53|PM|second|evening|| +75414|AAAAAAAAHJGCBAAA|75414|20|56|54|PM|second|evening|| +75415|AAAAAAAAIJGCBAAA|75415|20|56|55|PM|second|evening|| +75416|AAAAAAAAJJGCBAAA|75416|20|56|56|PM|second|evening|| +75417|AAAAAAAAKJGCBAAA|75417|20|56|57|PM|second|evening|| +75418|AAAAAAAALJGCBAAA|75418|20|56|58|PM|second|evening|| +75419|AAAAAAAAMJGCBAAA|75419|20|56|59|PM|second|evening|| +75420|AAAAAAAANJGCBAAA|75420|20|57|0|PM|second|evening|| +75421|AAAAAAAAOJGCBAAA|75421|20|57|1|PM|second|evening|| +75422|AAAAAAAAPJGCBAAA|75422|20|57|2|PM|second|evening|| +75423|AAAAAAAAAKGCBAAA|75423|20|57|3|PM|second|evening|| +75424|AAAAAAAABKGCBAAA|75424|20|57|4|PM|second|evening|| +75425|AAAAAAAACKGCBAAA|75425|20|57|5|PM|second|evening|| +75426|AAAAAAAADKGCBAAA|75426|20|57|6|PM|second|evening|| +75427|AAAAAAAAEKGCBAAA|75427|20|57|7|PM|second|evening|| +75428|AAAAAAAAFKGCBAAA|75428|20|57|8|PM|second|evening|| +75429|AAAAAAAAGKGCBAAA|75429|20|57|9|PM|second|evening|| +75430|AAAAAAAAHKGCBAAA|75430|20|57|10|PM|second|evening|| +75431|AAAAAAAAIKGCBAAA|75431|20|57|11|PM|second|evening|| +75432|AAAAAAAAJKGCBAAA|75432|20|57|12|PM|second|evening|| +75433|AAAAAAAAKKGCBAAA|75433|20|57|13|PM|second|evening|| +75434|AAAAAAAALKGCBAAA|75434|20|57|14|PM|second|evening|| +75435|AAAAAAAAMKGCBAAA|75435|20|57|15|PM|second|evening|| +75436|AAAAAAAANKGCBAAA|75436|20|57|16|PM|second|evening|| +75437|AAAAAAAAOKGCBAAA|75437|20|57|17|PM|second|evening|| +75438|AAAAAAAAPKGCBAAA|75438|20|57|18|PM|second|evening|| +75439|AAAAAAAAALGCBAAA|75439|20|57|19|PM|second|evening|| +75440|AAAAAAAABLGCBAAA|75440|20|57|20|PM|second|evening|| +75441|AAAAAAAACLGCBAAA|75441|20|57|21|PM|second|evening|| +75442|AAAAAAAADLGCBAAA|75442|20|57|22|PM|second|evening|| +75443|AAAAAAAAELGCBAAA|75443|20|57|23|PM|second|evening|| +75444|AAAAAAAAFLGCBAAA|75444|20|57|24|PM|second|evening|| +75445|AAAAAAAAGLGCBAAA|75445|20|57|25|PM|second|evening|| +75446|AAAAAAAAHLGCBAAA|75446|20|57|26|PM|second|evening|| +75447|AAAAAAAAILGCBAAA|75447|20|57|27|PM|second|evening|| +75448|AAAAAAAAJLGCBAAA|75448|20|57|28|PM|second|evening|| +75449|AAAAAAAAKLGCBAAA|75449|20|57|29|PM|second|evening|| +75450|AAAAAAAALLGCBAAA|75450|20|57|30|PM|second|evening|| +75451|AAAAAAAAMLGCBAAA|75451|20|57|31|PM|second|evening|| +75452|AAAAAAAANLGCBAAA|75452|20|57|32|PM|second|evening|| +75453|AAAAAAAAOLGCBAAA|75453|20|57|33|PM|second|evening|| +75454|AAAAAAAAPLGCBAAA|75454|20|57|34|PM|second|evening|| +75455|AAAAAAAAAMGCBAAA|75455|20|57|35|PM|second|evening|| +75456|AAAAAAAABMGCBAAA|75456|20|57|36|PM|second|evening|| +75457|AAAAAAAACMGCBAAA|75457|20|57|37|PM|second|evening|| +75458|AAAAAAAADMGCBAAA|75458|20|57|38|PM|second|evening|| +75459|AAAAAAAAEMGCBAAA|75459|20|57|39|PM|second|evening|| +75460|AAAAAAAAFMGCBAAA|75460|20|57|40|PM|second|evening|| +75461|AAAAAAAAGMGCBAAA|75461|20|57|41|PM|second|evening|| +75462|AAAAAAAAHMGCBAAA|75462|20|57|42|PM|second|evening|| +75463|AAAAAAAAIMGCBAAA|75463|20|57|43|PM|second|evening|| +75464|AAAAAAAAJMGCBAAA|75464|20|57|44|PM|second|evening|| +75465|AAAAAAAAKMGCBAAA|75465|20|57|45|PM|second|evening|| +75466|AAAAAAAALMGCBAAA|75466|20|57|46|PM|second|evening|| +75467|AAAAAAAAMMGCBAAA|75467|20|57|47|PM|second|evening|| +75468|AAAAAAAANMGCBAAA|75468|20|57|48|PM|second|evening|| +75469|AAAAAAAAOMGCBAAA|75469|20|57|49|PM|second|evening|| +75470|AAAAAAAAPMGCBAAA|75470|20|57|50|PM|second|evening|| +75471|AAAAAAAAANGCBAAA|75471|20|57|51|PM|second|evening|| +75472|AAAAAAAABNGCBAAA|75472|20|57|52|PM|second|evening|| +75473|AAAAAAAACNGCBAAA|75473|20|57|53|PM|second|evening|| +75474|AAAAAAAADNGCBAAA|75474|20|57|54|PM|second|evening|| +75475|AAAAAAAAENGCBAAA|75475|20|57|55|PM|second|evening|| +75476|AAAAAAAAFNGCBAAA|75476|20|57|56|PM|second|evening|| +75477|AAAAAAAAGNGCBAAA|75477|20|57|57|PM|second|evening|| +75478|AAAAAAAAHNGCBAAA|75478|20|57|58|PM|second|evening|| +75479|AAAAAAAAINGCBAAA|75479|20|57|59|PM|second|evening|| +75480|AAAAAAAAJNGCBAAA|75480|20|58|0|PM|second|evening|| +75481|AAAAAAAAKNGCBAAA|75481|20|58|1|PM|second|evening|| +75482|AAAAAAAALNGCBAAA|75482|20|58|2|PM|second|evening|| +75483|AAAAAAAAMNGCBAAA|75483|20|58|3|PM|second|evening|| +75484|AAAAAAAANNGCBAAA|75484|20|58|4|PM|second|evening|| +75485|AAAAAAAAONGCBAAA|75485|20|58|5|PM|second|evening|| +75486|AAAAAAAAPNGCBAAA|75486|20|58|6|PM|second|evening|| +75487|AAAAAAAAAOGCBAAA|75487|20|58|7|PM|second|evening|| +75488|AAAAAAAABOGCBAAA|75488|20|58|8|PM|second|evening|| +75489|AAAAAAAACOGCBAAA|75489|20|58|9|PM|second|evening|| +75490|AAAAAAAADOGCBAAA|75490|20|58|10|PM|second|evening|| +75491|AAAAAAAAEOGCBAAA|75491|20|58|11|PM|second|evening|| +75492|AAAAAAAAFOGCBAAA|75492|20|58|12|PM|second|evening|| +75493|AAAAAAAAGOGCBAAA|75493|20|58|13|PM|second|evening|| +75494|AAAAAAAAHOGCBAAA|75494|20|58|14|PM|second|evening|| +75495|AAAAAAAAIOGCBAAA|75495|20|58|15|PM|second|evening|| +75496|AAAAAAAAJOGCBAAA|75496|20|58|16|PM|second|evening|| +75497|AAAAAAAAKOGCBAAA|75497|20|58|17|PM|second|evening|| +75498|AAAAAAAALOGCBAAA|75498|20|58|18|PM|second|evening|| +75499|AAAAAAAAMOGCBAAA|75499|20|58|19|PM|second|evening|| +75500|AAAAAAAANOGCBAAA|75500|20|58|20|PM|second|evening|| +75501|AAAAAAAAOOGCBAAA|75501|20|58|21|PM|second|evening|| +75502|AAAAAAAAPOGCBAAA|75502|20|58|22|PM|second|evening|| +75503|AAAAAAAAAPGCBAAA|75503|20|58|23|PM|second|evening|| +75504|AAAAAAAABPGCBAAA|75504|20|58|24|PM|second|evening|| +75505|AAAAAAAACPGCBAAA|75505|20|58|25|PM|second|evening|| +75506|AAAAAAAADPGCBAAA|75506|20|58|26|PM|second|evening|| +75507|AAAAAAAAEPGCBAAA|75507|20|58|27|PM|second|evening|| +75508|AAAAAAAAFPGCBAAA|75508|20|58|28|PM|second|evening|| +75509|AAAAAAAAGPGCBAAA|75509|20|58|29|PM|second|evening|| +75510|AAAAAAAAHPGCBAAA|75510|20|58|30|PM|second|evening|| +75511|AAAAAAAAIPGCBAAA|75511|20|58|31|PM|second|evening|| +75512|AAAAAAAAJPGCBAAA|75512|20|58|32|PM|second|evening|| +75513|AAAAAAAAKPGCBAAA|75513|20|58|33|PM|second|evening|| +75514|AAAAAAAALPGCBAAA|75514|20|58|34|PM|second|evening|| +75515|AAAAAAAAMPGCBAAA|75515|20|58|35|PM|second|evening|| +75516|AAAAAAAANPGCBAAA|75516|20|58|36|PM|second|evening|| +75517|AAAAAAAAOPGCBAAA|75517|20|58|37|PM|second|evening|| +75518|AAAAAAAAPPGCBAAA|75518|20|58|38|PM|second|evening|| +75519|AAAAAAAAAAHCBAAA|75519|20|58|39|PM|second|evening|| +75520|AAAAAAAABAHCBAAA|75520|20|58|40|PM|second|evening|| +75521|AAAAAAAACAHCBAAA|75521|20|58|41|PM|second|evening|| +75522|AAAAAAAADAHCBAAA|75522|20|58|42|PM|second|evening|| +75523|AAAAAAAAEAHCBAAA|75523|20|58|43|PM|second|evening|| +75524|AAAAAAAAFAHCBAAA|75524|20|58|44|PM|second|evening|| +75525|AAAAAAAAGAHCBAAA|75525|20|58|45|PM|second|evening|| +75526|AAAAAAAAHAHCBAAA|75526|20|58|46|PM|second|evening|| +75527|AAAAAAAAIAHCBAAA|75527|20|58|47|PM|second|evening|| +75528|AAAAAAAAJAHCBAAA|75528|20|58|48|PM|second|evening|| +75529|AAAAAAAAKAHCBAAA|75529|20|58|49|PM|second|evening|| +75530|AAAAAAAALAHCBAAA|75530|20|58|50|PM|second|evening|| +75531|AAAAAAAAMAHCBAAA|75531|20|58|51|PM|second|evening|| +75532|AAAAAAAANAHCBAAA|75532|20|58|52|PM|second|evening|| +75533|AAAAAAAAOAHCBAAA|75533|20|58|53|PM|second|evening|| +75534|AAAAAAAAPAHCBAAA|75534|20|58|54|PM|second|evening|| +75535|AAAAAAAAABHCBAAA|75535|20|58|55|PM|second|evening|| +75536|AAAAAAAABBHCBAAA|75536|20|58|56|PM|second|evening|| +75537|AAAAAAAACBHCBAAA|75537|20|58|57|PM|second|evening|| +75538|AAAAAAAADBHCBAAA|75538|20|58|58|PM|second|evening|| +75539|AAAAAAAAEBHCBAAA|75539|20|58|59|PM|second|evening|| +75540|AAAAAAAAFBHCBAAA|75540|20|59|0|PM|second|evening|| +75541|AAAAAAAAGBHCBAAA|75541|20|59|1|PM|second|evening|| +75542|AAAAAAAAHBHCBAAA|75542|20|59|2|PM|second|evening|| +75543|AAAAAAAAIBHCBAAA|75543|20|59|3|PM|second|evening|| +75544|AAAAAAAAJBHCBAAA|75544|20|59|4|PM|second|evening|| +75545|AAAAAAAAKBHCBAAA|75545|20|59|5|PM|second|evening|| +75546|AAAAAAAALBHCBAAA|75546|20|59|6|PM|second|evening|| +75547|AAAAAAAAMBHCBAAA|75547|20|59|7|PM|second|evening|| +75548|AAAAAAAANBHCBAAA|75548|20|59|8|PM|second|evening|| +75549|AAAAAAAAOBHCBAAA|75549|20|59|9|PM|second|evening|| +75550|AAAAAAAAPBHCBAAA|75550|20|59|10|PM|second|evening|| +75551|AAAAAAAAACHCBAAA|75551|20|59|11|PM|second|evening|| +75552|AAAAAAAABCHCBAAA|75552|20|59|12|PM|second|evening|| +75553|AAAAAAAACCHCBAAA|75553|20|59|13|PM|second|evening|| +75554|AAAAAAAADCHCBAAA|75554|20|59|14|PM|second|evening|| +75555|AAAAAAAAECHCBAAA|75555|20|59|15|PM|second|evening|| +75556|AAAAAAAAFCHCBAAA|75556|20|59|16|PM|second|evening|| +75557|AAAAAAAAGCHCBAAA|75557|20|59|17|PM|second|evening|| +75558|AAAAAAAAHCHCBAAA|75558|20|59|18|PM|second|evening|| +75559|AAAAAAAAICHCBAAA|75559|20|59|19|PM|second|evening|| +75560|AAAAAAAAJCHCBAAA|75560|20|59|20|PM|second|evening|| +75561|AAAAAAAAKCHCBAAA|75561|20|59|21|PM|second|evening|| +75562|AAAAAAAALCHCBAAA|75562|20|59|22|PM|second|evening|| +75563|AAAAAAAAMCHCBAAA|75563|20|59|23|PM|second|evening|| +75564|AAAAAAAANCHCBAAA|75564|20|59|24|PM|second|evening|| +75565|AAAAAAAAOCHCBAAA|75565|20|59|25|PM|second|evening|| +75566|AAAAAAAAPCHCBAAA|75566|20|59|26|PM|second|evening|| +75567|AAAAAAAAADHCBAAA|75567|20|59|27|PM|second|evening|| +75568|AAAAAAAABDHCBAAA|75568|20|59|28|PM|second|evening|| +75569|AAAAAAAACDHCBAAA|75569|20|59|29|PM|second|evening|| +75570|AAAAAAAADDHCBAAA|75570|20|59|30|PM|second|evening|| +75571|AAAAAAAAEDHCBAAA|75571|20|59|31|PM|second|evening|| +75572|AAAAAAAAFDHCBAAA|75572|20|59|32|PM|second|evening|| +75573|AAAAAAAAGDHCBAAA|75573|20|59|33|PM|second|evening|| +75574|AAAAAAAAHDHCBAAA|75574|20|59|34|PM|second|evening|| +75575|AAAAAAAAIDHCBAAA|75575|20|59|35|PM|second|evening|| +75576|AAAAAAAAJDHCBAAA|75576|20|59|36|PM|second|evening|| +75577|AAAAAAAAKDHCBAAA|75577|20|59|37|PM|second|evening|| +75578|AAAAAAAALDHCBAAA|75578|20|59|38|PM|second|evening|| +75579|AAAAAAAAMDHCBAAA|75579|20|59|39|PM|second|evening|| +75580|AAAAAAAANDHCBAAA|75580|20|59|40|PM|second|evening|| +75581|AAAAAAAAODHCBAAA|75581|20|59|41|PM|second|evening|| +75582|AAAAAAAAPDHCBAAA|75582|20|59|42|PM|second|evening|| +75583|AAAAAAAAAEHCBAAA|75583|20|59|43|PM|second|evening|| +75584|AAAAAAAABEHCBAAA|75584|20|59|44|PM|second|evening|| +75585|AAAAAAAACEHCBAAA|75585|20|59|45|PM|second|evening|| +75586|AAAAAAAADEHCBAAA|75586|20|59|46|PM|second|evening|| +75587|AAAAAAAAEEHCBAAA|75587|20|59|47|PM|second|evening|| +75588|AAAAAAAAFEHCBAAA|75588|20|59|48|PM|second|evening|| +75589|AAAAAAAAGEHCBAAA|75589|20|59|49|PM|second|evening|| +75590|AAAAAAAAHEHCBAAA|75590|20|59|50|PM|second|evening|| +75591|AAAAAAAAIEHCBAAA|75591|20|59|51|PM|second|evening|| +75592|AAAAAAAAJEHCBAAA|75592|20|59|52|PM|second|evening|| +75593|AAAAAAAAKEHCBAAA|75593|20|59|53|PM|second|evening|| +75594|AAAAAAAALEHCBAAA|75594|20|59|54|PM|second|evening|| +75595|AAAAAAAAMEHCBAAA|75595|20|59|55|PM|second|evening|| +75596|AAAAAAAANEHCBAAA|75596|20|59|56|PM|second|evening|| +75597|AAAAAAAAOEHCBAAA|75597|20|59|57|PM|second|evening|| +75598|AAAAAAAAPEHCBAAA|75598|20|59|58|PM|second|evening|| +75599|AAAAAAAAAFHCBAAA|75599|20|59|59|PM|second|evening|| +75600|AAAAAAAABFHCBAAA|75600|21|0|0|PM|second|evening|| +75601|AAAAAAAACFHCBAAA|75601|21|0|1|PM|second|evening|| +75602|AAAAAAAADFHCBAAA|75602|21|0|2|PM|second|evening|| +75603|AAAAAAAAEFHCBAAA|75603|21|0|3|PM|second|evening|| +75604|AAAAAAAAFFHCBAAA|75604|21|0|4|PM|second|evening|| +75605|AAAAAAAAGFHCBAAA|75605|21|0|5|PM|second|evening|| +75606|AAAAAAAAHFHCBAAA|75606|21|0|6|PM|second|evening|| +75607|AAAAAAAAIFHCBAAA|75607|21|0|7|PM|second|evening|| +75608|AAAAAAAAJFHCBAAA|75608|21|0|8|PM|second|evening|| +75609|AAAAAAAAKFHCBAAA|75609|21|0|9|PM|second|evening|| +75610|AAAAAAAALFHCBAAA|75610|21|0|10|PM|second|evening|| +75611|AAAAAAAAMFHCBAAA|75611|21|0|11|PM|second|evening|| +75612|AAAAAAAANFHCBAAA|75612|21|0|12|PM|second|evening|| +75613|AAAAAAAAOFHCBAAA|75613|21|0|13|PM|second|evening|| +75614|AAAAAAAAPFHCBAAA|75614|21|0|14|PM|second|evening|| +75615|AAAAAAAAAGHCBAAA|75615|21|0|15|PM|second|evening|| +75616|AAAAAAAABGHCBAAA|75616|21|0|16|PM|second|evening|| +75617|AAAAAAAACGHCBAAA|75617|21|0|17|PM|second|evening|| +75618|AAAAAAAADGHCBAAA|75618|21|0|18|PM|second|evening|| +75619|AAAAAAAAEGHCBAAA|75619|21|0|19|PM|second|evening|| +75620|AAAAAAAAFGHCBAAA|75620|21|0|20|PM|second|evening|| +75621|AAAAAAAAGGHCBAAA|75621|21|0|21|PM|second|evening|| +75622|AAAAAAAAHGHCBAAA|75622|21|0|22|PM|second|evening|| +75623|AAAAAAAAIGHCBAAA|75623|21|0|23|PM|second|evening|| +75624|AAAAAAAAJGHCBAAA|75624|21|0|24|PM|second|evening|| +75625|AAAAAAAAKGHCBAAA|75625|21|0|25|PM|second|evening|| +75626|AAAAAAAALGHCBAAA|75626|21|0|26|PM|second|evening|| +75627|AAAAAAAAMGHCBAAA|75627|21|0|27|PM|second|evening|| +75628|AAAAAAAANGHCBAAA|75628|21|0|28|PM|second|evening|| +75629|AAAAAAAAOGHCBAAA|75629|21|0|29|PM|second|evening|| +75630|AAAAAAAAPGHCBAAA|75630|21|0|30|PM|second|evening|| +75631|AAAAAAAAAHHCBAAA|75631|21|0|31|PM|second|evening|| +75632|AAAAAAAABHHCBAAA|75632|21|0|32|PM|second|evening|| +75633|AAAAAAAACHHCBAAA|75633|21|0|33|PM|second|evening|| +75634|AAAAAAAADHHCBAAA|75634|21|0|34|PM|second|evening|| +75635|AAAAAAAAEHHCBAAA|75635|21|0|35|PM|second|evening|| +75636|AAAAAAAAFHHCBAAA|75636|21|0|36|PM|second|evening|| +75637|AAAAAAAAGHHCBAAA|75637|21|0|37|PM|second|evening|| +75638|AAAAAAAAHHHCBAAA|75638|21|0|38|PM|second|evening|| +75639|AAAAAAAAIHHCBAAA|75639|21|0|39|PM|second|evening|| +75640|AAAAAAAAJHHCBAAA|75640|21|0|40|PM|second|evening|| +75641|AAAAAAAAKHHCBAAA|75641|21|0|41|PM|second|evening|| +75642|AAAAAAAALHHCBAAA|75642|21|0|42|PM|second|evening|| +75643|AAAAAAAAMHHCBAAA|75643|21|0|43|PM|second|evening|| +75644|AAAAAAAANHHCBAAA|75644|21|0|44|PM|second|evening|| +75645|AAAAAAAAOHHCBAAA|75645|21|0|45|PM|second|evening|| +75646|AAAAAAAAPHHCBAAA|75646|21|0|46|PM|second|evening|| +75647|AAAAAAAAAIHCBAAA|75647|21|0|47|PM|second|evening|| +75648|AAAAAAAABIHCBAAA|75648|21|0|48|PM|second|evening|| +75649|AAAAAAAACIHCBAAA|75649|21|0|49|PM|second|evening|| +75650|AAAAAAAADIHCBAAA|75650|21|0|50|PM|second|evening|| +75651|AAAAAAAAEIHCBAAA|75651|21|0|51|PM|second|evening|| +75652|AAAAAAAAFIHCBAAA|75652|21|0|52|PM|second|evening|| +75653|AAAAAAAAGIHCBAAA|75653|21|0|53|PM|second|evening|| +75654|AAAAAAAAHIHCBAAA|75654|21|0|54|PM|second|evening|| +75655|AAAAAAAAIIHCBAAA|75655|21|0|55|PM|second|evening|| +75656|AAAAAAAAJIHCBAAA|75656|21|0|56|PM|second|evening|| +75657|AAAAAAAAKIHCBAAA|75657|21|0|57|PM|second|evening|| +75658|AAAAAAAALIHCBAAA|75658|21|0|58|PM|second|evening|| +75659|AAAAAAAAMIHCBAAA|75659|21|0|59|PM|second|evening|| +75660|AAAAAAAANIHCBAAA|75660|21|1|0|PM|second|evening|| +75661|AAAAAAAAOIHCBAAA|75661|21|1|1|PM|second|evening|| +75662|AAAAAAAAPIHCBAAA|75662|21|1|2|PM|second|evening|| +75663|AAAAAAAAAJHCBAAA|75663|21|1|3|PM|second|evening|| +75664|AAAAAAAABJHCBAAA|75664|21|1|4|PM|second|evening|| +75665|AAAAAAAACJHCBAAA|75665|21|1|5|PM|second|evening|| +75666|AAAAAAAADJHCBAAA|75666|21|1|6|PM|second|evening|| +75667|AAAAAAAAEJHCBAAA|75667|21|1|7|PM|second|evening|| +75668|AAAAAAAAFJHCBAAA|75668|21|1|8|PM|second|evening|| +75669|AAAAAAAAGJHCBAAA|75669|21|1|9|PM|second|evening|| +75670|AAAAAAAAHJHCBAAA|75670|21|1|10|PM|second|evening|| +75671|AAAAAAAAIJHCBAAA|75671|21|1|11|PM|second|evening|| +75672|AAAAAAAAJJHCBAAA|75672|21|1|12|PM|second|evening|| +75673|AAAAAAAAKJHCBAAA|75673|21|1|13|PM|second|evening|| +75674|AAAAAAAALJHCBAAA|75674|21|1|14|PM|second|evening|| +75675|AAAAAAAAMJHCBAAA|75675|21|1|15|PM|second|evening|| +75676|AAAAAAAANJHCBAAA|75676|21|1|16|PM|second|evening|| +75677|AAAAAAAAOJHCBAAA|75677|21|1|17|PM|second|evening|| +75678|AAAAAAAAPJHCBAAA|75678|21|1|18|PM|second|evening|| +75679|AAAAAAAAAKHCBAAA|75679|21|1|19|PM|second|evening|| +75680|AAAAAAAABKHCBAAA|75680|21|1|20|PM|second|evening|| +75681|AAAAAAAACKHCBAAA|75681|21|1|21|PM|second|evening|| +75682|AAAAAAAADKHCBAAA|75682|21|1|22|PM|second|evening|| +75683|AAAAAAAAEKHCBAAA|75683|21|1|23|PM|second|evening|| +75684|AAAAAAAAFKHCBAAA|75684|21|1|24|PM|second|evening|| +75685|AAAAAAAAGKHCBAAA|75685|21|1|25|PM|second|evening|| +75686|AAAAAAAAHKHCBAAA|75686|21|1|26|PM|second|evening|| +75687|AAAAAAAAIKHCBAAA|75687|21|1|27|PM|second|evening|| +75688|AAAAAAAAJKHCBAAA|75688|21|1|28|PM|second|evening|| +75689|AAAAAAAAKKHCBAAA|75689|21|1|29|PM|second|evening|| +75690|AAAAAAAALKHCBAAA|75690|21|1|30|PM|second|evening|| +75691|AAAAAAAAMKHCBAAA|75691|21|1|31|PM|second|evening|| +75692|AAAAAAAANKHCBAAA|75692|21|1|32|PM|second|evening|| +75693|AAAAAAAAOKHCBAAA|75693|21|1|33|PM|second|evening|| +75694|AAAAAAAAPKHCBAAA|75694|21|1|34|PM|second|evening|| +75695|AAAAAAAAALHCBAAA|75695|21|1|35|PM|second|evening|| +75696|AAAAAAAABLHCBAAA|75696|21|1|36|PM|second|evening|| +75697|AAAAAAAACLHCBAAA|75697|21|1|37|PM|second|evening|| +75698|AAAAAAAADLHCBAAA|75698|21|1|38|PM|second|evening|| +75699|AAAAAAAAELHCBAAA|75699|21|1|39|PM|second|evening|| +75700|AAAAAAAAFLHCBAAA|75700|21|1|40|PM|second|evening|| +75701|AAAAAAAAGLHCBAAA|75701|21|1|41|PM|second|evening|| +75702|AAAAAAAAHLHCBAAA|75702|21|1|42|PM|second|evening|| +75703|AAAAAAAAILHCBAAA|75703|21|1|43|PM|second|evening|| +75704|AAAAAAAAJLHCBAAA|75704|21|1|44|PM|second|evening|| +75705|AAAAAAAAKLHCBAAA|75705|21|1|45|PM|second|evening|| +75706|AAAAAAAALLHCBAAA|75706|21|1|46|PM|second|evening|| +75707|AAAAAAAAMLHCBAAA|75707|21|1|47|PM|second|evening|| +75708|AAAAAAAANLHCBAAA|75708|21|1|48|PM|second|evening|| +75709|AAAAAAAAOLHCBAAA|75709|21|1|49|PM|second|evening|| +75710|AAAAAAAAPLHCBAAA|75710|21|1|50|PM|second|evening|| +75711|AAAAAAAAAMHCBAAA|75711|21|1|51|PM|second|evening|| +75712|AAAAAAAABMHCBAAA|75712|21|1|52|PM|second|evening|| +75713|AAAAAAAACMHCBAAA|75713|21|1|53|PM|second|evening|| +75714|AAAAAAAADMHCBAAA|75714|21|1|54|PM|second|evening|| +75715|AAAAAAAAEMHCBAAA|75715|21|1|55|PM|second|evening|| +75716|AAAAAAAAFMHCBAAA|75716|21|1|56|PM|second|evening|| +75717|AAAAAAAAGMHCBAAA|75717|21|1|57|PM|second|evening|| +75718|AAAAAAAAHMHCBAAA|75718|21|1|58|PM|second|evening|| +75719|AAAAAAAAIMHCBAAA|75719|21|1|59|PM|second|evening|| +75720|AAAAAAAAJMHCBAAA|75720|21|2|0|PM|second|evening|| +75721|AAAAAAAAKMHCBAAA|75721|21|2|1|PM|second|evening|| +75722|AAAAAAAALMHCBAAA|75722|21|2|2|PM|second|evening|| +75723|AAAAAAAAMMHCBAAA|75723|21|2|3|PM|second|evening|| +75724|AAAAAAAANMHCBAAA|75724|21|2|4|PM|second|evening|| +75725|AAAAAAAAOMHCBAAA|75725|21|2|5|PM|second|evening|| +75726|AAAAAAAAPMHCBAAA|75726|21|2|6|PM|second|evening|| +75727|AAAAAAAAANHCBAAA|75727|21|2|7|PM|second|evening|| +75728|AAAAAAAABNHCBAAA|75728|21|2|8|PM|second|evening|| +75729|AAAAAAAACNHCBAAA|75729|21|2|9|PM|second|evening|| +75730|AAAAAAAADNHCBAAA|75730|21|2|10|PM|second|evening|| +75731|AAAAAAAAENHCBAAA|75731|21|2|11|PM|second|evening|| +75732|AAAAAAAAFNHCBAAA|75732|21|2|12|PM|second|evening|| +75733|AAAAAAAAGNHCBAAA|75733|21|2|13|PM|second|evening|| +75734|AAAAAAAAHNHCBAAA|75734|21|2|14|PM|second|evening|| +75735|AAAAAAAAINHCBAAA|75735|21|2|15|PM|second|evening|| +75736|AAAAAAAAJNHCBAAA|75736|21|2|16|PM|second|evening|| +75737|AAAAAAAAKNHCBAAA|75737|21|2|17|PM|second|evening|| +75738|AAAAAAAALNHCBAAA|75738|21|2|18|PM|second|evening|| +75739|AAAAAAAAMNHCBAAA|75739|21|2|19|PM|second|evening|| +75740|AAAAAAAANNHCBAAA|75740|21|2|20|PM|second|evening|| +75741|AAAAAAAAONHCBAAA|75741|21|2|21|PM|second|evening|| +75742|AAAAAAAAPNHCBAAA|75742|21|2|22|PM|second|evening|| +75743|AAAAAAAAAOHCBAAA|75743|21|2|23|PM|second|evening|| +75744|AAAAAAAABOHCBAAA|75744|21|2|24|PM|second|evening|| +75745|AAAAAAAACOHCBAAA|75745|21|2|25|PM|second|evening|| +75746|AAAAAAAADOHCBAAA|75746|21|2|26|PM|second|evening|| +75747|AAAAAAAAEOHCBAAA|75747|21|2|27|PM|second|evening|| +75748|AAAAAAAAFOHCBAAA|75748|21|2|28|PM|second|evening|| +75749|AAAAAAAAGOHCBAAA|75749|21|2|29|PM|second|evening|| +75750|AAAAAAAAHOHCBAAA|75750|21|2|30|PM|second|evening|| +75751|AAAAAAAAIOHCBAAA|75751|21|2|31|PM|second|evening|| +75752|AAAAAAAAJOHCBAAA|75752|21|2|32|PM|second|evening|| +75753|AAAAAAAAKOHCBAAA|75753|21|2|33|PM|second|evening|| +75754|AAAAAAAALOHCBAAA|75754|21|2|34|PM|second|evening|| +75755|AAAAAAAAMOHCBAAA|75755|21|2|35|PM|second|evening|| +75756|AAAAAAAANOHCBAAA|75756|21|2|36|PM|second|evening|| +75757|AAAAAAAAOOHCBAAA|75757|21|2|37|PM|second|evening|| +75758|AAAAAAAAPOHCBAAA|75758|21|2|38|PM|second|evening|| +75759|AAAAAAAAAPHCBAAA|75759|21|2|39|PM|second|evening|| +75760|AAAAAAAABPHCBAAA|75760|21|2|40|PM|second|evening|| +75761|AAAAAAAACPHCBAAA|75761|21|2|41|PM|second|evening|| +75762|AAAAAAAADPHCBAAA|75762|21|2|42|PM|second|evening|| +75763|AAAAAAAAEPHCBAAA|75763|21|2|43|PM|second|evening|| +75764|AAAAAAAAFPHCBAAA|75764|21|2|44|PM|second|evening|| +75765|AAAAAAAAGPHCBAAA|75765|21|2|45|PM|second|evening|| +75766|AAAAAAAAHPHCBAAA|75766|21|2|46|PM|second|evening|| +75767|AAAAAAAAIPHCBAAA|75767|21|2|47|PM|second|evening|| +75768|AAAAAAAAJPHCBAAA|75768|21|2|48|PM|second|evening|| +75769|AAAAAAAAKPHCBAAA|75769|21|2|49|PM|second|evening|| +75770|AAAAAAAALPHCBAAA|75770|21|2|50|PM|second|evening|| +75771|AAAAAAAAMPHCBAAA|75771|21|2|51|PM|second|evening|| +75772|AAAAAAAANPHCBAAA|75772|21|2|52|PM|second|evening|| +75773|AAAAAAAAOPHCBAAA|75773|21|2|53|PM|second|evening|| +75774|AAAAAAAAPPHCBAAA|75774|21|2|54|PM|second|evening|| +75775|AAAAAAAAAAICBAAA|75775|21|2|55|PM|second|evening|| +75776|AAAAAAAABAICBAAA|75776|21|2|56|PM|second|evening|| +75777|AAAAAAAACAICBAAA|75777|21|2|57|PM|second|evening|| +75778|AAAAAAAADAICBAAA|75778|21|2|58|PM|second|evening|| +75779|AAAAAAAAEAICBAAA|75779|21|2|59|PM|second|evening|| +75780|AAAAAAAAFAICBAAA|75780|21|3|0|PM|second|evening|| +75781|AAAAAAAAGAICBAAA|75781|21|3|1|PM|second|evening|| +75782|AAAAAAAAHAICBAAA|75782|21|3|2|PM|second|evening|| +75783|AAAAAAAAIAICBAAA|75783|21|3|3|PM|second|evening|| +75784|AAAAAAAAJAICBAAA|75784|21|3|4|PM|second|evening|| +75785|AAAAAAAAKAICBAAA|75785|21|3|5|PM|second|evening|| +75786|AAAAAAAALAICBAAA|75786|21|3|6|PM|second|evening|| +75787|AAAAAAAAMAICBAAA|75787|21|3|7|PM|second|evening|| +75788|AAAAAAAANAICBAAA|75788|21|3|8|PM|second|evening|| +75789|AAAAAAAAOAICBAAA|75789|21|3|9|PM|second|evening|| +75790|AAAAAAAAPAICBAAA|75790|21|3|10|PM|second|evening|| +75791|AAAAAAAAABICBAAA|75791|21|3|11|PM|second|evening|| +75792|AAAAAAAABBICBAAA|75792|21|3|12|PM|second|evening|| +75793|AAAAAAAACBICBAAA|75793|21|3|13|PM|second|evening|| +75794|AAAAAAAADBICBAAA|75794|21|3|14|PM|second|evening|| +75795|AAAAAAAAEBICBAAA|75795|21|3|15|PM|second|evening|| +75796|AAAAAAAAFBICBAAA|75796|21|3|16|PM|second|evening|| +75797|AAAAAAAAGBICBAAA|75797|21|3|17|PM|second|evening|| +75798|AAAAAAAAHBICBAAA|75798|21|3|18|PM|second|evening|| +75799|AAAAAAAAIBICBAAA|75799|21|3|19|PM|second|evening|| +75800|AAAAAAAAJBICBAAA|75800|21|3|20|PM|second|evening|| +75801|AAAAAAAAKBICBAAA|75801|21|3|21|PM|second|evening|| +75802|AAAAAAAALBICBAAA|75802|21|3|22|PM|second|evening|| +75803|AAAAAAAAMBICBAAA|75803|21|3|23|PM|second|evening|| +75804|AAAAAAAANBICBAAA|75804|21|3|24|PM|second|evening|| +75805|AAAAAAAAOBICBAAA|75805|21|3|25|PM|second|evening|| +75806|AAAAAAAAPBICBAAA|75806|21|3|26|PM|second|evening|| +75807|AAAAAAAAACICBAAA|75807|21|3|27|PM|second|evening|| +75808|AAAAAAAABCICBAAA|75808|21|3|28|PM|second|evening|| +75809|AAAAAAAACCICBAAA|75809|21|3|29|PM|second|evening|| +75810|AAAAAAAADCICBAAA|75810|21|3|30|PM|second|evening|| +75811|AAAAAAAAECICBAAA|75811|21|3|31|PM|second|evening|| +75812|AAAAAAAAFCICBAAA|75812|21|3|32|PM|second|evening|| +75813|AAAAAAAAGCICBAAA|75813|21|3|33|PM|second|evening|| +75814|AAAAAAAAHCICBAAA|75814|21|3|34|PM|second|evening|| +75815|AAAAAAAAICICBAAA|75815|21|3|35|PM|second|evening|| +75816|AAAAAAAAJCICBAAA|75816|21|3|36|PM|second|evening|| +75817|AAAAAAAAKCICBAAA|75817|21|3|37|PM|second|evening|| +75818|AAAAAAAALCICBAAA|75818|21|3|38|PM|second|evening|| +75819|AAAAAAAAMCICBAAA|75819|21|3|39|PM|second|evening|| +75820|AAAAAAAANCICBAAA|75820|21|3|40|PM|second|evening|| +75821|AAAAAAAAOCICBAAA|75821|21|3|41|PM|second|evening|| +75822|AAAAAAAAPCICBAAA|75822|21|3|42|PM|second|evening|| +75823|AAAAAAAAADICBAAA|75823|21|3|43|PM|second|evening|| +75824|AAAAAAAABDICBAAA|75824|21|3|44|PM|second|evening|| +75825|AAAAAAAACDICBAAA|75825|21|3|45|PM|second|evening|| +75826|AAAAAAAADDICBAAA|75826|21|3|46|PM|second|evening|| +75827|AAAAAAAAEDICBAAA|75827|21|3|47|PM|second|evening|| +75828|AAAAAAAAFDICBAAA|75828|21|3|48|PM|second|evening|| +75829|AAAAAAAAGDICBAAA|75829|21|3|49|PM|second|evening|| +75830|AAAAAAAAHDICBAAA|75830|21|3|50|PM|second|evening|| +75831|AAAAAAAAIDICBAAA|75831|21|3|51|PM|second|evening|| +75832|AAAAAAAAJDICBAAA|75832|21|3|52|PM|second|evening|| +75833|AAAAAAAAKDICBAAA|75833|21|3|53|PM|second|evening|| +75834|AAAAAAAALDICBAAA|75834|21|3|54|PM|second|evening|| +75835|AAAAAAAAMDICBAAA|75835|21|3|55|PM|second|evening|| +75836|AAAAAAAANDICBAAA|75836|21|3|56|PM|second|evening|| +75837|AAAAAAAAODICBAAA|75837|21|3|57|PM|second|evening|| +75838|AAAAAAAAPDICBAAA|75838|21|3|58|PM|second|evening|| +75839|AAAAAAAAAEICBAAA|75839|21|3|59|PM|second|evening|| +75840|AAAAAAAABEICBAAA|75840|21|4|0|PM|second|evening|| +75841|AAAAAAAACEICBAAA|75841|21|4|1|PM|second|evening|| +75842|AAAAAAAADEICBAAA|75842|21|4|2|PM|second|evening|| +75843|AAAAAAAAEEICBAAA|75843|21|4|3|PM|second|evening|| +75844|AAAAAAAAFEICBAAA|75844|21|4|4|PM|second|evening|| +75845|AAAAAAAAGEICBAAA|75845|21|4|5|PM|second|evening|| +75846|AAAAAAAAHEICBAAA|75846|21|4|6|PM|second|evening|| +75847|AAAAAAAAIEICBAAA|75847|21|4|7|PM|second|evening|| +75848|AAAAAAAAJEICBAAA|75848|21|4|8|PM|second|evening|| +75849|AAAAAAAAKEICBAAA|75849|21|4|9|PM|second|evening|| +75850|AAAAAAAALEICBAAA|75850|21|4|10|PM|second|evening|| +75851|AAAAAAAAMEICBAAA|75851|21|4|11|PM|second|evening|| +75852|AAAAAAAANEICBAAA|75852|21|4|12|PM|second|evening|| +75853|AAAAAAAAOEICBAAA|75853|21|4|13|PM|second|evening|| +75854|AAAAAAAAPEICBAAA|75854|21|4|14|PM|second|evening|| +75855|AAAAAAAAAFICBAAA|75855|21|4|15|PM|second|evening|| +75856|AAAAAAAABFICBAAA|75856|21|4|16|PM|second|evening|| +75857|AAAAAAAACFICBAAA|75857|21|4|17|PM|second|evening|| +75858|AAAAAAAADFICBAAA|75858|21|4|18|PM|second|evening|| +75859|AAAAAAAAEFICBAAA|75859|21|4|19|PM|second|evening|| +75860|AAAAAAAAFFICBAAA|75860|21|4|20|PM|second|evening|| +75861|AAAAAAAAGFICBAAA|75861|21|4|21|PM|second|evening|| +75862|AAAAAAAAHFICBAAA|75862|21|4|22|PM|second|evening|| +75863|AAAAAAAAIFICBAAA|75863|21|4|23|PM|second|evening|| +75864|AAAAAAAAJFICBAAA|75864|21|4|24|PM|second|evening|| +75865|AAAAAAAAKFICBAAA|75865|21|4|25|PM|second|evening|| +75866|AAAAAAAALFICBAAA|75866|21|4|26|PM|second|evening|| +75867|AAAAAAAAMFICBAAA|75867|21|4|27|PM|second|evening|| +75868|AAAAAAAANFICBAAA|75868|21|4|28|PM|second|evening|| +75869|AAAAAAAAOFICBAAA|75869|21|4|29|PM|second|evening|| +75870|AAAAAAAAPFICBAAA|75870|21|4|30|PM|second|evening|| +75871|AAAAAAAAAGICBAAA|75871|21|4|31|PM|second|evening|| +75872|AAAAAAAABGICBAAA|75872|21|4|32|PM|second|evening|| +75873|AAAAAAAACGICBAAA|75873|21|4|33|PM|second|evening|| +75874|AAAAAAAADGICBAAA|75874|21|4|34|PM|second|evening|| +75875|AAAAAAAAEGICBAAA|75875|21|4|35|PM|second|evening|| +75876|AAAAAAAAFGICBAAA|75876|21|4|36|PM|second|evening|| +75877|AAAAAAAAGGICBAAA|75877|21|4|37|PM|second|evening|| +75878|AAAAAAAAHGICBAAA|75878|21|4|38|PM|second|evening|| +75879|AAAAAAAAIGICBAAA|75879|21|4|39|PM|second|evening|| +75880|AAAAAAAAJGICBAAA|75880|21|4|40|PM|second|evening|| +75881|AAAAAAAAKGICBAAA|75881|21|4|41|PM|second|evening|| +75882|AAAAAAAALGICBAAA|75882|21|4|42|PM|second|evening|| +75883|AAAAAAAAMGICBAAA|75883|21|4|43|PM|second|evening|| +75884|AAAAAAAANGICBAAA|75884|21|4|44|PM|second|evening|| +75885|AAAAAAAAOGICBAAA|75885|21|4|45|PM|second|evening|| +75886|AAAAAAAAPGICBAAA|75886|21|4|46|PM|second|evening|| +75887|AAAAAAAAAHICBAAA|75887|21|4|47|PM|second|evening|| +75888|AAAAAAAABHICBAAA|75888|21|4|48|PM|second|evening|| +75889|AAAAAAAACHICBAAA|75889|21|4|49|PM|second|evening|| +75890|AAAAAAAADHICBAAA|75890|21|4|50|PM|second|evening|| +75891|AAAAAAAAEHICBAAA|75891|21|4|51|PM|second|evening|| +75892|AAAAAAAAFHICBAAA|75892|21|4|52|PM|second|evening|| +75893|AAAAAAAAGHICBAAA|75893|21|4|53|PM|second|evening|| +75894|AAAAAAAAHHICBAAA|75894|21|4|54|PM|second|evening|| +75895|AAAAAAAAIHICBAAA|75895|21|4|55|PM|second|evening|| +75896|AAAAAAAAJHICBAAA|75896|21|4|56|PM|second|evening|| +75897|AAAAAAAAKHICBAAA|75897|21|4|57|PM|second|evening|| +75898|AAAAAAAALHICBAAA|75898|21|4|58|PM|second|evening|| +75899|AAAAAAAAMHICBAAA|75899|21|4|59|PM|second|evening|| +75900|AAAAAAAANHICBAAA|75900|21|5|0|PM|second|evening|| +75901|AAAAAAAAOHICBAAA|75901|21|5|1|PM|second|evening|| +75902|AAAAAAAAPHICBAAA|75902|21|5|2|PM|second|evening|| +75903|AAAAAAAAAIICBAAA|75903|21|5|3|PM|second|evening|| +75904|AAAAAAAABIICBAAA|75904|21|5|4|PM|second|evening|| +75905|AAAAAAAACIICBAAA|75905|21|5|5|PM|second|evening|| +75906|AAAAAAAADIICBAAA|75906|21|5|6|PM|second|evening|| +75907|AAAAAAAAEIICBAAA|75907|21|5|7|PM|second|evening|| +75908|AAAAAAAAFIICBAAA|75908|21|5|8|PM|second|evening|| +75909|AAAAAAAAGIICBAAA|75909|21|5|9|PM|second|evening|| +75910|AAAAAAAAHIICBAAA|75910|21|5|10|PM|second|evening|| +75911|AAAAAAAAIIICBAAA|75911|21|5|11|PM|second|evening|| +75912|AAAAAAAAJIICBAAA|75912|21|5|12|PM|second|evening|| +75913|AAAAAAAAKIICBAAA|75913|21|5|13|PM|second|evening|| +75914|AAAAAAAALIICBAAA|75914|21|5|14|PM|second|evening|| +75915|AAAAAAAAMIICBAAA|75915|21|5|15|PM|second|evening|| +75916|AAAAAAAANIICBAAA|75916|21|5|16|PM|second|evening|| +75917|AAAAAAAAOIICBAAA|75917|21|5|17|PM|second|evening|| +75918|AAAAAAAAPIICBAAA|75918|21|5|18|PM|second|evening|| +75919|AAAAAAAAAJICBAAA|75919|21|5|19|PM|second|evening|| +75920|AAAAAAAABJICBAAA|75920|21|5|20|PM|second|evening|| +75921|AAAAAAAACJICBAAA|75921|21|5|21|PM|second|evening|| +75922|AAAAAAAADJICBAAA|75922|21|5|22|PM|second|evening|| +75923|AAAAAAAAEJICBAAA|75923|21|5|23|PM|second|evening|| +75924|AAAAAAAAFJICBAAA|75924|21|5|24|PM|second|evening|| +75925|AAAAAAAAGJICBAAA|75925|21|5|25|PM|second|evening|| +75926|AAAAAAAAHJICBAAA|75926|21|5|26|PM|second|evening|| +75927|AAAAAAAAIJICBAAA|75927|21|5|27|PM|second|evening|| +75928|AAAAAAAAJJICBAAA|75928|21|5|28|PM|second|evening|| +75929|AAAAAAAAKJICBAAA|75929|21|5|29|PM|second|evening|| +75930|AAAAAAAALJICBAAA|75930|21|5|30|PM|second|evening|| +75931|AAAAAAAAMJICBAAA|75931|21|5|31|PM|second|evening|| +75932|AAAAAAAANJICBAAA|75932|21|5|32|PM|second|evening|| +75933|AAAAAAAAOJICBAAA|75933|21|5|33|PM|second|evening|| +75934|AAAAAAAAPJICBAAA|75934|21|5|34|PM|second|evening|| +75935|AAAAAAAAAKICBAAA|75935|21|5|35|PM|second|evening|| +75936|AAAAAAAABKICBAAA|75936|21|5|36|PM|second|evening|| +75937|AAAAAAAACKICBAAA|75937|21|5|37|PM|second|evening|| +75938|AAAAAAAADKICBAAA|75938|21|5|38|PM|second|evening|| +75939|AAAAAAAAEKICBAAA|75939|21|5|39|PM|second|evening|| +75940|AAAAAAAAFKICBAAA|75940|21|5|40|PM|second|evening|| +75941|AAAAAAAAGKICBAAA|75941|21|5|41|PM|second|evening|| +75942|AAAAAAAAHKICBAAA|75942|21|5|42|PM|second|evening|| +75943|AAAAAAAAIKICBAAA|75943|21|5|43|PM|second|evening|| +75944|AAAAAAAAJKICBAAA|75944|21|5|44|PM|second|evening|| +75945|AAAAAAAAKKICBAAA|75945|21|5|45|PM|second|evening|| +75946|AAAAAAAALKICBAAA|75946|21|5|46|PM|second|evening|| +75947|AAAAAAAAMKICBAAA|75947|21|5|47|PM|second|evening|| +75948|AAAAAAAANKICBAAA|75948|21|5|48|PM|second|evening|| +75949|AAAAAAAAOKICBAAA|75949|21|5|49|PM|second|evening|| +75950|AAAAAAAAPKICBAAA|75950|21|5|50|PM|second|evening|| +75951|AAAAAAAAALICBAAA|75951|21|5|51|PM|second|evening|| +75952|AAAAAAAABLICBAAA|75952|21|5|52|PM|second|evening|| +75953|AAAAAAAACLICBAAA|75953|21|5|53|PM|second|evening|| +75954|AAAAAAAADLICBAAA|75954|21|5|54|PM|second|evening|| +75955|AAAAAAAAELICBAAA|75955|21|5|55|PM|second|evening|| +75956|AAAAAAAAFLICBAAA|75956|21|5|56|PM|second|evening|| +75957|AAAAAAAAGLICBAAA|75957|21|5|57|PM|second|evening|| +75958|AAAAAAAAHLICBAAA|75958|21|5|58|PM|second|evening|| +75959|AAAAAAAAILICBAAA|75959|21|5|59|PM|second|evening|| +75960|AAAAAAAAJLICBAAA|75960|21|6|0|PM|second|evening|| +75961|AAAAAAAAKLICBAAA|75961|21|6|1|PM|second|evening|| +75962|AAAAAAAALLICBAAA|75962|21|6|2|PM|second|evening|| +75963|AAAAAAAAMLICBAAA|75963|21|6|3|PM|second|evening|| +75964|AAAAAAAANLICBAAA|75964|21|6|4|PM|second|evening|| +75965|AAAAAAAAOLICBAAA|75965|21|6|5|PM|second|evening|| +75966|AAAAAAAAPLICBAAA|75966|21|6|6|PM|second|evening|| +75967|AAAAAAAAAMICBAAA|75967|21|6|7|PM|second|evening|| +75968|AAAAAAAABMICBAAA|75968|21|6|8|PM|second|evening|| +75969|AAAAAAAACMICBAAA|75969|21|6|9|PM|second|evening|| +75970|AAAAAAAADMICBAAA|75970|21|6|10|PM|second|evening|| +75971|AAAAAAAAEMICBAAA|75971|21|6|11|PM|second|evening|| +75972|AAAAAAAAFMICBAAA|75972|21|6|12|PM|second|evening|| +75973|AAAAAAAAGMICBAAA|75973|21|6|13|PM|second|evening|| +75974|AAAAAAAAHMICBAAA|75974|21|6|14|PM|second|evening|| +75975|AAAAAAAAIMICBAAA|75975|21|6|15|PM|second|evening|| +75976|AAAAAAAAJMICBAAA|75976|21|6|16|PM|second|evening|| +75977|AAAAAAAAKMICBAAA|75977|21|6|17|PM|second|evening|| +75978|AAAAAAAALMICBAAA|75978|21|6|18|PM|second|evening|| +75979|AAAAAAAAMMICBAAA|75979|21|6|19|PM|second|evening|| +75980|AAAAAAAANMICBAAA|75980|21|6|20|PM|second|evening|| +75981|AAAAAAAAOMICBAAA|75981|21|6|21|PM|second|evening|| +75982|AAAAAAAAPMICBAAA|75982|21|6|22|PM|second|evening|| +75983|AAAAAAAAANICBAAA|75983|21|6|23|PM|second|evening|| +75984|AAAAAAAABNICBAAA|75984|21|6|24|PM|second|evening|| +75985|AAAAAAAACNICBAAA|75985|21|6|25|PM|second|evening|| +75986|AAAAAAAADNICBAAA|75986|21|6|26|PM|second|evening|| +75987|AAAAAAAAENICBAAA|75987|21|6|27|PM|second|evening|| +75988|AAAAAAAAFNICBAAA|75988|21|6|28|PM|second|evening|| +75989|AAAAAAAAGNICBAAA|75989|21|6|29|PM|second|evening|| +75990|AAAAAAAAHNICBAAA|75990|21|6|30|PM|second|evening|| +75991|AAAAAAAAINICBAAA|75991|21|6|31|PM|second|evening|| +75992|AAAAAAAAJNICBAAA|75992|21|6|32|PM|second|evening|| +75993|AAAAAAAAKNICBAAA|75993|21|6|33|PM|second|evening|| +75994|AAAAAAAALNICBAAA|75994|21|6|34|PM|second|evening|| +75995|AAAAAAAAMNICBAAA|75995|21|6|35|PM|second|evening|| +75996|AAAAAAAANNICBAAA|75996|21|6|36|PM|second|evening|| +75997|AAAAAAAAONICBAAA|75997|21|6|37|PM|second|evening|| +75998|AAAAAAAAPNICBAAA|75998|21|6|38|PM|second|evening|| +75999|AAAAAAAAAOICBAAA|75999|21|6|39|PM|second|evening|| +76000|AAAAAAAABOICBAAA|76000|21|6|40|PM|second|evening|| +76001|AAAAAAAACOICBAAA|76001|21|6|41|PM|second|evening|| +76002|AAAAAAAADOICBAAA|76002|21|6|42|PM|second|evening|| +76003|AAAAAAAAEOICBAAA|76003|21|6|43|PM|second|evening|| +76004|AAAAAAAAFOICBAAA|76004|21|6|44|PM|second|evening|| +76005|AAAAAAAAGOICBAAA|76005|21|6|45|PM|second|evening|| +76006|AAAAAAAAHOICBAAA|76006|21|6|46|PM|second|evening|| +76007|AAAAAAAAIOICBAAA|76007|21|6|47|PM|second|evening|| +76008|AAAAAAAAJOICBAAA|76008|21|6|48|PM|second|evening|| +76009|AAAAAAAAKOICBAAA|76009|21|6|49|PM|second|evening|| +76010|AAAAAAAALOICBAAA|76010|21|6|50|PM|second|evening|| +76011|AAAAAAAAMOICBAAA|76011|21|6|51|PM|second|evening|| +76012|AAAAAAAANOICBAAA|76012|21|6|52|PM|second|evening|| +76013|AAAAAAAAOOICBAAA|76013|21|6|53|PM|second|evening|| +76014|AAAAAAAAPOICBAAA|76014|21|6|54|PM|second|evening|| +76015|AAAAAAAAAPICBAAA|76015|21|6|55|PM|second|evening|| +76016|AAAAAAAABPICBAAA|76016|21|6|56|PM|second|evening|| +76017|AAAAAAAACPICBAAA|76017|21|6|57|PM|second|evening|| +76018|AAAAAAAADPICBAAA|76018|21|6|58|PM|second|evening|| +76019|AAAAAAAAEPICBAAA|76019|21|6|59|PM|second|evening|| +76020|AAAAAAAAFPICBAAA|76020|21|7|0|PM|second|evening|| +76021|AAAAAAAAGPICBAAA|76021|21|7|1|PM|second|evening|| +76022|AAAAAAAAHPICBAAA|76022|21|7|2|PM|second|evening|| +76023|AAAAAAAAIPICBAAA|76023|21|7|3|PM|second|evening|| +76024|AAAAAAAAJPICBAAA|76024|21|7|4|PM|second|evening|| +76025|AAAAAAAAKPICBAAA|76025|21|7|5|PM|second|evening|| +76026|AAAAAAAALPICBAAA|76026|21|7|6|PM|second|evening|| +76027|AAAAAAAAMPICBAAA|76027|21|7|7|PM|second|evening|| +76028|AAAAAAAANPICBAAA|76028|21|7|8|PM|second|evening|| +76029|AAAAAAAAOPICBAAA|76029|21|7|9|PM|second|evening|| +76030|AAAAAAAAPPICBAAA|76030|21|7|10|PM|second|evening|| +76031|AAAAAAAAAAJCBAAA|76031|21|7|11|PM|second|evening|| +76032|AAAAAAAABAJCBAAA|76032|21|7|12|PM|second|evening|| +76033|AAAAAAAACAJCBAAA|76033|21|7|13|PM|second|evening|| +76034|AAAAAAAADAJCBAAA|76034|21|7|14|PM|second|evening|| +76035|AAAAAAAAEAJCBAAA|76035|21|7|15|PM|second|evening|| +76036|AAAAAAAAFAJCBAAA|76036|21|7|16|PM|second|evening|| +76037|AAAAAAAAGAJCBAAA|76037|21|7|17|PM|second|evening|| +76038|AAAAAAAAHAJCBAAA|76038|21|7|18|PM|second|evening|| +76039|AAAAAAAAIAJCBAAA|76039|21|7|19|PM|second|evening|| +76040|AAAAAAAAJAJCBAAA|76040|21|7|20|PM|second|evening|| +76041|AAAAAAAAKAJCBAAA|76041|21|7|21|PM|second|evening|| +76042|AAAAAAAALAJCBAAA|76042|21|7|22|PM|second|evening|| +76043|AAAAAAAAMAJCBAAA|76043|21|7|23|PM|second|evening|| +76044|AAAAAAAANAJCBAAA|76044|21|7|24|PM|second|evening|| +76045|AAAAAAAAOAJCBAAA|76045|21|7|25|PM|second|evening|| +76046|AAAAAAAAPAJCBAAA|76046|21|7|26|PM|second|evening|| +76047|AAAAAAAAABJCBAAA|76047|21|7|27|PM|second|evening|| +76048|AAAAAAAABBJCBAAA|76048|21|7|28|PM|second|evening|| +76049|AAAAAAAACBJCBAAA|76049|21|7|29|PM|second|evening|| +76050|AAAAAAAADBJCBAAA|76050|21|7|30|PM|second|evening|| +76051|AAAAAAAAEBJCBAAA|76051|21|7|31|PM|second|evening|| +76052|AAAAAAAAFBJCBAAA|76052|21|7|32|PM|second|evening|| +76053|AAAAAAAAGBJCBAAA|76053|21|7|33|PM|second|evening|| +76054|AAAAAAAAHBJCBAAA|76054|21|7|34|PM|second|evening|| +76055|AAAAAAAAIBJCBAAA|76055|21|7|35|PM|second|evening|| +76056|AAAAAAAAJBJCBAAA|76056|21|7|36|PM|second|evening|| +76057|AAAAAAAAKBJCBAAA|76057|21|7|37|PM|second|evening|| +76058|AAAAAAAALBJCBAAA|76058|21|7|38|PM|second|evening|| +76059|AAAAAAAAMBJCBAAA|76059|21|7|39|PM|second|evening|| +76060|AAAAAAAANBJCBAAA|76060|21|7|40|PM|second|evening|| +76061|AAAAAAAAOBJCBAAA|76061|21|7|41|PM|second|evening|| +76062|AAAAAAAAPBJCBAAA|76062|21|7|42|PM|second|evening|| +76063|AAAAAAAAACJCBAAA|76063|21|7|43|PM|second|evening|| +76064|AAAAAAAABCJCBAAA|76064|21|7|44|PM|second|evening|| +76065|AAAAAAAACCJCBAAA|76065|21|7|45|PM|second|evening|| +76066|AAAAAAAADCJCBAAA|76066|21|7|46|PM|second|evening|| +76067|AAAAAAAAECJCBAAA|76067|21|7|47|PM|second|evening|| +76068|AAAAAAAAFCJCBAAA|76068|21|7|48|PM|second|evening|| +76069|AAAAAAAAGCJCBAAA|76069|21|7|49|PM|second|evening|| +76070|AAAAAAAAHCJCBAAA|76070|21|7|50|PM|second|evening|| +76071|AAAAAAAAICJCBAAA|76071|21|7|51|PM|second|evening|| +76072|AAAAAAAAJCJCBAAA|76072|21|7|52|PM|second|evening|| +76073|AAAAAAAAKCJCBAAA|76073|21|7|53|PM|second|evening|| +76074|AAAAAAAALCJCBAAA|76074|21|7|54|PM|second|evening|| +76075|AAAAAAAAMCJCBAAA|76075|21|7|55|PM|second|evening|| +76076|AAAAAAAANCJCBAAA|76076|21|7|56|PM|second|evening|| +76077|AAAAAAAAOCJCBAAA|76077|21|7|57|PM|second|evening|| +76078|AAAAAAAAPCJCBAAA|76078|21|7|58|PM|second|evening|| +76079|AAAAAAAAADJCBAAA|76079|21|7|59|PM|second|evening|| +76080|AAAAAAAABDJCBAAA|76080|21|8|0|PM|second|evening|| +76081|AAAAAAAACDJCBAAA|76081|21|8|1|PM|second|evening|| +76082|AAAAAAAADDJCBAAA|76082|21|8|2|PM|second|evening|| +76083|AAAAAAAAEDJCBAAA|76083|21|8|3|PM|second|evening|| +76084|AAAAAAAAFDJCBAAA|76084|21|8|4|PM|second|evening|| +76085|AAAAAAAAGDJCBAAA|76085|21|8|5|PM|second|evening|| +76086|AAAAAAAAHDJCBAAA|76086|21|8|6|PM|second|evening|| +76087|AAAAAAAAIDJCBAAA|76087|21|8|7|PM|second|evening|| +76088|AAAAAAAAJDJCBAAA|76088|21|8|8|PM|second|evening|| +76089|AAAAAAAAKDJCBAAA|76089|21|8|9|PM|second|evening|| +76090|AAAAAAAALDJCBAAA|76090|21|8|10|PM|second|evening|| +76091|AAAAAAAAMDJCBAAA|76091|21|8|11|PM|second|evening|| +76092|AAAAAAAANDJCBAAA|76092|21|8|12|PM|second|evening|| +76093|AAAAAAAAODJCBAAA|76093|21|8|13|PM|second|evening|| +76094|AAAAAAAAPDJCBAAA|76094|21|8|14|PM|second|evening|| +76095|AAAAAAAAAEJCBAAA|76095|21|8|15|PM|second|evening|| +76096|AAAAAAAABEJCBAAA|76096|21|8|16|PM|second|evening|| +76097|AAAAAAAACEJCBAAA|76097|21|8|17|PM|second|evening|| +76098|AAAAAAAADEJCBAAA|76098|21|8|18|PM|second|evening|| +76099|AAAAAAAAEEJCBAAA|76099|21|8|19|PM|second|evening|| +76100|AAAAAAAAFEJCBAAA|76100|21|8|20|PM|second|evening|| +76101|AAAAAAAAGEJCBAAA|76101|21|8|21|PM|second|evening|| +76102|AAAAAAAAHEJCBAAA|76102|21|8|22|PM|second|evening|| +76103|AAAAAAAAIEJCBAAA|76103|21|8|23|PM|second|evening|| +76104|AAAAAAAAJEJCBAAA|76104|21|8|24|PM|second|evening|| +76105|AAAAAAAAKEJCBAAA|76105|21|8|25|PM|second|evening|| +76106|AAAAAAAALEJCBAAA|76106|21|8|26|PM|second|evening|| +76107|AAAAAAAAMEJCBAAA|76107|21|8|27|PM|second|evening|| +76108|AAAAAAAANEJCBAAA|76108|21|8|28|PM|second|evening|| +76109|AAAAAAAAOEJCBAAA|76109|21|8|29|PM|second|evening|| +76110|AAAAAAAAPEJCBAAA|76110|21|8|30|PM|second|evening|| +76111|AAAAAAAAAFJCBAAA|76111|21|8|31|PM|second|evening|| +76112|AAAAAAAABFJCBAAA|76112|21|8|32|PM|second|evening|| +76113|AAAAAAAACFJCBAAA|76113|21|8|33|PM|second|evening|| +76114|AAAAAAAADFJCBAAA|76114|21|8|34|PM|second|evening|| +76115|AAAAAAAAEFJCBAAA|76115|21|8|35|PM|second|evening|| +76116|AAAAAAAAFFJCBAAA|76116|21|8|36|PM|second|evening|| +76117|AAAAAAAAGFJCBAAA|76117|21|8|37|PM|second|evening|| +76118|AAAAAAAAHFJCBAAA|76118|21|8|38|PM|second|evening|| +76119|AAAAAAAAIFJCBAAA|76119|21|8|39|PM|second|evening|| +76120|AAAAAAAAJFJCBAAA|76120|21|8|40|PM|second|evening|| +76121|AAAAAAAAKFJCBAAA|76121|21|8|41|PM|second|evening|| +76122|AAAAAAAALFJCBAAA|76122|21|8|42|PM|second|evening|| +76123|AAAAAAAAMFJCBAAA|76123|21|8|43|PM|second|evening|| +76124|AAAAAAAANFJCBAAA|76124|21|8|44|PM|second|evening|| +76125|AAAAAAAAOFJCBAAA|76125|21|8|45|PM|second|evening|| +76126|AAAAAAAAPFJCBAAA|76126|21|8|46|PM|second|evening|| +76127|AAAAAAAAAGJCBAAA|76127|21|8|47|PM|second|evening|| +76128|AAAAAAAABGJCBAAA|76128|21|8|48|PM|second|evening|| +76129|AAAAAAAACGJCBAAA|76129|21|8|49|PM|second|evening|| +76130|AAAAAAAADGJCBAAA|76130|21|8|50|PM|second|evening|| +76131|AAAAAAAAEGJCBAAA|76131|21|8|51|PM|second|evening|| +76132|AAAAAAAAFGJCBAAA|76132|21|8|52|PM|second|evening|| +76133|AAAAAAAAGGJCBAAA|76133|21|8|53|PM|second|evening|| +76134|AAAAAAAAHGJCBAAA|76134|21|8|54|PM|second|evening|| +76135|AAAAAAAAIGJCBAAA|76135|21|8|55|PM|second|evening|| +76136|AAAAAAAAJGJCBAAA|76136|21|8|56|PM|second|evening|| +76137|AAAAAAAAKGJCBAAA|76137|21|8|57|PM|second|evening|| +76138|AAAAAAAALGJCBAAA|76138|21|8|58|PM|second|evening|| +76139|AAAAAAAAMGJCBAAA|76139|21|8|59|PM|second|evening|| +76140|AAAAAAAANGJCBAAA|76140|21|9|0|PM|second|evening|| +76141|AAAAAAAAOGJCBAAA|76141|21|9|1|PM|second|evening|| +76142|AAAAAAAAPGJCBAAA|76142|21|9|2|PM|second|evening|| +76143|AAAAAAAAAHJCBAAA|76143|21|9|3|PM|second|evening|| +76144|AAAAAAAABHJCBAAA|76144|21|9|4|PM|second|evening|| +76145|AAAAAAAACHJCBAAA|76145|21|9|5|PM|second|evening|| +76146|AAAAAAAADHJCBAAA|76146|21|9|6|PM|second|evening|| +76147|AAAAAAAAEHJCBAAA|76147|21|9|7|PM|second|evening|| +76148|AAAAAAAAFHJCBAAA|76148|21|9|8|PM|second|evening|| +76149|AAAAAAAAGHJCBAAA|76149|21|9|9|PM|second|evening|| +76150|AAAAAAAAHHJCBAAA|76150|21|9|10|PM|second|evening|| +76151|AAAAAAAAIHJCBAAA|76151|21|9|11|PM|second|evening|| +76152|AAAAAAAAJHJCBAAA|76152|21|9|12|PM|second|evening|| +76153|AAAAAAAAKHJCBAAA|76153|21|9|13|PM|second|evening|| +76154|AAAAAAAALHJCBAAA|76154|21|9|14|PM|second|evening|| +76155|AAAAAAAAMHJCBAAA|76155|21|9|15|PM|second|evening|| +76156|AAAAAAAANHJCBAAA|76156|21|9|16|PM|second|evening|| +76157|AAAAAAAAOHJCBAAA|76157|21|9|17|PM|second|evening|| +76158|AAAAAAAAPHJCBAAA|76158|21|9|18|PM|second|evening|| +76159|AAAAAAAAAIJCBAAA|76159|21|9|19|PM|second|evening|| +76160|AAAAAAAABIJCBAAA|76160|21|9|20|PM|second|evening|| +76161|AAAAAAAACIJCBAAA|76161|21|9|21|PM|second|evening|| +76162|AAAAAAAADIJCBAAA|76162|21|9|22|PM|second|evening|| +76163|AAAAAAAAEIJCBAAA|76163|21|9|23|PM|second|evening|| +76164|AAAAAAAAFIJCBAAA|76164|21|9|24|PM|second|evening|| +76165|AAAAAAAAGIJCBAAA|76165|21|9|25|PM|second|evening|| +76166|AAAAAAAAHIJCBAAA|76166|21|9|26|PM|second|evening|| +76167|AAAAAAAAIIJCBAAA|76167|21|9|27|PM|second|evening|| +76168|AAAAAAAAJIJCBAAA|76168|21|9|28|PM|second|evening|| +76169|AAAAAAAAKIJCBAAA|76169|21|9|29|PM|second|evening|| +76170|AAAAAAAALIJCBAAA|76170|21|9|30|PM|second|evening|| +76171|AAAAAAAAMIJCBAAA|76171|21|9|31|PM|second|evening|| +76172|AAAAAAAANIJCBAAA|76172|21|9|32|PM|second|evening|| +76173|AAAAAAAAOIJCBAAA|76173|21|9|33|PM|second|evening|| +76174|AAAAAAAAPIJCBAAA|76174|21|9|34|PM|second|evening|| +76175|AAAAAAAAAJJCBAAA|76175|21|9|35|PM|second|evening|| +76176|AAAAAAAABJJCBAAA|76176|21|9|36|PM|second|evening|| +76177|AAAAAAAACJJCBAAA|76177|21|9|37|PM|second|evening|| +76178|AAAAAAAADJJCBAAA|76178|21|9|38|PM|second|evening|| +76179|AAAAAAAAEJJCBAAA|76179|21|9|39|PM|second|evening|| +76180|AAAAAAAAFJJCBAAA|76180|21|9|40|PM|second|evening|| +76181|AAAAAAAAGJJCBAAA|76181|21|9|41|PM|second|evening|| +76182|AAAAAAAAHJJCBAAA|76182|21|9|42|PM|second|evening|| +76183|AAAAAAAAIJJCBAAA|76183|21|9|43|PM|second|evening|| +76184|AAAAAAAAJJJCBAAA|76184|21|9|44|PM|second|evening|| +76185|AAAAAAAAKJJCBAAA|76185|21|9|45|PM|second|evening|| +76186|AAAAAAAALJJCBAAA|76186|21|9|46|PM|second|evening|| +76187|AAAAAAAAMJJCBAAA|76187|21|9|47|PM|second|evening|| +76188|AAAAAAAANJJCBAAA|76188|21|9|48|PM|second|evening|| +76189|AAAAAAAAOJJCBAAA|76189|21|9|49|PM|second|evening|| +76190|AAAAAAAAPJJCBAAA|76190|21|9|50|PM|second|evening|| +76191|AAAAAAAAAKJCBAAA|76191|21|9|51|PM|second|evening|| +76192|AAAAAAAABKJCBAAA|76192|21|9|52|PM|second|evening|| +76193|AAAAAAAACKJCBAAA|76193|21|9|53|PM|second|evening|| +76194|AAAAAAAADKJCBAAA|76194|21|9|54|PM|second|evening|| +76195|AAAAAAAAEKJCBAAA|76195|21|9|55|PM|second|evening|| +76196|AAAAAAAAFKJCBAAA|76196|21|9|56|PM|second|evening|| +76197|AAAAAAAAGKJCBAAA|76197|21|9|57|PM|second|evening|| +76198|AAAAAAAAHKJCBAAA|76198|21|9|58|PM|second|evening|| +76199|AAAAAAAAIKJCBAAA|76199|21|9|59|PM|second|evening|| +76200|AAAAAAAAJKJCBAAA|76200|21|10|0|PM|second|evening|| +76201|AAAAAAAAKKJCBAAA|76201|21|10|1|PM|second|evening|| +76202|AAAAAAAALKJCBAAA|76202|21|10|2|PM|second|evening|| +76203|AAAAAAAAMKJCBAAA|76203|21|10|3|PM|second|evening|| +76204|AAAAAAAANKJCBAAA|76204|21|10|4|PM|second|evening|| +76205|AAAAAAAAOKJCBAAA|76205|21|10|5|PM|second|evening|| +76206|AAAAAAAAPKJCBAAA|76206|21|10|6|PM|second|evening|| +76207|AAAAAAAAALJCBAAA|76207|21|10|7|PM|second|evening|| +76208|AAAAAAAABLJCBAAA|76208|21|10|8|PM|second|evening|| +76209|AAAAAAAACLJCBAAA|76209|21|10|9|PM|second|evening|| +76210|AAAAAAAADLJCBAAA|76210|21|10|10|PM|second|evening|| +76211|AAAAAAAAELJCBAAA|76211|21|10|11|PM|second|evening|| +76212|AAAAAAAAFLJCBAAA|76212|21|10|12|PM|second|evening|| +76213|AAAAAAAAGLJCBAAA|76213|21|10|13|PM|second|evening|| +76214|AAAAAAAAHLJCBAAA|76214|21|10|14|PM|second|evening|| +76215|AAAAAAAAILJCBAAA|76215|21|10|15|PM|second|evening|| +76216|AAAAAAAAJLJCBAAA|76216|21|10|16|PM|second|evening|| +76217|AAAAAAAAKLJCBAAA|76217|21|10|17|PM|second|evening|| +76218|AAAAAAAALLJCBAAA|76218|21|10|18|PM|second|evening|| +76219|AAAAAAAAMLJCBAAA|76219|21|10|19|PM|second|evening|| +76220|AAAAAAAANLJCBAAA|76220|21|10|20|PM|second|evening|| +76221|AAAAAAAAOLJCBAAA|76221|21|10|21|PM|second|evening|| +76222|AAAAAAAAPLJCBAAA|76222|21|10|22|PM|second|evening|| +76223|AAAAAAAAAMJCBAAA|76223|21|10|23|PM|second|evening|| +76224|AAAAAAAABMJCBAAA|76224|21|10|24|PM|second|evening|| +76225|AAAAAAAACMJCBAAA|76225|21|10|25|PM|second|evening|| +76226|AAAAAAAADMJCBAAA|76226|21|10|26|PM|second|evening|| +76227|AAAAAAAAEMJCBAAA|76227|21|10|27|PM|second|evening|| +76228|AAAAAAAAFMJCBAAA|76228|21|10|28|PM|second|evening|| +76229|AAAAAAAAGMJCBAAA|76229|21|10|29|PM|second|evening|| +76230|AAAAAAAAHMJCBAAA|76230|21|10|30|PM|second|evening|| +76231|AAAAAAAAIMJCBAAA|76231|21|10|31|PM|second|evening|| +76232|AAAAAAAAJMJCBAAA|76232|21|10|32|PM|second|evening|| +76233|AAAAAAAAKMJCBAAA|76233|21|10|33|PM|second|evening|| +76234|AAAAAAAALMJCBAAA|76234|21|10|34|PM|second|evening|| +76235|AAAAAAAAMMJCBAAA|76235|21|10|35|PM|second|evening|| +76236|AAAAAAAANMJCBAAA|76236|21|10|36|PM|second|evening|| +76237|AAAAAAAAOMJCBAAA|76237|21|10|37|PM|second|evening|| +76238|AAAAAAAAPMJCBAAA|76238|21|10|38|PM|second|evening|| +76239|AAAAAAAAANJCBAAA|76239|21|10|39|PM|second|evening|| +76240|AAAAAAAABNJCBAAA|76240|21|10|40|PM|second|evening|| +76241|AAAAAAAACNJCBAAA|76241|21|10|41|PM|second|evening|| +76242|AAAAAAAADNJCBAAA|76242|21|10|42|PM|second|evening|| +76243|AAAAAAAAENJCBAAA|76243|21|10|43|PM|second|evening|| +76244|AAAAAAAAFNJCBAAA|76244|21|10|44|PM|second|evening|| +76245|AAAAAAAAGNJCBAAA|76245|21|10|45|PM|second|evening|| +76246|AAAAAAAAHNJCBAAA|76246|21|10|46|PM|second|evening|| +76247|AAAAAAAAINJCBAAA|76247|21|10|47|PM|second|evening|| +76248|AAAAAAAAJNJCBAAA|76248|21|10|48|PM|second|evening|| +76249|AAAAAAAAKNJCBAAA|76249|21|10|49|PM|second|evening|| +76250|AAAAAAAALNJCBAAA|76250|21|10|50|PM|second|evening|| +76251|AAAAAAAAMNJCBAAA|76251|21|10|51|PM|second|evening|| +76252|AAAAAAAANNJCBAAA|76252|21|10|52|PM|second|evening|| +76253|AAAAAAAAONJCBAAA|76253|21|10|53|PM|second|evening|| +76254|AAAAAAAAPNJCBAAA|76254|21|10|54|PM|second|evening|| +76255|AAAAAAAAAOJCBAAA|76255|21|10|55|PM|second|evening|| +76256|AAAAAAAABOJCBAAA|76256|21|10|56|PM|second|evening|| +76257|AAAAAAAACOJCBAAA|76257|21|10|57|PM|second|evening|| +76258|AAAAAAAADOJCBAAA|76258|21|10|58|PM|second|evening|| +76259|AAAAAAAAEOJCBAAA|76259|21|10|59|PM|second|evening|| +76260|AAAAAAAAFOJCBAAA|76260|21|11|0|PM|second|evening|| +76261|AAAAAAAAGOJCBAAA|76261|21|11|1|PM|second|evening|| +76262|AAAAAAAAHOJCBAAA|76262|21|11|2|PM|second|evening|| +76263|AAAAAAAAIOJCBAAA|76263|21|11|3|PM|second|evening|| +76264|AAAAAAAAJOJCBAAA|76264|21|11|4|PM|second|evening|| +76265|AAAAAAAAKOJCBAAA|76265|21|11|5|PM|second|evening|| +76266|AAAAAAAALOJCBAAA|76266|21|11|6|PM|second|evening|| +76267|AAAAAAAAMOJCBAAA|76267|21|11|7|PM|second|evening|| +76268|AAAAAAAANOJCBAAA|76268|21|11|8|PM|second|evening|| +76269|AAAAAAAAOOJCBAAA|76269|21|11|9|PM|second|evening|| +76270|AAAAAAAAPOJCBAAA|76270|21|11|10|PM|second|evening|| +76271|AAAAAAAAAPJCBAAA|76271|21|11|11|PM|second|evening|| +76272|AAAAAAAABPJCBAAA|76272|21|11|12|PM|second|evening|| +76273|AAAAAAAACPJCBAAA|76273|21|11|13|PM|second|evening|| +76274|AAAAAAAADPJCBAAA|76274|21|11|14|PM|second|evening|| +76275|AAAAAAAAEPJCBAAA|76275|21|11|15|PM|second|evening|| +76276|AAAAAAAAFPJCBAAA|76276|21|11|16|PM|second|evening|| +76277|AAAAAAAAGPJCBAAA|76277|21|11|17|PM|second|evening|| +76278|AAAAAAAAHPJCBAAA|76278|21|11|18|PM|second|evening|| +76279|AAAAAAAAIPJCBAAA|76279|21|11|19|PM|second|evening|| +76280|AAAAAAAAJPJCBAAA|76280|21|11|20|PM|second|evening|| +76281|AAAAAAAAKPJCBAAA|76281|21|11|21|PM|second|evening|| +76282|AAAAAAAALPJCBAAA|76282|21|11|22|PM|second|evening|| +76283|AAAAAAAAMPJCBAAA|76283|21|11|23|PM|second|evening|| +76284|AAAAAAAANPJCBAAA|76284|21|11|24|PM|second|evening|| +76285|AAAAAAAAOPJCBAAA|76285|21|11|25|PM|second|evening|| +76286|AAAAAAAAPPJCBAAA|76286|21|11|26|PM|second|evening|| +76287|AAAAAAAAAAKCBAAA|76287|21|11|27|PM|second|evening|| +76288|AAAAAAAABAKCBAAA|76288|21|11|28|PM|second|evening|| +76289|AAAAAAAACAKCBAAA|76289|21|11|29|PM|second|evening|| +76290|AAAAAAAADAKCBAAA|76290|21|11|30|PM|second|evening|| +76291|AAAAAAAAEAKCBAAA|76291|21|11|31|PM|second|evening|| +76292|AAAAAAAAFAKCBAAA|76292|21|11|32|PM|second|evening|| +76293|AAAAAAAAGAKCBAAA|76293|21|11|33|PM|second|evening|| +76294|AAAAAAAAHAKCBAAA|76294|21|11|34|PM|second|evening|| +76295|AAAAAAAAIAKCBAAA|76295|21|11|35|PM|second|evening|| +76296|AAAAAAAAJAKCBAAA|76296|21|11|36|PM|second|evening|| +76297|AAAAAAAAKAKCBAAA|76297|21|11|37|PM|second|evening|| +76298|AAAAAAAALAKCBAAA|76298|21|11|38|PM|second|evening|| +76299|AAAAAAAAMAKCBAAA|76299|21|11|39|PM|second|evening|| +76300|AAAAAAAANAKCBAAA|76300|21|11|40|PM|second|evening|| +76301|AAAAAAAAOAKCBAAA|76301|21|11|41|PM|second|evening|| +76302|AAAAAAAAPAKCBAAA|76302|21|11|42|PM|second|evening|| +76303|AAAAAAAAABKCBAAA|76303|21|11|43|PM|second|evening|| +76304|AAAAAAAABBKCBAAA|76304|21|11|44|PM|second|evening|| +76305|AAAAAAAACBKCBAAA|76305|21|11|45|PM|second|evening|| +76306|AAAAAAAADBKCBAAA|76306|21|11|46|PM|second|evening|| +76307|AAAAAAAAEBKCBAAA|76307|21|11|47|PM|second|evening|| +76308|AAAAAAAAFBKCBAAA|76308|21|11|48|PM|second|evening|| +76309|AAAAAAAAGBKCBAAA|76309|21|11|49|PM|second|evening|| +76310|AAAAAAAAHBKCBAAA|76310|21|11|50|PM|second|evening|| +76311|AAAAAAAAIBKCBAAA|76311|21|11|51|PM|second|evening|| +76312|AAAAAAAAJBKCBAAA|76312|21|11|52|PM|second|evening|| +76313|AAAAAAAAKBKCBAAA|76313|21|11|53|PM|second|evening|| +76314|AAAAAAAALBKCBAAA|76314|21|11|54|PM|second|evening|| +76315|AAAAAAAAMBKCBAAA|76315|21|11|55|PM|second|evening|| +76316|AAAAAAAANBKCBAAA|76316|21|11|56|PM|second|evening|| +76317|AAAAAAAAOBKCBAAA|76317|21|11|57|PM|second|evening|| +76318|AAAAAAAAPBKCBAAA|76318|21|11|58|PM|second|evening|| +76319|AAAAAAAAACKCBAAA|76319|21|11|59|PM|second|evening|| +76320|AAAAAAAABCKCBAAA|76320|21|12|0|PM|second|evening|| +76321|AAAAAAAACCKCBAAA|76321|21|12|1|PM|second|evening|| +76322|AAAAAAAADCKCBAAA|76322|21|12|2|PM|second|evening|| +76323|AAAAAAAAECKCBAAA|76323|21|12|3|PM|second|evening|| +76324|AAAAAAAAFCKCBAAA|76324|21|12|4|PM|second|evening|| +76325|AAAAAAAAGCKCBAAA|76325|21|12|5|PM|second|evening|| +76326|AAAAAAAAHCKCBAAA|76326|21|12|6|PM|second|evening|| +76327|AAAAAAAAICKCBAAA|76327|21|12|7|PM|second|evening|| +76328|AAAAAAAAJCKCBAAA|76328|21|12|8|PM|second|evening|| +76329|AAAAAAAAKCKCBAAA|76329|21|12|9|PM|second|evening|| +76330|AAAAAAAALCKCBAAA|76330|21|12|10|PM|second|evening|| +76331|AAAAAAAAMCKCBAAA|76331|21|12|11|PM|second|evening|| +76332|AAAAAAAANCKCBAAA|76332|21|12|12|PM|second|evening|| +76333|AAAAAAAAOCKCBAAA|76333|21|12|13|PM|second|evening|| +76334|AAAAAAAAPCKCBAAA|76334|21|12|14|PM|second|evening|| +76335|AAAAAAAAADKCBAAA|76335|21|12|15|PM|second|evening|| +76336|AAAAAAAABDKCBAAA|76336|21|12|16|PM|second|evening|| +76337|AAAAAAAACDKCBAAA|76337|21|12|17|PM|second|evening|| +76338|AAAAAAAADDKCBAAA|76338|21|12|18|PM|second|evening|| +76339|AAAAAAAAEDKCBAAA|76339|21|12|19|PM|second|evening|| +76340|AAAAAAAAFDKCBAAA|76340|21|12|20|PM|second|evening|| +76341|AAAAAAAAGDKCBAAA|76341|21|12|21|PM|second|evening|| +76342|AAAAAAAAHDKCBAAA|76342|21|12|22|PM|second|evening|| +76343|AAAAAAAAIDKCBAAA|76343|21|12|23|PM|second|evening|| +76344|AAAAAAAAJDKCBAAA|76344|21|12|24|PM|second|evening|| +76345|AAAAAAAAKDKCBAAA|76345|21|12|25|PM|second|evening|| +76346|AAAAAAAALDKCBAAA|76346|21|12|26|PM|second|evening|| +76347|AAAAAAAAMDKCBAAA|76347|21|12|27|PM|second|evening|| +76348|AAAAAAAANDKCBAAA|76348|21|12|28|PM|second|evening|| +76349|AAAAAAAAODKCBAAA|76349|21|12|29|PM|second|evening|| +76350|AAAAAAAAPDKCBAAA|76350|21|12|30|PM|second|evening|| +76351|AAAAAAAAAEKCBAAA|76351|21|12|31|PM|second|evening|| +76352|AAAAAAAABEKCBAAA|76352|21|12|32|PM|second|evening|| +76353|AAAAAAAACEKCBAAA|76353|21|12|33|PM|second|evening|| +76354|AAAAAAAADEKCBAAA|76354|21|12|34|PM|second|evening|| +76355|AAAAAAAAEEKCBAAA|76355|21|12|35|PM|second|evening|| +76356|AAAAAAAAFEKCBAAA|76356|21|12|36|PM|second|evening|| +76357|AAAAAAAAGEKCBAAA|76357|21|12|37|PM|second|evening|| +76358|AAAAAAAAHEKCBAAA|76358|21|12|38|PM|second|evening|| +76359|AAAAAAAAIEKCBAAA|76359|21|12|39|PM|second|evening|| +76360|AAAAAAAAJEKCBAAA|76360|21|12|40|PM|second|evening|| +76361|AAAAAAAAKEKCBAAA|76361|21|12|41|PM|second|evening|| +76362|AAAAAAAALEKCBAAA|76362|21|12|42|PM|second|evening|| +76363|AAAAAAAAMEKCBAAA|76363|21|12|43|PM|second|evening|| +76364|AAAAAAAANEKCBAAA|76364|21|12|44|PM|second|evening|| +76365|AAAAAAAAOEKCBAAA|76365|21|12|45|PM|second|evening|| +76366|AAAAAAAAPEKCBAAA|76366|21|12|46|PM|second|evening|| +76367|AAAAAAAAAFKCBAAA|76367|21|12|47|PM|second|evening|| +76368|AAAAAAAABFKCBAAA|76368|21|12|48|PM|second|evening|| +76369|AAAAAAAACFKCBAAA|76369|21|12|49|PM|second|evening|| +76370|AAAAAAAADFKCBAAA|76370|21|12|50|PM|second|evening|| +76371|AAAAAAAAEFKCBAAA|76371|21|12|51|PM|second|evening|| +76372|AAAAAAAAFFKCBAAA|76372|21|12|52|PM|second|evening|| +76373|AAAAAAAAGFKCBAAA|76373|21|12|53|PM|second|evening|| +76374|AAAAAAAAHFKCBAAA|76374|21|12|54|PM|second|evening|| +76375|AAAAAAAAIFKCBAAA|76375|21|12|55|PM|second|evening|| +76376|AAAAAAAAJFKCBAAA|76376|21|12|56|PM|second|evening|| +76377|AAAAAAAAKFKCBAAA|76377|21|12|57|PM|second|evening|| +76378|AAAAAAAALFKCBAAA|76378|21|12|58|PM|second|evening|| +76379|AAAAAAAAMFKCBAAA|76379|21|12|59|PM|second|evening|| +76380|AAAAAAAANFKCBAAA|76380|21|13|0|PM|second|evening|| +76381|AAAAAAAAOFKCBAAA|76381|21|13|1|PM|second|evening|| +76382|AAAAAAAAPFKCBAAA|76382|21|13|2|PM|second|evening|| +76383|AAAAAAAAAGKCBAAA|76383|21|13|3|PM|second|evening|| +76384|AAAAAAAABGKCBAAA|76384|21|13|4|PM|second|evening|| +76385|AAAAAAAACGKCBAAA|76385|21|13|5|PM|second|evening|| +76386|AAAAAAAADGKCBAAA|76386|21|13|6|PM|second|evening|| +76387|AAAAAAAAEGKCBAAA|76387|21|13|7|PM|second|evening|| +76388|AAAAAAAAFGKCBAAA|76388|21|13|8|PM|second|evening|| +76389|AAAAAAAAGGKCBAAA|76389|21|13|9|PM|second|evening|| +76390|AAAAAAAAHGKCBAAA|76390|21|13|10|PM|second|evening|| +76391|AAAAAAAAIGKCBAAA|76391|21|13|11|PM|second|evening|| +76392|AAAAAAAAJGKCBAAA|76392|21|13|12|PM|second|evening|| +76393|AAAAAAAAKGKCBAAA|76393|21|13|13|PM|second|evening|| +76394|AAAAAAAALGKCBAAA|76394|21|13|14|PM|second|evening|| +76395|AAAAAAAAMGKCBAAA|76395|21|13|15|PM|second|evening|| +76396|AAAAAAAANGKCBAAA|76396|21|13|16|PM|second|evening|| +76397|AAAAAAAAOGKCBAAA|76397|21|13|17|PM|second|evening|| +76398|AAAAAAAAPGKCBAAA|76398|21|13|18|PM|second|evening|| +76399|AAAAAAAAAHKCBAAA|76399|21|13|19|PM|second|evening|| +76400|AAAAAAAABHKCBAAA|76400|21|13|20|PM|second|evening|| +76401|AAAAAAAACHKCBAAA|76401|21|13|21|PM|second|evening|| +76402|AAAAAAAADHKCBAAA|76402|21|13|22|PM|second|evening|| +76403|AAAAAAAAEHKCBAAA|76403|21|13|23|PM|second|evening|| +76404|AAAAAAAAFHKCBAAA|76404|21|13|24|PM|second|evening|| +76405|AAAAAAAAGHKCBAAA|76405|21|13|25|PM|second|evening|| +76406|AAAAAAAAHHKCBAAA|76406|21|13|26|PM|second|evening|| +76407|AAAAAAAAIHKCBAAA|76407|21|13|27|PM|second|evening|| +76408|AAAAAAAAJHKCBAAA|76408|21|13|28|PM|second|evening|| +76409|AAAAAAAAKHKCBAAA|76409|21|13|29|PM|second|evening|| +76410|AAAAAAAALHKCBAAA|76410|21|13|30|PM|second|evening|| +76411|AAAAAAAAMHKCBAAA|76411|21|13|31|PM|second|evening|| +76412|AAAAAAAANHKCBAAA|76412|21|13|32|PM|second|evening|| +76413|AAAAAAAAOHKCBAAA|76413|21|13|33|PM|second|evening|| +76414|AAAAAAAAPHKCBAAA|76414|21|13|34|PM|second|evening|| +76415|AAAAAAAAAIKCBAAA|76415|21|13|35|PM|second|evening|| +76416|AAAAAAAABIKCBAAA|76416|21|13|36|PM|second|evening|| +76417|AAAAAAAACIKCBAAA|76417|21|13|37|PM|second|evening|| +76418|AAAAAAAADIKCBAAA|76418|21|13|38|PM|second|evening|| +76419|AAAAAAAAEIKCBAAA|76419|21|13|39|PM|second|evening|| +76420|AAAAAAAAFIKCBAAA|76420|21|13|40|PM|second|evening|| +76421|AAAAAAAAGIKCBAAA|76421|21|13|41|PM|second|evening|| +76422|AAAAAAAAHIKCBAAA|76422|21|13|42|PM|second|evening|| +76423|AAAAAAAAIIKCBAAA|76423|21|13|43|PM|second|evening|| +76424|AAAAAAAAJIKCBAAA|76424|21|13|44|PM|second|evening|| +76425|AAAAAAAAKIKCBAAA|76425|21|13|45|PM|second|evening|| +76426|AAAAAAAALIKCBAAA|76426|21|13|46|PM|second|evening|| +76427|AAAAAAAAMIKCBAAA|76427|21|13|47|PM|second|evening|| +76428|AAAAAAAANIKCBAAA|76428|21|13|48|PM|second|evening|| +76429|AAAAAAAAOIKCBAAA|76429|21|13|49|PM|second|evening|| +76430|AAAAAAAAPIKCBAAA|76430|21|13|50|PM|second|evening|| +76431|AAAAAAAAAJKCBAAA|76431|21|13|51|PM|second|evening|| +76432|AAAAAAAABJKCBAAA|76432|21|13|52|PM|second|evening|| +76433|AAAAAAAACJKCBAAA|76433|21|13|53|PM|second|evening|| +76434|AAAAAAAADJKCBAAA|76434|21|13|54|PM|second|evening|| +76435|AAAAAAAAEJKCBAAA|76435|21|13|55|PM|second|evening|| +76436|AAAAAAAAFJKCBAAA|76436|21|13|56|PM|second|evening|| +76437|AAAAAAAAGJKCBAAA|76437|21|13|57|PM|second|evening|| +76438|AAAAAAAAHJKCBAAA|76438|21|13|58|PM|second|evening|| +76439|AAAAAAAAIJKCBAAA|76439|21|13|59|PM|second|evening|| +76440|AAAAAAAAJJKCBAAA|76440|21|14|0|PM|second|evening|| +76441|AAAAAAAAKJKCBAAA|76441|21|14|1|PM|second|evening|| +76442|AAAAAAAALJKCBAAA|76442|21|14|2|PM|second|evening|| +76443|AAAAAAAAMJKCBAAA|76443|21|14|3|PM|second|evening|| +76444|AAAAAAAANJKCBAAA|76444|21|14|4|PM|second|evening|| +76445|AAAAAAAAOJKCBAAA|76445|21|14|5|PM|second|evening|| +76446|AAAAAAAAPJKCBAAA|76446|21|14|6|PM|second|evening|| +76447|AAAAAAAAAKKCBAAA|76447|21|14|7|PM|second|evening|| +76448|AAAAAAAABKKCBAAA|76448|21|14|8|PM|second|evening|| +76449|AAAAAAAACKKCBAAA|76449|21|14|9|PM|second|evening|| +76450|AAAAAAAADKKCBAAA|76450|21|14|10|PM|second|evening|| +76451|AAAAAAAAEKKCBAAA|76451|21|14|11|PM|second|evening|| +76452|AAAAAAAAFKKCBAAA|76452|21|14|12|PM|second|evening|| +76453|AAAAAAAAGKKCBAAA|76453|21|14|13|PM|second|evening|| +76454|AAAAAAAAHKKCBAAA|76454|21|14|14|PM|second|evening|| +76455|AAAAAAAAIKKCBAAA|76455|21|14|15|PM|second|evening|| +76456|AAAAAAAAJKKCBAAA|76456|21|14|16|PM|second|evening|| +76457|AAAAAAAAKKKCBAAA|76457|21|14|17|PM|second|evening|| +76458|AAAAAAAALKKCBAAA|76458|21|14|18|PM|second|evening|| +76459|AAAAAAAAMKKCBAAA|76459|21|14|19|PM|second|evening|| +76460|AAAAAAAANKKCBAAA|76460|21|14|20|PM|second|evening|| +76461|AAAAAAAAOKKCBAAA|76461|21|14|21|PM|second|evening|| +76462|AAAAAAAAPKKCBAAA|76462|21|14|22|PM|second|evening|| +76463|AAAAAAAAALKCBAAA|76463|21|14|23|PM|second|evening|| +76464|AAAAAAAABLKCBAAA|76464|21|14|24|PM|second|evening|| +76465|AAAAAAAACLKCBAAA|76465|21|14|25|PM|second|evening|| +76466|AAAAAAAADLKCBAAA|76466|21|14|26|PM|second|evening|| +76467|AAAAAAAAELKCBAAA|76467|21|14|27|PM|second|evening|| +76468|AAAAAAAAFLKCBAAA|76468|21|14|28|PM|second|evening|| +76469|AAAAAAAAGLKCBAAA|76469|21|14|29|PM|second|evening|| +76470|AAAAAAAAHLKCBAAA|76470|21|14|30|PM|second|evening|| +76471|AAAAAAAAILKCBAAA|76471|21|14|31|PM|second|evening|| +76472|AAAAAAAAJLKCBAAA|76472|21|14|32|PM|second|evening|| +76473|AAAAAAAAKLKCBAAA|76473|21|14|33|PM|second|evening|| +76474|AAAAAAAALLKCBAAA|76474|21|14|34|PM|second|evening|| +76475|AAAAAAAAMLKCBAAA|76475|21|14|35|PM|second|evening|| +76476|AAAAAAAANLKCBAAA|76476|21|14|36|PM|second|evening|| +76477|AAAAAAAAOLKCBAAA|76477|21|14|37|PM|second|evening|| +76478|AAAAAAAAPLKCBAAA|76478|21|14|38|PM|second|evening|| +76479|AAAAAAAAAMKCBAAA|76479|21|14|39|PM|second|evening|| +76480|AAAAAAAABMKCBAAA|76480|21|14|40|PM|second|evening|| +76481|AAAAAAAACMKCBAAA|76481|21|14|41|PM|second|evening|| +76482|AAAAAAAADMKCBAAA|76482|21|14|42|PM|second|evening|| +76483|AAAAAAAAEMKCBAAA|76483|21|14|43|PM|second|evening|| +76484|AAAAAAAAFMKCBAAA|76484|21|14|44|PM|second|evening|| +76485|AAAAAAAAGMKCBAAA|76485|21|14|45|PM|second|evening|| +76486|AAAAAAAAHMKCBAAA|76486|21|14|46|PM|second|evening|| +76487|AAAAAAAAIMKCBAAA|76487|21|14|47|PM|second|evening|| +76488|AAAAAAAAJMKCBAAA|76488|21|14|48|PM|second|evening|| +76489|AAAAAAAAKMKCBAAA|76489|21|14|49|PM|second|evening|| +76490|AAAAAAAALMKCBAAA|76490|21|14|50|PM|second|evening|| +76491|AAAAAAAAMMKCBAAA|76491|21|14|51|PM|second|evening|| +76492|AAAAAAAANMKCBAAA|76492|21|14|52|PM|second|evening|| +76493|AAAAAAAAOMKCBAAA|76493|21|14|53|PM|second|evening|| +76494|AAAAAAAAPMKCBAAA|76494|21|14|54|PM|second|evening|| +76495|AAAAAAAAANKCBAAA|76495|21|14|55|PM|second|evening|| +76496|AAAAAAAABNKCBAAA|76496|21|14|56|PM|second|evening|| +76497|AAAAAAAACNKCBAAA|76497|21|14|57|PM|second|evening|| +76498|AAAAAAAADNKCBAAA|76498|21|14|58|PM|second|evening|| +76499|AAAAAAAAENKCBAAA|76499|21|14|59|PM|second|evening|| +76500|AAAAAAAAFNKCBAAA|76500|21|15|0|PM|second|evening|| +76501|AAAAAAAAGNKCBAAA|76501|21|15|1|PM|second|evening|| +76502|AAAAAAAAHNKCBAAA|76502|21|15|2|PM|second|evening|| +76503|AAAAAAAAINKCBAAA|76503|21|15|3|PM|second|evening|| +76504|AAAAAAAAJNKCBAAA|76504|21|15|4|PM|second|evening|| +76505|AAAAAAAAKNKCBAAA|76505|21|15|5|PM|second|evening|| +76506|AAAAAAAALNKCBAAA|76506|21|15|6|PM|second|evening|| +76507|AAAAAAAAMNKCBAAA|76507|21|15|7|PM|second|evening|| +76508|AAAAAAAANNKCBAAA|76508|21|15|8|PM|second|evening|| +76509|AAAAAAAAONKCBAAA|76509|21|15|9|PM|second|evening|| +76510|AAAAAAAAPNKCBAAA|76510|21|15|10|PM|second|evening|| +76511|AAAAAAAAAOKCBAAA|76511|21|15|11|PM|second|evening|| +76512|AAAAAAAABOKCBAAA|76512|21|15|12|PM|second|evening|| +76513|AAAAAAAACOKCBAAA|76513|21|15|13|PM|second|evening|| +76514|AAAAAAAADOKCBAAA|76514|21|15|14|PM|second|evening|| +76515|AAAAAAAAEOKCBAAA|76515|21|15|15|PM|second|evening|| +76516|AAAAAAAAFOKCBAAA|76516|21|15|16|PM|second|evening|| +76517|AAAAAAAAGOKCBAAA|76517|21|15|17|PM|second|evening|| +76518|AAAAAAAAHOKCBAAA|76518|21|15|18|PM|second|evening|| +76519|AAAAAAAAIOKCBAAA|76519|21|15|19|PM|second|evening|| +76520|AAAAAAAAJOKCBAAA|76520|21|15|20|PM|second|evening|| +76521|AAAAAAAAKOKCBAAA|76521|21|15|21|PM|second|evening|| +76522|AAAAAAAALOKCBAAA|76522|21|15|22|PM|second|evening|| +76523|AAAAAAAAMOKCBAAA|76523|21|15|23|PM|second|evening|| +76524|AAAAAAAANOKCBAAA|76524|21|15|24|PM|second|evening|| +76525|AAAAAAAAOOKCBAAA|76525|21|15|25|PM|second|evening|| +76526|AAAAAAAAPOKCBAAA|76526|21|15|26|PM|second|evening|| +76527|AAAAAAAAAPKCBAAA|76527|21|15|27|PM|second|evening|| +76528|AAAAAAAABPKCBAAA|76528|21|15|28|PM|second|evening|| +76529|AAAAAAAACPKCBAAA|76529|21|15|29|PM|second|evening|| +76530|AAAAAAAADPKCBAAA|76530|21|15|30|PM|second|evening|| +76531|AAAAAAAAEPKCBAAA|76531|21|15|31|PM|second|evening|| +76532|AAAAAAAAFPKCBAAA|76532|21|15|32|PM|second|evening|| +76533|AAAAAAAAGPKCBAAA|76533|21|15|33|PM|second|evening|| +76534|AAAAAAAAHPKCBAAA|76534|21|15|34|PM|second|evening|| +76535|AAAAAAAAIPKCBAAA|76535|21|15|35|PM|second|evening|| +76536|AAAAAAAAJPKCBAAA|76536|21|15|36|PM|second|evening|| +76537|AAAAAAAAKPKCBAAA|76537|21|15|37|PM|second|evening|| +76538|AAAAAAAALPKCBAAA|76538|21|15|38|PM|second|evening|| +76539|AAAAAAAAMPKCBAAA|76539|21|15|39|PM|second|evening|| +76540|AAAAAAAANPKCBAAA|76540|21|15|40|PM|second|evening|| +76541|AAAAAAAAOPKCBAAA|76541|21|15|41|PM|second|evening|| +76542|AAAAAAAAPPKCBAAA|76542|21|15|42|PM|second|evening|| +76543|AAAAAAAAAALCBAAA|76543|21|15|43|PM|second|evening|| +76544|AAAAAAAABALCBAAA|76544|21|15|44|PM|second|evening|| +76545|AAAAAAAACALCBAAA|76545|21|15|45|PM|second|evening|| +76546|AAAAAAAADALCBAAA|76546|21|15|46|PM|second|evening|| +76547|AAAAAAAAEALCBAAA|76547|21|15|47|PM|second|evening|| +76548|AAAAAAAAFALCBAAA|76548|21|15|48|PM|second|evening|| +76549|AAAAAAAAGALCBAAA|76549|21|15|49|PM|second|evening|| +76550|AAAAAAAAHALCBAAA|76550|21|15|50|PM|second|evening|| +76551|AAAAAAAAIALCBAAA|76551|21|15|51|PM|second|evening|| +76552|AAAAAAAAJALCBAAA|76552|21|15|52|PM|second|evening|| +76553|AAAAAAAAKALCBAAA|76553|21|15|53|PM|second|evening|| +76554|AAAAAAAALALCBAAA|76554|21|15|54|PM|second|evening|| +76555|AAAAAAAAMALCBAAA|76555|21|15|55|PM|second|evening|| +76556|AAAAAAAANALCBAAA|76556|21|15|56|PM|second|evening|| +76557|AAAAAAAAOALCBAAA|76557|21|15|57|PM|second|evening|| +76558|AAAAAAAAPALCBAAA|76558|21|15|58|PM|second|evening|| +76559|AAAAAAAAABLCBAAA|76559|21|15|59|PM|second|evening|| +76560|AAAAAAAABBLCBAAA|76560|21|16|0|PM|second|evening|| +76561|AAAAAAAACBLCBAAA|76561|21|16|1|PM|second|evening|| +76562|AAAAAAAADBLCBAAA|76562|21|16|2|PM|second|evening|| +76563|AAAAAAAAEBLCBAAA|76563|21|16|3|PM|second|evening|| +76564|AAAAAAAAFBLCBAAA|76564|21|16|4|PM|second|evening|| +76565|AAAAAAAAGBLCBAAA|76565|21|16|5|PM|second|evening|| +76566|AAAAAAAAHBLCBAAA|76566|21|16|6|PM|second|evening|| +76567|AAAAAAAAIBLCBAAA|76567|21|16|7|PM|second|evening|| +76568|AAAAAAAAJBLCBAAA|76568|21|16|8|PM|second|evening|| +76569|AAAAAAAAKBLCBAAA|76569|21|16|9|PM|second|evening|| +76570|AAAAAAAALBLCBAAA|76570|21|16|10|PM|second|evening|| +76571|AAAAAAAAMBLCBAAA|76571|21|16|11|PM|second|evening|| +76572|AAAAAAAANBLCBAAA|76572|21|16|12|PM|second|evening|| +76573|AAAAAAAAOBLCBAAA|76573|21|16|13|PM|second|evening|| +76574|AAAAAAAAPBLCBAAA|76574|21|16|14|PM|second|evening|| +76575|AAAAAAAAACLCBAAA|76575|21|16|15|PM|second|evening|| +76576|AAAAAAAABCLCBAAA|76576|21|16|16|PM|second|evening|| +76577|AAAAAAAACCLCBAAA|76577|21|16|17|PM|second|evening|| +76578|AAAAAAAADCLCBAAA|76578|21|16|18|PM|second|evening|| +76579|AAAAAAAAECLCBAAA|76579|21|16|19|PM|second|evening|| +76580|AAAAAAAAFCLCBAAA|76580|21|16|20|PM|second|evening|| +76581|AAAAAAAAGCLCBAAA|76581|21|16|21|PM|second|evening|| +76582|AAAAAAAAHCLCBAAA|76582|21|16|22|PM|second|evening|| +76583|AAAAAAAAICLCBAAA|76583|21|16|23|PM|second|evening|| +76584|AAAAAAAAJCLCBAAA|76584|21|16|24|PM|second|evening|| +76585|AAAAAAAAKCLCBAAA|76585|21|16|25|PM|second|evening|| +76586|AAAAAAAALCLCBAAA|76586|21|16|26|PM|second|evening|| +76587|AAAAAAAAMCLCBAAA|76587|21|16|27|PM|second|evening|| +76588|AAAAAAAANCLCBAAA|76588|21|16|28|PM|second|evening|| +76589|AAAAAAAAOCLCBAAA|76589|21|16|29|PM|second|evening|| +76590|AAAAAAAAPCLCBAAA|76590|21|16|30|PM|second|evening|| +76591|AAAAAAAAADLCBAAA|76591|21|16|31|PM|second|evening|| +76592|AAAAAAAABDLCBAAA|76592|21|16|32|PM|second|evening|| +76593|AAAAAAAACDLCBAAA|76593|21|16|33|PM|second|evening|| +76594|AAAAAAAADDLCBAAA|76594|21|16|34|PM|second|evening|| +76595|AAAAAAAAEDLCBAAA|76595|21|16|35|PM|second|evening|| +76596|AAAAAAAAFDLCBAAA|76596|21|16|36|PM|second|evening|| +76597|AAAAAAAAGDLCBAAA|76597|21|16|37|PM|second|evening|| +76598|AAAAAAAAHDLCBAAA|76598|21|16|38|PM|second|evening|| +76599|AAAAAAAAIDLCBAAA|76599|21|16|39|PM|second|evening|| +76600|AAAAAAAAJDLCBAAA|76600|21|16|40|PM|second|evening|| +76601|AAAAAAAAKDLCBAAA|76601|21|16|41|PM|second|evening|| +76602|AAAAAAAALDLCBAAA|76602|21|16|42|PM|second|evening|| +76603|AAAAAAAAMDLCBAAA|76603|21|16|43|PM|second|evening|| +76604|AAAAAAAANDLCBAAA|76604|21|16|44|PM|second|evening|| +76605|AAAAAAAAODLCBAAA|76605|21|16|45|PM|second|evening|| +76606|AAAAAAAAPDLCBAAA|76606|21|16|46|PM|second|evening|| +76607|AAAAAAAAAELCBAAA|76607|21|16|47|PM|second|evening|| +76608|AAAAAAAABELCBAAA|76608|21|16|48|PM|second|evening|| +76609|AAAAAAAACELCBAAA|76609|21|16|49|PM|second|evening|| +76610|AAAAAAAADELCBAAA|76610|21|16|50|PM|second|evening|| +76611|AAAAAAAAEELCBAAA|76611|21|16|51|PM|second|evening|| +76612|AAAAAAAAFELCBAAA|76612|21|16|52|PM|second|evening|| +76613|AAAAAAAAGELCBAAA|76613|21|16|53|PM|second|evening|| +76614|AAAAAAAAHELCBAAA|76614|21|16|54|PM|second|evening|| +76615|AAAAAAAAIELCBAAA|76615|21|16|55|PM|second|evening|| +76616|AAAAAAAAJELCBAAA|76616|21|16|56|PM|second|evening|| +76617|AAAAAAAAKELCBAAA|76617|21|16|57|PM|second|evening|| +76618|AAAAAAAALELCBAAA|76618|21|16|58|PM|second|evening|| +76619|AAAAAAAAMELCBAAA|76619|21|16|59|PM|second|evening|| +76620|AAAAAAAANELCBAAA|76620|21|17|0|PM|second|evening|| +76621|AAAAAAAAOELCBAAA|76621|21|17|1|PM|second|evening|| +76622|AAAAAAAAPELCBAAA|76622|21|17|2|PM|second|evening|| +76623|AAAAAAAAAFLCBAAA|76623|21|17|3|PM|second|evening|| +76624|AAAAAAAABFLCBAAA|76624|21|17|4|PM|second|evening|| +76625|AAAAAAAACFLCBAAA|76625|21|17|5|PM|second|evening|| +76626|AAAAAAAADFLCBAAA|76626|21|17|6|PM|second|evening|| +76627|AAAAAAAAEFLCBAAA|76627|21|17|7|PM|second|evening|| +76628|AAAAAAAAFFLCBAAA|76628|21|17|8|PM|second|evening|| +76629|AAAAAAAAGFLCBAAA|76629|21|17|9|PM|second|evening|| +76630|AAAAAAAAHFLCBAAA|76630|21|17|10|PM|second|evening|| +76631|AAAAAAAAIFLCBAAA|76631|21|17|11|PM|second|evening|| +76632|AAAAAAAAJFLCBAAA|76632|21|17|12|PM|second|evening|| +76633|AAAAAAAAKFLCBAAA|76633|21|17|13|PM|second|evening|| +76634|AAAAAAAALFLCBAAA|76634|21|17|14|PM|second|evening|| +76635|AAAAAAAAMFLCBAAA|76635|21|17|15|PM|second|evening|| +76636|AAAAAAAANFLCBAAA|76636|21|17|16|PM|second|evening|| +76637|AAAAAAAAOFLCBAAA|76637|21|17|17|PM|second|evening|| +76638|AAAAAAAAPFLCBAAA|76638|21|17|18|PM|second|evening|| +76639|AAAAAAAAAGLCBAAA|76639|21|17|19|PM|second|evening|| +76640|AAAAAAAABGLCBAAA|76640|21|17|20|PM|second|evening|| +76641|AAAAAAAACGLCBAAA|76641|21|17|21|PM|second|evening|| +76642|AAAAAAAADGLCBAAA|76642|21|17|22|PM|second|evening|| +76643|AAAAAAAAEGLCBAAA|76643|21|17|23|PM|second|evening|| +76644|AAAAAAAAFGLCBAAA|76644|21|17|24|PM|second|evening|| +76645|AAAAAAAAGGLCBAAA|76645|21|17|25|PM|second|evening|| +76646|AAAAAAAAHGLCBAAA|76646|21|17|26|PM|second|evening|| +76647|AAAAAAAAIGLCBAAA|76647|21|17|27|PM|second|evening|| +76648|AAAAAAAAJGLCBAAA|76648|21|17|28|PM|second|evening|| +76649|AAAAAAAAKGLCBAAA|76649|21|17|29|PM|second|evening|| +76650|AAAAAAAALGLCBAAA|76650|21|17|30|PM|second|evening|| +76651|AAAAAAAAMGLCBAAA|76651|21|17|31|PM|second|evening|| +76652|AAAAAAAANGLCBAAA|76652|21|17|32|PM|second|evening|| +76653|AAAAAAAAOGLCBAAA|76653|21|17|33|PM|second|evening|| +76654|AAAAAAAAPGLCBAAA|76654|21|17|34|PM|second|evening|| +76655|AAAAAAAAAHLCBAAA|76655|21|17|35|PM|second|evening|| +76656|AAAAAAAABHLCBAAA|76656|21|17|36|PM|second|evening|| +76657|AAAAAAAACHLCBAAA|76657|21|17|37|PM|second|evening|| +76658|AAAAAAAADHLCBAAA|76658|21|17|38|PM|second|evening|| +76659|AAAAAAAAEHLCBAAA|76659|21|17|39|PM|second|evening|| +76660|AAAAAAAAFHLCBAAA|76660|21|17|40|PM|second|evening|| +76661|AAAAAAAAGHLCBAAA|76661|21|17|41|PM|second|evening|| +76662|AAAAAAAAHHLCBAAA|76662|21|17|42|PM|second|evening|| +76663|AAAAAAAAIHLCBAAA|76663|21|17|43|PM|second|evening|| +76664|AAAAAAAAJHLCBAAA|76664|21|17|44|PM|second|evening|| +76665|AAAAAAAAKHLCBAAA|76665|21|17|45|PM|second|evening|| +76666|AAAAAAAALHLCBAAA|76666|21|17|46|PM|second|evening|| +76667|AAAAAAAAMHLCBAAA|76667|21|17|47|PM|second|evening|| +76668|AAAAAAAANHLCBAAA|76668|21|17|48|PM|second|evening|| +76669|AAAAAAAAOHLCBAAA|76669|21|17|49|PM|second|evening|| +76670|AAAAAAAAPHLCBAAA|76670|21|17|50|PM|second|evening|| +76671|AAAAAAAAAILCBAAA|76671|21|17|51|PM|second|evening|| +76672|AAAAAAAABILCBAAA|76672|21|17|52|PM|second|evening|| +76673|AAAAAAAACILCBAAA|76673|21|17|53|PM|second|evening|| +76674|AAAAAAAADILCBAAA|76674|21|17|54|PM|second|evening|| +76675|AAAAAAAAEILCBAAA|76675|21|17|55|PM|second|evening|| +76676|AAAAAAAAFILCBAAA|76676|21|17|56|PM|second|evening|| +76677|AAAAAAAAGILCBAAA|76677|21|17|57|PM|second|evening|| +76678|AAAAAAAAHILCBAAA|76678|21|17|58|PM|second|evening|| +76679|AAAAAAAAIILCBAAA|76679|21|17|59|PM|second|evening|| +76680|AAAAAAAAJILCBAAA|76680|21|18|0|PM|second|evening|| +76681|AAAAAAAAKILCBAAA|76681|21|18|1|PM|second|evening|| +76682|AAAAAAAALILCBAAA|76682|21|18|2|PM|second|evening|| +76683|AAAAAAAAMILCBAAA|76683|21|18|3|PM|second|evening|| +76684|AAAAAAAANILCBAAA|76684|21|18|4|PM|second|evening|| +76685|AAAAAAAAOILCBAAA|76685|21|18|5|PM|second|evening|| +76686|AAAAAAAAPILCBAAA|76686|21|18|6|PM|second|evening|| +76687|AAAAAAAAAJLCBAAA|76687|21|18|7|PM|second|evening|| +76688|AAAAAAAABJLCBAAA|76688|21|18|8|PM|second|evening|| +76689|AAAAAAAACJLCBAAA|76689|21|18|9|PM|second|evening|| +76690|AAAAAAAADJLCBAAA|76690|21|18|10|PM|second|evening|| +76691|AAAAAAAAEJLCBAAA|76691|21|18|11|PM|second|evening|| +76692|AAAAAAAAFJLCBAAA|76692|21|18|12|PM|second|evening|| +76693|AAAAAAAAGJLCBAAA|76693|21|18|13|PM|second|evening|| +76694|AAAAAAAAHJLCBAAA|76694|21|18|14|PM|second|evening|| +76695|AAAAAAAAIJLCBAAA|76695|21|18|15|PM|second|evening|| +76696|AAAAAAAAJJLCBAAA|76696|21|18|16|PM|second|evening|| +76697|AAAAAAAAKJLCBAAA|76697|21|18|17|PM|second|evening|| +76698|AAAAAAAALJLCBAAA|76698|21|18|18|PM|second|evening|| +76699|AAAAAAAAMJLCBAAA|76699|21|18|19|PM|second|evening|| +76700|AAAAAAAANJLCBAAA|76700|21|18|20|PM|second|evening|| +76701|AAAAAAAAOJLCBAAA|76701|21|18|21|PM|second|evening|| +76702|AAAAAAAAPJLCBAAA|76702|21|18|22|PM|second|evening|| +76703|AAAAAAAAAKLCBAAA|76703|21|18|23|PM|second|evening|| +76704|AAAAAAAABKLCBAAA|76704|21|18|24|PM|second|evening|| +76705|AAAAAAAACKLCBAAA|76705|21|18|25|PM|second|evening|| +76706|AAAAAAAADKLCBAAA|76706|21|18|26|PM|second|evening|| +76707|AAAAAAAAEKLCBAAA|76707|21|18|27|PM|second|evening|| +76708|AAAAAAAAFKLCBAAA|76708|21|18|28|PM|second|evening|| +76709|AAAAAAAAGKLCBAAA|76709|21|18|29|PM|second|evening|| +76710|AAAAAAAAHKLCBAAA|76710|21|18|30|PM|second|evening|| +76711|AAAAAAAAIKLCBAAA|76711|21|18|31|PM|second|evening|| +76712|AAAAAAAAJKLCBAAA|76712|21|18|32|PM|second|evening|| +76713|AAAAAAAAKKLCBAAA|76713|21|18|33|PM|second|evening|| +76714|AAAAAAAALKLCBAAA|76714|21|18|34|PM|second|evening|| +76715|AAAAAAAAMKLCBAAA|76715|21|18|35|PM|second|evening|| +76716|AAAAAAAANKLCBAAA|76716|21|18|36|PM|second|evening|| +76717|AAAAAAAAOKLCBAAA|76717|21|18|37|PM|second|evening|| +76718|AAAAAAAAPKLCBAAA|76718|21|18|38|PM|second|evening|| +76719|AAAAAAAAALLCBAAA|76719|21|18|39|PM|second|evening|| +76720|AAAAAAAABLLCBAAA|76720|21|18|40|PM|second|evening|| +76721|AAAAAAAACLLCBAAA|76721|21|18|41|PM|second|evening|| +76722|AAAAAAAADLLCBAAA|76722|21|18|42|PM|second|evening|| +76723|AAAAAAAAELLCBAAA|76723|21|18|43|PM|second|evening|| +76724|AAAAAAAAFLLCBAAA|76724|21|18|44|PM|second|evening|| +76725|AAAAAAAAGLLCBAAA|76725|21|18|45|PM|second|evening|| +76726|AAAAAAAAHLLCBAAA|76726|21|18|46|PM|second|evening|| +76727|AAAAAAAAILLCBAAA|76727|21|18|47|PM|second|evening|| +76728|AAAAAAAAJLLCBAAA|76728|21|18|48|PM|second|evening|| +76729|AAAAAAAAKLLCBAAA|76729|21|18|49|PM|second|evening|| +76730|AAAAAAAALLLCBAAA|76730|21|18|50|PM|second|evening|| +76731|AAAAAAAAMLLCBAAA|76731|21|18|51|PM|second|evening|| +76732|AAAAAAAANLLCBAAA|76732|21|18|52|PM|second|evening|| +76733|AAAAAAAAOLLCBAAA|76733|21|18|53|PM|second|evening|| +76734|AAAAAAAAPLLCBAAA|76734|21|18|54|PM|second|evening|| +76735|AAAAAAAAAMLCBAAA|76735|21|18|55|PM|second|evening|| +76736|AAAAAAAABMLCBAAA|76736|21|18|56|PM|second|evening|| +76737|AAAAAAAACMLCBAAA|76737|21|18|57|PM|second|evening|| +76738|AAAAAAAADMLCBAAA|76738|21|18|58|PM|second|evening|| +76739|AAAAAAAAEMLCBAAA|76739|21|18|59|PM|second|evening|| +76740|AAAAAAAAFMLCBAAA|76740|21|19|0|PM|second|evening|| +76741|AAAAAAAAGMLCBAAA|76741|21|19|1|PM|second|evening|| +76742|AAAAAAAAHMLCBAAA|76742|21|19|2|PM|second|evening|| +76743|AAAAAAAAIMLCBAAA|76743|21|19|3|PM|second|evening|| +76744|AAAAAAAAJMLCBAAA|76744|21|19|4|PM|second|evening|| +76745|AAAAAAAAKMLCBAAA|76745|21|19|5|PM|second|evening|| +76746|AAAAAAAALMLCBAAA|76746|21|19|6|PM|second|evening|| +76747|AAAAAAAAMMLCBAAA|76747|21|19|7|PM|second|evening|| +76748|AAAAAAAANMLCBAAA|76748|21|19|8|PM|second|evening|| +76749|AAAAAAAAOMLCBAAA|76749|21|19|9|PM|second|evening|| +76750|AAAAAAAAPMLCBAAA|76750|21|19|10|PM|second|evening|| +76751|AAAAAAAAANLCBAAA|76751|21|19|11|PM|second|evening|| +76752|AAAAAAAABNLCBAAA|76752|21|19|12|PM|second|evening|| +76753|AAAAAAAACNLCBAAA|76753|21|19|13|PM|second|evening|| +76754|AAAAAAAADNLCBAAA|76754|21|19|14|PM|second|evening|| +76755|AAAAAAAAENLCBAAA|76755|21|19|15|PM|second|evening|| +76756|AAAAAAAAFNLCBAAA|76756|21|19|16|PM|second|evening|| +76757|AAAAAAAAGNLCBAAA|76757|21|19|17|PM|second|evening|| +76758|AAAAAAAAHNLCBAAA|76758|21|19|18|PM|second|evening|| +76759|AAAAAAAAINLCBAAA|76759|21|19|19|PM|second|evening|| +76760|AAAAAAAAJNLCBAAA|76760|21|19|20|PM|second|evening|| +76761|AAAAAAAAKNLCBAAA|76761|21|19|21|PM|second|evening|| +76762|AAAAAAAALNLCBAAA|76762|21|19|22|PM|second|evening|| +76763|AAAAAAAAMNLCBAAA|76763|21|19|23|PM|second|evening|| +76764|AAAAAAAANNLCBAAA|76764|21|19|24|PM|second|evening|| +76765|AAAAAAAAONLCBAAA|76765|21|19|25|PM|second|evening|| +76766|AAAAAAAAPNLCBAAA|76766|21|19|26|PM|second|evening|| +76767|AAAAAAAAAOLCBAAA|76767|21|19|27|PM|second|evening|| +76768|AAAAAAAABOLCBAAA|76768|21|19|28|PM|second|evening|| +76769|AAAAAAAACOLCBAAA|76769|21|19|29|PM|second|evening|| +76770|AAAAAAAADOLCBAAA|76770|21|19|30|PM|second|evening|| +76771|AAAAAAAAEOLCBAAA|76771|21|19|31|PM|second|evening|| +76772|AAAAAAAAFOLCBAAA|76772|21|19|32|PM|second|evening|| +76773|AAAAAAAAGOLCBAAA|76773|21|19|33|PM|second|evening|| +76774|AAAAAAAAHOLCBAAA|76774|21|19|34|PM|second|evening|| +76775|AAAAAAAAIOLCBAAA|76775|21|19|35|PM|second|evening|| +76776|AAAAAAAAJOLCBAAA|76776|21|19|36|PM|second|evening|| +76777|AAAAAAAAKOLCBAAA|76777|21|19|37|PM|second|evening|| +76778|AAAAAAAALOLCBAAA|76778|21|19|38|PM|second|evening|| +76779|AAAAAAAAMOLCBAAA|76779|21|19|39|PM|second|evening|| +76780|AAAAAAAANOLCBAAA|76780|21|19|40|PM|second|evening|| +76781|AAAAAAAAOOLCBAAA|76781|21|19|41|PM|second|evening|| +76782|AAAAAAAAPOLCBAAA|76782|21|19|42|PM|second|evening|| +76783|AAAAAAAAAPLCBAAA|76783|21|19|43|PM|second|evening|| +76784|AAAAAAAABPLCBAAA|76784|21|19|44|PM|second|evening|| +76785|AAAAAAAACPLCBAAA|76785|21|19|45|PM|second|evening|| +76786|AAAAAAAADPLCBAAA|76786|21|19|46|PM|second|evening|| +76787|AAAAAAAAEPLCBAAA|76787|21|19|47|PM|second|evening|| +76788|AAAAAAAAFPLCBAAA|76788|21|19|48|PM|second|evening|| +76789|AAAAAAAAGPLCBAAA|76789|21|19|49|PM|second|evening|| +76790|AAAAAAAAHPLCBAAA|76790|21|19|50|PM|second|evening|| +76791|AAAAAAAAIPLCBAAA|76791|21|19|51|PM|second|evening|| +76792|AAAAAAAAJPLCBAAA|76792|21|19|52|PM|second|evening|| +76793|AAAAAAAAKPLCBAAA|76793|21|19|53|PM|second|evening|| +76794|AAAAAAAALPLCBAAA|76794|21|19|54|PM|second|evening|| +76795|AAAAAAAAMPLCBAAA|76795|21|19|55|PM|second|evening|| +76796|AAAAAAAANPLCBAAA|76796|21|19|56|PM|second|evening|| +76797|AAAAAAAAOPLCBAAA|76797|21|19|57|PM|second|evening|| +76798|AAAAAAAAPPLCBAAA|76798|21|19|58|PM|second|evening|| +76799|AAAAAAAAAAMCBAAA|76799|21|19|59|PM|second|evening|| +76800|AAAAAAAABAMCBAAA|76800|21|20|0|PM|second|evening|| +76801|AAAAAAAACAMCBAAA|76801|21|20|1|PM|second|evening|| +76802|AAAAAAAADAMCBAAA|76802|21|20|2|PM|second|evening|| +76803|AAAAAAAAEAMCBAAA|76803|21|20|3|PM|second|evening|| +76804|AAAAAAAAFAMCBAAA|76804|21|20|4|PM|second|evening|| +76805|AAAAAAAAGAMCBAAA|76805|21|20|5|PM|second|evening|| +76806|AAAAAAAAHAMCBAAA|76806|21|20|6|PM|second|evening|| +76807|AAAAAAAAIAMCBAAA|76807|21|20|7|PM|second|evening|| +76808|AAAAAAAAJAMCBAAA|76808|21|20|8|PM|second|evening|| +76809|AAAAAAAAKAMCBAAA|76809|21|20|9|PM|second|evening|| +76810|AAAAAAAALAMCBAAA|76810|21|20|10|PM|second|evening|| +76811|AAAAAAAAMAMCBAAA|76811|21|20|11|PM|second|evening|| +76812|AAAAAAAANAMCBAAA|76812|21|20|12|PM|second|evening|| +76813|AAAAAAAAOAMCBAAA|76813|21|20|13|PM|second|evening|| +76814|AAAAAAAAPAMCBAAA|76814|21|20|14|PM|second|evening|| +76815|AAAAAAAAABMCBAAA|76815|21|20|15|PM|second|evening|| +76816|AAAAAAAABBMCBAAA|76816|21|20|16|PM|second|evening|| +76817|AAAAAAAACBMCBAAA|76817|21|20|17|PM|second|evening|| +76818|AAAAAAAADBMCBAAA|76818|21|20|18|PM|second|evening|| +76819|AAAAAAAAEBMCBAAA|76819|21|20|19|PM|second|evening|| +76820|AAAAAAAAFBMCBAAA|76820|21|20|20|PM|second|evening|| +76821|AAAAAAAAGBMCBAAA|76821|21|20|21|PM|second|evening|| +76822|AAAAAAAAHBMCBAAA|76822|21|20|22|PM|second|evening|| +76823|AAAAAAAAIBMCBAAA|76823|21|20|23|PM|second|evening|| +76824|AAAAAAAAJBMCBAAA|76824|21|20|24|PM|second|evening|| +76825|AAAAAAAAKBMCBAAA|76825|21|20|25|PM|second|evening|| +76826|AAAAAAAALBMCBAAA|76826|21|20|26|PM|second|evening|| +76827|AAAAAAAAMBMCBAAA|76827|21|20|27|PM|second|evening|| +76828|AAAAAAAANBMCBAAA|76828|21|20|28|PM|second|evening|| +76829|AAAAAAAAOBMCBAAA|76829|21|20|29|PM|second|evening|| +76830|AAAAAAAAPBMCBAAA|76830|21|20|30|PM|second|evening|| +76831|AAAAAAAAACMCBAAA|76831|21|20|31|PM|second|evening|| +76832|AAAAAAAABCMCBAAA|76832|21|20|32|PM|second|evening|| +76833|AAAAAAAACCMCBAAA|76833|21|20|33|PM|second|evening|| +76834|AAAAAAAADCMCBAAA|76834|21|20|34|PM|second|evening|| +76835|AAAAAAAAECMCBAAA|76835|21|20|35|PM|second|evening|| +76836|AAAAAAAAFCMCBAAA|76836|21|20|36|PM|second|evening|| +76837|AAAAAAAAGCMCBAAA|76837|21|20|37|PM|second|evening|| +76838|AAAAAAAAHCMCBAAA|76838|21|20|38|PM|second|evening|| +76839|AAAAAAAAICMCBAAA|76839|21|20|39|PM|second|evening|| +76840|AAAAAAAAJCMCBAAA|76840|21|20|40|PM|second|evening|| +76841|AAAAAAAAKCMCBAAA|76841|21|20|41|PM|second|evening|| +76842|AAAAAAAALCMCBAAA|76842|21|20|42|PM|second|evening|| +76843|AAAAAAAAMCMCBAAA|76843|21|20|43|PM|second|evening|| +76844|AAAAAAAANCMCBAAA|76844|21|20|44|PM|second|evening|| +76845|AAAAAAAAOCMCBAAA|76845|21|20|45|PM|second|evening|| +76846|AAAAAAAAPCMCBAAA|76846|21|20|46|PM|second|evening|| +76847|AAAAAAAAADMCBAAA|76847|21|20|47|PM|second|evening|| +76848|AAAAAAAABDMCBAAA|76848|21|20|48|PM|second|evening|| +76849|AAAAAAAACDMCBAAA|76849|21|20|49|PM|second|evening|| +76850|AAAAAAAADDMCBAAA|76850|21|20|50|PM|second|evening|| +76851|AAAAAAAAEDMCBAAA|76851|21|20|51|PM|second|evening|| +76852|AAAAAAAAFDMCBAAA|76852|21|20|52|PM|second|evening|| +76853|AAAAAAAAGDMCBAAA|76853|21|20|53|PM|second|evening|| +76854|AAAAAAAAHDMCBAAA|76854|21|20|54|PM|second|evening|| +76855|AAAAAAAAIDMCBAAA|76855|21|20|55|PM|second|evening|| +76856|AAAAAAAAJDMCBAAA|76856|21|20|56|PM|second|evening|| +76857|AAAAAAAAKDMCBAAA|76857|21|20|57|PM|second|evening|| +76858|AAAAAAAALDMCBAAA|76858|21|20|58|PM|second|evening|| +76859|AAAAAAAAMDMCBAAA|76859|21|20|59|PM|second|evening|| +76860|AAAAAAAANDMCBAAA|76860|21|21|0|PM|second|evening|| +76861|AAAAAAAAODMCBAAA|76861|21|21|1|PM|second|evening|| +76862|AAAAAAAAPDMCBAAA|76862|21|21|2|PM|second|evening|| +76863|AAAAAAAAAEMCBAAA|76863|21|21|3|PM|second|evening|| +76864|AAAAAAAABEMCBAAA|76864|21|21|4|PM|second|evening|| +76865|AAAAAAAACEMCBAAA|76865|21|21|5|PM|second|evening|| +76866|AAAAAAAADEMCBAAA|76866|21|21|6|PM|second|evening|| +76867|AAAAAAAAEEMCBAAA|76867|21|21|7|PM|second|evening|| +76868|AAAAAAAAFEMCBAAA|76868|21|21|8|PM|second|evening|| +76869|AAAAAAAAGEMCBAAA|76869|21|21|9|PM|second|evening|| +76870|AAAAAAAAHEMCBAAA|76870|21|21|10|PM|second|evening|| +76871|AAAAAAAAIEMCBAAA|76871|21|21|11|PM|second|evening|| +76872|AAAAAAAAJEMCBAAA|76872|21|21|12|PM|second|evening|| +76873|AAAAAAAAKEMCBAAA|76873|21|21|13|PM|second|evening|| +76874|AAAAAAAALEMCBAAA|76874|21|21|14|PM|second|evening|| +76875|AAAAAAAAMEMCBAAA|76875|21|21|15|PM|second|evening|| +76876|AAAAAAAANEMCBAAA|76876|21|21|16|PM|second|evening|| +76877|AAAAAAAAOEMCBAAA|76877|21|21|17|PM|second|evening|| +76878|AAAAAAAAPEMCBAAA|76878|21|21|18|PM|second|evening|| +76879|AAAAAAAAAFMCBAAA|76879|21|21|19|PM|second|evening|| +76880|AAAAAAAABFMCBAAA|76880|21|21|20|PM|second|evening|| +76881|AAAAAAAACFMCBAAA|76881|21|21|21|PM|second|evening|| +76882|AAAAAAAADFMCBAAA|76882|21|21|22|PM|second|evening|| +76883|AAAAAAAAEFMCBAAA|76883|21|21|23|PM|second|evening|| +76884|AAAAAAAAFFMCBAAA|76884|21|21|24|PM|second|evening|| +76885|AAAAAAAAGFMCBAAA|76885|21|21|25|PM|second|evening|| +76886|AAAAAAAAHFMCBAAA|76886|21|21|26|PM|second|evening|| +76887|AAAAAAAAIFMCBAAA|76887|21|21|27|PM|second|evening|| +76888|AAAAAAAAJFMCBAAA|76888|21|21|28|PM|second|evening|| +76889|AAAAAAAAKFMCBAAA|76889|21|21|29|PM|second|evening|| +76890|AAAAAAAALFMCBAAA|76890|21|21|30|PM|second|evening|| +76891|AAAAAAAAMFMCBAAA|76891|21|21|31|PM|second|evening|| +76892|AAAAAAAANFMCBAAA|76892|21|21|32|PM|second|evening|| +76893|AAAAAAAAOFMCBAAA|76893|21|21|33|PM|second|evening|| +76894|AAAAAAAAPFMCBAAA|76894|21|21|34|PM|second|evening|| +76895|AAAAAAAAAGMCBAAA|76895|21|21|35|PM|second|evening|| +76896|AAAAAAAABGMCBAAA|76896|21|21|36|PM|second|evening|| +76897|AAAAAAAACGMCBAAA|76897|21|21|37|PM|second|evening|| +76898|AAAAAAAADGMCBAAA|76898|21|21|38|PM|second|evening|| +76899|AAAAAAAAEGMCBAAA|76899|21|21|39|PM|second|evening|| +76900|AAAAAAAAFGMCBAAA|76900|21|21|40|PM|second|evening|| +76901|AAAAAAAAGGMCBAAA|76901|21|21|41|PM|second|evening|| +76902|AAAAAAAAHGMCBAAA|76902|21|21|42|PM|second|evening|| +76903|AAAAAAAAIGMCBAAA|76903|21|21|43|PM|second|evening|| +76904|AAAAAAAAJGMCBAAA|76904|21|21|44|PM|second|evening|| +76905|AAAAAAAAKGMCBAAA|76905|21|21|45|PM|second|evening|| +76906|AAAAAAAALGMCBAAA|76906|21|21|46|PM|second|evening|| +76907|AAAAAAAAMGMCBAAA|76907|21|21|47|PM|second|evening|| +76908|AAAAAAAANGMCBAAA|76908|21|21|48|PM|second|evening|| +76909|AAAAAAAAOGMCBAAA|76909|21|21|49|PM|second|evening|| +76910|AAAAAAAAPGMCBAAA|76910|21|21|50|PM|second|evening|| +76911|AAAAAAAAAHMCBAAA|76911|21|21|51|PM|second|evening|| +76912|AAAAAAAABHMCBAAA|76912|21|21|52|PM|second|evening|| +76913|AAAAAAAACHMCBAAA|76913|21|21|53|PM|second|evening|| +76914|AAAAAAAADHMCBAAA|76914|21|21|54|PM|second|evening|| +76915|AAAAAAAAEHMCBAAA|76915|21|21|55|PM|second|evening|| +76916|AAAAAAAAFHMCBAAA|76916|21|21|56|PM|second|evening|| +76917|AAAAAAAAGHMCBAAA|76917|21|21|57|PM|second|evening|| +76918|AAAAAAAAHHMCBAAA|76918|21|21|58|PM|second|evening|| +76919|AAAAAAAAIHMCBAAA|76919|21|21|59|PM|second|evening|| +76920|AAAAAAAAJHMCBAAA|76920|21|22|0|PM|second|evening|| +76921|AAAAAAAAKHMCBAAA|76921|21|22|1|PM|second|evening|| +76922|AAAAAAAALHMCBAAA|76922|21|22|2|PM|second|evening|| +76923|AAAAAAAAMHMCBAAA|76923|21|22|3|PM|second|evening|| +76924|AAAAAAAANHMCBAAA|76924|21|22|4|PM|second|evening|| +76925|AAAAAAAAOHMCBAAA|76925|21|22|5|PM|second|evening|| +76926|AAAAAAAAPHMCBAAA|76926|21|22|6|PM|second|evening|| +76927|AAAAAAAAAIMCBAAA|76927|21|22|7|PM|second|evening|| +76928|AAAAAAAABIMCBAAA|76928|21|22|8|PM|second|evening|| +76929|AAAAAAAACIMCBAAA|76929|21|22|9|PM|second|evening|| +76930|AAAAAAAADIMCBAAA|76930|21|22|10|PM|second|evening|| +76931|AAAAAAAAEIMCBAAA|76931|21|22|11|PM|second|evening|| +76932|AAAAAAAAFIMCBAAA|76932|21|22|12|PM|second|evening|| +76933|AAAAAAAAGIMCBAAA|76933|21|22|13|PM|second|evening|| +76934|AAAAAAAAHIMCBAAA|76934|21|22|14|PM|second|evening|| +76935|AAAAAAAAIIMCBAAA|76935|21|22|15|PM|second|evening|| +76936|AAAAAAAAJIMCBAAA|76936|21|22|16|PM|second|evening|| +76937|AAAAAAAAKIMCBAAA|76937|21|22|17|PM|second|evening|| +76938|AAAAAAAALIMCBAAA|76938|21|22|18|PM|second|evening|| +76939|AAAAAAAAMIMCBAAA|76939|21|22|19|PM|second|evening|| +76940|AAAAAAAANIMCBAAA|76940|21|22|20|PM|second|evening|| +76941|AAAAAAAAOIMCBAAA|76941|21|22|21|PM|second|evening|| +76942|AAAAAAAAPIMCBAAA|76942|21|22|22|PM|second|evening|| +76943|AAAAAAAAAJMCBAAA|76943|21|22|23|PM|second|evening|| +76944|AAAAAAAABJMCBAAA|76944|21|22|24|PM|second|evening|| +76945|AAAAAAAACJMCBAAA|76945|21|22|25|PM|second|evening|| +76946|AAAAAAAADJMCBAAA|76946|21|22|26|PM|second|evening|| +76947|AAAAAAAAEJMCBAAA|76947|21|22|27|PM|second|evening|| +76948|AAAAAAAAFJMCBAAA|76948|21|22|28|PM|second|evening|| +76949|AAAAAAAAGJMCBAAA|76949|21|22|29|PM|second|evening|| +76950|AAAAAAAAHJMCBAAA|76950|21|22|30|PM|second|evening|| +76951|AAAAAAAAIJMCBAAA|76951|21|22|31|PM|second|evening|| +76952|AAAAAAAAJJMCBAAA|76952|21|22|32|PM|second|evening|| +76953|AAAAAAAAKJMCBAAA|76953|21|22|33|PM|second|evening|| +76954|AAAAAAAALJMCBAAA|76954|21|22|34|PM|second|evening|| +76955|AAAAAAAAMJMCBAAA|76955|21|22|35|PM|second|evening|| +76956|AAAAAAAANJMCBAAA|76956|21|22|36|PM|second|evening|| +76957|AAAAAAAAOJMCBAAA|76957|21|22|37|PM|second|evening|| +76958|AAAAAAAAPJMCBAAA|76958|21|22|38|PM|second|evening|| +76959|AAAAAAAAAKMCBAAA|76959|21|22|39|PM|second|evening|| +76960|AAAAAAAABKMCBAAA|76960|21|22|40|PM|second|evening|| +76961|AAAAAAAACKMCBAAA|76961|21|22|41|PM|second|evening|| +76962|AAAAAAAADKMCBAAA|76962|21|22|42|PM|second|evening|| +76963|AAAAAAAAEKMCBAAA|76963|21|22|43|PM|second|evening|| +76964|AAAAAAAAFKMCBAAA|76964|21|22|44|PM|second|evening|| +76965|AAAAAAAAGKMCBAAA|76965|21|22|45|PM|second|evening|| +76966|AAAAAAAAHKMCBAAA|76966|21|22|46|PM|second|evening|| +76967|AAAAAAAAIKMCBAAA|76967|21|22|47|PM|second|evening|| +76968|AAAAAAAAJKMCBAAA|76968|21|22|48|PM|second|evening|| +76969|AAAAAAAAKKMCBAAA|76969|21|22|49|PM|second|evening|| +76970|AAAAAAAALKMCBAAA|76970|21|22|50|PM|second|evening|| +76971|AAAAAAAAMKMCBAAA|76971|21|22|51|PM|second|evening|| +76972|AAAAAAAANKMCBAAA|76972|21|22|52|PM|second|evening|| +76973|AAAAAAAAOKMCBAAA|76973|21|22|53|PM|second|evening|| +76974|AAAAAAAAPKMCBAAA|76974|21|22|54|PM|second|evening|| +76975|AAAAAAAAALMCBAAA|76975|21|22|55|PM|second|evening|| +76976|AAAAAAAABLMCBAAA|76976|21|22|56|PM|second|evening|| +76977|AAAAAAAACLMCBAAA|76977|21|22|57|PM|second|evening|| +76978|AAAAAAAADLMCBAAA|76978|21|22|58|PM|second|evening|| +76979|AAAAAAAAELMCBAAA|76979|21|22|59|PM|second|evening|| +76980|AAAAAAAAFLMCBAAA|76980|21|23|0|PM|second|evening|| +76981|AAAAAAAAGLMCBAAA|76981|21|23|1|PM|second|evening|| +76982|AAAAAAAAHLMCBAAA|76982|21|23|2|PM|second|evening|| +76983|AAAAAAAAILMCBAAA|76983|21|23|3|PM|second|evening|| +76984|AAAAAAAAJLMCBAAA|76984|21|23|4|PM|second|evening|| +76985|AAAAAAAAKLMCBAAA|76985|21|23|5|PM|second|evening|| +76986|AAAAAAAALLMCBAAA|76986|21|23|6|PM|second|evening|| +76987|AAAAAAAAMLMCBAAA|76987|21|23|7|PM|second|evening|| +76988|AAAAAAAANLMCBAAA|76988|21|23|8|PM|second|evening|| +76989|AAAAAAAAOLMCBAAA|76989|21|23|9|PM|second|evening|| +76990|AAAAAAAAPLMCBAAA|76990|21|23|10|PM|second|evening|| +76991|AAAAAAAAAMMCBAAA|76991|21|23|11|PM|second|evening|| +76992|AAAAAAAABMMCBAAA|76992|21|23|12|PM|second|evening|| +76993|AAAAAAAACMMCBAAA|76993|21|23|13|PM|second|evening|| +76994|AAAAAAAADMMCBAAA|76994|21|23|14|PM|second|evening|| +76995|AAAAAAAAEMMCBAAA|76995|21|23|15|PM|second|evening|| +76996|AAAAAAAAFMMCBAAA|76996|21|23|16|PM|second|evening|| +76997|AAAAAAAAGMMCBAAA|76997|21|23|17|PM|second|evening|| +76998|AAAAAAAAHMMCBAAA|76998|21|23|18|PM|second|evening|| +76999|AAAAAAAAIMMCBAAA|76999|21|23|19|PM|second|evening|| +77000|AAAAAAAAJMMCBAAA|77000|21|23|20|PM|second|evening|| +77001|AAAAAAAAKMMCBAAA|77001|21|23|21|PM|second|evening|| +77002|AAAAAAAALMMCBAAA|77002|21|23|22|PM|second|evening|| +77003|AAAAAAAAMMMCBAAA|77003|21|23|23|PM|second|evening|| +77004|AAAAAAAANMMCBAAA|77004|21|23|24|PM|second|evening|| +77005|AAAAAAAAOMMCBAAA|77005|21|23|25|PM|second|evening|| +77006|AAAAAAAAPMMCBAAA|77006|21|23|26|PM|second|evening|| +77007|AAAAAAAAANMCBAAA|77007|21|23|27|PM|second|evening|| +77008|AAAAAAAABNMCBAAA|77008|21|23|28|PM|second|evening|| +77009|AAAAAAAACNMCBAAA|77009|21|23|29|PM|second|evening|| +77010|AAAAAAAADNMCBAAA|77010|21|23|30|PM|second|evening|| +77011|AAAAAAAAENMCBAAA|77011|21|23|31|PM|second|evening|| +77012|AAAAAAAAFNMCBAAA|77012|21|23|32|PM|second|evening|| +77013|AAAAAAAAGNMCBAAA|77013|21|23|33|PM|second|evening|| +77014|AAAAAAAAHNMCBAAA|77014|21|23|34|PM|second|evening|| +77015|AAAAAAAAINMCBAAA|77015|21|23|35|PM|second|evening|| +77016|AAAAAAAAJNMCBAAA|77016|21|23|36|PM|second|evening|| +77017|AAAAAAAAKNMCBAAA|77017|21|23|37|PM|second|evening|| +77018|AAAAAAAALNMCBAAA|77018|21|23|38|PM|second|evening|| +77019|AAAAAAAAMNMCBAAA|77019|21|23|39|PM|second|evening|| +77020|AAAAAAAANNMCBAAA|77020|21|23|40|PM|second|evening|| +77021|AAAAAAAAONMCBAAA|77021|21|23|41|PM|second|evening|| +77022|AAAAAAAAPNMCBAAA|77022|21|23|42|PM|second|evening|| +77023|AAAAAAAAAOMCBAAA|77023|21|23|43|PM|second|evening|| +77024|AAAAAAAABOMCBAAA|77024|21|23|44|PM|second|evening|| +77025|AAAAAAAACOMCBAAA|77025|21|23|45|PM|second|evening|| +77026|AAAAAAAADOMCBAAA|77026|21|23|46|PM|second|evening|| +77027|AAAAAAAAEOMCBAAA|77027|21|23|47|PM|second|evening|| +77028|AAAAAAAAFOMCBAAA|77028|21|23|48|PM|second|evening|| +77029|AAAAAAAAGOMCBAAA|77029|21|23|49|PM|second|evening|| +77030|AAAAAAAAHOMCBAAA|77030|21|23|50|PM|second|evening|| +77031|AAAAAAAAIOMCBAAA|77031|21|23|51|PM|second|evening|| +77032|AAAAAAAAJOMCBAAA|77032|21|23|52|PM|second|evening|| +77033|AAAAAAAAKOMCBAAA|77033|21|23|53|PM|second|evening|| +77034|AAAAAAAALOMCBAAA|77034|21|23|54|PM|second|evening|| +77035|AAAAAAAAMOMCBAAA|77035|21|23|55|PM|second|evening|| +77036|AAAAAAAANOMCBAAA|77036|21|23|56|PM|second|evening|| +77037|AAAAAAAAOOMCBAAA|77037|21|23|57|PM|second|evening|| +77038|AAAAAAAAPOMCBAAA|77038|21|23|58|PM|second|evening|| +77039|AAAAAAAAAPMCBAAA|77039|21|23|59|PM|second|evening|| +77040|AAAAAAAABPMCBAAA|77040|21|24|0|PM|second|evening|| +77041|AAAAAAAACPMCBAAA|77041|21|24|1|PM|second|evening|| +77042|AAAAAAAADPMCBAAA|77042|21|24|2|PM|second|evening|| +77043|AAAAAAAAEPMCBAAA|77043|21|24|3|PM|second|evening|| +77044|AAAAAAAAFPMCBAAA|77044|21|24|4|PM|second|evening|| +77045|AAAAAAAAGPMCBAAA|77045|21|24|5|PM|second|evening|| +77046|AAAAAAAAHPMCBAAA|77046|21|24|6|PM|second|evening|| +77047|AAAAAAAAIPMCBAAA|77047|21|24|7|PM|second|evening|| +77048|AAAAAAAAJPMCBAAA|77048|21|24|8|PM|second|evening|| +77049|AAAAAAAAKPMCBAAA|77049|21|24|9|PM|second|evening|| +77050|AAAAAAAALPMCBAAA|77050|21|24|10|PM|second|evening|| +77051|AAAAAAAAMPMCBAAA|77051|21|24|11|PM|second|evening|| +77052|AAAAAAAANPMCBAAA|77052|21|24|12|PM|second|evening|| +77053|AAAAAAAAOPMCBAAA|77053|21|24|13|PM|second|evening|| +77054|AAAAAAAAPPMCBAAA|77054|21|24|14|PM|second|evening|| +77055|AAAAAAAAAANCBAAA|77055|21|24|15|PM|second|evening|| +77056|AAAAAAAABANCBAAA|77056|21|24|16|PM|second|evening|| +77057|AAAAAAAACANCBAAA|77057|21|24|17|PM|second|evening|| +77058|AAAAAAAADANCBAAA|77058|21|24|18|PM|second|evening|| +77059|AAAAAAAAEANCBAAA|77059|21|24|19|PM|second|evening|| +77060|AAAAAAAAFANCBAAA|77060|21|24|20|PM|second|evening|| +77061|AAAAAAAAGANCBAAA|77061|21|24|21|PM|second|evening|| +77062|AAAAAAAAHANCBAAA|77062|21|24|22|PM|second|evening|| +77063|AAAAAAAAIANCBAAA|77063|21|24|23|PM|second|evening|| +77064|AAAAAAAAJANCBAAA|77064|21|24|24|PM|second|evening|| +77065|AAAAAAAAKANCBAAA|77065|21|24|25|PM|second|evening|| +77066|AAAAAAAALANCBAAA|77066|21|24|26|PM|second|evening|| +77067|AAAAAAAAMANCBAAA|77067|21|24|27|PM|second|evening|| +77068|AAAAAAAANANCBAAA|77068|21|24|28|PM|second|evening|| +77069|AAAAAAAAOANCBAAA|77069|21|24|29|PM|second|evening|| +77070|AAAAAAAAPANCBAAA|77070|21|24|30|PM|second|evening|| +77071|AAAAAAAAABNCBAAA|77071|21|24|31|PM|second|evening|| +77072|AAAAAAAABBNCBAAA|77072|21|24|32|PM|second|evening|| +77073|AAAAAAAACBNCBAAA|77073|21|24|33|PM|second|evening|| +77074|AAAAAAAADBNCBAAA|77074|21|24|34|PM|second|evening|| +77075|AAAAAAAAEBNCBAAA|77075|21|24|35|PM|second|evening|| +77076|AAAAAAAAFBNCBAAA|77076|21|24|36|PM|second|evening|| +77077|AAAAAAAAGBNCBAAA|77077|21|24|37|PM|second|evening|| +77078|AAAAAAAAHBNCBAAA|77078|21|24|38|PM|second|evening|| +77079|AAAAAAAAIBNCBAAA|77079|21|24|39|PM|second|evening|| +77080|AAAAAAAAJBNCBAAA|77080|21|24|40|PM|second|evening|| +77081|AAAAAAAAKBNCBAAA|77081|21|24|41|PM|second|evening|| +77082|AAAAAAAALBNCBAAA|77082|21|24|42|PM|second|evening|| +77083|AAAAAAAAMBNCBAAA|77083|21|24|43|PM|second|evening|| +77084|AAAAAAAANBNCBAAA|77084|21|24|44|PM|second|evening|| +77085|AAAAAAAAOBNCBAAA|77085|21|24|45|PM|second|evening|| +77086|AAAAAAAAPBNCBAAA|77086|21|24|46|PM|second|evening|| +77087|AAAAAAAAACNCBAAA|77087|21|24|47|PM|second|evening|| +77088|AAAAAAAABCNCBAAA|77088|21|24|48|PM|second|evening|| +77089|AAAAAAAACCNCBAAA|77089|21|24|49|PM|second|evening|| +77090|AAAAAAAADCNCBAAA|77090|21|24|50|PM|second|evening|| +77091|AAAAAAAAECNCBAAA|77091|21|24|51|PM|second|evening|| +77092|AAAAAAAAFCNCBAAA|77092|21|24|52|PM|second|evening|| +77093|AAAAAAAAGCNCBAAA|77093|21|24|53|PM|second|evening|| +77094|AAAAAAAAHCNCBAAA|77094|21|24|54|PM|second|evening|| +77095|AAAAAAAAICNCBAAA|77095|21|24|55|PM|second|evening|| +77096|AAAAAAAAJCNCBAAA|77096|21|24|56|PM|second|evening|| +77097|AAAAAAAAKCNCBAAA|77097|21|24|57|PM|second|evening|| +77098|AAAAAAAALCNCBAAA|77098|21|24|58|PM|second|evening|| +77099|AAAAAAAAMCNCBAAA|77099|21|24|59|PM|second|evening|| +77100|AAAAAAAANCNCBAAA|77100|21|25|0|PM|second|evening|| +77101|AAAAAAAAOCNCBAAA|77101|21|25|1|PM|second|evening|| +77102|AAAAAAAAPCNCBAAA|77102|21|25|2|PM|second|evening|| +77103|AAAAAAAAADNCBAAA|77103|21|25|3|PM|second|evening|| +77104|AAAAAAAABDNCBAAA|77104|21|25|4|PM|second|evening|| +77105|AAAAAAAACDNCBAAA|77105|21|25|5|PM|second|evening|| +77106|AAAAAAAADDNCBAAA|77106|21|25|6|PM|second|evening|| +77107|AAAAAAAAEDNCBAAA|77107|21|25|7|PM|second|evening|| +77108|AAAAAAAAFDNCBAAA|77108|21|25|8|PM|second|evening|| +77109|AAAAAAAAGDNCBAAA|77109|21|25|9|PM|second|evening|| +77110|AAAAAAAAHDNCBAAA|77110|21|25|10|PM|second|evening|| +77111|AAAAAAAAIDNCBAAA|77111|21|25|11|PM|second|evening|| +77112|AAAAAAAAJDNCBAAA|77112|21|25|12|PM|second|evening|| +77113|AAAAAAAAKDNCBAAA|77113|21|25|13|PM|second|evening|| +77114|AAAAAAAALDNCBAAA|77114|21|25|14|PM|second|evening|| +77115|AAAAAAAAMDNCBAAA|77115|21|25|15|PM|second|evening|| +77116|AAAAAAAANDNCBAAA|77116|21|25|16|PM|second|evening|| +77117|AAAAAAAAODNCBAAA|77117|21|25|17|PM|second|evening|| +77118|AAAAAAAAPDNCBAAA|77118|21|25|18|PM|second|evening|| +77119|AAAAAAAAAENCBAAA|77119|21|25|19|PM|second|evening|| +77120|AAAAAAAABENCBAAA|77120|21|25|20|PM|second|evening|| +77121|AAAAAAAACENCBAAA|77121|21|25|21|PM|second|evening|| +77122|AAAAAAAADENCBAAA|77122|21|25|22|PM|second|evening|| +77123|AAAAAAAAEENCBAAA|77123|21|25|23|PM|second|evening|| +77124|AAAAAAAAFENCBAAA|77124|21|25|24|PM|second|evening|| +77125|AAAAAAAAGENCBAAA|77125|21|25|25|PM|second|evening|| +77126|AAAAAAAAHENCBAAA|77126|21|25|26|PM|second|evening|| +77127|AAAAAAAAIENCBAAA|77127|21|25|27|PM|second|evening|| +77128|AAAAAAAAJENCBAAA|77128|21|25|28|PM|second|evening|| +77129|AAAAAAAAKENCBAAA|77129|21|25|29|PM|second|evening|| +77130|AAAAAAAALENCBAAA|77130|21|25|30|PM|second|evening|| +77131|AAAAAAAAMENCBAAA|77131|21|25|31|PM|second|evening|| +77132|AAAAAAAANENCBAAA|77132|21|25|32|PM|second|evening|| +77133|AAAAAAAAOENCBAAA|77133|21|25|33|PM|second|evening|| +77134|AAAAAAAAPENCBAAA|77134|21|25|34|PM|second|evening|| +77135|AAAAAAAAAFNCBAAA|77135|21|25|35|PM|second|evening|| +77136|AAAAAAAABFNCBAAA|77136|21|25|36|PM|second|evening|| +77137|AAAAAAAACFNCBAAA|77137|21|25|37|PM|second|evening|| +77138|AAAAAAAADFNCBAAA|77138|21|25|38|PM|second|evening|| +77139|AAAAAAAAEFNCBAAA|77139|21|25|39|PM|second|evening|| +77140|AAAAAAAAFFNCBAAA|77140|21|25|40|PM|second|evening|| +77141|AAAAAAAAGFNCBAAA|77141|21|25|41|PM|second|evening|| +77142|AAAAAAAAHFNCBAAA|77142|21|25|42|PM|second|evening|| +77143|AAAAAAAAIFNCBAAA|77143|21|25|43|PM|second|evening|| +77144|AAAAAAAAJFNCBAAA|77144|21|25|44|PM|second|evening|| +77145|AAAAAAAAKFNCBAAA|77145|21|25|45|PM|second|evening|| +77146|AAAAAAAALFNCBAAA|77146|21|25|46|PM|second|evening|| +77147|AAAAAAAAMFNCBAAA|77147|21|25|47|PM|second|evening|| +77148|AAAAAAAANFNCBAAA|77148|21|25|48|PM|second|evening|| +77149|AAAAAAAAOFNCBAAA|77149|21|25|49|PM|second|evening|| +77150|AAAAAAAAPFNCBAAA|77150|21|25|50|PM|second|evening|| +77151|AAAAAAAAAGNCBAAA|77151|21|25|51|PM|second|evening|| +77152|AAAAAAAABGNCBAAA|77152|21|25|52|PM|second|evening|| +77153|AAAAAAAACGNCBAAA|77153|21|25|53|PM|second|evening|| +77154|AAAAAAAADGNCBAAA|77154|21|25|54|PM|second|evening|| +77155|AAAAAAAAEGNCBAAA|77155|21|25|55|PM|second|evening|| +77156|AAAAAAAAFGNCBAAA|77156|21|25|56|PM|second|evening|| +77157|AAAAAAAAGGNCBAAA|77157|21|25|57|PM|second|evening|| +77158|AAAAAAAAHGNCBAAA|77158|21|25|58|PM|second|evening|| +77159|AAAAAAAAIGNCBAAA|77159|21|25|59|PM|second|evening|| +77160|AAAAAAAAJGNCBAAA|77160|21|26|0|PM|second|evening|| +77161|AAAAAAAAKGNCBAAA|77161|21|26|1|PM|second|evening|| +77162|AAAAAAAALGNCBAAA|77162|21|26|2|PM|second|evening|| +77163|AAAAAAAAMGNCBAAA|77163|21|26|3|PM|second|evening|| +77164|AAAAAAAANGNCBAAA|77164|21|26|4|PM|second|evening|| +77165|AAAAAAAAOGNCBAAA|77165|21|26|5|PM|second|evening|| +77166|AAAAAAAAPGNCBAAA|77166|21|26|6|PM|second|evening|| +77167|AAAAAAAAAHNCBAAA|77167|21|26|7|PM|second|evening|| +77168|AAAAAAAABHNCBAAA|77168|21|26|8|PM|second|evening|| +77169|AAAAAAAACHNCBAAA|77169|21|26|9|PM|second|evening|| +77170|AAAAAAAADHNCBAAA|77170|21|26|10|PM|second|evening|| +77171|AAAAAAAAEHNCBAAA|77171|21|26|11|PM|second|evening|| +77172|AAAAAAAAFHNCBAAA|77172|21|26|12|PM|second|evening|| +77173|AAAAAAAAGHNCBAAA|77173|21|26|13|PM|second|evening|| +77174|AAAAAAAAHHNCBAAA|77174|21|26|14|PM|second|evening|| +77175|AAAAAAAAIHNCBAAA|77175|21|26|15|PM|second|evening|| +77176|AAAAAAAAJHNCBAAA|77176|21|26|16|PM|second|evening|| +77177|AAAAAAAAKHNCBAAA|77177|21|26|17|PM|second|evening|| +77178|AAAAAAAALHNCBAAA|77178|21|26|18|PM|second|evening|| +77179|AAAAAAAAMHNCBAAA|77179|21|26|19|PM|second|evening|| +77180|AAAAAAAANHNCBAAA|77180|21|26|20|PM|second|evening|| +77181|AAAAAAAAOHNCBAAA|77181|21|26|21|PM|second|evening|| +77182|AAAAAAAAPHNCBAAA|77182|21|26|22|PM|second|evening|| +77183|AAAAAAAAAINCBAAA|77183|21|26|23|PM|second|evening|| +77184|AAAAAAAABINCBAAA|77184|21|26|24|PM|second|evening|| +77185|AAAAAAAACINCBAAA|77185|21|26|25|PM|second|evening|| +77186|AAAAAAAADINCBAAA|77186|21|26|26|PM|second|evening|| +77187|AAAAAAAAEINCBAAA|77187|21|26|27|PM|second|evening|| +77188|AAAAAAAAFINCBAAA|77188|21|26|28|PM|second|evening|| +77189|AAAAAAAAGINCBAAA|77189|21|26|29|PM|second|evening|| +77190|AAAAAAAAHINCBAAA|77190|21|26|30|PM|second|evening|| +77191|AAAAAAAAIINCBAAA|77191|21|26|31|PM|second|evening|| +77192|AAAAAAAAJINCBAAA|77192|21|26|32|PM|second|evening|| +77193|AAAAAAAAKINCBAAA|77193|21|26|33|PM|second|evening|| +77194|AAAAAAAALINCBAAA|77194|21|26|34|PM|second|evening|| +77195|AAAAAAAAMINCBAAA|77195|21|26|35|PM|second|evening|| +77196|AAAAAAAANINCBAAA|77196|21|26|36|PM|second|evening|| +77197|AAAAAAAAOINCBAAA|77197|21|26|37|PM|second|evening|| +77198|AAAAAAAAPINCBAAA|77198|21|26|38|PM|second|evening|| +77199|AAAAAAAAAJNCBAAA|77199|21|26|39|PM|second|evening|| +77200|AAAAAAAABJNCBAAA|77200|21|26|40|PM|second|evening|| +77201|AAAAAAAACJNCBAAA|77201|21|26|41|PM|second|evening|| +77202|AAAAAAAADJNCBAAA|77202|21|26|42|PM|second|evening|| +77203|AAAAAAAAEJNCBAAA|77203|21|26|43|PM|second|evening|| +77204|AAAAAAAAFJNCBAAA|77204|21|26|44|PM|second|evening|| +77205|AAAAAAAAGJNCBAAA|77205|21|26|45|PM|second|evening|| +77206|AAAAAAAAHJNCBAAA|77206|21|26|46|PM|second|evening|| +77207|AAAAAAAAIJNCBAAA|77207|21|26|47|PM|second|evening|| +77208|AAAAAAAAJJNCBAAA|77208|21|26|48|PM|second|evening|| +77209|AAAAAAAAKJNCBAAA|77209|21|26|49|PM|second|evening|| +77210|AAAAAAAALJNCBAAA|77210|21|26|50|PM|second|evening|| +77211|AAAAAAAAMJNCBAAA|77211|21|26|51|PM|second|evening|| +77212|AAAAAAAANJNCBAAA|77212|21|26|52|PM|second|evening|| +77213|AAAAAAAAOJNCBAAA|77213|21|26|53|PM|second|evening|| +77214|AAAAAAAAPJNCBAAA|77214|21|26|54|PM|second|evening|| +77215|AAAAAAAAAKNCBAAA|77215|21|26|55|PM|second|evening|| +77216|AAAAAAAABKNCBAAA|77216|21|26|56|PM|second|evening|| +77217|AAAAAAAACKNCBAAA|77217|21|26|57|PM|second|evening|| +77218|AAAAAAAADKNCBAAA|77218|21|26|58|PM|second|evening|| +77219|AAAAAAAAEKNCBAAA|77219|21|26|59|PM|second|evening|| +77220|AAAAAAAAFKNCBAAA|77220|21|27|0|PM|second|evening|| +77221|AAAAAAAAGKNCBAAA|77221|21|27|1|PM|second|evening|| +77222|AAAAAAAAHKNCBAAA|77222|21|27|2|PM|second|evening|| +77223|AAAAAAAAIKNCBAAA|77223|21|27|3|PM|second|evening|| +77224|AAAAAAAAJKNCBAAA|77224|21|27|4|PM|second|evening|| +77225|AAAAAAAAKKNCBAAA|77225|21|27|5|PM|second|evening|| +77226|AAAAAAAALKNCBAAA|77226|21|27|6|PM|second|evening|| +77227|AAAAAAAAMKNCBAAA|77227|21|27|7|PM|second|evening|| +77228|AAAAAAAANKNCBAAA|77228|21|27|8|PM|second|evening|| +77229|AAAAAAAAOKNCBAAA|77229|21|27|9|PM|second|evening|| +77230|AAAAAAAAPKNCBAAA|77230|21|27|10|PM|second|evening|| +77231|AAAAAAAAALNCBAAA|77231|21|27|11|PM|second|evening|| +77232|AAAAAAAABLNCBAAA|77232|21|27|12|PM|second|evening|| +77233|AAAAAAAACLNCBAAA|77233|21|27|13|PM|second|evening|| +77234|AAAAAAAADLNCBAAA|77234|21|27|14|PM|second|evening|| +77235|AAAAAAAAELNCBAAA|77235|21|27|15|PM|second|evening|| +77236|AAAAAAAAFLNCBAAA|77236|21|27|16|PM|second|evening|| +77237|AAAAAAAAGLNCBAAA|77237|21|27|17|PM|second|evening|| +77238|AAAAAAAAHLNCBAAA|77238|21|27|18|PM|second|evening|| +77239|AAAAAAAAILNCBAAA|77239|21|27|19|PM|second|evening|| +77240|AAAAAAAAJLNCBAAA|77240|21|27|20|PM|second|evening|| +77241|AAAAAAAAKLNCBAAA|77241|21|27|21|PM|second|evening|| +77242|AAAAAAAALLNCBAAA|77242|21|27|22|PM|second|evening|| +77243|AAAAAAAAMLNCBAAA|77243|21|27|23|PM|second|evening|| +77244|AAAAAAAANLNCBAAA|77244|21|27|24|PM|second|evening|| +77245|AAAAAAAAOLNCBAAA|77245|21|27|25|PM|second|evening|| +77246|AAAAAAAAPLNCBAAA|77246|21|27|26|PM|second|evening|| +77247|AAAAAAAAAMNCBAAA|77247|21|27|27|PM|second|evening|| +77248|AAAAAAAABMNCBAAA|77248|21|27|28|PM|second|evening|| +77249|AAAAAAAACMNCBAAA|77249|21|27|29|PM|second|evening|| +77250|AAAAAAAADMNCBAAA|77250|21|27|30|PM|second|evening|| +77251|AAAAAAAAEMNCBAAA|77251|21|27|31|PM|second|evening|| +77252|AAAAAAAAFMNCBAAA|77252|21|27|32|PM|second|evening|| +77253|AAAAAAAAGMNCBAAA|77253|21|27|33|PM|second|evening|| +77254|AAAAAAAAHMNCBAAA|77254|21|27|34|PM|second|evening|| +77255|AAAAAAAAIMNCBAAA|77255|21|27|35|PM|second|evening|| +77256|AAAAAAAAJMNCBAAA|77256|21|27|36|PM|second|evening|| +77257|AAAAAAAAKMNCBAAA|77257|21|27|37|PM|second|evening|| +77258|AAAAAAAALMNCBAAA|77258|21|27|38|PM|second|evening|| +77259|AAAAAAAAMMNCBAAA|77259|21|27|39|PM|second|evening|| +77260|AAAAAAAANMNCBAAA|77260|21|27|40|PM|second|evening|| +77261|AAAAAAAAOMNCBAAA|77261|21|27|41|PM|second|evening|| +77262|AAAAAAAAPMNCBAAA|77262|21|27|42|PM|second|evening|| +77263|AAAAAAAAANNCBAAA|77263|21|27|43|PM|second|evening|| +77264|AAAAAAAABNNCBAAA|77264|21|27|44|PM|second|evening|| +77265|AAAAAAAACNNCBAAA|77265|21|27|45|PM|second|evening|| +77266|AAAAAAAADNNCBAAA|77266|21|27|46|PM|second|evening|| +77267|AAAAAAAAENNCBAAA|77267|21|27|47|PM|second|evening|| +77268|AAAAAAAAFNNCBAAA|77268|21|27|48|PM|second|evening|| +77269|AAAAAAAAGNNCBAAA|77269|21|27|49|PM|second|evening|| +77270|AAAAAAAAHNNCBAAA|77270|21|27|50|PM|second|evening|| +77271|AAAAAAAAINNCBAAA|77271|21|27|51|PM|second|evening|| +77272|AAAAAAAAJNNCBAAA|77272|21|27|52|PM|second|evening|| +77273|AAAAAAAAKNNCBAAA|77273|21|27|53|PM|second|evening|| +77274|AAAAAAAALNNCBAAA|77274|21|27|54|PM|second|evening|| +77275|AAAAAAAAMNNCBAAA|77275|21|27|55|PM|second|evening|| +77276|AAAAAAAANNNCBAAA|77276|21|27|56|PM|second|evening|| +77277|AAAAAAAAONNCBAAA|77277|21|27|57|PM|second|evening|| +77278|AAAAAAAAPNNCBAAA|77278|21|27|58|PM|second|evening|| +77279|AAAAAAAAAONCBAAA|77279|21|27|59|PM|second|evening|| +77280|AAAAAAAABONCBAAA|77280|21|28|0|PM|second|evening|| +77281|AAAAAAAACONCBAAA|77281|21|28|1|PM|second|evening|| +77282|AAAAAAAADONCBAAA|77282|21|28|2|PM|second|evening|| +77283|AAAAAAAAEONCBAAA|77283|21|28|3|PM|second|evening|| +77284|AAAAAAAAFONCBAAA|77284|21|28|4|PM|second|evening|| +77285|AAAAAAAAGONCBAAA|77285|21|28|5|PM|second|evening|| +77286|AAAAAAAAHONCBAAA|77286|21|28|6|PM|second|evening|| +77287|AAAAAAAAIONCBAAA|77287|21|28|7|PM|second|evening|| +77288|AAAAAAAAJONCBAAA|77288|21|28|8|PM|second|evening|| +77289|AAAAAAAAKONCBAAA|77289|21|28|9|PM|second|evening|| +77290|AAAAAAAALONCBAAA|77290|21|28|10|PM|second|evening|| +77291|AAAAAAAAMONCBAAA|77291|21|28|11|PM|second|evening|| +77292|AAAAAAAANONCBAAA|77292|21|28|12|PM|second|evening|| +77293|AAAAAAAAOONCBAAA|77293|21|28|13|PM|second|evening|| +77294|AAAAAAAAPONCBAAA|77294|21|28|14|PM|second|evening|| +77295|AAAAAAAAAPNCBAAA|77295|21|28|15|PM|second|evening|| +77296|AAAAAAAABPNCBAAA|77296|21|28|16|PM|second|evening|| +77297|AAAAAAAACPNCBAAA|77297|21|28|17|PM|second|evening|| +77298|AAAAAAAADPNCBAAA|77298|21|28|18|PM|second|evening|| +77299|AAAAAAAAEPNCBAAA|77299|21|28|19|PM|second|evening|| +77300|AAAAAAAAFPNCBAAA|77300|21|28|20|PM|second|evening|| +77301|AAAAAAAAGPNCBAAA|77301|21|28|21|PM|second|evening|| +77302|AAAAAAAAHPNCBAAA|77302|21|28|22|PM|second|evening|| +77303|AAAAAAAAIPNCBAAA|77303|21|28|23|PM|second|evening|| +77304|AAAAAAAAJPNCBAAA|77304|21|28|24|PM|second|evening|| +77305|AAAAAAAAKPNCBAAA|77305|21|28|25|PM|second|evening|| +77306|AAAAAAAALPNCBAAA|77306|21|28|26|PM|second|evening|| +77307|AAAAAAAAMPNCBAAA|77307|21|28|27|PM|second|evening|| +77308|AAAAAAAANPNCBAAA|77308|21|28|28|PM|second|evening|| +77309|AAAAAAAAOPNCBAAA|77309|21|28|29|PM|second|evening|| +77310|AAAAAAAAPPNCBAAA|77310|21|28|30|PM|second|evening|| +77311|AAAAAAAAAAOCBAAA|77311|21|28|31|PM|second|evening|| +77312|AAAAAAAABAOCBAAA|77312|21|28|32|PM|second|evening|| +77313|AAAAAAAACAOCBAAA|77313|21|28|33|PM|second|evening|| +77314|AAAAAAAADAOCBAAA|77314|21|28|34|PM|second|evening|| +77315|AAAAAAAAEAOCBAAA|77315|21|28|35|PM|second|evening|| +77316|AAAAAAAAFAOCBAAA|77316|21|28|36|PM|second|evening|| +77317|AAAAAAAAGAOCBAAA|77317|21|28|37|PM|second|evening|| +77318|AAAAAAAAHAOCBAAA|77318|21|28|38|PM|second|evening|| +77319|AAAAAAAAIAOCBAAA|77319|21|28|39|PM|second|evening|| +77320|AAAAAAAAJAOCBAAA|77320|21|28|40|PM|second|evening|| +77321|AAAAAAAAKAOCBAAA|77321|21|28|41|PM|second|evening|| +77322|AAAAAAAALAOCBAAA|77322|21|28|42|PM|second|evening|| +77323|AAAAAAAAMAOCBAAA|77323|21|28|43|PM|second|evening|| +77324|AAAAAAAANAOCBAAA|77324|21|28|44|PM|second|evening|| +77325|AAAAAAAAOAOCBAAA|77325|21|28|45|PM|second|evening|| +77326|AAAAAAAAPAOCBAAA|77326|21|28|46|PM|second|evening|| +77327|AAAAAAAAABOCBAAA|77327|21|28|47|PM|second|evening|| +77328|AAAAAAAABBOCBAAA|77328|21|28|48|PM|second|evening|| +77329|AAAAAAAACBOCBAAA|77329|21|28|49|PM|second|evening|| +77330|AAAAAAAADBOCBAAA|77330|21|28|50|PM|second|evening|| +77331|AAAAAAAAEBOCBAAA|77331|21|28|51|PM|second|evening|| +77332|AAAAAAAAFBOCBAAA|77332|21|28|52|PM|second|evening|| +77333|AAAAAAAAGBOCBAAA|77333|21|28|53|PM|second|evening|| +77334|AAAAAAAAHBOCBAAA|77334|21|28|54|PM|second|evening|| +77335|AAAAAAAAIBOCBAAA|77335|21|28|55|PM|second|evening|| +77336|AAAAAAAAJBOCBAAA|77336|21|28|56|PM|second|evening|| +77337|AAAAAAAAKBOCBAAA|77337|21|28|57|PM|second|evening|| +77338|AAAAAAAALBOCBAAA|77338|21|28|58|PM|second|evening|| +77339|AAAAAAAAMBOCBAAA|77339|21|28|59|PM|second|evening|| +77340|AAAAAAAANBOCBAAA|77340|21|29|0|PM|second|evening|| +77341|AAAAAAAAOBOCBAAA|77341|21|29|1|PM|second|evening|| +77342|AAAAAAAAPBOCBAAA|77342|21|29|2|PM|second|evening|| +77343|AAAAAAAAACOCBAAA|77343|21|29|3|PM|second|evening|| +77344|AAAAAAAABCOCBAAA|77344|21|29|4|PM|second|evening|| +77345|AAAAAAAACCOCBAAA|77345|21|29|5|PM|second|evening|| +77346|AAAAAAAADCOCBAAA|77346|21|29|6|PM|second|evening|| +77347|AAAAAAAAECOCBAAA|77347|21|29|7|PM|second|evening|| +77348|AAAAAAAAFCOCBAAA|77348|21|29|8|PM|second|evening|| +77349|AAAAAAAAGCOCBAAA|77349|21|29|9|PM|second|evening|| +77350|AAAAAAAAHCOCBAAA|77350|21|29|10|PM|second|evening|| +77351|AAAAAAAAICOCBAAA|77351|21|29|11|PM|second|evening|| +77352|AAAAAAAAJCOCBAAA|77352|21|29|12|PM|second|evening|| +77353|AAAAAAAAKCOCBAAA|77353|21|29|13|PM|second|evening|| +77354|AAAAAAAALCOCBAAA|77354|21|29|14|PM|second|evening|| +77355|AAAAAAAAMCOCBAAA|77355|21|29|15|PM|second|evening|| +77356|AAAAAAAANCOCBAAA|77356|21|29|16|PM|second|evening|| +77357|AAAAAAAAOCOCBAAA|77357|21|29|17|PM|second|evening|| +77358|AAAAAAAAPCOCBAAA|77358|21|29|18|PM|second|evening|| +77359|AAAAAAAAADOCBAAA|77359|21|29|19|PM|second|evening|| +77360|AAAAAAAABDOCBAAA|77360|21|29|20|PM|second|evening|| +77361|AAAAAAAACDOCBAAA|77361|21|29|21|PM|second|evening|| +77362|AAAAAAAADDOCBAAA|77362|21|29|22|PM|second|evening|| +77363|AAAAAAAAEDOCBAAA|77363|21|29|23|PM|second|evening|| +77364|AAAAAAAAFDOCBAAA|77364|21|29|24|PM|second|evening|| +77365|AAAAAAAAGDOCBAAA|77365|21|29|25|PM|second|evening|| +77366|AAAAAAAAHDOCBAAA|77366|21|29|26|PM|second|evening|| +77367|AAAAAAAAIDOCBAAA|77367|21|29|27|PM|second|evening|| +77368|AAAAAAAAJDOCBAAA|77368|21|29|28|PM|second|evening|| +77369|AAAAAAAAKDOCBAAA|77369|21|29|29|PM|second|evening|| +77370|AAAAAAAALDOCBAAA|77370|21|29|30|PM|second|evening|| +77371|AAAAAAAAMDOCBAAA|77371|21|29|31|PM|second|evening|| +77372|AAAAAAAANDOCBAAA|77372|21|29|32|PM|second|evening|| +77373|AAAAAAAAODOCBAAA|77373|21|29|33|PM|second|evening|| +77374|AAAAAAAAPDOCBAAA|77374|21|29|34|PM|second|evening|| +77375|AAAAAAAAAEOCBAAA|77375|21|29|35|PM|second|evening|| +77376|AAAAAAAABEOCBAAA|77376|21|29|36|PM|second|evening|| +77377|AAAAAAAACEOCBAAA|77377|21|29|37|PM|second|evening|| +77378|AAAAAAAADEOCBAAA|77378|21|29|38|PM|second|evening|| +77379|AAAAAAAAEEOCBAAA|77379|21|29|39|PM|second|evening|| +77380|AAAAAAAAFEOCBAAA|77380|21|29|40|PM|second|evening|| +77381|AAAAAAAAGEOCBAAA|77381|21|29|41|PM|second|evening|| +77382|AAAAAAAAHEOCBAAA|77382|21|29|42|PM|second|evening|| +77383|AAAAAAAAIEOCBAAA|77383|21|29|43|PM|second|evening|| +77384|AAAAAAAAJEOCBAAA|77384|21|29|44|PM|second|evening|| +77385|AAAAAAAAKEOCBAAA|77385|21|29|45|PM|second|evening|| +77386|AAAAAAAALEOCBAAA|77386|21|29|46|PM|second|evening|| +77387|AAAAAAAAMEOCBAAA|77387|21|29|47|PM|second|evening|| +77388|AAAAAAAANEOCBAAA|77388|21|29|48|PM|second|evening|| +77389|AAAAAAAAOEOCBAAA|77389|21|29|49|PM|second|evening|| +77390|AAAAAAAAPEOCBAAA|77390|21|29|50|PM|second|evening|| +77391|AAAAAAAAAFOCBAAA|77391|21|29|51|PM|second|evening|| +77392|AAAAAAAABFOCBAAA|77392|21|29|52|PM|second|evening|| +77393|AAAAAAAACFOCBAAA|77393|21|29|53|PM|second|evening|| +77394|AAAAAAAADFOCBAAA|77394|21|29|54|PM|second|evening|| +77395|AAAAAAAAEFOCBAAA|77395|21|29|55|PM|second|evening|| +77396|AAAAAAAAFFOCBAAA|77396|21|29|56|PM|second|evening|| +77397|AAAAAAAAGFOCBAAA|77397|21|29|57|PM|second|evening|| +77398|AAAAAAAAHFOCBAAA|77398|21|29|58|PM|second|evening|| +77399|AAAAAAAAIFOCBAAA|77399|21|29|59|PM|second|evening|| +77400|AAAAAAAAJFOCBAAA|77400|21|30|0|PM|second|evening|| +77401|AAAAAAAAKFOCBAAA|77401|21|30|1|PM|second|evening|| +77402|AAAAAAAALFOCBAAA|77402|21|30|2|PM|second|evening|| +77403|AAAAAAAAMFOCBAAA|77403|21|30|3|PM|second|evening|| +77404|AAAAAAAANFOCBAAA|77404|21|30|4|PM|second|evening|| +77405|AAAAAAAAOFOCBAAA|77405|21|30|5|PM|second|evening|| +77406|AAAAAAAAPFOCBAAA|77406|21|30|6|PM|second|evening|| +77407|AAAAAAAAAGOCBAAA|77407|21|30|7|PM|second|evening|| +77408|AAAAAAAABGOCBAAA|77408|21|30|8|PM|second|evening|| +77409|AAAAAAAACGOCBAAA|77409|21|30|9|PM|second|evening|| +77410|AAAAAAAADGOCBAAA|77410|21|30|10|PM|second|evening|| +77411|AAAAAAAAEGOCBAAA|77411|21|30|11|PM|second|evening|| +77412|AAAAAAAAFGOCBAAA|77412|21|30|12|PM|second|evening|| +77413|AAAAAAAAGGOCBAAA|77413|21|30|13|PM|second|evening|| +77414|AAAAAAAAHGOCBAAA|77414|21|30|14|PM|second|evening|| +77415|AAAAAAAAIGOCBAAA|77415|21|30|15|PM|second|evening|| +77416|AAAAAAAAJGOCBAAA|77416|21|30|16|PM|second|evening|| +77417|AAAAAAAAKGOCBAAA|77417|21|30|17|PM|second|evening|| +77418|AAAAAAAALGOCBAAA|77418|21|30|18|PM|second|evening|| +77419|AAAAAAAAMGOCBAAA|77419|21|30|19|PM|second|evening|| +77420|AAAAAAAANGOCBAAA|77420|21|30|20|PM|second|evening|| +77421|AAAAAAAAOGOCBAAA|77421|21|30|21|PM|second|evening|| +77422|AAAAAAAAPGOCBAAA|77422|21|30|22|PM|second|evening|| +77423|AAAAAAAAAHOCBAAA|77423|21|30|23|PM|second|evening|| +77424|AAAAAAAABHOCBAAA|77424|21|30|24|PM|second|evening|| +77425|AAAAAAAACHOCBAAA|77425|21|30|25|PM|second|evening|| +77426|AAAAAAAADHOCBAAA|77426|21|30|26|PM|second|evening|| +77427|AAAAAAAAEHOCBAAA|77427|21|30|27|PM|second|evening|| +77428|AAAAAAAAFHOCBAAA|77428|21|30|28|PM|second|evening|| +77429|AAAAAAAAGHOCBAAA|77429|21|30|29|PM|second|evening|| +77430|AAAAAAAAHHOCBAAA|77430|21|30|30|PM|second|evening|| +77431|AAAAAAAAIHOCBAAA|77431|21|30|31|PM|second|evening|| +77432|AAAAAAAAJHOCBAAA|77432|21|30|32|PM|second|evening|| +77433|AAAAAAAAKHOCBAAA|77433|21|30|33|PM|second|evening|| +77434|AAAAAAAALHOCBAAA|77434|21|30|34|PM|second|evening|| +77435|AAAAAAAAMHOCBAAA|77435|21|30|35|PM|second|evening|| +77436|AAAAAAAANHOCBAAA|77436|21|30|36|PM|second|evening|| +77437|AAAAAAAAOHOCBAAA|77437|21|30|37|PM|second|evening|| +77438|AAAAAAAAPHOCBAAA|77438|21|30|38|PM|second|evening|| +77439|AAAAAAAAAIOCBAAA|77439|21|30|39|PM|second|evening|| +77440|AAAAAAAABIOCBAAA|77440|21|30|40|PM|second|evening|| +77441|AAAAAAAACIOCBAAA|77441|21|30|41|PM|second|evening|| +77442|AAAAAAAADIOCBAAA|77442|21|30|42|PM|second|evening|| +77443|AAAAAAAAEIOCBAAA|77443|21|30|43|PM|second|evening|| +77444|AAAAAAAAFIOCBAAA|77444|21|30|44|PM|second|evening|| +77445|AAAAAAAAGIOCBAAA|77445|21|30|45|PM|second|evening|| +77446|AAAAAAAAHIOCBAAA|77446|21|30|46|PM|second|evening|| +77447|AAAAAAAAIIOCBAAA|77447|21|30|47|PM|second|evening|| +77448|AAAAAAAAJIOCBAAA|77448|21|30|48|PM|second|evening|| +77449|AAAAAAAAKIOCBAAA|77449|21|30|49|PM|second|evening|| +77450|AAAAAAAALIOCBAAA|77450|21|30|50|PM|second|evening|| +77451|AAAAAAAAMIOCBAAA|77451|21|30|51|PM|second|evening|| +77452|AAAAAAAANIOCBAAA|77452|21|30|52|PM|second|evening|| +77453|AAAAAAAAOIOCBAAA|77453|21|30|53|PM|second|evening|| +77454|AAAAAAAAPIOCBAAA|77454|21|30|54|PM|second|evening|| +77455|AAAAAAAAAJOCBAAA|77455|21|30|55|PM|second|evening|| +77456|AAAAAAAABJOCBAAA|77456|21|30|56|PM|second|evening|| +77457|AAAAAAAACJOCBAAA|77457|21|30|57|PM|second|evening|| +77458|AAAAAAAADJOCBAAA|77458|21|30|58|PM|second|evening|| +77459|AAAAAAAAEJOCBAAA|77459|21|30|59|PM|second|evening|| +77460|AAAAAAAAFJOCBAAA|77460|21|31|0|PM|second|evening|| +77461|AAAAAAAAGJOCBAAA|77461|21|31|1|PM|second|evening|| +77462|AAAAAAAAHJOCBAAA|77462|21|31|2|PM|second|evening|| +77463|AAAAAAAAIJOCBAAA|77463|21|31|3|PM|second|evening|| +77464|AAAAAAAAJJOCBAAA|77464|21|31|4|PM|second|evening|| +77465|AAAAAAAAKJOCBAAA|77465|21|31|5|PM|second|evening|| +77466|AAAAAAAALJOCBAAA|77466|21|31|6|PM|second|evening|| +77467|AAAAAAAAMJOCBAAA|77467|21|31|7|PM|second|evening|| +77468|AAAAAAAANJOCBAAA|77468|21|31|8|PM|second|evening|| +77469|AAAAAAAAOJOCBAAA|77469|21|31|9|PM|second|evening|| +77470|AAAAAAAAPJOCBAAA|77470|21|31|10|PM|second|evening|| +77471|AAAAAAAAAKOCBAAA|77471|21|31|11|PM|second|evening|| +77472|AAAAAAAABKOCBAAA|77472|21|31|12|PM|second|evening|| +77473|AAAAAAAACKOCBAAA|77473|21|31|13|PM|second|evening|| +77474|AAAAAAAADKOCBAAA|77474|21|31|14|PM|second|evening|| +77475|AAAAAAAAEKOCBAAA|77475|21|31|15|PM|second|evening|| +77476|AAAAAAAAFKOCBAAA|77476|21|31|16|PM|second|evening|| +77477|AAAAAAAAGKOCBAAA|77477|21|31|17|PM|second|evening|| +77478|AAAAAAAAHKOCBAAA|77478|21|31|18|PM|second|evening|| +77479|AAAAAAAAIKOCBAAA|77479|21|31|19|PM|second|evening|| +77480|AAAAAAAAJKOCBAAA|77480|21|31|20|PM|second|evening|| +77481|AAAAAAAAKKOCBAAA|77481|21|31|21|PM|second|evening|| +77482|AAAAAAAALKOCBAAA|77482|21|31|22|PM|second|evening|| +77483|AAAAAAAAMKOCBAAA|77483|21|31|23|PM|second|evening|| +77484|AAAAAAAANKOCBAAA|77484|21|31|24|PM|second|evening|| +77485|AAAAAAAAOKOCBAAA|77485|21|31|25|PM|second|evening|| +77486|AAAAAAAAPKOCBAAA|77486|21|31|26|PM|second|evening|| +77487|AAAAAAAAALOCBAAA|77487|21|31|27|PM|second|evening|| +77488|AAAAAAAABLOCBAAA|77488|21|31|28|PM|second|evening|| +77489|AAAAAAAACLOCBAAA|77489|21|31|29|PM|second|evening|| +77490|AAAAAAAADLOCBAAA|77490|21|31|30|PM|second|evening|| +77491|AAAAAAAAELOCBAAA|77491|21|31|31|PM|second|evening|| +77492|AAAAAAAAFLOCBAAA|77492|21|31|32|PM|second|evening|| +77493|AAAAAAAAGLOCBAAA|77493|21|31|33|PM|second|evening|| +77494|AAAAAAAAHLOCBAAA|77494|21|31|34|PM|second|evening|| +77495|AAAAAAAAILOCBAAA|77495|21|31|35|PM|second|evening|| +77496|AAAAAAAAJLOCBAAA|77496|21|31|36|PM|second|evening|| +77497|AAAAAAAAKLOCBAAA|77497|21|31|37|PM|second|evening|| +77498|AAAAAAAALLOCBAAA|77498|21|31|38|PM|second|evening|| +77499|AAAAAAAAMLOCBAAA|77499|21|31|39|PM|second|evening|| +77500|AAAAAAAANLOCBAAA|77500|21|31|40|PM|second|evening|| +77501|AAAAAAAAOLOCBAAA|77501|21|31|41|PM|second|evening|| +77502|AAAAAAAAPLOCBAAA|77502|21|31|42|PM|second|evening|| +77503|AAAAAAAAAMOCBAAA|77503|21|31|43|PM|second|evening|| +77504|AAAAAAAABMOCBAAA|77504|21|31|44|PM|second|evening|| +77505|AAAAAAAACMOCBAAA|77505|21|31|45|PM|second|evening|| +77506|AAAAAAAADMOCBAAA|77506|21|31|46|PM|second|evening|| +77507|AAAAAAAAEMOCBAAA|77507|21|31|47|PM|second|evening|| +77508|AAAAAAAAFMOCBAAA|77508|21|31|48|PM|second|evening|| +77509|AAAAAAAAGMOCBAAA|77509|21|31|49|PM|second|evening|| +77510|AAAAAAAAHMOCBAAA|77510|21|31|50|PM|second|evening|| +77511|AAAAAAAAIMOCBAAA|77511|21|31|51|PM|second|evening|| +77512|AAAAAAAAJMOCBAAA|77512|21|31|52|PM|second|evening|| +77513|AAAAAAAAKMOCBAAA|77513|21|31|53|PM|second|evening|| +77514|AAAAAAAALMOCBAAA|77514|21|31|54|PM|second|evening|| +77515|AAAAAAAAMMOCBAAA|77515|21|31|55|PM|second|evening|| +77516|AAAAAAAANMOCBAAA|77516|21|31|56|PM|second|evening|| +77517|AAAAAAAAOMOCBAAA|77517|21|31|57|PM|second|evening|| +77518|AAAAAAAAPMOCBAAA|77518|21|31|58|PM|second|evening|| +77519|AAAAAAAAANOCBAAA|77519|21|31|59|PM|second|evening|| +77520|AAAAAAAABNOCBAAA|77520|21|32|0|PM|second|evening|| +77521|AAAAAAAACNOCBAAA|77521|21|32|1|PM|second|evening|| +77522|AAAAAAAADNOCBAAA|77522|21|32|2|PM|second|evening|| +77523|AAAAAAAAENOCBAAA|77523|21|32|3|PM|second|evening|| +77524|AAAAAAAAFNOCBAAA|77524|21|32|4|PM|second|evening|| +77525|AAAAAAAAGNOCBAAA|77525|21|32|5|PM|second|evening|| +77526|AAAAAAAAHNOCBAAA|77526|21|32|6|PM|second|evening|| +77527|AAAAAAAAINOCBAAA|77527|21|32|7|PM|second|evening|| +77528|AAAAAAAAJNOCBAAA|77528|21|32|8|PM|second|evening|| +77529|AAAAAAAAKNOCBAAA|77529|21|32|9|PM|second|evening|| +77530|AAAAAAAALNOCBAAA|77530|21|32|10|PM|second|evening|| +77531|AAAAAAAAMNOCBAAA|77531|21|32|11|PM|second|evening|| +77532|AAAAAAAANNOCBAAA|77532|21|32|12|PM|second|evening|| +77533|AAAAAAAAONOCBAAA|77533|21|32|13|PM|second|evening|| +77534|AAAAAAAAPNOCBAAA|77534|21|32|14|PM|second|evening|| +77535|AAAAAAAAAOOCBAAA|77535|21|32|15|PM|second|evening|| +77536|AAAAAAAABOOCBAAA|77536|21|32|16|PM|second|evening|| +77537|AAAAAAAACOOCBAAA|77537|21|32|17|PM|second|evening|| +77538|AAAAAAAADOOCBAAA|77538|21|32|18|PM|second|evening|| +77539|AAAAAAAAEOOCBAAA|77539|21|32|19|PM|second|evening|| +77540|AAAAAAAAFOOCBAAA|77540|21|32|20|PM|second|evening|| +77541|AAAAAAAAGOOCBAAA|77541|21|32|21|PM|second|evening|| +77542|AAAAAAAAHOOCBAAA|77542|21|32|22|PM|second|evening|| +77543|AAAAAAAAIOOCBAAA|77543|21|32|23|PM|second|evening|| +77544|AAAAAAAAJOOCBAAA|77544|21|32|24|PM|second|evening|| +77545|AAAAAAAAKOOCBAAA|77545|21|32|25|PM|second|evening|| +77546|AAAAAAAALOOCBAAA|77546|21|32|26|PM|second|evening|| +77547|AAAAAAAAMOOCBAAA|77547|21|32|27|PM|second|evening|| +77548|AAAAAAAANOOCBAAA|77548|21|32|28|PM|second|evening|| +77549|AAAAAAAAOOOCBAAA|77549|21|32|29|PM|second|evening|| +77550|AAAAAAAAPOOCBAAA|77550|21|32|30|PM|second|evening|| +77551|AAAAAAAAAPOCBAAA|77551|21|32|31|PM|second|evening|| +77552|AAAAAAAABPOCBAAA|77552|21|32|32|PM|second|evening|| +77553|AAAAAAAACPOCBAAA|77553|21|32|33|PM|second|evening|| +77554|AAAAAAAADPOCBAAA|77554|21|32|34|PM|second|evening|| +77555|AAAAAAAAEPOCBAAA|77555|21|32|35|PM|second|evening|| +77556|AAAAAAAAFPOCBAAA|77556|21|32|36|PM|second|evening|| +77557|AAAAAAAAGPOCBAAA|77557|21|32|37|PM|second|evening|| +77558|AAAAAAAAHPOCBAAA|77558|21|32|38|PM|second|evening|| +77559|AAAAAAAAIPOCBAAA|77559|21|32|39|PM|second|evening|| +77560|AAAAAAAAJPOCBAAA|77560|21|32|40|PM|second|evening|| +77561|AAAAAAAAKPOCBAAA|77561|21|32|41|PM|second|evening|| +77562|AAAAAAAALPOCBAAA|77562|21|32|42|PM|second|evening|| +77563|AAAAAAAAMPOCBAAA|77563|21|32|43|PM|second|evening|| +77564|AAAAAAAANPOCBAAA|77564|21|32|44|PM|second|evening|| +77565|AAAAAAAAOPOCBAAA|77565|21|32|45|PM|second|evening|| +77566|AAAAAAAAPPOCBAAA|77566|21|32|46|PM|second|evening|| +77567|AAAAAAAAAAPCBAAA|77567|21|32|47|PM|second|evening|| +77568|AAAAAAAABAPCBAAA|77568|21|32|48|PM|second|evening|| +77569|AAAAAAAACAPCBAAA|77569|21|32|49|PM|second|evening|| +77570|AAAAAAAADAPCBAAA|77570|21|32|50|PM|second|evening|| +77571|AAAAAAAAEAPCBAAA|77571|21|32|51|PM|second|evening|| +77572|AAAAAAAAFAPCBAAA|77572|21|32|52|PM|second|evening|| +77573|AAAAAAAAGAPCBAAA|77573|21|32|53|PM|second|evening|| +77574|AAAAAAAAHAPCBAAA|77574|21|32|54|PM|second|evening|| +77575|AAAAAAAAIAPCBAAA|77575|21|32|55|PM|second|evening|| +77576|AAAAAAAAJAPCBAAA|77576|21|32|56|PM|second|evening|| +77577|AAAAAAAAKAPCBAAA|77577|21|32|57|PM|second|evening|| +77578|AAAAAAAALAPCBAAA|77578|21|32|58|PM|second|evening|| +77579|AAAAAAAAMAPCBAAA|77579|21|32|59|PM|second|evening|| +77580|AAAAAAAANAPCBAAA|77580|21|33|0|PM|second|evening|| +77581|AAAAAAAAOAPCBAAA|77581|21|33|1|PM|second|evening|| +77582|AAAAAAAAPAPCBAAA|77582|21|33|2|PM|second|evening|| +77583|AAAAAAAAABPCBAAA|77583|21|33|3|PM|second|evening|| +77584|AAAAAAAABBPCBAAA|77584|21|33|4|PM|second|evening|| +77585|AAAAAAAACBPCBAAA|77585|21|33|5|PM|second|evening|| +77586|AAAAAAAADBPCBAAA|77586|21|33|6|PM|second|evening|| +77587|AAAAAAAAEBPCBAAA|77587|21|33|7|PM|second|evening|| +77588|AAAAAAAAFBPCBAAA|77588|21|33|8|PM|second|evening|| +77589|AAAAAAAAGBPCBAAA|77589|21|33|9|PM|second|evening|| +77590|AAAAAAAAHBPCBAAA|77590|21|33|10|PM|second|evening|| +77591|AAAAAAAAIBPCBAAA|77591|21|33|11|PM|second|evening|| +77592|AAAAAAAAJBPCBAAA|77592|21|33|12|PM|second|evening|| +77593|AAAAAAAAKBPCBAAA|77593|21|33|13|PM|second|evening|| +77594|AAAAAAAALBPCBAAA|77594|21|33|14|PM|second|evening|| +77595|AAAAAAAAMBPCBAAA|77595|21|33|15|PM|second|evening|| +77596|AAAAAAAANBPCBAAA|77596|21|33|16|PM|second|evening|| +77597|AAAAAAAAOBPCBAAA|77597|21|33|17|PM|second|evening|| +77598|AAAAAAAAPBPCBAAA|77598|21|33|18|PM|second|evening|| +77599|AAAAAAAAACPCBAAA|77599|21|33|19|PM|second|evening|| +77600|AAAAAAAABCPCBAAA|77600|21|33|20|PM|second|evening|| +77601|AAAAAAAACCPCBAAA|77601|21|33|21|PM|second|evening|| +77602|AAAAAAAADCPCBAAA|77602|21|33|22|PM|second|evening|| +77603|AAAAAAAAECPCBAAA|77603|21|33|23|PM|second|evening|| +77604|AAAAAAAAFCPCBAAA|77604|21|33|24|PM|second|evening|| +77605|AAAAAAAAGCPCBAAA|77605|21|33|25|PM|second|evening|| +77606|AAAAAAAAHCPCBAAA|77606|21|33|26|PM|second|evening|| +77607|AAAAAAAAICPCBAAA|77607|21|33|27|PM|second|evening|| +77608|AAAAAAAAJCPCBAAA|77608|21|33|28|PM|second|evening|| +77609|AAAAAAAAKCPCBAAA|77609|21|33|29|PM|second|evening|| +77610|AAAAAAAALCPCBAAA|77610|21|33|30|PM|second|evening|| +77611|AAAAAAAAMCPCBAAA|77611|21|33|31|PM|second|evening|| +77612|AAAAAAAANCPCBAAA|77612|21|33|32|PM|second|evening|| +77613|AAAAAAAAOCPCBAAA|77613|21|33|33|PM|second|evening|| +77614|AAAAAAAAPCPCBAAA|77614|21|33|34|PM|second|evening|| +77615|AAAAAAAAADPCBAAA|77615|21|33|35|PM|second|evening|| +77616|AAAAAAAABDPCBAAA|77616|21|33|36|PM|second|evening|| +77617|AAAAAAAACDPCBAAA|77617|21|33|37|PM|second|evening|| +77618|AAAAAAAADDPCBAAA|77618|21|33|38|PM|second|evening|| +77619|AAAAAAAAEDPCBAAA|77619|21|33|39|PM|second|evening|| +77620|AAAAAAAAFDPCBAAA|77620|21|33|40|PM|second|evening|| +77621|AAAAAAAAGDPCBAAA|77621|21|33|41|PM|second|evening|| +77622|AAAAAAAAHDPCBAAA|77622|21|33|42|PM|second|evening|| +77623|AAAAAAAAIDPCBAAA|77623|21|33|43|PM|second|evening|| +77624|AAAAAAAAJDPCBAAA|77624|21|33|44|PM|second|evening|| +77625|AAAAAAAAKDPCBAAA|77625|21|33|45|PM|second|evening|| +77626|AAAAAAAALDPCBAAA|77626|21|33|46|PM|second|evening|| +77627|AAAAAAAAMDPCBAAA|77627|21|33|47|PM|second|evening|| +77628|AAAAAAAANDPCBAAA|77628|21|33|48|PM|second|evening|| +77629|AAAAAAAAODPCBAAA|77629|21|33|49|PM|second|evening|| +77630|AAAAAAAAPDPCBAAA|77630|21|33|50|PM|second|evening|| +77631|AAAAAAAAAEPCBAAA|77631|21|33|51|PM|second|evening|| +77632|AAAAAAAABEPCBAAA|77632|21|33|52|PM|second|evening|| +77633|AAAAAAAACEPCBAAA|77633|21|33|53|PM|second|evening|| +77634|AAAAAAAADEPCBAAA|77634|21|33|54|PM|second|evening|| +77635|AAAAAAAAEEPCBAAA|77635|21|33|55|PM|second|evening|| +77636|AAAAAAAAFEPCBAAA|77636|21|33|56|PM|second|evening|| +77637|AAAAAAAAGEPCBAAA|77637|21|33|57|PM|second|evening|| +77638|AAAAAAAAHEPCBAAA|77638|21|33|58|PM|second|evening|| +77639|AAAAAAAAIEPCBAAA|77639|21|33|59|PM|second|evening|| +77640|AAAAAAAAJEPCBAAA|77640|21|34|0|PM|second|evening|| +77641|AAAAAAAAKEPCBAAA|77641|21|34|1|PM|second|evening|| +77642|AAAAAAAALEPCBAAA|77642|21|34|2|PM|second|evening|| +77643|AAAAAAAAMEPCBAAA|77643|21|34|3|PM|second|evening|| +77644|AAAAAAAANEPCBAAA|77644|21|34|4|PM|second|evening|| +77645|AAAAAAAAOEPCBAAA|77645|21|34|5|PM|second|evening|| +77646|AAAAAAAAPEPCBAAA|77646|21|34|6|PM|second|evening|| +77647|AAAAAAAAAFPCBAAA|77647|21|34|7|PM|second|evening|| +77648|AAAAAAAABFPCBAAA|77648|21|34|8|PM|second|evening|| +77649|AAAAAAAACFPCBAAA|77649|21|34|9|PM|second|evening|| +77650|AAAAAAAADFPCBAAA|77650|21|34|10|PM|second|evening|| +77651|AAAAAAAAEFPCBAAA|77651|21|34|11|PM|second|evening|| +77652|AAAAAAAAFFPCBAAA|77652|21|34|12|PM|second|evening|| +77653|AAAAAAAAGFPCBAAA|77653|21|34|13|PM|second|evening|| +77654|AAAAAAAAHFPCBAAA|77654|21|34|14|PM|second|evening|| +77655|AAAAAAAAIFPCBAAA|77655|21|34|15|PM|second|evening|| +77656|AAAAAAAAJFPCBAAA|77656|21|34|16|PM|second|evening|| +77657|AAAAAAAAKFPCBAAA|77657|21|34|17|PM|second|evening|| +77658|AAAAAAAALFPCBAAA|77658|21|34|18|PM|second|evening|| +77659|AAAAAAAAMFPCBAAA|77659|21|34|19|PM|second|evening|| +77660|AAAAAAAANFPCBAAA|77660|21|34|20|PM|second|evening|| +77661|AAAAAAAAOFPCBAAA|77661|21|34|21|PM|second|evening|| +77662|AAAAAAAAPFPCBAAA|77662|21|34|22|PM|second|evening|| +77663|AAAAAAAAAGPCBAAA|77663|21|34|23|PM|second|evening|| +77664|AAAAAAAABGPCBAAA|77664|21|34|24|PM|second|evening|| +77665|AAAAAAAACGPCBAAA|77665|21|34|25|PM|second|evening|| +77666|AAAAAAAADGPCBAAA|77666|21|34|26|PM|second|evening|| +77667|AAAAAAAAEGPCBAAA|77667|21|34|27|PM|second|evening|| +77668|AAAAAAAAFGPCBAAA|77668|21|34|28|PM|second|evening|| +77669|AAAAAAAAGGPCBAAA|77669|21|34|29|PM|second|evening|| +77670|AAAAAAAAHGPCBAAA|77670|21|34|30|PM|second|evening|| +77671|AAAAAAAAIGPCBAAA|77671|21|34|31|PM|second|evening|| +77672|AAAAAAAAJGPCBAAA|77672|21|34|32|PM|second|evening|| +77673|AAAAAAAAKGPCBAAA|77673|21|34|33|PM|second|evening|| +77674|AAAAAAAALGPCBAAA|77674|21|34|34|PM|second|evening|| +77675|AAAAAAAAMGPCBAAA|77675|21|34|35|PM|second|evening|| +77676|AAAAAAAANGPCBAAA|77676|21|34|36|PM|second|evening|| +77677|AAAAAAAAOGPCBAAA|77677|21|34|37|PM|second|evening|| +77678|AAAAAAAAPGPCBAAA|77678|21|34|38|PM|second|evening|| +77679|AAAAAAAAAHPCBAAA|77679|21|34|39|PM|second|evening|| +77680|AAAAAAAABHPCBAAA|77680|21|34|40|PM|second|evening|| +77681|AAAAAAAACHPCBAAA|77681|21|34|41|PM|second|evening|| +77682|AAAAAAAADHPCBAAA|77682|21|34|42|PM|second|evening|| +77683|AAAAAAAAEHPCBAAA|77683|21|34|43|PM|second|evening|| +77684|AAAAAAAAFHPCBAAA|77684|21|34|44|PM|second|evening|| +77685|AAAAAAAAGHPCBAAA|77685|21|34|45|PM|second|evening|| +77686|AAAAAAAAHHPCBAAA|77686|21|34|46|PM|second|evening|| +77687|AAAAAAAAIHPCBAAA|77687|21|34|47|PM|second|evening|| +77688|AAAAAAAAJHPCBAAA|77688|21|34|48|PM|second|evening|| +77689|AAAAAAAAKHPCBAAA|77689|21|34|49|PM|second|evening|| +77690|AAAAAAAALHPCBAAA|77690|21|34|50|PM|second|evening|| +77691|AAAAAAAAMHPCBAAA|77691|21|34|51|PM|second|evening|| +77692|AAAAAAAANHPCBAAA|77692|21|34|52|PM|second|evening|| +77693|AAAAAAAAOHPCBAAA|77693|21|34|53|PM|second|evening|| +77694|AAAAAAAAPHPCBAAA|77694|21|34|54|PM|second|evening|| +77695|AAAAAAAAAIPCBAAA|77695|21|34|55|PM|second|evening|| +77696|AAAAAAAABIPCBAAA|77696|21|34|56|PM|second|evening|| +77697|AAAAAAAACIPCBAAA|77697|21|34|57|PM|second|evening|| +77698|AAAAAAAADIPCBAAA|77698|21|34|58|PM|second|evening|| +77699|AAAAAAAAEIPCBAAA|77699|21|34|59|PM|second|evening|| +77700|AAAAAAAAFIPCBAAA|77700|21|35|0|PM|second|evening|| +77701|AAAAAAAAGIPCBAAA|77701|21|35|1|PM|second|evening|| +77702|AAAAAAAAHIPCBAAA|77702|21|35|2|PM|second|evening|| +77703|AAAAAAAAIIPCBAAA|77703|21|35|3|PM|second|evening|| +77704|AAAAAAAAJIPCBAAA|77704|21|35|4|PM|second|evening|| +77705|AAAAAAAAKIPCBAAA|77705|21|35|5|PM|second|evening|| +77706|AAAAAAAALIPCBAAA|77706|21|35|6|PM|second|evening|| +77707|AAAAAAAAMIPCBAAA|77707|21|35|7|PM|second|evening|| +77708|AAAAAAAANIPCBAAA|77708|21|35|8|PM|second|evening|| +77709|AAAAAAAAOIPCBAAA|77709|21|35|9|PM|second|evening|| +77710|AAAAAAAAPIPCBAAA|77710|21|35|10|PM|second|evening|| +77711|AAAAAAAAAJPCBAAA|77711|21|35|11|PM|second|evening|| +77712|AAAAAAAABJPCBAAA|77712|21|35|12|PM|second|evening|| +77713|AAAAAAAACJPCBAAA|77713|21|35|13|PM|second|evening|| +77714|AAAAAAAADJPCBAAA|77714|21|35|14|PM|second|evening|| +77715|AAAAAAAAEJPCBAAA|77715|21|35|15|PM|second|evening|| +77716|AAAAAAAAFJPCBAAA|77716|21|35|16|PM|second|evening|| +77717|AAAAAAAAGJPCBAAA|77717|21|35|17|PM|second|evening|| +77718|AAAAAAAAHJPCBAAA|77718|21|35|18|PM|second|evening|| +77719|AAAAAAAAIJPCBAAA|77719|21|35|19|PM|second|evening|| +77720|AAAAAAAAJJPCBAAA|77720|21|35|20|PM|second|evening|| +77721|AAAAAAAAKJPCBAAA|77721|21|35|21|PM|second|evening|| +77722|AAAAAAAALJPCBAAA|77722|21|35|22|PM|second|evening|| +77723|AAAAAAAAMJPCBAAA|77723|21|35|23|PM|second|evening|| +77724|AAAAAAAANJPCBAAA|77724|21|35|24|PM|second|evening|| +77725|AAAAAAAAOJPCBAAA|77725|21|35|25|PM|second|evening|| +77726|AAAAAAAAPJPCBAAA|77726|21|35|26|PM|second|evening|| +77727|AAAAAAAAAKPCBAAA|77727|21|35|27|PM|second|evening|| +77728|AAAAAAAABKPCBAAA|77728|21|35|28|PM|second|evening|| +77729|AAAAAAAACKPCBAAA|77729|21|35|29|PM|second|evening|| +77730|AAAAAAAADKPCBAAA|77730|21|35|30|PM|second|evening|| +77731|AAAAAAAAEKPCBAAA|77731|21|35|31|PM|second|evening|| +77732|AAAAAAAAFKPCBAAA|77732|21|35|32|PM|second|evening|| +77733|AAAAAAAAGKPCBAAA|77733|21|35|33|PM|second|evening|| +77734|AAAAAAAAHKPCBAAA|77734|21|35|34|PM|second|evening|| +77735|AAAAAAAAIKPCBAAA|77735|21|35|35|PM|second|evening|| +77736|AAAAAAAAJKPCBAAA|77736|21|35|36|PM|second|evening|| +77737|AAAAAAAAKKPCBAAA|77737|21|35|37|PM|second|evening|| +77738|AAAAAAAALKPCBAAA|77738|21|35|38|PM|second|evening|| +77739|AAAAAAAAMKPCBAAA|77739|21|35|39|PM|second|evening|| +77740|AAAAAAAANKPCBAAA|77740|21|35|40|PM|second|evening|| +77741|AAAAAAAAOKPCBAAA|77741|21|35|41|PM|second|evening|| +77742|AAAAAAAAPKPCBAAA|77742|21|35|42|PM|second|evening|| +77743|AAAAAAAAALPCBAAA|77743|21|35|43|PM|second|evening|| +77744|AAAAAAAABLPCBAAA|77744|21|35|44|PM|second|evening|| +77745|AAAAAAAACLPCBAAA|77745|21|35|45|PM|second|evening|| +77746|AAAAAAAADLPCBAAA|77746|21|35|46|PM|second|evening|| +77747|AAAAAAAAELPCBAAA|77747|21|35|47|PM|second|evening|| +77748|AAAAAAAAFLPCBAAA|77748|21|35|48|PM|second|evening|| +77749|AAAAAAAAGLPCBAAA|77749|21|35|49|PM|second|evening|| +77750|AAAAAAAAHLPCBAAA|77750|21|35|50|PM|second|evening|| +77751|AAAAAAAAILPCBAAA|77751|21|35|51|PM|second|evening|| +77752|AAAAAAAAJLPCBAAA|77752|21|35|52|PM|second|evening|| +77753|AAAAAAAAKLPCBAAA|77753|21|35|53|PM|second|evening|| +77754|AAAAAAAALLPCBAAA|77754|21|35|54|PM|second|evening|| +77755|AAAAAAAAMLPCBAAA|77755|21|35|55|PM|second|evening|| +77756|AAAAAAAANLPCBAAA|77756|21|35|56|PM|second|evening|| +77757|AAAAAAAAOLPCBAAA|77757|21|35|57|PM|second|evening|| +77758|AAAAAAAAPLPCBAAA|77758|21|35|58|PM|second|evening|| +77759|AAAAAAAAAMPCBAAA|77759|21|35|59|PM|second|evening|| +77760|AAAAAAAABMPCBAAA|77760|21|36|0|PM|second|evening|| +77761|AAAAAAAACMPCBAAA|77761|21|36|1|PM|second|evening|| +77762|AAAAAAAADMPCBAAA|77762|21|36|2|PM|second|evening|| +77763|AAAAAAAAEMPCBAAA|77763|21|36|3|PM|second|evening|| +77764|AAAAAAAAFMPCBAAA|77764|21|36|4|PM|second|evening|| +77765|AAAAAAAAGMPCBAAA|77765|21|36|5|PM|second|evening|| +77766|AAAAAAAAHMPCBAAA|77766|21|36|6|PM|second|evening|| +77767|AAAAAAAAIMPCBAAA|77767|21|36|7|PM|second|evening|| +77768|AAAAAAAAJMPCBAAA|77768|21|36|8|PM|second|evening|| +77769|AAAAAAAAKMPCBAAA|77769|21|36|9|PM|second|evening|| +77770|AAAAAAAALMPCBAAA|77770|21|36|10|PM|second|evening|| +77771|AAAAAAAAMMPCBAAA|77771|21|36|11|PM|second|evening|| +77772|AAAAAAAANMPCBAAA|77772|21|36|12|PM|second|evening|| +77773|AAAAAAAAOMPCBAAA|77773|21|36|13|PM|second|evening|| +77774|AAAAAAAAPMPCBAAA|77774|21|36|14|PM|second|evening|| +77775|AAAAAAAAANPCBAAA|77775|21|36|15|PM|second|evening|| +77776|AAAAAAAABNPCBAAA|77776|21|36|16|PM|second|evening|| +77777|AAAAAAAACNPCBAAA|77777|21|36|17|PM|second|evening|| +77778|AAAAAAAADNPCBAAA|77778|21|36|18|PM|second|evening|| +77779|AAAAAAAAENPCBAAA|77779|21|36|19|PM|second|evening|| +77780|AAAAAAAAFNPCBAAA|77780|21|36|20|PM|second|evening|| +77781|AAAAAAAAGNPCBAAA|77781|21|36|21|PM|second|evening|| +77782|AAAAAAAAHNPCBAAA|77782|21|36|22|PM|second|evening|| +77783|AAAAAAAAINPCBAAA|77783|21|36|23|PM|second|evening|| +77784|AAAAAAAAJNPCBAAA|77784|21|36|24|PM|second|evening|| +77785|AAAAAAAAKNPCBAAA|77785|21|36|25|PM|second|evening|| +77786|AAAAAAAALNPCBAAA|77786|21|36|26|PM|second|evening|| +77787|AAAAAAAAMNPCBAAA|77787|21|36|27|PM|second|evening|| +77788|AAAAAAAANNPCBAAA|77788|21|36|28|PM|second|evening|| +77789|AAAAAAAAONPCBAAA|77789|21|36|29|PM|second|evening|| +77790|AAAAAAAAPNPCBAAA|77790|21|36|30|PM|second|evening|| +77791|AAAAAAAAAOPCBAAA|77791|21|36|31|PM|second|evening|| +77792|AAAAAAAABOPCBAAA|77792|21|36|32|PM|second|evening|| +77793|AAAAAAAACOPCBAAA|77793|21|36|33|PM|second|evening|| +77794|AAAAAAAADOPCBAAA|77794|21|36|34|PM|second|evening|| +77795|AAAAAAAAEOPCBAAA|77795|21|36|35|PM|second|evening|| +77796|AAAAAAAAFOPCBAAA|77796|21|36|36|PM|second|evening|| +77797|AAAAAAAAGOPCBAAA|77797|21|36|37|PM|second|evening|| +77798|AAAAAAAAHOPCBAAA|77798|21|36|38|PM|second|evening|| +77799|AAAAAAAAIOPCBAAA|77799|21|36|39|PM|second|evening|| +77800|AAAAAAAAJOPCBAAA|77800|21|36|40|PM|second|evening|| +77801|AAAAAAAAKOPCBAAA|77801|21|36|41|PM|second|evening|| +77802|AAAAAAAALOPCBAAA|77802|21|36|42|PM|second|evening|| +77803|AAAAAAAAMOPCBAAA|77803|21|36|43|PM|second|evening|| +77804|AAAAAAAANOPCBAAA|77804|21|36|44|PM|second|evening|| +77805|AAAAAAAAOOPCBAAA|77805|21|36|45|PM|second|evening|| +77806|AAAAAAAAPOPCBAAA|77806|21|36|46|PM|second|evening|| +77807|AAAAAAAAAPPCBAAA|77807|21|36|47|PM|second|evening|| +77808|AAAAAAAABPPCBAAA|77808|21|36|48|PM|second|evening|| +77809|AAAAAAAACPPCBAAA|77809|21|36|49|PM|second|evening|| +77810|AAAAAAAADPPCBAAA|77810|21|36|50|PM|second|evening|| +77811|AAAAAAAAEPPCBAAA|77811|21|36|51|PM|second|evening|| +77812|AAAAAAAAFPPCBAAA|77812|21|36|52|PM|second|evening|| +77813|AAAAAAAAGPPCBAAA|77813|21|36|53|PM|second|evening|| +77814|AAAAAAAAHPPCBAAA|77814|21|36|54|PM|second|evening|| +77815|AAAAAAAAIPPCBAAA|77815|21|36|55|PM|second|evening|| +77816|AAAAAAAAJPPCBAAA|77816|21|36|56|PM|second|evening|| +77817|AAAAAAAAKPPCBAAA|77817|21|36|57|PM|second|evening|| +77818|AAAAAAAALPPCBAAA|77818|21|36|58|PM|second|evening|| +77819|AAAAAAAAMPPCBAAA|77819|21|36|59|PM|second|evening|| +77820|AAAAAAAANPPCBAAA|77820|21|37|0|PM|second|evening|| +77821|AAAAAAAAOPPCBAAA|77821|21|37|1|PM|second|evening|| +77822|AAAAAAAAPPPCBAAA|77822|21|37|2|PM|second|evening|| +77823|AAAAAAAAAAADBAAA|77823|21|37|3|PM|second|evening|| +77824|AAAAAAAABAADBAAA|77824|21|37|4|PM|second|evening|| +77825|AAAAAAAACAADBAAA|77825|21|37|5|PM|second|evening|| +77826|AAAAAAAADAADBAAA|77826|21|37|6|PM|second|evening|| +77827|AAAAAAAAEAADBAAA|77827|21|37|7|PM|second|evening|| +77828|AAAAAAAAFAADBAAA|77828|21|37|8|PM|second|evening|| +77829|AAAAAAAAGAADBAAA|77829|21|37|9|PM|second|evening|| +77830|AAAAAAAAHAADBAAA|77830|21|37|10|PM|second|evening|| +77831|AAAAAAAAIAADBAAA|77831|21|37|11|PM|second|evening|| +77832|AAAAAAAAJAADBAAA|77832|21|37|12|PM|second|evening|| +77833|AAAAAAAAKAADBAAA|77833|21|37|13|PM|second|evening|| +77834|AAAAAAAALAADBAAA|77834|21|37|14|PM|second|evening|| +77835|AAAAAAAAMAADBAAA|77835|21|37|15|PM|second|evening|| +77836|AAAAAAAANAADBAAA|77836|21|37|16|PM|second|evening|| +77837|AAAAAAAAOAADBAAA|77837|21|37|17|PM|second|evening|| +77838|AAAAAAAAPAADBAAA|77838|21|37|18|PM|second|evening|| +77839|AAAAAAAAABADBAAA|77839|21|37|19|PM|second|evening|| +77840|AAAAAAAABBADBAAA|77840|21|37|20|PM|second|evening|| +77841|AAAAAAAACBADBAAA|77841|21|37|21|PM|second|evening|| +77842|AAAAAAAADBADBAAA|77842|21|37|22|PM|second|evening|| +77843|AAAAAAAAEBADBAAA|77843|21|37|23|PM|second|evening|| +77844|AAAAAAAAFBADBAAA|77844|21|37|24|PM|second|evening|| +77845|AAAAAAAAGBADBAAA|77845|21|37|25|PM|second|evening|| +77846|AAAAAAAAHBADBAAA|77846|21|37|26|PM|second|evening|| +77847|AAAAAAAAIBADBAAA|77847|21|37|27|PM|second|evening|| +77848|AAAAAAAAJBADBAAA|77848|21|37|28|PM|second|evening|| +77849|AAAAAAAAKBADBAAA|77849|21|37|29|PM|second|evening|| +77850|AAAAAAAALBADBAAA|77850|21|37|30|PM|second|evening|| +77851|AAAAAAAAMBADBAAA|77851|21|37|31|PM|second|evening|| +77852|AAAAAAAANBADBAAA|77852|21|37|32|PM|second|evening|| +77853|AAAAAAAAOBADBAAA|77853|21|37|33|PM|second|evening|| +77854|AAAAAAAAPBADBAAA|77854|21|37|34|PM|second|evening|| +77855|AAAAAAAAACADBAAA|77855|21|37|35|PM|second|evening|| +77856|AAAAAAAABCADBAAA|77856|21|37|36|PM|second|evening|| +77857|AAAAAAAACCADBAAA|77857|21|37|37|PM|second|evening|| +77858|AAAAAAAADCADBAAA|77858|21|37|38|PM|second|evening|| +77859|AAAAAAAAECADBAAA|77859|21|37|39|PM|second|evening|| +77860|AAAAAAAAFCADBAAA|77860|21|37|40|PM|second|evening|| +77861|AAAAAAAAGCADBAAA|77861|21|37|41|PM|second|evening|| +77862|AAAAAAAAHCADBAAA|77862|21|37|42|PM|second|evening|| +77863|AAAAAAAAICADBAAA|77863|21|37|43|PM|second|evening|| +77864|AAAAAAAAJCADBAAA|77864|21|37|44|PM|second|evening|| +77865|AAAAAAAAKCADBAAA|77865|21|37|45|PM|second|evening|| +77866|AAAAAAAALCADBAAA|77866|21|37|46|PM|second|evening|| +77867|AAAAAAAAMCADBAAA|77867|21|37|47|PM|second|evening|| +77868|AAAAAAAANCADBAAA|77868|21|37|48|PM|second|evening|| +77869|AAAAAAAAOCADBAAA|77869|21|37|49|PM|second|evening|| +77870|AAAAAAAAPCADBAAA|77870|21|37|50|PM|second|evening|| +77871|AAAAAAAAADADBAAA|77871|21|37|51|PM|second|evening|| +77872|AAAAAAAABDADBAAA|77872|21|37|52|PM|second|evening|| +77873|AAAAAAAACDADBAAA|77873|21|37|53|PM|second|evening|| +77874|AAAAAAAADDADBAAA|77874|21|37|54|PM|second|evening|| +77875|AAAAAAAAEDADBAAA|77875|21|37|55|PM|second|evening|| +77876|AAAAAAAAFDADBAAA|77876|21|37|56|PM|second|evening|| +77877|AAAAAAAAGDADBAAA|77877|21|37|57|PM|second|evening|| +77878|AAAAAAAAHDADBAAA|77878|21|37|58|PM|second|evening|| +77879|AAAAAAAAIDADBAAA|77879|21|37|59|PM|second|evening|| +77880|AAAAAAAAJDADBAAA|77880|21|38|0|PM|second|evening|| +77881|AAAAAAAAKDADBAAA|77881|21|38|1|PM|second|evening|| +77882|AAAAAAAALDADBAAA|77882|21|38|2|PM|second|evening|| +77883|AAAAAAAAMDADBAAA|77883|21|38|3|PM|second|evening|| +77884|AAAAAAAANDADBAAA|77884|21|38|4|PM|second|evening|| +77885|AAAAAAAAODADBAAA|77885|21|38|5|PM|second|evening|| +77886|AAAAAAAAPDADBAAA|77886|21|38|6|PM|second|evening|| +77887|AAAAAAAAAEADBAAA|77887|21|38|7|PM|second|evening|| +77888|AAAAAAAABEADBAAA|77888|21|38|8|PM|second|evening|| +77889|AAAAAAAACEADBAAA|77889|21|38|9|PM|second|evening|| +77890|AAAAAAAADEADBAAA|77890|21|38|10|PM|second|evening|| +77891|AAAAAAAAEEADBAAA|77891|21|38|11|PM|second|evening|| +77892|AAAAAAAAFEADBAAA|77892|21|38|12|PM|second|evening|| +77893|AAAAAAAAGEADBAAA|77893|21|38|13|PM|second|evening|| +77894|AAAAAAAAHEADBAAA|77894|21|38|14|PM|second|evening|| +77895|AAAAAAAAIEADBAAA|77895|21|38|15|PM|second|evening|| +77896|AAAAAAAAJEADBAAA|77896|21|38|16|PM|second|evening|| +77897|AAAAAAAAKEADBAAA|77897|21|38|17|PM|second|evening|| +77898|AAAAAAAALEADBAAA|77898|21|38|18|PM|second|evening|| +77899|AAAAAAAAMEADBAAA|77899|21|38|19|PM|second|evening|| +77900|AAAAAAAANEADBAAA|77900|21|38|20|PM|second|evening|| +77901|AAAAAAAAOEADBAAA|77901|21|38|21|PM|second|evening|| +77902|AAAAAAAAPEADBAAA|77902|21|38|22|PM|second|evening|| +77903|AAAAAAAAAFADBAAA|77903|21|38|23|PM|second|evening|| +77904|AAAAAAAABFADBAAA|77904|21|38|24|PM|second|evening|| +77905|AAAAAAAACFADBAAA|77905|21|38|25|PM|second|evening|| +77906|AAAAAAAADFADBAAA|77906|21|38|26|PM|second|evening|| +77907|AAAAAAAAEFADBAAA|77907|21|38|27|PM|second|evening|| +77908|AAAAAAAAFFADBAAA|77908|21|38|28|PM|second|evening|| +77909|AAAAAAAAGFADBAAA|77909|21|38|29|PM|second|evening|| +77910|AAAAAAAAHFADBAAA|77910|21|38|30|PM|second|evening|| +77911|AAAAAAAAIFADBAAA|77911|21|38|31|PM|second|evening|| +77912|AAAAAAAAJFADBAAA|77912|21|38|32|PM|second|evening|| +77913|AAAAAAAAKFADBAAA|77913|21|38|33|PM|second|evening|| +77914|AAAAAAAALFADBAAA|77914|21|38|34|PM|second|evening|| +77915|AAAAAAAAMFADBAAA|77915|21|38|35|PM|second|evening|| +77916|AAAAAAAANFADBAAA|77916|21|38|36|PM|second|evening|| +77917|AAAAAAAAOFADBAAA|77917|21|38|37|PM|second|evening|| +77918|AAAAAAAAPFADBAAA|77918|21|38|38|PM|second|evening|| +77919|AAAAAAAAAGADBAAA|77919|21|38|39|PM|second|evening|| +77920|AAAAAAAABGADBAAA|77920|21|38|40|PM|second|evening|| +77921|AAAAAAAACGADBAAA|77921|21|38|41|PM|second|evening|| +77922|AAAAAAAADGADBAAA|77922|21|38|42|PM|second|evening|| +77923|AAAAAAAAEGADBAAA|77923|21|38|43|PM|second|evening|| +77924|AAAAAAAAFGADBAAA|77924|21|38|44|PM|second|evening|| +77925|AAAAAAAAGGADBAAA|77925|21|38|45|PM|second|evening|| +77926|AAAAAAAAHGADBAAA|77926|21|38|46|PM|second|evening|| +77927|AAAAAAAAIGADBAAA|77927|21|38|47|PM|second|evening|| +77928|AAAAAAAAJGADBAAA|77928|21|38|48|PM|second|evening|| +77929|AAAAAAAAKGADBAAA|77929|21|38|49|PM|second|evening|| +77930|AAAAAAAALGADBAAA|77930|21|38|50|PM|second|evening|| +77931|AAAAAAAAMGADBAAA|77931|21|38|51|PM|second|evening|| +77932|AAAAAAAANGADBAAA|77932|21|38|52|PM|second|evening|| +77933|AAAAAAAAOGADBAAA|77933|21|38|53|PM|second|evening|| +77934|AAAAAAAAPGADBAAA|77934|21|38|54|PM|second|evening|| +77935|AAAAAAAAAHADBAAA|77935|21|38|55|PM|second|evening|| +77936|AAAAAAAABHADBAAA|77936|21|38|56|PM|second|evening|| +77937|AAAAAAAACHADBAAA|77937|21|38|57|PM|second|evening|| +77938|AAAAAAAADHADBAAA|77938|21|38|58|PM|second|evening|| +77939|AAAAAAAAEHADBAAA|77939|21|38|59|PM|second|evening|| +77940|AAAAAAAAFHADBAAA|77940|21|39|0|PM|second|evening|| +77941|AAAAAAAAGHADBAAA|77941|21|39|1|PM|second|evening|| +77942|AAAAAAAAHHADBAAA|77942|21|39|2|PM|second|evening|| +77943|AAAAAAAAIHADBAAA|77943|21|39|3|PM|second|evening|| +77944|AAAAAAAAJHADBAAA|77944|21|39|4|PM|second|evening|| +77945|AAAAAAAAKHADBAAA|77945|21|39|5|PM|second|evening|| +77946|AAAAAAAALHADBAAA|77946|21|39|6|PM|second|evening|| +77947|AAAAAAAAMHADBAAA|77947|21|39|7|PM|second|evening|| +77948|AAAAAAAANHADBAAA|77948|21|39|8|PM|second|evening|| +77949|AAAAAAAAOHADBAAA|77949|21|39|9|PM|second|evening|| +77950|AAAAAAAAPHADBAAA|77950|21|39|10|PM|second|evening|| +77951|AAAAAAAAAIADBAAA|77951|21|39|11|PM|second|evening|| +77952|AAAAAAAABIADBAAA|77952|21|39|12|PM|second|evening|| +77953|AAAAAAAACIADBAAA|77953|21|39|13|PM|second|evening|| +77954|AAAAAAAADIADBAAA|77954|21|39|14|PM|second|evening|| +77955|AAAAAAAAEIADBAAA|77955|21|39|15|PM|second|evening|| +77956|AAAAAAAAFIADBAAA|77956|21|39|16|PM|second|evening|| +77957|AAAAAAAAGIADBAAA|77957|21|39|17|PM|second|evening|| +77958|AAAAAAAAHIADBAAA|77958|21|39|18|PM|second|evening|| +77959|AAAAAAAAIIADBAAA|77959|21|39|19|PM|second|evening|| +77960|AAAAAAAAJIADBAAA|77960|21|39|20|PM|second|evening|| +77961|AAAAAAAAKIADBAAA|77961|21|39|21|PM|second|evening|| +77962|AAAAAAAALIADBAAA|77962|21|39|22|PM|second|evening|| +77963|AAAAAAAAMIADBAAA|77963|21|39|23|PM|second|evening|| +77964|AAAAAAAANIADBAAA|77964|21|39|24|PM|second|evening|| +77965|AAAAAAAAOIADBAAA|77965|21|39|25|PM|second|evening|| +77966|AAAAAAAAPIADBAAA|77966|21|39|26|PM|second|evening|| +77967|AAAAAAAAAJADBAAA|77967|21|39|27|PM|second|evening|| +77968|AAAAAAAABJADBAAA|77968|21|39|28|PM|second|evening|| +77969|AAAAAAAACJADBAAA|77969|21|39|29|PM|second|evening|| +77970|AAAAAAAADJADBAAA|77970|21|39|30|PM|second|evening|| +77971|AAAAAAAAEJADBAAA|77971|21|39|31|PM|second|evening|| +77972|AAAAAAAAFJADBAAA|77972|21|39|32|PM|second|evening|| +77973|AAAAAAAAGJADBAAA|77973|21|39|33|PM|second|evening|| +77974|AAAAAAAAHJADBAAA|77974|21|39|34|PM|second|evening|| +77975|AAAAAAAAIJADBAAA|77975|21|39|35|PM|second|evening|| +77976|AAAAAAAAJJADBAAA|77976|21|39|36|PM|second|evening|| +77977|AAAAAAAAKJADBAAA|77977|21|39|37|PM|second|evening|| +77978|AAAAAAAALJADBAAA|77978|21|39|38|PM|second|evening|| +77979|AAAAAAAAMJADBAAA|77979|21|39|39|PM|second|evening|| +77980|AAAAAAAANJADBAAA|77980|21|39|40|PM|second|evening|| +77981|AAAAAAAAOJADBAAA|77981|21|39|41|PM|second|evening|| +77982|AAAAAAAAPJADBAAA|77982|21|39|42|PM|second|evening|| +77983|AAAAAAAAAKADBAAA|77983|21|39|43|PM|second|evening|| +77984|AAAAAAAABKADBAAA|77984|21|39|44|PM|second|evening|| +77985|AAAAAAAACKADBAAA|77985|21|39|45|PM|second|evening|| +77986|AAAAAAAADKADBAAA|77986|21|39|46|PM|second|evening|| +77987|AAAAAAAAEKADBAAA|77987|21|39|47|PM|second|evening|| +77988|AAAAAAAAFKADBAAA|77988|21|39|48|PM|second|evening|| +77989|AAAAAAAAGKADBAAA|77989|21|39|49|PM|second|evening|| +77990|AAAAAAAAHKADBAAA|77990|21|39|50|PM|second|evening|| +77991|AAAAAAAAIKADBAAA|77991|21|39|51|PM|second|evening|| +77992|AAAAAAAAJKADBAAA|77992|21|39|52|PM|second|evening|| +77993|AAAAAAAAKKADBAAA|77993|21|39|53|PM|second|evening|| +77994|AAAAAAAALKADBAAA|77994|21|39|54|PM|second|evening|| +77995|AAAAAAAAMKADBAAA|77995|21|39|55|PM|second|evening|| +77996|AAAAAAAANKADBAAA|77996|21|39|56|PM|second|evening|| +77997|AAAAAAAAOKADBAAA|77997|21|39|57|PM|second|evening|| +77998|AAAAAAAAPKADBAAA|77998|21|39|58|PM|second|evening|| +77999|AAAAAAAAALADBAAA|77999|21|39|59|PM|second|evening|| +78000|AAAAAAAABLADBAAA|78000|21|40|0|PM|second|evening|| +78001|AAAAAAAACLADBAAA|78001|21|40|1|PM|second|evening|| +78002|AAAAAAAADLADBAAA|78002|21|40|2|PM|second|evening|| +78003|AAAAAAAAELADBAAA|78003|21|40|3|PM|second|evening|| +78004|AAAAAAAAFLADBAAA|78004|21|40|4|PM|second|evening|| +78005|AAAAAAAAGLADBAAA|78005|21|40|5|PM|second|evening|| +78006|AAAAAAAAHLADBAAA|78006|21|40|6|PM|second|evening|| +78007|AAAAAAAAILADBAAA|78007|21|40|7|PM|second|evening|| +78008|AAAAAAAAJLADBAAA|78008|21|40|8|PM|second|evening|| +78009|AAAAAAAAKLADBAAA|78009|21|40|9|PM|second|evening|| +78010|AAAAAAAALLADBAAA|78010|21|40|10|PM|second|evening|| +78011|AAAAAAAAMLADBAAA|78011|21|40|11|PM|second|evening|| +78012|AAAAAAAANLADBAAA|78012|21|40|12|PM|second|evening|| +78013|AAAAAAAAOLADBAAA|78013|21|40|13|PM|second|evening|| +78014|AAAAAAAAPLADBAAA|78014|21|40|14|PM|second|evening|| +78015|AAAAAAAAAMADBAAA|78015|21|40|15|PM|second|evening|| +78016|AAAAAAAABMADBAAA|78016|21|40|16|PM|second|evening|| +78017|AAAAAAAACMADBAAA|78017|21|40|17|PM|second|evening|| +78018|AAAAAAAADMADBAAA|78018|21|40|18|PM|second|evening|| +78019|AAAAAAAAEMADBAAA|78019|21|40|19|PM|second|evening|| +78020|AAAAAAAAFMADBAAA|78020|21|40|20|PM|second|evening|| +78021|AAAAAAAAGMADBAAA|78021|21|40|21|PM|second|evening|| +78022|AAAAAAAAHMADBAAA|78022|21|40|22|PM|second|evening|| +78023|AAAAAAAAIMADBAAA|78023|21|40|23|PM|second|evening|| +78024|AAAAAAAAJMADBAAA|78024|21|40|24|PM|second|evening|| +78025|AAAAAAAAKMADBAAA|78025|21|40|25|PM|second|evening|| +78026|AAAAAAAALMADBAAA|78026|21|40|26|PM|second|evening|| +78027|AAAAAAAAMMADBAAA|78027|21|40|27|PM|second|evening|| +78028|AAAAAAAANMADBAAA|78028|21|40|28|PM|second|evening|| +78029|AAAAAAAAOMADBAAA|78029|21|40|29|PM|second|evening|| +78030|AAAAAAAAPMADBAAA|78030|21|40|30|PM|second|evening|| +78031|AAAAAAAAANADBAAA|78031|21|40|31|PM|second|evening|| +78032|AAAAAAAABNADBAAA|78032|21|40|32|PM|second|evening|| +78033|AAAAAAAACNADBAAA|78033|21|40|33|PM|second|evening|| +78034|AAAAAAAADNADBAAA|78034|21|40|34|PM|second|evening|| +78035|AAAAAAAAENADBAAA|78035|21|40|35|PM|second|evening|| +78036|AAAAAAAAFNADBAAA|78036|21|40|36|PM|second|evening|| +78037|AAAAAAAAGNADBAAA|78037|21|40|37|PM|second|evening|| +78038|AAAAAAAAHNADBAAA|78038|21|40|38|PM|second|evening|| +78039|AAAAAAAAINADBAAA|78039|21|40|39|PM|second|evening|| +78040|AAAAAAAAJNADBAAA|78040|21|40|40|PM|second|evening|| +78041|AAAAAAAAKNADBAAA|78041|21|40|41|PM|second|evening|| +78042|AAAAAAAALNADBAAA|78042|21|40|42|PM|second|evening|| +78043|AAAAAAAAMNADBAAA|78043|21|40|43|PM|second|evening|| +78044|AAAAAAAANNADBAAA|78044|21|40|44|PM|second|evening|| +78045|AAAAAAAAONADBAAA|78045|21|40|45|PM|second|evening|| +78046|AAAAAAAAPNADBAAA|78046|21|40|46|PM|second|evening|| +78047|AAAAAAAAAOADBAAA|78047|21|40|47|PM|second|evening|| +78048|AAAAAAAABOADBAAA|78048|21|40|48|PM|second|evening|| +78049|AAAAAAAACOADBAAA|78049|21|40|49|PM|second|evening|| +78050|AAAAAAAADOADBAAA|78050|21|40|50|PM|second|evening|| +78051|AAAAAAAAEOADBAAA|78051|21|40|51|PM|second|evening|| +78052|AAAAAAAAFOADBAAA|78052|21|40|52|PM|second|evening|| +78053|AAAAAAAAGOADBAAA|78053|21|40|53|PM|second|evening|| +78054|AAAAAAAAHOADBAAA|78054|21|40|54|PM|second|evening|| +78055|AAAAAAAAIOADBAAA|78055|21|40|55|PM|second|evening|| +78056|AAAAAAAAJOADBAAA|78056|21|40|56|PM|second|evening|| +78057|AAAAAAAAKOADBAAA|78057|21|40|57|PM|second|evening|| +78058|AAAAAAAALOADBAAA|78058|21|40|58|PM|second|evening|| +78059|AAAAAAAAMOADBAAA|78059|21|40|59|PM|second|evening|| +78060|AAAAAAAANOADBAAA|78060|21|41|0|PM|second|evening|| +78061|AAAAAAAAOOADBAAA|78061|21|41|1|PM|second|evening|| +78062|AAAAAAAAPOADBAAA|78062|21|41|2|PM|second|evening|| +78063|AAAAAAAAAPADBAAA|78063|21|41|3|PM|second|evening|| +78064|AAAAAAAABPADBAAA|78064|21|41|4|PM|second|evening|| +78065|AAAAAAAACPADBAAA|78065|21|41|5|PM|second|evening|| +78066|AAAAAAAADPADBAAA|78066|21|41|6|PM|second|evening|| +78067|AAAAAAAAEPADBAAA|78067|21|41|7|PM|second|evening|| +78068|AAAAAAAAFPADBAAA|78068|21|41|8|PM|second|evening|| +78069|AAAAAAAAGPADBAAA|78069|21|41|9|PM|second|evening|| +78070|AAAAAAAAHPADBAAA|78070|21|41|10|PM|second|evening|| +78071|AAAAAAAAIPADBAAA|78071|21|41|11|PM|second|evening|| +78072|AAAAAAAAJPADBAAA|78072|21|41|12|PM|second|evening|| +78073|AAAAAAAAKPADBAAA|78073|21|41|13|PM|second|evening|| +78074|AAAAAAAALPADBAAA|78074|21|41|14|PM|second|evening|| +78075|AAAAAAAAMPADBAAA|78075|21|41|15|PM|second|evening|| +78076|AAAAAAAANPADBAAA|78076|21|41|16|PM|second|evening|| +78077|AAAAAAAAOPADBAAA|78077|21|41|17|PM|second|evening|| +78078|AAAAAAAAPPADBAAA|78078|21|41|18|PM|second|evening|| +78079|AAAAAAAAAABDBAAA|78079|21|41|19|PM|second|evening|| +78080|AAAAAAAABABDBAAA|78080|21|41|20|PM|second|evening|| +78081|AAAAAAAACABDBAAA|78081|21|41|21|PM|second|evening|| +78082|AAAAAAAADABDBAAA|78082|21|41|22|PM|second|evening|| +78083|AAAAAAAAEABDBAAA|78083|21|41|23|PM|second|evening|| +78084|AAAAAAAAFABDBAAA|78084|21|41|24|PM|second|evening|| +78085|AAAAAAAAGABDBAAA|78085|21|41|25|PM|second|evening|| +78086|AAAAAAAAHABDBAAA|78086|21|41|26|PM|second|evening|| +78087|AAAAAAAAIABDBAAA|78087|21|41|27|PM|second|evening|| +78088|AAAAAAAAJABDBAAA|78088|21|41|28|PM|second|evening|| +78089|AAAAAAAAKABDBAAA|78089|21|41|29|PM|second|evening|| +78090|AAAAAAAALABDBAAA|78090|21|41|30|PM|second|evening|| +78091|AAAAAAAAMABDBAAA|78091|21|41|31|PM|second|evening|| +78092|AAAAAAAANABDBAAA|78092|21|41|32|PM|second|evening|| +78093|AAAAAAAAOABDBAAA|78093|21|41|33|PM|second|evening|| +78094|AAAAAAAAPABDBAAA|78094|21|41|34|PM|second|evening|| +78095|AAAAAAAAABBDBAAA|78095|21|41|35|PM|second|evening|| +78096|AAAAAAAABBBDBAAA|78096|21|41|36|PM|second|evening|| +78097|AAAAAAAACBBDBAAA|78097|21|41|37|PM|second|evening|| +78098|AAAAAAAADBBDBAAA|78098|21|41|38|PM|second|evening|| +78099|AAAAAAAAEBBDBAAA|78099|21|41|39|PM|second|evening|| +78100|AAAAAAAAFBBDBAAA|78100|21|41|40|PM|second|evening|| +78101|AAAAAAAAGBBDBAAA|78101|21|41|41|PM|second|evening|| +78102|AAAAAAAAHBBDBAAA|78102|21|41|42|PM|second|evening|| +78103|AAAAAAAAIBBDBAAA|78103|21|41|43|PM|second|evening|| +78104|AAAAAAAAJBBDBAAA|78104|21|41|44|PM|second|evening|| +78105|AAAAAAAAKBBDBAAA|78105|21|41|45|PM|second|evening|| +78106|AAAAAAAALBBDBAAA|78106|21|41|46|PM|second|evening|| +78107|AAAAAAAAMBBDBAAA|78107|21|41|47|PM|second|evening|| +78108|AAAAAAAANBBDBAAA|78108|21|41|48|PM|second|evening|| +78109|AAAAAAAAOBBDBAAA|78109|21|41|49|PM|second|evening|| +78110|AAAAAAAAPBBDBAAA|78110|21|41|50|PM|second|evening|| +78111|AAAAAAAAACBDBAAA|78111|21|41|51|PM|second|evening|| +78112|AAAAAAAABCBDBAAA|78112|21|41|52|PM|second|evening|| +78113|AAAAAAAACCBDBAAA|78113|21|41|53|PM|second|evening|| +78114|AAAAAAAADCBDBAAA|78114|21|41|54|PM|second|evening|| +78115|AAAAAAAAECBDBAAA|78115|21|41|55|PM|second|evening|| +78116|AAAAAAAAFCBDBAAA|78116|21|41|56|PM|second|evening|| +78117|AAAAAAAAGCBDBAAA|78117|21|41|57|PM|second|evening|| +78118|AAAAAAAAHCBDBAAA|78118|21|41|58|PM|second|evening|| +78119|AAAAAAAAICBDBAAA|78119|21|41|59|PM|second|evening|| +78120|AAAAAAAAJCBDBAAA|78120|21|42|0|PM|second|evening|| +78121|AAAAAAAAKCBDBAAA|78121|21|42|1|PM|second|evening|| +78122|AAAAAAAALCBDBAAA|78122|21|42|2|PM|second|evening|| +78123|AAAAAAAAMCBDBAAA|78123|21|42|3|PM|second|evening|| +78124|AAAAAAAANCBDBAAA|78124|21|42|4|PM|second|evening|| +78125|AAAAAAAAOCBDBAAA|78125|21|42|5|PM|second|evening|| +78126|AAAAAAAAPCBDBAAA|78126|21|42|6|PM|second|evening|| +78127|AAAAAAAAADBDBAAA|78127|21|42|7|PM|second|evening|| +78128|AAAAAAAABDBDBAAA|78128|21|42|8|PM|second|evening|| +78129|AAAAAAAACDBDBAAA|78129|21|42|9|PM|second|evening|| +78130|AAAAAAAADDBDBAAA|78130|21|42|10|PM|second|evening|| +78131|AAAAAAAAEDBDBAAA|78131|21|42|11|PM|second|evening|| +78132|AAAAAAAAFDBDBAAA|78132|21|42|12|PM|second|evening|| +78133|AAAAAAAAGDBDBAAA|78133|21|42|13|PM|second|evening|| +78134|AAAAAAAAHDBDBAAA|78134|21|42|14|PM|second|evening|| +78135|AAAAAAAAIDBDBAAA|78135|21|42|15|PM|second|evening|| +78136|AAAAAAAAJDBDBAAA|78136|21|42|16|PM|second|evening|| +78137|AAAAAAAAKDBDBAAA|78137|21|42|17|PM|second|evening|| +78138|AAAAAAAALDBDBAAA|78138|21|42|18|PM|second|evening|| +78139|AAAAAAAAMDBDBAAA|78139|21|42|19|PM|second|evening|| +78140|AAAAAAAANDBDBAAA|78140|21|42|20|PM|second|evening|| +78141|AAAAAAAAODBDBAAA|78141|21|42|21|PM|second|evening|| +78142|AAAAAAAAPDBDBAAA|78142|21|42|22|PM|second|evening|| +78143|AAAAAAAAAEBDBAAA|78143|21|42|23|PM|second|evening|| +78144|AAAAAAAABEBDBAAA|78144|21|42|24|PM|second|evening|| +78145|AAAAAAAACEBDBAAA|78145|21|42|25|PM|second|evening|| +78146|AAAAAAAADEBDBAAA|78146|21|42|26|PM|second|evening|| +78147|AAAAAAAAEEBDBAAA|78147|21|42|27|PM|second|evening|| +78148|AAAAAAAAFEBDBAAA|78148|21|42|28|PM|second|evening|| +78149|AAAAAAAAGEBDBAAA|78149|21|42|29|PM|second|evening|| +78150|AAAAAAAAHEBDBAAA|78150|21|42|30|PM|second|evening|| +78151|AAAAAAAAIEBDBAAA|78151|21|42|31|PM|second|evening|| +78152|AAAAAAAAJEBDBAAA|78152|21|42|32|PM|second|evening|| +78153|AAAAAAAAKEBDBAAA|78153|21|42|33|PM|second|evening|| +78154|AAAAAAAALEBDBAAA|78154|21|42|34|PM|second|evening|| +78155|AAAAAAAAMEBDBAAA|78155|21|42|35|PM|second|evening|| +78156|AAAAAAAANEBDBAAA|78156|21|42|36|PM|second|evening|| +78157|AAAAAAAAOEBDBAAA|78157|21|42|37|PM|second|evening|| +78158|AAAAAAAAPEBDBAAA|78158|21|42|38|PM|second|evening|| +78159|AAAAAAAAAFBDBAAA|78159|21|42|39|PM|second|evening|| +78160|AAAAAAAABFBDBAAA|78160|21|42|40|PM|second|evening|| +78161|AAAAAAAACFBDBAAA|78161|21|42|41|PM|second|evening|| +78162|AAAAAAAADFBDBAAA|78162|21|42|42|PM|second|evening|| +78163|AAAAAAAAEFBDBAAA|78163|21|42|43|PM|second|evening|| +78164|AAAAAAAAFFBDBAAA|78164|21|42|44|PM|second|evening|| +78165|AAAAAAAAGFBDBAAA|78165|21|42|45|PM|second|evening|| +78166|AAAAAAAAHFBDBAAA|78166|21|42|46|PM|second|evening|| +78167|AAAAAAAAIFBDBAAA|78167|21|42|47|PM|second|evening|| +78168|AAAAAAAAJFBDBAAA|78168|21|42|48|PM|second|evening|| +78169|AAAAAAAAKFBDBAAA|78169|21|42|49|PM|second|evening|| +78170|AAAAAAAALFBDBAAA|78170|21|42|50|PM|second|evening|| +78171|AAAAAAAAMFBDBAAA|78171|21|42|51|PM|second|evening|| +78172|AAAAAAAANFBDBAAA|78172|21|42|52|PM|second|evening|| +78173|AAAAAAAAOFBDBAAA|78173|21|42|53|PM|second|evening|| +78174|AAAAAAAAPFBDBAAA|78174|21|42|54|PM|second|evening|| +78175|AAAAAAAAAGBDBAAA|78175|21|42|55|PM|second|evening|| +78176|AAAAAAAABGBDBAAA|78176|21|42|56|PM|second|evening|| +78177|AAAAAAAACGBDBAAA|78177|21|42|57|PM|second|evening|| +78178|AAAAAAAADGBDBAAA|78178|21|42|58|PM|second|evening|| +78179|AAAAAAAAEGBDBAAA|78179|21|42|59|PM|second|evening|| +78180|AAAAAAAAFGBDBAAA|78180|21|43|0|PM|second|evening|| +78181|AAAAAAAAGGBDBAAA|78181|21|43|1|PM|second|evening|| +78182|AAAAAAAAHGBDBAAA|78182|21|43|2|PM|second|evening|| +78183|AAAAAAAAIGBDBAAA|78183|21|43|3|PM|second|evening|| +78184|AAAAAAAAJGBDBAAA|78184|21|43|4|PM|second|evening|| +78185|AAAAAAAAKGBDBAAA|78185|21|43|5|PM|second|evening|| +78186|AAAAAAAALGBDBAAA|78186|21|43|6|PM|second|evening|| +78187|AAAAAAAAMGBDBAAA|78187|21|43|7|PM|second|evening|| +78188|AAAAAAAANGBDBAAA|78188|21|43|8|PM|second|evening|| +78189|AAAAAAAAOGBDBAAA|78189|21|43|9|PM|second|evening|| +78190|AAAAAAAAPGBDBAAA|78190|21|43|10|PM|second|evening|| +78191|AAAAAAAAAHBDBAAA|78191|21|43|11|PM|second|evening|| +78192|AAAAAAAABHBDBAAA|78192|21|43|12|PM|second|evening|| +78193|AAAAAAAACHBDBAAA|78193|21|43|13|PM|second|evening|| +78194|AAAAAAAADHBDBAAA|78194|21|43|14|PM|second|evening|| +78195|AAAAAAAAEHBDBAAA|78195|21|43|15|PM|second|evening|| +78196|AAAAAAAAFHBDBAAA|78196|21|43|16|PM|second|evening|| +78197|AAAAAAAAGHBDBAAA|78197|21|43|17|PM|second|evening|| +78198|AAAAAAAAHHBDBAAA|78198|21|43|18|PM|second|evening|| +78199|AAAAAAAAIHBDBAAA|78199|21|43|19|PM|second|evening|| +78200|AAAAAAAAJHBDBAAA|78200|21|43|20|PM|second|evening|| +78201|AAAAAAAAKHBDBAAA|78201|21|43|21|PM|second|evening|| +78202|AAAAAAAALHBDBAAA|78202|21|43|22|PM|second|evening|| +78203|AAAAAAAAMHBDBAAA|78203|21|43|23|PM|second|evening|| +78204|AAAAAAAANHBDBAAA|78204|21|43|24|PM|second|evening|| +78205|AAAAAAAAOHBDBAAA|78205|21|43|25|PM|second|evening|| +78206|AAAAAAAAPHBDBAAA|78206|21|43|26|PM|second|evening|| +78207|AAAAAAAAAIBDBAAA|78207|21|43|27|PM|second|evening|| +78208|AAAAAAAABIBDBAAA|78208|21|43|28|PM|second|evening|| +78209|AAAAAAAACIBDBAAA|78209|21|43|29|PM|second|evening|| +78210|AAAAAAAADIBDBAAA|78210|21|43|30|PM|second|evening|| +78211|AAAAAAAAEIBDBAAA|78211|21|43|31|PM|second|evening|| +78212|AAAAAAAAFIBDBAAA|78212|21|43|32|PM|second|evening|| +78213|AAAAAAAAGIBDBAAA|78213|21|43|33|PM|second|evening|| +78214|AAAAAAAAHIBDBAAA|78214|21|43|34|PM|second|evening|| +78215|AAAAAAAAIIBDBAAA|78215|21|43|35|PM|second|evening|| +78216|AAAAAAAAJIBDBAAA|78216|21|43|36|PM|second|evening|| +78217|AAAAAAAAKIBDBAAA|78217|21|43|37|PM|second|evening|| +78218|AAAAAAAALIBDBAAA|78218|21|43|38|PM|second|evening|| +78219|AAAAAAAAMIBDBAAA|78219|21|43|39|PM|second|evening|| +78220|AAAAAAAANIBDBAAA|78220|21|43|40|PM|second|evening|| +78221|AAAAAAAAOIBDBAAA|78221|21|43|41|PM|second|evening|| +78222|AAAAAAAAPIBDBAAA|78222|21|43|42|PM|second|evening|| +78223|AAAAAAAAAJBDBAAA|78223|21|43|43|PM|second|evening|| +78224|AAAAAAAABJBDBAAA|78224|21|43|44|PM|second|evening|| +78225|AAAAAAAACJBDBAAA|78225|21|43|45|PM|second|evening|| +78226|AAAAAAAADJBDBAAA|78226|21|43|46|PM|second|evening|| +78227|AAAAAAAAEJBDBAAA|78227|21|43|47|PM|second|evening|| +78228|AAAAAAAAFJBDBAAA|78228|21|43|48|PM|second|evening|| +78229|AAAAAAAAGJBDBAAA|78229|21|43|49|PM|second|evening|| +78230|AAAAAAAAHJBDBAAA|78230|21|43|50|PM|second|evening|| +78231|AAAAAAAAIJBDBAAA|78231|21|43|51|PM|second|evening|| +78232|AAAAAAAAJJBDBAAA|78232|21|43|52|PM|second|evening|| +78233|AAAAAAAAKJBDBAAA|78233|21|43|53|PM|second|evening|| +78234|AAAAAAAALJBDBAAA|78234|21|43|54|PM|second|evening|| +78235|AAAAAAAAMJBDBAAA|78235|21|43|55|PM|second|evening|| +78236|AAAAAAAANJBDBAAA|78236|21|43|56|PM|second|evening|| +78237|AAAAAAAAOJBDBAAA|78237|21|43|57|PM|second|evening|| +78238|AAAAAAAAPJBDBAAA|78238|21|43|58|PM|second|evening|| +78239|AAAAAAAAAKBDBAAA|78239|21|43|59|PM|second|evening|| +78240|AAAAAAAABKBDBAAA|78240|21|44|0|PM|second|evening|| +78241|AAAAAAAACKBDBAAA|78241|21|44|1|PM|second|evening|| +78242|AAAAAAAADKBDBAAA|78242|21|44|2|PM|second|evening|| +78243|AAAAAAAAEKBDBAAA|78243|21|44|3|PM|second|evening|| +78244|AAAAAAAAFKBDBAAA|78244|21|44|4|PM|second|evening|| +78245|AAAAAAAAGKBDBAAA|78245|21|44|5|PM|second|evening|| +78246|AAAAAAAAHKBDBAAA|78246|21|44|6|PM|second|evening|| +78247|AAAAAAAAIKBDBAAA|78247|21|44|7|PM|second|evening|| +78248|AAAAAAAAJKBDBAAA|78248|21|44|8|PM|second|evening|| +78249|AAAAAAAAKKBDBAAA|78249|21|44|9|PM|second|evening|| +78250|AAAAAAAALKBDBAAA|78250|21|44|10|PM|second|evening|| +78251|AAAAAAAAMKBDBAAA|78251|21|44|11|PM|second|evening|| +78252|AAAAAAAANKBDBAAA|78252|21|44|12|PM|second|evening|| +78253|AAAAAAAAOKBDBAAA|78253|21|44|13|PM|second|evening|| +78254|AAAAAAAAPKBDBAAA|78254|21|44|14|PM|second|evening|| +78255|AAAAAAAAALBDBAAA|78255|21|44|15|PM|second|evening|| +78256|AAAAAAAABLBDBAAA|78256|21|44|16|PM|second|evening|| +78257|AAAAAAAACLBDBAAA|78257|21|44|17|PM|second|evening|| +78258|AAAAAAAADLBDBAAA|78258|21|44|18|PM|second|evening|| +78259|AAAAAAAAELBDBAAA|78259|21|44|19|PM|second|evening|| +78260|AAAAAAAAFLBDBAAA|78260|21|44|20|PM|second|evening|| +78261|AAAAAAAAGLBDBAAA|78261|21|44|21|PM|second|evening|| +78262|AAAAAAAAHLBDBAAA|78262|21|44|22|PM|second|evening|| +78263|AAAAAAAAILBDBAAA|78263|21|44|23|PM|second|evening|| +78264|AAAAAAAAJLBDBAAA|78264|21|44|24|PM|second|evening|| +78265|AAAAAAAAKLBDBAAA|78265|21|44|25|PM|second|evening|| +78266|AAAAAAAALLBDBAAA|78266|21|44|26|PM|second|evening|| +78267|AAAAAAAAMLBDBAAA|78267|21|44|27|PM|second|evening|| +78268|AAAAAAAANLBDBAAA|78268|21|44|28|PM|second|evening|| +78269|AAAAAAAAOLBDBAAA|78269|21|44|29|PM|second|evening|| +78270|AAAAAAAAPLBDBAAA|78270|21|44|30|PM|second|evening|| +78271|AAAAAAAAAMBDBAAA|78271|21|44|31|PM|second|evening|| +78272|AAAAAAAABMBDBAAA|78272|21|44|32|PM|second|evening|| +78273|AAAAAAAACMBDBAAA|78273|21|44|33|PM|second|evening|| +78274|AAAAAAAADMBDBAAA|78274|21|44|34|PM|second|evening|| +78275|AAAAAAAAEMBDBAAA|78275|21|44|35|PM|second|evening|| +78276|AAAAAAAAFMBDBAAA|78276|21|44|36|PM|second|evening|| +78277|AAAAAAAAGMBDBAAA|78277|21|44|37|PM|second|evening|| +78278|AAAAAAAAHMBDBAAA|78278|21|44|38|PM|second|evening|| +78279|AAAAAAAAIMBDBAAA|78279|21|44|39|PM|second|evening|| +78280|AAAAAAAAJMBDBAAA|78280|21|44|40|PM|second|evening|| +78281|AAAAAAAAKMBDBAAA|78281|21|44|41|PM|second|evening|| +78282|AAAAAAAALMBDBAAA|78282|21|44|42|PM|second|evening|| +78283|AAAAAAAAMMBDBAAA|78283|21|44|43|PM|second|evening|| +78284|AAAAAAAANMBDBAAA|78284|21|44|44|PM|second|evening|| +78285|AAAAAAAAOMBDBAAA|78285|21|44|45|PM|second|evening|| +78286|AAAAAAAAPMBDBAAA|78286|21|44|46|PM|second|evening|| +78287|AAAAAAAAANBDBAAA|78287|21|44|47|PM|second|evening|| +78288|AAAAAAAABNBDBAAA|78288|21|44|48|PM|second|evening|| +78289|AAAAAAAACNBDBAAA|78289|21|44|49|PM|second|evening|| +78290|AAAAAAAADNBDBAAA|78290|21|44|50|PM|second|evening|| +78291|AAAAAAAAENBDBAAA|78291|21|44|51|PM|second|evening|| +78292|AAAAAAAAFNBDBAAA|78292|21|44|52|PM|second|evening|| +78293|AAAAAAAAGNBDBAAA|78293|21|44|53|PM|second|evening|| +78294|AAAAAAAAHNBDBAAA|78294|21|44|54|PM|second|evening|| +78295|AAAAAAAAINBDBAAA|78295|21|44|55|PM|second|evening|| +78296|AAAAAAAAJNBDBAAA|78296|21|44|56|PM|second|evening|| +78297|AAAAAAAAKNBDBAAA|78297|21|44|57|PM|second|evening|| +78298|AAAAAAAALNBDBAAA|78298|21|44|58|PM|second|evening|| +78299|AAAAAAAAMNBDBAAA|78299|21|44|59|PM|second|evening|| +78300|AAAAAAAANNBDBAAA|78300|21|45|0|PM|second|evening|| +78301|AAAAAAAAONBDBAAA|78301|21|45|1|PM|second|evening|| +78302|AAAAAAAAPNBDBAAA|78302|21|45|2|PM|second|evening|| +78303|AAAAAAAAAOBDBAAA|78303|21|45|3|PM|second|evening|| +78304|AAAAAAAABOBDBAAA|78304|21|45|4|PM|second|evening|| +78305|AAAAAAAACOBDBAAA|78305|21|45|5|PM|second|evening|| +78306|AAAAAAAADOBDBAAA|78306|21|45|6|PM|second|evening|| +78307|AAAAAAAAEOBDBAAA|78307|21|45|7|PM|second|evening|| +78308|AAAAAAAAFOBDBAAA|78308|21|45|8|PM|second|evening|| +78309|AAAAAAAAGOBDBAAA|78309|21|45|9|PM|second|evening|| +78310|AAAAAAAAHOBDBAAA|78310|21|45|10|PM|second|evening|| +78311|AAAAAAAAIOBDBAAA|78311|21|45|11|PM|second|evening|| +78312|AAAAAAAAJOBDBAAA|78312|21|45|12|PM|second|evening|| +78313|AAAAAAAAKOBDBAAA|78313|21|45|13|PM|second|evening|| +78314|AAAAAAAALOBDBAAA|78314|21|45|14|PM|second|evening|| +78315|AAAAAAAAMOBDBAAA|78315|21|45|15|PM|second|evening|| +78316|AAAAAAAANOBDBAAA|78316|21|45|16|PM|second|evening|| +78317|AAAAAAAAOOBDBAAA|78317|21|45|17|PM|second|evening|| +78318|AAAAAAAAPOBDBAAA|78318|21|45|18|PM|second|evening|| +78319|AAAAAAAAAPBDBAAA|78319|21|45|19|PM|second|evening|| +78320|AAAAAAAABPBDBAAA|78320|21|45|20|PM|second|evening|| +78321|AAAAAAAACPBDBAAA|78321|21|45|21|PM|second|evening|| +78322|AAAAAAAADPBDBAAA|78322|21|45|22|PM|second|evening|| +78323|AAAAAAAAEPBDBAAA|78323|21|45|23|PM|second|evening|| +78324|AAAAAAAAFPBDBAAA|78324|21|45|24|PM|second|evening|| +78325|AAAAAAAAGPBDBAAA|78325|21|45|25|PM|second|evening|| +78326|AAAAAAAAHPBDBAAA|78326|21|45|26|PM|second|evening|| +78327|AAAAAAAAIPBDBAAA|78327|21|45|27|PM|second|evening|| +78328|AAAAAAAAJPBDBAAA|78328|21|45|28|PM|second|evening|| +78329|AAAAAAAAKPBDBAAA|78329|21|45|29|PM|second|evening|| +78330|AAAAAAAALPBDBAAA|78330|21|45|30|PM|second|evening|| +78331|AAAAAAAAMPBDBAAA|78331|21|45|31|PM|second|evening|| +78332|AAAAAAAANPBDBAAA|78332|21|45|32|PM|second|evening|| +78333|AAAAAAAAOPBDBAAA|78333|21|45|33|PM|second|evening|| +78334|AAAAAAAAPPBDBAAA|78334|21|45|34|PM|second|evening|| +78335|AAAAAAAAAACDBAAA|78335|21|45|35|PM|second|evening|| +78336|AAAAAAAABACDBAAA|78336|21|45|36|PM|second|evening|| +78337|AAAAAAAACACDBAAA|78337|21|45|37|PM|second|evening|| +78338|AAAAAAAADACDBAAA|78338|21|45|38|PM|second|evening|| +78339|AAAAAAAAEACDBAAA|78339|21|45|39|PM|second|evening|| +78340|AAAAAAAAFACDBAAA|78340|21|45|40|PM|second|evening|| +78341|AAAAAAAAGACDBAAA|78341|21|45|41|PM|second|evening|| +78342|AAAAAAAAHACDBAAA|78342|21|45|42|PM|second|evening|| +78343|AAAAAAAAIACDBAAA|78343|21|45|43|PM|second|evening|| +78344|AAAAAAAAJACDBAAA|78344|21|45|44|PM|second|evening|| +78345|AAAAAAAAKACDBAAA|78345|21|45|45|PM|second|evening|| +78346|AAAAAAAALACDBAAA|78346|21|45|46|PM|second|evening|| +78347|AAAAAAAAMACDBAAA|78347|21|45|47|PM|second|evening|| +78348|AAAAAAAANACDBAAA|78348|21|45|48|PM|second|evening|| +78349|AAAAAAAAOACDBAAA|78349|21|45|49|PM|second|evening|| +78350|AAAAAAAAPACDBAAA|78350|21|45|50|PM|second|evening|| +78351|AAAAAAAAABCDBAAA|78351|21|45|51|PM|second|evening|| +78352|AAAAAAAABBCDBAAA|78352|21|45|52|PM|second|evening|| +78353|AAAAAAAACBCDBAAA|78353|21|45|53|PM|second|evening|| +78354|AAAAAAAADBCDBAAA|78354|21|45|54|PM|second|evening|| +78355|AAAAAAAAEBCDBAAA|78355|21|45|55|PM|second|evening|| +78356|AAAAAAAAFBCDBAAA|78356|21|45|56|PM|second|evening|| +78357|AAAAAAAAGBCDBAAA|78357|21|45|57|PM|second|evening|| +78358|AAAAAAAAHBCDBAAA|78358|21|45|58|PM|second|evening|| +78359|AAAAAAAAIBCDBAAA|78359|21|45|59|PM|second|evening|| +78360|AAAAAAAAJBCDBAAA|78360|21|46|0|PM|second|evening|| +78361|AAAAAAAAKBCDBAAA|78361|21|46|1|PM|second|evening|| +78362|AAAAAAAALBCDBAAA|78362|21|46|2|PM|second|evening|| +78363|AAAAAAAAMBCDBAAA|78363|21|46|3|PM|second|evening|| +78364|AAAAAAAANBCDBAAA|78364|21|46|4|PM|second|evening|| +78365|AAAAAAAAOBCDBAAA|78365|21|46|5|PM|second|evening|| +78366|AAAAAAAAPBCDBAAA|78366|21|46|6|PM|second|evening|| +78367|AAAAAAAAACCDBAAA|78367|21|46|7|PM|second|evening|| +78368|AAAAAAAABCCDBAAA|78368|21|46|8|PM|second|evening|| +78369|AAAAAAAACCCDBAAA|78369|21|46|9|PM|second|evening|| +78370|AAAAAAAADCCDBAAA|78370|21|46|10|PM|second|evening|| +78371|AAAAAAAAECCDBAAA|78371|21|46|11|PM|second|evening|| +78372|AAAAAAAAFCCDBAAA|78372|21|46|12|PM|second|evening|| +78373|AAAAAAAAGCCDBAAA|78373|21|46|13|PM|second|evening|| +78374|AAAAAAAAHCCDBAAA|78374|21|46|14|PM|second|evening|| +78375|AAAAAAAAICCDBAAA|78375|21|46|15|PM|second|evening|| +78376|AAAAAAAAJCCDBAAA|78376|21|46|16|PM|second|evening|| +78377|AAAAAAAAKCCDBAAA|78377|21|46|17|PM|second|evening|| +78378|AAAAAAAALCCDBAAA|78378|21|46|18|PM|second|evening|| +78379|AAAAAAAAMCCDBAAA|78379|21|46|19|PM|second|evening|| +78380|AAAAAAAANCCDBAAA|78380|21|46|20|PM|second|evening|| +78381|AAAAAAAAOCCDBAAA|78381|21|46|21|PM|second|evening|| +78382|AAAAAAAAPCCDBAAA|78382|21|46|22|PM|second|evening|| +78383|AAAAAAAAADCDBAAA|78383|21|46|23|PM|second|evening|| +78384|AAAAAAAABDCDBAAA|78384|21|46|24|PM|second|evening|| +78385|AAAAAAAACDCDBAAA|78385|21|46|25|PM|second|evening|| +78386|AAAAAAAADDCDBAAA|78386|21|46|26|PM|second|evening|| +78387|AAAAAAAAEDCDBAAA|78387|21|46|27|PM|second|evening|| +78388|AAAAAAAAFDCDBAAA|78388|21|46|28|PM|second|evening|| +78389|AAAAAAAAGDCDBAAA|78389|21|46|29|PM|second|evening|| +78390|AAAAAAAAHDCDBAAA|78390|21|46|30|PM|second|evening|| +78391|AAAAAAAAIDCDBAAA|78391|21|46|31|PM|second|evening|| +78392|AAAAAAAAJDCDBAAA|78392|21|46|32|PM|second|evening|| +78393|AAAAAAAAKDCDBAAA|78393|21|46|33|PM|second|evening|| +78394|AAAAAAAALDCDBAAA|78394|21|46|34|PM|second|evening|| +78395|AAAAAAAAMDCDBAAA|78395|21|46|35|PM|second|evening|| +78396|AAAAAAAANDCDBAAA|78396|21|46|36|PM|second|evening|| +78397|AAAAAAAAODCDBAAA|78397|21|46|37|PM|second|evening|| +78398|AAAAAAAAPDCDBAAA|78398|21|46|38|PM|second|evening|| +78399|AAAAAAAAAECDBAAA|78399|21|46|39|PM|second|evening|| +78400|AAAAAAAABECDBAAA|78400|21|46|40|PM|second|evening|| +78401|AAAAAAAACECDBAAA|78401|21|46|41|PM|second|evening|| +78402|AAAAAAAADECDBAAA|78402|21|46|42|PM|second|evening|| +78403|AAAAAAAAEECDBAAA|78403|21|46|43|PM|second|evening|| +78404|AAAAAAAAFECDBAAA|78404|21|46|44|PM|second|evening|| +78405|AAAAAAAAGECDBAAA|78405|21|46|45|PM|second|evening|| +78406|AAAAAAAAHECDBAAA|78406|21|46|46|PM|second|evening|| +78407|AAAAAAAAIECDBAAA|78407|21|46|47|PM|second|evening|| +78408|AAAAAAAAJECDBAAA|78408|21|46|48|PM|second|evening|| +78409|AAAAAAAAKECDBAAA|78409|21|46|49|PM|second|evening|| +78410|AAAAAAAALECDBAAA|78410|21|46|50|PM|second|evening|| +78411|AAAAAAAAMECDBAAA|78411|21|46|51|PM|second|evening|| +78412|AAAAAAAANECDBAAA|78412|21|46|52|PM|second|evening|| +78413|AAAAAAAAOECDBAAA|78413|21|46|53|PM|second|evening|| +78414|AAAAAAAAPECDBAAA|78414|21|46|54|PM|second|evening|| +78415|AAAAAAAAAFCDBAAA|78415|21|46|55|PM|second|evening|| +78416|AAAAAAAABFCDBAAA|78416|21|46|56|PM|second|evening|| +78417|AAAAAAAACFCDBAAA|78417|21|46|57|PM|second|evening|| +78418|AAAAAAAADFCDBAAA|78418|21|46|58|PM|second|evening|| +78419|AAAAAAAAEFCDBAAA|78419|21|46|59|PM|second|evening|| +78420|AAAAAAAAFFCDBAAA|78420|21|47|0|PM|second|evening|| +78421|AAAAAAAAGFCDBAAA|78421|21|47|1|PM|second|evening|| +78422|AAAAAAAAHFCDBAAA|78422|21|47|2|PM|second|evening|| +78423|AAAAAAAAIFCDBAAA|78423|21|47|3|PM|second|evening|| +78424|AAAAAAAAJFCDBAAA|78424|21|47|4|PM|second|evening|| +78425|AAAAAAAAKFCDBAAA|78425|21|47|5|PM|second|evening|| +78426|AAAAAAAALFCDBAAA|78426|21|47|6|PM|second|evening|| +78427|AAAAAAAAMFCDBAAA|78427|21|47|7|PM|second|evening|| +78428|AAAAAAAANFCDBAAA|78428|21|47|8|PM|second|evening|| +78429|AAAAAAAAOFCDBAAA|78429|21|47|9|PM|second|evening|| +78430|AAAAAAAAPFCDBAAA|78430|21|47|10|PM|second|evening|| +78431|AAAAAAAAAGCDBAAA|78431|21|47|11|PM|second|evening|| +78432|AAAAAAAABGCDBAAA|78432|21|47|12|PM|second|evening|| +78433|AAAAAAAACGCDBAAA|78433|21|47|13|PM|second|evening|| +78434|AAAAAAAADGCDBAAA|78434|21|47|14|PM|second|evening|| +78435|AAAAAAAAEGCDBAAA|78435|21|47|15|PM|second|evening|| +78436|AAAAAAAAFGCDBAAA|78436|21|47|16|PM|second|evening|| +78437|AAAAAAAAGGCDBAAA|78437|21|47|17|PM|second|evening|| +78438|AAAAAAAAHGCDBAAA|78438|21|47|18|PM|second|evening|| +78439|AAAAAAAAIGCDBAAA|78439|21|47|19|PM|second|evening|| +78440|AAAAAAAAJGCDBAAA|78440|21|47|20|PM|second|evening|| +78441|AAAAAAAAKGCDBAAA|78441|21|47|21|PM|second|evening|| +78442|AAAAAAAALGCDBAAA|78442|21|47|22|PM|second|evening|| +78443|AAAAAAAAMGCDBAAA|78443|21|47|23|PM|second|evening|| +78444|AAAAAAAANGCDBAAA|78444|21|47|24|PM|second|evening|| +78445|AAAAAAAAOGCDBAAA|78445|21|47|25|PM|second|evening|| +78446|AAAAAAAAPGCDBAAA|78446|21|47|26|PM|second|evening|| +78447|AAAAAAAAAHCDBAAA|78447|21|47|27|PM|second|evening|| +78448|AAAAAAAABHCDBAAA|78448|21|47|28|PM|second|evening|| +78449|AAAAAAAACHCDBAAA|78449|21|47|29|PM|second|evening|| +78450|AAAAAAAADHCDBAAA|78450|21|47|30|PM|second|evening|| +78451|AAAAAAAAEHCDBAAA|78451|21|47|31|PM|second|evening|| +78452|AAAAAAAAFHCDBAAA|78452|21|47|32|PM|second|evening|| +78453|AAAAAAAAGHCDBAAA|78453|21|47|33|PM|second|evening|| +78454|AAAAAAAAHHCDBAAA|78454|21|47|34|PM|second|evening|| +78455|AAAAAAAAIHCDBAAA|78455|21|47|35|PM|second|evening|| +78456|AAAAAAAAJHCDBAAA|78456|21|47|36|PM|second|evening|| +78457|AAAAAAAAKHCDBAAA|78457|21|47|37|PM|second|evening|| +78458|AAAAAAAALHCDBAAA|78458|21|47|38|PM|second|evening|| +78459|AAAAAAAAMHCDBAAA|78459|21|47|39|PM|second|evening|| +78460|AAAAAAAANHCDBAAA|78460|21|47|40|PM|second|evening|| +78461|AAAAAAAAOHCDBAAA|78461|21|47|41|PM|second|evening|| +78462|AAAAAAAAPHCDBAAA|78462|21|47|42|PM|second|evening|| +78463|AAAAAAAAAICDBAAA|78463|21|47|43|PM|second|evening|| +78464|AAAAAAAABICDBAAA|78464|21|47|44|PM|second|evening|| +78465|AAAAAAAACICDBAAA|78465|21|47|45|PM|second|evening|| +78466|AAAAAAAADICDBAAA|78466|21|47|46|PM|second|evening|| +78467|AAAAAAAAEICDBAAA|78467|21|47|47|PM|second|evening|| +78468|AAAAAAAAFICDBAAA|78468|21|47|48|PM|second|evening|| +78469|AAAAAAAAGICDBAAA|78469|21|47|49|PM|second|evening|| +78470|AAAAAAAAHICDBAAA|78470|21|47|50|PM|second|evening|| +78471|AAAAAAAAIICDBAAA|78471|21|47|51|PM|second|evening|| +78472|AAAAAAAAJICDBAAA|78472|21|47|52|PM|second|evening|| +78473|AAAAAAAAKICDBAAA|78473|21|47|53|PM|second|evening|| +78474|AAAAAAAALICDBAAA|78474|21|47|54|PM|second|evening|| +78475|AAAAAAAAMICDBAAA|78475|21|47|55|PM|second|evening|| +78476|AAAAAAAANICDBAAA|78476|21|47|56|PM|second|evening|| +78477|AAAAAAAAOICDBAAA|78477|21|47|57|PM|second|evening|| +78478|AAAAAAAAPICDBAAA|78478|21|47|58|PM|second|evening|| +78479|AAAAAAAAAJCDBAAA|78479|21|47|59|PM|second|evening|| +78480|AAAAAAAABJCDBAAA|78480|21|48|0|PM|second|evening|| +78481|AAAAAAAACJCDBAAA|78481|21|48|1|PM|second|evening|| +78482|AAAAAAAADJCDBAAA|78482|21|48|2|PM|second|evening|| +78483|AAAAAAAAEJCDBAAA|78483|21|48|3|PM|second|evening|| +78484|AAAAAAAAFJCDBAAA|78484|21|48|4|PM|second|evening|| +78485|AAAAAAAAGJCDBAAA|78485|21|48|5|PM|second|evening|| +78486|AAAAAAAAHJCDBAAA|78486|21|48|6|PM|second|evening|| +78487|AAAAAAAAIJCDBAAA|78487|21|48|7|PM|second|evening|| +78488|AAAAAAAAJJCDBAAA|78488|21|48|8|PM|second|evening|| +78489|AAAAAAAAKJCDBAAA|78489|21|48|9|PM|second|evening|| +78490|AAAAAAAALJCDBAAA|78490|21|48|10|PM|second|evening|| +78491|AAAAAAAAMJCDBAAA|78491|21|48|11|PM|second|evening|| +78492|AAAAAAAANJCDBAAA|78492|21|48|12|PM|second|evening|| +78493|AAAAAAAAOJCDBAAA|78493|21|48|13|PM|second|evening|| +78494|AAAAAAAAPJCDBAAA|78494|21|48|14|PM|second|evening|| +78495|AAAAAAAAAKCDBAAA|78495|21|48|15|PM|second|evening|| +78496|AAAAAAAABKCDBAAA|78496|21|48|16|PM|second|evening|| +78497|AAAAAAAACKCDBAAA|78497|21|48|17|PM|second|evening|| +78498|AAAAAAAADKCDBAAA|78498|21|48|18|PM|second|evening|| +78499|AAAAAAAAEKCDBAAA|78499|21|48|19|PM|second|evening|| +78500|AAAAAAAAFKCDBAAA|78500|21|48|20|PM|second|evening|| +78501|AAAAAAAAGKCDBAAA|78501|21|48|21|PM|second|evening|| +78502|AAAAAAAAHKCDBAAA|78502|21|48|22|PM|second|evening|| +78503|AAAAAAAAIKCDBAAA|78503|21|48|23|PM|second|evening|| +78504|AAAAAAAAJKCDBAAA|78504|21|48|24|PM|second|evening|| +78505|AAAAAAAAKKCDBAAA|78505|21|48|25|PM|second|evening|| +78506|AAAAAAAALKCDBAAA|78506|21|48|26|PM|second|evening|| +78507|AAAAAAAAMKCDBAAA|78507|21|48|27|PM|second|evening|| +78508|AAAAAAAANKCDBAAA|78508|21|48|28|PM|second|evening|| +78509|AAAAAAAAOKCDBAAA|78509|21|48|29|PM|second|evening|| +78510|AAAAAAAAPKCDBAAA|78510|21|48|30|PM|second|evening|| +78511|AAAAAAAAALCDBAAA|78511|21|48|31|PM|second|evening|| +78512|AAAAAAAABLCDBAAA|78512|21|48|32|PM|second|evening|| +78513|AAAAAAAACLCDBAAA|78513|21|48|33|PM|second|evening|| +78514|AAAAAAAADLCDBAAA|78514|21|48|34|PM|second|evening|| +78515|AAAAAAAAELCDBAAA|78515|21|48|35|PM|second|evening|| +78516|AAAAAAAAFLCDBAAA|78516|21|48|36|PM|second|evening|| +78517|AAAAAAAAGLCDBAAA|78517|21|48|37|PM|second|evening|| +78518|AAAAAAAAHLCDBAAA|78518|21|48|38|PM|second|evening|| +78519|AAAAAAAAILCDBAAA|78519|21|48|39|PM|second|evening|| +78520|AAAAAAAAJLCDBAAA|78520|21|48|40|PM|second|evening|| +78521|AAAAAAAAKLCDBAAA|78521|21|48|41|PM|second|evening|| +78522|AAAAAAAALLCDBAAA|78522|21|48|42|PM|second|evening|| +78523|AAAAAAAAMLCDBAAA|78523|21|48|43|PM|second|evening|| +78524|AAAAAAAANLCDBAAA|78524|21|48|44|PM|second|evening|| +78525|AAAAAAAAOLCDBAAA|78525|21|48|45|PM|second|evening|| +78526|AAAAAAAAPLCDBAAA|78526|21|48|46|PM|second|evening|| +78527|AAAAAAAAAMCDBAAA|78527|21|48|47|PM|second|evening|| +78528|AAAAAAAABMCDBAAA|78528|21|48|48|PM|second|evening|| +78529|AAAAAAAACMCDBAAA|78529|21|48|49|PM|second|evening|| +78530|AAAAAAAADMCDBAAA|78530|21|48|50|PM|second|evening|| +78531|AAAAAAAAEMCDBAAA|78531|21|48|51|PM|second|evening|| +78532|AAAAAAAAFMCDBAAA|78532|21|48|52|PM|second|evening|| +78533|AAAAAAAAGMCDBAAA|78533|21|48|53|PM|second|evening|| +78534|AAAAAAAAHMCDBAAA|78534|21|48|54|PM|second|evening|| +78535|AAAAAAAAIMCDBAAA|78535|21|48|55|PM|second|evening|| +78536|AAAAAAAAJMCDBAAA|78536|21|48|56|PM|second|evening|| +78537|AAAAAAAAKMCDBAAA|78537|21|48|57|PM|second|evening|| +78538|AAAAAAAALMCDBAAA|78538|21|48|58|PM|second|evening|| +78539|AAAAAAAAMMCDBAAA|78539|21|48|59|PM|second|evening|| +78540|AAAAAAAANMCDBAAA|78540|21|49|0|PM|second|evening|| +78541|AAAAAAAAOMCDBAAA|78541|21|49|1|PM|second|evening|| +78542|AAAAAAAAPMCDBAAA|78542|21|49|2|PM|second|evening|| +78543|AAAAAAAAANCDBAAA|78543|21|49|3|PM|second|evening|| +78544|AAAAAAAABNCDBAAA|78544|21|49|4|PM|second|evening|| +78545|AAAAAAAACNCDBAAA|78545|21|49|5|PM|second|evening|| +78546|AAAAAAAADNCDBAAA|78546|21|49|6|PM|second|evening|| +78547|AAAAAAAAENCDBAAA|78547|21|49|7|PM|second|evening|| +78548|AAAAAAAAFNCDBAAA|78548|21|49|8|PM|second|evening|| +78549|AAAAAAAAGNCDBAAA|78549|21|49|9|PM|second|evening|| +78550|AAAAAAAAHNCDBAAA|78550|21|49|10|PM|second|evening|| +78551|AAAAAAAAINCDBAAA|78551|21|49|11|PM|second|evening|| +78552|AAAAAAAAJNCDBAAA|78552|21|49|12|PM|second|evening|| +78553|AAAAAAAAKNCDBAAA|78553|21|49|13|PM|second|evening|| +78554|AAAAAAAALNCDBAAA|78554|21|49|14|PM|second|evening|| +78555|AAAAAAAAMNCDBAAA|78555|21|49|15|PM|second|evening|| +78556|AAAAAAAANNCDBAAA|78556|21|49|16|PM|second|evening|| +78557|AAAAAAAAONCDBAAA|78557|21|49|17|PM|second|evening|| +78558|AAAAAAAAPNCDBAAA|78558|21|49|18|PM|second|evening|| +78559|AAAAAAAAAOCDBAAA|78559|21|49|19|PM|second|evening|| +78560|AAAAAAAABOCDBAAA|78560|21|49|20|PM|second|evening|| +78561|AAAAAAAACOCDBAAA|78561|21|49|21|PM|second|evening|| +78562|AAAAAAAADOCDBAAA|78562|21|49|22|PM|second|evening|| +78563|AAAAAAAAEOCDBAAA|78563|21|49|23|PM|second|evening|| +78564|AAAAAAAAFOCDBAAA|78564|21|49|24|PM|second|evening|| +78565|AAAAAAAAGOCDBAAA|78565|21|49|25|PM|second|evening|| +78566|AAAAAAAAHOCDBAAA|78566|21|49|26|PM|second|evening|| +78567|AAAAAAAAIOCDBAAA|78567|21|49|27|PM|second|evening|| +78568|AAAAAAAAJOCDBAAA|78568|21|49|28|PM|second|evening|| +78569|AAAAAAAAKOCDBAAA|78569|21|49|29|PM|second|evening|| +78570|AAAAAAAALOCDBAAA|78570|21|49|30|PM|second|evening|| +78571|AAAAAAAAMOCDBAAA|78571|21|49|31|PM|second|evening|| +78572|AAAAAAAANOCDBAAA|78572|21|49|32|PM|second|evening|| +78573|AAAAAAAAOOCDBAAA|78573|21|49|33|PM|second|evening|| +78574|AAAAAAAAPOCDBAAA|78574|21|49|34|PM|second|evening|| +78575|AAAAAAAAAPCDBAAA|78575|21|49|35|PM|second|evening|| +78576|AAAAAAAABPCDBAAA|78576|21|49|36|PM|second|evening|| +78577|AAAAAAAACPCDBAAA|78577|21|49|37|PM|second|evening|| +78578|AAAAAAAADPCDBAAA|78578|21|49|38|PM|second|evening|| +78579|AAAAAAAAEPCDBAAA|78579|21|49|39|PM|second|evening|| +78580|AAAAAAAAFPCDBAAA|78580|21|49|40|PM|second|evening|| +78581|AAAAAAAAGPCDBAAA|78581|21|49|41|PM|second|evening|| +78582|AAAAAAAAHPCDBAAA|78582|21|49|42|PM|second|evening|| +78583|AAAAAAAAIPCDBAAA|78583|21|49|43|PM|second|evening|| +78584|AAAAAAAAJPCDBAAA|78584|21|49|44|PM|second|evening|| +78585|AAAAAAAAKPCDBAAA|78585|21|49|45|PM|second|evening|| +78586|AAAAAAAALPCDBAAA|78586|21|49|46|PM|second|evening|| +78587|AAAAAAAAMPCDBAAA|78587|21|49|47|PM|second|evening|| +78588|AAAAAAAANPCDBAAA|78588|21|49|48|PM|second|evening|| +78589|AAAAAAAAOPCDBAAA|78589|21|49|49|PM|second|evening|| +78590|AAAAAAAAPPCDBAAA|78590|21|49|50|PM|second|evening|| +78591|AAAAAAAAAADDBAAA|78591|21|49|51|PM|second|evening|| +78592|AAAAAAAABADDBAAA|78592|21|49|52|PM|second|evening|| +78593|AAAAAAAACADDBAAA|78593|21|49|53|PM|second|evening|| +78594|AAAAAAAADADDBAAA|78594|21|49|54|PM|second|evening|| +78595|AAAAAAAAEADDBAAA|78595|21|49|55|PM|second|evening|| +78596|AAAAAAAAFADDBAAA|78596|21|49|56|PM|second|evening|| +78597|AAAAAAAAGADDBAAA|78597|21|49|57|PM|second|evening|| +78598|AAAAAAAAHADDBAAA|78598|21|49|58|PM|second|evening|| +78599|AAAAAAAAIADDBAAA|78599|21|49|59|PM|second|evening|| +78600|AAAAAAAAJADDBAAA|78600|21|50|0|PM|second|evening|| +78601|AAAAAAAAKADDBAAA|78601|21|50|1|PM|second|evening|| +78602|AAAAAAAALADDBAAA|78602|21|50|2|PM|second|evening|| +78603|AAAAAAAAMADDBAAA|78603|21|50|3|PM|second|evening|| +78604|AAAAAAAANADDBAAA|78604|21|50|4|PM|second|evening|| +78605|AAAAAAAAOADDBAAA|78605|21|50|5|PM|second|evening|| +78606|AAAAAAAAPADDBAAA|78606|21|50|6|PM|second|evening|| +78607|AAAAAAAAABDDBAAA|78607|21|50|7|PM|second|evening|| +78608|AAAAAAAABBDDBAAA|78608|21|50|8|PM|second|evening|| +78609|AAAAAAAACBDDBAAA|78609|21|50|9|PM|second|evening|| +78610|AAAAAAAADBDDBAAA|78610|21|50|10|PM|second|evening|| +78611|AAAAAAAAEBDDBAAA|78611|21|50|11|PM|second|evening|| +78612|AAAAAAAAFBDDBAAA|78612|21|50|12|PM|second|evening|| +78613|AAAAAAAAGBDDBAAA|78613|21|50|13|PM|second|evening|| +78614|AAAAAAAAHBDDBAAA|78614|21|50|14|PM|second|evening|| +78615|AAAAAAAAIBDDBAAA|78615|21|50|15|PM|second|evening|| +78616|AAAAAAAAJBDDBAAA|78616|21|50|16|PM|second|evening|| +78617|AAAAAAAAKBDDBAAA|78617|21|50|17|PM|second|evening|| +78618|AAAAAAAALBDDBAAA|78618|21|50|18|PM|second|evening|| +78619|AAAAAAAAMBDDBAAA|78619|21|50|19|PM|second|evening|| +78620|AAAAAAAANBDDBAAA|78620|21|50|20|PM|second|evening|| +78621|AAAAAAAAOBDDBAAA|78621|21|50|21|PM|second|evening|| +78622|AAAAAAAAPBDDBAAA|78622|21|50|22|PM|second|evening|| +78623|AAAAAAAAACDDBAAA|78623|21|50|23|PM|second|evening|| +78624|AAAAAAAABCDDBAAA|78624|21|50|24|PM|second|evening|| +78625|AAAAAAAACCDDBAAA|78625|21|50|25|PM|second|evening|| +78626|AAAAAAAADCDDBAAA|78626|21|50|26|PM|second|evening|| +78627|AAAAAAAAECDDBAAA|78627|21|50|27|PM|second|evening|| +78628|AAAAAAAAFCDDBAAA|78628|21|50|28|PM|second|evening|| +78629|AAAAAAAAGCDDBAAA|78629|21|50|29|PM|second|evening|| +78630|AAAAAAAAHCDDBAAA|78630|21|50|30|PM|second|evening|| +78631|AAAAAAAAICDDBAAA|78631|21|50|31|PM|second|evening|| +78632|AAAAAAAAJCDDBAAA|78632|21|50|32|PM|second|evening|| +78633|AAAAAAAAKCDDBAAA|78633|21|50|33|PM|second|evening|| +78634|AAAAAAAALCDDBAAA|78634|21|50|34|PM|second|evening|| +78635|AAAAAAAAMCDDBAAA|78635|21|50|35|PM|second|evening|| +78636|AAAAAAAANCDDBAAA|78636|21|50|36|PM|second|evening|| +78637|AAAAAAAAOCDDBAAA|78637|21|50|37|PM|second|evening|| +78638|AAAAAAAAPCDDBAAA|78638|21|50|38|PM|second|evening|| +78639|AAAAAAAAADDDBAAA|78639|21|50|39|PM|second|evening|| +78640|AAAAAAAABDDDBAAA|78640|21|50|40|PM|second|evening|| +78641|AAAAAAAACDDDBAAA|78641|21|50|41|PM|second|evening|| +78642|AAAAAAAADDDDBAAA|78642|21|50|42|PM|second|evening|| +78643|AAAAAAAAEDDDBAAA|78643|21|50|43|PM|second|evening|| +78644|AAAAAAAAFDDDBAAA|78644|21|50|44|PM|second|evening|| +78645|AAAAAAAAGDDDBAAA|78645|21|50|45|PM|second|evening|| +78646|AAAAAAAAHDDDBAAA|78646|21|50|46|PM|second|evening|| +78647|AAAAAAAAIDDDBAAA|78647|21|50|47|PM|second|evening|| +78648|AAAAAAAAJDDDBAAA|78648|21|50|48|PM|second|evening|| +78649|AAAAAAAAKDDDBAAA|78649|21|50|49|PM|second|evening|| +78650|AAAAAAAALDDDBAAA|78650|21|50|50|PM|second|evening|| +78651|AAAAAAAAMDDDBAAA|78651|21|50|51|PM|second|evening|| +78652|AAAAAAAANDDDBAAA|78652|21|50|52|PM|second|evening|| +78653|AAAAAAAAODDDBAAA|78653|21|50|53|PM|second|evening|| +78654|AAAAAAAAPDDDBAAA|78654|21|50|54|PM|second|evening|| +78655|AAAAAAAAAEDDBAAA|78655|21|50|55|PM|second|evening|| +78656|AAAAAAAABEDDBAAA|78656|21|50|56|PM|second|evening|| +78657|AAAAAAAACEDDBAAA|78657|21|50|57|PM|second|evening|| +78658|AAAAAAAADEDDBAAA|78658|21|50|58|PM|second|evening|| +78659|AAAAAAAAEEDDBAAA|78659|21|50|59|PM|second|evening|| +78660|AAAAAAAAFEDDBAAA|78660|21|51|0|PM|second|evening|| +78661|AAAAAAAAGEDDBAAA|78661|21|51|1|PM|second|evening|| +78662|AAAAAAAAHEDDBAAA|78662|21|51|2|PM|second|evening|| +78663|AAAAAAAAIEDDBAAA|78663|21|51|3|PM|second|evening|| +78664|AAAAAAAAJEDDBAAA|78664|21|51|4|PM|second|evening|| +78665|AAAAAAAAKEDDBAAA|78665|21|51|5|PM|second|evening|| +78666|AAAAAAAALEDDBAAA|78666|21|51|6|PM|second|evening|| +78667|AAAAAAAAMEDDBAAA|78667|21|51|7|PM|second|evening|| +78668|AAAAAAAANEDDBAAA|78668|21|51|8|PM|second|evening|| +78669|AAAAAAAAOEDDBAAA|78669|21|51|9|PM|second|evening|| +78670|AAAAAAAAPEDDBAAA|78670|21|51|10|PM|second|evening|| +78671|AAAAAAAAAFDDBAAA|78671|21|51|11|PM|second|evening|| +78672|AAAAAAAABFDDBAAA|78672|21|51|12|PM|second|evening|| +78673|AAAAAAAACFDDBAAA|78673|21|51|13|PM|second|evening|| +78674|AAAAAAAADFDDBAAA|78674|21|51|14|PM|second|evening|| +78675|AAAAAAAAEFDDBAAA|78675|21|51|15|PM|second|evening|| +78676|AAAAAAAAFFDDBAAA|78676|21|51|16|PM|second|evening|| +78677|AAAAAAAAGFDDBAAA|78677|21|51|17|PM|second|evening|| +78678|AAAAAAAAHFDDBAAA|78678|21|51|18|PM|second|evening|| +78679|AAAAAAAAIFDDBAAA|78679|21|51|19|PM|second|evening|| +78680|AAAAAAAAJFDDBAAA|78680|21|51|20|PM|second|evening|| +78681|AAAAAAAAKFDDBAAA|78681|21|51|21|PM|second|evening|| +78682|AAAAAAAALFDDBAAA|78682|21|51|22|PM|second|evening|| +78683|AAAAAAAAMFDDBAAA|78683|21|51|23|PM|second|evening|| +78684|AAAAAAAANFDDBAAA|78684|21|51|24|PM|second|evening|| +78685|AAAAAAAAOFDDBAAA|78685|21|51|25|PM|second|evening|| +78686|AAAAAAAAPFDDBAAA|78686|21|51|26|PM|second|evening|| +78687|AAAAAAAAAGDDBAAA|78687|21|51|27|PM|second|evening|| +78688|AAAAAAAABGDDBAAA|78688|21|51|28|PM|second|evening|| +78689|AAAAAAAACGDDBAAA|78689|21|51|29|PM|second|evening|| +78690|AAAAAAAADGDDBAAA|78690|21|51|30|PM|second|evening|| +78691|AAAAAAAAEGDDBAAA|78691|21|51|31|PM|second|evening|| +78692|AAAAAAAAFGDDBAAA|78692|21|51|32|PM|second|evening|| +78693|AAAAAAAAGGDDBAAA|78693|21|51|33|PM|second|evening|| +78694|AAAAAAAAHGDDBAAA|78694|21|51|34|PM|second|evening|| +78695|AAAAAAAAIGDDBAAA|78695|21|51|35|PM|second|evening|| +78696|AAAAAAAAJGDDBAAA|78696|21|51|36|PM|second|evening|| +78697|AAAAAAAAKGDDBAAA|78697|21|51|37|PM|second|evening|| +78698|AAAAAAAALGDDBAAA|78698|21|51|38|PM|second|evening|| +78699|AAAAAAAAMGDDBAAA|78699|21|51|39|PM|second|evening|| +78700|AAAAAAAANGDDBAAA|78700|21|51|40|PM|second|evening|| +78701|AAAAAAAAOGDDBAAA|78701|21|51|41|PM|second|evening|| +78702|AAAAAAAAPGDDBAAA|78702|21|51|42|PM|second|evening|| +78703|AAAAAAAAAHDDBAAA|78703|21|51|43|PM|second|evening|| +78704|AAAAAAAABHDDBAAA|78704|21|51|44|PM|second|evening|| +78705|AAAAAAAACHDDBAAA|78705|21|51|45|PM|second|evening|| +78706|AAAAAAAADHDDBAAA|78706|21|51|46|PM|second|evening|| +78707|AAAAAAAAEHDDBAAA|78707|21|51|47|PM|second|evening|| +78708|AAAAAAAAFHDDBAAA|78708|21|51|48|PM|second|evening|| +78709|AAAAAAAAGHDDBAAA|78709|21|51|49|PM|second|evening|| +78710|AAAAAAAAHHDDBAAA|78710|21|51|50|PM|second|evening|| +78711|AAAAAAAAIHDDBAAA|78711|21|51|51|PM|second|evening|| +78712|AAAAAAAAJHDDBAAA|78712|21|51|52|PM|second|evening|| +78713|AAAAAAAAKHDDBAAA|78713|21|51|53|PM|second|evening|| +78714|AAAAAAAALHDDBAAA|78714|21|51|54|PM|second|evening|| +78715|AAAAAAAAMHDDBAAA|78715|21|51|55|PM|second|evening|| +78716|AAAAAAAANHDDBAAA|78716|21|51|56|PM|second|evening|| +78717|AAAAAAAAOHDDBAAA|78717|21|51|57|PM|second|evening|| +78718|AAAAAAAAPHDDBAAA|78718|21|51|58|PM|second|evening|| +78719|AAAAAAAAAIDDBAAA|78719|21|51|59|PM|second|evening|| +78720|AAAAAAAABIDDBAAA|78720|21|52|0|PM|second|evening|| +78721|AAAAAAAACIDDBAAA|78721|21|52|1|PM|second|evening|| +78722|AAAAAAAADIDDBAAA|78722|21|52|2|PM|second|evening|| +78723|AAAAAAAAEIDDBAAA|78723|21|52|3|PM|second|evening|| +78724|AAAAAAAAFIDDBAAA|78724|21|52|4|PM|second|evening|| +78725|AAAAAAAAGIDDBAAA|78725|21|52|5|PM|second|evening|| +78726|AAAAAAAAHIDDBAAA|78726|21|52|6|PM|second|evening|| +78727|AAAAAAAAIIDDBAAA|78727|21|52|7|PM|second|evening|| +78728|AAAAAAAAJIDDBAAA|78728|21|52|8|PM|second|evening|| +78729|AAAAAAAAKIDDBAAA|78729|21|52|9|PM|second|evening|| +78730|AAAAAAAALIDDBAAA|78730|21|52|10|PM|second|evening|| +78731|AAAAAAAAMIDDBAAA|78731|21|52|11|PM|second|evening|| +78732|AAAAAAAANIDDBAAA|78732|21|52|12|PM|second|evening|| +78733|AAAAAAAAOIDDBAAA|78733|21|52|13|PM|second|evening|| +78734|AAAAAAAAPIDDBAAA|78734|21|52|14|PM|second|evening|| +78735|AAAAAAAAAJDDBAAA|78735|21|52|15|PM|second|evening|| +78736|AAAAAAAABJDDBAAA|78736|21|52|16|PM|second|evening|| +78737|AAAAAAAACJDDBAAA|78737|21|52|17|PM|second|evening|| +78738|AAAAAAAADJDDBAAA|78738|21|52|18|PM|second|evening|| +78739|AAAAAAAAEJDDBAAA|78739|21|52|19|PM|second|evening|| +78740|AAAAAAAAFJDDBAAA|78740|21|52|20|PM|second|evening|| +78741|AAAAAAAAGJDDBAAA|78741|21|52|21|PM|second|evening|| +78742|AAAAAAAAHJDDBAAA|78742|21|52|22|PM|second|evening|| +78743|AAAAAAAAIJDDBAAA|78743|21|52|23|PM|second|evening|| +78744|AAAAAAAAJJDDBAAA|78744|21|52|24|PM|second|evening|| +78745|AAAAAAAAKJDDBAAA|78745|21|52|25|PM|second|evening|| +78746|AAAAAAAALJDDBAAA|78746|21|52|26|PM|second|evening|| +78747|AAAAAAAAMJDDBAAA|78747|21|52|27|PM|second|evening|| +78748|AAAAAAAANJDDBAAA|78748|21|52|28|PM|second|evening|| +78749|AAAAAAAAOJDDBAAA|78749|21|52|29|PM|second|evening|| +78750|AAAAAAAAPJDDBAAA|78750|21|52|30|PM|second|evening|| +78751|AAAAAAAAAKDDBAAA|78751|21|52|31|PM|second|evening|| +78752|AAAAAAAABKDDBAAA|78752|21|52|32|PM|second|evening|| +78753|AAAAAAAACKDDBAAA|78753|21|52|33|PM|second|evening|| +78754|AAAAAAAADKDDBAAA|78754|21|52|34|PM|second|evening|| +78755|AAAAAAAAEKDDBAAA|78755|21|52|35|PM|second|evening|| +78756|AAAAAAAAFKDDBAAA|78756|21|52|36|PM|second|evening|| +78757|AAAAAAAAGKDDBAAA|78757|21|52|37|PM|second|evening|| +78758|AAAAAAAAHKDDBAAA|78758|21|52|38|PM|second|evening|| +78759|AAAAAAAAIKDDBAAA|78759|21|52|39|PM|second|evening|| +78760|AAAAAAAAJKDDBAAA|78760|21|52|40|PM|second|evening|| +78761|AAAAAAAAKKDDBAAA|78761|21|52|41|PM|second|evening|| +78762|AAAAAAAALKDDBAAA|78762|21|52|42|PM|second|evening|| +78763|AAAAAAAAMKDDBAAA|78763|21|52|43|PM|second|evening|| +78764|AAAAAAAANKDDBAAA|78764|21|52|44|PM|second|evening|| +78765|AAAAAAAAOKDDBAAA|78765|21|52|45|PM|second|evening|| +78766|AAAAAAAAPKDDBAAA|78766|21|52|46|PM|second|evening|| +78767|AAAAAAAAALDDBAAA|78767|21|52|47|PM|second|evening|| +78768|AAAAAAAABLDDBAAA|78768|21|52|48|PM|second|evening|| +78769|AAAAAAAACLDDBAAA|78769|21|52|49|PM|second|evening|| +78770|AAAAAAAADLDDBAAA|78770|21|52|50|PM|second|evening|| +78771|AAAAAAAAELDDBAAA|78771|21|52|51|PM|second|evening|| +78772|AAAAAAAAFLDDBAAA|78772|21|52|52|PM|second|evening|| +78773|AAAAAAAAGLDDBAAA|78773|21|52|53|PM|second|evening|| +78774|AAAAAAAAHLDDBAAA|78774|21|52|54|PM|second|evening|| +78775|AAAAAAAAILDDBAAA|78775|21|52|55|PM|second|evening|| +78776|AAAAAAAAJLDDBAAA|78776|21|52|56|PM|second|evening|| +78777|AAAAAAAAKLDDBAAA|78777|21|52|57|PM|second|evening|| +78778|AAAAAAAALLDDBAAA|78778|21|52|58|PM|second|evening|| +78779|AAAAAAAAMLDDBAAA|78779|21|52|59|PM|second|evening|| +78780|AAAAAAAANLDDBAAA|78780|21|53|0|PM|second|evening|| +78781|AAAAAAAAOLDDBAAA|78781|21|53|1|PM|second|evening|| +78782|AAAAAAAAPLDDBAAA|78782|21|53|2|PM|second|evening|| +78783|AAAAAAAAAMDDBAAA|78783|21|53|3|PM|second|evening|| +78784|AAAAAAAABMDDBAAA|78784|21|53|4|PM|second|evening|| +78785|AAAAAAAACMDDBAAA|78785|21|53|5|PM|second|evening|| +78786|AAAAAAAADMDDBAAA|78786|21|53|6|PM|second|evening|| +78787|AAAAAAAAEMDDBAAA|78787|21|53|7|PM|second|evening|| +78788|AAAAAAAAFMDDBAAA|78788|21|53|8|PM|second|evening|| +78789|AAAAAAAAGMDDBAAA|78789|21|53|9|PM|second|evening|| +78790|AAAAAAAAHMDDBAAA|78790|21|53|10|PM|second|evening|| +78791|AAAAAAAAIMDDBAAA|78791|21|53|11|PM|second|evening|| +78792|AAAAAAAAJMDDBAAA|78792|21|53|12|PM|second|evening|| +78793|AAAAAAAAKMDDBAAA|78793|21|53|13|PM|second|evening|| +78794|AAAAAAAALMDDBAAA|78794|21|53|14|PM|second|evening|| +78795|AAAAAAAAMMDDBAAA|78795|21|53|15|PM|second|evening|| +78796|AAAAAAAANMDDBAAA|78796|21|53|16|PM|second|evening|| +78797|AAAAAAAAOMDDBAAA|78797|21|53|17|PM|second|evening|| +78798|AAAAAAAAPMDDBAAA|78798|21|53|18|PM|second|evening|| +78799|AAAAAAAAANDDBAAA|78799|21|53|19|PM|second|evening|| +78800|AAAAAAAABNDDBAAA|78800|21|53|20|PM|second|evening|| +78801|AAAAAAAACNDDBAAA|78801|21|53|21|PM|second|evening|| +78802|AAAAAAAADNDDBAAA|78802|21|53|22|PM|second|evening|| +78803|AAAAAAAAENDDBAAA|78803|21|53|23|PM|second|evening|| +78804|AAAAAAAAFNDDBAAA|78804|21|53|24|PM|second|evening|| +78805|AAAAAAAAGNDDBAAA|78805|21|53|25|PM|second|evening|| +78806|AAAAAAAAHNDDBAAA|78806|21|53|26|PM|second|evening|| +78807|AAAAAAAAINDDBAAA|78807|21|53|27|PM|second|evening|| +78808|AAAAAAAAJNDDBAAA|78808|21|53|28|PM|second|evening|| +78809|AAAAAAAAKNDDBAAA|78809|21|53|29|PM|second|evening|| +78810|AAAAAAAALNDDBAAA|78810|21|53|30|PM|second|evening|| +78811|AAAAAAAAMNDDBAAA|78811|21|53|31|PM|second|evening|| +78812|AAAAAAAANNDDBAAA|78812|21|53|32|PM|second|evening|| +78813|AAAAAAAAONDDBAAA|78813|21|53|33|PM|second|evening|| +78814|AAAAAAAAPNDDBAAA|78814|21|53|34|PM|second|evening|| +78815|AAAAAAAAAODDBAAA|78815|21|53|35|PM|second|evening|| +78816|AAAAAAAABODDBAAA|78816|21|53|36|PM|second|evening|| +78817|AAAAAAAACODDBAAA|78817|21|53|37|PM|second|evening|| +78818|AAAAAAAADODDBAAA|78818|21|53|38|PM|second|evening|| +78819|AAAAAAAAEODDBAAA|78819|21|53|39|PM|second|evening|| +78820|AAAAAAAAFODDBAAA|78820|21|53|40|PM|second|evening|| +78821|AAAAAAAAGODDBAAA|78821|21|53|41|PM|second|evening|| +78822|AAAAAAAAHODDBAAA|78822|21|53|42|PM|second|evening|| +78823|AAAAAAAAIODDBAAA|78823|21|53|43|PM|second|evening|| +78824|AAAAAAAAJODDBAAA|78824|21|53|44|PM|second|evening|| +78825|AAAAAAAAKODDBAAA|78825|21|53|45|PM|second|evening|| +78826|AAAAAAAALODDBAAA|78826|21|53|46|PM|second|evening|| +78827|AAAAAAAAMODDBAAA|78827|21|53|47|PM|second|evening|| +78828|AAAAAAAANODDBAAA|78828|21|53|48|PM|second|evening|| +78829|AAAAAAAAOODDBAAA|78829|21|53|49|PM|second|evening|| +78830|AAAAAAAAPODDBAAA|78830|21|53|50|PM|second|evening|| +78831|AAAAAAAAAPDDBAAA|78831|21|53|51|PM|second|evening|| +78832|AAAAAAAABPDDBAAA|78832|21|53|52|PM|second|evening|| +78833|AAAAAAAACPDDBAAA|78833|21|53|53|PM|second|evening|| +78834|AAAAAAAADPDDBAAA|78834|21|53|54|PM|second|evening|| +78835|AAAAAAAAEPDDBAAA|78835|21|53|55|PM|second|evening|| +78836|AAAAAAAAFPDDBAAA|78836|21|53|56|PM|second|evening|| +78837|AAAAAAAAGPDDBAAA|78837|21|53|57|PM|second|evening|| +78838|AAAAAAAAHPDDBAAA|78838|21|53|58|PM|second|evening|| +78839|AAAAAAAAIPDDBAAA|78839|21|53|59|PM|second|evening|| +78840|AAAAAAAAJPDDBAAA|78840|21|54|0|PM|second|evening|| +78841|AAAAAAAAKPDDBAAA|78841|21|54|1|PM|second|evening|| +78842|AAAAAAAALPDDBAAA|78842|21|54|2|PM|second|evening|| +78843|AAAAAAAAMPDDBAAA|78843|21|54|3|PM|second|evening|| +78844|AAAAAAAANPDDBAAA|78844|21|54|4|PM|second|evening|| +78845|AAAAAAAAOPDDBAAA|78845|21|54|5|PM|second|evening|| +78846|AAAAAAAAPPDDBAAA|78846|21|54|6|PM|second|evening|| +78847|AAAAAAAAAAEDBAAA|78847|21|54|7|PM|second|evening|| +78848|AAAAAAAABAEDBAAA|78848|21|54|8|PM|second|evening|| +78849|AAAAAAAACAEDBAAA|78849|21|54|9|PM|second|evening|| +78850|AAAAAAAADAEDBAAA|78850|21|54|10|PM|second|evening|| +78851|AAAAAAAAEAEDBAAA|78851|21|54|11|PM|second|evening|| +78852|AAAAAAAAFAEDBAAA|78852|21|54|12|PM|second|evening|| +78853|AAAAAAAAGAEDBAAA|78853|21|54|13|PM|second|evening|| +78854|AAAAAAAAHAEDBAAA|78854|21|54|14|PM|second|evening|| +78855|AAAAAAAAIAEDBAAA|78855|21|54|15|PM|second|evening|| +78856|AAAAAAAAJAEDBAAA|78856|21|54|16|PM|second|evening|| +78857|AAAAAAAAKAEDBAAA|78857|21|54|17|PM|second|evening|| +78858|AAAAAAAALAEDBAAA|78858|21|54|18|PM|second|evening|| +78859|AAAAAAAAMAEDBAAA|78859|21|54|19|PM|second|evening|| +78860|AAAAAAAANAEDBAAA|78860|21|54|20|PM|second|evening|| +78861|AAAAAAAAOAEDBAAA|78861|21|54|21|PM|second|evening|| +78862|AAAAAAAAPAEDBAAA|78862|21|54|22|PM|second|evening|| +78863|AAAAAAAAABEDBAAA|78863|21|54|23|PM|second|evening|| +78864|AAAAAAAABBEDBAAA|78864|21|54|24|PM|second|evening|| +78865|AAAAAAAACBEDBAAA|78865|21|54|25|PM|second|evening|| +78866|AAAAAAAADBEDBAAA|78866|21|54|26|PM|second|evening|| +78867|AAAAAAAAEBEDBAAA|78867|21|54|27|PM|second|evening|| +78868|AAAAAAAAFBEDBAAA|78868|21|54|28|PM|second|evening|| +78869|AAAAAAAAGBEDBAAA|78869|21|54|29|PM|second|evening|| +78870|AAAAAAAAHBEDBAAA|78870|21|54|30|PM|second|evening|| +78871|AAAAAAAAIBEDBAAA|78871|21|54|31|PM|second|evening|| +78872|AAAAAAAAJBEDBAAA|78872|21|54|32|PM|second|evening|| +78873|AAAAAAAAKBEDBAAA|78873|21|54|33|PM|second|evening|| +78874|AAAAAAAALBEDBAAA|78874|21|54|34|PM|second|evening|| +78875|AAAAAAAAMBEDBAAA|78875|21|54|35|PM|second|evening|| +78876|AAAAAAAANBEDBAAA|78876|21|54|36|PM|second|evening|| +78877|AAAAAAAAOBEDBAAA|78877|21|54|37|PM|second|evening|| +78878|AAAAAAAAPBEDBAAA|78878|21|54|38|PM|second|evening|| +78879|AAAAAAAAACEDBAAA|78879|21|54|39|PM|second|evening|| +78880|AAAAAAAABCEDBAAA|78880|21|54|40|PM|second|evening|| +78881|AAAAAAAACCEDBAAA|78881|21|54|41|PM|second|evening|| +78882|AAAAAAAADCEDBAAA|78882|21|54|42|PM|second|evening|| +78883|AAAAAAAAECEDBAAA|78883|21|54|43|PM|second|evening|| +78884|AAAAAAAAFCEDBAAA|78884|21|54|44|PM|second|evening|| +78885|AAAAAAAAGCEDBAAA|78885|21|54|45|PM|second|evening|| +78886|AAAAAAAAHCEDBAAA|78886|21|54|46|PM|second|evening|| +78887|AAAAAAAAICEDBAAA|78887|21|54|47|PM|second|evening|| +78888|AAAAAAAAJCEDBAAA|78888|21|54|48|PM|second|evening|| +78889|AAAAAAAAKCEDBAAA|78889|21|54|49|PM|second|evening|| +78890|AAAAAAAALCEDBAAA|78890|21|54|50|PM|second|evening|| +78891|AAAAAAAAMCEDBAAA|78891|21|54|51|PM|second|evening|| +78892|AAAAAAAANCEDBAAA|78892|21|54|52|PM|second|evening|| +78893|AAAAAAAAOCEDBAAA|78893|21|54|53|PM|second|evening|| +78894|AAAAAAAAPCEDBAAA|78894|21|54|54|PM|second|evening|| +78895|AAAAAAAAADEDBAAA|78895|21|54|55|PM|second|evening|| +78896|AAAAAAAABDEDBAAA|78896|21|54|56|PM|second|evening|| +78897|AAAAAAAACDEDBAAA|78897|21|54|57|PM|second|evening|| +78898|AAAAAAAADDEDBAAA|78898|21|54|58|PM|second|evening|| +78899|AAAAAAAAEDEDBAAA|78899|21|54|59|PM|second|evening|| +78900|AAAAAAAAFDEDBAAA|78900|21|55|0|PM|second|evening|| +78901|AAAAAAAAGDEDBAAA|78901|21|55|1|PM|second|evening|| +78902|AAAAAAAAHDEDBAAA|78902|21|55|2|PM|second|evening|| +78903|AAAAAAAAIDEDBAAA|78903|21|55|3|PM|second|evening|| +78904|AAAAAAAAJDEDBAAA|78904|21|55|4|PM|second|evening|| +78905|AAAAAAAAKDEDBAAA|78905|21|55|5|PM|second|evening|| +78906|AAAAAAAALDEDBAAA|78906|21|55|6|PM|second|evening|| +78907|AAAAAAAAMDEDBAAA|78907|21|55|7|PM|second|evening|| +78908|AAAAAAAANDEDBAAA|78908|21|55|8|PM|second|evening|| +78909|AAAAAAAAODEDBAAA|78909|21|55|9|PM|second|evening|| +78910|AAAAAAAAPDEDBAAA|78910|21|55|10|PM|second|evening|| +78911|AAAAAAAAAEEDBAAA|78911|21|55|11|PM|second|evening|| +78912|AAAAAAAABEEDBAAA|78912|21|55|12|PM|second|evening|| +78913|AAAAAAAACEEDBAAA|78913|21|55|13|PM|second|evening|| +78914|AAAAAAAADEEDBAAA|78914|21|55|14|PM|second|evening|| +78915|AAAAAAAAEEEDBAAA|78915|21|55|15|PM|second|evening|| +78916|AAAAAAAAFEEDBAAA|78916|21|55|16|PM|second|evening|| +78917|AAAAAAAAGEEDBAAA|78917|21|55|17|PM|second|evening|| +78918|AAAAAAAAHEEDBAAA|78918|21|55|18|PM|second|evening|| +78919|AAAAAAAAIEEDBAAA|78919|21|55|19|PM|second|evening|| +78920|AAAAAAAAJEEDBAAA|78920|21|55|20|PM|second|evening|| +78921|AAAAAAAAKEEDBAAA|78921|21|55|21|PM|second|evening|| +78922|AAAAAAAALEEDBAAA|78922|21|55|22|PM|second|evening|| +78923|AAAAAAAAMEEDBAAA|78923|21|55|23|PM|second|evening|| +78924|AAAAAAAANEEDBAAA|78924|21|55|24|PM|second|evening|| +78925|AAAAAAAAOEEDBAAA|78925|21|55|25|PM|second|evening|| +78926|AAAAAAAAPEEDBAAA|78926|21|55|26|PM|second|evening|| +78927|AAAAAAAAAFEDBAAA|78927|21|55|27|PM|second|evening|| +78928|AAAAAAAABFEDBAAA|78928|21|55|28|PM|second|evening|| +78929|AAAAAAAACFEDBAAA|78929|21|55|29|PM|second|evening|| +78930|AAAAAAAADFEDBAAA|78930|21|55|30|PM|second|evening|| +78931|AAAAAAAAEFEDBAAA|78931|21|55|31|PM|second|evening|| +78932|AAAAAAAAFFEDBAAA|78932|21|55|32|PM|second|evening|| +78933|AAAAAAAAGFEDBAAA|78933|21|55|33|PM|second|evening|| +78934|AAAAAAAAHFEDBAAA|78934|21|55|34|PM|second|evening|| +78935|AAAAAAAAIFEDBAAA|78935|21|55|35|PM|second|evening|| +78936|AAAAAAAAJFEDBAAA|78936|21|55|36|PM|second|evening|| +78937|AAAAAAAAKFEDBAAA|78937|21|55|37|PM|second|evening|| +78938|AAAAAAAALFEDBAAA|78938|21|55|38|PM|second|evening|| +78939|AAAAAAAAMFEDBAAA|78939|21|55|39|PM|second|evening|| +78940|AAAAAAAANFEDBAAA|78940|21|55|40|PM|second|evening|| +78941|AAAAAAAAOFEDBAAA|78941|21|55|41|PM|second|evening|| +78942|AAAAAAAAPFEDBAAA|78942|21|55|42|PM|second|evening|| +78943|AAAAAAAAAGEDBAAA|78943|21|55|43|PM|second|evening|| +78944|AAAAAAAABGEDBAAA|78944|21|55|44|PM|second|evening|| +78945|AAAAAAAACGEDBAAA|78945|21|55|45|PM|second|evening|| +78946|AAAAAAAADGEDBAAA|78946|21|55|46|PM|second|evening|| +78947|AAAAAAAAEGEDBAAA|78947|21|55|47|PM|second|evening|| +78948|AAAAAAAAFGEDBAAA|78948|21|55|48|PM|second|evening|| +78949|AAAAAAAAGGEDBAAA|78949|21|55|49|PM|second|evening|| +78950|AAAAAAAAHGEDBAAA|78950|21|55|50|PM|second|evening|| +78951|AAAAAAAAIGEDBAAA|78951|21|55|51|PM|second|evening|| +78952|AAAAAAAAJGEDBAAA|78952|21|55|52|PM|second|evening|| +78953|AAAAAAAAKGEDBAAA|78953|21|55|53|PM|second|evening|| +78954|AAAAAAAALGEDBAAA|78954|21|55|54|PM|second|evening|| +78955|AAAAAAAAMGEDBAAA|78955|21|55|55|PM|second|evening|| +78956|AAAAAAAANGEDBAAA|78956|21|55|56|PM|second|evening|| +78957|AAAAAAAAOGEDBAAA|78957|21|55|57|PM|second|evening|| +78958|AAAAAAAAPGEDBAAA|78958|21|55|58|PM|second|evening|| +78959|AAAAAAAAAHEDBAAA|78959|21|55|59|PM|second|evening|| +78960|AAAAAAAABHEDBAAA|78960|21|56|0|PM|second|evening|| +78961|AAAAAAAACHEDBAAA|78961|21|56|1|PM|second|evening|| +78962|AAAAAAAADHEDBAAA|78962|21|56|2|PM|second|evening|| +78963|AAAAAAAAEHEDBAAA|78963|21|56|3|PM|second|evening|| +78964|AAAAAAAAFHEDBAAA|78964|21|56|4|PM|second|evening|| +78965|AAAAAAAAGHEDBAAA|78965|21|56|5|PM|second|evening|| +78966|AAAAAAAAHHEDBAAA|78966|21|56|6|PM|second|evening|| +78967|AAAAAAAAIHEDBAAA|78967|21|56|7|PM|second|evening|| +78968|AAAAAAAAJHEDBAAA|78968|21|56|8|PM|second|evening|| +78969|AAAAAAAAKHEDBAAA|78969|21|56|9|PM|second|evening|| +78970|AAAAAAAALHEDBAAA|78970|21|56|10|PM|second|evening|| +78971|AAAAAAAAMHEDBAAA|78971|21|56|11|PM|second|evening|| +78972|AAAAAAAANHEDBAAA|78972|21|56|12|PM|second|evening|| +78973|AAAAAAAAOHEDBAAA|78973|21|56|13|PM|second|evening|| +78974|AAAAAAAAPHEDBAAA|78974|21|56|14|PM|second|evening|| +78975|AAAAAAAAAIEDBAAA|78975|21|56|15|PM|second|evening|| +78976|AAAAAAAABIEDBAAA|78976|21|56|16|PM|second|evening|| +78977|AAAAAAAACIEDBAAA|78977|21|56|17|PM|second|evening|| +78978|AAAAAAAADIEDBAAA|78978|21|56|18|PM|second|evening|| +78979|AAAAAAAAEIEDBAAA|78979|21|56|19|PM|second|evening|| +78980|AAAAAAAAFIEDBAAA|78980|21|56|20|PM|second|evening|| +78981|AAAAAAAAGIEDBAAA|78981|21|56|21|PM|second|evening|| +78982|AAAAAAAAHIEDBAAA|78982|21|56|22|PM|second|evening|| +78983|AAAAAAAAIIEDBAAA|78983|21|56|23|PM|second|evening|| +78984|AAAAAAAAJIEDBAAA|78984|21|56|24|PM|second|evening|| +78985|AAAAAAAAKIEDBAAA|78985|21|56|25|PM|second|evening|| +78986|AAAAAAAALIEDBAAA|78986|21|56|26|PM|second|evening|| +78987|AAAAAAAAMIEDBAAA|78987|21|56|27|PM|second|evening|| +78988|AAAAAAAANIEDBAAA|78988|21|56|28|PM|second|evening|| +78989|AAAAAAAAOIEDBAAA|78989|21|56|29|PM|second|evening|| +78990|AAAAAAAAPIEDBAAA|78990|21|56|30|PM|second|evening|| +78991|AAAAAAAAAJEDBAAA|78991|21|56|31|PM|second|evening|| +78992|AAAAAAAABJEDBAAA|78992|21|56|32|PM|second|evening|| +78993|AAAAAAAACJEDBAAA|78993|21|56|33|PM|second|evening|| +78994|AAAAAAAADJEDBAAA|78994|21|56|34|PM|second|evening|| +78995|AAAAAAAAEJEDBAAA|78995|21|56|35|PM|second|evening|| +78996|AAAAAAAAFJEDBAAA|78996|21|56|36|PM|second|evening|| +78997|AAAAAAAAGJEDBAAA|78997|21|56|37|PM|second|evening|| +78998|AAAAAAAAHJEDBAAA|78998|21|56|38|PM|second|evening|| +78999|AAAAAAAAIJEDBAAA|78999|21|56|39|PM|second|evening|| +79000|AAAAAAAAJJEDBAAA|79000|21|56|40|PM|second|evening|| +79001|AAAAAAAAKJEDBAAA|79001|21|56|41|PM|second|evening|| +79002|AAAAAAAALJEDBAAA|79002|21|56|42|PM|second|evening|| +79003|AAAAAAAAMJEDBAAA|79003|21|56|43|PM|second|evening|| +79004|AAAAAAAANJEDBAAA|79004|21|56|44|PM|second|evening|| +79005|AAAAAAAAOJEDBAAA|79005|21|56|45|PM|second|evening|| +79006|AAAAAAAAPJEDBAAA|79006|21|56|46|PM|second|evening|| +79007|AAAAAAAAAKEDBAAA|79007|21|56|47|PM|second|evening|| +79008|AAAAAAAABKEDBAAA|79008|21|56|48|PM|second|evening|| +79009|AAAAAAAACKEDBAAA|79009|21|56|49|PM|second|evening|| +79010|AAAAAAAADKEDBAAA|79010|21|56|50|PM|second|evening|| +79011|AAAAAAAAEKEDBAAA|79011|21|56|51|PM|second|evening|| +79012|AAAAAAAAFKEDBAAA|79012|21|56|52|PM|second|evening|| +79013|AAAAAAAAGKEDBAAA|79013|21|56|53|PM|second|evening|| +79014|AAAAAAAAHKEDBAAA|79014|21|56|54|PM|second|evening|| +79015|AAAAAAAAIKEDBAAA|79015|21|56|55|PM|second|evening|| +79016|AAAAAAAAJKEDBAAA|79016|21|56|56|PM|second|evening|| +79017|AAAAAAAAKKEDBAAA|79017|21|56|57|PM|second|evening|| +79018|AAAAAAAALKEDBAAA|79018|21|56|58|PM|second|evening|| +79019|AAAAAAAAMKEDBAAA|79019|21|56|59|PM|second|evening|| +79020|AAAAAAAANKEDBAAA|79020|21|57|0|PM|second|evening|| +79021|AAAAAAAAOKEDBAAA|79021|21|57|1|PM|second|evening|| +79022|AAAAAAAAPKEDBAAA|79022|21|57|2|PM|second|evening|| +79023|AAAAAAAAALEDBAAA|79023|21|57|3|PM|second|evening|| +79024|AAAAAAAABLEDBAAA|79024|21|57|4|PM|second|evening|| +79025|AAAAAAAACLEDBAAA|79025|21|57|5|PM|second|evening|| +79026|AAAAAAAADLEDBAAA|79026|21|57|6|PM|second|evening|| +79027|AAAAAAAAELEDBAAA|79027|21|57|7|PM|second|evening|| +79028|AAAAAAAAFLEDBAAA|79028|21|57|8|PM|second|evening|| +79029|AAAAAAAAGLEDBAAA|79029|21|57|9|PM|second|evening|| +79030|AAAAAAAAHLEDBAAA|79030|21|57|10|PM|second|evening|| +79031|AAAAAAAAILEDBAAA|79031|21|57|11|PM|second|evening|| +79032|AAAAAAAAJLEDBAAA|79032|21|57|12|PM|second|evening|| +79033|AAAAAAAAKLEDBAAA|79033|21|57|13|PM|second|evening|| +79034|AAAAAAAALLEDBAAA|79034|21|57|14|PM|second|evening|| +79035|AAAAAAAAMLEDBAAA|79035|21|57|15|PM|second|evening|| +79036|AAAAAAAANLEDBAAA|79036|21|57|16|PM|second|evening|| +79037|AAAAAAAAOLEDBAAA|79037|21|57|17|PM|second|evening|| +79038|AAAAAAAAPLEDBAAA|79038|21|57|18|PM|second|evening|| +79039|AAAAAAAAAMEDBAAA|79039|21|57|19|PM|second|evening|| +79040|AAAAAAAABMEDBAAA|79040|21|57|20|PM|second|evening|| +79041|AAAAAAAACMEDBAAA|79041|21|57|21|PM|second|evening|| +79042|AAAAAAAADMEDBAAA|79042|21|57|22|PM|second|evening|| +79043|AAAAAAAAEMEDBAAA|79043|21|57|23|PM|second|evening|| +79044|AAAAAAAAFMEDBAAA|79044|21|57|24|PM|second|evening|| +79045|AAAAAAAAGMEDBAAA|79045|21|57|25|PM|second|evening|| +79046|AAAAAAAAHMEDBAAA|79046|21|57|26|PM|second|evening|| +79047|AAAAAAAAIMEDBAAA|79047|21|57|27|PM|second|evening|| +79048|AAAAAAAAJMEDBAAA|79048|21|57|28|PM|second|evening|| +79049|AAAAAAAAKMEDBAAA|79049|21|57|29|PM|second|evening|| +79050|AAAAAAAALMEDBAAA|79050|21|57|30|PM|second|evening|| +79051|AAAAAAAAMMEDBAAA|79051|21|57|31|PM|second|evening|| +79052|AAAAAAAANMEDBAAA|79052|21|57|32|PM|second|evening|| +79053|AAAAAAAAOMEDBAAA|79053|21|57|33|PM|second|evening|| +79054|AAAAAAAAPMEDBAAA|79054|21|57|34|PM|second|evening|| +79055|AAAAAAAAANEDBAAA|79055|21|57|35|PM|second|evening|| +79056|AAAAAAAABNEDBAAA|79056|21|57|36|PM|second|evening|| +79057|AAAAAAAACNEDBAAA|79057|21|57|37|PM|second|evening|| +79058|AAAAAAAADNEDBAAA|79058|21|57|38|PM|second|evening|| +79059|AAAAAAAAENEDBAAA|79059|21|57|39|PM|second|evening|| +79060|AAAAAAAAFNEDBAAA|79060|21|57|40|PM|second|evening|| +79061|AAAAAAAAGNEDBAAA|79061|21|57|41|PM|second|evening|| +79062|AAAAAAAAHNEDBAAA|79062|21|57|42|PM|second|evening|| +79063|AAAAAAAAINEDBAAA|79063|21|57|43|PM|second|evening|| +79064|AAAAAAAAJNEDBAAA|79064|21|57|44|PM|second|evening|| +79065|AAAAAAAAKNEDBAAA|79065|21|57|45|PM|second|evening|| +79066|AAAAAAAALNEDBAAA|79066|21|57|46|PM|second|evening|| +79067|AAAAAAAAMNEDBAAA|79067|21|57|47|PM|second|evening|| +79068|AAAAAAAANNEDBAAA|79068|21|57|48|PM|second|evening|| +79069|AAAAAAAAONEDBAAA|79069|21|57|49|PM|second|evening|| +79070|AAAAAAAAPNEDBAAA|79070|21|57|50|PM|second|evening|| +79071|AAAAAAAAAOEDBAAA|79071|21|57|51|PM|second|evening|| +79072|AAAAAAAABOEDBAAA|79072|21|57|52|PM|second|evening|| +79073|AAAAAAAACOEDBAAA|79073|21|57|53|PM|second|evening|| +79074|AAAAAAAADOEDBAAA|79074|21|57|54|PM|second|evening|| +79075|AAAAAAAAEOEDBAAA|79075|21|57|55|PM|second|evening|| +79076|AAAAAAAAFOEDBAAA|79076|21|57|56|PM|second|evening|| +79077|AAAAAAAAGOEDBAAA|79077|21|57|57|PM|second|evening|| +79078|AAAAAAAAHOEDBAAA|79078|21|57|58|PM|second|evening|| +79079|AAAAAAAAIOEDBAAA|79079|21|57|59|PM|second|evening|| +79080|AAAAAAAAJOEDBAAA|79080|21|58|0|PM|second|evening|| +79081|AAAAAAAAKOEDBAAA|79081|21|58|1|PM|second|evening|| +79082|AAAAAAAALOEDBAAA|79082|21|58|2|PM|second|evening|| +79083|AAAAAAAAMOEDBAAA|79083|21|58|3|PM|second|evening|| +79084|AAAAAAAANOEDBAAA|79084|21|58|4|PM|second|evening|| +79085|AAAAAAAAOOEDBAAA|79085|21|58|5|PM|second|evening|| +79086|AAAAAAAAPOEDBAAA|79086|21|58|6|PM|second|evening|| +79087|AAAAAAAAAPEDBAAA|79087|21|58|7|PM|second|evening|| +79088|AAAAAAAABPEDBAAA|79088|21|58|8|PM|second|evening|| +79089|AAAAAAAACPEDBAAA|79089|21|58|9|PM|second|evening|| +79090|AAAAAAAADPEDBAAA|79090|21|58|10|PM|second|evening|| +79091|AAAAAAAAEPEDBAAA|79091|21|58|11|PM|second|evening|| +79092|AAAAAAAAFPEDBAAA|79092|21|58|12|PM|second|evening|| +79093|AAAAAAAAGPEDBAAA|79093|21|58|13|PM|second|evening|| +79094|AAAAAAAAHPEDBAAA|79094|21|58|14|PM|second|evening|| +79095|AAAAAAAAIPEDBAAA|79095|21|58|15|PM|second|evening|| +79096|AAAAAAAAJPEDBAAA|79096|21|58|16|PM|second|evening|| +79097|AAAAAAAAKPEDBAAA|79097|21|58|17|PM|second|evening|| +79098|AAAAAAAALPEDBAAA|79098|21|58|18|PM|second|evening|| +79099|AAAAAAAAMPEDBAAA|79099|21|58|19|PM|second|evening|| +79100|AAAAAAAANPEDBAAA|79100|21|58|20|PM|second|evening|| +79101|AAAAAAAAOPEDBAAA|79101|21|58|21|PM|second|evening|| +79102|AAAAAAAAPPEDBAAA|79102|21|58|22|PM|second|evening|| +79103|AAAAAAAAAAFDBAAA|79103|21|58|23|PM|second|evening|| +79104|AAAAAAAABAFDBAAA|79104|21|58|24|PM|second|evening|| +79105|AAAAAAAACAFDBAAA|79105|21|58|25|PM|second|evening|| +79106|AAAAAAAADAFDBAAA|79106|21|58|26|PM|second|evening|| +79107|AAAAAAAAEAFDBAAA|79107|21|58|27|PM|second|evening|| +79108|AAAAAAAAFAFDBAAA|79108|21|58|28|PM|second|evening|| +79109|AAAAAAAAGAFDBAAA|79109|21|58|29|PM|second|evening|| +79110|AAAAAAAAHAFDBAAA|79110|21|58|30|PM|second|evening|| +79111|AAAAAAAAIAFDBAAA|79111|21|58|31|PM|second|evening|| +79112|AAAAAAAAJAFDBAAA|79112|21|58|32|PM|second|evening|| +79113|AAAAAAAAKAFDBAAA|79113|21|58|33|PM|second|evening|| +79114|AAAAAAAALAFDBAAA|79114|21|58|34|PM|second|evening|| +79115|AAAAAAAAMAFDBAAA|79115|21|58|35|PM|second|evening|| +79116|AAAAAAAANAFDBAAA|79116|21|58|36|PM|second|evening|| +79117|AAAAAAAAOAFDBAAA|79117|21|58|37|PM|second|evening|| +79118|AAAAAAAAPAFDBAAA|79118|21|58|38|PM|second|evening|| +79119|AAAAAAAAABFDBAAA|79119|21|58|39|PM|second|evening|| +79120|AAAAAAAABBFDBAAA|79120|21|58|40|PM|second|evening|| +79121|AAAAAAAACBFDBAAA|79121|21|58|41|PM|second|evening|| +79122|AAAAAAAADBFDBAAA|79122|21|58|42|PM|second|evening|| +79123|AAAAAAAAEBFDBAAA|79123|21|58|43|PM|second|evening|| +79124|AAAAAAAAFBFDBAAA|79124|21|58|44|PM|second|evening|| +79125|AAAAAAAAGBFDBAAA|79125|21|58|45|PM|second|evening|| +79126|AAAAAAAAHBFDBAAA|79126|21|58|46|PM|second|evening|| +79127|AAAAAAAAIBFDBAAA|79127|21|58|47|PM|second|evening|| +79128|AAAAAAAAJBFDBAAA|79128|21|58|48|PM|second|evening|| +79129|AAAAAAAAKBFDBAAA|79129|21|58|49|PM|second|evening|| +79130|AAAAAAAALBFDBAAA|79130|21|58|50|PM|second|evening|| +79131|AAAAAAAAMBFDBAAA|79131|21|58|51|PM|second|evening|| +79132|AAAAAAAANBFDBAAA|79132|21|58|52|PM|second|evening|| +79133|AAAAAAAAOBFDBAAA|79133|21|58|53|PM|second|evening|| +79134|AAAAAAAAPBFDBAAA|79134|21|58|54|PM|second|evening|| +79135|AAAAAAAAACFDBAAA|79135|21|58|55|PM|second|evening|| +79136|AAAAAAAABCFDBAAA|79136|21|58|56|PM|second|evening|| +79137|AAAAAAAACCFDBAAA|79137|21|58|57|PM|second|evening|| +79138|AAAAAAAADCFDBAAA|79138|21|58|58|PM|second|evening|| +79139|AAAAAAAAECFDBAAA|79139|21|58|59|PM|second|evening|| +79140|AAAAAAAAFCFDBAAA|79140|21|59|0|PM|second|evening|| +79141|AAAAAAAAGCFDBAAA|79141|21|59|1|PM|second|evening|| +79142|AAAAAAAAHCFDBAAA|79142|21|59|2|PM|second|evening|| +79143|AAAAAAAAICFDBAAA|79143|21|59|3|PM|second|evening|| +79144|AAAAAAAAJCFDBAAA|79144|21|59|4|PM|second|evening|| +79145|AAAAAAAAKCFDBAAA|79145|21|59|5|PM|second|evening|| +79146|AAAAAAAALCFDBAAA|79146|21|59|6|PM|second|evening|| +79147|AAAAAAAAMCFDBAAA|79147|21|59|7|PM|second|evening|| +79148|AAAAAAAANCFDBAAA|79148|21|59|8|PM|second|evening|| +79149|AAAAAAAAOCFDBAAA|79149|21|59|9|PM|second|evening|| +79150|AAAAAAAAPCFDBAAA|79150|21|59|10|PM|second|evening|| +79151|AAAAAAAAADFDBAAA|79151|21|59|11|PM|second|evening|| +79152|AAAAAAAABDFDBAAA|79152|21|59|12|PM|second|evening|| +79153|AAAAAAAACDFDBAAA|79153|21|59|13|PM|second|evening|| +79154|AAAAAAAADDFDBAAA|79154|21|59|14|PM|second|evening|| +79155|AAAAAAAAEDFDBAAA|79155|21|59|15|PM|second|evening|| +79156|AAAAAAAAFDFDBAAA|79156|21|59|16|PM|second|evening|| +79157|AAAAAAAAGDFDBAAA|79157|21|59|17|PM|second|evening|| +79158|AAAAAAAAHDFDBAAA|79158|21|59|18|PM|second|evening|| +79159|AAAAAAAAIDFDBAAA|79159|21|59|19|PM|second|evening|| +79160|AAAAAAAAJDFDBAAA|79160|21|59|20|PM|second|evening|| +79161|AAAAAAAAKDFDBAAA|79161|21|59|21|PM|second|evening|| +79162|AAAAAAAALDFDBAAA|79162|21|59|22|PM|second|evening|| +79163|AAAAAAAAMDFDBAAA|79163|21|59|23|PM|second|evening|| +79164|AAAAAAAANDFDBAAA|79164|21|59|24|PM|second|evening|| +79165|AAAAAAAAODFDBAAA|79165|21|59|25|PM|second|evening|| +79166|AAAAAAAAPDFDBAAA|79166|21|59|26|PM|second|evening|| +79167|AAAAAAAAAEFDBAAA|79167|21|59|27|PM|second|evening|| +79168|AAAAAAAABEFDBAAA|79168|21|59|28|PM|second|evening|| +79169|AAAAAAAACEFDBAAA|79169|21|59|29|PM|second|evening|| +79170|AAAAAAAADEFDBAAA|79170|21|59|30|PM|second|evening|| +79171|AAAAAAAAEEFDBAAA|79171|21|59|31|PM|second|evening|| +79172|AAAAAAAAFEFDBAAA|79172|21|59|32|PM|second|evening|| +79173|AAAAAAAAGEFDBAAA|79173|21|59|33|PM|second|evening|| +79174|AAAAAAAAHEFDBAAA|79174|21|59|34|PM|second|evening|| +79175|AAAAAAAAIEFDBAAA|79175|21|59|35|PM|second|evening|| +79176|AAAAAAAAJEFDBAAA|79176|21|59|36|PM|second|evening|| +79177|AAAAAAAAKEFDBAAA|79177|21|59|37|PM|second|evening|| +79178|AAAAAAAALEFDBAAA|79178|21|59|38|PM|second|evening|| +79179|AAAAAAAAMEFDBAAA|79179|21|59|39|PM|second|evening|| +79180|AAAAAAAANEFDBAAA|79180|21|59|40|PM|second|evening|| +79181|AAAAAAAAOEFDBAAA|79181|21|59|41|PM|second|evening|| +79182|AAAAAAAAPEFDBAAA|79182|21|59|42|PM|second|evening|| +79183|AAAAAAAAAFFDBAAA|79183|21|59|43|PM|second|evening|| +79184|AAAAAAAABFFDBAAA|79184|21|59|44|PM|second|evening|| +79185|AAAAAAAACFFDBAAA|79185|21|59|45|PM|second|evening|| +79186|AAAAAAAADFFDBAAA|79186|21|59|46|PM|second|evening|| +79187|AAAAAAAAEFFDBAAA|79187|21|59|47|PM|second|evening|| +79188|AAAAAAAAFFFDBAAA|79188|21|59|48|PM|second|evening|| +79189|AAAAAAAAGFFDBAAA|79189|21|59|49|PM|second|evening|| +79190|AAAAAAAAHFFDBAAA|79190|21|59|50|PM|second|evening|| +79191|AAAAAAAAIFFDBAAA|79191|21|59|51|PM|second|evening|| +79192|AAAAAAAAJFFDBAAA|79192|21|59|52|PM|second|evening|| +79193|AAAAAAAAKFFDBAAA|79193|21|59|53|PM|second|evening|| +79194|AAAAAAAALFFDBAAA|79194|21|59|54|PM|second|evening|| +79195|AAAAAAAAMFFDBAAA|79195|21|59|55|PM|second|evening|| +79196|AAAAAAAANFFDBAAA|79196|21|59|56|PM|second|evening|| +79197|AAAAAAAAOFFDBAAA|79197|21|59|57|PM|second|evening|| +79198|AAAAAAAAPFFDBAAA|79198|21|59|58|PM|second|evening|| +79199|AAAAAAAAAGFDBAAA|79199|21|59|59|PM|second|evening|| +79200|AAAAAAAABGFDBAAA|79200|22|0|0|PM|second|evening|| +79201|AAAAAAAACGFDBAAA|79201|22|0|1|PM|second|evening|| +79202|AAAAAAAADGFDBAAA|79202|22|0|2|PM|second|evening|| +79203|AAAAAAAAEGFDBAAA|79203|22|0|3|PM|second|evening|| +79204|AAAAAAAAFGFDBAAA|79204|22|0|4|PM|second|evening|| +79205|AAAAAAAAGGFDBAAA|79205|22|0|5|PM|second|evening|| +79206|AAAAAAAAHGFDBAAA|79206|22|0|6|PM|second|evening|| +79207|AAAAAAAAIGFDBAAA|79207|22|0|7|PM|second|evening|| +79208|AAAAAAAAJGFDBAAA|79208|22|0|8|PM|second|evening|| +79209|AAAAAAAAKGFDBAAA|79209|22|0|9|PM|second|evening|| +79210|AAAAAAAALGFDBAAA|79210|22|0|10|PM|second|evening|| +79211|AAAAAAAAMGFDBAAA|79211|22|0|11|PM|second|evening|| +79212|AAAAAAAANGFDBAAA|79212|22|0|12|PM|second|evening|| +79213|AAAAAAAAOGFDBAAA|79213|22|0|13|PM|second|evening|| +79214|AAAAAAAAPGFDBAAA|79214|22|0|14|PM|second|evening|| +79215|AAAAAAAAAHFDBAAA|79215|22|0|15|PM|second|evening|| +79216|AAAAAAAABHFDBAAA|79216|22|0|16|PM|second|evening|| +79217|AAAAAAAACHFDBAAA|79217|22|0|17|PM|second|evening|| +79218|AAAAAAAADHFDBAAA|79218|22|0|18|PM|second|evening|| +79219|AAAAAAAAEHFDBAAA|79219|22|0|19|PM|second|evening|| +79220|AAAAAAAAFHFDBAAA|79220|22|0|20|PM|second|evening|| +79221|AAAAAAAAGHFDBAAA|79221|22|0|21|PM|second|evening|| +79222|AAAAAAAAHHFDBAAA|79222|22|0|22|PM|second|evening|| +79223|AAAAAAAAIHFDBAAA|79223|22|0|23|PM|second|evening|| +79224|AAAAAAAAJHFDBAAA|79224|22|0|24|PM|second|evening|| +79225|AAAAAAAAKHFDBAAA|79225|22|0|25|PM|second|evening|| +79226|AAAAAAAALHFDBAAA|79226|22|0|26|PM|second|evening|| +79227|AAAAAAAAMHFDBAAA|79227|22|0|27|PM|second|evening|| +79228|AAAAAAAANHFDBAAA|79228|22|0|28|PM|second|evening|| +79229|AAAAAAAAOHFDBAAA|79229|22|0|29|PM|second|evening|| +79230|AAAAAAAAPHFDBAAA|79230|22|0|30|PM|second|evening|| +79231|AAAAAAAAAIFDBAAA|79231|22|0|31|PM|second|evening|| +79232|AAAAAAAABIFDBAAA|79232|22|0|32|PM|second|evening|| +79233|AAAAAAAACIFDBAAA|79233|22|0|33|PM|second|evening|| +79234|AAAAAAAADIFDBAAA|79234|22|0|34|PM|second|evening|| +79235|AAAAAAAAEIFDBAAA|79235|22|0|35|PM|second|evening|| +79236|AAAAAAAAFIFDBAAA|79236|22|0|36|PM|second|evening|| +79237|AAAAAAAAGIFDBAAA|79237|22|0|37|PM|second|evening|| +79238|AAAAAAAAHIFDBAAA|79238|22|0|38|PM|second|evening|| +79239|AAAAAAAAIIFDBAAA|79239|22|0|39|PM|second|evening|| +79240|AAAAAAAAJIFDBAAA|79240|22|0|40|PM|second|evening|| +79241|AAAAAAAAKIFDBAAA|79241|22|0|41|PM|second|evening|| +79242|AAAAAAAALIFDBAAA|79242|22|0|42|PM|second|evening|| +79243|AAAAAAAAMIFDBAAA|79243|22|0|43|PM|second|evening|| +79244|AAAAAAAANIFDBAAA|79244|22|0|44|PM|second|evening|| +79245|AAAAAAAAOIFDBAAA|79245|22|0|45|PM|second|evening|| +79246|AAAAAAAAPIFDBAAA|79246|22|0|46|PM|second|evening|| +79247|AAAAAAAAAJFDBAAA|79247|22|0|47|PM|second|evening|| +79248|AAAAAAAABJFDBAAA|79248|22|0|48|PM|second|evening|| +79249|AAAAAAAACJFDBAAA|79249|22|0|49|PM|second|evening|| +79250|AAAAAAAADJFDBAAA|79250|22|0|50|PM|second|evening|| +79251|AAAAAAAAEJFDBAAA|79251|22|0|51|PM|second|evening|| +79252|AAAAAAAAFJFDBAAA|79252|22|0|52|PM|second|evening|| +79253|AAAAAAAAGJFDBAAA|79253|22|0|53|PM|second|evening|| +79254|AAAAAAAAHJFDBAAA|79254|22|0|54|PM|second|evening|| +79255|AAAAAAAAIJFDBAAA|79255|22|0|55|PM|second|evening|| +79256|AAAAAAAAJJFDBAAA|79256|22|0|56|PM|second|evening|| +79257|AAAAAAAAKJFDBAAA|79257|22|0|57|PM|second|evening|| +79258|AAAAAAAALJFDBAAA|79258|22|0|58|PM|second|evening|| +79259|AAAAAAAAMJFDBAAA|79259|22|0|59|PM|second|evening|| +79260|AAAAAAAANJFDBAAA|79260|22|1|0|PM|second|evening|| +79261|AAAAAAAAOJFDBAAA|79261|22|1|1|PM|second|evening|| +79262|AAAAAAAAPJFDBAAA|79262|22|1|2|PM|second|evening|| +79263|AAAAAAAAAKFDBAAA|79263|22|1|3|PM|second|evening|| +79264|AAAAAAAABKFDBAAA|79264|22|1|4|PM|second|evening|| +79265|AAAAAAAACKFDBAAA|79265|22|1|5|PM|second|evening|| +79266|AAAAAAAADKFDBAAA|79266|22|1|6|PM|second|evening|| +79267|AAAAAAAAEKFDBAAA|79267|22|1|7|PM|second|evening|| +79268|AAAAAAAAFKFDBAAA|79268|22|1|8|PM|second|evening|| +79269|AAAAAAAAGKFDBAAA|79269|22|1|9|PM|second|evening|| +79270|AAAAAAAAHKFDBAAA|79270|22|1|10|PM|second|evening|| +79271|AAAAAAAAIKFDBAAA|79271|22|1|11|PM|second|evening|| +79272|AAAAAAAAJKFDBAAA|79272|22|1|12|PM|second|evening|| +79273|AAAAAAAAKKFDBAAA|79273|22|1|13|PM|second|evening|| +79274|AAAAAAAALKFDBAAA|79274|22|1|14|PM|second|evening|| +79275|AAAAAAAAMKFDBAAA|79275|22|1|15|PM|second|evening|| +79276|AAAAAAAANKFDBAAA|79276|22|1|16|PM|second|evening|| +79277|AAAAAAAAOKFDBAAA|79277|22|1|17|PM|second|evening|| +79278|AAAAAAAAPKFDBAAA|79278|22|1|18|PM|second|evening|| +79279|AAAAAAAAALFDBAAA|79279|22|1|19|PM|second|evening|| +79280|AAAAAAAABLFDBAAA|79280|22|1|20|PM|second|evening|| +79281|AAAAAAAACLFDBAAA|79281|22|1|21|PM|second|evening|| +79282|AAAAAAAADLFDBAAA|79282|22|1|22|PM|second|evening|| +79283|AAAAAAAAELFDBAAA|79283|22|1|23|PM|second|evening|| +79284|AAAAAAAAFLFDBAAA|79284|22|1|24|PM|second|evening|| +79285|AAAAAAAAGLFDBAAA|79285|22|1|25|PM|second|evening|| +79286|AAAAAAAAHLFDBAAA|79286|22|1|26|PM|second|evening|| +79287|AAAAAAAAILFDBAAA|79287|22|1|27|PM|second|evening|| +79288|AAAAAAAAJLFDBAAA|79288|22|1|28|PM|second|evening|| +79289|AAAAAAAAKLFDBAAA|79289|22|1|29|PM|second|evening|| +79290|AAAAAAAALLFDBAAA|79290|22|1|30|PM|second|evening|| +79291|AAAAAAAAMLFDBAAA|79291|22|1|31|PM|second|evening|| +79292|AAAAAAAANLFDBAAA|79292|22|1|32|PM|second|evening|| +79293|AAAAAAAAOLFDBAAA|79293|22|1|33|PM|second|evening|| +79294|AAAAAAAAPLFDBAAA|79294|22|1|34|PM|second|evening|| +79295|AAAAAAAAAMFDBAAA|79295|22|1|35|PM|second|evening|| +79296|AAAAAAAABMFDBAAA|79296|22|1|36|PM|second|evening|| +79297|AAAAAAAACMFDBAAA|79297|22|1|37|PM|second|evening|| +79298|AAAAAAAADMFDBAAA|79298|22|1|38|PM|second|evening|| +79299|AAAAAAAAEMFDBAAA|79299|22|1|39|PM|second|evening|| +79300|AAAAAAAAFMFDBAAA|79300|22|1|40|PM|second|evening|| +79301|AAAAAAAAGMFDBAAA|79301|22|1|41|PM|second|evening|| +79302|AAAAAAAAHMFDBAAA|79302|22|1|42|PM|second|evening|| +79303|AAAAAAAAIMFDBAAA|79303|22|1|43|PM|second|evening|| +79304|AAAAAAAAJMFDBAAA|79304|22|1|44|PM|second|evening|| +79305|AAAAAAAAKMFDBAAA|79305|22|1|45|PM|second|evening|| +79306|AAAAAAAALMFDBAAA|79306|22|1|46|PM|second|evening|| +79307|AAAAAAAAMMFDBAAA|79307|22|1|47|PM|second|evening|| +79308|AAAAAAAANMFDBAAA|79308|22|1|48|PM|second|evening|| +79309|AAAAAAAAOMFDBAAA|79309|22|1|49|PM|second|evening|| +79310|AAAAAAAAPMFDBAAA|79310|22|1|50|PM|second|evening|| +79311|AAAAAAAAANFDBAAA|79311|22|1|51|PM|second|evening|| +79312|AAAAAAAABNFDBAAA|79312|22|1|52|PM|second|evening|| +79313|AAAAAAAACNFDBAAA|79313|22|1|53|PM|second|evening|| +79314|AAAAAAAADNFDBAAA|79314|22|1|54|PM|second|evening|| +79315|AAAAAAAAENFDBAAA|79315|22|1|55|PM|second|evening|| +79316|AAAAAAAAFNFDBAAA|79316|22|1|56|PM|second|evening|| +79317|AAAAAAAAGNFDBAAA|79317|22|1|57|PM|second|evening|| +79318|AAAAAAAAHNFDBAAA|79318|22|1|58|PM|second|evening|| +79319|AAAAAAAAINFDBAAA|79319|22|1|59|PM|second|evening|| +79320|AAAAAAAAJNFDBAAA|79320|22|2|0|PM|second|evening|| +79321|AAAAAAAAKNFDBAAA|79321|22|2|1|PM|second|evening|| +79322|AAAAAAAALNFDBAAA|79322|22|2|2|PM|second|evening|| +79323|AAAAAAAAMNFDBAAA|79323|22|2|3|PM|second|evening|| +79324|AAAAAAAANNFDBAAA|79324|22|2|4|PM|second|evening|| +79325|AAAAAAAAONFDBAAA|79325|22|2|5|PM|second|evening|| +79326|AAAAAAAAPNFDBAAA|79326|22|2|6|PM|second|evening|| +79327|AAAAAAAAAOFDBAAA|79327|22|2|7|PM|second|evening|| +79328|AAAAAAAABOFDBAAA|79328|22|2|8|PM|second|evening|| +79329|AAAAAAAACOFDBAAA|79329|22|2|9|PM|second|evening|| +79330|AAAAAAAADOFDBAAA|79330|22|2|10|PM|second|evening|| +79331|AAAAAAAAEOFDBAAA|79331|22|2|11|PM|second|evening|| +79332|AAAAAAAAFOFDBAAA|79332|22|2|12|PM|second|evening|| +79333|AAAAAAAAGOFDBAAA|79333|22|2|13|PM|second|evening|| +79334|AAAAAAAAHOFDBAAA|79334|22|2|14|PM|second|evening|| +79335|AAAAAAAAIOFDBAAA|79335|22|2|15|PM|second|evening|| +79336|AAAAAAAAJOFDBAAA|79336|22|2|16|PM|second|evening|| +79337|AAAAAAAAKOFDBAAA|79337|22|2|17|PM|second|evening|| +79338|AAAAAAAALOFDBAAA|79338|22|2|18|PM|second|evening|| +79339|AAAAAAAAMOFDBAAA|79339|22|2|19|PM|second|evening|| +79340|AAAAAAAANOFDBAAA|79340|22|2|20|PM|second|evening|| +79341|AAAAAAAAOOFDBAAA|79341|22|2|21|PM|second|evening|| +79342|AAAAAAAAPOFDBAAA|79342|22|2|22|PM|second|evening|| +79343|AAAAAAAAAPFDBAAA|79343|22|2|23|PM|second|evening|| +79344|AAAAAAAABPFDBAAA|79344|22|2|24|PM|second|evening|| +79345|AAAAAAAACPFDBAAA|79345|22|2|25|PM|second|evening|| +79346|AAAAAAAADPFDBAAA|79346|22|2|26|PM|second|evening|| +79347|AAAAAAAAEPFDBAAA|79347|22|2|27|PM|second|evening|| +79348|AAAAAAAAFPFDBAAA|79348|22|2|28|PM|second|evening|| +79349|AAAAAAAAGPFDBAAA|79349|22|2|29|PM|second|evening|| +79350|AAAAAAAAHPFDBAAA|79350|22|2|30|PM|second|evening|| +79351|AAAAAAAAIPFDBAAA|79351|22|2|31|PM|second|evening|| +79352|AAAAAAAAJPFDBAAA|79352|22|2|32|PM|second|evening|| +79353|AAAAAAAAKPFDBAAA|79353|22|2|33|PM|second|evening|| +79354|AAAAAAAALPFDBAAA|79354|22|2|34|PM|second|evening|| +79355|AAAAAAAAMPFDBAAA|79355|22|2|35|PM|second|evening|| +79356|AAAAAAAANPFDBAAA|79356|22|2|36|PM|second|evening|| +79357|AAAAAAAAOPFDBAAA|79357|22|2|37|PM|second|evening|| +79358|AAAAAAAAPPFDBAAA|79358|22|2|38|PM|second|evening|| +79359|AAAAAAAAAAGDBAAA|79359|22|2|39|PM|second|evening|| +79360|AAAAAAAABAGDBAAA|79360|22|2|40|PM|second|evening|| +79361|AAAAAAAACAGDBAAA|79361|22|2|41|PM|second|evening|| +79362|AAAAAAAADAGDBAAA|79362|22|2|42|PM|second|evening|| +79363|AAAAAAAAEAGDBAAA|79363|22|2|43|PM|second|evening|| +79364|AAAAAAAAFAGDBAAA|79364|22|2|44|PM|second|evening|| +79365|AAAAAAAAGAGDBAAA|79365|22|2|45|PM|second|evening|| +79366|AAAAAAAAHAGDBAAA|79366|22|2|46|PM|second|evening|| +79367|AAAAAAAAIAGDBAAA|79367|22|2|47|PM|second|evening|| +79368|AAAAAAAAJAGDBAAA|79368|22|2|48|PM|second|evening|| +79369|AAAAAAAAKAGDBAAA|79369|22|2|49|PM|second|evening|| +79370|AAAAAAAALAGDBAAA|79370|22|2|50|PM|second|evening|| +79371|AAAAAAAAMAGDBAAA|79371|22|2|51|PM|second|evening|| +79372|AAAAAAAANAGDBAAA|79372|22|2|52|PM|second|evening|| +79373|AAAAAAAAOAGDBAAA|79373|22|2|53|PM|second|evening|| +79374|AAAAAAAAPAGDBAAA|79374|22|2|54|PM|second|evening|| +79375|AAAAAAAAABGDBAAA|79375|22|2|55|PM|second|evening|| +79376|AAAAAAAABBGDBAAA|79376|22|2|56|PM|second|evening|| +79377|AAAAAAAACBGDBAAA|79377|22|2|57|PM|second|evening|| +79378|AAAAAAAADBGDBAAA|79378|22|2|58|PM|second|evening|| +79379|AAAAAAAAEBGDBAAA|79379|22|2|59|PM|second|evening|| +79380|AAAAAAAAFBGDBAAA|79380|22|3|0|PM|second|evening|| +79381|AAAAAAAAGBGDBAAA|79381|22|3|1|PM|second|evening|| +79382|AAAAAAAAHBGDBAAA|79382|22|3|2|PM|second|evening|| +79383|AAAAAAAAIBGDBAAA|79383|22|3|3|PM|second|evening|| +79384|AAAAAAAAJBGDBAAA|79384|22|3|4|PM|second|evening|| +79385|AAAAAAAAKBGDBAAA|79385|22|3|5|PM|second|evening|| +79386|AAAAAAAALBGDBAAA|79386|22|3|6|PM|second|evening|| +79387|AAAAAAAAMBGDBAAA|79387|22|3|7|PM|second|evening|| +79388|AAAAAAAANBGDBAAA|79388|22|3|8|PM|second|evening|| +79389|AAAAAAAAOBGDBAAA|79389|22|3|9|PM|second|evening|| +79390|AAAAAAAAPBGDBAAA|79390|22|3|10|PM|second|evening|| +79391|AAAAAAAAACGDBAAA|79391|22|3|11|PM|second|evening|| +79392|AAAAAAAABCGDBAAA|79392|22|3|12|PM|second|evening|| +79393|AAAAAAAACCGDBAAA|79393|22|3|13|PM|second|evening|| +79394|AAAAAAAADCGDBAAA|79394|22|3|14|PM|second|evening|| +79395|AAAAAAAAECGDBAAA|79395|22|3|15|PM|second|evening|| +79396|AAAAAAAAFCGDBAAA|79396|22|3|16|PM|second|evening|| +79397|AAAAAAAAGCGDBAAA|79397|22|3|17|PM|second|evening|| +79398|AAAAAAAAHCGDBAAA|79398|22|3|18|PM|second|evening|| +79399|AAAAAAAAICGDBAAA|79399|22|3|19|PM|second|evening|| +79400|AAAAAAAAJCGDBAAA|79400|22|3|20|PM|second|evening|| +79401|AAAAAAAAKCGDBAAA|79401|22|3|21|PM|second|evening|| +79402|AAAAAAAALCGDBAAA|79402|22|3|22|PM|second|evening|| +79403|AAAAAAAAMCGDBAAA|79403|22|3|23|PM|second|evening|| +79404|AAAAAAAANCGDBAAA|79404|22|3|24|PM|second|evening|| +79405|AAAAAAAAOCGDBAAA|79405|22|3|25|PM|second|evening|| +79406|AAAAAAAAPCGDBAAA|79406|22|3|26|PM|second|evening|| +79407|AAAAAAAAADGDBAAA|79407|22|3|27|PM|second|evening|| +79408|AAAAAAAABDGDBAAA|79408|22|3|28|PM|second|evening|| +79409|AAAAAAAACDGDBAAA|79409|22|3|29|PM|second|evening|| +79410|AAAAAAAADDGDBAAA|79410|22|3|30|PM|second|evening|| +79411|AAAAAAAAEDGDBAAA|79411|22|3|31|PM|second|evening|| +79412|AAAAAAAAFDGDBAAA|79412|22|3|32|PM|second|evening|| +79413|AAAAAAAAGDGDBAAA|79413|22|3|33|PM|second|evening|| +79414|AAAAAAAAHDGDBAAA|79414|22|3|34|PM|second|evening|| +79415|AAAAAAAAIDGDBAAA|79415|22|3|35|PM|second|evening|| +79416|AAAAAAAAJDGDBAAA|79416|22|3|36|PM|second|evening|| +79417|AAAAAAAAKDGDBAAA|79417|22|3|37|PM|second|evening|| +79418|AAAAAAAALDGDBAAA|79418|22|3|38|PM|second|evening|| +79419|AAAAAAAAMDGDBAAA|79419|22|3|39|PM|second|evening|| +79420|AAAAAAAANDGDBAAA|79420|22|3|40|PM|second|evening|| +79421|AAAAAAAAODGDBAAA|79421|22|3|41|PM|second|evening|| +79422|AAAAAAAAPDGDBAAA|79422|22|3|42|PM|second|evening|| +79423|AAAAAAAAAEGDBAAA|79423|22|3|43|PM|second|evening|| +79424|AAAAAAAABEGDBAAA|79424|22|3|44|PM|second|evening|| +79425|AAAAAAAACEGDBAAA|79425|22|3|45|PM|second|evening|| +79426|AAAAAAAADEGDBAAA|79426|22|3|46|PM|second|evening|| +79427|AAAAAAAAEEGDBAAA|79427|22|3|47|PM|second|evening|| +79428|AAAAAAAAFEGDBAAA|79428|22|3|48|PM|second|evening|| +79429|AAAAAAAAGEGDBAAA|79429|22|3|49|PM|second|evening|| +79430|AAAAAAAAHEGDBAAA|79430|22|3|50|PM|second|evening|| +79431|AAAAAAAAIEGDBAAA|79431|22|3|51|PM|second|evening|| +79432|AAAAAAAAJEGDBAAA|79432|22|3|52|PM|second|evening|| +79433|AAAAAAAAKEGDBAAA|79433|22|3|53|PM|second|evening|| +79434|AAAAAAAALEGDBAAA|79434|22|3|54|PM|second|evening|| +79435|AAAAAAAAMEGDBAAA|79435|22|3|55|PM|second|evening|| +79436|AAAAAAAANEGDBAAA|79436|22|3|56|PM|second|evening|| +79437|AAAAAAAAOEGDBAAA|79437|22|3|57|PM|second|evening|| +79438|AAAAAAAAPEGDBAAA|79438|22|3|58|PM|second|evening|| +79439|AAAAAAAAAFGDBAAA|79439|22|3|59|PM|second|evening|| +79440|AAAAAAAABFGDBAAA|79440|22|4|0|PM|second|evening|| +79441|AAAAAAAACFGDBAAA|79441|22|4|1|PM|second|evening|| +79442|AAAAAAAADFGDBAAA|79442|22|4|2|PM|second|evening|| +79443|AAAAAAAAEFGDBAAA|79443|22|4|3|PM|second|evening|| +79444|AAAAAAAAFFGDBAAA|79444|22|4|4|PM|second|evening|| +79445|AAAAAAAAGFGDBAAA|79445|22|4|5|PM|second|evening|| +79446|AAAAAAAAHFGDBAAA|79446|22|4|6|PM|second|evening|| +79447|AAAAAAAAIFGDBAAA|79447|22|4|7|PM|second|evening|| +79448|AAAAAAAAJFGDBAAA|79448|22|4|8|PM|second|evening|| +79449|AAAAAAAAKFGDBAAA|79449|22|4|9|PM|second|evening|| +79450|AAAAAAAALFGDBAAA|79450|22|4|10|PM|second|evening|| +79451|AAAAAAAAMFGDBAAA|79451|22|4|11|PM|second|evening|| +79452|AAAAAAAANFGDBAAA|79452|22|4|12|PM|second|evening|| +79453|AAAAAAAAOFGDBAAA|79453|22|4|13|PM|second|evening|| +79454|AAAAAAAAPFGDBAAA|79454|22|4|14|PM|second|evening|| +79455|AAAAAAAAAGGDBAAA|79455|22|4|15|PM|second|evening|| +79456|AAAAAAAABGGDBAAA|79456|22|4|16|PM|second|evening|| +79457|AAAAAAAACGGDBAAA|79457|22|4|17|PM|second|evening|| +79458|AAAAAAAADGGDBAAA|79458|22|4|18|PM|second|evening|| +79459|AAAAAAAAEGGDBAAA|79459|22|4|19|PM|second|evening|| +79460|AAAAAAAAFGGDBAAA|79460|22|4|20|PM|second|evening|| +79461|AAAAAAAAGGGDBAAA|79461|22|4|21|PM|second|evening|| +79462|AAAAAAAAHGGDBAAA|79462|22|4|22|PM|second|evening|| +79463|AAAAAAAAIGGDBAAA|79463|22|4|23|PM|second|evening|| +79464|AAAAAAAAJGGDBAAA|79464|22|4|24|PM|second|evening|| +79465|AAAAAAAAKGGDBAAA|79465|22|4|25|PM|second|evening|| +79466|AAAAAAAALGGDBAAA|79466|22|4|26|PM|second|evening|| +79467|AAAAAAAAMGGDBAAA|79467|22|4|27|PM|second|evening|| +79468|AAAAAAAANGGDBAAA|79468|22|4|28|PM|second|evening|| +79469|AAAAAAAAOGGDBAAA|79469|22|4|29|PM|second|evening|| +79470|AAAAAAAAPGGDBAAA|79470|22|4|30|PM|second|evening|| +79471|AAAAAAAAAHGDBAAA|79471|22|4|31|PM|second|evening|| +79472|AAAAAAAABHGDBAAA|79472|22|4|32|PM|second|evening|| +79473|AAAAAAAACHGDBAAA|79473|22|4|33|PM|second|evening|| +79474|AAAAAAAADHGDBAAA|79474|22|4|34|PM|second|evening|| +79475|AAAAAAAAEHGDBAAA|79475|22|4|35|PM|second|evening|| +79476|AAAAAAAAFHGDBAAA|79476|22|4|36|PM|second|evening|| +79477|AAAAAAAAGHGDBAAA|79477|22|4|37|PM|second|evening|| +79478|AAAAAAAAHHGDBAAA|79478|22|4|38|PM|second|evening|| +79479|AAAAAAAAIHGDBAAA|79479|22|4|39|PM|second|evening|| +79480|AAAAAAAAJHGDBAAA|79480|22|4|40|PM|second|evening|| +79481|AAAAAAAAKHGDBAAA|79481|22|4|41|PM|second|evening|| +79482|AAAAAAAALHGDBAAA|79482|22|4|42|PM|second|evening|| +79483|AAAAAAAAMHGDBAAA|79483|22|4|43|PM|second|evening|| +79484|AAAAAAAANHGDBAAA|79484|22|4|44|PM|second|evening|| +79485|AAAAAAAAOHGDBAAA|79485|22|4|45|PM|second|evening|| +79486|AAAAAAAAPHGDBAAA|79486|22|4|46|PM|second|evening|| +79487|AAAAAAAAAIGDBAAA|79487|22|4|47|PM|second|evening|| +79488|AAAAAAAABIGDBAAA|79488|22|4|48|PM|second|evening|| +79489|AAAAAAAACIGDBAAA|79489|22|4|49|PM|second|evening|| +79490|AAAAAAAADIGDBAAA|79490|22|4|50|PM|second|evening|| +79491|AAAAAAAAEIGDBAAA|79491|22|4|51|PM|second|evening|| +79492|AAAAAAAAFIGDBAAA|79492|22|4|52|PM|second|evening|| +79493|AAAAAAAAGIGDBAAA|79493|22|4|53|PM|second|evening|| +79494|AAAAAAAAHIGDBAAA|79494|22|4|54|PM|second|evening|| +79495|AAAAAAAAIIGDBAAA|79495|22|4|55|PM|second|evening|| +79496|AAAAAAAAJIGDBAAA|79496|22|4|56|PM|second|evening|| +79497|AAAAAAAAKIGDBAAA|79497|22|4|57|PM|second|evening|| +79498|AAAAAAAALIGDBAAA|79498|22|4|58|PM|second|evening|| +79499|AAAAAAAAMIGDBAAA|79499|22|4|59|PM|second|evening|| +79500|AAAAAAAANIGDBAAA|79500|22|5|0|PM|second|evening|| +79501|AAAAAAAAOIGDBAAA|79501|22|5|1|PM|second|evening|| +79502|AAAAAAAAPIGDBAAA|79502|22|5|2|PM|second|evening|| +79503|AAAAAAAAAJGDBAAA|79503|22|5|3|PM|second|evening|| +79504|AAAAAAAABJGDBAAA|79504|22|5|4|PM|second|evening|| +79505|AAAAAAAACJGDBAAA|79505|22|5|5|PM|second|evening|| +79506|AAAAAAAADJGDBAAA|79506|22|5|6|PM|second|evening|| +79507|AAAAAAAAEJGDBAAA|79507|22|5|7|PM|second|evening|| +79508|AAAAAAAAFJGDBAAA|79508|22|5|8|PM|second|evening|| +79509|AAAAAAAAGJGDBAAA|79509|22|5|9|PM|second|evening|| +79510|AAAAAAAAHJGDBAAA|79510|22|5|10|PM|second|evening|| +79511|AAAAAAAAIJGDBAAA|79511|22|5|11|PM|second|evening|| +79512|AAAAAAAAJJGDBAAA|79512|22|5|12|PM|second|evening|| +79513|AAAAAAAAKJGDBAAA|79513|22|5|13|PM|second|evening|| +79514|AAAAAAAALJGDBAAA|79514|22|5|14|PM|second|evening|| +79515|AAAAAAAAMJGDBAAA|79515|22|5|15|PM|second|evening|| +79516|AAAAAAAANJGDBAAA|79516|22|5|16|PM|second|evening|| +79517|AAAAAAAAOJGDBAAA|79517|22|5|17|PM|second|evening|| +79518|AAAAAAAAPJGDBAAA|79518|22|5|18|PM|second|evening|| +79519|AAAAAAAAAKGDBAAA|79519|22|5|19|PM|second|evening|| +79520|AAAAAAAABKGDBAAA|79520|22|5|20|PM|second|evening|| +79521|AAAAAAAACKGDBAAA|79521|22|5|21|PM|second|evening|| +79522|AAAAAAAADKGDBAAA|79522|22|5|22|PM|second|evening|| +79523|AAAAAAAAEKGDBAAA|79523|22|5|23|PM|second|evening|| +79524|AAAAAAAAFKGDBAAA|79524|22|5|24|PM|second|evening|| +79525|AAAAAAAAGKGDBAAA|79525|22|5|25|PM|second|evening|| +79526|AAAAAAAAHKGDBAAA|79526|22|5|26|PM|second|evening|| +79527|AAAAAAAAIKGDBAAA|79527|22|5|27|PM|second|evening|| +79528|AAAAAAAAJKGDBAAA|79528|22|5|28|PM|second|evening|| +79529|AAAAAAAAKKGDBAAA|79529|22|5|29|PM|second|evening|| +79530|AAAAAAAALKGDBAAA|79530|22|5|30|PM|second|evening|| +79531|AAAAAAAAMKGDBAAA|79531|22|5|31|PM|second|evening|| +79532|AAAAAAAANKGDBAAA|79532|22|5|32|PM|second|evening|| +79533|AAAAAAAAOKGDBAAA|79533|22|5|33|PM|second|evening|| +79534|AAAAAAAAPKGDBAAA|79534|22|5|34|PM|second|evening|| +79535|AAAAAAAAALGDBAAA|79535|22|5|35|PM|second|evening|| +79536|AAAAAAAABLGDBAAA|79536|22|5|36|PM|second|evening|| +79537|AAAAAAAACLGDBAAA|79537|22|5|37|PM|second|evening|| +79538|AAAAAAAADLGDBAAA|79538|22|5|38|PM|second|evening|| +79539|AAAAAAAAELGDBAAA|79539|22|5|39|PM|second|evening|| +79540|AAAAAAAAFLGDBAAA|79540|22|5|40|PM|second|evening|| +79541|AAAAAAAAGLGDBAAA|79541|22|5|41|PM|second|evening|| +79542|AAAAAAAAHLGDBAAA|79542|22|5|42|PM|second|evening|| +79543|AAAAAAAAILGDBAAA|79543|22|5|43|PM|second|evening|| +79544|AAAAAAAAJLGDBAAA|79544|22|5|44|PM|second|evening|| +79545|AAAAAAAAKLGDBAAA|79545|22|5|45|PM|second|evening|| +79546|AAAAAAAALLGDBAAA|79546|22|5|46|PM|second|evening|| +79547|AAAAAAAAMLGDBAAA|79547|22|5|47|PM|second|evening|| +79548|AAAAAAAANLGDBAAA|79548|22|5|48|PM|second|evening|| +79549|AAAAAAAAOLGDBAAA|79549|22|5|49|PM|second|evening|| +79550|AAAAAAAAPLGDBAAA|79550|22|5|50|PM|second|evening|| +79551|AAAAAAAAAMGDBAAA|79551|22|5|51|PM|second|evening|| +79552|AAAAAAAABMGDBAAA|79552|22|5|52|PM|second|evening|| +79553|AAAAAAAACMGDBAAA|79553|22|5|53|PM|second|evening|| +79554|AAAAAAAADMGDBAAA|79554|22|5|54|PM|second|evening|| +79555|AAAAAAAAEMGDBAAA|79555|22|5|55|PM|second|evening|| +79556|AAAAAAAAFMGDBAAA|79556|22|5|56|PM|second|evening|| +79557|AAAAAAAAGMGDBAAA|79557|22|5|57|PM|second|evening|| +79558|AAAAAAAAHMGDBAAA|79558|22|5|58|PM|second|evening|| +79559|AAAAAAAAIMGDBAAA|79559|22|5|59|PM|second|evening|| +79560|AAAAAAAAJMGDBAAA|79560|22|6|0|PM|second|evening|| +79561|AAAAAAAAKMGDBAAA|79561|22|6|1|PM|second|evening|| +79562|AAAAAAAALMGDBAAA|79562|22|6|2|PM|second|evening|| +79563|AAAAAAAAMMGDBAAA|79563|22|6|3|PM|second|evening|| +79564|AAAAAAAANMGDBAAA|79564|22|6|4|PM|second|evening|| +79565|AAAAAAAAOMGDBAAA|79565|22|6|5|PM|second|evening|| +79566|AAAAAAAAPMGDBAAA|79566|22|6|6|PM|second|evening|| +79567|AAAAAAAAANGDBAAA|79567|22|6|7|PM|second|evening|| +79568|AAAAAAAABNGDBAAA|79568|22|6|8|PM|second|evening|| +79569|AAAAAAAACNGDBAAA|79569|22|6|9|PM|second|evening|| +79570|AAAAAAAADNGDBAAA|79570|22|6|10|PM|second|evening|| +79571|AAAAAAAAENGDBAAA|79571|22|6|11|PM|second|evening|| +79572|AAAAAAAAFNGDBAAA|79572|22|6|12|PM|second|evening|| +79573|AAAAAAAAGNGDBAAA|79573|22|6|13|PM|second|evening|| +79574|AAAAAAAAHNGDBAAA|79574|22|6|14|PM|second|evening|| +79575|AAAAAAAAINGDBAAA|79575|22|6|15|PM|second|evening|| +79576|AAAAAAAAJNGDBAAA|79576|22|6|16|PM|second|evening|| +79577|AAAAAAAAKNGDBAAA|79577|22|6|17|PM|second|evening|| +79578|AAAAAAAALNGDBAAA|79578|22|6|18|PM|second|evening|| +79579|AAAAAAAAMNGDBAAA|79579|22|6|19|PM|second|evening|| +79580|AAAAAAAANNGDBAAA|79580|22|6|20|PM|second|evening|| +79581|AAAAAAAAONGDBAAA|79581|22|6|21|PM|second|evening|| +79582|AAAAAAAAPNGDBAAA|79582|22|6|22|PM|second|evening|| +79583|AAAAAAAAAOGDBAAA|79583|22|6|23|PM|second|evening|| +79584|AAAAAAAABOGDBAAA|79584|22|6|24|PM|second|evening|| +79585|AAAAAAAACOGDBAAA|79585|22|6|25|PM|second|evening|| +79586|AAAAAAAADOGDBAAA|79586|22|6|26|PM|second|evening|| +79587|AAAAAAAAEOGDBAAA|79587|22|6|27|PM|second|evening|| +79588|AAAAAAAAFOGDBAAA|79588|22|6|28|PM|second|evening|| +79589|AAAAAAAAGOGDBAAA|79589|22|6|29|PM|second|evening|| +79590|AAAAAAAAHOGDBAAA|79590|22|6|30|PM|second|evening|| +79591|AAAAAAAAIOGDBAAA|79591|22|6|31|PM|second|evening|| +79592|AAAAAAAAJOGDBAAA|79592|22|6|32|PM|second|evening|| +79593|AAAAAAAAKOGDBAAA|79593|22|6|33|PM|second|evening|| +79594|AAAAAAAALOGDBAAA|79594|22|6|34|PM|second|evening|| +79595|AAAAAAAAMOGDBAAA|79595|22|6|35|PM|second|evening|| +79596|AAAAAAAANOGDBAAA|79596|22|6|36|PM|second|evening|| +79597|AAAAAAAAOOGDBAAA|79597|22|6|37|PM|second|evening|| +79598|AAAAAAAAPOGDBAAA|79598|22|6|38|PM|second|evening|| +79599|AAAAAAAAAPGDBAAA|79599|22|6|39|PM|second|evening|| +79600|AAAAAAAABPGDBAAA|79600|22|6|40|PM|second|evening|| +79601|AAAAAAAACPGDBAAA|79601|22|6|41|PM|second|evening|| +79602|AAAAAAAADPGDBAAA|79602|22|6|42|PM|second|evening|| +79603|AAAAAAAAEPGDBAAA|79603|22|6|43|PM|second|evening|| +79604|AAAAAAAAFPGDBAAA|79604|22|6|44|PM|second|evening|| +79605|AAAAAAAAGPGDBAAA|79605|22|6|45|PM|second|evening|| +79606|AAAAAAAAHPGDBAAA|79606|22|6|46|PM|second|evening|| +79607|AAAAAAAAIPGDBAAA|79607|22|6|47|PM|second|evening|| +79608|AAAAAAAAJPGDBAAA|79608|22|6|48|PM|second|evening|| +79609|AAAAAAAAKPGDBAAA|79609|22|6|49|PM|second|evening|| +79610|AAAAAAAALPGDBAAA|79610|22|6|50|PM|second|evening|| +79611|AAAAAAAAMPGDBAAA|79611|22|6|51|PM|second|evening|| +79612|AAAAAAAANPGDBAAA|79612|22|6|52|PM|second|evening|| +79613|AAAAAAAAOPGDBAAA|79613|22|6|53|PM|second|evening|| +79614|AAAAAAAAPPGDBAAA|79614|22|6|54|PM|second|evening|| +79615|AAAAAAAAAAHDBAAA|79615|22|6|55|PM|second|evening|| +79616|AAAAAAAABAHDBAAA|79616|22|6|56|PM|second|evening|| +79617|AAAAAAAACAHDBAAA|79617|22|6|57|PM|second|evening|| +79618|AAAAAAAADAHDBAAA|79618|22|6|58|PM|second|evening|| +79619|AAAAAAAAEAHDBAAA|79619|22|6|59|PM|second|evening|| +79620|AAAAAAAAFAHDBAAA|79620|22|7|0|PM|second|evening|| +79621|AAAAAAAAGAHDBAAA|79621|22|7|1|PM|second|evening|| +79622|AAAAAAAAHAHDBAAA|79622|22|7|2|PM|second|evening|| +79623|AAAAAAAAIAHDBAAA|79623|22|7|3|PM|second|evening|| +79624|AAAAAAAAJAHDBAAA|79624|22|7|4|PM|second|evening|| +79625|AAAAAAAAKAHDBAAA|79625|22|7|5|PM|second|evening|| +79626|AAAAAAAALAHDBAAA|79626|22|7|6|PM|second|evening|| +79627|AAAAAAAAMAHDBAAA|79627|22|7|7|PM|second|evening|| +79628|AAAAAAAANAHDBAAA|79628|22|7|8|PM|second|evening|| +79629|AAAAAAAAOAHDBAAA|79629|22|7|9|PM|second|evening|| +79630|AAAAAAAAPAHDBAAA|79630|22|7|10|PM|second|evening|| +79631|AAAAAAAAABHDBAAA|79631|22|7|11|PM|second|evening|| +79632|AAAAAAAABBHDBAAA|79632|22|7|12|PM|second|evening|| +79633|AAAAAAAACBHDBAAA|79633|22|7|13|PM|second|evening|| +79634|AAAAAAAADBHDBAAA|79634|22|7|14|PM|second|evening|| +79635|AAAAAAAAEBHDBAAA|79635|22|7|15|PM|second|evening|| +79636|AAAAAAAAFBHDBAAA|79636|22|7|16|PM|second|evening|| +79637|AAAAAAAAGBHDBAAA|79637|22|7|17|PM|second|evening|| +79638|AAAAAAAAHBHDBAAA|79638|22|7|18|PM|second|evening|| +79639|AAAAAAAAIBHDBAAA|79639|22|7|19|PM|second|evening|| +79640|AAAAAAAAJBHDBAAA|79640|22|7|20|PM|second|evening|| +79641|AAAAAAAAKBHDBAAA|79641|22|7|21|PM|second|evening|| +79642|AAAAAAAALBHDBAAA|79642|22|7|22|PM|second|evening|| +79643|AAAAAAAAMBHDBAAA|79643|22|7|23|PM|second|evening|| +79644|AAAAAAAANBHDBAAA|79644|22|7|24|PM|second|evening|| +79645|AAAAAAAAOBHDBAAA|79645|22|7|25|PM|second|evening|| +79646|AAAAAAAAPBHDBAAA|79646|22|7|26|PM|second|evening|| +79647|AAAAAAAAACHDBAAA|79647|22|7|27|PM|second|evening|| +79648|AAAAAAAABCHDBAAA|79648|22|7|28|PM|second|evening|| +79649|AAAAAAAACCHDBAAA|79649|22|7|29|PM|second|evening|| +79650|AAAAAAAADCHDBAAA|79650|22|7|30|PM|second|evening|| +79651|AAAAAAAAECHDBAAA|79651|22|7|31|PM|second|evening|| +79652|AAAAAAAAFCHDBAAA|79652|22|7|32|PM|second|evening|| +79653|AAAAAAAAGCHDBAAA|79653|22|7|33|PM|second|evening|| +79654|AAAAAAAAHCHDBAAA|79654|22|7|34|PM|second|evening|| +79655|AAAAAAAAICHDBAAA|79655|22|7|35|PM|second|evening|| +79656|AAAAAAAAJCHDBAAA|79656|22|7|36|PM|second|evening|| +79657|AAAAAAAAKCHDBAAA|79657|22|7|37|PM|second|evening|| +79658|AAAAAAAALCHDBAAA|79658|22|7|38|PM|second|evening|| +79659|AAAAAAAAMCHDBAAA|79659|22|7|39|PM|second|evening|| +79660|AAAAAAAANCHDBAAA|79660|22|7|40|PM|second|evening|| +79661|AAAAAAAAOCHDBAAA|79661|22|7|41|PM|second|evening|| +79662|AAAAAAAAPCHDBAAA|79662|22|7|42|PM|second|evening|| +79663|AAAAAAAAADHDBAAA|79663|22|7|43|PM|second|evening|| +79664|AAAAAAAABDHDBAAA|79664|22|7|44|PM|second|evening|| +79665|AAAAAAAACDHDBAAA|79665|22|7|45|PM|second|evening|| +79666|AAAAAAAADDHDBAAA|79666|22|7|46|PM|second|evening|| +79667|AAAAAAAAEDHDBAAA|79667|22|7|47|PM|second|evening|| +79668|AAAAAAAAFDHDBAAA|79668|22|7|48|PM|second|evening|| +79669|AAAAAAAAGDHDBAAA|79669|22|7|49|PM|second|evening|| +79670|AAAAAAAAHDHDBAAA|79670|22|7|50|PM|second|evening|| +79671|AAAAAAAAIDHDBAAA|79671|22|7|51|PM|second|evening|| +79672|AAAAAAAAJDHDBAAA|79672|22|7|52|PM|second|evening|| +79673|AAAAAAAAKDHDBAAA|79673|22|7|53|PM|second|evening|| +79674|AAAAAAAALDHDBAAA|79674|22|7|54|PM|second|evening|| +79675|AAAAAAAAMDHDBAAA|79675|22|7|55|PM|second|evening|| +79676|AAAAAAAANDHDBAAA|79676|22|7|56|PM|second|evening|| +79677|AAAAAAAAODHDBAAA|79677|22|7|57|PM|second|evening|| +79678|AAAAAAAAPDHDBAAA|79678|22|7|58|PM|second|evening|| +79679|AAAAAAAAAEHDBAAA|79679|22|7|59|PM|second|evening|| +79680|AAAAAAAABEHDBAAA|79680|22|8|0|PM|second|evening|| +79681|AAAAAAAACEHDBAAA|79681|22|8|1|PM|second|evening|| +79682|AAAAAAAADEHDBAAA|79682|22|8|2|PM|second|evening|| +79683|AAAAAAAAEEHDBAAA|79683|22|8|3|PM|second|evening|| +79684|AAAAAAAAFEHDBAAA|79684|22|8|4|PM|second|evening|| +79685|AAAAAAAAGEHDBAAA|79685|22|8|5|PM|second|evening|| +79686|AAAAAAAAHEHDBAAA|79686|22|8|6|PM|second|evening|| +79687|AAAAAAAAIEHDBAAA|79687|22|8|7|PM|second|evening|| +79688|AAAAAAAAJEHDBAAA|79688|22|8|8|PM|second|evening|| +79689|AAAAAAAAKEHDBAAA|79689|22|8|9|PM|second|evening|| +79690|AAAAAAAALEHDBAAA|79690|22|8|10|PM|second|evening|| +79691|AAAAAAAAMEHDBAAA|79691|22|8|11|PM|second|evening|| +79692|AAAAAAAANEHDBAAA|79692|22|8|12|PM|second|evening|| +79693|AAAAAAAAOEHDBAAA|79693|22|8|13|PM|second|evening|| +79694|AAAAAAAAPEHDBAAA|79694|22|8|14|PM|second|evening|| +79695|AAAAAAAAAFHDBAAA|79695|22|8|15|PM|second|evening|| +79696|AAAAAAAABFHDBAAA|79696|22|8|16|PM|second|evening|| +79697|AAAAAAAACFHDBAAA|79697|22|8|17|PM|second|evening|| +79698|AAAAAAAADFHDBAAA|79698|22|8|18|PM|second|evening|| +79699|AAAAAAAAEFHDBAAA|79699|22|8|19|PM|second|evening|| +79700|AAAAAAAAFFHDBAAA|79700|22|8|20|PM|second|evening|| +79701|AAAAAAAAGFHDBAAA|79701|22|8|21|PM|second|evening|| +79702|AAAAAAAAHFHDBAAA|79702|22|8|22|PM|second|evening|| +79703|AAAAAAAAIFHDBAAA|79703|22|8|23|PM|second|evening|| +79704|AAAAAAAAJFHDBAAA|79704|22|8|24|PM|second|evening|| +79705|AAAAAAAAKFHDBAAA|79705|22|8|25|PM|second|evening|| +79706|AAAAAAAALFHDBAAA|79706|22|8|26|PM|second|evening|| +79707|AAAAAAAAMFHDBAAA|79707|22|8|27|PM|second|evening|| +79708|AAAAAAAANFHDBAAA|79708|22|8|28|PM|second|evening|| +79709|AAAAAAAAOFHDBAAA|79709|22|8|29|PM|second|evening|| +79710|AAAAAAAAPFHDBAAA|79710|22|8|30|PM|second|evening|| +79711|AAAAAAAAAGHDBAAA|79711|22|8|31|PM|second|evening|| +79712|AAAAAAAABGHDBAAA|79712|22|8|32|PM|second|evening|| +79713|AAAAAAAACGHDBAAA|79713|22|8|33|PM|second|evening|| +79714|AAAAAAAADGHDBAAA|79714|22|8|34|PM|second|evening|| +79715|AAAAAAAAEGHDBAAA|79715|22|8|35|PM|second|evening|| +79716|AAAAAAAAFGHDBAAA|79716|22|8|36|PM|second|evening|| +79717|AAAAAAAAGGHDBAAA|79717|22|8|37|PM|second|evening|| +79718|AAAAAAAAHGHDBAAA|79718|22|8|38|PM|second|evening|| +79719|AAAAAAAAIGHDBAAA|79719|22|8|39|PM|second|evening|| +79720|AAAAAAAAJGHDBAAA|79720|22|8|40|PM|second|evening|| +79721|AAAAAAAAKGHDBAAA|79721|22|8|41|PM|second|evening|| +79722|AAAAAAAALGHDBAAA|79722|22|8|42|PM|second|evening|| +79723|AAAAAAAAMGHDBAAA|79723|22|8|43|PM|second|evening|| +79724|AAAAAAAANGHDBAAA|79724|22|8|44|PM|second|evening|| +79725|AAAAAAAAOGHDBAAA|79725|22|8|45|PM|second|evening|| +79726|AAAAAAAAPGHDBAAA|79726|22|8|46|PM|second|evening|| +79727|AAAAAAAAAHHDBAAA|79727|22|8|47|PM|second|evening|| +79728|AAAAAAAABHHDBAAA|79728|22|8|48|PM|second|evening|| +79729|AAAAAAAACHHDBAAA|79729|22|8|49|PM|second|evening|| +79730|AAAAAAAADHHDBAAA|79730|22|8|50|PM|second|evening|| +79731|AAAAAAAAEHHDBAAA|79731|22|8|51|PM|second|evening|| +79732|AAAAAAAAFHHDBAAA|79732|22|8|52|PM|second|evening|| +79733|AAAAAAAAGHHDBAAA|79733|22|8|53|PM|second|evening|| +79734|AAAAAAAAHHHDBAAA|79734|22|8|54|PM|second|evening|| +79735|AAAAAAAAIHHDBAAA|79735|22|8|55|PM|second|evening|| +79736|AAAAAAAAJHHDBAAA|79736|22|8|56|PM|second|evening|| +79737|AAAAAAAAKHHDBAAA|79737|22|8|57|PM|second|evening|| +79738|AAAAAAAALHHDBAAA|79738|22|8|58|PM|second|evening|| +79739|AAAAAAAAMHHDBAAA|79739|22|8|59|PM|second|evening|| +79740|AAAAAAAANHHDBAAA|79740|22|9|0|PM|second|evening|| +79741|AAAAAAAAOHHDBAAA|79741|22|9|1|PM|second|evening|| +79742|AAAAAAAAPHHDBAAA|79742|22|9|2|PM|second|evening|| +79743|AAAAAAAAAIHDBAAA|79743|22|9|3|PM|second|evening|| +79744|AAAAAAAABIHDBAAA|79744|22|9|4|PM|second|evening|| +79745|AAAAAAAACIHDBAAA|79745|22|9|5|PM|second|evening|| +79746|AAAAAAAADIHDBAAA|79746|22|9|6|PM|second|evening|| +79747|AAAAAAAAEIHDBAAA|79747|22|9|7|PM|second|evening|| +79748|AAAAAAAAFIHDBAAA|79748|22|9|8|PM|second|evening|| +79749|AAAAAAAAGIHDBAAA|79749|22|9|9|PM|second|evening|| +79750|AAAAAAAAHIHDBAAA|79750|22|9|10|PM|second|evening|| +79751|AAAAAAAAIIHDBAAA|79751|22|9|11|PM|second|evening|| +79752|AAAAAAAAJIHDBAAA|79752|22|9|12|PM|second|evening|| +79753|AAAAAAAAKIHDBAAA|79753|22|9|13|PM|second|evening|| +79754|AAAAAAAALIHDBAAA|79754|22|9|14|PM|second|evening|| +79755|AAAAAAAAMIHDBAAA|79755|22|9|15|PM|second|evening|| +79756|AAAAAAAANIHDBAAA|79756|22|9|16|PM|second|evening|| +79757|AAAAAAAAOIHDBAAA|79757|22|9|17|PM|second|evening|| +79758|AAAAAAAAPIHDBAAA|79758|22|9|18|PM|second|evening|| +79759|AAAAAAAAAJHDBAAA|79759|22|9|19|PM|second|evening|| +79760|AAAAAAAABJHDBAAA|79760|22|9|20|PM|second|evening|| +79761|AAAAAAAACJHDBAAA|79761|22|9|21|PM|second|evening|| +79762|AAAAAAAADJHDBAAA|79762|22|9|22|PM|second|evening|| +79763|AAAAAAAAEJHDBAAA|79763|22|9|23|PM|second|evening|| +79764|AAAAAAAAFJHDBAAA|79764|22|9|24|PM|second|evening|| +79765|AAAAAAAAGJHDBAAA|79765|22|9|25|PM|second|evening|| +79766|AAAAAAAAHJHDBAAA|79766|22|9|26|PM|second|evening|| +79767|AAAAAAAAIJHDBAAA|79767|22|9|27|PM|second|evening|| +79768|AAAAAAAAJJHDBAAA|79768|22|9|28|PM|second|evening|| +79769|AAAAAAAAKJHDBAAA|79769|22|9|29|PM|second|evening|| +79770|AAAAAAAALJHDBAAA|79770|22|9|30|PM|second|evening|| +79771|AAAAAAAAMJHDBAAA|79771|22|9|31|PM|second|evening|| +79772|AAAAAAAANJHDBAAA|79772|22|9|32|PM|second|evening|| +79773|AAAAAAAAOJHDBAAA|79773|22|9|33|PM|second|evening|| +79774|AAAAAAAAPJHDBAAA|79774|22|9|34|PM|second|evening|| +79775|AAAAAAAAAKHDBAAA|79775|22|9|35|PM|second|evening|| +79776|AAAAAAAABKHDBAAA|79776|22|9|36|PM|second|evening|| +79777|AAAAAAAACKHDBAAA|79777|22|9|37|PM|second|evening|| +79778|AAAAAAAADKHDBAAA|79778|22|9|38|PM|second|evening|| +79779|AAAAAAAAEKHDBAAA|79779|22|9|39|PM|second|evening|| +79780|AAAAAAAAFKHDBAAA|79780|22|9|40|PM|second|evening|| +79781|AAAAAAAAGKHDBAAA|79781|22|9|41|PM|second|evening|| +79782|AAAAAAAAHKHDBAAA|79782|22|9|42|PM|second|evening|| +79783|AAAAAAAAIKHDBAAA|79783|22|9|43|PM|second|evening|| +79784|AAAAAAAAJKHDBAAA|79784|22|9|44|PM|second|evening|| +79785|AAAAAAAAKKHDBAAA|79785|22|9|45|PM|second|evening|| +79786|AAAAAAAALKHDBAAA|79786|22|9|46|PM|second|evening|| +79787|AAAAAAAAMKHDBAAA|79787|22|9|47|PM|second|evening|| +79788|AAAAAAAANKHDBAAA|79788|22|9|48|PM|second|evening|| +79789|AAAAAAAAOKHDBAAA|79789|22|9|49|PM|second|evening|| +79790|AAAAAAAAPKHDBAAA|79790|22|9|50|PM|second|evening|| +79791|AAAAAAAAALHDBAAA|79791|22|9|51|PM|second|evening|| +79792|AAAAAAAABLHDBAAA|79792|22|9|52|PM|second|evening|| +79793|AAAAAAAACLHDBAAA|79793|22|9|53|PM|second|evening|| +79794|AAAAAAAADLHDBAAA|79794|22|9|54|PM|second|evening|| +79795|AAAAAAAAELHDBAAA|79795|22|9|55|PM|second|evening|| +79796|AAAAAAAAFLHDBAAA|79796|22|9|56|PM|second|evening|| +79797|AAAAAAAAGLHDBAAA|79797|22|9|57|PM|second|evening|| +79798|AAAAAAAAHLHDBAAA|79798|22|9|58|PM|second|evening|| +79799|AAAAAAAAILHDBAAA|79799|22|9|59|PM|second|evening|| +79800|AAAAAAAAJLHDBAAA|79800|22|10|0|PM|second|evening|| +79801|AAAAAAAAKLHDBAAA|79801|22|10|1|PM|second|evening|| +79802|AAAAAAAALLHDBAAA|79802|22|10|2|PM|second|evening|| +79803|AAAAAAAAMLHDBAAA|79803|22|10|3|PM|second|evening|| +79804|AAAAAAAANLHDBAAA|79804|22|10|4|PM|second|evening|| +79805|AAAAAAAAOLHDBAAA|79805|22|10|5|PM|second|evening|| +79806|AAAAAAAAPLHDBAAA|79806|22|10|6|PM|second|evening|| +79807|AAAAAAAAAMHDBAAA|79807|22|10|7|PM|second|evening|| +79808|AAAAAAAABMHDBAAA|79808|22|10|8|PM|second|evening|| +79809|AAAAAAAACMHDBAAA|79809|22|10|9|PM|second|evening|| +79810|AAAAAAAADMHDBAAA|79810|22|10|10|PM|second|evening|| +79811|AAAAAAAAEMHDBAAA|79811|22|10|11|PM|second|evening|| +79812|AAAAAAAAFMHDBAAA|79812|22|10|12|PM|second|evening|| +79813|AAAAAAAAGMHDBAAA|79813|22|10|13|PM|second|evening|| +79814|AAAAAAAAHMHDBAAA|79814|22|10|14|PM|second|evening|| +79815|AAAAAAAAIMHDBAAA|79815|22|10|15|PM|second|evening|| +79816|AAAAAAAAJMHDBAAA|79816|22|10|16|PM|second|evening|| +79817|AAAAAAAAKMHDBAAA|79817|22|10|17|PM|second|evening|| +79818|AAAAAAAALMHDBAAA|79818|22|10|18|PM|second|evening|| +79819|AAAAAAAAMMHDBAAA|79819|22|10|19|PM|second|evening|| +79820|AAAAAAAANMHDBAAA|79820|22|10|20|PM|second|evening|| +79821|AAAAAAAAOMHDBAAA|79821|22|10|21|PM|second|evening|| +79822|AAAAAAAAPMHDBAAA|79822|22|10|22|PM|second|evening|| +79823|AAAAAAAAANHDBAAA|79823|22|10|23|PM|second|evening|| +79824|AAAAAAAABNHDBAAA|79824|22|10|24|PM|second|evening|| +79825|AAAAAAAACNHDBAAA|79825|22|10|25|PM|second|evening|| +79826|AAAAAAAADNHDBAAA|79826|22|10|26|PM|second|evening|| +79827|AAAAAAAAENHDBAAA|79827|22|10|27|PM|second|evening|| +79828|AAAAAAAAFNHDBAAA|79828|22|10|28|PM|second|evening|| +79829|AAAAAAAAGNHDBAAA|79829|22|10|29|PM|second|evening|| +79830|AAAAAAAAHNHDBAAA|79830|22|10|30|PM|second|evening|| +79831|AAAAAAAAINHDBAAA|79831|22|10|31|PM|second|evening|| +79832|AAAAAAAAJNHDBAAA|79832|22|10|32|PM|second|evening|| +79833|AAAAAAAAKNHDBAAA|79833|22|10|33|PM|second|evening|| +79834|AAAAAAAALNHDBAAA|79834|22|10|34|PM|second|evening|| +79835|AAAAAAAAMNHDBAAA|79835|22|10|35|PM|second|evening|| +79836|AAAAAAAANNHDBAAA|79836|22|10|36|PM|second|evening|| +79837|AAAAAAAAONHDBAAA|79837|22|10|37|PM|second|evening|| +79838|AAAAAAAAPNHDBAAA|79838|22|10|38|PM|second|evening|| +79839|AAAAAAAAAOHDBAAA|79839|22|10|39|PM|second|evening|| +79840|AAAAAAAABOHDBAAA|79840|22|10|40|PM|second|evening|| +79841|AAAAAAAACOHDBAAA|79841|22|10|41|PM|second|evening|| +79842|AAAAAAAADOHDBAAA|79842|22|10|42|PM|second|evening|| +79843|AAAAAAAAEOHDBAAA|79843|22|10|43|PM|second|evening|| +79844|AAAAAAAAFOHDBAAA|79844|22|10|44|PM|second|evening|| +79845|AAAAAAAAGOHDBAAA|79845|22|10|45|PM|second|evening|| +79846|AAAAAAAAHOHDBAAA|79846|22|10|46|PM|second|evening|| +79847|AAAAAAAAIOHDBAAA|79847|22|10|47|PM|second|evening|| +79848|AAAAAAAAJOHDBAAA|79848|22|10|48|PM|second|evening|| +79849|AAAAAAAAKOHDBAAA|79849|22|10|49|PM|second|evening|| +79850|AAAAAAAALOHDBAAA|79850|22|10|50|PM|second|evening|| +79851|AAAAAAAAMOHDBAAA|79851|22|10|51|PM|second|evening|| +79852|AAAAAAAANOHDBAAA|79852|22|10|52|PM|second|evening|| +79853|AAAAAAAAOOHDBAAA|79853|22|10|53|PM|second|evening|| +79854|AAAAAAAAPOHDBAAA|79854|22|10|54|PM|second|evening|| +79855|AAAAAAAAAPHDBAAA|79855|22|10|55|PM|second|evening|| +79856|AAAAAAAABPHDBAAA|79856|22|10|56|PM|second|evening|| +79857|AAAAAAAACPHDBAAA|79857|22|10|57|PM|second|evening|| +79858|AAAAAAAADPHDBAAA|79858|22|10|58|PM|second|evening|| +79859|AAAAAAAAEPHDBAAA|79859|22|10|59|PM|second|evening|| +79860|AAAAAAAAFPHDBAAA|79860|22|11|0|PM|second|evening|| +79861|AAAAAAAAGPHDBAAA|79861|22|11|1|PM|second|evening|| +79862|AAAAAAAAHPHDBAAA|79862|22|11|2|PM|second|evening|| +79863|AAAAAAAAIPHDBAAA|79863|22|11|3|PM|second|evening|| +79864|AAAAAAAAJPHDBAAA|79864|22|11|4|PM|second|evening|| +79865|AAAAAAAAKPHDBAAA|79865|22|11|5|PM|second|evening|| +79866|AAAAAAAALPHDBAAA|79866|22|11|6|PM|second|evening|| +79867|AAAAAAAAMPHDBAAA|79867|22|11|7|PM|second|evening|| +79868|AAAAAAAANPHDBAAA|79868|22|11|8|PM|second|evening|| +79869|AAAAAAAAOPHDBAAA|79869|22|11|9|PM|second|evening|| +79870|AAAAAAAAPPHDBAAA|79870|22|11|10|PM|second|evening|| +79871|AAAAAAAAAAIDBAAA|79871|22|11|11|PM|second|evening|| +79872|AAAAAAAABAIDBAAA|79872|22|11|12|PM|second|evening|| +79873|AAAAAAAACAIDBAAA|79873|22|11|13|PM|second|evening|| +79874|AAAAAAAADAIDBAAA|79874|22|11|14|PM|second|evening|| +79875|AAAAAAAAEAIDBAAA|79875|22|11|15|PM|second|evening|| +79876|AAAAAAAAFAIDBAAA|79876|22|11|16|PM|second|evening|| +79877|AAAAAAAAGAIDBAAA|79877|22|11|17|PM|second|evening|| +79878|AAAAAAAAHAIDBAAA|79878|22|11|18|PM|second|evening|| +79879|AAAAAAAAIAIDBAAA|79879|22|11|19|PM|second|evening|| +79880|AAAAAAAAJAIDBAAA|79880|22|11|20|PM|second|evening|| +79881|AAAAAAAAKAIDBAAA|79881|22|11|21|PM|second|evening|| +79882|AAAAAAAALAIDBAAA|79882|22|11|22|PM|second|evening|| +79883|AAAAAAAAMAIDBAAA|79883|22|11|23|PM|second|evening|| +79884|AAAAAAAANAIDBAAA|79884|22|11|24|PM|second|evening|| +79885|AAAAAAAAOAIDBAAA|79885|22|11|25|PM|second|evening|| +79886|AAAAAAAAPAIDBAAA|79886|22|11|26|PM|second|evening|| +79887|AAAAAAAAABIDBAAA|79887|22|11|27|PM|second|evening|| +79888|AAAAAAAABBIDBAAA|79888|22|11|28|PM|second|evening|| +79889|AAAAAAAACBIDBAAA|79889|22|11|29|PM|second|evening|| +79890|AAAAAAAADBIDBAAA|79890|22|11|30|PM|second|evening|| +79891|AAAAAAAAEBIDBAAA|79891|22|11|31|PM|second|evening|| +79892|AAAAAAAAFBIDBAAA|79892|22|11|32|PM|second|evening|| +79893|AAAAAAAAGBIDBAAA|79893|22|11|33|PM|second|evening|| +79894|AAAAAAAAHBIDBAAA|79894|22|11|34|PM|second|evening|| +79895|AAAAAAAAIBIDBAAA|79895|22|11|35|PM|second|evening|| +79896|AAAAAAAAJBIDBAAA|79896|22|11|36|PM|second|evening|| +79897|AAAAAAAAKBIDBAAA|79897|22|11|37|PM|second|evening|| +79898|AAAAAAAALBIDBAAA|79898|22|11|38|PM|second|evening|| +79899|AAAAAAAAMBIDBAAA|79899|22|11|39|PM|second|evening|| +79900|AAAAAAAANBIDBAAA|79900|22|11|40|PM|second|evening|| +79901|AAAAAAAAOBIDBAAA|79901|22|11|41|PM|second|evening|| +79902|AAAAAAAAPBIDBAAA|79902|22|11|42|PM|second|evening|| +79903|AAAAAAAAACIDBAAA|79903|22|11|43|PM|second|evening|| +79904|AAAAAAAABCIDBAAA|79904|22|11|44|PM|second|evening|| +79905|AAAAAAAACCIDBAAA|79905|22|11|45|PM|second|evening|| +79906|AAAAAAAADCIDBAAA|79906|22|11|46|PM|second|evening|| +79907|AAAAAAAAECIDBAAA|79907|22|11|47|PM|second|evening|| +79908|AAAAAAAAFCIDBAAA|79908|22|11|48|PM|second|evening|| +79909|AAAAAAAAGCIDBAAA|79909|22|11|49|PM|second|evening|| +79910|AAAAAAAAHCIDBAAA|79910|22|11|50|PM|second|evening|| +79911|AAAAAAAAICIDBAAA|79911|22|11|51|PM|second|evening|| +79912|AAAAAAAAJCIDBAAA|79912|22|11|52|PM|second|evening|| +79913|AAAAAAAAKCIDBAAA|79913|22|11|53|PM|second|evening|| +79914|AAAAAAAALCIDBAAA|79914|22|11|54|PM|second|evening|| +79915|AAAAAAAAMCIDBAAA|79915|22|11|55|PM|second|evening|| +79916|AAAAAAAANCIDBAAA|79916|22|11|56|PM|second|evening|| +79917|AAAAAAAAOCIDBAAA|79917|22|11|57|PM|second|evening|| +79918|AAAAAAAAPCIDBAAA|79918|22|11|58|PM|second|evening|| +79919|AAAAAAAAADIDBAAA|79919|22|11|59|PM|second|evening|| +79920|AAAAAAAABDIDBAAA|79920|22|12|0|PM|second|evening|| +79921|AAAAAAAACDIDBAAA|79921|22|12|1|PM|second|evening|| +79922|AAAAAAAADDIDBAAA|79922|22|12|2|PM|second|evening|| +79923|AAAAAAAAEDIDBAAA|79923|22|12|3|PM|second|evening|| +79924|AAAAAAAAFDIDBAAA|79924|22|12|4|PM|second|evening|| +79925|AAAAAAAAGDIDBAAA|79925|22|12|5|PM|second|evening|| +79926|AAAAAAAAHDIDBAAA|79926|22|12|6|PM|second|evening|| +79927|AAAAAAAAIDIDBAAA|79927|22|12|7|PM|second|evening|| +79928|AAAAAAAAJDIDBAAA|79928|22|12|8|PM|second|evening|| +79929|AAAAAAAAKDIDBAAA|79929|22|12|9|PM|second|evening|| +79930|AAAAAAAALDIDBAAA|79930|22|12|10|PM|second|evening|| +79931|AAAAAAAAMDIDBAAA|79931|22|12|11|PM|second|evening|| +79932|AAAAAAAANDIDBAAA|79932|22|12|12|PM|second|evening|| +79933|AAAAAAAAODIDBAAA|79933|22|12|13|PM|second|evening|| +79934|AAAAAAAAPDIDBAAA|79934|22|12|14|PM|second|evening|| +79935|AAAAAAAAAEIDBAAA|79935|22|12|15|PM|second|evening|| +79936|AAAAAAAABEIDBAAA|79936|22|12|16|PM|second|evening|| +79937|AAAAAAAACEIDBAAA|79937|22|12|17|PM|second|evening|| +79938|AAAAAAAADEIDBAAA|79938|22|12|18|PM|second|evening|| +79939|AAAAAAAAEEIDBAAA|79939|22|12|19|PM|second|evening|| +79940|AAAAAAAAFEIDBAAA|79940|22|12|20|PM|second|evening|| +79941|AAAAAAAAGEIDBAAA|79941|22|12|21|PM|second|evening|| +79942|AAAAAAAAHEIDBAAA|79942|22|12|22|PM|second|evening|| +79943|AAAAAAAAIEIDBAAA|79943|22|12|23|PM|second|evening|| +79944|AAAAAAAAJEIDBAAA|79944|22|12|24|PM|second|evening|| +79945|AAAAAAAAKEIDBAAA|79945|22|12|25|PM|second|evening|| +79946|AAAAAAAALEIDBAAA|79946|22|12|26|PM|second|evening|| +79947|AAAAAAAAMEIDBAAA|79947|22|12|27|PM|second|evening|| +79948|AAAAAAAANEIDBAAA|79948|22|12|28|PM|second|evening|| +79949|AAAAAAAAOEIDBAAA|79949|22|12|29|PM|second|evening|| +79950|AAAAAAAAPEIDBAAA|79950|22|12|30|PM|second|evening|| +79951|AAAAAAAAAFIDBAAA|79951|22|12|31|PM|second|evening|| +79952|AAAAAAAABFIDBAAA|79952|22|12|32|PM|second|evening|| +79953|AAAAAAAACFIDBAAA|79953|22|12|33|PM|second|evening|| +79954|AAAAAAAADFIDBAAA|79954|22|12|34|PM|second|evening|| +79955|AAAAAAAAEFIDBAAA|79955|22|12|35|PM|second|evening|| +79956|AAAAAAAAFFIDBAAA|79956|22|12|36|PM|second|evening|| +79957|AAAAAAAAGFIDBAAA|79957|22|12|37|PM|second|evening|| +79958|AAAAAAAAHFIDBAAA|79958|22|12|38|PM|second|evening|| +79959|AAAAAAAAIFIDBAAA|79959|22|12|39|PM|second|evening|| +79960|AAAAAAAAJFIDBAAA|79960|22|12|40|PM|second|evening|| +79961|AAAAAAAAKFIDBAAA|79961|22|12|41|PM|second|evening|| +79962|AAAAAAAALFIDBAAA|79962|22|12|42|PM|second|evening|| +79963|AAAAAAAAMFIDBAAA|79963|22|12|43|PM|second|evening|| +79964|AAAAAAAANFIDBAAA|79964|22|12|44|PM|second|evening|| +79965|AAAAAAAAOFIDBAAA|79965|22|12|45|PM|second|evening|| +79966|AAAAAAAAPFIDBAAA|79966|22|12|46|PM|second|evening|| +79967|AAAAAAAAAGIDBAAA|79967|22|12|47|PM|second|evening|| +79968|AAAAAAAABGIDBAAA|79968|22|12|48|PM|second|evening|| +79969|AAAAAAAACGIDBAAA|79969|22|12|49|PM|second|evening|| +79970|AAAAAAAADGIDBAAA|79970|22|12|50|PM|second|evening|| +79971|AAAAAAAAEGIDBAAA|79971|22|12|51|PM|second|evening|| +79972|AAAAAAAAFGIDBAAA|79972|22|12|52|PM|second|evening|| +79973|AAAAAAAAGGIDBAAA|79973|22|12|53|PM|second|evening|| +79974|AAAAAAAAHGIDBAAA|79974|22|12|54|PM|second|evening|| +79975|AAAAAAAAIGIDBAAA|79975|22|12|55|PM|second|evening|| +79976|AAAAAAAAJGIDBAAA|79976|22|12|56|PM|second|evening|| +79977|AAAAAAAAKGIDBAAA|79977|22|12|57|PM|second|evening|| +79978|AAAAAAAALGIDBAAA|79978|22|12|58|PM|second|evening|| +79979|AAAAAAAAMGIDBAAA|79979|22|12|59|PM|second|evening|| +79980|AAAAAAAANGIDBAAA|79980|22|13|0|PM|second|evening|| +79981|AAAAAAAAOGIDBAAA|79981|22|13|1|PM|second|evening|| +79982|AAAAAAAAPGIDBAAA|79982|22|13|2|PM|second|evening|| +79983|AAAAAAAAAHIDBAAA|79983|22|13|3|PM|second|evening|| +79984|AAAAAAAABHIDBAAA|79984|22|13|4|PM|second|evening|| +79985|AAAAAAAACHIDBAAA|79985|22|13|5|PM|second|evening|| +79986|AAAAAAAADHIDBAAA|79986|22|13|6|PM|second|evening|| +79987|AAAAAAAAEHIDBAAA|79987|22|13|7|PM|second|evening|| +79988|AAAAAAAAFHIDBAAA|79988|22|13|8|PM|second|evening|| +79989|AAAAAAAAGHIDBAAA|79989|22|13|9|PM|second|evening|| +79990|AAAAAAAAHHIDBAAA|79990|22|13|10|PM|second|evening|| +79991|AAAAAAAAIHIDBAAA|79991|22|13|11|PM|second|evening|| +79992|AAAAAAAAJHIDBAAA|79992|22|13|12|PM|second|evening|| +79993|AAAAAAAAKHIDBAAA|79993|22|13|13|PM|second|evening|| +79994|AAAAAAAALHIDBAAA|79994|22|13|14|PM|second|evening|| +79995|AAAAAAAAMHIDBAAA|79995|22|13|15|PM|second|evening|| +79996|AAAAAAAANHIDBAAA|79996|22|13|16|PM|second|evening|| +79997|AAAAAAAAOHIDBAAA|79997|22|13|17|PM|second|evening|| +79998|AAAAAAAAPHIDBAAA|79998|22|13|18|PM|second|evening|| +79999|AAAAAAAAAIIDBAAA|79999|22|13|19|PM|second|evening|| +80000|AAAAAAAABIIDBAAA|80000|22|13|20|PM|second|evening|| +80001|AAAAAAAACIIDBAAA|80001|22|13|21|PM|second|evening|| +80002|AAAAAAAADIIDBAAA|80002|22|13|22|PM|second|evening|| +80003|AAAAAAAAEIIDBAAA|80003|22|13|23|PM|second|evening|| +80004|AAAAAAAAFIIDBAAA|80004|22|13|24|PM|second|evening|| +80005|AAAAAAAAGIIDBAAA|80005|22|13|25|PM|second|evening|| +80006|AAAAAAAAHIIDBAAA|80006|22|13|26|PM|second|evening|| +80007|AAAAAAAAIIIDBAAA|80007|22|13|27|PM|second|evening|| +80008|AAAAAAAAJIIDBAAA|80008|22|13|28|PM|second|evening|| +80009|AAAAAAAAKIIDBAAA|80009|22|13|29|PM|second|evening|| +80010|AAAAAAAALIIDBAAA|80010|22|13|30|PM|second|evening|| +80011|AAAAAAAAMIIDBAAA|80011|22|13|31|PM|second|evening|| +80012|AAAAAAAANIIDBAAA|80012|22|13|32|PM|second|evening|| +80013|AAAAAAAAOIIDBAAA|80013|22|13|33|PM|second|evening|| +80014|AAAAAAAAPIIDBAAA|80014|22|13|34|PM|second|evening|| +80015|AAAAAAAAAJIDBAAA|80015|22|13|35|PM|second|evening|| +80016|AAAAAAAABJIDBAAA|80016|22|13|36|PM|second|evening|| +80017|AAAAAAAACJIDBAAA|80017|22|13|37|PM|second|evening|| +80018|AAAAAAAADJIDBAAA|80018|22|13|38|PM|second|evening|| +80019|AAAAAAAAEJIDBAAA|80019|22|13|39|PM|second|evening|| +80020|AAAAAAAAFJIDBAAA|80020|22|13|40|PM|second|evening|| +80021|AAAAAAAAGJIDBAAA|80021|22|13|41|PM|second|evening|| +80022|AAAAAAAAHJIDBAAA|80022|22|13|42|PM|second|evening|| +80023|AAAAAAAAIJIDBAAA|80023|22|13|43|PM|second|evening|| +80024|AAAAAAAAJJIDBAAA|80024|22|13|44|PM|second|evening|| +80025|AAAAAAAAKJIDBAAA|80025|22|13|45|PM|second|evening|| +80026|AAAAAAAALJIDBAAA|80026|22|13|46|PM|second|evening|| +80027|AAAAAAAAMJIDBAAA|80027|22|13|47|PM|second|evening|| +80028|AAAAAAAANJIDBAAA|80028|22|13|48|PM|second|evening|| +80029|AAAAAAAAOJIDBAAA|80029|22|13|49|PM|second|evening|| +80030|AAAAAAAAPJIDBAAA|80030|22|13|50|PM|second|evening|| +80031|AAAAAAAAAKIDBAAA|80031|22|13|51|PM|second|evening|| +80032|AAAAAAAABKIDBAAA|80032|22|13|52|PM|second|evening|| +80033|AAAAAAAACKIDBAAA|80033|22|13|53|PM|second|evening|| +80034|AAAAAAAADKIDBAAA|80034|22|13|54|PM|second|evening|| +80035|AAAAAAAAEKIDBAAA|80035|22|13|55|PM|second|evening|| +80036|AAAAAAAAFKIDBAAA|80036|22|13|56|PM|second|evening|| +80037|AAAAAAAAGKIDBAAA|80037|22|13|57|PM|second|evening|| +80038|AAAAAAAAHKIDBAAA|80038|22|13|58|PM|second|evening|| +80039|AAAAAAAAIKIDBAAA|80039|22|13|59|PM|second|evening|| +80040|AAAAAAAAJKIDBAAA|80040|22|14|0|PM|second|evening|| +80041|AAAAAAAAKKIDBAAA|80041|22|14|1|PM|second|evening|| +80042|AAAAAAAALKIDBAAA|80042|22|14|2|PM|second|evening|| +80043|AAAAAAAAMKIDBAAA|80043|22|14|3|PM|second|evening|| +80044|AAAAAAAANKIDBAAA|80044|22|14|4|PM|second|evening|| +80045|AAAAAAAAOKIDBAAA|80045|22|14|5|PM|second|evening|| +80046|AAAAAAAAPKIDBAAA|80046|22|14|6|PM|second|evening|| +80047|AAAAAAAAALIDBAAA|80047|22|14|7|PM|second|evening|| +80048|AAAAAAAABLIDBAAA|80048|22|14|8|PM|second|evening|| +80049|AAAAAAAACLIDBAAA|80049|22|14|9|PM|second|evening|| +80050|AAAAAAAADLIDBAAA|80050|22|14|10|PM|second|evening|| +80051|AAAAAAAAELIDBAAA|80051|22|14|11|PM|second|evening|| +80052|AAAAAAAAFLIDBAAA|80052|22|14|12|PM|second|evening|| +80053|AAAAAAAAGLIDBAAA|80053|22|14|13|PM|second|evening|| +80054|AAAAAAAAHLIDBAAA|80054|22|14|14|PM|second|evening|| +80055|AAAAAAAAILIDBAAA|80055|22|14|15|PM|second|evening|| +80056|AAAAAAAAJLIDBAAA|80056|22|14|16|PM|second|evening|| +80057|AAAAAAAAKLIDBAAA|80057|22|14|17|PM|second|evening|| +80058|AAAAAAAALLIDBAAA|80058|22|14|18|PM|second|evening|| +80059|AAAAAAAAMLIDBAAA|80059|22|14|19|PM|second|evening|| +80060|AAAAAAAANLIDBAAA|80060|22|14|20|PM|second|evening|| +80061|AAAAAAAAOLIDBAAA|80061|22|14|21|PM|second|evening|| +80062|AAAAAAAAPLIDBAAA|80062|22|14|22|PM|second|evening|| +80063|AAAAAAAAAMIDBAAA|80063|22|14|23|PM|second|evening|| +80064|AAAAAAAABMIDBAAA|80064|22|14|24|PM|second|evening|| +80065|AAAAAAAACMIDBAAA|80065|22|14|25|PM|second|evening|| +80066|AAAAAAAADMIDBAAA|80066|22|14|26|PM|second|evening|| +80067|AAAAAAAAEMIDBAAA|80067|22|14|27|PM|second|evening|| +80068|AAAAAAAAFMIDBAAA|80068|22|14|28|PM|second|evening|| +80069|AAAAAAAAGMIDBAAA|80069|22|14|29|PM|second|evening|| +80070|AAAAAAAAHMIDBAAA|80070|22|14|30|PM|second|evening|| +80071|AAAAAAAAIMIDBAAA|80071|22|14|31|PM|second|evening|| +80072|AAAAAAAAJMIDBAAA|80072|22|14|32|PM|second|evening|| +80073|AAAAAAAAKMIDBAAA|80073|22|14|33|PM|second|evening|| +80074|AAAAAAAALMIDBAAA|80074|22|14|34|PM|second|evening|| +80075|AAAAAAAAMMIDBAAA|80075|22|14|35|PM|second|evening|| +80076|AAAAAAAANMIDBAAA|80076|22|14|36|PM|second|evening|| +80077|AAAAAAAAOMIDBAAA|80077|22|14|37|PM|second|evening|| +80078|AAAAAAAAPMIDBAAA|80078|22|14|38|PM|second|evening|| +80079|AAAAAAAAANIDBAAA|80079|22|14|39|PM|second|evening|| +80080|AAAAAAAABNIDBAAA|80080|22|14|40|PM|second|evening|| +80081|AAAAAAAACNIDBAAA|80081|22|14|41|PM|second|evening|| +80082|AAAAAAAADNIDBAAA|80082|22|14|42|PM|second|evening|| +80083|AAAAAAAAENIDBAAA|80083|22|14|43|PM|second|evening|| +80084|AAAAAAAAFNIDBAAA|80084|22|14|44|PM|second|evening|| +80085|AAAAAAAAGNIDBAAA|80085|22|14|45|PM|second|evening|| +80086|AAAAAAAAHNIDBAAA|80086|22|14|46|PM|second|evening|| +80087|AAAAAAAAINIDBAAA|80087|22|14|47|PM|second|evening|| +80088|AAAAAAAAJNIDBAAA|80088|22|14|48|PM|second|evening|| +80089|AAAAAAAAKNIDBAAA|80089|22|14|49|PM|second|evening|| +80090|AAAAAAAALNIDBAAA|80090|22|14|50|PM|second|evening|| +80091|AAAAAAAAMNIDBAAA|80091|22|14|51|PM|second|evening|| +80092|AAAAAAAANNIDBAAA|80092|22|14|52|PM|second|evening|| +80093|AAAAAAAAONIDBAAA|80093|22|14|53|PM|second|evening|| +80094|AAAAAAAAPNIDBAAA|80094|22|14|54|PM|second|evening|| +80095|AAAAAAAAAOIDBAAA|80095|22|14|55|PM|second|evening|| +80096|AAAAAAAABOIDBAAA|80096|22|14|56|PM|second|evening|| +80097|AAAAAAAACOIDBAAA|80097|22|14|57|PM|second|evening|| +80098|AAAAAAAADOIDBAAA|80098|22|14|58|PM|second|evening|| +80099|AAAAAAAAEOIDBAAA|80099|22|14|59|PM|second|evening|| +80100|AAAAAAAAFOIDBAAA|80100|22|15|0|PM|second|evening|| +80101|AAAAAAAAGOIDBAAA|80101|22|15|1|PM|second|evening|| +80102|AAAAAAAAHOIDBAAA|80102|22|15|2|PM|second|evening|| +80103|AAAAAAAAIOIDBAAA|80103|22|15|3|PM|second|evening|| +80104|AAAAAAAAJOIDBAAA|80104|22|15|4|PM|second|evening|| +80105|AAAAAAAAKOIDBAAA|80105|22|15|5|PM|second|evening|| +80106|AAAAAAAALOIDBAAA|80106|22|15|6|PM|second|evening|| +80107|AAAAAAAAMOIDBAAA|80107|22|15|7|PM|second|evening|| +80108|AAAAAAAANOIDBAAA|80108|22|15|8|PM|second|evening|| +80109|AAAAAAAAOOIDBAAA|80109|22|15|9|PM|second|evening|| +80110|AAAAAAAAPOIDBAAA|80110|22|15|10|PM|second|evening|| +80111|AAAAAAAAAPIDBAAA|80111|22|15|11|PM|second|evening|| +80112|AAAAAAAABPIDBAAA|80112|22|15|12|PM|second|evening|| +80113|AAAAAAAACPIDBAAA|80113|22|15|13|PM|second|evening|| +80114|AAAAAAAADPIDBAAA|80114|22|15|14|PM|second|evening|| +80115|AAAAAAAAEPIDBAAA|80115|22|15|15|PM|second|evening|| +80116|AAAAAAAAFPIDBAAA|80116|22|15|16|PM|second|evening|| +80117|AAAAAAAAGPIDBAAA|80117|22|15|17|PM|second|evening|| +80118|AAAAAAAAHPIDBAAA|80118|22|15|18|PM|second|evening|| +80119|AAAAAAAAIPIDBAAA|80119|22|15|19|PM|second|evening|| +80120|AAAAAAAAJPIDBAAA|80120|22|15|20|PM|second|evening|| +80121|AAAAAAAAKPIDBAAA|80121|22|15|21|PM|second|evening|| +80122|AAAAAAAALPIDBAAA|80122|22|15|22|PM|second|evening|| +80123|AAAAAAAAMPIDBAAA|80123|22|15|23|PM|second|evening|| +80124|AAAAAAAANPIDBAAA|80124|22|15|24|PM|second|evening|| +80125|AAAAAAAAOPIDBAAA|80125|22|15|25|PM|second|evening|| +80126|AAAAAAAAPPIDBAAA|80126|22|15|26|PM|second|evening|| +80127|AAAAAAAAAAJDBAAA|80127|22|15|27|PM|second|evening|| +80128|AAAAAAAABAJDBAAA|80128|22|15|28|PM|second|evening|| +80129|AAAAAAAACAJDBAAA|80129|22|15|29|PM|second|evening|| +80130|AAAAAAAADAJDBAAA|80130|22|15|30|PM|second|evening|| +80131|AAAAAAAAEAJDBAAA|80131|22|15|31|PM|second|evening|| +80132|AAAAAAAAFAJDBAAA|80132|22|15|32|PM|second|evening|| +80133|AAAAAAAAGAJDBAAA|80133|22|15|33|PM|second|evening|| +80134|AAAAAAAAHAJDBAAA|80134|22|15|34|PM|second|evening|| +80135|AAAAAAAAIAJDBAAA|80135|22|15|35|PM|second|evening|| +80136|AAAAAAAAJAJDBAAA|80136|22|15|36|PM|second|evening|| +80137|AAAAAAAAKAJDBAAA|80137|22|15|37|PM|second|evening|| +80138|AAAAAAAALAJDBAAA|80138|22|15|38|PM|second|evening|| +80139|AAAAAAAAMAJDBAAA|80139|22|15|39|PM|second|evening|| +80140|AAAAAAAANAJDBAAA|80140|22|15|40|PM|second|evening|| +80141|AAAAAAAAOAJDBAAA|80141|22|15|41|PM|second|evening|| +80142|AAAAAAAAPAJDBAAA|80142|22|15|42|PM|second|evening|| +80143|AAAAAAAAABJDBAAA|80143|22|15|43|PM|second|evening|| +80144|AAAAAAAABBJDBAAA|80144|22|15|44|PM|second|evening|| +80145|AAAAAAAACBJDBAAA|80145|22|15|45|PM|second|evening|| +80146|AAAAAAAADBJDBAAA|80146|22|15|46|PM|second|evening|| +80147|AAAAAAAAEBJDBAAA|80147|22|15|47|PM|second|evening|| +80148|AAAAAAAAFBJDBAAA|80148|22|15|48|PM|second|evening|| +80149|AAAAAAAAGBJDBAAA|80149|22|15|49|PM|second|evening|| +80150|AAAAAAAAHBJDBAAA|80150|22|15|50|PM|second|evening|| +80151|AAAAAAAAIBJDBAAA|80151|22|15|51|PM|second|evening|| +80152|AAAAAAAAJBJDBAAA|80152|22|15|52|PM|second|evening|| +80153|AAAAAAAAKBJDBAAA|80153|22|15|53|PM|second|evening|| +80154|AAAAAAAALBJDBAAA|80154|22|15|54|PM|second|evening|| +80155|AAAAAAAAMBJDBAAA|80155|22|15|55|PM|second|evening|| +80156|AAAAAAAANBJDBAAA|80156|22|15|56|PM|second|evening|| +80157|AAAAAAAAOBJDBAAA|80157|22|15|57|PM|second|evening|| +80158|AAAAAAAAPBJDBAAA|80158|22|15|58|PM|second|evening|| +80159|AAAAAAAAACJDBAAA|80159|22|15|59|PM|second|evening|| +80160|AAAAAAAABCJDBAAA|80160|22|16|0|PM|second|evening|| +80161|AAAAAAAACCJDBAAA|80161|22|16|1|PM|second|evening|| +80162|AAAAAAAADCJDBAAA|80162|22|16|2|PM|second|evening|| +80163|AAAAAAAAECJDBAAA|80163|22|16|3|PM|second|evening|| +80164|AAAAAAAAFCJDBAAA|80164|22|16|4|PM|second|evening|| +80165|AAAAAAAAGCJDBAAA|80165|22|16|5|PM|second|evening|| +80166|AAAAAAAAHCJDBAAA|80166|22|16|6|PM|second|evening|| +80167|AAAAAAAAICJDBAAA|80167|22|16|7|PM|second|evening|| +80168|AAAAAAAAJCJDBAAA|80168|22|16|8|PM|second|evening|| +80169|AAAAAAAAKCJDBAAA|80169|22|16|9|PM|second|evening|| +80170|AAAAAAAALCJDBAAA|80170|22|16|10|PM|second|evening|| +80171|AAAAAAAAMCJDBAAA|80171|22|16|11|PM|second|evening|| +80172|AAAAAAAANCJDBAAA|80172|22|16|12|PM|second|evening|| +80173|AAAAAAAAOCJDBAAA|80173|22|16|13|PM|second|evening|| +80174|AAAAAAAAPCJDBAAA|80174|22|16|14|PM|second|evening|| +80175|AAAAAAAAADJDBAAA|80175|22|16|15|PM|second|evening|| +80176|AAAAAAAABDJDBAAA|80176|22|16|16|PM|second|evening|| +80177|AAAAAAAACDJDBAAA|80177|22|16|17|PM|second|evening|| +80178|AAAAAAAADDJDBAAA|80178|22|16|18|PM|second|evening|| +80179|AAAAAAAAEDJDBAAA|80179|22|16|19|PM|second|evening|| +80180|AAAAAAAAFDJDBAAA|80180|22|16|20|PM|second|evening|| +80181|AAAAAAAAGDJDBAAA|80181|22|16|21|PM|second|evening|| +80182|AAAAAAAAHDJDBAAA|80182|22|16|22|PM|second|evening|| +80183|AAAAAAAAIDJDBAAA|80183|22|16|23|PM|second|evening|| +80184|AAAAAAAAJDJDBAAA|80184|22|16|24|PM|second|evening|| +80185|AAAAAAAAKDJDBAAA|80185|22|16|25|PM|second|evening|| +80186|AAAAAAAALDJDBAAA|80186|22|16|26|PM|second|evening|| +80187|AAAAAAAAMDJDBAAA|80187|22|16|27|PM|second|evening|| +80188|AAAAAAAANDJDBAAA|80188|22|16|28|PM|second|evening|| +80189|AAAAAAAAODJDBAAA|80189|22|16|29|PM|second|evening|| +80190|AAAAAAAAPDJDBAAA|80190|22|16|30|PM|second|evening|| +80191|AAAAAAAAAEJDBAAA|80191|22|16|31|PM|second|evening|| +80192|AAAAAAAABEJDBAAA|80192|22|16|32|PM|second|evening|| +80193|AAAAAAAACEJDBAAA|80193|22|16|33|PM|second|evening|| +80194|AAAAAAAADEJDBAAA|80194|22|16|34|PM|second|evening|| +80195|AAAAAAAAEEJDBAAA|80195|22|16|35|PM|second|evening|| +80196|AAAAAAAAFEJDBAAA|80196|22|16|36|PM|second|evening|| +80197|AAAAAAAAGEJDBAAA|80197|22|16|37|PM|second|evening|| +80198|AAAAAAAAHEJDBAAA|80198|22|16|38|PM|second|evening|| +80199|AAAAAAAAIEJDBAAA|80199|22|16|39|PM|second|evening|| +80200|AAAAAAAAJEJDBAAA|80200|22|16|40|PM|second|evening|| +80201|AAAAAAAAKEJDBAAA|80201|22|16|41|PM|second|evening|| +80202|AAAAAAAALEJDBAAA|80202|22|16|42|PM|second|evening|| +80203|AAAAAAAAMEJDBAAA|80203|22|16|43|PM|second|evening|| +80204|AAAAAAAANEJDBAAA|80204|22|16|44|PM|second|evening|| +80205|AAAAAAAAOEJDBAAA|80205|22|16|45|PM|second|evening|| +80206|AAAAAAAAPEJDBAAA|80206|22|16|46|PM|second|evening|| +80207|AAAAAAAAAFJDBAAA|80207|22|16|47|PM|second|evening|| +80208|AAAAAAAABFJDBAAA|80208|22|16|48|PM|second|evening|| +80209|AAAAAAAACFJDBAAA|80209|22|16|49|PM|second|evening|| +80210|AAAAAAAADFJDBAAA|80210|22|16|50|PM|second|evening|| +80211|AAAAAAAAEFJDBAAA|80211|22|16|51|PM|second|evening|| +80212|AAAAAAAAFFJDBAAA|80212|22|16|52|PM|second|evening|| +80213|AAAAAAAAGFJDBAAA|80213|22|16|53|PM|second|evening|| +80214|AAAAAAAAHFJDBAAA|80214|22|16|54|PM|second|evening|| +80215|AAAAAAAAIFJDBAAA|80215|22|16|55|PM|second|evening|| +80216|AAAAAAAAJFJDBAAA|80216|22|16|56|PM|second|evening|| +80217|AAAAAAAAKFJDBAAA|80217|22|16|57|PM|second|evening|| +80218|AAAAAAAALFJDBAAA|80218|22|16|58|PM|second|evening|| +80219|AAAAAAAAMFJDBAAA|80219|22|16|59|PM|second|evening|| +80220|AAAAAAAANFJDBAAA|80220|22|17|0|PM|second|evening|| +80221|AAAAAAAAOFJDBAAA|80221|22|17|1|PM|second|evening|| +80222|AAAAAAAAPFJDBAAA|80222|22|17|2|PM|second|evening|| +80223|AAAAAAAAAGJDBAAA|80223|22|17|3|PM|second|evening|| +80224|AAAAAAAABGJDBAAA|80224|22|17|4|PM|second|evening|| +80225|AAAAAAAACGJDBAAA|80225|22|17|5|PM|second|evening|| +80226|AAAAAAAADGJDBAAA|80226|22|17|6|PM|second|evening|| +80227|AAAAAAAAEGJDBAAA|80227|22|17|7|PM|second|evening|| +80228|AAAAAAAAFGJDBAAA|80228|22|17|8|PM|second|evening|| +80229|AAAAAAAAGGJDBAAA|80229|22|17|9|PM|second|evening|| +80230|AAAAAAAAHGJDBAAA|80230|22|17|10|PM|second|evening|| +80231|AAAAAAAAIGJDBAAA|80231|22|17|11|PM|second|evening|| +80232|AAAAAAAAJGJDBAAA|80232|22|17|12|PM|second|evening|| +80233|AAAAAAAAKGJDBAAA|80233|22|17|13|PM|second|evening|| +80234|AAAAAAAALGJDBAAA|80234|22|17|14|PM|second|evening|| +80235|AAAAAAAAMGJDBAAA|80235|22|17|15|PM|second|evening|| +80236|AAAAAAAANGJDBAAA|80236|22|17|16|PM|second|evening|| +80237|AAAAAAAAOGJDBAAA|80237|22|17|17|PM|second|evening|| +80238|AAAAAAAAPGJDBAAA|80238|22|17|18|PM|second|evening|| +80239|AAAAAAAAAHJDBAAA|80239|22|17|19|PM|second|evening|| +80240|AAAAAAAABHJDBAAA|80240|22|17|20|PM|second|evening|| +80241|AAAAAAAACHJDBAAA|80241|22|17|21|PM|second|evening|| +80242|AAAAAAAADHJDBAAA|80242|22|17|22|PM|second|evening|| +80243|AAAAAAAAEHJDBAAA|80243|22|17|23|PM|second|evening|| +80244|AAAAAAAAFHJDBAAA|80244|22|17|24|PM|second|evening|| +80245|AAAAAAAAGHJDBAAA|80245|22|17|25|PM|second|evening|| +80246|AAAAAAAAHHJDBAAA|80246|22|17|26|PM|second|evening|| +80247|AAAAAAAAIHJDBAAA|80247|22|17|27|PM|second|evening|| +80248|AAAAAAAAJHJDBAAA|80248|22|17|28|PM|second|evening|| +80249|AAAAAAAAKHJDBAAA|80249|22|17|29|PM|second|evening|| +80250|AAAAAAAALHJDBAAA|80250|22|17|30|PM|second|evening|| +80251|AAAAAAAAMHJDBAAA|80251|22|17|31|PM|second|evening|| +80252|AAAAAAAANHJDBAAA|80252|22|17|32|PM|second|evening|| +80253|AAAAAAAAOHJDBAAA|80253|22|17|33|PM|second|evening|| +80254|AAAAAAAAPHJDBAAA|80254|22|17|34|PM|second|evening|| +80255|AAAAAAAAAIJDBAAA|80255|22|17|35|PM|second|evening|| +80256|AAAAAAAABIJDBAAA|80256|22|17|36|PM|second|evening|| +80257|AAAAAAAACIJDBAAA|80257|22|17|37|PM|second|evening|| +80258|AAAAAAAADIJDBAAA|80258|22|17|38|PM|second|evening|| +80259|AAAAAAAAEIJDBAAA|80259|22|17|39|PM|second|evening|| +80260|AAAAAAAAFIJDBAAA|80260|22|17|40|PM|second|evening|| +80261|AAAAAAAAGIJDBAAA|80261|22|17|41|PM|second|evening|| +80262|AAAAAAAAHIJDBAAA|80262|22|17|42|PM|second|evening|| +80263|AAAAAAAAIIJDBAAA|80263|22|17|43|PM|second|evening|| +80264|AAAAAAAAJIJDBAAA|80264|22|17|44|PM|second|evening|| +80265|AAAAAAAAKIJDBAAA|80265|22|17|45|PM|second|evening|| +80266|AAAAAAAALIJDBAAA|80266|22|17|46|PM|second|evening|| +80267|AAAAAAAAMIJDBAAA|80267|22|17|47|PM|second|evening|| +80268|AAAAAAAANIJDBAAA|80268|22|17|48|PM|second|evening|| +80269|AAAAAAAAOIJDBAAA|80269|22|17|49|PM|second|evening|| +80270|AAAAAAAAPIJDBAAA|80270|22|17|50|PM|second|evening|| +80271|AAAAAAAAAJJDBAAA|80271|22|17|51|PM|second|evening|| +80272|AAAAAAAABJJDBAAA|80272|22|17|52|PM|second|evening|| +80273|AAAAAAAACJJDBAAA|80273|22|17|53|PM|second|evening|| +80274|AAAAAAAADJJDBAAA|80274|22|17|54|PM|second|evening|| +80275|AAAAAAAAEJJDBAAA|80275|22|17|55|PM|second|evening|| +80276|AAAAAAAAFJJDBAAA|80276|22|17|56|PM|second|evening|| +80277|AAAAAAAAGJJDBAAA|80277|22|17|57|PM|second|evening|| +80278|AAAAAAAAHJJDBAAA|80278|22|17|58|PM|second|evening|| +80279|AAAAAAAAIJJDBAAA|80279|22|17|59|PM|second|evening|| +80280|AAAAAAAAJJJDBAAA|80280|22|18|0|PM|second|evening|| +80281|AAAAAAAAKJJDBAAA|80281|22|18|1|PM|second|evening|| +80282|AAAAAAAALJJDBAAA|80282|22|18|2|PM|second|evening|| +80283|AAAAAAAAMJJDBAAA|80283|22|18|3|PM|second|evening|| +80284|AAAAAAAANJJDBAAA|80284|22|18|4|PM|second|evening|| +80285|AAAAAAAAOJJDBAAA|80285|22|18|5|PM|second|evening|| +80286|AAAAAAAAPJJDBAAA|80286|22|18|6|PM|second|evening|| +80287|AAAAAAAAAKJDBAAA|80287|22|18|7|PM|second|evening|| +80288|AAAAAAAABKJDBAAA|80288|22|18|8|PM|second|evening|| +80289|AAAAAAAACKJDBAAA|80289|22|18|9|PM|second|evening|| +80290|AAAAAAAADKJDBAAA|80290|22|18|10|PM|second|evening|| +80291|AAAAAAAAEKJDBAAA|80291|22|18|11|PM|second|evening|| +80292|AAAAAAAAFKJDBAAA|80292|22|18|12|PM|second|evening|| +80293|AAAAAAAAGKJDBAAA|80293|22|18|13|PM|second|evening|| +80294|AAAAAAAAHKJDBAAA|80294|22|18|14|PM|second|evening|| +80295|AAAAAAAAIKJDBAAA|80295|22|18|15|PM|second|evening|| +80296|AAAAAAAAJKJDBAAA|80296|22|18|16|PM|second|evening|| +80297|AAAAAAAAKKJDBAAA|80297|22|18|17|PM|second|evening|| +80298|AAAAAAAALKJDBAAA|80298|22|18|18|PM|second|evening|| +80299|AAAAAAAAMKJDBAAA|80299|22|18|19|PM|second|evening|| +80300|AAAAAAAANKJDBAAA|80300|22|18|20|PM|second|evening|| +80301|AAAAAAAAOKJDBAAA|80301|22|18|21|PM|second|evening|| +80302|AAAAAAAAPKJDBAAA|80302|22|18|22|PM|second|evening|| +80303|AAAAAAAAALJDBAAA|80303|22|18|23|PM|second|evening|| +80304|AAAAAAAABLJDBAAA|80304|22|18|24|PM|second|evening|| +80305|AAAAAAAACLJDBAAA|80305|22|18|25|PM|second|evening|| +80306|AAAAAAAADLJDBAAA|80306|22|18|26|PM|second|evening|| +80307|AAAAAAAAELJDBAAA|80307|22|18|27|PM|second|evening|| +80308|AAAAAAAAFLJDBAAA|80308|22|18|28|PM|second|evening|| +80309|AAAAAAAAGLJDBAAA|80309|22|18|29|PM|second|evening|| +80310|AAAAAAAAHLJDBAAA|80310|22|18|30|PM|second|evening|| +80311|AAAAAAAAILJDBAAA|80311|22|18|31|PM|second|evening|| +80312|AAAAAAAAJLJDBAAA|80312|22|18|32|PM|second|evening|| +80313|AAAAAAAAKLJDBAAA|80313|22|18|33|PM|second|evening|| +80314|AAAAAAAALLJDBAAA|80314|22|18|34|PM|second|evening|| +80315|AAAAAAAAMLJDBAAA|80315|22|18|35|PM|second|evening|| +80316|AAAAAAAANLJDBAAA|80316|22|18|36|PM|second|evening|| +80317|AAAAAAAAOLJDBAAA|80317|22|18|37|PM|second|evening|| +80318|AAAAAAAAPLJDBAAA|80318|22|18|38|PM|second|evening|| +80319|AAAAAAAAAMJDBAAA|80319|22|18|39|PM|second|evening|| +80320|AAAAAAAABMJDBAAA|80320|22|18|40|PM|second|evening|| +80321|AAAAAAAACMJDBAAA|80321|22|18|41|PM|second|evening|| +80322|AAAAAAAADMJDBAAA|80322|22|18|42|PM|second|evening|| +80323|AAAAAAAAEMJDBAAA|80323|22|18|43|PM|second|evening|| +80324|AAAAAAAAFMJDBAAA|80324|22|18|44|PM|second|evening|| +80325|AAAAAAAAGMJDBAAA|80325|22|18|45|PM|second|evening|| +80326|AAAAAAAAHMJDBAAA|80326|22|18|46|PM|second|evening|| +80327|AAAAAAAAIMJDBAAA|80327|22|18|47|PM|second|evening|| +80328|AAAAAAAAJMJDBAAA|80328|22|18|48|PM|second|evening|| +80329|AAAAAAAAKMJDBAAA|80329|22|18|49|PM|second|evening|| +80330|AAAAAAAALMJDBAAA|80330|22|18|50|PM|second|evening|| +80331|AAAAAAAAMMJDBAAA|80331|22|18|51|PM|second|evening|| +80332|AAAAAAAANMJDBAAA|80332|22|18|52|PM|second|evening|| +80333|AAAAAAAAOMJDBAAA|80333|22|18|53|PM|second|evening|| +80334|AAAAAAAAPMJDBAAA|80334|22|18|54|PM|second|evening|| +80335|AAAAAAAAANJDBAAA|80335|22|18|55|PM|second|evening|| +80336|AAAAAAAABNJDBAAA|80336|22|18|56|PM|second|evening|| +80337|AAAAAAAACNJDBAAA|80337|22|18|57|PM|second|evening|| +80338|AAAAAAAADNJDBAAA|80338|22|18|58|PM|second|evening|| +80339|AAAAAAAAENJDBAAA|80339|22|18|59|PM|second|evening|| +80340|AAAAAAAAFNJDBAAA|80340|22|19|0|PM|second|evening|| +80341|AAAAAAAAGNJDBAAA|80341|22|19|1|PM|second|evening|| +80342|AAAAAAAAHNJDBAAA|80342|22|19|2|PM|second|evening|| +80343|AAAAAAAAINJDBAAA|80343|22|19|3|PM|second|evening|| +80344|AAAAAAAAJNJDBAAA|80344|22|19|4|PM|second|evening|| +80345|AAAAAAAAKNJDBAAA|80345|22|19|5|PM|second|evening|| +80346|AAAAAAAALNJDBAAA|80346|22|19|6|PM|second|evening|| +80347|AAAAAAAAMNJDBAAA|80347|22|19|7|PM|second|evening|| +80348|AAAAAAAANNJDBAAA|80348|22|19|8|PM|second|evening|| +80349|AAAAAAAAONJDBAAA|80349|22|19|9|PM|second|evening|| +80350|AAAAAAAAPNJDBAAA|80350|22|19|10|PM|second|evening|| +80351|AAAAAAAAAOJDBAAA|80351|22|19|11|PM|second|evening|| +80352|AAAAAAAABOJDBAAA|80352|22|19|12|PM|second|evening|| +80353|AAAAAAAACOJDBAAA|80353|22|19|13|PM|second|evening|| +80354|AAAAAAAADOJDBAAA|80354|22|19|14|PM|second|evening|| +80355|AAAAAAAAEOJDBAAA|80355|22|19|15|PM|second|evening|| +80356|AAAAAAAAFOJDBAAA|80356|22|19|16|PM|second|evening|| +80357|AAAAAAAAGOJDBAAA|80357|22|19|17|PM|second|evening|| +80358|AAAAAAAAHOJDBAAA|80358|22|19|18|PM|second|evening|| +80359|AAAAAAAAIOJDBAAA|80359|22|19|19|PM|second|evening|| +80360|AAAAAAAAJOJDBAAA|80360|22|19|20|PM|second|evening|| +80361|AAAAAAAAKOJDBAAA|80361|22|19|21|PM|second|evening|| +80362|AAAAAAAALOJDBAAA|80362|22|19|22|PM|second|evening|| +80363|AAAAAAAAMOJDBAAA|80363|22|19|23|PM|second|evening|| +80364|AAAAAAAANOJDBAAA|80364|22|19|24|PM|second|evening|| +80365|AAAAAAAAOOJDBAAA|80365|22|19|25|PM|second|evening|| +80366|AAAAAAAAPOJDBAAA|80366|22|19|26|PM|second|evening|| +80367|AAAAAAAAAPJDBAAA|80367|22|19|27|PM|second|evening|| +80368|AAAAAAAABPJDBAAA|80368|22|19|28|PM|second|evening|| +80369|AAAAAAAACPJDBAAA|80369|22|19|29|PM|second|evening|| +80370|AAAAAAAADPJDBAAA|80370|22|19|30|PM|second|evening|| +80371|AAAAAAAAEPJDBAAA|80371|22|19|31|PM|second|evening|| +80372|AAAAAAAAFPJDBAAA|80372|22|19|32|PM|second|evening|| +80373|AAAAAAAAGPJDBAAA|80373|22|19|33|PM|second|evening|| +80374|AAAAAAAAHPJDBAAA|80374|22|19|34|PM|second|evening|| +80375|AAAAAAAAIPJDBAAA|80375|22|19|35|PM|second|evening|| +80376|AAAAAAAAJPJDBAAA|80376|22|19|36|PM|second|evening|| +80377|AAAAAAAAKPJDBAAA|80377|22|19|37|PM|second|evening|| +80378|AAAAAAAALPJDBAAA|80378|22|19|38|PM|second|evening|| +80379|AAAAAAAAMPJDBAAA|80379|22|19|39|PM|second|evening|| +80380|AAAAAAAANPJDBAAA|80380|22|19|40|PM|second|evening|| +80381|AAAAAAAAOPJDBAAA|80381|22|19|41|PM|second|evening|| +80382|AAAAAAAAPPJDBAAA|80382|22|19|42|PM|second|evening|| +80383|AAAAAAAAAAKDBAAA|80383|22|19|43|PM|second|evening|| +80384|AAAAAAAABAKDBAAA|80384|22|19|44|PM|second|evening|| +80385|AAAAAAAACAKDBAAA|80385|22|19|45|PM|second|evening|| +80386|AAAAAAAADAKDBAAA|80386|22|19|46|PM|second|evening|| +80387|AAAAAAAAEAKDBAAA|80387|22|19|47|PM|second|evening|| +80388|AAAAAAAAFAKDBAAA|80388|22|19|48|PM|second|evening|| +80389|AAAAAAAAGAKDBAAA|80389|22|19|49|PM|second|evening|| +80390|AAAAAAAAHAKDBAAA|80390|22|19|50|PM|second|evening|| +80391|AAAAAAAAIAKDBAAA|80391|22|19|51|PM|second|evening|| +80392|AAAAAAAAJAKDBAAA|80392|22|19|52|PM|second|evening|| +80393|AAAAAAAAKAKDBAAA|80393|22|19|53|PM|second|evening|| +80394|AAAAAAAALAKDBAAA|80394|22|19|54|PM|second|evening|| +80395|AAAAAAAAMAKDBAAA|80395|22|19|55|PM|second|evening|| +80396|AAAAAAAANAKDBAAA|80396|22|19|56|PM|second|evening|| +80397|AAAAAAAAOAKDBAAA|80397|22|19|57|PM|second|evening|| +80398|AAAAAAAAPAKDBAAA|80398|22|19|58|PM|second|evening|| +80399|AAAAAAAAABKDBAAA|80399|22|19|59|PM|second|evening|| +80400|AAAAAAAABBKDBAAA|80400|22|20|0|PM|second|evening|| +80401|AAAAAAAACBKDBAAA|80401|22|20|1|PM|second|evening|| +80402|AAAAAAAADBKDBAAA|80402|22|20|2|PM|second|evening|| +80403|AAAAAAAAEBKDBAAA|80403|22|20|3|PM|second|evening|| +80404|AAAAAAAAFBKDBAAA|80404|22|20|4|PM|second|evening|| +80405|AAAAAAAAGBKDBAAA|80405|22|20|5|PM|second|evening|| +80406|AAAAAAAAHBKDBAAA|80406|22|20|6|PM|second|evening|| +80407|AAAAAAAAIBKDBAAA|80407|22|20|7|PM|second|evening|| +80408|AAAAAAAAJBKDBAAA|80408|22|20|8|PM|second|evening|| +80409|AAAAAAAAKBKDBAAA|80409|22|20|9|PM|second|evening|| +80410|AAAAAAAALBKDBAAA|80410|22|20|10|PM|second|evening|| +80411|AAAAAAAAMBKDBAAA|80411|22|20|11|PM|second|evening|| +80412|AAAAAAAANBKDBAAA|80412|22|20|12|PM|second|evening|| +80413|AAAAAAAAOBKDBAAA|80413|22|20|13|PM|second|evening|| +80414|AAAAAAAAPBKDBAAA|80414|22|20|14|PM|second|evening|| +80415|AAAAAAAAACKDBAAA|80415|22|20|15|PM|second|evening|| +80416|AAAAAAAABCKDBAAA|80416|22|20|16|PM|second|evening|| +80417|AAAAAAAACCKDBAAA|80417|22|20|17|PM|second|evening|| +80418|AAAAAAAADCKDBAAA|80418|22|20|18|PM|second|evening|| +80419|AAAAAAAAECKDBAAA|80419|22|20|19|PM|second|evening|| +80420|AAAAAAAAFCKDBAAA|80420|22|20|20|PM|second|evening|| +80421|AAAAAAAAGCKDBAAA|80421|22|20|21|PM|second|evening|| +80422|AAAAAAAAHCKDBAAA|80422|22|20|22|PM|second|evening|| +80423|AAAAAAAAICKDBAAA|80423|22|20|23|PM|second|evening|| +80424|AAAAAAAAJCKDBAAA|80424|22|20|24|PM|second|evening|| +80425|AAAAAAAAKCKDBAAA|80425|22|20|25|PM|second|evening|| +80426|AAAAAAAALCKDBAAA|80426|22|20|26|PM|second|evening|| +80427|AAAAAAAAMCKDBAAA|80427|22|20|27|PM|second|evening|| +80428|AAAAAAAANCKDBAAA|80428|22|20|28|PM|second|evening|| +80429|AAAAAAAAOCKDBAAA|80429|22|20|29|PM|second|evening|| +80430|AAAAAAAAPCKDBAAA|80430|22|20|30|PM|second|evening|| +80431|AAAAAAAAADKDBAAA|80431|22|20|31|PM|second|evening|| +80432|AAAAAAAABDKDBAAA|80432|22|20|32|PM|second|evening|| +80433|AAAAAAAACDKDBAAA|80433|22|20|33|PM|second|evening|| +80434|AAAAAAAADDKDBAAA|80434|22|20|34|PM|second|evening|| +80435|AAAAAAAAEDKDBAAA|80435|22|20|35|PM|second|evening|| +80436|AAAAAAAAFDKDBAAA|80436|22|20|36|PM|second|evening|| +80437|AAAAAAAAGDKDBAAA|80437|22|20|37|PM|second|evening|| +80438|AAAAAAAAHDKDBAAA|80438|22|20|38|PM|second|evening|| +80439|AAAAAAAAIDKDBAAA|80439|22|20|39|PM|second|evening|| +80440|AAAAAAAAJDKDBAAA|80440|22|20|40|PM|second|evening|| +80441|AAAAAAAAKDKDBAAA|80441|22|20|41|PM|second|evening|| +80442|AAAAAAAALDKDBAAA|80442|22|20|42|PM|second|evening|| +80443|AAAAAAAAMDKDBAAA|80443|22|20|43|PM|second|evening|| +80444|AAAAAAAANDKDBAAA|80444|22|20|44|PM|second|evening|| +80445|AAAAAAAAODKDBAAA|80445|22|20|45|PM|second|evening|| +80446|AAAAAAAAPDKDBAAA|80446|22|20|46|PM|second|evening|| +80447|AAAAAAAAAEKDBAAA|80447|22|20|47|PM|second|evening|| +80448|AAAAAAAABEKDBAAA|80448|22|20|48|PM|second|evening|| +80449|AAAAAAAACEKDBAAA|80449|22|20|49|PM|second|evening|| +80450|AAAAAAAADEKDBAAA|80450|22|20|50|PM|second|evening|| +80451|AAAAAAAAEEKDBAAA|80451|22|20|51|PM|second|evening|| +80452|AAAAAAAAFEKDBAAA|80452|22|20|52|PM|second|evening|| +80453|AAAAAAAAGEKDBAAA|80453|22|20|53|PM|second|evening|| +80454|AAAAAAAAHEKDBAAA|80454|22|20|54|PM|second|evening|| +80455|AAAAAAAAIEKDBAAA|80455|22|20|55|PM|second|evening|| +80456|AAAAAAAAJEKDBAAA|80456|22|20|56|PM|second|evening|| +80457|AAAAAAAAKEKDBAAA|80457|22|20|57|PM|second|evening|| +80458|AAAAAAAALEKDBAAA|80458|22|20|58|PM|second|evening|| +80459|AAAAAAAAMEKDBAAA|80459|22|20|59|PM|second|evening|| +80460|AAAAAAAANEKDBAAA|80460|22|21|0|PM|second|evening|| +80461|AAAAAAAAOEKDBAAA|80461|22|21|1|PM|second|evening|| +80462|AAAAAAAAPEKDBAAA|80462|22|21|2|PM|second|evening|| +80463|AAAAAAAAAFKDBAAA|80463|22|21|3|PM|second|evening|| +80464|AAAAAAAABFKDBAAA|80464|22|21|4|PM|second|evening|| +80465|AAAAAAAACFKDBAAA|80465|22|21|5|PM|second|evening|| +80466|AAAAAAAADFKDBAAA|80466|22|21|6|PM|second|evening|| +80467|AAAAAAAAEFKDBAAA|80467|22|21|7|PM|second|evening|| +80468|AAAAAAAAFFKDBAAA|80468|22|21|8|PM|second|evening|| +80469|AAAAAAAAGFKDBAAA|80469|22|21|9|PM|second|evening|| +80470|AAAAAAAAHFKDBAAA|80470|22|21|10|PM|second|evening|| +80471|AAAAAAAAIFKDBAAA|80471|22|21|11|PM|second|evening|| +80472|AAAAAAAAJFKDBAAA|80472|22|21|12|PM|second|evening|| +80473|AAAAAAAAKFKDBAAA|80473|22|21|13|PM|second|evening|| +80474|AAAAAAAALFKDBAAA|80474|22|21|14|PM|second|evening|| +80475|AAAAAAAAMFKDBAAA|80475|22|21|15|PM|second|evening|| +80476|AAAAAAAANFKDBAAA|80476|22|21|16|PM|second|evening|| +80477|AAAAAAAAOFKDBAAA|80477|22|21|17|PM|second|evening|| +80478|AAAAAAAAPFKDBAAA|80478|22|21|18|PM|second|evening|| +80479|AAAAAAAAAGKDBAAA|80479|22|21|19|PM|second|evening|| +80480|AAAAAAAABGKDBAAA|80480|22|21|20|PM|second|evening|| +80481|AAAAAAAACGKDBAAA|80481|22|21|21|PM|second|evening|| +80482|AAAAAAAADGKDBAAA|80482|22|21|22|PM|second|evening|| +80483|AAAAAAAAEGKDBAAA|80483|22|21|23|PM|second|evening|| +80484|AAAAAAAAFGKDBAAA|80484|22|21|24|PM|second|evening|| +80485|AAAAAAAAGGKDBAAA|80485|22|21|25|PM|second|evening|| +80486|AAAAAAAAHGKDBAAA|80486|22|21|26|PM|second|evening|| +80487|AAAAAAAAIGKDBAAA|80487|22|21|27|PM|second|evening|| +80488|AAAAAAAAJGKDBAAA|80488|22|21|28|PM|second|evening|| +80489|AAAAAAAAKGKDBAAA|80489|22|21|29|PM|second|evening|| +80490|AAAAAAAALGKDBAAA|80490|22|21|30|PM|second|evening|| +80491|AAAAAAAAMGKDBAAA|80491|22|21|31|PM|second|evening|| +80492|AAAAAAAANGKDBAAA|80492|22|21|32|PM|second|evening|| +80493|AAAAAAAAOGKDBAAA|80493|22|21|33|PM|second|evening|| +80494|AAAAAAAAPGKDBAAA|80494|22|21|34|PM|second|evening|| +80495|AAAAAAAAAHKDBAAA|80495|22|21|35|PM|second|evening|| +80496|AAAAAAAABHKDBAAA|80496|22|21|36|PM|second|evening|| +80497|AAAAAAAACHKDBAAA|80497|22|21|37|PM|second|evening|| +80498|AAAAAAAADHKDBAAA|80498|22|21|38|PM|second|evening|| +80499|AAAAAAAAEHKDBAAA|80499|22|21|39|PM|second|evening|| +80500|AAAAAAAAFHKDBAAA|80500|22|21|40|PM|second|evening|| +80501|AAAAAAAAGHKDBAAA|80501|22|21|41|PM|second|evening|| +80502|AAAAAAAAHHKDBAAA|80502|22|21|42|PM|second|evening|| +80503|AAAAAAAAIHKDBAAA|80503|22|21|43|PM|second|evening|| +80504|AAAAAAAAJHKDBAAA|80504|22|21|44|PM|second|evening|| +80505|AAAAAAAAKHKDBAAA|80505|22|21|45|PM|second|evening|| +80506|AAAAAAAALHKDBAAA|80506|22|21|46|PM|second|evening|| +80507|AAAAAAAAMHKDBAAA|80507|22|21|47|PM|second|evening|| +80508|AAAAAAAANHKDBAAA|80508|22|21|48|PM|second|evening|| +80509|AAAAAAAAOHKDBAAA|80509|22|21|49|PM|second|evening|| +80510|AAAAAAAAPHKDBAAA|80510|22|21|50|PM|second|evening|| +80511|AAAAAAAAAIKDBAAA|80511|22|21|51|PM|second|evening|| +80512|AAAAAAAABIKDBAAA|80512|22|21|52|PM|second|evening|| +80513|AAAAAAAACIKDBAAA|80513|22|21|53|PM|second|evening|| +80514|AAAAAAAADIKDBAAA|80514|22|21|54|PM|second|evening|| +80515|AAAAAAAAEIKDBAAA|80515|22|21|55|PM|second|evening|| +80516|AAAAAAAAFIKDBAAA|80516|22|21|56|PM|second|evening|| +80517|AAAAAAAAGIKDBAAA|80517|22|21|57|PM|second|evening|| +80518|AAAAAAAAHIKDBAAA|80518|22|21|58|PM|second|evening|| +80519|AAAAAAAAIIKDBAAA|80519|22|21|59|PM|second|evening|| +80520|AAAAAAAAJIKDBAAA|80520|22|22|0|PM|second|evening|| +80521|AAAAAAAAKIKDBAAA|80521|22|22|1|PM|second|evening|| +80522|AAAAAAAALIKDBAAA|80522|22|22|2|PM|second|evening|| +80523|AAAAAAAAMIKDBAAA|80523|22|22|3|PM|second|evening|| +80524|AAAAAAAANIKDBAAA|80524|22|22|4|PM|second|evening|| +80525|AAAAAAAAOIKDBAAA|80525|22|22|5|PM|second|evening|| +80526|AAAAAAAAPIKDBAAA|80526|22|22|6|PM|second|evening|| +80527|AAAAAAAAAJKDBAAA|80527|22|22|7|PM|second|evening|| +80528|AAAAAAAABJKDBAAA|80528|22|22|8|PM|second|evening|| +80529|AAAAAAAACJKDBAAA|80529|22|22|9|PM|second|evening|| +80530|AAAAAAAADJKDBAAA|80530|22|22|10|PM|second|evening|| +80531|AAAAAAAAEJKDBAAA|80531|22|22|11|PM|second|evening|| +80532|AAAAAAAAFJKDBAAA|80532|22|22|12|PM|second|evening|| +80533|AAAAAAAAGJKDBAAA|80533|22|22|13|PM|second|evening|| +80534|AAAAAAAAHJKDBAAA|80534|22|22|14|PM|second|evening|| +80535|AAAAAAAAIJKDBAAA|80535|22|22|15|PM|second|evening|| +80536|AAAAAAAAJJKDBAAA|80536|22|22|16|PM|second|evening|| +80537|AAAAAAAAKJKDBAAA|80537|22|22|17|PM|second|evening|| +80538|AAAAAAAALJKDBAAA|80538|22|22|18|PM|second|evening|| +80539|AAAAAAAAMJKDBAAA|80539|22|22|19|PM|second|evening|| +80540|AAAAAAAANJKDBAAA|80540|22|22|20|PM|second|evening|| +80541|AAAAAAAAOJKDBAAA|80541|22|22|21|PM|second|evening|| +80542|AAAAAAAAPJKDBAAA|80542|22|22|22|PM|second|evening|| +80543|AAAAAAAAAKKDBAAA|80543|22|22|23|PM|second|evening|| +80544|AAAAAAAABKKDBAAA|80544|22|22|24|PM|second|evening|| +80545|AAAAAAAACKKDBAAA|80545|22|22|25|PM|second|evening|| +80546|AAAAAAAADKKDBAAA|80546|22|22|26|PM|second|evening|| +80547|AAAAAAAAEKKDBAAA|80547|22|22|27|PM|second|evening|| +80548|AAAAAAAAFKKDBAAA|80548|22|22|28|PM|second|evening|| +80549|AAAAAAAAGKKDBAAA|80549|22|22|29|PM|second|evening|| +80550|AAAAAAAAHKKDBAAA|80550|22|22|30|PM|second|evening|| +80551|AAAAAAAAIKKDBAAA|80551|22|22|31|PM|second|evening|| +80552|AAAAAAAAJKKDBAAA|80552|22|22|32|PM|second|evening|| +80553|AAAAAAAAKKKDBAAA|80553|22|22|33|PM|second|evening|| +80554|AAAAAAAALKKDBAAA|80554|22|22|34|PM|second|evening|| +80555|AAAAAAAAMKKDBAAA|80555|22|22|35|PM|second|evening|| +80556|AAAAAAAANKKDBAAA|80556|22|22|36|PM|second|evening|| +80557|AAAAAAAAOKKDBAAA|80557|22|22|37|PM|second|evening|| +80558|AAAAAAAAPKKDBAAA|80558|22|22|38|PM|second|evening|| +80559|AAAAAAAAALKDBAAA|80559|22|22|39|PM|second|evening|| +80560|AAAAAAAABLKDBAAA|80560|22|22|40|PM|second|evening|| +80561|AAAAAAAACLKDBAAA|80561|22|22|41|PM|second|evening|| +80562|AAAAAAAADLKDBAAA|80562|22|22|42|PM|second|evening|| +80563|AAAAAAAAELKDBAAA|80563|22|22|43|PM|second|evening|| +80564|AAAAAAAAFLKDBAAA|80564|22|22|44|PM|second|evening|| +80565|AAAAAAAAGLKDBAAA|80565|22|22|45|PM|second|evening|| +80566|AAAAAAAAHLKDBAAA|80566|22|22|46|PM|second|evening|| +80567|AAAAAAAAILKDBAAA|80567|22|22|47|PM|second|evening|| +80568|AAAAAAAAJLKDBAAA|80568|22|22|48|PM|second|evening|| +80569|AAAAAAAAKLKDBAAA|80569|22|22|49|PM|second|evening|| +80570|AAAAAAAALLKDBAAA|80570|22|22|50|PM|second|evening|| +80571|AAAAAAAAMLKDBAAA|80571|22|22|51|PM|second|evening|| +80572|AAAAAAAANLKDBAAA|80572|22|22|52|PM|second|evening|| +80573|AAAAAAAAOLKDBAAA|80573|22|22|53|PM|second|evening|| +80574|AAAAAAAAPLKDBAAA|80574|22|22|54|PM|second|evening|| +80575|AAAAAAAAAMKDBAAA|80575|22|22|55|PM|second|evening|| +80576|AAAAAAAABMKDBAAA|80576|22|22|56|PM|second|evening|| +80577|AAAAAAAACMKDBAAA|80577|22|22|57|PM|second|evening|| +80578|AAAAAAAADMKDBAAA|80578|22|22|58|PM|second|evening|| +80579|AAAAAAAAEMKDBAAA|80579|22|22|59|PM|second|evening|| +80580|AAAAAAAAFMKDBAAA|80580|22|23|0|PM|second|evening|| +80581|AAAAAAAAGMKDBAAA|80581|22|23|1|PM|second|evening|| +80582|AAAAAAAAHMKDBAAA|80582|22|23|2|PM|second|evening|| +80583|AAAAAAAAIMKDBAAA|80583|22|23|3|PM|second|evening|| +80584|AAAAAAAAJMKDBAAA|80584|22|23|4|PM|second|evening|| +80585|AAAAAAAAKMKDBAAA|80585|22|23|5|PM|second|evening|| +80586|AAAAAAAALMKDBAAA|80586|22|23|6|PM|second|evening|| +80587|AAAAAAAAMMKDBAAA|80587|22|23|7|PM|second|evening|| +80588|AAAAAAAANMKDBAAA|80588|22|23|8|PM|second|evening|| +80589|AAAAAAAAOMKDBAAA|80589|22|23|9|PM|second|evening|| +80590|AAAAAAAAPMKDBAAA|80590|22|23|10|PM|second|evening|| +80591|AAAAAAAAANKDBAAA|80591|22|23|11|PM|second|evening|| +80592|AAAAAAAABNKDBAAA|80592|22|23|12|PM|second|evening|| +80593|AAAAAAAACNKDBAAA|80593|22|23|13|PM|second|evening|| +80594|AAAAAAAADNKDBAAA|80594|22|23|14|PM|second|evening|| +80595|AAAAAAAAENKDBAAA|80595|22|23|15|PM|second|evening|| +80596|AAAAAAAAFNKDBAAA|80596|22|23|16|PM|second|evening|| +80597|AAAAAAAAGNKDBAAA|80597|22|23|17|PM|second|evening|| +80598|AAAAAAAAHNKDBAAA|80598|22|23|18|PM|second|evening|| +80599|AAAAAAAAINKDBAAA|80599|22|23|19|PM|second|evening|| +80600|AAAAAAAAJNKDBAAA|80600|22|23|20|PM|second|evening|| +80601|AAAAAAAAKNKDBAAA|80601|22|23|21|PM|second|evening|| +80602|AAAAAAAALNKDBAAA|80602|22|23|22|PM|second|evening|| +80603|AAAAAAAAMNKDBAAA|80603|22|23|23|PM|second|evening|| +80604|AAAAAAAANNKDBAAA|80604|22|23|24|PM|second|evening|| +80605|AAAAAAAAONKDBAAA|80605|22|23|25|PM|second|evening|| +80606|AAAAAAAAPNKDBAAA|80606|22|23|26|PM|second|evening|| +80607|AAAAAAAAAOKDBAAA|80607|22|23|27|PM|second|evening|| +80608|AAAAAAAABOKDBAAA|80608|22|23|28|PM|second|evening|| +80609|AAAAAAAACOKDBAAA|80609|22|23|29|PM|second|evening|| +80610|AAAAAAAADOKDBAAA|80610|22|23|30|PM|second|evening|| +80611|AAAAAAAAEOKDBAAA|80611|22|23|31|PM|second|evening|| +80612|AAAAAAAAFOKDBAAA|80612|22|23|32|PM|second|evening|| +80613|AAAAAAAAGOKDBAAA|80613|22|23|33|PM|second|evening|| +80614|AAAAAAAAHOKDBAAA|80614|22|23|34|PM|second|evening|| +80615|AAAAAAAAIOKDBAAA|80615|22|23|35|PM|second|evening|| +80616|AAAAAAAAJOKDBAAA|80616|22|23|36|PM|second|evening|| +80617|AAAAAAAAKOKDBAAA|80617|22|23|37|PM|second|evening|| +80618|AAAAAAAALOKDBAAA|80618|22|23|38|PM|second|evening|| +80619|AAAAAAAAMOKDBAAA|80619|22|23|39|PM|second|evening|| +80620|AAAAAAAANOKDBAAA|80620|22|23|40|PM|second|evening|| +80621|AAAAAAAAOOKDBAAA|80621|22|23|41|PM|second|evening|| +80622|AAAAAAAAPOKDBAAA|80622|22|23|42|PM|second|evening|| +80623|AAAAAAAAAPKDBAAA|80623|22|23|43|PM|second|evening|| +80624|AAAAAAAABPKDBAAA|80624|22|23|44|PM|second|evening|| +80625|AAAAAAAACPKDBAAA|80625|22|23|45|PM|second|evening|| +80626|AAAAAAAADPKDBAAA|80626|22|23|46|PM|second|evening|| +80627|AAAAAAAAEPKDBAAA|80627|22|23|47|PM|second|evening|| +80628|AAAAAAAAFPKDBAAA|80628|22|23|48|PM|second|evening|| +80629|AAAAAAAAGPKDBAAA|80629|22|23|49|PM|second|evening|| +80630|AAAAAAAAHPKDBAAA|80630|22|23|50|PM|second|evening|| +80631|AAAAAAAAIPKDBAAA|80631|22|23|51|PM|second|evening|| +80632|AAAAAAAAJPKDBAAA|80632|22|23|52|PM|second|evening|| +80633|AAAAAAAAKPKDBAAA|80633|22|23|53|PM|second|evening|| +80634|AAAAAAAALPKDBAAA|80634|22|23|54|PM|second|evening|| +80635|AAAAAAAAMPKDBAAA|80635|22|23|55|PM|second|evening|| +80636|AAAAAAAANPKDBAAA|80636|22|23|56|PM|second|evening|| +80637|AAAAAAAAOPKDBAAA|80637|22|23|57|PM|second|evening|| +80638|AAAAAAAAPPKDBAAA|80638|22|23|58|PM|second|evening|| +80639|AAAAAAAAAALDBAAA|80639|22|23|59|PM|second|evening|| +80640|AAAAAAAABALDBAAA|80640|22|24|0|PM|second|evening|| +80641|AAAAAAAACALDBAAA|80641|22|24|1|PM|second|evening|| +80642|AAAAAAAADALDBAAA|80642|22|24|2|PM|second|evening|| +80643|AAAAAAAAEALDBAAA|80643|22|24|3|PM|second|evening|| +80644|AAAAAAAAFALDBAAA|80644|22|24|4|PM|second|evening|| +80645|AAAAAAAAGALDBAAA|80645|22|24|5|PM|second|evening|| +80646|AAAAAAAAHALDBAAA|80646|22|24|6|PM|second|evening|| +80647|AAAAAAAAIALDBAAA|80647|22|24|7|PM|second|evening|| +80648|AAAAAAAAJALDBAAA|80648|22|24|8|PM|second|evening|| +80649|AAAAAAAAKALDBAAA|80649|22|24|9|PM|second|evening|| +80650|AAAAAAAALALDBAAA|80650|22|24|10|PM|second|evening|| +80651|AAAAAAAAMALDBAAA|80651|22|24|11|PM|second|evening|| +80652|AAAAAAAANALDBAAA|80652|22|24|12|PM|second|evening|| +80653|AAAAAAAAOALDBAAA|80653|22|24|13|PM|second|evening|| +80654|AAAAAAAAPALDBAAA|80654|22|24|14|PM|second|evening|| +80655|AAAAAAAAABLDBAAA|80655|22|24|15|PM|second|evening|| +80656|AAAAAAAABBLDBAAA|80656|22|24|16|PM|second|evening|| +80657|AAAAAAAACBLDBAAA|80657|22|24|17|PM|second|evening|| +80658|AAAAAAAADBLDBAAA|80658|22|24|18|PM|second|evening|| +80659|AAAAAAAAEBLDBAAA|80659|22|24|19|PM|second|evening|| +80660|AAAAAAAAFBLDBAAA|80660|22|24|20|PM|second|evening|| +80661|AAAAAAAAGBLDBAAA|80661|22|24|21|PM|second|evening|| +80662|AAAAAAAAHBLDBAAA|80662|22|24|22|PM|second|evening|| +80663|AAAAAAAAIBLDBAAA|80663|22|24|23|PM|second|evening|| +80664|AAAAAAAAJBLDBAAA|80664|22|24|24|PM|second|evening|| +80665|AAAAAAAAKBLDBAAA|80665|22|24|25|PM|second|evening|| +80666|AAAAAAAALBLDBAAA|80666|22|24|26|PM|second|evening|| +80667|AAAAAAAAMBLDBAAA|80667|22|24|27|PM|second|evening|| +80668|AAAAAAAANBLDBAAA|80668|22|24|28|PM|second|evening|| +80669|AAAAAAAAOBLDBAAA|80669|22|24|29|PM|second|evening|| +80670|AAAAAAAAPBLDBAAA|80670|22|24|30|PM|second|evening|| +80671|AAAAAAAAACLDBAAA|80671|22|24|31|PM|second|evening|| +80672|AAAAAAAABCLDBAAA|80672|22|24|32|PM|second|evening|| +80673|AAAAAAAACCLDBAAA|80673|22|24|33|PM|second|evening|| +80674|AAAAAAAADCLDBAAA|80674|22|24|34|PM|second|evening|| +80675|AAAAAAAAECLDBAAA|80675|22|24|35|PM|second|evening|| +80676|AAAAAAAAFCLDBAAA|80676|22|24|36|PM|second|evening|| +80677|AAAAAAAAGCLDBAAA|80677|22|24|37|PM|second|evening|| +80678|AAAAAAAAHCLDBAAA|80678|22|24|38|PM|second|evening|| +80679|AAAAAAAAICLDBAAA|80679|22|24|39|PM|second|evening|| +80680|AAAAAAAAJCLDBAAA|80680|22|24|40|PM|second|evening|| +80681|AAAAAAAAKCLDBAAA|80681|22|24|41|PM|second|evening|| +80682|AAAAAAAALCLDBAAA|80682|22|24|42|PM|second|evening|| +80683|AAAAAAAAMCLDBAAA|80683|22|24|43|PM|second|evening|| +80684|AAAAAAAANCLDBAAA|80684|22|24|44|PM|second|evening|| +80685|AAAAAAAAOCLDBAAA|80685|22|24|45|PM|second|evening|| +80686|AAAAAAAAPCLDBAAA|80686|22|24|46|PM|second|evening|| +80687|AAAAAAAAADLDBAAA|80687|22|24|47|PM|second|evening|| +80688|AAAAAAAABDLDBAAA|80688|22|24|48|PM|second|evening|| +80689|AAAAAAAACDLDBAAA|80689|22|24|49|PM|second|evening|| +80690|AAAAAAAADDLDBAAA|80690|22|24|50|PM|second|evening|| +80691|AAAAAAAAEDLDBAAA|80691|22|24|51|PM|second|evening|| +80692|AAAAAAAAFDLDBAAA|80692|22|24|52|PM|second|evening|| +80693|AAAAAAAAGDLDBAAA|80693|22|24|53|PM|second|evening|| +80694|AAAAAAAAHDLDBAAA|80694|22|24|54|PM|second|evening|| +80695|AAAAAAAAIDLDBAAA|80695|22|24|55|PM|second|evening|| +80696|AAAAAAAAJDLDBAAA|80696|22|24|56|PM|second|evening|| +80697|AAAAAAAAKDLDBAAA|80697|22|24|57|PM|second|evening|| +80698|AAAAAAAALDLDBAAA|80698|22|24|58|PM|second|evening|| +80699|AAAAAAAAMDLDBAAA|80699|22|24|59|PM|second|evening|| +80700|AAAAAAAANDLDBAAA|80700|22|25|0|PM|second|evening|| +80701|AAAAAAAAODLDBAAA|80701|22|25|1|PM|second|evening|| +80702|AAAAAAAAPDLDBAAA|80702|22|25|2|PM|second|evening|| +80703|AAAAAAAAAELDBAAA|80703|22|25|3|PM|second|evening|| +80704|AAAAAAAABELDBAAA|80704|22|25|4|PM|second|evening|| +80705|AAAAAAAACELDBAAA|80705|22|25|5|PM|second|evening|| +80706|AAAAAAAADELDBAAA|80706|22|25|6|PM|second|evening|| +80707|AAAAAAAAEELDBAAA|80707|22|25|7|PM|second|evening|| +80708|AAAAAAAAFELDBAAA|80708|22|25|8|PM|second|evening|| +80709|AAAAAAAAGELDBAAA|80709|22|25|9|PM|second|evening|| +80710|AAAAAAAAHELDBAAA|80710|22|25|10|PM|second|evening|| +80711|AAAAAAAAIELDBAAA|80711|22|25|11|PM|second|evening|| +80712|AAAAAAAAJELDBAAA|80712|22|25|12|PM|second|evening|| +80713|AAAAAAAAKELDBAAA|80713|22|25|13|PM|second|evening|| +80714|AAAAAAAALELDBAAA|80714|22|25|14|PM|second|evening|| +80715|AAAAAAAAMELDBAAA|80715|22|25|15|PM|second|evening|| +80716|AAAAAAAANELDBAAA|80716|22|25|16|PM|second|evening|| +80717|AAAAAAAAOELDBAAA|80717|22|25|17|PM|second|evening|| +80718|AAAAAAAAPELDBAAA|80718|22|25|18|PM|second|evening|| +80719|AAAAAAAAAFLDBAAA|80719|22|25|19|PM|second|evening|| +80720|AAAAAAAABFLDBAAA|80720|22|25|20|PM|second|evening|| +80721|AAAAAAAACFLDBAAA|80721|22|25|21|PM|second|evening|| +80722|AAAAAAAADFLDBAAA|80722|22|25|22|PM|second|evening|| +80723|AAAAAAAAEFLDBAAA|80723|22|25|23|PM|second|evening|| +80724|AAAAAAAAFFLDBAAA|80724|22|25|24|PM|second|evening|| +80725|AAAAAAAAGFLDBAAA|80725|22|25|25|PM|second|evening|| +80726|AAAAAAAAHFLDBAAA|80726|22|25|26|PM|second|evening|| +80727|AAAAAAAAIFLDBAAA|80727|22|25|27|PM|second|evening|| +80728|AAAAAAAAJFLDBAAA|80728|22|25|28|PM|second|evening|| +80729|AAAAAAAAKFLDBAAA|80729|22|25|29|PM|second|evening|| +80730|AAAAAAAALFLDBAAA|80730|22|25|30|PM|second|evening|| +80731|AAAAAAAAMFLDBAAA|80731|22|25|31|PM|second|evening|| +80732|AAAAAAAANFLDBAAA|80732|22|25|32|PM|second|evening|| +80733|AAAAAAAAOFLDBAAA|80733|22|25|33|PM|second|evening|| +80734|AAAAAAAAPFLDBAAA|80734|22|25|34|PM|second|evening|| +80735|AAAAAAAAAGLDBAAA|80735|22|25|35|PM|second|evening|| +80736|AAAAAAAABGLDBAAA|80736|22|25|36|PM|second|evening|| +80737|AAAAAAAACGLDBAAA|80737|22|25|37|PM|second|evening|| +80738|AAAAAAAADGLDBAAA|80738|22|25|38|PM|second|evening|| +80739|AAAAAAAAEGLDBAAA|80739|22|25|39|PM|second|evening|| +80740|AAAAAAAAFGLDBAAA|80740|22|25|40|PM|second|evening|| +80741|AAAAAAAAGGLDBAAA|80741|22|25|41|PM|second|evening|| +80742|AAAAAAAAHGLDBAAA|80742|22|25|42|PM|second|evening|| +80743|AAAAAAAAIGLDBAAA|80743|22|25|43|PM|second|evening|| +80744|AAAAAAAAJGLDBAAA|80744|22|25|44|PM|second|evening|| +80745|AAAAAAAAKGLDBAAA|80745|22|25|45|PM|second|evening|| +80746|AAAAAAAALGLDBAAA|80746|22|25|46|PM|second|evening|| +80747|AAAAAAAAMGLDBAAA|80747|22|25|47|PM|second|evening|| +80748|AAAAAAAANGLDBAAA|80748|22|25|48|PM|second|evening|| +80749|AAAAAAAAOGLDBAAA|80749|22|25|49|PM|second|evening|| +80750|AAAAAAAAPGLDBAAA|80750|22|25|50|PM|second|evening|| +80751|AAAAAAAAAHLDBAAA|80751|22|25|51|PM|second|evening|| +80752|AAAAAAAABHLDBAAA|80752|22|25|52|PM|second|evening|| +80753|AAAAAAAACHLDBAAA|80753|22|25|53|PM|second|evening|| +80754|AAAAAAAADHLDBAAA|80754|22|25|54|PM|second|evening|| +80755|AAAAAAAAEHLDBAAA|80755|22|25|55|PM|second|evening|| +80756|AAAAAAAAFHLDBAAA|80756|22|25|56|PM|second|evening|| +80757|AAAAAAAAGHLDBAAA|80757|22|25|57|PM|second|evening|| +80758|AAAAAAAAHHLDBAAA|80758|22|25|58|PM|second|evening|| +80759|AAAAAAAAIHLDBAAA|80759|22|25|59|PM|second|evening|| +80760|AAAAAAAAJHLDBAAA|80760|22|26|0|PM|second|evening|| +80761|AAAAAAAAKHLDBAAA|80761|22|26|1|PM|second|evening|| +80762|AAAAAAAALHLDBAAA|80762|22|26|2|PM|second|evening|| +80763|AAAAAAAAMHLDBAAA|80763|22|26|3|PM|second|evening|| +80764|AAAAAAAANHLDBAAA|80764|22|26|4|PM|second|evening|| +80765|AAAAAAAAOHLDBAAA|80765|22|26|5|PM|second|evening|| +80766|AAAAAAAAPHLDBAAA|80766|22|26|6|PM|second|evening|| +80767|AAAAAAAAAILDBAAA|80767|22|26|7|PM|second|evening|| +80768|AAAAAAAABILDBAAA|80768|22|26|8|PM|second|evening|| +80769|AAAAAAAACILDBAAA|80769|22|26|9|PM|second|evening|| +80770|AAAAAAAADILDBAAA|80770|22|26|10|PM|second|evening|| +80771|AAAAAAAAEILDBAAA|80771|22|26|11|PM|second|evening|| +80772|AAAAAAAAFILDBAAA|80772|22|26|12|PM|second|evening|| +80773|AAAAAAAAGILDBAAA|80773|22|26|13|PM|second|evening|| +80774|AAAAAAAAHILDBAAA|80774|22|26|14|PM|second|evening|| +80775|AAAAAAAAIILDBAAA|80775|22|26|15|PM|second|evening|| +80776|AAAAAAAAJILDBAAA|80776|22|26|16|PM|second|evening|| +80777|AAAAAAAAKILDBAAA|80777|22|26|17|PM|second|evening|| +80778|AAAAAAAALILDBAAA|80778|22|26|18|PM|second|evening|| +80779|AAAAAAAAMILDBAAA|80779|22|26|19|PM|second|evening|| +80780|AAAAAAAANILDBAAA|80780|22|26|20|PM|second|evening|| +80781|AAAAAAAAOILDBAAA|80781|22|26|21|PM|second|evening|| +80782|AAAAAAAAPILDBAAA|80782|22|26|22|PM|second|evening|| +80783|AAAAAAAAAJLDBAAA|80783|22|26|23|PM|second|evening|| +80784|AAAAAAAABJLDBAAA|80784|22|26|24|PM|second|evening|| +80785|AAAAAAAACJLDBAAA|80785|22|26|25|PM|second|evening|| +80786|AAAAAAAADJLDBAAA|80786|22|26|26|PM|second|evening|| +80787|AAAAAAAAEJLDBAAA|80787|22|26|27|PM|second|evening|| +80788|AAAAAAAAFJLDBAAA|80788|22|26|28|PM|second|evening|| +80789|AAAAAAAAGJLDBAAA|80789|22|26|29|PM|second|evening|| +80790|AAAAAAAAHJLDBAAA|80790|22|26|30|PM|second|evening|| +80791|AAAAAAAAIJLDBAAA|80791|22|26|31|PM|second|evening|| +80792|AAAAAAAAJJLDBAAA|80792|22|26|32|PM|second|evening|| +80793|AAAAAAAAKJLDBAAA|80793|22|26|33|PM|second|evening|| +80794|AAAAAAAALJLDBAAA|80794|22|26|34|PM|second|evening|| +80795|AAAAAAAAMJLDBAAA|80795|22|26|35|PM|second|evening|| +80796|AAAAAAAANJLDBAAA|80796|22|26|36|PM|second|evening|| +80797|AAAAAAAAOJLDBAAA|80797|22|26|37|PM|second|evening|| +80798|AAAAAAAAPJLDBAAA|80798|22|26|38|PM|second|evening|| +80799|AAAAAAAAAKLDBAAA|80799|22|26|39|PM|second|evening|| +80800|AAAAAAAABKLDBAAA|80800|22|26|40|PM|second|evening|| +80801|AAAAAAAACKLDBAAA|80801|22|26|41|PM|second|evening|| +80802|AAAAAAAADKLDBAAA|80802|22|26|42|PM|second|evening|| +80803|AAAAAAAAEKLDBAAA|80803|22|26|43|PM|second|evening|| +80804|AAAAAAAAFKLDBAAA|80804|22|26|44|PM|second|evening|| +80805|AAAAAAAAGKLDBAAA|80805|22|26|45|PM|second|evening|| +80806|AAAAAAAAHKLDBAAA|80806|22|26|46|PM|second|evening|| +80807|AAAAAAAAIKLDBAAA|80807|22|26|47|PM|second|evening|| +80808|AAAAAAAAJKLDBAAA|80808|22|26|48|PM|second|evening|| +80809|AAAAAAAAKKLDBAAA|80809|22|26|49|PM|second|evening|| +80810|AAAAAAAALKLDBAAA|80810|22|26|50|PM|second|evening|| +80811|AAAAAAAAMKLDBAAA|80811|22|26|51|PM|second|evening|| +80812|AAAAAAAANKLDBAAA|80812|22|26|52|PM|second|evening|| +80813|AAAAAAAAOKLDBAAA|80813|22|26|53|PM|second|evening|| +80814|AAAAAAAAPKLDBAAA|80814|22|26|54|PM|second|evening|| +80815|AAAAAAAAALLDBAAA|80815|22|26|55|PM|second|evening|| +80816|AAAAAAAABLLDBAAA|80816|22|26|56|PM|second|evening|| +80817|AAAAAAAACLLDBAAA|80817|22|26|57|PM|second|evening|| +80818|AAAAAAAADLLDBAAA|80818|22|26|58|PM|second|evening|| +80819|AAAAAAAAELLDBAAA|80819|22|26|59|PM|second|evening|| +80820|AAAAAAAAFLLDBAAA|80820|22|27|0|PM|second|evening|| +80821|AAAAAAAAGLLDBAAA|80821|22|27|1|PM|second|evening|| +80822|AAAAAAAAHLLDBAAA|80822|22|27|2|PM|second|evening|| +80823|AAAAAAAAILLDBAAA|80823|22|27|3|PM|second|evening|| +80824|AAAAAAAAJLLDBAAA|80824|22|27|4|PM|second|evening|| +80825|AAAAAAAAKLLDBAAA|80825|22|27|5|PM|second|evening|| +80826|AAAAAAAALLLDBAAA|80826|22|27|6|PM|second|evening|| +80827|AAAAAAAAMLLDBAAA|80827|22|27|7|PM|second|evening|| +80828|AAAAAAAANLLDBAAA|80828|22|27|8|PM|second|evening|| +80829|AAAAAAAAOLLDBAAA|80829|22|27|9|PM|second|evening|| +80830|AAAAAAAAPLLDBAAA|80830|22|27|10|PM|second|evening|| +80831|AAAAAAAAAMLDBAAA|80831|22|27|11|PM|second|evening|| +80832|AAAAAAAABMLDBAAA|80832|22|27|12|PM|second|evening|| +80833|AAAAAAAACMLDBAAA|80833|22|27|13|PM|second|evening|| +80834|AAAAAAAADMLDBAAA|80834|22|27|14|PM|second|evening|| +80835|AAAAAAAAEMLDBAAA|80835|22|27|15|PM|second|evening|| +80836|AAAAAAAAFMLDBAAA|80836|22|27|16|PM|second|evening|| +80837|AAAAAAAAGMLDBAAA|80837|22|27|17|PM|second|evening|| +80838|AAAAAAAAHMLDBAAA|80838|22|27|18|PM|second|evening|| +80839|AAAAAAAAIMLDBAAA|80839|22|27|19|PM|second|evening|| +80840|AAAAAAAAJMLDBAAA|80840|22|27|20|PM|second|evening|| +80841|AAAAAAAAKMLDBAAA|80841|22|27|21|PM|second|evening|| +80842|AAAAAAAALMLDBAAA|80842|22|27|22|PM|second|evening|| +80843|AAAAAAAAMMLDBAAA|80843|22|27|23|PM|second|evening|| +80844|AAAAAAAANMLDBAAA|80844|22|27|24|PM|second|evening|| +80845|AAAAAAAAOMLDBAAA|80845|22|27|25|PM|second|evening|| +80846|AAAAAAAAPMLDBAAA|80846|22|27|26|PM|second|evening|| +80847|AAAAAAAAANLDBAAA|80847|22|27|27|PM|second|evening|| +80848|AAAAAAAABNLDBAAA|80848|22|27|28|PM|second|evening|| +80849|AAAAAAAACNLDBAAA|80849|22|27|29|PM|second|evening|| +80850|AAAAAAAADNLDBAAA|80850|22|27|30|PM|second|evening|| +80851|AAAAAAAAENLDBAAA|80851|22|27|31|PM|second|evening|| +80852|AAAAAAAAFNLDBAAA|80852|22|27|32|PM|second|evening|| +80853|AAAAAAAAGNLDBAAA|80853|22|27|33|PM|second|evening|| +80854|AAAAAAAAHNLDBAAA|80854|22|27|34|PM|second|evening|| +80855|AAAAAAAAINLDBAAA|80855|22|27|35|PM|second|evening|| +80856|AAAAAAAAJNLDBAAA|80856|22|27|36|PM|second|evening|| +80857|AAAAAAAAKNLDBAAA|80857|22|27|37|PM|second|evening|| +80858|AAAAAAAALNLDBAAA|80858|22|27|38|PM|second|evening|| +80859|AAAAAAAAMNLDBAAA|80859|22|27|39|PM|second|evening|| +80860|AAAAAAAANNLDBAAA|80860|22|27|40|PM|second|evening|| +80861|AAAAAAAAONLDBAAA|80861|22|27|41|PM|second|evening|| +80862|AAAAAAAAPNLDBAAA|80862|22|27|42|PM|second|evening|| +80863|AAAAAAAAAOLDBAAA|80863|22|27|43|PM|second|evening|| +80864|AAAAAAAABOLDBAAA|80864|22|27|44|PM|second|evening|| +80865|AAAAAAAACOLDBAAA|80865|22|27|45|PM|second|evening|| +80866|AAAAAAAADOLDBAAA|80866|22|27|46|PM|second|evening|| +80867|AAAAAAAAEOLDBAAA|80867|22|27|47|PM|second|evening|| +80868|AAAAAAAAFOLDBAAA|80868|22|27|48|PM|second|evening|| +80869|AAAAAAAAGOLDBAAA|80869|22|27|49|PM|second|evening|| +80870|AAAAAAAAHOLDBAAA|80870|22|27|50|PM|second|evening|| +80871|AAAAAAAAIOLDBAAA|80871|22|27|51|PM|second|evening|| +80872|AAAAAAAAJOLDBAAA|80872|22|27|52|PM|second|evening|| +80873|AAAAAAAAKOLDBAAA|80873|22|27|53|PM|second|evening|| +80874|AAAAAAAALOLDBAAA|80874|22|27|54|PM|second|evening|| +80875|AAAAAAAAMOLDBAAA|80875|22|27|55|PM|second|evening|| +80876|AAAAAAAANOLDBAAA|80876|22|27|56|PM|second|evening|| +80877|AAAAAAAAOOLDBAAA|80877|22|27|57|PM|second|evening|| +80878|AAAAAAAAPOLDBAAA|80878|22|27|58|PM|second|evening|| +80879|AAAAAAAAAPLDBAAA|80879|22|27|59|PM|second|evening|| +80880|AAAAAAAABPLDBAAA|80880|22|28|0|PM|second|evening|| +80881|AAAAAAAACPLDBAAA|80881|22|28|1|PM|second|evening|| +80882|AAAAAAAADPLDBAAA|80882|22|28|2|PM|second|evening|| +80883|AAAAAAAAEPLDBAAA|80883|22|28|3|PM|second|evening|| +80884|AAAAAAAAFPLDBAAA|80884|22|28|4|PM|second|evening|| +80885|AAAAAAAAGPLDBAAA|80885|22|28|5|PM|second|evening|| +80886|AAAAAAAAHPLDBAAA|80886|22|28|6|PM|second|evening|| +80887|AAAAAAAAIPLDBAAA|80887|22|28|7|PM|second|evening|| +80888|AAAAAAAAJPLDBAAA|80888|22|28|8|PM|second|evening|| +80889|AAAAAAAAKPLDBAAA|80889|22|28|9|PM|second|evening|| +80890|AAAAAAAALPLDBAAA|80890|22|28|10|PM|second|evening|| +80891|AAAAAAAAMPLDBAAA|80891|22|28|11|PM|second|evening|| +80892|AAAAAAAANPLDBAAA|80892|22|28|12|PM|second|evening|| +80893|AAAAAAAAOPLDBAAA|80893|22|28|13|PM|second|evening|| +80894|AAAAAAAAPPLDBAAA|80894|22|28|14|PM|second|evening|| +80895|AAAAAAAAAAMDBAAA|80895|22|28|15|PM|second|evening|| +80896|AAAAAAAABAMDBAAA|80896|22|28|16|PM|second|evening|| +80897|AAAAAAAACAMDBAAA|80897|22|28|17|PM|second|evening|| +80898|AAAAAAAADAMDBAAA|80898|22|28|18|PM|second|evening|| +80899|AAAAAAAAEAMDBAAA|80899|22|28|19|PM|second|evening|| +80900|AAAAAAAAFAMDBAAA|80900|22|28|20|PM|second|evening|| +80901|AAAAAAAAGAMDBAAA|80901|22|28|21|PM|second|evening|| +80902|AAAAAAAAHAMDBAAA|80902|22|28|22|PM|second|evening|| +80903|AAAAAAAAIAMDBAAA|80903|22|28|23|PM|second|evening|| +80904|AAAAAAAAJAMDBAAA|80904|22|28|24|PM|second|evening|| +80905|AAAAAAAAKAMDBAAA|80905|22|28|25|PM|second|evening|| +80906|AAAAAAAALAMDBAAA|80906|22|28|26|PM|second|evening|| +80907|AAAAAAAAMAMDBAAA|80907|22|28|27|PM|second|evening|| +80908|AAAAAAAANAMDBAAA|80908|22|28|28|PM|second|evening|| +80909|AAAAAAAAOAMDBAAA|80909|22|28|29|PM|second|evening|| +80910|AAAAAAAAPAMDBAAA|80910|22|28|30|PM|second|evening|| +80911|AAAAAAAAABMDBAAA|80911|22|28|31|PM|second|evening|| +80912|AAAAAAAABBMDBAAA|80912|22|28|32|PM|second|evening|| +80913|AAAAAAAACBMDBAAA|80913|22|28|33|PM|second|evening|| +80914|AAAAAAAADBMDBAAA|80914|22|28|34|PM|second|evening|| +80915|AAAAAAAAEBMDBAAA|80915|22|28|35|PM|second|evening|| +80916|AAAAAAAAFBMDBAAA|80916|22|28|36|PM|second|evening|| +80917|AAAAAAAAGBMDBAAA|80917|22|28|37|PM|second|evening|| +80918|AAAAAAAAHBMDBAAA|80918|22|28|38|PM|second|evening|| +80919|AAAAAAAAIBMDBAAA|80919|22|28|39|PM|second|evening|| +80920|AAAAAAAAJBMDBAAA|80920|22|28|40|PM|second|evening|| +80921|AAAAAAAAKBMDBAAA|80921|22|28|41|PM|second|evening|| +80922|AAAAAAAALBMDBAAA|80922|22|28|42|PM|second|evening|| +80923|AAAAAAAAMBMDBAAA|80923|22|28|43|PM|second|evening|| +80924|AAAAAAAANBMDBAAA|80924|22|28|44|PM|second|evening|| +80925|AAAAAAAAOBMDBAAA|80925|22|28|45|PM|second|evening|| +80926|AAAAAAAAPBMDBAAA|80926|22|28|46|PM|second|evening|| +80927|AAAAAAAAACMDBAAA|80927|22|28|47|PM|second|evening|| +80928|AAAAAAAABCMDBAAA|80928|22|28|48|PM|second|evening|| +80929|AAAAAAAACCMDBAAA|80929|22|28|49|PM|second|evening|| +80930|AAAAAAAADCMDBAAA|80930|22|28|50|PM|second|evening|| +80931|AAAAAAAAECMDBAAA|80931|22|28|51|PM|second|evening|| +80932|AAAAAAAAFCMDBAAA|80932|22|28|52|PM|second|evening|| +80933|AAAAAAAAGCMDBAAA|80933|22|28|53|PM|second|evening|| +80934|AAAAAAAAHCMDBAAA|80934|22|28|54|PM|second|evening|| +80935|AAAAAAAAICMDBAAA|80935|22|28|55|PM|second|evening|| +80936|AAAAAAAAJCMDBAAA|80936|22|28|56|PM|second|evening|| +80937|AAAAAAAAKCMDBAAA|80937|22|28|57|PM|second|evening|| +80938|AAAAAAAALCMDBAAA|80938|22|28|58|PM|second|evening|| +80939|AAAAAAAAMCMDBAAA|80939|22|28|59|PM|second|evening|| +80940|AAAAAAAANCMDBAAA|80940|22|29|0|PM|second|evening|| +80941|AAAAAAAAOCMDBAAA|80941|22|29|1|PM|second|evening|| +80942|AAAAAAAAPCMDBAAA|80942|22|29|2|PM|second|evening|| +80943|AAAAAAAAADMDBAAA|80943|22|29|3|PM|second|evening|| +80944|AAAAAAAABDMDBAAA|80944|22|29|4|PM|second|evening|| +80945|AAAAAAAACDMDBAAA|80945|22|29|5|PM|second|evening|| +80946|AAAAAAAADDMDBAAA|80946|22|29|6|PM|second|evening|| +80947|AAAAAAAAEDMDBAAA|80947|22|29|7|PM|second|evening|| +80948|AAAAAAAAFDMDBAAA|80948|22|29|8|PM|second|evening|| +80949|AAAAAAAAGDMDBAAA|80949|22|29|9|PM|second|evening|| +80950|AAAAAAAAHDMDBAAA|80950|22|29|10|PM|second|evening|| +80951|AAAAAAAAIDMDBAAA|80951|22|29|11|PM|second|evening|| +80952|AAAAAAAAJDMDBAAA|80952|22|29|12|PM|second|evening|| +80953|AAAAAAAAKDMDBAAA|80953|22|29|13|PM|second|evening|| +80954|AAAAAAAALDMDBAAA|80954|22|29|14|PM|second|evening|| +80955|AAAAAAAAMDMDBAAA|80955|22|29|15|PM|second|evening|| +80956|AAAAAAAANDMDBAAA|80956|22|29|16|PM|second|evening|| +80957|AAAAAAAAODMDBAAA|80957|22|29|17|PM|second|evening|| +80958|AAAAAAAAPDMDBAAA|80958|22|29|18|PM|second|evening|| +80959|AAAAAAAAAEMDBAAA|80959|22|29|19|PM|second|evening|| +80960|AAAAAAAABEMDBAAA|80960|22|29|20|PM|second|evening|| +80961|AAAAAAAACEMDBAAA|80961|22|29|21|PM|second|evening|| +80962|AAAAAAAADEMDBAAA|80962|22|29|22|PM|second|evening|| +80963|AAAAAAAAEEMDBAAA|80963|22|29|23|PM|second|evening|| +80964|AAAAAAAAFEMDBAAA|80964|22|29|24|PM|second|evening|| +80965|AAAAAAAAGEMDBAAA|80965|22|29|25|PM|second|evening|| +80966|AAAAAAAAHEMDBAAA|80966|22|29|26|PM|second|evening|| +80967|AAAAAAAAIEMDBAAA|80967|22|29|27|PM|second|evening|| +80968|AAAAAAAAJEMDBAAA|80968|22|29|28|PM|second|evening|| +80969|AAAAAAAAKEMDBAAA|80969|22|29|29|PM|second|evening|| +80970|AAAAAAAALEMDBAAA|80970|22|29|30|PM|second|evening|| +80971|AAAAAAAAMEMDBAAA|80971|22|29|31|PM|second|evening|| +80972|AAAAAAAANEMDBAAA|80972|22|29|32|PM|second|evening|| +80973|AAAAAAAAOEMDBAAA|80973|22|29|33|PM|second|evening|| +80974|AAAAAAAAPEMDBAAA|80974|22|29|34|PM|second|evening|| +80975|AAAAAAAAAFMDBAAA|80975|22|29|35|PM|second|evening|| +80976|AAAAAAAABFMDBAAA|80976|22|29|36|PM|second|evening|| +80977|AAAAAAAACFMDBAAA|80977|22|29|37|PM|second|evening|| +80978|AAAAAAAADFMDBAAA|80978|22|29|38|PM|second|evening|| +80979|AAAAAAAAEFMDBAAA|80979|22|29|39|PM|second|evening|| +80980|AAAAAAAAFFMDBAAA|80980|22|29|40|PM|second|evening|| +80981|AAAAAAAAGFMDBAAA|80981|22|29|41|PM|second|evening|| +80982|AAAAAAAAHFMDBAAA|80982|22|29|42|PM|second|evening|| +80983|AAAAAAAAIFMDBAAA|80983|22|29|43|PM|second|evening|| +80984|AAAAAAAAJFMDBAAA|80984|22|29|44|PM|second|evening|| +80985|AAAAAAAAKFMDBAAA|80985|22|29|45|PM|second|evening|| +80986|AAAAAAAALFMDBAAA|80986|22|29|46|PM|second|evening|| +80987|AAAAAAAAMFMDBAAA|80987|22|29|47|PM|second|evening|| +80988|AAAAAAAANFMDBAAA|80988|22|29|48|PM|second|evening|| +80989|AAAAAAAAOFMDBAAA|80989|22|29|49|PM|second|evening|| +80990|AAAAAAAAPFMDBAAA|80990|22|29|50|PM|second|evening|| +80991|AAAAAAAAAGMDBAAA|80991|22|29|51|PM|second|evening|| +80992|AAAAAAAABGMDBAAA|80992|22|29|52|PM|second|evening|| +80993|AAAAAAAACGMDBAAA|80993|22|29|53|PM|second|evening|| +80994|AAAAAAAADGMDBAAA|80994|22|29|54|PM|second|evening|| +80995|AAAAAAAAEGMDBAAA|80995|22|29|55|PM|second|evening|| +80996|AAAAAAAAFGMDBAAA|80996|22|29|56|PM|second|evening|| +80997|AAAAAAAAGGMDBAAA|80997|22|29|57|PM|second|evening|| +80998|AAAAAAAAHGMDBAAA|80998|22|29|58|PM|second|evening|| +80999|AAAAAAAAIGMDBAAA|80999|22|29|59|PM|second|evening|| +81000|AAAAAAAAJGMDBAAA|81000|22|30|0|PM|second|evening|| +81001|AAAAAAAAKGMDBAAA|81001|22|30|1|PM|second|evening|| +81002|AAAAAAAALGMDBAAA|81002|22|30|2|PM|second|evening|| +81003|AAAAAAAAMGMDBAAA|81003|22|30|3|PM|second|evening|| +81004|AAAAAAAANGMDBAAA|81004|22|30|4|PM|second|evening|| +81005|AAAAAAAAOGMDBAAA|81005|22|30|5|PM|second|evening|| +81006|AAAAAAAAPGMDBAAA|81006|22|30|6|PM|second|evening|| +81007|AAAAAAAAAHMDBAAA|81007|22|30|7|PM|second|evening|| +81008|AAAAAAAABHMDBAAA|81008|22|30|8|PM|second|evening|| +81009|AAAAAAAACHMDBAAA|81009|22|30|9|PM|second|evening|| +81010|AAAAAAAADHMDBAAA|81010|22|30|10|PM|second|evening|| +81011|AAAAAAAAEHMDBAAA|81011|22|30|11|PM|second|evening|| +81012|AAAAAAAAFHMDBAAA|81012|22|30|12|PM|second|evening|| +81013|AAAAAAAAGHMDBAAA|81013|22|30|13|PM|second|evening|| +81014|AAAAAAAAHHMDBAAA|81014|22|30|14|PM|second|evening|| +81015|AAAAAAAAIHMDBAAA|81015|22|30|15|PM|second|evening|| +81016|AAAAAAAAJHMDBAAA|81016|22|30|16|PM|second|evening|| +81017|AAAAAAAAKHMDBAAA|81017|22|30|17|PM|second|evening|| +81018|AAAAAAAALHMDBAAA|81018|22|30|18|PM|second|evening|| +81019|AAAAAAAAMHMDBAAA|81019|22|30|19|PM|second|evening|| +81020|AAAAAAAANHMDBAAA|81020|22|30|20|PM|second|evening|| +81021|AAAAAAAAOHMDBAAA|81021|22|30|21|PM|second|evening|| +81022|AAAAAAAAPHMDBAAA|81022|22|30|22|PM|second|evening|| +81023|AAAAAAAAAIMDBAAA|81023|22|30|23|PM|second|evening|| +81024|AAAAAAAABIMDBAAA|81024|22|30|24|PM|second|evening|| +81025|AAAAAAAACIMDBAAA|81025|22|30|25|PM|second|evening|| +81026|AAAAAAAADIMDBAAA|81026|22|30|26|PM|second|evening|| +81027|AAAAAAAAEIMDBAAA|81027|22|30|27|PM|second|evening|| +81028|AAAAAAAAFIMDBAAA|81028|22|30|28|PM|second|evening|| +81029|AAAAAAAAGIMDBAAA|81029|22|30|29|PM|second|evening|| +81030|AAAAAAAAHIMDBAAA|81030|22|30|30|PM|second|evening|| +81031|AAAAAAAAIIMDBAAA|81031|22|30|31|PM|second|evening|| +81032|AAAAAAAAJIMDBAAA|81032|22|30|32|PM|second|evening|| +81033|AAAAAAAAKIMDBAAA|81033|22|30|33|PM|second|evening|| +81034|AAAAAAAALIMDBAAA|81034|22|30|34|PM|second|evening|| +81035|AAAAAAAAMIMDBAAA|81035|22|30|35|PM|second|evening|| +81036|AAAAAAAANIMDBAAA|81036|22|30|36|PM|second|evening|| +81037|AAAAAAAAOIMDBAAA|81037|22|30|37|PM|second|evening|| +81038|AAAAAAAAPIMDBAAA|81038|22|30|38|PM|second|evening|| +81039|AAAAAAAAAJMDBAAA|81039|22|30|39|PM|second|evening|| +81040|AAAAAAAABJMDBAAA|81040|22|30|40|PM|second|evening|| +81041|AAAAAAAACJMDBAAA|81041|22|30|41|PM|second|evening|| +81042|AAAAAAAADJMDBAAA|81042|22|30|42|PM|second|evening|| +81043|AAAAAAAAEJMDBAAA|81043|22|30|43|PM|second|evening|| +81044|AAAAAAAAFJMDBAAA|81044|22|30|44|PM|second|evening|| +81045|AAAAAAAAGJMDBAAA|81045|22|30|45|PM|second|evening|| +81046|AAAAAAAAHJMDBAAA|81046|22|30|46|PM|second|evening|| +81047|AAAAAAAAIJMDBAAA|81047|22|30|47|PM|second|evening|| +81048|AAAAAAAAJJMDBAAA|81048|22|30|48|PM|second|evening|| +81049|AAAAAAAAKJMDBAAA|81049|22|30|49|PM|second|evening|| +81050|AAAAAAAALJMDBAAA|81050|22|30|50|PM|second|evening|| +81051|AAAAAAAAMJMDBAAA|81051|22|30|51|PM|second|evening|| +81052|AAAAAAAANJMDBAAA|81052|22|30|52|PM|second|evening|| +81053|AAAAAAAAOJMDBAAA|81053|22|30|53|PM|second|evening|| +81054|AAAAAAAAPJMDBAAA|81054|22|30|54|PM|second|evening|| +81055|AAAAAAAAAKMDBAAA|81055|22|30|55|PM|second|evening|| +81056|AAAAAAAABKMDBAAA|81056|22|30|56|PM|second|evening|| +81057|AAAAAAAACKMDBAAA|81057|22|30|57|PM|second|evening|| +81058|AAAAAAAADKMDBAAA|81058|22|30|58|PM|second|evening|| +81059|AAAAAAAAEKMDBAAA|81059|22|30|59|PM|second|evening|| +81060|AAAAAAAAFKMDBAAA|81060|22|31|0|PM|second|evening|| +81061|AAAAAAAAGKMDBAAA|81061|22|31|1|PM|second|evening|| +81062|AAAAAAAAHKMDBAAA|81062|22|31|2|PM|second|evening|| +81063|AAAAAAAAIKMDBAAA|81063|22|31|3|PM|second|evening|| +81064|AAAAAAAAJKMDBAAA|81064|22|31|4|PM|second|evening|| +81065|AAAAAAAAKKMDBAAA|81065|22|31|5|PM|second|evening|| +81066|AAAAAAAALKMDBAAA|81066|22|31|6|PM|second|evening|| +81067|AAAAAAAAMKMDBAAA|81067|22|31|7|PM|second|evening|| +81068|AAAAAAAANKMDBAAA|81068|22|31|8|PM|second|evening|| +81069|AAAAAAAAOKMDBAAA|81069|22|31|9|PM|second|evening|| +81070|AAAAAAAAPKMDBAAA|81070|22|31|10|PM|second|evening|| +81071|AAAAAAAAALMDBAAA|81071|22|31|11|PM|second|evening|| +81072|AAAAAAAABLMDBAAA|81072|22|31|12|PM|second|evening|| +81073|AAAAAAAACLMDBAAA|81073|22|31|13|PM|second|evening|| +81074|AAAAAAAADLMDBAAA|81074|22|31|14|PM|second|evening|| +81075|AAAAAAAAELMDBAAA|81075|22|31|15|PM|second|evening|| +81076|AAAAAAAAFLMDBAAA|81076|22|31|16|PM|second|evening|| +81077|AAAAAAAAGLMDBAAA|81077|22|31|17|PM|second|evening|| +81078|AAAAAAAAHLMDBAAA|81078|22|31|18|PM|second|evening|| +81079|AAAAAAAAILMDBAAA|81079|22|31|19|PM|second|evening|| +81080|AAAAAAAAJLMDBAAA|81080|22|31|20|PM|second|evening|| +81081|AAAAAAAAKLMDBAAA|81081|22|31|21|PM|second|evening|| +81082|AAAAAAAALLMDBAAA|81082|22|31|22|PM|second|evening|| +81083|AAAAAAAAMLMDBAAA|81083|22|31|23|PM|second|evening|| +81084|AAAAAAAANLMDBAAA|81084|22|31|24|PM|second|evening|| +81085|AAAAAAAAOLMDBAAA|81085|22|31|25|PM|second|evening|| +81086|AAAAAAAAPLMDBAAA|81086|22|31|26|PM|second|evening|| +81087|AAAAAAAAAMMDBAAA|81087|22|31|27|PM|second|evening|| +81088|AAAAAAAABMMDBAAA|81088|22|31|28|PM|second|evening|| +81089|AAAAAAAACMMDBAAA|81089|22|31|29|PM|second|evening|| +81090|AAAAAAAADMMDBAAA|81090|22|31|30|PM|second|evening|| +81091|AAAAAAAAEMMDBAAA|81091|22|31|31|PM|second|evening|| +81092|AAAAAAAAFMMDBAAA|81092|22|31|32|PM|second|evening|| +81093|AAAAAAAAGMMDBAAA|81093|22|31|33|PM|second|evening|| +81094|AAAAAAAAHMMDBAAA|81094|22|31|34|PM|second|evening|| +81095|AAAAAAAAIMMDBAAA|81095|22|31|35|PM|second|evening|| +81096|AAAAAAAAJMMDBAAA|81096|22|31|36|PM|second|evening|| +81097|AAAAAAAAKMMDBAAA|81097|22|31|37|PM|second|evening|| +81098|AAAAAAAALMMDBAAA|81098|22|31|38|PM|second|evening|| +81099|AAAAAAAAMMMDBAAA|81099|22|31|39|PM|second|evening|| +81100|AAAAAAAANMMDBAAA|81100|22|31|40|PM|second|evening|| +81101|AAAAAAAAOMMDBAAA|81101|22|31|41|PM|second|evening|| +81102|AAAAAAAAPMMDBAAA|81102|22|31|42|PM|second|evening|| +81103|AAAAAAAAANMDBAAA|81103|22|31|43|PM|second|evening|| +81104|AAAAAAAABNMDBAAA|81104|22|31|44|PM|second|evening|| +81105|AAAAAAAACNMDBAAA|81105|22|31|45|PM|second|evening|| +81106|AAAAAAAADNMDBAAA|81106|22|31|46|PM|second|evening|| +81107|AAAAAAAAENMDBAAA|81107|22|31|47|PM|second|evening|| +81108|AAAAAAAAFNMDBAAA|81108|22|31|48|PM|second|evening|| +81109|AAAAAAAAGNMDBAAA|81109|22|31|49|PM|second|evening|| +81110|AAAAAAAAHNMDBAAA|81110|22|31|50|PM|second|evening|| +81111|AAAAAAAAINMDBAAA|81111|22|31|51|PM|second|evening|| +81112|AAAAAAAAJNMDBAAA|81112|22|31|52|PM|second|evening|| +81113|AAAAAAAAKNMDBAAA|81113|22|31|53|PM|second|evening|| +81114|AAAAAAAALNMDBAAA|81114|22|31|54|PM|second|evening|| +81115|AAAAAAAAMNMDBAAA|81115|22|31|55|PM|second|evening|| +81116|AAAAAAAANNMDBAAA|81116|22|31|56|PM|second|evening|| +81117|AAAAAAAAONMDBAAA|81117|22|31|57|PM|second|evening|| +81118|AAAAAAAAPNMDBAAA|81118|22|31|58|PM|second|evening|| +81119|AAAAAAAAAOMDBAAA|81119|22|31|59|PM|second|evening|| +81120|AAAAAAAABOMDBAAA|81120|22|32|0|PM|second|evening|| +81121|AAAAAAAACOMDBAAA|81121|22|32|1|PM|second|evening|| +81122|AAAAAAAADOMDBAAA|81122|22|32|2|PM|second|evening|| +81123|AAAAAAAAEOMDBAAA|81123|22|32|3|PM|second|evening|| +81124|AAAAAAAAFOMDBAAA|81124|22|32|4|PM|second|evening|| +81125|AAAAAAAAGOMDBAAA|81125|22|32|5|PM|second|evening|| +81126|AAAAAAAAHOMDBAAA|81126|22|32|6|PM|second|evening|| +81127|AAAAAAAAIOMDBAAA|81127|22|32|7|PM|second|evening|| +81128|AAAAAAAAJOMDBAAA|81128|22|32|8|PM|second|evening|| +81129|AAAAAAAAKOMDBAAA|81129|22|32|9|PM|second|evening|| +81130|AAAAAAAALOMDBAAA|81130|22|32|10|PM|second|evening|| +81131|AAAAAAAAMOMDBAAA|81131|22|32|11|PM|second|evening|| +81132|AAAAAAAANOMDBAAA|81132|22|32|12|PM|second|evening|| +81133|AAAAAAAAOOMDBAAA|81133|22|32|13|PM|second|evening|| +81134|AAAAAAAAPOMDBAAA|81134|22|32|14|PM|second|evening|| +81135|AAAAAAAAAPMDBAAA|81135|22|32|15|PM|second|evening|| +81136|AAAAAAAABPMDBAAA|81136|22|32|16|PM|second|evening|| +81137|AAAAAAAACPMDBAAA|81137|22|32|17|PM|second|evening|| +81138|AAAAAAAADPMDBAAA|81138|22|32|18|PM|second|evening|| +81139|AAAAAAAAEPMDBAAA|81139|22|32|19|PM|second|evening|| +81140|AAAAAAAAFPMDBAAA|81140|22|32|20|PM|second|evening|| +81141|AAAAAAAAGPMDBAAA|81141|22|32|21|PM|second|evening|| +81142|AAAAAAAAHPMDBAAA|81142|22|32|22|PM|second|evening|| +81143|AAAAAAAAIPMDBAAA|81143|22|32|23|PM|second|evening|| +81144|AAAAAAAAJPMDBAAA|81144|22|32|24|PM|second|evening|| +81145|AAAAAAAAKPMDBAAA|81145|22|32|25|PM|second|evening|| +81146|AAAAAAAALPMDBAAA|81146|22|32|26|PM|second|evening|| +81147|AAAAAAAAMPMDBAAA|81147|22|32|27|PM|second|evening|| +81148|AAAAAAAANPMDBAAA|81148|22|32|28|PM|second|evening|| +81149|AAAAAAAAOPMDBAAA|81149|22|32|29|PM|second|evening|| +81150|AAAAAAAAPPMDBAAA|81150|22|32|30|PM|second|evening|| +81151|AAAAAAAAAANDBAAA|81151|22|32|31|PM|second|evening|| +81152|AAAAAAAABANDBAAA|81152|22|32|32|PM|second|evening|| +81153|AAAAAAAACANDBAAA|81153|22|32|33|PM|second|evening|| +81154|AAAAAAAADANDBAAA|81154|22|32|34|PM|second|evening|| +81155|AAAAAAAAEANDBAAA|81155|22|32|35|PM|second|evening|| +81156|AAAAAAAAFANDBAAA|81156|22|32|36|PM|second|evening|| +81157|AAAAAAAAGANDBAAA|81157|22|32|37|PM|second|evening|| +81158|AAAAAAAAHANDBAAA|81158|22|32|38|PM|second|evening|| +81159|AAAAAAAAIANDBAAA|81159|22|32|39|PM|second|evening|| +81160|AAAAAAAAJANDBAAA|81160|22|32|40|PM|second|evening|| +81161|AAAAAAAAKANDBAAA|81161|22|32|41|PM|second|evening|| +81162|AAAAAAAALANDBAAA|81162|22|32|42|PM|second|evening|| +81163|AAAAAAAAMANDBAAA|81163|22|32|43|PM|second|evening|| +81164|AAAAAAAANANDBAAA|81164|22|32|44|PM|second|evening|| +81165|AAAAAAAAOANDBAAA|81165|22|32|45|PM|second|evening|| +81166|AAAAAAAAPANDBAAA|81166|22|32|46|PM|second|evening|| +81167|AAAAAAAAABNDBAAA|81167|22|32|47|PM|second|evening|| +81168|AAAAAAAABBNDBAAA|81168|22|32|48|PM|second|evening|| +81169|AAAAAAAACBNDBAAA|81169|22|32|49|PM|second|evening|| +81170|AAAAAAAADBNDBAAA|81170|22|32|50|PM|second|evening|| +81171|AAAAAAAAEBNDBAAA|81171|22|32|51|PM|second|evening|| +81172|AAAAAAAAFBNDBAAA|81172|22|32|52|PM|second|evening|| +81173|AAAAAAAAGBNDBAAA|81173|22|32|53|PM|second|evening|| +81174|AAAAAAAAHBNDBAAA|81174|22|32|54|PM|second|evening|| +81175|AAAAAAAAIBNDBAAA|81175|22|32|55|PM|second|evening|| +81176|AAAAAAAAJBNDBAAA|81176|22|32|56|PM|second|evening|| +81177|AAAAAAAAKBNDBAAA|81177|22|32|57|PM|second|evening|| +81178|AAAAAAAALBNDBAAA|81178|22|32|58|PM|second|evening|| +81179|AAAAAAAAMBNDBAAA|81179|22|32|59|PM|second|evening|| +81180|AAAAAAAANBNDBAAA|81180|22|33|0|PM|second|evening|| +81181|AAAAAAAAOBNDBAAA|81181|22|33|1|PM|second|evening|| +81182|AAAAAAAAPBNDBAAA|81182|22|33|2|PM|second|evening|| +81183|AAAAAAAAACNDBAAA|81183|22|33|3|PM|second|evening|| +81184|AAAAAAAABCNDBAAA|81184|22|33|4|PM|second|evening|| +81185|AAAAAAAACCNDBAAA|81185|22|33|5|PM|second|evening|| +81186|AAAAAAAADCNDBAAA|81186|22|33|6|PM|second|evening|| +81187|AAAAAAAAECNDBAAA|81187|22|33|7|PM|second|evening|| +81188|AAAAAAAAFCNDBAAA|81188|22|33|8|PM|second|evening|| +81189|AAAAAAAAGCNDBAAA|81189|22|33|9|PM|second|evening|| +81190|AAAAAAAAHCNDBAAA|81190|22|33|10|PM|second|evening|| +81191|AAAAAAAAICNDBAAA|81191|22|33|11|PM|second|evening|| +81192|AAAAAAAAJCNDBAAA|81192|22|33|12|PM|second|evening|| +81193|AAAAAAAAKCNDBAAA|81193|22|33|13|PM|second|evening|| +81194|AAAAAAAALCNDBAAA|81194|22|33|14|PM|second|evening|| +81195|AAAAAAAAMCNDBAAA|81195|22|33|15|PM|second|evening|| +81196|AAAAAAAANCNDBAAA|81196|22|33|16|PM|second|evening|| +81197|AAAAAAAAOCNDBAAA|81197|22|33|17|PM|second|evening|| +81198|AAAAAAAAPCNDBAAA|81198|22|33|18|PM|second|evening|| +81199|AAAAAAAAADNDBAAA|81199|22|33|19|PM|second|evening|| +81200|AAAAAAAABDNDBAAA|81200|22|33|20|PM|second|evening|| +81201|AAAAAAAACDNDBAAA|81201|22|33|21|PM|second|evening|| +81202|AAAAAAAADDNDBAAA|81202|22|33|22|PM|second|evening|| +81203|AAAAAAAAEDNDBAAA|81203|22|33|23|PM|second|evening|| +81204|AAAAAAAAFDNDBAAA|81204|22|33|24|PM|second|evening|| +81205|AAAAAAAAGDNDBAAA|81205|22|33|25|PM|second|evening|| +81206|AAAAAAAAHDNDBAAA|81206|22|33|26|PM|second|evening|| +81207|AAAAAAAAIDNDBAAA|81207|22|33|27|PM|second|evening|| +81208|AAAAAAAAJDNDBAAA|81208|22|33|28|PM|second|evening|| +81209|AAAAAAAAKDNDBAAA|81209|22|33|29|PM|second|evening|| +81210|AAAAAAAALDNDBAAA|81210|22|33|30|PM|second|evening|| +81211|AAAAAAAAMDNDBAAA|81211|22|33|31|PM|second|evening|| +81212|AAAAAAAANDNDBAAA|81212|22|33|32|PM|second|evening|| +81213|AAAAAAAAODNDBAAA|81213|22|33|33|PM|second|evening|| +81214|AAAAAAAAPDNDBAAA|81214|22|33|34|PM|second|evening|| +81215|AAAAAAAAAENDBAAA|81215|22|33|35|PM|second|evening|| +81216|AAAAAAAABENDBAAA|81216|22|33|36|PM|second|evening|| +81217|AAAAAAAACENDBAAA|81217|22|33|37|PM|second|evening|| +81218|AAAAAAAADENDBAAA|81218|22|33|38|PM|second|evening|| +81219|AAAAAAAAEENDBAAA|81219|22|33|39|PM|second|evening|| +81220|AAAAAAAAFENDBAAA|81220|22|33|40|PM|second|evening|| +81221|AAAAAAAAGENDBAAA|81221|22|33|41|PM|second|evening|| +81222|AAAAAAAAHENDBAAA|81222|22|33|42|PM|second|evening|| +81223|AAAAAAAAIENDBAAA|81223|22|33|43|PM|second|evening|| +81224|AAAAAAAAJENDBAAA|81224|22|33|44|PM|second|evening|| +81225|AAAAAAAAKENDBAAA|81225|22|33|45|PM|second|evening|| +81226|AAAAAAAALENDBAAA|81226|22|33|46|PM|second|evening|| +81227|AAAAAAAAMENDBAAA|81227|22|33|47|PM|second|evening|| +81228|AAAAAAAANENDBAAA|81228|22|33|48|PM|second|evening|| +81229|AAAAAAAAOENDBAAA|81229|22|33|49|PM|second|evening|| +81230|AAAAAAAAPENDBAAA|81230|22|33|50|PM|second|evening|| +81231|AAAAAAAAAFNDBAAA|81231|22|33|51|PM|second|evening|| +81232|AAAAAAAABFNDBAAA|81232|22|33|52|PM|second|evening|| +81233|AAAAAAAACFNDBAAA|81233|22|33|53|PM|second|evening|| +81234|AAAAAAAADFNDBAAA|81234|22|33|54|PM|second|evening|| +81235|AAAAAAAAEFNDBAAA|81235|22|33|55|PM|second|evening|| +81236|AAAAAAAAFFNDBAAA|81236|22|33|56|PM|second|evening|| +81237|AAAAAAAAGFNDBAAA|81237|22|33|57|PM|second|evening|| +81238|AAAAAAAAHFNDBAAA|81238|22|33|58|PM|second|evening|| +81239|AAAAAAAAIFNDBAAA|81239|22|33|59|PM|second|evening|| +81240|AAAAAAAAJFNDBAAA|81240|22|34|0|PM|second|evening|| +81241|AAAAAAAAKFNDBAAA|81241|22|34|1|PM|second|evening|| +81242|AAAAAAAALFNDBAAA|81242|22|34|2|PM|second|evening|| +81243|AAAAAAAAMFNDBAAA|81243|22|34|3|PM|second|evening|| +81244|AAAAAAAANFNDBAAA|81244|22|34|4|PM|second|evening|| +81245|AAAAAAAAOFNDBAAA|81245|22|34|5|PM|second|evening|| +81246|AAAAAAAAPFNDBAAA|81246|22|34|6|PM|second|evening|| +81247|AAAAAAAAAGNDBAAA|81247|22|34|7|PM|second|evening|| +81248|AAAAAAAABGNDBAAA|81248|22|34|8|PM|second|evening|| +81249|AAAAAAAACGNDBAAA|81249|22|34|9|PM|second|evening|| +81250|AAAAAAAADGNDBAAA|81250|22|34|10|PM|second|evening|| +81251|AAAAAAAAEGNDBAAA|81251|22|34|11|PM|second|evening|| +81252|AAAAAAAAFGNDBAAA|81252|22|34|12|PM|second|evening|| +81253|AAAAAAAAGGNDBAAA|81253|22|34|13|PM|second|evening|| +81254|AAAAAAAAHGNDBAAA|81254|22|34|14|PM|second|evening|| +81255|AAAAAAAAIGNDBAAA|81255|22|34|15|PM|second|evening|| +81256|AAAAAAAAJGNDBAAA|81256|22|34|16|PM|second|evening|| +81257|AAAAAAAAKGNDBAAA|81257|22|34|17|PM|second|evening|| +81258|AAAAAAAALGNDBAAA|81258|22|34|18|PM|second|evening|| +81259|AAAAAAAAMGNDBAAA|81259|22|34|19|PM|second|evening|| +81260|AAAAAAAANGNDBAAA|81260|22|34|20|PM|second|evening|| +81261|AAAAAAAAOGNDBAAA|81261|22|34|21|PM|second|evening|| +81262|AAAAAAAAPGNDBAAA|81262|22|34|22|PM|second|evening|| +81263|AAAAAAAAAHNDBAAA|81263|22|34|23|PM|second|evening|| +81264|AAAAAAAABHNDBAAA|81264|22|34|24|PM|second|evening|| +81265|AAAAAAAACHNDBAAA|81265|22|34|25|PM|second|evening|| +81266|AAAAAAAADHNDBAAA|81266|22|34|26|PM|second|evening|| +81267|AAAAAAAAEHNDBAAA|81267|22|34|27|PM|second|evening|| +81268|AAAAAAAAFHNDBAAA|81268|22|34|28|PM|second|evening|| +81269|AAAAAAAAGHNDBAAA|81269|22|34|29|PM|second|evening|| +81270|AAAAAAAAHHNDBAAA|81270|22|34|30|PM|second|evening|| +81271|AAAAAAAAIHNDBAAA|81271|22|34|31|PM|second|evening|| +81272|AAAAAAAAJHNDBAAA|81272|22|34|32|PM|second|evening|| +81273|AAAAAAAAKHNDBAAA|81273|22|34|33|PM|second|evening|| +81274|AAAAAAAALHNDBAAA|81274|22|34|34|PM|second|evening|| +81275|AAAAAAAAMHNDBAAA|81275|22|34|35|PM|second|evening|| +81276|AAAAAAAANHNDBAAA|81276|22|34|36|PM|second|evening|| +81277|AAAAAAAAOHNDBAAA|81277|22|34|37|PM|second|evening|| +81278|AAAAAAAAPHNDBAAA|81278|22|34|38|PM|second|evening|| +81279|AAAAAAAAAINDBAAA|81279|22|34|39|PM|second|evening|| +81280|AAAAAAAABINDBAAA|81280|22|34|40|PM|second|evening|| +81281|AAAAAAAACINDBAAA|81281|22|34|41|PM|second|evening|| +81282|AAAAAAAADINDBAAA|81282|22|34|42|PM|second|evening|| +81283|AAAAAAAAEINDBAAA|81283|22|34|43|PM|second|evening|| +81284|AAAAAAAAFINDBAAA|81284|22|34|44|PM|second|evening|| +81285|AAAAAAAAGINDBAAA|81285|22|34|45|PM|second|evening|| +81286|AAAAAAAAHINDBAAA|81286|22|34|46|PM|second|evening|| +81287|AAAAAAAAIINDBAAA|81287|22|34|47|PM|second|evening|| +81288|AAAAAAAAJINDBAAA|81288|22|34|48|PM|second|evening|| +81289|AAAAAAAAKINDBAAA|81289|22|34|49|PM|second|evening|| +81290|AAAAAAAALINDBAAA|81290|22|34|50|PM|second|evening|| +81291|AAAAAAAAMINDBAAA|81291|22|34|51|PM|second|evening|| +81292|AAAAAAAANINDBAAA|81292|22|34|52|PM|second|evening|| +81293|AAAAAAAAOINDBAAA|81293|22|34|53|PM|second|evening|| +81294|AAAAAAAAPINDBAAA|81294|22|34|54|PM|second|evening|| +81295|AAAAAAAAAJNDBAAA|81295|22|34|55|PM|second|evening|| +81296|AAAAAAAABJNDBAAA|81296|22|34|56|PM|second|evening|| +81297|AAAAAAAACJNDBAAA|81297|22|34|57|PM|second|evening|| +81298|AAAAAAAADJNDBAAA|81298|22|34|58|PM|second|evening|| +81299|AAAAAAAAEJNDBAAA|81299|22|34|59|PM|second|evening|| +81300|AAAAAAAAFJNDBAAA|81300|22|35|0|PM|second|evening|| +81301|AAAAAAAAGJNDBAAA|81301|22|35|1|PM|second|evening|| +81302|AAAAAAAAHJNDBAAA|81302|22|35|2|PM|second|evening|| +81303|AAAAAAAAIJNDBAAA|81303|22|35|3|PM|second|evening|| +81304|AAAAAAAAJJNDBAAA|81304|22|35|4|PM|second|evening|| +81305|AAAAAAAAKJNDBAAA|81305|22|35|5|PM|second|evening|| +81306|AAAAAAAALJNDBAAA|81306|22|35|6|PM|second|evening|| +81307|AAAAAAAAMJNDBAAA|81307|22|35|7|PM|second|evening|| +81308|AAAAAAAANJNDBAAA|81308|22|35|8|PM|second|evening|| +81309|AAAAAAAAOJNDBAAA|81309|22|35|9|PM|second|evening|| +81310|AAAAAAAAPJNDBAAA|81310|22|35|10|PM|second|evening|| +81311|AAAAAAAAAKNDBAAA|81311|22|35|11|PM|second|evening|| +81312|AAAAAAAABKNDBAAA|81312|22|35|12|PM|second|evening|| +81313|AAAAAAAACKNDBAAA|81313|22|35|13|PM|second|evening|| +81314|AAAAAAAADKNDBAAA|81314|22|35|14|PM|second|evening|| +81315|AAAAAAAAEKNDBAAA|81315|22|35|15|PM|second|evening|| +81316|AAAAAAAAFKNDBAAA|81316|22|35|16|PM|second|evening|| +81317|AAAAAAAAGKNDBAAA|81317|22|35|17|PM|second|evening|| +81318|AAAAAAAAHKNDBAAA|81318|22|35|18|PM|second|evening|| +81319|AAAAAAAAIKNDBAAA|81319|22|35|19|PM|second|evening|| +81320|AAAAAAAAJKNDBAAA|81320|22|35|20|PM|second|evening|| +81321|AAAAAAAAKKNDBAAA|81321|22|35|21|PM|second|evening|| +81322|AAAAAAAALKNDBAAA|81322|22|35|22|PM|second|evening|| +81323|AAAAAAAAMKNDBAAA|81323|22|35|23|PM|second|evening|| +81324|AAAAAAAANKNDBAAA|81324|22|35|24|PM|second|evening|| +81325|AAAAAAAAOKNDBAAA|81325|22|35|25|PM|second|evening|| +81326|AAAAAAAAPKNDBAAA|81326|22|35|26|PM|second|evening|| +81327|AAAAAAAAALNDBAAA|81327|22|35|27|PM|second|evening|| +81328|AAAAAAAABLNDBAAA|81328|22|35|28|PM|second|evening|| +81329|AAAAAAAACLNDBAAA|81329|22|35|29|PM|second|evening|| +81330|AAAAAAAADLNDBAAA|81330|22|35|30|PM|second|evening|| +81331|AAAAAAAAELNDBAAA|81331|22|35|31|PM|second|evening|| +81332|AAAAAAAAFLNDBAAA|81332|22|35|32|PM|second|evening|| +81333|AAAAAAAAGLNDBAAA|81333|22|35|33|PM|second|evening|| +81334|AAAAAAAAHLNDBAAA|81334|22|35|34|PM|second|evening|| +81335|AAAAAAAAILNDBAAA|81335|22|35|35|PM|second|evening|| +81336|AAAAAAAAJLNDBAAA|81336|22|35|36|PM|second|evening|| +81337|AAAAAAAAKLNDBAAA|81337|22|35|37|PM|second|evening|| +81338|AAAAAAAALLNDBAAA|81338|22|35|38|PM|second|evening|| +81339|AAAAAAAAMLNDBAAA|81339|22|35|39|PM|second|evening|| +81340|AAAAAAAANLNDBAAA|81340|22|35|40|PM|second|evening|| +81341|AAAAAAAAOLNDBAAA|81341|22|35|41|PM|second|evening|| +81342|AAAAAAAAPLNDBAAA|81342|22|35|42|PM|second|evening|| +81343|AAAAAAAAAMNDBAAA|81343|22|35|43|PM|second|evening|| +81344|AAAAAAAABMNDBAAA|81344|22|35|44|PM|second|evening|| +81345|AAAAAAAACMNDBAAA|81345|22|35|45|PM|second|evening|| +81346|AAAAAAAADMNDBAAA|81346|22|35|46|PM|second|evening|| +81347|AAAAAAAAEMNDBAAA|81347|22|35|47|PM|second|evening|| +81348|AAAAAAAAFMNDBAAA|81348|22|35|48|PM|second|evening|| +81349|AAAAAAAAGMNDBAAA|81349|22|35|49|PM|second|evening|| +81350|AAAAAAAAHMNDBAAA|81350|22|35|50|PM|second|evening|| +81351|AAAAAAAAIMNDBAAA|81351|22|35|51|PM|second|evening|| +81352|AAAAAAAAJMNDBAAA|81352|22|35|52|PM|second|evening|| +81353|AAAAAAAAKMNDBAAA|81353|22|35|53|PM|second|evening|| +81354|AAAAAAAALMNDBAAA|81354|22|35|54|PM|second|evening|| +81355|AAAAAAAAMMNDBAAA|81355|22|35|55|PM|second|evening|| +81356|AAAAAAAANMNDBAAA|81356|22|35|56|PM|second|evening|| +81357|AAAAAAAAOMNDBAAA|81357|22|35|57|PM|second|evening|| +81358|AAAAAAAAPMNDBAAA|81358|22|35|58|PM|second|evening|| +81359|AAAAAAAAANNDBAAA|81359|22|35|59|PM|second|evening|| +81360|AAAAAAAABNNDBAAA|81360|22|36|0|PM|second|evening|| +81361|AAAAAAAACNNDBAAA|81361|22|36|1|PM|second|evening|| +81362|AAAAAAAADNNDBAAA|81362|22|36|2|PM|second|evening|| +81363|AAAAAAAAENNDBAAA|81363|22|36|3|PM|second|evening|| +81364|AAAAAAAAFNNDBAAA|81364|22|36|4|PM|second|evening|| +81365|AAAAAAAAGNNDBAAA|81365|22|36|5|PM|second|evening|| +81366|AAAAAAAAHNNDBAAA|81366|22|36|6|PM|second|evening|| +81367|AAAAAAAAINNDBAAA|81367|22|36|7|PM|second|evening|| +81368|AAAAAAAAJNNDBAAA|81368|22|36|8|PM|second|evening|| +81369|AAAAAAAAKNNDBAAA|81369|22|36|9|PM|second|evening|| +81370|AAAAAAAALNNDBAAA|81370|22|36|10|PM|second|evening|| +81371|AAAAAAAAMNNDBAAA|81371|22|36|11|PM|second|evening|| +81372|AAAAAAAANNNDBAAA|81372|22|36|12|PM|second|evening|| +81373|AAAAAAAAONNDBAAA|81373|22|36|13|PM|second|evening|| +81374|AAAAAAAAPNNDBAAA|81374|22|36|14|PM|second|evening|| +81375|AAAAAAAAAONDBAAA|81375|22|36|15|PM|second|evening|| +81376|AAAAAAAABONDBAAA|81376|22|36|16|PM|second|evening|| +81377|AAAAAAAACONDBAAA|81377|22|36|17|PM|second|evening|| +81378|AAAAAAAADONDBAAA|81378|22|36|18|PM|second|evening|| +81379|AAAAAAAAEONDBAAA|81379|22|36|19|PM|second|evening|| +81380|AAAAAAAAFONDBAAA|81380|22|36|20|PM|second|evening|| +81381|AAAAAAAAGONDBAAA|81381|22|36|21|PM|second|evening|| +81382|AAAAAAAAHONDBAAA|81382|22|36|22|PM|second|evening|| +81383|AAAAAAAAIONDBAAA|81383|22|36|23|PM|second|evening|| +81384|AAAAAAAAJONDBAAA|81384|22|36|24|PM|second|evening|| +81385|AAAAAAAAKONDBAAA|81385|22|36|25|PM|second|evening|| +81386|AAAAAAAALONDBAAA|81386|22|36|26|PM|second|evening|| +81387|AAAAAAAAMONDBAAA|81387|22|36|27|PM|second|evening|| +81388|AAAAAAAANONDBAAA|81388|22|36|28|PM|second|evening|| +81389|AAAAAAAAOONDBAAA|81389|22|36|29|PM|second|evening|| +81390|AAAAAAAAPONDBAAA|81390|22|36|30|PM|second|evening|| +81391|AAAAAAAAAPNDBAAA|81391|22|36|31|PM|second|evening|| +81392|AAAAAAAABPNDBAAA|81392|22|36|32|PM|second|evening|| +81393|AAAAAAAACPNDBAAA|81393|22|36|33|PM|second|evening|| +81394|AAAAAAAADPNDBAAA|81394|22|36|34|PM|second|evening|| +81395|AAAAAAAAEPNDBAAA|81395|22|36|35|PM|second|evening|| +81396|AAAAAAAAFPNDBAAA|81396|22|36|36|PM|second|evening|| +81397|AAAAAAAAGPNDBAAA|81397|22|36|37|PM|second|evening|| +81398|AAAAAAAAHPNDBAAA|81398|22|36|38|PM|second|evening|| +81399|AAAAAAAAIPNDBAAA|81399|22|36|39|PM|second|evening|| +81400|AAAAAAAAJPNDBAAA|81400|22|36|40|PM|second|evening|| +81401|AAAAAAAAKPNDBAAA|81401|22|36|41|PM|second|evening|| +81402|AAAAAAAALPNDBAAA|81402|22|36|42|PM|second|evening|| +81403|AAAAAAAAMPNDBAAA|81403|22|36|43|PM|second|evening|| +81404|AAAAAAAANPNDBAAA|81404|22|36|44|PM|second|evening|| +81405|AAAAAAAAOPNDBAAA|81405|22|36|45|PM|second|evening|| +81406|AAAAAAAAPPNDBAAA|81406|22|36|46|PM|second|evening|| +81407|AAAAAAAAAAODBAAA|81407|22|36|47|PM|second|evening|| +81408|AAAAAAAABAODBAAA|81408|22|36|48|PM|second|evening|| +81409|AAAAAAAACAODBAAA|81409|22|36|49|PM|second|evening|| +81410|AAAAAAAADAODBAAA|81410|22|36|50|PM|second|evening|| +81411|AAAAAAAAEAODBAAA|81411|22|36|51|PM|second|evening|| +81412|AAAAAAAAFAODBAAA|81412|22|36|52|PM|second|evening|| +81413|AAAAAAAAGAODBAAA|81413|22|36|53|PM|second|evening|| +81414|AAAAAAAAHAODBAAA|81414|22|36|54|PM|second|evening|| +81415|AAAAAAAAIAODBAAA|81415|22|36|55|PM|second|evening|| +81416|AAAAAAAAJAODBAAA|81416|22|36|56|PM|second|evening|| +81417|AAAAAAAAKAODBAAA|81417|22|36|57|PM|second|evening|| +81418|AAAAAAAALAODBAAA|81418|22|36|58|PM|second|evening|| +81419|AAAAAAAAMAODBAAA|81419|22|36|59|PM|second|evening|| +81420|AAAAAAAANAODBAAA|81420|22|37|0|PM|second|evening|| +81421|AAAAAAAAOAODBAAA|81421|22|37|1|PM|second|evening|| +81422|AAAAAAAAPAODBAAA|81422|22|37|2|PM|second|evening|| +81423|AAAAAAAAABODBAAA|81423|22|37|3|PM|second|evening|| +81424|AAAAAAAABBODBAAA|81424|22|37|4|PM|second|evening|| +81425|AAAAAAAACBODBAAA|81425|22|37|5|PM|second|evening|| +81426|AAAAAAAADBODBAAA|81426|22|37|6|PM|second|evening|| +81427|AAAAAAAAEBODBAAA|81427|22|37|7|PM|second|evening|| +81428|AAAAAAAAFBODBAAA|81428|22|37|8|PM|second|evening|| +81429|AAAAAAAAGBODBAAA|81429|22|37|9|PM|second|evening|| +81430|AAAAAAAAHBODBAAA|81430|22|37|10|PM|second|evening|| +81431|AAAAAAAAIBODBAAA|81431|22|37|11|PM|second|evening|| +81432|AAAAAAAAJBODBAAA|81432|22|37|12|PM|second|evening|| +81433|AAAAAAAAKBODBAAA|81433|22|37|13|PM|second|evening|| +81434|AAAAAAAALBODBAAA|81434|22|37|14|PM|second|evening|| +81435|AAAAAAAAMBODBAAA|81435|22|37|15|PM|second|evening|| +81436|AAAAAAAANBODBAAA|81436|22|37|16|PM|second|evening|| +81437|AAAAAAAAOBODBAAA|81437|22|37|17|PM|second|evening|| +81438|AAAAAAAAPBODBAAA|81438|22|37|18|PM|second|evening|| +81439|AAAAAAAAACODBAAA|81439|22|37|19|PM|second|evening|| +81440|AAAAAAAABCODBAAA|81440|22|37|20|PM|second|evening|| +81441|AAAAAAAACCODBAAA|81441|22|37|21|PM|second|evening|| +81442|AAAAAAAADCODBAAA|81442|22|37|22|PM|second|evening|| +81443|AAAAAAAAECODBAAA|81443|22|37|23|PM|second|evening|| +81444|AAAAAAAAFCODBAAA|81444|22|37|24|PM|second|evening|| +81445|AAAAAAAAGCODBAAA|81445|22|37|25|PM|second|evening|| +81446|AAAAAAAAHCODBAAA|81446|22|37|26|PM|second|evening|| +81447|AAAAAAAAICODBAAA|81447|22|37|27|PM|second|evening|| +81448|AAAAAAAAJCODBAAA|81448|22|37|28|PM|second|evening|| +81449|AAAAAAAAKCODBAAA|81449|22|37|29|PM|second|evening|| +81450|AAAAAAAALCODBAAA|81450|22|37|30|PM|second|evening|| +81451|AAAAAAAAMCODBAAA|81451|22|37|31|PM|second|evening|| +81452|AAAAAAAANCODBAAA|81452|22|37|32|PM|second|evening|| +81453|AAAAAAAAOCODBAAA|81453|22|37|33|PM|second|evening|| +81454|AAAAAAAAPCODBAAA|81454|22|37|34|PM|second|evening|| +81455|AAAAAAAAADODBAAA|81455|22|37|35|PM|second|evening|| +81456|AAAAAAAABDODBAAA|81456|22|37|36|PM|second|evening|| +81457|AAAAAAAACDODBAAA|81457|22|37|37|PM|second|evening|| +81458|AAAAAAAADDODBAAA|81458|22|37|38|PM|second|evening|| +81459|AAAAAAAAEDODBAAA|81459|22|37|39|PM|second|evening|| +81460|AAAAAAAAFDODBAAA|81460|22|37|40|PM|second|evening|| +81461|AAAAAAAAGDODBAAA|81461|22|37|41|PM|second|evening|| +81462|AAAAAAAAHDODBAAA|81462|22|37|42|PM|second|evening|| +81463|AAAAAAAAIDODBAAA|81463|22|37|43|PM|second|evening|| +81464|AAAAAAAAJDODBAAA|81464|22|37|44|PM|second|evening|| +81465|AAAAAAAAKDODBAAA|81465|22|37|45|PM|second|evening|| +81466|AAAAAAAALDODBAAA|81466|22|37|46|PM|second|evening|| +81467|AAAAAAAAMDODBAAA|81467|22|37|47|PM|second|evening|| +81468|AAAAAAAANDODBAAA|81468|22|37|48|PM|second|evening|| +81469|AAAAAAAAODODBAAA|81469|22|37|49|PM|second|evening|| +81470|AAAAAAAAPDODBAAA|81470|22|37|50|PM|second|evening|| +81471|AAAAAAAAAEODBAAA|81471|22|37|51|PM|second|evening|| +81472|AAAAAAAABEODBAAA|81472|22|37|52|PM|second|evening|| +81473|AAAAAAAACEODBAAA|81473|22|37|53|PM|second|evening|| +81474|AAAAAAAADEODBAAA|81474|22|37|54|PM|second|evening|| +81475|AAAAAAAAEEODBAAA|81475|22|37|55|PM|second|evening|| +81476|AAAAAAAAFEODBAAA|81476|22|37|56|PM|second|evening|| +81477|AAAAAAAAGEODBAAA|81477|22|37|57|PM|second|evening|| +81478|AAAAAAAAHEODBAAA|81478|22|37|58|PM|second|evening|| +81479|AAAAAAAAIEODBAAA|81479|22|37|59|PM|second|evening|| +81480|AAAAAAAAJEODBAAA|81480|22|38|0|PM|second|evening|| +81481|AAAAAAAAKEODBAAA|81481|22|38|1|PM|second|evening|| +81482|AAAAAAAALEODBAAA|81482|22|38|2|PM|second|evening|| +81483|AAAAAAAAMEODBAAA|81483|22|38|3|PM|second|evening|| +81484|AAAAAAAANEODBAAA|81484|22|38|4|PM|second|evening|| +81485|AAAAAAAAOEODBAAA|81485|22|38|5|PM|second|evening|| +81486|AAAAAAAAPEODBAAA|81486|22|38|6|PM|second|evening|| +81487|AAAAAAAAAFODBAAA|81487|22|38|7|PM|second|evening|| +81488|AAAAAAAABFODBAAA|81488|22|38|8|PM|second|evening|| +81489|AAAAAAAACFODBAAA|81489|22|38|9|PM|second|evening|| +81490|AAAAAAAADFODBAAA|81490|22|38|10|PM|second|evening|| +81491|AAAAAAAAEFODBAAA|81491|22|38|11|PM|second|evening|| +81492|AAAAAAAAFFODBAAA|81492|22|38|12|PM|second|evening|| +81493|AAAAAAAAGFODBAAA|81493|22|38|13|PM|second|evening|| +81494|AAAAAAAAHFODBAAA|81494|22|38|14|PM|second|evening|| +81495|AAAAAAAAIFODBAAA|81495|22|38|15|PM|second|evening|| +81496|AAAAAAAAJFODBAAA|81496|22|38|16|PM|second|evening|| +81497|AAAAAAAAKFODBAAA|81497|22|38|17|PM|second|evening|| +81498|AAAAAAAALFODBAAA|81498|22|38|18|PM|second|evening|| +81499|AAAAAAAAMFODBAAA|81499|22|38|19|PM|second|evening|| +81500|AAAAAAAANFODBAAA|81500|22|38|20|PM|second|evening|| +81501|AAAAAAAAOFODBAAA|81501|22|38|21|PM|second|evening|| +81502|AAAAAAAAPFODBAAA|81502|22|38|22|PM|second|evening|| +81503|AAAAAAAAAGODBAAA|81503|22|38|23|PM|second|evening|| +81504|AAAAAAAABGODBAAA|81504|22|38|24|PM|second|evening|| +81505|AAAAAAAACGODBAAA|81505|22|38|25|PM|second|evening|| +81506|AAAAAAAADGODBAAA|81506|22|38|26|PM|second|evening|| +81507|AAAAAAAAEGODBAAA|81507|22|38|27|PM|second|evening|| +81508|AAAAAAAAFGODBAAA|81508|22|38|28|PM|second|evening|| +81509|AAAAAAAAGGODBAAA|81509|22|38|29|PM|second|evening|| +81510|AAAAAAAAHGODBAAA|81510|22|38|30|PM|second|evening|| +81511|AAAAAAAAIGODBAAA|81511|22|38|31|PM|second|evening|| +81512|AAAAAAAAJGODBAAA|81512|22|38|32|PM|second|evening|| +81513|AAAAAAAAKGODBAAA|81513|22|38|33|PM|second|evening|| +81514|AAAAAAAALGODBAAA|81514|22|38|34|PM|second|evening|| +81515|AAAAAAAAMGODBAAA|81515|22|38|35|PM|second|evening|| +81516|AAAAAAAANGODBAAA|81516|22|38|36|PM|second|evening|| +81517|AAAAAAAAOGODBAAA|81517|22|38|37|PM|second|evening|| +81518|AAAAAAAAPGODBAAA|81518|22|38|38|PM|second|evening|| +81519|AAAAAAAAAHODBAAA|81519|22|38|39|PM|second|evening|| +81520|AAAAAAAABHODBAAA|81520|22|38|40|PM|second|evening|| +81521|AAAAAAAACHODBAAA|81521|22|38|41|PM|second|evening|| +81522|AAAAAAAADHODBAAA|81522|22|38|42|PM|second|evening|| +81523|AAAAAAAAEHODBAAA|81523|22|38|43|PM|second|evening|| +81524|AAAAAAAAFHODBAAA|81524|22|38|44|PM|second|evening|| +81525|AAAAAAAAGHODBAAA|81525|22|38|45|PM|second|evening|| +81526|AAAAAAAAHHODBAAA|81526|22|38|46|PM|second|evening|| +81527|AAAAAAAAIHODBAAA|81527|22|38|47|PM|second|evening|| +81528|AAAAAAAAJHODBAAA|81528|22|38|48|PM|second|evening|| +81529|AAAAAAAAKHODBAAA|81529|22|38|49|PM|second|evening|| +81530|AAAAAAAALHODBAAA|81530|22|38|50|PM|second|evening|| +81531|AAAAAAAAMHODBAAA|81531|22|38|51|PM|second|evening|| +81532|AAAAAAAANHODBAAA|81532|22|38|52|PM|second|evening|| +81533|AAAAAAAAOHODBAAA|81533|22|38|53|PM|second|evening|| +81534|AAAAAAAAPHODBAAA|81534|22|38|54|PM|second|evening|| +81535|AAAAAAAAAIODBAAA|81535|22|38|55|PM|second|evening|| +81536|AAAAAAAABIODBAAA|81536|22|38|56|PM|second|evening|| +81537|AAAAAAAACIODBAAA|81537|22|38|57|PM|second|evening|| +81538|AAAAAAAADIODBAAA|81538|22|38|58|PM|second|evening|| +81539|AAAAAAAAEIODBAAA|81539|22|38|59|PM|second|evening|| +81540|AAAAAAAAFIODBAAA|81540|22|39|0|PM|second|evening|| +81541|AAAAAAAAGIODBAAA|81541|22|39|1|PM|second|evening|| +81542|AAAAAAAAHIODBAAA|81542|22|39|2|PM|second|evening|| +81543|AAAAAAAAIIODBAAA|81543|22|39|3|PM|second|evening|| +81544|AAAAAAAAJIODBAAA|81544|22|39|4|PM|second|evening|| +81545|AAAAAAAAKIODBAAA|81545|22|39|5|PM|second|evening|| +81546|AAAAAAAALIODBAAA|81546|22|39|6|PM|second|evening|| +81547|AAAAAAAAMIODBAAA|81547|22|39|7|PM|second|evening|| +81548|AAAAAAAANIODBAAA|81548|22|39|8|PM|second|evening|| +81549|AAAAAAAAOIODBAAA|81549|22|39|9|PM|second|evening|| +81550|AAAAAAAAPIODBAAA|81550|22|39|10|PM|second|evening|| +81551|AAAAAAAAAJODBAAA|81551|22|39|11|PM|second|evening|| +81552|AAAAAAAABJODBAAA|81552|22|39|12|PM|second|evening|| +81553|AAAAAAAACJODBAAA|81553|22|39|13|PM|second|evening|| +81554|AAAAAAAADJODBAAA|81554|22|39|14|PM|second|evening|| +81555|AAAAAAAAEJODBAAA|81555|22|39|15|PM|second|evening|| +81556|AAAAAAAAFJODBAAA|81556|22|39|16|PM|second|evening|| +81557|AAAAAAAAGJODBAAA|81557|22|39|17|PM|second|evening|| +81558|AAAAAAAAHJODBAAA|81558|22|39|18|PM|second|evening|| +81559|AAAAAAAAIJODBAAA|81559|22|39|19|PM|second|evening|| +81560|AAAAAAAAJJODBAAA|81560|22|39|20|PM|second|evening|| +81561|AAAAAAAAKJODBAAA|81561|22|39|21|PM|second|evening|| +81562|AAAAAAAALJODBAAA|81562|22|39|22|PM|second|evening|| +81563|AAAAAAAAMJODBAAA|81563|22|39|23|PM|second|evening|| +81564|AAAAAAAANJODBAAA|81564|22|39|24|PM|second|evening|| +81565|AAAAAAAAOJODBAAA|81565|22|39|25|PM|second|evening|| +81566|AAAAAAAAPJODBAAA|81566|22|39|26|PM|second|evening|| +81567|AAAAAAAAAKODBAAA|81567|22|39|27|PM|second|evening|| +81568|AAAAAAAABKODBAAA|81568|22|39|28|PM|second|evening|| +81569|AAAAAAAACKODBAAA|81569|22|39|29|PM|second|evening|| +81570|AAAAAAAADKODBAAA|81570|22|39|30|PM|second|evening|| +81571|AAAAAAAAEKODBAAA|81571|22|39|31|PM|second|evening|| +81572|AAAAAAAAFKODBAAA|81572|22|39|32|PM|second|evening|| +81573|AAAAAAAAGKODBAAA|81573|22|39|33|PM|second|evening|| +81574|AAAAAAAAHKODBAAA|81574|22|39|34|PM|second|evening|| +81575|AAAAAAAAIKODBAAA|81575|22|39|35|PM|second|evening|| +81576|AAAAAAAAJKODBAAA|81576|22|39|36|PM|second|evening|| +81577|AAAAAAAAKKODBAAA|81577|22|39|37|PM|second|evening|| +81578|AAAAAAAALKODBAAA|81578|22|39|38|PM|second|evening|| +81579|AAAAAAAAMKODBAAA|81579|22|39|39|PM|second|evening|| +81580|AAAAAAAANKODBAAA|81580|22|39|40|PM|second|evening|| +81581|AAAAAAAAOKODBAAA|81581|22|39|41|PM|second|evening|| +81582|AAAAAAAAPKODBAAA|81582|22|39|42|PM|second|evening|| +81583|AAAAAAAAALODBAAA|81583|22|39|43|PM|second|evening|| +81584|AAAAAAAABLODBAAA|81584|22|39|44|PM|second|evening|| +81585|AAAAAAAACLODBAAA|81585|22|39|45|PM|second|evening|| +81586|AAAAAAAADLODBAAA|81586|22|39|46|PM|second|evening|| +81587|AAAAAAAAELODBAAA|81587|22|39|47|PM|second|evening|| +81588|AAAAAAAAFLODBAAA|81588|22|39|48|PM|second|evening|| +81589|AAAAAAAAGLODBAAA|81589|22|39|49|PM|second|evening|| +81590|AAAAAAAAHLODBAAA|81590|22|39|50|PM|second|evening|| +81591|AAAAAAAAILODBAAA|81591|22|39|51|PM|second|evening|| +81592|AAAAAAAAJLODBAAA|81592|22|39|52|PM|second|evening|| +81593|AAAAAAAAKLODBAAA|81593|22|39|53|PM|second|evening|| +81594|AAAAAAAALLODBAAA|81594|22|39|54|PM|second|evening|| +81595|AAAAAAAAMLODBAAA|81595|22|39|55|PM|second|evening|| +81596|AAAAAAAANLODBAAA|81596|22|39|56|PM|second|evening|| +81597|AAAAAAAAOLODBAAA|81597|22|39|57|PM|second|evening|| +81598|AAAAAAAAPLODBAAA|81598|22|39|58|PM|second|evening|| +81599|AAAAAAAAAMODBAAA|81599|22|39|59|PM|second|evening|| +81600|AAAAAAAABMODBAAA|81600|22|40|0|PM|second|evening|| +81601|AAAAAAAACMODBAAA|81601|22|40|1|PM|second|evening|| +81602|AAAAAAAADMODBAAA|81602|22|40|2|PM|second|evening|| +81603|AAAAAAAAEMODBAAA|81603|22|40|3|PM|second|evening|| +81604|AAAAAAAAFMODBAAA|81604|22|40|4|PM|second|evening|| +81605|AAAAAAAAGMODBAAA|81605|22|40|5|PM|second|evening|| +81606|AAAAAAAAHMODBAAA|81606|22|40|6|PM|second|evening|| +81607|AAAAAAAAIMODBAAA|81607|22|40|7|PM|second|evening|| +81608|AAAAAAAAJMODBAAA|81608|22|40|8|PM|second|evening|| +81609|AAAAAAAAKMODBAAA|81609|22|40|9|PM|second|evening|| +81610|AAAAAAAALMODBAAA|81610|22|40|10|PM|second|evening|| +81611|AAAAAAAAMMODBAAA|81611|22|40|11|PM|second|evening|| +81612|AAAAAAAANMODBAAA|81612|22|40|12|PM|second|evening|| +81613|AAAAAAAAOMODBAAA|81613|22|40|13|PM|second|evening|| +81614|AAAAAAAAPMODBAAA|81614|22|40|14|PM|second|evening|| +81615|AAAAAAAAANODBAAA|81615|22|40|15|PM|second|evening|| +81616|AAAAAAAABNODBAAA|81616|22|40|16|PM|second|evening|| +81617|AAAAAAAACNODBAAA|81617|22|40|17|PM|second|evening|| +81618|AAAAAAAADNODBAAA|81618|22|40|18|PM|second|evening|| +81619|AAAAAAAAENODBAAA|81619|22|40|19|PM|second|evening|| +81620|AAAAAAAAFNODBAAA|81620|22|40|20|PM|second|evening|| +81621|AAAAAAAAGNODBAAA|81621|22|40|21|PM|second|evening|| +81622|AAAAAAAAHNODBAAA|81622|22|40|22|PM|second|evening|| +81623|AAAAAAAAINODBAAA|81623|22|40|23|PM|second|evening|| +81624|AAAAAAAAJNODBAAA|81624|22|40|24|PM|second|evening|| +81625|AAAAAAAAKNODBAAA|81625|22|40|25|PM|second|evening|| +81626|AAAAAAAALNODBAAA|81626|22|40|26|PM|second|evening|| +81627|AAAAAAAAMNODBAAA|81627|22|40|27|PM|second|evening|| +81628|AAAAAAAANNODBAAA|81628|22|40|28|PM|second|evening|| +81629|AAAAAAAAONODBAAA|81629|22|40|29|PM|second|evening|| +81630|AAAAAAAAPNODBAAA|81630|22|40|30|PM|second|evening|| +81631|AAAAAAAAAOODBAAA|81631|22|40|31|PM|second|evening|| +81632|AAAAAAAABOODBAAA|81632|22|40|32|PM|second|evening|| +81633|AAAAAAAACOODBAAA|81633|22|40|33|PM|second|evening|| +81634|AAAAAAAADOODBAAA|81634|22|40|34|PM|second|evening|| +81635|AAAAAAAAEOODBAAA|81635|22|40|35|PM|second|evening|| +81636|AAAAAAAAFOODBAAA|81636|22|40|36|PM|second|evening|| +81637|AAAAAAAAGOODBAAA|81637|22|40|37|PM|second|evening|| +81638|AAAAAAAAHOODBAAA|81638|22|40|38|PM|second|evening|| +81639|AAAAAAAAIOODBAAA|81639|22|40|39|PM|second|evening|| +81640|AAAAAAAAJOODBAAA|81640|22|40|40|PM|second|evening|| +81641|AAAAAAAAKOODBAAA|81641|22|40|41|PM|second|evening|| +81642|AAAAAAAALOODBAAA|81642|22|40|42|PM|second|evening|| +81643|AAAAAAAAMOODBAAA|81643|22|40|43|PM|second|evening|| +81644|AAAAAAAANOODBAAA|81644|22|40|44|PM|second|evening|| +81645|AAAAAAAAOOODBAAA|81645|22|40|45|PM|second|evening|| +81646|AAAAAAAAPOODBAAA|81646|22|40|46|PM|second|evening|| +81647|AAAAAAAAAPODBAAA|81647|22|40|47|PM|second|evening|| +81648|AAAAAAAABPODBAAA|81648|22|40|48|PM|second|evening|| +81649|AAAAAAAACPODBAAA|81649|22|40|49|PM|second|evening|| +81650|AAAAAAAADPODBAAA|81650|22|40|50|PM|second|evening|| +81651|AAAAAAAAEPODBAAA|81651|22|40|51|PM|second|evening|| +81652|AAAAAAAAFPODBAAA|81652|22|40|52|PM|second|evening|| +81653|AAAAAAAAGPODBAAA|81653|22|40|53|PM|second|evening|| +81654|AAAAAAAAHPODBAAA|81654|22|40|54|PM|second|evening|| +81655|AAAAAAAAIPODBAAA|81655|22|40|55|PM|second|evening|| +81656|AAAAAAAAJPODBAAA|81656|22|40|56|PM|second|evening|| +81657|AAAAAAAAKPODBAAA|81657|22|40|57|PM|second|evening|| +81658|AAAAAAAALPODBAAA|81658|22|40|58|PM|second|evening|| +81659|AAAAAAAAMPODBAAA|81659|22|40|59|PM|second|evening|| +81660|AAAAAAAANPODBAAA|81660|22|41|0|PM|second|evening|| +81661|AAAAAAAAOPODBAAA|81661|22|41|1|PM|second|evening|| +81662|AAAAAAAAPPODBAAA|81662|22|41|2|PM|second|evening|| +81663|AAAAAAAAAAPDBAAA|81663|22|41|3|PM|second|evening|| +81664|AAAAAAAABAPDBAAA|81664|22|41|4|PM|second|evening|| +81665|AAAAAAAACAPDBAAA|81665|22|41|5|PM|second|evening|| +81666|AAAAAAAADAPDBAAA|81666|22|41|6|PM|second|evening|| +81667|AAAAAAAAEAPDBAAA|81667|22|41|7|PM|second|evening|| +81668|AAAAAAAAFAPDBAAA|81668|22|41|8|PM|second|evening|| +81669|AAAAAAAAGAPDBAAA|81669|22|41|9|PM|second|evening|| +81670|AAAAAAAAHAPDBAAA|81670|22|41|10|PM|second|evening|| +81671|AAAAAAAAIAPDBAAA|81671|22|41|11|PM|second|evening|| +81672|AAAAAAAAJAPDBAAA|81672|22|41|12|PM|second|evening|| +81673|AAAAAAAAKAPDBAAA|81673|22|41|13|PM|second|evening|| +81674|AAAAAAAALAPDBAAA|81674|22|41|14|PM|second|evening|| +81675|AAAAAAAAMAPDBAAA|81675|22|41|15|PM|second|evening|| +81676|AAAAAAAANAPDBAAA|81676|22|41|16|PM|second|evening|| +81677|AAAAAAAAOAPDBAAA|81677|22|41|17|PM|second|evening|| +81678|AAAAAAAAPAPDBAAA|81678|22|41|18|PM|second|evening|| +81679|AAAAAAAAABPDBAAA|81679|22|41|19|PM|second|evening|| +81680|AAAAAAAABBPDBAAA|81680|22|41|20|PM|second|evening|| +81681|AAAAAAAACBPDBAAA|81681|22|41|21|PM|second|evening|| +81682|AAAAAAAADBPDBAAA|81682|22|41|22|PM|second|evening|| +81683|AAAAAAAAEBPDBAAA|81683|22|41|23|PM|second|evening|| +81684|AAAAAAAAFBPDBAAA|81684|22|41|24|PM|second|evening|| +81685|AAAAAAAAGBPDBAAA|81685|22|41|25|PM|second|evening|| +81686|AAAAAAAAHBPDBAAA|81686|22|41|26|PM|second|evening|| +81687|AAAAAAAAIBPDBAAA|81687|22|41|27|PM|second|evening|| +81688|AAAAAAAAJBPDBAAA|81688|22|41|28|PM|second|evening|| +81689|AAAAAAAAKBPDBAAA|81689|22|41|29|PM|second|evening|| +81690|AAAAAAAALBPDBAAA|81690|22|41|30|PM|second|evening|| +81691|AAAAAAAAMBPDBAAA|81691|22|41|31|PM|second|evening|| +81692|AAAAAAAANBPDBAAA|81692|22|41|32|PM|second|evening|| +81693|AAAAAAAAOBPDBAAA|81693|22|41|33|PM|second|evening|| +81694|AAAAAAAAPBPDBAAA|81694|22|41|34|PM|second|evening|| +81695|AAAAAAAAACPDBAAA|81695|22|41|35|PM|second|evening|| +81696|AAAAAAAABCPDBAAA|81696|22|41|36|PM|second|evening|| +81697|AAAAAAAACCPDBAAA|81697|22|41|37|PM|second|evening|| +81698|AAAAAAAADCPDBAAA|81698|22|41|38|PM|second|evening|| +81699|AAAAAAAAECPDBAAA|81699|22|41|39|PM|second|evening|| +81700|AAAAAAAAFCPDBAAA|81700|22|41|40|PM|second|evening|| +81701|AAAAAAAAGCPDBAAA|81701|22|41|41|PM|second|evening|| +81702|AAAAAAAAHCPDBAAA|81702|22|41|42|PM|second|evening|| +81703|AAAAAAAAICPDBAAA|81703|22|41|43|PM|second|evening|| +81704|AAAAAAAAJCPDBAAA|81704|22|41|44|PM|second|evening|| +81705|AAAAAAAAKCPDBAAA|81705|22|41|45|PM|second|evening|| +81706|AAAAAAAALCPDBAAA|81706|22|41|46|PM|second|evening|| +81707|AAAAAAAAMCPDBAAA|81707|22|41|47|PM|second|evening|| +81708|AAAAAAAANCPDBAAA|81708|22|41|48|PM|second|evening|| +81709|AAAAAAAAOCPDBAAA|81709|22|41|49|PM|second|evening|| +81710|AAAAAAAAPCPDBAAA|81710|22|41|50|PM|second|evening|| +81711|AAAAAAAAADPDBAAA|81711|22|41|51|PM|second|evening|| +81712|AAAAAAAABDPDBAAA|81712|22|41|52|PM|second|evening|| +81713|AAAAAAAACDPDBAAA|81713|22|41|53|PM|second|evening|| +81714|AAAAAAAADDPDBAAA|81714|22|41|54|PM|second|evening|| +81715|AAAAAAAAEDPDBAAA|81715|22|41|55|PM|second|evening|| +81716|AAAAAAAAFDPDBAAA|81716|22|41|56|PM|second|evening|| +81717|AAAAAAAAGDPDBAAA|81717|22|41|57|PM|second|evening|| +81718|AAAAAAAAHDPDBAAA|81718|22|41|58|PM|second|evening|| +81719|AAAAAAAAIDPDBAAA|81719|22|41|59|PM|second|evening|| +81720|AAAAAAAAJDPDBAAA|81720|22|42|0|PM|second|evening|| +81721|AAAAAAAAKDPDBAAA|81721|22|42|1|PM|second|evening|| +81722|AAAAAAAALDPDBAAA|81722|22|42|2|PM|second|evening|| +81723|AAAAAAAAMDPDBAAA|81723|22|42|3|PM|second|evening|| +81724|AAAAAAAANDPDBAAA|81724|22|42|4|PM|second|evening|| +81725|AAAAAAAAODPDBAAA|81725|22|42|5|PM|second|evening|| +81726|AAAAAAAAPDPDBAAA|81726|22|42|6|PM|second|evening|| +81727|AAAAAAAAAEPDBAAA|81727|22|42|7|PM|second|evening|| +81728|AAAAAAAABEPDBAAA|81728|22|42|8|PM|second|evening|| +81729|AAAAAAAACEPDBAAA|81729|22|42|9|PM|second|evening|| +81730|AAAAAAAADEPDBAAA|81730|22|42|10|PM|second|evening|| +81731|AAAAAAAAEEPDBAAA|81731|22|42|11|PM|second|evening|| +81732|AAAAAAAAFEPDBAAA|81732|22|42|12|PM|second|evening|| +81733|AAAAAAAAGEPDBAAA|81733|22|42|13|PM|second|evening|| +81734|AAAAAAAAHEPDBAAA|81734|22|42|14|PM|second|evening|| +81735|AAAAAAAAIEPDBAAA|81735|22|42|15|PM|second|evening|| +81736|AAAAAAAAJEPDBAAA|81736|22|42|16|PM|second|evening|| +81737|AAAAAAAAKEPDBAAA|81737|22|42|17|PM|second|evening|| +81738|AAAAAAAALEPDBAAA|81738|22|42|18|PM|second|evening|| +81739|AAAAAAAAMEPDBAAA|81739|22|42|19|PM|second|evening|| +81740|AAAAAAAANEPDBAAA|81740|22|42|20|PM|second|evening|| +81741|AAAAAAAAOEPDBAAA|81741|22|42|21|PM|second|evening|| +81742|AAAAAAAAPEPDBAAA|81742|22|42|22|PM|second|evening|| +81743|AAAAAAAAAFPDBAAA|81743|22|42|23|PM|second|evening|| +81744|AAAAAAAABFPDBAAA|81744|22|42|24|PM|second|evening|| +81745|AAAAAAAACFPDBAAA|81745|22|42|25|PM|second|evening|| +81746|AAAAAAAADFPDBAAA|81746|22|42|26|PM|second|evening|| +81747|AAAAAAAAEFPDBAAA|81747|22|42|27|PM|second|evening|| +81748|AAAAAAAAFFPDBAAA|81748|22|42|28|PM|second|evening|| +81749|AAAAAAAAGFPDBAAA|81749|22|42|29|PM|second|evening|| +81750|AAAAAAAAHFPDBAAA|81750|22|42|30|PM|second|evening|| +81751|AAAAAAAAIFPDBAAA|81751|22|42|31|PM|second|evening|| +81752|AAAAAAAAJFPDBAAA|81752|22|42|32|PM|second|evening|| +81753|AAAAAAAAKFPDBAAA|81753|22|42|33|PM|second|evening|| +81754|AAAAAAAALFPDBAAA|81754|22|42|34|PM|second|evening|| +81755|AAAAAAAAMFPDBAAA|81755|22|42|35|PM|second|evening|| +81756|AAAAAAAANFPDBAAA|81756|22|42|36|PM|second|evening|| +81757|AAAAAAAAOFPDBAAA|81757|22|42|37|PM|second|evening|| +81758|AAAAAAAAPFPDBAAA|81758|22|42|38|PM|second|evening|| +81759|AAAAAAAAAGPDBAAA|81759|22|42|39|PM|second|evening|| +81760|AAAAAAAABGPDBAAA|81760|22|42|40|PM|second|evening|| +81761|AAAAAAAACGPDBAAA|81761|22|42|41|PM|second|evening|| +81762|AAAAAAAADGPDBAAA|81762|22|42|42|PM|second|evening|| +81763|AAAAAAAAEGPDBAAA|81763|22|42|43|PM|second|evening|| +81764|AAAAAAAAFGPDBAAA|81764|22|42|44|PM|second|evening|| +81765|AAAAAAAAGGPDBAAA|81765|22|42|45|PM|second|evening|| +81766|AAAAAAAAHGPDBAAA|81766|22|42|46|PM|second|evening|| +81767|AAAAAAAAIGPDBAAA|81767|22|42|47|PM|second|evening|| +81768|AAAAAAAAJGPDBAAA|81768|22|42|48|PM|second|evening|| +81769|AAAAAAAAKGPDBAAA|81769|22|42|49|PM|second|evening|| +81770|AAAAAAAALGPDBAAA|81770|22|42|50|PM|second|evening|| +81771|AAAAAAAAMGPDBAAA|81771|22|42|51|PM|second|evening|| +81772|AAAAAAAANGPDBAAA|81772|22|42|52|PM|second|evening|| +81773|AAAAAAAAOGPDBAAA|81773|22|42|53|PM|second|evening|| +81774|AAAAAAAAPGPDBAAA|81774|22|42|54|PM|second|evening|| +81775|AAAAAAAAAHPDBAAA|81775|22|42|55|PM|second|evening|| +81776|AAAAAAAABHPDBAAA|81776|22|42|56|PM|second|evening|| +81777|AAAAAAAACHPDBAAA|81777|22|42|57|PM|second|evening|| +81778|AAAAAAAADHPDBAAA|81778|22|42|58|PM|second|evening|| +81779|AAAAAAAAEHPDBAAA|81779|22|42|59|PM|second|evening|| +81780|AAAAAAAAFHPDBAAA|81780|22|43|0|PM|second|evening|| +81781|AAAAAAAAGHPDBAAA|81781|22|43|1|PM|second|evening|| +81782|AAAAAAAAHHPDBAAA|81782|22|43|2|PM|second|evening|| +81783|AAAAAAAAIHPDBAAA|81783|22|43|3|PM|second|evening|| +81784|AAAAAAAAJHPDBAAA|81784|22|43|4|PM|second|evening|| +81785|AAAAAAAAKHPDBAAA|81785|22|43|5|PM|second|evening|| +81786|AAAAAAAALHPDBAAA|81786|22|43|6|PM|second|evening|| +81787|AAAAAAAAMHPDBAAA|81787|22|43|7|PM|second|evening|| +81788|AAAAAAAANHPDBAAA|81788|22|43|8|PM|second|evening|| +81789|AAAAAAAAOHPDBAAA|81789|22|43|9|PM|second|evening|| +81790|AAAAAAAAPHPDBAAA|81790|22|43|10|PM|second|evening|| +81791|AAAAAAAAAIPDBAAA|81791|22|43|11|PM|second|evening|| +81792|AAAAAAAABIPDBAAA|81792|22|43|12|PM|second|evening|| +81793|AAAAAAAACIPDBAAA|81793|22|43|13|PM|second|evening|| +81794|AAAAAAAADIPDBAAA|81794|22|43|14|PM|second|evening|| +81795|AAAAAAAAEIPDBAAA|81795|22|43|15|PM|second|evening|| +81796|AAAAAAAAFIPDBAAA|81796|22|43|16|PM|second|evening|| +81797|AAAAAAAAGIPDBAAA|81797|22|43|17|PM|second|evening|| +81798|AAAAAAAAHIPDBAAA|81798|22|43|18|PM|second|evening|| +81799|AAAAAAAAIIPDBAAA|81799|22|43|19|PM|second|evening|| +81800|AAAAAAAAJIPDBAAA|81800|22|43|20|PM|second|evening|| +81801|AAAAAAAAKIPDBAAA|81801|22|43|21|PM|second|evening|| +81802|AAAAAAAALIPDBAAA|81802|22|43|22|PM|second|evening|| +81803|AAAAAAAAMIPDBAAA|81803|22|43|23|PM|second|evening|| +81804|AAAAAAAANIPDBAAA|81804|22|43|24|PM|second|evening|| +81805|AAAAAAAAOIPDBAAA|81805|22|43|25|PM|second|evening|| +81806|AAAAAAAAPIPDBAAA|81806|22|43|26|PM|second|evening|| +81807|AAAAAAAAAJPDBAAA|81807|22|43|27|PM|second|evening|| +81808|AAAAAAAABJPDBAAA|81808|22|43|28|PM|second|evening|| +81809|AAAAAAAACJPDBAAA|81809|22|43|29|PM|second|evening|| +81810|AAAAAAAADJPDBAAA|81810|22|43|30|PM|second|evening|| +81811|AAAAAAAAEJPDBAAA|81811|22|43|31|PM|second|evening|| +81812|AAAAAAAAFJPDBAAA|81812|22|43|32|PM|second|evening|| +81813|AAAAAAAAGJPDBAAA|81813|22|43|33|PM|second|evening|| +81814|AAAAAAAAHJPDBAAA|81814|22|43|34|PM|second|evening|| +81815|AAAAAAAAIJPDBAAA|81815|22|43|35|PM|second|evening|| +81816|AAAAAAAAJJPDBAAA|81816|22|43|36|PM|second|evening|| +81817|AAAAAAAAKJPDBAAA|81817|22|43|37|PM|second|evening|| +81818|AAAAAAAALJPDBAAA|81818|22|43|38|PM|second|evening|| +81819|AAAAAAAAMJPDBAAA|81819|22|43|39|PM|second|evening|| +81820|AAAAAAAANJPDBAAA|81820|22|43|40|PM|second|evening|| +81821|AAAAAAAAOJPDBAAA|81821|22|43|41|PM|second|evening|| +81822|AAAAAAAAPJPDBAAA|81822|22|43|42|PM|second|evening|| +81823|AAAAAAAAAKPDBAAA|81823|22|43|43|PM|second|evening|| +81824|AAAAAAAABKPDBAAA|81824|22|43|44|PM|second|evening|| +81825|AAAAAAAACKPDBAAA|81825|22|43|45|PM|second|evening|| +81826|AAAAAAAADKPDBAAA|81826|22|43|46|PM|second|evening|| +81827|AAAAAAAAEKPDBAAA|81827|22|43|47|PM|second|evening|| +81828|AAAAAAAAFKPDBAAA|81828|22|43|48|PM|second|evening|| +81829|AAAAAAAAGKPDBAAA|81829|22|43|49|PM|second|evening|| +81830|AAAAAAAAHKPDBAAA|81830|22|43|50|PM|second|evening|| +81831|AAAAAAAAIKPDBAAA|81831|22|43|51|PM|second|evening|| +81832|AAAAAAAAJKPDBAAA|81832|22|43|52|PM|second|evening|| +81833|AAAAAAAAKKPDBAAA|81833|22|43|53|PM|second|evening|| +81834|AAAAAAAALKPDBAAA|81834|22|43|54|PM|second|evening|| +81835|AAAAAAAAMKPDBAAA|81835|22|43|55|PM|second|evening|| +81836|AAAAAAAANKPDBAAA|81836|22|43|56|PM|second|evening|| +81837|AAAAAAAAOKPDBAAA|81837|22|43|57|PM|second|evening|| +81838|AAAAAAAAPKPDBAAA|81838|22|43|58|PM|second|evening|| +81839|AAAAAAAAALPDBAAA|81839|22|43|59|PM|second|evening|| +81840|AAAAAAAABLPDBAAA|81840|22|44|0|PM|second|evening|| +81841|AAAAAAAACLPDBAAA|81841|22|44|1|PM|second|evening|| +81842|AAAAAAAADLPDBAAA|81842|22|44|2|PM|second|evening|| +81843|AAAAAAAAELPDBAAA|81843|22|44|3|PM|second|evening|| +81844|AAAAAAAAFLPDBAAA|81844|22|44|4|PM|second|evening|| +81845|AAAAAAAAGLPDBAAA|81845|22|44|5|PM|second|evening|| +81846|AAAAAAAAHLPDBAAA|81846|22|44|6|PM|second|evening|| +81847|AAAAAAAAILPDBAAA|81847|22|44|7|PM|second|evening|| +81848|AAAAAAAAJLPDBAAA|81848|22|44|8|PM|second|evening|| +81849|AAAAAAAAKLPDBAAA|81849|22|44|9|PM|second|evening|| +81850|AAAAAAAALLPDBAAA|81850|22|44|10|PM|second|evening|| +81851|AAAAAAAAMLPDBAAA|81851|22|44|11|PM|second|evening|| +81852|AAAAAAAANLPDBAAA|81852|22|44|12|PM|second|evening|| +81853|AAAAAAAAOLPDBAAA|81853|22|44|13|PM|second|evening|| +81854|AAAAAAAAPLPDBAAA|81854|22|44|14|PM|second|evening|| +81855|AAAAAAAAAMPDBAAA|81855|22|44|15|PM|second|evening|| +81856|AAAAAAAABMPDBAAA|81856|22|44|16|PM|second|evening|| +81857|AAAAAAAACMPDBAAA|81857|22|44|17|PM|second|evening|| +81858|AAAAAAAADMPDBAAA|81858|22|44|18|PM|second|evening|| +81859|AAAAAAAAEMPDBAAA|81859|22|44|19|PM|second|evening|| +81860|AAAAAAAAFMPDBAAA|81860|22|44|20|PM|second|evening|| +81861|AAAAAAAAGMPDBAAA|81861|22|44|21|PM|second|evening|| +81862|AAAAAAAAHMPDBAAA|81862|22|44|22|PM|second|evening|| +81863|AAAAAAAAIMPDBAAA|81863|22|44|23|PM|second|evening|| +81864|AAAAAAAAJMPDBAAA|81864|22|44|24|PM|second|evening|| +81865|AAAAAAAAKMPDBAAA|81865|22|44|25|PM|second|evening|| +81866|AAAAAAAALMPDBAAA|81866|22|44|26|PM|second|evening|| +81867|AAAAAAAAMMPDBAAA|81867|22|44|27|PM|second|evening|| +81868|AAAAAAAANMPDBAAA|81868|22|44|28|PM|second|evening|| +81869|AAAAAAAAOMPDBAAA|81869|22|44|29|PM|second|evening|| +81870|AAAAAAAAPMPDBAAA|81870|22|44|30|PM|second|evening|| +81871|AAAAAAAAANPDBAAA|81871|22|44|31|PM|second|evening|| +81872|AAAAAAAABNPDBAAA|81872|22|44|32|PM|second|evening|| +81873|AAAAAAAACNPDBAAA|81873|22|44|33|PM|second|evening|| +81874|AAAAAAAADNPDBAAA|81874|22|44|34|PM|second|evening|| +81875|AAAAAAAAENPDBAAA|81875|22|44|35|PM|second|evening|| +81876|AAAAAAAAFNPDBAAA|81876|22|44|36|PM|second|evening|| +81877|AAAAAAAAGNPDBAAA|81877|22|44|37|PM|second|evening|| +81878|AAAAAAAAHNPDBAAA|81878|22|44|38|PM|second|evening|| +81879|AAAAAAAAINPDBAAA|81879|22|44|39|PM|second|evening|| +81880|AAAAAAAAJNPDBAAA|81880|22|44|40|PM|second|evening|| +81881|AAAAAAAAKNPDBAAA|81881|22|44|41|PM|second|evening|| +81882|AAAAAAAALNPDBAAA|81882|22|44|42|PM|second|evening|| +81883|AAAAAAAAMNPDBAAA|81883|22|44|43|PM|second|evening|| +81884|AAAAAAAANNPDBAAA|81884|22|44|44|PM|second|evening|| +81885|AAAAAAAAONPDBAAA|81885|22|44|45|PM|second|evening|| +81886|AAAAAAAAPNPDBAAA|81886|22|44|46|PM|second|evening|| +81887|AAAAAAAAAOPDBAAA|81887|22|44|47|PM|second|evening|| +81888|AAAAAAAABOPDBAAA|81888|22|44|48|PM|second|evening|| +81889|AAAAAAAACOPDBAAA|81889|22|44|49|PM|second|evening|| +81890|AAAAAAAADOPDBAAA|81890|22|44|50|PM|second|evening|| +81891|AAAAAAAAEOPDBAAA|81891|22|44|51|PM|second|evening|| +81892|AAAAAAAAFOPDBAAA|81892|22|44|52|PM|second|evening|| +81893|AAAAAAAAGOPDBAAA|81893|22|44|53|PM|second|evening|| +81894|AAAAAAAAHOPDBAAA|81894|22|44|54|PM|second|evening|| +81895|AAAAAAAAIOPDBAAA|81895|22|44|55|PM|second|evening|| +81896|AAAAAAAAJOPDBAAA|81896|22|44|56|PM|second|evening|| +81897|AAAAAAAAKOPDBAAA|81897|22|44|57|PM|second|evening|| +81898|AAAAAAAALOPDBAAA|81898|22|44|58|PM|second|evening|| +81899|AAAAAAAAMOPDBAAA|81899|22|44|59|PM|second|evening|| +81900|AAAAAAAANOPDBAAA|81900|22|45|0|PM|second|evening|| +81901|AAAAAAAAOOPDBAAA|81901|22|45|1|PM|second|evening|| +81902|AAAAAAAAPOPDBAAA|81902|22|45|2|PM|second|evening|| +81903|AAAAAAAAAPPDBAAA|81903|22|45|3|PM|second|evening|| +81904|AAAAAAAABPPDBAAA|81904|22|45|4|PM|second|evening|| +81905|AAAAAAAACPPDBAAA|81905|22|45|5|PM|second|evening|| +81906|AAAAAAAADPPDBAAA|81906|22|45|6|PM|second|evening|| +81907|AAAAAAAAEPPDBAAA|81907|22|45|7|PM|second|evening|| +81908|AAAAAAAAFPPDBAAA|81908|22|45|8|PM|second|evening|| +81909|AAAAAAAAGPPDBAAA|81909|22|45|9|PM|second|evening|| +81910|AAAAAAAAHPPDBAAA|81910|22|45|10|PM|second|evening|| +81911|AAAAAAAAIPPDBAAA|81911|22|45|11|PM|second|evening|| +81912|AAAAAAAAJPPDBAAA|81912|22|45|12|PM|second|evening|| +81913|AAAAAAAAKPPDBAAA|81913|22|45|13|PM|second|evening|| +81914|AAAAAAAALPPDBAAA|81914|22|45|14|PM|second|evening|| +81915|AAAAAAAAMPPDBAAA|81915|22|45|15|PM|second|evening|| +81916|AAAAAAAANPPDBAAA|81916|22|45|16|PM|second|evening|| +81917|AAAAAAAAOPPDBAAA|81917|22|45|17|PM|second|evening|| +81918|AAAAAAAAPPPDBAAA|81918|22|45|18|PM|second|evening|| +81919|AAAAAAAAAAAEBAAA|81919|22|45|19|PM|second|evening|| +81920|AAAAAAAABAAEBAAA|81920|22|45|20|PM|second|evening|| +81921|AAAAAAAACAAEBAAA|81921|22|45|21|PM|second|evening|| +81922|AAAAAAAADAAEBAAA|81922|22|45|22|PM|second|evening|| +81923|AAAAAAAAEAAEBAAA|81923|22|45|23|PM|second|evening|| +81924|AAAAAAAAFAAEBAAA|81924|22|45|24|PM|second|evening|| +81925|AAAAAAAAGAAEBAAA|81925|22|45|25|PM|second|evening|| +81926|AAAAAAAAHAAEBAAA|81926|22|45|26|PM|second|evening|| +81927|AAAAAAAAIAAEBAAA|81927|22|45|27|PM|second|evening|| +81928|AAAAAAAAJAAEBAAA|81928|22|45|28|PM|second|evening|| +81929|AAAAAAAAKAAEBAAA|81929|22|45|29|PM|second|evening|| +81930|AAAAAAAALAAEBAAA|81930|22|45|30|PM|second|evening|| +81931|AAAAAAAAMAAEBAAA|81931|22|45|31|PM|second|evening|| +81932|AAAAAAAANAAEBAAA|81932|22|45|32|PM|second|evening|| +81933|AAAAAAAAOAAEBAAA|81933|22|45|33|PM|second|evening|| +81934|AAAAAAAAPAAEBAAA|81934|22|45|34|PM|second|evening|| +81935|AAAAAAAAABAEBAAA|81935|22|45|35|PM|second|evening|| +81936|AAAAAAAABBAEBAAA|81936|22|45|36|PM|second|evening|| +81937|AAAAAAAACBAEBAAA|81937|22|45|37|PM|second|evening|| +81938|AAAAAAAADBAEBAAA|81938|22|45|38|PM|second|evening|| +81939|AAAAAAAAEBAEBAAA|81939|22|45|39|PM|second|evening|| +81940|AAAAAAAAFBAEBAAA|81940|22|45|40|PM|second|evening|| +81941|AAAAAAAAGBAEBAAA|81941|22|45|41|PM|second|evening|| +81942|AAAAAAAAHBAEBAAA|81942|22|45|42|PM|second|evening|| +81943|AAAAAAAAIBAEBAAA|81943|22|45|43|PM|second|evening|| +81944|AAAAAAAAJBAEBAAA|81944|22|45|44|PM|second|evening|| +81945|AAAAAAAAKBAEBAAA|81945|22|45|45|PM|second|evening|| +81946|AAAAAAAALBAEBAAA|81946|22|45|46|PM|second|evening|| +81947|AAAAAAAAMBAEBAAA|81947|22|45|47|PM|second|evening|| +81948|AAAAAAAANBAEBAAA|81948|22|45|48|PM|second|evening|| +81949|AAAAAAAAOBAEBAAA|81949|22|45|49|PM|second|evening|| +81950|AAAAAAAAPBAEBAAA|81950|22|45|50|PM|second|evening|| +81951|AAAAAAAAACAEBAAA|81951|22|45|51|PM|second|evening|| +81952|AAAAAAAABCAEBAAA|81952|22|45|52|PM|second|evening|| +81953|AAAAAAAACCAEBAAA|81953|22|45|53|PM|second|evening|| +81954|AAAAAAAADCAEBAAA|81954|22|45|54|PM|second|evening|| +81955|AAAAAAAAECAEBAAA|81955|22|45|55|PM|second|evening|| +81956|AAAAAAAAFCAEBAAA|81956|22|45|56|PM|second|evening|| +81957|AAAAAAAAGCAEBAAA|81957|22|45|57|PM|second|evening|| +81958|AAAAAAAAHCAEBAAA|81958|22|45|58|PM|second|evening|| +81959|AAAAAAAAICAEBAAA|81959|22|45|59|PM|second|evening|| +81960|AAAAAAAAJCAEBAAA|81960|22|46|0|PM|second|evening|| +81961|AAAAAAAAKCAEBAAA|81961|22|46|1|PM|second|evening|| +81962|AAAAAAAALCAEBAAA|81962|22|46|2|PM|second|evening|| +81963|AAAAAAAAMCAEBAAA|81963|22|46|3|PM|second|evening|| +81964|AAAAAAAANCAEBAAA|81964|22|46|4|PM|second|evening|| +81965|AAAAAAAAOCAEBAAA|81965|22|46|5|PM|second|evening|| +81966|AAAAAAAAPCAEBAAA|81966|22|46|6|PM|second|evening|| +81967|AAAAAAAAADAEBAAA|81967|22|46|7|PM|second|evening|| +81968|AAAAAAAABDAEBAAA|81968|22|46|8|PM|second|evening|| +81969|AAAAAAAACDAEBAAA|81969|22|46|9|PM|second|evening|| +81970|AAAAAAAADDAEBAAA|81970|22|46|10|PM|second|evening|| +81971|AAAAAAAAEDAEBAAA|81971|22|46|11|PM|second|evening|| +81972|AAAAAAAAFDAEBAAA|81972|22|46|12|PM|second|evening|| +81973|AAAAAAAAGDAEBAAA|81973|22|46|13|PM|second|evening|| +81974|AAAAAAAAHDAEBAAA|81974|22|46|14|PM|second|evening|| +81975|AAAAAAAAIDAEBAAA|81975|22|46|15|PM|second|evening|| +81976|AAAAAAAAJDAEBAAA|81976|22|46|16|PM|second|evening|| +81977|AAAAAAAAKDAEBAAA|81977|22|46|17|PM|second|evening|| +81978|AAAAAAAALDAEBAAA|81978|22|46|18|PM|second|evening|| +81979|AAAAAAAAMDAEBAAA|81979|22|46|19|PM|second|evening|| +81980|AAAAAAAANDAEBAAA|81980|22|46|20|PM|second|evening|| +81981|AAAAAAAAODAEBAAA|81981|22|46|21|PM|second|evening|| +81982|AAAAAAAAPDAEBAAA|81982|22|46|22|PM|second|evening|| +81983|AAAAAAAAAEAEBAAA|81983|22|46|23|PM|second|evening|| +81984|AAAAAAAABEAEBAAA|81984|22|46|24|PM|second|evening|| +81985|AAAAAAAACEAEBAAA|81985|22|46|25|PM|second|evening|| +81986|AAAAAAAADEAEBAAA|81986|22|46|26|PM|second|evening|| +81987|AAAAAAAAEEAEBAAA|81987|22|46|27|PM|second|evening|| +81988|AAAAAAAAFEAEBAAA|81988|22|46|28|PM|second|evening|| +81989|AAAAAAAAGEAEBAAA|81989|22|46|29|PM|second|evening|| +81990|AAAAAAAAHEAEBAAA|81990|22|46|30|PM|second|evening|| +81991|AAAAAAAAIEAEBAAA|81991|22|46|31|PM|second|evening|| +81992|AAAAAAAAJEAEBAAA|81992|22|46|32|PM|second|evening|| +81993|AAAAAAAAKEAEBAAA|81993|22|46|33|PM|second|evening|| +81994|AAAAAAAALEAEBAAA|81994|22|46|34|PM|second|evening|| +81995|AAAAAAAAMEAEBAAA|81995|22|46|35|PM|second|evening|| +81996|AAAAAAAANEAEBAAA|81996|22|46|36|PM|second|evening|| +81997|AAAAAAAAOEAEBAAA|81997|22|46|37|PM|second|evening|| +81998|AAAAAAAAPEAEBAAA|81998|22|46|38|PM|second|evening|| +81999|AAAAAAAAAFAEBAAA|81999|22|46|39|PM|second|evening|| +82000|AAAAAAAABFAEBAAA|82000|22|46|40|PM|second|evening|| +82001|AAAAAAAACFAEBAAA|82001|22|46|41|PM|second|evening|| +82002|AAAAAAAADFAEBAAA|82002|22|46|42|PM|second|evening|| +82003|AAAAAAAAEFAEBAAA|82003|22|46|43|PM|second|evening|| +82004|AAAAAAAAFFAEBAAA|82004|22|46|44|PM|second|evening|| +82005|AAAAAAAAGFAEBAAA|82005|22|46|45|PM|second|evening|| +82006|AAAAAAAAHFAEBAAA|82006|22|46|46|PM|second|evening|| +82007|AAAAAAAAIFAEBAAA|82007|22|46|47|PM|second|evening|| +82008|AAAAAAAAJFAEBAAA|82008|22|46|48|PM|second|evening|| +82009|AAAAAAAAKFAEBAAA|82009|22|46|49|PM|second|evening|| +82010|AAAAAAAALFAEBAAA|82010|22|46|50|PM|second|evening|| +82011|AAAAAAAAMFAEBAAA|82011|22|46|51|PM|second|evening|| +82012|AAAAAAAANFAEBAAA|82012|22|46|52|PM|second|evening|| +82013|AAAAAAAAOFAEBAAA|82013|22|46|53|PM|second|evening|| +82014|AAAAAAAAPFAEBAAA|82014|22|46|54|PM|second|evening|| +82015|AAAAAAAAAGAEBAAA|82015|22|46|55|PM|second|evening|| +82016|AAAAAAAABGAEBAAA|82016|22|46|56|PM|second|evening|| +82017|AAAAAAAACGAEBAAA|82017|22|46|57|PM|second|evening|| +82018|AAAAAAAADGAEBAAA|82018|22|46|58|PM|second|evening|| +82019|AAAAAAAAEGAEBAAA|82019|22|46|59|PM|second|evening|| +82020|AAAAAAAAFGAEBAAA|82020|22|47|0|PM|second|evening|| +82021|AAAAAAAAGGAEBAAA|82021|22|47|1|PM|second|evening|| +82022|AAAAAAAAHGAEBAAA|82022|22|47|2|PM|second|evening|| +82023|AAAAAAAAIGAEBAAA|82023|22|47|3|PM|second|evening|| +82024|AAAAAAAAJGAEBAAA|82024|22|47|4|PM|second|evening|| +82025|AAAAAAAAKGAEBAAA|82025|22|47|5|PM|second|evening|| +82026|AAAAAAAALGAEBAAA|82026|22|47|6|PM|second|evening|| +82027|AAAAAAAAMGAEBAAA|82027|22|47|7|PM|second|evening|| +82028|AAAAAAAANGAEBAAA|82028|22|47|8|PM|second|evening|| +82029|AAAAAAAAOGAEBAAA|82029|22|47|9|PM|second|evening|| +82030|AAAAAAAAPGAEBAAA|82030|22|47|10|PM|second|evening|| +82031|AAAAAAAAAHAEBAAA|82031|22|47|11|PM|second|evening|| +82032|AAAAAAAABHAEBAAA|82032|22|47|12|PM|second|evening|| +82033|AAAAAAAACHAEBAAA|82033|22|47|13|PM|second|evening|| +82034|AAAAAAAADHAEBAAA|82034|22|47|14|PM|second|evening|| +82035|AAAAAAAAEHAEBAAA|82035|22|47|15|PM|second|evening|| +82036|AAAAAAAAFHAEBAAA|82036|22|47|16|PM|second|evening|| +82037|AAAAAAAAGHAEBAAA|82037|22|47|17|PM|second|evening|| +82038|AAAAAAAAHHAEBAAA|82038|22|47|18|PM|second|evening|| +82039|AAAAAAAAIHAEBAAA|82039|22|47|19|PM|second|evening|| +82040|AAAAAAAAJHAEBAAA|82040|22|47|20|PM|second|evening|| +82041|AAAAAAAAKHAEBAAA|82041|22|47|21|PM|second|evening|| +82042|AAAAAAAALHAEBAAA|82042|22|47|22|PM|second|evening|| +82043|AAAAAAAAMHAEBAAA|82043|22|47|23|PM|second|evening|| +82044|AAAAAAAANHAEBAAA|82044|22|47|24|PM|second|evening|| +82045|AAAAAAAAOHAEBAAA|82045|22|47|25|PM|second|evening|| +82046|AAAAAAAAPHAEBAAA|82046|22|47|26|PM|second|evening|| +82047|AAAAAAAAAIAEBAAA|82047|22|47|27|PM|second|evening|| +82048|AAAAAAAABIAEBAAA|82048|22|47|28|PM|second|evening|| +82049|AAAAAAAACIAEBAAA|82049|22|47|29|PM|second|evening|| +82050|AAAAAAAADIAEBAAA|82050|22|47|30|PM|second|evening|| +82051|AAAAAAAAEIAEBAAA|82051|22|47|31|PM|second|evening|| +82052|AAAAAAAAFIAEBAAA|82052|22|47|32|PM|second|evening|| +82053|AAAAAAAAGIAEBAAA|82053|22|47|33|PM|second|evening|| +82054|AAAAAAAAHIAEBAAA|82054|22|47|34|PM|second|evening|| +82055|AAAAAAAAIIAEBAAA|82055|22|47|35|PM|second|evening|| +82056|AAAAAAAAJIAEBAAA|82056|22|47|36|PM|second|evening|| +82057|AAAAAAAAKIAEBAAA|82057|22|47|37|PM|second|evening|| +82058|AAAAAAAALIAEBAAA|82058|22|47|38|PM|second|evening|| +82059|AAAAAAAAMIAEBAAA|82059|22|47|39|PM|second|evening|| +82060|AAAAAAAANIAEBAAA|82060|22|47|40|PM|second|evening|| +82061|AAAAAAAAOIAEBAAA|82061|22|47|41|PM|second|evening|| +82062|AAAAAAAAPIAEBAAA|82062|22|47|42|PM|second|evening|| +82063|AAAAAAAAAJAEBAAA|82063|22|47|43|PM|second|evening|| +82064|AAAAAAAABJAEBAAA|82064|22|47|44|PM|second|evening|| +82065|AAAAAAAACJAEBAAA|82065|22|47|45|PM|second|evening|| +82066|AAAAAAAADJAEBAAA|82066|22|47|46|PM|second|evening|| +82067|AAAAAAAAEJAEBAAA|82067|22|47|47|PM|second|evening|| +82068|AAAAAAAAFJAEBAAA|82068|22|47|48|PM|second|evening|| +82069|AAAAAAAAGJAEBAAA|82069|22|47|49|PM|second|evening|| +82070|AAAAAAAAHJAEBAAA|82070|22|47|50|PM|second|evening|| +82071|AAAAAAAAIJAEBAAA|82071|22|47|51|PM|second|evening|| +82072|AAAAAAAAJJAEBAAA|82072|22|47|52|PM|second|evening|| +82073|AAAAAAAAKJAEBAAA|82073|22|47|53|PM|second|evening|| +82074|AAAAAAAALJAEBAAA|82074|22|47|54|PM|second|evening|| +82075|AAAAAAAAMJAEBAAA|82075|22|47|55|PM|second|evening|| +82076|AAAAAAAANJAEBAAA|82076|22|47|56|PM|second|evening|| +82077|AAAAAAAAOJAEBAAA|82077|22|47|57|PM|second|evening|| +82078|AAAAAAAAPJAEBAAA|82078|22|47|58|PM|second|evening|| +82079|AAAAAAAAAKAEBAAA|82079|22|47|59|PM|second|evening|| +82080|AAAAAAAABKAEBAAA|82080|22|48|0|PM|second|evening|| +82081|AAAAAAAACKAEBAAA|82081|22|48|1|PM|second|evening|| +82082|AAAAAAAADKAEBAAA|82082|22|48|2|PM|second|evening|| +82083|AAAAAAAAEKAEBAAA|82083|22|48|3|PM|second|evening|| +82084|AAAAAAAAFKAEBAAA|82084|22|48|4|PM|second|evening|| +82085|AAAAAAAAGKAEBAAA|82085|22|48|5|PM|second|evening|| +82086|AAAAAAAAHKAEBAAA|82086|22|48|6|PM|second|evening|| +82087|AAAAAAAAIKAEBAAA|82087|22|48|7|PM|second|evening|| +82088|AAAAAAAAJKAEBAAA|82088|22|48|8|PM|second|evening|| +82089|AAAAAAAAKKAEBAAA|82089|22|48|9|PM|second|evening|| +82090|AAAAAAAALKAEBAAA|82090|22|48|10|PM|second|evening|| +82091|AAAAAAAAMKAEBAAA|82091|22|48|11|PM|second|evening|| +82092|AAAAAAAANKAEBAAA|82092|22|48|12|PM|second|evening|| +82093|AAAAAAAAOKAEBAAA|82093|22|48|13|PM|second|evening|| +82094|AAAAAAAAPKAEBAAA|82094|22|48|14|PM|second|evening|| +82095|AAAAAAAAALAEBAAA|82095|22|48|15|PM|second|evening|| +82096|AAAAAAAABLAEBAAA|82096|22|48|16|PM|second|evening|| +82097|AAAAAAAACLAEBAAA|82097|22|48|17|PM|second|evening|| +82098|AAAAAAAADLAEBAAA|82098|22|48|18|PM|second|evening|| +82099|AAAAAAAAELAEBAAA|82099|22|48|19|PM|second|evening|| +82100|AAAAAAAAFLAEBAAA|82100|22|48|20|PM|second|evening|| +82101|AAAAAAAAGLAEBAAA|82101|22|48|21|PM|second|evening|| +82102|AAAAAAAAHLAEBAAA|82102|22|48|22|PM|second|evening|| +82103|AAAAAAAAILAEBAAA|82103|22|48|23|PM|second|evening|| +82104|AAAAAAAAJLAEBAAA|82104|22|48|24|PM|second|evening|| +82105|AAAAAAAAKLAEBAAA|82105|22|48|25|PM|second|evening|| +82106|AAAAAAAALLAEBAAA|82106|22|48|26|PM|second|evening|| +82107|AAAAAAAAMLAEBAAA|82107|22|48|27|PM|second|evening|| +82108|AAAAAAAANLAEBAAA|82108|22|48|28|PM|second|evening|| +82109|AAAAAAAAOLAEBAAA|82109|22|48|29|PM|second|evening|| +82110|AAAAAAAAPLAEBAAA|82110|22|48|30|PM|second|evening|| +82111|AAAAAAAAAMAEBAAA|82111|22|48|31|PM|second|evening|| +82112|AAAAAAAABMAEBAAA|82112|22|48|32|PM|second|evening|| +82113|AAAAAAAACMAEBAAA|82113|22|48|33|PM|second|evening|| +82114|AAAAAAAADMAEBAAA|82114|22|48|34|PM|second|evening|| +82115|AAAAAAAAEMAEBAAA|82115|22|48|35|PM|second|evening|| +82116|AAAAAAAAFMAEBAAA|82116|22|48|36|PM|second|evening|| +82117|AAAAAAAAGMAEBAAA|82117|22|48|37|PM|second|evening|| +82118|AAAAAAAAHMAEBAAA|82118|22|48|38|PM|second|evening|| +82119|AAAAAAAAIMAEBAAA|82119|22|48|39|PM|second|evening|| +82120|AAAAAAAAJMAEBAAA|82120|22|48|40|PM|second|evening|| +82121|AAAAAAAAKMAEBAAA|82121|22|48|41|PM|second|evening|| +82122|AAAAAAAALMAEBAAA|82122|22|48|42|PM|second|evening|| +82123|AAAAAAAAMMAEBAAA|82123|22|48|43|PM|second|evening|| +82124|AAAAAAAANMAEBAAA|82124|22|48|44|PM|second|evening|| +82125|AAAAAAAAOMAEBAAA|82125|22|48|45|PM|second|evening|| +82126|AAAAAAAAPMAEBAAA|82126|22|48|46|PM|second|evening|| +82127|AAAAAAAAANAEBAAA|82127|22|48|47|PM|second|evening|| +82128|AAAAAAAABNAEBAAA|82128|22|48|48|PM|second|evening|| +82129|AAAAAAAACNAEBAAA|82129|22|48|49|PM|second|evening|| +82130|AAAAAAAADNAEBAAA|82130|22|48|50|PM|second|evening|| +82131|AAAAAAAAENAEBAAA|82131|22|48|51|PM|second|evening|| +82132|AAAAAAAAFNAEBAAA|82132|22|48|52|PM|second|evening|| +82133|AAAAAAAAGNAEBAAA|82133|22|48|53|PM|second|evening|| +82134|AAAAAAAAHNAEBAAA|82134|22|48|54|PM|second|evening|| +82135|AAAAAAAAINAEBAAA|82135|22|48|55|PM|second|evening|| +82136|AAAAAAAAJNAEBAAA|82136|22|48|56|PM|second|evening|| +82137|AAAAAAAAKNAEBAAA|82137|22|48|57|PM|second|evening|| +82138|AAAAAAAALNAEBAAA|82138|22|48|58|PM|second|evening|| +82139|AAAAAAAAMNAEBAAA|82139|22|48|59|PM|second|evening|| +82140|AAAAAAAANNAEBAAA|82140|22|49|0|PM|second|evening|| +82141|AAAAAAAAONAEBAAA|82141|22|49|1|PM|second|evening|| +82142|AAAAAAAAPNAEBAAA|82142|22|49|2|PM|second|evening|| +82143|AAAAAAAAAOAEBAAA|82143|22|49|3|PM|second|evening|| +82144|AAAAAAAABOAEBAAA|82144|22|49|4|PM|second|evening|| +82145|AAAAAAAACOAEBAAA|82145|22|49|5|PM|second|evening|| +82146|AAAAAAAADOAEBAAA|82146|22|49|6|PM|second|evening|| +82147|AAAAAAAAEOAEBAAA|82147|22|49|7|PM|second|evening|| +82148|AAAAAAAAFOAEBAAA|82148|22|49|8|PM|second|evening|| +82149|AAAAAAAAGOAEBAAA|82149|22|49|9|PM|second|evening|| +82150|AAAAAAAAHOAEBAAA|82150|22|49|10|PM|second|evening|| +82151|AAAAAAAAIOAEBAAA|82151|22|49|11|PM|second|evening|| +82152|AAAAAAAAJOAEBAAA|82152|22|49|12|PM|second|evening|| +82153|AAAAAAAAKOAEBAAA|82153|22|49|13|PM|second|evening|| +82154|AAAAAAAALOAEBAAA|82154|22|49|14|PM|second|evening|| +82155|AAAAAAAAMOAEBAAA|82155|22|49|15|PM|second|evening|| +82156|AAAAAAAANOAEBAAA|82156|22|49|16|PM|second|evening|| +82157|AAAAAAAAOOAEBAAA|82157|22|49|17|PM|second|evening|| +82158|AAAAAAAAPOAEBAAA|82158|22|49|18|PM|second|evening|| +82159|AAAAAAAAAPAEBAAA|82159|22|49|19|PM|second|evening|| +82160|AAAAAAAABPAEBAAA|82160|22|49|20|PM|second|evening|| +82161|AAAAAAAACPAEBAAA|82161|22|49|21|PM|second|evening|| +82162|AAAAAAAADPAEBAAA|82162|22|49|22|PM|second|evening|| +82163|AAAAAAAAEPAEBAAA|82163|22|49|23|PM|second|evening|| +82164|AAAAAAAAFPAEBAAA|82164|22|49|24|PM|second|evening|| +82165|AAAAAAAAGPAEBAAA|82165|22|49|25|PM|second|evening|| +82166|AAAAAAAAHPAEBAAA|82166|22|49|26|PM|second|evening|| +82167|AAAAAAAAIPAEBAAA|82167|22|49|27|PM|second|evening|| +82168|AAAAAAAAJPAEBAAA|82168|22|49|28|PM|second|evening|| +82169|AAAAAAAAKPAEBAAA|82169|22|49|29|PM|second|evening|| +82170|AAAAAAAALPAEBAAA|82170|22|49|30|PM|second|evening|| +82171|AAAAAAAAMPAEBAAA|82171|22|49|31|PM|second|evening|| +82172|AAAAAAAANPAEBAAA|82172|22|49|32|PM|second|evening|| +82173|AAAAAAAAOPAEBAAA|82173|22|49|33|PM|second|evening|| +82174|AAAAAAAAPPAEBAAA|82174|22|49|34|PM|second|evening|| +82175|AAAAAAAAAABEBAAA|82175|22|49|35|PM|second|evening|| +82176|AAAAAAAABABEBAAA|82176|22|49|36|PM|second|evening|| +82177|AAAAAAAACABEBAAA|82177|22|49|37|PM|second|evening|| +82178|AAAAAAAADABEBAAA|82178|22|49|38|PM|second|evening|| +82179|AAAAAAAAEABEBAAA|82179|22|49|39|PM|second|evening|| +82180|AAAAAAAAFABEBAAA|82180|22|49|40|PM|second|evening|| +82181|AAAAAAAAGABEBAAA|82181|22|49|41|PM|second|evening|| +82182|AAAAAAAAHABEBAAA|82182|22|49|42|PM|second|evening|| +82183|AAAAAAAAIABEBAAA|82183|22|49|43|PM|second|evening|| +82184|AAAAAAAAJABEBAAA|82184|22|49|44|PM|second|evening|| +82185|AAAAAAAAKABEBAAA|82185|22|49|45|PM|second|evening|| +82186|AAAAAAAALABEBAAA|82186|22|49|46|PM|second|evening|| +82187|AAAAAAAAMABEBAAA|82187|22|49|47|PM|second|evening|| +82188|AAAAAAAANABEBAAA|82188|22|49|48|PM|second|evening|| +82189|AAAAAAAAOABEBAAA|82189|22|49|49|PM|second|evening|| +82190|AAAAAAAAPABEBAAA|82190|22|49|50|PM|second|evening|| +82191|AAAAAAAAABBEBAAA|82191|22|49|51|PM|second|evening|| +82192|AAAAAAAABBBEBAAA|82192|22|49|52|PM|second|evening|| +82193|AAAAAAAACBBEBAAA|82193|22|49|53|PM|second|evening|| +82194|AAAAAAAADBBEBAAA|82194|22|49|54|PM|second|evening|| +82195|AAAAAAAAEBBEBAAA|82195|22|49|55|PM|second|evening|| +82196|AAAAAAAAFBBEBAAA|82196|22|49|56|PM|second|evening|| +82197|AAAAAAAAGBBEBAAA|82197|22|49|57|PM|second|evening|| +82198|AAAAAAAAHBBEBAAA|82198|22|49|58|PM|second|evening|| +82199|AAAAAAAAIBBEBAAA|82199|22|49|59|PM|second|evening|| +82200|AAAAAAAAJBBEBAAA|82200|22|50|0|PM|second|evening|| +82201|AAAAAAAAKBBEBAAA|82201|22|50|1|PM|second|evening|| +82202|AAAAAAAALBBEBAAA|82202|22|50|2|PM|second|evening|| +82203|AAAAAAAAMBBEBAAA|82203|22|50|3|PM|second|evening|| +82204|AAAAAAAANBBEBAAA|82204|22|50|4|PM|second|evening|| +82205|AAAAAAAAOBBEBAAA|82205|22|50|5|PM|second|evening|| +82206|AAAAAAAAPBBEBAAA|82206|22|50|6|PM|second|evening|| +82207|AAAAAAAAACBEBAAA|82207|22|50|7|PM|second|evening|| +82208|AAAAAAAABCBEBAAA|82208|22|50|8|PM|second|evening|| +82209|AAAAAAAACCBEBAAA|82209|22|50|9|PM|second|evening|| +82210|AAAAAAAADCBEBAAA|82210|22|50|10|PM|second|evening|| +82211|AAAAAAAAECBEBAAA|82211|22|50|11|PM|second|evening|| +82212|AAAAAAAAFCBEBAAA|82212|22|50|12|PM|second|evening|| +82213|AAAAAAAAGCBEBAAA|82213|22|50|13|PM|second|evening|| +82214|AAAAAAAAHCBEBAAA|82214|22|50|14|PM|second|evening|| +82215|AAAAAAAAICBEBAAA|82215|22|50|15|PM|second|evening|| +82216|AAAAAAAAJCBEBAAA|82216|22|50|16|PM|second|evening|| +82217|AAAAAAAAKCBEBAAA|82217|22|50|17|PM|second|evening|| +82218|AAAAAAAALCBEBAAA|82218|22|50|18|PM|second|evening|| +82219|AAAAAAAAMCBEBAAA|82219|22|50|19|PM|second|evening|| +82220|AAAAAAAANCBEBAAA|82220|22|50|20|PM|second|evening|| +82221|AAAAAAAAOCBEBAAA|82221|22|50|21|PM|second|evening|| +82222|AAAAAAAAPCBEBAAA|82222|22|50|22|PM|second|evening|| +82223|AAAAAAAAADBEBAAA|82223|22|50|23|PM|second|evening|| +82224|AAAAAAAABDBEBAAA|82224|22|50|24|PM|second|evening|| +82225|AAAAAAAACDBEBAAA|82225|22|50|25|PM|second|evening|| +82226|AAAAAAAADDBEBAAA|82226|22|50|26|PM|second|evening|| +82227|AAAAAAAAEDBEBAAA|82227|22|50|27|PM|second|evening|| +82228|AAAAAAAAFDBEBAAA|82228|22|50|28|PM|second|evening|| +82229|AAAAAAAAGDBEBAAA|82229|22|50|29|PM|second|evening|| +82230|AAAAAAAAHDBEBAAA|82230|22|50|30|PM|second|evening|| +82231|AAAAAAAAIDBEBAAA|82231|22|50|31|PM|second|evening|| +82232|AAAAAAAAJDBEBAAA|82232|22|50|32|PM|second|evening|| +82233|AAAAAAAAKDBEBAAA|82233|22|50|33|PM|second|evening|| +82234|AAAAAAAALDBEBAAA|82234|22|50|34|PM|second|evening|| +82235|AAAAAAAAMDBEBAAA|82235|22|50|35|PM|second|evening|| +82236|AAAAAAAANDBEBAAA|82236|22|50|36|PM|second|evening|| +82237|AAAAAAAAODBEBAAA|82237|22|50|37|PM|second|evening|| +82238|AAAAAAAAPDBEBAAA|82238|22|50|38|PM|second|evening|| +82239|AAAAAAAAAEBEBAAA|82239|22|50|39|PM|second|evening|| +82240|AAAAAAAABEBEBAAA|82240|22|50|40|PM|second|evening|| +82241|AAAAAAAACEBEBAAA|82241|22|50|41|PM|second|evening|| +82242|AAAAAAAADEBEBAAA|82242|22|50|42|PM|second|evening|| +82243|AAAAAAAAEEBEBAAA|82243|22|50|43|PM|second|evening|| +82244|AAAAAAAAFEBEBAAA|82244|22|50|44|PM|second|evening|| +82245|AAAAAAAAGEBEBAAA|82245|22|50|45|PM|second|evening|| +82246|AAAAAAAAHEBEBAAA|82246|22|50|46|PM|second|evening|| +82247|AAAAAAAAIEBEBAAA|82247|22|50|47|PM|second|evening|| +82248|AAAAAAAAJEBEBAAA|82248|22|50|48|PM|second|evening|| +82249|AAAAAAAAKEBEBAAA|82249|22|50|49|PM|second|evening|| +82250|AAAAAAAALEBEBAAA|82250|22|50|50|PM|second|evening|| +82251|AAAAAAAAMEBEBAAA|82251|22|50|51|PM|second|evening|| +82252|AAAAAAAANEBEBAAA|82252|22|50|52|PM|second|evening|| +82253|AAAAAAAAOEBEBAAA|82253|22|50|53|PM|second|evening|| +82254|AAAAAAAAPEBEBAAA|82254|22|50|54|PM|second|evening|| +82255|AAAAAAAAAFBEBAAA|82255|22|50|55|PM|second|evening|| +82256|AAAAAAAABFBEBAAA|82256|22|50|56|PM|second|evening|| +82257|AAAAAAAACFBEBAAA|82257|22|50|57|PM|second|evening|| +82258|AAAAAAAADFBEBAAA|82258|22|50|58|PM|second|evening|| +82259|AAAAAAAAEFBEBAAA|82259|22|50|59|PM|second|evening|| +82260|AAAAAAAAFFBEBAAA|82260|22|51|0|PM|second|evening|| +82261|AAAAAAAAGFBEBAAA|82261|22|51|1|PM|second|evening|| +82262|AAAAAAAAHFBEBAAA|82262|22|51|2|PM|second|evening|| +82263|AAAAAAAAIFBEBAAA|82263|22|51|3|PM|second|evening|| +82264|AAAAAAAAJFBEBAAA|82264|22|51|4|PM|second|evening|| +82265|AAAAAAAAKFBEBAAA|82265|22|51|5|PM|second|evening|| +82266|AAAAAAAALFBEBAAA|82266|22|51|6|PM|second|evening|| +82267|AAAAAAAAMFBEBAAA|82267|22|51|7|PM|second|evening|| +82268|AAAAAAAANFBEBAAA|82268|22|51|8|PM|second|evening|| +82269|AAAAAAAAOFBEBAAA|82269|22|51|9|PM|second|evening|| +82270|AAAAAAAAPFBEBAAA|82270|22|51|10|PM|second|evening|| +82271|AAAAAAAAAGBEBAAA|82271|22|51|11|PM|second|evening|| +82272|AAAAAAAABGBEBAAA|82272|22|51|12|PM|second|evening|| +82273|AAAAAAAACGBEBAAA|82273|22|51|13|PM|second|evening|| +82274|AAAAAAAADGBEBAAA|82274|22|51|14|PM|second|evening|| +82275|AAAAAAAAEGBEBAAA|82275|22|51|15|PM|second|evening|| +82276|AAAAAAAAFGBEBAAA|82276|22|51|16|PM|second|evening|| +82277|AAAAAAAAGGBEBAAA|82277|22|51|17|PM|second|evening|| +82278|AAAAAAAAHGBEBAAA|82278|22|51|18|PM|second|evening|| +82279|AAAAAAAAIGBEBAAA|82279|22|51|19|PM|second|evening|| +82280|AAAAAAAAJGBEBAAA|82280|22|51|20|PM|second|evening|| +82281|AAAAAAAAKGBEBAAA|82281|22|51|21|PM|second|evening|| +82282|AAAAAAAALGBEBAAA|82282|22|51|22|PM|second|evening|| +82283|AAAAAAAAMGBEBAAA|82283|22|51|23|PM|second|evening|| +82284|AAAAAAAANGBEBAAA|82284|22|51|24|PM|second|evening|| +82285|AAAAAAAAOGBEBAAA|82285|22|51|25|PM|second|evening|| +82286|AAAAAAAAPGBEBAAA|82286|22|51|26|PM|second|evening|| +82287|AAAAAAAAAHBEBAAA|82287|22|51|27|PM|second|evening|| +82288|AAAAAAAABHBEBAAA|82288|22|51|28|PM|second|evening|| +82289|AAAAAAAACHBEBAAA|82289|22|51|29|PM|second|evening|| +82290|AAAAAAAADHBEBAAA|82290|22|51|30|PM|second|evening|| +82291|AAAAAAAAEHBEBAAA|82291|22|51|31|PM|second|evening|| +82292|AAAAAAAAFHBEBAAA|82292|22|51|32|PM|second|evening|| +82293|AAAAAAAAGHBEBAAA|82293|22|51|33|PM|second|evening|| +82294|AAAAAAAAHHBEBAAA|82294|22|51|34|PM|second|evening|| +82295|AAAAAAAAIHBEBAAA|82295|22|51|35|PM|second|evening|| +82296|AAAAAAAAJHBEBAAA|82296|22|51|36|PM|second|evening|| +82297|AAAAAAAAKHBEBAAA|82297|22|51|37|PM|second|evening|| +82298|AAAAAAAALHBEBAAA|82298|22|51|38|PM|second|evening|| +82299|AAAAAAAAMHBEBAAA|82299|22|51|39|PM|second|evening|| +82300|AAAAAAAANHBEBAAA|82300|22|51|40|PM|second|evening|| +82301|AAAAAAAAOHBEBAAA|82301|22|51|41|PM|second|evening|| +82302|AAAAAAAAPHBEBAAA|82302|22|51|42|PM|second|evening|| +82303|AAAAAAAAAIBEBAAA|82303|22|51|43|PM|second|evening|| +82304|AAAAAAAABIBEBAAA|82304|22|51|44|PM|second|evening|| +82305|AAAAAAAACIBEBAAA|82305|22|51|45|PM|second|evening|| +82306|AAAAAAAADIBEBAAA|82306|22|51|46|PM|second|evening|| +82307|AAAAAAAAEIBEBAAA|82307|22|51|47|PM|second|evening|| +82308|AAAAAAAAFIBEBAAA|82308|22|51|48|PM|second|evening|| +82309|AAAAAAAAGIBEBAAA|82309|22|51|49|PM|second|evening|| +82310|AAAAAAAAHIBEBAAA|82310|22|51|50|PM|second|evening|| +82311|AAAAAAAAIIBEBAAA|82311|22|51|51|PM|second|evening|| +82312|AAAAAAAAJIBEBAAA|82312|22|51|52|PM|second|evening|| +82313|AAAAAAAAKIBEBAAA|82313|22|51|53|PM|second|evening|| +82314|AAAAAAAALIBEBAAA|82314|22|51|54|PM|second|evening|| +82315|AAAAAAAAMIBEBAAA|82315|22|51|55|PM|second|evening|| +82316|AAAAAAAANIBEBAAA|82316|22|51|56|PM|second|evening|| +82317|AAAAAAAAOIBEBAAA|82317|22|51|57|PM|second|evening|| +82318|AAAAAAAAPIBEBAAA|82318|22|51|58|PM|second|evening|| +82319|AAAAAAAAAJBEBAAA|82319|22|51|59|PM|second|evening|| +82320|AAAAAAAABJBEBAAA|82320|22|52|0|PM|second|evening|| +82321|AAAAAAAACJBEBAAA|82321|22|52|1|PM|second|evening|| +82322|AAAAAAAADJBEBAAA|82322|22|52|2|PM|second|evening|| +82323|AAAAAAAAEJBEBAAA|82323|22|52|3|PM|second|evening|| +82324|AAAAAAAAFJBEBAAA|82324|22|52|4|PM|second|evening|| +82325|AAAAAAAAGJBEBAAA|82325|22|52|5|PM|second|evening|| +82326|AAAAAAAAHJBEBAAA|82326|22|52|6|PM|second|evening|| +82327|AAAAAAAAIJBEBAAA|82327|22|52|7|PM|second|evening|| +82328|AAAAAAAAJJBEBAAA|82328|22|52|8|PM|second|evening|| +82329|AAAAAAAAKJBEBAAA|82329|22|52|9|PM|second|evening|| +82330|AAAAAAAALJBEBAAA|82330|22|52|10|PM|second|evening|| +82331|AAAAAAAAMJBEBAAA|82331|22|52|11|PM|second|evening|| +82332|AAAAAAAANJBEBAAA|82332|22|52|12|PM|second|evening|| +82333|AAAAAAAAOJBEBAAA|82333|22|52|13|PM|second|evening|| +82334|AAAAAAAAPJBEBAAA|82334|22|52|14|PM|second|evening|| +82335|AAAAAAAAAKBEBAAA|82335|22|52|15|PM|second|evening|| +82336|AAAAAAAABKBEBAAA|82336|22|52|16|PM|second|evening|| +82337|AAAAAAAACKBEBAAA|82337|22|52|17|PM|second|evening|| +82338|AAAAAAAADKBEBAAA|82338|22|52|18|PM|second|evening|| +82339|AAAAAAAAEKBEBAAA|82339|22|52|19|PM|second|evening|| +82340|AAAAAAAAFKBEBAAA|82340|22|52|20|PM|second|evening|| +82341|AAAAAAAAGKBEBAAA|82341|22|52|21|PM|second|evening|| +82342|AAAAAAAAHKBEBAAA|82342|22|52|22|PM|second|evening|| +82343|AAAAAAAAIKBEBAAA|82343|22|52|23|PM|second|evening|| +82344|AAAAAAAAJKBEBAAA|82344|22|52|24|PM|second|evening|| +82345|AAAAAAAAKKBEBAAA|82345|22|52|25|PM|second|evening|| +82346|AAAAAAAALKBEBAAA|82346|22|52|26|PM|second|evening|| +82347|AAAAAAAAMKBEBAAA|82347|22|52|27|PM|second|evening|| +82348|AAAAAAAANKBEBAAA|82348|22|52|28|PM|second|evening|| +82349|AAAAAAAAOKBEBAAA|82349|22|52|29|PM|second|evening|| +82350|AAAAAAAAPKBEBAAA|82350|22|52|30|PM|second|evening|| +82351|AAAAAAAAALBEBAAA|82351|22|52|31|PM|second|evening|| +82352|AAAAAAAABLBEBAAA|82352|22|52|32|PM|second|evening|| +82353|AAAAAAAACLBEBAAA|82353|22|52|33|PM|second|evening|| +82354|AAAAAAAADLBEBAAA|82354|22|52|34|PM|second|evening|| +82355|AAAAAAAAELBEBAAA|82355|22|52|35|PM|second|evening|| +82356|AAAAAAAAFLBEBAAA|82356|22|52|36|PM|second|evening|| +82357|AAAAAAAAGLBEBAAA|82357|22|52|37|PM|second|evening|| +82358|AAAAAAAAHLBEBAAA|82358|22|52|38|PM|second|evening|| +82359|AAAAAAAAILBEBAAA|82359|22|52|39|PM|second|evening|| +82360|AAAAAAAAJLBEBAAA|82360|22|52|40|PM|second|evening|| +82361|AAAAAAAAKLBEBAAA|82361|22|52|41|PM|second|evening|| +82362|AAAAAAAALLBEBAAA|82362|22|52|42|PM|second|evening|| +82363|AAAAAAAAMLBEBAAA|82363|22|52|43|PM|second|evening|| +82364|AAAAAAAANLBEBAAA|82364|22|52|44|PM|second|evening|| +82365|AAAAAAAAOLBEBAAA|82365|22|52|45|PM|second|evening|| +82366|AAAAAAAAPLBEBAAA|82366|22|52|46|PM|second|evening|| +82367|AAAAAAAAAMBEBAAA|82367|22|52|47|PM|second|evening|| +82368|AAAAAAAABMBEBAAA|82368|22|52|48|PM|second|evening|| +82369|AAAAAAAACMBEBAAA|82369|22|52|49|PM|second|evening|| +82370|AAAAAAAADMBEBAAA|82370|22|52|50|PM|second|evening|| +82371|AAAAAAAAEMBEBAAA|82371|22|52|51|PM|second|evening|| +82372|AAAAAAAAFMBEBAAA|82372|22|52|52|PM|second|evening|| +82373|AAAAAAAAGMBEBAAA|82373|22|52|53|PM|second|evening|| +82374|AAAAAAAAHMBEBAAA|82374|22|52|54|PM|second|evening|| +82375|AAAAAAAAIMBEBAAA|82375|22|52|55|PM|second|evening|| +82376|AAAAAAAAJMBEBAAA|82376|22|52|56|PM|second|evening|| +82377|AAAAAAAAKMBEBAAA|82377|22|52|57|PM|second|evening|| +82378|AAAAAAAALMBEBAAA|82378|22|52|58|PM|second|evening|| +82379|AAAAAAAAMMBEBAAA|82379|22|52|59|PM|second|evening|| +82380|AAAAAAAANMBEBAAA|82380|22|53|0|PM|second|evening|| +82381|AAAAAAAAOMBEBAAA|82381|22|53|1|PM|second|evening|| +82382|AAAAAAAAPMBEBAAA|82382|22|53|2|PM|second|evening|| +82383|AAAAAAAAANBEBAAA|82383|22|53|3|PM|second|evening|| +82384|AAAAAAAABNBEBAAA|82384|22|53|4|PM|second|evening|| +82385|AAAAAAAACNBEBAAA|82385|22|53|5|PM|second|evening|| +82386|AAAAAAAADNBEBAAA|82386|22|53|6|PM|second|evening|| +82387|AAAAAAAAENBEBAAA|82387|22|53|7|PM|second|evening|| +82388|AAAAAAAAFNBEBAAA|82388|22|53|8|PM|second|evening|| +82389|AAAAAAAAGNBEBAAA|82389|22|53|9|PM|second|evening|| +82390|AAAAAAAAHNBEBAAA|82390|22|53|10|PM|second|evening|| +82391|AAAAAAAAINBEBAAA|82391|22|53|11|PM|second|evening|| +82392|AAAAAAAAJNBEBAAA|82392|22|53|12|PM|second|evening|| +82393|AAAAAAAAKNBEBAAA|82393|22|53|13|PM|second|evening|| +82394|AAAAAAAALNBEBAAA|82394|22|53|14|PM|second|evening|| +82395|AAAAAAAAMNBEBAAA|82395|22|53|15|PM|second|evening|| +82396|AAAAAAAANNBEBAAA|82396|22|53|16|PM|second|evening|| +82397|AAAAAAAAONBEBAAA|82397|22|53|17|PM|second|evening|| +82398|AAAAAAAAPNBEBAAA|82398|22|53|18|PM|second|evening|| +82399|AAAAAAAAAOBEBAAA|82399|22|53|19|PM|second|evening|| +82400|AAAAAAAABOBEBAAA|82400|22|53|20|PM|second|evening|| +82401|AAAAAAAACOBEBAAA|82401|22|53|21|PM|second|evening|| +82402|AAAAAAAADOBEBAAA|82402|22|53|22|PM|second|evening|| +82403|AAAAAAAAEOBEBAAA|82403|22|53|23|PM|second|evening|| +82404|AAAAAAAAFOBEBAAA|82404|22|53|24|PM|second|evening|| +82405|AAAAAAAAGOBEBAAA|82405|22|53|25|PM|second|evening|| +82406|AAAAAAAAHOBEBAAA|82406|22|53|26|PM|second|evening|| +82407|AAAAAAAAIOBEBAAA|82407|22|53|27|PM|second|evening|| +82408|AAAAAAAAJOBEBAAA|82408|22|53|28|PM|second|evening|| +82409|AAAAAAAAKOBEBAAA|82409|22|53|29|PM|second|evening|| +82410|AAAAAAAALOBEBAAA|82410|22|53|30|PM|second|evening|| +82411|AAAAAAAAMOBEBAAA|82411|22|53|31|PM|second|evening|| +82412|AAAAAAAANOBEBAAA|82412|22|53|32|PM|second|evening|| +82413|AAAAAAAAOOBEBAAA|82413|22|53|33|PM|second|evening|| +82414|AAAAAAAAPOBEBAAA|82414|22|53|34|PM|second|evening|| +82415|AAAAAAAAAPBEBAAA|82415|22|53|35|PM|second|evening|| +82416|AAAAAAAABPBEBAAA|82416|22|53|36|PM|second|evening|| +82417|AAAAAAAACPBEBAAA|82417|22|53|37|PM|second|evening|| +82418|AAAAAAAADPBEBAAA|82418|22|53|38|PM|second|evening|| +82419|AAAAAAAAEPBEBAAA|82419|22|53|39|PM|second|evening|| +82420|AAAAAAAAFPBEBAAA|82420|22|53|40|PM|second|evening|| +82421|AAAAAAAAGPBEBAAA|82421|22|53|41|PM|second|evening|| +82422|AAAAAAAAHPBEBAAA|82422|22|53|42|PM|second|evening|| +82423|AAAAAAAAIPBEBAAA|82423|22|53|43|PM|second|evening|| +82424|AAAAAAAAJPBEBAAA|82424|22|53|44|PM|second|evening|| +82425|AAAAAAAAKPBEBAAA|82425|22|53|45|PM|second|evening|| +82426|AAAAAAAALPBEBAAA|82426|22|53|46|PM|second|evening|| +82427|AAAAAAAAMPBEBAAA|82427|22|53|47|PM|second|evening|| +82428|AAAAAAAANPBEBAAA|82428|22|53|48|PM|second|evening|| +82429|AAAAAAAAOPBEBAAA|82429|22|53|49|PM|second|evening|| +82430|AAAAAAAAPPBEBAAA|82430|22|53|50|PM|second|evening|| +82431|AAAAAAAAAACEBAAA|82431|22|53|51|PM|second|evening|| +82432|AAAAAAAABACEBAAA|82432|22|53|52|PM|second|evening|| +82433|AAAAAAAACACEBAAA|82433|22|53|53|PM|second|evening|| +82434|AAAAAAAADACEBAAA|82434|22|53|54|PM|second|evening|| +82435|AAAAAAAAEACEBAAA|82435|22|53|55|PM|second|evening|| +82436|AAAAAAAAFACEBAAA|82436|22|53|56|PM|second|evening|| +82437|AAAAAAAAGACEBAAA|82437|22|53|57|PM|second|evening|| +82438|AAAAAAAAHACEBAAA|82438|22|53|58|PM|second|evening|| +82439|AAAAAAAAIACEBAAA|82439|22|53|59|PM|second|evening|| +82440|AAAAAAAAJACEBAAA|82440|22|54|0|PM|second|evening|| +82441|AAAAAAAAKACEBAAA|82441|22|54|1|PM|second|evening|| +82442|AAAAAAAALACEBAAA|82442|22|54|2|PM|second|evening|| +82443|AAAAAAAAMACEBAAA|82443|22|54|3|PM|second|evening|| +82444|AAAAAAAANACEBAAA|82444|22|54|4|PM|second|evening|| +82445|AAAAAAAAOACEBAAA|82445|22|54|5|PM|second|evening|| +82446|AAAAAAAAPACEBAAA|82446|22|54|6|PM|second|evening|| +82447|AAAAAAAAABCEBAAA|82447|22|54|7|PM|second|evening|| +82448|AAAAAAAABBCEBAAA|82448|22|54|8|PM|second|evening|| +82449|AAAAAAAACBCEBAAA|82449|22|54|9|PM|second|evening|| +82450|AAAAAAAADBCEBAAA|82450|22|54|10|PM|second|evening|| +82451|AAAAAAAAEBCEBAAA|82451|22|54|11|PM|second|evening|| +82452|AAAAAAAAFBCEBAAA|82452|22|54|12|PM|second|evening|| +82453|AAAAAAAAGBCEBAAA|82453|22|54|13|PM|second|evening|| +82454|AAAAAAAAHBCEBAAA|82454|22|54|14|PM|second|evening|| +82455|AAAAAAAAIBCEBAAA|82455|22|54|15|PM|second|evening|| +82456|AAAAAAAAJBCEBAAA|82456|22|54|16|PM|second|evening|| +82457|AAAAAAAAKBCEBAAA|82457|22|54|17|PM|second|evening|| +82458|AAAAAAAALBCEBAAA|82458|22|54|18|PM|second|evening|| +82459|AAAAAAAAMBCEBAAA|82459|22|54|19|PM|second|evening|| +82460|AAAAAAAANBCEBAAA|82460|22|54|20|PM|second|evening|| +82461|AAAAAAAAOBCEBAAA|82461|22|54|21|PM|second|evening|| +82462|AAAAAAAAPBCEBAAA|82462|22|54|22|PM|second|evening|| +82463|AAAAAAAAACCEBAAA|82463|22|54|23|PM|second|evening|| +82464|AAAAAAAABCCEBAAA|82464|22|54|24|PM|second|evening|| +82465|AAAAAAAACCCEBAAA|82465|22|54|25|PM|second|evening|| +82466|AAAAAAAADCCEBAAA|82466|22|54|26|PM|second|evening|| +82467|AAAAAAAAECCEBAAA|82467|22|54|27|PM|second|evening|| +82468|AAAAAAAAFCCEBAAA|82468|22|54|28|PM|second|evening|| +82469|AAAAAAAAGCCEBAAA|82469|22|54|29|PM|second|evening|| +82470|AAAAAAAAHCCEBAAA|82470|22|54|30|PM|second|evening|| +82471|AAAAAAAAICCEBAAA|82471|22|54|31|PM|second|evening|| +82472|AAAAAAAAJCCEBAAA|82472|22|54|32|PM|second|evening|| +82473|AAAAAAAAKCCEBAAA|82473|22|54|33|PM|second|evening|| +82474|AAAAAAAALCCEBAAA|82474|22|54|34|PM|second|evening|| +82475|AAAAAAAAMCCEBAAA|82475|22|54|35|PM|second|evening|| +82476|AAAAAAAANCCEBAAA|82476|22|54|36|PM|second|evening|| +82477|AAAAAAAAOCCEBAAA|82477|22|54|37|PM|second|evening|| +82478|AAAAAAAAPCCEBAAA|82478|22|54|38|PM|second|evening|| +82479|AAAAAAAAADCEBAAA|82479|22|54|39|PM|second|evening|| +82480|AAAAAAAABDCEBAAA|82480|22|54|40|PM|second|evening|| +82481|AAAAAAAACDCEBAAA|82481|22|54|41|PM|second|evening|| +82482|AAAAAAAADDCEBAAA|82482|22|54|42|PM|second|evening|| +82483|AAAAAAAAEDCEBAAA|82483|22|54|43|PM|second|evening|| +82484|AAAAAAAAFDCEBAAA|82484|22|54|44|PM|second|evening|| +82485|AAAAAAAAGDCEBAAA|82485|22|54|45|PM|second|evening|| +82486|AAAAAAAAHDCEBAAA|82486|22|54|46|PM|second|evening|| +82487|AAAAAAAAIDCEBAAA|82487|22|54|47|PM|second|evening|| +82488|AAAAAAAAJDCEBAAA|82488|22|54|48|PM|second|evening|| +82489|AAAAAAAAKDCEBAAA|82489|22|54|49|PM|second|evening|| +82490|AAAAAAAALDCEBAAA|82490|22|54|50|PM|second|evening|| +82491|AAAAAAAAMDCEBAAA|82491|22|54|51|PM|second|evening|| +82492|AAAAAAAANDCEBAAA|82492|22|54|52|PM|second|evening|| +82493|AAAAAAAAODCEBAAA|82493|22|54|53|PM|second|evening|| +82494|AAAAAAAAPDCEBAAA|82494|22|54|54|PM|second|evening|| +82495|AAAAAAAAAECEBAAA|82495|22|54|55|PM|second|evening|| +82496|AAAAAAAABECEBAAA|82496|22|54|56|PM|second|evening|| +82497|AAAAAAAACECEBAAA|82497|22|54|57|PM|second|evening|| +82498|AAAAAAAADECEBAAA|82498|22|54|58|PM|second|evening|| +82499|AAAAAAAAEECEBAAA|82499|22|54|59|PM|second|evening|| +82500|AAAAAAAAFECEBAAA|82500|22|55|0|PM|second|evening|| +82501|AAAAAAAAGECEBAAA|82501|22|55|1|PM|second|evening|| +82502|AAAAAAAAHECEBAAA|82502|22|55|2|PM|second|evening|| +82503|AAAAAAAAIECEBAAA|82503|22|55|3|PM|second|evening|| +82504|AAAAAAAAJECEBAAA|82504|22|55|4|PM|second|evening|| +82505|AAAAAAAAKECEBAAA|82505|22|55|5|PM|second|evening|| +82506|AAAAAAAALECEBAAA|82506|22|55|6|PM|second|evening|| +82507|AAAAAAAAMECEBAAA|82507|22|55|7|PM|second|evening|| +82508|AAAAAAAANECEBAAA|82508|22|55|8|PM|second|evening|| +82509|AAAAAAAAOECEBAAA|82509|22|55|9|PM|second|evening|| +82510|AAAAAAAAPECEBAAA|82510|22|55|10|PM|second|evening|| +82511|AAAAAAAAAFCEBAAA|82511|22|55|11|PM|second|evening|| +82512|AAAAAAAABFCEBAAA|82512|22|55|12|PM|second|evening|| +82513|AAAAAAAACFCEBAAA|82513|22|55|13|PM|second|evening|| +82514|AAAAAAAADFCEBAAA|82514|22|55|14|PM|second|evening|| +82515|AAAAAAAAEFCEBAAA|82515|22|55|15|PM|second|evening|| +82516|AAAAAAAAFFCEBAAA|82516|22|55|16|PM|second|evening|| +82517|AAAAAAAAGFCEBAAA|82517|22|55|17|PM|second|evening|| +82518|AAAAAAAAHFCEBAAA|82518|22|55|18|PM|second|evening|| +82519|AAAAAAAAIFCEBAAA|82519|22|55|19|PM|second|evening|| +82520|AAAAAAAAJFCEBAAA|82520|22|55|20|PM|second|evening|| +82521|AAAAAAAAKFCEBAAA|82521|22|55|21|PM|second|evening|| +82522|AAAAAAAALFCEBAAA|82522|22|55|22|PM|second|evening|| +82523|AAAAAAAAMFCEBAAA|82523|22|55|23|PM|second|evening|| +82524|AAAAAAAANFCEBAAA|82524|22|55|24|PM|second|evening|| +82525|AAAAAAAAOFCEBAAA|82525|22|55|25|PM|second|evening|| +82526|AAAAAAAAPFCEBAAA|82526|22|55|26|PM|second|evening|| +82527|AAAAAAAAAGCEBAAA|82527|22|55|27|PM|second|evening|| +82528|AAAAAAAABGCEBAAA|82528|22|55|28|PM|second|evening|| +82529|AAAAAAAACGCEBAAA|82529|22|55|29|PM|second|evening|| +82530|AAAAAAAADGCEBAAA|82530|22|55|30|PM|second|evening|| +82531|AAAAAAAAEGCEBAAA|82531|22|55|31|PM|second|evening|| +82532|AAAAAAAAFGCEBAAA|82532|22|55|32|PM|second|evening|| +82533|AAAAAAAAGGCEBAAA|82533|22|55|33|PM|second|evening|| +82534|AAAAAAAAHGCEBAAA|82534|22|55|34|PM|second|evening|| +82535|AAAAAAAAIGCEBAAA|82535|22|55|35|PM|second|evening|| +82536|AAAAAAAAJGCEBAAA|82536|22|55|36|PM|second|evening|| +82537|AAAAAAAAKGCEBAAA|82537|22|55|37|PM|second|evening|| +82538|AAAAAAAALGCEBAAA|82538|22|55|38|PM|second|evening|| +82539|AAAAAAAAMGCEBAAA|82539|22|55|39|PM|second|evening|| +82540|AAAAAAAANGCEBAAA|82540|22|55|40|PM|second|evening|| +82541|AAAAAAAAOGCEBAAA|82541|22|55|41|PM|second|evening|| +82542|AAAAAAAAPGCEBAAA|82542|22|55|42|PM|second|evening|| +82543|AAAAAAAAAHCEBAAA|82543|22|55|43|PM|second|evening|| +82544|AAAAAAAABHCEBAAA|82544|22|55|44|PM|second|evening|| +82545|AAAAAAAACHCEBAAA|82545|22|55|45|PM|second|evening|| +82546|AAAAAAAADHCEBAAA|82546|22|55|46|PM|second|evening|| +82547|AAAAAAAAEHCEBAAA|82547|22|55|47|PM|second|evening|| +82548|AAAAAAAAFHCEBAAA|82548|22|55|48|PM|second|evening|| +82549|AAAAAAAAGHCEBAAA|82549|22|55|49|PM|second|evening|| +82550|AAAAAAAAHHCEBAAA|82550|22|55|50|PM|second|evening|| +82551|AAAAAAAAIHCEBAAA|82551|22|55|51|PM|second|evening|| +82552|AAAAAAAAJHCEBAAA|82552|22|55|52|PM|second|evening|| +82553|AAAAAAAAKHCEBAAA|82553|22|55|53|PM|second|evening|| +82554|AAAAAAAALHCEBAAA|82554|22|55|54|PM|second|evening|| +82555|AAAAAAAAMHCEBAAA|82555|22|55|55|PM|second|evening|| +82556|AAAAAAAANHCEBAAA|82556|22|55|56|PM|second|evening|| +82557|AAAAAAAAOHCEBAAA|82557|22|55|57|PM|second|evening|| +82558|AAAAAAAAPHCEBAAA|82558|22|55|58|PM|second|evening|| +82559|AAAAAAAAAICEBAAA|82559|22|55|59|PM|second|evening|| +82560|AAAAAAAABICEBAAA|82560|22|56|0|PM|second|evening|| +82561|AAAAAAAACICEBAAA|82561|22|56|1|PM|second|evening|| +82562|AAAAAAAADICEBAAA|82562|22|56|2|PM|second|evening|| +82563|AAAAAAAAEICEBAAA|82563|22|56|3|PM|second|evening|| +82564|AAAAAAAAFICEBAAA|82564|22|56|4|PM|second|evening|| +82565|AAAAAAAAGICEBAAA|82565|22|56|5|PM|second|evening|| +82566|AAAAAAAAHICEBAAA|82566|22|56|6|PM|second|evening|| +82567|AAAAAAAAIICEBAAA|82567|22|56|7|PM|second|evening|| +82568|AAAAAAAAJICEBAAA|82568|22|56|8|PM|second|evening|| +82569|AAAAAAAAKICEBAAA|82569|22|56|9|PM|second|evening|| +82570|AAAAAAAALICEBAAA|82570|22|56|10|PM|second|evening|| +82571|AAAAAAAAMICEBAAA|82571|22|56|11|PM|second|evening|| +82572|AAAAAAAANICEBAAA|82572|22|56|12|PM|second|evening|| +82573|AAAAAAAAOICEBAAA|82573|22|56|13|PM|second|evening|| +82574|AAAAAAAAPICEBAAA|82574|22|56|14|PM|second|evening|| +82575|AAAAAAAAAJCEBAAA|82575|22|56|15|PM|second|evening|| +82576|AAAAAAAABJCEBAAA|82576|22|56|16|PM|second|evening|| +82577|AAAAAAAACJCEBAAA|82577|22|56|17|PM|second|evening|| +82578|AAAAAAAADJCEBAAA|82578|22|56|18|PM|second|evening|| +82579|AAAAAAAAEJCEBAAA|82579|22|56|19|PM|second|evening|| +82580|AAAAAAAAFJCEBAAA|82580|22|56|20|PM|second|evening|| +82581|AAAAAAAAGJCEBAAA|82581|22|56|21|PM|second|evening|| +82582|AAAAAAAAHJCEBAAA|82582|22|56|22|PM|second|evening|| +82583|AAAAAAAAIJCEBAAA|82583|22|56|23|PM|second|evening|| +82584|AAAAAAAAJJCEBAAA|82584|22|56|24|PM|second|evening|| +82585|AAAAAAAAKJCEBAAA|82585|22|56|25|PM|second|evening|| +82586|AAAAAAAALJCEBAAA|82586|22|56|26|PM|second|evening|| +82587|AAAAAAAAMJCEBAAA|82587|22|56|27|PM|second|evening|| +82588|AAAAAAAANJCEBAAA|82588|22|56|28|PM|second|evening|| +82589|AAAAAAAAOJCEBAAA|82589|22|56|29|PM|second|evening|| +82590|AAAAAAAAPJCEBAAA|82590|22|56|30|PM|second|evening|| +82591|AAAAAAAAAKCEBAAA|82591|22|56|31|PM|second|evening|| +82592|AAAAAAAABKCEBAAA|82592|22|56|32|PM|second|evening|| +82593|AAAAAAAACKCEBAAA|82593|22|56|33|PM|second|evening|| +82594|AAAAAAAADKCEBAAA|82594|22|56|34|PM|second|evening|| +82595|AAAAAAAAEKCEBAAA|82595|22|56|35|PM|second|evening|| +82596|AAAAAAAAFKCEBAAA|82596|22|56|36|PM|second|evening|| +82597|AAAAAAAAGKCEBAAA|82597|22|56|37|PM|second|evening|| +82598|AAAAAAAAHKCEBAAA|82598|22|56|38|PM|second|evening|| +82599|AAAAAAAAIKCEBAAA|82599|22|56|39|PM|second|evening|| +82600|AAAAAAAAJKCEBAAA|82600|22|56|40|PM|second|evening|| +82601|AAAAAAAAKKCEBAAA|82601|22|56|41|PM|second|evening|| +82602|AAAAAAAALKCEBAAA|82602|22|56|42|PM|second|evening|| +82603|AAAAAAAAMKCEBAAA|82603|22|56|43|PM|second|evening|| +82604|AAAAAAAANKCEBAAA|82604|22|56|44|PM|second|evening|| +82605|AAAAAAAAOKCEBAAA|82605|22|56|45|PM|second|evening|| +82606|AAAAAAAAPKCEBAAA|82606|22|56|46|PM|second|evening|| +82607|AAAAAAAAALCEBAAA|82607|22|56|47|PM|second|evening|| +82608|AAAAAAAABLCEBAAA|82608|22|56|48|PM|second|evening|| +82609|AAAAAAAACLCEBAAA|82609|22|56|49|PM|second|evening|| +82610|AAAAAAAADLCEBAAA|82610|22|56|50|PM|second|evening|| +82611|AAAAAAAAELCEBAAA|82611|22|56|51|PM|second|evening|| +82612|AAAAAAAAFLCEBAAA|82612|22|56|52|PM|second|evening|| +82613|AAAAAAAAGLCEBAAA|82613|22|56|53|PM|second|evening|| +82614|AAAAAAAAHLCEBAAA|82614|22|56|54|PM|second|evening|| +82615|AAAAAAAAILCEBAAA|82615|22|56|55|PM|second|evening|| +82616|AAAAAAAAJLCEBAAA|82616|22|56|56|PM|second|evening|| +82617|AAAAAAAAKLCEBAAA|82617|22|56|57|PM|second|evening|| +82618|AAAAAAAALLCEBAAA|82618|22|56|58|PM|second|evening|| +82619|AAAAAAAAMLCEBAAA|82619|22|56|59|PM|second|evening|| +82620|AAAAAAAANLCEBAAA|82620|22|57|0|PM|second|evening|| +82621|AAAAAAAAOLCEBAAA|82621|22|57|1|PM|second|evening|| +82622|AAAAAAAAPLCEBAAA|82622|22|57|2|PM|second|evening|| +82623|AAAAAAAAAMCEBAAA|82623|22|57|3|PM|second|evening|| +82624|AAAAAAAABMCEBAAA|82624|22|57|4|PM|second|evening|| +82625|AAAAAAAACMCEBAAA|82625|22|57|5|PM|second|evening|| +82626|AAAAAAAADMCEBAAA|82626|22|57|6|PM|second|evening|| +82627|AAAAAAAAEMCEBAAA|82627|22|57|7|PM|second|evening|| +82628|AAAAAAAAFMCEBAAA|82628|22|57|8|PM|second|evening|| +82629|AAAAAAAAGMCEBAAA|82629|22|57|9|PM|second|evening|| +82630|AAAAAAAAHMCEBAAA|82630|22|57|10|PM|second|evening|| +82631|AAAAAAAAIMCEBAAA|82631|22|57|11|PM|second|evening|| +82632|AAAAAAAAJMCEBAAA|82632|22|57|12|PM|second|evening|| +82633|AAAAAAAAKMCEBAAA|82633|22|57|13|PM|second|evening|| +82634|AAAAAAAALMCEBAAA|82634|22|57|14|PM|second|evening|| +82635|AAAAAAAAMMCEBAAA|82635|22|57|15|PM|second|evening|| +82636|AAAAAAAANMCEBAAA|82636|22|57|16|PM|second|evening|| +82637|AAAAAAAAOMCEBAAA|82637|22|57|17|PM|second|evening|| +82638|AAAAAAAAPMCEBAAA|82638|22|57|18|PM|second|evening|| +82639|AAAAAAAAANCEBAAA|82639|22|57|19|PM|second|evening|| +82640|AAAAAAAABNCEBAAA|82640|22|57|20|PM|second|evening|| +82641|AAAAAAAACNCEBAAA|82641|22|57|21|PM|second|evening|| +82642|AAAAAAAADNCEBAAA|82642|22|57|22|PM|second|evening|| +82643|AAAAAAAAENCEBAAA|82643|22|57|23|PM|second|evening|| +82644|AAAAAAAAFNCEBAAA|82644|22|57|24|PM|second|evening|| +82645|AAAAAAAAGNCEBAAA|82645|22|57|25|PM|second|evening|| +82646|AAAAAAAAHNCEBAAA|82646|22|57|26|PM|second|evening|| +82647|AAAAAAAAINCEBAAA|82647|22|57|27|PM|second|evening|| +82648|AAAAAAAAJNCEBAAA|82648|22|57|28|PM|second|evening|| +82649|AAAAAAAAKNCEBAAA|82649|22|57|29|PM|second|evening|| +82650|AAAAAAAALNCEBAAA|82650|22|57|30|PM|second|evening|| +82651|AAAAAAAAMNCEBAAA|82651|22|57|31|PM|second|evening|| +82652|AAAAAAAANNCEBAAA|82652|22|57|32|PM|second|evening|| +82653|AAAAAAAAONCEBAAA|82653|22|57|33|PM|second|evening|| +82654|AAAAAAAAPNCEBAAA|82654|22|57|34|PM|second|evening|| +82655|AAAAAAAAAOCEBAAA|82655|22|57|35|PM|second|evening|| +82656|AAAAAAAABOCEBAAA|82656|22|57|36|PM|second|evening|| +82657|AAAAAAAACOCEBAAA|82657|22|57|37|PM|second|evening|| +82658|AAAAAAAADOCEBAAA|82658|22|57|38|PM|second|evening|| +82659|AAAAAAAAEOCEBAAA|82659|22|57|39|PM|second|evening|| +82660|AAAAAAAAFOCEBAAA|82660|22|57|40|PM|second|evening|| +82661|AAAAAAAAGOCEBAAA|82661|22|57|41|PM|second|evening|| +82662|AAAAAAAAHOCEBAAA|82662|22|57|42|PM|second|evening|| +82663|AAAAAAAAIOCEBAAA|82663|22|57|43|PM|second|evening|| +82664|AAAAAAAAJOCEBAAA|82664|22|57|44|PM|second|evening|| +82665|AAAAAAAAKOCEBAAA|82665|22|57|45|PM|second|evening|| +82666|AAAAAAAALOCEBAAA|82666|22|57|46|PM|second|evening|| +82667|AAAAAAAAMOCEBAAA|82667|22|57|47|PM|second|evening|| +82668|AAAAAAAANOCEBAAA|82668|22|57|48|PM|second|evening|| +82669|AAAAAAAAOOCEBAAA|82669|22|57|49|PM|second|evening|| +82670|AAAAAAAAPOCEBAAA|82670|22|57|50|PM|second|evening|| +82671|AAAAAAAAAPCEBAAA|82671|22|57|51|PM|second|evening|| +82672|AAAAAAAABPCEBAAA|82672|22|57|52|PM|second|evening|| +82673|AAAAAAAACPCEBAAA|82673|22|57|53|PM|second|evening|| +82674|AAAAAAAADPCEBAAA|82674|22|57|54|PM|second|evening|| +82675|AAAAAAAAEPCEBAAA|82675|22|57|55|PM|second|evening|| +82676|AAAAAAAAFPCEBAAA|82676|22|57|56|PM|second|evening|| +82677|AAAAAAAAGPCEBAAA|82677|22|57|57|PM|second|evening|| +82678|AAAAAAAAHPCEBAAA|82678|22|57|58|PM|second|evening|| +82679|AAAAAAAAIPCEBAAA|82679|22|57|59|PM|second|evening|| +82680|AAAAAAAAJPCEBAAA|82680|22|58|0|PM|second|evening|| +82681|AAAAAAAAKPCEBAAA|82681|22|58|1|PM|second|evening|| +82682|AAAAAAAALPCEBAAA|82682|22|58|2|PM|second|evening|| +82683|AAAAAAAAMPCEBAAA|82683|22|58|3|PM|second|evening|| +82684|AAAAAAAANPCEBAAA|82684|22|58|4|PM|second|evening|| +82685|AAAAAAAAOPCEBAAA|82685|22|58|5|PM|second|evening|| +82686|AAAAAAAAPPCEBAAA|82686|22|58|6|PM|second|evening|| +82687|AAAAAAAAAADEBAAA|82687|22|58|7|PM|second|evening|| +82688|AAAAAAAABADEBAAA|82688|22|58|8|PM|second|evening|| +82689|AAAAAAAACADEBAAA|82689|22|58|9|PM|second|evening|| +82690|AAAAAAAADADEBAAA|82690|22|58|10|PM|second|evening|| +82691|AAAAAAAAEADEBAAA|82691|22|58|11|PM|second|evening|| +82692|AAAAAAAAFADEBAAA|82692|22|58|12|PM|second|evening|| +82693|AAAAAAAAGADEBAAA|82693|22|58|13|PM|second|evening|| +82694|AAAAAAAAHADEBAAA|82694|22|58|14|PM|second|evening|| +82695|AAAAAAAAIADEBAAA|82695|22|58|15|PM|second|evening|| +82696|AAAAAAAAJADEBAAA|82696|22|58|16|PM|second|evening|| +82697|AAAAAAAAKADEBAAA|82697|22|58|17|PM|second|evening|| +82698|AAAAAAAALADEBAAA|82698|22|58|18|PM|second|evening|| +82699|AAAAAAAAMADEBAAA|82699|22|58|19|PM|second|evening|| +82700|AAAAAAAANADEBAAA|82700|22|58|20|PM|second|evening|| +82701|AAAAAAAAOADEBAAA|82701|22|58|21|PM|second|evening|| +82702|AAAAAAAAPADEBAAA|82702|22|58|22|PM|second|evening|| +82703|AAAAAAAAABDEBAAA|82703|22|58|23|PM|second|evening|| +82704|AAAAAAAABBDEBAAA|82704|22|58|24|PM|second|evening|| +82705|AAAAAAAACBDEBAAA|82705|22|58|25|PM|second|evening|| +82706|AAAAAAAADBDEBAAA|82706|22|58|26|PM|second|evening|| +82707|AAAAAAAAEBDEBAAA|82707|22|58|27|PM|second|evening|| +82708|AAAAAAAAFBDEBAAA|82708|22|58|28|PM|second|evening|| +82709|AAAAAAAAGBDEBAAA|82709|22|58|29|PM|second|evening|| +82710|AAAAAAAAHBDEBAAA|82710|22|58|30|PM|second|evening|| +82711|AAAAAAAAIBDEBAAA|82711|22|58|31|PM|second|evening|| +82712|AAAAAAAAJBDEBAAA|82712|22|58|32|PM|second|evening|| +82713|AAAAAAAAKBDEBAAA|82713|22|58|33|PM|second|evening|| +82714|AAAAAAAALBDEBAAA|82714|22|58|34|PM|second|evening|| +82715|AAAAAAAAMBDEBAAA|82715|22|58|35|PM|second|evening|| +82716|AAAAAAAANBDEBAAA|82716|22|58|36|PM|second|evening|| +82717|AAAAAAAAOBDEBAAA|82717|22|58|37|PM|second|evening|| +82718|AAAAAAAAPBDEBAAA|82718|22|58|38|PM|second|evening|| +82719|AAAAAAAAACDEBAAA|82719|22|58|39|PM|second|evening|| +82720|AAAAAAAABCDEBAAA|82720|22|58|40|PM|second|evening|| +82721|AAAAAAAACCDEBAAA|82721|22|58|41|PM|second|evening|| +82722|AAAAAAAADCDEBAAA|82722|22|58|42|PM|second|evening|| +82723|AAAAAAAAECDEBAAA|82723|22|58|43|PM|second|evening|| +82724|AAAAAAAAFCDEBAAA|82724|22|58|44|PM|second|evening|| +82725|AAAAAAAAGCDEBAAA|82725|22|58|45|PM|second|evening|| +82726|AAAAAAAAHCDEBAAA|82726|22|58|46|PM|second|evening|| +82727|AAAAAAAAICDEBAAA|82727|22|58|47|PM|second|evening|| +82728|AAAAAAAAJCDEBAAA|82728|22|58|48|PM|second|evening|| +82729|AAAAAAAAKCDEBAAA|82729|22|58|49|PM|second|evening|| +82730|AAAAAAAALCDEBAAA|82730|22|58|50|PM|second|evening|| +82731|AAAAAAAAMCDEBAAA|82731|22|58|51|PM|second|evening|| +82732|AAAAAAAANCDEBAAA|82732|22|58|52|PM|second|evening|| +82733|AAAAAAAAOCDEBAAA|82733|22|58|53|PM|second|evening|| +82734|AAAAAAAAPCDEBAAA|82734|22|58|54|PM|second|evening|| +82735|AAAAAAAAADDEBAAA|82735|22|58|55|PM|second|evening|| +82736|AAAAAAAABDDEBAAA|82736|22|58|56|PM|second|evening|| +82737|AAAAAAAACDDEBAAA|82737|22|58|57|PM|second|evening|| +82738|AAAAAAAADDDEBAAA|82738|22|58|58|PM|second|evening|| +82739|AAAAAAAAEDDEBAAA|82739|22|58|59|PM|second|evening|| +82740|AAAAAAAAFDDEBAAA|82740|22|59|0|PM|second|evening|| +82741|AAAAAAAAGDDEBAAA|82741|22|59|1|PM|second|evening|| +82742|AAAAAAAAHDDEBAAA|82742|22|59|2|PM|second|evening|| +82743|AAAAAAAAIDDEBAAA|82743|22|59|3|PM|second|evening|| +82744|AAAAAAAAJDDEBAAA|82744|22|59|4|PM|second|evening|| +82745|AAAAAAAAKDDEBAAA|82745|22|59|5|PM|second|evening|| +82746|AAAAAAAALDDEBAAA|82746|22|59|6|PM|second|evening|| +82747|AAAAAAAAMDDEBAAA|82747|22|59|7|PM|second|evening|| +82748|AAAAAAAANDDEBAAA|82748|22|59|8|PM|second|evening|| +82749|AAAAAAAAODDEBAAA|82749|22|59|9|PM|second|evening|| +82750|AAAAAAAAPDDEBAAA|82750|22|59|10|PM|second|evening|| +82751|AAAAAAAAAEDEBAAA|82751|22|59|11|PM|second|evening|| +82752|AAAAAAAABEDEBAAA|82752|22|59|12|PM|second|evening|| +82753|AAAAAAAACEDEBAAA|82753|22|59|13|PM|second|evening|| +82754|AAAAAAAADEDEBAAA|82754|22|59|14|PM|second|evening|| +82755|AAAAAAAAEEDEBAAA|82755|22|59|15|PM|second|evening|| +82756|AAAAAAAAFEDEBAAA|82756|22|59|16|PM|second|evening|| +82757|AAAAAAAAGEDEBAAA|82757|22|59|17|PM|second|evening|| +82758|AAAAAAAAHEDEBAAA|82758|22|59|18|PM|second|evening|| +82759|AAAAAAAAIEDEBAAA|82759|22|59|19|PM|second|evening|| +82760|AAAAAAAAJEDEBAAA|82760|22|59|20|PM|second|evening|| +82761|AAAAAAAAKEDEBAAA|82761|22|59|21|PM|second|evening|| +82762|AAAAAAAALEDEBAAA|82762|22|59|22|PM|second|evening|| +82763|AAAAAAAAMEDEBAAA|82763|22|59|23|PM|second|evening|| +82764|AAAAAAAANEDEBAAA|82764|22|59|24|PM|second|evening|| +82765|AAAAAAAAOEDEBAAA|82765|22|59|25|PM|second|evening|| +82766|AAAAAAAAPEDEBAAA|82766|22|59|26|PM|second|evening|| +82767|AAAAAAAAAFDEBAAA|82767|22|59|27|PM|second|evening|| +82768|AAAAAAAABFDEBAAA|82768|22|59|28|PM|second|evening|| +82769|AAAAAAAACFDEBAAA|82769|22|59|29|PM|second|evening|| +82770|AAAAAAAADFDEBAAA|82770|22|59|30|PM|second|evening|| +82771|AAAAAAAAEFDEBAAA|82771|22|59|31|PM|second|evening|| +82772|AAAAAAAAFFDEBAAA|82772|22|59|32|PM|second|evening|| +82773|AAAAAAAAGFDEBAAA|82773|22|59|33|PM|second|evening|| +82774|AAAAAAAAHFDEBAAA|82774|22|59|34|PM|second|evening|| +82775|AAAAAAAAIFDEBAAA|82775|22|59|35|PM|second|evening|| +82776|AAAAAAAAJFDEBAAA|82776|22|59|36|PM|second|evening|| +82777|AAAAAAAAKFDEBAAA|82777|22|59|37|PM|second|evening|| +82778|AAAAAAAALFDEBAAA|82778|22|59|38|PM|second|evening|| +82779|AAAAAAAAMFDEBAAA|82779|22|59|39|PM|second|evening|| +82780|AAAAAAAANFDEBAAA|82780|22|59|40|PM|second|evening|| +82781|AAAAAAAAOFDEBAAA|82781|22|59|41|PM|second|evening|| +82782|AAAAAAAAPFDEBAAA|82782|22|59|42|PM|second|evening|| +82783|AAAAAAAAAGDEBAAA|82783|22|59|43|PM|second|evening|| +82784|AAAAAAAABGDEBAAA|82784|22|59|44|PM|second|evening|| +82785|AAAAAAAACGDEBAAA|82785|22|59|45|PM|second|evening|| +82786|AAAAAAAADGDEBAAA|82786|22|59|46|PM|second|evening|| +82787|AAAAAAAAEGDEBAAA|82787|22|59|47|PM|second|evening|| +82788|AAAAAAAAFGDEBAAA|82788|22|59|48|PM|second|evening|| +82789|AAAAAAAAGGDEBAAA|82789|22|59|49|PM|second|evening|| +82790|AAAAAAAAHGDEBAAA|82790|22|59|50|PM|second|evening|| +82791|AAAAAAAAIGDEBAAA|82791|22|59|51|PM|second|evening|| +82792|AAAAAAAAJGDEBAAA|82792|22|59|52|PM|second|evening|| +82793|AAAAAAAAKGDEBAAA|82793|22|59|53|PM|second|evening|| +82794|AAAAAAAALGDEBAAA|82794|22|59|54|PM|second|evening|| +82795|AAAAAAAAMGDEBAAA|82795|22|59|55|PM|second|evening|| +82796|AAAAAAAANGDEBAAA|82796|22|59|56|PM|second|evening|| +82797|AAAAAAAAOGDEBAAA|82797|22|59|57|PM|second|evening|| +82798|AAAAAAAAPGDEBAAA|82798|22|59|58|PM|second|evening|| +82799|AAAAAAAAAHDEBAAA|82799|22|59|59|PM|second|evening|| +82800|AAAAAAAABHDEBAAA|82800|23|0|0|PM|third|evening|| +82801|AAAAAAAACHDEBAAA|82801|23|0|1|PM|third|evening|| +82802|AAAAAAAADHDEBAAA|82802|23|0|2|PM|third|evening|| +82803|AAAAAAAAEHDEBAAA|82803|23|0|3|PM|third|evening|| +82804|AAAAAAAAFHDEBAAA|82804|23|0|4|PM|third|evening|| +82805|AAAAAAAAGHDEBAAA|82805|23|0|5|PM|third|evening|| +82806|AAAAAAAAHHDEBAAA|82806|23|0|6|PM|third|evening|| +82807|AAAAAAAAIHDEBAAA|82807|23|0|7|PM|third|evening|| +82808|AAAAAAAAJHDEBAAA|82808|23|0|8|PM|third|evening|| +82809|AAAAAAAAKHDEBAAA|82809|23|0|9|PM|third|evening|| +82810|AAAAAAAALHDEBAAA|82810|23|0|10|PM|third|evening|| +82811|AAAAAAAAMHDEBAAA|82811|23|0|11|PM|third|evening|| +82812|AAAAAAAANHDEBAAA|82812|23|0|12|PM|third|evening|| +82813|AAAAAAAAOHDEBAAA|82813|23|0|13|PM|third|evening|| +82814|AAAAAAAAPHDEBAAA|82814|23|0|14|PM|third|evening|| +82815|AAAAAAAAAIDEBAAA|82815|23|0|15|PM|third|evening|| +82816|AAAAAAAABIDEBAAA|82816|23|0|16|PM|third|evening|| +82817|AAAAAAAACIDEBAAA|82817|23|0|17|PM|third|evening|| +82818|AAAAAAAADIDEBAAA|82818|23|0|18|PM|third|evening|| +82819|AAAAAAAAEIDEBAAA|82819|23|0|19|PM|third|evening|| +82820|AAAAAAAAFIDEBAAA|82820|23|0|20|PM|third|evening|| +82821|AAAAAAAAGIDEBAAA|82821|23|0|21|PM|third|evening|| +82822|AAAAAAAAHIDEBAAA|82822|23|0|22|PM|third|evening|| +82823|AAAAAAAAIIDEBAAA|82823|23|0|23|PM|third|evening|| +82824|AAAAAAAAJIDEBAAA|82824|23|0|24|PM|third|evening|| +82825|AAAAAAAAKIDEBAAA|82825|23|0|25|PM|third|evening|| +82826|AAAAAAAALIDEBAAA|82826|23|0|26|PM|third|evening|| +82827|AAAAAAAAMIDEBAAA|82827|23|0|27|PM|third|evening|| +82828|AAAAAAAANIDEBAAA|82828|23|0|28|PM|third|evening|| +82829|AAAAAAAAOIDEBAAA|82829|23|0|29|PM|third|evening|| +82830|AAAAAAAAPIDEBAAA|82830|23|0|30|PM|third|evening|| +82831|AAAAAAAAAJDEBAAA|82831|23|0|31|PM|third|evening|| +82832|AAAAAAAABJDEBAAA|82832|23|0|32|PM|third|evening|| +82833|AAAAAAAACJDEBAAA|82833|23|0|33|PM|third|evening|| +82834|AAAAAAAADJDEBAAA|82834|23|0|34|PM|third|evening|| +82835|AAAAAAAAEJDEBAAA|82835|23|0|35|PM|third|evening|| +82836|AAAAAAAAFJDEBAAA|82836|23|0|36|PM|third|evening|| +82837|AAAAAAAAGJDEBAAA|82837|23|0|37|PM|third|evening|| +82838|AAAAAAAAHJDEBAAA|82838|23|0|38|PM|third|evening|| +82839|AAAAAAAAIJDEBAAA|82839|23|0|39|PM|third|evening|| +82840|AAAAAAAAJJDEBAAA|82840|23|0|40|PM|third|evening|| +82841|AAAAAAAAKJDEBAAA|82841|23|0|41|PM|third|evening|| +82842|AAAAAAAALJDEBAAA|82842|23|0|42|PM|third|evening|| +82843|AAAAAAAAMJDEBAAA|82843|23|0|43|PM|third|evening|| +82844|AAAAAAAANJDEBAAA|82844|23|0|44|PM|third|evening|| +82845|AAAAAAAAOJDEBAAA|82845|23|0|45|PM|third|evening|| +82846|AAAAAAAAPJDEBAAA|82846|23|0|46|PM|third|evening|| +82847|AAAAAAAAAKDEBAAA|82847|23|0|47|PM|third|evening|| +82848|AAAAAAAABKDEBAAA|82848|23|0|48|PM|third|evening|| +82849|AAAAAAAACKDEBAAA|82849|23|0|49|PM|third|evening|| +82850|AAAAAAAADKDEBAAA|82850|23|0|50|PM|third|evening|| +82851|AAAAAAAAEKDEBAAA|82851|23|0|51|PM|third|evening|| +82852|AAAAAAAAFKDEBAAA|82852|23|0|52|PM|third|evening|| +82853|AAAAAAAAGKDEBAAA|82853|23|0|53|PM|third|evening|| +82854|AAAAAAAAHKDEBAAA|82854|23|0|54|PM|third|evening|| +82855|AAAAAAAAIKDEBAAA|82855|23|0|55|PM|third|evening|| +82856|AAAAAAAAJKDEBAAA|82856|23|0|56|PM|third|evening|| +82857|AAAAAAAAKKDEBAAA|82857|23|0|57|PM|third|evening|| +82858|AAAAAAAALKDEBAAA|82858|23|0|58|PM|third|evening|| +82859|AAAAAAAAMKDEBAAA|82859|23|0|59|PM|third|evening|| +82860|AAAAAAAANKDEBAAA|82860|23|1|0|PM|third|evening|| +82861|AAAAAAAAOKDEBAAA|82861|23|1|1|PM|third|evening|| +82862|AAAAAAAAPKDEBAAA|82862|23|1|2|PM|third|evening|| +82863|AAAAAAAAALDEBAAA|82863|23|1|3|PM|third|evening|| +82864|AAAAAAAABLDEBAAA|82864|23|1|4|PM|third|evening|| +82865|AAAAAAAACLDEBAAA|82865|23|1|5|PM|third|evening|| +82866|AAAAAAAADLDEBAAA|82866|23|1|6|PM|third|evening|| +82867|AAAAAAAAELDEBAAA|82867|23|1|7|PM|third|evening|| +82868|AAAAAAAAFLDEBAAA|82868|23|1|8|PM|third|evening|| +82869|AAAAAAAAGLDEBAAA|82869|23|1|9|PM|third|evening|| +82870|AAAAAAAAHLDEBAAA|82870|23|1|10|PM|third|evening|| +82871|AAAAAAAAILDEBAAA|82871|23|1|11|PM|third|evening|| +82872|AAAAAAAAJLDEBAAA|82872|23|1|12|PM|third|evening|| +82873|AAAAAAAAKLDEBAAA|82873|23|1|13|PM|third|evening|| +82874|AAAAAAAALLDEBAAA|82874|23|1|14|PM|third|evening|| +82875|AAAAAAAAMLDEBAAA|82875|23|1|15|PM|third|evening|| +82876|AAAAAAAANLDEBAAA|82876|23|1|16|PM|third|evening|| +82877|AAAAAAAAOLDEBAAA|82877|23|1|17|PM|third|evening|| +82878|AAAAAAAAPLDEBAAA|82878|23|1|18|PM|third|evening|| +82879|AAAAAAAAAMDEBAAA|82879|23|1|19|PM|third|evening|| +82880|AAAAAAAABMDEBAAA|82880|23|1|20|PM|third|evening|| +82881|AAAAAAAACMDEBAAA|82881|23|1|21|PM|third|evening|| +82882|AAAAAAAADMDEBAAA|82882|23|1|22|PM|third|evening|| +82883|AAAAAAAAEMDEBAAA|82883|23|1|23|PM|third|evening|| +82884|AAAAAAAAFMDEBAAA|82884|23|1|24|PM|third|evening|| +82885|AAAAAAAAGMDEBAAA|82885|23|1|25|PM|third|evening|| +82886|AAAAAAAAHMDEBAAA|82886|23|1|26|PM|third|evening|| +82887|AAAAAAAAIMDEBAAA|82887|23|1|27|PM|third|evening|| +82888|AAAAAAAAJMDEBAAA|82888|23|1|28|PM|third|evening|| +82889|AAAAAAAAKMDEBAAA|82889|23|1|29|PM|third|evening|| +82890|AAAAAAAALMDEBAAA|82890|23|1|30|PM|third|evening|| +82891|AAAAAAAAMMDEBAAA|82891|23|1|31|PM|third|evening|| +82892|AAAAAAAANMDEBAAA|82892|23|1|32|PM|third|evening|| +82893|AAAAAAAAOMDEBAAA|82893|23|1|33|PM|third|evening|| +82894|AAAAAAAAPMDEBAAA|82894|23|1|34|PM|third|evening|| +82895|AAAAAAAAANDEBAAA|82895|23|1|35|PM|third|evening|| +82896|AAAAAAAABNDEBAAA|82896|23|1|36|PM|third|evening|| +82897|AAAAAAAACNDEBAAA|82897|23|1|37|PM|third|evening|| +82898|AAAAAAAADNDEBAAA|82898|23|1|38|PM|third|evening|| +82899|AAAAAAAAENDEBAAA|82899|23|1|39|PM|third|evening|| +82900|AAAAAAAAFNDEBAAA|82900|23|1|40|PM|third|evening|| +82901|AAAAAAAAGNDEBAAA|82901|23|1|41|PM|third|evening|| +82902|AAAAAAAAHNDEBAAA|82902|23|1|42|PM|third|evening|| +82903|AAAAAAAAINDEBAAA|82903|23|1|43|PM|third|evening|| +82904|AAAAAAAAJNDEBAAA|82904|23|1|44|PM|third|evening|| +82905|AAAAAAAAKNDEBAAA|82905|23|1|45|PM|third|evening|| +82906|AAAAAAAALNDEBAAA|82906|23|1|46|PM|third|evening|| +82907|AAAAAAAAMNDEBAAA|82907|23|1|47|PM|third|evening|| +82908|AAAAAAAANNDEBAAA|82908|23|1|48|PM|third|evening|| +82909|AAAAAAAAONDEBAAA|82909|23|1|49|PM|third|evening|| +82910|AAAAAAAAPNDEBAAA|82910|23|1|50|PM|third|evening|| +82911|AAAAAAAAAODEBAAA|82911|23|1|51|PM|third|evening|| +82912|AAAAAAAABODEBAAA|82912|23|1|52|PM|third|evening|| +82913|AAAAAAAACODEBAAA|82913|23|1|53|PM|third|evening|| +82914|AAAAAAAADODEBAAA|82914|23|1|54|PM|third|evening|| +82915|AAAAAAAAEODEBAAA|82915|23|1|55|PM|third|evening|| +82916|AAAAAAAAFODEBAAA|82916|23|1|56|PM|third|evening|| +82917|AAAAAAAAGODEBAAA|82917|23|1|57|PM|third|evening|| +82918|AAAAAAAAHODEBAAA|82918|23|1|58|PM|third|evening|| +82919|AAAAAAAAIODEBAAA|82919|23|1|59|PM|third|evening|| +82920|AAAAAAAAJODEBAAA|82920|23|2|0|PM|third|evening|| +82921|AAAAAAAAKODEBAAA|82921|23|2|1|PM|third|evening|| +82922|AAAAAAAALODEBAAA|82922|23|2|2|PM|third|evening|| +82923|AAAAAAAAMODEBAAA|82923|23|2|3|PM|third|evening|| +82924|AAAAAAAANODEBAAA|82924|23|2|4|PM|third|evening|| +82925|AAAAAAAAOODEBAAA|82925|23|2|5|PM|third|evening|| +82926|AAAAAAAAPODEBAAA|82926|23|2|6|PM|third|evening|| +82927|AAAAAAAAAPDEBAAA|82927|23|2|7|PM|third|evening|| +82928|AAAAAAAABPDEBAAA|82928|23|2|8|PM|third|evening|| +82929|AAAAAAAACPDEBAAA|82929|23|2|9|PM|third|evening|| +82930|AAAAAAAADPDEBAAA|82930|23|2|10|PM|third|evening|| +82931|AAAAAAAAEPDEBAAA|82931|23|2|11|PM|third|evening|| +82932|AAAAAAAAFPDEBAAA|82932|23|2|12|PM|third|evening|| +82933|AAAAAAAAGPDEBAAA|82933|23|2|13|PM|third|evening|| +82934|AAAAAAAAHPDEBAAA|82934|23|2|14|PM|third|evening|| +82935|AAAAAAAAIPDEBAAA|82935|23|2|15|PM|third|evening|| +82936|AAAAAAAAJPDEBAAA|82936|23|2|16|PM|third|evening|| +82937|AAAAAAAAKPDEBAAA|82937|23|2|17|PM|third|evening|| +82938|AAAAAAAALPDEBAAA|82938|23|2|18|PM|third|evening|| +82939|AAAAAAAAMPDEBAAA|82939|23|2|19|PM|third|evening|| +82940|AAAAAAAANPDEBAAA|82940|23|2|20|PM|third|evening|| +82941|AAAAAAAAOPDEBAAA|82941|23|2|21|PM|third|evening|| +82942|AAAAAAAAPPDEBAAA|82942|23|2|22|PM|third|evening|| +82943|AAAAAAAAAAEEBAAA|82943|23|2|23|PM|third|evening|| +82944|AAAAAAAABAEEBAAA|82944|23|2|24|PM|third|evening|| +82945|AAAAAAAACAEEBAAA|82945|23|2|25|PM|third|evening|| +82946|AAAAAAAADAEEBAAA|82946|23|2|26|PM|third|evening|| +82947|AAAAAAAAEAEEBAAA|82947|23|2|27|PM|third|evening|| +82948|AAAAAAAAFAEEBAAA|82948|23|2|28|PM|third|evening|| +82949|AAAAAAAAGAEEBAAA|82949|23|2|29|PM|third|evening|| +82950|AAAAAAAAHAEEBAAA|82950|23|2|30|PM|third|evening|| +82951|AAAAAAAAIAEEBAAA|82951|23|2|31|PM|third|evening|| +82952|AAAAAAAAJAEEBAAA|82952|23|2|32|PM|third|evening|| +82953|AAAAAAAAKAEEBAAA|82953|23|2|33|PM|third|evening|| +82954|AAAAAAAALAEEBAAA|82954|23|2|34|PM|third|evening|| +82955|AAAAAAAAMAEEBAAA|82955|23|2|35|PM|third|evening|| +82956|AAAAAAAANAEEBAAA|82956|23|2|36|PM|third|evening|| +82957|AAAAAAAAOAEEBAAA|82957|23|2|37|PM|third|evening|| +82958|AAAAAAAAPAEEBAAA|82958|23|2|38|PM|third|evening|| +82959|AAAAAAAAABEEBAAA|82959|23|2|39|PM|third|evening|| +82960|AAAAAAAABBEEBAAA|82960|23|2|40|PM|third|evening|| +82961|AAAAAAAACBEEBAAA|82961|23|2|41|PM|third|evening|| +82962|AAAAAAAADBEEBAAA|82962|23|2|42|PM|third|evening|| +82963|AAAAAAAAEBEEBAAA|82963|23|2|43|PM|third|evening|| +82964|AAAAAAAAFBEEBAAA|82964|23|2|44|PM|third|evening|| +82965|AAAAAAAAGBEEBAAA|82965|23|2|45|PM|third|evening|| +82966|AAAAAAAAHBEEBAAA|82966|23|2|46|PM|third|evening|| +82967|AAAAAAAAIBEEBAAA|82967|23|2|47|PM|third|evening|| +82968|AAAAAAAAJBEEBAAA|82968|23|2|48|PM|third|evening|| +82969|AAAAAAAAKBEEBAAA|82969|23|2|49|PM|third|evening|| +82970|AAAAAAAALBEEBAAA|82970|23|2|50|PM|third|evening|| +82971|AAAAAAAAMBEEBAAA|82971|23|2|51|PM|third|evening|| +82972|AAAAAAAANBEEBAAA|82972|23|2|52|PM|third|evening|| +82973|AAAAAAAAOBEEBAAA|82973|23|2|53|PM|third|evening|| +82974|AAAAAAAAPBEEBAAA|82974|23|2|54|PM|third|evening|| +82975|AAAAAAAAACEEBAAA|82975|23|2|55|PM|third|evening|| +82976|AAAAAAAABCEEBAAA|82976|23|2|56|PM|third|evening|| +82977|AAAAAAAACCEEBAAA|82977|23|2|57|PM|third|evening|| +82978|AAAAAAAADCEEBAAA|82978|23|2|58|PM|third|evening|| +82979|AAAAAAAAECEEBAAA|82979|23|2|59|PM|third|evening|| +82980|AAAAAAAAFCEEBAAA|82980|23|3|0|PM|third|evening|| +82981|AAAAAAAAGCEEBAAA|82981|23|3|1|PM|third|evening|| +82982|AAAAAAAAHCEEBAAA|82982|23|3|2|PM|third|evening|| +82983|AAAAAAAAICEEBAAA|82983|23|3|3|PM|third|evening|| +82984|AAAAAAAAJCEEBAAA|82984|23|3|4|PM|third|evening|| +82985|AAAAAAAAKCEEBAAA|82985|23|3|5|PM|third|evening|| +82986|AAAAAAAALCEEBAAA|82986|23|3|6|PM|third|evening|| +82987|AAAAAAAAMCEEBAAA|82987|23|3|7|PM|third|evening|| +82988|AAAAAAAANCEEBAAA|82988|23|3|8|PM|third|evening|| +82989|AAAAAAAAOCEEBAAA|82989|23|3|9|PM|third|evening|| +82990|AAAAAAAAPCEEBAAA|82990|23|3|10|PM|third|evening|| +82991|AAAAAAAAADEEBAAA|82991|23|3|11|PM|third|evening|| +82992|AAAAAAAABDEEBAAA|82992|23|3|12|PM|third|evening|| +82993|AAAAAAAACDEEBAAA|82993|23|3|13|PM|third|evening|| +82994|AAAAAAAADDEEBAAA|82994|23|3|14|PM|third|evening|| +82995|AAAAAAAAEDEEBAAA|82995|23|3|15|PM|third|evening|| +82996|AAAAAAAAFDEEBAAA|82996|23|3|16|PM|third|evening|| +82997|AAAAAAAAGDEEBAAA|82997|23|3|17|PM|third|evening|| +82998|AAAAAAAAHDEEBAAA|82998|23|3|18|PM|third|evening|| +82999|AAAAAAAAIDEEBAAA|82999|23|3|19|PM|third|evening|| +83000|AAAAAAAAJDEEBAAA|83000|23|3|20|PM|third|evening|| +83001|AAAAAAAAKDEEBAAA|83001|23|3|21|PM|third|evening|| +83002|AAAAAAAALDEEBAAA|83002|23|3|22|PM|third|evening|| +83003|AAAAAAAAMDEEBAAA|83003|23|3|23|PM|third|evening|| +83004|AAAAAAAANDEEBAAA|83004|23|3|24|PM|third|evening|| +83005|AAAAAAAAODEEBAAA|83005|23|3|25|PM|third|evening|| +83006|AAAAAAAAPDEEBAAA|83006|23|3|26|PM|third|evening|| +83007|AAAAAAAAAEEEBAAA|83007|23|3|27|PM|third|evening|| +83008|AAAAAAAABEEEBAAA|83008|23|3|28|PM|third|evening|| +83009|AAAAAAAACEEEBAAA|83009|23|3|29|PM|third|evening|| +83010|AAAAAAAADEEEBAAA|83010|23|3|30|PM|third|evening|| +83011|AAAAAAAAEEEEBAAA|83011|23|3|31|PM|third|evening|| +83012|AAAAAAAAFEEEBAAA|83012|23|3|32|PM|third|evening|| +83013|AAAAAAAAGEEEBAAA|83013|23|3|33|PM|third|evening|| +83014|AAAAAAAAHEEEBAAA|83014|23|3|34|PM|third|evening|| +83015|AAAAAAAAIEEEBAAA|83015|23|3|35|PM|third|evening|| +83016|AAAAAAAAJEEEBAAA|83016|23|3|36|PM|third|evening|| +83017|AAAAAAAAKEEEBAAA|83017|23|3|37|PM|third|evening|| +83018|AAAAAAAALEEEBAAA|83018|23|3|38|PM|third|evening|| +83019|AAAAAAAAMEEEBAAA|83019|23|3|39|PM|third|evening|| +83020|AAAAAAAANEEEBAAA|83020|23|3|40|PM|third|evening|| +83021|AAAAAAAAOEEEBAAA|83021|23|3|41|PM|third|evening|| +83022|AAAAAAAAPEEEBAAA|83022|23|3|42|PM|third|evening|| +83023|AAAAAAAAAFEEBAAA|83023|23|3|43|PM|third|evening|| +83024|AAAAAAAABFEEBAAA|83024|23|3|44|PM|third|evening|| +83025|AAAAAAAACFEEBAAA|83025|23|3|45|PM|third|evening|| +83026|AAAAAAAADFEEBAAA|83026|23|3|46|PM|third|evening|| +83027|AAAAAAAAEFEEBAAA|83027|23|3|47|PM|third|evening|| +83028|AAAAAAAAFFEEBAAA|83028|23|3|48|PM|third|evening|| +83029|AAAAAAAAGFEEBAAA|83029|23|3|49|PM|third|evening|| +83030|AAAAAAAAHFEEBAAA|83030|23|3|50|PM|third|evening|| +83031|AAAAAAAAIFEEBAAA|83031|23|3|51|PM|third|evening|| +83032|AAAAAAAAJFEEBAAA|83032|23|3|52|PM|third|evening|| +83033|AAAAAAAAKFEEBAAA|83033|23|3|53|PM|third|evening|| +83034|AAAAAAAALFEEBAAA|83034|23|3|54|PM|third|evening|| +83035|AAAAAAAAMFEEBAAA|83035|23|3|55|PM|third|evening|| +83036|AAAAAAAANFEEBAAA|83036|23|3|56|PM|third|evening|| +83037|AAAAAAAAOFEEBAAA|83037|23|3|57|PM|third|evening|| +83038|AAAAAAAAPFEEBAAA|83038|23|3|58|PM|third|evening|| +83039|AAAAAAAAAGEEBAAA|83039|23|3|59|PM|third|evening|| +83040|AAAAAAAABGEEBAAA|83040|23|4|0|PM|third|evening|| +83041|AAAAAAAACGEEBAAA|83041|23|4|1|PM|third|evening|| +83042|AAAAAAAADGEEBAAA|83042|23|4|2|PM|third|evening|| +83043|AAAAAAAAEGEEBAAA|83043|23|4|3|PM|third|evening|| +83044|AAAAAAAAFGEEBAAA|83044|23|4|4|PM|third|evening|| +83045|AAAAAAAAGGEEBAAA|83045|23|4|5|PM|third|evening|| +83046|AAAAAAAAHGEEBAAA|83046|23|4|6|PM|third|evening|| +83047|AAAAAAAAIGEEBAAA|83047|23|4|7|PM|third|evening|| +83048|AAAAAAAAJGEEBAAA|83048|23|4|8|PM|third|evening|| +83049|AAAAAAAAKGEEBAAA|83049|23|4|9|PM|third|evening|| +83050|AAAAAAAALGEEBAAA|83050|23|4|10|PM|third|evening|| +83051|AAAAAAAAMGEEBAAA|83051|23|4|11|PM|third|evening|| +83052|AAAAAAAANGEEBAAA|83052|23|4|12|PM|third|evening|| +83053|AAAAAAAAOGEEBAAA|83053|23|4|13|PM|third|evening|| +83054|AAAAAAAAPGEEBAAA|83054|23|4|14|PM|third|evening|| +83055|AAAAAAAAAHEEBAAA|83055|23|4|15|PM|third|evening|| +83056|AAAAAAAABHEEBAAA|83056|23|4|16|PM|third|evening|| +83057|AAAAAAAACHEEBAAA|83057|23|4|17|PM|third|evening|| +83058|AAAAAAAADHEEBAAA|83058|23|4|18|PM|third|evening|| +83059|AAAAAAAAEHEEBAAA|83059|23|4|19|PM|third|evening|| +83060|AAAAAAAAFHEEBAAA|83060|23|4|20|PM|third|evening|| +83061|AAAAAAAAGHEEBAAA|83061|23|4|21|PM|third|evening|| +83062|AAAAAAAAHHEEBAAA|83062|23|4|22|PM|third|evening|| +83063|AAAAAAAAIHEEBAAA|83063|23|4|23|PM|third|evening|| +83064|AAAAAAAAJHEEBAAA|83064|23|4|24|PM|third|evening|| +83065|AAAAAAAAKHEEBAAA|83065|23|4|25|PM|third|evening|| +83066|AAAAAAAALHEEBAAA|83066|23|4|26|PM|third|evening|| +83067|AAAAAAAAMHEEBAAA|83067|23|4|27|PM|third|evening|| +83068|AAAAAAAANHEEBAAA|83068|23|4|28|PM|third|evening|| +83069|AAAAAAAAOHEEBAAA|83069|23|4|29|PM|third|evening|| +83070|AAAAAAAAPHEEBAAA|83070|23|4|30|PM|third|evening|| +83071|AAAAAAAAAIEEBAAA|83071|23|4|31|PM|third|evening|| +83072|AAAAAAAABIEEBAAA|83072|23|4|32|PM|third|evening|| +83073|AAAAAAAACIEEBAAA|83073|23|4|33|PM|third|evening|| +83074|AAAAAAAADIEEBAAA|83074|23|4|34|PM|third|evening|| +83075|AAAAAAAAEIEEBAAA|83075|23|4|35|PM|third|evening|| +83076|AAAAAAAAFIEEBAAA|83076|23|4|36|PM|third|evening|| +83077|AAAAAAAAGIEEBAAA|83077|23|4|37|PM|third|evening|| +83078|AAAAAAAAHIEEBAAA|83078|23|4|38|PM|third|evening|| +83079|AAAAAAAAIIEEBAAA|83079|23|4|39|PM|third|evening|| +83080|AAAAAAAAJIEEBAAA|83080|23|4|40|PM|third|evening|| +83081|AAAAAAAAKIEEBAAA|83081|23|4|41|PM|third|evening|| +83082|AAAAAAAALIEEBAAA|83082|23|4|42|PM|third|evening|| +83083|AAAAAAAAMIEEBAAA|83083|23|4|43|PM|third|evening|| +83084|AAAAAAAANIEEBAAA|83084|23|4|44|PM|third|evening|| +83085|AAAAAAAAOIEEBAAA|83085|23|4|45|PM|third|evening|| +83086|AAAAAAAAPIEEBAAA|83086|23|4|46|PM|third|evening|| +83087|AAAAAAAAAJEEBAAA|83087|23|4|47|PM|third|evening|| +83088|AAAAAAAABJEEBAAA|83088|23|4|48|PM|third|evening|| +83089|AAAAAAAACJEEBAAA|83089|23|4|49|PM|third|evening|| +83090|AAAAAAAADJEEBAAA|83090|23|4|50|PM|third|evening|| +83091|AAAAAAAAEJEEBAAA|83091|23|4|51|PM|third|evening|| +83092|AAAAAAAAFJEEBAAA|83092|23|4|52|PM|third|evening|| +83093|AAAAAAAAGJEEBAAA|83093|23|4|53|PM|third|evening|| +83094|AAAAAAAAHJEEBAAA|83094|23|4|54|PM|third|evening|| +83095|AAAAAAAAIJEEBAAA|83095|23|4|55|PM|third|evening|| +83096|AAAAAAAAJJEEBAAA|83096|23|4|56|PM|third|evening|| +83097|AAAAAAAAKJEEBAAA|83097|23|4|57|PM|third|evening|| +83098|AAAAAAAALJEEBAAA|83098|23|4|58|PM|third|evening|| +83099|AAAAAAAAMJEEBAAA|83099|23|4|59|PM|third|evening|| +83100|AAAAAAAANJEEBAAA|83100|23|5|0|PM|third|evening|| +83101|AAAAAAAAOJEEBAAA|83101|23|5|1|PM|third|evening|| +83102|AAAAAAAAPJEEBAAA|83102|23|5|2|PM|third|evening|| +83103|AAAAAAAAAKEEBAAA|83103|23|5|3|PM|third|evening|| +83104|AAAAAAAABKEEBAAA|83104|23|5|4|PM|third|evening|| +83105|AAAAAAAACKEEBAAA|83105|23|5|5|PM|third|evening|| +83106|AAAAAAAADKEEBAAA|83106|23|5|6|PM|third|evening|| +83107|AAAAAAAAEKEEBAAA|83107|23|5|7|PM|third|evening|| +83108|AAAAAAAAFKEEBAAA|83108|23|5|8|PM|third|evening|| +83109|AAAAAAAAGKEEBAAA|83109|23|5|9|PM|third|evening|| +83110|AAAAAAAAHKEEBAAA|83110|23|5|10|PM|third|evening|| +83111|AAAAAAAAIKEEBAAA|83111|23|5|11|PM|third|evening|| +83112|AAAAAAAAJKEEBAAA|83112|23|5|12|PM|third|evening|| +83113|AAAAAAAAKKEEBAAA|83113|23|5|13|PM|third|evening|| +83114|AAAAAAAALKEEBAAA|83114|23|5|14|PM|third|evening|| +83115|AAAAAAAAMKEEBAAA|83115|23|5|15|PM|third|evening|| +83116|AAAAAAAANKEEBAAA|83116|23|5|16|PM|third|evening|| +83117|AAAAAAAAOKEEBAAA|83117|23|5|17|PM|third|evening|| +83118|AAAAAAAAPKEEBAAA|83118|23|5|18|PM|third|evening|| +83119|AAAAAAAAALEEBAAA|83119|23|5|19|PM|third|evening|| +83120|AAAAAAAABLEEBAAA|83120|23|5|20|PM|third|evening|| +83121|AAAAAAAACLEEBAAA|83121|23|5|21|PM|third|evening|| +83122|AAAAAAAADLEEBAAA|83122|23|5|22|PM|third|evening|| +83123|AAAAAAAAELEEBAAA|83123|23|5|23|PM|third|evening|| +83124|AAAAAAAAFLEEBAAA|83124|23|5|24|PM|third|evening|| +83125|AAAAAAAAGLEEBAAA|83125|23|5|25|PM|third|evening|| +83126|AAAAAAAAHLEEBAAA|83126|23|5|26|PM|third|evening|| +83127|AAAAAAAAILEEBAAA|83127|23|5|27|PM|third|evening|| +83128|AAAAAAAAJLEEBAAA|83128|23|5|28|PM|third|evening|| +83129|AAAAAAAAKLEEBAAA|83129|23|5|29|PM|third|evening|| +83130|AAAAAAAALLEEBAAA|83130|23|5|30|PM|third|evening|| +83131|AAAAAAAAMLEEBAAA|83131|23|5|31|PM|third|evening|| +83132|AAAAAAAANLEEBAAA|83132|23|5|32|PM|third|evening|| +83133|AAAAAAAAOLEEBAAA|83133|23|5|33|PM|third|evening|| +83134|AAAAAAAAPLEEBAAA|83134|23|5|34|PM|third|evening|| +83135|AAAAAAAAAMEEBAAA|83135|23|5|35|PM|third|evening|| +83136|AAAAAAAABMEEBAAA|83136|23|5|36|PM|third|evening|| +83137|AAAAAAAACMEEBAAA|83137|23|5|37|PM|third|evening|| +83138|AAAAAAAADMEEBAAA|83138|23|5|38|PM|third|evening|| +83139|AAAAAAAAEMEEBAAA|83139|23|5|39|PM|third|evening|| +83140|AAAAAAAAFMEEBAAA|83140|23|5|40|PM|third|evening|| +83141|AAAAAAAAGMEEBAAA|83141|23|5|41|PM|third|evening|| +83142|AAAAAAAAHMEEBAAA|83142|23|5|42|PM|third|evening|| +83143|AAAAAAAAIMEEBAAA|83143|23|5|43|PM|third|evening|| +83144|AAAAAAAAJMEEBAAA|83144|23|5|44|PM|third|evening|| +83145|AAAAAAAAKMEEBAAA|83145|23|5|45|PM|third|evening|| +83146|AAAAAAAALMEEBAAA|83146|23|5|46|PM|third|evening|| +83147|AAAAAAAAMMEEBAAA|83147|23|5|47|PM|third|evening|| +83148|AAAAAAAANMEEBAAA|83148|23|5|48|PM|third|evening|| +83149|AAAAAAAAOMEEBAAA|83149|23|5|49|PM|third|evening|| +83150|AAAAAAAAPMEEBAAA|83150|23|5|50|PM|third|evening|| +83151|AAAAAAAAANEEBAAA|83151|23|5|51|PM|third|evening|| +83152|AAAAAAAABNEEBAAA|83152|23|5|52|PM|third|evening|| +83153|AAAAAAAACNEEBAAA|83153|23|5|53|PM|third|evening|| +83154|AAAAAAAADNEEBAAA|83154|23|5|54|PM|third|evening|| +83155|AAAAAAAAENEEBAAA|83155|23|5|55|PM|third|evening|| +83156|AAAAAAAAFNEEBAAA|83156|23|5|56|PM|third|evening|| +83157|AAAAAAAAGNEEBAAA|83157|23|5|57|PM|third|evening|| +83158|AAAAAAAAHNEEBAAA|83158|23|5|58|PM|third|evening|| +83159|AAAAAAAAINEEBAAA|83159|23|5|59|PM|third|evening|| +83160|AAAAAAAAJNEEBAAA|83160|23|6|0|PM|third|evening|| +83161|AAAAAAAAKNEEBAAA|83161|23|6|1|PM|third|evening|| +83162|AAAAAAAALNEEBAAA|83162|23|6|2|PM|third|evening|| +83163|AAAAAAAAMNEEBAAA|83163|23|6|3|PM|third|evening|| +83164|AAAAAAAANNEEBAAA|83164|23|6|4|PM|third|evening|| +83165|AAAAAAAAONEEBAAA|83165|23|6|5|PM|third|evening|| +83166|AAAAAAAAPNEEBAAA|83166|23|6|6|PM|third|evening|| +83167|AAAAAAAAAOEEBAAA|83167|23|6|7|PM|third|evening|| +83168|AAAAAAAABOEEBAAA|83168|23|6|8|PM|third|evening|| +83169|AAAAAAAACOEEBAAA|83169|23|6|9|PM|third|evening|| +83170|AAAAAAAADOEEBAAA|83170|23|6|10|PM|third|evening|| +83171|AAAAAAAAEOEEBAAA|83171|23|6|11|PM|third|evening|| +83172|AAAAAAAAFOEEBAAA|83172|23|6|12|PM|third|evening|| +83173|AAAAAAAAGOEEBAAA|83173|23|6|13|PM|third|evening|| +83174|AAAAAAAAHOEEBAAA|83174|23|6|14|PM|third|evening|| +83175|AAAAAAAAIOEEBAAA|83175|23|6|15|PM|third|evening|| +83176|AAAAAAAAJOEEBAAA|83176|23|6|16|PM|third|evening|| +83177|AAAAAAAAKOEEBAAA|83177|23|6|17|PM|third|evening|| +83178|AAAAAAAALOEEBAAA|83178|23|6|18|PM|third|evening|| +83179|AAAAAAAAMOEEBAAA|83179|23|6|19|PM|third|evening|| +83180|AAAAAAAANOEEBAAA|83180|23|6|20|PM|third|evening|| +83181|AAAAAAAAOOEEBAAA|83181|23|6|21|PM|third|evening|| +83182|AAAAAAAAPOEEBAAA|83182|23|6|22|PM|third|evening|| +83183|AAAAAAAAAPEEBAAA|83183|23|6|23|PM|third|evening|| +83184|AAAAAAAABPEEBAAA|83184|23|6|24|PM|third|evening|| +83185|AAAAAAAACPEEBAAA|83185|23|6|25|PM|third|evening|| +83186|AAAAAAAADPEEBAAA|83186|23|6|26|PM|third|evening|| +83187|AAAAAAAAEPEEBAAA|83187|23|6|27|PM|third|evening|| +83188|AAAAAAAAFPEEBAAA|83188|23|6|28|PM|third|evening|| +83189|AAAAAAAAGPEEBAAA|83189|23|6|29|PM|third|evening|| +83190|AAAAAAAAHPEEBAAA|83190|23|6|30|PM|third|evening|| +83191|AAAAAAAAIPEEBAAA|83191|23|6|31|PM|third|evening|| +83192|AAAAAAAAJPEEBAAA|83192|23|6|32|PM|third|evening|| +83193|AAAAAAAAKPEEBAAA|83193|23|6|33|PM|third|evening|| +83194|AAAAAAAALPEEBAAA|83194|23|6|34|PM|third|evening|| +83195|AAAAAAAAMPEEBAAA|83195|23|6|35|PM|third|evening|| +83196|AAAAAAAANPEEBAAA|83196|23|6|36|PM|third|evening|| +83197|AAAAAAAAOPEEBAAA|83197|23|6|37|PM|third|evening|| +83198|AAAAAAAAPPEEBAAA|83198|23|6|38|PM|third|evening|| +83199|AAAAAAAAAAFEBAAA|83199|23|6|39|PM|third|evening|| +83200|AAAAAAAABAFEBAAA|83200|23|6|40|PM|third|evening|| +83201|AAAAAAAACAFEBAAA|83201|23|6|41|PM|third|evening|| +83202|AAAAAAAADAFEBAAA|83202|23|6|42|PM|third|evening|| +83203|AAAAAAAAEAFEBAAA|83203|23|6|43|PM|third|evening|| +83204|AAAAAAAAFAFEBAAA|83204|23|6|44|PM|third|evening|| +83205|AAAAAAAAGAFEBAAA|83205|23|6|45|PM|third|evening|| +83206|AAAAAAAAHAFEBAAA|83206|23|6|46|PM|third|evening|| +83207|AAAAAAAAIAFEBAAA|83207|23|6|47|PM|third|evening|| +83208|AAAAAAAAJAFEBAAA|83208|23|6|48|PM|third|evening|| +83209|AAAAAAAAKAFEBAAA|83209|23|6|49|PM|third|evening|| +83210|AAAAAAAALAFEBAAA|83210|23|6|50|PM|third|evening|| +83211|AAAAAAAAMAFEBAAA|83211|23|6|51|PM|third|evening|| +83212|AAAAAAAANAFEBAAA|83212|23|6|52|PM|third|evening|| +83213|AAAAAAAAOAFEBAAA|83213|23|6|53|PM|third|evening|| +83214|AAAAAAAAPAFEBAAA|83214|23|6|54|PM|third|evening|| +83215|AAAAAAAAABFEBAAA|83215|23|6|55|PM|third|evening|| +83216|AAAAAAAABBFEBAAA|83216|23|6|56|PM|third|evening|| +83217|AAAAAAAACBFEBAAA|83217|23|6|57|PM|third|evening|| +83218|AAAAAAAADBFEBAAA|83218|23|6|58|PM|third|evening|| +83219|AAAAAAAAEBFEBAAA|83219|23|6|59|PM|third|evening|| +83220|AAAAAAAAFBFEBAAA|83220|23|7|0|PM|third|evening|| +83221|AAAAAAAAGBFEBAAA|83221|23|7|1|PM|third|evening|| +83222|AAAAAAAAHBFEBAAA|83222|23|7|2|PM|third|evening|| +83223|AAAAAAAAIBFEBAAA|83223|23|7|3|PM|third|evening|| +83224|AAAAAAAAJBFEBAAA|83224|23|7|4|PM|third|evening|| +83225|AAAAAAAAKBFEBAAA|83225|23|7|5|PM|third|evening|| +83226|AAAAAAAALBFEBAAA|83226|23|7|6|PM|third|evening|| +83227|AAAAAAAAMBFEBAAA|83227|23|7|7|PM|third|evening|| +83228|AAAAAAAANBFEBAAA|83228|23|7|8|PM|third|evening|| +83229|AAAAAAAAOBFEBAAA|83229|23|7|9|PM|third|evening|| +83230|AAAAAAAAPBFEBAAA|83230|23|7|10|PM|third|evening|| +83231|AAAAAAAAACFEBAAA|83231|23|7|11|PM|third|evening|| +83232|AAAAAAAABCFEBAAA|83232|23|7|12|PM|third|evening|| +83233|AAAAAAAACCFEBAAA|83233|23|7|13|PM|third|evening|| +83234|AAAAAAAADCFEBAAA|83234|23|7|14|PM|third|evening|| +83235|AAAAAAAAECFEBAAA|83235|23|7|15|PM|third|evening|| +83236|AAAAAAAAFCFEBAAA|83236|23|7|16|PM|third|evening|| +83237|AAAAAAAAGCFEBAAA|83237|23|7|17|PM|third|evening|| +83238|AAAAAAAAHCFEBAAA|83238|23|7|18|PM|third|evening|| +83239|AAAAAAAAICFEBAAA|83239|23|7|19|PM|third|evening|| +83240|AAAAAAAAJCFEBAAA|83240|23|7|20|PM|third|evening|| +83241|AAAAAAAAKCFEBAAA|83241|23|7|21|PM|third|evening|| +83242|AAAAAAAALCFEBAAA|83242|23|7|22|PM|third|evening|| +83243|AAAAAAAAMCFEBAAA|83243|23|7|23|PM|third|evening|| +83244|AAAAAAAANCFEBAAA|83244|23|7|24|PM|third|evening|| +83245|AAAAAAAAOCFEBAAA|83245|23|7|25|PM|third|evening|| +83246|AAAAAAAAPCFEBAAA|83246|23|7|26|PM|third|evening|| +83247|AAAAAAAAADFEBAAA|83247|23|7|27|PM|third|evening|| +83248|AAAAAAAABDFEBAAA|83248|23|7|28|PM|third|evening|| +83249|AAAAAAAACDFEBAAA|83249|23|7|29|PM|third|evening|| +83250|AAAAAAAADDFEBAAA|83250|23|7|30|PM|third|evening|| +83251|AAAAAAAAEDFEBAAA|83251|23|7|31|PM|third|evening|| +83252|AAAAAAAAFDFEBAAA|83252|23|7|32|PM|third|evening|| +83253|AAAAAAAAGDFEBAAA|83253|23|7|33|PM|third|evening|| +83254|AAAAAAAAHDFEBAAA|83254|23|7|34|PM|third|evening|| +83255|AAAAAAAAIDFEBAAA|83255|23|7|35|PM|third|evening|| +83256|AAAAAAAAJDFEBAAA|83256|23|7|36|PM|third|evening|| +83257|AAAAAAAAKDFEBAAA|83257|23|7|37|PM|third|evening|| +83258|AAAAAAAALDFEBAAA|83258|23|7|38|PM|third|evening|| +83259|AAAAAAAAMDFEBAAA|83259|23|7|39|PM|third|evening|| +83260|AAAAAAAANDFEBAAA|83260|23|7|40|PM|third|evening|| +83261|AAAAAAAAODFEBAAA|83261|23|7|41|PM|third|evening|| +83262|AAAAAAAAPDFEBAAA|83262|23|7|42|PM|third|evening|| +83263|AAAAAAAAAEFEBAAA|83263|23|7|43|PM|third|evening|| +83264|AAAAAAAABEFEBAAA|83264|23|7|44|PM|third|evening|| +83265|AAAAAAAACEFEBAAA|83265|23|7|45|PM|third|evening|| +83266|AAAAAAAADEFEBAAA|83266|23|7|46|PM|third|evening|| +83267|AAAAAAAAEEFEBAAA|83267|23|7|47|PM|third|evening|| +83268|AAAAAAAAFEFEBAAA|83268|23|7|48|PM|third|evening|| +83269|AAAAAAAAGEFEBAAA|83269|23|7|49|PM|third|evening|| +83270|AAAAAAAAHEFEBAAA|83270|23|7|50|PM|third|evening|| +83271|AAAAAAAAIEFEBAAA|83271|23|7|51|PM|third|evening|| +83272|AAAAAAAAJEFEBAAA|83272|23|7|52|PM|third|evening|| +83273|AAAAAAAAKEFEBAAA|83273|23|7|53|PM|third|evening|| +83274|AAAAAAAALEFEBAAA|83274|23|7|54|PM|third|evening|| +83275|AAAAAAAAMEFEBAAA|83275|23|7|55|PM|third|evening|| +83276|AAAAAAAANEFEBAAA|83276|23|7|56|PM|third|evening|| +83277|AAAAAAAAOEFEBAAA|83277|23|7|57|PM|third|evening|| +83278|AAAAAAAAPEFEBAAA|83278|23|7|58|PM|third|evening|| +83279|AAAAAAAAAFFEBAAA|83279|23|7|59|PM|third|evening|| +83280|AAAAAAAABFFEBAAA|83280|23|8|0|PM|third|evening|| +83281|AAAAAAAACFFEBAAA|83281|23|8|1|PM|third|evening|| +83282|AAAAAAAADFFEBAAA|83282|23|8|2|PM|third|evening|| +83283|AAAAAAAAEFFEBAAA|83283|23|8|3|PM|third|evening|| +83284|AAAAAAAAFFFEBAAA|83284|23|8|4|PM|third|evening|| +83285|AAAAAAAAGFFEBAAA|83285|23|8|5|PM|third|evening|| +83286|AAAAAAAAHFFEBAAA|83286|23|8|6|PM|third|evening|| +83287|AAAAAAAAIFFEBAAA|83287|23|8|7|PM|third|evening|| +83288|AAAAAAAAJFFEBAAA|83288|23|8|8|PM|third|evening|| +83289|AAAAAAAAKFFEBAAA|83289|23|8|9|PM|third|evening|| +83290|AAAAAAAALFFEBAAA|83290|23|8|10|PM|third|evening|| +83291|AAAAAAAAMFFEBAAA|83291|23|8|11|PM|third|evening|| +83292|AAAAAAAANFFEBAAA|83292|23|8|12|PM|third|evening|| +83293|AAAAAAAAOFFEBAAA|83293|23|8|13|PM|third|evening|| +83294|AAAAAAAAPFFEBAAA|83294|23|8|14|PM|third|evening|| +83295|AAAAAAAAAGFEBAAA|83295|23|8|15|PM|third|evening|| +83296|AAAAAAAABGFEBAAA|83296|23|8|16|PM|third|evening|| +83297|AAAAAAAACGFEBAAA|83297|23|8|17|PM|third|evening|| +83298|AAAAAAAADGFEBAAA|83298|23|8|18|PM|third|evening|| +83299|AAAAAAAAEGFEBAAA|83299|23|8|19|PM|third|evening|| +83300|AAAAAAAAFGFEBAAA|83300|23|8|20|PM|third|evening|| +83301|AAAAAAAAGGFEBAAA|83301|23|8|21|PM|third|evening|| +83302|AAAAAAAAHGFEBAAA|83302|23|8|22|PM|third|evening|| +83303|AAAAAAAAIGFEBAAA|83303|23|8|23|PM|third|evening|| +83304|AAAAAAAAJGFEBAAA|83304|23|8|24|PM|third|evening|| +83305|AAAAAAAAKGFEBAAA|83305|23|8|25|PM|third|evening|| +83306|AAAAAAAALGFEBAAA|83306|23|8|26|PM|third|evening|| +83307|AAAAAAAAMGFEBAAA|83307|23|8|27|PM|third|evening|| +83308|AAAAAAAANGFEBAAA|83308|23|8|28|PM|third|evening|| +83309|AAAAAAAAOGFEBAAA|83309|23|8|29|PM|third|evening|| +83310|AAAAAAAAPGFEBAAA|83310|23|8|30|PM|third|evening|| +83311|AAAAAAAAAHFEBAAA|83311|23|8|31|PM|third|evening|| +83312|AAAAAAAABHFEBAAA|83312|23|8|32|PM|third|evening|| +83313|AAAAAAAACHFEBAAA|83313|23|8|33|PM|third|evening|| +83314|AAAAAAAADHFEBAAA|83314|23|8|34|PM|third|evening|| +83315|AAAAAAAAEHFEBAAA|83315|23|8|35|PM|third|evening|| +83316|AAAAAAAAFHFEBAAA|83316|23|8|36|PM|third|evening|| +83317|AAAAAAAAGHFEBAAA|83317|23|8|37|PM|third|evening|| +83318|AAAAAAAAHHFEBAAA|83318|23|8|38|PM|third|evening|| +83319|AAAAAAAAIHFEBAAA|83319|23|8|39|PM|third|evening|| +83320|AAAAAAAAJHFEBAAA|83320|23|8|40|PM|third|evening|| +83321|AAAAAAAAKHFEBAAA|83321|23|8|41|PM|third|evening|| +83322|AAAAAAAALHFEBAAA|83322|23|8|42|PM|third|evening|| +83323|AAAAAAAAMHFEBAAA|83323|23|8|43|PM|third|evening|| +83324|AAAAAAAANHFEBAAA|83324|23|8|44|PM|third|evening|| +83325|AAAAAAAAOHFEBAAA|83325|23|8|45|PM|third|evening|| +83326|AAAAAAAAPHFEBAAA|83326|23|8|46|PM|third|evening|| +83327|AAAAAAAAAIFEBAAA|83327|23|8|47|PM|third|evening|| +83328|AAAAAAAABIFEBAAA|83328|23|8|48|PM|third|evening|| +83329|AAAAAAAACIFEBAAA|83329|23|8|49|PM|third|evening|| +83330|AAAAAAAADIFEBAAA|83330|23|8|50|PM|third|evening|| +83331|AAAAAAAAEIFEBAAA|83331|23|8|51|PM|third|evening|| +83332|AAAAAAAAFIFEBAAA|83332|23|8|52|PM|third|evening|| +83333|AAAAAAAAGIFEBAAA|83333|23|8|53|PM|third|evening|| +83334|AAAAAAAAHIFEBAAA|83334|23|8|54|PM|third|evening|| +83335|AAAAAAAAIIFEBAAA|83335|23|8|55|PM|third|evening|| +83336|AAAAAAAAJIFEBAAA|83336|23|8|56|PM|third|evening|| +83337|AAAAAAAAKIFEBAAA|83337|23|8|57|PM|third|evening|| +83338|AAAAAAAALIFEBAAA|83338|23|8|58|PM|third|evening|| +83339|AAAAAAAAMIFEBAAA|83339|23|8|59|PM|third|evening|| +83340|AAAAAAAANIFEBAAA|83340|23|9|0|PM|third|evening|| +83341|AAAAAAAAOIFEBAAA|83341|23|9|1|PM|third|evening|| +83342|AAAAAAAAPIFEBAAA|83342|23|9|2|PM|third|evening|| +83343|AAAAAAAAAJFEBAAA|83343|23|9|3|PM|third|evening|| +83344|AAAAAAAABJFEBAAA|83344|23|9|4|PM|third|evening|| +83345|AAAAAAAACJFEBAAA|83345|23|9|5|PM|third|evening|| +83346|AAAAAAAADJFEBAAA|83346|23|9|6|PM|third|evening|| +83347|AAAAAAAAEJFEBAAA|83347|23|9|7|PM|third|evening|| +83348|AAAAAAAAFJFEBAAA|83348|23|9|8|PM|third|evening|| +83349|AAAAAAAAGJFEBAAA|83349|23|9|9|PM|third|evening|| +83350|AAAAAAAAHJFEBAAA|83350|23|9|10|PM|third|evening|| +83351|AAAAAAAAIJFEBAAA|83351|23|9|11|PM|third|evening|| +83352|AAAAAAAAJJFEBAAA|83352|23|9|12|PM|third|evening|| +83353|AAAAAAAAKJFEBAAA|83353|23|9|13|PM|third|evening|| +83354|AAAAAAAALJFEBAAA|83354|23|9|14|PM|third|evening|| +83355|AAAAAAAAMJFEBAAA|83355|23|9|15|PM|third|evening|| +83356|AAAAAAAANJFEBAAA|83356|23|9|16|PM|third|evening|| +83357|AAAAAAAAOJFEBAAA|83357|23|9|17|PM|third|evening|| +83358|AAAAAAAAPJFEBAAA|83358|23|9|18|PM|third|evening|| +83359|AAAAAAAAAKFEBAAA|83359|23|9|19|PM|third|evening|| +83360|AAAAAAAABKFEBAAA|83360|23|9|20|PM|third|evening|| +83361|AAAAAAAACKFEBAAA|83361|23|9|21|PM|third|evening|| +83362|AAAAAAAADKFEBAAA|83362|23|9|22|PM|third|evening|| +83363|AAAAAAAAEKFEBAAA|83363|23|9|23|PM|third|evening|| +83364|AAAAAAAAFKFEBAAA|83364|23|9|24|PM|third|evening|| +83365|AAAAAAAAGKFEBAAA|83365|23|9|25|PM|third|evening|| +83366|AAAAAAAAHKFEBAAA|83366|23|9|26|PM|third|evening|| +83367|AAAAAAAAIKFEBAAA|83367|23|9|27|PM|third|evening|| +83368|AAAAAAAAJKFEBAAA|83368|23|9|28|PM|third|evening|| +83369|AAAAAAAAKKFEBAAA|83369|23|9|29|PM|third|evening|| +83370|AAAAAAAALKFEBAAA|83370|23|9|30|PM|third|evening|| +83371|AAAAAAAAMKFEBAAA|83371|23|9|31|PM|third|evening|| +83372|AAAAAAAANKFEBAAA|83372|23|9|32|PM|third|evening|| +83373|AAAAAAAAOKFEBAAA|83373|23|9|33|PM|third|evening|| +83374|AAAAAAAAPKFEBAAA|83374|23|9|34|PM|third|evening|| +83375|AAAAAAAAALFEBAAA|83375|23|9|35|PM|third|evening|| +83376|AAAAAAAABLFEBAAA|83376|23|9|36|PM|third|evening|| +83377|AAAAAAAACLFEBAAA|83377|23|9|37|PM|third|evening|| +83378|AAAAAAAADLFEBAAA|83378|23|9|38|PM|third|evening|| +83379|AAAAAAAAELFEBAAA|83379|23|9|39|PM|third|evening|| +83380|AAAAAAAAFLFEBAAA|83380|23|9|40|PM|third|evening|| +83381|AAAAAAAAGLFEBAAA|83381|23|9|41|PM|third|evening|| +83382|AAAAAAAAHLFEBAAA|83382|23|9|42|PM|third|evening|| +83383|AAAAAAAAILFEBAAA|83383|23|9|43|PM|third|evening|| +83384|AAAAAAAAJLFEBAAA|83384|23|9|44|PM|third|evening|| +83385|AAAAAAAAKLFEBAAA|83385|23|9|45|PM|third|evening|| +83386|AAAAAAAALLFEBAAA|83386|23|9|46|PM|third|evening|| +83387|AAAAAAAAMLFEBAAA|83387|23|9|47|PM|third|evening|| +83388|AAAAAAAANLFEBAAA|83388|23|9|48|PM|third|evening|| +83389|AAAAAAAAOLFEBAAA|83389|23|9|49|PM|third|evening|| +83390|AAAAAAAAPLFEBAAA|83390|23|9|50|PM|third|evening|| +83391|AAAAAAAAAMFEBAAA|83391|23|9|51|PM|third|evening|| +83392|AAAAAAAABMFEBAAA|83392|23|9|52|PM|third|evening|| +83393|AAAAAAAACMFEBAAA|83393|23|9|53|PM|third|evening|| +83394|AAAAAAAADMFEBAAA|83394|23|9|54|PM|third|evening|| +83395|AAAAAAAAEMFEBAAA|83395|23|9|55|PM|third|evening|| +83396|AAAAAAAAFMFEBAAA|83396|23|9|56|PM|third|evening|| +83397|AAAAAAAAGMFEBAAA|83397|23|9|57|PM|third|evening|| +83398|AAAAAAAAHMFEBAAA|83398|23|9|58|PM|third|evening|| +83399|AAAAAAAAIMFEBAAA|83399|23|9|59|PM|third|evening|| +83400|AAAAAAAAJMFEBAAA|83400|23|10|0|PM|third|evening|| +83401|AAAAAAAAKMFEBAAA|83401|23|10|1|PM|third|evening|| +83402|AAAAAAAALMFEBAAA|83402|23|10|2|PM|third|evening|| +83403|AAAAAAAAMMFEBAAA|83403|23|10|3|PM|third|evening|| +83404|AAAAAAAANMFEBAAA|83404|23|10|4|PM|third|evening|| +83405|AAAAAAAAOMFEBAAA|83405|23|10|5|PM|third|evening|| +83406|AAAAAAAAPMFEBAAA|83406|23|10|6|PM|third|evening|| +83407|AAAAAAAAANFEBAAA|83407|23|10|7|PM|third|evening|| +83408|AAAAAAAABNFEBAAA|83408|23|10|8|PM|third|evening|| +83409|AAAAAAAACNFEBAAA|83409|23|10|9|PM|third|evening|| +83410|AAAAAAAADNFEBAAA|83410|23|10|10|PM|third|evening|| +83411|AAAAAAAAENFEBAAA|83411|23|10|11|PM|third|evening|| +83412|AAAAAAAAFNFEBAAA|83412|23|10|12|PM|third|evening|| +83413|AAAAAAAAGNFEBAAA|83413|23|10|13|PM|third|evening|| +83414|AAAAAAAAHNFEBAAA|83414|23|10|14|PM|third|evening|| +83415|AAAAAAAAINFEBAAA|83415|23|10|15|PM|third|evening|| +83416|AAAAAAAAJNFEBAAA|83416|23|10|16|PM|third|evening|| +83417|AAAAAAAAKNFEBAAA|83417|23|10|17|PM|third|evening|| +83418|AAAAAAAALNFEBAAA|83418|23|10|18|PM|third|evening|| +83419|AAAAAAAAMNFEBAAA|83419|23|10|19|PM|third|evening|| +83420|AAAAAAAANNFEBAAA|83420|23|10|20|PM|third|evening|| +83421|AAAAAAAAONFEBAAA|83421|23|10|21|PM|third|evening|| +83422|AAAAAAAAPNFEBAAA|83422|23|10|22|PM|third|evening|| +83423|AAAAAAAAAOFEBAAA|83423|23|10|23|PM|third|evening|| +83424|AAAAAAAABOFEBAAA|83424|23|10|24|PM|third|evening|| +83425|AAAAAAAACOFEBAAA|83425|23|10|25|PM|third|evening|| +83426|AAAAAAAADOFEBAAA|83426|23|10|26|PM|third|evening|| +83427|AAAAAAAAEOFEBAAA|83427|23|10|27|PM|third|evening|| +83428|AAAAAAAAFOFEBAAA|83428|23|10|28|PM|third|evening|| +83429|AAAAAAAAGOFEBAAA|83429|23|10|29|PM|third|evening|| +83430|AAAAAAAAHOFEBAAA|83430|23|10|30|PM|third|evening|| +83431|AAAAAAAAIOFEBAAA|83431|23|10|31|PM|third|evening|| +83432|AAAAAAAAJOFEBAAA|83432|23|10|32|PM|third|evening|| +83433|AAAAAAAAKOFEBAAA|83433|23|10|33|PM|third|evening|| +83434|AAAAAAAALOFEBAAA|83434|23|10|34|PM|third|evening|| +83435|AAAAAAAAMOFEBAAA|83435|23|10|35|PM|third|evening|| +83436|AAAAAAAANOFEBAAA|83436|23|10|36|PM|third|evening|| +83437|AAAAAAAAOOFEBAAA|83437|23|10|37|PM|third|evening|| +83438|AAAAAAAAPOFEBAAA|83438|23|10|38|PM|third|evening|| +83439|AAAAAAAAAPFEBAAA|83439|23|10|39|PM|third|evening|| +83440|AAAAAAAABPFEBAAA|83440|23|10|40|PM|third|evening|| +83441|AAAAAAAACPFEBAAA|83441|23|10|41|PM|third|evening|| +83442|AAAAAAAADPFEBAAA|83442|23|10|42|PM|third|evening|| +83443|AAAAAAAAEPFEBAAA|83443|23|10|43|PM|third|evening|| +83444|AAAAAAAAFPFEBAAA|83444|23|10|44|PM|third|evening|| +83445|AAAAAAAAGPFEBAAA|83445|23|10|45|PM|third|evening|| +83446|AAAAAAAAHPFEBAAA|83446|23|10|46|PM|third|evening|| +83447|AAAAAAAAIPFEBAAA|83447|23|10|47|PM|third|evening|| +83448|AAAAAAAAJPFEBAAA|83448|23|10|48|PM|third|evening|| +83449|AAAAAAAAKPFEBAAA|83449|23|10|49|PM|third|evening|| +83450|AAAAAAAALPFEBAAA|83450|23|10|50|PM|third|evening|| +83451|AAAAAAAAMPFEBAAA|83451|23|10|51|PM|third|evening|| +83452|AAAAAAAANPFEBAAA|83452|23|10|52|PM|third|evening|| +83453|AAAAAAAAOPFEBAAA|83453|23|10|53|PM|third|evening|| +83454|AAAAAAAAPPFEBAAA|83454|23|10|54|PM|third|evening|| +83455|AAAAAAAAAAGEBAAA|83455|23|10|55|PM|third|evening|| +83456|AAAAAAAABAGEBAAA|83456|23|10|56|PM|third|evening|| +83457|AAAAAAAACAGEBAAA|83457|23|10|57|PM|third|evening|| +83458|AAAAAAAADAGEBAAA|83458|23|10|58|PM|third|evening|| +83459|AAAAAAAAEAGEBAAA|83459|23|10|59|PM|third|evening|| +83460|AAAAAAAAFAGEBAAA|83460|23|11|0|PM|third|evening|| +83461|AAAAAAAAGAGEBAAA|83461|23|11|1|PM|third|evening|| +83462|AAAAAAAAHAGEBAAA|83462|23|11|2|PM|third|evening|| +83463|AAAAAAAAIAGEBAAA|83463|23|11|3|PM|third|evening|| +83464|AAAAAAAAJAGEBAAA|83464|23|11|4|PM|third|evening|| +83465|AAAAAAAAKAGEBAAA|83465|23|11|5|PM|third|evening|| +83466|AAAAAAAALAGEBAAA|83466|23|11|6|PM|third|evening|| +83467|AAAAAAAAMAGEBAAA|83467|23|11|7|PM|third|evening|| +83468|AAAAAAAANAGEBAAA|83468|23|11|8|PM|third|evening|| +83469|AAAAAAAAOAGEBAAA|83469|23|11|9|PM|third|evening|| +83470|AAAAAAAAPAGEBAAA|83470|23|11|10|PM|third|evening|| +83471|AAAAAAAAABGEBAAA|83471|23|11|11|PM|third|evening|| +83472|AAAAAAAABBGEBAAA|83472|23|11|12|PM|third|evening|| +83473|AAAAAAAACBGEBAAA|83473|23|11|13|PM|third|evening|| +83474|AAAAAAAADBGEBAAA|83474|23|11|14|PM|third|evening|| +83475|AAAAAAAAEBGEBAAA|83475|23|11|15|PM|third|evening|| +83476|AAAAAAAAFBGEBAAA|83476|23|11|16|PM|third|evening|| +83477|AAAAAAAAGBGEBAAA|83477|23|11|17|PM|third|evening|| +83478|AAAAAAAAHBGEBAAA|83478|23|11|18|PM|third|evening|| +83479|AAAAAAAAIBGEBAAA|83479|23|11|19|PM|third|evening|| +83480|AAAAAAAAJBGEBAAA|83480|23|11|20|PM|third|evening|| +83481|AAAAAAAAKBGEBAAA|83481|23|11|21|PM|third|evening|| +83482|AAAAAAAALBGEBAAA|83482|23|11|22|PM|third|evening|| +83483|AAAAAAAAMBGEBAAA|83483|23|11|23|PM|third|evening|| +83484|AAAAAAAANBGEBAAA|83484|23|11|24|PM|third|evening|| +83485|AAAAAAAAOBGEBAAA|83485|23|11|25|PM|third|evening|| +83486|AAAAAAAAPBGEBAAA|83486|23|11|26|PM|third|evening|| +83487|AAAAAAAAACGEBAAA|83487|23|11|27|PM|third|evening|| +83488|AAAAAAAABCGEBAAA|83488|23|11|28|PM|third|evening|| +83489|AAAAAAAACCGEBAAA|83489|23|11|29|PM|third|evening|| +83490|AAAAAAAADCGEBAAA|83490|23|11|30|PM|third|evening|| +83491|AAAAAAAAECGEBAAA|83491|23|11|31|PM|third|evening|| +83492|AAAAAAAAFCGEBAAA|83492|23|11|32|PM|third|evening|| +83493|AAAAAAAAGCGEBAAA|83493|23|11|33|PM|third|evening|| +83494|AAAAAAAAHCGEBAAA|83494|23|11|34|PM|third|evening|| +83495|AAAAAAAAICGEBAAA|83495|23|11|35|PM|third|evening|| +83496|AAAAAAAAJCGEBAAA|83496|23|11|36|PM|third|evening|| +83497|AAAAAAAAKCGEBAAA|83497|23|11|37|PM|third|evening|| +83498|AAAAAAAALCGEBAAA|83498|23|11|38|PM|third|evening|| +83499|AAAAAAAAMCGEBAAA|83499|23|11|39|PM|third|evening|| +83500|AAAAAAAANCGEBAAA|83500|23|11|40|PM|third|evening|| +83501|AAAAAAAAOCGEBAAA|83501|23|11|41|PM|third|evening|| +83502|AAAAAAAAPCGEBAAA|83502|23|11|42|PM|third|evening|| +83503|AAAAAAAAADGEBAAA|83503|23|11|43|PM|third|evening|| +83504|AAAAAAAABDGEBAAA|83504|23|11|44|PM|third|evening|| +83505|AAAAAAAACDGEBAAA|83505|23|11|45|PM|third|evening|| +83506|AAAAAAAADDGEBAAA|83506|23|11|46|PM|third|evening|| +83507|AAAAAAAAEDGEBAAA|83507|23|11|47|PM|third|evening|| +83508|AAAAAAAAFDGEBAAA|83508|23|11|48|PM|third|evening|| +83509|AAAAAAAAGDGEBAAA|83509|23|11|49|PM|third|evening|| +83510|AAAAAAAAHDGEBAAA|83510|23|11|50|PM|third|evening|| +83511|AAAAAAAAIDGEBAAA|83511|23|11|51|PM|third|evening|| +83512|AAAAAAAAJDGEBAAA|83512|23|11|52|PM|third|evening|| +83513|AAAAAAAAKDGEBAAA|83513|23|11|53|PM|third|evening|| +83514|AAAAAAAALDGEBAAA|83514|23|11|54|PM|third|evening|| +83515|AAAAAAAAMDGEBAAA|83515|23|11|55|PM|third|evening|| +83516|AAAAAAAANDGEBAAA|83516|23|11|56|PM|third|evening|| +83517|AAAAAAAAODGEBAAA|83517|23|11|57|PM|third|evening|| +83518|AAAAAAAAPDGEBAAA|83518|23|11|58|PM|third|evening|| +83519|AAAAAAAAAEGEBAAA|83519|23|11|59|PM|third|evening|| +83520|AAAAAAAABEGEBAAA|83520|23|12|0|PM|third|evening|| +83521|AAAAAAAACEGEBAAA|83521|23|12|1|PM|third|evening|| +83522|AAAAAAAADEGEBAAA|83522|23|12|2|PM|third|evening|| +83523|AAAAAAAAEEGEBAAA|83523|23|12|3|PM|third|evening|| +83524|AAAAAAAAFEGEBAAA|83524|23|12|4|PM|third|evening|| +83525|AAAAAAAAGEGEBAAA|83525|23|12|5|PM|third|evening|| +83526|AAAAAAAAHEGEBAAA|83526|23|12|6|PM|third|evening|| +83527|AAAAAAAAIEGEBAAA|83527|23|12|7|PM|third|evening|| +83528|AAAAAAAAJEGEBAAA|83528|23|12|8|PM|third|evening|| +83529|AAAAAAAAKEGEBAAA|83529|23|12|9|PM|third|evening|| +83530|AAAAAAAALEGEBAAA|83530|23|12|10|PM|third|evening|| +83531|AAAAAAAAMEGEBAAA|83531|23|12|11|PM|third|evening|| +83532|AAAAAAAANEGEBAAA|83532|23|12|12|PM|third|evening|| +83533|AAAAAAAAOEGEBAAA|83533|23|12|13|PM|third|evening|| +83534|AAAAAAAAPEGEBAAA|83534|23|12|14|PM|third|evening|| +83535|AAAAAAAAAFGEBAAA|83535|23|12|15|PM|third|evening|| +83536|AAAAAAAABFGEBAAA|83536|23|12|16|PM|third|evening|| +83537|AAAAAAAACFGEBAAA|83537|23|12|17|PM|third|evening|| +83538|AAAAAAAADFGEBAAA|83538|23|12|18|PM|third|evening|| +83539|AAAAAAAAEFGEBAAA|83539|23|12|19|PM|third|evening|| +83540|AAAAAAAAFFGEBAAA|83540|23|12|20|PM|third|evening|| +83541|AAAAAAAAGFGEBAAA|83541|23|12|21|PM|third|evening|| +83542|AAAAAAAAHFGEBAAA|83542|23|12|22|PM|third|evening|| +83543|AAAAAAAAIFGEBAAA|83543|23|12|23|PM|third|evening|| +83544|AAAAAAAAJFGEBAAA|83544|23|12|24|PM|third|evening|| +83545|AAAAAAAAKFGEBAAA|83545|23|12|25|PM|third|evening|| +83546|AAAAAAAALFGEBAAA|83546|23|12|26|PM|third|evening|| +83547|AAAAAAAAMFGEBAAA|83547|23|12|27|PM|third|evening|| +83548|AAAAAAAANFGEBAAA|83548|23|12|28|PM|third|evening|| +83549|AAAAAAAAOFGEBAAA|83549|23|12|29|PM|third|evening|| +83550|AAAAAAAAPFGEBAAA|83550|23|12|30|PM|third|evening|| +83551|AAAAAAAAAGGEBAAA|83551|23|12|31|PM|third|evening|| +83552|AAAAAAAABGGEBAAA|83552|23|12|32|PM|third|evening|| +83553|AAAAAAAACGGEBAAA|83553|23|12|33|PM|third|evening|| +83554|AAAAAAAADGGEBAAA|83554|23|12|34|PM|third|evening|| +83555|AAAAAAAAEGGEBAAA|83555|23|12|35|PM|third|evening|| +83556|AAAAAAAAFGGEBAAA|83556|23|12|36|PM|third|evening|| +83557|AAAAAAAAGGGEBAAA|83557|23|12|37|PM|third|evening|| +83558|AAAAAAAAHGGEBAAA|83558|23|12|38|PM|third|evening|| +83559|AAAAAAAAIGGEBAAA|83559|23|12|39|PM|third|evening|| +83560|AAAAAAAAJGGEBAAA|83560|23|12|40|PM|third|evening|| +83561|AAAAAAAAKGGEBAAA|83561|23|12|41|PM|third|evening|| +83562|AAAAAAAALGGEBAAA|83562|23|12|42|PM|third|evening|| +83563|AAAAAAAAMGGEBAAA|83563|23|12|43|PM|third|evening|| +83564|AAAAAAAANGGEBAAA|83564|23|12|44|PM|third|evening|| +83565|AAAAAAAAOGGEBAAA|83565|23|12|45|PM|third|evening|| +83566|AAAAAAAAPGGEBAAA|83566|23|12|46|PM|third|evening|| +83567|AAAAAAAAAHGEBAAA|83567|23|12|47|PM|third|evening|| +83568|AAAAAAAABHGEBAAA|83568|23|12|48|PM|third|evening|| +83569|AAAAAAAACHGEBAAA|83569|23|12|49|PM|third|evening|| +83570|AAAAAAAADHGEBAAA|83570|23|12|50|PM|third|evening|| +83571|AAAAAAAAEHGEBAAA|83571|23|12|51|PM|third|evening|| +83572|AAAAAAAAFHGEBAAA|83572|23|12|52|PM|third|evening|| +83573|AAAAAAAAGHGEBAAA|83573|23|12|53|PM|third|evening|| +83574|AAAAAAAAHHGEBAAA|83574|23|12|54|PM|third|evening|| +83575|AAAAAAAAIHGEBAAA|83575|23|12|55|PM|third|evening|| +83576|AAAAAAAAJHGEBAAA|83576|23|12|56|PM|third|evening|| +83577|AAAAAAAAKHGEBAAA|83577|23|12|57|PM|third|evening|| +83578|AAAAAAAALHGEBAAA|83578|23|12|58|PM|third|evening|| +83579|AAAAAAAAMHGEBAAA|83579|23|12|59|PM|third|evening|| +83580|AAAAAAAANHGEBAAA|83580|23|13|0|PM|third|evening|| +83581|AAAAAAAAOHGEBAAA|83581|23|13|1|PM|third|evening|| +83582|AAAAAAAAPHGEBAAA|83582|23|13|2|PM|third|evening|| +83583|AAAAAAAAAIGEBAAA|83583|23|13|3|PM|third|evening|| +83584|AAAAAAAABIGEBAAA|83584|23|13|4|PM|third|evening|| +83585|AAAAAAAACIGEBAAA|83585|23|13|5|PM|third|evening|| +83586|AAAAAAAADIGEBAAA|83586|23|13|6|PM|third|evening|| +83587|AAAAAAAAEIGEBAAA|83587|23|13|7|PM|third|evening|| +83588|AAAAAAAAFIGEBAAA|83588|23|13|8|PM|third|evening|| +83589|AAAAAAAAGIGEBAAA|83589|23|13|9|PM|third|evening|| +83590|AAAAAAAAHIGEBAAA|83590|23|13|10|PM|third|evening|| +83591|AAAAAAAAIIGEBAAA|83591|23|13|11|PM|third|evening|| +83592|AAAAAAAAJIGEBAAA|83592|23|13|12|PM|third|evening|| +83593|AAAAAAAAKIGEBAAA|83593|23|13|13|PM|third|evening|| +83594|AAAAAAAALIGEBAAA|83594|23|13|14|PM|third|evening|| +83595|AAAAAAAAMIGEBAAA|83595|23|13|15|PM|third|evening|| +83596|AAAAAAAANIGEBAAA|83596|23|13|16|PM|third|evening|| +83597|AAAAAAAAOIGEBAAA|83597|23|13|17|PM|third|evening|| +83598|AAAAAAAAPIGEBAAA|83598|23|13|18|PM|third|evening|| +83599|AAAAAAAAAJGEBAAA|83599|23|13|19|PM|third|evening|| +83600|AAAAAAAABJGEBAAA|83600|23|13|20|PM|third|evening|| +83601|AAAAAAAACJGEBAAA|83601|23|13|21|PM|third|evening|| +83602|AAAAAAAADJGEBAAA|83602|23|13|22|PM|third|evening|| +83603|AAAAAAAAEJGEBAAA|83603|23|13|23|PM|third|evening|| +83604|AAAAAAAAFJGEBAAA|83604|23|13|24|PM|third|evening|| +83605|AAAAAAAAGJGEBAAA|83605|23|13|25|PM|third|evening|| +83606|AAAAAAAAHJGEBAAA|83606|23|13|26|PM|third|evening|| +83607|AAAAAAAAIJGEBAAA|83607|23|13|27|PM|third|evening|| +83608|AAAAAAAAJJGEBAAA|83608|23|13|28|PM|third|evening|| +83609|AAAAAAAAKJGEBAAA|83609|23|13|29|PM|third|evening|| +83610|AAAAAAAALJGEBAAA|83610|23|13|30|PM|third|evening|| +83611|AAAAAAAAMJGEBAAA|83611|23|13|31|PM|third|evening|| +83612|AAAAAAAANJGEBAAA|83612|23|13|32|PM|third|evening|| +83613|AAAAAAAAOJGEBAAA|83613|23|13|33|PM|third|evening|| +83614|AAAAAAAAPJGEBAAA|83614|23|13|34|PM|third|evening|| +83615|AAAAAAAAAKGEBAAA|83615|23|13|35|PM|third|evening|| +83616|AAAAAAAABKGEBAAA|83616|23|13|36|PM|third|evening|| +83617|AAAAAAAACKGEBAAA|83617|23|13|37|PM|third|evening|| +83618|AAAAAAAADKGEBAAA|83618|23|13|38|PM|third|evening|| +83619|AAAAAAAAEKGEBAAA|83619|23|13|39|PM|third|evening|| +83620|AAAAAAAAFKGEBAAA|83620|23|13|40|PM|third|evening|| +83621|AAAAAAAAGKGEBAAA|83621|23|13|41|PM|third|evening|| +83622|AAAAAAAAHKGEBAAA|83622|23|13|42|PM|third|evening|| +83623|AAAAAAAAIKGEBAAA|83623|23|13|43|PM|third|evening|| +83624|AAAAAAAAJKGEBAAA|83624|23|13|44|PM|third|evening|| +83625|AAAAAAAAKKGEBAAA|83625|23|13|45|PM|third|evening|| +83626|AAAAAAAALKGEBAAA|83626|23|13|46|PM|third|evening|| +83627|AAAAAAAAMKGEBAAA|83627|23|13|47|PM|third|evening|| +83628|AAAAAAAANKGEBAAA|83628|23|13|48|PM|third|evening|| +83629|AAAAAAAAOKGEBAAA|83629|23|13|49|PM|third|evening|| +83630|AAAAAAAAPKGEBAAA|83630|23|13|50|PM|third|evening|| +83631|AAAAAAAAALGEBAAA|83631|23|13|51|PM|third|evening|| +83632|AAAAAAAABLGEBAAA|83632|23|13|52|PM|third|evening|| +83633|AAAAAAAACLGEBAAA|83633|23|13|53|PM|third|evening|| +83634|AAAAAAAADLGEBAAA|83634|23|13|54|PM|third|evening|| +83635|AAAAAAAAELGEBAAA|83635|23|13|55|PM|third|evening|| +83636|AAAAAAAAFLGEBAAA|83636|23|13|56|PM|third|evening|| +83637|AAAAAAAAGLGEBAAA|83637|23|13|57|PM|third|evening|| +83638|AAAAAAAAHLGEBAAA|83638|23|13|58|PM|third|evening|| +83639|AAAAAAAAILGEBAAA|83639|23|13|59|PM|third|evening|| +83640|AAAAAAAAJLGEBAAA|83640|23|14|0|PM|third|evening|| +83641|AAAAAAAAKLGEBAAA|83641|23|14|1|PM|third|evening|| +83642|AAAAAAAALLGEBAAA|83642|23|14|2|PM|third|evening|| +83643|AAAAAAAAMLGEBAAA|83643|23|14|3|PM|third|evening|| +83644|AAAAAAAANLGEBAAA|83644|23|14|4|PM|third|evening|| +83645|AAAAAAAAOLGEBAAA|83645|23|14|5|PM|third|evening|| +83646|AAAAAAAAPLGEBAAA|83646|23|14|6|PM|third|evening|| +83647|AAAAAAAAAMGEBAAA|83647|23|14|7|PM|third|evening|| +83648|AAAAAAAABMGEBAAA|83648|23|14|8|PM|third|evening|| +83649|AAAAAAAACMGEBAAA|83649|23|14|9|PM|third|evening|| +83650|AAAAAAAADMGEBAAA|83650|23|14|10|PM|third|evening|| +83651|AAAAAAAAEMGEBAAA|83651|23|14|11|PM|third|evening|| +83652|AAAAAAAAFMGEBAAA|83652|23|14|12|PM|third|evening|| +83653|AAAAAAAAGMGEBAAA|83653|23|14|13|PM|third|evening|| +83654|AAAAAAAAHMGEBAAA|83654|23|14|14|PM|third|evening|| +83655|AAAAAAAAIMGEBAAA|83655|23|14|15|PM|third|evening|| +83656|AAAAAAAAJMGEBAAA|83656|23|14|16|PM|third|evening|| +83657|AAAAAAAAKMGEBAAA|83657|23|14|17|PM|third|evening|| +83658|AAAAAAAALMGEBAAA|83658|23|14|18|PM|third|evening|| +83659|AAAAAAAAMMGEBAAA|83659|23|14|19|PM|third|evening|| +83660|AAAAAAAANMGEBAAA|83660|23|14|20|PM|third|evening|| +83661|AAAAAAAAOMGEBAAA|83661|23|14|21|PM|third|evening|| +83662|AAAAAAAAPMGEBAAA|83662|23|14|22|PM|third|evening|| +83663|AAAAAAAAANGEBAAA|83663|23|14|23|PM|third|evening|| +83664|AAAAAAAABNGEBAAA|83664|23|14|24|PM|third|evening|| +83665|AAAAAAAACNGEBAAA|83665|23|14|25|PM|third|evening|| +83666|AAAAAAAADNGEBAAA|83666|23|14|26|PM|third|evening|| +83667|AAAAAAAAENGEBAAA|83667|23|14|27|PM|third|evening|| +83668|AAAAAAAAFNGEBAAA|83668|23|14|28|PM|third|evening|| +83669|AAAAAAAAGNGEBAAA|83669|23|14|29|PM|third|evening|| +83670|AAAAAAAAHNGEBAAA|83670|23|14|30|PM|third|evening|| +83671|AAAAAAAAINGEBAAA|83671|23|14|31|PM|third|evening|| +83672|AAAAAAAAJNGEBAAA|83672|23|14|32|PM|third|evening|| +83673|AAAAAAAAKNGEBAAA|83673|23|14|33|PM|third|evening|| +83674|AAAAAAAALNGEBAAA|83674|23|14|34|PM|third|evening|| +83675|AAAAAAAAMNGEBAAA|83675|23|14|35|PM|third|evening|| +83676|AAAAAAAANNGEBAAA|83676|23|14|36|PM|third|evening|| +83677|AAAAAAAAONGEBAAA|83677|23|14|37|PM|third|evening|| +83678|AAAAAAAAPNGEBAAA|83678|23|14|38|PM|third|evening|| +83679|AAAAAAAAAOGEBAAA|83679|23|14|39|PM|third|evening|| +83680|AAAAAAAABOGEBAAA|83680|23|14|40|PM|third|evening|| +83681|AAAAAAAACOGEBAAA|83681|23|14|41|PM|third|evening|| +83682|AAAAAAAADOGEBAAA|83682|23|14|42|PM|third|evening|| +83683|AAAAAAAAEOGEBAAA|83683|23|14|43|PM|third|evening|| +83684|AAAAAAAAFOGEBAAA|83684|23|14|44|PM|third|evening|| +83685|AAAAAAAAGOGEBAAA|83685|23|14|45|PM|third|evening|| +83686|AAAAAAAAHOGEBAAA|83686|23|14|46|PM|third|evening|| +83687|AAAAAAAAIOGEBAAA|83687|23|14|47|PM|third|evening|| +83688|AAAAAAAAJOGEBAAA|83688|23|14|48|PM|third|evening|| +83689|AAAAAAAAKOGEBAAA|83689|23|14|49|PM|third|evening|| +83690|AAAAAAAALOGEBAAA|83690|23|14|50|PM|third|evening|| +83691|AAAAAAAAMOGEBAAA|83691|23|14|51|PM|third|evening|| +83692|AAAAAAAANOGEBAAA|83692|23|14|52|PM|third|evening|| +83693|AAAAAAAAOOGEBAAA|83693|23|14|53|PM|third|evening|| +83694|AAAAAAAAPOGEBAAA|83694|23|14|54|PM|third|evening|| +83695|AAAAAAAAAPGEBAAA|83695|23|14|55|PM|third|evening|| +83696|AAAAAAAABPGEBAAA|83696|23|14|56|PM|third|evening|| +83697|AAAAAAAACPGEBAAA|83697|23|14|57|PM|third|evening|| +83698|AAAAAAAADPGEBAAA|83698|23|14|58|PM|third|evening|| +83699|AAAAAAAAEPGEBAAA|83699|23|14|59|PM|third|evening|| +83700|AAAAAAAAFPGEBAAA|83700|23|15|0|PM|third|evening|| +83701|AAAAAAAAGPGEBAAA|83701|23|15|1|PM|third|evening|| +83702|AAAAAAAAHPGEBAAA|83702|23|15|2|PM|third|evening|| +83703|AAAAAAAAIPGEBAAA|83703|23|15|3|PM|third|evening|| +83704|AAAAAAAAJPGEBAAA|83704|23|15|4|PM|third|evening|| +83705|AAAAAAAAKPGEBAAA|83705|23|15|5|PM|third|evening|| +83706|AAAAAAAALPGEBAAA|83706|23|15|6|PM|third|evening|| +83707|AAAAAAAAMPGEBAAA|83707|23|15|7|PM|third|evening|| +83708|AAAAAAAANPGEBAAA|83708|23|15|8|PM|third|evening|| +83709|AAAAAAAAOPGEBAAA|83709|23|15|9|PM|third|evening|| +83710|AAAAAAAAPPGEBAAA|83710|23|15|10|PM|third|evening|| +83711|AAAAAAAAAAHEBAAA|83711|23|15|11|PM|third|evening|| +83712|AAAAAAAABAHEBAAA|83712|23|15|12|PM|third|evening|| +83713|AAAAAAAACAHEBAAA|83713|23|15|13|PM|third|evening|| +83714|AAAAAAAADAHEBAAA|83714|23|15|14|PM|third|evening|| +83715|AAAAAAAAEAHEBAAA|83715|23|15|15|PM|third|evening|| +83716|AAAAAAAAFAHEBAAA|83716|23|15|16|PM|third|evening|| +83717|AAAAAAAAGAHEBAAA|83717|23|15|17|PM|third|evening|| +83718|AAAAAAAAHAHEBAAA|83718|23|15|18|PM|third|evening|| +83719|AAAAAAAAIAHEBAAA|83719|23|15|19|PM|third|evening|| +83720|AAAAAAAAJAHEBAAA|83720|23|15|20|PM|third|evening|| +83721|AAAAAAAAKAHEBAAA|83721|23|15|21|PM|third|evening|| +83722|AAAAAAAALAHEBAAA|83722|23|15|22|PM|third|evening|| +83723|AAAAAAAAMAHEBAAA|83723|23|15|23|PM|third|evening|| +83724|AAAAAAAANAHEBAAA|83724|23|15|24|PM|third|evening|| +83725|AAAAAAAAOAHEBAAA|83725|23|15|25|PM|third|evening|| +83726|AAAAAAAAPAHEBAAA|83726|23|15|26|PM|third|evening|| +83727|AAAAAAAAABHEBAAA|83727|23|15|27|PM|third|evening|| +83728|AAAAAAAABBHEBAAA|83728|23|15|28|PM|third|evening|| +83729|AAAAAAAACBHEBAAA|83729|23|15|29|PM|third|evening|| +83730|AAAAAAAADBHEBAAA|83730|23|15|30|PM|third|evening|| +83731|AAAAAAAAEBHEBAAA|83731|23|15|31|PM|third|evening|| +83732|AAAAAAAAFBHEBAAA|83732|23|15|32|PM|third|evening|| +83733|AAAAAAAAGBHEBAAA|83733|23|15|33|PM|third|evening|| +83734|AAAAAAAAHBHEBAAA|83734|23|15|34|PM|third|evening|| +83735|AAAAAAAAIBHEBAAA|83735|23|15|35|PM|third|evening|| +83736|AAAAAAAAJBHEBAAA|83736|23|15|36|PM|third|evening|| +83737|AAAAAAAAKBHEBAAA|83737|23|15|37|PM|third|evening|| +83738|AAAAAAAALBHEBAAA|83738|23|15|38|PM|third|evening|| +83739|AAAAAAAAMBHEBAAA|83739|23|15|39|PM|third|evening|| +83740|AAAAAAAANBHEBAAA|83740|23|15|40|PM|third|evening|| +83741|AAAAAAAAOBHEBAAA|83741|23|15|41|PM|third|evening|| +83742|AAAAAAAAPBHEBAAA|83742|23|15|42|PM|third|evening|| +83743|AAAAAAAAACHEBAAA|83743|23|15|43|PM|third|evening|| +83744|AAAAAAAABCHEBAAA|83744|23|15|44|PM|third|evening|| +83745|AAAAAAAACCHEBAAA|83745|23|15|45|PM|third|evening|| +83746|AAAAAAAADCHEBAAA|83746|23|15|46|PM|third|evening|| +83747|AAAAAAAAECHEBAAA|83747|23|15|47|PM|third|evening|| +83748|AAAAAAAAFCHEBAAA|83748|23|15|48|PM|third|evening|| +83749|AAAAAAAAGCHEBAAA|83749|23|15|49|PM|third|evening|| +83750|AAAAAAAAHCHEBAAA|83750|23|15|50|PM|third|evening|| +83751|AAAAAAAAICHEBAAA|83751|23|15|51|PM|third|evening|| +83752|AAAAAAAAJCHEBAAA|83752|23|15|52|PM|third|evening|| +83753|AAAAAAAAKCHEBAAA|83753|23|15|53|PM|third|evening|| +83754|AAAAAAAALCHEBAAA|83754|23|15|54|PM|third|evening|| +83755|AAAAAAAAMCHEBAAA|83755|23|15|55|PM|third|evening|| +83756|AAAAAAAANCHEBAAA|83756|23|15|56|PM|third|evening|| +83757|AAAAAAAAOCHEBAAA|83757|23|15|57|PM|third|evening|| +83758|AAAAAAAAPCHEBAAA|83758|23|15|58|PM|third|evening|| +83759|AAAAAAAAADHEBAAA|83759|23|15|59|PM|third|evening|| +83760|AAAAAAAABDHEBAAA|83760|23|16|0|PM|third|evening|| +83761|AAAAAAAACDHEBAAA|83761|23|16|1|PM|third|evening|| +83762|AAAAAAAADDHEBAAA|83762|23|16|2|PM|third|evening|| +83763|AAAAAAAAEDHEBAAA|83763|23|16|3|PM|third|evening|| +83764|AAAAAAAAFDHEBAAA|83764|23|16|4|PM|third|evening|| +83765|AAAAAAAAGDHEBAAA|83765|23|16|5|PM|third|evening|| +83766|AAAAAAAAHDHEBAAA|83766|23|16|6|PM|third|evening|| +83767|AAAAAAAAIDHEBAAA|83767|23|16|7|PM|third|evening|| +83768|AAAAAAAAJDHEBAAA|83768|23|16|8|PM|third|evening|| +83769|AAAAAAAAKDHEBAAA|83769|23|16|9|PM|third|evening|| +83770|AAAAAAAALDHEBAAA|83770|23|16|10|PM|third|evening|| +83771|AAAAAAAAMDHEBAAA|83771|23|16|11|PM|third|evening|| +83772|AAAAAAAANDHEBAAA|83772|23|16|12|PM|third|evening|| +83773|AAAAAAAAODHEBAAA|83773|23|16|13|PM|third|evening|| +83774|AAAAAAAAPDHEBAAA|83774|23|16|14|PM|third|evening|| +83775|AAAAAAAAAEHEBAAA|83775|23|16|15|PM|third|evening|| +83776|AAAAAAAABEHEBAAA|83776|23|16|16|PM|third|evening|| +83777|AAAAAAAACEHEBAAA|83777|23|16|17|PM|third|evening|| +83778|AAAAAAAADEHEBAAA|83778|23|16|18|PM|third|evening|| +83779|AAAAAAAAEEHEBAAA|83779|23|16|19|PM|third|evening|| +83780|AAAAAAAAFEHEBAAA|83780|23|16|20|PM|third|evening|| +83781|AAAAAAAAGEHEBAAA|83781|23|16|21|PM|third|evening|| +83782|AAAAAAAAHEHEBAAA|83782|23|16|22|PM|third|evening|| +83783|AAAAAAAAIEHEBAAA|83783|23|16|23|PM|third|evening|| +83784|AAAAAAAAJEHEBAAA|83784|23|16|24|PM|third|evening|| +83785|AAAAAAAAKEHEBAAA|83785|23|16|25|PM|third|evening|| +83786|AAAAAAAALEHEBAAA|83786|23|16|26|PM|third|evening|| +83787|AAAAAAAAMEHEBAAA|83787|23|16|27|PM|third|evening|| +83788|AAAAAAAANEHEBAAA|83788|23|16|28|PM|third|evening|| +83789|AAAAAAAAOEHEBAAA|83789|23|16|29|PM|third|evening|| +83790|AAAAAAAAPEHEBAAA|83790|23|16|30|PM|third|evening|| +83791|AAAAAAAAAFHEBAAA|83791|23|16|31|PM|third|evening|| +83792|AAAAAAAABFHEBAAA|83792|23|16|32|PM|third|evening|| +83793|AAAAAAAACFHEBAAA|83793|23|16|33|PM|third|evening|| +83794|AAAAAAAADFHEBAAA|83794|23|16|34|PM|third|evening|| +83795|AAAAAAAAEFHEBAAA|83795|23|16|35|PM|third|evening|| +83796|AAAAAAAAFFHEBAAA|83796|23|16|36|PM|third|evening|| +83797|AAAAAAAAGFHEBAAA|83797|23|16|37|PM|third|evening|| +83798|AAAAAAAAHFHEBAAA|83798|23|16|38|PM|third|evening|| +83799|AAAAAAAAIFHEBAAA|83799|23|16|39|PM|third|evening|| +83800|AAAAAAAAJFHEBAAA|83800|23|16|40|PM|third|evening|| +83801|AAAAAAAAKFHEBAAA|83801|23|16|41|PM|third|evening|| +83802|AAAAAAAALFHEBAAA|83802|23|16|42|PM|third|evening|| +83803|AAAAAAAAMFHEBAAA|83803|23|16|43|PM|third|evening|| +83804|AAAAAAAANFHEBAAA|83804|23|16|44|PM|third|evening|| +83805|AAAAAAAAOFHEBAAA|83805|23|16|45|PM|third|evening|| +83806|AAAAAAAAPFHEBAAA|83806|23|16|46|PM|third|evening|| +83807|AAAAAAAAAGHEBAAA|83807|23|16|47|PM|third|evening|| +83808|AAAAAAAABGHEBAAA|83808|23|16|48|PM|third|evening|| +83809|AAAAAAAACGHEBAAA|83809|23|16|49|PM|third|evening|| +83810|AAAAAAAADGHEBAAA|83810|23|16|50|PM|third|evening|| +83811|AAAAAAAAEGHEBAAA|83811|23|16|51|PM|third|evening|| +83812|AAAAAAAAFGHEBAAA|83812|23|16|52|PM|third|evening|| +83813|AAAAAAAAGGHEBAAA|83813|23|16|53|PM|third|evening|| +83814|AAAAAAAAHGHEBAAA|83814|23|16|54|PM|third|evening|| +83815|AAAAAAAAIGHEBAAA|83815|23|16|55|PM|third|evening|| +83816|AAAAAAAAJGHEBAAA|83816|23|16|56|PM|third|evening|| +83817|AAAAAAAAKGHEBAAA|83817|23|16|57|PM|third|evening|| +83818|AAAAAAAALGHEBAAA|83818|23|16|58|PM|third|evening|| +83819|AAAAAAAAMGHEBAAA|83819|23|16|59|PM|third|evening|| +83820|AAAAAAAANGHEBAAA|83820|23|17|0|PM|third|evening|| +83821|AAAAAAAAOGHEBAAA|83821|23|17|1|PM|third|evening|| +83822|AAAAAAAAPGHEBAAA|83822|23|17|2|PM|third|evening|| +83823|AAAAAAAAAHHEBAAA|83823|23|17|3|PM|third|evening|| +83824|AAAAAAAABHHEBAAA|83824|23|17|4|PM|third|evening|| +83825|AAAAAAAACHHEBAAA|83825|23|17|5|PM|third|evening|| +83826|AAAAAAAADHHEBAAA|83826|23|17|6|PM|third|evening|| +83827|AAAAAAAAEHHEBAAA|83827|23|17|7|PM|third|evening|| +83828|AAAAAAAAFHHEBAAA|83828|23|17|8|PM|third|evening|| +83829|AAAAAAAAGHHEBAAA|83829|23|17|9|PM|third|evening|| +83830|AAAAAAAAHHHEBAAA|83830|23|17|10|PM|third|evening|| +83831|AAAAAAAAIHHEBAAA|83831|23|17|11|PM|third|evening|| +83832|AAAAAAAAJHHEBAAA|83832|23|17|12|PM|third|evening|| +83833|AAAAAAAAKHHEBAAA|83833|23|17|13|PM|third|evening|| +83834|AAAAAAAALHHEBAAA|83834|23|17|14|PM|third|evening|| +83835|AAAAAAAAMHHEBAAA|83835|23|17|15|PM|third|evening|| +83836|AAAAAAAANHHEBAAA|83836|23|17|16|PM|third|evening|| +83837|AAAAAAAAOHHEBAAA|83837|23|17|17|PM|third|evening|| +83838|AAAAAAAAPHHEBAAA|83838|23|17|18|PM|third|evening|| +83839|AAAAAAAAAIHEBAAA|83839|23|17|19|PM|third|evening|| +83840|AAAAAAAABIHEBAAA|83840|23|17|20|PM|third|evening|| +83841|AAAAAAAACIHEBAAA|83841|23|17|21|PM|third|evening|| +83842|AAAAAAAADIHEBAAA|83842|23|17|22|PM|third|evening|| +83843|AAAAAAAAEIHEBAAA|83843|23|17|23|PM|third|evening|| +83844|AAAAAAAAFIHEBAAA|83844|23|17|24|PM|third|evening|| +83845|AAAAAAAAGIHEBAAA|83845|23|17|25|PM|third|evening|| +83846|AAAAAAAAHIHEBAAA|83846|23|17|26|PM|third|evening|| +83847|AAAAAAAAIIHEBAAA|83847|23|17|27|PM|third|evening|| +83848|AAAAAAAAJIHEBAAA|83848|23|17|28|PM|third|evening|| +83849|AAAAAAAAKIHEBAAA|83849|23|17|29|PM|third|evening|| +83850|AAAAAAAALIHEBAAA|83850|23|17|30|PM|third|evening|| +83851|AAAAAAAAMIHEBAAA|83851|23|17|31|PM|third|evening|| +83852|AAAAAAAANIHEBAAA|83852|23|17|32|PM|third|evening|| +83853|AAAAAAAAOIHEBAAA|83853|23|17|33|PM|third|evening|| +83854|AAAAAAAAPIHEBAAA|83854|23|17|34|PM|third|evening|| +83855|AAAAAAAAAJHEBAAA|83855|23|17|35|PM|third|evening|| +83856|AAAAAAAABJHEBAAA|83856|23|17|36|PM|third|evening|| +83857|AAAAAAAACJHEBAAA|83857|23|17|37|PM|third|evening|| +83858|AAAAAAAADJHEBAAA|83858|23|17|38|PM|third|evening|| +83859|AAAAAAAAEJHEBAAA|83859|23|17|39|PM|third|evening|| +83860|AAAAAAAAFJHEBAAA|83860|23|17|40|PM|third|evening|| +83861|AAAAAAAAGJHEBAAA|83861|23|17|41|PM|third|evening|| +83862|AAAAAAAAHJHEBAAA|83862|23|17|42|PM|third|evening|| +83863|AAAAAAAAIJHEBAAA|83863|23|17|43|PM|third|evening|| +83864|AAAAAAAAJJHEBAAA|83864|23|17|44|PM|third|evening|| +83865|AAAAAAAAKJHEBAAA|83865|23|17|45|PM|third|evening|| +83866|AAAAAAAALJHEBAAA|83866|23|17|46|PM|third|evening|| +83867|AAAAAAAAMJHEBAAA|83867|23|17|47|PM|third|evening|| +83868|AAAAAAAANJHEBAAA|83868|23|17|48|PM|third|evening|| +83869|AAAAAAAAOJHEBAAA|83869|23|17|49|PM|third|evening|| +83870|AAAAAAAAPJHEBAAA|83870|23|17|50|PM|third|evening|| +83871|AAAAAAAAAKHEBAAA|83871|23|17|51|PM|third|evening|| +83872|AAAAAAAABKHEBAAA|83872|23|17|52|PM|third|evening|| +83873|AAAAAAAACKHEBAAA|83873|23|17|53|PM|third|evening|| +83874|AAAAAAAADKHEBAAA|83874|23|17|54|PM|third|evening|| +83875|AAAAAAAAEKHEBAAA|83875|23|17|55|PM|third|evening|| +83876|AAAAAAAAFKHEBAAA|83876|23|17|56|PM|third|evening|| +83877|AAAAAAAAGKHEBAAA|83877|23|17|57|PM|third|evening|| +83878|AAAAAAAAHKHEBAAA|83878|23|17|58|PM|third|evening|| +83879|AAAAAAAAIKHEBAAA|83879|23|17|59|PM|third|evening|| +83880|AAAAAAAAJKHEBAAA|83880|23|18|0|PM|third|evening|| +83881|AAAAAAAAKKHEBAAA|83881|23|18|1|PM|third|evening|| +83882|AAAAAAAALKHEBAAA|83882|23|18|2|PM|third|evening|| +83883|AAAAAAAAMKHEBAAA|83883|23|18|3|PM|third|evening|| +83884|AAAAAAAANKHEBAAA|83884|23|18|4|PM|third|evening|| +83885|AAAAAAAAOKHEBAAA|83885|23|18|5|PM|third|evening|| +83886|AAAAAAAAPKHEBAAA|83886|23|18|6|PM|third|evening|| +83887|AAAAAAAAALHEBAAA|83887|23|18|7|PM|third|evening|| +83888|AAAAAAAABLHEBAAA|83888|23|18|8|PM|third|evening|| +83889|AAAAAAAACLHEBAAA|83889|23|18|9|PM|third|evening|| +83890|AAAAAAAADLHEBAAA|83890|23|18|10|PM|third|evening|| +83891|AAAAAAAAELHEBAAA|83891|23|18|11|PM|third|evening|| +83892|AAAAAAAAFLHEBAAA|83892|23|18|12|PM|third|evening|| +83893|AAAAAAAAGLHEBAAA|83893|23|18|13|PM|third|evening|| +83894|AAAAAAAAHLHEBAAA|83894|23|18|14|PM|third|evening|| +83895|AAAAAAAAILHEBAAA|83895|23|18|15|PM|third|evening|| +83896|AAAAAAAAJLHEBAAA|83896|23|18|16|PM|third|evening|| +83897|AAAAAAAAKLHEBAAA|83897|23|18|17|PM|third|evening|| +83898|AAAAAAAALLHEBAAA|83898|23|18|18|PM|third|evening|| +83899|AAAAAAAAMLHEBAAA|83899|23|18|19|PM|third|evening|| +83900|AAAAAAAANLHEBAAA|83900|23|18|20|PM|third|evening|| +83901|AAAAAAAAOLHEBAAA|83901|23|18|21|PM|third|evening|| +83902|AAAAAAAAPLHEBAAA|83902|23|18|22|PM|third|evening|| +83903|AAAAAAAAAMHEBAAA|83903|23|18|23|PM|third|evening|| +83904|AAAAAAAABMHEBAAA|83904|23|18|24|PM|third|evening|| +83905|AAAAAAAACMHEBAAA|83905|23|18|25|PM|third|evening|| +83906|AAAAAAAADMHEBAAA|83906|23|18|26|PM|third|evening|| +83907|AAAAAAAAEMHEBAAA|83907|23|18|27|PM|third|evening|| +83908|AAAAAAAAFMHEBAAA|83908|23|18|28|PM|third|evening|| +83909|AAAAAAAAGMHEBAAA|83909|23|18|29|PM|third|evening|| +83910|AAAAAAAAHMHEBAAA|83910|23|18|30|PM|third|evening|| +83911|AAAAAAAAIMHEBAAA|83911|23|18|31|PM|third|evening|| +83912|AAAAAAAAJMHEBAAA|83912|23|18|32|PM|third|evening|| +83913|AAAAAAAAKMHEBAAA|83913|23|18|33|PM|third|evening|| +83914|AAAAAAAALMHEBAAA|83914|23|18|34|PM|third|evening|| +83915|AAAAAAAAMMHEBAAA|83915|23|18|35|PM|third|evening|| +83916|AAAAAAAANMHEBAAA|83916|23|18|36|PM|third|evening|| +83917|AAAAAAAAOMHEBAAA|83917|23|18|37|PM|third|evening|| +83918|AAAAAAAAPMHEBAAA|83918|23|18|38|PM|third|evening|| +83919|AAAAAAAAANHEBAAA|83919|23|18|39|PM|third|evening|| +83920|AAAAAAAABNHEBAAA|83920|23|18|40|PM|third|evening|| +83921|AAAAAAAACNHEBAAA|83921|23|18|41|PM|third|evening|| +83922|AAAAAAAADNHEBAAA|83922|23|18|42|PM|third|evening|| +83923|AAAAAAAAENHEBAAA|83923|23|18|43|PM|third|evening|| +83924|AAAAAAAAFNHEBAAA|83924|23|18|44|PM|third|evening|| +83925|AAAAAAAAGNHEBAAA|83925|23|18|45|PM|third|evening|| +83926|AAAAAAAAHNHEBAAA|83926|23|18|46|PM|third|evening|| +83927|AAAAAAAAINHEBAAA|83927|23|18|47|PM|third|evening|| +83928|AAAAAAAAJNHEBAAA|83928|23|18|48|PM|third|evening|| +83929|AAAAAAAAKNHEBAAA|83929|23|18|49|PM|third|evening|| +83930|AAAAAAAALNHEBAAA|83930|23|18|50|PM|third|evening|| +83931|AAAAAAAAMNHEBAAA|83931|23|18|51|PM|third|evening|| +83932|AAAAAAAANNHEBAAA|83932|23|18|52|PM|third|evening|| +83933|AAAAAAAAONHEBAAA|83933|23|18|53|PM|third|evening|| +83934|AAAAAAAAPNHEBAAA|83934|23|18|54|PM|third|evening|| +83935|AAAAAAAAAOHEBAAA|83935|23|18|55|PM|third|evening|| +83936|AAAAAAAABOHEBAAA|83936|23|18|56|PM|third|evening|| +83937|AAAAAAAACOHEBAAA|83937|23|18|57|PM|third|evening|| +83938|AAAAAAAADOHEBAAA|83938|23|18|58|PM|third|evening|| +83939|AAAAAAAAEOHEBAAA|83939|23|18|59|PM|third|evening|| +83940|AAAAAAAAFOHEBAAA|83940|23|19|0|PM|third|evening|| +83941|AAAAAAAAGOHEBAAA|83941|23|19|1|PM|third|evening|| +83942|AAAAAAAAHOHEBAAA|83942|23|19|2|PM|third|evening|| +83943|AAAAAAAAIOHEBAAA|83943|23|19|3|PM|third|evening|| +83944|AAAAAAAAJOHEBAAA|83944|23|19|4|PM|third|evening|| +83945|AAAAAAAAKOHEBAAA|83945|23|19|5|PM|third|evening|| +83946|AAAAAAAALOHEBAAA|83946|23|19|6|PM|third|evening|| +83947|AAAAAAAAMOHEBAAA|83947|23|19|7|PM|third|evening|| +83948|AAAAAAAANOHEBAAA|83948|23|19|8|PM|third|evening|| +83949|AAAAAAAAOOHEBAAA|83949|23|19|9|PM|third|evening|| +83950|AAAAAAAAPOHEBAAA|83950|23|19|10|PM|third|evening|| +83951|AAAAAAAAAPHEBAAA|83951|23|19|11|PM|third|evening|| +83952|AAAAAAAABPHEBAAA|83952|23|19|12|PM|third|evening|| +83953|AAAAAAAACPHEBAAA|83953|23|19|13|PM|third|evening|| +83954|AAAAAAAADPHEBAAA|83954|23|19|14|PM|third|evening|| +83955|AAAAAAAAEPHEBAAA|83955|23|19|15|PM|third|evening|| +83956|AAAAAAAAFPHEBAAA|83956|23|19|16|PM|third|evening|| +83957|AAAAAAAAGPHEBAAA|83957|23|19|17|PM|third|evening|| +83958|AAAAAAAAHPHEBAAA|83958|23|19|18|PM|third|evening|| +83959|AAAAAAAAIPHEBAAA|83959|23|19|19|PM|third|evening|| +83960|AAAAAAAAJPHEBAAA|83960|23|19|20|PM|third|evening|| +83961|AAAAAAAAKPHEBAAA|83961|23|19|21|PM|third|evening|| +83962|AAAAAAAALPHEBAAA|83962|23|19|22|PM|third|evening|| +83963|AAAAAAAAMPHEBAAA|83963|23|19|23|PM|third|evening|| +83964|AAAAAAAANPHEBAAA|83964|23|19|24|PM|third|evening|| +83965|AAAAAAAAOPHEBAAA|83965|23|19|25|PM|third|evening|| +83966|AAAAAAAAPPHEBAAA|83966|23|19|26|PM|third|evening|| +83967|AAAAAAAAAAIEBAAA|83967|23|19|27|PM|third|evening|| +83968|AAAAAAAABAIEBAAA|83968|23|19|28|PM|third|evening|| +83969|AAAAAAAACAIEBAAA|83969|23|19|29|PM|third|evening|| +83970|AAAAAAAADAIEBAAA|83970|23|19|30|PM|third|evening|| +83971|AAAAAAAAEAIEBAAA|83971|23|19|31|PM|third|evening|| +83972|AAAAAAAAFAIEBAAA|83972|23|19|32|PM|third|evening|| +83973|AAAAAAAAGAIEBAAA|83973|23|19|33|PM|third|evening|| +83974|AAAAAAAAHAIEBAAA|83974|23|19|34|PM|third|evening|| +83975|AAAAAAAAIAIEBAAA|83975|23|19|35|PM|third|evening|| +83976|AAAAAAAAJAIEBAAA|83976|23|19|36|PM|third|evening|| +83977|AAAAAAAAKAIEBAAA|83977|23|19|37|PM|third|evening|| +83978|AAAAAAAALAIEBAAA|83978|23|19|38|PM|third|evening|| +83979|AAAAAAAAMAIEBAAA|83979|23|19|39|PM|third|evening|| +83980|AAAAAAAANAIEBAAA|83980|23|19|40|PM|third|evening|| +83981|AAAAAAAAOAIEBAAA|83981|23|19|41|PM|third|evening|| +83982|AAAAAAAAPAIEBAAA|83982|23|19|42|PM|third|evening|| +83983|AAAAAAAAABIEBAAA|83983|23|19|43|PM|third|evening|| +83984|AAAAAAAABBIEBAAA|83984|23|19|44|PM|third|evening|| +83985|AAAAAAAACBIEBAAA|83985|23|19|45|PM|third|evening|| +83986|AAAAAAAADBIEBAAA|83986|23|19|46|PM|third|evening|| +83987|AAAAAAAAEBIEBAAA|83987|23|19|47|PM|third|evening|| +83988|AAAAAAAAFBIEBAAA|83988|23|19|48|PM|third|evening|| +83989|AAAAAAAAGBIEBAAA|83989|23|19|49|PM|third|evening|| +83990|AAAAAAAAHBIEBAAA|83990|23|19|50|PM|third|evening|| +83991|AAAAAAAAIBIEBAAA|83991|23|19|51|PM|third|evening|| +83992|AAAAAAAAJBIEBAAA|83992|23|19|52|PM|third|evening|| +83993|AAAAAAAAKBIEBAAA|83993|23|19|53|PM|third|evening|| +83994|AAAAAAAALBIEBAAA|83994|23|19|54|PM|third|evening|| +83995|AAAAAAAAMBIEBAAA|83995|23|19|55|PM|third|evening|| +83996|AAAAAAAANBIEBAAA|83996|23|19|56|PM|third|evening|| +83997|AAAAAAAAOBIEBAAA|83997|23|19|57|PM|third|evening|| +83998|AAAAAAAAPBIEBAAA|83998|23|19|58|PM|third|evening|| +83999|AAAAAAAAACIEBAAA|83999|23|19|59|PM|third|evening|| +84000|AAAAAAAABCIEBAAA|84000|23|20|0|PM|third|evening|| +84001|AAAAAAAACCIEBAAA|84001|23|20|1|PM|third|evening|| +84002|AAAAAAAADCIEBAAA|84002|23|20|2|PM|third|evening|| +84003|AAAAAAAAECIEBAAA|84003|23|20|3|PM|third|evening|| +84004|AAAAAAAAFCIEBAAA|84004|23|20|4|PM|third|evening|| +84005|AAAAAAAAGCIEBAAA|84005|23|20|5|PM|third|evening|| +84006|AAAAAAAAHCIEBAAA|84006|23|20|6|PM|third|evening|| +84007|AAAAAAAAICIEBAAA|84007|23|20|7|PM|third|evening|| +84008|AAAAAAAAJCIEBAAA|84008|23|20|8|PM|third|evening|| +84009|AAAAAAAAKCIEBAAA|84009|23|20|9|PM|third|evening|| +84010|AAAAAAAALCIEBAAA|84010|23|20|10|PM|third|evening|| +84011|AAAAAAAAMCIEBAAA|84011|23|20|11|PM|third|evening|| +84012|AAAAAAAANCIEBAAA|84012|23|20|12|PM|third|evening|| +84013|AAAAAAAAOCIEBAAA|84013|23|20|13|PM|third|evening|| +84014|AAAAAAAAPCIEBAAA|84014|23|20|14|PM|third|evening|| +84015|AAAAAAAAADIEBAAA|84015|23|20|15|PM|third|evening|| +84016|AAAAAAAABDIEBAAA|84016|23|20|16|PM|third|evening|| +84017|AAAAAAAACDIEBAAA|84017|23|20|17|PM|third|evening|| +84018|AAAAAAAADDIEBAAA|84018|23|20|18|PM|third|evening|| +84019|AAAAAAAAEDIEBAAA|84019|23|20|19|PM|third|evening|| +84020|AAAAAAAAFDIEBAAA|84020|23|20|20|PM|third|evening|| +84021|AAAAAAAAGDIEBAAA|84021|23|20|21|PM|third|evening|| +84022|AAAAAAAAHDIEBAAA|84022|23|20|22|PM|third|evening|| +84023|AAAAAAAAIDIEBAAA|84023|23|20|23|PM|third|evening|| +84024|AAAAAAAAJDIEBAAA|84024|23|20|24|PM|third|evening|| +84025|AAAAAAAAKDIEBAAA|84025|23|20|25|PM|third|evening|| +84026|AAAAAAAALDIEBAAA|84026|23|20|26|PM|third|evening|| +84027|AAAAAAAAMDIEBAAA|84027|23|20|27|PM|third|evening|| +84028|AAAAAAAANDIEBAAA|84028|23|20|28|PM|third|evening|| +84029|AAAAAAAAODIEBAAA|84029|23|20|29|PM|third|evening|| +84030|AAAAAAAAPDIEBAAA|84030|23|20|30|PM|third|evening|| +84031|AAAAAAAAAEIEBAAA|84031|23|20|31|PM|third|evening|| +84032|AAAAAAAABEIEBAAA|84032|23|20|32|PM|third|evening|| +84033|AAAAAAAACEIEBAAA|84033|23|20|33|PM|third|evening|| +84034|AAAAAAAADEIEBAAA|84034|23|20|34|PM|third|evening|| +84035|AAAAAAAAEEIEBAAA|84035|23|20|35|PM|third|evening|| +84036|AAAAAAAAFEIEBAAA|84036|23|20|36|PM|third|evening|| +84037|AAAAAAAAGEIEBAAA|84037|23|20|37|PM|third|evening|| +84038|AAAAAAAAHEIEBAAA|84038|23|20|38|PM|third|evening|| +84039|AAAAAAAAIEIEBAAA|84039|23|20|39|PM|third|evening|| +84040|AAAAAAAAJEIEBAAA|84040|23|20|40|PM|third|evening|| +84041|AAAAAAAAKEIEBAAA|84041|23|20|41|PM|third|evening|| +84042|AAAAAAAALEIEBAAA|84042|23|20|42|PM|third|evening|| +84043|AAAAAAAAMEIEBAAA|84043|23|20|43|PM|third|evening|| +84044|AAAAAAAANEIEBAAA|84044|23|20|44|PM|third|evening|| +84045|AAAAAAAAOEIEBAAA|84045|23|20|45|PM|third|evening|| +84046|AAAAAAAAPEIEBAAA|84046|23|20|46|PM|third|evening|| +84047|AAAAAAAAAFIEBAAA|84047|23|20|47|PM|third|evening|| +84048|AAAAAAAABFIEBAAA|84048|23|20|48|PM|third|evening|| +84049|AAAAAAAACFIEBAAA|84049|23|20|49|PM|third|evening|| +84050|AAAAAAAADFIEBAAA|84050|23|20|50|PM|third|evening|| +84051|AAAAAAAAEFIEBAAA|84051|23|20|51|PM|third|evening|| +84052|AAAAAAAAFFIEBAAA|84052|23|20|52|PM|third|evening|| +84053|AAAAAAAAGFIEBAAA|84053|23|20|53|PM|third|evening|| +84054|AAAAAAAAHFIEBAAA|84054|23|20|54|PM|third|evening|| +84055|AAAAAAAAIFIEBAAA|84055|23|20|55|PM|third|evening|| +84056|AAAAAAAAJFIEBAAA|84056|23|20|56|PM|third|evening|| +84057|AAAAAAAAKFIEBAAA|84057|23|20|57|PM|third|evening|| +84058|AAAAAAAALFIEBAAA|84058|23|20|58|PM|third|evening|| +84059|AAAAAAAAMFIEBAAA|84059|23|20|59|PM|third|evening|| +84060|AAAAAAAANFIEBAAA|84060|23|21|0|PM|third|evening|| +84061|AAAAAAAAOFIEBAAA|84061|23|21|1|PM|third|evening|| +84062|AAAAAAAAPFIEBAAA|84062|23|21|2|PM|third|evening|| +84063|AAAAAAAAAGIEBAAA|84063|23|21|3|PM|third|evening|| +84064|AAAAAAAABGIEBAAA|84064|23|21|4|PM|third|evening|| +84065|AAAAAAAACGIEBAAA|84065|23|21|5|PM|third|evening|| +84066|AAAAAAAADGIEBAAA|84066|23|21|6|PM|third|evening|| +84067|AAAAAAAAEGIEBAAA|84067|23|21|7|PM|third|evening|| +84068|AAAAAAAAFGIEBAAA|84068|23|21|8|PM|third|evening|| +84069|AAAAAAAAGGIEBAAA|84069|23|21|9|PM|third|evening|| +84070|AAAAAAAAHGIEBAAA|84070|23|21|10|PM|third|evening|| +84071|AAAAAAAAIGIEBAAA|84071|23|21|11|PM|third|evening|| +84072|AAAAAAAAJGIEBAAA|84072|23|21|12|PM|third|evening|| +84073|AAAAAAAAKGIEBAAA|84073|23|21|13|PM|third|evening|| +84074|AAAAAAAALGIEBAAA|84074|23|21|14|PM|third|evening|| +84075|AAAAAAAAMGIEBAAA|84075|23|21|15|PM|third|evening|| +84076|AAAAAAAANGIEBAAA|84076|23|21|16|PM|third|evening|| +84077|AAAAAAAAOGIEBAAA|84077|23|21|17|PM|third|evening|| +84078|AAAAAAAAPGIEBAAA|84078|23|21|18|PM|third|evening|| +84079|AAAAAAAAAHIEBAAA|84079|23|21|19|PM|third|evening|| +84080|AAAAAAAABHIEBAAA|84080|23|21|20|PM|third|evening|| +84081|AAAAAAAACHIEBAAA|84081|23|21|21|PM|third|evening|| +84082|AAAAAAAADHIEBAAA|84082|23|21|22|PM|third|evening|| +84083|AAAAAAAAEHIEBAAA|84083|23|21|23|PM|third|evening|| +84084|AAAAAAAAFHIEBAAA|84084|23|21|24|PM|third|evening|| +84085|AAAAAAAAGHIEBAAA|84085|23|21|25|PM|third|evening|| +84086|AAAAAAAAHHIEBAAA|84086|23|21|26|PM|third|evening|| +84087|AAAAAAAAIHIEBAAA|84087|23|21|27|PM|third|evening|| +84088|AAAAAAAAJHIEBAAA|84088|23|21|28|PM|third|evening|| +84089|AAAAAAAAKHIEBAAA|84089|23|21|29|PM|third|evening|| +84090|AAAAAAAALHIEBAAA|84090|23|21|30|PM|third|evening|| +84091|AAAAAAAAMHIEBAAA|84091|23|21|31|PM|third|evening|| +84092|AAAAAAAANHIEBAAA|84092|23|21|32|PM|third|evening|| +84093|AAAAAAAAOHIEBAAA|84093|23|21|33|PM|third|evening|| +84094|AAAAAAAAPHIEBAAA|84094|23|21|34|PM|third|evening|| +84095|AAAAAAAAAIIEBAAA|84095|23|21|35|PM|third|evening|| +84096|AAAAAAAABIIEBAAA|84096|23|21|36|PM|third|evening|| +84097|AAAAAAAACIIEBAAA|84097|23|21|37|PM|third|evening|| +84098|AAAAAAAADIIEBAAA|84098|23|21|38|PM|third|evening|| +84099|AAAAAAAAEIIEBAAA|84099|23|21|39|PM|third|evening|| +84100|AAAAAAAAFIIEBAAA|84100|23|21|40|PM|third|evening|| +84101|AAAAAAAAGIIEBAAA|84101|23|21|41|PM|third|evening|| +84102|AAAAAAAAHIIEBAAA|84102|23|21|42|PM|third|evening|| +84103|AAAAAAAAIIIEBAAA|84103|23|21|43|PM|third|evening|| +84104|AAAAAAAAJIIEBAAA|84104|23|21|44|PM|third|evening|| +84105|AAAAAAAAKIIEBAAA|84105|23|21|45|PM|third|evening|| +84106|AAAAAAAALIIEBAAA|84106|23|21|46|PM|third|evening|| +84107|AAAAAAAAMIIEBAAA|84107|23|21|47|PM|third|evening|| +84108|AAAAAAAANIIEBAAA|84108|23|21|48|PM|third|evening|| +84109|AAAAAAAAOIIEBAAA|84109|23|21|49|PM|third|evening|| +84110|AAAAAAAAPIIEBAAA|84110|23|21|50|PM|third|evening|| +84111|AAAAAAAAAJIEBAAA|84111|23|21|51|PM|third|evening|| +84112|AAAAAAAABJIEBAAA|84112|23|21|52|PM|third|evening|| +84113|AAAAAAAACJIEBAAA|84113|23|21|53|PM|third|evening|| +84114|AAAAAAAADJIEBAAA|84114|23|21|54|PM|third|evening|| +84115|AAAAAAAAEJIEBAAA|84115|23|21|55|PM|third|evening|| +84116|AAAAAAAAFJIEBAAA|84116|23|21|56|PM|third|evening|| +84117|AAAAAAAAGJIEBAAA|84117|23|21|57|PM|third|evening|| +84118|AAAAAAAAHJIEBAAA|84118|23|21|58|PM|third|evening|| +84119|AAAAAAAAIJIEBAAA|84119|23|21|59|PM|third|evening|| +84120|AAAAAAAAJJIEBAAA|84120|23|22|0|PM|third|evening|| +84121|AAAAAAAAKJIEBAAA|84121|23|22|1|PM|third|evening|| +84122|AAAAAAAALJIEBAAA|84122|23|22|2|PM|third|evening|| +84123|AAAAAAAAMJIEBAAA|84123|23|22|3|PM|third|evening|| +84124|AAAAAAAANJIEBAAA|84124|23|22|4|PM|third|evening|| +84125|AAAAAAAAOJIEBAAA|84125|23|22|5|PM|third|evening|| +84126|AAAAAAAAPJIEBAAA|84126|23|22|6|PM|third|evening|| +84127|AAAAAAAAAKIEBAAA|84127|23|22|7|PM|third|evening|| +84128|AAAAAAAABKIEBAAA|84128|23|22|8|PM|third|evening|| +84129|AAAAAAAACKIEBAAA|84129|23|22|9|PM|third|evening|| +84130|AAAAAAAADKIEBAAA|84130|23|22|10|PM|third|evening|| +84131|AAAAAAAAEKIEBAAA|84131|23|22|11|PM|third|evening|| +84132|AAAAAAAAFKIEBAAA|84132|23|22|12|PM|third|evening|| +84133|AAAAAAAAGKIEBAAA|84133|23|22|13|PM|third|evening|| +84134|AAAAAAAAHKIEBAAA|84134|23|22|14|PM|third|evening|| +84135|AAAAAAAAIKIEBAAA|84135|23|22|15|PM|third|evening|| +84136|AAAAAAAAJKIEBAAA|84136|23|22|16|PM|third|evening|| +84137|AAAAAAAAKKIEBAAA|84137|23|22|17|PM|third|evening|| +84138|AAAAAAAALKIEBAAA|84138|23|22|18|PM|third|evening|| +84139|AAAAAAAAMKIEBAAA|84139|23|22|19|PM|third|evening|| +84140|AAAAAAAANKIEBAAA|84140|23|22|20|PM|third|evening|| +84141|AAAAAAAAOKIEBAAA|84141|23|22|21|PM|third|evening|| +84142|AAAAAAAAPKIEBAAA|84142|23|22|22|PM|third|evening|| +84143|AAAAAAAAALIEBAAA|84143|23|22|23|PM|third|evening|| +84144|AAAAAAAABLIEBAAA|84144|23|22|24|PM|third|evening|| +84145|AAAAAAAACLIEBAAA|84145|23|22|25|PM|third|evening|| +84146|AAAAAAAADLIEBAAA|84146|23|22|26|PM|third|evening|| +84147|AAAAAAAAELIEBAAA|84147|23|22|27|PM|third|evening|| +84148|AAAAAAAAFLIEBAAA|84148|23|22|28|PM|third|evening|| +84149|AAAAAAAAGLIEBAAA|84149|23|22|29|PM|third|evening|| +84150|AAAAAAAAHLIEBAAA|84150|23|22|30|PM|third|evening|| +84151|AAAAAAAAILIEBAAA|84151|23|22|31|PM|third|evening|| +84152|AAAAAAAAJLIEBAAA|84152|23|22|32|PM|third|evening|| +84153|AAAAAAAAKLIEBAAA|84153|23|22|33|PM|third|evening|| +84154|AAAAAAAALLIEBAAA|84154|23|22|34|PM|third|evening|| +84155|AAAAAAAAMLIEBAAA|84155|23|22|35|PM|third|evening|| +84156|AAAAAAAANLIEBAAA|84156|23|22|36|PM|third|evening|| +84157|AAAAAAAAOLIEBAAA|84157|23|22|37|PM|third|evening|| +84158|AAAAAAAAPLIEBAAA|84158|23|22|38|PM|third|evening|| +84159|AAAAAAAAAMIEBAAA|84159|23|22|39|PM|third|evening|| +84160|AAAAAAAABMIEBAAA|84160|23|22|40|PM|third|evening|| +84161|AAAAAAAACMIEBAAA|84161|23|22|41|PM|third|evening|| +84162|AAAAAAAADMIEBAAA|84162|23|22|42|PM|third|evening|| +84163|AAAAAAAAEMIEBAAA|84163|23|22|43|PM|third|evening|| +84164|AAAAAAAAFMIEBAAA|84164|23|22|44|PM|third|evening|| +84165|AAAAAAAAGMIEBAAA|84165|23|22|45|PM|third|evening|| +84166|AAAAAAAAHMIEBAAA|84166|23|22|46|PM|third|evening|| +84167|AAAAAAAAIMIEBAAA|84167|23|22|47|PM|third|evening|| +84168|AAAAAAAAJMIEBAAA|84168|23|22|48|PM|third|evening|| +84169|AAAAAAAAKMIEBAAA|84169|23|22|49|PM|third|evening|| +84170|AAAAAAAALMIEBAAA|84170|23|22|50|PM|third|evening|| +84171|AAAAAAAAMMIEBAAA|84171|23|22|51|PM|third|evening|| +84172|AAAAAAAANMIEBAAA|84172|23|22|52|PM|third|evening|| +84173|AAAAAAAAOMIEBAAA|84173|23|22|53|PM|third|evening|| +84174|AAAAAAAAPMIEBAAA|84174|23|22|54|PM|third|evening|| +84175|AAAAAAAAANIEBAAA|84175|23|22|55|PM|third|evening|| +84176|AAAAAAAABNIEBAAA|84176|23|22|56|PM|third|evening|| +84177|AAAAAAAACNIEBAAA|84177|23|22|57|PM|third|evening|| +84178|AAAAAAAADNIEBAAA|84178|23|22|58|PM|third|evening|| +84179|AAAAAAAAENIEBAAA|84179|23|22|59|PM|third|evening|| +84180|AAAAAAAAFNIEBAAA|84180|23|23|0|PM|third|evening|| +84181|AAAAAAAAGNIEBAAA|84181|23|23|1|PM|third|evening|| +84182|AAAAAAAAHNIEBAAA|84182|23|23|2|PM|third|evening|| +84183|AAAAAAAAINIEBAAA|84183|23|23|3|PM|third|evening|| +84184|AAAAAAAAJNIEBAAA|84184|23|23|4|PM|third|evening|| +84185|AAAAAAAAKNIEBAAA|84185|23|23|5|PM|third|evening|| +84186|AAAAAAAALNIEBAAA|84186|23|23|6|PM|third|evening|| +84187|AAAAAAAAMNIEBAAA|84187|23|23|7|PM|third|evening|| +84188|AAAAAAAANNIEBAAA|84188|23|23|8|PM|third|evening|| +84189|AAAAAAAAONIEBAAA|84189|23|23|9|PM|third|evening|| +84190|AAAAAAAAPNIEBAAA|84190|23|23|10|PM|third|evening|| +84191|AAAAAAAAAOIEBAAA|84191|23|23|11|PM|third|evening|| +84192|AAAAAAAABOIEBAAA|84192|23|23|12|PM|third|evening|| +84193|AAAAAAAACOIEBAAA|84193|23|23|13|PM|third|evening|| +84194|AAAAAAAADOIEBAAA|84194|23|23|14|PM|third|evening|| +84195|AAAAAAAAEOIEBAAA|84195|23|23|15|PM|third|evening|| +84196|AAAAAAAAFOIEBAAA|84196|23|23|16|PM|third|evening|| +84197|AAAAAAAAGOIEBAAA|84197|23|23|17|PM|third|evening|| +84198|AAAAAAAAHOIEBAAA|84198|23|23|18|PM|third|evening|| +84199|AAAAAAAAIOIEBAAA|84199|23|23|19|PM|third|evening|| +84200|AAAAAAAAJOIEBAAA|84200|23|23|20|PM|third|evening|| +84201|AAAAAAAAKOIEBAAA|84201|23|23|21|PM|third|evening|| +84202|AAAAAAAALOIEBAAA|84202|23|23|22|PM|third|evening|| +84203|AAAAAAAAMOIEBAAA|84203|23|23|23|PM|third|evening|| +84204|AAAAAAAANOIEBAAA|84204|23|23|24|PM|third|evening|| +84205|AAAAAAAAOOIEBAAA|84205|23|23|25|PM|third|evening|| +84206|AAAAAAAAPOIEBAAA|84206|23|23|26|PM|third|evening|| +84207|AAAAAAAAAPIEBAAA|84207|23|23|27|PM|third|evening|| +84208|AAAAAAAABPIEBAAA|84208|23|23|28|PM|third|evening|| +84209|AAAAAAAACPIEBAAA|84209|23|23|29|PM|third|evening|| +84210|AAAAAAAADPIEBAAA|84210|23|23|30|PM|third|evening|| +84211|AAAAAAAAEPIEBAAA|84211|23|23|31|PM|third|evening|| +84212|AAAAAAAAFPIEBAAA|84212|23|23|32|PM|third|evening|| +84213|AAAAAAAAGPIEBAAA|84213|23|23|33|PM|third|evening|| +84214|AAAAAAAAHPIEBAAA|84214|23|23|34|PM|third|evening|| +84215|AAAAAAAAIPIEBAAA|84215|23|23|35|PM|third|evening|| +84216|AAAAAAAAJPIEBAAA|84216|23|23|36|PM|third|evening|| +84217|AAAAAAAAKPIEBAAA|84217|23|23|37|PM|third|evening|| +84218|AAAAAAAALPIEBAAA|84218|23|23|38|PM|third|evening|| +84219|AAAAAAAAMPIEBAAA|84219|23|23|39|PM|third|evening|| +84220|AAAAAAAANPIEBAAA|84220|23|23|40|PM|third|evening|| +84221|AAAAAAAAOPIEBAAA|84221|23|23|41|PM|third|evening|| +84222|AAAAAAAAPPIEBAAA|84222|23|23|42|PM|third|evening|| +84223|AAAAAAAAAAJEBAAA|84223|23|23|43|PM|third|evening|| +84224|AAAAAAAABAJEBAAA|84224|23|23|44|PM|third|evening|| +84225|AAAAAAAACAJEBAAA|84225|23|23|45|PM|third|evening|| +84226|AAAAAAAADAJEBAAA|84226|23|23|46|PM|third|evening|| +84227|AAAAAAAAEAJEBAAA|84227|23|23|47|PM|third|evening|| +84228|AAAAAAAAFAJEBAAA|84228|23|23|48|PM|third|evening|| +84229|AAAAAAAAGAJEBAAA|84229|23|23|49|PM|third|evening|| +84230|AAAAAAAAHAJEBAAA|84230|23|23|50|PM|third|evening|| +84231|AAAAAAAAIAJEBAAA|84231|23|23|51|PM|third|evening|| +84232|AAAAAAAAJAJEBAAA|84232|23|23|52|PM|third|evening|| +84233|AAAAAAAAKAJEBAAA|84233|23|23|53|PM|third|evening|| +84234|AAAAAAAALAJEBAAA|84234|23|23|54|PM|third|evening|| +84235|AAAAAAAAMAJEBAAA|84235|23|23|55|PM|third|evening|| +84236|AAAAAAAANAJEBAAA|84236|23|23|56|PM|third|evening|| +84237|AAAAAAAAOAJEBAAA|84237|23|23|57|PM|third|evening|| +84238|AAAAAAAAPAJEBAAA|84238|23|23|58|PM|third|evening|| +84239|AAAAAAAAABJEBAAA|84239|23|23|59|PM|third|evening|| +84240|AAAAAAAABBJEBAAA|84240|23|24|0|PM|third|evening|| +84241|AAAAAAAACBJEBAAA|84241|23|24|1|PM|third|evening|| +84242|AAAAAAAADBJEBAAA|84242|23|24|2|PM|third|evening|| +84243|AAAAAAAAEBJEBAAA|84243|23|24|3|PM|third|evening|| +84244|AAAAAAAAFBJEBAAA|84244|23|24|4|PM|third|evening|| +84245|AAAAAAAAGBJEBAAA|84245|23|24|5|PM|third|evening|| +84246|AAAAAAAAHBJEBAAA|84246|23|24|6|PM|third|evening|| +84247|AAAAAAAAIBJEBAAA|84247|23|24|7|PM|third|evening|| +84248|AAAAAAAAJBJEBAAA|84248|23|24|8|PM|third|evening|| +84249|AAAAAAAAKBJEBAAA|84249|23|24|9|PM|third|evening|| +84250|AAAAAAAALBJEBAAA|84250|23|24|10|PM|third|evening|| +84251|AAAAAAAAMBJEBAAA|84251|23|24|11|PM|third|evening|| +84252|AAAAAAAANBJEBAAA|84252|23|24|12|PM|third|evening|| +84253|AAAAAAAAOBJEBAAA|84253|23|24|13|PM|third|evening|| +84254|AAAAAAAAPBJEBAAA|84254|23|24|14|PM|third|evening|| +84255|AAAAAAAAACJEBAAA|84255|23|24|15|PM|third|evening|| +84256|AAAAAAAABCJEBAAA|84256|23|24|16|PM|third|evening|| +84257|AAAAAAAACCJEBAAA|84257|23|24|17|PM|third|evening|| +84258|AAAAAAAADCJEBAAA|84258|23|24|18|PM|third|evening|| +84259|AAAAAAAAECJEBAAA|84259|23|24|19|PM|third|evening|| +84260|AAAAAAAAFCJEBAAA|84260|23|24|20|PM|third|evening|| +84261|AAAAAAAAGCJEBAAA|84261|23|24|21|PM|third|evening|| +84262|AAAAAAAAHCJEBAAA|84262|23|24|22|PM|third|evening|| +84263|AAAAAAAAICJEBAAA|84263|23|24|23|PM|third|evening|| +84264|AAAAAAAAJCJEBAAA|84264|23|24|24|PM|third|evening|| +84265|AAAAAAAAKCJEBAAA|84265|23|24|25|PM|third|evening|| +84266|AAAAAAAALCJEBAAA|84266|23|24|26|PM|third|evening|| +84267|AAAAAAAAMCJEBAAA|84267|23|24|27|PM|third|evening|| +84268|AAAAAAAANCJEBAAA|84268|23|24|28|PM|third|evening|| +84269|AAAAAAAAOCJEBAAA|84269|23|24|29|PM|third|evening|| +84270|AAAAAAAAPCJEBAAA|84270|23|24|30|PM|third|evening|| +84271|AAAAAAAAADJEBAAA|84271|23|24|31|PM|third|evening|| +84272|AAAAAAAABDJEBAAA|84272|23|24|32|PM|third|evening|| +84273|AAAAAAAACDJEBAAA|84273|23|24|33|PM|third|evening|| +84274|AAAAAAAADDJEBAAA|84274|23|24|34|PM|third|evening|| +84275|AAAAAAAAEDJEBAAA|84275|23|24|35|PM|third|evening|| +84276|AAAAAAAAFDJEBAAA|84276|23|24|36|PM|third|evening|| +84277|AAAAAAAAGDJEBAAA|84277|23|24|37|PM|third|evening|| +84278|AAAAAAAAHDJEBAAA|84278|23|24|38|PM|third|evening|| +84279|AAAAAAAAIDJEBAAA|84279|23|24|39|PM|third|evening|| +84280|AAAAAAAAJDJEBAAA|84280|23|24|40|PM|third|evening|| +84281|AAAAAAAAKDJEBAAA|84281|23|24|41|PM|third|evening|| +84282|AAAAAAAALDJEBAAA|84282|23|24|42|PM|third|evening|| +84283|AAAAAAAAMDJEBAAA|84283|23|24|43|PM|third|evening|| +84284|AAAAAAAANDJEBAAA|84284|23|24|44|PM|third|evening|| +84285|AAAAAAAAODJEBAAA|84285|23|24|45|PM|third|evening|| +84286|AAAAAAAAPDJEBAAA|84286|23|24|46|PM|third|evening|| +84287|AAAAAAAAAEJEBAAA|84287|23|24|47|PM|third|evening|| +84288|AAAAAAAABEJEBAAA|84288|23|24|48|PM|third|evening|| +84289|AAAAAAAACEJEBAAA|84289|23|24|49|PM|third|evening|| +84290|AAAAAAAADEJEBAAA|84290|23|24|50|PM|third|evening|| +84291|AAAAAAAAEEJEBAAA|84291|23|24|51|PM|third|evening|| +84292|AAAAAAAAFEJEBAAA|84292|23|24|52|PM|third|evening|| +84293|AAAAAAAAGEJEBAAA|84293|23|24|53|PM|third|evening|| +84294|AAAAAAAAHEJEBAAA|84294|23|24|54|PM|third|evening|| +84295|AAAAAAAAIEJEBAAA|84295|23|24|55|PM|third|evening|| +84296|AAAAAAAAJEJEBAAA|84296|23|24|56|PM|third|evening|| +84297|AAAAAAAAKEJEBAAA|84297|23|24|57|PM|third|evening|| +84298|AAAAAAAALEJEBAAA|84298|23|24|58|PM|third|evening|| +84299|AAAAAAAAMEJEBAAA|84299|23|24|59|PM|third|evening|| +84300|AAAAAAAANEJEBAAA|84300|23|25|0|PM|third|evening|| +84301|AAAAAAAAOEJEBAAA|84301|23|25|1|PM|third|evening|| +84302|AAAAAAAAPEJEBAAA|84302|23|25|2|PM|third|evening|| +84303|AAAAAAAAAFJEBAAA|84303|23|25|3|PM|third|evening|| +84304|AAAAAAAABFJEBAAA|84304|23|25|4|PM|third|evening|| +84305|AAAAAAAACFJEBAAA|84305|23|25|5|PM|third|evening|| +84306|AAAAAAAADFJEBAAA|84306|23|25|6|PM|third|evening|| +84307|AAAAAAAAEFJEBAAA|84307|23|25|7|PM|third|evening|| +84308|AAAAAAAAFFJEBAAA|84308|23|25|8|PM|third|evening|| +84309|AAAAAAAAGFJEBAAA|84309|23|25|9|PM|third|evening|| +84310|AAAAAAAAHFJEBAAA|84310|23|25|10|PM|third|evening|| +84311|AAAAAAAAIFJEBAAA|84311|23|25|11|PM|third|evening|| +84312|AAAAAAAAJFJEBAAA|84312|23|25|12|PM|third|evening|| +84313|AAAAAAAAKFJEBAAA|84313|23|25|13|PM|third|evening|| +84314|AAAAAAAALFJEBAAA|84314|23|25|14|PM|third|evening|| +84315|AAAAAAAAMFJEBAAA|84315|23|25|15|PM|third|evening|| +84316|AAAAAAAANFJEBAAA|84316|23|25|16|PM|third|evening|| +84317|AAAAAAAAOFJEBAAA|84317|23|25|17|PM|third|evening|| +84318|AAAAAAAAPFJEBAAA|84318|23|25|18|PM|third|evening|| +84319|AAAAAAAAAGJEBAAA|84319|23|25|19|PM|third|evening|| +84320|AAAAAAAABGJEBAAA|84320|23|25|20|PM|third|evening|| +84321|AAAAAAAACGJEBAAA|84321|23|25|21|PM|third|evening|| +84322|AAAAAAAADGJEBAAA|84322|23|25|22|PM|third|evening|| +84323|AAAAAAAAEGJEBAAA|84323|23|25|23|PM|third|evening|| +84324|AAAAAAAAFGJEBAAA|84324|23|25|24|PM|third|evening|| +84325|AAAAAAAAGGJEBAAA|84325|23|25|25|PM|third|evening|| +84326|AAAAAAAAHGJEBAAA|84326|23|25|26|PM|third|evening|| +84327|AAAAAAAAIGJEBAAA|84327|23|25|27|PM|third|evening|| +84328|AAAAAAAAJGJEBAAA|84328|23|25|28|PM|third|evening|| +84329|AAAAAAAAKGJEBAAA|84329|23|25|29|PM|third|evening|| +84330|AAAAAAAALGJEBAAA|84330|23|25|30|PM|third|evening|| +84331|AAAAAAAAMGJEBAAA|84331|23|25|31|PM|third|evening|| +84332|AAAAAAAANGJEBAAA|84332|23|25|32|PM|third|evening|| +84333|AAAAAAAAOGJEBAAA|84333|23|25|33|PM|third|evening|| +84334|AAAAAAAAPGJEBAAA|84334|23|25|34|PM|third|evening|| +84335|AAAAAAAAAHJEBAAA|84335|23|25|35|PM|third|evening|| +84336|AAAAAAAABHJEBAAA|84336|23|25|36|PM|third|evening|| +84337|AAAAAAAACHJEBAAA|84337|23|25|37|PM|third|evening|| +84338|AAAAAAAADHJEBAAA|84338|23|25|38|PM|third|evening|| +84339|AAAAAAAAEHJEBAAA|84339|23|25|39|PM|third|evening|| +84340|AAAAAAAAFHJEBAAA|84340|23|25|40|PM|third|evening|| +84341|AAAAAAAAGHJEBAAA|84341|23|25|41|PM|third|evening|| +84342|AAAAAAAAHHJEBAAA|84342|23|25|42|PM|third|evening|| +84343|AAAAAAAAIHJEBAAA|84343|23|25|43|PM|third|evening|| +84344|AAAAAAAAJHJEBAAA|84344|23|25|44|PM|third|evening|| +84345|AAAAAAAAKHJEBAAA|84345|23|25|45|PM|third|evening|| +84346|AAAAAAAALHJEBAAA|84346|23|25|46|PM|third|evening|| +84347|AAAAAAAAMHJEBAAA|84347|23|25|47|PM|third|evening|| +84348|AAAAAAAANHJEBAAA|84348|23|25|48|PM|third|evening|| +84349|AAAAAAAAOHJEBAAA|84349|23|25|49|PM|third|evening|| +84350|AAAAAAAAPHJEBAAA|84350|23|25|50|PM|third|evening|| +84351|AAAAAAAAAIJEBAAA|84351|23|25|51|PM|third|evening|| +84352|AAAAAAAABIJEBAAA|84352|23|25|52|PM|third|evening|| +84353|AAAAAAAACIJEBAAA|84353|23|25|53|PM|third|evening|| +84354|AAAAAAAADIJEBAAA|84354|23|25|54|PM|third|evening|| +84355|AAAAAAAAEIJEBAAA|84355|23|25|55|PM|third|evening|| +84356|AAAAAAAAFIJEBAAA|84356|23|25|56|PM|third|evening|| +84357|AAAAAAAAGIJEBAAA|84357|23|25|57|PM|third|evening|| +84358|AAAAAAAAHIJEBAAA|84358|23|25|58|PM|third|evening|| +84359|AAAAAAAAIIJEBAAA|84359|23|25|59|PM|third|evening|| +84360|AAAAAAAAJIJEBAAA|84360|23|26|0|PM|third|evening|| +84361|AAAAAAAAKIJEBAAA|84361|23|26|1|PM|third|evening|| +84362|AAAAAAAALIJEBAAA|84362|23|26|2|PM|third|evening|| +84363|AAAAAAAAMIJEBAAA|84363|23|26|3|PM|third|evening|| +84364|AAAAAAAANIJEBAAA|84364|23|26|4|PM|third|evening|| +84365|AAAAAAAAOIJEBAAA|84365|23|26|5|PM|third|evening|| +84366|AAAAAAAAPIJEBAAA|84366|23|26|6|PM|third|evening|| +84367|AAAAAAAAAJJEBAAA|84367|23|26|7|PM|third|evening|| +84368|AAAAAAAABJJEBAAA|84368|23|26|8|PM|third|evening|| +84369|AAAAAAAACJJEBAAA|84369|23|26|9|PM|third|evening|| +84370|AAAAAAAADJJEBAAA|84370|23|26|10|PM|third|evening|| +84371|AAAAAAAAEJJEBAAA|84371|23|26|11|PM|third|evening|| +84372|AAAAAAAAFJJEBAAA|84372|23|26|12|PM|third|evening|| +84373|AAAAAAAAGJJEBAAA|84373|23|26|13|PM|third|evening|| +84374|AAAAAAAAHJJEBAAA|84374|23|26|14|PM|third|evening|| +84375|AAAAAAAAIJJEBAAA|84375|23|26|15|PM|third|evening|| +84376|AAAAAAAAJJJEBAAA|84376|23|26|16|PM|third|evening|| +84377|AAAAAAAAKJJEBAAA|84377|23|26|17|PM|third|evening|| +84378|AAAAAAAALJJEBAAA|84378|23|26|18|PM|third|evening|| +84379|AAAAAAAAMJJEBAAA|84379|23|26|19|PM|third|evening|| +84380|AAAAAAAANJJEBAAA|84380|23|26|20|PM|third|evening|| +84381|AAAAAAAAOJJEBAAA|84381|23|26|21|PM|third|evening|| +84382|AAAAAAAAPJJEBAAA|84382|23|26|22|PM|third|evening|| +84383|AAAAAAAAAKJEBAAA|84383|23|26|23|PM|third|evening|| +84384|AAAAAAAABKJEBAAA|84384|23|26|24|PM|third|evening|| +84385|AAAAAAAACKJEBAAA|84385|23|26|25|PM|third|evening|| +84386|AAAAAAAADKJEBAAA|84386|23|26|26|PM|third|evening|| +84387|AAAAAAAAEKJEBAAA|84387|23|26|27|PM|third|evening|| +84388|AAAAAAAAFKJEBAAA|84388|23|26|28|PM|third|evening|| +84389|AAAAAAAAGKJEBAAA|84389|23|26|29|PM|third|evening|| +84390|AAAAAAAAHKJEBAAA|84390|23|26|30|PM|third|evening|| +84391|AAAAAAAAIKJEBAAA|84391|23|26|31|PM|third|evening|| +84392|AAAAAAAAJKJEBAAA|84392|23|26|32|PM|third|evening|| +84393|AAAAAAAAKKJEBAAA|84393|23|26|33|PM|third|evening|| +84394|AAAAAAAALKJEBAAA|84394|23|26|34|PM|third|evening|| +84395|AAAAAAAAMKJEBAAA|84395|23|26|35|PM|third|evening|| +84396|AAAAAAAANKJEBAAA|84396|23|26|36|PM|third|evening|| +84397|AAAAAAAAOKJEBAAA|84397|23|26|37|PM|third|evening|| +84398|AAAAAAAAPKJEBAAA|84398|23|26|38|PM|third|evening|| +84399|AAAAAAAAALJEBAAA|84399|23|26|39|PM|third|evening|| +84400|AAAAAAAABLJEBAAA|84400|23|26|40|PM|third|evening|| +84401|AAAAAAAACLJEBAAA|84401|23|26|41|PM|third|evening|| +84402|AAAAAAAADLJEBAAA|84402|23|26|42|PM|third|evening|| +84403|AAAAAAAAELJEBAAA|84403|23|26|43|PM|third|evening|| +84404|AAAAAAAAFLJEBAAA|84404|23|26|44|PM|third|evening|| +84405|AAAAAAAAGLJEBAAA|84405|23|26|45|PM|third|evening|| +84406|AAAAAAAAHLJEBAAA|84406|23|26|46|PM|third|evening|| +84407|AAAAAAAAILJEBAAA|84407|23|26|47|PM|third|evening|| +84408|AAAAAAAAJLJEBAAA|84408|23|26|48|PM|third|evening|| +84409|AAAAAAAAKLJEBAAA|84409|23|26|49|PM|third|evening|| +84410|AAAAAAAALLJEBAAA|84410|23|26|50|PM|third|evening|| +84411|AAAAAAAAMLJEBAAA|84411|23|26|51|PM|third|evening|| +84412|AAAAAAAANLJEBAAA|84412|23|26|52|PM|third|evening|| +84413|AAAAAAAAOLJEBAAA|84413|23|26|53|PM|third|evening|| +84414|AAAAAAAAPLJEBAAA|84414|23|26|54|PM|third|evening|| +84415|AAAAAAAAAMJEBAAA|84415|23|26|55|PM|third|evening|| +84416|AAAAAAAABMJEBAAA|84416|23|26|56|PM|third|evening|| +84417|AAAAAAAACMJEBAAA|84417|23|26|57|PM|third|evening|| +84418|AAAAAAAADMJEBAAA|84418|23|26|58|PM|third|evening|| +84419|AAAAAAAAEMJEBAAA|84419|23|26|59|PM|third|evening|| +84420|AAAAAAAAFMJEBAAA|84420|23|27|0|PM|third|evening|| +84421|AAAAAAAAGMJEBAAA|84421|23|27|1|PM|third|evening|| +84422|AAAAAAAAHMJEBAAA|84422|23|27|2|PM|third|evening|| +84423|AAAAAAAAIMJEBAAA|84423|23|27|3|PM|third|evening|| +84424|AAAAAAAAJMJEBAAA|84424|23|27|4|PM|third|evening|| +84425|AAAAAAAAKMJEBAAA|84425|23|27|5|PM|third|evening|| +84426|AAAAAAAALMJEBAAA|84426|23|27|6|PM|third|evening|| +84427|AAAAAAAAMMJEBAAA|84427|23|27|7|PM|third|evening|| +84428|AAAAAAAANMJEBAAA|84428|23|27|8|PM|third|evening|| +84429|AAAAAAAAOMJEBAAA|84429|23|27|9|PM|third|evening|| +84430|AAAAAAAAPMJEBAAA|84430|23|27|10|PM|third|evening|| +84431|AAAAAAAAANJEBAAA|84431|23|27|11|PM|third|evening|| +84432|AAAAAAAABNJEBAAA|84432|23|27|12|PM|third|evening|| +84433|AAAAAAAACNJEBAAA|84433|23|27|13|PM|third|evening|| +84434|AAAAAAAADNJEBAAA|84434|23|27|14|PM|third|evening|| +84435|AAAAAAAAENJEBAAA|84435|23|27|15|PM|third|evening|| +84436|AAAAAAAAFNJEBAAA|84436|23|27|16|PM|third|evening|| +84437|AAAAAAAAGNJEBAAA|84437|23|27|17|PM|third|evening|| +84438|AAAAAAAAHNJEBAAA|84438|23|27|18|PM|third|evening|| +84439|AAAAAAAAINJEBAAA|84439|23|27|19|PM|third|evening|| +84440|AAAAAAAAJNJEBAAA|84440|23|27|20|PM|third|evening|| +84441|AAAAAAAAKNJEBAAA|84441|23|27|21|PM|third|evening|| +84442|AAAAAAAALNJEBAAA|84442|23|27|22|PM|third|evening|| +84443|AAAAAAAAMNJEBAAA|84443|23|27|23|PM|third|evening|| +84444|AAAAAAAANNJEBAAA|84444|23|27|24|PM|third|evening|| +84445|AAAAAAAAONJEBAAA|84445|23|27|25|PM|third|evening|| +84446|AAAAAAAAPNJEBAAA|84446|23|27|26|PM|third|evening|| +84447|AAAAAAAAAOJEBAAA|84447|23|27|27|PM|third|evening|| +84448|AAAAAAAABOJEBAAA|84448|23|27|28|PM|third|evening|| +84449|AAAAAAAACOJEBAAA|84449|23|27|29|PM|third|evening|| +84450|AAAAAAAADOJEBAAA|84450|23|27|30|PM|third|evening|| +84451|AAAAAAAAEOJEBAAA|84451|23|27|31|PM|third|evening|| +84452|AAAAAAAAFOJEBAAA|84452|23|27|32|PM|third|evening|| +84453|AAAAAAAAGOJEBAAA|84453|23|27|33|PM|third|evening|| +84454|AAAAAAAAHOJEBAAA|84454|23|27|34|PM|third|evening|| +84455|AAAAAAAAIOJEBAAA|84455|23|27|35|PM|third|evening|| +84456|AAAAAAAAJOJEBAAA|84456|23|27|36|PM|third|evening|| +84457|AAAAAAAAKOJEBAAA|84457|23|27|37|PM|third|evening|| +84458|AAAAAAAALOJEBAAA|84458|23|27|38|PM|third|evening|| +84459|AAAAAAAAMOJEBAAA|84459|23|27|39|PM|third|evening|| +84460|AAAAAAAANOJEBAAA|84460|23|27|40|PM|third|evening|| +84461|AAAAAAAAOOJEBAAA|84461|23|27|41|PM|third|evening|| +84462|AAAAAAAAPOJEBAAA|84462|23|27|42|PM|third|evening|| +84463|AAAAAAAAAPJEBAAA|84463|23|27|43|PM|third|evening|| +84464|AAAAAAAABPJEBAAA|84464|23|27|44|PM|third|evening|| +84465|AAAAAAAACPJEBAAA|84465|23|27|45|PM|third|evening|| +84466|AAAAAAAADPJEBAAA|84466|23|27|46|PM|third|evening|| +84467|AAAAAAAAEPJEBAAA|84467|23|27|47|PM|third|evening|| +84468|AAAAAAAAFPJEBAAA|84468|23|27|48|PM|third|evening|| +84469|AAAAAAAAGPJEBAAA|84469|23|27|49|PM|third|evening|| +84470|AAAAAAAAHPJEBAAA|84470|23|27|50|PM|third|evening|| +84471|AAAAAAAAIPJEBAAA|84471|23|27|51|PM|third|evening|| +84472|AAAAAAAAJPJEBAAA|84472|23|27|52|PM|third|evening|| +84473|AAAAAAAAKPJEBAAA|84473|23|27|53|PM|third|evening|| +84474|AAAAAAAALPJEBAAA|84474|23|27|54|PM|third|evening|| +84475|AAAAAAAAMPJEBAAA|84475|23|27|55|PM|third|evening|| +84476|AAAAAAAANPJEBAAA|84476|23|27|56|PM|third|evening|| +84477|AAAAAAAAOPJEBAAA|84477|23|27|57|PM|third|evening|| +84478|AAAAAAAAPPJEBAAA|84478|23|27|58|PM|third|evening|| +84479|AAAAAAAAAAKEBAAA|84479|23|27|59|PM|third|evening|| +84480|AAAAAAAABAKEBAAA|84480|23|28|0|PM|third|evening|| +84481|AAAAAAAACAKEBAAA|84481|23|28|1|PM|third|evening|| +84482|AAAAAAAADAKEBAAA|84482|23|28|2|PM|third|evening|| +84483|AAAAAAAAEAKEBAAA|84483|23|28|3|PM|third|evening|| +84484|AAAAAAAAFAKEBAAA|84484|23|28|4|PM|third|evening|| +84485|AAAAAAAAGAKEBAAA|84485|23|28|5|PM|third|evening|| +84486|AAAAAAAAHAKEBAAA|84486|23|28|6|PM|third|evening|| +84487|AAAAAAAAIAKEBAAA|84487|23|28|7|PM|third|evening|| +84488|AAAAAAAAJAKEBAAA|84488|23|28|8|PM|third|evening|| +84489|AAAAAAAAKAKEBAAA|84489|23|28|9|PM|third|evening|| +84490|AAAAAAAALAKEBAAA|84490|23|28|10|PM|third|evening|| +84491|AAAAAAAAMAKEBAAA|84491|23|28|11|PM|third|evening|| +84492|AAAAAAAANAKEBAAA|84492|23|28|12|PM|third|evening|| +84493|AAAAAAAAOAKEBAAA|84493|23|28|13|PM|third|evening|| +84494|AAAAAAAAPAKEBAAA|84494|23|28|14|PM|third|evening|| +84495|AAAAAAAAABKEBAAA|84495|23|28|15|PM|third|evening|| +84496|AAAAAAAABBKEBAAA|84496|23|28|16|PM|third|evening|| +84497|AAAAAAAACBKEBAAA|84497|23|28|17|PM|third|evening|| +84498|AAAAAAAADBKEBAAA|84498|23|28|18|PM|third|evening|| +84499|AAAAAAAAEBKEBAAA|84499|23|28|19|PM|third|evening|| +84500|AAAAAAAAFBKEBAAA|84500|23|28|20|PM|third|evening|| +84501|AAAAAAAAGBKEBAAA|84501|23|28|21|PM|third|evening|| +84502|AAAAAAAAHBKEBAAA|84502|23|28|22|PM|third|evening|| +84503|AAAAAAAAIBKEBAAA|84503|23|28|23|PM|third|evening|| +84504|AAAAAAAAJBKEBAAA|84504|23|28|24|PM|third|evening|| +84505|AAAAAAAAKBKEBAAA|84505|23|28|25|PM|third|evening|| +84506|AAAAAAAALBKEBAAA|84506|23|28|26|PM|third|evening|| +84507|AAAAAAAAMBKEBAAA|84507|23|28|27|PM|third|evening|| +84508|AAAAAAAANBKEBAAA|84508|23|28|28|PM|third|evening|| +84509|AAAAAAAAOBKEBAAA|84509|23|28|29|PM|third|evening|| +84510|AAAAAAAAPBKEBAAA|84510|23|28|30|PM|third|evening|| +84511|AAAAAAAAACKEBAAA|84511|23|28|31|PM|third|evening|| +84512|AAAAAAAABCKEBAAA|84512|23|28|32|PM|third|evening|| +84513|AAAAAAAACCKEBAAA|84513|23|28|33|PM|third|evening|| +84514|AAAAAAAADCKEBAAA|84514|23|28|34|PM|third|evening|| +84515|AAAAAAAAECKEBAAA|84515|23|28|35|PM|third|evening|| +84516|AAAAAAAAFCKEBAAA|84516|23|28|36|PM|third|evening|| +84517|AAAAAAAAGCKEBAAA|84517|23|28|37|PM|third|evening|| +84518|AAAAAAAAHCKEBAAA|84518|23|28|38|PM|third|evening|| +84519|AAAAAAAAICKEBAAA|84519|23|28|39|PM|third|evening|| +84520|AAAAAAAAJCKEBAAA|84520|23|28|40|PM|third|evening|| +84521|AAAAAAAAKCKEBAAA|84521|23|28|41|PM|third|evening|| +84522|AAAAAAAALCKEBAAA|84522|23|28|42|PM|third|evening|| +84523|AAAAAAAAMCKEBAAA|84523|23|28|43|PM|third|evening|| +84524|AAAAAAAANCKEBAAA|84524|23|28|44|PM|third|evening|| +84525|AAAAAAAAOCKEBAAA|84525|23|28|45|PM|third|evening|| +84526|AAAAAAAAPCKEBAAA|84526|23|28|46|PM|third|evening|| +84527|AAAAAAAAADKEBAAA|84527|23|28|47|PM|third|evening|| +84528|AAAAAAAABDKEBAAA|84528|23|28|48|PM|third|evening|| +84529|AAAAAAAACDKEBAAA|84529|23|28|49|PM|third|evening|| +84530|AAAAAAAADDKEBAAA|84530|23|28|50|PM|third|evening|| +84531|AAAAAAAAEDKEBAAA|84531|23|28|51|PM|third|evening|| +84532|AAAAAAAAFDKEBAAA|84532|23|28|52|PM|third|evening|| +84533|AAAAAAAAGDKEBAAA|84533|23|28|53|PM|third|evening|| +84534|AAAAAAAAHDKEBAAA|84534|23|28|54|PM|third|evening|| +84535|AAAAAAAAIDKEBAAA|84535|23|28|55|PM|third|evening|| +84536|AAAAAAAAJDKEBAAA|84536|23|28|56|PM|third|evening|| +84537|AAAAAAAAKDKEBAAA|84537|23|28|57|PM|third|evening|| +84538|AAAAAAAALDKEBAAA|84538|23|28|58|PM|third|evening|| +84539|AAAAAAAAMDKEBAAA|84539|23|28|59|PM|third|evening|| +84540|AAAAAAAANDKEBAAA|84540|23|29|0|PM|third|evening|| +84541|AAAAAAAAODKEBAAA|84541|23|29|1|PM|third|evening|| +84542|AAAAAAAAPDKEBAAA|84542|23|29|2|PM|third|evening|| +84543|AAAAAAAAAEKEBAAA|84543|23|29|3|PM|third|evening|| +84544|AAAAAAAABEKEBAAA|84544|23|29|4|PM|third|evening|| +84545|AAAAAAAACEKEBAAA|84545|23|29|5|PM|third|evening|| +84546|AAAAAAAADEKEBAAA|84546|23|29|6|PM|third|evening|| +84547|AAAAAAAAEEKEBAAA|84547|23|29|7|PM|third|evening|| +84548|AAAAAAAAFEKEBAAA|84548|23|29|8|PM|third|evening|| +84549|AAAAAAAAGEKEBAAA|84549|23|29|9|PM|third|evening|| +84550|AAAAAAAAHEKEBAAA|84550|23|29|10|PM|third|evening|| +84551|AAAAAAAAIEKEBAAA|84551|23|29|11|PM|third|evening|| +84552|AAAAAAAAJEKEBAAA|84552|23|29|12|PM|third|evening|| +84553|AAAAAAAAKEKEBAAA|84553|23|29|13|PM|third|evening|| +84554|AAAAAAAALEKEBAAA|84554|23|29|14|PM|third|evening|| +84555|AAAAAAAAMEKEBAAA|84555|23|29|15|PM|third|evening|| +84556|AAAAAAAANEKEBAAA|84556|23|29|16|PM|third|evening|| +84557|AAAAAAAAOEKEBAAA|84557|23|29|17|PM|third|evening|| +84558|AAAAAAAAPEKEBAAA|84558|23|29|18|PM|third|evening|| +84559|AAAAAAAAAFKEBAAA|84559|23|29|19|PM|third|evening|| +84560|AAAAAAAABFKEBAAA|84560|23|29|20|PM|third|evening|| +84561|AAAAAAAACFKEBAAA|84561|23|29|21|PM|third|evening|| +84562|AAAAAAAADFKEBAAA|84562|23|29|22|PM|third|evening|| +84563|AAAAAAAAEFKEBAAA|84563|23|29|23|PM|third|evening|| +84564|AAAAAAAAFFKEBAAA|84564|23|29|24|PM|third|evening|| +84565|AAAAAAAAGFKEBAAA|84565|23|29|25|PM|third|evening|| +84566|AAAAAAAAHFKEBAAA|84566|23|29|26|PM|third|evening|| +84567|AAAAAAAAIFKEBAAA|84567|23|29|27|PM|third|evening|| +84568|AAAAAAAAJFKEBAAA|84568|23|29|28|PM|third|evening|| +84569|AAAAAAAAKFKEBAAA|84569|23|29|29|PM|third|evening|| +84570|AAAAAAAALFKEBAAA|84570|23|29|30|PM|third|evening|| +84571|AAAAAAAAMFKEBAAA|84571|23|29|31|PM|third|evening|| +84572|AAAAAAAANFKEBAAA|84572|23|29|32|PM|third|evening|| +84573|AAAAAAAAOFKEBAAA|84573|23|29|33|PM|third|evening|| +84574|AAAAAAAAPFKEBAAA|84574|23|29|34|PM|third|evening|| +84575|AAAAAAAAAGKEBAAA|84575|23|29|35|PM|third|evening|| +84576|AAAAAAAABGKEBAAA|84576|23|29|36|PM|third|evening|| +84577|AAAAAAAACGKEBAAA|84577|23|29|37|PM|third|evening|| +84578|AAAAAAAADGKEBAAA|84578|23|29|38|PM|third|evening|| +84579|AAAAAAAAEGKEBAAA|84579|23|29|39|PM|third|evening|| +84580|AAAAAAAAFGKEBAAA|84580|23|29|40|PM|third|evening|| +84581|AAAAAAAAGGKEBAAA|84581|23|29|41|PM|third|evening|| +84582|AAAAAAAAHGKEBAAA|84582|23|29|42|PM|third|evening|| +84583|AAAAAAAAIGKEBAAA|84583|23|29|43|PM|third|evening|| +84584|AAAAAAAAJGKEBAAA|84584|23|29|44|PM|third|evening|| +84585|AAAAAAAAKGKEBAAA|84585|23|29|45|PM|third|evening|| +84586|AAAAAAAALGKEBAAA|84586|23|29|46|PM|third|evening|| +84587|AAAAAAAAMGKEBAAA|84587|23|29|47|PM|third|evening|| +84588|AAAAAAAANGKEBAAA|84588|23|29|48|PM|third|evening|| +84589|AAAAAAAAOGKEBAAA|84589|23|29|49|PM|third|evening|| +84590|AAAAAAAAPGKEBAAA|84590|23|29|50|PM|third|evening|| +84591|AAAAAAAAAHKEBAAA|84591|23|29|51|PM|third|evening|| +84592|AAAAAAAABHKEBAAA|84592|23|29|52|PM|third|evening|| +84593|AAAAAAAACHKEBAAA|84593|23|29|53|PM|third|evening|| +84594|AAAAAAAADHKEBAAA|84594|23|29|54|PM|third|evening|| +84595|AAAAAAAAEHKEBAAA|84595|23|29|55|PM|third|evening|| +84596|AAAAAAAAFHKEBAAA|84596|23|29|56|PM|third|evening|| +84597|AAAAAAAAGHKEBAAA|84597|23|29|57|PM|third|evening|| +84598|AAAAAAAAHHKEBAAA|84598|23|29|58|PM|third|evening|| +84599|AAAAAAAAIHKEBAAA|84599|23|29|59|PM|third|evening|| +84600|AAAAAAAAJHKEBAAA|84600|23|30|0|PM|third|evening|| +84601|AAAAAAAAKHKEBAAA|84601|23|30|1|PM|third|evening|| +84602|AAAAAAAALHKEBAAA|84602|23|30|2|PM|third|evening|| +84603|AAAAAAAAMHKEBAAA|84603|23|30|3|PM|third|evening|| +84604|AAAAAAAANHKEBAAA|84604|23|30|4|PM|third|evening|| +84605|AAAAAAAAOHKEBAAA|84605|23|30|5|PM|third|evening|| +84606|AAAAAAAAPHKEBAAA|84606|23|30|6|PM|third|evening|| +84607|AAAAAAAAAIKEBAAA|84607|23|30|7|PM|third|evening|| +84608|AAAAAAAABIKEBAAA|84608|23|30|8|PM|third|evening|| +84609|AAAAAAAACIKEBAAA|84609|23|30|9|PM|third|evening|| +84610|AAAAAAAADIKEBAAA|84610|23|30|10|PM|third|evening|| +84611|AAAAAAAAEIKEBAAA|84611|23|30|11|PM|third|evening|| +84612|AAAAAAAAFIKEBAAA|84612|23|30|12|PM|third|evening|| +84613|AAAAAAAAGIKEBAAA|84613|23|30|13|PM|third|evening|| +84614|AAAAAAAAHIKEBAAA|84614|23|30|14|PM|third|evening|| +84615|AAAAAAAAIIKEBAAA|84615|23|30|15|PM|third|evening|| +84616|AAAAAAAAJIKEBAAA|84616|23|30|16|PM|third|evening|| +84617|AAAAAAAAKIKEBAAA|84617|23|30|17|PM|third|evening|| +84618|AAAAAAAALIKEBAAA|84618|23|30|18|PM|third|evening|| +84619|AAAAAAAAMIKEBAAA|84619|23|30|19|PM|third|evening|| +84620|AAAAAAAANIKEBAAA|84620|23|30|20|PM|third|evening|| +84621|AAAAAAAAOIKEBAAA|84621|23|30|21|PM|third|evening|| +84622|AAAAAAAAPIKEBAAA|84622|23|30|22|PM|third|evening|| +84623|AAAAAAAAAJKEBAAA|84623|23|30|23|PM|third|evening|| +84624|AAAAAAAABJKEBAAA|84624|23|30|24|PM|third|evening|| +84625|AAAAAAAACJKEBAAA|84625|23|30|25|PM|third|evening|| +84626|AAAAAAAADJKEBAAA|84626|23|30|26|PM|third|evening|| +84627|AAAAAAAAEJKEBAAA|84627|23|30|27|PM|third|evening|| +84628|AAAAAAAAFJKEBAAA|84628|23|30|28|PM|third|evening|| +84629|AAAAAAAAGJKEBAAA|84629|23|30|29|PM|third|evening|| +84630|AAAAAAAAHJKEBAAA|84630|23|30|30|PM|third|evening|| +84631|AAAAAAAAIJKEBAAA|84631|23|30|31|PM|third|evening|| +84632|AAAAAAAAJJKEBAAA|84632|23|30|32|PM|third|evening|| +84633|AAAAAAAAKJKEBAAA|84633|23|30|33|PM|third|evening|| +84634|AAAAAAAALJKEBAAA|84634|23|30|34|PM|third|evening|| +84635|AAAAAAAAMJKEBAAA|84635|23|30|35|PM|third|evening|| +84636|AAAAAAAANJKEBAAA|84636|23|30|36|PM|third|evening|| +84637|AAAAAAAAOJKEBAAA|84637|23|30|37|PM|third|evening|| +84638|AAAAAAAAPJKEBAAA|84638|23|30|38|PM|third|evening|| +84639|AAAAAAAAAKKEBAAA|84639|23|30|39|PM|third|evening|| +84640|AAAAAAAABKKEBAAA|84640|23|30|40|PM|third|evening|| +84641|AAAAAAAACKKEBAAA|84641|23|30|41|PM|third|evening|| +84642|AAAAAAAADKKEBAAA|84642|23|30|42|PM|third|evening|| +84643|AAAAAAAAEKKEBAAA|84643|23|30|43|PM|third|evening|| +84644|AAAAAAAAFKKEBAAA|84644|23|30|44|PM|third|evening|| +84645|AAAAAAAAGKKEBAAA|84645|23|30|45|PM|third|evening|| +84646|AAAAAAAAHKKEBAAA|84646|23|30|46|PM|third|evening|| +84647|AAAAAAAAIKKEBAAA|84647|23|30|47|PM|third|evening|| +84648|AAAAAAAAJKKEBAAA|84648|23|30|48|PM|third|evening|| +84649|AAAAAAAAKKKEBAAA|84649|23|30|49|PM|third|evening|| +84650|AAAAAAAALKKEBAAA|84650|23|30|50|PM|third|evening|| +84651|AAAAAAAAMKKEBAAA|84651|23|30|51|PM|third|evening|| +84652|AAAAAAAANKKEBAAA|84652|23|30|52|PM|third|evening|| +84653|AAAAAAAAOKKEBAAA|84653|23|30|53|PM|third|evening|| +84654|AAAAAAAAPKKEBAAA|84654|23|30|54|PM|third|evening|| +84655|AAAAAAAAALKEBAAA|84655|23|30|55|PM|third|evening|| +84656|AAAAAAAABLKEBAAA|84656|23|30|56|PM|third|evening|| +84657|AAAAAAAACLKEBAAA|84657|23|30|57|PM|third|evening|| +84658|AAAAAAAADLKEBAAA|84658|23|30|58|PM|third|evening|| +84659|AAAAAAAAELKEBAAA|84659|23|30|59|PM|third|evening|| +84660|AAAAAAAAFLKEBAAA|84660|23|31|0|PM|third|evening|| +84661|AAAAAAAAGLKEBAAA|84661|23|31|1|PM|third|evening|| +84662|AAAAAAAAHLKEBAAA|84662|23|31|2|PM|third|evening|| +84663|AAAAAAAAILKEBAAA|84663|23|31|3|PM|third|evening|| +84664|AAAAAAAAJLKEBAAA|84664|23|31|4|PM|third|evening|| +84665|AAAAAAAAKLKEBAAA|84665|23|31|5|PM|third|evening|| +84666|AAAAAAAALLKEBAAA|84666|23|31|6|PM|third|evening|| +84667|AAAAAAAAMLKEBAAA|84667|23|31|7|PM|third|evening|| +84668|AAAAAAAANLKEBAAA|84668|23|31|8|PM|third|evening|| +84669|AAAAAAAAOLKEBAAA|84669|23|31|9|PM|third|evening|| +84670|AAAAAAAAPLKEBAAA|84670|23|31|10|PM|third|evening|| +84671|AAAAAAAAAMKEBAAA|84671|23|31|11|PM|third|evening|| +84672|AAAAAAAABMKEBAAA|84672|23|31|12|PM|third|evening|| +84673|AAAAAAAACMKEBAAA|84673|23|31|13|PM|third|evening|| +84674|AAAAAAAADMKEBAAA|84674|23|31|14|PM|third|evening|| +84675|AAAAAAAAEMKEBAAA|84675|23|31|15|PM|third|evening|| +84676|AAAAAAAAFMKEBAAA|84676|23|31|16|PM|third|evening|| +84677|AAAAAAAAGMKEBAAA|84677|23|31|17|PM|third|evening|| +84678|AAAAAAAAHMKEBAAA|84678|23|31|18|PM|third|evening|| +84679|AAAAAAAAIMKEBAAA|84679|23|31|19|PM|third|evening|| +84680|AAAAAAAAJMKEBAAA|84680|23|31|20|PM|third|evening|| +84681|AAAAAAAAKMKEBAAA|84681|23|31|21|PM|third|evening|| +84682|AAAAAAAALMKEBAAA|84682|23|31|22|PM|third|evening|| +84683|AAAAAAAAMMKEBAAA|84683|23|31|23|PM|third|evening|| +84684|AAAAAAAANMKEBAAA|84684|23|31|24|PM|third|evening|| +84685|AAAAAAAAOMKEBAAA|84685|23|31|25|PM|third|evening|| +84686|AAAAAAAAPMKEBAAA|84686|23|31|26|PM|third|evening|| +84687|AAAAAAAAANKEBAAA|84687|23|31|27|PM|third|evening|| +84688|AAAAAAAABNKEBAAA|84688|23|31|28|PM|third|evening|| +84689|AAAAAAAACNKEBAAA|84689|23|31|29|PM|third|evening|| +84690|AAAAAAAADNKEBAAA|84690|23|31|30|PM|third|evening|| +84691|AAAAAAAAENKEBAAA|84691|23|31|31|PM|third|evening|| +84692|AAAAAAAAFNKEBAAA|84692|23|31|32|PM|third|evening|| +84693|AAAAAAAAGNKEBAAA|84693|23|31|33|PM|third|evening|| +84694|AAAAAAAAHNKEBAAA|84694|23|31|34|PM|third|evening|| +84695|AAAAAAAAINKEBAAA|84695|23|31|35|PM|third|evening|| +84696|AAAAAAAAJNKEBAAA|84696|23|31|36|PM|third|evening|| +84697|AAAAAAAAKNKEBAAA|84697|23|31|37|PM|third|evening|| +84698|AAAAAAAALNKEBAAA|84698|23|31|38|PM|third|evening|| +84699|AAAAAAAAMNKEBAAA|84699|23|31|39|PM|third|evening|| +84700|AAAAAAAANNKEBAAA|84700|23|31|40|PM|third|evening|| +84701|AAAAAAAAONKEBAAA|84701|23|31|41|PM|third|evening|| +84702|AAAAAAAAPNKEBAAA|84702|23|31|42|PM|third|evening|| +84703|AAAAAAAAAOKEBAAA|84703|23|31|43|PM|third|evening|| +84704|AAAAAAAABOKEBAAA|84704|23|31|44|PM|third|evening|| +84705|AAAAAAAACOKEBAAA|84705|23|31|45|PM|third|evening|| +84706|AAAAAAAADOKEBAAA|84706|23|31|46|PM|third|evening|| +84707|AAAAAAAAEOKEBAAA|84707|23|31|47|PM|third|evening|| +84708|AAAAAAAAFOKEBAAA|84708|23|31|48|PM|third|evening|| +84709|AAAAAAAAGOKEBAAA|84709|23|31|49|PM|third|evening|| +84710|AAAAAAAAHOKEBAAA|84710|23|31|50|PM|third|evening|| +84711|AAAAAAAAIOKEBAAA|84711|23|31|51|PM|third|evening|| +84712|AAAAAAAAJOKEBAAA|84712|23|31|52|PM|third|evening|| +84713|AAAAAAAAKOKEBAAA|84713|23|31|53|PM|third|evening|| +84714|AAAAAAAALOKEBAAA|84714|23|31|54|PM|third|evening|| +84715|AAAAAAAAMOKEBAAA|84715|23|31|55|PM|third|evening|| +84716|AAAAAAAANOKEBAAA|84716|23|31|56|PM|third|evening|| +84717|AAAAAAAAOOKEBAAA|84717|23|31|57|PM|third|evening|| +84718|AAAAAAAAPOKEBAAA|84718|23|31|58|PM|third|evening|| +84719|AAAAAAAAAPKEBAAA|84719|23|31|59|PM|third|evening|| +84720|AAAAAAAABPKEBAAA|84720|23|32|0|PM|third|evening|| +84721|AAAAAAAACPKEBAAA|84721|23|32|1|PM|third|evening|| +84722|AAAAAAAADPKEBAAA|84722|23|32|2|PM|third|evening|| +84723|AAAAAAAAEPKEBAAA|84723|23|32|3|PM|third|evening|| +84724|AAAAAAAAFPKEBAAA|84724|23|32|4|PM|third|evening|| +84725|AAAAAAAAGPKEBAAA|84725|23|32|5|PM|third|evening|| +84726|AAAAAAAAHPKEBAAA|84726|23|32|6|PM|third|evening|| +84727|AAAAAAAAIPKEBAAA|84727|23|32|7|PM|third|evening|| +84728|AAAAAAAAJPKEBAAA|84728|23|32|8|PM|third|evening|| +84729|AAAAAAAAKPKEBAAA|84729|23|32|9|PM|third|evening|| +84730|AAAAAAAALPKEBAAA|84730|23|32|10|PM|third|evening|| +84731|AAAAAAAAMPKEBAAA|84731|23|32|11|PM|third|evening|| +84732|AAAAAAAANPKEBAAA|84732|23|32|12|PM|third|evening|| +84733|AAAAAAAAOPKEBAAA|84733|23|32|13|PM|third|evening|| +84734|AAAAAAAAPPKEBAAA|84734|23|32|14|PM|third|evening|| +84735|AAAAAAAAAALEBAAA|84735|23|32|15|PM|third|evening|| +84736|AAAAAAAABALEBAAA|84736|23|32|16|PM|third|evening|| +84737|AAAAAAAACALEBAAA|84737|23|32|17|PM|third|evening|| +84738|AAAAAAAADALEBAAA|84738|23|32|18|PM|third|evening|| +84739|AAAAAAAAEALEBAAA|84739|23|32|19|PM|third|evening|| +84740|AAAAAAAAFALEBAAA|84740|23|32|20|PM|third|evening|| +84741|AAAAAAAAGALEBAAA|84741|23|32|21|PM|third|evening|| +84742|AAAAAAAAHALEBAAA|84742|23|32|22|PM|third|evening|| +84743|AAAAAAAAIALEBAAA|84743|23|32|23|PM|third|evening|| +84744|AAAAAAAAJALEBAAA|84744|23|32|24|PM|third|evening|| +84745|AAAAAAAAKALEBAAA|84745|23|32|25|PM|third|evening|| +84746|AAAAAAAALALEBAAA|84746|23|32|26|PM|third|evening|| +84747|AAAAAAAAMALEBAAA|84747|23|32|27|PM|third|evening|| +84748|AAAAAAAANALEBAAA|84748|23|32|28|PM|third|evening|| +84749|AAAAAAAAOALEBAAA|84749|23|32|29|PM|third|evening|| +84750|AAAAAAAAPALEBAAA|84750|23|32|30|PM|third|evening|| +84751|AAAAAAAAABLEBAAA|84751|23|32|31|PM|third|evening|| +84752|AAAAAAAABBLEBAAA|84752|23|32|32|PM|third|evening|| +84753|AAAAAAAACBLEBAAA|84753|23|32|33|PM|third|evening|| +84754|AAAAAAAADBLEBAAA|84754|23|32|34|PM|third|evening|| +84755|AAAAAAAAEBLEBAAA|84755|23|32|35|PM|third|evening|| +84756|AAAAAAAAFBLEBAAA|84756|23|32|36|PM|third|evening|| +84757|AAAAAAAAGBLEBAAA|84757|23|32|37|PM|third|evening|| +84758|AAAAAAAAHBLEBAAA|84758|23|32|38|PM|third|evening|| +84759|AAAAAAAAIBLEBAAA|84759|23|32|39|PM|third|evening|| +84760|AAAAAAAAJBLEBAAA|84760|23|32|40|PM|third|evening|| +84761|AAAAAAAAKBLEBAAA|84761|23|32|41|PM|third|evening|| +84762|AAAAAAAALBLEBAAA|84762|23|32|42|PM|third|evening|| +84763|AAAAAAAAMBLEBAAA|84763|23|32|43|PM|third|evening|| +84764|AAAAAAAANBLEBAAA|84764|23|32|44|PM|third|evening|| +84765|AAAAAAAAOBLEBAAA|84765|23|32|45|PM|third|evening|| +84766|AAAAAAAAPBLEBAAA|84766|23|32|46|PM|third|evening|| +84767|AAAAAAAAACLEBAAA|84767|23|32|47|PM|third|evening|| +84768|AAAAAAAABCLEBAAA|84768|23|32|48|PM|third|evening|| +84769|AAAAAAAACCLEBAAA|84769|23|32|49|PM|third|evening|| +84770|AAAAAAAADCLEBAAA|84770|23|32|50|PM|third|evening|| +84771|AAAAAAAAECLEBAAA|84771|23|32|51|PM|third|evening|| +84772|AAAAAAAAFCLEBAAA|84772|23|32|52|PM|third|evening|| +84773|AAAAAAAAGCLEBAAA|84773|23|32|53|PM|third|evening|| +84774|AAAAAAAAHCLEBAAA|84774|23|32|54|PM|third|evening|| +84775|AAAAAAAAICLEBAAA|84775|23|32|55|PM|third|evening|| +84776|AAAAAAAAJCLEBAAA|84776|23|32|56|PM|third|evening|| +84777|AAAAAAAAKCLEBAAA|84777|23|32|57|PM|third|evening|| +84778|AAAAAAAALCLEBAAA|84778|23|32|58|PM|third|evening|| +84779|AAAAAAAAMCLEBAAA|84779|23|32|59|PM|third|evening|| +84780|AAAAAAAANCLEBAAA|84780|23|33|0|PM|third|evening|| +84781|AAAAAAAAOCLEBAAA|84781|23|33|1|PM|third|evening|| +84782|AAAAAAAAPCLEBAAA|84782|23|33|2|PM|third|evening|| +84783|AAAAAAAAADLEBAAA|84783|23|33|3|PM|third|evening|| +84784|AAAAAAAABDLEBAAA|84784|23|33|4|PM|third|evening|| +84785|AAAAAAAACDLEBAAA|84785|23|33|5|PM|third|evening|| +84786|AAAAAAAADDLEBAAA|84786|23|33|6|PM|third|evening|| +84787|AAAAAAAAEDLEBAAA|84787|23|33|7|PM|third|evening|| +84788|AAAAAAAAFDLEBAAA|84788|23|33|8|PM|third|evening|| +84789|AAAAAAAAGDLEBAAA|84789|23|33|9|PM|third|evening|| +84790|AAAAAAAAHDLEBAAA|84790|23|33|10|PM|third|evening|| +84791|AAAAAAAAIDLEBAAA|84791|23|33|11|PM|third|evening|| +84792|AAAAAAAAJDLEBAAA|84792|23|33|12|PM|third|evening|| +84793|AAAAAAAAKDLEBAAA|84793|23|33|13|PM|third|evening|| +84794|AAAAAAAALDLEBAAA|84794|23|33|14|PM|third|evening|| +84795|AAAAAAAAMDLEBAAA|84795|23|33|15|PM|third|evening|| +84796|AAAAAAAANDLEBAAA|84796|23|33|16|PM|third|evening|| +84797|AAAAAAAAODLEBAAA|84797|23|33|17|PM|third|evening|| +84798|AAAAAAAAPDLEBAAA|84798|23|33|18|PM|third|evening|| +84799|AAAAAAAAAELEBAAA|84799|23|33|19|PM|third|evening|| +84800|AAAAAAAABELEBAAA|84800|23|33|20|PM|third|evening|| +84801|AAAAAAAACELEBAAA|84801|23|33|21|PM|third|evening|| +84802|AAAAAAAADELEBAAA|84802|23|33|22|PM|third|evening|| +84803|AAAAAAAAEELEBAAA|84803|23|33|23|PM|third|evening|| +84804|AAAAAAAAFELEBAAA|84804|23|33|24|PM|third|evening|| +84805|AAAAAAAAGELEBAAA|84805|23|33|25|PM|third|evening|| +84806|AAAAAAAAHELEBAAA|84806|23|33|26|PM|third|evening|| +84807|AAAAAAAAIELEBAAA|84807|23|33|27|PM|third|evening|| +84808|AAAAAAAAJELEBAAA|84808|23|33|28|PM|third|evening|| +84809|AAAAAAAAKELEBAAA|84809|23|33|29|PM|third|evening|| +84810|AAAAAAAALELEBAAA|84810|23|33|30|PM|third|evening|| +84811|AAAAAAAAMELEBAAA|84811|23|33|31|PM|third|evening|| +84812|AAAAAAAANELEBAAA|84812|23|33|32|PM|third|evening|| +84813|AAAAAAAAOELEBAAA|84813|23|33|33|PM|third|evening|| +84814|AAAAAAAAPELEBAAA|84814|23|33|34|PM|third|evening|| +84815|AAAAAAAAAFLEBAAA|84815|23|33|35|PM|third|evening|| +84816|AAAAAAAABFLEBAAA|84816|23|33|36|PM|third|evening|| +84817|AAAAAAAACFLEBAAA|84817|23|33|37|PM|third|evening|| +84818|AAAAAAAADFLEBAAA|84818|23|33|38|PM|third|evening|| +84819|AAAAAAAAEFLEBAAA|84819|23|33|39|PM|third|evening|| +84820|AAAAAAAAFFLEBAAA|84820|23|33|40|PM|third|evening|| +84821|AAAAAAAAGFLEBAAA|84821|23|33|41|PM|third|evening|| +84822|AAAAAAAAHFLEBAAA|84822|23|33|42|PM|third|evening|| +84823|AAAAAAAAIFLEBAAA|84823|23|33|43|PM|third|evening|| +84824|AAAAAAAAJFLEBAAA|84824|23|33|44|PM|third|evening|| +84825|AAAAAAAAKFLEBAAA|84825|23|33|45|PM|third|evening|| +84826|AAAAAAAALFLEBAAA|84826|23|33|46|PM|third|evening|| +84827|AAAAAAAAMFLEBAAA|84827|23|33|47|PM|third|evening|| +84828|AAAAAAAANFLEBAAA|84828|23|33|48|PM|third|evening|| +84829|AAAAAAAAOFLEBAAA|84829|23|33|49|PM|third|evening|| +84830|AAAAAAAAPFLEBAAA|84830|23|33|50|PM|third|evening|| +84831|AAAAAAAAAGLEBAAA|84831|23|33|51|PM|third|evening|| +84832|AAAAAAAABGLEBAAA|84832|23|33|52|PM|third|evening|| +84833|AAAAAAAACGLEBAAA|84833|23|33|53|PM|third|evening|| +84834|AAAAAAAADGLEBAAA|84834|23|33|54|PM|third|evening|| +84835|AAAAAAAAEGLEBAAA|84835|23|33|55|PM|third|evening|| +84836|AAAAAAAAFGLEBAAA|84836|23|33|56|PM|third|evening|| +84837|AAAAAAAAGGLEBAAA|84837|23|33|57|PM|third|evening|| +84838|AAAAAAAAHGLEBAAA|84838|23|33|58|PM|third|evening|| +84839|AAAAAAAAIGLEBAAA|84839|23|33|59|PM|third|evening|| +84840|AAAAAAAAJGLEBAAA|84840|23|34|0|PM|third|evening|| +84841|AAAAAAAAKGLEBAAA|84841|23|34|1|PM|third|evening|| +84842|AAAAAAAALGLEBAAA|84842|23|34|2|PM|third|evening|| +84843|AAAAAAAAMGLEBAAA|84843|23|34|3|PM|third|evening|| +84844|AAAAAAAANGLEBAAA|84844|23|34|4|PM|third|evening|| +84845|AAAAAAAAOGLEBAAA|84845|23|34|5|PM|third|evening|| +84846|AAAAAAAAPGLEBAAA|84846|23|34|6|PM|third|evening|| +84847|AAAAAAAAAHLEBAAA|84847|23|34|7|PM|third|evening|| +84848|AAAAAAAABHLEBAAA|84848|23|34|8|PM|third|evening|| +84849|AAAAAAAACHLEBAAA|84849|23|34|9|PM|third|evening|| +84850|AAAAAAAADHLEBAAA|84850|23|34|10|PM|third|evening|| +84851|AAAAAAAAEHLEBAAA|84851|23|34|11|PM|third|evening|| +84852|AAAAAAAAFHLEBAAA|84852|23|34|12|PM|third|evening|| +84853|AAAAAAAAGHLEBAAA|84853|23|34|13|PM|third|evening|| +84854|AAAAAAAAHHLEBAAA|84854|23|34|14|PM|third|evening|| +84855|AAAAAAAAIHLEBAAA|84855|23|34|15|PM|third|evening|| +84856|AAAAAAAAJHLEBAAA|84856|23|34|16|PM|third|evening|| +84857|AAAAAAAAKHLEBAAA|84857|23|34|17|PM|third|evening|| +84858|AAAAAAAALHLEBAAA|84858|23|34|18|PM|third|evening|| +84859|AAAAAAAAMHLEBAAA|84859|23|34|19|PM|third|evening|| +84860|AAAAAAAANHLEBAAA|84860|23|34|20|PM|third|evening|| +84861|AAAAAAAAOHLEBAAA|84861|23|34|21|PM|third|evening|| +84862|AAAAAAAAPHLEBAAA|84862|23|34|22|PM|third|evening|| +84863|AAAAAAAAAILEBAAA|84863|23|34|23|PM|third|evening|| +84864|AAAAAAAABILEBAAA|84864|23|34|24|PM|third|evening|| +84865|AAAAAAAACILEBAAA|84865|23|34|25|PM|third|evening|| +84866|AAAAAAAADILEBAAA|84866|23|34|26|PM|third|evening|| +84867|AAAAAAAAEILEBAAA|84867|23|34|27|PM|third|evening|| +84868|AAAAAAAAFILEBAAA|84868|23|34|28|PM|third|evening|| +84869|AAAAAAAAGILEBAAA|84869|23|34|29|PM|third|evening|| +84870|AAAAAAAAHILEBAAA|84870|23|34|30|PM|third|evening|| +84871|AAAAAAAAIILEBAAA|84871|23|34|31|PM|third|evening|| +84872|AAAAAAAAJILEBAAA|84872|23|34|32|PM|third|evening|| +84873|AAAAAAAAKILEBAAA|84873|23|34|33|PM|third|evening|| +84874|AAAAAAAALILEBAAA|84874|23|34|34|PM|third|evening|| +84875|AAAAAAAAMILEBAAA|84875|23|34|35|PM|third|evening|| +84876|AAAAAAAANILEBAAA|84876|23|34|36|PM|third|evening|| +84877|AAAAAAAAOILEBAAA|84877|23|34|37|PM|third|evening|| +84878|AAAAAAAAPILEBAAA|84878|23|34|38|PM|third|evening|| +84879|AAAAAAAAAJLEBAAA|84879|23|34|39|PM|third|evening|| +84880|AAAAAAAABJLEBAAA|84880|23|34|40|PM|third|evening|| +84881|AAAAAAAACJLEBAAA|84881|23|34|41|PM|third|evening|| +84882|AAAAAAAADJLEBAAA|84882|23|34|42|PM|third|evening|| +84883|AAAAAAAAEJLEBAAA|84883|23|34|43|PM|third|evening|| +84884|AAAAAAAAFJLEBAAA|84884|23|34|44|PM|third|evening|| +84885|AAAAAAAAGJLEBAAA|84885|23|34|45|PM|third|evening|| +84886|AAAAAAAAHJLEBAAA|84886|23|34|46|PM|third|evening|| +84887|AAAAAAAAIJLEBAAA|84887|23|34|47|PM|third|evening|| +84888|AAAAAAAAJJLEBAAA|84888|23|34|48|PM|third|evening|| +84889|AAAAAAAAKJLEBAAA|84889|23|34|49|PM|third|evening|| +84890|AAAAAAAALJLEBAAA|84890|23|34|50|PM|third|evening|| +84891|AAAAAAAAMJLEBAAA|84891|23|34|51|PM|third|evening|| +84892|AAAAAAAANJLEBAAA|84892|23|34|52|PM|third|evening|| +84893|AAAAAAAAOJLEBAAA|84893|23|34|53|PM|third|evening|| +84894|AAAAAAAAPJLEBAAA|84894|23|34|54|PM|third|evening|| +84895|AAAAAAAAAKLEBAAA|84895|23|34|55|PM|third|evening|| +84896|AAAAAAAABKLEBAAA|84896|23|34|56|PM|third|evening|| +84897|AAAAAAAACKLEBAAA|84897|23|34|57|PM|third|evening|| +84898|AAAAAAAADKLEBAAA|84898|23|34|58|PM|third|evening|| +84899|AAAAAAAAEKLEBAAA|84899|23|34|59|PM|third|evening|| +84900|AAAAAAAAFKLEBAAA|84900|23|35|0|PM|third|evening|| +84901|AAAAAAAAGKLEBAAA|84901|23|35|1|PM|third|evening|| +84902|AAAAAAAAHKLEBAAA|84902|23|35|2|PM|third|evening|| +84903|AAAAAAAAIKLEBAAA|84903|23|35|3|PM|third|evening|| +84904|AAAAAAAAJKLEBAAA|84904|23|35|4|PM|third|evening|| +84905|AAAAAAAAKKLEBAAA|84905|23|35|5|PM|third|evening|| +84906|AAAAAAAALKLEBAAA|84906|23|35|6|PM|third|evening|| +84907|AAAAAAAAMKLEBAAA|84907|23|35|7|PM|third|evening|| +84908|AAAAAAAANKLEBAAA|84908|23|35|8|PM|third|evening|| +84909|AAAAAAAAOKLEBAAA|84909|23|35|9|PM|third|evening|| +84910|AAAAAAAAPKLEBAAA|84910|23|35|10|PM|third|evening|| +84911|AAAAAAAAALLEBAAA|84911|23|35|11|PM|third|evening|| +84912|AAAAAAAABLLEBAAA|84912|23|35|12|PM|third|evening|| +84913|AAAAAAAACLLEBAAA|84913|23|35|13|PM|third|evening|| +84914|AAAAAAAADLLEBAAA|84914|23|35|14|PM|third|evening|| +84915|AAAAAAAAELLEBAAA|84915|23|35|15|PM|third|evening|| +84916|AAAAAAAAFLLEBAAA|84916|23|35|16|PM|third|evening|| +84917|AAAAAAAAGLLEBAAA|84917|23|35|17|PM|third|evening|| +84918|AAAAAAAAHLLEBAAA|84918|23|35|18|PM|third|evening|| +84919|AAAAAAAAILLEBAAA|84919|23|35|19|PM|third|evening|| +84920|AAAAAAAAJLLEBAAA|84920|23|35|20|PM|third|evening|| +84921|AAAAAAAAKLLEBAAA|84921|23|35|21|PM|third|evening|| +84922|AAAAAAAALLLEBAAA|84922|23|35|22|PM|third|evening|| +84923|AAAAAAAAMLLEBAAA|84923|23|35|23|PM|third|evening|| +84924|AAAAAAAANLLEBAAA|84924|23|35|24|PM|third|evening|| +84925|AAAAAAAAOLLEBAAA|84925|23|35|25|PM|third|evening|| +84926|AAAAAAAAPLLEBAAA|84926|23|35|26|PM|third|evening|| +84927|AAAAAAAAAMLEBAAA|84927|23|35|27|PM|third|evening|| +84928|AAAAAAAABMLEBAAA|84928|23|35|28|PM|third|evening|| +84929|AAAAAAAACMLEBAAA|84929|23|35|29|PM|third|evening|| +84930|AAAAAAAADMLEBAAA|84930|23|35|30|PM|third|evening|| +84931|AAAAAAAAEMLEBAAA|84931|23|35|31|PM|third|evening|| +84932|AAAAAAAAFMLEBAAA|84932|23|35|32|PM|third|evening|| +84933|AAAAAAAAGMLEBAAA|84933|23|35|33|PM|third|evening|| +84934|AAAAAAAAHMLEBAAA|84934|23|35|34|PM|third|evening|| +84935|AAAAAAAAIMLEBAAA|84935|23|35|35|PM|third|evening|| +84936|AAAAAAAAJMLEBAAA|84936|23|35|36|PM|third|evening|| +84937|AAAAAAAAKMLEBAAA|84937|23|35|37|PM|third|evening|| +84938|AAAAAAAALMLEBAAA|84938|23|35|38|PM|third|evening|| +84939|AAAAAAAAMMLEBAAA|84939|23|35|39|PM|third|evening|| +84940|AAAAAAAANMLEBAAA|84940|23|35|40|PM|third|evening|| +84941|AAAAAAAAOMLEBAAA|84941|23|35|41|PM|third|evening|| +84942|AAAAAAAAPMLEBAAA|84942|23|35|42|PM|third|evening|| +84943|AAAAAAAAANLEBAAA|84943|23|35|43|PM|third|evening|| +84944|AAAAAAAABNLEBAAA|84944|23|35|44|PM|third|evening|| +84945|AAAAAAAACNLEBAAA|84945|23|35|45|PM|third|evening|| +84946|AAAAAAAADNLEBAAA|84946|23|35|46|PM|third|evening|| +84947|AAAAAAAAENLEBAAA|84947|23|35|47|PM|third|evening|| +84948|AAAAAAAAFNLEBAAA|84948|23|35|48|PM|third|evening|| +84949|AAAAAAAAGNLEBAAA|84949|23|35|49|PM|third|evening|| +84950|AAAAAAAAHNLEBAAA|84950|23|35|50|PM|third|evening|| +84951|AAAAAAAAINLEBAAA|84951|23|35|51|PM|third|evening|| +84952|AAAAAAAAJNLEBAAA|84952|23|35|52|PM|third|evening|| +84953|AAAAAAAAKNLEBAAA|84953|23|35|53|PM|third|evening|| +84954|AAAAAAAALNLEBAAA|84954|23|35|54|PM|third|evening|| +84955|AAAAAAAAMNLEBAAA|84955|23|35|55|PM|third|evening|| +84956|AAAAAAAANNLEBAAA|84956|23|35|56|PM|third|evening|| +84957|AAAAAAAAONLEBAAA|84957|23|35|57|PM|third|evening|| +84958|AAAAAAAAPNLEBAAA|84958|23|35|58|PM|third|evening|| +84959|AAAAAAAAAOLEBAAA|84959|23|35|59|PM|third|evening|| +84960|AAAAAAAABOLEBAAA|84960|23|36|0|PM|third|evening|| +84961|AAAAAAAACOLEBAAA|84961|23|36|1|PM|third|evening|| +84962|AAAAAAAADOLEBAAA|84962|23|36|2|PM|third|evening|| +84963|AAAAAAAAEOLEBAAA|84963|23|36|3|PM|third|evening|| +84964|AAAAAAAAFOLEBAAA|84964|23|36|4|PM|third|evening|| +84965|AAAAAAAAGOLEBAAA|84965|23|36|5|PM|third|evening|| +84966|AAAAAAAAHOLEBAAA|84966|23|36|6|PM|third|evening|| +84967|AAAAAAAAIOLEBAAA|84967|23|36|7|PM|third|evening|| +84968|AAAAAAAAJOLEBAAA|84968|23|36|8|PM|third|evening|| +84969|AAAAAAAAKOLEBAAA|84969|23|36|9|PM|third|evening|| +84970|AAAAAAAALOLEBAAA|84970|23|36|10|PM|third|evening|| +84971|AAAAAAAAMOLEBAAA|84971|23|36|11|PM|third|evening|| +84972|AAAAAAAANOLEBAAA|84972|23|36|12|PM|third|evening|| +84973|AAAAAAAAOOLEBAAA|84973|23|36|13|PM|third|evening|| +84974|AAAAAAAAPOLEBAAA|84974|23|36|14|PM|third|evening|| +84975|AAAAAAAAAPLEBAAA|84975|23|36|15|PM|third|evening|| +84976|AAAAAAAABPLEBAAA|84976|23|36|16|PM|third|evening|| +84977|AAAAAAAACPLEBAAA|84977|23|36|17|PM|third|evening|| +84978|AAAAAAAADPLEBAAA|84978|23|36|18|PM|third|evening|| +84979|AAAAAAAAEPLEBAAA|84979|23|36|19|PM|third|evening|| +84980|AAAAAAAAFPLEBAAA|84980|23|36|20|PM|third|evening|| +84981|AAAAAAAAGPLEBAAA|84981|23|36|21|PM|third|evening|| +84982|AAAAAAAAHPLEBAAA|84982|23|36|22|PM|third|evening|| +84983|AAAAAAAAIPLEBAAA|84983|23|36|23|PM|third|evening|| +84984|AAAAAAAAJPLEBAAA|84984|23|36|24|PM|third|evening|| +84985|AAAAAAAAKPLEBAAA|84985|23|36|25|PM|third|evening|| +84986|AAAAAAAALPLEBAAA|84986|23|36|26|PM|third|evening|| +84987|AAAAAAAAMPLEBAAA|84987|23|36|27|PM|third|evening|| +84988|AAAAAAAANPLEBAAA|84988|23|36|28|PM|third|evening|| +84989|AAAAAAAAOPLEBAAA|84989|23|36|29|PM|third|evening|| +84990|AAAAAAAAPPLEBAAA|84990|23|36|30|PM|third|evening|| +84991|AAAAAAAAAAMEBAAA|84991|23|36|31|PM|third|evening|| +84992|AAAAAAAABAMEBAAA|84992|23|36|32|PM|third|evening|| +84993|AAAAAAAACAMEBAAA|84993|23|36|33|PM|third|evening|| +84994|AAAAAAAADAMEBAAA|84994|23|36|34|PM|third|evening|| +84995|AAAAAAAAEAMEBAAA|84995|23|36|35|PM|third|evening|| +84996|AAAAAAAAFAMEBAAA|84996|23|36|36|PM|third|evening|| +84997|AAAAAAAAGAMEBAAA|84997|23|36|37|PM|third|evening|| +84998|AAAAAAAAHAMEBAAA|84998|23|36|38|PM|third|evening|| +84999|AAAAAAAAIAMEBAAA|84999|23|36|39|PM|third|evening|| +85000|AAAAAAAAJAMEBAAA|85000|23|36|40|PM|third|evening|| +85001|AAAAAAAAKAMEBAAA|85001|23|36|41|PM|third|evening|| +85002|AAAAAAAALAMEBAAA|85002|23|36|42|PM|third|evening|| +85003|AAAAAAAAMAMEBAAA|85003|23|36|43|PM|third|evening|| +85004|AAAAAAAANAMEBAAA|85004|23|36|44|PM|third|evening|| +85005|AAAAAAAAOAMEBAAA|85005|23|36|45|PM|third|evening|| +85006|AAAAAAAAPAMEBAAA|85006|23|36|46|PM|third|evening|| +85007|AAAAAAAAABMEBAAA|85007|23|36|47|PM|third|evening|| +85008|AAAAAAAABBMEBAAA|85008|23|36|48|PM|third|evening|| +85009|AAAAAAAACBMEBAAA|85009|23|36|49|PM|third|evening|| +85010|AAAAAAAADBMEBAAA|85010|23|36|50|PM|third|evening|| +85011|AAAAAAAAEBMEBAAA|85011|23|36|51|PM|third|evening|| +85012|AAAAAAAAFBMEBAAA|85012|23|36|52|PM|third|evening|| +85013|AAAAAAAAGBMEBAAA|85013|23|36|53|PM|third|evening|| +85014|AAAAAAAAHBMEBAAA|85014|23|36|54|PM|third|evening|| +85015|AAAAAAAAIBMEBAAA|85015|23|36|55|PM|third|evening|| +85016|AAAAAAAAJBMEBAAA|85016|23|36|56|PM|third|evening|| +85017|AAAAAAAAKBMEBAAA|85017|23|36|57|PM|third|evening|| +85018|AAAAAAAALBMEBAAA|85018|23|36|58|PM|third|evening|| +85019|AAAAAAAAMBMEBAAA|85019|23|36|59|PM|third|evening|| +85020|AAAAAAAANBMEBAAA|85020|23|37|0|PM|third|evening|| +85021|AAAAAAAAOBMEBAAA|85021|23|37|1|PM|third|evening|| +85022|AAAAAAAAPBMEBAAA|85022|23|37|2|PM|third|evening|| +85023|AAAAAAAAACMEBAAA|85023|23|37|3|PM|third|evening|| +85024|AAAAAAAABCMEBAAA|85024|23|37|4|PM|third|evening|| +85025|AAAAAAAACCMEBAAA|85025|23|37|5|PM|third|evening|| +85026|AAAAAAAADCMEBAAA|85026|23|37|6|PM|third|evening|| +85027|AAAAAAAAECMEBAAA|85027|23|37|7|PM|third|evening|| +85028|AAAAAAAAFCMEBAAA|85028|23|37|8|PM|third|evening|| +85029|AAAAAAAAGCMEBAAA|85029|23|37|9|PM|third|evening|| +85030|AAAAAAAAHCMEBAAA|85030|23|37|10|PM|third|evening|| +85031|AAAAAAAAICMEBAAA|85031|23|37|11|PM|third|evening|| +85032|AAAAAAAAJCMEBAAA|85032|23|37|12|PM|third|evening|| +85033|AAAAAAAAKCMEBAAA|85033|23|37|13|PM|third|evening|| +85034|AAAAAAAALCMEBAAA|85034|23|37|14|PM|third|evening|| +85035|AAAAAAAAMCMEBAAA|85035|23|37|15|PM|third|evening|| +85036|AAAAAAAANCMEBAAA|85036|23|37|16|PM|third|evening|| +85037|AAAAAAAAOCMEBAAA|85037|23|37|17|PM|third|evening|| +85038|AAAAAAAAPCMEBAAA|85038|23|37|18|PM|third|evening|| +85039|AAAAAAAAADMEBAAA|85039|23|37|19|PM|third|evening|| +85040|AAAAAAAABDMEBAAA|85040|23|37|20|PM|third|evening|| +85041|AAAAAAAACDMEBAAA|85041|23|37|21|PM|third|evening|| +85042|AAAAAAAADDMEBAAA|85042|23|37|22|PM|third|evening|| +85043|AAAAAAAAEDMEBAAA|85043|23|37|23|PM|third|evening|| +85044|AAAAAAAAFDMEBAAA|85044|23|37|24|PM|third|evening|| +85045|AAAAAAAAGDMEBAAA|85045|23|37|25|PM|third|evening|| +85046|AAAAAAAAHDMEBAAA|85046|23|37|26|PM|third|evening|| +85047|AAAAAAAAIDMEBAAA|85047|23|37|27|PM|third|evening|| +85048|AAAAAAAAJDMEBAAA|85048|23|37|28|PM|third|evening|| +85049|AAAAAAAAKDMEBAAA|85049|23|37|29|PM|third|evening|| +85050|AAAAAAAALDMEBAAA|85050|23|37|30|PM|third|evening|| +85051|AAAAAAAAMDMEBAAA|85051|23|37|31|PM|third|evening|| +85052|AAAAAAAANDMEBAAA|85052|23|37|32|PM|third|evening|| +85053|AAAAAAAAODMEBAAA|85053|23|37|33|PM|third|evening|| +85054|AAAAAAAAPDMEBAAA|85054|23|37|34|PM|third|evening|| +85055|AAAAAAAAAEMEBAAA|85055|23|37|35|PM|third|evening|| +85056|AAAAAAAABEMEBAAA|85056|23|37|36|PM|third|evening|| +85057|AAAAAAAACEMEBAAA|85057|23|37|37|PM|third|evening|| +85058|AAAAAAAADEMEBAAA|85058|23|37|38|PM|third|evening|| +85059|AAAAAAAAEEMEBAAA|85059|23|37|39|PM|third|evening|| +85060|AAAAAAAAFEMEBAAA|85060|23|37|40|PM|third|evening|| +85061|AAAAAAAAGEMEBAAA|85061|23|37|41|PM|third|evening|| +85062|AAAAAAAAHEMEBAAA|85062|23|37|42|PM|third|evening|| +85063|AAAAAAAAIEMEBAAA|85063|23|37|43|PM|third|evening|| +85064|AAAAAAAAJEMEBAAA|85064|23|37|44|PM|third|evening|| +85065|AAAAAAAAKEMEBAAA|85065|23|37|45|PM|third|evening|| +85066|AAAAAAAALEMEBAAA|85066|23|37|46|PM|third|evening|| +85067|AAAAAAAAMEMEBAAA|85067|23|37|47|PM|third|evening|| +85068|AAAAAAAANEMEBAAA|85068|23|37|48|PM|third|evening|| +85069|AAAAAAAAOEMEBAAA|85069|23|37|49|PM|third|evening|| +85070|AAAAAAAAPEMEBAAA|85070|23|37|50|PM|third|evening|| +85071|AAAAAAAAAFMEBAAA|85071|23|37|51|PM|third|evening|| +85072|AAAAAAAABFMEBAAA|85072|23|37|52|PM|third|evening|| +85073|AAAAAAAACFMEBAAA|85073|23|37|53|PM|third|evening|| +85074|AAAAAAAADFMEBAAA|85074|23|37|54|PM|third|evening|| +85075|AAAAAAAAEFMEBAAA|85075|23|37|55|PM|third|evening|| +85076|AAAAAAAAFFMEBAAA|85076|23|37|56|PM|third|evening|| +85077|AAAAAAAAGFMEBAAA|85077|23|37|57|PM|third|evening|| +85078|AAAAAAAAHFMEBAAA|85078|23|37|58|PM|third|evening|| +85079|AAAAAAAAIFMEBAAA|85079|23|37|59|PM|third|evening|| +85080|AAAAAAAAJFMEBAAA|85080|23|38|0|PM|third|evening|| +85081|AAAAAAAAKFMEBAAA|85081|23|38|1|PM|third|evening|| +85082|AAAAAAAALFMEBAAA|85082|23|38|2|PM|third|evening|| +85083|AAAAAAAAMFMEBAAA|85083|23|38|3|PM|third|evening|| +85084|AAAAAAAANFMEBAAA|85084|23|38|4|PM|third|evening|| +85085|AAAAAAAAOFMEBAAA|85085|23|38|5|PM|third|evening|| +85086|AAAAAAAAPFMEBAAA|85086|23|38|6|PM|third|evening|| +85087|AAAAAAAAAGMEBAAA|85087|23|38|7|PM|third|evening|| +85088|AAAAAAAABGMEBAAA|85088|23|38|8|PM|third|evening|| +85089|AAAAAAAACGMEBAAA|85089|23|38|9|PM|third|evening|| +85090|AAAAAAAADGMEBAAA|85090|23|38|10|PM|third|evening|| +85091|AAAAAAAAEGMEBAAA|85091|23|38|11|PM|third|evening|| +85092|AAAAAAAAFGMEBAAA|85092|23|38|12|PM|third|evening|| +85093|AAAAAAAAGGMEBAAA|85093|23|38|13|PM|third|evening|| +85094|AAAAAAAAHGMEBAAA|85094|23|38|14|PM|third|evening|| +85095|AAAAAAAAIGMEBAAA|85095|23|38|15|PM|third|evening|| +85096|AAAAAAAAJGMEBAAA|85096|23|38|16|PM|third|evening|| +85097|AAAAAAAAKGMEBAAA|85097|23|38|17|PM|third|evening|| +85098|AAAAAAAALGMEBAAA|85098|23|38|18|PM|third|evening|| +85099|AAAAAAAAMGMEBAAA|85099|23|38|19|PM|third|evening|| +85100|AAAAAAAANGMEBAAA|85100|23|38|20|PM|third|evening|| +85101|AAAAAAAAOGMEBAAA|85101|23|38|21|PM|third|evening|| +85102|AAAAAAAAPGMEBAAA|85102|23|38|22|PM|third|evening|| +85103|AAAAAAAAAHMEBAAA|85103|23|38|23|PM|third|evening|| +85104|AAAAAAAABHMEBAAA|85104|23|38|24|PM|third|evening|| +85105|AAAAAAAACHMEBAAA|85105|23|38|25|PM|third|evening|| +85106|AAAAAAAADHMEBAAA|85106|23|38|26|PM|third|evening|| +85107|AAAAAAAAEHMEBAAA|85107|23|38|27|PM|third|evening|| +85108|AAAAAAAAFHMEBAAA|85108|23|38|28|PM|third|evening|| +85109|AAAAAAAAGHMEBAAA|85109|23|38|29|PM|third|evening|| +85110|AAAAAAAAHHMEBAAA|85110|23|38|30|PM|third|evening|| +85111|AAAAAAAAIHMEBAAA|85111|23|38|31|PM|third|evening|| +85112|AAAAAAAAJHMEBAAA|85112|23|38|32|PM|third|evening|| +85113|AAAAAAAAKHMEBAAA|85113|23|38|33|PM|third|evening|| +85114|AAAAAAAALHMEBAAA|85114|23|38|34|PM|third|evening|| +85115|AAAAAAAAMHMEBAAA|85115|23|38|35|PM|third|evening|| +85116|AAAAAAAANHMEBAAA|85116|23|38|36|PM|third|evening|| +85117|AAAAAAAAOHMEBAAA|85117|23|38|37|PM|third|evening|| +85118|AAAAAAAAPHMEBAAA|85118|23|38|38|PM|third|evening|| +85119|AAAAAAAAAIMEBAAA|85119|23|38|39|PM|third|evening|| +85120|AAAAAAAABIMEBAAA|85120|23|38|40|PM|third|evening|| +85121|AAAAAAAACIMEBAAA|85121|23|38|41|PM|third|evening|| +85122|AAAAAAAADIMEBAAA|85122|23|38|42|PM|third|evening|| +85123|AAAAAAAAEIMEBAAA|85123|23|38|43|PM|third|evening|| +85124|AAAAAAAAFIMEBAAA|85124|23|38|44|PM|third|evening|| +85125|AAAAAAAAGIMEBAAA|85125|23|38|45|PM|third|evening|| +85126|AAAAAAAAHIMEBAAA|85126|23|38|46|PM|third|evening|| +85127|AAAAAAAAIIMEBAAA|85127|23|38|47|PM|third|evening|| +85128|AAAAAAAAJIMEBAAA|85128|23|38|48|PM|third|evening|| +85129|AAAAAAAAKIMEBAAA|85129|23|38|49|PM|third|evening|| +85130|AAAAAAAALIMEBAAA|85130|23|38|50|PM|third|evening|| +85131|AAAAAAAAMIMEBAAA|85131|23|38|51|PM|third|evening|| +85132|AAAAAAAANIMEBAAA|85132|23|38|52|PM|third|evening|| +85133|AAAAAAAAOIMEBAAA|85133|23|38|53|PM|third|evening|| +85134|AAAAAAAAPIMEBAAA|85134|23|38|54|PM|third|evening|| +85135|AAAAAAAAAJMEBAAA|85135|23|38|55|PM|third|evening|| +85136|AAAAAAAABJMEBAAA|85136|23|38|56|PM|third|evening|| +85137|AAAAAAAACJMEBAAA|85137|23|38|57|PM|third|evening|| +85138|AAAAAAAADJMEBAAA|85138|23|38|58|PM|third|evening|| +85139|AAAAAAAAEJMEBAAA|85139|23|38|59|PM|third|evening|| +85140|AAAAAAAAFJMEBAAA|85140|23|39|0|PM|third|evening|| +85141|AAAAAAAAGJMEBAAA|85141|23|39|1|PM|third|evening|| +85142|AAAAAAAAHJMEBAAA|85142|23|39|2|PM|third|evening|| +85143|AAAAAAAAIJMEBAAA|85143|23|39|3|PM|third|evening|| +85144|AAAAAAAAJJMEBAAA|85144|23|39|4|PM|third|evening|| +85145|AAAAAAAAKJMEBAAA|85145|23|39|5|PM|third|evening|| +85146|AAAAAAAALJMEBAAA|85146|23|39|6|PM|third|evening|| +85147|AAAAAAAAMJMEBAAA|85147|23|39|7|PM|third|evening|| +85148|AAAAAAAANJMEBAAA|85148|23|39|8|PM|third|evening|| +85149|AAAAAAAAOJMEBAAA|85149|23|39|9|PM|third|evening|| +85150|AAAAAAAAPJMEBAAA|85150|23|39|10|PM|third|evening|| +85151|AAAAAAAAAKMEBAAA|85151|23|39|11|PM|third|evening|| +85152|AAAAAAAABKMEBAAA|85152|23|39|12|PM|third|evening|| +85153|AAAAAAAACKMEBAAA|85153|23|39|13|PM|third|evening|| +85154|AAAAAAAADKMEBAAA|85154|23|39|14|PM|third|evening|| +85155|AAAAAAAAEKMEBAAA|85155|23|39|15|PM|third|evening|| +85156|AAAAAAAAFKMEBAAA|85156|23|39|16|PM|third|evening|| +85157|AAAAAAAAGKMEBAAA|85157|23|39|17|PM|third|evening|| +85158|AAAAAAAAHKMEBAAA|85158|23|39|18|PM|third|evening|| +85159|AAAAAAAAIKMEBAAA|85159|23|39|19|PM|third|evening|| +85160|AAAAAAAAJKMEBAAA|85160|23|39|20|PM|third|evening|| +85161|AAAAAAAAKKMEBAAA|85161|23|39|21|PM|third|evening|| +85162|AAAAAAAALKMEBAAA|85162|23|39|22|PM|third|evening|| +85163|AAAAAAAAMKMEBAAA|85163|23|39|23|PM|third|evening|| +85164|AAAAAAAANKMEBAAA|85164|23|39|24|PM|third|evening|| +85165|AAAAAAAAOKMEBAAA|85165|23|39|25|PM|third|evening|| +85166|AAAAAAAAPKMEBAAA|85166|23|39|26|PM|third|evening|| +85167|AAAAAAAAALMEBAAA|85167|23|39|27|PM|third|evening|| +85168|AAAAAAAABLMEBAAA|85168|23|39|28|PM|third|evening|| +85169|AAAAAAAACLMEBAAA|85169|23|39|29|PM|third|evening|| +85170|AAAAAAAADLMEBAAA|85170|23|39|30|PM|third|evening|| +85171|AAAAAAAAELMEBAAA|85171|23|39|31|PM|third|evening|| +85172|AAAAAAAAFLMEBAAA|85172|23|39|32|PM|third|evening|| +85173|AAAAAAAAGLMEBAAA|85173|23|39|33|PM|third|evening|| +85174|AAAAAAAAHLMEBAAA|85174|23|39|34|PM|third|evening|| +85175|AAAAAAAAILMEBAAA|85175|23|39|35|PM|third|evening|| +85176|AAAAAAAAJLMEBAAA|85176|23|39|36|PM|third|evening|| +85177|AAAAAAAAKLMEBAAA|85177|23|39|37|PM|third|evening|| +85178|AAAAAAAALLMEBAAA|85178|23|39|38|PM|third|evening|| +85179|AAAAAAAAMLMEBAAA|85179|23|39|39|PM|third|evening|| +85180|AAAAAAAANLMEBAAA|85180|23|39|40|PM|third|evening|| +85181|AAAAAAAAOLMEBAAA|85181|23|39|41|PM|third|evening|| +85182|AAAAAAAAPLMEBAAA|85182|23|39|42|PM|third|evening|| +85183|AAAAAAAAAMMEBAAA|85183|23|39|43|PM|third|evening|| +85184|AAAAAAAABMMEBAAA|85184|23|39|44|PM|third|evening|| +85185|AAAAAAAACMMEBAAA|85185|23|39|45|PM|third|evening|| +85186|AAAAAAAADMMEBAAA|85186|23|39|46|PM|third|evening|| +85187|AAAAAAAAEMMEBAAA|85187|23|39|47|PM|third|evening|| +85188|AAAAAAAAFMMEBAAA|85188|23|39|48|PM|third|evening|| +85189|AAAAAAAAGMMEBAAA|85189|23|39|49|PM|third|evening|| +85190|AAAAAAAAHMMEBAAA|85190|23|39|50|PM|third|evening|| +85191|AAAAAAAAIMMEBAAA|85191|23|39|51|PM|third|evening|| +85192|AAAAAAAAJMMEBAAA|85192|23|39|52|PM|third|evening|| +85193|AAAAAAAAKMMEBAAA|85193|23|39|53|PM|third|evening|| +85194|AAAAAAAALMMEBAAA|85194|23|39|54|PM|third|evening|| +85195|AAAAAAAAMMMEBAAA|85195|23|39|55|PM|third|evening|| +85196|AAAAAAAANMMEBAAA|85196|23|39|56|PM|third|evening|| +85197|AAAAAAAAOMMEBAAA|85197|23|39|57|PM|third|evening|| +85198|AAAAAAAAPMMEBAAA|85198|23|39|58|PM|third|evening|| +85199|AAAAAAAAANMEBAAA|85199|23|39|59|PM|third|evening|| +85200|AAAAAAAABNMEBAAA|85200|23|40|0|PM|third|evening|| +85201|AAAAAAAACNMEBAAA|85201|23|40|1|PM|third|evening|| +85202|AAAAAAAADNMEBAAA|85202|23|40|2|PM|third|evening|| +85203|AAAAAAAAENMEBAAA|85203|23|40|3|PM|third|evening|| +85204|AAAAAAAAFNMEBAAA|85204|23|40|4|PM|third|evening|| +85205|AAAAAAAAGNMEBAAA|85205|23|40|5|PM|third|evening|| +85206|AAAAAAAAHNMEBAAA|85206|23|40|6|PM|third|evening|| +85207|AAAAAAAAINMEBAAA|85207|23|40|7|PM|third|evening|| +85208|AAAAAAAAJNMEBAAA|85208|23|40|8|PM|third|evening|| +85209|AAAAAAAAKNMEBAAA|85209|23|40|9|PM|third|evening|| +85210|AAAAAAAALNMEBAAA|85210|23|40|10|PM|third|evening|| +85211|AAAAAAAAMNMEBAAA|85211|23|40|11|PM|third|evening|| +85212|AAAAAAAANNMEBAAA|85212|23|40|12|PM|third|evening|| +85213|AAAAAAAAONMEBAAA|85213|23|40|13|PM|third|evening|| +85214|AAAAAAAAPNMEBAAA|85214|23|40|14|PM|third|evening|| +85215|AAAAAAAAAOMEBAAA|85215|23|40|15|PM|third|evening|| +85216|AAAAAAAABOMEBAAA|85216|23|40|16|PM|third|evening|| +85217|AAAAAAAACOMEBAAA|85217|23|40|17|PM|third|evening|| +85218|AAAAAAAADOMEBAAA|85218|23|40|18|PM|third|evening|| +85219|AAAAAAAAEOMEBAAA|85219|23|40|19|PM|third|evening|| +85220|AAAAAAAAFOMEBAAA|85220|23|40|20|PM|third|evening|| +85221|AAAAAAAAGOMEBAAA|85221|23|40|21|PM|third|evening|| +85222|AAAAAAAAHOMEBAAA|85222|23|40|22|PM|third|evening|| +85223|AAAAAAAAIOMEBAAA|85223|23|40|23|PM|third|evening|| +85224|AAAAAAAAJOMEBAAA|85224|23|40|24|PM|third|evening|| +85225|AAAAAAAAKOMEBAAA|85225|23|40|25|PM|third|evening|| +85226|AAAAAAAALOMEBAAA|85226|23|40|26|PM|third|evening|| +85227|AAAAAAAAMOMEBAAA|85227|23|40|27|PM|third|evening|| +85228|AAAAAAAANOMEBAAA|85228|23|40|28|PM|third|evening|| +85229|AAAAAAAAOOMEBAAA|85229|23|40|29|PM|third|evening|| +85230|AAAAAAAAPOMEBAAA|85230|23|40|30|PM|third|evening|| +85231|AAAAAAAAAPMEBAAA|85231|23|40|31|PM|third|evening|| +85232|AAAAAAAABPMEBAAA|85232|23|40|32|PM|third|evening|| +85233|AAAAAAAACPMEBAAA|85233|23|40|33|PM|third|evening|| +85234|AAAAAAAADPMEBAAA|85234|23|40|34|PM|third|evening|| +85235|AAAAAAAAEPMEBAAA|85235|23|40|35|PM|third|evening|| +85236|AAAAAAAAFPMEBAAA|85236|23|40|36|PM|third|evening|| +85237|AAAAAAAAGPMEBAAA|85237|23|40|37|PM|third|evening|| +85238|AAAAAAAAHPMEBAAA|85238|23|40|38|PM|third|evening|| +85239|AAAAAAAAIPMEBAAA|85239|23|40|39|PM|third|evening|| +85240|AAAAAAAAJPMEBAAA|85240|23|40|40|PM|third|evening|| +85241|AAAAAAAAKPMEBAAA|85241|23|40|41|PM|third|evening|| +85242|AAAAAAAALPMEBAAA|85242|23|40|42|PM|third|evening|| +85243|AAAAAAAAMPMEBAAA|85243|23|40|43|PM|third|evening|| +85244|AAAAAAAANPMEBAAA|85244|23|40|44|PM|third|evening|| +85245|AAAAAAAAOPMEBAAA|85245|23|40|45|PM|third|evening|| +85246|AAAAAAAAPPMEBAAA|85246|23|40|46|PM|third|evening|| +85247|AAAAAAAAAANEBAAA|85247|23|40|47|PM|third|evening|| +85248|AAAAAAAABANEBAAA|85248|23|40|48|PM|third|evening|| +85249|AAAAAAAACANEBAAA|85249|23|40|49|PM|third|evening|| +85250|AAAAAAAADANEBAAA|85250|23|40|50|PM|third|evening|| +85251|AAAAAAAAEANEBAAA|85251|23|40|51|PM|third|evening|| +85252|AAAAAAAAFANEBAAA|85252|23|40|52|PM|third|evening|| +85253|AAAAAAAAGANEBAAA|85253|23|40|53|PM|third|evening|| +85254|AAAAAAAAHANEBAAA|85254|23|40|54|PM|third|evening|| +85255|AAAAAAAAIANEBAAA|85255|23|40|55|PM|third|evening|| +85256|AAAAAAAAJANEBAAA|85256|23|40|56|PM|third|evening|| +85257|AAAAAAAAKANEBAAA|85257|23|40|57|PM|third|evening|| +85258|AAAAAAAALANEBAAA|85258|23|40|58|PM|third|evening|| +85259|AAAAAAAAMANEBAAA|85259|23|40|59|PM|third|evening|| +85260|AAAAAAAANANEBAAA|85260|23|41|0|PM|third|evening|| +85261|AAAAAAAAOANEBAAA|85261|23|41|1|PM|third|evening|| +85262|AAAAAAAAPANEBAAA|85262|23|41|2|PM|third|evening|| +85263|AAAAAAAAABNEBAAA|85263|23|41|3|PM|third|evening|| +85264|AAAAAAAABBNEBAAA|85264|23|41|4|PM|third|evening|| +85265|AAAAAAAACBNEBAAA|85265|23|41|5|PM|third|evening|| +85266|AAAAAAAADBNEBAAA|85266|23|41|6|PM|third|evening|| +85267|AAAAAAAAEBNEBAAA|85267|23|41|7|PM|third|evening|| +85268|AAAAAAAAFBNEBAAA|85268|23|41|8|PM|third|evening|| +85269|AAAAAAAAGBNEBAAA|85269|23|41|9|PM|third|evening|| +85270|AAAAAAAAHBNEBAAA|85270|23|41|10|PM|third|evening|| +85271|AAAAAAAAIBNEBAAA|85271|23|41|11|PM|third|evening|| +85272|AAAAAAAAJBNEBAAA|85272|23|41|12|PM|third|evening|| +85273|AAAAAAAAKBNEBAAA|85273|23|41|13|PM|third|evening|| +85274|AAAAAAAALBNEBAAA|85274|23|41|14|PM|third|evening|| +85275|AAAAAAAAMBNEBAAA|85275|23|41|15|PM|third|evening|| +85276|AAAAAAAANBNEBAAA|85276|23|41|16|PM|third|evening|| +85277|AAAAAAAAOBNEBAAA|85277|23|41|17|PM|third|evening|| +85278|AAAAAAAAPBNEBAAA|85278|23|41|18|PM|third|evening|| +85279|AAAAAAAAACNEBAAA|85279|23|41|19|PM|third|evening|| +85280|AAAAAAAABCNEBAAA|85280|23|41|20|PM|third|evening|| +85281|AAAAAAAACCNEBAAA|85281|23|41|21|PM|third|evening|| +85282|AAAAAAAADCNEBAAA|85282|23|41|22|PM|third|evening|| +85283|AAAAAAAAECNEBAAA|85283|23|41|23|PM|third|evening|| +85284|AAAAAAAAFCNEBAAA|85284|23|41|24|PM|third|evening|| +85285|AAAAAAAAGCNEBAAA|85285|23|41|25|PM|third|evening|| +85286|AAAAAAAAHCNEBAAA|85286|23|41|26|PM|third|evening|| +85287|AAAAAAAAICNEBAAA|85287|23|41|27|PM|third|evening|| +85288|AAAAAAAAJCNEBAAA|85288|23|41|28|PM|third|evening|| +85289|AAAAAAAAKCNEBAAA|85289|23|41|29|PM|third|evening|| +85290|AAAAAAAALCNEBAAA|85290|23|41|30|PM|third|evening|| +85291|AAAAAAAAMCNEBAAA|85291|23|41|31|PM|third|evening|| +85292|AAAAAAAANCNEBAAA|85292|23|41|32|PM|third|evening|| +85293|AAAAAAAAOCNEBAAA|85293|23|41|33|PM|third|evening|| +85294|AAAAAAAAPCNEBAAA|85294|23|41|34|PM|third|evening|| +85295|AAAAAAAAADNEBAAA|85295|23|41|35|PM|third|evening|| +85296|AAAAAAAABDNEBAAA|85296|23|41|36|PM|third|evening|| +85297|AAAAAAAACDNEBAAA|85297|23|41|37|PM|third|evening|| +85298|AAAAAAAADDNEBAAA|85298|23|41|38|PM|third|evening|| +85299|AAAAAAAAEDNEBAAA|85299|23|41|39|PM|third|evening|| +85300|AAAAAAAAFDNEBAAA|85300|23|41|40|PM|third|evening|| +85301|AAAAAAAAGDNEBAAA|85301|23|41|41|PM|third|evening|| +85302|AAAAAAAAHDNEBAAA|85302|23|41|42|PM|third|evening|| +85303|AAAAAAAAIDNEBAAA|85303|23|41|43|PM|third|evening|| +85304|AAAAAAAAJDNEBAAA|85304|23|41|44|PM|third|evening|| +85305|AAAAAAAAKDNEBAAA|85305|23|41|45|PM|third|evening|| +85306|AAAAAAAALDNEBAAA|85306|23|41|46|PM|third|evening|| +85307|AAAAAAAAMDNEBAAA|85307|23|41|47|PM|third|evening|| +85308|AAAAAAAANDNEBAAA|85308|23|41|48|PM|third|evening|| +85309|AAAAAAAAODNEBAAA|85309|23|41|49|PM|third|evening|| +85310|AAAAAAAAPDNEBAAA|85310|23|41|50|PM|third|evening|| +85311|AAAAAAAAAENEBAAA|85311|23|41|51|PM|third|evening|| +85312|AAAAAAAABENEBAAA|85312|23|41|52|PM|third|evening|| +85313|AAAAAAAACENEBAAA|85313|23|41|53|PM|third|evening|| +85314|AAAAAAAADENEBAAA|85314|23|41|54|PM|third|evening|| +85315|AAAAAAAAEENEBAAA|85315|23|41|55|PM|third|evening|| +85316|AAAAAAAAFENEBAAA|85316|23|41|56|PM|third|evening|| +85317|AAAAAAAAGENEBAAA|85317|23|41|57|PM|third|evening|| +85318|AAAAAAAAHENEBAAA|85318|23|41|58|PM|third|evening|| +85319|AAAAAAAAIENEBAAA|85319|23|41|59|PM|third|evening|| +85320|AAAAAAAAJENEBAAA|85320|23|42|0|PM|third|evening|| +85321|AAAAAAAAKENEBAAA|85321|23|42|1|PM|third|evening|| +85322|AAAAAAAALENEBAAA|85322|23|42|2|PM|third|evening|| +85323|AAAAAAAAMENEBAAA|85323|23|42|3|PM|third|evening|| +85324|AAAAAAAANENEBAAA|85324|23|42|4|PM|third|evening|| +85325|AAAAAAAAOENEBAAA|85325|23|42|5|PM|third|evening|| +85326|AAAAAAAAPENEBAAA|85326|23|42|6|PM|third|evening|| +85327|AAAAAAAAAFNEBAAA|85327|23|42|7|PM|third|evening|| +85328|AAAAAAAABFNEBAAA|85328|23|42|8|PM|third|evening|| +85329|AAAAAAAACFNEBAAA|85329|23|42|9|PM|third|evening|| +85330|AAAAAAAADFNEBAAA|85330|23|42|10|PM|third|evening|| +85331|AAAAAAAAEFNEBAAA|85331|23|42|11|PM|third|evening|| +85332|AAAAAAAAFFNEBAAA|85332|23|42|12|PM|third|evening|| +85333|AAAAAAAAGFNEBAAA|85333|23|42|13|PM|third|evening|| +85334|AAAAAAAAHFNEBAAA|85334|23|42|14|PM|third|evening|| +85335|AAAAAAAAIFNEBAAA|85335|23|42|15|PM|third|evening|| +85336|AAAAAAAAJFNEBAAA|85336|23|42|16|PM|third|evening|| +85337|AAAAAAAAKFNEBAAA|85337|23|42|17|PM|third|evening|| +85338|AAAAAAAALFNEBAAA|85338|23|42|18|PM|third|evening|| +85339|AAAAAAAAMFNEBAAA|85339|23|42|19|PM|third|evening|| +85340|AAAAAAAANFNEBAAA|85340|23|42|20|PM|third|evening|| +85341|AAAAAAAAOFNEBAAA|85341|23|42|21|PM|third|evening|| +85342|AAAAAAAAPFNEBAAA|85342|23|42|22|PM|third|evening|| +85343|AAAAAAAAAGNEBAAA|85343|23|42|23|PM|third|evening|| +85344|AAAAAAAABGNEBAAA|85344|23|42|24|PM|third|evening|| +85345|AAAAAAAACGNEBAAA|85345|23|42|25|PM|third|evening|| +85346|AAAAAAAADGNEBAAA|85346|23|42|26|PM|third|evening|| +85347|AAAAAAAAEGNEBAAA|85347|23|42|27|PM|third|evening|| +85348|AAAAAAAAFGNEBAAA|85348|23|42|28|PM|third|evening|| +85349|AAAAAAAAGGNEBAAA|85349|23|42|29|PM|third|evening|| +85350|AAAAAAAAHGNEBAAA|85350|23|42|30|PM|third|evening|| +85351|AAAAAAAAIGNEBAAA|85351|23|42|31|PM|third|evening|| +85352|AAAAAAAAJGNEBAAA|85352|23|42|32|PM|third|evening|| +85353|AAAAAAAAKGNEBAAA|85353|23|42|33|PM|third|evening|| +85354|AAAAAAAALGNEBAAA|85354|23|42|34|PM|third|evening|| +85355|AAAAAAAAMGNEBAAA|85355|23|42|35|PM|third|evening|| +85356|AAAAAAAANGNEBAAA|85356|23|42|36|PM|third|evening|| +85357|AAAAAAAAOGNEBAAA|85357|23|42|37|PM|third|evening|| +85358|AAAAAAAAPGNEBAAA|85358|23|42|38|PM|third|evening|| +85359|AAAAAAAAAHNEBAAA|85359|23|42|39|PM|third|evening|| +85360|AAAAAAAABHNEBAAA|85360|23|42|40|PM|third|evening|| +85361|AAAAAAAACHNEBAAA|85361|23|42|41|PM|third|evening|| +85362|AAAAAAAADHNEBAAA|85362|23|42|42|PM|third|evening|| +85363|AAAAAAAAEHNEBAAA|85363|23|42|43|PM|third|evening|| +85364|AAAAAAAAFHNEBAAA|85364|23|42|44|PM|third|evening|| +85365|AAAAAAAAGHNEBAAA|85365|23|42|45|PM|third|evening|| +85366|AAAAAAAAHHNEBAAA|85366|23|42|46|PM|third|evening|| +85367|AAAAAAAAIHNEBAAA|85367|23|42|47|PM|third|evening|| +85368|AAAAAAAAJHNEBAAA|85368|23|42|48|PM|third|evening|| +85369|AAAAAAAAKHNEBAAA|85369|23|42|49|PM|third|evening|| +85370|AAAAAAAALHNEBAAA|85370|23|42|50|PM|third|evening|| +85371|AAAAAAAAMHNEBAAA|85371|23|42|51|PM|third|evening|| +85372|AAAAAAAANHNEBAAA|85372|23|42|52|PM|third|evening|| +85373|AAAAAAAAOHNEBAAA|85373|23|42|53|PM|third|evening|| +85374|AAAAAAAAPHNEBAAA|85374|23|42|54|PM|third|evening|| +85375|AAAAAAAAAINEBAAA|85375|23|42|55|PM|third|evening|| +85376|AAAAAAAABINEBAAA|85376|23|42|56|PM|third|evening|| +85377|AAAAAAAACINEBAAA|85377|23|42|57|PM|third|evening|| +85378|AAAAAAAADINEBAAA|85378|23|42|58|PM|third|evening|| +85379|AAAAAAAAEINEBAAA|85379|23|42|59|PM|third|evening|| +85380|AAAAAAAAFINEBAAA|85380|23|43|0|PM|third|evening|| +85381|AAAAAAAAGINEBAAA|85381|23|43|1|PM|third|evening|| +85382|AAAAAAAAHINEBAAA|85382|23|43|2|PM|third|evening|| +85383|AAAAAAAAIINEBAAA|85383|23|43|3|PM|third|evening|| +85384|AAAAAAAAJINEBAAA|85384|23|43|4|PM|third|evening|| +85385|AAAAAAAAKINEBAAA|85385|23|43|5|PM|third|evening|| +85386|AAAAAAAALINEBAAA|85386|23|43|6|PM|third|evening|| +85387|AAAAAAAAMINEBAAA|85387|23|43|7|PM|third|evening|| +85388|AAAAAAAANINEBAAA|85388|23|43|8|PM|third|evening|| +85389|AAAAAAAAOINEBAAA|85389|23|43|9|PM|third|evening|| +85390|AAAAAAAAPINEBAAA|85390|23|43|10|PM|third|evening|| +85391|AAAAAAAAAJNEBAAA|85391|23|43|11|PM|third|evening|| +85392|AAAAAAAABJNEBAAA|85392|23|43|12|PM|third|evening|| +85393|AAAAAAAACJNEBAAA|85393|23|43|13|PM|third|evening|| +85394|AAAAAAAADJNEBAAA|85394|23|43|14|PM|third|evening|| +85395|AAAAAAAAEJNEBAAA|85395|23|43|15|PM|third|evening|| +85396|AAAAAAAAFJNEBAAA|85396|23|43|16|PM|third|evening|| +85397|AAAAAAAAGJNEBAAA|85397|23|43|17|PM|third|evening|| +85398|AAAAAAAAHJNEBAAA|85398|23|43|18|PM|third|evening|| +85399|AAAAAAAAIJNEBAAA|85399|23|43|19|PM|third|evening|| +85400|AAAAAAAAJJNEBAAA|85400|23|43|20|PM|third|evening|| +85401|AAAAAAAAKJNEBAAA|85401|23|43|21|PM|third|evening|| +85402|AAAAAAAALJNEBAAA|85402|23|43|22|PM|third|evening|| +85403|AAAAAAAAMJNEBAAA|85403|23|43|23|PM|third|evening|| +85404|AAAAAAAANJNEBAAA|85404|23|43|24|PM|third|evening|| +85405|AAAAAAAAOJNEBAAA|85405|23|43|25|PM|third|evening|| +85406|AAAAAAAAPJNEBAAA|85406|23|43|26|PM|third|evening|| +85407|AAAAAAAAAKNEBAAA|85407|23|43|27|PM|third|evening|| +85408|AAAAAAAABKNEBAAA|85408|23|43|28|PM|third|evening|| +85409|AAAAAAAACKNEBAAA|85409|23|43|29|PM|third|evening|| +85410|AAAAAAAADKNEBAAA|85410|23|43|30|PM|third|evening|| +85411|AAAAAAAAEKNEBAAA|85411|23|43|31|PM|third|evening|| +85412|AAAAAAAAFKNEBAAA|85412|23|43|32|PM|third|evening|| +85413|AAAAAAAAGKNEBAAA|85413|23|43|33|PM|third|evening|| +85414|AAAAAAAAHKNEBAAA|85414|23|43|34|PM|third|evening|| +85415|AAAAAAAAIKNEBAAA|85415|23|43|35|PM|third|evening|| +85416|AAAAAAAAJKNEBAAA|85416|23|43|36|PM|third|evening|| +85417|AAAAAAAAKKNEBAAA|85417|23|43|37|PM|third|evening|| +85418|AAAAAAAALKNEBAAA|85418|23|43|38|PM|third|evening|| +85419|AAAAAAAAMKNEBAAA|85419|23|43|39|PM|third|evening|| +85420|AAAAAAAANKNEBAAA|85420|23|43|40|PM|third|evening|| +85421|AAAAAAAAOKNEBAAA|85421|23|43|41|PM|third|evening|| +85422|AAAAAAAAPKNEBAAA|85422|23|43|42|PM|third|evening|| +85423|AAAAAAAAALNEBAAA|85423|23|43|43|PM|third|evening|| +85424|AAAAAAAABLNEBAAA|85424|23|43|44|PM|third|evening|| +85425|AAAAAAAACLNEBAAA|85425|23|43|45|PM|third|evening|| +85426|AAAAAAAADLNEBAAA|85426|23|43|46|PM|third|evening|| +85427|AAAAAAAAELNEBAAA|85427|23|43|47|PM|third|evening|| +85428|AAAAAAAAFLNEBAAA|85428|23|43|48|PM|third|evening|| +85429|AAAAAAAAGLNEBAAA|85429|23|43|49|PM|third|evening|| +85430|AAAAAAAAHLNEBAAA|85430|23|43|50|PM|third|evening|| +85431|AAAAAAAAILNEBAAA|85431|23|43|51|PM|third|evening|| +85432|AAAAAAAAJLNEBAAA|85432|23|43|52|PM|third|evening|| +85433|AAAAAAAAKLNEBAAA|85433|23|43|53|PM|third|evening|| +85434|AAAAAAAALLNEBAAA|85434|23|43|54|PM|third|evening|| +85435|AAAAAAAAMLNEBAAA|85435|23|43|55|PM|third|evening|| +85436|AAAAAAAANLNEBAAA|85436|23|43|56|PM|third|evening|| +85437|AAAAAAAAOLNEBAAA|85437|23|43|57|PM|third|evening|| +85438|AAAAAAAAPLNEBAAA|85438|23|43|58|PM|third|evening|| +85439|AAAAAAAAAMNEBAAA|85439|23|43|59|PM|third|evening|| +85440|AAAAAAAABMNEBAAA|85440|23|44|0|PM|third|evening|| +85441|AAAAAAAACMNEBAAA|85441|23|44|1|PM|third|evening|| +85442|AAAAAAAADMNEBAAA|85442|23|44|2|PM|third|evening|| +85443|AAAAAAAAEMNEBAAA|85443|23|44|3|PM|third|evening|| +85444|AAAAAAAAFMNEBAAA|85444|23|44|4|PM|third|evening|| +85445|AAAAAAAAGMNEBAAA|85445|23|44|5|PM|third|evening|| +85446|AAAAAAAAHMNEBAAA|85446|23|44|6|PM|third|evening|| +85447|AAAAAAAAIMNEBAAA|85447|23|44|7|PM|third|evening|| +85448|AAAAAAAAJMNEBAAA|85448|23|44|8|PM|third|evening|| +85449|AAAAAAAAKMNEBAAA|85449|23|44|9|PM|third|evening|| +85450|AAAAAAAALMNEBAAA|85450|23|44|10|PM|third|evening|| +85451|AAAAAAAAMMNEBAAA|85451|23|44|11|PM|third|evening|| +85452|AAAAAAAANMNEBAAA|85452|23|44|12|PM|third|evening|| +85453|AAAAAAAAOMNEBAAA|85453|23|44|13|PM|third|evening|| +85454|AAAAAAAAPMNEBAAA|85454|23|44|14|PM|third|evening|| +85455|AAAAAAAAANNEBAAA|85455|23|44|15|PM|third|evening|| +85456|AAAAAAAABNNEBAAA|85456|23|44|16|PM|third|evening|| +85457|AAAAAAAACNNEBAAA|85457|23|44|17|PM|third|evening|| +85458|AAAAAAAADNNEBAAA|85458|23|44|18|PM|third|evening|| +85459|AAAAAAAAENNEBAAA|85459|23|44|19|PM|third|evening|| +85460|AAAAAAAAFNNEBAAA|85460|23|44|20|PM|third|evening|| +85461|AAAAAAAAGNNEBAAA|85461|23|44|21|PM|third|evening|| +85462|AAAAAAAAHNNEBAAA|85462|23|44|22|PM|third|evening|| +85463|AAAAAAAAINNEBAAA|85463|23|44|23|PM|third|evening|| +85464|AAAAAAAAJNNEBAAA|85464|23|44|24|PM|third|evening|| +85465|AAAAAAAAKNNEBAAA|85465|23|44|25|PM|third|evening|| +85466|AAAAAAAALNNEBAAA|85466|23|44|26|PM|third|evening|| +85467|AAAAAAAAMNNEBAAA|85467|23|44|27|PM|third|evening|| +85468|AAAAAAAANNNEBAAA|85468|23|44|28|PM|third|evening|| +85469|AAAAAAAAONNEBAAA|85469|23|44|29|PM|third|evening|| +85470|AAAAAAAAPNNEBAAA|85470|23|44|30|PM|third|evening|| +85471|AAAAAAAAAONEBAAA|85471|23|44|31|PM|third|evening|| +85472|AAAAAAAABONEBAAA|85472|23|44|32|PM|third|evening|| +85473|AAAAAAAACONEBAAA|85473|23|44|33|PM|third|evening|| +85474|AAAAAAAADONEBAAA|85474|23|44|34|PM|third|evening|| +85475|AAAAAAAAEONEBAAA|85475|23|44|35|PM|third|evening|| +85476|AAAAAAAAFONEBAAA|85476|23|44|36|PM|third|evening|| +85477|AAAAAAAAGONEBAAA|85477|23|44|37|PM|third|evening|| +85478|AAAAAAAAHONEBAAA|85478|23|44|38|PM|third|evening|| +85479|AAAAAAAAIONEBAAA|85479|23|44|39|PM|third|evening|| +85480|AAAAAAAAJONEBAAA|85480|23|44|40|PM|third|evening|| +85481|AAAAAAAAKONEBAAA|85481|23|44|41|PM|third|evening|| +85482|AAAAAAAALONEBAAA|85482|23|44|42|PM|third|evening|| +85483|AAAAAAAAMONEBAAA|85483|23|44|43|PM|third|evening|| +85484|AAAAAAAANONEBAAA|85484|23|44|44|PM|third|evening|| +85485|AAAAAAAAOONEBAAA|85485|23|44|45|PM|third|evening|| +85486|AAAAAAAAPONEBAAA|85486|23|44|46|PM|third|evening|| +85487|AAAAAAAAAPNEBAAA|85487|23|44|47|PM|third|evening|| +85488|AAAAAAAABPNEBAAA|85488|23|44|48|PM|third|evening|| +85489|AAAAAAAACPNEBAAA|85489|23|44|49|PM|third|evening|| +85490|AAAAAAAADPNEBAAA|85490|23|44|50|PM|third|evening|| +85491|AAAAAAAAEPNEBAAA|85491|23|44|51|PM|third|evening|| +85492|AAAAAAAAFPNEBAAA|85492|23|44|52|PM|third|evening|| +85493|AAAAAAAAGPNEBAAA|85493|23|44|53|PM|third|evening|| +85494|AAAAAAAAHPNEBAAA|85494|23|44|54|PM|third|evening|| +85495|AAAAAAAAIPNEBAAA|85495|23|44|55|PM|third|evening|| +85496|AAAAAAAAJPNEBAAA|85496|23|44|56|PM|third|evening|| +85497|AAAAAAAAKPNEBAAA|85497|23|44|57|PM|third|evening|| +85498|AAAAAAAALPNEBAAA|85498|23|44|58|PM|third|evening|| +85499|AAAAAAAAMPNEBAAA|85499|23|44|59|PM|third|evening|| +85500|AAAAAAAANPNEBAAA|85500|23|45|0|PM|third|evening|| +85501|AAAAAAAAOPNEBAAA|85501|23|45|1|PM|third|evening|| +85502|AAAAAAAAPPNEBAAA|85502|23|45|2|PM|third|evening|| +85503|AAAAAAAAAAOEBAAA|85503|23|45|3|PM|third|evening|| +85504|AAAAAAAABAOEBAAA|85504|23|45|4|PM|third|evening|| +85505|AAAAAAAACAOEBAAA|85505|23|45|5|PM|third|evening|| +85506|AAAAAAAADAOEBAAA|85506|23|45|6|PM|third|evening|| +85507|AAAAAAAAEAOEBAAA|85507|23|45|7|PM|third|evening|| +85508|AAAAAAAAFAOEBAAA|85508|23|45|8|PM|third|evening|| +85509|AAAAAAAAGAOEBAAA|85509|23|45|9|PM|third|evening|| +85510|AAAAAAAAHAOEBAAA|85510|23|45|10|PM|third|evening|| +85511|AAAAAAAAIAOEBAAA|85511|23|45|11|PM|third|evening|| +85512|AAAAAAAAJAOEBAAA|85512|23|45|12|PM|third|evening|| +85513|AAAAAAAAKAOEBAAA|85513|23|45|13|PM|third|evening|| +85514|AAAAAAAALAOEBAAA|85514|23|45|14|PM|third|evening|| +85515|AAAAAAAAMAOEBAAA|85515|23|45|15|PM|third|evening|| +85516|AAAAAAAANAOEBAAA|85516|23|45|16|PM|third|evening|| +85517|AAAAAAAAOAOEBAAA|85517|23|45|17|PM|third|evening|| +85518|AAAAAAAAPAOEBAAA|85518|23|45|18|PM|third|evening|| +85519|AAAAAAAAABOEBAAA|85519|23|45|19|PM|third|evening|| +85520|AAAAAAAABBOEBAAA|85520|23|45|20|PM|third|evening|| +85521|AAAAAAAACBOEBAAA|85521|23|45|21|PM|third|evening|| +85522|AAAAAAAADBOEBAAA|85522|23|45|22|PM|third|evening|| +85523|AAAAAAAAEBOEBAAA|85523|23|45|23|PM|third|evening|| +85524|AAAAAAAAFBOEBAAA|85524|23|45|24|PM|third|evening|| +85525|AAAAAAAAGBOEBAAA|85525|23|45|25|PM|third|evening|| +85526|AAAAAAAAHBOEBAAA|85526|23|45|26|PM|third|evening|| +85527|AAAAAAAAIBOEBAAA|85527|23|45|27|PM|third|evening|| +85528|AAAAAAAAJBOEBAAA|85528|23|45|28|PM|third|evening|| +85529|AAAAAAAAKBOEBAAA|85529|23|45|29|PM|third|evening|| +85530|AAAAAAAALBOEBAAA|85530|23|45|30|PM|third|evening|| +85531|AAAAAAAAMBOEBAAA|85531|23|45|31|PM|third|evening|| +85532|AAAAAAAANBOEBAAA|85532|23|45|32|PM|third|evening|| +85533|AAAAAAAAOBOEBAAA|85533|23|45|33|PM|third|evening|| +85534|AAAAAAAAPBOEBAAA|85534|23|45|34|PM|third|evening|| +85535|AAAAAAAAACOEBAAA|85535|23|45|35|PM|third|evening|| +85536|AAAAAAAABCOEBAAA|85536|23|45|36|PM|third|evening|| +85537|AAAAAAAACCOEBAAA|85537|23|45|37|PM|third|evening|| +85538|AAAAAAAADCOEBAAA|85538|23|45|38|PM|third|evening|| +85539|AAAAAAAAECOEBAAA|85539|23|45|39|PM|third|evening|| +85540|AAAAAAAAFCOEBAAA|85540|23|45|40|PM|third|evening|| +85541|AAAAAAAAGCOEBAAA|85541|23|45|41|PM|third|evening|| +85542|AAAAAAAAHCOEBAAA|85542|23|45|42|PM|third|evening|| +85543|AAAAAAAAICOEBAAA|85543|23|45|43|PM|third|evening|| +85544|AAAAAAAAJCOEBAAA|85544|23|45|44|PM|third|evening|| +85545|AAAAAAAAKCOEBAAA|85545|23|45|45|PM|third|evening|| +85546|AAAAAAAALCOEBAAA|85546|23|45|46|PM|third|evening|| +85547|AAAAAAAAMCOEBAAA|85547|23|45|47|PM|third|evening|| +85548|AAAAAAAANCOEBAAA|85548|23|45|48|PM|third|evening|| +85549|AAAAAAAAOCOEBAAA|85549|23|45|49|PM|third|evening|| +85550|AAAAAAAAPCOEBAAA|85550|23|45|50|PM|third|evening|| +85551|AAAAAAAAADOEBAAA|85551|23|45|51|PM|third|evening|| +85552|AAAAAAAABDOEBAAA|85552|23|45|52|PM|third|evening|| +85553|AAAAAAAACDOEBAAA|85553|23|45|53|PM|third|evening|| +85554|AAAAAAAADDOEBAAA|85554|23|45|54|PM|third|evening|| +85555|AAAAAAAAEDOEBAAA|85555|23|45|55|PM|third|evening|| +85556|AAAAAAAAFDOEBAAA|85556|23|45|56|PM|third|evening|| +85557|AAAAAAAAGDOEBAAA|85557|23|45|57|PM|third|evening|| +85558|AAAAAAAAHDOEBAAA|85558|23|45|58|PM|third|evening|| +85559|AAAAAAAAIDOEBAAA|85559|23|45|59|PM|third|evening|| +85560|AAAAAAAAJDOEBAAA|85560|23|46|0|PM|third|evening|| +85561|AAAAAAAAKDOEBAAA|85561|23|46|1|PM|third|evening|| +85562|AAAAAAAALDOEBAAA|85562|23|46|2|PM|third|evening|| +85563|AAAAAAAAMDOEBAAA|85563|23|46|3|PM|third|evening|| +85564|AAAAAAAANDOEBAAA|85564|23|46|4|PM|third|evening|| +85565|AAAAAAAAODOEBAAA|85565|23|46|5|PM|third|evening|| +85566|AAAAAAAAPDOEBAAA|85566|23|46|6|PM|third|evening|| +85567|AAAAAAAAAEOEBAAA|85567|23|46|7|PM|third|evening|| +85568|AAAAAAAABEOEBAAA|85568|23|46|8|PM|third|evening|| +85569|AAAAAAAACEOEBAAA|85569|23|46|9|PM|third|evening|| +85570|AAAAAAAADEOEBAAA|85570|23|46|10|PM|third|evening|| +85571|AAAAAAAAEEOEBAAA|85571|23|46|11|PM|third|evening|| +85572|AAAAAAAAFEOEBAAA|85572|23|46|12|PM|third|evening|| +85573|AAAAAAAAGEOEBAAA|85573|23|46|13|PM|third|evening|| +85574|AAAAAAAAHEOEBAAA|85574|23|46|14|PM|third|evening|| +85575|AAAAAAAAIEOEBAAA|85575|23|46|15|PM|third|evening|| +85576|AAAAAAAAJEOEBAAA|85576|23|46|16|PM|third|evening|| +85577|AAAAAAAAKEOEBAAA|85577|23|46|17|PM|third|evening|| +85578|AAAAAAAALEOEBAAA|85578|23|46|18|PM|third|evening|| +85579|AAAAAAAAMEOEBAAA|85579|23|46|19|PM|third|evening|| +85580|AAAAAAAANEOEBAAA|85580|23|46|20|PM|third|evening|| +85581|AAAAAAAAOEOEBAAA|85581|23|46|21|PM|third|evening|| +85582|AAAAAAAAPEOEBAAA|85582|23|46|22|PM|third|evening|| +85583|AAAAAAAAAFOEBAAA|85583|23|46|23|PM|third|evening|| +85584|AAAAAAAABFOEBAAA|85584|23|46|24|PM|third|evening|| +85585|AAAAAAAACFOEBAAA|85585|23|46|25|PM|third|evening|| +85586|AAAAAAAADFOEBAAA|85586|23|46|26|PM|third|evening|| +85587|AAAAAAAAEFOEBAAA|85587|23|46|27|PM|third|evening|| +85588|AAAAAAAAFFOEBAAA|85588|23|46|28|PM|third|evening|| +85589|AAAAAAAAGFOEBAAA|85589|23|46|29|PM|third|evening|| +85590|AAAAAAAAHFOEBAAA|85590|23|46|30|PM|third|evening|| +85591|AAAAAAAAIFOEBAAA|85591|23|46|31|PM|third|evening|| +85592|AAAAAAAAJFOEBAAA|85592|23|46|32|PM|third|evening|| +85593|AAAAAAAAKFOEBAAA|85593|23|46|33|PM|third|evening|| +85594|AAAAAAAALFOEBAAA|85594|23|46|34|PM|third|evening|| +85595|AAAAAAAAMFOEBAAA|85595|23|46|35|PM|third|evening|| +85596|AAAAAAAANFOEBAAA|85596|23|46|36|PM|third|evening|| +85597|AAAAAAAAOFOEBAAA|85597|23|46|37|PM|third|evening|| +85598|AAAAAAAAPFOEBAAA|85598|23|46|38|PM|third|evening|| +85599|AAAAAAAAAGOEBAAA|85599|23|46|39|PM|third|evening|| +85600|AAAAAAAABGOEBAAA|85600|23|46|40|PM|third|evening|| +85601|AAAAAAAACGOEBAAA|85601|23|46|41|PM|third|evening|| +85602|AAAAAAAADGOEBAAA|85602|23|46|42|PM|third|evening|| +85603|AAAAAAAAEGOEBAAA|85603|23|46|43|PM|third|evening|| +85604|AAAAAAAAFGOEBAAA|85604|23|46|44|PM|third|evening|| +85605|AAAAAAAAGGOEBAAA|85605|23|46|45|PM|third|evening|| +85606|AAAAAAAAHGOEBAAA|85606|23|46|46|PM|third|evening|| +85607|AAAAAAAAIGOEBAAA|85607|23|46|47|PM|third|evening|| +85608|AAAAAAAAJGOEBAAA|85608|23|46|48|PM|third|evening|| +85609|AAAAAAAAKGOEBAAA|85609|23|46|49|PM|third|evening|| +85610|AAAAAAAALGOEBAAA|85610|23|46|50|PM|third|evening|| +85611|AAAAAAAAMGOEBAAA|85611|23|46|51|PM|third|evening|| +85612|AAAAAAAANGOEBAAA|85612|23|46|52|PM|third|evening|| +85613|AAAAAAAAOGOEBAAA|85613|23|46|53|PM|third|evening|| +85614|AAAAAAAAPGOEBAAA|85614|23|46|54|PM|third|evening|| +85615|AAAAAAAAAHOEBAAA|85615|23|46|55|PM|third|evening|| +85616|AAAAAAAABHOEBAAA|85616|23|46|56|PM|third|evening|| +85617|AAAAAAAACHOEBAAA|85617|23|46|57|PM|third|evening|| +85618|AAAAAAAADHOEBAAA|85618|23|46|58|PM|third|evening|| +85619|AAAAAAAAEHOEBAAA|85619|23|46|59|PM|third|evening|| +85620|AAAAAAAAFHOEBAAA|85620|23|47|0|PM|third|evening|| +85621|AAAAAAAAGHOEBAAA|85621|23|47|1|PM|third|evening|| +85622|AAAAAAAAHHOEBAAA|85622|23|47|2|PM|third|evening|| +85623|AAAAAAAAIHOEBAAA|85623|23|47|3|PM|third|evening|| +85624|AAAAAAAAJHOEBAAA|85624|23|47|4|PM|third|evening|| +85625|AAAAAAAAKHOEBAAA|85625|23|47|5|PM|third|evening|| +85626|AAAAAAAALHOEBAAA|85626|23|47|6|PM|third|evening|| +85627|AAAAAAAAMHOEBAAA|85627|23|47|7|PM|third|evening|| +85628|AAAAAAAANHOEBAAA|85628|23|47|8|PM|third|evening|| +85629|AAAAAAAAOHOEBAAA|85629|23|47|9|PM|third|evening|| +85630|AAAAAAAAPHOEBAAA|85630|23|47|10|PM|third|evening|| +85631|AAAAAAAAAIOEBAAA|85631|23|47|11|PM|third|evening|| +85632|AAAAAAAABIOEBAAA|85632|23|47|12|PM|third|evening|| +85633|AAAAAAAACIOEBAAA|85633|23|47|13|PM|third|evening|| +85634|AAAAAAAADIOEBAAA|85634|23|47|14|PM|third|evening|| +85635|AAAAAAAAEIOEBAAA|85635|23|47|15|PM|third|evening|| +85636|AAAAAAAAFIOEBAAA|85636|23|47|16|PM|third|evening|| +85637|AAAAAAAAGIOEBAAA|85637|23|47|17|PM|third|evening|| +85638|AAAAAAAAHIOEBAAA|85638|23|47|18|PM|third|evening|| +85639|AAAAAAAAIIOEBAAA|85639|23|47|19|PM|third|evening|| +85640|AAAAAAAAJIOEBAAA|85640|23|47|20|PM|third|evening|| +85641|AAAAAAAAKIOEBAAA|85641|23|47|21|PM|third|evening|| +85642|AAAAAAAALIOEBAAA|85642|23|47|22|PM|third|evening|| +85643|AAAAAAAAMIOEBAAA|85643|23|47|23|PM|third|evening|| +85644|AAAAAAAANIOEBAAA|85644|23|47|24|PM|third|evening|| +85645|AAAAAAAAOIOEBAAA|85645|23|47|25|PM|third|evening|| +85646|AAAAAAAAPIOEBAAA|85646|23|47|26|PM|third|evening|| +85647|AAAAAAAAAJOEBAAA|85647|23|47|27|PM|third|evening|| +85648|AAAAAAAABJOEBAAA|85648|23|47|28|PM|third|evening|| +85649|AAAAAAAACJOEBAAA|85649|23|47|29|PM|third|evening|| +85650|AAAAAAAADJOEBAAA|85650|23|47|30|PM|third|evening|| +85651|AAAAAAAAEJOEBAAA|85651|23|47|31|PM|third|evening|| +85652|AAAAAAAAFJOEBAAA|85652|23|47|32|PM|third|evening|| +85653|AAAAAAAAGJOEBAAA|85653|23|47|33|PM|third|evening|| +85654|AAAAAAAAHJOEBAAA|85654|23|47|34|PM|third|evening|| +85655|AAAAAAAAIJOEBAAA|85655|23|47|35|PM|third|evening|| +85656|AAAAAAAAJJOEBAAA|85656|23|47|36|PM|third|evening|| +85657|AAAAAAAAKJOEBAAA|85657|23|47|37|PM|third|evening|| +85658|AAAAAAAALJOEBAAA|85658|23|47|38|PM|third|evening|| +85659|AAAAAAAAMJOEBAAA|85659|23|47|39|PM|third|evening|| +85660|AAAAAAAANJOEBAAA|85660|23|47|40|PM|third|evening|| +85661|AAAAAAAAOJOEBAAA|85661|23|47|41|PM|third|evening|| +85662|AAAAAAAAPJOEBAAA|85662|23|47|42|PM|third|evening|| +85663|AAAAAAAAAKOEBAAA|85663|23|47|43|PM|third|evening|| +85664|AAAAAAAABKOEBAAA|85664|23|47|44|PM|third|evening|| +85665|AAAAAAAACKOEBAAA|85665|23|47|45|PM|third|evening|| +85666|AAAAAAAADKOEBAAA|85666|23|47|46|PM|third|evening|| +85667|AAAAAAAAEKOEBAAA|85667|23|47|47|PM|third|evening|| +85668|AAAAAAAAFKOEBAAA|85668|23|47|48|PM|third|evening|| +85669|AAAAAAAAGKOEBAAA|85669|23|47|49|PM|third|evening|| +85670|AAAAAAAAHKOEBAAA|85670|23|47|50|PM|third|evening|| +85671|AAAAAAAAIKOEBAAA|85671|23|47|51|PM|third|evening|| +85672|AAAAAAAAJKOEBAAA|85672|23|47|52|PM|third|evening|| +85673|AAAAAAAAKKOEBAAA|85673|23|47|53|PM|third|evening|| +85674|AAAAAAAALKOEBAAA|85674|23|47|54|PM|third|evening|| +85675|AAAAAAAAMKOEBAAA|85675|23|47|55|PM|third|evening|| +85676|AAAAAAAANKOEBAAA|85676|23|47|56|PM|third|evening|| +85677|AAAAAAAAOKOEBAAA|85677|23|47|57|PM|third|evening|| +85678|AAAAAAAAPKOEBAAA|85678|23|47|58|PM|third|evening|| +85679|AAAAAAAAALOEBAAA|85679|23|47|59|PM|third|evening|| +85680|AAAAAAAABLOEBAAA|85680|23|48|0|PM|third|evening|| +85681|AAAAAAAACLOEBAAA|85681|23|48|1|PM|third|evening|| +85682|AAAAAAAADLOEBAAA|85682|23|48|2|PM|third|evening|| +85683|AAAAAAAAELOEBAAA|85683|23|48|3|PM|third|evening|| +85684|AAAAAAAAFLOEBAAA|85684|23|48|4|PM|third|evening|| +85685|AAAAAAAAGLOEBAAA|85685|23|48|5|PM|third|evening|| +85686|AAAAAAAAHLOEBAAA|85686|23|48|6|PM|third|evening|| +85687|AAAAAAAAILOEBAAA|85687|23|48|7|PM|third|evening|| +85688|AAAAAAAAJLOEBAAA|85688|23|48|8|PM|third|evening|| +85689|AAAAAAAAKLOEBAAA|85689|23|48|9|PM|third|evening|| +85690|AAAAAAAALLOEBAAA|85690|23|48|10|PM|third|evening|| +85691|AAAAAAAAMLOEBAAA|85691|23|48|11|PM|third|evening|| +85692|AAAAAAAANLOEBAAA|85692|23|48|12|PM|third|evening|| +85693|AAAAAAAAOLOEBAAA|85693|23|48|13|PM|third|evening|| +85694|AAAAAAAAPLOEBAAA|85694|23|48|14|PM|third|evening|| +85695|AAAAAAAAAMOEBAAA|85695|23|48|15|PM|third|evening|| +85696|AAAAAAAABMOEBAAA|85696|23|48|16|PM|third|evening|| +85697|AAAAAAAACMOEBAAA|85697|23|48|17|PM|third|evening|| +85698|AAAAAAAADMOEBAAA|85698|23|48|18|PM|third|evening|| +85699|AAAAAAAAEMOEBAAA|85699|23|48|19|PM|third|evening|| +85700|AAAAAAAAFMOEBAAA|85700|23|48|20|PM|third|evening|| +85701|AAAAAAAAGMOEBAAA|85701|23|48|21|PM|third|evening|| +85702|AAAAAAAAHMOEBAAA|85702|23|48|22|PM|third|evening|| +85703|AAAAAAAAIMOEBAAA|85703|23|48|23|PM|third|evening|| +85704|AAAAAAAAJMOEBAAA|85704|23|48|24|PM|third|evening|| +85705|AAAAAAAAKMOEBAAA|85705|23|48|25|PM|third|evening|| +85706|AAAAAAAALMOEBAAA|85706|23|48|26|PM|third|evening|| +85707|AAAAAAAAMMOEBAAA|85707|23|48|27|PM|third|evening|| +85708|AAAAAAAANMOEBAAA|85708|23|48|28|PM|third|evening|| +85709|AAAAAAAAOMOEBAAA|85709|23|48|29|PM|third|evening|| +85710|AAAAAAAAPMOEBAAA|85710|23|48|30|PM|third|evening|| +85711|AAAAAAAAANOEBAAA|85711|23|48|31|PM|third|evening|| +85712|AAAAAAAABNOEBAAA|85712|23|48|32|PM|third|evening|| +85713|AAAAAAAACNOEBAAA|85713|23|48|33|PM|third|evening|| +85714|AAAAAAAADNOEBAAA|85714|23|48|34|PM|third|evening|| +85715|AAAAAAAAENOEBAAA|85715|23|48|35|PM|third|evening|| +85716|AAAAAAAAFNOEBAAA|85716|23|48|36|PM|third|evening|| +85717|AAAAAAAAGNOEBAAA|85717|23|48|37|PM|third|evening|| +85718|AAAAAAAAHNOEBAAA|85718|23|48|38|PM|third|evening|| +85719|AAAAAAAAINOEBAAA|85719|23|48|39|PM|third|evening|| +85720|AAAAAAAAJNOEBAAA|85720|23|48|40|PM|third|evening|| +85721|AAAAAAAAKNOEBAAA|85721|23|48|41|PM|third|evening|| +85722|AAAAAAAALNOEBAAA|85722|23|48|42|PM|third|evening|| +85723|AAAAAAAAMNOEBAAA|85723|23|48|43|PM|third|evening|| +85724|AAAAAAAANNOEBAAA|85724|23|48|44|PM|third|evening|| +85725|AAAAAAAAONOEBAAA|85725|23|48|45|PM|third|evening|| +85726|AAAAAAAAPNOEBAAA|85726|23|48|46|PM|third|evening|| +85727|AAAAAAAAAOOEBAAA|85727|23|48|47|PM|third|evening|| +85728|AAAAAAAABOOEBAAA|85728|23|48|48|PM|third|evening|| +85729|AAAAAAAACOOEBAAA|85729|23|48|49|PM|third|evening|| +85730|AAAAAAAADOOEBAAA|85730|23|48|50|PM|third|evening|| +85731|AAAAAAAAEOOEBAAA|85731|23|48|51|PM|third|evening|| +85732|AAAAAAAAFOOEBAAA|85732|23|48|52|PM|third|evening|| +85733|AAAAAAAAGOOEBAAA|85733|23|48|53|PM|third|evening|| +85734|AAAAAAAAHOOEBAAA|85734|23|48|54|PM|third|evening|| +85735|AAAAAAAAIOOEBAAA|85735|23|48|55|PM|third|evening|| +85736|AAAAAAAAJOOEBAAA|85736|23|48|56|PM|third|evening|| +85737|AAAAAAAAKOOEBAAA|85737|23|48|57|PM|third|evening|| +85738|AAAAAAAALOOEBAAA|85738|23|48|58|PM|third|evening|| +85739|AAAAAAAAMOOEBAAA|85739|23|48|59|PM|third|evening|| +85740|AAAAAAAANOOEBAAA|85740|23|49|0|PM|third|evening|| +85741|AAAAAAAAOOOEBAAA|85741|23|49|1|PM|third|evening|| +85742|AAAAAAAAPOOEBAAA|85742|23|49|2|PM|third|evening|| +85743|AAAAAAAAAPOEBAAA|85743|23|49|3|PM|third|evening|| +85744|AAAAAAAABPOEBAAA|85744|23|49|4|PM|third|evening|| +85745|AAAAAAAACPOEBAAA|85745|23|49|5|PM|third|evening|| +85746|AAAAAAAADPOEBAAA|85746|23|49|6|PM|third|evening|| +85747|AAAAAAAAEPOEBAAA|85747|23|49|7|PM|third|evening|| +85748|AAAAAAAAFPOEBAAA|85748|23|49|8|PM|third|evening|| +85749|AAAAAAAAGPOEBAAA|85749|23|49|9|PM|third|evening|| +85750|AAAAAAAAHPOEBAAA|85750|23|49|10|PM|third|evening|| +85751|AAAAAAAAIPOEBAAA|85751|23|49|11|PM|third|evening|| +85752|AAAAAAAAJPOEBAAA|85752|23|49|12|PM|third|evening|| +85753|AAAAAAAAKPOEBAAA|85753|23|49|13|PM|third|evening|| +85754|AAAAAAAALPOEBAAA|85754|23|49|14|PM|third|evening|| +85755|AAAAAAAAMPOEBAAA|85755|23|49|15|PM|third|evening|| +85756|AAAAAAAANPOEBAAA|85756|23|49|16|PM|third|evening|| +85757|AAAAAAAAOPOEBAAA|85757|23|49|17|PM|third|evening|| +85758|AAAAAAAAPPOEBAAA|85758|23|49|18|PM|third|evening|| +85759|AAAAAAAAAAPEBAAA|85759|23|49|19|PM|third|evening|| +85760|AAAAAAAABAPEBAAA|85760|23|49|20|PM|third|evening|| +85761|AAAAAAAACAPEBAAA|85761|23|49|21|PM|third|evening|| +85762|AAAAAAAADAPEBAAA|85762|23|49|22|PM|third|evening|| +85763|AAAAAAAAEAPEBAAA|85763|23|49|23|PM|third|evening|| +85764|AAAAAAAAFAPEBAAA|85764|23|49|24|PM|third|evening|| +85765|AAAAAAAAGAPEBAAA|85765|23|49|25|PM|third|evening|| +85766|AAAAAAAAHAPEBAAA|85766|23|49|26|PM|third|evening|| +85767|AAAAAAAAIAPEBAAA|85767|23|49|27|PM|third|evening|| +85768|AAAAAAAAJAPEBAAA|85768|23|49|28|PM|third|evening|| +85769|AAAAAAAAKAPEBAAA|85769|23|49|29|PM|third|evening|| +85770|AAAAAAAALAPEBAAA|85770|23|49|30|PM|third|evening|| +85771|AAAAAAAAMAPEBAAA|85771|23|49|31|PM|third|evening|| +85772|AAAAAAAANAPEBAAA|85772|23|49|32|PM|third|evening|| +85773|AAAAAAAAOAPEBAAA|85773|23|49|33|PM|third|evening|| +85774|AAAAAAAAPAPEBAAA|85774|23|49|34|PM|third|evening|| +85775|AAAAAAAAABPEBAAA|85775|23|49|35|PM|third|evening|| +85776|AAAAAAAABBPEBAAA|85776|23|49|36|PM|third|evening|| +85777|AAAAAAAACBPEBAAA|85777|23|49|37|PM|third|evening|| +85778|AAAAAAAADBPEBAAA|85778|23|49|38|PM|third|evening|| +85779|AAAAAAAAEBPEBAAA|85779|23|49|39|PM|third|evening|| +85780|AAAAAAAAFBPEBAAA|85780|23|49|40|PM|third|evening|| +85781|AAAAAAAAGBPEBAAA|85781|23|49|41|PM|third|evening|| +85782|AAAAAAAAHBPEBAAA|85782|23|49|42|PM|third|evening|| +85783|AAAAAAAAIBPEBAAA|85783|23|49|43|PM|third|evening|| +85784|AAAAAAAAJBPEBAAA|85784|23|49|44|PM|third|evening|| +85785|AAAAAAAAKBPEBAAA|85785|23|49|45|PM|third|evening|| +85786|AAAAAAAALBPEBAAA|85786|23|49|46|PM|third|evening|| +85787|AAAAAAAAMBPEBAAA|85787|23|49|47|PM|third|evening|| +85788|AAAAAAAANBPEBAAA|85788|23|49|48|PM|third|evening|| +85789|AAAAAAAAOBPEBAAA|85789|23|49|49|PM|third|evening|| +85790|AAAAAAAAPBPEBAAA|85790|23|49|50|PM|third|evening|| +85791|AAAAAAAAACPEBAAA|85791|23|49|51|PM|third|evening|| +85792|AAAAAAAABCPEBAAA|85792|23|49|52|PM|third|evening|| +85793|AAAAAAAACCPEBAAA|85793|23|49|53|PM|third|evening|| +85794|AAAAAAAADCPEBAAA|85794|23|49|54|PM|third|evening|| +85795|AAAAAAAAECPEBAAA|85795|23|49|55|PM|third|evening|| +85796|AAAAAAAAFCPEBAAA|85796|23|49|56|PM|third|evening|| +85797|AAAAAAAAGCPEBAAA|85797|23|49|57|PM|third|evening|| +85798|AAAAAAAAHCPEBAAA|85798|23|49|58|PM|third|evening|| +85799|AAAAAAAAICPEBAAA|85799|23|49|59|PM|third|evening|| +85800|AAAAAAAAJCPEBAAA|85800|23|50|0|PM|third|evening|| +85801|AAAAAAAAKCPEBAAA|85801|23|50|1|PM|third|evening|| +85802|AAAAAAAALCPEBAAA|85802|23|50|2|PM|third|evening|| +85803|AAAAAAAAMCPEBAAA|85803|23|50|3|PM|third|evening|| +85804|AAAAAAAANCPEBAAA|85804|23|50|4|PM|third|evening|| +85805|AAAAAAAAOCPEBAAA|85805|23|50|5|PM|third|evening|| +85806|AAAAAAAAPCPEBAAA|85806|23|50|6|PM|third|evening|| +85807|AAAAAAAAADPEBAAA|85807|23|50|7|PM|third|evening|| +85808|AAAAAAAABDPEBAAA|85808|23|50|8|PM|third|evening|| +85809|AAAAAAAACDPEBAAA|85809|23|50|9|PM|third|evening|| +85810|AAAAAAAADDPEBAAA|85810|23|50|10|PM|third|evening|| +85811|AAAAAAAAEDPEBAAA|85811|23|50|11|PM|third|evening|| +85812|AAAAAAAAFDPEBAAA|85812|23|50|12|PM|third|evening|| +85813|AAAAAAAAGDPEBAAA|85813|23|50|13|PM|third|evening|| +85814|AAAAAAAAHDPEBAAA|85814|23|50|14|PM|third|evening|| +85815|AAAAAAAAIDPEBAAA|85815|23|50|15|PM|third|evening|| +85816|AAAAAAAAJDPEBAAA|85816|23|50|16|PM|third|evening|| +85817|AAAAAAAAKDPEBAAA|85817|23|50|17|PM|third|evening|| +85818|AAAAAAAALDPEBAAA|85818|23|50|18|PM|third|evening|| +85819|AAAAAAAAMDPEBAAA|85819|23|50|19|PM|third|evening|| +85820|AAAAAAAANDPEBAAA|85820|23|50|20|PM|third|evening|| +85821|AAAAAAAAODPEBAAA|85821|23|50|21|PM|third|evening|| +85822|AAAAAAAAPDPEBAAA|85822|23|50|22|PM|third|evening|| +85823|AAAAAAAAAEPEBAAA|85823|23|50|23|PM|third|evening|| +85824|AAAAAAAABEPEBAAA|85824|23|50|24|PM|third|evening|| +85825|AAAAAAAACEPEBAAA|85825|23|50|25|PM|third|evening|| +85826|AAAAAAAADEPEBAAA|85826|23|50|26|PM|third|evening|| +85827|AAAAAAAAEEPEBAAA|85827|23|50|27|PM|third|evening|| +85828|AAAAAAAAFEPEBAAA|85828|23|50|28|PM|third|evening|| +85829|AAAAAAAAGEPEBAAA|85829|23|50|29|PM|third|evening|| +85830|AAAAAAAAHEPEBAAA|85830|23|50|30|PM|third|evening|| +85831|AAAAAAAAIEPEBAAA|85831|23|50|31|PM|third|evening|| +85832|AAAAAAAAJEPEBAAA|85832|23|50|32|PM|third|evening|| +85833|AAAAAAAAKEPEBAAA|85833|23|50|33|PM|third|evening|| +85834|AAAAAAAALEPEBAAA|85834|23|50|34|PM|third|evening|| +85835|AAAAAAAAMEPEBAAA|85835|23|50|35|PM|third|evening|| +85836|AAAAAAAANEPEBAAA|85836|23|50|36|PM|third|evening|| +85837|AAAAAAAAOEPEBAAA|85837|23|50|37|PM|third|evening|| +85838|AAAAAAAAPEPEBAAA|85838|23|50|38|PM|third|evening|| +85839|AAAAAAAAAFPEBAAA|85839|23|50|39|PM|third|evening|| +85840|AAAAAAAABFPEBAAA|85840|23|50|40|PM|third|evening|| +85841|AAAAAAAACFPEBAAA|85841|23|50|41|PM|third|evening|| +85842|AAAAAAAADFPEBAAA|85842|23|50|42|PM|third|evening|| +85843|AAAAAAAAEFPEBAAA|85843|23|50|43|PM|third|evening|| +85844|AAAAAAAAFFPEBAAA|85844|23|50|44|PM|third|evening|| +85845|AAAAAAAAGFPEBAAA|85845|23|50|45|PM|third|evening|| +85846|AAAAAAAAHFPEBAAA|85846|23|50|46|PM|third|evening|| +85847|AAAAAAAAIFPEBAAA|85847|23|50|47|PM|third|evening|| +85848|AAAAAAAAJFPEBAAA|85848|23|50|48|PM|third|evening|| +85849|AAAAAAAAKFPEBAAA|85849|23|50|49|PM|third|evening|| +85850|AAAAAAAALFPEBAAA|85850|23|50|50|PM|third|evening|| +85851|AAAAAAAAMFPEBAAA|85851|23|50|51|PM|third|evening|| +85852|AAAAAAAANFPEBAAA|85852|23|50|52|PM|third|evening|| +85853|AAAAAAAAOFPEBAAA|85853|23|50|53|PM|third|evening|| +85854|AAAAAAAAPFPEBAAA|85854|23|50|54|PM|third|evening|| +85855|AAAAAAAAAGPEBAAA|85855|23|50|55|PM|third|evening|| +85856|AAAAAAAABGPEBAAA|85856|23|50|56|PM|third|evening|| +85857|AAAAAAAACGPEBAAA|85857|23|50|57|PM|third|evening|| +85858|AAAAAAAADGPEBAAA|85858|23|50|58|PM|third|evening|| +85859|AAAAAAAAEGPEBAAA|85859|23|50|59|PM|third|evening|| +85860|AAAAAAAAFGPEBAAA|85860|23|51|0|PM|third|evening|| +85861|AAAAAAAAGGPEBAAA|85861|23|51|1|PM|third|evening|| +85862|AAAAAAAAHGPEBAAA|85862|23|51|2|PM|third|evening|| +85863|AAAAAAAAIGPEBAAA|85863|23|51|3|PM|third|evening|| +85864|AAAAAAAAJGPEBAAA|85864|23|51|4|PM|third|evening|| +85865|AAAAAAAAKGPEBAAA|85865|23|51|5|PM|third|evening|| +85866|AAAAAAAALGPEBAAA|85866|23|51|6|PM|third|evening|| +85867|AAAAAAAAMGPEBAAA|85867|23|51|7|PM|third|evening|| +85868|AAAAAAAANGPEBAAA|85868|23|51|8|PM|third|evening|| +85869|AAAAAAAAOGPEBAAA|85869|23|51|9|PM|third|evening|| +85870|AAAAAAAAPGPEBAAA|85870|23|51|10|PM|third|evening|| +85871|AAAAAAAAAHPEBAAA|85871|23|51|11|PM|third|evening|| +85872|AAAAAAAABHPEBAAA|85872|23|51|12|PM|third|evening|| +85873|AAAAAAAACHPEBAAA|85873|23|51|13|PM|third|evening|| +85874|AAAAAAAADHPEBAAA|85874|23|51|14|PM|third|evening|| +85875|AAAAAAAAEHPEBAAA|85875|23|51|15|PM|third|evening|| +85876|AAAAAAAAFHPEBAAA|85876|23|51|16|PM|third|evening|| +85877|AAAAAAAAGHPEBAAA|85877|23|51|17|PM|third|evening|| +85878|AAAAAAAAHHPEBAAA|85878|23|51|18|PM|third|evening|| +85879|AAAAAAAAIHPEBAAA|85879|23|51|19|PM|third|evening|| +85880|AAAAAAAAJHPEBAAA|85880|23|51|20|PM|third|evening|| +85881|AAAAAAAAKHPEBAAA|85881|23|51|21|PM|third|evening|| +85882|AAAAAAAALHPEBAAA|85882|23|51|22|PM|third|evening|| +85883|AAAAAAAAMHPEBAAA|85883|23|51|23|PM|third|evening|| +85884|AAAAAAAANHPEBAAA|85884|23|51|24|PM|third|evening|| +85885|AAAAAAAAOHPEBAAA|85885|23|51|25|PM|third|evening|| +85886|AAAAAAAAPHPEBAAA|85886|23|51|26|PM|third|evening|| +85887|AAAAAAAAAIPEBAAA|85887|23|51|27|PM|third|evening|| +85888|AAAAAAAABIPEBAAA|85888|23|51|28|PM|third|evening|| +85889|AAAAAAAACIPEBAAA|85889|23|51|29|PM|third|evening|| +85890|AAAAAAAADIPEBAAA|85890|23|51|30|PM|third|evening|| +85891|AAAAAAAAEIPEBAAA|85891|23|51|31|PM|third|evening|| +85892|AAAAAAAAFIPEBAAA|85892|23|51|32|PM|third|evening|| +85893|AAAAAAAAGIPEBAAA|85893|23|51|33|PM|third|evening|| +85894|AAAAAAAAHIPEBAAA|85894|23|51|34|PM|third|evening|| +85895|AAAAAAAAIIPEBAAA|85895|23|51|35|PM|third|evening|| +85896|AAAAAAAAJIPEBAAA|85896|23|51|36|PM|third|evening|| +85897|AAAAAAAAKIPEBAAA|85897|23|51|37|PM|third|evening|| +85898|AAAAAAAALIPEBAAA|85898|23|51|38|PM|third|evening|| +85899|AAAAAAAAMIPEBAAA|85899|23|51|39|PM|third|evening|| +85900|AAAAAAAANIPEBAAA|85900|23|51|40|PM|third|evening|| +85901|AAAAAAAAOIPEBAAA|85901|23|51|41|PM|third|evening|| +85902|AAAAAAAAPIPEBAAA|85902|23|51|42|PM|third|evening|| +85903|AAAAAAAAAJPEBAAA|85903|23|51|43|PM|third|evening|| +85904|AAAAAAAABJPEBAAA|85904|23|51|44|PM|third|evening|| +85905|AAAAAAAACJPEBAAA|85905|23|51|45|PM|third|evening|| +85906|AAAAAAAADJPEBAAA|85906|23|51|46|PM|third|evening|| +85907|AAAAAAAAEJPEBAAA|85907|23|51|47|PM|third|evening|| +85908|AAAAAAAAFJPEBAAA|85908|23|51|48|PM|third|evening|| +85909|AAAAAAAAGJPEBAAA|85909|23|51|49|PM|third|evening|| +85910|AAAAAAAAHJPEBAAA|85910|23|51|50|PM|third|evening|| +85911|AAAAAAAAIJPEBAAA|85911|23|51|51|PM|third|evening|| +85912|AAAAAAAAJJPEBAAA|85912|23|51|52|PM|third|evening|| +85913|AAAAAAAAKJPEBAAA|85913|23|51|53|PM|third|evening|| +85914|AAAAAAAALJPEBAAA|85914|23|51|54|PM|third|evening|| +85915|AAAAAAAAMJPEBAAA|85915|23|51|55|PM|third|evening|| +85916|AAAAAAAANJPEBAAA|85916|23|51|56|PM|third|evening|| +85917|AAAAAAAAOJPEBAAA|85917|23|51|57|PM|third|evening|| +85918|AAAAAAAAPJPEBAAA|85918|23|51|58|PM|third|evening|| +85919|AAAAAAAAAKPEBAAA|85919|23|51|59|PM|third|evening|| +85920|AAAAAAAABKPEBAAA|85920|23|52|0|PM|third|evening|| +85921|AAAAAAAACKPEBAAA|85921|23|52|1|PM|third|evening|| +85922|AAAAAAAADKPEBAAA|85922|23|52|2|PM|third|evening|| +85923|AAAAAAAAEKPEBAAA|85923|23|52|3|PM|third|evening|| +85924|AAAAAAAAFKPEBAAA|85924|23|52|4|PM|third|evening|| +85925|AAAAAAAAGKPEBAAA|85925|23|52|5|PM|third|evening|| +85926|AAAAAAAAHKPEBAAA|85926|23|52|6|PM|third|evening|| +85927|AAAAAAAAIKPEBAAA|85927|23|52|7|PM|third|evening|| +85928|AAAAAAAAJKPEBAAA|85928|23|52|8|PM|third|evening|| +85929|AAAAAAAAKKPEBAAA|85929|23|52|9|PM|third|evening|| +85930|AAAAAAAALKPEBAAA|85930|23|52|10|PM|third|evening|| +85931|AAAAAAAAMKPEBAAA|85931|23|52|11|PM|third|evening|| +85932|AAAAAAAANKPEBAAA|85932|23|52|12|PM|third|evening|| +85933|AAAAAAAAOKPEBAAA|85933|23|52|13|PM|third|evening|| +85934|AAAAAAAAPKPEBAAA|85934|23|52|14|PM|third|evening|| +85935|AAAAAAAAALPEBAAA|85935|23|52|15|PM|third|evening|| +85936|AAAAAAAABLPEBAAA|85936|23|52|16|PM|third|evening|| +85937|AAAAAAAACLPEBAAA|85937|23|52|17|PM|third|evening|| +85938|AAAAAAAADLPEBAAA|85938|23|52|18|PM|third|evening|| +85939|AAAAAAAAELPEBAAA|85939|23|52|19|PM|third|evening|| +85940|AAAAAAAAFLPEBAAA|85940|23|52|20|PM|third|evening|| +85941|AAAAAAAAGLPEBAAA|85941|23|52|21|PM|third|evening|| +85942|AAAAAAAAHLPEBAAA|85942|23|52|22|PM|third|evening|| +85943|AAAAAAAAILPEBAAA|85943|23|52|23|PM|third|evening|| +85944|AAAAAAAAJLPEBAAA|85944|23|52|24|PM|third|evening|| +85945|AAAAAAAAKLPEBAAA|85945|23|52|25|PM|third|evening|| +85946|AAAAAAAALLPEBAAA|85946|23|52|26|PM|third|evening|| +85947|AAAAAAAAMLPEBAAA|85947|23|52|27|PM|third|evening|| +85948|AAAAAAAANLPEBAAA|85948|23|52|28|PM|third|evening|| +85949|AAAAAAAAOLPEBAAA|85949|23|52|29|PM|third|evening|| +85950|AAAAAAAAPLPEBAAA|85950|23|52|30|PM|third|evening|| +85951|AAAAAAAAAMPEBAAA|85951|23|52|31|PM|third|evening|| +85952|AAAAAAAABMPEBAAA|85952|23|52|32|PM|third|evening|| +85953|AAAAAAAACMPEBAAA|85953|23|52|33|PM|third|evening|| +85954|AAAAAAAADMPEBAAA|85954|23|52|34|PM|third|evening|| +85955|AAAAAAAAEMPEBAAA|85955|23|52|35|PM|third|evening|| +85956|AAAAAAAAFMPEBAAA|85956|23|52|36|PM|third|evening|| +85957|AAAAAAAAGMPEBAAA|85957|23|52|37|PM|third|evening|| +85958|AAAAAAAAHMPEBAAA|85958|23|52|38|PM|third|evening|| +85959|AAAAAAAAIMPEBAAA|85959|23|52|39|PM|third|evening|| +85960|AAAAAAAAJMPEBAAA|85960|23|52|40|PM|third|evening|| +85961|AAAAAAAAKMPEBAAA|85961|23|52|41|PM|third|evening|| +85962|AAAAAAAALMPEBAAA|85962|23|52|42|PM|third|evening|| +85963|AAAAAAAAMMPEBAAA|85963|23|52|43|PM|third|evening|| +85964|AAAAAAAANMPEBAAA|85964|23|52|44|PM|third|evening|| +85965|AAAAAAAAOMPEBAAA|85965|23|52|45|PM|third|evening|| +85966|AAAAAAAAPMPEBAAA|85966|23|52|46|PM|third|evening|| +85967|AAAAAAAAANPEBAAA|85967|23|52|47|PM|third|evening|| +85968|AAAAAAAABNPEBAAA|85968|23|52|48|PM|third|evening|| +85969|AAAAAAAACNPEBAAA|85969|23|52|49|PM|third|evening|| +85970|AAAAAAAADNPEBAAA|85970|23|52|50|PM|third|evening|| +85971|AAAAAAAAENPEBAAA|85971|23|52|51|PM|third|evening|| +85972|AAAAAAAAFNPEBAAA|85972|23|52|52|PM|third|evening|| +85973|AAAAAAAAGNPEBAAA|85973|23|52|53|PM|third|evening|| +85974|AAAAAAAAHNPEBAAA|85974|23|52|54|PM|third|evening|| +85975|AAAAAAAAINPEBAAA|85975|23|52|55|PM|third|evening|| +85976|AAAAAAAAJNPEBAAA|85976|23|52|56|PM|third|evening|| +85977|AAAAAAAAKNPEBAAA|85977|23|52|57|PM|third|evening|| +85978|AAAAAAAALNPEBAAA|85978|23|52|58|PM|third|evening|| +85979|AAAAAAAAMNPEBAAA|85979|23|52|59|PM|third|evening|| +85980|AAAAAAAANNPEBAAA|85980|23|53|0|PM|third|evening|| +85981|AAAAAAAAONPEBAAA|85981|23|53|1|PM|third|evening|| +85982|AAAAAAAAPNPEBAAA|85982|23|53|2|PM|third|evening|| +85983|AAAAAAAAAOPEBAAA|85983|23|53|3|PM|third|evening|| +85984|AAAAAAAABOPEBAAA|85984|23|53|4|PM|third|evening|| +85985|AAAAAAAACOPEBAAA|85985|23|53|5|PM|third|evening|| +85986|AAAAAAAADOPEBAAA|85986|23|53|6|PM|third|evening|| +85987|AAAAAAAAEOPEBAAA|85987|23|53|7|PM|third|evening|| +85988|AAAAAAAAFOPEBAAA|85988|23|53|8|PM|third|evening|| +85989|AAAAAAAAGOPEBAAA|85989|23|53|9|PM|third|evening|| +85990|AAAAAAAAHOPEBAAA|85990|23|53|10|PM|third|evening|| +85991|AAAAAAAAIOPEBAAA|85991|23|53|11|PM|third|evening|| +85992|AAAAAAAAJOPEBAAA|85992|23|53|12|PM|third|evening|| +85993|AAAAAAAAKOPEBAAA|85993|23|53|13|PM|third|evening|| +85994|AAAAAAAALOPEBAAA|85994|23|53|14|PM|third|evening|| +85995|AAAAAAAAMOPEBAAA|85995|23|53|15|PM|third|evening|| +85996|AAAAAAAANOPEBAAA|85996|23|53|16|PM|third|evening|| +85997|AAAAAAAAOOPEBAAA|85997|23|53|17|PM|third|evening|| +85998|AAAAAAAAPOPEBAAA|85998|23|53|18|PM|third|evening|| +85999|AAAAAAAAAPPEBAAA|85999|23|53|19|PM|third|evening|| +86000|AAAAAAAABPPEBAAA|86000|23|53|20|PM|third|evening|| +86001|AAAAAAAACPPEBAAA|86001|23|53|21|PM|third|evening|| +86002|AAAAAAAADPPEBAAA|86002|23|53|22|PM|third|evening|| +86003|AAAAAAAAEPPEBAAA|86003|23|53|23|PM|third|evening|| +86004|AAAAAAAAFPPEBAAA|86004|23|53|24|PM|third|evening|| +86005|AAAAAAAAGPPEBAAA|86005|23|53|25|PM|third|evening|| +86006|AAAAAAAAHPPEBAAA|86006|23|53|26|PM|third|evening|| +86007|AAAAAAAAIPPEBAAA|86007|23|53|27|PM|third|evening|| +86008|AAAAAAAAJPPEBAAA|86008|23|53|28|PM|third|evening|| +86009|AAAAAAAAKPPEBAAA|86009|23|53|29|PM|third|evening|| +86010|AAAAAAAALPPEBAAA|86010|23|53|30|PM|third|evening|| +86011|AAAAAAAAMPPEBAAA|86011|23|53|31|PM|third|evening|| +86012|AAAAAAAANPPEBAAA|86012|23|53|32|PM|third|evening|| +86013|AAAAAAAAOPPEBAAA|86013|23|53|33|PM|third|evening|| +86014|AAAAAAAAPPPEBAAA|86014|23|53|34|PM|third|evening|| +86015|AAAAAAAAAAAFBAAA|86015|23|53|35|PM|third|evening|| +86016|AAAAAAAABAAFBAAA|86016|23|53|36|PM|third|evening|| +86017|AAAAAAAACAAFBAAA|86017|23|53|37|PM|third|evening|| +86018|AAAAAAAADAAFBAAA|86018|23|53|38|PM|third|evening|| +86019|AAAAAAAAEAAFBAAA|86019|23|53|39|PM|third|evening|| +86020|AAAAAAAAFAAFBAAA|86020|23|53|40|PM|third|evening|| +86021|AAAAAAAAGAAFBAAA|86021|23|53|41|PM|third|evening|| +86022|AAAAAAAAHAAFBAAA|86022|23|53|42|PM|third|evening|| +86023|AAAAAAAAIAAFBAAA|86023|23|53|43|PM|third|evening|| +86024|AAAAAAAAJAAFBAAA|86024|23|53|44|PM|third|evening|| +86025|AAAAAAAAKAAFBAAA|86025|23|53|45|PM|third|evening|| +86026|AAAAAAAALAAFBAAA|86026|23|53|46|PM|third|evening|| +86027|AAAAAAAAMAAFBAAA|86027|23|53|47|PM|third|evening|| +86028|AAAAAAAANAAFBAAA|86028|23|53|48|PM|third|evening|| +86029|AAAAAAAAOAAFBAAA|86029|23|53|49|PM|third|evening|| +86030|AAAAAAAAPAAFBAAA|86030|23|53|50|PM|third|evening|| +86031|AAAAAAAAABAFBAAA|86031|23|53|51|PM|third|evening|| +86032|AAAAAAAABBAFBAAA|86032|23|53|52|PM|third|evening|| +86033|AAAAAAAACBAFBAAA|86033|23|53|53|PM|third|evening|| +86034|AAAAAAAADBAFBAAA|86034|23|53|54|PM|third|evening|| +86035|AAAAAAAAEBAFBAAA|86035|23|53|55|PM|third|evening|| +86036|AAAAAAAAFBAFBAAA|86036|23|53|56|PM|third|evening|| +86037|AAAAAAAAGBAFBAAA|86037|23|53|57|PM|third|evening|| +86038|AAAAAAAAHBAFBAAA|86038|23|53|58|PM|third|evening|| +86039|AAAAAAAAIBAFBAAA|86039|23|53|59|PM|third|evening|| +86040|AAAAAAAAJBAFBAAA|86040|23|54|0|PM|third|evening|| +86041|AAAAAAAAKBAFBAAA|86041|23|54|1|PM|third|evening|| +86042|AAAAAAAALBAFBAAA|86042|23|54|2|PM|third|evening|| +86043|AAAAAAAAMBAFBAAA|86043|23|54|3|PM|third|evening|| +86044|AAAAAAAANBAFBAAA|86044|23|54|4|PM|third|evening|| +86045|AAAAAAAAOBAFBAAA|86045|23|54|5|PM|third|evening|| +86046|AAAAAAAAPBAFBAAA|86046|23|54|6|PM|third|evening|| +86047|AAAAAAAAACAFBAAA|86047|23|54|7|PM|third|evening|| +86048|AAAAAAAABCAFBAAA|86048|23|54|8|PM|third|evening|| +86049|AAAAAAAACCAFBAAA|86049|23|54|9|PM|third|evening|| +86050|AAAAAAAADCAFBAAA|86050|23|54|10|PM|third|evening|| +86051|AAAAAAAAECAFBAAA|86051|23|54|11|PM|third|evening|| +86052|AAAAAAAAFCAFBAAA|86052|23|54|12|PM|third|evening|| +86053|AAAAAAAAGCAFBAAA|86053|23|54|13|PM|third|evening|| +86054|AAAAAAAAHCAFBAAA|86054|23|54|14|PM|third|evening|| +86055|AAAAAAAAICAFBAAA|86055|23|54|15|PM|third|evening|| +86056|AAAAAAAAJCAFBAAA|86056|23|54|16|PM|third|evening|| +86057|AAAAAAAAKCAFBAAA|86057|23|54|17|PM|third|evening|| +86058|AAAAAAAALCAFBAAA|86058|23|54|18|PM|third|evening|| +86059|AAAAAAAAMCAFBAAA|86059|23|54|19|PM|third|evening|| +86060|AAAAAAAANCAFBAAA|86060|23|54|20|PM|third|evening|| +86061|AAAAAAAAOCAFBAAA|86061|23|54|21|PM|third|evening|| +86062|AAAAAAAAPCAFBAAA|86062|23|54|22|PM|third|evening|| +86063|AAAAAAAAADAFBAAA|86063|23|54|23|PM|third|evening|| +86064|AAAAAAAABDAFBAAA|86064|23|54|24|PM|third|evening|| +86065|AAAAAAAACDAFBAAA|86065|23|54|25|PM|third|evening|| +86066|AAAAAAAADDAFBAAA|86066|23|54|26|PM|third|evening|| +86067|AAAAAAAAEDAFBAAA|86067|23|54|27|PM|third|evening|| +86068|AAAAAAAAFDAFBAAA|86068|23|54|28|PM|third|evening|| +86069|AAAAAAAAGDAFBAAA|86069|23|54|29|PM|third|evening|| +86070|AAAAAAAAHDAFBAAA|86070|23|54|30|PM|third|evening|| +86071|AAAAAAAAIDAFBAAA|86071|23|54|31|PM|third|evening|| +86072|AAAAAAAAJDAFBAAA|86072|23|54|32|PM|third|evening|| +86073|AAAAAAAAKDAFBAAA|86073|23|54|33|PM|third|evening|| +86074|AAAAAAAALDAFBAAA|86074|23|54|34|PM|third|evening|| +86075|AAAAAAAAMDAFBAAA|86075|23|54|35|PM|third|evening|| +86076|AAAAAAAANDAFBAAA|86076|23|54|36|PM|third|evening|| +86077|AAAAAAAAODAFBAAA|86077|23|54|37|PM|third|evening|| +86078|AAAAAAAAPDAFBAAA|86078|23|54|38|PM|third|evening|| +86079|AAAAAAAAAEAFBAAA|86079|23|54|39|PM|third|evening|| +86080|AAAAAAAABEAFBAAA|86080|23|54|40|PM|third|evening|| +86081|AAAAAAAACEAFBAAA|86081|23|54|41|PM|third|evening|| +86082|AAAAAAAADEAFBAAA|86082|23|54|42|PM|third|evening|| +86083|AAAAAAAAEEAFBAAA|86083|23|54|43|PM|third|evening|| +86084|AAAAAAAAFEAFBAAA|86084|23|54|44|PM|third|evening|| +86085|AAAAAAAAGEAFBAAA|86085|23|54|45|PM|third|evening|| +86086|AAAAAAAAHEAFBAAA|86086|23|54|46|PM|third|evening|| +86087|AAAAAAAAIEAFBAAA|86087|23|54|47|PM|third|evening|| +86088|AAAAAAAAJEAFBAAA|86088|23|54|48|PM|third|evening|| +86089|AAAAAAAAKEAFBAAA|86089|23|54|49|PM|third|evening|| +86090|AAAAAAAALEAFBAAA|86090|23|54|50|PM|third|evening|| +86091|AAAAAAAAMEAFBAAA|86091|23|54|51|PM|third|evening|| +86092|AAAAAAAANEAFBAAA|86092|23|54|52|PM|third|evening|| +86093|AAAAAAAAOEAFBAAA|86093|23|54|53|PM|third|evening|| +86094|AAAAAAAAPEAFBAAA|86094|23|54|54|PM|third|evening|| +86095|AAAAAAAAAFAFBAAA|86095|23|54|55|PM|third|evening|| +86096|AAAAAAAABFAFBAAA|86096|23|54|56|PM|third|evening|| +86097|AAAAAAAACFAFBAAA|86097|23|54|57|PM|third|evening|| +86098|AAAAAAAADFAFBAAA|86098|23|54|58|PM|third|evening|| +86099|AAAAAAAAEFAFBAAA|86099|23|54|59|PM|third|evening|| +86100|AAAAAAAAFFAFBAAA|86100|23|55|0|PM|third|evening|| +86101|AAAAAAAAGFAFBAAA|86101|23|55|1|PM|third|evening|| +86102|AAAAAAAAHFAFBAAA|86102|23|55|2|PM|third|evening|| +86103|AAAAAAAAIFAFBAAA|86103|23|55|3|PM|third|evening|| +86104|AAAAAAAAJFAFBAAA|86104|23|55|4|PM|third|evening|| +86105|AAAAAAAAKFAFBAAA|86105|23|55|5|PM|third|evening|| +86106|AAAAAAAALFAFBAAA|86106|23|55|6|PM|third|evening|| +86107|AAAAAAAAMFAFBAAA|86107|23|55|7|PM|third|evening|| +86108|AAAAAAAANFAFBAAA|86108|23|55|8|PM|third|evening|| +86109|AAAAAAAAOFAFBAAA|86109|23|55|9|PM|third|evening|| +86110|AAAAAAAAPFAFBAAA|86110|23|55|10|PM|third|evening|| +86111|AAAAAAAAAGAFBAAA|86111|23|55|11|PM|third|evening|| +86112|AAAAAAAABGAFBAAA|86112|23|55|12|PM|third|evening|| +86113|AAAAAAAACGAFBAAA|86113|23|55|13|PM|third|evening|| +86114|AAAAAAAADGAFBAAA|86114|23|55|14|PM|third|evening|| +86115|AAAAAAAAEGAFBAAA|86115|23|55|15|PM|third|evening|| +86116|AAAAAAAAFGAFBAAA|86116|23|55|16|PM|third|evening|| +86117|AAAAAAAAGGAFBAAA|86117|23|55|17|PM|third|evening|| +86118|AAAAAAAAHGAFBAAA|86118|23|55|18|PM|third|evening|| +86119|AAAAAAAAIGAFBAAA|86119|23|55|19|PM|third|evening|| +86120|AAAAAAAAJGAFBAAA|86120|23|55|20|PM|third|evening|| +86121|AAAAAAAAKGAFBAAA|86121|23|55|21|PM|third|evening|| +86122|AAAAAAAALGAFBAAA|86122|23|55|22|PM|third|evening|| +86123|AAAAAAAAMGAFBAAA|86123|23|55|23|PM|third|evening|| +86124|AAAAAAAANGAFBAAA|86124|23|55|24|PM|third|evening|| +86125|AAAAAAAAOGAFBAAA|86125|23|55|25|PM|third|evening|| +86126|AAAAAAAAPGAFBAAA|86126|23|55|26|PM|third|evening|| +86127|AAAAAAAAAHAFBAAA|86127|23|55|27|PM|third|evening|| +86128|AAAAAAAABHAFBAAA|86128|23|55|28|PM|third|evening|| +86129|AAAAAAAACHAFBAAA|86129|23|55|29|PM|third|evening|| +86130|AAAAAAAADHAFBAAA|86130|23|55|30|PM|third|evening|| +86131|AAAAAAAAEHAFBAAA|86131|23|55|31|PM|third|evening|| +86132|AAAAAAAAFHAFBAAA|86132|23|55|32|PM|third|evening|| +86133|AAAAAAAAGHAFBAAA|86133|23|55|33|PM|third|evening|| +86134|AAAAAAAAHHAFBAAA|86134|23|55|34|PM|third|evening|| +86135|AAAAAAAAIHAFBAAA|86135|23|55|35|PM|third|evening|| +86136|AAAAAAAAJHAFBAAA|86136|23|55|36|PM|third|evening|| +86137|AAAAAAAAKHAFBAAA|86137|23|55|37|PM|third|evening|| +86138|AAAAAAAALHAFBAAA|86138|23|55|38|PM|third|evening|| +86139|AAAAAAAAMHAFBAAA|86139|23|55|39|PM|third|evening|| +86140|AAAAAAAANHAFBAAA|86140|23|55|40|PM|third|evening|| +86141|AAAAAAAAOHAFBAAA|86141|23|55|41|PM|third|evening|| +86142|AAAAAAAAPHAFBAAA|86142|23|55|42|PM|third|evening|| +86143|AAAAAAAAAIAFBAAA|86143|23|55|43|PM|third|evening|| +86144|AAAAAAAABIAFBAAA|86144|23|55|44|PM|third|evening|| +86145|AAAAAAAACIAFBAAA|86145|23|55|45|PM|third|evening|| +86146|AAAAAAAADIAFBAAA|86146|23|55|46|PM|third|evening|| +86147|AAAAAAAAEIAFBAAA|86147|23|55|47|PM|third|evening|| +86148|AAAAAAAAFIAFBAAA|86148|23|55|48|PM|third|evening|| +86149|AAAAAAAAGIAFBAAA|86149|23|55|49|PM|third|evening|| +86150|AAAAAAAAHIAFBAAA|86150|23|55|50|PM|third|evening|| +86151|AAAAAAAAIIAFBAAA|86151|23|55|51|PM|third|evening|| +86152|AAAAAAAAJIAFBAAA|86152|23|55|52|PM|third|evening|| +86153|AAAAAAAAKIAFBAAA|86153|23|55|53|PM|third|evening|| +86154|AAAAAAAALIAFBAAA|86154|23|55|54|PM|third|evening|| +86155|AAAAAAAAMIAFBAAA|86155|23|55|55|PM|third|evening|| +86156|AAAAAAAANIAFBAAA|86156|23|55|56|PM|third|evening|| +86157|AAAAAAAAOIAFBAAA|86157|23|55|57|PM|third|evening|| +86158|AAAAAAAAPIAFBAAA|86158|23|55|58|PM|third|evening|| +86159|AAAAAAAAAJAFBAAA|86159|23|55|59|PM|third|evening|| +86160|AAAAAAAABJAFBAAA|86160|23|56|0|PM|third|evening|| +86161|AAAAAAAACJAFBAAA|86161|23|56|1|PM|third|evening|| +86162|AAAAAAAADJAFBAAA|86162|23|56|2|PM|third|evening|| +86163|AAAAAAAAEJAFBAAA|86163|23|56|3|PM|third|evening|| +86164|AAAAAAAAFJAFBAAA|86164|23|56|4|PM|third|evening|| +86165|AAAAAAAAGJAFBAAA|86165|23|56|5|PM|third|evening|| +86166|AAAAAAAAHJAFBAAA|86166|23|56|6|PM|third|evening|| +86167|AAAAAAAAIJAFBAAA|86167|23|56|7|PM|third|evening|| +86168|AAAAAAAAJJAFBAAA|86168|23|56|8|PM|third|evening|| +86169|AAAAAAAAKJAFBAAA|86169|23|56|9|PM|third|evening|| +86170|AAAAAAAALJAFBAAA|86170|23|56|10|PM|third|evening|| +86171|AAAAAAAAMJAFBAAA|86171|23|56|11|PM|third|evening|| +86172|AAAAAAAANJAFBAAA|86172|23|56|12|PM|third|evening|| +86173|AAAAAAAAOJAFBAAA|86173|23|56|13|PM|third|evening|| +86174|AAAAAAAAPJAFBAAA|86174|23|56|14|PM|third|evening|| +86175|AAAAAAAAAKAFBAAA|86175|23|56|15|PM|third|evening|| +86176|AAAAAAAABKAFBAAA|86176|23|56|16|PM|third|evening|| +86177|AAAAAAAACKAFBAAA|86177|23|56|17|PM|third|evening|| +86178|AAAAAAAADKAFBAAA|86178|23|56|18|PM|third|evening|| +86179|AAAAAAAAEKAFBAAA|86179|23|56|19|PM|third|evening|| +86180|AAAAAAAAFKAFBAAA|86180|23|56|20|PM|third|evening|| +86181|AAAAAAAAGKAFBAAA|86181|23|56|21|PM|third|evening|| +86182|AAAAAAAAHKAFBAAA|86182|23|56|22|PM|third|evening|| +86183|AAAAAAAAIKAFBAAA|86183|23|56|23|PM|third|evening|| +86184|AAAAAAAAJKAFBAAA|86184|23|56|24|PM|third|evening|| +86185|AAAAAAAAKKAFBAAA|86185|23|56|25|PM|third|evening|| +86186|AAAAAAAALKAFBAAA|86186|23|56|26|PM|third|evening|| +86187|AAAAAAAAMKAFBAAA|86187|23|56|27|PM|third|evening|| +86188|AAAAAAAANKAFBAAA|86188|23|56|28|PM|third|evening|| +86189|AAAAAAAAOKAFBAAA|86189|23|56|29|PM|third|evening|| +86190|AAAAAAAAPKAFBAAA|86190|23|56|30|PM|third|evening|| +86191|AAAAAAAAALAFBAAA|86191|23|56|31|PM|third|evening|| +86192|AAAAAAAABLAFBAAA|86192|23|56|32|PM|third|evening|| +86193|AAAAAAAACLAFBAAA|86193|23|56|33|PM|third|evening|| +86194|AAAAAAAADLAFBAAA|86194|23|56|34|PM|third|evening|| +86195|AAAAAAAAELAFBAAA|86195|23|56|35|PM|third|evening|| +86196|AAAAAAAAFLAFBAAA|86196|23|56|36|PM|third|evening|| +86197|AAAAAAAAGLAFBAAA|86197|23|56|37|PM|third|evening|| +86198|AAAAAAAAHLAFBAAA|86198|23|56|38|PM|third|evening|| +86199|AAAAAAAAILAFBAAA|86199|23|56|39|PM|third|evening|| +86200|AAAAAAAAJLAFBAAA|86200|23|56|40|PM|third|evening|| +86201|AAAAAAAAKLAFBAAA|86201|23|56|41|PM|third|evening|| +86202|AAAAAAAALLAFBAAA|86202|23|56|42|PM|third|evening|| +86203|AAAAAAAAMLAFBAAA|86203|23|56|43|PM|third|evening|| +86204|AAAAAAAANLAFBAAA|86204|23|56|44|PM|third|evening|| +86205|AAAAAAAAOLAFBAAA|86205|23|56|45|PM|third|evening|| +86206|AAAAAAAAPLAFBAAA|86206|23|56|46|PM|third|evening|| +86207|AAAAAAAAAMAFBAAA|86207|23|56|47|PM|third|evening|| +86208|AAAAAAAABMAFBAAA|86208|23|56|48|PM|third|evening|| +86209|AAAAAAAACMAFBAAA|86209|23|56|49|PM|third|evening|| +86210|AAAAAAAADMAFBAAA|86210|23|56|50|PM|third|evening|| +86211|AAAAAAAAEMAFBAAA|86211|23|56|51|PM|third|evening|| +86212|AAAAAAAAFMAFBAAA|86212|23|56|52|PM|third|evening|| +86213|AAAAAAAAGMAFBAAA|86213|23|56|53|PM|third|evening|| +86214|AAAAAAAAHMAFBAAA|86214|23|56|54|PM|third|evening|| +86215|AAAAAAAAIMAFBAAA|86215|23|56|55|PM|third|evening|| +86216|AAAAAAAAJMAFBAAA|86216|23|56|56|PM|third|evening|| +86217|AAAAAAAAKMAFBAAA|86217|23|56|57|PM|third|evening|| +86218|AAAAAAAALMAFBAAA|86218|23|56|58|PM|third|evening|| +86219|AAAAAAAAMMAFBAAA|86219|23|56|59|PM|third|evening|| +86220|AAAAAAAANMAFBAAA|86220|23|57|0|PM|third|evening|| +86221|AAAAAAAAOMAFBAAA|86221|23|57|1|PM|third|evening|| +86222|AAAAAAAAPMAFBAAA|86222|23|57|2|PM|third|evening|| +86223|AAAAAAAAANAFBAAA|86223|23|57|3|PM|third|evening|| +86224|AAAAAAAABNAFBAAA|86224|23|57|4|PM|third|evening|| +86225|AAAAAAAACNAFBAAA|86225|23|57|5|PM|third|evening|| +86226|AAAAAAAADNAFBAAA|86226|23|57|6|PM|third|evening|| +86227|AAAAAAAAENAFBAAA|86227|23|57|7|PM|third|evening|| +86228|AAAAAAAAFNAFBAAA|86228|23|57|8|PM|third|evening|| +86229|AAAAAAAAGNAFBAAA|86229|23|57|9|PM|third|evening|| +86230|AAAAAAAAHNAFBAAA|86230|23|57|10|PM|third|evening|| +86231|AAAAAAAAINAFBAAA|86231|23|57|11|PM|third|evening|| +86232|AAAAAAAAJNAFBAAA|86232|23|57|12|PM|third|evening|| +86233|AAAAAAAAKNAFBAAA|86233|23|57|13|PM|third|evening|| +86234|AAAAAAAALNAFBAAA|86234|23|57|14|PM|third|evening|| +86235|AAAAAAAAMNAFBAAA|86235|23|57|15|PM|third|evening|| +86236|AAAAAAAANNAFBAAA|86236|23|57|16|PM|third|evening|| +86237|AAAAAAAAONAFBAAA|86237|23|57|17|PM|third|evening|| +86238|AAAAAAAAPNAFBAAA|86238|23|57|18|PM|third|evening|| +86239|AAAAAAAAAOAFBAAA|86239|23|57|19|PM|third|evening|| +86240|AAAAAAAABOAFBAAA|86240|23|57|20|PM|third|evening|| +86241|AAAAAAAACOAFBAAA|86241|23|57|21|PM|third|evening|| +86242|AAAAAAAADOAFBAAA|86242|23|57|22|PM|third|evening|| +86243|AAAAAAAAEOAFBAAA|86243|23|57|23|PM|third|evening|| +86244|AAAAAAAAFOAFBAAA|86244|23|57|24|PM|third|evening|| +86245|AAAAAAAAGOAFBAAA|86245|23|57|25|PM|third|evening|| +86246|AAAAAAAAHOAFBAAA|86246|23|57|26|PM|third|evening|| +86247|AAAAAAAAIOAFBAAA|86247|23|57|27|PM|third|evening|| +86248|AAAAAAAAJOAFBAAA|86248|23|57|28|PM|third|evening|| +86249|AAAAAAAAKOAFBAAA|86249|23|57|29|PM|third|evening|| +86250|AAAAAAAALOAFBAAA|86250|23|57|30|PM|third|evening|| +86251|AAAAAAAAMOAFBAAA|86251|23|57|31|PM|third|evening|| +86252|AAAAAAAANOAFBAAA|86252|23|57|32|PM|third|evening|| +86253|AAAAAAAAOOAFBAAA|86253|23|57|33|PM|third|evening|| +86254|AAAAAAAAPOAFBAAA|86254|23|57|34|PM|third|evening|| +86255|AAAAAAAAAPAFBAAA|86255|23|57|35|PM|third|evening|| +86256|AAAAAAAABPAFBAAA|86256|23|57|36|PM|third|evening|| +86257|AAAAAAAACPAFBAAA|86257|23|57|37|PM|third|evening|| +86258|AAAAAAAADPAFBAAA|86258|23|57|38|PM|third|evening|| +86259|AAAAAAAAEPAFBAAA|86259|23|57|39|PM|third|evening|| +86260|AAAAAAAAFPAFBAAA|86260|23|57|40|PM|third|evening|| +86261|AAAAAAAAGPAFBAAA|86261|23|57|41|PM|third|evening|| +86262|AAAAAAAAHPAFBAAA|86262|23|57|42|PM|third|evening|| +86263|AAAAAAAAIPAFBAAA|86263|23|57|43|PM|third|evening|| +86264|AAAAAAAAJPAFBAAA|86264|23|57|44|PM|third|evening|| +86265|AAAAAAAAKPAFBAAA|86265|23|57|45|PM|third|evening|| +86266|AAAAAAAALPAFBAAA|86266|23|57|46|PM|third|evening|| +86267|AAAAAAAAMPAFBAAA|86267|23|57|47|PM|third|evening|| +86268|AAAAAAAANPAFBAAA|86268|23|57|48|PM|third|evening|| +86269|AAAAAAAAOPAFBAAA|86269|23|57|49|PM|third|evening|| +86270|AAAAAAAAPPAFBAAA|86270|23|57|50|PM|third|evening|| +86271|AAAAAAAAAABFBAAA|86271|23|57|51|PM|third|evening|| +86272|AAAAAAAABABFBAAA|86272|23|57|52|PM|third|evening|| +86273|AAAAAAAACABFBAAA|86273|23|57|53|PM|third|evening|| +86274|AAAAAAAADABFBAAA|86274|23|57|54|PM|third|evening|| +86275|AAAAAAAAEABFBAAA|86275|23|57|55|PM|third|evening|| +86276|AAAAAAAAFABFBAAA|86276|23|57|56|PM|third|evening|| +86277|AAAAAAAAGABFBAAA|86277|23|57|57|PM|third|evening|| +86278|AAAAAAAAHABFBAAA|86278|23|57|58|PM|third|evening|| +86279|AAAAAAAAIABFBAAA|86279|23|57|59|PM|third|evening|| +86280|AAAAAAAAJABFBAAA|86280|23|58|0|PM|third|evening|| +86281|AAAAAAAAKABFBAAA|86281|23|58|1|PM|third|evening|| +86282|AAAAAAAALABFBAAA|86282|23|58|2|PM|third|evening|| +86283|AAAAAAAAMABFBAAA|86283|23|58|3|PM|third|evening|| +86284|AAAAAAAANABFBAAA|86284|23|58|4|PM|third|evening|| +86285|AAAAAAAAOABFBAAA|86285|23|58|5|PM|third|evening|| +86286|AAAAAAAAPABFBAAA|86286|23|58|6|PM|third|evening|| +86287|AAAAAAAAABBFBAAA|86287|23|58|7|PM|third|evening|| +86288|AAAAAAAABBBFBAAA|86288|23|58|8|PM|third|evening|| +86289|AAAAAAAACBBFBAAA|86289|23|58|9|PM|third|evening|| +86290|AAAAAAAADBBFBAAA|86290|23|58|10|PM|third|evening|| +86291|AAAAAAAAEBBFBAAA|86291|23|58|11|PM|third|evening|| +86292|AAAAAAAAFBBFBAAA|86292|23|58|12|PM|third|evening|| +86293|AAAAAAAAGBBFBAAA|86293|23|58|13|PM|third|evening|| +86294|AAAAAAAAHBBFBAAA|86294|23|58|14|PM|third|evening|| +86295|AAAAAAAAIBBFBAAA|86295|23|58|15|PM|third|evening|| +86296|AAAAAAAAJBBFBAAA|86296|23|58|16|PM|third|evening|| +86297|AAAAAAAAKBBFBAAA|86297|23|58|17|PM|third|evening|| +86298|AAAAAAAALBBFBAAA|86298|23|58|18|PM|third|evening|| +86299|AAAAAAAAMBBFBAAA|86299|23|58|19|PM|third|evening|| +86300|AAAAAAAANBBFBAAA|86300|23|58|20|PM|third|evening|| +86301|AAAAAAAAOBBFBAAA|86301|23|58|21|PM|third|evening|| +86302|AAAAAAAAPBBFBAAA|86302|23|58|22|PM|third|evening|| +86303|AAAAAAAAACBFBAAA|86303|23|58|23|PM|third|evening|| +86304|AAAAAAAABCBFBAAA|86304|23|58|24|PM|third|evening|| +86305|AAAAAAAACCBFBAAA|86305|23|58|25|PM|third|evening|| +86306|AAAAAAAADCBFBAAA|86306|23|58|26|PM|third|evening|| +86307|AAAAAAAAECBFBAAA|86307|23|58|27|PM|third|evening|| +86308|AAAAAAAAFCBFBAAA|86308|23|58|28|PM|third|evening|| +86309|AAAAAAAAGCBFBAAA|86309|23|58|29|PM|third|evening|| +86310|AAAAAAAAHCBFBAAA|86310|23|58|30|PM|third|evening|| +86311|AAAAAAAAICBFBAAA|86311|23|58|31|PM|third|evening|| +86312|AAAAAAAAJCBFBAAA|86312|23|58|32|PM|third|evening|| +86313|AAAAAAAAKCBFBAAA|86313|23|58|33|PM|third|evening|| +86314|AAAAAAAALCBFBAAA|86314|23|58|34|PM|third|evening|| +86315|AAAAAAAAMCBFBAAA|86315|23|58|35|PM|third|evening|| +86316|AAAAAAAANCBFBAAA|86316|23|58|36|PM|third|evening|| +86317|AAAAAAAAOCBFBAAA|86317|23|58|37|PM|third|evening|| +86318|AAAAAAAAPCBFBAAA|86318|23|58|38|PM|third|evening|| +86319|AAAAAAAAADBFBAAA|86319|23|58|39|PM|third|evening|| +86320|AAAAAAAABDBFBAAA|86320|23|58|40|PM|third|evening|| +86321|AAAAAAAACDBFBAAA|86321|23|58|41|PM|third|evening|| +86322|AAAAAAAADDBFBAAA|86322|23|58|42|PM|third|evening|| +86323|AAAAAAAAEDBFBAAA|86323|23|58|43|PM|third|evening|| +86324|AAAAAAAAFDBFBAAA|86324|23|58|44|PM|third|evening|| +86325|AAAAAAAAGDBFBAAA|86325|23|58|45|PM|third|evening|| +86326|AAAAAAAAHDBFBAAA|86326|23|58|46|PM|third|evening|| +86327|AAAAAAAAIDBFBAAA|86327|23|58|47|PM|third|evening|| +86328|AAAAAAAAJDBFBAAA|86328|23|58|48|PM|third|evening|| +86329|AAAAAAAAKDBFBAAA|86329|23|58|49|PM|third|evening|| +86330|AAAAAAAALDBFBAAA|86330|23|58|50|PM|third|evening|| +86331|AAAAAAAAMDBFBAAA|86331|23|58|51|PM|third|evening|| +86332|AAAAAAAANDBFBAAA|86332|23|58|52|PM|third|evening|| +86333|AAAAAAAAODBFBAAA|86333|23|58|53|PM|third|evening|| +86334|AAAAAAAAPDBFBAAA|86334|23|58|54|PM|third|evening|| +86335|AAAAAAAAAEBFBAAA|86335|23|58|55|PM|third|evening|| +86336|AAAAAAAABEBFBAAA|86336|23|58|56|PM|third|evening|| +86337|AAAAAAAACEBFBAAA|86337|23|58|57|PM|third|evening|| +86338|AAAAAAAADEBFBAAA|86338|23|58|58|PM|third|evening|| +86339|AAAAAAAAEEBFBAAA|86339|23|58|59|PM|third|evening|| +86340|AAAAAAAAFEBFBAAA|86340|23|59|0|PM|third|evening|| +86341|AAAAAAAAGEBFBAAA|86341|23|59|1|PM|third|evening|| +86342|AAAAAAAAHEBFBAAA|86342|23|59|2|PM|third|evening|| +86343|AAAAAAAAIEBFBAAA|86343|23|59|3|PM|third|evening|| +86344|AAAAAAAAJEBFBAAA|86344|23|59|4|PM|third|evening|| +86345|AAAAAAAAKEBFBAAA|86345|23|59|5|PM|third|evening|| +86346|AAAAAAAALEBFBAAA|86346|23|59|6|PM|third|evening|| +86347|AAAAAAAAMEBFBAAA|86347|23|59|7|PM|third|evening|| +86348|AAAAAAAANEBFBAAA|86348|23|59|8|PM|third|evening|| +86349|AAAAAAAAOEBFBAAA|86349|23|59|9|PM|third|evening|| +86350|AAAAAAAAPEBFBAAA|86350|23|59|10|PM|third|evening|| +86351|AAAAAAAAAFBFBAAA|86351|23|59|11|PM|third|evening|| +86352|AAAAAAAABFBFBAAA|86352|23|59|12|PM|third|evening|| +86353|AAAAAAAACFBFBAAA|86353|23|59|13|PM|third|evening|| +86354|AAAAAAAADFBFBAAA|86354|23|59|14|PM|third|evening|| +86355|AAAAAAAAEFBFBAAA|86355|23|59|15|PM|third|evening|| +86356|AAAAAAAAFFBFBAAA|86356|23|59|16|PM|third|evening|| +86357|AAAAAAAAGFBFBAAA|86357|23|59|17|PM|third|evening|| +86358|AAAAAAAAHFBFBAAA|86358|23|59|18|PM|third|evening|| +86359|AAAAAAAAIFBFBAAA|86359|23|59|19|PM|third|evening|| +86360|AAAAAAAAJFBFBAAA|86360|23|59|20|PM|third|evening|| +86361|AAAAAAAAKFBFBAAA|86361|23|59|21|PM|third|evening|| +86362|AAAAAAAALFBFBAAA|86362|23|59|22|PM|third|evening|| +86363|AAAAAAAAMFBFBAAA|86363|23|59|23|PM|third|evening|| +86364|AAAAAAAANFBFBAAA|86364|23|59|24|PM|third|evening|| +86365|AAAAAAAAOFBFBAAA|86365|23|59|25|PM|third|evening|| +86366|AAAAAAAAPFBFBAAA|86366|23|59|26|PM|third|evening|| +86367|AAAAAAAAAGBFBAAA|86367|23|59|27|PM|third|evening|| +86368|AAAAAAAABGBFBAAA|86368|23|59|28|PM|third|evening|| +86369|AAAAAAAACGBFBAAA|86369|23|59|29|PM|third|evening|| +86370|AAAAAAAADGBFBAAA|86370|23|59|30|PM|third|evening|| +86371|AAAAAAAAEGBFBAAA|86371|23|59|31|PM|third|evening|| +86372|AAAAAAAAFGBFBAAA|86372|23|59|32|PM|third|evening|| +86373|AAAAAAAAGGBFBAAA|86373|23|59|33|PM|third|evening|| +86374|AAAAAAAAHGBFBAAA|86374|23|59|34|PM|third|evening|| +86375|AAAAAAAAIGBFBAAA|86375|23|59|35|PM|third|evening|| +86376|AAAAAAAAJGBFBAAA|86376|23|59|36|PM|third|evening|| +86377|AAAAAAAAKGBFBAAA|86377|23|59|37|PM|third|evening|| +86378|AAAAAAAALGBFBAAA|86378|23|59|38|PM|third|evening|| +86379|AAAAAAAAMGBFBAAA|86379|23|59|39|PM|third|evening|| +86380|AAAAAAAANGBFBAAA|86380|23|59|40|PM|third|evening|| +86381|AAAAAAAAOGBFBAAA|86381|23|59|41|PM|third|evening|| +86382|AAAAAAAAPGBFBAAA|86382|23|59|42|PM|third|evening|| +86383|AAAAAAAAAHBFBAAA|86383|23|59|43|PM|third|evening|| +86384|AAAAAAAABHBFBAAA|86384|23|59|44|PM|third|evening|| +86385|AAAAAAAACHBFBAAA|86385|23|59|45|PM|third|evening|| +86386|AAAAAAAADHBFBAAA|86386|23|59|46|PM|third|evening|| +86387|AAAAAAAAEHBFBAAA|86387|23|59|47|PM|third|evening|| +86388|AAAAAAAAFHBFBAAA|86388|23|59|48|PM|third|evening|| +86389|AAAAAAAAGHBFBAAA|86389|23|59|49|PM|third|evening|| +86390|AAAAAAAAHHBFBAAA|86390|23|59|50|PM|third|evening|| +86391|AAAAAAAAIHBFBAAA|86391|23|59|51|PM|third|evening|| +86392|AAAAAAAAJHBFBAAA|86392|23|59|52|PM|third|evening|| +86393|AAAAAAAAKHBFBAAA|86393|23|59|53|PM|third|evening|| +86394|AAAAAAAALHBFBAAA|86394|23|59|54|PM|third|evening|| +86395|AAAAAAAAMHBFBAAA|86395|23|59|55|PM|third|evening|| +86396|AAAAAAAANHBFBAAA|86396|23|59|56|PM|third|evening|| +86397|AAAAAAAAOHBFBAAA|86397|23|59|57|PM|third|evening|| +86398|AAAAAAAAPHBFBAAA|86398|23|59|58|PM|third|evening|| +86399|AAAAAAAAAIBFBAAA|86399|23|59|59|PM|third|evening|| diff --git a/spark-connector/spark-common/src/test/resources/data/tpcds/warehouse/warehouse.dat b/spark-connector/spark-common/src/test/resources/data/tpcds/warehouse/warehouse.dat new file mode 100644 index 00000000000..40a34febebb --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/data/tpcds/warehouse/warehouse.dat @@ -0,0 +1,5 @@ +1|AAAAAAAABAAAAAAA|Conventional childr|977787|651|6th |Parkway|Suite 470|Fairview|Williamson County|TN|35709|United States|-5| +2|AAAAAAAACAAAAAAA|Important issues liv|138504|600|View First|Avenue|Suite P|Fairview|Williamson County|TN|35709|United States|-5| +3|AAAAAAAADAAAAAAA|Doors canno|294242|534|Ash Laurel|Dr.|Suite 0|Fairview|Williamson County|TN|35709|United States|-5| +4|AAAAAAAAEAAAAAAA|Bad cards must make.|621234|368|Wilson Elm|Drive|Suite 80|Fairview|Williamson County|TN|35709|United States|-5| +5|AAAAAAAAFAAAAAAA|||||||Fairview|Williamson County|TN|35709|United States|| diff --git a/spark-connector/spark-common/src/test/resources/data/tpcds/web_page/web_page.dat b/spark-connector/spark-common/src/test/resources/data/tpcds/web_page/web_page.dat new file mode 100644 index 00000000000..c180402911b --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/data/tpcds/web_page/web_page.dat @@ -0,0 +1,60 @@ +1|AAAAAAAABAAAAAAA|1997-09-03||2450810|2452620|Y|98539|http://www.foo.com|welcome|2531|8|3|4| +2|AAAAAAAACAAAAAAA|1997-09-03|2000-09-02|2450814|2452580|N||http://www.foo.com|protected|1564|4|3|1| +3|AAAAAAAACAAAAAAA|2000-09-03||2450814|2452611|N||http://www.foo.com|feedback|1564|4|3|4| +4|AAAAAAAAEAAAAAAA|1997-09-03|1999-09-03|2450812|2452579|N||http://www.foo.com|general|3732|18|7|1| +5|AAAAAAAAEAAAAAAA|1999-09-04|2001-09-02|2450812|2452597|N||http://www.foo.com|welcome|3732|18|3|1| +6|AAAAAAAAEAAAAAAA|2001-09-03||2450814|2452597|N||http://www.foo.com|ad|3732|18|7|4| +7|AAAAAAAAHAAAAAAA|1997-09-03||2450815|2452574|N||http://www.foo.com|feedback|3034|18|7|4| +8|AAAAAAAAIAAAAAAA|1997-09-03|2000-09-02|2450815|2452646|Y|1898|http://www.foo.com|protected|3128|12|2|4| +9|AAAAAAAAIAAAAAAA|2000-09-03||2450807|2452579|Y|84146|http://www.foo.com|welcome|3128|13|5|3| +10|AAAAAAAAKAAAAAAA|1997-09-03|1999-09-03||2452623|N||http://www.foo.com|||||| +11|AAAAAAAAKAAAAAAA|1999-09-04|2001-09-02|2450814|2452611|N||http://www.foo.com|welcome|7046|23|4|4| +12|AAAAAAAAKAAAAAAA|2001-09-03||2450815|2452611|N||http://www.foo.com|protected|7046|17|4|4| +13|AAAAAAAANAAAAAAA|1997-09-03||2450807|2452629|N||http://www.foo.com|protected|2281|6|4|1| +14|AAAAAAAAOAAAAAAA|1997-09-03|2000-09-02|2450810|2452639|N||http://www.foo.com|dynamic|5676|19|6|0| +15|AAAAAAAAOAAAAAAA|2000-09-03||2450810|2452639|N||http://www.foo.com|dynamic|2469|10|5|2| +16|AAAAAAAAABAAAAAA|1997-09-03|1999-09-03|2450814|2452601|Y|33463|http://www.foo.com|feedback|701|2|1|4| +17|AAAAAAAAABAAAAAA|1999-09-04|2001-09-02|2450812|2452645|N||http://www.foo.com|general|701|11|1|3| +18|AAAAAAAAABAAAAAA|2001-09-03||2450812|2452608|N||http://www.foo.com|ad|4080|11|6|3| +19|AAAAAAAADBAAAAAA|1997-09-03||2450808|2452648|Y|57610|http://www.foo.com|general|2347|9|7|4| +20|AAAAAAAAEBAAAAAA|1997-09-03|2000-09-02|2450809|2452555|Y|46487|http://www.foo.com|ad|1147|3|6|0| +21|AAAAAAAAEBAAAAAA|2000-09-03||2450809|2452555|Y|10897|http://www.foo.com|general|1147|3|6|4| +22|AAAAAAAAGBAAAAAA|1997-09-03|1999-09-03|2450812|2452565|Y|20213|http://www.foo.com|general|5663|25|3|4| +23|AAAAAAAAGBAAAAAA|1999-09-04|2001-09-02|2450812|2452623|Y|20213|http://www.foo.com|order|4729|23|6|4| +24|AAAAAAAAGBAAAAAA|2001-09-03||2450812|2452646|Y|20213|http://www.foo.com|dynamic|5918|23|6|1| +25|AAAAAAAAJBAAAAAA|1997-09-03||2450811|2452620|N||http://www.foo.com|feedback|1526|9|4|2| +26|AAAAAAAAKBAAAAAA|1997-09-03|2000-09-02|2450812|2452636|Y|98376|http://www.foo.com|ad|1826|9|3|1| +27|AAAAAAAAKBAAAAAA|2000-09-03||2450812|2452607|Y|98376|http://www.foo.com|protected|1553|9|1|1| +28|AAAAAAAAMBAAAAAA|1997-09-03|1999-09-03|2450807|2452572|N||http://www.foo.com|protected|1308|4|1|2| +29|AAAAAAAAMBAAAAAA|1999-09-04|2001-09-02|2450808|2452611|N||http://www.foo.com|order|1308|4|1|2| +30|AAAAAAAAMBAAAAAA|2001-09-03||2450808|2452611|N||http://www.foo.com|general|3872|18|1|4| +31|AAAAAAAAPBAAAAAA|1997-09-03||2450810|2452596|N||http://www.foo.com|general|1732|3|6|0| +32|AAAAAAAAACAAAAAA|1997-09-03|2000-09-02|2450808|2452585|N||http://www.foo.com|welcome|5104|20|7|4| +33|AAAAAAAAACAAAAAA|2000-09-03||2450808|2452585|N||http://www.foo.com|protected|2129|7|1|0| +34|AAAAAAAACCAAAAAA|1997-09-03|1999-09-03|2450808|2452616|N||http://www.foo.com|welcome|2726|12|5|2| +35|AAAAAAAACCAAAAAA|1999-09-04|2001-09-02|2450808|2452591|N||http://www.foo.com|protected|2726|12|1|2| +36|AAAAAAAACCAAAAAA|2001-09-03||2450812|2452613|N||http://www.foo.com|dynamic|2726|3|1|2| +37|AAAAAAAAFCAAAAAA|1997-09-03||2450809|2452556|N||http://www.foo.com|ad|3076|15|3|0| +38|AAAAAAAAGCAAAAAA|1997-09-03|2000-09-02|2450811|2452583|Y|37285|http://www.foo.com|general|3096|18|3|0| +39|AAAAAAAAGCAAAAAA|2000-09-03||2450815|2452583|N||http://www.foo.com|general|3096|18|3|0| +40|AAAAAAAAICAAAAAA|1997-09-03|1999-09-03|2450813|2452576|N||http://www.foo.com|general|4402|18|4|2| +41|AAAAAAAAICAAAAAA|1999-09-04|2001-09-02|2450813|2452579|Y|16769|http://www.foo.com|welcome|784|3|4|4| +42|AAAAAAAAICAAAAAA|2001-09-03||2450813|2452579|Y|60150|http://www.foo.com|dynamic|1451|3|4|4| +43|AAAAAAAALCAAAAAA|1997-09-03||2450814|2452580|Y|64793|http://www.foo.com|ad|3760|12|3|2| +44|AAAAAAAAMCAAAAAA|1997-09-03|2000-09-02|2450811|2452602|Y|92078|http://www.foo.com|ad|4179|19|7|1| +45|AAAAAAAAMCAAAAAA|2000-09-03||2450811|2452575|Y|98633|http://www.foo.com|feedback|4584|19|7|4| +46|AAAAAAAAOCAAAAAA|1997-09-03|1999-09-03|2450809|2452574|N||http://www.foo.com|protected|1711|4|5|1| +47|AAAAAAAAOCAAAAAA|1999-09-04|2001-09-02|2450815|2452574|N||http://www.foo.com|welcome|1711|4|5|1| +48|AAAAAAAAOCAAAAAA|2001-09-03||2450815|2452622|N||http://www.foo.com|ad|1732|9|5|1| +49|AAAAAAAABDAAAAAA|1997-09-03||2450809|2452618|N||http://www.foo.com|order|4894|20|3|2| +50|AAAAAAAACDAAAAAA|1997-09-03|2000-09-02|2450808|2452615|N||http://www.foo.com|welcome|5262|16|5|2| +51|AAAAAAAACDAAAAAA|2000-09-03||2450811|2452564|N||http://www.foo.com|general|3423|19|7|1| +52|AAAAAAAAEDAAAAAA|1997-09-03|1999-09-03|2450815|2452606|N||http://www.foo.com|welcome|3306|21|7|1| +53|AAAAAAAAEDAAAAAA|1999-09-04|2001-09-02|2450808|2452636|N||http://www.foo.com|dynamic|3306|21|7|1| +54|AAAAAAAAEDAAAAAA|2001-09-03||2450808|2452629|N||http://www.foo.com|protected|1931|7|2|2| +55|AAAAAAAAHDAAAAAA|1997-09-03||2450811|2452549|N||http://www.foo.com|order|3788|19|1|0| +56|AAAAAAAAIDAAAAAA|1997-09-03|2000-09-02|2450815|2452554|N||http://www.foo.com|protected|5733|24|2|2| +57|AAAAAAAAIDAAAAAA|2000-09-03||2450811|2452568|N||http://www.foo.com|ad|5733|16|2|2| +58|AAAAAAAAKDAAAAAA|1997-09-03|1999-09-03|2450813|2452619|Y|7625|http://www.foo.com|ad|6577|24|4|3| +59|AAAAAAAAKDAAAAAA|1999-09-04|2001-09-02|2450813|2452624|Y|80555|http://www.foo.com|general|6577|24|2|3| +60|AAAAAAAAKDAAAAAA|2001-09-03||2450813|2452566|Y|80555|http://www.foo.com|welcome|6577|24|2|3| diff --git a/spark-connector/spark-common/src/test/resources/data/tpcds/web_returns/web_returns.dat b/spark-connector/spark-common/src/test/resources/data/tpcds/web_returns/web_returns.dat new file mode 100644 index 00000000000..080eef05cfe --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/data/tpcds/web_returns/web_returns.dat @@ -0,0 +1,7176 @@ +2451653|7022|10402|46224|1011635|3446|4057|46224|1011635|3446|4057|56|23|1|10|698.20|13.96|712.16|18.63|820.30|300.22|382.06|15.92|852.89| +2451627|64915|15464|3811|18405|199|48793|3811|18405|199|48793|13|9|1|47|1248.79|49.95|1298.74|61.81|709.23|262.24|128.25|858.30|820.99| +2452798||9559||31639||18790||31639|2038|18790||11|2|11||25.52|||16.72||16.36|165.47|| +2452243|23325|5559|47687|663037|1234|26984|47687|663037|1234|26984|29|26|5|33|1820.94|54.62|1875.56|70.16|1000.23|1802.73|14.56|3.65|1125.01| +2452305|73981|7665|92552|1104998|2244|19090|92552|1104998|2244|19090|49|1|5|18|82.08|7.38|89.46|35.63|1342.26|38.57|19.14|24.37|1385.27| +2451288|74891|7555|36650|1309189|128|15420|36650|1309189|128|15420|1|17|7|16|1956.00|156.48|2112.48|22.56|221.76|215.16|504.84|1236.00|400.80| +2451337|44242|4856|75377|1291615|653|16891|75377|1291615|653|16891|16|24|7|3|134.40|0.00|134.40|78.06|108.78|108.86|15.06|10.48|186.84| +2451278|48347|1225|68887|1781163|3486|35762|68887|1781163|3486|35762|2|20|7|43|520.30|20.81|541.11|54.02|3468.38|504.69|14.36|1.25|3543.21| +2452098|72741|6146|27559|1010209|2812|43415|27559|1010209|2812|43415|1|15|10|4|79.68|2.39|82.07|11.40|54.52|58.16|1.50|20.02|68.31| +2452065|46137|16559|26204|217410|1416|28222|26204|217410|1416|28222|13|32|11|28|707.00|28.28|735.28|16.84|382.20|424.20|248.86|33.94|427.32| +2451936|25850|8285|31623|1546796|816|20389|31623|1546796|816|20389|5|15|11|6|708.66|21.25|729.91|25.19|146.10|474.80|144.99|88.87|192.54| +2452508|71077|97|30010|1589848|6933|36371|30010|1589848|6933|36371|21|11|12|26|528.84|37.01|565.85|90.97|237.12|449.51|23.00|56.33|365.10| +2451590|30553|13172|28036|1376937|2285|11586|28036|1376937|2285|11586|13|16|13|16|417.60|25.05|442.65|22.13|539.36|50.11|128.62|238.87|586.54| +2452495||9797|19758||2243||19758|1696824|||49||14|4||||11.84||9.39|||| +2452566|67292|12055|24337|970065|593|4321|24337|970065|593|4321|17|2|14|5|219.85|15.38|235.23|28.94|233.90|147.29|42.81|29.75|278.22| +||4933|||2358||14932|418645|||||15|24||||54.71|62.40||129.49|60.94|143.37| +2452567|26030|5107|62543|1118206|6669|32732|62543|1118206|6669|32732|42|25|16|40|1479.60|103.57|1583.17|4.99|856.80|665.82|219.72|594.06|965.36| +2452569|29129|5703|888|217580|5919|2658|888|217580|5919|2658|30|13|17|10|69.00|6.21|75.21|60.35|19.70|31.05|11.38|26.57|86.26| +2451692|37442|14911|74623|1176118|1613|27108|74623|1176118|1613|27108|40|15|18|11|1628.66|32.57|1661.23|13.11|506.22|1237.78|355.70|35.18|551.90| +2451495|70505|8533|87084|1360384|4437|14537|87084|1360384|4437|14537|16|20|18|23|447.58|17.90|465.48|30.56|302.22|331.20|51.20|65.18|350.68| +2451396|74839|2288|||||26767||3439|27775||33|22|||0.00|||75.90|485.00|||| +2451271|72837|10759|35141|718131|2368|35405|35141|718131|2368|35405|22|12|22|30|247.20|9.88|257.08|23.57|717.30|229.89|15.05|2.26|750.75| +2451430||7130|78764||||78764|1060754|5497|||27|22|79||0.00|0.00|92.68|||0.00|0.00|| +2452236|23926|13961|30618|8492|4766|9440|30618|8492|4766|9440|41|17|24|9|107.46|3.22|110.68|54.67|33.03|33.31|68.21|5.94|90.92| +2452273|38436|13283|29071|1421648|5454|5396|29071|1421648|5454|5396|49|18|24|8|252.08|15.12|267.20|62.43|12.72|133.60|29.62|88.86|90.27| +2451372|50248|1042|30691|813091|5891|17091|30691|813091|5891|17091|49|21|26|77|1088.78|43.55|1132.33|1.05|3811.50|163.31|397.95|527.52|3856.10| +2451514|46416|15325|90970|854030|2690|2550|90970|854030|2690|2550|1|13|26|10|362.50|3.62|366.12|68.99|164.80|290.00|60.17|12.33|237.41| +2451529|45968|5942|943|58420|2297|21421|943|58420|2297|21421|13|23|26|35|340.20|13.60|353.80|50.62|453.60|336.79|2.25|1.16|517.82| +2450982|37442|12086|74331|1565742|5600|22258|74331|1565742|5600|22258|56|32|27|4|19.20|1.72|20.92|63.20|41.36|4.60|10.22|4.38|106.28| +2450976|42382|12544|63968|169725|1529|23286|63968|169725|1529|23286|8|6|27|11|380.49|15.21|395.70|76.70|77.66|144.58|11.79|224.12|169.57| +2452492|36787|13269|12839|1060269|5299|15082|12839|1060269|5299|15082|25|5|29|11|400.84|20.04|420.88|16.27|99.11|308.64|18.44|73.76|135.42| +2451423|75368|4009|64553|1687614|2228|7289|64553|1687614|2228|7289|28|26|30|19|168.15|3.36|171.51|94.69|107.54|117.70|42.88|7.57|205.59| +2451476|48229|3460|91962|12631|2900|22369|91962|12631|2900|22369|8|20|30|5|18.20|0.54|18.74|11.66|8.25|2.54|5.79|9.87|20.45| +|43497|3046||1006429|3340|26251|90318|1006429|3340||||32|7|470.26||484.36|99.14||470.26|0.00||| +2451486|253|15937|15754|998787|3867|37347|15754|998787|3867|37347|4|13|32|2|98.40|8.85|107.25|16.33|131.22|51.16|27.87|19.37|156.40| +2451420|61938|11521|25841|1808855|5916|9330|25841|1808855|5916|9330|56|22|32|44|4496.36|314.74|4811.10|5.38|1570.80|4316.50|151.08|28.78|1890.92| +2451622|68694|16490|46758|1477303|5095|8440|46758|1477303|5095|8440|44|9|33|5|1231.20|24.62|1255.82|13.24|580.20|295.48|823.43|112.29|618.06| +2451734|55182|17581|2341|914970|5999|11428|2341|914970|5999|11428|20|17|33|7|88.34|5.30|93.64|24.45|109.27|6.18|6.57|75.59|139.02| +2451697|47575|14282|8237|1089705|4773|31841|8237|1089705|4773|31841|37|8|33|3|164.85|1.64|166.49|3.49|57.06|6.59|137.68|20.58|62.19| +2451615|73881|7993|37870|705124|5978|28032|37870|705124|5978|28032|10|1|34|32|4588.16|183.52|4771.68|51.92|2136.96|917.63|3523.70|146.83|2372.40| +2452120|67792|791|15039|1600609|4132|6502|15039|1600609|4132|6502|19|14|36|16|496.96|34.78|531.74|71.96|773.12|134.17|261.20|101.59|879.86| +2452135|32975|12139|68233|680344|3655|13766|68233|680344|3655|13766|3|19|36|33|1560.57|0.00|1560.57|73.33|845.46|1201.63|326.63|32.31|918.79| +2452038|60285|13993|96096|1039516|6936|47109|96096|1039516|6936|47109|15|2|36|91|15813.07|632.52|16445.59|65.73|8008.91|13441.10|1352.02|1019.95|8707.16| +2451873|61736|3050|62984|1914824|7017|47257|62984|1914824|7017|47257|19|13|38|54|2243.70|179.49|2423.19|60.32|1075.68|628.23|1373.14|242.33|1315.49| +2451812|17353|8467|15013|691848|1836|26624|15013|691848|1836|26624|13|33|38|11|15.18|0.00|15.18|42.63|5.50|9.71|1.53|3.94|48.13| +2452054|39091|9725|17582|1892903|2624|22392|17582|1892903|2624|22392|13|27|39|27|3078.54|30.78|3109.32|79.71|1835.19|1015.91|453.77|1608.86|1945.68| +2452413|27263|12283|8730|781278|1291|27963|8730|781278|1291|27963|45|2|40|13|1286.22|51.44|1337.66|97.11|95.16|424.45|17.23|844.54|243.71| +2452476|28293|16639|15777|803717|3657|25150|15777|803717|3657|25150|43|35|40|9|103.32|1.03|104.35|31.68|2.79|76.45|7.25|19.62|35.50| +2451905|55597|13928|11775|962709|43|8624|11775|962709|43|8624|11|20|41|56|4452.00|311.64|4763.64|63.92|3368.96|1958.88|797.79|1695.33|3744.52| +2451845|42388|16676|39546|1391011|5229|1847|39546|1391011|5229|1847|53|29|41|29|1387.36|69.36|1456.72|14.34|818.67|194.23|608.49|584.64|902.37| +2451261|73439|16081||||6964||||6964|25||42|||||58.60|192.76|80.02|54.48|39.46|| +2452078|22372|551|34386|435559|2685|28258|34386|435559|2685|28258|7|12|43|58|4257.20|85.14|4342.34|99.74|1530.04|2469.17|1609.22|178.81|1714.92| +2451335|79473|139|96442|1432457|2425|22840|96442|1432457|2425|22840|8|29|44|13|45.63|3.65|49.28|49.66|0.00|39.24|1.53|4.86|53.31| +2451441|11487|8912|65405|1361990|4975|22407|65405|1361990|4975|22407|20|1|45|43|4438.89|133.16|4572.05|83.51|466.98|532.66|2187.48|1718.75|683.65| +2452105|59457|1791|45913|1423984|4102|39373|45913|1423984|4102|39373|7|30|46|61|704.55|49.31|753.86|33.45|3012.79|295.91|302.39|106.25|3095.55| +2451677|33194|9268|78918|1366834|2750|13819|78918|1366834|2750|13819|38|23|48|9|216.81|17.34|234.15|19.81|45.63|21.68|35.12|160.01|82.78| +2452561|35610|13365|53081|788648|5815|34582|53081|788648|5815|34582|27|29|49|79|10628.66|744.00|11372.66|88.50|2361.31|8502.92|1658.07|467.67|3193.81| +2452060|60418|8915|35908|793212|2516|38620|35908|793212|2516|38620|13|32|51|9|355.32|10.65|365.97|99.51|193.77|309.12|6.00|40.20|303.93| +2452077|13035|3950|57668|1078154|4240|44373|57668|1078154|4240|44373|25|33|51|58|6574.88|460.24|7035.12|57.17|3221.32|1380.72|4051.44|1142.72|3738.73| +|71745|5139|||2111|28323|||2111||37|8|52||765.16||834.02|32.04|229.40||215.16|351.06|330.30| +2451900|32899|1417|80971|1300813|208|13250|80971|1300813|208|13250|32|28|53|2|97.88|1.95|99.83|22.48|13.64|80.26|5.46|12.16|38.07| +2451958|26993|5285|65631|1375925|1419|42466|65631|1375925|1419|42466|29|35|53|13|1368.77|82.12|1450.89|88.01|727.09|191.62|235.43|941.72|897.22| +||16082|||6738|12460||1500216|||53|20|55|7|70.70|4.94||38.47||42.42|4.24||71.20| +||2203|||||61199|||21007|25|35|56||1717.20|154.54|1871.74|3.62|356.40||||| +2451384|77186|7999|58380|1708805|5983|5608|58380|1708805|5983|5608|34|23|57|19|2002.03|120.12|2122.15|71.12|302.10|1301.31|119.12|581.60|493.34| +2451443|45709|2144|93800|1777124|1248|16619|93800|1777124|1248|16619|20|2|58|9|740.34|22.21|762.55|37.92|174.78|222.10|67.37|450.87|234.91| +2451759|67735|14096|20138|821188|5885|40135|20138|821188|5885|40135|38|31|59|26|789.36|71.04|860.40|6.20|134.94|149.97|421.99|217.40|212.18| +2451560|59013|1702|23883|539803|6923|38843|23883|539803|6923|38843|13|25|63|31|191.58|3.83|195.41|1.63|739.35|42.14|127.02|22.42|744.81| +||14062|40670|103015||||103015|2160|33705|||64|36|1086.12|32.58|1118.70|4.75||||543.61|| +2451547|32382|17270|28101|99551|3001|15251|28101|99551|3001|15251|26|29|66|7|368.97|0.00|368.97|48.34|12.81|99.62|37.70|231.65|61.15| +2452027|26463|17030|33973|277684|1748|22259|33973|277684|1748|22259|32|34|69|57|5985.57|359.13|6344.70|89.65|1056.21|2394.22|2944.90|646.45|1504.99| +2452341|68091|6577|20911|1511560|5409|5658|20911|1511560|5409|5658|51|18|72|20|3750.00|187.50|3937.50|62.87|742.20|187.50|1959.37|1603.13|992.57| +2452196|78918|7013|27224|1559926|6228|18130|27224|1559926|6228|18130|57|32|72|2|72.30|4.33|76.63|96.96|24.44|65.79|5.46|1.05|125.73| +2452538|5244|15671|42195|94659|276|37493|42195|94659|276|37493|29|11|73|4|208.84|0.00|208.84|90.81|202.68|154.54|19.00|35.30|293.49| +2452648|69373|1533|56326|571505|5084|34663|56326|571505|5084|34663|57|4|75|35|2668.05|80.04|2748.09|62.33|361.20|1414.06|463.97|790.02|503.57| +2452670|41829|5143|72961|4573|4185|3397|72961|4573|4185|3397|55|17|75|23|1807.80|0.00|1807.80|14.00|766.36|379.63|1142.53|285.64|780.36| +2451916|34658|10039|92828|990174|2720|23666|92828|990174|2720|23666|19|17|76|72|2581.92|77.45|2659.37|20.09|5286.96|568.02|2013.90|0.00|5384.50| +2451616|51966|14284|64626|1703073|4720|47567|64626|1703073|4720|47567|43|24|77|8|25.28|0.75|26.03|61.93|0.00|11.12|12.17|1.99|62.68| +||7711|59099||6659||59099|1634079|||||77||1934.50|||45.51|||||838.39| +2451617|28361|8932|56939|1603349|4959|17369|56939|1603349|4959|17369|7|1|79|21|919.59|18.39|937.98|21.38|509.04|478.18|8.82|432.59|548.81| +2451871|54886|12658|11537|1097050|610|24523|11537|1097050|610|24523|8|27|79|33|606.87|36.41|643.28|32.97|290.73|54.61|276.13|276.13|360.11| +2451726|26757|10174|7212|1794817|4822|49674|7212|1794817|4822|49674|19|33|81|17|371.28|29.70|400.98|28.14|85.68|226.48|50.68|94.12|143.52| +2452486|46513|14904|54862|897323|1384|32245|54862|897323|1384|32245|21|32|83|20|104.00|0.00|104.00|21.18|333.00|86.32|9.19|8.49|354.18| +2452557|65926|1311|92087|1208787|7087|11034|92087|1208787|7087|11034|43|3|83|29|300.15|0.00|300.15|44.01|2552.00|168.08|109.61|22.46|2596.01| +2451280|25318|5443|20396|1857247|3251|9702|20396|1857247|3251|9702|19|34|84|2|96.36|3.85|100.21|85.73|33.88|30.83|43.90|21.63|123.46| +2451393|62509|16586|93660|1078409|3158|46707|93660|1078409|3158|46707|55|26|84|18|265.50|2.65|268.15|49.10|165.24|45.13|103.57|116.80|216.99| +2452735||6987|95006|533478|2797||95006|533478||469|||85||20.20||||15.50|19.59|||102.28| +2452663|51913|7290|95493|1729053|5813|973|95493|1729053|5813|973|60|13|85|34|567.80|5.67|573.47|72.34|1817.30|22.71|196.23|348.86|1895.31| +2451723|72393|16273|23324|159786|601|4216|23324|159786|601|4216|52|13|86|16|56.48|0.00|56.48|78.57|53.60|46.31|2.64|7.53|132.17| +2451599|60380|3092|96909|361910|864|41065|96909|361910|864|41065|43|10|86|22|3938.88|157.55|4096.43|78.70|1140.04|590.83|1941.86|1406.19|1376.29| +2452181|69772|3371|40789|1625335|281|18027|40789|1625335|281|18027|5|3|87|11|1976.59|39.53|2016.12|74.21|951.61|316.25|1278.46|381.88|1065.35| +|67250|7460||254698||26424||254698||||34|91|||89.14|1079.59|||||2.98|| +2452382|45230|7749|2884|1353432|7180|19328|2884|1353432|7180|19328|23|14|92|15|54.30|1.62|55.92|48.55|37.95|53.21|1.07|0.02|88.12| +2452555|53480|9885|43174|1417259|6897|4535|43174|1417259|6897|4535|33|2|92|18|1303.56|117.32|1420.88|71.91|477.90|404.10|638.61|260.85|667.13| +2452790|34442|13597|49372|463802|4985|9815|49372|463802|4985|9815|27|29|93|21|1923.81|115.42|2039.23|57.75|72.03|1365.90|530.01|27.90|245.20| +2452775|70628|7113|51176|1018443|4910|49377|51176|1018443|4910|49377|51|30|93|88|11567.60|115.67|11683.27|84.79|5192.88|4280.01|1967.64|5319.95|5393.34| +2452056|77444|4185|29134|701863|3839|19768|29134|701863|3839|19768|19|23|95|11|1553.64|124.29|1677.93|50.63|493.46|450.55|573.60|529.49|668.38| +||4231||253586|7137|7935|2788|253586||7935||12|96|||||||175.30|10.67||| +2452085|44364|397|86385|1635385|3585|19385|86385|1635385|3585|19385|47|3|98|44|96.80|6.77|103.57|87.44|549.56|90.02|2.23|4.55|643.77| +2452151|61395|9447|11425|1792938|51|27517|11425|1792938|51|27517|7|25|98|41|1643.69|0.00|1643.69|42.33|182.45|1380.69|163.06|99.94|224.78| +2451635|79446|1328|75662|1085284|2553|29822|75662|1085284|2553|29822|22|14|100|22|2492.60|49.85|2542.45|6.19|1077.78|2093.78|327.03|71.79|1133.82| +2452628|60416|5989|68084|235070|5256|10242|68084|235070|5256|10242|33|14|101|15|755.10|15.10|770.20|58.98|485.40|407.75|93.78|253.57|559.48| +2451386|56182|2684|53523|1341309|1542|13328|53523|1341309|1542|13328|56|10|102|63|793.17|7.93|801.10|63.89|335.79|0.00|682.12|111.05|407.61| +2451461|12953|8336|47572|281063|507|19998|47572|281063|507|19998|55|31|102|21|1251.39|12.51|1263.90|68.51|235.62|838.43|111.49|301.47|316.64| +2451410|46409|16840|44818|1524677|983|2842|44818|1524677|983|2842|28|19|102|5|87.15|4.35|91.50|63.38|115.40|34.86|4.18|48.11|183.13| +2452031|56843|15851|2635|1719760|2038|3116|2635|1719760|2038|3116|31|9|103|4|25.28|0.75|26.03|51.35|32.20|20.72|1.04|3.52|84.30| +2451630|36986|1975|74401|1501553|5944|47518|74401|1501553|5944|47518|19|17|104|27|524.88|36.74|561.62|98.42|378.00|47.23|367.79|109.86|513.16| +2452965|64055|12030|87174|1327898|3822|31393|87174|1327898|3822|31393|54|2|105|41|380.48|30.43|410.91|54.41|309.14|327.21|11.18|42.09|393.98| +2451745|51810|4561|47890|647802|2914|12826|47890|647802|2914|12826|4|30|106|57|10771.86|107.71|10879.57|64.16|4598.76|538.59|1534.99|8698.28|4770.63| +2451857|74574|13357|83613|599090|520|42750|83613|599090|520|42750|49|4|106|2|81.78|1.63|83.41|89.31|12.96|46.61|0.35|34.82|103.90| +2452699|75135|17131|25765|901970|5728|2686|25765|901970|5728|2686|36|11|107|59|5709.43|285.47|5994.90|78.92|2602.49|57.09|3504.45|2147.89|2966.88| +2451357|57768|16429|70181|1262444|6354|19017|70181|1262444|6354|19017|56|1|108|19|857.85|51.47|909.32|6.38|354.92|797.80|4.20|55.85|412.77| +2451900|34630|17033|42647|426674|1901|5528|42647|426674|1901|5528|44|25|109|43|1506.72|75.33|1582.05|76.34|527.18|964.30|59.66|482.76|678.85| +2451985|39519|5587|90602|329148|6094|9393|90602|329148|6094|9393|56|22|109|2|332.68|19.96|352.64|4.54|117.62|36.59|210.22|85.87|142.12| +2451981|44094|968|1585|1022397|1196|2168|1585|1022397|1196|2168|50|35|110|8|398.56|11.95|410.51|56.87|75.68|63.76|56.91|277.89|144.50| +2452375|28890|1219|88083|71791|3499|17207|88083|71791|3499|17207|21|28|111|4|174.72|12.23|186.95|12.80|88.96|12.23|30.87|131.62|113.99| +2452389|41006|12715|79850|1847316|4829|26295|79850|1847316|4829|26295|19|20|111|28|1792.84|107.57|1900.41|34.60|227.64|1093.63|370.58|328.63|369.81| +2451203|37455|11576|97225|1020916|1054|39545|97225|1020916|1054|39545|49|4|112|67|4822.66|96.45|4919.11|87.63|518.58|4629.75|98.38|94.53|702.66| +2452452|62802|12413|2765|3012|59|33506|2765|3012|59|33506|21|6|114|3|195.09|15.60|210.69|93.27|144.93|115.10|44.79|35.20|253.80| +2451731|2553|15752|96745|29063|3781|18499|96745|29063|3781|18499|53|4|116|5|1100.85|44.03|1144.88|97.23|671.25|55.04|805.27|240.54|812.51| +2451657|73389|769|22735|1504060|2538|663|22735|1504060|2538|663|11|10|116|71|1970.96|177.38|2148.34|3.99|1504.49|39.41|1719.07|212.48|1685.86| +2451501|65646|17455|92593|1190903|6813|30723|92593|1190903|6813|30723|31|1|117|8|149.20|10.44|159.64|31.75|188.16|56.69|28.67|63.84|230.35| +2452689|46127|11179|18481||1712|14704|18481|877073|1712||6||119||32.30|0.96|33.26|63.11||10.33|||80.22| +2451281|64869|2194|71155|351456|5357|11|71155|351456|5357|11|32|10|122|21|70.56|5.64|76.20|34.23|6.72|26.10|28.89|15.57|46.59| +2451304|32760|12247|89139|676463|2187|24758|89139|676463|2187|24758|8|29|122|53|2937.26|234.98|3172.24|70.37|2264.16|734.31|859.15|1343.80|2569.51| +2452716|26880|16521|98310|730718|1526|44334|98310|730718|1526|44334|43|3|123|37|6641.13|265.64|6906.77|91.31|3760.68|5777.78|707.94|155.41|4117.63| +2452513|78127|3675|72190|211339|88|39637|72190|211339|88|39637|19|10|123|51|1797.24|71.88|1869.12|57.79|2331.72|323.50|176.84|1296.90|2461.39| +2452628|40092|17239|86362|1411621|6127|9786|86362|1411621|6127|9786|42|7|123|71|884.66|35.38|920.04|23.51|1020.27|460.02|254.78|169.86|1079.16| +2451727|44661|8252|62890|1674763|6636|37309|62890|1674763|6636|37309|14|20|124|73|3682.12|73.64|3755.76|39.06|78.11|1509.66|1194.85|977.61|190.81| +2451824|25490|15266|54506|1277298|2490|21682|54506|1277298|2490|21682|14|32|124|42|5403.72|432.29|5836.01|89.28|2909.76|2701.86|270.18|2431.68|3431.33| +2452813|52198|17040|87082|572217|905|23640|87082|572217|905|23640|36|12|125|34|934.66|0.00|934.66|92.07|272.34|542.10|369.00|23.56|364.41| +2452726|29100|6513|68020|1128323|2226|34482|68020|1128323|2226|34482|55|3|125|2|498.76|34.91|533.67|98.25|157.50|194.51|282.95|21.30|290.66| +2451726|58130|5323|9712|181928|544|3913|9712|181928|544|3913|25|19|126|30|378.30|22.69|400.99|33.27|0.00|139.97|47.66|190.67|55.96| +2451592|17279|5071|96467|381153|1039|8878|96467|381153|1039|8878|50|35|126|12|1138.80|56.94|1195.74|43.34|618.96|694.66|253.15|190.99|719.24| +2452636|51270|7380|20891|1145852|6366|42480|20891|1145852|6366|42480|24|21|127|39|2588.82|232.99|2821.81|2.21|0.00|362.43|868.29|1358.10|235.20| +2451812|74469|17528|17374|1283053|732|18011|17374|1283053|732|18011|25|10|128|36|1954.44|78.17|2032.61|65.24|231.12|1035.85|257.20|661.39|374.53| +2451671|61435|9262|58531|1841383|3882|31934|58531|1841383|3882|31934|28|5|128|14|687.68|61.89|749.57|42.00|547.26|453.86|35.07|198.75|651.15| +2451809|58455|6772|50350|238523|1896|9622|50350|238523|1896|9622|22|21|128|5|157.50|4.72|162.22|43.52|41.25|137.02|1.02|19.46|89.49| +2451270|43952|4801|31763|1514954|1345|34136|31763|1514954|1345|34136|55|1|129|5|354.65|21.27|375.92|44.77|73.15|70.93|243.99|39.73|139.19| +2451763|67846|17011|79328|74724|5673|31469|79328|74724|5673|31469|2|6|130|38|3191.62|95.74|3287.36|39.89|1914.82|861.73|232.98|2096.91|2050.45| +2451315|33516|9865|93564|1570320|5923|31079|93564|1570320|5923|31079|25|33|133|1|14.14|0.56|14.70|64.61|3.53|13.71|0.31|0.12|68.70| +2451359|46100|524|36495|123214|5580|29546|36495|123214|5580|29546|56|3|133|5|334.25|10.02|344.27|4.24|250.70|307.51|5.88|20.86|264.96| +2452773|45115|16983|83715|1211771|3027|9530|83715|1211771|3027|9530|18|5|134|7|763.91|68.75|832.66|78.99|109.13|122.22|532.60|109.09|256.87| +2452668|47742|14835|64630|1364810|1437|49264|64630|1364810|1437|49264|18|17|134|18|457.56|32.02|489.58|10.53|152.46|430.10|10.70|16.76|195.01| +2452577|23222|10519|42680|1788617|2554|891|42680|1788617|2554|891|15|33|134|15|648.60|38.91|687.51|89.01|157.95|512.39|25.87|110.34|285.87| +2452712|31502|3597|33664|1128366|5915|15064|33664|1128366|5915|15064|15|24|134|27|184.14|12.88|197.02|13.50|115.02|160.20|1.43|22.51|141.40| +2452783||12363||1721964|||24767||3961|33158|57|30|134||211.32|2.11||14.06|||||| +2451800|44907|4166|4444|1203328|5765|8649|4444|1203328|5765|8649|43|11|135|76|7185.04|143.70|7328.74|50.91|1771.56|3233.26|2568.65|1383.13|1966.17| +2452297|67338|1461|37879|1531939|6446|2106|37879|1531939|6446|2106|51|2|136|8|131.28|6.56|137.84|2.43|28.24|84.01|18.90|28.37|37.23| +2452235|61818|14849|94282|1272679|5246|43975|94282|1272679|5246|43975|21|31|136|66|5686.56|227.46|5914.02|40.68|3629.34|2786.41|899.04|2001.11|3897.48| +2452454|44021|6931|66423|1412735|5894|22253|66423|1412735|5894|22253|1|32|136|43|3775.83|0.00|3775.83|60.80|2559.79|0.00|2794.11|981.72|2620.59| +2452280|49266|14861|27529|655802|1322|49995|27529|655802|1322|49995|1|21|136|31|2306.71|0.00|2306.71|89.66|1082.52|576.67|121.10|1608.94|1172.18| +2451669|19847|14845|13755|1169374|993|34612|13755|1169374|993|34612|1|1|137|93|700.29|63.02|763.31|5.78|47.43|469.19|9.24|221.86|116.23| +2451595|55884|9380|78166|1875090|4814|15338|78166|1875090|4814|15338|1|33|138|31|442.68|39.84|482.52|2.41|2080.72|150.51|75.96|216.21|2122.97| +2451515|74804|3692|21773|1404553|6180|19760|21773|1404553|6180|19760|31|32|138|42|4413.36|308.93|4722.29|60.27|238.56|3265.88|963.88|183.60|607.76| +2451291|16899|17320|67941|660216|3244|37519|67941|660216|3244|37519|52|28|139|21|151.62|13.64|165.26|77.43|96.18|116.74|23.36|11.52|187.25| +2451444|30184|9139|40917|915281|1292|32056|40917|915281|1292|32056|16|28|139|28|698.04|62.82|760.86|22.50|40.04|677.09|18.43|2.52|125.36| +2451260|45986|7216|45558|1271056|5354|36852|45558|1271056|5354|36852|46|14|139|34|1701.02|51.03|1752.05|35.11|531.42|1598.95|63.28|38.79|617.56| +2451355|64606|10705|40863|1306557|6818|8230|40863|1306557|6818|8230|7|30|140|13|203.06|2.03|205.09|56.04|64.09|6.09|120.15|76.82|122.16| +||2588|||3687||68076|643705||18116|1|17|141||4.56||4.87||0.86|0.04|3.97||23.28| +2451641|78997|13561|33016|445850|3484|2765|33016|445850|3484|2765|43|8|141|14|1214.36|72.86|1287.22|38.50|830.90|570.74|495.58|148.04|942.26| +2451646|58522|11299|64397|854748|5099|21403|64397|854748|5099|21403|46|27|142|46|134.78|12.13|146.91|64.12|1.38|134.78|0.00|0.00|77.63| +2451564|37191|9601|54302|1733659|3863|11220|54302|1733659|3863|11220|46|31|142|52|5554.12|55.54|5609.66|83.14|1437.28|5387.49|151.63|15.00|1575.96| +2451105|30380|13696|17930|351263|3443|30823|17930|351263|3443|30823|40|12|143|49|2148.16|42.96|2191.12|40.86|736.47|2062.23|56.71|29.22|820.29| +|18453|12949|4877|990329|6981|26080||||26080|||144||54.00|||26.09||24.30||25.55|| +2451877|74022|1115|62415|412674|980|41239|62415|412674|980|41239|38|24|146|17|477.19|19.08|496.27|94.94|169.15|357.89|97.82|21.48|283.17| +2451931|11441|17881|72405|42997|6836|43475|72405|42997|6836|43475|35|15|146|9|594.09|41.58|635.67|99.75|223.65|582.20|4.63|7.26|364.98| +2451952|72639|6662|25687|331184|2281|30978|25687|331184|2281|30978|55|31|147|2|56.32|1.68|58.00|95.08|57.54|32.10|10.89|13.33|154.30| +2452497|31120|5767|97143|744653|3810|28967|97143|744653|3810|28967|13|3|148|10|376.00|0.00|376.00|5.99|67.10|30.08|214.47|131.45|73.09| +2452338|75688|15803|71902|232164|6473|47582|71902|232164|6473|47582|57|24|149|22|1400.08|28.00|1428.08|23.49|744.26|112.00|154.56|1133.52|795.75| +2452321|32331|17695|50164|1419112|6907|28702|50164|1419112|6907|28702|31|10|149|13|299.26|14.96|314.22|83.36|299.26|134.66|134.97|29.63|397.58| +2452585|76529|4458|66648|1053258|6268|3678|66648|1053258|6268|3678|36|33|151|9|52.02|0.00|52.02|99.26|57.24|16.12|34.46|1.44|156.50| +2452558|59507|9379|32508|458056|6004|752|32508|458056|6004|752|7|23|151|2|10.64|0.42|11.06|42.53|8.62|3.08|5.21|2.35|51.57| +2452431|66172|14315|9732|1904822|1159|21449|9732|1904822|1159|21449|45|30|152|26|696.02|13.92|709.94|22.50|221.26|320.16|75.17|300.69|257.68| +2451707|61736|5240|81941|1761461|2581|9748|81941|1761461|2581|9748|2|32|153|19|287.09|0.00|287.09|39.45|574.18|25.83|41.80|219.46|613.63| +2451571|50207|6460|5037|633572|2107|22242|5037|633572|2107|22242|7|14|153|55|628.65|31.43|660.08|46.16|2435.95|125.73|382.21|120.71|2513.54| +2451501|66402|5491|46376|1387022|6068|29914|46376|1387022|6068|29914|13|31|153|14|1409.10|0.00|1409.10|32.11|199.22|211.36|634.80|562.94|231.33| +2451907|68584|7031|13708|1723305|902|4946|13708|1723305|902|4946|19|15|155|16|1702.56|136.20|1838.76|34.61|619.04|1447.17|191.54|63.85|789.85| +2451445|47384|15853|67068|1528175|3682|22742|67068|1528175|3682|22742|38|2|156|4|108.16|1.08|109.24|15.07|57.00|94.09|4.36|9.71|73.15| +2452870|17174|16872|78757|668776|4042|44061|78757|668776|4042|44061|45|17|157|37|597.55|23.90|621.45|5.41|659.71|268.89|207.05|121.61|689.02| +2452629|46611|11259|76627|908125|4423|15521|76627|908125|4423|15521|51|11|159|7|12.81|0.38|13.19|35.10|0.70|1.53|6.54|4.74|36.18| +2452772|61876|8217|46971|1799140|5709|5525|46971|1799140|5709|5525|39|19|159|9|860.85|34.43|895.28|96.49|264.87|637.02|100.72|123.11|395.79| +2451618|73969|15745||296806||43344|80737|296806||43344||10|160|19|44.08||||102.98|3.08|7.38|33.62|| +2451647|69222|15530|43215|1324659|6103|46747|43215|1324659|6103|46747|8|32|160|41|1341.93|26.83|1368.76|54.95|298.07|1341.93|0.00|0.00|379.85| +2451809|70800|1532|34446|425086|4526|22319|34446|425086|4526|22319|58|24|160|3|40.89|0.00|40.89|80.15|79.08|21.67|13.64|5.58|159.23| +2452200|15201|11709|51703|1903869|1988|10154|51703|1903869|1988|10154|3|14|161|53|3673.43|220.40|3893.83|19.13|592.54|1763.24|955.09|955.10|832.07| +2452062|46294|7043|56554|1149852|7150|2448|56554|1149852|7150|2448|9|16|161|37|742.96|0.00|742.96|97.28|275.65|542.36|168.50|32.10|372.93| +2452815|19799|72|27864|1284143|6022|44254|27864|1284143|6022|44254|12|9|164|11|243.65|17.05|260.70|7.94|53.24|58.47|55.55|129.63|78.23| +2452773|70588|1963|65407|1231216|2482|19606|65407|1231216|2482|19606|51|32|165|6|212.40|14.86|227.26|72.62|3.60|146.55|39.51|26.34|91.08| +2452673|69138|11808|41510|1298761|6059|41757|41510|1298761|6059|41757|30|16|165|25|2282.00|45.64|2327.64|15.50|588.75|1369.20|164.30|748.50|649.89| +2452272|30759|9707|87083|1336011|139|29867|87083|1336011|139|29867|49|22|166|44|391.16|31.29|422.45|7.80|1075.80|207.31|62.50|121.35|1114.89| +2452769|57764|11799|26199|33405|4611|35817|26199|33405|4611|35817|36|30|171|14|1084.30|0.00|1084.30|59.20|713.44|422.87|403.47|257.96|772.64| +2451575|79271|343|5861|1917676|2644|42859|5861|1917676|2644|42859|55|25|173|27|1268.46|76.10|1344.56|39.24|128.25|126.84|856.21|285.41|243.59| +2451568|34046|9010|51754|1701368|3782|43749|51754|1701368|3782|43749|46|9|173|27|343.44|6.86|350.30|79.07|231.93|164.85|55.36|123.23|317.86| +|75175|15050|41510||6305|4879|41510|1578484||4879|19||173||||1172.03|37.79||||524.63|338.60| +2451077|50278|15386|75580|1352068|109|13350|75580|1352068|109|13350|34|10|174|17|377.40|18.87|396.27|82.85|43.86|241.53|6.79|129.08|145.58| +2451113|61235|6640|35017|989992|3367|6742|35017|989992|3367|6742|19|17|174|6|4.62|0.41|5.03|91.55|6.54|2.31|1.66|0.65|98.50| +2451114|69061|13502|57503|1258217|3284|43998|57503|1258217|3284|43998|10|21|174|28|0.00|0.00|0.00|45.91|702.52|0.00|0.00|0.00|748.43| +2452134|27349|8183|51112|287150|7188|38026|51112|287150|7188|38026|55|12|175|91|20323.94|0.00|20323.94|16.28|8129.03|17275.34|487.77|2560.83|8145.31| +||11971|56794||340|36913||||||1|176||358.92||||||||| +2452421|43122|12013|91487|1229591|5295|5752|91487|1229591|5295|5752|1|2|177|3|279.48|16.76|296.24|31.88|0.00|39.12|132.19|108.17|48.64| +2452246|72396|6979|10718|1507554|2790|15979|10718|1507554|2790|15979|43|11|177|23|672.52|13.45|685.97|82.02|622.15|585.09|40.21|47.22|717.62| +2451698|49590|14383|93085|1729540|42|42097|93085|1729540|42|42097|13|20|178|31|856.53|77.08|933.61|51.28|164.92|188.43|527.79|140.31|293.28| +2451708|75926|8965|79203|328418|2786|34354|79203|328418|2786|34354|31|8|178|4|52.12|4.69|56.81|37.88|15.80|13.55|16.97|21.60|58.37| +2451646|65916|14269|57267|503265|6063|2814|57267|503265|6063|2814|50|23|178|15|3178.80|158.94|3337.74|11.13|1651.35|1494.03|269.56|1415.21|1821.42| +2452085|67782|8041|83327|1796512|4897|43635|83327|1796512|4897|43635|53|16|179|15|1368.90|13.68|1382.58|48.98|608.40|82.13|102.94|1183.83|671.06| +2451965|38332|12158|51816|518822|3875|13728|51816|518822|3875|13728|1|2|179|57|863.55|34.54|898.09|82.72|479.94|293.60|541.45|28.50|597.20| +2452105|11332|11521|18120|700033|793|29106|18120|700033|793|29106|25|8|179|56|4683.28|280.99|4964.27|52.05|1984.08|1873.31|758.69|2051.28|2317.12| +2451954|30243|16597|17518|440362|1606|46497|17518|440362|1606|46497|44|5|180|68|2711.16|216.89|2928.05|11.94|120.36|2277.37|82.42|351.37|349.19| +2452792|32759|13735|1287|1161084|3281|39878|1287|1161084|3281|39878|9|30|182|1|0.00|0.00|0.00|86.02|0.31|0.00|0.00|0.00|86.33| +2451093|19768|7996|77474||||77474|1691292||36222||6|183|4|141.12||143.94||52.68||87.13||| +2451023|25678|15062|54262|724601|4540|26479|54262|724601|4540|26479|56|24|183|26|115.96|2.31|118.27|3.24|10.92|2.31|87.51|26.14|16.47| +2452470|25426|10319|89293|615604|7162|9520|89293|615604|7162|9520|11|8|184|19|356.44|32.07|388.51|29.13|237.50|0.00|310.10|46.34|298.70| +2452395|29169|13371|35414|48508|1602|45143|35414|48508|1602|45143|47|34|185|3|19.77|0.79|20.56|90.69|5.04|13.44|0.50|5.83|96.52| +2451892|74845|12671|96577|1821662|5947|8232|96577|1821662|5947|8232|11|10|186|6|42.54|2.12|44.66|19.21|4.68|15.31|0.54|26.69|26.01| +2451907|72326|16442|68166|1875876|2339|1249|68166|1875876|2339|1249|14|2|186|4|20.28|1.21|21.49|40.48|0.24|14.19|5.78|0.31|41.93| +2452529|78098|12071|77095|1851220|4898|32976|77095|1851220|4898|32976|21|16|187|35|852.95|25.58|878.53|55.74|327.25|554.41|65.67|232.87|408.57| +2452360|46617|3019|53624||||53624|||||31|187|||46.49|710.65|47.58||518.04||75.99|| +2452447|60513|12965|74383|847710|1037|15211|74383|847710|1037|15211|37|32|187|16|64.16|4.49|68.65|74.33|342.40|23.73|10.10|30.33|421.22| +2452545|65727|16573|69391|422268|5545|45622|69391|422268|5545|45622|23|27|187|28|297.92|23.83|321.75|56.22|923.44|169.81|94.80|33.31|1003.49| +2452571|47104|11427|82762|1856645|4928|11211|82762|1856645|4928|11211|48|31|189|60|299.40|17.96|317.36|74.60|133.20|59.88|153.29|86.23|225.76| +2452570|27777|17310|22670|1464059|1448|35284|22670|1464059|1448|35284|21|7|189|38|1783.72|35.67|1819.39|38.36|36.86|1712.37|44.95|26.40|110.89| +2452534|18816|787|72105|510212|3519|8873|72105|510212|3519|8873|12|29|189|32|608.96|6.08|615.04|12.60|46.72|170.50|302.53|135.93|65.40| +2452533|23066|16962|74474|1560403|1979|2308|74474|1560403|1979|2308|45|13|189|35|673.40|40.40|713.80|25.26|74.55|424.24|27.40|221.76|140.21| +2452696|33495|5631|16890|1579696|902|3708|16890|1579696|902|3708|9|29|189|63|3817.80|267.24|4085.04|28.69|1908.90|1221.69|882.67|1713.44|2204.83| +2452019|72205|9655|87161|1309013|6418|35070|87161|1309013|6418|35070|43|33|193|37|372.96|14.91|387.87|54.91|202.76|365.50|0.96|6.50|272.58| +2451986|74748|581|64772|841546|3920|35094|64772|841546|3920|35094|35|8|193|10|203.50|10.17|213.67|29.11|260.00|113.96|75.21|14.33|299.28| +2451906|71247|3853|58000|231026|1205|37784|58000|231026|1205|37784|17|25|195|2|123.42|3.70|127.12|62.60|23.84|25.91|77.03|20.48|90.14| +2451971|26686|115|19305|670509|5713|27717|19305|670509|5713|27717|35|2|196|17|103.87|0.00|103.87|47.30|20.74|70.63|21.27|11.97|68.04| +2451252|61475|7795|40697|1714544|5191|46238|40697|1714544|5191|46238|55|22|197|20|882.00|44.10|926.10|2.07|653.40|485.10|349.27|47.63|699.57| +2452076|40245|5246|74149|1296610|3871|24379|74149|1296610|3871|24379|43|32|198|8|12.64|0.63|13.27|89.40|5.60|9.98|1.03|1.63|95.63| +2452902|30694|9765|16352|1785597|6489|43334|16352|1785597|6489|43334|6|16|201|9|70.38|0.00|70.38|55.46|50.31|38.70|15.20|16.48|105.77| +2452709|20035|4386|99477|1540007|4537|33020|99477|1540007|4537|33020|25|30|201|77|2616.46|78.49|2694.95|73.88|693.77|1072.74|1065.16|478.56|846.14| +2452711|33447|14875|17990|997022|1654|3486|17990|997022|1654|3486|6|23|202|13|34.32|3.08|37.40|75.36|143.13|32.60|0.22|1.50|221.57| +2451846|73375|12685|71744|1871785|4179|18173|71744|1871785|4179|18173|56|19|203|3|122.82|3.68|126.50|8.65|68.76|62.63|28.28|31.91|81.09| +2451866|80236|6733|40958|1727122|6533|25897|40958|1727122|6533|25897|59|20|203|8|101.20|6.07|107.27|86.84|364.48|76.91|23.07|1.22|457.39| +2452041|63899|13011|68106|1745559|5412|12065|68106|1745559|5412|12065|53|32|204|1|164.12|9.84|173.96|67.42|88.00|21.33|55.68|87.11|165.26| +2452159|76627|7589|75816|1345626|6636|40446|75816|1345626|6636|40446|31|33|204|15|34.50|2.41|36.91|11.66|276.00|23.11|3.75|7.64|290.07| +2452061|23757|3915|2296|566935|5974|16613|2296|566935|5974|16613|37|21|204|16|543.04|10.86|553.90|64.33|91.68|38.01|90.90|414.13|166.87| +2451675|43146|8860|39654|1314487|3720|27706|39654|1314487|3720|27706|32|17|206|42|236.04|18.88|254.92|27.07|265.44|9.44|33.99|192.61|311.39| +2452278|44490|249|79558|264997|4483|20322|79558|264997|4483|20322|35|17|207|28|3333.12|233.31|3566.43|62.32|1807.68|2599.83|366.64|366.65|2103.31| +2450980|67877|13216|66532|1748836|4740|11444|66532|1748836|4740|11444|38|25|208|29|1077.35|86.18|1163.53|64.83|731.67|646.41|86.18|344.76|882.68| +2451103|71668|3661|77804|453650|3896|189|77804|453650|3896|189|44|30|208|27|1527.93|30.55|1558.48|87.60|645.84|1527.93|0.00|0.00|763.99| +2452246|80588|10739|69013|1863961|4862|43010|69013|1863961|4862|43010|41|7|209|9|0.00|0.00|0.00|10.45|155.25|0.00|0.00|0.00|165.70| +2452316|70694|8725|2762|207268|2621|35974|2762|207268|2621|35974|11|5|209|36|3015.00|150.75|3165.75|40.57|1206.00|1869.30|263.51|882.19|1397.32| +2452348|42546|11483||||30581|34478|878277|||57|5|209||||514.62||1443.60|355.91|||| +2452009|19321|4676|83663|524844|1225|||524844|1225|38406|||211||30.78|||93.39||||3.02|| +2451920|33795|14183|34864|1195014|5917|28420|34864|1195014|5917|28420|20|17|211|4|225.60|11.28|236.88|6.56|214.84|194.01|26.85|4.74|232.68| +2452616|21111|8142|45581|256976|6771|24966|45581|256976|6771|24966|1|13|213|71|13275.58|265.51|13541.09|71.06|3204.23|2389.60|3918.95|6967.03|3540.80| +2451384|42631|9763|3730|882712|5694|45476|3730|882712|5694|45476|40|8|214|10|81.50|3.26|84.76|59.74|10.40|10.59|19.14|51.77|73.40| +2452016|29220|16652|55148|211402|6856|20310|55148|211402|6856|20310|31|31|216|30|106.20|0.00|106.20|16.48|197.40|86.02|12.71|7.47|213.88| +2452000|37490|16273|90056|93435|2861|14640|90056|93435|2861|14640|19|23|216|41|1485.84|118.86|1604.70|27.89|565.80|549.76|9.36|926.72|712.55| +2452442|69015|15251|92599|910385|4171|33557|92599|910385|4171|33557|33|20|217|4|131.32|9.19|140.51|94.42|12.92|30.20|91.00|10.12|116.53| +2452367|65620|2881|44677|1784700|6323|42346|44677|1784700|6323|42346|7|22|217|78|1271.40|12.71|1284.11|72.52|705.90|1233.25|6.48|31.67|791.13| +2452378|68335|12181|49000|371535|5270|45758|49000|371535|5270|45758|35|18|217|56|4981.76|149.45|5131.21|72.31|2687.44|4483.58|378.61|119.57|2909.20| +2450999|66604|12454|91464|44345|6026|34954|91464|44345|6026|34954|44|6|218|6|201.06|16.08|217.14|69.69|111.66|60.31|32.37|108.38|197.43| +2451070|6360|11935|24834|1263106|6680|35456|24834|1263106|6680|35456|13|8|218|3|237.15|11.85|249.00|36.86|137.70|75.88|98.37|62.90|186.41| +2452102|17048|629|29639|1157762|4638|19161|29639|1157762|4638|19161|20|23|219|2|64.24|5.13|69.37|16.31|37.84|8.99|12.15|43.10|59.28| +2451961|39062|8869|44533|361505|2124|41543|44533|361505|2124|41543|37|33|219|38|1748.76|52.46|1801.22|29.29|26.22|716.99|577.79|453.98|107.97| +2451365|37196|8218|5858|1012650|3442|3481|5858|1012650|3442|3481|43|5|220|3|26.07|2.34|28.41|25.64|25.53|16.94|1.82|7.31|53.51| +2451408|69841|14060|79988|1644989|2790|46991|79988|1644989|2790|46991|38|25|220|44|776.16|15.52|791.68|14.38|582.12|279.41|158.96|337.79|612.02| +2451123|3581|16795|97463|673234|4558|43129|97463|673234|4558|43129|31|8|221|10|598.70|5.98|604.68|47.83|288.60|203.55|185.72|209.43|342.41| +2451223|33152|5072|97984|523740|6296|41299|97984|523740|6296|41299|25|9|222|12|752.04|15.04|767.08|6.15|208.32|646.75|22.11|83.18|229.51| +2452679|28602|10011|21577|505904|3478|36205|21577|505904|3478|36205|51|23|225|21|0.00|0.00|0.00|51.49|402.78|0.00|0.00|0.00|454.27| +2452822|60526|17899|10448|29479|3710|8341|10448|29479|3710|8341|54|4|225|14|428.12|12.84|440.96|59.43|820.54|299.68|7.70|120.74|892.81| +2451977|68718|4195|77534|1120704|7074|33844|77534|1120704|7074|33844|20|33|226|34|3881.44|0.00|3881.44|43.32|1940.72|1707.83|1630.20|543.41|1984.04| +2451403|73317|6337|94779|1038088|3797|34306|94779|1038088|3797|34306|58|29|228|57|600.21|12.00|612.21|21.49|668.61|174.06|315.35|110.80|702.10| +2451477|25201|6554|18521|1608664|3607|34503|18521|1608664|3607|34503|46|16|228|28|1782.76|17.82|1800.58|71.03|1018.64|124.79|215.53|1442.44|1107.49| +2451525|45831|12286|21750|1370819|335|41404|21750|1370819|335|41404|49|16|229|26|2287.22|0.00|2287.22|37.56|1586.26|1944.13|298.48|44.61|1623.82| +2451568|75136|6860|33981|486596|2411|45826|33981|486596|2411|45826|43|28|229|28|612.92|0.00|612.92|36.05|197.40|361.62|57.79|193.51|233.45| +2452110|25909|3847|78499|835148|6997|40446|78499|835148|6997|40446|25|3|231|23|1957.53|156.60|2114.13|44.53|907.12|861.31|1041.40|54.82|1108.25| +2452378|6141|10357|88920|963171|975|35626|88920|963171|975|35626|31|8|232|3|11.85|0.71|12.56|66.80|83.13|2.48|7.40|1.97|150.64| +2452701|27120|2586|63777|1291455|4380|48058|63777|1291455|4380|48058|18|2|233|22|1845.58|147.64|1993.22|39.09|1318.24|203.01|1231.92|410.65|1504.97| +2452546|67744|11382|94822|171546|1423|5747|94822|171546|1423|5747|45|19|234|3|489.42|19.57|508.99|45.91|42.81|58.73|215.34|215.35|108.29| +2451466|66181|14462|43770|666610|2250|29090|43770|666610|2250|29090|25|17|237|22|642.18|57.79|699.97|46.21|242.22|468.79|136.97|36.42|346.22| +2451089|74288|5701|74240|994772|6151|5130|74240|994772|6151|5130|56|8|238|12|0.00|0.00|0.00|64.16|241.92|0.00|0.00|0.00|306.08| +2450911|70001|11864|20033|934543|1500|45610|20033|934543|1500|45610|10|16|238|2|56.52|3.39|59.91|43.73|56.52|41.82|5.29|9.41|103.64| +2451011|51686|15680|63646|1219200|1601|19555|63646|1219200|1601|19555|31|8|238|21|4456.83|401.11|4857.94|30.67|1378.44|401.11|2717.33|1338.39|1810.22| +2452840|46167|13155|62773|1252017|908|42152|62773|1252017|908|42152|57|1|241|35|7610.40|228.31|7838.71|78.73|2663.50|2054.80|111.11|5444.49|2970.54| +||14596|76470|1261294|518|5695|76470|1261294||5695||32|242||||64.89||||39.55||81.61| +2452711|19383|12253||345662|795|32328|||795|32328||11|244||38.07||41.11|23.23||26.64|8.68|2.75|| +2452783|65519|10308|59892|1089914|6858|7627|59892|1089914|6858|7627|60|23|244|2|176.46|0.00|176.46|95.27|25.20|142.93|23.13|10.40|120.47| +2452082|58666|14174|34572|1797402|679|8709|34572|1797402|679|8709|7|12|245|22|2155.56|150.88|2306.44|8.87|1092.08|1940.00|68.97|146.59|1251.83| +2452386|71898|6801|16885|966314|4190|4019|16885|966314|4190|4019|19|9|246|26|2074.54|0.00|2074.54|55.19|1003.86|1452.17|497.89|124.48|1059.05| +2451420|67075|16444|64128|1525746|964|18182|64128|1525746|964|18182|19|22|247|2|77.80|2.33|80.13|52.77|70.28|41.23|10.23|26.34|125.38| +2451325|70391|11743|32807|1451903|599|11248|32807|1451903|599|11248|50|25|247|26|179.14|5.37|184.51|75.99|637.26|127.18|47.80|4.16|718.62| +2451335|75395|1285|64946|1028069|2839|43609|64946|1028069|2839|43609|31|20|248|18|126.72|11.40|138.12|5.56|231.30|27.87|42.50|56.35|248.26| +2451343|40184|14653|59900|1912191|5282|773|59900|1912191|5282|773|16|4|248|29|3387.78|237.14|3624.92|89.38|859.56|846.94|1168.78|1372.06|1186.08| +|42779|9292||1564391|||97924||3305|||17|248|12||0.84|||255.12|9.81|1.99|5.12|| +2452394|61396|4723|63512|412889|3866|21643|63512|412889|3866|21643|59|8|250|6|27.18|1.35|28.53|79.73|312.60|8.69|15.34|3.15|393.68| +2452363|64389|13201|10232|236641|3850|13059|10232|236641|3850|13059|25|31|251|42|4317.18|388.54|4705.72|65.04|2349.90|1381.49|2407.26|528.43|2803.48| +2452383|2322|5631|54348|1376446|1744|6642|54348|1376446|1744|6642|31|30|253|11|223.63|20.12|243.75|85.51|28.82|33.54|174.88|15.21|134.45| +2452347|45889|2253|58057|681709|2161|15013|58057|681709|2161|15013|55|17|254|4|245.28|12.26|257.54|26.66|65.40|78.48|133.44|33.36|104.32| +2451965|62545|11915|35005|434837|4622|22054|35005|434837|4622|22054|20|30|255|38|63.46|5.07|68.53|70.73|15.20|50.13|8.39|4.94|91.00| +2451985|44611|3128|89306|337124|3342|9560|89306|337124|3342|9560|25|28|255|14|1493.24|119.45|1612.69|75.91|535.22|403.17|490.53|599.54|730.58| +2451424|49423|12685|7321|964738|6602|14866|7321|964738|6602|14866|56|23|256|23|716.68|43.00|759.68|88.69|353.51|129.00|417.25|170.43|485.20| +2451092|15476|3548|35043|992223|250|34630|35043|992223|250|34630|32|1|258|16|181.92|16.37|198.29|5.07|77.92|143.71|6.49|31.72|99.36| +2451548|53248|4423|15377|64748|2693|432|15377|64748|2693|432|40|10|259|15|432.00|30.24|462.24|37.01|545.70|138.24|99.87|193.89|612.95| +2452655|73554|9153|7508|1440517|5173|582|7508|1440517|5173|582|24|24|260|49|2773.89|83.21|2857.10|73.31|1133.37|1886.24|603.60|284.05|1289.89| +2451350|39937|10696|3220|1867305|190|5075|3220|1867305|190|5075|37|11|261|1|100.52|6.03|106.55|16.98|24.71|6.03|62.36|32.13|47.72| +2451258|85139|13382|18899|435040|1934|32875|18899|435040|1934|32875|14|17|261|16|1326.72|26.53|1353.25|99.22|1027.04|742.96|198.47|385.29|1152.79| +2452383|77464|3079|10425|1596581|337|12540|10425|1596581|337|12540|47|15|262|12|51.24|3.58|54.82|37.30|17.04|19.98|1.56|29.70|57.92| +2451257|56801|11929|11868|1438974|5280|37186|11868|1438974|5280|37186|4|2|263|8|93.44|8.40|101.84|84.63|381.68|6.54|9.55|77.35|474.71| +2451157|61099|112|78976|79252|5481|38510|78976|79252|5481|38510|50|5|264|12|68.88|0.68|69.56|51.95|334.80|42.70|9.68|16.50|387.43| +2451212|76704|16981|75138|422944|1150|23356|75138|422944|1150|23356|13|30|264|2|152.76|1.52|154.28|3.89|14.54|99.29|16.04|37.43|19.95| +2451267|46213|9700|14332|1637009|2086|25163|14332|1637009|2086|25163|58|22|264|5|196.60|11.79|208.39|39.60|96.30|171.04|16.10|9.46|147.69| +2452134|48039|9387|29695|150092|5569|44453|29695|150092|5569|44453|57|12|265|21|271.95|2.71|274.66|53.32|116.55|5.43|26.65|239.87|172.58| +2452168|65141|2147|90803|542493|6536|48179|90803|542493|6536|48179|25|21|265|41|1336.60|80.19|1416.79|47.38|55.35|962.35|157.18|217.07|182.92| +2451056|68753|12547|60316|1871|579|22087|60316|1871|579|22087|50|24|266|65|799.50|15.99|815.49|14.46|507.00|287.82|378.64|133.04|537.45| +2451294|19824|6046|68844|1881174|6915|340|68844|1881174|6915|340|38|2|267|41|530.54|42.44|572.98|5.56|89.38|106.10|80.64|343.80|137.38| +|35362|5314|||6627||19209|774518|6627|||28|268|||6.88||||32.47||65.28|176.90| +2451174|5506|848|98553|86435|5517|16999|98553|86435|5517|16999|46|35|269|11|162.03|1.62|163.65|51.54|101.20|113.42|22.84|25.77|154.36| +2451985|61864|17831|64310|1368819|6128|4637|64310|1368819|6128|4637|14|26|270|62|5987.96|119.75|6107.71|48.39|3849.58|3293.37|2586.80|107.79|4017.72| +||7613|24496|1082053|||24496||||21|11|272|78|88.14|3.52|91.66|||52.88|||145.05| +2452358|79684|979||521875|||67695||1208|8541|15|35|272|5||22.25|578.60||163.15||||249.16| +2452122|74428|2384|43090|1601176|5215|15701|43090|1601176|5215|15701|26|21|273|3|153.33|0.00|153.33|26.86|35.70|59.79|16.83|76.71|62.56| +2452558|44986|13014||1403975|50||85853|||||13|275|16|27.68|||27.58|512.48||0.19||| +2452143|44789|11673|39741|270260|3179|42498|39741|270260|3179|42498|17|5|276|11|524.70|47.22|571.92|6.05|215.27|487.97|28.28|8.45|268.54| +2451043|59463|13753|21297|784918|3739|18207|21297|784918|3739|18207|44|9|277|5|25.05|2.00|27.05|76.19|185.70|17.53|7.14|0.38|263.89| +2451396|19126|11092|11311|444163|6615|35067|11311|444163|6615|35067|37|16|278|16|160.16|3.20|163.36|68.64|154.56|65.66|90.72|3.78|226.40| +2451258|84559|14044|40182|1036566|4950|45734|40182|1036566|4950|45734|7|8|278|11|8.80|0.79|9.59|98.28|5.06|4.84|0.23|3.73|104.13| +2451210|48307|13712|58179|1880210|5441|32672|58179|1880210|5441|32672|7|34|279|36|771.84|30.87|802.71|41.57|4148.64|223.83|274.00|274.01|4221.08| +||5560||1791892|1456|13373||||13373|34|28|281|14|24.22|||31.76|||12.51|3.96|77.51| +2451097|21354|6082|39363|44554|3698|4089|39363|44554|3698|4089|58|35|281|6|640.74|32.03|672.77|6.42|6.60|134.55|450.50|55.69|45.05| +2450925|16066|193|13275|363739|4156|5820|13275|363739|4156|5820|44|15|281|51|2536.74|76.10|2612.84|85.91|164.73|228.30|946.46|1361.98|326.74| +2451266|59537|8632|53031|662146||||662146|||46|31|282||330.60|23.14||96.41|764.52||26.97|25.93|884.07| +2451277|26638|14617|93792|1863889|5186|12083|93792|1863889|5186|12083|16|25|282|59|752.25|15.04|767.29|32.99|2633.17|526.57|58.67|167.01|2681.20| +2452351|46035|2987|97840|500406|2172|4385|97840|500406|2172|4385|53|7|283|38|418.76|12.56|431.32|96.72|159.98|297.31|63.15|58.30|269.26| +2452204|69888|14611|26700|1175765|4830|21095|26700|1175765|4830|21095|21|2|283|19|219.64|19.76|239.40|20.61|261.63|10.98|181.53|27.13|302.00| +2452126|71599|14859|71078|643374|3223|15919|71078|643374|3223|15919|13|19|283|2|165.88|3.31|169.19|55.92|45.88|145.97|15.52|4.39|105.11| +2451138|45505|82|50479|1094863|3251|13551|50479|1094863|3251|13551|32|27|285|32|899.84|80.98|980.82|63.82|961.28|377.93|360.11|161.80|1106.08| +2451012|39740|14840|88788|1649968|748|36775|88788|1649968|748|36775|16|19|285|54|4475.52|313.28|4788.80|3.48|331.02|2237.76|492.30|1745.46|647.78| +2451231|68904|4315|33691|1627332|4973|36967|33691|1627332|4973|36967|28|27|286|6|51.06|4.08|55.14|33.53|84.54|14.29|19.48|17.29|122.15| +2451323|71939|11774|60129|1209936|6590|41244|60129|1209936|6590|41244|38|14|286|9|191.16|0.00|191.16|46.20|263.70|105.13|68.82|17.21|309.90| +2451425|36388|15595|63364|1347317|870|30023|63364|1347317|870|30023|40|23|287|27|1720.98|86.04|1807.02|2.14|1132.11|1032.58|227.17|461.23|1220.29| +2451532|83208|3328|44647|1617567|440|28960|44647|1617567|440|28960|52|34|287|69|631.35|56.82|688.17|45.64|1943.04|473.51|20.51|137.33|2045.50| +2452149|62148|524|78604|915942|4480|1418|78604|915942|4480|1418|59|7|288|11|90.86|4.54|95.40|27.78|59.29|80.86|2.70|7.30|91.61| +2451932|44361|14930|37586|71175|2364|31553|37586|71175|2364|31553|31|17|288|34|997.22|39.88|1037.10|16.91|562.70|169.52|248.31|579.39|619.49| +2452045|68287|9872|64467|1153472|430|588|64467|1153472|430|588|55|2|288|83|278.05|8.34|286.39|85.20|928.77|47.26|156.93|73.86|1022.31| +2452176|78872|4015|49675|1597822|722|12069|49675|1597822|722|12069|53|28|290|11|254.10|0.00|254.10|19.41|152.46|231.23|8.91|13.96|171.87| +2452235|42256|1881|93018|1425586|2154|43522|93018|1425586|2154|43522|33|12|290|58|107.88|2.15|110.03|86.54|9.86|91.69|3.23|12.96|98.55| +||12261|||||||5626||7||291|10|573.40||607.80|||||104.59|| +2451784|47739|566|83352|1008363|5821|28079|83352|1008363|5821|28079|26|13|292|1|31.09|1.86|32.95|50.42|9.69|0.31|14.77|16.01|61.97| +2451821|35227|2150|67185|1030957|3882|46454|67185|1030957|3882|46454|38|20|293|60|549.00|5.49|554.49|38.26|235.20|274.50|68.62|205.88|278.95| +2452188|47091|13967|13226|1497413|3553|12093|13226|1497413|3553|12093|37|32|294|21|1677.27|50.31|1727.58|48.58|26.88|1408.90|53.67|214.70|125.77| +2452179|64643|17377|88138|1517671|5189|44839|88138|1517671|5189|44839|57|20|294|28|497.84|4.97|502.81|17.75|945.84|59.74|13.14|424.96|968.56| +2452790|29887|9427|85628|919578|6681|19031|85628|919578|6681|19031|43|8|295|32|1809.28|0.00|1809.28|47.37|834.88|1628.35|57.89|123.04|882.25| +2452733|65191|3930|73053|363227|3801|45622|73053|363227|3801|45622|54|15|296|54|594.54|17.83|612.37|69.12|0.00|0.00|541.03|53.51|86.95| +2452723|50141|4461|60764|1219766|6768|36570|60764|1219766|6768|36570|39|12|296|48|2389.44|143.36|2532.80|25.23|2766.72|1218.61|948.37|222.46|2935.31| +2452637|32102|13266|95006|1168974|542|44957|95006|1168974|542|44957|24|4|296|6|639.48|38.36|677.84|40.08|0.00|204.63|300.04|134.81|78.44| +2452615|71413|8197|84832|1391476|4520|43564|84832|1391476|4520|43564|9|33|296|23|1070.65|53.53|1124.18|66.95|264.73|331.90|679.65|59.10|385.21| +2452042|48411|9872|89102|833858|3061|39864|89102|833858|3061|39864|55|18|297|28|329.84|0.00|329.84|41.92|219.80|128.63|50.30|150.91|261.72| +2452150|47056|3095|59633|180405|4824|16796|59633|180405|4824|16796|23|15|297|5|471.70|18.86|490.56|71.80|333.80|160.37|74.71|236.62|424.46| +2452653|74476|3468|25264|1651608|2399|26343|25264|1651608|2399|26343|48|16|298|12|15.60|0.00|15.60|14.64|501.72|0.62|0.29|14.69|516.36| +2451703|15077|2138|66154|1794873|4745|11417|66154|1794873|4745|11417|46|33|300|15|55.65|4.45|60.10|91.88|1394.70|2.22|35.79|17.64|1491.03| +2451370|42630|6427|33577|995818|12|26653|33577|995818|12|26653|14|34|301|1|77.98|1.55|79.53|88.66|10.49|28.07|10.48|39.43|100.70| +2452623|44346|16227|87506|1358636|1319|48449|87506|1358636|1319|48449|55|20|302|2|79.90|6.39|86.29|13.02|10.70|62.32|13.71|3.87|30.11| +2452665|46067|8365|76892|1123716|493|29717|76892|1123716|493|29717|45|35|302|7|390.81|31.26|422.07|24.20|138.39|300.92|39.55|50.34|193.85| +2451233|8526|13219|98093|1784102|6111|26873|98093|1784102|6111|26873|34|17|303|3|283.53|5.67|289.20|82.56|180.42|113.41|102.07|68.05|268.65| +2451556|33048|2125|68345|716560|2822|34237|68345|716560|2822|34237|4|21|304|28|3007.48|120.29|3127.77|21.21|719.04|2255.61|684.20|67.67|860.54| +2451444|35390|14122|32346|1579758|3170|22982|32346|1579758|3170|22982|50|32|304|48|11070.24|442.80|11513.04|28.10|4584.48|664.21|4578.65|5827.38|5055.38| +2451455||4124|||4069|17567|35120|1436218|4069|17567||1|304||86.04|7.74|93.78||209.16|23.23|47.10||| +2451335|38177|15662|22649|1171516|5630|11297|22649|1171516|5630|11297|40|8|305|26|668.46|40.10|708.56|89.66|0.00|367.65|105.28|195.53|129.76| +2451451|32728|15931|85340|1208518|5296|9674|85340|1208518|5296|9674|26|9|305|25|707.75|28.31|736.06|57.61|429.25|84.93|118.33|504.49|515.17| +2451631|75589|4160|58016|251642|21|16047|58016|251642|21|16047|2|20|308|23|2303.68|69.11|2372.79|16.55|1763.64|1105.76|263.54|934.38|1849.30| +2451213|43560|17533|90618|336382|993|32357|90618|336382|993|32357|13|2|310|19|1124.23|78.69|1202.92|64.21|587.67|539.63|333.22|251.38|730.57| +2451167|70078|1390|60763|457218|5320|44975|60763|457218|5320|44975|46|1|310|27|1490.67|104.34|1595.01|57.99|941.22|1281.97|58.43|150.27|1103.55| +2451331|46523|8431|3800|339115|30|25698|3800|339115|30|25698|8|34|310|33|645.15|25.80|670.95|88.82|919.05|406.44|224.38|14.33|1033.67| +2451357|54411|12926|43593|935143|448|10281|43593|935143|448|10281|25|25|310|36|960.48|86.44|1046.92|46.86|1729.08|461.03|119.86|379.59|1862.38| +2451292|74461|14422|50081|1042248|815|8182|50081|1042248|815|8182|49|9|311|4|20.80|0.62|21.42|44.45|39.40|14.76|4.28|1.76|84.47| +2451237|3742|7580|45421|1560639|6257|17875|45421|1560639|6257|17875|32|33|311|4|10.92|0.76|11.68|5.04|5.72|4.25|4.93|1.74|11.52| +2452446|61632|7731|84744|531345|2393|30194|84744|531345|2393|30194|7|6|312|34|3140.58|251.24|3391.82|88.85|467.50|2041.37|1022.26|76.95|807.59| +2452283|51645|15109|29637|637624|1258|46045|29637|637624|1258|46045|43|18|312|60|1027.80|30.83|1058.63|78.55|538.20|729.73|295.08|2.99|647.58| +2451346|26471|7135|45453|1792162|1271|13980|45453|1792162|1271|13980|2|14|314|19|2188.42|87.53|2275.95|19.69|84.17|175.07|583.87|1429.48|191.39| +2451313|67710|487|21071|1450661|7098|25135|21071|1450661|7098|25135|44|26|314|4|39.56|2.76|42.32|57.88|5.64|19.78|19.38|0.40|66.28| +2452743|70573|16068|9987|1813696|605|26314|9987|1813696|605|26314|45|8|315|27|404.46|20.22|424.68|60.10|300.24|129.42|214.53|60.51|380.56| +2452690|18162|1243|37978|159510|5395|2527|37978|159510|5395|2527|49|32|315|3|43.29|3.46|46.75|67.29|24.36|2.16|38.25|2.88|95.11| +2452263|67167|899|47459|155949|439|42082|47459|155949|439|42082|27|18|318|27|3129.84|93.89|3223.73|4.98|3129.84|31.29|2354.89|743.66|3228.71| +2452551|362|17964|4519|357056|3993|44530|4519|357056|3993|44530|36|21|320|32|512.64|30.75|543.39|2.15|425.60|256.32|220.43|35.89|458.50| +2451208|68740|1684|57152|631374|4443|6665|57152|631374|4443|6665|22|6|321|43|5424.88|433.99|5858.87|93.08|463.11|162.74|368.34|4893.80|990.18| +2451214|33702|12511|86394|1682943|2339|48535|86394|1682943|2339|48535|31|12|321|73|727.81|36.39|764.20|14.19|800.81|131.00|185.01|411.80|851.39| +|27247|10864|||||68564|1027409|207||14|4|323||52.55|||||17.86|12.83|21.86|78.90| +2451251|42501|3415|69379|1508454|1482|36110|69379|1508454|1482|36110|52|22|324|5|705.95|28.23|734.18|50.45|270.15|472.98|223.65|9.32|348.83| +2451403|47962|9679|92685|1395580|4812|46947|92685|1395580|4812|46947|20|20|324|33|1776.06|17.76|1793.82|54.75|1522.29|1651.73|2.48|121.85|1594.80| +2451194|41548|4634|34505|743125|2498|9071|34505|743125|2498|9071|1|21|325|12|219.60|15.37|234.97|46.22|85.92|8.78|46.38|164.44|147.51| +2451646|49250|5366|13124|966061|1045|14829|13124|966061|1045|14829|16|21|326|26|3394.82|237.63|3632.45|76.89|444.34|1561.61|1429.90|403.31|758.86| +2451244|66431|17924|41758|1643878|3151|16424|41758|1643878|3151|16424|52|11|327|6|94.20|8.47|102.67|69.35|20.76|86.66|1.65|5.89|98.58| +2451244|83495|5746|65213|186896|6626|46309|65213|186896|6626|46309|50|26|327|84|6720.84|403.25|7124.09|90.08|4565.40|4166.92|1583.43|970.49|5058.73| +|22796|3703|||||5960|||||29|328||8164.80|||76.37|214.56|816.48||293.94|617.52| +2451490|27236|11221|87845|1632407|3369|4378|87845|1632407|3369|4378|46|35|331|18|474.84|9.49|484.33|83.48|265.86|242.16|25.59|207.09|358.83| +2451541|57222|10504|35255|1761079|5750|17174|35255|1761079|5750|17174|31|9|331|52|910.52|63.73|974.25|38.42|334.36|154.78|733.06|22.68|436.51| +2452679|28607|12276|71071|551330|1636|14342|71071|551330|1636|14342|49|4|332|45|2617.65|104.70|2722.35|81.04|731.70|1780.00|527.71|309.94|917.44| +||11287||||48504|25890|1111028|||39|29|332||28.08|0.56|||25.12||22.75||49.00| +2452844|69752|9913|22724|543057|1790|29723|22724|543057|1790|29723|36|7|332|11|1181.40|47.25|1228.65|77.23|131.23|35.44|1100.12|45.84|255.71| +2452833|57796|12063|70920|135659|5645|31231|70920|135659|5645|31231|1|30|332|70|630.70|0.00|630.70|30.55|908.60|441.49|132.44|56.77|939.15| +2451956|74537|11669|24506|1091885|864|26290|24506|1091885|864|26290|56|6|334|1|44.00|1.32|45.32|87.82|28.34|4.84|8.61|30.55|117.48| +2450966|33696|6070|47617|351772|374|42529|47617|351772|374|42529|28|23|336|18|1731.42|103.88|1835.30|96.01|255.06|1523.64|149.60|58.18|454.95| +2451211|24292|1735|83914|1791478|5089|43053|83914|1791478|5089|43053|4|31|338|30|1484.40|133.59|1617.99|57.15|479.40|950.01|229.78|304.61|670.14| +2452501|68445|9873|95441|680460|232|22051|95441|680460|232|22051|42|4|340|61|1648.22|16.48|1664.70|79.49|606.95|280.19|164.16|1203.87|702.92| +||16506|77600||590||77600||590||18||340|22|66.66||71.32|82.62|||0.62||| +2451697|71511|16256|30200|1396623|1313|17725|30200|1396623|1313|17725|56|34|341|17|2154.24|172.33|2326.57|55.41|247.18|258.50|454.97|1440.77|474.92| +2452031|19113|4523|72569|452913|4104|24048|72569|452913|4104|24048|31|23|344|18|374.76|14.99|389.75|68.38|447.66|228.60|39.46|106.70|531.03| +2452296|61966|8891|34385|913955|4725|36295|34385|913955|4725|36295|57|15|345|2|329.78|0.00|329.78|35.60|16.64|267.12|49.50|13.16|52.24| +2452501|33142|7605|48129|1354981|4586|44238|48129|1354981|4586|44238|55|34|345|3|3.54|0.14|3.68|96.54|7.08|3.54|0.00|0.00|103.76| +2452329|76186|2745|24343|1873811|3726|25994|24343|1873811|3726|25994|9|8|346|42|1808.52|144.68|1953.20|26.18|1126.02|325.53|949.11|533.88|1296.88| +2452067|27287|3553|40103|1627663|3670|34430|40103|1627663|3670|34430|31|30|348|14|933.10|65.31|998.41|47.41|399.84|475.88|457.22|0.00|512.56| +2451901|45970|7460|18219|531324|6029|734|18219|531324|6029|734|55|23|348|45|3562.20|71.24|3633.44|8.29|3062.25|1175.52|1980.94|405.74|3141.78| +2452252|34340|4317|42867|1227042|3217|14192|42867|1227042|3217|14192|55|7|349|65|7900.10|316.00|8216.10|79.51|3903.25|6004.07|1023.85|872.18|4298.76| +2451082|34632|8978|87282|1483680|4525|43323|87282|1483680|4525|43323|2|29|352|11|895.73|0.00|895.73|16.69|285.01|716.58|121.82|57.33|301.70| +2451136|69613|7609|26260|1827002|3744|46933|26260|1827002|3744|46933|22|12|352|3|249.48|4.98|254.46|7.98|44.67|102.28|61.82|85.38|57.63| +2450974|68133|5950|78369|1234242|2915|12788|78369|1234242|2915|12788|28|9|352|29|350.61|0.00|350.61|47.54|267.09|115.70|70.47|164.44|314.63| +2451444|57350|16694|285|1827609|2933|21857|285|1827609|2933|21857|4|21|353|65|471.90|9.43|481.33|78.36|300.30|320.89|67.95|83.06|388.09| +2452447|7033|11581|47055|1890204|906|23608|47055|1890204|906|23608|36|12|355|46|800.40|16.00|816.40|44.48|355.58|416.20|345.78|38.42|416.06| +2451742|7935|13777|60674|2978|1729|8480|60674|2978|1729|8480|26|9|357|32|323.84|9.71|333.55|5.00|259.20|71.24|189.45|63.15|273.91| +2451820|60902|13550|35395|1327201|4960|45966|35395|1327201|4960|45966|43|27|357|83|2855.20|142.76|2997.96|63.60|2548.93|1284.84|533.92|1036.44|2755.29| +2452471|41352|17219|28298|1412295|2339|4383|28298|1412295|2339|4383|37|31|360|12|302.64|6.05|308.69|95.56|496.32|154.34|140.88|7.42|597.93| +2451219|42020|5170|8694|1081238|2582|5479|8694|1081238|2582|5479|58|8|361|19|2104.44|84.17|2188.61|88.38|318.82|1073.26|577.46|453.72|491.37| +2451059|4976|15284|84901|936648|4042|32989|84901|936648|4042|32989|19|18|361|45|2864.25|200.49|3064.74|35.00|599.40|1260.27|112.27|1491.71|834.89| +2450977|47390|2558|83061|1500663|665|43420|83061|1500663|665|43420|8|5|361|34|2181.78|130.90|2312.68|80.44|2493.56|196.36|615.48|1369.94|2704.90| +2451713|50328|3076|98709|1861376|2090|29557|98709|1861376|2090|29557|2|3|363|45|6048.00|241.92|6289.92|73.96|902.70|2782.08|2024.87|1241.05|1218.58| +2452679|70380|6157|64243|1746050|5457|39664|64243|1746050|5457|39664|30|22|364|11|88.99|5.33|94.32|58.87|645.48|20.46|4.79|63.74|709.68| +2452805|57516|4539|8479|642091|2903|19715|8479|642091|2903|19715|39|3|364|76|0.00|0.00|0.00|71.68|1682.64|0.00|0.00|0.00|1754.32| +2452800|61755|2904|40407|1091405|3650|9048|40407|1091405|3650|9048|45|3|364|1|50.06|3.50|53.56|23.47|25.03|17.52|31.23|1.31|52.00| +2451576|16475|2470|63853|56173|6093|45213|63853|56173|6093|45213|55|28|365|6|733.92|7.33|741.25|47.05|92.76|300.90|225.17|207.85|147.14| +2451578|43902|7802|41822|1494874|7126|46178|41822|1494874|7126|46178|28|31|366|45|10581.30|529.06|11110.36|43.82|3105.00|8782.47|881.42|917.41|3677.88| +2451384|52713|5425|49056|488602|595|29741|49056|488602|595|29741|40|18|367|3|42.75|1.71|44.46|45.20|13.50|14.96|14.72|13.07|60.41| +2451457|5544|10570|28872|736171|6670|21969|28872|736171|6670|21969|49|18|367|8|173.20|3.46|176.66|75.02|481.20|1.73|80.59|90.88|559.68| +2452649|76912|1581|28205|977327|1202|34277|28205|977327|1202|34277|57|11|368|40|104.40|5.22|109.62|84.98|0.00|87.69|2.17|14.54|90.20| +2452578|22507|11496|73888|610247|6253|27812|73888|610247|6253|27812|57|35|369|32|102.40|1.02|103.42|68.97|84.16|74.75|24.88|2.77|154.15| +2452614|18431|16320|76611|41204|4997|36790|76611|41204|4997|36790|15|18|370|7|935.55|74.84|1010.39|43.29|212.59|729.72|148.19|57.64|330.72| +2452395|62302|14437|7435|93666|344|10975|7435|93666|344|10975|55|17|372|16|1392.80|0.00|1392.80|10.21|785.60|1128.16|214.35|50.29|795.81| +2451980|19706|12665|4759|510016|5673|24530|4759|510016|5673|24530|49|32|373|1|65.45|2.61|68.06|52.11|20.40|18.32|24.97|22.16|75.12| +2451942|64453|14179|84899|242299|1652|35452|84899|242299|1652|35452|44|25|373|32|1672.96|117.10|1790.06|2.78|856.32|1488.93|31.28|152.75|976.20| +2452065|46405|9697|33602|1270277|5705|7533|33602|1270277|5705|7533|50|30|373|63|13803.93|552.15|14356.08|4.46|3807.72|4969.41|3798.84|5035.68|4364.33| +2452387|50827|11261|99533|536184|4482|28380|99533|536184|4482|28380|39|17|374|4|123.16|7.38|130.54|69.75|150.08|0.00|2.46|120.70|227.21| +2452472|75798|53|68523|164894|653|3658|68523|164894|653|3658|1|30|374|10|42.60|0.00|42.60|56.79|653.20|28.96|7.63|6.01|709.99| +2452428|55100|9903|89538|518665|2992|44872|89538|518665|2992|44872|17|30|374|18|188.64|9.43|198.07|78.24|172.80|145.25|36.01|7.38|260.47| +2451701|67168|10441|25409|1185647|2685|39323|25409|1185647|2685|39323|40|31|376|4|222.28|17.78|240.06|59.88|60.40|177.82|36.90|7.56|138.06| +2452371|79856|6107|99536|557454|2125|47643|99536|557454|2125|47643|23|25|377|3|350.07|24.50|374.57|94.24|34.98|241.54|27.13|81.40|153.72| +2452274|82700|3169|71982|260431|4880|28129|71982|260431|4880|28129|27|17|379|1|3.28|0.09|3.37|64.76|1.83|2.59|0.35|0.34|66.68| +2451189|47341|3241|18502|1488871|487|8903|18502|1488871|487|8903|44|24|380|43|202.10|12.12|214.22|6.17|57.19|24.25|149.39|28.46|75.48| +2452248|32674|2737|44219|1762146|3273|38847|44219|1762146|3273|38847|7|13|381|10|24.30|0.00|24.30|15.53|40.90|17.49|3.47|3.34|56.43| +2452354|16716|2387|63675|1159607|4692|45824|63675|1159607|4692|45824|13|30|381|32|2844.48|227.55|3072.03|4.75|206.08|881.78|372.91|1589.79|438.38| +2451090|67522|9109|81338|343360|3782|19004|81338|343360|3782|19004|20|34|382|47|912.74|18.25|930.99|46.65|1053.27|702.80|83.97|125.97|1118.17| +2452001|61664|17269|20569|206481|6724|35126|20569|206481|6724|35126|17|24|383|81|5375.16|215.00|5590.16|71.93|1087.02|3171.34|1035.79|1168.03|1373.95| +2452096|35628|1553|68425|1319389|6800|44164|68425|1319389|6800|44164|41|34|383|4|153.32|10.73|164.05|63.17|47.56|38.33|82.79|32.20|121.46| +2451975|62535|836|762|389951|3493|41882|762|389951|3493|41882|32|30|384|42|2021.88|121.31|2143.19|17.73|3791.34|1556.84|460.38|4.66|3930.38| +2452069|66411|4199|30227|1915147|3267|20987|30227|1915147|3267|20987|32|11|384|11|69.85|3.49|73.34|92.46|34.87|30.73|28.55|10.57|130.82| +2452732|36115|9615|79848|244660|3072|38284|79848|244660|3072|38284|60|26|387|20|2792.20|0.00|2792.20|37.34|1456.80|1647.39|583.85|560.96|1494.14| +2451657||15866||||||1054048|5079|||23|388|||0.13|||36.17|||0.05|52.91| +2451168|22322|15682|18286|171766|3404|2799|18286|171766|3404|2799|7|8|389|4|486.56|24.32|510.88|87.07|262.40|243.28|75.41|167.87|373.79| +2451111|34586|8329|65731|1059996|1461|46479|65731|1059996|1461|46479|10|17|389|3|446.04|0.00|446.04|5.12|229.20|120.43|19.53|306.08|234.32| +2451231|10018|16858|18286|171766|3404|2799|18286|171766|3404|2799|28|5|389|6|203.40|12.20|215.60|28.31|136.50|144.41|58.99|0.00|177.01| +2452381|25344|13739|70394|737973|5552|34731|70394|737973|5552|34731|5|17|390|24|834.48|33.37|867.85|86.55|335.76|417.24|45.89|371.35|455.68| +2452357|66195|13245|33419|531039|2259|43479|33419|531039|2259|43479|31|20|390|3|355.74|17.78|373.52|97.15|155.31|266.80|82.71|6.23|270.24| +2451004|70169|4562|54661|403767|73|30779|54661|403767|73|30779|40|8|392|37|5286.93|158.60|5445.53|49.71|2861.21|3489.37|1366.14|431.42|3069.52| +2451522|54567|12046|50221|1444132|90|35601|50221|1444132|90|35601|38|16|394|11|115.17|3.45|118.62|98.36|76.78|97.89|0.17|17.11|178.59| +2451553||844||1763144|4753|||1763144|4753|43562|8||394|6|500.10|20.00|520.10|25.22|||||| +|45897|9830|41999|1122881||21045||1122881|||25|24|395|3||2.78||2.44||18.07|||36.00| +2451881|75690|17633|31487|208331|3128|24372|31487|208331|3128|24372|7|30|395|71|270.51|0.00|270.51|68.60|55.38|129.84|81.58|59.09|123.98| +2452292|62855|16033|39252|22033|3661|42|39252|22033|3661|42|59|14|397|28|303.52|27.31|330.83|38.71|714.28|6.07|145.75|151.70|780.30| +2452372|67516|7323|77787|486436|4685|46181|77787|486436|4685|46181|57|5|397|21|2242.17|0.00|2242.17|82.15|1861.44|336.32|495.52|1410.33|1943.59| +2452077|76307|1661|26619|966019|7019|47219|26619|966019|7019|47219|55|2|398|4|590.76|41.35|632.11|7.49|237.72|307.19|258.04|25.53|286.56| +2452051|60921|13646|15369|1587739|1262|29185|15369|1587739|1262|29185|47|2|398|10|52.20|2.08|54.28|79.78|43.20|7.83|2.21|42.16|125.06| +2452161|22914|1004|93125|875761|3197|36633|93125|875761|3197|36633|32|9|398|23|2659.03|79.77|2738.80|15.24|553.84|957.25|629.65|1072.13|648.85| +2452879|15760|10345|8176|1149544|5665|45433|8176|1149544|5665|45433|7|33|400|3|0.60|0.03|0.63|53.84|0.00|0.58|0.01|0.01|53.87| +2452434|53713|10833|11612|1122834|4903|8925|11612|1122834|4903|8925|35|8|401|73|7062.75|494.39|7557.14|23.17|613.93|423.76|6174.26|464.73|1131.49| +2452717|35576|14311|40100|641271|1642|44813|40100|641271|1642|44813|25|29|402|26|1072.50|64.35|1136.85|59.52|812.50|997.42|38.29|36.79|936.37| +2452344|39345|6291|44685|802635|719|36596|44685|802635|719|36596|3|2|403|6|140.04|11.20|151.24|15.34|230.10|110.63|28.52|0.89|256.64| +2452410|69897|2149|29606|902885|6564|11796|29606|902885|6564|11796|27|23|403|7|353.85|14.15|368.00|91.29|142.24|258.31|10.50|85.04|247.68| +2452665|29072|17121|96906|1718565|4224|23883|96906|1718565|4224|23883|33|4|405|31|411.37|20.56|431.93|72.09|1197.22|382.57|13.24|15.56|1289.87| +|41756|5566||||19056||||19056|||406|9||5.59|||||59.60||144.12| +2452903|68533|660|31112|507511|3110|41509|31112|507511|3110|41509|13|27|407|5|584.35|0.00|584.35|20.47|142.65|11.68|355.05|217.62|163.12| +2452707|54540|5928|475|334505||5271||||5271|||408||5471.20|164.13||40.18|0.00||||204.31| +2452848|76804|3237|89457|1017540|3670|15353|89457|1017540|3670|15353|33|35|408|63|2123.73|21.23|2144.96|72.40|578.97|1210.52|721.43|191.78|672.60| +2451623|66143|1951|84062|1766999|6736|30873|84062|1766999|6736|30873|52|4|413|1|0.82|0.06|0.88|97.23|37.73|0.59|0.16|0.07|135.02| +2451566|63096|12644|24073|736617|8|24199|24073|736617|8|24199|38|15|413|13|430.04|17.20|447.24|73.13|37.83|90.30|261.59|78.15|128.16| +2451759|55189|9142|57641|615344|247|42350|57641|615344|247|42350|46|8|413|55|4015.55|40.15|4055.70|90.00|133.65|1766.84|314.81|1933.90|263.80| +2451706|35709|2906|4486|346173|4997|8479|4486|346173|4997|8479|16|26|413|34|622.54|24.90|647.44|21.17|2179.40|87.15|42.83|492.56|2225.47| +2451460|16106|3190|79227|838114|6601|41888|79227|838114|6601|41888|31|25|414|24|596.40|29.82|626.22|23.34|109.20|17.89|214.04|364.47|162.36| +|31371|8833||794193|||49303|||3173||23|415|54||44.12|2250.56|||||416.58|1857.06| +2451352|77857|5270|511|1136695|79|25863|511|1136695|79|25863|25|16|416|69|291.87|17.51|309.38|88.10|774.18|186.79|35.72|69.36|879.79| +2451361|36591|5323|5839|208520|5601|18329|5839|208520|5601|18329|20|25|416|68|118.32|0.00|118.32|67.69|1479.68|63.89|40.27|14.16|1547.37| +2451176|60223|11449|70158|426538|4871|4851|70158|426538|4871|4851|10|29|417|25|367.00|25.69|392.69|64.42|111.50|121.11|218.84|27.05|201.61| +||9088||819028||15956||819028||||4|417||0.00|0.00|0.00|93.25|||0.00||| +2451113|44910|127|46371|1403817|6863|29109|46371|1403817|6863|29109|14|25|417|47|463.42|13.90|477.32|20.98|109.04|213.17|212.71|37.54|143.92| +2452311|30598|7073|63527|1884021|2562|26703|63527|1884021|2562|26703|47|23|418|8|1744.96|139.59|1884.55|64.41|994.16|837.58|390.17|517.21|1198.16| +2452232|70585|16611|45778|1232783|235|40|45778|1232783|235|40|13|14|419|3|295.05|11.80|306.85|74.58|189.66|227.18|41.40|26.47|276.04| +2452377|38739|8993|99978|337196|4814|479|99978|337196|4814|479|59|8|420|8|31.36|1.56|32.92|40.55|13.20|14.73|0.66|15.97|55.31| +2452312|64935|17045|77462|509014|566|42918|77462|509014|566|42918|53|3|420|13|100.10|9.00|109.10|6.49|400.40|55.05|26.57|18.48|415.89| +2452397|58088|5091|68967|529142|6104|25223|68967|529142|6104|25223|3|7|420|10|852.40|76.71|929.11|96.64|773.50|443.24|139.11|270.05|946.85| +2452098|78716|13280|20516|1811259|6002|47945|20516|1811259|6002|47945|29|16|422|5|756.55|30.26|786.81|19.72|573.15|52.95|42.21|661.39|623.13| +2452278|38618|6725|14510|322585|5860|47935|14510|322585|5860|47935|27|15|423|2|3.64|0.14|3.78|44.77|20.68|2.98|0.34|0.32|65.59| +2452195||13981|67997||2696||||2696|25869||26|423|31||||||1539.41|||| +2451267|37475|9110|68785|106534|4283|35585|68785|106534|4283|35585|31|15|424|1|6.54|0.06|6.60|82.40|24.66|0.00|1.43|5.11|107.12| +2451198|66651|8018|64669|1504421|5773|29125|64669|1504421|5773|29125|34|7|425|42|2169.30|173.54|2342.84|23.46|288.96|780.94|985.73|402.63|485.96| +2451996|63964|15757|71783|1400414|2998|26829|71783|1400414|2998|26829|37|13|426|22|355.08|3.55|358.63|20.37|307.12|323.12|7.99|23.97|331.04| +2452878|43944|16572|95739|1185586|5386||95739||5386|30386|||427|||||19.63|||||| +2452196|53204|15585|82951|1116077|3956|31882|82951|1116077|3956|31882|1|5|428|35|2714.60|81.43|2796.03|95.49|218.75|2090.24|280.96|343.40|395.67| +2452821|63996|16159|91062|1177495|6775|33208|91062|1177495|6775|33208|15|14|430|9|285.39|2.85|288.24|49.06|456.57|54.22|83.22|147.95|508.48| +2452467|58970|8940|71847|940264|2281|16651|71847|940264|2281|16651|48|6|432|7|120.54|7.23|127.77|40.10|156.66|44.59|74.43|1.52|203.99| +2452581|23830|3969|84430|79655|4033|42058|84430|79655|4033|42058|37|34|433|18|425.34|25.52|450.86|89.82|765.54|161.62|65.93|197.79|880.88| +2452466|32522|14611|29347|543889|6431|22973|29347|543889|6431|22973|47|5|433|4|340.72|23.85|364.57|39.97|227.16|218.06|68.68|53.98|290.98| +2452585|72459|2019|44579|1242133|6887|44841|44579|1242133|6887|44841|42|20|435|5|158.15|1.58|159.73|21.25|294.70|22.14|43.52|92.49|317.53| +2452120|33391|2099|84856|861879|2729|5700|84856|861879|2729|5700|33|7|439|14|116.62|9.32|125.94|19.87|24.22|69.97|34.05|12.60|53.41| +2451627|50051|10094|61419|477207|4995|38383|61419|477207|4995|38383|10|16|440|15|89.40|2.68|92.08|7.62|6.30|70.62|3.56|15.22|16.60| +2451701|54510|3118|79429|1874014|2999|38784|79429|1874014|2999|38784|46|7|440|26|500.76|10.01|510.77|13.50|942.50|245.37|74.06|181.33|966.01| +2452115|65179|14594|57422|240861|5189|46881|57422|240861|5189|46881|8|9|442|57|3200.55|256.04|3456.59|29.06|290.70|2944.50|217.64|38.41|575.80| +2452032|74367|2443|37715|1711959|2203|32447|37715|1711959|2203|32447|19|29|443|17|859.18|0.00|859.18|38.12|859.18|34.36|453.65|371.17|897.30| +2452125|44624|6391|99701|1794225|1549|24873|99701|1794225|1549|24873|50|4|444|32|1894.72|37.89|1932.61|93.60|1208.64|1856.82|20.46|17.44|1340.13| +2451851|47044|10099|69376|1657135|6847|13406|69376|1657135|6847|13406|25|1|446|19|325.09|22.75|347.84|35.87|162.45|68.26|38.52|218.31|221.07| +2451751|61971|12421|44180|1805156|532|32708|44180|1805156|532|32708|7|19|447|51|1348.95|121.40|1470.35|59.67|1601.91|472.13|35.07|841.75|1782.98| +2451879|26526|2714|15919|198289|1459|34982|15919|198289|1459|34982|19|20|447|3|360.69|18.03|378.72|64.79|24.87|14.42|10.38|335.89|107.69| +2451431|68891|4072|78317|245702|6334|20966|78317|245702|6334|20966|31|13|448|30|2547.90|50.95|2598.85|42.07|1663.80|1579.69|222.68|745.53|1756.82| +2452527|64462|15039|74127|528863|2799|36288|74127|528863|2799|36288|37|22|450|24|1504.56|60.18|1564.74|58.70|1013.28|210.63|258.78|1035.15|1132.16| +2452447||10865||1861615|1127|||1861615||2639|29||451||490.86||||2111.13|255.24||122.53|2200.63| +2451646|53532|9164|80968|307552|4983|30767|80968|307552|4983|30767|37|28|452|12|200.04|4.00|204.04|67.21|767.16|198.03|1.66|0.35|838.37| +2451797|72046|6938|93542|590383|6777|5971|93542|590383|6777|5971|11|25|452|2|189.30|13.25|202.55|18.16|49.80|32.18|28.28|128.84|81.21| +2452742|75179|17430|19541|897570|2446|21322|19541|897570|2446|21322|12|26|453|12|2009.28|60.27|2069.55|35.42|271.44|1004.64|70.32|934.32|367.13| +2451253|34195|13990|61567|966706|7045|10984|61567|966706|7045|10984|38|26|454|35|3995.95|79.91|4075.86|90.29|1269.10|2477.48|470.72|1047.75|1439.30| +2451824|37076|8737|72535|1475357|132|46154|72535|1475357|132|46154|37|3|455|11|1623.82|16.23|1640.05|21.44|0.00|470.90|357.40|795.52|37.67| +2452153|65069|2125|46549|1683185|6221|14504|46549|1683185|6221|14504|7|24|456|6|903.72|0.00|903.72|35.82|144.54|614.52|274.74|14.46|180.36| +2451970|41480|8791|95656|1235714|572|13430|95656|1235714|572|13430|43|23|457|1|104.04|0.00|104.04|31.40|28.61|72.82|20.60|10.62|60.01| +2452790|56319|12792|52807|979473|3292|23111|52807|979473|3292|23111|24|22|458|28|814.80|8.14|822.94|64.70|101.64|277.03|532.39|5.38|174.48| +2452737|21999|2935|26134|1464699|2464|10629|26134|1464699|2464|10629|37|14|458|49|259.70|23.37|283.07|59.65|96.04|171.40|65.34|22.96|179.06| +2452674|49022|11436|56930|103771|5812|47053|56930|103771|5812|47053|60|2|460|9|122.40|0.00|122.40|75.14|13.86|105.26|13.54|3.60|89.00| +2452710|46611|882|63262|568098|4134|34617|63262|568098|4134|34617|6|30|462|6|0.00|0.00|0.00|89.47|231.78|0.00|0.00|0.00|321.25| +2452762|53690|14425|56181|845929|6877|33825|56181|845929|6877|33825|19|21|462|10|674.90|26.99|701.89|83.02|230.20|33.74|423.16|218.00|340.21| +2452210|35397|6943||1346168|||3636||||55||465||15.00||15.00|57.87|||||| +2452127|79509|15782|52623|138974|1772|19723|52623|138974|1772|19723|13|26|465|2|61.12|1.83|62.95|77.46|90.24|18.94|21.09|21.09|169.53| +2452088|71360|1373|11308|1074825|6342|5859|11308|1074825|6342|5859|7|6|466|28|2528.12|101.12|2629.24|79.86|51.52|808.99|962.71|756.42|232.50| +2452437|19612|1129|83089|975396|2103|11210|83089|975396|2103|11210|47|17|467|22|519.86|10.39|530.25|91.43|535.04|301.51|67.68|150.67|636.86| +2451412|72875|8080|5767|523000|5033|15066|5767|523000|5033|15066|38|13|468|9|758.52|53.09|811.61|25.63|502.11|364.08|70.99|323.45|580.83| +2451234|35941|13819|55562|1739802|42|16682|55562|1739802|42|16682|50|7|468|17|1335.86|26.71|1362.57|88.53|557.94|347.32|850.14|138.40|673.18| +2451342|44964|3097|19293|854772|1098|977|19293|854772|1098|977|50|5|468|68|318.24|6.36|324.60|19.60|531.08|12.72|274.96|30.56|557.04| +2451807|31732|4112|9696|454067|4485|40856|9696|454067|4485|40856|49|29|469|5|93.10|6.51|99.61|11.66|240.95|6.51|2.59|84.00|259.12| +2451821|71691|5758|46656|288691|1573|2008|46656|288691|1573|2008|58|27|469|19|776.15|31.04|807.19|1.16|211.66|116.42|145.14|514.59|243.86| +2451962|23905|8693|16380|678460|4987|23514|16380|678460|4987|23514|31|24|470|5|533.10|10.66|543.76|60.76|210.00|127.94|307.92|97.24|281.42| +2451917|14553|17309|65662|876980|6698|23216|65662|876980|6698|23216|43|15|470|45|4228.65|253.71|4482.36|28.45|1592.55|2790.90|618.23|819.52|1874.71| +2452080|17054|9644|7052|826916|380|30644|7052|826916|380|30644|43|3|470|3|128.01|6.40|134.41|50.62|0.00|30.72|42.80|54.49|57.02| +2452021|53605|1207|73009|341221|6586|46798|73009|341221|6586|46798|8|22|470|35|2703.40|162.20|2865.60|83.50|1081.15|648.81|1171.11|883.48|1326.85| +2451529|12512|11305|88495|156765|6188|29611|88495|156765|6188|29611|49|5|471|9|442.44|35.39|477.83|53.46|188.01|243.34|111.49|87.61|276.86| +2452580|65796|11559|32032|162062|6492|6522|32032|162062|6492|6522|24|27|473|48|2477.28|0.00|2477.28|40.72|908.16|1164.32|144.42|1168.54|948.88| +2452763|58273|15361|89995|460789|4830|48871|89995|460789|4830|48871|13|18|474|21|106.89|6.41|113.30|81.18|25.20|12.82|84.66|9.41|112.79| +2452853|20549|17079|47736|1626320|2504|17135|47736|1626320|2504|17135|30|26|474|40|865.60|77.90|943.50|0.57|54.00|380.86|426.57|58.17|132.47| +2452905|33134|13633|14690|502684|2678|34272|14690|502684|2678|34272|18|21|474|53|222.60|8.90|231.50|34.27|1426.76|155.82|29.38|37.40|1469.93| +2452836|60056|8748|68553|154687|4468|41402|68553|154687|4468|41402|31|9|475|9|1064.70|85.17|1149.87|84.22|549.00|404.58|132.02|528.10|718.39| +2452708|11114|6303|27355|619688|5621|20754|27355|619688|5621|20754|51|5|475|1|45.67|2.74|48.41|32.03|18.37|1.37|6.64|37.66|53.14| +2451775|77867|16268|2330|930611|1292|26373|2330|930611|1292|26373|8|29|477|2|30.66|1.83|32.49|14.44|9.58|28.51|0.58|1.57|25.85| +2451709|47485|14048|15308|228310|6912|10714|15308|228310|6912|10714|26|9|477|5|410.60|32.84|443.44|82.53|323.75|369.54|12.72|28.34|439.12| +2451548|77998|2362|15833|91093|6353|27613|15833|91093|6353|27613|40|20|477|4|232.28|4.64|236.92|12.91|434.28|190.46|24.67|17.15|451.83| +2452159|33704|13633|6353|243200|3600|1647|6353|243200|3600|1647|43|33|478|3|189.66|7.58|197.24|90.60|53.22|125.17|62.55|1.94|151.40| +2452246|9132|1317|54497|1118685|4826|30614|54497|1118685|4826|30614|9|4|478|1|11.68|0.58|12.26|6.72|5.16|11.32|0.08|0.28|12.46| +2452261|61677|10659|72584|410244|351|29258|72584|410244|351|29258|19|33|478|5|5.45|0.00|5.45|88.79|3.05|3.21|0.40|1.84|91.84| +2451789|66396|14666|86235|1694740|45|43350|86235|1694740|45|43350|31|35|479|13|653.38|13.06|666.44|95.11|290.29|372.42|70.24|210.72|398.46| +2451796||12346|||||89390|1252161||||2|479|1||0.00|5.81|||4.99||0.69|111.71| +2451677|66370|3434|39787|1314774|608|18795|39787|1314774|608|18795|22|33|479|6|22.62|0.90|23.52|84.11|19.68|3.39|13.65|5.58|104.69| +2452244|22964|5649|79546|1691045|1744|43243|79546|1691045|1744|43243|3|34|480|4|114.68|0.00|114.68|2.96|125.12|113.53|0.77|0.38|128.08| +2451827|70558|4766|10550|414900|403|21906|10550|414900|403|21906|47|25|481|39|1522.56|137.03|1659.59|47.19|340.47|943.98|520.72|57.86|524.69| +2451826|55928|17408|58334|244217|5747|36430|58334|244217|5747|36430|11|28|481|39|1125.15|0.00|1125.15|54.83|409.11|866.36|150.09|108.70|463.94| +2452121|49202|13052|32363|1071280|4150|34667|32363|1071280|4150|34667|37|3|482|21|1216.74|0.00|1216.74|4.67|672.42|559.70|151.11|505.93|677.09| +2452719|72694|4161|98790|541588|2433|11678|98790|541588|2433|11678|45|12|484|23|357.65|32.18|389.83|77.97|117.53|146.63|21.10|189.92|227.68| +2452792|64346|9199|33869|1882238|6607|30529|33869|1882238|6607|30529|36|26|484|9|753.57|52.74|806.31|28.79|393.48|625.46|108.89|19.22|475.01| +2452641|41580|9906|26562|1542902|842|8382|26562|1542902|842|8382|55|9|484|12|347.16|31.24|378.40|55.86|88.56|104.14|153.10|89.92|175.66| +2451748|52174|12710|84641|1416158|828|35545|84641|1416158|828|35545|8|30|485|1|96.46|5.78|102.24|34.89|7.71|56.91|12.65|26.90|48.38| +2452425|53|5223|81263|292141|5066|23591|81263|292141|5066|23591|37|19|487|29|1748.99|0.00|1748.99|61.66|365.40|699.59|619.14|430.26|427.06| +2451907|69601|10507|78753|1032434|2562|9043|78753|1032434|2562|9043|17|10|488|25|3211.50|128.46|3339.96|68.59|487.75|2504.97|565.22|141.31|684.80| +2452603|66385|16908|29191|1613696|601|34306|29191|1613696|601|34306|21|26|491|18|1108.08|0.00|1108.08|71.08|360.72|775.65|6.64|325.79|431.80| +2452130|46385|14575|92585|546896|6454|49565|92585|546896|6454|49565|27|25|492|11|237.05|4.74|241.79|42.36|181.83|113.78|94.91|28.36|228.93| +2452146|81254|11163|41046|1629313|427|17894|41046|1629313|427|17894|27|10|492|11|88.22|2.64|90.86|92.23|8.36|24.70|1.90|61.62|103.23| +2452147|62609|9979|48727|897925|6323|29474|48727|897925|6323|29474|33|20|492|32|72.64|2.17|74.81|63.42|37.12|11.62|31.73|29.29|102.71| +2452408|70320|8669|68201|517287|5573|18612|68201|517287|5573|18612|19|29|493|75|1564.50|93.87|1658.37|84.80|898.50|453.70|722.02|388.78|1077.17| +||13619||137684|155||||155||47||494|46||336.75||35.88|1877.72|0.00|336.75||| +2452512|52572|6469|76113|1164027|1094|34208|76113|1164027|1094|34208|21|15|495|43|7813.10|0.00|7813.10|40.92|1544.13|3750.28|3737.79|325.03|1585.05| +2451258|60064|14542|83888|808395|6855|16515|83888|808395|6855|16515|14|7|499|10|2036.00|101.80|2137.80|98.75|641.70|0.00|1649.16|386.84|842.25| +2451224|60840|10472|38019|113130|2994|7705|38019|113130|2994|7705|14|29|499|42|794.64|55.62|850.26|83.60|722.40|532.40|89.16|173.08|861.62| +2451626|68426|3538|38376|1699423|6870|31070|38376|1699423|6870|31070|50|3|500|30|1567.20|141.04|1708.24|46.77|117.30|1410.48|72.09|84.63|305.11| +2451588|47091|7186|14887|675171|3455|27292|14887|675171|3455|27292|32|1|500|10|267.80|8.03|275.83|17.88|364.50|24.10|104.79|138.91|390.41| +2451715|34630|3661|18574|613306|1649|2672|18574|613306|1649|2672|40|6|501|3|84.36|4.21|88.57|79.58|4.20|52.30|3.20|28.86|87.99| +2451631|22112|16531|59391|1909468|5145|38469|59391|1909468|5145|38469|46|2|501|4|229.60|2.29|231.89|84.53|70.08|197.45|1.92|30.23|156.90| +2452174|14817|17681|76758|245572|1586|800|76758|245572|1586|800|35|29|502|79|2655.19|106.20|2761.39|87.51|1480.46|1911.73|468.37|275.09|1674.17| +2452199|52086|7943|51013|517281|1949|47417|51013|517281|1949|47417|31|28|503|25|1439.25|71.96|1511.21|14.61|248.00|863.55|253.30|322.40|334.57| +2452887|61184|5328|17953|991464|175|4086|17953|991464|175|4086|55|27|504|19|1308.53|52.34|1360.87|4.96|1594.67|1138.42|1.70|168.41|1651.97| +2451591|32290|5818|36346|1812345|791|11190|36346|1812345|791|11190|50|20|505|15|0.00|0.00|0.00|69.02|293.40|0.00|0.00|0.00|362.42| +2452053|72175|3845|33300|443416|1979|495|33300|443416|1979|495|38|27|506|2|53.10|0.53|53.63|23.71|18.10|38.76|14.05|0.29|42.34| +2452164|34834|656|72685|629833|581|10129|72685|629833|581|10129|49|5|506|22|332.86|26.62|359.48|9.36|152.68|146.45|113.71|72.70|188.66| +2451400|34581|757|69348|804|2307|11010|69348|804|2307|11010|1|1|507|11|1280.18|64.00|1344.18|74.44|884.51|345.64|654.17|280.37|1022.95| +2451289|40212|10204|67583|1663453|2523|2393|67583|1663453|2523|2393|46|14|507|2|129.20|1.29|130.49|71.39|8.06|32.30|11.62|85.28|80.74| +2452209|70525|8863|30345|316581|2817|11053|30345|316581|2817|11053|37|34|510|1|50.37|2.51|52.88|95.49|36.13|8.05|41.47|0.85|134.13| +2452192|66645|12331|76514|724646|6378|44110|76514|724646|6378|44110|11|35|511|13|282.75|2.82|285.57|41.12|267.02|33.93|191.59|57.23|310.96| +2452063|46217|187|11052|1494466|3833|23647|11052|1494466|3833|23647|20|9|511|5|57.30|1.71|59.01|7.81|67.75|26.93|26.42|3.95|77.27| +2451111|35124|5102|43943|1231545|1900|29455|43943|1231545|1900|29455|52|31|513|24|1656.24|66.24|1722.48|27.17|858.72|1026.86|106.99|522.39|952.13| +2452476|36552|3085|87168|165214|1660|48553|87168|165214|1660|48553|5|9|514|61|2123.41|169.87|2293.28|40.05|1445.70|637.02|178.36|1308.03|1655.62| +2452312|65387|14047|56431|653318|3805|2692|56431|653318|3805|2692|45|35|514|24|2870.16|86.10|2956.26|79.54|1161.60|459.22|602.73|1808.21|1327.24| +2451980|43913|11695|65481|964821|608|4795|65481|964821|608|4795|20|35|516|2|271.46|19.00|290.46|96.16|86.18|217.16|48.32|5.98|201.34| +2452074|31388|5330|52280|1506762|3197|37632|52280|1506762|3197|37632|8|18|516|1|3.57|0.28|3.85|15.38|3.35|0.60|0.05|2.92|19.01| +2452338|53724|13685|38759|565420|4434|5448|38759|565420|4434|5448|57|4|518|50|5022.00|401.76|5423.76|22.32|1575.50|3666.06|1166.10|189.84|1999.58| +2452443|38714|4777|23163|381042|2358|33625|23163|381042|2358|33625|33|27|518|9|440.91|26.45|467.36|22.36|134.73|308.63|5.29|126.99|183.54| +2452852|66593|10831|32018|114682|6946|44010|32018|114682|6946|44010|60|29|519|26|78.00|3.90|81.90|49.72|146.12|64.74|11.13|2.13|199.74| +2452316|45063|755|22582|921363|4591|29372|22582|921363|4591|29372|53|34|520|19|435.67|8.71|444.38|53.58|1394.41|121.98|116.06|197.63|1456.70| +2452432|75028|11007|16112|627630|2701|35419|16112|627630|2701|35419|3|8|520|1|51.68|0.00|51.68|93.73|16.99|3.10|4.85|43.73|110.72| +2452462|35052|16284|29054|405570|6086|1402|29054|405570|6086|1402|39|23|522|8|202.80|8.11|210.91|79.11|155.68|119.65|69.01|14.14|242.90| +2452442|69856|11091|86663|1017264|6712|13713|86663|1017264|6712|13713|51|32|522|4|65.32|0.65|65.97|57.97|226.20|60.74|3.52|1.06|284.82| +2451321|32907|2047|20292|1416337|1535|9533|20292|1416337|1535|9533|58|33|523|1|45.14|2.70|47.84|21.21|25.39|29.79|0.30|15.05|49.30| +2451333|53709|14236|63975|910762|2957|12990|63975|910762|2957|12990|10|14|523|9|347.13|3.47|350.60|58.92|105.84|222.16|1.24|123.73|168.23| +2452065|66063|17309|1522|1142982|6842|1055|1522|1142982|6842|1055|41|34|524|3|299.34|26.94|326.28|83.51|3.24|41.90|200.80|56.64|113.69| +2451921|39966|787|58514|74261|3161|41708|58514|74261|3161|41708|38|29|524|59|234.82|11.74|246.56|43.08|234.82|46.96|133.38|54.48|289.64| +2451562|7574|3644|52491|1750959|180|15848|52491|1750959|180|15848|10|32|526|46|692.30|0.00|692.30|46.87|225.40|560.76|119.70|11.84|272.27| +2451694|44202|11728|8340|955255|1370|5532|8340|955255|1370|5532|14|11|527|1|50.38|3.02|53.40|63.41|3.28|47.35|0.51|2.52|69.71| +2452379|38573|4379|20352|1087229|4859|2089|20352|1087229|4859|2089|29|33|528|39|1174.29|82.20|1256.49|98.74|765.96|1080.34|47.91|46.04|946.90| +2452262|20676|737|97623|345503|2983|41616|97623|345503|2983|41616|15|19|528|33|2033.46|81.33|2114.79|47.21|1220.01|813.38|988.26|231.82|1348.55| +2451765|44640|254|99244|1534773|2146|27000|99244|1534773|2146|27000|59|25|529|9|20.70|0.82|21.52|27.36|41.49|20.07|0.23|0.40|69.67| +2451793|44925|6479|75340|1103743|6193|39043|75340|1103743|6193|39043|8|18|529|45|1699.65|0.00|1699.65|53.72|553.95|1580.67|115.41|3.57|607.67| +2451675|80161|11939|32145|1222207|6316|41178|32145|1222207|6316|41178|11|1|529|6|256.98|5.13|262.11|63.18|194.94|218.43|32.76|5.79|263.25| +2451202|66123|13930|1334|1582953|2972|41791|1334|1582953|2972|41791|58|13|530|9|305.19|24.41|329.60|23.01|49.14|12.20|240.25|52.74|96.56| +2452363|32777|16139|64911|1007177|3043|49709|64911|1007177|3043|49709|45|23|531|25|177.50|0.00|177.50|85.42|516.25|133.12|16.86|27.52|601.67| +2451296|45207|16064|27600|140126|652|15578|27600|140126|652|15578|14|26|532|16|629.28|50.34|679.62|50.94|58.88|578.93|45.81|4.54|160.16| +2451388|49498|12799|70730|308310|3490|25070|70730|308310|3490|25070|55|15|532|12|1363.32|0.00|1363.32|20.84|984.60|749.82|263.80|349.70|1005.44| +2452046|70508|9908|44015|693316|6170|49471|44015|693316|6170|49471|17|18|533|40|7966.00|79.66|8045.66|85.89|785.20|2549.12|1841.73|3575.15|950.75| +2451296|39500|14738|51127|186294|1108|39475|51127|186294|1108|39475|55|29|534|48|3071.04|245.68|3316.72|3.15|3169.92|1996.17|53.74|1021.13|3418.75| +2452370|81986|12865|85323|1324972|4621|24670|85323|1324972|4621|24670|9|4|535|22|966.02|28.98|995.00|33.12|53.46|850.09|42.89|73.04|115.56| +2452214|72624|10843|28438|701352|2713|19227|28438|701352|2713|19227|21|12|535|6|84.30|2.52|86.82|87.76|67.74|59.01|1.77|23.52|158.02| +2452369|31592|3489|55423|1861429|5035|14641|55423|1861429|5035|14641|53|34|535|65|438.75|17.55|456.30|83.09|180.70|302.73|73.45|62.57|281.34| +2452834|44459|15001|88722|327315|6708|148|88722|327315|6708|148|48|31|536|13|138.19|1.38|139.57|0.94|102.44|120.22|8.44|9.53|104.76| +2451769|33268|4388|63133|1529698|1063|22028|63133|1529698|1063|22028|31|6|537|40|628.00|56.52|684.52|84.67|2367.60|508.68|41.76|77.56|2508.79| +2452481|54990|14103|34716|1728646|4976|22906|34716|1728646|4976|22906|15|10|538|29|308.56|27.77|336.33|44.62|462.84|249.93|41.04|17.59|535.23| +2451252|35671|14498|9257|393718|6979|48287|9257|393718|6979|48287|32|13|542|14|25.48|0.50|25.98|81.80|16.38|23.69|0.76|1.03|98.68| +2451349|70622|8371|14802|1826113|4624|35182|14802|1826113|4624|35182|28|10|542|47|2092.91|62.78|2155.69|94.57|277.30|1318.53|410.42|363.96|434.65| +2451533|60084|4813|52244|1436817|3790|33163|52244|1436817|3790|33163|7|5|543|23|1028.79|51.43|1080.22|60.55|318.32|113.16|137.34|778.29|430.30| +|8310|15229|||2003||||||||545|4||26.54||89.37|177.88||44.90|83.41|293.79| +2451382|11095|2762|80081|35202|6323|22244|80081|35202|6323|22244|13|14|547|45|420.75|29.45|450.20|34.44|3788.10|408.12|4.92|7.71|3851.99| +2451287|49728|8797|2286|976589|1292|38842|2286|976589|1292|38842|32|2|548|29|1252.22|12.52|1264.74|30.56|1762.33|200.35|830.97|220.90|1805.41| +2452716|3841|8970|12891|1913338|2585|7385|12891|1913338|2585|7385|25|23|549|5|15.20|1.36|16.56|20.16|100.60|13.52|0.06|1.62|122.12| +2451500|47550|10870|70371|1612807|4843|38832|70371|1612807|4843|38832|31|4|551|25|3369.75|269.58|3639.33|23.69|205.25|1010.92|353.82|2005.01|498.52| +2452906||4734||16859||||16859|612|||30|553|34||7.96|207.20||1395.70||||1423.16| +2452731|65415|3157|27681|1244613|5545|7677|27681|1244613|5545|7677|57|15|553|27|603.18|48.25|651.43|61.51|186.03|271.43|92.89|238.86|295.79| +2452080|66885|8773|79814|677481|6348|5615|79814|677481|6348|5615|9|19|554|6|463.08|18.52|481.60|38.62|298.74|134.29|279.47|49.32|355.88| +2452263|8001|11935|61163|986231|4452|11873|61163|986231|4452|11873|57|16|554|4|80.20|0.80|81.00|19.88|49.64|58.54|21.66|0.00|70.32| +2452230||1757|||3991|15074||||15074|19|6|555|20|1721.00||1738.21|41.09|491.60||58.68||| +2451598|55289|4946|16149|726030|6758|29039|16149|726030|6758|29039|1|2|556|44|242.88|21.85|264.73|98.22|2794.00|87.43|133.68|21.77|2914.07| +2452099|73798|9781|18697|1730398|946|24647|18697|1730398|946|24647|26|9|557|3|63.45|0.00|63.45|60.97|6.87|31.72|23.79|7.94|67.84| +2451653|32172|6259|17587|862596|4405|6614|17587|862596|4405|6614|16|20|558|13|11.57|0.46|12.03|40.72|13.26|7.28|2.44|1.85|54.44| +2451483|36996|15670|36274|1664380|1286|21792|36274|1664380|1286|21792|52|31|558|1|36.57|1.09|37.66|82.77|25.75|32.91|2.45|1.21|109.61| +2451400|33462|4237|46973|972552|4084|24816|46973|972552|4084|24816|38|22|559|56|3759.28|150.37|3909.65|93.36|1683.36|2894.64|760.88|103.76|1927.09| +2451062|36186|3416|20170|1812380|3790|26000|20170|1812380|3790|26000|19|7|561|5|134.45|8.06|142.51|81.83|124.30|99.49|34.96|0.00|214.19| +2452056|50014|11468|2549|512192|2635|23525|2549|512192|2635|23525|19|5|563|64|3675.52|257.28|3932.80|20.02|7685.76|3271.21|173.85|230.46|7963.06| +2451223|76270|3706|41445|1554563|834|38305|41445|1554563|834|38305|26|23|564|3|418.80|25.12|443.92|36.09|20.43|67.00|38.69|313.11|81.64| +2452491|38518|15483|38640|1833519|398|34477|38640|1833519|398|34477|9|21|571|35|4067.35|284.71|4352.06|70.60|813.40|2318.38|1661.52|87.45|1168.71| +2452407|17335|3018|65571|619193|1519|33614|65571|619193|1519|33614|49|3|571|36|370.08|3.70|373.78|71.20|634.32|240.55|51.81|77.72|709.22| +2452041|19271|14990|35723|244845|3567|13136|35723|244845|3567|13136|2|10|573|85|8277.30|413.86|8691.16|48.60|3762.10|1407.14|3091.57|3778.59|4224.56| +2451964|66997|1967|64333|1604105|5124|46096|64333|1604105|5124|46096|47|23|573|7|615.23|43.06|658.29|19.60|187.95|233.78|240.31|141.14|250.61| +2452568|71344|3079|51955|1451479|1403|10527|51955|1451479|1403|10527|45|20|574|11|481.69|0.00|481.69|58.27|30.03|274.56|55.92|151.21|88.30| +2452527|3060|17550|99311|187706|5701|6896|99311|187706|5701|6896|43|13|574|9|107.28|0.00|107.28|54.43|9.18|84.75|17.57|4.96|63.61| +2452484|46525|2955|11068|1819003|2938|34920|11068|1819003|2938|34920|60|22|574|21|646.38|32.31|678.69|29.14|114.03|252.08|67.03|327.27|175.48| +2451043|72749|10267|57753|596034|3115|21843|57753|596034|3115|21843|46|18|577|12|384.24|19.21|403.45|56.06|156.12|126.79|141.59|115.86|231.39| +2451131|59546|13003|43622|490734|2646|24958|43622|490734|2646|24958|10|25|577|48|2749.92|164.99|2914.91|22.74|845.76|0.00|549.98|2199.94|1033.49| +2452628|6673|7057|39416|216524|5185|32693|39416|216524|5185|32693|25|8|578|6|397.26|27.80|425.06|3.81|64.20|11.91|381.49|3.86|95.81| +2452749|79367|13818|86508|398310|4065|14220|86508|398310|4065|14220|42|6|578|59|329.22|3.29|332.51|81.22|36.58|207.40|36.54|85.28|121.09| +2452196|42994|2717|72492|640642|4345|7295|72492|640642|4345|7295|1|16|579|6|528.60|26.43|555.03|63.92|37.74|322.44|24.73|181.43|128.09| +2452028|14687|10123|8684|968093|1102|34511|8684|968093|1102|34511|50|10|580|4|291.32|26.21|317.53|72.10|269.76|37.87|139.39|114.06|368.07| +2452026|66123|1715|86623|1458897|4324|20551|86623|1458897|4324|20551|23|32|580|25|2134.75|64.04|2198.79|1.98|573.50|2070.70|6.40|57.65|639.52| +2452030|72069|434|11522|1513616|2557|39851|11522|1513616|2557|39851|8|1|580|1|2.83|0.14|2.97|86.50|1.65|2.29|0.34|0.20|88.29| +2452454|14601|6889|44637|1060908|3226|10297|44637|1060908|3226|10297|11|22|583|51|3532.77|70.65|3603.42|6.25|569.67|2684.90|618.94|228.93|646.57| +||7397|||6087||55537|1878812|6087||35|34|583|87||391.76|5988.47|||||1057.78|2283.01| +2452483|14709|16329|68795|1543318|6641|22364|68795|1543318|6641|22364|23|4|584|8|49.68|2.98|52.66|55.34|33.12|42.72|6.26|0.70|91.44| +2451783|14407|15128|86620|459166|6559|49905|86620|459166|6559|49905|19|26|585|25|990.75|49.53|1040.28|70.82|231.00|49.53|696.50|244.72|351.35| +2451321|28840|1264|89904|1355976|6048|13320|89904|1355976|6048|13320|49|2|586|4|7.72|0.38|8.10|45.19|115.84|7.56|0.09|0.07|161.41| +2451363|37265|5755|8539|1236314|6042||8539|1236314|||14|28|586|||0.00|45.36|||29.93|10.64|4.79|| +2452425|20500|7943|76770|582825|4080|41735|76770|582825|4080|41735|57|15|588|24|93.84|4.69|98.53|95.26|429.60|62.87|14.86|16.11|529.55| +2452327|54293|7543|64173|69536|6546|7509|64173|69536|6546|7509|43|18|588|13|478.01|14.34|492.35|24.74|51.74|234.22|80.45|163.34|90.82| +2452017|39009|15527|37193|728154|1162|4523|37193|728154|1162|4523|23|8|589|60|2198.40|109.92|2308.32|93.76|2650.80|1055.23|537.28|605.89|2854.48| +2452098|41709|7943|74722|1448492|1109|38479|74722|1448492|1109|38479|32|2|589|4|941.20|56.47|997.67|85.47|138.68|225.88|257.51|457.81|280.62| +2451870|79406|4658|51015|1596650|1838|42673|51015|1596650|1838|42673|8|34|589|34|341.36|10.24|351.60|64.97|409.70|116.06|78.85|146.45|484.91| +2452028|33956|7075|93942|1252574|1253|43485|93942|1252574|1253|43485|2|29|589|58|3.48|0.24|3.72|49.02|174.58|0.13|0.30|3.05|223.84| +2451693|25604|4951|77139|751937|7182|4827|77139|751937|7182|4827|38|25|593|19|2187.09|65.61|2252.70|89.06|914.85|459.28|1537.75|190.06|1069.52| +2451138|44092|2966|40218|1001975|6132|44289|40218|1001975|6132|44289|50|10|594|52|128.44|1.28|129.72|3.20|60.84|20.55|18.34|89.55|65.32| +2451204||4051||1742074|1862|31250|23086||||50|29|594|45||202.87||21.30|||31.95|120.21|| +2451345|77839|11311|54301|401159|5264|4969|54301|401159|5264|4969|7|27|595|7|611.87|48.94|660.81|61.35|230.23|562.92|38.67|10.28|340.52| +2452900|73734|4873|65413|881249|7038|38074|65413|881249|7038|38074|9|14|596|14|1391.74|55.66|1447.40|95.45|490.00|487.10|72.37|832.27|641.11| +2452048|7537|6805|84629|69822|4615|5761|84629|69822|4615|5761|5|35|597|31|3232.99|161.64|3394.63|88.52|631.47|484.94|164.88|2583.17|881.63| +2452270|74695|2405|63282|1100166|6203|3040|63282|1100166|6203|3040|27|14|598|68|656.88|13.13|670.01|37.60|1188.64|177.35|302.10|177.43|1239.37| +2452395|72023|57|85176|1764209|4042|19075|85176|1764209|4042|19075|43|28|599|7|9.24|0.55|9.79|49.38|3.36|8.68|0.19|0.37|53.29| +2450937|46424|12085|44334|1139245|2956|37067|44334|1139245|2956|37067|34|23|600|35|672.35|47.06|719.41|12.05|288.05|349.62|287.22|35.51|347.16| +2451001|26584|11726|74416|1834114|4212|663|74416|1834114|4212|663|1|12|600|49|9557.45|860.17|10417.62|96.66|5202.33|3153.95|3073.68|3329.82|6159.16| +2450946|53668|14755|78104|213066|5181|28496|78104|213066|5181|28496|2|8|600|27|126.36|3.79|130.15|53.80|221.40|111.19|8.49|6.68|278.99| +2451097|62664|2290|53114|401252|2190|46728|53114|401252|2190|46728|1|21|601|21|344.19|27.53|371.72|51.26|418.11|340.74|2.82|0.63|496.90| +2452260|40803|5235|29828|750632|4236|21040|29828|750632|4236|21040|5|19|602|21|381.57|7.63|389.20|43.25|671.58|289.99|5.49|86.09|722.46| +2452142|79606|10899|27867|1835978|4556|4144|27867|1835978|4556|4144|23|8|602|3|214.02|4.28|218.30|16.42|10.68|96.30|47.08|70.64|31.38| +2452252|71673|17845|13486|1673297|1108|1719|13486|1673297|1108|1719|7|28|602|11|399.08|35.91|434.99|69.04|41.91|231.46|5.02|162.60|146.86| +2452167|40422|9241|91578|1622191|4004|36217|91578|1622191|4004|36217|31|29|602|24|941.52|56.49|998.01|37.39|443.04|160.05|429.80|351.67|536.92| +2451708|62233|14698|99053|1261231|6609|3940|99053|1261231|6609|3940|1|26|603|2|79.96|4.79|84.75|47.67|42.32|7.19|50.93|21.84|94.78| +2451679|46171|13700|35040|1018410|5333|3103|35040|1018410|5333|3103|58|27|603|3|180.24|7.20|187.44|44.35|92.31|131.57|30.66|18.01|143.86| +2451699|75420|15652|35208|1396464|920|44176|35208|1396464|920|44176|46|20|603|2|118.18|9.45|127.63|24.36|27.72|76.81|2.06|39.31|61.53| +2452700|72659|1087|97928|248640|6552|64|97928|248640|6552|64|9|33|605|7|437.01|34.96|471.97|55.45|227.99|69.92|326.71|40.38|318.40| +2452784|64341|16812|1687|1825267|5647|37274|1687|1825267|5647|37274|49|34|605|10|18.30|0.36|18.66|62.95|6.30|15.55|2.03|0.72|69.61| +2452366|60845|13639|16257|1145578|852|34573|16257|1145578|852|34573|7|13|606|4|285.80|22.86|308.66|60.90|21.16|148.61|5.48|131.71|104.92| +2452136|72960|7981|87056|630748|4840|13732|87056|630748|4840|13732|53|16|607|1|1.29|0.00|1.29|87.06|5.84|0.36|0.07|0.86|92.90| +2451956|48356|836|15216|870676|4536|13996|15216|870676|4536|13996|38|33|607|3|66.54|1.99|68.53|36.08|3.69|28.61|9.10|28.83|41.76| +2452313|53623|6163|13637|326112|4187|29909|13637|326112|4187|29909|5|11|608|12|909.72|45.48|955.20|29.46|341.04|9.09|648.45|252.18|415.98| +2451668|35822|3982|14156|598117|4217|26974|14156|598117|4217|26974|49|26|610|13|461.24|41.51|502.75|63.61|103.48|161.43|80.94|218.87|208.60| +2451717|37858|4873|4248|1358119|6390|39861|4248|1358119|6390|39861|22|29|610|17|1367.65|54.70|1422.35|6.29|203.15|1203.53|60.72|103.40|264.14| +2452745|49077|3933|46892|869761|6230|45852|46892|869761|6230|45852|39|22|611|69|993.60|69.55|1063.15|15.02|205.62|486.86|501.67|5.07|290.19| +2452806|50965|8833|41058|597281|6304|44527|41058|597281|6304|44527|7|5|611|9|110.16|8.81|118.97|84.06|17.91|1.10|25.08|83.98|110.78| +2452732|33230|9120|25468|1798970|119|17621|25468|1798970|119|17621|39|9|612|86|1842.12|165.79|2007.91|82.70|1137.78|1068.42|471.95|301.75|1386.27| +2451186|16673|6241|43488|437226|3764|37549|43488|437226|3764|37549|13|4|613|57|496.47|34.75|531.22|30.44|1228.35|278.02|152.91|65.54|1293.54| +2452002|15055|8105|74023|1755002|1228|1407|74023|1755002|1228|1407|37|2|614|10|132.00|1.32|133.32|57.98|15.60|75.24|34.05|22.71|74.90| +2451182|63542|10267|29539|457518|6650|35782|29539|457518|6650|35782|55|5|617|3|124.80|8.73|133.53|6.53|73.17|98.59|24.63|1.58|88.43| +2451230|72264|7670|95376|656556|7198|46517|95376|656556|7198|46517|7|5|617|5|82.75|2.48|85.23|36.91|124.10|3.31|15.88|63.56|163.49| +2452284|37270|14543|13868|1165751|6434|23117|13868|1165751|6434|23117|9|5|618|23|73.37|0.00|73.37|80.59|90.62|15.40|25.50|32.47|171.21| +2452377|38777|11345|47090|1015551|3565|29979|47090|1015551|3565|29979|17|24|618|12|691.32|34.56|725.88|45.34|724.32|414.79|143.79|132.74|804.22| +2451289|43753|9722|19359|1875539|3719|21452|19359|1875539|3719|21452|40|20|619|13|150.28|0.00|150.28|67.72|216.06|43.58|46.94|59.76|283.78| +2451104|32420|829|40124|1843892|||40124||2860|18228|49||620|||10.30|525.58|||371.00||28.86|234.09| +2452365|69386|10705|95322|692990|3458|45126|95322|692990|3458|45126|11|2|621|20|2726.80|245.41|2972.21|68.17|1378.40|545.36|763.50|1417.94|1691.98| +2452478|76379|9021|70035|914045|3208|10418|70035|914045|3208|10418|35|2|621|28|1289.12|51.56|1340.68|80.08|618.24|373.84|73.22|842.06|749.88| +2452782|25289|14953|29550|1301570|343|21116|29550|1301570|343|21116|3|33|624|89|1794.24|71.76|1866.00|46.49|608.76|1596.87|55.26|142.11|727.01| +2452143|32266|9515|54955|1164022|736|5850|54955|1164022|736|5850|2|13|625|14|651.84|13.03|664.87|34.85|639.10|52.14|425.78|173.92|686.98| +2451286|45629|5114|70145|1878466|1187|42708|70145|1878466|1187|42708|34|31|626|68|101.32|2.02|103.34|12.23|187.00|3.03|54.05|44.24|201.25| +2451803|40919|17575|99023|203476|2329|41182|99023|203476|2329|41182|25|23|627|16|88.16|7.05|95.21|55.37|40.80|71.40|9.72|7.04|103.22| +2451887|40701|15182|61858|62751|4444|42890|61858|62751|4444|42890|50|28|627|17|1154.64|69.27|1223.91|9.00|400.52|992.99|45.26|116.39|478.79| +2452619|75697|2538|58881|1702575|4669|35916|58881|1702575|4669|35916|6|5|628|31|486.08|38.88|524.96|89.17|640.77|301.36|149.62|35.10|768.82| +2451488|72973|5473|61825||4203||||4203||||630|56|363.44||396.14|79.58||174.45|90.71||| +2451602|35397|3290|48847|1054508|6216|37877|48847|1054508|6216|37877|28|10|631|10|59.90|4.19|64.09|98.23|575.30|50.91|5.66|3.33|677.72| +2451585|48345|9938|50928|498406|4284|35762|50928|498406|4284|35762|25|2|631|52|1139.84|68.39|1208.23|45.94|333.84|752.29|81.38|306.17|448.17| +2451552|72468|4663|26426|1141374|4322|30423|26426|1141374|4322|30423|40|27|631|30|280.20|2.80|283.00|41.28|776.10|246.57|11.77|21.86|820.18| +2452366|62457|11713|23996|1256937|7031|12372|23996|1256937|7031|12372|21|15|632|71|1571.23|15.71|1586.94|67.19|2155.56|1539.80|7.85|23.58|2238.46| +2452772|25693|1947|47245|750749|4653|9357|47245|750749|4653|9357|45|16|633|82|1036.48|10.36|1046.84|92.90|1295.60|466.41|370.54|199.53|1398.86| +2451713|40104|16246|16397|216644|4891|27138|16397|216644|4891|27138|58|26|634|10|449.70|8.99|458.69|18.14|132.80|166.38|192.65|90.67|159.93| +2451067|14762|3076|23709|708654|3199|6991|23709|708654|3199|6991|56|33|635|35|0.00|0.00|0.00|58.73|218.40|0.00|0.00|0.00|277.13| +||21|25326||1444|31903|25326|279385||31903|9||636||1053.15|||76.68|481.95|||134.18|| +2452065|20969|164|21694|206859|3224|32389|21694|206859|3224|32389|47|34|637|5|33.25|0.66|33.91|26.45|72.55|4.98|19.78|8.49|99.66| +2452663|28063|5485|73547|1573298|1049|30000|73547|1573298|1049|30000|25|15|638|41|68.88|1.37|70.25|97.91|103.32|57.17|7.72|3.99|202.60| +2451981|74957|6643|79511|1287616|2121|7379|79511|1287616|2121|7379|13|29|639|23|113.85|3.41|117.26|28.84|281.29|29.60|56.44|27.81|313.54| +2451277|41149|133|81626|123423|3620|29417|81626|123423|3620|29417|38|26|640|10|7.10|0.35|7.45|57.83|5.30|0.07|6.67|0.36|63.48| +2451580|41615|12901|76445|1088891|6137|38583|76445|1088891|6137|38583|2|25|644|40|4877.20|341.40|5218.60|7.89|1286.00|926.66|2488.84|1461.70|1635.29| +2451446|72138|6280|90772|724706|593|47280|90772|724706|593|47280|19|14|644|32|5888.64|58.88|5947.52|90.20|512.00|2002.13|3070.34|816.17|661.08| +2451550|48910|5446|37147|1732161|2375|40542|37147|1732161|2375|40542|58|19|645|19|1040.06|72.80|1112.86|65.08|100.51|436.82|506.72|96.52|238.39| +2451403|24712|6607|16133|620862|5591|41120|16133|620862|5591|41120|13|16|646|6|735.36|22.06|757.42|78.88|183.84|205.90|243.55|285.91|284.78| +2451461|75785|6944|13113|549170|874|32931|13113|549170|874|32931|26|35|646|6|231.06|13.86|244.92|80.06|123.78|0.00|83.18|147.88|217.70| +2452142|18368|9223|96207|187943|2879|5815|96207|187943|2879|5815|19|24|647|37|472.86|0.00|472.86|8.26|0.00|472.86|0.00|0.00|8.26| +2451716|76614|6230|63790|171409|1428|34247|63790|171409|1428|34247|10|19|648|20|847.60|76.28|923.88|99.88|514.60|228.85|222.75|396.00|690.76| +2451739|61240|5186|50756|167017|2478|31186|50756|167017|2478|31186|46|34|648|1|8.44|0.42|8.86|13.02|4.07|3.62|3.08|1.74|17.51| +2451654|43801|1672|55450|1843980|6110|240|55450|1843980|6110|240|26|10|648|22|115.94|2.31|118.25|10.79|56.76|27.82|1.76|86.36|69.86| +2451709|51418|8668|22057|786535|1108|36948|22057|786535|1108|36948|7|30|649|71|148.39|13.35|161.74|32.10|340.09|56.38|9.20|82.81|385.54| +||2283||||48374|||1764|48374|25||650||3774.92||4114.66|45.35|910.86|2793.44||422.04|1295.95| +|71810|11530|17103||5802|47128|||||||651||3848.07|346.32|4194.39||755.51||182.40|48.49|| +2452508|71425|4865|72087|640091|95|48452|72087|640091|95|48452|21|22|652|43|401.19|32.09|433.28|51.25|256.28|361.07|22.46|17.66|339.62| +2451885|38165|5516|57753|1518521|3736|43351|57753|1518521|3736|43351|7|12|653|30|843.60|8.43|852.03|35.40|937.50|843.60|0.00|0.00|981.33| +2451614|44308|14311|22053|575307|6961|47415|22053|575307|6961|47415|56|27|654|8|61.84|5.56|67.40|48.86|284.48|57.51|2.59|1.74|338.90| +2451724|67530|5377|9878|507294|4006|21306|9878|507294|4006|21306|56|34|654|52|798.20|15.96|814.16|82.18|704.60|638.56|46.29|113.35|802.74| +2452812|69234|2679|94076|534007|3538|2269|94076|534007|3538|2269|31|13|655|40|1328.00|79.68|1407.68|43.26|806.40|571.04|52.98|703.98|929.34| +2452152|77832|4677|90036|1490052|6068|32531|90036|1490052|6068|32531|47|34|656|25|1782.25|160.40|1942.65|69.27|1230.50|213.87|784.19|784.19|1460.17| +2452872|78466|15477|33783|1692436|3536|1836|33783|1692436|3536|1836|24|17|657|20|1939.20|135.74|2074.94|18.59|897.20|639.93|584.67|714.60|1051.53| +2451599|48217|6385|83152|1424793|6434|24875|83152|1424793|6434|24875|31|4|658|1|132.88|9.30|142.18|0.94|54.48|130.22|0.63|2.03|64.72| +2451625|37264|12367|48812|300938|711|6037|48812|300938|711|6037|56|7|658|59|2396.58|215.69|2612.27|64.48|374.06|1509.84|434.50|452.24|654.23| +2451520||746|44140|1339022|5551|30080||1339022|5551|30080||12|658|84||50.71||||912.79|3492.96|665.33|| +2451487|82860|379|60665|523156|6447|40138|60665|523156|6447|40138|43|4|658|15|31.65|0.63|32.28|24.11|179.55|21.52|2.32|7.81|204.29| +2452497|40248|16361|45657|1103986|3562|17891|45657|1103986|3562|17891|25|29|660|69|4117.23|41.17|4158.40|26.62|202.17|576.41|283.26|3257.56|269.96| +2451367|38544|9151|12260|252508|6356|27451|12260|252508|6356|27451|49|20|661|11|154.88|1.54|156.42|56.50|37.51|133.19|17.78|3.91|95.55| +2452508|33727|8785|41142|656130|5071|37259|41142|656130|5071|37259|51|15|662|7|36.12|0.00|36.12|18.30|12.53|36.12|0.00|0.00|30.83| +2452632|73307|1129|76162|858534|3306|22878|76162|858534|3306|22878|37|25|663|38|190.00|3.80|193.80|27.52|1140.76|131.10|13.54|45.36|1172.08| +2452494|61610|1735|90717|760267|6170|47720|90717|760267|6170|47720|1|31|664|4|105.96|8.47|114.43|72.93|81.12|38.14|54.25|13.57|162.52| +2451986|20521|2615|22525|1565073|6821|15722|22525|1565073|6821|15722|49|10|665|23|54.05|3.78|57.83|21.68|974.28|5.94|26.46|21.65|999.74| +2452111|62417|4871|65741|155863|32|49754|65741|155863|32|49754|37|29|665|25|1779.50|142.36|1921.86|49.29|1403.00|1174.47|6.05|598.98|1594.65| +2452074|26967|7553|60177|78871|1118|44965|60177|78871|1118|44965|7|8|666|65|243.75|2.43|246.18|86.64|487.50|121.87|2.43|119.45|576.57| +2452671|62125|4603|1353|198745|2090|38682|1353|198745|2090|38682|9|30|668|8|451.60|13.54|465.14|34.24|482.40|261.92|108.11|81.57|530.18| +2452085|67340|11803|69615|228362|3909|8703|69615|228362|3909|8703|41|17|669|80|5595.20|447.61|6042.81|60.42|632.80|3245.21|1245.49|1104.50|1140.83| +2451993|74133|13579|33535|342266|4244|11375|33535|342266|4244|11375|37|32|671|8|1660.48|116.23|1776.71|53.22|749.20|1162.33|498.15|0.00|918.65| +2452003|72330|9685|54020|277158|7143|34281|54020|277158|7143|34281|13|18|671|30|25.80|1.80|27.60|46.02|116.40|17.28|6.56|1.96|164.22| +2451226||10855||||||1542195|1885||||672||27.70||28.25||7.20||14.46||21.06| +2451485|9141|2209|19408|671380|170|27334|19408|671380|170|27334|8|18|674|6|229.20|0.00|229.20|80.80|108.54|135.22|2.81|91.17|189.34| +2452310|25554|6817|53222|1226249|876|21503|53222|1226249|876|21503|45|30|675|40|1623.20|81.16|1704.36|79.85|2028.80|292.17|918.41|412.62|2189.81| +2452359|73145|3457|40307|182961|4368|17375|40307|182961|4368|17375|37|6|675|61|8857.20|265.71|9122.91|43.55|4992.24|5225.74|3413.57|217.89|5301.50| +2452440|67865|919|78728|1095130|4779|43581|78728|1095130|4779|43581|35|7|676|7|272.16|0.00|272.16|39.95|177.52|187.79|80.15|4.22|217.47| +2452529|61213|10791|72868|852033|798|5563|72868|852033|798|5563|9|19|676|47|1149.62|11.49|1161.11|22.26|396.68|643.78|465.37|40.47|430.43| +2452072|28720|1937|10362|90064|2566|47915|10362|90064|2566|47915|37|21|678|25|989.75|59.38|1049.13|47.07|400.50|910.57|68.88|10.30|506.95| +2451275|45755|3997|53867|284483|1499|22468|53867|284483|1499|22468|1|35|679|26|214.76|19.32|234.08|66.52|117.00|57.98|17.24|139.54|202.84| +2452600|39867|4791|49021|1473245|1022|7646|49021|1473245|1022|7646|3|27|680|50|1151.00|23.02|1174.02|1.09|1239.50|529.46|441.29|180.25|1263.61| +2452602|77326|1629|||852|9594|||852|9594|36|13|680|3|||12.81||2.16|2.41||1.26|63.92| +2451877|60266|4718|18802|1797055|5355|45208|18802|1797055|5355|45208|25|17|682|7|823.20|32.92|856.12|35.92|98.77|650.32|93.35|79.53|167.61| +2451817|53150|8912|92876|1375749|5822|30695|92876|1375749|5822|30695|43|31|682|5|329.45|23.06|352.51|51.51|123.05|322.86|0.13|6.46|197.62| +|14294|13319|99679|616508||31319|||5690||37|29|683|20|573.80|0.00|573.80||0.00||41.77||8.70| +2452028||4676|29874||5251|9916|||5251|9916|43||683||1611.84||1756.90|82.24|3223.68|467.43|778.19|366.22|| +2452114|62834|15967|77290|535169|248|30974|77290|535169|248|30974|23|12|683|2|28.14|1.68|29.82|94.19|4.50|27.57|0.19|0.38|100.37| +2452032|73470|1231|23168|1881310|7005|1500|23168|1881310|7005|1500|41|14|683|53|2413.09|24.13|2437.22|75.47|386.90|1061.75|1229.71|121.63|486.50| +2452066|84417|12373|21158|1211499|143|34066|21158|1211499|143|34066|23|11|684|31|993.24|69.52|1062.76|76.53|116.87|387.36|521.05|84.83|262.92| +2452088|67781|14054|61504|1156086|7115|23297|61504|1156086|7115|23297|11|7|684|4|66.24|1.98|68.22|38.99|101.60|31.13|26.33|8.78|142.57| +2452057|59711|15854|59983|1802547|7064|48028|59983|1802547|7064|48028|37|34|686|34|3168.80|0.00|3168.80|88.39|2851.92|1267.52|1406.94|494.34|2940.31| +2451964|70498|12235|15362|1280893|6424|28355|15362|1280893|6424|28355|19|3|686|2|43.18|2.59|45.77|65.69|29.68|16.40|11.24|15.54|97.96| +2452140||4825|55959|579526|5493||55959|579526|||||686|3|59.40|||37.19|47.52|11.28|||84.71| +2452043|78100|3113|91836|1682674|1959|44797|91836|1682674|1959|44797|44|11|687|22|1750.32|122.52|1872.84|47.05|700.04|1190.21|520.90|39.21|869.61| +2451924|57199|15008|26859|67755|6251|16300|26859|67755|6251|16300|5|32|687|5|245.35|9.81|255.16|60.42|188.25|41.70|34.62|169.03|258.48| +2451979|47062|12620|64636|1360734|6785|83|64636|1360734|6785|83|7|21|687|27|0.00|0.00|0.00|99.60|276.21|0.00|0.00|0.00|375.81| +2451396|72886|12836|47933|764507|3834|28008|47933|764507|3834|28008|40|3|690|4|127.44|3.82|131.26|38.48|9.48|56.07|54.24|17.13|51.78| +2452324|40879|4177|55697|117271|2398|19172|55697|117271|2398|19172|29|32|691|15|2312.85|69.38|2382.23|40.76|902.55|855.75|1151.10|306.00|1012.69| +2452692|35083|3063|95345|1514004|1110|33769|95345|1514004|1110|33769|9|10|692|67|905.84|36.23|942.07|39.90|2978.15|371.39|427.56|106.89|3054.28| +2451324|36813|152|43843|1080956|469|8335|43843|1080956|469|8335|20|21|695|1|101.32|0.00|101.32|65.59|58.11|74.97|2.63|23.72|123.70| +2451318|63408|9367|64745|1838650|1802|17707|64745|1838650|1802|17707|38|18|695|30|694.20|62.47|756.67|80.58|570.60|590.07|73.93|30.20|713.65| +2451190|9033|9895|49418|1223980|142|44657|49418|1223980|142|44657|49|5|695|2|280.08|2.80|282.88|27.61|36.30|14.00|2.66|263.42|66.71| +2451227|31289|1102|40125|1711567|609|6098|40125|1711567|609|6098|28|9|696|50|2074.00|62.22|2136.22|64.41|553.00|518.50|311.10|1244.40|679.63| +2451206|46902|17263|12400|1016009|6818|28474|12400|1016009|6818|28474|43|33|696|1|51.26|2.05|53.31|52.95|44.49|29.73|16.79|4.74|99.49| +2451998||10429||1308502|394||65410|1308502||||18|697||503.50|35.24|538.74|65.10|127.70|493.43|||228.04| +2451996|69376|8231|16629|1629886|2442|8062|16629|1629886|2442|8062|29|14|697|11|57.42|3.44|60.86|33.50|16.61|51.10|3.72|2.60|53.55| +2451173|21204|13538|65449|69614|6579|31944|65449|69614|6579|31944|14|26|698|41|1396.46|27.92|1424.38|25.21|465.35|139.64|640.97|615.85|518.48| +||9898|22808|||35076||84113|5418|35076|34||698|73|211.70|14.81|226.51||4667.62|122.78|||| +2451372|40897|5758|11298|286833|6368|20303|11298|286833|6368|20303|44|33|699|33|4652.01|186.08|4838.09|10.79|2401.08|1349.08|1981.75|1321.18|2597.95| +2452747|74116|14754|12499|281160|574|24435|12499|281160|574|24435|55|34|700|65|5150.60|0.00|5150.60|51.64|6581.25|3553.91|367.23|1229.46|6632.89| +2451988|45190|8159|56525|618797|3022|38094|56525|618797|3022|38094|11|22|703|10|519.80|46.78|566.58|93.75|99.00|509.40|7.59|2.81|239.53| +2451929|46098|12745|66373|1732804|5988|18019|66373|1732804|5988|18019|20|11|705|10|610.00|12.20|622.20|24.85|248.50|469.70|127.67|12.63|285.55| +2452151|43236|8359|89958|965885|565|28492|89958|965885|565|28492|53|18|705|10|125.80|7.54|133.34|20.85|338.70|91.83|11.21|22.76|367.09| +2451940|60949|10508|84835|1264335|2235|36982|84835|1264335|2235|36982|7|29|705|64|263.04|5.26|268.30|20.34|168.32|207.80|11.04|44.20|193.92| +2451943|25603|12032|8845|747074|1656|37085|8845|747074|1656|37085|23|21|705|13|527.93|31.67|559.60|52.53|791.96|480.41|19.95|27.57|876.16| +2451693|64176|16807|97394|701917|6440|27363|97394|701917|6440|27363|1|21|706|57|10342.65|103.42|10446.07|44.97|1524.18|6619.29|3723.36|0.00|1672.57| +2452546|58122|2005|35826|1542517|6753|11723|35826|1542517|6753|11723|36|33|707|32|4542.08|363.36|4905.44|95.92|843.52|3497.40|783.51|261.17|1302.80| +2452706|50017|8880|84424|271542|4707|5025|84424|271542|4707|5025|27|12|707|56|760.48|22.81|783.29|18.71|582.96|479.10|123.80|157.58|624.48| +2452491|7021|427|34205|197098|2038|23378|34205|197098|2038|23378|30|4|707|74|75.48|4.52|80.00|55.21|341.14|29.43|44.20|1.85|400.87| +2451070|1971|12481|45821|659795|3369|12990|45821|659795|3369|12990|43|28|708|12|2490.24|49.80|2540.04|49.29|1059.00|1792.97|216.15|481.12|1158.09| +2451874|71899|2294||1554336|465|44994|3760|||44994||29|709|||343.18|7206.85|||3569.10|2207.36||474.94| +2451584|25859|7459|73238|1310975|712|20496|73238|1310975|712|20496|13|22|710|49|844.27|67.54|911.81|76.44|2674.42|135.08|368.77|340.42|2818.40| +2451659|45351|3548|12225|1404404|5383|24762|12225|1404404|5383|24762|31|15|710|69|24.15|0.48|24.63|80.82|237.36|3.86|17.65|2.64|318.66| +2451711|43869|8638|52109|488605|3948|3244|52109|488605|3948|3244|52|21|710|54|2701.08|135.05|2836.13|40.25|1902.42|2566.02|74.28|60.78|2077.72| +2451543|85001|3452|47074|1444259|6597|8182|47074|1444259|6597|8182|37|19|711|41|3558.39|71.16|3629.55|6.63|2874.10|71.16|1011.29|2475.94|2951.89| +2451555|70773|15958|78789|73269|6949|21429|78789|73269|6949|21429|58|19|712|8|381.92|34.37|416.29|42.72|0.00|183.32|115.18|83.42|77.09| +2451680|57486|9769|53974|163866|3005|39344|53974|163866|3005|39344|8|3|712|78|85.02|0.85|85.87|45.36|652.86|25.50|7.73|51.79|699.07| +2451780|258|4609|27552|477441|4577|46113|27552|477441|4577|46113|14|12|712|85|2350.25|47.00|2397.25|16.94|118.15|94.01|564.06|1692.18|182.09| +2451538|70651|5146|96233|42997|608|35372|96233|42997|608|35372|52|6|713|4|180.88|14.47|195.35|29.63|24.24|3.61|161.31|15.96|68.34| +2451546|35975|16976|85642|1510542|6995|26695|85642|1510542|6995|26695|10|14|713|17|1144.10|22.88|1166.98|29.68|963.39|102.96|312.34|728.80|1015.95| +2451525|29105|2554|32326|1917659|3792|21125|32326|1917659|3792|21125|46|33|713|41|399.34|0.00|399.34|27.20|445.67|275.54|87.89|35.91|472.87| +2452514|58594|13881|17516|1490656|2996|2536|17516|1490656|2996|2536|35|30|717|7|71.19|3.55|74.74|0.71|36.40|29.18|7.14|34.87|40.66| +2451296|71152|3530|27415|1670454|693|17332|27415|1670454|693|17332|44|11|719|46|1919.58|95.97|2015.55|81.00|1002.34|1132.55|338.42|448.61|1179.31| +2451213|19142|596|80744|1483863|1735|10854|80744|1483863|1735|10854|10|14|719|10|438.20|17.52|455.72|28.35|352.50|197.19|48.20|192.81|398.37| +2451970|37940|17419|8131|697854|3224|21747|8131|697854|3224|21747|43|1|721|8|296.96|14.84|311.80|89.49|0.00|124.72|87.84|84.40|104.33| +2451933|69494|12199|742|35679|2263|12000|742|35679|2263|12000|59|6|721|2|128.40|3.85|132.25|52.54|10.84|21.82|18.11|88.47|67.23| +2452250|40650|8091|15864|1504219|1727|8435|15864|1504219|1727|8435|43|34|722|56|1583.12|94.98|1678.10|48.28|3166.24|1029.02|338.00|216.10|3309.50| +2451922|34883|2135|44995|1468167|939|36911|44995|1468167|939|36911|25|9|724|4|110.88|7.76|118.64|47.80|129.36|26.61|67.41|16.86|184.92| +2452101|73913|15227|89412|960130|2048|13166|89412|960130|2048|13166|43|26|724|5|190.40|0.00|190.40|79.87|436.85|36.17|3.08|151.15|516.72| +2452086|60779|896|55663|553505|1747|20789|55663|553505|1747|20789|56|33|724|2|88.36|7.06|95.42|61.03|59.30|17.67|43.82|26.87|127.39| +2451983|41621|13322|56749|1683072|948|2071|56749|1683072|948|2071|32|4|724|75|985.50|9.85|995.35|4.44|454.50|315.36|113.92|556.22|468.79| +2451099|79590|13412|74329|1658050|1418|36339|74329|1658050|1418|36339|50|27|726|57|2132.37|149.26|2281.63|31.74|359.67|1108.83|92.11|931.43|540.67| +|18089|410|||452|8774||307777|452||||726|12|56.40||60.91|||||31.95|| +2451391|70178|8707|56608|1222826|2291|6109|56608|1222826|2291|6109|31|30|727|41|992.20|89.29|1081.49|59.66|555.14|138.90|503.44|349.86|704.09| +2451401|37419|11581|50598|1708483|7168|5500|50598|1708483|7168|5500|31|9|727|32|2492.48|224.32|2716.80|54.16|813.12|2168.45|103.68|220.35|1091.60| +2451625|40820|652|75423|899038|5406|24621|75423|899038|5406|24621|31|17|731|3|514.80|25.74|540.54|34.13|122.04|303.73|99.20|111.87|181.91| +2451967|47384|11357|93969|236327|5885|30996|93969|236327|5885|30996|20|34|733|3|257.34|0.00|257.34|14.74|170.40|2.57|124.83|129.94|185.14| +2451898|45862|7577|89296|1790140|3790|7382|89296|1790140|3790|7382|43|33|734|73|2689.32|161.35|2850.67|65.33|1445.40|2554.85|44.37|90.10|1672.08| +2451908|59466|1895|92218|1145390|1409|3028|92218|1145390|1409|3028|13|5|734|13|391.69|27.41|419.10|89.16|335.66|78.33|125.34|188.02|452.23| +2451737|14023|5999|84850|14579|2355|7284|84850|14579|2355|7284|14|17|734|11|268.40|24.15|292.55|36.98|380.16|24.15|97.70|146.55|441.29| +2451816|76918|9149|61562|64114|266|16818|61562|64114|266|16818|37|8|734|11|1768.36|159.15|1927.51|53.32|214.28|1485.42|237.66|45.28|426.75| +2451206|44039|10178|94332|1902286|1877|25751|94332|1902286|1877|25751|19|15|736|20|125.20|10.01|135.21|31.26|198.20|27.54|85.94|11.72|239.47| +2451182|74048|11170|22267|1579354|888|16422|22267|1579354|888|16422|25|17|736|41|5211.10|208.44|5419.54|26.15|2210.72|3543.54|1150.61|516.95|2445.31| +2451222|39072|910|51938|703369|2000|19478|51938|703369|2000|19478|20|3|736|6|470.40|42.33|512.73|63.05|74.82|221.08|234.36|14.96|180.20| +2451809|45963|5348|32546|910811|3476|18988|32546|910811|3476|18988|7|10|737|31|1221.09|73.26|1294.35|71.17|466.86|549.49|161.18|510.42|611.29| +2451778|74921|6478|42735|464072|162|25099|42735|464072|162|25099|50|30|737|59|6258.13|250.32|6508.45|28.39|4059.20|688.39|946.85|4622.89|4337.91| +2451650|36836|2098|51064|922579|6094|6409|51064|922579|6094|6409|25|27|737|4|17.40|1.21|18.61|16.45|0.56|13.05|4.30|0.05|18.22| +2452603|12296|9348|57267|582105|1343|39381|57267|582105|1343|39381|42|11|738|6|501.90|5.01|506.91|94.27|106.74|95.36|264.25|142.29|206.02| +2452403|46730|6041|43859|1897756|4806|23903|43859|1897756|4806|23903|35|4|739|21|393.33|11.79|405.12|3.02|418.11|365.79|3.85|23.69|432.92| +2452440|67432|13205|88262|1558777|1739|49054|88262|1558777|1739|49054|11|33|739|8|443.28|31.02|474.30|36.78|18.40|390.08|51.60|1.60|86.20| +2452376||1613|10321|741276||37350|10321||3814||45|28|739|68||||11.05|543.32|747.70|||| +2452441|48819|4725|74434|1883182|6330|25831|74434|1883182|6330|25831|55|27|739|4|351.48|3.51|354.99|4.41|245.24|17.57|277.14|56.77|253.16| +2452139|44738|14657|5804|309143|4035|20527|5804|309143|4035|20527|8|24|740|77|3907.75|117.23|4024.98|25.27|1096.48|1992.95|957.40|957.40|1238.98| +2451898|52925|11924|44577|375353|176|8599|44577|375353|176|8599|2|18|740|9|1830.69|91.53|1922.22|72.73|762.75|750.58|939.69|140.42|927.01| +2451657||7639||933658|3737|||||||27|741||393.60|||||110.20|172.87|110.53|281.49| +2451665|41523|14084|64577|584372|4967|42362|64577|584372|4967|42362|43|30|741|6|194.82|7.79|202.61|72.94|438.42|56.49|11.06|127.27|519.15| +2451536|36574|2275|29437|1197287|1584|33034|29437|1197287|1584|33034|14|20|741|80|833.60|16.67|850.27|41.67|105.60|266.75|272.08|294.77|163.94| +2452811|61016|6840|37388|888806|224|16842|37388|888806|224|16842|48|8|742|12|456.96|31.98|488.94|48.91|212.76|438.68|15.35|2.93|293.65| +2452750|55169|14844|88499|979013|5527|22841|88499|979013|5527|22841|36|30|742|13|2179.97|65.39|2245.36|81.68|741.65|1373.38|516.21|290.38|888.72| +2452740|35140|9331|53346|1238233|5167|28454|53346|1238233|5167|28454|13|2|742|38|4558.10|227.90|4786.00|10.07|2223.38|957.20|396.09|3204.81|2461.35| +2452556|14380|7377|58387|672071|3355|18639|58387|672071|3355|18639|7|34|743|35|722.75|7.22|729.97|17.69|2439.50|115.64|84.99|522.12|2464.41| +2451471|37499|8077|80331|367471|6258|5398|80331|367471|6258|5398|55|23|744|1|6.57|0.00|6.57|90.80|19.20|4.53|0.55|1.49|110.00| +2451215|50391|4879|93494|356285|6276|19467|93494|356285|6276|19467|1|1|745|24|515.76|46.41|562.17|87.86|468.96|201.14|144.72|169.90|603.23| +2451953|33594|7952|96445|11323|7001|37832|96445|11323|7001|37832|29|6|746|19|1300.36|78.02|1378.38|77.14|1155.96|286.07|963.57|50.72|1311.12| +2451854|36712|86|20319|727454|2942|14830|20319|727454|2942|14830|26|7|746|52|4707.56|376.60|5084.16|65.75|903.24|2542.08|411.44|1754.04|1345.59| +2451951|41482|12218|35437|1684278|5919|49160|35437|1684278|5919|49160|59|35|746|8|160.72|11.25|171.97|86.67|19.36|135.00|12.60|13.12|117.28| +2451398|61140|16075|87920|201607|6141|3475|87920|201607|6141|3475|46|5|747|4|677.88|20.33|698.21|15.29|183.80|122.01|505.84|50.03|219.42| +2451625|15730|7382|79825|988013|7048|38883|79825|988013|7048|38883|19|19|749|2|160.32|6.41|166.73|7.52|37.82|158.71|1.48|0.13|51.75| +2451755|67571|9106|29513|236545|2036|7715|29513|236545|2036|7715|37|24|749|4|85.76|2.57|88.33|6.83|115.60|34.30|10.29|41.17|125.00| +2451765|29608|7696|5999|395393|6034|13875|5999|395393|6034|13875|20|16|749|29|3130.55|125.22|3255.77|24.64|70.18|313.05|2310.35|507.15|220.04| +2451701|67183|12919|87873|1336874|3828|23982|87873|1336874|3828|23982|8|30|749|13|1259.57|50.38|1309.95|76.08|662.87|302.29|851.97|105.31|789.33| +2451053|5764|9488|29330|1247713|943|24526|29330|1247713|943|24526|50|7|751|48|2151.84|150.62|2302.46|90.00|1050.72|559.47|621.02|971.35|1291.34| +2450901|35146|13417|64990|1024677|4011|28498|64990|1024677|4011|28498|34|18|751|4|186.20|11.17|197.37|45.26|20.96|104.27|18.84|63.09|77.39| +2452510|24158|3847|49639|1646731|6223|7268|49639|1646731|6223|7268|39|23|752|20|1879.80|75.19|1954.99|6.28|250.60|1579.03|264.67|36.10|332.07| +2452517|41218|8643|41701|1223335|4969|37003|41701|1223335|4969|37003|30|5|752|18|800.82|64.06|864.88|51.99|592.56|120.12|435.64|245.06|708.61| +2450992|47994|10172|67088|825106|3124|20342|67088|825106|3124|20342|10|18|753|6|106.74|7.47|114.21|87.22|33.84|2.13|27.19|77.42|128.53| +2451537|83329|16681|23110|1327374|85|42349|23110|1327374|85|42349|26|23|754|73|4842.82|290.56|5133.38|93.06|662.11|1694.98|2297.92|849.92|1045.73| +2451609|69652|6157|87241|617585|5176|11920|87241|617585|5176|11920|44|29|754|9|494.28|0.00|494.28|22.55|10.35|291.62|123.62|79.04|32.90| +2452214|33906|6085|56362|616724|1886|9001|56362|616724|1886|9001|5|6|755|3|56.16|1.12|57.28|13.70|129.90|28.08|18.81|9.27|144.72| +2451718|75888|4741|83124|1175451||6817|83124||452||56||757|12|1006.32||1066.69|48.44|||573.20||566.13| +2452717|56793|15097|59412|1907970|2928|21933|59412|1907970|2928|21933|18|23|759|32|1555.84|108.90|1664.74|96.91|1842.56|108.90|1418.00|28.94|2048.37| +2451605|28176|1988|36782|1343818|2054|35843|36782|1343818|2054|35843|4|15|760|46|2630.28|52.60|2682.88|65.97|1181.74|2130.52|374.82|124.94|1300.31| +2451923|32904|7376|98098|742006|667|27328|98098|742006|667|27328|29|15|761|7|93.45|0.93|94.38|64.35|244.09|13.08|39.38|40.99|309.37| +2451983|19722|5192|52742|404752|6009|11619|52742|404752|6009|11619|50|20|761|69|3647.34|182.36|3829.70|32.17|4310.43|0.00|3464.97|182.37|4524.96| +|35239|9369||1205956|||||1655|||3|762||4.51|0.18|4.69|||2.07|||74.02| +2452832|66365|15156|13852|644318|31|17697|13852|644318|31|17697|60|22|762|58|8297.48|663.79|8961.27|56.69|691.36|1410.57|2066.07|4820.84|1411.84| +2452371|67532|4027||1711441|||||1137|34080||1|764|||0.00|||||30.97|26.40|607.50| +|77035|12013|73085|686794||10212||||||29|765||6401.37||6593.41|75.42|757.02|128.02|||| +2452587|76872|6979|48236|1023945|2054|35363|48236|1023945|2054|35363|49|3|765|5|11.75|0.82|12.57|52.88|108.50|7.87|1.66|2.22|162.20| +2452483|57952|13947|59295|905834|20|14959|59295|905834|20|14959|21|35|766|2|107.18|5.35|112.53|66.04|41.56|61.09|15.20|30.89|112.95| +|30987|15731|22207||1722|||1264188|1722|45256||29|766||95.28||||24.00||1.52|8.01|| +2452277|44212|7011|675|1804131|34|39490|675|1804131|34|39490|9|13|766|43|470.42|14.11|484.53|31.63|2118.18|432.78|3.38|34.26|2163.92| +2451778|58787|14660|53402|601923|4491|37459|53402|601923|4491|37459|50|30|767|50|2774.00|249.66|3023.66|13.24|2774.00|1081.86|1099.89|592.25|3036.90| +2451356|35199|17383|35648|1862618|2035|41405|35648|1862618|2035|41405|4|22|768|53|3502.24|315.20|3817.44|32.39|1072.19|2556.63|255.31|690.30|1419.78| +2451380|17103|8509|48|1787168|3088|33808|48|1787168|3088|33808|46|35|768|34|1687.08|0.00|1687.08|57.57|1095.14|1383.40|261.16|42.52|1152.71| +2452590|49436|2835|5155|1698106|4657|48008|5155|1698106|4657|48008|13|7|770|26|4016.48|361.48|4377.96|41.02|1396.98|281.15|1456.77|2278.56|1799.48| +2452658|60647|12930|41525|1464596|3220|37891|41525|1464596|3220|37891|30|16|770|3|53.64|3.21|56.85|69.51|29.25|41.30|11.84|0.50|101.97| +2452513|66700|4891|68793|1655392|2438|48637|68793|1655392|2438|48637|53|1|772|29|1280.06|38.40|1318.46|10.85|88.16|128.00|806.44|345.62|137.41| +2452501|32895|6857|17815|1845772|4929|24933|17815|1845772|4929|24933|39|22|772|9|8.64|0.51|9.15|18.35|11.52|5.44|2.04|1.16|30.38| +2452443|57853|17477|39336|476006|5123|44240|39336|476006|5123|44240|5|5|772|5|159.45|6.37|165.82|15.30|120.95|105.23|43.37|10.85|142.62| +|73427|12649|27669||602|||655159||||20|772||76.25|||26.47|234.00|53.37||5.95|267.33| +2451743|66909|16795|68005|999712|5019|33126|68005|999712|5019|33126|56|32|774|17|119.00|9.52|128.52|45.54|109.48|74.97|13.20|30.83|164.54| +2452778|47204|5961|94711|1375790|4516|20842|94711|1375790|4516|20842|30|17|775|12|868.44|26.05|894.49|21.15|234.72|112.89|279.55|476.00|281.92| +2452339|48165|12145|74550|1557956|6162|35168|74550|1557956|6162|35168|25|25|776|18|89.82|2.69|92.51|68.07|539.46|34.13|6.12|49.57|610.22| +2452289|63403|12931|71556|1438948|5893|38885|71556|1438948|5893|38885|31|33|779|20|358.20|14.32|372.52|25.56|102.20|111.04|232.33|14.83|142.08| +2451143|65523|16147||831853|||1360||||22||783||989.30|||47.14|||113.57||424.16| +2451335|66114|5785|73613|993048|6436|48224|73613|993048|6436|48224|58|2|788|36|5380.92|161.42|5542.34|69.45|2337.12|807.13|1646.56|2927.23|2567.99| +2452072|36418|3251|58459|1915267|875|8036|58459|1915267|875|8036|51|17|789|43|2259.22|67.77|2326.99|32.27|881.50|1445.90|740.12|73.20|981.54| +2451148|68187|12295|54405|1302909|566|16270|54405|1302909|566|16270|37|35|790|36|122.04|7.32|129.36|4.78|65.16|31.73|82.18|8.13|77.26| +2451324|73095|5176|30151|113518|6838|16158|30151|113518|6838|16158|37|2|791|24|39.84|1.19|41.03|65.81|293.28|6.37|9.37|24.10|360.28| +2451274|59358|6794|74433|168363|1046|28976|74433|168363|1046|28976|38|20|791|27|196.02|17.64|213.66|23.75|0.00|168.57|12.90|14.55|41.39| +2451308|39684|12260|85069|776213|3357|13254|85069|776213|3357|13254|25|29|792|7|342.16|30.79|372.95|92.47|40.60|242.93|89.30|9.93|163.86| +2451347|55578|15734|78057|1472383|3156|32282|78057|1472383|3156|32282|58|24|792|15|244.20|9.76|253.96|70.43|556.35|78.14|43.17|122.89|636.54| +2452187|66193|4215|77313|1082261|4809|8110|77313|1082261|4809|8110|57|22|793|61|8826.70|529.60|9356.30|35.50|1586.61|8561.89|169.47|95.34|2151.71| +2451556|65742|5791|42185|579872|312|31152|42185|579872|312|31152|1|19|794|1|7.76|0.38|8.14|36.09|0.40|7.52|0.08|0.16|36.87| +2451701|76321|676|33340|1269577|567|18804|33340|1269577|567|18804|38|7|796|12|44.88|2.69|47.57|58.30|21.84|30.96|11.55|2.37|82.83| +2451000|51628|11743|60424|1062368|6130|42397|60424|1062368|6130|42397|43|33|797|9|50.31|4.52|54.83|41.10|13.68|48.29|0.80|1.22|59.30| +2451074|73912|5044|60526|363992|1858|77|60526|363992|1858|77|20|21|797|53|3523.44|0.00|3523.44|60.06|773.27|1973.12|170.53|1379.79|833.33| +2452411|35266|5017|53004|1803456|3508|4407|53004|1803456|3508|4407|15|3|798|11|555.17|33.31|588.48|63.60|795.85|510.75|34.64|9.78|892.76| +2452536|29104|199|57655|1466001|3147|46293|57655|1466001|3147|46293|48|17|799|8|3.76|0.03|3.79|9.24|152.00|1.65|0.97|1.14|161.27| +2452572|3646|6045|59747|773587|1380|30420|59747|773587|1380|30420|15|19|799|23|424.81|0.00|424.81|11.69|315.56|382.32|14.87|27.62|327.25| +2451531|83636|3194|38090|1714619||43583|38090||3501|43583|7|34|802||1409.54||1494.11||333.41|690.67|||| +2451688|37655|10330|76495|477531|4167|27250|76495|477531|4167|27250|13|10|802|53|305.81|27.52|333.33|71.61|50.88|73.39|69.72|162.70|150.01| +2452063|42140|4663|69335|551116|5697|27878|69335|551116|5697|27878|23|31|803|19|428.45|4.28|432.73|39.38|473.67|154.24|222.11|52.10|517.33| +2451869|27310|7184|66869|999562|608|38854|66869|999562|608|38854|56|16|803|60|106.20|0.00|106.20|24.84|49.80|56.28|25.95|23.97|74.64| +2451799|38026|6139|86636|756405|5374|13143|86636|756405|5374|13143|8|29|804|72|4783.68|382.69|5166.37|92.72|2858.40|2439.67|1242.32|1101.69|3333.81| +2452064|53996|7073|46993|1053973|6200|14380|46993|1053973|6200|14380|20|4|805|10|669.50|13.39|682.89|19.55|152.10|174.07|460.74|34.69|185.04| +2451699|61939|8540|60319|421150|4381|33212|60319|421150|4381|33212|7|25|806|1|57.83|2.31|60.14|93.34|0.00|13.87|10.11|33.85|95.65| +2452465|31520|4383|57977|1620955|3933|2111|57977|1620955|3933|2111|45|23|807|5|63.95|5.11|69.06|9.24|14.50|8.31|25.59|30.05|28.85| +2452546|38855|10849|30704|130006|2108|44210|30704|130006|2108|44210|9|14|808|1|55.29|1.10|56.39|43.70|82.94|7.74|14.26|33.29|127.74| +2452464|60075|13477|79777|1258096|4862|45981|79777|1258096|4862|45981|49|24|810|16|886.40|62.04|948.44|64.79|223.84|274.78|373.08|238.54|350.67| +2452051|49197|215|3895||4613|30572|||||||813|||177.60||6.04||2024.70|1527.42|0.00|2043.94| +2452001|68431|11318|71180|453042|1751|13213|71180|453042|1751|13213|56|29|814|1|1.07|0.01|1.08|0.64|8.61|0.16|0.60|0.31|9.26| +2452115|48302|4195|38729|1646815|1301|8987|38729|1646815|1301|8987|56|23|815|31|889.39|17.78|907.17|59.19|98.58|809.34|59.23|20.82|175.55| +2452338|50913|8625|98455|449727|6952|46624|98455|449727|6952|46624|47|27|816|8|0.32|0.02|0.34|35.39|16.16|0.12|0.08|0.12|51.57| +2452045|44505|2753|50932|1397941|4150|4712|50932|1397941|4150|4712|53|21|817|12|91.56|5.49|97.05|75.08|96.36|25.63|12.52|53.41|176.93| +2452104|36532|16073|8725|569046|1367|13688|8725|569046|1367|13688|38|29|817|2|190.32|3.80|194.12|79.50|17.62|171.28|16.75|2.29|100.92| +2452593|78292|13995|63610|1425829|4048|32220|63610|1425829|4048|32220|39|14|818|13|2568.93|0.00|2568.93|7.26|1027.52|2055.14|380.20|133.59|1034.78| +2451509|24621|16483|64856|252137|5465|30393|64856|252137|5465|30393|50|17|819|3|125.28|10.02|135.30|28.70|22.26|125.28|0.00|0.00|60.98| +2451495|33759|13862|78816|502486|2956|47026|78816|502486|2956|47026|55|28|824|5|96.30|7.70|104.00|36.79|75.65|85.70|2.01|8.59|120.14| +2451081|61538|10426|21453|601743|1233|43876|21453|601743|1233|43876|37|19|825|13|1307.15|117.64|1424.79|96.41|1102.01|261.43|962.06|83.66|1316.06| +2451150|66210|11812|48333|143251|3769|19840|48333|143251|3769|19840|37|6|825|16|224.96|20.24|245.20|44.97|58.88|20.24|180.15|24.57|124.09| +2451611|25577|6607|88090|853801|3959|16470|88090|853801|3959|16470|40|2|826|46|6536.60|196.09|6732.69|32.86|777.86|2483.90|2188.45|1864.25|1006.81| +2451596|66923|12529|3776|1807126|4876|23826|3776|1807126|4876|23826|19|29|826|13|12.61|1.00|13.61|23.14|1.56|8.95|2.37|1.29|25.70| +2451257|39630|11695|1582|1163800|4865|19130|1582|1163800|4865|19130|22|1|828|11|794.75|31.79|826.54|83.54|241.01|87.42|502.20|205.13|356.34| +2451050|71323|15896|72360|788947|4287|34027|72360|788947|4287|34027|38|14|828|4|303.80|15.19|318.99|42.85|143.20|121.52|178.63|3.65|201.24| +2452811|34272|3829|6007|23049|2844|42286|6007|23049|2844|42286|6|21|829|4|23.52|2.11|25.63|38.96|98.48|3.76|16.00|3.76|139.55| +2452783|32594|858|66712|1112982|6899|1969|66712|1112982|6899|1969|15|6|829|6|55.62|5.00|60.62|99.60|247.92|18.35|1.49|35.78|352.52| +2452757|25649|2970|72265|522917|416|4268|72265|522917|416|4268|24|7|830|73|901.55|18.03|919.58|27.26|1352.69|459.79|48.59|393.17|1397.98| +2452844|16201|16683|99260|415998|2336|31074|99260|415998|2336|31074|57|14|830|18|2102.04|42.04|2144.08|16.47|1320.48|1513.46|164.80|423.78|1378.99| +2451044|68143|8180|29021|1060062|703|32191|29021|1060062|703|32191|50|26|831|75|480.00|33.60|513.60|69.91|1386.75|172.80|3.07|304.13|1490.26| +2452843|54862|12285|34802|607396|1143|28937|34802|607396|1143|28937|9|27|833|13|225.68|2.25|227.93|69.90|101.40|11.28|12.86|201.54|173.55| +2452918|58735|6697|70669|1212468|5914|40113|70669|1212468|5914|40113|3|8|833|7|290.08|0.00|290.08|61.62|31.36|107.32|180.93|1.83|92.98| +2452119|45996|17768|71373|153646|366|20239|71373|153646|366|20239|38|28|837|30|1045.50|73.18|1118.68|82.03|1085.70|386.83|217.36|441.31|1240.91| +2451927|74314|2803|21694|1606742|1866|13361|21694|1606742|1866|13361|31|22|837|14|279.30|19.55|298.85|72.80|123.62|195.51|78.76|5.03|215.97| +2452361|41321|3665|49595|1232222|449|42723|49595|1232222|449|42723|45|2|838|4|99.76|0.00|99.76|19.06|68.36|58.85|24.54|16.37|87.42| +2451817|78776|11354|55876|767786|3249|5159|55876|767786|3249|5159|32|3|840|9|495.18|24.75|519.93|38.33|165.06|79.22|370.20|45.76|228.14| +2452002||10555|21284||5414|24679|21284||||44|33|840|||827.07|11165.47|95.79||4859.04|2520.50|2958.86|2384.46| +2452696|70489|2491|74542||3469|14556|||3469|14556||34|841||||3331.95|77.74||1035.17||2133.75|| +2452756|24078|906|96470|125316|2029|998|96470|125316|2029|998|49|10|842|13|15.86|0.47|16.33|64.78|319.15|7.45|8.07|0.34|384.40| +2451352|27291|11306|68159|1273614|4222|6077|68159|1273614|4222|6077|31|23|843|29|304.21|15.21|319.42|40.08|161.24|91.26|119.25|93.70|216.53| +2451284|81629|13220|9396|688712|1275|14591|9396|688712|1275|14591|40|28|843|60|2800.80|28.00|2828.80|82.73|903.60|1596.45|650.34|554.01|1014.33| +2451350|30257|5119|20878|2575|3025|11075|20878|2575|3025|11075|38|29|843|32|55.68|2.22|57.90|7.14|743.04|7.79|8.14|39.75|752.40| +2451680|3817|1351|34104|1223321|3291|35261|34104|1223321|3291|35261|37|8|844|1|28.77|2.30|31.07|23.62|4.98|0.28|20.51|7.98|30.90| +2451765|64315|2054|48901|441994|7087|5780|48901|441994|7087|5780|31|23|844|17|445.23|31.16|476.39|5.18|72.59|222.61|140.25|82.37|108.93| +2451646|36359|12907|91543|989249|3755|29861|91543|989249|3755|29861|37|20|844|6|474.48|23.72|498.20|5.75|245.82|346.37|42.27|85.84|275.29| +2451179|37339|9031|||2875||82877|1120476|2875||38||845|32||238.63|4215.91||44.16||||| +2451461|48640|13996|53725|923501|3677|25453|53725|923501|3677|25453|20|22|849|7|246.47|19.71|266.18|69.18|14.49|207.03|26.03|13.41|103.38| +2451451|69191|10309|47233|1702397|4798|42265|47233|1702397|4798|42265|13|31|849|1|193.71|17.43|211.14|83.20|0.00|112.35|67.52|13.84|100.63| +2451486|27367|13702|20506|201292|2078|17311|20506|201292|2078|17311|14|11|849|42|253.26|0.00|253.26|90.05|261.66|243.12|2.43|7.71|351.71| +2451370|42533|58|4544|203025|5506|6034|4544|203025|5506|6034|26|27|850|86|2476.80|148.60|2625.40|19.50|4000.72|1857.60|303.40|315.80|4168.82| +2451988|58974|12287|13570|1610499|5028|45557|13570|1610499|5028|45557|17|30|851|3|22.38|0.22|22.60|38.44|99.21|8.28|12.97|1.13|137.87| +2451957|76162|5237|84715|169090|3418|36193|84715|169090|3418|36193|13|8|851|5|243.50|2.43|245.93|82.77|171.65|43.83|103.82|95.85|256.85| +2452959|56897|2227|90745|1477141|3537|11933|90745|1477141|3537|11933|15|19|852|33|3327.72|299.49|3627.21|59.49|1440.12|1564.02|1481.50|282.20|1799.10| +2452884|26481|14169|74027|1572553|5126|9252|74027|1572553|5126|9252|1|34|852|21|488.04|0.00|488.04|77.35|262.71|463.63|7.32|17.09|340.06| +2452823|65611|9147|31067|1480982|497|43612|31067|1480982|497|43612|54|8|852|44|44.00|1.32|45.32|56.21|56.32|35.20|4.48|4.32|113.85| +2452843|58066|2118|13543|1266414|1238|21309|13543|1266414|1238|21309|60|10|852|14|153.58|7.67|161.25|24.69|95.34|75.25|33.68|44.65|127.70| +2452446|27717|8055|90242|651464|1486|19908|90242|651464|1486|19908|33|23|853|25|367.25|22.03|389.28|39.39|306.00|209.33|137.39|20.53|367.42| +2452479|82430|16859|1671|1638334|3797|28060|1671|1638334|3797|28060|35|26|853|5|384.10|30.72|414.82|35.90|203.70|261.18|38.10|84.82|270.32| +2451727|69418|8000|68750|181504|3505|42706|68750|181504|3505|42706|56|23|854|67|5002.89|250.14|5253.03|7.08|2648.51|650.37|3438.49|914.03|2905.73| +2452424|80657|10663|56043|1368665|3334|23156|56043|1368665|3334|23156|49|1|855|74|27.38|0.54|27.92|29.42|10.36|16.70|5.23|5.45|40.32| +2452518|21176|2267|93761|685239|3917|595|93761|685239|3917|595|37|33|855|2|16.76|1.00|17.76|28.83|18.84|14.24|1.18|1.34|48.67| +2450999|62457|11458|78129|1681589|696|49356|78129|1681589|696|49356|37|8|856|10|353.60|21.21|374.81|54.61|74.00|247.52|38.18|67.90|149.82| +2452225|38094|12125|97740|1715462|4737|22059|97740|1715462|4737|22059|9|8|858|42|3438.54|103.15|3541.69|42.47|1533.00|3197.84|64.98|175.72|1678.62| +2451274|62591|9572|96911|684937|4610|32283|96911|684937|4610|32283|4|1|859|30|576.60|5.76|582.36|21.39|1183.80|80.72|371.91|123.97|1210.95| +2451257|75682|11311|14167|1257052|5184|34069|14167|1257052|5184|34069|13|11|859|2|101.54|0.00|101.54|6.93|121.06|70.06|4.40|27.08|127.99| +2452470|69460|3205|14060|823855|6450|42692|14060|823855|6450|42692|5|26|861|6|20.40|1.02|21.42|47.55|40.86|11.42|0.35|8.63|89.43| +2452555|75021|105|46797|546547|2297|48447|46797|546547|2297|48447|37|20|861|37|8411.95|336.47|8748.42|9.52|277.13|3617.13|1917.92|2876.90|623.12| +2452338|48846|14913|90847|1563251|2855|40859|90847|1563251|2855|40859|55|14|861|30|1270.20|25.40|1295.60|20.82|712.50|139.72|1006.12|124.36|758.72| +|49148|12283|70695|1264019|||70695|1264019|||41||861|||4.15|56.09|46.99|51.94||||| +||14285||||3319||504455|200|3319|27|34|862|3||17.74|271.21|36.77||45.62|||71.79| +|16117|10657|32088|1141876||31052|32088|||||21|863|67|134.67|||||5.38|82.74||123.98| +2451885|61052|4286|13955|141600|892|8984|13955|141600|892|8984|44|12|863|22|271.70|5.43|277.13|29.93|648.12|141.28|22.17|108.25|683.48| +2452088|72308|6068|27110|1127257|1004|37551|27110|1127257|1004|37551|8|26|865|19|578.55|34.71|613.26|84.42|597.93|329.77|101.99|146.79|717.06| +2452787|13061|15636|26510|429698|2133|8121|26510|429698|2133|8121|39|20|866|60|3048.60|243.88|3292.48|76.81|247.20|1463.32|523.14|1062.14|567.89| +2452813|25539|16536|26022|476571|5967|28916|26022|476571|5967|28916|18|22|866|4|167.00|15.03|182.03|60.87|87.12|167.00|0.00|0.00|163.02| +2451348|38675|3313|29527|915921|7115|38262|29527|915921|7115|38262|49|11|867|65|753.35|45.20|798.55|5.36|860.60|632.81|91.61|28.93|911.16| +2451228|73039|11443|76635|130912|836|32713|76635|130912|836|32713|56|26|867|1|95.42|5.72|101.14|7.90|16.35|8.58|74.68|12.16|29.97| +2452584|35292|10399|66277|516452|2980|29955|66277|516452|2980|29955|36|24|868|60|3868.20|77.36|3945.56|39.07|1513.80|2630.37|557.02|680.81|1630.23| +2452378|10831|14433|93305|977198|5891|9784|93305|977198|5891|9784|55|32|868|5|198.55|7.94|206.49|95.28|78.85|125.08|71.26|2.21|182.07| +2452351|69093|12205|6401|1838388|3175|39962|6401|1838388|3175|39962|45|14|868|56|271.04|18.97|290.01|40.22|203.28|0.00|162.62|108.42|262.47| +2452303|32654|17057|57656|981712|968|28177|57656|981712|968|28177|9|32|869|10|26.00|1.30|27.30|85.17|32.50|12.74|9.81|3.45|118.97| +||16287|12256|1363159|2062|44165|12256||2062|44165|25||870|32|101.12|8.08||53.97|||||1073.89| +2451216|61252|14122|59212|1811352|4292|28832|59212|1811352|4292|28832|22|24|871|13|1153.62|103.82|1257.44|63.03|140.27|346.08|314.94|492.60|307.12| +2452170|64454|11383|95416|623415|4567|35366|95416|623415|4567|35366|33|13|873|9|412.65|12.37|425.02|30.89|6.48|202.19|42.09|168.37|49.74| +2452484|71524|1857|57382|1295522|2909|38696|57382|1295522|2909|38696|17|31|874|1|25.46|2.29|27.75|82.60|0.30|16.29|2.56|6.61|85.19| +2452415|61559|15355|87549|713502|255|40608|87549|713502|255|40608|39|28|875|16|952.16|9.52|961.68|84.71|237.92|485.60|247.27|219.29|332.15| +2451797|71056|11941|49142|1245253|2517|48628|49142|1245253|2517|48628|35|3|878|25|745.75|37.28|783.03|73.60|115.25|223.72|349.76|172.27|226.13| +2452640|38609|2011|68394|1555489|4184|8479|68394|1555489|4184|8479|3|32|879|13|193.70|0.00|193.70|75.35|895.83|83.29|24.29|86.12|971.18| +2452802|50359|9081|69525|1132238|2598|111|69525|1132238|2598|111|33|15|879|48|668.64|53.49|722.13|76.36|510.72|661.95|4.54|2.15|640.57| +2452770|24166|13674|88329|1227766|803|10640|88329|1227766|803|10640|54|17|879|9|72.00|1.44|73.44|97.21|261.18|25.20|15.91|30.89|359.83| +2452867|71334|5244|83040|990528|6369|1857|83040|990528|6369|1857|1|31|879|1|45.10|0.90|46.00|31.51|63.76|26.60|8.51|9.99|96.17| +2451687|50282|15622|54205|1302899|5546|35793|54205|1302899|5546|35793|49|20|880|12|1178.88|35.36|1214.24|91.83|23.76|58.94|526.37|593.57|150.95| +2451119|34250|7078|53112|521880|5848|12216|53112|521880|5848|12216|16|23|881|3|0.00|0.00|0.00|34.15|89.22|0.00|0.00|0.00|123.37| +2452813|68329|10179|42809|1321051|446|12688|42809|1321051|446|12688|12|4|883|41|3136.09|0.00|3136.09|55.52|1862.22|533.13|1275.45|1327.51|1917.74| +2452811|67718|17700|92214|1188415|7016|30370|92214|1188415|7016|30370|33|2|888|1|0.37|0.03|0.40|92.74|4.72|0.09|0.10|0.18|97.49| +2451865|37615|13369|44314|1001457|3400|40143|44314|1001457|3400|40143|35|1|889|63|1830.15|146.41|1976.56|21.29|121.59|1482.42|142.56|205.17|289.29| +|70040|10963|83209|1547535||||1547535|||20||889|||16.67||5.37||||33.85|63.09| +2451285|59518|1730|85256|1437194|1932|15070|85256|1437194|1932|15070|52|13|890|17|124.10|3.72|127.82|92.82|806.82|86.87|21.96|15.27|903.36| +2451895|20218|5185|98856|703862|2115|9568|98856|703862|2115|9568|14|2|891|3|95.31|7.62|102.93|12.74|9.75|63.85|12.26|19.20|30.11| +||9494||1686097||41465||1686097||41465|||891|3||5.82|296.82|||142.59|||142.22| +2452198|35304|9923|73799|425979|5359|1139|73799|425979|5359|1139|59|35|892|3|81.36|3.25|84.61|14.19|203.46|25.22|11.22|44.92|220.90| +2452637|52145|9378|86469|1641317|2118|36119|86469|1641317|2118|36119|37|7|893|1|5.21|0.36|5.57|39.18|3.47|3.69|0.83|0.69|43.01| +2451949|68180|17972|70618|328243|3115|11540|70618|328243|3115|11540|56|22|894|97|1385.16|96.96|1482.12|63.22|167.81|567.91|294.21|523.04|327.99| +2452608|60526|786|1300|1705960|1373|16033|1300|1705960|1373|16033|45|19|895|27|498.42|44.85|543.27|31.22|380.16|9.96|24.42|464.04|456.23| +2452581|67260|2965||||16206||||16206||35|896||5039.76|251.98|5291.74|90.14|1911.76|1461.53||3542.45|| +2451055|58185|16576|26401|1515394|2434|44627|26401|1515394|2434|44627|40|21|897|86|3627.48|290.19|3917.67|24.85|1339.02|580.39|1279.77|1767.32|1654.06| +2451013|69841|4204|73638|1102924|3057|12343|73638|1102924|3057|12343|1|28|897|25|1334.50|106.76|1441.26|82.50|539.00|827.39|294.12|212.99|728.26| +2451472|43684|11947|60362|947825|88|37504|60362|947825|88|37504|1|33|898|12|1105.20|66.31|1171.51|35.05|396.72|1072.04|7.29|25.87|498.08| +2452858|76464|6121|1743|505378|2966|1801|1743|505378|2966|1801|18|7|902|68|3901.16|156.04|4057.20|3.42|2030.48|351.10|923.01|2627.05|2189.94| +2452028||15313|42633||4214||42633||4214|12028|47||903||||1386.38||||265.96|757.00|297.65| +|48662|1387|||5670|13216|20978|1865724|5670|13216|||903|2||8.74||4.81|103.36||125.89||116.91| +2452369|65319|9965|95782|669812|1442|27919|95782|669812|1442|27919|29|19|904|69|600.99|30.04|631.03|12.32|362.94|246.40|251.75|102.84|405.30| +2452399|69983|3933|3622|1434087|2599|28264|3622|1434087|2599|28264|51|30|904|78|8711.04|696.88|9407.92|24.87|1984.32|609.77|8020.25|81.02|2706.07| +2452766|28712|12246|4033|1349908|230|32152|4033|1349908|230|32152|42|32|905|37|56.24|5.06|61.30|58.22|84.36|10.12|29.97|16.15|147.64| +2452845|78946|13920|16855|1382834|4774|7575|16855|1382834|4774|7575|37|17|905|57|572.28|28.61|600.89|39.88|733.02|383.42|37.77|151.09|801.51| +2452404|35333|15053|55935|404158|3228|18651|55935|404158|3228|18651|47|15|906|6|720.24|43.21|763.45|1.19|273.18|316.90|358.97|44.37|317.58| +2452567|10875|5121|39369|796691|413|19735|39369|796691|413|19735|1|17|907|36|1258.20|50.32|1308.52|9.35|419.40|931.06|32.71|294.43|479.07| +2452592|70309|2754|52504|1411858|12|40566|52504|1411858|12|40566|43|8|907|4|7.08|0.00|7.08|53.08|22.92|1.20|4.05|1.83|76.00| +2452579|34802|2037|4555|1866707|6459|43411|4555|1866707|6459|43411|13|16|907|3|328.62|6.57|335.19|85.64|126.39|46.00|45.21|237.41|218.60| +2452755|66401|16215|66450|188908|966|37471|66450|188908|966|37471|1|31|908|7|63.98|0.00|63.98|16.70|60.48|38.38|24.06|1.54|77.18| +2450973|8897|8431|79297|143976|3855|7334|79297|143976|3855|7334|25|21|910|4|535.00|32.10|567.10|3.94|145.28|160.50|348.28|26.22|181.32| +2450987|9039|16490|58866|1352692|1718|47144|58866|1352692|1718|47144|16|22|910|59|5390.83|107.81|5498.64|48.57|836.03|269.54|4506.73|614.56|992.41| +2452815||2430|69027|||31949|||4221|31949|19||911||1805.70|54.17||78.97|||250.27|1140.12|| +2452778|72742|11946|1413|65161|3756|1504|1413|65161|3756|1504|21|8|911|11|24.86|1.74|26.60|85.16|16.72|0.24|0.73|23.89|103.62| +2452183|22774|10819|33719|729500||||729500||47415||34|912|24|1471.92||1501.35|||1030.34||4.42|590.51| +2452311|82099|14363|66403|564030|6456|41827|66403|564030|6456|41827|49|32|913|59|1092.09|87.36|1179.45|79.10|1782.39|961.03|110.09|20.97|1948.85| +2451810|48278|3290|44392|99926|3107|34641|44392|99926|3107|34641|10|6|914|13|79.43|2.38|81.81|75.66|367.64|34.94|43.15|1.34|445.68| +2451305|78334|4213|18154|800603|4652|8301|18154|800603|4652|8301|28|18|915|75|8750.25|0.00|8750.25|15.28|984.00|6037.67|1274.91|1437.67|999.28| +2452011|57332|10676|65358|331834|2310|29186|65358|331834|2310|29186|25|27|916|74|2391.68|95.66|2487.34|28.10|3232.32|2391.68|0.00|0.00|3356.08| +2451399|36621|7898|32046|1395256|2019|18782|32046|1395256|2019|18782|37|9|918|6|188.76|9.43|198.19|54.21|0.00|1.88|153.24|33.64|63.64| +2451357||6079||1477160|||||3777||40|23|918|40||299.20||68.20|3222.40||||| +2450916|36053|4981|96645|1473236|1027|49218|96645|1473236|1027|49218|31|12|920|32|1060.16|42.40|1102.56|74.87|1120.64|106.01|95.41|858.74|1237.91| +2451024|69644|3326|68849|1473633|6417|33954|68849|1473633|6417|33954|20|9|920|41|8913.40|445.67|9359.07|49.97|2561.27|8913.40|0.00|0.00|3056.91| +2451292|40585|5767|79520|806123|6326|22882|79520|806123|6326|22882|55|1|922|1|129.82|7.78|137.60|38.16|10.59|72.69|49.13|8.00|56.53| +2451150|75941|15619|74372|1299864|5003|14895|74372|1299864|5003|14895|20|25|923|55|4313.10|388.17|4701.27|31.71|202.95|1380.19|87.98|2844.93|622.83| +2451687|65902|5017|14825|375253|6081|35709|14825|375253|6081|35709|55|21|924|41|66.01|0.00|66.01|36.63|264.86|1.98|40.97|23.06|301.49| +2451764|43635|9131||||||1495824||32886|23||924||2860.59||3003.61||874.01|||377.60|| +2451848|33589|9571|99149|1085963|4424|44838|99149|1085963|4424|44838|38|31|924|16|901.60|18.03|919.63|45.19|156.00|405.72|19.83|476.05|219.22| +2451744|33424|11546|47596|171926|5503|19833|47596|171926|5503|19833|19|4|924|15|1589.85|47.69|1637.54|64.31|612.75|333.86|1255.99|0.00|724.75| +2451038|34378|17764|38893|1672552|2211|41870|38893|1672552|2211|41870|58|27|926|70|8392.30|251.76|8644.06|64.16|0.00|2098.07|2895.34|3398.89|315.92| +2451104|61877|14113|31358|1762356|5001|5646|31358|1762356|5001|5646|4|30|926|69|10809.54|540.47|11350.01|82.94|7442.34|8755.72|2053.82|0.00|8065.75| +||13569||||||623307||15153|||928|||5.23|70.67||20.20|||10.25|109.64| +2452326|45233|11305|76588|150654|720|30786|76588|150654|720|30786|21|24|930|55|244.20|21.97|266.17|61.00|5.50|192.91|19.49|31.80|88.47| +2452417|76651|10763|73332|1063291|6850|32409|73332|1063291|6850|32409|21|25|930|39|5155.02|309.30|5464.32|97.67|2238.21|1649.60|2664.11|841.31|2645.18| +2452539|9748|6715|75353|952275|6444|4166|75353|952275|6444|4166|55|19|930|14|34.30|0.00|34.30|30.79|54.88|18.52|14.83|0.95|85.67| +||17045||1755550||6175|27061|1755550||6175|||931|11|||883.69||127.93||190.28|338.29|208.37| +2452352|50177|555|6725|1669698|1824|1197|6725|1669698|1824|1197|37|33|932|13|205.53|18.49|224.02|12.12|150.67|53.43|130.80|21.30|181.28| +2452340|43947|8525|51461|499063|1512|8314|51461|499063|1512|8314|33|17|932|2|155.48|13.99|169.47|74.31|4.84|113.50|27.28|14.70|93.14| +2452086|69430|4919|88818|890047|4076|4552|88818|890047|4076|4552|15|5|933|24|601.44|0.00|601.44|26.20|108.24|414.99|180.85|5.60|134.44| +2451813|60639|2510|33687|264698|4156|12414|33687|264698|4156|12414|26|28|934|14|2407.72|48.15|2455.87|78.32|1061.48|1203.86|385.23|818.63|1187.95| +2451604|85275|8660|90513|356051|2036|29974|90513|356051|2036|29974|52|28|935|19|3298.78|263.90|3562.68|4.92|106.40|3265.79|32.00|0.99|375.22| +2452412|66455|3421|18020|829563|1106|12202|18020|829563|1106|12202|11|24|936|76|102.60|5.13|107.73|7.34|60.80|73.87|7.75|20.98|73.27| +2451395|59543|8012|51680|619044|808|21419|51680|619044|808|21419|8|7|938|2|85.20|4.26|89.46|81.23|50.12|66.45|13.68|5.07|135.61| +2452766|27514|258|7955|535351|4347|33390|7955|535351|4347|33390|42|20|939|8|27.52|0.27|27.79|51.82|376.32|19.26|0.99|7.27|428.41| +2452781|25848|1699|5397|327919|5641|32163|5397|327919|5641|32163|13|2|939|69|26.22|0.78|27.00|23.61|60.72|13.63|3.65|8.94|85.11| +2452408|25895|229||864959|||68040||4278|20750||28|940||4398.19||4662.08|43.97|2174.86|1803.25||1920.26|2482.72| +2452427|8541|12919|59724|331967|5010|47653|59724|331967|5010|47653|19|28|942|11|144.87|0.00|144.87|26.31|248.27|130.38|2.17|12.32|274.58| +2452293|13825|12399|69071|1049459|3447|31435|69071|1049459|3447|31435|1|22|943|31|613.80|42.96|656.76|80.29|465.93|220.96|176.77|216.07|589.18| +2451212|32429|5270|98129|945166|4556|26393|98129|945166|4556|26393|4|1|946|14|807.66|48.45|856.11|16.99|393.96|72.68|51.44|683.54|459.40| +2451211||5323||790158||47601|22060|||||8|946|2|7.88|0.70|8.58||0.90||||53.82| +2451568|77644|12550|48748|1246617|3686|26755|48748|1246617|3686|26755|40|6|947|26|2070.90|20.70|2091.60|59.04|828.36|579.85|104.37|1386.68|908.10| +2451449|71217|763|67749|1880463|3577|27538|67749|1880463|3577|27538|55|24|947|55|124.85|7.49|132.34|41.12|811.25|66.17|51.05|7.63|859.86| +2451719|64443|7454|93371|1016450|5482|4961|93371|1016450|5482|4961|43|25|947|20|583.20|34.99|618.19|20.46|923.40|244.94|290.90|47.36|978.85| +2451419|60118|9691|93533|1145001|6069|40337|93533|1145001|6069|40337|26|29|947|60|1222.80|73.36|1296.16|54.80|1459.80|586.94|101.73|534.13|1587.96| +2451479|62564|17341|3858|1834685|2947|5240|3858|1834685|2947|5240|20|22|947|13|59.54|1.78|61.32|77.27|5.85|20.83|36.00|2.71|84.90| +2451555|30796|17924|17411|1779149|2487|31025|17411|1779149|2487|31025|56|31|947|47|2694.04|161.64|2855.68|2.43|0.00|2451.57|174.57|67.90|164.07| +2452648|23391|13015|62033|1035268|6550|41385|62033|1035268|6550|41385|33|11|948|58|1450.00|58.00|1508.00|14.61|29.58|826.50|548.68|74.82|102.19| +2451258|62807|11120|40569|1409099|2876|21406|40569|1409099|2876|21406|14|7|949|63|2024.82|101.24|2126.06|19.17|628.11|485.95|938.71|600.16|748.52| +2451341|61199|1748|19272|430656|3640|15024|19272|430656|3640|15024|43|13|949|6|270.72|18.95|289.67|43.77|315.84|246.35|18.03|6.34|378.56| +2451313|2207|5641|49026|1615462|6698|44781|49026|1615462|6698|44781|22|1|949|77|662.20|0.00|662.20|52.10|261.03|278.12|69.13|314.95|313.13| +2451220|28677|1460|1203|274321|5839|37757|1203|274321|5839|37757|10|35|950|42|2500.26|225.02|2725.28|56.98|1798.44|1350.14|322.03|828.09|2080.44| +2451284|65097|13939|43715|1856964|3460|20756|43715|1856964|3460|20756|1|10|950|61|2427.19|218.44|2645.63|50.80|254.98|582.52|1217.48|627.19|524.22| +2452470|70493|15075|23705|675592|1526|19013|23705|675592|1526|19013|57|29|951|8|53.84|0.00|53.84|93.51|5.76|38.22|1.09|14.53|99.27| +2451205|76567|7082|50191|687640|6336|23032|50191|687640|6336|23032|44|4|952|70|804.30|48.25|852.55|33.38|0.00|233.24|159.89|411.17|81.63| +2451411|40738|13318|95870|609164|5204|8580|95870|609164|5204|8580|10|7|952|23|1373.10|123.57|1496.67|87.12|1437.04|727.74|51.62|593.74|1647.73| +||10423|17940|||||1827883|5379||49|3|954||85.29|7.67||||34.96||41.78|51.27| +2452459|31361|6371|60407|1870247|887|18280|60407|1870247|887|18280|9|5|955|1|55.79|2.78|58.57|19.71|10.59|13.94|34.31|7.54|33.08| +2452394|26643|10083|42561|764189|6617|42645|42561|764189|6617|42645|31|1|955|30|408.60|12.25|420.85|4.94|58.20|44.94|138.19|225.47|75.39| +2451950|35514|11285|15618|635812|3559|3753|15618|635812|3559|3753|17|21|956|67|3476.63|0.00|3476.63|4.91|1206.00|2294.57|354.61|827.45|1210.91| +2451999|70297|6236|76748|1195753|5605|30657|76748|1195753|5605|30657|59|33|956|45|973.35|48.66|1022.01|96.45|1520.55|330.93|334.05|308.37|1665.66| +2452025|48761|1913|65608|1189253|2851|8496|65608|1189253|2851|8496|2|34|956|20|1558.20|0.00|1558.20|82.35|1414.00|264.89|1073.44|219.87|1496.35| +2451905|28521|11123|30998|247052|5506|1960|30998|247052|5506|1960|20|29|956|21|952.77|9.52|962.29|97.09|109.83|771.74|141.20|39.83|216.44| +2452201||1469|||5340|40657||||40657||2|957||781.25||820.31|90.78|382.25||||| +2452273|67872|6525|39072|275433|4594|36155|39072|275433|4594|36155|15|22|957|1|29.88|1.49|31.37|51.36|40.94|27.19|0.13|2.56|93.79| +2451984|60561|12313|29699|260034|2369|10257|29699|260034|2369|10257|20|35|958|14|86.38|5.18|91.56|5.48|218.96|74.28|0.60|11.50|229.62| +2451809|55361|4775|89515|1648410|1705|33400|89515|1648410|1705|33400|5|7|958|8|1218.64|48.74|1267.38|60.71|211.92|365.59|307.09|545.96|321.37| +2451193|74624|9008|69512|1079760|4408|11009|69512|1079760|4408|11009|1|17|959|24|1242.24|37.26|1279.50|14.67|478.80|757.76|198.63|285.85|530.73| +2451259|71856|5092|56437|1267326|4262|35998|56437|1267326|4262|35998|22|12|959|18|1768.32|159.14|1927.46|90.61|715.32|282.93|950.64|534.75|965.07| +2451083|28417|14821|37609|1098721|2539|44793|37609|1098721|2539|44793|43|11|960|2|7.42|0.44|7.86|78.71|4.94|0.81|6.01|0.60|84.09| +2450989|68618|2845|51184|870388|4792|8396|51184|870388|4792|8396|32|23|960|70|2380.70|23.80|2404.50|44.10|937.30|309.49|1491.27|579.94|1005.20| +2451657|50027|10867|60609|497028|1447|2666|60609|497028|1447|2666|16|17|962|6|256.62|0.00|256.62|52.87|152.88|41.05|153.05|62.52|205.75| +2451668|66162|16240|41031|1702782|6133|23531|41031|1702782|6133|23531|52|29|962|1|10.58|0.52|11.10|99.25|1.43|7.72|0.85|2.01|101.20| +2452695|7738|6264|52559|1246538|517|13696|52559|1246538|517|13696|7|16|963|26|258.44|15.50|273.94|94.64|184.60|85.28|41.55|131.61|294.74| +2451814|44388|16543|66463|146767|7071|26575|66463|146767|7071|26575|23|14|964|2|84.96|5.94|90.90|68.82|75.70|7.64|31.70|45.62|150.46| +2451436|36377|14278|94951|1020929|5307|48085|94951|1020929|5307|48085|34|6|966|62|1600.22|96.01|1696.23|46.53|861.80|288.03|223.07|1089.12|1004.34| +2452343|71491|7043|77412|95300|1635|45123|77412|95300|1635|45123|29|4|967|5|15.40|0.92|16.32|70.46|23.80|7.85|1.73|5.82|95.18| +2451693|33447|2636|42325|75487|1002|48564|42325|75487|1002|48564|56|18|968|29|319.00|22.33|341.33|82.38|30.16|111.65|182.46|24.89|134.87| +2451091|26999|17842|13615|1346570|3878|42833|13615|1346570|3878|42833|49|7|969|5|454.50|31.81|486.31|15.32|201.00|118.17|117.71|218.62|248.13| +2451676|23396|13994|62431|710775|1635|14588|62431|710775|1635|14588|37|19|971|31|3808.66|228.51|4037.17|66.96|100.13|2856.49|409.43|542.74|395.60| +2452687|28589|3150|74582|1255158|87|39863|74582|1255158|87|39863|45|13|972|1|5.04|0.25|5.29|7.22|0.40|1.71|2.93|0.40|7.87| +2452805|14578|2937|33187|1667066|4945|6871|33187|1667066|4945|6871|36|22|975|11|321.31|6.42|327.73|79.69|803.22|247.40|40.65|33.26|889.33| +2452406|67247|3221|53725|906574|2623|33072|53725|906574|2623|33072|55|13|977|6|79.68|5.57|85.25|96.22|12.30|75.69|3.63|0.36|114.09| +2451774|68876|3283|92761|1331863|3318|13973|92761|1331863|3318|13973|55|15|978|60|101.40|0.00|101.40|50.88|46.80|70.98|17.03|13.39|97.68| +2452062|15205|11105|24543|482554|2965|9776|24543|482554|2965|9776|19|19|981|10|341.80|0.00|341.80|69.44|161.90|211.91|79.23|50.66|231.34| +2452168|34784|1040|94498|491754|1010|26266|94498|491754|1010|26266|20|10|981|64|1131.52|45.26|1176.78|42.44|85.12|577.07|327.12|227.33|172.82| +2452041|15700|2510|61297|440365|1386|24854|61297|440365|1386|24854|53|13|981|17|757.52|37.87|795.39|9.40|76.50|719.64|3.03|34.85|123.77| +2452561|67680|693|46599|650508|4464|25620|46599|650508|4464|25620|49|17|982|11|668.36|33.41|701.77|10.55|275.11|467.85|152.38|48.13|319.07| +2451839|50240|11444|61384|1154872|3113|45754|61384|1154872|3113|45754|43|9|984|8|263.04|0.00|263.04|47.70|123.28|139.41|101.37|22.26|170.98| +|77768|17640|29609|784872|6535||29609|784872||26198||4|986|29|1200.60|108.05||24.73|485.17|552.27|538.11|110.22|| +||17639|81274|12022|||81274|12022|1170||||987|35|328.30||344.71||515.90|||160.71|| +2452398|81418|16587|75209|627810|2811|36212|75209|627810|2811|36212|15|12|987|13|363.09|14.52|377.61|40.32|294.97|236.00|96.58|30.51|349.81| +2452708|39751|3228|41647|701218|2742|47513|41647|701218|2742|47513|45|28|989|20|27.00|2.43|29.43|1.43|298.40|22.68|4.10|0.22|302.26| +2451594|9134|1282|63955|1565352|1085|23549|||1085|23549||6|990|||||40.06|||||| +2451400||3979|49266|385709||948|||||25|13|991|5|474.45||||80.75|||338.48|140.86| +2451281|38763|1384|91066|1509604|5389|17527|91066|1509604|5389|17527|50|33|992|35|4076.10|0.00|4076.10|28.49|1041.60|1671.20|553.12|1851.78|1070.09| +2451495|81983|16348|17|1800317|4264|40611|17|1800317|4264|40611|1|21|992|63|369.81|33.28|403.09|0.96|641.97|232.98|135.46|1.37|676.21| +2451966|64477|7735|55029|504852|5522|36945|55029|504852|5522|36945|41|7|993|19|863.36|77.70|941.06|97.39|482.41|751.12|42.65|69.59|657.50| +2451878|62358|10526|33931|1533850|3737|44057|33931|1533850|3737|44057|32|24|993|24|774.96|23.24|798.20|50.65|129.12|588.96|40.92|145.08|203.01| +2452155|51724|1088|8065|1059444|1670|44696|8065|1059444|1670|44696|53|23|994|22|2867.92|0.00|2867.92|95.81|832.48|2867.92|0.00|0.00|928.29| +2451262|8850|11584|84365|1772017|5269|16968|84365|1772017|5269|16968|19|10|997|31|686.03|61.74|747.77|34.99|53.94|535.10|70.93|80.00|150.67| +2451236|52834|7105|28533|1710167|2201|20635|28533|1710167|2201|20635|1|24|997|58|11059.44|442.37|11501.81|31.73|921.62|8183.98|1581.50|1293.96|1395.72| +2451275|49577|13270|90929|1732713|97|23834|90929|1732713|97|23834|8|26|997|94|790.54|39.52|830.06|83.60|430.52|134.39|629.90|26.25|553.64| +2451404||7322|29870|1314224|1778|2532|29870|1314224|1778|||35|998|22|1705.22|85.26|||248.60|1551.75||50.65|| +2451394|21441|2152|15249|244191|333|28475|15249|244191|333|28475|25|2|1000|12|52.80|2.11|54.91|27.68|52.80|18.48|32.60|1.72|82.59| +2451508|45539|14776|39644|1879611|3334|17091|39644|1879611|3334|17091|7|25|1000|46|517.04|10.34|527.38|52.76|142.14|77.55|162.61|276.88|205.24| +|51236|1501||1108286||23911||1108286||23911|49|25|1001||859.52|68.76|||889.10|833.73|||997.51| +2451825|60895|3142|14004|1054314|6624|46134|14004|1054314|6624|46134|10|7|1001|51|740.01|51.80|791.81|25.15|723.69|170.20|569.81|0.00|800.64| +2452731|69077|7974|53867|107132|1997|42862|53867|107132|1997|42862|13|30|1002|23|1086.06|65.16|1151.22|51.91|1327.56|282.37|168.77|634.92|1444.63| +2452742|51876|16801|1362|1060215|1115|3215|1362|1060215|1115|3215|49|14|1003|13|1036.36|31.09|1067.45|13.40|671.71|93.27|877.07|66.02|716.20| +2452716|54724|15757|18949|1627038|5880|22769|18949|1627038|5880|22769|42|22|1003|19|273.60|16.41|290.01|38.37|222.30|24.62|189.22|59.76|277.08| +2452827|32226|3865|27570|173480|990|21300|27570|173480|990|21300|19|21|1003|69|5404.77|270.23|5675.00|57.15|3949.56|972.85|930.70|3501.22|4276.94| +2452854|64039|16267|74977|1724124|824|5524|74977|1724124|824|5524|48|18|1004|13|531.05|42.48|573.53|4.03|176.93|47.79|478.42|4.84|223.44| +2452807|15324|3061|62348|1714120|6692|2064|62348|1714120|6692|2064|43|16|1004|4|310.88|27.97|338.85|53.31|36.00|133.67|115.18|62.03|117.28| +2451141|49018|14632|93763|473141|2119|31097|93763|473141|2119|31097|52|35|1005|30|4499.10|269.94|4769.04|54.87|1621.80|1484.70|542.59|2471.81|1946.61| +2452475|73201|14729|82075|1725540|6605|20870|82075|1725540|6605|20870|39|15|1006|50|382.50|26.77|409.27|77.20|220.50|237.15|117.73|27.62|324.47| +2451369|15900|8756|80829|1149936|196|30103|80829|1149936|196|30103|44|6|1007|44|7843.44|470.60|8314.04|90.18|3826.24|3372.67|89.41|4381.36|4387.02| +2451818|66476|4531|27289|1823596|1503|45410|27289|1823596|1503|45410|47|2|1008|55|1317.25|118.55|1435.80|63.76|782.10|922.07|284.52|110.66|964.41| +2451432|35679|13561|18419|348793|6014|45188|18419|348793|6014|45188|43|21|1009|5|274.20|8.22|282.42|53.23|325.60|54.84|87.74|131.62|387.05| +2451275|65181|6415|27073||3231||27073|722421|||||1009||21.27||||||||| +2451133|26166|5302|97656|440266|2076|4286|97656|440266|2076|4286|32|27|1009|10|134.90|10.79|145.69|14.58|11.40|8.09|49.45|77.36|36.77| +2452452|80924|7113|47720|1311402|4331|40013|47720|1311402|4331|40013|21|29|1010|56|1515.36|0.00|1515.36|77.73|82.32|515.22|580.08|420.06|160.05| +2452671|51468|17845|93167|1055484|3848|12965|93167|1055484|3848|12965|24|3|1011|58|1155.36|69.32|1224.68|92.72|149.64|369.71|102.13|683.52|311.68| +2451766|28488|835|36529|1677934|6986|38838|36529|1677934|6986|38838|56|8|1013|7|410.97|28.76|439.73|79.40|465.01|41.09|125.75|244.13|573.17| +2451225|25187|16492|89695|727207|4719|2631|89695|727207|4719|2631|49|32|1014|3|691.92|62.27|754.19|41.30|350.22|76.11|283.27|332.54|453.79| +2451123|26122|913|90550|209539|4648|13440|90550|209539|4648|13440|2|18|1014|61|1159.00|11.59|1170.59|77.28|597.19|1089.46|47.28|22.26|686.06| +2452758|75238|8850|80619|1795790|2961|5379|80619|1795790|2961|5379|30|15|1015|18|239.58|19.16|258.74|94.61|379.26|206.03|1.67|31.88|493.03| +2452708|18894|14853|52546|509686|2026|34319|52546|509686|2026|34319|60|17|1015|17|830.96|66.47|897.43|96.95|169.83|797.72|7.64|25.60|333.25| +2452640|47218|1645|91052|1720849|1846|26396|91052|1720849|1846|26396|13|22|1015|4|811.48|0.00|811.48|61.65|143.20|332.70|234.60|244.18|204.85| +2452599|58366|6666|46627|1304407|5387|47967|46627|1304407|5387|47967|45|2|1016|72|3641.76|72.83|3714.59|90.60|363.60|2002.96|950.50|688.30|527.03| +2452574|78389|16026|37444|1454706|3168|26830|37444|1454706|3168|26830|48|24|1016|1|32.77|2.29|35.06|57.24|46.52|14.41|9.18|9.18|106.05| +2451197|7425|16696|81058|817410|4515|38067|81058|817410|4515|38067|1|24|1017|56|243.04|0.00|243.04|54.74|136.08|211.44|14.22|17.38|190.82| +2451864|72118|10502|78800|816858|2116|48174|78800|816858|2116|48174|38|30|1019|2|51.22|1.02|52.24|67.31|9.02|0.51|31.94|18.77|77.35| +2451704|45572|17443|17456|797121|6386|42451|17456|797121|6386|42451|56|1|1019|6|172.38|1.72|174.10|8.61|13.08|122.38|25.50|24.50|23.41| +2452333|29115|15099|83349|1790677|5674|35811|83349|1790677|5674|35811|7|25|1020|19|33.44|2.67|36.11|40.91|61.37|16.05|3.65|13.74|104.95| +2452278|69507|17807|18825|891375|2325|9675|18825|891375|2325|9675|41|17|1020|43|855.70|42.78|898.48|74.30|297.13|855.70|0.00|0.00|414.21| +2450898|23388|5572|2295|474756|5617|27678|2295|474756|5617|27678|50|27|1021|12|1391.28|69.56|1460.84|67.18|993.72|1057.37|237.07|96.84|1130.46| +2450879|78418|7390|21532|507768|4236|16364|21532|507768|4236|16364|4|33|1021|6|268.92|18.82|287.74|50.54|243.54|236.64|11.94|20.34|312.90| +2450913|68160|3248|49018|1841292|766|11040|49018|1841292|766|11040|34|11|1022|68|773.84|15.47|789.31|88.05|39.44|216.67|479.16|78.01|142.96| +2451020|57303|5644|22408|1140292|6623|3707|22408|1140292|6623|3707|49|29|1022|43|2963.56|266.72|3230.28|90.87|683.70|2874.65|12.44|76.47|1041.29| +2451011|45662|8038|86956|1619882|4008|34934|86956|1619882|4008|34934|56|35|1022|62|2349.18|46.98|2396.16|66.40|1897.20|2043.78|284.02|21.38|2010.58| +2451075|70835|1486|38465|1368691|2564|19590|38465|1368691|2564|19590|34|13|1022|23|959.79|0.00|959.79|94.18|1279.72|844.61|25.33|89.85|1373.90| +2451784|74144|1310|12634|577394|768|4743|12634|577394|768|4743|20|18|1024|23|734.39|7.34|741.73|1.17|277.38|727.04|1.98|5.37|285.89| +2451782|72121|4778|63523|1709942|361|2380|63523|1709942|361|2380|19|19|1024|5|233.15|6.99|240.14|57.22|155.45|76.93|21.87|134.35|219.66| +2451365|42226|9128|95482|1275404|5698|13610|95482|1275404|5698|13610|10|18|1025|39|899.34|0.00|899.34|88.35|72.15|143.89|7.55|747.90|160.50| +2452736|5864|3985|37427|1529280|2286|28092|37427|1529280|2286|28092|37|27|1026|41|628.12|0.00|628.12|26.70|1136.93|119.34|386.67|122.11|1163.63| +2452402|69481|15967|63227|1265974|6321|29468|63227|1265974|6321|29468|41|15|1027|2|69.08|3.45|72.53|17.42|46.04|0.00|8.98|60.10|66.91| +2452081|38120|12104|18522|437653|6384|36762|18522|437653|6384|36762|47|7|1029|28|155.12|4.65|159.77|67.23|23.52|131.85|11.86|11.41|95.40| +2452380|54933|745|3260|1629348|236|6724|3260|1629348|236|6724|31|4|1030|1|8.18|0.65|8.83|36.02|6.25|6.05|1.21|0.92|42.92| +2452572|61190|9429|12912|754745|4578|30858|12912|754745|4578|30858|45|16|1031|9|225.27|4.50|229.77|4.73|23.94|94.61|101.91|28.75|33.17| +2452574|71217|11437|57684|444969|2654|18339|57684|444969|2654|18339|31|1|1031|89|759.17|45.55|804.72|85.51|310.61|516.23|233.22|9.72|441.67| +2451922|4110|17491|91258|794161|1417|37073|91258|794161|1417|37073|41|26|1033|20|880.60|35.22|915.82|10.30|352.20|581.19|191.62|107.79|397.72| +||17767||1899802|||64182||222|35042|8|26|1033||448.76|8.97||67.93|||70.94|5.35|413.34| +2452372|80645|7333|80570|904521|2072|33223|80570|904521|2072|33223|33|22|1034|2|90.34|2.71|93.05|48.84|59.74|56.91|17.71|15.72|111.29| +2452384|23311|187|74479|1708157|1035|2760|74479|1708157|1035|2760|45|7|1034|41|1034.84|41.39|1076.23|40.64|528.49|631.25|209.86|193.73|610.52| +2452413|68312|5623|81716|148016|269|48122|81716|148016|269|48122|51|6|1034|33|883.08|44.15|927.23|86.63|1324.62|158.95|144.82|579.31|1455.40| +2452227|79667|7379|70607|1352964|5721|18078|70607|1352964|5721|18078|5|12|1036|12|972.00|19.44|991.44|23.21|114.96|699.84|239.50|32.66|157.61| +2452183|28857|2631|89552|977967|6029|12044|89552|977967|6029|12044|17|28|1036|73|4733.32|283.99|5017.31|58.43|3628.83|331.33|2993.35|1408.64|3971.25| +2452043|13522|7629|44332|523604|3676|40595|44332|523604|3676|40595|5|19|1036|17|3244.62|129.78|3374.40|62.54|474.81|3147.28|59.37|37.97|667.13| +2452033|78893|10911|25561|1512598|6035|14719|25561|1512598|6035|14719|43|13|1036|13|611.91|48.95|660.86|63.85|289.77|354.90|46.26|210.75|402.57| +2451995|34743|933|21815|1774392|2169|25899|21815|1774392|2169|25899|31|34|1038|33|578.82|11.57|590.39|85.54|171.60|272.04|52.15|254.63|268.71| +2451930|48991|10111|42587|303533|1279|21778|42587|303533|1279|21778|19|27|1039|24|747.36|22.42|769.78|3.92|126.24|523.15|179.36|44.85|152.58| +2451884|69885|7556|4239|1241650|3861|34472|4239|1241650|3861|34472|59|26|1039|32|3484.16|104.52|3588.68|51.02|344.32|627.14|2057.05|799.97|499.86| +|60703|14002|34478|1155124||26816||||26816||2|1041|4|||342.36||111.16||37.25|181.87|| +2452602|9059|15061|69004|1227222|2240|38058|69004|1227222|2240|38058|24|14|1042|17|304.64|6.09|310.73|37.32|49.64|12.18|81.88|210.58|93.05| +2451985|51958|2489|77373|1127673|1620|37167|77373|1127673|1620|37167|55|23|1043|29|3126.20|31.26|3157.46|37.49|837.81|2563.48|382.64|180.08|906.56| +2451978|37351|9068|86463|1492768|7073|2178|86463|1492768|7073|2178|14|9|1043|19|221.16|2.21|223.37|13.46|54.34|44.23|84.92|92.01|70.01| +2452079|65345|10385|68532|291834|7136|14485|68532|291834|7136|14485|59|19|1043|64|161.92|3.23|165.15|86.05|1731.84|63.14|85.93|12.85|1821.12| +2452078|65125|2059|37342|1050980|1065|37103|37342|1050980|1065|37103|43|31|1044|41|2038.93|122.33|2161.26|64.64|1997.11|1712.70|244.67|81.56|2184.08| +2452004|71823|12950|75001|1518775|949|1123|75001|1518775|949|1123|14|30|1044|13|561.21|50.50|611.71|92.09|489.84|291.82|51.18|218.21|632.43| +2451765|28832|9311|75699|1095934|4969|33604|75699|1095934|4969|33604|53|5|1046|1|57.47|1.14|58.61|11.96|33.52|48.84|7.68|0.95|46.62| +2451339|40746|7195|29088|1422714|6385|49259|29088|1422714|6385|49259|7|19|1047|74|1589.52|31.79|1621.31|95.63|453.62|921.92|574.13|93.47|581.04| +2451359|70726|14864|65872|773823|4527|38831|65872|773823|4527|38831|56|12|1047|63|6489.63|454.27|6943.90|28.79|3167.64|4477.84|362.12|1649.67|3650.70| +2451293|62563|6406|98170|1708711|2005|34546|98170|1708711|2005|34546|13|33|1047|42|4909.38|392.75|5302.13|66.46|3201.66|4172.97|633.31|103.10|3660.87| +2452797|64053|16797|57467|315774|1728|45682|57467|315774|1728|45682|54|12|1048|26|2366.52|141.99|2508.51|71.91|959.40|2200.86|13.25|152.41|1173.30| +2452740|40109|5449|2055|425758|5014|5470|2055|425758|5014|5470|48|12|1048|6|129.18|0.00|129.18|10.66|134.76|24.54|53.36|51.28|145.42| +2451880|65903|919|70637|122538|39|36340|70637|122538|39|36340|55|30|1049|12|459.12|27.54|486.66|85.71|446.40|9.18|364.45|85.49|559.65| +2452273|76479|15727|13914|1751942|2770|44798|13914|1751942|2770|44798|51|4|1051|69|1482.81|14.82|1497.63|11.89|92.46|1141.76|255.78|85.27|119.17| +2452322|68913|215|64039|773564|2642|7320|64039|773564|2642|7320|15|35|1054|58|3251.48|130.05|3381.53|41.95|295.22|650.29|1014.46|1586.73|467.22| +2452046|22394|4184|5592|1224395|798|31154|5592|1224395|798|31154|8|18|1056|34|201.62|8.06|209.68|60.90|605.20|131.05|38.10|32.47|674.16| +2452091|74883|17126|76218|433802|2732|7091|76218|433802|2732|7091|50|21|1057|7|663.60|19.90|683.50|44.95|139.65|46.45|141.94|475.21|204.50| +2451961|29793|49|70795|1125625|6502|26979|70795|1125625|6502|26979|59|12|1057|20|1666.00|33.32|1699.32|19.51|686.00|349.86|315.87|1000.27|738.83| +2451790|59412|2321|5916|706006|1296|41739|5916|706006|1296|41739|19|20|1058|34|91.12|0.00|91.12|18.11|91.12|38.27|47.56|5.29|109.23| +2452042|37390|11156|72875|464800|2325|17050|72875|464800|2325|17050|26|8|1059|2|158.64|3.17|161.81|9.30|39.66|38.07|6.02|114.55|52.13| +2452198|19735|15029|6019|687666|6560|1007|6019|687666|6560|1007|53|11|1061|46|4381.50|306.70|4688.20|42.99|2014.34|1226.82|2555.29|599.39|2364.03| +|37712|4523||1333537|739|||1333537||40741|59|19|1061|36|||2075.06||||64.71|79.11|| +2452174||8229||212224|2781||||||33||1061||247.52|12.37|259.89|65.48||96.53|3.01||| +2451109|72665|5533|51614|55532|7097|30662|51614|55532|7097|30662|44|27|1062|1|70.27|6.32|76.59|63.13|22.19|21.08|46.73|2.46|91.64| +2452327|72395|1725|13756|1262836|269|2549|13756|1262836|269|2549|27|33|1063|3|33.96|2.71|36.67|71.52|130.17|8.49|0.25|25.22|204.40| +2452263|32650|16143|88368|1607795|1975|21402|88368|1607795|1975|21402|51|26|1063|15|269.85|5.39|275.24|42.82|36.90|215.88|33.46|20.51|85.11| +2452084|4393|839|33152||5710||33152|||44836|25||1063||2349.10|70.47|2419.57||234.65||515.16||| +2452195|40759|4197|1985|1446925|3077|12184|1985|1446925|3077|12184|5|28|1064|23|1087.90|10.87|1098.77|5.22|2115.54|892.07|139.03|56.80|2131.63| +2452369|62608|5867|30947|1557506|6465|10624|30947|1557506|6465|10624|5|22|1065|14|50.26|2.51|52.77|10.25|604.24|37.19|7.05|6.02|617.00| +2452226|80989|9200|43372|25293|14|14735|43372|25293|14|14735|23|11|1066|10|499.50|39.96|539.46|91.65|999.00|354.64|130.37|14.49|1130.61| +2452002|37563|551|5851|872649|4247|43845|5851|872649|4247|43845|7|28|1066|6|151.44|13.62|165.06|40.19|319.74|1.51|89.95|59.98|373.55| +2451966|40300|385|30815|1096287|4606|4925|30815|1096287|4606|4925|55|23|1066|20|1396.20|41.88|1438.08|9.47|1241.00|698.10|544.51|153.59|1292.35| +2452160|67638|7166|39388||||39388|1093175|||7||1066|9|42.12||43.38|24.44|2.79||||| +2451425|6982|16510|83644|1295050|5656|13509|83644|1295050|5656|13509|16|10|1067|70|81.90|2.45|84.35|56.98|50.40|68.79|4.71|8.40|109.83| +2451510|79618|5642|84247|1642781|962|33496|84247|1642781|962|33496|55|33|1068|34|2590.46|25.90|2616.36|98.78|1952.96|647.61|777.14|1165.71|2077.64| +2452210|40007|9938|53162|57071|4180|36936|53162|57071|4180|36936|23|32|1069|5|815.85|48.95|864.80|92.65|230.70|726.10|24.23|65.52|372.30| +2452035|48544|4663|31036|1874584|6132|24033|31036|1874584|6132|24033|53|12|1069|43|2566.24|0.00|2566.24|57.58|1518.76|2155.64|225.83|184.77|1576.34| +2452699|43778|729|44246|632002|6511|14667|44246|632002|6511|14667|13|10|1073|5|35.75|0.35|36.10|60.25|21.70|18.59|11.84|5.32|82.30| +2451969|46390|1771|17550|998497|2291|38885|17550|998497|2291|38885|49|5|1074|82|3044.66|30.44|3075.10|99.61|2202.52|943.84|546.21|1554.61|2332.57| +2452369|18135|1561|34885|1685475|2112|49902|34885|1685475|2112|49902|21|23|1075|3|144.03|0.00|144.03|53.80|68.88|47.52|1.93|94.58|122.68| +2452420|40961|401|53492|1201919|2346|35573|53492|1201919|2346|35573|43|17|1075|3|112.44|0.00|112.44|87.53|74.97|105.69|3.17|3.58|162.50| +2452345|36414|4055|78707|220717|1574|15631|78707|220717|1574|15631|33|4|1076|35|1251.25|0.00|1251.25|57.87|12.95|750.75|425.42|75.08|70.82| +|38947|6861||||48329|66101|702279|||47||1076||0.84|0.07|||1.34||||| +2451631|75424|4486|68970|1462527|6484|8888|68970|1462527|6484|8888|2|29|1077|26|824.72|16.49|841.21|1.42|98.54|470.09|237.60|117.03|116.45| +2451675|62122|11699|82677|633608|2139|33023|82677|633608|2139|33023|2|17|1078|18|1025.64|10.25|1035.89|29.68|540.54|410.25|547.69|67.70|580.47| +2451688|59598|12794|87740|985341|542|14943|87740|985341|542|14943|19|18|1078|3|113.37|7.93|121.30|42.13|128.49|4.53|30.47|78.37|178.55| +2451618|46541|11642|44023|209861|4053|2378|44023|209861|4053|2378|7|9|1079|7|43.33|1.29|44.62|75.69|10.78|27.29|11.22|4.82|87.76| +2451779|86325|12560|56702|1342345|1141|22784|56702|1342345|1141|22784|10|13|1079|22|2620.42|235.83|2856.25|4.87|705.32|471.67|128.92|2019.83|946.02| +2452911|67108|462|61146|1411793|3240|2687|61146|1411793|3240|2687|1|10|1080|6|2.70|0.00|2.70|37.94|13.14|0.70|0.30|1.70|51.08| +2450972|65953|3932|63200|1354369|3491|20613|63200|1354369|3491|20613|37|30|1081|11|362.12|25.34|387.46|43.45|181.06|318.66|16.08|27.38|249.85| +2451082|16445|11576|97240|1533703|3766|8629|97240|1533703|3766|8629|8|27|1081|9|733.86|14.67|748.53|90.83|126.09|432.97|6.01|294.88|231.59| +2450887|36691|721|86945|138072|7152|30679|86945|138072|7152|30679|46|5|1081|81|1441.80|0.00|1441.80|15.20|2090.61|1153.44|268.17|20.19|2105.81| +2451117|54612|772|57941|1477836|2884|29932|57941|1477836|2884|29932|34|10|1081|13|597.09|5.97|603.06|54.93|49.66|65.67|467.64|63.78|110.56| +2451656|23202|16712|15543|1710492|2688|37|15543|1710492|2688|37|44|30|1082|30|0.00|0.00|0.00|74.16|358.80|0.00|0.00|0.00|432.96| +2451778|13748|12038|60394|742123|5452|20734|60394|742123|5452|20734|56|14|1082|14|264.74|23.82|288.56|15.00|676.48|116.48|127.50|20.76|715.30| +2452414|72559|15783|63064|1449809|6154|99|63064|1449809|6154|99|45|31|1083|2|245.00|9.80|254.80|69.55|156.94|225.40|14.89|4.71|236.29| +2451248|67462|16198|75331|930060|789|49071|75331|930060|789|49071|4|8|1084|8|305.52|6.11|311.63|6.14|17.92|265.80|9.53|30.19|30.17| +2451271|82722|814|80101|1286526|3798|12623|80101|1286526|3798|12623|44|24|1084|49|608.58|0.00|608.58|12.30|566.93|298.20|80.69|229.69|579.23| +2451161|68743|301|736|622997|1611|39025|736|622997|1611|39025|19|20|1085|11|497.97|14.93|512.90|82.21|36.08|144.41|155.56|198.00|133.22| +2451297|52919|10886|7983|1802807|384|24808|7983|1802807|384|24808|20|18|1086|5|216.85|19.51|236.36|5.29|3.45|149.62|34.28|32.95|28.25| +2452343|61289|10939|42223|1683500|3624|21701|42223|1683500|3624|21701|35|25|1087|2|0.94|0.01|0.95|35.15|1.32|0.22|0.27|0.45|36.48| +2452174|75386|14475|26067|1058686|4105|18324|26067|1058686|4105|18324|11|30|1087|63|6179.67|247.18|6426.85|20.04|1382.22|4016.78|389.32|1773.57|1649.44| +2452821|50978|13809|7152|1298402|4052|11349|7152|1298402|4052|11349|37|33|1089|33|1638.78|114.71|1753.49|13.95|1990.23|1081.59|228.44|328.75|2118.89| +2452726|38655|4992|63623|173679|535|32591|63623|173679|535|32591|21|20|1089|18|106.74|1.06|107.80|53.75|783.00|59.77|33.34|13.63|837.81| +2450986|16006|16114|89658|360161|2311|18814|89658|360161|2311|18814|34|33|1090|6|365.52|3.65|369.17|98.92|416.52|149.86|47.44|168.22|519.09| +2452361|57600|1099|31054|63697|340|35783|31054|63697|340|35783|43|12|1092|1|63.69|3.82|67.51|5.51|46.20|38.21|6.11|19.37|55.53| +2452444|20795|1353|14692|1586485|7078|20871|14692|1586485|7078|20871|45|14|1093|22|566.50|39.65|606.15|9.10|0.00|107.63|18.35|440.52|48.75| +2452429|48819|2499|1326|1770408|3124|39481|1326|1770408|3124|39481|9|26|1093|87|601.17|30.05|631.22|10.60|522.00|258.50|143.92|198.75|562.65| +|81881|713|||958|11048|90825|1445315||11048|38|12|1094||||||52.65||3.93||| +2452314|81099|3927|24849|1150148|247|34746|24849|1150148|247|34746|49|25|1095|12|33.00|0.66|33.66|99.44|67.80|23.76|8.22|1.02|167.90| +2451761|72697|13465|15932|1678638|5344|46850|15932|1678638|5344|46850|43|21|1096|67|3292.38|263.39|3555.77|56.67|1579.86|2633.90|638.72|19.76|1899.92| +2451980|7037|7709|59530|639383|2436|4689|59530|639383|2436|4689|47|19|1096|28|368.76|7.37|376.13|81.29|105.28|173.31|15.63|179.82|193.94| +2451926|26611|1082|21835|1217405|4803|4858|21835|1217405|4803|4858|49|7|1097|28|109.48|1.09|110.57|0.90|65.24|29.55|0.79|79.14|67.23| +2451949|73888|1277|23614|8100|4939|37825|23614|8100|4939|37825|13|14|1097|18|100.98|3.02|104.00|8.83|568.62|57.55|39.08|4.35|580.47| +2452031|45307|5927|24135|1482529|6076|39270|24135|1482529|6076|39270|43|31|1097|1|12.23|1.10|13.33|51.84|26.75|8.92|2.41|0.90|79.69| +2452051|40408|14717|46510|146985|613|45088|46510|146985|613|45088|50|19|1097|5|63.40|4.43|67.83|91.22|11.50|13.94|47.48|1.98|107.15| +2451527|10149|16502|67905|1045043|3428|23366|67905|1045043|3428|23366|25|10|1098|37|234.58|16.42|251.00|87.19|430.31|173.58|7.32|53.68|533.92| +2451307|66620|13570|10768|1722914|660|47206|10768|1722914|660|47206|32|6|1099|29|1340.09|13.40|1353.49|7.56|114.26|884.45|27.33|428.31|135.22| +2451102|63030|16148|53149|428169|436|19856|53149|428169|436|19856|32|12|1099|35|1706.60|0.00|1706.60|11.87|834.75|648.50|941.70|116.40|846.62| +2452709|27357|13771|79793|1657206|6741|47869|79793|1657206|6741|47869|51|7|1100|7|487.55|0.00|487.55|80.00|243.74|156.01|92.83|238.71|323.74| +2452856|1128|6403|49327|690903|479|8855|49327|690903|479|8855|51|33|1100|21|935.97|18.71|954.68|62.41|1911.21|617.74|238.67|79.56|1992.33| +2452937|76777|10173|53784|157033|635|1037|53784|157033|635|1037|33|26|1100|11|179.30|5.37|184.67|82.27|19.03|150.61|19.22|9.47|106.67| +2451666||3584|79570|1692030|3690|30150|79570|1692030||30150|16||1101||5034.92||5085.26||3036.88||||3163.03| +2451724|58841|13912|9410|1156163||40022|9410|1156163||40022|||1101|||||31.48|997.90|416.40|||1121.91| +2451688|52106|1534|40547|1812897|6400|7150|40547|1812897|6400|7150|58|21|1101|4|6.08|0.06|6.14|45.38|61.12|6.08|0.00|0.00|106.56| +2452508|39871|2088|21431|996762|1026|31674|21431|996762|1026|31674|30|4|1103|32|849.28|76.43|925.71|62.33|718.72|645.45|114.14|89.69|857.48| +2452281|57876|1401|20194|1606807|973|43986|20194|1606807|973|43986|27|30|1104|1|8.84|0.44|9.28|40.88|21.70|8.75|0.08|0.01|63.02| +2452344|45530|8643|22631|1121874|6364|38007|22631|1121874|6364|38007|41|13|1104|56|2757.44|27.57|2785.01|9.76|226.80|1516.59|496.34|744.51|264.13| +2452300|44356|7751|34506|339219|3667|5576|34506|339219|3667|5576|21|5|1104|1|12.46|0.62|13.08|16.33|0.52|0.12|12.09|0.25|17.47| +2450872|80639|1534|15304|1348712|1689|28205|15304|1348712|1689|28205|37|14|1107|20|605.80|30.29|636.09|13.87|363.40|315.01|136.67|154.12|407.56| +2450895|46639|16534|48960|1026997|6634|32671|48960|1026997|6634|32671|44|16|1107|83|795.14|63.61|858.75|31.85|4330.94|143.12|293.40|358.62|4426.40| +2452829|70825|996|51484|450148|5612|41076|51484|450148|5612|41076|27|5|1108|11|645.26|51.62|696.88|31.85|344.85|619.44|20.91|4.91|428.32| +2452739|10518|885|63759|47663|4367|19871|63759|47663|4367|19871|36|23|1108|24|2110.56|105.52|2216.08|53.31|275.28|590.95|1063.72|455.89|434.11| +||7552|296|1634307|||296|1634307|5918||20||1109|16|||574.08||397.92|||80.43|| +2451269|71987|3880|85268|1530220|5572|49724|85268|1530220|5572|49724|22|10|1109|35|1176.70|105.90|1282.60|39.61|931.35|776.62|272.05|128.03|1076.86| +||12056|52419|||1963||176369||1963|10|4|1109|4||||||80.46|||| +2451941|63066|10727|93921|613656|3344|8279|93921|613656|3344|8279|19|5|1112|21|2225.37|155.77|2381.14|75.12|1369.41|1223.95|520.73|480.69|1600.30| +2451336|35503|9068|353|671474|1795|13716|353|671474|1795|13716|13|34|1114|2|33.92|0.67|34.59|25.33|33.92|22.72|3.58|7.62|59.92| +2451118|48098|8758|68576|923938|3300|43862|68576|923938|3300|43862|43|24|1114|1|10.41|0.41|10.82|1.38|25.45|3.12|5.17|2.12|27.24| +2452074|25341|61|74410|846304|3798|18892|74410|846304|3798|18892|13|11|1115|5|554.30|5.54|559.84|72.10|132.55|182.91|96.56|274.83|210.19| +2452105|37870|47|99657|145552|1494|25036|99657|145552|1494|25036|43|23|1115|25|1400.25|70.01|1470.26|41.69|1077.00|238.04|1045.98|116.23|1188.70| +2451376|32149|1792|10782|1421878|974|38070|10782|1421878|974|38070|13|34|1116|51|1387.20|69.36|1456.56|92.53|372.30|832.32|116.52|438.36|534.19| +2452831|73601|4854|92889|1546197|5105|28813|92889|1546197|5105|28813|60|18|1117|40|2689.60|242.06|2931.66|50.89|1013.20|1936.51|90.37|662.72|1306.15| +2452694|40192|1123|99275|1254381|3087|2993|99275|1254381|3087|2993|12|16|1117|21|2209.62|154.67|2364.29|95.19|143.43|1436.25|54.13|719.24|393.29| +2452034|37793|14419|96166|344308|6131|4174|96166|344308|6131|4174|56|34|1119|16|1046.40|62.78|1109.18|87.09|298.88|324.38|642.59|79.43|448.75| +2451923|74227|14965|43903|644470|5837|45204|43903|644470|5837|45204|26|8|1119|8|754.32|30.17|784.49|99.95|399.36|528.02|122.20|104.10|529.48| +2451557|51409|2422|66862|1403326|1390|45007|66862|1403326|1390|45007|28|4|1121|3|33.78|0.67|34.45|6.16|26.79|20.94|5.26|7.58|33.62| +2451797|35209|8891|31612|314529|2246|20763|31612|314529|2246|20763|41|28|1122|9|94.77|2.84|97.61|68.08|126.36|3.79|1.81|89.17|197.28| +2451694|34499|2399|58914|1722356|4198|21593|58914|1722356|4198|21593|32|10|1122|44|511.28|46.01|557.29|43.31|154.88|316.99|19.42|174.87|244.20| +2451883|70444|17090|67310|466918|5726|42981|67310|466918|5726|42981|43|13|1122|57|138.51|5.54|144.05|52.50|31.92|11.08|49.69|77.74|89.96| +2451215|73015|16928|1451|866059|3820|29581|1451|866059|3820|29581|56|22|1123|44|2771.56|166.29|2937.85|38.74|748.88|776.03|1516.60|478.93|953.91| +2451611|26268|7303|61774|1328959|144|27729|61774|1328959|144|27729|20|7|1124|33|1096.59|21.93|1118.52|70.70|783.42|328.97|145.84|621.78|876.05| +2451787|14575|12994|10014|792661|3461|20408|10014|792661|3461|20408|46|4|1124|1|0.00|0.00|0.00|1.11|59.71|0.00|0.00|0.00|60.82| +2451247|51624|11491|59164|1339715|5066|25570|59164|1339715|5066|25570|43|9|1128|46|2049.76|81.99|2131.75|75.86|40.02|286.96|1092.93|669.87|197.87| +|28911|8227||1518149||30894|||||12|9|1129||492.30||526.76|||||261.13|910.83| +2452703|52935|12859|5365|1089700|4788|45123|5365|1089700|4788|45123|37|3|1131|4|229.72|13.78|243.50|13.10|106.04|133.23|90.70|5.79|132.92| +2452711|44354|9199|93579|621290|6154|17818|93579|621290|6154|17818|57|1|1131|15|437.85|26.27|464.12|64.94|6.60|258.33|107.71|71.81|97.81| +2451387|505|224|69700|1707725|597|19469|69700|1707725|597|19469|44|32|1132|34|2184.50|21.84|2206.34|0.85|535.84|43.69|749.28|1391.53|558.53| +2451912|19543|289|62367|822795|2776|21957|62367|822795|2776|21957|47|24|1133|3|40.59|0.81|41.40|65.41|11.82|39.37|0.24|0.98|78.04| +2452444|40991|8569|95256|812081|6059|16084|95256|812081|6059|16084|51|2|1134|45|362.70|14.50|377.20|80.87|1270.35|130.57|141.59|90.54|1365.72| +2451767|26189|16777|9337|396286|6082|45878|9337|396286|6082|45878|2|22|1135|8|2.32|0.11|2.43|78.22|41.76|2.13|0.07|0.12|120.09| +2452205||16401||1485361|4681||||4681|4001|19||1136||81.65|7.34|88.99|65.08|86.60||30.19|40.03|159.02| +2452121||15637|50207||2174||||||29|20|1136|99||226.82|11568.26|36.51||907.31||8555.99|| +2452201|18119|16819|45741|693478|2015|7352|45741|693478|2015|7352|57|5|1136|77|2581.81|206.54|2788.35|90.39|1106.49|2246.17|234.94|100.70|1403.42| +2451652|36636|17065|8301|1215992|4483|33374|8301|1215992|4483|33374|37|22|1137|10|513.00|35.91|548.91|56.30|438.40|97.47|62.32|353.21|530.61| +2451709|70372|6589|21487|1264463|5392|36368|21487|1264463|5392|36368|1|11|1138|10|672.20|6.72|678.92|98.12|422.30|295.76|210.80|165.64|527.14| +2451465|60189|220|30663|1657923|2430|45690|30663|1657923|2430|45690|44|11|1138|2|271.98|19.03|291.01|58.75|83.26|247.50|15.91|8.57|161.04| +2451926|84890|16967|90949|831686|1670|29607|90949|831686|1670|29607|44|35|1139|3|680.19|27.20|707.39|95.09|206.34|455.72|60.60|163.87|328.63| +2451965|62734|10403|26197|1067346|2095|31244|26197|1067346|2095|31244|14|13|1139|47|417.83|0.00|417.83|56.56|1221.53|363.51|4.88|49.44|1278.09| +2451857|17918|1735|15918|1554812|906|25000|15918|1554812|906|25000|25|16|1139|31|2987.16|268.84|3256.00|30.17|2987.16|89.61|2810.62|86.93|3286.17| +2452081|66807|14756|71255|1838228|4048|5821|71255|1838228|4048|5821|29|13|1139|33|1761.87|88.09|1849.96|52.23|0.00|1233.30|517.99|10.58|140.32| +2452671|29821|3439|79587|1507371|1555|22539|79587|1507371|1555|22539|13|8|1140|29|249.11|0.00|249.11|87.05|560.57|104.62|76.57|67.92|647.62| +2451802|64867|16675|52743|629170|1597|36424|52743|629170|1597|36424|8|33|1142|46|2182.24|87.28|2269.52|38.66|2139.46|21.82|151.22|2009.20|2265.40| +2451609|49532|14851|20386|85211|789|5214|20386|85211|789|5214|7|15|1143|3|104.07|5.20|109.27|98.10|54.51|79.09|7.24|17.74|157.81| +2452350|34173|501|28720|1558826|5285|44991|28720|1558826|5285|44991|3|9|1144|17|253.64|17.75|271.39|95.98|326.23|238.42|7.61|7.61|439.96| +2451973|44847|974|19405|1239048|2691|12687|19405|1239048|2691|12687|17|15|1146|6|522.12|36.54|558.66|22.38|220.86|360.26|35.60|126.26|279.78| +2451884|75096|263|61838|667936|7051|7716|61838|667936|7051|7716|59|3|1146|48|633.12|6.33|639.45|6.11|99.36|63.31|495.73|74.08|111.80| +2452327|78176|11827|21821|142713|3605|37250|21821|142713|3605|37250|15|33|1147|5|91.60|5.49|97.09|70.89|30.50|51.29|25.39|14.92|106.88| +2452690|9578|4507|51096|1864316|736|18403|51096|1864316|736|18403|21|20|1149|18|604.26|18.12|622.38|16.97|443.16|235.66|335.42|33.18|478.25| +2452813|34586|12597|70084|1877015|5899|10783|70084|1877015|5899|10783|33|24|1149|5|1.90|0.11|2.01|65.02|2.30|0.19|0.30|1.41|67.43| +2452286|54528|14793|3353|469591|1076|5714|3353|469591|1076|5714|17|19|1150|10|665.80|26.63|692.43|91.67|179.20|545.95|111.46|8.39|297.50| +2451373|81605|17380|42057|1646063|2869|7228|42057|1646063|2869|7228|22|14|1151|16|1642.24|0.00|1642.24|24.15|1263.36|1609.39|13.14|19.71|1287.51| +2452178|72723|10993|27813|1146650|1840|9077|27813|1146650|1840|9077|41|15|1153|16|978.24|68.47|1046.71|0.90|306.40|684.76|246.52|46.96|375.77| +||12985||1106652|3983||98121||||||1154|4|50.60||52.62|9.62||20.24||10.33|98.08| +|18330|8441|||||3164||||59||1154|36||17.49|600.69|89.31|474.12||||580.92| +2452509|73564|14707|52598|1194219|1793|28214|52598|1194219|1793|28214|21|24|1155|60|3631.20|0.00|3631.20|75.52|2094.60|1488.79|214.24|1928.17|2170.12| +2452231|34112|17991|66825|953677|1776|3475|66825|953677|1776|3475|9|5|1155|3|143.28|8.59|151.87|30.47|74.19|34.38|6.53|102.37|113.25| +2452190|72206|14107|76211|1111959|1754|27549|76211|1111959|1754|27549|56|23|1156|11|767.69|23.03|790.72|30.82|225.72|368.49|127.74|271.46|279.57| +2452035|46445|12152|20875|1717741|4092|12607|20875|1717741|4092|12607|26|19|1156|2|82.86|0.00|82.86|56.73|25.34|14.91|66.59|1.36|82.07| +2452005|69085|16159|94920|330334|4948|27162|94920|330334|4948|27162|32|21|1156|28|2661.96|239.57|2901.53|59.04|1832.88|2449.00|129.90|83.06|2131.49| +2451964|81253|6854|70389|1605149|262|12622|70389|1605149|262|12622|20|21|1156|19|208.24|12.49|220.73|7.29|15.96|72.88|40.60|94.76|35.74| +2452765|39344|5215|28921|1565837|1153|42469|28921|1565837|1153|42469|24|1|1157|17|2439.84|146.39|2586.23|43.41|1365.10|1659.09|499.68|281.07|1554.90| +2451214|7050|3800|1713|221739|1318|11744|1713|221739|1318|11744|50|21|1159|2|128.44|11.55|139.99|92.14|29.84|115.59|9.76|3.09|133.53| +2451663|64043|9755|85095|1045804|3313|6022|85095|1045804|3313|6022|7|6|1160|1|52.48|1.57|54.05|20.26|15.88|36.73|4.41|11.34|37.71| +2451868|70409|775|39035|1420107|1179|47451|39035|1420107|1179|47451|50|12|1160|6|1216.62|36.49|1253.11|71.21|442.38|778.63|74.45|363.54|550.08| +2452068|18630|16003|77975|1227761|747|48933|77975|1227761|747|48933|1|17|1161|48|1481.28|29.62|1510.90|38.47|330.72|503.63|948.32|29.33|398.81| +2451803|57713|3028|5964|204768|6325|41129|5964|204768|6325|41129|8|24|1162|81|5303.07|477.27|5780.34|59.28|2849.58|5303.07|0.00|0.00|3386.13| +2452798|40555|13089|58629|1426897|5212|12327|58629|1426897|5212|12327|39|33|1163|21|391.02|15.64|406.66|33.88|638.19|211.15|16.18|163.69|687.71| +2452703|15335|2931|81791|1280467|390|49466|81791|1280467|390|49466|12|24|1163|22|3993.88|239.63|4233.51|38.10|1774.96|3874.06|115.02|4.80|2052.69| +||8810||||40853|45354|||40853|||1164|17|1436.84||||302.43||650.32|240.53|401.53| +2451365|34811|13549|12506|271013|5073|13133|12506|271013|5073|13133|26|17|1164|20|263.20|15.79|278.99|63.70|275.00|118.44|92.64|52.12|354.49| +2452738|73213|7182|1320|1618970|246|28226|1320|1618970|246|28226|51|25|1166|8|642.88|51.43|694.31|72.45|369.04|276.43|256.51|109.94|492.92| +2452554|30399|12087|68896|1254113|2083|31300|68896|1254113|2083|31300|41|16|1167|5|567.25|39.70|606.95|15.42|8.45|192.86|262.07|112.32|63.57| +2452315|40114|5723|31677|1233903|5729|32755|31677|1233903|5729|32755|41|23|1168|20|292.00|2.92|294.92|84.64|343.00|67.16|186.61|38.23|430.56| +||16636|73015||4454||||4454|26390|28||1169|1||0.00|55.94||14.27||19.13|2.13|19.48| +2452019|7286|8851|56904|1313372|2640|17908|56904|1313372|2640|17908|41|5|1171|48|10.08|0.90|10.98|90.11|51.84|4.83|2.52|2.73|142.85| +2452283|28296|16247|34177|1788567|6157|23300|34177|1788567|6157|23300|41|26|1172|22|124.30|7.45|131.75|48.26|364.54|123.05|0.13|1.12|420.25| +2452429|54040|15311|15440|1505805|1815|33904|15440|1505805|1815|33904|29|30|1172|5|23.35|1.16|24.51|60.37|31.90|16.57|3.25|3.53|93.43| +2452291|74844|15597|95149|753797|445|38693|95149|753797|445|38693|21|19|1172|88|4263.60|255.81|4519.41|67.88|270.16|1193.80|1811.18|1258.62|593.85| +2451517|38594|13990|75916|1216492|5068|48444|75916|1216492|5068|48444|26|27|1173|96|659.52|6.59|666.11|51.86|229.44|606.75|39.57|13.20|287.89| +2451678|43014|9877|72289|620777|18|22059|72289|620777|18|22059|44|17|1173|2|54.84|2.74|57.58|90.79|83.98|1.09|12.90|40.85|177.51| +2452435||11310||1043006|||5490||5769|6485|||1175|16|1336.00|93.52||70.95|756.16|841.68||103.81|| +2451027|20155|12956|28701|312336|5924|1959|28701|312336|5924|1959|37|2|1177|33|678.81|20.36|699.17|60.60|235.95|291.88|259.24|127.69|316.91| +2451915|70847|5960|81595|1897476|6157|9238|81595|1897476|6157|9238|29|12|1179|12|228.00|15.96|243.96|82.38|285.00|150.48|32.55|44.97|383.34| +2451894|16146|8147|38809|1317713|5770|6674|38809|1317713|5770|6674|53|35|1179|36|187.92|13.15|201.07|63.17|399.96|142.81|30.67|14.44|476.28| +2452050|57055|8467|36366|802252|3338|34024|36366|802252|3338|34024|49|27|1179|5|292.30|17.53|309.83|14.72|15.35|265.99|25.78|0.53|47.60| +2452499|27236|7749|4043|1608229|3968|35354|4043|1608229|3968|35354|7|4|1180|63|7239.96|144.79|7384.75|74.44|4247.46|651.59|4611.85|1976.52|4466.69| +2452490|36808|8307|30046|1534736|4579|21269|30046|1534736|4579|21269|21|17|1180|37|5366.48|375.65|5742.13|59.23|2493.43|3058.89|715.35|1592.24|2928.31| +2452468|45883|9211|41103|1107959|2015|27671|41103|1107959|2015|27671|57|17|1180|1|157.71|4.73|162.44|59.63|39.42|12.61|39.17|105.93|103.78| +2451599|70242|223|53653|419548|2690|585|53653|419548|2690|585|10|24|1181|12|1570.08|125.60|1695.68|26.53|517.80|1428.77|100.33|40.98|669.93| +2451576|59156|14536|63614|488852|6890|22528|63614|488852|6890|22528|20|4|1181|8|188.08|5.64|193.72|72.48|188.08|171.15|2.70|14.23|266.20| +2451682|44007|11372|42715|1309751|6459|29881|42715|1309751|6459|29881|28|26|1181|66|7480.44|448.82|7929.26|28.15|2552.22|4114.24|1716.76|1649.44|3029.19| +2452615|73099|14989|6415|596965|2362|14112|6415|596965|2362|14112|9|18|1182|83|4894.51|48.94|4943.45|62.14|2446.84|1027.84|1121.33|2745.34|2557.92| +2452762|28789|14305|95761|46142|4170|14998|95761|46142|4170|14998|48|4|1182|22|2122.78|84.91|2207.69|66.46|1243.22|1804.36|194.23|124.19|1394.59| +2452593|56990|14376|54250|652802|954|39106|54250|652802|954|39106|37|24|1182|5|387.00|11.61|398.61|63.68|136.55|201.24|117.02|68.74|211.84| +2451343||14938|61380|350711|6089|||||28267|28|34|1183|8|14.80|||98.21||12.58|0.82|1.40|118.33| +2451293|76850|11869|51955|338388|4821|9654|51955|338388|4821|9654|1|31|1183|41|3803.57|266.24|4069.81|1.74|2535.85|608.57|1948.95|1246.05|2803.83| +2452415|30841|17269|20459|1409061|5216|5371|20459|1409061|5216|5371|17|30|1184|81|8939.16|804.52|9743.68|56.76|694.98|983.30|7398.94|556.92|1556.26| +2452344|48645|10669|70767|1336198|2429|35860|70767|1336198|2429|35860|3|27|1184|28|1208.76|108.78|1317.54|59.78|890.68|217.57|475.77|515.42|1059.24| +2452420|58946|15823|74689|256101|1160|34972|74689|256101|1160|34972|51|23|1185|17|1595.11|111.65|1706.76|48.50|334.39|1036.82|474.54|83.75|494.54| +2452238|50046|11535|52063|1095892|521|36303|52063|1095892|521|36303|31|35|1185|2|224.74|11.23|235.97|63.72|161.82|24.72|54.00|146.02|236.77| +2452390|9526|2005|55948|236469|6143|8664|55948|236469|6143|8664|3|21|1185|77|3856.93|347.12|4204.05|28.26|5179.79|1002.80|2654.34|199.79|5555.17| +2451807|34412|4327|17622|974332|6242|24952|17622|974332|6242|24952|19|22|1186|1|3.09|0.21|3.30|57.87|21.06|2.84|0.04|0.21|79.14| +2451184|45126|4069||||46059|12820|442182||46059||30|1187|26|1269.32|88.85|1358.17|81.46||583.88||61.69|195.53| +2451329|18152|8540|97831|427100|5969|24438|97831|427100|5969|24438|37|34|1187|40|5.60|0.39|5.99|96.54|82.40|1.51|3.92|0.17|179.33| +2451411|69234|1748|692|1809370|95|43973|692|1809370|95|43973|20|19|1188|52|536.64|48.29|584.93|55.64|1395.68|262.95|180.63|93.06|1499.61| +2451266|55732|5641|19399|446271|1943|38015|19399|446271|1943|38015|58|22|1188|32|147.84|11.82|159.66|76.72|10.88|141.92|0.94|4.98|99.42| +2451268|64111|3544|2747|1740510|226|49942|2747|1740510|226|49942|44|32|1188|44|638.88|51.11|689.99|75.10|1353.00|63.88|339.25|235.75|1479.21| +2451300|26963|10660|50284|1508156|2028|1100|50284|1508156|2028|1100|37|11|1189|28|1484.84|29.69|1514.53|33.08|864.08|14.84|499.80|970.20|926.85| +2452187|72420|14213|5720|468517|5241|8124|5720|468517|5241|8124|47|17|1190|81|351.54|17.57|369.11|30.74|72.09|108.97|53.36|189.21|120.40| +2451243|37265|68|91964|1372475|1339|2250|91964|1372475|1339|2250|26|27|1191|45|0.00|0.00|0.00|10.69|312.75|0.00|0.00|0.00|323.44| +2451287|75533|14630|13155|1138065|6175|22638|13155|1138065|6175|22638|37|3|1191|24|385.68|30.85|416.53|29.71|819.84|104.13|247.76|33.79|880.40| +2451633|69335|16154|65830|1337198|2519|40287|65830|1337198|2519|40287|10|1|1192|10|111.50|1.11|112.61|88.39|34.50|23.41|77.51|10.58|124.00| +2451729|28936|11678|41427|1320965|2103|30841|41427|1320965|2103|30841|2|22|1192|34|2545.58|76.36|2621.94|84.97|2616.30|1527.34|203.64|814.60|2777.63| +|37264|10922||1211782|5183|2984|59181||5183||37|10|1194|1|1.35||1.37||5.14||0.04||| +2451518|41194|11620|35409|546397|3785|33573|35409|546397|3785|33573|52|24|1194|35|702.45|35.12|737.57|3.23|432.25|533.86|136.55|32.04|470.60| +2451535|62491|3118|42795|1661441|1687|5533|42795|1661441|1687|5533|40|23|1194|20|2500.00|225.00|2725.00|71.58|833.20|575.00|173.25|1751.75|1129.78| +2452710|21308|14565|11057|504726|4625|3197|11057|504726|4625|3197|51|29|1195|9|716.58|35.82|752.40|15.65|83.25|494.44|182.15|39.99|134.72| +2452464|72563|10077|15274|313516|205|28447|15274|313516|205|28447|18|32|1195|37|1979.87|138.59|2118.46|17.20|1576.94|1148.32|33.26|798.29|1732.73| +2452478|23281|6039|45661|407691|921|43751|45661|407691|921|43751|15|23|1195|4|701.76|63.15|764.91|11.27|331.60|119.29|244.63|337.84|406.02| +2452558|21434|14892|60201|809440|3926|35212|60201|809440|3926|35212|9|31|1195|93|2025.54|101.27|2126.81|34.60|692.85|1174.81|110.59|740.14|828.72| +2451833|71380|17513|35179|485352|5478|13651|35179|485352|5478|13651|29|5|1196|66|0.00|0.00|0.00|89.81|1904.10|0.00|0.00|0.00|1993.91| +2452394|27641|3438|57428|549520|3212|21704|57428|549520|3212|21704|49|21|1197|1|2.72|0.21|2.93|1.64|14.20|0.35|0.11|2.26|16.05| +2452227|79554|6989|44702|1329274|3446|17218|44702|1329274|3446|17218|43|25|1198|2|51.22|4.09|55.31|78.23|96.42|8.70|39.11|3.41|178.74| +2451129|57088|17818|24307|1322677|247|10217|24307|1322677|247|10217|34|4|1201|58|838.10|67.04|905.14|1.56|588.12|167.62|53.63|616.85|656.72| +2451971|49268|4297|29758|776568|4178|6235|29758|776568|4178|6235|55|15|1203|17|274.89|13.74|288.63|24.95|710.09|104.45|74.99|95.45|748.78| +2452193|31426|4562|61754|279629|6304|45779|61754|279629|6304|45779|31|16|1203|15|9.75|0.78|10.53|87.05|39.00|0.68|7.89|1.18|126.83| +2451605|48973|4807|10676|915427|3378|28976|10676|915427|3378|28976|49|14|1204|18|840.06|0.00|840.06|50.78|335.88|764.45|18.14|57.47|386.66| +2451752|16261|8239|69446|965391|3383|39728|69446|965391|3383|39728|22|13|1204|60|828.60|41.43|870.03|7.54|1074.00|812.02|7.95|8.63|1122.97| +2451607|68698|1094|41514|605342|723|34504|41514|605342|723|34504|56|7|1204|43|923.64|73.89|997.53|38.93|733.58|175.49|411.48|336.67|846.40| +2452240|47294|14411|6029|1243174|5919|43464|6029|1243174|5919|43464|37|32|1207|56|2035.04|122.10|2157.14|19.23|1061.76|488.40|123.73|1422.91|1203.09| +2452465|73994|157|37694|899987|4680|8926|37694|899987|4680|8926|41|27|1207|11|1368.29|95.78|1464.07|33.83|884.07|533.63|534.18|300.48|1013.68| +2452295|71683|15581|2349|890692|5082|8272|2349|890692|5082|8272|43|23|1207|18|1242.36|37.27|1279.63|72.13|506.16|273.31|203.50|765.55|615.56| +2451512|37059|3478|1266|1055649|1679|9662|1266|1055649|1679|9662|46|8|1208|18|83.16|4.98|88.14|4.34|93.42|23.28|6.58|53.30|102.74| +2451430|62865|12154|35739|1749877|1168|44906|35739|1749877|1168|44906|37|14|1208|2|62.66|3.13|65.79|14.87|30.16|6.89|19.51|36.26|48.16| +2451379|34090|17995|29327|443991|5855|43766|29327|443991|5855|43766|43|4|1208|22|4264.26|255.85|4520.11|88.70|1405.80|596.99|660.10|3007.17|1750.35| +2452786|70798|4381|42402|508098|1441|12337|42402|508098|1441|12337|7|22|1211|42|6887.16|275.48|7162.64|19.04|4081.14|1101.94|5033.14|752.08|4375.66| +2452572|83267|4047|79607|144611|4015|19372|79607|144611|4015|19372|6|24|1211|14|1430.38|85.82|1516.20|30.07|678.86|801.01|610.48|18.89|794.75| +2451712|42781|10789|41625|358953|281|26409|41625|358953|281|26409|25|3|1212|4|88.80|0.88|89.68|78.58|1.88|58.60|8.45|21.75|81.34| +2451886|66323|9731|52481|267200|3072|30991|52481|267200|3072|30991|32|12|1213|12|410.16|0.00|410.16|60.69|320.40|73.82|124.44|211.90|381.09| +2452072|46024|8990|71206|82705|6651|20150|71206|82705|6651|20150|2|25|1213|42|560.28|22.41|582.69|34.00|154.98|330.56|20.67|209.05|211.39| +|32906|17213|||||70891|424053||31930|47|18|1214||205.28|6.15||80.16|189.48|176.54|10.63|18.11|275.79| +2452325|26741|17949|62132|493058|5137|17216|62132|493058|5137|17216|31|4|1214|55|4367.55|87.35|4454.90|98.61|2092.75|3974.47|110.06|283.02|2278.71| +2452298|14428|17449|53872|412072|5472|39672|53872|412072|5472|39672|59|8|1214|4|81.56|1.63|83.19|75.12|141.88|33.43|25.99|22.14|218.63| +2452298|48165|10389|44069|353844|4183|43853|44069|353844|4183|43853|55|16|1214|55|4281.75|342.54|4624.29|14.58|518.65|3511.03|393.06|377.66|875.77| +2451269|68934|15361|43456|1727831|5359|22934|43456|1727831|5359|22934|50|13|1215|14|605.36|48.42|653.78|96.78|770.56|411.64|139.47|54.25|915.76| +||5918|15233|424816|5985|49318|15233|424816|5985|49318||24|1215|8|751.20|15.02|||130.96|555.88|91.80||215.05| +2451853||7826|88500|1639060||25380|||3220||55||1217||909.81|54.58|||849.15|||98.26|| +2451501|70254|15175|37670|1761872|2168|42748|37670|1761872|2168|42748|50|5|1218|28|473.48|42.61|516.09|94.33|631.40|355.11|49.71|68.66|768.34| +2452643|65414|16543|39660|1894101|2495|12536|39660|1894101|2495|12536|48|13|1219|5|420.55|16.82|437.37|10.59|52.55|328.02|72.17|20.36|79.96| +2452574|67968|12309|65848|1770884|3473|28509|65848|1770884|3473|28509|54|27|1219|46|2968.84|89.06|3057.90|0.79|853.76|1246.91|154.97|1566.96|943.61| +2452492|55512|4944|93986|1787137|3488|35392|93986|1787137|3488|35392|24|29|1219|60|1618.20|16.18|1634.38|77.08|1330.20|1488.74|16.82|112.64|1423.46| +2451156|77442|2785|7145|464543|2741|27339|7145|464543|2741|27339|8|25|1221|98|1236.76|86.57|1323.33|65.77|2408.84|1150.18|83.98|2.60|2561.18| +||1489|||2382||96330||||13||1221|33||||89.12||184.61|144.68||105.29| +2451744|64851|9317|10619|1269000|628|22256|10619|1269000|628|22256|35|3|1225|6|437.16|4.37|441.53|23.92|100.50|48.08|311.26|77.82|128.79| +2452510|52116|17177|75504|908116|6681|26046|75504|908116|6681|26046|57|20|1229|23|41.63|0.41|42.04|38.02|541.65|22.06|8.80|10.77|580.08| +2452410|78008|7963|69508|1085429|4550|24424|69508|1085429|4550|24424|15|21|1229|3|263.16|21.05|284.21|43.64|154.80|150.00|110.89|2.27|219.49| +2451986|44719|15679|43553|1909444|4135|34826|43553|1909444|4135|34826|49|26|1230|10|777.30|23.31|800.61|43.91|461.50|544.11|179.55|53.64|528.72| +|53507|17257|60016||5878|32497|60016|619601|5878||||1230|||11.70|597.15||305.37|345.41|12.00|228.04|395.00| +2451837|37308|16549|65947|29036|6078|3967|65947|29036|6078|3967|53|18|1230|52|7749.56|542.46|8292.02|15.20|3040.96|697.46|1833.54|5218.56|3598.62| +2451911|37771|14443|5341|766092|3692|7319|5341|766092|3692|7319|55|11|1231|4|143.04|5.72|148.76|72.97|37.08|125.87|12.19|4.98|115.77| +|68261|16075||1045530||47337|85450|1045530||47337|53|17|1231|30|||981.29|||761.19|118.49||247.82| +2451180||17107|46512|513439||||513439||||25|1232|28|33.60|||||13.77||13.09|| +2452538|59184|1987|10459|1626156|5053|47150|10459|1626156|5053|47150|59|19|1233|46|7876.58|393.82|8270.40|98.07|4149.66|6773.85|385.95|716.78|4641.55| +2452382|53045|2947||1184852|||18490|||376|||1234|23|||361.51||32.20|92.15|23.60||| +2451351|77516|7532|59566|1702285|7051|28617|59566|1702285|7051|28617|55|32|1235|13|243.23|19.45|262.68|24.20|218.92|19.45|20.14|203.64|262.57| +2451398|25870|6769|70996|678435|6321|1360|70996|678435|6321|1360|44|15|1235|13|665.21|19.95|685.16|63.64|354.77|186.25|431.06|47.90|438.36| +||937|56530|||24306|56530|896973|||57||1237|||||||298.39||2140.77|| +2451961|39752|13562|2290|163211|2179|26300|2290|163211|2179|26300|23|35|1238|18|2496.06|149.76|2645.82|16.90|0.00|1248.03|212.16|1035.87|166.66| +2451725|80316|2689|8331|1399610|3689|10168|8331|1399610|3689|10168|16|16|1239|1|70.74|3.53|74.27|76.07|69.24|60.83|2.08|7.83|148.84| +2451586|17524|2968|57000|1181666|6332|22598|57000|1181666|6332|22598|22|33|1239|56|113.12|2.26|115.38|16.85|59.92|1.13|51.51|60.48|79.03| +2451959|83716|16994|80371|1185259|6147|4188|80371|1185259|6147|4188|32|21|1240|76|1166.60|0.00|1166.60|96.97|1231.20|139.99|277.18|749.43|1328.17| +2452042|8976|17879|49283|1377205|5927|48249|49283|1377205|5927|48249|31|32|1240|27|981.45|39.25|1020.70|81.35|227.34|412.20|261.85|307.40|347.94| +2451419|74183|12757|51850|1203610|970|3530|51850|1203610|970|3530|58|13|1241|17|0.00|0.00|0.00|14.22|118.15|0.00|0.00|0.00|132.37| +2451328|51304|7388|73979|715675|4571|44267|73979|715675|4571|44267|16|26|1241|33|278.52|13.92|292.44|40.96|1254.33|19.49|56.98|202.05|1309.21| +|39138|2187||1534786||15839||1534786|||||1244||612.75|12.25|625.00|86.42||435.05|51.53||| +2452408|78921|1843|74214|1352123|6032|341|74214|1352123|6032|341|59|20|1244|8|51.60|2.06|53.66|96.44|189.28|36.63|12.87|2.10|287.78| +2451457|17468|12914|30837|337069|4101|21933|30837|337069|4101|21933|46|18|1247|5|333.35|6.66|340.01|90.39|194.45|130.00|140.31|63.04|291.50| +2451203|74135|17947|63631|985691|4998|4658|63631|985691|4998|4658|50|11|1248|4|123.40|7.40|130.80|27.62|165.96|78.97|33.76|10.67|200.98| +2451283|74783|16850|7173|1030121|6669|8017|7173|1030121|6669|8017|32|24|1248|35|149.10|1.49|150.59|43.85|358.05|55.16|69.51|24.43|403.39| +2451769|51815|13897|47701|768429|2757|33485|47701|768429|2757|33485|13|21|1251|16|186.40|1.86|188.26|82.30|798.72|54.05|44.99|87.36|882.88| +2452607|31667|15355|17962|1200910|2705|20100|17962|1200910|2705|20100|27|1|1252|14|1919.26|172.73|2091.99|29.76|83.44|1535.40|341.63|42.23|285.93| +2452686|2924|17067|16977|296693|56|26972|16977|296693|56|26972|45|20|1252|8|30.40|1.82|32.22|7.77|43.28|3.34|23.00|4.06|52.87| +2452533|42204|6907|81472|1399923|1574|2425|81472|1399923|1574|2425|48|4|1252|6|141.90|12.77|154.67|42.62|24.66|18.44|49.38|74.08|80.05| +2452468|24369|3013|10834|101056|6478|25053|10834|101056|6478|25053|51|6|1253|22|3479.96|69.59|3549.55|3.29|1077.12|1461.58|1069.74|948.64|1150.00| +2452420|33570|14085|90245|390531|4770|256|90245|390531|4770|256|1|28|1253|44|707.08|28.28|735.36|80.00|391.16|466.67|91.35|149.06|499.44| +2451250|69457|9406|55509|652122|6335|34595|55509|652122|6335|34595|25|28|1254|2|362.74|0.00|362.74|83.19|106.96|39.90|193.70|129.14|190.15| +2451386|38124|13678|39537|370274|5458|36195|39537|370274|5458|36195|40|11|1254|14|707.56|14.15|721.71|8.30|609.98|226.41|187.64|293.51|632.43| +2451950|62870|7316|20239|297461|2683|25505|20239|297461|2683|25505|5|15|1255|54|2310.66|207.95|2518.61|80.43|957.96|1386.39|600.77|323.50|1246.34| +2452034|55563|13853|19203|1668332|3061|11837|19203|1668332|3061|11837|55|33|1255|1|17.15|1.02|18.17|15.40|9.67|16.29|0.43|0.43|26.09| +2451462|53057|10063|41655|919789|6276|49563|41655|919789|6276|49563|40|12|1256|4|20.68|0.00|20.68|90.56|12.92|13.64|2.60|4.44|103.48| +2451689|47544|16759|2429|155036|1690|14744|2429|155036|1690|14744|55|4|1256|1|36.94|0.73|37.67|32.76|12.00|36.57|0.27|0.10|45.49| +2451352|60799|10372|4405|1658128|2698|38021|4405|1658128|2698|38021|49|5|1257|41|3038.10|243.04|3281.14|69.94|893.39|1792.47|722.46|523.17|1206.37| +2452318|57767|3049|83007|1834031|255|40479|83007|1834031|255|40479|3|22|1258|16|1277.12|51.08|1328.20|34.75|1222.72|140.48|863.84|272.80|1308.55| +2452341|4348|13629|71297|653629|1561|16693|71297|653629|1561|16693|33|8|1258|28|147.00|11.76|158.76|91.99|156.52|83.79|43.61|19.60|260.27| +2452321|7726|6951|52306|1605957|3255|24906|52306|1605957|3255|24906|17|31|1258|6|463.44|27.80|491.24|59.85|33.48|46.34|296.14|120.96|121.13| +2452398|38587|10923|26482|1446375|6091|38141|26482|1446375|6091|38141|41|6|1258|7|18.83|0.75|19.58|12.08|16.10|13.18|1.07|4.58|28.93| +2452734|70749|1669|47104|72288|2272|29856|47104|72288|2272|29856|48|15|1260|19|2203.81|110.19|2314.00|6.92|944.49|1300.24|831.28|72.29|1061.60| +2451308|79140|3478|51096|1111381|5266|38751|51096|1111381|5266|38751|44|29|1261|22|414.48|33.15|447.63|38.48|333.08|82.89|162.47|169.12|404.71| +2452200|30548|4343|90654|1043775|4096|13217|90654|1043775|4096|13217|21|22|1263|28|2068.08|0.00|2068.08|11.90|442.96|1447.65|496.34|124.09|454.86| +2452204|32986|9467|2235|1606300|5565|9230|2235|1606300|5565|9230|9|29|1263|52|2489.76|124.48|2614.24|6.82|1278.16|1817.52|658.79|13.45|1409.46| +2452250|21763|13847|61117|1870411|1305|45399|61117|1870411|1305|45399|37|33|1263|4|151.24|4.53|155.77|79.73|46.36|146.70|0.99|3.55|130.62| +2451979|79751|2947|94893|1170467|2394|17521|94893|1170467|2394|17521|32|30|1264|64|6519.68|65.19|6584.87|39.36|5312.64|5737.31|633.71|148.66|5417.19| +2451584|56848|223|39919|1502869|5019|22369|39919|1502869|5019|22369|55|12|1265|36|1322.64|39.67|1362.31|57.07|515.88|1190.37|111.10|21.17|612.62| +2451567|56182|8017|50447|1579447|2847|2647|50447|1579447|2847|2647|16|3|1265|8|532.16|47.89|580.05|24.98|806.88|250.11|222.81|59.24|879.75| +2451544|51772|2833|39361|614902|5196|17137|39361|614902|5196|17137|49|24|1265|4|400.08|12.00|412.08|39.10|271.24|392.07|6.40|1.61|322.34| +2451545|33400|610|49072|1636400|6128|45056|49072|1636400|6128|45056|26|14|1265|19|1606.64|64.26|1670.90|42.90|566.96|1092.51|51.41|462.72|674.12| +|41119|777||||22084||||||24|1268|2|22.80||24.62|57.72|||2.87|1.24|| +2452059|43410|11054|63013|1128492|818|49897|63013|1128492|818|49897|11|30|1269|39|2632.50|52.65|2685.15|40.34|478.53|868.72|123.46|1640.32|571.52| +2451588|54172|14440|70535|74536|4490|26491|70535|74536|4490|26491|10|13|1270|52|774.80|69.73|844.53|5.07|270.92|100.72|377.48|296.60|345.72| +2451592|18786|13558|87111|876598|3332|20019|87111|876598|3332|20019|25|17|1270|4|3.36|0.10|3.46|46.41|41.76|1.00|1.34|1.02|88.27| +2451512|43634|17488|21822|942890|7158|21873|21822|942890|7158|21873|26|28|1271|19|889.20|0.00|889.20|36.23|64.79|266.76|124.48|497.96|101.02| +2451480|40260|15097|38062|1276363|1417|20518|38062|1276363|1417|20518|44|20|1271|8|118.96|3.56|122.52|69.61|45.76|27.36|54.96|36.64|118.93| +2452373|31530|15595|25836|669789|1742|9695|25836|669789|1742|9695|19|11|1272|22|978.34|58.70|1037.04|91.54|362.34|616.35|228.05|133.94|512.58| +2452338|40139|11765|54464|161617|4417|6370|54464|161617|4417|6370|49|27|1272|65|434.85|26.09|460.94|99.86|252.20|234.81|104.02|96.02|378.15| +2452275|27757|6367|33286|365546|5453|3313|33286|365546|5453|3313|13|4|1272|9|1568.61|62.74|1631.35|76.76|689.22|250.97|711.52|606.12|828.72| +2452814|59893|8316|51426|1465183|6140|8697|51426|1465183|6140|8697|54|14|1273|13|885.04|53.10|938.14|1.34|238.94|646.07|107.53|131.44|293.38| +2452885|36922|3135|25735|677141|6147|48706|25735|677141|6147|48706|37|35|1273|13|239.98|7.19|247.17|73.55|26.65|151.18|15.09|73.71|107.39| +2451386||12439|97425||||97425|394954||21259|4||1275||||||50.25||196.14|364.26|| +2451280|59918|15943|96413|598411|6362|20360|96413|598411|6362|20360|14|29|1275|7|171.71|6.86|178.57|93.30|37.66|60.09|69.20|42.42|137.82| +2451215|68409|6104|10005|1501012|6866|45120|10005|1501012|6866|45120|32|22|1275|5|724.45|50.71|775.16|95.22|205.70|137.64|88.02|498.79|351.63| +2451531|39558|17776|61849|706393|890|22587|61849|706393|890|22587|16|23|1276|29|2061.32|0.00|2061.32|30.73|1030.66|535.94|549.13|976.25|1061.39| +2451742|69170|15968|29503|1355835|2167|40499|29503|1355835|2167|40499|38|2|1277|30|312.60|9.37|321.97|18.77|1062.90|287.59|16.50|8.51|1091.04| +2451112|68937|5084|5421|1024048|6675|36055|5421|1024048|6675|36055|26|3|1278|1|42.99|3.86|46.85|85.65|79.11|29.66|8.13|5.20|168.62| +2451491|79046|5998||||25838||||25838|37|31|1280|||||64.49|||||| +2452476|40953|9445|41332|420593|1901|46409|41332|420593|1901|46409|13|8|1282|5|758.75|30.35|789.10|49.59|379.35|166.92|390.60|201.23|459.29| +2452971|78280|14631|77304|83988|2225|14509|77304|83988|2225|14509|36|20|1283|16|524.48|31.46|555.94|31.31|104.80|451.05|22.76|50.67|167.57| +2452678|62484|10674|40066|67225|431|10790|40066|67225|431|10790|13|24|1283|47|1610.69|64.42|1675.11|86.96|20.21|805.34|789.24|16.11|171.59| +2452374|71896|9591|8184|1664415|4646|32477|8184|1664415|4646|32477|33|16|1284|7|450.10|13.50|463.60|87.03|233.73|157.53|81.91|210.66|334.26| +2452311|42269|1479|47517|1695234|3001|49602|47517|1695234|3001|49602|53|12|1284|22|399.30|35.93|435.23|61.67|1524.60|159.72|105.41|134.17|1622.20| +2452472|54059|13361|75590|50773|5156|18339|75590|50773|5156|18339|29|22|1285|2|18.02|0.18|18.20|25.38|26.48|14.23|1.89|1.90|52.04| +2451444|43396|4834|31792|1400455|4318|21781|31792|1400455|4318|21781|28|26|1286|23|908.73|81.78|990.51|77.40|836.05|599.76|305.88|3.09|995.23| +2451190|38272|8498|32362|936822|4035|43648|32362|936822|4035|43648|37|33|1287|9|218.97|8.75|227.72|32.13|301.14|210.21|2.71|6.05|342.02| +2451614|38326|4816|97719|1430535|6446|9694|97719|1430535|6446|9694|25|31|1289|14|126.70|1.26|127.96|87.06|44.24|82.35|34.59|9.76|132.56| +2452721|80894|15099|82623|1740073|3123|29373|82623|1740073|3123|29373|42|2|1290|74|954.60|66.82|1021.42|55.75|1622.82|448.66|197.31|308.63|1745.39| +|67041|12769||1205191|474||||474|4510|44||1292|16||172.73|2640.41|32.60||1554.63|730.44||1027.89| +2451566|29152|4057|24174|404955|1736|7670|24174|404955|1736|7670|2|24|1293|22|235.62|7.06|242.68|64.43|104.72|75.39|112.16|48.07|176.21| +2451650|67651|12562|48925|349250|3528|44206|48925|349250|3528|44206|37|3|1293|69|5560.71|222.42|5783.13|28.13|2780.01|1612.60|1776.64|2171.47|3030.56| +2451417||16238||1315457|1797||36317||1797|37737||22|1294|9|14.94|||55.70|14.40||||71.44| +2451179||6067|92116|||||36281||9811|56||1294|13|||185.77||||0.90|89.34|| +||13624|||5257||5759||5257|32559|4|26|1294|||3.33|40.35|77.52|35.80||7.59|13.52|| +2451337|60175|14324|53485|1053054|5776|10545|53485|1053054|5776|10545|55|31|1294|69|5348.88|0.00|5348.88|99.84|3234.03|2513.97|113.39|2721.52|3333.87| +2452438|7958|4909|3039|975488|5984|4833|3039|975488|5984|4833|47|1|1295|25|2216.50|66.49|2282.99|64.21|738.75|1307.73|799.71|109.06|869.45| +2452869|10525|8089|2008|724382|2309|283|2008|724382|2309|283|36|4|1297|5|72.90|0.00|72.90|85.68|56.85|28.43|41.80|2.67|142.53| +2452769|23146|16969|6853|1627970|1887|5404|6853|1627970|1887|5404|57|11|1297|3|110.13|7.70|117.83|94.66|18.00|40.74|52.73|16.66|120.36| +2452734|11530|17737|68451|1179538|4578|8065|68451|1179538|4578|8065|3|20|1299|4|194.36|7.77|202.13|86.75|205.44|34.98|3.18|156.20|299.96| +||15543|86366|404849||15909|86366|404849|||3||1300|15|183.60|3.67|187.27|63.13|115.20|178.09|||182.00| +2451934|61254|533|51042|668368|447|7773|51042|668368|447|7773|35|3|1301|16|448.96|8.97|457.93|64.05|26.40|67.34|141.19|240.43|99.42| +2452364|69395|16353|66528|303501|4921|43494|66528|303501|4921|43494|55|22|1302|70|9524.20|190.48|9714.68|54.72|3809.40|1142.90|5615.47|2765.83|4054.60| +2451440|46055|11588|17012|946255|2698|13141|17012|946255|2698|13141|58|21|1303|15|339.60|27.16|366.76|97.60|27.00|288.66|14.26|36.68|151.76| +2451590|69719|15764|75124|1071773|2022|5471|75124|1071773|2022|5471|49|12|1303|11|109.78|6.58|116.36|80.42|585.75|36.22|2.94|70.62|672.75| +2451525|4967|16442|87116|153269|222|41128|87116|153269|222|41128|38|26|1303|34|495.38|29.72|525.10|69.51|67.32|178.33|291.68|25.37|166.55| +2451229|64302|2611|7029|274769|4909|29849|7029|274769|4909|29849|20|2|1304|84|6725.88|67.25|6793.13|96.31|3087.84|3430.19|1087.57|2208.12|3251.40| +||5270|36369|||45199||1060128|6640|45199|28||1304||731.72|14.63|||||24.07||| +2451629|6892|10744|32613|825979|145|35111|32613|825979|145|35111|2|31|1305|14|1989.40|99.47|2088.87|47.84|217.98|576.92|324.87|1087.61|365.29| +2452009|18508|2894|28026|1529645|6464|48883|28026|1529645|6464|48883|20|10|1306|32|565.12|22.60|587.72|58.66|85.44|0.00|293.86|271.26|166.70| +2452525|68565|8631|62702|1135831|6560|47689|62702|1135831|6560|47689|13|33|1307|80|2616.80|52.33|2669.13|36.52|1785.60|2067.27|5.49|544.04|1874.45| +2452680|45580|13770|95860|1551612|5764|36763|95860|1551612|5764|36763|25|14|1307|28|202.16|2.02|204.18|23.75|328.72|80.86|31.53|89.77|354.49| +2452491|44621|5083|2606|165221|1789|37557|2606|165221|1789|37557|27|34|1307|3|23.34|1.63|24.97|51.18|66.72|12.60|1.82|8.92|119.53| +2451584|59852|3454|59168|1286052|6183|40667|59168|1286052|6183|40667|13|17|1310|18|239.76|0.00|239.76|17.99|639.72|105.49|116.81|17.46|657.71| +||8121|64834|1135878|5322|45519|||5322|45519|59|1|1311|25||60.16|6076.41|90.66||||178.69|| +2452087|74011|1907|19717|912012|5554|20696|19717|912012|5554|20696|17|34|1311|52|1128.92|56.44|1185.36|33.62|219.96|914.42|126.55|87.95|310.02| +2452200|55147|17233|65427|1806670|1960|17650|65427|1806670|1960|17650|55|34|1311|48|3015.36|241.22|3256.58|13.10|1190.40|150.76|2091.15|773.45|1444.72| +2452132|33722|2599|25083|368166|849|7485|25083|368166|849|7485|11|26|1312|44|755.92|30.23|786.15|68.94|1197.24|151.18|314.46|290.28|1296.41| +2452134|72205|11731|54029|1009489|5749|8409|54029|1009489|5749|8409|43|12|1312|73|4067.56|284.72|4352.28|88.02|1949.83|3457.42|353.88|256.26|2322.57| +2452370|54340|15797|52032|577200|1921|27489|52032|577200|1921|27489|31|15|1313|5|69.70|4.87|74.57|45.21|107.45|27.18|34.01|8.51|157.53| +2452483|11479|13723|92380|1168113|1446|29979|92380|1168113|1446|29979|11|6|1313|12|15.60|1.40|17.00|20.09|439.80|6.55|6.96|2.09|461.29| +2451264|66856|13892|32026|1296687|2148|39162|32026|1296687|2148|39162|8|4|1315|66|10681.44|427.25|11108.69|66.87|5272.74|1602.21|3540.89|5538.34|5766.86| +2451251|72159|15770|55264|1334387|2310|45433|55264|1334387|2310|45433|49|14|1315|17|284.92|22.79|307.71|69.87|257.72|122.51|56.84|105.57|350.38| +2452249|76282|15791|22691|1199191|138|16238|22691|1199191|138|16238|59|6|1316|45|217.35|8.69|226.04|49.03|2971.35|136.93|32.16|48.26|3029.07| +2452383|25262|2459|26066|354457|3295|33286|26066|354457|3295|33286|11|11|1319|72|1321.20|13.21|1334.41|8.76|36.00|1149.44|44.65|127.11|57.97| +2452431|23322|3445|90597|564818|2592|38813|90597|564818|2592|38813|53|16|1319|16|111.68|3.35|115.03|58.25|55.84|11.16|39.20|61.32|117.44| +2452408|74492|9015|32528|382684|1640|37796|32528|382684|1640|37796|11|22|1320|6|1254.06|62.70|1316.76|48.57|80.04|1040.86|134.31|78.89|191.31| +2452896|51198|319|69170|1467939|1908|36277|69170|1467939|1908|36277|33|8|1321|13|746.98|7.46|754.44|80.37|391.30|380.95|230.59|135.44|479.13| +||7525|34230|||18874||752876||18874|||1322||2255.61|||||248.11|||| +2452117|52089|3405|16354|269006|411|17863|16354|269006|411|17863|3|34|1322|73|2643.33|0.00|2643.33|19.74|1089.89|1400.96|534.21|708.16|1109.63| +2451582|82512|15265|85135|843340|3945|34550|85135|843340|3945|34550|1|30|1323|60|6020.40|361.22|6381.62|53.19|3010.20|1986.73|1048.75|2984.92|3424.61| +2451622|30939|14977|86480|858656|2832|32561|86480|858656|2832|32561|31|32|1323|1|81.94|0.81|82.75|19.07|83.76|72.92|3.69|5.33|103.64| +2451678|14699|1138|45826|1611888|797|35259|45826|1611888|797|35259|4|22|1323|14|878.50|26.35|904.85|97.17|146.30|720.37|113.85|44.28|269.82| +2451229|23025|11042|34635|1514603|4524|24092|34635|1514603|4524|24092|1|31|1324|23|4292.95|171.71|4464.66|99.80|1543.53|4121.23|85.86|85.86|1815.04| +2451828|49678|7669|33889|618612|6888|36364|33889|618612|6888|36364|59|6|1325|19|93.48|2.80|96.28|31.62|210.52|62.63|1.23|29.62|244.94| +|14607|15667|||1949|15252||701580||15252|11|19|1325|5|||13.71||5.25|2.42|10.69||92.30| +2452303|78248|14961|64233|1273636|286|18089|64233|1273636|286|18089|49|6|1326|13|1954.16|19.54|1973.70|90.55|555.10|957.53|39.86|956.77|665.19| +2451172|9734|10450|68366|1333941|3963|12338|68366|1333941|3963|12338|32|9|1327|15|1726.95|103.61|1830.56|50.97|980.10|310.85|991.27|424.83|1134.68| +2451272|71079|6884|72446|219040|5281|12275|72446|219040|5281|12275|22|28|1327|43|1559.18|77.95|1637.13|26.37|1137.78|1434.44|89.81|34.93|1242.10| +2451179|53549|9685|89616|1281430|3197|33411|89616|1281430|3197|33411|40|16|1327|6|305.16|0.00|305.16|19.59|157.62|143.42|74.40|87.34|177.21| +2452697|64733|978|34922|1214144|2966|10188|34922|1214144|2966|10188|31|3|1328|3|81.93|6.55|88.48|57.81|62.25|45.06|1.84|35.03|126.61| +2452618|48312|12751|29932|1857657|5382|5907|29932|1857657|5382|5907|15|20|1332|3|98.85|0.98|99.83|42.45|11.91|65.24|29.57|4.04|55.34| +2451894|30471|7208|52806|135915|3024|45333|52806|135915|3024|45333|59|5|1333|2|234.56|4.69|239.25|1.55|95.74|232.21|0.65|1.70|101.98| +2452427|44602|15091|25724|567587|1450|47713|25724|567587|1450|47713|13|1|1334|36|4003.56|280.24|4283.80|56.79|571.68|3563.16|48.44|391.96|908.71| +2452528|15648|8244|46186|769563|4940|17164|46186|769563|4940|17164|24|33|1334|9|45.45|3.18|48.63|52.58|207.99|23.63|19.85|1.97|263.75| +2452611|49537|2149|61577|978943|5909|14428|61577|978943|5909|14428|51|17|1334|22|78.32|7.04|85.36|30.54|407.88|36.81|9.54|31.97|445.46| +2451695|9370|12538|22872|1011952|1832|3712|22872|1011952|1832|3712|16|24|1335|4|433.00|34.64|467.64|78.63|239.96|142.89|14.50|275.61|353.23| +||4648||||||103696||41725|||1337||845.54|16.91|862.45|95.70|0.00|727.16|101.80||| +2451601|72726|9092|32112|1633399|1439|47926|32112|1633399|1439|47926|28|30|1338|5|462.70|32.38|495.08|59.13|67.45|397.92|45.99|18.79|158.96| +2451619|68987|16327|79275|589031|2340|40849|79275|589031|2340|40849|46|10|1338|28|3290.84|164.54|3455.38|80.51|2067.24|1053.06|1029.37|1208.41|2312.29| +2451546|44346|16388|12874|266052|2430|26808|12874|266052|2430|26808|19|21|1338|6|915.66|0.00|915.66|14.42|348.78|759.99|99.62|56.05|363.20| +2451032|76392|52|99121|1731726|2731|49736|99121|1731726|2731|49736|44|1|1339|24|1785.84|17.85|1803.69|17.05|830.64|1107.22|481.82|196.80|865.54| +2451982|68461|3841|94355|590322|6227|41765|94355|590322|6227|41765|32|17|1341|16|129.60|5.18|134.78|50.00|135.36|90.72|27.99|10.89|190.54| +2452234|15133|11107|15736|269949|4515|43081|15736|269949|4515|43081|13|18|1344|11|404.03|28.28|432.31|14.74|53.79|28.28|285.57|90.18|96.81| +2451671|74510|12121|72851|142654|457|48613|72851|142654|457|48613|37|32|1345|7|960.82|38.43|999.25|86.09|593.46|220.98|14.79|725.05|717.98| +2452212|17834|4247|77139|931593|447|454|77139|931593|447|454|47|8|1346|55|2033.90|81.35|2115.25|52.83|128.70|1728.81|292.88|12.21|262.88| +2452077||13285|5074|742564|2101|||742564|2101||7|10|1346||219.84|0.00|219.84||213.92|211.04|7.30||| +2452514|39335|17305|62008|1629759|2310|37308|62008|1629759|2310|37308|57|8|1348|53|5823.11|291.15|6114.26|56.17|3095.73|2271.01|3161.36|390.74|3443.05| +2451843|19546|5222|69852|1075671|4337|12203|69852|1075671|4337|12203|32|27|1349|19|3146.59|62.93|3209.52|48.20|169.10|912.51|469.15|1764.93|280.23| +2452872|11257|12906|1754|1250784|6661|27691|1754|1250784|6661|27691|27|10|1350|56|2088.24|20.88|2109.12|79.82|944.72|459.41|16.28|1612.55|1045.42| +2452626|26494|8851|84386|423925|1864|15756|84386|423925|1864|15756|45|6|1350|25|816.00|8.16|824.16|63.27|670.25|807.84|2.61|5.55|741.68| +||2890|61825||||||3575|8097||34|1351||1612.23|80.61|1692.84||997.96|1418.76|148.97||| +2452731|53062|15039|78176|1462511|4093|42028|78176|1462511|4093|42028|27|18|1352|47|3783.97|113.51|3897.48|22.81|900.99|983.83|1568.07|1232.07|1037.31| +2452867|31152|16897|86587|431098|3162|43273|86587|431098|3162|43273|12|23|1352|46|1916.82|134.17|2050.99|40.89|1797.22|1782.64|16.10|118.08|1972.28| +2452789|69075|8862|40239|205076|2368|25295|40239|205076|2368|25295|24|11|1352|41|1039.35|20.78|1060.13|26.41|11.48|363.77|378.32|297.26|58.67| +2451281|44262|10804|79864|174828|3039|33203|79864|174828|3039|33203|16|13|1353|4|245.64|22.10|267.74|24.41|152.24|171.94|42.74|30.96|198.75| +2451777|49793|5720|82878|957744|6563|45429|82878|957744|6563|45429|31|2|1354|16|413.44|4.13|417.57|72.99|193.76|194.31|219.13|0.00|270.88| +2452025|13975|9635|29913|506533|4306|4926|29913|506533|4306|4926|8|20|1355|10|1082.00|21.64|1103.64|74.93|336.30|97.38|118.15|866.47|432.87| +2452030|57786|11749|19920|240496|5519|10095|19920|240496|5519|10095|7|15|1355|4|135.44|2.70|138.14|46.78|1.32|13.54|18.28|103.62|50.80| +2452052|65399|12823|46347|1108926|7058|34437|46347|1108926|7058|34437|38|1|1355|5|910.00|63.70|973.70|2.80|334.70|691.60|58.96|159.44|401.20| +2452013|25288|15061|64280|299984|841|5345|64280|299984|841|5345|32|25|1356|14|77.98|5.45|83.43|70.53|77.98|51.46|17.76|8.76|153.96| +2451248|72744|685|4205|824882|3512|38942|4205|824882|3512|38942|31|29|1357|1|52.07|0.52|52.59|5.21|22.09|45.82|3.93|2.32|27.82| +2452260|59087|17167|6730|1814395|6060|14125|6730|1814395|6060|14125|5|7|1358|14|2864.68|85.94|2950.62|77.06|172.48|1117.22|297.06|1450.40|335.48| +2452358|82058|16789|13312|1873921|483|16245|13312|1873921|483|16245|59|3|1358|25|539.75|16.19|555.94|35.67|215.75|145.73|263.99|130.03|267.61| +2452155|61682|13887|45873|1426977|4881|35185|45873|1426977|4881|35185|5|6|1358|21|594.51|5.94|600.45|32.68|490.14|17.83|74.96|501.72|528.76| +2451512|14127|15151|70202|1237058|6314|8770|70202|1237058|6314|8770|10|24|1359|3|81.21|3.24|84.45|46.22|31.08|25.17|38.10|17.94|80.54| +2452670|3617|17478|89516|358054|2592|39177|89516|358054|2592|39177|51|7|1360|8|79.92|5.59|85.51|83.95|284.16|2.39|77.53|0.00|373.70| +2451405|60880|955|89607|1906420|833|30046|89607|1906420|833|30046|44|31|1361|10|565.00|22.60|587.60|62.25|88.20|180.80|257.41|126.79|173.05| +2451321|44323|6758|12657|747710|6763|36169|12657|747710|6763|36169|1|12|1361|18|963.18|9.63|972.81|22.95|225.18|173.37|292.22|497.59|257.76| +2451530|42889|5311|7166|119193|5173|32353|7166|119193|5173|32353|22|27|1365|3|35.01|2.10|37.11|56.94|11.28|33.95|0.09|0.97|70.32| +2451601|81986|5266|47688|1303008|6681|24801|47688|1303008|6681|24801|44|30|1365|30|999.00|69.93|1068.93|4.41|1023.90|299.70|118.88|580.42|1098.24| +2451331|26524|3562|98201|1856035|4622|16456|98201|1856035|4622|16456|25|23|1366|59|752.84|67.75|820.59|87.38|1129.26|715.19|11.67|25.98|1284.39| +2452266|72797|15321|28208|601659|1063|32867|28208|601659|1063|32867|41|8|1367|21|920.85|55.25|976.10|84.80|1350.51|405.17|443.48|72.20|1490.56| +2452226|34942|3051|||||19987||||49||1367|6|172.92|12.10||95.46|6.90||24.90|99.61|| +2452798|70592|5377|21536|1909602|4068|4534|21536|1909602|4068|4534|21|4|1368|7|138.95|12.50|151.45|11.01|166.74|86.14|11.61|41.20|190.25| +2452834|75544|16155|54628|794995|639|29545|54628|794995|639|29545|9|5|1369|13|507.39|35.51|542.90|98.49|70.98|25.36|482.03|0.00|204.98| +2452709|63457|2436|1930|531750|3970|13390|1930|531750|3970|13390|24|12|1369|1|7.35|0.44|7.79|21.12|32.34|5.43|1.51|0.41|53.90| +2451207|5141|5560|74324|1914327|730|10686|74324|1914327|730|10686|2|10|1370|63|1551.69|0.00|1551.69|19.03|0.00|822.39|415.70|313.60|19.03| +2451318|58096|17998|72906|1309926|3746|15119|72906|1309926|3746|15119|50|7|1370|52|632.84|0.00|632.84|39.70|253.24|512.60|8.41|111.83|292.94| +2451171|78783|5290|792|1585867|260|42092|792|1585867|260|42092|43|1|1371|20|2007.80|40.15|2047.95|2.55|1392.40|1465.69|487.89|54.22|1435.10| +2452664|31928|8544|30132|929894|928|6773|30132|929894|928|6773|18|5|1372|6|283.32|5.66|288.98|13.13|102.18|45.33|164.21|73.78|120.97| +2452895|42900|6103|86648|1486388|1328|44668|86648|1486388|1328|44668|24|28|1372|46|536.82|21.47|558.29|28.85|2524.94|472.40|26.41|38.01|2575.26| +2452797|71749|14391|61112|1103336|5560|21337|61112|1103336|5560|21337|60|34|1372|22|21.78|1.74|23.52|69.48|219.78|12.41|1.59|7.78|291.00| +2451223|45162|8113|24198|1253934|2047|27902|24198|1253934|2047|27902|20|1|1373|68|3687.64|36.87|3724.51|9.17|294.44|1364.42|116.16|2207.06|340.48| +2451283|16097|865|81362|1288016|5423|34077|81362|1288016|5423|34077|56|12|1373|17|10.20|0.71|10.91|82.33|20.40|9.07|0.31|0.82|103.44| +2452714|77375|6030|19303|587682|5661|32393|19303|587682|5661|32393|33|25|1374|47|1968.36|0.00|1968.36|97.77|1289.68|1023.54|500.75|444.07|1387.45| +2451856|33942|6035|95714|461539|164|38389|95714|461539|164|38389|29|1|1376|22|1.76|0.12|1.88|50.35|37.84|0.51|0.53|0.72|88.31| +2451736|43336|17653|67268|1500221|4821|32221|67268|1500221|4821|32221|7|29|1377|4|287.36|0.00|287.36|65.13|54.12|163.79|87.73|35.84|119.25| +2452655|32741|14844|60459|1382793|2727|26661|60459|1382793|2727|26661|24|20|1379|17|643.62|19.30|662.92|70.74|1072.70|251.01|298.38|94.23|1162.74| +2452705|67479|1593|71969|1483110||||1483110|||36|13|1379||135.54|8.13|143.67||180.72||44.64||| +2452725|76888|313|88047|1623421|5995|9369|88047|1623421|5995|9369|57|30|1380|93|1868.37|130.78|1999.15|2.75|3063.42|1606.79|138.63|122.95|3196.95| +2452790|72363|13086|11817|1149034|4298|25915|11817|1149034|4298|25915|27|6|1381|30|2321.40|162.49|2483.89|28.09|299.40|1880.33|48.51|392.56|489.98| +2452591|54880|816|58695|940379|2510|8994|58695|940379|2510|8994|37|7|1381|8|325.52|22.78|348.30|91.59|114.08|260.41|38.41|26.70|228.45| +2452329|5376|6439|72953|698827|1054|48081|72953|698827|1054|48081|1|27|1382|2|31.76|0.63|32.39|85.96|18.90|2.54|23.66|5.56|105.49| +2452090|38433|16082|53078|1646656|681|23506|53078|1646656|681|23506|50|27|1383|16|132.64|7.95|140.59|55.06|455.20|86.21|45.03|1.40|518.21| +2452516|79215|10367|22181|1668118|4808|31098|22181|1668118|4808|31098|27|27|1384|16|148.80|1.48|150.28|13.06|5.92|13.39|93.43|41.98|20.46| +2452272|30875|915|45866|1915250|7034|10371|45866|1915250|7034|10371|59|23|1384|37|1792.28|161.30|1953.58|87.09|417.36|1541.36|50.18|200.74|665.75| +2452012|59745|3458|83628|1553999|7165|48505|83628|1553999|7165|48505|1|27|1386|32|1239.04|49.56|1288.60|76.58|1045.44|433.66|169.12|636.26|1171.58| +2451949|70941|14246|75696|1620860|3271|4435|75696|1620860|3271|4435|1|26|1386|3|19.05|1.14|20.19|64.83|69.33|14.47|3.66|0.92|135.30| +2451919|69022|17675|5631|161845|6859|27873|5631|161845|6859|27873|59|22|1386|2|8.04|0.64|8.68|10.96|0.60|2.01|3.37|2.66|12.20| +2451874|70778|1172|21109|160498|240|46829|21109|160498|240|46829|13|1|1386|7|437.29|17.49|454.78|26.01|53.55|0.00|52.47|384.82|97.05| +2451377|19846|13711|1093|425962|4431|45300|1093|425962|4431|45300|58|22|1387|53|563.39|45.07|608.46|95.07|650.31|28.16|123.10|412.13|790.45| +2451481|45037|7918|34648|589771|447|4370|34648|589771|447|4370|38|22|1387|32|901.44|72.11|973.55|17.05|2524.16|225.36|486.77|189.31|2613.32| +|39554|12824|17191|961716||33613|||||43||1387|32||5.52|557.84|||121.51||176.64|| +2452091|24382|14531|12351|490086|7021|33956|12351|490086|7021|33956|44|28|1389|4|112.32|0.00|112.32|23.97|18.72|11.23|60.65|40.44|42.69| +2452187|56000|8396|39937|1104875|6260|32798|39937|1104875|6260|32798|8|32|1389|3|159.63|1.59|161.22|89.61|28.59|28.73|3.92|126.98|119.79| +2452038|33607|12151|47831|1237312|4393|1474|47831|1237312|4393|1474|26|22|1389|48|5.28|0.00|5.28|8.26|82.08|4.01|0.40|0.87|90.34| +2451103|74579|5932|16668|839995|3322|48249|16668|839995|3322|48249|16|26|1390|12|772.20|30.88|803.08|44.35|133.80|308.88|463.32|0.00|209.03| +2452452|85005|9627|15829|226122|862|12755|15829|226122|862|12755|1|15|1392|34|541.28|37.88|579.16|25.46|584.12|405.96|43.30|92.02|647.46| +2451698|69892|4894|6587|26207|674|7494|6587|26207|674|7494|52|9|1393|50|1392.00|83.52|1475.52|52.70|1334.00|83.52|418.71|889.77|1470.22| +2452237|42825|12279|30159|1189081|2403|27725|30159|1189081|2403|27725|35|16|1394|2|191.12|3.82|194.94|9.44|23.88|172.00|12.81|6.31|37.14| +2451782|76612|5791|39137|717786|6388||39137|||6237||2|1397|||59.85||33.36|||359.39||| +2451397|76535|1870|61923|265883|1843|46156|61923|265883|1843|46156|26|21|1401|4|63.12|4.41|67.53|19.35|95.64|46.07|16.36|0.69|119.40| +2451514|44632|17662|85865|1099839|2966|29340|85865|1099839|2966|29340|43|10|1402|30|327.90|16.39|344.29|11.16|200.40|318.06|3.73|6.11|227.95| +2451497||4201|||||23234||4395|47999|||1402|34||||70.43|976.48||||1145.21| +2451129||5024||1398509|1316||14902|1398509|1316||46|35|1405|||16.50|||||22.78||125.94| +2451186|68289|1306|14413|654824|3588|11599|14413|654824|3588|11599|10|10|1405|4|269.68|2.69|272.37|29.14|134.84|266.98|0.81|1.89|166.67| +2452678|69015|5064|15646|851381|7116|41251|15646|851381|7116|41251|51|34|1407|45|1561.05|109.27|1670.32|99.40|780.30|1030.29|477.68|53.08|988.97| +2452700|25867|4032|74604|1770256|2708|17960|74604|1770256|2708|17960|19|28|1407|47|1723.96|137.91|1861.87|65.87|449.79|0.00|1499.84|224.12|653.57| +2451934|28157|13589|96227|500295|1963|27631|96227|500295|1963|27631|59|21|1408|19|368.03|7.36|375.39|79.51|208.43|14.72|183.72|169.59|295.30| +2452024|67499|13583|78180|1521235|3937|32239|78180|1521235|3937|32239|2|15|1408|60|8058.60|402.93|8461.53|6.49|2332.80|6849.81|229.67|979.12|2742.22| +2451854|28213|9773|20677|1215683|4736|16942|20677|1215683|4736|16942|50|35|1411|5|185.05|14.80|199.85|34.07|110.15|46.26|22.20|116.59|159.02| +2451655|28196|1423|25610|1561503|2196|47289|25610|1561503|2196|47289|47|12|1411|29|87.58|2.62|90.20|65.58|2.90|64.80|2.27|20.51|71.10| +2451325|55926|3290|63025|1159031|1437|25890|63025|1159031|1437|25890|34|33|1412|11|2999.59|209.97|3209.56|85.47|947.21|2219.69|218.37|561.53|1242.65| +2451195|67015|4462|14639|526489|4223|40890|14639|526489|4223|40890|34|26|1412|38|794.20|0.00|794.20|3.96|1227.40|627.41|15.01|151.78|1231.36| +2452773|12450|10519|39361|1007810|4259|31108|39361|1007810|4259|31108|33|29|1413|14|128.94|5.15|134.09|18.25|10.22|10.31|69.99|48.64|33.62| +2452710|45607|3597|18379|909507|635|26563|18379|909507|635|26563|27|23|1413|1|51.17|0.00|51.17|3.29|1.76|2.55|36.46|12.16|5.05| +2451922|34808|6500|92267|1202622|6130|17238|92267|1202622|6130|17238|17|2|1414|9|612.00|55.08|667.08|0.98|522.99|483.48|120.80|7.72|579.05| +2452674|44726|6679|99364|1568669|1974|14432|99364|1568669|1974|14432|12|15|1415|5|112.20|5.61|117.81|74.94|184.05|33.66|57.33|21.21|264.60| +2451801|62363|9082|45008|111254|2653|6499|45008|111254|2653|6499|8|19|1416|70|1339.10|26.78|1365.88|21.53|816.90|281.21|465.47|592.42|865.21| +2451842|34499|10063|47063|627594||15762|47063|627594||15762|55||1416|19||||26.39|1066.09|92.79|1533.92|229.21|1185.27| +2451685|73372|6002|25301|1263239|3577|4715|25301|1263239|3577|4715|2|2|1416|4|545.36|5.45|550.81|41.85|260.56|419.92|71.50|53.94|307.86| +2451643|44126|5293|9838|1176621|6604|29340|9838|1176621|6604|29340|2|2|1416|10|488.40|39.07|527.47|22.88|185.20|346.76|69.40|72.24|247.15| +2451841|27842|3187|33221|1484942|3063|337|33221|1484942|3063|337|31|11|1417|6|149.52|4.48|154.00|53.78|106.38|73.26|66.34|9.92|164.64| +2452052|40045|533|87223|1339711|3846|38334|87223|1339711|3846|38334|32|6|1421|3|235.89|2.35|238.24|67.80|121.65|148.61|60.22|27.06|191.80| +2451941|65129|10988|27227|1871916|2205|35741|27227|1871916|2205|35741|7|22|1421|53|17.49|0.52|18.01|98.68|62.54|13.11|3.06|1.32|161.74| +2452169|66747|421||28323|||53559||5487|9898|7||1422|1||||68.04||2.99||6.88|110.91| +2452247|64088|4895|17378|607589|3847|33705|17378|607589|3847|33705|23|4|1422|48|862.56|69.00|931.56|58.23|837.60|517.53|279.47|65.56|964.83| +2452546|66830|10722|94913|889780|4647|20314|94913|889780|4647|20314|1|1|1423|10|1049.30|83.94|1133.24|21.40|181.60|839.44|163.69|46.17|286.94| +2452751|82748|12357|95106|1842672|5438|1804|95106|1842672|5438|1804|3|10|1423|48|4533.60|181.34|4714.94|44.71|2755.68|2176.12|2098.15|259.33|2981.73| +2452580|74383|919|13325|272563|248|45486|13325|272563|248|45486|57|32|1423|7|27.09|2.16|29.25|74.13|318.85|8.39|10.65|8.05|395.14| +2452293|55306|831|39051|616615|2532|28096|39051|616615|2532|28096|27|18|1424|6|59.22|2.96|62.18|84.26|46.02|46.78|2.98|9.46|133.24| +2452302|60754|8633|98348|123189|6410|23137|98348|123189|6410|23137|41|32|1424|18|218.70|2.18|220.88|42.06|364.50|153.09|45.92|19.69|408.74| +2451362|71940|9728|22602|957887|1972|12010|22602|957887|1972|12010|8|26|1425|23|5543.23|110.86|5654.09|56.29|3012.54|3492.23|820.40|1230.60|3179.69| +2451880|58629|2618|64467|38343|5819|19248|64467|38343|5819|19248|41|7|1426|30|392.40|3.92|396.32|37.09|377.70|313.92|3.92|74.56|418.71| +2452497|18690|16913|43136|553742|7195|15001|43136|553742|7195|15001|3|30|1427|11|168.52|3.37|171.89|36.46|58.96|42.13|46.76|79.63|98.79| +2452325|45658|14059|45319|661478|2053|6645|45319|661478|2053|6645|3|14|1427|17|1184.73|94.77|1279.50|81.27|98.60|639.75|201.64|343.34|274.64| +2452239|61824|14159|72449|683296|3296|35343|72449|683296|3296|35343|7|19|1427|1|0.00|0.00|0.00|94.78|91.12|0.00|0.00|0.00|185.90| +2452669|32817|1399|48588|912880|7125|39017|48588|912880|7125|39017|6|26|1428|1|17.18|0.51|17.69|69.70|50.53|7.21|1.69|8.28|120.74| +2452317|2567|5077|76908|561084|6413|2942|76908|561084|6413|2942|3|28|1429|26|1517.10|121.36|1638.46|79.81|787.28|622.01|751.87|143.22|988.45| +2452367|15540|16605|92580|879042|5504|5166|92580|879042|5504|5166|15|4|1429|30|613.50|24.54|638.04|98.67|456.90|239.26|104.78|269.46|580.11| +2452568|48441|3990|41454|547395|983|43724|41454|547395|983|43724|39|19|1430|18|1610.10|64.40|1674.50|48.33|614.34|466.92|468.70|674.48|727.07| +2452530|61239|6201|81386|466185|5031|42230|81386|466185|5031|42230|24|12|1430|29|3116.63|280.49|3397.12|94.60|464.00|218.16|260.86|2637.61|839.09| +2452468|54888|6247|39716|1542965|3367|49416|39716|1542965|3367|49416|60|14|1430|38|1026.76|51.33|1078.09|62.68|1093.26|472.30|121.98|432.48|1207.27| +2451435|1312|14554|85570|1694689|5408|32527|85570|1694689|5408|32527|56|25|1431|2|41.50|1.24|42.74|20.67|109.70|24.90|1.16|15.44|131.61| +2452698|76744|17814|62282|102438|1794|7950|62282|102438|1794|7950|3|22|1432|64|2533.12|50.66|2583.78|80.16|2092.80|2457.12|69.92|6.08|2223.62| +2451325|13650|8695|54728|849708|7088|21268|54728|849708|7088|21268|56|28|1434|19|887.11|0.00|887.11|35.71|363.66|372.58|72.03|442.50|399.37| +|27100|12860|3248||3011||3248|226553|3011|11116||5|1434||502.18|20.08||4.60|30.10||||| +2451210|43073|1096|43561|470763|5165|12767|43561|470763|5165|12767|34|2|1434|12|941.04|37.64|978.68|61.50|238.08|611.67|256.90|72.47|337.22| +2451274|74463|11245|45045|865637|3890|12707|45045|865637|3890|12707|7|9|1436|17|111.18|0.00|111.18|49.12|21.42|62.26|35.71|13.21|70.54| +2451436|75547|4843|45045|865637|3890|12707|45045|865637|3890|12707|37|27|1436|6|467.82|4.67|472.49|60.46|55.32|4.67|83.36|379.79|120.45| +2451879|64422|4964|95492|701005|918|35231|95492|701005|918|35231|29|32|1438|33|1385.67|69.28|1454.95|43.35|1283.04|803.68|512.15|69.84|1395.67| +2452168|67274|8083|44217|1548937|4057|34777|44217|1548937|4057|34777|27|22|1442|28|329.00|6.58|335.58|75.58|441.84|253.33|13.62|62.05|524.00| +2452564|35600|14689|27429|764151|6120|40042|27429|764151|6120|40042|13|33|1443|52|371.80|14.87|386.67|10.51|552.24|230.51|121.50|19.79|577.62| +2452474|42153|17119|16592|1553939|2133|10280|16592|1553939|2133|10280|21|16|1443|38|34.20|1.02|35.22|45.15|182.40|2.39|28.31|3.50|228.57| +2452219|78935|10975|39289|193862|3635|9808|39289|193862|3635|9808|5|17|1444|65|1055.60|63.33|1118.93|83.30|447.85|601.69|403.97|49.94|594.48| +2451356|74861|17678|79601|1130337|1873|26609|79601|1130337|1873|26609|8|15|1447|5|304.80|9.14|313.94|24.75|312.65|259.08|33.83|11.89|346.54| +2451811|37982|3839|80328|1476572|4016|46213|80328|1476572|4016|46213|38|13|1449|1|63.67|5.73|69.40|75.68|74.29|1.91|58.05|3.71|155.70| +2452383|16040|14631|51283|1367134|5832|1683|51283|1367134|5832|1683|1|17|1450|11|539.77|21.59|561.36|29.43|556.60|221.30|213.37|105.10|607.62| +2452289|41347|11299|79775|1465452|7129|31606|79775|1465452|7129|31606|47|18|1450|2|218.66|2.18|220.84|93.97|9.92|118.07|35.20|65.39|106.07| +2452176|68582|9601|24294|668296|651|3053|24294|668296|651|3053|23|28|1450|56|4142.88|248.57|4391.45|18.83|819.84|3977.16|13.25|152.47|1087.24| +2452439|68935|15185|46560|689531|1302|20720|46560|689531|1302|20720|9|35|1450|1|2.18|0.10|2.28|80.39|4.64|0.95|0.52|0.71|85.13| +2452170|46431|13623|13896|1533632|2168|3951|13896|1533632|2168|3951|7|12|1450|48|8166.72|490.00|8656.72|71.36|1425.60|5308.36|1543.51|1314.85|1986.96| +2452418|70423|6971|76675|129954|4480|3759|76675|129954|4480|3759|33|25|1450|7|851.20|34.04|885.24|56.13|353.85|672.44|42.90|135.86|444.02| +2451555|36094|7264|61054|611135|1663|32991|61054|611135|1663|32991|20|30|1451|33|248.16|17.37|265.53|93.89|443.52|193.56|51.32|3.28|554.78| +2451640|7428|1436|34467|1566905|943|19381|34467|1566905|943|19381|52|26|1451|56|419.44|12.58|432.02|25.33|838.88|71.30|45.25|302.89|876.79| +2451542|42704|7618|3700|26018|336|49007|3700|26018|336|49007|40|3|1451|1|17.61|1.23|18.84|53.47|6.60|14.08|1.30|2.23|61.30| +2451259|46186|1100|9885|928563|4041|23919|9885|928563|4041|23919|52|19|1454|17|143.14|0.00|143.14|99.23|308.55|133.12|7.81|2.21|407.78| +||17737||1416561|5071|1534|97298|1416561|5071||47|13|1455|||0.00|1136.52|76.08|25.08||||101.16| +2452502|69005|6897|87683|377693|1750|5360|87683|377693|1750|5360|45|9|1455|8|440.88|35.27|476.15|52.90|277.60|13.22|38.48|389.18|365.77| +||4145|68669|1881300||||1881300||8915|35|33|1455|62|174.84||185.33||115.32|||30.85|| +2451321|72723|16084|22546|1371332|1271|44057|22546|1371332|1271|44057|1|14|1457|49|2630.81|0.00|2630.81|86.71|1456.28|420.92|1944.70|265.19|1542.99| +2452503|9539|6153|44060|1824512|964|36663|44060|1824512|964|36663|9|19|1458|3|226.50|20.38|246.88|25.87|29.04|110.98|36.96|78.56|75.29| +2451157|64271|2980|89338||6503|2109|89338|129697|6503|2109||29|1459||211.32|8.45|219.77||3240.00|188.07|4.65|18.60|| +2451637|67160|15872|81825|1484293|4361|19182|81825|1484293|4361|19182|43|16|1461|37|1919.56|115.17|2034.73|2.85|657.49|1574.03|228.04|117.49|775.51| +2452729|33707|16245||1279163||14926|44905||5421||||1463||591.39|||4.62|||274.17|86.58|| +2451269|13913|13201|58316|1078552|4741|14577|58316|1078552|4741|14577|26|24|1467|31|1159.71|92.77|1252.48|0.70|2899.74|208.74|741.75|209.22|2993.21| +2451973|74530|4793|32555|715956|2910|44711|32555|715956|2910|44711|13|16|1468|63|782.46|0.00|782.46|11.32|82.53|305.15|429.57|47.74|93.85| +2451570|40381|1960|64596|715736|6076|34016|64596|715736|6076|34016|50|3|1469|25|170.75|11.95|182.70|12.43|88.25|70.00|86.64|14.11|112.63| +2452152|54480|15815|80878|1706220|1162|20104|80878|1706220|1162|20104|56|11|1470|49|928.55|0.00|928.55|18.35|2105.04|204.28|340.40|383.87|2123.39| +2451569|42337|8572|22977|907329|2528|28127|22977|907329|2528|28127|31|14|1474|21|344.19|3.44|347.63|11.46|0.00|165.21|134.23|44.75|14.90| +2451695|71880|13232|44146|1771416|1439|42309|44146|1771416|1439|42309|8|22|1475|22|1946.56|116.79|2063.35|22.89|665.94|1498.85|447.71|0.00|805.62| +2451427|31259|3853|77896|1056795|4494|3346|77896|1056795|4494|3346|40|28|1476|12|453.60|4.53|458.13|21.72|469.80|276.69|35.38|141.53|496.05| +2451291|64524|106|12613|1149415|4170|25772|12613|1149415|4170|25772|14|5|1476|27|167.94|8.39|176.33|43.97|91.26|83.97|52.06|31.91|143.62| +2452788|43634|10351|97704|1691028|1952|49676|97704|1691028|1952|49676|3|35|1478|40|367.60|7.35|374.95|47.51|16.40|272.02|72.64|22.94|71.26| +2452757|56780|4740|35387|1880530|4873|46416|35387|1880530|4873|46416|9|1|1478|3|25.83|0.77|26.60|51.06|51.69|23.24|0.95|1.64|103.52| +2452510|58919|14595|79394|1235667|1540|38260|79394|1235667|1540|38260|5|12|1481|51|0.00|0.00|0.00|54.67|2446.47|0.00|0.00|0.00|2501.14| +2452495|79122|17507|65741|999642|4743|3844|65741|999642|4743|3844|45|12|1481|44|57.20|1.71|58.91|42.77|58.52|20.02|5.57|31.61|103.00| +2452179|18249|15481|97325|1810101|477|33300|97325|1810101|477|33300|15|16|1482|18|358.38|21.50|379.88|51.82|268.74|229.36|11.61|117.41|342.06| +2452849|78095|6477|5195|861214|2480|6946|5195|861214|2480|6946|36|18|1483|5|341.90|20.51|362.41|35.91|80.00|105.98|40.10|195.82|136.42| +|38500|6813|95874|430326|1578|39583||430326|||21||1483|40|917.20|45.86||||632.86|||| +2451262|77874|11323|28981|1160418|1455|42092|28981|1160418|1455|42092|44|24|1485|41|2624.82|0.00|2624.82|8.47|124.64|2178.60|165.10|281.12|133.11| +2452762|63202|10411|97948|404067|2539|33858|97948|404067|2539|33858|3|13|1486|27|1638.09|32.76|1670.85|67.65|1389.96|1375.99|15.72|246.38|1490.37| +2452472|18266|17533|41968|160362|3203|8397|41968|160362|3203|8397|7|12|1487|10|72.10|0.00|72.10|20.03|43.20|34.60|16.12|21.38|63.23| +2451423|67385|11296|13347|1141172|6597|46869|13347|1141172|6597|46869|55|21|1489|29|1161.45|0.00|1161.45|70.97|243.02|348.43|666.67|146.35|313.99| +|71905|5194||708239|5537|6035|22941||5537|6035|||1489||2891.84|115.67||58.58|||91.96||| +2451878|18860|5819|81466|1186361|409|12504|81466|1186361|409|12504|13|32|1490|13|833.56|16.67|850.23|91.70|329.03|683.51|129.04|21.01|437.40| +2451872|50269|14635|81201|1178347|3140|7933|81201|1178347|3140|7933|56|5|1490|10|1933.20|115.99|2049.19|64.33|593.10|425.30|663.47|844.43|773.42| +2452770|62977|6255|53306|267019|4732|7198|53306|267019|4732|7198|42|32|1491|8|929.68|65.07|994.75|25.16|398.40|790.22|65.54|73.92|488.63| +2452855|7678|8208|2501|1339554|5407|20860|2501|1339554|5407|20860|45|11|1491|10|216.50|2.16|218.66|64.87|63.10|186.19|30.31|0.00|130.13| +2451730|53815|2978|91361|1419116|3224|28579|91361|1419116|3224|28579|8|13|1492|16|1077.28|86.18|1163.46|67.96|511.36|635.59|295.93|145.76|665.50| +2452016|66974|17732|72076|831490|6457|21471|72076|831490|6457|21471|7|23|1492|31|4943.88|346.07|5289.95|36.88|1167.15|3609.03|573.98|760.87|1550.10| +2452160|67006|14777|48762|512610|1611|39059|48762|512610|1611|39059|41|29|1493|5|418.55|4.18|422.73|61.99|226.70|364.13|28.29|26.13|292.87| +2452453|18343|7765|28877|423874|6071|29868|28877|423874|6071|29868|39|3|1494|7|86.45|0.86|87.31|77.88|41.86|37.17|44.35|4.93|120.60| +2452648|61578|13434|88855|1749890|4172|46454|88855|1749890|4172|46454|55|2|1495|49|3662.26|219.73|3881.99|8.35|1192.17|329.60|433.24|2899.42|1420.25| +2451828|32343|13340|73646|1074165|6684|3|73646|1074165|6684|3|50|32|1497|7|941.43|56.48|997.91|16.91|382.41|131.80|728.66|80.97|455.80| +2451898|22226|4388|98993|1008024|3455|14839|98993|1008024|3455|14839|11|23|1497|2|27.50|0.82|28.32|24.38|5.78|21.17|1.58|4.75|30.98| +2452756|61104|3618|88571|1485983|3842|7254|88571|1485983|3842|7254|39|4|1498|32|1304.00|39.12|1343.12|71.53|604.16|260.80|219.07|824.13|714.81| +2452776|71459|3025||1638718|1935|10705||1638718|1935||||1498|6|||151.30|26.05|||||77.87| +2452651|40389|6403|11732|1731442|1905|13615|11732|1731442|1905|13615|42|20|1498|67|18.76|0.56|19.32|28.87|754.42|3.93|10.82|4.01|783.85| +2451173|25674|3301|95728|1274791|6654|26917|95728|1274791|6654|26917|20|15|1501|8|774.72|30.98|805.70|48.34|228.56|309.88|371.87|92.97|307.88| +||12031|||4699|||774051||9747|26||1502|21||22.57||||76.20|||591.97| +2451744|75926|12973|31770|915976|6135|33894|31770|915976|6135|33894|53|17|1502|39|590.07|47.20|637.27|55.43|244.53|64.90|204.81|320.36|347.16| +2450876|10044|9742|96146|881863|5980|32097|96146|881863|5980|32097|16|19|1504|34|83.64|5.01|88.65|53.80|1593.92|22.58|24.42|36.64|1652.73| +2451879|63465|13487|66524|950562|1000|41438|66524|950562|1000|41438|11|11|1505|50|0.00|0.00|0.00|89.89|414.00|0.00|0.00|0.00|503.89| +2451139|82016|3752|86753|220453|2953|21853|86753|220453|2953|21853|4|34|1507|51|9485.49|0.00|9485.49|22.64|6280.65|6355.27|1972.03|1158.19|6303.29| +2451141|34804|15592|11592|1333723|1454|41985|11592|1333723|1454|41985|44|28|1507|33|326.70|3.26|329.96|45.44|788.37|147.01|55.70|123.99|837.07| +2452647|24607|16326|41527|346673|25|49157|41527|346673|25|49157|27|10|1508|1|55.00|3.30|58.30|43.75|88.48|28.05|2.15|24.80|135.53| +2451595|49620|4064|94729|439881|6633|18585|94729|439881|6633|18585|14|35|1509|78|3439.02|171.95|3610.97|89.53|572.52|2166.58|1221.54|50.90|834.00| +2452889|81480|8829|94205|1494790|5775|18760|94205|1494790|5775|18760|12|21|1512|2|30.50|2.44|32.94|74.63|8.96|11.28|2.69|16.53|86.03| +2452737|67608|14868|80472|1668620|6815|38563|80472|1668620|6815|38563|1|18|1512|1|2.85|0.11|2.96|37.89|14.98|2.28|0.21|0.36|52.98| +2452529|57815|9567|78771|16356|1494|20632|78771|16356|1494|20632|55|9|1513|6|349.80|0.00|349.80|66.30|246.18|118.93|6.92|223.95|312.48| +2452473|19477|1995|48377|1113621|2065|46509|48377|1113621|2065|46509|59|30|1513|55|4445.65|0.00|4445.65|51.02|2444.75|1333.69|2831.88|280.08|2495.77| +2451470|81790|7273|97520|1586396|2025|7254|97520|1586396|2025|7254|19|29|1514|2|45.32|0.00|45.32|24.09|49.44|33.99|7.02|4.31|73.53| +2451434|68763|4405|76244|149246|3401|25203|76244|149246|3401|25203|26|14|1514|47|3576.23|0.00|3576.23|8.67|1164.19|572.19|2793.75|210.29|1172.86| +|49278|6653|99192||6963||99192|1536901|6963|20672|21||1515|||21.65|||63.20|||123.44|170.09| +2452276|22129|4337|68438|194205|5572|5386|68438|194205|5572|5386|1|15|1515|45|19.35|0.19|19.54|17.34|111.15|6.96|3.34|9.05|128.68| +2452431|17312|9525|49461|1797654|6262|8051|49461|1797654|6262|8051|31|28|1516|11|367.29|18.36|385.65|4.50|264.00|102.84|227.42|37.03|286.86| +2452282|50296|13049|73836|246252|3915|33131|73836|246252|3915|33131|9|16|1517|93|1249.92|37.49|1287.41|14.88|1028.58|374.97|96.24|778.71|1080.95| +||4763|37006|||44924|37006||6818|44924|||1517|26||129.60||28.94|1974.96|370.29|444.35|1036.82|| +2452787|35497|13668|27272|||49386|27272||3850|49386|51||1520|35||||||295.57|41.38||426.09| +2452906|62332|14241|58280|363179|3631|25330|58280|363179|3631|25330|55|1|1520|2|49.82|0.99|50.81|8.94|40.08|42.34|4.86|2.62|50.01| +2452349|64051|6233|21939|539404|3716|40381|21939|539404|3716|40381|45|24|1522|33|1270.17|38.10|1308.27|26.38|702.57|1257.46|0.12|12.59|767.05| +2452351|45096|13919|55441|1566559|5324|30089|55441|1566559|5324|30089|5|2|1522|23|1433.36|71.66|1505.02|74.76|822.71|659.34|270.90|503.12|969.13| +2451657|70956|13348|2801|1620087|573|8659|2801|1620087|573|8659|52|7|1523|17|1946.67|58.40|2005.07|69.10|663.51|194.66|823.44|928.57|791.01| +2452748|58425|7515|65846|1276359|2872|39785|65846|1276359|2872|39785|60|13|1524|25|618.00|49.44|667.44|97.91|125.75|562.38|10.01|45.61|273.10| +2451742|64271|14701|63441|1146556|3624|28339|63441|1146556|3624|28339|47|19|1525|47|5097.62|0.00|5097.62|70.13|2825.17|3109.54|1948.31|39.77|2895.30| +2451794|61349|12272|51810|946072|1581|290|51810|946072|1581|290|37|8|1525|14|2415.28|72.45|2487.73|7.99|413.98|1352.55|53.13|1009.60|494.42| +2451799|41508|12307|35794|1410929|4911|48893|35794|1410929|4911|48893|2|29|1525|25|1301.75|78.10|1379.85|83.23|367.00|299.40|661.55|340.80|528.33| +2452189|8235|308|79706|585604|2255|14106|79706|585604|2255|14106|2|19|1526|13|346.32|20.77|367.09|98.50|46.15|124.67|201.70|19.95|165.42| +2452010|59093|16789|35735|1281674|5213|28752|35735|1281674|5213|28752|47|13|1526|38|709.08|14.18|723.26|25.53|1013.08|319.08|265.20|124.80|1052.79| +2452437|41039|6457|34325|1693607|1294|1191|34325|1693607|1294|1191|17|11|1527|29|1041.39|0.00|1041.39|63.13|388.89|781.04|252.53|7.82|452.02| +2452450|59823|5499|45243|462399|6755|29158|45243|462399|6755|29158|37|28|1528|24|108.00|9.72|117.72|50.67|40.32|55.08|25.40|27.52|100.71| +2452730|19178|4233|78767|954134|2301|19668|78767|954134|2301|19668|18|19|1530|59|183.49|0.00|183.49|34.30|130.39|60.55|89.74|33.20|164.69| +2451091|67592|9301|81074|1824951|1627|46780|81074|1824951|1627|46780|22|11|1533|14|967.40|87.06|1054.46|68.67|351.82|831.96|130.02|5.42|507.55| +2451084|68706|7741|85973|1213342|3911|6480|85973|1213342|3911|6480|10|26|1533|51|1781.43|142.51|1923.94|78.96|54.06|1656.72|3.74|120.97|275.53| +2451400|68847|13598|40032|1149056|6433|47057|40032|1149056|6433|47057|56|3|1534|47|136.77|8.20|144.97|94.74|4.23|64.28|3.62|68.87|107.17| +2451251|39718|6056|24133|810785|7037|40089|24133|810785|7037|40089|26|11|1535|4|519.20|46.72|565.92|31.06|38.72|0.00|493.24|25.96|116.50| +2451269|50337|8834|32493|1371363|633|32303|32493|1371363|633|32303|1|8|1535|7|22.54|0.45|22.99|85.56|25.20|14.20|1.66|6.68|111.21| +2452502|35321|435|35122|570500|1078|14409|35122|570500|1078|14409|53|3|1536|13|274.30|5.48|279.78|27.10|174.85|202.98|14.26|57.06|207.43| +2451295|62617|2443|84179|1097304|829|28001|84179|1097304|829|28001|8|20|1537|11|1613.48|0.00|1613.48|98.93|0.00|484.04|847.08|282.36|98.93| +2451138|36257|12286|54630|1159477|1124|47971|54630|1159477|1124|47971|56|23|1537|19|1386.62|41.59|1428.21|26.93|24.32|637.84|584.04|164.74|92.84| +2451176|62797|15488|28944|1766382|5067|44905|28944|1766382|5067|44905|20|9|1537|1|37.12|1.48|38.60|7.29|3.33|27.84|4.91|4.37|12.10| +2451119|65969|10168|24171|1109583|595|12007|24171|1109583|595|12007|46|34|1539|82|9273.38|92.73|9366.11|23.54|323.08|8531.50|727.04|14.84|439.35| +2451347|19684|6736|35709|282198|1134|47270|35709|282198|1134|47270|58|17|1539|1|90.88|5.45|96.33|41.59|16.52|40.89|23.99|26.00|63.56| +2451613|37715|12688|55096|1365594|2492|12590|55096|1365594|2492|12590|10|33|1541|19|1973.34|98.66|2072.00|88.25|902.69|1065.60|390.32|517.42|1089.60| +2452006|13545|12379|75529|870195|2461|23127|75529|870195|2461|23127|29|1|1542|23|45.54|2.27|47.81|97.62|243.11|42.80|0.98|1.76|343.00| +2452197|75371|3133|86537|1055837|2323|31882|86537|1055837|2323|31882|5|31|1543|15|2075.85|186.82|2262.67|91.03|1037.85|1411.57|245.78|418.50|1315.70| +2452661|71606|6631|23793|878423|3453|28883|23793|878423|3453|28883|1|6|1544|42|798.00|39.90|837.90|45.83|144.90|574.56|26.81|196.63|230.63| +2451571|62025|13159|35305|475133|2161|33236|35305|475133|2161|33236|22|21|1546|4|10.40|0.20|10.60|97.16|197.88|7.59|1.60|1.21|295.24| +2452706|14606|4357|28962|522034|3106|3378|28962|522034|3106|3378|55|29|1547|17|739.16|0.00|739.16|51.55|112.71|421.32|63.56|254.28|164.26| +2452749|58144|9655|64400|1309345|4690|49635|64400|1309345|4690|49635|24|15|1547|55|156.75|10.97|167.72|86.77|267.85|43.89|36.11|76.75|365.59| +2452826|71169|2154|99439|576733|74|2168|99439|576733|74|2168|42|20|1547|8|321.76|28.95|350.71|63.65|33.52|3.21|273.95|44.60|126.12| +2452810|7843|15501|15927|1217393|459|30725|15927|1217393|459|30725|21|21|1549|34|2866.88|258.01|3124.89|58.54|2011.78|0.00|1146.75|1720.13|2328.33| +2452655|55286|1935|72081|637519|3757|2442|72081|637519|3757|2442|57|20|1549|66|477.84|19.11|496.95|97.89|7646.76|95.56|194.96|187.32|7763.76| +2451700|16503|562|9534|52659|4937|21615|9534|52659|4937|21615|44|12|1550|7|628.46|37.70|666.16|17.27|234.85|389.64|114.63|124.19|289.82| +2451618|62219|5630|84073|1648056|4039|4822|84073|1648056|4039|4822|14|26|1550|3|65.82|2.63|68.45|46.79|46.56|50.68|13.47|1.67|95.98| +||16022||309720|5569||74224|309720|||56||1550|||468.67|5676.19|5.05||1926.78|3083.89||| +2452899|71850|9522|5996|1412846|4496|10946|5996|1412846|4496|10946|12|31|1553|73|3925.94|353.33|4279.27|46.78|844.61|117.77|1180.53|2627.64|1244.72| +2452533|41778|8245|8390|994053|869|25732|8390|994053|869|25732|48|34|1555|35|474.25|37.94|512.19|5.05|204.05|369.91|28.17|76.17|247.04| +2452487|75918|11497|14114|607140|3994|8442|14114|607140|3994|8442|31|3|1555|5|79.20|0.79|79.99|72.01|69.30|69.69|8.17|1.34|142.10| +||6583||1222395|6564|||1222395|||42||1555||252.80|7.58||69.63|661.28|108.70|73.49|70.61|738.49| +2452492||636||7431|7146|7308|83716|7431|||||1555|||40.94|552.71|48.06|||4.75|471.20|| +2452509|62686|9006|14114|607140|3994|8442|14114|607140|3994|8442|60|18|1555|46|3256.34|65.12|3321.46|41.95|132.48|390.76|2263.80|601.78|239.55| +2451956|32577|17245|7194|764630|2866|40302|7194|764630|2866|40302|17|6|1557|33|751.74|30.06|781.80|40.05|542.85|210.48|119.07|422.19|612.96| +2451830|81914|931|11160|168085|563|16641|11160|168085|563|16641|29|16|1557|38|3847.12|76.94|3924.06|32.29|1784.10|1615.79|535.51|1695.82|1893.33| +2451406|21930|11431|75946|353312|278|12091|75946|353312|278|12091|38|22|1558|31|3043.58|182.61|3226.19|77.90|1057.10|2952.27|69.39|21.92|1317.61| +2452332|46885|2123|56161|1190303|3645|17387|56161|1190303|3645|17387|3|24|1560|19|1574.34|125.94|1700.28|24.73|819.85|739.93|442.23|392.18|970.52| +2452524|28502|7461|55804|1654035|3913|49344|55804|1654035|3913|49344|7|11|1560|1|10.85|0.75|11.60|68.85|0.00|10.30|0.30|0.25|69.60| +2452373|50895|1907|35538|1076484|1877|10823|35538|1076484|1877|10823|41|28|1560|19|1851.17|74.04|1925.21|83.15|631.56|703.44|1021.47|126.26|788.75| +||16264||1326656||||||21950|||1562||417.60|4.17|421.77|||41.76|||| +2451128|31833|16184|63454|1649150|5599|22448|63454|1649150|5599|22448|44|1|1563|60|709.20|42.55|751.75|52.62|531.60|482.25|111.20|115.75|626.77| +2451263|33728|3862|8207|585058|2262|26313|8207|585058|2262|26313|26|8|1563|45|689.40|13.78|703.18|76.91|98.10|372.27|221.99|95.14|188.79| +2451584|73907|6008|57205|534081|5004|22680|57205|534081|5004|22680|14|10|1564|31|146.94|0.00|146.94|71.77|3091.63|36.73|79.35|30.86|3163.40| +||14475|29017|725473|||||776||45|20|1566|19|422.37|12.67|435.04|91.32|178.60|363.23|30.75||282.59| +2451700||14626|26314||5041||26314|264301|5041|10628||19|1567|22|253.44|||87.29|166.32|78.56|||268.81| +2451738|17240|14156|96127|1331089|2498|12660|96127|1331089|2498|12660|38|21|1567|2|13.76|0.13|13.89|47.77|56.34|1.92|8.64|3.20|104.24| +2451962|52366|14822|28280|207369|3258|26700|28280|207369|3258|26700|59|7|1568|61|5944.45|178.33|6122.78|30.80|3293.39|3210.00|2242.24|492.21|3502.52| +2451354|43162|15514|23002|950636|4670|29104|23002|950636|4670|29104|2|28|1569|3|6.42|0.25|6.67|75.89|36.27|0.12|2.70|3.60|112.41| +2452410|41025|1217|9702|8202|4655|18308|9702|8202|4655|18308|43|12|1570|8|124.96|4.99|129.95|80.99|35.12|46.23|46.45|32.28|121.10| +2452469|16545|10205|2384|428621|4905|32342|2384|428621|4905|32342|21|21|1570|6|0.00|0.00|0.00|91.84|67.56|0.00|0.00|0.00|159.40| +2452422|15836|823|46548|1607623|251|8126|46548|1607623|251|8126|19|35|1570|66|1240.14|99.21|1339.35|13.61|812.46|359.64|61.63|818.87|925.28| +2451652|79692|5800|9031|302852|3073|49294|9031|302852|3073|49294|7|7|1572|3|184.77|16.62|201.39|12.38|65.97|184.77|0.00|0.00|94.97| +2451588|18828|1310|53005|1481807|3409|44658|53005|1481807|3409|44658|38|12|1572|15|913.20|73.05|986.25|62.67|189.45|365.28|21.91|526.01|325.17| +2451640|48185|10372|92291|1216283|3828|42173|92291|1216283|3828|42173|8|9|1572|28|1803.20|0.00|1803.20|90.84|243.60|901.60|847.50|54.10|334.44| +2451934|34075|769|18114|732601|6288|9528|18114|732601|6288|9528|11|23|1573|22|917.84|64.24|982.08|44.95|191.18|807.69|87.01|23.14|300.37| +2452447|63043|13867|83884|942141|5198|22655|83884|942141|5198|22655|29|12|1574|51|4277.88|213.89|4491.77|17.78|809.37|1197.80|2156.05|924.03|1041.04| +2452371|49559|13215|31981|1581401|68|11135|31981|1581401|68|11135|13|21|1574|6|34.44|0.68|35.12|60.72|120.66|7.57|15.85|11.02|182.06| +2452559|78335|13191|60331|1861479|1027|33422|60331|1861479|1027|33422|35|35|1574|62|228.78|0.00|228.78|1.47|523.28|203.61|9.81|15.36|524.75| +2451480|65205|15904|9963|442407|5251|1695|9963|442407|5251|1695|43|19|1575|1|35.23|1.76|36.99|10.39|16.20|25.36|1.38|8.49|28.35| +2452055|26057|17425|42652|307688|6771|9807|42652|307688|6771|9807|43|34|1577|23|709.78|56.78|766.56|99.06|650.67|170.34|204.98|334.46|806.51| +2452200|64942|3359|1570|873522|4321|26273|1570|873522|4321|26273|37|11|1577|49|1226.47|61.32|1287.79|18.92|1362.69|1189.67|10.67|26.13|1442.93| +2452212|57854|10831|81791|416477|1563|7896|81791|416477|1563|7896|3|27|1577|51|471.75|42.45|514.20|45.85|1415.76|14.15|242.52|215.08|1504.06| +2452325|53572|10063|38068|906103|891|7279|38068|906103|891|7279|17|23|1578|24|947.28|28.41|975.69|76.85|0.00|18.94|482.73|445.61|105.26| +2451278|55781|15853|6807|1066551|5495|18086|6807|1066551|5495|18086|1|29|1579|27|622.08|37.32|659.40|29.89|223.02|385.68|113.47|122.93|290.23| +2451535|74034|11140|42798|1609077|2109|24388|42798|1609077|2109|24388|1|7|1581|9|353.07|7.06|360.13|38.54|36.81|35.30|282.81|34.96|82.41| +2451621|32048|17533|23044|26218|5392|2166|23044|26218|5392|2166|25|13|1581|3|482.70|33.78|516.48|52.99|265.83|458.56|4.10|20.04|352.60| +2451736|15265|2696|25309|1177312|1668|45671|25309|1177312|1668|45671|14|5|1582|14|231.56|2.31|233.87|97.42|125.86|50.94|43.34|137.28|225.59| +2451662|58831|13706|81226|1664904|3335|1013|81226|1664904|3335|1013|52|35|1582|91|1399.58|0.00|1399.58|69.82|77.35|475.85|729.74|193.99|147.17| +2452670|74154|15019|10398|1790629|460|18691|10398|1790629|460|18691|7|2|1583|82|5111.88|357.83|5469.71|37.25|2595.30|3373.84|1512.09|225.95|2990.38| +2452557|71259|14262|3504|1077637|6970|13103|3504|1077637|6970|13103|25|9|1583|27|287.55|0.00|287.55|78.30|594.27|129.39|124.94|33.22|672.57| +2452131|16377|5353|8983|989863|1590|16870|8983|989863|1590|16870|53|14|1585|78|1025.70|0.00|1025.70|12.68|5698.68|20.51|1005.19|0.00|5711.36| +2452115|75327|9361|97676|1306217|4358|46499|97676|1306217|4358|46499|37|10|1585|27|1230.12|36.90|1267.02|97.34|102.33|615.06|55.35|559.71|236.57| +2451435|79709|10094|75295|737044|3146|7695|75295|737044|3146|7695|34|31|1586|13|287.95|11.51|299.46|94.09|112.58|241.87|39.62|6.46|218.18| +2451725|6529|4064|4409|766454|1260|18638|4409|766454|1260|18638|56|28|1586|63|3710.70|0.00|3710.70|81.54|1855.35|3116.98|368.10|225.62|1936.89| +2451403|81544|15358|62147|1868733|2519|19505|62147|1868733|2519|19505|32|26|1587|6|410.40|36.93|447.33|87.16|180.18|287.28|7.38|115.74|304.27| +||16652|32421|676760||23838|32421|676760|1899|23838|26|22|1587|5|0.00|0.00||65.10||0.00||0.00|75.00| +2451799|70922|8887|11586|329250|2114|39778|11586|329250|2114|39778|10|29|1588|22|342.98|13.71|356.69|90.66|205.70|308.68|18.52|15.78|310.07| +2451612|38109|8287|66075|1299735|2195|34655|66075|1299735|2195|34655|34|12|1588|7|365.05|3.65|368.70|19.33|186.13|36.50|91.99|236.56|209.11| +2451769|73728|10552|39591|1878292|1793|4447|39591|1878292|1793|4447|2|26|1588|9|77.58|6.98|84.56|31.35|443.70|65.94|10.70|0.94|482.03| +2451498|70124|5636|10150|1224013|1076|31739|10150|1224013|1076|31739|31|23|1589|32|286.08|11.44|297.52|40.67|58.24|243.16|39.91|3.01|110.35| +2451578|48675|308|90312|672994|476|29605|90312|672994|476|29605|19|6|1589|35|698.95|27.95|726.90|67.03|163.10|307.53|39.14|352.28|258.08| +2451503|71710|14113|62194|19971|5748|17972|62194|19971|5748|17972|40|15|1591|8|15.36|0.92|16.28|9.42|25.60|9.83|1.38|4.15|35.94| +2451658|66913|568|17462|1065279|5449|44819|17462|1065279|5449|44819|8|1|1591|7|502.67|25.13|527.80|47.20|371.56|30.16|415.80|56.71|443.89| +2451483|38397|17420|48313|889946|1838|12155|48313|889946|1838|12155|7|29|1593|7|892.85|80.35|973.20|71.81|393.40|517.85|22.50|352.50|545.56| +2451757|43493|5726|90986|1446633|3080|28680|90986|1446633|3080|28680|43|20|1596|6|1396.62|41.89|1438.51|26.65|163.38|1061.43|187.70|147.49|231.92| +2452792|74264|3811|20363|92734|4305|8276|20363|92734|4305|8276|21|22|1597|72|13706.64|137.06|13843.70|39.40|4312.08|6305.05|2368.50|5033.09|4488.54| +2452941|44685|13093|74845|345915|4185|13655|74845|345915|4185|13655|43|10|1597|65|227.50|4.55|232.05|5.12|468.65|45.50|182.00|0.00|478.32| +2452505|62060|11793|1645|1125274|4903|27732|1645|1125274|4903|27732|25|15|1598|56|617.12|||||370.27|||| +2452309|55419|17339|25709|471765|574|36183|25709|471765|574|36183|33|6|1598|28|24.92|0.99|25.91|23.56|96.32|11.96|0.25|12.71|120.87| +2451559|54511|13544|1240|1333500|5407|38914|1240|1333500|5407|38914|58|24|1600|9|76.41|0.76|77.17|82.72|23.40|52.72|17.05|6.64|106.88| +2452700|74342|9516|23284|167702|120|4538|23284|167702|120|4538|19|29|1602|5|338.35|0.00|338.35|37.50|169.15|10.15|157.53|170.67|206.65| +2452676|76723|12001|35074|1407349|2024|26746|35074|1407349|2024|26746|30|35|1602|35|2440.90|122.04|2562.94|36.47|1351.70|1171.63|456.93|812.34|1510.21| +2452658|20250|4197|68249|1193978|1307|31436|68249|1193978|1307|31436|24|35|1604|98|814.38|0.00|814.38|35.90|814.38|65.15|292.19|457.04|850.28| +2451440|31265|4501|53144|726217|2429|49933|53144|726217|2429|49933|2|34|1607|42|1602.30|48.06|1650.36|61.74|2617.02|368.52|1196.76|37.02|2726.82| +2452528|45355|12249|46617|982353|5289|4178|46617|982353|5289|4178|3|15|1608|29|1291.37|90.39|1381.76|57.43|32.19|400.32|552.45|338.60|180.01| +2452759|70685|14967|36326|13315|4704|10493|36326|13315|4704|10493|39|19|1608|51|6011.37|541.02|6552.39|12.47|1138.83|2885.45|2188.14|937.78|1692.32| +2451974|70605|17264|6160|1587663|5166|39869|6160|1587663|5166|39869|20|22|1609|5|936.90|56.21|993.11|86.74|33.85|252.96|198.34|485.60|176.80| +2451972|69633|632|18328|1749717|1106|27695|18328|1749717|1106|27695|23|2|1609|25|38.00|1.52|39.52|99.74|52.00|29.64|0.66|7.70|153.26| +2451983|70309|1255|78858|1465011|5211|5364|78858|1465011|5211|5364|19|8|1609|22|2435.84|170.50|2606.34|92.43|906.40|1607.65|198.76|629.43|1169.33| +2452662|36505|684|43331|1704463|2748|25480|43331|1704463|2748|25480|49|22|1611|23|198.95|0.00|198.95|78.16|51.75|1.98|13.78|183.19|129.91| +2451366|66365|5659|99197|1118954|1464|27174|99197|1118954|1464|27174|28|16|1612|56|5479.04|0.00|5479.04|83.81|490.56|2027.24|3072.10|379.70|574.37| +2451462|71727|8557|37325|722723|568|23966|37325|722723|568|23966|55|15|1612|43|1253.45|100.27|1353.72|52.83|0.00|889.94|199.93|163.58|153.10| +2452735|79168|17767|22423|1883746|1422|16745|22423|1883746|1422|16745|45|12|1613|26|1279.46|102.35|1381.81|28.21|316.16|626.93|554.65|97.88|446.72| +2452795|44617|9846|98732|690069|6558|22037|98732|690069|6558|22037|21|8|1613|14|539.56|37.76|577.32|82.44|168.84|307.54|106.72|125.30|289.04| +2451476|43101|15704|47637|678703|3016|82|47637|678703|3016|82|22|19|1614|17|2396.15|167.73|2563.88|56.83|1302.20|790.72|417.41|1188.02|1526.76| +2451452|41021|15200|42884|201971|5058|32545|42884|201971|5058|32545|52|29|1614|35|98.70|8.88|107.58|91.43|362.25|55.27|26.92|16.51|462.56| +2451301|49684|15734|28366|1030540|1514|40888|28366|1030540|1514|40888|20|2|1614|33|1961.52|19.61|1981.13|43.14|21.45|215.76|1571.18|174.58|84.20| +2451511|59209|2602|93617|194322|7027|18932|93617|194322|7027|18932|52|5|1616|21|401.94|0.00|401.94|65.17|125.58|152.73|59.81|189.40|190.75| +2451590|78738|1034|4313|644525|3937|12549|4313|644525|3937|12549|43|33|1616|11|310.86|15.54|326.40|62.37|285.56|118.12|119.49|73.25|363.47| +2451293|61229|13636|46522|566887|4852|31617|46522|566887|4852|31617|2|33|1616|35|1190.70|11.90|1202.60|51.26|357.00|583.44|534.38|72.88|420.16| +2451363|40345|5954|57759|1776540|1260|38646|57759|1776540|1260|38646|31|28|1617|3|41.70|2.08|43.78|64.72|45.87|32.52|0.55|8.63|112.67| +2452149|46681|9887|25063|1848141|372|18203|25063|1848141|372|18203|20|5|1618|19|3398.15|101.94|3500.09|81.74|1634.57|1835.00|375.15|1188.00|1818.25| +2451977|10027|6869|94343|1858791|5639|22487|94343|1858791|5639|22487|49|20|1618|50|844.50|0.00|844.50|63.55|63.00|67.56|155.38|621.56|126.55| +|53402|8875|36357|1529565|||36357||||56||1618|10|||135.78||||||| +2451197|65675|15386|65504|1740852|1447|21595|65504|1740852|1447|21595|8|12|1619|1|45.79|2.28|48.07|44.59|11.28|13.27|29.59|2.93|58.15| +2451145|49595|16292|56372|20720|621|25369|56372|20720|621|25369|55|25|1619|46|4607.36|414.66|5022.02|6.13|4006.14|92.14|3205.80|1309.42|4426.93| +2451049|37082|10417|43970|727551|285|11066|43970|727551|285|11066|43|10|1619|2|27.04|0.27|27.31|70.73|41.16|0.27|23.02|3.75|112.16| +2451133|59150|6310|37328|166865|2002|23139|37328|166865|2002|23139|7|23|1619|7|16.87|0.84|17.71|55.91|73.29|10.45|1.79|4.63|130.04| +2451924|26834|8155|25713|603387|4614|35041|25713|603387|4614|35041|31|29|1620|11|142.78|4.28|147.06|46.25|102.63|81.38|0.61|60.79|153.16| +2452910|41451|12393|76597|1423976|5954|23162|76597|1423976|5954|23162|18|22|1621|84|3514.56|246.01|3760.57|89.23|2913.96|2425.04|555.65|533.87|3249.20| +|35063|17049|72037|||4724|72037|||4724|37|25|1621||772.34|||11.44|0.00|84.95|494.92|192.47|80.95| +2451657|2005|13027|50868|1148629|3543|20504|50868|1148629|3543|20504|4|14|1622|16|154.40|10.80|165.20|5.40|110.24|6.17|48.91|99.32|126.44| +2451484|32703|13084|43084|1086974|464|7907|43084|1086974|464|7907|38|27|1623|12|851.52|59.60|911.12|39.69|510.84|195.84|419.63|236.05|610.13| +||14893|44914|1777825||23122||1777825|6766|23122|43|6|1625|||||74.70|||86.99||1177.80| +2451762|35979|12107|17034|832594|4154|4114|17034|832594|4154|4114|23|29|1626|16|1735.20|17.35|1752.55|44.32|69.28|607.32|902.30|225.58|130.95| +2452701|32095|5833|6116|87760|2651|9895|6116|87760|2651|9895|49|15|1627|43|4636.69|370.93|5007.62|49.82|762.82|2040.14|1843.55|753.00|1183.57| +2451713|4387|6526|81030|697344|4058|7572|81030|697344|4058|7572|8|10|1628|4|6.48|0.06|6.54|45.12|195.08|1.81|3.68|0.99|240.26| +2451334|70471|4316|46249|1339324|5199|721|46249|1339324|5199|721|2|25|1631|44|496.76|19.87|516.63|89.50|16.72|24.83|226.52|245.41|126.09| +2451338|21157|13177|31296|37713|1283|23700|31296|37713|1283|23700|46|12|1632|75|6353.25|508.26|6861.51|64.22|3291.75|5717.92|266.83|368.50|3864.23| +2451194|63565|10633|3307|60355|1050|22545|3307|60355|1050|22545|22|28|1632|5|377.25|11.31|388.56|25.95|232.60|49.04|118.15|210.06|269.86| +2451215|24924|4934|87758|1468350|3695|7887|87758|1468350|3695|7887|49|28|1632|48|65.76|3.28|69.04|73.67|0.48|50.63|6.95|8.18|77.43| +2451670|42382|12877|25241|801387|6686|48385|25241|801387|6686|48385|8|8|1633|59|9067.71|181.35|9249.06|71.73|4051.53|5621.98|1998.52|1447.21|4304.61| +2452462|38290|14603|37564|1031457|2597|28090|37564|1031457|2597|28090|49|29|1634|9|1469.79|29.39|1499.18|46.31|697.05|499.72|252.21|717.86|772.75| +2452204|60366|5173|44707|100564|6021|678|44707|100564|6021|678|23|12|1635|5|79.80|2.39|82.19|68.57|124.20|71.82|2.95|5.03|195.16| +2452006|73187|12649|56494|834290|2039|40188|56494|834290|2039|40188|25|31|1636|74|6606.72|396.40|7003.12|54.19|1390.46|4955.04|429.43|1222.25|1841.05| +2451958|8802|7501|3565|428967|3616|44218|3565|428967|3616|44218|43|4|1637|3|367.11|29.36|396.47|9.89|61.17|18.35|48.82|299.94|100.42| +2451594|42541|14908|51798|559892|2833|45727|51798|559892|2833|45727|13|13|1638|54|4752.00|0.00|4752.00|23.27|2851.20|3991.68|402.96|357.36|2874.47| +2452002|38270|1694|378|913985|6792|21152|378|913985|6792|21152|26|19|1642|28|1095.08|65.70|1160.78|30.75|285.60|558.49|273.66|262.93|382.05| +2451041|63256|3772|52979|293688|6523|25875|52979|293688|6523|25875|8|1|1643|32|794.24|7.94|802.18|42.96|624.96|381.23|272.58|140.43|675.86| +2452707|49240|5199|7260|1105633|3653|18873|7260|1105633|3653|18873|13|12|1646|6|19.20|0.76|19.96|20.70|7.68|11.52|7.60|0.08|29.14| +2451415|76870|14338|29867|314711|4355|15952|29867|314711|4355|15952|16|34|1647|2|99.94|1.99|101.93|70.94|37.00|0.00|64.96|34.98|109.93| +2452667|79730|313|88562|92583|2204|48578|88562|92583|2204|48578|9|12|1648|12|32.28|2.25|34.53|91.00|199.20|19.04|6.09|7.15|292.45| +2452444|75444|9054|41145|310685|3425|18612|41145|310685|3425|18612|3|17|1649|3|174.42|8.72|183.14|79.63|138.21|113.37|21.97|39.08|226.56| +2452586|32516|1116|78678|292664|21|7987|78678|292664|21|7987|39|8|1649|95|16163.30|1454.69|17617.99|56.25|7163.00|10021.24|6142.06|0.00|8673.94| +2452647|41328|6330|46746|312529|1865|35601|46746|312529|1865|35601|37|4|1650|8|110.72|2.21|112.93|94.30|87.84|99.64|6.31|4.77|184.35| +2452624|25782|840|39706|1603233|3560|43487|39706|1603233|3560|43487|6|13|1650|50|631.00|31.55|662.55|80.62|82.00|536.35|88.97|5.68|194.17| +2452594|75105|15408|38685|487744|5956|17015|38685|487744|5956|17015|27|15|1650|64|1766.40|35.32|1801.72|99.26|539.52|1342.46|122.94|301.00|674.10| +2452649|50559|2485|20731|1315685|5392|3146|20731|1315685|5392|3146|42|29|1650|39|163.41|11.43|174.84|71.93|961.74|138.89|19.86|4.66|1045.10| +2451296|54693|12176|2860|1340658|2456|48654|2860|1340658|2456|48654|4|22|1651|48|4927.68|98.55|5026.23|12.71|5576.16|1281.19|1021.01|2625.48|5687.42| +2451231|60316|13366|9844|35130|2463|33796|9844|35130|2463|33796|34|22|1651|49|323.89|6.47|330.36|97.55|755.58|129.55|21.37|172.97|859.60| +2451611|67332|8950|38629|835068|6707|41146|38629|835068|6707|41146|43|13|1652|14|425.46|17.01|442.47|57.80|451.92|323.34|46.97|55.15|526.73| +2451587|71500|256|50193|983584|6575|3966|50193|983584|6575|3966|34|7|1652|53|3463.02|34.63|3497.65|7.81|2803.70|484.82|2620.81|357.39|2846.14| +2452488|76032|1788|46384|234893|5802|19511|46384|234893|5802|19511|25|35|1653|26|865.54|60.58|926.12|27.70|932.10|354.87|469.81|40.86|1020.38| +2452568|19497|3993|47655|1201907|6112|26764|47655|1201907|6112|26764|13|15|1653|9|355.32|17.76|373.08|58.64|45.81|99.48|63.96|191.88|122.21| +2452407|71420|1451|14602|983104|3594|30858|14602|983104|3594|30858|3|24|1655|62|403.62|20.18|423.80|59.60|2364.68|68.61|50.25|284.76|2444.46| +2451756|76138|15347|33022|1305365|5355|24098|33022|1305365|5355|24098|41|3|1656|93|2316.63|138.99|2455.62|49.22|3972.03|2131.29|137.15|48.19|4160.24| +2451848|66410|7273|27484|1557658|632|38806|27484|1557658|632|38806|25|23|1656|16|990.72|9.90|1000.62|50.52|177.28|59.44|363.19|568.09|237.70| +2452771|70246|13350|79680|77909|5385|47261|79680|77909|5385|47261|33|16|1657|7|1332.52|79.95|1412.47|38.63|503.37|373.10|700.37|259.05|621.95| +2452605|28722|17904|36812|565905|5751|33597|36812|565905|5751|33597|1|29|1657|6|745.80|37.29|783.09|10.08|23.28|484.77|46.98|214.05|70.65| +2452857|76998|757|72489|780583|4891|24948|72489|780583|4891|24948|54|11|1658|14|1407.00|14.07|1421.07|50.48|14.70|998.97|248.89|159.14|79.25| +2451189|27059|5266|84579|74009|5839|21269|84579|74009|5839|21269|37|24|1659|5|146.45|2.92|149.37|20.07|59.65|10.25|34.05|102.15|82.64| +2451159|18978|10339|||79|||||18998|25||1659|||95.38|1685.16|20.79|2517.78|620.01|843.69|126.08|| +2451817|48781|2369|88531|597108|1238|5415|88531|597108|1238|5415|29|1|1660|6|81.24|4.06|85.30|29.96|10.86|45.49|29.67|6.08|44.88| +2451820|77250|15824|70607|193340|3273|14453|70607|193340|3273|14453|7|8|1660|22|469.04|14.07|483.11|86.74|95.26|229.82|193.76|45.46|196.07| +2451116|51533|5942|16103|772720|4984|9648|16103|772720|4984|9648|34|28|1661|35|1518.65|15.18|1533.83|33.22|249.55|485.96|433.72|598.97|297.95| +2451091|7974|12043|83448|455653|1105|11357|83448|455653|1105|11357|32|27|1661|50|214.50|17.16|231.66|30.42|150.50|188.76|1.54|24.20|198.08| +2452601|79711|3967|22925|1346743|4608|44026|22925|1346743|4608|44026|3|18|1662|1|217.00|17.36|234.36|41.85|51.45|56.42|141.31|19.27|110.66| +2452631|4628|5035|60969|782195|4574|14153|60969|782195|4574|14153|13|11|1662|36|2469.96|49.39|2519.35|16.83|2590.56|889.18|822.00|758.78|2656.78| +2452442|68980|2257|54605|1554236|5867|43545|54605|1554236|5867|43545|47|18|1664|59|2875.07|28.75|2903.82|93.73|252.52|1782.54|371.46|721.07|375.00| +2452172|22429|7909|2226|688018|5010|28402|2226|688018|5010|28402|5|6|1664|48|813.12|24.39|837.51|20.84|750.24|390.29|152.21|270.62|795.47| +2452714|18879|4200|81064|635894|324|30754|81064|635894|324|30754|12|21|1665|12|142.32|9.96|152.28|64.77|36.36|78.27|28.18|35.87|111.09| +2452187|62692|1801|44357|1414594|831|43115|44357|1414594|831|43115|35|7|1666|1|22.24|0.44|22.68|3.07|35.79|14.45|1.55|6.24|39.30| +|8445|15523|28574|||23261||775203|||13||1669|59|3839.13||3954.30|1.36||||6.15|1436.36| +2452794|42059|1425|84168|918434|3375|10508|84168|918434|3375|10508|30|32|1669|4|51.44|1.54|52.98|19.38|61.76|47.83|0.32|3.29|82.68| +2452660|33271|16501|99646|1235814|2829|47797|99646|1235814|2829|47797|25|9|1669|12|2071.68|165.73|2237.41|59.50|391.32|41.43|1502.38|527.87|616.55| +2452796|69587|13428|51612|219277|2589|34654|51612|219277|2589|34654|45|18|1669|51|525.30|0.00|525.30|2.06|417.69|178.60|152.54|194.16|419.75| +2452808|66055|3867|93886|1588244|3849|16607|93886|1588244|3849|16607|25|14|1670|20|497.00|9.94|506.94|92.37|359.00|422.45|14.16|60.39|461.31| +2452730|38758|3877|76481|518170|5807|31548|76481|518170|5807|31548|48|31|1671|14|1600.76|128.06|1728.82|72.39|469.56|1600.76|0.00|0.00|670.01| +2452620|27548|1134|15597|1044487|6177|27467|15597|1044487|6177|27467|54|13|1671|23|511.52|35.80|547.32|68.12|369.38|501.28|5.52|4.72|473.30| +2452573|4869|4662|22046|279185|1924|19063|22046|279185|1924|19063|33|16|1671|1|90.77|5.44|96.21|55.99|63.03|28.13|13.15|49.49|124.46| +2452535|16321|3333|12232|1859931|4877|576|12232|1859931|4877|576|7|30|1672|16|217.76|13.06|230.82|74.82|172.48|17.42|142.24|58.10|260.36| +2452310|16257|15031|11382|228108|2787|45113|11382|228108|2787|45113|59|16|1672|2|44.72|3.13|47.85|66.10|4.74|30.40|12.17|2.15|73.97| +2452756|3382|16938|63818|830805|992|25579|63818|830805|992|25579|7|24|1673|11|190.08|15.20|205.28|16.11|56.98|28.51|74.32|87.25|88.29| +2452875|66581|5751|47492|198863|1434|20005|47492|198863|1434|20005|39|27|1673|21|508.20|25.41|533.61|73.76|33.81|10.16|249.02|249.02|132.98| +2452752|66469|3117|10804|560337|5070|39403|10804|560337|5070|39403|9|11|1675|30|952.50|19.05|971.55|1.07|72.30|676.27|85.63|190.60|92.42| +2452755|45203|6679|88821|1697559|5022|22187|88821|1697559|5022|22187|18|16|1676|13|173.94|6.95|180.89|50.23|181.87|55.66|36.66|81.62|239.05| +2451858|53317|12367|35985|258089|3393|20744|35985|258089|3393|20744|25|35|1677|26|390.78|11.72|402.50|81.65|128.44|230.56|64.08|96.14|221.81| +2452411|73721|15857|70043|596435|1580|13972|70043|596435|1580|13972|13|5|1678|31|498.48|44.86|543.34|7.91|1019.59|34.89|148.34|315.25|1072.36| +2452361|77099|4725|60213|971199|2985||60213||2985||30||1679|8||0.00|||||||128.80| +2451569|69576|4504|52461|1660760|4259|11358|52461|1660760|4259|11358|10|25|1680|14|713.44|21.40|734.84|35.06|16.38|242.56|277.81|193.07|72.84| +2452607|54220|5928|99705|438592|5126|413|99705|438592|5126|413|25|34|1681|9|1503.99|60.15|1564.14|52.89|1030.14|992.63|25.56|485.80|1143.18| +|41221|12973|90533||||90533||6034|14408|7||1682|37|8220.66||8713.89|49.27||4274.74||157.84|| +2451284|60520|2191|14543|1142230|5470|7110|14543|1142230|5470|7110|49|1|1683|27|4702.86|141.08|4843.94|92.94|1518.48|3809.31|62.54|831.01|1752.50| +2451565|46441|8305|49056|1828947|391|40282|49056|1828947|391|40282|14|16|1684|9|871.47|34.85|906.32|81.08|38.70|732.03|101.79|37.65|154.63| +2451562|71699|13963|9046|553156|6866|3776|9046|553156|6866|3776|8|14|1686|43|1749.67|17.49|1767.16|98.16|131.15|17.49|121.25|1610.93|246.80| +2451608|18311|6368|3794|1468028|262|18896|3794|1468028|262|18896|22|30|1686|1|42.65|3.83|46.48|43.96|19.26|37.53|3.43|1.69|67.05| +2452634|32983|17364|26899|906374|249|38571|26899|906374|249|38571|39|30|1687|11|300.96|15.04|316.00|53.98|71.61|18.05|192.37|90.54|140.63| +2451590|82360|7222|82485|639101|6917|3180|82485|639101|6917|3180|55|23|1688|82|9404.58|846.41|10250.99|31.39|887.24|3573.74|524.77|5306.07|1765.04| +2451463|14987|121|20148|1110355|6162|24769|20148|1110355|6162|24769|4|16|1689|21|1509.06|135.81|1644.87|56.56|528.15|1388.33|74.85|45.88|720.52| +2451496|47353|350|17819|17682|745|35408|17819|17682|745|35408|32|14|1689|1|21.31|0.00|21.31|10.23|37.30|7.67|4.91|8.73|47.53| +2451488|70389|13532|62216|901698|1180|19062|62216|901698|1180|19062|49|8|1689|2|107.44|3.22|110.66|21.23|42.96|1.07|43.61|62.76|67.41| +2452771|18656|1075|85004|1341528|2499|33870|85004|1341528|2499|33870|43|12|1690|85|13967.20|838.03|14805.23|93.21|3910.00|3491.80|2304.58|8170.82|4841.24| +2452036|78734|3533|8657|1339673|2689|40905|8657|1339673|2689|40905|20|16|1691|60|3907.20|195.36|4102.56|62.78|2629.80|3438.33|337.58|131.29|2887.94| +2452023|60532|6941|51292|1127029|5519|3656|51292|1127029|5519|3656|26|7|1691|3|120.90|1.20|122.10|34.70|56.61|70.12|20.81|29.97|92.51| +2451485|57812|4336|3230|17730|2277|1624|3230|17730|2277|1624|7|17|1692|23|201.71|2.01|203.72|60.30|512.21|199.69|0.80|1.22|574.52| +2452007|36085|14767|22917|1824863|3962|26261|22917|1824863|3962|26261|8|4|1693|1|125.28|2.50|127.78|6.86|50.11|111.49|6.89|6.90|59.47| +2451584|30205|4304|76648|1176648|3048|26648|76648|1176648|3048|26648|26|13|1694|12|29.88|2.39|32.27|41.97|4.56|24.80|0.86|4.22|48.92| +2451648|12991|9775|14562|1653829|3496|21963|14562|1653829|3496|21963|13|35|1694|25|170.50|0.00|170.50|46.81|36.00|47.74|94.52|28.24|82.81| +2451655||332|49169||6652|26817||1362087||26817||23|1694|19|||436.77||||156.87|191.75|756.88| +2452134|59343|1415|51032|290346|5213|11327|51032|290346|5213|11327|25|30|1695|14|1566.74|0.00|1566.74|27.44|898.94|62.66|1413.83|90.25|926.38| +2451954|62114|14123|98539|209494|6849|604|98539|209494|6849|604|59|11|1695|29|3748.25|37.48|3785.73|18.74|0.00|1724.19|1072.75|951.31|56.22| +2452026|18696|337|58494|1350609|3171|28133|58494|1350609|3171|28133|32|8|1695|1|207.16|4.14|211.30|28.88|22.78|205.08|1.47|0.61|55.80| +2451255|65839|12056|48157|1141925|6540|45955|48157|1141925|6540|45955|37|1|1696|62|2548.20|76.44|2624.64|97.57|3320.72|1223.13|371.01|954.06|3494.73| +2452556|58210|5511|34982|1667261|2740|23019|34982|1667261|2740|23019|45|13|1698|32|1453.12|14.53|1467.65|57.77|1695.36|1351.40|6.10|95.62|1767.66| +2451184|17130|11018|2816|1219828|5193|17005|2816|1219828|5193|17005|10|34|1699|23|659.18|0.00|659.18|45.00|32.89|39.55|408.95|210.68|77.89| +2451191|35509|9554|31384|241609|6234|44859|31384|241609|6234|44859|55|9|1699|13|497.12|44.74|541.86|22.90|70.98|129.25|18.39|349.48|138.62| +2451308|68216|4052|86858|53346|1434|33922|86858|53346|1434|33922|40|4|1702|2|20.42|0.61|21.03|71.19|20.86|8.98|11.09|0.35|92.66| +2451951|10676|1028|58103|688246|3636|36579|58103|688246|3636|36579|23|1|1704|3|334.56|13.38|347.94|19.74|197.67|200.73|96.35|37.48|230.79| +2451976|1847|1105|90110|1186342|2574|5206|90110|1186342|2574|5206|19|6|1706|17|378.42|11.35|389.77|2.08|378.42|234.62|60.39|83.41|391.85| +2452511|30639|11671|52868|1454532|4196|12660|52868|1454532|4196|12660|53|23|1708|7|16.66|0.99|17.65|27.99|34.16|14.49|2.08|0.09|63.14| +2452330|74682|17073|60505|||37559||||37559|7||1708|18|786.78|||11.72||723.83|39.02||319.15| +2452252|55808|14229|53539|725318|2250|15229|53539|725318|2250|15229|51|29|1708|2|120.74|6.03|126.77|15.14|136.84|47.08|30.20|43.46|158.01| +2452014|44921|3668|56983|460498|3213|15528|56983|460498|3213|15528|50|3|1710|29|206.19|2.06|208.25|70.54|68.73|65.98|88.33|51.88|141.33| +2451825|21945|9719|52442|1281627|5212|23597|52442|1281627|5212|23597|49|17|1710|27|421.47|29.50|450.97|65.60|210.60|332.96|38.94|49.57|305.70| +2452314|83350|4525|1805|1768202|3846|26243|1805|1768202|3846|26243|51|26|1711|12|669.48|46.86|716.34|10.43|19.92|495.41|156.66|17.41|77.21| +2452226|68706|8637|61910|633436|962|9288|61910|633436|962|9288|59|29|1711|12|410.28|0.00|410.28|18.09|102.48|82.05|164.11|164.12|120.57| +2451529|4002|11672|95044|451731|4418|40305|95044|451731|4418|40305|50|3|1712|43|469.13|9.38|478.51|67.15|656.61|365.92|4.12|99.09|733.14| +2451507|72570|11888|48403|227041|2032|30623|48403|227041|2032|30623|38|33|1713|6|4.98|0.24|5.22|82.15|204.42|2.68|1.19|1.11|286.81| +2452516|20730|3675|32146|886866|2833|49953|32146|886866|2833|49953|41|4|1714|10|253.40|7.60|261.00|21.04|760.40|43.07|143.02|67.31|789.04| +2452530|56301|8333|29599|1012430|2014|4045|29599|1012430|2014|4045|57|15|1714|6|564.48|28.22|592.70|63.97|40.32|107.25|411.50|45.73|132.51| +2452379|70810|14417|69378|1806571|564|32557|69378|1806571|564|32557|17|12|1714|5|305.75|15.28|321.03|21.03|261.00|73.38|132.45|99.92|297.31| +2452099|37938|15344|78582|380283|831|15332|78582|380283|831|15332|35|9|1715|24|1259.76|25.19|1284.95|5.75|532.32|617.28|115.64|526.84|563.26| +2452551|61892|11405|91258|723789|320|8451|91258|723789|320|8451|13|15|1716|46|416.30|8.32|424.62|7.21|425.96|391.32|6.49|18.49|441.49| +2452355|26340|15583|55707|504866|3978|45537|55707|504866|3978|45537|47|2|1716|31|512.43|40.99|553.42|32.67|256.06|102.48|229.57|180.38|329.72| +2452458|63315|16999|49284|1743214|1944|40227|49284|1743214|1944|40227|53|25|1716|30|2287.20|137.23|2424.43|39.20|1019.10|663.28|178.63|1445.29|1195.53| +2452440|47380|1117|48641|1888928|1215|31102|48641|1888928|1215|31102|49|32|1716|44|2198.68|109.93|2308.61|15.38|631.84|2088.74|48.37|61.57|757.15| +2451635|67826|11218|31808|107180|1752|23924|31808|107180|1752|23924|26|23|1717|20|2738.00|191.66|2929.66|1.97|199.00|903.54|825.50|1008.96|392.63| +2451564|18800|14146|28131|706877|470|33216|28131|706877|470|33216|56|35|1717|39|181.74|0.00|181.74|22.18|493.35|56.33|101.58|23.83|515.53| +2451793|76372|4405|30747|403323|3099|41275|30747|403323|3099|41275|25|4|1717|20|1487.60|119.00|1606.60|97.14|285.40|1309.08|37.48|141.04|501.54| +2452064|38267|3745|46448|907422|5996|15370|46448|907422|5996|15370|43|14|1719|2|27.32|0.54|27.86|88.88|12.90|1.36|6.23|19.73|102.32| +2452172|2891|7597|39637|1518731|2178|26872|39637|1518731|2178|26872|13|15|1719|14|2343.46|117.17|2460.63|59.05|236.88|234.34|1729.47|379.65|413.10| +2452830|12513|14353|79386|1275411|3436|31414|79386|1275411|3436|31414|15|20|1720|28|4241.16|84.82|4325.98|16.27|1781.08|1272.34|2850.06|118.76|1882.17| +2452388|79588|7975|23041|770596|104|47659|23041|770596|104|47659|3|10|1721|21|204.75|14.33|219.08|32.14|173.25|2.04|70.94|131.77|219.72| +2452415|58885|11633|69566|838795|3224|26453|69566|838795|3224|26453|59|4|1721|7|672.77|13.45|686.22|11.73|390.67|343.11|260.43|69.23|415.85| +2452262|37126|17285|38910|460999|1488|46024|38910|460999|1488|46024|13|15|1721|26|857.48|77.17|934.65|70.26|1286.22|85.74|277.82|493.92|1433.65| +2452443|74117|14455|37060|1596157|2101|27198|37060|1596157|2101|27198|60|12|1722|23|1858.86|111.53|1970.39|60.40|0.00|371.77|1070.70|416.39|171.93| +2452543|59909|943|27718|1093160|202|20491|27718|1093160|202|20491|30|10|1722|15|1726.65|0.00|1726.65|11.68|479.55|1692.11|1.03|33.51|491.23| +2451974|25438|1645|19918|1810882|1399|42316|19918|1810882|1399|42316|13|25|1724|80|1016.80|61.00|1077.80|38.16|489.60|376.21|486.84|153.75|588.76| +2452035||5965|54684|||45089|54684|1820168||45089|2||1724|33|1197.90||1281.75|29.16|446.49||491.61||| +2452811|30094|7659|5753|414149|2898|3694|5753|414149|2898|3694|43|17|1725|36|1429.56|71.47|1501.03|20.10|0.00|1329.49|68.04|32.03|91.57| +2452523|6218|2694|74447|787469|844|14666|74447|787469|844|14666|39|6|1726|8|273.76|10.95|284.71|98.98|354.88|210.79|33.37|29.60|464.81| +2451221|82612|6524|48407|445302|150|37445|48407|445302|150|37445|25|11|1728|18|2399.76|95.99|2495.75|23.24|685.62|1703.82|334.05|361.89|804.85| +2451976|44951|1937|93525|1721417|5662|45154|93525|1721417|5662|45154|25|27|1729|7|71.12|0.71|71.83|15.56|569.38|63.29|7.67|0.16|585.65| +2452136|69066|8216|20348|1099991|5587|19630|20348|1099991|5587|19630|32|12|1729|11|692.89|0.00|692.89|21.59|377.08|637.45|29.38|26.06|398.67| +2451575|5439|10036|6433|391036|4086|6289|6433|391036|4086|6289|43|15|1730|50|1245.50|99.64|1345.14|95.67|872.00|909.21|272.39|63.90|1067.31| +2452683|75391|16980|67746|1863533|4120|7507|67746|1863533|4120|7507|12|26|1732|34|1499.06|59.96|1559.02|61.31|386.58|1034.35|320.64|144.07|507.85| +2452688|82433|15648|60419|1727996||27903|||6773|27903|39||1732|||3.00|40.56||19.60|7.13||25.26|76.30| +2451172||1897|73854|||44837|73854||4525|44837||26|1733|36|43.56|1.30||33.59|118.08||14.46|2.97|| +2451234|48331|1465|27709|1060049|1189|24282|27709|1060049|1189|24282|58|23|1733|40|383.60|0.00|383.60|75.57|383.60|253.17|125.21|5.22|459.17| +2451173|37766|5803|53136|1425098|613|42528|53136|1425098|613|42528|1|25|1734|3|55.35|4.98|60.33|42.77|25.41|36.53|16.18|2.64|73.16| +2451363|50971|3094|59528|190771|6014|33657|59528|190771|6014|33657|1|25|1734|23|3974.63|357.71|4332.34|32.55|1048.11|3497.67|271.86|205.10|1438.37| +2451697|11888|949|64038|1307127|1416|23305|64038|1307127|1416|23305|50|1|1735|12|306.12|0.00|306.12|59.22|126.00|183.67|79.59|42.86|185.22| +2452524|43401|5317|80966|771053|4693|28333|80966|771053|4693|28333|27|9|1736|9|383.85|7.67|391.52|29.11|437.13|134.34|94.81|154.70|473.91| +2451516|34664|10118|21935|809066|5444|38622|21935|809066|5444|38622|13|19|1737|16|638.56|44.69|683.25|93.83|812.64|517.23|29.11|92.22|951.16| +2451706|45353|2924|54089|1153857|6425|39440|54089|1153857|6425|39440|28|7|1738|1|2.88|0.11|2.99|5.70|2.72|2.18|0.18|0.52|8.53| +2451790|74627|16406|94390|152283|6529|48422|94390|152283|6529|48422|11|6|1740|4|378.88|34.09|412.97|63.33|97.76|106.08|111.84|160.96|195.18| +2452290|66298|2755|30407|1645922|5437|18152|30407|1645922|5437|18152|47|5|1742|21|1743.00|34.86|1777.86|5.39|1479.87|34.86|290.38|1417.76|1520.12| +2452404|74491|2517|90753|1542378|1203|33628|90753|1542378|1203|33628|25|21|1742|6|163.38|3.26|166.64|53.17|88.80|98.02|30.06|35.30|145.23| +2452457|60308|10403|31144|606749|5554|31159|31144|606749|5554|31159|59|31|1742|17|776.90|31.07|807.97|82.35|141.27|411.75|332.28|32.87|254.69| +2452304|56113|1735|87357|50173|4989|25405|87357|50173|4989|25405|39|5|1742|20|2116.00|190.44|2306.44|16.54|1014.80|1502.36|245.45|368.19|1221.78| +2452720|58862|10242|86731|1131498|265|28585|86731|1131498|265|28585|25|20|1743|16|2498.72|174.91|2673.63|84.36|99.84|224.88|773.10|1500.74|359.11| +2451249|19324|13390|653|459256|5506|31309|653|459256|5506|31309|13|3|1744|5|179.35|5.38|184.73|30.16|52.55|130.92|39.71|8.72|88.09| +2451688|69042|11366|29180|795702|3824|25946|29180|795702|3824|25946|32|1|1748|24|528.96|5.28|534.24|61.42|73.68|491.93|10.36|26.67|140.38| +2451160|18640|14080|64282|1151789|2496|48403|64282|1151789|2496|48403|44|18|1749|33|648.78|19.46|668.24|48.41|190.74|103.80|506.83|38.15|258.61| +2452679|76539|10482|67264|982425|1939|29500|67264|982425|1939|29500|33|5|1750|9|874.89|69.99|944.88|71.51|157.95|332.45|173.58|368.86|299.45| +2451414|12658|12506|84920|599410|3947|22837|84920|599410|3947|22837|32|21|1751|8|457.68|9.15|466.83|46.62|94.88|306.64|33.22|117.82|150.65| +2451400|43304|10520|67321|888944|7|2285|67321|888944|7|2285|26|5|1751|73|807.38|0.00|807.38|51.28|330.69|662.05|1.45|143.88|381.97| +2451552|51341|10718|75358|1572173|3035|46250|75358|1572173|3035|46250|56|16|1751|59|1381.78|96.72|1478.50|72.24|1054.92|1174.51|35.23|172.04|1223.88| +2451221|37720|8456|92363|462853|5343|8233|92363|462853|5343|8233|19|28|1752|2|157.60|0.00|157.60|85.94|76.08|146.56|8.83|2.21|162.02| +2451312|65456|7388|42348|273552|6356|25207|42348|273552|6356|25207|20|17|1752|52|2977.52|59.55|3037.07|81.52|1086.28|1191.00|625.28|1161.24|1227.35| +2451680|69926|6889|51396|1707182|215|48001|51396|1707182|215|48001|32|13|1755|6|1016.34|0.00|1016.34|44.02|524.94|630.13|115.86|270.35|568.96| +2451864|37535|3692|77646|643117|3435|12953|77646|643117|3435|12953|20|7|1755|34|1405.90|70.29|1476.19|34.93|731.68|154.64|1113.62|137.64|836.90| +2451585|40613|8150|1513|1213176|4392|46855|1513|1213176|4392|46855|38|9|1756|8|1084.56|32.53|1117.09|92.35|0.00|86.76|369.18|628.62|124.88| +2451535|49410|2930|6812|1405084|7003|7675|6812|1405084|7003|7675|32|1|1756|1|10.14|0.10|10.24|64.17|20.29|6.89|2.47|0.78|84.56| +2451995|14146|10933|55872|343191|3310|7029|55872|343191|3310|7029|17|18|1757|7|274.54|19.21|293.75|74.19|411.81|79.61|3.89|191.04|505.21| +2452049|56878|9919|33250|619770|7090|16410|33250|619770|7090|16410|25|27|1757|12|198.48|9.92|208.40|32.16|992.40|160.76|33.57|4.15|1034.48| +2451929||5483||||||1253114|2638|42162||17|1757|||||||3143.77||94.66|| +2452528|55260|7728|35705|935726|6147|3415|35705|935726|6147|3415|48|16|1758|7|43.19|3.88|47.07|40.43|159.88|0.43|5.98|36.78|204.19| +2452649|3956|7813|56242|493599|909|44266|56242|493599|909|44266|33|15|1758|10|220.80|11.04|231.84|37.93|64.20|134.68|6.02|80.10|113.17| +|26470|11811|32917|361324||42472||||42472|31|4|1760|11|246.51|14.79|||49.28|189.81||44.23|| +2452456|79625|4665|47305|1485854|6450|40646|47305|1485854|6450|40646|31|33|1760|5|9.25|0.18|9.43|68.49|148.00|5.08|3.29|0.88|216.67| +2452311|55204|5083|11117|58028|3339|46650|11117|58028|3339|46650|3|19|1763|75|8994.75|809.52|9804.27|26.49|5053.50|7825.43|526.19|643.13|5889.51| +2452303|82747|5419|76340|48708|6823|38093|76340|48708|6823|38093|37|6|1763|15|16.35|1.47|17.82|72.72|2.10|1.47|9.67|5.21|76.29| +2452751||7215|89859|242198|6537|||242198|6537|31676|13||1764|20|640.40||640.40||||26.25||166.69| +2451668|39103|8803|55691|1820697|103|23909|55691|1820697|103|23909|58|33|1766|4|22.60|0.00|22.60|2.31|45.20|8.58|7.29|6.73|47.51| +2452639|32972|2040|12124|1868698|1272|20999|12124|1868698|1272|20999|6|26|1768|3|51.90|0.00|51.90|89.54|177.93|43.07|3.62|5.21|267.47| +2452661|56606|13386|61788|775013|3438|263|61788|775013|3438|263|36|3|1768|61|1369.45|68.47|1437.92|72.29|1712.27|205.41|209.52|954.52|1853.03| +||6979|21852||||||||49|25|1769|1|||61.33||13.64||||| +2451220|75142|616|1534|810885|5036|46387|1534|810885|5036|46387|4|17|1769|30|1076.70|64.60|1141.30|47.22|333.00|678.32|107.56|290.82|444.82| +2452453|17088|16921|5758|854345|6132|12272|5758|854345|6132|12272|51|25|1770|9|796.86|0.00|796.86|73.09|259.38|207.18|88.45|501.23|332.47| +2451407|56493|17506|38759|864203|6447|29844|38759|864203|6447|29844|19|8|1771|17|167.45|15.07|182.52|31.74|109.31|83.72|53.58|30.15|156.12| +2452176|47647|3709|22262|769116|6017|16918|22262|769116|6017|16918|55|23|1772|7|27.79|0.27|28.06|33.24|7.14|5.28|18.90|3.61|40.65| +2452339|59178|10833|17959|126989|5619|4649|17959|126989|5619|4649|31|34|1772|4|143.72|0.00|143.72|96.86|14.60|74.73|62.09|6.90|111.46| +2452815|65279|9691|75538|1413693|201|29156|75538|1413693|201|29156|9|12|1775|45|1082.25|32.46|1114.71|77.48|836.10|584.41|169.26|328.58|946.04| +2452792|57965|1494|66886|1794693|6500|15860|66886|1794693|6500|15860|48|35|1775|1|128.92|7.73|136.65|28.75|17.98|91.53|13.83|23.56|54.46| +2452746|38746|17925|5649|139295|2894|26140|5649|139295|2894|26140|25|27|1775|2|7.98|0.23|8.21|35.97|2.66|5.26|2.12|0.60|38.86| +|41917|16554|||5179||||||||1775|||3.00|303.32||||||| +2452399|13590|15093|61154|1298347|4340|47933|61154|1298347|4340|47933|55|2|1776|67|1216.05|109.44|1325.49|62.63|560.79|1045.80|28.94|141.31|732.86| +2452392|55482|9129|95801|824|1847|46517|95801|824|1847|46517|41|2|1776|24|1786.56|71.46|1858.02|57.56|370.32|196.52|556.51|1033.53|499.34| +2451581|49667|5716|||5398|||||42032||16|1779||813.20||821.33||378.10||123.44|144.92|| +2451732|43963|5026|52159|601679|4799|49519|52159|601679|4799|49519|32|11|1779|57|1601.70|112.11|1713.81|50.49|880.08|1425.51|56.38|119.81|1042.68| +2452339|18612|13033|96391|1470590|7189|26588|96391|1470590|7189|26588|9|12|1780|50|4968.50|397.48|5365.98|43.20|3168.00|2384.88|981.77|1601.85|3608.68| +2452733|38346|3375|60607|767000|6993|36139|60607|767000|6993|36139|42|30|1782|4|121.08|8.47|129.55|48.92|74.24|13.31|86.21|21.56|131.63| +2452790|19734|5119|51015|1154993|4924|38455|51015|1154993|4924|38455|3|12|1782|77|2701.93|189.13|2891.06|94.38|385.77|621.44|20.80|2059.69|669.28| +||1303|51321||3629|2136|||||1||1784|25|1901.25||||353.75||866.97|654.03|454.27| +2451455|45688|8074|11366|1757722|878|15681|11366|1757722|878|15681|7|4|1785|77|5366.90|268.34|5635.24|85.52|2423.19|1610.07|3381.14|375.69|2777.05| +2451578|46681|15154|55523|1429835|500|42012|55523|1429835|500|42012|44|5|1785|82|7183.20|0.00|7183.20|87.14|1699.04|6536.71|38.78|607.71|1786.18| +2451350|68315|2197|32458|1161377|3096|45615|32458|1161377|3096|45615|26|26|1786|10|1521.60|45.64|1567.24|31.09|164.90|1308.57|31.95|181.08|241.63| +2451315|74631|6076|56853|545788|4676|32411|56853|545788|4676|32411|50|25|1786|64|5399.68|377.97|5777.65|6.32|781.44|4481.73|238.66|679.29|1165.73| +2451247|31419|17864|81164||5430|6363||551697||6363|||1786|44||90.31|1595.55|10.53||105.36|909.92|489.96|| +2452298|24331|12523|40038|895397|4356|32562|40038|895397|4356|32562|49|12|1787|18|525.42|42.03|567.45|50.30|339.84|110.33|261.50|153.59|432.17| +2451650|42917|10717|60893|1152155|6970|12232|60893|1152155|6970|12232|2|18|1788|31|731.60|29.26|760.86|28.01|66.96|146.32|146.32|438.96|124.23| +2451475|67028|13460|14489|333079|6516|16306|14489|333079|6516|16306|28|28|1788|2|94.24|2.82|97.06|25.82|53.54|59.37|4.18|30.69|82.18| +2452812|43727|2659|72968|832275|3229|5383|72968|832275|3229|5383|12|32|1789|29|937.86|0.00|937.86|67.27|803.88|393.90|21.75|522.21|871.15| +2452698|57771|3471|5159|1192722|3838|15801|5159|1192722|3838|15801|43|26|1789|8|439.92|17.59|457.51|62.98|77.60|96.78|171.57|171.57|158.17| +2452882|37399|15468|96076|1177771|1419|43467|96076|1177771|1419|43467|6|29|1789|22|889.46|35.57|925.03|21.62|134.64|889.46|0.00|0.00|191.83| +2452090|26135|5081|22906|1559231|1603|47175|22906|1559231|1603|47175|26|3|1793|10|429.30|17.17|446.47|19.11|294.10|364.90|57.96|6.44|330.38| +2451944|42363|889|1218|1193401|5137|39720|1218|1193401|5137|39720|11|17|1795|28|117.04|5.85|122.89|71.62|153.16|10.53|78.81|27.70|230.63| +2451988|58368|13855|97710|1883099|935|36724|97710|1883099|935|36724|23|12|1795|2|13.46|0.40|13.86|35.49|19.24|9.82|1.09|2.55|55.13| +2452141|72539|11545|15192|423203|6061|10472|15192|423203|6061|10472|55|34|1795|67|1790.24|161.12|1951.36|30.61|1957.07|1002.53|567.15|220.56|2148.80| +2452901|3791|11772|88632|1715833|3034|47835|88632|1715833|3034|47835|30|28|1796|4|572.96|0.00|572.96|3.18|280.64|538.58|17.87|16.51|283.82| +2452725|69398|9787|22030|1296678|126|31974|22030|1296678|126|31974|37|7|1796|14|622.30|56.00|678.30|58.41|248.92|161.79|239.46|221.05|363.33| +2451289|55255|3577|58620|185370|4920|15270|58620|185370|4920|15270|4|28|1797|17|372.81|33.55|406.36|7.52|111.69|283.33|64.42|25.06|152.76| +2452485|32554|10675|62576|1018375|1374|36373|62576|1018375|1374|36373|23|32|1801|5|302.05|3.02|305.07|66.23|115.65|3.02|140.54|158.49|184.90| +2452291|36193|8117|16484|1574627|5123|17266|16484|1574627|5123|17266|19|19|1802|9|343.80|0.00|343.80|61.25|194.67|336.92|5.50|1.38|255.92| +2452357|31466|10959|97043|1430621|6246|18624|97043|1430621|6246|18624|1|20|1803|79|5195.04|467.55|5662.59|58.81|2650.45|4415.78|615.61|163.65|3176.81| +2452437|81942|14467|36786|462370|4754|47091|36786|462370|4754|47091|35|28|1803|3|62.37|3.74|66.11|16.51|8.91|10.60|10.87|40.90|29.16| +2452826|69685|5655|70943|1106990|4637|40684|70943|1106990|4637|40684|19|22|1804|75|10151.25|203.02|10354.27|44.56|1864.50|7714.95|219.26|2217.04|2112.08| +2452661|23524|9399|83054|248817|5427|14837|83054|248817|5427|14837|48|3|1804|3|56.13|4.49|60.62|62.95|39.51|21.32|25.41|9.40|106.95| +2452815|65422|1725|494|82709|5277|22645|494|82709|5277|22645|27|7|1804|38|3242.92|259.43|3502.35|79.22|420.28|3242.92|0.00|0.00|758.93| +2452409|48025|7917|43913|1686130|2747|14164|43913|1686130|2747|14164|7|3|1805|45|1683.90|101.03|1784.93|79.93|2164.95|1077.69|290.98|315.23|2345.91| +2452404|32807|301|29713|450601|4289|42377|29713|450601|4289|42377|9|3|1806|37|1273.54|63.67|1337.21|14.72|1179.19|827.80|436.82|8.92|1257.58| +2452656|36657|5706|25469|594142|6368|15841|25469|594142|6368|15841|12|30|1807|15|1767.30|159.05|1926.35|72.30|846.75|1643.58|82.89|40.83|1078.10| +2452832|42924|3882|96025|1754924|5823|31875|96025|1754924|5823|31875|39|34|1808|63|2104.20|105.21|2209.41|32.74|274.05|2104.20|0.00|0.00|412.00| +2452829|65612|12057|48447|288667|3504|18579|48447|288667|3504|18579|30|15|1808|78|5867.16|410.70|6277.86|55.79|2615.34|5104.42|472.89|289.85|3081.83| +2452768|44082|17443|89565|1439436|2460|27531|89565|1439436|2460|27531|33|34|1808|37|3401.78|204.10|3605.88|60.50|1910.31|3163.65|133.35|104.78|2174.91| +2452616|66107|15151|75901|688454|5360|43466|75901|688454|5360|43466|12|8|1808|6|375.60|26.29|401.89|15.77|123.54|90.14|128.45|157.01|165.60| +2451325|33926|13999|67929|408761|3640|20872|67929|408761|3640|20872|52|23|1809|11|387.09|15.48|402.57|31.87|71.94|383.21|1.35|2.53|119.29| +2451701|72462|15566|16302|242261|4220|27026|16302|242261|4220|27026|2|16|1810|1|41.31|2.89|44.20|29.27|22.48|19.41|11.38|10.52|54.64| +2451648|32723|2908|15806|912835|1864|43246|15806|912835|1864|43246|20|16|1810|35|114.10|7.98|122.08|26.75|315.70|20.53|17.77|75.80|350.43| +2452435|61399|2573|83747|1659956|5318|28327|83747|1659956|5318|28327|13|6|1811|63|1311.03|39.33|1350.36|15.85|2184.84|1232.36|0.78|77.89|2240.02| +2450984|28125|2365|71402|1632712|2375|29285|71402|1632712|2375|29285|34|17|1815|58|6976.82|348.84|7325.66|89.87|2593.76|1534.90|4299.11|1142.81|3032.47| +2451226|22214|2242|52376|1467410|4397|36984|52376|1467410|4397|36984|8|13|1815|70|3492.30|174.61|3666.91|88.00|4725.00|174.61|364.94|2952.75|4987.61| +2451052|31988|5338|2280|1368917|1954|49391|2280|1368917|1954|49391|50|18|1815|10|182.20|7.28|189.48|79.49|269.90|36.44|14.57|131.19|356.67| +2451047|7723|12781|48318|444330|6742|31907|48318|444330|6742|31907|55|9|1815|57|10922.91|546.14|11469.05|65.51|2080.50|4806.08|2874.91|3241.92|2692.15| +2452334|57113|17081|46980|354345|463|13428|46980|354345|463|13428|7|34|1816|2|29.96|0.59|30.55|41.41|13.14|17.67|10.44|1.85|55.14| +2451305|30819|4390|13119|994196|2651|29510|13119|994196|2651|29510|34|8|1817|24|465.84|4.65|470.49|53.15|372.72|326.08|104.82|34.94|430.52| +2451290|78957|16568|51028|1676515|402|24689|51028|1676515|402|24689|49|18|1817|18|1264.50|75.87|1340.37|80.34|97.20|113.80|414.25|736.45|253.41| +||6594|55409|1820120||35589|55409|1820120||35589|18||1818|2|60.10|3.00||82.35||5.40|||| +2452142|17240|8941|72914|915692|6023|19601|72914|915692|6023|19601|57|33|1819|14|19.46|0.58|20.04|96.90|240.94|10.11|9.25|0.10|338.42| +2452306|79009|11435|74120|181331|2142|38553|74120|181331|2142|38553|23|6|1819|7|154.77|7.73|162.50|61.24|10.57|71.19|20.89|62.69|79.54| +2452273|77366|1355|18136|369216|3049|8882|18136|369216|3049|8882|1|35|1821|24|1825.44|146.03|1971.47|99.09|730.08|1150.02|553.84|121.58|975.20| +2452308|28968|373|79305|132635|2765|16895|79305|132635|2765|16895|25|17|1821|4|138.20|12.43|150.63|95.71|221.12|8.29|127.31|2.60|329.26| +2451464|65744|9109|41650|1443076|4502|1928|41650|1443076|4502|1928|40|32|1822|41|97.58|4.87|102.45|26.48|376.38|20.49|60.13|16.96|407.73| +2452297|14991|1847|33120|1239513|3506|43852|33120|1239513|3506|43852|49|28|1824|23|38.18|0.00|38.18|92.58|25.30|11.45|0.53|26.20|117.88| +2452468|14392|10577|55292|1885960|7028|38496|55292|1885960|7028|38496|49|29|1824|43|1061.24|42.44|1103.68|15.68|349.59|1061.24|0.00|0.00|407.71| +2451851|63721|2395|91281|1520804|5748|19113|91281|1520804|5748|19113|41|23|1825|81|1669.41|150.24|1819.65|57.07|417.15|1318.83|21.03|329.55|624.46| +2451365|57967|10108|68605|||28914|68605||||7|19|1827|10|||290.82|12.21|141.30|269.08|||| +2451221|44797|1012|43620|1138283|946|48809|43620|1138283|946|48809|32|2|1829|7|443.45|22.17|465.62|83.22|297.85|146.33|148.56|148.56|403.24| +2451718|46474|3751|36951|309117|930|12296|36951|309117|930|12296|19|28|1832|6|170.88|10.25|181.13|62.95|152.34|5.12|16.57|149.19|225.54| +2451722|75924|823|36189|1115616|7043|10070|36189|1115616|7043|10070|7|25|1832|76|4079.68|285.57|4365.25|54.95|665.00|4038.88|16.32|24.48|1005.52| +2451617|65646|5938|66695|1368636|977|20518|66695|1368636|977|20518|49|9|1832|29|1790.75|71.63|1862.38|16.92|235.48|1647.49|45.84|97.42|324.03| +2451651|57816|9680|55978|1021591|4451|32464|55978|1021591|4451|32464|16|9|1832|46|4898.08|440.82|5338.90|70.32|3351.10|3232.73|1182.39|482.96|3862.24| +2452822|48968|4872|18610|1060423|989|6755|18610|1060423|989|6755|21|26|1833|26|1033.76|10.33|1044.09|73.66|800.28|155.06|184.52|694.18|884.27| +2451651|58037|3514|7154|755705|609|13160|7154|755705|609|13160|55|6|1834|3|152.34|9.14|161.48|11.65|228.51|77.69|73.90|0.75|249.30| +2451269|39199|17948|93334|666908|4835|41609|93334|666908|4835|41609|49|24|1835|29|2372.49|189.79|2562.28|41.81|56.26|1708.19|192.64|471.66|287.86| +2451443|15258|566|34000|426733|5066|25799|34000|426733|5066|25799|4|2|1837|34|4797.06|287.82|5084.88|40.39|1114.18|4557.20|232.66|7.20|1442.39| +|29715|4544|75306||||75306|1785487||42296|||1837|2|||79.72|31.57|5.92|62.36|15.41|1.17|38.27| +2451961|59231|5389|34877|1135946|3768|42437|34877|1135946|3768|42437|37|13|1839|6|385.98|11.57|397.55|81.17|180.90|84.91|177.63|123.44|273.64| +2452155|78560|1501|14397|814840|6483|11726|14397|814840|6483|11726|26|7|1840|66|6969.60|487.87|7457.47|26.75|1204.50|2369.66|3357.95|1241.99|1719.12| +2451986|29722|3103|21121|1898232|4496|4007|21121|1898232|4496|4007|13|17|1840|50|85.00|5.10|90.10|71.99|41.50|74.80|6.73|3.47|118.59| +2451765|74689|7312|13278|1594404|2377|22303|13278|1594404|2377|22303|8|33|1841|20|452.40|18.09|470.49|10.19|95.20|303.10|149.30|0.00|123.48| +2451672|26630|8930|13918|179526|3427|44545|13918|179526|3427|44545|22|20|1841|4|205.96|16.47|222.43|92.91|0.00|72.08|61.58|72.30|109.38| +2452048|12069|3205|79149|1900379|4809|10792|79149|1900379|4809|10792|13|33|1843|8|716.72|14.33|731.05|67.44|342.08|186.34|297.01|233.37|423.85| +2452845|61164|8442|55124|1739061|1398|40935|55124|1739061|1398|40935|31|27|1844|7|875.07|17.50|892.57|68.75|56.42|323.77|99.23|452.07|142.67| +2452750|58321|15759|77582|960476|4970|28264|77582|960476|4970|28264|21|19|1844|37|2160.80|108.04|2268.84|5.41|135.05|605.02|902.35|653.43|248.50| +2452164|65714|8221|97828|283072|3163|24407|97828|283072|3163|24407|25|27|1845|1|1.28|0.10|1.38|30.18|1.28|1.10|0.10|0.08|31.56| +||11520|63186|771873|4560||63186||4560|37247|||1846|5|||445.15|||183.78|||| +2452185|20969|14235|48931|1483218|1858|898|48931|1483218|1858|898|13|14|1848|26|1622.14|81.10|1703.24|70.96|654.94|145.99|1269.48|206.67|807.00| +2451318|38071|9556|95527|1175380|5986|14192|95527|1175380|5986|14192|16|15|1850|8|344.88|17.24|362.12|26.18|783.84|106.91|64.25|173.72|827.26| +2451300|35783|3514|90731|561253|5822|1591|90731|561253|5822|1591|37|31|1850|8|1011.04|60.66|1071.70|30.77|584.48|101.10|327.57|582.37|675.91| +2451606|61380|13646|83109|1840724|6339|21107|83109|1840724|6339|21107|26|18|1851|24|60.48|3.02|63.50|39.07|302.40|13.30|3.30|43.88|344.49| +2452810||11994|36382|1201593|851||36382|||15262|1|3|1852|||192.14|2593.92||1182.33||1726.63|458.99|| +2451881|48287|8888|57107|1483956|2452|24948|57107|1483956|2452|24948|29|4|1854|2|366.88|3.66|370.54|32.29|130.44|14.67|147.92|204.29|166.39| +2451660|71013|3025|89529|1898625|521|38017|89529|1898625|521|38017|55|33|1854|8|0.00|0.00|0.00|88.94|91.76|0.00|0.00|0.00|180.70| +2452018|33377|8353|75328|1695487|1599|2111|75328|1695487|1599|2111|20|32|1855|17|107.61|2.15|109.76|91.69|14.79|87.16|15.33|5.12|108.63| +2451945|57270|12614|86208|1031182|4603|31177|86208|1031182|4603|31177|50|25|1856|13|216.32|6.48|222.80|0.72|618.28|32.44|51.48|132.40|625.48| +2451235|61112|8047|13549|1366883|5464|35198|13549|1366883|5464|35198|58|5|1858|65|8199.10|0.00|8199.10|79.68|4346.55|1639.82|1771.00|4788.28|4426.23| +2451688|46344|16498|30448|717425|7155|45332|30448|717425|7155|45332|46|3|1859|5|22.70|2.04|24.74|87.46|6.80|17.70|1.50|3.50|96.30| +2451654|69186|6259|96998|1344182|4966|17797|96998|1344182|4966|17797|28|3|1860|1|22.89|1.14|24.03|2.66|2.11|11.67|7.18|4.04|5.91| +||7783|43951|574119||38455|43951||3487||22|18|1860|18||||66.79|||||730.64| +|20761|659|68493||4820|23207|||||||1861|27||||4.42||68.94|||| +2451713|65651|2912|36691|1282001|3311|34668|36691|1282001|3311|34668|37|27|1862|4|139.56|5.58|145.14|47.46|73.28|40.47|35.67|63.42|126.32| +2451986|40373|5017|94376|1245114|5452|27790|94376|1245114|5452|27790|11|24|1863|4|186.08|11.16|197.24|12.35|83.08|135.83|7.53|42.72|106.59| +2451443|44766|1976|46492|1831793|4294|26748|46492|1831793|4294|26748|34|24|1864|83|97.11|4.85|101.96|71.34|126.16|70.89|25.95|0.27|202.35| +2451542|45864|13303|36274|90222|4617|18565|36274|90222|4617|18565|14|28|1865|84|710.64|14.21|724.85|93.10|100.80|334.00|210.91|165.73|208.11| +2451877|34744|3656|14879|1396195|3558|26121|14879|1396195|3558|26121|25|34|1866|66|5677.98|0.00|5677.98|16.76|4637.16|3861.02|1617.09|199.87|4653.92| +2452007|46723|11840|99826|1047289|7105|4968|99826|1047289|7105|4968|50|27|1866|1|90.31|3.61|93.92|75.89|4.57|68.63|12.79|8.89|84.07| +2451363|69393|1453|58577|1429973|6169|19612|58577|1429973|6169|19612|10|30|1868|13|3575.52|321.79|3897.31|12.50|1191.84|3504.00|22.17|49.35|1526.13| +2451503|46095|4900|76378|1643168|4872|32089|76378|1643168|4872|32089|1|24|1868|20|104.60|9.41|114.01|85.93|274.80|47.07|12.08|45.45|370.14| +2452526|22501|6705|45156|1729000|2797|43794|45156|1729000|2797|43794|23|28|1870|26|1238.12|12.38|1250.50|82.28|283.66|433.34|378.24|426.54|378.32| +2452611|13554|16023|19908|126949|5543|28137|19908|126949|5543|28137|9|22|1871|49|1874.74|149.97|2024.71|7.78|574.28|1874.74|0.00|0.00|732.03| +2452684|43186|10485|42712|671640|2168|18296|42712|671640|2168|18296|36|17|1871|13|216.71|13.00|229.71|29.03|143.26|179.86|16.95|19.90|185.29| +2452458|34667|3637|81995|1309239|5283|29374|81995|1309239|5283|29374|43|7|1871|19|1216.95|12.16|1229.11|42.85|351.88|888.37|216.86|111.72|406.89| +2452120|69166|13023|60425|1893567|5556|26298|60425|1893567|5556|26298|11|21|1872|9|14.49|0.28|14.77|39.32|421.74|8.83|1.52|4.14|461.34| +2452276|64443|13627|64873|1714537|5001|48265|64873|1714537|5001|48265|35|23|1872|14|382.06|15.28|397.34|27.51|79.52|122.25|49.36|210.45|122.31| +2452230|2471|14861|54553|418915|6124|44486|54553|418915|6124|44486|37|30|1872|15|1454.10|14.54|1468.64|44.18|1035.45|116.32|615.37|722.41|1094.17| +2451310|72520|11990|76173|489207|6688|8522|76173|489207|6688|8522|49|30|1874|27|302.67|27.24|329.91|0.64|2907.09|111.98|76.27|114.42|2934.97| +2451330|27184|11644|89805|362403|1401|5246|89805|362403|1401|5246|26|13|1875|17|150.62|0.00|150.62|33.50|16.66|122.00|5.72|22.90|50.16| +2452383|84814|7171|43046|1701713|6427|43541|43046|1701713|6427|43541|33|31|1877|43|5512.17|0.00|5512.17|3.03|1717.85|4464.85|209.46|837.86|1720.88| +2451207|505|17030|69103|1747629|108|33834|69103|1747629|108|33834|46|34|1878|3|114.03|1.14|115.17|45.15|168.75|5.70|12.99|95.34|215.04| +2451344|1852|15442|5859|315689|5166|25043|5859|315689|5166|25043|10|6|1878|56|10888.64|435.54|11324.18|57.59|989.52|3702.13|2802.73|4383.78|1482.65| +2451287|37377|14192|58597|1297433|3469|9952|58597|1297433|3469|9952|10|15|1878|16|101.12|2.02|103.14|94.66|21.92|45.50|16.68|38.94|118.60| +2451334|32886|16042|42059|765453|6000|12194|42059|765453|6000|12194|38|4|1879|32|1649.60|131.96|1781.56|60.15|441.28|49.48|1136.08|464.04|633.39| +2451419|61635|472|54867|1225976|638|29347|54867|1225976|638|29347|7|12|1879|7|280.49|11.21|291.70|22.80|146.72|16.82|60.64|203.03|180.73| +2452484|20297|10435|97621|1811264|554|39397|97621|1811264|554|39397|37|16|1880|5|598.65|53.87|652.52|98.90|329.55|125.71|222.28|250.66|482.32| +2451128|46972|10112|16749|939079|4609|37692|16749|939079|4609|37692|14|8|1881|55|36.30|1.08|37.38|60.11|990.00|35.57|0.24|0.49|1051.19| +2452027|58801|11072|31496|1361626|1003|41933|31496|1361626|1003|41933|5|16|1882|79|804.22|40.21|844.43|78.37|1072.82|418.19|374.44|11.59|1191.40| +2451728|70535|683|82230|306519|2008|23497|82230|306519|2008|23497|59|33|1883|19|2153.65|0.00|2153.65|91.59|717.82|1722.92|68.91|361.82|809.41| +2451756|43938|11615|6970|420584|6998|33812|6970|420584|6998|33812|49|1|1883|20|2037.60|183.38|2220.98|6.08|569.20|1568.95|379.60|89.05|758.66| +2452302|83708|12667|4234|654016|4598|35580|4234|654016|4598|35580|9|27|1884|74|2080.88|187.27|2268.15|99.56|1986.16|499.41|1091.21|490.26|2272.99| +2451663|18197|16573|17964|972990|369|30195|17964|972990|369|30195|10|27|1886|72|1285.92|115.73|1401.65|8.98|0.00|822.98|138.88|324.06|124.71| +2451565|33821|7480|93754|604315|5276|46237|93754|604315|5276|46237|50|3|1886|2|29.94|2.69|32.63|63.06|38.24|14.67|14.35|0.92|103.99| +2451792|37453|5347|85898|1595312|726|7340|85898|1595312|726|7340|31|21|1887|58|731.38|65.82|797.20|33.80|24.94|497.33|91.27|142.78|124.56| +2451949|81769|10507|9016|305768|5720|5672|9016|305768|5720|5672|59|1|1887|3|172.71|15.54|188.25|19.25|86.34|132.98|23.04|16.69|121.13| +2451796|8232|13885|39083|474551|1572|29840|39083|474551|1572|29840|20|19|1887|47|3495.39|139.81|3635.20|50.43|1003.92|2516.68|29.36|949.35|1194.16| +2451782|61261|9181|64361|704057|4553|19049|64361|704057|4553|19049|13|19|1888|25|727.25|0.00|727.25|68.47|1882.50|443.62|99.27|184.36|1950.97| +2451993|46951|4202|85641|1690824|6854|13237|85641|1690824|6854|13237|5|23|1889|54|6325.56|63.25|6388.81|5.86|2874.96|1834.41|4176.76|314.39|2944.07| +2451421|60108|386|92384|1831794|1157|12167|92384|1831794|1157|12167|22|5|1892|10|392.40|3.92|396.32|13.65|318.40|376.70|6.12|9.58|335.97| +2451478|35340|6134|45362|1580728|1247|49366|45362|1580728|1247|49366|14|15|1892|12|666.00|39.96|705.96|98.26|259.92|552.78|35.09|78.13|398.14| +2451482|44756|8900|55187|1897965|2343|32674|55187|1897965|2343|32674|4|20|1892|49|2630.81|157.84|2788.65|6.24|1527.33|736.62|719.79|1174.40|1691.41| +2451402|62573|4598|75793|925187|5781|34375|75793|925187|5781|34375|52|33|1892|62|3896.70|350.70|4247.40|20.97|116.56|2221.11|1390.73|284.86|488.23| +2451601|44342|9446|55187|1897965|2343|32674|55187|1897965|2343|32674|31|21|1892|13|388.44|31.07|419.51|57.57|101.53|229.17|28.66|130.61|190.17| +2451911|33020|3005|18130|1891347|4564|8581|18130|1891347|4564|8581|37|18|1893|28|1179.92|11.79|1191.71|40.12|37.80|1014.73|145.36|19.83|89.71| +2451882|45946|3563|8821|1919009|6797|44185|8821|1919009|6797|44185|50|28|1893|51|7027.29|281.09|7308.38|1.93|2810.61|5762.37|455.37|809.55|3093.63| +2451330|70350|9404|40293|939788|4883|7531|40293|939788|4883|7531|44|9|1894|30|3498.90|279.91|3778.81|19.49|2226.60|1539.51|391.87|1567.52|2526.00| +2451300|49694|17528|35530|880736|4789|24842|35530|880736|4789|24842|31|31|1894|36|2514.24|75.42|2589.66|9.56|2324.52|1081.12|1017.51|415.61|2409.50| +2452730|51330|8541|93193|930258|6970|16835|93193|930258|6970|16835|27|29|1895|41|1553.49|31.06|1584.55|77.97|2193.09|388.37|699.07|466.05|2302.12| +2452749|35946|3567|35259|969769|5079|19236|35259|969769|5079|19236|6|13|1895|15|358.50|0.00|358.50|50.63|12.15|351.33|2.79|4.38|62.78| +2451851|44818|15205|96774|1845953|6332|45864|96774|1845953|6332|45864|55|9|1896|3|0.57|0.04|0.61|56.69|2.91|0.30|0.15|0.12|59.64| +|86182|17069||1048380|6377||59536|1048380|||||1896|||25.21|385.37|||280.92||74.49|370.21| +2452234|42653|13331|79206|107327|1048|31616|79206|107327|1048|31616|17|12|1897|2|35.56|2.84|38.40|42.03|58.76|22.40|12.89|0.27|103.63| +2451641|67429|17030|25302|631954|4206|25258|25302|631954|4206|25258|28|29|1898|21|1664.88|83.24|1748.12|97.90|470.40|1431.79|104.89|128.20|651.54| +2451703|56041|1364|50426|1748416|6759|39949|50426|1748416|6759|39949|26|30|1898|38|610.28|42.71|652.99|85.01|138.70|555.35|43.39|11.54|266.42| +2451558|34147|1328|64280|272201|6169|37690|64280|272201|6169|37690|38|29|1898|7|87.64|1.75|89.39|81.14|20.86|7.01|78.21|2.42|103.75| +2452067|36373|14018|21688|1513973|7058|2190|21688|1513973|7058|2190|50|25|1899|4|295.48|0.00|295.48|20.26|525.32|189.10|65.95|40.43|545.58| +2451997|47248|16813|34529|51231|7133|682|34529|51231|7133|682|2|1|1899|29|2601.30|52.02|2653.32|24.69|1474.07|52.02|2498.29|50.99|1550.78| +2451799|72483|3853|99238|1020197|1156|3715|99238|1020197|1156|3715|2|18|1899|2|127.68|1.27|128.95|43.10|57.44|120.01|2.91|4.76|101.81| +2451697|68440|8089|71035|280539|90|30394|71035|280539|90|30394|55|24|1900|1|2.72|0.10|2.82|28.51|12.47|2.44|0.00|0.28|41.08| +2451825|27062|13424|76082|87660|485|44063|76082|87660|485|44063|26|32|1900|49|2782.71|166.96|2949.67|53.67|989.31|1530.49|100.17|1152.05|1209.94| +2452701|52620|505|60666|708708|5550|30792|60666|708708|5550|30792|30|27|1901|3|84.33|2.52|86.85|78.12|51.72|32.04|50.72|1.57|132.36| +2452804|64825|435|24987|848842|3676|1808|24987|848842|3676|1808|54|17|1901|2|9.40|0.84|10.24|21.16|1.28|2.91|0.64|5.85|23.28| +2451316|67556|6526|3794|1119081|415|6102|3794|1119081|415|6102|32|20|1902|51|5428.44|271.42|5699.86|54.55|3334.38|2659.93|1882.58|885.93|3660.35| +2451940|22211|5|62073|204355|5484|8966|62073|204355|5484|8966|47|10|1905|1|159.94|6.39|166.33|55.38|37.43|113.55|6.95|39.44|99.20| +2451318|27564|4162|47260|424944|1828|17159|47260|424944|1828|17159|14|15|1908|51|554.37|16.63|571.00|7.78|1980.84|299.35|38.25|216.77|2005.25| +2451247|44173|11083|26961|1067227|293|9359|26961|1067227|293|9359|20|10|1911|20|335.60|16.78|352.38|37.27|107.20|90.61|58.79|186.20|161.25| +2452351|75731|4217|32776|568801|3673|46898|32776|568801|3673|46898|5|12|1912|86|2033.04|121.98|2155.02|15.29|1251.30|101.65|405.59|1525.80|1388.57| +2452351|81810|1871|76397|295697|3797|12697|76397|295697|3797|12697|41|30|1912|90|16487.10|164.87|16651.97|89.42|445.50|11540.97|3313.90|1632.23|699.79| +2452391|51312|5683|49955|861599|3290|20934|49955|861599|3290|20934|1|35|1914|44|1923.24|38.46|1961.70|75.88|2000.24|96.16|475.04|1352.04|2114.58| +2452245|27705|13253|11844|1463912|5580|10848|11844|1463912|5580|10848|15|26|1914|14|143.08|5.72|148.80|72.38|114.52|17.16|98.21|27.71|192.62| +2452206|68497|13353|9200|1730092|2537|25829|9200|1730092|2537|25829|29|10|1915|22|3571.70|321.45|3893.15|44.18|378.18|2107.30|1420.46|43.94|743.81| +2452026|16565|14195|55806|820581|1003|39378|55806|820581|1003|39378|39|11|1915|27|1789.29|89.46|1878.75|83.09|57.51|912.53|587.42|289.34|230.06| +2452088|68233|6329|80286|666454|1069|3284|80286|666454|1069|3284|51|35|1915|14|679.42|6.79|686.21|43.36|194.04|563.91|51.97|63.54|244.19| +2451553|5550|17143|41248|334710|4972|27587|41248|334710|4972|27587|52|12|1916|11|693.44|34.67|728.11|34.95|180.51|443.80|169.75|79.89|250.13| +2452025|82316|8678|28013|1786275|537|35599|28013|1786275|537|35599|7|11|1917|65|1823.25|127.62|1950.87|28.70|233.35|948.09|472.58|402.58|389.67| +2451324|16526|3574|76525|1250772|572|25172|76525|1250772|572|25172|14|26|1918|14|127.12|3.81|130.93|97.33|262.08|19.06|37.82|70.24|363.22| +2451213|74277|8563|47556|1792375|3947|44366|47556|1792375|3947|44366|55|8|1919|12|74.76|5.98|80.74|79.47|2.88|3.73|41.90|29.13|88.33| +2451724|39686|10664|80711|1786704|3897|25443|80711|1786704|3897|25443|38|9|1920|31|462.21|41.59|503.80|89.98|82.15|420.61|21.21|20.39|213.72| +2451500|61968|7147|76691|354160|1676|25592|76691|354160|1676|25592|2|7|1920|5|174.95|1.74|176.69|74.79|316.25|24.49|37.61|112.85|392.78| +2452794|57255|2329|77726|1525623|5873|31770|77726|1525623|5873|31770|19|8|1922|8|223.36|17.86|241.22|65.78|305.60|223.36|0.00|0.00|389.24| +2452286|59463|10603|39326|717391|5856|36321|39326|717391|5856|36321|13|30|1923|12|41.64|0.41|42.05|46.45|111.24|23.73|5.01|12.90|158.10| +2452344|51271|13905|83149|178886|4670|46007|83149|178886|4670|46007|33|5|1923|13|229.84|6.89|236.73|37.85|257.40|20.68|129.67|79.49|302.14| +2451188|16539|4838|94299|628286|1473|37860|94299|628286|1473|37860|25|12|1924|12|56.40|1.12|57.52|57.79|174.60|32.71|11.13|12.56|233.51| +2451153|63419|5020|36209|1005322|5635|4748|36209|1005322|5635|4748|50|34|1924|29|6643.61|597.92|7241.53|21.56|2684.24|3653.98|1973.15|1016.48|3303.72| +2451625|30043|1304|51546|1073440|5734|4428|51546|1073440|5734|4428|50|27|1926|83|7285.74|145.71|7431.45|38.80|4163.28|291.42|3147.44|3846.88|4347.79| +2452202|45471|2777|39867|1023651|1388|46372|39867|1023651|1388|46372|55|20|1927|43|422.69|38.04|460.73|20.30|563.30|295.88|10.14|116.67|621.64| +2451198|69266|13036|72209|1404651|5093|48335|72209|1404651|5093|48335|40|22|1928|36|3441.60|34.41|3476.01|95.69|1346.40|2890.94|132.15|418.51|1476.50| +2451234|5227|3760|33879|1483219|3759|19546|33879|1483219|3759|19546|7|18|1928|12|704.64|42.27|746.91|48.55|139.20|324.13|34.24|346.27|230.02| +2451507|55635|740|80983|1164191|5799|11007|80983|1164191|5799|11007|4|35|1929|2|33.30|2.66|35.96|31.56|1.34|25.97|3.07|4.26|35.56| +2451538|27617|5485|67577|535559|3541|42770|67577|535559|3541|42770|26|34|1929|5|14.30|0.42|14.72|74.38|14.30|1.71|10.70|1.89|89.10| +||601|66694|627579|4464||66694|627579|4464||11|28|1933|22|||571.12|||474.02||58.26|329.14| +2451599|52533|15598|71286|1828446|4359|29072|71286|1828446|4359|29072|1|29|1934|5|140.15|2.80|142.95|55.68|15.15|28.03|43.72|68.40|73.63| +2451683|36021|8509|51184|35734|1531|11681|51184|35734|1531|11681|13|16|1934|12|1084.44|97.59|1182.03|83.99|38.64|97.59|858.55|128.30|220.22| +2452802|70262|11983|16149|1109667|5488|14023|16149|1109667|5488|14023|9|22|1935|12|533.52|37.34|570.86|38.47|207.48|272.09|70.58|190.85|283.29| +2452610|34107|9903|54987|1529027|5114|5201|54987|1529027|5114|5201|33|16|1936|22|1110.12|33.30|1143.42|98.85|941.82|943.60|1.66|164.86|1073.97| +2452545|45349|15600||1751834|849|27017|86819|1751834|849||9||1936|||34.04|||1562.00||949.85|2216.34|1628.96| +2451601|45971|17096|64994|1573758|4922|45333|64994|1573758|4922|45333|14|26|1937|8|171.04|3.42|174.46|41.80|332.56|25.65|114.85|30.54|377.78| +||15200||1676367||8020|26717|1676367|417|8020||21|1938|12||||21.92||||48.72|592.50| +2451256|62975|12260|12657|1845909|3961|45166|12657|1845909|3961|45166|44|24|1938|45|4021.65|80.43|4102.08|45.83|228.15|80.43|157.64|3783.58|354.41| +2452753|68182|7039|79926|623203|433|38510|79926|623203|433|38510|33|11|1939|21|915.81|27.47|943.28|4.17|0.00|476.22|127.48|312.11|31.64| +2452028|44659|5978|97076|1413693|2053|46978|97076|1413693|2053|46978|31|19|1941|4|36.96|1.10|38.06|48.23|98.64|18.48|10.16|8.32|147.97| +2451967|72742|13381|79980|187248|563|3831|79980|187248|563|3831|35|22|1941|17|2307.24|184.57|2491.81|5.77|489.26|1684.28|386.23|236.73|679.60| +2452078|74682|884|76853|1307746|1839|23532|76853|1307746|1839|23532|17|2|1941|60|13285.20|797.11|14082.31|76.14|2391.00|11026.71|2168.15|90.34|3264.25| +2452162|15600|11243|6150|493580|3057|21687|6150|493580|3057|21687|26|26|1942|30|2927.10|87.81|3014.91|53.35|284.40|2136.78|308.22|482.10|425.56| +2452074|70221|8789|27693|356523|2953|21736|27693|356523|2953|21736|8|31|1942|11|1105.72|11.05|1116.77|12.94|257.95|265.37|680.68|159.67|281.94| +2451982|13418|10142|57620|1198074|3113|14654|57620|1198074|3113|14654|49|25|1942|1|12.63|0.25|12.88|98.97|18.62|1.13|6.67|4.83|117.84| +2451610|41338|14170|53825|1786078|2331|20984|53825|1786078|2331|20984|44|17|1943|62|230.02|11.50|241.52|44.72|1517.76|227.71|0.46|1.85|1573.98| +2451357|66758|15188|31855|663860|7065|28670|31855|663860|7065|28670|55|24|1945|26|204.88|4.09|208.97|4.19|172.90|165.95|18.68|20.25|181.18| +2451220|70992|14431|26831|144367|1103|45886|26831|144367|1103|45886|58|31|1945|62|4402.00|264.12|4666.12|19.94|1158.16|2641.20|176.08|1584.72|1442.22| +2452286|68522|15969|39931|791556|4781|35206|39931|791556|4781|35206|15|31|1946|24|474.96|4.74|479.70|26.11|129.36|56.99|62.69|355.28|160.21| +2452057|74862|14653|32292|800847|3959|19193|32292|800847|3959|19193|25|24|1947|10|411.20|8.22|419.42|71.97|729.00|16.44|98.69|296.07|809.19| +2451958|71791|7201|20104|1036546|988|45430|20104|1036546|988|45430|2|30|1947|1|8.63|0.17|8.80|55.13|6.19|6.38|0.31|1.94|61.49| +2452514|46679|5238|93826|762869|7112|25355|93826|762869|7112|25355|9|7|1948|31|1590.30|31.80|1622.10|74.24|103.54|1097.30|315.52|177.48|209.58| +2452554|81065|12561|58998|136890|4735|36580|58998|136890|4735|36580|25|24|1949|55|5489.55|164.68|5654.23|4.11|1761.65|2140.92|1406.42|1942.21|1930.44| +2452534|4163|16069|19576|747618|3660|4502|19576|747618|3660|4502|42|2|1950|72|3906.00|39.06|3945.06|54.94|3794.40|3437.28|98.43|370.29|3888.40| +2452633|44561|9207|31094|212887|5880|44120|31094|212887|5880|44120|24|5|1950|2|45.84|2.29|48.13|30.29|140.42|45.84|0.00|0.00|173.00| +2452492|58957|3300|76099|1692848|2844|26840|76099|1692848|2844|26840|30|2|1950|1|26.46|2.38|28.84|51.86|11.97|4.76|14.75|6.95|66.21| +2452498|46532|2628|92108|611678|401|7571|92108|611678|401|7571|25|34|1951|28|220.64|4.41|225.05|67.11|1030.68|214.02|1.32|5.30|1102.20| +2452535|62695|9330|9178|888606|1634|29062|9178|888606|1634|29062|6|22|1951|4|272.04|13.60|285.64|99.44|158.68|195.86|38.09|38.09|271.72| +2452479|18665|15151|27171|1207453|535|33970|27171|1207453|535|33970|21|35|1951|11|886.38|8.86|895.24|71.70|69.63|274.77|24.46|587.15|150.19| +2452526|74605|2243|57376|1115504|1632|7360|57376|1115504|1632|7360|51|3|1952|19|1436.21|114.89|1551.10|20.92|460.94|244.15|11.92|1180.14|596.75| +2452348|49831|12781|14688|1597964|1687|9410|14688|1597964|1687|9410|41|3|1952|32|1067.20|10.67|1077.87|72.29|717.12|64.03|561.77|441.40|800.08| +||8384|62463||5707|||793685|5707|37776|56|4|1953||3464.26|||||||1784.80|1160.78| +2452752|40636|16755|10607|1354479|5104|23376|10607|1354479|5104|23376|30|11|1954|47|7941.12|317.64|8258.76|69.86|661.76|5558.78|190.58|2191.76|1049.26| +2452741|48623|2079|66908|1726988|4221|31501|66908|1726988|4221|31501|24|3|1954|8|179.60|8.98|188.58|55.21|179.60|0.00|145.47|34.13|243.79| +2452740|59863|1933|13678|464130|1782|14587|13678|464130|1782|14587|49|8|1954|5|14.15|1.13|15.28|10.04|9.75|11.03|0.28|2.84|20.92| +2451304|63954|9331|3950|1301958|2719|10327|3950|1301958|2719|10327|13|19|1955|48|270.72|16.24|286.96|63.42|948.48|200.33|29.56|40.83|1028.14| +2452806|6669|261|24173|1322034|7095|17356|24173|1322034|7095|17356|33|35|1956|15|105.15|9.46|114.61|42.50|4.20|13.66|57.63|33.86|56.16| +2451860|62479|16111|13485|1412656|5552|49547|13485|1412656|5552|49547|5|29|1957|3|77.70|0.00|77.70|71.83|83.07|17.09|18.18|42.43|154.90| +2451887|7444|14899|60619|1024262|4705|14301|60619|1024262|4705|14301|11|2|1957|3|301.23|0.00|301.23|1.01|137.52|117.47|126.79|56.97|138.53| +2451931|73658|7621|57021|1249834|7000|17413|57021|1249834|7000|17413|23|16|1957|58|1191.90|107.27|1299.17|61.10|777.20|166.86|184.50|840.54|945.57| +2451871|12172|12287|59414|935847|1433|32266|59414|935847|1433|32266|59|33|1957|35|1752.80|0.00|1752.80|79.93|1475.95|1595.04|94.65|63.11|1555.88| +2451490|55737|10136|77195|1662616|4837|41858|77195|1662616|4837|41858|13|27|1958|50|1569.50|141.25|1710.75|35.48|572.50|282.51|926.63|360.36|749.23| +2452897|67221|7737|1526|343345|4811|37830|1526|343345|4811|37830|25|5|1960|1|39.96|1.99|41.95|1.88|4.90|15.58|17.79|6.59|8.77| +2452743|77030|17892|96507|811886|4512|26738|96507|811886|4512|26738|31|10|1960|26|25.74|1.80|27.54|47.46|86.32|12.61|6.17|6.96|135.58| +2451087|57850|6212|38012|842260|4461|11862|38012|842260|4461|11862|50|2|1961|9|387.36|23.24|410.60|71.11|205.11|135.57|188.84|62.95|299.46| +||3189||385597|581||89013|||28365|||1962|||27.39|418.69|||375.64|0.15|15.51|595.92| +2450960|68039|10976|72134|658064|1594|8324|72134|658064|1594|8324|7|14|1963|1|11.64|0.00|11.64|88.04|16.93|7.10|0.81|3.73|104.97| +2452348|70272|7037|17327|1111304|4481|32458|17327|1111304|4481|32458|45|33|1964|83|15689.49|1412.05|17101.54|29.32|1394.40|9413.69|2824.11|3451.69|2835.77| +||7890||51289|5759||66419|51289|||||1965||937.80|||76.39|249.24||||391.27| +2452533||11331|37729|748976|5470|24364|||5470|24364||24|1965|12||13.34|161.66|75.42|||74.23|60.75|| +2452708|25129|11364|78|602695|4512|13129|78|602695|4512|13129|51|26|1966|30|144.90|0.00|144.90|24.22|368.10|118.81|9.65|16.44|392.32| +2451222|66108|10741|34648|1475878|308|44738|34648|1475878|308|44738|4|18|1968|9|569.07|28.45|597.52|44.06|159.30|523.54|18.21|27.32|231.81| +2452640|19926|5826|16494|70580|6619|15105|16494|70580|6619|15105|37|35|1969|53|1655.72|99.34|1755.06|69.65|643.42|1357.69|160.93|137.10|812.41| +2450969|27297|6470|95202|936408|3214|4820|95202|936408|3214|4820|25|18|1970|8|33.12|1.98|35.10|69.20|389.84|13.91|12.67|6.54|461.02| +2452394|53473|15156|25269|1480549|1782|18262|25269|1480549|1782|18262|54|15|1972|10|71.90|0.00|71.90|23.36|767.60|29.47|4.24|38.19|790.96| +2452434|67894|17247|32192|875497|3955|60|32192|875497|3955|60|25|22|1972|13|38.09|0.76|38.85|65.80|53.95|37.70|0.30|0.09|120.51| +|29491|15445|1355|1697832|709||1355|||||29|1973||||612.99|||||36.44|507.91| +2452208|8826|1261|98710|681943|823|44456|98710|681943|823|44456|31|23|1973|8|1348.32|0.00|1348.32|30.75|149.76|364.04|314.96|669.32|180.51| +2452649|47210|5329|23654|554274|2847|21067|23654|554274|2847|21067|57|2|1974|25|1208.50|84.59|1293.09|86.27|329.50|277.95|474.58|455.97|500.36| +2452076|35785|15032|34053|422104|5355|29006|34053|422104|5355|29006|19|12|1975|23|328.44|13.13|341.57|84.06|131.33|328.44|0.00|0.00|228.52| +2451312|30492|15790|23220|75846|3225|32204|23220|75846|3225|32204|28|22|1976|10|779.70|38.98|818.68|52.57|207.30|7.79|694.71|77.20|298.85| +2452754|36208|15711|98069|170934|2199|36264|98069|170934|2199|36264|7|27|1977|51|4027.98|322.23|4350.21|95.35|4271.76|241.67|2006.74|1779.57|4689.34| +2452746|31671|10272|61648|1193643|3591|15539|61648|1193643|3591|15539|21|33|1977|3|238.05|9.52|247.57|79.91|18.03|233.28|3.76|1.01|107.46| +2451363|51719|10712|50528|80620|7065|27910|50528|80620|7065|27910|8|27|1978|29|1103.16|0.00|1103.16|38.05|2965.25|529.51|441.71|131.94|3003.30| +2451815|33136|1399|74263|1239320|596|3140|74263|1239320|596|3140|55|31|1980|32|20.48|1.84|22.32|33.61|82.56|12.90|2.72|4.86|118.01| +2451215|71984|14786|44982|1896258|5134|6763|44982|1896258|5134|6763|22|29|1981|27|2129.22|21.29|2150.51|57.86|1987.20|1405.28|680.50|43.44|2066.35| +2451245|58618|15392|77295|1065124|2106|15935|77295|1065124|2106|15935|44|22|1981|51|5448.33|217.93|5666.26|78.83|718.08|980.69|3440.08|1027.56|1014.84| +2451414|51023|14702||||44973|72873||||20|15|1981||5.48||5.58||31.97||0.08|0.36|| +2452667|77693|17757|39305|605437|3569|101|39305|605437|3569|101|7|13|1982|84|11853.24|118.53|11971.77|81.56|6052.20|1777.98|2619.56|7455.70|6252.29| +2451778|39182|9092|68924|1470747|4217|46440|68924|1470747|4217|46440|23|14|1983|17|2536.40|202.91|2739.31|94.55|810.56|2409.58|92.57|34.25|1108.02| +2452371|24681|15635|71561|803501|5488|2628|71561|803501|5488|2628|39|22|1984|25|3263.25|293.69|3556.94|44.73|511.75|554.75|975.06|1733.44|850.17| +2452294|15774|5763|23391|677081|4371|49661|23391|677081|4371|49661|25|10|1985|11|962.50|77.00|1039.50|48.54|127.38|558.25|303.18|101.07|252.92| +2452183|29955|14465|75181|218356|1084|37859|75181|218356|1084|37859|17|21|1985|41|7473.48|597.87|8071.35|15.90|862.23|822.08|4057.35|2594.05|1476.00| +2452147|60560|8865|35505|1013850|5442|8587|35505|1013850|5442|8587|25|10|1985|18|1288.26|103.06|1391.32|19.92|71.46|283.41|411.98|592.87|194.44| +2450934|27556|11600|42901|1145249|5997|47098|42901|1145249|5997|47098|10|14|1987|2|92.42|2.77|95.19|75.09|118.10|77.63|1.77|13.02|195.96| +2452849|82132|7983|80500|61329|2158|18234|80500|61329|2158|18234|9|20|1988|1|82.45|7.42|89.87|60.55|27.48|50.29|8.68|23.48|95.45| +2452706|73915|7404|19314|953740|966|22992|19314|953740|966|22992|12|30|1988|1|39.74|3.57|43.31|34.12|9.66|2.78|14.04|22.92|47.35| +2452514|65775|1391|52625|442585|6945|19258|52625|442585|6945|19258|11|6|1989|7|587.86|0.00|587.86|78.66|159.04|517.31|32.45|38.10|237.70| +2451147|40181|14180|93697|277409|6274|23186|93697|277409|6274|23186|14|24|1990|12|93.84|3.75|97.59|22.95|82.08|55.36|7.31|31.17|108.78| +2452408|71619|9681|50123|1475771|6666|20714|50123|1475771|6666|20714|39|24|1991|31|560.48|11.20|571.68|65.80|980.84|280.24|266.22|14.02|1057.84| +2452404|39435|13703|13544|1678376|4361|7993|13544|1678376|4361|7993|15|24|1991|3|38.34|1.91|40.25|27.01|110.52|36.80|0.56|0.98|139.44| +2452571|30382|9009|45289|594128|2967|3053|45289|594128|2967|3053|6|14|1995|46|483.00|9.66|492.66|19.69|141.22|149.73|209.96|123.31|170.57| +2452567|47645|387|62803|732188|1952|28573|62803|732188|1952|28573|55|6|1995|54|188.46|0.00|188.46|98.22|1005.48|175.26|6.86|6.34|1103.70| +2452571|77115|6906|59314|1380691|6068|41045|59314|1380691|6068|41045|24|30|1995|74|663.78|59.74|723.52|43.86|1517.74|325.25|270.82|67.71|1621.34| +2451136|57163|4094|79036|1371501|3166|27278|79036|1371501|3166|27278|50|15|1996|49|148.96|4.46|153.42|61.72|208.25|47.66|74.96|26.34|274.43| +2452230|76495|4753|6149|1036274|5152|27630|6149|1036274|5152|27630|51|9|1998|2|125.04|8.75|133.79|52.32|80.76|75.02|44.51|5.51|141.83| +2452064||13987|||4575|41045||1369705|4575||15||1998|1|||3.82||23.87||0.32|0.15|| +2451199|68587|3314|6132|1215215|298|34934|6132|1215215|298|34934|14|20|1999|12|120.60|2.41|123.01|79.08|37.92|32.56|36.97|51.07|119.41| +2451058|12428|4462|48471|528162|3053|7944|48471|528162|3053|7944|28|32|1999|32|62.40|4.36|66.76|9.98|187.52|16.22|3.23|42.95|201.86| +2451946|70240|13790|81770|795325|4433|39941|81770|795325|4433|39941|19|33|2001|30|5324.40|479.19|5803.59|93.29|1005.60|2662.20|1437.58|1224.62|1578.08| +2452388|81878|13469|81250|1584615|1580|46145|81250|1584615|1580|46145|9|15|2002|5|153.70|4.61|158.31|47.25|87.85|53.79|26.97|72.94|139.71| +2451242|36592|8720|12637|15109|5581|28053|12637|15109|5581|28053|26|4|2004|2|77.28|4.63|81.91|70.86|72.56|57.96|14.10|5.22|148.05| +2452080|65337|10141|95652|1021575|3051|21774|95652|1021575|3051|21774|44|6|2005|12|370.56|33.35|403.91|99.14|39.84|74.11|118.58|177.87|172.33| +||15836|92930|||1816|92930|1750692|||50|15|2006|7||3.47|90.27|||19.09||52.82|| +2451390|68407|11626|7004|505966|4167|19163|7004|505966|4167|19163|2|14|2007|1|10.61|0.74|11.35|70.71|65.04|2.86|7.36|0.39|136.49| +2452690|21391|16747|90419|1786934|3096|25611|90419|1786934|3096|25611|57|27|2008|11|999.90|29.99|1029.89|15.66|222.20|199.98|31.99|767.93|267.85| +2451673|8059|4838|83327|1462147|4967|13834|83327|1462147|4967|13834|28|14|2010|36|640.44|51.23|691.67|17.25|945.72|12.80|169.46|458.18|1014.20| +2451758|40434|9091|46016|405069|4569|28422|46016|405069|4569|28422|53|11|2012|55|5314.65|425.17|5739.82|72.77|2872.65|4039.13|905.61|369.91|3370.59| +2451897|16980|10817|46580|996119|3258|37997|46580|996119|3258|37997|53|18|2012|12|1058.88|31.76|1090.64|95.24|239.04|10.58|891.05|157.25|366.04| +2451812|81094|2579|14463|412171|4279|14387|14463|412171|4279|14387|1|15|2012|19|134.33|0.00|134.33|6.25|0.00|114.18|12.49|7.66|6.25| +2451700|68107|6047|57920|1077832|6191|15303|57920|1077832|6191|15303|25|1|2012|74|8524.06|85.24|8609.30|0.98|1033.04|5370.15|473.08|2680.83|1119.26| +2450980|58070|10100|40813|1036602|2344|28086|40813|1036602|2344|28086|49|27|2014|2|6.96|0.00|6.96|58.49|10.24|6.61|0.18|0.17|68.73| +2452054|73289|1639|16023|1416372|5074|45376|16023|1416372|5074|45376|43|14|2015|4|146.56|10.25|156.81|36.00|9.76|67.41|7.91|71.24|56.01| +2451367|76439|10348|31949|1675867|1032|45350|31949|1675867|1032|45350|50|7|2016|11|261.47|5.22|266.69|57.81|87.12|156.88|29.28|75.31|150.15| +2452019|65757|2265|99862|1735404|5346|6841|99862|1735404|5346|6841|33|6|2017|34|1017.28|91.55|1108.83|25.26|1212.78|905.37|1.11|110.80|1329.59| +2451615|6737|9127|5817|255440|3910|733|5817|255440|3910|733|52|28|2018|6|1263.30|0.00|1263.30|71.45|416.10|75.79|296.87|890.64|487.55| +2451708|68841|2050|9760|1598875|6837|44752|9760|1598875|6837|44752|8|19|2018|59|1204.78|60.23|1265.01|24.17|323.32|253.00|866.11|85.67|407.72| +||1689|76204|1717180|2956|24732|||2956||24|6|2019||165.48||168.78||18.27|13.23|118.75||72.10| +2452649|15065|900|92473|514047|6021|42795|92473|514047|6021|42795|24|12|2019|25|304.25|3.04|307.29|41.28|486.75|130.82|86.71|86.72|531.07| +2451751|48581|14384|2963|867540|6517|23094|2963|867540|6517|23094|55|26|2020|15|3014.25|0.00|3014.25|96.16|789.45|1265.98|909.10|839.17|885.61| +2451694||8914||887249|||96194||2304|20159||13|2020||||978.08||311.20|300.20||434.33|| +2451618|48913|8930|25685|1341596|6754|43065|25685|1341596|6754|43065|4|35|2020|92|12407.12|0.00|12407.12|61.12|250.24|3349.92|7064.61|1992.59|311.36| +2452905|32805|6114|71601|550652|6150|12401|71601|550652|6150|12401|9|3|2021|8|367.04|29.36|396.40|19.72|67.28|143.14|91.79|132.11|116.36| +2451425|58961|8293|99726|1913889|4052|23015|99726|1913889|4052|23015|49|23|2023|19|3.80|0.26|4.06|29.79|32.68|1.74|1.85|0.21|62.73| +2451248|44111|2707|33516|668470|671|22872|33516|668470|671|22872|4|24|2023|18|1139.22|68.35|1207.57|53.54|169.56|1093.65|36.00|9.57|291.45| +2452874|62708|5262|81371|1904949|3374|49799|81371|1904949|3374|49799|13|3|2024|2|62.26|1.86|64.12|62.13|266.06|46.69|12.14|3.43|330.05| +2451723|86287|13414||||||||46884||35|2025||886.62||||396.12||||502.54| +2451514|75889|2353|26879|499274|1622|11217|26879|499274|1622|11217|52|15|2025|67|527.29|47.45|574.74|26.11|597.64|195.09|9.96|322.24|671.20| +2452076|21349|5888|7324|709830|1136|8089|7324|709830|1136|8089|13|22|2026|1|6.05|0.24|6.29|68.25|8.65|3.93|0.59|1.53|77.14| +2451093|54559|12788|81363|1827613|710|18160|81363|1827613|710|18160|46|11|2027|26|390.26|15.61|405.87|14.43|127.92|269.27|76.22|44.77|157.96| +2451035|62847|4393|95860|1009443|4979|6115|95860|1009443|4979|6115|10|30|2027|29|895.23|0.00|895.23|71.13|226.20|859.42|32.58|3.23|297.33| +2452625|15825|14706|89063|1467249|188|3574|89063|1467249|188|3574|54|23|2028|54|3576.96|250.38|3827.34|45.06|500.58|1323.47|2118.28|135.21|796.02| +2452614|42399|4995|67544|943384|4377|35817|67544|943384|4377|35817|6|13|2028|5|334.95|30.14|365.09|70.16|82.75|177.52|85.01|72.42|183.05| +2452863|30441|2013|55464|1502356|4095|46234|55464|1502356|4095|46234|37|16|2029|16|542.40|10.84|553.24|73.94|160.16|37.96|464.08|40.36|244.94| +2452569|53523|10332|33568|1778393|4371|7996|33568|1778393|4371|7996|24|17|2030|10|54.80|0.54|55.34|26.18|231.40|20.82|6.11|27.87|258.12| +2452148|75952|10533|66738|1533246|554|28662|66738|1533246|554|28662|1|25|2032|51|1970.13|39.40|2009.53|57.69|706.86|197.01|1187.99|585.13|803.95| +2451948|12287|2695|64914|1752200|3893|22278|64914|1752200|3893|22278|43|1|2036|39|3710.07|296.80|4006.87|3.11|1434.42|2337.34|507.91|864.82|1734.33| +2452072|46703|569|92344|69715|7086|49210|92344|69715|7086|49210|25|14|2038|12|181.56|9.07|190.63|28.67|496.20|168.85|12.45|0.26|533.94| +2452758|58829|1461|94048|1382200|5478|4719|94048|1382200|5478|4719|1|33|2040|1|118.20|0.00|118.20|41.91|0.00|44.91|47.63|25.66|41.91| +2452588|65845|15444|35446|590112|7178|36244|35446|590112|7178|36244|21|35|2041|32|1665.28|133.22|1798.50|29.25|221.76|932.55|512.91|219.82|384.23| +2452606|75565|13233|52465|680356|3094|36985|52465|680356|3094|36985|19|9|2041|46|72.68|2.90|75.58|27.30|25.76|51.60|15.17|5.91|55.96| +2451307|65814|5576|55681|964635|1942|28496|55681|964635|1942|28496|46|31|2045|10|78.90|2.36|81.26|50.57|7.80|73.37|3.48|2.05|60.73| +2451088|58441|5410|46686|150083|5033|19630|46686|150083|5033|19630|37|31|2045|1|60.64|0.60|61.24|68.95|16.17|35.77|1.49|23.38|85.72| +2452585|78292|8340|53450|1113755|907|24459|53450|1113755|907|24459|1|9|2046|42|1464.54|131.80|1596.34|22.22|1930.32|336.84|315.75|811.95|2084.34| +2452565|24988|13242|52043|1104833|2423|15166|52043|1104833|2423|15166|24|29|2046|11|588.28|41.17|629.45|75.94|363.66|17.64|228.25|342.39|480.77| +2452300|28177|8453|90209|299046|2283|25073|90209|299046|2283|25073|41|18|2049|8|31.12|2.17|33.29|27.37|93.52|0.62|14.94|15.56|123.06| +2451997|17097|12098|52751|37552|3153|13554|52751|37552|3153|13554|53|32|2051|3|142.71|5.70|148.41|97.17|119.49|115.59|2.98|24.14|222.36| +2452098|68557|3620|8911|1815881|4804|16927|8911|1815881|4804|16927|31|17|2051|1|109.71|9.87|119.58|78.22|98.74|71.31|19.58|18.82|186.83| +2451808|44928|14387|83617|490079|988|23897|83617|490079|988|23897|32|25|2052|1|61.82|0.00|61.82|49.23|23.18|0.61|10.40|50.81|72.41| +2452354|67473|9357|77373|1759265|6310|555|77373|1759265|6310|555|21|4|2053|19|570.95|51.38|622.33|93.24|241.49|428.21|57.09|85.65|386.11| +2452015|79887|11972|55988|437198|361|25571|55988|437198|361|25571|5|9|2055|9|232.11|0.00|232.11|24.20|68.22|148.55|41.78|41.78|92.42| +2451920|54976|9709|66767|550058|6149|47840|66767|550058|6149|47840|37|11|2057|8|75.36|0.75|76.11|99.61|124.56|11.30|40.99|23.07|224.92| +2451979|70870|5533|38018|1743790|3962|13334|38018|1743790|3962|13334|11|32|2057|6|852.42|17.04|869.46|4.50|274.98|818.32|5.11|28.99|296.52| +|7377|17618|53715||3765|38790|53715||3765|38790|25||2060|86|831.62||881.51||4437.60||0.00||4573.58| +2451267|6933|1076|82365|18216|2067|31518|82365|18216|2067|31518|52|29|2061|19|2234.21|156.39|2390.60|16.46|455.05|245.76|795.38|1193.07|627.90| +||443||408814|||37659||||45|25|2065|7|476.21|33.33|||309.54|||105.87|| +2452411|43413|17445|23265|1247972|2032|14181|23265|1247972|2032|14181|31|16|2065|6|53.76|0.00|53.76|66.48|89.58|11.28|14.01|28.47|156.06| +2452455|40187|3445|55052|1419064|7076|33888|55052|1419064|7076|33888|39|35|2065|57|8645.76|172.91|8818.67|94.20|3966.63|6570.77|477.24|1597.75|4233.74| +2452276|43756|9351|69316|287071|6073|38675|69316|287071|6073|38675|49|23|2068|2|313.32|12.53|325.85|87.71|41.02|3.13|139.58|170.61|141.26| +2452448|71163|14949|67120|344426|1379|37932|67120|344426|1379|37932|23|31|2069|53|1279.42|38.38|1317.80|12.87|2132.72|1151.47|1.27|126.68|2183.97| +2451153|73389|5384|76631|1913310|1989||76631||||52||2070|53|561.80||584.27||||333.70|3.38|482.59| +2452239|29865|14821|49880|1314518|1909|28500|49880|1314518|1909|28500|29|30|2071|23|918.62|9.18|927.80|78.99|577.30|918.62|0.00|0.00|665.47| +2451557|80442|14348|46648|1144256|4264|21825|46648|1144256|4264|21825|49|26|2072|7|32.76|0.98|33.74|26.07|65.59|1.31|10.69|20.76|92.64| +2452284|73146|4969|67365|1327344|570|41749|67365|1327344|570|41749|9|19|2073|4|269.52|8.08|277.60|35.25|231.00|150.93|103.17|15.42|274.33| +2452119|64116|12439|87657|415347|2590|35033|87657|415347|2590|35033|39|7|2073|38|671.84|33.59|705.43|26.13|448.02|631.52|30.24|10.08|507.74| +2452669|34133|14631|96593|414081|7023|23161|96593|414081|7023|23161|6|6|2075|5|23.35|0.00|23.35|44.35|22.40|13.54|0.78|9.03|66.75| +2452841|61958|4195|43326|1142617|1108|4399|43326|1142617|1108|4399|51|10|2075|25|458.25|9.16|467.41|74.07|0.00|174.13|36.93|247.19|83.23| +2451642|70825|17174|89645|1429460|6875|39890|89645|1429460|6875|39890|46|6|2076|13|1019.59|40.78|1060.37|92.72|92.69|407.83|446.58|165.18|226.19| +2451470|82210|5341|87447|1581916|2338|18807|87447|1581916|2338|18807|22|18|2076|75|3994.50|39.94|4034.44|66.35|2271.00|79.89|548.04|3366.57|2377.29| +2452697|58574|7141|21421|405207|1088|6118|21421|405207|1088|6118|33|9|2077|7|370.23|18.51|388.74|73.78|162.68|225.84|142.94|1.45|254.97| +2451926|53053|5702|96236|386496|4756|27416|96236|386496|4756|27416|26|13|2078|38|2981.10|238.48|3219.58|14.76|808.26|2176.20|796.85|8.05|1061.50| +2452558|38409|4806|96029|1127942|1102|2262|96029|1127942|1102|2262|33|25|2079|5|829.30|74.63|903.93|78.07|385.45|306.84|475.43|47.03|538.15| +2452575|24223|3738|66285|1449698|5404|5070|66285|1449698|5404|5070|39|27|2079|23|494.96|44.54|539.50|95.22|175.49|183.13|305.59|6.24|315.25| +2452803|28616|4285|76769|705626|5330|41787|76769|705626|5330|41787|7|32|2081|12|1102.08|44.08|1146.16|69.35|440.76|165.31|252.92|683.85|554.19| +2452123|57500|638|74736|1850089|2595|41548|74736|1850089|2595|41548|20|10|2082|1|97.22|3.88|101.10|91.11|0.00|3.88|93.34|0.00|94.99| +||2161|29661||||||2921|9104|41||2082|4||2.90||95.66||21.65|5.76|4.91|| +2451974|36458|14759|29724|1821752|980|25808|29724|1821752|980|25808|17|11|2082|14|1066.38|85.31|1151.69|11.11|123.34|906.42|22.39|137.57|219.76| +2452139|8580|5846|59815|1910378|3341|47904|59815|1910378|3341|47904|49|30|2082|9|165.06|13.20|178.26|39.77|371.43|113.89|12.28|38.89|424.40| +2451649|36418|12614|81968|384300|3432|10964|81968|384300|3432|10964|4|14|2083|8|84.88|5.94|90.82|10.45|152.72|44.13|17.52|23.23|169.11| +2451793|52222|2851|65723|77278|386|42741|65723|77278|386|42741|4|10|2083|1|115.93|8.11|124.04|66.06|27.75|62.60|38.39|14.94|101.92| +2451236|78645|16699|83774|460063|2752|7841|83774|460063|2752|7841|1|9|2084|47|3660.36|109.81|3770.17|93.11|699.36|3660.36|0.00|0.00|902.28| +|43955|12643||341946|5612|2125||||2125|59||2085|6|||407.24|23.81|||||| +2452352|31270|697|23396|44691|5986|928|23396|44691|5986|928|43|32|2085|6|269.88|24.28|294.16|51.51|247.44|156.53|41.93|71.42|323.23| +2452394|27161|10775|42428|58476|2524|23772|42428|58476|2524|23772|31|5|2085|11|35.31|1.41|36.72|53.69|270.93|1.05|32.88|1.38|326.03| +2452663|27383|15159|20603|1015295|5634|1973|20603|1015295|5634|1973|18|31|2086|3|93.69|4.68|98.37|50.71|72.27|26.23|64.76|2.70|127.66| +2452228|59745|13409|47063|493093|2676|5459|47063|493093|2676|5459|49|11|2089|21|3228.33|161.41|3389.74|75.09|1277.01|1033.06|658.58|1536.69|1513.51| +2452246|55563|11203|36878|861971|7064|44557|36878|861971|7064|44557|35|10|2089|14|676.90|47.38|724.28|81.72|104.58|358.75|3.18|314.97|233.68| +2452792|70711|8833|38830|954598|2366|16534|38830|954598|2366|16534|48|15|2090|39|2473.77|98.95|2572.72|6.00|717.99|890.55|1124.08|459.14|822.94| +2452751|74415|11577|8810|1329377|1144|13758|8810|1329377|1144|13758|3|29|2090|16|697.12|62.74|759.86|2.46|266.56|515.86|179.44|1.82|331.76| +2451894|56442|15686|83483|1506496|3628|20391|83483|1506496|3628|20391|50|10|2091|17|2403.63|72.10|2475.73|37.80|1090.21|2115.19|109.60|178.84|1200.11| +|38735|13816||515402||27611|||130|||33|2095|11|167.20||180.57||117.04|0.00|||146.33| +2451694|26040|10837|99589|431933|3924|37068|99589|431933|3924|37068|56|21|2095|1|75.85|6.06|81.91|57.83|7.74|18.96|43.23|13.66|71.63| +2451590|82258|16670|50281|595905|1176|38447|50281|595905|1176|38447|7|4|2095|21|2202.27|132.13|2334.40|70.83|1637.58|220.22|39.64|1942.41|1840.54| +2451034|80737|1820|56065|914863|3274|25707|56065|914863|3274|25707|32|1|2096|17|2331.72|93.26|2424.98|26.02|1304.24|1375.71|688.32|267.69|1423.52| +2451212|26023|9451|80515|876312|2956|46000|80515|876312|2956|46000|19|33|2097|8|154.88|13.93|168.81|24.32|154.88|61.95|80.84|12.09|193.13| +2452293|72343|37|82744|466434|2924|33414|82744|466434|2924|33414|7|5|2098|7|651.14|45.57|696.71|40.83|176.82|136.73|401.23|113.18|263.22| +2452292|10964|6689|75869|1469205|5898|37426|75869|1469205|5898|37426|45|13|2098|2|18.74|0.93|19.67|55.38|9.36|16.86|0.54|1.34|65.67| +2452432|5173|5985|75869|1469205|5898|37426|75869|1469205|5898|37426|13|9|2098|4|271.52|5.43|276.95|73.64|142.24|92.31|130.82|48.39|221.31| +2451656|34103|17894|21843|707183|2123|37910|21843|707183|2123|37910|20|34|2099|23|782.69|23.48|806.17|26.83|341.09|516.57|45.24|220.88|391.40| +2451116|49173|13918|77367|965205|6643|46881|77367|965205|6643|46881|56|21|2100|60|1822.20|0.00|1822.20|39.88|388.20|965.76|847.87|8.57|428.08| +2451263|67656|15046|81099|635955|6011|18467|81099|635955|6011|18467|46|18|2100|3|8.76|0.78|9.54|40.94|52.59|2.97|4.05|1.74|94.31| +2451123|30804|15565|1668|1600020|748|17808|1668|1600020|748|17808|31|9|2100|12|706.80|42.40|749.20|53.14|345.84|579.57|94.15|33.08|441.38| +2452468|27426|10349|15138|402477|1369|46261|15138|402477|1369|46261|53|25|2102|5|456.65|27.39|484.04|90.24|154.45|77.63|140.23|238.79|272.08| +2451801|45486|15746|32498|920185|2719|20006|32498|920185|2719|20006|26|29|2103|24|343.20|30.88|374.08|99.66|124.80|236.80|30.85|75.55|255.34| +2451762|33063|7730|25538|1178620|5302|9584|25538|1178620|5302|9584|22|20|2103|38|481.84|33.72|515.56|47.94|291.84|77.09|230.70|174.05|373.50| +2451878|41992|14026|7645|518136|980|2224|7645|518136|980|2224|13|25|2103|62|5246.44|52.46|5298.90|87.23|1165.60|4931.65|100.73|214.06|1305.29| +2452760|15261|16518|89961|868347|2286|18672|89961|868347|2286|18672|54|24|2104|7|641.48|32.07|673.55|74.36|19.81|237.34|133.36|270.78|126.24| +2452825|64277|8647|15416|865987|1005|1223|15416|865987|1005|1223|36|6|2104|12|115.80|4.63|120.43|50.30|453.84|99.58|13.30|2.92|508.77| +2452818|60531|14088|98375|626252|4882|8359|98375|626252|4882|8359|24|7|2106|4|23.76|0.00|23.76|97.99|28.48|15.20|3.33|5.23|126.47| +2452082|48579|16325|28618|1180185|3399|45766|28618|1180185|3399|45766|38|20|2107|7|838.95|58.72|897.67|85.60|262.15|33.55|64.43|740.97|406.47| +2451776|21211|17902|88354|822596|2085|48727|88354|822596|2085|48727|43|25|2108|70|1801.80|144.14|1945.94|67.89|1961.40|594.59|338.01|869.20|2173.43| +2451753|76386|5408|60637|1383721|5205|44289|60637|1383721|5205|44289|8|28|2108|32|3866.88|270.68|4137.56|81.58|1076.80|2320.12|30.93|1515.83|1429.06| +2451654|24752|2954|98988|1609215|5842|31669|98988|1609215|5842|31669|31|23|2108|49|946.68|66.26|1012.94|69.50|401.80|463.87|53.10|429.71|537.56| +2452491|80782|17973|20591|1223664|1937|36657|20591|1223664|1937|36657|57|2|2109|31|109.43|3.28|112.71|73.73|506.85|55.80|4.82|48.81|583.86| +2452469|79096|13365|61942|664030|5318|24606|61942|664030|5318|24606|19|24|2109|55|3896.75|350.70|4247.45|36.92|0.00|1402.83|548.66|1945.26|387.62| +2452478|12771|9815|48811|311705|3752|31446|48811|311705|3752|31446|27|26|2109|40|2349.60|93.98|2443.58|84.95|3356.80|516.91|384.86|1447.83|3535.73| +2451625|79995|6032|62820|1139565|2263|19008|62820|1139565|2263|19008|1|4|2112|92|9347.20|280.41|9627.61|4.08|5608.32|6636.51|108.42|2602.27|5892.81| +2451698|43168|9529|67898|260243|2188|22133|67898|260243|2188|22133|49|10|2112|38|888.06|0.00|888.06|77.64|213.56|257.53|88.27|542.26|291.20| +2451669|9597|17570|24724|890880|4589|46298|24724|890880|4589|46298|8|8|2112|32|1569.60|109.87|1679.47|47.42|669.76|1444.03|75.34|50.23|827.05| +2451709|57761|4616|89038|409853|7115|41577|89038|409853|7115|41577|14|20|2112|47|1674.14|100.44|1774.58|32.13|797.12|318.08|1356.06|0.00|929.69| +2452687|76592|14209|65484|278007|2977|47100|65484|278007|2977|47100|15|14|2113|21|376.32|30.10|406.42|71.31|752.85|180.63|156.55|39.14|854.26| +2452830|72524|14904|84524|1393944|2564|32337|84524|1393944|2564|32337|33|22|2113|57|1794.36|53.83|1848.19|18.52|288.42|897.18|556.25|340.93|360.77| +2452464||15841||296929|4496|36063|69362|296929|4496||||2114|25|558.50|0.00|558.50|51.86|113.25||260.48||| +2452417|35019|8761|92894|24069|491|37713|92894|24069|491|37713|47|9|2114|19|1348.81|94.41|1443.22|93.40|575.89|957.65|289.45|101.71|763.70| +2451157|63634|12319|80407|1098011|101|38245|80407|1098011|101|38245|2|13|2115|28|1562.68|31.25|1593.93|76.15|1070.72|1453.29|98.45|10.94|1178.12| +2451331|34934|3049|27360|831878|5643|31808|27360|831878|5643|31808|14|19|2117|39|3997.50|79.95|4077.45|24.34|2112.63|1319.17|1981.96|696.37|2216.92| +2451367|37016|11932|64691|1219388|1638|15488|64691|1219388|1638|15488|8|33|2117|26|231.14|0.00|231.14|37.76|114.14|173.35|2.31|55.48|151.90| +2451303|51801|11755|21839|269470|3148|6426|21839|269470|3148|6426|28|10|2117|99|1939.41|135.75|2075.16|8.21|3036.33|678.79|252.12|1008.50|3180.29| +2452074|16610|439|99742|1662139|2583|30580|99742|1662139|2583|30580|13|10|2118|17|1960.44|156.83|2117.27|72.32|749.53|999.82|451.49|509.13|978.68| +2452162|27534|16055|60173|1082058|6790|41075|60173|1082058|6790|41075|38|24|2118|7|30.24|1.51|31.75|82.47|35.63|6.65|8.49|15.10|119.61| +2452681|17926|16401|80900|1518725|103|46681|80900|1518725|103|46681|15|33|2119|41|517.01|5.17|522.18|94.61|273.47|0.00|113.74|403.27|373.25| +2451692|46562|3452|75334|29067|753|8086|75334|29067|753|8086|8|20|2120|12|289.56|14.47|304.03|91.10|55.32|98.45|63.06|128.05|160.89| +2451786|76608|3323|40009|643666|923|26627|40009|643666|923|26627|38|28|2120|34|571.54|40.00|611.54|97.01|433.84|91.44|340.87|139.23|570.85| +2451725|47127|14360|50987|8871|708|6145|50987|8871|708|6145|44|31|2121|1|154.77|13.92|168.69|89.83|92.86|125.36|6.76|22.65|196.61| +2452530|43833|14487|62524|855266|361|31856|62524|855266|361|31856|33|2|2123|25|675.00|33.75|708.75|41.50|1548.75|128.25|289.77|256.98|1624.00| +2452224|75428|8913|75400|723359|6653|28768|75400|723359|6653|28768|45|9|2124|2|20.66|1.85|22.51|92.96|6.08|18.80|0.39|1.47|100.89| +2451205|43582|15698|86051||2661|8519|86051||2661|8519|55|17|2125||18.72|0.37||92.90|||7.31|4.30|195.09| +2451947|57581|11432|41009|1388791|4220|49602|41009|1388791|4220|49602|49|31|2126|6|382.32|26.76|409.08|70.34|96.66|237.03|52.30|92.99|193.76| +2452631|72673|3333|68773|1115565|757|19502|68773|1115565|757|19502|43|28|2127|12|500.16|10.00|510.16|51.98|460.68|240.07|249.68|10.41|522.66| +2452515|80109|17451|1538|1674080|7069|44858|1538|1674080|7069|44858|25|13|2127|48|3079.20|215.54|3294.74|68.39|384.48|2863.65|150.88|64.67|668.41| +2452533|30007|1851|30439|6278|2117|11509|30439|6278|2117|11509|9|9|2127|22|12.32|0.86|13.18|1.80|36.96|11.08|0.03|1.21|39.62| +2452756|45438|8484|72539|484097|902|46507|72539|484097|902|46507|9|13|2128|6|153.96|9.23|163.19|34.62|39.84|144.72|7.48|1.76|83.69| +2451745|45525|10165|66790|1291925|5413|37348|66790|1291925|5413|37348|8|1|2130|2|107.24|7.50|114.74|78.25|104.34|21.44|80.65|5.15|190.09| +||5197|89400|1410375|6103|33878|89400|1410375||33878|44|24|2131|38||147.15|2599.67||214.32|1030.05||725.46|| +2451013|80744|16579|85334|661693|5605|48317|85334|661693|5605|48317|38|33|2132|42|2239.02|201.51|2440.53|21.17|1403.22|1656.87|564.68|17.47|1625.90| +2451032|68804|2996|94249|376279|7109|39939|94249|376279|7109|39939|10|34|2132|25|402.25|28.15|430.40|20.15|25.00|189.05|38.37|174.83|73.30| +2451130|37425|9374|43167|807821|5275|3929|43167|807821|5275|3929|19|16|2132|1|40.87|2.45|43.32|87.90|16.97|11.44|18.24|11.19|107.32| +2452726|73954|12189|88440|109232|3177|9169|88440|109232|3177|9169|25|17|2133|34|3214.02|160.70|3374.72|50.67|0.00|96.42|1745.85|1371.75|211.37| +2451528|53058|17072|58918|1523818|1871|7124|58918|1523818|1871|7124|10|11|2134|27|1800.09|36.00|1836.09|90.57|1199.88|1440.07|147.60|212.42|1326.45| +2451571|14763|10345|44418|1815021|3577|26180|44418|1815021|3577|26180|31|9|2134|16|349.60|27.96|377.56|85.17|341.12|321.63|22.09|5.88|454.25| +2452791|70704|8298|87773|424710|847|1784|87773|424710|847|1784|25|18|2135|17|533.63|5.33|538.96|51.28|76.16|229.46|51.70|252.47|132.77| +||9140|13625||6744||13625|775961|||||2136||3355.02|301.95|3656.97|14.49|1332.85||1949.27||| +2452812|68389|17803|||2277||54722|1727123||30678|54|20|2138|20||||21.82|413.40|2456.16||439.77|610.66| +2451354|50116|7153|83284|25925|3766|48007|83284|25925|3766|48007|32|11|2140|32|21.44|0.64|22.08|35.19|150.08|11.57|6.51|3.36|185.91| +2451400|64255|16813|67589|395923|5410|10944|67589|395923|5410|10944|1|4|2141|71|5366.89|0.00|5366.89|4.59|3328.48|2683.44|590.35|2093.10|3333.07| +2451072|40135|13198|20917|678020|276|16179|20917|678020|276|16179|34|5|2142|67|32.83|2.62|35.45|9.18|1029.12|7.22|25.09|0.52|1040.92| +2451009|9780|788|22054|181469|4437|26252|22054|181469|4437|26252|25|33|2142|2|44.40|1.33|45.73|78.74|0.00|41.73|1.70|0.97|80.07| +2450972|49334|17458|21668|857818|815|19765|21668|857818|815|19765|19|11|2143|26|1902.94|57.08|1960.02|19.14|710.84|1236.91|119.88|546.15|787.06| +2450991|5418|5347|48079|615806|2286|26366|48079|615806|2286|26366|58|7|2143|40|660.00|19.80|679.80|14.08|215.20|184.80|47.52|427.68|249.08| +2451033|74398|15517|86157|1785376|1842|6661|86157|1785376|1842|6661|20|31|2143|43|2967.00|89.01|3056.01|21.14|1374.28|860.43|400.24|1706.33|1484.43| +2452434|69666|12747|6023|1711242|6061|46080|6023|1711242|6061|46080|45|6|2144|11|682.11|40.92|723.03|15.08|10.23|334.23|333.96|13.92|66.23| +2451522|67261|4780|8348|1301102|6656|44610|8348|1301102|6656|44610|50|22|2147|8|491.60|9.83|501.43|98.35|206.40|176.97|289.45|25.18|314.58| +2451642|35734|12806|82549|1727628|3907|30986|82549|1727628|3907|30986|25|27|2147|82|307.50|3.07|310.57|45.18|104.14|221.40|43.91|42.19|152.39| +2451673|58665|11168|5743|1334862|4781|35900|5743|1334862|4781|35900|19|19|2147|14|195.30|9.76|205.06|35.85|238.70|9.76|9.27|176.27|284.31| +2452944|60599|9465|60906|519326|146|2566|60906|519326|146|2566|55|29|2148|21|1095.15|76.66|1171.81|26.40|484.89|43.80|809.53|241.82|587.95| +|63712|8310|60521|||10125||238091|||45|15|2148||12.60|0.88||80.90||3.15|6.14|3.31|| +2452763|74770|11982|94290|1407804|6565|31679|94290|1407804|6565|31679|7|10|2148|8|19.20|0.57|19.77|94.75|236.96|13.05|3.38|2.77|332.28| +2452856|79580|12045|81113|215574|7188|29202|81113|215574|7188|29202|36|33|2148|16|154.56|6.18|160.74|36.88|58.72|38.64|44.04|71.88|101.78| +2452818|62695|15781|18457|397689|3674|18459|18457|397689|3674|18459|57|30|2148|13|1301.69|26.03|1327.72|26.07|1105.26|234.30|544.36|523.03|1157.36| +2451310|61874|11596|6520|1100669|5218|24167|6520|1100669|5218|24167|32|8|2150|45|1002.60|20.05|1022.65|97.73|769.95|150.39|42.61|809.60|887.73| +2451290|30290|16849|52587|1491314|4888|6862|52587|1491314|4888|6862|2|29|2150|15|99.15|8.92|108.07|11.75|7.80|85.26|5.13|8.76|28.47| +2451396|48030|5510|48137|1445018|3852|26286|48137|1445018|3852|26286|7|5|2151|84|6226.08|124.52|6350.60|10.34|494.76|2428.17|1974.91|1823.00|629.62| +2451454|45853|13975|52556|85450|2344|30038|52556|85450|2344|30038|56|28|2151|6|854.76|68.38|923.14|11.91|207.54|812.02|26.92|15.82|287.83| +||13376|48137|1445018|||48137|1445018||26286|14||2151|3|428.07|||68.35|210.72|393.82|31.51||| +2452344|48990|12149|19320|1502290|2860|8677|19320|1502290|2860|8677|11|34|2154|27|250.02|17.50|267.52|75.58|189.00|2.50|49.50|198.02|282.08| +2452652|80182|8538|95343|420765|6634|33656|95343|420765|6634|33656|42|28|2156|26|2505.36|175.37|2680.73|23.94|1176.76|876.87|1025.94|602.55|1376.07| +2451801|73091|12560|47241|752167|6693|18419|47241|752167|6693|18419|29|34|2157|32|468.80|4.68|473.48|82.43|441.28|135.95|183.06|149.79|528.39| +2451733|81701|2551|91199|1412304|3809|16914|91199|1412304|3809|16914|26|16|2157|39|2234.70|134.08|2368.78|16.03|774.54|44.69|897.90|1292.11|924.65| +2451610||14228|31236||||31236|1244152|4161||1||2158|25|||3321.81||||344.79|239.61|| +2452520|57213|4437|43785|98616|3047|11078|43785|98616|3047|11078|13|33|2161|16|119.36|8.35|127.71|58.73|127.84|23.87|56.33|39.16|194.92| +2451337|68060|12868|82245|1432157|6069|44381|82245|1432157|6069|44381|44|33|2162|41|947.51|85.27|1032.78|3.23|2084.85|293.72|19.61|634.18|2173.35| +2452118|65549|7375|4239|288851|4663|1675|4239|288851|4663|1675|7|34|2163|6|0.12|0.00|0.12|83.92|7.92|0.11|0.00|0.01|91.84| +2452171|76854|8155|36530|1033617|5104|40944|36530|1033617|5104|40944|59|24|2163|17|487.39|0.00|487.39|83.55|318.58|297.30|152.07|38.02|402.13| +2451881|61028|5489|46852|1342052|99|33346|46852|1342052|99|33346|47|29|2164|38|493.62|19.74|513.36|31.56|1481.24|138.21|78.19|277.22|1532.54| +2451768|66197|11879|80363|500833|5656|11679|80363|500833|5656|11679|50|13|2164|2|31.44|0.00|31.44|8.30|30.58|0.94|6.40|24.10|38.88| +2451980|24885|13268|92986|1247818|6650|43482|92986|1247818|6650|43482|26|24|2164|8|656.32|0.00|656.32|7.16|492.24|374.10|93.13|189.09|499.40| +||12889|96159|||23454||1769124|489||||2164|8|||74.11||29.12|8.39|33.84||| +2450989|42626|17197|81705|32170|5835|32347|81705|32170|5835|32347|14|19|2167|83|8661.88|259.85|8921.73|14.89|885.61|259.85|5209.25|3192.78|1160.35| +2451003|37793|7006|468|1426547|626|15505|468|1426547|626|15505|52|13|2167|30|5838.30|408.68|6246.98|12.76|2250.90|1284.42|2049.24|2504.64|2672.34| +2451467|50746|13972|95618|72349|280|38211|95618|72349|280|38211|49|27|2168|34|1289.62|116.06|1405.68|74.59|24.14|889.83|319.83|79.96|214.79| +2451397|65049|10177|47609|395819|4829|37839|47609|395819|4829|37839|1|28|2168|48|732.48|29.29|761.77|41.90|121.92|395.53|47.17|289.78|193.11| +2452842|50354|1392|2412|398456|547|40591|2412|398456|547|40591|15|9|2169|24|1220.16|24.40|1244.56|74.77|162.48|890.71|125.19|204.26|261.65| +2452405|11497|12723|26880|900515|1350|5385|26880|900515|1350|5385|17|2|2170|4|37.00|0.37|37.37|65.11|8.52|17.02|9.39|10.59|74.00| +2452740|50847|10209|12147|48818|4242|34913|12147|48818|4242|34913|19|30|2171|3|36.90|1.84|38.74|45.45|43.35|28.78|6.98|1.14|90.64| +2451916|78169|11282|99681|657992|1550|46661|99681|657992|1550|46661|50|25|2173|30|1162.80|104.65|1267.45|59.19|156.90|488.37|161.86|512.57|320.74| +2452130|25808|14267|55036|1093109|4429|44502|55036|1093109|4429|44502|49|2|2173|6|64.74|2.58|67.32|89.38|1.08|11.65|51.49|1.60|93.04| +2451212|78975|4588|49207|20172|2967|12121|49207|20172|2967|12121|55|12|2175|3|143.58|4.30|147.88|48.48|73.53|103.37|12.06|28.15|126.31| +2451323|59569|16226|5162|1783858|2107|34803|5162|1783858|2107|34803|46|21|2175|71|861.23|43.06|904.29|9.99|597.82|17.22|607.68|236.33|650.87| +2452295|60354|1905|82153|439474|5948|37269|82153|439474|5948|37269|23|28|2176|46|6638.72|199.16|6837.88|96.18|265.42|597.48|5557.94|483.30|560.76| +||2421|96786|||6682|96786||3799|6682||32|2176||||277.85|60.55|963.90|||175.96|| +2451289|49335|17516|87241|1370606|7124|6489|87241|1370606|7124|6489|16|35|2178|7|383.88|11.51|395.39|37.90|267.82|299.42|62.50|21.96|317.23| +2452121|8415|12251|52069|1620996|676|23603|52069|1620996|676|23603|5|9|2179|27|621.54|55.93|677.47|26.05|621.54|111.87|315.99|193.68|703.52| +2451052|70966|2972|32123|770256|3142|18428|32123|770256|3142|18428|50|10|2181|55|6219.95|124.39|6344.34|93.05|3431.45|0.00|3047.77|3172.18|3648.89| +2451298|70224|14065|26509|428366|176|7233|26509|428366|176|7233|49|14|2182|54|2982.42|268.41|3250.83|92.48|1924.02|2266.63|236.21|479.58|2284.91| +2451159||14624||||13850|3673|1503332|2991||10|12|2182||340.64|||33.71|113.52|146.47|||| +2451188|38669|4880|24888|1060737|5386|23635|24888|1060737|5386|23635|43|3|2182|3|67.95|4.07|72.02|53.33|77.67|59.79|0.73|7.43|135.07| +||10060|9520|1249550|1227|44457||1249550|1227|44457|||2184||181.46||181.46||12.30|||19.51|| +2451236|64093|7366|20737|288926|5562|25751|20737|288926|5562|25751|52|4|2184|56|3009.44|90.28|3099.72|13.80|1218.00|2016.32|585.94|407.18|1322.08| +2451548|31492|12650|67821|1587299|1530|6608|67821|1587299|1530|6608|31|35|2185|25|596.25|35.77|632.02|26.12|389.25|232.53|225.50|138.22|451.14| +2452064|37904|6908|9492|104889|4286|14530|9492|104889|4286|14530|23|22|2187|31|657.20|13.14|670.34|88.85|699.36|420.60|182.18|54.42|801.35| +|65034|8807||||13466||1567011||13466|25|2|2188|47||||92.07|354.85||201.11|603.34|725.38| +2451343|60012|2302|16454|1898075|496|8117|16454|1898075|496|8117|37|18|2190|9|535.86|37.51|573.37|67.12|82.44|53.58|361.71|120.57|187.07| +2451439|60553|4568|88567|1891294|6021|28395|88567|1891294|6021|28395|43|17|2190|14|1105.44|77.38|1182.82|29.45|659.68|243.19|500.10|362.15|766.51| +2451242|48655|15409|30441|33447|2053|13059|30441|33447|2053|13059|10|17|2190|4|8.84|0.61|9.45|96.99|0.08|6.18|1.09|1.57|97.68| +2451304|23841|11330|39311|1910421|731|14641|39311|1910421|731|14641|7|9|2191|1|16.44|0.49|16.93|58.03|2.24|3.12|8.79|4.53|60.76| +2452727|54016|7569|79502|586926|5550|27774|79502|586926|5550|27774|25|16|2192|26|1123.20|89.85|1213.05|38.96|302.64|572.83|313.71|236.66|431.45| +2452324|23545|4189|80331|703610|2089|43721|80331|703610|2089|43721|7|33|2193|2|56.16|5.05|61.21|41.17|15.02|10.67|10.91|34.58|61.24| +2452460|70850|9659|75429|1659861|1093|32278|75429|1659861|1093|32278|11|16|2193|32|999.36|49.96|1049.32|11.64|742.08|159.89|444.91|394.56|803.68| +2452213|64198|6519|3120|846701|1482|29510|3120|846701|1482|29510|59|10|2194|39|1163.37|0.00|1163.37|79.90|329.55|930.69|2.32|230.36|409.45| +2452381|26324|14477|90857|682960|4663|49966|90857|682960|4663|49966|33|18|2195|5|316.55|0.00|316.55|82.07|115.65|79.13|90.21|147.21|197.72| +2452514|4720|739|24182|1167598|2214|4430|24182|1167598|2214|4430|21|3|2195|25|2606.50|26.06|2632.56|98.92|684.50|2371.91|7.03|227.56|809.48| +2451659|82528|1939|97938|1166701|2664|38227|97938|1166701|2664|38227|52|20|2196|26|868.14|43.40|911.54|63.48|192.92|121.53|149.32|597.29|299.80| +2451682|14553|11680|35115|1243217|6072|6974|35115|1243217|6072|6974|10|13|2196|14|124.74|6.23|130.97|4.90|142.66|12.47|61.74|50.53|153.79| +2452800|84149|13908|56510|1852556|249|20295|56510|1852556|249|20295|13|2|2197|34|26.86|1.34|28.20|15.10|96.22|14.50|10.75|1.61|112.66| +2452827|76675|15597|12014|625487|5007|34080|12014|625487|5007|34080|42|2|2197|35|432.25|17.29|449.54|79.23|339.85|229.09|8.12|195.04|436.37| +2451282|43446|8938|65137|153620|4503|15386|65137|153620|4503|15386|40|15|2199|5|19.30|0.19|19.49|56.81|84.30|12.15|5.93|1.22|141.30| +2451246|30100|4348|66081|264066|451|37636|66081|264066|451|37636|40|7|2199|89|778.75|62.30|841.05|20.12|181.56|311.50|224.28|242.97|263.98| +2451110|38215|17726|38374|95487|7000|11313|38374|95487|7000|11313|55|15|2199|35|2258.55|158.09|2416.64|86.72|120.40|1716.49|97.57|444.49|365.21| +|43949|1694|87303||1684||87303|1649917|1684|||26|2202|||3.12|65.64|||26.25|7.97||126.41| +2452593|45022|8319|98837|273141|1845|45396|98837|273141|1845|45396|51|32|2204|21|120.54|10.84|131.38|30.96|87.78|74.73|19.69|26.12|129.58| +2452515|14051|3888|32323|861733|7143|39306|32323|861733|7143|39306|30|12|2204|23|1038.45|41.53|1079.98|74.37|357.42|768.45|99.90|170.10|473.32| +2452558|747|15819|69015|945998|581|14011|69015|945998|581|14011|19|13|2204|7|133.98|4.01|137.99|65.95|30.17|14.73|81.09|38.16|100.13| +2452664|37119|12582|18173|701696|5219|47095|18173|701696|5219|47095|49|19|2206|24|1332.24|79.93|1412.17|5.92|222.00|293.09|789.75|249.40|307.85| +|46166|8096|98890|||||1816302|4114|32326||3|2208||592.50||592.50||||||461.61| +2451134|55853|376|59645|750228|11|15747|59645|750228|11|15747|46|16|2208|1|40.77|2.03|42.80|38.22|1.11|16.30|3.67|20.80|41.36| +2452551|61910|2802|880|334083|4086|11289|880|334083|4086|11289|9|6|2209|3|605.13|42.35|647.48|52.17|249.93|163.38|53.01|388.74|344.45| +2452416|45416|2303|65704|1569487|1717|17500|65704|1569487|1717|17500|35|23|2210|1|8.70|0.52|9.22|91.84|46.41|3.13|3.73|1.84|138.77| +2452474|5702|12583|71134|1480360|2833|14859|71134|1480360|2833|14859|47|14|2210|17|715.53|42.93|758.46|50.43|370.94|214.65|105.18|395.70|464.30| +2451335|73056|14533|2651|33359|67|23575|2651|33359|67|23575|43|31|2211|6|737.76|7.37|745.13|12.24|167.04|604.96|69.05|63.75|186.65| +2451040|39658|12188|71957|340941|1125|9309|71957|340941|1125|9309|32|3|2212|51|132.09|3.96|136.05|25.36|3700.56|35.66|62.67|33.76|3729.88| +2451086|45568|15547|75211|1265700|4189|34278|75211|1265700|4189|34278|28|14|2212|73|2134.52|149.41|2283.93|30.73|1381.16|1728.96|121.66|283.90|1561.30| +2452329|39704|1725|38687|213205|2923|9441|38687|213205|2923|9441|53|7|2213|1|13.65|0.95|14.60|35.98|8.03|6.14|3.00|4.51|44.96| +2452194|33897|15689|86828|1706369|2263|24957|86828|1706369|2263|24957|13|7|2213|8|397.76|19.88|417.64|3.39|157.92|194.90|24.34|178.52|181.19| +2451663||14167|||4528|4248|87935|||4248|||2214|||81.49||||465.68||642.64|| +2451447|57106|9388|88448|238239|2783|18527|88448|238239|2783|18527|34|3|2216|36|1953.00|136.71|2089.71|59.34|2394.36|1249.92|421.84|281.24|2590.41| +2451359|36147|3850|97239|648363|4314|35658|97239|648363|4314|35658|52|35|2217|50|6577.00|591.93|7168.93|89.39|1006.50|1973.10|4465.78|138.12|1687.82| +2451389|4767|14666|58678|1312539|6753|4214|58678|1312539|6753|4214|56|30|2217|23|166.29|3.32|169.61|20.93|48.99|0.00|36.58|129.71|73.24| +2452584|70421|9211|49308|1159555|5914|18393|49308|1159555|5914|18393|42|5|2218|1|33.01|1.32|34.33|82.21|18.57|15.51|6.82|10.68|102.10| +2452396|57605|5988|41595|1843047|5652|7904|41595|1843047|5652|7904|13|7|2218|12|71.04|0.00|71.04|17.00|104.28|39.78|13.12|18.14|121.28| +2452675|2150|12847|18229|79655|5881|15660|18229|79655|5881|15660|49|7|2219|58|264.48|21.15|285.63|61.74|6.96|42.31|59.98|162.19|89.85| +2452646|37160|3654|45839|367273|1154|1435|45839|367273|1154|1435|21|19|2219|1|41.54|0.83|42.37|56.89|46.73|18.27|21.87|1.40|104.45| +2450985|4677|4774|24996|245857|318|5579|24996|245857|318|5579|32|34|2220|7|1221.64|97.73|1319.37|13.31|298.90|732.98|146.59|342.07|409.94| +2451191||10054|98047||||98047|266290|||46|34|2220|||96.66|2513.38|76.28|1118.60||677.89||1291.54| +2450934|46368|5222|15276|1331825|5252|30304|15276|1331825|5252|30304|7|33|2220|20|474.80|4.74|479.54|98.39|431.60|308.62|99.70|66.48|534.73| +2452268|4443|6273|73221|1733962|2703|13444|73221|1733962|2703|13444|53|21|2221|48|3396.96|203.81|3600.77|91.76|1970.88|1120.99|2025.61|250.36|2266.45| +2452320|76908|5931|73495|828672|296|38320|73495|828672|296|38320|9|8|2221|21|83.58|5.85|89.43|50.24|55.65|51.81|17.79|13.98|111.74| +|40581|11389||||39091||1856521|||||2221||662.22|0.00|||122.46|509.90||144.71|174.03| +2450962|26782|14371|7670|1850697|2971|10398|7670|1850697|2971|10398|25|29|2223|14|312.48|3.12|315.60|81.22|406.14|62.49|114.99|135.00|490.48| +2452370|34140|6413|85723|389906|5525|13826|85723|389906|5525|13826|45|14|2224|3|4.50|0.27|4.77|91.32|6.00|0.27|2.15|2.08|97.59| +2452384|47154|14683|85723|389906|5525|13826|85723|389906|5525|13826|59|30|2224|3|14.79|0.29|15.08|21.92|172.59|4.43|1.55|8.81|194.80| +2451387|79954|10537|48183|1056584|185|7386|48183|1056584|185|7386|14|9|2228|24|972.72|87.54|1060.26|25.77|185.28|466.90|25.29|480.53|298.59| +2452756|16048|5449|4442|1747704|2705|7329|4442|1747704|2705|7329|57|14|2229|10|479.50|14.38|493.88|32.64|309.00|441.14|21.86|16.50|356.02| +2452787|78995|13267|80394|1335367|6693|40066|80394|1335367|6693|40066|3|1|2229|21|348.81|3.48|352.29|90.11|424.20|38.36|74.50|235.95|517.79| +2452703|75849|2358|44500|773185|2317|14202|44500|773185|2317|14202|12|1|2229|32|4.48|0.35|4.83|71.11|12.16|2.46|0.52|1.50|83.62| +2451436|64045|16010|78969|1603019|1469|12672|78969|1603019|1469|12672|7|13|2230|22|1657.04|149.13|1806.17|49.92|223.30|695.95|720.81|240.28|422.35| +2452625|33158|9139|21856|715562|2868|12174|21856|715562|2868|12174|15|20|2233|49|7596.47|607.71|8204.18|22.18|3632.86|531.75|70.64|6994.08|4262.75| +2452453|35928|14821|68639|1018581|2476|33218|68639|1018581|2476|33218|19|12|2233|26|1245.40|74.72|1320.12|92.29|595.66|1008.77|37.86|198.77|762.67| +2452453|71910|2845|57232|231280|6128|23376|57232|231280|6128|23376|54|21|2233|65|11352.90|681.17|12034.07|18.84|0.00|3065.28|1243.14|7044.48|700.01| +2451664|73985|16019|29343|1496197|5451|39905|29343|1496197|5451|39905|59|34|2234|35|0.00|0.00|0.00|19.83|86.80|0.00|0.00|0.00|106.63| +||10862|67321|406871||17924|67321|406871|4374|17924|11||2234||939.00|||||||471.01|| +2451794|61142|2690|35618|958774|4330|37086|35618|958774|4330|37086|37|12|2234|17|660.11|0.00|660.11|47.57|1138.15|541.29|46.33|72.49|1185.72| +2451802|30888|11591|21874|529976|2525|44627|21874|529976|2525|44627|26|8|2234|13|46.41|0.00|46.41|85.65|0.00|33.87|11.28|1.26|85.65| +2451223|21827|16402|30467|1670104|1388|2672|30467|1670104|1388|2672|46|35|2235|12|449.52|17.98|467.50|35.36|663.60|418.05|25.80|5.67|716.94| +2451200||15430||1493136|724|40312||||40312|26|20|2235|||22.90||||||350.43|| +2452414|66527|9685|34259|1179923|6740|21204|34259|1179923|6740|21204|39|24|2238|34|233.92|14.03|247.95|31.05|34.00|88.88|87.02|58.02|79.08| +2451783|35009|14909|63734|1371155|2129|47503|63734|1371155|2129|47503|47|7|2239|22|920.04|73.60|993.64|99.54|173.14|239.21|238.29|442.54|346.28| +2451722|54200|2074|15643|1184770|297|24224|15643|1184770|297|24224|26|4|2240|59|2611.93|104.47|2716.40|56.29|1439.01|261.19|963.80|1386.94|1599.77| +2451841|53809|3794|45932|962082|4632|20782|45932|962082|4632|20782|22|5|2240|4|299.68|17.98|317.66|73.41|125.44|8.99|104.64|186.05|216.83| +2451544|58842|16984|47566|151083|5800|34164|47566|151083|5800|34164|26|3|2240|1|42.07|0.84|42.91|36.39|33.37|27.76|3.43|10.88|70.60| +2451965|21367|8075|13972|94235|3745|27255|13972|94235|3745|27255|17|3|2244|11|156.20|6.24|162.44|32.80|160.82|156.20|0.00|0.00|199.86| +2451256|40333|14743|43105|1208188|1624|17907|43105|1208188|1624|17907|4|6|2245|1|30.20|1.81|32.01|89.23|53.83|12.38|0.71|17.11|144.87| +2451239|77785|13942|33026|1553984|6589|12794|33026|1553984|6589|12794|56|7|2245|3|424.47|33.95|458.42|96.49|199.74|12.73|148.22|263.52|330.18| +2451311|66884|1552|39056|489698|4787|41829|39056|489698|4787|41829|40|4|2245|13|54.21|0.00|54.21|38.60|217.10|8.67|11.38|34.16|255.70| +2451271|5528|9049|71066|677624|1335|34646|71066|677624|1335|34646|52|1|2246|26|43.42|3.03|46.45|95.51|286.26|3.47|2.79|37.16|384.80| +2451252|31015|7261|||805|||492611|805||49|18|2246|||269.62||75.58|461.70|1540.72|1779.54||806.90| +2451730|39826|8288|55458|1791250|6242|42881|55458|1791250|6242|42881|22|32|2248|22|2455.42|171.87|2627.29|77.04|946.66|1104.93|108.03|1242.46|1195.57| +2451170|19454|8524|75941|1639767|1193|3419|75941|1639767|1193|3419|25|25|2249|14|374.08|18.70|392.78|79.80|259.00|340.41|8.75|24.92|357.50| +2451907|55668|12395|78643|691014|5785|36556|78643|691014|5785|36556|23|1|2250|7|803.95|16.07|820.02|55.04|543.48|176.86|175.58|451.51|614.59| +2452273|5062|6009|40954|847264|1127|40637|40954|847264|1127|40637|3|13|2253|35|1709.05|0.00|1709.05|7.48|1407.35|1555.23|43.06|110.76|1414.83| +2452287|17357|16743|52795|1143574|7153|26732|52795|1143574|7153|26732|57|13|2254|69|13114.83|393.44|13508.27|7.71|3672.18|3147.55|8472.18|1495.10|4073.33| +2451887|22358|12997|81938|615707|4323|13292|81938|615707|4323|13292|25|24|2255|22|342.54|13.70|356.24|64.19|126.50|116.46|24.86|201.22|204.39| +2451929|71529|403|97394|1714855|3963|45824|97394|1714855|3963|45824|20|19|2255|20|260.20|23.41|283.61|1.51|81.80|226.37|25.37|8.46|106.72| +2451696|12978|9829|25603|94984|4012|24993|25603|94984|4012|24993|19|16|2256|58|3826.84|38.26|3865.10|30.65|944.24|2257.83|141.21|1427.80|1013.15| +2451937|8872|11785|97958|541674|1743|46659|97958|541674|1743|46659|5|11|2259|1|59.86|0.59|60.45|15.46|75.83|20.95|12.45|26.46|91.88| +2451944|34060|781|34400|834280|1360|30793|34400|834280|1360|30793|44|1|2259|8|40.64|2.84|43.48|15.88|10.56|35.35|3.49|1.80|29.28| +2451856|36760|10865|98664|1776398|5332|47466|98664|1776398|5332|47466|26|16|2259|82|5194.70|103.89|5298.59|51.31|576.46|1766.19|205.71|3222.80|731.66| +2451908|78863|16763|99738|582859|5627|21148|99738|582859|5627|21148|59|5|2259|18|32.76|0.00|32.76|9.26|4.68|15.06|17.70|0.00|13.94| +||2156||809187||||809187|||||2261||||191.87|||124.36|3.19||368.58| +|67165|10327|52859|436981|3950|40072||436981|3950||||2262|||60.58|1070.38|20.39|315.70||434.92||396.67| +2452474|61830|7062|63573|1092112|298|2884|63573|1092112|298|2884|57|11|2263|16|896.80|17.93|914.73|71.15|643.04|197.29|538.62|160.89|732.12| +2452517|3392|2304|63618|193574|4330|9933|63618|193574|4330|9933|37|9|2263|10|27.10|1.08|28.18|97.47|36.10|4.06|11.75|11.29|134.65| +2452465|45616|11187|98785|188295|2605|15715|98785|188295|2605|15715|12|27|2263|7|855.19|17.10|872.29|5.35|427.56|795.32|59.27|0.60|450.01| +2452467|47672|12563|18047|64943|6592|22688|18047|64943|6592|22688|31|1|2264|48|768.00|23.04|791.04|12.17|312.96|0.00|107.52|660.48|348.17| +2450897|23205|2518|80779|1179736|5893|30003|80779|1179736|5893|30003|55|21|2265|56|2770.32|193.92|2964.24|17.06|102.48|1662.19|487.57|620.56|313.46| +2450998|34395|13246|46151|1702535|2919|56|46151|1702535|2919|56|34|13|2266|7|795.06|39.75|834.81|23.41|225.26|159.01|559.72|76.33|288.42| +2451115|68252|8995|32009|821652|2048|16044|32009|821652|2048|16044|56|27|2266|16|7.04|0.00|7.04|91.87|71.04|5.13|1.20|0.71|162.91| +2452800|47195|3327|9114|674925|4289|18100|9114|674925|4289|18100|57|15|2267|53|969.90|0.00|969.90|15.33|2061.17|581.94|174.58|213.38|2076.50| +2451293|24518|5329|37714|1893570|1073|18576|37714|1893570|1073|18576|34|34|2268|1|72.39|5.79|78.18|99.81|49.92|33.29|3.91|35.19|155.52| +2451141|68764|1681|88764|670832|5347|41015|88764|670832|5347|41015|2|19|2268|74|420.32|4.20|424.52|82.17|1419.32|113.48|285.36|21.48|1505.69| +2452759|74990|16956|9207|1631805|2050|43495|9207|1631805|2050|43495|55|21|2269|84|11198.04|559.90|11757.94|23.49|2874.48|8734.47|1527.41|936.16|3457.87| +2452166|70527|9275|41434|1900717|1247|49730|41434|1900717|1247|49730|59|1|2270|33|3822.06|152.88|3974.94|60.28|238.59|1146.61|963.16|1712.29|451.75| +|72080|9457|36313||6811||36313||6811||41|22|2270|29|1558.75|109.11|1667.86|95.28|566.66|966.42||491.64|| +2452041|79588|4801|53388|253234|4280|36879|53388|253234|4280|36879|14|35|2270|35|793.80|71.44|865.24|94.14|337.40|325.45|449.61|18.74|502.98| +2452479|45756|13155|60933|1589754|4528|9349|60933|1589754|4528|9349|31|9|2272|58|1277.16|89.40|1366.56|9.71|1332.84|0.00|1072.81|204.35|1431.95| +2452450|70375|12189|75192|1541203|4014|39978|75192|1541203|4014|39978|43|2|2272|9|131.04|10.48|141.52|67.25|262.17|39.31|33.02|58.71|339.90| +2452750|55225|7069|39568|2285|2602|40166|39568|2285|2602|40166|31|20|2273|30|1234.20|49.36|1283.56|80.81|583.80|802.23|159.82|272.15|713.97| +2452084|33004|803|45985|1538442|2099|34556|45985|1538442|2099|34556|59|31|2274|7|370.23|0.00|370.23|62.14|370.23|55.53|308.40|6.30|432.37| +2451926|34698|12163|1902|1118583|5740|17494|1902|1118583|5740|17494|2|4|2274|10|640.30|44.82|685.12|18.08|9.00|83.23|72.41|484.66|71.90| +2451757|74745|4546|303|92413|2923|24186|303|92413|2923|24186|32|34|2276|3|88.89|0.00|88.89|27.42|42.78|74.66|2.41|11.82|70.20| +2451753|5621|16694|63603|225964|772|7533|63603|225964|772|7533|28|23|2276|8|465.60|27.93|493.53|88.26|113.92|74.49|191.64|199.47|230.11| +2451302|34906|12662|45012|31929|2846|35716|45012|31929|2846|35716|32|12|2278|71|9763.21|488.16|10251.37|16.01|4494.30|5272.13|224.55|4266.53|4998.47| +2451585|78459|8804|88843|901977|1864|32951|88843|901977|1864|32951|56|14|2279|62|13910.32|1112.82|15023.14|88.09|3057.22|10571.84|567.54|2770.94|4258.13| +2452756|57114|10389|470|867985|5547|6262|470|867985|5547|6262|25|1|2280|74|8892.58|177.85|9070.43|96.25|5052.72|5691.25|1088.45|2112.88|5326.82| +2452445|39050|12767|7661|1563611|3914|1864|7661|1563611|3914|1864|29|25|2281|25|268.50|8.05|276.55|87.46|438.25|0.00|24.16|244.34|533.76| +2451589|67490|8752|94103|366017|331|39445|94103|366017|331|39445|20|29|2282|2|88.08|7.04|95.12|76.52|22.96|59.89|7.32|20.87|106.52| +||12281||606290||26353|||401|||6|2283||||||113.16|139.23||188.13|195.78| +2452215|52314|16541|68398|1549998|2445|18445|68398|1549998|2445|18445|27|11|2285|10|58.00|2.32|60.32|72.52|9.60|36.54|13.09|8.37|84.44| +2452149|41075|6509|43384|1728286|1988|16537|43384|1728286|1988|16537|27|16|2285|8|851.84|8.51|860.35|99.84|135.20|732.58|51.28|67.98|243.55| +2451625|53307|116|25471|1763250|5029|13561|25471|1763250|5029|13561|13|8|2286|11|126.28|5.05|131.33|98.94|126.28|63.14|36.62|26.52|230.27| +2451615|60409|1291|26370|1165192|4014|13636|26370|1165192|4014|13636|34|6|2287|11|0.00|0.00|0.00|99.54|73.81|0.00|0.00|0.00|173.35| +2451089|27260|709|48591|784278|4765|30052|48591|784278|4765|30052|25|27|2290|9|286.29|25.76|312.05|95.32|76.77|85.88|164.33|36.08|197.85| +2452529|59801|15456|43055|1712087|7166|38598|43055|1712087|7166|38598|48|11|2291|1|38.43|1.92|40.35|70.92|47.71|7.68|5.22|25.53|120.55| +2452495|83727|13033|99|1706050|4801|44352|99|1706050|4801|44352|13|27|2292|23|49.91|3.99|53.90|4.19|37.72|26.95|11.25|11.71|45.90| +2452564|35702|3717|73496|509325|3107|47336|73496|509325|3107|47336|18|34|2292|8|1042.96|62.57|1105.53|67.31|431.92|490.19|392.46|160.31|561.80| +2451917|47035|596|35597|761378|5912|39693|35597|761378|5912|39693|1|2|2295|2|332.96|23.30|356.26|97.32|100.58|326.30|6.06|0.60|221.20| +2452706|35266|156|42909|267438|2014|22943|42909|267438|2014|22943|24|4|2297|30|1198.50|83.89|1282.39|24.07|1390.20|1066.66|2.63|129.21|1498.16| +2452748|6234|3267|5613|1502583|3200|47017|5613|1502583|3200|47017|30|29|2297|35|308.00|24.64|332.64|90.87|308.00|295.68|11.95|0.37|423.51| +2452165|72210|8541|64672|1759421|1817|39766|64672|1759421|1817|39766|59|1|2298|2|336.92|20.21|357.13|42.82|162.22|141.50|171.96|23.46|225.25| +2452248|27625|4591|68849|1884358|3067|8176|68849|1884358|3067|8176|55|25|2298|6|380.46|3.80|384.26|53.71|106.50|22.82|236.04|121.60|164.01| +2452091|77322|2755|12457|706176|6648|22367|12457|706176|6648|22367|23|15|2298|53|306.87|15.34|322.21|72.83|1928.67|236.28|55.76|14.83|2016.84| +2452158|29746|2517|65843|92761|5726|29844|65843|92761|5726|29844|3|3|2298|7|227.85|2.27|230.12|8.91|175.77|66.07|58.24|103.54|186.95| +2451674|44018|17671|90875|843657|2839|35174|90875|843657|2839|35174|1|32|2299|29|1484.51|118.76|1603.27|63.78|568.40|608.64|621.86|254.01|750.94| +2452347|1782|3679|39212|1842966|320|46427|39212|1842966|320|46427|23|27|2300|1|110.97|2.21|113.18|16.96|14.57|24.41|26.83|59.73|33.74| +||7219|||||68781|||13147|34||2302||1727.16|||||||98.80|| +2451270|18105|8432|81966|938740|667|5041|81966|938740|667|5041|58|8|2302|17|2319.48|139.16|2458.64|42.47|518.33|1484.46|400.80|434.22|699.96| +2451072|10614|14089|14686|860139|1592|46645|14686|860139|1592|46645|58|28|2302|73|511.00|10.22|521.22|44.31|883.30|465.01|17.01|28.98|937.83| +2452428|21081|14103|8573|850069|5165|23461|8573|850069|5165|23461|9|12|2303|14|223.30|0.00|223.30|69.16|873.18|37.96|124.17|61.17|942.34| +2452731|77333|5005|67309|198800|5044|41688|67309|198800|5044|41688|1|29|2306|50|7351.00|73.51|7424.51|55.91|3818.50|2572.85|3965.86|812.29|3947.92| +2452602|66796|8905|89682|1491922|1809|14049|89682|1491922|1809|14049|57|5|2306|11|1164.68|69.88|1234.56|6.55|291.17|58.23|265.54|840.91|367.60| +2452399|9288|4446|60093|824496|899|40455|60093|824496|899|40455|51|19|2307|48|72.96|3.64|76.60|47.71|1021.92|18.24|42.68|12.04|1073.27| +2452894|63984|565|76319|333152|5985|28065|76319|333152|5985|28065|7|22|2308|31|2546.34|203.70|2750.04|83.59|1488.62|2342.63|40.74|162.97|1775.91| +2452778|58701|7837|43986|1368458|6083|18508|43986|1368458|6083|18508|7|9|2308|12|480.48|43.24|523.72|98.10|368.76|192.19|135.49|152.80|510.10| +2451471|1445|2980|98962|447676|390|3551|98962|447676|390|3551|26|9|2309|61|2231.99|66.95|2298.94|49.18|3441.62|133.91|545.50|1552.58|3557.75| +2451524|39743|2497|63148|1125363|5178|35793|63148|1125363|5178|35793|40|8|2309|67|91.79|7.34|99.13|34.75|44.89|17.44|23.79|50.56|86.98| +2451480|57363|1594|33425|1429024|4623|24222|33425|1429024|4623|24222|22|8|2309|4|271.44|19.00|290.44|50.58|122.56|198.15|63.02|10.27|192.14| +2452319|35100|9457|37393|630709|5978|46094|37393|630709|5978|46094|3|11|2310|21|270.69|18.94|289.63|14.03|947.31|48.72|206.43|15.54|980.28| +2452421|45338|3541|98455|972313|1724|40382|98455|972313|1724|40382|5|3|2310|61|5222.21|469.99|5692.20|37.54|3839.95|208.88|1052.79|3960.54|4347.48| +2452411|41660|4801|78600|1680752|4951|10703|78600|1680752|4951|10703|5|19|2310|9|566.19|33.97|600.16|89.31|203.58|90.59|38.04|437.56|326.86| +2452386|70168|16811|87342|884640|7091|20389|87342|884640|7091|20389|9|2|2310|59|2053.79|164.30|2218.09|56.61|1401.84|1047.43|301.90|704.46|1622.75| +2452100|30145|9155|59942|955070|6198|48926|59942|955070|6198|48926|32|19|2311|2|245.66|14.73|260.39|86.31|33.96|181.78|30.02|33.86|135.00| +2451293||13268||||||297346|2152|19405||9|2313||1747.80||1852.66|91.15|403.35||||| +2452073|32178|1823|41401|402057|3513|40216|41401|402057|3513|40216|31|29|2314|89|21918.92|219.18|22138.10|37.80|995.91|20822.97|317.82|778.13|1252.89| +2452082|86018|8697|76345|1353281|3017|26000|76345|1353281|3017|26000|37|35|2315|67|31.49|2.83|34.32|67.33|361.13|1.88|24.57|5.04|431.29| +2452212|17360|15917|19674|647777|2280|40383|19674|647777|2280|40383|5|1|2315|33|268.29|13.41|281.70|44.49|0.00|244.14|12.07|12.08|57.90| +2452122|15934|14787|31661|312205|4749|20940|31661|312205|4749|20940|3|1|2315|36|3814.92|114.44|3929.36|40.84|745.20|1564.11|1935.69|315.12|900.48| +2452315|23146|8591|41822|1095806|4990|36574|41822|1095806|4990|36574|35|24|2316|9|501.57|30.09|531.66|36.49|51.39|386.20|2.30|113.07|117.97| +2452330|70947|16081|49482|1104892|6702|10959|49482|1104892|6702|10959|19|13|2316|1|151.17|6.04|157.21|78.78|0.00|101.28|8.48|41.41|84.82| +2451608|38024|1580|22663|1598643|4223|13803|22663|1598643|4223|13803|44|10|2317|59|5318.85|106.37|5425.22|26.31|2393.63|5318.85|0.00|0.00|2526.31| +2451611|67794|15841|80201|1473865|1648|48435|80201|1473865|1648|48435|55|27|2317|52|2430.48|0.00|2430.48|79.71|3608.80|534.70|1497.66|398.12|3688.51| +2452746||17209|29989|421265|988||29989||988|28711|25||2318|60|1633.20||1649.53||1261.80|||219.51|1289.36| +2452976|52282|16881|16364|349356|1195|21787|16364|349356|1195|21787|42|29|2318|28|2889.04|231.12|3120.16|65.10|1655.92|577.80|1409.85|901.39|1952.14| +2452819|36797|1311|70905|90047|4389|12331|70905|90047|4389|12331|42|34|2319|6|1362.12|13.62|1375.74|71.95|458.82|422.25|234.96|704.91|544.39| +2452855|73107|7219|80683|1543226|969|2312|80683|1543226|969|2312|15|16|2319|61|0.00|0.00|0.00|58.43|50.02|0.00|0.00|0.00|108.45| +2451388|16074|14713|50979|294530|834|49185|50979|294530|834|49185|10|8|2320|5|142.20|4.26|146.46|51.93|23.70|52.61|17.91|71.68|79.89| +2451214|3467|5960|97165|337792|5619|45799|97165|337792|5619|45799|26|7|2320|30|471.90|9.43|481.33|92.07|504.90|302.01|139.30|30.59|606.40| +2452872||525|24828|||7284||536180|5132|7284|31||2322|||41.13||35.21||795.18|||950.90| +2452165|4815|15327|58262|698158|4007|28703|58262|698158|4007|28703|5|13|2324|16|659.52|32.97|692.49|70.36|433.44|125.30|341.90|192.32|536.77| +2452625|57116|4692|44818|1139832|5599|32213|44818|1139832|5599|32213|49|1|2325|12|310.68|9.32|320.00|4.25|270.60|111.84|182.93|15.91|284.17| +2451586|22955|625|56496|227160|3871|11335|56496|227160|3871|11335|31|16|2326|5|823.35|49.40|872.75|47.75|44.25|386.97|253.10|183.28|141.40| +|7141|11698|||210|33850|75466||210||55||2326||||1035.32||||2.34|36.73|| +2452834|73596|6426|82860|322420|5980|45540|82860|322420|5980|45540|9|2|2327|22|618.42|55.65|674.07|23.82|59.40|111.31|375.26|131.85|138.87| +2452852|75392|11724|87144|1842794|5997|15247|87144|1842794|5997|15247|3|14|2327|21|652.47|32.62|685.09|69.19|1029.00|469.77|113.27|69.43|1130.81| +2452697|60021|8430|13768|804789|1057|30478|13768|804789|1057|30478|21|24|2327|5|637.90|0.00|637.90|82.65|39.85|236.02|301.41|100.47|122.50| +2452239|32375|10189|54419|695472|6478|48331|54419|695472|6478|48331|19|1|2328|78|7163.52|501.44|7664.96|24.40|2910.18|6590.43|108.88|464.21|3436.02| +2452275|54896|113|13314|1661513|3665|4217|13314|1661513|3665|4217|13|14|2328|7|447.65|13.42|461.07|52.85|167.86|308.87|8.32|130.46|234.13| +2452763|33958|9223|60246|1293127|2455|16536|60246|1293127|2455|16536|54|19|2329|10|164.20|0.00|164.20|29.61|124.80|137.92|19.18|7.10|154.41| +2452629|67453|17539|29622|1547650|878|46106|29622|1547650|878|46106|54|7|2329|77|4814.81|288.88|5103.69|72.07|4444.44|4188.88|600.89|25.04|4805.39| +2452760|60699|17781|95761|893110|2012|25761|95761|893110|2012|25761|30|19|2329|3|390.81|0.00|390.81|56.11|278.28|31.26|133.03|226.52|334.39| +2452641|68971|2617|59808|294340|5672|14204|59808|294340|5672|14204|55|5|2329|93|627.75|31.38|659.13|76.51|452.91|558.69|46.27|22.79|560.80| +2452320|33942|6753|11393|1901277|6361|13845|11393|1901277|6361|13845|41|31|2330|19|3464.27|207.85|3672.12|1.89|1220.37|3152.48|227.60|84.19|1430.11| +2451272|69581|10915|59366|999728|5690|852|59366|999728|5690|852|1|30|2331|4|15.32|0.00|15.32|45.89|11.48|12.71|1.85|0.76|57.37| +2451372|16767|8530|28986|1254236|6239|9489|28986|1254236|6239|9489|49|11|2332|18|121.14|0.00|121.14|41.05|24.84|35.13|86.01|0.00|65.89| +2451469|69348|9026|92066|720011|3956|19548|92066|720011|3956|19548|52|3|2334|1|32.76|2.29|35.05|59.99|15.56|8.19|23.34|1.23|77.84| +2451269||6511|||6207||37495|1297851|6207|42563||32|2334|1|59.34|2.37|61.71|||53.99||1.72|| +2452488|58022|3339|56017|853208|6046|15637|56017|853208|6046|15637|1|35|2335|10|588.50|35.31|623.81|39.55|237.30|211.86|350.27|26.37|312.16| +2452530|33495|10379|25709|1467153|1750|29947|25709|1467153|1750|29947|23|13|2335|24|3012.96|30.12|3043.08|28.11|969.60|2530.88|433.87|48.21|1027.83| +2452397|21278|14569||1369923|||95413|||||15|2335||4370.07|43.70|||1030.82|524.40|||1076.76| +2452504|58285|15625|32120|372741|5362|5630|32120|372741|5362|5630|23|30|2335|1|1.49|0.10|1.59|64.67|56.97|1.26|0.13|0.10|121.74| +|28378|857|153||4003|9528||951678|||25|18|2336||1379.18||||62.48|717.17|463.40||| +2451516|47956|7717|72174|1629806|1838|39823|72174|1629806|1838|39823|32|15|2337|4|140.32|12.62|152.94|39.70|82.24|50.51|10.77|79.04|134.56| +2452201|82915|2017|73725|1259222|5872|48122|73725|1259222|5872|48122|55|11|2338|81|1988.55|79.54|2068.09|84.70|477.09|795.42|298.28|894.85|641.33| +2452031|76644|11197|25145|1688112|5479|8046|25145|1688112|5479|8046|27|5|2338|1|182.61|1.82|184.43|74.48|14.80|122.34|59.66|0.61|91.10| +2452152|55897|4059|55244|1146521|4198|8675|55244|1146521|4198|8675|41|5|2338|49|4784.36|191.37|4975.73|40.41|3160.99|2296.49|597.08|1890.79|3392.77| +2452001|61877|9511|34806|578327|1048|5016|34806|578327|1048|5016|1|12|2340|75|9801.75|784.14|10585.89|12.10|2773.50|3626.64|3334.55|2840.56|3569.74| +2452079|61813|16171|50055|373150|3445|7740|50055|373150|3445|7740|11|18|2340|20|78.00|7.02|85.02|80.24|122.80|35.10|20.59|22.31|210.06| +|69694|16522||||||132126||14890|||2341|53|||4077.98|66.44||2581.47|197.16|962.64|| +2451494|62300|2815|84620|1276863|7153|26996|84620|1276863|7153|26996|25|20|2341|5|452.80|13.58|466.38|40.31|133.50|452.80|0.00|0.00|187.39| +2451919|70580|5990|57916|969165|5214|1663|57916|969165|5214|1663|7|12|2343|29|234.90|7.04|241.94|45.01|67.28|202.01|29.27|3.62|119.33| +2452900|43685|11448|95251|1063085|2472|32706|95251|1063085|2472|32706|6|11|2345|62|264.74|5.29|270.03|39.41|11.78|201.20|36.85|26.69|56.48| +2452770|70692|7050|76230|485824|6265|29059|76230|485824|6265|29059|15|19|2345|60|453.60|4.53|458.13|27.15|3629.40|167.83|57.15|228.62|3661.08| +2451950|78014|9035|81251|1230573|3895|37664|81251|1230573|3895|37664|50|1|2347|52|382.72|3.82|386.54|25.83|0.00|168.39|203.61|10.72|29.65| +2452096|47908|17450|81286|577459|832|17005|81286|577459|832|17005|1|10|2347|1|0.00|0.00|0.00|36.40|25.94|0.00|0.00|0.00|62.34| +2452193|63452|16827|23543|104055|3320|29832|23543|104055|3320|29832|31|20|2348|49|1068.69|74.80|1143.49|37.95|534.10|790.83|77.80|200.06|646.85| +|49265|11411|82287||1915||82287|1735301||28129||30|2348|||19.52|995.82|91.87|||43.05|435.34|| +2452290|70043|17643|85325|1617282|4886|23643|85325|1617282|4886|23643|27|20|2348|2|0.00|0.00|0.00|44.43|78.14|0.00|0.00|0.00|122.57| +2451972|76032|10195|73254|1404357|7107|4610|73254|1404357|7107|4610|1|9|2350|16|1749.60|0.00|1749.60|63.77|1407.20|1207.22|330.85|211.53|1470.97| +|1263|4471|||1957|28210|15498|376151|1957|28210|||2350|81||289.77|6085.32||613.98|4752.35||114.76|| +2451283|66304|16|85571|1292796|2697|49634|85571|1292796|2697|49634|43|20|2351|1|96.30|5.77|102.07|40.22|1.57|23.11|46.10|27.09|47.56| +2452038|34854|7748|40632|514382|132|9835|40632|514382|132|9835|53|13|2352|7|112.77|7.89|120.66|14.76|112.77|60.89|30.09|21.79|135.42| +2452020|73935|2150|33457|411626|3395|9564|33457|411626|3395|9564|50|16|2352|13|1838.85|147.10|1985.95|67.66|192.40|91.94|1677.03|69.88|407.16| +2452674|29998|12324|43880|1384361|1195|43676|43880|1384361|1195|43676|57|26|2353|15|1813.50|54.40|1867.90|26.90|318.15|924.88|168.83|719.79|399.45| +2452695|20515|787|24379|1513314|4649|45537|24379|1513314|4649|45537|33|5|2353|7|1217.93|60.89|1278.82|95.20|202.93|864.73|300.22|52.98|359.02| +2451895|77909|3677|62077|215435|793|30904|62077|215435|793|30904|56|10|2354|61|1286.49|64.32|1350.81|19.98|680.76|244.43|281.35|760.71|765.06| +2451333|30832|12271|6749|1590870|4591|48712|6749|1590870|4591|48712|2|8|2355|17|1102.79|11.02|1113.81|40.97|91.80|981.48|33.96|87.35|143.79| +2451373|69013|15170|38738|234459|933|19854|38738|234459|933|19854|43|10|2355|74|59.94|5.39|65.33|19.64|79.92|19.78|5.22|34.94|104.95| +2452782|19635|9637|81922|219997|3225|17700|81922|219997|3225|17700|39|26|2359|16|1051.84|21.03|1072.87|7.34|615.68|136.73|448.40|466.71|644.05| +2452667|32418|12168|39016|1838929|7195|41908|39016|1838929|7195|41908|15|17|2359|21|1378.44|55.13|1433.57|78.81|165.90|510.02|8.68|859.74|299.84| +2451029|74212|11698|16994|141950|4953|5909|16994|141950|4953|5909|38|11|2362|4|92.88|1.85|94.73|11.92|264.40|81.73|8.13|3.02|278.17| +2451148|70527|2509|408|784419|6030|44841|408|784419|6030|44841|32|2|2363|1|6.29|0.12|6.41|32.61|5.46|3.96|0.93|1.40|38.19| +2451555|35730|14074|14078|1081341|3004|43867|14078|1081341|3004|43867|44|25|2364|69|1164.72|69.88|1234.60|25.35|1553.19|407.65|83.27|673.80|1648.42| +2451969|32418|9152|36085|1138790|2742|5447|36085|1138790|2742|5447|20|23|2365|8|45.12|0.00|45.12|70.13|116.48|26.16|17.82|1.14|186.61| +2451777||2903|||||8383||||||2366||||112.77|64.35|38.25|13.96||43.92|107.97| +2451823|50618|2483|6052|1799888|3324|18360|6052|1799888|3324|18360|19|6|2366|6|1029.42|0.00|1029.42|74.75|420.18|452.94|345.88|230.60|494.93| +2452397|16401|7127|49575|717167|3159|47951|49575|717167|3159|47951|15|33|2368|23|2276.31|22.76|2299.07|21.24|0.00|1160.91|323.46|791.94|44.00| +2452721|17288|1569|7903|1094948|3146|32191|7903|1094948|3146|32191|55|15|2369|23|83.72|1.67|85.39|11.53|13.80|18.41|16.98|48.33|27.00| +2452810|73620|17649|42637|523635|5880|20525|42637|523635|5880|20525|15|13|2369|27|3624.48|326.20|3950.68|12.81|2135.70|1159.83|1749.90|714.75|2474.71| +2452716|7416|8619|82060|1825242|6824|29606|82060|1825242|6824|29606|55|33|2373|55|8935.30|804.17|9739.47|14.95|5385.60|7058.88|637.98|1238.44|6204.72| +2452773|65105|9681|83491|32273|1055|30590|83491|32273|1055|30590|30|15|2373|17|545.70|10.91|556.61|52.15|809.20|27.28|285.13|233.29|872.26| +2452417|61314|15091|60126|394999|5872|7545|60126|394999|5872|7545|42|5|2375|37|708.18|21.24|729.42|21.95|381.10|594.87|31.72|81.59|424.29| +2451729|79725|2572|63906|1149927|3595|8016|63906|1149927|3595|8016|16|25|2377|58|5152.14|257.60|5409.74|53.94|6133.50|0.00|2369.98|2782.16|6445.04| +2451673|27143|12308|92595|1404109|5223|37537|92595|1404109|5223|37537|55|18|2378|43|8599.57|687.96|9287.53|95.21|2834.99|4299.78|3826.81|472.98|3618.16| +2451601|60005|4942|73643|711922|154|16433|73643|711922|154|16433|34|33|2378|35|7.35|0.29|7.64|57.71|89.60|0.44|0.76|6.15|147.60| +2451647|72320|15586|1953|1705830|5707|32737|1953|1705830|5707|32737|40|17|2378|32|1325.12|39.75|1364.87|61.72|1282.56|477.04|203.53|644.55|1384.03| +2451610|70265|722|314|1032855|2643|20784|314|1032855|2643|20784|1|19|2378|28|50.96|4.07|55.03|79.35|138.60|20.38|4.89|25.69|222.02| +2452194|74435|1535|52061|639524|4587|34497|52061|639524|4587|34497|29|1|2379|6|285.12|11.40|296.52|99.89|22.80|171.07|72.99|41.06|134.09| +2451434|59802|12158|17267|1380009|5151|48646|17267|1380009|5151|48646|31|16|2381|7|138.53|2.77|141.30|76.22|0.00|8.31|11.71|118.51|78.99| +2451638|16814|16754|58023|1127025|1980|33382|58023|1127025|1980|33382|22|10|2381|27|41.58|1.66|43.24|22.00|0.00|21.20|16.91|3.47|23.66| +2451476|74119|14438|18756|466615|874|19533|18756|466615|874|19533|1|9|2381|24|2724.48|27.24|2751.72|32.21|696.24|1362.24|1307.75|54.49|755.69| +2452392|84037|9641|24383|1357942|2254|3766|24383|1357942|2254|3766|19|19|2382|15|227.85|6.83|234.68|29.06|41.85|209.62|8.02|10.21|77.74| +2452408|71597|47|16451|751833|4368|45750|16451|751833|4368|45750|5|1|2382|17|131.58|2.63|134.21|72.93|338.30|78.94|41.58|11.06|413.86| +2451818|64597|12949|58694|1413097|2253|27409|58694|1413097|2253|27409|31|24|2383|8|28.88|2.59|31.47|21.18|134.80|9.24|7.46|12.18|158.57| +2451292|49521|9121|45181|1491789|6397|2205|45181|1491789|6397|2205|38|23|2385|15|462.30|32.36|494.66|19.14|814.65|254.26|45.76|162.28|866.15| +2452555|43423|12869|60585|1033728|1271|23614|60585|1033728|1271|23614|53|19|2386|21|906.15|45.30|951.45|87.53|523.53|525.56|323.50|57.09|656.36| +2452473|56414|17887|63238|656100|5409|28671|63238|656100|5409|28671|39|11|2386|8|322.32|19.33|341.65|84.75|154.16|161.16|107.97|53.19|258.24| +2452501|69061|11697|13337|1715240|1590|18693|13337|1715240|1590|18693|9|17|2386|18|153.18|4.59|157.77|40.13|22.86|56.67|64.66|31.85|67.58| +2451843|31466|4789|21440|1797167|5247|25774|21440|1797167|5247|25774|50|15|2387|16|69.92|2.09|72.01|74.79|99.20|25.17|38.93|5.82|176.08| +2451992|57002|9950|6092|1570482|472|12909|6092|1570482|472|12909|20|8|2387|37|215.34|2.15|217.49|28.18|1669.44|38.76|95.35|81.23|1699.77| +2451413|74434|8474|74294|686575|6056|47137|74294|686575|6056|47137|26|1|2388|18|1671.84|100.31|1772.15|73.81|1124.10|1003.10|254.12|414.62|1298.22| +2452498||14899|5162||1988|||||44030|13||2390|53|230.55||248.99||67.84||66.47||| +2452819|35084|5820|38391|1542909|674|22392|38391|1542909|674|22392|15|16|2392|4|104.36|2.08|106.44|17.75|32.08|89.74|7.16|7.46|51.91| +2452731|60723|13110|52278|1597919|3913|9154|52278|1597919|3913|9154|45|22|2392|4|234.52|18.76|253.28|79.04|27.40|82.08|44.20|108.24|125.20| +2452737|19828|11433|96675|334412|4502|40639|96675|334412|4502|40639|42|12|2392|8|978.32|39.13|1017.45|25.72|434.80|264.14|678.47|35.71|499.65| +2452880|75581|16284|34114|1748962|963|25364|34114|1748962|963|25364|21|34|2392|10|105.00|4.20|109.20|9.91|52.50|101.85|2.29|0.86|66.61| +2451730|70706|15242|51735|794117|3252|834|51735|794117|3252|834|46|20|2393|25|4163.25|83.26|4246.51|49.68|1288.50|3830.19|286.43|46.63|1421.44| +2451606|53304|7948|12101|662354|2207|8460|12101|662354|2207|8460|50|1|2393|7|543.41|27.17|570.58|45.08|132.51|434.72|104.34|4.35|204.76| +2452261|34955|7673|77028|306236|4244|19852|77028|306236|4244|19852|5|23|2394|36|874.80|43.74|918.54|83.72|296.64|122.47|37.61|714.72|424.10| +2452148|69492|471|57663|1873807|7198|33789|57663|1873807|7198|33789|59|13|2394|2|113.04|2.26|115.30|12.77|135.02|0.00|4.52|108.52|150.05| +2451295|48457|8104|87602|894781|5960|43539|87602|894781|5960|43539|19|22|2395|5|100.95|8.07|109.02|52.30|35.00|45.42|34.42|21.11|95.37| +||14861|98986||||98986||5632||||2397|1|1.75||1.82||0.70|0.66|1.09||| +2452314|50758|5913|24262|1010712|5962|4012|24262|1010712|5962|4012|55|15|2397|11|79.86|4.79|84.65|18.64|269.72|5.59|57.93|16.34|293.15| +2451825|60568|15316|76291|1361007|4476|32345|76291|1361007|4476|32345|38|23|2399|83|649.89|51.99|701.88|92.51|34.03|565.40|8.44|76.05|178.53| +2451778|73471|11648|37844|160951|4858|44365|37844|160951|4858|44365|1|5|2399|21|4239.90|0.00|4239.90|27.06|584.64|127.19|3084.53|1028.18|611.70| +|64915|6103|||4380|19610|31520|||19610|13||2399|35|||704.77|||||496.23|| +2451487|57663|6220|72951|1820162|3373|2937|72951|1820162|3373|2937|7|16|2400|81|3991.68|119.75|4111.43|49.15|5246.37|2355.09|1096.51|540.08|5415.27| +2451966|51866|3703|56854|1834330|5853|129|56854|1834330|5853|129|25|31|2401|47|10644.56|638.67|11283.23|46.09|1086.17|9260.76|982.49|401.31|1770.93| +2452011|12664|17473|79074|1142890|1459|41275|79074|1142890|1459|41275|29|32|2401|74|71.04|6.39|77.43|22.48|1360.86|41.91|4.36|24.77|1389.73| +2451970|55268|2717|45774|1791477|6427|35730|45774|1791477|6427|35730|37|24|2401|56|2047.36|81.89|2129.25|2.70|271.04|2026.88|2.86|17.62|355.63| +2451168|67164|14044|72244|1030238|232|3779|72244|1030238|232|3779|4|24|2403|20|550.20|11.00|561.20|13.08|807.00|280.60|202.20|67.40|831.08| +2451437|56481|9223|10355|913764|5173|582|10355|913764|5173|582|46|29|2403|12|125.28|3.75|129.03|55.54|85.08|67.65|3.45|54.18|144.37| +2451202|8725|17440|36717|1802418|4919|43773|36717|1802418|4919|43773|58|24|2403|21|540.33|21.61|561.94|84.13|555.87|297.18|43.76|199.39|661.61| +2451296|64338|9073|90769|1405805|6441|18277|90769|1405805|6441|18277|28|10|2407|69|3276.12|229.32|3505.44|5.70|297.39|3145.07|60.28|70.77|532.41| +2452321|75530|14005|41222|816461|6547|26586|41222|816461|6547|26586|5|19|2410|33|2625.48|183.78|2809.26|49.61|665.61|341.31|571.04|1713.13|899.00| +2452294|57892|12301|50475|1053304|2322|9546|50475|1053304|2322|9546|33|30|2411|13|29.38|1.46|30.84|95.99|3.25|10.87|4.25|14.26|100.70| +2451100|49092|2800|50718|433917|7163|39609|50718|433917|7163|39609|56|9|2412|15|660.15|26.40|686.55|54.45|343.80|231.05|360.44|68.66|424.65| +2451136|36933|15656|39745|1212249|3153|39657|39745|1212249|3153|39657|43|31|2412|9|488.70|9.77|498.47|15.94|195.48|439.83|45.93|2.94|221.19| +2452398|72354|10787|99326|713215|5951|18240|99326|713215|5951|18240|21|8|2413|5|0.00|0.00|0.00|71.20|163.80|0.00|0.00|0.00|235.00| +2452356|47828|3487|96454|1915601|1595|3142|96454|1915601|1595|3142|45|5|2414|7|205.45|0.00|205.45|28.02|209.65|65.74|34.92|104.79|237.67| +2452416|36042|10049|11946|825852|6958|20464|11946|825852|6958|20464|37|27|2415|56|297.36|17.84|315.20|32.64|53.76|231.94|22.24|43.18|104.24| +2452326|33404|16123|64328|1509267|3777|22916|64328|1509267|3777|22916|35|25|2416|10|528.40|21.13|549.53|45.22|137.20|438.57|46.71|43.12|203.55| +2452384|51203|69|41823|1546548|5320|8845|41823|1546548|5320|8845|17|22|2416|3|370.89|0.00|370.89|63.24|111.24|185.44|114.97|70.48|174.48| +2452397|35365|1109|58684|1586682|2327|25125|58684|1586682|2327|25125|9|24|2416|14|889.98|80.09|970.07|85.05|561.40|195.79|69.41|624.78|726.54| +2452482|76296|15813|47648|486546|713|32250|47648|486546|713|32250|13|1|2417|19|316.73|22.17|338.90|62.25|0.00|262.88|17.23|36.62|84.42| +2452518|62908|3493|47850|1158883|316|46949|47850|1158883|316|46949|15|22|2417|1|53.76|2.68|56.44|5.96|34.56|29.56|14.27|9.93|43.20| +2451226|64916|355|25412|1616623|187|9398|25412|1616623|187|9398|20|6|2419|55|3015.10|60.30|3075.40|18.57|1491.05|1778.90|964.23|271.97|1569.92| +2451063|24421|12709|78629|454359|6536|44266|78629|454359|6536|44266|2|17|2419|32|1286.08|12.86|1298.94|53.34|547.84|12.86|50.92|1222.30|614.04| +2451267|29469|17336|39972|236201|2830|47812|39972|236201|2830|47812|56|32|2419|17|2105.28|63.15|2168.43|41.09|372.81|1915.80|43.58|145.90|477.05| +2452188|36963|4243|47236|1383716|2243|770|47236|1383716|2243|770|29|3|2421|1|34.83|2.78|37.61|27.48|1.16|25.07|1.95|7.81|31.42| +2452079|62062|4319|79573|1641513|4700|22640|79573|1641513|4700|22640|47|13|2421|17|73.44|5.14|78.58|58.15|40.80|25.70|34.37|13.37|104.09| +2452112||17169||63958|6020|5329|7496|||5329||11|2422||4295.23|85.90||99.42||3865.70|120.26||| +2452160|16510|12505|83695|203833|6018|41756|83695|203833|6018|41756|29|32|2422|14|972.86|48.64|1021.50|1.12|91.56|321.04|410.64|241.18|141.32| +2451056|73214|2792|61486|721513|5987|14461|61486|721513|5987|14461|37|1|2423|20|267.40|8.02|275.42|83.59|501.40|18.71|176.56|72.13|593.01| +2451153|67978|1690|67409|322051|4246|4088|67409|322051|4246|4088|50|20|2424|20|43.60|1.74|45.34|52.80|0.00|15.69|21.21|6.70|54.54| +2451108|40086|10246|36313|86207|4101|30748|36313|86207|4101|30748|2|28|2424|8|1859.76|0.00|1859.76|90.38|958.64|1487.80|148.78|223.18|1049.02| +2452217|86117|6951|21340|1644672|2404|43336|21340|1644672|2404|43336|27|20|2425|7|400.12|12.00|412.12|2.01|180.25|192.05|208.07|0.00|194.26| +2452254|70606|5095|56501|130773|598|20423|56501|130773|598|20423|15|27|2428|45|45.00|0.00|45.00|9.50|52.20|22.50|5.17|17.33|61.70| +2452278|44913|15201|39903|1097368|1680|29145|39903|1097368|1680|29145|11|12|2428|5|47.55|3.32|50.87|23.05|9.40|34.71|11.55|1.29|35.77| +|66846|9573|46693||3498||||3498|||18|2428||1215.39|0.00|1215.39|40.02|||226.42||| +2451459|79569|15662|93762|648190|1018|4246|93762|648190|1018|4246|37|35|2431|32|3693.76|36.93|3730.69|94.08|461.44|36.93|1426.16|2230.67|592.45| +2451555|54859|7468|34512|1392850|3188|1879|34512|1392850|3188|1879|37|18|2431|3|14.13|0.56|14.69|1.18|6.72|4.94|0.73|8.46|8.46| +2451523|41827|3799|79632|198025|3265|4058|79632|198025|3265|4058|37|28|2431|66|1178.76|70.72|1249.48|63.35|1296.90|683.68|64.36|430.72|1430.97| +2450991|62953|6136|72110|1607395|6680|14812|72110|1607395|6680|14812|19|4|2432|6|595.56|5.95|601.51|83.07|319.02|244.17|196.77|154.62|408.04| +2450997|44901|10243|75963|1186376|5142|31108|75963|1186376|5142|31108|31|23|2432|1|43.93|0.87|44.80|60.83|5.32|38.65|0.26|5.02|67.02| +2451890|75837|1549|17947|764520|2293|42513|17947|764520|2293|42513|2|5|2433|87|141.81|4.25|146.06|95.61|27.84|117.70|23.38|0.73|127.70| +2451891|11920|4367|47349|265047|4345|9243|47349|265047|4345|9243|25|31|2433|7|201.11|18.09|219.20|62.00|67.90|102.56|5.91|92.64|147.99| +2451246|62837|556|34161|134768|4622|1629|34161|134768|4622|1629|44|22|2434|97|618.86|6.18|625.04|54.73|944.78|216.60|4.02|398.24|1005.69| +2451350|57183|1454|67092|1149548|4004|36060|67092|1149548|4004|36060|25|34|2435|32|321.92|9.65|331.57|51.26|873.60|305.82|12.07|4.03|934.51| +2451467|24557|9439|61934|4500|4959|25980|61934|4500|4959|25980|31|25|2436|80|1957.60|19.57|1977.17|47.84|1304.80|313.21|624.86|1019.53|1372.21| +2451508|67614|5611|61038|1817590|3742|35894|61038|1817590|3742|35894|10|29|2436|4|197.72|5.93|203.65|62.04|124.16|177.94|1.97|17.81|192.13| +2451624|66831|7648|14472|298768|5817|5266|14472|298768|5817|5266|56|17|2436|3|257.10|0.00|257.10|83.36|22.47|182.54|67.84|6.72|105.83| +2452527|11162|2187|16504|756467|1583|2699|16504|756467|1583|2699|55|4|2437|8|66.48|5.98|72.46|23.10|454.48|43.87|7.46|15.15|483.56| +2452578||8388||||||||41517|12|2|2437||78.84||||22.68|10.24|37.04|31.56|38.63| +2452645|73413|12030|724|606528|1002|29631|724|606528|1002|29631|55|9|2440|49|2234.89|89.39|2324.28|80.45|1305.85|625.76|1158.57|450.56|1475.69| +2451668|63388|12170|60375|1043288|6201|12314|60375|1043288|6201|12314|25|8|2441|10|1233.90|111.05|1344.95|2.93|490.60|1073.49|83.41|77.00|604.58| +2452108|38567|17019|25752|1031947|542|18337|25752|1031947|542|18337|11|2|2443|20|1486.40|133.77|1620.17|70.84|405.20|1426.94|23.18|36.28|609.81| +2452152||9937|39439|1599559|||39439|1599559|||31|8|2443||7016.16||||505.92|0.00|841.93|6174.23|604.23| +2452430|6403|349|50360|1256473|6586|8252|50360|1256473|6586|8252|45|23|2445|1|6.32|0.31|6.63|54.19|12.65|0.63|4.55|1.14|67.15| +2451789|75946|1112|57226|842222|5618|5814|57226|842222|5618|5814|49|35|2446|61|9203.07|184.06|9387.13|6.93|2831.62|1748.58|4323.60|3130.89|3022.61| +2452761|78980|14388|58222|1221074|1526|17178|58222|1221074|1526|17178|45|19|2447|35|1257.90|75.47|1333.37|81.39|479.15|75.47|189.18|993.25|636.01| +2452722|59065|9150|76257|99487|2717|13547|76257|99487|2717|13547|15|9|2447|58|1247.00|0.00|1247.00|54.91|1247.00|660.91|164.10|421.99|1301.91| +2452673|74601|11305|33180|1713814|6801|46988|33180|1713814|6801|46988|54|16|2447|8|76.72|3.83|80.55|57.38|225.68|44.49|27.71|4.52|286.89| +2452821|33559|547|85634|1778314|5437|14087|85634|1778314|5437|14087|57|18|2447|9|319.95|3.19|323.14|3.23|421.74|204.76|100.21|14.98|428.16| +||15134||1525715||15881|||998|15881|||2448|34|3516.96|140.67||5.73|1274.66|1758.48|1107.84||| +2451369|78336|13507|61389|1579074|2806|37738|61389|1579074|2806|37738|44|29|2449|30|215.10|0.00|215.10|67.34|942.00|118.30|18.39|78.41|1009.34| +2452259|19463|10451|6424|1457060|3343|24826|6424|1457060|3343|24826|49|35|2450|10|770.60|7.70|778.30|29.98|689.50|747.48|12.48|10.64|727.18| +2451535|5212|5491|89983|1338244|3305|7213|89983|1338244|3305|7213|50|3|2451|1|3.12|0.03|3.15|88.84|51.57|2.21|0.55|0.36|140.44| +2451720|55065|12964||1174658|4379|18100||||18100|31|6|2451|19|||491.47|||361.38|13.25||898.39| +2451741|82576|12253|20746|111799|2499|49552|20746|111799|2499|49552|52|28|2451|1|195.63|9.78|205.41|72.11|107.74|89.98|20.07|85.58|189.63| +2451980|40884|17543|19430|1030553|2923|30893|19430|1030553|2923|30893|29|27|2452|26|70.46|4.93|75.39|65.74|370.50|40.16|6.06|24.24|441.17| +2451862|29431|9413|78034|1373191|5948|6305|78034|1373191|5948|6305|44|35|2452|68|4006.56|240.39|4246.95|46.08|2348.72|520.85|2858.28|627.43|2635.19| +2452432|9228|9067|65398|1142311|1271|39784|65398|1142311|1271|39784|19|23|2453|26|560.04|0.00|560.04|55.64|270.92|190.41|192.20|177.43|326.56| +2451226|73065|12373|70770|1194432|6447|5662|70770|1194432|6447|5662|52|18|2454|12|29.52|0.29|29.81|77.85|43.08|20.95|5.39|3.18|121.22| +2451379|43233|7358|76582|1565935|4088|18688|76582|1565935|4088|18688|31|3|2454|17|568.82|28.44|597.26|41.20|325.04|290.09|144.93|133.80|394.68| +2452588|57207|12210|44358|1262117|4766|21201|44358|1262117|4766|21201|1|27|2455|96|1348.80|13.48|1362.28|9.13|3147.84|795.79|66.36|486.65|3170.45| +2452753|30006|15168|80680|1168387|6047|28107|80680|1168387|6047|28107|51|7|2455|4|55.16|1.65|56.81|81.36|27.56|55.16|0.00|0.00|110.57| +2451516|19005|8018|77460|710388|5363|6291|77460|710388|5363|6291|38|13|2456|12|719.76|7.19|726.95|97.88|239.88|431.85|233.20|54.71|344.95| +2451553|35581|3800|1699|675222|6698|30621|1699|675222|6698|30621|32|30|2456|54|4084.56|245.07|4329.63|59.75|1750.14|2205.66|1164.91|713.99|2054.96| +2451670|55715|10789|9945|1740068|6191|10314|9945|1740068|6191|10314|32|2|2456|9|933.66|84.02|1017.68|24.81|429.75|858.96|63.49|11.21|538.58| +2452416|42597|1263|97089|490783|2077|24971|97089|490783|2077|24971|37|34|2457|5|596.60|17.89|614.49|3.56|224.80|113.35|449.42|33.83|246.25| +2452378|76063|7512|56958|565602|999|24043|56958|565602|999|24043|24|1|2457|54|399.60|0.00|399.60|72.57|1421.28|91.90|267.69|40.01|1493.85| +2451818|60712|7432|98421|737865|4156|16847|98421|737865|4156|16847|49|9|2458|55|4133.25|206.66|4339.91|38.68|1623.60|3967.92|100.85|64.48|1868.94| +2451721|26204|13724|7035|900385|6182|9932|7035|900385|6182|9932|2|21|2458|38|3484.60|174.23|3658.83|40.49|1431.84|2474.06|899.38|111.16|1646.56| +2451529|73713|5206|5291|781489||15532||781489|||8||2460|14|||500.53|43.49|290.64||||| +2451886|14744|2557|94019|297862|905|18701|94019|297862|905|18701|2|29|2461|42|341.88|20.51|362.39|23.95|2296.56|246.15|73.71|22.02|2341.02| +2452006|80913|2413|9583|749224|6409|9079|9583|749224|6409|9079|38|34|2461|2|231.36|2.31|233.67|54.41|73.44|94.85|125.58|10.93|130.16| +2451767|66548|745|74923|682980|2684|1141|74923|682980|2684|1141|1|6|2462|5|37.80|1.13|38.93|64.32|2.45|11.34|4.23|22.23|67.90| +2451619|45833|11420|14424|528082|2093|14151|14424|528082|2093|14151|37|8|2462|33|2147.64|150.33|2297.97|82.00|1112.43|687.24|993.07|467.33|1344.76| +2450999|72788|61|97094|181589|5284|24626|97094|181589|5284|24626|26|23|2463|22|1554.08|77.70|1631.78|56.42|271.26|77.70|959.64|516.74|405.38| +2450988|32740|6091|41944|571336|5881|23226|41944|571336|5881|23226|32|10|2463|48|405.12|12.15|417.27|62.93|1891.20|113.43|23.33|268.36|1966.28| +2452340|38347|1527|82976|1680237|2298|20406|82976|1680237|2298|20406|51|4|2464|29|866.23|25.98|892.21|73.33|157.47|69.29|223.14|573.80|256.78| +2452556|85831|14193|14450|1812250|2497|17497|14450|1812250|2497|17497|53|18|2466|1|86.86|0.86|87.72|43.93|51.28|1.73|53.63|31.50|96.07| +2452433|71089|1153|96724|54710|696|37082|96724|54710|696|37082|55|18|2466|8|378.96|30.31|409.27|89.67|250.88|45.47|176.74|156.75|370.86| +2452539|70608|13233|97164|30593|22|39898|97164|30593|22|39898|47|24|2466|5|23.40|1.40|24.80|40.28|101.50|19.65|3.48|0.27|143.18| +2452771|40706|13167|80434|1709948|5062|37376|80434|1709948|5062|37376|39|28|2467|91|1232.14|61.60|1293.74|50.26|2123.03|1219.81|9.86|2.47|2234.89| +2452716|47980|15933|26521|236482|4396|35957||236482||35957||14|2467|31|||||||557.24||| +2452050|47756|3269|74504|1397198|3092|28986|74504|1397198|3092|28986|7|31|2468|21|2950.29|265.52|3215.81|57.90|0.00|1563.65|166.39|1220.25|323.42| +2452179|81232|12571|15843|1848369|6048|9374|15843|1848369|6048|9374|39|26|2468|6|39.84|1.19|41.03|43.49|159.36|21.11|2.62|16.11|204.04| +2452177|36801|2043|86608|1768761|429|27142|86608|1768761|429|27142|23|24|2468|1|27.87|0.00|27.87|17.74|1.76|5.85|1.10|20.92|19.50| +2452895|54750|4050|3440|1674814|4235|13609|3440|1674814|4235|13609|54|10|2469|19|1493.78|119.50|1613.28|79.41|1459.20|478.00|609.46|406.32|1658.11| +2452574|35934|24|12432|440977|1522|31620|12432|440977|1522|31620|3|3|2470|7|306.32|24.50|330.82|87.42|158.27|199.10|64.33|42.89|270.19| +2452832|58803|14949|22264|1864083|3502|21321|22264|1864083|3502|21321|9|20|2470|15|1317.75|0.00|1317.75|94.34|0.00|724.76|260.91|332.08|94.34| +2451492|3404|2431|22540|898074|5608|11542|22540|898074|5608|11542|10|17|2473|14|101.08|1.01|102.09|82.29|5.60|23.24|60.71|17.13|88.90| +2452243|34991|10477|88692|1231838|5737|32836|88692|1231838|5737|32836|11|17|2474|25|30.25|2.11|32.36|43.92|189.75|26.62|3.30|0.33|235.78| +2452363|56114|4347|93930|1066930|5177|42224|93930|1066930|5177|42224|21|32|2474|7|233.24|20.99|254.23|83.72|136.08|25.65|47.74|159.85|240.79| +|81767|8835||905710|6267|||905710|||13|1|2475|48|1870.56||||504.00||701.46||| +2452287|32757|13549|14261|156857|5853|12002|14261|156857|5853|12002|17|8|2476|16|759.04|37.95|796.99|20.92|656.48|204.94|448.82|105.28|715.35| +2451755|4229|13394|17594|479617|4840|2416|17594|479617|4840|2416|32|4|2478|11|617.43|0.00|617.43|91.66|44.00|18.52|407.25|191.66|135.66| +2451718|15445|13772||1203685||40634||||40634|44|7|2478||||3104.67||710.22|1581.08||439.84|1039.13| +2452442|45806|7363|86310|52815|3320|49425|86310|52815|3320|49425|9|27|2479|2|17.58|0.17|17.75|90.61|23.46|2.10|4.64|10.84|114.24| +2451697|38427|6964|85035|1707681|3480|8126|85035|1707681|3480|8126|43|18|2481|33|98.34|0.98|99.32|46.78|222.75|97.35|0.90|0.09|270.51| +2451578|74567|902|92144|1339724|904|4884|92144|1339724|904|4884|20|14|2481|26|4358.90|348.71|4707.61|55.59|924.56|3181.99|1070.98|105.93|1328.86| +2451570|30951|1610|56800|596370|4293|6616|56800|596370|4293|6616|2|23|2481|35|3274.60|32.74|3307.34|39.71|183.75|785.90|920.81|1567.89|256.20| +2451970|35990|17509|30248|827632|4569|40353|30248|827632|4569|40353|14|29|2482|13|1062.75|53.13|1115.88|11.54|345.02|924.59|55.26|82.90|409.69| +2452123|20130|1249|51054|1288938|1222|33106|51054|1288938|1222|33106|23|27|2482|58|1722.02|154.98|1877.00|72.99|1085.76|275.52|1330.78|115.72|1313.73| +2451778|58888|14461|60824|583766|7061|40803|60824|583766|7061|40803|22|2|2483|43|5626.55|168.79|5795.34|29.82|1664.10|2925.80|1404.39|1296.36|1862.71| +||5702||600336|||53151|600336|||25|24|2483|11||9.19||58.69|||84.11||207.25| +2451687|80795|8000|54821|1890609|1150|16138|54821|1890609|1150|16138|49|32|2484|5|9.45|0.85|10.30|80.11|104.05|4.34|1.83|3.28|185.01| +2451679|36885|12883|42999|1049266|333|9847|42999|1049266|333|9847|55|33|2484|8|962.64|19.25|981.89|57.90|622.88|375.42|182.03|405.19|700.03| +2452229|36093|3573|2701|641891|6328|40718|2701|641891|6328|40718|49|29|2485|5|166.45|14.98|181.43|38.98|110.95|84.88|1.63|79.94|164.91| +2452575|28879|10957|24750|112258|5013|27768|24750|112258|5013|27768|7|32|2486|31|1997.64|59.92|2057.56|44.58|2183.33|459.45|1522.80|15.39|2287.83| +2452674|29049|3855|81460|235063|6266|31022|81460|235063|6266|31022|25|6|2486|32|734.08|44.04|778.12|34.02|893.76|14.68|597.10|122.30|971.82| +2451636|5877|15200|46332|984256|5733|11257|46332|984256|5733|11257|52|12|2487|34|3339.14|66.78|3405.92|14.29|1926.44|300.52|1549.69|1488.93|2007.51| +2451590|27146|9572|2529|83024|1119|15214|2529|83024|1119|15214|7|6|2487|55|4070.00|122.10|4192.10|1.66|1724.80|1546.60|580.38|1943.02|1848.56| +2451489|67625|15958|98565|575426|4734|36795|98565|575426|4734|36795|52|35|2487|16|1383.84|124.54|1508.38|97.75|1011.20|1356.16|19.37|8.31|1233.49| +||12260|||2460|3921|691|1492552|2460||56|33|2487|6|||580.54||16.56|265.39|270.25||| +2451485|77366|14386|74668|1837736|1157|13378|74668|1837736|1157|13378|44|28|2488|19|286.52|0.00|286.52|86.97|197.03|189.10|62.34|35.08|284.00| +2452300|81654|16609|91950|1256184|5218|29852|91950|1256184|5218|29852|31|13|2489|22|1514.70|75.73|1590.43|13.08|398.64|378.67|284.00|852.03|487.45| +2452426|16612|501|87990|1165427|6864|10301|87990|1165427|6864|10301|41|23|2490|37|832.50|8.32|840.82|2.25|262.70|632.70|47.95|151.85|273.27| +2452382|30417|3489|87734|1622206|2278|7950|87734|1622206|2278|7950|13|27|2490|11|137.94|11.03|148.97|79.67|104.17|45.52|24.95|67.47|194.87| +2452878|67684|3144|91046|440594|895|5596|91046|440594|895|5596|48|33|2491|5|1098.85|98.89|1197.74|66.29|221.95|1043.90|32.42|22.53|387.13| +2451939|45335|14105|658|1773279|7147|16968|658|1773279|7147|16968|11|16|2492|20|1313.20|0.00|1313.20|73.66|1125.60|656.60|164.15|492.45|1199.26| +2452100|4108|4855|83596|31925|5854|2583|83596|31925|5854|2583|50|14|2492|8|791.20|23.73|814.93|43.98|268.16|419.33|278.90|92.97|335.87| +2452061|46148|16040|50207|421480|3506|44732|50207|421480|3506|44732|25|30|2492|1|55.31|2.21|57.52|77.36|14.64|34.29|16.81|4.21|94.21| +2452842|5559|11118|89539|1624178|2370|27009|89539|1624178|2370|27009|12|32|2493|33|2552.88|0.00|2552.88|83.19|147.84|433.98|1356.09|762.81|231.03| +2451876|21422|10999|63222|1674170|4318|19666|63222|1674170|4318|19666|13|26|2494|10|365.90|3.65|369.55|6.90|339.80|197.58|102.67|65.65|350.35| +2452546|16656|17001|34391|1851419|6988|14307|34391|1851419|6988|14307|49|7|2495|1|90.92|2.72|93.64|67.28|6.81|78.19|3.43|9.30|76.81| +2452367|63261|13619|5437|252111|3585|29012|5437|252111|3585|29012|33|19|2495|25|920.25|27.60|947.85|27.91|147.00|855.83|49.60|14.82|202.51| +2452471|16660|16001|5065|35981|944|45860|5065|35981|944|45860|47|2|2495|7|129.15|0.00|129.15|27.40|66.15|36.16|58.58|34.41|93.55| +2452754|59821|5538|284|856284|6684|43884|284|856284|6684|43884|25|7|2496|4|303.08|6.06|309.14|84.55|221.48|181.84|105.47|15.77|312.09| +2452687|54335|9133|97320|1320318|4563|14408|97320|1320318|4563|14408|49|26|2496|4|170.52|6.82|177.34|44.20|68.20|143.23|14.46|12.83|119.22| +2452648|46032|8289|25439|554021|1803|1185|25439|554021|1803|1185|33|23|2496|43|4916.19|442.45|5358.64|14.64|779.16|934.07|1314.09|2668.03|1236.25| +2451263|46319|8368|61628|1868748|3068|46941|61628|1868748|3068|46941|52|7|2497|11|154.55|7.72|162.27|19.73|98.01|23.18|42.03|89.34|125.46| +2452534|76995|1644|39824|1360478|3885|34092|39824|1360478|3885|34092|15|13|2498|88|10487.84|839.02|11326.86|40.39|110.88|4824.40|1868.93|3794.51|990.29| +2452612|36503|84|95507|1616454|3801|1548|95507|1616454|3801|1548|60|14|2500|5|331.70|0.00|331.70|22.14|65.15|205.65|105.88|20.17|87.29| +2451738|34883|5807|73174|1396961|1995|24582|73174|1396961|1995|24582|17|3|2502|24|1968.72|157.49|2126.21|99.28|944.88|39.37|1524.18|405.17|1201.65| +2452599|12867|5229|36231|942936|841|32346|36231|942936|841|32346|57|26|2503|16|124.48|7.46|131.94|86.79|82.88|65.97|16.96|41.55|177.13| +2452667|82477|729|26224|1548102|1180|49058|26224|1548102|1180|49058|15|4|2503|41|7144.25|642.98|7787.23|75.00|2707.23|714.42|4886.67|1543.16|3425.21| +2451917|42383|8227|20846|850108|523|27738|20846|850108|523|27738|23|5|2505|33|1094.94|43.79|1138.73|66.87|816.75|613.16|404.69|77.09|927.41| +2451912|40976|13694|56360|1052047|5019|23134|56360|1052047|5019|23134|26|12|2505|29|2010.57|20.10|2030.67|16.14|50.17|1769.30|77.20|164.07|86.41| +2452687|80412|13447|67405|1692573|941|40462|67405|1692573|941|40462|6|15|2509|24|156.24|6.24|162.48|16.15|31.20|98.43|24.28|33.53|53.59| +2451471|20963|14146|68190|4675|1960|11998|68190|4675|1960|11998|19|17|2510|18|439.56|17.58|457.14|96.86|219.78|140.65|167.38|131.53|334.22| +2451328|69060|934|29255|1528152|7049|25546|29255|1528152|7049|25546|4|13|2512|1|19.03|1.71|20.74|61.31|23.19|1.90|16.95|0.18|86.21| +2452403|69816|12965|6644|1331037|3783|14529|6644|1331037|3783|14529|53|19|2514|94|7153.40|643.80|7797.20|41.72|2161.06|3934.37|1770.46|1448.57|2846.58| +2452307|54709|9267|71024|933266|661|14103|71024|933266|661|14103|1|6|2514|71|558.06|22.32|580.38|49.77|201.64|245.54|259.39|53.13|273.73| +2452288|39287|11439|11397|484664|1578|20045|11397|484664|1578|20045|23|24|2515|10|279.50|25.15|304.65|63.38|21.50|139.75|58.69|81.06|110.03| +2452389|75099|12991|34193|818982|218|607|34193|818982|218|607|53|17|2515|30|430.80|25.84|456.64|43.63|775.50|133.54|53.50|243.76|844.97| +2452309|65805|7119|19026|1808464|4302|42540|19026|1808464|4302|42540|27|22|2515|12|259.44|2.59|262.03|6.72|113.52|41.51|39.22|178.71|122.83| +2451952||12806|72905|||||1503458||21458|||2516|39||40.63||21.62|82.68|414.50|||144.93| +2452856|66328|1825|70538|1061683|5628|1220|70538|1061683|5628|1220|33|30|2517|9|558.45|0.00|558.45|49.56|304.65|430.00|115.60|12.85|354.21| +2452808|71193|13657|69603|1512816|4382|33548|69603|1512816|4382|33548|57|10|2517|79|1513.64|45.40|1559.04|84.79|527.72|181.63|306.36|1025.65|657.91| +2452891|77769|5541|18180|1380318|4503|36688|18180|1380318|4503|36688|25|34|2517|56|135.52|9.48|145.00|57.19|566.72|14.90|41.01|79.61|633.39| +2451532|80666|8731|||6935|15584|27237||||56||2518||582.32|40.76||78.79|305.68|395.97|152.80|33.55|425.23| +2451721|57752|2176|17027|290026|272|35671|17027|290026|272|35671|34|9|2519|18|194.76|15.58|210.34|86.28|21.78|142.17|27.87|24.72|123.64| +2451540|22299|4723|76163|1011248|733|22218|76163|1011248|733|22218|50|35|2523|28|479.92|14.39|494.31|52.77|279.44|81.58|155.35|242.99|346.60| +2451490|75664|8126|18334|19527|2767|12360|18334|19527|2767|12360|16|19|2523|8|308.16|18.48|326.64|91.34|355.52|261.93|18.49|27.74|465.34| +2450976|20466|17236|64027|603696|2565|38587|64027|603696|2565|38587|52|34|2524|56|3684.24|147.36|3831.60|70.88|1841.84|810.53|2816.23|57.48|2060.08| +2451379|17273|15932|68064|578867|4070|32073|68064|578867|4070|32073|1|6|2529|28|6235.88|311.79|6547.67|45.70|818.72|2556.71|1177.33|2501.84|1176.21| +2451629|14973|10621|51873|1158205|90|15222|51873|1158205|90|15222|46|13|2532|54|741.42|29.65|771.07|92.38|2118.96|511.57|135.61|94.24|2240.99| +2451716|74694|2224|56286|730803|4520|33437|56286|730803|4520|33437|16|24|2534|2|3.90|0.19|4.09|40.84|25.36|0.23|1.13|2.54|66.39| +2451795|6999|10697|71895|1880549|50|25951|71895|1880549|50|25951|1|32|2536|65|301.60|3.01|304.61|73.81|48.10|289.53|3.13|8.94|124.92| +2451992|41420|4511|45812|1627787|4515|47243|45812|1627787|4515|47243|55|25|2539|1|22.20|0.44|22.64|41.96|44.40|3.10|8.97|10.13|86.80| +2452397|64529|6493|8523|390043|5963|4236|8523|390043|5963|4236|35|34|2540|53|0.00|0.00|0.00|90.70|1421.46|0.00|0.00|0.00|1512.16| +2451896|18780|2605|67037|442634|6231|41028|67037|442634|6231|41028|8|28|2541|29|1166.96|11.66|1178.62|37.24|1471.46|46.67|829.01|291.28|1520.36| +2451948|49756|12899|3287|564266|3292|22718|3287|564266|3292|22718|43|12|2541|1|63.51|4.44|67.95|41.43|11.69|6.98|38.44|18.09|57.56| +2452496|8796|1163|15776|1092525|1274|13623|15776|1092525|1274|13623|3|26|2542|6|524.28|5.24|529.52|44.69|104.82|351.26|140.14|32.88|154.75| +2452075|69866|4345|20096|4686|4123|23960|20096|4686|4123|23960|50|18|2543|9|352.53|31.72|384.25|18.68|178.29|211.51|112.81|28.21|228.69| +2451937|73633|4723|96162|1216501|6793|33532|96162|1216501|6793|33532|50|25|2543|29|587.54|17.62|605.16|28.35|117.45|64.62|371.27|151.65|163.42| +2451939|45629|16598|81765|1602681|6444|48207|81765|1602681|6444|48207|56|15|2543|46|26.22|0.52|26.74|78.46|370.76|12.58|4.91|8.73|449.74| +2452050|33471|16411|65428|950522|2369|9463|65428|950522|2369|9463|44|23|2546|16|2679.04|241.11|2920.15|17.01|648.48|642.96|468.29|1567.79|906.60| +2452020|43741|9409|65662|1600180|1545|19663|65662|1600180|1545|19663|59|32|2548|8|501.84|35.12|536.96|64.38|179.76|386.41|69.25|46.18|279.26| +2451831|71254|11767|78737|1170339|1588|37590|78737|1170339|1588|37590|13|13|2548|2|210.36|4.20|214.56|44.57|75.72|92.55|10.60|107.21|124.49| +2451437||7058|||5404|13337||||13337||29|2549||||55.56|78.71|59.04||||140.39| +2451968|77126|8891|53504|962391|2031|29671|53504|962391|2031|29671|47|33|2550|6|82.80|5.79|88.59|29.56|9.90|46.36|25.50|10.94|45.25| +2451670|48601|16520|63936|1073938|3940|20742|63936|1073938|3940|20742|16|9|2551|13|175.50|10.53|186.03|44.49|24.96|42.12|48.01|85.37|79.98| +2451699|61096|3674|21935|103370|5158|28993|21935|103370|5158|28993|58|35|2551|16|612.00|30.60|642.60|36.07|179.52|214.20|350.06|47.74|246.19| +2451615|43449|15961|91558|1081032|5259|27533|91558|1081032|5259|27533|50|29|2551|59|1296.82|64.84|1361.66|56.58|273.76|1283.85|10.63|2.34|395.18| +2450981|47140|6133|76547|1235849|5551|406|76547|1235849|5551|406|2|1|2552|25|985.00|88.65|1073.65|52.27|343.50|807.70|88.65|88.65|484.42| +2451795|75649|16033|99234|1881434|6034|4987|99234|1881434|6034|4987|20|13|2553|12|411.12|8.22|419.34|91.82|194.76|49.33|358.17|3.62|294.80| +2452165|30031|16523|10865|1130230|2042|49407|10865|1130230|2042|49407|38|1|2554|6|40.02|1.20|41.22|89.56|38.64|16.00|18.73|5.29|129.40| +2451946|68010|6866|8525|329818|1064|46357|8525|329818|1064|46357|11|1|2562|42|405.30|12.15|417.45|18.38|926.52|186.43|98.49|120.38|957.05| +2451498|70250|11260|24353|97948|5970|39571|24353|97948|5970|39571|4|10|2564|6|132.72|7.96|140.68|8.87|156.84|130.06|1.51|1.15|173.67| +2451185|22052|2720|97050|1138222|6594|566|97050|1138222|6594|566|46|33|2566|8|267.44|8.02|275.46|61.02|329.84|21.39|184.53|61.52|398.88| +2451098|76248|13018|67548|262156|3164|46972|67548|262156|3164|46972|22|28|2566|58|9535.78|381.43|9917.21|17.07|4042.02|4005.02|2433.53|3097.23|4440.52| +2452387|40778|14777|88737|837889|6594|45346|88737|837889|6594|45346|3|17|2567|1|38.75|2.32|41.07|78.78|23.68|19.37|14.14|5.24|104.78| +2452403|69292|12257|11254|528556|5858|713|11254|528556|5858|713|31|33|2567|82|676.50|54.12|730.62|26.62|270.60|662.97|0.54|12.99|351.34| +2452469|31581|11595|56709|92896|1436|1223|56709|92896|1436|1223|15|13|2567|11|69.30|5.54|74.84|16.34|55.44|67.91|0.84|0.55|77.32| +||853|98873|1090751|||98873||6276|1754||23|2567||2765.01|||35.04|1367.93||||1430.62| +2451691|36823|8330|1592|1555600|8|11969|1592|1555600|8|11969|4|21|2570|8|522.48|26.12|548.60|17.83|16.32|485.90|24.14|12.44|60.27| +2451500|60123|17209|94058|526509|2160|14658|94058|526509|2160|14658|25|21|2570|43|2441.54|195.32|2636.86|99.96|1808.58|439.47|1942.00|60.07|2103.86| +2452027|48555|3847|41793|1846688|4383|47678|41793|1846688|4383|47678|7|21|2572|17|106.76|6.40|113.16|31.60|148.24|67.25|25.68|13.83|186.24| +2451977||8642|71662|1366319||12339|71662|1366319||12339||18|2572||428.67|||||77.16|||116.60| +2451734|34490|6925|7834|184085|6736|33387|7834|184085|6736|33387|44|26|2573|38|874.00|69.92|943.92|8.16|255.74|533.14|279.50|61.36|333.82| +2451104|63931|286|19919|1308297|2722|21547|19919|1308297|2722|21547|52|35|2574|21|242.13|4.84|246.97|78.48|707.70|167.06|37.53|37.54|791.02| +2451082||1519||307819|||63516|307819|||||2574|49|843.78||860.65|80.89|441.98||737.46||539.74| +2451645|16495|17726|99397|1809277|3157|12237|99397|1809277|3157|12237|34|33|2575|21|2535.75|177.50|2713.25|51.16|1074.99|278.93|1579.77|677.05|1303.65| +2451112|59053|8564|80357|906457|3757|46257|80357|906457|3757|46257|40|3|2576|16|822.88|65.83|888.71|58.11|117.44|493.72|161.28|167.88|241.38| +2452789|25879|9984|31658|1702272|6933|9547|31658|1702272|6933|9547|48|10|2577|20|760.40|22.81|783.21|26.78|334.40|319.36|414.57|26.47|383.99| +2452387|79177|7815|93932|741205|4031|42904|93932|741205|4031|42904|55|7|2578|7|791.91|15.83|807.74|12.30|161.77|261.33|95.50|435.08|189.90| +2452360|49718|1705|60485|962642|3999|15709|60485|962642|3999|15709|35|14|2578|5|13.20|1.05|14.25|75.64|50.90|8.18|0.75|4.27|127.59| +2452387|29304|6497|11607|1875588|6416|11197|11607|1875588|6416|11197|41|4|2579|2|219.38|0.00|219.38|14.33|67.02|0.00|32.90|186.48|81.35| +2452345|50948|6873|7807|81216|7072|48481|7807|81216|7072|48481|19|3|2579|76|6101.28|488.10|6589.38|4.81|1149.12|2196.46|3319.09|585.73|1642.03| +2452819|68514|6621|4691|323415|1339|24863|4691|323415|1339|24863|9|1|2586|2|79.46|7.15|86.61|30.11|2.48|14.30|29.32|35.84|39.74| +2452874|52917|11083|67389|1161702|15|21528|67389|1161702|15|21528|57|8|2586|19|797.05|31.88|828.93|15.50|311.41|71.73|239.35|485.97|358.79| +2452706|4563|7737|3849|705860|1918|13976|3849|705860|1918|13976|6|7|2587|26|249.34|0.00|249.34|60.11|186.94|69.81|41.29|138.24|247.05| +2452828|78871|3579|44075|1135844|3613|22582|44075|1135844|3613|22582|19|13|2587|12|592.32|29.61|621.93|92.57|98.64|473.85|79.37|39.10|220.82| +2452724|4098|17892|26729|1036095|3861|19674|26729|1036095|3861|19674|49|4|2587|46|1626.56|130.12|1756.68|63.04|101.66|666.88|940.48|19.20|294.82| +2452119|11318|6931|708|1470579|1835|20861|708|1470579|1835|20861|31|9|2588|16|360.80|18.04|378.84|65.17|969.60|241.73|63.10|55.97|1052.81| +2451657|26386|3793|29886|1720477|668|17259|29886|1720477|668|17259|10|20|2589|11|123.97|3.71|127.68|99.84|94.49|114.05|6.94|2.98|198.04| +2451916|23075|983|82201|1120858|3068|16125|82201|1120858|3068|16125|50|1|2590|18|463.50|27.81|491.31|89.28|128.70|32.44|331.91|99.15|245.79| +2451595|68571|17389||1386710||||1386710|1952|20747||17|2591|||||||||0.39|109.54| +2451669|58116|17282|26219|1184085|4351|30617|26219|1184085|4351|30617|10|31|2591|6|5.34|0.48|5.82|71.37|248.28|0.37|3.87|1.10|320.13| +2451592|58963|11089|7190|832298|959|12820|7190|832298|959|12820|34|17|2591|30|1014.30|60.85|1075.15|36.50|202.80|994.01|6.49|13.80|300.15| +2451691|73105|11380|86675|1305326|3624|2722|86675|1305326|3624|2722|28|11|2592|21|154.98|7.74|162.72|76.34|232.47|37.19|36.51|81.28|316.55| +2451586|40562|7690|32474|1856352|4230|38908|32474|1856352|4230|38908|20|9|2592|15|148.35|5.93|154.28|92.14|55.65|74.17|33.38|40.80|153.72| +2451580|17648|3466||||41903|56131|1197553|2528||52||2592|80|1068.80|32.06|||56.00||565.71|150.39|| +2451745|66580|14083|87414|1204815|4263|34464|87414|1204815|4263|34464|8|34|2592|14|92.54|0.00|92.54|48.17|103.46|78.65|10.00|3.89|151.63| +2452781|58590|6738|62862|869964|1066|6168|62862|869964|1066|6168|18|21|2593|3|148.29|1.48|149.77|84.13|346.05|16.31|71.26|60.72|431.66| +||3640||||48926||294526|3526|48926||4|2594||||512.81|63.95||286.98|||178.04| +2451437|71560|8332|28357|413966|1175|19184|28357|413966|1175|19184|55|30|2594|76|2546.00|203.68|2749.68|7.96|753.16|1934.96|140.53|470.51|964.80| +2452489|38497|4755|74449|1446982|5468|34754|74449|1446982|5468|34754|33|15|2595|2|438.78|39.49|478.27|81.74|200.88|100.91|229.75|108.12|322.11| +2452497|78297|5597|35202|560128|2301|35674|35202|560128|2301|35674|41|35|2595|58|66.12|0.00|66.12|20.60|265.06|26.44|29.76|9.92|285.66| +2452384|19106|3979|30951|1288550|5702|13301|30951|1288550|5702|13301|17|28|2595|20|106.80|0.00|106.80|54.80|51.20|8.54|88.43|9.83|106.00| +2452065|74338|5585|78271|1909044|2217|10990|78271|1909044|2217|10990|37|1|2596|57|2450.43|24.50|2474.93|13.50|1224.93|710.62|1565.82|173.99|1262.93| +2452104|24445|5107|71795|683251|6354|20210|71795|683251|6354|20210|5|25|2598|33|984.06|0.00|984.06|79.49|447.15|0.00|600.27|383.79|526.64| +2452189|11347|10447|83736|1716541|2499|34904|83736|1716541|2499|34904|57|28|2598|39|582.27|52.40|634.67|46.78|138.84|209.61|231.04|141.62|238.02| +2451947|69007|13448|78055|676549|4643|39137|78055|676549|4643|39137|20|13|2601|32|30.72|0.61|31.33|26.00|0.00|23.34|1.99|5.39|26.61| +2451512|67316|4189|57456|782361|866|22218|57456|782361|866|22218|19|4|2602|16|2298.56|206.87|2505.43|29.26|364.80|781.51|1001.25|515.80|600.93| +2451430|84465|6314|23357|1104761|5812|28416|23357|1104761|5812|28416|20|6|2604|18|2.16|0.00|2.16|27.20|91.26|0.75|0.43|0.98|118.46| +2451469|75016|9854|98328|1810142|6403|9864|98328|1810142|6403|9864|58|8|2604|27|2491.56|224.24|2715.80|34.08|2179.98|2366.98|13.70|110.88|2438.30| +2451545|45167|9212|63251|211478|6105|7532|63251|211478|6105|7532|28|14|2605|27|117.99|4.71|122.70|35.97|708.48|33.03|18.69|66.27|749.16| +2451630|85545|16864|99592|1698760|4328|34296|99592|1698760|4328|34296|2|9|2605|9|38.79|3.49|42.28|26.68|28.80|19.39|4.46|14.94|58.97| +2451515|31973|9793|99581|1247616|2498|12933|99581|1247616|2498|12933|43|34|2606|57|47.88|3.35|51.23|47.76|55.29|17.23|5.51|25.14|106.40| +2452398|59208|13725|80603|1799771|3739|1707|80603|1799771|3739|1707|7|28|2607|4|15.56|0.93|16.49|9.65|9.32|0.46|11.02|4.08|19.90| +2452561|23922|16199|45990|1197610|4077|3297|45990|1197610|4077|3297|35|13|2607|21|1153.74|46.14|1199.88|13.51|1127.91|922.99|53.07|177.68|1187.56| +2452519||12683|34260|||6574|34260|1765298||6574||35|2607||28.35|0.28|||96.75||||108.74| +2451863|68567|16387|28791|1880112|680|19601|28791|1880112|680|19601|50|18|2609|6|205.14|0.00|205.14|62.24|144.06|112.82|54.46|37.86|206.30| +2451699|61197|16742|39416|1653701|4339|49824|39416|1653701|4339|49824|7|23|2609|52|7126.08|285.04|7411.12|32.81|3958.76|213.78|6497.56|414.74|4276.61| +2452469|55194|13523|92515|475245|728|43058|92515|475245|728|43058|41|10|2610|6|938.64|84.47|1023.11|78.77|424.32|750.91|116.39|71.34|587.56| +2452502|57313|8729|53599|1199355|664|46420|53599|1199355|664|46420|5|19|2610|60|6911.40|207.34|7118.74|77.40|1525.80|1520.50|4420.53|970.37|1810.54| +2452340|69605|17349|3490|1149424|3711|40045|3490|1149424|3711|40045|29|32|2610|7|408.52|20.42|428.94|92.33|268.80|347.24|20.22|41.06|381.55| +2451965|10792|1613|15601|206070|6939|45008|15601|206070|6939|45008|23|22|2612|80|6088.00|487.04|6575.04|13.24|6849.60|913.20|3984.59|1190.21|7349.88| +2452422|49454|8505|86253|976917|1181|19092|86253|976917|1181|19092|1|20|2613|21|1021.44|40.85|1062.29|52.52|1087.38|469.86|419.20|132.38|1180.75| +2452328|39877|6051|82059|1691229|3599|47522|82059|1691229|3599|47522|13|5|2613|23|1529.04|30.58|1559.62|12.03|857.21|581.03|862.68|85.33|899.82| +2452540|80934|9891|87359|||47833||1583117|||||2613||||||||||203.43| +2452846|28830|8625|9005|1495787|169|19751|9005|1495787|169|19751|54|3|2615|18|1125.00|101.25|1226.25|86.76|162.72|180.00|623.70|321.30|350.73| +2452735|27935|3991|54437|1253380|3523|49666|54437|1253380|3523|49666|57|4|2615|53|11090.78|443.63|11534.41|72.04|1247.62|10314.42|248.43|527.93|1763.29| +2452926|21030|6547|19827|1864466|2352|45838|19827|1864466|2352|45838|31|15|2615|2|1.60|0.12|1.72|75.41|46.46|0.86|0.60|0.14|121.99| +2451811|58515|889|21971|599733|1495|38904|21971|599733|1495|38904|49|15|2616|2|15.04|0.90|15.94|82.02|10.20|2.25|10.74|2.05|93.12| +2452061|83200|9837|48373|65276|6179|30682|48373|65276|6179|30682|27|27|2617|57|4940.76|0.00|4940.76|12.39|4632.39|1778.67|2972.36|189.73|4644.78| +2452168|39954|9003|1159|694199|2086|34373|1159|694199|2086|34373|39|1|2617|37|2287.34|22.87|2310.21|72.36|1726.42|526.08|1074.36|686.90|1821.65| +2451989|33551|16351|79666|337016|6719|8469|79666|337016|6719|8469|25|16|2618|50|1849.00|92.45|1941.45|26.25|45.00|628.66|536.94|683.40|163.70| +2451673|65259|17510|47934|1546379|1977|2822|47934|1546379|1977|2822|34|13|2620|10|152.30|1.52|153.82|52.94|131.50|24.36|58.85|69.09|185.96| +2451667|43289|11671|21877|437401|478|2|21877|437401|478|2|19|19|2623|41|1063.54|95.71|1159.25|98.68|72.16|63.81|849.77|149.96|266.55| +2452071|60557|16553|69206|1473408|5610|35812|69206|1473408|5610|35812|47|1|2625|52|4836.52|241.82|5078.34|6.34|1875.12|4691.42|18.86|126.24|2123.28| +2452501|20036|11165|82949|1441182|3415|45201|82949|1441182|3415|45201|33|22|2626|69|4477.41|223.87|4701.28|73.11|5522.07|2775.99|935.78|765.64|5819.05| +2451827|16697|8911|23168|1757005|5242|3432|23168|1757005|5242|3432|26|27|2627|10|287.50|20.12|307.62|29.14|299.00|123.62|124.54|39.34|348.26| +2452583|26272|9018|82054|1531557|4260|7763|82054|1531557|4260|7763|42|13|2628|43|1750.96|17.50|1768.46|52.82|415.81|87.54|349.31|1314.11|486.13| +2452628|76050|14337|66172|740239|3044|29843|66172|740239|3044|29843|30|17|2628|21|259.98|18.19|278.17|44.61|589.26|181.98|7.02|70.98|652.06| +2452119|62277|15473|56503|1378015|774|21086|56503|1378015|774|21086|32|14|2630|50|8531.50|341.26|8872.76|64.26|3738.50|170.63|5852.60|2508.27|4144.02| +2452128|27327|13268|6465|1826319|126|17933|6465|1826319|126|17933|14|16|2630|44|87.12|0.87|87.99|15.42|0.00|38.33|25.85|22.94|16.29| +2452076|63677|12911|91994|1714034|4074|10514|91994|1714034|4074|10514|38|34|2630|11|476.85|33.37|510.22|40.53|65.78|19.07|137.33|320.45|139.68| +2452433|39228|8551|95521|531109|650|37438|95521|531109|650|37438|11|20|2631|13|336.96|26.95|363.91|4.12|144.43|94.34|87.34|155.28|175.50| +2451434|28142|8761|58286|1651856|1426|16196|58286|1651856|1426|16196|44|18|2632|17|1930.86|135.16|2066.02|87.97|754.63|868.88|870.82|191.16|977.76| +||17491|||6652|||||10932||4|2632|7|118.79||119.97||79.17||||148.93| +2451415|43231|3398|91578|249052|4666|8941|91578|249052|4666|8941|46|18|2633|20|671.00|26.84|697.84|73.10|353.00|161.04|418.16|91.80|452.94| +2451519|47682|11252|66800|1704424|5248|872|66800|1704424|5248|872|14|9|2633|30|488.10|34.16|522.26|76.16|155.40|419.76|28.70|39.64|265.72| +2451757|78190|4064|58799|147521|1490|48612|58799|147521|1490|48612|53|14|2634|32|1519.36|15.19|1534.55|10.50|1744.32|319.06|108.02|1092.28|1770.01| +2451715|4820|14893|42780|1727553|1926|35099|42780|1727553|1926|35099|5|21|2634|63|2556.54|127.82|2684.36|94.21|74.97|2428.71|90.75|37.08|297.00| +2451652|62202|2918|39618|634923|7028|18733|39618|634923|7028|18733|52|34|2635|42|2798.46|0.00|2798.46|74.25|1722.00|2042.87|45.33|710.26|1796.25| +2451601|70620|11534|38871|936934|6550|14213|38871|936934|6550|14213|38|7|2635|2|146.26|8.77|155.03|20.31|25.42|16.08|84.61|45.57|54.50| +2452088|75418|2749|88708|1709135|1915|26342|88708|1709135|1915|26342|32|32|2636|67|5149.62|205.98|5355.60|17.42|3457.20|3089.77|1235.91|823.94|3680.60| +2451959|39828|5081|93196|191905|6261|570|93196|191905|6261|570|31|26|2636|18|394.56|0.00|394.56|52.90|1099.26|82.85|99.74|211.97|1152.16| +2452016|46649|17678|68016|295842|115|31941|68016|295842|115|31941|37|30|2637|44|4462.04|178.48|4640.52|56.88|0.00|133.86|995.48|3332.70|235.36| +2451307|16123|9220|12467|638158|3049|24740|12467|638158|3049|24740|19|29|2639|51|9432.45|471.62|9904.07|2.21|3007.47|8866.50|107.53|458.42|3481.30| +2451241|47593|17815|76348|1238899|6603|8754|76348|1238899|6603|8754|43|29|2639|2|17.58|0.70|18.28|20.83|29.04|8.26|6.61|2.71|50.57| +||2065|98183|619956|||98183||||50||2639|||0.00||38.64|250.25|20.92|143.87||288.89| +2452242|69293|1105|97437|1031828|5866|6657|97437|1031828|5866|6657|31|10|2642|6|16.98|0.16|17.14|67.63|27.12|14.09|2.80|0.09|94.91| +2452144|55528|16895|40014|917496|2978|20060|40014|917496|2978|20060|49|8|2645|15|581.70|17.45|599.15|55.37|268.95|11.63|205.22|364.85|341.77| +2451280|11384|10810|81594|858207|5080|13159|81594|858207|5080|13159|22|17|2648|45|2727.45|190.92|2918.37|73.11|1684.35|2018.31|574.40|134.74|1948.38| +2451294|77119|7885|95255|1257900|545|41143|95255|1257900|545|41143|19|29|2648|1|3.27|0.13|3.40|41.25|3.36|1.53|1.56|0.18|44.74| +2451654|9199|4286|91474|250464|6701|19291|91474|250464|6701|19291|31|27|2649|32|391.04|15.64|406.68|51.63|347.52|336.29|22.99|31.76|414.79| +2452688|45361|6361|40213|280810|6560|24310|40213|280810|6560|24310|18|22|2650|25|2666.25|159.97|2826.22|8.45|1066.50|346.61|1183.01|1136.63|1234.92| +2451540||10240|||2952|642||932462|2952|||22|2651|30|1680.30|117.62||||1277.02|72.59|330.69|1197.42| +||913|||||32090|1853023|||||2652|||23.40|491.56|98.33|314.56|224.71||231.28|| +2452016|31533|17642|37138|133050|1409|6521|37138|133050|1409|6521|43|24|2655|2|65.88|5.92|71.80|77.74|81.38|52.70|5.27|7.91|165.04| +2451793||3494|42256||||42256||6815|46871|56||2656||194.40||211.89|32.73|0.00|141.91|||50.22| +2451717|37380|9107|94286|1264396|3306|29016|94286|1264396|3306|29016|50|10|2657|14|179.06|16.11|195.17|92.12|152.18|137.87|17.71|23.48|260.41| +2452110|20358|4547|33836|376399|4562|18325|33836|376399|4562|18325|5|25|2658|3|164.34|1.64|165.98|86.01|107.55|118.32|5.98|40.04|195.20| +2451981|81045|16694|93657|1522101|6898|3742|93657|1522101|6898|3742|2|30|2658|30|1327.20|119.44|1446.64|24.97|153.00|464.52|327.81|534.87|297.41| +2451916|36296|10223|69971|275993|3168|47990|69971|275993|3168|47990|20|1|2660|25|1158.75|11.58|1170.33|88.91|1376.00|950.17|139.74|68.84|1476.49| +2452767|43715|17817|50443|1770318|2993|14868|50443|1770318|2993|14868|19|12|2662|25|611.25|12.22|623.47|79.09|46.00|256.72|67.36|287.17|137.31| +2451423|39672|17461|8736|1601529|5522|31515|8736|1601529|5522|31515|37|3|2663|18|2220.84|155.45|2376.29|91.74|1634.22|2176.42|17.76|26.66|1881.41| +2451570|34321|4184|39111|489226|6858|29656|39111|489226|6858|29656|40|29|2663|50|337.50|10.12|347.62|68.94|1858.50|54.00|280.66|2.84|1937.56| +2451545|73924|7316|82676|1887857|6831|3945|82676|1887857|6831|3945|20|24|2665|23|241.50|16.90|258.40|49.36|346.38|120.75|85.73|35.02|412.64| +2451535|4897|7244|61805|97735|6465|41595|61805|97735|6465|41595|26|13|2667|2|130.30|1.30|131.60|49.92|40.26|13.03|60.98|56.29|91.48| +2452459|33860|493|47693|176139|6632|34278|47693|176139|6632|34278|25|10|2668|1|82.82|1.65|84.47|90.94|6.90|18.22|55.55|9.05|99.49| +2452316|73201|229|84942|1248343|6497|38298|84942|1248343|6497|38298|13|10|2668|1|94.53|4.72|99.25|84.74|27.44|69.95|3.93|20.65|116.90| +2451497|64340|17630|70352|1430349|4746|47543|70352|1430349|4746|47543|28|13|2669|3|2.22|0.04|2.26|19.70|3.48|0.95|0.58|0.69|23.22| +2451573|53258|13351|35311|1008481|2451|17221|35311|1008481|2451|17221|50|9|2669|20|727.60|7.27|734.87|93.56|531.00|189.17|204.60|333.83|631.83| +2451343|69882|8323|8648||7182|30896||1531115|7182||4||2669|||||22.68||||780.96|546.30| +2451537|34755|9322|||||||5091|35083|||2669||1367.12||1490.16||237.36|738.24|364.75|264.13|| +2451891|84234|497|65232|1234449|1619|34789|65232|1234449|1619|34789|11|34|2670|13|7.28|0.43|7.71|3.60|273.91|2.40|1.61|3.27|277.94| +2451226|43107|199|32576|1134934|2092|22050|32576|1134934|2092|22050|38|7|2671|15|180.75|9.03|189.78|85.99|281.25|54.22|37.95|88.58|376.27| +2451806|61368|2281|26521|1758883|2092|12901|26521|1758883|2092|12901|32|30|2672|23|606.74|36.40|643.14|66.20|191.13|588.53|14.02|4.19|293.73| +2451121|29076|8606|66241|368419|6597|26775|66241|368419|6597|26775|50|8|2673|5|74.70|0.00|74.70|49.23|13.80|0.00|22.41|52.29|63.03| +2452162|68807|2965|15483|1781681|4326|19724|15483|1781681|4326|19724|5|35|2674|42|891.66|44.58|936.24|16.50|3195.36|98.08|515.82|277.76|3256.44| +2452008|16752|15503|35971|385890|4256|38222|35971|385890|4256|38222|41|2|2675|17|18.70|0.56|19.26|46.67|17.34|6.54|1.33|10.83|64.57| +2452077|46590|215|6021|441668|5315|26162|6021|441668|5315|26162|56|24|2675|7|227.36|20.46|247.82|46.40|59.43|209.17|2.36|15.83|126.29| +2451662|32504|13504|38935|97304|5273|49595|38935|97304|5273|49595|26|25|2676|16|155.84|7.79|163.63|90.89|12.32|115.32|20.66|19.86|111.00| +2452240|43361|2977|67298|1658932|2213|41894|67298|1658932|2213|41894|47|11|2678|67|420.09|25.20|445.29|84.88|1120.91|390.68|23.82|5.59|1230.99| +2452056|70174|7941|99501|334792|1636|22927|99501|334792|1636|22927|37|7|2678|4|480.32|24.01|504.33|21.89|230.80|33.62|192.08|254.62|276.70| +|78244|9765|||1108|||1618466||46950|23|8|2678|23||||29.13|||||| +2452924|60997|14451|99447|558431|6662|44046|99447|558431|6662|44046|55|27|2684|43|1448.24|14.48|1462.72|93.58|784.32|72.41|701.67|674.16|892.38| +2451597|34641|2030|46234|1015496|2805|41667|46234|1015496|2805|41667|26|1|2685|8|81.76|0.81|82.57|98.17|25.52|13.89|59.04|8.83|124.50| +2451378|51556|1172|51660|29271|1635|5246|51660|29271|1635|5246|25|33|2685|46|8368.32|334.73|8703.05|75.06|4799.64|669.46|4080.39|3618.47|5209.43| +2451234|85923|12703|9710|1428633|3203|18126|9710|1428633|3203|18126|50|28|2686|10|1387.60|83.25|1470.85|51.43|563.70|568.91|130.99|687.70|698.38| +||11875|89758|548114|5657||89758||||39||2688||||230.87|80.53||61.11|34.70|130.54|| +2452784|37551|6036|89758|548114|5657|47427|89758|548114|5657|47427|51|18|2688|6|174.84|8.74|183.58|1.31|55.44|115.39|46.96|12.49|65.49| +2452546|57798|3901|48330|1495738|299|16907|48330|1495738|299|16907|25|11|2690|33|225.06|15.75|240.81|19.76|398.31|29.25|50.91|144.90|433.82| +2452341|50151|5059|49270|993333|596|34659|49270|993333|596|34659|7|24|2690|16|198.08|15.84|213.92|56.15|64.96|65.36|55.74|76.98|136.95| +2452477|37367|5569|24682|1342442|1002|37115|24682|1342442|1002|37115|15|1|2691|10|58.00|3.48|61.48|95.38|188.70|2.32|15.03|40.65|287.56| +2452520|74917|6075|21146|1699178|57|15689|21146|1699178|57|15689|53|27|2691|39|2215.20|177.21|2392.41|58.79|443.04|465.19|560.00|1190.01|679.04| +2452909|74751|10326|47333|644389|4198|8454|47333|644389|4198|8454|31|10|2692|48|27.84|1.67|29.51|5.08|33.12|16.14|9.00|2.70|39.87| +2452761|69770|888|85155|118292|582|29719|85155|118292|582|29719|21|29|2692|14|942.76|84.84|1027.60|92.29|601.16|282.82|26.39|633.55|778.29| +2452783|74600|7021|22104|1615524|944|39164|22104|1615524|944|39164|30|22|2692|37|1170.31|23.40|1193.71|84.67|96.20|479.82|331.43|359.06|204.27| +2452854|10469|9282|81679|1773256|5633|22057|81679|1773256|5633|22057|45|31|2693|52|2255.76|22.55|2278.31|16.53|708.76|1263.22|397.01|595.53|747.84| +2452356|6028|10545|||||37678|1359588|989|||22|2698||132.65|11.93||||41.12|79.63|11.90|125.66| +2452371|43676|15177|81272|1270467|1262|1657|81272|1270467|1262|1657|43|5|2698|1|9.01|0.72|9.73|68.67|13.17|1.62|2.73|4.66|82.56| +2452558|62369|15315|34137|936302|1667|7432|34137|936302|1667|7432|15|2|2699|50|2457.50|122.87|2580.37|79.70|597.50|1966.00|403.03|88.47|800.07| +2452524|50557|14427|47503|1021602|5748|25847|47503|1021602|5748|25847|12|17|2699|49|1654.73|66.18|1720.91|43.41|912.87|1059.02|220.41|375.30|1022.46| +2452745|30374|8985|69359|1368229|4346|4016|69359|1368229|4346|4016|19|23|2699|57|5395.05|377.65|5772.70|65.14|2094.18|4639.74|422.97|332.34|2536.97| +2451347|47792|8890|81545|780018|5691|17364|81545|780018|5691|17364|2|7|2702|38|4805.48|240.27|5045.75|53.72|1952.06|2883.28|57.66|1864.54|2246.05| +2451324|63097|4576|11910|487878|4646|14567|11910|487878|4646|14567|37|33|2702|13|1486.55|59.46|1546.01|83.71|821.86|1352.76|24.08|109.71|965.03| +2451840|69773|7907|57940|780037|981|38725|57940|780037|981|38725|23|19|2703|16|1168.96|35.06|1204.02|8.77|811.84|537.72|574.42|56.82|855.67| +2451895|75372|7736|15677|1562184|3538|36045|15677|1562184|3538|36045|37|2|2703|9|926.28|9.26|935.54|38.39|383.22|129.67|557.62|238.99|430.87| +2451841|76242|5816|20626|976131|83|38035|20626|976131|83|38035|14|8|2703|61|1413.98|84.83|1498.81|52.86|1996.53|452.47|749.97|211.54|2134.22| +2451976|76296|217|75669|926680|5338|4796|75669|926680|5338|4796|55|14|2703|15|736.65|36.83|773.48|90.67|839.55|22.09|643.10|71.46|967.05| +2452042|70604|6218|66415|37505|1748|45638|66415|37505|1748|45638|44|17|2704|2|65.40|0.00|65.40|37.38|138.08|57.55|3.92|3.93|175.46| +2451899|56703|7373|80365|666358|5598|36791|80365|666358|5598|36791|8|33|2704|28|47.60|2.38|49.98|49.89|17.64|43.79|0.34|3.47|69.91| +2452007|56564|12434||606430|2210|||606430|||||2705|13|1205.36||1277.68||||742.02||127.00| +2451945|26731|4123|49093|1646955|4370|34232|49093|1646955|4370|34232|37|6|2705|61|1269.41|12.69|1282.10|59.98|63.44|482.37|283.33|503.71|136.11| +2452727|74714|15924|59263|525805|5147|37289|59263|525805|5147|37289|51|21|2706|28|142.24|8.53|150.77|55.12|426.72|108.10|4.09|30.05|490.37| +2452526|63970|11202|86704|1717745|433|1474|86704|1717745|433|1474|42|2|2706|31|1393.14|27.86|1421.00|21.40|1325.25|626.91|275.84|490.39|1374.51| +2451267|6975|17666|61358|618593|3475|21910|61358|618593|3475|21910|34|23|2708|42|1516.20|60.64|1576.84|46.92|209.58|1516.20|0.00|0.00|317.14| +2451170|47222|8378|87040|1831605|3817|45582|87040|1831605|3817|45582|55|14|2708|18|388.44|7.76|396.20|60.31|468.54|330.17|41.37|16.90|536.61| +||16026||1767319|5125|11378||||||7|2709||434.88|21.74||74.57|161.92|373.99|44.44||258.23| +2452215|38128|7577|67590|1481274|558|23442|67590|1481274|558|23442|31|29|2710|20|22.40|0.67|23.07|23.10|14.60|12.32|1.20|8.88|38.37| +2451999|45124|9149|12079|653306|933|4560|12079|653306|933|4560|32|31|2710|5|359.70|17.98|377.68|20.61|286.10|194.23|54.60|110.87|324.69| +2452838|20385|12294|53727|665582|5790|14045|53727|665582|5790|14045|48|28|2711|62|1109.80|0.00|1109.80|44.32|1021.76|721.37|271.90|116.53|1066.08| +2452400|75693|9733|30645|731483|4768|23653|30645|731483|4768|23653|57|32|2712|46|242.42|0.00|242.42|42.51|404.34|43.63|192.82|5.97|446.85| +2451427|39011|9742|50074|188491|5661|42831|50074|188491|5661|42831|56|22|2713|24|112.32|10.10|122.42|2.16|317.04|78.62|18.53|15.17|329.30| +2451252|46405|1402|82450|894479|2155|||894479|||26||2713|63|2666.16|||43.51||693.20|||| +2451169|38257|5869|82725|1702452|579|24706|82725|1702452|579|24706|56|25|2714|31|659.37|13.18|672.55|96.82|410.75|283.52|206.71|169.14|520.75| +|60005|6072|24257|||||1386954|1651|||32|2715|5|419.00|25.14|444.14||179.55||31.42|282.83|219.70| +2451086|80703|7477|43868|888013|3711|31456|43868|888013|3711|31456|43|5|2716|94|775.50|38.77|814.27|67.32|553.66|85.30|365.80|324.40|659.75| +2451014|35549|16303|1739|1757356|1420|31037|1739|1757356|1420|31037|1|11|2716|33|798.60|31.94|830.54|83.55|462.99|543.04|122.66|132.90|578.48| +2452429|33842|14671|51432|1103345|2458|49171|51432|1103345|2458|49171|7|1|2717|31|2644.30|52.88|2697.18|7.66|1678.03|475.97|1366.04|802.29|1738.57| +2451145|57837|10501|33379|1110454|5482|42110|||5482||||2718|||8.47||8.45|36.72|127.06|228.28|68.20|53.64| +2452118|71631|6753|31006|361084|5562|34487|31006|361084|5562|34487|47|6|2719|4|168.72|10.12|178.84|71.76|89.32|11.81|6.27|150.64|171.20| +2452444|26275|8883|38728|1308899|6623|43547|38728|1308899|6623|43547|47|30|2720|18|888.30|44.41|932.71|74.89|439.38|124.36|229.18|534.76|558.68| +2452225|64448|12605|29754|698882|6057|6785|29754|698882|6057|6785|35|34|2723|25|1626.00|48.78|1674.78|22.32|570.00|146.34|443.89|1035.77|641.10| +2452244|18324|14027|80367|413049|6178|40107|80367|413049|6178|40107|39|31|2724|67|316.91|6.33|323.24|45.51|58.96|66.55|192.77|57.59|110.80| +2451242|34104|362|88164|1898103|5195|1934|88164|1898103|5195|1934|32|32|2725|78|804.96|0.00|804.96|95.10|2213.64|603.72|82.50|118.74|2308.74| +2451157|70618|484|15554|639195|436|124|15554|639195|436|124|13|15|2725|84|209.16|0.00|209.16|34.40|2238.60|8.36|98.39|102.41|2273.00| +2451266|45330|6706|92247|1450350|2904|18594|92247|1450350|2904|18594|16|31|2725|16|853.44|25.60|879.04|71.33|267.68|332.84|203.03|317.57|364.61| +2451429|32225|14018|94016|1753961|1553|36698|94016|1753961|1553|36698|34|26|2726|29|5735.04|229.40|5964.44|1.18|1911.68|4760.08|263.23|711.73|2142.26| +2452514|61796|17703|98885|1145132|179|17226|98885|1145132|179|17226|21|18|2727|16|1034.72|20.69|1055.41|46.01|0.00|600.13|408.51|26.08|66.70| +2452449|70107|4757|92364|546923|4682|33241|92364|546923|4682|33241|41|29|2728|28|2252.04|157.64|2409.68|85.05|440.44|382.84|1831.81|37.39|683.13| +||4449|60013|171325||2396||171325|684||25||2728||7071.12||7707.52|15.09||2121.33|247.48|4702.31|| +2451722|27458|16574|4528|1540350|172|17241|4528|1540350|172|17241|10|27|2729|2|40.66|1.21|41.87|82.23|9.56|17.48|5.79|17.39|93.00| +2451702|20420|16387|5438|998797|2556|39515|5438|998797|2556|39515|43|19|2732|39|8681.40|0.00|8681.40|11.38|4929.99|7031.93|1171.12|478.35|4941.37| +2452422|43981|3243|14733|633785|6964|40432|14733|633785|6964|40432|1|11|2733|17|109.65|9.86|119.51|72.82|1052.98|53.72|24.04|31.89|1135.66| +2451611|54355|5230|63676|290878|2080|21682|63676|290878|2080|21682|22|4|2734|16|808.64|72.77|881.41|76.60|715.20|371.97|362.43|74.24|864.57| +2451569|66365|14356|93238|877012|4786|28160|93238|877012|4786|28160|37|16|2734|57|1516.20|75.81|1592.01|84.33|1178.76|60.64|160.11|1295.45|1338.90| +2452153|17960|9273|92534|1413787|1887|26340|92534|1413787|1887|26340|25|24|2735|1|76.46|6.88|83.34|57.94|24.03|68.81|6.04|1.61|88.85| +2452195|42586|2379|90143|647518|1693|14668|90143|647518|1693|14668|37|23|2735|19|971.28|48.56|1019.84|62.69|49.78|543.91|64.10|363.27|161.03| +2451862|16882|13993|41558|1846043|6528|15013|41558|1846043|6528|15013|17|29|2736|2|35.40|0.70|36.10|71.51|12.10|12.03|21.96|1.41|84.31| +2451959|26008|7861|77800|1567909|5010|17886|77800|1567909|5010|17886|53|13|2736|6|289.20|0.00|289.20|28.11|148.74|63.62|173.69|51.89|176.85| +2451894|70969|13949|12421|50058|4495|28132|12421|50058|4495|28132|41|14|2736|25|1201.50|60.07|1261.57|62.74|1373.25|949.18|239.70|12.62|1496.06| +2452554|75343|11901|65347|394229|1158|10887|65347|394229|1158|10887|24|12|2737|39|896.22|53.77|949.99|90.54|933.66|107.54|189.28|599.40|1077.97| +2452736|82543|14403|22903|1196629|1908|19587|22903|1196629|1908|19587|57|33|2737|77|996.38|0.00|996.38|64.55|1733.27|278.98|308.48|408.92|1797.82| +2452421|62605|11808|94134|1860773|6165|8804|94134|1860773|6165|8804|13|9|2739|29|366.85|18.34|385.19|42.08|1330.52|161.41|112.99|92.45|1390.94| +2452534|69540|15717|72091|401542|593|844|72091|401542|593|844|31|12|2739|13|91.13|3.64|94.77|40.60|18.59|89.30|0.18|1.65|62.83| +2451188|25895|17906|85549|1417124|1052|32627|85549|1417124|1052|32627|19|9|2740|27|969.03|48.45|1017.48|81.03|1139.94|784.91|162.02|22.10|1269.42| +2452725|23524|17241|41966|14029|6092|47355|41966|14029|6092|47355|60|5|2741|12|333.12|6.66|339.78|87.41|176.88|36.64|83.01|213.47|270.95| +||6516|64592|798115||45067|64592||1591|45067|7||2742||478.72|9.57||5.49||335.10|38.77||| +2451072|60264|13588|5929|216542|5202|25062|5929|216542|5202|25062|32|6|2748|53|894.64|71.57|966.21|79.48|1473.93|653.08|147.35|94.21|1624.98| +2451956|46346|13093|67547|490205|5216|47427|67547|490205|5216|47427|43|5|2750|1|1.28|0.11|1.39|29.68|0.56|0.39|0.40|0.49|30.35| +2451866||4610|||210||88578|1820394|210||8|7|2750|21||4.35|||||34.89|1.08|| +2451832|65545|10946|86391|194623|4008|16240|86391|194623|4008|16240|43|32|2751|21|152.88|1.52|154.40|36.46|9.45|120.77|20.87|11.24|47.43| +2451740|32070|3283|57885|742209|5380|17351|57885|742209|5380|17351|49|5|2751|66|2396.46|191.71|2588.17|66.80|1391.28|23.96|2372.50|0.00|1649.79| +2451635||5000||||26369|14180|991294||26369|||2751|6||||93.92|||10.85||| +|70975|5381|12192|||9560||||9560|14||2751|17||||37.30|||27.77||654.04| +2451710|65421|10792|65803|1831355|107|6812|65803|1831355|107|6812|16|27|2753|19|848.54|8.48|857.02|47.62|89.30|466.69|133.64|248.21|145.40| +2451565|32592|10760|48732|1822453|527|35448|48732|1822453|527|35448|10|23|2753|17|398.65|31.89|430.54|68.83|1362.04|295.00|27.98|75.67|1462.76| +2451053|38938|17746|6666|1266481|7096|42511|6666|1266481|7096|42511|52|19|2754|57|9031.65|722.53|9754.18|71.51|6152.01|8218.80|341.39|471.46|6946.05| +2452595|61566|12747|24755|1457658|3808|18711|24755|1457658|3808|18711|51|15|2755|1|139.80|6.99|146.79|32.27|22.71|110.44|3.81|25.55|61.97| +2452482|42946|16261|96954|64483|6059|33988|96954|64483|6059|33988|1|14|2756|3|25.74|0.77|26.51|41.18|2.31|2.31|21.78|1.65|44.26| +2452413|25034|17205|91658|1581989|1920|35851|91658|1581989|1920|35851|23|9|2756|1|21.30|1.06|22.36|74.64|35.50|13.63|4.90|2.77|111.20| +2452440|73468|9737|49130|1236564|4445|35879|49130|1236564|4445|35879|21|14|2756|11|66.99|6.02|73.01|12.55|111.76|38.85|20.82|7.32|130.33| +2451182|49831|14012|23118|1747451|4937|3270|23118|1747451|4937|3270|38|30|2758|3|62.94|3.14|66.08|23.80|86.55|61.05|0.24|1.65|113.49| +2451728|56306|236|93332|1523854|6823|4192|93332|1523854|6823|4192|43|12|2759|10|1001.30|20.02|1021.32|73.41|166.80|10.01|109.04|882.25|260.23| +2452768|49685|17844|1999|291675|1351|34227|1999|291675|1351|34227|12|14|2760|7|3.29|0.19|3.48|65.98|8.89|1.61|1.52|0.16|75.06| +2452644|70761|7513|5688|525708|128|10148|5688|525708|128|10148|51|23|2760|11|436.04|30.52|466.56|44.27|170.94|13.08|21.14|401.82|245.73| +2452737|50451|2202||1473507||||1473507|||25|15|2761|13||||||46.61|174.68|137.25|328.25| +2452737|73223|6957|25290|1115706|2569|23832|25290|1115706|2569|23832|51|28|2761|69|5366.82|107.33|5474.15|82.13|3056.01|2039.39|1497.34|1830.09|3245.47| +2452830|14776|4014|51486|1846375|5617|15706|51486|1846375|5617|15706|27|19|2761|25|1131.50|45.26|1176.76|71.83|1297.75|769.42|50.69|311.39|1414.84| +2452661|32828|8989|2188|950618|6248|48278|2188|950618|6248|48278|51|28|2761|2|24.50|0.00|24.50|65.12|14.70|7.35|2.05|15.10|79.82| +2451522|72005|4190|62932|899425|7118|22764|62932|899425|7118|22764|4|27|2762|15|1628.55|16.28|1644.83|97.87|140.70|195.42|214.96|1218.17|254.85| +2451531|65684|79|97509|347840|124|6455|97509|347840|124|6455|26|15|2762|2|72.04|5.76|77.80|56.17|196.14|18.01|50.78|3.25|258.07| +2451684|64022|7681|89830|849497|4811|15725|89830|849497|4811|15725|19|32|2763|7|185.15|3.70|188.85|28.73|53.90|64.80|114.33|6.02|86.33| +2451764|61744|9920|26112|926954|1734|37290|26112|926954|1734|37290|4|22|2763|9|58.86|0.00|58.86|99.60|38.61|6.47|10.47|41.92|138.21| +2452405|28227|13999|69004|1223677|5103|976|69004|1223677|5103|976|59|7|2764|8|137.12|10.96|148.08|87.38|124.64|112.43|23.70|0.99|222.98| +2452417|69180|2871|40657|330926|3242|14311|40657|330926|3242|14311|9|1|2764|1|28.85|1.15|30.00|91.28|16.67|15.86|7.92|5.07|109.10| +2451980|68557|7628|5613|1196756|5946|36336|5613|1196756|5946|36336|47|4|2765|52|3503.24|140.12|3643.36|59.27|469.04|3258.01|76.02|169.21|668.43| +2452706||12093|||||16786|1156468|6550||||2766|||336.63|||412.19|2230.19||1522.85|779.48| +2452676|65387|17665|84717|986461|1452|36443|84717|986461|1452|36443|48|20|2766|25|290.00|2.90|292.90|67.01|58.00|182.70|5.36|101.94|127.91| +2452332|43129|13363|22613|1069970|6127|40284|22613|1069970|6127|40284|33|14|2767|1|127.04|10.16|137.20|31.83|12.21|76.22|34.55|16.27|54.20| +2452407|67408|10351|78021|391047|2826|26652|78021|391047|2826|26652|47|14|2767|18|753.84|22.61|776.45|85.73|590.04|422.15|275.30|56.39|698.38| +2452393|25246|12247|74755|280197|3686|16328|74755|280197|3686|16328|39|6|2767|9|1341.90|67.09|1408.99|13.47|290.52|442.82|863.11|35.97|371.08| +|44879|3380||1792572|3147|24169|95150||3147||46|19|2768||794.52|0.00||||460.82|236.92||361.63| +2451857|28489|17978|57736|695512|488|26617|57736|695512|488|26617|50|24|2768|80|273.60|13.68|287.28|29.86|212.80|158.68|33.32|81.60|256.34| +|1596|14360|85029||5123|20817|||5123||58|6|2768|39||96.96|||468.78|360.17|71.75|953.36|640.86| +2452340|81289|6857|39583|829324|4665|3606|39583|829324|4665|3606|57|26|2769|5|297.15|5.94|303.09|70.15|157.75|267.43|21.99|7.73|233.84| +2452353|64296|17823|94120|1763051|3182|34513|94120|1763051|3182|34513|13|13|2770|47|2372.09|166.04|2538.13|93.36|1778.95|1067.44|613.18|691.47|2038.35| +2452433|77722|259|63287|210299|6111|11876|63287|210299|6111|11876|15|31|2770|14|1825.60|164.30|1989.90|90.97|387.94|36.51|930.32|858.77|643.21| +2452462|19131|3585|63501|1138118|2688|22905|63501|1138118|2688|22905|59|6|2777|9|15.66|1.09|16.75|33.13|11.16|2.03|2.72|10.91|45.38| +2452497|46739|12797|62351|856735|7119|29103|62351|856735|7119|29103|7|35|2777|1|58.39|0.58|58.97|63.57|99.48|2.91|49.37|6.11|163.63| +2452471|68824|2839|99278|1826816|7154|4739|99278|1826816|7154|4739|3|29|2777|50|188.00|16.92|204.92|97.53|21.50|133.48|28.35|26.17|135.95| +2452679|42043|6906|31294|727802|663|24324|31294|727802|663|24324|25|24|2778|23|449.19|26.95|476.14|92.63|389.39|98.82|339.85|10.52|508.97| +2452856|12692|11701|19399|1296965|6084|22850|19399|1296965|6084|22850|7|10|2778|91|3271.45|196.28|3467.73|22.61|3271.45|294.43|595.40|2381.62|3490.34| +2452057|63821|2779|73917|1283378|5192|46253|73917|1283378|5192|46253|2|21|2782|9|100.08|2.00|102.08|37.06|291.15|7.00|67.94|25.14|330.21| +2452150|17112|10073|21024|313836|1048|23460|21024|313836|1048|23460|26|27|2783|14|1307.88|52.31|1360.19|68.54|742.28|1203.24|17.78|86.86|863.13| +2452216|71332|14207|1702|1350514|1373|18185|1702|1350514|1373|18185|15|32|2785|83|12771.21|383.13|13154.34|97.79|3765.71|10472.39|666.65|1632.17|4246.63| +2452173|81149|2053|34949|616728|1707|5886|34949|616728|1707|5886|11|21|2786|1|13.85|0.00|13.85|88.97|30.68|12.60|0.55|0.70|119.65| +2452175|51809|15083|6079|409833|3634|13835|6079|409833|3634|13835|41|30|2786|40|16.00|0.48|16.48|41.89|61.60|12.96|1.82|1.22|103.97| +2452370|16881|8735|23564|1393803|6089|46728|23564|1393803|6089|46728|25|18|2787|33|1254.00|12.54|1266.54|4.54|284.79|25.08|1081.44|147.48|301.87| +2452282|37365|3343|89382|37186|637|3241|89382|37186|637|3241|17|28|2789|44|0.00|0.00|0.00|68.08|54.12|0.00|0.00|0.00|122.20| +2452311|46490|3899|80953|1802422|3891|37360|80953|1802422|3891|37360|9|9|2789|4|72.16|2.88|75.04|25.54|83.92|15.15|29.64|27.37|112.34| +2452196|83400|17299|18358|1613816|6874|12332|18358|1613816|6874|12332|5|35|2792|5|66.55|5.32|71.87|26.42|332.95|0.66|28.99|36.90|364.69| +2452221|62803|12011|51824|1562425|6179|14733|51824|1562425|6179|14733|2|29|2792|29|478.79|38.30|517.09|59.41|277.24|33.51|360.67|84.61|374.95| +2452209|47532|395|1851|909374|5697|2420|1851|909374|5697|2420|31|11|2792|41|4554.28|182.17|4736.45|42.48|236.57|2322.68|1651.38|580.22|461.22| +2451372|61796|7831|11568|522681|3041|32954|11568|522681|3041|32954|14|1|2793|20|485.00|29.10|514.10|59.86|90.40|281.30|175.18|28.52|179.36| +2451464|62230|12824|49136|1648974|12|1850|49136|1648974|12|1850|40|17|2793|45|1610.10|96.60|1706.70|86.26|656.55|418.62|702.97|488.51|839.41| +2452115|68466|16340|74797|1620827|1257|25287|74797|1620827|1257|25287|38|7|2794|44|1315.16|92.06|1407.22|76.94|3791.48|1288.85|22.10|4.21|3960.48| +2452683|49116|2917|85561|226976|4038|700|85561|226976|4038|700|30|18|2795|18|1687.86|0.00|1687.86|94.67|602.82|1063.35|118.65|505.86|697.49| +2451649|44328|674|13831|1287205|4579|38800|13831|1287205|4579|38800|56|25|2797|15|425.40|12.76|438.16|93.15|150.15|212.70|180.79|31.91|256.06| +2451988|74269|10055|9685|44091|2050|6056|9685|44091|2050|6056|37|13|2798|36|1017.72|81.41|1099.13|34.84|158.04|753.11|148.18|116.43|274.29| +2452795|65214|13831|50148|662178|7008|36991|50148|662178|7008|36991|1|20|2799|30|133.80|2.67|136.47|8.00|120.60|66.90|35.45|31.45|131.27| +2452530|30760|10539|37019|726121|6870|33972|37019|726121|6870|33972|42|30|2800|47|5126.76|256.33|5383.09|16.40|4222.01|4357.74|284.53|484.49|4494.74| +2452110|50079|13361|58505|849264|5270|42876|58505|849264|5270|42876|23|8|2801|27|3559.14|177.95|3737.09|26.62|1149.93|783.01|360.89|2415.24|1354.50| +2452038|45697|4279|69974|849493|5812|43731|69974|849493|5812|43731|2|12|2801|4|215.64|6.46|222.10|34.34|105.24|166.04|10.41|39.19|146.04| +2451862|65768|16159|54054|1496534|6567|1047|54054|1496534|6567|1047|41|16|2803|41|6881.03|0.00|6881.03|94.18|2515.35|5298.39|522.27|1060.37|2609.53| +2451067|42036|15736|88498|943916|6981|32046|88498|943916|6981|32046|10|9|2804|7|18.62|0.18|18.80|5.37|23.80|2.97|14.39|1.26|29.35| +2451119|12578|15373|84379|1559529|1879|23429|84379|1559529|1879|23429|46|31|2805|12|1507.92|105.55|1613.47|41.44|170.16|1507.92|0.00|0.00|317.15| +|31052|6254|||||||6069|44550|37|6|2806||8655.50|605.88|9261.38|86.45|6701.10||279.57|1191.87|| +2452090|8163|2261|66550|1130098|2799|28347|66550|1130098|2799|28347|2|7|2806|61|0.00|0.00|0.00|23.91|170.19|0.00|0.00|0.00|194.10| +|71911|12545|52543||7184|47881||||47881|41||2806||1102.80|55.14|1157.94||241.20||350.91||| +2451603|33408|11189|39851|1378884|6717|4197|39851|1378884|6717|4197|55|35|2807|1|62.32|3.73|66.05|52.69|45.08|46.11|12.15|4.06|101.50| +2451681||8516|||3416|11144||701641|3416||11|7|2807|26||||10.56|||||986.34| +2451902|12542|2813|38216|1323591|6119|25494|38216|1323591|6119|25494|44|22|2808|22|1956.90|136.98|2093.88|46.17|1823.36|1506.81|405.08|45.01|2006.51| +||13903|||||||7086||44|8|2808|2|82.72|0.00||16.88|||38.46||| +2451876|75218|7748|25738|969098|3658|2218|25738|969098|3658|2218|32|16|2808|40|1324.00|105.92|1429.92|86.91|1624.80|251.56|718.53|353.91|1817.63| +2451552|28988|13897|43427|1196509|2744|23026|43427|1196509|2744|23026|4|12|2809|2|37.08|2.22|39.30|80.65|29.90|8.89|27.62|0.57|112.77| +2452403|36633|16299|58726|1284464|3355|2293|58726|1284464|3355|2293|17|7|2810|34|2441.20|219.70|2660.90|77.44|1723.12|1611.19|41.50|788.51|2020.26| +2452344|64476|14311|28454|1200981|3908|15635|28454|1200981|3908|15635|41|22|2810|23|1626.56|81.32|1707.88|87.26|1114.12|553.03|418.67|654.86|1282.70| +|16221|5579|41169|1537283|2550|14264|41169|||||31|2810||||2682.21|76.80||1593.39||573.63|1340.67| +2451268|14218|10940|66536|1744625|1114|36803|66536|1744625|1114|36803|55|20|2811|88|6379.12|0.00|6379.12|72.10|975.92|2168.90|1389.37|2820.85|1048.02| +2451324|63014|10939|12315|1152110|4352|36547|12315|1152110|4352|36547|34|33|2811|7|787.78|15.75|803.53|72.56|465.50|173.31|436.27|178.20|553.81| +2452655|69703|13563|96938|1275419|1900|12428|96938|1275419|1900|12428|54|15|2812|64|1569.92|141.29|1711.21|13.93|407.68|156.99|1059.69|353.24|562.90| +2452741|71023|13815|61675|145178|5481|47737|61675|145178|5481|47737|36|11|2812|14|230.58|16.14|246.72|30.92|302.68|140.65|44.96|44.97|349.74| +2452799|27880|17053|77230|1647963|5096|27429|77230|1647963|5096|27429|60|30|2814|36|1445.76|86.74|1532.50|20.27|540.00|852.99|361.58|231.19|647.01| +2452404|23972|5397|83797|546579|4208|29437|83797|546579|4208|29437|27|10|2816|17|123.93|3.71|127.64|15.94|82.62|6.19|45.91|71.83|102.27| +2452348|26460|4291|36727|1174018|2509|47800|36727|1174018|2509|47800|17|11|2816|4|384.72|19.23|403.95|17.44|52.64|69.24|223.99|91.49|89.31| +2452516|45257|1821|26965|1184445|5572|32252|26965|1184445|5572|32252|29|8|2816|6|419.04|0.00|419.04|22.27|86.70|205.32|192.34|21.38|108.97| +2452301|32009|9871|29643|1641790|1937|19331|29643|1641790|1937|19331|49|22|2817|14|111.86|7.83|119.69|74.03|63.98|71.59|19.32|20.95|145.84| +2452241|80808|9387|66019|75384|1196|44961|66019|75384|1196|44961|59|25|2817|3|142.89|2.85|145.74|74.60|54.54|81.44|21.50|39.95|131.99| +2451169|54843|14803|59031|1498360|4048|47284|59031|1498360|4048|47284|49|25|2818|13|635.70|25.42|661.12|27.11|225.94|432.27|44.75|158.68|278.47| +2451224|36426|14644|8829|1802592|708|471|8829|1802592|708|471|28|28|2820|13|108.68|3.26|111.94|41.20|10.40|51.07|47.81|9.80|54.86| +2451387|78768|17107|21227|362945|1816|45087|21227|362945|1816|45087|32|30|2822|20|90.00|0.90|90.90|64.18|15.60|64.80|6.30|18.90|80.68| +2452793|75980|11970|36118|976475|2432|30389|36118|976475|2432|30389|55|23|2823|48|1670.88|0.00|1670.88|34.88|428.16|1219.74|171.43|279.71|463.04| +2452699|71377|11646|33410|482775|5293|47058|33410|482775|5293|47058|3|32|2823|3|61.92|3.71|65.63|44.39|11.25|14.86|30.58|16.48|59.35| +2452889|23947|9183|26383|1796882|981|7527|26383|1796882|981|7527|15|3|2824|14|31.22|1.24|32.46|29.51|909.16|19.66|6.58|4.98|939.91| +2452425|45398|1145|96972|89531|2090|46296|96972|89531|2090|46296|35|32|2825|70|365.40|29.23|394.63|90.39|1369.90|317.89|34.68|12.83|1489.52| +2452157|65591|14723|36618|233488|3605|10522|36618|233488|3605|10522|27|12|2826|49|1167.18|11.67|1178.85|10.90|0.00|898.72|174.49|93.97|22.57| +2452076|75911|8713|18344|757972|353|15581|18344|757972|353|15581|13|10|2826|12|205.44|2.05|207.49|81.84|311.64|100.66|82.77|22.01|395.53| +2452202|11558|2397|21872|1419800|3681|9209|21872|1419800|3681|9209|49|31|2826|7|255.08|10.20|265.28|89.49|31.85|45.91|27.19|181.98|131.54| +2452544|5946|17407|47362|144837|5366|47254|47362|144837|5366|47254|25|27|2827|44|205.48|10.27|215.75|7.53|93.28|57.53|20.71|127.24|111.08| +2451192|33703|10111|1755|1688853|398|25543|1755|1688853|398|25543|26|29|2831|43|1729.03|69.16|1798.19|92.67|2564.09|1417.80|199.18|112.05|2725.92| +2450915|69514|17725|5126|1283554|7135|46716|5126|1283554|7135|46716|7|24|2831|2|40.20|1.20|41.40|91.00|18.66|27.73|1.87|10.60|110.86| +2450894|38068|9931|39563|1612381|1999|38817|39563|1612381|1999|38817|40|15|2831|71|3160.92|284.48|3445.40|58.47|939.33|410.91|2200.00|550.01|1282.28| +2451061|12746|9992|21215|690837|5612|49634|21215|690837|5612|49634|37|19|2833|33|3852.75|308.22|4160.97|59.20|1703.13|3274.83|277.40|300.52|2070.55| +2451236|68053|6367|47790|1033704|4418|32732|47790|1033704|4418|32732|34|18|2835|10|141.10|5.64|146.74|82.39|47.60|128.40|6.60|6.10|135.63| +2452321|76052|14755|72146|190889|2879|15269|72146|190889|2879|15269|35|20|2836|3|412.86|12.38|425.24|98.25|17.37|408.73|2.68|1.45|128.00| +2452101||6857||||41330|38506|606514|922|41330|||2840||||2535.30||||||| +2451944|86021|17477|31968|1078184|47|1063|31968|1078184|47|1063|55|22|2840|13|178.36|14.26|192.62|53.31|345.67|7.13|56.50|114.73|413.24| +||2869|||4832|19991||||19991|1||2840|||||||||213.15|| +2451290|2612|3992|81833|508232|4678|26677|81833|508232|4678|26677|4|35|2841|9|18.45|0.55|19.00|34.38|69.48|1.29|9.60|7.56|104.41| +2451319|82476|16004|42154|736208|3862|23916|42154|736208|3862|23916|31|10|2841|53|313.76|15.68|329.44|86.69|470.64|263.55|37.65|12.56|573.01| +2451297|70914|1348|93078|1022957|3236|45115|93078|1022957|3236|45115|56|26|2841|1|9.81|0.88|10.69|22.82|53.45|0.00|9.02|0.79|77.15| +2451354|27343|2266|93117|1635734|2704|20921|93117|1635734|2704|20921|20|7|2841|37|2139.71|171.17|2310.88|53.45|485.07|1861.54|66.76|211.41|709.69| +2451766||15200||991377|4154||18600||4154||22||2844|34||0.00|6251.92|11.12||4751.45||45.02|| +2451282|63572|7280|48187|1821219|7051|11330|48187|1821219|7051|11330|10|21|2845|48|2167.20|43.34|2210.54|71.45|1096.32|953.56|121.36|1092.28|1211.11| +2451327|24268|68|99293|1081864|3635|34606|99293|1081864|3635|34606|7|28|2848|33|2394.48|119.72|2514.20|6.03|1197.24|1316.96|226.27|851.25|1322.99| +2451408|45776|2606|55912|142682|3499|14716|55912|142682|3499|14716|40|16|2849|57|562.02|0.00|562.02|72.55|340.29|556.39|5.63|0.00|412.84| +||6374|8040||4174||||||||2849||1088.64|||||||359.26|2535.41| +2451622|38315|6716|2087|817177|3114|12204|2087|817177|3114|12204|19|28|2851|11|1205.27|84.36|1289.63|19.92|751.30|72.31|305.89|827.07|855.58| +2452537|26387|1659|17294|468347|6153|43559|17294|468347|6153|43559|25|12|2852|49|662.48|13.24|675.72|50.54|165.62|536.60|112.03|13.85|229.40| +2452741|34914|1209|56282|35435|988|36141|56282|35435|988|36141|7|16|2854|25|534.25|16.02|550.27|97.30|248.50|518.22|6.73|9.30|361.82| +2452824|40908|15240|22361|387066|5371|41676|22361|387066|5371|41676|19|14|2854|9|643.59|6.43|650.02|6.02|259.11|173.76|84.56|385.27|271.56| +2451488|79902|17110|20129|1768932|4582|39832|20129|1768932|4582|39832|52|23|2855|16|1250.72|62.53|1313.25|3.17|585.44|75.04|928.78|246.90|651.14| +2451547|71976|12493|93598|138388|5931|9521|93598|138388|5931|9521|1|21|2855|47|1873.89|74.95|1948.84|42.58|1018.49|93.69|1335.15|445.05|1136.02| +2451399|70478|481|48100|851537|3774|411|48100|851537|3774|411|26|2|2856|12|542.52|16.27|558.79|85.24|58.92|151.90|124.99|265.63|160.43| +2452048|17638|15619|88674|1728042|5457|46425|88674|1728042|5457|46425|17|12|2858|11|444.18|0.00|444.18|35.08|327.80|359.78|30.38|54.02|362.88| +2451861||11051|32609|926663||5971|32609||5517||||2861|||||84.75||1187.90|0.00|0.00|242.44| +2452082|13550|4351|21238|1385198|4005|17212|21238|1385198|4005|17212|50|30|2862|17|613.36|12.26|625.62|37.38|431.29|67.46|382.13|163.77|480.93| +2452073|15939|11489|97311|186215|6766|27270|97311|186215|6766|27270|53|27|2862|13|1694.29|16.94|1711.23|98.11|97.63|457.45|803.94|432.90|212.68| +|25997|11119|||992|36328|96814||||||2862||||239.77|||30.55||137.03|| +2451929|70599|2699|50625|70168|911|41701|50625|70168|911|41701|32|28|2863|13|72.54|5.80|78.34|41.08|114.01|57.30|1.98|13.26|160.89| +2452139|50166|12067|19133|1481260|3598|41668|19133|1481260|3598|41668|2|34|2864|1|40.59|0.40|40.99|87.68|46.53|3.24|37.35|0.00|134.61| +2452724|38694|7489|60765|831557|3949|37188|60765|831557|3949|37188|60|2|2865|60|2320.80|69.62|2390.42|65.14|1134.60|1183.60|1125.82|11.38|1269.36| +2452674|24850|6690|61072|1786676|1527|14331|61072|1786676|1527|14331|33|1|2865|19|3060.14|30.60|3090.74|97.02|446.12|2845.93|212.06|2.15|573.74| +2451910|72416|13847|2468|1879842|7169|43743|2468|1879842|7169|43743|43|20|2866|2|161.30|4.83|166.13|42.61|55.44|77.42|59.55|24.33|102.88| +2452005|66275|200|19419|742712|405|10898|19419|742712|405|10898|13|28|2868|51|568.14|11.36|579.50|64.57|1022.55|482.91|34.09|51.14|1098.48| +2451956|47655|15392|52315|622981|3600|6666|52315|622981|3600|6666|43|10|2869|24|247.92|2.47|250.39|89.97|30.96|96.68|145.19|6.05|123.40| +2452136|42168|889|23560|765572|5984|796|23560|765572|5984|796|53|18|2869|1|72.83|3.64|76.47|87.77|6.06|32.77|2.40|37.66|97.47| +2451943|46587|10118|69103|1831524|2392|44060|69103|1831524|2392|44060|25|26|2869|4|137.60|6.88|144.48|24.44|367.00|74.30|11.39|51.91|398.32| +2452161|52392|12697|66833|175927|2174|19621|66833|175927|2174|19621|47|9|2869|43|3882.04|77.64|3959.68|96.73|287.24|2018.66|819.88|1043.50|461.61| +2452393|54413|10595|8857|1290265|26|28234|8857|1290265|26|28234|57|4|2871|84|2774.52|83.23|2857.75|78.42|1672.44|305.19|49.38|2419.95|1834.09| +2452530|45260|12858|24104|1425253|5430|37799|24104|1425253|5430|37799|12|35|2872|6|36.12|0.36|36.48|84.31|180.84|19.50|4.81|11.81|265.51| +2452538|41808|9456|22274|185920|3166|33165|22274|185920|3166|33165|15|21|2872|1|56.32|5.06|61.38|41.07|82.14|7.88|3.39|45.05|128.27| +2452272|67683|4381|36099|1452069|5286|30456|36099|1452069|5286|30456|31|32|2875|3|4.23|0.25|4.48|88.69|16.20|1.14|2.90|0.19|105.14| +2451617|32493|3121|63584|1225198|1565|17179|63584|1225198|1565|17179|14|33|2876|4|290.64|26.15|316.79|9.24|135.28|46.50|129.39|114.75|170.67| +2452659|70089|10452|13020|557679|1538|29797|13020|557679|1538|29797|15|16|2877|7|117.95|8.25|126.20|40.33|54.04|20.05|94.96|2.94|102.62| +2451021|59971|8620|58380|1538627|6427|17827|58380|1538627|6427|17827|49|29|2879|5|214.35|10.71|225.06|28.92|89.00|197.20|7.20|9.95|128.63| +2451150|25069|4718|78186|910870|2401|27885|78186|910870|2401|27885|2|12|2880|17|85.85|7.72|93.57|68.31|45.73|50.65|30.62|4.58|121.76| +2451505|66163|740|10502|1350573|1397|17068|10502|1350573|1397|17068|32|17|2881|55|825.55|0.00|825.55|94.66|777.15|33.02|792.53|0.00|871.81| +2451296|39703|3326|75825|5640|702|33364|75825|5640|702|33364|58|15|2881|62|4431.76|44.31|4476.07|66.53|104.16|2038.60|1555.55|837.61|215.00| +2452801|34513|1974|46353|626325|5144|316|46353|626325|5144|316|37|34|2882|78|2792.40|55.84|2848.24|41.91|943.80|2764.47|3.91|24.02|1041.55| +2451933|33144|7892|11716|1888358|1800|35595|11716|1888358|1800|35595|31|22|2884|9|592.11|5.92|598.03|77.51|135.09|497.37|45.47|49.27|218.52| +2451961|17870|6920|33616|1852868|120|29325|33616|1852868|120|29325|20|10|2884|25|386.00|11.58|397.58|46.34|140.25|239.32|61.60|85.08|198.17| +||4013|5941|257851||||257851||27765|||2885|||8.56||98.71||4.28|29.66|394.18|257.53| +2452774|46955|7545|16246|1506133|6420|8354|16246|1506133|6420|8354|12|20|2889|62|944.88|9.44|954.32|76.01|453.22|387.40|61.32|496.16|538.67| +2452845|6774|6762|4095|1200634|6773|3712|4095|1200634|6773|3712|24|34|2889|45|1272.60|12.72|1285.32|12.65|967.05|318.15|133.62|820.83|992.42| +2451979|82690|16481|44562|1781810|4658|48706|44562|1781810|4658|48706|44|14|2890|7|3.22|0.19|3.41|11.66|94.78|0.70|2.24|0.28|106.63| +2451924|45355|10046|44036|628981|3973|20118|44036|628981|3973|20118|43|6|2890|67|660.62|6.60|667.22|76.05|206.36|587.95|57.40|15.27|289.01| +|37942|16243||1840917|4778||43809|1840917||||7|2891|24|133.44|9.34|||||6.00|54.05|753.77| +2452326|42331|16709|82388|1820880|172|11864|82388|1820880|172|11864|33|19|2892|52|249.08|7.47|256.55|97.60|407.68|57.28|15.34|176.46|512.75| +2452399|63074|8871|34508|1635306|3751|10549|34508|1635306|3751|10549|45|28|2892|54|4250.34|85.00|4335.34|60.32|971.46|892.57|3156.30|201.47|1116.78| +2452344|53316|15327|1023|1180058|6293|28528|1023|1180058|6293|28528|27|31|2892|96|13280.64|265.61|13546.25|82.24|155.52|5843.48|1933.66|5503.50|503.37| +|69150|11451||324330|||||1115|35453|31|29|2892|||50.94||||||61.14|| +2451238|32629|12374|50748|148178|6055|29485|50748|148178|6055|29485|4|4|2893|25|2026.00|182.34|2208.34|62.70|1256.00|749.62|612.66|663.72|1501.04| +2451497|9858|17314|84732|640267|602|49737|84732|640267|602|49737|46|22|2893|9|1258.56|12.58|1271.14|37.52|1098.36|1208.21|40.78|9.57|1148.46| +2451290|69094|12308|26075|1470209|1096|6383|26075|1470209|1096|6383|28|32|2893|47|3495.39|209.72|3705.11|93.68|1261.95|69.90|2432.09|993.40|1565.35| +2451431|68335|722|91018|1340919|1413|36665|91018|1340919|1413|36665|2|22|2893|1|6.69|0.00|6.69|11.91|2.02|2.74|1.26|2.69|13.93| +2451709|75899|12736|87933|602406|879|42152|87933|602406|879|42152|37|22|2894|24|0.00|0.00|0.00|23.61|212.64|0.00|0.00|0.00|236.25| +2451415|67627|17569|77879|349500|6721|14742|77879|349500|6721|14742|55|18|2895|41|1974.15|98.70|2072.85|37.59|267.32|256.63|257.62|1459.90|403.61| +2452798|65963|7545|82353|1208014|2122|26583|82353|1208014|2122|26583|42|16|2896|18|196.92|5.90|202.82|99.93|65.52|49.23|57.59|90.10|171.35| +2452752|46296|6762|24214|851814|5367|28567|24214|851814|5367|28567|49|34|2896|23|1903.94|152.31|2056.25|83.58|1459.58|266.55|1539.14|98.25|1695.47| +2451838|6261|16466|18726|690174|4822|16270|18726|690174|4822|16270|26|28|2897|1|82.18|7.39|89.57|45.39|14.67|42.73|28.00|11.45|67.45| +2451874||8809||938217||21985|10358|||21985|||2897||||3765.75|||1349.98|||| +2451197|66677|10336|62369|200331|4693|22255|62369|200331|4693|22255|43|7|2898|11|1165.01|11.65|1176.66|77.24|771.21|34.95|22.60|1107.46|860.10| +2451152|18493|14011|40056|931344|3879|10767|40056|931344|3879|10767|43|27|2898|45|2445.30|146.71|2592.01|66.52|73.80|2249.67|84.12|111.51|287.03| +2451156|37213|3673|11674|1503007|293|35226|11674|1503007|293|35226|34|35|2899|1|82.69|3.30|85.99|34.17|12.48|81.03|1.46|0.20|49.95| +2451411|23686|6331|2140|737566|3839|22065|2140|737566|3839|22065|20|22|2899|52|2544.36|25.44|2569.80|76.31|1073.28|50.88|2393.74|99.74|1175.03| +2452462|65563|12900|20028|545573|1071|5816|20028|545573|1071|5816|43|3|2901|1|68.69|0.68|69.37|2.31|20.69|14.42|36.90|17.37|23.68| +2452650|85373|3993|57497|290351|1252|14153|57497|290351|1252|14153|45|5|2901|79|7821.00|469.26|8290.26|76.15|1501.00|703.89|3060.35|4056.76|2046.41| +2452048|63988|4805|37261|566078|4942|40559|37261|566078|4942|40559|59|10|2902|80|1356.00|0.00|1356.00|10.66|2938.40|745.80|543.07|67.13|2949.06| +||13550||1279277|5312|||||32994|||2902|||||||||44.57|| +2451337|30830|15793|8460|1790893|1326|24959|8460|1790893|1326|24959|52|22|2903|2|27.60|0.82|28.42|15.33|7.36|18.21|6.38|3.01|23.51| +2451254|36893|15152|17818|391490|5609|6081|17818|391490|5609|6081|58|30|2903|34|4082.04|367.38|4449.42|20.20|177.48|1224.61|714.35|2143.08|565.06| +2452372|24090|9025|53245|1868669|2046|45870|53245|1868669|2046|45870|9|8|2906|28|72.24|4.33|76.57|25.44|521.36|30.34|33.93|7.97|551.13| +2452092|39370|4657|59950|1158531|5559|7740|59950|1158531|5559|7740|13|34|2907|43|4515.86|0.00|4515.86|4.76|733.58|3432.05|899.56|184.25|738.34| +2451874|44019|3163|82414|547951|4688|15025|82414|547951|4688|15025|38|35|2907|25|1749.00|52.47|1801.47|87.00|636.00|1329.24|386.17|33.59|775.47| +2451202|65535|10462|40092|1202089|886|6836|40092|1202089|886|6836|44|1|2908|18|46.26|1.38|47.64|16.62|347.40|25.44|3.33|17.49|365.40| +2452094|29284|16507|43557|1080155|4400|6998|43557|1080155|4400|6998|25|20|2909|12|7.80|0.39|8.19|54.08|63.96|7.02|0.69|0.09|118.43| +2452137|45845|9337|29890|1064733|6423|7266|29890|1064733|6423|7266|53|23|2909|33|2549.58|0.00|2549.58|61.54|1448.70|178.47|118.55|2252.56|1510.24| +2452008|25797|11043|57255|642195|5982|17369|57255|642195|5982|17369|51|4|2909|5|183.95|0.00|183.95|84.89|28.90|158.19|13.91|11.85|113.79| +2452078|19280|1369|40826|1283183|3493|29403|40826|1283183|3493|29403|7|8|2909|72|5335.20|373.46|5708.66|45.27|984.96|373.46|694.64|4267.10|1403.69| +2452027|55398|12191|24135|996604|2320|3989|24135|996604|2320|3989|50|7|2910|45|499.50|9.99|509.49|65.07|83.25|274.72|24.72|200.06|158.31| +2451381|23394|9388|91785|746422|2659|8096|91785|746422|2659|8096|13|21|2911|7|330.33|13.21|343.54|80.64|117.32|49.54|131.97|148.82|211.17| +2452837|33178|9775|9854|1387989|5914|25113|9854|1387989|5914|25113|25|4|2912|4|615.60|43.09|658.69|45.38|248.84|597.13|10.89|7.58|337.31| +2451240||11392|86959|1699698|6037|1976|86959||||56|20|2914||||||69.39|24.98|88.76|25.04|| +2451183|59940|9608|51800|401994|188|47582|51800|401994|188|47582|56|11|2914|4|308.64|12.34|320.98|68.31|149.88|299.38|8.42|0.84|230.53| +2452293|72491|6959|78604|919177|2150|28676|78604|919177|2150|28676|23|32|2915|10|42.20|0.42|42.62|29.42|496.40|37.55|1.25|3.40|526.24| +2452439|14857|363|35958|938814|2823|32|35958|938814|2823|32|47|18|2915|46|7592.76|379.63|7972.39|39.32|1641.28|5846.42|69.85|1676.49|2060.23| +2452705|24714|6582|1966|1759459|1752|48045|1966|1759459|1752|48045|49|28|2916|8|14.72|1.17|15.89|8.68|19.76|10.00|1.98|2.74|29.61| +2452877|35914|12996|70281|777227|3373|3119|70281|777227|3373|3119|48|13|2916|62|2242.54|156.97|2399.51|28.25|174.84|695.18|835.57|711.79|360.06| +2452812|73785|5184|26877|1596098|872|40046|26877|1596098|872|40046|43|24|2916|8|49.20|0.98|50.18|85.16|14.80|21.64|25.63|1.93|100.94| +2452894|46339|16561|69248|1395608|15|32775|69248|1395608|15|32775|51|19|2916|17|820.08|32.80|852.88|28.45|1284.69|41.00|483.02|296.06|1345.94| +2452394|75058|16567|31969|1275347|3525|2103|31969|1275347|3525|2103|5|13|2917|31|3224.93|96.74|3321.67|70.83|2170.62|2870.18|35.47|319.28|2338.19| +2452411|69112|7553|97580|1107634|1241|39295|97580|1107634|1241|39295|49|27|2917|6|570.78|34.24|605.02|5.94|127.50|114.15|251.14|205.49|167.68| +2452340|12529|2905|67770|278525|4833|3188|67770|278525|4833|3188|15|15|2918|65|0.00|0.00|0.00|64.68|5272.80|0.00|0.00|0.00|5337.48| +2452413|26121|9909|99738|1493115|2939|13563|99738|1493115|2939|13563|59|23|2918|20|998.60|9.98|1008.58|3.75|13.60|968.64|26.36|3.60|27.33| +2452339|58706|15845|74329|1713711|5093|10075|74329|1713711|5093|10075|39|5|2918|3|5.76|0.51|6.27|36.34|9.66|0.05|2.28|3.43|46.51| +2451307|28628|12646|18644|127230|1416|21202|18644|127230|1416|21202|22|31|2919|8|28.08|0.28|28.36|60.89|11.04|16.56|9.10|2.42|72.21| +2452565|67188|1173|78389|1808813|3237|27261||||27261|43|2|2920||||11.31|44.84||9.68||0.69|53.43| +2452602|24384|4446|32734|1728892|6650|23608|32734|1728892|6650|23608|42|33|2921|1|6.32|0.44|6.76|33.68|58.47|3.66|1.64|1.02|92.59| +2451818|48449|1621|5777|1805913|2|37738|5777|1805913|2|37738|13|31|2923|17|2521.27|25.21|2546.48|48.12|542.98|2496.05|16.14|9.08|616.31| +2451703|12547|10804|65887|787171|5702|48186|65887|787171|5702|48186|56|10|2923|38|0.00|0.00|0.00|39.42|1706.96|0.00|0.00|0.00|1746.38| +2451989|43820|5840|5777|1743208|3839|27717|5777|1743208|3839|27717|55|10|2924|2|78.50|6.28|84.78|96.90|27.58|48.67|5.07|24.76|130.76| +2451911|66919|4334|72976|606350|6477|17051|72976|606350|6477|17051|50|14|2924|48|171.84|3.43|175.27|76.53|563.52|80.76|7.28|83.80|643.48| +2451965|70557|15320|29748|902441|1534|41427|29748|902441|1534|41427|19|22|2924|20|381.60|11.44|393.04|68.83|82.40|270.93|28.77|81.90|162.67| +2452327|26596|6557|46630|532507|1231|9508|46630|532507|1231|9508|23|11|2925|29|1749.28|0.00|1749.28|79.54|821.86|594.75|761.98|392.55|901.40| +2452305|64766|17845|17477|1125338|6446|30307|17477|1125338|6446|30307|45|32|2925|4|219.00|8.76|227.76|45.38|117.12|96.36|116.50|6.14|171.26| +2451724|64869|10522|17407|58431|608|49185|17407|58431|608|49185|25|22|2926|60|5694.60|284.73|5979.33|28.57|3148.80|227.78|4100.11|1366.71|3462.10| +2451681|16024|11377|36970|69997|2671|1345|36970|69997|2671|1345|10|12|2926|8|80.96|1.61|82.57|96.88|269.84|61.52|18.27|1.17|368.33| +2452242|19131|13029|74423|1127939|5808|31724|74423|1127939|5808|31724|41|16|2927|71|2493.52|0.00|2493.52|12.71|1324.15|723.12|1221.57|548.83|1336.86| +2452172||11245|76925|1536804|5836|34068|76925||5836|||29|2927|14|1305.78|||5.86||548.42|280.22||| +2452126|42292|3517|23821|100200|579|7758|23821|100200|579|7758|23|15|2927|5|1.75|0.08|1.83|21.42|10.55|0.91|0.20|0.64|32.05| +2452856|45709|7641|20965|19768|6571|15374|20965|19768|6571|15374|21|25|2928|13|621.14|12.42|633.56|47.98|751.92|322.99|44.72|253.43|812.32| +2451285|75952|17776|59684|733500|5363|42826|59684|733500|5363|42826|46|24|2929|5|69.30|0.00|69.30|51.19|0.00|0.69|56.26|12.35|51.19| +2451409|74180|2863|17344|421485|6426|18520|17344|421485|6426|18520|50|2|2929|2|33.52|2.68|36.20|59.14|9.70|24.13|6.38|3.01|71.52| +2451233|69681|1243|58331|1083478|4625|20219|58331|1083478|4625|20219|49|16|2931|47|1726.78|69.07|1795.85|98.75|1104.97|1243.28|483.50|0.00|1272.79| +2451190|73020|12367|15479|943525|5171|24817|15479|943525|5171|24817|31|15|2931|12|21.36|1.70|23.06|97.79|6.96|17.94|1.33|2.09|106.45| +2451442|76469|6277|91847|313791|4535|17279|91847|313791|4535|17279|19|33|2932|2|248.72|9.94|258.66|58.34|76.98|218.87|14.02|15.83|145.26| +2451362|71127|13298|7807|735232|4257|23235|7807|735232|4257|23235|7|13|2932|78|464.10|23.20|487.30|17.24|237.12|37.12|132.36|294.62|277.56| +2451418|44088|9016|74518|1464607|5309|34968|74518|1464607|5309|34968|49|11|2932|42|179.76|16.17|195.93|44.20|1258.74|165.37|1.87|12.52|1319.11| +2451723|48492|15254|72448|1631574|4653|26932|72448|1631574|4653|26932|20|21|2935|20|123.60|11.12|134.72|87.53|39.80|90.22|21.69|11.69|138.45| +||17413|60272|726867|6709||60272|||||5|2937||||2079.16|21.49||||778.45|2100.65| +2451430|69746|2263|98645|1916430|5862|3647|98645|1916430|5862|3647|19|13|2938|6|929.34|27.88|957.22|20.57|185.82|278.80|468.38|182.16|234.27| +2452874|41476|15523|65426|393246|1066|40133|65426|393246|1066|40133|45|30|2940|45|1738.35|86.91|1825.26|34.43|701.10|1182.07|495.08|61.20|822.44| +2452626|57386|13303|10961|1526197|5880|316|10961|1526197|5880|316|31|11|2940|5|168.60|8.43|177.03|93.29|74.50|28.66|69.97|69.97|176.22| +2452711|49150|13428|8581|619184|1340|33543|8581|619184|1340|33543|1|33|2940|55|4728.35|283.70|5012.05|38.85|3994.65|2364.17|1087.52|1276.66|4317.20| +2452723|29338|1188|91672|1176546|4267|26341|91672|1176546|4267|26341|43|29|2940|7|318.71|0.00|318.71|79.82|133.28|299.58|12.81|6.32|213.10| +2452368|30653|3759|72779|1761713|6200|33534|72779|1761713|6200|33534|29|10|2941|27|605.07|30.25|635.32|82.24|351.00|242.02|315.85|47.20|463.49| +2451000|84721|812|53162|341803|2444|19485|53162|341803|2444|19485|40|24|2945|6|142.74|0.00|142.74|64.24|51.90|107.05|27.12|8.57|116.14| +2451593|81389|1969|67342|859574|5406|40838|67342|859574|5406|40838|19|13|2949|2|13.78|0.41|14.19|84.01|46.28|2.61|1.56|9.61|130.70| +2451702|25167|1537|73501|1499311|321|34578|73501|1499311|321|34578|34|10|2949|16|489.92|44.09|534.01|92.96|180.48|362.54|21.65|105.73|317.53| +2451776|68979|11977|11959|1198975|6288|26864|11959|1198975|6288|26864|47|34|2951|7|540.12|27.00|567.12|47.20|245.00|172.83|44.07|323.22|319.20| +2451286|38390|17372|6137|1233270|5171|40573|6137|1233270|5171|40573|34|2|2952|49|1350.93|108.07|1459.00|39.83|84.28|1350.93|0.00|0.00|232.18| +2452697|60336|4101|75556|837590|2024|11258|75556|837590|2024|11258|7|2|2953|12|1217.76|109.59|1327.35|81.25|71.52|584.52|329.28|303.96|262.36| +2452847|58949|7651|1150|946308|4713|20271|1150|946308|4713|20271|27|15|2954|4|41.68|2.50|44.18|19.22|106.16|20.00|6.72|14.96|127.88| +2451225|68634|10945|4746|1834849|2199|12349|4746|1834849|2199|12349|28|34|2955|15|524.55|26.22|550.77|65.04|1049.10|262.27|249.16|13.12|1140.36| +2451189|61163|1450|||4776||56982|1357679|4776||||2955|36|||1475.48|15.18|281.52|1037.22|||| +2451168|42420|2618|29258|992456|7|44358|29258|992456|7|44358|2|11|2955|48|574.08|11.48|585.56|83.17|2088.48|459.26|4.59|110.23|2183.13| +2452498|86297|471|37133|971952|2771|3590|37133|971952|2771|3590|59|27|2956|9|752.13|0.00|752.13|50.28|569.79|707.00|42.87|2.26|620.07| +2452347|37928|14535|1275|914018|8|7598|1275|914018|8|7598|27|2|2956|22|120.56|10.85|131.41|72.13|30.14|103.68|3.20|13.68|113.12| +2452008|64843|9350|53750|918550|4950|49350|53750|918550|4950|49350|49|28|2957|39|2343.12|117.15|2460.27|43.29|360.36|1569.89|525.79|247.44|520.80| +2452733|25170|13209|8539|1723528|1364|37953|8539|1723528|1364|37953|24|13|2958|20|1862.40|111.74|1974.14|31.35|20.00|800.83|116.77|944.80|163.09| +2451892|15790|1583|13631|1181281|2531|34981|13631|1181281|2531|34981|19|1|2959|32|1089.60|0.00|1089.60|89.58|600.32|795.40|285.37|8.83|689.90| +2451008|10469|7610|64227|609683|1913|7857|64227|609683|1913|7857|52|35|2960|9|856.80|17.13|873.93|61.52|291.24|848.23|4.45|4.12|369.89| +2452249|43315|9035|9463|574360|6810|18907|9463|574360|6810|18907|11|23|2961|18|323.28|0.00|323.28|56.45|272.88|48.49|68.69|206.10|329.33| +2451247|19576|14768|23641|511148|7102|14209|23641|511148|7102|14209|37|35|2962|13|102.70|6.16|108.86|11.05|94.77|55.45|21.26|25.99|111.98| +|61224|7342||||48958|29831||1100|48958|10||2962|33|324.72|6.49||14.99|1110.12|194.83||62.35|1131.60| +2451247|54887|15182|79170|1124921|2220|21930|79170|1124921|2220|21930|25|10|2962|25|221.00|2.21|223.21|66.68|164.50|119.34|36.59|65.07|233.39| +2451894|36380|6229|44559|53692|3978|21511|44559|53692|3978|21511|26|1|2964|73|6364.87|190.94|6555.81|77.32|4455.19|1654.86|3579.60|1130.41|4723.45| +2451939|64601|15362|59501|1713464|4227|27790|59501|1713464|4227|27790|17|5|2964|3|33.69|1.01|34.70|58.36|6.72|32.00|0.52|1.17|66.09| +2451426|77197|8614||458797||32620||458797||32620|58|20|2965|24||1.87|39.31|63.08||||17.98|259.59| +2451595|66039|4951|8155|363808|6214|22267|8155|363808|6214|22267|56|26|2965|1|27.19|0.27|27.46|39.51|43.51|25.83|0.50|0.86|83.29| +2451620|79235|7264|24762|92676|590|39657|24762|92676|590|39657|28|16|2965|57|6142.32|429.96|6572.28|24.84|2656.20|4975.27|513.50|653.55|3111.00| +2451914|39994|7085|41480|1054694|6308|19475|41480|1054694|6308|19475|32|12|2966|85|9269.25|370.77|9640.02|35.40|194.65|3151.54|1162.36|4955.35|600.82| +2452046|65203|17717|52706|1002914|2275|18083|52706|1002914|2275|18083|35|7|2966|13|1000.61|20.01|1020.62|43.07|333.45|840.51|83.25|76.85|396.53| +2451019|35234|10580|21003|1300262|5121|49580|21003|1300262|5121|49580|43|1|2968|33|57.42|4.01|61.43|39.07|6.27|10.90|34.42|12.10|49.35| +2452653|7819|3661|85839|686309|2332|12755|85839|686309|2332|12755|42|30|2969|34|5061.24|253.06|5314.30|47.69|862.58|3036.74|323.92|1700.58|1163.33| +2452853|46559|1944|55805|1772226|6600|11021|55805|1772226|6600|11021|42|15|2970|12|224.64|2.24|226.88|19.55|174.12|60.65|6.55|157.44|195.91| +2451419|68555|12265|78582|1729670|3605|31940|78582|1729670|3605|31940|19|21|2973|4|414.48|24.86|439.34|64.05|47.80|165.79|221.33|27.36|136.71| +2451263|16111|524|85243|57931|1819|27754|85243|57931|1819|27754|28|2|2973|15|181.80|1.81|183.61|59.08|113.25|14.54|167.26|0.00|174.14| +2451619|72671|9727|27301|1545960|4619|4078|27301|1545960|4619|4078|55|5|2974|4|174.28|0.00|174.28|74.14|146.40|15.68|126.88|31.72|220.54| +2451582|35094|7132|88977|543081|6432|43336|88977|543081|6432|43336|4|19|2974|77|114.73|1.14|115.87|52.21|41.58|63.10|38.72|12.91|94.93| +2452045|45296|607|31556|1839472|4188|32151|31556|1839472|4188|32151|41|11|2975|4|53.48|2.67|56.15|53.89|17.80|13.37|24.06|16.05|74.36| +2451935|68346|15955|95534|1469467|2153|42039|95534|1469467|2153|42039|23|29|2975|29|183.28|1.83|185.11|43.87|63.22|43.98|2.78|136.52|108.92| +2452024|52399|12176|48584|1272284|3984|19684|48584|1272284|3984|19684|50|2|2976|20|1314.60|26.29|1340.89|70.35|37.00|893.92|353.37|67.31|133.64| +2452120|13312|8630|46940|945773|6653|7086|46940|945773|6653|7086|47|15|2976|4|177.72|5.33|183.05|17.59|130.04|133.29|25.32|19.11|152.96| +2452015|42791|5468|37484|3634|631|34381|37484|3634|631|34381|47|5|2976|34|189.72|11.38|201.10|62.80|448.12|47.43|12.80|129.49|522.30| +2452047|76630|5213|13652|857781|310|3239|13652|857781|310|3239|38|30|2976|26|3618.16|72.36|3690.52|39.09|296.40|651.26|2373.52|593.38|407.85| +2451694|41108|8299|79761|64431|5901|18171|79761|64431|5901|18171|55|13|2977|12|111.60|0.00|111.60|48.81|99.24|36.82|45.61|29.17|148.05| +2451695|47379|2900|2770|1734136|751|47629|2770|1734136|751|47629|26|35|2977|13|154.18|9.25|163.43|7.22|290.03|90.96|55.63|7.59|306.50| +2452176|36940|9383|86232|125146|2460|36574|86232|125146|2460|36574|8|16|2978|41|6199.20|0.00|6199.20|95.88|2452.21|5021.35|341.57|836.28|2548.09| +2452047|69387|6584|88174|54062|4303|14544|88174|54062|4303|14544|43|14|2978|47|9086.04|817.74|9903.78|68.29|2666.31|1817.20|4070.55|3198.29|3552.34| +2451469|57800|7778|52462|1112437|2459|33634|52462|1112437|2459|33634|50|23|2981|16|49.28|3.94|53.22|32.54|101.28|39.91|5.34|4.03|137.76| +2452861|77681|7597|58566|1341864|6315|33613|58566|1341864|6315|33613|37|33|2982|29|3675.75|110.27|3786.02|90.45|1722.89|661.63|1386.49|1627.63|1923.61| +2452780|44114|6414|64744|1583817|6056|7487|64744|1583817|6056|7487|57|12|2982|58|158.34|12.66|171.00|21.02|202.42|57.00|67.89|33.45|236.10| +2451974|12440|11030|47988|1566482|4176|34717|47988|1566482|4176|34717|59|5|2983|2|231.58|9.26|240.84|82.51|94.08|41.68|170.91|18.99|185.85| +2452052|21942|49|52313|1127785|57|39882|52313|1127785|57|39882|32|20|2983|4|188.40|9.42|197.82|90.54|20.40|173.32|3.92|11.16|120.36| +2452783||16009|30197||5454|12106|30197|831202|||||2984|||19.77||62.69|351.54|296.57||52.40|434.00| +2452913|45283|11413|7336|482668|1553|14038|7336|482668|1553|14038|1|22|2984|42|0.00|0.00|0.00|3.33|1879.08|0.00|0.00|0.00|1882.41| +2452327|57004|11859|85691|1475147|156|16012|85691|1475147|156|16012|43|5|2985|14|1491.28|104.38|1595.66|98.57|360.50|1357.06|65.76|68.46|563.45| +2452276||13451|16410||566|28644||1195688|||33|17|2985||594.88||612.72|10.12|||299.52|15.77|1062.84| +2451906|70349|17941|58502|309354|2606|26305|58502|309354|2606|26305|49|17|2986|10|0.00|0.00|0.00|61.27|392.70|0.00|0.00|0.00|453.97| +2451368|74735|16910|73134|268472|4257|44795|73134|268472|4257|44795|49|12|2987|27|2534.76|202.78|2737.54|85.17|1067.31|988.55|355.62|1190.59|1355.26| +2451168|45927|6106|58994|667000|959|48918|58994|667000|959|48918|7|16|2988|30|4070.10|244.20|4314.30|70.60|268.20|2482.76|1396.85|190.49|583.00| +2451164|50087|1148|93348|1782132|6469|11206|93348|1782132|6469|11206|32|17|2990|4|45.68|4.11|49.79|65.20|25.80|3.19|39.09|3.40|95.11| +||2990|54596||160||||||13|13|2990||2980.44|||||1400.80|916.19||| +2451070|74925|10654|59225|533675|4478|9728|59225|533675|4478|9728|40|32|2991|18|954.00|57.24|1011.24|31.41|204.30|849.06|69.26|35.68|292.95| +2452347|46485|12161|81852|1522683|314|1898|81852|1522683|314|1898|33|10|2992|50|4141.50|124.24|4265.74|76.66|2840.00|3603.10|107.68|430.72|3040.90| +2451661|36816|13828|59331|1670085|6439|49993|59331|1670085|6439|49993|43|17|2993|4|30.16|0.30|30.46|14.05|64.12|3.01|10.86|16.29|78.47| +2451671|65373|17006|55666|413774|235|41096|55666|413774|235|41096|44|34|2993|21|160.02|9.60|169.62|29.45|96.60|142.41|9.33|8.28|135.65| +2452503|70951|7795|63676|1162799|6275|3798|63676|1162799|6275|3798|12|2|2994|56|8967.28|807.05|9774.33|95.77|2642.64|8518.91|416.98|31.39|3545.46| +2451228|54219|272|22565|476818|1471|7277|22565|476818|1471|7277|22|25|2995|5|129.35|7.76|137.11|95.77|69.65|7.76|29.18|92.41|173.18| +2451458|70317|8978|75303|46602|1901|18353|75303|46602|1901|18353|52|22|2995|1|28.50|1.71|30.21|83.33|12.21|3.70|23.06|1.74|97.25| +2452504|68160|6417|18962|374670|2025|21780|18962|374670|2025|21780|60|23|2996|9|484.20|0.00|484.20|14.92|128.16|392.20|11.96|80.04|143.08| +2452699|27927|13341|49106|1791922|4691|17107|49106|1791922|4691|17107|33|14|2997|10|216.00|10.80|226.80|47.32|110.70|153.36|47.60|15.04|168.82| +2451231|72817|12757|51870|509098|6326|3954|51870|509098|6326|3954|40|21|2998|53|3236.18|258.89|3495.07|30.37|1106.11|2815.47|302.91|117.80|1395.37| +2451203|61268|6626|89913|458178|4843|34308|89913|458178|4843|34308|49|11|2998|2|17.90|0.53|18.43|34.72|49.76|8.05|6.50|3.35|85.01| +||13627||||1307|92714|1475296|3125||49|20|2999|3||22.40|271.37|81.01|93.36|234.03||12.55|196.77| +2452717|33141|3847|86001|1469302|6824|5930|86001|1469302|6824|5930|7|29|3002|16|717.60|7.17|724.77|76.35|54.08|602.78|82.67|32.15|137.60| +2452704|15937|14605|40537|1819183|582|47581|40537|1819183|582|47581|57|7|3002|82|3933.54|314.68|4248.22|12.41|2587.92|983.38|1416.07|1534.09|2915.01| +2452813|84284|3222|5768|365145|5675|19852|5768|365145|5675|19852|18|7|3002|21|638.61|0.00|638.61|78.45|22.68|485.34|110.35|42.92|101.13| +2452752|42459|3753|85824|1375600|2176|9152|85824|1375600|2176|9152|19|18|3002|4|374.24|14.96|389.20|55.14|31.92|119.75|229.04|25.45|102.02| +2451378|40741|6955|64780|809288|4996|39904|64780|809288|4996|39904|49|11|3003|11|858.33|34.33|892.66|16.10|683.76|540.74|69.86|247.73|734.19| +2452857|21214|5178|10048|974422|4043|28064|10048|974422|4043|28064|27|34|3005|6|42.06|0.42|42.48|55.81|53.28|32.80|6.57|2.69|109.51| +2452873|6943|8581|26600|470067|4734|43001|26600|470067|4734|43001|12|31|3005|57|1199.28|59.96|1259.24|37.41|1744.77|11.99|831.10|356.19|1842.14| +2452762|45486|3999|5723|1457961|2199|18437|5723|1457961|2199|18437|6|1|3005|22|144.10|10.08|154.18|75.90|31.02|106.63|8.99|28.48|117.00| +2452781|57657|6949|51593|1275633|1626|20419|51593|1275633|1626|20419|43|1|3005|28|259.00|2.59|261.59|95.61|361.20|93.24|72.93|92.83|459.40| +|1643|8629||||41677||1069707|6892||||3007||||351.68|27.73|||2.08|206.20|464.77| +2452611|25273|6414|75504|1230210|1363|16069|75504|1230210|1363|16069|9|11|3007|4|218.80|6.56|225.36|10.07|29.80|80.95|12.40|125.45|46.43| +2452525|46171|5643|5639|922714|6989|16111|5639|922714|6989|16111|12|28|3007|29|4866.49|97.32|4963.81|48.77|3317.89|1849.26|1810.33|1206.90|3463.98| +2452832|30444|10959|53521|1661623|6078|8580|53521|1661623|6078|8580|9|21|3009|1|5.25|0.15|5.40|57.93|57.79|1.36|1.75|2.14|115.87| +2452074|75904|8059|86000|1910671|2989|18460|86000|1910671|2989|18460|9|35|3012|1|165.81|6.63|172.44|51.41|56.04|104.46|55.21|6.14|114.08| +2452246|31821|16269|38429|777494|2959|39224|38429|777494|2959|39224|29|22|3013|45|4314.15|172.56|4486.71|9.62|616.05|2545.34|601.39|1167.42|798.23| +2452658|42083|11241|20268|1012472|6276|24033|20268|1012472|6276|24033|27|28|3014|11|173.03|8.65|181.68|40.15|120.78|50.17|76.17|46.69|169.58| +2452387||4265|56844|||9808||1035832||9808|31||3015|18|657.72|52.61|||185.40||102.87|199.69|277.18| +2452462|64527|5799|30544|1856312|1280|648|30544|1856312|1280|648|13|26|3015|1|0.00|0.00|0.00|78.18|21.74|0.00|0.00|0.00|99.92| +2451208|61442|10214|3967|76222|1630|19838|3967|76222|1630|19838|19|5|3017|42|43.26|3.89|47.15|25.92|217.56|12.54|9.83|20.89|247.37| +2451306|45528|11252|77166|510248|3330|27659|77166|510248|3330|27659|32|15|3018|20|1236.00|12.36|1248.36|83.69|504.40|98.88|852.84|284.28|600.45| +2451376|69969|16724|57191|690211|4207|23401|57191|690211|4207|23401|7|30|3022|21|1910.37|57.31|1967.68|91.19|463.05|343.86|971.23|595.28|611.55| +2452074|48935|2689||1431079|6179|34032|74826|1431079|6179|||9|3024|||17.12|359.66||156.97||61.96||267.56| +2452089|60854|13609|98089|1375151|3860|25769|98089|1375151|3860|25769|50|29|3024|15|190.35|17.13|207.48|91.75|439.35|81.85|34.72|73.78|548.23| +2451964|80666|11159|11978|782925|1872|44819|11978|782925|1872|44819|23|1|3024|5|609.85|54.88|664.73|16.27|62.00|195.15|211.49|203.21|133.15| +2452073|82731|9911|34614|1154791|4215|29592|34614|1154791|4215|29592|31|25|3024|1|115.23|10.37|125.60|29.64|10.84|28.80|2.59|83.84|50.85| +2451622|77300|1472|69403|1383999|6325|5090|69403|1383999|6325|5090|34|26|3027|60|3921.60|274.51|4196.11|74.65|1907.40|3568.65|3.52|349.43|2256.56| +2451631|73937|15806|68634|686432|4230|16875|68634|686432|4230|16875|55|27|3027|69|480.24|14.40|494.64|92.92|144.21|14.40|368.01|97.83|251.53| +2451603|63007|7450|81983|303606|6029|32052|81983|303606|6029|32052|2|8|3027|73|63.51|5.08|68.59|76.65|655.54|47.63|9.05|6.83|737.27| +2452490|8988|16749|53379|1659120|4861|4202|53379|1659120|4861|4202|15|20|3029|12|1846.56|18.46|1865.02|76.26|101.16|794.02|947.28|105.26|195.88| +2451425|70015|17642|29490|1309741|2792|24643|29490|1309741|2792|24643|13|4|3030|22|2623.28|0.00|2623.28|69.21|758.56|1862.52|266.26|494.50|827.77| +2451443|24499|2348|64498|587439|427|47815|64498|587439|427|47815|7|24|3030|26|145.60|7.28|152.88|17.08|330.20|75.71|19.56|50.33|354.56| +2451945|6720|7573|76468|92674|6927|7533|76468|92674|6927|7533|11|28|3031|10|1142.40|34.27|1176.67|28.04|385.90|799.68|20.56|322.16|448.21| +2452730|68826|14281|31854|71500|334|37594|31854|71500|334|37594|19|26|3032|10|668.90|13.37|682.27|87.58|222.90|408.02|127.83|133.05|323.85| +2452536|69295|7673|86858|205793|7128|10016|86858|205793|7128|10016|7|6|3034|2|142.18|9.95|152.13|40.31|89.58|59.71|20.61|61.86|139.84| +2452386|34683|8407|27477|1876244|211|39378|27477|1876244|211|39378|13|16|3035|89|1270.92|88.96|1359.88|40.30|474.37|1194.66|68.63|7.63|603.63| +2452717|12048|1956|69322|940737|505|38320|69322|940737|505|38320|57|23|3036|11|291.83|0.00|291.83|61.39|137.83|274.32|17.51|0.00|199.22| +2452511|52854|15925|46792|1243327|7062|37197|46792|1243327|7062|37197|3|25|3037|32|1418.24|14.18|1432.42|74.44|1260.80|468.01|465.61|484.62|1349.42| +2451380|74868|14426|43977|1919040|4056|19919|43977|1919040|4056|19919|25|1|3038|6|106.08|8.48|114.56|55.32|118.56|96.53|2.96|6.59|182.36| +2451594|56455|416|56748|1794084|6620|2003|56748|1794084|6620|2003|19|26|3039|5|37.05|0.00|37.05|27.63|15.70|0.74|18.88|17.43|43.33| +2451720|59733|9820|98625|514565|4105|48492|98625|514565|4105|48492|49|8|3039|10|517.80|46.60|564.40|24.73|847.40|93.20|250.51|174.09|918.73| +2451523|64370|11749|77477|1015205|7148|10517|77477|1015205|7148|10517|20|22|3039|41|1155.79|46.23|1202.02|15.69|1274.28|1074.88|54.20|26.71|1336.20| +2452081||14753|24756|1027701|4599||24756|1027701|4599||||3040|35|2390.15|191.21|2581.36||||774.17||| +2452062|63527|12831|95937|1394901|5065|48429|95937|1394901|5065|48429|41|20|3041|46|2750.34|110.01|2860.35|98.35|549.70|1540.19|1101.23|108.92|758.06| +2452277|61046|11549|22920|1873793|3066|17539|22920|1873793|3066|17539|13|2|3041|38|1138.48|68.30|1206.78|96.14|319.58|626.16|174.18|338.14|484.02| +2451637|35091|4624|95268|952969|4623|36724|95268|952969|4623|36724|31|10|3043|1|71.12|0.00|71.12|64.94|32.67|23.46|5.24|42.42|97.61| +2451327|74600|4480|29599|1585221|843|38512|29599|1585221|843|38512|49|23|3044|8|285.12|19.95|305.07|65.99|231.60|282.26|1.85|1.01|317.54| +|62363|14731|89146|||18914|89146||6356|18914|||3045|||7.49||71.05||99.24||17.61|134.96| +2451714|49999|1072|84115|426055|5948|13888|84115|426055|5948|13888|26|35|3046|41|1942.58|77.70|2020.28|70.26|647.39|621.62|1162.44|158.52|795.35| +2451591|37999|5083|79711|1085253|3332|20842|79711|1085253|3332|20842|10|3|3046|36|3341.16|133.64|3474.80|89.02|1068.84|2372.22|697.63|271.31|1291.50| +2452298|33034|17143|93074|1381606|3738|24317|93074|1381606|3738|24317|45|21|3047|39|4697.16|0.00|4697.16|91.91|320.19|2818.29|1728.56|150.31|412.10| +2452481|46379|3055|52218|1630064|710|21309|52218|1630064|710|21309|37|6|3048|73|856.29|51.37|907.66|34.26|2569.60|428.14|214.07|214.08|2655.23| +2452279|57708|7213|19537|256795|453|6511|19537|256795|453|6511|27|24|3048|4|116.60|6.99|123.59|61.57|73.28|36.14|25.74|54.72|141.84| +||1643|93256||4315|35668|93256|1466162|4315||||3048|40|4847.20|96.94|||||2836.58||| +2452297|53968|4469|28557|1495512|1667|21869|28557|1495512|1667|21869|41|11|3048|23|1255.11|50.20|1305.31|95.28|989.00|1091.94|79.95|83.22|1134.48| +|41854|2583|32033|1211030|||32033|1211030|2827|24271|7|32|3048|16||||45.03|27.20|21.48|38.01|26.43|| +2451367|46624|11368|39922|1467904|1839|3421|39922|1467904|1839|3421|50|5|3051|14|739.06|59.12|798.18|88.59|689.78|169.98|364.21|204.87|837.49| +2451959|60213|983|81684|266680|1276|35872|81684|266680|1276|35872|14|29|3053|27|150.66|13.55|164.21|12.42|1168.83|61.77|67.55|21.34|1194.80| +2452901|25605|10101|59600|510768|336|40704|59600|510768|336|40704|15|17|3054|4|589.92|17.69|607.61|43.86|31.04|501.43|66.36|22.13|92.59| +|531|8737|40429|846481|2933|||||30585|48|22|3054|||100.32|2608.32|9.03||777.48||1245.98|| +2452898|12635|13009|62193|869244|5095|39746|62193|869244|5095|39746|19|8|3054|7|596.68|41.76|638.44|92.10|54.95|584.74|11.94|0.00|188.81| +2452674|71278|7429|57371|535356|894|40079|57371|535356|894|40079|45|15|3056|5|49.85|0.99|50.84|24.81|87.25|5.98|16.67|27.20|113.05| +2452791|82082|13044|15601|1831371|741|15711|15601|1831371|741|15711|9|10|3056|18|2188.62|65.65|2254.27|3.07|116.64|525.26|848.31|815.05|185.36| +2451545|40088|1267|78524|1877256|1941|19473|78524|1877256|1941|19473|40|32|3057|36|0.00|0.00|0.00|96.68|1222.20|0.00|0.00|0.00|1318.88| +2451495|80052|9250|3325|1247194|2263|8132|3325|1247194|2263|8132|16|2|3057|61|4886.71|244.33|5131.04|46.89|1437.16|2345.62|2337.80|203.29|1728.38| +2451475|30865|9136|49654|447940|626|21665|49654|447940|626|21665|38|26|3057|18|1016.46|10.16|1026.62|97.84|584.10|193.12|288.16|535.18|692.10| +2451486|19407|15578|4736|478010|2837|34111|4736|478010|2837|34111|56|18|3057|45|2124.45|84.97|2209.42|38.85|579.15|1465.87|625.65|32.93|702.97| +2451656|35579|10873|10103|1311251|1999|41547|10103|1311251|1999|41547|46|35|3057|1|46.00|3.68|49.68|11.02|3.63|44.16|1.03|0.81|18.33| +2452703|57293|939|66954|770447|6387|23080|66954|770447|6387|23080|43|6|3058|27|397.71|11.93|409.64|4.64|143.91|51.70|166.08|179.93|160.48| +2451582|57696|14575|91486|1343973|6060|15700|91486|1343973|6060|15700|8|22|3059|9|402.66|0.00|402.66|36.39|108.18|297.96|74.33|30.37|144.57| +2452490|75417|4767|87656|593096|3181|25723|87656|593096|3181|25723|31|12|3060|12|402.96|12.08|415.04|81.14|451.80|225.65|143.62|33.69|545.02| +2451964|35801|16109|80162|226135|6426|27828|80162|226135|6426|27828|47|32|3061|3|39.09|1.56|40.65|73.72|31.26|23.84|1.83|13.42|106.54| +2451616|8984|8704|60856|1156529|2249|3122|60856|1156529|2249|3122|37|3|3062|5|238.70|9.54|248.24|58.62|172.60|23.87|204.08|10.75|240.76| +2451707|27207|16922|42286|1397779|5625|32318|42286|1397779|5625|32318|1|32|3063|21|588.21|52.93|641.14|3.15|6.93|564.68|6.82|16.71|63.01| +2451442|52058|11012|41127|282092|3057|19222|41127|282092|3057|19222|46|27|3064|3|235.05|9.40|244.45|67.10|71.73|72.86|51.90|110.29|148.23| +2452727|49572|10611|54787|893211|1235|37706|54787|893211|1235|37706|15|25|3066|75|1777.50|88.87|1866.37|92.89|1608.00|426.60|1283.35|67.55|1789.76| +2452717|34289|6207|5491|1654307|1970|16033|5491|1654307|1970|16033|55|2|3066|8|49.92|0.49|50.41|9.39|48.72|20.46|25.63|3.83|58.60| +2452480|10296|2521|12198|367833|6668|16350|12198|367833|6668|16350|42|27|3067|13|73.32|1.46|74.78|8.39|195.78|35.19|14.10|24.03|205.63| +2452568|33704|13707|66380|1285346|2712|45278|66380|1285346|2712|45278|49|1|3067|16|353.44|14.13|367.57|31.33|186.56|289.82|33.71|29.91|232.02| +2452146|43765|12839|9375|1073367|5006|25445|9375|1073367|5006|25445|8|23|3068|9|1435.59|0.00|1435.59|2.14|478.53|990.55|44.50|400.54|480.67| +2452573|61046|6397|1670|1213564|3858|552|1670|1213564|3858|552|12|25|3069|50|1765.50|0.00|1765.50|71.67|2097.00|1641.91|59.32|64.27|2168.67| +2452846|86181|3253|55357|402372|6987|20002|55357|402372|6987|20002|19|34|3071|32|889.92|80.09|970.01|66.35|101.76|80.09|364.42|445.41|248.20| +2452753|18068|6559|97508|1289648|6141|26234|97508|1289648|6141|26234|9|11|3071|5|26.65|0.26|26.91|12.33|144.05|20.78|4.63|1.24|156.64| +2452426|58198|11487|8362|1512009|1256|36103|8362|1512009|1256|36103|3|7|3073|17|642.60|25.70|668.30|31.18|543.66|77.11|356.25|209.24|600.54| +2452652|49438|11991|73482|1282195|5308|5574|73482|1282195|5308|5574|9|5|3074|35|1563.80|78.19|1641.99|51.30|0.00|437.86|270.22|855.72|129.49| +2452747|45715|15319|10155|1055480|3558|3683|10155|1055480|3558|3683|27|2|3075|16|204.48|14.31|218.79|32.44|0.00|4.08|94.18|106.22|46.75| +2452455|75574|16669|54198|1015951|104|32657|54198|1015951|104|32657|25|35|3076|15|235.80|16.50|252.30|20.81|71.85|35.37|66.14|134.29|109.16| +2452323|12177|7|32816|839589|5915|36288|32816|839589|5915|36288|42|33|3076|13|393.25|15.73|408.98|28.11|112.32|393.25|0.00|0.00|156.16| +2451327|39867|13846|54847|1097362|677|23992|54847|1097362|677|23992|19|16|3077|14|2211.30|154.79|2366.09|53.80|293.02|331.69|827.02|1052.59|501.61| +2451009|10354|9614|47276|1126877|6125|14526|47276|1126877|6125|14526|22|35|3078|54|2185.92|109.29|2295.21|70.93|1271.16|939.94|548.23|697.75|1451.38| +2452539|30735|8965|85942|87606|6517|7381|85942|87606|6517|7381|31|18|3079|21|71.61|2.14|73.75|54.18|104.58|1.43|68.07|2.11|160.90| +2452303|53577|14213|84877|382657|437|49817|84877|382657|437|49817|35|7|3080|52|4561.44|45.61|4607.05|62.62|2181.40|3466.69|426.95|667.80|2289.63| +2451756|60201|17470|49176|1670971|6719|8114|49176|1670971|6719|8114|8|28|3081|24|3598.32|323.84|3922.16|26.39|505.92|2986.60|397.61|214.11|856.15| +2451696|69040|658|89587|1038299|6658|4570|89587|1038299|6658|4570|38|8|3081|15|185.70|5.57|191.27|88.96|454.05|51.99|14.70|119.01|548.58| +2452724|73128|10045|96|1883750|3002|5745|96|1883750|3002|5745|24|20|3084|17|91.46|2.74|94.20|14.19|32.98|32.92|51.51|7.03|49.91| +2452348|34594|16373|11784|689057|730|38403|11784|689057|730|38403|1|22|3085|28|810.88|8.10|818.98|74.70|2682.96|454.09|103.46|253.33|2765.76| +2451720|45474|9808|46143|112153|4116|37326|46143|112153|4116|37326|31|10|3086|21|135.24|5.40|140.64|32.25|56.28|70.32|20.12|44.80|93.93| +2451615|65144|2888|37125|292317|7156|12795|37125|292317|7156|12795|32|25|3086|8|383.92|7.67|391.59|24.26|81.28|241.86|72.45|69.61|113.21| +2452820|21386|15819|5692|453872|6852|34585|5692|453872|6852|34585|51|23|3087|4|301.16|9.03|310.19|6.67|50.16|33.12|58.96|209.08|65.86| +2452209|63569|8749|78255|827364|6073|39982|78255|827364|6073|39982|13|34|3088|42|1407.42|56.29|1463.71|69.64|326.34|98.51|1112.57|196.34|452.27| +2452318|46142|11757|51648|429355|1462|34816|51648|429355|1462|34816|13|12|3089|3|701.46|63.13|764.59|66.28|328.80|161.33|27.00|513.13|458.21| +2452397|34898|2285|26461|1290515|5322|14176|26461|1290515|5322|14176|19|24|3089|19|524.97|0.00|524.97|2.30|272.84|57.74|331.73|135.50|275.14| +2452347|61238|15103|56149|1517308|4514|17673|56149|1517308|4514|17673|59|11|3089|68|173.40|5.20|178.60|57.68|520.88|74.56|60.29|38.55|583.76| +2451052|25330|2338|49386|1335112|838|12917|49386|1335112|838|12917|37|33|3090|54|727.92|36.39|764.31|18.39|1252.26|647.84|44.84|35.24|1307.04| +2451041|46789|11756|26908|1718429|5150|6271|26908|1718429|5150|6271|22|31|3090|61|2604.70|78.14|2682.84|85.73|0.00|2318.18|108.87|177.65|163.87| +2451396|45330|16498|67616|551442|4068|36294|67616|551442|4068|36294|56|35|3091|26|1790.36|0.00|1790.36|97.05|1329.90|1468.09|170.80|151.47|1426.95| +2451222|37074|9637|38027|258404|2781|46405|38027|258404|2781|46405|58|16|3091|20|849.40|16.98|866.38|76.33|1352.80|331.26|424.87|93.27|1446.11| +2452128|77166|12457|58163|999726|5642|44358|58163|999726|5642|44358|7|3|3092|9|87.12|1.74|88.86|36.92|18.63|79.27|4.78|3.07|57.29| +2451444|76790|1495|95949|672996|4443|7937|95949|672996|4443|7937|34|28|3093|7|113.12|1.13|114.25|40.73|678.93|108.59|2.85|1.68|720.79| +2451469|53718|4976|27448|1493692|5089|9333|27448|1493692|5089|9333|13|33|3093|3|183.63|11.01|194.64|30.08|99.87|38.56|2.90|142.17|140.96| +2451367|60197|12566|88549|1248069|5400|41954|88549|1248069|5400|41954|34|32|3093|4|429.08|8.58|437.66|73.69|0.00|381.88|36.81|10.39|82.27| +2451479|72386|716|38015|680197|5132|12914|38015|680197|5132|12914|43|8|3093|11|998.25|79.86|1078.11|9.74|846.01|978.28|7.78|12.19|935.61| +2451526|40093|6208|82706|736391|2429|45714|82706|736391|2429|45714|46|13|3093|9|339.12|20.34|359.46|15.66|106.20|125.47|196.55|17.10|142.20| +2451310|79797|16252|52432|1330416|4847|10078|52432|1330416|4847|10078|14|25|3093|11|397.98|15.91|413.89|0.76|691.35|55.71|171.13|171.14|708.02| +2451784|69572|13577|85593|1827247|6854|43708|85593|1827247|6854|43708|35|1|3095|4|250.44|2.50|252.94|63.17|125.20|192.83|21.89|35.72|190.87| +2451021|49825|3038|46504|799052|1200|6148|46504|799052|1200|6148|2|9|3096|44|2188.12|131.28|2319.40|32.69|2009.48|875.24|774.59|538.29|2173.45| +2451057|70003|6169|95804|1403524|2891|44258|95804|1403524|2891|44258|50|19|3096|10|10.20|0.10|10.30|74.38|12.20|2.34|3.06|4.80|86.68| +2451704|35757|9422|86540|1172134|928|10122|86540|1172134|928|10122|32|5|3097|28|2391.76|167.42|2559.18|53.98|717.36|1530.72|533.84|327.20|938.76| +2451739|48791|5713||686721|3225||40617|686721|3225||||3099|5|603.75|24.15|627.90|87.58|374.45||22.82|85.86|486.18| +2451794|57580|7655|41068|1305150|4879|48161|41068|1305150|4879|48161|38|35|3099|63|551.25|49.61|600.86|31.77|34.02|176.40|146.19|228.66|115.40| +2451699|69931|5014|54318|563495|4225|44602|54318|563495|4225|44602|13|27|3100|75|1094.25|87.54|1181.79|1.15|938.25|470.52|349.28|274.45|1026.94| +|25083|4033||1388973|6751|18882|1595|1388973|6751|18882|47|6|3101|4||12.11|163.55||||83.60||| +2451295|39906|4888|52880|147257|686|25270|52880|147257|686|25270|16|16|3102|2|80.72|5.65|86.37|73.83|142.06|25.83|50.49|4.40|221.54| +2451365|19848|1423|38374|554467|3713|49006|38374|554467|3713|49006|50|21|3102|67|1543.01|138.87|1681.88|24.70|1136.99|725.21|580.63|237.17|1300.56| +2452596|38879|7671|85464|1011903|5989|3628|85464|1011903|5989|3628|18|5|3104|16|24.32|0.00|24.32|88.85|170.88|4.37|17.95|2.00|259.73| +2452678|21950|12210|75583|932841|2099|11757|75583|932841|2099|11757|42|2|3104|1|89.88|2.69|92.57|67.29|12.67|70.10|12.06|7.72|82.65| +2452814|34101|12321|76728|1766669|3010|29398|76728|1766669|3010|29398|37|26|3104|63|348.39|10.45|358.84|83.55|236.88|337.93|6.69|3.77|330.88| +2452355|61388|897|15435|172224|660|21849|15435|172224|660|21849|57|24|3106|22|895.18|17.90|913.08|35.26|263.34|277.50|142.06|475.62|316.50| +2452363|53008|14585|11254|1154743|6585|37227|11254|1154743|6585|37227|43|6|3106|46|443.90|0.00|443.90|32.82|982.10|128.73|56.73|258.44|1014.92| +2452350|69733|17201|73074|722330|6786|9242|73074|722330|6786|9242|37|11|3107|39|475.80|14.27|490.07|76.87|1367.73|314.02|53.38|108.40|1458.87| +2452379|68110|7167|42557|1305689|6068|34047|42557|1305689|6068|34047|29|17|3107|15|808.80|72.79|881.59|65.86|1550.25|56.61|594.23|157.96|1688.90| +2452034|66786|7475|37854|1448787|1320|37900|37854|1448787|1320|37900|14|28|3108|7|97.09|1.94|99.03|35.90|57.54|84.46|4.29|8.34|95.38| +2452377|77024|3077|72449|1086354|6659|7317|72449|1086354|6659|7317|13|10|3109|47|1529.38|137.64|1667.02|22.03|746.83|367.05|220.84|941.49|906.50| +2452541|20888|4590|71491|1540492|693|3741|71491|1540492|693|3741|18|4|3111|10|613.50|49.08|662.58|21.89|341.80|269.94|199.26|144.30|412.77| +2452516|55895|4911|22982|845861|7140|49619|22982|845861|7140|49619|39|28|3111|6|102.96|1.02|103.98|5.04|6.42|59.71|13.84|29.41|12.48| +2451401|42763|9104|15755|1473945|4582|13572|15755|1473945|4582|13572|13|31|3112|25|2579.50|154.77|2734.27|25.31|135.75|77.38|1175.99|1326.13|315.83| +2452336|67386|16725|36654|1561795|1783|36524|36654|1561795|1783|36524|47|31|3113|15|1092.30|10.92|1103.22|9.53|357.45|644.45|129.87|317.98|377.90| +2451947|37207|6743|54447|1189613|779|22745|54447|1189613|779|22745|43|4|3114|67|8801.12|88.01|8889.13|27.35|779.21|5632.71|95.05|3073.36|894.57| +2452025|68153|15361|83482|242238|1347|14456|83482|242238|1347|14456|26|26|3116|71|11652.52|0.00|11652.52|24.43|2648.30|5243.63|256.35|6152.54|2672.73| +2451416|62862|16093|29280|274194|1155|40116|29280|274194|1155|40116|13|27|3118|19|133.76|6.68|140.44|24.26|200.64|82.93|38.12|12.71|231.58| +2451331|33194|14665|16604|1329926|6448|12217|16604|1329926|6448|12217|13|26|3118|9|193.59|3.87|197.46|84.03|38.70|60.01|90.83|42.75|126.60| +2451212|48083|3896|79470|1906870|1823|21623|79470|1906870|1823|21623|34|2|3118|25|507.25|20.29|527.54|57.69|1174.75|15.21|113.16|378.88|1252.73| +2451307|74173|9050|11726|1083887|5201|41762|11726|1083887|5201|41762|40|7|3118|25|1941.25|0.00|1941.25|65.29|853.25|252.36|574.22|1114.67|918.54| +2451624|73759|7480|41043|1788540|4390|44287|41043|1788540|4390|44287|8|7|3119|41|3783.07|0.00|3783.07|80.92|2308.30|1513.22|885.24|1384.61|2389.22| +||14788|||1473|11108|47003||||13||3119||||160.12|9.09|184.76||70.71||193.85| +2451482|50140|11128|85056|675299|5895|7338|85056|675299|5895|7338|34|5|3119|46|3032.78|242.62|3275.40|37.85|742.44|1182.78|703.00|1147.00|1022.91| +2451275|49946|6598|90263|366566|2516|34019|90263|366566|2516|34019|13|9|3120|64|673.28|53.86|727.14|93.23|3788.80|585.75|69.14|18.39|3935.89| +2452147|45777|2671|786|1843493|1847|37354|786|1843493|1847|37354|13|15|3122|62|3217.80|160.89|3378.69|59.00|1324.94|514.84|2027.22|675.74|1544.83| +2451932|33296|15007|62487|527264|41|5571|62487|527264|41|5571|20|24|3122|37|273.43|10.93|284.36|97.23|1285.38|123.04|40.60|109.79|1393.54| +2452048|23133|581|66669|1519739|2762|22985|66669|1519739|2762|22985|3|7|3125|41|2758.07|193.06|2951.13|5.11|2138.97|2675.32|69.51|13.24|2337.14| +||3830||647429|2407|||647429|2407|30138|52||3126||105.70||113.09|||16.91|41.73|47.06|149.82| +2451358|72241|15724|28960|1269159|6958|31557|28960|1269159|6958|31557|25|8|3126|55|5393.85|269.69|5663.54|89.89|85.25|2750.86|1189.34|1453.65|444.83| +2451568|76377|13868|51523|349789|3608|29874|51523|349789|3608|29874|13|19|3126|23|718.06|21.54|739.60|70.41|107.64|193.87|214.91|309.28|199.59| +2451421|39028|3163|38360|1641141|3122|46350|38360|1641141|3122|46350|49|11|3126|35|438.90|30.72|469.62|18.85|1621.55|311.61|85.28|42.01|1671.12| +2451177|67811|16348|56552|28784|1816|30448|56552|28784|1816|30448|10|19|3127|53|133.03|1.33|134.36|65.16|255.46|42.56|37.99|52.48|321.95| +2451219|65975|1345|79405|1212014|110|40722|79405|1212014|110|40722|13|1|3127|11|60.17|2.40|62.57|34.55|3.19|40.31|18.66|1.20|40.14| +2452130|78305|14924|954|1242288|7175|1309|954|1242288|7175|1309|5|32|3128|72|416.88|25.01|441.89|40.09|42.48|304.32|45.02|67.54|107.58| +2452537|18791|9319|19354|1324908|3262|19663|19354|1324908|3262|19663|45|14|3130|6|157.14|4.71|161.85|3.59|66.24|25.14|91.08|40.92|74.54| +2452524|45866|495|79827|1081254|1881|30508|79827|1081254|1881|30508|19|15|3130|3|89.46|0.89|90.35|26.51|3.81|6.26|46.59|36.61|31.21| +2451869|17103|9145|62800|1854730|4260|1390|62800|1854730|4260|1390|37|26|3131|4|116.28|9.30|125.58|98.06|24.48|65.11|36.33|14.84|131.84| +2452763|62186|8178|58393|1527203|6060|16470|58393|1527203|6060|16470|51|9|3132|8|52.80|4.75|57.55|56.10|140.80|25.87|11.57|15.36|201.65| +2452683|54541|10986|67200|395694|4635|7929|67200|395694|4635|7929|7|29|3132|17|780.81|46.84|827.65|23.91|284.75|101.50|448.34|230.97|355.50| +2452318|69189|9377|67657|301340|2223|45106|67657|301340|2223|45106|25|24|3134|29|44.66|4.01|48.67|36.96|17.69|7.59|25.20|11.87|58.66| +2451204|41390|526|77733|750127|5755|45766|77733|750127|5755|45766|40|7|3135|14|1172.92|0.00|1172.92|88.01|1513.40|516.08|118.23|538.61|1601.41| +2451092|77858|15068||227850|||10838|||25121||32|3135|3||11.86|||79.08|151.87|||186.04| +2451672|17841|6211|96605|1263115|6872|39382|96605|1263115|6872|39382|1|6|3137|20|3340.00|167.00|3507.00|64.39|1670.00|901.80|1731.12|707.08|1901.39| +2452675|50553|14649|40180|964951|6969|39740|40180|964951|6969|39740|57|28|3139|32|14.72|0.14|14.86|39.42|2.88|3.68|6.40|4.64|42.44| +2452526||5941||699848|||97425|||7094|54||3139|||4.15||8.50|189.34|||43.12|201.99| +2452191|74838|14645|16977|1825604|4984|36411|16977|1825604|4984|36411|11|29|3141|25|2922.50|204.57|3127.07|71.42|748.25|642.95|1527.29|752.26|1024.24| +2452847|54686|9900|41999|1049492|2138|49231|41999|1049492|2138|49231|45|15|3142|70|239.40|0.00|239.40|73.03|559.30|146.03|27.07|66.30|632.33| +2452663|80947|14976|56846|271513|5827|37341|56846||||6|19|3142|79|2498.77|||99.09||224.88|||| +2452196|80370|3933|18462|399644|2426|38008|18462|399644|2426|38008|51|33|3143|26|1193.14|95.45|1288.59|73.18|364.52|1097.68|24.81|70.65|533.15| +2451554|79340|8878|43870|1737602|1734|37019|43870|1737602|1734|37019|13|22|3144|79|1869.93|18.69|1888.62|48.27|808.17|1308.95|420.73|140.25|875.13| +2451743||13954||1665313|||||||7||3146||2117.29||2265.50||||41.28||1995.89| +|77422|8742||||||||39120|24|26|3147|9|696.96||710.89|33.71|338.49||||386.13| +2451953|14469|7657|90788|35691|4556|5475|90788|35691|4556|5475|2|33|3148|10|58.10|5.22|63.32|42.17|406.70|36.60|1.29|20.21|454.09| +2451100|68146|4813|62270|1599791|4865|17986|62270|1599791|4865|17986|44|1|3149|25|104.25|9.38|113.63|55.57|343.00|50.04|13.55|40.66|407.95| +2452720|44166|3384|32702|1577093|3531|4722|32702|1577093|3531|4722|27|24|3150|13|153.27|7.66|160.93|48.87|72.28|121.08|28.97|3.22|128.81| +||14323|61452||196|34368|61452|||34368|14|3|3151|4||||83.96|169.28|||102.92|| +2452153|33991|3637|19947|1877506|312|37871|19947|1877506|312|37871|50|12|3151|18|447.12|13.41|460.53|97.84|664.02|438.17|2.95|6.00|775.27| +2451993|74748|17222|97931|526773|4062|2951|97931|526773|4062|2951|5|9|3151|75|8364.00|0.00|8364.00|33.18|5147.25|6774.84|1175.97|413.19|5180.43| +2452253|33573|3013|28358|376831|1751|24271|28358|376831|1751|24271|11|4|3152|5|60.80|0.00|60.80|65.24|177.75|32.22|9.14|19.44|242.99| +|34035|7934||1519719||6617||1519719|768|6617|43||3153|21|||||||226.33|22.39|213.19| +2451321|18871|17248|70155|667705|1608|45958|70155|667705|1608|45958|28|9|3154|51|744.60|7.44|752.04|35.36|138.21|528.66|172.75|43.19|181.01| +2451289|75341|2170|12247|549152|5610|10915|12247|549152|5610|10915|55|6|3154|42|1067.64|10.67|1078.31|27.99|1112.16|886.14|181.50|0.00|1150.82| +2452251|5061|5481|9165|199745|1078|24858|9165|199745|1078|24858|37|2|3155|64|4408.32|132.24|4540.56|35.33|948.48|484.91|1530.12|2393.29|1116.05| +2452262|18899|10089|20331|613836|2541|15646|20331|613836|2541|15646|37|15|3156|18|282.78|5.65|288.43|11.21|10.44|22.62|46.82|213.34|27.30| +2452269|72443|7801|8856|376108|2113|46965|8856|376108|2113|46965|21|26|3156|15|57.60|1.72|59.32|10.48|182.40|33.40|21.29|2.91|194.60| +2452363|64086|10711|11234|744500|4966|41079|11234|744500|4966|41079|9|35|3156|16|946.08|66.22|1012.30|22.17|585.12|122.99|567.93|255.16|673.51| +2451779|36266|4418|69969|218825|481|24937|69969|218825|481|24937|31|21|3157|1|7.89|0.00|7.89|95.71|37.91|4.49|0.13|3.27|133.62| +2452478|62851|8067|62637|812071|3905|2586|62637|812071|3905|2586|55|24|3158|4|63.60|4.45|68.05|48.09|26.88|62.96|0.34|0.30|79.42| +2451819|56132|10619|96625|419164|4903|33395|96625|419164|4903|33395|41|30|3159|5|21.20|0.84|22.04|90.79|19.30|5.30|11.92|3.98|110.93| +2451704|15274|17431|48119|696263|2454|31845|48119|696263|2454|31845|7|24|3159|3|222.12|19.99|242.11|75.86|54.84|195.46|12.26|14.40|150.69| +||1436|30609|||42937|||4961||||3163|4|||420.07|||||46.59|| +||11836||||4550|21532|||4550|||3165||84.96|5.09||6.20|||27.03|35.85|| +2451809|43259|2198|89272|1549776|1033|36290|89272|1549776|1033|36290|43|28|3166|19|1962.32|176.60|2138.92|58.10|267.52|922.29|52.00|988.03|502.22| +2452825|29436|17503|69792|839655|7118|3781|69792|839655|7118|3781|37|31|3167|7|448.84|40.39|489.23|69.89|23.59|121.18|183.48|144.18|133.87| +2451901||12115||899721|1407|37540||899721||37540|||3168|25|||70.98|37.37|273.25|||20.41|313.35| +2451811|81031|11936|42760|133396|385|6974|42760|133396|385|6974|8|32|3168|2|63.70|5.09|68.79|88.96|0.94|0.00|61.78|1.92|94.99| +2452277|82460|16967|53113|1799738|4763|1788|53113|1799738|4763|1788|57|6|3169|33|206.58|16.52|223.10|64.09|915.42|105.35|81.99|19.24|996.03| +2451398|71777|2995|98341|1125814|2887|35960|98341|1125814|2887|35960|22|11|3170|53|3452.95|207.17|3660.12|58.19|2193.14|1519.29|657.44|1276.22|2458.50| +2452102|68698|7850|18052|366255|58|31108|18052|366255|58|31108|25|23|3171|24|148.32|7.41|155.73|18.90|228.24|102.34|6.43|39.55|254.55| +2452076|54722|13255|39725|259573|3068|40916|39725|259573|3068|40916|53|30|3172|13|15.08|0.75|15.83|71.26|60.58|13.42|1.59|0.07|132.59| +2452208|36503|2147|53188|1789139|290|41441|53188|1789139|290|41441|51|15|3172|44|3329.04|133.16|3462.20|43.57|2784.32|632.51|1590.95|1105.58|2961.05| +2452132|74068|11957|97957|106708|4612|40563|97957|106708|4612|40563|19|31|3172|28|2336.04|186.88|2522.92|95.64|1038.24|1565.14|439.41|331.49|1320.76| +2452077|35254|13947|97380|763112|44|976|97380|763112|44|976|35|33|3172|17|698.70|62.88|761.58|35.07|210.80|447.16|110.67|140.87|308.75| +2451109|71101|15790|99206|177508|5057|35806|99206|177508|5057|35806|52|31|3174|19|1469.08|132.21|1601.29|86.02|618.45|1028.35|255.62|185.11|836.68| +2451007|4980|9806|34991|145001|3011|48174|34991|145001|3011|48174|32|7|3174|41|7220.10|0.00|7220.10|13.94|4195.53|649.80|5256.24|1314.06|4209.47| +2452718|47670|828|25559|1768123|1887|15251|25559|1768123|1887|15251|9|26|3175|9|993.15|19.86|1013.01|49.17|245.70|655.47|165.46|172.22|314.73| +2451068|14811|13562|74849|605568|6334|34653|74849|605568|6334|34653|26|28|3178|20|2462.80|197.02|2659.82|93.99|158.80|1477.68|334.94|650.18|449.81| +2452646|19551|25|82383|482573|6410|41400|82383|482573|6410|41400|3|11|3180|17|1318.01|65.90|1383.91|1.63|821.27|355.86|230.91|731.24|888.80| +2451994|60163|11966|95209|762289|4122|5602|95209|762289|4122|5602|55|24|3182|35|109.55|2.19|111.74|32.97|56.70|93.11|5.75|10.69|91.86| +|53832|5840|62635|1488956||21292||1488956|524|21292|14||3182||||||||||596.17| +2452194|69005|6213|21223|1029109|642|28575|21223|1029109|642|28575|35|17|3183|17|229.33|20.63|249.96|70.56|108.63|181.17|39.49|8.67|199.82| +2452629|34571|5673|38026|1466599|5219|28239|38026|1466599|5219|28239|51|32|3184|2|11.84|1.06|12.90|66.36|2.50|7.45|2.23|2.16|69.92| +2452646|35419|13332|71004|1689955|6059|29810|71004|1689955|6059|29810|24|23|3184|4|0.00|0.00|0.00|30.40|41.76|0.00|0.00|0.00|72.16| +2451289|32996|14300|60719|1578133|794|18655|60719|1578133|794|18655|20|14|3186|48|38.88|0.00|38.88|91.63|1052.16|14.38|21.56|2.94|1143.79| +2451342|18361|17434|47476|1626817|1358|1546|47476|1626817|1358|1546|1|1|3188|29|521.13|20.84|541.97|60.16|875.51|479.43|26.68|15.02|956.51| +2451199|46914|6764|77341|646248|3508|42415|77341|646248|3508|42415|7|28|3188|32|307.20|24.57|331.77|14.97|2099.84|21.50|131.42|154.28|2139.38| +2451299|40695|1994|56729|1909163|797|36831|56729|1909163|797|36831|43|5|3188|6|170.04|6.80|176.84|89.82|163.20|164.93|3.16|1.95|259.82| +2451628|9822|7654|71618|371134|3097|27013|71618|371134|3097|27013|22|16|3189|15|752.25|67.70|819.95|95.95|1074.60|692.07|39.11|21.07|1238.25| +2451588|69358|2305|39752|1241588|5824|31213|39752|1241588|5824|31213|1|25|3189|20|1616.20|64.64|1680.84|70.38|314.20|1519.22|67.88|29.10|449.22| +2451773|58661|8314|55786|609711|4436|49561|55786|609711|4436|49561|58|1|3189|55|1352.45|108.19|1460.64|73.47|2479.40|1230.72|52.34|69.39|2661.06| +2451347|54116|13490|52563|1516516|116|2469|52563|1516516|116|2469|13|16|3190|12|1071.84|42.87|1114.71|15.55|627.36|825.31|29.58|216.95|685.78| +2451487|77257|14098|41956|1830812|4115|47371|41956|1830812|4115|47371|26|14|3190|52|152.88|1.52|154.40|87.60|55.64|30.57|105.18|17.13|144.76| +2452341|64819|8021|46116|1587383|4203|33823|46116|1587383|4203|33823|33|7|3192|29|4553.58|136.60|4690.18|94.63|3140.41|318.75|2498.54|1736.29|3371.64| +2452331|44169|9103|77369|15459|1480|2544|77369|15459|1480|2544|31|11|3193|43|169.85|13.58|183.43|81.74|183.61|0.00|20.38|149.47|278.93| +2452408|33713|10971|78758|1228538|4365|11392|78758|1228538|4365|11392|55|2|3194|27|2527.74|0.00|2527.74|19.15|601.83|1263.87|1124.84|139.03|620.98| +2452385|70878|4307|40654|1844546|4438|13130|40654|1844546|4438|13130|51|25|3194|5|448.35|13.45|461.80|27.45|52.70|188.30|239.24|20.81|93.60| +2452299|29757|6251|81693|1161961|1429|27250|81693|1161961|1429|27250|15|34|3194|23|1401.39|84.08|1485.47|72.28|710.01|1275.26|56.75|69.38|866.37| +2451276|37510|494|45406|919376|5793|1363|45406|919376|5793|1363|55|25|3195|24|741.12|29.64|770.76|59.68|451.92|711.47|17.79|11.86|541.24| +2451369|55941|7429|13841|1380627|7060|21846|13841|1380627|7060|21846|13|18|3196|64|3751.68|337.65|4089.33|38.06|493.44|1012.95|465.58|2273.15|869.15| +2451433|61976|10393|70438|324479|5367|45008|70438|324479|5367|45008|4|16|3199|52|185.64|12.99|198.63|64.55|29.64|157.79|5.57|22.28|107.18| +2451881|66946|371|96637|650322|1160|13998|96637|650322|1160|13998|37|28|3200|47|814.51|73.30|887.81|94.91|184.24|219.91|267.57|327.03|352.45| +2451818|65662|17363|97998|1367304|1410|41116|97998|1367304|1410|41116|1|8|3201|2|3.48|0.10|3.58|35.80|6.98|1.49|0.69|1.30|42.88| +||11495||1246651|6860|40669|46795|1246651||40669|20|32|3201||0.00||0.00|||0.00|||341.25| +2452044|45596|17327|95751|1355945|6892|12286|95751|1355945|6892|12286|37|4|3201|25|1534.00|76.70|1610.70|22.29|865.75|1257.88|19.32|256.80|964.74| +2451856|56076|11561|75585|1353937|2336|34335|75585|1353937|2336|34335|29|7|3201|1|75.05|3.75|78.80|16.51|47.04|69.04|1.44|4.57|67.30| +2451825|17878|7733|46777|788058|4092|35326|46777|788058|4092|35326|2|29|3202|10|398.30|31.86|430.16|67.65|321.20|155.33|184.65|58.32|420.71| +2451938|64916|11054|15247|1787945|6596|19247|15247|1787945|6596|19247|14|7|3202|16|277.92|19.45|297.37|33.20|144.48|105.60|86.16|86.16|197.13| +2451787|19513|6695|44480|1414654|28|12202|44480|1414654|28|12202|23|19|3202|10|1791.40|143.31|1934.71|67.96|160.40|268.71|1202.92|319.77|371.67| +2451179|49181|15854|49020|898508|5596|29837|49020|898508|5596|29837|20|33|3203|4|214.44|12.86|227.30|2.90|238.84|15.01|33.90|165.53|254.60| +2451247|58128|2498|35982|1509978|1221|42817|35982|1509978|1221|42817|37|24|3203|24|388.80|34.99|423.79|18.27|242.88|23.32|310.65|54.83|296.14| +2451185|70480|3517|79270||5943|||||28703|||3204|||2.63|68.59|87.44||17.14|25.87||113.93| +2451194|25490|17114|84675|585334|1193|36252|84675|585334|1193|36252|44|15|3206|46|193.20|15.45|208.65|12.92|128.80|71.48|35.29|86.43|157.17| +2451192|71451|3500|68753|1193512|1918|30324|68753|1193512|1918|30324|10|24|3206|22|1350.58|13.50|1364.08|2.51|1075.36|621.26|299.02|430.30|1091.37| +2451237||11336|||6645||||||10|23|3206|16|606.08|6.06|||15.52||3.63|69.10|| +2452858|58099|13803|22889|1195825|314|450|22889|1195825|314|450|51|3|3207|4|189.20|0.00|189.20|1.57|20.00|145.68|10.00|33.52|21.57| +2452637|82716|14755|6108|1664260|5612|38117|6108|1664260|5612|38117|48|27|3207|3|33.84|0.67|34.51|81.52|39.99|32.82|0.62|0.40|122.18| +2452579|61675|8757|61152|176883|261|34839|61152|176883|261|34839|9|8|3207|34|1036.32|0.00|1036.32|54.08|16.66|839.41|55.13|141.78|70.74| +2451956|61356|7355|||1763|40708|33120||1763|40708||33|3209||9.58|0.67||69.15||4.02|2.44||| +2452260|68766|15063|46947|1474333|3319|24305|46947|1474333|3319|24305|25|7|3210|9|1221.30|36.63|1257.93|90.35|255.15|720.56|190.28|310.46|382.13| +2452560|79267|7317|12776|1575638|1347|25009|12776|1575638|1347|25009|31|1|3211|77|423.50|33.88|457.38|62.74|1025.64|186.34|111.46|125.70|1122.26| +||719|88935||||||2754||57|24|3211|32|6270.72|||17.51||||342.39|| +2452317|6734|7677|56094|817616|385|32307|56094|817616|385|32307|59|5|3211|13|997.75|0.00|997.75|77.90|126.23|439.01|44.69|514.05|204.13| +2452451|59202|1093|32610|261739|821|29503|32610|261739|821|29503|5|17|3211|13|1023.23|30.69|1053.92|5.63|166.27|931.13|33.15|58.95|202.59| +2452491|46814|17916|30063|1444696|6176|30009|30063|1444696|6176|30009|36|35|3212|10|298.60|0.00|298.60|38.76|106.30|256.79|22.15|19.66|145.06| +2452459|47600|1569||247033|3833|13833||247033|||55||3212|8||||97.63|||7.14|57.81|| +||627|21126||||21126||1829||11|31|3213|7|15.96|0.79||65.31|||3.76|6.14|| +2451022|67603|637|54061|427720|2179|37838|54061|427720|2179|37838|7|4|3215|11|185.90|5.57|191.47|31.37|254.98|176.60|2.04|7.26|291.92| +2452087|7958|7105|49339|782910|128|3746|49339|782910|128|3746|50|3|3216|26|315.12|18.90|334.02|32.63|1576.12|302.51|5.29|7.32|1627.65| +2452780|77550|10908|56668|323984|6053|1722|56668|323984|6053|1722|51|7|3217|16|1875.84|150.06|2025.90|91.14|908.80|1631.98|168.26|75.60|1150.00| +|45128|5743|||338||96345|841365||1358||21|3218||2473.68|||2.97|1884.74|717.36|632.27||| +2451755||11881||1632615|1032||86245|1632615|||||3220||97.52||100.44|||96.54|0.31|0.67|204.76| +|68814|7137|60073|1748827|353|||1748827|||51|9|3222|27|1338.93||1352.31|18.37||669.46|||| +2451815|70979|13130|38148|810039|2743|34208|38148|810039|2743|34208|50|17|3223|55|625.35|18.76|644.11|23.77|809.05|131.32|158.08|335.95|851.58| +2451528|45295|8587|93977|1362938|5852|21613|93977|1362938|5852|21613|52|3|3224|1|111.67|1.11|112.78|41.17|64.76|83.75|19.54|8.38|107.04| +2451521|36710|284|91526|194293|2660|12627|91526|194293|2660|12627|16|32|3224|74|6418.76|385.12|6803.88|33.17|3348.50|2824.25|790.79|2803.72|3766.79| +2451515|68922|1058|59683|967887|1691|12295|59683|967887|1691|12295|13|8|3224|39|2072.07|0.00|2072.07|99.40|609.18|1326.12|425.19|320.76|708.58| +2451650|81789|3394|82252|162546|4887|3181|82252|162546|4887|3181|31|21|3224|5|401.05|24.06|425.11|8.07|192.70|116.30|205.02|79.73|224.83| +2451330|37540|3620|22994|964193|4945|39744|22994|964193|4945|39744|34|8|3225|30|243.90|19.51|263.41|6.13|627.30|92.68|148.19|3.03|652.94| +2451338|48249|10507|68761|754519|3124|6929|68761|754519|3124|6929|58|17|3226|49|2226.56|155.85|2382.41|45.03|795.27|757.03|1146.23|323.30|996.15| +2451818|10788|1976|68929|1332478|6727|27725|68929|1332478|6727|27725|25|18|3227|85|331.50|6.63|338.13|70.86|867.00|328.18|2.15|1.17|944.49| +2452188|42942|2945|46787|712416|2045|9721|46787|712416|2045|9721|37|15|3229|48|4212.00|168.48|4380.48|18.92|3103.68|4212.00|0.00|0.00|3291.08| +2452409|24361|12037|84880|1686959|3885|14011|84880|1686959|3885|14011|19|27|3230|28|1764.00|70.56|1834.56|77.66|518.84|388.08|619.16|756.76|667.06| +2452400|74446|799|80368|957705|3462|12235|80368|957705|3462|12235|13|18|3230|24|862.56|43.12|905.68|88.98|0.00|457.15|202.70|202.71|132.10| +2452672|14668|12288|33610|910434|1658|30529|33610|910434|1658|30529|51|28|3231|54|419.04|4.19|423.23|78.04|1204.74|75.42|271.45|72.17|1286.97| +2451933||8636|688|||37186||1418454|||||3232||||837.73||151.32||208.74|536.77|| +2451972|57821|17420|48557|1501168|4626|3284|48557|1501168|4626|3284|8|8|3232|36|667.08|26.68|693.76|69.93|1413.00|413.58|101.40|152.10|1509.61| +2451998|43102|11347|37806|987610|1414|18818|37806|987610|1414|18818|49|25|3234|42|410.34|24.62|434.96|29.49|1179.78|82.06|233.07|95.21|1233.89| +2452067|5665|11480|73618|1301008|3598|18188|73618|1301008|3598|18188|55|22|3234|78|4847.70|242.38|5090.08|39.89|0.00|1939.08|2879.53|29.09|282.27| +2452182|57883|5936|48991|1898623|3808|3040|48991|1898623|3808|3040|5|21|3234|10|266.20|2.66|268.86|27.44|851.90|141.08|3.75|121.37|882.00| +2452212|32216|17719|78260|38537|2461|19138|78260|38537|2461|19138|56|22|3234|9|1326.24|26.52|1352.76|44.47|771.12|835.53|230.63|260.08|842.11| +2452048|63026|17119|23430|125851|3919|48340|23430|125851|3919|48340|35|2|3235|61|2155.13|64.65|2219.78|60.37|2901.77|668.09|104.09|1382.95|3026.79| +2451303|78585|8467|76286|162339|5192|30798|76286|162339|5192|30798|31|17|3236|17|2655.23|79.65|2734.88|97.81|303.45|398.28|1331.60|925.35|480.91| +2451882|13916|15223|66280|695761|4889|16370|66280|695761|4889|16370|1|27|3237|6|129.90|5.19|135.09|99.58|46.56|98.72|25.56|5.62|151.33| +2451940|30011|12529|25644|705726|6161|26643|25644|705726|6161|26643|49|1|3238|34|241.40|9.65|251.05|6.27|552.16|7.24|128.78|105.38|568.08| +2451992|11263|15692|40971|571316|4508|49653|40971|571316|4508|49653|13|34|3238|71|88.75|7.10|95.85|75.91|237.14|47.92|40.01|0.82|320.15| +2451186|43512|43|66660|24197|4934|19271|66660|24197|4934|19271|37|20|3241|18|1857.96|74.31|1932.27|48.16|488.88|1672.16|29.72|156.08|611.35| +2452855|61529|8739|88088|1863458|4875|27092|88088|1863458|4875|27092|24|25|3243|20|2962.00|29.62|2991.62|17.13|417.60|1184.80|1581.70|195.50|464.35| +2452848|73235|8559|87354|1835628|349|37423|87354|1835628|349|37423|42|6|3243|1|64.49|2.57|67.06|45.99|41.99|57.39|2.84|4.26|90.55| +2452764|67194|4047|6541|1425915|5289|23063|6541|1425915|5289|23063|25|12|3243|38|120.84|7.25|128.09|27.75|2848.10|21.75|13.87|85.22|2883.10| +2452831|80408|15393|21791|1456678|3118|48805|21791|1456678|3118|48805|6|10|3243|52|7534.80|678.13|8212.93|9.25|1202.24|1582.30|4107.22|1845.28|1889.62| +2452870|25647|889|92244|1883353|2462|9124|92244|1883353|2462|9124|57|19|3243|64|35.84|1.79|37.63|24.76|872.96|33.68|1.59|0.57|899.51| +2452656|24358|11079|47358|1656495|32|36016|47358|1656495|32|36016|42|10|3244|4|273.32|24.59|297.91|71.76|58.56|21.86|5.02|246.44|154.91| +2452782|77537|5868|17469|473486|5503|23520|17469|473486|5503|23520|12|26|3244|1|1.70|0.15|1.85|74.44|1.64|0.39|1.00|0.31|76.23| +||15609|37699|613662|||37699||4472|45235|55||3245|39|||739.87|23.53|111.54|41.48||474.50|183.47| +2452855|49596|14923|52081|588746|3458|32523|52081|588746|3458|32523|55|34|3245|16|527.84|10.55|538.39|60.36|9.28|15.83|450.56|61.45|80.19| +||15852|||||17555|477380|3158||51||3245|7|434.70|21.73||82.83||373.84|54.16|6.70|| +2451310|42093|12316|45862|56977|6045|27960|45862|56977|6045|27960|58|24|3246|41|3102.88|0.00|3102.88|90.34|1329.63|1272.18|1354.71|475.99|1419.97| +2452403|17723|14953|27594|1784877|913|16996|27594|1784877|913|16996|47|25|3247|81|1329.21|13.29|1342.50|13.50|2658.42|1315.91|3.05|10.25|2685.21| +2452296|13354|4021|88119|1340318|3717|17916|88119|1340318|3717|17916|55|25|3247|67|6984.75|349.23|7333.98|67.79|5308.41|3562.22|2635.34|787.19|5725.43| +2452321|33219|7607|54815|707809|4878|2778|54815|707809|4878|2778|49|33|3249|64|6353.92|381.23|6735.15|94.29|2874.24|190.61|1725.72|4437.59|3349.76| +2451311|45714|668|82073|1101210|347|1484|82073|1101210|347|1484|50|33|3250|14|193.06|9.65|202.71|46.98|579.32|131.28|59.92|1.86|635.95| +2452739|42688|2400|97646|475581|5869|27004|97646|475581|5869|27004|54|8|3253|40|1222.80|0.00|1222.80|39.66|140.00|782.59|299.34|140.87|179.66| +2452754|67774|11952|12465|1897226|5634|37195|12465|1897226|5634|37195|13|13|3253|2|92.78|4.63|97.41|80.82|198.22|63.09|17.51|12.18|283.67| +2451213|71395|11024|78995|1680941|2440|14786|78995|1680941|2440|14786|43|23|3255|19|1037.02|93.33|1130.35|46.78|145.92|425.17|36.71|575.14|286.03| +2451296|57109|17864|46170|85708|3999|21090|46170|85708|3999|21090|56|35|3255|24|2011.92|140.83|2152.75|0.53|429.12|865.12|688.08|458.72|570.48| +2451097|18225|15685|37123|1657210|4050|3737|37123|1657210|4050|3737|20|15|3258|29|382.51|34.42|416.93|54.01|503.44|0.00|225.68|156.83|591.87| +2451034|55059|9592|36380|1598534|5041|23195|36380|1598534|5041|23195|46|22|3258|6|1197.78|0.00|1197.78|98.11|212.10|658.77|145.53|393.48|310.21| +2451057|82036|14114|428|139095|1762|17229|428|139095|1762|17229|44|8|3258|54|883.98|44.19|928.17|49.52|785.70|220.99|304.97|358.02|879.41| +2451088|40504|6194|59356|534935|914|43646|59356|534935|914|43646|52|33|3262|22|34.10|0.68|34.78|42.46|86.02|31.71|1.76|0.63|129.16| +2451531|64190|3847|57766|380219|719|37572|57766|380219|719|37572|38|24|3263|5|140.80|8.44|149.24|76.31|152.50|1.40|72.48|66.92|237.25| +||17626|55159||||55159|182468||12239|||3264|||0.00|||43.90|69.84|22.59||81.86| +2451616|77408|13976|90778|1567653|1775|20250|90778|1567653|1775|20250|1|6|3264|26|233.74|7.01|240.75|45.87|340.08|170.63|40.39|22.72|392.96| +2452257|74119|2387|46092|191546|200|2454|46092|191546|200|2454|7|12|3265|22|3301.76|165.08|3466.84|76.76|1956.46|2707.44|53.48|540.84|2198.30| +2451997|19196|3818|14594|1846486|2025|33917|14594|1846486|2025|33917|23|13|3266|2|393.68|23.62|417.30|41.35|213.44|263.76|16.88|113.04|278.41| +2452116|41657|5203|21031|1239297|1563|30629|21031|1239297|1563|30629|55|30|3266|13|1557.01|0.00|1557.01|44.73|855.53|435.96|1121.05|0.00|900.26| +||6722||782746||||782746|88||||3266||||744.81|49.47||667.52|8.08||123.75| +2452744|42006|17197|82773|1536644|4115|15186|82773|1536644|4115|15186|57|29|3267|9|119.34|3.58|122.92|19.03|192.87|36.99|20.58|61.77|215.48| +2452821|25933|8887|65092|1503916|6293|2717|65092|1503916|6293|2717|9|10|3267|38|2092.28|104.61|2196.89|32.34|286.90|271.99|418.66|1401.63|423.85| +2452677|82501|4513|||3376||14675|950449|3376||||3270||||273.50|92.81|474.48|188.19|54.57||| +2452472|69996|16665|10669|1440914|5849|847|10669|1440914|5849|847|54|14|3271|29|1358.36|81.50|1439.86|41.67|135.72|135.83|317.85|904.68|258.89| +2451982|52997|481|40749|1068701|3053|605|40749|1068701|3053|605|5|3|3272|14|19.18|0.76|19.94|9.08|250.46|17.83|1.26|0.09|260.30| +2451926|56007|11101||875036|6529||49143||6529||49||3272|14|1601.74||1745.89||485.94|864.93|||| +2451964|967|1562|60438|292246|6454|36262|60438|292246|6454|36262|49|6|3272|15|119.55|2.39|121.94|94.39|182.55|53.79|11.83|53.93|279.33| +2452457|78283|13951|38434|689575|5516|6257|38434|689575|5516|6257|23|32|3273|36|1797.12|161.74|1958.86|8.47|513.36|1006.38|513.98|276.76|683.57| +2452533|63623|6147|80575|1888614|1900|26739|80575|1888614|1900|26739|47|16|3273|61|161.65|3.23|164.88|73.39|107.97|30.71|58.92|72.02|184.59| +2451938|40519|10565|37437|211678|766|43054|37437|211678|766|43054|2|3|3274|41|1665.83|33.31|1699.14|49.15|1735.12|1515.90|100.45|49.48|1817.58| +2452646|2965|13407|70516|1099429|7095|8408|70516|1099429|7095|8408|31|20|3275|18|684.90|6.84|691.74|27.44|24.30|267.11|217.25|200.54|58.58| +2452744|71834|17167|96339|24614|6936|34411|96339|24614|6936|34411|15|2|3275|2|68.50|4.11|72.61|20.66|252.96|53.43|3.31|11.76|277.73| +2452796|78731|11013|52132|1201130|528|4326|52132|1201130|528|4326|31|25|3276|15|1237.65|74.25|1311.90|52.93|595.80|470.30|230.20|537.15|722.98| +2452007|20351|15721|54428|1737040|4805|20170|54428|1737040|4805|20170|13|19|3277|24|2255.76|45.11|2300.87|22.68|356.16|789.51|439.87|1026.38|423.95| +2452147|10605|9239|6370|1818076|4629|23135|6370|1818076|4629|23135|35|18|3277|30|3659.40|256.15|3915.55|70.88|197.70|3037.30|292.38|329.72|524.73| +2452112|51331|3743|82513|720568|6623|1431|82513|720568|6623|1431|32|6|3277|85|8302.80|664.22|8967.02|68.41|2587.40|3238.09|4963.41|101.30|3320.03| +2451944|68712|4481|97224|1812523|5869|19968|97224|1812523|5869|19968|11|30|3277|68|4897.36|293.84|5191.20|49.05|2129.08|1469.20|3016.78|411.38|2471.97| +2451504|38288|8642|38010|428613|1616|23819|38010|428613|1616|23819|4|17|3278|33|347.49|17.37|364.86|46.56|1867.80|271.04|48.92|27.53|1931.73| +2451321|43180|9013|89730|26624|4671|40765|89730|26624|4671|40765|58|18|3279|3|96.63|4.83|101.46|40.86|70.86|17.39|26.14|53.10|116.55| +2452813|36566|16551|73779|218525|5671|9217|73779|218525|5671|9217|55|16|3283|21|3336.06|100.08|3436.14|36.66|2300.76|2869.01|317.59|149.46|2437.50| +|6115|15462|||902|33995||1132256|||||3283|||15.11|203.99|29.90|||130.99|6.90|556.33| +2452806|26438|12705|51662|1033965|621|19724|51662|1033965|621|19724|60|31|3283|31|2028.64|40.57|2069.21|89.59|1211.17|507.16|152.14|1369.34|1341.33| +2452195|26302|10625|69817|1078044|7071|29698|69817|1078044|7071|29698|51|33|3284|12|630.36|37.82|668.18|36.74|83.40|126.07|353.00|151.29|157.96| +2452856|50480|5971|58199|1127066|7133|20400|58199|1127066|7133|20400|21|24|3285|9|268.29|18.78|287.07|43.59|70.56|13.41|211.55|43.33|132.93| +2452967|73082|474|70480|1288468|1303|35691|70480|1288468|1303|35691|57|35|3285|1|16.00|1.12|17.12|58.58|9.41|14.72|0.30|0.98|69.11| +2452412|71039|15093|76675|1850976|4477|49178|76675|1850976|4477|49178|29|17|3286|15|690.90|13.81|704.71|0.55|312.00|221.08|314.77|155.05|326.36| +|74266|10304||786229||||786229||18291|47|21|3288|29|877.25||894.79|36.08|||||226.75| +2451629|49658|890|31705|1246459|4413|4767|31705|1246459|4413|4767|46|25|3291|36|1805.76|72.23|1877.99|40.73|1188.00|144.46|1096.45|564.85|1300.96| +2451650|60738|11374|50360|742096|3879|2415|50360|742096|3879|2415|43|15|3291|22|1288.76|77.32|1366.08|59.29|224.84|257.75|216.51|814.50|361.45| +2452489|67032|4851|19229|1822829|6429|28829|19229|1822829|6429|28829|9|5|3292|13|2310.10|0.00|2310.10|25.81|978.38|1848.08|309.55|152.47|1004.19| +2452383|72146|11917|92887|462688|1482|939|92887|462688|1482|939|19|8|3292|8|1475.84|14.75|1490.59|77.18|723.12|1106.88|77.48|291.48|815.05| +2451464|69793|17186|38710|846721|4779|40437|38710|846721|4779|40437|31|33|3293|9|97.83|6.84|104.67|22.37|269.10|70.43|6.02|21.38|298.31| +2451637|19131|16243|99444|1723107|4370|43680|99444|1723107|4370|43680|19|2|3293|73|1030.03|41.20|1071.23|87.02|666.49|41.20|19.77|969.06|794.71| +2451886|73710|1187|15180||||15180|1852615||49838|||3294|||0.12|2.24|27.01|||0.02|0.03|28.33| +2451978|79443|13619|23422|1586023|5071|23272|23422|1586023|5071|23272|55|21|3294|8|98.16|5.88|104.04|91.05|546.96|15.70|28.86|53.60|643.89| +2452806|72902|8736|876|556015|4754|42693|876|556015|4754|42693|49|24|3295|38|4737.84|142.13|4879.97|17.26|1848.70|1610.86|2095.07|1031.91|2008.09| +2451638|57097|4261|63127|480730|5533|47536|63127|480730|5533|47536|22|7|3296|58|104.40|5.22|109.62|1.85|0.00|14.61|23.34|66.45|7.07| +2451792|48692|2966|67854|285749|5691|38786|67854|285749|5691|38786|26|2|3296|4|2.92|0.26|3.18|95.81|11.72|1.54|0.46|0.92|107.79| +2451736|81633|12002|54492|980062|4385|14755|54492|980062|4385|14755|46|26|3296|37|7464.75|149.29|7614.04|77.50|3634.14|4180.26|1905.00|1379.49|3860.93| +2452745|16968|11853|78315|1061428|4541|32854|78315|1061428|4541|32854|39|4|3297|51|4988.82|349.21|5338.03|59.51|2230.23|3442.28|262.91|1283.63|2638.95| +2452702|82698|10705|7519|391156|1993|12477|7519|391156|1993|12477|7|29|3297|83|7553.00|0.00|7553.00|8.34|3098.39|2568.02|996.99|3987.99|3106.73| +2451313|2068|5365|50973|928915|1257|32752|50973|928915|1257|32752|58|30|3299|53|696.42|41.78|738.20|28.25|311.64|250.71|31.19|414.52|381.67| +2451312|21966|17486|67431|1811129|1580|41678|67431|1811129|1580|41678|4|18|3299|5|329.85|6.59|336.44|46.50|85.50|95.65|21.07|213.13|138.59| +2452371|60002|4531|83210|330964|3518|24919|83210|330964|3518|24919|53|35|3301|10|704.40|35.22|739.62|27.25|407.80|204.27|20.00|480.13|470.27| +2452481|55263|13075|35972|1357611|6450|7289|35972|1357611|6450|7289|49|26|3301|63|415.80|16.63|432.43|99.56|4.41|390.85|14.22|10.73|120.60| +2452387|52378|10885|29371|1443476|4781|1286|29371|1443476|4781|1286|39|29|3301|3|269.85|10.79|280.64|24.87|0.00|159.21|64.17|46.47|35.66| +2452395||11105|77427|||30092|77427||3737|30092|1||3301|4||||77.75|||9.32||| +2452668|46627|2472|36621|1142851|3621|37491|36621|1142851|3621|37491|27|24|3302|58|1289.34|103.14|1392.48|18.19|1987.66|657.56|360.11|271.67|2108.99| +2451239|28540|10807|76522|1803235|4422|30198|76522|1803235|4422|30198|40|4|3303|33|2180.97|43.61|2224.58|32.46|386.76|2093.73|65.43|21.81|462.83| +2451247|67802|10982|88677|169749|2468|1940|88677|169749|2468|1940|2|9|3303|12|571.68|28.58|600.26|59.71|959.64|383.02|7.54|181.12|1047.93| +2452429|29875|13674|52277|1272151|2425|21499|52277|1272151|2425|21499|31|21|3306|12|646.80|51.74|698.54|93.86|107.76|109.95|354.32|182.53|253.36| +2452465|47892|10476|92530|1325150|5770|13190|92530|1325150|5770|13190|49|13|3306|8|1363.04|81.78|1444.82|29.92|465.04|845.08|202.00|315.96|576.74| +2452383|59538|14555|11855|1302610|1012|45014|11855|1302610|1012|45014|31|14|3307|17|749.70|0.00|749.70|40.00|1221.79|622.25|113.43|14.02|1261.79| +2452446|69881|5231|47024|803067|1510|23106|47024|803067|1510|23106|17|35|3307|70|5980.10|538.20|6518.30|88.84|1388.10|1973.43|2924.86|1081.81|2015.14| +2452501|43783|9683|2769|1293681|1393|7905|2769|1293681|1393|7905|23|35|3307|11|1020.47|10.20|1030.67|12.58|441.65|438.80|133.78|447.89|464.43| +2452284|44192|12721|54148|71903|505|25860|54148|71903|505|25860|55|18|3307|22|3197.70|159.88|3357.58|22.94|279.62|511.63|1531.05|1155.02|462.44| +2452812|69060|14862|90762|1163745|6775|48558|90762|1163745|6775|48558|27|35|3308|63|2264.85|135.89|2400.74|52.11|429.03|1970.41|188.44|106.00|617.03| +2451731|64891|6740|38413|443430|6494|35911|38413|443430|6494|35911|37|18|3309|63|345.24|24.16|369.40|10.10|563.22|110.47|185.46|49.31|597.48| +2452031|51130|15824|24544|1385674|851|28781|24544|1385674|851|28781|50|18|3310|38|2299.00|137.94|2436.94|14.70|1877.58|551.76|576.58|1170.66|2030.22| +2451908|54286|7442|67546||7061|34595|67546|1539127|7061||||3310|1||2.38||96.05|91.88|33.32||3.69|190.31| +2451821|38103|11563|49845|1770251|5057|35416|49845|1770251|5057|35416|41|25|3310|20|1830.60|54.91|1885.51|59.65|684.00|1556.01|32.95|241.64|798.56| +2452695|39931|9768|89788||7192||89788|98490||27894|49|18|3312||70.47|4.22||31.55|102.51|1.40|||| +2451384|52948|5816|17734|1300863|4345|11874|17734|1300863|4345|11874|25|7|3314|15|189.30|1.89|191.19|75.87|57.30|166.58|22.49|0.23|135.06| +2452064|84283|4897|61229|965172|7162|7905|61229|965172|7162|7905|19|17|3315|8|1140.40|79.82|1220.22|21.32|166.56|535.98|314.29|290.13|267.70| +2451555|82850|604|84895|455680|5218|7603|84895|455680|5218|7603|49|34|3316|19|355.68|28.45|384.13|19.95|231.61|252.53|82.52|20.63|280.01| +2451753|43216|17383|10967|725314|461|15161|10967|725314|461|15161|32|32|3317|38|537.70|26.88|564.58|0.64|408.50|462.42|15.05|60.23|436.02| +2451615|69167|9902|5105|121751|5950|3396|5105|121751|5950|3396|14|28|3317|4|492.72|4.92|497.64|45.28|395.80|78.83|318.69|95.20|446.00| +2452006|37160|1079|46069|1401856|5290|1877|46069|1401856|5290|1877|14|18|3319|1|16.40|0.65|17.05|63.76|49.20|0.49|10.81|5.10|113.61| +2452905|42438|3006|55905|1891042|2179|2116|55905|1891042|2179|2116|37|10|3322|21|1286.67|64.33|1351.00|82.31|652.89|476.06|259.39|551.22|799.53| +||6467|66120||1996|26734|66120|1810858|1996|26734|||3323||149.27|||56.28|68.77||||| +2451914|77096|751|46100|1787247|6441|44435|46100|1787247|6441|44435|47|30|3323|1|9.23|0.36|9.59|2.75|17.63|1.47|3.25|4.51|20.74| +2451315|62190|7744|85819|528979|5739|14899|85819|528979|5739|14899|19|17|3324|54|5855.76|117.11|5972.87|66.84|2439.72|468.46|2316.53|3070.77|2623.67| +2452241|62218|17215|37428|1617429|6484|49047|37428|1617429|6484|49047|25|9|3325|3|30.69|1.53|32.22|9.00|56.70|19.02|1.05|10.62|67.23| +2452421|49559|8231|9240|1449856|72|9041|9240|1449856|72|9041|49|5|3325|52|1454.96|43.64|1498.60|11.60|719.68|14.54|1209.95|230.47|774.92| +2451791|80606|11716|79944|378985|3626|8267|79944|378985|3626|8267|14|29|3326|9|916.38|45.81|962.19|23.35|242.19|210.76|395.14|310.48|311.35| +2452428|44865|203|30076|771626|4823|32373|30076|771626|4823|32373|59|6|3328|53|209.88|18.88|228.76|76.15|788.64|92.34|38.78|78.76|883.67| +2452417|50199|7295|24182|1173491|4047|48156|24182|1173491|4047|48156|5|20|3330|42|372.54|18.62|391.16|39.73|188.16|182.54|76.00|114.00|246.51| +2451764|36658|11195||||||1710924|||1||3332|16|||257.12|83.58|95.84|154.27||99.77|179.42| +2451692|66802|12433|7168|155180|5145|12357|7168|155180|5145|12357|26|16|3332|10|585.40|23.41|608.81|41.99|33.10|257.57|288.49|39.34|98.50| +2451886|29288|12913|35137|18549|6314|48126|35137|18549|6314|48126|2|9|3332|21|418.95|16.75|435.70|58.44|73.92|175.95|228.42|14.58|149.11| +2451666|36495|4759|6082|339636|3943|42697|6082|339636|3943|42697|25|29|3336|19|288.42|17.30|305.72|49.31|125.40|147.09|115.89|25.44|192.01| +||6568||236150|||28230|236150|3270|3190||8|3336||3098.38|185.90|3284.28|||1456.23|1543.62||833.64| +2452511|26130|9419|49485|110213|6588|12116|49485|110213|6588|12116|57|31|3338|9|278.91|11.15|290.06|34.41|146.79|150.61|106.48|21.82|192.35| +2452380|82347|7859|98641|778518|5242|31519|98641|778518|5242|31519|3|3|3338|3|72.51|2.17|74.68|82.89|20.04|10.87|44.38|17.26|105.10| +2451912|8507|12607|60854|1897092|1683|34674|60854|1897092|1683|34674|53|25|3339|28|2408.28|0.00|2408.28|40.38|109.20|553.90|1335.15|519.23|149.58| +2452054|70326|16291|48202|1145739|444|49605|48202|1145739|444|49605|53|32|3339|5|557.15|33.42|590.57|22.37|244.15|89.14|421.20|46.81|299.94| +2452254|46670|9339|76139|1553666|4793|27120|76139|1553666|4793|27120|47|23|3341|8|611.68|36.70|648.38|87.12|352.64|165.15|35.72|410.81|476.46| +2452261|68575|7203|18888|863929|3370|4811|18888|863929|3370|4811|47|29|3341|11|444.29|35.54|479.83|76.55|394.90|111.07|36.65|296.57|506.99| +2452579|60422|9471|89073|509002|3331|413|89073|509002|3331|413|53|23|3342|81|8334.90|83.34|8418.24|44.36|2553.93|1917.02|1411.93|5005.95|2681.63| +2451180|76681|14929|27992|128615|3285|16708|27992|128615|3285|16708|28|10|3344|92|9133.76|456.68|9590.44|96.08|3760.96|8129.04|20.09|984.63|4313.72| +2451130|9819|11950|98184|116702|2867|3785|98184|116702|2867|3785|16|14|3345|1|79.80|1.59|81.39|30.40|12.21|7.98|63.20|8.62|44.20| +2452759|11729|3429|6986|896107|75|18796|6986|896107|75|18796|3|8|3348|4|3.40|0.00|3.40|51.26|11.36|1.97|0.58|0.85|62.62| +2452415|23809|5083|37441|1693525|2409|44940|37441|1693525|2409|44940|33|29|3349|3|79.62|3.98|83.60|17.51|0.00|3.18|42.80|33.64|21.49| +2452684|39702|5419|58145|1415006|5820|36681|58145|1415006|5820|36681|39|33|3349|5|26.85|1.61|28.46|84.10|72.95|8.05|8.46|10.34|158.66| +2452590|68639|3663|89481|1618393|5352|3464|89481|1618393|5352|3464|54|22|3349|3|31.23|1.87|33.10|97.66|42.93|25.29|0.23|5.71|142.46| +2451955|33124|2729|36587|388532|2077|775|36587|388532|2077|775|55|11|3351|1|146.49|7.32|153.81|43.50|3.90|70.31|3.04|73.14|54.72| +2451311|59579|9100|28179|71403|851|866|28179|71403|851|866|44|6|3352|22|4902.04|49.02|4951.06|12.46|1800.70|2990.24|879.42|1032.38|1862.18| +2451324|70307|1363|39888|1693326|4717|18155|39888|1693326|4717|18155|1|6|3352|46|245.64|22.10|267.74|29.59|361.10|142.47|16.50|86.67|412.79| +2451390|75775|3962|3646|1032726|3759|4039|3646|1032726|3759|4039|43|7|3355|3|25.53|2.04|27.57|25.10|23.55|3.31|21.33|0.89|50.69| +2452453|48669|16589|29645|1298294|4990|47639|29645|1298294|4990|47639|15|5|3356|18|1651.86|115.63|1767.49|41.39|224.28|231.26|525.62|894.98|381.30| +2452401|71022|6679|91020|904973|4526|3279|91020|904973|4526|3279|51|19|3356|61|1376.16|0.00|1376.16|17.59|1158.39|192.66|556.24|627.26|1175.98| +2452407|12581|3453|12789|1542186|2783|3380|12789|1542186|2783|3380|7|32|3357|55|125.40|11.28|136.68|53.06|106.70|57.68|27.08|40.64|171.04| +2451578|37232|16622|49662|30174|1886|12845|49662|30174|1886|12845|14|7|3358|39|836.55|66.92|903.47|12.68|895.05|418.27|46.01|372.27|974.65| +2451266|51584|8326|16442|1227034|473|15465|16442|1227034|473|15465|7|6|3361|3|51.48|4.11|55.59|84.12|9.84|39.63|1.06|10.79|98.07| +2452123|64448|9752|3829|1845709|2436|3963|3829|1845709|2436|3963|59|1|3362|65|327.60|16.38|343.98|18.27|835.25|45.86|194.40|87.34|869.90| +2451201|36052|2491|65207|1240511|3462|42766|65207|1240511|3462|42766|26|32|3364|33|142.56|0.00|142.56|54.29|196.02|19.95|74.79|47.82|250.31| +2452618|45384|390|26535|522132|338|9429|26535|522132|338|9429|48|13|3365|71|393.34|11.80|405.14|89.39|2054.03|314.67|17.30|61.37|2155.22| +2451143|49296|2140|41561|85050|1692|11181|41561|85050|1692|11181|28|28|3366|1|4.73|0.42|5.15|69.64|5.71|3.50|0.38|0.85|75.77| +2451922|67240|6925|78482|489209|6336|33463|78482|489209|6336|33463|49|19|3368|15|140.70|1.40|142.10|22.64|85.95|52.05|5.31|83.34|109.99| +2451981|65536|11444|37036|1139246|1903|26513|37036|1139246|1903|26513|14|24|3370|58|103.24|8.25|111.49|47.42|236.06|77.43|16.00|9.81|291.73| +2451870|18733|14258|1812|400427|6242|15257|1812|400427|6242|15257|56|27|3370|19|1261.79|75.70|1337.49|65.24|466.26|1110.37|81.76|69.66|607.20| +2452388|20731|9887|79459|471375|3291|2407|79459|471375|3291|2407|31|8|3372|31|4238.01|84.76|4322.77|50.38|882.88|0.00|2669.94|1568.07|1018.02| +2452726|68264|246|42589|967168|6500|7032|42589|967168|6500|7032|39|6|3373|7|260.54|13.02|273.56|73.68|144.69|33.87|104.26|122.41|231.39| +2452859|64385|3912|9202|118471|4093|4962|9202|118471|4093|4962|9|31|3373|29|2724.55|190.71|2915.26|67.86|1430.28|299.70|2230.86|193.99|1688.85| +2452143|39562|9973|80633|1527595|1310|33872|80633|1527595|1310|33872|31|17|3377|34|1006.06|10.06|1016.12|40.24|670.82|633.81|327.58|44.67|721.12| +2451977|56010|14657|80285|1584110|4382|41054|80285|1584110|4382|41054|43|35|3377|21|1103.34|33.10|1136.44|58.38|520.38|242.73|688.48|172.13|611.86| +2451640|49859|15628|28341|764454|3414|29527|28341|764454|3414|29527|2|32|3378|38|625.48|56.29|681.77|87.02|1876.82|318.99|159.37|147.12|2020.13| +2451614|51566|4018|95983|614778|4773|44768|95983|614778|4773|44768|4|20|3378|36|1236.60|86.56|1323.16|73.91|210.96|667.76|17.06|551.78|371.43| +2451343|40063|14380|88264|1251667|2717|35720|88264|1251667|2717|35720|49|32|3379|45|2230.20|0.00|2230.20|96.28|155.70|2007.18|129.35|93.67|251.98| +2451891|20380|15290|94831|87041|2451|10308|94831|87041|2451|10308|32|15|3380|8|21.36|1.92|23.28|78.99|6.00|13.88|3.21|4.27|86.91| +2451306|76949|547|43056|876812|2891|26185|43056|876812|2891|26185|14|9|3381|10|1730.50|34.61|1765.11|28.15|273.20|1107.52|18.68|604.30|335.96| +2452560|16904|9267|67599|987992|4738|21531|67599|987992|4738|21531|42|15|3383|35|2661.40|26.61|2688.01|88.76|75.95|2102.50|262.68|296.22|191.32| +2452544|77789|8304|88605|670401|550|20346|88605|670401|550|20346|48|27|3383|13|148.59|2.97|151.56|16.16|535.21|108.47|39.31|0.81|554.34| +2451272|49188|11084|55356|649613|4670|34527|55356|649613|4670|34527|49|24|3384|16|1097.76|21.95|1119.71|7.46|886.56|823.32|120.75|153.69|915.97| +2452288|77160|11101|2878|1496527|4576|1425|2878|1496527|4576|1425|51|26|3385|34|486.20|19.44|505.64|82.37|1999.20|165.30|198.95|121.95|2101.01| +2452334|78748|14017|7640|411464|2041|23418|7640|411464|2041|23418|45|10|3385|30|1294.20|116.47|1410.67|64.21|558.90|905.94|174.71|213.55|739.58| +2452537|68189|5223|62080|707437|3994|40904|62080|707437|3994|40904|33|19|3386|14|345.80|17.29|363.09|51.98|230.44|190.19|140.04|15.57|299.71| +||11599||1753097||22819|80589|1753097|5158|22819||9|3386|||36.65|||824.04|1136.38|33.36|52.18|891.51| +2452682|45456|12417|29053|844919|785|4651|29053|844919|785|4651|42|9|3387|8|1028.00|71.96|1099.96|93.33|408.40|750.44|11.10|266.46|573.69| +2451337|25328|8998|70401|794880|3359|39838|70401|794880|3359|39838|28|11|3388|16|48.64|0.97|49.61|51.89|62.56|42.80|2.74|3.10|115.42| +2451215|69829|7490|12835|519300|1765|43077|12835|519300|1765|43077|25|34|3388|2|59.76|5.37|65.13|44.20|5.42|20.91|37.29|1.56|54.99| +2451916|72790|1853|79921|446601|3681|42761|79921|446601|3681|42761|44|23|3389|6|296.34|14.81|311.15|81.37|179.70|242.99|13.33|40.02|275.88| +2452778|45114|2862|50152|1919486|204|13199|50152|1919486|204|13199|36|10|3393|37|1109.26|66.55|1175.81|17.09|739.63|133.11|165.94|810.21|823.27| +2451411|68229|7640|45442|1603194|4146|28298|45442|1603194|4146|28298|37|25|3394|7|282.87|22.62|305.49|49.94|75.11|79.20|20.36|183.31|147.67| +2451655|47705|11188|33117|1075145|3573|801|33117|1075145|3573|801|26|17|3395|15|865.20|0.00|865.20|45.37|77.85|259.56|314.93|290.71|123.22| +2452798|74455|7350|49805|420803|4601|7999|49805|420803|4601|7999|7|22|3397|33|535.59|0.00|535.59|56.86|622.71|64.27|89.55|381.77|679.57| +|53836|7935|94051|238760|||94051|238760|269|22178|||3399|2|68.98||74.49||2.80||||| +2452926|34523|2640|76467|1794774|3928|33035|76467|1794774|3928|33035|30|23|3399|86|5747.38|229.89|5977.27|36.32|249.40|114.94|56.32|5576.12|515.61| +2451085|61781|3139|39163|830050|137|9024|39163|830050|137|9024|32|4|3400|1|129.40|2.58|131.98|73.84|90.37|19.41|65.99|44.00|166.79| +2451072|52958|5158|76253|395649|4645|16794|76253|395649|4645|16794|26|23|3400|13|184.60|1.84|186.44|9.82|33.54|140.29|28.80|15.51|45.20| +2451026|19350|17048|90018|423040|6462|33084|90018|423040|6462|33084|13|19|3400|14|1635.62|81.78|1717.40|38.27|338.38|1602.90|26.50|6.22|458.43| +2452651|31975|3829|60647|779341|4435|11176|60647|779341|4435|11176|30|34|3404|6|34.20|2.39|36.59|89.89|19.92|5.47|24.99|3.74|112.20| +2452723|65505|11797|58691|351502|6713|46771|58691|351502|6713|46771|13|19|3404|3|560.31|16.80|577.11|80.02|243.90|554.70|3.19|2.42|340.72| +2452682||8193|92066|1279638|6410|32782|92066||6410|32782||14|3404||||209.98||||137.88||103.75| +2451257|38475|12692|22236|1839692|4701|11310|22236|1839692|4701|11310|49|4|3406|16|452.80|27.16|479.96|17.18|145.44|18.11|152.14|282.55|189.78| +2451151|68804|15541|25314|1390963|3459|9108|25314|1390963|3459|9108|40|14|3406|71|6487.27|129.74|6617.01|70.77|3454.15|6227.77|197.22|62.28|3654.66| +2452413|34841|17289|67214|1617145|1923|40654|67214|1617145|1923|40654|41|31|3407|38|293.74|5.87|299.61|8.03|231.04|49.93|207.23|36.58|244.94| +2452660|64386|1056|65285|685576|4714|29805|65285|685576|4714|29805|24|28|3408|30|991.50|79.32|1070.82|65.58|495.60|644.47|38.17|308.86|640.50| +2452469|73275|14779|671|1469941|2411|34081|671|1469941|2411|34081|7|26|3409|5|14.85|1.33|16.18|89.79|3.10|14.85|0.00|0.00|94.22| +2450968|37492|7|87117|861930|5543|1156|87117|861930|5543|1156|52|32|3410|2|274.14|0.00|274.14|84.99|80.44|68.53|145.98|59.63|165.43| +2451053|39849|12896|65598|1768149|3853|46404|65598|1768149|3853|46404|31|27|3410|7|37.31|3.35|40.66|49.24|401.10|20.89|2.79|13.63|453.69| +2451201|66300|17944|11922|709822|2122|9269|11922|709822|2122|9269|32|18|3410|51|2821.32|0.00|2821.32|54.42|1057.74|2341.69|33.57|446.06|1112.16| +|61983|17443||1623533|1092||||1092||31||3412||194.04|11.64|205.68|20.51|404.28||3.96|7.69|436.43| +2451928||14179|69695||5219|47781|||||2||3413|85||116.32|2442.77|||1279.54|481.57|565.34|2986.76| +2451766|63983|44|50979|582807|6188|35169|50979|582807|6188|35169|28|7|3414|64|1962.88|137.40|2100.28|4.61|586.24|1354.38|474.63|133.87|728.25| +2451666|66230|7837|61466|63934|3555|10376|61466|63934|3555|10376|43|31|3415|12|1038.96|41.55|1080.51|53.56|400.56|155.84|724.15|158.97|495.67| +2451577|21057|15241|74246|92849|4252|49655|74246|92849|4252|49655|43|10|3415|8|185.76|0.00|185.76|9.89|387.04|16.71|5.07|163.98|396.93| +2451678|76481|17984|13120|1898161|1249|7890|13120|1898161|1249|7890|22|32|3415|21|2428.44|24.28|2452.72|85.04|1309.98|1699.90|327.84|400.70|1419.30| +2451686|59566|15490|3592|1396581|3970|22159|3592|1396581|3970|22159|26|5|3416|7|638.96|51.11|690.07|24.30|15.96|364.20|274.76|0.00|91.37| +2451662|84435|16802|10336|1280031|4926|43021|10336|1280031|4926|43021|49|34|3416|24|3587.28|71.74|3659.02|50.04|797.04|2798.07|662.93|126.28|918.82| +2452067|36088|3806|27948|1226661|6621|3734|27948|1226661|6621|3734|25|17|3417|33|2559.15|0.00|2559.15|70.02|2066.79|2456.78|9.21|93.16|2136.81| +2452070|66309|9491|32059|1895551|196|24088|32059|1895551|196|24088|49|18|3417|40|1904.80|0.00|1904.80|53.50|2704.00|571.44|1160.02|173.34|2757.50| +2452497|52691|2821|3651|222982|2313|28444|3651|222982|2313|28444|39|2|3419|88|3089.68|216.27|3305.95|40.92|2738.56|1637.53|580.86|871.29|2995.75| +2452510|22999|17507|48895|464105|6868|49278|48895|464105|6868|49278|35|22|3419|72|76.32|3.81|80.13|52.89|2307.60|41.97|32.63|1.72|2364.30| +2452380|69089|5151|76149|787269|4436|17956|76149|787269|4436|17956|45|4|3419|18|1645.38|82.26|1727.64|26.89|571.32|131.63|393.57|1120.18|680.47| +2452439|73037|14781|51746|413446|746|11646|51746|413446|746|11646|53|29|3419|58|1408.82|42.26|1451.08|61.89|110.78|1000.26|81.71|326.85|214.93| +2452401|41883|8881|32736|170200|6417|3081|32736|170200|6417|3081|5|23|3419|43|2164.62|43.29|2207.91|78.33|1498.55|1407.00|659.12|98.50|1620.17| +2451271|56797|2158|67171|86617|16|48262|67171|86617|16|48262|58|26|3420|54|8305.20|332.20|8637.40|57.94|276.48|7806.88|353.80|144.52|666.62| +|36616|10279|4280|53463|||4280||1046|||17|3420||309.42||334.17|65.25|147.99||||237.99| +2451394|28935|9712|14271|1617038|4605|12972|14271|1617038|4605|12972|1|12|3420|8|225.52|15.78|241.30|23.55|191.20|33.82|103.51|88.19|230.53| +2451932|34060|8669|87617|651843|5222|31001|87617|651843|5222|31001|56|19|3422|2|101.04|0.00|101.04|3.55|16.84|54.56|22.77|23.71|20.39| +2452485|53097|14001|39946|1578974|6802|13030|39946|1578974|6802|13030|23|18|3424|8|999.68|29.99|1029.67|62.11|279.44|779.75|17.59|202.34|371.54| +2451825|39748|17984|64196|980596|6149|28502|64196|980596|6149|28502|17|27|3425|81|1619.19|0.00|1619.19|21.68|1121.04|1068.66|412.89|137.64|1142.72| +2452399|4537|17997|64880|1147114|901|10335|64880|1147114|901|10335|11|14|3427|52|1691.56|67.66|1759.22|95.01|805.48|422.89|38.06|1230.61|968.15| +2452166|58682|423|42547|1339818|3903|21332|42547|1339818|3903|21332|43|8|3427|42|689.22|55.13|744.35|63.31|645.96|482.45|47.55|159.22|764.40| +2451647|68196|1006|16931|948076|468|27613|16931|948076|468|27613|34|2|3428|19|338.39|30.45|368.84|21.97|435.29|87.98|70.11|180.30|487.71| +2451585|71989|16240|89199|1204255|111|18367|89199|1204255|111|18367|8|12|3428|6|617.46|49.39|666.85|29.77|229.86|364.30|32.91|220.25|309.02| +2452639||11898|56702|408821||39012||||39012|||3430||4548.96|136.46||15.48|1802.32|2683.88||1436.12|| +2451915|60558|15649|16408|1099872|4489|39153|16408|1099872|4489|39153|56|26|3432|1|4.93|0.34|5.27|67.24|3.09|0.93|1.56|2.44|70.67| +2451159|26964|15145|30281|587542|356|6370|30281|587542|356|6370|49|13|3434|2|117.70|2.35|120.05|70.39|20.84|5.88|92.81|19.01|93.58| +2451339|18384|14594|10648|435687|4279|33271|10648|435687|4279|33271|52|5|3435|36|2928.60|205.00|3133.60|95.00|2662.20|1112.86|472.09|1343.65|2962.20| +|49643|12781||1049578|6447|28869|71909|||28869|20||3436|1|||41.49||50.26|||36.20|57.49| +2452718|59745|16092|71601|419386|418|13003|71601|419386|418|13003|48|5|3437|7|477.40|23.87|501.27|0.65|278.46|224.37|164.46|88.57|302.98| +2451320|5058|10627|38618|1573906|3241|38929|38618|1573906|3241|38929|37|33|3438|52|2529.28|126.46|2655.74|16.87|889.72|2478.69|15.68|34.91|1033.05| +2452849|65267|11868|98232|214086|7093|39747|98232|214086|7093|39747|51|35|3440|3|203.67|14.25|217.92|10.49|179.43|52.95|129.61|21.11|204.17| +2452277|67946|9137|45367|46105|2843|23581|45367|46105|2843|23581|53|6|3443|52|3593.72|179.68|3773.40|92.89|3187.08|1221.86|1850.05|521.81|3459.65| +2452840|43553|17778|4727|1666169|2011|17453|4727|1666169|2011|17453|13|20|3445|47|791.48|0.00|791.48|79.31|98.70|71.23|453.75|266.50|178.01| +2452654|64104|2863|7826|212531|2836|5941|7826|212531|2836|5941|42|9|3445|16|380.00|19.00|399.00|0.92|220.96|262.20|10.60|107.20|240.88| +2451846|37983|1043|4992|1411601|1598|6776|4992|1411601|1598|6776|26|14|3446|42|2615.34|52.30|2667.64|24.87|151.62|2092.27|251.07|272.00|228.79| +2452018|1912|6111|20990|1165000|5010|15773|20990|1165000|5010|15773|49|2|3447|1|7.67|0.07|7.74|37.78|53.69|5.98|1.01|0.68|91.54| +2452218|69118|8271|21890|1775892|2694|25496|21890|1775892|2694|25496|41|20|3447|9|188.10|1.88|189.98|13.64|153.90|112.86|15.80|59.44|169.42| +2451984|39221|16561|62012|1429892|972|23252|62012|1429892|972|23252|14|35|3448|19|579.12|46.32|625.44|60.90|178.22|17.37|162.90|398.85|285.44| +2452014|69925|7585|93311|200117|1770|19776|93311|200117|1770|19776|8|10|3448|1|11.28|0.78|12.06|15.50|48.52|4.51|3.38|3.39|64.80| +2452092|85969|5093|73540|412250|3313|21623|73540|412250|3313|21623|17|12|3448|19|48.45|0.00|48.45|86.34|64.22|40.21|6.42|1.82|150.56| +2451746|48036|1679|19260|1463839|2818|14597|19260|1463839|2818|14597|35|9|3451|11|104.28|9.38|113.66|74.22|78.21|14.59|54.71|34.98|161.81| +2451925|62096|8263|600|135367|5334|20501|600|135367|5334|20501|55|2|3451|12|519.96|5.19|525.15|98.43|121.68|171.58|101.03|247.35|225.30| +2451913|64135|14515|81318|922253|5588|12523|81318|922253|5588|12523|53|21|3451|33|22.77|0.91|23.68|54.96|85.14|1.13|1.73|19.91|141.01| +2451926|37257|16123|70219|290072|4325|28578|70219|290072|4325|28578|56|18|3451|7|308.77|9.26|318.03|94.91|218.68|222.31|70.03|16.43|322.85| +2451926|71629|14207|65750|1420431|7112|30593|65750|1420431|7112|30593|11|4|3451|45|402.30|24.13|426.43|73.78|448.20|172.98|142.17|87.15|546.11| +2452703|10857|2640|39569|1024394|2819|41597|39569|1024394|2819|41597|25|13|3452|17|2369.80|94.79|2464.59|64.66|1030.37|1303.39|970.43|95.98|1189.82| +2452885|21572|7363|19302|511877|5252|1884|19302|511877|5252|1884|60|12|3452|75|1168.50|46.74|1215.24|97.51|2702.25|1028.28|50.47|89.75|2846.50| +2452596|61301|327|58037|1605473|3309|10392|58037|1605473|3309|10392|48|2|3452|9|734.04|29.36|763.40|63.13|154.53|88.08|284.22|361.74|247.02| +2452514|59960|16291|55565|1248916|247|47314|55565|1248916|247|47314|7|28|3453|30|3243.60|32.43|3276.03|2.18|2780.40|875.77|923.45|1444.38|2815.01| +2452440|5090|15193|77622|1015050|4831|33859|77622|1015050|4831|33859|13|6|3454|76|4902.76|147.08|5049.84|56.10|791.92|3480.95|767.77|654.04|995.10| +2452334|37499|13793|57771|1088171|5771|2124|57771|1088171|5771|2124|59|35|3456|9|363.60|14.54|378.14|10.36|306.18|3.63|226.78|133.19|331.08| +2452290|78268|9885|50944|594581|618|19102|50944|594581|618|19102|17|11|3456|36|5277.96|211.11|5489.07|80.34|1944.36|2586.20|2261.07|430.69|2235.81| +2451222|30891|14569|26366|52629|1292|3955|26366|52629|1292|3955|58|11|3457|22|152.02|6.08|158.10|47.95|89.54|27.36|23.68|100.98|143.57| +2452203|50420|12215|18912|1910245|3178|8911|18912|1910245|3178|8911|35|34|3460|19|1021.25|91.91|1113.16|63.73|609.71|980.40|29.82|11.03|765.35| +2452172|72098|9945|15179|1575860|2494|24375|15179|1575860|2494|24375|7|23|3460|1|20.61|1.85|22.46|90.96|2.74|13.19|4.08|3.34|95.55| +2451736|63035|14398|92384|1875844|4904|25517|92384|1875844|4904|25517|26|2|3462|5|345.25|27.62|372.87|36.97|79.10|120.83|33.66|190.76|143.69| +2451400|82486|6025|260|1157446|232|31465|260|1157446|232|31465|43|33|3463|42|865.62|8.65|874.27|48.89|53.76|268.34|412.12|185.16|111.30| +2451116|10899|9812|46034|1350259|1825|18835|46034|1350259|1825|18835|34|35|3464|6|38.52|0.38|38.90|16.65|161.82|35.82|0.13|2.57|178.85| +||14113|93048||2192|25611||1526420|||||3466|59||46.91|||1737.55||||1884.31| +2451637|70746|2521|96782|1595767|423|48101|96782|1595767|423|48101|49|16|3467|47|626.98|31.34|658.32|63.16|344.04|194.36|103.82|328.80|438.54| +2451662|65744|194|22851|1249565|1079|15346|22851|1249565|1079|15346|32|25|3468|47|3203.99|128.15|3332.14|58.46|4165.61|224.27|1430.26|1549.46|4352.22| +2452442|72322|17569|64383|712795|3607|46772|64383|712795|3607|46772|37|20|3469|40|7532.40|301.29|7833.69|68.34|2958.80|4971.38|230.49|2330.53|3328.43| +2451696|46780|422|31353|1107536|4166|37549|31353|1107536|4166|37549|58|8|3471|13|1633.71|130.69|1764.40|49.08|219.44|865.86|637.31|130.54|399.21| +2452104|42010|16825|85712|335752|3392|48232|85712|335752|3392|48232|7|30|3472|2|1.90|0.05|1.95|5.03|0.34|1.84|0.04|0.02|5.42| +||14593|78246|655633|2173|31913|78246|655633|2173|||24|3472|29|2924.36|||80.02|||173.12||808.46| +2451280|71780|10651|51916|1347938|2807|35629|51916|1347938|2807|35629|46|15|3473|18|517.32|15.51|532.83|43.04|396.54|72.42|360.36|84.54|455.09| +2452616|21919|289|5564|922988|6812|25036|5564|922988|6812|25036|9|15|3474|5|1094.70|87.57|1182.27|28.59|413.85|459.77|88.89|546.04|530.01| +2452782|69886|17403|97963|761221|4479|33784|97963|761221|4479|33784|37|24|3476|40|1921.60|153.72|2075.32|58.49|960.80|595.69|609.91|716.00|1173.01| +2452704|33862|3487|12130|447257|4784|21111|12130|447257|4784|21111|51|23|3477|21|66.78|5.34|72.12|48.51|13.44|61.43|0.42|4.93|67.29| +2452687|36739|7266|23587|247131|1875|15819|23587|247131|1875|15819|18|32|3481|8|208.80|0.00|208.80|80.55|175.84|12.52|149.17|47.11|256.39| +2452816|45573|12153|37119|476758|6797|4836|37119|476758|6797|4836|18|27|3482|12|64.68|1.29|65.97|74.12|64.68|43.33|20.92|0.43|140.09| +2452700|21389|6123|74967||6308|12602||||12602|54||3482|||73.61|||1076.32|||336.01|1168.60| +2452694||14517|61541|||15335|61541||4337|||15|3482|15|||||566.55|||562.58|687.18| +2451519|16360|8204|26444|145639|4782|17056|26444|145639|4782|17056|37|5|3483|33|2274.03|22.74|2296.77|97.02|2175.03|2046.62|161.46|65.95|2294.79| +2451941|46267|15287|75909|1513406|3256|9153|75909|1513406|3256|9153|59|11|3485|6|774.72|61.97|836.69|18.49|406.74|588.78|154.33|31.61|487.20| +2451860|43394|16981|90664|293860|703|21146|90664|293860|703|21146|44|35|3486|3|151.44|6.05|157.49|36.14|24.54|22.71|28.32|100.41|66.73| +2450990|79775|6838|40397|288079|3761|2643|40397|288079|3761|2643|46|15|3488|35|3976.35|79.52|4055.87|51.92|2364.25|2664.15|1246.59|65.61|2495.69| +2451846|61626|17858|70127|834142|6157|42925|70127|834142|6157|42925|19|21|3489|2|109.68|5.48|115.16|42.19|11.54|66.90|35.50|7.28|59.21| +2451894|22534|15173|69762|1036849|5536|28270|69762|1036849|5536|28270|43|4|3489|48|1252.80|87.69|1340.49|86.03|2341.92|1140.04|34.95|77.81|2515.64| +2451156|67920|1786|27219|956381|6343|20305|27219|956381|6343|20305|50|29|3492|66|7124.70|0.00|7124.70|95.43|4294.62|6554.72|541.48|28.50|4390.05| +||16393||||||1541222|5768|||1|3492|11|1350.03|121.50|1471.53|81.71||1215.02|78.30||234.23| +2452231|8557|7483|5050|1169937|6024|32511|5050|1169937|6024|32511|39|30|3493|15|396.45|23.78|420.23|35.76|274.50|372.66|12.13|11.66|334.04| +2451989|6424|8027|68478|868938|4151|21011|68478|868938|4151|21011|50|7|3494|77|1235.08|12.35|1247.43|16.93|178.64|494.03|429.80|311.25|207.92| +2450873|29403|14974|84136|1908181|5026|18671|84136|1908181|5026|18671|13|23|3496|9|450.45|9.00|459.45|51.02|74.25|4.50|321.08|124.87|134.27| +2451410|67887|4276|46635|910170|6858|5193|46635|910170|6858|5193|1|32|3497|1|0.69|0.05|0.74|49.63|0.59|0.05|0.34|0.30|50.27| +2452905|66002|13500|45724|217292|6813|24381|45724|217292|6813|24381|12|15|3498|13|784.16|39.20|823.36|39.88|139.36|117.62|459.91|206.63|218.44| +2452757|51067|2035|98094|203299|2551|13403|98094|203299|2551|13403|39|26|3498|1|139.93|1.39|141.32|42.91|69.96|100.74|39.19|0.00|114.26| +2452372|67064|17025|9503|296030|1310||9503|296030|||||3499||3292.32||||1923.84|954.77|1028.52||2198.56| +2452825|68554|12726|42682|1758569|6009|37896|42682|1758569|6009|37896|7|12|3500|8|61.36|1.84|63.20|81.80|257.68|5.52|40.20|15.64|341.32| +2452735|80920|6631|92618|1507245|1872|5299|92618|1507245|1872|5299|1|33|3500|20|2606.80|104.27|2711.07|70.60|912.40|1355.53|525.53|725.74|1087.27| +2452695||8187||345002|6543|43331||345002|||||3500|8|216.88|||23.32|230.00|2.16||75.16|272.83| +2452443|38771|3421|16293|1418189|4532|31275|16293|1418189|4532|31275|19|29|3501|38|108.30|2.16|110.46|71.57|96.52|34.65|61.86|11.79|170.25| +2452471|57761|4740|29297|1259306|4962|14171|29297|1259306|4962|14171|39|24|3503|2|200.08|12.00|212.08|13.44|102.12|188.07|6.84|5.17|127.56| +2452478||13305|57190|||||1531501|2259|32170||5|3503|24|||176.70||62.40|||36.22|84.82| +2452510|48228|17190|4635|1093941|222|35050|4635|1093941|222|35050|13|12|3503|12|2102.88|105.14|2208.02|61.78|100.08|841.15|643.48|618.25|267.00| +2452433|69087|7093|80404|330903|602|3901|80404|330903|602|3901|42|17|3504|5|59.25|4.14|63.39|8.32|23.95|0.59|13.49|45.17|36.41| +2451655|65358|1766|23721|1877768|4615|29062|23721|1877768|4615|29062|8|13|3505|65|4968.60|248.43|5217.03|14.89|1242.15|4322.68|148.56|497.36|1505.47| +2451467|27861|17032|66691|439274|5810|43546|66691|439274|5810|43546|55|4|3506|1|7.75|0.69|8.44|33.30|40.68|4.65|0.96|2.14|74.67| +2451100|79322|7738|2272|1827188|2857|1773|2272|1827188|2857|1773|50|5|3507|6|222.72|8.90|231.62|86.88|169.68|64.58|112.27|45.87|265.46| +2452286|71189|1261|71940|90060|2227|394|71940|90060|2227|394|45|19|3508|84|8626.80|517.60|9144.40|42.19|1257.48|3623.25|1501.06|3502.49|1817.27| +2452328|67245|8971|54203|1002391|4179|10720|54203|1002391|4179|10720|9|31|3510|35|1737.40|156.36|1893.76|67.25|1697.85|469.09|469.27|799.04|1921.46| +2452280|80325|15753|21|545236|6451|3266|21|545236|6451|3266|37|26|3510|1|34.07|1.02|35.09|68.92|14.80|3.40|1.84|28.83|84.74| +2452334|29069|1777|58407|1175127|2274|23741|58407|1175127|2274|23741|45|20|3510|10|777.70|0.00|777.70|48.04|416.60|147.76|302.37|327.57|464.64| +2452337|66288|65|42212|32712|5965|40065|42212|32712|5965|40065|31|29|3510|4|58.76|1.17|59.93|22.50|22.84|44.65|5.22|8.89|46.51| +2451073|60319|14737|38658|1274168|5678|15541|38658|1274168|5678|15541|43|24|3511|47|101.05|5.05|106.10|6.93|1011.44|60.63|8.08|32.34|1023.42| +2451020|45259|2584|13584|570269|6954|6086|13584|570269|6954|6086|49|31|3511|16|814.88|32.59|847.47|59.56|657.60|162.97|508.48|143.43|749.75| +2451960|59688|5108|2346|735311|5220|13266|2346|735311|5220|13266|56|33|3512|15|3373.05|134.92|3507.97|58.33|1484.10|2563.51|129.52|680.02|1677.35| +2451872|78231|16189|36257|1753582|2107|42985|36257|1753582|2107|42985|47|35|3513|1|6.47|0.06|6.53|64.65|0.32|3.36|1.92|1.19|65.03| +2451934|4692|6925|53423|875010|4244|25431|53423|875010|4244|25431|7|31|3515|22|355.30|3.55|358.85|36.48|1175.68|302.00|24.51|28.79|1215.71| +2451230|75703|280|53565|1761223|2481|29339|53565|1761223|2481|29339|19|3|3516|2|47.78|0.95|48.73|34.39|4.76|18.63|11.66|17.49|40.10| +2451730|20111|385|61708|383929|3397|2065|61708|383929|3397|2065|13|24|3517|55|6721.55|336.07|7057.62|34.88|1464.65|6654.33|42.34|24.88|1835.60| +2451555|32065|7166|19379|694657|3982|11307|19379|694657|3982|11307|55|27|3517|23|640.55|44.83|685.38|68.76|869.17|531.65|104.54|4.36|982.76| +2452216|54668|14851|88099|59391|6683|30775|88099|59391|6683|30775|59|5|3518|68|2430.32|97.21|2527.53|52.27|303.28|1555.40|621.19|253.73|452.76| +2452449|75924|6681|36092|159577|3415|15653|36092|159577|3415|15653|47|13|3518|4|101.68|5.08|106.76|31.93|3.32|65.07|8.42|28.19|40.33| +2452372|33856|3677|36492|1175457|22|49834|36492|1175457|22|49834|39|31|3520|23|1226.36|12.26|1238.62|40.21|158.24|1079.19|135.39|11.78|210.71| +2451518|15631|481|73310|51449|788|24480|73310|51449|788|24480|16|25|3521|30|1911.90|38.23|1950.13|72.28|0.00|994.18|605.69|312.03|110.51| +2452137|65450|14725|2772|1194369|2766|39563|2772|1194369|2766|39563|29|33|3523|41|75.44|3.01|78.45|13.10|79.13|46.77|26.66|2.01|95.24| +2451962|61393|14792|22451|1190698|6145|37592|22451|1190698|6145|37592|55|29|3525|71|3117.61|62.35|3179.96|95.33|1194.93|2681.14|126.57|309.90|1352.61| +2452324|71389|601|15165|1025757|1149|23388|15165|1025757|1149|23388|53|17|3526|9|428.67|30.00|458.67|42.82|220.05|21.43|187.33|219.91|292.87| +2452357|77889|14139|19225|1050837|6802|48814|19225|1050837|6802|48814|7|7|3526|47|2572.78|77.18|2649.96|81.99|2962.41|205.82|1065.13|1301.83|3121.58| +2452949|56716|8325|6998|983768|6138|37261|6998|983768|6138|37261|24|19|3528|1|2.88|0.00|2.88|93.93|12.35|1.92|0.54|0.42|106.28| +2452077|29127|5918|25577|145934|2291|45048|25577|145934|2291|45048|38|1|3529|61|8052.00|241.56|8293.56|55.46|4616.48|4348.08|703.74|3000.18|4913.50| +2452565|78862|13140|28152|1064519|1686|28100|28152|1064519|1686|28100|49|1|3530|1|0.24|0.01|0.25|13.25|0.74|0.00|0.01|0.23|14.00| +||975|64416|852053|6090||64416||6090|31727|36|15|3530|6||0.35|||237.48||0.61||| +2451181|51630|104|62550|230819|5041|43710|62550|230819|5041|43710|25|30|3534|1|1.01|0.08|1.09|86.53|0.49|0.67|0.20|0.14|87.10| +2450991|27904|14912|83466|33857|1048|26639|83466|33857|1048|26639|34|20|3534|1|27.73|2.21|29.94|3.95|31.51|15.80|3.22|8.71|37.67| +2451270|46429|12607|28723|219638|2553|1915|28723|219638|2553|1915|8|9|3536|25|2086.00|20.86|2106.86|62.71|834.25|813.54|623.50|648.96|917.82| +2451114||5344|28784|||49815||1137410|7189|49815|14|15|3536||450.73|||3.30|1288.27|||81.82|| +2451355|67730|11848|58830|1330560|5108|15658|58830|1330560|5108|15658|1|24|3536|65|92.95|4.64|97.59|45.69|467.35|91.09|0.66|1.20|517.68| +2452418|41989|12031|40867|37119|3324|29976|40867|37119|3324|29976|29|10|3539|48|1335.84|40.07|1375.91|87.90|419.04|173.65|1115.70|46.49|547.01| +2452246|63947|14635|96670|1553727|1184|641|96670|1553727|1184|641|57|33|3541|3|114.63|10.31|124.94|59.69|8.58|66.48|39.00|9.15|78.58| +2451523|20768|14704|36174|855739|4104|25669|36174|855739|4104|25669|20|21|3542|60|4936.80|296.20|5233.00|40.02|0.00|3307.65|325.83|1303.32|336.22| +2452522|19584|4847|96325|776389|1253|14517|96325|776389|1253|14517|25|10|3543|36|4396.32|0.00|4396.32|7.95|824.04|2989.49|984.78|422.05|831.99| +2452305|12853|7899|57052|974954|2456|17511|57052|974954|2456|17511|3|21|3543|21|120.75|4.83|125.58|29.07|2014.11|114.71|4.46|1.58|2048.01| +2452264|26499|13839|26469|157397|5925|30853|26469|157397|5925|30853|39|27|3543|66|1689.60|0.00|1689.60|20.16|2112.00|135.16|792.76|761.68|2132.16| +2451186|10732|506|32044|1629910|4576|18795|32044|1629910|4576|18795|4|22|3545|2|44.30|0.44|44.74|26.29|29.52|27.90|1.14|15.26|56.25| +2451168|34671|8294|19461|1113125|742|15159|19461|1113125|742|15159|58|10|3545|37|5002.77|50.02|5052.79|68.94|1756.39|2051.13|2685.99|265.65|1875.35| +2451124|70607|1472|11448|849899|2397|14448|11448|849899|2397|14448|4|23|3545|3|183.06|12.81|195.87|2.24|120.21|124.48|41.00|17.58|135.26| +2451151|61258|12217|2565|857026|5087|37195|2565|857026|5087|37195|58|3|3545|28|1008.00|60.48|1068.48|42.58|1181.60|887.04|65.31|55.65|1284.66| +2452656|41274|2599|66852|1485662|4472|3682|66852|1485662|4472|3682|51|22|3546|13|652.99|32.64|685.63|79.75|105.30|561.57|39.31|52.11|217.69| +2452749|43309|1215|32698|1595524|5903|4729|32698|1595524|5903|4729|60|22|3546|14|144.62|2.89|147.51|37.40|5.46|34.70|43.96|65.96|45.75| +2452498|1911|11183|42147|876995|6243|38691|42147|876995|6243|38691|49|12|3547|40|3266.00|130.64|3396.64|91.41|2110.40|2580.14|75.44|610.42|2332.45| +2452430|8011|7993|86280|1012913|6746|40932|86280|1012913|6746|40932|47|33|3548|3|24.39|2.19|26.58|64.32|0.00|1.70|14.97|7.72|66.51| +2452243|40837|821|49688|364419|397|33928|49688|364419|397|33928|33|9|3548|46|6278.54|502.28|6780.82|30.32|907.58|3641.55|2531.51|105.48|1440.18| +2452314|75259|1901|37506|72155|3957|1006|37506|72155|3957|1006|35|2|3550|1|9.18|0.09|9.27|1.26|19.69|5.96|2.44|0.78|21.04| +2451674|67620|11785|46143|398438|1580|47075|46143|398438|1580|47075|8|31|3551|10|67.70|1.35|69.05|93.47|321.70|19.63|9.13|38.94|416.52| +2451773||1756|31375|925314|6006||31375|925314||5498||4|3551||1499.67|89.98||||0.00|1229.72||| +2451543|62822|12842|19272|902180|2241|37949|19272|902180|2241|37949|44|8|3551|4|245.84|7.37|253.21|9.84|48.44|17.20|192.05|36.59|65.65| +2451285|75474|16954|34899|1810880|6697|30929|34899|1810880|6697|30929|28|32|3552|7|215.32|15.07|230.39|6.84|80.71|27.99|179.83|7.50|102.62| +2451259|74134|4480|73443|159737|878|37219|73443|159737|878|37219|44|14|3552|5|1.25|0.11|1.36|46.67|2.85|0.47|0.37|0.41|49.63| +2451258|51824|9094|34899|1810880|6697|30929|34899|1810880|6697|30929|14|9|3552|5|14.15|0.84|14.99|65.22|353.75|8.63|0.93|4.59|419.81| +2452769|41313|16323|2003|888664|1725|23986|2003|888664|1725|23986|37|4|3553|38|6038.20|0.00|6038.20|42.16|1940.66|3743.68|160.61|2133.91|1982.82| +2452681|40654|2364|285|1870330|5975|35620|285|1870330|5975|35620|15|35|3554|11|1333.97|13.33|1347.30|85.80|1215.39|787.04|541.46|5.47|1314.52| +2451232|67956|9343|31390|770835|1480|13678|31390|770835|1480|13678|37|14|3555|1|3.14|0.25|3.39|12.99|48.82|1.60|1.54|0.00|62.06| +2451308|70477|3302|27787|1473738|5736|44487|27787|1473738|5736|44487|8|9|3555|61|0.00|0.00|0.00|70.92|2172.21|0.00|0.00|0.00|2243.13| +2451391|14669|2923|21925|1568683|1041|49799|21925|1568683|1041|49799|16|28|3555|85|7691.65|461.49|8153.14|21.00|3978.00|6691.73|819.93|179.99|4460.49| +2451531|21737|11392|98260|348777|5694|26564|98260|348777|5694|26564|50|19|3555|6|354.00|17.70|371.70|70.81|3.78|77.88|173.95|102.17|92.29| +2452470|63070|10225|5427|169146|6818|16537|5427|169146|6818|16537|1|7|3556|13|2786.29|55.72|2842.01|85.90|682.37|278.62|601.84|1905.83|823.99| +2451707|51103|1588|48463|1668370|6677|31784|48463|1668370|6677|31784|38|8|3557|21|268.59|5.37|273.96|25.46|0.00|260.53|3.62|4.44|30.83| +2451699|38992|6470|18681|3164|3647|26930|18681|3164|3647|26930|50|25|3557|68|4133.04|0.00|4133.04|80.54|896.24|785.27|2778.64|569.13|976.78| +||10509|31520|||4488||||4488|43|30|3560|||103.01|1819.96|93.21||||201.92|| +2452487|28551|16995|91068|338794|2998|20567|91068|338794|2998|20567|7|27|3561|20|1969.60|19.69|1989.29|27.18|544.80|1181.76|354.52|433.32|591.67| +2452346|79061|6349|26801|728794|1540|8333|26801|728794|1540|8333|17|32|3562|10|71.00|2.13|73.13|71.68|92.30|46.15|16.64|8.21|166.11| +2452331|70579|10885|15612|789054|4449|30244|15612|789054|4449|30244|47|23|3562|5|8.85|0.70|9.55|34.79|222.45|1.94|1.65|5.26|257.94| +2452772|70747|3523|56752|685913|2274|27435|56752|685913|2274|27435|24|13|3563|24|1353.12|121.78|1474.90|10.98|312.24|1326.05|17.05|10.02|445.00| +2452135|61451|17715|37236|1903176|2363|1103|37236|1903176|2363|1103|19|7|3564|38|1605.88|112.41|1718.29|56.45|900.60|1011.70|374.33|219.85|1069.46| +2452584|38659|1977|20125|1474624|3876|28375|20125|1474624|3876|28375|49|22|3565|1|56.07|5.04|61.11|96.63|11.87|28.59|6.04|21.44|113.54| +2452524|72566|11485|25354|1277311|5268|8825|25354|1277311|5268|8825|19|29|3565|14|943.60|9.43|953.03|87.11|637.98|717.13|31.70|194.77|734.52| +2452505|56989|309|71294|830139|3384|32229|71294|830139|3384|32229|51|24|3565|66|1370.16|95.91|1466.07|43.21|777.48|0.00|232.92|1137.24|916.60| +2452629|18281|17167|51633|150731|4229|28927|51633|150731|4229|28927|9|33|3565|77|2068.99|82.75|2151.74|62.92|450.45|2068.99|0.00|0.00|596.12| +2452383|12235|8217|83779|1684476|3126|46576|83779|1684476|3126|46576|31|5|3566|51|945.03|47.25|992.28|23.89|1189.83|718.22|95.26|131.55|1260.97| +2452362|36259|17169|68752|1749185|3711|24914|68752|1749185|3711|24914|23|22|3566|23|1783.19|89.15|1872.34|12.32|1074.33|1587.03|1.96|194.20|1175.80| +2451128|66005|14293|57488|1322736|2384|2032|57488|1322736|2384|2032|14|25|3567|22|1814.34|163.29|1977.63|60.34|2010.36|72.57|905.72|836.05|2233.99| +2451195|6650|15409|51600|1863804|4008|13059|51600|1863804|4008|13059|34|25|3567|60|1889.40|18.89|1908.29|52.60|2073.60|377.88|1103.40|408.12|2145.09| +2451307|43973|12670|94197|1339557|117|26677|94197|1339557|117|26677|52|2|3567|12|250.80|15.04|265.84|16.47|524.40|150.48|32.10|68.22|555.91| +2451290|44788|7387|96722|238075|4911|38928|96722|238075|4911|38928|16|4|3567|15|727.95|14.55|742.50|60.23|786.15|698.83|3.20|25.92|860.93| +2451770|68992|2549|90905|5387|669|29551|90905|5387|669|29551|1|13|3569|1|3.78|0.30|4.08|44.97|1.72|1.70|0.58|1.50|46.99| +2451472|72453|11234|44780|14360|2340|38320|44780|14360|2340|38320|58|22|3571|12|664.44|13.28|677.72|6.00|1011.00|591.35|32.15|40.94|1030.28| +2452268|42520|581|96360|739507|6561|20811|96360|739507|6561|20811|41|23|3572|71|5748.87|172.46|5921.33|52.03|4166.28|4944.02|539.24|265.61|4390.77| +2451560|16845|3073|81151|1845797|1467|7563|81151|1845797|1467|7563|49|7|3575|46|8197.20|491.83|8689.03|42.20|4508.46|81.97|5437.20|2678.03|5042.49| +2451345|4486|9271|75277|48493|2862|5631|75277|48493|2862|5631|52|34|3576|2|280.74|22.45|303.19|76.64|64.78|115.10|16.56|149.08|163.87| +2451007|12838|6088|54527|1241946|5365|20384|54527|1241946|5365|20384|22|22|3577|53|67.84|4.07|71.91|50.85|2717.84|1.35|1.99|64.50|2772.76| +2451973|67399|9149|76400|627008|2863|16271|76400|627008|2863|16271|7|23|3578|15|53.25|1.06|54.31|3.39|23.25|50.58|0.10|2.57|27.70| +2452633|71287|15378|87795|22726|6810|2541|87795|22726|6810|2541|43|34|3580|16|77.28|6.18|83.46|63.34|425.12|21.63|6.67|48.98|494.64| +2452635|3421|12969|13040|1023184|81|44978|13040|1023184|81|44978|33|29|3580|62|2517.20|226.54|2743.74|13.59|562.96|1611.00|389.66|516.54|803.09| +2451305|45936|10795|18527|436324|2121|43918|18527|436324|2121|43918|20|5|3581|2|7.62|0.60|8.22|50.29|47.10|3.04|3.70|0.88|97.99| +2451269|49464|14854|50325|1882335|3945|18755|50325|1882335|3945|18755|28|23|3583|1|42.74|1.28|44.02|73.36|23.01|42.31|0.27|0.16|97.65| +2451322|49769|6697|||||29749|||10750||34|3583|||208.04|2808.56|||1638.32||0.00|1834.09| +2451435|45793|3068|81618|1066824|30|21283|81618|1066824|30|21283|13|21|3583|56|3841.04|268.87|4109.91|81.13|509.04|614.56|322.64|2903.84|859.04| +2451918|65087|15271|90615|1234272|7176|15233|90615|1234272|7176|15233|31|21|3584|40|670.40|20.11|690.51|73.83|446.80|549.72|19.30|101.38|540.74| +2451109|68146|7334|74190|24752|6161|19923|74190|24752|6161|19923|10|29|3585|21|271.95|24.47|296.42|41.90|243.81|250.19|18.71|3.05|310.18| +2450975|68692|17348|99323|315740|2910|16127|99323|315740|2910|16127|19|28|3585|3|63.63|5.09|68.72|93.65|29.94|62.35|1.25|0.03|128.68| +2451395|38885|13834|47763|1439715|1620|1972|47763|1439715|1620|1972|2|31|3586|11|118.03|7.08|125.11|94.79|18.26|97.96|15.45|4.62|120.13| +2452013|32572|10871|86342|472594|1246|1898|86342|472594|1246|1898|43|28|3589|6|302.22|9.06|311.28|52.22|327.42|27.19|123.76|151.27|388.70| +||16904||1222833|||49610|1222833|||20|28|3589||125.46|11.29|||50.66||12.40||| +2450991|74643|11848|76520|1621253|7139|7425|76520|1621253|7139|7425|38|25|3590|16|52.32|1.04|53.36|62.72|283.04|8.89|18.67|24.76|346.80| +||3044||||||267499|5146||19||3593|61||169.75|||113.46|565.83|435.69||296.52| +2452122|18706|6545|88227|1345068|1462|1903|88227|1345068|1462|1903|1|4|3595|45|1597.05|79.85|1676.90|18.52|1335.60|1165.84|275.97|155.24|1433.97| +2452087|74953|9518|34574|32362|6510|7449|34574|32362|6510|7449|44|15|3595|40|1020.80|40.83|1061.63|3.19|1758.00|867.68|68.90|84.22|1802.02| +2451377|7869|5770|89928|1444441|4907|8220|89928|1444441|4907|8220|20|7|3596|3|292.17|2.92|295.09|88.83|37.02|128.55|106.35|57.27|128.77| +2451286|65811|17692|40552|1447026|5500|39621|40552|1447026|5500|39621|34|14|3597|22|2227.06|178.16|2405.22|17.36|830.94|1224.88|751.63|250.55|1026.46| +2451211|53961|16420|14271|1330713|5202|8891|14271|1330713|5202|8891|16|26|3597|6|852.18|76.69|928.87|14.65|91.74|485.74|21.98|344.46|183.08| +2451807|18900|7949|15365|831569|2173|19577|15365|831569|2173|19577|49|6|3604|13|213.33|17.06|230.39|70.41|80.47|185.59|17.47|10.27|167.94| +2451770|68721|9776|49986|650756|4679|44249|49986|650756|4679|44249|2|18|3604|49|881.02|8.81|889.83|85.04|184.24|149.77|277.87|453.38|278.09| +2451725|70114|12401|97357|243490|5976|1709|97357|243490|5976|1709|50|19|3604|92|6121.68|183.65|6305.33|76.61|2225.48|2326.23|189.77|3605.68|2485.74| +2451796|56308|14639|68497|1770032|5167|31902|68497|1770032|5167|31902|41|16|3604|5|353.85|24.76|378.61|28.51|0.00|35.38|133.75|184.72|53.27| +||3589||1536611||39069|34782|1536611|840|39069|||3605|||14.61|176.96|||133.12||7.90|284.63| +2451730||11942||||22963|56121|1892786|4298|22963|31||3606||||572.01||34.60|5.60||111.04|129.05| +2452572|70210|4297|25560|533916|4319|10675|25560|533916|4319|10675|27|13|3609|22|2698.30|26.98|2725.28|49.82|1439.02|1538.03|754.17|406.10|1515.82| +2452649|43430|17703|45490|571622|1307|35039|45490|571622|1307|35039|12|24|3609|10|469.20|9.38|478.58|1.19|159.30|215.83|68.40|184.97|169.87| +2452334|48153|2423|2025|1385325|625|23125|2025|1385325|625|23125|53|35|3611|40|1733.20|69.32|1802.52|12.61|216.40|675.94|644.92|412.34|298.33| +2452350|55155|9731|35389|526625|2661|49897|35389|526625|2661|49897|25|10|3612|31|5074.08|152.22|5226.30|26.19|2537.04|4008.52|383.60|681.96|2715.45| +||5647|||||||||||3612|||1.18||25.24||||24.43|| +2452923|20716|10357|34126|401266|5559|39452|34126|401266|5559|39452|13|31|3614|1|26.22|1.83|28.05|72.80|4.16|1.04|19.13|6.05|78.79| +2452718|43081|6913|12420|824527|5434|29941|12420|824527|5434|29941|31|9|3614|6|211.14|12.66|223.80|28.82|16.20|200.58|0.31|10.25|57.68| +2452300|1503|8531|97096|1202326|5956|7186|97096|1202326|5956|7186|35|24|3615|1|18.88|1.51|20.39|75.76|13.19|10.38|5.86|2.64|90.46| +2452477|65976|11669|18844|1813027|3563|14946|18844|1813027|3563|14946|27|13|3615|28|1510.04|60.40|1570.44|69.57|598.08|875.82|285.39|348.83|728.05| +2452414|66980|3089|76698|1326860|1822|2784|76698|1326860|1822|2784|15|26|3615|17|1242.36|111.81|1354.17|24.95|94.01|583.90|408.24|250.22|230.77| +2452045|39493|17809|55116|1698372|5181|25990|55116|1698372|5181|25990|19|18|3616|31|3040.17|212.81|3252.98|44.57|1681.75|2553.74|413.46|72.97|1939.13| +2451344|46541|8338|48899|298009|272|1382|48899|298009|272|1382|1|13|3617|44|2945.36|176.72|3122.08|93.80|5184.08|382.89|2126.85|435.62|5454.60| +2451438|33042|1915|98467|451337|3407|30677|98467|451337|3407|30677|20|32|3617|1|14.28|1.14|15.42|22.38|6.05|10.42|3.78|0.08|29.57| +2452826|42405|13243|90700|1599657|2661|20865|90700|1599657|2661|20865|12|32|3618|7|457.38|0.00|457.38|97.71|55.16|402.49|34.58|20.31|152.87| +2452970|71973|16398|64777|1451294|3411|35528|64777|1451294|3411|35528|19|32|3618|18|2063.52|123.81|2187.33|36.09|1112.94|1877.80|26.00|159.72|1272.84| +2452789|43571|16417|57794|542355|3291|20156|57794|542355|3291|20156|1|3|3618|30|1158.90|69.53|1228.43|69.00|927.00|591.03|442.93|124.94|1065.53| +2452100|79712|15692|48542|1096597|4652|9954|48542|1096597|4652|9954|55|33|3619|14|114.38|6.86|121.24|80.66|44.52|102.94|8.92|2.52|132.04| +2452210|33382|10733|28776|1369756|1136|42116|28776|1369756|1136|42116|29|2|3619|1|133.89|4.01|137.90|47.82|5.88|25.43|30.36|78.10|57.71| +2452001|76700|1202|68850|391167|4684|21848|68850|391167|4684|21848|8|32|3621|57|6596.61|263.86|6860.47|60.87|3552.24|5079.38|91.03|1426.20|3876.97| +2451964|29869|4793|27609|1366537|5818|43146|27609|1366537|5818|43146|20|8|3621|43|1667.54|150.07|1817.61|37.59|3134.70|833.77|583.63|250.14|3322.36| +2452048|49511|16898|66707|1307309|1958|1407|66707|1307309|1958|1407|37|32|3621|10|245.20|9.80|255.00|10.46|245.20|19.61|49.62|175.97|265.46| +2451909|60266|14773|28897|1416082|1667|35699|28897|1416082|1667|35699|31|21|3622|7|50.82|4.06|54.88|4.65|66.92|24.39|11.62|14.81|75.63| +2451727|37091|9205|39832|508280|2681|44329|39832|508280|2681|44329|56|16|3622|46|2886.96|0.00|2886.96|24.00|3214.94|808.34|2037.04|41.58|3238.94| +2452061|35748|11603|1917|1152836|5355|43521|1917|1152836|5355|43521|29|34|3623|17|1217.37|109.56|1326.93|57.40|332.01|109.56|819.77|288.04|498.97| +2452090|41128|13946|42943|1218125|4860|25242|42943|1218125|4860|25242|47|13|3623|40|882.80|70.62|953.42|80.27|220.40|238.35|380.22|264.23|371.29| +2452740|82770|6967|57521|1593438|1802|9766|57521|1593438|1802|9766|9|23|3626|28|3853.36|0.00|3853.36|83.97|2969.12|2890.02|134.86|828.48|3053.09| +2451937|46719|4172|44523|107165|3407|27202|44523|107165|3407|27202|41|30|3627|60|943.80|56.62|1000.42|77.16|1161.60|358.64|70.21|514.95|1295.38| +2451977|13266|799|11604|1175528|4195|49536|11604|1175528|4195|49536|2|8|3628|2|420.30|4.20|424.50|12.30|182.12|4.20|83.22|332.88|198.62| +2451171|33701|5407|87197|794786|7175|33164|87197|794786|7175|33164|34|3|3629|15|1737.60|34.75|1772.35|92.36|113.25|399.64|949.95|388.01|240.36| +2451239|47252|3673|75108|936999|490|7181|75108|936999|490|7181|52|20|3629|59|6983.83|279.35|7263.18|71.70|4910.57|2653.85|2554.68|1775.30|5261.62| +2451303|84515|14282|70342|1817579|369|24759|70342|1817579|369|24759|13|13|3629|14|134.40|9.40|143.80|52.41|64.26|69.88|34.19|30.33|126.07| +2451358|69822|14107|70247|878918|4389|17907|70247|878918|4389|17907|34|7|3631|4|229.36|4.58|233.94|30.10|124.88|165.13|32.75|31.48|159.56| +2451309|45788|14516|58614|224764|5314|14264|58614|224764|5314|14264|37|11|3631|13|389.35|0.00|389.35|24.11|231.14|70.08|268.18|51.09|255.25| +2451600|65747|3043|45108|640663|1018|19020|45108|640663|1018|19020|49|9|3632|35|765.80|45.94|811.74|78.41|893.20|405.87|341.93|18.00|1017.55| +2451657|78721|3046|50185|1731062|2339|37216|50185|1731062|2339|37216|56|5|3632|3|253.47|7.60|261.07|22.07|5.55|63.36|125.47|64.64|35.22| +2451817|44021|11069|64381|1056911|2241|20818|64381|1056911|2241|20818|37|23|3633|5|72.95|5.83|78.78|10.93|141.05|48.87|10.35|13.73|157.81| +2451484|65298|12844|56289|463400|2511|41975|56289|463400|2511|41975|40|14|3635|51|1531.02|45.93|1576.95|55.44|334.05|903.30|514.73|112.99|435.42| +2451572|54943|13819|95584|1745718|1099|14080|95584|1745718|1099|14080|56|2|3636|27|948.24|18.96|967.20|54.53|289.71|597.39|192.96|157.89|363.20| +2452756|72850|9603|54114|904366|3065|917|54114|904366|3065|917|1|11|3637|29|374.97|7.49|382.46|65.59|117.45|277.47|82.87|14.63|190.53| +2452651|49389|13140|77254|1882254|3283|28807|77254|1882254|3283|28807|48|31|3637|6|99.18|2.97|102.15|38.40|11.64|44.63|50.18|4.37|53.01| +2451507|43161|3784|80730|1415121|265|34656|80730|1415121|265|34656|32|11|3641|3|208.02|12.48|220.50|79.92|88.59|124.81|9.98|73.23|180.99| +2452406|56474|995|85905|1706548|5144|22987|85905|1706548|5144|22987|11|1|3642|32|27.20|0.27|27.47|62.39|16.32|13.32|1.52|12.36|78.98| +2452291|18437|16931|19699|1388536|4573|20657|19699|1388536|4573|20657|33|33|3642|5|718.55|7.18|725.73|30.07|93.10|517.35|46.27|154.93|130.35| +2452332|40744|6793|89603|362792|4334|3123|89603|362792|4334|3123|47|3|3642|3|122.13|8.54|130.67|30.37|24.03|24.42|57.64|40.07|62.94| +2452686|63902|11349|45574|1151852|530|9208|45574|1151852|530|9208|6|15|3643|59|1026.60|30.79|1057.39|44.57|3508.73|728.88|253.06|44.66|3584.09| +2452756|2979|8863|77852|403329|6406|10283|77852|403329|6406|10283|54|5|3643|20|42.20|2.95|45.15|81.17|464.40|28.27|12.39|1.54|548.52| +2452024|68922|6572|33875|1464030|232|34|33875|1464030|232|34|13|12|3648|35|0.00|0.00|0.00|63.90|2032.80|0.00|0.00|0.00|2096.70| +2451981|1367|4001|33337|1902145|1706|914|33337|1902145|1706|914|26|4|3649|1|0.00|0.00|0.00|68.95|51.16|0.00|0.00|0.00|120.11| +|37542|16237|97731||1787|168|97731|673759|1787|||17|3649|||221.18|7593.86|13.31||2506.71|||1623.43| +2451609||16826|83350||||83350|1152032|||59||3650||||5211.86|||4076.60|||| +2451773|38602|11411|19172|153187|1955|47970|19172|153187|1955|47970|8|8|3650|2|21.32|1.49|22.81|81.88|42.64|5.75|8.40|7.17|126.01| +2451673|62716|17642|22119|1280100|1634|12815|22119|1280100|1634|12815|5|18|3650|10|449.50|31.46|480.96|54.96|0.00|80.91|342.78|25.81|86.42| +2451798|11464|7811|31704|679653|4402|1151|31704|679653|4402|1151|2|32|3650|52|1851.20|55.53|1906.73|38.20|944.32|481.31|972.62|397.27|1038.05| +2451324|83768|17203|33344|1702640|3583|24879|33344|1702640|3583|24879|40|26|3651|9|83.34|0.83|84.17|80.12|430.92|83.34|0.00|0.00|511.87| +2451257|42779|6433|30578|866611|6997|47783|30578|866611|6997|47783|8|3|3651|27|1062.99|63.77|1126.76|22.43|114.48|914.17|93.75|55.07|200.68| +2451254|65473|6014|35666|1257621|1623|23978|35666|1257621|1623|23978|2|12|3651|5|6.70|0.60|7.30|36.70|8.65|1.74|0.69|4.27|45.95| +2452558|13102|6175|84203|1575000|4997|30641|84203|1575000|4997|30641|35|2|3653|7|192.71|11.56|204.27|34.59|112.98|80.93|60.36|51.42|159.13| +2452433|68601|4815|84985|490070|3955|33840|84985|490070|3955|33840|5|22|3653|36|884.88|17.69|902.57|84.87|1548.36|389.34|203.17|292.37|1650.92| +2452540|31824|16116|52832|1074589|4346|14503|52832|1074589|4346|14503|6|35|3654|31|483.91|14.51|498.42|43.40|903.34|425.84|23.22|34.85|961.25| +2452438|47692|9897|74920|1296801|5082|28116|74920|1296801|5082|28116|3|15|3654|7|313.25|21.92|335.17|33.38|154.21|81.44|127.49|104.32|209.51| +2452536|73455|7087|35142|912369|443|1717|35142|912369|443|1717|48|17|3654|20|266.60|21.32|287.92|71.22|16.00|175.95|81.58|9.07|108.54| +2452416|54739|9265|63554|348784|5214|30044|63554|348784|5214|30044|49|19|3654|48|761.28|15.22|776.50|20.10|228.00|593.79|105.51|61.98|263.32| +2452143|60613|5227|78746|781823|1347|30071|78746|781823|1347|30071|37|26|3656|59|3056.79|183.40|3240.19|88.37|208.27|1008.74|962.58|1085.47|480.04| +2452038|69782|7975|62929|1682712|6495|26678|62929|1682712|6495|26678|8|24|3658|27|1314.36|78.86|1393.22|66.06|604.53|289.15|358.82|666.39|749.45| +2452072|58960|17285|44783|734055|1280|39705|44783|734055|1280|39705|25|21|3658|2|128.90|6.44|135.34|34.47|51.08|6.44|122.46|0.00|91.99| +2452112|64928|3567|64181|1725990|5752|43961|64181|1725990|5752|43961|49|9|3662|23|1457.74|102.04|1559.78|25.84|915.17|58.30|937.62|461.82|1043.05| +2452080|41837|2517|47009|1505872|6335|27598|47009|1505872|6335|27598|51|24|3662|27|1710.72|153.96|1864.68|88.37|1536.03|701.39|989.14|20.19|1778.36| +2452168|37856|11207|33132|1569099|3819|16186|33132|1569099|3819|16186|9|24|3663|44|635.36|19.06|654.42|27.75|1180.08|139.77|14.86|480.73|1226.89| +2451327|56998|6502|6219|421685|3904|10523|6219|421685|3904|10523|19|16|3664|7|823.69|65.89|889.58|78.24|351.54|247.10|443.97|132.62|495.67| +2451369|56368|14864|10169|815206|4690|12527|10169|815206|4690|12527|10|18|3665|73|8581.15|429.05|9010.20|24.97|3705.48|3175.02|432.49|4973.64|4159.50| +2451539|69434|7022|32057|1341329|1801|45026|32057|1341329|1801|45026|8|1|3667|5|485.05|19.40|504.45|79.71|187.20|247.37|230.54|7.14|286.31| +2451784|22634|16670|9060|1602971|3635|9946|9060|1602971|3635|9946|56|23|3667|24|99.12|6.93|106.05|89.65|297.84|37.66|33.80|27.66|394.42| +2451740|14587|3259|3825|558671|4717|31563|3825|558671|4717|31563|4|7|3669|8|47.04|2.35|49.39|97.80|18.00|5.17|23.86|18.01|118.15| +||6493||324827|2332|49484|20475||2332|49484|28|28|3670|52||173.65|||1488.24||||1705.32| +2452768|67761|7137|23943|378430|6870|24110|23943|378430|6870|24110|33|14|3671|44|2732.84|109.31|2842.15|94.28|311.96|2268.25|130.08|334.51|515.55| +2452731|68735|11502|34341|912206|4471|29183|34341|912206|4471|29183|49|7|3671|34|754.80|60.38|815.18|41.20|0.00|611.38|86.05|57.37|101.58| +2451188|69664|16340|68450|236042|4787|18332|68450|236042|4787|18332|49|21|3675|24|1374.72|96.23|1470.95|49.29|438.72|687.36|302.43|384.93|584.24| +2451363|77555|15844|84117|1577972|1427|30929|84117|1577972|1427|30929|50|28|3677|74|853.22|68.25|921.47|2.67|158.36|264.49|229.60|359.13|229.28| +|55109|12671|57788|992634|4280|41879|||4280||43||3678|||25.85|887.69|||284.40|||405.16| +2452166|26540|12788|45561|1849826|138|44003|45561|1849826|138|44003|32|25|3678|10|5.20|0.46|5.66|42.73|16.50|3.84|0.85|0.51|59.69| +|40238|15068|5617||6141||5617|1514679|6141||49|25|3679|||27.99||||436.64|||815.36| +2452445|70330|6865|60052|293276|3288|26125|60052|293276|3288|26125|57|20|3680|1|35.89|2.51|38.40|39.92|27.31|19.38|4.78|11.73|69.74| +2452422|40450|7363|97323|1836291|4059|3074|97323|1836291|4059|3074|29|4|3681|28|2527.28|227.45|2754.73|6.67|791.56|1617.45|191.06|718.77|1025.68| +2451147|65524|9074|96523|1576260|2844|45828|96523|1576260|2844|45828|52|31|3682|42|1322.16|79.32|1401.48|97.65|396.48|198.32|775.44|348.40|573.45| +2452443|10583|12887|75366|1900025|684|1696|75366|1900025|684|1696|11|11|3684|24|697.20|41.83|739.03|52.07|526.32|383.46|263.54|50.20|620.22| +2452298|46349|11897|17821|218037|5806|44422|17821|218037|5806|44422|55|12|3684|58|510.98|5.10|516.08|59.52|147.90|439.44|48.64|22.90|212.52| +2452001|84003|10483|37395|1361263|7084|31352|37395|1361263|7084|31352|26|6|3685|18|1174.68|93.97|1268.65|31.72|289.62|35.24|1105.25|34.19|415.31| +2452816|39450|11964|45739|164298|2857|28169|45739|164298|2857|28169|30|21|3686|5|322.25|16.11|338.36|93.00|181.00|257.80|16.75|47.70|290.11| +2452779|11187|4647|26822|956990|6358|32926|26822|956990|6358|32926|30|1|3686|26|541.06|37.87|578.93|32.89|1209.52|308.40|2.32|230.34|1280.28| +2452033|47170|3989|18821|1649801|3981|45714|18821|1649801|3981|45714|8|29|3687|15|35.40|1.41|36.81|89.77|18.45|26.90|6.29|2.21|109.63| +2450992|24120|5786|55565|667529|740|29551|55565|667529|740|29551|22|1|3688|77|4211.90|379.07|4590.97|62.30|1437.59|1937.47|1865.03|409.40|1878.96| +2451067|42097|4378|65588|357205|822|35239|65588|357205|822|35239|2|14|3688|1|86.51|5.19|91.70|62.10|51.90|46.71|39.40|0.40|119.19| +|33727|1375|40581|755896||||755896|3304|18723|13|16|3689||||510.34|9.71|90.84|0.00|401.33|94.15|115.41| +2452456|39012|9441|97095|1312273|5851|45829|97095|1312273|5851|45829|21|13|3690|26|808.34|0.00|808.34|84.30|109.98|598.17|60.94|149.23|194.28| +2452483|27174|15303|81239|1383103|2567|41584|81239|1383103|2567|41584|59|24|3690|28|6251.00|250.04|6501.04|72.69|3194.80|5875.94|195.03|180.03|3517.53| +2451985|29036|5309|43979|1836124|5422|15120|43979|1836124|5422|15120|55|2|3691|34|5690.58|284.52|5975.10|81.77|640.90|3585.06|1579.14|526.38|1007.19| +||5077||830041|6694|45747|||||29|7|3691||2631.96|||33.18|418.68||||| +2452022|44902|14798|53917|1223485|4700|29115|53917|1223485|4700|29115|11|15|3691|6|55.32|4.97|60.29|54.01|39.54|26.00|12.90|16.42|98.52| +2451913|30922|16604|42239|1000326|3566|18006|42239|1000326|3566|18006|25|8|3691|13|84.24|3.36|87.60|14.69|87.10|49.70|34.19|0.35|105.15| +2451990|60575|15665|69066|493905|5944|25136|69066|493905|5944|25136|55|11|3691|9|82.89|3.31|86.20|49.13|56.70|38.95|31.19|12.75|109.14| +2452142|48080|16285|26882|150020|2405|33543|26882|150020|2405|33543|31|12|3692|43|8763.40|175.26|8938.66|23.41|4687.43|5959.11|532.81|2271.48|4886.10| +2452032|49871|14975|14034|1565431|4475|23472|14034|1565431|4475|23472|14|35|3694|3|14.19|0.85|15.04|34.43|11.28|4.82|2.71|6.66|46.56| +2452551|78998|1836|57404|869073|3142|34011|57404|869073|3142|34011|49|3|3695|3|30.33|0.60|30.93|40.93|166.89|24.87|4.64|0.82|208.42| +2452064|76899|3674|44926|1360136|6546|36956|44926|1360136|6546|36956|31|19|3696|66|3564.00|249.48|3813.48|51.08|792.00|1639.44|904.54|1020.02|1092.56| +2451606|34699|3808|74564|1425192|4620|2495|74564|1425192|4620|2495|56|5|3697|1|4.89|0.09|4.98|45.69|10.11|2.83|0.35|1.71|55.89| +2452089|61909|1053|53352|1846385|2171|32404|53352|1846385|2171|32404|29|23|3699|52|3193.84|255.50|3449.34|77.71|1970.80|2491.19|548.06|154.59|2304.01| +2452127|44702|1423|64759|579190|1724|43402|64759|579190|1724|43402|11|1|3699|43|2951.52|88.54|3040.06|53.63|872.04|2538.30|136.36|276.86|1014.21| +2451672|15263|14315|64448|1833408|7168|15728|64448|1833408|7168|15728|25|18|3702|10|1131.70|56.58|1188.28|16.17|760.40|407.41|550.46|173.83|833.15| +2451180|71039|15496|16075|710927|3379|47831|16075|710927|3379|47831|2|13|3704|12|756.36|30.25|786.61|50.38|394.56|582.39|137.43|36.54|475.19| +2452733|73272|8317|62727|1489753|2826|47452|62727|1489753|2826|47452|12|12|3705|11|868.45|8.68|877.13|17.69|501.05|225.79|533.40|109.26|527.42| +2452149|63668|2363|62387|670977|5167|33310|62387|670977|5167|33310|29|16|3708|7|1025.43|71.78|1097.21|99.70|201.39|656.27|236.26|132.90|372.87| +2451240|62234|12458|22866|642085|3704|28123|22866|642085|3704|28123|37|30|3709|67|4994.85|449.53|5444.38|69.39|3132.25|249.74|759.21|3985.90|3651.17| +2451388|60530|14432|37880|1759899|1118|16737|37880|1759899|1118|16737|31|26|3710|7|1285.20|89.96|1375.16|82.47|195.51|925.34|165.53|194.33|367.94| +2452071|35604|14199|45410|1872399|2235|30824|45410|1872399|2235|30824|33|30|3712|42|266.28|5.32|271.60|74.70|433.02|31.95|56.23|178.10|513.04| +2452059|6832|4073|92068|1374386|3104|31022|92068|1374386|3104|31022|5|33|3713|11|82.72|3.30|86.02|76.71|114.62|36.39|42.62|3.71|194.63| +2451931|27997|15197|16554|1838033|1912|12991|16554|1838033|1912|12991|35|7|3713|57|6467.22|0.00|6467.22|10.50|1379.40|3168.93|758.60|2539.69|1389.90| +2451278|69306|9830|95595|276240|1332|25177|95595|276240|1332|25177|10|30|3714|13|70.20|3.51|73.71|5.39|224.90|30.88|5.89|33.43|233.80| +2452318|46106|13491|47012|937961|110|21059|47012|937961|110|21059|19|26|3715|62|12028.62|120.28|12148.90|82.18|4350.54|1202.86|6928.48|3897.28|4553.00| +2452317|62834|14099|31320|657000|5880|10807|31320|657000|5880|10807|43|9|3715|5|153.40|6.13|159.53|41.16|70.55|145.73|4.29|3.38|117.84| +2452271|44262|1589|38927|1234016|2705|37841|38927|1234016|2705|37841|11|9|3715|2|67.30|4.71|72.01|14.45|62.32|22.20|42.39|2.71|81.48| +2452404||6445||||4928|||4343||7||3717||357.20|10.71|||||52.83|129.35|| +2452562|5110|14119|10705|978638|6218|6598|10705|978638|6218|6598|47|28|3717|32|3276.80|65.53|3342.33|86.27|1450.24|2326.52|855.25|95.03|1602.04| +2452431|30978|15199|18924|980321|4918|47115|18924|980321|4918|47115|27|29|3717|7|473.55|37.88|511.43|14.36|7.00|71.03|185.15|217.37|59.24| +2452415|35027|16343|17713|539193|1120|39447|17713|539193|1120|39447|15|3|3717|15|0.00|0.00|0.00|38.79|74.10|0.00|0.00|0.00|112.89| +2452488|28702|369|57688|1900761|5434|40507|57688|1900761|5434|40507|11|20|3718|20|1369.20|0.00|1369.20|80.65|1908.80|561.37|638.18|169.65|1989.45| +2452342|71055|4455|35072|740252|7032|39365|35072|740252|7032|39365|55|33|3718|22|954.36|19.08|973.44|82.23|685.08|143.15|8.11|803.10|786.39| +2452263|76598|17289|95670|1362247|2871|2295|95670|1362247|2871|2295|57|22|3718|41|637.14|38.22|675.36|17.19|281.67|305.82|6.62|324.70|337.08| +2452454|44024|12774|13159|1115776|3640|13057|13159|1115776|3640|13057|24|33|3719|2|21.78|1.74|23.52|19.51|23.22|20.03|0.19|1.56|44.47| +2452483|25909|14445|39021|1736738|4855|35772|39021|1736738|4855|35772|49|1|3719|41|3759.70|187.98|3947.68|59.29|375.97|1767.05|996.32|996.33|623.24| +2451392|7035|8774|38430|462674|5765|11656|38430|462674|5765|11656|56|21|3720|8|77.28|6.18|83.46|66.03|121.04|48.68|1.14|27.46|193.25| +2451199|58606|6730|79321|944162|1450|41091|79321|944162|1450|41091|56|26|3720|8|43.36|0.86|44.22|81.01|93.92|7.80|29.51|6.05|175.79| +2452279|71720|15223|75929|1584011|6493|34128|75929|1584011|6493|34128|9|6|3721|68|12122.36|606.11|12728.47|32.02|4494.80|363.67|10347.64|1411.05|5132.93| +2452763|33727|7338|13329|1699013|3097|45581|13329|1699013|3097|45581|19|8|3722|47|2210.88|22.10|2232.98|49.19|93.06|1702.37|96.61|411.90|164.35| +||6928|55821|1915996||||1915996|6571||||3723|||||60.85|||76.74||1525.95| +2451245|20279|85|77259|419486|4113|26293|77259|419486|4113|26293|37|16|3723|12|12.84|0.64|13.48|30.33|231.84|1.92|9.71|1.21|262.81| +2451372|45647|11914|67979|997171|5916|12708|67979|997171|5916|12708|58|9|3723|9|369.00|0.00|369.00|44.33|228.69|284.13|31.40|53.47|273.02| +2452037||357|79733|1856479|||79733|1856479||12018|||3724|||93.66||91.32|||391.12||| +2451159|36025|13708|89769|1342175|7028|21434|89769|1342175|7028|21434|46|28|3726|3|555.39|5.55|560.94|51.71|73.65|38.87|495.85|20.67|130.91| +2452073|67632|9361|65951|379005|3259|5513|65951|379005|3259|5513|38|32|3727|39|134.16|6.70|140.86|42.29|24.18|40.24|48.83|45.09|73.17| +2452710|68949|9531|74707|138717|5574|46431|74707|138717|5574|46431|48|17|3729|35|1965.95|19.65|1985.60|36.70|540.40|393.19|1399.75|173.01|596.75| +2451173|73517|5770|55917|860075|6633|40038|55917|860075|6633|40038|19|25|3730|75|0.00|0.00|0.00|75.92|180.00|0.00|0.00|0.00|255.92| +2451210|5233|5200|23129|462463|7044|35978|23129|462463|7044|35978|44|9|3730|73|2357.90|141.47|2499.37|85.88|261.34|1131.79|1177.06|49.05|488.69| +2451236|4752|11192|46968|1883741|4161|35334|46968|1883741|4161|35334|31|10|3731|15|408.30|32.66|440.96|51.07|36.00|306.22|28.58|73.50|119.73| +2452516|17717|15905|16984|697007|6277|17947|16984|697007|6277|17947|9|21|3732|36|278.64|16.71|295.35|35.73|15.48|61.30|26.08|191.26|67.92| +2451994|65583|16001|87657|66271|2485|346|87657|66271|2485|346|26|7|3733|7|461.58|4.61|466.19|85.21|44.17|216.94|2.44|242.20|133.99| +2452115|60165|12121|93968|1633662|3403|43497|93968|1633662|3403|43497|50|28|3735|24|173.76|3.47|177.23|27.26|212.40|71.24|98.41|4.11|243.13| +2451127|55714|12128|10993|1814071|6699|32436|10993|1814071|6699|32436|58|28|3736|56|140.00|4.20|144.20|17.56|206.08|67.20|40.04|32.76|227.84| +|54711|17588|51875|43289||8117||43289|5903|8117|55|19|3736||121.65|1.21|||128.10|116.78|||| +2451883|48301|11915|97428|1432133|4438|47943|97428|1432133|4438|47943|20|5|3737|81|5114.34|409.14|5523.48|36.37|757.35|971.72|207.13|3935.49|1202.86| +2452122|37253|1587|43054|1134643|2232|27421|43054|1134643|2232|27421|21|23|3738|9|7.65|0.61|8.26|91.91|42.21|1.53|3.73|2.39|134.73| +2452053|66050|4971|35127|555215|5703|1791|35127|555215|5703|1791|37|6|3738|47|1516.22|45.48|1561.70|96.71|558.36|106.13|296.11|1113.98|700.55| +2451729|14923|17191|68384|851168|6305|21889|68384|851168|6305|21889|28|18|3739|29|2606.52|52.13|2658.65|71.75|953.52|130.32|1337.14|1139.06|1077.40| +2451844|34152|9349|255|1412526|4350|25374|255|1412526|4350|25374|49|32|3741|7|104.16|4.16|108.32|45.54|17.92|64.57|35.23|4.36|67.62| +2451815|56422|4913|90636|75765|5694|3623|90636|75765|5694|3623|25|7|3743|3|622.77|56.04|678.81|27.84|268.14|379.88|123.87|119.02|352.02| +2452047|35376|17243|94779|95832|6838|20644|94779|95832|6838|20644|5|24|3743|36|1450.08|0.00|1450.08|55.41|569.52|1131.06|76.56|242.46|624.93| +2452203|66331|11835|50837|1135410|2830|7803|50837|1135410|2830|7803|13|28|3744|56|656.88|0.00|656.88|99.33|347.76|479.52|99.32|78.04|447.09| +2452426|36347|6825|48684|1486201|4071|33141|48684|1486201|4071|33141|55|15|3745|30|220.50|19.84|240.34|87.22|576.60|105.84|110.07|4.59|683.66| +2451030|75369|12022|97394|1192010|2626|48395|97394|1192010|2626|48395|31|34|3746|5|336.05|20.16|356.21|18.13|294.00|141.14|89.65|105.26|332.29| +2451105|72820|14326|78679|1375716|753|37790|78679|1375716|753|37790|16|20|3746|9|297.09|26.73|323.82|19.24|18.54|68.33|41.17|187.59|64.51| +2451924|49627|17137|10423|692040|4857|30474|10423|692040|4857|30474|44|8|3747|31|528.86|26.44|555.30|12.62|170.19|349.04|21.57|158.25|209.25| +2452225|29221|13817|84849|533744|4239|13534|84849|533744|4239|13534|55|21|3748|72|3299.76|65.99|3365.75|15.24|4950.00|824.94|1286.90|1187.92|5031.23| +2452584|51270|16843|30512|1215261|4810|10759|30512|1215261|4810|10759|12|26|3750|48|2198.40|21.98|2220.38|11.93|1099.20|593.56|417.25|1187.59|1133.11| +2452031|85107|15525|48516|855412|6308|17157|48516|855412|6308|17157|51|34|3752|2|55.50|1.11|56.61|81.49|3.46|13.32|36.27|5.91|86.06| +2452195|63070|11723|23419|1520608|997|18586|23419|1520608|997|18586|21|30|3752|15|953.40|9.53|962.93|48.44|210.45|867.59|6.86|78.95|268.42| +2452121|25883|3363|53036|1480447|5011|21622|53036|1480447|5011|21622|41|12|3752|10|370.90|33.38|404.28|85.17|33.70|163.19|126.70|81.01|152.25| +2451265|43238|2257|66790|1670782|727|9472|66790|1670782|727|9472|10|16|3753|10|462.70|23.13|485.83|19.28|352.90|148.06|12.58|302.06|395.31| +2451877|81620|6355|98476|1660290|4151|8365|98476|1660290|4151|8365|1|35|3755|23|346.15|17.30|363.45|28.18|484.61|17.30|272.94|55.91|530.09| +2452887|46260|15727|53832|108796|5713|45830|53832|108796|5713|45830|33|26|3758|60|220.20|11.01|231.21|18.86|2206.20|156.34|22.98|40.88|2236.07| +2452783|72145|14935|9980|871618|5703|2941|9980|871618|5703|2941|55|13|3760|1|22.20|0.88|23.08|32.28|64.60|15.76|1.80|4.64|97.76| +2452362|39864|4247|76886|758904|4922|44140|76886|758904|4922|44140|31|10|3761|15|1836.15|0.00|1836.15|38.49|734.40|661.01|975.36|199.78|772.89| +2452482|56892|2163|36527|333151|2928|34352|36527|333151|2928|34352|39|3|3761|11|222.42|6.67|229.09|88.63|0.00|146.79|43.10|32.53|95.30| +2452372|61957|15503|53342|1437431|6673|5962|53342|1437431|6673|5962|31|9|3761|2|4.14|0.20|4.34|79.85|0.22|0.41|2.61|1.12|80.27| +|46906|11995|95081||3403|||704042|3403|||3|3763|29||1.96|99.98||||6.35|20.12|| +2451377|33447|11714|18201|583477|2353|32876|18201|583477|2353|32876|13|28|3763|8|174.88|1.74|176.62|86.05|13.04|152.14|21.83|0.91|100.83| +2451250|34037|598|93700|1312154|4655|28309|93700|1312154|4655|28309|2|19|3764|12|294.48|26.50|320.98|56.36|253.92|265.03|24.44|5.01|336.78| +2451462|61529|2404|73469|528149|76|15556|73469|528149|76|15556|37|25|3764|19|495.52|29.73|525.25|66.91|464.55|54.50|202.86|238.16|561.19| +2452303|77066|17667|80643||1090|||506643||20690||31|3766|7||||19.44|0.00|118.44|||| +2452419|11873|6559|58654|1183698|1542|38586|58654|1183698|1542|38586|47|25|3766|23|318.78|6.37|325.15|64.22|660.79|184.89|132.55|1.34|731.38| +2452461|38705|10471|21841|1142278|1915|19152|21841|1142278|1915|19152|1|2|3766|22|2508.88|175.62|2684.50|21.37|1003.42|351.24|151.03|2006.61|1200.41| +2452372|58396|5401|35342|137904|4113|15475|35342|137904|4113|15475|5|31|3766|26|4071.60|40.71|4112.31|22.56|1533.48|2727.97|1128.64|214.99|1596.75| +2452498|80064|11165|53442|767061|1833|25452|53442|767061|1833|25452|19|18|3766|1|108.77|3.26|112.03|25.05|58.33|47.85|50.56|10.36|86.64| +2452377||14701|53465|1048613||34109|53465|1048613|3761||21||3768|28||||83.48||31.92||7.03|| +2452498|40780|9594|32506|1540614|4722|33630|32506|1540614|4722|33630|6|11|3769|5|459.00|4.59|463.59|79.38|280.50|18.36|400.98|39.66|364.47| +2452813|61933|4362|27082|977674|3019|41564|27082|977674|3019|41564|7|8|3770|16|120.48|3.61|124.09|39.50|603.04|81.92|35.08|3.48|646.15| +2451601|65024|14384|33380|1338845|3984|7889|33380|1338845|3984|7889|1|24|3772|7|176.19|7.04|183.23|51.97|64.68|33.47|62.79|79.93|123.69| +2451600|54964|3650|99340|1371939|4985|9584|99340|1371939|4985|9584|25|17|3772|16|379.04|18.95|397.99|5.24|57.60|216.05|96.16|66.83|81.79| +2451467|65801|6613|12946|410205|6311|27217|12946|410205|6311|27217|32|31|3772|36|133.56|12.02|145.58|98.91|112.32|96.16|5.98|31.42|223.25| +|11054|7390||694665|3806||54677|||11394||8|3777|45||40.98|496.38||||||118.70| +2452323|52408|14193|48708|1226870|7117|47439|48708|1226870|7117|47439|51|27|3778|19|1844.14|165.97|2010.11|42.20|836.19|1604.40|21.57|218.17|1044.36| +2452366|63567|1153|48708|1226870||47439|48708|1226870|||57|26|3778|63|103.95||109.14|26.91|573.93|55.09|31.75|17.11|606.03| +2451962|36962|15380|7076|1578356|3683|37763|7076|1578356|3683|37763|1|26|3779|16|1250.40|87.52|1337.92|64.02|380.48|400.12|25.50|824.78|532.02| +2452643|69341|8529|38469|237387|6705|36423|38469|237387|6705|36423|51|31|3780|13|170.82|11.95|182.77|87.39|94.90|88.82|56.58|25.42|194.24| +2451773|77398|14342|53023|798102|4334|9813|53023|798102|4334|9813|14|26|3781|69|3777.06|37.77|3814.83|11.71|1831.26|604.32|2633.37|539.37|1880.74| +2451946|50296|14348|55759|1553907||33454||||33454|||3782|32||117.38|||493.12||108.65||642.10| +2451759|59604|12686|55759|1553907|4638|33454|55759|1553907|4638|33454|19|3|3782|9|29.16|0.87|30.03|52.10|476.37|22.16|6.93|0.07|529.34| +2451766|40878|332|51368|565268|321|45774|51368|565268|321|45774|28|21|3783|83|2397.87|23.97|2421.84|83.77|6235.79|1870.33|506.43|21.11|6343.53| +2451499|12508|2887|72548|369629|3863|26097|72548|369629|3863|26097|52|17|3784|52|625.04|31.25|656.29|39.88|138.84|481.28|57.50|86.26|209.97| +2451480|42863|746|62915|110810|1105|21800|62915|110810|1105|21800|31|20|3785|12|307.56|12.30|319.86|64.08|49.20|67.66|119.95|119.95|125.58| +2451402|8432|379|8530|906108|1639|20417|8530|906108|1639|20417|26|4|3785|37|2405.37|96.21|2501.58|88.20|784.40|2261.04|98.14|46.19|968.81| +2451288|46614|5494|93592|895617|3595|22420|93592|895617|3595|22420|31|35|3785|43|57.19|5.14|62.33|37.56|131.58|48.61|2.74|5.84|174.28| +|45838|9823||||31652|||2205|31652|21||3786|||0.45|6.10|23.35|||0.43|3.19|24.55| +|18310|14564||1739360|5373||25237|1739360|||||3787|||189.63||87.98|1835.13|||1782.60|2112.74| +2451279|48192|10714|85864|1061197|1777|35110|85864|1061197|1777|35110|44|4|3787|27|3592.89|0.00|3592.89|26.51|579.42|1006.00|181.08|2405.81|605.93| +2451244|35543|6242|65441|1585938|2882|27426|65441|1585938|2882|27426|38|19|3789|20|506.60|15.19|521.79|47.39|287.80|238.10|228.22|40.28|350.38| +2452733|29327|12757|||4962|||1402933|4962|||29|3790|26|975.00|19.50|994.50|79.11||29.25|189.15||| +2452059|44229|12259|21963|1918624|1685|39193|21963|1918624|1685|39193|11|1|3791|9|155.07|0.00|155.07|53.96|62.01|35.66|64.48|54.93|115.97| +2451785|15725|8654|4647|445793|1692|42438|4647|445793|1692|42438|47|10|3792|14|1637.44|0.00|1637.44|35.04|516.18|1031.58|127.23|478.63|551.22| +2451241|64834|1315|18157|1382166|3775|47384|18157|1382166|3775|47384|13|16|3793|9|729.63|65.66|795.29|18.77|567.45|211.59|378.16|139.88|651.88| +2451311|22782|6523|47731|821846|4761|38523|47731|821846|4761|38523|20|2|3793|22|1795.20|35.90|1831.10|30.88|1060.62|143.61|297.28|1354.31|1127.40| +2451402|20986|3746|7097|1570787|5677|37767|7097|1570787|5677|37767|1|12|3793|9|572.40|28.62|601.02|26.02|628.20|509.43|56.04|6.93|682.84| +2452582|44459|11985|73904|262371|6391|46058|73904|262371|6391|46058|27|6|3794|30|63.60|3.81|67.41|42.68|20.10|20.98|25.99|16.63|66.59| +2452278|7954|7449|6052|354777|749|47074|6052|354777|749|47074|3|16|3796|17|36.04|1.80|37.84|88.75|205.02|1.44|20.76|13.84|295.57| +2452752|49956|9453|89581|44679|4577|14028|89581|44679|4577|14028|37|9|3797|10|871.00|78.39|949.39|2.98|488.60|113.23|219.75|538.02|569.97| +2451933|49610|12013|755|1148956|2462|5664|755|1148956|2462|5664|35|7|3798|21|1190.49|95.23|1285.72|80.26|175.56|773.81|162.50|254.18|351.05| +2451992|70326|10085|4951|991117|4483|9849|4951|991117|4483|9849|8|3|3798|5|190.15|17.11|207.26|31.91|23.05|62.74|33.12|94.29|72.07| +2451347|40695|7178|92213|1755356|99|24042|92213|1755356|99|24042|8|2|3799|49|582.61|17.47|600.08|36.64|179.34|291.30|247.61|43.70|233.45| +2452541|44824|9204|55646|254670|3294|1071|55646|254670|3294|1071|24|26|3800|6|308.22|0.00|308.22|69.22|102.72|178.76|82.85|46.61|171.94| +2451745|78253|17096|99444|895190|2136|20682|99444|895190|2136|20682|38|8|3801|33|301.62|12.06|313.68|75.09|68.31|250.34|14.87|36.41|155.46| +2451048|62302|6364|10173|109160|4500|35087|10173|109160|4500|35087|26|10|3802|24|5023.68|401.89|5425.57|80.64|827.28|2562.07|24.61|2437.00|1309.81| +2451156|72224|10753|87322|328371|5867|2116|87322|328371|5867|2116|50|20|3802|45|302.85|27.25|330.10|11.41|224.55|145.36|124.41|33.08|263.21| +2451796|65972|2365|3382|1342236|2337|6791|3382|1342236|2337|6791|44|2|3803|26|20.28|0.00|20.28|91.50|18.98|0.81|13.23|6.24|110.48| +2452541||17802|||3285|25892|49365|1465525|3285|25892||4|3805||208.98|4.17|213.15|||154.64|30.43||| +2452557|39454|3168|667|989639|4211|7983|667|989639|4211|7983|37|35|3805|9|1151.01|57.55|1208.56|92.56|687.42|840.23|279.70|31.08|837.53| +2451793|59815|9865|98689|1248516|5990|9017|98689|1248516|5990|9017|26|18|3807|20|896.40|26.89|923.29|6.08|607.20|457.16|338.21|101.03|640.17| +2452024|4116|14930|44325|332067|6656|35998|44325|332067|6656|35998|26|21|3807|1|82.41|5.76|88.17|81.87|46.26|45.32|14.46|22.63|133.89| +2452025|40067|15319|807|63053|1299|23545|807|63053|1299|23545|5|23|3809|34|2353.82|0.00|2353.82|36.74|1176.74|894.45|218.90|1240.47|1213.48| +2451648|5822|17224|25883|330215|947|29679|25883|330215|947|29679|34|4|3810|63|721.98|7.21|729.19|46.21|1035.09|339.33|355.86|26.79|1088.51| +2451810|25249|14596|17618|120578|6738|37251|17618|120578|6738|37251|56|24|3810|3|168.99|10.13|179.12|69.02|68.64|10.13|36.53|122.33|147.79| +||16737||195303|||39152|195303||3958|1||3812|28||215.11|5593.07|69.13|640.08||||| +2451324|62340|17881|55437|498709|6334|10406|55437|498709|6334|10406|2|34|3821|18|488.16|34.17|522.33|92.32|69.66|209.90|200.34|77.92|196.15| +2451306|19695|9860|90415|969959|6656|7400|90415|969959|6656|7400|55|11|3821|63|1871.10|18.71|1889.81|45.10|1927.80|542.61|26.56|1301.93|1991.61| +2451659|47656|4442|51638|851893|4082|36425|51638|851893|4082|36425|1|33|3822|53|2125.30|191.27|2316.57|21.00|187.09|977.63|401.68|745.99|399.36| +2452441|72917|9937|18959|450622|2638|40301|18959|450622|2638|40301|15|1|3824|46|2387.40|119.37|2506.77|9.58|646.30|1026.58|54.43|1306.39|775.25| +2450892|76603|7646|80575|1470799|1023|45647|80575|1470799|1023|45647|40|21|3825|22|122.76|3.68|126.44|11.22|1186.68|79.79|13.75|29.22|1201.58| +2451066|69747|9266|82637|394458|3879|40900|82637|394458|3879|40900|2|16|3825|64|117.76|1.17|118.93|13.60|457.60|24.72|52.10|40.94|472.37| +2450914|32460|4606|24289|1570836|136|18636|24289|1570836|136|18636|50|3|3825|44|1671.56|66.86|1738.42|79.90|135.08|534.89|22.73|1113.94|281.84| +2451123|76538|6049|34865||203|7719|||203|7719|32|20|3825||7196.11||7411.99||3096.01||||3411.51| +||14788|24757||3821|30553|||||||3825|2|96.16|3.84|100.00||||4.73||| +2451848|28308|8618|97037|766271|5105|7186|97037|766271|5105|7186|25|32|3826|26|752.96|60.23|813.19|63.87|256.62|587.30|165.66|0.00|380.72| +2451720|45578|13997|21735|689579|3023|8867|21735|689579|3023|8867|7|14|3826|20|2879.40|143.97|3023.37|33.17|1199.80|1785.22|273.54|820.64|1376.94| +2451714|36556|14767|4980|1128644|661|7878|4980|1128644|661|7878|35|6|3826|1|51.47|0.00|51.47|97.91|100.79|9.26|12.24|29.97|198.70| +2450946|80263|5101|26761|57692|6576|11060|26761|57692|6576|11060|58|5|3828|24|1056.24|95.06|1151.30|45.66|821.52|591.49|246.31|218.44|962.24| +2450924|78320|17989|9895|79468|1041|18614|9895|79468|1041|18614|43|27|3828|7|524.58|47.21|571.79|38.52|174.86|110.16|49.73|364.69|260.59| +2452439|22598|2505|93352|1847636|1120|49404|93352|1847636|1120|49404|59|11|3829|12|233.88|2.33|236.21|52.38|136.32|159.03|59.88|14.97|191.03| +2452490|66346|12343|32197|1522086|4029|35154|32197|1522086|4029|35154|15|30|3829|73|860.67|8.60|869.27|51.92|1865.15|860.67|0.00|0.00|1925.67| +2452375|72303|11811|64916|25616|1869|26922|64916|25616|1869|26922|33|24|3829|21|1744.47|34.88|1779.35|87.00|910.14|1081.57|397.74|265.16|1032.02| +2452292|58527|2713|77016|110841|666|15291|77016|110841|666||39||3829|9|155.52||155.52|77.59|1062.72|102.64|49.70||| +||11889|35593|1186697|2954|37211|35593|1186697|2954|37211||6|3830|11|890.45||961.68|88.48|||17.90||430.20| +|51655|7329|35593|1186697|||35593|1186697||||19|3830|||30.72|414.75|52.26|||90.09||162.54| +2452412|29942|12931|35258|467488|6118|4748|35258|467488|6118|4748|55|20|3830|74|327.08|29.43|356.51|76.62|366.30|35.97|261.99|29.12|472.35| +2452692|43869|9024|60222|1063958|3341|25077|60222|1063958|3341|25077|55|14|3831|40|1585.20|31.70|1616.90|70.96|1513.20|1141.34|79.89|363.97|1615.86| +2452638|43487|9324|65360|422641|369|49250|65360|422641|369|49250|54|7|3831|32|1013.12|81.04|1094.16|69.93|1182.08|901.67|13.37|98.08|1333.05| +2452743|4435|4728|54876|6618|7160|14102|54876|6618|7160|14102|1|32|3831|62|580.94|11.61|592.55|68.48|633.64|23.23|351.35|206.36|713.73| +2451735|46671|2075|80956|797774|1439|17504|80956|797774|1439|17504|25|33|3832|23|2061.26|82.45|2143.71|78.66|1053.40|1442.88|371.02|247.36|1214.51| +2451831|37745|10937|74993|444326|2106|35039|74993|444326|2106|35039|53|3|3832|11|570.57|22.82|593.39|22.45|268.40|422.22|83.07|65.28|313.67| +2451134|19559|2200||51912|934||||934|||29|3833|18|32.04||34.92||342.72|14.73|||| +2451328|54881|4012|48108|407980|3852|171|48108|407980|3852|171|49|22|3833|58|6878.22|137.56|7015.78|94.49|1810.18|2751.28|1320.62|2806.32|2042.23| +2451335|13058|7816|87665|1319713|4561|35409|87665|1319713|4561|35409|40|35|3833|9|1100.79|22.01|1122.80|41.74|133.38|517.37|501.74|81.68|197.13| +2451207|39628|4882|29223|289056|2136|10816|29223|289056|2136|10816|32|20|3833|1|30.35|0.30|30.65|94.17|18.35|22.15|6.31|1.89|112.82| +2452659|39082|3462|26276|1077133|790|37200|26276|1077133|790|37200|36|25|3835|9|166.23|14.96|181.19|94.85|243.81|78.12|61.67|26.44|353.62| +2452391|65469|485|21365|1785566|3014|36015|21365|1785566|3014|36015|37|23|3836|1|65.51|1.31|66.82|35.20|17.40|32.75|23.58|9.18|53.91| +2451102|3057|9688|68753|852585|2017|20249|68753|852585|2017|20249|32|4|3837|21|145.11|13.05|158.16|57.29|798.63|63.84|17.06|64.21|868.97| +2452757|65181|7057|74887|238337|4187|20437|74887|238337|4187|20437|42|22|3838|31|748.34|67.35|815.69|19.77|21.08|344.23|367.74|36.37|108.20| +2452766|48496|10837|95003|399797|2111|19493|95003|399797|2111|19493|31|24|3838|32|415.04|24.90|439.94|11.56|718.40|240.72|19.17|155.15|754.86| +2452875|36986|10842|37814|974449|2637|15625|37814|974449|2637|15625|39|20|3838|48|7776.48|466.58|8243.06|89.19|3993.12|311.05|3658.06|3807.37|4548.89| +2452449|40132|9933|65435|1671684|2733|42182|65435|1671684|2733|42182|29|16|3839|60|3312.60|132.50|3445.10|14.35|1009.80|2981.34|16.56|314.70|1156.65| +2452367|37345|15509|26831|1175119|7054|27342|26831|1175119|7054|27342|39|31|3840|29|22.04|1.76|23.80|99.46|26.10|17.63|4.32|0.09|127.32| +2451460|10307|10454|43157|405515|6273|22678|43157|405515|6273|22678|8|35|3841|46|61.64|3.69|65.33|1.57|8.28|28.35|26.29|7.00|13.54| +2451392|72525|1988|86661|1312361|14|15667|86661|1312361|14|15667|13|27|3841|55|713.35|42.80|756.15|7.92|656.15|156.93|133.54|422.88|706.87| +2452420|50157|7533|25987|228382|4730|34725|25987|228382|4730|34725|27|19|3842|43|3240.05|0.00|3240.05|45.42|704.34|3078.04|77.76|84.25|749.76| +2451164|21100|6032|98659|45625|1391|8357|98659|45625|1391|8357|28|28|3844|15|52.20|0.52|52.72|97.26|146.10|25.57|22.90|3.73|243.88| +2452730|72532|1303|55383|1153370|4157|8144|55383|1153370|4157|8144|13|20|3845|89|2849.78|85.49|2935.27|24.92|1477.40|1852.35|49.87|947.56|1587.81| +||2069|16984|1504848||||1504848||36223|47||3847||567.54|39.72|||298.32||43.59||398.29| +2451922|79118|14995|83549|1017630|5758|21039|83549|1017630|5758|21039|55|1|3847|23|593.40|41.53|634.93|76.63|208.84|130.54|152.74|310.12|327.00| +2451882|48339|12143|85302|366842|4735|9875|85302|366842|4735|9875|37|14|3847|1|25.08|1.75|26.83|74.81|5.84|0.50|14.25|10.33|82.40| +2451826|20174|1106|96986|1208821|2038|43147|96986|1208821|2038|43147|55|7|3847|20|852.00|25.56|877.56|80.32|401.40|119.28|51.29|681.43|507.28| +2451940|18492|17714|98135|1251972|6962|45199|98135|1251972|6962|45199|17|12|3848|2|40.20|3.21|43.41|66.11|4.02|33.76|0.19|6.25|73.34| +2451911|55695|11135|25615|830240|2865|14243|25615|830240|2865|14243|5|11|3848|1|68.36|6.15|74.51|1.65|13.67|47.16|2.75|18.45|21.47| +||6415||100828|956||37500|||41131|7|6|3848|42||||||||3.67|| +2451375|81103|8404|41938|226290|3089|49841|41938|226290|3089|49841|28|22|3849|1|30.63|1.83|32.46|77.99|5.64|2.45|18.59|9.59|85.46| +2451657|58253|5149|15730|114768|1524|41784|15730|114768|1524|41784|7|19|3850|58|4217.18|210.85|4428.03|52.18|6560.38|3289.40|890.66|37.12|6823.41| +2451627|66098|7466|65323|1460861|4752|39843|65323|1460861|4752|39843|50|16|3850|21|717.36|50.21|767.57|87.46|71.61|695.83|9.68|11.85|209.28| +2452300|60492|9979|16181|504110|5639|768|16181|504110|5639|768|35|32|3851|35|1117.90|100.61|1218.51|83.25|141.05|67.07|378.29|672.54|324.91| +2452351|61522|10421|28428|1618867|859|21251|28428|1618867|859|21251|17|2|3851|3|42.81|0.85|43.66|51.43|79.05|27.82|13.19|1.80|131.33| +2452215|37127|11041|21755|506539|6114|37123|21755|506539|6114|37123|43|27|3853|50|0.00|0.00|0.00|94.10|135.00|0.00|0.00|0.00|229.10| +2451551|61174|3658|94654|61989|2669|14336|94654|61989|2669|14336|2|26|3855|2|8.42|0.25|8.67|16.45|10.82|1.43|5.45|1.54|27.52| +2451858|63264|9182|79676|1171633|37|5241|79676|1171633|37|5241|23|16|3856|3|64.62|3.23|67.85|99.66|12.72|49.75|0.59|14.28|115.61| +2451644|74683|5113|99007|1427527|847|39367|99007|1427527|847|39367|50|32|3856|43|4379.55|350.36|4729.91|78.46|265.31|2978.09|434.45|967.01|694.13| +2451882|75601|13241|58635|1055750|4512|44827|58635|1055750|4512|44827|26|21|3857|17|289.51|11.58|301.09|38.55|81.60|5.79|167.39|116.33|131.73| +2451853|48982|1915|45632|944477|5275|22873|45632|944477|5275|22873|20|15|3857|25|1253.75|100.30|1354.05|53.09|1111.75|689.56|163.61|400.58|1265.14| +2452678|19368|17017|48759|442812|2818|10871|48759|442812|2818|10871|3|18|3858|23|2134.40|128.06|2262.46|4.37|667.00|811.07|767.53|555.80|799.43| +2451371|38548|12199|69080|303709|1185|23814|69080|303709|1185|23814|2|27|3859|26|5565.04|166.95|5731.99|80.76|355.16|3450.32|274.91|1839.81|602.87| +2451274|47257|14857|40038|1387393|1148|1303|40038|1387393|1148|1303|16|1|3859|10|8.90|0.80|9.70|50.18|0.10|7.65|0.11|1.14|51.08| +2452665|61523|3091|1614|363506|6998|22490|1614|363506|6998|22490|15|32|3860|20|204.80|14.33|219.13|25.18|162.20|122.88|22.11|59.81|201.71| +2452731|42392|17281|53016|1328255|6694|48733|53016|1328255|6694|48733|37|35|3860|22|191.62|7.66|199.28|6.07|0.00|187.78|0.42|3.42|13.73| +2451724|21045|9874|44499|529578|257|22583|44499|529578|257|22583|8|19|3861|1|17.21|0.17|17.38|54.33|7.34|12.39|4.19|0.63|61.84| +2451624|20875|4028|47336|796658|6333|25208|47336|796658|6333|25208|50|9|3861|9|583.47|0.00|583.47|75.74|202.95|402.59|23.51|157.37|278.69| +2451183|36843|3764|59511|1733546|6428|6863|59511|1733546|6428|6863|46|26|3863|20|1406.60|0.00|1406.60|32.37|1019.80|1280.00|72.16|54.44|1052.17| +2451171|29827|14965|25247|1668618|7189|6160|25247|1668618|7189|6160|10|30|3863|14|1408.82|98.61|1507.43|52.70|533.54|1084.79|298.10|25.93|684.85| +2451289|70558|15964|76906|437788|1038|1076|76906|437788|1038|1076|38|29|3863|52|5249.92|419.99|5669.91|87.51|590.20|3254.95|438.89|1556.08|1097.70| +2451347|49959|2776|21946|1433838|1283|29128|21946|1433838|1283|29128|8|35|3864|21|677.46|13.54|691.00|82.98|204.12|426.79|15.04|235.63|300.64| +2451428|77255|16993|38342|1106237|4532|14427|38342|1106237|4532|14427|37|9|3865|37|3205.31|160.26|3365.57|18.51|2438.67|480.79|1852.67|871.85|2617.44| +2451541|37122|13660|88728|385462|2996|4977|88728|385462|2996|4977|7|18|3865|15|921.90|55.31|977.21|8.67|943.95|811.27|4.42|106.21|1007.93| +2451207|37064|8659|33788|899576|2164|44352|33788|899576|2164|44352|4|2|3866|33|271.26|5.42|276.68|37.96|178.53|100.36|46.14|124.76|221.91| +2450952|73145|10394|72565|959841|717|9946|72565|959841|717|9946|37|18|3866|19|654.74|52.37|707.11|14.55|139.08|294.63|111.63|248.48|206.00| +2452847|72276|577|86646|1684168|3643|36765|86646|1684168|3643|36765|43|16|3868|29|2404.39|24.04|2428.43|13.86|1831.93|817.49|1380.60|206.30|1869.83| +2451269|77354|9242|71887|1294815|2990|47118|71887|1294815|2990|47118|43|20|3869|3|14.76|1.32|16.08|51.61|14.76|2.95|11.21|0.60|67.69| +2451279|49271|10010|45860|631005|2103|27201|45860|631005|2103|27201|13|15|3870|47|924.49|64.71|989.20|12.65|429.11|591.67|19.96|312.86|506.47| +2451389|18167|14053|5641|1436609|1624|8639|5641|1436609|1624|8639|8|12|3870|25|2052.00|20.52|2072.52|43.65|866.25|1231.20|16.41|804.39|930.42| +2452757||9453|73696|1214804|6312|46220|73696|1214804|6312||30||3871|23|139.84||||0.00||||37.47| +2451557||17500||1025906|2540|49127||1025906|2540||19||3873|6|59.28|0.59|59.87|76.52|||25.27|3.78|| +2452283|43785|3173|21745|891137|6129|46321|21745|891137|6129|46321|55|32|3874|31|1235.97|24.71|1260.68|80.13|226.30|630.34|448.16|157.47|331.14| +2451288|49625|12202|66373|773977|4781|42385|66373|773977|4781|42385|44|29|3875|33|4798.86|47.98|4846.84|20.12|685.41|1391.66|2964.26|442.94|753.51| +2451332|21316|13018|98186|450981|3423|45018|98186|450981|3423|45018|14|2|3876|24|1134.72|45.38|1180.10|46.75|539.04|1089.33|37.67|7.72|631.17| +2451432|41280|2834|6310|858513|4763|11366|6310|858513|4763|11366|38|10|3878|60|5261.40|157.84|5419.24|84.50|0.00|1946.71|961.26|2353.43|242.34| +2452790|15812|1845|88753|571425|4144|34416|88753|571425|4144|34416|12|32|3879|31|83.39|1.66|85.05|70.21|778.72|41.69|27.10|14.60|850.59| +2452462|42866|7599|29565|1155955|3145|28335|29565|1155955|3145|28335|21|30|3880|14|950.88|38.03|988.91|67.38|411.18|465.93|334.61|150.34|516.59| +2452510|56119|10651|70751|1470694|6020|26258|70751|1470694|6020|26258|23|21|3880|43|1225.93|36.77|1262.70|76.55|1114.56|1078.81|64.73|82.39|1227.88| +2452548|75877|869|97067|1797348|829|7910|97067|1797348|829|7910|27|24|3880|55|1375.55|27.51|1403.06|85.54|216.15|729.04|90.51|556.00|329.20| +2451202|72307|10096|5496|1752557|4065|41126|5496|1752557|4065|41126|58|11|3882|16|802.88|56.20|859.08|10.19|45.44|754.70|13.97|34.21|111.83| +2452614||9414||||40009|23609||||54||3883|55||70.91|||||62.00|454.69|| +|64136|12871||471316|3750|||471316|3750|22184|36||3884|7||3.14||46.86|24.85|108.38||22.41|74.85| +2451986|28344|7655|50527|746299|3671|39490|50527|746299|3671|39490|7|16|3885|47|2188.79|196.99|2385.78|1.24|1242.21|1707.25|390.04|91.50|1440.44| +2452002|64232|13813|12444|978140|636|29932|12444|978140|636|29932|49|31|3886|21|579.81|17.39|597.20|45.00|396.69|376.87|16.23|186.71|459.08| +2452009|66507|1703|59683|1211669|3655|9641|59683|1211669|3655|9641|19|20|3886|12|422.28|29.55|451.83|13.70|222.24|0.00|38.00|384.28|265.49| +2451261|71464|15877|27307|662962|7064|27119|27307|662962|7064|27119|1|35|3890|54|4925.88|147.77|5073.65|64.06|2514.24|3743.66|130.04|1052.18|2726.07| +2451895|62502|25|43516|1293033|4597|19761|43516|1293033|4597|19761|43|15|3894|35|1431.85|14.31|1446.16|38.69|400.75|114.54|316.15|1001.16|453.75| +2451926|78869|14695|70272|156350|5981|29259|70272|156350|5981|29259|43|9|3894|26|784.16|23.52|807.68|0.81|1437.80|635.16|102.81|46.19|1462.13| +2452450|61595|7251|85505|1656987|3669|39598|85505|1656987|3669|39598|35|30|3895|1|6.85|0.47|7.32|68.33|19.95|1.98|0.73|4.14|88.75| +2452275|44819|12411|225|566961|5344|33280|225|566961|5344|33280|1|27|3896|1|136.62|1.36|137.98|43.40|62.88|12.29|68.38|55.95|107.64| +2451761|66924|4159|59954|1414280|3006|31285|59954|1414280|3006|31285|55|4|3898|47|460.60|36.84|497.44|16.72|198.81|64.48|289.16|106.96|252.37| +2451895|81010|853|47777|1347014|3404|241|47777|1347014|3404|241|37|11|3899|14|1414.84|42.44|1457.28|2.96|240.52|56.59|135.82|1222.43|285.92| +2452017|63206|12551|93932|706016|1300|14984|93932|706016|1300|14984|7|23|3899|14|323.26|25.86|349.12|46.59|16.52|161.63|124.45|37.18|88.97| +2452735|85674|8928|59159|1417223|1334|42198|59159|1417223|1334|42198|18|24|3900|2|27.70|1.93|29.63|16.56|10.26|14.95|1.78|10.97|28.75| +2452837|44102|162|2371|1635834|4497|14360|2371|1635834|4497|14360|51|25|3901|35|57.75|0.00|57.75|89.35|0.00|41.58|5.33|10.84|89.35| +2452713|46272|13008|59720|871337|4554|33771|59720|871337|4554|33771|37|4|3901|12|150.84|3.01|153.85|56.35|110.64|49.77|5.05|96.02|170.00| +2452877|77535|17145|44407|1035804|3201|39798|44407|1035804|3201|39798|27|7|3901|8|260.96|20.87|281.83|40.39|272.08|28.70|181.16|51.10|333.34| +2451255|74487|11173|72270|1278599|6528|2010|72270|1278599|6528|2010|7|30|3902|7|840.21|75.61|915.82|91.66|246.12|596.54|143.76|99.91|413.39| +2451172|45817|7273|50108|609691|6074|29704|50108|609691|6074|29704|58|19|3902|17|2270.69|0.00|2270.69|62.50|172.72|885.56|41.55|1343.58|235.22| +2451203|50412|14146|57344|1238686|6428|17723|57344|1238686|6428|17723|13|6|3902|37|2662.15|53.24|2715.39|79.78|139.86|2582.28|19.16|60.71|272.88| +2451220|63052|4405|55050|554761|872|26183|55050|554761|872|26183|55|11|3902|26|363.48|18.17|381.65|1.85|156.52|58.15|149.61|155.72|176.54| +2451803|72149|6664|54093|1581997|3501|41005|54093|1581997|3501|41005|37|34|3903|20|1767.00|159.03|1926.03|76.45|723.80|583.11|1053.66|130.23|959.28| +2451653|44740|15379|39225|1355362|3852|3189|39225|1355362|3852|3189|32|27|3904|12|31.44|2.51|33.95|88.92|43.56|2.20|26.90|2.34|134.99| +2451414|44751|10357|18403|1593451|146|5594|18403|1593451|146|5594|22|19|3906|12|428.88|21.44|450.32|62.03|241.56|184.41|29.33|215.14|325.03| +|20238|17360||1296491|||||3205||2|30|3906||751.35|0.00||97.55|890.55||486.87|189.35|988.10| +|49313|7612||193867|4469|2718|74018||4469|2718|20||3909||||105.17|46.60|90.22|52.06||1.05|| +2452786||17721|18294|||||1906673|4299|25125|6||3910|||31.62|1085.62|28.90||200.26|554.93|298.81|126.32| +2452637|16785|315|39806|327857|708|3159|39806|327857|708|3159|57|5|3910|53|666.74|26.66|693.40|33.08|3222.40|660.07|0.66|6.01|3282.14| +2452777|68479|14052|26209|1240986|5810|44634|26209|1240986|5810|44634|39|6|3910|10|88.50|4.42|92.92|85.66|26.30|34.51|37.25|16.74|116.38| +2452666|40164|17895|26265|830994|76|41205|26265|830994|76|41205|60|13|3910|35|291.55|2.91|294.46|64.01|178.15|183.67|72.27|35.61|245.07| +2451852|37759|6523|28050|1312774|2298|12622|28050|1312774|2298|12622|26|21|3911|14|194.04|9.70|203.74|60.52|1330.84|79.55|41.21|73.28|1401.06| +2451708|43637|9908|20802|1318468|3334|29047|20802|1318468|3334|29047|59|25|3911|7|87.64|5.25|92.89|63.44|45.22|80.62|0.35|6.67|113.91| +2451138|34815|1093|40652|1744743|5187|41231|40652|1744743|5187|41231|16|24|3912|5|48.30|2.41|50.71|67.02|376.85|7.24|30.38|10.68|446.28| +2451251|27688|15853|67|1162494|474|37701|67|1162494|474|37701|32|26|3912|14|80.22|5.61|85.83|51.15|26.18|57.75|19.77|2.70|82.94| +2451983|47499|12368|62940|46653|1566|18526|62940|46653|1566|18526|17|28|3913|5|316.65|3.16|319.81|46.41|180.90|94.99|106.39|115.27|230.47| +2452809|40447|17539|94571|1285052|7180|49708|94571|1285052|7180|49708|33|2|3914|54|1732.32|34.64|1766.96|84.14|1128.60|1576.41|12.47|143.44|1247.38| +2452043|18541|13010|71700|645445|1590|5688|71700|645445|1590|5688|47|4|3915|31|533.20|0.00|533.20|3.36|343.48|501.20|10.56|21.44|346.84| +2451307|48959|9136|23162|930503|2644|33185|23162|930503|2644|33185|8|6|3916|46|6.90|0.00|6.90|13.80|57.96|3.51|2.77|0.62|71.76| +2452218|5255|12239|96652|305994|2536|44278|96652|305994|2536|44278|25|2|3917|92|431.48|12.94|444.42|13.85|99.36|431.48|0.00|0.00|126.15| +2451346|46913|7423|1834|542828|6669|16863|1834|542828|6669|16863|13|3|3918|16|437.44|30.62|468.06|49.50|617.76|240.59|102.36|94.49|697.88| +2451382|67686|16099|37963|1610551|4386|43421|37963|1610551|4386|43421|32|5|3918|6|168.30|11.78|180.08|44.78|18.24|114.44|30.70|23.16|74.80| +2452046|41830|6935|1005|1348407|3362|11164|1005|1348407|3362|11164|13|17|3920|28|3639.16|218.34|3857.50|85.83|519.68|800.61|340.62|2497.93|823.85| +2451882|29611|12101|19339|1499862|6785|20908|19339|1499862|6785|20908|41|29|3920|1|17.85|0.17|18.02|61.80|14.16|16.42|0.47|0.96|76.13| +2452089|47937|9901|82065|1156601|737|5273|82065|1156601|737|5273|38|34|3920|18|3076.92|92.30|3169.22|49.17|688.14|30.76|1492.61|1553.55|829.61| +2452057|65370|665|67803|706419|5835|34498|67803|706419|5835|34498|49|15|3920|4|83.08|3.32|86.40|34.71|109.80|3.32|25.52|54.24|147.83| +2451275|12609|17564|75240|756145|6748|3963|75240|756145|6748|3963|31|23|3922|63|418.95|0.00|418.95|91.73|379.89|398.00|13.19|7.76|471.62| +2451316|54097|2168|36387|1915045|5350|12408|36387|1915045|5350|12408|28|30|3922|40|4308.40|0.00|4308.40|93.40|3034.00|990.93|165.87|3151.60|3127.40| +2451445|26536|2971|85793|805811|5476|37094|85793|805811|5476|37094|10|19|3923|82|18368.82|1469.50|19838.32|23.76|9508.72|9000.72|3934.60|5433.50|11001.98| +2451338|33866|6802|63386|1755037|5535|2433|63386|1755037|5535|2433|26|25|3923|17|651.27|19.53|670.80|62.37|444.55|162.81|126.99|361.47|526.45| +2451481|61359|2674|60312|82835|3311|34987|60312|82835|3311|34987|34|9|3923|10|1140.30|0.00|1140.30|59.96|228.00|307.88|749.17|83.25|287.96| +2451973|36728|10673|37445|1197453|3861|34316|37445|1197453|3861|34316|26|8|3926|64|2586.24|181.03|2767.27|10.44|539.52|879.32|699.83|1007.09|730.99| +2451707|64603|4315|4682|1830783|1684|7832|4682|1830783|1684|7832|55|30|3927|23|1155.29|69.31|1224.60|11.40|709.55|300.37|282.12|572.80|790.26| +2452587|41892|11016|3991|672799|993|7399|3991|672799|993|7399|12|32|3928|23|1490.40|119.23|1609.63|49.25|1330.78|1490.40|0.00|0.00|1499.26| +2452782|68436|4728|44772|1579298|6271|7644|44772|1579298|6271|7644|37|33|3928|1|63.66|1.27|64.93|46.63|30.61|3.18|59.27|1.21|78.51| +2452407|44319|1955|78194|1395601|1361|768|78194|1395601|1361|768|49|9|3929|25|732.00|0.00|732.00|86.12|747.00|43.92|688.08|0.00|833.12| +2452273|19541|13341|56422|1779567|665|21810|56422|1779567|665|21810|31|19|3930|45|1795.95|71.83|1867.78|63.32|1959.30|915.93|774.41|105.61|2094.45| +2451932|18815|12991|62503|1913884|3218|32999|62503|1913884|3218|32999|38|1|3934|18|634.50|6.34|640.84|14.45|902.88|412.42|115.48|106.60|923.67| +2451754|53057|13909|525|1757837|2255|22758|525|1757837|2255|22758|56|14|3934|15|728.70|21.86|750.56|62.61|811.65|590.24|60.92|77.54|896.12| +2451817|62039|3857|91327|250392|2257|16122|91327|250392|2257|16122|1|17|3934|26|317.72|3.17|320.89|23.57|120.38|146.15|166.42|5.15|147.12| +2452421|76501|8431||551297||||||28860|||3935|||1.81||69.44|16.98|2.06|||| +2452442|79427|1237|17085|148181|5596|21938|17085|148181|5596|21938|47|13|3936|54|4162.32|41.62|4203.94|97.85|647.46|1373.56|2035.79|752.97|786.93| +2452287|54742|4187|36776|1183747|2718|3689|36776|1183747|2718|3689|55|3|3936|36|554.40|11.08|565.48|28.65|418.32|288.28|71.85|194.27|458.05| +2451197|20137|13502|69734|1525242|1103|2564|69734|1525242|1103|2564|1|2|3937|32|3376.32|303.86|3680.18|15.06|1542.08|810.31|1975.82|590.19|1861.00| +2451292|66576|11882|38263|1640031|3046|40014|38263|1640031|3046|40014|40|9|3937|84|649.32|19.47|668.79|59.84|524.16|142.85|476.08|30.39|603.47| +2451310|44721|15002|66246|1779808|123|6485|66246|1779808|123|6485|44|17|3938|19|2417.37|96.69|2514.06|3.59|265.24|1257.03|69.62|1090.72|365.52| +2451170|45745|3376|95241|695879|3364|849|95241|695879|3364|849|50|25|3938|1|1.68|0.00|1.68|62.28|1.35|1.41|0.25|0.02|63.63| +2452293|50721|1887|16215|1105767|2966|38965|16215|1105767|2966|38965|39|12|3939|38|918.46|27.55|946.01|79.43|1307.20|624.55|238.06|55.85|1414.18| +2452552|44199|85|28649|1374880|2264|16848|28649|1374880|2264|16848|42|16|3941|31|894.04|17.88|911.92|71.80|868.62|98.34|7.95|787.75|958.30| +2452709|61974|17535|66408|701582|2356|16777|66408|701582|2356|16777|49|3|3941|67|3614.65|36.14|3650.79|94.08|1806.99|433.75|349.89|2831.01|1937.21| +2451279|20439|16351|21319|522441|3116|1438|21319|522441|3116|1438|19|28|3942|5|298.65|14.93|313.58|39.89|139.35|197.10|47.72|53.83|194.17| +2451414|50248|12592|24756|391450|3297|28791|24756|391450|3297|28791|10|24|3942|74|2331.00|93.24|2424.24|10.47|4014.50|629.37|1293.23|408.40|4118.21| +2451335|73509|17882|953|593143|4886|28276|953|593143|4886|28276|1|15|3942|34|1968.94|98.44|2067.38|93.66|888.42|492.23|118.13|1358.58|1080.52| +2451236|75664|10432|79614|396422|1230|26038|79614|396422|1230|26038|46|1|3944|30|1495.20|0.00|1495.20|86.11|229.80|717.69|738.63|38.88|315.91| +2451412|7297|2768|4738|512605|6268|44370|4738|512605|6268|44370|56|21|3944|11|1145.32|45.81|1191.13|65.58|822.25|847.53|288.85|8.94|933.64| +2451776|67499|5000|16048|421484|6473|23909|16048|421484|6473|23909|34|14|3945|10|11.00|0.33|11.33|13.08|2.60|1.98|5.32|3.70|16.01| +2451624|72278|17882|17513|937100|5934|8721|17513|937100|5934|8721|49|4|3945|5|941.65|18.83|960.48|94.61|321.00|367.24|126.37|448.04|434.44| +2451815|64163|163|1947|304192|390|18635|1947|304192|390|18635|20|31|3945|27|0.00|0.00|0.00|66.32|718.74|0.00|0.00|0.00|785.06| +2451745|41746|5354|57737|1352192|6647|49102|57737|1352192|6647|49102|2|32|3945|32|487.36|34.11|521.47|78.21|566.40|233.93|235.68|17.75|678.72| +2452280|44747|6571|64862|507932|4602|872|64862|507932|4602|872|53|29|3946|38|2255.68|22.55|2278.23|20.64|1396.50|1894.77|43.30|317.61|1439.69| +2452653|7829|5430|77430|459323|3263|35603|77430|459323|3263|35603|9|29|3947|27|1148.04|0.00|1148.04|91.09|424.71|447.73|231.10|469.21|515.80| +2452557|35714|4789|13915|802467|6219|22771|13915|802467|6219|22771|19|18|3947|27|2459.16|122.95|2582.11|73.17|255.96|1844.37|116.81|497.98|452.08| +2452740|30346|9951|1119|1351521|323|44725|1119|1351521|323|44725|18|22|3947|19|34.01|1.02|35.03|12.00|31.54|13.26|8.50|12.25|44.56| +2452780||4968|3643|1115372|||3643||2654||31||3947|55|833.25||908.24||||||| +2452268|34119|8369|39164|162204|797|11437|39164|162204|797|11437|23|6|3948|9|1142.82|0.00|1142.82|28.68|88.83|1005.68|5.48|131.66|117.51| +2452464|70978|7991|92606|1235272|3985|22251|92606|1235272|3985|22251|39|28|3949|3|458.43|0.00|458.43|28.72|223.98|380.49|49.88|28.06|252.70| +2451168|31117|1778|24326|899687|6248|30009|24326|899687|6248|30009|28|10|3950|72|6721.20|537.69|7258.89|76.04|2172.24|1680.30|4133.53|907.37|2785.97| +2452386|77301|17611|57126|1698451|3329|1854|57126|1698451|3329|1854|13|21|3951|2|162.36|12.98|175.34|1.56|23.76|99.03|58.26|5.07|38.30| +2452256|27757|681|82426|58616|3959|32549|82426|58616|3959|32549|57|23|3951|2|37.36|1.12|38.48|91.22|18.08|33.99|1.68|1.69|110.42| +2451213|69166|11353|45049|697049|1049|46696|45049|697049|1049|46696|32|5|3952|9|19.35|0.77|20.12|54.49|107.46|5.99|11.62|1.74|162.72| +2451114|40763|15013|96951|949878|4052|15826|96951|949878|4052|15826|14|11|3952|64|1996.16|79.84|2076.00|95.07|1181.44|339.34|82.84|1573.98|1356.35| +2452407|60094|13515|65157|267479|5839|1009|65157|267479|5839|1009|60|18|3953|30|1945.80|19.45|1965.25|85.84|207.00|19.45|1078.75|847.60|312.29| +2452658||7177||||36588|||3684|36588|42||3953||||2348.29||857.20|2046.68|105.56|2.16|| +|58360|8019||1467254||||1467254|6713|42572|42|9|3953|33||337.14|5153.49||2408.01|1926.54|||2788.67| +2452890|41394|13860|4093|342804|3915|31426|4093|342804|3915|31426|12|12|3954|3|25.38|0.50|25.88|48.84|24.87|0.76|9.60|15.02|74.21| +2452662|56219|10530|97983|1175365|2794|13776|97983|1175365|2794|13776|18|12|3954|25|3486.75|174.33|3661.08|71.31|626.75|1255.23|245.46|1986.06|872.39| +2452790|945|199|509|572326|5390|2407|509|572326|5390|2407|33|35|3954|27|151.20|7.56|158.76|49.26|529.20|92.23|5.89|53.08|586.02| +2452503|67927|2693|58996|603810|6228|9542|58996|603810|6228|9542|7|7|3955|4|1.72|0.08|1.80|53.58|5.00|1.16|0.07|0.49|58.66| +2452420|30731|17193|58996|603810|6228|9542|58996|603810|6228|9542|25|25|3955|27|4430.97|354.47|4785.44|88.41|2840.40|2437.03|1595.15|398.79|3283.28| +2452500|29278|2819|35993|1901161|5882|18250|35993|1901161|5882|18250|5|11|3955|49|7793.45|701.41|8494.86|62.85|2894.43|0.00|935.21|6858.24|3658.69| +2451770|36400|1678|44305|1652683|6308|3486|44305|1652683|6308|3486|50|2|3956|7|262.92|0.00|262.92|37.37|283.15|65.73|161.69|35.50|320.52| +2452043|55280|1339|88953|876593|1033|39026|88953|876593|1033|39026|2|14|3957|3|234.63|0.00|234.63|31.70|5.64|105.58|54.20|74.85|37.34| +2451963|70779|4591|61988|534869|3750|5431|61988|534869|3750|5431|43|10|3959|24|40.80|0.00|40.80|49.45|44.40|32.64|6.36|1.80|93.85| +2452019|24596|7286|54797|259627|457|28887|54797|259627|457|28887|8|5|3959|21|861.42|68.91|930.33|95.19|600.39|249.81|91.74|519.87|764.49| +2452703|62174|16773|94969|1062198|1427|1856|94969|1062198|1427|1856|13|32|3960|33|360.69|0.00|360.69|97.24|266.64|331.83|6.34|22.52|363.88| +2451646|71094|550|19375|864465|2548|16046|19375|864465|2548|16046|52|21|3961|21|442.89|31.00|473.89|10.62|153.30|39.86|374.81|28.22|194.92| +2452508|56379|9543|28863|771463|4492|30268|28863|771463|4492|30268|12|17|3962|33|182.82|14.62|197.44|14.11|72.93|73.12|54.85|54.85|101.66| +2452244|42241|8371|18517|102929|4141|22553|18517|102929|4141|22553|49|5|3966|16|13.44|0.26|13.70|34.54|12.48|1.34|11.85|0.25|47.28| +2452293|5814|16577|51655|303899|5390|43634|51655|303899|5390|43634|41|29|3966|32|56.00|4.48|60.48|36.69|79.04|19.04|17.74|19.22|120.21| +2452106|77048|7813|44768|1770697|6226|13755|44768|1770697|6226|13755|5|29|3968|3|13.74|0.68|14.42|32.36|56.73|1.37|10.39|1.98|89.77| +||2584||1586753||||1586753|||||3969||||||||220.30||| +2451732|32873|11761|48703|1188762|2421|15633|48703|1188762|2421|15633|25|7|3970|23|575.46|0.00|575.46|52.98|177.10|460.36|59.85|55.25|230.08| +2451710|78347|425|25932|1894118|4704|42537|25932|1894118|4704|42537|49|27|3970|4|46.32|2.31|48.63|27.59|95.36|37.05|3.52|5.75|125.26| +2451918|30852|4345|68265|1041491|5117|3543|68265|1041491|5117|3543|43|24|3971|31|584.97|52.64|637.61|99.41|690.06|561.57|15.91|7.49|842.11| +2451932|18207|16598|83622|1517731|5887|17596|83622|1517731|5887|17596|26|33|3971|5|1024.10|10.24|1034.34|21.59|12.60|296.98|625.32|101.80|44.43| +2451952|44873|8126|66098|1066508|71|2834|66098|1066508|71|2834|41|6|3971|38|321.86|22.53|344.39|82.17|531.24|151.27|92.11|78.48|635.94| +2451116|39786|12416|18349|1870225|5748|20424|18349|1870225|5748|20424|1|2|3972|77|843.92|16.87|860.79|50.96|545.93|118.14|674.97|50.81|613.76| +2452479|50157|6085|13502|1381855|4453|39779|13502|1381855|4453|39779|37|33|3973|3|63.33|3.79|67.12|75.83|104.31|58.89|0.26|4.18|183.93| +2451299|20691|12710|34608|284816|271|24079|34608|284816|271|24079|28|5|3975|41|229.19|11.45|240.64|61.20|145.14|151.26|69.35|8.58|217.79| +2451841|44125|2132|90681|8382|3236|24537|90681|8382|3236|24537|23|33|3976|12|1356.84|27.13|1383.97|66.55|818.40|678.42|624.14|54.28|912.08| +|67117|10274|||3104|||342846|||35|15|3976|75|||12096.57||||387.56||7783.06| +2452431|54235|5856|65816|29860|2257|21101|65816|29860|2257|21101|24|29|3977|18|767.88|61.43|829.31|42.30|489.78|560.55|64.27|143.06|593.51| +2451280||10870|56173|1748693|60|12980|56173|||12980|13|27|3978|44||46.36|2364.72|43.92|2822.60||630.59|945.90|| +2451069|67355|8731|22373|1492720|4314|2261|22373|1492720|4314|2261|43|5|3978|81|315.90|28.43|344.33|40.15|402.57|110.56|12.32|193.02|471.15| +2452791|72038|501|7364|262077|2390|16303|7364|262077|2390|16303|60|33|3979|2|89.26|6.24|95.50|29.88|0.00|58.01|13.43|17.82|36.12| +2452811|23656|3489|37449|41506|1116|22326|37449|41506|1116|22326|27|12|3979|26|540.54|32.43|572.97|76.41|788.32|378.37|72.97|89.20|897.16| +2452605|71625|17089|78688|737344||||737344|||9|25|3979||107.84|||99.34|103.52|||48.69|209.33| +2451394|30471|15824|6804|1254850|143|17389|6804|1254850|143|17389|1|18|3980|1|22.57|2.03|24.60|57.40|55.84|18.95|2.78|0.84|115.27| +2452395|48764|12681|63930|376928|3973|41371|63930|376928|3973|41371|57|26|3982|70|9927.40|198.54|10125.94|8.48|1085.70|6552.08|3274.06|101.26|1292.72| +2451799|68590|11756|14713|298222|2531|46440|14713|298222|2531|46440|47|12|3983|11|331.87|26.54|358.41|20.20|30.58|225.67|95.58|10.62|77.32| +2451923|35314|260|79017|1334707|3197|487|79017|1334707|3197|487|11|20|3983|3|156.87|12.54|169.41|17.58|61.08|10.98|30.63|115.26|91.20| +||15577|||1347||44928|708514|1347||||3983||||299.55|95.85||||8.25|274.76| +2451475|1128|2290|73703|1369206|3462|34565|73703|1369206|3462|34565|14|19|3984|29|4437.00|177.48|4614.48|11.85|1321.53|3948.93|463.66|24.41|1510.86| +2451638|60973|10828|72094|352796|1145|21894|72094|352796|1145|21894|1|19|3984|15|713.40|49.93|763.33|83.56|1562.70|178.35|101.65|433.40|1696.19| +2451594|60869|1132|37303|1722528|106|4084|37303|1722528|106|4084|25|16|3986|33|2379.30|190.34|2569.64|70.63|903.87|1546.54|782.79|49.97|1164.84| +2451495|19492|32|14394|778058|569|48233|14394|778058|569|48233|52|1|3986|44|583.44|52.50|635.94|22.01|448.80|583.44|0.00|0.00|523.31| +2451272|60987|9829|14963|584535|4507|21279|14963|584535|4507|21279|8|26|3987|12|2109.60|168.76|2278.36|94.32|727.44|295.34|1052.27|761.99|990.52| +2451264|42012|5551|82861|544045|4429|4766|82861|544045|4429|4766|22|28|3987|20|37.20|2.23|39.43|94.53|24.80|26.04|5.02|6.14|121.56| +2451374|80380|10603|4132|1524646|3113|33980|4132|1524646|3113|33980|32|28|3987|13|372.97|7.45|380.42|44.77|566.41|234.97|22.08|115.92|618.63| +2451305|63093|1621|99525|426406|6087|30168|99525|426406|6087|30168|20|22|3987|7|83.51|7.51|91.02|51.55|48.02|52.61|28.42|2.48|107.08| +2451262|67610|13904|11257|1067843|6029|34968|11257|1067843|6029|34968|13|29|3987|28|4089.40|81.78|4171.18|42.28|1645.56|2821.68|976.14|291.58|1769.62| +2451586|61902|4297|41628|1008899|617|31488|41628|1008899|617|31488|22|8|3988|2|75.06|1.50|76.56|66.30|2.60|31.52|24.38|19.16|70.40| +2452533|68781|13759|23564|703045|2526|33207|23564|703045|2526|33207|9|17|3991|20|415.80|16.63|432.43|93.29|207.80|395.01|5.82|14.97|317.72| +2451619|58752|7106|71914|1174374|1634|30494|71914|1174374|1634|30494|52|12|3992|27|5571.72|334.30|5906.02|2.14|3261.33|5181.69|156.01|234.02|3597.77| +2451821|30743|17350|59161|281223|6132|3041|59161|281223|6132|3041|32|30|3992|35|649.95|19.49|669.44|71.85|258.65|116.99|474.33|58.63|349.99| +2451123||326||||||1686980|6831|31835|4|31|3993|31||||71.64||1498.81|1305.84||| +2452613|69126|12057|74821|1687601|739|12811|74821|1687601|739|12811|21|29|3995|1|41.87|2.09|43.96|35.98|22.54|11.72|27.13|3.02|60.61| +2451466|75725|6362|41419|974053|2687|14121|41419|974053|2687|14121|32|13|3996|8|677.36|6.77|684.13|9.52|288.88|243.84|26.01|407.51|305.17| +2451349|74780|4552|54186|215434|291|13910|54186|215434|291|13910|10|7|3996|18|54.72|1.64|56.36|28.95|241.02|38.30|15.59|0.83|271.61| +2451265|47601|13166|81249|411210|2418|11179|81249|411210|2418|11179|2|11|3997|19|378.86|18.94|397.80|78.20|113.62|98.50|78.50|201.86|210.76| +|44041|3862||1184137|5407|4324|467||5407|4324||28|3998||35.84|0.00|||116.64|||0.00|| +2452614|65125|14778|98649|1475446|1228|25433|98649|1475446|1228|25433|7|9|4000|5|296.80|0.00|296.80|64.28|179.00|92.00|110.59|94.21|243.28| +2452117|60040|10910|60331|254305|4279|15500|60331|254305|4279|15500|13|14|4001|8|537.36|48.36|585.72|23.97|32.08|32.24|343.48|161.64|104.41| +2451885|15629|6623|49589|60067|5894|40703|49589|60067|5894|40703|25|26|4001|70|434.00|13.02|447.02|3.41|140.70|156.24|261.09|16.67|157.13| +2452652|30434|2784|68670|1637343|4416|23136|68670|1637343|4416|23136|39|26|4002|19|600.40|42.02|642.42|43.00|952.47|144.09|77.57|378.74|1037.49| +2452672|59968|2245|57281|1460281|1234|39034|57281|1460281|1234|39034|24|18|4002|4|144.04|4.32|148.36|93.56|93.20|46.09|78.36|19.59|191.08| +2452610|18144|9036|54374|513180|2386|19992|54374|513180|2386|19992|21|3|4002|6|360.78|14.43|375.21|82.56|56.94|61.33|5.98|293.47|153.93| +2451711|83774|17708|34667|1678102|3137|14572|34667|1678102|3137|14572|7|24|4003|10|358.10|28.64|386.74|40.24|232.00|243.50|56.15|58.45|300.88| +2452177|49484|10519|3810|142225|2687|5902|3810|142225|2687|5902|27|31|4007|19|881.98|35.27|917.25|13.34|349.03|220.49|535.80|125.69|397.64| +2452189|74851|5773|63024|303816|6655|49847|63024|303816|6655|49847|23|7|4007|56|2294.88|22.94|2317.82|94.10|1233.68|114.74|1308.08|872.06|1350.72| +2452780|68814|1260|36002|1175309|216|8323|36002|1175309|216|8323|33|17|4008|41|242.72|19.41|262.13|17.99|15.99|242.72|0.00|0.00|53.39| +2452822|78019|15306|58676|301194|3265|14583|58676|301194|3265|14583|42|25|4010|41|578.10|52.02|630.12|51.86|72.16|450.91|71.22|55.97|176.04| +2452838|42478|17299|34900|613073|5293|47066|34900|613073|5293|47066|12|25|4011|2|251.10|17.57|268.67|81.33|92.36|47.70|71.19|132.21|191.26| +2452370|62413|8223|33838|202016|1394|6772|33838|202016|1394|6772|47|17|4013|1|25.95|2.33|28.28|53.11|14.09|8.04|16.47|1.44|69.53| +2450962|72679|10712|17030|416194|4511|18475|17030|416194|4511|18475|52|21|4014|69|444.36|13.33|457.69|13.12|192.51|391.03|28.79|24.54|218.96| +2450856|57460|1861|91240|1866965|4737|10509|91240|1866965|4737|10509|16|25|4014|12|944.16|37.76|981.92|26.69|62.88|75.53|208.47|660.16|127.33| +2451132|68669|17912|69630|1257168|5859|22997|69630|1257168|5859|22997|55|22|4014|13|452.40|4.52|456.92|89.51|694.72|366.44|77.36|8.60|788.75| +2451874|67867|10568|48237|1240300|1163|16826|48237|1240300|1163|16826|2|29|4015|7|616.35|43.14|659.49|44.10|343.35|61.63|388.30|166.42|430.59| +2451807|42927|9536|8656|504968|6770|18514|8656|504968|6770|18514|47|2|4015|87|713.40|49.93|763.33|51.91|46.98|14.26|321.60|377.54|148.82| +||11672||657477||25770|58354||247||49|10|4016|9||35.43||||159.44|123.47||| +2451800|57665|6077|60792|866712|6679|8999|60792|866712|6679|8999|50|28|4017|57|4422.06|265.32|4687.38|24.14|429.78|3979.85|159.19|283.02|719.24| +2451645|55394|1879|11390|1694495|2047|35952|11390|1694495|2047|35952|46|1|4018|4|715.92|50.11|766.03|12.47|266.56|472.50|63.28|180.14|329.14| +2451288|21456|2842|14102|332776|6844|43214|14102|332776|6844|43214|16|23|4019|6|848.58|33.94|882.52|76.45|365.76|8.48|243.62|596.48|476.15| +2452743|26307|5553|64844|1319045|2046|41894|64844|1319045|2046|41894|39|6|4021|14|1895.04|170.55|2065.59|96.19|800.94|644.31|462.77|787.96|1067.68| +2451346|59308|8390|37917|491497|3477|35457|37917|491497|3477|35457|40|12|4022|9|579.60|52.16|631.76|51.28|289.80|75.34|494.17|10.09|393.24| +2451408|49649|11800|76131|189611|4338|41818|76131|189611|4338|41818|7|14|4022|7|999.81|69.98|1069.79|69.07|344.75|579.88|109.18|310.75|483.80| +2451499|67848|8989|34622|18524|6426|33928|34622|18524|6426|33928|8|18|4022|2|142.30|11.38|153.68|75.33|82.38|135.18|2.20|4.92|169.09| +2451362|73800|7816|53395|1011930|4065|40553|53395|1011930|4065|40553|4|4|4022|1|92.80|1.85|94.65|43.40|54.13|14.84|60.80|17.16|99.38| +2452849|49000|14811|66550|489332|5714|6096|66550|489332|5714|6096|43|35|4023|12|635.52|0.00|635.52|5.36|301.68|482.99|32.03|120.50|307.04| +2452818|32796|14787|77355|437615|6675|12135|77355|437615|6675|12135|25|25|4023|5|122.00|2.44|124.44|86.95|49.80|81.74|32.20|8.06|139.19| +2451925|41246|10987|79362|671372|2229|239|79362|671372|2229|239|44|10|4024|7|68.11|4.08|72.19|56.84|74.34|40.86|24.25|3.00|135.26| +2451089|56094|4534|86837|782513|4142|2618|86837|782513|4142|2618|58|13|4025|19|2139.02|128.34|2267.36|6.43|115.52|513.36|975.39|650.27|250.29| +2451937|71539|8927|52390|771965|6387|17162|52390|771965|6387|17162|11|6|4026|31|1661.91|116.33|1778.24|85.28|224.44|880.81|492.09|289.01|426.05| +2452067|38752|9896|78185|804864|3990|41069|78185|804864|3990|41069|49|26|4026|37|1636.14|16.36|1652.50|88.94|363.34|638.09|778.47|219.58|468.64| +2451492|35023|10564|40252|1220822|2545|28715|40252|1220822|2545|28715|26|4|4027|28|370.16|11.10|381.26|24.66|479.36|22.20|247.05|100.91|515.12| +2451420||6088|25791||2311|||233651||24571|10||4027|15|1531.95|30.63|1562.58||306.30|||292.30|389.75| +2452227|73735|15407|29975|106743|1464|19032|29975|106743|1464|19032|43|6|4029|8|252.08|12.60|264.68|84.70|592.40|110.91|66.34|74.83|689.70| +2451851|80080|12206|77053|1098092|6331|13370|77053|1098092|6331|13370|41|34|4033|14|468.02|9.36|477.38|29.10|448.56|154.44|206.96|106.62|487.02| +2451900|12969|1925|56928|1911960|2545|17977|56928|1911960|2545|17977|29|1|4033|26|1585.22|47.55|1632.77|49.83|964.86|396.30|867.91|321.01|1062.24| +2452071|31590|16829|49733|1268895|6104|48913|49733|1268895|6104|48913|17|28|4033|9|384.57|15.38|399.95|90.30|76.86|34.61|335.96|14.00|182.54| +2451973|50269|1265|91766|151333|4500|31267|91766|151333|4500|31267|44|7|4033|35|911.40|82.02|993.42|38.02|151.90|45.57|225.11|640.72|271.94| +2452207|83009|5625|32972|||||1454645||||17|4034||9785.47|880.69||6.63|2266.07||1453.14||| +2452761|45895|16650|34626|1915264|4349|45787|34626|1915264|4349|45787|39|1|4035|22|3132.80|0.00|3132.80|49.84|1608.64|2349.60|313.28|469.92|1658.48| +2451899|68099|11737|96322|63824|1279|47981|96322|63824|1279|47981|13|12|4036|13|852.93|34.11|887.04|64.53|133.77|631.16|33.26|188.51|232.41| +2452875|76835|5952|96686|892449|5459|14422|96686|892449|5459|14422|27|1|4038|27|137.43|2.74|140.17|69.32|1696.68|30.23|53.60|53.60|1768.74| +2451563|12216|13666|81157|1681092|4227|562|81157|1681092|4227|562|19|4|4040|54|2340.36|46.80|2387.16|11.38|3470.58|1825.48|509.73|5.15|3528.76| +2451318|34651|17641|68210|1466237|4264|41138|68210|1466237|4264|41138|40|12|4040|34|1224.68|85.72|1310.40|68.76|1224.68|306.17|716.43|202.08|1379.16| +2452034|26870|8072|837|1064392|5547|11102|837|1064392|5547|11102|41|32|4041|1|3.89|0.31|4.20|14.37|20.27|1.01|2.64|0.24|34.95| +2451961|73316|4999|29643|624285|6927|26769|29643|624285|6927|26769|59|29|4041|2|17.32|0.69|18.01|41.77|30.96|1.90|2.31|13.11|73.42| +2452301|35719|17243|51681|300452|7176|15547|51681|300452|7176|15547|55|27|4043|27|1973.70|98.68|2072.38|50.80|714.69|78.94|1800.02|94.74|864.17| +2452457|72973|14925|77538|115212|4886|5360|77538|115212|4886|5360|15|4|4044|8|623.04|43.61|666.65|65.95|220.24|485.97|9.59|127.48|329.80| +2452435|79524|7793|7488|994554|3667|36333|7488|994554|3667|36333|37|15|4044|35|50.05|0.50|50.55|67.48|50.05|40.04|3.10|6.91|118.03| +2451775|55385|5482|6099|488191|3883|22328|6099|488191|3883|22328|32|8|4045|3|376.92|22.61|399.53|8.71|196.47|342.99|22.39|11.54|227.79| +2451523||5941||1510605||22116|||||4|12|4045|24|||||1337.76|1270.87|12.04||| +||7406|39556|||4438|39556|1595850|3344||10||4046||||376.83|||238.18|||| +2451493|58408|11894|9872|1280208|3344|1727|9872|1280208|3344|1727|26|32|4046|6|35.64|1.42|37.06|17.28|20.34|35.64|0.00|0.00|39.04| +2451494|62612|13411|57657|1549141|6225|47756|57657|1549141|6225|47756|32|7|4046|8|130.24|5.20|135.44|36.21|10.64|89.86|15.74|24.64|52.05| +2451341|54161|4312|44067|707609|2351|29893|44067|707609|2351|29893|52|11|4046|4|158.04|7.90|165.94|5.90|75.80|139.07|9.67|9.30|89.60| +2452192|19874|11457|9744|1580033|3922|25011|9744|1580033|3922|25011|59|32|4047|32|632.64|44.28|676.92|75.10|434.88|158.16|66.42|408.06|554.26| +2452145|18299|14917|99680|634589|3098|1207|99680|634589|3098|1207|59|4|4047|23|71.53|3.57|75.10|46.89|18.63|65.09|2.06|4.38|69.09| +2452467|28702|1657|3661|433619|2330|2688|3661|433619|2330|2688|43|3|4048|37|1685.72|16.85|1702.57|33.35|775.89|438.28|1110.22|137.22|826.09| +2452150|74226|12679|86035|1815097|4512|46374|86035|1815097|4512|46374|15|12|4051|18|2572.92|231.56|2804.48|81.43|605.34|1466.56|1106.36|0.00|918.33| +|64902|11341||686732|1229|||686732|1229|24879|25||4052|4|449.24|31.44|480.68||95.84|179.69|||129.69| +2451343|72827|1492|50963|1414883|403|27876|50963|1414883|403|27876|56|19|4052|7|974.75|0.00|974.75|30.15|157.50|682.32|35.09|257.34|187.65| +2451291|6826|4570|60905|800602|1899|5196|60905|800602|1899|5196|38|3|4052|9|355.14|7.10|362.24|10.51|233.64|301.86|52.74|0.54|251.25| +2451570|63836|13898|9354|1699100|1999|12945|9354|1699100|1999|12945|43|32|4054|8|177.12|1.77|178.89|43.43|159.36|92.10|12.75|72.27|204.56| +2452836|55675|17676|30255|873060|6265|4223|30255|873060|6265|4223|57|19|4056|72|3632.40|36.32|3668.72|36.19|1393.92|1489.28|685.79|1457.33|1466.43| +2452693|31727|7044|1459|1373177|1342|49460|1459|1373177|1342|49460|37|22|4056|2|18.70|0.56|19.26|28.38|0.00|9.35|9.25|0.10|28.94| +2452501|67235|4887|21579|56376|1126|38723|21579|56376|1126|38723|35|30|4057|40|1656.80|66.27|1723.07|55.83|184.00|1540.82|26.67|89.31|306.10| +2451275|69503|12253|51464|600370|1276|44582|51464|600370|1276|44582|25|10|4059|3|42.12|0.42|42.54|26.92|54.93|7.58|15.88|18.66|82.27| +2451211|45438|13693|34376|184549|4322|46895|34376|184549|4322|46895|55|7|4060|13|227.11|6.81|233.92|40.23|321.75|177.14|7.99|41.98|368.79| +2451322|49262|331|27871|1094653|5882|27111|27871|1094653|5882|27111|25|9|4060|1|204.30|4.08|208.38|90.63|37.14|194.08|1.22|9.00|131.85| +2451139|44407|8642|10340|1856987|6834|18681|10340|1856987|6834|18681|44|2|4060|32|2058.24|185.24|2243.48|38.95|467.52|1955.32|58.66|44.26|691.71| +2452613|63195|11949|82541|1615548|2602|14009|82541|1615548|2602|14009|55|28|4064|21|151.83|4.55|156.38|92.71|208.95|22.77|27.10|101.96|306.21| +2452732|76532|12877|61804|935446|2335|37177|61804|935446|2335|37177|12|5|4064|6|85.32|0.00|85.32|7.72|77.58|49.48|22.93|12.91|85.30| +2451554|23211|5950|64110|135077|803|48188|64110|135077|803|48188|56|3|4065|31|866.45|8.66|875.11|55.36|254.82|242.60|393.02|230.83|318.84| +2452227|74865|49|16384|1625851|918|1232|16384|1625851|918|1232|15|14|4066|67|678.04|54.24|732.28|72.19|775.19|535.65|25.63|116.76|901.62| +2452764|52898|11109|72133|452614|5895|36329|72133|452614|5895|36329|27|25|4067|6|73.74|5.89|79.63|5.53|36.42|8.11|22.31|43.32|47.84| +2452455|30552|4663|85898|423899|7147|19948|85898|423899|7147|19948|55|31|4070|16|169.12|11.83|180.95|28.79|413.44|155.59|7.30|6.23|454.06| +2451810|67306|2942|71067|1758147|427|14707|71067|1758147|427|14707|31|29|4071|7|66.43|5.31|71.74|16.09|0.70|51.15|7.33|7.95|22.10| +2451257|53486|5836|76846|605018|4390|6162|76846|605018|4390|6162|49|32|4072|2|12.88|0.00|12.88|71.13|0.22|5.15|0.92|6.81|71.35| +2451323|36271|14047|4331|938526|5874|27622|4331|938526|5874|27622|2|16|4072|3|132.12|9.24|141.36|14.82|92.19|99.09|26.75|6.28|116.25| +2451372|30531|9152|58716|441121|6373|19178|58716|441121|6373|19178|13|25|4072|9|269.73|2.69|272.42|56.94|311.22|169.92|33.93|65.88|370.85| +2451361|44343|16240|3139|1277864|5036|39008|3139|1277864|5036|39008|37|18|4072|7|100.10|2.00|102.10|35.43|650.93|27.02|14.61|58.47|688.36| +2450983|68570|16238|4023|119590|3510|5830|4023|119590|3510|5830|4|14|4074|17|23.80|1.42|25.22|91.80|64.94|12.37|4.91|6.52|158.16| +2450931|33139|15193|5129|309709|4689|17669|5129|309709|4689|17669|44|16|4074|40|531.20|47.80|579.00|75.82|2174.00|58.43|316.75|156.02|2297.62| +2452458|28035|36|83858|140890|4675|34907|83858|140890|4675|34907|25|30|4076|62|4772.14|381.77|5153.91|96.30|1105.46|3865.43|616.56|290.15|1583.53| +2452051|70783|15195|90398|1409262|6973|13884|90398|1409262|6973|13884|37|13|4077|83|2372.97|118.64|2491.61|53.58|2372.97|261.02|1393.88|718.07|2545.19| +2452210|17535|16055|84360|1829891|2622|40553|84360|1829891|2622|40553|37|9|4077|19|695.78|62.62|758.40|30.51|389.50|619.24|31.38|45.16|482.63| +2451285|63117|7861|55082|1632018|4601|5537|55082|1632018|4601|5537|2|32|4078|79|13016.04|650.80|13666.84|50.03|670.71|3644.49|8434.39|937.16|1371.54| +2452638|45114|10389|68990|470202|2967|24132|68990|470202|2967|24132|19|19|4079|18|182.34|9.11|191.45|58.77|107.64|92.99|67.90|21.45|175.52| +2452606|44935|3733|63967|235312|2657|22402|63967|235312|2657|22402|36|29|4079|26|689.00|48.23|737.23|7.59|1469.78|137.80|99.21|451.99|1525.60| +2451936|3681|3287|2206|822197|3435|13826|2206|822197|3435|13826|56|21|4080|20|1785.20|160.66|1945.86|85.31|1009.00|1106.82|210.29|468.09|1254.97| +2451968|50661|16589|60040|202447|1207|21214|60040|202447|1207|21214|35|18|4080|21|1140.51|45.62|1186.13|89.03|115.29|467.60|430.66|242.25|249.94| +2450928|74222|4052|83472|226947|4022|37097|83472|226947|4022|37097|38|11|4081|19|2328.07|162.96|2491.03|45.83|201.02|721.70|1365.41|240.96|409.81| +2450996|43539|10930|3357|1711397|5351|5615|3357|1711397|5351|5615|55|28|4081|72|2383.20|95.32|2478.52|39.78|2019.60|1072.44|26.21|1284.55|2154.70| +2450996|14135|17320|85828|1565673|3471|33716|85828|1565673|3471|33716|20|10|4081|12|340.20|0.00|340.20|74.06|86.88|231.33|102.33|6.54|160.94| +2451533|81478|13376|63826|737389|6952|21315|63826|737389|6952|21315|25|29|4083|5|1.10|0.08|1.18|30.86|0.35|0.30|0.35|0.45|31.29| +2451448|79104|5995|82360|1197161|1209|48010|82360|1197161|1209|48010|50|20|4083|77|4046.35|80.92|4127.27|80.35|2386.23|3601.25|231.45|213.65|2547.50| +2451934|72343|1975|41353|1871118|883|34648|41353|1871118|883|34648|23|15|4085|1|197.86|7.91|205.77|41.43|39.08|73.20|37.39|87.27|88.42| +2450952|75018|14696|42958|1613964|1323|38682|42958|1613964|1323|38682|37|13|4087|18|252.00|17.64|269.64|73.31|462.06|35.28|69.35|147.37|553.01| +2451090|66060|12187|55104|180627|550|44520|55104|180627|550|44520|16|2|4087|27|75.60|6.80|82.40|91.32|5.67|26.46|5.40|43.74|103.79| +2450899|55332|9559|42912|33628|3544|7907|42912|33628|3544|7907|8|10|4087|90|124.20|4.96|129.16|63.77|2235.60|99.36|4.22|20.62|2304.33| +2451250|44861|10513|78127|277781|4988|21042|78127|277781|4988|21042|19|32|4088|1|105.87|3.17|109.04|19.63|65.63|102.69|2.38|0.80|88.43| +2452001|71135|17753|66622|1837601|227|2453|66622|1837601|227|2453|17|5|4089|34|2568.36|25.68|2594.04|33.36|40.46|1078.71|819.30|670.35|99.50| +2452027|15594|9277|53941|1915004|67|18283|53941|1915004|67|18283|41|24|4089|75|2924.25|58.48|2982.73|46.26|797.25|1666.82|50.29|1207.14|901.99| +2451906|54640|11227|70489|1254301|4066|4231|70489|1254301|4066|4231|2|35|4089|33|3747.48|299.79|4047.27|77.02|115.83|562.12|1401.55|1783.81|492.64| +2451846|79183|11341|11180|748595|2810|45425|11180|748595|2810|45425|37|32|4090|1|34.73|2.77|37.50|33.03|30.87|12.85|3.71|18.17|66.67| +2451603|57280|17341|55746|1236414|5435|19703|55746|1236414|5435|19703|53|29|4090|4|399.00|3.99|402.99|77.12|117.88|199.50|67.83|131.67|198.99| +2451646|78730|4157|30924|1512379|3787|23242|30924|1512379|3787|23242|25|30|4090|71|166.14|1.66|167.80|47.05|604.21|103.00|16.41|46.73|652.92| +2451736|43618|8090|42994|663003|2212|37068|42994|663003|2212|37068|56|1|4090|31|1865.58|37.31|1902.89|86.30|2343.91|37.31|146.26|1682.01|2467.52| +2451662|32488|5747|59124|100435|6099|5810|59124|100435|6099|5810|50|32|4090|16|2783.36|222.66|3006.02|59.83|0.00|501.00|1574.82|707.54|282.49| +2452600|30913|13681|18000|404253|2506|30759|18000|404253|2506|30759|45|32|4091|35|517.30|31.03|548.33|74.79|74.90|444.87|68.80|3.63|180.72| +2452653|52978|7890|85351|70480|2809|33138|85351|70480|2809|33138|13|5|4091|26|659.88|39.59|699.47|6.45|534.82|184.76|327.83|147.29|580.86| +2452424|75877|12947|89429|947374|6119|17664|89429|947374|6119|17664|9|29|4094|35|1331.05|13.31|1344.36|26.57|178.50|1291.11|23.96|15.98|218.38| +2452636|67892|2323|75331|892035|2692|48596|75331|892035|2692|48596|9|14|4096|31|1956.72|78.26|2034.98|19.76|1002.23|1565.37|250.46|140.89|1100.25| +2452488|71585|6837|63271|919353|235|35517|63271|919353|235|35517|25|24|4096|71|171.11|6.84|177.95|67.49|1370.30|78.71|51.74|40.66|1444.63| +2451106|63349|1333|4563|1426959|1355|34904|4563|1426959|1355|34904|52|28|4097|18|65.34|0.65|65.99|46.79|145.44|60.76|2.83|1.75|192.88| +||114|453|84821|||453|84821||36404||30|4098|42|1322.16|0.00|1322.16|93.03||594.97||501.77|| +2452784|40982|13422|37896|905137|1102|13795|37896|905137|1102|13795|48|35|4098|65|914.55|0.00|914.55|45.73|1463.80|759.07|68.41|87.07|1509.53| +2452725|44765|10105|74055|1893152|4696|29440|74055|1893152|4696|29440|48|2|4099|1|7.84|0.47|8.31|58.05|5.60|3.52|1.46|2.86|64.12| +2451238|27891|10550|45935|||44399||754772||44399|||4101|||||67.15||38.12|16.01||| +2451116|28780|7918|20846|111979|712|25045|20846|111979|712|25045|56|20|4102|12|79.56|7.16|86.72|20.86|901.80|38.98|34.89|5.69|929.82| +2452684|39680|14556|51856|537897|5985|15226|51856|537897|5985|15226|55|23|4104|34|2358.58|23.58|2382.16|71.35|1164.84|306.61|923.38|1128.59|1259.77| +2451750|47226|259|55167|1722066|7012|1511|55167|1722066|7012|1511|2|8|4105|4|182.52|1.82|184.34|51.82|298.68|82.13|67.26|33.13|352.32| +2452061|74181|15854||1789500|1086||24761|||43825||28|4106|9|169.20||181.04|||52.45|||255.72| +2452178|65861|15307|37662|1622147||||1622147|3785||7||4107||702.05|||96.29||||235.20|780.23| +2452022|80456|6491|44089|424884|3726|44921|44089|424884|3726|44921|7|12|4107|71|4936.63|49.36|4985.99|64.84|2829.35|3011.34|173.27|1752.02|2943.55| +2451359|57428|17887|41652|1688856|6907|28111|41652|1688856|6907|28111|52|29|4109|56|4910.08|392.80|5302.88|93.70|1540.56|2847.84|1505.43|556.81|2027.06| +2451657|72305|2905|65384|1662799|1814|7229|65384|1662799|1814|7229|25|6|4110|2|44.24|3.53|47.77|21.09|181.42|27.87|9.49|6.88|206.04| +2452379|72745|8007|95554|901918|6235|23352|95554|901918|6235|23352|3|3|4111|4|164.44|0.00|164.44|6.44|146.16|26.31|87.02|51.11|152.60| +2452499|43629|11347|3780|1801945|5357|9969|3780|1801945|5357|9969|45|12|4113|41|123.82|8.66|132.48|1.69|127.10|110.19|12.53|1.10|137.45| +2452376|9040|9995|70075|1904224|2726|20875|70075|1904224|2726|20875|59|22|4113|7|156.94|9.41|166.35|49.48|25.55|69.05|30.76|57.13|84.44| +2452013|63643|15523|84836|595338|5040|13189|84836|595338|5040|13189|19|5|4114|21|131.67|10.53|142.20|61.90|1449.21|61.88|56.52|13.27|1521.64| +2452087|17406|14381|18374|565655|3336|41370|18374|565655|3336|41370|55|1|4114|9|0.00|0.00|0.00|27.87|636.93|0.00|0.00|0.00|664.80| +2452023|15575|359|51955|147339|4770|21354|51955|147339|4770|21354|47|2|4117|27|833.76|41.68|875.44|77.91|306.99|400.20|190.76|242.80|426.58| +2452051|43241|9275|37771|535300|5629|23558|37771|535300|5629|23558|19|17|4117|9|678.87|13.57|692.44|13.91|270.09|244.39|60.82|373.66|297.57| +2452056|49940|4907|9283|168459|6727|25035|9283|168459|6727|25035|37|17|4117|62|171.74|3.43|175.17|28.68|3011.96|39.50|112.40|19.84|3044.07| +||3217||864005|5643|26060||||26060||19|4118||1137.30|56.86||||||10.24|| +2452183|28148|839|37855|1684107|6712|8564|37855|1684107|6712|8564|2|34|4118|53|3875.36|310.02|4185.38|95.18|1328.71|3642.83|69.75|162.78|1733.91| +2451680|49574|3572|98350|885049|6148|41247|98350|885049|6148|41247|22|28|4119|8|911.28|18.22|929.50|42.26|284.72|510.31|12.02|388.95|345.20| +2451375|33900|12764|12470|1333879|441|33850|12470|1333879|441|33850|13|2|4123|50|213.00|17.04|230.04|11.36|250.00|10.65|62.72|139.63|278.40| +2451267|83318|4306|83620|1517685|6950|20662|83620|1517685|6950|20662|37|26|4126|14|226.24|4.52|230.76|27.94|16.80|42.98|130.11|53.15|49.26| +2451462|30067|12134|35182|1663680|531|45829|35182|1663680|531|45829|43|15|4126|10|275.60|19.29|294.89|41.12|224.60|71.65|20.39|183.56|285.01| +2451469|26908|8720|87514|1206060|1053|37599|87514|1206060|1053|37599|8|11|4126|5|200.70|16.05|216.75|59.14|112.40|150.52|7.52|42.66|187.59| +2451781|85070|17398|27810|235418|2226|19834|27810|235418|2226|19834|20|35|4127|45|1934.55|154.76|2089.31|86.03|628.65|1354.18|504.92|75.45|869.44| +2452720|46444|4611|85077|1006174|5671|44368|85077|1006174|5671|44368|3|28|4128|56|4649.68|418.47|5068.15|68.15|1599.92|3068.78|347.79|1233.11|2086.54| +2451992|5765|15289|75589|255603|2017|11631|75589|255603|2017|11631|13|2|4129|39|763.23|68.69|831.92|90.91|145.08|625.84|127.77|9.62|304.68| +2452102|58989|6181|28484|1292196|261|29926|28484|1292196|261|29926|57|12|4129|11|31.79|0.63|32.42|72.72|651.75|9.85|21.06|0.88|725.10| +2451631|77642|8965|49617|120248|2526|21557|49617|120248|2526|21557|46|17|4131|2|86.62|3.46|90.08|97.85|60.48|41.57|27.03|18.02|161.79| +2451587|42782|14269|37255|1437287|6919|104|37255|1437287|6919|104|49|29|4131|13|233.74|11.68|245.42|60.12|81.77|28.04|125.47|80.23|153.57| +2451611|74791|236|54754|1526694|587|5280|54754|1526694|587|5280|58|23|4131|10|210.50|4.21|214.71|58.41|304.10|202.08|8.33|0.09|366.72| +2452626|36386|6501|72571|1578103|2835|11214|72571|1578103|2835|11214|15|25|4132|7|396.06|7.92|403.98|44.73|365.61|43.56|137.47|215.03|418.26| +2452849|47452|9625|40616|1581244|5072|31700|40616|1581244|5072|31700|60|29|4132|62|2745.98|0.00|2745.98|47.14|1269.76|1757.42|405.30|583.26|1316.90| +2452065|37731|3289|25877|532516|6355|22594|25877|532516|6355|22594|38|14|4133|91|1313.13|118.18|1431.31|65.81|3677.31|118.18|322.63|872.32|3861.30| +2452473|62278|13667|63231|867262|2493|19324|63231|867262|2493|19324|51|33|4134|8|282.96|22.63|305.59|20.28|88.88|274.47|3.14|5.35|131.79| +2451546|75069|7807|36887|187658|29|43200|36887|187658|29|43200|14|11|4135|51|2645.88|26.45|2672.33|48.36|32.64|2222.53|194.74|228.61|107.45| +2451707|58187|9385|38242|582061|2680|48946|38242|582061|2680|48946|28|6|4135|7|586.60|52.79|639.39|88.27|117.32|199.44|301.98|85.18|258.38| +2451969|59942|6641|7398|1493997|949|1948|7398|1493997|949|1948|5|33|4136|14|718.34|57.46|775.80|25.98|169.82|294.51|241.58|182.25|253.26| +2451333|57218|14782|2675|1200570|3618|5066|2675|1200570|3618|5066|49|24|4137|5|128.65|10.29|138.94|79.68|314.55|79.76|3.42|45.47|404.52| +2451889|47892|12827|19505|899590|2522|17007|19505|899590|2522|17007|44|18|4139|5|95.35|2.86|98.21|18.06|209.80|35.27|57.07|3.01|230.72| +2451877|64797|14906|62605|378109|1166|5470|62605|378109|1166|5470|5|12|4142|11|649.11|6.49|655.60|85.07|165.66|0.00|285.60|363.51|257.22| +2451714|77593|4364|26001|303149|4650|23798|26001|303149|4650|23798|16|3|4143|23|227.01|9.08|236.09|47.32|178.02|202.03|17.98|7.00|234.42| +2451655|68231|6841|176|552482|2084|11167|176|552482|2084|11167|19|34|4144|52|6512.48|520.99|7033.47|73.91|342.68|1107.12|5243.19|162.17|937.58| +2452767|26497|9684|44687|1025611|6088|41412|44687|1025611|6088|41412|21|22|4145|55|2545.95|50.91|2596.86|55.16|1709.40|1502.11|104.38|939.46|1815.47| +2451692|39207|15343|71672|1500427|5182|25937|71672|1500427|5182|25937|4|15|4146|2|251.38|5.02|256.40|86.05|179.02|115.63|42.08|93.67|270.09| +2452613|39975|5773|35538|188210|3729|34448|35538|188210|3729|34448|25|23|4147|42|989.52|89.05|1078.57|27.39|679.98|841.09|10.39|138.04|796.42| +2452464|35120|3565|68793|729274|5402|25130|68793|729274|5402|25130|1|29|4147|28|12.88|0.38|13.26|54.33|157.08|2.57|0.10|10.21|211.79| +|36231|10423|67251|1675881||24388||1675881|||12||4148||||499.86||89.76||||156.09| +2451935|35317|17099|78057|1712798|6386|15927|78057|1712798|6386|15927|59|34|4149|7|583.38|23.33|606.71|49.01|80.15|64.17|150.57|368.64|152.49| +2452089|75921|13847|45831|317456|2681|15506|45831|317456|2681|15506|5|25|4149|4|135.56|2.71|138.27|30.64|179.44|112.51|18.90|4.15|212.79| +2451890|18137|17531|62382|1203214|46|33278|62382|1203214|46|33278|44|16|4149|42|1988.70|178.98|2167.68|73.30|506.10|934.68|31.62|1022.40|758.38| +2451898|75222|12719|43364|1661490|5216|16942|43364|1661490|5216|16942|41|26|4149|1|13.07|0.39|13.46|46.09|36.32|5.88|4.45|2.74|82.80| +2452140|49051|11474|32381|1386290|3046|12955|32381|1386290|3046|12955|56|26|4149|22|0.00|0.00|0.00|15.69|372.68|0.00|0.00|0.00|388.37| +2451986|72169|7549|78491|1528297|3350|13156|78491|1528297|3350|13156|37|28|4150|35|2094.75|146.63|2241.38|65.15|1132.25|1173.06|479.27|442.42|1344.03| +2451105|59086|634|45069|933793|1717|29241|45069|933793|1717|29241|56|8|4152|13|54.86|0.00|54.86|42.34|50.05|48.82|2.77|3.27|92.39| +2452329|76486|4043|48915|1265709|6503|42097|48915|1265709|6503|42097|31|23|4158|18|220.50|15.43|235.93|51.68|595.62|92.61|124.05|3.84|662.73| +2452251|64010|16971|34054|1246387|4320|25053|34054|1246387|4320|25053|55|32|4160|16|1425.76|114.06|1539.82|39.62|862.88|1211.89|190.34|23.53|1016.56| +2451550|20234|2380|64137|1509210|1649|21734|64137|1509210|1649|21734|20|13|4161|9|541.44|32.48|573.92|8.96|297.27|16.24|63.02|462.18|338.71| +2451300|70296|7039|24770|160307|3044|22228|24770|160307|3044|22228|38|34|4162|16|138.72|5.54|144.26|2.93|116.80|19.42|108.56|10.74|125.27| +2452392|30114|5899|35676|1900609|5095|24428|35676|1900609|5095|24428|31|10|4163|89|331.08|3.31|334.39|69.21|808.12|251.62|50.05|29.41|880.64| +2451760|50425|14980|89058|567674|5137|2600|89058|567674|5137|2600|25|16|4165|1|63.64|3.18|66.82|9.84|13.19|7.63|40.32|15.69|26.21| +2451424|36279|9166|18928|1259559|637|9668|18928|1259559|637|9668|14|26|4165|31|3755.96|300.47|4056.43|51.87|2414.59|976.54|1111.76|1667.66|2766.93| +2451469|63004|7324|84133|971996|1812|24428|84133|971996|1812|24428|49|25|4165|21|4700.22|329.01|5029.23|10.71|1812.09|3478.16|867.66|354.40|2151.81| +2451958|26068|7049|10510|264016|4251|42807|10510|264016|4251|42807|31|16|4166|25|492.00|4.92|496.92|57.35|250.25|103.32|194.34|194.34|312.52| +2451982|65674|557|67315|363306|5250|28794|67315|363306|5250|28794|47|2|4166|30|1645.50|98.73|1744.23|70.00|860.70|1217.67|376.49|51.34|1029.43| +2451325|50337|5302|92164|272773|4582|23544|92164|272773|4582|23544|2|24|4167|2|5.88|0.47|6.35|55.81|4.98|0.47|2.70|2.71|61.26| +2451761|31015|7646|63548|1811304|1013|4322|63548|1811304|1013|4322|7|5|4170|49|3265.36|163.26|3428.62|28.69|2684.71|2253.09|627.60|384.67|2876.66| +2451826|73816|7129|42974|1282088|402|716|42974|1282088|402|716|13|13|4170|70|2223.90|155.67|2379.57|46.45|1383.90|1289.86|756.57|177.47|1586.02| +2451620|54780|17936|28388|813758|281|40451|28388|813758|281|40451|20|1|4175|70|9391.20|187.82|9579.02|1.24|1633.10|5728.63|988.89|2673.68|1822.16| +2451540|69044|8683|6652|1121606|3760|13114|6652|1121606|3760|13114|25|22|4175|61|7023.54|210.70|7234.24|0.78|6704.51|1896.35|4922.10|205.09|6915.99| +2450989|78495|2330|82117|367738|4964|36733|82117|367738|4964|36733|49|32|4176|1|79.47|5.56|85.03|44.94|15.32|22.25|8.58|48.64|65.82| +||12796|82117|||36733|82117|367738|4964|36733|7||4176||2171.04|43.42|2214.46|||1454.59||444.20|| +2451399|59786|12638|715|116157|4890|23493|715|116157|4890|23493|50|3|4177|15|119.55|7.17|126.72|7.29|624.45|40.64|2.36|76.55|638.91| +2451289|76234|15502|2687|593934|1628|16475|2687|593934|1628|16475|56|33|4177|23|1918.20|115.09|2033.29|65.11|547.86|1630.47|224.42|63.31|728.06| +2452449|59510|9937|26049|249906|5316|12773|26049|249906|5316|12773|1|1|4179|6|215.94|15.11|231.05|91.37|265.02|142.52|4.40|69.02|371.50| +2451325|64530|1970|23327|646515|903|22844|23327|646515|903|22844|2|35|4180|4|591.68|29.58|621.26|28.76|208.80|272.17|230.04|89.47|267.14| +2451197|82684|592|38722|753522|5095|43901|38722|753522|5095|43901|40|2|4182|74|3848.00|192.40|4040.40|29.25|769.60|923.52|1696.19|1228.29|991.25| +2451672|54720|326|59610|1137627|91|24508|59610|1137627|91|24508|37|19|4185|4|228.80|20.59|249.39|51.06|74.92|13.72|53.77|161.31|146.57| +2451731|71324|2140|8830|881747|1511|6028|8830|881747|1511|6028|38|31|4185|59|248.39|7.45|255.84|77.42|2980.68|62.09|80.10|106.20|3065.55| +||14389||||23025|1269|1589172|275|23025|28||4186|7||0.00|||||||| +2451450|72499|15547|62745|906778|764|6750|62745|906778|764|6750|4|28|4186|29|826.50|24.79|851.29|98.51|287.10|173.56|58.76|594.18|410.40| +2451402|79090|17318|13789|1759064|4339|32014|13789|1759064|4339|32014|40|3|4186|36|2980.44|119.21|3099.65|47.04|583.56|1043.15|1549.83|387.46|749.81| +2452358|41825|5767|99521|1634420|72|10971|99521|1634420|72|10971|49|9|4188|4|120.32|3.60|123.92|38.15|84.68|109.49|7.90|2.93|126.43| +2452360|38049|2607|80426|1812937|5448|21559|80426|1812937|5448|21559|41|20|4188|25|2783.25|166.99|2950.24|28.68|1491.00|918.47|540.78|1324.00|1686.67| +2451255|71880|1249|51790|683620|2650|11280|51790|683620|2650|11280|43|28|4189|1|26.46|1.58|28.04|72.13|10.29|3.96|1.57|20.93|84.00| +2451094|80458|13552|82285|1425883|5834|33432|82285|1425883|5834|33432|2|26|4189|71|8838.79|795.49|9634.28|1.52|2945.79|707.10|4228.47|3903.22|3742.80| +2452068|5304|17671|45779|912776|6620|29664|45779|912776|6620|29664|35|32|4190|26|936.78|18.73|955.51|90.44|0.00|28.10|54.52|854.16|109.17| +2452558|59089|6264|96574|1461239|4304|37416|96574|1461239|4304|37416|31|25|4191|11|1255.54|12.55|1268.09|80.88|488.29|590.10|425.88|239.56|581.72| +2452770|67948|13729|1188|702053|871|4536|1188|702053|871|4536|48|35|4192|4|41.72|1.25|42.97|78.16|20.84|22.94|9.95|8.83|100.25| +|75544|16929||180082|5067||697||||||4197|||||36.36|||1.31|0.99|67.58| +2452588|54077|3390|2922|165108|2989|44035|2922|165108|2989|44035|13|6|4198|22|954.14|0.00|954.14|17.50|47.52|190.82|656.45|106.87|65.02| +2452487|60108|3282|94376|1810993|5657|45121|94376|1810993|5657|45121|25|1|4198|75|1026.00|30.78|1056.78|67.14|933.00|287.28|583.58|155.14|1030.92| +2451881|54064|1540|44444|365315|5739|14210|44444|365315|5739|14210|20|35|4199|16|1587.04|95.22|1682.26|23.85|383.68|301.53|925.56|359.95|502.75| +2451560|69292|7214|9011|978353|4048|21790|9011|978353|4048|21790|19|13|4199|34|3505.74|245.40|3751.14|54.87|1752.70|280.45|2773.74|451.55|2052.97| +2452887|71613|10897|27757|1218241|3572|32456|27757|1218241|3572|32456|33|3|4200|30|1679.10|16.79|1695.89|40.71|877.20|1544.77|75.22|59.11|934.70| +2452716||3831|596||6755||596||||||4200|33||60.32|730.55|||361.92|114.07|194.24|| +2452891|20572|10927|27553|1483593|2386|28026|27553|1483593|2386|28026|31|30|4200|1|103.90|1.03|104.93|13.33|32.53|70.65|17.62|15.63|46.89| +2451977|41141|3971|1327|1128000|5073|14099|1327|1128000|5073|14099|11|35|4201|18|1312.02|0.00|1312.02|39.49|629.64|590.40|649.45|72.17|669.13| +2451934|18889|7603|58222|8401|4980|18406|58222|8401|4980|18406|14|10|4204|38|232.56|13.95|246.51|47.80|2094.56|134.88|31.25|66.43|2156.31| +2451774|42070|1435|4401|356953|4705|25210|4401|356953|4705|25210|46|28|4205|5|184.80|7.39|192.19|38.21|141.65|129.36|2.77|52.67|187.25| +2451790|21028|1336|38997|1743552|1707|20262|38997|1743552|1707|20262|2|19|4205|6|30.60|0.00|30.60|49.07|10.92|17.44|6.18|6.98|59.99| +2452833|83321|5071|62770|1047804|5191|18578|62770|1047804|5191|18578|51|17|4206|16|1804.00|72.16|1876.16|7.30|460.16|1569.48|56.28|178.24|539.62| +2452802|38592|8646|10765|331113|5861|49009|10765|331113|5861|49009|49|8|4208|45|2503.80|175.26|2679.06|20.45|595.80|300.45|1806.74|396.61|791.51| +2450950|8593|386|92991|236077|763|12202|92991|236077|763|12202|38|4|4209|53|2848.22|199.37|3047.59|28.19|1740.52|996.87|555.40|1295.95|1968.08| +2451772|53978|1772|26453|803746|3886|10826|26453|803746|3886|10826|55|21|4210|1|63.64|3.81|67.45|28.22|17.43|1.27|16.83|45.54|49.46| +|41568|7393|25545|||18020|25545|||||32|4210|20|684.60||739.36|||||96.26|596.14| +2452789|78943|17757|12911|1370386|1814|13289|12911|1370386|1814|13289|7|5|4211|23|642.16|19.26|661.42|33.96|535.21|597.20|8.09|36.87|588.43| +2452119|4163|10232|49012|748139|4066|40440|49012|748139|4066|40440|7|7|4214|15|1282.05|115.38|1397.43|3.92|433.05|1243.58|11.92|26.55|552.35| +2452553|74235|15510|55696|1498962|4981|34647|55696|1498962|4981|34647|42|7|4215|8|223.84|17.90|241.74|23.10|187.20|123.11|44.32|56.41|228.20| +2452464|4697|15057|52384|1773817|4403|43789|52384|1773817|4403|43789|24|6|4215|56|2849.84|28.49|2878.33|25.98|1658.16|2023.38|8.26|818.20|1712.63| +2452049|40388|15857|85233|596457|5281|4858|85233|596457|5281|4858|20|35|4218|32|166.08|4.98|171.06|62.10|564.80|144.48|7.99|13.61|631.88| +2452040|67054|9761|24814|955229|6891|33353|24814|955229|6891|33353|25|16|4218|21|170.73|3.41|174.14|13.40|536.97|29.02|75.10|66.61|553.78| +2452212|63245|537|28862|1356241|2467|22693|28862|1356241|2467|22693|43|24|4219|11|1035.54|62.13|1097.67|34.48|67.87|186.39|339.66|509.49|164.48| +2452346|63876|8919|44940|244037|1534|16231|44940|244037|1534|16231|35|24|4219|18|281.70|25.35|307.05|41.09|9.54|0.00|225.36|56.34|75.98| +2451742|69371|15115|83734|1807256|7131|27453|83734|1807256|7131|27453|20|29|4220|35|380.10|22.80|402.90|26.44|950.25|83.62|44.47|252.01|999.49| +2452168|33448|10121|85989|745900|1011|46922|85989|745900|1011|46922|33|5|4221|20|64.20|0.00|64.20|97.77|0.00|28.89|6.70|28.61|97.77| +2452302|14685|2623|38792|950590|6035|43033|38792|950590|6035|43033|47|16|4221|18|338.40|20.30|358.70|79.51|213.30|64.29|139.79|134.32|313.11| +2451979|45847|1545|67712|535533|1754|36775|67712|535533|1754|36775|21|16|4221|16|4361.76|392.55|4754.31|8.95|1068.16|4361.76|0.00|0.00|1469.66| +2452092|82235|15919|10484|1380213|5142|9318|10484|1380213|5142|9318|49|26|4221|21|1794.87|0.00|1794.87|39.46|1051.26|1130.76|66.41|597.70|1090.72| +2451313|34579|8368|12169|1068901|2433|9565|12169|1068901|2433|9565|14|22|4222|5|171.00|10.26|181.26|40.19|104.05|63.27|60.32|47.41|154.50| +2451271|55145|2728|16317|1393070|5023|36976|16317|1393070|5023|36976|49|4|4222|15|51.45|4.63|56.08|72.01|15.45|23.66|2.22|25.57|92.09| +2451092|20557|3106|48214|1160033|1099|35718|48214|1160033|1099|35718|49|20|4222|47|2014.42|120.86|2135.28|87.17|774.56|564.03|1116.80|333.59|982.59| +2452183|74615|11779|52474|289001|3928|9655|52474|289001|3928|9655|19|29|4224|13|1367.60|27.35|1394.95|44.94|558.74|1271.86|90.95|4.79|631.03| +2452203|31268|2325|8747|3314|6681|20495|8747|3314|6681|20495|17|9|4224|43|430.00|4.30|434.30|52.49|133.73|236.50|36.76|156.74|190.52| +|65110|1319||186247|5271|48295|||||23||4228|15||||62.31|51.90|85.23|||122.52| +2452212|54152|12461|66475|965430|385|10140|66475|965430|385|10140|33|27|4228|18|204.66|6.13|210.79|30.43|14.58|194.42|6.14|4.10|51.14| +2451857|72422|1879|52123|475226|3482|11385|52123|475226|3482|11385|31|27|4229|29|1840.05|0.00|1840.05|20.86|224.75|110.40|415.11|1314.54|245.61| +2452868|70113|6693|13343|447219|3048|47724|13343|447219|3048|47724|18|24|4230|43|3200.92|32.00|3232.92|68.29|960.19|1248.35|1405.85|546.72|1060.48| +2451250|43905|1750|33210|1767495|3733|23971|33210|1767495|3733|23971|56|8|4232|10|633.20|50.65|683.85|47.32|22.20|82.31|506.81|44.08|120.17| +2452019|26883|6365|87687|243993|3499|3452|87687|243993|3499|3452|37|10|4233|11|334.29|20.05|354.34|8.16|45.21|200.57|120.34|13.38|73.42| +2451216|71123|13604|9071|1384940|3209|45478|9071|1384940|3209|45478|13|17|4234|74|1606.54|80.32|1686.86|40.88|655.64|1220.97|370.14|15.43|776.84| +2451205|41991|6589|92714|1442986|1258|3883|92714|1442986|1258|3883|50|16|4234|6|654.00|32.70|686.70|54.20|697.62|130.80|151.72|371.48|784.52| +2452466|32837|6737|52897|1076614|5484|13601|52897|1076614|5484|13601|41|2|4235|1|0.04|0.00|0.04|14.78|0.08|0.00|0.02|0.02|14.86| +2451750|73484|12874|65477|1378330|5583|18389|65477|1378330|5583|18389|16|8|4236|63|147.42|8.84|156.26|76.13|2070.81|41.27|6.36|99.79|2155.78| +2451735|930|5263|35420|1105275|377|41032|35420|1105275|377|41032|31|22|4236|3|44.10|0.88|44.98|15.63|12.90|33.51|10.06|0.53|29.41| +2450994|70170|4591|34218|1436673|2328|23936|34218|1436673|2328|23936|19|11|4237|1|66.39|3.98|70.37|50.41|42.15|3.98|26.21|36.20|96.54| +2452658|67625|14131|98609|55573|3384|31148|98609|55573|3384|31148|55|13|4238|59|1052.56|42.10|1094.66|64.20|307.98|947.30|86.31|18.95|414.28| +2452861|17578|12144|62148|1730776|4204|21985|62148|1730776|4204|21985|30|21|4238|27|1100.52|22.01|1122.53|12.16|341.01|484.22|603.97|12.33|375.18| +2452808|18375|17287|42008|1216395|29|24863|42008|1216395|29|24863|37|28|4238|56|940.24|56.41|996.65|33.32|511.28|808.60|78.98|52.66|601.01| +2452909|73738|17181|2097|568702|5354|29606|2097|568702|5354|29606|51|24|4238|19|279.11|16.74|295.85|37.50|237.12|136.76|105.33|37.02|291.36| +2451980|31560|10325|12741|1742342|3590|40791|12741|1742342|3590|40791|14|25|4239|49|360.64|21.63|382.27|56.26|811.44|212.77|90.20|57.67|889.33| +2452751|51639|4791|8437|1409188|339|33890|8437|1409188|339|33890|60|26|4240|14|187.74|13.14|200.88|45.75|79.38|26.28|24.21|137.25|138.27| +2452745|73234|9486|42487|824378|2269|30960|42487|824378|2269|30960|51|19|4240|2|25.98|0.25|26.23|77.26|26.52|9.09|10.47|6.42|104.03| +2452761|64714|16687|84733|952831|3776|5474|84733|952831|3776|5474|6|26|4240|4|403.24|16.12|419.36|63.30|102.96|334.68|55.53|13.03|182.38| +2452801|74802|16509|7428|750653|3925|41597|7428|750653|3925|41597|19|2|4240|42|2396.52|143.79|2540.31|21.63|196.98|1150.32|12.46|1233.74|362.40| +2452726|39074|11172|48131|1052745|6159|33926|48131|1052745|6159|33926|21|23|4241|12|1505.16|135.46|1640.62|37.44|1263.24|903.09|584.00|18.07|1436.14| +2452237|7780|5683|74794|833458|122|38786|74794|833458|122|38786|49|33|4242|34|351.90|24.63|376.53|34.24|655.86|123.16|45.74|183.00|714.73| +2452100|57128|12197|7253|746843|833|11223|7253|746843|833|11223|31|6|4243|67|1677.01|0.00|1677.01|65.76|1057.93|1056.51|359.89|260.61|1123.69| +2452725|30838|6565|18687|1869101|5915|40682|18687|1869101|5915|40682|31|19|4244|10|499.20|0.00|499.20|72.29|242.40|79.87|352.23|67.10|314.69| +2452729|68877|15582|9065|1789680|1942|5757|9065|1789680|1942|5757|25|10|4245|19|1944.08|58.32|2002.40|16.08|1817.16|505.46|129.47|1309.15|1891.56| +2452443|30459|10591|8614|1357601|1788|38822|8614|1357601|1788|38822|15|8|4246|2|63.10|5.04|68.14|39.88|52.18|24.60|26.95|11.55|97.10| +2451353|63709|211|6936|720236|3936|4036|6936|720236|3936|4036|31|33|4247|1|61.96|4.95|66.91|97.58|0.62|17.34|7.58|37.04|103.15| +2452261|23704|14249|40518|391705|939|45726|40518|391705|939|45726|5|12|4249|3|48.00|0.00|48.00|33.51|52.35|15.84|23.47|8.69|85.86| +2452218|50813|2593|80608|664744|80|29416|80608|664744|80|29416|41|35|4249|90|97.20|1.94|99.14|95.37|54.90|8.74|27.42|61.04|152.21| +2452930|17841|6381|66074|1772990|5153|4116|66074|1772990|5153|4116|1|18|4252|47|5219.82|0.00|5219.82|69.37|0.00|1304.95|2427.21|1487.66|69.37| +2452804|52366|13065|72574|875513|2005|37344|72574|875513|2005|37344|33|21|4252|23|370.07|25.90|395.97|34.39|407.10|51.80|92.29|225.98|467.39| +2452771||15159|17031|1028557|||17031|1028557||42056|||4252|15||11.57|1169.27|19.33||683.04||223.10|| +2452799|42834|6519|27195|1031219|6796|10420|27195|1031219|6796|10420|27|13|4254|51|3050.31|244.02|3294.33|21.55|1953.81|1189.62|1786.26|74.43|2219.38| +2452690|33465|10287|14686|1421847|5808|30969|14686|1421847|5808|30969|43|7|4254|5|964.90|57.89|1022.79|58.83|19.25|212.27|195.68|556.95|135.97| +2452252|64797|7763|47099|64928|4004|33433|47099|64928|4004|33433|17|17|4255|17|3488.74|104.66|3593.40|5.75|1690.31|313.98|1968.35|1206.41|1800.72| +||17684|75167|1881010|6853||75167|1881010||42696|||4256||24.24|0.00||||||3.03|| +2451118|43887|13828|41833|457302|6724|8193|41833|457302|6724|8193|44|31|4259|68|5718.12|514.63|6232.75|45.93|857.48|4174.22|1466.70|77.20|1418.04| +2451033|20994|7034|28586|1723269|799|35529|28586|1723269|799|35529|31|4|4259|60|4614.00|184.56|4798.56|52.76|0.00|1568.76|639.50|2405.74|237.32| +2450949|66260|10916|85429|1166327|4025|6123|85429|1166327|4025|6123|52|16|4259|25|1400.75|70.03|1470.78|57.98|357.25|476.25|453.00|471.50|485.26| +2452523|46098|12097|5455|1081498|741|46784|5455|1081498|741|46784|7|15|4260|36|1829.52|91.47|1920.99|28.10|1371.96|18.29|1394.64|416.59|1491.53| +2451821|40800|11414|16807|1256237|114|8744|16807|1256237|114|8744|22|30|4261|18|225.36|13.52|238.88|24.84|661.14|54.08|159.29|11.99|699.50| +2452872|60179|15192|76099|524182|1865|17148|76099|524182|1865|17148|49|6|4262|7|408.87|32.70|441.57|99.76|105.14|335.27|9.56|64.04|237.60| +2452691|29461|2832|49371|280908|5598|18335|49371|280908|5598|18335|12|33|4262|32|2915.52|58.31|2973.83|94.66|1328.00|1545.22|698.85|671.45|1480.97| +2452587|41784|675|74973|1409037|5948|2012|74973|1409037|5948|2012|37|29|4263|41|834.76|8.34|843.10|31.72|304.22|16.69|719.90|98.17|344.28| +2452735|23288|12463|90066|1677215|2411|12760|90066|1677215|2411|12760|3|35|4264|9|98.64|7.89|106.53|2.28|201.33|25.64|32.85|40.15|211.50| +2452655|37325|17586|6041|301639|6037|21282|6041|301639|6037|21282|19|24|4264|32|781.76|46.90|828.66|55.02|946.24|515.96|204.66|61.14|1048.16| +|21387|4873|96565||2969||||2969|||15|4264|62||9.00||13.58|2521.54|99.02|||2544.12| +2452144|63987|9779|64427|1544149|2718|20487|64427|1544149|2718|20487|23|24|4265|1|204.68|6.14|210.82|62.88|48.53|204.68|0.00|0.00|117.55| +2452633|49170|639|48256|1691229|249|4869|48256|1691229|249|4869|37|17|4266|3|212.25|8.49|220.74|85.20|20.07|23.34|28.33|160.58|113.76| +2452751|70326|7866|41814|1461769|5724|48526|41814|1461769|5724|48526|24|15|4268|33|2756.82|110.27|2867.09|74.78|594.66|1488.68|317.03|951.11|779.71| +2452234|72467|4487|1217|434484|104|124|1217|434484|104|124|49|6|4269|3|16.98|1.18|18.16|74.11|40.80|7.64|6.53|2.81|116.09| +2452185|71391|8407|32479|179119|1406|38893|32479|179119|1406|38893|27|6|4269|3|45.21|0.45|45.66|1.01|18.84|13.11|1.92|30.18|20.30| +2451204|28970|16714|64467|29967|1867|19767|64467|29967|1867|19767|7|7|4271|95|1195.10|11.95|1207.05|32.37|977.55|1135.34|41.83|17.93|1021.87| +2451249|18998|3646|10806|990205|804|2556|10806|990205|804|2556|32|26|4271|7|103.04|1.03|104.07|36.28|0.00|39.15|34.50|29.39|37.31| +2451185|78357|5408|96855|1738657|4012|23414|96855|1738657|4012|23414|56|13|4272|23|99.59|8.96|108.55|65.01|77.51|54.77|14.79|30.03|151.48| +2451294|73418|3079|51|348975|5546|20917|51|348975|5546|20917|52|4|4272|4|138.64|0.00|138.64|2.99|84.88|58.22|18.49|61.93|87.87| +2451586|24416|12428|76312|1623861|4210|11006|76312|1623861|4210|11006|2|6|4273|7|3.36|0.00|3.36|61.27|49.28|1.88|0.38|1.10|110.55| +2451609|36229|13430|99908|1451006|4504|9602|99908|1451006|4504|9602|7|12|4273|10|437.00|30.59|467.59|1.65|126.50|262.20|15.73|159.07|158.74| +2451739|12070|10240|16059|1385768|2230|26339|16059|1385768|2230|26339|13|26|4273|17|74.46|2.23|76.69|42.72|468.52|64.78|0.96|8.72|513.47| +2451808|64847|262|25480|327675|3917|912|25480|327675|3917|912|13|33|4274|18|1515.96|60.63|1576.59|25.28|806.40|576.06|56.39|883.51|892.31| +2452643|19724|4065|65559|1288851|2990|43882|65559|1288851|2990|43882|45|24|4275|4|484.40|33.90|518.30|73.84|159.04|368.14|19.76|96.50|266.78| +2452723||3751|193|1591271|5196|15074|193|1591271||15074|45||4275||3587.40|0.00|3587.40||0.00||||| +2451974|71158|9803|45581|1285966|2107|47852|45581|1285966|2107|47852|17|1|4276|5|195.55|17.59|213.14|77.77|3.40|125.15|27.45|42.95|98.76| +2451927|23386|4811|9877|1410145|4813|41928|9877|1410145|4813|41928|59|28|4276|52|2075.84|103.79|2179.63|92.63|1050.92|622.75|406.86|1046.23|1247.34| +||5143|95027||||95027||1792|22951|53||4276|30|2627.70|236.49||24.60||||19.45|| +2451408||12200||1493655|6128|25801|||||49|21|4277||||||15.57|34.97||13.75|119.75| +2451640|61631|16312|74878|1532054|6383|46712|74878|1532054|6383|46712|32|23|4277|79|3570.80|71.41|3642.21|2.15|10503.84|1821.10|892.34|857.36|10577.40| +2451607|37432|12805|70225|863793|2961|3329|70225|863793|2961|3329|20|10|4277|8|292.32|8.76|301.08|89.62|239.20|125.69|84.98|81.65|337.58| +2451455|14789|5617|16647|1864136|6025|32314|16647|1864136|6025|32314|22|12|4278|9|1532.70|45.98|1578.68|94.57|971.37|15.32|1502.20|15.18|1111.92| +2452553|81482|7158|2603|549799|3842|42238|2603|549799|3842|42238|36|30|4281|15|259.05|2.59|261.64|76.29|251.40|240.91|13.96|4.18|330.28| +2451160|33237|7969|74094|1016903|4159|3368|74094|1016903|4159|3368|22|26|4284|70|2253.30|0.00|2253.30|75.55|3505.60|1104.11|540.11|609.08|3581.15| +2451292|57462|5278|63818|439226|5387|11195|63818|439226|5387|11195|2|2|4285|43|4851.26|97.02|4948.28|61.04|501.81|1309.84|1097.84|2443.58|659.87| +2451148|25251|13321|11657|513018|3979|10893|11657|513018|3979|10893|8|4|4285|16|1380.96|27.61|1408.57|11.60|1046.88|1353.34|20.99|6.63|1086.09| +2452013|8010|9247|74542|1195710|6831|47599|74542|1195710|6831|47599|20|1|4286|6|689.76|13.79|703.55|52.81|62.70|193.13|278.11|218.52|129.30| +2452293|10777|1657|33092|429444|5796|18595|33092|429444|5796|18595|31|29|4288|28|202.44|4.04|206.48|49.27|2026.64|26.31|59.88|116.25|2079.95| +2452506|73970|613|32955|416570|4538|48953|32955|416570|4538|48953|57|34|4289|13|857.35|0.00|857.35|57.49|342.94|308.64|131.69|417.02|400.43| +2452353|76316|17785|83743|77225|1907|35836|83743|77225|1907|35836|1|24|4289|49|1433.25|128.99|1562.24|50.46|762.44|673.62|113.94|645.69|941.89| +2452941|72084|14778|70426|39155|2637|18966|70426|39155|2637|18966|57|24|4290|24|1405.44|42.16|1447.60|23.40|122.16|815.15|100.34|489.95|187.72| +2451217|77917|5545|88772|1799481|590|28452|88772|1799481|590|28452|8|29|4291|10|473.20|23.66|496.86|89.67|171.60|113.56|284.11|75.53|284.93| +2451311|5410|7339|79405|75957|2109|38214|79405|75957|2109|38214|31|34|4291|17|163.54|1.63|165.17|31.50|74.63|150.45|6.80|6.29|107.76| +2452596|66980|14748|34303|1236047|3744|43088|34303|1236047|3744|43088|7|11|4292|20|20.20|1.41|21.61|78.98|32.20|6.06|0.14|14.00|112.59| +2452758|42543|2809|94818|984173|3128|47330|94818|984173|3128|47330|7|12|4292|38|0.00|0.00|0.00|78.06|68.78|0.00|0.00|0.00|146.84| +2452878|74181|10567|39521|501841|5761|45728|39521|501841|5761|45728|7|20|4294|32|123.20|6.16|129.36|43.25|36.16|51.74|33.58|37.88|85.57| +2451303|70506|2314|42886|750827|1777|9171|42886|750827|1777|9171|55|28|4295|4|251.52|20.12|271.64|44.15|22.32|65.39|124.70|61.43|86.59| +2451335|71171|4034|7972|938458|591|37077|7972|938458|591|37077|22|5|4295|5|446.90|26.81|473.71|60.21|271.05|259.20|37.54|150.16|358.07| +2452102|72253|7124|90314|1685045|2176|36507|90314|1685045|2176|36507|43|25|4296|1|4.95|0.34|5.29|26.01|23.12|1.13|1.71|2.11|49.47| +2452384|35648|4927|63807|679104|4801|25298|63807|679104|4801|25298|9|26|4297|34|3477.52|278.20|3755.72|12.36|1961.46|2051.73|983.79|442.00|2252.02| +2451756|29054|16705|11477|1092944|6858|44725|11477|1092944|6858|44725|55|3|4298|54|1864.62|167.81|2032.43|57.34|3400.38|1025.54|587.35|251.73|3625.53| +2452108|74502|9611|9749|672616|4730|42797|9749|672616|4730|42797|26|23|4299|26|1634.88|130.79|1765.67|13.89|1235.26|98.09|1260.16|276.63|1379.94| +2451602|68834|8416|10958|133116|1321|36726|10958|133116|1321|36726|22|19|4301|17|1180.48|35.41|1215.89|88.60|406.98|637.45|124.89|418.14|530.99| +2451568|48671|4826|29387|842865|2743|44621|29387|842865|2743|44621|37|34|4301|6|147.42|10.31|157.73|38.40|91.14|95.82|44.37|7.23|139.85| +2451693|34743|8332|55853|1600832|4211|3990|55853|1600832|4211|3990|40|10|4301|12|667.08|26.68|693.76|58.35|452.64|26.68|397.04|243.36|537.67| +2451299|73565|10198|19615|1509079|4143|1607|19615|1509079|4143|1607|7|21|4302|1|0.43|0.00|0.43|0.79|5.65|0.12|0.02|0.29|6.44| +2452252|63824|16275|36715|1225818|6921|41224|36715|1225818|6921|41224|51|32|4303|10|844.30|25.32|869.62|87.80|449.30|464.36|170.97|208.97|562.42| +2451504|58958|10867|29344|453125|4459|22193|29344|453125|4459|22193|2|21|4304|35|1557.15|15.57|1572.72|17.71|287.35|436.00|661.47|459.68|320.63| +2451255|63150|10987|86989|1379746|2456|30366|86989|1379746|2456|30366|1|13|4305|31|3051.95|91.55|3143.50|19.00|2305.78|610.39|805.71|1635.85|2416.33| +2452514|59183|17073|69697|551597|2297|37397|69697|551597|2297|37397|39|5|4306|4|29.72|2.67|32.39|12.79|124.84|2.67|20.82|6.23|140.30| +2452228|38404|6473|9472|264110|6395|49833|9472|264110|6395|49833|33|22|4307|42|4691.82|140.75|4832.57|97.03|1451.94|2627.41|1981.83|82.58|1689.72| +2452096|60181|1315|52457|436390|4323|18256|52457|436390|4323|18256|25|26|4307|17|629.51|0.00|629.51|40.80|656.88|497.31|97.82|34.38|697.68| +2451210|56172|14054|10835|1619068|101|39934|10835|1619068|101|39934|49|32|4309|56|1195.60|0.00|1195.60|26.34|117.60|370.63|305.23|519.74|143.94| +2451619|60248|16876|29851|1063515|1532|14396|29851|1063515|1532|14396|31|1|4312|57|2352.39|23.52|2375.91|56.97|2200.77|1176.19|823.34|352.86|2281.26| +2452794|11033|14937|41418|895747|476|10005|41418|895747|476|10005|48|4|4314|21|692.16|62.29|754.45|86.42|239.19|318.39|115.86|257.91|387.90| +2452806|77865|4518|50013|843103|1793|39636|50013|843103|1793|39636|15|11|4314|22|362.12|7.24|369.36|89.49|379.94|39.83|99.90|222.39|476.67| +2452478|74370|14484|96627|1364078|5929|22580|96627|1364078|5929|22580|12|6|4315|1|128.24|2.56|130.80|49.67|11.22|38.47|34.11|55.66|63.45| +2452393|46239|3852|59576|491788|2400|20165|59576|491788|2400|20165|24|8|4315|29|662.94|19.88|682.82|31.74|190.53|656.31|2.98|3.65|242.15| +2451338|81443|511|23762|1663293|3731|23402|23762|1663293|3731|23402|26|4|4316|10|126.00|10.08|136.08|65.97|49.00|45.36|33.86|46.78|125.05| +2452339|7927|6931|48576|1359148|4312|23638|48576|1359148|4312|23638|55|29|4319|10|251.90|10.07|261.97|35.46|407.00|115.87|28.56|107.47|452.53| +2451333|12726|1078|40593|1274567|988|4562|40593|1274567|988|4562|43|29|4320|52|1349.92|13.49|1363.41|90.06|767.52|863.94|417.94|68.04|871.07| +2452437|38944|8953|94951|244201|298|46795|94951|244201|298|46795|35|3|4321|42|2247.84|157.34|2405.18|54.73|1316.70|561.96|1247.55|438.33|1528.77| +2452405|65918|17537|15302|1728146|1790|38234|15302|1728146|1790|38234|39|23|4321|22|2065.36|165.22|2230.58|65.72|484.44|454.37|386.63|1224.36|715.38| +2451694|64942|4121|9749|1095262|5222|19935|9749|1095262|5222|19935|8|24|4323|22|342.98|17.14|360.12|21.53|788.92|277.81|56.04|9.13|827.59| +2451752|32853|5090|44235|249769|1256|3190|44235|249769|1256|3190|11|3|4323|84|7686.84|691.81|8378.65|18.19|4891.32|6072.60|1485.10|129.14|5601.32| +2451416|24633|3661|12347|1639916|1306|18092|12347|1639916|1306|18092|55|4|4324|8|145.84|13.12|158.96|74.43|52.24|123.96|12.47|9.41|139.79| +2451248|44996|16531|27351|918511|3624|36784|27351|918511|3624|36784|50|29|4324|29|635.39|31.76|667.15|45.10|2150.93|451.12|53.43|130.84|2227.79| +2452655|39475|14112|5072|876396|1320|16597|5072|876396|1320|16597|24|9|4325|31|693.47|34.67|728.14|16.17|2542.62|450.75|2.42|240.30|2593.46| +2452434|21977|17845|26858|1523148|3085|43375|26858|1523148|3085|43375|31|29|4329|31|216.07|10.80|226.87|29.04|1442.43|116.67|88.46|10.94|1482.27| +2452484|58332|15377|871|1512933|1795|36657|871|1512933|1795|36657|1|9|4329|52|1269.84|76.19|1346.03|45.46|243.36|215.87|959.11|94.86|365.01| +2452380|74408|11467|39637|1608915|593||39637||593|38271|31|34|4329|53|||1894.07||||775.17||1064.63| +2451530|14759|8635|22529|979434|3539||22529|||32444|||4331||586.46||592.32|||433.98|82.33|70.15|170.81| +2452151|22260|17925|63610|61108|1359|12057|63610|61108|1359|12057|17|22|4332|3|83.16|3.32|86.48|39.81|46.20|40.74|4.24|38.18|89.33| +2452702|85372|493|65936|256686|2989|18939|65936|256686|2989|18939|30|27|4333|46|242.42|14.54|256.96|45.03|242.42|26.66|187.71|28.05|301.99| +2452703|46603|17463|98780|1552080|580|40680|98780|1552080|580|40680|25|8|4333|22|919.82|36.79|956.61|0.56|468.82|294.34|369.03|256.45|506.17| +2451162|23759|16844|87764|1305403|7042|48681|87764|1305403|7042|48681|40|30|4334|1|40.41|2.02|42.43|40.64|12.98|23.84|4.63|11.94|55.64| +2452166|36384|5859|9715|1190569|223|11477|9715|1190569|223|11477|13|18|4336|24|77.28|3.09|80.37|54.42|1213.92|17.00|45.21|15.07|1271.43| +2452186|18567|3369|13135|1398277|2172|44067|13135|1398277|2172|44067|45|7|4336|43|625.22|43.76|668.98|14.33|1563.48|62.52|151.92|410.78|1621.57| +2452202|47493|10271|37747|716411|3522|47386|37747|716411|3522|47386|17|16|4336|50|667.50|6.67|674.17|28.69|0.00|340.42|29.43|297.65|35.36| +2451916|24875|6067|75115|802547|5979|47811|75115|802547|5979|47811|13|16|4337|11|25.30|0.50|25.80|62.99|14.30|17.45|4.71|3.14|77.79| +2451829|26906|6410|69348|1483268|2341|16261|69348|1483268|2341|16261|7|10|4337|1|11.25|0.56|11.81|19.01|9.84|9.22|0.75|1.28|29.41| +2451847|8446|13625|78833|938167|5948|20082|78833|938167|5948|20082|8|15|4337|48|4435.68|0.00|4435.68|23.30|700.32|709.70|782.45|2943.53|723.62| +2451840|61783|14324|83106|1483045|937|48029|83106|1483045|937|48029|29|18|4337|20|777.60|7.77|785.37|65.04|432.00|723.16|3.81|50.63|504.81| +2452733|5739|11029|36102|29474|893|26712|36102|29474|893|26712|7|14|4338|15|22.95|1.83|24.78|56.33|34.80|13.99|1.43|7.53|92.96| +2452382|66066|17049|11822|841258|3847|3683|11822|841258|3847|3683|41|28|4340|14|244.58|9.78|254.36|85.46|47.60|205.44|36.00|3.14|142.84| +2452085|7589|9223|3546|98089|185|36281|3546|98089|185|36281|11|21|4341|23|956.11|86.04|1042.15|74.08|19.32|889.18|46.18|20.75|179.44| +2451928|51993|11048|2230|1735921|4812|48903|2230|1735921|4812|48903|38|6|4341|3|117.57|0.00|117.57|8.19|132.66|29.39|10.58|77.60|140.85| +||8384|86160||||||||31|2|4342||1222.52|24.45|1246.97|96.87|970.32|1051.36|34.23||| +2451188|63029|10964|22787|106350|7066|32982|22787|106350|7066|32982|58|29|4343|1|25.04|2.00|27.04|98.38|5.53|16.02|7.75|1.27|105.91| +2451200|72357|8560|71057|1141670|7130|33743|71057|1141670|7130|33743|34|15|4343|81|716.04|0.00|716.04|98.28|358.02|171.84|397.26|146.94|456.30| +2451136|14227|5312|14148|36617|4686|4402|14148|36617|4686|4402|49|33|4343|6|822.90|24.68|847.58|42.16|66.36|781.75|6.99|34.16|133.20| +2452496|28210|13747|97457|1013314|4818|47875|97457|1013314|4818|47875|17|6|4345|77|66.22|1.32|67.54|65.00|20.79|43.04|15.06|8.12|87.11| +2451336|79225|2804|15002|1280719|5636|7753|15002|1280719|5636|7753|49|3|4346|46|91.54|1.83|93.37|35.67|96.14|52.17|28.34|11.03|133.64| +2451181|80366|8458|23043|743565|5334|27856|23043|743565|5334|27856|26|35|4346|15|152.55|3.05|155.60|2.81|60.90|71.69|46.09|34.77|66.76| +2451244|59709|11984|78333|1651109|238|14214|78333|1651109|238|14214|58|35|4346|3|40.44|1.61|42.05|4.32|28.23|37.20|0.61|2.63|34.16| +2452023|8653|536|82195|667191|5340|18736|82195|667191|5340|18736|56|10|4347|4|47.24|0.94|48.18|62.92|10.88|25.50|18.26|3.48|74.74| +2451765|44773|9488|22854|1077797|1540|33683|22854|1077797|1540|33683|38|35|4348|7|49.00|0.98|49.98|41.50|256.06|19.60|22.93|6.47|298.54| +2451824|58999|4844|69404|1891451|1945|6039|69404|1891451|1945|6039|40|24|4348|31|2989.64|209.27|3198.91|64.23|1719.88|1464.92|1478.97|45.75|1993.38| +2451568|75851|13208|59539|148274|3762|49650|59539|148274|3762|49650|56|32|4348|3|17.64|0.00|17.64|9.67|7.77|9.17|2.87|5.60|17.44| +2451453|70204|11473|40640|1146336|32|35281|40640|1146336|32|35281|1|5|4349|28|1046.92|0.00|1046.92|13.96|574.00|324.54|621.24|101.14|587.96| +2451360|28429|16852|34214|1242033|6299|8918|34214|1242033|6299|8918|40|24|4349|25|1492.75|104.49|1597.24|83.05|1137.25|447.82|323.92|721.01|1324.79| +2451532|27260|7826|1917|1758020|1770|26720|1917|1758020|1770|26720|20|21|4350|85|435.20|4.35|439.55|21.76|217.60|161.02|84.99|189.19|243.71| +2451583|18009|17756|26519|1667189|1106|38576|26519|1667189|1106|38576|58|35|4352|2|104.24|2.08|106.32|27.56|7.26|60.45|5.25|38.54|36.90| +2451429|71159|1162|47685|1410607|1976|14098|47685|1410607|1976|14098|43|30|4352|2|17.54|0.35|17.89|76.37|16.28|8.41|4.65|4.48|93.00| +2451473|28008|4861|80098|1730132|3813|21094|80098|1730132|3813|21094|58|13|4352|37|2228.51|66.85|2295.36|83.33|1178.45|646.26|1550.60|31.65|1328.63| +|79238|8504|43442|774050|||43442|774050||47619|44||4353|6|518.40||544.32||372.60||||| +2452029|80469|16463|26195|411510|2872|17387|26195|411510|2872|17387|29|23|4354|2|372.70|3.72|376.42|27.41|70.98|316.79|33.54|22.37|102.11| +2452044|81722|16124|63062|436336|1909|1020|63062|436336|1909|1020|49|34|4354|1|157.71|4.73|162.44|87.79|76.54|55.19|17.42|85.10|169.06| +|72575|1739|37437|765673|||37437||2709||49||4354|8||18.47|480.39|53.94||124.71|161.86||271.53| +2451799|34499|3257|13501|54435|6216|3197|13501|54435|6216|3197|2|3|4354|32|255.36|20.42|275.78|63.88|937.28|232.37|5.05|17.94|1021.58| +2452020|61921|2917|45603|737419|3635|46251|45603|737419|3635|46251|20|10|4354|12|438.48|39.46|477.94|92.60|666.48|232.39|26.79|179.30|798.54| +2452429|45541|9699|46321|250995|6069|49543|46321|250995|6069|49543|33|27|4356|2|63.06|1.89|64.95|12.20|87.08|2.52|59.32|1.22|101.17| +2452408|34315|2183|46336|1490659|4582|21305|46336|1490659|4582|21305|5|3|4356|16|2823.36|0.00|2823.36|60.76|32.80|2597.49|29.36|196.51|93.56| +2452860|73897|10219|31137|335927|4717|14707|31137|335927|4717|14707|54|33|4357|1|13.23|1.19|14.42|68.30|1.96|10.84|1.33|1.06|71.45| +2452738|77530|12465|64909|1581779|6202|47072|64909|1581779|6202|47072|45|29|4357|2|322.62|3.22|325.84|36.37|120.98|258.09|52.26|12.27|160.57| +2452662|47833|5917|41994|514327|2660|3793|41994|514327|2660|3793|39|23|4357|10|525.90|36.81|562.71|9.57|674.20|157.77|110.43|257.70|720.58| +2452744|74795|7537|60426|343162|7051|5387|60426|343162|7051|5387|37|8|4357|42|3714.90|0.00|3714.90|38.44|559.02|3306.26|314.65|93.99|597.46| +2452719|69407|13230|43044|1210161|3118|2354|43044|1210161|3118|2354|19|30|4359|76|93.48|6.54|100.02|19.85|8.36|23.37|38.56|31.55|34.75| +2452769|63359|14430|94251|763868|3085|34655|94251|763868|3085|34655|48|3|4359|11|86.46|6.05|92.51|15.39|18.15|52.74|7.75|25.97|39.59| +2452607|20127|2253|3976|1679105|6234|32163|3976|1679105|6234|32163|1|14|4359|13|1329.77|119.67|1449.44|91.89|443.17|930.83|107.71|291.23|654.73| +2451685|18404|13339|65827|1538302|377|16805|65827|1538302|377|16805|58|29|4360|2|361.72|25.32|387.04|28.41|155.58|137.45|40.36|183.91|209.31| +2451510|73834|10168|32699|174147|5176|33069|32699|174147|5176|33069|46|35|4363|1|93.77|2.81|96.58|1.32|57.70|40.32|46.50|6.95|61.83| +2451676|70437|2842|40151|273914|477|49840|40151|273914|477|49840|26|6|4363|2|107.02|8.56|115.58|34.56|34.34|54.58|30.41|22.03|77.46| +2451454|70920|6793|59848|1810108|2768|19828|59848|1810108|2768|19828|14|8|4365|30|331.80|9.95|341.75|79.57|580.50|245.53|6.03|80.24|670.02| +2451416|40340|17438|32950|733342|6181|23020|32950|733342|6181|23020|40|33|4365|1|2.32|0.06|2.38|17.78|1.01|0.90|1.32|0.10|18.85| +2452780|53401|13447|84614|1186323|6079|23435|84614|1186323|6079|23435|24|32|4366|38|7685.88|538.01|8223.89|63.53|2794.90|768.58|2974.43|3942.87|3396.44| +2452876|71024|6444|71067|521635|1450|6418|71067|521635|1450|6418|31|2|4366|32|2796.80|223.74|3020.54|83.45|815.68|0.00|307.64|2489.16|1122.87| +2452845|20259|5280|89617|777912|4254|4149|89617|777912|4254|4149|57|5|4366|57|6767.61|473.73|7241.34|72.89|1605.69|2233.31|3083.32|1450.98|2152.31| +2451738|53110|12412|86842|658518|3794|37470|86842|658518|3794|37470|13|11|4367|7|11.13|0.77|11.90|86.39|40.25|9.23|1.53|0.37|127.41| +2452167|66398|1255|23189|1036343|2297|7051|23189|1036343|2297|7051|25|21|4368|4|716.68|21.50|738.18|55.02|143.32|645.01|26.51|45.16|219.84| +2451627|72036|3883|30230|1694010|4190|30370|30230|1694010|4190|30370|31|9|4369|34|1367.14|0.00|1367.14|93.12|585.82|1271.44|56.46|39.24|678.94| +2451811|15627|1910|21514|1350715|3469|21423|21514|1350715|3469|21423|10|16|4370|13|349.83|6.99|356.82|25.32|0.00|108.44|202.76|38.63|32.31| +2451537|70016|5173|78883|446502|74|42093|78883|446502|74|42093|56|30|4370|7|218.61|8.74|227.35|14.35|83.37|109.30|89.63|19.68|106.46| +|69234|10794|33224|||48544|||||||4371|||3.18||8.98|60.64|27.09|5.26|47.35|| +2452873|69398|15769|38389|1116494|2952|37857|38389|1116494|2952|37857|6|31|4371|46|722.20|36.11|758.31|43.10|268.18|541.65|50.55|130.00|347.39| +2451413|74743|7660|14006|1202243|5680|42317|14006|1202243|5680|42317|1|17|4372|1|0.00|0.00|0.00|21.13|23.80|0.00|0.00|0.00|44.93| +2452568|43011|5292|56519|303882|6798|5361|56519|303882|6798|5361|3|18|4374|42|1835.82|91.79|1927.61|5.48|850.08|1487.01|170.91|177.90|947.35| +2451211|66112|12176|25485|1326618|6075|38585|25485|1326618|6075|38585|46|35|4376|10|201.80|18.16|219.96|78.62|20.40|52.46|116.48|32.86|117.18| +2451234|20615|9272|10593|1548002|6658|44067|10593|1548002|6658|44067|38|32|4376|2|126.86|7.61|134.47|37.69|64.50|10.14|32.68|84.04|109.80| +2452042|34978|1382|86657|887812|4614|21769|86657|887812|4614|21769|41|13|4377|43|86.86|1.73|88.59|39.44|3227.58|47.77|19.54|19.55|3268.75| +2451354|72503|7004|74417|1501348|4726|7304|74417|1501348|4726|7304|55|10|4379|67|4481.63|89.63|4571.26|74.25|243.88|4347.18|26.89|107.56|407.76| +2451365|68632|14420|38646|1920536|5697|43183|38646|1920536|5697|43183|49|12|4379|14|91.98|7.35|99.33|25.63|5.74|90.14|0.23|1.61|38.72| +2452699|2176|14106|43257|564081|4905|38576|43257|564081|4905|38576|18|17|4380|22|573.32|34.39|607.71|41.09|18.48|458.65|110.08|4.59|93.96| +2452590|53498|4369|1930|1853747|2811|39828|1930|1853747|2811|39828|12|29|4380|28|1554.56|77.72|1632.28|47.79|310.80|1430.19|92.03|32.34|436.31| +2452534|74299|10849|65488|935172|2809|36893|65488|935172|2809|36893|5|1|4381|42|3651.06|182.55|3833.61|39.10|0.00|620.68|1090.93|1939.45|221.65| +|47403|308||1917365||2269||1917365||2269|14||4382||1557.84||1557.84||||475.14||610.50| +2451279|16031|6052|13230|284390|4350|11863|13230|284390|4350|11863|32|34|4382|14|36.54|1.46|38.00|6.82|3.36|24.11|11.06|1.37|11.64| +2451291|75791|6922|48397|578290|6136|429|48397|578290|6136|429|38|31|4383|24|79.92|2.39|82.31|58.56|230.16|76.72|0.03|3.17|291.11| +2451313|59594|11212|24933|1761838|2296|49154|24933|1761838|2296|49154|20|31|4383|11|195.14|0.00|195.14|79.69|55.77|50.73|129.96|14.45|135.46| +2452122|27922|16297|18242|1374096|4350|27804|18242|1374096|4350|27804|13|11|4384|1|97.64|3.90|101.54|22.67|18.30|80.06|7.91|9.67|44.87| +2452026|13345|14497|4968|57516|4464|32212|4968|57516|4464|32212|50|8|4384|16|2183.36|174.66|2358.02|26.14|1157.76|131.00|513.09|1539.27|1358.56| +2451549|73386|14488|78560|754211|6309|18760|78560|754211|6309|18760|50|6|4386|9|700.65|63.05|763.70|64.77|432.27|91.08|262.11|347.46|560.09| +2451583|46937|10735|94087|331000|6313|32473|94087|331000|6313|32473|7|5|4386|74|6711.80|0.00|6711.80|68.04|3213.82|3959.96|2366.58|385.26|3281.86| +2452717|28655|4471|88828|577839|3|31367|88828|577839|3|31367|6|21|4388|41|2042.21|20.42|2062.63|36.77|1063.13|735.19|562.01|745.01|1120.32| +2451293|27762|15592|59688|1571696|6457|32065|59688|1571696|6457|32065|14|9|4389|44|33.00|0.33|33.33|53.62|26.40|14.52|3.69|14.79|80.35| +2452307|74143|16889|90891|531682|6520|42158|90891|531682|6520|42158|53|7|4390|51|13281.42|531.25|13812.67|43.99|265.20|9562.62|1636.27|2082.53|840.44| +2451582|28049|16309|50611|1651849|6240|7431|50611|1651849|6240|7431|55|34|4392|48|2502.24|25.02|2527.26|98.85|1383.36|700.62|378.34|1423.28|1507.23| +2451370|30171|3751|70154|1492034|761|19888|70154|1492034|761|19888|43|21|4393|24|153.84|9.23|163.07|88.99|283.68|124.61|7.89|21.34|381.90| +2452578|13185|4849|45676|300629|4382|45335|45676|300629|4382|45335|45|10|4394|9|1325.07|79.50|1404.57|86.95|128.16|1219.06|104.94|1.07|294.61| +2452444|30988|4953|94891|866282|73|19864|94891|866282|73|19864|39|35|4394|65|2291.25|137.47|2428.72|20.44|176.15|1580.96|383.55|326.74|334.06| +2451435||13879|11714|1098908||49343|11714|1098908||||11|4395|17|||620.07|15.56||||213.99|240.04| +2452220|25432|8023|86605|726164|3676|19635|86605|726164|3676|19635|29|26|4396|4|33.88|1.35|35.23|66.13|110.16|10.50|7.94|15.44|177.64| +2452474|5357|2981|20710|1024916|5475|37681|20710|1024916|5475|37681|29|3|4397|12|11.04|0.44|11.48|66.54|2.52|6.95|3.68|0.41|69.50| +2452543|27417|3071|49613|1339644|965|26559|49613|1339644|965|26559|45|21|4397|30|3051.60|30.51|3082.11|40.76|1243.20|1800.44|988.41|262.75|1314.47| +2452638|46927|5299|31973|1331361|4702|11690|31973|1331361|4702|11690|18|7|4398|30|2315.70|0.00|2315.70|95.29|0.00|1713.61|367.27|234.82|95.29| +2452502|72617|15204|53455|694921|2434|26700|53455|694921|2434|26700|37|16|4398|36|2411.64|48.23|2459.87|96.92|0.00|361.74|840.45|1209.45|145.15| +2451291|40456|15814|40645|732293|1988|39236|40645|732293|1988|39236|31|18|4399|68|2231.76|44.63|2276.39|8.96|2329.00|178.54|102.66|1950.56|2382.59| +2451466||12493|78038||3580|30751||1858809||30751||32|4399|7|133.00|||21.28|28.00|23.94||76.35|| +2451561|65118|16292|2701|747449|2150|33651|2701|747449|2150|33651|13|6|4400|21|1406.79|112.54|1519.33|93.08|81.69|1167.63|90.88|148.28|287.31| +2451546|75119|6640|89123|443439|4955|8118|89123|443439|4955|8118|1|9|4400|18|174.06|6.96|181.02|94.31|167.40|73.10|75.72|25.24|268.67| +2452808|39292|16743|69083|556002|2121|17440|69083|556002|2121|17440|21|14|4403|2|0.00|0.00|0.00|76.97|52.96|0.00|0.00|0.00|129.93| +2452801|83810|5275|85806|1704287|5615|47743|85806|1704287|5615|47743|24|7|4403|53|5291.52|317.49|5609.01|57.28|892.52|4444.87|431.79|414.86|1267.29| +2452751|71922|1194|52045|440317|6762|2663|52045|440317|6762|2663|54|14|4403|23|1403.92|28.07|1431.99|1.64|0.00|1277.56|6.31|120.05|29.71| +2452711|29123|9438|14256|1600369|6482|27795|14256|1600369|6482|27795|37|18|4403|3|48.96|2.44|51.40|45.79|8.88|3.91|42.34|2.71|57.11| +2451776|36895|15740|36151|1710721|6891|36308|36151|1710721|6891|36308|56|13|4405|22|3293.40|65.86|3359.26|7.50|1181.18|658.68|1686.22|948.50|1254.54| +2451707|59683|15991|54993|104569|1345|12121|54993|104569|1345|12121|34|2|4407|44|5918.88|236.75|6155.63|85.24|246.40|828.64|2239.70|2850.54|568.39| +2451624|65944|7228|47716|1428439|1962|5132|47716|1428439|1962|5132|14|17|4407|35|3540.95|177.04|3717.99|13.17|2376.50|0.00|1133.10|2407.85|2566.71| +2452571|11786|1351|82628|1918756|6084|46212|82628|1918756|6084|46212|25|28|4409|49|4013.10|120.39|4133.49|62.78|1480.78|3852.57|89.89|70.64|1663.95| +2452013|64609|6043|58249|561349|2849|43949|58249|561349|2849|43949|35|26|4410|16|31.52|1.89|33.41|17.51|69.44|11.97|12.12|7.43|88.84| +2452134|48322|10603|10404|869432|6070|4550|10404|869432|6070|4550|25|20|4410|94|53.58|4.28|57.86|61.77|1405.30|10.71|11.57|31.30|1471.35| +2451682|37281|15970|37433|82358|4083|12961|37433|82358|4083|12961|22|32|4411|5|61.95|4.95|66.90|42.36|23.40|50.79|6.80|4.36|70.71| +2451483|47737|1732|96732|1397277|6975|33120|96732|1397277|6975|33120|1|28|4413|3|341.01|17.05|358.06|49.81|102.30|265.98|69.77|5.26|169.16| +2451274|54178|17470|15051|16488|2278|49315|15051|16488|2278|49315|49|20|4413|37|4219.11|253.14|4472.25|93.54|2012.06|4219.11|0.00|0.00|2358.74| +2452290|41222|14175|35532|1532714|1096|11831|35532|1532714|1096|11831|3|14|4414|33|701.58|42.09|743.67|60.10|656.37|245.55|4.56|451.47|758.56| +2451380|30833|8836|76483|1775413|3143|45273|76483|1775413|3143|45273|20|24|4416|28|1322.16|105.77|1427.93|7.01|215.88|793.29|296.16|232.71|328.66| +|33779|595|76620|||||67222|||43|30|4417|3||5.97||13.20||20.91|||56.52| +2452311|73716|9021|43581|348416|6051|1733|43581|348416|6051|1733|3|5|4417|20|1840.00|147.20|1987.20|94.43|306.60|993.60|270.84|575.56|548.23| +2452248|14256|2071|12125|1257314|6503|36492|12125|1257314|6503|36492|35|28|4417|24|2934.48|117.37|3051.85|80.25|1137.84|1995.44|882.69|56.35|1335.46| +2452086|20059|4123|90129|61296|910|40877|90129|61296|910|40877|43|21|4417|50|6527.00|195.81|6722.81|13.82|2101.50|5743.76|783.24|0.00|2311.13| +2452360|28285|16249|58578|667867|5156|16045|58578|667867|5156|16045|31|23|4417|32|726.08|0.00|726.08|59.68|29.44|14.52|697.32|14.24|89.12| +2451175|73889|8965|45435|815438|1664|14067|45435|815438|1664|14067|20|12|4418|53|0.00|0.00|0.00|50.19|2064.88|0.00|0.00|0.00|2115.07| +2451140|56317|14269|11343|192769|1748|15574|11343|192769|1748|15574|56|4|4418|35|1443.40|57.73|1501.13|4.05|603.40|1327.92|110.86|4.62|665.18| +2451280|39218|5732|93629|1126434|4039|41644|93629|1126434|4039|41644|28|18|4422|63|3715.74|37.15|3752.89|45.76|3715.74|854.62|1087.22|1773.90|3798.65| +2451259|36438|16981|66313|356224|5335|17246|66313|356224|5335|17246|43|34|4422|20|422.40|29.56|451.96|45.53|208.60|202.75|171.32|48.33|283.69| +2451419|78764|12050|21905|1223938|4371|45604|21905|1223938|4371|45604|49|30|4423|15|1012.95|10.12|1023.07|93.24|655.50|496.34|232.47|284.14|758.86| +2451493|38484|15380|66211|1023887|763|38039|66211|1023887|763|38039|38|8|4423|6|621.24|24.84|646.08|36.73|98.04|410.01|90.82|120.41|159.61| +2451727|72122|15901|35937|916160|2336|34559|35937|916160|2336|34559|8|1|4425|52|2124.20|148.69|2272.89|66.40|1994.20|1975.50|104.09|44.61|2209.29| +2451134|63098|3298|66704|1398417|4083|33749|66704|1398417|4083|33749|52|20|4426|44|2233.88|111.69|2345.57|79.09|770.00|379.75|1483.30|370.83|960.78| +2451399|5662|17587|91244|679776|2261|25993|91244|679776|2261|25993|20|19|4428|6|49.32|0.98|50.30|98.67|148.08|36.00|6.66|6.66|247.73| +2452749|59911|7243|43883|1177407|4178|47702|43883|1177407|4178|47702|36|26|4429|51|339.15|13.56|352.71|28.47|132.60|230.62|67.28|41.25|174.63| +2452580|72147|9961|89879|1568903|5880|20104|89879|1568903|5880|20104|57|12|4429|7|116.13|9.29|125.42|10.25|45.08|51.09|36.42|28.62|64.62| +2452583|81039|14731|65111|1092933|1375|15041|65111|1092933|1375|15041|30|23|4429|38|246.62|9.86|256.48|16.20|287.66|115.91|78.42|52.29|313.72| +2452532|33837|3273|85314|205807|3547|17287|85314|205807|3547|17287|24|23|4429|16|1154.08|46.16|1200.24|95.57|87.36|161.57|764.23|228.28|229.09| +2451406|21548|3958|13953|235682|4258|31587|13953|235682|4258|31587|25|15|4430|19|0.00|0.00|0.00|51.78|607.24|0.00|0.00|0.00|659.02| +2451501|55955|13249|72179|1471644|3556|17421|72179|1471644|3556|17421|44|17|4430|25|266.75|2.66|269.41|79.33|54.25|213.40|14.93|38.42|136.24| +2452117|60430|722|89762|218265|2768|42918|89762|218265|2768|42918|1|26|4431|10|1401.90|14.01|1415.91|46.19|734.30|925.25|90.56|386.09|794.50| +2451072|49746|17632|31365|1273236|1860|36884|31365|1273236|1860|36884|25|20|4432|66|14669.82|293.39|14963.21|57.87|5363.16|6601.41|5970.62|2097.79|5714.42| +2451314|75127|8338|95872|1122189|4059|48729|95872|1122189|4059|48729|4|1|4433|1|76.40|6.87|83.27|59.76|16.01|46.60|8.04|21.76|82.64| +2451710|72199|5350|93048|867745|1195|18292|93048|867745|1195|18292|1|10|4437|35|28.35|1.13|29.48|98.74|341.25|13.32|13.82|1.21|441.12| +2452447|76940|17035|96199|||7323||599107||7323|||4438|44|||2334.20|64.41||210.07||169.94|356.13| +2451645|50639|17470|29819|1500946|4873|22800|29819|1500946|4873|22800|28|13|4440|11|955.57|38.22|993.79|0.66|129.14|363.11|562.83|29.63|168.02| +2452915|5950|2676|54412|1590790|3521|27899|54412|1590790|3521|27899|12|19|4441|40|288.00|2.88|290.88|61.49|152.00|129.60|42.76|115.64|216.37| +2452870|75465|456|19954|1447509|5017|44972|19954|1447509|5017|44972|51|25|4441|26|12.48|1.12|13.60|75.72|533.00|4.99|2.47|5.02|609.84| +2452836|49761|14085|97854|1847823|3823|3483|97854|1847823|3823|3483|54|10|4441|38|41.42|2.48|43.90|59.24|623.58|40.59|0.79|0.04|685.30| +2452710|68849|15417|42702|1717723|4391|26212|42702|1717723|4391|26212|3|26|4442|2|309.94|15.49|325.43|74.27|156.78|179.76|118.46|11.72|246.54| +2452710|81581|15954|24066|368208|703|16045|24066|368208|703|16045|43|6|4442|27|188.73|0.00|188.73|8.96|1384.56|9.43|23.30|156.00|1393.52| +2451367|78276|1028|||||38061|||9821|||4444||51.76|1.03||4.16|46.32|||5.11|| +2451579|2298|17498|18655|1054554|2100|799|18655|1054554|2100|799|2|12|4446|36|6053.76|121.07|6174.83|9.78|2548.80|1573.97|2015.90|2463.89|2679.65| +2451559|75243|1612|84575|175890|3652|22567|84575|175890|3652|22567|14|2|4448|61|107.36|9.66|117.02|36.45|1021.14|47.23|33.07|27.06|1067.25| +2451580|45115|6812|49877|1668873|5469|34065|49877|1668873|5469|34065|16|8|4449|45|2376.90|71.30|2448.20|59.35|576.00|1022.06|948.38|406.46|706.65| +2452622|32868|15711|99161|389938|2315|24292|99161|389938|2315|24292|51|24|4450|9|338.04|6.76|344.80|85.70|22.50|47.32|154.08|136.64|114.96| +2452669|62996|2499|39100|795055|5057|25812|39100|795055|5057|25812|21|17|4450|7|548.66|38.40|587.06|71.79|93.66|384.06|52.67|111.93|203.85| +2451294|68167|14456|76037|1393102|5720|48738|76037|1393102|5720|48738|2|21|4452|43|2885.73|173.14|3058.87|23.15|2176.66|2337.44|241.24|307.05|2372.95| +2451447|74389|14653|43411|1806739|1267|30195|43411|1806739|1267|30195|19|32|4452|20|843.40|25.30|868.70|96.54|91.00|236.15|72.87|534.38|212.84| +2452187|66327|16871|29343|359795|1800|7052|29343|359795|1800|7052|41|1|4453|48|1765.92|35.31|1801.23|94.47|1765.92|1642.30|56.86|66.76|1895.70| +2452185|40591|16869|86695|1569770|492|17967|86695|1569770|492|17967|47|26|4453|17|277.10|19.39|296.49|9.39|38.76|36.02|96.43|144.65|67.54| +2452170|37338|9111|84790|1526754|5165|3129|84790|1526754|5165|3129|5|20|4453|56|7185.36|143.70|7329.06|41.84|145.04|862.24|4995.26|1327.86|330.58| +2451913|55548|4027|42333|4943|4353|44563|42333|4943|4353|44563|14|3|4454|56|211.68|16.93|228.61|94.86|172.48|52.92|125.42|33.34|284.27| +2451815|81437|4280|89074|137456|6638|21467|89074|137456|6638|21467|26|24|4457|16|698.08|55.84|753.92|47.65|38.72|551.48|117.28|29.32|142.21| +2451272|77373|8854|3049|906046|6196|35546|3049|906046|6196|35546|19|11|4461|10|686.80|61.81|748.61|54.71|98.10|130.49|150.20|406.11|214.62| +2451824|18814|8845|3882|153559|1636|46066|3882|153559|1636|46066|23|1|4462|19|1730.90|121.16|1852.06|28.13|200.64|1263.55|462.67|4.68|349.93| +2452704|33196|3411|85554|1390050|146|2595|85554|1390050|146|2595|49|4|4464|61|3839.95|0.00|3839.95|43.51|548.39|2303.97|1382.38|153.60|591.90| +2452569|43770|15037|2107|793034|2361|15288|2107|793034|2361|15288|51|5|4464|18|3.42|0.10|3.52|13.80|36.36|2.29|0.10|1.03|50.26| +2451706|15419|350|42477|1717704|3778|20252|42477|1717704|3778|20252|2|22|4465|37|937.95|9.37|947.32|75.71|1178.45|328.28|469.44|140.23|1263.53| +2451101|36127|15980|97354|1547409|3864|45519|97354|1547409|3864|45519|2|21|4466|18|275.22|0.00|275.22|88.85|181.98|145.86|112.54|16.82|270.83| +2451691|32048|2194|75943|1623196|1296|17796|75943|1623196|1296|17796|22|27|4467|54|9522.90|476.14|9999.04|51.97|2825.28|857.06|2773.06|5892.78|3353.39| +2451624|73574|8822|87173|1051021|1022|15478|87173|1051021|1022|15478|43|22|4467|1|117.92|8.25|126.17|9.66|99.16|17.68|52.12|48.12|117.07| +2451568|29588|13304|17229|1239287|5345|24603|17229|1239287|5345|24603|37|14|4467|5|105.50|6.33|111.83|67.87|52.75|89.67|5.06|10.77|126.95| +2452346|16247|2215|83431|1745690|4749|47008|83431|1745690|4749|47008|25|17|4468|14|223.72|0.00|223.72|46.52|615.44|46.98|109.57|67.17|661.96| +2452397||8507|||4352||78799|1551399|4352||25||4468|14|||||705.18|523.60|272.08|173.96|818.06| +2452891|75191|10237|20450|627410|770|23777|20450|627410|770|23777|13|26|4470|11|1828.09|146.24|1974.33|62.91|851.40|402.17|99.81|1326.11|1060.55| +2452748|15806|11421|63186|1094317|648|14132|63186|1094317|648|14132|33|31|4470|28|305.48|0.00|305.48|41.74|49.56|164.95|75.88|64.65|91.30| +2452971|71261|17755|43114|614931|2395|40612|43114|614931|2395|40612|43|6|4470|39|1528.02|61.12|1589.14|7.42|1437.93|382.00|1123.09|22.93|1506.47| +2452471|49972|13763|91053|63821|1742|34910|91053|63821|1742|34910|25|12|4471|9|35.55|0.00|35.55|1.58|13.05|23.10|6.10|6.35|14.63| +2452795|48900|12909|91275|275186|1944|17502|91275|275186|1944|17502|30|29|4472|23|190.44|7.61|198.05|41.41|31.74|119.97|40.16|30.31|80.76| +2451805|60571|1312||||13915|73297||4760|13915||29|4473|21|1305.15|91.36|||771.12|247.97||697.74|940.50| +2452043|16605|16435|14646|1248063|1480|45297|14646|1248063|1480|45297|20|15|4474|5|27.05|1.89|28.94|59.17|36.05|0.27|4.82|21.96|97.11| +2452396|41661|8733|61400|1218209|4971|37780|61400|1218209|4971|37780|33|18|4477|21|492.45|9.84|502.29|46.92|1231.23|187.13|180.13|125.19|1287.99| +2452492|28668|4427|64753|1355248|6096|20591|64753|1355248|6096|20591|51|30|4477|1|36.42|1.82|38.24|77.19|2.10|34.96|0.52|0.94|81.11| +2452500|71610|1321|27580|332004|5228|5205|27580|332004|5228|5205|29|25|4477|2|8.04|0.64|8.68|5.50|7.62|5.46|0.36|2.22|13.76| +2452221|36926|6035|29722|1510342|4209|4876|29722|1510342|4209|4876|33|29|4478|49|3871.00|348.39|4219.39|70.68|2377.48|1819.37|636.00|1415.63|2796.55| +2452904|61461|11514|37902|1527252|1049|21599|37902|1527252|1049|21599|24|27|4479|45|5474.25|383.19|5857.44|7.47|1390.05|821.13|3722.49|930.63|1780.71| +2452773|46774|9031|46475|145859|796|29733|46475|145859|796|29733|33|24|4479|20|194.40|9.72|204.12|72.52|454.00|79.70|3.44|111.26|536.24| +2452683|82456|9861|93984|1612576|415|31407|93984|1612576|415|31407|43|1|4479|20|1677.80|50.33|1728.13|10.08|19.20|318.78|924.13|434.89|79.61| +2452640|26895|16129|23218|384598|1531|34911|23218|384598|1531|34911|19|32|4480|27|1681.29|67.25|1748.54|8.75|1200.96|907.89|177.88|595.52|1276.96| +2452614|38919|16381|81560|1767036|2112|26388|81560|1767036|2112|26388|60|32|4480|41|16.81|1.00|17.81|53.43|409.59|8.40|8.07|0.34|464.02| +2451257|78572|12742|51473|1896625|4977|6882|51473|1896625|4977|6882|2|7|4481|16|892.16|26.76|918.92|1.16|501.76|187.35|704.81|0.00|529.68| +2451382|74387|2317|6427|375300|4173|22693|6427|375300|4173|22693|50|3|4481|10|172.80|5.18|177.98|60.88|120.50|29.37|103.26|40.17|186.56| +2452041|27926|17953|54906|890745|3031|24070|54906|890745|3031|24070|50|7|4482|6|573.42|22.93|596.35|10.43|58.50|269.50|36.47|267.45|91.86| +2451890|65440|2563|91041|1803538|3235|30532|91041|1803538|3235|30532|53|5|4482|49|5602.66|280.13|5882.79|13.20|2374.05|2801.33|896.42|1904.91|2667.38| +2451774|44980|6995|86790|1782929|4221|25960|86790|1782929|4221|25960|56|33|4483|2|181.50|7.26|188.76|40.99|40.32|58.08|114.78|8.64|88.57| +2451899|74590|14390|79839|496772|6505|21438|79839|496772|6505|21438|7|1|4483|41|3289.02|296.01|3585.03|95.38|1525.61|2368.09|414.41|506.52|1917.00| +2451269|49031|5431|29303|1140808|2713|6665|29303|1140808|2713|6665|1|7|4484|39|1886.82|56.60|1943.42|74.98|607.23|1754.74|17.17|114.91|738.81| +2451257|30156|17305|26973|1785863|3953|2443|26973|1785863|3953|2443|34|4|4484|84|1465.80|58.63|1524.43|91.10|1233.96|762.21|689.51|14.08|1383.69| +2451170|26541|9176|29820|1721865|5110|14755|29820|1721865|5110|14755|56|25|4484|19|887.68|35.50|923.18|6.92|766.65|35.50|818.09|34.09|809.07| +2451131|65430|5728|29466|686068|2670|34072|29466|686068|2670|34072|22|26|4484|24|1587.36|15.87|1603.23|52.70|2503.44|158.73|1342.91|85.72|2572.01| +2452575|42958|7909|58498|496189|1433|21924|58498|496189|1433|21924|36|14|4485|16|1643.20|98.59|1741.79|58.54|802.40|98.59|741.41|803.20|959.53| +2452511|58181|12879|22148|1751329|3263|47644|22148|1751329|3263|47644|13|19|4485|12|462.00|4.62|466.62|75.94|98.52|318.78|47.26|95.96|179.08| +2451578|44039|12049|97156|403554|3105|33903|97156|403554|3105|33903|16|30|4486|4|382.80|30.62|413.42|85.84|77.52|359.83|10.33|12.64|193.98| +2451732|80251|818|23319|555174|1782|37237|23319|555174|1782|37237|16|24|4486|19|599.83|11.99|611.82|90.10|543.21|83.97|216.66|299.20|645.30| +2451689|50282|12997|73325|1915133|1741|13102|73325|1915133|1741|13102|46|13|4486|31|6437.77|515.02|6952.79|20.64|1355.32|5021.46|665.66|750.65|1890.98| +2452014|47392|15799|43508|1201956|404|22452|43508|1201956|404|22452|55|2|4487|2|177.92|7.11|185.03|39.07|103.14|28.46|71.74|77.72|149.32| +2452254|73690|1601|59889|1559742|3242|6695|59889|1559742|3242|6695|29|21|4488|17|1428.34|0.00|1428.34|16.28|75.14|1128.38|272.96|27.00|91.42| +2452237|56145|6889|54004|445240|2876|7712|54004|445240|2876|7712|59|11|4488|14|637.98|57.41|695.39|53.18|57.96|599.70|36.36|1.92|168.55| +2451839|43394|3461|26638|1062906|774|5442|26638|1062906|774|5442|55|10|4491|16|1562.72|93.76|1656.48|21.83|959.84|125.01|402.55|1035.16|1075.43| +2451759|66752|8075|44735|352338|5541|9544|44735|352338|5541|9544|26|27|4491|51|4479.84|403.18|4883.02|63.97|2850.90|89.59|1975.61|2414.64|3318.05| +2451699|59467|3622|27515|760869|3023|13577|27515|760869|3023|13577|46|25|4494|4|315.52|9.46|324.98|25.29|60.24|97.81|113.20|104.51|94.99| +2451751|47698|10297|18831|766895|2606|29917|18831|766895|2606|29917|49|16|4494|70|1499.40|59.97|1559.37|98.97|512.40|119.95|27.58|1351.87|671.34| +2451964|60348|4571||||||||||24|4495|35||||||679.70|||| +2452074|70094|16553|43395|53579|916|8300|43395|53579|916|8300|11|22|4495|6|365.34|7.30|372.64|50.16|8.58|171.70|94.88|98.76|66.04| +2452171|23321|3839|45111|724332|6510|28622|45111|724332|6510|28622|43|19|4495|28|2931.60|263.84|3195.44|24.08|1025.92|2579.80|49.25|302.55|1313.84| +|68847|6552||8550||6671|43666|8550|||42||4496|1|||54.34|92.12|71.22|38.20|2.49|13.12|| +2452684|21876|7968|30190|1684327|6464|21801|30190|1684327|6464|21801|30|19|4496|26|406.12|24.36|430.48|74.00|76.18|60.91|10.35|334.86|174.54| +2452777|31464|4167|9258|1575677|1296|6115|9258|1575677|1296|6115|25|22|4496|60|34.20|2.39|36.59|67.86|66.00|23.94|0.20|10.06|136.25| +2451260|36468|818|22695|1083437|4179|30121|22695|1083437|4179|30121|4|8|4497|25|839.00|0.00|839.00|92.13|610.00|687.98|57.38|93.64|702.13| +2451651|37042|13255|6328|873497|4666|30588|6328|873497|4666|30588|46|26|4500|3|6.15|0.06|6.21|39.27|50.61|4.61|0.81|0.73|89.94| +2451896|57610|11423|61938|1315965|1992|9219|61938|1315965|1992|9219|44|16|4501|18|416.34|37.47|453.81|14.03|666.18|133.22|260.47|22.65|717.68| +2451900|43328|1471|51094|520156|6371|12586|51094|520156|6371|12586|7|32|4502|19|9.31|0.65|9.96|58.30|174.80|2.88|1.92|4.51|233.75| +2451429|51796|12014|74223|1492348|5320|30292|74223|1492348|5320|30292|25|9|4503|11|102.52|2.05|104.57|93.42|153.78|100.46|1.79|0.27|249.25| +2451673|37626|9856|95805|740506|6807|45461|95805|740506|6807|45461|43|18|4504|34|45.22|0.45|45.67|72.61|19.72|4.06|14.81|26.35|92.78| +2452066|75076|10829|29051|1517952|6053|7354|29051|1517952|6053|7354|53|22|4505|31|2660.73|159.64|2820.37|16.73|1169.01|2341.44|312.90|6.39|1345.38| +2452004|69544|15902|87776|379199|5345|35535|87776|379199|5345|35535|25|13|4505|2|59.60|2.38|61.98|6.06|52.72|25.62|24.46|9.52|61.16| +|81640|344|62959||1929|48614|62959|||48614||23|4505|12|229.80|2.29|232.09||72.36|149.37|61.12||174.09| +2451979|67785|16349|80186|588585|5431|17430|80186|588585|5431|17430|49|11|4505|61|8174.00|326.96|8500.96|87.19|1021.75|2697.42|4436.02|1040.56|1435.90| +2451999|7155|13127|24857|1675608|1559|11510|24857|1675608|1559|11510|8|1|4508|9|653.67|45.75|699.42|8.90|383.94|209.17|346.71|97.79|438.59| +2451993|33680|6761|65065|1262424|5383|25989|65065|1262424|5383|25989|17|3|4508|46|3018.06|0.00|3018.06|98.48|2029.52|241.44|194.36|2582.26|2128.00| +2451921|41753|11630|97962|1104051|4540|20229|97962|1104051|4540|20229|41|28|4508|7|584.50|11.69|596.19|31.12|6.16|333.16|133.21|118.13|48.97| +2452736|12120|4233|29842|581703|5917|11331|29842|581703|5917|11331|3|32|4509|37|2505.27|175.36|2680.63|54.22|780.70|2004.21|195.41|305.65|1010.28| +2451185|28882|1288|8348|932702|2656|43057|8348|932702|2656|43057|13|10|4510|17|750.38|52.52|802.90|65.80|421.26|240.12|341.87|168.39|539.58| +2451121|31301|3362|60267|5174|128|34635|60267|5174|128|34635|13|8|4511|94|11664.46|816.51|12480.97|50.71|5340.14|2682.82|1616.69|7364.95|6207.36| +2451130|78107|13892|31855|744293|731|20416|31855|744293|731|20416|22|6|4511|17|404.60|24.27|428.87|3.52|235.96|400.55|3.15|0.90|263.75| +2451246|10773|15770|21216|1769586|1603|21173|21216|1769586|1603|21173|56|17|4511|24|3671.52|183.57|3855.09|49.90|1136.40|1909.19|1004.52|757.81|1369.87| +2451872|38394|17849|29820|620824|981|5985|29820|620824|981|5985|49|11|4512|16|211.04|8.44|219.48|86.44|8.00|48.53|63.37|99.14|102.88| +2452360|82397|7929|93649|819775|5901|16027|93649|819775|5901|16027|25|27|4513|32|3061.12|0.00|3061.12|77.62|3443.84|1132.61|713.54|1214.97|3521.46| +2452512|43326|8993|38542|1449691|3593|11495|38542|1449691|3593|11495|43|9|4513|5|355.15|7.10|362.25|89.04|314.90|159.81|19.53|175.81|411.04| +2452418|74452|10655|30943|570844|4698|17799|30943|570844|4698|17799|55|4|4513|30|384.60|30.76|415.36|50.55|1423.80|26.92|325.48|32.20|1505.11| +2451540|37753|140|23254|316489|4880|48392|23254|316489|4880|48392|28|30|4514|45|2621.25|104.85|2726.10|36.76|270.00|209.70|1278.12|1133.43|411.61| +2451729|71230|5396|79422|2798|2974|35103|79422|2798|2974|35103|26|33|4514|81|7473.06|0.00|7473.06|80.63|112.59|1644.07|2389.88|3439.11|193.22| +2451936|74401|5840|74862|792056|6497|20891|74862|792056|6497|20891|2|9|4515|1|34.59|1.38|35.97|86.28|67.80|21.09|12.82|0.68|155.46| +2451196|72826|7910|52846|237817|835|15806|52846|237817|835|15806|8|24|4517|6|76.02|0.00|76.02|35.22|86.34|25.08|46.35|4.59|121.56| +2451646|44643|6208|42762|1700243|477|14711|42762|1700243|477|14711|4|21|4519|1|42.00|2.52|44.52|92.04|18.20|0.84|25.51|15.65|112.76| +2451249|67106|6922|3939|1321718|2250|43182|3939|1321718|2250|43182|32|18|4521|17|146.88|11.75|158.63|95.41|92.99|26.43|26.49|93.96|200.15| +2451251|39836|11212|73016|1316493|7170|8247|73016|1316493|7170|8247|14|5|4521|17|2265.59|90.62|2356.21|28.01|1328.04|362.49|247.40|1655.70|1446.67| +2451774|11998|1727|93735|521565|4595|28425|93735|521565|4595|28425|13|26|4522|28|3713.08|259.91|3972.99|19.21|2455.32|1076.79|1819.04|817.25|2734.44| +2451882|48844|1117|79861|1024282|2750|15618|79861|1024282|2750|15618|35|6|4522|73|1542.49|138.82|1681.31|66.45|1428.61|1126.01|308.19|108.29|1633.88| +2452816|29058|1548|46389|670525|708|1244|46389|670525|708|1244|37|22|4523|15|690.30|34.51|724.81|54.25|512.55|593.65|81.18|15.47|601.31| +2452192|33947|11408|50577|110784|2638|48892|50577|110784|2638|48892|29|9|4524|4|88.00|7.92|95.92|31.07|167.24|18.48|52.14|17.38|206.23| +2451058|77230|16870|97484|56715|6346|41530|97484|56715|6346|41530|13|34|4525|51|9473.25|663.12|10136.37|84.65|2205.75|0.00|1042.05|8431.20|2953.52| +2451614|30874|2773|46316|1520210|5657|42351|46316|1520210|5657|42351|20|8|4526|6|247.02|4.94|251.96|72.97|64.44|83.98|146.73|16.31|142.35| +2451357|40500|8176|21284|45852|5895|2701|21284|45852|5895|2701|28|16|4528|44|2058.32|144.08|2202.40|67.32|416.24|1337.90|655.58|64.84|627.64| +2451241|64500|11470|26489|1125849|4762|12122|26489|1125849|4762|12122|52|11|4528|10|56.60|2.83|59.43|99.41|110.20|56.03|0.11|0.46|212.44| +2451289|61854|10984|56807|549312|217|40322|56807|549312|217|40322|25|12|4528|6|855.66|42.78|898.44|31.72|30.90|94.12|38.07|723.47|105.40| +2451892|35143|15781|43993|586770|4747|32371|43993|586770|4747|32371|44|5|4530|3|8.79|0.00|8.79|11.05|1.74|8.52|0.01|0.26|12.79| +2451734|31132|712||1287378||1017|9913|1287378|4279|1017|4||4531|||||52.57|477.12||||| +2451125|70370|13396|75803|912204|2558|14559|75803|912204|2558|14559|56|34|4533|51|186.66|0.00|186.66|7.18|979.71|154.92|0.95|30.79|986.89| +2451790|78465|8390|56201|1423425|2649|33026|56201|1423425|2649|33026|13|13|4534|18|1554.30|0.00|1554.30|80.27|706.50|1243.44|108.80|202.06|786.77| +2451750|33849|6745|47212|1244995|6425|42655|47212|1244995|6425|42655|26|28|4534|85|227.80|13.66|241.46|45.56|113.90|88.84|8.33|130.63|173.12| +2451235|43489|12190|37139|1581924|1909|20294|37139|1581924|1909|20294|58|27|4535|13|322.01|25.76|347.77|17.45|823.03|161.00|45.08|115.93|866.24| +2451247|55926|9799|28592|107452|4712|27972|28592|107452|4712|27972|8|27|4535|22|434.28|34.74|469.02|46.25|2544.52|186.74|235.16|12.38|2625.51| +2452629|39498|3049|19483|1164633|3736|24486|19483|1164633|3736|24486|42|14|4536|35|1903.30|133.23|2036.53|87.56|353.15|799.38|629.23|474.69|573.94| +2451032|45548|14192|64253|227737|6421||64253||6421|7905||4|4537||||||46.30|155.02|83.71||108.23| +2451340|64914|5638|13396|152278|760|14442|13396|152278|760|14442|40|3|4538|50|406.00|4.06|410.06|2.03|565.50|40.60|124.23|241.17|571.59| +2451155|69300|832|4037|1248113|6142|29418|4037|1248113|6142|29418|13|12|4538|6|63.90|0.00|63.90|76.19|207.84|18.53|14.51|30.86|284.03| +2451274|14313|11636|35000|104901|6449|45950|35000|104901|6449|45950|32|25|4538|5|118.75|8.31|127.06|16.80|93.45|58.18|44.21|16.36|118.56| +2451311|57733|2035|96931|1782390|4649|24908|96931|1782390|4649|24908|28|18|4538|19|285.76|5.71|291.47|70.61|142.88|265.75|5.60|14.41|219.20| +2451093|27076|10543|55130|1323023|1316|10056|55130|1323023|1316|10056|22|31|4538|22|1083.72|43.34|1127.06|85.89|971.74|455.16|131.99|496.57|1100.97| +2451685|42284|14192|81877|1800116|5108|35347|81877|1800116|5108|35347|20|7|4544|16|2835.36|28.35|2863.71|4.95|430.56|2495.11|299.42|40.83|463.86| +2451313|37891|3622|33130|1067616|55|39741|33130|1067616|55|39741|19|8|4545|66|8471.76|677.74|9149.50|28.93|1239.48|8048.17|139.78|283.81|1946.15| +2452283|53712|17521|96524|479298|4919|49293|96524|479298|4919|49293|39|4|4546|29|2842.29|170.53|3012.82|62.63|1177.40|198.96|185.03|2458.30|1410.56| +2452490|78252|16809|61794|1517070|645|5072|61794|1517070|645|5072|57|7|4546|22|1608.64|112.60|1721.24|46.17|1103.08|289.55|976.12|342.97|1261.85| +2451270|27056|6220|80963|261105|3647|2542|80963|261105|3647|2542|10|12|4547|66|1852.62|55.57|1908.19|83.59|992.64|1445.04|232.32|175.26|1131.80| +2451076|46504|13628|95633|788907|581|37855|95633|788907|581|37855|38|16|4548|4|5.68|0.51|6.19|34.03|6.84|2.44|3.11|0.13|41.38| +2451185|43800|6950|53676|1490817|3958|38346|53676|1490817|3958|38346|49|11|4548|7|91.21|2.73|93.94|52.30|5.32|78.44|7.53|5.24|60.35| +2451155|37441|1411|22322|415236|4950|25111|22322|415236|4950|25111|46|19|4548|1|3.93|0.19|4.12|88.92|14.16|0.19|3.62|0.12|103.27| +2452216|34205|12721|72187|893012|3084|8756|72187|893012|3084|8756|47|21|4549|49|11500.30|1035.02|12535.32|61.52|1290.66|11270.29|75.90|154.11|2387.20| +2451360|78583|7474|51193|27097|1401|29658|51193|27097|1401|29658|26|10|4550|76|1130.88|33.92|1164.80|70.42|48.64|339.26|356.22|435.40|152.98| +2451716|44242|10015|23992|1637967|3495|28670|23992|1637967|3495|28670|1|7|4551|7|446.81|17.87|464.68|52.66|496.44|259.14|150.13|37.54|566.97| +2451779|19938|5996|58796|1695621|3646|10824|58796|1695621|3646|10824|14|34|4551|7|110.74|1.10|111.84|53.13|32.13|94.12|1.32|15.30|86.36| +2451061|81770|15694|42184|485081|331|13628|42184|485081|331|13628|28|26|4552|11|1051.93|31.55|1083.48|47.17|370.37|189.34|517.55|345.04|449.09| +2451203|76565|10426|61126|176426|973|15473|61126|176426|973|15473|58|32|4552|4|151.44|12.11|163.55|56.26|144.24|112.06|32.68|6.70|212.61| +2451835|67915|16826|17242|1852101|560|7819|17242|1852101|560|7819|35|33|4553|11|394.35|11.83|406.18|94.27|113.30|185.34|43.89|165.12|219.40| +2451806|22728|2090|30316|599484|3499|36314|30316|599484|3499|36314|2|8|4554|62|10218.22|817.45|11035.67|52.24|4280.48|4393.83|2504.48|3319.91|5150.17| +2451820||14792|||||||41||||4556|2|||17.97||74.68|16.45|||155.68| +2451029||14120|72397||5269|24552||928033||24552|10|25|4560|7|251.51|2.51||||||2.37|| +2451109|53808|4714|77107|1488811|1762|666|77107|1488811|1762|666|7|14|4560|42|1482.60|14.82|1497.42|59.47|2763.18|148.26|1080.81|253.53|2837.47| +2451959|75681|1625|24686|1309963|5640|42517|24686|1309963|5640|42517|56|22|4562|9|47.97|0.00|47.97|93.26|1.98|32.13|6.17|9.67|95.24| +2452023|60432|8978|98838|1870389|1140|37938|98838|1870389|1140|37938|14|21|4562|6|641.22|12.82|654.04|44.05|206.82|141.06|320.10|180.06|263.69| +2451672|46145|7193|80933|214514|6495|16076|80933|214514|6495|16076|37|12|4563|16|700.48|0.00|700.48|32.72|175.04|14.00|137.29|549.19|207.76| +||1015||1239048||9575|||4335||||4564|||33.71|||374.40||||| +2451236|44927|2587|90319|1811879|7039|18599|90319|1811879|7039|18599|52|17|4565|45|4859.55|437.35|5296.90|30.61|3064.95|437.35|3891.53|530.67|3532.91| +2451531|67245|12766|13365|1865747|882|16017|13365|1865747|882|16017|26|6|4566|2|19.52|1.56|21.08|12.45|30.68|5.66|0.55|13.31|44.69| +2451794|9449|13418|24006|928207|6808|45056|24006|928207|6808|45056|10|34|4567|28|6126.12|490.08|6616.20|2.83|403.76|2818.01|3076.54|231.57|896.67| +2451561|65989|16669|26996|358139|482|8825|26996|358139|482|8825|1|28|4567|24|182.64|0.00|182.64|8.59|296.88|78.53|99.94|4.17|305.47| +2451788|46392|7|7952|1600267|5382|28097|7952|1600267|5382|28097|52|5|4567|11|807.29|8.07|815.36|54.93|273.24|395.57|139.98|271.74|336.24| +||5644|7600||4274||7600|865020|4274|||5|4568|1||1.05||77.00|||||101.42| +2452332|42293|10343|37450|524067|1884|28854|37450|524067|1884|28854|43|9|4572|12|16.92|0.67|17.59|7.94|73.44|3.04|9.71|4.17|82.05| +2452382|41137|9453|20927|67879|3184|42136|20927|67879|3184|42136|13|28|4572|39|1127.10|67.62|1194.72|70.91|2017.08|507.19|551.71|68.20|2155.61| +2452262|46248|5247|78359|246011|4108|1432|78359|246011|4108|1432|17|16|4572|15|1071.15|96.40|1167.55|25.29|187.80|867.63|14.24|189.28|309.49| +2451693|41452|14546|35971|1288301|2678|5008|35971|1288301|2678|5008|38|25|4574|53|6440.56|322.02|6762.58|37.19|706.49|6247.34|36.71|156.51|1065.70| +2451662|50350|12442|52540|62222|3904|6833|52540|62222|3904|6833|14|25|4574|11|68.86|1.37|70.23|84.11|60.17|59.90|8.33|0.63|145.65| +2452688|69967|5190|2448|135678|3755|20985|2448|135678|3755|20985|51|16|4575|5|75.10|6.00|81.10|32.37|43.80|51.06|0.96|23.08|82.17| +2451662|81388|868|98219|1483209|1799|35989|98219|1483209|1799|35989|43|22|4580|25|83.75|7.53|91.28|92.33|43.50|72.02|7.38|4.35|143.36| +2451652|32118|15680|70398|1760085|4172|47459|70398|1760085|4172|47459|58|8|4580|6|47.52|4.27|51.79|7.05|52.56|38.01|0.66|8.85|63.88| +2451005|10977|17383|5361|189853|3937|20329|5361|189853|3937|20329|19|13|4582|5|215.30|4.30|219.60|92.87|45.05|92.57|66.27|56.46|142.22| +2451078|14975|13148|31542|1397275|608|30741|31542|1397275|608|30741|55|27|4582|22|1372.80|41.18|1413.98|24.67|1307.46|631.48|303.94|437.38|1373.31| +2452122|19431|14681|53592|1042341|1137|13133|53592|1042341|1137|13133|43|27|4583|26|894.40|17.88|912.28|70.29|841.88|590.30|176.37|127.73|930.05| +2452322|61029|6723|6417|506560|2993|42546|6417|506560|2993|42546|3|14|4583|18|1276.92|38.30|1315.22|92.37|120.42|1136.45|136.25|4.22|251.09| +2451795|5012|16480|38373|1532707|2241|31375|38373|1532707|2241|31375|8|4|4584|20|666.00|59.94|725.94|50.59|428.20|632.70|9.99|23.31|538.73| +2451715|21402|4850|14672|1065095|4671|20694|14672|1065095|4671|20694|2|25|4585|47|1312.24|118.10|1430.34|8.12|1148.21|957.93|290.53|63.78|1274.43| +2451620|75096|10694||1194724||46964|1757|1194724|44||||4585|54|108.54|1.08||60.50|25.38||||86.96| +2451346|38164|235|86375|347417|6059|6701|86375|347417|6059|6701|10|16|4586|53|7202.70|504.18|7706.88|85.80|3041.14|7058.64|2.88|141.18|3631.12| +2451274|8621|6740|88088|952579|5070|2361|88088|952579|5070|2361|46|25|4586|3|147.00|5.88|152.88|53.87|103.20|102.90|36.16|7.94|162.95| +2452202|45918|1307|25402|1575406|4257|7861|25402|1575406|4257|7861|17|22|4587|15|108.60|2.17|110.77|11.03|86.85|85.79|21.89|0.92|100.05| +2452240|58314|11480|90515|1246986|610|45481|90515|1246986|610|45481|47|35|4587|4|56.92|1.13|58.05|86.77|148.68|40.98|6.69|9.25|236.58| +2451242|28137|16762|44110|468768|5073|4931|44110|468768|5073|4931|20|35|4589|9|106.38|3.19|109.57|8.32|83.97|44.67|38.87|22.84|95.48| +2452525|37551|5667|42133|362117|6548|41779|42133|362117|6548|41779|35|30|4590|46|1244.30|99.54|1343.84|57.69|2058.50|298.63|595.77|349.90|2215.73| +2452765|40086|5088|14325|119610|4949|2320|14325|119610|4949|2320|27|26|4592|16|578.40|23.13|601.53|49.34|169.28|196.65|374.11|7.64|241.75| +2452488|33027|11568|67170|712316|662|30208|67170|712316|662|30208|7|12|4593|4|90.28|4.51|94.79|94.73|72.20|31.59|50.47|8.22|171.44| +2452524|26658|16854|18347|630984|876|34967|18347|630984|876|34967|43|26|4593|56|1575.84|63.03|1638.87|92.09|818.72|583.06|724.72|268.06|973.84| +2452737|2573|3405|77336|1739702|5482|18271|77336|1739702|5482|18271|60|31|4594|11|60.61|0.60|61.21|35.78|191.29|7.27|46.40|6.94|227.67| +2452788|66703|8547|15416|1541055|5094|13486|15416|1541055|5094|13486|36|30|4594|30|851.40|76.62|928.02|36.95|53.10|527.86|323.54|0.00|166.67| +2452534|6139|8581|12892|1782520|||12892||2195|18223|||4595||1010.40||1040.71|63.96|77.70|484.99|||171.97| +2452467|60059|6949|13196|561265|6581|3897|13196|561265|6581|3897|1|2|4595|1|39.73|3.57|43.30|35.59|41.94|0.39|19.67|19.67|81.10| +2452599|66741|1333|65710|1914005|4700|30642|65710|1914005|4700|30642|24|2|4595|61|3111.00|279.99|3390.99|90.74|3499.57|1213.29|1347.37|550.34|3870.30| +2451423|4684|7513|31958|1020568|5625|34682|31958|1020568|5625|34682|55|25|4597|4|256.56|7.69|264.25|23.26|29.60|43.61|10.64|202.31|60.55| +2451771|52647|4564|33062|508932|5249|38719|33062|508932|5249|38719|28|24|4598|50|1040.00|0.00|1040.00|85.23|614.50|135.20|18.09|886.71|699.73| +2451750|12844|12826|5818|537603|5741|326|5818|537603|5741|326|19|9|4598|8|458.96|22.94|481.90|49.51|30.56|192.76|186.34|79.86|103.01| +|58994|4479||698122|3229|23136||||23136|51|10|4599||766.94|||23.39|395.56|||73.79|418.95| +2452346|28409|12581|37249|541122|2242|32115|37249|541122|2242|32115|57|20|4601|12|795.84|71.62|867.46|50.95|719.28|652.58|31.51|111.75|841.85| +2452473|59205|17013|11030|1177814|1398|36935|11030|1177814|1398|36935|7|10|4601|3|26.34|0.79|27.13|54.09|115.92|19.75|5.53|1.06|170.80| +2452653|42739|6744|79471|707585|1299|9413|79471|707585|1299|9413|13|14|4602|27|4182.84|0.00|4182.84|97.75|1207.44|209.14|993.42|2980.28|1305.19| +2452822|56851|13170|53764|360746|881|28663|53764|360746|881|28663|57|4|4604|13|110.89|7.76|118.65|21.21|277.42|89.82|2.73|18.34|306.39| +2452117|67463|595|60601|156896|6791|286|60601|156896|6791|286|39|31|4605|26|0.00|0.00|0.00|81.35|1004.38|0.00|0.00|0.00|1085.73| +2452146|82893|2071|25399|1882381|3716|17098|25399|1882381|3716|17098|57|14|4605|4|426.44|38.37|464.81|66.83|89.24|230.27|78.46|117.71|194.44| +2451187|57209|7990|50724|356544|6011|27078|50724|356544|6011|27078|52|3|4607|40|2439.60|195.16|2634.76|98.13|906.00|1439.36|910.21|90.03|1199.29| +2451206|30344|11560|18003|940747|1091|49435|18003|940747|1091|49435|1|6|4607|50|11137.00|111.37|11248.37|12.60|3426.50|6904.94|338.56|3893.50|3550.47| +2451401|48138|4621|9829|1705089|7102|45562|9829|1705089|7102|45562|44|4|4608|30|1650.00|99.00|1749.00|28.53|1885.50|1254.00|146.52|249.48|2013.03| +2451376|29216|4237|14770|960113|1456|17999|14770|960113|1456|17999|32|1|4608|75|9450.75|94.50|9545.25|13.74|2100.00|4819.88|277.85|4353.02|2208.24| +||2725||953394|400|2653||953394||2653|53|30|4609||2057.88|41.15||37.22|||111.13|12.35|5163.01| +2452461|69740|14529|50032|1210760|1088|33416|50032|1210760|1088|33416|35|11|4609|34|2147.44|42.94|2190.38|83.83|429.42|1138.14|504.65|504.65|556.19| +2452066|74788|845|29275|113276|1277|35725|29275|113276|1277|35725|1|1|4610|54|3249.18|32.49|3281.67|92.04|850.50|1462.13|428.89|1358.16|975.03| +2452542|45366|16745|92703|1769242|628||||628|17614|7|15|4612|10|||||||2.61|4.87|339.73| +2451920|45866|16705|62331|1329091|6604|33317|62331|1329091|6604|33317|55|31|4613|11|1345.74|121.11|1466.85|72.92|455.73|1090.04|155.97|99.73|649.76| +2451814|33853|4028|85395|457648|5548|9001|85395|457648|5548|9001|41|11|4613|72|417.60|8.35|425.95|54.33|2159.28|12.52|186.33|218.75|2221.96| +2451611|11670|686|50831|459452|5320|15941|50831|459452|5320|15941|50|33|4613|31|2829.99|28.29|2858.28|42.83|1271.93|2433.79|249.60|146.60|1343.05| +2452026|35686|6866|11639|1825700|4561|575|11639|1825700|4561|575|55|2|4614|17|1627.41|113.91|1741.32|27.91|697.34|1220.55|20.34|386.52|839.16| +2452174|38300|13969|4874|1822988|3949|18910|4874|1822988|3949|18910|13|20|4614|2|130.36|1.30|131.66|26.13|60.02|67.78|1.87|60.71|87.45| +2451282|67563|9052|31248|1559866|931|21196|31248|1559866|931|21196|49|35|4615|13|282.36|25.41|307.77|54.37|242.06|192.00|3.61|86.75|321.84| +2451290|36616|1114|38268|517206|2659|21957|38268|517206|2659|21957|2|31|4615|8|281.60|22.52|304.12|76.72|45.36|149.24|5.29|127.07|144.60| +2452692|58171|12823|47092|165035|2178|28521|47092|165035|2178|28521|27|7|4618|45|553.95|27.69|581.64|24.04|874.80|49.85|302.46|201.64|926.53| +2451489|59181|17944|54558|1044399|1440|34081|54558|1044399|1440|34081|46|14|4619|6|344.10|0.00|344.10|83.58|187.20|89.46|221.53|33.11|270.78| +2451695|29635|17675|45775|103072|6416|5313|45775|103072|6416|5313|37|31|4623|1|78.14|4.68|82.82|46.71|34.01|57.82|12.39|7.93|85.40| +2451703|29414|1172|10263|663473|3741|38861|10263|663473|3741|38861|13|35|4623|2|37.94|1.89|39.83|23.98|56.90|13.65|12.63|11.66|82.77| +2451869|72052|5503|15854|474394|2181|11121|15854|474394|2181|11121|38|18|4623|2|64.06|3.84|67.90|25.73|54.46|19.21|14.80|30.05|84.03| +2452837|54182|1089|48285|883397|2862|31127|48285|883397|2862|31127|15|11|4626|7|200.13|4.00|204.13|24.51|84.63|116.07|36.14|47.92|113.14| +2451998|43837|13883|62387|1071245|3303|36961|62387|1071245|3303|36961|17|16|4627|2|6.64|0.46|7.10|66.09|2.76|1.46|0.05|5.13|69.31| +2451998|78673|3431|39977|315772|5060|43809|39977|315772|5060|43809|32|7|4629|50|2554.00|0.00|2554.00|8.16|2137.00|664.04|1644.26|245.70|2145.16| +2452054|75108|8245|47241|222852|63|32827|47241|222852|63|32827|59|8|4629|10|978.80|68.51|1047.31|45.79|241.30|538.34|96.90|343.56|355.60| +2451363|46714|9439|42165|1779906|3694|11035|42165|1779906|3694|11035|22|25|4630|36|639.72|0.00|639.72|33.40|456.84|498.98|61.92|78.82|490.24| +2451311|48233|17678|60994|1666055|2763|15471|60994|1666055|2763|15471|25|20|4630|37|2549.67|0.00|2549.67|71.88|579.42|764.90|1070.86|713.91|651.30| +2451211|32370|8746|31331|968702|5018|40247|31331|968702|5018|40247|50|27|4630|13|66.04|1.32|67.36|74.66|19.37|11.88|25.99|28.17|95.35| +2451081|64852|2636|12395|292545|695|36045|12395|292545|695|36045|28|2|4632|13|99.84|3.99|103.83|44.99|353.21|52.91|44.58|2.35|402.19| +2451118|68356|2296|86084|1585341|5398|13502|86084|1585341|5398|13502|40|6|4632|67|1068.65|96.17|1164.82|55.73|152.09|758.74|30.99|278.92|303.99| +||6835|75305|||31944||350569||31944||25|4633||398.00||405.96|59.72|79.50|131.34|223.99||147.18| +2452212||13779|||863|7140||1024139||||26|4635|49|3726.94||4025.09|||149.07|||3881.52| +2452206|78756|11337|25080|793467|5407|39794|25080|793467|5407|39794|21|17|4635|2|19.64|0.58|20.22|30.32|55.64|2.94|3.00|13.70|86.54| +2452695|17020|16749|55437|1186364|6091|32171|55437|1186364|6091|32171|27|9|4637|14|1315.58|39.46|1355.04|7.74|552.44|1236.64|22.89|56.05|599.64| +2452132|50717|9380|54753|96122|4691|37707|54753|96122|4691|37707|19|31|4638|36|441.72|26.50|468.22|86.46|100.44|79.50|152.13|210.09|213.40| +2452037|15563|2537|49390|1738345|6053|47361|49390|1738345|6053|47361|5|13|4638|95|443.65|31.05|474.70|93.41|129.20|283.93|68.67|91.05|253.66| +2451968|33783|7397|95418|708280|7095|11157|95418|708280|7095|11157|29|31|4638|60|4579.20|0.00|4579.20|36.17|666.00|1969.05|756.94|1853.21|702.17| +2451622|36545|13940|29549|1400219|6089|39159|29549|1400219|6089|39159|25|32|4639|1|2.53|0.17|2.70|50.61|1.62|0.93|1.15|0.45|52.40| +2451482|22197|15652|84568|1538791|661|46884|84568|1538791|661|46884|46|33|4639|4|149.12|0.00|149.12|30.80|104.08|86.48|24.42|38.22|134.88| +2451304|72659|14755|69546|1300383|5459|9770|69546|1300383|5459|9770|26|18|4640|13|2905.50|145.27|3050.77|94.43|1264.77|2382.51|26.14|496.85|1504.47| +2451331|75327|11198|71462|962149|3236|1523|71462|962149|3236|1523|26|23|4641|31|699.67|41.98|741.65|53.30|1669.04|356.83|54.85|287.99|1764.32| +2452130|69721|2583|41464|1093510|6061|28054|41464|1093510|6061|28054|57|10|4642|55|9661.85|0.00|9661.85|68.96|4582.05|8599.04|212.56|850.25|4651.01| +2452176|74574|8227|80834|1910541|5048|32355|80834|1910541|5048|32355|13|34|4642|57|11154.33|334.62|11488.95|98.66|6354.93|0.00|4573.27|6581.06|6788.21| +2451270|30641|7957|91066|255628|2143|33105|91066|255628|2143|33105|4|8|4643|3|59.82|2.99|62.81|42.74|139.59|29.31|14.33|16.18|185.32| +2452769|74034|7405|88317|1138285|4253|29421|88317|1138285|4253|29421|57|5|4644|3|38.82|3.49|42.31|72.51|2.58|36.10|1.68|1.04|78.58| +2451491|39624|3874|94415|780655|142|3582|94415|780655|142|3582|7|27|4645|35|0.00|0.00|0.00|48.13|1196.30|0.00|0.00|0.00|1244.43| +|42334|17402|83085|1675193|||83085|||48162|||4645||1027.40|92.46|||182.90||167.77||337.70| +2452815|71265|16725|16415|1733821|6427|49433|16415|1733821|6427|49433|36|27|4647|4|41.44|2.07|43.51|32.38|4.16|4.55|24.34|12.55|38.61| +2451164|46998|10180|17597|149196|6395|28441|17597|149196|6395|28441|16|30|4648|64|6266.24|250.64|6516.88|32.11|6015.36|5827.60|179.84|258.80|6298.11| +2452721|74108|6121|575|1805234|3493|9352|575|1805234|3493|9352|36|11|4649|14|894.04|0.00|894.04|46.45|121.80|572.18|144.83|177.03|168.25| +2452093|60619|6638|8373|||19763|8373|||19763|1||4650|24|3046.32|243.70||93.64|1029.12||2385.88|325.35|| +2452111|61327|14099|58957|384558|5359|6207|58957|384558|5359|6207|32|32|4650|49|5082.77|203.31|5286.08|96.20|324.38|1778.96|2147.47|1156.34|623.89| +2452851|36538|7410|60370|93721|4672|36423|60370|93721|4672|36423|9|29|4651|4|248.56|9.94|258.50|81.71|0.00|116.82|36.88|94.86|91.65| +2451809|71529|3254|76760|191995|4830|4465|76760|191995|4830|4465|8|7|4652|1|79.98|1.59|81.57|45.75|54.46|46.38|20.83|12.77|101.80| +2451788|68533|5666|9431|551878|4278|31125|9431|551878|4278|31125|5|8|4652|1|4.32|0.04|4.36|42.63|64.94|2.41|1.05|0.86|107.61| +2451702||9991|68766||||68766|||23262|37||4652|72|||8411.27|26.83|5833.44||918.77||| +2452102|20343|9517|35490|429267|5444|22868|35490|429267|5444|22868|19|29|4653|2|74.18|0.00|74.18|36.40|46.70|10.38|31.26|32.54|83.10| +2451970|44238|9637|81970|49180|1543|38753|81970|49180|1543|38753|2|32|4653|53|6806.26|340.31|7146.57|85.88|1338.78|4015.69|1199.94|1590.63|1764.97| +2452526|6015|4968|17168|900994|2420|17046|17168|900994|2420|17046|1|27|4654|30|855.90|59.91|915.81|24.26|1806.90|25.67|24.90|805.33|1891.07| +2451755|55720|3115|35358|1525853|795|42490|35358|1525853|795|42490|22|4|4655|24|16.80|0.16|16.96|25.64|559.44|11.59|2.34|2.87|585.24| +2452700|46231|12081|16769|1108999|429|8659|16769|1108999|429|8659|30|16|4656|18|1204.74|72.28|1277.02|91.02|68.76|783.08|383.71|37.95|232.06| +|68909|7947|17456|1816707|405|6103|||||42|10|4656|2|||76.96||||||77.63| +2452655|73478|15996|50583|300102|1974|14693|50583|300102|1974|14693|60|21|4656|53|246.98|19.75|266.73|85.66|4041.78|41.98|24.60|180.40|4147.19| +2452707|35398|11445|8552|1415936|2167|20751|8552|1415936|2167|20751|42|6|4656|4|157.56|3.15|160.71|32.37|85.16|89.80|11.51|56.25|120.68| +2452051|54388|557|36494|846853|2459|20018|36494|846853|2459|20018|41|13|4657|10|453.00|18.12|471.12|75.80|233.80|0.00|317.10|135.90|327.72| +2450987|22356|10964|59681|828334|6587|49240|59681|828334|6587|49240|4|33|4659|4|403.80|20.19|423.99|82.36|182.36|383.61|1.61|18.58|284.91| +2451154|37097|4076|22721|357|5546|11182|22721|357|5546|11182|32|25|4659|73|5126.79|256.33|5383.12|15.01|2365.93|4255.23|854.12|17.44|2637.27| +2450861|50104|9079|23759|642600|641|17882|23759|642600|641|17882|56|23|4659|40|58.40|3.50|61.90|7.92|256.40|53.14|3.10|2.16|267.82| +2452228|76999|15671|5656|1746828|800|21172|5656|1746828|800|21172|55|14|4660|27|1264.68|88.52|1353.20|29.01|632.34|101.17|1105.33|58.18|749.87| +2452629|40160|9780|32201|1125313|4025|9584|32201|1125313|4025|9584|36|26|4661|12|24.24|0.48|24.72|59.31|157.68|23.27|0.38|0.59|217.47| +2452546|50701|13381|89846|915578|2910|21842|89846|915578|2910|21842|45|13|4661|66|4683.36|421.50|5104.86|41.93|2146.32|2856.84|1388.15|438.37|2609.75| +2451212|72309|13198|67240|721925|257|3342|67240|721925|257|3342|56|23|4662|6|120.78|9.66|130.44|87.25|273.42|106.28|0.43|14.07|370.33| +2451296|64657|7472|||6918||43515|1887840|6918||58||4662|5|265.45|||59.35|165.90||57.07||241.17| +2452432|76875|10047|76848|653632|1969|6353|76848|653632|1969|6353|51|29|4668|84|283.08|25.47|308.55|98.15|871.92|220.80|41.10|21.18|995.54| +2451225|55227|12766|59499|679563|7180|7644|59499|679563|7180|7644|13|29|4671|38|1007.00|80.56|1087.56|67.67|310.84|382.66|99.89|524.45|459.07| +2452755|67255|828|1528|26015|3302|40989|1528|26015|3302|40989|55|34|4672|51|3207.90|0.00|3207.90|26.86|2229.21|930.29|478.29|1799.32|2256.07| +2452853|16541|16843|39368|978256|1944|35279|39368|978256|1944|35279|48|16|4672|24|10.80|0.00|10.80|79.79|0.00|6.04|1.47|3.29|79.79| +2452139|17248|1499|28043|1457241|6439|47237|28043|1457241|6439|47237|41|25|4673|13|206.05|6.18|212.23|66.42|173.68|113.32|40.80|51.93|246.28| +2451706|38674|10904|75287|1600454|1974|3141|75287|1600454|1974|3141|53|12|4675|56|747.04|7.47|754.51|33.32|747.04|672.33|53.79|20.92|787.83| +||158||1227001|4265|3976||1227001|||46|27|4676|2|137.14|8.22|145.36|64.91|||||157.53| +2451872|67090|10565|89813|1762976|5292|34408|89813|1762976|5292|34408|1|15|4678|66|11259.60|337.78|11597.38|14.23|671.88|0.00|9570.66|1688.94|1023.89| +2452243|54818|12019|31460|1911481|2702|37123|31460|1911481|2702|37123|47|15|4680|29|240.70|16.84|257.54|66.67|131.95|21.66|118.28|100.76|215.46| +2452220|20769|11299|12517|932619|1968|870|12517|932619|1968|870|49|1|4680|28|366.24|3.66|369.90|70.62|626.08|347.92|7.14|11.18|700.36| +2452181|75947|4195|11337|497798|259|38767|11337|497798|259|38767|15|22|4680|11|755.26|15.10|770.36|2.03|495.22|271.89|468.86|14.51|512.35| +2451981|17712|6542|53252|514813|4374|7582|53252|514813|4374|7582|44|22|4682|45|1325.70|39.77|1365.47|95.16|687.15|821.93|80.60|423.17|822.08| +2451999|86223|7022|83288|1298470|4405|49587|83288|1298470|4405|49587|49|34|4682|4|193.20|1.93|195.13|49.69|128.80|86.94|75.44|30.82|180.42| +2452422|22143|8117|5770|29138|859|43380|5770|29138|859|43380|51|32|4685|12|111.24|0.00|111.24|51.22|87.60|94.55|1.16|15.53|138.82| +2452532|67572|11913|47345|1665927|4156|11138|47345|1665927|4156|11138|59|30|4685|1|3.71|0.07|3.78|73.41|1.27|1.26|0.39|2.06|74.75| +|37298|4801||545842||36312|14854||6877|36312|13|13|4685||42.21|0.00|42.21|53.63||21.10||3.80|77.21| +2452745|71084|17394|92185|858599|2966|22980|92185|858599|2966|22980|24|11|4687|45|6376.50|255.06|6631.56|73.53|3666.15|2040.48|1691.04|2644.98|3994.74| +||3585|50044|277862|6480|19051|50044|||19051|49|5|4687|3|||150.69||25.62|25.11|40.04||| +2452523|75575|12798|8433|1025098|2563|45675|8433|1025098|2563|45675|43|9|4687|49|3090.43|123.61|3214.04|67.71|1776.74|1328.88|1356.39|405.16|1968.06| +2452591|26366|1536|59183|1271166|1549|13532|59183|1271166|1549|13532|54|1|4687|35|498.75|14.96|513.71|56.52|249.20|428.92|61.45|8.38|320.68| +2452370|68535|5671|14623|767089|3555|28821|14623|767089|3555|28821|33|34|4688|10|872.30|78.50|950.80|97.24|495.10|732.73|90.72|48.85|670.84| +2452412|30113|10411|55149|840204|1259|36714|55149|840204|1259|36714|27|35|4688|1|3.57|0.28|3.85|56.41|9.84|1.74|1.29|0.54|66.53| +2452381|5387|16437|3311|97671|6078|44838|3311|97671|6078|44838|51|14|4688|51|277.44|16.64|294.08|86.19|360.06|194.20|69.92|13.32|462.89| +2451444|39235|6865|31578|1261954|4777|18800|31578|1261954|4777|18800|37|14|4691|2|102.96|4.11|107.07|19.12|19.92|60.74|22.79|19.43|43.15| +2451390|61144|10006|16302|1564347|3145|31990|16302|1564347|3145|31990|46|35|4691|19|606.86|0.00|606.86|80.19|471.96|236.67|325.76|44.43|552.15| +2452028|67600|10237|12353|1247941|4776|47211|12353|1247941|4776|47211|29|29|4692|52|223.60|2.23|225.83|26.25|310.96|67.08|137.73|18.79|339.44| +2452084|42592|4034|73524|845126|5975|35577|73524|845126|5975|35577|47|18|4692|71|11332.31|793.26|12125.57|28.07|765.38|10652.37|156.38|523.56|1586.71| +2452176|10457|17807|80552|832010|6221|23279|80552|832010|6221|23279|14|34|4692|3|14.97|1.19|16.16|3.79|33.51|8.68|0.69|5.60|38.49| +|63854|17839||1656977|3623|||1656977||||19|4693|56|511.84|||69.65|170.24|291.74||193.69|| +2452345|81201|7461|37655|1406722|1742|37962|37655|1406722|1742|37962|9|26|4693|40|568.80|22.75|591.55|68.30|240.80|193.39|319.09|56.32|331.85| +2452377|28370|15105|49151|319801|2451|7501|49151|319801|2451|7501|3|6|4693|19|18.05|0.36|18.41|90.16|67.64|14.25|0.95|2.85|158.16| +2451474|58578|14810|35871|578352|4480|22561|35871|578352|4480|22561|38|11|4694|67|1130.96|33.92|1164.88|67.54|601.66|916.07|58.02|156.87|703.12| +2451568|72707|11758|16223|525622|308|43760|16223|525622|308|43760|10|22|4696|19|547.01|32.82|579.83|59.53|95.00|218.80|203.49|124.72|187.35| +2451867|69703|835|72278|912421|1811|35954|72278|912421|1811|35954|28|8|4697|3|203.97|12.23|216.20|75.91|105.27|104.02|7.99|91.96|193.41| +2451677|78354|1255|43225|1361094|563|20079|43225|1361094|563|20079|52|12|4697|57|13252.50|795.15|14047.65|21.27|1386.81|8084.02|516.84|4651.64|2203.23| +2451776|64602|1753|42157|722648|6339|9277|42157|722648|6339|9277|2|5|4697|56|2357.04|188.56|2545.60|26.30|1469.44|94.28|1221.89|1040.87|1684.30| +2452433|42013|6987|76443|1666022|6001|45933|76443|1666022|6001|45933|9|20|4698|33|1197.90|11.97|1209.87|46.57|0.00|814.57|241.49|141.84|58.54| +2452270|69848|5881|9084|1396557|2430|44256|9084|1396557|2430|44256|1|2|4698|7|505.61|20.22|525.83|16.75|269.29|91.00|265.35|149.26|306.26| +2451937|4298|11321|8889|872910|3331|24552|8889|872910|3331|24552|8|11|4699|72|3726.72|260.87|3987.59|34.14|6011.28|3465.84|104.35|156.53|6306.29| +2452001|70985|16088|30488|823139|590|4794|30488|823139|590|4794|41|1|4699|41|135.30|1.35|136.65|92.44|87.74|79.82|29.40|26.08|181.53| +2452613|28144|13473|24811|961386|4361|17689|24811|961386|4361|17689|18|29|4700|14|3063.90|183.83|3247.73|28.46|1514.66|2389.84|121.33|552.73|1726.95| +2452494|22979|16833|73742|771179|6663|8547|73742|771179|6663|8547|47|7|4701|23|274.62|21.96|296.58|2.37|240.35|263.63|9.56|1.43|264.68| +2452594|66196|10781|5641|1080262|2530|12798|5641|1080262|2530|12798|33|19|4701|19|70.11|5.60|75.71|21.58|702.43|19.63|17.16|33.32|729.61| +2451448|37824|16903|94813|1195619|5578|23184|94813|1195619|5578|23184|37|31|4703|19|535.42|5.35|540.77|47.05|1677.70|112.43|21.14|401.85|1730.10| +2451068|33544|17299|29448|760157|5266|44375|29448|760157|5266|44375|52|3|4706|48|142.08|11.36|153.44|50.24|319.68|112.24|22.08|7.76|381.28| +2451221|20592|1633|31228|1095776|2371|20519|31228|1095776|2371|20519|58|11|4706|81|17022.96|510.68|17533.64|56.82|4489.02|9532.85|6216.79|1273.32|5056.52| +2452668|47277|17431|12165|638809|6606|42450|12165|638809|6606|42450|36|30|4709|1|9.41|0.47|9.88|8.84|15.41|1.97|7.29|0.15|24.72| +2451989|60474|3415|73805|736953|2948|39343|73805|736953|2948|39343|59|22|4710|13|236.60|2.36|238.96|82.92|76.05|141.96|22.71|71.93|161.33| +2452879|39021|10956|89513|1235736|2759|14535|89513|1235736|2759|14535|39|8|4712|68|16036.44|801.82|16838.26|44.96|2732.92|1122.55|2386.22|12527.67|3579.70| +2452024|77351|10682|84021|1064876|6531|37386|84021|1064876|6531|37386|37|3|4713|3|73.38|6.60|79.98|8.23|125.04|12.47|14.00|46.91|139.87| +2452054|43004|16442|82252|1404018|3737|13056|82252|1404018|3737|13056|13|9|4714|31|894.04|0.00|894.04|0.55|484.22|268.21|269.10|356.73|484.77| +2451917|52802|4049|62496|1810183|7117|16804|62496|1810183|7117|16804|13|12|4714|2|0.66|0.01|0.67|29.22|2.46|0.29|0.17|0.20|31.69| +2451923|23869|16003|15044|1792769|3741|17066|15044|1792769|3741|17066|35|28|4718|14|148.82|13.39|162.21|48.51|117.04|37.20|25.67|85.95|178.94| +2451927|23807|5797|20651|1235391|3284|30824|20651|1235391|3284|30824|19|32|4718|34|2546.94|178.28|2725.22|55.74|1528.30|50.93|923.52|1572.49|1762.32| +2451231|39237|11413|61807|1733361|2068|6422|61807|1733361|2068|6422|10|19|4719|63|3694.32|0.00|3694.32|13.65|2228.94|1773.27|1556.05|365.00|2242.59| +2452324|11478|17165|14386|998871|3030|3232|14386|998871|3030|3232|21|22|4722|10|850.60|8.50|859.10|20.05|453.00|136.09|521.59|192.92|481.55| +2452314|79010|15859|4585|44835|5085|14182|4585|44835|5085|14182|29|24|4723|1|22.75|1.13|23.88|98.55|16.55|15.69|4.44|2.62|116.23| +||3589|||3462|1286||10038|3462||||4724|8||||24.01||51.68||4.90|119.29| +2451243|23919|11692|12783|1914220|904|25141|12783|1914220|904|25141|32|24|4724|80|1979.20|19.79|1998.99|61.03|835.20|890.64|195.94|892.62|916.02| +2452157|36372|13391|80815|645789|4010|3784|80815|645789|4010|3784|38|17|4730|16|1389.44|125.04|1514.48|7.63|23.04|472.40|339.30|577.74|155.71| +2452026|66084|4249|99818|264395|619|10043|99818|264395|619|10043|1|21|4730|24|1000.08|90.00|1090.08|71.86|370.32|260.02|192.41|547.65|532.18| +2452415|9956|4017|68078|387870|5615|41407|68078|387870|5615|41407|21|35|4731|10|903.90|63.27|967.17|79.75|465.60|153.66|232.57|517.67|608.62| +2451649|52960|11122|23923|1119880|5084|41041|23923|1119880|5084|41041|22|11|4732|36|5492.88|219.71|5712.59|20.23|2644.56|274.64|4957.32|260.92|2884.50| +|35984|4634|6442|||19677|6442|1911089||||4|4733||523.80||555.22|7.73|246.40||356.18|167.62|| +2451644|43274|4754|39655|1160177|4252|19574|39655|1160177|4252|19574|4|5|4733|32|1293.44|103.47|1396.91|13.15|137.92|970.08|58.20|265.16|254.54| +2452540|37805|463|8588|401816|3844|26672|8588|401816|3844|26672|33|16|4734|30|878.10|17.56|895.66|14.77|198.60|799.07|51.36|27.67|230.93| +2452400|69955|9189|16318|114371|4871|23724|16318|114371|4871|23724|39|13|4734|25|346.00|17.30|363.30|16.09|153.75|335.62|5.91|4.47|187.14| +2452486|70893|7755|76043|1023511|579|22047|76043|1023511|579|22047|57|25|4736|22|293.92|5.87|299.79|95.09|402.16|99.93|124.15|69.84|503.12| +2452613|64815|13765|8941|316944|947|27750|8941|316944|947|27750|42|16|4736|1|1.90|0.05|1.95|98.20|1.30|1.02|0.54|0.34|99.55| +2451229|78620|3254|36157|988810|2263|6869|36157|988810|2263|6869|10|21|4738|13|3081.39|30.81|3112.20|73.66|253.76|616.27|1306.51|1158.61|358.23| +2451211|77447|5852|11506|740820|2934|17048|11506|740820|2934|17048|37|9|4740|42|1441.86|72.09|1513.95|94.26|777.42|605.58|100.35|735.93|943.77| +2451187|25383|9424|79176|598249|6122|23195|79176||||||4740|7|284.97||290.66|11.73||||16.93|| +||14701||||4175||289924|||||4741||||4720.26||1833.52|||661.30|2146.23| +2451973|31716|4814|27726|1649687|3648|10009|27726|1649687|3648|10009|37|22|4745|49|2875.32|28.75|2904.07|93.23|1996.75|143.76|1802.82|928.74|2118.73| +2452122|35432|11852|3418|1269442|7019|47843|3418|1269442|7019|47843|29|18|4745|5|217.95|6.53|224.48|19.04|199.00|30.51|71.22|116.22|224.57| +2452452|66303|16515|99008|1484342|5323|24257|99008|1484342|5323|24257|27|33|4746|11|281.82|14.09|295.91|51.23|86.13|140.91|126.81|14.10|151.45| +2452520|71895|12075|54496|1215382|4621|33107|54496|1215382|4621|33107|19|12|4747|4|113.80|3.41|117.21|41.23|138.40|15.93|91.99|5.88|183.04| +2452246|61485|1123|78582|1238334|486|26238|78582|1238334|486|26238|43|18|4750|7|291.55|2.91|294.46|6.50|110.67|52.47|71.72|167.36|120.08| +2452358|23869|2493|98698|1696247|196|19745|98698|1696247|196|19745|3|32|4750|4|21.80|0.00|21.80|31.25|54.48|14.82|1.88|5.10|85.73| +2452855|5970|14515|69798|779608|6571|38334|69798|779608|6571|38334|33|18|4751|8|156.96|1.56|158.52|38.28|156.96|32.96|32.24|91.76|196.80| +2451123|42538|1592|48897|1195546|948|38797|48897|1195546|948|38797|32|32|4752|3|30.36|2.73|33.09|79.98|51.15|30.36|0.00|0.00|133.86| +2451619|41930|6514|88564|1447777|237|3050|88564|1447777|237|3050|10|27|4756|86|8907.88|0.00|8907.88|63.54|6190.28|8195.24|655.62|57.02|6253.82| +2451946|14999|6656|23025|1434268|2311|11107|23025|1434268|2311|11107|7|23|4757|5|128.20|10.25|138.45|92.08|155.20|49.99|35.19|43.02|257.53| +2451890|48106|4903|74845|708026|407|14035|74845|708026|407|14035|19|17|4757|15|2179.35|65.38|2244.73|78.32|1078.65|871.74|379.20|928.41|1222.35| +2451915|77435|10142|12604|1885770|2936|10502|12604|1885770|2936|10502|17|8|4758|32|5336.64|53.36|5390.00|73.60|2494.40|2775.05|1946.80|614.79|2621.36| +2452082|34421|5041|94137|572369|2954|39186|94137|572369|2954|39186|29|1|4759|10|239.30|7.17|246.47|22.95|327.40|11.96|56.83|170.51|357.52| +2452229|24138|17833|16758|260957|2756|10955|16758|260957|2756|10955|33|4|4759|20|138.80|12.49|151.29|80.69|291.60|13.88|94.93|29.99|384.78| +2452192|45540|5209|56768|111228|3288|44948|56768|111228|3288|44948|17|22|4759|45|105.30|1.05|106.35|17.54|275.40|54.75|49.03|1.52|293.99| +2452229|9948|11845|60220|1366768|6116|49864|60220|1366768|6116|49864|11|27|4759|5|182.60|14.60|197.20|3.15|67.25|91.30|69.38|21.92|85.00| +2452857|66134|13911|73600|1651763|7173|37736|73600|1651763|7173|37736|39|30|4760|20|1500.40|30.00|1530.40|87.53|381.00|660.17|815.02|25.21|498.53| +2451245|73561|50|36141|317017|740|23663|36141|317017|740|23663|46|9|4762|2|310.92|0.00|310.92|44.77|176.66|161.67|10.44|138.81|221.43| +2451160|71734|361|27726|1137170|5014|27611|27726|1137170|5014|27611|52|15|4762|23|1196.69|35.90|1232.59|84.10|875.15|586.37|146.47|463.85|995.15| +2451144|29059|5308|68468|190764|1507|6250|68468|190764|1507|6250|31|21|4762|40|928.80|65.01|993.81|99.18|3884.40|325.08|549.38|54.34|4048.59| +2451662|47848|12385|21644|584108|972|1789|21644|584108|972|1789|55|26|4764|5|81.55|2.44|83.99|50.21|203.95|79.91|1.19|0.45|256.60| +2451910|75832|10331|8937|1302848|1559|1070|8937|1302848|1559|1070|47|35|4765|74|6153.84|492.30|6646.14|6.45|3454.32|3630.76|2094.15|428.93|3953.07| +2452332|50688|13457|48083|1873373|4263|21153|48083|1873373|4263|21153|19|25|4766|79|5097.08|356.79|5453.87|16.72|2718.39|509.70|3486.40|1100.98|3091.90| +2451670|48074|9296|4850|211825|249|13168|4850|211825|249|13168|14|16|4768|56|2500.96|175.06|2676.02|19.62|378.56|2425.93|57.02|18.01|573.24| +2451489|27911|15052|65046|1135891|3889|46734|65046|1135891|3889|46734|31|23|4768|9|364.59|10.93|375.52|93.97|24.03|149.48|163.48|51.63|128.93| +2451526|27860|9907|89132|819291|650|33609|89132|819291|650|33609|43|33|4768|76|5174.84|258.74|5433.58|80.85|3242.92|1811.19|1883.64|1480.01|3582.51| +2452274|29646|11535|54644|1707935|7017|45807|54644|1707935|7017|45807|23|17|4769|5|417.25|20.86|438.11|44.93|257.95|287.90|21.98|107.37|323.74| +2451623|71309|1438|4715|1608639|6963|39640|4715|1608639|6963|39640|38|15|4770|60|492.00|9.84|501.84|85.23|144.60|418.20|5.16|68.64|239.67| +2451638|76892|4354|64951|1442979|4028|20814|64951|1442979|4028|20814|10|18|4770|21|961.38|38.45|999.83|35.79|24.57|653.73|203.04|104.61|98.81| +2451711|6102|6694|88573|920509|5692|30875|88573|920509|5692|30875|31|18|4770|11|17.82|0.53|18.35|50.88|267.74|6.77|3.97|7.08|319.15| +||7024|||6747||||6747||37||4770|12|533.40|21.33|554.73|29.72|||||| +2452098|33949|17159|9351|373380|1056|28332|9351|373380|1056|28332|23|2|4771|10|1354.70|27.09|1381.79|46.83|613.20|514.78|638.33|201.59|687.12| +|14550|9631||||11491||||||13|4772||||||||173.37|166.58|| +2452775|75334|5598|22844|445929|1014|30452|22844|445929|1014|30452|12|35|4773|16|133.60|8.01|141.61|58.27|154.88|49.43|34.50|49.67|221.16| +2451403|63308|4126|95427|656336|3680|43142|95427|656336|3680|43142|37|10|4774|8|294.24|2.94|297.18|10.72|18.72|288.35|0.76|5.13|32.38| +|72002|12823|||2080|||1545000|2080|||2|4774|3|222.87|15.60|238.47||37.92|60.17|||145.67| +2451432|75932|7963|89805|63178|551|1877|89805|63178|551|1877|43|28|4774|4|296.64|20.76|317.40|52.14|97.72|32.63|108.24|155.77|170.62| +2452576|70553|10729|27623|1776763|6744|5135|27623|1776763|6744|5135|60|20|4775|31|0.00|0.00|0.00|69.58|273.11|0.00|0.00|0.00|342.69| +2451729|66598|8276|14742|883873|4557|23241|14742|883873|4557|23241|46|34|4778|26|10.66|0.21|10.87|89.51|28.86|7.03|3.26|0.37|118.58| +2451672|26099|11696|7120|1506848|2129|25857|7120|1506848|2129|25857|31|24|4778|51|812.43|64.99|877.42|69.47|2624.97|609.32|20.31|182.80|2759.43| +2451489|46113|12982|75895|700952|4056|46360|75895|700952|4056|46360|4|10|4779|12|40.20|2.41|42.61|24.67|214.80|21.70|11.84|6.66|241.88| +2451478|12308|11293|20096|390013|4730|34247|20096|390013|4730|34247|10|4|4781|25|1403.50|70.17|1473.67|50.92|701.75|519.29|884.21|0.00|822.84| +2452426|64640|11515|95990|1754995|1200|46958|95990|1754995|1200|46958|55|1|4783|11|85.25|6.82|92.07|97.75|78.32|29.83|23.83|31.59|182.89| +2452409|37256|17381|78191|1392544|50|49203|78191|1392544|50|49203|19|9|4783|5|311.75|15.58|327.33|43.12|207.80|77.93|91.18|142.64|266.50| +2452400|49674|4499|24313|1207589|4912|23388|24313|1207589|4912|23388|45|34|4783|89|3530.63|0.00|3530.63|91.86|199.36|1941.84|1588.79|0.00|291.22| +||4417|39630||||||6536|16389|||4783|20|156.00|12.48|||2.00|107.64|||| +2452813|70867|5371|2449|1358716|4583|36850|2449|1358716|4583|36850|18|30|4784|15|1116.45|22.32|1138.77|5.47|285.30|524.73|455.62|136.10|313.09| +2452840|61228|4293|12051|919012|5173|26934|12051|919012|5173|26934|36|15|4784|7|720.58|43.23|763.81|33.78|237.02|461.17|20.75|238.66|314.03| +2452881|59713|12|54227|1495602|3024|21199|54227|1495602|3024|21199|15|7|4785|69|3467.25|173.36|3640.61|28.55|75.21|1386.90|1539.45|540.90|277.12| +2452874|42720|2286|65329|1809593|4704|26615|65329|1809593|4704|26615|51|12|4785|15|271.50|8.14|279.64|64.16|135.75|160.18|34.50|76.82|208.05| +2452067|37174|2756||||42497|36723||||17|7|4788|6|13.62|0.54|||||||| +2452862|46714|8349|35195|1041044|1740|34789|35195|1041044|1740|34789|9|31|4789|52|2094.56|104.72|2199.28|25.58|551.20|1382.40|306.22|405.94|681.50| +2451322|19663|15938|97259|886961|5463|31565|97259|886961|5463|31565|19|34|4790|11|197.78|15.82|213.60|40.49|423.83|17.80|41.39|138.59|480.14| +2451321|56380|13315|14618|1448848|1478|20155|14618|1448848|1478|20155|10|2|4790|52|4906.72|343.47|5250.19|66.97|4713.80|2894.96|1046.11|965.65|5124.24| +2451406|81062|1105|64392|745992|6345|7898|64392|745992|6345|7898|2|35|4792|10|278.20|2.78|280.98|73.57|196.30|197.52|15.32|65.36|272.65| +2451272|76035|11767|65337|1621561|4985|19609|65337|1621561|4985|19609|34|11|4792|1|37.80|0.37|38.17|5.46|6.20|18.90|10.58|8.32|12.03| +2451824|73220|5983|15647|1384416|738|28707|15647|1384416|738|28707|32|28|4793|45|147.60|5.90|153.50|39.42|147.60|138.74|0.88|7.98|192.92| +2451740|74373|2002|45993|1591890|6940|38837|45993|1591890|6940|38837|43|25|4793|4|178.64|10.71|189.35|88.45|89.32|101.82|68.36|8.46|188.48| +2452446|45792|289|38996|1622066|4250|35380|38996|1622066|4250|35380|33|33|4794|16|1879.52|37.59|1917.11|26.86|1217.76|1804.33|70.67|4.52|1282.21| +2452292|66211|11103|11509|498456|3803|29150|11509|498456|3803|29150|33|32|4794|16|688.32|41.29|729.61|48.94|723.52|406.10|126.99|155.23|813.75| +2452342|33740|4235|22473|1360041|5609|23177|22473|1360041|5609|23177|59|27|4794|55|223.30|11.16|234.46|5.50|542.30|138.44|35.64|49.22|558.96| +2451377|74691|7252|25315|166860|3958|8303|25315|166860|3958|8303|56|16|4795|29|721.52|50.50|772.02|59.74|549.84|555.57|53.10|112.85|660.08| +2452106|64186|7715|10308|70007|4906|20252|10308|70007|4906|20252|37|35|4797|16|638.88|51.11|689.99|35.02|352.00|562.21|26.83|49.84|438.13| +2451991||14150|||1000||88561|1344604|1000||11||4797|15||48.26|||1129.05|58.98|109.77||| +2452044|63024|16976|27463|793149|5682|46568|27463|793149|5682|46568|44|26|4797|18|142.38|11.39|153.77|58.81|533.88|113.90|13.38|15.10|604.08| +2451208|73446|17042|96382|1204154|4726|37251|96382|1204154|4726|37251|2|29|4798|10|295.10|23.60|318.70|96.62|286.70|200.66|55.71|38.73|406.92| +2451332|25330|16267|92606|1709144|82|1820|92606|1709144|82|1820|20|11|4798|19|275.50|16.53|292.03|8.88|220.40|165.30|16.53|93.67|245.81| +2451987|19145|13175|75049|415192|5735|3478|75049|415192|5735|3478|29|18|4799|2|32.10|2.88|34.98|22.19|25.68|22.14|6.17|3.79|50.75| +2451577|34933|4112|84988|681957|879|6248|84988|681957|879|6248|16|9|4802|39|1650.87|115.56|1766.43|20.83|645.06|974.01|311.35|365.51|781.45| +2451605|59057|14839|56570|1539755|2940|36525|56570|1539755|2940|36525|38|3|4802|1|15.26|0.76|16.02|52.07|3.90|9.91|0.10|5.25|56.73| +2452631|73090|12079|22518|1096293|3668|7443|22518|1096293|3668|7443|31|18|4803|57|412.11|28.84|440.95|2.23|257.64|391.50|13.80|6.81|288.71| +2451489|42124|6001|55156|21200|844|13688|55156|21200|844|13688|13|25|4804|38|3652.94|36.52|3689.46|97.60|1686.06|1132.41|1310.67|1209.86|1820.18| +2451897|74965|9686|92529|1337238|5084|7416|92529|1337238|5084|7416|26|22|4805|53|2970.65|237.65|3208.30|34.20|594.13|712.95|1241.73|1015.97|865.98| +2451601|42490|5791|63754|12477|3247|15170|63754|12477|3247|15170|58|19|4806|17|152.83|1.52|154.35|67.46|91.63|68.77|77.33|6.73|160.61| +2451151|43378|6076|66425|1614505|3738|29418|66425|1614505|3738|29418|50|19|4808|17|44.03|1.32|45.35|32.28|169.83|42.70|0.31|1.02|203.43| +2451079|75198|12619|90878|1371468|3258|25448|90878|1371468|3258|25448|58|33|4808|36|3754.44|225.26|3979.70|86.87|41.04|3341.45|355.17|57.82|353.17| +2452705|51153|10761|8829|1249748|1067|15186|8829|1249748|1067|15186|24|23|4809|1|36.68|1.46|38.14|78.59|0.87|23.84|7.19|5.65|80.92| +2452670|36873|16219|80987|1052740|6893|27046|80987|1052740|6893|27046|39|21|4809|9|454.23|31.79|486.02|72.21|227.07|145.35|114.28|194.60|331.07| +||3851|||5971||||||55|32|4810|6|129.48|6.47||69.51||||6.30|| +|65040|16435||582445|5937|39476|43753|582445|5937||||4810|20||19.53||65.19|1465.20||||| +|56758|10609||626488|5717||78459||||59||4810|||3.81||||||33.87|| +2452519|43801|10945|88127|1760883|6439|28395|88127|1760883|6439|28395|18|6|4811|20|1267.00|88.69|1355.69|48.66|422.20|291.41|282.92|692.67|559.55| +2452508|55037|14136|3359|374668|3577|18486|3359|374668|3577|18486|54|22|4811|13|1305.07|91.35|1396.42|92.81|152.62|913.54|375.86|15.67|336.78| +2452597|71008|13017|87851|1585610|2922|41434|87851|1585610|2922|41434|3|25|4811|30|112.20|0.00|112.20|40.47|142.80|42.63|7.65|61.92|183.27| +2452474|67054|11850|63338|1076839|740||63338||740||57||4811|27||3.92|69.26||||26.97||| +2451223|17948|7168|40049|92344|6239|4534|40049|92344|6239|4534|13|9|4812|9|161.28|0.00|161.28|13.73|158.13|117.73|40.06|3.49|171.86| +2451392|47238|925|29334|324250|1566|39282|29334|324250|1566|39282|2|33|4812|55|600.05|18.00|618.05|83.86|218.35|360.03|55.20|184.82|320.21| +2452425|56507|17233|16903|1386394|1038|9329|16903|1386394|1038|9329|39|9|4813|7|130.48|1.30|131.78|9.48|32.62|28.70|74.29|27.49|43.40| +2452474|68384|13181|49932|1044611|1047|9139|49932|1044611|1047|9139|53|21|4814|42|811.86|64.94|876.80|19.04|405.72|202.96|170.49|438.41|489.70| +2452456|75493|2753|36583|977762|2588|25367|36583|977762|2588|25367|31|17|4814|7|11.48|0.34|11.82|7.68|15.12|2.18|3.34|5.96|23.14| +2452317|68146|1931|62519|822115|3664|27660|62519|822115|3664|27660|53|8|4816|40|896.00|0.00|896.00|61.44|258.00|125.44|454.63|315.93|319.44| +2452191|28587|15607|43710|1310182|4207|5032|43710|1310182|4207|5032|37|9|4816|27|1007.91|10.07|1017.98|11.62|671.76|10.07|927.99|69.85|693.45| +2452200|40803|14321|96126|88923|3112|29148|96126|88923|3112|29148|3|3|4816|72|1840.32|36.80|1877.12|13.77|380.16|1693.09|145.75|1.48|430.73| +|61284|12304||561378|5854|37977||561378||||10|4817|37||||79.13|156.88|743.24|||264.59| +2451724|60038|12416|64920|1214180|5393|37006|64920|1214180|5393|37006|40|14|4817|99|3037.32|151.86|3189.18|22.91|781.11|1852.76|675.19|509.37|955.88| +2451639|46772|13579|70769||3516|23313||1534119|3516|23313||9|4817|32||0.34||||11.16|||| +2452003|26676|10661|78017|1426139|1014|21936|78017|1426139|1014|21936|17|22|4818|12|99.12|1.98|101.10|37.28|123.96|68.39|25.19|5.54|163.22| +2451905|31011|9145|94715|48563|5258|12706|94715|48563|5258|12706|43|3|4818|18|932.94|9.32|942.26|70.12|81.00|233.23|419.82|279.89|160.44| +2452129|19691|12621|46993|602433|3473|37713|46993|602433|3473|37713|37|16|4821|21|842.73|25.28|868.01|1.75|137.34|328.66|282.73|231.34|164.37| +2452137|50618|7257|2096|3621|346|16671|2096|3621|346|16671|19|24|4821|20|1051.60|84.12|1135.72|87.56|222.40|578.38|33.12|440.10|394.08| +2452651|17103|9714|40167|1682335|5303|22671|40167|1682335|5303|22671|25|30|4822|27|677.16|33.85|711.01|89.72|362.61|196.37|139.42|341.37|486.18| +2451353|30243|17875|52593|1859361|5329|3344|52593|1859361|5329|3344|52|22|4826|3|7.74|0.46|8.20|26.13|4.26|1.08|1.99|4.67|30.85| +2451332|34842|17776|54837|1232830|776|12769|54837|1232830|776|12769|43|1|4826|2|97.32|8.75|106.07|46.48|53.90|41.84|40.50|14.98|109.13| +2452113|64615|205|93507|80509|6711|22913|93507|80509|6711|22913|17|6|4827|16|624.32|31.21|655.53|43.56|246.40|249.72|14.98|359.62|321.17| +2452583|36908|9597|21086|1754588|2137|39239|21086|1754588|2137|39239|30|5|4828|50|3483.50|278.68|3762.18|60.02|2612.50|592.19|549.34|2341.97|2951.20| +2452378|1007|13543|80427|1408639|851|16216|80427|1408639|851|16216|41|1|4830|32|1296.96|64.84|1361.80|6.48|764.80|998.65|17.89|280.42|836.12| +2452413|70958|17089|23876|1005876|5123|13923|23876|1005876|5123|13923|51|24|4831|11|422.40|4.22|426.62|17.12|135.08|168.96|202.75|50.69|156.42| +2452237|63853|12339|20899|59046|40|30587|20899|59046|40|30587|7|3|4831|19|1078.06|0.00|1078.06|13.14|351.50|657.61|8.40|412.05|364.64| +2452173||7173|||5329|5221|||5329||||4832|||0.00|4966.16||||||| +2452351|33403|7283|60986|1802556|1454|46370|60986|1802556|1454|46370|43|29|4832|16|1737.28|52.11|1789.39|81.46|152.00|1181.35|183.45|372.48|285.57| +2451191|47795|2872|17953|929910|6314|20271|17953|929910|6314|20271|40|32|4833|3|42.93|2.14|45.07|48.02|7.14|33.48|0.37|9.08|57.30| +2451255|20488|15142|48961|454740|2472|7004|48961|454740|2472|7004|19|10|4833|5|40.60|0.00|40.60|13.91|257.20|18.67|8.99|12.94|271.11| +2451782|65629|14606|17861|942762|6416|1317|17861|942762|6416|1317|34|33|4836|34|136.68|1.36|138.04|53.22|957.44|120.27|13.29|3.12|1012.02| +||3955|26402|700994|6433|31025||700994|||||4837||2298.78|0.00|2298.78|32.55|||926.87||1860.05| +2452794|52188|6330|22810|1412319|2628|1337|22810|1412319|2628|1337|48|14|4838|6|532.74|42.61|575.35|69.89|75.30|516.75|11.03|4.96|187.80| +2452670|69535|15408|15933|1183447|561|49228|15933|1183447|561|49228|27|8|4838|13|849.55|8.49|858.04|54.22|391.04|696.63|91.75|61.17|453.75| +2452868|78715|2485|51317|1328817|3564|33464|51317|1328817|3564|33464|57|32|4838|12|217.20|17.37|234.57|68.09|76.44|160.72|50.26|6.22|161.90| +2451479|71909|10135|46761|1739381|6848|36315|46761|1739381|6848|36315|19|5|4839|24|476.88|4.76|481.64|35.65|59.52|429.19|19.07|28.62|99.93| +2451273|78883|140|31011|1251574|6537|5100|31011|1251574|6537|5100|44|3|4841|23|707.94|7.07|715.01|61.35|249.78|637.14|52.39|18.41|318.20| +2451347|36795|4678|10553|552771|942|23560|10553|552771|942|23560|46|17|4841|2|148.44|13.35|161.79|53.73|148.44|117.26|19.95|11.23|215.52| +2452867|40891|11511|55029|1636024|4572|36367|55029|1636024|4572|36367|13|5|4843|40|3134.40|31.34|3165.74|87.32|1790.80|720.91|410.29|2003.20|1909.46| +2452932|78807|5221|85266|1656563|5460|35557|85266|1656563|5460|35557|48|2|4843|3|152.13|12.17|164.30|75.41|68.43|42.59|51.48|58.06|156.01| +2452797|67483|7749|93939|296804|22|7687|93939|296804|22|7687|1|30|4843|1|107.19|7.50|114.69|73.16|78.60|11.79|30.52|64.88|159.26| +2451952|59727|17519|44837|1598250|1263|15476|44837|1598250|1263|15476|26|9|4844|10|1765.60|158.90|1924.50|15.93|111.50|600.30|302.97|862.33|286.33| +2451742|57658|11815|39201|1480453|458|42910|39201|1480453|458|42910|31|29|4845|2|11.40|0.00|11.40|33.36|29.40|7.52|2.25|1.63|62.76| +2451569|68466|12088|61257|1222925|3346|41767|61257|1222925|3346|41767|7|20|4845|8|101.12|7.07|108.19|62.58|18.72|77.86|20.23|3.03|88.37| +2451686|73575|4916|62932|1690649|5566|8483|62932|1690649|5566|8483|31|10|4846|5|240.80|0.00|240.80|45.83|128.10|21.67|208.17|10.96|173.93| +2451257|56657|14791|91368|38892|6416|37140|91368|38892|6416|37140|55|33|4847|8|1159.52|92.76|1252.28|99.34|830.64|46.38|144.70|968.44|1022.74| +2451109|70806|15674|6043|1390349|4608|18914|6043|1390349|4608|18914|14|8|4848|60|1930.80|77.23|2008.03|17.12|3342.60|212.38|1185.70|532.72|3436.95| +2451292|48422|17311|69837|1805108|3226|24497|69837|1805108|3226|24497|28|34|4850|54|3406.32|306.56|3712.88|81.10|2373.84|1839.41|814.79|752.12|2761.50| +2452165|61155|15017|5487|379368|2049|19930|5487|379368|2049|19930|26|33|4851|57|4191.78|41.91|4233.69|5.40|2954.88|2976.16|364.68|850.94|3002.19| +2451975|41168|14666|69192|209236|5280|39324|69192|209236|5280|39324|19|7|4851|46|1519.38|30.38|1549.76|89.13|1969.26|1413.02|87.21|19.15|2088.77| +2451702|9288|9188|18493|423547|5401|12502|18493|423547|5401|12502|34|7|4852|2|163.16|9.78|172.94|78.63|93.22|66.89|73.16|23.11|181.63| +2451642|28290|15998|28701|1049096|338|10733|28701|1049096|338|10733|13|26|4852|14|0.00|0.00|0.00|32.46|80.36|0.00|0.00|0.00|112.82| +2451611|2284|14776|67878|1810636|4594|15705|67878|1810636|4594|15705|43|19|4852|27|438.75|35.10|473.85|85.79|196.29|394.87|17.55|26.33|317.18| +2451656|78124|9508|91949|949823|6097|10371|91949|949823|6097|10371|31|8|4853|4|593.44|41.54|634.98|34.90|215.76|498.48|13.29|81.67|292.20| +2451723|41912|2947|36157|186733|2109|9485|36157|186733|2109|9485|32|25|4853|55|62.15|4.35|66.50|19.83|207.35|16.15|43.70|2.30|231.53| +2452387|27611|13897|30077|874483|4842|6848|30077|874483|4842|6848|3|30|4854|36|1703.52|153.31|1856.83|6.11|638.64|851.76|349.22|502.54|798.06| +2452289||9287|39763||||39763||2562|35785|11||4854||349.08|||72.38|150.96|132.65|||251.26| +2452432||8869|96476||2734|||||3063|23||4854|27|||473.21||||70.76|39.81|| +2452160|40227|1307|43679|1735253|1674|32848|43679|1735253|1674|32848|55|8|4855|33|1889.91|151.19|2041.10|20.40|559.02|604.77|848.19|436.95|730.61| +|59813|2135|69467|1903445|3823|49801|||3823|49801|15||4857|||27.35|574.45||176.05||201.60|99.31|263.48| +2452316|75651|13323|6430|46343|1103|47416|6430|46343|1103|47416|35|20|4857|31|434.00|8.68|442.68|24.36|146.01|73.78|302.58|57.64|179.05| +2452438|55854|3863|44266|1734013|1084|40522|44266|1734013|1084|40522|43|8|4857|2|229.62|4.59|234.21|36.58|119.60|80.36|141.79|7.47|160.77| +2452514|54949|14585|44266|1734013|1084|40522|44266|1734013|1084|40522|43|29|4857|13|0.00|0.00|0.00|83.75|276.90|0.00|0.00|0.00|360.65| +2452268|38377|15387|34178|1396681|2384|17687|34178|1396681|2384|17687|9|32|4858|31|748.34|67.35|815.69|34.96|69.13|501.38|224.73|22.23|171.44| +2452388|26445|6311|339|1859719|1052|5232|339|1859719|1052|5232|1|13|4858|8|216.24|6.48|222.72|55.89|189.92|142.71|45.58|27.95|252.29| +2452461|36509|16909|3062|1168366|517|38668|3062|1168366|517|38668|35|32|4858|8|412.08|4.12|416.20|93.61|124.96|342.02|65.85|4.21|222.69| +2451365|66122|4141|94899|1411907|915|13523|94899|1411907|915|13523|26|19|4860|38|904.78|27.14|931.92|4.07|180.88|380.00|278.13|246.65|212.09| +2452415|65933|5737|78089|1731161|4680|2152|78089|1731161|4680|2152|41|5|4861|58|2387.28|95.49|2482.77|96.58|1050.38|692.31|1203.42|491.55|1242.45| +2452579|77767|8573|46342|1558775|5961|47147|46342|1558775|5961|47147|59|32|4861|10|791.00|15.82|806.82|18.51|275.10|522.06|134.47|134.47|309.43| +2452357|43755|3001|91711|444061|2364|14314|91711|444061|2364|14314|19|25|4862|41|2077.88|41.55|2119.43|58.44|1114.79|41.55|40.72|1995.61|1214.78| +2452010|76746|16190|77009|1597829|4036|6351|77009|1597829|4036|6351|1|24|4863|27|1124.55|78.71|1203.26|74.51|1124.55|528.53|393.37|202.65|1277.77| +2452050|60056|5546|24117|1697322|1727|46132|24117|1697322|1727|46132|19|8|4863|19|451.82|18.07|469.89|95.30|513.57|338.86|44.05|68.91|626.94| +|12329|7743||861649||||||31764||6|4864||3193.56|127.74||||||325.75|| +2452618|66439|4693|35488|734557|6779|23448|35488|734557|6779|23448|6|29|4864|4|195.12|5.85|200.97|85.98|53.20|1.95|121.69|71.48|145.03| +2452665|77264|13261|90897|962348|6599|97|90897|962348|6599|97|24|28|4865|54|1236.06|0.00|1236.06|41.30|473.58|889.96|162.66|183.44|514.88| +2451789|73343|12613|15960|1904931|5902|6826|15960|1904931|5902|6826|20|22|4866|11|314.60|6.29|320.89|61.40|64.35|144.71|69.65|100.24|132.04| +2451721|33960|12985|75925|1542263|1401|4139|75925|1542263|1401|4139|50|33|4866|2|181.62|0.00|181.62|80.73|124.70|65.38|80.20|36.04|205.43| +2451672|63777|12596|96273|1489676|5479|24082|96273|1489676|5479|24082|40|10|4866|23|255.30|5.10|260.40|52.37|255.30|117.43|113.05|24.82|312.77| +2451600|39047|15380|48666|732729|3992|855|48666|732729|3992|855|10|21|4866|3|376.08|30.08|406.16|92.10|102.57|259.49|57.12|59.47|224.75| +2451767|44703|15758|56723|434050|2224|2351|56723|434050|2224|2351|13|17|4866|16|1808.64|0.00|1808.64|81.02|371.52|1067.09|363.35|378.20|452.54| +2450887|68179|15632|35712|1814034|3556|5478|35712|1814034|3556|5478|19|5|4867|6|569.16|5.69|574.85|78.26|151.14|267.50|72.39|229.27|235.09| +2450986|40503|8401|56364|654927|5490|42853|56364|654927|5490|42853|31|6|4867|50|208.50|10.42|218.92|51.49|21.50|37.53|47.87|123.10|83.41| +||13189|85021||||85021||||32|22|4867||4058.53|||99.07||||121.76|| +2451013|34985|14582|8145|1109576|2729|49981|8145|1109576|2729|49981|40|32|4867|38|3257.74|195.46|3453.20|85.77|2028.44|3160.00|54.73|43.01|2309.67| +2451107|17249|8842|67993|475222|851|42880|67993|475222|851|42880|20|31|4867|37|16.65|1.16|17.81|53.43|43.66|10.65|4.14|1.86|98.25| +2451662|72339|3682|876|716083|3643|16803|876|716083|3643|16803|46|10|4869|59|7141.36|214.24|7355.60|11.63|1311.57|4927.53|929.80|1284.03|1537.44| +2451734|45217|4924|98106|1430023|5587|11151|98106|1430023|5587|11151|44|23|4870|45|1296.45|103.71|1400.16|10.73|664.20|427.82|703.59|165.04|778.64| +2452411|67166|5711|17114|1177013|4465|46764|17114|1177013|4465|46764|41|16|4871|24|581.28|40.68|621.96|2.71|62.64|308.07|265.01|8.20|106.03| +2452450|46601|575|23249|8500|2104|41308|23249|8500|2104|41308|37|13|4872|15|1480.20|133.21|1613.41|46.91|770.85|251.63|847.71|380.86|950.97| +2452432|71108|2733|33428|754754|1680|35406|33428|754754|1680|35406|39|24|4872|44|4855.40|0.00|4855.40|92.98|1336.72|1602.28|162.65|3090.47|1429.70| +2451797||752||9953|||58873|9953|6186||||4873|32|824.96||824.96|30.41||247.48|||| +2451818|25266|3325|27256|556401|393|21938|27256|556401|393|21938|41|12|4874|1|139.23|1.39|140.62|6.93|41.39|84.93|30.95|23.35|49.71| +2451862|8964|10525|19896|1212030|964|30298|19896|1212030|964|30298|56|32|4874|25|684.75|0.00|684.75|85.57|57.00|479.32|82.17|123.26|142.57| +2451103|20734|3151|92574|68896|418|42787|92574|68896|418|42787|38|21|4875|4|106.72|9.60|116.32|66.32|43.04|46.95|58.57|1.20|118.96| +2451133|21823|6506|76994|1186119|1197|1922|76994|1186119|1197|1922|28|11|4875|40|268.40|10.73|279.13|7.97|82.80|241.56|13.68|13.16|101.50| +2451154|72433|4718|57687|146944|2601|6705|57687|146944|2601|6705|16|24|4875|5|82.05|1.64|83.69|59.59|28.80|6.56|21.13|54.36|90.03| +2451715|35400|392|50706|1415205|6457|16556|50706|1415205|6457|16556|40|35|4877|50|3750.00|300.00|4050.00|8.56|2100.00|1012.50|1752.00|985.50|2408.56| +2451753|20681|12571|67207|176242|124|16359|67207|176242|124|16359|16|35|4877|6|196.98|0.00|196.98|12.50|346.74|195.01|0.25|1.72|359.24| +2451386|49334|2122|31163|1706642|74|17553|31163|1706642|74|17553|4|21|4878|10|332.10|3.32|335.42|53.05|141.80|252.39|78.11|1.60|198.17| +2451413|15294|17233|82609|1122738|2867|38596|82609|1122738|2867|38596|32|15|4878|30|2014.50|120.87|2135.37|12.81|705.00|664.78|728.84|620.88|838.68| +2452187|62713|10793|94335|819482|4182|129|94335|819482|4182|129|57|10|4879|30|348.00|24.36|372.36|33.01|203.10|104.40|29.23|214.37|260.47| +2452557|29474|11940|43935|1570609|130|48051|43935|1570609|130|48051|1|10|4880|25|121.75|7.30|129.05|12.98|121.75|21.91|37.93|61.91|142.03| +2452743|58050|1243|63753|647881|6856|46184|63753|647881|6856|46184|12|14|4882|50|3063.50|0.00|3063.50|23.47|1718.50|2113.81|788.24|161.45|1741.97| +2452580|30573|747|83675|1222299|2170|4794|83675|1222299|2170|4794|55|29|4882|3|199.32|5.97|205.29|32.41|66.42|141.51|5.78|52.03|104.80| +2452715|9273|5496||1860621|3095||21700|1860621||16016|33|25|4882|6|||57.81|28.70||33.81|18.03||| +2452584|78094|14397|27992|659167|4295|43423|27992|659167|4295|43423|9|20|4882|47|6.58|0.52|7.10|58.91|26.79|5.79|0.53|0.26|86.22| +||4615|49455||3056|25274||287460||25274|31|2|4884|32|||2859.75|97.06|||||| +2452497|42375|9003|49455|287460|3056|25274|49455|287460|3056|25274|17|27|4884|16|631.20|50.49|681.69|54.34|502.56|265.10|43.93|322.17|607.39| +2452822|35877|10878|55397|1914641|1532|42776|55397|1914641|1532|42776|7|23|4886|2|181.12|1.81|182.93|97.31|103.04|92.37|63.90|24.85|202.16| +2451368|74146|15022|41188|1482607|1626|37045|41188|1482607|1626|37045|58|21|4889|30|957.30|67.01|1024.31|4.43|360.90|210.60|7.46|739.24|432.34| +2452390|19064|5131|24851|1310882|5266|24497|24851|1310882|5266|24497|39|4|4890|78|3147.30|125.89|3273.19|23.91|1333.80|912.71|1854.70|379.89|1483.60| +2451881|21512|1037|3031|1378672|3113|47954|3031|1378672|3113|47954|20|25|4891|21|88.20|4.41|92.61|6.71|73.50|88.20|0.00|0.00|84.62| +2452762|31699|17973|38118|1448204|4690|23976|38118|1448204|4690|23976|6|5|4893|8|313.60|9.40|323.00|93.41|161.92|304.19|6.21|3.20|264.73| +2452733|76421|9816|43989|1148340|1044|34948|43989|1148340|1044|34948|13|34|4893|2|13.86|0.55|14.41|78.52|10.32|11.50|0.59|1.77|89.39| +2452800|31602|8883|38438|1079426|6814|35402|38438|1079426|6814|35402|31|32|4893|43|79.12|3.95|83.07|52.91|0.00|45.09|31.98|2.05|56.86| +2452328|43634|17731|63370|403161|3752|43990|63370|403161|3752|43990|59|20|4894|4|326.12|19.56|345.68|17.90|5.80|169.58|57.91|98.63|43.26| +2451213|8477|8438|79742|552784|2979|41221|79742|552784|2979|41221|31|7|4895|20|696.80|41.80|738.60|73.20|944.20|139.36|133.78|423.66|1059.20| +2451137|80975|17564|61358|1537753|101|7696|61358|1537753|101|7696|1|15|4895|5|14.30|0.71|15.01|57.44|66.20|3.43|10.32|0.55|124.35| +2452344|27346|7843|18686|1478234|6229|15777|18686|1478234|6229|15777|21|16|4897|15|326.40|3.26|329.66|12.48|345.60|212.16|103.95|10.29|361.34| +2451669|15174|17683|52114|977372|6630|28688|52114|977372|6630|28688|10|28|4900|43|1807.29|90.36|1897.65|39.15|1505.86|1120.51|487.61|199.17|1635.37| +2451825|73205|10598|35785|821871|757|9290|35785|821871|757|9290|13|11|4900|4|307.76|0.00|307.76|68.74|11.36|141.56|99.72|66.48|80.10| +2452412|17376|743|43635|1351762|7089|13216|43635|1351762|7089|13216|1|8|4902|17|1757.12|140.56|1897.68|28.26|563.21|70.28|16.86|1669.98|732.03| +2452321|50752|12049|92690|394322|3954|26786|92690|394322|3954|26786|5|23|4904|3|539.37|5.39|544.76|7.69|204.96|210.35|98.70|230.32|218.04| +2452370|61684|4011|79806|722547|2088|7982|79806|722547|2088|7982|3|16|4904|23|741.29|14.82|756.11|29.20|56.81|444.77|124.53|171.99|100.83| +2452308|44853|6159|94251|1152921|2791|19014|94251|1152921|2791|19014|17|10|4904|13|235.95|21.23|257.18|78.22|310.57|49.54|108.11|78.30|410.02| +2452422|71977|821|15505|14758|58|48958|15505|14758|58|48958|49|19|4905|33|89.43|1.78|91.21|97.39|42.57|0.89|43.38|45.16|141.74| +2451338|33070|8587|53453|861149|2798|36447|53453|861149|2798|36447|25|25|4906|14|1051.26|94.61|1145.87|60.95|434.98|441.52|189.01|420.73|590.54| +2451494|67422|3766|38311|180761|5164|9567|38311|180761|5164|9567|1|25|4906|39|466.44|37.31|503.75|16.45|48.36|116.61|132.93|216.90|102.12| +2451422|21728|16282|3140|17949|4758|27967|3140|17949|4758|27967|1|6|4908|24|1264.08|37.92|1302.00|62.34|296.16|214.89|818.36|230.83|396.42| +2451885|69610|7898|69294|1504026|3558|41043|69294|1504026|3558|41043|26|31|4910|54|11855.70|237.11|12092.81|12.15|4041.36|1896.91|5377.74|4581.05|4290.62| +2451350|27207|4945|42233|1789242|604|46013|42233|1789242|604|46013|22|9|4914|62|955.42|19.10|974.52|13.06|157.48|535.03|323.70|96.69|189.64| +2451782|24162|2033|50802|1785510|2171|15232|50802|1785510|2171|15232|47|17|4916|21|307.86|24.62|332.48|55.57|1400.07|141.61|146.30|19.95|1480.26| +2451957|34023|15787|36085|1287692|4099|28106|36085|1287692|4099|28106|17|18|4916|1|90.50|1.81|92.31|22.39|47.71|2.71|9.65|78.14|71.91| +2451730|80479|5996|98927|1752019|664|26956|98927|1752019|664|26956|26|28|4917|83|839.96|41.99|881.95|21.62|735.38|545.97|67.61|226.38|798.99| +2451558|60050|7012|47335|1817619|703|4987|47335|1817619|703|4987|40|6|4917|36|3612.60|72.25|3684.85|3.63|1565.28|2167.56|447.96|997.08|1641.16| +2451620|23796|8761|65326|512893|13|17133|65326|512893|13|17133|41|32|4919|5|90.35|5.42|95.77|95.44|25.80|37.04|2.13|51.18|126.66| +2451898|44593|3410|94411|1282936|7108|44033|94411|1282936|7108|44033|14|4|4919|28|633.64|25.34|658.98|64.33|613.20|297.81|33.58|302.25|702.87| +2451294|30710|10256|46828|1319837|4493|35549|46828|1319837|4493|35549|44|4|4920|6|12.90|0.77|13.67|21.10|19.32|0.77|4.97|7.16|41.19| +2451476|20481|7465|18249|843943|1990|38884|18249|843943|1990|38884|58|2|4921|23|431.48|8.62|440.10|74.72|120.98|51.77|182.26|197.45|204.32| +2451448|42419|13582|60996|1464945|94|40396|60996|1464945|94|40396|26|21|4921|58|1400.70|112.05|1512.75|42.50|314.94|490.24|582.69|327.77|469.49| +2451580|57190|9103|41157|1499387|6417|45847|41157|1499387|6417|45847|43|7|4921|32|1674.24|50.22|1724.46|76.75|195.52|552.49|1009.57|112.18|322.49| +2452344|54716|8893|29377|532204|6678|27152|29377|532204|6678|27152|45|10|4923|20|272.60|8.17|280.77|13.51|144.60|43.61|206.09|22.90|166.28| +2452548|56121|7379|49839|1629237|6682|927|49839|1629237|6682|927|39|5|4923|45|1760.85|158.47|1919.32|15.90|283.95|792.38|503.60|464.87|458.32| +2451958|66583|16394|49024|1031413|1355|7697|49024|1031413|1355|7697|31|11|4924|8|1239.92|99.19|1339.11|46.03|984.08|1004.33|106.01|129.58|1129.30| +2452712|55383|10314|77159|665571|5536|37548|77159|665571|5536|37548|18|19|4928|5|80.00|1.60|81.60|78.05|166.15|37.60|33.49|8.91|245.80| +2451466|9389|2324|84424|1190174|1524|46921|84424|1190174|1524|46921|14|8|4929|11|921.91|27.65|949.56|26.97|582.23|165.94|559.41|196.56|636.85| +2451496|83867|9428|53329|1900705|2034|10163|53329|1900705|2034|10163|4|21|4929|63|640.71|12.81|653.52|52.93|56.70|378.01|170.75|91.95|122.44| +2452763|43482|10023|95426|572321|1216|16511|95426|572321|1216|16511|3|27|4930|10|117.60|8.23|125.83|40.68|1011.50|79.96|27.85|9.79|1060.41| +2452730|69746|12618|38723|1049206|4842|40525|38723|1049206|4842|40525|42|11|4930|1|15.53|1.24|16.77|32.47|7.36|13.82|0.25|1.46|41.07| +2452744|5155|8370|18663|1377256|4778|47121|18663|1377256|4778|47121|31|19|4930|9|8.73|0.17|8.90|96.68|70.29|2.00|1.61|5.12|167.14| +2452667|43004|11853|67300|270992|3131|7270|67300|270992|3131|7270|13|10|4930|30|54.60|1.09|55.69|84.40|437.70|25.66|25.46|3.48|523.19| +2451841|75151|10003|45645|782644|4443|25889|45645|782644|4443|25889|29|34|4931|1|6.97|0.55|7.52|99.73|0.48|1.95|4.51|0.51|100.76| +2451881|58670|9065|80428|457126|6271|47369|80428|457126|6271|47369|20|9|4931|75|155.25|12.42|167.67|52.70|1090.50|0.00|74.52|80.73|1155.62| +2451465|74876|13264|35052|889768|1731|9247|35052|889768|1731|9247|31|2|4932|71|1929.07|115.74|2044.81|10.60|206.61|1350.34|277.79|300.94|332.95| +2451369|48231|1192|38489|1413419|5149|7279|38489|1413419|5149|7279|7|34|4932|3|66.66|3.33|69.99|67.07|76.68|4.66|26.66|35.34|147.08| +2452403|32521|9301|73133|1059607|4928|30649|73133|1059607|4928|30649|37|14|4933|8|20.96|0.00|20.96|94.65|128.64|1.46|12.48|7.02|223.29| +2452573|68787|570|75570|935625|5280|41688|75570|935625|5280|41688|51|11|4933|86|2781.24|27.81|2809.05|11.33|115.24|834.37|798.21|1148.66|154.38| +2452605|29309|1203|98325|358911|650|23236|98325|358911|650|23236|60|14|4933|27|193.05|3.86|196.91|66.00|310.50|137.06|19.03|36.96|380.36| +2452621|27894|10224|43904|1690289|1027|612|43904|1690289|1027|612|18|16|4933|1|117.33|5.86|123.19|97.73|22.92|52.79|42.59|21.95|126.51| +2452319|53421|5763|90206|485261|7069|23324|90206|485261|7069|23324|19|30|4934|9|3.60|0.10|3.70|16.65|2.52|1.76|0.80|1.04|19.27| +2452068|66991|13387|81312|1105721|1330|1739|81312|1105721|1330|1739|50|2|4935|43|1697.21|135.77|1832.98|64.78|407.21|1018.32|407.33|271.56|607.76| +2452118|18694|7091|24763|864266|5016|37319|24763|864266|5016|37319|31|19|4935|8|418.56|0.00|418.56|61.58|364.56|16.74|120.54|281.28|426.14| +2451952|39082|5407|29074|1546809|3297|35432|29074|1546809|3297|35432|25|26|4935|82|19074.84|381.49|19456.33|35.89|601.88|0.00|10681.91|8392.93|1019.26| +2451881|58988|7544|11473|1413393|1713|30033|11473|1413393|1713|30033|14|8|4935|4|148.72|4.46|153.18|90.20|172.84|7.43|118.68|22.61|267.50| +2451694|77103|4117|77643|424813|2030|3200|77643|424813|2030|3200|44|11|4937|12|608.76|12.17|620.93|15.13|345.00|243.50|84.00|281.26|372.30| +2451688|41356|12865|8038|832302|2502|14713|8038|832302|2502|14713|7|15|4938|44|335.72|23.50|359.22|85.17|223.96|23.50|149.86|162.36|332.63| +2451821|22982|500|60449|305637|4425|17213|60449|305637|4425|17213|19|33|4938|13|2640.69|158.44|2799.13|70.95|728.39|580.95|1462.41|597.33|957.78| +2452508|72979|7437|30091|682823|6402|39534|30091|682823|6402|39534|51|7|4943|7|618.31|24.73|643.04|82.02|224.84|228.77|381.74|7.80|331.59| +2452322|12759|12259|14533|142143|153|22963|14533|142143|153|22963|51|3|4943|7|854.42|51.26|905.68|14.15|146.09|751.88|95.36|7.18|211.50| +2452451|54699|8781|15682|409874|2019|24611|15682|409874|2019|24611|15|25|4943|21|621.60|31.08|652.68|51.22|419.16|304.58|250.44|66.58|501.46| +2452092|53787|9305|23450|1544824|4245|32866|23450|1544824|4245|32866|47|17|4944|4|156.40|6.25|162.65|96.66|156.40|114.17|36.31|5.92|259.31| +2452602|22758|4135|16698|1135865|3832|17399|16698|1135865|3832|17399|25|1|4948|4|386.80|27.07|413.87|59.84|62.56|193.40|145.05|48.35|149.47| +2451740|78192|2924|70006||3014|33518|70006||3014|33518|||4949|29||39.51||86.16||177.81|283.66||| +2451868|32337|12421|61562|127706|3803|31100|61562|127706|3803|31100|14|9|4949|10|502.50|30.15|532.65|40.85|178.30|412.05|77.78|12.67|249.30| +2452691|71343|1389|14609|520551|5693|30435|14609|520551|5693|30435|7|14|4951|53|9161.05|549.66|9710.71|52.97|3772.01|1190.93|6216.69|1753.43|4374.64| +2452897|27596|16641|29166|862790|2014|9638|29166|862790|2014|9638|37|30|4952|8|718.48|50.29|768.77|73.47|85.84|474.19|229.63|14.66|209.60| +2451900|35764|17204|76756|1493428|2100|7572|76756|1493428|2100|7572|20|16|4953|42|1744.26|139.54|1883.80|46.22|1682.10|1657.04|66.28|20.94|1867.86| +2452785|51195|1245|3942|1339535|1881|17474|3942|1339535|1881|17474|57|18|4955|37|1820.77|127.45|1948.22|39.27|116.55|1474.82|280.21|65.74|283.27| +2452505|21155|8407|27758|1541839|5967|20495|27758|1541839|5967|20495|31|34|4958|17|347.31|27.78|375.09|79.25|403.58|340.36|3.47|3.48|510.61| +|33984|3253||870028||15104|26890|870028|3566|15104||18|4958||7488.51||||2754.20|6065.69|1308.99|113.83|3210.33| +2452396|25505|13529|42059|213364|5469|28374|42059|213364|5469|28374|35|28|4958|79|3209.77|288.87|3498.64|83.79|2818.72|2760.40|238.16|211.21|3191.38| +2452270|55439|11445|90219|1065056|5846|28236|90219|1065056|5846|28236|57|35|4959|25|279.75|11.19|290.94|98.13|82.25|44.76|89.29|145.70|191.57| +2452464|58771|10813|89070|1917242|2661|29680|89070|1917242|2661|29680|1|5|4961|1|10.38|0.20|10.58|28.93|49.85|6.53|0.80|3.05|78.98| +|22691|17185||8082||48918|||2900||||4961|13||5.45||16.61||||34.90|| +2452871|40382|3367|56544|141033|5522|38411|56544|141033|5522|38411|43|30|4962|26|1534.52|138.10|1672.62|85.33|1273.74|1335.03|17.95|181.54|1497.17| +2451169|37640|5078|47256|555495|1687|6726|47256|555495|1687|6726|32|4|4963|29|669.90|53.59|723.49|77.82|957.00|241.16|111.47|317.27|1088.41| +2451062|9596|1843|78643|1399322|1248|24774|78643|1399322|1248|24774|50|28|4963|3|13.95|0.69|14.64|70.83|6.63|11.02|1.87|1.06|78.15| +2451775|20090|7904|18283|1230043|4203|42716|18283|1230043|4203|42716|16|4|4964|12|927.24|83.45|1010.69|47.09|536.04|927.24|0.00|0.00|666.58| +2451969|45613|5869|2121|1456632|1990|43301|2121|1456632|1990|43301|35|4|4967|26|82.16|7.39|89.55|19.51|987.22|12.32|46.79|23.05|1014.12| +2452138|77136|6908|41929|328877|5425|44373|41929|328877|5425|44373|31|33|4969|8|7.52|0.37|7.89|52.59|167.04|2.78|1.89|2.85|220.00| +2452147|27425|332|44317|350452|2987|29075|44317|350452|2987|29075|53|33|4969|36|550.80|33.04|583.84|49.97|341.28|418.60|92.54|39.66|424.29| +2451102|77414|9013|95565|1841827|4869|32661|95565|1841827|4869|32661|19|8|4971|21|1002.75|0.00|1002.75|47.11|686.07|651.78|217.60|133.37|733.18| +2451286|53293|3758|96247|228954|6814|19121|96247|228954|6814|19121|19|17|4971|87|2021.01|181.89|2202.90|54.14|443.70|949.87|856.91|214.23|679.73| +2452375|64094|10731|18120|1754937|5001|26218|18120|1754937|5001|26218|55|4|4972|49|2567.11|179.69|2746.80|98.85|716.87|2515.76|49.80|1.55|995.41| +2452401|21225|6379||||37377||||||16|4972|5|316.05||331.85|41.97|101.10||||158.87| +2452725|47558|11299|54457|1282461|1665|41316|54457|1282461|1665|41316|13|17|4973|7|150.50|3.01|153.51|12.33|65.31|139.96|8.11|2.43|80.65| +||17335|28886|437661|5636|9211|||5636|||26|4974||||2186.10|7.90|63.00||92.02|178.64|175.00| +2451541|36550|17515|97864|786301|785|47222|97864|786301|785|47222|25|23|4974|76|3774.16|264.19|4038.35|12.38|3990.00|37.74|3512.23|224.19|4266.57| +2451515|45886|1495||222100|7103||68297|222100|7103|14906|56|6|4974||||267.57||110.21|||76.93|| +2451448|33293|12566|76170|482246|675|9504|76170|482246|675|9504|7|17|4974|26|927.42|27.82|955.24|53.24|511.68|797.58|20.77|109.07|592.74| +2451602|18906|6208|12437|1210888|2586|28237|12437|1210888|2586|28237|34|21|4974|53|6832.76|478.29|7311.05|83.90|3564.78|6491.12|116.15|225.49|4126.97| +2451359|54465|5929|81040|214789|985|46734|81040|214789|985|46734|58|29|4975|66|4731.54|189.26|4920.80|52.91|1764.18|4400.33|33.12|298.09|2006.35| +2451343|62375|7496|25541|837828|4868|29955|25541|837828|4868|29955|58|23|4975|51|249.39|2.49|251.88|67.59|1310.70|2.49|93.82|153.08|1380.78| +2451931|67589|10151|91999|995846|46|3846|91999|995846|46|3846|8|18|4977|57|620.73|49.65|670.38|26.99|174.42|68.28|154.68|397.77|251.06| +2452055|18477|14465|31035|1326525|415|25552|31035|1326525|415|25552|7|7|4977|73|7257.66|580.61|7838.27|5.63|3244.12|3773.98|2647.59|836.09|3830.36| +2452083|68870|8948|61833|1828256|5879|21502|61833|1828256|5879|21502|19|9|4977|26|1913.86|0.00|1913.86|19.85|472.68|1033.48|422.58|457.80|492.53| +2451580|32910|11210|72890|218412|4381|39503|72890|218412|4381|39503|56|6|4978|94|8243.80|329.75|8573.55|86.69|7213.56|8243.80|0.00|0.00|7630.00| +2451643|50238|3994|78853|1914418|3583|22748|78853|1914418|3583|22748|25|1|4978|19|3218.41|32.18|3250.59|47.67|1048.61|482.76|547.13|2188.52|1128.46| +2451229|61316|3268|80339|47995|2329|35665|80339|47995|2329|35665|26|24|4979|52|895.44|0.00|895.44|93.20|1268.80|886.48|6.09|2.87|1362.00| +||2995|95876|1360084||14947|||5539|14947|||4979|13|51.61|0.51|52.12|7.36|11.83|43.86|0.54|7.21|19.70| +2451336|62114|2822|61879|564332|3585|11591|61879|564332|3585|11591|58|19|4981|17|887.74|17.75|905.49|17.87|312.29|284.07|217.32|386.35|347.91| +2452327|29332|15221|74820|1288021|7175|25529|74820|1288021|7175|25529|39|12|4982|52|9972.04|698.04|10670.08|19.46|4481.88|6980.42|329.07|2662.55|5199.38| +2452476|42083|9505|99310|1765584|6611|19285|99310|1765584|6611|19285|55|31|4982|25|555.75|5.55|561.30|53.38|1072.00|38.90|289.43|227.42|1130.93| +2451926|19960|9941|63488|1074388|4935|17435|63488|1074388|4935|17435|53|24|4983|1|11.34|0.79|12.13|64.66|5.87|4.53|0.47|6.34|71.32| +2452092|72595|16279|53312|309700|4488|22029|53312|309700|4488|22029|29|5|4983|31|1535.74|76.78|1612.52|22.32|1939.67|1028.94|466.25|40.55|2038.77| +2451129|65772|17521|1186|1667574|3209|3244|1186|1667574|3209|3244|7|20|4984|21|175.14|14.01|189.15|48.14|121.80|89.32|12.87|72.95|183.95| +2452094|46259|14807|23335|1340023|1864|32105|23335|1340023|1864|32105|20|22|4988|12|29.04|0.29|29.33|14.82|1.44|24.39|4.37|0.28|16.55| +2451943|49348|271|16447|815784|6768|1352|16447|815784|6768|1352|59|6|4988|58|10081.56|302.44|10384.00|67.92|2458.62|3326.91|3850.15|2904.50|2828.98| +2452930|73447|16569|56850|40094|5291|40935|56850|40094|5291|40935|12|10|4989|20|1379.40|124.14|1503.54|51.27|1026.40|1131.10|14.89|233.41|1201.81| +2452643|49154|17937|11057|1721057|6257|47057|11057|1721057|6257|47057|36|30|4989|70|10625.30|318.75|10944.05|65.54|5405.40|7862.72|1381.29|1381.29|5789.69| +2451278|13819|9038|31994|466941|288|44835|31994|466941|288|44835|13|12|4990|12|1364.28|54.57|1418.85|65.19|1395.24|886.78|420.20|57.30|1515.00| +2452168|34988|11987|16637|1197128|3666|19004|16637|1197128|3666|19004|43|17|4991|5|433.85|4.33|438.18|36.35|141.35|21.69|300.87|111.29|182.03| +2452067|71443|14006|97203|277453|6503|19906|97203|277453|6503|19906|31|24|4994|53|5647.68|338.86|5986.54|20.14|4054.50|4969.95|521.85|155.88|4413.50| +2452416|46197|2143|37720|745158|3796|32387|37720|745158|3796|32387|9|8|4997|1|75.93|0.00|75.93|59.94|33.46|3.79|51.94|20.20|93.40| +2452376|47646|14461|62604|946821|5438|27702|62604|946821|5438|27702|3|33|4997|77|159.39|0.00|159.39|98.64|927.85|124.32|1.40|33.67|1026.49| +2451275|73894|7222|13545|1600380|5968|22756|13545|1600380|5968|22756|26|20|4998|42|2689.68|188.27|2877.95|81.90|1430.52|430.34|835.95|1423.39|1700.69| +2451056|36875|14674|62158|467061|6011|2161|62158|467061|6011|2161|16|15|4998|4|11.52|0.69|12.21|95.26|9.20|7.71|0.87|2.94|105.15| +2451899|32037|5636|77721|1682189|2210|2678|77721|1682189|2210|2678|2|3|4999|51|2300.10|92.00|2392.10|15.43|2195.55|23.00|683.13|1593.97|2302.98| +2451553|64234|12734|78736|1193205|6874|22943|78736|1193205|6874|22943|43|22|5000|94|595.02|53.55|648.57|88.42|4640.78|220.15|243.66|131.21|4782.75| +2451826|39123|7558|2798|331281|4564|42647|2798|331281|4564|42647|14|24|5000|20|4487.60|359.00|4846.60|21.58|942.20|3051.56|229.76|1206.28|1322.78| +2451770|32488|9116|35004|136456|3508|27313|35004|136456|3508|27313|38|28|5000|55|13462.35|1211.61|14673.96|11.50|438.90|673.11|10487.17|2302.07|1662.01| +2451555|85353|16426|76671|221774|5277|41933|76671|221774|5277|41933|56|32|5001|9|143.19|4.29|147.48|95.17|92.43|37.22|57.22|48.75|191.89| +2451715|52005|8365|75218|1092468|5271|39321|75218|1092468|5271|39321|40|35|5001|43|4329.67|216.48|4546.15|46.23|1082.31|2597.80|1541.36|190.51|1345.02| +2451582|41551|11780|47623|1007209|1548|20734|47623|1007209|1548|20734|52|22|5001|10|277.50|24.97|302.47|42.22|704.60|241.42|30.30|5.78|771.79| +2451832||10567||918597|1378|43006||||43006|53||5002||4265.12||4648.98||2095.35|1535.44|1555.91||2482.33| +2452827|51822|3553|50727|619162|1644|22126|50727|619162|1644|22126|15|7|5007|22|1602.04|144.18|1746.22|75.30|345.84|1057.34|152.51|392.19|565.32| +2452588|18035|7461|78985|1617033|1128|49976|78985|1617033|1128|49976|12|3|5007|21|1874.88|56.24|1931.12|42.97|1090.11|506.21|68.43|1300.24|1189.32| +2452678|63809|2877|72354|||29233||960098|1889|||13|5007|||||||957.95|478.98||| +2452081|80060|17453|63784|916275|6366|15257|63784|916275|6366|15257|3|10|5008|16|488.48|4.88|493.36|16.43|27.04|254.00|140.68|93.80|48.35| +2451121|5321|4972|22952|1601086|6067|2201|22952|1601086|6067|2201|56|9|5009|6|252.78|5.05|257.83|79.45|540.00|60.66|84.53|107.59|624.50| +2451054|14942|9292|5175|202794|4413|2832|5175|202794|4413|2832|20|26|5009|30|1372.50|13.72|1386.22|71.70|2338.50|892.12|62.44|417.94|2423.92| +2451965|61312|16352|7010|1706707|2404|32501|7010|1706707|2404|32501|13|10|5011|16|1516.96|30.33|1547.29|87.65|593.60|364.07|611.03|541.86|711.58| +2451591|66480|15224|81774|306521|4068|26862|81774|306521|4068|26862|8|31|5012|2|207.18|8.28|215.46|76.31|35.86|58.01|55.19|93.98|120.45| +2451596|44653|158|18085|965693|6901|30509|18085|965693|6901|30509|31|4|5012|12|931.56|65.20|996.76|8.92|481.80|428.51|241.46|261.59|555.92| +2451711|21884|14158|97194|1583170|2704|1290|97194|1583170|2704|1290|40|10|5012|41|1211.96|60.59|1272.55|49.91|358.34|1115.00|17.45|79.51|468.84| +2451768|54106|10466|52819|1428582|1145|15308|52819|1428582|1145|15308|56|1|5013|78|11544.00|461.76|12005.76|30.21|3794.70|2308.80|3417.02|5818.18|4286.67| +2452310|45947|2637|1975|256919|216|47513|1975|256919|216|47513|7|31|5014|8|1306.80|0.00|1306.80|95.33|597.36|287.49|835.83|183.48|692.69| +2452035|33835|3817|98112|1108954|6196|33485|98112|1108954|6196|33485|7|11|5015|16|1911.36|57.34|1968.70|38.05|1005.92|401.38|1434.48|75.50|1101.31| +2451969|66655|16916|30887|926013|1586|33912|30887|926013|1586|33912|1|32|5015|18|540.00|0.00|540.00|41.27|990.00|124.20|145.53|270.27|1031.27| +2452038|19829|3665|89603|939834|1265|8296|89603|939834|1265|8296|20|10|5015|41|1189.41|107.04|1296.45|81.62|1401.79|666.06|235.50|287.85|1590.45| +2452866|50488|14382|91805|1214057|5556|12208|91805|1214057|5556|12208|33|17|5018|4|5.92|0.11|6.03|20.02|62.16|4.55|1.21|0.16|82.29| +2452702|70900|13428|72763|1521513|5463|27813|72763|1521513|5463|27813|39|5|5018|13|320.84|0.00|320.84|99.00|127.01|35.29|177.04|108.51|226.01| +2450996|69828|4444|5620|110354|3690|11546|5620|110354|3690|11546|52|1|5020|9|711.18|0.00|711.18|82.74|223.74|547.60|155.40|8.18|306.48| +|79149|5161||1448766|4360|17201|8725||4360||49||5020||||51.77||||||| +2452419|27111|3447|82534|532011|3888|8965|82534|532011|3888|8965|1|17|5021|50|1498.50|74.92|1573.42|22.11|80.50|674.32|140.11|684.07|177.53| +2451892|49344|3887|35828|207313|3151|1036|35828|207313|3151|1036|19|2|5022|28|2172.80|152.09|2324.89|49.75|1409.24|347.64|949.08|876.08|1611.08| +2451966|46647|217|68543|582996|6602|26655|68543|582996|6602|26655|29|14|5023|5|219.95|13.19|233.14|44.10|17.10|107.77|102.08|10.10|74.39| +2452230|68138|13965|6113|1296127|6494|12508|6113|1296127|6494|12508|15|16|5024|22|332.86|23.30|356.16|6.48|237.82|83.21|82.38|167.27|267.60| +2452253|14638|4799|2282|859104|4726|46748|2282|859104|4726|46748|27|10|5024|64|311.04|12.44|323.48|2.95|66.56|220.83|12.62|77.59|81.95| +2452566|79592|3819|42267|341237|5852|48984|42267|341237|5852|48984|31|3|5026|3|21.48|1.07|22.55|39.90|123.54|5.37|15.78|0.33|164.51| +2451557|56397|1126|16690|615853|3016|41779|16690|615853|3016|41779|50|8|5028|50|5781.50|115.63|5897.13|74.74|4247.50|3584.53|812.87|1384.10|4437.87| +2452573|78240|12187|12311|654700|5889|31078|12311|654700|5889|31078|39|19|5029|31|514.60|20.58|535.18|71.72|1939.67|339.63|82.23|92.74|2031.97| +2452696|32582|645|78984|1371032|4233|47481|78984|1371032|4233|47481|13|5|5029|11|825.44|24.76|850.20|33.87|195.47|231.12|588.37|5.95|254.10| +2452748|73896|9504|78104|274534|6964|47394|78104|274534|6964|47394|43|10|5029|25|1532.25|122.58|1654.83|69.87|1751.25|367.74|337.70|826.81|1943.70| +2452712|36265|1386|83056|677307|2311|10562|83056|677307|2311|10562|36|20|5030|11|80.63|3.22|83.85|69.93|197.45|8.06|34.83|37.74|270.60| +2451541|63889|11287|9866|1644262|705|15101|9866|1644262|705|15101|28|34|5031|15|325.05|0.00|325.05|21.53|214.65|260.04|3.25|61.76|236.18| +2451220|43221|1333|79704|337258|2012|13566|79704|337258|2012|13566|13|7|5032|81|411.48|28.80|440.28|10.31|2939.49|312.72|93.82|4.94|2978.60| +2451870|73290|3991|29777|134080|4783|13486|29777|134080|4783|13486|29|4|5033|38|2277.34|113.86|2391.20|65.70|1007.00|1844.64|151.44|281.26|1186.56| +||5581|27575|||10444|||6021|||17|5033||600.73|42.05|642.78||274.04|||233.45|409.78| +2451765|5035|3686|54238|578658|678|8698|54238|578658|678|8698|19|23|5035|4|183.68|7.34|191.02|80.99|64.48|73.47|2.20|108.01|152.81| +2451690|38279|730|77497|920865|1480|48848|77497|920865|1480|48848|20|14|5035|68|7815.24|547.06|8362.30|28.15|943.16|3360.55|2895.54|1559.15|1518.37| +2451668|57146|14254|73883|768142|1601|48307|73883|768142|1601|48307|25|26|5035|11|825.11|33.00|858.11|76.12|529.54|561.07|174.26|89.78|638.66| +2451842|75076|17606|17248|458621|1594|15320|17248|458621|1594|15320|37|31|5035|33|693.33|48.53|741.86|97.21|71.61|395.19|98.38|199.76|217.35| +2451584|5387|16796|76347|1026405|1263|34121|76347|1026405|1263|34121|34|32|5036|32|331.52|0.00|331.52|91.83|967.04|318.25|5.97|7.30|1058.87| +2452473|8170|6385|41214|273316|7018|8720|41214|273316|7018|8720|55|9|5038|6|1164.66|104.81|1269.47|32.00|223.50|978.31|175.16|11.19|360.31| +2452102|74082|6437|12286|976512|5538|164|12286|976512|5538|164|37|8|5040|36|2316.60|208.49|2525.09|86.24|899.28|1876.44|413.75|26.41|1194.01| +2451953|26722|12211|2326|288289|5452|44262|2326|288289|5452|44262|32|4|5040|21|1805.58|72.22|1877.80|1.12|481.32|1191.68|411.31|202.59|554.66| +|70736|9955||||19066||1878336|7101|19066|||5042||298.32|20.88||96.84|||157.04|123.39|378.72| +2452447|81462|10895|60571|361266|1514|30209|60571|361266|1514|30209|1|11|5042|12|290.64|2.90|293.54|86.13|1050.96|84.28|130.00|76.36|1139.99| +2452172|22365|10539|36971|1402365|206|38447|36971|1402365|206|38447|39|17|5043|13|728.65|7.28|735.93|8.86|137.80|0.00|174.87|553.78|153.94| +||11787|11824|||||123291||46978|13||5043||674.00|13.48||||114.58||453.14|1367.07| +2452487|45134|16445|13287|1590199|2311|15270|13287|1590199|2311|15270|15|14|5045|29|2317.97|23.17|2341.14|40.62|1633.86|370.87|1012.49|934.61|1697.65| +2452294|60816|11185|69452|1163180|4908|43436|69452|1163180|4908|43436|11|33|5046|9|902.34|0.00|902.34|46.85|649.71|505.31|138.96|258.07|696.56| +2452225|34640|17211|83313|1009852|3238|2224|83313|1009852|3238|2224|7|33|5046|21|861.42|60.29|921.71|48.30|346.50|206.74|202.95|451.73|455.09| +2451785|48762|5515|28296|935538|5627|8069|28296|935538|5627|8069|7|13|5047|43|3455.48|241.88|3697.36|52.43|325.94|2384.28|492.75|578.45|620.25| +2452022|70113|17543|27321|1050339|2604|4822|27321|1050339|2604|4822|29|28|5048|41|4356.25|304.93|4661.18|67.42|1698.22|0.00|1873.18|2483.07|2070.57| +2451709|73564|3818|49329|1593492|1655|25371|49329|1593492|1655|25371|37|27|5049|2|22.68|1.58|24.26|51.37|40.82|17.23|2.07|3.38|93.77| +2451787|8986|7784|13165|1180393|5221|39696|13165|1180393|5221|39696|25|13|5049|8|219.44|4.38|223.82|18.26|20.24|87.77|73.73|57.94|42.88| +2452184|22524|465|34908|1837980|5499|26618|34908|1837980|5499|26618|15|15|5054|63|1044.54|94.00|1138.54|26.76|2938.95|0.00|752.06|292.48|3059.71| +2452066|38738|4001|94777|1295565|1153|388|94777|1295565|1153|388|13|20|5054|16|479.04|9.58|488.62|95.36|110.56|234.72|171.02|73.30|215.50| +2452165|86106|7345|35440|1183550|460|48217|35440|1183550|460|48217|9|16|5054|2|228.84|0.00|228.84|85.88|15.00|210.53|8.78|9.53|100.88| +2451933|11325|8171|15812|1700075|4785|38248|15812|1700075|4785|38248|41|2|5058|77|342.65|23.98|366.63|84.73|2571.03|185.03|45.70|111.92|2679.74| +2452217|14881|12707|74879|196675|4071|2667|74879|196675|4071|2667|11|5|5059|82|3162.74|284.64|3447.38|15.36|3575.20|2024.15|170.78|967.81|3875.20| +2452205|69593|3443|46894|1579428|5115|9649|46894|1579428|5115|9649|57|1|5059|1|0.75|0.00|0.75|30.36|1.89|0.00|0.14|0.61|32.25| +2451379|44886|4540|89639|1530098|6110|30169|89639|1530098|6110|30169|4|15|5060|39|686.40|54.91|741.31|62.35|196.17|652.08|1.02|33.30|313.43| +2452560|48719|8669|17438|730371|4104|23437|17438|730371|4104|23437|5|31|5061|49|5135.69|359.49|5495.18|29.10|1486.66|1643.42|803.22|2689.05|1875.25| +2451797|49881|7993|91181|865740|2699|40411|91181|865740|2699|40411|58|31|5063|4|76.56|0.00|76.56|9.34|12.44|76.56|0.00|0.00|21.78| +2451570|38571|4741|58743|1692626|2156|29639|58743|1692626|2156|29639|7|31|5064|7|182.14|3.64|185.78|53.82|70.56|173.03|2.27|6.84|128.02| +2451456|74689|16204|88029|616057|6038|32419|88029|616057|6038|32419|1|16|5064|77|2255.33|157.87|2413.20|5.82|48.51|1420.85|784.41|50.07|212.20| +2452150|72295|1247|67451|1076818|5738|49458|67451|1076818|5738|49458|31|2|5065|25|294.00|14.70|308.70|8.00|1581.25|64.68|165.11|64.21|1603.95| +2451070|72203|6961|9839|1331600|5808|34016|9839|1331600|5808|34016|58|25|5068|48|554.40|16.63|571.03|46.89|2311.68|227.30|140.65|186.45|2375.20| +2451004|43692|17731|16935|1703537|539|6341|16935|1703537|539|6341|49|23|5068|8|178.96|0.00|178.96|10.62|4.96|0.00|127.06|51.90|15.58| +||17798||1347370|||57200|||35357|||5069|51|1342.32|67.11|1409.43|53.33|778.77|926.20|128.99||| +2451569|46128|5480|53127|829751|328|1752|53127|829751|328|1752|38|15|5069|28|5340.44|213.61|5554.05|44.97|2756.32|2830.43|2359.40|150.61|3014.90| +2452271|13112|5113|83502|1846278|1501|48677|83502|1846278|1501|48677|7|29|5071|11|761.20|68.50|829.70|72.26|867.46|274.03|472.55|14.62|1008.22| +2452268|11699|6229|52999|1782622|2198|44221|52999|1782622|2198|44221|15|13|5071|12|473.76|4.73|478.49|34.40|243.60|213.19|244.93|15.64|282.73| +2452267|45472|7985|73137|758964|4791|18218|73137|758964|4791|18218|59|11|5071|16|127.68|10.21|137.89|48.48|510.72|99.59|13.48|14.61|569.41| +2451090|27388|6355|11059|636019|6226|38386|11059|636019|6226|38386|1|20|5072|2|42.38|1.27|43.65|18.65|54.08|2.11|36.64|3.63|74.00| +2451093|72539|4933|11007|1710088|3216|15097|11007|1710088|3216|15097|13|9|5072|12|34.44|0.00|34.44|38.79|0.00|29.96|2.77|1.71|38.79| +2450978|42626|3151|4564|1828909|5607|9952||1828909||9952||3|5072|33|1471.14||1559.40||43.23|||123.58|168.88| +2452369|20246|9099|90412|904806|1589|21380|90412|904806|1589|21380|11|9|5074|7|376.25|15.05|391.30|56.67|250.81|169.31|130.37|76.57|322.53| +2452113|74799|6787|25295|757410|4725|16440|25295|757410|4725|16440|55|12|5075|3|11.07|0.11|11.18|85.26|8.10|2.10|1.34|7.63|93.47| +2452135|37959|13862|86031|1281350|6269|388|86031|1281350|6269|388|44|19|5075|8|38.24|0.76|39.00|69.20|16.00|4.97|9.31|23.96|85.96| +2452651|23669|16641|40844|131281|5365|41002|40844|131281|5365|41002|39|29|5076|13|685.36|54.82|740.18|23.12|116.35|178.19|55.78|451.39|194.29| +2452246|19302|15409|49438|911738|6838|35138|49438|911738|6838|35138|13|1|5077|2|171.08|8.55|179.63|40.88|84.68|59.87|12.23|98.98|134.11| +2452163|5628|14579|79797|38032|3820|15608|79797|38032|3820|15608|11|5|5077|7|120.12|0.00|120.12|44.90|12.18|46.84|17.58|55.70|57.08| +2451977|21990|7387|31523|1253284|1892|35253|31523|1253284|1892|35253|55|20|5077|36|1166.76|11.66|1178.42|57.90|2486.16|478.37|681.50|6.89|2555.72| +2452217|64410|13407|73811|1818759|154|21502|73811|1818759|154|21502|19|21|5079|50|931.00|37.24|968.24|39.17|1325.00|195.51|551.61|183.88|1401.41| +2452205|33196|5637|76435|1902233|5375|31989|76435|1902233|5375|31989|31|19|5079|5|65.95|5.27|71.22|82.61|102.90|32.31|31.28|2.36|190.78| +2451303|64539|1192|40018|510098|178|34258|40018|510098|178|34258|20|9|5080|31|131.44|2.62|134.06|35.55|965.34|67.03|49.59|14.82|1003.51| +2452647|11179|5094|39012|135988|5411|25987|39012|135988|5411|25987|30|19|5081|60|2336.40|70.09|2406.49|7.00|212.40|747.64|730.82|857.94|289.49| +2451005|72367|1928|74837|230999|314|40476|74837|230999|314|40476|19|14|5082|3|106.92|6.41|113.33|28.25|69.18|11.76|58.04|37.12|103.84| +2451018|34286|14678|96825|395286|6547|3808|96825|395286|6547|3808|34|12|5082|2|55.30|0.00|55.30|37.90|82.96|22.67|28.38|4.25|120.86| +2450975|66259|3091|25537|1851785|4880|2775|25537|1851785|4880|2775|13|33|5082|5|404.20|20.21|424.41|1.60|180.80|351.65|37.83|14.72|202.61| +2451573|56249|12794|37062|904294|2373|18052|37062|904294|2373|18052|10|27|5083|24|258.00|15.48|273.48|34.23|100.32|87.72|134.52|35.76|150.03| +2452319|75015|12045|83222|219912|955|16445|83222|219912|955|16445|3|1|5084|6|192.72|0.00|192.72|83.40|436.80|53.96|13.87|124.89|520.20| +2452045|82536|13473|69289|768819|1949|2726|69289|768819|1949|2726|59|33|5085|33|1946.34|175.17|2121.51|93.06|950.07|622.82|211.76|1111.76|1218.30| +2452482|53987|3361|60319|1860497|2628|33606|60319|1860497|2628|33606|7|23|5086|37|133.94|5.35|139.29|96.59|871.72|103.13|0.92|29.89|973.66| +2452442|42515|73|35173|932991|5656|11074|35173|932991|5656|11074|19|30|5086|3|165.36|4.96|170.32|23.63|95.91|137.24|13.77|14.35|124.50| +||16383||1537273|6428|15536||1537273|6428|15536|||5088|||3.15||48.42||||21.77|173.01| +2451963|71672|17900|44907|1398818|1176|30287|44907|1398818|1176|30287|23|19|5090|11|347.82|10.43|358.25|57.20|141.46|135.64|186.71|25.47|209.09| +2451956|51699|7139|81261|1468023|7138|6700|81261|1468023|7138|6700|29|21|5090|5|69.20|3.46|72.66|16.15|82.70|15.91|43.69|9.60|102.31| +2452841|14327|16575|61971|538362|7200|47191|61971|538362|7200|47191|30|14|5092|23|1354.01|121.86|1475.87|41.69|119.83|974.88|136.48|242.65|283.38| +2452937|58525|9336|20837|1196065|3373|1803|20837|1196065|3373|1803|12|4|5093|5|740.25|7.40|747.65|36.09|79.30|429.34|245.61|65.30|122.79| +2452834|81404|9871|19612|1799569|3079|47436|19612|1799569|3079|47436|55|7|5093|11|257.18|7.71|264.89|98.71|210.43|249.46|4.63|3.09|316.85| +2451430|57117|6538|3427|302288|5149|10|3427|302288|5149|10|58|34|5094|32|1498.56|134.87|1633.43|62.90|759.68|809.22|585.93|103.41|957.45| +2451298|75798|12278|85124|1782751|1531|16358|85124|1782751|1531|16358|37|4|5094|3|61.68|3.08|64.76|40.50|39.84|48.72|10.36|2.60|83.42| +2451649|40399|11194|34580|1529821|6662|17903|34580|1529821|6662|17903|58|35|5095|77|1667.05|150.03|1817.08|19.77|1982.75|1300.29|139.36|227.40|2152.55| +2451775|63855|7234|88093||3644|33196||370445||33196|||5095|4|341.40|17.07|358.47|67.03|216.48|310.67|||| +2452706|65652|11295|22731|1872460|589|49518|22731|1872460|589|49518|13|29|5096|14|373.24|33.59|406.83|10.75|462.70|339.64|6.38|27.22|507.04| +2452703|77471|6138|14046|264397|3548|29499|14046|264397|3548|29499|51|28|5097|9|855.27|17.10|872.37|72.86|967.77|59.86|262.48|532.93|1057.73| +2451795|29358|3128|67261|351094|3727|26760|67261|351094|3727|26760|44|11|5098|6|93.48|8.41|101.89|26.42|65.94|60.76|0.65|32.07|100.77| +2451706|15728|6997|27575|1716000|2825|17650|27575|1716000|2825|17650|7|33|5098|5|631.70|31.58|663.28|90.85|223.70|511.67|114.02|6.01|346.13| +2452683|62351|3727|38079|127297|962|16980|38079|127297|962|16980|36|24|5099|15|1670.40|100.22|1770.62|57.85|801.75|16.70|926.07|727.63|959.82| +||11868|||3727||5827|1070377||33877|24|32|5099|||195.86|2372.12||641.70||1401.51||886.26| +2452311|37778|15845|38195|1095619|1490|28514|38195|1095619|1490|28514|13|16|5100|18|1736.10|138.88|1874.98|98.10|791.46|677.07|52.95|1006.08|1028.44| +2452277|70558|8431|85882|39337|6425|13974|85882|39337|6425|13974|57|26|5101|24|62.40|3.12|65.52|3.10|18.00|9.36|32.35|20.69|24.22| +2452268|74617|16491|21535|309825|5315|2005|21535|309825|5315|2005|53|26|5101|32|6472.00|64.72|6536.72|62.24|3635.84|1876.88|4503.21|91.91|3762.80| +||13319|||||72431|||43265|||5102||37.76||||||1.95|2.21|| +2451443|69169|10694|80807|1318862|917|21772|80807|1318862|917|21772|22|1|5103|88|9612.24|0.00|9612.24|88.85|5455.12|2883.67|5113.71|1614.86|5543.97| +2451480|30247|910|2078|632770|3462|16107|2078|632770|3462|16107|26|30|5103|27|2303.91|0.00|2303.91|10.03|218.16|1704.89|539.11|59.91|228.19| +2451436|28003|17350|59462|59871|2680|27442|59462|59871|2680|27442|8|9|5103|42|233.10|16.31|249.41|60.70|1032.36|41.95|17.20|173.95|1109.37| +2452291|28451|3657|64932|1395533|4181|23229|64932|1395533|4181|23229|7|25|5106|1|61.39|4.29|65.68|96.76|18.54|56.47|2.21|2.71|119.59| +2452379|79077|43|77425|1748568|4958|28901|77425|1748568|4958|28901|7|23|5106|41|2370.21|94.80|2465.01|2.96|1221.80|1090.29|755.15|524.77|1319.56| +2451934||1507|29119|704291|5863|26588|29119|704291|5863|26588|19|8|5107|1|193.70||211.13|18.00|72.64|||18.77|108.07| +2451991|39701|5815|91921|1749318|3124|24615|91921|1749318|3124|24615|41|14|5107|25|169.25|6.77|176.02|4.96|103.50|79.54|28.70|61.01|115.23| +2452074|71479|5719|82711|1362937|7163|32989|82711|1362937|7163|32989|37|21|5107|67|2564.09|205.12|2769.21|9.01|1520.90|717.94|166.15|1680.00|1735.03| +2451993|28521|8015|24511|1416416|6721|8226|24511|1416416|6721|8226|55|25|5108|80|857.60|25.72|883.32|59.31|388.00|68.60|173.58|615.42|473.03| +2451996|17059|12385|12749|945492|3035|34978|12749|945492|3035|34978|50|6|5108|40|749.20|22.47|771.67|69.90|576.00|142.34|24.27|582.59|668.37| +2452044|73735|17435|47647|834833|772|6758|47647|834833|772|6758|43|7|5108|3|25.65|2.30|27.95|62.87|6.48|2.05|22.65|0.95|71.65| +2452808|74798|15192|535|270013|6691|14969|535|270013|6691|14969|25|19|5109|21|3390.87|101.72|3492.59|46.19|629.16|542.53|626.63|2221.71|777.07| +2451914||10175||||||879993|6654||56|29|5110||||1081.85||||||| +2451867|54389|17101|4852|1562264|6429|7841|4852|1562264|6429|7841|17|6|5110|2|132.22|1.32|133.54|26.32|27.54|59.49|51.63|21.10|55.18| +2451864|37570|14974|80674|1858657|1440|13023|80674|1858657|1440|13023|46|18|5111|56|21.84|0.65|22.49|35.22|34.16|10.48|9.31|2.05|70.03| +2451700|27939|10171|96640|261582|6524|33466|96640|261582|6524|33466|43|14|5111|3|3.90|0.27|4.17|59.45|10.11|3.86|0.02|0.02|69.83| +2451415|69135|8914|61510|222918|1926|15334|61510|222918|1926|15334|56|3|5112|42|257.88|2.57|260.45|89.49|141.54|30.94|199.70|27.24|233.60| +2451433|44585|9121|8466|601270|6427|45231|8466|601270|6427|45231|20|14|5112|1|80.75|7.26|88.01|62.85|26.91|29.07|16.02|35.66|97.02| +2452725|8196|4189|5436|130065|4294|37723|5436|130065|4294|37723|27|17|5114|8|283.76|25.53|309.29|81.64|5.20|130.52|99.60|53.64|112.37| +2452744|70951|9660|43390|1176685|5180|27675|43390|1176685|5180|27675|39|30|5114|10|55.60|5.00|60.60|58.60|519.20|2.22|41.63|11.75|582.80| +2451608|43128|11660|49764|1802743|1769|21548|49764|1802743|1769|21548|4|23|5115|14|2046.94|102.34|2149.28|92.36|1049.72|1187.22|60.18|799.54|1244.42| +2451427|78567|17797|50629|664318|6807|45296|50629|664318|6807|45296|52|25|5118|5|617.85|0.00|617.85|18.66|137.30|438.67|26.87|152.31|155.96| +2451522|64751|7798|72710|816548|33|11071|72710|816548|33|11071|25|4|5118|64|3130.24|0.00|3130.24|84.62|1213.44|2316.37|170.91|642.96|1298.06| +2451337|63724|15956|53301|1121258|2368|38725|53301|1121258|2368|38725|44|11|5118|6|151.68|13.65|165.33|34.29|59.22|133.47|1.63|16.58|107.16| +2451900|58067|16448|32519|216008|4744|37033|32519|216008|4744|37033|50|29|5120|4|4.00|0.36|4.36|91.14|92.44|3.28|0.33|0.39|183.94| +2452082|70079|7103|96320|553653|5386|4366|96320|553653|5386|4366|50|13|5120|47|387.28|23.23|410.51|2.16|1550.06|50.34|26.95|309.99|1575.45| +2452081|18323|8653|71162|861892|5775|28058|71162|861892|5775|28058|37|31|5120|64|24.32|1.70|26.02|45.23|223.36|15.07|8.60|0.65|270.29| +2451840|31199|10121|20153|1602667|1534|3648|20153|1602667|1534|3648|20|30|5121|96|5143.68|51.43|5195.11|26.33|1503.36|720.11|1636.72|2786.85|1581.12| +2451207|37213|13837|74884|733037|3190|28543|74884|733037|3190|28543|25|16|5122|67|1072.67|42.90|1115.57|30.77|5543.58|42.90|988.57|41.20|5617.25| +2451170|73955|17191|62134|1045110|1686|46662|62134|1045110|1686|46662|49|3|5122|46|1226.82|73.60|1300.42|58.67|17.94|687.01|264.50|275.31|150.21| +2452323|46403|2913|40833|1023981|5882|13030|40833|1023981|5882|13030|1|5|5126|24|806.64|64.53|871.17|23.85|17.28|282.32|325.07|199.25|105.66| +2450883|70079|13942|41246|1040001|5956|46264|41246|1040001|5956|46264|28|5|5127|32|3246.72|194.80|3441.52|69.09|2291.84|2337.63|372.72|536.37|2555.73| +2451595|57909|14954|38861|523284|6107|8130|38861|523284|6107|8130|14|30|5128|5|453.40|36.27|489.67|14.36|181.35|172.29|47.78|233.33|231.98| +2451355|84646|11938|20677|426187|4850|19160|20677|426187|4850|19160|14|32|5129|2|76.78|4.60|81.38|46.86|30.36|75.24|1.06|0.48|81.82| +2451208|18660|7789|88201|533713|2872|10384|88201|533713|2872|10384|32|20|5129|23|2634.42|131.72|2766.14|45.03|446.43|1923.12|384.10|327.20|623.18| +2452285|72767|1305|88618|1264643|1821|41846|88618|1264643|1821|41846|23|32|5131|46|1012.92|40.51|1053.43|90.61|559.82|921.75|61.99|29.18|690.94| +2452289|24484|16647|28128|191742|3003|14617|28128|191742|3003|14617|41|35|5131|27|657.18|46.00|703.18|66.07|1544.67|545.45|53.63|58.10|1656.74| +2451824|65969|2641|66593|1027839|732|34778|66593|1027839|732|34778|23|35|5134|15|556.35|27.81|584.16|2.97|241.65|450.64|24.31|81.40|272.43| +2452014|12996|16268|69163|1667981|3246|29711|69163|1667981|3246|29711|20|32|5139|11|1224.19|0.00|1224.19|18.19|602.91|526.40|341.91|355.88|621.10| +2451778|70781|6785|39268|582243|6465|11087|39268|582243|6465|11087|50|20|5139|1|93.33|1.86|95.19|17.44|18.66|27.99|28.74|36.60|37.96| +2452441|24794|12653|95443|525418|4352|25076|95443|525418|4352|25076|15|31|5140|17|1859.46|111.56|1971.02|55.32|732.53|911.13|682.79|265.54|899.41| +2451419|34825|6962|54055|441851|1200|22949|54055|441851|1200|22949|52|35|5141|78|2331.42|116.57|2447.99|29.83|621.66|1585.36|440.17|305.89|768.06| +|58278|9572|29822||||29822||5755||1||5142||66.64||71.97|73.61|122.24|20.65|||201.18| +2452280|67077|4463|12579|1341244|3509|31374|12579|1341244|3509|31374|43|12|5143|12|36.96|0.36|37.32|42.14|6.00|9.60|23.52|3.84|48.50| +2451730|80692|14035|83969|354813|2457|44054|83969|354813|2457|44054|17|2|5145|42|2430.96|0.00|2430.96|80.52|767.34|583.43|277.12|1570.41|847.86| +2451753|10070|14399|71768|1340183|3445|27860|71768|1340183|3445|27860|19|31|5145|6|87.96|6.15|94.11|43.06|63.66|36.94|20.40|30.62|112.87| +2452057|64099|3245|22044|1049710|5729|41395|22044|1049710|5729|41395|14|19|5146|1|60.57|1.81|62.38|42.26|35.74|58.75|1.82|0.00|79.81| +2452103|63290|7171|23663|955340|6217|9894|23663|955340|6217|9894|44|24|5146|40|418.00|4.18|422.18|2.04|480.40|401.28|0.16|16.56|486.62| +||8737|52712||2413|46687|||||41|32|5146||341.64|3.41||51.78||283.56|35.42|22.66|| +2452876|24856|789|57810|1416532|3607|15129|57810|1416532|3607|15129|30|11|5147|13|1073.02|42.92|1115.94|88.60|559.78|53.65|723.75|295.62|691.30| +2452138|16760|10767|12389|842375|5161|29994|12389|842375|5161|29994|31|24|5148|10|224.80|8.99|233.79|8.41|206.00|155.11|48.08|21.61|223.40| +2452101||5569||||||940244||26241|45|22|5148|16|898.72|||7.63|390.72|||476.42|| +2452064|13907|6075|21547|535713|6679|4845|21547|535713|6679|4845|7|24|5148|39|4583.28|412.49|4995.77|14.67|723.45|3804.12|70.12|709.04|1150.61| +2452160|11230|14351|4021|1020850|2479|3708|4021|1020850|2479|3708|41|10|5149|19|1105.99|44.23|1150.22|2.83|915.23|1072.81|10.28|22.90|962.29| +2452421|15925|8157|50713|1636081|7049|29617|50713|1636081|7049|29617|23|10|5150|19|137.75|4.13|141.88|65.06|76.19|48.21|86.85|2.69|145.38| +2452312|36739|3825|19113||138|||361802|||57|14|5150||0.00|0.00||93.15|596.19|0.00|0.00|0.00|| +2452314|15498|1489|58782|394535|1135|45735|58782|394535|1135|45735|5|28|5150|5|71.95|2.15|74.10|77.67|204.80|27.34|44.61|0.00|284.62| +2452286|65354|12261|91184|1451603|822|18041|91184|1451603|822|18041|55|30|5154|51|290.70|14.53|305.23|35.41|58.14|247.09|13.51|30.10|108.08| +2451943|63068|2249|6989|258380|1826|29607|6989|258380|1826|29607|8|1|5155|68|8462.60|423.13|8885.73|72.77|3552.32|7362.46|99.01|1001.13|4048.22| +2452019|75658|4903|43479|1101653|1427|19201|43479|1101653|1427|19201|7|28|5155|49|4385.50|350.84|4736.34|24.11|1783.60|3683.82|596.42|105.26|2158.55| +2452622|78029|16029|63535|19284|3386|39535|63535|19284|3386|39535|57|34|5156|83|6954.57|556.36|7510.93|66.73|2742.32|2712.28|3054.44|1187.85|3365.41| +||17041|52420|||||841557||43078||24|5156||202.50|10.12||34.11|231.00||||| +2451201|45004|17720|72507|246322|937|46305|72507|246322|937|46305|32|31|5158|33|41.58|0.00|41.58|14.16|67.65|11.64|29.04|0.90|81.81| +2450950|33894|16882|49311|463152|6146|33587|49311|463152|6146|33587|20|9|5159|93|19299.36|1157.96|20457.32|24.49|6502.56|385.98|4917.47|13995.91|7685.01| +2451482|60172|2732|8843|1786070|1697|22524|8843|1786070|1697|22524|16|8|5160|5|643.35|57.90|701.25|52.05|26.50|315.24|190.30|137.81|136.45| +2451683|65022|15580|24198|148331|911|16644|24198|148331|911|16644|56|4|5160|54|7730.64|77.30|7807.94|79.87|2995.38|4251.85|3478.79|0.00|3152.55| +2452677|77289|13596|36677|1895153|829|39305|36677|1895153|829|39305|60|16|5162|55|8390.25|503.41|8893.66|25.74|372.90|6460.49|1042.07|887.69|902.05| +2452283|72237|7255|31042|1310663|5931|27552|31042|1310663|5931|27552|43|35|5163|14|0.00|0.00|0.00|69.73|116.06|0.00|0.00|0.00|185.79| +2452272|71024|10301|3059|1644715|6724|16380|3059|1644715|6724|16380|33|13|5163|5|67.40|2.02|69.42|95.08|16.40|2.02|42.49|22.89|113.50| +2452418|15031|11477|41790|1691712|4387|13909|41790|1691712|4387|13909|31|15|5165|94|20843.56|1042.17|21885.73|62.52|5210.42|1459.04|13569.16|5815.36|6315.11| +2452105|43550|649|89347|1514626|1505|39231|89347|1514626|1505|39231|59|35|5167|9|979.29|29.37|1008.66|82.03|776.07|19.58|105.56|854.15|887.47| +2451525|16567|11404|1102|156877|2605|49180|1102|156877|2605|49180|38|19|5168|12|55.08|4.40|59.48|45.67|319.68|2.20|43.89|8.99|369.75| +2451024|79585|5932|24216|1110270|3524|42778|24216|1110270|3524|42778|49|19|5169|57|629.85|31.49|661.34|61.33|558.03|88.17|227.50|314.18|650.85| +2451026|27302|16411|91843|464731|6866|1354|91843|464731|6866|1354|1|19|5169|52|3474.12|138.96|3613.08|46.82|679.64|2570.84|478.73|424.55|865.42| +2452604|82434|699|45611|1723062|5666|34717|45611|1723062|5666|34717|51|16|5170|13|394.55|7.89|402.44|82.36|942.50|217.00|37.28|140.27|1032.75| +2452587||12489||956198|6255||||||37||5170|||||34.80|146.20|409.24|10.46||| +2452080|66082|17825|7312|1124005|3898|40191|7312|1124005|3898|40191|7|26|5171|3|53.94|4.31|58.25|46.53|74.85|25.35|16.58|12.01|125.69| +2451903|62070|10520|21869|902099|1176|16253|21869|902099|1176|16253|38|3|5171|7|247.31|19.78|267.09|54.15|366.38|49.46|21.76|176.09|440.31| +2451675|67969|2942|59370|699398|1379|44560|59370|699398|1379|44560|46|5|5172|9|666.90|40.01|706.91|89.43|374.58|160.05|111.50|395.35|504.02| +2451392|47743|9625|35591|1394612|1280|30701|35591|1394612|1280|30701|58|11|5173|83|6177.69|370.66|6548.35|82.49|6775.29|4200.82|1739.64|237.23|7228.44| +2452224|68479|3075|18334|736785|2083|14534|18334|736785|2083|14534|39|4|5174|3|22.44|2.01|24.45|5.46|10.08|10.09|3.82|8.53|17.55| +2452152|69486|299|27735|1245395|4655|36315|27735|1245395|4655|36315|55|4|5174|59|1793.01|53.79|1846.80|79.73|2903.39|1685.42|95.75|11.84|3036.91| +2451749|77485|7256|71005|1474978|2951|8477|71005|1474978|2951|8477|8|31|5175|27|3303.99|0.00|3303.99|80.71|2221.56|1156.39|1395.94|751.66|2302.27| +2451777|84685|15833|91730|1856030|7177|1877|91730|1856030|7177|1877|35|32|5176|15|1399.80|125.98|1525.78|23.43|1615.20|489.93|263.86|646.01|1764.61| +2451257|33375|15640|3670||1638|47296||997010||47296|28||5177|27|881.55|26.44|907.99||20.79||||54.29| +2451303|42152|15226|28290|62018|4546|15474|28290|62018|4546|15474|52|21|5177|33|4069.89|244.19|4314.08|99.80|1302.18|1546.55|883.16|1640.18|1646.17| +2451750|65247|14582|30377|1319298|5819|42387|30377|1319298|5819|42387|50|7|5178|22|3499.32|244.95|3744.27|66.53|1775.62|3219.37|229.55|50.40|2087.10| +2451723|68322|8843|50477|352566|4255|24344|50477|352566|4255|24344|25|28|5178|6|272.46|19.07|291.53|26.34|174.36|5.44|85.44|181.58|219.77| +2451742|24460|7495||1855951|6155||||6155|||20|5178|||44.16|4461.10||1899.26|309.18|1437.71||1990.87| +2452500|20825|9949|53712|25206|5500|1794|53712|25206|5500|1794|39|13|5181|62|432.14|8.64|440.78|7.51|1441.50|220.39|23.29|188.46|1457.65| +2452522|67630|6527|83676|1187716|5064|26179|83676|1187716|5064|26179|21|24|5182|13|13.13|0.52|13.65|67.22|111.80|9.19|1.30|2.64|179.54| +2451416|86074|8773|69351|1608509|4467|472|69351|1608509|4467|472|44|7|5183|13|1105.78|99.52|1205.30|91.84|127.53|11.05|1061.88|32.85|318.89| +2451263|55719|15130||804882||22482|92635|804882||22482|14|26|5184|15|496.20||540.85|84.46||401.92|42.42|51.86|| +||16402|||1001|19246|33311|273956|||26||5184||12.18|0.97|13.15|24.61|3.19|5.11|||28.77| +2452403|72465|12871|82477|1910086|2002|49822|82477|1910086|2002|49822|19|24|5185|13|223.08|11.15|234.23|14.97|23.40|180.69|16.53|25.86|49.52| +2452485|59960|3912|30170|718536|3702|3315|30170|718536|3702|3315|48|23|5185|34|395.08|7.90|402.98|76.76|141.78|19.75|108.84|266.49|226.44| +2452791|41164|14577|82935|1527103|6824|48945|82935|1527103|6824|48945|45|32|5187|13|681.85|20.45|702.30|87.86|664.30|197.73|459.91|24.21|772.61| +2452661|33172|876|71123|1733887|4651|19815|71123|1733887|4651|19815|60|12|5187|28|283.92|22.71|306.63|12.20|203.56|150.47|86.74|46.71|238.47| +2451165|47052|9091|4931|1414463|4795|49127|4931|1414463|4795|49127|43|6|5188|5|37.15|2.97|40.12|39.26|19.40|34.92|0.02|2.21|61.63| +2451681|27951|2749|34996|1130377|4511|20692|34996|1130377|4511|20692|13|26|5189|8|129.68|0.00|129.68|8.88|60.48|101.15|4.56|23.97|69.36| +2451664|4522|13099|31404|1013610|1863|40916|31404|1013610|1863|40916|25|35|5189|16|62.40|4.99|67.39|74.10|39.04|45.55|3.70|13.15|118.13| +2451686|69861|9656|8445|1713821|7197|32173|8445|1713821|7197|32173|4|11|5189|51|12658.20|506.32|13164.52|2.90|6197.01|2405.05|7074.67|3178.48|6706.23| +2451674|47181|13906|48876|45448|1573|11745|48876|45448|1573|11745|38|11|5189|12|800.76|32.03|832.79|7.12|9.72|64.06|486.22|250.48|48.87| +2451153|34687|11293|79442|1074242|4242|18595|79442|1074242|4242|18595|28|1|5191|10|65.50|3.27|68.77|59.18|192.00|17.68|3.34|44.48|254.45| +2451181|11092|14614|55732|192805|1878|13398|55732|192805|1878|13398|10|15|5191|5|13.25|0.39|13.64|63.70|4.40|6.89|2.60|3.76|68.49| +2451129|62180|7154|1803|1252932|1214|29896|1803|1252932|1214|29896|1|18|5191|7|772.59|0.00|772.59|32.12|17.36|726.23|25.03|21.33|49.48| +2451214|33370|919|56280|1521803|1373|12496|56280|1521803|1373|12496|49|2|5191|2|48.68|0.97|49.65|95.36|87.14|9.24|14.98|24.46|183.47| +2451280|28124|13630|58377|100609|2394|36626|58377|100609|2394|36626|13|12|5191|5|152.60|7.63|160.23|72.56|102.45|28.99|97.65|25.96|182.64| +2452394|35770|2617|18338|775298|2305|17265|18338|775298|2305|17265|31|9|5192|30|681.90|47.73|729.63|78.10|1120.50|286.39|288.72|106.79|1246.33| +||9484|59053|1413050|1447||||1447||||5193||302.40|||19.90|97.55|90.72|||| +2451656||1628|||5785||69836||5785|33500||34|5193||||5955.38|21.81||1057.49|3245.96||411.41| +2451669|31894|10700|82921|1274982|6596|16657|82921|1274982|6596|16657|43|21|5193|61|558.15|27.90|586.05|78.66|148.84|106.04|370.73|81.38|255.40| +2451692|73121|6556|61865|237539|6013|12134|61865|237539|6013|12134|31|22|5193|50|3053.00|91.59|3144.59|34.79|515.50|1892.86|464.05|696.09|641.88| +||13537|80015|1360889||||1360889|5316|||18|5195||1139.04|||||||8.21|| +2452664|23580|11113|92651|577330|3256|49935|92651|577330|3256|49935|9|2|5195|23|3093.04|30.93|3123.97|3.87|1200.14|1206.28|1018.85|867.91|1234.94| +2451143|63671|250|19512|1322632|5305|1978|19512|1322632|5305|1978|34|11|5197|1|72.22|2.16|74.38|38.52|4.15|11.55|23.05|37.62|44.83| +2451067|31872|9067|62247|1916376|3305|42634|62247|1916376|3305|42634|8|23|5197|9|509.67|45.87|555.54|78.95|166.95|346.57|53.82|109.28|291.77| +2452759|75054|9267|64076|109591|5059|24127|64076|109591|5059|24127|25|27|5198|68|11098.96|554.94|11653.90|45.78|4917.08|776.92|103.22|10218.82|5517.80| +2451929|63606|17623|58148|1789747|2499|34898|58148|1789747|2499|34898|8|28|5199|3|60.45|0.00|60.45|83.30|181.38|13.29|14.61|32.55|264.68| +2451987|9714|3961|71661|529363|218|25520|71661|529363|218|25520|2|28|5199|8|272.24|2.72|274.96|34.89|176.16|0.00|182.40|89.84|213.77| +2452408|51979|12571|47952|402362|1925|735|47952|402362|1925|735|5|29|5201|7|37.73|2.64|40.37|46.64|117.04|25.65|1.32|10.76|166.32| +2452287|31847|5113|71635|1581772|7109|22799|71635|1581772|7109|22799|45|7|5201|42|640.92|32.04|672.96|15.45|34.02|115.36|283.80|241.76|81.51| +2451386|22487|601|6168|625119|4070|35421|6168|625119|4070|35421|50|29|5202|28|1201.48|0.00|1201.48|12.59|680.40|216.26|315.27|669.95|692.99| +2451367|43703|13231|96266|1344850|1081|29265|96266|1344850|1081|29265|37|10|5202|29|168.20|10.09|178.29|45.48|74.82|6.72|37.14|124.34|130.39| +2452635|6608|17335|50235|148994|1706|8018|50235|148994|1706|8018|60|1|5205|11|1210.44|12.10|1222.54|83.04|165.55|907.83|45.39|257.22|260.69| +2452766|10314|8299|32320|593400|5327|41207|32320|593400|5327|41207|42|18|5206|14|91.14|4.55|95.69|10.93|486.50|16.40|56.05|18.69|501.98| +2452683|47452|16221|21761|60126|3291|31256|21761|60126|3291|31256|3|5|5206|77|6200.04|124.00|6324.04|37.78|332.64|3658.02|1194.74|1347.28|494.42| +2452049|37116|6389|10227|567817|4254|29444|10227|567817|4254|29444|5|20|5207|42|1906.38|133.44|2039.82|49.86|393.54|305.02|1441.22|160.14|576.84| +2451548|71272|15610|70763|441230|4497|41011|70763|441230|4497|41011|16|3|5208|37|4445.55|133.36|4578.91|16.07|3449.14|4401.09|30.67|13.79|3598.57| +2451430|28074|10732|23785|800955|6925|28095|23785|800955|6925|28095|40|15|5209|14|806.82|56.47|863.29|39.10|968.24|508.29|146.27|152.26|1063.81| +2452188|82026|10729|66932|632134|2538|6807|66932|632134|2538|6807|1|31|5210|9|365.58|25.59|391.17|29.92|82.17|204.72|112.60|48.26|137.68| +2452098|58258|1985|50230|809127|24|13721|50230|809127|24|13721|55|23|5210|18|169.56|11.86|181.42|73.27|143.46|110.21|40.95|18.40|228.59| +2452267|47603|12149|1344|1808040|6289|12585|1344|1808040|6289|12585|45|23|5210|61|1127.89|0.00|1127.89|24.00|705.16|124.06|401.53|602.30|729.16| +2451112|72145|4558|54915|256937|2606|32628|54915|256937|2606|32628|10|33|5211|15|251.85|15.11|266.96|77.14|187.80|110.81|86.03|55.01|280.05| +2451934|39441|8587|79395|1282731|1267|36203|79395|1282731|1267|36203|2|11|5212|12|900.96|72.07|973.03|57.22|1453.32|423.45|444.08|33.43|1582.61| +2452099|33273|4057|||||66276|474265|4654||||5216|||0.00|||||||223.53| +2451738|25349|8305|56924|1081695|1313|19284|56924|1081695|1313|19284|56|17|5218|33|285.12|2.85|287.97|11.91|35.64|122.60|17.87|144.65|50.40| +2451638|75998|15290|94301|1045524|1547|17217|94301|1045524|1547|17217|19|32|5218|43|4444.91|311.14|4756.05|99.75|3809.80|355.59|2003.76|2085.56|4220.69| +2451687|33843|5612|46430|652671|6916|23399|46430|652671|6916|23399|32|6|5218|24|195.84|9.79|205.63|83.45|527.52|45.04|22.62|128.18|620.76| +2451229|58627|11786|43833|475947|3261|37775|43833|475947|3261|37775|2|33|5219|4|14.36|1.14|15.50|82.58|37.12|11.48|1.90|0.98|120.84| +2452479|30936|12681|95430|1099024|6971|42565|95430|1099024|6971|42565|36|12|5220|26|1292.72|25.85|1318.57|28.31|396.50|1266.86|19.65|6.21|450.66| +2452412|8374|2613|42790|1881691|3439|49187|42790|1881691|3439|49187|19|31|5220|6|923.76|18.47|942.23|0.71|499.86|526.54|107.24|289.98|519.04| +2452603|69516|4005|28083|19683|4436|29236|28083|19683|4436|29236|51|20|5220|1|23.09|0.92|24.01|54.79|3.84|1.61|9.88|11.60|59.55| +2451820|38583|9101|89612|68140|4268|26843|89612|68140|4268|26843|37|15|5221|33|4723.29|188.93|4912.22|4.33|1898.82|472.32|3273.24|977.73|2092.08| +2451874|66956|3943|40205|249918|5231|16897|40205|249918|5231|16897|29|21|5221|33|3146.88|94.40|3241.28|85.06|3290.10|1227.28|1650.85|268.75|3469.56| +2452177|52789|7753|63403|1120566|3682|22445|63403|1120566|3682|22445|5|7|5224|3|277.35|5.54|282.89|60.03|72.33|94.29|95.19|87.87|137.90| +2452264|15963|10121|23460|1531923|1672|43536|23460|1531923|1672|43536|31|23|5224|54|5099.76|407.98|5507.74|99.01|2846.34|1121.94|1034.23|2943.59|3353.33| +2451720|33095|1075|44312|1855518|1924|28683|44312|1855518|1924|28683|29|25|5225|62|13350.46|934.53|14284.99|97.53|3000.18|12415.92|794.35|140.19|4032.24| +2451315|62900|1072|40838|1919925|1765|12005|40838|1919925|1765|12005|14|12|5226|8|193.76|1.93|195.69|61.04|0.00|110.44|79.98|3.34|62.97| +2451460|50032|16321|13253|1849920|4187|40054|13253|1849920|4187|40054|28|23|5226|46|84.64|1.69|86.33|92.31|42.32|2.53|21.34|60.77|136.32| +|42811|13497|||||||3480||17|26|5227|4|||68.47||160.68||28.35|26.17|196.06| +2452026|83905|9461|60580|1576538|3112|10388|60580|1576538|3112|10388|43|7|5227|26|1531.66|91.89|1623.55|29.51|930.02|413.54|1039.85|78.27|1051.42| +2451627|31614|5785|96028|1210766|2751|36289|96028|1210766|2751|36289|7|26|5228|40|1586.80|79.34|1666.14|39.81|80.80|126.94|1109.49|350.37|199.95| +2451687|59828|8125|11873|1591028|583|26185|11873|1591028|583|26185|8|19|5228|50|1021.00|61.26|1082.26|67.50|102.00|541.13|239.93|239.94|230.76| +2451796||11908|71245||588||71245|546893|588|13083|||5228|48|||3039.06||||||2924.87| +2452309||10815|54651|475907||43572|54651||||3|12|5230|17|428.40||||||11.56||| +2451390|55918|8545|27250|1621317|2137|2604|27250|1621317|2137|2604|14|30|5231|30|950.10|57.00|1007.10|16.41|1345.80|741.07|29.26|179.77|1419.21| +2451622|41922|9566|77616|840971|5479|16834|77616|840971|5479|16834|4|23|5233|13|1937.39|154.99|2092.38|20.55|787.67|1937.39|0.00|0.00|963.21| +2451468|71677|15487|3128|1110915|5455|16442|3128|1110915|5455|16442|46|33|5233|13|345.54|0.00|345.54|91.16|63.96|127.84|217.70|0.00|155.12| +2451793|72171|2455|69394|222229|3511|46746|69394|222229|3511|46746|10|32|5234|8|476.32|0.00|476.32|99.96|317.52|423.92|28.82|23.58|417.48| +2452387|56776|15137|56673|460747|2421|44142|56673|460747|2421|44142|47|6|5235|21|2612.82|130.64|2743.46|99.72|2146.20|2508.30|71.07|33.45|2376.56| +2452329|62218|8543|27147|481721|2695|33316|27147|481721|2695|33316|27|24|5235|16|150.72|13.56|164.28|16.15|0.00|36.17|49.25|65.30|29.71| +2452426|67267|1673|61215|643339|2263|26387|61215|643339|2263|26387|17|3|5236|1|70.51|2.11|72.62|61.95|51.95|52.17|9.72|8.62|116.01| +2452438|71777|16415|21335|667570|2690|5273|21335|667570|2690|5273|3|11|5236|79|4417.68|397.59|4815.27|15.83|422.65|2473.90|1457.83|485.95|836.07| +2452313|29484|17837|74788|1633144|3147|43503|74788|1633144|3147|43503|15|28|5237|11|1589.28|79.46|1668.74|62.42|279.40|1430.35|76.28|82.65|421.28| +2452762|66942|16056|86489|1652897|6505|20913|86489|1652897|6505|20913|37|31|5239|18|354.42|14.17|368.59|22.04|82.44|301.25|29.24|23.93|118.65| +2452723||6337|83684|940096|||83684||||||5239|||||||135.91||225.43|| +2452711|28287|14449|78917|1379995|626|104|78917|1379995|626|104|21|23|5239|19|145.92|0.00|145.92|3.65|44.84|81.71|30.17|34.04|48.49| +2451385|35412|8983|89518|1908370|4822|45627|89518|1908370|4822|45627|26|32|5240|10|157.40|0.00|157.40|46.50|629.70|119.62|25.31|12.47|676.20| +2452285|7577|17837|3923|1047351|3579|30207|3923|1047351|3579|30207|9|17|5241|1|27.07|1.35|28.42|39.88|13.98|7.30|3.75|16.02|55.21| +2452277|75411|3475|61014|1866416|1865|3267|61014|1866416|1865|3267|51|16|5241|7|14.49|1.30|15.79|27.40|10.43|11.01|3.48|0.00|39.13| +2452113|66897|3625|53174|1207303|2232|21114|53174|1207303|2232|21114|7|29|5243|21|955.50|0.00|955.50|38.18|1698.48|745.29|31.53|178.68|1736.66| +2451287|65795|5137|72941|1720270|2799|31728|72941|1720270|2799|31728|14|17|5244|10|263.40|2.63|266.03|66.78|865.60|44.77|43.72|174.91|935.01| +2451585|58637|12170|79861|1390420|1332|9091|79861|1390420|1332|9091|37|1|5245|55|1315.60|39.46|1355.06|24.02|1440.45|618.33|585.70|111.57|1503.93| +2451614|44150|7819|28096|1546562|6181|30247|28096|1546562|6181|30247|20|5|5246|76|3068.88|0.00|3068.88|28.47|2685.08|889.97|719.04|1459.87|2713.55| +||5180||1326177||||1326177|5941||52||5246|||||18.22||545.94||24.84|678.75| +2451889|16665|5936|48722|1481569|5969|35216|48722|1481569|5969|35216|23|3|5247|32|150.72|12.05|162.77|57.83|100.48|119.06|27.54|4.12|170.36| +2451925|39293|17719|91830|1675492|7154|34063|91830|1675492|7154|34063|35|4|5247|6|616.92|6.16|623.08|32.97|19.08|573.73|12.52|30.67|58.21| +2452355|72009|11737|56029|1334667|505|15943|56029|1334667|505|15943|57|18|5248|55|2191.75|21.91|2213.66|83.07|770.00|526.02|899.49|766.24|874.98| +2452150|56747|8793|1291|408818|1945|49472|1291|408818|1945|49472|13|7|5248|67|9063.09|90.63|9153.72|95.22|6591.46|7159.84|171.29|1731.96|6777.31| +2452470|65744|3039|87943|60157|3571|4585|87943|60157|3571|4585|45|12|5248|19|1761.87|105.71|1867.58|42.39|383.04|158.56|817.68|785.63|531.14| +2452391|78123|6169|82899|1772641|6736|20031|82899|1772641|6736|20031|3|3|5248|34|262.48|2.62|265.10|99.10|262.48|115.49|101.42|45.57|364.20| +2452341|71852|11931|69712|449857|6802|48194|69712|449857|6802|48194|51|16|5248|17|974.78|77.98|1052.76|87.92|1502.97|97.47|587.79|289.52|1668.87| +2452202|78344|9517|20658|396354|1650|43699|20658|396354|1650|43699|1|13|5248|31|1303.55|39.10|1342.65|7.51|374.17|430.17|751.10|122.28|420.78| +2451599|32184|13304|83189|1866110|5933|22005|83189|1866110|5933|22005|1|18|5250|8|466.96|9.33|476.29|17.32|245.36|289.51|95.82|81.63|272.01| +2451474|67560|3559|53857|49203|4549|3048|53857|49203|4549|3048|55|20|5251|82|8900.28|534.01|9434.29|52.91|1644.10|6675.21|1090.28|1134.79|2231.02| +2451733|70239|667|23|115411|5999|49787|23|115411|5999|49787|38|23|5252|45|2362.05|188.96|2551.01|14.80|1494.45|165.34|1383.92|812.79|1698.21| +2451767|37814|3979|28827|1250989|3711|26963|28827|1250989|3711|26963|37|25|5252|14|517.30|41.38|558.68|95.99|382.76|367.28|10.50|139.52|520.13| +2452135|47375|12685|17873|574101|5529|48557|17873|574101|5529|48557|31|30|5255|22|2962.30|118.49|3080.79|22.83|1501.50|2547.57|29.03|385.70|1642.82| +2452072|10292|1051|98872|545320|1917|47195|98872|545320|1917|47195|50|23|5255|13|1592.24|47.76|1640.00|82.87|220.09|1019.03|51.58|521.63|350.72| +2451689|15463|8926|41911|1349117|6276|42282|41911|1349117|6276|42282|49|4|5256|4|337.64|16.88|354.52|39.62|187.56|6.75|277.94|52.95|244.06| +2451160||15499||105267||2383|66509|||2383|4||5257|||20.47||54.98||||90.29|992.91| +||8407||||26886|71687|1456871|1255||59|10|5258|1||0.00||58.99|0.06||0.00|0.00|59.05| +2452126|70010|4431|92002|147767|3885|25250|92002|147767|3885|25250|39|14|5259|62|4810.58|384.84|5195.42|52.87|1939.36|2116.65|1293.08|1400.85|2377.07| +||16837|63020|1594037|1854|30918|63020|1594037||30918|14|19|5260||7.30|0.29|7.59|46.57|8.65|7.30|0.00|0.00|55.51| +2451765|18099|12169|22992|917746|2900|9207|22992|917746|2900|9207|1|7|5261|59|1805.40|144.43|1949.83|46.76|709.18|541.62|1225.86|37.92|900.37| +2451248|35380|7612|12579|1613214|2296|27331|12579|1613214|2296|27331|26|10|5262|1|19.44|1.55|20.99|29.73|31.82|6.02|2.01|11.41|63.10| +2451349|6825|17893|58471|1912323|961|44924|58471|1912323|961|44924|25|6|5262|43|1192.39|95.39|1287.78|30.03|1222.06|357.71|217.01|617.67|1347.48| +2451211|48143|13196|40968|1846114|3613|2712|40968|1846114|3613|2712|14|7|5264|24|297.12|11.88|309.00|20.71|378.24|202.04|72.26|22.82|410.83| +2452415|32949|13461|86270|1206040|6610|39580|86270|1206040|6610|39580|3|24|5266|51|9300.36|465.01|9765.37|77.07|2492.88|2883.11|1411.79|5005.46|3034.96| +2451513|47409|410|41787|1806223|1059|36695|41787|1806223|1059|36695|58|24|5268|77|5869.71|528.27|6397.98|96.58|6684.37|3052.24|1155.16|1662.31|7309.22| +2451566|38035|3796|85934|1301998|2415|32479|85934|1301998|2415|32479|43|32|5268|5|93.00|0.00|93.00|62.48|39.50|56.73|36.27|0.00|101.98| +2451489|45248|17590|64905|621221|5890||64905|||18159|||5269||303.92|||78.35|52.49||||146.03| +2452052|37129|8543|5828|1167074|673|12719|5828|1167074|673|12719|20|19|5270|59|2092.14|125.52|2217.66|92.87|578.20|1129.75|452.32|510.07|796.59| +2451476|77278|2572|16307|1657192|1277|30209|16307|1657192|1277|30209|56|1|5271|4|16.16|0.16|16.32|93.60|64.64|11.31|2.32|2.53|158.40| +2452775|63395|11739|34493|664592|5891|27943|34493|664592|5891|27943|25|30|5273|19|2510.85|125.54|2636.39|19.85|1702.21|728.14|1586.61|196.10|1847.60| +2452750|77092|1201|52938|402982|3779|4623|52938|402982|3779|4623|57|6|5274|65|3276.00|262.08|3538.08|40.01|3276.00|1900.08|1252.08|123.84|3578.09| +2451651|34712|11360|10782|562188|5594|35047|10782|562188|5594|35047|31|25|5275|88|1659.68|116.17|1775.85|17.09|1498.64|647.27|283.47|728.94|1631.90| +2451854|56621|17389|||7075|44609|49160||7075|44609|2|12|5275|42|1197.84||1305.64|||515.07|||| +2451537|35710|14896|95|83318|3694|8917|95|83318|3694|8917|49|13|5276|25|0.00|0.00|0.00|11.22|833.25|0.00|0.00|0.00|844.47| +2452179|40473|16697|10874|1887910|5746|35982|10874|1887910|5746|35982|29|10|5279|72|3175.20|222.26|3397.46|90.35|3042.72|603.28|1080.20|1491.72|3355.33| +2451989|53675|2255|46160|1109987|1014|3641|46160|1109987|1014|3641|56|1|5279|28|5293.40|105.86|5399.26|60.49|756.00|317.60|248.79|4727.01|922.35| +2452385|49213|3175|14875|1140664|3700|20289|14875|1140664|3700|20289|6|34|5280|17|1418.31|85.09|1503.40|28.52|243.10|411.30|382.66|624.35|356.71| +2452573|22617|3273|66809|476779|702|14672|66809|476779|702|14672|1|9|5280|22|2148.96|171.91|2320.87|74.74|994.84|2148.96|0.00|0.00|1241.49| +2452499|75846|17715|57463|992435|5454|10426|57463|992435|5454|10426|45|2|5280|25|98.50|5.91|104.41|67.96|57.50|83.72|2.51|12.27|131.37| +2452197|73969|3617|34392|1282454|4532|11494|34392|1282454|4532|11494|5|24|5281|3|556.68|5.56|562.24|83.28|209.64|467.61|67.69|21.38|298.48| +2451292|16913|10858|8692|394421|503|27432|8692|394421|503|27432|50|10|5283|1|37.58|1.50|39.08|59.72|54.44|2.25|26.49|8.84|115.66| +2451403|58210|8752|64338|193746|4718|36923|64338|193746|4718|36923|32|24|5284|54|1226.88|85.88|1312.76|52.07|1191.24|417.13|80.97|728.78|1329.19| +2452864|62389|7441|84741|1025557|773|22389|84741|1025557|773|22389|48|13|5285|43|455.80|41.02|496.82|39.22|736.59|113.95|194.85|147.00|816.83| +2452781|17608|4074|55396|1521776|5756|31736|55396|1521776|5756|31736|25|14|5286|1|64.74|3.88|68.62|19.58|9.38|50.49|13.53|0.72|32.84| +2451372|64742|11746|23095|686246|6997|24948|23095|686246|6997|24948|25|23|5287|8|847.60|8.47|856.07|12.27|234.40|610.27|135.27|102.06|255.14| +2451378||7243||550411|2886|16961||550411|2886||40||5287|2|303.20|24.25|327.45|||151.60||139.48|| +2451342|57223|6214|25651|813773|295|35617|25651|813773|295|35617|38|19|5287|17|761.77|22.85|784.62|99.57|645.66|182.82|283.68|295.27|768.08| +2451315|70805|3175|50380|539844|3708|27172|50380|539844|3708|27172|55|32|5287|13|1717.82|85.89|1803.71|81.98|204.49|360.74|1044.95|312.13|372.36| +2451932|41178|7925|40541|864438|6735|45479|40541|864438|6735|45479|5|23|5288|3|53.61|2.68|56.29|33.14|71.49|46.10|5.78|1.73|107.31| +2452590|44553|13065|60479|22114|6949|44584|60479|22114|6949|44584|25|17|5289|22|573.54|22.94|596.48|54.81|0.00|177.79|178.08|217.67|77.75| +2452611|9555|3937|11527|846126|1525|41724|11527|846126|1525|41724|54|28|5289|18|2959.74|236.77|3196.51|4.85|429.48|651.14|92.34|2216.26|671.10| +2452488|6688|16014|9027|70586|1298|4857|9027|70586|1298|4857|27|10|5289|12|315.36|18.92|334.28|73.40|0.00|132.45|23.77|159.14|92.32| +2451389|17596|17965|88638|1115483|3928|2373|88638|1115483|3928|2373|43|16|5290|6|488.46|19.53|507.99|73.14|276.60|19.53|103.16|365.77|369.27| +2451304|70735|12292|59750|1697277|851|23625|59750|1697277|851|23625|8|8|5290|4|139.76|2.79|142.55|87.03|9.72|27.95|76.03|35.78|99.54| +2451668|61649|16330||1820068|3537|||1820068||6206||15|5291|14|141.12|4.23||36.27|||||| +2452807|26112|3657|13671|1050254|2837|3420|13671|1050254|2837|3420|60|34|5292|36|3193.20|0.00|3193.20|82.53|199.44|2586.49|546.03|60.68|281.97| +2452866|39934|14029|69976|404601|1179|4957|69976|404601|1179|4957|12|1|5292|5|55.95|4.47|60.42|53.07|24.45|40.28|1.56|14.11|81.99| +2452353|48965|12709|9292|1113876|3707|40691|9292|1113876|3707|40691|55|1|5294|94|195.52|1.95|197.47|5.87|83.66|50.83|68.00|76.69|91.48| +2452314|48866|967|9424|528828|2185|40742|9424|528828|2185|40742|41|6|5294|26|10.92|0.32|11.24|18.23|372.32|10.15|0.29|0.48|390.87| +2452689|9728|4983|13738|1659105|4025|17792|13738|1659105|4025|17792|15|7|5297|16|1074.56|21.49|1096.05|42.99|56.48|838.15|172.57|63.84|120.96| +2451910|30906|10031|58490|602278|2066|607|58490|602278|2066|607|35|23|5298|3|83.88|1.67|85.55|55.66|5.85|68.78|7.24|7.86|63.18| +2452299|48618|10789|57526|112742|5911|43527|57526|112742|5911|43527|45|32|5299|16|1984.32|0.00|1984.32|7.71|482.56|1984.32|0.00|0.00|490.27| +2451826|40748|10778|78139|548649|6806|32116|78139|548649|6806|32116|1|17|5300|2|20.02|1.80|21.82|0.51|5.22|16.81|0.16|3.05|7.53| +2451883|26624|4340|80211|275017|4223|28676|80211|275017|4223|28676|53|23|5300|18|321.84|9.65|331.49|55.99|605.16|212.41|13.13|96.30|670.80| +2452349|67904|15537|21920|1431392|4864|47136|21920|1431392|4864|47136|49|20|5301|32|98.88|5.93|104.81|55.51|167.36|31.64|41.68|25.56|228.80| +2452352|44144|10181|31097|1711176|2102|13781|31097|1711176|2102|13781|27|27|5301|7|170.31|13.62|183.93|1.55|73.01|11.92|142.55|15.84|88.18| +|64202|3083||1163113||25478|48354|1163113|3119|25478|37|31|5302||1029.69|||80.08||761.97||157.96|732.44| +2451213|60166|8824|87278|1166991|5104|44770|87278|1166991|5104|44770|8|31|5303|47|2644.22|79.32|2723.54|81.46|1395.43|2088.93|77.74|477.55|1556.21| +2452077|36871|17463|5511|1462693|6722|14704|5511|1462693|6722|14704|23|14|5304|5|99.50|5.97|105.47|69.15|232.15|75.62|0.23|23.65|307.27| +2452118|62251|11517|1494|1358036|978|35473|1494|1358036|978|35473|19|16|5304|8|1043.12|83.44|1126.56|38.77|154.48|406.81|496.32|139.99|276.69| +2452138|48586|5333|53099|1207577|4808|13686|53099|1207577|4808|13686|21|3|5304|34|6606.20|264.24|6870.44|68.26|1016.26|1321.24|4386.51|898.45|1348.76| +2452247|41950|7193|56639|1008613|1387|35761|56639|1008613|1387|35761|33|15|5305|17|0.00|0.00|0.00|15.63|788.46|0.00|0.00|0.00|804.09| +2452443|64091|6581|47365|1730593|3774|21802|47365|1730593|3774|21802|31|2|5305|2|0.00|0.00|0.00|9.72|12.44|0.00|0.00|0.00|22.16| +2451227|49606|5858|77558|175670|4629|35941|77558|175670|4629|35941|4|22|5306|6|132.06|10.56|142.62|76.08|84.06|110.93|5.28|15.85|170.70| +2451381|16614|10315|8599|217513|1627|37741|8599|217513|1627|37741|13|12|5307|3|183.63|9.18|192.81|37.98|22.62|157.92|6.94|18.77|69.78| +2451758|28299|8525|36294|402264|4234|14204|36294|402264|4234|14204|25|21|5308|62|11.78|0.23|12.01|55.81|290.16|6.12|3.45|2.21|346.20| +2451666|73258|4382|24229|265033|5484|44735|24229|265033|5484|44735|17|27|5308|28|4207.00|294.49|4501.49|22.18|1477.00|2692.48|560.37|954.15|1793.67| +2451834|72163|2300|50858|1093852|846|40287|50858|1093852|846|40287|55|10|5309|1|199.09|15.92|215.01|57.97|134.59|25.88|12.12|161.09|208.48| +2451779|39214|2914|29149|679933|6717|6348|29149|679933|6717|6348|22|34|5309|9|589.59|35.37|624.96|9.01|267.93|11.79|346.68|231.12|312.31| +2451903|76887|9434|5784|1592489|1947|39005|5784|1592489|1947|39005|20|5|5310|4|356.84|28.54|385.38|41.71|194.64|303.31|38.54|14.99|264.89| +2451517||3157||226494|3350||49285|||4206||18|5311|5|57.50|1.72||||8.05||14.35|| +2451403|59854|5462|42140|569526|6159|5545|42140|569526|6159|5545|55|31|5311|25|0.00|0.00|0.00|76.56|227.75|0.00|0.00|0.00|304.31| +2452082|29327|8465|32683|1025434|2985|22536|32683|1025434|2985|22536|47|34|5312|29|561.15|44.89|606.04|71.80|18.56|331.07|131.14|98.94|135.25| +2452649|7934|17173|10583|1460559|5288|36017|10583|1460559|5288|36017|19|27|5314|3|3.81|0.22|4.03|58.80|0.99|3.46|0.10|0.25|60.01| +2452823||17467||653985||32457|88749||||57|20|5314|||||||3712.30||93.01|| +2451727|15537|17548|28444|246537|6230|32723|28444|246537|6230|32723|38|15|5315|23|2268.26|158.77|2427.03|37.58|467.59|2109.48|85.74|73.04|663.94| +2452315|64533|5927|63194|765673|2552|19031|63194|765673|2552|19031|11|29|5316|1|49.82|2.98|52.80|74.43|24.91|36.36|10.09|3.37|102.32| +2452275|67204|4609|10119|1081302|4932|44515|10119|1081302|4932|44515|7|34|5317|12|1068.48|10.68|1079.16|99.43|424.44|192.32|367.98|508.18|534.55| +2451645|65069|12796|42676|195930|1537|41191|42676|195930|1537|41191|38|19|5318|4|130.40|6.52|136.92|74.60|120.52|20.86|109.54|0.00|201.64| +2452452|69401|7141|53637|525189|4388|35940|53637|525189|4388|35940|48|29|5319|38|1031.32|30.93|1062.25|59.81|80.56|195.95|116.95|718.42|171.30| +2452056|43123|8624|27946|502577|3608|6639|27946|502577|3608|6639|32|15|5321|52|2595.32|25.95|2621.27|69.55|1748.76|2387.69|70.59|137.04|1844.26| +2452041|82524|3991|93507|58527|3100|6873|93507|58527|3100|6873|25|15|5321|26|1421.42|14.21|1435.63|18.55|473.72|1179.77|130.49|111.16|506.48| +2452784|57575|15393|81238|1369586|6287|32188|81238|1369586|6287|32188|1|13|5322|37|923.52|0.00|923.52|77.87|442.52|0.00|64.64|858.88|520.39| +2451763|75669|12370|50138||3583||50138||||58||5323|8|483.76|||96.09|60.40|304.76|30.43||156.49| +2451787|44043|3754|16836|1051996|756|30316|16836|1051996|756|30316|25|4|5323|18|476.46|33.35|509.81|64.95|114.84|200.11|232.13|44.22|213.14| +2452510|49642|3763|5449|1558298|4394|29290|5449|1558298|4394|29290|25|34|5324|20|1154.60|0.00|1154.60|20.02|60.60|785.12|214.29|155.19|80.62| +|69986|6987||1416703||11232|61215||2144|||13|5324||||448.49||71.19|||2.74|152.88| +2452501|24146|11435|63055|441963|1671|26579|63055|441963|1671|26579|3|32|5324|6|255.18|10.20|265.38|23.62|5.28|96.96|151.89|6.33|39.10| +2451011|29442|12034|5628|327763|5545|40080|5628|327763|5545|40080|7|27|5326|15|2175.15|195.76|2370.91|72.27|1837.50|1000.56|845.70|328.89|2105.53| +2452079|39328|1149|57845|1669859|3120|17934|57845|1669859|3120|17934|39|15|5327|17|99.96|4.99|104.95|87.29|20.74|94.96|1.25|3.75|113.02| +2452150|71350|5161|53077|168575|2120|20418|53077|168575|2120|20418|3|24|5327|7|512.82|20.51|533.33|46.15|214.62|328.20|121.84|62.78|281.28| +2451696|31921|17203|44823|588007|1944|27081|44823|588007|1944|27081|50|4|5328|35|1708.35|153.75|1862.10|48.27|1192.45|973.75|683.17|51.43|1394.47| +2451837|10815|4897|41182|1686633|2037|20641|41182|1686633|2037|20641|50|24|5328|36|1774.08|35.48|1809.56|4.73|2634.12|532.22|161.44|1080.42|2674.33| +2451768|31848|17914|82853|261658|1616|10821|82853|261658|1616|10821|38|19|5328|40|320.80|25.66|346.46|92.01|152.80|73.78|51.87|195.15|270.47| +2452500|73815|2655|28947|197273|5952|35831|28947|197273|5952|35831|37|19|5330|19|4640.37|46.40|4686.77|75.22|1367.62|2877.02|828.77|934.58|1489.24| +2452609|76936|17361|26374|305149|5171|29193|26374|305149|5171|29193|24|34|5330|47|2924.81|29.24|2954.05|23.28|590.79|1842.63|670.95|411.23|643.31| +||9079||225922|704|25486||225922|704|||9|5331|4|81.20||86.07|12.62||58.46|||70.25| +2452489|21527|1477|15697|1799508|1272|10283|15697|1799508|1272|10283|23|25|5332|52|3078.92|123.15|3202.07|1.55|729.04|677.36|2233.45|168.11|853.74| +2452399|5888|13139|44972|1184135|4898|2461|44972|1184135|4898|2461|51|17|5334|10|362.90|18.14|381.04|80.98|225.20|221.36|42.46|99.08|324.32| +|51013|263||||9735|56353|1674724||9735|||5334|8|2047.68|143.33|2191.01||550.40||159.72|249.82|| +2452791||17881|52593|674833|1873||52593||1873||49|24|5337||3982.78|0.00|3982.78|61.44|||877.80||2550.56| +2452846|43443|9555|63966|704399|3232|22465|63966|704399|3232|22465|15|33|5339|36|1187.64|35.62|1223.26|53.59|553.32|178.14|726.84|282.66|642.53| +2452875|70658|1207|94668|1540582|6943|27304|94668|1540582|6943|27304|54|33|5340|2|87.02|6.96|93.98|22.38|48.48|43.51|37.41|6.10|77.82| +2452856|69478|16482|86770|468768|4013|6411|86770|468768|4013|6411|15|3|5340|16|74.08|4.44|78.52|68.21|12.32|12.59|8.60|52.89|84.97| +2452433|65789|10175|49131|1640727|1570|1166|49131|1640727|1570|1166|19|26|5341|67|1153.74|11.53|1165.27|87.91|1025.77|369.19|470.73|313.82|1125.21| +2452316|75258|15225|65795|204458|3121|46537|65795|204458|3121|46537|39|5|5343|15|64.80|5.18|69.98|37.16|363.45|8.42|37.77|18.61|405.79| +2452890|30061|246|14452|1366916|4533|33397|14452|1366916|4533|33397|24|4|5345|18|2570.58|231.35|2801.93|70.02|421.38|257.05|2035.90|277.63|722.75| +2452821||10345|||4909||||4909|||35|5345||||560.88||||||471.70| +2451169|70317|3106|43589|1784534|2279|39224|43589|1784534|2279|39224|46|1|5346|11|198.66|17.87|216.53|92.35|154.55|154.95|33.21|10.50|264.77| +2452296|47865|11653|81174|396773|4819|48865|81174|396773|4819|48865|39|35|5347|8|1155.92|104.03|1259.95|73.61|570.96|161.82|765.45|228.65|748.60| +2452384|40929|13410|12538|647675|1212|10749|12538|647675|1212|10749|9|29|5348|29|60.03|1.20|61.23|90.97|1506.84|38.41|20.97|0.65|1599.01| +2451133|63526|10490|30487|1652575|3469|14723|30487|1652575|3469|14723|46|18|5352|25|1493.00|0.00|1493.00|57.29|232.25|627.06|571.52|294.42|289.54| +2451366|36736|4507|41177|319610|4443|7676|41177|319610|4443|7676|19|21|5353|78|3696.42|0.00|3696.42|57.77|1029.60|3178.92|20.70|496.80|1087.37| +2451234||16846|52214||4632|8441|52214||4632||||5353|29|||1472.39||1688.67|337.70|932.07||| +2451668|47570|4640|20118|509855|3592|41729|20118|509855|3592|41729|49|13|5355|33|529.32|10.58|539.90|85.09|291.06|460.50|11.01|57.81|386.73| +2452387|44922|16907|43437|171966|848|18530|43437|171966|848|18530|29|13|5357|23|161.46|14.53|175.99|3.09|74.06|80.73|75.07|5.66|91.68| +2452449|17675|2499|86733|683239|4098|9757|86733|683239|4098|9757|1|7|5357|74|79.18|3.16|82.34|79.57|146.52|76.01|2.15|1.02|229.25| +2452379|70813|11739|92967|1101608|3496|30537|92967|1101608|3496|30537|47|10|5357|13|148.07|2.96|151.03|79.83|53.04|84.39|2.54|61.14|135.83| +2451338|74940|15976|30974|547640|6353|41419|30974|547640|6353|41419|13|18|5358|2|57.10|2.85|59.95|23.00|41.34|9.70|10.90|36.50|67.19| +2452043||5060||1501469||23296||||23296|13||5359||452.70||461.75|48.95||9.05|||| +2452086|22003|16916|79632|43316|3800|19884|79632|43316|3800|19884|47|9|5360|6|17.70|0.53|18.23|4.09|10.68|5.66|9.87|2.17|15.30| +2452149|37118|15383|64649|1386267|5132|41597|64649|1386267|5132|41597|20|1|5360|27|691.20|48.38|739.58|55.11|426.33|559.87|76.17|55.16|529.82| +2452906|50132|15307|79800|1488594|2541|15688|79800|1488594|2541|15688|27|11|5362|41|1050.83|0.00|1050.83|15.33|648.21|336.26|71.45|643.12|663.54| +2452153|24137|13922|46272|1716996|5673|5150|46272|1716996|5673|5150|2|18|5363|1|93.60|6.55|100.15|75.07|21.23|58.03|9.60|25.97|102.85| +2452106|78135|6566|61634|844113|545|48577|61634|844113|545|48577|32|29|5363|12|2094.72|167.57|2262.29|49.12|1341.84|1110.20|226.43|758.09|1558.53| +2452798|60137|2646|43869|352940|2011|39882|43869|352940|2011|39882|55|6|5364|49|2555.84|178.90|2734.74|68.09|884.45|613.40|1573.37|369.07|1131.44| +2452743|31134|17989|23392|1443223|1101|46179|23392|1443223|1101|46179|21|8|5365|19|83.03|3.32|86.35|75.09|59.85|53.96|25.87|3.20|138.26| +2451261|72471|12782|90978|234316|2101|22239|90978|234316|2101|22239|2|3|5366|24|1241.76|37.25|1279.01|7.99|620.88|124.17|55.87|1061.72|666.12| +2451752|68363|17329|51442|755733|6071|43962|51442|755733|6071|43962|55|24|5368|7|15.89|0.79|16.68|89.34|334.74|2.86|11.46|1.57|424.87| +2451994|64506|463|37802|792396|1037|22031|37802|792396|1037|22031|32|23|5368|14|382.20|3.82|386.02|53.54|985.74|256.07|105.94|20.19|1043.10| +2451942|21756|15173|90749|1046923|2297|23671|90749|1046923|2297|23671|11|29|5368|6|102.60|1.02|103.62|31.51|62.16|36.93|43.99|21.68|94.69| +2451727|32857|12944|28347|1271434|121|44008|28347|1271434|121|44008|19|13|5370|38|993.70|29.81|1023.51|96.44|311.22|953.95|20.27|19.48|437.47| +2451675|21343|979|57865|293981|2097|31413|57865|293981|2097|31413|50|5|5370|47|4277.47|128.32|4405.79|46.18|266.96|983.81|1449.21|1844.45|441.46| +2451770|60263|14350|51881|814084|3040|396|51881|814084|3040|396|16|21|5370|61|1402.39|14.02|1416.41|95.42|2805.39|154.26|1035.94|212.19|2914.83| +||3338|72980|141520||||||12600|||5371||||||0.90|||2.40|| +2452732|65658|9480|68405|1835795|5585|45375|68405|1835795|5585|45375|31|5|5372|22|475.42|33.27|508.69|73.29|723.58|318.53|43.92|112.97|830.14| +2452766|27259|6115|63150|1336752|6707|27109|63150|1336752|6707|27109|21|33|5372|17|1734.51|0.00|1734.51|3.30|0.00|156.10|473.52|1104.89|3.30| +2452203|15134|14117|4597|1739228|4612|5243|4597|1739228|4612|5243|33|18|5373|17|1257.15|88.00|1345.15|46.66|0.00|905.14|130.24|221.77|134.66| +2452289|73857|227|72787|1713679|1324|5816|72787|1713679|1324|5816|25|17|5373|2|33.30|2.33|35.63|85.92|3.70|9.99|0.93|22.38|91.95| +2451172|74999|13198|25694|1645830|6319|41655|25694|1645830|6319|41655|14|27|5375|19|467.02|28.02|495.04|17.77|69.16|121.42|58.75|286.85|114.95| +2451179|54526|17372|85116|77947|6778|44362|85116|77947|6778|44362|1|28|5375|22|1986.38|158.91|2145.29|78.69|171.60|1350.73|171.62|464.03|409.20| +2452607|74550|12475|8064|1530217|6134|32140|8064|1530217|6134|32140|27|7|5377|19|808.07|48.48|856.55|62.66|403.94|234.34|562.25|11.48|515.08| +2452777|45118|1875|2660|1398749|3285|23374|2660|1398749|3285|23374|21|7|5377|26|29.90|2.69|32.59|74.63|1504.10|29.30|0.48|0.12|1581.42| +2452618|15314|8281|35809|586121|1233|48745|35809|586121|1233|48745|3|22|5377|33|3479.85|34.79|3514.64|67.45|186.12|1879.11|1440.66|160.08|288.36| +2451230|43836|3940|38227|1398269|3911|38353|38227|1398269|3911|38353|8|2|5378|20|564.20|11.28|575.48|47.07|285.20|490.85|38.87|34.48|343.55| +2451189|71873|13580|60534|789124|2067|34657|60534|789124|2067|34657|32|32|5378|43|1032.43|92.91|1125.34|45.22|254.13|929.18|94.99|8.26|392.26| +2451653|59628|3824|7814|305281|3548|18215|7814|305281|3548|18215|34|10|5380|6|69.18|1.38|70.56|67.07|59.34|44.27|22.16|2.75|127.79| +2451990|46435|7850|31903|970064|7072|41633|31903|970064|7072|41633|11|10|5381|5|10.15|0.81|10.96|67.49|7.80|3.14|5.95|1.06|76.10| +||17315|2670||||2670|||||35|5381|||1.45|37.77|56.60||||13.79|64.53| +2451899|69972|13052|98693|585765|5990|9815|98693|585765|5990|9815|26|4|5382|33|3922.71|117.68|4040.39|60.07|2315.61|3726.57|100.03|96.11|2493.36| +2451928|44541|4964|37782|1633152|75|11445|37782|1633152|75|11445|44|9|5382|26|45.50|0.45|45.95|14.21|182.52|45.50|0.00|0.00|197.18| +2452073|65158|7111|85922|1383889|1503|5870|85922|1383889|1503|5870|39|2|5383|8|1064.00|0.00|1064.00|49.33|182.00|904.40|63.84|95.76|231.33| +2452285|77870|4935|82173|1658224|6628|8279|82173|1658224|6628|8279|37|1|5384|6|227.76|6.83|234.59|27.48|124.20|177.65|20.04|30.07|158.51| +2452294|37404|11719|3380|161784|4988|592|3380|161784|4988|592|53|28|5384|59|405.92|24.35|430.27|9.08|1798.32|109.59|127.42|168.91|1831.75| +2452127|77253|5366|83064|644917|6770|46223|83064|644917|6770|46223|2|24|5385|1|94.05|5.64|99.69|34.21|78.38|50.78|30.28|12.99|118.23| +2451991|4775|1808|41085|116816|2147|25831|41085|116816|2147|25831|20|33|5385|11|121.66|10.94|132.60|20.43|20.68|72.99|24.82|23.85|52.05| +2451448|13285|14380|65165|1582126|5040|23601|65165|1582126|5040|23601|43|35|5386|38|3626.72|290.13|3916.85|9.72|894.90|2574.97|799.33|252.42|1194.75| +2451434|78677|5125|97994|976311|5929|529|97994|976311|5929|529|37|17|5386|65|0.00|0.00|0.00|64.27|543.40|0.00|0.00|0.00|607.67| +2452817|9726|11373|78717|1442030|896|16609|78717|1442030|896|16609|49|21|5387|23|394.68|15.78|410.46|4.84|1014.99|280.22|78.97|35.49|1035.61| +2452850|61522|5586|59695|1201129|4963|27597|59695|1201129|4963|27597|39|18|5388|12|258.36|5.16|263.52|62.39|81.60|121.42|31.49|105.45|149.15| +2451633|65268|17840|49141|376033|4078|34970|49141|376033|4078|34970|52|28|5389|3|305.49|12.21|317.70|71.49|40.50|97.75|101.79|105.95|124.20| +2452572|73963|13249|1430|193818|2653|33441|1430|193818|2653|33441|25|21|5390|66|246.18|0.00|246.18|58.77|339.90|59.08|1.87|185.23|398.67| +2452713|72152|13806|35639|363436|4386|28536|35639|363436|4386|28536|60|20|5390|6|187.44|3.74|191.18|93.54|67.80|11.24|176.20|0.00|165.08| +2452559|21184|13659|88449|1648741|1833|24925|88449|1648741|1833|24925|43|17|5392|51|8443.05|84.43|8527.48|20.24|3039.09|6838.87|1283.34|320.84|3143.76| +2451660|75606|13424|18992|1388817|3089|161|18992|1388817|3089|161|22|32|5396|1|19.27|0.00|19.27|11.57|13.04|6.55|8.64|4.08|24.61| +2451504|7658|901|55468|874263|705|1100|55468|874263|705|1100|2|16|5398|61|11219.73|448.78|11668.51|69.82|2739.51|8639.19|2451.51|129.03|3258.11| +2451431|47921|17227|93181|243438|2942|44399|93181|243438|2942|44399|7|25|5399|33|2965.38|207.57|3172.95|27.99|889.35|474.46|1619.09|871.83|1124.91| +2451124|28635|12715|79244|436747|7050|7353|79244|436747|7050|7353|52|24|5400|65|2432.95|170.30|2603.25|6.73|546.65|1508.42|795.09|129.44|723.68| +2451105|67727|13852|27892|1623357|3269|16781|27892|1623357|3269|16781|14|23|5400|1|74.29|5.20|79.49|30.82|1.45|5.94|68.35|0.00|37.47| +2451738|70823|3352|87087|1527281|1075|34869|87087|1527281|1075|34869|13|33|5401|2|92.30|1.84|94.14|57.94|130.32|48.91|24.29|19.10|190.10| +2451584|54682|11702|5636|515316|643|13570|5636|515316|643|13570|22|12|5401|5|65.30|3.91|69.21|23.19|78.35|48.32|0.50|16.48|105.45| +2451694|4165|17053|79073|1709073|6273|19873|79073|1709073|6273|19873|44|29|5401|50|838.00|41.90|879.90|36.87|253.00|16.76|648.77|172.47|331.77| +2452462|45860|13633|64766|1671409|3652|34695|64766|1671409|3652|34695|19|15|5402|64|729.60|0.00|729.60|18.22|446.72|80.25|175.32|474.03|464.94| +2451068|36302|3818|55187|1693356|4325|30494|55187|1693356|4325|30494|38|25|5403|11|605.22|48.41|653.63|11.70|100.87|18.15|399.20|187.87|160.98| +2451096|62595|14680|88107|1035817|6374|19731|88107|1035817|6374|19731|56|24|5403|30|1866.60|111.99|1978.59|72.43|1306.50|746.64|884.76|235.20|1490.92| +2452443|70055|2187|48740|1272979|4418|14610|48740|1272979|4418|14610|43|12|5404|8|593.04|11.86|604.90|87.14|431.28|349.89|92.39|150.76|530.28| +2452517|72804|9037|64610|919949|6841|9733|64610|919949|6841|9733|25|19|5404|23|1441.64|43.24|1484.88|53.74|600.53|43.24|1020.83|377.57|697.51| +2451803|47222|6799|11054|285433|4259|43085|11054|285433|4259|43085|7|29|5406|35|3536.75|106.10|3642.85|45.27|1991.85|2192.78|497.26|846.71|2143.22| +2452876|26503|9210|18779|381805|1278|7104|18779|381805|1278|7104|9|21|5407|49|6928.11|69.28|6997.39|40.02|3024.77|6373.86|288.21|266.04|3134.07| +2452728|69038|8502|95627|754338|2649|560|95627|754338|2649|560|1|23|5407|3|116.85|5.84|122.69|58.32|64.92|32.71|34.49|49.65|129.08| +|35730|12436|13139|||18540|13139|460508||18540|43|12|5408|8||80.45||41.40|268.16||241.37|429.11|| +2451391|79607|16864|38709|443086|263|19040|38709|443086|263|19040|7|30|5409|15|739.35|59.14|798.49|4.30|763.20|221.80|481.32|36.23|826.64| +2451367|63854|5636|32415|1480918|4174|14230|32415|1480918|4174|14230|7|26|5410|77|503.58|10.07|513.65|21.40|359.59|352.50|116.33|34.75|391.06| +2451518|67545|2548|91636|1121266|496|22526|91636|1121266|496|22526|13|21|5410|4|211.20|0.00|211.20|79.90|138.60|50.68|33.70|126.82|218.50| +2452307||14803||||5525||684304|4775|5525|||5411|87|||||2090.61|||199.65|2234.54| +2452157|1056|435|7987|1647303|1372|13888|7987|1647303|1372|13888|59|33|5415|33|97.02|4.85|101.87|76.60|125.07|0.97|24.01|72.04|206.52| +2452079|27391|16291|45365|56783|5354|49125|45365|56783|5354|49125|1|22|5415|5|101.85|3.05|104.90|79.29|87.30|19.35|63.52|18.98|169.64| +2451706|73238|3202|51934|1573605|1676|2547|51934|1573605|1676|2547|32|28|5417|19|634.03|50.72|684.75|95.94|54.53|450.16|12.87|171.00|201.19| +2451909|57389|2749|4733|1868964|5195|10626|4733|1868964|5195|10626|47|20|5418|7|136.01|5.44|141.45|15.92|231.21|74.80|59.37|1.84|252.57| +2451866|47186|9656|22523|1004381|1439|48497|22523|1004381|1439|48497|41|13|5418|66|9046.62|452.33|9498.95|28.94|6082.56|4432.84|3967.85|645.93|6563.83| +2452059|23077|8669|35620|1718354|4288|41022|35620|1718354|4288|41022|20|6|5420|2|9.80|0.29|10.09|96.79|2.28|3.62|0.12|6.06|99.36| +2452886|57300|11268|23669|1082395|5474|27353|23669|1082395|5474|27353|51|18|5421|6|21.30|1.06|22.36|72.23|20.70|7.02|8.13|6.15|93.99| +2452840|17718|6637|43798|490551|4857|34363|43798|490551|4857|34363|49|28|5421|2|42.62|2.98|45.60|20.53|23.82|4.68|16.69|21.25|47.33| +2452669|64810|16977|94618|612920|4484|27905|94618|612920|4484|27905|18|17|5421|6|18.66|1.11|19.77|17.15|41.04|5.97|12.56|0.13|59.30| +2451190|69621|9205|82780|1915728|2276|11624|82780|1915728|2276|11624|28|24|5426|20|57.40|5.16|62.56|41.97|786.20|53.95|1.44|2.01|833.33| +2452722|73277|11550|64339|1842086|3033|19933|64339|1842086|3033|19933|49|33|5428|15|484.80|19.39|504.19|62.24|234.75|387.84|96.96|0.00|316.38| +2451719|68849|13150|73259|1756140|4268|30349|73259|1756140|4268|30349|10|19|5431|25|1325.00|53.00|1378.00|10.20|265.00|463.75|129.18|732.07|328.20| +2451654|25935|6409|57121|1078553|6738|9770|57121|1078553|6738|9770|7|11|5431|89|20605.28|0.00|20605.28|47.78|5151.32|7211.84|2410.81|10982.63|5199.10| +2452658|44330|8365|59303|672337|1018|46452|59303|672337|1018|46452|51|22|5432|74|5279.90|263.99|5543.89|5.66|3472.08|2903.94|1092.94|1283.02|3741.73| +2452828|38748|11781|72754|1462261|6615|36969|72754|1462261|6615|36969|54|31|5432|14|273.84|2.73|276.57|0.56|182.56|71.19|137.80|64.85|185.85| +2451702|29138|3091|2889|370720|951|20335|2889|370720|951|20335|22|24|5433|34|311.78|0.00|311.78|88.79|79.22|218.24|33.67|59.87|168.01| +2451791|36460|12508|97713|1124848|6383|39118|97713|1124848|6383|39118|25|30|5433|7|451.57|40.64|492.21|13.77|179.27|13.54|188.35|249.68|233.68| +2451626|9562|14503|34793|1407343|3446|27949|34793|1407343|3446|27949|40|9|5433|11|606.43|42.45|648.88|56.31|619.30|497.27|94.96|14.20|718.06| +2451771|59950|5014|45581|1345473|1812|48504|45581|1345473|1812|48504|38|25|5433|19|1047.09|10.47|1057.56|50.27|378.67|712.02|26.80|308.27|439.41| +2452853|52904|11049|69056|382976|896|8016|69056|382976|896|8016|18|24|5435|59|18.88|0.18|19.06|81.78|0.00|17.74|0.22|0.92|81.96| +2452835|22034|295|69034|1852985|6183|34981|69034|1852985|6183|34981|1|4|5435|1|58.94|3.53|62.47|90.97|16.94|40.66|5.66|12.62|111.44| +2451288||10303||1550404||||1550404|1945||7||5436|5|||88.03|89.44|126.95|||0.77|219.77| +2452431|51483|7249|5328|108474|4773|9919|5328|108474|4773|9919|25|9|5437|26|861.38|25.84|887.22|77.79|269.88|379.00|405.19|77.19|373.51| +2452592|70309|12043|8883|620485|6487|47736|8883|620485|6487|47736|60|32|5437|26|3507.14|245.49|3752.63|26.15|368.94|3156.42|38.57|312.15|640.58| +2452237|41685|6009|1240|320648|4409|35370|1240|320648|4409|35370|27|32|5438|2|0.00|0.00|0.00|87.64|47.00|0.00|0.00|0.00|134.64| +|43149|1797||326251|||55937|326251|||||5438|||30.54|539.58||||||| +2452067|67827|683|32436|1760521|6559|44597|32436|1760521|6559|44597|59|26|5438|10|27.00|2.43|29.43|59.13|22.40|21.60|1.51|3.89|83.96| +2451876|54382|2915|6968|969045|369|2893|6968|969045|369|2893|32|7|5439|3|442.92|17.71|460.63|70.08|206.70|22.14|21.03|399.75|294.49| +2451808|46018|4154|92649|1199659|6669|16879|92649|1199659|6669|16879|43|24|5439|55|1504.80|45.14|1549.94|42.85|533.50|1053.36|334.06|117.38|621.49| +2452623|31972|4632|18071|565244|2464|40037|18071|565244|2464|40037|19|13|5441|8|344.80|20.68|365.48|38.99|539.76|320.66|7.72|16.42|599.43| +2452558|59146|15756|43606|943907|4208|24956|43606|943907|4208|24956|48|16|5441|3|128.88|1.28|130.16|62.68|103.11|46.39|31.34|51.15|167.07| +2452709|53230|15822|58082|585555|1028|4101|58082|585555|1028|4101|43|12|5441|11|17.71|0.17|17.88|55.02|443.30|4.42|2.25|11.04|498.49| +2451563|31311|15733|45624|1397160|173|10902|45624|1397160|173|10902|20|13|5442|17|24.99|0.24|25.23|8.22|250.58|10.99|11.48|2.52|259.04| +2452486|39592|2593|76938|1802577|5369|15808|76938|1802577|5369|15808|60|10|5443|4|244.72|14.68|259.40|15.33|244.72|90.54|123.34|30.84|274.73| +2452156|70250|343|51901|1441662|6976|15537|51901|1441662|6976|15537|25|35|5444|1|43.62|2.18|45.80|95.00|40.13|22.68|20.94|0.00|137.31| +2452170|50323|3083|8107|1088580|700|2020|8107|1088580|700|2020|1|16|5444|48|781.44|70.32|851.76|46.74|1445.76|734.55|18.75|28.14|1562.82| +2452044|74161|16167|73312|125903|4894|285|73312|125903|4894|285|1|33|5444|1|35.98|0.35|36.33|8.67|9.13|12.59|4.91|18.48|18.15| +2451260|45768|1249|68808|1340179|2750|9768|68808|1340179|2750|9768|22|11|5445|13|310.96|18.65|329.61|36.99|336.96|93.28|132.78|84.90|392.60| +2451337|75162|8510|52212|1196117|1175|48233|52212|1196117|1175|48233|26|2|5445|41|902.82|81.25|984.07|28.90|3694.51|153.47|509.55|239.80|3804.66| +2451416|72163|4270|25149|907750|1998|10199|25149|907750|1998|10199|34|8|5445|53|5924.34|355.46|6279.80|53.97|473.82|2606.70|3184.93|132.71|883.25| +2451492||5893||||43212|3130|653926|4582||31|13|5447||8.69|0.00||99.45|10.14||||| +2451652|49766|17197|79737|1790490|1596|25949|79737|1790490|1596|25949|38|6|5447|17|2058.36|144.08|2202.44|59.62|1194.42|988.01|299.69|770.66|1398.12| +2451723|40810|4378|77581|31034|2534|43987|77581|31034|2534|43987|7|6|5447|74|2942.24|88.26|3030.50|79.74|1681.28|1883.03|974.47|84.74|1849.28| +2452421|65194|8473|61277|744053|3982|11511|61277|744053|3982|11511|55|12|5448|62|2507.28|25.07|2532.35|6.18|2507.28|977.83|1223.56|305.89|2538.53| +2452455|25801|5647|52405|387795|5585|14175|52405|387795|5585|14175|39|29|5448|81|12975.39|0.00|12975.39|98.21|2413.80|2595.07|9446.09|934.23|2512.01| +2452452|26582|4439|59668|1070250|6879|17461|59668|1070250|6879|17461|31|27|5450|3|256.14|17.92|274.06|81.20|198.48|25.61|48.41|182.12|297.60| +2452117|38914|11996|78641|279801|6914|20074|78641|279801|6914|20074|26|32|5451|13|454.74|18.18|472.92|27.69|82.03|363.79|82.76|8.19|127.90| +2452107|55835|5155|34717|1035939|5608|12430|34717|1035939|5608|12430|38|6|5451|1|40.41|0.00|40.41|53.88|49.39|22.22|0.72|17.47|103.27| +2452082|42670|2651|10334|1447057|5380|28503|10334|1447057|5380|28503|47|21|5451|5|761.00|0.00|761.00|98.19|158.85|0.00|30.44|730.56|257.04| +2452034|80851|15583|96278|830918|5205|5845|96278|830918|5205|5845|43|12|5451|12|149.88|8.99|158.87|56.55|74.88|1.49|138.00|10.39|140.42| +2452239|66035|7989|33657|394482|5260|6438|33657|394482|5260|6438|53|2|5452|2|24.70|0.49|25.19|78.66|89.82|22.72|0.11|1.87|168.97| +2451431|30844|8995|42611|1143782|4953|20124|42611|1143782|4953|20124|37|32|5453|70|275.10|24.75|299.85|94.39|716.80|198.07|39.28|37.75|835.94| +2451328|69375|520|80363|1203868|5420|8525|80363|1203868|5420|8525|28|9|5454|31|834.83|66.78|901.61|66.47|295.43|183.66|234.42|416.75|428.68| +2451442|82014|11392|69999|1159|1119|38279|69999|1159|1119|38279|49|32|5455|7|353.92|7.07|360.99|65.79|353.92|127.41|138.17|88.34|426.78| +2451763|70406|500|62419|84070|3321|21419|62419|84070|3321|21419|31|7|5457|38|490.58|4.90|495.48|37.62|451.06|426.80|47.83|15.95|493.58| +2451961|64149|10355|59159|657330|5454|41178|59159|657330|5454|41178|2|18|5458|64|53.12|3.18|56.30|73.92|695.04|26.02|10.29|16.81|772.14| +2452006|28922|6077|51207|69339|2671|17203|51207|69339|2671|17203|56|7|5458|6|234.00|7.02|241.02|77.65|314.28|18.72|81.80|133.48|398.95| +2452301|69111|7489|34258|291560|862|44964|34258|291560|862|44964|33|16|5459|32|1201.28|36.03|1237.31|61.60|430.08|648.69|16.57|536.02|527.71| +2452367|45683|5389|8285|1868762|4086|963|8285|1868762|4086|963|33|7|5460|10|762.50|38.12|800.62|29.11|152.50|396.50|263.52|102.48|219.73| +2452003|69246|12686|54852|802928|6651|46374|54852|802928|6651|46374|50|18|5461|21|1309.35|117.84|1427.19|2.12|1309.35|1296.25|11.52|1.58|1429.31| +2451781|54610|7339|48866|862317|921|20772|48866|862317|921|20772|14|32|5463|19|1492.45|44.77|1537.22|7.75|447.64|417.88|515.79|558.78|500.16| +2451720|61215|15928|12373|1624149|5078|5254|12373|1624149|5078|5254|46|3|5463|2|244.64|4.89|249.53|91.66|15.78|193.26|4.11|47.27|112.33| +2452638|45243|10698|32461|1571350|3486|12375|32461|1571350|3486|12375|1|2|5464|6|35.16|2.46|37.62|58.31|0.36|26.01|4.94|4.21|61.13| +2452715|36154|2253|65839|809531|5670|13409|65839|809531|5670|13409|24|21|5464|25|560.00|16.80|576.80|14.76|609.50|448.00|94.08|17.92|641.06| +2451906|17971|12659|65740|1504934|1281|25275|65740|1504934|1281|25275|8|31|5465|38|383.04|19.15|402.19|89.64|547.58|137.89|53.93|191.22|656.37| +2451977|55112|4451|81274|1321920|4966|38012|81274|1321920|4966|38012|49|8|5465|53|1145.86|57.29|1203.15|52.41|859.13|939.60|167.07|39.19|968.83| +2451503|56121|10027|34385|509169|2800|26384|34385|509169|2800|26384|55|22|5466|64|1437.44|14.37|1451.81|54.29|159.36|603.72|383.51|450.21|228.02| +2451941|46503|15950|58138|1708375|1365|43955|58138|1708375|1365|43955|14|10|5467|36|335.16|13.40|348.56|85.04|72.72|251.37|31.00|52.79|171.16| +2451646||17644|20439||5969|2734||404404|5969||||5469||||242.61|||40.43||88.43|| +2452115|43185|2120|46846|1272515|584|49453|46846|1272515|584|49453|59|5|5471|30|1374.00|123.66|1497.66|36.00|503.70|1319.04|8.24|46.72|663.36| +2451460|53042|14168|86110|781324|3385|10999|86110|781324|3385|10999|46|9|5472|14|478.52|28.71|507.23|27.63|194.32|411.52|53.60|13.40|250.66| +2451256|12972|5509|68263|1335041|1819|38197|68263|1335041|1819|38197|26|31|5472|44|3844.72|38.44|3883.16|34.54|183.04|884.28|1865.07|1095.37|256.02| +2451509|49795|9146|88079|484508|6537|14966|88079|484508|6537|14966|10|7|5472|5|43.90|0.43|44.33|82.68|195.60|32.48|2.96|8.46|278.71| +2452290|11412|2595|14986|1537834|1929|16377|14986|1537834|1929|16377|17|24|5473|11|835.89|33.43|869.32|4.13|1180.08|777.37|12.28|46.24|1217.64| +2452312|75834|14545|50356|1742975|6794|2566|50356|1742975|6794|2566|17|21|5474|68|1134.24|102.08|1236.32|98.31|82.28|306.24|803.16|24.84|282.67| +2452204|21438|6463|25003|142337|6071|26205|25003|142337|6071|26205|9|1|5474|17|51.85|0.51|52.36|90.76|230.86|11.92|22.36|17.57|322.13| +2451899|26263|12662|47379|151128|6524|9920|47379|151128|6524|9920|31|7|5475|58|4696.26|281.77|4978.03|64.76|164.72|3052.56|1446.45|197.25|511.25| +2452068|8831|4219|48994|222495|1596|48650|48994|222495|1596|48650|26|2|5475|22|446.38|17.85|464.23|0.91|0.00|58.02|194.18|194.18|18.76| +2451846|8115|6019|27864|1755139|4814|6936|27864|1755139|4814|6936|13|30|5476|14|1492.12|89.52|1581.64|79.42|792.68|373.03|604.30|514.79|961.62| +2451558|50951|15787|94316|745464|7012|28560|94316|745464|7012|28560|8|21|5477|7|30.38|0.91|31.29|30.34|233.31|0.60|4.76|25.02|264.56| +2451750|61817|7045|62963|1471918|5320|31875|62963|1471918|5320|31875|31|24|5477|5|145.95|4.37|150.32|96.72|76.80|35.02|63.23|47.70|177.89| +||17357|38662|||6615|||||37||5480||||||0.00||130.18|303.77|78.09| +2451243|21203|17164|14874|16846|4418|11990|14874|16846|4418|11990|55|12|5482|1|24.97|0.99|25.96|5.96|21.30|14.73|6.34|3.90|28.25| +2451983|28487|11197|63864|1718308|5999|15290|63864|1718308|5999|15290|20|16|5483|14|358.40|28.67|387.07|60.48|328.58|265.21|66.16|27.03|417.73| +2452165|27521|1519|29531|730682|3033|27784|29531|730682|3033|27784|20|30|5483|54|1341.36|0.00|1341.36|73.69|945.00|871.88|140.84|328.64|1018.69| +2452094|70200|9403|30100|358341|5782|41623|30100|358341|5782|41623|26|2|5483|49|1702.75|136.22|1838.97|97.28|3012.52|1225.98|348.04|128.73|3246.02| +2452067|14191|5353|16772|642751|377|4756|16772|642751|377|4756|56|6|5483|10|1071.60|53.58|1125.18|40.02|480.80|32.14|634.07|405.39|574.40| +2452625|49723|9651|75229|972733|5084|47435|75229|972733|5084|47435|9|15|5485|3|224.73|17.97|242.70|10.73|57.63|11.23|17.08|196.42|86.33| +2452285|41012|6797|26316|31142|4326|27663|26316|31142|4326|27663|47|22|5486|3|457.35|0.00|457.35|34.38|102.15|429.90|21.13|6.32|136.53| +2452353|54848|6147|39959|1008141|4323|32505|39959|1008141|4323|32505|31|27|5486|18|2174.94|43.49|2218.43|21.37|181.08|1022.22|864.54|288.18|245.94| +2452397|47559|10701|97268|1636397|3526|21455|97268|1636397|3526|21455|57|20|5488|7|317.52|15.87|333.39|6.57|98.63|38.10|248.68|30.74|121.07| +2452430|63755|5089|88794|1711792|5190|47835|88794|1711792|5190|47835|1|12|5490|10|177.10|12.39|189.49|63.72|213.80|37.19|5.59|134.32|289.91| +2451096|50257|8924|18103|30717|6531||||6531|36745|||5492|44|1136.96|0.00|1136.96|7.36|||187.60|187.60|| +2451185|57118|9559|19089|1447359|876|25146|19089|1447359|876|25146|2|33|5492|7|187.46|16.87|204.33|71.43|15.33|118.09|45.09|24.28|103.63| +2451875|72960|15944|39126|944678|3383|6135|39126|944678|3383|6135|55|21|5493|17|884.00|0.00|884.00|30.59|336.77|786.76|64.17|33.07|367.36| +2451859|22389|8563|56194|195351|2668|26065|56194|195351|2668|26065|44|1|5493|7|102.06|6.12|108.18|60.03|43.40|51.03|9.69|41.34|109.55| +2451652|73516|15061|59417|1673715|1966|40664|59417|1673715|1966|40664|56|31|5493|5|186.30|5.58|191.88|7.30|402.10|173.25|11.09|1.96|414.98| +2452333|77217|16293|79133|1312518|6703|18535|79133|1312518|6703|18535|18|4|5494|23|268.64|24.17|292.81|86.76|374.21|96.71|55.01|116.92|485.14| +2452081|77875|13757|54976|1013134|6492|33450|54976|1013134|6492|33450|15|17|5495|9|819.90|16.39|836.29|38.76|380.61|32.79|732.01|55.10|435.76| +2452239|74627|7009|20589|962151|1760|34922|20589|962151|1760|34922|23|6|5495|16|25.28|1.01|26.29|14.45|74.24|21.99|3.15|0.14|89.70| +2451494|57999|9746|68024|1703241|6105|13322|68024|1703241|6105|13322|14|35|5496|9|925.38|18.50|943.88|34.03|43.02|842.09|78.29|5.00|95.55| +2451592|72231|10678|70309|1277838|2967|15696|70309|1277838|2967|15696|28|22|5496|25|67.25|2.01|69.26|26.70|1482.75|13.45|27.43|26.37|1511.46| +2451924|53331|6158|68580|536306|479|36605|68580|536306|479|36605|14|27|5498|12|1011.48|10.11|1021.59|27.90|29.28|940.67|29.03|41.78|67.29| +2452723|49638|16287|72978|1274908|3238|23168|72978|1274908|3238|23168|27|11|5500|7|574.14|45.93|620.07|93.17|34.79|166.50|12.22|395.42|173.89| +2452642|55432|16920|22523|788741|3406|12824|22523|788741|3406|12824|60|16|5500|12|264.84|21.18|286.02|18.17|27.36|66.21|119.17|79.46|66.71| +2452735|39968|8235|47791|497388|5738|6488|47791|497388|5738|6488|1|29|5500|3|109.47|9.85|119.32|13.12|9.93|56.92|31.53|21.02|32.90| +2451318|25526|15860|22048|1486404|713|49469|22048|1486404|713|49469|1|16|5502|71|2574.46|154.46|2728.92|71.64|1444.14|2497.22|75.69|1.55|1670.24| +2451332|31307|14677|60806|1807510|7014|33718|60806|1807510|7014|33718|43|2|5502|24|1380.24|82.81|1463.05|80.04|966.24|165.62|255.07|959.55|1129.09| +2451458|28235|5728||||21792||867686|2739|21792|10||5502|16||||61.30|118.08||||179.38| +2451402|50877|9824|50970|1424238|2753|6821|50970|1424238|2753|6821|58|6|5502|42|283.08|16.98|300.06|71.81|339.78|206.64|55.80|20.64|428.57| +2451473|79870|11072|23772|1522197|5422|4247|23772|1522197|5422|4247|55|2|5502|47|350.62|14.02|364.64|47.99|788.66|234.91|72.89|42.82|850.67| +2450972|51772|5875|23313|1694854|7195|43536|23313|1694854|7195|43536|58|5|5503|14|1109.78|88.78|1198.56|37.87|625.94|388.42|577.08|144.28|752.59| +2451008|76084|9820|47047|1025370|1646|13969|47047|1025370|1646|13969|40|1|5503|58|5799.42|347.96|6147.38|10.95|1670.98|4175.58|600.82|1023.02|2029.89| +2452048|23190|506|3786|171904|3234|24185|3786|171904|3234|24185|56|33|5504|4|73.40|2.20|75.60|66.34|97.88|42.57|28.05|2.78|166.42| +2452514|51152|7044|75036|1041419|6202|22185|75036|1041419|6202|22185|55|15|5505|16|280.48|5.60|286.08|90.89|136.64|171.09|67.82|41.57|233.13| +2452526|59282|5223|27688|330347|5889|6659|27688|330347|5889|6659|30|31|5505|81|7193.61|0.00|7193.61|28.68|3269.97|287.74|4005.40|2900.47|3298.65| +2452645|64440|9451|60722|1770907|2339|45324|60722|1770907|2339|45324|33|16|5506|10|30.40|0.30|30.70|2.51|365.20|18.54|0.59|11.27|368.01| +2451541|79833|14467|4061|72237|6813|44142|4061|72237|6813|44142|28|23|5508|10|51.20|4.60|55.80|62.81|76.80|48.64|1.35|1.21|144.21| +2451742|59351|13639|8306|957631|2603|24328|8306|957631|2603|24328|37|27|5508|30|80.10|7.20|87.30|9.00|23.70|9.61|52.16|18.33|39.90| +2451611|34373|12242|70020|1612154|3088|1622|70020|1612154|3088|1622|20|6|5508|13|1178.32|11.78|1190.10|79.38|490.88|506.67|503.73|167.92|582.04| +|31865|1663||1487024||40449|92688|1487024||40449|58|14|5508|59|||4045.09|50.88|||||| +2452112|63511|8252|42635|84366|4050|40934|42635|84366|4050|40934|47|35|5509|75|2083.50|104.17|2187.67|61.25|2717.25|1458.45|287.52|337.53|2882.67| +2452002|26330|7633|69194|619384|5574|28164|69194|619384|5574|28164|11|26|5509|69|1118.49|67.10|1185.59|70.26|394.68|928.34|121.69|68.46|532.04| +2452196|58315|17955|20717|1062721|5953|46228|20717|1062721|5953|46228|57|9|5510|46|1071.34|64.28|1135.62|64.90|3461.96|846.35|150.74|74.25|3591.14| +2452089|81179|7615|37925|1294777|4429|11281|37925|1294777|4429|11281|31|28|5510|21|2787.12|83.61|2870.73|36.00|2680.02|864.00|1153.87|769.25|2799.63| +2452608|25368|9787|23010|1579619|4581|15590|23010|1579619|4581|15590|12|28|5511|3|99.60|1.99|101.59|11.03|12.15|54.78|13.89|30.93|25.17| +2452189|48397|16771|45470|1621511|6352|41240|45470|1621511|6352|41240|57|10|5514|42|1827.42|109.64|1937.06|32.11|1790.04|1681.22|14.62|131.58|1931.79| +2451701|57202|3530|60505|1451347|2892|13664|60505|1451347|2892|13664|22|21|5516|21|642.60|51.40|694.00|46.39|82.74|366.28|265.26|11.06|180.53| +2451575|67549|1822|38692|1640416|4187|9558|38692|1640416|4187|9558|56|18|5516|14|691.18|13.82|705.00|47.48|735.42|525.29|18.24|147.65|796.72| +2451694|57735|367|27931|300445|4159|30673|27931|300445|4159|30673|37|1|5517|5|124.75|4.99|129.74|81.91|47.45|22.45|51.15|51.15|134.35| +2451718|16680|11398|63012|1024788|5317|15493|63012|1024788|5317|15493|4|13|5517|6|151.92|1.51|153.43|36.69|245.94|77.47|38.71|35.74|284.14| +2451572|74710|1318|56085|117635|5585|49935|56085|117635|5585|49935|44|2|5518|16|826.08|57.82|883.90|28.15|116.16|776.51|47.58|1.99|202.13| +2452826|67435|6979|61654|814479|3751|42576|61654|814479|3751|42576|48|27|5520|7|295.33|17.71|313.04|79.69|105.98|20.67|244.44|30.22|203.38| +2452670|43222|10867|83421|651604|1834|15217|83421|651604|1834|15217|9|12|5520|63|5580.54|334.83|5915.37|77.14|2712.78|4408.62|410.17|761.75|3124.75| +2452837|26652|16933|36480|1350543|2206|34269|36480|1350543|2206|34269|51|12|5521|38|855.38|51.32|906.70|24.02|2410.72|342.15|205.29|307.94|2486.06| +2452731|68820|3015|23175|123113|4298|12236|23175|123113|4298|12236|6|12|5521|1|49.65|0.99|50.64|10.94|50.99|9.93|24.22|15.50|62.92| +2451345|24524|16918|8923|1266235|2747|35306|8923|1266235|2747|35306|38|35|5522|6|83.16|4.98|88.14|68.14|75.36|2.49|47.59|33.08|148.48| +2451139|69542|8773|6521|1630337|4553|31569|6521|1630337|4553|31569|55|35|5523|57|2107.29|189.65|2296.94|80.57|3230.76|863.98|1007.08|236.23|3500.98| +2451005|18874|11596|13192|737772|7152|34485|13192|737772|7152|34485|8|8|5524|7|488.04|14.64|502.68|94.36|102.27|229.37|219.86|38.81|211.27| +2450977|34335|13222|76160|1043555|4197|41992|76160|1043555|4197|41992|50|15|5524|12|1553.16|0.00|1553.16|16.61|115.56|1397.84|69.89|85.43|132.17| +2451242|70466|17179|36704|1393949|7194|22886|36704|1393949|7194|22886|56|8|5525|28|1762.04|158.58|1920.62|8.17|1762.04|193.82|1536.85|31.37|1928.79| +2451834|76542|4237|57111|965589|4467|42145|57111|965589|4467|42145|19|27|5526|43|2061.85|185.56|2247.41|85.79|1867.49|350.51|239.58|1471.76|2138.84| +2451903|62792|13501|45569|157084|1846|27408|45569|157084|1846|27408|29|13|5526|2|59.68|4.77|64.45|91.20|14.32|38.19|5.37|16.12|110.29| +2452642|76742|17700|68860|1738473|4086|30852|68860|1738473|4086|30852|42|3|5529|57|1519.62|91.17|1610.79|15.57|156.18|744.61|193.75|581.26|262.92| +2452623|61027|8502|30833|1470147|3061|13622|30833|1470147|3061|13622|9|8|5529|20|243.40|2.43|245.83|47.36|64.20|92.49|146.38|4.53|113.99| +2452769|72810|12325|6614|689458|4655|7099|6614|689458|4655|7099|25|11|5529|3|289.20|20.24|309.44|69.56|204.12|179.30|7.69|102.21|293.92| +2451727|52967|14785|53131|1329328|4278|43228|53131|1329328|4278|43228|35|3|5530|45|3073.05|215.11|3288.16|54.49|1634.40|3042.31|5.53|25.21|1904.00| +2451778|83971|6833|11455|1489121|2787|32453|11455|1489121|2787|32453|20|33|5531|56|3185.84|286.72|3472.56|14.87|4779.04|318.58|2494.51|372.75|5080.63| +2451871|31472|16637|2406|1357237|1315|16593|2406|1357237|1315|16593|55|4|5531|21|2254.77|157.83|2412.60|18.23|716.10|2006.74|151.29|96.74|892.16| +2452859|69401|727|7390|131845|1100|14355|7390|131845|1100|14355|48|22|5532|29|330.89|9.92|340.81|42.81|355.54|221.69|43.68|65.52|408.27| +2452800|22697|1845|31953|1237755|5510|47265|31953|1237755|5510|47265|18|7|5532|24|1714.32|85.71|1800.03|95.89|1552.56|565.72|183.77|964.83|1734.16| +2452847|68189|3600|90051|946787|323|8259|90051|946787|323|8259|15|22|5532|15|198.90|15.91|214.81|6.29|45.45|39.78|85.92|73.20|67.65| +2451200|47829|17734|59059|1514274|336|27998|59059|1514274|336|27998|44|33|5533|34|2601.68|156.10|2757.78|76.45|3002.20|1925.24|547.91|128.53|3234.75| +2451276|61827|17692|81841|537973|3705|7884|81841|537973|3705|7884|19|10|5533|4|272.92|13.64|286.56|69.57|73.20|21.83|72.81|178.28|156.41| +2451200|79284|11992|43161|150828|6848|34915|43161|150828|6848|34915|20|14|5533|13|434.59|26.07|460.66|4.84|193.05|112.99|321.60|0.00|223.96| +2451683|4040|3673|49515|1351924|5886|42695|49515|1351924|5886|42695|7|18|5535|1|28.77|0.28|29.05|54.13|11.73|23.30|3.44|2.03|66.14| +2452674|13785|2929|62880|330201|3475|42396|62880|330201|3475|42396|12|21|5536|7|142.38|7.11|149.49|90.91|291.55|9.96|72.83|59.59|389.57| +2452683|70358|6930|52133|673991|5002|33260|52133|673991|5002|33260|45|23|5536|12|763.32|0.00|763.32|36.83|258.72|351.12|399.83|12.37|295.55| +2452607|73490|16629|81901|1171105|1556|10360|81901|1171105|1556|10360|39|35|5536|88|5518.48|331.10|5849.58|71.53|4291.76|5077.00|203.08|238.40|4694.39| +2452589|48242|14994|95829|1192411|5346|40328|95829|1192411|5346|40328|54|4|5536|44|0.00|0.00|0.00|79.81|1039.28|0.00|0.00|0.00|1119.09| +2451546|26232|4988|27538|197887|3883|33032|27538|197887|3883|33032|20|9|5538|44|216.92|10.84|227.76|85.11|159.72|28.19|60.39|128.34|255.67| +2451476|5102|15664|13213|1161891|969|34047|13213|1161891|969|34047|46|22|5541|62|4491.90|404.27|4896.17|43.24|4117.42|1078.05|375.52|3038.33|4564.93| +2451182|34648|11912|8699|175414|82|44797|8699|175414|82|44797|25|28|5542|18|132.12|7.92|140.04|7.54|230.04|17.17|66.67|48.28|245.50| +2451393|30373|16153|41757|1577803|7002|25048|41757|1577803|7002|25048|16|13|5542|3|323.46|12.93|336.39|56.60|52.56|84.09|11.96|227.41|122.09| +2452588|45786|7329|72962|870380|6551|49122|72962|870380|6551|49122|39|19|5545|5|172.80|10.36|183.16|89.41|10.15|148.60|22.26|1.94|109.92| +2452587|71990|9549|94153|1835568|983|9198|94153|1835568|983|9198|1|10|5545|8|289.36|2.89|292.25|53.20|439.36|196.76|24.07|68.53|495.45| +2451913|59367|8857|58317|242968|5219|10670|58317|242968|5219|10670|53|6|5546|46|479.32|28.75|508.07|40.47|1159.20|177.34|166.08|135.90|1228.42| +2451783|33257|2287|51649|249937|5320|17057|51649|249937|5320|17057|38|25|5546|6|419.28|4.19|423.47|71.98|0.00|83.85|197.90|137.53|76.17| +2452101||7883||1169560||15547|65390||4930|15547||22|5547|14||||37.65|144.90|572.20|||285.25| +2452207|57141|15327|80304|834112|5967|39822|80304|834112|5967|39822|5|12|5547|4|43.68|2.18|45.86|90.53|166.12|22.27|7.49|13.92|258.83| +2452146|70794|7259|24657|89967|6477|13787|24657|89967|6477|13787|19|19|5547|47|2310.05|161.70|2471.75|1.66|1123.77|554.41|1334.28|421.36|1287.13| +||10630|82989|389356||35737|82989|389356|4170|35737||15|5550|28|3744.44|224.66|||2012.64||||| +2450907|50324|2396|75795|1132319|4843|27767|75795|1132319|4843|27767|50|32|5550|72|10298.88|411.95|10710.83|79.78|2184.48|5252.42|2573.69|2472.77|2676.21| +2452144|70554|14707|26717|235823|2529|10835|26717|235823|2529|10835|13|4|5552|8|1562.48|78.12|1640.60|40.27|791.12|1187.48|127.50|247.50|909.51| +2452426|59253|3235|19331|1346390|1449|6108|19331|1346390|1449|6108|35|34|5553|4|304.96|27.44|332.40|5.18|214.84|219.57|26.47|58.92|247.46| +2452328|54506|6207|8272|1393242|6659|43629|8272|1393242|6659|43629|53|11|5553|23|47.38|0.00|47.38|98.94|616.17|6.15|18.55|22.68|715.11| +2452173|53324|16145|38196|115476|2356|32083|38196|115476|2356|32083|26|35|5554|56|720.16|50.41|770.57|18.17|3169.04|417.69|245.00|57.47|3237.62| +2452048|64318|16265|22139|356514|6089|24464|22139|356514|6089|24464|13|25|5555|47|1075.36|96.78|1172.14|56.45|438.98|225.82|229.37|620.17|592.21| +2452472|40012|9493|42470|785727|3384|37041|42470|785727|3384|37041|24|20|5556|54|1216.08|60.80|1276.88|8.68|369.90|1143.11|56.91|16.06|439.38| +2452598|46769|10759|87464|860324|1184|41197|87464|860324|1184|41197|36|7|5556|51|1850.28|0.00|1850.28|11.37|1497.87|740.11|466.27|643.90|1509.24| +2452676|40508|17769|13347|735275|1556|21437|13347|735275|1556|21437|57|24|5559|10|39.80|0.79|40.59|60.57|139.30|4.77|30.82|4.21|200.66| +2451640|14477|10916|92350|1382280|5457|17434|92350|1382280|5457|17434|40|29|5560|4|34.16|0.34|34.50|60.29|66.84|27.66|2.27|4.23|127.47| +2452288|37859|13585|||||47749|||4687|1||5561|6|169.74||||27.60||||| +2452026|68672|12541|59565|935388|4011|30634|59565|935388|4011|30634|7|18|5563|13|518.96|5.18|524.14|61.49|237.25|202.39|170.94|145.63|303.92| +2451993|73273|14513|88077|96428|1932|7083|88077|96428|1932|7083|14|20|5563|91|7236.32|72.36|7308.68|42.37|3514.42|2966.89|3244.76|1024.67|3629.15| +2452825|80507|1123|34470|433474|2078|39529|34470|433474|2078|39529|60|22|5564|14|1256.22|37.68|1293.90|47.00|300.86|1193.40|50.88|11.94|385.54| +2452734||2493|39115||4489|29129||1231402||29129|||5564||895.58||940.35|51.52|||8.50|36.28|| +2452827|46523|10386|30922|765027|6332|34790|30922|765027|6332|34790|43|16|5564|2|36.96|0.36|37.32|20.07|14.14|18.11|16.39|2.46|34.57| +2452755|72868|15645|8397|672364|4684|17051|8397|672364|4684|17051|18|15|5565|17|647.87|19.43|667.30|63.94|89.25|414.63|44.31|188.93|172.62| +2452922|73576|4302|94046|1421336|5779|40669|94046|1421336|5779|40669|31|5|5565|53|416.58|24.99|441.57|32.25|2917.12|58.32|89.56|268.70|2974.36| +2452430|54549|16053|88754|984983|2012|22594|88754|984983|2012|22594|11|14|5568|40|651.20|32.56|683.76|50.14|410.40|104.19|218.80|328.21|493.10| +2451976|38968|3493|70107|405907|2154|26754|70107|405907|2154|26754|17|8|5570|19|2167.52|86.70|2254.22|75.25|852.34|1473.91|651.99|41.62|1014.29| +2451951|50094|7561|40353|972478|1403|44328|40353|972478|1403|44328|43|16|5570|70|3132.50|187.95|3320.45|58.12|2631.30|1660.22|161.95|1310.33|2877.37| +2451969|72845|6881|46621|26233|3445|48257|46621|26233|3445|48257|17|29|5570|20|672.60|33.63|706.23|71.62|228.80|235.41|332.26|104.93|334.05| +2452692|16783|3133|53748|1078354|4560|25166|53748|1078354|4560|25166|12|2|5572|14|1892.80|151.42|2044.22|72.70|1038.66|1135.68|325.56|431.56|1262.78| +2451322|39574|5462|44722|956518|2181|29199|44722|956518|2181|29199|40|16|5573|23|0.00|0.00|0.00|69.11|117.99|0.00|0.00|0.00|187.10| +2451417|69361|8929|28975|504045|4715|41785|28975|504045|4715|41785|2|8|5573|38|199.50|5.98|205.48|12.15|997.88|1.99|158.00|39.51|1016.01| +2451347|38403|12817|17899|636600|3748|1296|17899|636600|3748|1296|1|23|5573|14|343.84|24.06|367.90|18.27|612.92|82.52|5.22|256.10|655.25| +2452561|40273|7737|95203|1517084|5012|13293|95203|1517084|5012|13293|43|22|5574|52|6195.80|309.79|6505.59|48.28|911.04|5886.01|210.65|99.14|1269.11| +2451823|71480|16679|19758|625349|987|11378|19758|625349|987|11378|11|33|5575|38|2357.52|23.57|2381.09|89.50|883.88|872.28|579.24|906.00|996.95| +2451062|16786|8086|15414|1016040|313|23739|15414|1016040|313|23739|37|26|5576|26|105.04|6.30|111.34|47.66|77.22|22.05|80.50|2.49|131.18| +2451288|45048|4270|92|668202|5559|12069|92|668202|5559|12069|56|26|5576|62|8366.90|585.68|8952.58|65.63|4078.36|6275.17|167.33|1924.40|4729.67| +2452059|44973|13063|70262|1730|6445|9160|70262|1730|6445|9160|35|31|5577|27|3923.64|313.89|4237.53|86.65|1291.68|1883.34|1040.55|999.75|1692.22| +2452373|67435|3913|5209|1661305|6201|21497|5209|1661305|6201|21497|59|18|5578|14|938.42|75.07|1013.49|39.90|169.40|272.14|426.41|239.87|284.37| +2452510|69984|5179|47336|394223|2710|35550|47336|394223|2710|35550|45|35|5579|39|3443.70|34.43|3478.13|35.84|529.62|241.05|960.79|2241.86|599.89| +2451160|30703|8852|16207|453556|6105|21807|16207|453556|6105|21807|22|22|5581|20|600.80|36.04|636.84|38.94|585.00|6.00|113.01|481.79|659.98| +2452310|1976|14789|99326|70629|6732|33235|99326|70629|6732|33235|51|35|5582|22|399.96|31.99|431.95|66.82|312.40|307.96|62.56|29.44|411.21| +2452207|74659|8303|40593|1369714|2835|14803|40593|1369714|2835|14803|41|18|5582|3|455.97|22.79|478.76|39.81|212.76|31.91|364.69|59.37|275.36| +2452140|37537|5947|96373|1155936|5099|45062|96373|1155936|5099|45062|19|34|5583|18|703.80|56.30|760.10|38.22|7.56|267.44|427.63|8.73|102.08| +2452459|56213|153|13203|583069|935|46001|13203|583069|935|46001|45|5|5584|79|9235.89|92.35|9328.24|2.30|354.71|1754.81|4114.59|3366.49|449.36| +||9367|||2155|33537|40897|838726||33537||4|5584|1||3.17|||20.68|81.41|||85.47| +2451272|74370|12844|77588|522014|5193|36019|77588|522014|5193|36019|10|29|5587|1|88.32|1.76|90.08|12.71|46.86|50.34|22.02|15.96|61.33| +2451376|21155|10777|28587|813894|5954|8814|28587|813894|5954|8814|22|31|5587|27|903.69|72.29|975.98|74.74|187.65|587.39|66.42|249.88|334.68| +2451298|28803|8455|38645|656658|1071|2684|38645|656658|1071|2684|46|7|5587|23|3069.12|276.22|3345.34|70.38|39.79|1319.72|594.79|1154.61|386.39| +2451472|58174|1240|16287|1659910|3469|29753|16287|1659910|3469|29753|38|21|5587|4|27.96|1.39|29.35|7.76|61.56|11.18|15.10|1.68|70.71| +2451227|29610|6835|36980|1770501|2775|43849|36980|1770501|2775|43849|58|14|5587|70|436.10|17.44|453.54|33.47|277.20|161.35|236.28|38.47|328.11| +2452128|35311|12197|25284|577257|2383|11956|25284|577257|2383|11956|55|30|5588|7|475.16|4.75|479.91|28.17|401.38|313.60|1.61|159.95|434.30| +2452041|18330|113|83324|347638|2799|37160|83324|347638|2799|37160|44|25|5589|42|1773.66|124.15|1897.81|81.21|46.62|1489.87|124.86|158.93|251.98| +||3919||1198731|3951||91911|1198731|3951||||5590|9||||||78.31|38.11|173.65|400.90| +2451927|73243|11144|6027|1823095|6563|2784|6027|1823095|6563|2784|31|18|5590|70|3451.70|103.55|3555.25|3.61|3835.30|2830.39|118.04|503.27|3942.46| +2451557|29960|9091|47349|29191|6179|22766|47349|29191|6179|22766|34|26|5591|4|17.00|0.17|17.17|56.18|13.60|16.66|0.25|0.09|69.95| +2452591|33300|8541|88228|330313|4751|5636|88228|330313|4751|5636|45|2|5592|19|260.30|10.41|270.71|78.96|205.39|5.20|158.16|96.94|294.76| +2452622|59510|879|96517|1619730|3743|5756|96517|1619730|3743|5756|30|4|5592|53|1279.95|12.79|1292.74|90.51|1279.95|678.37|138.36|463.22|1383.25| +2452509|49384|17051|15237|1118468|2155|15479|15237|1118468|2155|15479|43|2|5594|1|5.62|0.50|6.12|6.58|7.90|2.81|1.15|1.66|14.98| +2452651|79250|9253|87468|1120109|7150|14991|87468|1120109|7150|14991|1|2|5595|3|75.06|2.25|77.31|94.14|3.12|49.53|20.42|5.11|99.51| +2452762|16572|13641|43079|855107|1535|13163|43079|855107|1535|13163|39|14|5595|5|18.95|1.13|20.08|3.67|14.20|9.09|5.91|3.95|19.00| +2451810|13237|5468|7012|336139|1266|49193|7012|336139|1266|49193|32|30|5596|5|737.70|59.01|796.71|97.59|151.55|66.39|208.10|463.21|308.15| +2451826|58913|1072|19114|1693369|3977|31785|19114|1693369|3977|31785|37|32|5596|3|9.33|0.00|9.33|33.33|30.66|6.62|1.89|0.82|63.99| +2451640|78508|16321|2484|696916|2548|24180|2484|696916|2548|24180|46|22|5596|23|632.04|6.32|638.36|3.92|461.15|436.10|70.53|125.41|471.39| +2451570|50301|3928|29022|1367525|3981|10037|29022|1367525|3981|10037|56|7|5596|50|4471.00|134.13|4605.13|80.73|912.50|625.94|346.05|3499.01|1127.36| +2452061|72250|12035|37549|845583|4370|7957|37549|845583|4370|7957|31|1|5597|30|2377.50|190.20|2567.70|61.50|1334.70|2068.42|58.72|250.36|1586.40| +2452060|64264|5768|88461|1447284|6907|7377|88461|1447284|6907|7377|5|20|5597|13|344.50|0.00|344.50|85.45|262.99|68.90|13.78|261.82|348.44| +2452695|72051|5166|70899|309766|633|27147|70899|309766|633|27147|54|1|5598|25|4590.50|91.81|4682.31|78.66|1311.50|2616.58|1796.26|177.66|1481.97| +2452548|63375|17064|58644|606917|343|44616|58644|606917|343|44616|60|23|5598|37|279.72|0.00|279.72|43.61|378.88|27.97|183.77|67.98|422.49| +2451916|64261|11093|12724|574775|6779|45630|12724|574775|6779|45630|7|17|5599|1|35.63|0.00|35.63|53.58|0.71|35.27|0.02|0.34|54.29| +2452257|40804|3433|50877|770441|6005|44369|50877|770441|6005|44369|29|5|5600|71|6789.02|543.12|7332.14|66.29|3633.78|4888.09|1558.76|342.17|4243.19| +||17263|||616|||46152||31527|||5600||122.50|7.35|||43.75||21.59||92.86| +2452292|8318|7149|82331|983409|1287|21965|82331|983409|1287|21965|35|11|5600|85|4245.75|212.28|4458.03|47.58|1929.50|3396.60|297.20|551.95|2189.36| +2451292|19370|776|25447|524820|2146|43519|25447|524820|2146|43519|58|24|5601|16|543.20|0.00|543.20|55.50|318.40|173.82|88.65|280.73|373.90| +2451165|39155|6308|23257|880869|5234|34046|23257|880869|5234|34046|26|15|5601|1|49.62|1.98|51.60|12.70|17.91|23.81|3.87|21.94|32.59| +2452761|18064|13932|99813|1319385|2157|31682|99813|1319385|2157|31682|18|7|5603|24|0.00|0.00|0.00|86.98|17.28|0.00|0.00|0.00|104.26| +2452653|63174|12564|89078|259149|4773|9997|89078|259149|4773|9997|24|5|5603|37|1758.61|158.27|1916.88|30.89|1074.48|105.51|132.24|1520.86|1263.64| +2452552|56308|15109|53089|435361|833|10658|53089|435361|833|10658|18|26|5604|7|324.10|0.00|324.10|54.36|58.17|6.48|22.23|295.39|112.53| +2452112|59863|13105|74853|1463843|3233|42223|74853|1463843|3233|42223|41|28|5605|11|1177.11|23.54|1200.65|85.54|578.93|1012.31|34.60|130.20|688.01| +2452739|40113|8670|79316|952535|954|36573|79316|952535|954|36573|60|14|5606|5|469.45|18.77|488.22|27.56|5.15|258.19|80.27|130.99|51.48| +2452350|49681|10577|29754|1220349|1344|3139|29754|1220349|1344|3139|3|21|5608|1|143.25|11.46|154.71|23.20|18.28|117.46|1.80|23.99|52.94| +||7552||||39471|57774|1287022|||25||5612||13354.74||||||516.82|5225.72|| +2451706|16672|12379|36662|241707|1952|39444|36662|241707|1952|39444|16|13|5613|68|6310.40|315.52|6625.92|77.36|2845.80|6057.98|224.65|27.77|3238.68| +||7906||739418|2370|1675|||2370|1675|||5613|||135.90||95.26|990.99|1579.92|||| +2451198|66536|12601|30039|1542931|5776|668|30039|1542931|5776|668|43|34|5614|24|2494.56|49.89|2544.45|16.67|1191.36|1272.22|1063.43|158.91|1257.92| +2451212|73887|265|73771|1655877|1183|15336|73771|1655877|1183|15336|26|9|5616|3|13.92|1.11|15.03|1.13|139.38|10.30|1.33|2.29|141.62| +2451242|43901|11017|26085|1594038|391|22791|26085|1594038|391|22791|8|18|5616|42|1725.78|69.03|1794.81|75.10|585.06|258.86|484.08|982.84|729.19| +2451638|60697|7285|69960|1522029|5298|32567|69960|1522029|5298|32567|19|13|5617|70|10450.30|418.01|10868.31|43.03|3666.60|4180.12|2821.58|3448.60|4127.64| +2451608|47174|14348|74535|727108|2081|12701|74535|727108|2081|12701|40|14|5617|10|668.90|40.13|709.03|10.67|370.00|628.76|28.90|11.24|420.80| +2451438|26282|3874|31860|1720096|332|29368|31860|1720096|332|29368|2|28|5618|15|844.05|59.08|903.13|82.36|296.85|270.09|298.45|275.51|438.29| +2451408|65000|8324|98889|70287|885|33530|98889|70287|885|33530|56|27|5618|28|236.88|16.58|253.46|41.63|140.28|229.77|0.42|6.69|198.49| +2451543|56600|8054|57588|598479|4170|6308|57588|598479|4170|6308|14|29|5621|42|1584.66|126.77|1711.43|17.71|253.26|475.39|1075.99|33.28|397.74| +2451643|57460|11716|4698|650229|3960|23985|4698|650229|3960|23985|20|16|5621|1|1.80|0.05|1.85|90.86|1.25|1.24|0.04|0.52|92.16| +2451237|76826|121|31918|1743790|5709|21628|31918|1743790|5709|21628|44|13|5623|53|86.92|2.60|89.52|50.52|43.99|40.85|37.31|8.76|97.11| +2451213|25709|7663|94420|685778|6736|32894|94420|685778|6736|32894|13|35|5623|8|477.36|14.32|491.68|42.48|88.08|386.66|43.53|47.17|144.88| +2451253|37130|7591|39209|1782296|6536|39223|39209|1782296|6536|39223|43|1|5623|11|0.99|0.03|1.02|40.40|5.39|0.24|0.28|0.47|45.82| +2452673|45604|3156|36773|308634|2542|37250|36773|308634|2542|37250|57|19|5624|38|3991.14|279.37|4270.51|78.75|177.08|2235.03|368.78|1387.33|535.20| +2452824|39211|7554|25946|653401|2103|33958|25946|653401|2103|33958|60|32|5624|36|143.28|1.43|144.71|48.60|222.12|95.99|16.55|30.74|272.15| +2452747|33423|2001|86704|1674272|1840|16208|86704|1674272|1840|16208|13|3|5624|39|496.08|19.84|515.92|69.93|151.71|0.00|307.56|188.52|241.48| +2452533|17669|9330|19596|453058|3320|5182|19596|453058|3320|5182|39|23|5624|33|439.89|35.19|475.08|28.51|499.95|303.52|77.73|58.64|563.65| +2451066|75817|15436|63886|320181|4476|37171|63886|320181|4476|37171|13|22|5625|2|165.44|11.58|177.02|97.54|30.46|94.30|62.60|8.54|139.58| +2451364|73790|14947|67133|1541280|5827|8374|67133|1541280|5827|8374|26|24|5626|39|271.83|13.59|285.42|68.73|90.48|190.28|37.51|44.04|172.80| +2451970|5335|1157|50012|1716132|652|35972|50012|1716132|652|35972|7|25|5627|12|493.68|39.49|533.17|54.12|145.80|306.08|37.52|150.08|239.41| +2452789|61194|5382|3768|1370650|1532|14414|3768|1370650|1532|14414|9|22|5628|38|187.34|16.86|204.20|10.52|468.54|179.84|5.62|1.88|495.92| +2452832|82713|14773|13594|1650350|3459|16615|13594|1650350|3459|16615|30|4|5628|22|315.48|15.77|331.25|86.97|408.32|145.12|13.62|156.74|511.06| +2452404|60460|2745|36964|1408654|7191|6528|36964|1408654|7191|6528|37|14|5629|41|4020.46|80.40|4100.86|52.37|2261.56|2171.04|203.43|1645.99|2394.33| +2452350|64752|5101|6487|1653940|1393|31293|6487|1653940|1393|31293|29|26|5629|3|18.06|0.54|18.60|91.68|29.25|0.72|3.12|14.22|121.47| +2452104|65341|12536|65154|883784|1614|24291|65154|883784|1614|24291|7|31|5630|1|38.54|0.77|39.31|53.55|3.35|1.15|14.20|23.19|57.67| +2452103|45368|6752|297|68123|5102|34081|297|68123|5102|34081|41|4|5630|13|360.62|3.60|364.22|70.13|90.09|3.60|160.65|196.37|163.82| +2452130|26275|12179|86088|541250|5212|2774|86088|541250|5212|2774|7|16|5630|52|190.32|3.80|194.12|71.69|82.68|146.54|7.00|36.78|158.17| +|35631|5120||1490282|725|23921|57086|1490282|725|23921|10|15|5636|19|15.96||16.59|54.54|||5.56|5.14|| +2451527|40355|17533|83355|1214436|1164|25092|83355|1214436|1164|25092|34|33|5636|36|791.28|15.82|807.10|71.91|276.84|514.33|199.40|77.55|364.57| +2451618|67609|13927|2483|267401|6719|49190|2483|267401|6719|49190|37|21|5636|21|1347.78|80.86|1428.64|33.92|549.99|1118.65|64.15|164.98|664.77| +2451462|31114|7082|5837|484566|542|48871|5837|484566|542|48871|4|21|5639|71|3853.17|308.25|4161.42|12.75|2697.29|2042.18|271.64|1539.35|3018.29| +2452783|38732|11965|17584|379016|5248|327|17584|379016|5248|327|42|8|5640|1|19.27|0.96|20.23|99.91|72.28|15.22|1.41|2.64|173.15| +2451277|65884|9067|79613|354364|5915|33913|79613|354364|5915|33913|32|4|5642|1|75.28|3.01|78.29|49.61|53.77|7.52|56.24|11.52|106.39| +2451092|65808|5528|67417|1021452|7040|7075|67417|1021452|7040|7075|2|24|5645|2|70.98|1.41|72.39|3.02|96.52|9.93|60.43|0.62|100.95| +2452368|78003|7465|28469|1920375|2328|21481|28469|1920375|2328|21481|35|8|5650|5|248.70|0.00|248.70|71.46|331.65|154.19|83.16|11.35|403.11| +|53659|17689|84723||6201|48460||307962|6201|48460|36||5652|7|134.26|2.68||||5.37|74.75|54.14|| +2451324|48797|13294|50437|271321|2986|12335|50437|271321|2986|12335|56|3|5653|23|223.79|17.90|241.69|60.22|134.09|100.70|28.31|94.78|212.21| +2451589|73916|4513|10075|1836119|1010|19054|10075|1836119|1010|19054|37|17|5655|25|1858.25|92.91|1951.16|3.99|1247.25|613.22|522.91|722.12|1344.15| +2451627|8151|2344|33828|1533669|3910|17304|33828|1533669|3910|17304|58|31|5655|26|1821.82|127.52|1949.34|24.39|855.14|236.83|332.84|1252.15|1007.05| +2451530|44978|5956|92800|1246310|1420|29730|92800|1246310|1420|29730|1|13|5655|52|1548.56|0.00|1548.56|34.76|1140.88|1533.07|10.53|4.96|1175.64| +2451636|38340|1198|53253|1847623|5193|6363|53253|1847623|5193|6363|46|30|5656|1|8.45|0.76|9.21|37.21|67.64|7.77|0.59|0.09|105.61| +2451555|1266|14383|42908|803840|7172|14104|42908|803840|7172|14104|55|21|5656|17|633.08|56.97|690.05|96.60|0.00|75.96|384.41|172.71|153.57| +2451746|62386|8572|56208|642311|5661|11364|56208|642311|5661|11364|52|11|5656|15|376.80|0.00|376.80|32.24|354.15|11.30|3.65|361.85|386.39| +2451751|76831|6988|13639|987445|6498|29104|13639|987445|6498|29104|20|21|5656|39|381.81|0.00|381.81|46.82|39.78|125.99|145.81|110.01|86.60| +2452039|29138|12824|80620|1299979|3691|5403|80620|1299979|3691|5403|29|16|5658|8|456.40|41.07|497.47|28.56|53.68|191.68|39.70|225.02|123.31| +2452037|37908|3745|40588|1434104|67|14430|40588|1434104|67|14430|25|29|5658|18|2330.82|46.61|2377.43|67.89|1054.98|1584.95|193.92|551.95|1169.48| +2452425|10356|16933|11828|1895677|679|42928|11828|1895677|679|42928|3|6|5659|4|294.08|26.46|320.54|69.03|193.24|117.63|135.86|40.59|288.73| +2452489|26121|1215|27233|1106896|4819|1428|27233|1106896|4819|1428|9|28|5660|32|3266.88|130.67|3397.55|47.11|45.76|1992.79|624.30|649.79|223.54| +2451217|29072|13702|42647|1646388|82|10623|42647|1646388|82|10623|50|11|5662|84|12390.84|371.72|12762.56|10.25|4383.96|9788.76|1014.81|1587.27|4765.93| +2451525|21918|5005|31275|1727392|2709|14379|31275|1727392|2709|14379|43|9|5663|1|121.66|8.51|130.17|84.32|25.22|0.00|13.38|108.28|118.05| +2451408|14718|17236|56726|1105619|1359|29452|56726|1105619|1359|29452|4|7|5663|91|1897.35|75.89|1973.24|72.82|1167.53|322.54|188.97|1385.84|1316.24| +2452767|62828|4671|39676|432207|1185|31316|39676|432207|1185|31316|48|28|5664|1|12.74|0.63|13.37|51.81|10.92|4.96|7.54|0.24|63.36| +2451566|12448|8474|41594|34769|6791|44413|41594|34769|6791|44413|32|13|5666|62|1033.54|0.00|1033.54|84.61|3101.24|475.42|290.22|267.90|3185.85| +2451151|28981|17114|68999|309738|6030|18369|68999|309738|6030|18369|13|32|5667|30|31.80|0.95|32.75|76.97|256.50|31.80|0.00|0.00|334.42| +2451796|39614|4927|99321|504616|3158|14853|99321|504616|3158|14853|31|7|5668|4|23.40|2.10|25.50|90.04|9.00|20.59|1.48|1.33|101.14| +2451936|50392|9317|87789|1091724|4459|4041|87789|1091724|4459|4041|32|35|5669|48|872.16|61.05|933.21|18.09|2243.04|279.09|545.62|47.45|2322.18| +2452097|32554|15659|68238|747530|4422|23361|68238|747530|4422|23361|47|10|5669|41|1864.68|55.94|1920.62|29.73|1214.01|484.81|41.39|1338.48|1299.68| +2452371|55612|15963|31759|1496339|2166|12746|31759|1496339|2166|12746|47|10|5670|2|123.04|8.61|131.65|52.67|69.92|93.51|23.32|6.21|131.20| +2452332|78029|1857|38210|800964|6471|34025|38210|800964|6471|34025|55|13|5670|10|74.10|0.74|74.84|39.38|16.00|32.60|12.45|29.05|56.12| +2452383|56264|16857|95498|264314|683|37899|95498|264314|683|37899|1|6|5670|27|1755.54|105.33|1860.87|4.29|444.96|614.43|125.52|1015.59|554.58| +2452095|70436|15056|1779|1135797|3062|13080|1779|1135797|3062|13080|59|11|5671|56|90.16|4.50|94.66|11.79|32.48|60.40|25.89|3.87|48.77| +2451940|57648|13958|82362|1502790|5787|1557|82362|1502790|5787|1557|13|14|5671|27|1874.34|37.48|1911.82|70.15|823.77|1312.03|449.84|112.47|931.40| +2452040|20839|275|4845|1469152|2706|15013|4845|1469152|2706|15013|32|32|5671|3|3.42|0.00|3.42|99.45|5.97|2.18|0.35|0.89|105.42| +2451643|75189|7166|6631|1593164|3697|20230|6631|1593164|3697|20230|55|5|5672|10|1010.60|90.95|1101.55|30.56|383.70|141.48|825.66|43.46|505.21| +2452905|74811|1185|3481|763557|2833|39309|3481|763557|2833|39309|37|30|5673|26|28.86|1.73|30.59|91.54|9.62|26.26|1.30|1.30|102.89| +2452711|43422|13471|47361|1197159|1357|15602|47361|1197159|1357|15602|55|9|5673|2|155.94|3.11|159.05|83.32|1.68|90.44|48.47|17.03|88.11| +2452330|43588|14073|43082|271598|4914|47477|43082|271598|4914|47477|27|25|5674|75|1670.25|83.51|1753.76|54.70|0.00|1586.73|48.44|35.08|138.21| +2452286|71138|3863|85621|1575104|2987|47270|85621|1575104|2987|47270|39|33|5675|61|3636.21|109.08|3745.29|87.06|2389.37|2908.96|36.36|690.89|2585.51| +2452073|86296|11483|21188|1186504|5867|36383|21188|1186504|5867|36383|8|20|5677|9|106.02|9.54|115.56|28.55|50.13|92.23|4.55|9.24|88.22| +2451914|63432|13934|4188|1109918|448|6578|4188|1109918|448|6578|49|28|5677|1|2.68|0.02|2.70|48.84|2.41|1.84|0.77|0.07|51.27| +2452027|15248|5119|38581|312156|2084|1259|38581|312156|2084|1259|59|1|5677|7|61.88|0.00|61.88|47.62|112.35|55.69|0.24|5.95|159.97| +2452096|71478|7769|57651|63344|5390|17836|57651|63344|5390|17836|8|31|5677|39|375.96|0.00|375.96|86.60|347.10|90.23|142.86|142.87|433.70| +2451767|17579|1922|39480|407072|1064|35056|39480|407072|1064|35056|13|11|5678|80|681.60|40.89|722.49|27.28|6544.80|218.11|375.42|88.07|6612.97| +2451872|65412|10087|23850|542891|3979|23467|23850|542891|3979|23467|19|4|5678|33|0.00|0.00|0.00|7.10|976.47|0.00|0.00|0.00|983.57| +2452115|71082|13469|86403|102183|2763|30143|86403|102183|2763|30143|59|13|5679|3|254.19|0.00|254.19|7.53|50.28|119.46|74.10|60.63|57.81| +2452273|78793|12213|75999|1437269|2892|33715|75999|1437269|2892|33715|55|24|5680|45|647.10|6.47|653.57|61.43|1126.80|19.41|100.43|527.26|1194.70| +2452887|64209|6307|27262|1317669|2923|38530|27262|1317669|2923|38530|42|23|5681|47|124.55|1.24|125.79|34.62|2.82|39.85|36.42|48.28|38.68| +2451814|29035|15284|66233|96670|5954|40391|66233|96670|5954|40391|11|29|5682|45|6786.00|339.30|7125.30|27.60|4989.60|5428.80|719.31|637.89|5356.50| +2451507|61915|3562|59216|181355|7047|47539|59216|181355|7047|47539|22|18|5684|25|806.25|32.25|838.50|30.35|0.00|169.31|89.17|547.77|62.60| +2451618|72819|6973|6770|1595713|4656|14799|6770|1595713|4656|14799|10|32|5684|49|7753.27|465.19|8218.46|1.47|2237.83|3411.43|260.51|4081.33|2704.49| +2451841|33472|17149|35312|425994|6581|19611|35312|425994|6581|19611|23|22|5685|16|1332.64|106.61|1439.25|89.33|1102.88|1132.74|37.98|161.92|1298.82| +2452071|3820|6745|93314|1151541|4968|12795|93314|1151541|4968|12795|26|15|5686|47|5392.78|107.85|5500.63|66.37|437.10|3828.87|93.83|1470.08|611.32| +2451811|67002|15484|4102|1046640|825|43410|4102|1046640|825|43410|20|26|5687|10|307.80|18.46|326.26|11.54|140.20|289.33|6.83|11.64|170.20| +2451711|42489|16196|82989|645331|4401|41551|82989|645331|4401|41551|7|16|5687|31|2608.03|182.56|2790.59|63.49|1872.40|1773.46|175.25|659.32|2118.45| +2451747||4918|||3427|||1113836|||31||5687||894.75||939.48|28.47|732.00||||| +2451592|34445|4856|14252|1240118|1184|15450|14252|1240118|1184|15450|32|19|5688|16|575.84|0.00|575.84|89.05|1151.84|460.67|69.10|46.07|1240.89| +2451695|65969|4729|28739|869854|2616|17331|28739|869854|2616|17331|43|20|5688|68|5279.52|52.79|5332.31|80.01|794.24|422.36|3302.86|1554.30|927.04| +2451734|59928|6818|68831|1478415|2046|43277|68831|1478415|2046|43277|14|8|5688|28|1386.00|41.58|1427.58|66.92|1900.64|1178.10|91.47|116.43|2009.14| +|26925|10843|||3634|48675|14799|1212240|3634|48675|||5689||||141.30||77.18||56.79|37.87|| +2451339|74898|10723|58480|982510|6987|29417|58480|982510|6987|29417|32|8|5689|83|2284.99|182.79|2467.78|4.23|2924.92|708.34|788.32|788.33|3111.94| +2451467|71321|8972|86577|350131|6932|16933|86577|350131|6932|16933|37|17|5689|8|35.84|0.71|36.55|3.51|179.36|18.27|11.06|6.51|183.58| +2451682|58984|241|73123|1645992|5261|30130|73123|1645992|5261|30130|46|27|5690|11|208.01|8.32|216.33|17.59|44.66|191.36|13.81|2.84|70.57| +2452566|35057|5017|61871|1381964|104|39397|61871|1381964|104|39397|30|14|5692|8|97.92|5.87|103.79|3.92|293.76|82.25|6.42|9.25|303.55| +2451821|36022|15218|45640|1445833|5059|27045|45640|1445833|5059|27045|31|4|5693|53|5399.11|0.00|5399.11|84.39|1893.16|3401.43|499.42|1498.26|1977.55| +2451927|71093|6263|14815|567183|3551|3919|14815|567183|3551|3919|26|6|5694|6|97.92|0.97|98.89|8.36|2.28|30.35|47.97|19.60|11.61| +2451967|77820|6092|62015|1511792|5569|41346|62015|1511792|5569|41346|44|10|5695|41|191.47|17.23|208.70|39.44|84.05|32.54|100.12|58.81|140.72| +2451235|65879|5626|22401|1422004|3207|46363|22401|1422004|3207|46363|20|32|5697|7|157.78|7.88|165.66|76.33|439.67|39.44|71.00|47.34|523.88| +2452014|71460|10255|10015|1506552|6642|19179|10015|1506552|6642|19179|31|35|5698|14|237.58|19.00|256.58|71.21|90.02|192.43|32.50|12.65|180.23| +2451851|58992|11987|24600|1535223|1046|23669|24600|1535223|1046|23669|29|8|5698|33|5114.67|358.02|5472.69|40.00|397.65|1738.98|3038.12|337.57|795.67| +2451582|82329|7034|30761|667988|5215|42442|30761|667988|5215|42442|46|4|5699|31|2373.98|118.69|2492.67|9.12|314.03|1780.48|356.10|237.40|441.84| +||8606||282197||12465|64616||4931||||5699|38|490.58|||||0.00||299.26|| +2451728|38736|4387|38902|422704|106|8308|38902|422704|106|8308|25|22|5702|90|4745.70|189.82|4935.52|9.36|582.30|1708.45|2065.33|971.92|781.48| +2451604|69794|5708|4778|703511|5797|11730|4778|703511|5797|11730|19|24|5702|23|6.44|0.32|6.76|90.06|25.30|4.95|0.05|1.44|115.68| +2451831|31212|1256|44193|224181|2922|12910|44193|224181|2922|12910|2|26|5703|2|40.90|3.27|44.17|43.78|7.78|39.26|0.22|1.42|54.83| +2451423|29817|11365|38424|1917972|720|26268|38424|1917972|720|26268|32|16|5704|4|99.28|5.95|105.23|89.39|0.00|55.59|7.42|36.27|95.34| +2451409|68973|7246|81200|1013175|3103|4631|81200|1013175|3103|4631|34|34|5704|1|142.18|8.53|150.71|79.73|103.03|12.79|81.51|47.88|191.29| +2452630|52292|17731|28450|265063|6923|1936|28450|265063|6923|1936|48|18|5705|18|1096.74|0.00|1096.74|28.49|926.46|153.54|405.57|537.63|954.95| +2451228|58537|5011|54496|1300824|3505|10633|54496|1300824|3505|10633|44|5|5706|75|2912.25|58.24|2970.49|40.54|295.50|2271.55|205.02|435.68|394.28| +2452442|73602|16245|6592|1025097|7155|4460|6592|1025097|7155|4460|33|29|5707|10|1685.00|0.00|1685.00|14.05|825.30|454.95|246.01|984.04|839.35| +|17020|6385|22447||317|37652||661782|317||||5711|14|659.68||712.45|79.32|508.90||441.52||640.99| +2452299|43991|747|24656|833768|4833|7545|24656|833768|4833|7545|41|6|5711|13|581.62|5.81|587.43|30.43|249.21|343.15|4.76|233.71|285.45| +2452462|55508|7683|88314|634555|1223|48181|88314|634555|1223|48181|11|31|5711|2|60.08|5.40|65.48|0.80|69.48|51.06|7.75|1.27|75.68| +2452468|27125|553|91721|1523034|1994|22907|91721|1523034|1994|22907|9|17|5711|52|9950.20|796.01|10746.21|41.63|4102.80|6865.63|2313.42|771.15|4940.44| +2451915||5852|50328|||35408|50328|||35408|38|4|5713|6||10.63|162.55|||95.70||12.94|| +2451989|42183|13121|68691|90459|5643|48973|68691|90459|5643|48973|29|27|5713|78|4984.20|448.57|5432.77|40.11|387.66|1046.68|3189.39|748.13|876.34| +2451858|77679|991|95675|570713|1398|14436|95675|570713|1398|14436|55|29|5713|11|60.39|4.22|64.61|48.31|80.52|34.42|0.77|25.20|133.05| +2451309|67240|17036|78031|1636604|3977|1750|78031|1636604|3977|1750|32|4|5714|13|527.80|26.39|554.19|17.96|12.09|21.11|385.08|121.61|56.44| +2451069|69732|16904|44815|131059|4856|21853|44815|131059|4856|21853|58|16|5714|59|285.56|0.00|285.56|21.24|404.74|265.57|16.99|3.00|425.98| +2451391|30101|13558|18082|227871|3707|26343|18082|227871|3707|26343|16|25|5715|9|34.02|2.04|36.06|78.56|229.86|28.57|3.70|1.75|310.46| +2452721|32950|17409|83726|1547115|2857|37846|83726|1547115|2857|37846|43|9|5716|19|1489.60|74.48|1564.08|11.83|138.51|0.00|670.32|819.28|224.82| +2452608|76608|1315|19249|565277|3305|31733|19249|565277|3305|31733|39|16|5716|57|2561.01|0.00|2561.01|89.53|1517.91|742.69|1491.02|327.30|1607.44| +2452664|70852|16194|40721|1313671|4221|21571|40721|1313671|4221|21571|54|30|5717|39|2799.42|195.95|2995.37|34.34|3253.38|1175.75|1169.04|454.63|3483.67| +2452770|68665|7110|20293|1228971|4496|42374|20293|1228971|4496|42374|39|16|5717|25|1552.00|139.68|1691.68|17.68|689.75|341.44|1186.34|24.22|847.11| +2451809|47425|8921|33989|1849681|4220|41912|33989|1849681|4220|41912|32|25|5718|26|18.98|1.13|20.11|55.13|626.86|6.64|8.76|3.58|683.12| +2452284|49364|8367|21285|1307487|4889|49444|21285|1307487|4889|49444|3|9|5719|5|13.85|0.96|14.81|65.19|11.75|6.92|4.43|2.50|77.90| +2452741|72763|11617|44849|1199808|720|25279|44849|1199808|720|25279|30|26|5720|4|49.96|0.00|49.96|35.87|110.64|0.99|33.29|15.68|146.51| +2451310|55621|15980|87818|1227911|2861|547|87818|1227911|2861|547|34|35|5721|3|30.66|2.75|33.41|93.11|0.00|3.98|16.27|10.41|95.86| +2451703|1662|2263|42847|785508|969|44030|42847|785508|969|44030|50|11|5722|4|12.92|0.51|13.43|56.80|45.32|7.23|4.55|1.14|102.63| +2451352|27199|5491|88995|294121|5647|29220|88995|294121|5647|29220|25|11|5726|52|1801.28|54.03|1855.31|92.51|1309.88|270.19|780.85|750.24|1456.42| +2451288|84833|12964|34613|295540|3667|20594|34613|295540|3667|20594|56|9|5726|5|358.55|3.58|362.13|89.49|0.00|286.84|57.36|14.35|93.07| +2451428|19922|9266|39583|1718125|833|21462|39583|1718125|833|21462|1|1|5726|46|2496.88|149.81|2646.69|4.74|1317.44|1148.56|1078.65|269.67|1471.99| +2451045|64747|12392|79304|1691990|5123|35409|79304|1691990|5123|35409|1|10|5729|3|77.64|0.00|77.64|7.92|134.88|44.25|13.68|19.71|142.80| +2451562|29360|799|79432|570982|5285|15588|79432|570982|5285|15588|1|20|5730|21|374.64|0.00|374.64|22.98|13.86|59.94|34.61|280.09|36.84| +2452261|21624|12737|21178|514395|3612|45182|21178|514395|3612|45182|59|15|5731|37|2223.33|88.93|2312.26|95.27|0.00|1289.53|112.05|821.75|184.20| +2452378|72211|13875|63143|853393|7196|49446|63143|853393|7196|49446|39|7|5733|18|708.12|0.00|708.12|58.08|731.70|283.24|246.43|178.45|789.78| +2451967|20167|17990|43256|1499551|2246|39341|43256|1499551|2246|39341|1|19|5734|31|3117.36|124.69|3242.05|19.62|3919.02|1028.72|62.65|2025.99|4063.33| +2451840|62910|11069|8527|169305|6883|39661|8527|169305|6883|39661|5|4|5734|12|1716.12|0.00|1716.12|75.27|346.68|1698.95|3.77|13.40|421.95| +2452361|35364|15381|14979|175857|6335|27213|14979|175857|6335|27213|23|22|5735|40|40.00|3.60|43.60|74.67|56.00|10.40|0.88|28.72|134.27| +2451215|46284|10106|38642|1113827|1459|37091|38642|1113827|1459|37091|46|34|5736|18|2791.98|251.27|3043.25|77.13|997.02|1060.95|380.82|1350.21|1325.42| +2451130|40491|10742|95605|845308|258|29961|95605|845308|258|29961|26|1|5736|43|1402.23|70.11|1472.34|74.60|1498.98|560.89|328.12|513.22|1643.69| +2451173|32709|730|97968|305298|6228|40358|97968|305298|6228|40358|55|32|5739|66|543.18|0.00|543.18|67.45|199.98|141.22|305.48|96.48|267.43| +2452012|18439|10718|53063|565642|268|14094|53063|565642|268|14094|5|4|5741|11|530.86|31.85|562.71|76.87|232.21|440.61|65.88|24.37|340.93| +2451862|23914|10585|30234|||48844|||5574|48844|||5743|17|||343.25|||271.33|17.78|37.80|98.42| +2451822|69630|5713|1136|153218|6900|1382|1136|153218|6900|1382|25|13|5744|6|582.60|11.65|594.25|91.67|120.54|361.21|179.32|42.07|223.86| +2451589|26548|17564|29311|675889|5314|9092|29311|675889|5314|9092|46|9|5745|1|127.99|11.51|139.50|63.38|30.02|95.99|11.84|20.16|104.91| +2451757|1730|7334|55206|273827|4095|30316|55206|273827|4095|30316|47|11|5746|5|173.20|12.12|185.32|97.17|0.00|136.82|36.38|0.00|109.29| +|28983|6169|29088|576159|6077|8748|||||||5747||1646.00|98.76|||814.40|1036.98||176.62|| +2452702|65629|11931|35167|151380|2440|37853|35167|151380|2440|37853|48|28|5747|58|3341.96|33.41|3375.37|50.14|164.72|467.87|2615.42|258.67|248.27| +2452065|42226|15089|61114|466387|6238|34025|61114|466387|6238|34025|15|32|5748|96|9870.72|296.12|10166.84|75.17|508.80|6317.26|1456.91|2096.55|880.09| +2451979|44230|14275|72807|1291533|4212|26938|72807|1291533|4212|26938|29|20|5748|4|12.00|0.84|12.84|84.60|7.20|6.36|0.62|5.02|92.64| +2452056|56020|7555|40971|696462|1906|47750|40971|696462|1906|47750|47|8|5748|10|415.30|0.00|415.30|75.12|190.30|290.71|114.62|9.97|265.42| +2451455|59786|14146|32140|1366674|408|2142|32140|1366674|408|2142|20|19|5749|1|12.56|1.13|13.69|8.77|1.37|8.03|1.63|2.90|11.27| +2451347||5551||1377332|||20367||||14||5749|11||1.05|22.17|||||6.47|| +2452098|6855|2531|73960|867315|1470|47225|73960|867315|1470|47225|57|30|5750|7|542.43|48.81|591.24|52.91|681.52|124.75|183.77|233.91|783.24| +2452220|62727|11047|41876|1704384|2892|27000|41876|1704384|2892|27000|7|8|5750|71|234.30|14.05|248.35|90.25|2464.41|53.88|146.14|34.28|2568.71| +2452173|26452|3117|95331|411852|1173|43294|95331|411852|1173|43294|41|28|5750|17|1779.90|0.00|1779.90|49.37|686.46|106.79|167.31|1505.80|735.83| +2452172|70308|1555|31420|1783125|6477|48582|31420|1783125|6477|48582|17|18|5750|11|1117.16|44.68|1161.84|21.92|97.68|413.34|302.64|401.18|164.28| +2452271|10401|11995|21564|980258|2599|43293|21564|980258|2599|43293|9|33|5751|15|0.00|0.00|0.00|60.36|25.95|0.00|0.00|0.00|86.31| +2451332|26092|8876|6109|1480218|6327|10389|6109|1480218|6327|10389|56|28|5752|19|249.28|12.46|261.74|59.03|116.09|22.43|108.88|117.97|187.58| +2452294|33303|13493|33989|726044|1103|37088|33989|726044|1103|37088|11|13|5754|73|2522.88|227.05|2749.93|51.67|2601.72|479.34|1512.21|531.33|2880.44| +2452401|74175|4649|50793|1753955|5917|43479|50793|1753955|5917|43479|37|33|5754|36|1342.80|26.85|1369.65|85.42|863.28|711.68|271.38|359.74|975.55| +2451223|84026|16945|89400|1054160|873|7186|89400|1054160|873|7186|56|8|5756|37|1150.70|23.01|1173.71|95.70|852.48|1150.70|0.00|0.00|971.19| +2452199|59989|49|56584|1742421|7105|44942|56584|1742421|7105|44942|33|18|5757|6|17.22|1.03|18.25|48.13|60.30|7.57|0.19|9.46|109.46| +2452492|42268|16709|93479|792881|5883|1638|93479|792881|5883|1638|17|10|5758|34|491.64|39.33|530.97|41.17|93.84|442.47|20.65|28.52|174.34| +2452489|67132|16287|58618|571587|4909|17078|58618|571587|4909|17078|35|2|5758|52|276.12|5.52|281.64|28.60|3453.84|207.09|58.67|10.36|3487.96| +2451175|80988|4036|7366|791392|3418|28244|7366|791392|3418|28244|16|25|5759|76|95.76|6.70|102.46|85.93|34.20|32.55|48.03|15.18|126.83| +2452813|72670|13614|93826|1127457|1088|37919|93826|1127457|1088|37919|49|5|5762|28|966.84|77.34|1044.18|15.09|836.08|280.38|6.86|679.60|928.51| +2452225|35153|11491|44625|1682528|6478|49228|44625|1682528|6478|49228|21|24|5763|5|3.65|0.29|3.94|17.28|5.85|2.73|0.25|0.67|23.42| +2452172|80169|3909|37895|93444|4351|12306|37895|93444|4351|12306|35|24|5763|20|212.20|14.85|227.05|36.43|16.20|174.00|15.66|22.54|67.48| +2452733|44094|11101|30818|1401006|1594|11382|30818|1401006|1594|11382|24|33|5764|5|118.50|10.66|129.16|99.22|14.80|72.28|2.31|43.91|124.68| +2451986|43305|12341|65976|414117|658|43999|65976|414117|658|43999|11|25|5765|39|2614.17|130.70|2744.87|65.23|274.95|2352.75|49.66|211.76|470.88| +2451322|47931|17335|28556|1867945|1287|5076|28556|1867945|1287|5076|31|25|5766|12|1596.12|47.88|1644.00|97.15|542.04|1516.31|75.81|4.00|687.07| +2451311|82381|7018|10138|153459|4780|36901|10138|153459|4780|36901|55|27|5766|21|1493.94|134.45|1628.39|9.59|139.44|209.15|423.98|860.81|283.48| +2451906|64136|17065|85810|422129|6895|10014|85810|422129|6895|10014|44|17|5767|76|1973.72|157.89|2131.61|79.39|877.04|1381.60|550.67|41.45|1114.32| +2451101|36858|76|54285|1547521|6357|42793|54285|1547521|6357|42793|43|26|5769|70|1415.40|127.38|1542.78|10.96|0.00|452.92|192.49|769.99|138.34| +2451146|34885|3086||1038832||||1038832|||||5769|44|1275.12||1338.87||456.72|688.56|||614.41| +2452689|62899|5737|29211|324015|2019|10423|29211|324015|2019|10423|45|25|5770|8|515.60|46.40|562.00|6.01|133.68|335.14|54.13|126.33|186.09| +2452132|42285|2207|10079|438296|466|4683|10079|438296|466|4683|43|16|5771|14|1936.62|174.29|2110.91|55.44|456.96|968.31|745.59|222.72|686.69| +2452154|66318|4335|58355|189673|5791|4709|58355|189673|5791|4709|41|18|5771|34|71.06|3.55|74.61|42.62|284.92|13.50|32.23|25.33|331.09| +2452556|13413|9789|60137|1879866|1995|49324|60137|1879866|1995|49324|31|20|5772|15|305.70|0.00|305.70|65.41|1100.55|103.93|127.11|74.66|1165.96| +2452045|31964|10217|60458|1368567|4676|36385|60458|1368567|4676|36385|29|21|5773|5|187.00|16.83|203.83|53.85|55.70|181.39|3.92|1.69|126.38| +2452058|15710|12497|36877|772008|3139|25823|36877|772008|3139|25823|32|27|5773|44|1719.96|51.59|1771.55|29.75|278.52|395.59|781.37|543.00|359.86| +2452066|36715|7735|84562|856759|6556|7953|84562|856759|6556|7953|2|20|5773|11|699.60|41.97|741.57|80.84|0.00|55.96|70.80|572.84|122.81| +2451309|69857|4936|13602|961054|251|23551|13602|961054|251|23551|26|32|5776|13|1439.75|115.18|1554.93|90.05|332.15|921.44|342.08|176.23|537.38| +2451316|32265|6604|15976|462840|1257|34874|15976|462840|1257|34874|10|17|5776|61|2924.34|175.46|3099.80|95.45|46.36|1813.09|166.68|944.57|317.27| +2451285|79152|2450|50495|848828|1114|17047|50495|848828|1114|17047|13|2|5776|22|543.84|16.31|560.15|10.71|478.50|522.08|15.44|6.32|505.52| +2452646|8727|5017|70036|1813178|673|47015|70036|1813178|673|47015|21|4|5778|20|860.40|77.43|937.83|75.76|563.20|808.77|32.52|19.11|716.39| +2452598|3146|1435|45795|1545929|6863|38150|45795|1545929|6863|38150|1|7|5778|2|147.90|4.43|152.33|38.55|40.32|65.07|9.93|72.90|83.30| +2452666|58949|5277|33928|31705|5835|7565|33928|31705|5835|7565|51|27|5780|22|465.74|4.65|470.39|17.70|284.46|125.74|200.60|139.40|306.81| +||9738|65012|1039753||8882||1039753|94||27|13|5781||||||||||| +2452720|73648|1735|42342|357392|2042|12292|42342|357392|2042|12292|7|20|5782|51|33.15|1.32|34.47|96.12|1362.21|2.32|10.79|20.04|1459.65| +2452789|27667|3595|65649|859961|4320|43879|65649|859961|4320|43879|37|31|5782|42|3206.70|192.40|3399.10|29.45|852.18|2244.69|606.06|355.95|1074.03| +2452767|76075|691|86536|1565923|3263|49403|86536|1565923|3263|49403|57|17|5782|59|207.68|0.00|207.68|70.25|312.11|74.76|103.67|29.25|382.36| +2452223|70622|10205|27173|1496841|||27173|1496841|||||5783|72||21.25||81.18|708.48|170.03||226.15|| +2452170|42081|4065|24584|488910|6036|3962|24584|488910|6036|3962|53|30|5783|9|132.93|6.64|139.57|93.04|40.86|19.93|32.77|80.23|140.54| +2451787|28050|17942|48211|936987|163|25739|48211|936987|163|25739|32|33|5786|19|105.83|4.23|110.06|46.62|979.07|42.33|24.76|38.74|1029.92| +2451698|10238|4027|94377|637849|521|11193|94377|637849|521|11193|56|35|5788|35|2300.90|138.05|2438.95|89.57|172.90|1472.57|115.96|712.37|400.52| +|58113|4438|90113|615687|108|||615687|108|929||23|5789||106.85|2.13||32.62|22.05|25.64|55.22||56.80| +2451115|36021|1777|5503|801664|2625|2739|5503|801664|2625|2739|25|21|5789|30|802.20|32.08|834.28|68.83|154.20|657.80|67.86|76.54|255.11| +2451258|64991|12542|68551|1179187|6623|9706|68551|1179187|6623|9706|7|7|5789|28|2345.84|93.83|2439.67|50.80|409.08|867.96|458.14|1019.74|553.71| +2452611|6872|4362|93626|1612534|595|22656|93626|1612534|595|22656|7|1|5790|24|2702.40|0.00|2702.40|19.07|803.28|2080.84|93.23|528.33|822.35| +2452850|26211|1206|5401|6982|2963|30944|5401|6982|2963|30944|48|10|5790|48|289.92|23.19|313.11|4.55|1039.20|55.08|232.49|2.35|1066.94| +2452810|68895|17697|10027|||1004||1531686||1004|||5790|30|1112.40|44.49||32.55||433.83|346.07||| +2451975|34532|5747|97865|1756739|4060|20934|97865|1756739|4060|20934|50|9|5791|1|94.25|1.88|96.13|28.37|17.89|67.86|3.16|23.23|48.14| +2451957|48661|13861|22567|1390045|3123|15001|22567|1390045|3123|15001|38|14|5791|43|60.20|0.00|60.20|83.48|340.99|53.57|1.79|4.84|424.47| +2451327|36537|11116|59293|693911|3282|19853|59293|693911|3282|19853|58|30|5792|18|139.50|0.00|139.50|52.33|362.70|39.06|96.42|4.02|415.03| +2451340|85668|6811|11458|869935|2812|39289|11458|869935|2812|39289|2|16|5792|18|935.82|28.07|963.89|99.71|493.74|93.58|741.17|101.07|621.52| +2451452|59944|3574||1766884|6993||60375|1766884||33502||22|5794||968.52|58.11|1026.63|40.67|595.84||154.57||694.62| +2451328|17800|8852|40099|1552899|5699|6099|40099|1552899|5699|6099|43|4|5794|9|193.05|7.72|200.77|77.07|79.92|48.26|124.51|20.28|164.71| +2451278|46308|10666|45372|34187|6202|41017|45372|34187|6202|41017|40|24|5794|4|195.80|9.79|205.59|72.71|154.60|125.31|12.68|57.81|237.10| +2451398|67951|17260|22718|563308|4698|16135|22718|563308|4698|16135|8|27|5794|12|188.52|13.19|201.71|7.88|196.32|131.96|6.78|49.78|217.39| +2452107|73187|6463|63810|1343055|4347|17992|63810|1343055|4347|17992|20|28|5796|5|419.20|37.72|456.92|24.03|112.25|205.40|44.89|168.91|174.00| +2452051|42046|3179|33299|1121854|4456|1411|33299|1121854|4456|1411|25|14|5796|15|2090.25|41.80|2132.05|71.19|1148.55|1233.24|797.01|60.00|1261.54| +2451571|71647|17539|88271|1181980|3371|48390|88271|1181980|3371|48390|32|29|5797|8|10.48|0.83|11.31|41.20|3.44|9.11|0.27|1.10|45.47| +2451467|37250|13546|61851|877946|2841|40536|61851|877946|2841|40536|49|33|5797|41|5.33|0.15|5.48|89.94|62.32|3.83|0.24|1.26|152.41| +2451569|43076|17731|24405|124898|2991|29084|24405|124898|2991|29084|25|15|5798|16|460.96|13.82|474.78|17.97|125.60|304.23|147.32|9.41|157.39| +2451543|20556|3922|53903|201275|1447|17219|53903|201275|1447|17219|4|32|5798|42|621.18|49.69|670.87|17.75|194.88|80.75|270.21|270.22|262.32| +2451892||14357||1908939||35318|12126|1908939||35318|||5799|3|10.05|0.80|10.85||||||| +|36616|12774||569322||||569322|5937|6105|30||5800|15|987.75|59.26|1047.01||0.00|325.95||52.95|| +2452696|65449|15072|88759|921216|73|38930|88759|921216|73|38930|31|20|5800|24|1025.28|20.50|1045.78|61.20|78.00|717.69|212.23|95.36|159.70| +2452463|78126|7685|93993|452378|1516|39101|93993|452378|1516|39101|25|35|5801|3|120.63|6.03|126.66|80.43|13.38|25.33|13.34|81.96|99.84| +2452178|46708|16783|21223|114949|5828|38707|21223|114949|5828|38707|5|8|5802|69|164.91|4.94|169.85|69.99|45.54|97.29|24.34|43.28|120.47| +2451352|74523|10036|79624|104204|2831|9011|79624|104204|2831|9011|14|30|5803|20|312.00|0.00|312.00|87.95|398.80|221.52|69.66|20.82|486.75| +2451438|3050|1844|50664|1746225|1339|14100|50664|1746225|1339|14100|32|5|5803|62|5617.82|280.89|5898.71|25.52|2395.68|4831.32|291.00|495.50|2702.09| +2452233|78391|10941|83244|412830|2416|39602|83244|412830|2416|39602|7|6|5804|15|288.30|23.06|311.36|8.95|381.45|43.24|98.02|147.04|413.46| +2452420|25353|15271|62354|105973|1945|26717|62354|105973|1945|26717|29|3|5804|2|52.58|3.15|55.73|10.76|36.48|50.47|1.39|0.72|50.39| +2451501|40542|20|85755|354698|88|8231|85755|354698|88|8231|7|28|5805|35|835.45|0.00|835.45|20.40|512.05|467.85|316.13|51.47|532.45| +2451792|52922|15517|20013|702004|1595|29539|20013|702004|1595|29539|49|28|5805|36|498.24|39.85|538.09|93.31|398.52|343.78|109.66|44.80|531.68| +2451704|68907|11632|94627||1970|17418|94627|694075|1970|17418|||5805||314.49|28.30|342.79|77.84|234.63|||96.59|340.77| +2451392|74262|3100|41597|392930|3016|7149|41597|392930|3016|7149|31|29|5806|27|27.54|0.27|27.81|15.45|13.77|23.68|2.20|1.66|29.49| +2451106|69545|859|2856|480167|4678|37236|2856|480167|4678|37236|22|11|5807|41|608.44|30.42|638.86|29.95|743.74|498.92|89.80|19.72|804.11| +2452430|5608|17658|45360|1724980|600|36620|45360|1724980|600|36620|27|4|5808|35|185.50|3.71|189.21|53.32|90.30|163.24|5.11|17.15|147.33| +2452553|80569|11040|17905|783111|764|4770|17905|783111|764|4770|9|28|5808|31|2160.70|0.00|2160.70|47.60|1200.32|280.89|1353.46|526.35|1247.92| +2452372|73224|11341|13768||3753||13768||3753|44969||19|5808||||||17.04||53.89|71.45|| +2451465|38203|13708|23680|1273364|6695|46426|23680|1273364|6695|46426|40|35|5810|28|440.72|0.00|440.72|17.66|1212.40|334.94|38.08|67.70|1230.06| +2451635|70003|646|22786|1587620|1607|16394|22786|1587620|1607|16394|37|28|5810|21|282.45|22.59|305.04|33.08|250.11|62.13|136.59|83.73|305.78| +2451131|0|5749|97386|1307098|810|42522|97386|1307098|810|42522|2|3|5811|25|1816.00|127.12|1943.12|14.07|688.00|980.64|735.11|100.25|829.19| +2451051|41934|4033|73384|561654|3877|46147|73384|561654|3877|46147|34|16|5811|82|2283.70|137.02|2420.72|8.64|2045.90|525.25|281.35|1477.10|2191.56| +2451521|64749|8012|28517|708791|6665|2186|28517|708791|6665|2186|34|14|5812|3|341.58|23.91|365.49|27.45|113.85|78.56|52.60|210.42|165.21| +2451578|24365|6007|22432|1162016|800|21984|22432|1162016|800|21984|14|1|5814|9|442.44|22.12|464.56|59.68|78.93|305.28|134.41|2.75|160.73| +2451357|40825|16544|88577|208389|6154|6366|88577|208389|6154|6366|1|27|5814|25|517.50|46.57|564.07|50.43|528.25|108.67|339.32|69.51|625.25| +2451438|73692|3008|57634|1485491|4995|32852|57634|1485491|4995|32852|38|3|5814|14|153.72|4.61|158.33|34.96|49.56|84.54|65.72|3.46|89.13| +2451946|77758|3577|56865|767864|463|46262|56865|767864|463|46262|7|9|5816|7|258.51|7.75|266.26|51.73|106.82|235.24|13.03|10.24|166.30| +2451510|58386|13291||||||479054||4921|1|8|5817|36|2546.64|||38.98|||||2518.30| +2452119|39869|11561|53935|256153|5124|17342|53935|256153|5124|17342|32|22|5818|8|1116.08|44.64|1160.72|16.39|433.28|223.21|151.78|741.09|494.31| +2452012|62941|499|90126|528385|3797|44809|90126|528385|3797|44809|53|5|5818|41|4291.06|343.28|4634.34|52.62|1589.16|2660.45|766.38|864.23|1985.06| +2451279|75340|860|73974|693979|3333|4513|73974|693979|3333|4513|25|21|5820|30|693.60|41.61|735.21|41.79|224.10|436.96|2.56|254.08|307.50| +2451154|48628|5074|28|1010339|1450|38961|28|1010339|1450|38961|43|12|5820|75|4857.00|48.57|4905.57|35.07|416.25|4322.73|470.15|64.12|499.89| +2452536|28875|510|71474|515403|1732|30014|71474|515403|1732|30014|13|26|5821|26|0.00|0.00|0.00|78.52|2748.98|0.00|0.00|0.00|2827.50| +2452342|72998|1830|19921|1193384|5247|9110|19921|1193384|5247|9110|9|10|5821|23|921.84|55.31|977.15|9.98|1547.44|507.01|128.59|286.24|1612.73| +2452525|80046|10267|94540|635803|1866|38729|94540|635803|1866|38729|60|29|5821|89|537.56|21.50|559.06|91.73|867.75|365.54|106.65|65.37|980.98| +2452670|41011|3951|57367|1581709|6498|8887|57367|1581709|6498|8887|55|22|5822|10|241.90|14.51|256.41|98.40|131.60|217.71|22.73|1.46|244.51| +2452528|45335|9121|32366|1069555|3991|8027|32366|1069555|3991|8027|19|12|5822|9|239.22|9.56|248.78|14.57|132.93|193.76|29.54|15.92|157.06| +2452599|1979|2982|72188|232342|4496|16603|72188|232342|4496|16603|3|30|5823|63|1431.99|100.23|1532.22|31.45|1372.14|701.67|29.21|701.11|1503.82| +2451805|57227|8041|70416|624048|2127|3759|70416|624048|2127|3759|31|27|5824|30|724.80|36.24|761.04|27.62|186.30|594.33|36.53|93.94|250.16| +2451101|29959|5545|56813|1279335|6657|20779|56813|1279335|6657|20779|58|17|5825|1|45.01|2.70|47.71|25.88|111.36|15.30|14.85|14.86|139.94| +2451117|44417|7339|38800|1826903|1406|10262|38800|1826903|1406|10262|44|31|5825|33|242.88|4.85|247.73|57.88|2430.78|218.59|4.85|19.44|2493.51| +2451753||4360||||46387|||6679|||1|5826|67|9826.89|||87.58|2141.32|3537.68|1006.27|5282.94|2327.16| +2451652|62940|1204|64338|447719|347|23728|64338|447719|347|23728|46|21|5826|27|419.04|29.33|448.37|40.79|67.50|138.28|196.53|84.23|137.62| +2451722|72299|6826|16559|473803|7047|12738|16559|473803|7047|12738|44|14|5826|10|47.10|2.82|49.92|3.59|44.90|15.07|21.78|10.25|51.31| +2452279|14616|8187|11677|1301059|488|45070|11677|1301059|488|45070|5|24|5830|22|2100.56|147.03|2247.59|42.13|1417.90|2079.55|16.38|4.63|1607.06| +2452474|40445|9765|5424|659905|7186|30114|5424|659905|7186|30114|43|32|5831|69|480.24|43.22|523.46|49.49|641.01|393.79|7.78|78.67|733.72| +2451898|17628|6860|92145|1742558|1771|20584|92145|1742558|1771|20584|55|8|5832|1|37.53|3.00|40.53|56.34|40.53|17.63|0.39|19.51|99.87| +2451986|54891|7303|46457|905387|2270|7953|46457|905387|2270|7953|43|35|5833|49|9530.99|667.16|10198.15|5.61|1058.89|1620.26|6012.15|1898.58|1731.66| +2451044|80813|8467|97226|500063|5300|23337|97226|500063|5300|23337|14|20|5834|17|1812.37|144.98|1957.35|40.19|757.01|1304.90|111.64|395.83|942.18| +2450906|66172|10180|65891|631357|23|27442|65891|631357|23|27442|8|21|5834|8|197.68|7.90|205.58|14.00|265.52|53.37|4.32|139.99|287.42| +2452186|32081|10829|4230|1761168|506|3091|4230|1761168|506|3091|23|13|5835|8|32.64|0.00|32.64|30.13|95.76|3.59|26.43|2.62|125.89| +2452224|43117|5959|63183|1412286|1836|11339|63183|1412286|1836|11339|15|15|5835|15|244.95|22.04|266.99|45.53|13.20|164.11|43.65|37.19|80.77| +|30502|4258|92020|1366698||40548|||6623|40548|46||5837||931.80||931.80|8.10|||||427.90| +2451224|40135|17722|98726|597351|776|16648|98726|597351|776|16648|4|10|5837|18|128.34|1.28|129.62|18.92|167.76|21.81|52.19|54.34|187.96| +2451964|55483|683|16531|236562|6546|35377|16531|236562|6546|35377|31|14|5838|50|35.00|1.05|36.05|62.46|1547.00|8.05|4.31|22.64|1610.51| +2452720|64126|10158|85246|416583|3920|20010|85246|416583|3920|20010|45|23|5840|17|664.36|0.00|664.36|43.60|114.41|272.38|31.35|360.63|158.01| +2452822|38852|10591|46617|1355571|5029|17569|46617|1355571|5029|17569|6|1|5840|15|1435.35|100.47|1535.82|30.98|346.35|1263.10|172.25|0.00|477.80| +2452720|12076|12282|60017|448040|4063|14886|60017|448040|4063|14886|24|21|5840|57|6349.80|317.49|6667.29|66.99|3457.05|825.47|4032.76|1491.57|3841.53| +2452773|44492|5934|98919|325520|5368|23616|98919|325520|5368|23616|12|35|5840|76|633.08|37.98|671.06|85.49|2896.36|405.17|25.07|202.84|3019.83| +2451959|57124|5495|96503|65065|1627|35789|96503|65065|1627|35789|20|13|5842|56|786.24|39.31|825.55|65.60|540.40|314.49|438.72|33.03|645.31| +2451661|47714|6872|24185|162904|2423|30342|24185|162904|2423|30342|35|21|5845|2|1.62|0.11|1.73|64.25|3.70|1.23|0.08|0.31|68.06| +2451705|48390|6551|5757|961634|1864|1741|5757|961634|1864|1741|56|22|5845|93|19213.80|1729.24|20943.04|60.00|7280.97|2305.65|9468.56|7439.59|9070.21| +2451715|52128|7967|81046|1867566|2086|34253|81046|1867566|2086|34253|25|13|5845|60|11262.00|225.24|11487.24|68.68|6596.40|10811.52|315.33|135.15|6890.32| +2451292||3710||1231418|345||33244|1231418|345|16119||7|5847||177.12||178.89|54.40|543.18|||80.49|599.35| +2452590|45593|5556|23209|1002627|2398|30216|23209|1002627|2398|30216|13|1|5849|11|15.51|0.15|15.66|45.03|165.88|7.91|1.52|6.08|211.06| +2452406|30064|14449|25521|151183|1645|46507|25521|151183|1645|46507|33|19|5849|36|805.32|56.37|861.69|96.16|334.08|684.52|47.11|73.69|486.61| +||5191|60467||||60467|||18806|5|17|5850|79|417.91|8.35|||136.67|351.04|40.79|26.08|| +2452851|16358|10957|42474|76802|6777|29105|42474|76802|6777|29105|48|32|5851|85|2319.65|23.19|2342.84|92.82|1020.00|927.86|751.56|640.23|1136.01| +2452658|71229|1791|22780|178097|5414|32331|22780|178097|5414|32331|21|12|5852|31|816.54|8.16|824.70|20.90|366.42|718.55|73.49|24.50|395.48| +2452666|49534|13719|1474|450960|4799|14638|1474|450960|4799|14638|54|9|5852|11|281.05|2.81|283.86|15.65|145.86|157.38|76.67|47.00|164.32| +2452249|69385|10807|81878|208988|4098|20455|81878|208988|4098|20455|55|34|5853|81|1296.00|12.96|1308.96|70.49|4862.43|751.68|468.11|76.21|4945.88| +2452363|21465|17503|72898|682981|1864|9100|72898|682981|1864|9100|35|26|5853|35|5260.15|263.00|5523.15|9.53|452.20|3261.29|1998.86|0.00|724.73| +2452400|76895|14313|21276|253553|5830|32107|21276|253553|5830|32107|43|29|5853|15|3170.40|63.40|3233.80|7.00|1585.20|1902.24|431.17|836.99|1655.60| +2451890|43184|13568|26734|1319231|6575|7472|26734|1319231|6575|7472|43|9|5855|7|445.13|26.70|471.83|61.75|58.66|44.51|208.32|192.30|147.11| +2451589|39387|12901|65163|1865526|1889|252|65163|1865526|1889|252|10|21|5856|6|552.54|22.10|574.64|32.72|124.50|547.01|5.36|0.17|179.32| +2451663|70483|7369|8020|1275597|4421|43645|8020|1275597|4421|43645|22|28|5856|31|3823.54|305.88|4129.42|22.99|1579.14|267.64|817.85|2738.05|1908.01| +2452269|34780|2221|54429|979989|6349|23909|54429|979989|6349|23909|19|16|5858|36|510.12|40.80|550.92|56.35|534.60|249.95|78.05|182.12|631.75| +2451650|4522|10789|50439||1073|20437||1364556|1073|20437|4|21|5859|||||||40.60||17.66|| +2451599|63923|817|17459|265283|2260|4484|17459|265283|2260|4484|1|6|5859|17|361.93|28.95|390.88|86.12|133.11|285.92|64.60|11.41|248.18| +2451571|28828|8236|93349|1377174|999|26471|93349|1377174|999|26471|46|25|5859|19|527.06|42.16|569.22|7.15|306.66|73.78|72.52|380.76|355.97| +2451717|52430|6193|22835|627092|5396|23253|22835|627092|5396|23253|31|1|5860|5|830.40|0.00|830.40|90.94|106.45|631.10|89.68|109.62|197.39| +2451772|57310|12313|62770|544065|2960|20302|62770|544065|2960|20302|10|26|5862|4|646.44|6.46|652.90|75.88|300.36|452.50|36.84|157.10|382.70| +2452374|34911|16505|53643|782890|490|17737|53643|782890|490|17737|11|13|5863|15|70.35|4.92|75.27|78.64|1.05|50.65|17.73|1.97|84.61| +|31296|8151|57985|1110166||28434|57985|1110166|5100|28434|45|14|5863|||||26.53|||142.21|21.26|| +2451192|47149|5896|53297|418711|6172|22386|53297|418711|6172|22386|55|22|5864|47|1103.09|77.21|1180.30|56.34|385.87|1103.09|0.00|0.00|519.42| +|17523|6298||1168248|670|45139||1168248|670|45139|16|8|5866|67|1017.73|30.53||||915.95|47.83||92.99| +2452633|44411|11718|6475|1563443|4411|6626|6475|1563443|4411|6626|54|18|5867|33|1338.15|0.00|1338.15|73.30|494.01|735.98|463.67|138.50|567.31| +2452656|67386|13098|78060|1540852|5691|10883|78060|1540852|5691|10883|51|17|5867|13|174.33|6.97|181.30|42.13|229.45|108.08|39.75|26.50|278.55| +2451026|66235|3514|13420|913242|5064|17286|13420|913242|5064|17286|26|8|5868|44|2354.00|70.62|2424.62|25.64|604.12|541.42|725.03|1087.55|700.38| +2451098|46253|3320|69297|960313|4576|4392|69297|960313|4576|4392|25|35|5871|6|508.62|40.68|549.30|98.89|279.48|305.17|75.27|128.18|419.05| +2451447|14701|9346|87983|964762|3141|18367|87983|964762|3141|18367|55|5|5875|2|1.44|0.04|1.48|28.94|9.42|1.15|0.03|0.26|38.40| +2451400|20883|6350|25788|855185|6982|24379|25788|855185|6982|24379|19|4|5877|3|245.04|19.60|264.64|91.63|21.99|215.63|16.76|12.65|133.22| +2451688|39352|9682|88093|731584|1922|44213|88093|731584|1922|44213|16|17|5879|34|6380.78|319.03|6699.81|41.28|2233.12|5678.89|357.96|343.93|2593.43| +2451765|74755|14167|25142|545151|4360|10816|25142|545151|4360|10816|58|19|5879|6|32.04|1.28|33.32|61.15|360.48|8.01|0.96|23.07|422.91| +2451587|45909|12721|88093|731584|1922|44213|88093|731584|1922|44213|13|20|5879|10|353.20|17.66|370.86|12.35|65.70|314.34|8.16|30.70|95.71| +2451734|60432|6902|61702|330036|6017|31551|61702|330036|6017|31551|31|19|5879|1|69.26|5.54|74.80|2.61|21.70|3.46|28.95|36.85|29.85| +2450897|71414|7447|5838|1339649|5907|22118|5838|1339649|5907|22118|40|13|5880|48|165.60|14.90|180.50|41.87|1259.04|48.02|10.58|107.00|1315.81| +2452434|26663|15089|6158|1804325|892|19859|6158|1804325|892|19859|45|6|5881|16|1540.16|92.40|1632.56|5.84|982.08|754.67|39.27|746.22|1080.32| +2452344|43326|4989|20468|1732975|5482|17589|20468|1732975|5482|17589|51|31|5881|3|87.84|6.14|93.98|7.33|17.07|12.29|66.48|9.07|30.54| +2452316|8775|14275|7045|528035|5210|42319|7045|528035|5210|42319|3|2|5881|36|310.68|15.53|326.21|13.07|224.64|304.46|6.09|0.13|253.24| +2452604|27068|9723|44892|1356797|4349|19054|44892|1356797|4349|19054|9|23|5882|1|129.03|9.03|138.06|46.80|79.14|20.64|30.34|78.05|134.97| +2452533|70636|14088|29064|346677|6690|11150|29064|346677|6690|11150|60|13|5882|41|67.24|4.70|71.94|19.34|457.56|16.81|28.74|21.69|481.60| +2451514|66067|16282|||6486|28430|56645|1603742||28430||5|5883||1421.37|||||824.39|316.39||336.43| +2451441|21135|11090|92016|1783933|6250|44567|92016|1783933|6250|44567|8|29|5883|31|1660.36|83.01|1743.37|78.93|142.60|1212.06|130.00|318.30|304.54| +2452267|39070|5005|64933|1287265|1950|4682|64933|1287265|1950|4682|11|1|5884|5|405.05|4.05|409.10|1.39|108.00|40.50|240.60|123.95|113.44| +2452450|58821|8905|11713|329089|1665|30241|11713|329089|1665|30241|39|7|5884|71|2737.05|0.00|2737.05|3.70|2169.05|1724.34|293.68|719.03|2172.75| +2451285|34751|9889|50738|1871235|2132|44629|50738|1871235|2132|44629|44|9|5885|78|1390.74|97.35|1488.09|29.54|864.24|13.90|137.68|1239.16|991.13| +2451880|71298|15881|23640|323690|6587|12237|23640|323690|6587|12237|17|5|5886|16|203.68|16.29|219.97|81.06|136.80|46.84|53.32|103.52|234.15| +2451541|6542|2900|99154|232387|6773|23606|99154|232387|6773|23606|22|21|5888|48|6558.24|393.49|6951.73|18.52|946.56|3213.53|2207.50|1137.21|1358.57| +2451557|72531|4190|67497|584579|5661|37943|67497|584579|5661|37943|55|22|5888|10|23.60|0.94|24.54|61.53|212.80|17.70|2.71|3.19|275.27| +2451619|41643|11360|55867|826237|4254|21424|55867|826237|4254|21424|2|33|5888|52|1450.80|14.50|1465.30|92.59|900.12|116.06|1067.79|266.95|1007.21| +2452323|28877|6993|12838|861157|6629|45748|12838|861157|6629|45748|59|10|5889|24|0.00|0.00|0.00|73.87|898.32|0.00|0.00|0.00|972.19| +2452478|61327|1575|21840|1913334|6781|3028|21840|1913334|6781|3028|18|24|5890|14|987.42|69.11|1056.53|71.07|793.52|493.71|266.60|227.11|933.70| +2452105|22590|6797|96408|1255472|1472|49481|96408|1255472|1472|49481|17|29|5893|15|1831.05|109.86|1940.91|45.38|835.80|769.04|435.42|626.59|991.04| +2452620|68770|13752|36018|853715|212|31156|36018|853715|212|31156|27|2|5894|3|366.12|10.98|377.10|85.46|24.39|318.52|36.65|10.95|120.83| +2452634|6736|6309|84300|1305779|6458|33137|84300|1305779|6458|33137|39|6|5894|70|1385.30|0.00|1385.30|36.14|1582.70|1288.32|67.88|29.10|1618.84| +2451735|81069|6469|58476|1801293|2863|2480|58476|1801293|2863|2480|37|31|5896|3|276.48|24.88|301.36|6.15|3.84|27.64|154.28|94.56|34.87| +|60374|14449|||1950||39586|944768|||7|15|5898||2009.52|120.57|2130.09|48.86|1445.04||676.00||| +2452683|57362|11433|58004||2181|||848269|2181||||5899|5|788.90||836.23|38.45|165.10||15.78||250.88| +2452683|43560|16284|47612|1329561|663|3365|47612|1329561|663|3365|36|8|5899|70|72.80|3.64|76.44|46.65|193.90|10.19|46.95|15.66|244.19| +2451019|20438|14059|29148|709005|2462|7919|29148|709005|2462|7919|37|25|5900|13|123.63|9.89|133.52|53.71|39.13|108.79|11.42|3.42|102.73| +2450928|43496|3325|16646|1783301|4356|18211|16646|1783301|4356|18211|14|14|5900|3|2.19|0.13|2.32|22.26|2.01|1.29|0.10|0.80|24.40| +2451112|62345|158|46173|1254245|6764|39283|46173|1254245|6764|39283|22|21|5900|2|94.78|5.68|100.46|1.76|194.82|94.78|0.00|0.00|202.26| +2452372|3925|13673|86730|1386980|2783|9833|86730|1386980|2783|9833|41|2|5901|41|1332.09|0.00|1332.09|31.80|1880.67|386.30|274.27|671.52|1912.47| +2451430|56023|1480|65333|1236716|899|46282|65333|1236716|899|46282|10|13|5902|11|189.75|5.69|195.44|55.27|271.15|45.54|70.66|73.55|332.11| +2451600|30960|812|79159|1590746|4733|35473|79159|1590746|4733|35473|25|31|5903|38|4005.96|40.05|4046.01|32.88|235.60|560.83|2273.78|1171.35|308.53| +2451484|71386|16195|51000|124281|6009|46490|51000|124281|6009|46490|25|35|5903|10|325.80|13.03|338.83|1.27|149.10|6.51|70.24|249.05|163.40| +2451454|53187|6878|95241|578081|5274|21667|95241|578081|5274|21667|1|22|5903|12|38.16|3.05|41.21|15.39|19.08|29.00|4.03|5.13|37.52| +2451599|32182|6524|42398|501935|5025|1362|42398|501935|5025|1362|55|8|5903|47|2096.20|83.84|2180.04|32.45|419.24|293.46|414.63|1388.11|535.53| +2451572|61732|17491|57062|1522709|356|26450|57062|1522709|356|26450|13|24|5903|54|7624.26|304.97|7929.23|29.97|2964.60|2516.00|153.24|4955.02|3299.54| +2451745|63845|8824|99069|900725|1134|38390|99069|900725|1134|38390|25|2|5904|54|97.74|6.84|104.58|12.14|166.86|22.48|69.23|6.03|185.84| +2451750|69191|17885|21497|874265|3386|154|21497|874265|3386|154|32|28|5905|26|3709.68|111.29|3820.97|31.61|1785.94|667.74|973.42|2068.52|1928.84| +2451963|75180|10177|33325|1544153|6981|28609|33325|1544153|6981|28609|38|30|5906|12|793.08|7.93|801.01|61.39|594.84|626.53|119.91|46.64|664.16| +2451894|40364|16331|65631|1677|4123|46616|65631|1677|4123|46616|56|4|5906|71|6067.66|485.41|6553.07|15.38|478.54|1820.29|3567.79|679.58|979.33| +2452270|79917|14363|19325|780160|6619|13629|19325|780160|6619|13629|17|7|5907|37|880.60|70.44|951.04|48.63|361.49|493.13|189.86|197.61|480.56| +2451556|75271|11545|62672|306524|4329|27781|62672|306524|4329|27781|1|27|5908|5|236.20|18.89|255.09|3.85|107.00|210.21|0.77|25.22|129.74| +2451414|43336|8851|36808|686223|2838|13853|36808|686223|2838|13853|8|26|5909|3|171.36|1.71|173.07|5.78|126.78|17.13|60.14|94.09|134.27| +2452851|49212|5397|68526|1089700|5274|6448|68526|1089700|5274|6448|45|35|5910|16|1841.44|18.41|1859.85|18.13|411.84|497.18|120.98|1223.28|448.38| +2452822|62822|13309|64472|1760789|3953|26717|64472|1760789|3953|26717|37|19|5910|35|578.20|46.25|624.45|52.14|1423.80|57.82|20.81|499.57|1522.19| +2452126|36656|8324|63568|632902|7036|3617|63568|632902|7036|3617|47|21|5911|4|150.28|12.02|162.30|18.06|64.64|30.05|102.19|18.04|94.72| +2452086|45060|10111|63646|1790567|4241|32362|63646|1790567|4241|32362|23|25|5911|39|5127.72|0.00|5127.72|40.11|820.17|358.94|190.75|4578.03|860.28| +2452344|82517|10855|65473|202018|3566|2620|65473|202018|3566|2620|39|33|5912|85|733.55|22.00|755.55|69.67|3212.15|212.72|276.03|244.80|3303.82| +||7443|81542|||17328|||568||49||5915|||||||300.48|1002.61||| +|49391|4724||1712654|1255|37903|84406|||||24|5916||0.00|0.00|||||||| +2451174|60541|14996|39203|217177|751|16725|39203|217177|751|16725|8|34|5916|10|579.00|40.53|619.53|8.34|505.10|266.34|143.82|168.84|553.97| +2451291|65527|17947|4781|325673|3812|10704|4781|325673|3812|10704|4|25|5916|28|20.44|0.20|20.64|62.73|847.84|16.35|2.86|1.23|910.77| +2451750|67738|12805|23862|1807594|3326|1105|23862|1807594|3326|1105|8|24|5918|53|2264.69|181.17|2445.86|70.45|357.22|2128.80|97.84|38.05|608.84| +2452734|79939|6105|74249|1347798|1794|943|74249|1347798|1794|943|24|1|5919|80|4878.40|341.48|5219.88|20.69|1806.40|4439.34|87.81|351.25|2168.57| +2451642|14799|9964|87994|1770479|3717|43402|87994|1770479|3717|43402|32|13|5920|66|289.74|11.58|301.32|43.75|105.60|275.25|10.86|3.63|160.93| +2451951|42175|12734|76233|826527|5268|25562|76233|826527|5268|25562|32|8|5921|7|88.90|6.22|95.12|72.79|38.08|74.67|7.96|6.27|117.09| +2452935|45072|16483|93514|776878|1617|11866|93514|776878|1617|11866|48|23|5923|8|165.12|1.65|166.77|65.16|35.84|151.91|7.26|5.95|102.65| +2452499|33853|12215|78279|646742|4466|4314|78279|646742|4466|4314|17|8|5924|28|1153.60|23.07|1176.67|58.54|548.24|680.62|245.94|227.04|629.85| +2452368|26343|1707|3801|1670467|7086|19352|3801|1670467|7086|19352|3|21|5924|12|268.08|13.40|281.48|29.58|119.88|58.97|14.63|194.48|162.86| +2452864|16966|8898|94787|41374|1208|8995|94787|41374|1208|8995|54|31|5925|33|2325.51|69.76|2395.27|30.52|1219.68|1186.01|364.64|774.86|1319.96| +2450969|77649|11918|15933|864548|4363|9425|15933|864548|4363|9425|26|20|5926|66|1989.24|179.03|2168.27|75.46|67.98|954.83|351.69|682.72|322.47| +2451126|60687|6508|60396|1728297|4998|21299|60396|1728297|4998|21299|8|9|5926|15|1653.75|49.61|1703.36|30.88|501.60|264.60|291.72|1097.43|582.09| +2452243|24036|5115|85128|1787275|2222|1569|85128|1787275|2222|1569|47|18|5927|38|3741.10|224.46|3965.56|32.69|1558.76|1608.67|85.29|2047.14|1815.91| +2452184|61022|13371|46543|1261024|5905|11586|46543|1261024|5905|11586|29|19|5927|2|5.98|0.17|6.15|94.84|55.88|3.04|2.61|0.33|150.89| +2452703|68682|2265|36247|1111435|4223|26211|36247|1111435|4223|26211|48|24|5928|5|498.65|0.00|498.65|86.75|353.20|388.94|41.68|68.03|439.95| +2452603|39176|8827|57288|1904467|5599|19978|57288|1904467|5599|19978|25|11|5928|28|4241.44|84.82|4326.26|71.52|1113.28|3308.32|503.88|429.24|1269.62| +2452547|68708|9907|73269|892334|3846|36511|73269|892334|3846|36511|13|29|5929|5|297.70|11.90|309.60|93.42|40.70|181.59|81.27|34.84|146.02| +2452452|56782|5751|12517|1215000|4683|13166|12517|1215000|4683|13166|30||5929||321.88|||11.89|186.42|241.41||79.67|| +2452300|63504|10733|69793|695495|4750|3652|69793|695495|4750|3652|35|5|5931|36|1658.88|149.29|1808.17|91.73|715.32|630.37|596.53|431.98|956.34| +2451253|40540|14401|70058|505198|6385|49572|70058|505198|6385|49572|34|25|5932|7|56.00|4.48|60.48|10.89|22.96|32.48|4.46|19.06|38.33| +2451282|32983|12620|91574|274087|5847|29560|91574|274087|5847|29560|2|27|5932|6|206.28|6.18|212.46|98.30|14.70|18.56|82.59|105.13|119.18| +2451334|49245|284|60887|1749893|3699|12752|60887|1749893|3699|12752|10|22|5935|13|671.84|26.87|698.71|70.72|515.06|53.74|166.88|451.22|612.65| +2451349|65689|4490|47821|1475594|1320|35446|47821|1475594|1320|35446|55|21|5936|8|21.52|1.07|22.59|9.83|8.08|12.26|6.20|3.06|18.98| +2451246|71060|9556|46141|387180|5019|44458|46141|387180|5019|44458|14|27|5936|19|628.71|44.00|672.71|84.56|100.89|320.64|46.21|261.86|229.45| +2451319|70835|3514|76474|1508271|2468|38665|76474|1508271|2468|38665|25|27|5936|2|174.54|6.98|181.52|63.31|162.36|34.90|89.36|50.28|232.65| +2452327|82621|13281|77483|1165749|3262|4728|77483|1165749|3262|4728|53|25|5937|17|299.88|17.99|317.87|58.72|1281.46|164.93|101.21|33.74|1358.17| +2452304|21151|12429|56697|39712|7174|16189|56697|39712|7174|16189|19|15|5937|5|193.30|0.00|193.30|75.12|116.70|92.78|97.50|3.02|191.82| +2451607|50575|11656|16262|605146|5677|5761|16262|605146|5677|5761|44|27|5938|45|10395.90|831.67|11227.57|15.70|324.45|6237.54|1039.59|3118.77|1171.82| +2452304|57247|8271|89377|1104008|1486|16564|89377|1104008|1486|16564|49|6|5940|21|559.02|27.95|586.97|87.14|583.38|329.82|208.57|20.63|698.47| +2452644|39053|14971|35809|1402694|4779|46911|35809|1402694|4779|46911|49|20|5941|24|1597.92|79.89|1677.81|54.10|152.16|974.73|448.69|174.50|286.15| +2452812|32739|15453|79746|902707|2821|38535|79746|902707|2821|38535|37|28|5941|42|637.14|19.11|656.25|52.12|155.40|293.08|127.30|216.76|226.63| +2451288|12217|15974|11437|367575|6543|31078|11437|367575|6543|31078|2|31|5942|21|902.58|36.10|938.68|81.41|180.39|812.32|28.88|61.38|297.90| +2451293|77950|7177|64667|1388004|5741|38278|64667|1388004|5741|38278|56|10|5942|23|1577.34|110.41|1687.75|5.45|122.59|962.17|159.94|455.23|238.45| +2451629|14963|10105|99620|859867|2514|36361|99620|859867|2514|36361|10|7|5945|25|82.50|3.30|85.80|89.13|845.75|17.32|1.30|63.88|938.18| +2451837|7622|5137|70097|1502992|4240|9488|70097|1502992|4240|9488|13|23|5946|4|152.84|0.00|152.84|22.33|42.32|87.11|27.60|38.13|64.65| +2451857|30441|5659|53829|780764|5746|15881|53829|780764|5746|15881|35|28|5946|3|188.16|5.64|193.80|8.64|16.11|107.25|5.66|75.25|30.39| +2452666|68245|1209|88592|1162569|5346|24076|88592|1162569|5346|24076|25|4|5947|22|925.76|55.54|981.30|18.67|717.42|111.09|407.33|407.34|791.63| +2451607|23885|4630|43999|1868130|6214|41498|43999|1868130|6214|41498|58|1|5948|59|1696.84|50.90|1747.74|87.51|424.21|610.86|977.38|108.60|562.62| +2452087|61446|3044|80330|714601|5319|30390|80330|714601|5319|30390|49|28|5949|61|2765.74|82.97|2848.71|83.97|3207.99|442.51|650.50|1672.73|3374.93| +2452317|46696|6875|67245|1518732|266|13353|67245|1518732|266|13353|53|22|5950|28|2179.80|43.59|2223.39|22.53|1652.56|414.16|423.75|1341.89|1718.68| +2452376|56915|12975|2838|1555669|6900|20531|2838|1555669|6900|20531|3|24|5950|19|712.31|49.86|762.17|69.61|414.58|199.44|384.65|128.22|534.05| +2452408|69238|3777|1818|1414774|2483|6639|1818|1414774|2483|6639|25|25|5951|1|11.29|0.00|11.29|85.21|1.61|7.56|3.39|0.34|86.82| +2452426|19502|15905|16901|940928|4955|3382|16901|940928|4955|3382|43|3|5951|6|319.14|22.33|341.47|91.44|39.30|181.90|20.58|116.66|153.07| +2452218|54468|11725|98971|635223|2228|4833|98971|635223|2228|4833|19|2|5952|3|97.59|7.80|105.39|97.78|48.27|7.80|60.15|29.64|153.85| +2452217|74597|13231|94891|1098011|4331|27851|94891|1098011|4331|27851|13|28|5952|3|24.00|1.92|25.92|3.47|0.48|0.72|5.35|17.93|5.87| +2452841|70415|11719|23776|1651649|3169|3489|23776|1651649|3169|3489|45|20|5953|6|362.82|0.00|362.82|95.72|466.50|279.37|32.54|50.91|562.22| +|61428|14895|||||9352|1577097||40940|13||5953|||1.74||79.96|1.45|17.11|8.54|9.27|83.15| +2451293|58027|13528|63365|1685890|5568|44893|63365|1685890|5568|44893|58|15|5954|7|186.27|14.90|201.17|36.15|70.63|150.87|28.32|7.08|121.68| +|53113|14186|72778|||33019|72778|1411925||33019|49||5955|12||41.73||42.85|143.40|792.89|120.18||227.98| +|27498|12309|26366|1081656|593|44730||1081656|593|44730||5|5956|1||||62.14|63.88|31.17|19.74||127.55| +2451242|55006|15955|28976|1024185|5417|2935|28976|1024185|5417|2935|55|19|5959|16|296.00|23.68|319.68|83.49|357.60|254.56|13.26|28.18|464.77| +2451288|75563|718|28976|1024185|5417|2935|28976|1024185|5417|2935|10|32|5959|14|503.44|0.00|503.44|2.29|198.24|20.13|357.64|125.67|200.53| +2451255|74907|1093|44090|195282|2474|17266|44090|195282|2474|17266|10|28|5959|29|838.10|58.66|896.76|97.31|203.29|662.09|52.80|123.21|359.26| +2451455|77807|15853|91604|1280079|4107|6135|91604|1280079|4107|6135|7|24|5959|30|562.50|39.37|601.87|40.40|1038.30|286.87|220.50|55.13|1118.07| +2451964|23432|14069|35756|392479|2802|13925|35756|392479|2802|13925|55|5|5961|90|5531.40|276.57|5807.97|24.51|1088.10|719.08|866.21|3946.11|1389.18| +2451782|58892|7624|56226|1172163|2500|33237|56226|1172163|2500|33237|55|5|5962|97|3065.20|183.91|3249.11|72.96|4845.15|2973.24|8.27|83.69|5102.02| +2452148|49892|6931|44649|346802|1402|7555|44649|346802|1402|7555|57|6|5963|97|316.22|6.32|322.54|30.24|69.84|268.78|36.52|10.92|106.40| +2452229|4964|8996|70809|648831|1253|42475|70809|648831|1253|42475|17|16|5964|23|419.98|20.99|440.97|58.18|980.26|130.19|252.11|37.68|1059.43| +2452088|32845|12866|84924|1868229|2734|33992|84924|1868229|2734|33992|49|23|5964|53|1477.64|73.88|1551.52|86.58|1449.02|531.95|330.99|614.70|1609.48| +2451853|53110|6394|81551|461297|3796|12695|81551|461297|3796|12695|32|18|5965|44|1490.72|44.72|1535.44|56.06|919.16|1460.90|27.73|2.09|1019.94| +2451675|81327|7600|34193|1618269|2345|46021|34193|1618269|2345|46021|7|22|5965|58|1571.80|141.46|1713.26|4.78|3143.60|943.08|201.19|427.53|3289.84| +2452622|75915|15375|89756|1245693|6877|6861|89756|1245693|6877|6861|18|26|5966|43|2970.44|118.81|3089.25|42.17|79.12|2406.05|530.52|33.87|240.10| +2452777|55990|2715|42305|1641527|1902|19124|42305|1641527|1902|19124|60|20|5966|24|609.84|48.78|658.62|2.10|125.04|567.15|42.26|0.43|175.92| +2452645|34985|3159|45257|1268429|5201|8020|45257|1268429|5201|8020|54|29|5968|15|268.65|16.11|284.76|52.52|68.25|10.74|170.22|87.69|136.88| +2451813|8395|9701|79235|345785|4003|37054|79235|345785|4003|37054|49|2|5969|70|7746.90|309.87|8056.77|66.99|3821.30|2324.07|4284.03|1138.80|4198.16| +2451645|34185|2542|74774|1818853|4848|1550|74774|1818853|4848|1550|44|23|5970|49|2646.98|238.22|2885.20|34.51|407.19|2170.52|381.16|95.30|679.92| +2452361|74299|2305|42492|899099|4153|6360|42492|899099|4153|6360|13|1|5971|5|603.80|18.11|621.91|19.78|58.90|440.77|74.99|88.04|96.79| +2452411|23825|16637|65712|1141040|1521|43202|65712|1141040|1521|43202|19|12|5971|28|5343.52|213.74|5557.26|21.37|160.16|1335.88|721.37|3286.27|395.27| +2452392|72832|10457|98747|252182|2417|8652|98747|252182|2417|8652|9|26|5972|45|501.30|15.03|516.33|62.71|2785.05|190.49|292.16|18.65|2862.79| +2452407|41798|16517|42854|1127589|324|27012|42854|1127589|324|27012|43|35|5972|7|52.99|0.52|53.51|80.37|286.44|49.81|2.60|0.58|367.33| +2452488|75394|4043|68122|1865441|360|18479|68122|1865441|360|18479|41|25|5974|3|3.87|0.00|3.87|18.54|4.77|1.16|0.65|2.06|23.31| +2451697|67283|10886|13871|1288107|5096|33285|13871|1288107|5096|33285|16|25|5975|31|2683.36|53.66|2737.02|42.42|1923.24|0.00|885.50|1797.86|2019.32| +2451233|8151|12266|34491|1530574|3810|3046|34491|1530574|3810|3046|13|4|5976|5|274.50|24.70|299.20|45.64|268.80|57.64|36.86|180.00|339.14| +2451371|19444|10664|94514|1076690|3666|33795|94514|1076690|3666|33795|52|27|5976|6|474.42|33.20|507.62|97.72|60.54|431.72|40.99|1.71|191.46| +2451253|24205|17900|83321|99553|6185|23617|83321|99553|6185|23617|14|1|5976|8|1695.44|152.58|1848.02|80.40|866.96|186.49|573.40|935.55|1099.94| +2451963|47378|9625|93546|189191|4389|36387|93546|189191|4389|36387|7|21|5978|2|10.12|0.60|10.72|80.55|13.80|6.88|0.16|3.08|94.95| +2452491|43901|453|96452|144854|3656|25658|96452|144854|3656|25658|45|12|5979|6|577.86|34.67|612.53|53.00|213.54|404.50|46.80|126.56|301.21| +2452448|51911|7261|72748|1490925|4655|3585|72748|1490925|4655|3585|57|20|5979|2|93.24|1.86|95.10|80.49|68.26|53.14|27.26|12.84|150.61| +2452541|62216|9349|16988|426428|4078|47109|16988|426428|4078|47109|59|10|5979|16|591.84|11.83|603.67|51.62|25.60|325.51|111.85|154.48|89.05| +2452078|38790|10220|58193|1211364|6935|34459|58193|1211364|6935|34459|14|16|5980|11|374.11|0.00|374.11|28.63|66.99|258.13|9.27|106.71|95.62| +2451555|22244|4916|4770|497435|500|10718|4770|497435|500|10718|49|14|5981|22|52.14|3.12|55.26|92.53|10.56|37.01|12.86|2.27|106.21| +2452538|40883|10849|8118|914820|722|32224|8118|914820|722|32224|7|21|5982|26|2113.80|147.96|2261.76|64.05|1020.24|1416.24|6.97|690.59|1232.25| +2452839|47002|16623|2907|986815|323|9078|2907|986815|323|9078|30|33|5983|18|131.22|6.56|137.78|7.59|411.12|127.28|0.11|3.83|425.27| +2452647|29221|1710|46038|865614|390|18366|46038|865614|390|18366|3|29|5983|7|14.84|1.33|16.17|50.42|153.93|2.52|8.62|3.70|205.68| +2451920|70992|9787|63324|1406149|974|6599|63324|1406149|974|6599|20|4|5984|7|88.48|7.96|96.44|34.47|96.88|13.27|18.05|57.16|139.31| +2451312|67069|16846|93331|660303|2828|23000|93331|660303|2828|23000|20|8|5986|13|184.08|1.84|185.92|66.03|265.07|51.54|30.48|102.06|332.94| +2451157|19422|5920|66991|1296097|2356|48662|66991|1296097|2356|48662|49|22|5986|40|580.80|29.04|609.84|70.02|1051.20|325.24|97.11|158.45|1150.26| +2451393|57341|1294|50594|1258849|4351|37453|50594|1258849|4351|37453|40|26|5986|4|439.48|17.57|457.05|19.67|65.44|136.23|300.21|3.04|102.68| +2452008|79962|14755|36448|416031|2014|45550|36448|416031|2014|45550|49|6|5987|17|892.84|17.85|910.69|64.01|83.64|446.42|366.06|80.36|165.50| +2451590|32539|13898|59119|1077432|6145|34458|59119|1077432|6145|34458|58|10|5990|17|244.12|19.52|263.64|74.19|24.31|231.91|9.52|2.69|118.02| +2451720|40574|10795|50612|1555953|2894|44282|50612|1555953|2894|44282|31|35|5990|16|1082.88|43.31|1126.19|41.92|541.44|86.63|597.75|398.50|626.67| +2451705|82748|13568|26898|424255|12|27369|26898|424255|12|27369|26|14|5990|1|49.58|2.47|52.05|80.20|52.89|37.18|10.66|1.74|135.56| +2452633|37778|17959|24417|1859133|249|3412|24417|1859133|249|3412|33|30|5992|4|431.36|0.00|431.36|39.76|37.72|194.11|158.95|78.30|77.48| +2452717|30844|15282|75752|1566608|5391|34415|75752|1566608|5391|34415|7|3|5992|24|915.12|54.90|970.02|72.26|1689.60|91.51|469.45|354.16|1816.76| +2451402|43228|6259|88239|218199|4959|28119|88239|218199|4959|28119|50|14|5993|4|214.92|15.04|229.96|74.27|27.20|197.72|7.05|10.15|116.51| +2451517|6378|14024|31796|373027|3058|31489|31796|373027|3058|31489|58|12|5997|3|512.88|0.00|512.88|74.22|276.60|497.49|13.85|1.54|350.82| +2451749|45746|8878|4083|258998|1560|35275|4083|258998|1560|35275|31|26|6000|38|1108.84|55.44|1164.28|28.51|701.86|831.63|11.08|266.13|785.81| +2452667|69900|3645|46220|67298|5976|254|46220|67298|5976|254|12|22|6001|47|19.74|0.59|20.33|53.83|851.17|5.72|12.89|1.13|905.59| +2452830|28767|2965|27690|1548715|1293|3471|27690|1548715|1293|3471|60|8|6002|5|44.20|1.32|45.52|98.56|24.90|38.01|2.22|3.97|124.78| +2452736|51648|3939|19099|434341|6830|16119|19099|434341|6830|16119|12|9|6002|46|1686.36|101.18|1787.54|59.06|1073.18|590.22|263.07|833.07|1233.42| +2452599|43699|14341|23815|382492|3569|29799|23815|382492|3569|29799|57|4|6002|7|194.18|7.76|201.94|75.96|44.80|77.67|11.65|104.86|128.52| +|5364|2799|55215|1900061|3307|||||||16|6002|2|9.30||||||2.53|5.38|100.36| +2451432|28782|82|50262|1655229|6596|49563|50262|1655229|6596|49563|56|18|6003|3|142.92|7.14|150.06|10.30|51.27|32.87|9.90|100.15|68.71| +2451981|39684|2447|667|507081|1942|2756|667|507081|1942|2756|35|13|6004|1|78.56|5.49|84.05|4.22|91.30|47.13|5.34|26.09|101.01| +2451755|44039|11245|69196|1376880|2964|31048|69196|1376880|2964|31048|5|8|6006|9|50.04|4.50|54.54|79.74|50.04|37.53|7.25|5.26|134.28| +2452014|39452|3227|12393|1576606|1619|14185|12393|1576606|1619|14185|41|4|6007|11|53.02|3.18|56.20|49.09|92.84|22.79|27.20|3.03|145.11| +2452046|76066|4904|47009|1091206|6603|25200|47009|1091206|6603|25200|26|4|6007|63|2777.67|55.55|2833.22|50.38|1767.78|527.75|44.99|2204.93|1873.71| +2452076|52706|1490|38298|1192720|1189|36811|38298|1192720|1189|36811|8|20|6007|66|480.48|0.00|480.48|47.85|320.10|413.21|45.07|22.20|367.95| +2451643|62059|12577|20529|977191|1853|31315|20529|977191|1853|31315|7|24|6008|3|230.46|0.00|230.46|25.58|84.75|124.44|64.67|41.35|110.33| +2451761|57365|12622|65790|1139190|5557|10353|65790|1139190|5557|10353|2|13|6008|82|1657.22|99.43|1756.65|94.97|4557.56|1474.92|142.19|40.11|4751.96| +2451707|76680|9050|50696|1904896|1143|22143|50696|1904896|1143|22143|20|1|6008|9|9.63|0.09|9.72|19.13|16.92|8.95|0.57|0.11|36.14| +2451641|82093|14284|67883|948598|3713|46428|67883|948598|3713|46428|22|3|6010|5|165.35|0.00|165.35|63.70|92.55|147.16|0.90|17.29|156.25| +2451258|26947|4471|32592|689164|583|2355|32592|689164|583|2355|20|8|6013|53|85.86|1.71|87.57|22.26|85.86|78.13|4.25|3.48|109.83| +2451121|78976|14714|85972|791936|1900|29064|85972|791936|1900|29064|14|18|6013|10|294.90|0.00|294.90|19.45|262.10|212.32|28.07|54.51|281.55| +2452505|49684|14607|74354|1035107|2260|36213|74354|1035107|2260|36213|3|11|6014|44|61.60|1.23|62.83|35.11|1543.52|3.69|20.26|37.65|1579.86| +2452711|54536|1545|61920|470713|1553|46746|61920|470713|1553|46746|37|22|6015|27|407.43|4.07|411.50|27.96|558.63|264.82|32.80|109.81|590.66| +|7840|7074|||6745||||||24||6015|74||||||523.47|70.76|1344.57|1281.53| +2452753|54227|9828|10952|1461270|2788|17106|10952|1461270|2788|17106|3|12|6015|13|13.78|1.24|15.02|61.29|198.12|7.99|0.52|5.27|260.65| +2451199|3867|244|87983|1445282|181|9480|87983|1445282|181|9480|50|21|6016|17|726.75|29.07|755.82|4.52|789.14|450.58|121.51|154.66|822.73| +||7438|27415||4262|21662|27415|471215||||15|6017|89|||948.74|69.15||||26.47|1112.23| +2452286|73745|12853|98495|735021|3071|28545|98495|735021|||53||6018|6|134.94||||||0.00|0.00|| +2452713|62585|37|82319|1887100|5034|47015|82319|1887100|5034|47015|24|17|6019|20|179.20|8.96|188.16|51.63|30.20|84.22|83.58|11.40|90.79| +2452867|53138|11707|53738|1910163|6141|27719|53738|1910163|6141|27719|57|1|6019|38|1143.04|34.29|1177.33|82.90|607.24|800.12|96.01|246.91|724.43| +2452741|72094|3121|76564|1775986|208|8030|76564|1775986|208|8030|7|11|6021|38|3101.94|124.07|3226.01|25.75|144.02|961.60|1219.99|920.35|293.84| +2452795|60565|6523|51271|1472511|4151|34144|51271|1472511|4151|34144|1|4|6021|62|0.00|0.00|0.00|41.23|804.14|0.00|0.00|0.00|845.37| +2452812|36617|3739|67792|1547890|7188|24133|67792|1547890|7188|24133|33|13|6021|29|470.67|28.24|498.91|76.70|444.57|197.68|62.78|210.21|549.51| +2452006|38899|2063|27318|1641779|1840|44701|27318|1641779|1840|44701|35|5|6022|7|419.65|12.58|432.23|47.63|222.18|293.75|80.57|45.33|282.39| +2452028|35787|5414|71929|468647|2165|4883|71929|468647|2165|4883|19|20|6022|8|43.76|0.00|43.76|74.31|5.60|26.25|9.28|8.23|79.91| +2452053|9321|7483|78244|1295429|6214|22246|78244|1295429|6214|22246|23|18|6022|4|241.08|4.82|245.90|74.26|0.00|115.71|53.90|71.47|79.08| +2452201|17764|6233|9932|305675|5771|23114|9932|305675|5771|23114|59|18|6023|38|1402.96|98.20|1501.16|41.81|935.18|645.36|621.23|136.37|1075.19| +2452310|32864|7403|60106|871922|91|19507|60106|871922|91|19507|9|9|6025|13|52.91|2.11|55.02|62.16|30.55|49.73|0.28|2.90|94.82| +2452486|73891|7935|66058|184376|1094|18259|66058|184376|1094|18259|19|9|6025|2|67.42|1.34|68.76|98.17|61.56|8.09|55.17|4.16|161.07| +2452446|52309|3781|17992|1582659|4926|16793|17992|1582659|4926|16793|45|14|6026|19|1544.32|15.44|1559.76|41.60|1229.11|679.50|605.37|259.45|1286.15| +2452226|2552|7514|42875|1711067|1212|19804|42875|1711067|1212|19804|13|25|6027|4|444.48|4.44|448.92|82.95|33.32|275.57|69.25|99.66|120.71| +2452106|69850|929|80209|891895|3581|30867|80209|891895|3581|30867|25|31|6027|81|2664.90|0.00|2664.90|90.10|769.50|2211.86|90.60|362.44|859.60| +2451235|46647|9712|63481|1717634|6987|48740|63481|1717634|6987|48740|25|23|6028|5|159.30|14.33|173.63|98.21|10.60|81.24|39.81|38.25|123.14| +2452422|40347|2292|19660|1905108|2909|1557|19660|1905108|2909|1557|30|27|6030|19|307.04|15.35|322.39|17.60|200.26|282.47|6.38|18.19|233.21| +2452579|73356|9607|99994|1487032|4362|23743|99994|1487032|4362|23743|15|19|6030|60|189.60|5.68|195.28|83.85|902.40|109.96|35.04|44.60|991.93| +2452548|81240|1620|79872|103658|3091|33277|79872|103658|3091|33277|27|20|6030|51|5796.66|115.93|5912.59|38.34|901.68|1043.39|3327.28|1425.99|1055.95| +2452742|75115|14875|51488|1627988|488|45035|51488|1627988|488|45035|21|7|6031|12|738.48|29.53|768.01|29.97|167.40|487.39|112.99|138.10|226.90| +2451283|61750|2492|87948|640041|2181|49921|87948|640041|2181|49921|1|24|6033|23|1229.12|61.45|1290.57|49.03|92.00|749.76|119.84|359.52|202.48| +2451731|60641|9524|50458|302789|2673|30604|50458|302789|2673|30604|7|24|6035|28|300.72|6.01|306.73|72.04|182.84|72.17|182.84|45.71|260.89| +2451507|55662|9181|45829|504779|3729|37032|45829|504779|3729|37032|7|32|6035|40|1395.60|125.60|1521.20|38.00|332.00|251.20|709.52|434.88|495.60| +2451724|33351|5666|3032|1708921|57|11146|3032|1708921|57|11146|16|28|6035|14|1147.86|34.43|1182.29|26.58|186.06|447.66|350.10|350.10|247.07| +2452375|21293|16069|15491|650601|2958|16868|15491|650601|2958|16868|7|9|6036|43|3401.30|34.01|3435.31|78.67|877.63|1394.53|260.88|1745.89|990.31| +|35670|4073||6409|580|7951|5038||580||||6036|32||1.57|||131.52||0.98|1.12|152.49| +2452485|75842|3299|10460|501985|5157|31082|10460|501985|5157|31082|55|11|6036|67|1626.09|32.52|1658.61|23.33|1417.72|1252.08|172.04|201.97|1473.57| +2452447|19818|2933|55237|165043|3202|47361|55237|165043|3202|47361|15|27|6036|2|34.98|0.00|34.98|21.22|2.44|23.78|4.81|6.39|23.66| +2452144|70026|14863|23415|920209|5356|8150|23415|920209|5356|8150|43|6|6037|11|312.07|6.24|318.31|48.43|61.71|283.98|7.58|20.51|116.38| +2452089|65721|10589|56354|589334|6667|48047|56354|589334|6667|48047|7|15|6037|2|206.56|16.52|223.08|62.95|25.02|84.68|40.22|81.66|104.49| +2451978|44865|5171|3292|557750|2608|37066|3292|557750|2608|37066|49|29|6037|13|454.61|22.73|477.34|65.33|70.33|350.04|15.68|88.89|158.39| +2452139|64688|9284|97263|212866|4022|41178|97263|212866|4022|41178|29|24|6037|4|17.68|0.17|17.85|45.34|6.68|10.96|2.21|4.51|52.19| +2451625|68820|16231|99515|1633490|4712|20687|99515|1633490|4712|20687|49|3|6038|72|1409.04|112.72|1521.76|87.90|0.00|408.62|540.22|460.20|200.62| +2451604|6688|13465|32472|1267079|4086|28293|32472|1267079|4086|28293|40|25|6038|53|709.67|63.87|773.54|36.13|157.41|702.57|5.25|1.85|257.41| +2452394|54252|5643|54014|1891551|4288|20178|54014|1891551|4288|20178|19|26|6041|26|142.22|12.79|155.01|2.89|1253.20|85.33|44.94|11.95|1268.88| +2452289|78268|10289|95672|932953|3034|6315|95672|932953|3034|6315|25|2|6041|2|76.78|6.14|82.92|46.87|0.00|24.56|51.17|1.05|53.01| +2452611||849|86979|||16382|86979||4181||||6044|37|2610.35||||2666.96||877.07|1211.21|| +2452604|3885|8173|52644|1802659|4674|17842|52644|1802659|4674|17842|45|8|6044|3|13.74|0.27|14.01|67.38|5.79|9.20|1.77|2.77|73.44| +2452541|62679|6637|2461|452677|3340|5156|2461|452677|3340|5156|3|14|6044|64|1811.84|90.59|1902.43|94.39|2416.00|1050.86|7.60|753.38|2600.98| +2452438|16433|6288|19991|82232|2520|49161|19991|82232|2520|49161|45|33|6046|18|3413.88|307.24|3721.12|11.23|120.42|2833.52|412.05|168.31|438.89| +2451774|77271|2731|7254|587309|3811|45913|7254|587309|3811|45913|10|35|6047|1|37.03|2.22|39.25|99.94|11.16|7.77|6.14|23.12|113.32| +2451560|55822|955|37803|892460|3917|30927|37803|892460|3917|30927|19|25|6047|9|434.97|0.00|434.97|88.11|51.12|278.38|81.42|75.17|139.23| +2452356|54076|11461|73205|482862|6919|19729|73205|482862|6919|19729|29|24|6048|5|85.40|5.97|91.37|98.28|61.00|58.07|24.87|2.46|165.25| +2451131|43973|15976|73473|1271511|2349|34340|73473|1271511|2349|34340|13|21|6049|64|426.24|4.26|430.50|42.82|238.08|387.87|28.77|9.60|285.16| +2451168|44108|6220|46695|1557043|1791|6986|46695|1557043|1791|6986|43|30|6049|2|166.26|1.66|167.92|75.42|25.24|93.10|45.35|27.81|102.32| +2451174|41903|5864|83449|792765|5728|23844|83449|792765|5728|23844|10|24|6049|2|6.20|0.55|6.75|18.85|3.04|4.83|0.09|1.28|22.44| +2451677|73880|9892|37950|1044224|6051|8725|37950|1044224|6051|8725|58|8|6052|2|178.18|10.69|188.87|81.94|124.08|128.28|7.48|42.42|216.71| +2451531|82219|11437|9990|1381182|5908|8053|9990|1381182|5908|8053|52|17|6052|34|423.64|38.12|461.76|0.60|659.26|135.56|69.13|218.95|697.98| +2452709|62485|150|83171|1120083|1348|29860|83171|1120083|1348|29860|37|3|6053|64|7648.64|382.43|8031.07|6.09|0.00|2600.53|4341.37|706.74|388.52| +2452759|30715|1789|61456|1062910|11|1065|61456|1062910|11|1065|55|10|6053|9|342.54|27.40|369.94|34.11|71.55|208.94|6.68|126.92|133.06| +2452906|57492|10752|191|1469842|6340|41191|191|1469842|6340|41191|36|24|6053|43|9255.32|277.65|9532.97|45.95|0.00|3146.80|4214.87|1893.65|323.60| +2452858|70264|10933|39742|521844|5193|28895|39742|521844|5193|28895|57|16|6053|46|2616.94|235.52|2852.46|20.81|1892.90|157.01|1525.15|934.78|2149.23| +2452655|76952|7551|88605|985402|1846|6243|88605|985402|1846|6243|36|8|6053|6|321.90|25.75|347.65|55.99|62.94|218.89|22.66|80.35|144.68| +|56526|3667||145412||42488||145412|||37||6054|2|147.28|||||45.65||77.24|| +2451168|34496|7286|65831|1759496|5405|5053|65831|1759496|5405|5053|20|30|6054|31|1607.66|32.15|1639.81|54.71|1969.12|401.91|397.89|807.86|2055.98| +|61216|14546||1493300|5103|6106|79450|1493300|5103|6106||16|6055|10|4.00||4.20|||2.40|1.28||42.90| +2451512|19797|16264|36880|356280|480|7080|36880|356280|480|7080|20|20|6055|15|109.20|9.82|119.02|47.28|417.00|17.47|35.77|55.96|474.10| +2451507|69616|3661|53949|709384|5172|17407|53949|709384|5172|17407|20|33|6055|56|5146.96|411.75|5558.71|51.95|5146.96|1698.49|2344.95|1103.52|5610.66| +2451876|33604|14233|79446|1737527|5208|9689|79446|1737527|5208|9689|2|25|6056|15|872.40|52.34|924.74|48.36|484.65|715.36|43.97|113.07|585.35| +2451756|32702|13724|16444|828767|6337|37907|16444|828767|6337|37907|25|35|6056|75|1860.75|55.82|1916.57|87.81|80.25|1749.10|5.58|106.07|223.88| +2451736|46632|637|27578|1441550|322|26294|27578|1441550|322|26294|7|28|6056|28|1118.04|44.72|1162.76|55.70|298.20|268.32|229.42|620.30|398.62| +2451813|44397|9092|29393|210564|2135|12106|29393|210564|2135|12106|10|24|6057|1|85.01|7.65|92.66|16.54|23.33|60.35|9.61|15.05|47.52| +2451810|78800|14851|29531|1211202|73|18897|29531|1211202|73|18897|34|33|6057|23|372.14|33.49|405.63|7.63|37.03|189.79|82.05|100.30|78.15| +2452205|36056|16989|42139|1827885|6431|42130|42139|1827885|6431|42130|3|5|6058|52|4549.48|272.96|4822.44|66.22|727.48|1819.79|655.12|2074.57|1066.66| +2451954|42888|3776|91131|376551|6771|4238|91131|376551|6771|4238|43|17|6059|12|119.64|2.39|122.03|40.83|119.64|88.53|10.88|20.23|162.86| +2451945|82550|3182|81479|65787|2095|38450|81479|65787|2095|38450|31|5|6059|39|178.23|16.04|194.27|36.56|109.59|44.55|53.47|80.21|162.19| +2452603|61378|16231|12290|1250434|4578|30722|12290|1250434|4578|30722|42|34|6061|17|168.64|3.37|172.01|98.69|76.33|20.23|136.53|11.88|178.39| +2451892|18071|583|42581|131983|1385|48387|42581|131983|1385|48387|20|11|6062|33|2257.86|203.20|2461.06|9.27|637.89|925.72|452.92|879.22|850.36| +2452118|18001|17864|99579|1178481|1383|32732|99579|1178481|1383|32732|43|28|6062|13|341.25|13.65|354.90|55.79|156.13|122.85|78.62|139.78|225.57| +2452387|75186|9343|8175|797672|2722|10219|8175|797672|2722|10219|45|23|6064|22|461.78|18.47|480.25|70.52|629.64|364.80|45.58|51.40|718.63| +2452302|73702|3303|22889|11607|3525|35843|22889|11607|3525|35843|3|9|6064|11|243.10|4.86|247.96|8.95|457.05|2.43|52.94|187.73|470.86| +2452342|36757|9609|79629|1510584|5539|19294|79629|1510584|5539|19294|37|6|6064|46|10059.74|301.79|10361.53|58.63|3869.06|8349.58|513.04|1197.12|4229.48| +2452476|42139|11463|23680|80546|3012|4725|23680|80546|3012|4725|15|30|6064|70|567.70|34.06|601.76|52.95|499.10|5.67|112.40|449.63|586.11| +2451372|67079|5266|42148|1267601|1854|5260|42148|1267601|1854|5260|52|13|6066|16|656.80|39.40|696.20|19.11|340.16|19.70|235.72|401.38|398.67| +2451232||7855|||6217||14262|||39971|10|19|6066|||||33.94|||1496.41|2340.56|| +2451210|57399|17518|62848|1484316|1784|4852|62848|1484316|1784|4852|46|3|6066|20|1346.20|53.84|1400.04|55.94|336.40|498.09|178.10|670.01|446.18| +2451638|26780|6505|17497|1138729|5114|20699|17497|1138729|5114|20699|38|20|6067|36|618.48|0.00|618.48|26.85|515.52|327.79|186.04|104.65|542.37| +2451621|34537|16342|1426|1535891|3956|29221|1426|1535891|3956|29221|1|6|6067|37|704.85|49.33|754.18|12.37|903.91|563.88|1.40|139.57|965.61| +2452252|24098|16187|9629|508412|4442|20025|9629|508412|4442|20025|37|14|6068|29|1097.65|65.85|1163.50|19.87|1191.90|1053.74|1.75|42.16|1277.62| +2452330|71851|4297|10384|129915|3846|42624|10384|129915|3846|42624|53|29|6068|3|26.76|0.53|27.29|68.40|40.14|15.25|9.78|1.73|109.07| +2452406|39657|9337|19147|777459|4571|47683|19147|777459|4571|47683|49|20|6068|2|21.36|1.28|22.64|72.00|9.68|7.26|6.62|7.48|82.96| +2452311|37310|17703|15218|1209961|2304|11047|15218|1209961|2304|11047|11|10|6068|24|79.68|6.37|86.05|47.70|52.08|62.15|11.92|5.61|106.15| +2452685||17010|49310|1883801||41677|49310||||||6070|44|||2087.77||2098.80|1908.82||54.89|| +2452174|73205|731|2956|298607|4305|32403|2956|298607|4305|32403|31|30|6071|69|3267.84|0.00|3267.84|81.54|612.72|2352.84|384.30|530.70|694.26| +2451961|49424|17606|62996|1187646|743|15393|62996|1187646|743|15393|29|12|6071|4|81.16|2.43|83.59|43.68|27.04|6.49|26.88|47.79|73.15| +2451321|77821|13333|17928|424115|1502|8889|17928|424115|1502|8889|50|31|6073|29|2576.36|206.10|2782.46|17.31|317.84|1777.68|495.18|303.50|541.25| +2451553|19732|13228|73563|1520789|3568|8394|73563|1520789|3568|8394|56|34|6077|2|83.80|5.86|89.66|32.95|14.78|57.82|23.90|2.08|53.59| +2451618|28468|2131|39927|464944|4808|19872|39927|464944|4808|19872|44|32|6078|32|524.80|31.48|556.28|62.25|1874.88|456.57|30.70|37.53|1968.61| +2452275|46524|11173|1642|617760|631|23502|1642|617760|631|23502|11|34|6080|14|1942.92|58.28|2001.20|98.35|626.08|1690.34|123.76|128.82|782.71| +2452245|25615|9829|8256|715774|892|34410|8256|715774|892|34410|47|22|6080|47|1502.59|75.12|1577.71|86.17|965.85|736.26|429.14|337.19|1127.14| +2452259|69589|16863|24793|1237159|3925|43891|24793|1237159|3925|43891|55|34|6082|14|1349.18|80.95|1430.13|3.01|306.60|458.72|792.50|97.96|390.56| +2452328|56864|13477|61961|553515|5069|270|61961|553515|5069|270|33|18|6086|6|93.60|4.68|98.28|95.12|69.66|11.23|47.77|34.60|169.46| diff --git a/spark-connector/spark-common/src/test/resources/data/tpcds/web_sales/web_sales.dat b/spark-connector/spark-common/src/test/resources/data/tpcds/web_sales/web_sales.dat new file mode 100644 index 00000000000..4eda143ad3f --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/data/tpcds/web_sales/web_sales.dat @@ -0,0 +1,7193 @@ +2451383|73313|2451482|4591|83074|596485|1096|40907|85919|41329|1140|1351|43|4|4|5|6|1|57|33.59|59.45|38.04|1220.37|2168.28|1914.63|3388.65|50.95|1149.18|575.70|1019.10|1070.05|1594.80|1645.75|-895.53| +2451383|73313|2451411|3566|83074|596485|1096|40907|85919|41329|1140|1351|28|7|3|2|271|1|38|29.83|48.92|26.41|855.38|1003.58|1133.54|1858.96|30.10|0.00|910.86|1003.58|1033.68|1914.44|1944.54|-129.96| +2451383|73313|2451413|7286|83074|596485|1096|40907|85919|41329|1140|1351|58|28|10|5|300|1|32|49.72|107.89|97.10|345.28|3107.20|1591.04|3452.48|124.28|0.00|828.48|3107.20|3231.48|3935.68|4059.96|1516.16| +2451383|73313|2451393|2755|83074|596485|1096|40907|85919|41329|1140|1351|2|7|6|1|63|1|65|69.18|112.07|22.41|5827.90|1456.65|4496.70|7284.55|29.13|0.00|2476.50|1456.65|1485.78|3933.15|3962.28|-3040.05| +2451383|73313|2451502|2516|83074|596485|1096|40907|85919|41329|1140|1351|56|16|2|5|18|1|58|36.62|41.38|16.13|1464.50|935.54|2123.96|2400.04|84.19|0.00|167.62|935.54|1019.73|1103.16|1187.35|-1188.42| +2451383|73313|2451421|16966|83074|596485|1096|40907|85919|41329|1140|1351|50|7|2|2|185|1|90|72.05|161.39|27.43|12056.40|2468.70|6484.50|14525.10|74.06|0.00|4647.60|2468.70|2542.76|7116.30|7190.36|-4015.80| +2451383|73313|2451457|10402|83074|596485|1096|40907|85919|41329|1140|1351|56|16|16|3|293|1|15|83.92|174.55|69.82|1570.95|1047.30|1258.80|2618.25|20.94|0.00|392.70|1047.30|1068.24|1440.00|1460.94|-211.50| +2451383|73313|2451430|1735|83074|596485|1096|40907|85919|41329|1140|1351|25|19|14|3|202|1|16|45.27|128.56|68.13|966.88|1090.08|724.32|2056.96|87.20|0.00|370.24|1090.08|1177.28|1460.32|1547.52|365.76| +2451383|73313|2451458|15464|83074|596485|1096|40907|85919|41329|1140|1351|13|26|8|5|49|1|54|53.45|60.39|26.57|1826.28|1434.78|2886.30|3261.06|48.78|215.21|488.70|1219.57|1268.35|1708.27|1757.05|-1666.73| +2452625|34964|2452702|8925|42296|436090|2684|37278|57428|1487225|3907|30901|9|13|16|3|267|2|47|49.64|68.99|12.41|2659.26|583.27|2333.08|3242.53|52.49|0.00|1134.58|583.27|635.76|1717.85|1770.34|-1749.81| +2452625|34964|2452638|11041|42296|436090|2684|37278|57428|1487225|3907|30901|1|13|16|2|221|2|12|7.24|12.16|11.91|3.00|142.92|86.88|145.92|10.00|0.00|29.16|142.92|152.92|172.08|182.08|56.04| +2452625|34964|2452645|645|42296|436090|2684|37278|57428|1487225|3907|30901|15|21|7|5|246|2|6|45.21|67.36|30.31|222.30|181.86|271.26|404.16|16.36|0.00|185.88|181.86|198.22|367.74|384.10|-89.40| +2452625|34964|2452690|12453|42296|436090|2684|37278|57428|1487225|3907|30901|57|12|3|5|237|2|43|9.62|13.08|9.02|174.58|387.86|413.66|562.44|19.39|0.00|117.82|387.86|407.25|505.68|525.07|-25.80| +2452625|34964|2452707|13831|42296|436090|2684|37278|57428|1487225|3907|30901|18|19|1|2|223|2|57|35.95|62.55|43.78|1069.89|2495.46|2049.15|3565.35|224.59|0.00|962.16|2495.46|2720.05|3457.62|3682.21|446.31| +2452625|34964|2452640|9559|42296|436090|2684|37278|57428|1487225|3907|30901|6|21|6|4|72|2|93|65.80|152.65|29.00|11499.45|2697.00|6119.40|14196.45|215.76|0.00|7097.76|2697.00|2912.76|9794.76|10010.52|-3422.40| +2452625|34964|2452634|8085|42296|436090|2684|37278|57428|1487225|3907|30901|45|3|18|2|276|2|93|55.42|127.46|1.27|11735.67|118.11|5154.06|11853.78|4.72|0.00|1421.97|118.11|122.83|1540.08|1544.80|-5035.95| +2452625|34964|2452734|12081|42296|436090|2684|37278|57428|1487225|3907|30901|9|27|3|2|94|2|82|48.65|91.94|8.27|6860.94|678.14|3989.30|7539.08|13.56|0.00|2788.82|678.14|691.70|3466.96|3480.52|-3311.16| +2452625|34964|2452739|15739|42296|436090|2684|37278|57428|1487225|3907|30901|24|3|9|4|99|2|33|80.23|90.65|55.29|1166.88|1824.57|2647.59|2991.45|0.00|0.00|478.50|1824.57|1824.57|2303.07|2303.07|-823.02| +2451754|1529|2451801|17431|74800|1219525|3450|3375|84180|293885|5550|41475|49|8|2|2|259|3|65|5.37|10.31|2.06|536.25|133.90|349.05|670.15|7.47|9.37|26.65|124.53|132.00|151.18|158.65|-224.52| +2451754|1529|2451791|4694|74800|1219525|3450|3375|84180|293885|5550|41475|13|25|9|5|91|3|74|74.66|83.61|52.67|2289.56|3897.58|5524.84|6187.14|116.92|0.00|1546.60|3897.58|4014.50|5444.18|5561.10|-1627.26| +2451754|1529|2451814|2189|74800|1219525|3450|3375|84180|293885|5550|41475|8|26|3|4|18|3|26|77.15|198.27|109.04|2319.98|2835.04|2005.90|5155.02|36.28|2381.43|206.18|453.61|489.89|659.79|696.07|-1552.29| +2451754|1529|2451765|151|74800|1219525|3450|3375|84180|293885|5550|41475|23|7|3|5|167|3|25|99.00|292.05|262.84|730.25|6571.00|2475.00|7301.25|65.71|0.00|3285.50|6571.00|6636.71|9856.50|9922.21|4096.00| +2451754|1529|2451855|10793|74800|1219525|3450|3375|84180|293885|5550|41475|11|14|7|4|298|3|80|14.55|42.34|27.09|1220.00|2167.20|1164.00|3387.20|43.34|0.00|609.60|2167.20|2210.54|2776.80|2820.14|1003.20| +2451754|1529|2451862|12782|74800|1219525|3450|3375|84180|293885|5550|41475|20|2|13|4|275|3|54|64.58|160.80|120.60|2170.80|6512.40|3487.32|8683.20|586.11|0.00|3646.62|6512.40|7098.51|10159.02|10745.13|3025.08| +2451754|1529|2451809|15769|74800|1219525|3450|3375|84180|293885|5550|41475|35|17|20|1|162|3|30|54.81|81.11|22.71|1752.00|681.30|1644.30|2433.30|6.81|0.00|0.00|681.30|688.11|681.30|688.11|-963.00| +2451754|1529|2451782|5549|74800|1219525|3450|3375|84180|293885|5550|41475|29|26|6|3|170|3|19|87.13|170.77|155.40|292.03|2952.60|1655.47|3244.63|88.57|0.00|486.59|2952.60|3041.17|3439.19|3527.76|1297.13| +2451754|1529|2451760|12475|74800|1219525|3450|3375|84180|293885|5550|41475|44|11|16|4|232|3|29|86.36|90.67|68.90|631.33|1998.10|2504.44|2629.43|159.84|0.00|236.64|1998.10|2157.94|2234.74|2394.58|-506.34| +2451754|1529|2451812|5324|74800|1219525|3450|3375|84180|293885|5550|41475|26|26|4|4|277|3|2|4.21|10.90|0.54|20.72|1.08|8.42|21.80|0.07|0.00|5.88|1.08|1.15|6.96|7.03|-7.34| +2451754|1529|2451846|15686|74800|1219525|3450|3375|84180|293885|5550|41475|53|11|18|2|79|3|42|56.94|107.04|102.75|180.18|4315.50|2391.48|4495.68|43.15|0.00|1708.14|4315.50|4358.65|6023.64|6066.79|1924.02| +2451754|1529|2451858|12977|74800|1219525|3450|3375|84180|293885|5550|41475|26|14|12|1|50|3|91|88.66|242.04|183.95|5286.19|16739.45|8068.06|22025.64|502.18|0.00|0.00|16739.45|17241.63|16739.45|17241.63|8671.39| +2451754|1529|2451758|1874|74800|1219525|3450|3375|84180|293885|5550|41475|43|29|15|3|98|3|10|32.29|45.85|30.71|151.40|307.10|322.90|458.50|0.61|276.39|137.50|30.71|31.32|168.21|168.82|-292.19| +2451754|1529|2451801|8281|74800|1219525|3450|3375|84180|293885|5550|41475|26|1|18|5|59|3|1|66.29|174.34|137.72|36.62|137.72|66.29|174.34|4.62|71.61|45.32|66.11|70.73|111.43|116.05|-0.18| +2451754|1529|2451763|17867|74800|1219525|3450|3375|84180|293885|5550|41475|7|5|19|5|14|3|7|66.22|145.68|144.22|10.22|1009.54|463.54|1019.76|80.76|0.00|193.69|1009.54|1090.30|1203.23|1283.99|546.00| +2451754|1529|2451807|10760|74800|1219525|3450|3375|84180|293885|5550|41475|47|25|2|3|17|3|10|87.53|150.55|58.71|918.40|587.10|875.30|1505.50|27.12|135.03|316.10|452.07|479.19|768.17|795.29|-423.23| +2451483|77832|2451501|11842|89187|512359|4331|19456|87757|1249940|2525|37697|38|13|11|3|192|4|83|4.36|10.15|4.66|455.67|386.78|361.88|842.45|27.07|0.00|151.06|386.78|413.85|537.84|564.91|24.90| +2451483|77832|2451554|6718|89187|512359|4331|19456|87757|1249940|2525|37697|1|10|4|4|74|4|20|71.90|103.53|85.92|352.20|1718.40|1438.00|2070.60|154.65|0.00|227.60|1718.40|1873.05|1946.00|2100.65|280.40| +2451483|77832|2451542|3592|89187|512359|4331|19456|87757|1249940|2525|37697|38|20|12|5|201|4|26|81.47|126.27|27.77|2561.00|722.02|2118.22|3283.02|50.54|0.00|1411.54|722.02|772.56|2133.56|2184.10|-1396.20| +2451483|77832|2451563|14779|89187|512359|4331|19456|87757|1249940|2525|37697|16|22|12|2|125|4|32|30.81|40.05|22.02|576.96|704.64|985.92|1281.60|35.23|0.00|358.72|704.64|739.87|1063.36|1098.59|-281.28| +2451483|77832|2451593|5908|89187|512359|4331|19456|87757|1249940|2525|37697|55|8|2|1|93|4|44|68.96|143.43|61.67|3597.44|2713.48|3034.24|6310.92|135.67|0.00|0.00|2713.48|2849.15|2713.48|2849.15|-320.76| +2451483|77832|2451487|7636|89187|512359|4331|19456|87757|1249940|2525|37697|26|4|12|1|30|4|41|6.21|11.92|1.31|435.01|53.71|254.61|488.72|3.75|0.00|190.24|53.71|57.46|243.95|247.70|-200.90| +2451483|77832|2451487|520|89187|512359|4331|19456|87757|1249940|2525|37697|37|2|2|5|265|4|22|71.63|113.89|69.47|977.24|1528.34|1575.86|2505.58|137.55|0.00|1077.34|1528.34|1665.89|2605.68|2743.23|-47.52| +2451483|77832|2451569|12574|89187|512359|4331|19456|87757|1249940|2525|37697|13|16|11|5|202|4|51|43.92|94.86|52.17|2177.19|2660.67|2239.92|4837.86|102.17|106.42|1209.21|2554.25|2656.42|3763.46|3865.63|314.33| +2451483|77832|2451536|997|89187|512359|4331|19456|87757|1249940|2525|37697|20|26|20|3|159|4|50|15.41|17.41|6.26|557.50|313.00|770.50|870.50|9.20|159.63|330.50|153.37|162.57|483.87|493.07|-617.13| +2451483|77832|2451521|13090|89187|512359|4331|19456|87757|1249940|2525|37697|19|25|6|2|105|4|65|34.29|75.43|18.10|3726.45|1176.50|2228.85|4902.95|70.59|0.00|1176.50|1176.50|1247.09|2353.00|2423.59|-1052.35| +2451483|77832|2451518|1600|89187|512359|4331|19456|87757|1249940|2525|37697|46|13|3|1|249|4|37|35.86|89.29|0.00|3303.73|0.00|1326.82|3303.73|0.00|0.00|1090.02|0.00|0.00|1090.02|1090.02|-1326.82| +2451945|25492|2451977|16217|66395|1553493|1391|32089|22505|1123302|983|19681|31|1|6|1|166|5|57|69.75|192.51|102.03|5157.36|5815.71|3975.75|10973.07|174.47|0.00|2633.40|5815.71|5990.18|8449.11|8623.58|1839.96| +2451945|25492|2451960|14577|66395|1553493|1391|32089|22505|1123302|983|19681|55|17|20|1|252|5|50|42.11|58.95|46.57|619.00|2328.50|2105.50|2947.50|162.99|0.00|58.50|2328.50|2491.49|2387.00|2549.99|223.00| +2451945|25492|2452011|5559|66395|1553493|1391|32089|22505|1123302|983|19681|29|5|1|2|230|5|89|70.02|77.72|55.18|2006.06|4911.02|6231.78|6917.08|147.33|0.00|2766.12|4911.02|5058.35|7677.14|7824.47|-1320.76| +2451945|25492|2451956|8633|66395|1553493|1391|32089|22505|1123302|983|19681|21|9|19|3|220|5|69|21.00|24.36|13.64|739.68|941.16|1449.00|1680.84|9.41|0.00|487.14|941.16|950.57|1428.30|1437.71|-507.84| +2451945|25492|2451954|14617|66395|1553493|1391|32089|22505|1123302|983|19681|29|27|16|5|34|5|26|26.29|28.65|18.33|268.32|476.58|683.54|744.90|33.36|0.00|186.16|476.58|509.94|662.74|696.10|-206.96| +2451945|25492|2451949|11747|66395|1553493|1391|32089|22505|1123302|983|19681|51|1|9|4|62|5|94|66.79|165.63|66.25|9341.72|6227.50|6278.26|15569.22|560.47|0.00|2491.00|6227.50|6787.97|8718.50|9278.97|-50.76| +2451945|25492|2452018|7243|66395|1553493|1391|32089|22505|1123302|983|19681|29|21|14|5|133|5|37|16.03|20.67|0.62|741.85|22.94|593.11|764.79|0.00|0.00|367.04|22.94|22.94|389.98|389.98|-570.17| +2451945|25492|2452044|875|66395|1553493|1391|32089|22505|1123302|983|19681|35|17|10|1|35|5|51|10.48|30.18|23.84|323.34|1215.84|534.48|1539.18|23.58|36.47|122.91|1179.37|1202.95|1302.28|1325.86|644.89| +2451945|25492|2452046|6215|66395|1553493|1391|32089|22505|1123302|983|19681|57|19|8|4|170|5|45|67.42|113.26|41.90|3211.20|1885.50|3033.90|5096.70|56.56|0.00|2497.05|1885.50|1942.06|4382.55|4439.11|-1148.40| +2451945|25492|2452060|10215|66395|1553493|1391|32089|22505|1123302|983|19681|23|7|6|3|10|5|3|47.96|84.88|1.69|249.57|5.07|143.88|254.64|0.20|0.00|30.54|5.07|5.27|35.61|35.81|-138.81| +2451945|25492|2452065|7665|66395|1553493|1391|32089|22505|1123302|983|19681|49|13|2|4|36|5|71|68.25|152.19|4.56|10481.73|323.76|4845.75|10805.49|29.13|0.00|648.23|323.76|352.89|971.99|1001.12|-4521.99| +2451945|25492|2451997|9961|66395|1553493|1391|32089|22505|1123302|983|19681|59|3|7|2|54|5|57|25.26|42.18|2.53|2260.05|144.21|1439.82|2404.26|8.65|0.00|624.72|144.21|152.86|768.93|777.58|-1295.61| +2451945|25492|2451977|14731|66395|1553493|1391|32089|22505|1123302|983|19681|3|27|16|3|110|5|66|66.72|166.80|90.07|5064.18|5944.62|4403.52|11008.80|416.12|0.00|1651.32|5944.62|6360.74|7595.94|8012.06|1541.10| +2451945|25492|2452005|3175|66395|1553493|1391|32089|22505|1123302|983|19681|43|17|9|1|16|5|60|30.82|56.70|29.48|1633.20|1768.80|1849.20|3402.00|0.00|1715.73|475.80|53.07|53.07|528.87|528.87|-1796.13| +2451945|25492|2452002|6363|66395|1553493|1391|32089|22505|1123302|983|19681|3|21|7|1|112|5|13|19.64|34.17|17.76|213.33|230.88|255.32|444.21|6.92|0.00|75.40|230.88|237.80|306.28|313.20|-24.44| +2451945|25492|2451998|3273|66395|1553493|1391|32089|22505|1123302|983|19681|15|17|7|3|259|5|63|9.36|19.74|16.38|211.68|1031.94|589.68|1243.62|35.39|526.28|522.27|505.66|541.05|1027.93|1063.32|-84.02| +2451534|69300|2451633|4879|89701|202409|1164|31472|63718|215362|6843|25951|43|28|1|4|17|6|12|56.04|96.38|89.63|81.00|1075.56|672.48|1156.56|96.80|0.00|69.36|1075.56|1172.36|1144.92|1241.72|403.08| +2451534|69300|2451644|16009|89701|202409|1164|31472|63718|215362|6843|25951|16|25|10|4|18|6|73|36.58|41.70|35.44|456.98|2587.12|2670.34|3044.10|77.61|0.00|1247.57|2587.12|2664.73|3834.69|3912.30|-83.22| +2451534|69300|2451610|7712|89701|202409|1164|31472|63718|215362|6843|25951|58|28|10|5|277|6|91|48.63|65.16|3.25|5633.81|295.75|4425.33|5929.56|2.95|0.00|2489.76|295.75|298.70|2785.51|2788.46|-4129.58| +2451534|69300|2451651|11413|89701|202409|1164|31472|63718|215362|6843|25951|26|10|8|5|42|6|51|42.52|121.60|38.91|4217.19|1984.41|2168.52|6201.60|0.00|754.07|557.94|1230.34|1230.34|1788.28|1788.28|-938.18| +2451534|69300|2451639|14210|89701|202409|1164|31472|63718|215362|6843|25951|8|25|1|4|217|6|1|99.76|206.50|0.00|206.50|0.00|99.76|206.50|0.00|0.00|33.04|0.00|0.00|33.04|33.04|-99.76| +2451534|69300|2451583|6529|89701|202409|1164|31472|63718|215362|6843|25951|52|1|5|1|230|6|83|39.67|47.60|34.74|1067.38|2883.42|3292.61|3950.80|259.50|0.00|1026.71|2883.42|3142.92|3910.13|4169.63|-409.19| +2451534|69300|2451576|13015|89701|202409|1164|31472|63718|215362|6843|25951|37|13|5|3|91|6|63|97.02|121.27|103.07|1146.60|6493.41|6112.26|7640.01|12.98|6233.67|2597.49|259.74|272.72|2857.23|2870.21|-5852.52| +2451534|69300|2451604|15250|89701|202409|1164|31472|63718|215362|6843|25951|38|10|2|3|114|6|11|16.62|23.76|2.61|232.65|28.71|182.82|261.36|2.29|0.00|73.15|28.71|31.00|101.86|104.15|-154.11| +2451178|76807|2451181|12710|48271|1161066|3908|32303|96091|233980|782|16777|2|1|10|2|201|7|47|92.35|229.02|114.51|5381.97|5381.97|4340.45|10763.94|53.81|0.00|4843.35|5381.97|5435.78|10225.32|10279.13|1041.52| +2451178|76807|2451279|7555|48271|1161066|3908|32303|96091|233980|782|16777|1|16|4|2|100|7|44|52.52|126.04|122.25|166.76|5379.00|2310.88|5545.76|430.32|0.00|1219.68|5379.00|5809.32|6598.68|7029.00|3068.12| +2451178|76807|2451240|11416|48271|1161066|3908|32303|96091|233980|782|16777|10|2|15|1|27|7|87|4.47|4.82|3.85|84.39|334.95|388.89|419.34|30.14|0.00|108.75|334.95|365.09|443.70|473.84|-53.94| +2451178|76807|2451210|4856|48271|1161066|3908|32303|96091|233980|782|16777|16|10|3|5|267|7|9|70.18|106.67|44.80|556.83|403.20|631.62|960.03|0.00|249.98|470.34|153.22|153.22|623.56|623.56|-478.40| +2451178|76807|2451201|13342|48271|1161066|3908|32303|96091|233980|782|16777|49|1|20|1|95|7|13|49.14|63.88|31.94|415.22|415.22|638.82|830.44|6.56|87.19|215.80|328.03|334.59|543.83|550.39|-310.79| +2451178|76807|2451287|6628|48271|1161066|3908|32303|96091|233980|782|16777|16|20|6|3|250|7|52|51.52|79.85|59.08|1080.04|3072.16|2679.04|4152.20|0.00|0.00|2034.24|3072.16|3072.16|5106.40|5106.40|393.12| +2451178|76807|2451268|15808|48271|1161066|3908|32303|96091|233980|782|16777|1|8|19|5|24|7|1|64.29|145.93|135.71|10.22|135.71|64.29|145.93|0.00|0.00|72.96|135.71|135.71|208.67|208.67|71.42| +2451178|76807|2451243|1225|48271|1161066|3908|32303|96091|233980|782|16777|2|1|20|1|35|7|44|94.24|201.67|12.10|8341.08|532.40|4146.56|8873.48|21.29|0.00|4170.32|532.40|553.69|4702.72|4724.01|-3614.16| +2452190|51596|2452220|6047|77108|1572482|5456|22830|53825|1895897|5705|33479|41|29|19|3|244|8|76|55.63|72.31|54.23|1374.08|4121.48|4227.88|5495.56|82.42|0.00|164.16|4121.48|4203.90|4285.64|4368.06|-106.40| +2452190|51596|2452307|4603|77108|1572482|5456|22830|53825|1895897|5705|33479|49|21|20|1|15|8|33|31.08|66.20|35.74|1005.18|1179.42|1025.64|2184.60|106.14|0.00|633.27|1179.42|1285.56|1812.69|1918.83|153.78| +2452190|51596|2452216|7563|77108|1572482|5456|22830|53825|1895897|5705|33479|17|19|8|1|244|8|55|45.57|83.84|66.23|968.55|3642.65|2506.35|4611.20|218.55|0.00|1383.25|3642.65|3861.20|5025.90|5244.45|1136.30| +2452190|51596|2452213|9823|77108|1572482|5456|22830|53825|1895897|5705|33479|41|17|14|2|190|8|70|34.71|71.50|50.05|1501.50|3503.50|2429.70|5005.00|0.00|0.00|2051.70|3503.50|3503.50|5555.20|5555.20|1073.80| +2452190|51596|2452242|2247|77108|1572482|5456|22830|53825|1895897|5705|33479|7|13|17|5|60|8|10|28.39|67.00|7.37|596.30|73.70|283.90|670.00|1.47|0.00|180.90|73.70|75.17|254.60|256.07|-210.20| +2452190|51596|2452221|13143|77108|1572482|5456|22830|53825|1895897|5705|33479|59|21|2|3|157|8|87|88.69|201.32|30.19|14888.31|2626.53|7716.03|17514.84|0.00|0.00|1050.09|2626.53|2626.53|3676.62|3676.62|-5089.50| +2452190|51596|2452276|6545|77108|1572482|5456|22830|53825|1895897|5705|33479|25|15|13|3|173|8|22|46.33|81.54|44.84|807.40|986.48|1019.26|1793.88|49.32|0.00|807.18|986.48|1035.80|1793.66|1842.98|-32.78| +2452190|51596|2452305|12675|77108|1572482|5456|22830|53825|1895897|5705|33479|21|3|13|4|148|8|96|79.34|204.69|83.92|11593.92|8056.32|7616.64|19650.24|644.50|0.00|392.64|8056.32|8700.82|8448.96|9093.46|439.68| +2452190|51596|2452267|14961|77108|1572482|5456|22830|53825|1895897|5705|33479|53|19|13|1|169|8|91|44.37|108.70|61.95|4254.25|5637.45|4037.67|9891.70|56.37|0.00|197.47|5637.45|5693.82|5834.92|5891.29|1599.78| +2452190|51596|2452309|9843|77108|1572482|5456|22830|53825|1895897|5705|33479|1|17|4|2|197|8|44|70.93|75.18|10.52|2845.04|462.88|3120.92|3307.92|36.66|4.62|1223.64|458.26|494.92|1681.90|1718.56|-2662.66| +2452190|51596|2452211|2351|77108|1572482|5456|22830|53825|1895897|5705|33479|7|11|18|5|42|8|33|54.70|140.03|137.22|92.73|4528.26|1805.10|4620.99|90.56|0.00|2079.33|4528.26|4618.82|6607.59|6698.15|2723.16| +2452190|51596|2452243|9519|77108|1572482|5456|22830|53825|1895897|5705|33479|33|29|3|1|286|8|59|25.18|37.01|0.37|2161.76|21.83|1485.62|2183.59|0.74|3.27|567.58|18.56|19.30|586.14|586.88|-1467.06| +2452190|51596|2452305|8767|77108|1572482|5456|22830|53825|1895897|5705|33479|17|23|5|3|28|8|64|5.57|7.07|1.83|335.36|117.12|356.48|452.48|0.82|105.40|153.60|11.72|12.54|165.32|166.14|-344.76| +2452190|51596|2452192|7747|77108|1572482|5456|22830|53825|1895897|5705|33479|9|5|11|4|207|8|44|58.29|112.49|22.49|3960.00|989.56|2564.76|4949.56|89.06|0.00|247.28|989.56|1078.62|1236.84|1325.90|-1575.20| +2452190|51596|2452242|5899|77108|1572482|5456|22830|53825|1895897|5705|33479|51|9|17|4|108|8|93|18.56|18.93|3.97|1391.28|369.21|1726.08|1760.49|0.00|0.00|704.01|369.21|369.21|1073.22|1073.22|-1356.87| +2450900|56256|2450988|6169|12676|1815460|4197|8581|21248|1833962|39|14023|4|19|10|2|174|9|16|96.14|256.69|10.26|3942.88|164.16|1538.24|4107.04|0.00|0.00|2053.44|164.16|164.16|2217.60|2217.60|-1374.08| +2450900|56256|2450943|11930|12676|1815460|4197|8581|21248|1833962|39|14023|14|20|17|5|36|9|29|76.54|210.48|134.70|2197.62|3906.30|2219.66|6103.92|156.25|0.00|365.98|3906.30|4062.55|4272.28|4428.53|1686.64| +2450900|56256|2450988|9517|12676|1815460|4197|8581|21248|1833962|39|14023|7|4|2|4|293|9|75|77.80|126.03|113.42|945.75|8506.50|5835.00|9452.25|58.69|2637.01|2646.00|5869.49|5928.18|8515.49|8574.18|34.49| +2450900|56256|2451000|9637|12676|1815460|4197|8581|21248|1833962|39|14023|10|13|11|2|153|9|92|47.71|59.16|33.12|2395.68|3047.04|4389.32|5442.72|60.94|0.00|2012.96|3047.04|3107.98|5060.00|5120.94|-1342.28| +2450900|56256|2450994|13790|12676|1815460|4197|8581|21248|1833962|39|14023|28|16|6|5|217|9|30|84.41|243.10|126.41|3500.70|3792.30|2532.30|7293.00|303.38|0.00|802.20|3792.30|4095.68|4594.50|4897.88|1260.00| +2450900|56256|2450983|14032|12676|1815460|4197|8581|21248|1833962|39|14023|46|7|4|4|131|9|94|93.91|145.56|1.45|13546.34|136.30|8827.54|13682.64|8.17|0.00|2325.56|136.30|144.47|2461.86|2470.03|-8691.24| +2450900|56256|2450902|5780|12676|1815460|4197|8581|21248|1833962|39|14023|32|28|13|3|256|9|79|74.37|135.35|102.86|2566.71|8125.94|5875.23|10692.65|162.51|0.00|1496.26|8125.94|8288.45|9622.20|9784.71|2250.71| +2450900|56256|2451020|8896|12676|1815460|4197|8581|21248|1833962|39|14023|26|14|9|3|267|9|6|10.40|24.85|12.92|71.58|77.52|62.40|149.10|1.55|0.00|41.70|77.52|79.07|119.22|120.77|15.12| +2450900|56256|2450989|5962|12676|1815460|4197|8581|21248|1833962|39|14023|50|7|12|5|178|9|19|34.34|72.11|26.68|863.17|506.92|652.46|1370.09|10.13|0.00|287.66|506.92|517.05|794.58|804.71|-145.54| +2450900|56256|2450971|12166|12676|1815460|4197|8581|21248|1833962|39|14023|31|19|1|5|186|9|10|51.43|153.77|59.97|938.00|599.70|514.30|1537.70|5.99|0.00|445.90|599.70|605.69|1045.60|1051.59|85.40| +2450900|56256|2450922|10303|12676|1815460|4197|8581|21248|1833962|39|14023|4|16|18|2|73|9|50|43.85|83.31|4.99|3916.00|249.50|2192.50|4165.50|7.48|0.00|999.50|249.50|256.98|1249.00|1256.48|-1943.00| +2450900|56256|2450994|4864|12676|1815460|4197|8581|21248|1833962|39|14023|2|10|13|2|184|9|32|6.05|7.19|3.37|122.24|107.84|193.60|230.08|8.62|0.00|78.08|107.84|116.46|185.92|194.54|-85.76| +2450900|56256|2450975|15307|12676|1815460|4197|8581|21248|1833962|39|14023|20|28|9|3|80|9|19|99.99|266.97|248.28|355.11|4717.32|1899.81|5072.43|94.34|0.00|1115.87|4717.32|4811.66|5833.19|5927.53|2817.51| +2450900|56256|2450906|9236|12676|1815460|4197|8581|21248|1833962|39|14023|46|20|18|2|69|9|63|21.25|23.16|21.53|102.69|1356.39|1338.75|1459.08|40.69|0.00|116.55|1356.39|1397.08|1472.94|1513.63|17.64| +2450900|56256|2450980|16741|12676|1815460|4197|8581|21248|1833962|39|14023|40|7|17|2|140|9|5|2.57|5.37|2.04|16.65|10.20|12.85|26.85|0.30|0.00|7.20|10.20|10.50|17.40|17.70|-2.65| +2451821|29187|2451843|6797|1204|1306794|31|21268|98174|67833|6132|42569|56|2|5|3|11|10|39|15.52|43.61|30.52|510.51|1190.28|605.28|1700.79|0.00|1047.44|135.72|142.84|142.84|278.56|278.56|-462.44| +2451821|29187|2451839|13951|1204|1306794|31|21268|98174|67833|6132|42569|55|2|6|5|14|10|11|28.39|52.52|47.79|52.03|525.69|312.29|577.72|15.77|0.00|254.10|525.69|541.46|779.79|795.56|213.40| +2451821|29187|2451902|6146|1204|1306794|31|21268|98174|67833|6132|42569|1|26|17|5|143|10|11|18.69|34.95|19.92|165.33|219.12|205.59|384.45|6.57|0.00|3.74|219.12|225.69|222.86|229.43|13.53| +2451821|29187|2451823|4937|1204|1306794|31|21268|98174|67833|6132|42569|8|17|3|5|44|10|78|70.93|90.08|36.93|4145.70|2880.54|5532.54|7026.24|57.61|0.00|280.80|2880.54|2938.15|3161.34|3218.95|-2652.00| +2451821|29187|2451932|15797|1204|1306794|31|21268|98174|67833|6132|42569|23|1|3|5|14|10|17|18.29|28.71|19.23|161.16|326.91|310.93|488.07|16.34|0.00|4.76|326.91|343.25|331.67|348.01|15.98| +2451821|29187|2451926|12637|1204|1306794|31|21268|98174|67833|6132|42569|53|29|10|3|53|10|62|47.55|87.01|52.20|2158.22|3236.40|2948.10|5394.62|258.91|0.00|2697.00|3236.40|3495.31|5933.40|6192.31|288.30| +2451821|29187|2451904|6605|1204|1306794|31|21268|98174|67833|6132|42569|23|11|7|3|2|10|50|22.47|29.88|21.51|418.50|1075.50|1123.50|1494.00|10.75|0.00|702.00|1075.50|1086.25|1777.50|1788.25|-48.00| +2451821|29187|2451878|10915|1204|1306794|31|21268|98174|67833|6132|42569|20|8|19|3|197|10|77|32.04|54.78|23.00|2447.06|1771.00|2467.08|4218.06|88.55|0.00|1771.00|1771.00|1859.55|3542.00|3630.55|-696.08| +2451821|29187|2451828|2450|1204|1306794|31|21268|98174|67833|6132|42569|14|5|8|5|25|10|5|32.51|88.10|0.00|440.50|0.00|162.55|440.50|0.00|0.00|92.50|0.00|0.00|92.50|92.50|-162.55| +2451879|60291|2451988|11383|77402|480194|5386|17778|12527|1601689|4554|4946|38|11|2|3|61|11|90|7.65|11.16|5.91|472.50|531.90|688.50|1004.40|10.63|0.00|0.00|531.90|542.53|531.90|542.53|-156.60| +2451879|60291|2451965|5690|77402|480194|5386|17778|12527|1601689|4554|4946|55|11|11|4|104|11|80|17.07|40.28|24.57|1256.80|1965.60|1365.60|3222.40|137.59|0.00|1385.60|1965.60|2103.19|3351.20|3488.79|600.00| +2451879|60291|2451917|1729|77402|480194|5386|17778|12527|1601689|4554|4946|47|11|8|5|162|11|84|74.02|184.30|49.76|11301.36|4179.84|6217.68|15481.20|83.59|0.00|1392.72|4179.84|4263.43|5572.56|5656.15|-2037.84| +2451879|60291|2451963|6401|77402|480194|5386|17778|12527|1601689|4554|4946|35|19|2|4|143|11|35|84.47|192.59|136.73|1955.10|4785.55|2956.45|6740.65|28.71|4067.71|1347.85|717.84|746.55|2065.69|2094.40|-2238.61| +2451879|60291|2451990|13879|77402|480194|5386|17778|12527|1601689|4554|4946|25|23|11|3|32|11|81|35.78|81.22|30.05|4144.77|2434.05|2898.18|6578.82|0.00|973.62|2630.88|1460.43|1460.43|4091.31|4091.31|-1437.75| +2451879|60291|2451881|16559|77402|480194|5386|17778|12527|1601689|4554|4946|13|14|12|3|57|11|61|23.14|68.26|25.25|2623.61|1540.25|1411.54|4163.86|34.50|677.71|124.44|862.54|897.04|986.98|1021.48|-549.00| +2451879|60291|2451978|11587|77402|480194|5386|17778|12527|1601689|4554|4946|2|20|20|4|278|11|63|37.68|55.76|10.59|2845.71|667.17|2373.84|3512.88|33.35|0.00|983.43|667.17|700.52|1650.60|1683.95|-1706.67| +2451879|60291|2451984|2701|77402|480194|5386|17778|12527|1601689|4554|4946|43|7|16|3|100|11|9|55.58|58.91|49.48|84.87|445.32|500.22|530.19|35.62|0.00|100.71|445.32|480.94|546.03|581.65|-54.90| +2451879|60291|2451928|7520|77402|480194|5386|17778|12527|1601689|4554|4946|26|19|3|1|42|11|91|65.31|146.94|58.77|8023.47|5348.07|5943.21|13371.54|106.96|0.00|3476.20|5348.07|5455.03|8824.27|8931.23|-595.14| +2451879|60291|2451909|8285|77402|480194|5386|17778|12527|1601689|4554|4946|5|1|19|2|235|11|6|50.74|121.77|118.11|21.96|708.66|304.44|730.62|21.25|0.00|21.90|708.66|729.91|730.56|751.81|404.22| +2452251|49599|2452366|13601|96115|1463304|93|17729|4676|601676|3343|35332|17|23|5|3|21|12|88|9.73|21.60|15.98|494.56|1406.24|856.24|1900.80|6.18|1251.55|893.20|154.69|160.87|1047.89|1054.07|-701.55| +2452251|49599|2452351|10743|96115|1463304|93|17729|4676|601676|3343|35332|23|5|4|3|299|12|41|40.16|107.22|42.88|2637.94|1758.08|1646.56|4396.02|158.22|0.00|219.76|1758.08|1916.30|1977.84|2136.06|111.52| +2452251|49599|2452338|97|96115|1463304|93|17729|4676|601676|3343|35332|21|11|4|2|60|12|39|33.74|70.17|20.34|1943.37|793.26|1315.86|2736.63|8.32|674.27|492.57|118.99|127.31|611.56|619.88|-1196.87| +2452251|49599|2452325|1747|96115|1463304|93|17729|4676|601676|3343|35332|55|7|20|3|59|12|80|11.10|11.10|10.10|80.00|808.00|888.00|888.00|8.08|0.00|408.00|808.00|816.08|1216.00|1224.08|-80.00| +2452251|49599|2452359|11851|96115|1463304|93|17729|4676|601676|3343|35332|55|21|4|4|117|12|8|39.57|96.15|17.30|630.80|138.40|316.56|769.20|4.15|0.00|253.76|138.40|142.55|392.16|396.31|-178.16| +2452251|49599|2452369|2425|96115|1463304|93|17729|4676|601676|3343|35332|45|9|20|2|27|12|65|31.71|58.98|13.56|2952.30|881.40|2061.15|3833.70|61.69|0.00|1379.95|881.40|943.09|2261.35|2323.04|-1179.75| +2452251|49599|2452330|10011|96115|1463304|93|17729|4676|601676|3343|35332|19|7|8|4|23|12|26|41.97|101.14|42.47|1525.42|1104.22|1091.22|2629.64|77.29|0.00|157.56|1104.22|1181.51|1261.78|1339.07|13.00| +2452251|49599|2452317|17899|96115|1463304|93|17729|4676|601676|3343|35332|55|11|14|5|83|12|60|36.20|36.92|5.90|1861.20|354.00|2172.00|2215.20|14.16|0.00|243.60|354.00|368.16|597.60|611.76|-1818.00| +2452251|49599|2452338|8655|96115|1463304|93|17729|4676|601676|3343|35332|5|9|17|2|274|12|60|22.51|55.82|22.88|1976.40|1372.80|1350.60|3349.20|27.45|0.00|602.40|1372.80|1400.25|1975.20|2002.65|22.20| +2451351|75714|2451415|5839|62682|719464|6293|46322|46110|1632520|5162|2391|49|1|4|5|86|13|14|61.61|90.56|72.44|253.68|1014.16|862.54|1267.84|40.56|0.00|215.46|1014.16|1054.72|1229.62|1270.18|151.62| +2451351|75714|2451450|15520|62682|719464|6293|46322|46110|1632520|5162|2391|7|13|20|1|39|13|60|99.12|195.26|85.91|6561.00|5154.60|5947.20|11715.60|8.76|4278.31|5506.20|876.29|885.05|6382.49|6391.25|-5070.91| +2451351|75714|2451401|13735|62682|719464|6293|46322|46110|1632520|5162|2391|16|13|5|3|264|13|50|60.92|69.44|26.38|2153.00|1319.00|3046.00|3472.00|118.71|0.00|416.50|1319.00|1437.71|1735.50|1854.21|-1727.00| +2451351|75714|2451432|7954|62682|719464|6293|46322|46110|1632520|5162|2391|58|22|17|4|281|13|33|24.77|39.63|13.47|863.28|444.51|817.41|1307.79|31.11|0.00|326.70|444.51|475.62|771.21|802.32|-372.90| +2451351|75714|2451470|15424|62682|719464|6293|46322|46110|1632520|5162|2391|52|22|14|4|154|13|11|17.78|26.13|25.34|8.69|278.74|195.58|287.43|19.51|0.00|83.27|278.74|298.25|362.01|381.52|83.16| +2451351|75714|2451411|3235|62682|719464|6293|46322|46110|1632520|5162|2391|20|25|12|4|206|13|45|26.47|33.61|11.76|983.25|529.20|1191.15|1512.45|0.37|523.90|604.80|5.30|5.67|610.10|610.47|-1185.85| +2451351|75714|2451442|16711|62682|719464|6293|46322|46110|1632520|5162|2391|32|25|6|4|22|13|78|48.82|83.97|71.37|982.80|5566.86|3807.96|6549.66|55.66|0.00|1113.06|5566.86|5622.52|6679.92|6735.58|1758.90| +2451351|75714|2451377|13172|62682|719464|6293|46322|46110|1632520|5162|2391|13|4|19|2|208|13|23|85.64|108.76|26.10|1901.18|600.30|1969.72|2501.48|36.01|0.00|0.00|600.30|636.31|600.30|636.31|-1369.42| +2451351|75714|2451354|10610|62682|719464|6293|46322|46110|1632520|5162|2391|26|26|7|5|90|13|22|73.91|108.64|56.49|1147.30|1242.78|1626.02|2390.08|14.91|1056.36|740.74|186.42|201.33|927.16|942.07|-1439.60| +2452224|44494|2452292|3255|85452|861000|2948|39343|23682|678237|1587|16335|3|11|10|4|175|14|77|80.72|151.75|100.15|3973.20|7711.55|6215.44|11684.75|154.23|0.00|4907.21|7711.55|7865.78|12618.76|12772.99|1496.11| +2452224|44494|2452253|5667|85452|861000|2948|39343|23682|678237|1587|16335|23|9|18|1|247|14|20|66.38|106.87|101.52|107.00|2030.40|1327.60|2137.40|182.73|0.00|384.60|2030.40|2213.13|2415.00|2597.73|702.80| +2452224|44494|2452337|9797|85452|861000|2948|39343|23682|678237|1587|16335|49|7|6|5|204|14|62|10.82|24.88|2.73|1373.30|169.26|670.84|1542.56|1.69|0.00|107.88|169.26|170.95|277.14|278.83|-501.58| +2452224|44494|2452333|9305|85452|861000|2948|39343|23682|678237|1587|16335|11|25|20|2|184|14|36|78.32|127.66|14.04|4090.32|505.44|2819.52|4595.76|20.21|0.00|1194.84|505.44|525.65|1700.28|1720.49|-2314.08| +2452224|44494|2452272|12933|85452|861000|2948|39343|23682|678237|1587|16335|15|13|9|3|57|14|82|60.20|89.09|68.59|1681.00|5624.38|4936.40|7305.38|281.21|0.00|2702.72|5624.38|5905.59|8327.10|8608.31|687.98| +2452224|44494|2452272|9991|85452|861000|2948|39343|23682|678237|1587|16335|21|9|9|3|47|14|54|18.34|36.12|4.69|1697.22|253.26|990.36|1950.48|10.13|0.00|838.62|253.26|263.39|1091.88|1102.01|-737.10| +2452224|44494|2452322|13531|85452|861000|2948|39343|23682|678237|1587|16335|33|21|18|2|106|14|47|48.25|127.86|52.42|3545.68|2463.74|2267.75|6009.42|73.91|0.00|480.34|2463.74|2537.65|2944.08|3017.99|195.99| +2452224|44494|2452275|17957|85452|861000|2948|39343|23682|678237|1587|16335|35|15|18|3|98|14|21|77.06|97.86|17.61|1685.25|369.81|1618.26|2055.06|7.39|0.00|267.12|369.81|377.20|636.93|644.32|-1248.45| +2452224|44494|2452341|2195|85452|861000|2948|39343|23682|678237|1587|16335|25|17|12|5|138|14|33|69.15|132.07|101.69|1002.54|3355.77|2281.95|4358.31|268.46|0.00|2048.31|3355.77|3624.23|5404.08|5672.54|1073.82| +2452224|44494|2452334|12055|85452|861000|2948|39343|23682|678237|1587|16335|17|13|18|1|24|14|17|39.48|93.56|43.97|843.03|747.49|671.16|1590.52|52.32|0.00|699.72|747.49|799.81|1447.21|1499.53|76.33| +2452224|44494|2452310|14737|85452|861000|2948|39343|23682|678237|1587|16335|23|11|13|2|101|14|89|76.89|180.69|83.11|8684.62|7396.79|6843.21|16081.41|147.93|0.00|3055.37|7396.79|7544.72|10452.16|10600.09|553.58| +2452224|44494|2452252|8823|85452|861000|2948|39343|23682|678237|1587|16335|9|9|15|5|245|14|21|85.86|179.44|111.25|1431.99|2336.25|1803.06|3768.24|70.08|0.00|866.67|2336.25|2406.33|3202.92|3273.00|533.19| +2452224|44494|2452271|13363|85452|861000|2948|39343|23682|678237|1587|16335|25|19|14|3|82|14|63|37.23|80.41|53.87|1672.02|3393.81|2345.49|5065.83|57.01|543.00|1063.44|2850.81|2907.82|3914.25|3971.26|505.32| +2452224|44494|2452326|10351|85452|861000|2948|39343|23682|678237|1587|16335|3|13|2|2|262|14|55|76.31|110.64|61.95|2677.95|3407.25|4197.05|6085.20|68.14|0.00|2920.50|3407.25|3475.39|6327.75|6395.89|-789.80| +2452224|44494|2452334|9733|85452|861000|2948|39343|23682|678237|1587|16335|59|13|14|1|208|14|51|62.40|133.53|26.70|5448.33|1361.70|3182.40|6810.03|81.70|0.00|3404.76|1361.70|1443.40|4766.46|4848.16|-1820.70| +2452224|44494|2452280|2585|85452|861000|2948|39343|23682|678237|1587|16335|21|17|14|2|287|14|95|23.49|46.74|1.86|4263.60|176.70|2231.55|4440.30|7.06|0.00|2220.15|176.70|183.76|2396.85|2403.91|-2054.85| +2451405|67593|2451414|8326|363|3898|586|36874|85171|612056|5032|41320|28|14|11|4|257|15|45|60.01|106.21|105.14|48.15|4731.30|2700.45|4779.45|141.93|0.00|1433.70|4731.30|4873.23|6165.00|6306.93|2030.85| +2451405|67593|2451508|11812|363|3898|586|36874|85171|612056|5032|41320|49|14|4|2|3|15|71|16.54|20.67|6.20|1027.37|440.20|1174.34|1467.57|35.21|0.00|308.14|440.20|475.41|748.34|783.55|-734.14| +2451405|67593|2451459|13489|363|3898|586|36874|85171|612056|5032|41320|4|2|18|4|283|15|21|83.67|190.76|87.74|2163.42|1842.54|1757.07|4005.96|110.55|0.00|1361.85|1842.54|1953.09|3204.39|3314.94|85.47| +2451405|67593|2451517|11924|363|3898|586|36874|85171|612056|5032|41320|56|2|15|3|221|15|56|49.27|104.45|47.00|3217.20|2632.00|2759.12|5849.20|131.60|0.00|2632.00|2632.00|2763.60|5264.00|5395.60|-127.12| +2451405|67593|2451455|12512|363|3898|586|36874|85171|612056|5032|41320|25|19|7|4|91|15|62|32.09|74.76|12.70|3847.72|787.40|1989.58|4635.12|15.74|0.00|2131.56|787.40|803.14|2918.96|2934.70|-1202.18| +2451405|67593|2451490|6355|363|3898|586|36874|85171|612056|5032|41320|31|19|14|5|149|15|57|73.99|181.27|74.32|6096.15|4236.24|4217.43|10332.39|254.17|0.00|3202.83|4236.24|4490.41|7439.07|7693.24|18.81| +2451405|67593|2451449|2542|363|3898|586|36874|85171|612056|5032|41320|1|26|7|5|53|15|100|57.64|69.74|39.75|2999.00|3975.00|5764.00|6974.00|318.00|0.00|139.00|3975.00|4293.00|4114.00|4432.00|-1789.00| +2451405|67593|2451518|4933|363|3898|586|36874|85171|612056|5032|41320|1|26|17|2|120|15|56|16.59|21.73|13.68|450.80|766.08|929.04|1216.88|61.28|0.00|72.80|766.08|827.36|838.88|900.16|-162.96| +2451405|67593|2451454|7576|363|3898|586|36874|85171|612056|5032|41320|7|10|11|3|298|15|28|7.92|14.41|13.11|36.40|367.08|221.76|403.48|3.67|0.00|68.32|367.08|370.75|435.40|439.07|145.32| +2451405|67593|2451506|14248|363|3898|586|36874|85171|612056|5032|41320|44|22|1|3|122|15|94|4.94|12.25|3.06|863.86|287.64|464.36|1151.50|0.00|0.00|79.90|287.64|287.64|367.54|367.54|-176.72| +2451405|67593|2451426|3151|363|3898|586|36874|85171|612056|5032|41320|26|26|14|4|224|15|74|24.48|66.58|64.58|148.00|4778.92|1811.52|4926.92|95.57|0.00|2315.46|4778.92|4874.49|7094.38|7189.95|2967.40| +2451405|67593|2451475|6506|363|3898|586|36874|85171|612056|5032|41320|2|13|15|5|98|15|5|46.13|63.19|1.89|306.50|9.45|230.65|315.95|0.00|9.16|6.30|0.29|0.29|6.59|6.59|-230.36| +2452342|70397|2452347|15786|21567|477008|4096|3937|43522|468964|2307|36195|49|6|1|4|147|16|59|87.34|133.63|44.09|5282.86|2601.31|5153.06|7884.17|208.10|0.00|1813.07|2601.31|2809.41|4414.38|4622.48|-2551.75| +2452342|70397|2452432|16849|21567|477008|4096|3937|43522|468964|2307|36195|15|1|17|5|86|16|37|14.92|22.23|11.11|411.44|411.07|552.04|822.51|10.19|156.20|279.35|254.87|265.06|534.22|544.41|-297.17| +2452342|70397|2452402|5107|21567|477008|4096|3937|43522|468964|2307|36195|42|24|13|4|52|16|54|24.87|64.91|36.99|1507.68|1997.46|1342.98|3505.14|139.82|0.00|1086.48|1997.46|2137.28|3083.94|3223.76|654.48| +2452342|70397|2452406|1146|21567|477008|4096|3937|43522|468964|2307|36195|6|27|8|4|160|16|59|41.64|65.79|55.92|582.33|3299.28|2456.76|3881.61|32.99|0.00|232.46|3299.28|3332.27|3531.74|3564.73|842.52| +2452342|70397|2452379|8895|21567|477008|4096|3937|43522|468964|2307|36195|37|7|6|3|123|16|2|94.72|115.55|80.88|69.34|161.76|189.44|231.10|9.70|0.00|50.84|161.76|171.46|212.60|222.30|-27.68| +2452342|70397|2452425|10447|21567|477008|4096|3937|43522|468964|2307|36195|49|24|16|3|137|16|29|80.40|149.54|44.86|3035.72|1300.94|2331.60|4336.66|91.06|0.00|737.18|1300.94|1392.00|2038.12|2129.18|-1030.66| +2452342|70397|2452378|13224|21567|477008|4096|3937|43522|468964|2307|36195|39|27|18|2|146|16|57|59.56|109.59|31.78|4435.17|1811.46|3394.92|6246.63|108.68|0.00|1124.04|1811.46|1920.14|2935.50|3044.18|-1583.46| +2452342|70397|2452423|5721|21567|477008|4096|3937|43522|468964|2307|36195|36|9|12|4|146|16|22|50.53|55.58|3.33|1149.50|73.26|1111.66|1222.76|0.02|72.52|232.32|0.74|0.76|233.06|233.08|-1110.92| +2452386|7275|2452453|14461|93708|166860|5612|38411|33483|816413|3114|37113|51|30|7|1|245|17|14|28.04|62.52|38.76|332.64|542.64|392.56|875.28|23.17|211.62|437.64|331.02|354.19|768.66|791.83|-61.54| +2452386|7275|2452466|13263|93708|166860|5612|38411|33483|816413|3114|37113|33|7|11|3|115|17|64|53.31|100.22|51.11|3143.04|3271.04|3411.84|6414.08|196.26|0.00|1667.20|3271.04|3467.30|4938.24|5134.50|-140.80| +2452386|7275|2452501|4512|93708|166860|5612|38411|33483|816413|3114|37113|21|25|2|1|280|17|13|98.13|255.13|35.71|2852.46|464.23|1275.69|3316.69|9.28|0.00|1658.28|464.23|473.51|2122.51|2131.79|-811.46| +2452386|7275|2452449|5703|93708|166860|5612|38411|33483|816413|3114|37113|30|30|12|2|239|17|82|34.74|98.66|6.90|7524.32|565.80|2848.68|8090.12|50.92|0.00|3639.98|565.80|616.72|4205.78|4256.70|-2282.88| +2452386|7275|2452413|16303|93708|166860|5612|38411|33483|816413|3114|37113|21|9|9|3|197|17|100|59.08|76.21|10.66|6555.00|1066.00|5908.00|7621.00|63.96|0.00|3505.00|1066.00|1129.96|4571.00|4634.96|-4842.00| +2452386|7275|2452387|6657|93708|166860|5612|38411|33483|816413|3114|37113|21|12|8|3|88|17|20|78.32|151.15|75.57|1511.60|1511.40|1566.40|3023.00|30.22|0.00|665.00|1511.40|1541.62|2176.40|2206.62|-55.00| +2452386|7275|2452404|972|93708|166860|5612|38411|33483|816413|3114|37113|48|9|16|2|188|17|82|78.80|187.54|61.88|10304.12|5074.16|6461.60|15378.28|202.96|0.00|922.50|5074.16|5277.12|5996.66|6199.62|-1387.44| +2452386|7275|2452441|14377|93708|166860|5612|38411|33483|816413|3114|37113|7|19|6|5|21|17|90|79.43|114.37|18.29|8647.20|1646.10|7148.70|10293.30|98.76|0.00|0.00|1646.10|1744.86|1646.10|1744.86|-5502.60| +2452386|7275|2452438|14581|93708|166860|5612|38411|33483|816413|3114|37113|39|18|11|3|232|17|7|58.74|61.67|6.16|388.57|43.12|411.18|431.69|0.97|23.71|176.96|19.41|20.38|196.37|197.34|-391.77| +2451422|33648|2451514|2096|4238|1802467|6743|172|86677|1630659|4106|17182|43|22|19|4|169|18|97|45.06|130.67|53.57|7478.70|5196.29|4370.82|12674.99|103.92|0.00|6337.01|5196.29|5300.21|11533.30|11637.22|825.47| +2451422|33648|2451484|15748|4238|1802467|6743|172|86677|1630659|4106|17182|14|10|1|1|261|18|77|72.83|135.46|27.09|8344.49|2085.93|5607.91|10430.42|0.00|0.00|2294.60|2085.93|2085.93|4380.53|4380.53|-3521.98| +2451422|33648|2451468|667|4238|1802467|6743|172|86677|1630659|4106|17182|49|13|9|1|186|18|70|29.06|60.15|10.82|3453.10|757.40|2034.20|4210.50|8.48|651.36|631.40|106.04|114.52|737.44|745.92|-1928.16| +2451422|33648|2451446|3130|4238|1802467|6743|172|86677|1630659|4106|17182|26|13|20|3|166|18|83|2.78|6.28|4.64|136.12|385.12|230.74|521.24|30.80|0.00|9.96|385.12|415.92|395.08|425.88|154.38| +2451422|33648|2451510|3979|4238|1802467|6743|172|86677|1630659|4106|17182|46|19|6|2|268|18|100|94.73|161.04|27.37|13367.00|2737.00|9473.00|16104.00|82.11|0.00|322.00|2737.00|2819.11|3059.00|3141.11|-6736.00| +2451422|33648|2451488|14911|4238|1802467|6743|172|86677|1630659|4106|17182|40|19|16|5|51|18|12|87.76|200.09|148.06|624.36|1776.72|1053.12|2401.08|35.53|0.00|192.00|1776.72|1812.25|1968.72|2004.25|723.60| +2451422|33648|2451425|8533|4238|1802467|6743|172|86677|1630659|4106|17182|16|2|5|3|131|18|62|40.23|48.67|19.46|1811.02|1206.52|2494.26|3017.54|48.26|0.00|271.56|1206.52|1254.78|1478.08|1526.34|-1287.74| +2451422|33648|2451527|14008|4238|1802467|6743|172|86677|1630659|4106|17182|7|20|8|3|265|18|52|6.61|10.90|8.61|119.08|447.72|343.72|566.80|18.17|188.04|135.72|259.68|277.85|395.40|413.57|-84.04| +2451422|33648|2451429|16834|4238|1802467|6743|172|86677|1630659|4106|17182|32|20|12|3|254|18|78|50.44|89.27|1.78|6824.22|138.84|3934.32|6963.06|8.33|0.00|2018.64|138.84|147.17|2157.48|2165.81|-3795.48| +2452616|42506|2452627|6168|57659|1154202|7098|44794|66274|257323|2776|72|33|12|2|3|137|19|77|45.57|61.97|28.50|2577.19|2194.50|3508.89|4771.69|65.83|0.00|381.15|2194.50|2260.33|2575.65|2641.48|-1314.39| +2452616|42506|2452672|13206|57659|1154202|7098|44794|66274|257323|2776|72|39|6|6|4|42|19|7|89.03|121.08|107.76|93.24|754.32|623.21|847.56|15.08|0.00|364.42|754.32|769.40|1118.74|1133.82|131.11| +2452616|42506|2452629|1614|57659|1154202|7098|44794|66274|257323|2776|72|21|24|16|5|181|19|94|5.59|5.81|4.88|87.42|458.72|525.46|546.14|0.00|0.00|109.04|458.72|458.72|567.76|567.76|-66.74| +2452616|42506|2452679|5625|57659|1154202|7098|44794|66274|257323|2776|72|3|1|10|2|51|19|80|26.60|36.70|0.73|2877.60|58.40|2128.00|2936.00|4.67|0.00|997.60|58.40|63.07|1056.00|1060.67|-2069.60| +2452616|42506|2452710|4725|57659|1154202|7098|44794|66274|257323|2776|72|33|19|3|1|192|19|64|48.61|72.91|10.93|3966.72|699.52|3111.04|4666.24|0.00|125.91|652.80|573.61|573.61|1226.41|1226.41|-2537.43| +2452616|42506|2452631|3513|57659|1154202|7098|44794|66274|257323|2776|72|57|12|16|1|36|19|7|10.98|28.43|25.87|17.92|181.09|76.86|199.01|12.67|0.00|71.61|181.09|193.76|252.70|265.37|104.23| +2452616|42506|2452715|8749|57659|1154202|7098|44794|66274|257323|2776|72|25|30|17|4|21|19|5|12.83|31.17|7.79|116.90|38.95|64.15|155.85|0.77|0.00|71.65|38.95|39.72|110.60|111.37|-25.20| +2452616|42506|2452730|7059|57659|1154202|7098|44794|66274|257323|2776|72|45|13|1|5|229|19|13|24.50|69.09|68.39|9.10|889.07|318.50|898.17|8.89|0.00|8.97|889.07|897.96|898.04|906.93|570.57| +2452616|42506|2452692|14997|57659|1154202|7098|44794|66274|257323|2776|72|37|9|1|2|203|19|51|86.10|102.45|61.47|2089.98|3134.97|4391.10|5224.95|31.34|0.00|1149.03|3134.97|3166.31|4284.00|4315.34|-1256.13| +2452616|42506|2452728|9345|57659|1154202|7098|44794|66274|257323|2776|72|43|13|12|2|89|19|51|7.02|9.89|5.73|212.16|292.23|358.02|504.39|2.92|0.00|166.26|292.23|295.15|458.49|461.41|-65.79| +2452616|42506|2452721|6288|57659|1154202|7098|44794|66274|257323|2776|72|19|24|4|3|265|19|52|41.66|112.06|45.94|3438.24|2388.88|2166.32|5827.12|71.66|0.00|465.92|2388.88|2460.54|2854.80|2926.46|222.56| +2452616|42506|2452653|17221|57659|1154202|7098|44794|66274|257323|2776|72|13|25|9|5|121|19|100|2.60|4.13|0.41|372.00|41.00|260.00|413.00|2.05|0.00|152.00|41.00|43.05|193.00|195.05|-219.00| +2452616|42506|2452639|7623|57659|1154202|7098|44794|66274|257323|2776|72|55|1|1|4|84|19|24|24.55|49.10|12.76|872.16|306.24|589.20|1178.40|6.12|0.00|365.28|306.24|312.36|671.52|677.64|-282.96| +2452616|42506|2452665|9475|57659|1154202|7098|44794|66274|257323|2776|72|27|27|7|2|300|19|62|41.84|122.59|84.58|2356.62|5243.96|2594.08|7600.58|367.07|0.00|2736.06|5243.96|5611.03|7980.02|8347.09|2649.88| +2452616|42506|2452725|17175|57659|1154202|7098|44794|66274|257323|2776|72|37|7|9|4|169|19|43|26.54|70.06|25.92|1898.02|1114.56|1141.22|3012.58|11.14|0.00|1174.76|1114.56|1125.70|2289.32|2300.46|-26.66| +2451576|44795|2451594|10861|76619|942296|2020|13697|44747|484545|5528|1605|19|5|19|2|275|20|68|48.01|52.81|7.92|3052.52|538.56|3264.68|3591.08|48.47|0.00|1328.04|538.56|587.03|1866.60|1915.07|-2726.12| +2451576|44795|2451679|11627|76619|942296|2020|13697|44747|484545|5528|1605|43|25|14|5|129|20|55|7.39|18.03|5.22|704.55|287.10|406.45|991.65|17.22|0.00|277.20|287.10|304.32|564.30|581.52|-119.35| +2451576|44795|2451598|9169|76619|942296|2020|13697|44747|484545|5528|1605|11|23|8|4|7|20|15|73.98|221.94|137.60|1265.10|2064.00|1109.70|3329.10|61.92|0.00|366.15|2064.00|2125.92|2430.15|2492.07|954.30| +2451576|44795|2451690|16862|76619|942296|2020|13697|44747|484545|5528|1605|49|2|19|1|90|20|81|53.11|152.42|74.68|6296.94|6049.08|4301.91|12346.02|60.49|0.00|6049.08|6049.08|6109.57|12098.16|12158.65|1747.17| +2451576|44795|2451607|17935|76619|942296|2020|13697|44747|484545|5528|1605|8|7|13|2|98|20|1|59.35|135.31|13.53|121.78|13.53|59.35|135.31|1.21|0.00|60.88|13.53|14.74|74.41|75.62|-45.82| +2451576|44795|2451633|12731|76619|942296|2020|13697|44747|484545|5528|1605|7|13|20|3|46|20|69|81.98|237.74|16.64|15255.90|1148.16|5656.62|16404.06|103.33|0.00|5413.05|1148.16|1251.49|6561.21|6664.54|-4508.46| +2451576|44795|2451644|12902|76619|942296|2020|13697|44747|484545|5528|1605|35|7|20|1|52|20|97|5.42|13.49|0.00|1308.53|0.00|525.74|1308.53|0.00|0.00|549.02|0.00|0.00|549.02|549.02|-525.74| +2451576|44795|2451629|4214|76619|942296|2020|13697|44747|484545|5528|1605|38|7|8|2|286|20|93|30.19|47.70|19.08|2661.66|1774.44|2807.67|4436.10|29.27|798.49|2173.41|975.95|1005.22|3149.36|3178.63|-1831.72| +2451576|44795|2451682|182|76619|942296|2020|13697|44747|484545|5528|1605|29|14|14|2|220|20|98|73.29|199.34|63.78|13284.88|6250.44|7182.42|19535.32|125.00|0.00|5078.36|6250.44|6375.44|11328.80|11453.80|-931.98| +2451576|44795|2451601|13207|76619|942296|2020|13697|44747|484545|5528|1605|55|8|2|4|206|20|26|61.93|114.57|25.20|2323.62|655.20|1610.18|2978.82|39.31|0.00|804.18|655.20|694.51|1459.38|1498.69|-954.98| +2451576|44795|2451641|17396|76619|942296|2020|13697|44747|484545|5528|1605|49|17|13|2|111|20|36|38.67|100.15|76.11|865.44|2739.96|1392.12|3605.40|219.19|0.00|1153.44|2739.96|2959.15|3893.40|4112.59|1347.84| +2451576|44795|2451586|11947|76619|942296|2020|13697|44747|484545|5528|1605|49|26|20|4|25|20|11|9.24|13.95|0.13|152.02|1.43|101.64|153.45|0.04|0.00|16.83|1.43|1.47|18.26|18.30|-100.21| +2452580|26067|2452637|8988|94484|771125|5366|44407|94484|771125|5366|44407|9|18|20|4|208|21|96|95.27|191.49|44.04|14155.20|4227.84|9145.92|18383.04|253.67|0.00|1469.76|4227.84|4481.51|5697.60|5951.27|-4918.08| +2452580|26067|2452626|4747|94484|771125|5366|44407|94484|771125|5366|44407|55|21|14|5|13|21|54|40.63|63.38|62.11|68.58|3353.94|2194.02|3422.52|167.69|0.00|684.18|3353.94|3521.63|4038.12|4205.81|1159.92| +2452580|26067|2452696|5886|94484|771125|5366|44407|94484|771125|5366|44407|27|27|7|2|30|21|25|51.53|53.07|19.10|849.25|477.50|1288.25|1326.75|15.66|85.95|344.75|391.55|407.21|736.30|751.96|-896.70| +2452580|26067|2452613|12723|94484|771125|5366|44407|94484|771125|5366|44407|24|9|5|5|28|21|33|29.50|71.98|1.43|2328.15|47.19|973.50|2375.34|0.00|0.00|23.43|47.19|47.19|70.62|70.62|-926.31| +2452580|26067|2452625|297|94484|771125|5366|44407|94484|771125|5366|44407|48|12|1|3|125|21|84|4.13|4.25|3.23|85.68|271.32|346.92|357.00|5.42|0.00|85.68|271.32|276.74|357.00|362.42|-75.60| +2452580|26067|2452665|2941|94484|771125|5366|44407|94484|771125|5366|44407|25|24|8|2|293|21|39|43.71|47.64|33.82|538.98|1318.98|1704.69|1857.96|10.28|290.17|408.72|1028.81|1039.09|1437.53|1447.81|-675.88| +2452580|26067|2452661|16111|94484|771125|5366|44407|94484|771125|5366|44407|33|15|16|4|182|21|21|14.11|36.54|16.80|414.54|352.80|296.31|767.34|28.22|0.00|329.91|352.80|381.02|682.71|710.93|56.49| +2452580|26067|2452687|15121|94484|771125|5366|44407|94484|771125|5366|44407|13|6|19|5|51|21|77|92.60|99.08|13.87|6561.17|1067.99|7130.20|7629.16|67.49|224.27|2440.90|843.72|911.21|3284.62|3352.11|-6286.48| +2452580|26067|2452586|16717|94484|771125|5366|44407|94484|771125|5366|44407|42|25|12|4|273|21|41|20.34|39.05|28.50|432.55|1168.50|833.94|1601.05|105.16|0.00|0.00|1168.50|1273.66|1168.50|1273.66|334.56| +2451176|80914|2451251|6176|11428|674591|5401|8164|93398|771622|4900|42910|22|28|1|2|153|22|94|11.47|30.62|2.75|2619.78|258.50|1078.18|2878.28|10.34|0.00|517.94|258.50|268.84|776.44|786.78|-819.68| +2451176|80914|2451181|16384|11428|674591|5401|8164|93398|771622|4900|42910|40|4|15|2|249|22|73|70.12|100.27|56.15|3220.76|4098.95|5118.76|7319.71|204.94|0.00|1170.92|4098.95|4303.89|5269.87|5474.81|-1019.81| +2451176|80914|2451197|2276|11428|674591|5401|8164|93398|771622|4900|42910|22|28|18|1|264|22|60|28.39|38.61|38.22|23.40|2293.20|1703.40|2316.60|45.86|0.00|648.60|2293.20|2339.06|2941.80|2987.66|589.80| +2451176|80914|2451241|9493|11428|674591|5401|8164|93398|771622|4900|42910|58|20|19|5|208|22|48|25.97|61.28|60.66|29.76|2911.68|1246.56|2941.44|145.58|0.00|911.52|2911.68|3057.26|3823.20|3968.78|1665.12| +2451176|80914|2451252|2288|11428|674591|5401|8164|93398|771622|4900|42910|52|10|8|5|79|22|23|17.12|49.30|31.05|419.75|714.15|393.76|1133.90|0.00|0.00|238.05|714.15|714.15|952.20|952.20|320.39| +2451176|80914|2451245|338|11428|674591|5401|8164|93398|771622|4900|42910|20|26|10|3|189|22|77|46.14|114.42|62.93|3964.73|4845.61|3552.78|8810.34|145.36|0.00|880.88|4845.61|4990.97|5726.49|5871.85|1292.83| +2451176|80914|2451188|10759|11428|674591|5401|8164|93398|771622|4900|42910|22|16|16|3|120|22|61|54.61|82.46|8.24|4527.42|502.64|3331.21|5030.06|13.47|165.87|1156.56|336.77|350.24|1493.33|1506.80|-2994.44| +2451176|80914|2451243|3160|11428|674591|5401|8164|93398|771622|4900|42910|46|1|18|5|232|22|56|31.95|75.08|24.02|2859.36|1345.12|1789.20|4204.48|80.70|0.00|1513.12|1345.12|1425.82|2858.24|2938.94|-444.08| +2451176|80914|2451214|7922|11428|674591|5401|8164|93398|771622|4900|42910|2|16|16|5|67|22|17|12.11|19.98|7.19|217.43|122.23|205.87|339.66|9.77|0.00|108.63|122.23|132.00|230.86|240.63|-83.64| +2451176|80914|2451252|7130|11428|674591|5401|8164|93398|771622|4900|42910|34|28|5|2|292|22|95|93.80|153.83|0.00|14613.85|0.00|8911.00|14613.85|0.00|0.00|4529.60|0.00|0.00|4529.60|4529.60|-8911.00| +2451176|80914|2451265|8936|11428|674591|5401|8164|93398|771622|4900|42910|38|10|5|4|10|22|69|60.00|169.20|59.22|7588.62|4086.18|4140.00|11674.80|94.79|1716.19|2451.57|2369.99|2464.78|4821.56|4916.35|-1770.01| +2451176|80914|2451205|4015|11428|674591|5401|8164|93398|771622|4900|42910|52|10|12|1|160|22|41|16.08|47.75|14.32|1370.63|587.12|659.28|1957.75|46.96|0.00|489.13|587.12|634.08|1076.25|1123.21|-72.16| +2451176|80914|2451184|5860|11428|674591|5401|8164|93398|771622|4900|42910|44|16|15|3|34|22|96|75.08|191.45|72.75|11395.20|6984.00|7207.68|18379.20|139.68|0.00|8453.76|6984.00|7123.68|15437.76|15577.44|-223.68| +2451103|82709|2451163|2984|49487|1494913|1492|10871|87709|790511|3455|14834|32|28|3|1|279|23|28|93.59|181.56|137.98|1220.24|3863.44|2620.52|5083.68|0.00|0.00|1575.84|3863.44|3863.44|5439.28|5439.28|1242.92| +2451103|82709|2451211|3211|49487|1494913|1492|10871|87709|790511|3455|14834|52|14|16|4|85|23|17|76.74|177.26|141.80|602.82|2410.60|1304.58|3013.42|216.95|0.00|903.89|2410.60|2627.55|3314.49|3531.44|1106.02| +2451103|82709|2451194|13238|49487|1494913|1492|10871|87709|790511|3455|14834|40|14|15|2|269|23|65|72.31|84.60|21.99|4069.65|1429.35|4700.15|5499.00|85.76|0.00|2529.15|1429.35|1515.11|3958.50|4044.26|-3270.80| +2451103|82709|2451122|17275|49487|1494913|1492|10871|87709|790511|3455|14834|19|19|20|5|227|23|98|28.06|46.57|35.39|1095.64|3468.22|2749.88|4563.86|0.00|0.00|1003.52|3468.22|3468.22|4471.74|4471.74|718.34| +2451103|82709|2451150|9319|49487|1494913|1492|10871|87709|790511|3455|14834|40|14|5|2|299|23|44|26.04|27.34|9.29|794.20|408.76|1145.76|1202.96|36.78|0.00|384.56|408.76|445.54|793.32|830.10|-737.00| +2451103|82709|2451181|15206|49487|1494913|1492|10871|87709|790511|3455|14834|13|20|5|2|136|23|43|97.87|275.01|239.25|1537.68|10287.75|4208.41|11825.43|102.87|0.00|1773.75|10287.75|10390.62|12061.50|12164.37|6079.34| +2451103|82709|2451210|17566|49487|1494913|1492|10871|87709|790511|3455|14834|52|7|11|3|55|23|66|39.45|108.09|19.45|5850.24|1283.70|2603.70|7133.94|51.34|0.00|998.58|1283.70|1335.04|2282.28|2333.62|-1320.00| +2451103|82709|2451204|10165|49487|1494913|1492|10871|87709|790511|3455|14834|56|25|4|3|212|23|67|73.95|91.69|57.76|2273.31|3869.92|4954.65|6143.23|154.79|0.00|2579.50|3869.92|4024.71|6449.42|6604.21|-1084.73| +2451103|82709|2451209|10820|49487|1494913|1492|10871|87709|790511|3455|14834|14|10|11|4|224|23|89|30.66|80.94|1.61|7060.37|143.29|2728.74|7203.66|2.86|0.00|575.83|143.29|146.15|719.12|721.98|-2585.45| +2451103|82709|2451197|5684|49487|1494913|1492|10871|87709|790511|3455|14834|1|19|12|4|46|23|42|83.16|216.21|30.26|7809.90|1270.92|3492.72|9080.82|50.83|0.00|272.16|1270.92|1321.75|1543.08|1593.91|-2221.80| +2451103|82709|2451149|17750|49487|1494913|1492|10871|87709|790511|3455|14834|34|22|6|3|137|23|69|57.14|124.56|39.85|5844.99|2749.65|3942.66|8594.64|27.49|0.00|2320.47|2749.65|2777.14|5070.12|5097.61|-1193.01| +2451103|82709|2451165|13312|49487|1494913|1492|10871|87709|790511|3455|14834|7|19|6|5|21|23|35|76.29|226.58|149.54|2696.40|5233.90|2670.15|7930.30|366.37|0.00|1110.20|5233.90|5600.27|6344.10|6710.47|2563.75| +2451103|82709|2451200|6910|49487|1494913|1492|10871|87709|790511|3455|14834|32|13|20|3|226|23|92|28.87|66.11|57.51|791.20|5290.92|2656.04|6082.12|476.18|0.00|1398.40|5290.92|5767.10|6689.32|7165.50|2634.88| +2451103|82709|2451158|8020|49487|1494913|1492|10871|87709|790511|3455|14834|20|8|9|3|10|23|40|55.03|69.88|69.88|0.00|2795.20|2201.20|2795.20|195.66|0.00|866.40|2795.20|2990.86|3661.60|3857.26|594.00| +2451103|82709|2451178|494|49487|1494913|1492|10871|87709|790511|3455|14834|32|8|16|4|232|23|11|16.33|23.84|20.74|34.10|228.14|179.63|262.24|0.00|0.00|65.56|228.14|228.14|293.70|293.70|48.51| +2451103|82709|2451198|15157|49487|1494913|1492|10871|87709|790511|3455|14834|19|7|5|4|240|23|95|24.25|45.83|4.12|3962.45|391.40|2303.75|4353.85|31.31|0.00|1131.45|391.40|422.71|1522.85|1554.16|-1912.35| +2452102|35718|2452189|141|9167|1219556|4698|9887|29071|1421648|5454|5396|19|3|1|5|269|24|89|72.85|135.50|89.43|4100.23|7959.27|6483.65|12059.50|557.14|0.00|2532.05|7959.27|8516.41|10491.32|11048.46|1475.62| +2452102|35718|2452140|1327|9167|1219556|4698|9887|29071|1421648|5454|5396|59|27|11|5|139|24|61|63.48|128.22|57.69|4302.33|3519.09|3872.28|7821.42|70.38|0.00|3050.00|3519.09|3589.47|6569.09|6639.47|-353.19| +2452102|35718|2452179|247|9167|1219556|4698|9887|29071|1421648|5454|5396|19|25|6|4|172|24|52|57.06|81.02|34.02|2444.00|1769.04|2967.12|4213.04|7.43|1662.89|1853.28|106.15|113.58|1959.43|1966.86|-2860.97| +2452102|35718|2452105|4679|9167|1219556|4698|9887|29071|1421648|5454|5396|5|29|7|4|111|24|92|62.22|138.75|79.08|5489.64|7275.36|5724.24|12765.00|436.52|0.00|1403.92|7275.36|7711.88|8679.28|9115.80|1551.12| +2452102|35718|2452222|13961|9167|1219556|4698|9887|29071|1421648|5454|5396|41|5|5|4|286|24|11|28.90|45.95|11.94|374.11|131.34|317.90|505.45|3.94|0.00|4.95|131.34|135.28|136.29|140.23|-186.56| +2452102|35718|2452214|4113|9167|1219556|4698|9887|29071|1421648|5454|5396|5|29|20|5|277|24|72|14.58|32.22|24.48|557.28|1762.56|1049.76|2319.84|105.75|0.00|69.12|1762.56|1868.31|1831.68|1937.43|712.80| +2452102|35718|2452157|5759|9167|1219556|4698|9887|29071|1421648|5454|5396|15|1|4|5|123|24|70|26.61|54.55|0.00|3818.50|0.00|1862.70|3818.50|0.00|0.00|801.50|0.00|0.00|801.50|801.50|-1862.70| +2452102|35718|2452188|14839|9167|1219556|4698|9887|29071|1421648|5454|5396|9|1|14|2|40|24|94|17.65|48.18|22.64|2400.76|2128.16|1659.10|4528.92|37.24|1596.12|1720.20|532.04|569.28|2252.24|2289.48|-1127.06| +2452102|35718|2452207|5397|9167|1219556|4698|9887|29071|1421648|5454|5396|11|29|1|5|79|24|87|43.68|63.33|23.43|3471.30|2038.41|3800.16|5509.71|11.00|937.66|1873.11|1100.75|1111.75|2973.86|2984.86|-2699.41| +2452102|35718|2452116|17291|9167|1219556|4698|9887|29071|1421648|5454|5396|45|9|17|4|244|24|5|90.67|216.70|82.34|671.80|411.70|453.35|1083.50|12.14|168.79|21.65|242.91|255.05|264.56|276.70|-210.44| +2452102|35718|2452214|12895|9167|1219556|4698|9887|29071|1421648|5454|5396|27|25|5|4|116|24|51|97.78|158.40|128.30|1535.10|6543.30|4986.78|8078.40|588.89|0.00|2261.85|6543.30|7132.19|8805.15|9394.04|1556.52| +2452102|35718|2452138|4231|9167|1219556|4698|9887|29071|1421648|5454|5396|13|21|5|1|120|24|31|41.49|90.86|33.61|1774.75|1041.91|1286.19|2816.66|52.09|0.00|27.90|1041.91|1094.00|1069.81|1121.90|-244.28| +2452102|35718|2452111|13283|9167|1219556|4698|9887|29071|1421648|5454|5396|49|25|4|2|30|24|37|21.11|39.89|31.51|310.06|1165.87|781.07|1475.93|69.95|0.00|545.75|1165.87|1235.82|1711.62|1781.57|384.80| +2452102|35718|2452114|2645|9167|1219556|4698|9887|29071|1421648|5454|5396|39|5|3|1|289|24|77|48.80|136.15|34.03|7863.24|2620.31|3757.60|10483.55|235.82|0.00|4193.42|2620.31|2856.13|6813.73|7049.55|-1137.29| +2452377|62495|2452440|10213|47151|1504206|2061|34363|10761|431235|5575|39430|6|6|3|3|63|25|32|19.66|22.21|2.22|639.68|71.04|629.12|710.72|5.68|0.00|42.56|71.04|76.72|113.60|119.28|-558.08| +2452377|62495|2452409|7221|47151|1504206|2061|34363|10761|431235|5575|39430|21|13|20|3|99|25|2|58.24|112.98|76.82|72.32|153.64|116.48|225.96|10.75|0.00|2.24|153.64|164.39|155.88|166.63|37.16| +2452377|62495|2452469|5673|47151|1504206|2061|34363|10761|431235|5575|39430|12|27|2|5|285|25|43|61.06|164.86|112.10|2268.68|4820.30|2625.58|7088.98|0.00|0.00|425.27|4820.30|4820.30|5245.57|5245.57|2194.72| +2452377|62495|2452471|7197|47151|1504206|2061|34363|10761|431235|5575|39430|30|27|3|2|138|25|12|67.06|80.47|64.37|193.20|772.44|804.72|965.64|26.57|108.14|463.44|664.30|690.87|1127.74|1154.31|-140.42| +2452377|62495|2452475|3379|47151|1504206|2061|34363|10761|431235|5575|39430|7|13|17|2|127|25|26|30.08|45.12|24.81|528.06|645.06|782.08|1173.12|29.35|225.77|410.54|419.29|448.64|829.83|859.18|-362.79| +2452377|62495|2452378|5790|47151|1504206|2061|34363|10761|431235|5575|39430|33|3|20|5|190|25|73|50.03|124.57|28.65|7002.16|2091.45|3652.19|9093.61|104.57|0.00|818.33|2091.45|2196.02|2909.78|3014.35|-1560.74| +2452377|62495|2452460|5571|47151|1504206|2061|34363|10761|431235|5575|39430|21|3|6|5|231|25|22|88.96|152.12|3.04|3279.76|66.88|1957.12|3346.64|6.01|0.00|1639.66|66.88|72.89|1706.54|1712.55|-1890.24| +2452377|62495|2452441|13332|47151|1504206|2061|34363|10761|431235|5575|39430|7|1|8|5|232|25|50|57.17|160.07|67.22|4642.50|3361.00|2858.50|8003.50|33.61|0.00|1440.50|3361.00|3394.61|4801.50|4835.11|502.50| +2452377|62495|2452452|4761|47151|1504206|2061|34363|10761|431235|5575|39430|13|30|16|4|295|25|93|88.00|150.48|109.85|3778.59|10216.05|8184.00|13994.64|0.00|0.00|2658.87|10216.05|10216.05|12874.92|12874.92|2032.05| +2452377|62495|2452437|4549|47151|1504206|2061|34363|10761|431235|5575|39430|36|3|16|2|81|25|29|68.48|169.14|3.38|4807.04|98.02|1985.92|4905.06|1.00|81.35|931.77|16.67|17.67|948.44|949.44|-1969.25| +2452377|62495|2452441|4833|47151|1504206|2061|34363|10761|431235|5575|39430|49|25|1|1|134|25|78|82.00|89.38|54.52|2719.08|4252.56|6396.00|6971.64|212.62|0.00|2369.64|4252.56|4465.18|6622.20|6834.82|-2143.44| +2452377|62495|2452425|7099|47151|1504206|2061|34363|10761|431235|5575|39430|24|21|13|5|204|25|47|60.04|144.69|34.72|5168.59|1631.84|2821.88|6800.43|130.54|0.00|1359.71|1631.84|1762.38|2991.55|3122.09|-1190.04| +2452377|62495|2452483|5472|47151|1504206|2061|34363|10761|431235|5575|39430|51|30|14|3|252|25|1|78.44|228.26|15.97|212.29|15.97|78.44|228.26|1.18|1.11|95.86|14.86|16.04|110.72|111.90|-63.58| +2451287|82606|2451312|3586|95407|970818|2582|4346|54186|951760|2438|21402|2|2|12|2|32|26|81|22.29|40.79|33.03|628.56|2675.43|1805.49|3303.99|80.26|0.00|1090.26|2675.43|2755.69|3765.69|3845.95|869.94| +2451287|82606|2451303|1528|95407|970818|2582|4346|54186|951760|2438|21402|56|7|9|4|169|26|92|38.04|101.94|61.16|3751.76|5626.72|3499.68|9378.48|56.26|0.00|3938.52|5626.72|5682.98|9565.24|9621.50|2127.04| +2451287|82606|2451343|2494|95407|970818|2582|4346|54186|951760|2438|21402|25|8|7|4|228|26|55|50.21|55.73|16.16|2176.35|888.80|2761.55|3065.15|35.55|0.00|1501.50|888.80|924.35|2390.30|2425.85|-1872.75| +2451287|82606|2451355|1042|95407|970818|2582|4346|54186|951760|2438|21402|49|4|14|5|96|26|85|71.87|176.80|14.14|13826.10|1201.90|6108.95|15028.00|48.07|0.00|6160.80|1201.90|1249.97|7362.70|7410.77|-4907.05| +2451287|82606|2451295|5878|95407|970818|2582|4346|54186|951760|2438|21402|19|16|5|4|140|26|31|64.70|84.75|55.93|893.42|1733.83|2005.70|2627.25|104.02|0.00|604.19|1733.83|1837.85|2338.02|2442.04|-271.87| +2451287|82606|2451304|8710|95407|970818|2582|4346|54186|951760|2438|21402|16|1|10|2|171|26|34|56.10|141.37|52.30|3028.38|1778.20|1907.40|4806.58|34.14|640.15|1393.66|1138.05|1172.19|2531.71|2565.85|-769.35| +2451287|82606|2451360|13885|95407|970818|2582|4346|54186|951760|2438|21402|2|28|8|4|257|26|79|13.35|18.28|10.23|635.95|808.17|1054.65|1444.12|0.00|783.92|173.01|24.25|24.25|197.26|197.26|-1030.40| +2451287|82606|2451395|3788|95407|970818|2582|4346|54186|951760|2438|21402|2|14|14|5|176|26|15|56.43|62.07|41.58|307.35|623.70|846.45|931.05|18.21|168.39|102.30|455.31|473.52|557.61|575.82|-391.14| +2451287|82606|2451392|17788|95407|970818|2582|4346|54186|951760|2438|21402|49|8|5|5|275|26|6|33.83|99.12|25.77|440.10|154.62|202.98|594.72|12.36|0.00|285.42|154.62|166.98|440.04|452.40|-48.36| +2451287|82606|2451399|2090|95407|970818|2582|4346|54186|951760|2438|21402|46|26|1|1|208|26|69|88.19|179.90|8.99|11792.79|620.31|6085.11|12413.10|6.20|0.00|2854.53|620.31|626.51|3474.84|3481.04|-5464.80| +2451287|82606|2451405|10462|95407|970818|2582|4346|54186|951760|2438|21402|58|2|11|3|82|26|12|16.74|43.69|13.54|361.80|162.48|200.88|524.28|9.74|0.00|230.64|162.48|172.22|393.12|402.86|-38.40| +2451287|82606|2451362|4612|95407|970818|2582|4346|54186|951760|2438|21402|52|28|20|3|263|26|3|89.46|104.66|36.63|204.09|109.89|268.38|313.98|9.89|0.00|119.31|109.89|119.78|229.20|239.09|-158.49| +2451287|82606|2451407|15325|95407|970818|2582|4346|54186|951760|2438|21402|1|28|12|4|283|26|56|42.81|65.92|36.25|1661.52|2030.00|2397.36|3691.52|20.30|0.00|1291.92|2030.00|2050.30|3321.92|3342.22|-367.36| +2451287|82606|2451342|16459|95407|970818|2582|4346|54186|951760|2438|21402|32|16|8|5|149|26|54|67.29|91.51|90.59|49.68|4891.86|3633.66|4941.54|195.67|0.00|1531.44|4891.86|5087.53|6423.30|6618.97|1258.20| +2451287|82606|2451317|14662|95407|970818|2582|4346|54186|951760|2438|21402|55|20|17|3|246|26|36|47.37|132.16|9.25|4424.76|333.00|1705.32|4757.76|26.64|0.00|1950.48|333.00|359.64|2283.48|2310.12|-1372.32| +2451287|82606|2451306|5942|95407|970818|2582|4346|54186|951760|2438|21402|13|28|10|3|173|26|44|62.57|162.05|9.72|6702.52|427.68|2753.08|7130.20|17.10|0.00|213.84|427.68|444.78|641.52|658.62|-2325.40| +2450884|35444|2450962|8684|31979|1234440|4901|13715|46998|1484245|5851|33912|52|28|13|4|182|27|33|94.47|225.78|56.44|5588.22|1862.52|3117.51|7450.74|55.13|484.25|1191.96|1378.27|1433.40|2570.23|2625.36|-1739.24| +2450884|35444|2450948|10546|31979|1234440|4901|13715|46998|1484245|5851|33912|50|4|17|4|278|27|2|42.83|116.92|12.86|208.12|25.72|85.66|233.84|2.31|0.00|56.12|25.72|28.03|81.84|84.15|-59.94| +2450884|35444|2450918|14170|31979|1234440|4901|13715|46998|1484245|5851|33912|20|8|9|3|197|27|77|28.35|44.22|2.21|3234.77|170.17|2182.95|3404.94|0.00|0.00|135.52|170.17|170.17|305.69|305.69|-2012.78| +2450884|35444|2450928|12086|31979|1234440|4901|13715|46998|1484245|5851|33912|56|4|14|5|59|27|11|27.78|36.94|4.80|353.54|52.80|305.58|406.34|4.75|0.00|186.89|52.80|57.55|239.69|244.44|-252.78| +2450884|35444|2450947|11474|31979|1234440|4901|13715|46998|1484245|5851|33912|22|4|14|3|29|27|15|47.22|76.49|64.25|183.60|963.75|708.30|1147.35|0.00|0.00|195.00|963.75|963.75|1158.75|1158.75|255.45| +2450884|35444|2450888|6254|31979|1234440|4901|13715|46998|1484245|5851|33912|31|20|9|4|259|27|36|8.88|15.00|11.70|118.80|421.20|319.68|540.00|25.27|0.00|178.20|421.20|446.47|599.40|624.67|101.52| +2450884|35444|2450937|8206|31979|1234440|4901|13715|46998|1484245|5851|33912|14|13|6|3|207|27|86|71.43|127.14|16.52|9513.32|1420.72|6142.98|10934.04|28.41|0.00|983.84|1420.72|1449.13|2404.56|2432.97|-4722.26| +2450884|35444|2450889|2260|31979|1234440|4901|13715|46998|1484245|5851|33912|13|1|5|2|246|27|61|21.60|28.29|25.17|190.32|1535.37|1317.60|1725.69|15.35|0.00|275.72|1535.37|1550.72|1811.09|1826.44|217.77| +2450884|35444|2450940|12544|31979|1234440|4901|13715|46998|1484245|5851|33912|8|26|3|5|4|27|25|49.73|70.61|34.59|900.50|864.75|1243.25|1765.25|34.59|0.00|229.25|864.75|899.34|1094.00|1128.59|-378.50| +2450884|35444|2450984|13010|31979|1234440|4901|13715|46998|1484245|5851|33912|2|22|9|4|266|27|49|32.26|71.93|69.05|141.12|3383.45|1580.74|3524.57|23.68|1015.03|140.63|2368.42|2392.10|2509.05|2532.73|787.68| +2450884|35444|2450914|2648|31979|1234440|4901|13715|46998|1484245|5851|33912|22|13|2|5|225|27|51|30.34|39.13|13.69|1297.44|698.19|1547.34|1995.63|55.85|0.00|0.00|698.19|754.04|698.19|754.04|-849.15| +2450884|35444|2450887|17770|31979|1234440|4901|13715|46998|1484245|5851|33912|56|28|1|2|3|27|67|35.92|65.01|5.85|3963.72|391.95|2406.64|4355.67|7.83|0.00|1175.85|391.95|399.78|1567.80|1575.63|-2014.69| +2450884|35444|2450927|9118|31979|1234440|4901|13715|46998|1484245|5851|33912|8|13|12|4|216|27|66|68.00|85.68|40.26|2997.72|2657.16|4488.00|5654.88|239.14|0.00|678.48|2657.16|2896.30|3335.64|3574.78|-1830.84| +2450884|35444|2450929|7588|31979|1234440|4901|13715|46998|1484245|5851|33912|25|1|16|4|2|27|11|4.36|10.07|2.21|86.46|24.31|47.96|110.77|1.21|0.00|6.60|24.31|25.52|30.91|32.12|-23.65| +2450884|35444|2450973|17959|31979|1234440|4901|13715|46998|1484245|5851|33912|8|22|16|5|294|27|14|21.94|56.60|35.09|301.14|491.26|307.16|792.40|44.21|0.00|118.86|491.26|535.47|610.12|654.33|184.10| +2450884|35444|2450952|11053|31979|1234440|4901|13715|46998|1484245|5851|33912|4|26|15|3|234|27|14|24.32|51.55|38.66|180.46|541.24|340.48|721.70|48.71|0.00|180.32|541.24|589.95|721.56|770.27|200.76| +2451039|62333|2451050|11522|24997|871580|6163|36746|10052|1758318|4848|35431|7|19|16|2|298|28|30|66.66|189.98|182.38|228.00|5471.40|1999.80|5699.40|273.57|0.00|56.70|5471.40|5744.97|5528.10|5801.67|3471.60| +2451039|62333|2451062|11197|24997|871580|6163|36746|10052|1758318|4848|35431|16|10|6|3|260|28|33|44.85|64.13|28.21|1185.36|930.93|1480.05|2116.29|83.78|0.00|169.29|930.93|1014.71|1100.22|1184.00|-549.12| +2451039|62333|2451109|17164|24997|871580|6163|36746|10052|1758318|4848|35431|1|26|10|1|69|28|56|85.54|148.83|104.18|2500.40|5834.08|4790.24|8334.48|233.36|0.00|1833.44|5834.08|6067.44|7667.52|7900.88|1043.84| +2451039|62333|2451113|4058|24997|871580|6163|36746|10052|1758318|4848|35431|43|14|14|2|183|28|63|15.93|30.90|27.19|233.73|1712.97|1003.59|1946.70|137.03|0.00|856.17|1712.97|1850.00|2569.14|2706.17|709.38| +2451039|62333|2451155|11881|24997|871580|6163|36746|10052|1758318|4848|35431|25|2|11|5|209|28|28|84.48|119.11|39.30|2234.68|1100.40|2365.44|3335.08|55.02|0.00|800.24|1100.40|1155.42|1900.64|1955.66|-1265.04| +2451039|62333|2451043|3721|24997|871580|6163|36746|10052|1758318|4848|35431|58|7|18|2|256|28|89|97.81|97.81|86.07|1044.86|7660.23|8705.09|8705.09|536.21|0.00|1914.39|7660.23|8196.44|9574.62|10110.83|-1044.86| +2451039|62333|2451155|7190|24997|871580|6163|36746|10052|1758318|4848|35431|46|19|19|5|14|28|1|99.95|175.91|66.84|109.07|66.84|99.95|175.91|0.19|47.45|33.42|19.39|19.58|52.81|53.00|-80.56| +2451039|62333|2451122|286|24997|871580|6163|36746|10052|1758318|4848|35431|26|20|15|2|219|28|39|62.06|73.23|8.78|2513.55|342.42|2420.34|2855.97|0.00|219.14|771.03|123.28|123.28|894.31|894.31|-2297.06| +2451039|62333|2451074|1519|24997|871580|6163|36746|10052|1758318|4848|35431|14|28|12|2|269|28|34|71.14|165.75|109.39|1916.24|3719.26|2418.76|5635.50|0.00|0.00|56.10|3719.26|3719.26|3775.36|3775.36|1300.50| +2451039|62333|2451118|9403|24997|871580|6163|36746|10052|1758318|4848|35431|38|22|4|3|235|28|74|74.71|132.98|117.02|1181.04|8659.48|5528.54|9840.52|173.18|0.00|196.10|8659.48|8832.66|8855.58|9028.76|3130.94| +2451039|62333|2451087|5353|24997|871580|6163|36746|10052|1758318|4848|35431|56|20|16|1|33|28|75|45.15|130.93|60.22|5303.25|4516.50|3386.25|9819.75|0.00|0.00|687.00|4516.50|4516.50|5203.50|5203.50|1130.25| +2451039|62333|2451048|13004|24997|871580|6163|36746|10052|1758318|4848|35431|19|26|18|2|93|28|25|45.99|57.48|27.01|761.75|675.25|1149.75|1437.00|54.02|0.00|43.00|675.25|729.27|718.25|772.27|-474.50| +2452220|41441|2452320|11879|57028|556763|1298|3033|18962|1496846|5391|47079|17|13|12|2|45|29|91|39.18|64.25|34.69|2689.96|3156.79|3565.38|5846.75|220.97|0.00|1695.33|3156.79|3377.76|4852.12|5073.09|-408.59| +2452220|41441|2452314|17021|57028|556763|1298|3033|18962|1496846|5391|47079|15|7|20|3|239|29|37|60.58|164.77|74.14|3353.31|2743.18|2241.46|6096.49|1.64|2578.58|1340.88|164.60|166.24|1505.48|1507.12|-2076.86| +2452220|41441|2452229|4193|57028|556763|1298|3033|18962|1496846|5391|47079|31|21|19|2|95|29|82|23.49|50.97|45.36|460.02|3719.52|1926.18|4179.54|185.97|0.00|208.28|3719.52|3905.49|3927.80|4113.77|1793.34| +2452220|41441|2452281|13269|57028|556763|1298|3033|18962|1496846|5391|47079|25|9|17|2|244|29|55|34.69|39.19|36.44|151.25|2004.20|1907.95|2155.45|27.05|1463.06|172.15|541.14|568.19|713.29|740.34|-1366.81| +2452220|41441|2452285|12889|57028|556763|1298|3033|18962|1496846|5391|47079|53|3|17|5|220|29|49|22.48|56.64|24.35|1582.21|1193.15|1101.52|2775.36|107.38|0.00|999.11|1193.15|1300.53|2192.26|2299.64|91.63| +2452220|41441|2452279|12911|57028|556763|1298|3033|18962|1496846|5391|47079|5|15|20|5|120|29|89|97.71|276.51|218.44|5168.23|19441.16|8696.19|24609.39|0.00|0.00|2460.85|19441.16|19441.16|21902.01|21902.01|10744.97| +2452220|41441|2452256|5731|57028|556763|1298|3033|18962|1496846|5391|47079|3|29|11|4|107|29|56|96.93|107.59|71.00|2049.04|3976.00|5428.08|6025.04|155.06|874.72|1204.56|3101.28|3256.34|4305.84|4460.90|-2326.80| +2452220|41441|2452337|3275|57028|556763|1298|3033|18962|1496846|5391|47079|3|5|7|3|262|29|27|73.21|201.32|16.10|5000.94|434.70|1976.67|5435.64|13.04|0.00|652.05|434.70|447.74|1086.75|1099.79|-1541.97| +2452220|41441|2452276|12777|57028|556763|1298|3033|18962|1496846|5391|47079|33|29|16|2|115|29|87|97.21|228.44|43.40|16098.48|3775.80|8457.27|19874.28|226.54|0.00|7353.24|3775.80|4002.34|11129.04|11355.58|-4681.47| +2452220|41441|2452282|2765|57028|556763|1298|3033|18962|1496846|5391|47079|9|11|7|3|279|29|9|95.82|248.17|213.42|312.75|1920.78|862.38|2233.53|172.87|0.00|960.39|1920.78|2093.65|2881.17|3054.04|1058.40| +2451150|41465|2451252|13568|73211|866401|5991|7181|36734|1886542|798|2788|1|19|6|2|21|30|98|40.69|90.33|56.90|3276.14|5576.20|3987.62|8852.34|111.52|0.00|1061.34|5576.20|5687.72|6637.54|6749.06|1588.58| +2451150|41465|2451178|9163|73211|866401|5991|7181|36734|1886542|798|2788|28|7|3|2|241|30|83|76.03|146.73|111.51|2923.26|9255.33|6310.49|12178.59|283.21|6108.51|1460.80|3146.82|3430.03|4607.62|4890.83|-3163.67| +2451150|41465|2451255|16190|73211|866401|5991|7181|36734|1886542|798|2788|22|26|3|5|267|30|35|8.53|12.53|10.90|57.05|381.50|298.55|438.55|34.33|0.00|12.95|381.50|415.83|394.45|428.78|82.95| +2451150|41465|2451192|5546|73211|866401|5991|7181|36734|1886542|798|2788|43|26|13|5|147|30|34|91.37|161.72|9.70|5168.68|329.80|3106.58|5498.48|9.89|0.00|824.50|329.80|339.69|1154.30|1164.19|-2776.78| +2451150|41465|2451184|4009|73211|866401|5991|7181|36734|1886542|798|2788|28|4|14|5|253|30|85|17.11|35.41|8.85|2257.60|752.25|1454.35|3009.85|6.92|406.21|270.30|346.04|352.96|616.34|623.26|-1108.31| +2451150|41465|2451198|9025|73211|866401|5991|7181|36734|1886542|798|2788|4|19|14|1|105|30|50|97.54|258.48|18.09|12019.50|904.50|4877.00|12924.00|54.27|0.00|1550.50|904.50|958.77|2455.00|2509.27|-3972.50| +2451150|41465|2451257|9368|73211|866401|5991|7181|36734|1886542|798|2788|8|25|11|1|134|30|71|39.71|41.29|12.79|2023.50|908.09|2819.41|2931.59|81.72|0.00|1406.51|908.09|989.81|2314.60|2396.32|-1911.32| +2451150|41465|2451235|7987|73211|866401|5991|7181|36734|1886542|798|2788|43|10|20|4|228|30|37|81.68|93.11|42.83|1860.36|1584.71|3022.16|3445.07|79.23|0.00|413.29|1584.71|1663.94|1998.00|2077.23|-1437.45| +2451150|41465|2451187|4868|73211|866401|5991|7181|36734|1886542|798|2788|49|22|16|3|160|30|73|63.15|153.45|104.34|3585.03|7616.82|4609.95|11201.85|457.00|0.00|3584.30|7616.82|8073.82|11201.12|11658.12|3006.87| +2451150|41465|2451240|3460|73211|866401|5991|7181|36734|1886542|798|2788|8|22|12|4|289|30|14|14.06|16.59|3.64|181.30|50.96|196.84|232.26|0.15|45.86|88.20|5.10|5.25|93.30|93.45|-191.74| +2451037|54024|2451081|13742|29544|1314548|5952|34556|18322|1519490|6768|24572|40|4|15|2|39|31|10|1.41|3.10|1.67|14.30|16.70|14.10|31.00|0.00|0.00|10.20|16.70|16.70|26.90|26.90|2.60| +2451037|54024|2451154|9745|29544|1314548|5952|34556|18322|1519490|6768|24572|44|22|4|3|152|31|66|97.80|226.89|176.97|3294.72|11680.02|6454.80|14974.74|467.20|0.00|6588.78|11680.02|12147.22|18268.80|18736.00|5225.22| +2451037|54024|2451130|15829|29544|1314548|5952|34556|18322|1519490|6768|24572|56|22|15|3|254|31|27|37.83|101.00|9.09|2481.57|245.43|1021.41|2727.00|2.74|211.06|1090.80|34.37|37.11|1125.17|1127.91|-987.04| +2451037|54024|2451095|16804|29544|1314548|5952|34556|18322|1519490|6768|24572|40|28|19|3|207|31|94|42.95|113.81|73.97|3744.96|6953.18|4037.30|10698.14|66.75|3615.65|3209.16|3337.53|3404.28|6546.69|6613.44|-699.77| +2451037|54024|2451139|9320|29544|1314548|5952|34556|18322|1519490|6768|24572|34|26|20|4|265|31|34|85.40|245.09|117.64|4333.30|3999.76|2903.60|8333.06|279.98|0.00|3749.86|3999.76|4279.74|7749.62|8029.60|1096.16| +2451037|54024|2451065|8722|29544|1314548|5952|34556|18322|1519490|6768|24572|16|10|16|3|103|31|33|59.36|163.24|163.24|0.00|5386.92|1958.88|5386.92|323.21|0.00|269.28|5386.92|5710.13|5656.20|5979.41|3428.04| +2451037|54024|2451092|16910|29544|1314548|5952|34556|18322|1519490|6768|24572|37|10|16|1|126|31|69|13.02|28.12|12.65|1067.43|872.85|898.38|1940.28|24.78|253.12|115.92|619.73|644.51|735.65|760.43|-278.65| +2451037|54024|2451150|12733|29544|1314548|5952|34556|18322|1519490|6768|24572|58|22|11|3|115|31|68|16.39|39.33|6.68|2220.20|454.24|1114.52|2674.44|18.16|0.00|801.72|454.24|472.40|1255.96|1274.12|-660.28| +2451037|54024|2451117|7844|29544|1314548|5952|34556|18322|1519490|6768|24572|22|26|9|1|216|31|21|88.24|231.18|152.57|1650.81|3203.97|1853.04|4854.78|0.00|0.00|339.78|3203.97|3203.97|3543.75|3543.75|1350.93| +2451037|54024|2451104|13897|29544|1314548|5952|34556|18322|1519490|6768|24572|28|13|6|3|27|31|37|72.48|118.14|115.77|87.69|4283.49|2681.76|4371.18|133.64|2056.07|261.96|2227.42|2361.06|2489.38|2623.02|-454.34| +2451037|54024|2451132|13141|29544|1314548|5952|34556|18322|1519490|6768|24572|2|19|18|1|64|31|48|32.40|97.20|89.42|373.44|4292.16|1555.20|4665.60|128.76|0.00|792.96|4292.16|4420.92|5085.12|5213.88|2736.96| +2451037|54024|2451125|3916|29544|1314548|5952|34556|18322|1519490|6768|24572|20|1|1|1|241|31|20|7.63|10.22|4.29|118.60|85.80|152.60|204.40|6.00|0.00|94.00|85.80|91.80|179.80|185.80|-66.80| +2451037|54024|2451110|15034|29544|1314548|5952|34556|18322|1519490|6768|24572|8|13|2|4|66|31|24|92.87|157.87|0.00|3788.88|0.00|2228.88|3788.88|0.00|0.00|1742.88|0.00|0.00|1742.88|1742.88|-2228.88| +2451379|64549|2451383|8041|67273|1005447|3621|28548|7907|1111442|1023|4797|19|26|15|1|130|32|89|28.90|60.69|57.65|270.56|5130.85|2572.10|5401.41|102.61|0.00|1566.40|5130.85|5233.46|6697.25|6799.86|2558.75| +2451379|64549|2451470|2894|67273|1005447|3621|28548|7907|1111442|1023|4797|4|4|3|1|116|32|68|38.55|67.07|30.18|2508.52|2052.24|2621.40|4560.76|61.56|0.00|2188.92|2052.24|2113.80|4241.16|4302.72|-569.16| +2451379|64549|2451395|12158|67273|1005447|3621|28548|7907|1111442|1023|4797|14|22|13|2|68|32|19|28.69|81.76|23.71|1102.95|450.49|545.11|1553.44|18.01|0.00|776.72|450.49|468.50|1227.21|1245.22|-94.62| +2451379|64549|2451482|16754|67273|1005447|3621|28548|7907|1111442|1023|4797|10|4|20|2|114|32|6|82.63|122.29|100.27|132.12|601.62|495.78|733.74|12.03|0.00|300.78|601.62|613.65|902.40|914.43|105.84| +2451379|64549|2451496|14227|67273|1005447|3621|28548|7907|1111442|1023|4797|43|13|15|5|128|32|22|51.82|123.84|108.97|327.14|2397.34|1140.04|2724.48|95.89|0.00|980.76|2397.34|2493.23|3378.10|3473.99|1257.30| +2451379|64549|2451442|3043|67273|1005447|3621|28548|7907|1111442|1023|4797|7|22|3|4|262|32|25|55.48|159.22|156.03|79.75|3900.75|1387.00|3980.50|351.06|0.00|875.50|3900.75|4251.81|4776.25|5127.31|2513.75| +2451379|64549|2451471|2078|67273|1005447|3621|28548|7907|1111442|1023|4797|20|14|8|5|202|32|85|86.33|224.45|181.80|3625.25|15453.00|7338.05|19078.25|199.34|8808.21|8775.40|6644.79|6844.13|15420.19|15619.53|-693.26| +2451379|64549|2451414|4274|67273|1005447|3621|28548|7907|1111442|1023|4797|43|26|6|3|50|32|40|38.71|99.48|42.77|2268.40|1710.80|1548.40|3979.20|17.10|0.00|1949.60|1710.80|1727.90|3660.40|3677.50|162.40| +2451379|64549|2451478|14438|67273|1005447|3621|28548|7907|1111442|1023|4797|1|8|16|2|1|32|81|82.74|226.70|36.27|15424.83|2937.87|6701.94|18362.70|235.02|0.00|7528.14|2937.87|3172.89|10466.01|10701.03|-3764.07| +2451379|64549|2451497|3046|67273|1005447|3621|28548|7907|1111442|1023|4797|2|7|3|5|227|32|40|59.52|94.63|67.18|1098.00|2687.20|2380.80|3785.20|80.61|0.00|1703.20|2687.20|2767.81|4390.40|4471.01|306.40| +2451379|64549|2451428|1933|67273|1005447|3621|28548|7907|1111442|1023|4797|55|22|13|4|248|32|3|24.49|29.14|4.95|72.57|14.85|73.47|87.42|0.29|0.00|19.23|14.85|15.14|34.08|34.37|-58.62| +2451379|64549|2451462|15937|67273|1005447|3621|28548|7907|1111442|1023|4797|4|13|11|5|169|32|37|92.05|273.38|49.20|8294.66|1820.40|3405.85|10115.06|163.83|0.00|1213.60|1820.40|1984.23|3034.00|3197.83|-1585.45| +2451379|64549|2451391|11521|67273|1005447|3621|28548|7907|1111442|1023|4797|56|10|3|5|117|32|54|48.67|123.13|102.19|1130.76|5518.26|2628.18|6649.02|386.27|0.00|1130.22|5518.26|5904.53|6648.48|7034.75|2890.08| +2451517|31608|2451570|8431|88444|869059|5674|16336|33131|1695169|4228|25643|20|1|1|3|218|33|86|95.64|159.71|71.86|7555.10|6179.96|8225.04|13735.06|308.99|0.00|5905.62|6179.96|6488.95|12085.58|12394.57|-2045.08| +2451517|31608|2451536|12926|88444|869059|5674|16336|33131|1695169|4228|25643|37|10|20|4|78|33|3|23.57|31.34|14.10|51.72|42.30|70.71|94.02|2.53|0.00|5.64|42.30|44.83|47.94|50.47|-28.41| +2451517|31608|2451576|16490|88444|869059|5674|16336|33131|1695169|4228|25643|44|25|16|4|112|33|31|98.28|283.04|246.24|1140.80|7633.44|3046.68|8774.24|152.66|0.00|1666.87|7633.44|7786.10|9300.31|9452.97|4586.76| +2451517|31608|2451612|7090|88444|869059|5674|16336|33131|1695169|4228|25643|31|4|20|4|151|33|26|58.85|73.56|38.98|899.08|1013.48|1530.10|1912.56|60.80|0.00|382.46|1013.48|1074.28|1395.94|1456.74|-516.62| +2451517|31608|2451582|5407|88444|869059|5674|16336|33131|1695169|4228|25643|28|7|13|2|211|33|21|83.48|170.29|103.87|1394.82|2181.27|1753.08|3576.09|65.43|0.00|1179.99|2181.27|2246.70|3361.26|3426.69|428.19| +2451517|31608|2451631|1807|88444|869059|5674|16336|33131|1695169|4228|25643|16|1|20|2|201|33|87|99.48|239.74|33.56|17937.66|2919.72|8654.76|20857.38|184.52|613.14|10428.69|2306.58|2491.10|12735.27|12919.79|-6348.18| +2451517|31608|2451566|17581|88444|869059|5674|16336|33131|1695169|4228|25643|20|7|10|3|179|33|84|12.17|33.22|12.62|1730.40|1060.08|1022.28|2790.48|63.60|0.00|641.76|1060.08|1123.68|1701.84|1765.44|37.80| +2451517|31608|2451570|3673|88444|869059|5674|16336|33131|1695169|4228|25643|10|26|15|3|241|33|11|79.96|188.70|7.54|1992.76|82.94|879.56|2075.70|2.48|0.00|498.08|82.94|85.42|581.02|583.50|-796.62| +2451517|31608|2451583|14282|88444|869059|5674|16336|33131|1695169|4228|25643|37|28|11|2|176|33|7|36.32|70.46|54.95|108.57|384.65|254.24|493.22|3.84|0.00|177.52|384.65|388.49|562.17|566.01|130.41| +2451517|31608|2451519|3955|88444|869059|5674|16336|33131|1695169|4228|25643|13|25|16|2|135|33|65|73.89|191.37|99.51|5970.90|6468.15|4802.85|12439.05|517.45|0.00|3233.75|6468.15|6985.60|9701.90|10219.35|1665.30| +2451517|31608|2451610|9076|88444|869059|5674|16336|33131|1695169|4228|25643|10|7|2|5|208|33|57|98.95|131.60|121.07|600.21|6900.99|5640.15|7501.20|621.08|0.00|2550.18|6900.99|7522.07|9451.17|10072.25|1260.84| +2451517|31608|2451621|7544|88444|869059|5674|16336|33131|1695169|4228|25643|46|19|16|2|118|33|8|50.93|58.56|43.92|117.12|351.36|407.44|468.48|3.51|0.00|4.64|351.36|354.87|356.00|359.51|-56.08| +2451328|78353|2451355|5629|56679|1142880|1081|39282|47435|1683458|1885|16086|7|25|2|1|70|34|5|85.54|100.93|52.48|242.25|262.40|427.70|504.65|13.12|0.00|80.70|262.40|275.52|343.10|356.22|-165.30| +2451328|78353|2451388|1348|56679|1142880|1081|39282|47435|1683458|1885|16086|26|2|17|5|213|34|41|28.24|78.78|41.75|1518.23|1711.75|1157.84|3229.98|17.11|0.00|1130.37|1711.75|1728.86|2842.12|2859.23|553.91| +2451328|78353|2451367|2266|56679|1142880|1081|39282|47435|1683458|1885|16086|19|10|15|4|93|34|77|66.31|174.39|125.56|3759.91|9668.12|5105.87|13428.03|386.72|0.00|6176.17|9668.12|10054.84|15844.29|16231.01|4562.25| +2451328|78353|2451401|7195|56679|1142880|1081|39282|47435|1683458|1885|16086|40|7|4|2|110|34|96|41.23|77.92|77.14|74.88|7405.44|3958.08|7480.32|148.10|0.00|3665.28|7405.44|7553.54|11070.72|11218.82|3447.36| +2451328|78353|2451420|13102|56679|1142880|1081|39282|47435|1683458|1885|16086|38|8|3|3|216|34|94|18.44|25.81|10.84|1407.18|1018.96|1733.36|2426.14|10.18|0.00|581.86|1018.96|1029.14|1600.82|1611.00|-714.40| +2451328|78353|2451421|7993|56679|1142880|1081|39282|47435|1683458|1885|16086|10|7|2|5|141|34|55|71.17|196.42|143.38|2917.20|7885.90|3914.35|10803.10|315.43|0.00|431.75|7885.90|8201.33|8317.65|8633.08|3971.55| +2451328|78353|2451386|13804|56679|1142880|1081|39282|47435|1683458|1885|16086|46|19|9|5|201|34|52|52.68|150.66|75.33|3917.16|3917.16|2739.36|7834.32|9.40|3760.47|1174.68|156.69|166.09|1331.37|1340.77|-2582.67| +2451328|78353|2451355|136|56679|1142880|1081|39282|47435|1683458|1885|16086|1|28|9|5|214|34|48|59.41|141.39|57.96|4004.64|2782.08|2851.68|6786.72|111.28|0.00|610.56|2782.08|2893.36|3392.64|3503.92|-69.60| +2451328|78353|2451400|8824|56679|1142880|1081|39282|47435|1683458|1885|16086|37|13|15|3|241|34|26|90.63|265.54|249.60|414.44|6489.60|2356.38|6904.04|124.60|259.58|2209.22|6230.02|6354.62|8439.24|8563.84|3873.64| +2451328|78353|2451406|4708|56679|1142880|1081|39282|47435|1683458|1885|16086|20|8|19|3|47|34|70|81.95|117.18|17.57|6972.70|1229.90|5736.50|8202.60|110.69|0.00|574.00|1229.90|1340.59|1803.90|1914.59|-4506.60| +2451328|78353|2451409|15313|56679|1142880|1081|39282|47435|1683458|1885|16086|16|2|12|5|235|34|29|31.69|62.74|55.21|218.37|1601.09|919.01|1819.46|0.00|0.00|873.19|1601.09|1601.09|2474.28|2474.28|682.08| +2451328|78353|2451417|10262|56679|1142880|1081|39282|47435|1683458|1885|16086|19|10|2|3|241|34|15|88.55|170.90|54.68|1743.30|820.20|1328.25|2563.50|65.61|0.00|640.80|820.20|885.81|1461.00|1526.61|-508.05| +2451328|78353|2451415|14864|56679|1142880|1081|39282|47435|1683458|1885|16086|13|14|19|2|108|34|98|84.37|165.36|71.10|9237.48|6967.80|8268.26|16205.28|124.02|766.45|486.08|6201.35|6325.37|6687.43|6811.45|-2066.91| +2451889|42440|2451936|12247|90495|1378316|3290|31464|32170|1831235|5680|41854|50|13|15|3|144|35|82|48.03|73.96|22.92|4185.28|1879.44|3938.46|6064.72|18.79|0.00|2789.64|1879.44|1898.23|4669.08|4687.87|-2059.02| +2451889|42440|2451938|4739|90495|1378316|3290|31464|32170|1831235|5680|41854|43|2|2|2|62|35|3|88.91|130.69|3.92|380.31|11.76|266.73|392.07|0.35|0.00|137.22|11.76|12.11|148.98|149.33|-254.97| +2451889|42440|2451892|13304|90495|1378316|3290|31464|32170|1831235|5680|41854|25|26|15|1|257|35|10|25.49|52.76|12.13|406.30|121.30|254.90|527.60|2.42|0.00|58.00|121.30|123.72|179.30|181.72|-133.60| +2451889|42440|2451922|12923|90495|1378316|3290|31464|32170|1831235|5680|41854|14|2|16|2|81|35|15|29.09|73.59|11.03|938.40|165.45|436.35|1103.85|3.07|62.87|132.45|102.58|105.65|235.03|238.10|-333.77| +2451889|42440|2451982|17189|90495|1378316|3290|31464|32170|1831235|5680|41854|14|5|18|1|281|35|32|84.10|206.04|30.90|5604.48|988.80|2691.20|6593.28|79.10|0.00|725.12|988.80|1067.90|1713.92|1793.02|-1702.40| +2451889|42440|2451906|9506|90495|1378316|3290|31464|32170|1831235|5680|41854|41|5|18|3|138|35|37|70.28|200.29|104.15|3557.18|3853.55|2600.36|7410.73|38.53|0.00|222.00|3853.55|3892.08|4075.55|4114.08|1253.19| +2451889|42440|2451952|16405|90495|1378316|3290|31464|32170|1831235|5680|41854|53|8|16|2|135|35|95|13.14|29.82|19.08|1020.30|1812.60|1248.30|2832.90|36.25|0.00|1217.90|1812.60|1848.85|3030.50|3066.75|564.30| +2451889|42440|2451944|10424|90495|1378316|3290|31464|32170|1831235|5680|41854|53|23|20|5|80|35|82|18.11|34.77|3.47|2566.60|284.54|1485.02|2851.14|8.53|0.00|1197.20|284.54|293.07|1481.74|1490.27|-1200.48| +2451889|42440|2451983|13165|90495|1378316|3290|31464|32170|1831235|5680|41854|8|19|18|3|131|35|16|77.39|152.45|44.21|1731.84|707.36|1238.24|2439.20|0.00|0.00|341.44|707.36|707.36|1048.80|1048.80|-530.88| +2451889|42440|2451965|11816|90495|1378316|3290|31464|32170|1831235|5680|41854|50|11|16|1|96|35|65|8.32|20.96|11.73|599.95|762.45|540.80|1362.40|0.00|0.00|558.35|762.45|762.45|1320.80|1320.80|221.65| +2451889|42440|2451932|13645|90495|1378316|3290|31464|32170|1831235|5680|41854|29|8|12|2|169|35|4|41.77|125.31|71.42|215.56|285.68|167.08|501.24|25.71|0.00|10.00|285.68|311.39|295.68|321.39|118.60| +2451889|42440|2452008|12920|90495|1378316|3290|31464|32170|1831235|5680|41854|35|25|18|3|28|35|51|43.08|62.89|53.45|481.44|2725.95|2197.08|3207.39|81.77|0.00|256.53|2725.95|2807.72|2982.48|3064.25|528.87| +2451889|42440|2451943|8045|90495|1378316|3290|31464|32170|1831235|5680|41854|38|14|13|3|53|35|94|94.33|275.44|82.63|18124.14|7767.22|8867.02|25891.36|39.61|3805.93|5695.46|3961.29|4000.90|9656.75|9696.36|-4905.73| +2451889|42440|2451933|2234|90495|1378316|3290|31464|32170|1831235|5680|41854|31|11|13|3|110|35|38|84.90|135.84|71.99|2426.30|2735.62|3226.20|5161.92|27.35|0.00|980.40|2735.62|2762.97|3716.02|3743.37|-490.58| +2451973|61232|2452037|3687|32782|1743944|5506|19115|59321|467323|7048|23010|49|9|8|1|141|36|83|86.21|154.31|58.63|7941.44|4866.29|7155.43|12807.73|145.98|0.00|2689.20|4866.29|5012.27|7555.49|7701.47|-2289.14| +2451973|61232|2452089|791|32782|1743944|5506|19115|59321|467323|7048|23010|19|1|2|3|244|36|76|92.79|172.58|31.06|10755.52|2360.56|7052.04|13116.08|165.23|0.00|130.72|2360.56|2525.79|2491.28|2656.51|-4691.48| +2451973|61232|2452014|731|32782|1743944|5506|19115|59321|467323|7048|23010|33|13|7|5|300|36|33|72.55|161.06|80.53|2657.49|2657.49|2394.15|5314.98|53.14|0.00|1700.49|2657.49|2710.63|4357.98|4411.12|263.34| +2451973|61232|2452008|8293|32782|1743944|5506|19115|59321|467323|7048|23010|33|29|16|4|235|36|17|2.31|5.12|1.02|69.70|17.34|39.27|87.04|1.56|0.00|35.53|17.34|18.90|52.87|54.43|-21.93| +2451973|61232|2452083|2707|32782|1743944|5506|19115|59321|467323|7048|23010|59|25|6|4|52|36|77|13.47|15.35|5.06|792.33|389.62|1037.19|1181.95|19.48|0.00|0.00|389.62|409.10|389.62|409.10|-647.57| +2451973|61232|2452060|4817|32782|1743944|5506|19115|59321|467323|7048|23010|59|7|8|2|136|36|62|88.75|154.42|146.69|479.26|9094.78|5502.50|9574.04|90.94|0.00|1722.98|9094.78|9185.72|10817.76|10908.70|3592.28| +2451973|61232|2452060|1287|32782|1743944|5506|19115|59321|467323|7048|23010|41|25|12|4|266|36|8|3.01|3.79|2.84|7.60|22.72|24.08|30.32|0.07|21.81|5.12|0.91|0.98|6.03|6.10|-23.17| +2451973|61232|2452032|14255|32782|1743944|5506|19115|59321|467323|7048|23010|41|21|4|4|27|36|53|35.38|68.99|36.56|1718.79|1937.68|1875.14|3656.47|135.63|0.00|1279.42|1937.68|2073.31|3217.10|3352.73|62.54| +2451973|61232|2452058|17607|32782|1743944|5506|19115|59321|467323|7048|23010|17|11|10|2|34|36|75|40.94|81.47|3.25|5866.50|243.75|3070.50|6110.25|0.00|0.00|1710.75|243.75|243.75|1954.50|1954.50|-2826.75| +2451973|61232|2452092|2657|32782|1743944|5506|19115|59321|467323|7048|23010|59|1|19|5|41|36|100|28.71|81.82|5.72|7610.00|572.00|2871.00|8182.00|5.37|34.32|572.00|537.68|543.05|1109.68|1115.05|-2333.32| +2451973|61232|2451979|12139|32782|1743944|5506|19115|59321|467323|7048|23010|3|23|12|2|42|36|53|71.93|98.54|47.29|2716.25|2506.37|3812.29|5222.62|0.00|0.00|1252.92|2506.37|2506.37|3759.29|3759.29|-1305.92| +2451973|61232|2451984|12193|32782|1743944|5506|19115|59321|467323|7048|23010|29|17|16|3|120|36|65|17.34|28.09|16.01|785.20|1040.65|1127.10|1825.85|31.21|0.00|54.60|1040.65|1071.86|1095.25|1126.46|-86.45| +2451973|61232|2452038|8799|32782|1743944|5506|19115|59321|467323|7048|23010|19|25|3|1|209|36|55|27.25|77.39|17.02|3320.35|936.10|1498.75|4256.45|65.52|0.00|978.45|936.10|1001.62|1914.55|1980.07|-562.65| +2451973|61232|2452010|13993|32782|1743944|5506|19115|59321|467323|7048|23010|15|29|9|1|283|36|95|86.47|225.68|173.77|4931.45|16508.15|8214.65|21439.60|660.32|0.00|1286.30|16508.15|17168.47|17794.45|18454.77|8293.50| +2451540|67832|2451558|16492|9872|1545781|4443|21952|9872|1545781|4443|21952|8|10|4|5|16|37|61|81.13|180.10|151.28|1758.02|9228.08|4948.93|10986.10|184.56|0.00|1098.61|9228.08|9412.64|10326.69|10511.25|4279.15| +2451540|67832|2451621|7573|9872|1545781|4443|21952|9872|1545781|4443|21952|4|1|9|5|198|37|100|9.51|25.86|25.08|78.00|2508.00|951.00|2586.00|50.16|0.00|206.00|2508.00|2558.16|2714.00|2764.16|1557.00| +2451540|67832|2451606|12754|9872|1545781|4443|21952|9872|1545781|4443|21952|13|2|7|4|234|37|97|8.74|17.65|15.17|240.56|1471.49|847.78|1712.05|0.00|0.00|342.41|1471.49|1471.49|1813.90|1813.90|623.71| +2451540|67832|2451574|160|9872|1545781|4443|21952|9872|1545781|4443|21952|7|7|14|4|274|37|67|7.84|11.36|3.18|548.06|213.06|525.28|761.12|10.65|0.00|288.77|213.06|223.71|501.83|512.48|-312.22| +2451540|67832|2451622|8320|9872|1545781|4443|21952|9872|1545781|4443|21952|34|7|20|4|105|37|51|64.03|125.49|27.60|4992.39|1407.60|3265.53|6399.99|126.68|0.00|959.82|1407.60|1534.28|2367.42|2494.10|-1857.93| +2451540|67832|2451656|3481|9872|1545781|4443|21952|9872|1545781|4443|21952|20|7|10|3|29|37|23|45.25|132.58|62.31|1616.21|1433.13|1040.75|3049.34|57.32|0.00|213.44|1433.13|1490.45|1646.57|1703.89|392.38| +2451540|67832|2451609|913|9872|1545781|4443|21952|9872|1545781|4443|21952|22|25|14|5|286|37|65|64.72|152.09|13.68|8996.65|889.20|4206.80|9885.85|17.78|0.00|4745.00|889.20|906.98|5634.20|5651.98|-3317.60| +2451540|67832|2451579|6655|9872|1545781|4443|21952|9872|1545781|4443|21952|20|7|4|2|87|37|28|39.63|53.10|40.35|357.00|1129.80|1109.64|1486.80|45.19|0.00|416.08|1129.80|1174.99|1545.88|1591.07|20.16| +2451540|67832|2451638|16921|9872|1545781|4443|21952|9872|1545781|4443|21952|43|19|15|3|281|37|26|82.12|167.52|155.79|304.98|4050.54|2135.12|4355.52|283.53|0.00|1175.98|4050.54|4334.07|5226.52|5510.05|1915.42| +2451540|67832|2451600|4333|9872|1545781|4443|21952|9872|1545781|4443|21952|7|10|18|5|72|37|69|22.87|45.74|5.03|2808.99|347.07|1578.03|3156.06|27.76|0.00|251.85|347.07|374.83|598.92|626.68|-1230.96| +2451540|67832|2451622|1357|9872|1545781|4443|21952|9872|1545781|4443|21952|46|16|7|2|227|37|5|23.05|50.47|45.42|25.25|227.10|115.25|252.35|6.81|0.00|111.00|227.10|233.91|338.10|344.91|111.85| +2451540|67832|2451558|572|9872|1545781|4443|21952|9872|1545781|4443|21952|13|20|5|2|196|37|73|84.68|116.85|85.30|2303.15|6226.90|6181.64|8530.05|373.61|0.00|3752.93|6226.90|6600.51|9979.83|10353.44|45.26| +2451540|67832|2451651|13006|9872|1545781|4443|21952|9872|1545781|4443|21952|19|26|11|5|19|37|80|66.87|106.32|42.52|5104.00|3401.60|5349.60|8505.60|102.04|0.00|509.60|3401.60|3503.64|3911.20|4013.24|-1948.00| +2451540|67832|2451657|15260|9872|1545781|4443|21952|9872|1545781|4443|21952|44|10|19|2|128|37|62|61.83|127.36|81.51|2842.70|5053.62|3833.46|7896.32|0.00|0.00|3316.38|5053.62|5053.62|8370.00|8370.00|1220.16| +2451540|67832|2451647|17144|9872|1545781|4443|21952|9872|1545781|4443|21952|40|2|13|4|245|37|11|17.89|48.12|17.32|338.80|190.52|196.79|529.32|1.90|0.00|79.31|190.52|192.42|269.83|271.73|-6.27| +2451540|67832|2451586|1510|9872|1545781|4443|21952|9872|1545781|4443|21952|43|10|20|4|41|37|25|13.02|23.69|10.89|320.00|272.25|325.50|592.25|8.16|0.00|100.50|272.25|280.41|372.75|380.91|-53.25| +2451706|58174|2451794|3050|87550|500431|2112|46593|56646|1344568|36|41670|19|8|20|3|133|38|80|53.70|56.92|41.55|1229.60|3324.00|4296.00|4553.60|265.92|0.00|136.00|3324.00|3589.92|3460.00|3725.92|-972.00| +2451706|58174|2451816|5960|87550|500431|2112|46593|56646|1344568|36|41670|29|14|8|1|91|38|77|81.68|120.88|84.61|2792.79|6514.97|6289.36|9307.76|195.44|0.00|1023.33|6514.97|6710.41|7538.30|7733.74|225.61| +2451706|58174|2451771|13355|87550|500431|2112|46593|56646|1344568|36|41670|32|5|2|3|254|38|65|57.53|65.58|34.75|2003.95|2258.75|3739.45|4262.70|0.00|0.00|681.85|2258.75|2258.75|2940.60|2940.60|-1480.70| +2451706|58174|2451773|4837|87550|500431|2112|46593|56646|1344568|36|41670|25|29|17|4|231|38|81|27.99|53.18|13.82|3188.16|1119.42|2267.19|4307.58|37.27|705.23|732.24|414.19|451.46|1146.43|1183.70|-1853.00| +2451706|58174|2451823|2807|87550|500431|2112|46593|56646|1344568|36|41670|50|7|5|3|231|38|60|95.90|226.32|83.73|8555.40|5023.80|5754.00|13579.20|43.20|2863.56|0.00|2160.24|2203.44|2160.24|2203.44|-3593.76| +2451706|58174|2451760|668|87550|500431|2112|46593|56646|1344568|36|41670|53|5|16|4|222|38|90|40.03|70.45|19.72|4565.70|1774.80|3602.70|6340.50|124.23|0.00|2536.20|1774.80|1899.03|4311.00|4435.23|-1827.90| +2451706|58174|2451725|8147|87550|500431|2112|46593|56646|1344568|36|41670|14|14|15|2|202|38|63|23.04|66.35|45.78|1295.91|2884.14|1451.52|4180.05|93.44|547.98|501.48|2336.16|2429.60|2837.64|2931.08|884.64| +2451706|58174|2451803|8467|87550|500431|2112|46593|56646|1344568|36|41670|13|19|13|1|199|38|61|1.68|2.03|1.38|39.65|84.18|102.48|123.83|0.00|0.00|6.10|84.18|84.18|90.28|90.28|-18.30| +2451706|58174|2451811|15536|87550|500431|2112|46593|56646|1344568|36|41670|43|7|3|5|17|38|20|84.56|235.92|75.49|3208.60|1509.80|1691.20|4718.40|30.19|0.00|2170.40|1509.80|1539.99|3680.20|3710.39|-181.40| +2451771|7639|2451841|8414|59990|963621|852|10083|35808|780901|1776|10207|35|7|10|3|59|39|55|24.63|53.20|17.55|1960.75|965.25|1354.65|2926.00|38.61|0.00|409.20|965.25|1003.86|1374.45|1413.06|-389.40| +2451771|7639|2451845|12727|59990|963621|852|10083|35808|780901|1776|10207|1|29|12|4|6|39|51|14.23|34.00|15.98|919.02|814.98|725.73|1734.00|57.04|0.00|797.64|814.98|872.02|1612.62|1669.66|89.25| +2451771|7639|2451812|2453|59990|963621|852|10083|35808|780901|1776|10207|59|7|1|5|279|39|79|81.48|190.66|173.50|1355.64|13706.50|6436.92|15062.14|137.06|0.00|6325.53|13706.50|13843.56|20032.03|20169.09|7269.58| +2451771|7639|2451780|11585|59990|963621|852|10083|35808|780901|1776|10207|5|25|3|3|43|39|15|2.97|7.90|1.65|93.75|24.75|44.55|118.50|1.23|0.00|31.95|24.75|25.98|56.70|57.93|-19.80| +2451771|7639|2451833|15116|59990|963621|852|10083|35808|780901|1776|10207|53|29|16|4|92|39|54|42.66|102.81|45.23|3109.32|2442.42|2303.64|5551.74|122.12|0.00|1498.50|2442.42|2564.54|3940.92|4063.04|138.78| +2451771|7639|2451786|7105|59990|963621|852|10083|35808|780901|1776|10207|17|25|17|3|32|39|37|54.94|154.38|1.54|5655.08|56.98|2032.78|5712.06|0.56|0.00|1770.45|56.98|57.54|1827.43|1827.99|-1975.80| +2451771|7639|2451823|3059|59990|963621|852|10083|35808|780901|1776|10207|49|8|5|5|65|39|88|69.90|118.83|39.21|7006.56|3450.48|6151.20|10457.04|138.01|0.00|418.00|3450.48|3588.49|3868.48|4006.49|-2700.72| +2451771|7639|2451861|13933|59990|963621|852|10083|35808|780901|1776|10207|11|26|12|5|8|39|77|62.72|149.27|16.41|10230.22|1263.57|4829.44|11493.79|15.03|1048.76|2643.41|214.81|229.84|2858.22|2873.25|-4614.63| +2451771|7639|2451824|13271|59990|963621|852|10083|35808|780901|1776|10207|31|19|14|3|112|39|34|52.75|132.40|54.28|2656.08|1845.52|1793.50|4501.60|110.73|0.00|1935.62|1845.52|1956.25|3781.14|3891.87|52.02| +2451771|7639|2451825|3371|59990|963621|852|10083|35808|780901|1776|10207|14|11|7|3|249|39|57|89.30|173.24|51.97|6912.39|2962.29|5090.10|9874.68|177.73|0.00|592.23|2962.29|3140.02|3554.52|3732.25|-2127.81| +2451771|7639|2451878|9725|59990|963621|852|10083|35808|780901|1776|10207|13|29|10|3|46|39|35|86.33|219.27|114.02|3683.75|3990.70|3021.55|7674.45|39.90|0.00|2378.95|3990.70|4030.60|6369.65|6409.55|969.15| +2451771|7639|2451834|745|59990|963621|852|10083|35808|780901|1776|10207|20|23|3|1|262|39|44|77.85|159.59|135.65|1053.36|5968.60|3425.40|7021.96|84.75|1730.89|3300.00|4237.71|4322.46|7537.71|7622.46|812.31| +2451771|7639|2451788|31|59990|963621|852|10083|35808|780901|1776|10207|2|1|3|2|194|39|100|21.26|35.50|0.35|3515.00|35.00|2126.00|3550.00|2.45|0.00|497.00|35.00|37.45|532.00|534.45|-2091.00| +2451771|7639|2451852|17555|59990|963621|852|10083|35808|780901|1776|10207|47|7|14|2|117|39|15|87.02|127.91|0.00|1918.65|0.00|1305.30|1918.65|0.00|0.00|0.00|0.00|0.00|0.00|0.00|-1305.30| +2451771|7639|2451845|2315|59990|963621|852|10083|35808|780901|1776|10207|56|19|4|5|269|39|17|42.71|70.04|49.72|345.44|845.24|726.07|1190.68|67.61|0.00|35.70|845.24|912.85|880.94|948.55|119.17| +2451771|7639|2451821|10237|59990|963621|852|10083|35808|780901|1776|10207|20|29|2|4|13|39|90|59.60|107.28|80.46|2413.80|7241.40|5364.00|9655.20|651.72|0.00|2413.80|7241.40|7893.12|9655.20|10306.92|1877.40| +2452232|81936|2452241|14203|16812|1614684|1803|33675|8730|781278|1291|27963|53|3|4|4|294|40|92|24.49|57.30|27.50|2741.60|2530.00|2253.08|5271.60|75.90|0.00|579.60|2530.00|2605.90|3109.60|3185.50|276.92| +2452232|81936|2452302|7355|16812|1614684|1803|33675|8730|781278|1291|27963|39|29|10|5|90|40|15|42.87|100.31|62.19|571.80|932.85|643.05|1504.65|0.93|914.19|315.90|18.66|19.59|334.56|335.49|-624.39| +2452232|81936|2452308|14733|16812|1614684|1803|33675|8730|781278|1291|27963|53|21|12|1|300|40|25|74.12|146.75|127.67|477.00|3191.75|1853.00|3668.75|127.67|0.00|807.00|3191.75|3319.42|3998.75|4126.42|1338.75| +2452232|81936|2452284|1275|16812|1614684|1803|33675|8730|781278|1291|27963|17|13|15|3|31|40|80|57.49|166.14|142.88|1860.80|11430.40|4599.20|13291.20|800.12|0.00|4784.80|11430.40|12230.52|16215.20|17015.32|6831.20| +2452232|81936|2452318|12561|16812|1614684|1803|33675|8730|781278|1291|27963|33|9|3|3|246|40|59|32.09|74.12|9.63|3804.91|568.17|1893.31|4373.08|45.45|0.00|961.70|568.17|613.62|1529.87|1575.32|-1325.14| +2452232|81936|2452252|3205|16812|1614684|1803|33675|8730|781278|1291|27963|1|5|5|4|76|40|87|97.37|127.55|42.09|7435.02|3661.83|8471.19|11096.85|183.09|0.00|3661.83|3661.83|3844.92|7323.66|7506.75|-4809.36| +2452232|81936|2452327|5783|16812|1614684|1803|33675|8730|781278|1291|27963|25|5|6|1|49|40|63|75.34|89.65|31.37|3671.64|1976.31|4746.42|5647.95|158.10|0.00|1072.89|1976.31|2134.41|3049.20|3207.30|-2770.11| +2452232|81936|2452346|2551|16812|1614684|1803|33675|8730|781278|1291|27963|43|9|1|4|210|40|24|52.53|123.44|104.92|444.48|2518.08|1260.72|2962.56|176.26|0.00|888.72|2518.08|2694.34|3406.80|3583.06|1257.36| +2452232|81936|2452307|105|16812|1614684|1803|33675|8730|781278|1291|27963|15|21|4|2|110|40|21|5.29|7.67|1.68|125.79|35.28|111.09|161.07|0.00|0.00|69.09|35.28|35.28|104.37|104.37|-75.81| +2452232|81936|2452294|14913|16812|1614684|1803|33675|8730|781278|1291|27963|13|1|12|1|297|40|78|25.29|55.13|20.94|2666.82|1633.32|1972.62|4300.14|6.86|1290.32|859.56|343.00|349.86|1202.56|1209.42|-1629.62| +2452232|81936|2452287|12283|16812|1614684|1803|33675|8730|781278|1291|27963|45|23|14|3|295|40|43|85.42|122.15|98.94|998.03|4254.42|3673.06|5252.45|170.17|0.00|1050.49|4254.42|4424.59|5304.91|5475.08|581.36| +2452232|81936|2452341|457|16812|1614684|1803|33675|8730|781278|1291|27963|33|21|12|1|217|40|25|62.96|160.54|77.05|2087.25|1926.25|1574.00|4013.50|0.00|0.00|1284.25|1926.25|1926.25|3210.50|3210.50|352.25| +2452232|81936|2452236|14863|16812|1614684|1803|33675|8730|781278|1291|27963|27|17|4|4|174|40|74|99.04|251.56|218.85|2420.54|16194.90|7328.96|18615.44|0.00|0.00|2792.02|16194.90|16194.90|18986.92|18986.92|8865.94| +2452232|81936|2452281|16639|16812|1614684|1803|33675|8730|781278|1291|27963|43|3|2|2|295|40|18|13.05|15.52|11.48|72.72|206.64|234.90|279.36|2.04|2.06|66.96|204.58|206.62|271.54|273.58|-30.32| +2452232|81936|2452310|17059|16812|1614684|1803|33675|8730|781278|1291|27963|11|27|2|2|285|40|45|29.82|36.38|20.00|737.10|900.00|1341.90|1637.10|35.28|459.00|229.05|441.00|476.28|670.05|705.33|-900.90| +2452232|81936|2452339|10589|16812|1614684|1803|33675|8730|781278|1291|27963|53|7|11|3|145|40|76|30.31|60.01|40.20|1505.56|3055.20|2303.56|4560.76|91.65|0.00|1003.20|3055.20|3146.85|4058.40|4150.05|751.64| +2451695|44895|2451749|7655|77062|920710|4805|35253|84180|554438|1797|34645|8|7|10|4|218|41|5|8.40|19.57|3.32|81.25|16.60|42.00|97.85|0.49|0.00|28.35|16.60|17.09|44.95|45.44|-25.40| +2451695|44895|2451780|13928|77062|920710|4805|35253|84180|554438|1797|34645|11|26|9|2|165|41|83|88.07|214.89|79.50|11237.37|6598.50|7309.81|17835.87|461.89|0.00|3744.96|6598.50|7060.39|10343.46|10805.35|-711.31| +2451695|44895|2451804|4763|77062|920710|4805|35253|84180|554438|1797|34645|37|25|17|1|295|41|40|21.24|28.67|21.21|298.40|848.40|849.60|1146.80|25.45|0.00|217.60|848.40|873.85|1066.00|1091.45|-1.20| +2451695|44895|2451735|2305|77062|920710|4805|35253|84180|554438|1797|34645|31|14|7|1|35|41|15|51.21|139.29|30.64|1629.75|459.60|768.15|2089.35|27.57|0.00|626.70|459.60|487.17|1086.30|1113.87|-308.55| +2451695|44895|2451745|16801|77062|920710|4805|35253|84180|554438|1797|34645|37|5|17|4|248|41|19|13.01|16.00|1.92|267.52|36.48|247.19|304.00|0.00|35.75|152.00|0.73|0.73|152.73|152.73|-246.46| +2451695|44895|2451705|8315|77062|920710|4805|35253|84180|554438|1797|34645|50|14|20|5|97|41|62|12.02|16.58|8.29|513.98|513.98|745.24|1027.96|35.97|0.00|431.52|513.98|549.95|945.50|981.47|-231.26| +2451695|44895|2451735|6833|77062|920710|4805|35253|84180|554438|1797|34645|56|11|3|3|80|41|15|50.82|95.03|68.42|399.15|1026.30|762.30|1425.45|41.05|0.00|399.00|1026.30|1067.35|1425.30|1466.35|264.00| +2451695|44895|2451751|17527|77062|920710|4805|35253|84180|554438|1797|34645|56|14|14|5|266|41|25|53.52|80.28|62.61|441.75|1565.25|1338.00|2007.00|46.95|0.00|441.50|1565.25|1612.20|2006.75|2053.70|227.25| +2451695|44895|2451768|9673|77062|920710|4805|35253|84180|554438|1797|34645|29|23|2|4|236|41|95|6.85|9.59|1.05|811.30|99.75|650.75|911.05|1.99|0.00|218.50|99.75|101.74|318.25|320.24|-551.00| +2451695|44895|2451755|15757|77062|920710|4805|35253|84180|554438|1797|34645|2|25|1|2|270|41|28|14.49|22.60|9.26|373.52|259.28|405.72|632.80|20.74|0.00|107.52|259.28|280.02|366.80|387.54|-146.44| +2451695|44895|2451792|16676|77062|920710|4805|35253|84180|554438|1797|34645|53|14|9|5|34|41|36|59.88|78.44|47.84|1101.60|1722.24|2155.68|2823.84|86.11|0.00|847.08|1722.24|1808.35|2569.32|2655.43|-433.44| +2451695|44895|2451748|3134|77062|920710|4805|35253|84180|554438|1797|34645|11|8|14|3|242|41|81|53.84|129.21|10.33|9629.28|836.73|4361.04|10466.01|33.46|0.00|3453.03|836.73|870.19|4289.76|4323.22|-3524.31| +2451695|44895|2451756|16637|77062|920710|4805|35253|84180|554438|1797|34645|53|29|20|5|150|41|59|37.38|43.73|41.98|103.25|2476.82|2205.42|2580.07|99.07|0.00|489.70|2476.82|2575.89|2966.52|3065.59|271.40| +2451695|44895|2451729|3865|77062|920710|4805|35253|84180|554438|1797|34645|37|17|12|2|265|41|28|90.88|238.10|40.47|5533.64|1133.16|2544.64|6666.80|79.32|0.00|2800.00|1133.16|1212.48|3933.16|4012.48|-1411.48| +2451695|44895|2451760|3164|77062|920710|4805|35253|84180|554438|1797|34645|11|19|14|5|179|41|35|9.44|23.50|13.86|337.40|485.10|330.40|822.50|19.40|0.00|8.05|485.10|504.50|493.15|512.55|154.70| +2451131|72439|2451172|5929|65471|863021|1371|10921|59379|860833|2324|37874|7|4|19|5|231|42|17|25.72|46.81|41.19|95.54|700.23|437.24|795.77|1.68|616.20|119.34|84.03|85.71|203.37|205.05|-353.21| +2451131|72439|2451207|8590|65471|863021|1371|10921|59379|860833|2324|37874|49|19|8|3|108|42|7|79.50|87.45|26.23|428.54|183.61|556.50|612.15|3.08|132.19|110.18|51.42|54.50|161.60|164.68|-505.08| +2451131|72439|2451251|3091|65471|863021|1371|10921|59379|860833|2324|37874|40|1|4|2|107|42|98|1.12|1.93|0.77|113.68|75.46|109.76|189.14|3.01|0.00|63.70|75.46|78.47|139.16|142.17|-34.30| +2451131|72439|2451200|12508|65471|863021|1371|10921|59379|860833|2324|37874|16|4|17|1|45|42|52|41.04|92.34|30.47|3217.24|1584.44|2134.08|4801.68|126.75|0.00|671.84|1584.44|1711.19|2256.28|2383.03|-549.64| +2451131|72439|2451201|16081|65471|863021|1371|10921|59379|860833|2324|37874|25|14|16|1|46|42|14|73.48|117.56|43.49|1036.98|608.86|1028.72|1645.84|3.16|529.70|674.66|79.16|82.32|753.82|756.98|-949.56| +2451131|72439|2451219|14503|65471|863021|1371|10921|59379|860833|2324|37874|25|20|19|4|102|42|76|53.02|127.24|0.00|9670.24|0.00|4029.52|9670.24|0.00|0.00|3093.96|0.00|0.00|3093.96|3093.96|-4029.52| +2451131|72439|2451215|5014|65471|863021|1371|10921|59379|860833|2324|37874|44|1|19|2|18|42|8|43.48|86.96|26.08|487.04|208.64|347.84|695.68|10.43|0.00|278.24|208.64|219.07|486.88|497.31|-139.20| +2451131|72439|2451152|3364|65471|863021|1371|10921|59379|860833|2324|37874|16|10|16|3|223|42|71|94.86|175.49|59.66|8223.93|4235.86|6735.06|12459.79|84.71|0.00|6105.29|4235.86|4320.57|10341.15|10425.86|-2499.20| +2451939|48209|2451994|11871|92793|1723260|6174|24241|88257|297951|136|38603|27|23|3|1|269|43|14|46.91|85.84|16.30|973.56|228.20|656.74|1201.76|0.18|209.94|132.16|18.26|18.44|150.42|150.60|-638.48| +2451939|48209|2451966|9141|92793|1723260|6174|24241|88257|297951|136|38603|1|5|5|2|219|43|19|52.65|61.60|14.78|889.58|280.82|1000.35|1170.40|25.27|0.00|315.97|280.82|306.09|596.79|622.06|-719.53| +2451939|48209|2452057|12785|92793|1723260|6174|24241|88257|297951|136|38603|49|11|17|3|219|43|8|6.43|9.51|5.42|32.72|43.36|51.44|76.08|3.90|0.00|4.56|43.36|47.26|47.92|51.82|-8.08| +2451939|48209|2451979|9201|92793|1723260|6174|24241|88257|297951|136|38603|49|27|10|4|145|43|51|43.09|50.41|40.83|488.58|2082.33|2197.59|2570.91|145.76|0.00|1208.19|2082.33|2228.09|3290.52|3436.28|-115.26| +2451939|48209|2451954|551|92793|1723260|6174|24241|88257|297951|136|38603|7|3|10|3|79|43|100|86.90|114.70|73.40|4130.00|7340.00|8690.00|11470.00|42.57|5211.40|4014.00|2128.60|2171.17|6142.60|6185.17|-6561.40| +2451939|48209|2451950|349|92793|1723260|6174|24241|88257|297951|136|38603|39|19|7|4|204|43|30|56.32|78.84|19.71|1773.90|591.30|1689.60|2365.20|41.39|0.00|236.40|591.30|632.69|827.70|869.09|-1098.30| +2451939|48209|2452055|17201|92793|1723260|6174|24241|88257|297951|136|38603|5|11|19|2|165|43|61|62.21|179.16|46.58|8087.38|2841.38|3794.81|10928.76|28.41|0.00|5027.01|2841.38|2869.79|7868.39|7896.80|-953.43| +2451939|48209|2452009|385|92793|1723260|6174|24241|88257|297951|136|38603|15|15|18|5|225|43|74|57.16|106.31|15.94|6687.38|1179.56|4229.84|7866.94|1.76|1144.17|314.50|35.39|37.15|349.89|351.65|-4194.45| +2451134|80873|2451161|17071|97122|1431191|2060|19329|57166|125023|2336|15605|31|7|19|2|222|44|16|69.46|163.23|19.58|2298.40|313.28|1111.36|2611.68|25.06|0.00|26.08|313.28|338.34|339.36|364.42|-798.08| +2451134|80873|2451173|5578|97122|1431191|2060|19329|57166|125023|2336|15605|1|28|19|1|200|44|60|90.06|230.55|209.80|1245.00|12588.00|5403.60|13833.00|251.76|0.00|1936.20|12588.00|12839.76|14524.20|14775.96|7184.40| +2451134|80873|2451193|973|97122|1431191|2060|19329|57166|125023|2336|15605|19|16|8|2|103|44|53|65.28|129.90|120.80|482.30|6402.40|3459.84|6884.70|128.04|0.00|1583.11|6402.40|6530.44|7985.51|8113.55|2942.56| +2451134|80873|2451190|3820|97122|1431191|2060|19329|57166|125023|2336|15605|43|26|19|1|193|44|10|80.30|165.41|107.51|579.00|1075.10|803.00|1654.10|83.21|150.51|827.00|924.59|1007.80|1751.59|1834.80|121.59| +2451134|80873|2451229|3223|97122|1431191|2060|19329|57166|125023|2336|15605|19|20|12|1|127|44|27|83.59|242.41|116.35|3403.62|3141.45|2256.93|6545.07|94.24|0.00|1832.49|3141.45|3235.69|4973.94|5068.18|884.52| +2451134|80873|2451217|14728|97122|1431191|2060|19329|57166|125023|2336|15605|16|28|1|5|266|44|23|50.49|122.18|21.99|2304.37|505.77|1161.27|2810.14|19.82|257.94|1095.95|247.83|267.65|1343.78|1363.60|-913.44| +2451134|80873|2451222|3889|97122|1431191|2060|19329|57166|125023|2336|15605|2|13|2|2|239|44|52|96.16|190.39|91.38|5148.52|4751.76|5000.32|9900.28|237.58|0.00|1187.68|4751.76|4989.34|5939.44|6177.02|-248.56| +2451134|80873|2451223|13294|97122|1431191|2060|19329|57166|125023|2336|15605|52|13|20|3|233|44|28|47.87|102.92|26.75|2132.76|749.00|1340.36|2881.76|5.24|644.14|1412.04|104.86|110.10|1516.90|1522.14|-1235.50| +2451134|80873|2451211|139|97122|1431191|2060|19329|57166|125023|2336|15605|8|8|8|1|21|44|36|20.77|39.04|3.51|1279.08|126.36|747.72|1405.44|10.10|0.00|351.36|126.36|136.46|477.72|487.82|-621.36| +2451150|39372|2451203|8912|39927|464905|4730|16108|60212|1659758|1889|40114|20|1|1|3|278|45|94|76.32|135.84|103.23|3065.34|9703.62|7174.08|12768.96|291.10|0.00|1786.94|9703.62|9994.72|11490.56|11781.66|2529.54| +2451150|39372|2451237|14362|39927|464905|4730|16108|60212|1659758|1889|40114|14|1|11|4|227|45|47|2.20|2.31|1.15|54.52|54.05|103.40|108.57|2.70|0.00|31.02|54.05|56.75|85.07|87.77|-49.35| +2451150|39372|2451249|1502|39927|464905|4730|16108|60212|1659758|1889|40114|38|19|20|5|167|45|25|86.71|234.98|129.23|2643.75|3230.75|2167.75|5874.50|32.30|0.00|117.25|3230.75|3263.05|3348.00|3380.30|1063.00| +2451150|39372|2451164|10166|39927|464905|4730|16108|60212|1659758|1889|40114|1|28|12|4|6|45|3|85.74|168.05|31.92|408.39|95.76|257.22|504.15|3.83|0.00|191.55|95.76|99.59|287.31|291.14|-161.46| +2451150|39372|2451233|7315|39927|464905|4730|16108|60212|1659758|1889|40114|13|25|16|2|158|45|33|50.34|61.91|9.28|1736.79|306.24|1661.22|2043.03|21.43|0.00|408.54|306.24|327.67|714.78|736.21|-1354.98| +2451150|39372|2451188|10448|39927|464905|4730|16108|60212|1659758|1889|40114|16|19|11|2|230|45|97|38.73|89.85|85.35|436.50|8278.95|3756.81|8715.45|82.78|0.00|2352.25|8278.95|8361.73|10631.20|10713.98|4522.14| +2451150|39372|2451167|2414|39927|464905|4730|16108|60212|1659758|1889|40114|7|26|4|4|14|45|26|53.43|110.06|3.30|2775.76|85.80|1389.18|2861.56|0.00|5.14|1258.92|80.66|80.66|1339.58|1339.58|-1308.52| +2451150|39372|2451261|11431|39927|464905|4730|16108|60212|1659758|1889|40114|58|16|11|5|149|45|76|60.25|160.26|80.13|6089.88|6089.88|4579.00|12179.76|16.44|4445.61|4750.00|1644.27|1660.71|6394.27|6410.71|-2934.73| +2452004|58928|2452009|515|81081|164538|2395|11052|47597|1141632|3070|1327|5|1|12|5|281|46|14|21.02|49.81|44.33|76.72|620.62|294.28|697.34|12.41|0.00|264.88|620.62|633.03|885.50|897.91|326.34| +2452004|58928|2452036|4843|81081|164538|2395|11052|47597|1141632|3070|1327|59|29|7|2|277|46|19|62.38|84.21|30.31|1024.10|575.89|1185.22|1599.99|46.07|0.00|783.94|575.89|621.96|1359.83|1405.90|-609.33| +2452004|58928|2452051|3385|81081|164538|2395|11052|47597|1141632|3070|1327|17|25|4|5|69|46|13|3.52|8.55|8.03|6.76|104.39|45.76|111.15|5.21|0.00|6.63|104.39|109.60|111.02|116.23|58.63| +2452004|58928|2452114|1441|81081|164538|2395|11052|47597|1141632|3070|1327|27|19|19|3|67|46|59|14.65|31.64|28.79|168.15|1698.61|864.35|1866.76|0.00|0.00|709.18|1698.61|1698.61|2407.79|2407.79|834.26| +2452004|58928|2452033|1791|81081|164538|2395|11052|47597|1141632|3070|1327|7|21|11|3|138|46|62|36.49|105.09|11.55|5799.48|716.10|2262.38|6515.58|50.12|0.00|65.10|716.10|766.22|781.20|831.32|-1546.28| +2452004|58928|2452035|11565|81081|164538|2395|11052|47597|1141632|3070|1327|33|7|18|2|16|46|46|42.48|72.64|60.29|568.10|2773.34|1954.08|3341.44|55.46|0.00|1302.72|2773.34|2828.80|4076.06|4131.52|819.26| +2452004|58928|2452081|11099|81081|164538|2395|11052|47597|1141632|3070|1327|7|9|3|4|45|46|90|61.80|130.39|19.55|9975.60|1759.50|5562.00|11735.10|158.35|0.00|585.90|1759.50|1917.85|2345.40|2503.75|-3802.50| +2452004|58928|2452094|12233|81081|164538|2395|11052|47597|1141632|3070|1327|57|19|1|5|246|46|89|54.96|146.74|96.84|4441.10|8618.76|4891.44|13059.86|444.72|1206.62|4309.38|7412.14|7856.86|11721.52|12166.24|2520.70| +2452004|58928|2452006|13719|81081|164538|2395|11052|47597|1141632|3070|1327|49|1|7|3|49|46|99|26.43|47.83|43.52|426.69|4308.48|2616.57|4735.17|86.16|0.00|141.57|4308.48|4394.64|4450.05|4536.21|1691.91| +2452004|58928|2452029|5447|81081|164538|2395|11052|47597|1141632|3070|1327|25|19|20|2|197|46|96|24.79|26.52|25.45|102.72|2443.20|2379.84|2545.92|48.86|0.00|75.84|2443.20|2492.06|2519.04|2567.90|63.36| +2452004|58928|2452083|9161|81081|164538|2395|11052|47597|1141632|3070|1327|43|13|9|1|163|46|69|76.67|86.63|65.83|1435.20|4542.27|5290.23|5977.47|234.38|635.91|2689.62|3906.36|4140.74|6595.98|6830.36|-1383.87| +2452004|58928|2452112|11823|81081|164538|2395|11052|47597|1141632|3070|1327|29|21|7|3|179|46|65|87.93|177.61|42.62|8774.35|2770.30|5715.45|11544.65|83.10|0.00|4271.15|2770.30|2853.40|7041.45|7124.55|-2945.15| +2452004|58928|2452110|15015|81081|164538|2395|11052|47597|1141632|3070|1327|55|13|3|2|258|46|47|33.51|45.23|28.49|786.78|1339.03|1574.97|2125.81|80.34|0.00|722.39|1339.03|1419.37|2061.42|2141.76|-235.94| +2452004|58928|2452020|6013|81081|164538|2395|11052|47597|1141632|3070|1327|53|3|4|4|54|46|23|4.36|8.19|3.19|115.00|73.37|100.28|188.37|2.20|0.00|41.40|73.37|75.57|114.77|116.97|-26.91| +2451533|33585|2451542|10142|76684|1465338|6439|14293|32665|1518423|6749|20203|7|13|3|1|32|47|91|98.12|195.25|117.15|7107.10|10660.65|8928.92|17767.75|95.94|1066.06|5329.87|9594.59|9690.53|14924.46|15020.40|665.67| +2451533|33585|2451572|15604|76684|1465338|6439|14293|32665|1518423|6749|20203|49|1|17|3|72|47|10|4.55|8.19|0.73|74.60|7.30|45.50|81.90|0.07|0.00|17.10|7.30|7.37|24.40|24.47|-38.20| +2451533|33585|2451553|1456|76684|1465338|6439|14293|32665|1518423|6749|20203|50|13|11|2|183|47|12|7.67|18.79|10.14|103.80|121.68|92.04|225.48|3.65|0.00|108.12|121.68|125.33|229.80|233.45|29.64| +2451533|33585|2451555|1297|76684|1465338|6439|14293|32665|1518423|6749|20203|55|2|13|4|215|47|87|43.05|121.83|65.78|4876.35|5722.86|3745.35|10599.21|400.60|0.00|635.10|5722.86|6123.46|6357.96|6758.56|1977.51| +2451533|33585|2451635|4952|76684|1465338|6439|14293|32665|1518423|6749|20203|40|20|18|5|235|47|38|22.24|64.94|41.56|888.44|1579.28|845.12|2467.72|78.96|0.00|1159.76|1579.28|1658.24|2739.04|2818.00|734.16| +2451533|33585|2451634|8497|76684|1465338|6439|14293|32665|1518423|6749|20203|13|14|19|5|41|47|58|3.32|6.04|5.97|4.06|346.26|192.56|350.32|11.63|55.40|41.76|290.86|302.49|332.62|344.25|98.30| +2451533|33585|2451621|44|76684|1465338|6439|14293|32665|1518423|6749|20203|26|2|10|3|49|47|24|4.61|8.75|0.61|195.36|14.64|110.64|210.00|1.31|0.00|69.12|14.64|15.95|83.76|85.07|-96.00| +2451533|33585|2451579|7444|76684|1465338|6439|14293|32665|1518423|6749|20203|58|1|15|4|163|47|4|90.91|113.63|31.81|327.28|127.24|363.64|454.52|8.90|0.00|168.16|127.24|136.14|295.40|304.30|-236.40| +2451533|33585|2451644|9674|76684|1465338|6439|14293|32665|1518423|6749|20203|19|8|10|3|193|47|67|2.83|6.19|0.24|398.65|16.08|189.61|414.73|0.48|0.00|57.62|16.08|16.56|73.70|74.18|-173.53| +2451439|19447|2451495|2968|83448|868542|4036|13577|76051|209298|4059|9200|8|20|7|1|172|48|40|55.26|147.54|116.55|1239.60|4662.00|2210.40|5901.60|419.58|0.00|530.80|4662.00|5081.58|5192.80|5612.38|2451.60| +2451439|19447|2451510|16682|83448|868542|4036|13577|76051|209298|4059|9200|52|2|14|1|8|48|61|86.80|165.78|48.07|7180.31|2932.27|5294.80|10112.58|80.93|909.00|3640.48|2023.27|2104.20|5663.75|5744.68|-3271.53| +2451439|19447|2451535|11158|83448|868542|4036|13577|76051|209298|4059|9200|43|19|18|4|27|48|55|96.71|278.52|272.94|306.90|15011.70|5319.05|15318.60|900.70|0.00|7199.50|15011.70|15912.40|22211.20|23111.90|9692.65| +2451439|19447|2451497|178|83448|868542|4036|13577|76051|209298|4059|9200|14|16|10|1|162|48|88|65.09|102.19|85.83|1439.68|7553.04|5727.92|8992.72|226.59|0.00|3686.32|7553.04|7779.63|11239.36|11465.95|1825.12| +2451439|19447|2451538|17092|83448|868542|4036|13577|76051|209298|4059|9200|2|2|12|2|242|48|80|3.23|6.33|2.34|319.20|187.20|258.40|506.40|0.26|174.09|222.40|13.11|13.37|235.51|235.77|-245.29| +2451439|19447|2451465|9268|83448|868542|4036|13577|76051|209298|4059|9200|38|25|19|5|151|48|25|60.97|63.40|24.09|982.75|602.25|1524.25|1585.00|48.18|0.00|63.25|602.25|650.43|665.50|713.68|-922.00| +2451439|19447|2451542|9910|83448|868542|4036|13577|76051|209298|4059|9200|52|4|9|5|171|48|20|84.82|199.32|41.85|3149.40|837.00|1696.40|3986.40|14.56|351.54|597.80|485.46|500.02|1083.26|1097.82|-1210.94| +2451439|19447|2451450|17834|83448|868542|4036|13577|76051|209298|4059|9200|7|25|5|4|273|48|100|25.69|45.72|2.74|4298.00|274.00|2569.00|4572.00|16.44|0.00|2011.00|274.00|290.44|2285.00|2301.44|-2295.00| +2451439|19447|2451490|17704|83448|868542|4036|13577|76051|209298|4059|9200|50|20|13|3|40|48|4|34.27|92.18|21.20|283.92|84.80|137.08|368.72|7.63|0.00|99.52|84.80|92.43|184.32|191.95|-52.28| +2451439|19447|2451485|16357|83448|868542|4036|13577|76051|209298|4059|9200|37|25|4|5|226|48|35|83.90|100.68|27.18|2572.50|951.30|2936.50|3523.80|57.07|0.00|1515.15|951.30|1008.37|2466.45|2523.52|-1985.20| +2451439|19447|2451528|7276|83448|868542|4036|13577|76051|209298|4059|9200|19|4|6|4|259|48|4|31.95|51.12|20.95|120.68|83.80|127.80|204.48|5.86|0.00|94.04|83.80|89.66|177.84|183.70|-44.00| +2451439|19447|2451507|17149|83448|868542|4036|13577|76051|209298|4059|9200|26|19|6|5|111|48|42|79.67|137.82|59.26|3299.52|2488.92|3346.14|5788.44|199.11|0.00|1273.44|2488.92|2688.03|3762.36|3961.47|-857.22| +2451439|19447|2451555|14000|83448|868542|4036|13577|76051|209298|4059|9200|52|19|13|3|26|48|49|21.45|58.12|27.31|1509.69|1338.19|1051.05|2847.88|0.00|0.00|113.68|1338.19|1338.19|1451.87|1451.87|287.14| +2451439|19447|2451466|17306|83448|868542|4036|13577|76051|209298|4059|9200|13|26|11|3|245|48|41|65.98|141.19|0.00|5788.79|0.00|2705.18|5788.79|0.00|0.00|578.51|0.00|0.00|578.51|578.51|-2705.18| +2451439|19447|2451554|4916|83448|868542|4036|13577|76051|209298|4059|9200|58|28|17|2|224|48|19|15.99|42.05|24.80|327.75|471.20|303.81|798.95|18.84|0.00|207.67|471.20|490.04|678.87|697.71|167.39| +2451439|19447|2451511|7819|83448|868542|4036|13577|76051|209298|4059|9200|22|16|3|5|97|48|29|24.69|48.88|5.37|1261.79|155.73|716.01|1417.52|0.00|0.00|269.12|155.73|155.73|424.85|424.85|-560.28| +2452291|9182|2452394|1281|74159|77626|3940|49054|71649|1811922|2702|9816|45|30|7|4|141|49|65|43.80|47.74|15.75|2079.35|1023.75|2847.00|3103.10|10.23|0.00|589.55|1023.75|1033.98|1613.30|1623.53|-1823.25| +2452291|9182|2452346|12033|74159|77626|3940|49054|71649|1811922|2702|9816|3|27|6|3|57|49|24|15.33|31.73|23.16|205.68|555.84|367.92|761.52|8.00|466.90|251.28|88.94|96.94|340.22|348.22|-278.98| +2452291|9182|2452392|10713|74159|77626|3940|49054|71649|1811922|2702|9816|18|1|16|3|83|49|40|71.16|183.59|38.55|5801.60|1542.00|2846.40|7343.60|92.52|0.00|440.40|1542.00|1634.52|1982.40|2074.92|-1304.40| +2452291|9182|2452381|1861|74159|77626|3940|49054|71649|1811922|2702|9816|27|27|11|1|51|49|56|42.41|51.31|49.77|86.24|2787.12|2374.96|2873.36|83.61|0.00|1149.12|2787.12|2870.73|3936.24|4019.85|412.16| +2452291|9182|2452389|17913|74159|77626|3940|49054|71649|1811922|2702|9816|21|30|19|3|87|49|32|68.14|196.24|49.06|4709.76|1569.92|2180.48|6279.68|0.00|0.00|2009.28|1569.92|1569.92|3579.20|3579.20|-610.56| +2452291|9182|2452318|17973|74159|77626|3940|49054|71649|1811922|2702|9816|9|7|6|3|257|49|14|77.88|144.07|36.01|1512.84|504.14|1090.32|2016.98|25.20|0.00|1008.42|504.14|529.34|1512.56|1537.76|-586.18| +2452291|9182|2452395|6399|74159|77626|3940|49054|71649|1811922|2702|9816|27|9|16|4|105|49|25|24.75|48.01|4.80|1080.25|120.00|618.75|1200.25|7.20|0.00|588.00|120.00|127.20|708.00|715.20|-498.75| +2452291|9182|2452350|13365|74159|77626|3940|49054|71649|1811922|2702|9816|27|1|8|3|45|49|81|65.92|166.11|134.54|2557.17|10897.74|5339.52|13454.91|762.84|0.00|5650.56|10897.74|11660.58|16548.30|17311.14|5558.22| +2452291|9182|2452382|9816|74159|77626|3940|49054|71649|1811922|2702|9816|1|1|11|1|181|49|22|96.03|189.17|3.78|4078.58|83.16|2112.66|4161.74|0.83|0.00|166.32|83.16|83.99|249.48|250.31|-2029.50| +2452291|9182|2452294|8883|74159|77626|3940|49054|71649|1811922|2702|9816|37|24|2|2|12|49|91|42.85|53.13|42.50|967.33|3867.50|3899.35|4834.83|142.32|2088.45|1111.11|1779.05|1921.37|2890.16|3032.48|-2120.30| +2452291|9182|2452340|6372|74159|77626|3940|49054|71649|1811922|2702|9816|9|21|20|1|112|49|20|97.01|186.25|80.08|2123.40|1601.60|1940.20|3725.00|144.14|0.00|782.20|1601.60|1745.74|2383.80|2527.94|-338.60| +2452291|9182|2452305|10083|74159|77626|3940|49054|71649|1811922|2702|9816|45|15|13|3|226|49|5|62.93|71.11|17.77|266.70|88.85|314.65|355.55|7.99|0.00|88.85|88.85|96.84|177.70|185.69|-225.80| +2452291|9182|2452329|7494|74159|77626|3940|49054|71649|1811922|2702|9816|6|12|19|5|158|49|20|37.84|55.24|5.52|994.40|110.40|756.80|1104.80|0.00|37.53|110.40|72.87|72.87|183.27|183.27|-683.93| +2452586|44689|2452649|1647|99181|380076|4171|7866|75251|339248|2151|19446|13|12|12|1|10|50|66|22.44|48.47|21.32|1791.90|1407.12|1481.04|3199.02|0.00|0.00|671.22|1407.12|1407.12|2078.34|2078.34|-73.92| +2452586|44689|2452666|4338|99181|380076|4171|7866|75251|339248|2151|19446|37|12|16|3|20|50|34|56.07|164.28|67.35|3295.62|2289.90|1906.38|5585.52|206.09|0.00|2457.52|2289.90|2495.99|4747.42|4953.51|383.52| +2452586|44689|2452698|11691|99181|380076|4171|7866|75251|339248|2151|19446|7|13|17|2|120|50|41|85.28|160.32|17.63|5850.29|722.83|3496.48|6573.12|28.91|0.00|3220.55|722.83|751.74|3943.38|3972.29|-2773.65| +2452586|44689|2452612|7927|99181|380076|4171|7866|75251|339248|2151|19446|27|25|2|3|17|50|20|75.71|80.25|52.96|545.80|1059.20|1514.20|1605.00|57.19|105.92|176.40|953.28|1010.47|1129.68|1186.87|-560.92| +2452586|44689|2452604|543|99181|380076|4171|7866|75251|339248|2151|19446|31|19|1|5|59|50|14|85.93|118.58|101.97|232.54|1427.58|1203.02|1660.12|114.20|0.00|464.80|1427.58|1541.78|1892.38|2006.58|224.56| +2452586|44689|2452645|6390|99181|380076|4171|7866|75251|339248|2151|19446|13|15|6|2|25|50|59|76.14|85.27|0.85|4980.78|50.15|4492.26|5030.93|2.00|0.00|1257.29|50.15|52.15|1307.44|1309.44|-4442.11| +2452586|44689|2452644|6151|99181|380076|4171|7866|75251|339248|2151|19446|42|3|20|3|176|50|28|65.64|183.13|139.17|1230.88|3896.76|1837.92|5127.64|77.93|0.00|717.64|3896.76|3974.69|4614.40|4692.33|2058.84| +2452586|44689|2452639|10747|99181|380076|4171|7866|75251|339248|2151|19446|48|1|18|2|89|50|18|11.78|15.07|8.28|122.22|149.04|212.04|271.26|0.00|0.00|78.66|149.04|149.04|227.70|227.70|-63.00| +2451876|17566|2451956|8201|31908|1837229|2550|26718|54961|1074489|5881|25249|5|2|1|3|82|51|88|51.83|51.83|5.70|4059.44|501.60|4561.04|4561.04|25.08|0.00|1550.56|501.60|526.68|2052.16|2077.24|-4059.44| +2451876|17566|2451957|2687|31908|1837229|2550|26718|54961|1074489|5881|25249|56|29|1|4|280|51|33|98.35|116.05|27.85|2910.60|919.05|3245.55|3829.65|0.00|0.00|1033.89|919.05|919.05|1952.94|1952.94|-2326.50| +2451876|17566|2451989|7226|31908|1837229|2550|26718|54961|1074489|5881|25249|19|26|8|2|26|51|47|90.57|120.45|110.81|453.08|5208.07|4256.79|5661.15|260.40|0.00|2037.92|5208.07|5468.47|7245.99|7506.39|951.28| +2451876|17566|2451906|4313|31908|1837229|2550|26718|54961|1074489|5881|25249|23|1|3|2|261|51|97|5.40|16.03|7.69|808.98|745.93|523.80|1554.91|29.83|0.00|481.12|745.93|775.76|1227.05|1256.88|222.13| +2451876|17566|2451891|8105|31908|1837229|2550|26718|54961|1074489|5881|25249|53|5|16|2|215|51|48|99.64|102.62|37.96|3103.68|1822.08|4782.72|4925.76|0.00|400.85|1132.80|1421.23|1421.23|2554.03|2554.03|-3361.49| +2451876|17566|2451926|11180|31908|1837229|2550|26718|54961|1074489|5881|25249|50|25|20|2|214|51|33|71.60|158.23|30.06|4229.61|991.98|2362.80|5221.59|59.51|0.00|1775.07|991.98|1051.49|2767.05|2826.56|-1370.82| +2451876|17566|2451897|13339|31908|1837229|2550|26718|54961|1074489|5881|25249|49|7|6|3|40|51|24|84.73|249.10|42.34|4962.24|1016.16|2033.52|5978.40|25.30|172.74|1972.80|843.42|868.72|2816.22|2841.52|-1190.10| +2451876|17566|2451888|10244|31908|1837229|2550|26718|54961|1074489|5881|25249|59|8|7|2|124|51|72|40.11|89.84|17.96|5175.36|1293.12|2887.92|6468.48|38.79|0.00|0.00|1293.12|1331.91|1293.12|1331.91|-1594.80| +2451876|17566|2451891|10115|31908|1837229|2550|26718|54961|1074489|5881|25249|26|7|7|2|57|51|51|27.21|67.75|0.00|3455.25|0.00|1387.71|3455.25|0.00|0.00|587.01|0.00|0.00|587.01|587.01|-1387.71| +2451876|17566|2451888|14384|31908|1837229|2550|26718|54961|1074489|5881|25249|25|14|13|5|60|51|85|81.44|190.56|190.56|0.00|16197.60|6922.40|16197.60|1457.78|0.00|5668.65|16197.60|17655.38|21866.25|23324.03|9275.20| +2451876|17566|2451927|8915|31908|1837229|2550|26718|54961|1074489|5881|25249|13|7|14|1|258|51|31|86.29|89.74|39.48|1558.06|1223.88|2674.99|2781.94|36.71|0.00|500.65|1223.88|1260.59|1724.53|1761.24|-1451.11| +2451876|17566|2451984|12025|31908|1837229|2550|26718|54961|1074489|5881|25249|50|11|3|3|140|51|38|86.64|160.28|36.86|4689.96|1400.68|3292.32|6090.64|56.02|0.00|121.60|1400.68|1456.70|1522.28|1578.30|-1891.64| +2451876|17566|2451931|3950|31908|1837229|2550|26718|54961|1074489|5881|25249|25|23|11|2|46|51|66|54.50|113.36|113.36|0.00|7481.76|3597.00|7481.76|523.72|0.00|523.38|7481.76|8005.48|8005.14|8528.86|3884.76| +2452301|48530|2452380|17928|80393|184753|1913|39826|71596|197177|3259|13619|9|3|2|4|226|52|7|27.24|35.41|24.07|79.38|168.49|190.68|247.87|0.00|121.31|4.90|47.18|47.18|52.08|52.08|-143.50| +2452301|48530|2452408|7254|80393|184753|1913|39826|71596|197177|3259|13619|42|15|5|2|269|52|31|61.74|98.16|84.41|426.25|2616.71|1913.94|3042.96|157.00|654.17|1308.20|1962.54|2119.54|3270.74|3427.74|48.60| +2452301|48530|2452346|4341|80393|184753|1913|39826|71596|197177|3259|13619|25|21|12|5|231|52|28|32.59|62.57|32.53|841.12|910.84|912.52|1751.96|61.20|145.73|122.36|765.11|826.31|887.47|948.67|-147.41| +2452301|48530|2452329|17148|80393|184753|1913|39826|71596|197177|3259|13619|27|9|6|2|68|52|43|16.40|21.64|0.86|893.54|36.98|705.20|930.52|2.95|0.00|288.10|36.98|39.93|325.08|328.03|-668.22| +2452301|48530|2452336|12138|80393|184753|1913|39826|71596|197177|3259|13619|7|18|14|4|47|52|98|66.56|133.12|50.58|8088.92|4956.84|6522.88|13045.76|99.13|0.00|6000.54|4956.84|5055.97|10957.38|11056.51|-1566.04| +2452301|48530|2452357|4842|80393|184753|1913|39826|71596|197177|3259|13619|27|15|15|2|126|52|86|11.22|25.02|21.26|323.36|1828.36|964.92|2151.72|91.41|0.00|408.50|1828.36|1919.77|2236.86|2328.27|863.44| +2452301|48530|2452363|16311|80393|184753|1913|39826|71596|197177|3259|13619|30|3|19|1|46|52|64|70.26|89.93|73.74|1036.16|4719.36|4496.64|5755.52|0.00|0.00|1093.12|4719.36|4719.36|5812.48|5812.48|222.72| +2452301|48530|2452406|2331|80393|184753|1913|39826|71596|197177|3259|13619|33|13|14|4|201|52|61|17.47|48.91|13.20|2178.31|805.20|1065.67|2983.51|32.20|0.00|298.29|805.20|837.40|1103.49|1135.69|-260.47| +2452301|48530|2452388|17394|80393|184753|1913|39826|71596|197177|3259|13619|31|21|19|4|79|52|76|48.57|96.16|69.23|2046.68|5261.48|3691.32|7308.16|315.68|0.00|145.92|5261.48|5577.16|5407.40|5723.08|1570.16| +2452301|48530|2452394|7717|80393|184753|1913|39826|71596|197177|3259|13619|30|24|6|5|257|52|42|12.06|20.14|16.31|160.86|685.02|506.52|845.88|13.70|0.00|329.70|685.02|698.72|1014.72|1028.42|178.50| +2452301|48530|2452420|3585|80393|184753|1913|39826|71596|197177|3259|13619|21|12|11|4|104|52|74|7.00|19.25|18.67|42.92|1381.58|518.00|1424.50|69.07|0.00|355.94|1381.58|1450.65|1737.52|1806.59|863.58| +2452301|48530|2452355|12798|80393|184753|1913|39826|71596|197177|3259|13619|31|24|11|2|53|52|67|73.74|85.53|16.25|4641.76|1088.75|4940.58|5730.51|65.32|0.00|1833.12|1088.75|1154.07|2921.87|2987.19|-3851.83| +2452301|48530|2452369|13914|80393|184753|1913|39826|71596|197177|3259|13619|36|7|20|1|255|52|80|34.04|69.44|8.33|4888.80|666.40|2723.20|5555.20|59.97|0.00|2166.40|666.40|726.37|2832.80|2892.77|-2056.80| +2452301|48530|2452346|1536|80393|184753|1913|39826|71596|197177|3259|13619|33|21|5|2|196|52|72|23.71|34.37|6.53|2004.48|470.16|1707.12|2474.64|42.31|0.00|172.80|470.16|512.47|642.96|685.27|-1236.96| +2452301|48530|2452351|3312|80393|184753|1913|39826|71596|197177|3259|13619|57|19|18|4|87|52|10|47.43|71.14|58.33|128.10|583.30|474.30|711.40|23.33|0.00|135.10|583.30|606.63|718.40|741.73|109.00| +2452301|48530|2452363|5139|80393|184753|1913|39826|71596|197177|3259|13619|37|9|1|2|293|52|40|33.47|68.94|20.68|1930.40|827.20|1338.80|2757.60|74.44|0.00|1158.00|827.20|901.64|1985.20|2059.64|-511.60| +2451691|45951|2451809|1417|54041|1071263|5732|49354|42172|1238469|3067|31889|32|17|14|1|98|53|48|39.94|113.82|48.94|3114.24|2349.12|1917.12|5463.36|46.98|0.00|2021.28|2349.12|2396.10|4370.40|4417.38|432.00| +2451691|45951|2451705|13189|54041|1071263|5732|49354|42172|1238469|3067|31889|13|1|15|4|103|53|35|20.49|37.49|23.61|485.80|826.35|717.15|1312.15|0.00|0.00|314.65|826.35|826.35|1141.00|1141.00|109.20| +2451691|45951|2451722|5285|54041|1071263|5732|49354|42172|1238469|3067|31889|29|25|4|5|46|53|36|73.45|164.52|105.29|2132.28|3790.44|2644.20|5922.72|54.58|2880.73|2013.48|909.71|964.29|2923.19|2977.77|-1734.49| +2451691|45951|2451777|14582|54041|1071263|5732|49354|42172|1238469|3067|31889|37|1|3|5|7|53|38|44.37|100.71|56.39|1684.16|2142.82|1686.06|3826.98|0.00|0.00|344.28|2142.82|2142.82|2487.10|2487.10|456.76| +2451691|45951|2451705|16154|54041|1071263|5732|49354|42172|1238469|3067|31889|37|26|12|2|142|53|97|55.62|140.71|46.43|9145.16|4503.71|5395.14|13648.87|0.00|0.00|5322.39|4503.71|4503.71|9826.10|9826.10|-891.43| +2451691|45951|2451703|14507|54041|1071263|5732|49354|42172|1238469|3067|31889|13|1|5|2|186|53|83|57.56|123.75|122.51|102.92|10168.33|4777.48|10271.25|711.78|0.00|3388.89|10168.33|10880.11|13557.22|14269.00|5390.85| +2451691|45951|2451719|8843|54041|1071263|5732|49354|42172|1238469|3067|31889|55|5|5|2|249|53|26|11.63|25.58|24.04|40.04|625.04|302.38|665.08|18.75|0.00|166.14|625.04|643.79|791.18|809.93|322.66| +2451691|45951|2451705|2435|54041|1071263|5732|49354|42172|1238469|3067|31889|35|11|7|3|189|53|38|5.89|13.84|3.46|394.44|131.48|223.82|525.92|2.76|39.44|73.34|92.04|94.80|165.38|168.14|-131.78| +2452594|39658|2452689|10080|17082|911361|4040|39519|83234|1635849|1956|19435|3|7|6|3|100|54|39|54.46|84.41|70.90|526.89|2765.10|2123.94|3291.99|221.20|0.00|296.01|2765.10|2986.30|3061.11|3282.31|641.16| +2452594|39658|2452638|15561|17082|911361|4040|39519|83234|1635849|1956|19435|49|7|16|3|107|54|18|80.04|226.51|38.50|3384.18|693.00|1440.72|4077.18|0.00|200.97|1263.78|492.03|492.03|1755.81|1755.81|-948.69| +2452594|39658|2452614|14373|17082|911361|4040|39519|83234|1635849|1956|19435|43|7|16|1|33|54|45|66.53|90.48|42.52|2158.20|1913.40|2993.85|4071.60|153.07|0.00|814.05|1913.40|2066.47|2727.45|2880.52|-1080.45| +2452594|39658|2452661|3228|17082|911361|4040|39519|83234|1635849|1956|19435|7|12|18|5|102|54|58|33.83|73.41|24.22|2853.02|1404.76|1962.14|4257.78|42.14|0.00|808.52|1404.76|1446.90|2213.28|2255.42|-557.38| +2452594|39658|2452670|14709|17082|911361|4040|39519|83234|1635849|1956|19435|12|27|12|5|38|54|85|94.67|131.59|65.79|5593.00|5592.15|8046.95|11185.15|167.76|0.00|4250.00|5592.15|5759.91|9842.15|10009.91|-2454.80| +2452594|39658|2452658|13281|17082|911361|4040|39519|83234|1635849|1956|19435|1|1|14|2|190|54|87|98.33|232.05|215.80|1413.75|18774.60|8554.71|20188.35|1284.18|4505.90|4239.51|14268.70|15552.88|18508.21|19792.39|5713.99| +2452594|39658|2452603|4905|17082|911361|4040|39519|83234|1635849|1956|19435|45|3|11|2|156|54|82|39.36|104.69|39.78|5322.62|3261.96|3227.52|8584.58|0.00|0.00|600.24|3261.96|3261.96|3862.20|3862.20|34.44| +2452594|39658|2452599|13983|17082|911361|4040|39519|83234|1635849|1956|19435|54|9|6|4|255|54|5|59.63|119.26|9.54|548.60|47.70|298.15|596.30|0.95|0.00|262.35|47.70|48.65|310.05|311.00|-250.45| +2452594|39658|2452666|15555|17082|911361|4040|39519|83234|1635849|1956|19435|21|24|20|3|149|54|77|44.08|52.45|47.20|404.25|3634.40|3394.16|4038.65|0.00|0.00|847.77|3634.40|3634.40|4482.17|4482.17|240.24| +2451788|53647|2451904|2129|9887|1273192|5650|4555|63252|1721892|5176|41681|37|17|6|3|270|55|3|72.56|214.77|133.15|244.86|399.45|217.68|644.31|31.95|0.00|122.40|399.45|431.40|521.85|553.80|181.77| +2451788|53647|2451791|17666|9887|1273192|5650|4555|63252|1721892|5176|41681|7|17|7|5|220|55|62|80.41|159.21|113.03|2863.16|7007.86|4985.42|9871.02|0.00|0.00|2862.54|7007.86|7007.86|9870.40|9870.40|2022.44| +2451788|53647|2451903|6559|9887|1273192|5650|4555|63252|1721892|5176|41681|49|19|15|5|235|55|28|50.96|137.08|5.48|3684.80|153.44|1426.88|3838.24|7.67|0.00|575.68|153.44|161.11|729.12|736.79|-1273.44| +2451788|53647|2451849|15277|9887|1273192|5650|4555|63252|1721892|5176|41681|19|13|4|1|21|55|19|28.96|77.61|23.28|1032.27|442.32|550.24|1474.59|8.84|0.00|265.24|442.32|451.16|707.56|716.40|-107.92| +2451788|53647|2451859|8378|9887|1273192|5650|4555|63252|1721892|5176|41681|14|19|2|4|136|55|57|70.89|192.82|179.32|769.50|10221.24|4040.73|10990.74|0.00|0.00|4176.39|10221.24|10221.24|14397.63|14397.63|6180.51| +2451788|53647|2451843|10471|9887|1273192|5650|4555|63252|1721892|5176|41681|55|19|9|3|7|55|100|80.00|94.40|19.82|7458.00|1982.00|8000.00|9440.00|59.46|0.00|1604.00|1982.00|2041.46|3586.00|3645.46|-6018.00| +2451788|53647|2451877|12053|9887|1273192|5650|4555|63252|1721892|5176|41681|44|11|19|2|82|55|72|33.05|56.51|44.07|895.68|3173.04|2379.60|4068.72|158.65|0.00|1667.52|3173.04|3331.69|4840.56|4999.21|793.44| +2451788|53647|2451898|5401|9887|1273192|5650|4555|63252|1721892|5176|41681|31|26|5|3|1|55|93|2.16|2.70|1.05|153.45|97.65|200.88|251.10|2.92|0.00|16.74|97.65|100.57|114.39|117.31|-103.23| +2451788|53647|2451876|16082|9887|1273192|5650|4555|63252|1721892|5176|41681|53|20|8|2|159|55|62|14.92|16.56|10.10|400.52|626.20|925.04|1026.72|43.83|0.00|451.36|626.20|670.03|1077.56|1121.39|-298.84| +2451788|53647|2451886|11165|9887|1273192|5650|4555|63252|1721892|5176|41681|53|13|17|5|150|55|96|74.35|179.18|173.80|516.48|16684.80|7137.60|17201.28|333.69|0.00|3783.36|16684.80|17018.49|20468.16|20801.85|9547.20| +2451788|53647|2451885|12013|9887|1273192|5650|4555|63252|1721892|5176|41681|50|11|9|4|59|55|26|96.74|155.75|10.90|3766.10|283.40|2515.24|4049.50|17.00|0.00|1538.68|283.40|300.40|1822.08|1839.08|-2231.84| +2451788|53647|2451858|17660|9887|1273192|5650|4555|63252|1721892|5176|41681|44|7|15|1|157|55|73|39.00|79.17|43.54|2600.99|3178.42|2847.00|5779.41|254.27|0.00|2715.60|3178.42|3432.69|5894.02|6148.29|331.42| +2451788|53647|2451866|6979|9887|1273192|5650|4555|63252|1721892|5176|41681|17|29|11|4|220|55|44|56.84|91.51|68.63|1006.72|3019.72|2500.96|4026.44|150.98|0.00|483.12|3019.72|3170.70|3502.84|3653.82|518.76| +2451788|53647|2451809|11509|9887|1273192|5650|4555|63252|1721892|5176|41681|8|14|11|4|294|55|94|93.40|243.77|63.38|16956.66|5957.72|8779.60|22914.38|59.57|0.00|8248.50|5957.72|6017.29|14206.22|14265.79|-2821.88| +2451788|53647|2451847|10085|9887|1273192|5650|4555|63252|1721892|5176|41681|14|26|3|4|235|55|35|54.20|81.30|7.31|2589.65|255.85|1897.00|2845.50|7.67|0.00|512.05|255.85|263.52|767.90|775.57|-1641.15| +2450832|77425|2450877|7783|91077|845709|341|42573|63692|506079|4916|45195|19|22|11|3|32|56|90|43.05|68.01|0.68|6059.70|61.20|3874.50|6120.90|1.22|0.00|1774.80|61.20|62.42|1836.00|1837.22|-3813.30| +2450832|77425|2450940|2336|91077|845709|341|42573|63692|506079|4916|45195|8|28|15|5|225|56|81|41.30|89.62|68.11|1742.31|5516.91|3345.30|7259.22|496.52|0.00|2032.29|5516.91|6013.43|7549.20|8045.72|2171.61| +2450832|77425|2450839|15643|91077|845709|341|42573|63692|506079|4916|45195|32|16|11|3|75|56|83|55.35|148.89|116.13|2719.08|9638.79|4594.05|12357.87|192.77|0.00|370.18|9638.79|9831.56|10008.97|10201.74|5044.74| +2450832|77425|2450917|16540|91077|845709|341|42573|63692|506079|4916|45195|14|13|6|5|34|56|57|60.02|158.45|134.68|1354.89|7676.76|3421.14|9031.65|460.60|0.00|541.50|7676.76|8137.36|8218.26|8678.86|4255.62| +2450832|77425|2450862|17467|91077|845709|341|42573|63692|506079|4916|45195|58|10|2|3|84|56|100|73.38|109.33|22.95|8638.00|2295.00|7338.00|10933.00|2.29|2249.10|1639.00|45.90|48.19|1684.90|1687.19|-7292.10| +2450832|77425|2450844|11918|91077|845709|341|42573|63692|506079|4916|45195|49|8|2|4|86|56|86|19.69|58.47|11.10|4073.82|954.60|1693.34|5028.42|85.91|0.00|2162.04|954.60|1040.51|3116.64|3202.55|-738.74| +2450832|77425|2450873|15670|91077|845709|341|42573|63692|506079|4916|45195|34|22|16|5|114|56|39|35.66|67.04|52.29|575.25|2039.31|1390.74|2614.56|40.78|0.00|522.60|2039.31|2080.09|2561.91|2602.69|648.57| +2450832|77425|2450937|2203|91077|845709|341|42573|63692|506079|4916|45195|25|13|1|2|296|56|42|94.50|120.01|63.60|2369.22|2671.20|3969.00|5040.42|45.67|2163.67|1360.80|507.53|553.20|1868.33|1914.00|-3461.47| +2450832|77425|2450881|748|91077|845709|341|42573|63692|506079|4916|45195|56|16|9|4|39|56|73|21.98|51.65|29.44|1621.33|2149.12|1604.54|3770.45|42.98|0.00|602.98|2149.12|2192.10|2752.10|2795.08|544.58| +2450832|77425|2450836|13480|91077|845709|341|42573|63692|506079|4916|45195|49|7|20|4|278|56|79|91.52|175.71|144.08|2498.77|11382.32|7230.08|13881.09|0.00|0.00|1248.99|11382.32|11382.32|12631.31|12631.31|4152.24| +2451251|30422|2451275|985|15972|27168|411|4807|80022|1239567|4842|45685|58|8|10|5|20|57|53|96.07|221.92|199.72|1176.60|10585.16|5091.71|11761.76|635.10|0.00|3645.87|10585.16|11220.26|14231.03|14866.13|5493.45| +2451251|30422|2451360|1765|15972|27168|411|4807|80022|1239567|4842|45685|10|13|8|4|250|57|23|12.07|15.32|14.40|21.16|331.20|277.61|352.36|6.62|0.00|155.02|331.20|337.82|486.22|492.84|53.59| +2451251|30422|2451320|17870|15972|27168|411|4807|80022|1239567|4842|45685|32|10|5|1|10|57|62|84.90|141.78|32.60|6769.16|2021.20|5263.80|8790.36|121.27|0.00|438.96|2021.20|2142.47|2460.16|2581.43|-3242.60| +2451251|30422|2451264|13990|15972|27168|411|4807|80022|1239567|4842|45685|28|22|2|4|289|57|51|96.36|102.14|77.62|1250.52|3958.62|4914.36|5209.14|106.88|2177.24|52.02|1781.38|1888.26|1833.40|1940.28|-3132.98| +2451251|30422|2451267|4466|15972|27168|411|4807|80022|1239567|4842|45685|44|14|10|5|150|57|40|39.22|114.91|57.45|2298.40|2298.00|1568.80|4596.40|68.94|0.00|1516.80|2298.00|2366.94|3814.80|3883.74|729.20| +2451251|30422|2451312|7999|15972|27168|411|4807|80022|1239567|4842|45685|34|13|16|5|274|57|56|70.51|198.83|105.37|5233.76|5900.72|3948.56|11134.48|354.04|0.00|445.20|5900.72|6254.76|6345.92|6699.96|1952.16| +2451251|30422|2451278|7358|15972|27168|411|4807|80022|1239567|4842|45685|32|7|11|3|175|57|14|57.41|99.31|82.42|236.46|1153.88|803.74|1390.34|0.00|0.00|625.52|1153.88|1153.88|1779.40|1779.40|350.14| +2451251|30422|2451335|9877|15972|27168|411|4807|80022|1239567|4842|45685|20|28|19|3|117|57|16|35.92|41.30|33.45|125.60|535.20|574.72|660.80|32.11|0.00|92.48|535.20|567.31|627.68|659.79|-39.52| +2451251|30422|2451330|11149|15972|27168|411|4807|80022|1239567|4842|45685|10|26|5|1|227|57|97|95.63|234.29|72.62|15681.99|7044.14|9276.11|22726.13|493.08|0.00|3862.54|7044.14|7537.22|10906.68|11399.76|-2231.97| +2451284|24184|2451287|1759|18917|614640|763|48039|54232|918925|6761|48084|1|19|6|5|141|58|74|20.42|33.89|7.11|1981.72|526.14|1511.08|2507.86|1.57|499.83|877.64|26.31|27.88|903.95|905.52|-1484.77| +2451284|24184|2451344|11342|18917|614640|763|48039|54232|918925|6761|48084|13|7|11|3|32|58|36|97.13|177.74|127.97|1791.72|4606.92|3496.68|6398.64|0.00|1520.28|2879.28|3086.64|3086.64|5965.92|5965.92|-410.04| +2451284|24184|2451386|6380|18917|614640|763|48039|54232|918925|6761|48084|56|28|1|2|153|58|1|19.35|35.60|7.83|27.77|7.83|19.35|35.60|0.07|0.00|10.68|7.83|7.90|18.51|18.58|-11.52| +2451284|24184|2451291|4376|18917|614640|763|48039|54232|918925|6761|48084|44|8|17|4|184|58|47|4.16|9.48|0.28|432.40|13.16|195.52|445.56|0.35|4.34|111.39|8.82|9.17|120.21|120.56|-186.70| +2451284|24184|2451319|17689|18917|614640|763|48039|54232|918925|6761|48084|26|14|2|1|163|58|57|64.09|66.65|55.98|608.19|3190.86|3653.13|3799.05|159.54|0.00|531.81|3190.86|3350.40|3722.67|3882.21|-462.27| +2451284|24184|2451291|2144|18917|614640|763|48039|54232|918925|6761|48084|20|20|4|2|260|58|40|73.71|114.25|82.26|1279.60|3290.40|2948.40|4570.00|98.71|0.00|319.60|3290.40|3389.11|3610.00|3708.71|342.00| +2451284|24184|2451403|5528|18917|614640|763|48039|54232|918925|6761|48084|52|4|19|2|208|58|26|89.45|222.73|184.86|984.62|4806.36|2325.70|5790.98|144.19|0.00|0.00|4806.36|4950.55|4806.36|4950.55|2480.66| +2451284|24184|2451403|13064|18917|614640|763|48039|54232|918925|6761|48084|26|7|17|2|20|58|13|72.56|203.89|73.40|1696.37|954.20|943.28|2650.57|47.71|0.00|795.08|954.20|1001.91|1749.28|1796.99|10.92| +2451284|24184|2451368|12380|18917|614640|763|48039|54232|918925|6761|48084|14|8|2|4|183|58|27|33.73|99.50|31.84|1826.82|859.68|910.71|2686.50|68.77|0.00|1262.52|859.68|928.45|2122.20|2190.97|-51.03| +2451284|24184|2451292|8167|18917|614640|763|48039|54232|918925|6761|48084|26|28|3|1|219|58|43|46.62|105.36|94.82|453.22|4077.26|2004.66|4530.48|203.86|0.00|45.15|4077.26|4281.12|4122.41|4326.27|2072.60| +2451284|24184|2451309|6302|18917|614640|763|48039|54232|918925|6761|48084|55|26|10|1|25|58|31|19.14|47.85|0.00|1483.35|0.00|593.34|1483.35|0.00|0.00|533.82|0.00|0.00|533.82|533.82|-593.34| +2451284|24184|2451346|3937|18917|614640|763|48039|54232|918925|6761|48084|10|8|4|5|159|58|30|65.83|117.17|9.37|3234.00|281.10|1974.90|3515.10|1.34|146.17|1405.80|134.93|136.27|1540.73|1542.07|-1839.97| +2451284|24184|2451350|17437|18917|614640|763|48039|54232|918925|6761|48084|58|1|12|3|124|58|62|4.19|5.40|1.94|214.52|120.28|259.78|334.80|7.21|0.00|156.86|120.28|127.49|277.14|284.35|-139.50| +2451484|62929|2451509|17162|33521|1812017|2960|29903|54903|853546|4685|35981|38|19|7|2|234|59|97|72.15|98.12|53.96|4283.52|5234.12|6998.55|9517.64|209.36|0.00|2474.47|5234.12|5443.48|7708.59|7917.95|-1764.43| +2451484|62929|2451596|14096|33521|1812017|2960|29903|54903|853546|4685|35981|38|26|8|4|93|59|72|30.97|39.95|30.36|690.48|2185.92|2229.84|2876.40|196.73|0.00|229.68|2185.92|2382.65|2415.60|2612.33|-43.92| +2451484|62929|2451572|13633|33521|1812017|2960|29903|54903|853546|4685|35981|34|14|8|3|38|59|30|26.19|41.64|36.64|150.00|1099.20|785.70|1249.20|87.93|0.00|224.70|1099.20|1187.13|1323.90|1411.83|313.50| +2451484|62929|2451582|1316|33521|1812017|2960|29903|54903|853546|4685|35981|50|8|17|1|168|59|12|92.35|121.90|81.67|482.76|980.04|1108.20|1462.80|78.40|0.00|614.28|980.04|1058.44|1594.32|1672.72|-128.16| +2451484|62929|2451563|13132|33521|1812017|2960|29903|54903|853546|4685|35981|32|4|2|3|157|59|88|85.97|132.39|38.39|8272.00|3378.32|7565.36|11650.32|135.13|0.00|2213.20|3378.32|3513.45|5591.52|5726.65|-4187.04| +2451484|62929|2451568|10658|33521|1812017|2960|29903|54903|853546|4685|35981|43|1|7|1|102|59|70|82.70|138.10|26.23|7830.90|1836.10|5789.00|9667.00|73.44|0.00|1159.90|1836.10|1909.54|2996.00|3069.44|-3952.90| +2451484|62929|2451531|1057|33521|1812017|2960|29903|54903|853546|4685|35981|25|14|6|1|136|59|53|38.90|114.75|97.53|912.66|5169.09|2061.70|6081.75|310.14|0.00|2249.85|5169.09|5479.23|7418.94|7729.08|3107.39| +2451484|62929|2451604|5918|33521|1812017|2960|29903|54903|853546|4685|35981|43|22|12|5|185|59|28|56.48|109.57|97.51|337.68|2730.28|1581.44|3067.96|191.11|0.00|920.36|2730.28|2921.39|3650.64|3841.75|1148.84| +2451484|62929|2451589|4420|33521|1812017|2960|29903|54903|853546|4685|35981|22|20|20|4|128|59|54|8.92|25.42|18.30|384.48|988.20|481.68|1372.68|19.76|0.00|178.20|988.20|1007.96|1166.40|1186.16|506.52| +2451484|62929|2451522|16933|33521|1812017|2960|29903|54903|853546|4685|35981|34|7|10|1|292|59|81|47.85|121.06|38.73|6668.73|3137.13|3875.85|9805.86|237.16|501.94|882.09|2635.19|2872.35|3517.28|3754.44|-1240.66| +2451060|34923|2451146|8629|4759|535841|6570|49652|1574|810653|3345|15274|40|14|12|4|73|60|42|6.80|17.34|14.56|116.76|611.52|285.60|728.28|36.69|0.00|57.96|611.52|648.21|669.48|706.17|325.92| +2451060|34923|2451151|1736|4759|535841|6570|49652|1574|810653|3345|15274|40|26|7|5|53|60|66|75.14|103.69|67.39|2395.80|4447.74|4959.24|6843.54|88.95|0.00|67.98|4447.74|4536.69|4515.72|4604.67|-511.50| +2451060|34923|2451068|6412|4759|535841|6570|49652|1574|810653|3345|15274|38|14|13|1|46|60|27|34.07|93.35|1.86|2470.23|50.22|919.89|2520.45|3.51|0.00|1083.78|50.22|53.73|1134.00|1137.51|-869.67| +2451060|34923|2451100|16339|4759|535841|6570|49652|1574|810653|3345|15274|7|14|2|3|207|60|99|15.08|17.34|14.91|240.57|1476.09|1492.92|1716.66|0.00|0.00|858.33|1476.09|1476.09|2334.42|2334.42|-16.83| +2451060|34923|2451110|1556|4759|535841|6570|49652|1574|810653|3345|15274|58|19|10|5|227|60|95|8.34|10.67|4.05|628.90|384.75|792.30|1013.65|23.08|0.00|384.75|384.75|407.83|769.50|792.58|-407.55| +2451060|34923|2451100|14683|4759|535841|6570|49652|1574|810653|3345|15274|16|20|16|3|280|60|18|26.23|71.34|23.54|860.40|423.72|472.14|1284.12|24.78|148.30|269.64|275.42|300.20|545.06|569.84|-196.72| +2451060|34923|2451098|5642|4759|535841|6570|49652|1574|810653|3345|15274|46|14|9|5|131|60|86|11.15|28.09|8.98|1643.46|772.28|958.90|2415.74|15.44|0.00|0.00|772.28|787.72|772.28|787.72|-186.62| +2451060|34923|2451101|10288|4759|535841|6570|49652|1574|810653|3345|15274|25|16|2|5|101|60|71|93.32|154.91|117.73|2639.78|8358.83|6625.72|10998.61|488.99|2925.59|2859.17|5433.24|5922.23|8292.41|8781.40|-1192.48| +2451060|34923|2451180|211|4759|535841|6570|49652|1574|810653|3345|15274|22|13|12|5|105|60|47|74.89|179.73|120.41|2788.04|5659.27|3519.83|8447.31|169.77|0.00|2787.57|5659.27|5829.04|8446.84|8616.61|2139.44| +2450821|49678|2450839|4424|85218|575823|828|14233|85218|575823|828|14233|10|22|1|2|46|61|8|58.27|117.70|1.17|932.24|9.36|466.16|941.60|0.30|1.68|423.68|7.68|7.98|431.36|431.66|-458.48| +2450821|49678|2450861|2311|85218|575823|828|14233|85218|575823|828|14233|8|20|17|1|52|61|3|41.26|49.51|14.85|103.98|44.55|123.78|148.53|3.56|0.00|60.87|44.55|48.11|105.42|108.98|-79.23| +2450821|49678|2450896|15494|85218|575823|828|14233|85218|575823|828|14233|43|25|14|5|106|61|10|64.43|169.45|138.94|305.10|1389.40|644.30|1694.50|0.00|0.00|474.40|1389.40|1389.40|1863.80|1863.80|745.10| +2450821|49678|2450858|8002|85218|575823|828|14233|85218|575823|828|14233|25|8|4|3|122|61|48|30.57|49.21|16.73|1559.04|803.04|1467.36|2362.08|48.90|104.39|519.36|698.65|747.55|1218.01|1266.91|-768.71| +2450821|49678|2450849|16477|85218|575823|828|14233|85218|575823|828|14233|1|26|17|5|53|61|50|41.35|124.05|96.75|1365.00|4837.50|2067.50|6202.50|387.00|0.00|1984.50|4837.50|5224.50|6822.00|7209.00|2770.00| +2450821|49678|2450904|10138|85218|575823|828|14233|85218|575823|828|14233|10|8|4|1|55|61|65|95.74|256.58|141.11|7505.55|9172.15|6223.10|16677.70|275.16|0.00|166.40|9172.15|9447.31|9338.55|9613.71|2949.05| +2450821|49678|2450839|8452|85218|575823|828|14233|85218|575823|828|14233|32|16|18|2|119|61|21|49.67|67.05|9.38|1211.07|196.98|1043.07|1408.05|5.90|0.00|563.22|196.98|202.88|760.20|766.10|-846.09| +2450821|49678|2450880|3343|85218|575823|828|14233|85218|575823|828|14233|50|19|14|5|119|61|32|42.65|96.38|79.99|524.48|2559.68|1364.80|3084.16|230.37|0.00|215.68|2559.68|2790.05|2775.36|3005.73|1194.88| +2450821|49678|2450859|17122|85218|575823|828|14233|85218|575823|828|14233|31|22|1|2|16|61|99|19.42|49.52|43.57|589.05|4313.43|1922.58|4902.48|86.26|0.00|1470.15|4313.43|4399.69|5783.58|5869.84|2390.85| +2450821|49678|2450924|13783|85218|575823|828|14233|85218|575823|828|14233|10|28|4|3|102|61|47|12.51|21.51|8.60|606.77|404.20|587.97|1010.97|16.16|0.00|90.71|404.20|420.36|494.91|511.07|-183.77| +2450821|49678|2450912|10184|85218|575823|828|14233|85218|575823|828|14233|20|16|7|3|206|61|67|61.72|135.16|112.18|1539.66|7516.06|4135.24|9055.72|526.12|0.00|3622.02|7516.06|8042.18|11138.08|11664.20|3380.82| +2450821|49678|2450935|2155|85218|575823|828|14233|85218|575823|828|14233|52|19|16|4|42|61|96|79.59|227.62|34.14|18574.08|3277.44|7640.64|21851.52|0.00|0.00|10925.76|3277.44|3277.44|14203.20|14203.20|-4363.20| +2450821|49678|2450861|5764|85218|575823|828|14233|85218|575823|828|14233|10|7|2|5|261|61|24|73.07|99.37|32.79|1597.92|786.96|1753.68|2384.88|31.47|0.00|786.96|786.96|818.43|1573.92|1605.39|-966.72| +2451505|60774|2451610|14443|85517|1119630|496|14209|56648|1762228|2101|47014|8|7|13|2|181|62|48|73.14|182.11|100.16|3933.60|4807.68|3510.72|8741.28|384.61|0.00|524.16|4807.68|5192.29|5331.84|5716.45|1296.96| +2451505|60774|2451597|9254|85517|1119630|496|14209|56648|1762228|2101|47014|8|16|13|2|231|62|68|19.89|27.64|23.77|263.16|1616.36|1352.52|1879.52|145.47|0.00|883.32|1616.36|1761.83|2499.68|2645.15|263.84| +2451505|60774|2451524|16075|85517|1119630|496|14209|56648|1762228|2101|47014|10|25|10|4|101|62|52|71.03|166.92|23.36|7465.12|1214.72|3693.56|8679.84|0.00|0.00|1909.44|1214.72|1214.72|3124.16|3124.16|-2478.84| +2451505|60774|2451517|1474|85517|1119630|496|14209|56648|1762228|2101|47014|31|22|18|1|277|62|27|15.73|42.62|32.81|264.87|885.87|424.71|1150.74|26.57|0.00|402.57|885.87|912.44|1288.44|1315.01|461.16| +2451505|60774|2451597|3532|85517|1119630|496|14209|56648|1762228|2101|47014|34|16|10|1|162|62|82|6.88|18.98|3.22|1292.32|264.04|564.16|1556.36|5.28|0.00|77.08|264.04|269.32|341.12|346.40|-300.12| +2451505|60774|2451620|16036|85517|1119630|496|14209|56648|1762228|2101|47014|1|16|10|3|86|62|35|37.25|76.73|42.96|1181.95|1503.60|1303.75|2685.55|75.18|0.00|590.80|1503.60|1578.78|2094.40|2169.58|199.85| +2451505|60774|2451517|6940|85517|1119630|496|14209|56648|1762228|2101|47014|56|4|7|1|15|62|5|36.93|55.02|31.91|115.55|159.55|184.65|275.10|0.00|0.00|112.75|159.55|159.55|272.30|272.30|-25.10| +2451505|60774|2451596|8252|85517|1119630|496|14209|56648|1762228|2101|47014|38|10|15|1|47|62|94|63.72|159.93|94.35|6164.52|8868.90|5989.68|15033.42|46.11|4257.07|5561.98|4611.83|4657.94|10173.81|10219.92|-1377.85| +2451500|66633|2451550|1702|53134|14005|4476|20147|38730|1354568|5113|9184|13|2|17|2|257|63|69|60.11|88.36|6.18|5670.42|426.42|4147.59|6096.84|8.52|0.00|1097.10|426.42|434.94|1523.52|1532.04|-3721.17| +2451500|66633|2451531|12037|53134|14005|4476|20147|38730|1354568|5113|9184|26|1|8|5|292|63|77|72.59|213.41|93.90|9202.27|7230.30|5589.43|16432.57|274.75|1735.27|5093.55|5495.03|5769.78|10588.58|10863.33|-94.40| +2451500|66633|2451577|7369|53134|14005|4476|20147|38730|1354568|5113|9184|32|14|9|5|281|63|30|31.03|43.44|38.22|156.60|1146.60|930.90|1303.20|8.48|722.35|364.80|424.25|432.73|789.05|797.53|-506.65| +2451500|66633|2451592|7378|53134|14005|4476|20147|38730|1354568|5113|9184|46|2|20|1|139|63|21|82.15|210.30|159.82|1060.08|3356.22|1725.15|4416.30|201.37|0.00|1589.70|3356.22|3557.59|4945.92|5147.29|1631.07| +2451500|66633|2451603|6536|53134|14005|4476|20147|38730|1354568|5113|9184|55|14|5|5|142|63|31|43.29|115.58|40.45|2329.03|1253.95|1341.99|3582.98|32.35|714.75|644.80|539.20|571.55|1184.00|1216.35|-802.79| +2451500|66633|2451616|15130|53134|14005|4476|20147|38730|1354568|5113|9184|26|19|16|4|5|63|53|63.95|171.38|128.53|2271.05|6812.09|3389.35|9083.14|272.48|0.00|1271.47|6812.09|7084.57|8083.56|8356.04|3422.74| +2451500|66633|2451577|14287|53134|14005|4476|20147|38730|1354568|5113|9184|46|16|4|4|114|63|68|46.08|98.61|46.34|3554.36|3151.12|3133.44|6705.48|126.04|0.00|2145.40|3151.12|3277.16|5296.52|5422.56|17.68| +2451500|66633|2451597|799|53134|14005|4476|20147|38730|1354568|5113|9184|55|2|9|1|270|63|80|92.96|242.62|50.95|15333.60|4076.00|7436.80|19409.60|326.08|0.00|7957.60|4076.00|4402.08|12033.60|12359.68|-3360.80| +2451500|66633|2451582|16402|53134|14005|4476|20147|38730|1354568|5113|9184|31|20|16|3|243|63|66|43.62|88.11|51.10|2442.66|3372.60|2878.92|5815.26|0.00|0.00|1860.54|3372.60|3372.60|5233.14|5233.14|493.68| +2451500|66633|2451511|15430|53134|14005|4476|20147|38730|1354568|5113|9184|52|22|17|2|140|63|14|91.59|262.86|202.40|846.44|2833.60|1282.26|3680.04|170.01|0.00|1508.78|2833.60|3003.61|4342.38|4512.39|1551.34| +2451500|66633|2451517|2630|53134|14005|4476|20147|38730|1354568|5113|9184|50|26|5|3|61|63|50|16.33|42.78|39.35|171.50|1967.50|816.50|2139.00|108.01|767.32|342.00|1200.18|1308.19|1542.18|1650.19|383.68| +2451500|66633|2451503|4636|53134|14005|4476|20147|38730|1354568|5113|9184|1|1|18|3|8|63|84|37.17|97.01|77.60|1630.44|6518.40|3122.28|8148.84|391.10|0.00|407.40|6518.40|6909.50|6925.80|7316.90|3396.12| +2451073|44810|2451090|3319|782|1915675|4968|10214|77544|1088240|5246|17339|34|25|18|3|95|64|17|13.85|15.23|1.67|230.52|28.39|235.45|258.91|1.70|0.00|67.15|28.39|30.09|95.54|97.24|-207.06| +2451073|44810|2451158|14062|782|1915675|4968|10214|77544|1088240|5246|17339|43|10|20|4|228|64|80|26.74|58.02|30.17|2228.00|2413.60|2139.20|4641.60|72.40|0.00|2273.60|2413.60|2486.00|4687.20|4759.60|274.40| +2451073|44810|2451178|4930|782|1915675|4968|10214|77544|1088240|5246|17339|46|10|8|3|65|64|87|48.02|129.65|50.56|6880.83|4398.72|4177.74|11279.55|87.97|0.00|2029.71|4398.72|4486.69|6428.43|6516.40|220.98| +2451073|44810|2451140|5008|782|1915675|4968|10214|77544|1088240|5246|17339|10|20|6|4|19|64|51|38.61|44.40|22.64|1109.76|1154.64|1969.11|2264.40|46.18|0.00|814.98|1154.64|1200.82|1969.62|2015.80|-814.47| +2451073|44810|2451176|7285|782|1915675|4968|10214|77544|1088240|5246|17339|19|28|10|5|300|64|96|44.34|91.34|71.24|1929.60|6839.04|4256.64|8768.64|478.73|0.00|3506.88|6839.04|7317.77|10345.92|10824.65|2582.40| +2451073|44810|2451171|10772|782|1915675|4968|10214|77544|1088240|5246|17339|43|7|3|5|47|64|27|16.39|22.94|7.79|409.05|210.33|442.53|619.38|2.10|0.00|129.87|210.33|212.43|340.20|342.30|-232.20| +2451073|44810|2451122|238|782|1915675|4968|10214|77544|1088240|5246|17339|14|10|4|2|120|64|76|23.26|47.91|33.05|1129.36|2511.80|1767.76|3641.16|75.35|0.00|728.08|2511.80|2587.15|3239.88|3315.23|744.04| +2451073|44810|2451131|14348|782|1915675|4968|10214|77544|1088240|5246|17339|25|8|4|3|115|64|31|41.17|87.69|14.90|2256.49|461.90|1276.27|2718.39|12.56|147.80|815.30|314.10|326.66|1129.40|1141.96|-962.17| +2451072|31719|2451182|12742|26772|378122|5919|6069|88160|1267011|1719|33916|28|22|12|4|176|65|59|14.71|17.94|0.53|1027.19|31.27|867.89|1058.46|0.93|0.00|73.75|31.27|32.20|105.02|105.95|-836.62| +2451072|31719|2451108|14242|26772|378122|5919|6069|88160|1267011|1719|33916|28|28|15|3|188|65|57|65.79|185.52|89.04|5499.36|5075.28|3750.03|10574.64|101.50|0.00|3701.01|5075.28|5176.78|8776.29|8877.79|1325.25| +2451072|31719|2451088|604|26772|378122|5919|6069|88160|1267011|1719|33916|16|10|13|5|20|65|66|60.18|158.27|14.24|9505.98|939.84|3971.88|10445.82|56.39|0.00|312.84|939.84|996.23|1252.68|1309.07|-3032.04| +2451072|31719|2451154|7054|26772|378122|5919|6069|88160|1267011|1719|33916|16|2|8|4|170|65|16|1.79|5.10|2.24|45.76|35.84|28.64|81.60|1.43|0.00|40.80|35.84|37.27|76.64|78.07|7.20| +2451072|31719|2451121|2317|26772|378122|5919|6069|88160|1267011|1719|33916|31|10|16|1|36|65|7|28.33|54.39|11.96|297.01|83.72|198.31|380.73|0.07|76.18|41.86|7.54|7.61|49.40|49.47|-190.77| +2451072|31719|2451156|5776|26772|378122|5919|6069|88160|1267011|1719|33916|55|4|18|3|205|65|67|29.45|55.36|7.19|3227.39|481.73|1973.15|3709.12|28.90|0.00|259.29|481.73|510.63|741.02|769.92|-1491.42| +2451072|31719|2451075|12526|26772|378122|5919|6069|88160|1267011|1719|33916|19|7|18|2|46|65|64|10.38|25.22|4.03|1356.16|257.92|664.32|1614.08|1.70|87.69|807.04|170.23|171.93|977.27|978.97|-494.09| +2451072|31719|2451174|12316|26772|378122|5919|6069|88160|1267011|1719|33916|44|20|2|3|231|65|58|28.90|61.55|44.93|963.96|2605.94|1676.20|3569.90|234.53|0.00|749.36|2605.94|2840.47|3355.30|3589.83|929.74| +2451072|31719|2451082|1952|26772|378122|5919|6069|88160|1267011|1719|33916|8|2|12|4|129|65|40|93.48|193.50|158.67|1393.20|6346.80|3739.20|7740.00|444.27|0.00|2322.00|6346.80|6791.07|8668.80|9113.07|2607.60| +2451072|31719|2451185|12106|26772|378122|5919|6069|88160|1267011|1719|33916|16|2|12|2|55|65|96|40.38|105.39|66.39|3744.00|6373.44|3876.48|10117.44|0.00|0.00|404.16|6373.44|6373.44|6777.60|6777.60|2496.96| +2451072|31719|2451118|5893|26772|378122|5919|6069|88160|1267011|1719|33916|52|14|6|2|78|65|68|62.77|134.95|9.44|8534.68|641.92|4268.36|9176.60|38.51|0.00|2660.84|641.92|680.43|3302.76|3341.27|-3626.44| +2451072|31719|2451088|13250|26772|378122|5919|6069|88160|1267011|1719|33916|25|4|10|5|190|65|63|80.84|164.10|19.69|9097.83|1240.47|5092.92|10338.30|0.00|0.00|4341.96|1240.47|1240.47|5582.43|5582.43|-3852.45| +2451334|82725|2451344|11893|97747|570993|2639|24285|28739|188604|6823|26022|50|20|16|1|116|66|24|19.98|43.35|0.86|1019.76|20.64|479.52|1040.40|1.85|0.00|478.56|20.64|22.49|499.20|501.05|-458.88| +2451334|82725|2451381|1832|97747|570993|2639|24285|28739|188604|6823|26022|10|1|6|1|249|66|27|49.18|121.47|40.08|2197.53|1082.16|1327.86|3279.69|26.62|638.47|787.05|443.69|470.31|1230.74|1257.36|-884.17| +2451334|82725|2451356|17270|97747|570993|2639|24285|28739|188604|6823|26022|26|1|18|3|195|66|60|30.05|61.30|52.71|515.40|3162.60|1803.00|3678.00|0.00|0.00|882.60|3162.60|3162.60|4045.20|4045.20|1359.60| +2451334|82725|2451346|17582|97747|570993|2639|24285|28739|188604|6823|26022|7|20|5|2|196|66|7|2.47|4.59|0.78|26.67|5.46|17.29|32.13|0.05|2.83|7.00|2.63|2.68|9.63|9.68|-14.66| +2451334|82725|2451414|17662|97747|570993|2639|24285|28739|188604|6823|26022|37|22|8|1|44|66|42|74.77|198.14|160.49|1581.30|6740.58|3140.34|8321.88|5.39|6470.95|1331.40|269.63|275.02|1601.03|1606.42|-2870.71| +2451334|82725|2451454|3098|97747|570993|2639|24285|28739|188604|6823|26022|49|1|10|1|35|66|94|17.90|51.19|50.67|48.88|4762.98|1682.60|4811.86|428.66|0.00|529.22|4762.98|5191.64|5292.20|5720.86|3080.38| +2451334|82725|2451412|4201|97747|570993|2639|24285|28739|188604|6823|26022|19|26|15|4|70|66|73|39.86|40.25|28.17|881.84|2056.41|2909.78|2938.25|0.00|0.00|1145.37|2056.41|2056.41|3201.78|3201.78|-853.37| +2451334|82725|2451437|715|97747|570993|2639|24285|28739|188604|6823|26022|10|16|12|5|18|66|91|32.27|40.66|15.85|2257.71|1442.35|2936.57|3700.06|13.55|86.54|665.21|1355.81|1369.36|2021.02|2034.57|-1580.76| +2451334|82725|2451353|2410|97747|570993|2639|24285|28739|188604|6823|26022|37|7|19|4|199|66|81|23.28|67.27|65.92|109.35|5339.52|1885.68|5448.87|427.16|0.00|1089.45|5339.52|5766.68|6428.97|6856.13|3453.84| +2451334|82725|2451384|5395|97747|570993|2639|24285|28739|188604|6823|26022|7|2|3|4|35|66|47|74.21|110.57|105.04|259.91|4936.88|3487.87|5196.79|246.84|0.00|1558.99|4936.88|5183.72|6495.87|6742.71|1449.01| +2451334|82725|2451409|13192|97747|570993|2639|24285|28739|188604|6823|26022|1|19|9|3|7|66|15|29.03|63.86|5.74|871.80|86.10|435.45|957.90|4.30|0.00|316.05|86.10|90.40|402.15|406.45|-349.35| +2451334|82725|2451355|2180|97747|570993|2639|24285|28739|188604|6823|26022|44|20|2|5|268|66|89|7.96|9.63|7.41|197.58|659.49|708.44|857.07|6.59|0.00|33.82|659.49|666.08|693.31|699.90|-48.95| +2451871|52842|2451877|5653|862|752153|4691|20829|1777|1782422|3196|25334|53|11|15|4|280|67|35|18.04|33.55|14.09|681.10|493.15|631.40|1174.25|4.93|0.00|457.80|493.15|498.08|950.95|955.88|-138.25| +2451871|52842|2451902|16922|862|752153|4691|20829|1777|1782422|3196|25334|29|11|17|5|46|67|11|62.05|84.38|2.53|900.35|27.83|682.55|928.18|0.00|0.00|389.73|27.83|27.83|417.56|417.56|-654.72| +2451871|52842|2451885|5545|862|752153|4691|20829|1777|1782422|3196|25334|26|14|2|1|290|67|47|20.16|38.30|9.95|1332.45|467.65|947.52|1800.10|9.35|0.00|647.66|467.65|477.00|1115.31|1124.66|-479.87| +2451871|52842|2451950|12067|862|752153|4691|20829|1777|1782422|3196|25334|19|7|8|4|23|67|18|19.92|59.36|40.36|342.00|726.48|358.56|1068.48|5.81|653.83|256.32|72.65|78.46|328.97|334.78|-285.91| +2451871|52842|2451955|10259|862|752153|4691|20829|1777|1782422|3196|25334|41|20|11|1|71|67|38|88.90|149.35|116.49|1248.68|4426.62|3378.20|5675.30|88.53|0.00|1816.02|4426.62|4515.15|6242.64|6331.17|1048.42| +2451871|52842|2451968|7339|862|752153|4691|20829|1777|1782422|3196|25334|8|29|8|3|1|67|36|67.81|84.76|78.82|213.84|2837.52|2441.16|3051.36|170.25|0.00|1281.24|2837.52|3007.77|4118.76|4289.01|396.36| +2451871|52842|2451891|4406|862|752153|4691|20829|1777|1782422|3196|25334|7|25|2|1|160|67|11|75.10|190.75|162.13|314.82|1783.43|826.10|2098.25|0.00|0.00|860.20|1783.43|1783.43|2643.63|2643.63|957.33| +2451871|52842|2451953|10022|862|752153|4691|20829|1777|1782422|3196|25334|26|1|18|3|225|67|68|76.76|98.25|93.33|334.56|6346.44|5219.68|6681.00|441.71|825.03|2337.84|5521.41|5963.12|7859.25|8300.96|301.73| +2451871|52842|2451881|9539|862|752153|4691|20829|1777|1782422|3196|25334|14|14|18|5|15|67|67|55.03|106.20|6.37|6688.61|426.79|3687.01|7115.40|21.33|0.00|995.62|426.79|448.12|1422.41|1443.74|-3260.22| +2451871|52842|2451907|12617|862|752153|4691|20829|1777|1782422|3196|25334|37|5|17|1|225|67|81|91.76|128.46|110.47|1457.19|8948.07|7432.56|10405.26|243.38|2863.38|4786.29|6084.69|6328.07|10870.98|11114.36|-1347.87| +2451871|52842|2451966|6175|862|752153|4691|20829|1777|1782422|3196|25334|31|5|20|4|241|67|10|26.54|31.58|6.94|246.40|69.40|265.40|315.80|0.69|0.00|60.00|69.40|70.09|129.40|130.09|-196.00| +2451871|52842|2451898|4814|862|752153|4691|20829|1777|1782422|3196|25334|59|19|20|5|40|67|16|44.60|119.97|37.19|1324.48|595.04|713.60|1919.52|11.90|0.00|844.48|595.04|606.94|1439.52|1451.42|-118.56| +2451871|52842|2451900|4322|862|752153|4691|20829|1777|1782422|3196|25334|55|2|13|4|88|67|10|24.47|49.67|49.67|0.00|496.70|244.70|496.70|0.00|0.00|4.90|496.70|496.70|501.60|501.60|252.00| +2451871|52842|2451883|15929|862|752153|4691|20829|1777|1782422|3196|25334|56|29|11|2|183|67|35|99.45|124.31|88.26|1261.75|3089.10|3480.75|4350.85|278.01|0.00|1696.80|3089.10|3367.11|4785.90|5063.91|-391.65| +2451871|52842|2451989|8369|862|752153|4691|20829|1777|1782422|3196|25334|31|17|9|2|152|67|5|97.24|216.84|86.73|650.55|433.65|486.20|1084.20|17.34|0.00|520.40|433.65|450.99|954.05|971.39|-52.55| +2451111|7790|2451145|16916|34219|956323|827|44131|96141|526889|2843|7347|56|7|16|1|26|68|9|36.11|81.60|38.35|389.25|345.15|324.99|734.40|20.70|0.00|58.68|345.15|365.85|403.83|424.53|20.16| +2451111|7790|2451168|15382|34219|956323|827|44131|96141|526889|2843|7347|31|7|9|4|162|68|3|59.37|143.08|123.04|60.12|369.12|178.11|429.24|8.08|99.66|154.50|269.46|277.54|423.96|432.04|91.35| +2451111|7790|2451148|3664|34219|956323|827|44131|96141|526889|2843|7347|38|19|10|5|190|68|43|4.78|11.28|8.00|141.04|344.00|205.54|485.04|20.64|0.00|154.80|344.00|364.64|498.80|519.44|138.46| +2451111|7790|2451144|13448|34219|956323|827|44131|96141|526889|2843|7347|44|13|1|5|259|68|18|44.75|115.00|57.50|1035.00|1035.00|805.50|2070.00|72.45|0.00|496.80|1035.00|1107.45|1531.80|1604.25|229.50| +2451111|7790|2451219|17134|34219|956323|827|44131|96141|526889|2843|7347|16|7|12|5|155|68|90|35.34|100.01|2.00|8820.90|180.00|3180.60|9000.90|10.80|0.00|2790.00|180.00|190.80|2970.00|2980.80|-3000.60| +2451111|7790|2451209|6670|34219|956323|827|44131|96141|526889|2843|7347|55|22|6|5|84|68|4|38.27|61.23|60.61|2.48|242.44|153.08|244.92|0.24|237.59|41.60|4.85|5.09|46.45|46.69|-148.23| +2451111|7790|2451197|283|34219|956323|827|44131|96141|526889|2843|7347|4|4|13|3|250|68|31|66.00|143.88|58.99|2631.59|1828.69|2046.00|4460.28|164.58|0.00|1070.43|1828.69|1993.27|2899.12|3063.70|-217.31| +2451111|7790|2451203|13963|34219|956323|827|44131|96141|526889|2843|7347|20|8|12|1|220|68|37|35.05|101.29|87.10|525.03|3222.70|1296.85|3747.73|0.00|0.00|74.74|3222.70|3222.70|3297.44|3297.44|1925.85| +2451111|7790|2451220|3607|34219|956323|827|44131|96141|526889|2843|7347|43|1|7|3|109|68|72|55.53|126.05|1.26|8984.88|90.72|3998.16|9075.60|1.81|0.00|181.44|90.72|92.53|272.16|273.97|-3907.44| +2451111|7790|2451195|908|34219|956323|827|44131|96141|526889|2843|7347|28|25|4|2|203|68|90|66.09|138.78|29.14|9867.60|2622.60|5948.10|12490.20|26.22|0.00|2747.70|2622.60|2648.82|5370.30|5396.52|-3325.50| +2451111|7790|2451119|11198|34219|956323|827|44131|96141|526889|2843|7347|38|8|17|2|64|68|22|45.23|117.14|114.79|51.70|2525.38|995.06|2577.08|101.01|0.00|1004.96|2525.38|2626.39|3530.34|3631.35|1530.32| +2451111|7790|2451171|4136|34219|956323|827|44131|96141|526889|2843|7347|16|16|19|4|129|68|84|25.23|66.60|33.30|2797.20|2797.20|2119.32|5594.40|139.86|0.00|614.88|2797.20|2937.06|3412.08|3551.94|677.88| +2451111|7790|2451114|5320|34219|956323|827|44131|96141|526889|2843|7347|58|2|14|1|188|68|69|71.68|177.76|110.21|4660.95|7604.49|4945.92|12265.44|182.50|5323.14|3556.95|2281.35|2463.85|5838.30|6020.80|-2664.57| +2451111|7790|2451160|6368|34219|956323|827|44131|96141|526889|2843|7347|20|28|6|5|281|68|38|5.19|7.57|2.72|184.30|103.36|197.22|287.66|1.03|0.00|68.78|103.36|104.39|172.14|173.17|-93.86| +2451887|75024|2451965|11261|64487|1617286|3238|36037|38806|1840690|5987|25986|19|11|15|1|250|69|64|55.44|99.23|45.64|3429.76|2920.96|3548.16|6350.72|175.25|0.00|888.96|2920.96|3096.21|3809.92|3985.17|-627.20| +2451887|75024|2451937|14857|64487|1617286|3238|36037|38806|1840690|5987|25986|47|2|20|5|183|69|67|44.05|123.34|13.56|7355.26|908.52|2951.35|8263.78|36.34|0.00|1487.40|908.52|944.86|2395.92|2432.26|-2042.83| +2451887|75024|2451917|17383|64487|1617286|3238|36037|38806|1840690|5987|25986|2|2|2|5|295|69|38|61.16|101.52|89.33|463.22|3394.54|2324.08|3857.76|4.07|3292.70|1465.66|101.84|105.91|1567.50|1571.57|-2222.24| +2451887|75024|2451965|53|64487|1617286|3238|36037|38806|1840690|5987|25986|1|23|3|4|135|69|76|16.82|25.90|20.72|393.68|1574.72|1278.32|1968.40|15.74|0.00|98.04|1574.72|1590.46|1672.76|1688.50|296.40| +2451887|75024|2451908|9902|64487|1617286|3238|36037|38806|1840690|5987|25986|55|19|12|1|113|69|95|37.28|65.98|36.28|2821.50|3446.60|3541.60|6268.10|21.36|2378.15|1190.35|1068.45|1089.81|2258.80|2280.16|-2473.15| +2451887|75024|2451892|4103|64487|1617286|3238|36037|38806|1840690|5987|25986|23|1|3|5|277|69|54|41.12|80.18|7.21|3940.38|389.34|2220.48|4329.72|0.00|0.00|1428.30|389.34|389.34|1817.64|1817.64|-1831.14| +2451887|75024|2451954|15599|64487|1617286|3238|36037|38806|1840690|5987|25986|55|1|4|4|77|69|27|35.53|83.85|47.79|973.62|1290.33|959.31|2263.95|64.51|0.00|837.54|1290.33|1354.84|2127.87|2192.38|331.02| +2451887|75024|2451896|8509|64487|1617286|3238|36037|38806|1840690|5987|25986|29|25|8|3|268|69|22|22.84|63.72|59.89|84.26|1317.58|502.48|1401.84|92.23|0.00|42.02|1317.58|1409.81|1359.60|1451.83|815.10| +2451887|75024|2451954|13148|64487|1617286|3238|36037|38806|1840690|5987|25986|19|2|4|1|68|69|30|57.10|91.36|82.22|274.20|2466.60|1713.00|2740.80|172.66|0.00|794.70|2466.60|2639.26|3261.30|3433.96|753.60| +2451887|75024|2451975|17030|64487|1617286|3238|36037|38806|1840690|5987|25986|32|23|20|5|230|69|73|89.79|154.43|105.01|3607.66|7665.73|6554.67|11273.39|459.94|0.00|2141.82|7665.73|8125.67|9807.55|10267.49|1111.06| +2451973|69074|2451989|13799|18177|860787|2597|32760|84347|736031|4943|13106|51|29|17|1|215|70|42|5.71|16.67|13.33|140.28|559.86|239.82|700.14|0.00|67.18|168.00|492.68|492.68|660.68|660.68|252.86| +2451973|69074|2452044|7633|18177|860787|2597|32760|84347|736031|4943|13106|25|19|7|2|264|70|2|69.25|165.50|152.26|26.48|304.52|138.50|331.00|15.22|0.00|6.62|304.52|319.74|311.14|326.36|166.02| +2451973|69074|2451986|7721|18177|860787|2597|32760|84347|736031|4943|13106|1|7|20|3|179|70|27|74.91|140.83|70.41|1901.34|1901.07|2022.57|3802.41|171.09|0.00|684.18|1901.07|2072.16|2585.25|2756.34|-121.50| +2451973|69074|2452085|10441|18177|860787|2597|32760|84347|736031|4943|13106|57|27|19|4|149|70|13|79.90|120.64|119.43|15.73|1552.59|1038.70|1568.32|62.10|0.00|219.44|1552.59|1614.69|1772.03|1834.13|513.89| +2451973|69074|2452020|2911|18177|860787|2597|32760|84347|736031|4943|13106|21|29|9|5|184|70|8|78.72|109.42|93.00|131.36|744.00|629.76|875.36|14.88|0.00|175.04|744.00|758.88|919.04|933.92|114.24| +2451973|69074|2451980|9355|18177|860787|2597|32760|84347|736031|4943|13106|21|25|5|2|266|70|2|12.04|31.18|11.84|38.68|23.68|24.08|62.36|0.71|0.00|23.68|23.68|24.39|47.36|48.07|-0.40| +2451973|69074|2452062|1819|18177|860787|2597|32760|84347|736031|4943|13106|47|11|11|3|268|70|36|4.70|13.63|0.00|490.68|0.00|169.20|490.68|0.00|0.00|73.44|0.00|0.00|73.44|73.44|-169.20| +2451973|69074|2451998|8945|18177|860787|2597|32760|84347|736031|4943|13106|45|17|8|2|73|70|86|13.92|16.70|1.16|1336.44|99.76|1197.12|1436.20|0.00|0.00|617.48|99.76|99.76|717.24|717.24|-1097.36| +2451973|69074|2452053|6473|18177|860787|2597|32760|84347|736031|4943|13106|59|1|12|3|244|70|23|63.23|186.52|93.26|2144.98|2144.98|1454.29|4289.96|128.69|0.00|1158.28|2144.98|2273.67|3303.26|3431.95|690.69| +2451973|69074|2452038|3261|18177|860787|2597|32760|84347|736031|4943|13106|29|19|5|5|235|70|81|26.34|47.67|0.00|3861.27|0.00|2133.54|3861.27|0.00|0.00|463.32|0.00|0.00|463.32|463.32|-2133.54| +2451973|69074|2452013|4767|18177|860787|2597|32760|84347|736031|4943|13106|25|15|13|1|76|70|82|7.62|10.13|3.24|564.98|265.68|624.84|830.66|1.75|90.33|165.64|175.35|177.10|340.99|342.74|-449.49| +2451348|41853|2451456|16196|48452|1550645|1991|43784|15572|1372301|6669|48462|40|13|13|4|55|71|16|67.06|103.94|86.27|282.72|1380.32|1072.96|1663.04|0.00|0.00|332.48|1380.32|1380.32|1712.80|1712.80|307.36| +2451348|41853|2451452|8570|48452|1550645|1991|43784|15572|1372301|6669|48462|2|8|11|4|300|71|28|82.63|111.55|24.54|2436.28|687.12|2313.64|3123.40|54.96|0.00|437.08|687.12|742.08|1124.20|1179.16|-1626.52| +2451348|41853|2451444|10940|48452|1550645|1991|43784|15572|1372301|6669|48462|2|10|10|4|258|71|20|62.98|64.23|35.96|565.40|719.20|1259.60|1284.60|21.57|0.00|603.60|719.20|740.77|1322.80|1344.37|-540.40| +2451348|41853|2451438|8035|48452|1550645|1991|43784|15572|1372301|6669|48462|22|16|3|2|284|71|27|6.48|6.93|0.06|185.49|1.62|174.96|187.11|0.08|0.00|11.07|1.62|1.70|12.69|12.77|-173.34| +2451348|41853|2451468|4918|48452|1550645|1991|43784|15572|1372301|6669|48462|46|2|13|4|245|71|70|62.89|96.85|49.39|3322.20|3457.30|4402.30|6779.50|103.71|0.00|1355.90|3457.30|3561.01|4813.20|4916.91|-945.00| +2451348|41853|2451378|14281|48452|1550645|1991|43784|15572|1372301|6669|48462|44|26|15|4|40|71|70|65.38|179.79|145.62|2391.90|10193.40|4576.60|12585.30|0.00|0.00|628.60|10193.40|10193.40|10822.00|10822.00|5616.80| +2451348|41853|2451446|10939|48452|1550645|1991|43784|15572|1372301|6669|48462|8|1|17|3|229|71|9|99.54|287.67|198.49|802.62|1786.41|895.86|2589.03|7.68|1018.25|1268.55|768.16|775.84|2036.71|2044.39|-127.70| +2451348|41853|2451396|14474|48452|1550645|1991|43784|15572|1372301|6669|48462|56|19|1|4|230|71|14|36.18|41.24|31.34|138.60|438.76|506.52|577.36|30.71|0.00|63.42|438.76|469.47|502.18|532.89|-67.76| +2451348|41853|2451371|13408|48452|1550645|1991|43784|15572|1372301|6669|48462|28|2|12|4|129|71|14|36.63|105.49|50.63|768.04|708.82|512.82|1476.86|21.26|0.00|472.50|708.82|730.08|1181.32|1202.58|196.00| +2451348|41853|2451436|2527|48452|1550645|1991|43784|15572|1372301|6669|48462|32|16|11|3|278|71|85|70.71|178.89|114.48|5474.85|9730.80|6010.35|15205.65|326.95|1556.92|1064.20|8173.88|8500.83|9238.08|9565.03|2163.53| +2451348|41853|2451423|11995|48452|1550645|1991|43784|15572|1372301|6669|48462|46|20|15|2|46|71|45|31.55|93.70|46.85|2108.25|2108.25|1419.75|4216.50|84.33|0.00|843.30|2108.25|2192.58|2951.55|3035.88|688.50| +2451348|41853|2451359|2012|48452|1550645|1991|43784|15572|1372301|6669|48462|32|16|18|2|6|71|35|8.79|10.54|5.58|173.60|195.30|307.65|368.90|7.81|0.00|84.70|195.30|203.11|280.00|287.81|-112.35| +2451348|41853|2451354|11714|48452|1550645|1991|43784|15572|1372301|6669|48462|43|13|12|5|255|71|53|52.37|101.59|28.44|3876.95|1507.32|2775.61|5384.27|120.58|0.00|968.84|1507.32|1627.90|2476.16|2596.74|-1268.29| +2452129|70279|2452216|6577|90064|644490|6116|46095|90064|644490|6116|46095|51|19|7|1|105|72|23|69.02|195.32|187.50|179.86|4312.50|1587.46|4492.36|4.31|4226.25|1931.54|86.25|90.56|2017.79|2022.10|-1501.21| +2452129|70279|2452223|6831|90064|644490|6116|46095|90064|644490|6116|46095|47|21|18|5|2|72|100|95.29|141.02|66.27|7475.00|6627.00|9529.00|14102.00|596.43|0.00|282.00|6627.00|7223.43|6909.00|7505.43|-2902.00| +2452129|70279|2452235|5997|90064|644490|6116|46095|90064|644490|6116|46095|13|19|3|3|116|72|6|31.86|75.50|53.60|131.40|321.60|191.16|453.00|25.72|0.00|95.10|321.60|347.32|416.70|442.42|130.44| +2452129|70279|2452192|8669|90064|644490|6116|46095|90064|644490|6116|46095|37|1|13|5|277|72|52|34.99|74.17|57.11|887.12|2969.72|1819.48|3856.84|237.57|0.00|1426.88|2969.72|3207.29|4396.60|4634.17|1150.24| +2452129|70279|2452187|599|90064|644490|6116|46095|90064|644490|6116|46095|39|25|19|2|128|72|43|69.38|165.12|51.18|4899.42|2200.74|2983.34|7100.16|198.06|0.00|1703.66|2200.74|2398.80|3904.40|4102.46|-782.60| +2452129|70279|2452178|7013|90064|644490|6116|46095|90064|644490|6116|46095|57|25|20|2|214|72|14|42.79|50.92|36.15|206.78|506.10|599.06|712.88|20.95|156.89|85.54|349.21|370.16|434.75|455.70|-249.85| +2452129|70279|2452131|2405|90064|644490|6116|46095|90064|644490|6116|46095|57|13|5|3|271|72|38|21.33|46.92|2.34|1694.04|88.92|810.54|1782.96|0.88|0.00|320.72|88.92|89.80|409.64|410.52|-721.62| +2452129|70279|2452141|12447|90064|644490|6116|46095|90064|644490|6116|46095|45|1|9|2|182|72|10|6.46|19.12|14.14|49.80|141.40|64.60|191.20|5.65|0.00|66.90|141.40|147.05|208.30|213.95|76.80| +2452129|70279|2452249|2677|90064|644490|6116|46095|90064|644490|6116|46095|55|7|7|3|149|72|82|66.14|190.48|102.85|7185.66|8433.70|5423.48|15619.36|337.34|0.00|7497.26|8433.70|8771.04|15930.96|16268.30|3010.22| +2452228|66853|2452245|2337|92090|1415581|3519|18210|49682|1303843|330|41821|23|27|3|5|57|73|51|80.92|216.05|183.64|1652.91|9365.64|4126.92|11018.55|655.59|0.00|4296.75|9365.64|10021.23|13662.39|14317.98|5238.72| +2452228|66853|2452335|15643|92090|1415581|3519|18210|49682|1303843|330|41821|15|9|19|1|153|73|53|3.08|8.90|7.20|90.10|381.60|163.24|471.70|26.71|0.00|112.89|381.60|408.31|494.49|521.20|218.36| +2452228|66853|2452312|16541|92090|1415581|3519|18210|49682|1303843|330|41821|41|3|3|4|275|73|68|69.65|160.89|130.32|2078.76|8861.76|4736.20|10940.52|177.23|0.00|2734.96|8861.76|9038.99|11596.72|11773.95|4125.56| +2452228|66853|2452268|17467|92090|1415581|3519|18210|49682|1303843|330|41821|23|9|5|3|88|73|10|6.80|17.88|17.70|1.80|177.00|68.00|178.80|12.39|0.00|3.50|177.00|189.39|180.50|192.89|109.00| +2452228|66853|2452283|11919|92090|1415581|3519|18210|49682|1303843|330|41821|49|23|16|5|174|73|37|74.08|108.15|70.29|1400.82|2600.73|2740.96|4001.55|52.01|0.00|79.92|2600.73|2652.74|2680.65|2732.66|-140.23| +2452228|66853|2452316|15671|92090|1415581|3519|18210|49682|1303843|330|41821|29|7|3|2|31|73|12|55.64|153.56|52.21|1216.20|626.52|667.68|1842.72|0.00|0.00|55.20|626.52|626.52|681.72|681.72|-41.16| +2452228|66853|2452237|2203|92090|1415581|3519|18210|49682|1303843|330|41821|3|9|18|1|254|73|91|56.53|137.93|53.79|7656.74|4894.89|5144.23|12551.63|293.69|0.00|2007.46|4894.89|5188.58|6902.35|7196.04|-249.34| +2452228|66853|2452321|749|92090|1415581|3519|18210|49682|1303843|330|41821|51|27|2|3|271|73|65|28.20|31.30|18.46|834.60|1199.90|1833.00|2034.50|35.99|0.00|81.25|1199.90|1235.89|1281.15|1317.14|-633.10| +2452228|66853|2452320|13481|92090|1415581|3519|18210|49682|1303843|330|41821|13|27|14|1|201|73|15|51.67|88.35|13.25|1126.50|198.75|775.05|1325.25|0.00|0.00|119.25|198.75|198.75|318.00|318.00|-576.30| +2452562|82677|2452639|14736|2199|133121|4043|37765|3979|20382|4978|35500|45|1|19|2|258|74|24|91.85|194.72|118.77|1822.80|2850.48|2204.40|4673.28|171.02|0.00|1308.48|2850.48|3021.50|4158.96|4329.98|646.08| +2452562|82677|2452614|16410|2199|133121|4043|37765|3979|20382|4978|35500|57|9|18|4|150|74|9|28.82|72.33|20.97|462.24|188.73|259.38|650.97|16.98|0.00|123.66|188.73|205.71|312.39|329.37|-70.65| +2452562|82677|2452623|17329|2199|133121|4043|37765|3979|20382|4978|35500|9|3|2|2|212|74|55|37.45|60.66|2.42|3203.20|133.10|2059.75|3336.30|10.64|0.00|700.15|133.10|143.74|833.25|843.89|-1926.65| +2452562|82677|2452623|14535|2199|133121|4043|37765|3979|20382|4978|35500|60|21|5|1|37|74|95|65.05|152.86|105.47|4502.05|10019.65|6179.75|14521.70|400.78|0.00|4501.10|10019.65|10420.43|14520.75|14921.53|3839.90| +2452562|82677|2452636|17859|2199|133121|4043|37765|3979|20382|4978|35500|36|12|7|3|69|74|14|30.69|57.69|29.42|395.78|411.88|429.66|807.66|1.44|267.72|104.86|144.16|145.60|249.02|250.46|-285.50| +2452562|82677|2452602|463|2199|133121|4043|37765|3979|20382|4978|35500|42|6|8|3|265|74|7|33.43|61.51|60.27|8.68|421.89|234.01|430.57|33.75|0.00|25.83|421.89|455.64|447.72|481.47|187.88| +2452562|82677|2452610|4521|2199|133121|4043|37765|3979|20382|4978|35500|57|19|8|4|140|74|93|89.92|267.06|197.62|6457.92|18378.66|8362.56|24836.58|1286.50|0.00|6456.99|18378.66|19665.16|24835.65|26122.15|10016.10| +2452562|82677|2452650|13027|2199|133121|4043|37765|3979|20382|4978|35500|7|6|15|4|83|74|93|57.52|95.48|79.24|1510.32|7369.32|5349.36|8879.64|221.07|0.00|1597.74|7369.32|7590.39|8967.06|9188.13|2019.96| +2452562|82677|2452592|4476|2199|133121|4043|37765|3979|20382|4978|35500|21|13|3|4|85|74|69|42.30|64.71|20.06|3080.85|1384.14|2918.70|4464.99|55.36|0.00|1160.58|1384.14|1439.50|2544.72|2600.08|-1534.56| +2452497|48409|2452592|15625|32727|170314|1054|28641|45134|1247619|5708|34095|3|1|10|4|148|75|28|28.67|73.68|26.52|1320.48|742.56|802.76|2063.04|14.85|0.00|267.96|742.56|757.41|1010.52|1025.37|-60.20| +2452497|48409|2452617|15531|32727|170314|1054|28641|45134|1247619|5708|34095|45|9|17|4|41|75|1|69.82|203.87|112.12|91.75|112.12|69.82|203.87|4.48|0.00|16.30|112.12|116.60|128.42|132.90|42.30| +2452497|48409|2452609|9804|32727|170314|1054|28641|45134|1247619|5708|34095|19|6|16|1|79|75|66|45.59|85.70|40.27|2998.38|2657.82|3008.94|5656.20|79.73|0.00|2036.10|2657.82|2737.55|4693.92|4773.65|-351.12| +2452497|48409|2452523|4812|32727|170314|1054|28641|45134|1247619|5708|34095|45|1|16|4|2|75|69|94.01|175.79|82.62|6428.73|5700.78|6486.69|12129.51|228.03|0.00|2304.60|5700.78|5928.81|8005.38|8233.41|-785.91| +2452497|48409|2452507|8593|32727|170314|1054|28641|45134|1247619|5708|34095|51|21|16|1|259|75|63|73.12|131.61|92.12|2487.87|5803.56|4606.56|8291.43|174.10|0.00|0.00|5803.56|5977.66|5803.56|5977.66|1197.00| +2452497|48409|2452556|7383|32727|170314|1054|28641|45134|1247619|5708|34095|60|15|19|5|198|75|5|41.65|61.64|31.43|151.05|157.15|208.25|308.20|1.57|0.00|86.25|157.15|158.72|243.40|244.97|-51.10| +2452497|48409|2452535|1533|32727|170314|1054|28641|45134|1247619|5708|34095|57|15|14|3|42|75|84|33.23|79.41|76.23|267.12|6403.32|2791.32|6670.44|192.09|0.00|1600.20|6403.32|6595.41|8003.52|8195.61|3612.00| +2452497|48409|2452513|5143|32727|170314|1054|28641|45134|1247619|5708|34095|55|9|5|2|139|75|85|67.81|85.44|78.60|581.40|6681.00|5763.85|7262.40|0.00|2872.83|144.50|3808.17|3808.17|3952.67|3952.67|-1955.68| +2452497|48409|2452548|9108|32727|170314|1054|28641|45134|1247619|5708|34095|25|24|11|3|272|75|84|17.84|28.72|26.42|193.20|2219.28|1498.56|2412.48|177.54|0.00|1157.52|2219.28|2396.82|3376.80|3554.34|720.72| +2451713|63167|2451791|2120|39792|761829|2619|44256|64598|42136|273|11110|14|7|10|1|22|76|27|2.88|8.29|5.63|71.82|152.01|77.76|223.83|9.12|0.00|89.37|152.01|161.13|241.38|250.50|74.25| +2451713|63167|2451768|10111|39792|761829|2619|44256|64598|42136|273|11110|25|17|15|1|154|76|47|45.92|56.02|44.81|526.87|2106.07|2158.24|2632.94|42.12|0.00|894.88|2106.07|2148.19|3000.95|3043.07|-52.17| +2451713|63167|2451769|17725|39792|761829|2619|44256|64598|42136|273|11110|31|23|8|4|120|76|3|95.48|220.55|134.53|258.06|403.59|286.44|661.65|20.17|0.00|165.39|403.59|423.76|568.98|589.15|117.15| +2451713|63167|2451736|10034|39792|761829|2619|44256|64598|42136|273|11110|44|2|11|5|239|76|95|44.80|51.07|24.51|2523.20|2328.45|4256.00|4851.65|23.28|0.00|2036.80|2328.45|2351.73|4365.25|4388.53|-1927.55| +2451713|63167|2451772|7556|39792|761829|2619|44256|64598|42136|273|11110|53|23|14|4|11|76|88|58.87|148.94|1.48|12976.48|130.24|5180.56|13106.72|6.51|0.00|5242.16|130.24|136.75|5372.40|5378.91|-5050.32| +2451713|63167|2451728|9931|39792|761829|2619|44256|64598|42136|273|11110|32|20|8|2|9|76|35|41.56|44.88|16.15|1005.55|565.25|1454.60|1570.80|39.56|0.00|251.30|565.25|604.81|816.55|856.11|-889.35| +2451713|63167|2451785|3104|39792|761829|2619|44256|64598|42136|273|11110|26|20|8|3|295|76|6|65.00|68.25|51.87|98.28|311.22|390.00|409.50|12.44|0.00|114.66|311.22|323.66|425.88|438.32|-78.78| +2451713|63167|2451804|10039|39792|761829|2619|44256|64598|42136|273|11110|19|13|7|4|51|76|85|80.56|170.78|35.86|11468.20|3048.10|6847.60|14516.30|91.44|0.00|1741.65|3048.10|3139.54|4789.75|4881.19|-3799.50| +2451415|58912|2451496|5114|21966|1889753|3140|18927|22618|1818536|4688|14875|1|14|5|2|59|77|81|23.54|27.54|1.10|2141.64|89.10|1906.74|2230.74|2.67|0.00|780.03|89.10|91.77|869.13|871.80|-1817.64| +2451415|58912|2451489|11374|21966|1889753|3140|18927|22618|1818536|4688|14875|34|28|15|5|195|77|99|10.71|17.99|2.15|1568.16|212.85|1060.29|1781.01|14.89|0.00|462.33|212.85|227.74|675.18|690.07|-847.44| +2451415|58912|2451485|14284|21966|1889753|3140|18927|22618|1818536|4688|14875|43|20|10|2|151|77|19|39.97|52.76|3.16|942.40|60.04|759.43|1002.44|1.80|0.00|240.54|60.04|61.84|300.58|302.38|-699.39| +2451415|58912|2451524|7711|21966|1889753|3140|18927|22618|1818536|4688|14875|46|13|3|1|272|77|86|29.63|37.63|36.50|97.18|3139.00|2548.18|3236.18|251.12|0.00|420.54|3139.00|3390.12|3559.54|3810.66|590.82| +2451415|58912|2451535|8974|21966|1889753|3140|18927|22618|1818536|4688|14875|14|7|3|4|158|77|22|6.18|18.10|10.31|171.38|226.82|135.96|398.20|13.60|0.00|55.66|226.82|240.42|282.48|296.08|90.86| +2451415|58912|2451479|13370|21966|1889753|3140|18927|22618|1818536|4688|14875|13|20|5|4|293|77|44|96.31|276.40|38.69|10459.24|1702.36|4237.64|12161.60|85.11|0.00|4986.08|1702.36|1787.47|6688.44|6773.55|-2535.28| +2451415|58912|2451490|13900|21966|1889753|3140|18927|22618|1818536|4688|14875|32|1|12|1|87|77|45|2.78|3.36|0.70|119.70|31.50|125.10|151.20|1.89|0.00|10.35|31.50|33.39|41.85|43.74|-93.60| +2451415|58912|2451509|2014|21966|1889753|3140|18927|22618|1818536|4688|14875|46|2|10|5|213|77|11|71.24|213.00|176.79|398.31|1944.69|783.64|2343.00|155.57|0.00|984.06|1944.69|2100.26|2928.75|3084.32|1161.05| +2451415|58912|2451531|16210|21966|1889753|3140|18927|22618|1818536|4688|14875|37|19|1|2|216|77|12|99.79|237.50|232.75|57.00|2793.00|1197.48|2850.00|195.51|0.00|826.44|2793.00|2988.51|3619.44|3814.95|1595.52| +2451415|58912|2451437|16700|21966|1889753|3140|18927|22618|1818536|4688|14875|31|14|20|4|111|77|39|3.73|5.52|3.80|67.08|148.20|145.47|215.28|2.96|0.00|88.14|148.20|151.16|236.34|239.30|2.73| +2451415|58912|2451454|4490|21966|1889753|3140|18927|22618|1818536|4688|14875|7|10|18|2|259|77|89|30.01|33.01|29.04|353.33|2584.56|2670.89|2937.89|25.84|0.00|646.14|2584.56|2610.40|3230.70|3256.54|-86.33| +2451415|58912|2451478|13081|21966|1889753|3140|18927|22618|1818536|4688|14875|22|4|14|3|239|77|76|78.34|219.35|24.12|14837.48|1833.12|5953.84|16670.60|54.99|0.00|2666.84|1833.12|1888.11|4499.96|4554.95|-4120.72| +2451415|58912|2451434|14827|21966|1889753|3140|18927|22618|1818536|4688|14875|28|25|1|4|37|77|82|25.18|33.99|16.65|1421.88|1365.30|2064.76|2787.18|95.57|0.00|1281.66|1365.30|1460.87|2646.96|2742.53|-699.46| +2451415|58912|2451433|6271|21966|1889753|3140|18927|22618|1818536|4688|14875|44|2|8|2|133|77|37|71.18|170.12|134.39|1322.01|4972.43|2633.66|6294.44|99.44|0.00|2580.38|4972.43|5071.87|7552.81|7652.25|2338.77| +2451415|58912|2451432|8536|21966|1889753|3140|18927|22618|1818536|4688|14875|37|1|13|3|60|77|82|62.59|78.23|63.36|1219.34|5195.52|5132.38|6414.86|0.00|0.00|833.12|5195.52|5195.52|6028.64|6028.64|63.14| +2452521|43762|2452588|14743|45554|22294|3481|21421|54274|489532|4135|45322|31|6|7|1|225|78|79|64.22|151.55|25.76|9937.41|2035.04|5073.38|11972.45|142.45|0.00|4668.90|2035.04|2177.49|6703.94|6846.39|-3038.34| +2452521|43762|2452533|13944|45554|22294|3481|21421|54274|489532|4135|45322|42|18|7|3|26|78|37|97.45|277.73|272.17|205.72|10070.29|3605.65|10276.01|402.81|0.00|2157.84|10070.29|10473.10|12228.13|12630.94|6464.64| +2452521|43762|2452572|12499|45554|22294|3481|21421|54274|489532|4135|45322|49|30|12|2|232|78|93|80.37|111.71|82.66|2701.65|7687.38|7474.41|10389.03|0.00|0.00|3843.69|7687.38|7687.38|11531.07|11531.07|212.97| +2452521|43762|2452581|14359|45554|22294|3481|21421|54274|489532|4135|45322|3|21|13|2|84|78|63|93.71|236.14|226.69|595.35|14281.47|5903.73|14876.82|856.88|0.00|4164.93|14281.47|15138.35|18446.40|19303.28|8377.74| +2452521|43762|2452523|7284|45554|22294|3481|21421|54274|489532|4135|45322|31|7|12|5|35|78|14|70.49|143.79|4.31|1952.72|60.34|986.86|2013.06|0.00|0.00|140.84|60.34|60.34|201.18|201.18|-926.52| +2452521|43762|2452561|17269|45554|22294|3481|21421|54274|489532|4135|45322|3|1|13|2|171|78|22|58.15|126.18|105.99|444.18|2331.78|1279.30|2775.96|93.27|0.00|860.42|2331.78|2425.05|3192.20|3285.47|1052.48| +2452521|43762|2452538|1917|45554|22294|3481|21421|54274|489532|4135|45322|18|1|10|2|141|78|100|43.66|55.44|40.47|1497.00|4047.00|4366.00|5544.00|21.85|2954.31|1275.00|1092.69|1114.54|2367.69|2389.54|-3273.31| +2452521|43762|2452564|1554|45554|22294|3481|21421|54274|489532|4135|45322|48|30|19|3|4|78|69|11.85|31.40|26.37|347.07|1819.53|817.65|2166.60|37.84|873.37|563.04|946.16|984.00|1509.20|1547.04|128.51| +2452521|43762|2452597|11617|45554|22294|3481|21421|54274|489532|4135|45322|21|1|18|3|8|78|22|52.48|94.46|23.61|1558.70|519.42|1154.56|2078.12|10.38|0.00|124.52|519.42|529.80|643.94|654.32|-635.14| +2452521|43762|2452617|4495|45554|22294|3481|21421|54274|489532|4135|45322|1|19|2|1|203|78|72|72.49|138.45|34.61|7476.48|2491.92|5219.28|9968.40|0.00|0.00|2391.84|2491.92|2491.92|4883.76|4883.76|-2727.36| +2452521|43762|2452534|10669|45554|22294|3481|21421|54274|489532|4135|45322|30|24|9|4|236|78|77|40.98|86.05|63.67|1723.26|4902.59|3155.46|6625.85|392.20|0.00|662.20|4902.59|5294.79|5564.79|5956.99|1747.13| +2452521|43762|2452526|15780|45554|22294|3481|21421|54274|489532|4135|45322|21|13|3|2|288|78|83|64.71|110.00|38.50|5934.50|3195.50|5370.93|9130.00|63.91|0.00|2647.70|3195.50|3259.41|5843.20|5907.11|-2175.43| +2452521|43762|2452580|17544|45554|22294|3481|21421|54274|489532|4135|45322|54|25|2|3|227|78|26|66.54|135.07|109.40|667.42|2844.40|1730.04|3511.82|113.77|0.00|1545.18|2844.40|2958.17|4389.58|4503.35|1114.36| +2451535|77199|2451618|17995|81942|173188|4787|46833|81942|173188|4787|46833|58|28|10|5|270|79|72|67.23|201.01|116.58|6078.96|8393.76|4840.56|14472.72|587.56|0.00|6078.24|8393.76|8981.32|14472.00|15059.56|3553.20| +2451535|77199|2451581|2270|81942|173188|4787|46833|81942|173188|4787|46833|34|20|11|4|184|79|80|31.13|75.33|33.89|3315.20|2711.20|2490.40|6026.40|46.36|2196.07|360.80|515.13|561.49|875.93|922.29|-1975.27| +2451535|77199|2451624|1574|81942|173188|4787|46833|81942|173188|4787|46833|55|4|8|1|131|79|24|23.39|68.76|24.06|1072.80|577.44|561.36|1650.24|34.64|0.00|330.00|577.44|612.08|907.44|942.08|16.08| +2451535|77199|2451590|8932|81942|173188|4787|46833|81942|173188|4787|46833|7|1|11|3|195|79|21|31.41|78.21|43.79|722.82|919.59|659.61|1642.41|18.39|0.00|738.99|919.59|937.98|1658.58|1676.97|259.98| +2451535|77199|2451541|6610|81942|173188|4787|46833|81942|173188|4787|46833|22|20|20|4|188|79|15|25.21|41.59|28.69|193.50|430.35|378.15|623.85|0.00|0.00|162.15|430.35|430.35|592.50|592.50|52.20| +2451535|77199|2451609|7684|81942|173188|4787|46833|81942|173188|4787|46833|26|20|18|5|62|79|35|36.43|42.25|27.04|532.35|946.40|1275.05|1478.75|10.22|690.87|428.75|255.53|265.75|684.28|694.50|-1019.52| +2451535|77199|2451592|2456|81942|173188|4787|46833|81942|173188|4787|46833|55|14|18|3|195|79|10|28.97|57.36|0.57|567.90|5.70|289.70|573.60|0.28|0.00|57.30|5.70|5.98|63.00|63.28|-284.00| +2451535|77199|2451562|16660|81942|173188|4787|46833|81942|173188|4787|46833|1|13|16|5|214|79|70|12.83|22.45|20.87|110.60|1460.90|898.10|1571.50|14.60|0.00|314.30|1460.90|1475.50|1775.20|1789.80|562.80| +2451535|77199|2451619|1046|81942|173188|4787|46833|81942|173188|4787|46833|58|8|10|5|193|79|56|35.44|81.51|51.35|1688.96|2875.60|1984.64|4564.56|172.53|0.00|365.12|2875.60|3048.13|3240.72|3413.25|890.96| +2451535|77199|2451619|15106|81942|173188|4787|46833|81942|173188|4787|46833|1|14|18|3|38|79|91|42.84|80.53|33.82|4250.61|3077.62|3898.44|7328.23|75.70|553.97|2197.65|2523.65|2599.35|4721.30|4797.00|-1374.79| +2451535|77199|2451627|5564|81942|173188|4787|46833|81942|173188|4787|46833|14|25|11|1|111|79|74|12.01|32.42|7.45|1847.78|551.30|888.74|2399.08|19.29|165.39|695.60|385.91|405.20|1081.51|1100.80|-502.83| +2451535|77199|2451649|6667|81942|173188|4787|46833|81942|173188|4787|46833|16|19|8|4|57|79|79|42.16|86.00|4.30|6454.30|339.70|3330.64|6794.00|20.38|0.00|2649.66|339.70|360.08|2989.36|3009.74|-2990.94| +2451535|77199|2451647|12658|81942|173188|4787|46833|81942|173188|4787|46833|8|19|5|5|55|79|35|20.50|38.33|18.39|697.90|643.65|717.50|1341.55|10.04|476.30|80.15|167.35|177.39|247.50|257.54|-550.15| +2451535|77199|2451546|6452|81942|173188|4787|46833|81942|173188|4787|46833|28|14|1|4|77|79|55|50.17|110.37|86.08|1335.95|4734.40|2759.35|6070.35|426.09|0.00|1881.55|4734.40|5160.49|6615.95|7042.04|1975.05| +2450945|18919|2451064|10981|35692|452928|1811|48647|19010|1843359|2849|21685|37|28|11|2|123|80|73|81.35|159.44|28.69|9544.75|2094.37|5938.55|11639.12|18.63|230.38|3491.59|1863.99|1882.62|5355.58|5374.21|-4074.56| +2450945|18919|2450967|3040|35692|452928|1811|48647|19010|1843359|2849|21685|44|10|6|4|232|80|96|87.24|199.77|75.91|11890.56|7287.36|8375.04|19177.92|364.36|0.00|8246.40|7287.36|7651.72|15533.76|15898.12|-1087.68| +2450945|18919|2451006|16549|35692|452928|1811|48647|19010|1843359|2849|21685|1|25|5|2|229|80|18|90.24|158.82|115.93|772.02|2086.74|1624.32|2858.76|83.46|0.00|514.44|2086.74|2170.20|2601.18|2684.64|462.42| +2450945|18919|2450977|16306|35692|452928|1811|48647|19010|1843359|2849|21685|40|25|8|5|72|80|25|87.00|94.83|62.58|806.25|1564.50|2175.00|2370.75|62.58|0.00|237.00|1564.50|1627.08|1801.50|1864.08|-610.50| +2450945|18919|2451004|7606|35692|452928|1811|48647|19010|1843359|2849|21685|14|2|6|3|127|80|79|31.76|58.43|32.72|2031.09|2584.88|2509.04|4615.97|180.94|0.00|1107.58|2584.88|2765.82|3692.46|3873.40|75.84| +2450945|18919|2451060|17500|35692|452928|1811|48647|19010|1843359|2849|21685|50|7|15|1|194|80|90|11.50|29.09|11.34|1597.50|1020.60|1035.00|2618.10|81.64|0.00|314.10|1020.60|1102.24|1334.70|1416.34|-14.40| +2450945|18919|2450982|8419|35692|452928|1811|48647|19010|1843359|2849|21685|55|25|18|3|215|80|30|91.66|210.81|8.43|6071.40|252.90|2749.80|6324.30|10.11|0.00|2466.30|252.90|263.01|2719.20|2729.31|-2496.90| +2450945|18919|2451001|11504|35692|452928|1811|48647|19010|1843359|2849|21685|32|14|9|5|258|80|3|41.78|98.18|52.03|138.45|156.09|125.34|294.54|2.34|39.02|55.95|117.07|119.41|173.02|175.36|-8.27| +2450945|18919|2450966|8362|35692|452928|1811|48647|19010|1843359|2849|21685|37|13|5|5|128|80|76|1.01|1.23|0.25|74.48|19.00|76.76|93.48|0.76|0.00|46.36|19.00|19.76|65.36|66.12|-57.76| +2450945|18919|2450973|17227|35692|452928|1811|48647|19010|1843359|2849|21685|58|28|13|3|203|80|14|13.25|34.45|16.88|245.98|236.32|185.50|482.30|18.90|0.00|67.48|236.32|255.22|303.80|322.70|50.82| +2451499|69985|2451532|14104|99295|784813|5531|20202|23185|261525|5556|26274|1|1|1|3|218|81|55|48.37|92.38|9.23|4573.25|507.65|2660.35|5080.90|34.11|20.30|0.00|487.35|521.46|487.35|521.46|-2173.00| +2451499|69985|2451512|17002|99295|784813|5531|20202|23185|261525|5556|26274|7|14|5|2|126|81|100|76.39|176.46|155.28|2118.00|15528.00|7639.00|17646.00|621.12|0.00|2470.00|15528.00|16149.12|17998.00|18619.12|7889.00| +2451499|69985|2451529|4855|99295|784813|5531|20202|23185|261525|5556|26274|8|4|11|2|170|81|31|29.79|51.23|32.27|587.76|1000.37|923.49|1588.13|70.02|0.00|476.16|1000.37|1070.39|1476.53|1546.55|76.88| +2451499|69985|2451506|4369|99295|784813|5531|20202|23185|261525|5556|26274|4|7|18|4|60|81|59|88.07|226.33|43.00|10816.47|2537.00|5196.13|13353.47|0.00|0.00|2269.73|2537.00|2537.00|4806.73|4806.73|-2659.13| +2451499|69985|2451549|16040|99295|784813|5531|20202|23185|261525|5556|26274|34|4|18|5|115|81|36|42.03|82.37|7.41|2698.56|266.76|1513.08|2965.32|18.67|0.00|504.00|266.76|285.43|770.76|789.43|-1246.32| +2451499|69985|2451523|17599|99295|784813|5531|20202|23185|261525|5556|26274|50|14|20|5|277|81|7|48.19|143.60|63.18|562.94|442.26|337.33|1005.20|13.26|0.00|60.27|442.26|455.52|502.53|515.79|104.93| +2451499|69985|2451603|10174|99295|784813|5531|20202|23185|261525|5556|26274|19|2|14|1|151|81|56|67.76|84.02|21.84|3482.08|1223.04|3794.56|4705.12|97.84|0.00|1505.28|1223.04|1320.88|2728.32|2826.16|-2571.52| +2451499|69985|2451519|16180|99295|784813|5531|20202|23185|261525|5556|26274|44|28|17|4|111|81|8|32.81|95.80|45.98|398.56|367.84|262.48|766.40|22.07|0.00|383.20|367.84|389.91|751.04|773.11|105.36| +2451499|69985|2451547|10978|99295|784813|5531|20202|23185|261525|5556|26274|16|22|5|3|28|81|22|6.87|12.84|1.54|248.60|33.88|151.14|282.48|1.01|0.00|104.50|33.88|34.89|138.38|139.39|-117.26| +2451499|69985|2451588|10832|99295|784813|5531|20202|23185|261525|5556|26274|25|1|6|4|272|81|71|61.18|115.63|35.84|5665.09|2544.64|4343.78|8209.73|0.00|0.00|1231.14|2544.64|2544.64|3775.78|3775.78|-1799.14| +2451499|69985|2451505|10066|99295|784813|5531|20202|23185|261525|5556|26274|10|19|17|4|181|81|62|22.20|48.84|33.21|969.06|2059.02|1376.40|3028.08|0.00|0.00|1392.52|2059.02|2059.02|3451.54|3451.54|682.62| +2451499|69985|2451543|2539|99295|784813|5531|20202|23185|261525|5556|26274|55|13|6|5|34|81|18|28.74|57.76|46.78|197.64|842.04|517.32|1039.68|16.84|0.00|72.72|842.04|858.88|914.76|931.60|324.72| +2451499|69985|2451572|17101|99295|784813|5531|20202|23185|261525|5556|26274|37|16|12|2|182|81|71|32.15|86.16|18.95|4771.91|1345.45|2282.65|6117.36|107.63|0.00|672.37|1345.45|1453.08|2017.82|2125.45|-937.20| +2451499|69985|2451547|15265|99295|784813|5531|20202|23185|261525|5556|26274|14|19|12|4|129|81|97|82.95|181.66|167.12|1410.38|16210.64|8046.15|17621.02|1296.85|0.00|1584.98|16210.64|17507.49|17795.62|19092.47|8164.49| +2452050|48249|2452132|11503|55611|301065|119|31126|37013|61520|2288|9695|1|13|13|2|168|82|84|75.43|136.52|135.15|115.08|11352.60|6336.12|11467.68|234.99|8741.50|1031.52|2611.10|2846.09|3642.62|3877.61|-3725.02| +2452050|48249|2452147|5983|55611|301065|119|31126|37013|61520|2288|9695|35|11|17|3|39|82|75|88.75|118.03|86.16|2390.25|6462.00|6656.25|8852.25|323.10|0.00|3275.25|6462.00|6785.10|9737.25|10060.35|-194.25| +2452050|48249|2452136|4559|55611|301065|119|31126|37013|61520|2288|9695|29|17|10|1|155|82|62|79.36|161.10|16.11|8989.38|998.82|4920.32|9988.20|59.92|0.00|2396.92|998.82|1058.74|3395.74|3455.66|-3921.50| +2452050|48249|2452067|2393|55611|301065|119|31126|37013|61520|2288|9695|11|17|9|2|272|82|4|21.44|41.80|3.76|152.16|15.04|85.76|167.20|0.67|1.50|15.04|13.54|14.21|28.58|29.25|-72.22| +2452050|48249|2452132|10725|55611|301065|119|31126|37013|61520|2288|9695|51|1|16|1|136|82|55|29.56|54.09|48.68|297.55|2677.40|1625.80|2974.95|26.77|0.00|178.20|2677.40|2704.17|2855.60|2882.37|1051.60| +2452050|48249|2452161|14371|55611|301065|119|31126|37013|61520|2288|9695|55|3|9|1|233|82|71|11.34|30.16|25.33|342.93|1798.43|805.14|2141.36|143.87|0.00|770.35|1798.43|1942.30|2568.78|2712.65|993.29| +2452050|48249|2452156|2003|55611|301065|119|31126|37013|61520|2288|9695|13|13|7|4|44|82|87|70.32|82.27|76.51|501.12|6656.37|6117.84|7157.49|532.50|0.00|286.23|6656.37|7188.87|6942.60|7475.10|538.53| +2452050|48249|2452075|17851|55611|301065|119|31126|37013|61520|2288|9695|33|19|6|4|65|82|96|43.73|98.82|70.16|2751.36|6735.36|4198.08|9486.72|67.35|0.00|284.16|6735.36|6802.71|7019.52|7086.87|2537.28| +2452050|48249|2452085|6651|55611|301065|119|31126|37013|61520|2288|9695|17|9|1|2|83|82|21|17.57|18.97|8.91|211.26|187.11|368.97|398.37|3.74|0.00|159.18|187.11|190.85|346.29|350.03|-181.86| +2452050|48249|2452121|3715|55611|301065|119|31126|37013|61520|2288|9695|33|7|1|3|85|82|52|30.23|44.43|21.32|1201.72|1108.64|1571.96|2310.36|0.00|0.00|530.92|1108.64|1108.64|1639.56|1639.56|-463.32| +2452050|48249|2452079|13589|55611|301065|119|31126|37013|61520|2288|9695|3|5|7|3|299|82|61|50.15|117.85|56.56|3738.69|3450.16|3059.15|7188.85|310.51|0.00|1725.08|3450.16|3760.67|5175.24|5485.75|391.01| +2452339|12892|2452404|14904|2983|359033|683|37533|64757|1356958|4530|580|21|30|16|2|18|83|43|54.51|104.11|5.20|4253.13|223.60|2343.93|4476.73|0.00|0.00|402.48|223.60|223.60|626.08|626.08|-2120.33| +2452339|12892|2452342|15805|2983|359033|683|37533|64757|1356958|4530|580|1|1|11|3|98|83|25|34.43|40.62|9.74|772.00|243.50|860.75|1015.50|19.48|0.00|111.50|243.50|262.98|355.00|374.48|-617.25| +2452339|12892|2452430|12426|2983|359033|683|37533|64757|1356958|4530|580|36|7|17|3|269|83|54|56.58|92.22|9.22|4482.00|497.88|3055.32|4979.88|5.92|413.24|547.56|84.64|90.56|632.20|638.12|-2970.68| +2452339|12892|2452428|6852|2983|359033|683|37533|64757|1356958|4530|580|55|9|9|3|70|83|54|84.72|166.05|146.12|1076.22|7890.48|4574.88|8966.70|0.00|0.00|3945.24|7890.48|7890.48|11835.72|11835.72|3315.60| +2452339|12892|2452396|6093|2983|359033|683|37533|64757|1356958|4530|580|25|7|15|1|60|83|36|8.38|16.34|1.63|529.56|58.68|301.68|588.24|1.17|0.00|182.16|58.68|59.85|240.84|242.01|-243.00| +2452339|12892|2452450|4315|2983|359033|683|37533|64757|1356958|4530|580|42|18|20|3|26|83|73|18.35|40.18|29.33|792.05|2141.09|1339.55|2933.14|85.64|0.00|1289.91|2141.09|2226.73|3431.00|3516.64|801.54| +2452339|12892|2452391|5802|2983|359033|683|37533|64757|1356958|4530|580|7|1|5|4|186|83|97|49.55|111.98|6.71|10211.19|650.87|4806.35|10862.06|52.06|0.00|2280.47|650.87|702.93|2931.34|2983.40|-4155.48| +2452339|12892|2452422|9583|2983|359033|683|37533|64757|1356958|4530|580|1|15|2|4|216|83|46|85.54|205.29|192.97|566.72|8876.62|3934.84|9443.34|56.81|8166.49|2455.02|710.13|766.94|3165.15|3221.96|-3224.71| +2452339|12892|2452412|11775|2983|359033|683|37533|64757|1356958|4530|580|39|9|20|5|110|83|71|67.93|91.70|79.77|847.03|5663.67|4823.03|6510.70|0.00|0.00|1366.75|5663.67|5663.67|7030.42|7030.42|840.64| +2452339|12892|2452445|17749|2983|359033|683|37533|64757|1356958|4530|580|49|1|10|1|252|83|58|72.32|90.40|5.42|4928.84|314.36|4194.56|5243.20|0.00|216.90|733.70|97.46|97.46|831.16|831.16|-4097.10| +2452339|12892|2452411|1311|2983|359033|683|37533|64757|1356958|4530|580|43|15|17|3|289|83|82|86.28|258.84|10.35|20376.18|848.70|7074.96|21224.88|0.00|0.00|2546.92|848.70|848.70|3395.62|3395.62|-6226.26| +2452339|12892|2452374|10374|2983|359033|683|37533|64757|1356958|4530|580|60|24|17|4|191|83|68|47.61|111.40|83.55|1893.80|5681.40|3237.48|7575.20|454.51|0.00|757.52|5681.40|6135.91|6438.92|6893.43|2443.92| +2452339|12892|2452382|4753|2983|359033|683|37533|64757|1356958|4530|580|43|13|15|5|255|83|60|7.96|15.44|11.42|241.20|685.20|477.60|926.40|0.00|0.00|259.20|685.20|685.20|944.40|944.40|207.60| +2452339|12892|2452453|9564|2983|359033|683|37533|64757|1356958|4530|580|39|12|12|3|204|83|25|24.87|51.97|42.61|234.00|1065.25|621.75|1299.25|95.87|0.00|38.75|1065.25|1161.12|1104.00|1199.87|443.50| +2451121|45872|2451129|10498|36671|1428748|25|3702|22184|1122252|5580|10057|52|10|12|3|227|84|85|49.82|93.16|19.56|6256.00|1662.60|4234.70|7918.60|83.13|0.00|3959.30|1662.60|1745.73|5621.90|5705.03|-2572.10| +2451121|45872|2451233|13384|36671|1428748|25|3702|22184|1122252|5580|10057|22|20|20|4|218|84|50|24.08|58.51|18.13|2019.00|906.50|1204.00|2925.50|63.45|0.00|1023.50|906.50|969.95|1930.00|1993.45|-297.50| +2451121|45872|2451239|5200|36671|1428748|25|3702|22184|1122252|5580|10057|1|2|16|2|118|84|31|90.76|258.66|82.77|5452.59|2565.87|2813.56|8018.46|101.60|25.65|2405.29|2540.22|2641.82|4945.51|5047.11|-273.34| +2451121|45872|2451137|5443|36671|1428748|25|3702|22184|1122252|5580|10057|19|8|10|2|117|84|24|40.12|52.95|48.18|114.48|1156.32|962.88|1270.80|46.25|0.00|152.40|1156.32|1202.57|1308.72|1354.97|193.44| +2451121|45872|2451240|17638|36671|1428748|25|3702|22184|1122252|5580|10057|40|13|20|1|129|84|60|30.58|78.28|33.66|2677.20|2019.60|1834.80|4696.80|161.56|0.00|281.40|2019.60|2181.16|2301.00|2462.56|184.80| +2451121|45872|2451122|8791|36671|1428748|25|3702|22184|1122252|5580|10057|14|22|19|1|93|84|94|55.83|122.26|61.13|5746.22|5746.22|5248.02|11492.44|459.69|0.00|1493.66|5746.22|6205.91|7239.88|7699.57|498.20| +2451121|45872|2451163|13981|36671|1428748|25|3702|22184|1122252|5580|10057|38|28|10|2|277|84|98|29.69|35.33|25.08|1004.50|2457.84|2909.62|3462.34|49.15|0.00|830.06|2457.84|2506.99|3287.90|3337.05|-451.78| +2451121|45872|2451188|16586|36671|1428748|25|3702|22184|1122252|5580|10057|55|20|7|3|156|84|37|17.19|27.84|14.75|484.33|545.75|636.03|1030.08|5.45|0.00|185.37|545.75|551.20|731.12|736.57|-90.28| +2451121|45872|2451234|10358|36671|1428748|25|3702|22184|1122252|5580|10057|50|20|3|5|17|84|93|31.26|34.07|32.02|190.65|2977.86|2907.18|3168.51|59.55|0.00|728.19|2977.86|3037.41|3706.05|3765.60|70.68| +2451121|45872|2451234|4436|36671|1428748|25|3702|22184|1122252|5580|10057|2|19|5|3|78|84|61|10.28|28.98|8.40|1255.38|512.40|627.08|1767.78|25.82|143.47|300.12|368.93|394.75|669.05|694.87|-258.15| +2451121|45872|2451234|3698|36671|1428748|25|3702|22184|1122252|5580|10057|20|26|20|1|249|84|29|26.12|39.18|12.14|784.16|352.06|757.48|1136.22|17.60|0.00|420.21|352.06|369.66|772.27|789.87|-405.42| +2452497|65461|2452533|3763|96461|1344599|737|5275|76678|1083332|6889|17027|27|25|15|4|266|85|27|54.20|115.98|92.78|626.40|2505.06|1463.40|3131.46|40.58|1828.69|406.89|676.37|716.95|1083.26|1123.84|-787.03| +2452497|65461|2452557|2407|96461|1344599|737|5275|76678|1083332|6889|17027|60|30|20|5|180|85|29|15.04|29.92|15.55|416.73|450.95|436.16|867.68|31.56|0.00|407.74|450.95|482.51|858.69|890.25|14.79| +2452497|65461|2452502|6225|96461|1344599|737|5275|76678|1083332|6889|17027|3|1|20|4|208|85|33|90.72|192.32|151.93|1332.87|5013.69|2993.76|6346.56|0.00|0.00|2855.82|5013.69|5013.69|7869.51|7869.51|2019.93| +2452497|65461|2452558|6987|96461|1344599|737|5275|76678|1083332|6889|17027|9|19|15|5|62|85|91|3.87|7.23|4.04|290.29|367.64|352.17|657.93|33.08|0.00|203.84|367.64|400.72|571.48|604.56|15.47| +2452497|65461|2452541|11436|96461|1344599|737|5275|76678|1083332|6889|17027|42|24|3|4|262|85|26|65.75|75.61|51.41|629.20|1336.66|1709.50|1965.86|80.19|0.00|963.04|1336.66|1416.85|2299.70|2379.89|-372.84| +2452497|65461|2452510|7419|96461|1344599|737|5275|76678|1083332|6889|17027|36|18|7|3|26|85|9|70.40|119.68|119.68|0.00|1077.12|633.60|1077.12|64.62|0.00|387.72|1077.12|1141.74|1464.84|1529.46|443.52| +2452497|65461|2452546|9234|96461|1344599|737|5275|76678|1083332|6889|17027|9|21|20|1|52|85|76|23.72|36.29|29.75|497.04|2261.00|1802.72|2758.04|158.27|0.00|1241.08|2261.00|2419.27|3502.08|3660.35|458.28| +2452497|65461|2452522|13327|96461|1344599|737|5275|76678|1083332|6889|17027|24|7|16|3|130|85|57|62.93|173.68|145.89|1584.03|8315.73|3587.01|9899.76|131.38|1746.30|4850.70|6569.43|6700.81|11420.13|11551.51|2982.42| +2452497|65461|2452565|5881|96461|1344599|737|5275|76678|1083332|6889|17027|25|21|9|5|208|85|35|85.68|231.33|106.41|4372.20|3724.35|2998.80|8096.55|37.24|0.00|1052.45|3724.35|3761.59|4776.80|4814.04|725.55| +2452497|65461|2452504|3465|96461|1344599|737|5275|76678|1083332|6889|17027|15|18|20|3|109|85|7|16.15|44.41|18.20|183.47|127.40|113.05|310.87|10.19|0.00|6.16|127.40|137.59|133.56|143.75|14.35| +2452497|65461|2452504|7290|96461|1344599|737|5275|76678|1083332|6889|17027|60|24|17|4|11|85|69|98.55|111.36|16.70|6531.54|1152.30|6799.95|7683.84|11.52|0.00|2151.42|1152.30|1163.82|3303.72|3315.24|-5647.65| +2452497|65461|2452534|9801|96461|1344599|737|5275|76678|1083332|6889|17027|13|19|13|1|19|85|58|78.73|187.37|67.45|6955.36|3912.10|4566.34|10867.46|195.60|0.00|3477.10|3912.10|4107.70|7389.20|7584.80|-654.24| +2452497|65461|2452507|7959|96461|1344599|737|5275|76678|1083332|6889|17027|21|7|10|1|149|85|45|35.10|74.06|46.65|1233.45|2099.25|1579.50|3332.70|0.00|1196.57|1133.10|902.68|902.68|2035.78|2035.78|-676.82| +2452497|65461|2452581|4401|96461|1344599|737|5275|76678|1083332|6889|17027|7|30|19|3|237|85|53|75.46|141.11|118.53|1196.74|6282.09|3999.38|7478.83|56.53|4397.46|3514.96|1884.63|1941.16|5399.59|5456.12|-2114.75| +2451408|62038|2451502|3448|21094|1129773|5299|42378|21094|1129773|5299|42378|55|14|8|5|285|86|25|72.25|137.99|91.07|1173.00|2276.75|1806.25|3449.75|85.37|569.18|1724.75|1707.57|1792.94|3432.32|3517.69|-98.68| +2451408|62038|2451491|15367|21094|1129773|5299|42378|21094|1129773|5299|42378|55|13|13|2|108|86|29|83.60|181.41|14.51|4840.10|420.79|2424.40|5260.89|37.87|0.00|1052.12|420.79|458.66|1472.91|1510.78|-2003.61| +2451408|62038|2451525|16273|21094|1129773|5299|42378|21094|1129773|5299|42378|52|16|1|3|15|86|98|16.66|17.65|3.53|1383.76|345.94|1632.68|1729.70|0.00|0.00|414.54|345.94|345.94|760.48|760.48|-1286.74| +2451408|62038|2451427|3241|21094|1129773|5299|42378|21094|1129773|5299|42378|58|2|4|1|248|86|4|79.84|216.36|205.54|43.28|822.16|319.36|865.44|49.32|0.00|86.52|822.16|871.48|908.68|958.00|502.80| +2451408|62038|2451502|8605|21094|1129773|5299|42378|21094|1129773|5299|42378|38|1|9|2|25|86|86|83.21|232.15|71.96|13776.34|6188.56|7156.06|19964.90|183.80|4146.33|6587.60|2042.23|2226.03|8629.83|8813.63|-5113.83| +2451408|62038|2451461|3092|21094|1129773|5299|42378|21094|1129773|5299|42378|43|22|12|5|185|86|41|80.96|235.59|179.04|2318.55|7340.64|3319.36|9659.19|293.62|0.00|4443.17|7340.64|7634.26|11783.81|12077.43|4021.28| +2451408|62038|2451443|8834|21094|1129773|5299|42378|21094|1129773|5299|42378|43|14|14|5|116|86|53|56.87|79.04|32.40|2471.92|1717.20|3014.11|4189.12|137.37|0.00|1633.46|1717.20|1854.57|3350.66|3488.03|-1296.91| +2451408|62038|2451466|17228|21094|1129773|5299|42378|21094|1129773|5299|42378|10|26|5|3|234|86|76|37.99|94.59|33.10|4673.24|2515.60|2887.24|7188.84|50.31|0.00|430.92|2515.60|2565.91|2946.52|2996.83|-371.64| +2451408|62038|2451477|9988|21094|1129773|5299|42378|21094|1129773|5299|42378|55|20|4|5|163|86|30|90.13|228.93|169.40|1785.90|5082.00|2703.90|6867.90|204.29|1677.06|3090.30|3404.94|3609.23|6495.24|6699.53|701.04| +2451408|62038|2451453|16633|21094|1129773|5299|42378|21094|1129773|5299|42378|40|14|15|2|239|86|92|29.54|76.50|35.95|3730.60|3307.40|2717.68|7038.00|297.66|0.00|1336.76|3307.40|3605.06|4644.16|4941.82|589.72| +2451408|62038|2451497|16057|21094|1129773|5299|42378|21094|1129773|5299|42378|1|4|2|4|293|86|33|76.59|229.77|156.24|2426.49|5155.92|2527.47|7582.41|257.79|0.00|1743.72|5155.92|5413.71|6899.64|7157.43|2628.45| +2451408|62038|2451437|13754|21094|1129773|5299|42378|21094|1129773|5299|42378|20|10|1|4|104|86|19|73.94|158.23|129.74|541.31|2465.06|1404.86|3006.37|98.60|0.00|691.41|2465.06|2563.66|3156.47|3255.07|1060.20| +2451408|62038|2451436|14476|21094|1129773|5299|42378|21094|1129773|5299|42378|13|13|20|5|30|86|71|5.85|10.53|10.10|30.53|717.10|415.35|747.63|50.19|0.00|246.37|717.10|767.29|963.47|1013.66|301.75| +2452050|17840|2452089|927|49208|175411|6414|25017|84577|1855388|1626|43076|57|19|11|2|283|87|46|42.65|46.48|42.29|192.74|1945.34|1961.90|2138.08|77.81|0.00|277.84|1945.34|2023.15|2223.18|2300.99|-16.56| +2452050|17840|2452120|8415|49208|175411|6414|25017|84577|1855388|1626|43076|25|27|15|1|257|87|11|43.07|80.11|13.61|731.50|149.71|473.77|881.21|0.00|0.00|255.53|149.71|149.71|405.24|405.24|-324.06| +2452050|17840|2452147|12727|49208|175411|6414|25017|84577|1855388|1626|43076|35|23|9|1|220|87|52|89.77|266.61|218.62|2495.48|11368.24|4668.04|13863.72|0.00|6366.21|6377.28|5002.03|5002.03|11379.31|11379.31|333.99| +2452050|17840|2452153|2453|49208|175411|6414|25017|84577|1855388|1626|43076|53|17|8|5|52|87|92|75.61|124.00|58.28|6046.24|5361.76|6956.12|11408.00|53.61|0.00|4791.36|5361.76|5415.37|10153.12|10206.73|-1594.36| +2452050|17840|2452125|11585|49208|175411|6414|25017|84577|1855388|1626|43076|29|5|11|4|177|87|16|87.55|229.38|135.33|1504.80|2165.28|1400.80|3670.08|108.26|0.00|403.68|2165.28|2273.54|2568.96|2677.22|764.48| +2452050|17840|2452097|15117|49208|175411|6414|25017|84577|1855388|1626|43076|29|29|15|2|142|87|61|91.13|236.93|144.52|5637.01|8815.72|5558.93|14452.73|88.15|0.00|4624.41|8815.72|8903.87|13440.13|13528.28|3256.79| +2452050|17840|2452154|7105|49208|175411|6414|25017|84577|1855388|1626|43076|31|7|5|3|44|87|78|86.00|253.70|81.18|13456.56|6332.04|6708.00|19788.60|379.92|0.00|1978.86|6332.04|6711.96|8310.90|8690.82|-375.96| +2452050|17840|2452138|3059|49208|175411|6414|25017|84577|1855388|1626|43076|25|5|6|1|19|87|30|42.68|84.93|23.78|1834.50|713.40|1280.40|2547.90|6.06|592.12|942.60|121.28|127.34|1063.88|1069.94|-1159.12| +2452050|17840|2452066|13933|49208|175411|6414|25017|84577|1855388|1626|43076|39|13|14|1|118|87|89|91.71|104.54|22.99|7257.95|2046.11|8162.19|9304.06|102.30|0.00|1953.55|2046.11|2148.41|3999.66|4101.96|-6116.08| +2452050|17840|2452074|13271|49208|175411|6414|25017|84577|1855388|1626|43076|23|3|19|4|46|87|63|72.87|197.47|195.49|124.74|12315.87|4590.81|12440.61|0.00|2955.80|1741.32|9360.07|9360.07|11101.39|11101.39|4769.26| +2452050|17840|2452072|3371|49208|175411|6414|25017|84577|1855388|1626|43076|5|3|11|2|96|87|11|94.40|221.84|179.69|463.65|1976.59|1038.40|2440.24|28.46|553.44|365.97|1423.15|1451.61|1789.12|1817.58|384.75| +2452050|17840|2452122|9725|49208|175411|6414|25017|84577|1855388|1626|43076|13|5|6|2|95|87|27|69.30|120.58|13.26|2897.64|358.02|1871.10|3255.66|21.48|0.00|911.52|358.02|379.50|1269.54|1291.02|-1513.08| +2452050|17840|2452103|745|49208|175411|6414|25017|84577|1855388|1626|43076|13|9|6|2|8|87|67|61.18|176.81|104.31|4857.50|6988.77|4099.06|11846.27|628.98|0.00|1065.97|6988.77|7617.75|8054.74|8683.72|2889.71| +2452050|17840|2452070|31|49208|175411|6414|25017|84577|1855388|1626|43076|13|3|7|2|197|87|53|65.33|163.32|97.99|3462.49|5193.47|3462.49|8655.96|207.73|0.00|2510.08|5193.47|5401.20|7703.55|7911.28|1730.98| +2451375|59503|2451468|9092|74942|1114698|854|36610|76788|822852|3078|18834|50|20|6|1|266|88|54|94.76|276.69|146.64|7022.70|7918.56|5117.04|14941.26|0.00|0.00|5378.40|7918.56|7918.56|13296.96|13296.96|2801.52| +2451375|59503|2451407|14851|74942|1114698|854|36610|76788|822852|3078|18834|34|8|19|3|167|88|4|21.28|35.32|27.54|31.12|110.16|85.12|141.28|8.81|0.00|66.40|110.16|118.97|176.56|185.37|25.04| +2451375|59503|2451400|373|74942|1114698|854|36610|76788|822852|3078|18834|4|7|5|3|164|88|89|6.60|10.95|6.35|409.40|565.15|587.40|974.55|45.21|0.00|136.17|565.15|610.36|701.32|746.53|-22.25| +2451375|59503|2451401|6680|74942|1114698|854|36610|76788|822852|3078|18834|49|10|17|1|92|88|1|93.73|269.00|166.78|102.22|166.78|93.73|269.00|6.67|0.00|26.90|166.78|173.45|193.68|200.35|73.05| +2451375|59503|2451418|16792|74942|1114698|854|36610|76788|822852|3078|18834|20|2|3|1|222|88|93|42.11|82.53|48.69|3147.12|4528.17|3916.23|7675.29|181.12|0.00|3837.18|4528.17|4709.29|8365.35|8546.47|611.94| +2451375|59503|2451395|12230|74942|1114698|854|36610|76788|822852|3078|18834|55|26|7|3|39|88|3|38.33|110.00|23.10|260.70|69.30|114.99|330.00|2.07|0.00|102.30|69.30|71.37|171.60|173.67|-45.69| +2451375|59503|2451485|7939|74942|1114698|854|36610|76788|822852|3078|18834|32|13|4|4|284|88|71|17.52|26.63|5.59|1493.84|396.89|1243.92|1890.73|35.72|0.00|510.49|396.89|432.61|907.38|943.10|-847.03| +2451375|59503|2451462|692|74942|1114698|854|36610|76788|822852|3078|18834|55|16|3|2|51|88|6|62.28|66.63|18.65|287.88|111.90|373.68|399.78|5.59|0.00|119.88|111.90|117.49|231.78|237.37|-261.78| +2451375|59503|2451490|16327|74942|1114698|854|36610|76788|822852|3078|18834|13|19|20|3|93|88|66|40.39|64.22|37.88|1738.44|2500.08|2665.74|4238.52|150.00|0.00|1143.78|2500.08|2650.08|3643.86|3793.86|-165.66| +2451375|59503|2451459|871|74942|1114698|854|36610|76788|822852|3078|18834|14|8|2|4|93|88|56|24.16|55.08|31.94|1295.84|1788.64|1352.96|3084.48|16.63|125.20|554.96|1663.44|1680.07|2218.40|2235.03|310.48| +2451375|59503|2451457|9541|74942|1114698|854|36610|76788|822852|3078|18834|22|8|18|1|44|88|82|62.62|120.85|93.05|2279.60|7630.10|5134.84|9909.70|534.10|0.00|594.50|7630.10|8164.20|8224.60|8758.70|2495.26| +2451513|55518|2451541|17686|91083|39218|4953|48288|47292|749108|5493|41228|44|26|18|4|123|89|47|90.20|207.46|188.78|877.96|8872.66|4239.40|9750.62|377.97|2573.07|4582.50|6299.59|6677.56|10882.09|11260.06|2060.19| +2451513|55518|2451530|9146|91083|39218|4953|48288|47292|749108|5493|41228|1|7|7|5|193|89|57|59.85|96.95|22.29|4255.62|1270.53|3411.45|5526.15|76.23|0.00|2265.18|1270.53|1346.76|3535.71|3611.94|-2140.92| +2451513|55518|2451523|2116|91083|39218|4953|48288|47292|749108|5493|41228|32|16|4|4|114|89|11|3.04|7.90|4.10|41.80|45.10|33.44|86.90|2.70|0.00|41.69|45.10|47.80|86.79|89.49|11.66| +2451513|55518|2451515|6578|91083|39218|4953|48288|47292|749108|5493|41228|58|13|17|2|284|89|55|29.43|36.78|3.31|1840.85|182.05|1618.65|2022.90|1.82|0.00|485.10|182.05|183.87|667.15|668.97|-1436.60| +2451513|55518|2451577|14935|91083|39218|4953|48288|47292|749108|5493|41228|34|13|9|1|163|89|16|32.31|73.98|71.02|47.36|1136.32|516.96|1183.68|79.54|0.00|165.60|1136.32|1215.86|1301.92|1381.46|619.36| +2451513|55518|2451570|3484|91083|39218|4953|48288|47292|749108|5493|41228|16|22|2|5|65|89|59|93.42|184.03|90.17|5537.74|5320.03|5511.78|10857.77|266.00|0.00|4017.31|5320.03|5586.03|9337.34|9603.34|-191.75| +2451513|55518|2451624|2138|91083|39218|4953|48288|47292|749108|5493|41228|10|19|1|5|216|89|26|19.61|20.98|19.51|38.22|507.26|509.86|545.48|0.00|0.00|223.60|507.26|507.26|730.86|730.86|-2.60| +2451513|55518|2451597|14116|91083|39218|4953|48288|47292|749108|5493|41228|19|4|3|3|153|89|46|77.46|182.03|40.04|6531.54|1841.84|3563.16|8373.38|55.25|0.00|2176.72|1841.84|1897.09|4018.56|4073.81|-1721.32| +2451513|55518|2451599|2749|91083|39218|4953|48288|47292|749108|5493|41228|50|8|14|3|62|89|53|94.41|276.62|47.02|12168.80|2492.06|5003.73|14660.86|224.28|0.00|5717.64|2492.06|2716.34|8209.70|8433.98|-2511.67| +2451513|55518|2451592|5080|91083|39218|4953|48288|47292|749108|5493|41228|58|22|7|2|154|89|62|69.83|83.79|3.35|4987.28|207.70|4329.46|5194.98|16.61|0.00|0.00|207.70|224.31|207.70|224.31|-4121.76| +2451513|55518|2451593|6814|91083|39218|4953|48288|47292|749108|5493|41228|55|4|11|1|274|89|87|69.06|158.14|23.72|11694.54|2063.64|6008.22|13758.18|5.57|2001.73|6328.38|61.91|67.48|6390.29|6395.86|-5946.31| +2452274|6180|2452357|1873|87541|1076044|6500|12956|437|1493380|1124|1980|15|29|9|1|50|90|63|53.88|120.15|72.09|3027.78|4541.67|3394.44|7569.45|181.66|0.00|3708.81|4541.67|4723.33|8250.48|8432.14|1147.23| +2452274|6180|2452339|8519|87541|1076044|6500|12956|437|1493380|1124|1980|55|7|3|4|255|90|14|96.62|185.51|87.18|1376.62|1220.52|1352.68|2597.14|36.61|0.00|519.40|1220.52|1257.13|1739.92|1776.53|-132.16| +2452274|6180|2452391|7679|87541|1076044|6500|12956|437|1493380|1124|1980|33|7|1|3|205|90|70|76.07|126.27|1.26|8750.70|88.20|5324.90|8838.90|1.76|0.00|795.20|88.20|89.96|883.40|885.16|-5236.70| +2452274|6180|2452311|1597|87541|1076044|6500|12956|437|1493380|1124|1980|55|13|6|5|124|90|48|97.38|120.75|109.88|521.76|5274.24|4674.24|5796.00|0.00|0.00|2782.08|5274.24|5274.24|8056.32|8056.32|600.00| +2452274|6180|2452318|16025|87541|1076044|6500|12956|437|1493380|1124|1980|1|3|3|4|222|90|7|83.36|240.91|197.54|303.59|1382.78|583.52|1686.37|69.13|0.00|522.76|1382.78|1451.91|1905.54|1974.67|799.26| +2452274|6180|2452290|8603|87541|1076044|6500|12956|437|1493380|1124|1980|53|29|16|4|55|90|43|21.04|49.65|33.76|683.27|1451.68|904.72|2134.95|58.06|0.00|191.78|1451.68|1509.74|1643.46|1701.52|546.96| +2452274|6180|2452360|7791|87541|1076044|6500|12956|437|1493380|1124|1980|55|23|3|4|278|90|98|82.93|208.98|73.14|13312.32|7167.72|8127.14|20480.04|116.11|3297.15|5733.98|3870.57|3986.68|9604.55|9720.66|-4256.57| +2452274|6180|2452329|14505|87541|1076044|6500|12956|437|1493380|1124|1980|37|5|20|2|204|90|39|7.66|13.09|1.83|439.14|71.37|298.74|510.51|3.56|0.00|45.63|71.37|74.93|117.00|120.56|-227.37| +2452274|6180|2452368|4713|87541|1076044|6500|12956|437|1493380|1124|1980|57|25|4|3|62|90|7|17.68|33.76|33.42|2.38|233.94|123.76|236.32|0.00|0.00|75.60|233.94|233.94|309.54|309.54|110.18| +2452274|6180|2452356|1083|87541|1076044|6500|12956|437|1493380|1124|1980|3|11|17|1|145|90|73|26.38|36.40|15.65|1514.75|1142.45|1925.74|2657.20|62.37|102.82|690.58|1039.63|1102.00|1730.21|1792.58|-886.11| +2452274|6180|2452323|9071|87541|1076044|6500|12956|437|1493380|1124|1980|45|7|2|3|164|90|36|11.07|13.39|2.81|380.88|101.16|398.52|482.04|1.70|79.91|0.00|21.25|22.95|21.25|22.95|-377.27| +2452274|6180|2452315|1277|87541|1076044|6500|12956|437|1493380|1124|1980|47|21|11|3|145|90|42|78.09|102.29|81.83|859.32|3436.86|3279.78|4296.18|60.83|1409.11|1116.78|2027.75|2088.58|3144.53|3205.36|-1252.03| +2451398|35797|2451462|3367|90554|1506619|2684|23149|24539|640349|3297|32962|14|1|14|2|280|91|70|27.11|46.90|26.73|1411.90|1871.10|1897.70|3283.00|74.84|0.00|1181.60|1871.10|1945.94|3052.70|3127.54|-26.60| +2451398|35797|2451498|2122|90554|1506619|2684|23149|24539|640349|3297|32962|43|8|1|4|97|91|87|95.95|253.30|141.84|9697.02|12340.08|8347.65|22037.10|123.40|0.00|5728.95|12340.08|12463.48|18069.03|18192.43|3992.43| +2451398|35797|2451505|5426|90554|1506619|2684|23149|24539|640349|3297|32962|7|14|2|1|170|91|1|13.57|26.19|2.35|23.84|2.35|13.57|26.19|0.16|0.00|3.66|2.35|2.51|6.01|6.17|-11.22| +2451398|35797|2451435|7460|90554|1506619|2684|23149|24539|640349|3297|32962|31|13|8|4|123|91|39|92.49|94.33|66.03|1103.70|2575.17|3607.11|3678.87|231.76|0.00|36.66|2575.17|2806.93|2611.83|2843.59|-1031.94| +2451398|35797|2451478|8120|90554|1506619|2684|23149|24539|640349|3297|32962|25|13|1|5|109|91|80|35.51|100.49|42.20|4663.20|3376.00|2840.80|8039.20|135.04|0.00|2089.60|3376.00|3511.04|5465.60|5600.64|535.20| +2451398|35797|2451501|1906|90554|1506619|2684|23149|24539|640349|3297|32962|20|8|6|5|84|91|52|17.40|48.54|43.20|277.68|2246.40|904.80|2524.08|112.32|0.00|1034.80|2246.40|2358.72|3281.20|3393.52|1341.60| +2451398|35797|2451429|10436|90554|1506619|2684|23149|24539|640349|3297|32962|19|7|11|5|129|91|47|57.26|142.57|76.98|3082.73|3618.06|2691.22|6700.79|36.18|0.00|2278.09|3618.06|3654.24|5896.15|5932.33|926.84| +2451398|35797|2451509|5974|90554|1506619|2684|23149|24539|640349|3297|32962|10|19|1|5|269|91|75|3.81|6.47|5.17|97.50|387.75|285.75|485.25|2.94|314.07|130.50|73.68|76.62|204.18|207.12|-212.07| +2452210|33897|2452267|8255|73296|1855583|5070|8110|76380|1396321|524|48764|39|15|6|4|62|92|83|81.21|176.22|77.53|8191.27|6434.99|6740.43|14626.26|450.44|0.00|1169.47|6434.99|6885.43|7604.46|8054.90|-305.44| +2452210|33897|2452230|5221|73296|1855583|5070|8110|76380|1396321|524|48764|47|17|17|5|273|92|76|96.40|188.94|52.90|10339.04|4020.40|7326.40|14359.44|241.22|0.00|1292.00|4020.40|4261.62|5312.40|5553.62|-3306.00| +2452210|33897|2452273|7749|73296|1855583|5070|8110|76380|1396321|524|48764|23|29|18|1|77|92|38|16.92|36.20|3.62|1238.04|137.56|642.96|1375.60|4.12|0.00|632.70|137.56|141.68|770.26|774.38|-505.40| +2452210|33897|2452273|16029|73296|1855583|5070|8110|76380|1396321|524|48764|25|9|7|4|251|92|42|87.38|187.86|24.42|6864.48|1025.64|3669.96|7890.12|30.76|0.00|3234.84|1025.64|1056.40|4260.48|4291.24|-2644.32| +2452210|33897|2452289|17041|73296|1855583|5070|8110|76380|1396321|524|48764|21|25|15|5|72|92|91|1.53|1.63|1.51|10.92|137.41|139.23|148.33|4.12|0.00|38.22|137.41|141.53|175.63|179.75|-1.82| +2452210|33897|2452304|3193|73296|1855583|5070|8110|76380|1396321|524|48764|43|11|17|1|272|92|9|50.87|108.35|31.42|692.37|282.78|457.83|975.15|5.65|0.00|458.28|282.78|288.43|741.06|746.71|-175.05| +2452210|33897|2452327|2151|73296|1855583|5070|8110|76380|1396321|524|48764|29|27|20|4|168|92|91|48.95|127.75|16.60|10114.65|1510.60|4454.45|11625.25|45.31|0.00|930.02|1510.60|1555.91|2440.62|2485.93|-2943.85| +2452210|33897|2452256|17951|73296|1855583|5070|8110|76380|1396321|524|48764|27|15|2|3|177|92|37|45.72|67.66|6.76|2253.30|250.12|1691.64|2503.42|20.00|0.00|1101.49|250.12|270.12|1351.61|1371.61|-1441.52| +2452210|33897|2452328|9885|73296|1855583|5070|8110|76380|1396321|524|48764|33|21|2|3|67|92|46|67.06|80.47|72.42|370.30|3331.32|3084.76|3701.62|113.93|2065.41|407.10|1265.91|1379.84|1673.01|1786.94|-1818.85| +2452601|71061|2452606|1957|11894|1711901|355|23562|70838|1051564|5630|44884|39|19|3|3|296|93|21|93.55|198.32|71.39|2665.53|1499.19|1964.55|4164.72|0.00|0.00|1041.18|1499.19|1499.19|2540.37|2540.37|-465.36| +2452601|71061|2452610|2727|11894|1711901|355|23562|70838|1051564|5630|44884|15|19|12|4|219|93|46|21.36|33.74|29.69|186.30|1365.74|982.56|1552.04|13.65|0.00|713.92|1365.74|1379.39|2079.66|2093.31|383.18| +2452601|71061|2452666|17401|11894|1711901|355|23562|70838|1051564|5630|44884|12|19|1|5|59|93|9|28.50|61.56|59.71|16.65|537.39|256.50|554.04|48.36|0.00|160.65|537.39|585.75|698.04|746.40|280.89| +2452601|71061|2452672|7489|11894|1711901|355|23562|70838|1051564|5630|44884|49|13|9|3|230|93|46|88.52|185.89|76.21|5045.28|3505.66|4071.92|8550.94|280.45|0.00|4104.12|3505.66|3786.11|7609.78|7890.23|-566.26| +2452601|71061|2452624|11707|11894|1711901|355|23562|70838|1051564|5630|44884|55|7|17|3|156|93|72|1.22|3.41|2.01|100.80|144.72|87.84|245.52|8.68|0.00|19.44|144.72|153.40|164.16|172.84|56.88| +2452601|71061|2452617|1410|11894|1711901|355|23562|70838|1051564|5630|44884|57|1|13|5|90|93|100|82.83|223.64|187.85|3579.00|18785.00|8283.00|22364.00|1690.65|0.00|6038.00|18785.00|20475.65|24823.00|26513.65|10502.00| +2452601|71061|2452607|9013|11894|1711901|355|23562|70838|1051564|5630|44884|33|19|3|3|146|93|30|4.82|11.03|4.30|201.90|129.00|144.60|330.90|0.46|105.78|138.90|23.22|23.68|162.12|162.58|-121.38| +2452601|71061|2452655|9663|11894|1711901|355|23562|70838|1051564|5630|44884|42|9|9|3|77|93|4|84.68|186.29|50.29|544.00|201.16|338.72|745.16|15.61|6.03|312.96|195.13|210.74|508.09|523.70|-143.59| +2452601|71061|2452659|6690|11894|1711901|355|23562|70838|1051564|5630|44884|45|19|17|1|188|93|9|24.71|44.97|39.12|52.65|352.08|222.39|404.73|0.00|0.00|89.01|352.08|352.08|441.09|441.09|129.69| +2452601|71061|2452679|753|11894|1711901|355|23562|70838|1051564|5630|44884|12|27|2|2|255|93|51|84.32|145.03|140.67|222.36|7174.17|4300.32|7396.53|215.22|0.00|2292.45|7174.17|7389.39|9466.62|9681.84|2873.85| +2452601|71061|2452699|5985|11894|1711901|355|23562|70838|1051564|5630|44884|15|3|6|5|44|93|73|96.45|236.30|134.69|7417.53|9832.37|7040.85|17249.90|884.91|0.00|5174.97|9832.37|10717.28|15007.34|15892.25|2791.52| +2452601|71061|2452665|3759|11894|1711901|355|23562|70838|1051564|5630|44884|30|3|9|1|150|93|55|2.20|2.70|0.29|132.55|15.95|121.00|148.50|0.47|0.00|38.50|15.95|16.42|54.45|54.92|-105.05| +2452601|71061|2452711|13681|11894|1711901|355|23562|70838|1051564|5630|44884|39|15|6|4|212|93|17|51.89|91.84|62.45|499.63|1061.65|882.13|1561.28|10.61|0.00|733.72|1061.65|1072.26|1795.37|1805.98|179.52| +2452601|71061|2452690|13597|11894|1711901|355|23562|70838|1051564|5630|44884|27|6|19|3|254|93|43|66.20|114.52|91.61|985.13|3939.23|2846.60|4924.36|236.35|0.00|1428.03|3939.23|4175.58|5367.26|5603.61|1092.63| +2452601|71061|2452669|7113|11894|1711901|355|23562|70838|1051564|5630|44884|51|9|7|4|288|93|93|99.73|268.27|131.45|12724.26|12224.85|9274.89|24949.11|122.24|0.00|11725.44|12224.85|12347.09|23950.29|24072.53|2949.96| +2451033|74433|2451050|625|89077|352774|5271|21768|60926|1006342|5659|15356|43|20|3|2|264|94|87|47.91|52.70|2.63|4356.09|228.81|4168.17|4584.90|6.86|0.00|182.70|228.81|235.67|411.51|418.37|-3939.36| +2451033|74433|2451146|4345|89077|352774|5271|21768|60926|1006342|5659|15356|20|1|1|3|38|94|90|20.37|56.42|21.43|3149.10|1928.70|1833.30|5077.80|57.86|0.00|1269.00|1928.70|1986.56|3197.70|3255.56|95.40| +2451033|74433|2451129|4723|89077|352774|5271|21768|60926|1006342|5659|15356|46|2|20|5|63|94|4|75.98|103.33|43.39|239.76|173.56|303.92|413.32|12.14|0.00|136.36|173.56|185.70|309.92|322.06|-130.36| +2451033|74433|2451047|16598|89077|352774|5271|21768|60926|1006342|5659|15356|44|20|9|5|268|94|84|13.80|14.49|13.62|73.08|1144.08|1159.20|1217.16|91.52|0.00|486.36|1144.08|1235.60|1630.44|1721.96|-15.12| +2451033|74433|2451090|8126|89077|352774|5271|21768|60926|1006342|5659|15356|31|8|17|1|265|94|6|49.55|65.90|30.97|209.58|185.82|297.30|395.40|7.43|0.00|86.94|185.82|193.25|272.76|280.19|-111.48| +2451033|74433|2451043|9208|89077|352774|5271|21768|60926|1006342|5659|15356|1|19|19|5|74|94|25|78.43|104.31|84.49|495.50|2112.25|1960.75|2607.75|84.49|0.00|651.75|2112.25|2196.74|2764.00|2848.49|151.50| +2451033|74433|2451118|16892|89077|352774|5271|21768|60926|1006342|5659|15356|37|2|15|3|138|94|46|2.52|4.53|2.49|93.84|114.54|115.92|208.38|2.29|0.00|85.10|114.54|116.83|199.64|201.93|-1.38| +2451033|74433|2451116|11698|89077|352774|5271|21768|60926|1006342|5659|15356|32|22|17|2|237|94|34|55.77|72.50|29.72|1454.52|1010.48|1896.18|2465.00|10.10|0.00|468.18|1010.48|1020.58|1478.66|1488.76|-885.70| +2451033|74433|2451048|12202|89077|352774|5271|21768|60926|1006342|5659|15356|22|20|10|4|248|94|27|53.16|141.40|25.45|3130.65|687.15|1435.32|3817.80|61.84|0.00|381.78|687.15|748.99|1068.93|1130.77|-748.17| +2451033|74433|2451122|12794|89077|352774|5271|21768|60926|1006342|5659|15356|8|10|4|2|150|94|40|42.56|73.62|33.86|1590.40|1354.40|1702.40|2944.80|0.00|0.00|147.20|1354.40|1354.40|1501.60|1501.60|-348.00| +2451994|28063|2452030|11963|66345|115505|4218|9378|74719|1163832|2011|27971|21|9|19|5|264|95|85|25.61|58.39|49.04|794.75|4168.40|2176.85|4963.15|375.15|0.00|545.70|4168.40|4543.55|4714.10|5089.25|1991.55| +2451994|28063|2452080|7785|66345|115505|4218|9378|74719|1163832|2011|27971|59|29|3|1|39|95|51|47.21|66.09|5.94|3067.65|302.94|2407.71|3370.59|0.00|0.00|808.86|302.94|302.94|1111.80|1111.80|-2104.77| +2451994|28063|2452007|423|66345|115505|4218|9378|74719|1163832|2011|27971|19|5|16|4|162|95|52|80.50|212.52|89.25|6410.04|4641.00|4186.00|11051.04|232.05|0.00|5525.52|4641.00|4873.05|10166.52|10398.57|455.00| +2451994|28063|2451998|9279|66345|115505|4218|9378|74719|1163832|2011|27971|3|3|19|4|76|95|10|90.41|256.76|233.65|231.10|2336.50|904.10|2567.60|130.84|467.30|641.90|1869.20|2000.04|2511.10|2641.94|965.10| +2451994|28063|2452028|4647|66345|115505|4218|9378|74719|1163832|2011|27971|19|29|20|5|203|95|91|91.05|249.47|54.88|17707.69|4994.08|8285.55|22701.77|0.00|0.00|9307.48|4994.08|4994.08|14301.56|14301.56|-3291.47| +2451994|28063|2452063|11609|66345|115505|4218|9378|74719|1163832|2011|27971|39|3|17|4|294|95|70|27.23|48.19|25.54|1585.50|1787.80|1906.10|3373.30|0.00|214.53|235.90|1573.27|1573.27|1809.17|1809.17|-332.83| +2451994|28063|2452041|17461|66345|115505|4218|9378|74719|1163832|2011|27971|17|3|15|1|214|95|17|17.17|46.01|24.84|359.89|422.28|291.89|782.17|1.94|325.15|86.02|97.13|99.07|183.15|185.09|-194.76| +2451994|28063|2452069|5857|66345|115505|4218|9378|74719|1163832|2011|27971|15|15|8|3|278|95|61|83.76|226.15|90.46|8277.09|5518.06|5109.36|13795.15|379.64|772.52|6207.36|4745.54|5125.18|10952.90|11332.54|-363.82| +2451994|28063|2451998|15813|66345|115505|4218|9378|74719|1163832|2011|27971|23|19|8|3|251|95|75|90.47|264.17|184.91|5944.50|13868.25|6785.25|19812.75|554.73|0.00|6141.75|13868.25|14422.98|20010.00|20564.73|7083.00| +2451994|28063|2452101|3493|66345|115505|4218|9378|74719|1163832|2011|27971|3|5|1|2|73|95|23|42.55|69.35|6.24|1451.53|143.52|978.65|1595.05|8.61|0.00|270.94|143.52|152.13|414.46|423.07|-835.13| +2451994|28063|2452044|3201|66345|115505|4218|9378|74719|1163832|2011|27971|35|29|2|4|133|95|46|76.70|125.78|62.89|2892.94|2892.94|3528.20|5785.88|144.64|0.00|1330.32|2892.94|3037.58|4223.26|4367.90|-635.26| +2451994|28063|2452097|2515|66345|115505|4218|9378|74719|1163832|2011|27971|59|7|18|2|196|95|1|48.80|121.02|62.93|58.09|62.93|48.80|121.02|5.66|0.00|13.31|62.93|68.59|76.24|81.90|14.13| +2451994|28063|2452003|4185|66345|115505|4218|9378|74719|1163832|2011|27971|19|21|9|5|201|95|82|57.30|166.17|141.24|2044.26|11581.68|4698.60|13625.94|926.53|0.00|0.00|11581.68|12508.21|11581.68|12508.21|6883.08| +2451994|28063|2452082|15793|66345|115505|4218|9378|74719|1163832|2011|27971|15|11|17|5|233|95|69|9.73|17.80|4.45|921.15|307.05|671.37|1228.20|14.24|128.96|24.15|178.09|192.33|202.24|216.48|-493.28| +2451994|28063|2452107|11407|66345|115505|4218|9378|74719|1163832|2011|27971|35|9|19|1|160|95|88|83.98|152.00|147.44|401.28|12974.72|7390.24|13376.00|648.73|0.00|5885.44|12974.72|13623.45|18860.16|19508.89|5584.48| +2451153|47564|2451160|14839|11730|1455722|867|14059|94985|1532915|5188|44733|43|14|1|2|70|96|44|73.04|151.92|110.90|1804.88|4879.60|3213.76|6684.48|243.98|0.00|534.60|4879.60|5123.58|5414.20|5658.18|1665.84| +2451153|47564|2451238|5396|11730|1455722|867|14059|94985|1532915|5188|44733|34|2|3|4|5|96|1|31.17|70.44|10.56|59.88|10.56|31.17|70.44|0.52|0.00|21.13|10.56|11.08|31.69|32.21|-20.61| +2451153|47564|2451267|17290|11730|1455722|867|14059|94985|1532915|5188|44733|58|14|9|2|175|96|41|31.32|82.37|42.83|1621.14|1756.03|1284.12|3377.17|11.58|1176.54|1181.62|579.49|591.07|1761.11|1772.69|-704.63| +2451153|47564|2451206|12895|11730|1455722|867|14059|94985|1532915|5188|44733|8|8|2|2|139|96|32|69.25|85.87|82.43|110.08|2637.76|2216.00|2747.84|29.01|2057.45|164.80|580.31|609.32|745.11|774.12|-1635.69| +2451153|47564|2451202|4231|11730|1455722|867|14059|94985|1532915|5188|44733|1|20|14|4|184|96|25|63.03|158.83|38.11|3018.00|952.75|1575.75|3970.75|85.74|0.00|833.75|952.75|1038.49|1786.50|1872.24|-623.00| +2451153|47564|2451229|13282|11730|1455722|867|14059|94985|1532915|5188|44733|37|26|9|2|292|96|32|97.08|161.15|17.72|4589.76|567.04|3106.56|5156.80|39.69|0.00|2114.24|567.04|606.73|2681.28|2720.97|-2539.52| +2451153|47564|2451231|2644|11730|1455722|867|14059|94985|1532915|5188|44733|14|19|2|2|129|96|85|54.99|89.08|77.49|985.15|6586.65|4674.15|7571.80|395.19|0.00|1513.85|6586.65|6981.84|8100.50|8495.69|1912.50| +2451153|47564|2451269|13309|11730|1455722|867|14059|94985|1532915|5188|44733|50|2|8|4|110|96|28|5.29|13.06|4.70|234.08|131.60|148.12|365.68|3.94|0.00|87.64|131.60|135.54|219.24|223.18|-16.52| +2451153|47564|2451204|7669|11730|1455722|867|14059|94985|1532915|5188|44733|28|25|1|4|187|96|10|97.44|136.41|9.54|1268.70|95.40|974.40|1364.10|0.95|0.00|559.20|95.40|96.35|654.60|655.55|-879.00| +2451153|47564|2451250|4291|11730|1455722|867|14059|94985|1532915|5188|44733|2|7|3|5|257|96|23|61.56|110.19|69.41|937.94|1596.43|1415.88|2534.37|9.73|622.60|1190.94|973.83|983.56|2164.77|2174.50|-442.05| +2451153|47564|2451211|17960|11730|1455722|867|14059|94985|1532915|5188|44733|26|20|8|5|2|96|23|82.94|150.12|127.60|517.96|2934.80|1907.62|3452.76|234.78|0.00|103.50|2934.80|3169.58|3038.30|3273.08|1027.18| +2451153|47564|2451235|2564|11730|1455722|867|14059|94985|1532915|5188|44733|26|1|11|3|75|96|27|98.09|169.69|67.87|2749.14|1832.49|2648.43|4581.63|128.27|0.00|1740.96|1832.49|1960.76|3573.45|3701.72|-815.94| +2451153|47564|2451180|2956|11730|1455722|867|14059|94985|1532915|5188|44733|19|28|20|5|180|96|14|44.45|120.01|76.80|604.94|1075.20|622.30|1680.14|64.51|0.00|806.40|1075.20|1139.71|1881.60|1946.11|452.90| +2451153|47564|2451199|1820|11730|1455722|867|14059|94985|1532915|5188|44733|50|16|15|3|251|96|45|5.65|7.06|1.20|263.70|54.00|254.25|317.70|0.19|34.02|76.05|19.98|20.17|96.03|96.22|-234.27| +2451153|47564|2451170|15667|11730|1455722|867|14059|94985|1532915|5188|44733|20|19|2|4|226|96|43|2.61|4.01|2.24|76.11|96.32|112.23|172.43|1.21|76.09|55.04|20.23|21.44|75.27|76.48|-92.00| +2452499|43025|2452599|7215|12874|940613|5152|41644|73596|1392185|5261|24553|55|25|11|1|104|97|17|87.83|173.90|99.12|1271.26|1685.04|1493.11|2956.30|151.65|0.00|236.47|1685.04|1836.69|1921.51|2073.16|191.93| +2452499|43025|2452619|2490|12874|940613|5152|41644|73596|1392185|5261|24553|19|9|20|5|110|97|5|36.97|107.21|38.59|343.10|192.95|184.85|536.05|0.92|177.51|262.65|15.44|16.36|278.09|279.01|-169.41| +2452499|43025|2452597|12775|12874|940613|5152|41644|73596|1392185|5261|24553|37|6|7|3|82|97|21|53.32|86.37|23.31|1324.26|489.51|1119.72|1813.77|4.89|0.00|0.00|489.51|494.40|489.51|494.40|-630.21| +2452499|43025|2452510|16795|12874|940613|5152|41644|73596|1392185|5261|24553|9|6|1|4|57|97|7|52.27|59.58|5.95|375.41|41.65|365.89|417.06|1.87|10.41|8.33|31.24|33.11|39.57|41.44|-334.65| +2452499|43025|2452525|4854|12874|940613|5152|41644|73596|1392185|5261|24553|21|18|17|2|70|97|16|43.63|126.96|44.43|1320.48|710.88|698.08|2031.36|7.10|0.00|304.64|710.88|717.98|1015.52|1022.62|12.80| +2452499|43025|2452551|12751|12874|940613|5152|41644|73596|1392185|5261|24553|36|24|3|4|105|97|68|57.75|137.44|71.46|4486.64|4859.28|3927.00|9345.92|97.18|0.00|1308.32|4859.28|4956.46|6167.60|6264.78|932.28| +2452499|43025|2452616|1123|12874|940613|5152|41644|73596|1392185|5261|24553|42|3|6|5|224|97|71|37.79|65.37|55.56|696.51|3944.76|2683.09|4641.27|355.02|0.00|1484.61|3944.76|4299.78|5429.37|5784.39|1261.67| +2452499|43025|2452554|2493|12874|940613|5152|41644|73596|1392185|5261|24553|27|18|7|2|47|97|25|54.18|104.56|57.50|1176.50|1437.50|1354.50|2614.00|0.00|0.00|548.75|1437.50|1437.50|1986.25|1986.25|83.00| +2452499|43025|2452557|17769|12874|940613|5152|41644|73596|1392185|5261|24553|18|24|18|4|97|97|7|34.06|45.64|28.29|121.45|198.03|238.42|319.48|5.94|0.00|79.87|198.03|203.97|277.90|283.84|-40.39| +2451962|57038|2452053|5093|16297|370935|1573|7811|15799|667647|4125|13163|47|17|4|4|197|98|60|59.78|117.76|37.68|4804.80|2260.80|3586.80|7065.60|180.86|0.00|847.80|2260.80|2441.66|3108.60|3289.46|-1326.00| +2451962|57038|2452024|14793|16297|370935|1573|7811|15799|667647|4125|13163|49|25|11|2|150|98|61|23.45|34.94|12.92|1343.22|788.12|1430.45|2131.34|7.88|0.00|127.49|788.12|796.00|915.61|923.49|-642.33| +2451962|57038|2452012|7305|16297|370935|1573|7811|15799|667647|4125|13163|59|25|6|4|82|98|22|49.32|118.86|36.84|1804.44|810.48|1085.04|2614.92|8.10|0.00|601.26|810.48|818.58|1411.74|1419.84|-274.56| +2451962|57038|2452062|6135|16297|370935|1573|7811|15799|667647|4125|13163|19|25|16|4|52|98|22|44.85|69.96|23.08|1031.36|507.76|986.70|1539.12|0.00|0.00|384.78|507.76|507.76|892.54|892.54|-478.94| +2451962|57038|2452022|3583|16297|370935|1573|7811|15799|667647|4125|13163|31|5|7|1|68|98|7|86.36|217.62|145.80|502.74|1020.60|604.52|1523.34|0.00|0.00|411.25|1020.60|1020.60|1431.85|1431.85|416.08| +2451962|57038|2452000|11701|16297|370935|1573|7811|15799|667647|4125|13163|1|19|19|3|157|98|57|6.48|12.18|9.25|167.01|527.25|369.36|694.26|31.63|0.00|347.13|527.25|558.88|874.38|906.01|157.89| +2451962|57038|2452034|397|16297|370935|1573|7811|15799|667647|4125|13163|47|29|13|1|122|98|73|61.75|73.48|2.20|5203.44|160.60|4507.75|5364.04|11.24|0.00|321.20|160.60|171.84|481.80|493.04|-4347.15| +2451962|57038|2452041|8901|16297|370935|1573|7811|15799|667647|4125|13163|39|23|11|3|145|98|24|93.38|210.10|132.36|1865.76|3176.64|2241.12|5042.40|63.53|0.00|1512.72|3176.64|3240.17|4689.36|4752.89|935.52| +2451962|57038|2452011|4599|16297|370935|1573|7811|15799|667647|4125|13163|35|15|8|1|271|98|76|60.67|100.10|90.09|760.76|6846.84|4610.92|7607.60|547.74|0.00|380.00|6846.84|7394.58|7226.84|7774.58|2235.92| +2451962|57038|2452074|17183|16297|370935|1573|7811|15799|667647|4125|13163|5|9|17|1|258|98|71|92.33|141.26|46.61|6720.15|3309.31|6555.43|10029.46|99.27|0.00|802.30|3309.31|3408.58|4111.61|4210.88|-3246.12| +2451962|57038|2452080|13859|16297|370935|1573|7811|15799|667647|4125|13163|15|11|14|2|270|98|5|66.92|88.33|84.79|17.70|423.95|334.60|441.65|33.91|0.00|0.00|423.95|457.86|423.95|457.86|89.35| +2451962|57038|2452072|10529|16297|370935|1573|7811|15799|667647|4125|13163|57|9|14|5|136|98|33|99.49|143.26|97.41|1513.05|3214.53|3283.17|4727.58|12.53|1960.86|2221.89|1253.67|1266.20|3475.56|3488.09|-2029.50| +2451962|57038|2452071|9447|16297|370935|1573|7811|15799|667647|4125|13163|7|17|7|2|130|98|41|22.73|63.64|40.09|965.55|1643.69|931.93|2609.24|0.00|0.00|547.76|1643.69|1643.69|2191.45|2191.45|711.76| +2451543|25265|2451633|9364|19821|1761697|4373|3049|77876|54460|1124|31000|56|1|6|3|60|99|93|78.00|226.98|199.74|2533.32|18575.82|7254.00|21109.14|105.88|7987.60|6332.37|10588.22|10694.10|16920.59|17026.47|3334.22| +2451543|25265|2451629|13628|19821|1761697|4373|3049|77876|54460|1124|31000|55|25|11|1|14|99|62|25.08|50.66|40.52|628.68|2512.24|1554.96|3140.92|200.97|0.00|1067.64|2512.24|2713.21|3579.88|3780.85|957.28| +2451543|25265|2451562|12397|19821|1761697|4373|3049|77876|54460|1124|31000|2|10|17|3|219|99|42|48.19|137.34|105.75|1326.78|4441.50|2023.98|5768.28|0.00|1465.69|461.16|2975.81|2975.81|3436.97|3436.97|951.83| +2451543|25265|2451619|6244|19821|1761697|4373|3049|77876|54460|1124|31000|8|22|19|4|206|99|20|18.62|47.48|18.99|569.80|379.80|372.40|949.60|19.36|56.97|246.80|322.83|342.19|569.63|588.99|-49.57| +2451543|25265|2451640|13684|19821|1761697|4373|3049|77876|54460|1124|31000|58|22|14|4|191|99|5|35.16|60.82|55.34|27.40|276.70|175.80|304.10|24.90|0.00|149.00|276.70|301.60|425.70|450.60|100.90| +2451543|25265|2451655|6950|19821|1761697|4373|3049|77876|54460|1124|31000|52|19|6|4|185|99|29|70.39|175.97|131.97|1276.00|3827.13|2041.31|5103.13|153.08|0.00|1785.82|3827.13|3980.21|5612.95|5766.03|1785.82| +2451543|25265|2451653|1411|19821|1761697|4373|3049|77876|54460|1124|31000|44|10|19|5|218|99|64|79.63|137.75|26.17|7141.12|1674.88|5096.32|8816.00|133.99|0.00|881.28|1674.88|1808.87|2556.16|2690.15|-3421.44| +2451543|25265|2451600|10819|19821|1761697|4373|3049|77876|54460|1124|31000|31|14|20|5|7|99|71|16.54|34.56|16.58|1276.58|1177.18|1174.34|2453.76|58.85|0.00|490.61|1177.18|1236.03|1667.79|1726.64|2.84| +2451543|25265|2451662|11755|19821|1761697|4373|3049|77876|54460|1124|31000|46|14|2|3|297|99|45|73.53|183.08|148.29|1565.55|6673.05|3308.85|8238.60|266.92|0.00|3295.35|6673.05|6939.97|9968.40|10235.32|3364.20| +2451543|25265|2451600|4777|19821|1761697|4373|3049|77876|54460|1124|31000|32|1|5|4|103|99|86|35.37|78.52|8.63|6010.54|742.18|3041.82|6752.72|22.26|0.00|2565.38|742.18|764.44|3307.56|3329.82|-2299.64| +2451543|25265|2451643|10922|19821|1761697|4373|3049|77876|54460|1124|31000|10|25|10|2|94|99|87|91.98|102.09|66.35|3109.38|5772.45|8002.26|8881.83|387.90|923.59|976.14|4848.86|5236.76|5825.00|6212.90|-3153.40| +2451543|25265|2451568|11620|19821|1761697|4373|3049|77876|54460|1124|31000|1|28|19|3|87|99|46|11.57|20.13|13.08|324.30|601.68|532.22|925.98|6.73|517.44|287.04|84.24|90.97|371.28|378.01|-447.98| +2451489|45554|2451561|15598|87996|1424698|3447|49349|2066|1191653|1455|2957|34|2|6|3|277|100|21|27.35|77.40|33.28|926.52|698.88|574.35|1625.40|20.96|0.00|81.27|698.88|719.84|780.15|801.11|124.53| +2451489|45554|2451539|8509|87996|1424698|3447|49349|2066|1191653|1455|2957|56|2|5|1|34|100|80|43.50|76.56|48.23|2266.40|3858.40|3480.00|6124.80|192.92|0.00|2816.80|3858.40|4051.32|6675.20|6868.12|378.40| +2451489|45554|2451569|13148|87996|1424698|3447|49349|2066|1191653|1455|2957|37|2|12|2|25|100|76|99.78|189.58|45.49|10950.84|3457.24|7583.28|14408.08|69.14|0.00|1008.52|3457.24|3526.38|4465.76|4534.90|-4126.04| +2451489|45554|2451564|17030|87996|1424698|3447|49349|2066|1191653|1455|2957|1|7|17|5|223|100|59|3.45|9.48|1.32|481.44|77.88|203.55|559.32|1.55|0.00|257.24|77.88|79.43|335.12|336.67|-125.67| +2451489|45554|2451602|15322|87996|1424698|3447|49349|2066|1191653|1455|2957|8|10|10|1|55|100|89|61.04|87.28|9.60|6913.52|854.40|5432.56|7767.92|59.80|0.00|1165.01|854.40|914.20|2019.41|2079.21|-4578.16| +2451489|45554|2451552|724|87996|1424698|3447|49349|2066|1191653|1455|2957|46|16|4|1|181|100|99|63.05|162.66|58.55|10306.89|5796.45|6241.95|16103.34|64.92|4984.94|3380.85|811.51|876.43|4192.36|4257.28|-5430.44| +2451489|45554|2451609|1364|87996|1424698|3447|49349|2066|1191653|1455|2957|49|22|6|5|167|100|3|76.54|137.77|79.90|173.61|239.70|229.62|413.31|9.58|0.00|99.18|239.70|249.28|338.88|348.46|10.08| +2451489|45554|2451542|15442|87996|1424698|3447|49349|2066|1191653|1455|2957|2|14|4|5|139|100|71|13.07|36.20|23.89|874.01|1696.19|927.97|2570.20|84.80|0.00|796.62|1696.19|1780.99|2492.81|2577.61|768.22| +2451489|45554|2451532|1328|87996|1424698|3447|49349|2066|1191653|1455|2957|22|13|19|4|246|100|86|79.34|153.12|113.30|3424.52|9743.80|6823.24|13168.32|194.87|0.00|6320.14|9743.80|9938.67|16063.94|16258.81|2920.56| +2451489|45554|2451596|14347|87996|1424698|3447|49349|2066|1191653|1455|2957|40|20|1|1|71|100|21|61.75|160.55|97.93|1315.02|2056.53|1296.75|3371.55|185.08|0.00|640.50|2056.53|2241.61|2697.03|2882.11|759.78| +2452540|73343|2452639|16620|44578|1919699|2373|21094|44578|1919699|2373|21094|9|27|10|1|62|101|31|85.50|143.64|53.14|2805.50|1647.34|2650.50|4452.84|16.80|807.19|311.55|840.15|856.95|1151.70|1168.50|-1810.35| +2452540|73343|2452549|10699|44578|1919699|2373|21094|44578|1919699|2373|21094|51|7|15|1|187|101|4|92.93|104.08|55.16|195.68|220.64|371.72|416.32|19.85|0.00|174.84|220.64|240.49|395.48|415.33|-151.08| +2452540|73343|2452632|15708|44578|1919699|2373|21094|44578|1919699|2373|21094|55|27|10|4|145|101|22|51.47|91.61|83.36|181.50|1833.92|1132.34|2015.42|42.91|403.46|584.32|1430.46|1473.37|2014.78|2057.69|298.12| +2452540|73343|2452599|9795|44578|1919699|2373|21094|44578|1919699|2373|21094|48|24|13|4|292|101|50|29.22|56.39|27.63|1438.00|1381.50|1461.00|2819.50|13.81|0.00|817.50|1381.50|1395.31|2199.00|2212.81|-79.50| +2452540|73343|2452589|5989|44578|1919699|2373|21094|44578|1919699|2373|21094|33|12|8|2|139|101|28|29.97|89.91|50.34|1107.96|1409.52|839.16|2517.48|28.19|0.00|679.56|1409.52|1437.71|2089.08|2117.27|570.36| +2452540|73343|2452647|10687|44578|1919699|2373|21094|44578|1919699|2373|21094|55|1|7|3|116|101|19|86.14|221.37|177.09|841.32|3364.71|1636.66|4206.03|134.58|0.00|925.30|3364.71|3499.29|4290.01|4424.59|1728.05| +2452540|73343|2452598|13404|44578|1919699|2373|21094|44578|1919699|2373|21094|39|7|18|5|159|101|4|69.96|189.59|145.98|174.44|583.92|279.84|758.36|5.83|0.00|303.32|583.92|589.75|887.24|893.07|304.08| +2452540|73343|2452626|16807|44578|1919699|2373|21094|44578|1919699|2373|21094|13|13|10|3|143|101|73|49.90|139.72|81.03|4284.37|5915.19|3642.70|10199.56|414.06|0.00|3161.63|5915.19|6329.25|9076.82|9490.88|2272.49| +2452540|73343|2452599|16297|44578|1919699|2373|21094|44578|1919699|2373|21094|57|3|15|3|251|101|5|59.15|131.31|51.21|400.50|256.05|295.75|656.55|7.68|0.00|13.10|256.05|263.73|269.15|276.83|-39.70| +2451304|70566|2451368|2684|10393|1718135|1077|18819|16008|411354|845|13387|56|19|18|1|117|102|64|8.30|13.69|12.59|70.40|805.76|531.20|876.16|2.65|539.85|157.44|265.91|268.56|423.35|426.00|-265.29| +2451304|70566|2451380|16666|10393|1718135|1077|18819|16008|411354|845|13387|1|8|6|5|234|102|29|64.27|187.66|182.03|163.27|5278.87|1863.83|5442.14|316.73|0.00|2612.03|5278.87|5595.60|7890.90|8207.63|3415.04| +2451304|70566|2451368|7840|10393|1718135|1077|18819|16008|411354|845|13387|37|28|14|5|206|102|100|97.87|126.25|10.10|11615.00|1010.00|9787.00|12625.00|20.20|0.00|126.00|1010.00|1030.20|1136.00|1156.20|-8777.00| +2451304|70566|2451331|13642|10393|1718135|1077|18819|16008|411354|845|13387|28|10|10|1|55|102|68|70.29|194.00|64.02|8838.64|4353.36|4779.72|13192.00|0.00|1131.87|3034.16|3221.49|3221.49|6255.65|6255.65|-1558.23| +2451304|70566|2451368|11551|10393|1718135|1077|18819|16008|411354|845|13387|38|14|19|2|55|102|78|70.29|203.13|203.13|0.00|15844.14|5482.62|15844.14|950.64|0.00|5228.34|15844.14|16794.78|21072.48|22023.12|10361.52| +2451304|70566|2451310|8336|10393|1718135|1077|18819|16008|411354|845|13387|55|16|3|4|65|102|42|51.11|86.37|59.59|1124.76|2502.78|2146.62|3627.54|25.02|0.00|1704.78|2502.78|2527.80|4207.56|4232.58|356.16| +2451304|70566|2451408|12763|10393|1718135|1077|18819|16008|411354|845|13387|7|10|8|5|132|102|82|41.04|119.42|62.09|4701.06|5091.38|3365.28|9792.44|203.65|0.00|3525.18|5091.38|5295.03|8616.56|8820.21|1726.10| +2451304|70566|2451401|16840|10393|1718135|1077|18819|16008|411354|845|13387|28|1|4|5|56|102|48|18.86|51.29|17.43|1625.28|836.64|905.28|2461.92|41.83|0.00|1107.84|836.64|878.47|1944.48|1986.31|-68.64| +2451304|70566|2451389|1750|10393|1718135|1077|18819|16008|411354|845|13387|14|7|10|1|202|102|46|25.47|55.01|49.50|253.46|2277.00|1171.62|2530.46|50.54|1434.51|25.30|842.49|893.03|867.79|918.33|-329.13| +2451304|70566|2451393|1904|10393|1718135|1077|18819|16008|411354|845|13387|58|25|6|4|262|102|22|79.71|216.01|153.36|1378.30|3373.92|1753.62|4752.22|168.69|0.00|1140.48|3373.92|3542.61|4514.40|4683.09|1620.30| +2451304|70566|2451306|4153|10393|1718135|1077|18819|16008|411354|845|13387|37|8|8|4|300|102|66|40.18|73.93|27.35|3074.28|1805.10|2651.88|4879.38|0.00|0.00|390.06|1805.10|1805.10|2195.16|2195.16|-846.78| +2451304|70566|2451317|2420|10393|1718135|1077|18819|16008|411354|845|13387|37|4|20|4|98|102|98|79.64|166.44|6.65|15659.42|651.70|7804.72|16311.12|4.10|241.12|3261.44|410.58|414.68|3672.02|3676.12|-7394.14| +2451304|70566|2451357|14186|10393|1718135|1077|18819|16008|411354|845|13387|4|25|13|3|163|102|31|26.89|43.29|14.28|899.31|442.68|833.59|1341.99|4.42|0.00|630.54|442.68|447.10|1073.22|1077.64|-390.91| +2451801|66921|2451807|9499|49335|1467614|2693|31772|92727|1146590|5056|8982|25|1|9|2|265|103|63|67.89|105.90|31.77|4670.19|2001.51|4277.07|6671.70|60.04|0.00|1267.56|2001.51|2061.55|3269.07|3329.11|-2275.56| +2451801|66921|2451869|12812|49335|1467614|2693|31772|92727|1146590|5056|8982|13|11|15|2|236|103|38|11.96|22.72|22.49|8.74|854.62|454.48|863.36|59.82|0.00|379.62|854.62|914.44|1234.24|1294.06|400.14| +2451801|66921|2451913|15851|49335|1467614|2693|31772|92727|1146590|5056|8982|31|5|13|3|273|103|17|21.30|28.75|6.32|381.31|107.44|362.10|488.75|3.22|0.00|24.31|107.44|110.66|131.75|134.97|-254.66| +2451801|66921|2451818|13183|49335|1467614|2693|31772|92727|1146590|5056|8982|11|13|8|4|153|103|42|90.25|91.15|29.16|2603.58|1224.72|3790.50|3828.30|36.74|0.00|306.18|1224.72|1261.46|1530.90|1567.64|-2565.78| +2451801|66921|2451809|9577|49335|1467614|2693|31772|92727|1146590|5056|8982|2|13|5|3|271|103|53|24.93|29.91|16.15|729.28|855.95|1321.29|1585.23|8.55|0.00|443.61|855.95|864.50|1299.56|1308.11|-465.34| +2451801|66921|2451812|13781|49335|1467614|2693|31772|92727|1146590|5056|8982|2|8|8|1|141|103|34|67.19|124.97|77.48|1614.66|2634.32|2284.46|4248.98|56.90|1685.96|1317.16|948.36|1005.26|2265.52|2322.42|-1336.10| +2451801|66921|2451855|12643|49335|1467614|2693|31772|92727|1146590|5056|8982|59|26|1|5|109|103|80|38.90|93.36|41.07|4183.20|3285.60|3112.00|7468.80|165.59|1445.66|3360.80|1839.94|2005.53|5200.74|5366.33|-1272.06| +2451801|66921|2451853|697|49335|1467614|2693|31772|92727|1146590|5056|8982|31|8|14|3|32|103|45|97.50|206.70|128.15|3534.75|5766.75|4387.50|9301.50|0.00|0.00|4185.45|5766.75|5766.75|9952.20|9952.20|1379.25| +2451801|66921|2451806|8258|49335|1467614|2693|31772|92727|1146590|5056|8982|13|19|6|4|5|103|9|13.43|29.00|28.13|7.83|253.17|120.87|261.00|22.78|0.00|39.15|253.17|275.95|292.32|315.10|132.30| +2451801|66921|2451843|13649|49335|1467614|2693|31772|92727|1146590|5056|8982|11|19|4|5|209|103|10|11.20|24.19|20.31|38.80|203.10|112.00|241.90|4.06|0.00|53.20|203.10|207.16|256.30|260.36|91.10| +2451801|66921|2451868|10775|49335|1467614|2693|31772|92727|1146590|5056|8982|13|7|11|3|235|103|30|74.32|217.75|71.85|4377.00|2155.50|2229.60|6532.50|43.11|0.00|391.80|2155.50|2198.61|2547.30|2590.41|-74.10| +2451801|66921|2451821|4183|49335|1467614|2693|31772|92727|1146590|5056|8982|26|17|14|1|1|103|45|39.20|111.32|38.96|3256.20|1753.20|1764.00|5009.40|105.19|0.00|250.20|1753.20|1858.39|2003.40|2108.59|-10.80| +2451801|66921|2451876|11579|49335|1467614|2693|31772|92727|1146590|5056|8982|13|20|12|3|187|103|14|40.94|73.69|26.52|660.38|371.28|573.16|1031.66|3.71|0.00|92.82|371.28|374.99|464.10|467.81|-201.88| +2451801|66921|2451838|3997|49335|1467614|2693|31772|92727|1146590|5056|8982|8|8|12|2|229|103|36|67.84|107.86|4.31|3727.80|155.16|2442.24|3882.96|9.30|0.00|1669.32|155.16|164.46|1824.48|1833.78|-2287.08| +2451399|78997|2451452|2566|20053|1385627|1|45975|74401|1501553|5944|47518|25|16|5|1|117|104|66|19.51|25.36|20.54|318.12|1355.64|1287.66|1673.76|0.00|0.00|485.10|1355.64|1355.64|1840.74|1840.74|67.98| +2451399|78997|2451457|16580|20053|1385627|1|45975|74401|1501553|5944|47518|37|13|2|5|128|104|64|15.97|28.10|20.51|485.76|1312.64|1022.08|1798.40|65.63|0.00|449.28|1312.64|1378.27|1761.92|1827.55|290.56| +2451399|78997|2451473|7327|20053|1385627|1|45975|74401|1501553|5944|47518|50|20|3|5|250|104|45|15.18|16.84|0.50|735.30|22.50|683.10|757.80|1.35|0.00|280.35|22.50|23.85|302.85|304.20|-660.60| +2451399|78997|2451400|1975|20053|1385627|1|45975|74401|1501553|5944|47518|19|28|17|4|194|104|30|19.45|28.59|19.44|274.50|583.20|583.50|857.70|40.82|0.00|265.80|583.20|624.02|849.00|889.82|-0.30| +2451399|78997|2451488|7981|20053|1385627|1|45975|74401|1501553|5944|47518|10|28|11|5|109|104|93|57.81|126.60|86.08|3768.36|8005.44|5376.33|11773.80|80.05|0.00|5650.68|8005.44|8085.49|13656.12|13736.17|2629.11| +2451399|78997|2451419|5366|20053|1385627|1|45975|74401|1501553|5944|47518|13|20|15|4|83|104|20|71.81|113.45|38.57|1497.60|771.40|1436.20|2269.00|23.14|0.00|22.60|771.40|794.54|794.00|817.14|-664.80| +2451399|78997|2451460|836|20053|1385627|1|45975|74401|1501553|5944|47518|2|25|17|1|272|104|29|33.43|60.84|38.93|635.39|1128.97|969.47|1764.36|2.48|880.59|70.47|248.38|250.86|318.85|321.33|-721.09| +2451399|78997|2451415|28|20053|1385627|1|45975|74401|1501553|5944|47518|13|7|8|5|212|104|93|61.68|119.65|71.79|4450.98|6676.47|5736.24|11127.45|97.47|1802.64|3783.24|4873.83|4971.30|8657.07|8754.54|-862.41| +2451399|78997|2451505|15856|20053|1385627|1|45975|74401|1501553|5944|47518|44|4|16|4|162|104|43|46.75|67.32|28.27|1679.15|1215.61|2010.25|2894.76|97.24|0.00|781.31|1215.61|1312.85|1996.92|2094.16|-794.64| +2451399|78997|2451446|1808|20053|1385627|1|45975|74401|1501553|5944|47518|22|10|13|2|274|104|48|96.88|268.35|83.18|8888.16|3992.64|4650.24|12880.80|39.92|0.00|3219.84|3992.64|4032.56|7212.48|7252.40|-657.60| +2451399|78997|2451442|11455|20053|1385627|1|45975|74401|1501553|5944|47518|46|2|7|5|33|104|46|13.01|35.77|24.32|526.70|1118.72|598.46|1645.42|78.31|0.00|822.48|1118.72|1197.03|1941.20|2019.51|520.26| +2452631|48867|2452639|4227|53098|1635631|6517|3050|32917|1061448|5802|47535|30|18|3|2|51|105|61|79.32|134.05|50.93|5070.32|3106.73|4838.52|8177.05|186.40|0.00|3516.04|3106.73|3293.13|6622.77|6809.17|-1731.79| +2452631|48867|2452729|12030|53098|1635631|6517|3050|32917|1061448|5802|47535|54|19|6|4|72|105|68|43.97|58.04|9.28|3315.68|631.04|2989.96|3946.72|45.43|63.10|1736.04|567.94|613.37|2303.98|2349.41|-2422.02| +2452631|48867|2452667|6007|53098|1635631|6517|3050|32917|1061448|5802|47535|48|15|19|3|214|105|58|87.04|208.02|191.37|965.70|11099.46|5048.32|12065.16|332.98|0.00|2895.36|11099.46|11432.44|13994.82|14327.80|6051.14| +2452631|48867|2452667|16545|53098|1635631|6517|3050|32917|1061448|5802|47535|57|15|17|1|215|105|86|32.79|37.05|22.97|1210.88|1975.42|2819.94|3186.30|79.01|0.00|381.84|1975.42|2054.43|2357.26|2436.27|-844.52| +2452631|48867|2452709|3009|53098|1635631|6517|3050|32917|1061448|5802|47535|1|18|7|5|26|105|11|13.74|36.13|6.50|325.93|71.50|151.14|397.43|1.43|0.00|150.92|71.50|72.93|222.42|223.85|-79.64| +2452631|48867|2452672|10311|53098|1635631|6517|3050|32917|1061448|5802|47535|51|1|19|4|122|105|65|55.00|164.45|23.02|9192.95|1496.30|3575.00|10689.25|74.81|0.00|5130.45|1496.30|1571.11|6626.75|6701.56|-2078.70| +2452631|48867|2452724|14829|53098|1635631|6517|3050|32917|1061448|5802|47535|18|9|11|5|106|105|79|38.58|42.05|36.58|432.13|2889.82|3047.82|3321.95|115.59|0.00|1494.68|2889.82|3005.41|4384.50|4500.09|-158.00| +2452631|48867|2452677|253|53098|1635631|6517|3050|32917|1061448|5802|47535|37|21|3|5|227|105|16|50.94|69.27|16.62|842.40|265.92|815.04|1108.32|13.29|0.00|398.88|265.92|279.21|664.80|678.09|-549.12| +2451543|52256|2451607|7480|64090|1631987|5931|5828|68628|931425|4366|33910|46|4|15|2|99|106|2|58.91|174.37|31.38|285.98|62.76|117.82|348.74|1.58|40.16|83.68|22.60|24.18|106.28|107.86|-95.22| +2451543|52256|2451635|6049|64090|1631987|5931|5828|68628|931425|4366|33910|25|4|13|5|100|106|61|40.57|87.63|53.45|2084.98|3260.45|2474.77|5345.43|36.51|2738.77|53.07|521.68|558.19|574.75|611.26|-1953.09| +2451543|52256|2451610|12253|64090|1631987|5931|5828|68628|931425|4366|33910|26|22|13|4|105|106|38|27.85|37.59|5.26|1228.54|199.88|1058.30|1428.42|9.99|0.00|471.20|199.88|209.87|671.08|681.07|-858.42| +2451543|52256|2451597|14788|64090|1631987|5931|5828|68628|931425|4366|33910|32|14|19|5|131|106|99|60.39|169.69|6.78|16128.09|671.22|5978.61|16799.31|0.00|0.00|1175.13|671.22|671.22|1846.35|1846.35|-5307.39| +2451543|52256|2451660|15571|64090|1631987|5931|5828|68628|931425|4366|33910|25|13|4|3|132|106|7|59.17|130.17|0.00|911.19|0.00|414.19|911.19|0.00|0.00|346.22|0.00|0.00|346.22|346.22|-414.19| +2451543|52256|2451656|8302|64090|1631987|5931|5828|68628|931425|4366|33910|14|22|3|2|248|106|23|13.54|38.31|9.57|661.02|220.11|311.42|881.13|6.60|0.00|26.22|220.11|226.71|246.33|252.93|-91.31| +2451543|52256|2451574|4561|64090|1631987|5931|5828|68628|931425|4366|33910|4|26|12|3|271|106|70|72.72|212.34|188.98|1635.20|13228.60|5090.40|14863.80|132.28|0.00|1337.70|13228.60|13360.88|14566.30|14698.58|8138.20| +2451543|52256|2451559|5906|64090|1631987|5931|5828|68628|931425|4366|33910|7|28|3|4|235|106|17|93.18|165.86|14.92|2565.98|253.64|1584.06|2819.62|12.68|0.00|873.97|253.64|266.32|1127.61|1140.29|-1330.42| +2451543|52256|2451551|11128|64090|1631987|5931|5828|68628|931425|4366|33910|26|16|11|3|135|106|84|74.07|186.65|63.46|10347.96|5330.64|6221.88|15678.60|266.53|0.00|3762.36|5330.64|5597.17|9093.00|9359.53|-891.24| +2451543|52256|2451624|13357|64090|1631987|5931|5828|68628|931425|4366|33910|49|10|14|5|196|106|10|26.53|49.87|40.89|89.80|408.90|265.30|498.70|8.17|0.00|4.90|408.90|417.07|413.80|421.97|143.60| +2451543|52256|2451661|10306|64090|1631987|5931|5828|68628|931425|4366|33910|1|2|3|4|275|106|87|66.12|76.69|71.32|467.19|6204.84|5752.44|6672.03|62.04|0.00|1133.61|6204.84|6266.88|7338.45|7400.49|452.40| +2452514|44258|2452551|9702|84254|286093|2332|40971|56255|1211796|3288|16280|21|21|14|2|50|107|87|81.12|200.36|4.00|17083.32|348.00|7057.44|17431.32|0.00|0.00|8540.79|348.00|348.00|8888.79|8888.79|-6709.44| +2452514|44258|2452631|1641|84254|286093|2332|40971|56255|1211796|3288|16280|33|25|9|2|68|107|6|32.83|52.85|41.22|69.78|247.32|196.98|317.10|0.00|0.00|34.86|247.32|247.32|282.18|282.18|50.34| +2452514|44258|2452551|4099|84254|286093|2332|40971|56255|1211796|3288|16280|51|15|17|1|268|107|13|8.21|13.13|0.52|163.93|6.76|106.73|170.69|0.00|0.00|27.30|6.76|6.76|34.06|34.06|-99.97| +2452514|44258|2452582|8958|84254|286093|2332|40971|56255|1211796|3288|16280|45|24|11|2|256|107|68|68.75|76.31|67.15|622.88|4566.20|4675.00|5189.08|0.00|4246.56|2542.52|319.64|319.64|2862.16|2862.16|-4355.36| +2452514|44258|2452527|17131|84254|286093|2332|40971|56255|1211796|3288|16280|36|7|3|4|8|107|81|71.52|142.32|96.77|3689.55|7838.37|5793.12|11527.92|391.91|0.00|2536.11|7838.37|8230.28|10374.48|10766.39|2045.25| +2452514|44258|2452571|7831|84254|286093|2332|40971|56255|1211796|3288|16280|24|19|15|3|228|107|75|14.97|24.25|10.42|1037.25|781.50|1122.75|1818.75|31.26|0.00|836.25|781.50|812.76|1617.75|1649.01|-341.25| +2452514|44258|2452613|12825|84254|286093|2332|40971|56255|1211796|3288|16280|36|12|7|5|23|107|70|26.02|34.60|5.53|2034.90|387.10|1821.40|2422.00|27.09|0.00|1040.90|387.10|414.19|1428.00|1455.09|-1434.30| +2452514|44258|2452574|3745|84254|286093|2332|40971|56255|1211796|3288|16280|55|27|3|4|78|107|45|76.52|205.07|166.10|1753.65|7474.50|3443.40|9228.15|597.96|0.00|1476.45|7474.50|8072.46|8950.95|9548.91|4031.10| +2451262|36450|2451375|16|64734|1100959|1184|28609|89258|1248852|2084|47109|20|20|4|4|237|108|4|1.52|3.72|2.93|3.16|11.72|6.08|14.88|0.82|0.00|6.84|11.72|12.54|18.56|19.38|5.64| +2451262|36450|2451378|2408|64734|1100959|1184|28609|89258|1248852|2084|47109|50|2|8|4|80|108|42|57.11|100.51|2.01|4137.00|84.42|2398.62|4221.42|0.00|0.00|379.68|84.42|84.42|464.10|464.10|-2314.20| +2451262|36450|2451360|16178|64734|1100959|1184|28609|89258|1248852|2084|47109|37|28|18|4|130|108|90|22.11|55.93|50.33|504.00|4529.70|1989.90|5033.70|407.67|0.00|804.60|4529.70|4937.37|5334.30|5741.97|2539.80| +2451262|36450|2451305|11641|64734|1100959|1184|28609|89258|1248852|2084|47109|28|4|11|2|230|108|36|8.38|8.54|0.42|292.32|15.12|301.68|307.44|0.30|0.00|73.44|15.12|15.42|88.56|88.86|-286.56| +2451262|36450|2451354|1897|64734|1100959|1184|28609|89258|1248852|2084|47109|1|28|19|3|200|108|90|20.82|27.27|25.08|197.10|2257.20|1873.80|2454.30|180.57|0.00|564.30|2257.20|2437.77|2821.50|3002.07|383.40| +2451262|36450|2451281|1465|64734|1100959|1184|28609|89258|1248852|2084|47109|32|26|18|5|72|108|2|65.83|168.52|0.00|337.04|0.00|131.66|337.04|0.00|0.00|20.22|0.00|0.00|20.22|20.22|-131.66| +2451262|36450|2451344|16429|64734|1100959|1184|28609|89258|1248852|2084|47109|56|14|17|1|275|108|25|78.25|155.71|45.15|2764.00|1128.75|1956.25|3892.75|67.72|0.00|1868.50|1128.75|1196.47|2997.25|3064.97|-827.50| +2451262|36450|2451322|9544|64734|1100959|1184|28609|89258|1248852|2084|47109|40|4|12|4|16|108|12|59.96|137.90|119.97|215.16|1439.64|719.52|1654.80|86.37|0.00|463.32|1439.64|1526.01|1902.96|1989.33|720.12| +2451262|36450|2451348|5617|64734|1100959|1184|28609|89258|1248852|2084|47109|38|28|20|5|180|108|3|20.17|56.87|7.96|146.73|23.88|60.51|170.61|0.71|0.00|56.28|23.88|24.59|80.16|80.87|-36.63| +2451262|36450|2451285|10292|64734|1100959|1184|28609|89258|1248852|2084|47109|31|19|4|3|112|108|91|1.49|2.74|0.43|210.21|39.13|135.59|249.34|0.78|0.00|57.33|39.13|39.91|96.46|97.24|-96.46| +2451262|36450|2451378|14335|64734|1100959|1184|28609|89258|1248852|2084|47109|25|26|15|1|47|108|80|33.54|74.12|42.98|2491.20|3438.40|2683.20|5929.60|275.07|0.00|2016.00|3438.40|3713.47|5454.40|5729.47|755.20| +2451262|36450|2451334|2402|64734|1100959|1184|28609|89258|1248852|2084|47109|34|8|2|4|266|108|97|76.34|90.08|36.93|5155.55|3582.21|7404.98|8737.76|35.82|0.00|436.50|3582.21|3618.03|4018.71|4054.53|-3822.77| +2451734|36966|2451816|1841|44486|616499|6559|1019|74634|1620204|4752|3965|2|17|6|3|300|109|54|79.85|189.24|52.98|7358.04|2860.92|4311.90|10218.96|257.48|0.00|3474.36|2860.92|3118.40|6335.28|6592.76|-1450.98| +2451734|36966|2451784|9719|44486|616499|6559|1019|74634|1620204|4752|3965|50|29|1|2|273|109|36|61.67|178.84|110.88|2446.56|3991.68|2220.12|6438.24|319.33|0.00|3090.24|3991.68|4311.01|7081.92|7401.25|1771.56| +2451734|36966|2451768|17579|44486|616499|6559|1019|74634|1620204|4752|3965|26|29|6|2|205|109|48|18.39|45.60|39.21|306.72|1882.08|882.72|2188.80|37.64|0.00|678.24|1882.08|1919.72|2560.32|2597.96|999.36| +2451734|36966|2451848|7657|44486|616499|6559|1019|74634|1620204|4752|3965|26|25|2|1|280|109|60|22.50|46.80|19.18|1657.20|1150.80|1350.00|2808.00|67.66|184.12|0.00|966.68|1034.34|966.68|1034.34|-383.32| +2451734|36966|2451854|8921|44486|616499|6559|1019|74634|1620204|4752|3965|59|23|17|4|191|109|9|58.35|61.85|40.82|189.27|367.38|525.15|556.65|3.67|0.00|55.62|367.38|371.05|423.00|426.67|-157.77| +2451734|36966|2451853|10304|44486|616499|6559|1019|74634|1620204|4752|3965|35|7|3|4|135|109|62|47.58|56.14|25.82|1879.84|1600.84|2949.96|3480.68|64.03|0.00|835.14|1600.84|1664.87|2435.98|2500.01|-1349.12| +2451734|36966|2451801|17033|44486|616499|6559|1019|74634|1620204|4752|3965|44|2|14|3|112|109|83|25.29|58.41|35.04|1939.71|2908.32|2099.07|4848.03|145.41|0.00|678.11|2908.32|3053.73|3586.43|3731.84|809.25| +2451734|36966|2451811|17624|44486|616499|6559|1019|74634|1620204|4752|3965|47|11|15|2|109|109|69|11.22|32.08|27.58|310.50|1903.02|774.18|2213.52|20.55|875.38|287.73|1027.64|1048.19|1315.37|1335.92|253.46| +2451734|36966|2451751|16097|44486|616499|6559|1019|74634|1620204|4752|3965|59|26|11|5|139|109|32|44.18|69.80|23.03|1496.64|736.96|1413.76|2233.60|29.47|0.00|558.40|736.96|766.43|1295.36|1324.83|-676.80| +2451734|36966|2451763|5587|44486|616499|6559|1019|74634|1620204|4752|3965|56|20|20|2|114|109|26|73.70|168.03|166.34|43.94|4324.84|1916.20|4368.78|259.49|0.00|1572.74|4324.84|4584.33|5897.58|6157.07|2408.64| +2451734|36966|2451774|4003|44486|616499|6559|1019|74634|1620204|4752|3965|50|13|15|3|241|109|97|58.70|78.65|47.97|2975.96|4653.09|5693.90|7629.05|372.24|0.00|152.29|4653.09|5025.33|4805.38|5177.62|-1040.81| +2451734|36966|2451793|8551|44486|616499|6559|1019|74634|1620204|4752|3965|49|20|17|3|149|109|17|53.85|54.92|25.81|494.87|438.77|915.45|933.64|13.16|0.00|18.53|438.77|451.93|457.30|470.46|-476.68| +2451734|36966|2451804|14111|44486|616499|6559|1019|74634|1620204|4752|3965|55|1|17|5|26|109|33|14.68|42.86|30.85|396.33|1018.05|484.44|1414.38|20.36|0.00|254.43|1018.05|1038.41|1272.48|1292.84|533.61| +2451734|36966|2451754|14189|44486|616499|6559|1019|74634|1620204|4752|3965|35|11|4|4|277|109|70|20.80|41.60|2.08|2766.40|145.60|1456.00|2912.00|10.19|0.00|698.60|145.60|155.79|844.20|854.39|-1310.40| +2451734|36966|2451839|1970|44486|616499|6559|1019|74634|1620204|4752|3965|5|23|4|1|25|109|11|88.95|128.97|82.54|510.73|907.94|978.45|1418.67|45.39|0.00|0.00|907.94|953.33|907.94|953.33|-70.51| +2451734|36966|2451828|4805|44486|616499|6559|1019|74634|1620204|4752|3965|50|13|11|2|26|109|87|89.49|259.52|41.52|18966.00|3612.24|7785.63|22578.24|97.53|361.22|7676.01|3251.02|3348.55|10927.03|11024.56|-4534.61| +2451904|38622|2452018|9077|11364|1823136|508|280|1585|1022397|1196|2168|55|5|8|3|288|110|36|8.73|24.96|15.22|350.64|547.92|314.28|898.56|43.83|0.00|305.28|547.92|591.75|853.20|897.03|233.64| +2451904|38622|2451952|7544|11364|1823136|508|280|1585|1022397|1196|2168|53|20|12|1|67|110|46|94.91|282.83|280.00|130.18|12880.00|4365.86|13010.18|644.00|0.00|5203.98|12880.00|13524.00|18083.98|18727.98|8514.14| +2451904|38622|2451970|2309|11364|1823136|508|280|1585|1022397|1196|2168|35|23|3|4|98|110|18|93.44|231.73|222.46|166.86|4004.28|1681.92|4171.14|80.08|0.00|583.92|4004.28|4084.36|4588.20|4668.28|2322.36| +2451904|38622|2452021|13505|11364|1823136|508|280|1585|1022397|1196|2168|20|20|18|3|48|110|4|23.94|44.28|33.65|42.52|134.60|95.76|177.12|6.73|0.00|5.28|134.60|141.33|139.88|146.61|38.84| +2451904|38622|2451947|3020|11364|1823136|508|280|1585|1022397|1196|2168|8|25|4|2|293|110|93|63.14|70.71|0.70|6510.93|65.10|5872.02|6576.03|2.60|0.00|1709.34|65.10|67.70|1774.44|1777.04|-5806.92| +2451904|38622|2451963|1196|11364|1823136|508|280|1585|1022397|1196|2168|43|7|16|3|10|110|35|96.55|100.41|55.22|1581.65|1932.70|3379.25|3514.35|1.93|1836.06|1335.25|96.64|98.57|1431.89|1433.82|-3282.61| +2451904|38622|2451915|6331|11364|1823136|508|280|1585|1022397|1196|2168|11|17|5|3|34|110|90|69.39|172.08|86.04|7743.60|7743.60|6245.10|15487.20|542.05|0.00|1083.60|7743.60|8285.65|8827.20|9369.25|1498.50| +2451904|38622|2451968|968|11364|1823136|508|280|1585|1022397|1196|2168|50|1|9|4|182|110|9|30.38|49.82|49.82|0.00|448.38|273.42|448.38|13.45|0.00|197.28|448.38|461.83|645.66|659.11|174.96| +2451904|38622|2451959|13127|11364|1823136|508|280|1585|1022397|1196|2168|37|20|20|2|294|110|12|83.42|115.11|112.80|27.72|1353.60|1001.04|1381.32|40.60|0.00|469.56|1353.60|1394.20|1823.16|1863.76|352.56| +2451904|38622|2451963|6761|11364|1823136|508|280|1585|1022397|1196|2168|55|29|2|4|149|110|45|18.11|51.43|34.97|740.70|1573.65|814.95|2314.35|78.68|0.00|486.00|1573.65|1652.33|2059.65|2138.33|758.70| +2451904|38622|2452001|11630|11364|1823136|508|280|1585|1022397|1196|2168|17|8|18|4|247|110|53|95.41|209.90|159.52|2670.14|8454.56|5056.73|11124.70|338.18|0.00|2224.94|8454.56|8792.74|10679.50|11017.68|3397.83| +2451904|38622|2451975|15901|11364|1823136|508|280|1585|1022397|1196|2168|50|7|5|3|81|110|42|85.97|112.62|29.28|3500.28|1229.76|3610.74|4730.04|86.08|0.00|2081.10|1229.76|1315.84|3310.86|3396.94|-2380.98| +2451904|38622|2451974|17156|11364|1823136|508|280|1585|1022397|1196|2168|13|23|17|2|27|110|86|89.21|96.34|63.58|2817.36|5467.88|7672.06|8285.24|313.85|984.21|2070.88|4483.67|4797.52|6554.55|6868.40|-3188.39| +2452248|60861|2452283|14015|95403|1522037|5024|7658|52062|966341|5866|34900|15|11|4|2|113|111|5|2.28|6.36|6.16|1.00|30.80|11.40|31.80|0.30|0.00|15.90|30.80|31.10|46.70|47.00|19.40| +2452248|60861|2452284|5785|95403|1522037|5024|7658|52062|966341|5866|34900|53|5|16|2|88|111|10|3.19|4.68|0.65|40.30|6.50|31.90|46.80|0.13|0.00|6.00|6.50|6.63|12.50|12.63|-25.40| +2452248|60861|2452311|8125|95403|1522037|5024|7658|52062|966341|5866|34900|45|25|3|3|43|111|62|52.55|118.23|96.94|1319.98|6010.28|3258.10|7330.26|420.71|0.00|1979.04|6010.28|6430.99|7989.32|8410.03|2752.18| +2452248|60861|2452252|5099|95403|1522037|5024|7658|52062|966341|5866|34900|35|23|6|5|271|111|5|96.31|205.14|108.72|482.10|543.60|481.55|1025.70|5.43|0.00|0.00|543.60|549.03|543.60|549.03|62.05| +2452248|60861|2452320|6593|95403|1522037|5024|7658|52062|966341|5866|34900|45|25|13|5|260|111|93|97.38|282.40|220.27|5778.09|20485.11|9056.34|26263.20|409.70|0.00|9192.12|20485.11|20894.81|29677.23|30086.93|11428.77| +2452248|60861|2452346|11909|95403|1522037|5024|7658|52062|966341|5866|34900|49|3|16|3|300|111|19|4.25|6.03|1.92|78.09|36.48|80.75|114.57|0.50|19.69|37.62|16.79|17.29|54.41|54.91|-63.96| +2452248|60861|2452252|1219|95403|1522037|5024|7658|52062|966341|5866|34900|21|15|15|2|269|111|61|60.64|79.43|43.68|2180.75|2664.48|3699.04|4845.23|186.51|0.00|823.50|2664.48|2850.99|3487.98|3674.49|-1034.56| +2452248|60861|2452301|607|95403|1522037|5024|7658|52062|966341|5866|34900|9|23|9|4|236|111|28|98.39|225.31|6.75|6119.68|189.00|2754.92|6308.68|15.12|0.00|2901.92|189.00|204.12|3090.92|3106.04|-2565.92| +2452248|60861|2452328|2369|95403|1522037|5024|7658|52062|966341|5866|34900|7|7|14|4|274|111|52|1.53|4.08|0.81|170.04|42.12|79.56|212.16|0.84|0.00|22.88|42.12|42.96|65.00|65.84|-37.44| +2452248|60861|2452276|12715|95403|1522037|5024|7658|52062|966341|5866|34900|19|13|1|3|192|111|51|60.51|101.65|64.03|1918.62|3265.53|3086.01|5184.15|11.75|3069.59|1554.99|195.94|207.69|1750.93|1762.68|-2890.07| +2452248|60861|2452362|14245|95403|1522037|5024|7658|52062|966341|5866|34900|25|21|2|5|208|111|53|12.46|23.54|11.29|649.25|598.37|660.38|1247.62|47.86|0.00|162.18|598.37|646.23|760.55|808.41|-62.01| +2452248|60861|2452363|11263|95403|1522037|5024|7658|52062|966341|5866|34900|59|23|14|3|25|111|99|40.55|66.50|65.17|131.67|6451.83|4014.45|6583.50|63.22|3290.43|723.69|3161.40|3224.62|3885.09|3948.31|-853.05| +2451102|51665|2451114|3932|38145|706220|7095|3217|18735|1616489|5889|28764|20|22|20|3|149|112|77|33.03|36.00|1.80|2633.40|138.60|2543.31|2772.00|5.54|0.00|471.24|138.60|144.14|609.84|615.38|-2404.71| +2451102|51665|2451146|11294|38145|706220|7095|3217|18735|1616489|5889|28764|22|28|8|3|271|112|75|36.50|86.87|83.39|261.00|6254.25|2737.50|6515.25|562.88|0.00|1954.50|6254.25|6817.13|8208.75|8771.63|3516.75| +2451102|51665|2451207|8833|38145|706220|7095|3217|18735|1616489|5889|28764|13|28|6|4|212|112|22|86.95|149.55|0.00|3290.10|0.00|1912.90|3290.10|0.00|0.00|1348.82|0.00|0.00|1348.82|1348.82|-1912.90| +2451102|51665|2451200|9751|38145|706220|7095|3217|18735|1616489|5889|28764|2|28|8|1|181|112|14|23.44|30.94|16.08|208.04|225.12|328.16|433.16|6.75|0.00|0.00|225.12|231.87|225.12|231.87|-103.04| +2451102|51665|2451169|5707|38145|706220|7095|3217|18735|1616489|5889|28764|52|26|18|5|72|112|78|62.85|175.98|170.70|411.84|13314.60|4902.30|13726.44|197.05|3461.79|3568.50|9852.81|10049.86|13421.31|13618.36|4950.51| +2451102|51665|2451181|11576|38145|706220|7095|3217|18735|1616489|5889|28764|49|28|5|3|98|112|69|29.32|77.40|71.98|373.98|4966.62|2023.08|5340.60|71.51|1390.65|2029.29|3575.97|3647.48|5605.26|5676.77|1552.89| +2451102|51665|2451124|14108|38145|706220|7095|3217|18735|1616489|5889|28764|19|19|13|1|206|112|95|84.72|158.42|125.15|3160.65|11889.25|8048.40|15049.90|0.00|0.00|2558.35|11889.25|11889.25|14447.60|14447.60|3840.85| +2451102|51665|2451151|9788|38145|706220|7095|3217|18735|1616489|5889|28764|7|22|3|1|209|112|8|9.21|19.06|9.72|74.72|77.76|73.68|152.48|1.64|36.54|60.96|41.22|42.86|102.18|103.82|-32.46| +2451102|51665|2451130|578|38145|706220|7095|3217|18735|1616489|5889|28764|31|1|19|2|32|112|2|62.33|152.08|48.66|206.84|97.32|124.66|304.16|3.40|29.19|48.66|68.13|71.53|116.79|120.19|-56.53| +2451102|51665|2451189|721|38145|706220|7095|3217|18735|1616489|5889|28764|7|19|6|2|88|112|34|7.80|23.40|14.27|310.42|485.18|265.20|795.60|43.66|0.00|111.18|485.18|528.84|596.36|640.02|219.98| +2451102|51665|2451219|772|38145|706220|7095|3217|18735|1616489|5889|28764|4|8|7|4|94|112|54|18.72|25.64|5.38|1094.04|290.52|1010.88|1384.56|2.90|0.00|262.98|290.52|293.42|553.50|556.40|-720.36| +2451102|51665|2451159|2521|38145|706220|7095|3217|18735|1616489|5889|28764|19|16|18|2|66|112|3|97.95|225.28|148.68|229.80|446.04|293.85|675.84|19.00|129.35|195.99|316.69|335.69|512.68|531.68|22.84| +2451102|51665|2451174|11065|38145|706220|7095|3217|18735|1616489|5889|28764|31|10|6|1|36|112|21|19.69|27.76|26.92|17.64|565.32|413.49|582.96|11.30|0.00|5.67|565.32|576.62|570.99|582.29|151.83| +2451878|54989|2451916|17690|11575|599644|2113|44229|81040|173113|2036|26552|23|14|4|5|93|113|30|85.87|211.24|173.21|1140.90|5196.30|2576.10|6337.20|311.77|0.00|63.30|5196.30|5508.07|5259.60|5571.37|2620.20| +2451878|54989|2451933|6803|11575|599644|2113|44229|81040|173113|2036|26552|29|20|7|3|299|113|29|57.63|154.44|46.33|3135.19|1343.57|1671.27|4478.76|0.00|0.00|1074.74|1343.57|1343.57|2418.31|2418.31|-327.70| +2451878|54989|2451928|2059|11575|599644|2113|44229|81040|173113|2036|26552|38|25|3|3|223|113|54|4.79|10.15|9.54|32.94|515.16|258.66|548.10|1.13|401.82|180.36|113.34|114.47|293.70|294.83|-145.32| +2451878|54989|2451955|2675|11575|599644|2113|44229|81040|173113|2036|26552|49|23|12|2|169|113|6|77.77|174.98|69.99|629.94|419.94|466.62|1049.88|0.25|394.74|0.00|25.20|25.45|25.20|25.45|-441.42| +2451878|54989|2451945|3013|11575|599644|2113|44229|81040|173113|2036|26552|49|13|2|4|59|113|27|96.35|188.84|173.73|407.97|4690.71|2601.45|5098.68|281.44|0.00|1835.46|4690.71|4972.15|6526.17|6807.61|2089.26| +2451878|54989|2451952|17425|11575|599644|2113|44229|81040|173113|2036|26552|49|5|11|1|94|113|7|17.75|44.90|26.94|125.72|188.58|124.25|314.30|11.31|0.00|91.14|188.58|199.89|279.72|291.03|64.33| +2451878|54989|2451996|12950|11575|599644|2113|44229|81040|173113|2036|26552|14|11|14|5|203|113|75|10.67|16.53|10.57|447.00|792.75|800.25|1239.75|63.42|0.00|446.25|792.75|856.17|1239.00|1302.42|-7.50| +2451878|54989|2451950|455|11575|599644|2113|44229|81040|173113|2036|26552|43|14|4|5|296|113|51|96.52|126.44|63.22|3224.22|3224.22|4922.52|6448.44|128.96|0.00|386.58|3224.22|3353.18|3610.80|3739.76|-1698.30| +2451878|54989|2451965|4298|11575|599644|2113|44229|81040|173113|2036|26552|5|1|6|1|279|113|68|3.77|6.10|5.73|25.16|389.64|256.36|414.80|7.79|0.00|207.40|389.64|397.43|597.04|604.83|133.28| +2451878|54989|2451915|7639|11575|599644|2113|44229|81040|173113|2036|26552|29|25|1|1|14|113|54|10.56|30.30|5.75|1325.70|310.50|570.24|1636.20|1.86|124.20|408.78|186.30|188.16|595.08|596.94|-383.94| +2451878|54989|2451959|3359|11575|599644|2113|44229|81040|173113|2036|26552|44|2|14|3|232|113|51|73.77|178.52|7.14|8740.38|364.14|3762.27|9104.52|14.56|0.00|4278.90|364.14|378.70|4643.04|4657.60|-3398.13| +2451878|54989|2451922|14084|11575|599644|2113|44229|81040|173113|2036|26552|26|14|12|1|290|113|38|71.70|204.34|183.90|776.72|6988.20|2724.60|7764.92|209.64|0.00|2795.28|6988.20|7197.84|9783.48|9993.12|4263.60| +2451878|54989|2451937|16837|11575|599644|2113|44229|81040|173113|2036|26552|35|13|6|3|237|113|38|26.48|72.55|68.92|137.94|2618.96|1006.24|2756.90|0.00|497.60|358.34|2121.36|2121.36|2479.70|2479.70|1115.12| +2451878|54989|2451901|10831|11575|599644|2113|44229|81040|173113|2036|26552|17|23|5|3|111|113|81|59.30|116.82|68.92|3879.90|5582.52|4803.30|9462.42|502.42|0.00|283.50|5582.52|6084.94|5866.02|6368.44|779.22| +2452221|67128|2452253|10055|81152|1540093|1787|13481|21177|206842|2257|12351|29|3|2|4|16|114|64|58.78|88.75|6.21|5282.56|397.44|3761.92|5680.00|15.89|0.00|1817.60|397.44|413.33|2215.04|2230.93|-3364.48| +2452221|67128|2452339|5223|81152|1540093|1787|13481|21177|206842|2257|12351|5|1|12|5|131|114|39|88.29|183.64|167.11|644.67|6517.29|3443.31|7161.96|586.55|0.00|644.28|6517.29|7103.84|7161.57|7748.12|3073.98| +2452221|67128|2452313|14329|81152|1540093|1787|13481|21177|206842|2257|12351|25|23|1|2|16|114|21|97.11|247.63|240.20|156.03|5044.20|2039.31|5200.23|252.21|0.00|1715.91|5044.20|5296.41|6760.11|7012.32|3004.89| +2452221|67128|2452307|223|81152|1540093|1787|13481|21177|206842|2257|12351|49|23|9|4|266|114|42|82.60|246.97|22.22|9439.50|933.24|3469.20|10372.74|1.95|867.91|3215.52|65.33|67.28|3280.85|3282.80|-3403.87| +2452221|67128|2452243|12413|81152|1540093|1787|13481|21177|206842|2257|12351|21|5|5|2|2|114|35|67.82|185.82|65.03|4227.65|2276.05|2373.70|6503.70|182.08|0.00|780.15|2276.05|2458.13|3056.20|3238.28|-97.65| +2452221|67128|2452230|11599|81152|1540093|1787|13481|21177|206842|2257|12351|5|19|7|4|61|114|67|35.04|46.60|23.30|1561.10|1561.10|2347.68|3122.20|15.61|0.00|936.66|1561.10|1576.71|2497.76|2513.37|-786.58| +2452221|67128|2452297|8017|81152|1540093|1787|13481|21177|206842|2257|12351|49|25|11|4|120|114|61|22.02|42.05|9.67|1975.18|589.87|1343.22|2565.05|29.49|0.00|307.44|589.87|619.36|897.31|926.80|-753.35| +2452221|67128|2452271|2833|81152|1540093|1787|13481|21177|206842|2257|12351|33|13|17|2|157|114|23|69.64|114.20|91.36|525.32|2101.28|1601.72|2626.60|105.06|0.00|603.98|2101.28|2206.34|2705.26|2810.32|499.56| +2452221|67128|2452275|11175|81152|1540093|1787|13481|21177|206842|2257|12351|3|1|13|2|164|114|34|83.22|182.25|134.86|1611.26|4585.24|2829.48|6196.50|0.00|0.00|1053.32|4585.24|4585.24|5638.56|5638.56|1755.76| +2452221|67128|2452341|611|81152|1540093|1787|13481|21177|206842|2257|12351|57|19|4|5|6|114|5|53.98|121.99|50.01|359.90|250.05|269.90|609.95|22.50|0.00|134.15|250.05|272.55|384.20|406.70|-19.85| +2452221|67128|2452226|6647|81152|1540093|1787|13481|21177|206842|2257|12351|11|13|5|1|77|114|12|69.88|79.66|43.81|430.20|525.72|838.56|955.92|42.05|0.00|248.52|525.72|567.77|774.24|816.29|-312.84| +2452221|67128|2452234|5085|81152|1540093|1787|13481|21177|206842|2257|12351|25|13|1|5|199|114|49|54.72|154.31|138.87|756.56|6804.63|2681.28|7561.19|476.32|0.00|3478.02|6804.63|7280.95|10282.65|10758.97|4123.35| +2452221|67128|2452228|14537|81152|1540093|1787|13481|21177|206842|2257|12351|43|3|9|4|136|114|63|73.81|200.76|68.25|8348.13|4299.75|4650.03|12647.88|343.98|0.00|3667.86|4299.75|4643.73|7967.61|8311.59|-350.28| +2452221|67128|2452310|11373|81152|1540093|1787|13481|21177|206842|2257|12351|39|11|5|1|160|114|80|6.68|7.94|6.74|96.00|539.20|534.40|635.20|43.13|0.00|113.60|539.20|582.33|652.80|695.93|4.80| +2452221|67128|2452233|9291|81152|1540093|1787|13481|21177|206842|2257|12351|27|15|12|4|283|114|25|31.10|53.49|14.44|976.25|361.00|777.50|1337.25|0.00|0.00|173.75|361.00|361.00|534.75|534.75|-416.50| +2452221|67128|2452297|10619|81152|1540093|1787|13481|21177|206842|2257|12351|29|9|8|3|238|114|33|5.29|10.47|5.65|159.06|186.45|174.57|345.51|9.24|70.85|124.08|115.60|124.84|239.68|248.92|-58.97| +2451151|15600|2451221|11689|6582|1340820|5458|13649|79843|1871584|3563|34601|1|7|7|2|50|115|37|96.20|169.31|145.60|877.27|5387.20|3559.40|6264.47|215.48|0.00|62.53|5387.20|5602.68|5449.73|5665.21|1827.80| +2451151|15600|2451253|15208|6582|1340820|5458|13649|79843|1871584|3563|34601|31|28|4|5|289|115|92|71.31|110.53|51.94|5390.28|4778.48|6560.52|10168.76|382.27|0.00|406.64|4778.48|5160.75|5185.12|5567.39|-1782.04| +2451151|15600|2451257|4948|6582|1340820|5458|13649|79843|1871584|3563|34601|1|16|20|3|49|115|52|59.19|102.99|4.11|5141.76|213.72|3077.88|5355.48|6.41|0.00|2249.00|213.72|220.13|2462.72|2469.13|-2864.16| +2451151|15600|2451257|13024|6582|1340820|5458|13649|79843|1871584|3563|34601|50|10|6|1|276|115|49|55.13|135.06|90.49|2183.93|4434.01|2701.37|6617.94|177.36|0.00|727.65|4434.01|4611.37|5161.66|5339.02|1732.64| +2451151|15600|2451181|13054|6582|1340820|5458|13649|79843|1871584|3563|34601|26|20|8|2|159|115|6|49.20|126.44|69.54|341.40|417.24|295.20|758.64|25.03|0.00|22.74|417.24|442.27|439.98|465.01|122.04| +2451151|15600|2451270|1378|6582|1340820|5458|13649|79843|1871584|3563|34601|10|28|14|3|282|115|15|8.04|18.09|14.83|48.90|222.45|120.60|271.35|4.87|60.06|40.65|162.39|167.26|203.04|207.91|41.79| +2451151|15600|2451211|10928|6582|1340820|5458|13649|79843|1871584|3563|34601|40|25|8|3|5|115|49|74.34|155.37|138.27|837.90|6775.23|3642.66|7613.13|406.51|0.00|3121.30|6775.23|7181.74|9896.53|10303.04|3132.57| +2451151|15600|2451262|1732|6582|1340820|5458|13649|79843|1871584|3563|34601|16|14|14|3|12|115|51|25.73|72.30|29.64|2175.66|1511.64|1312.23|3687.30|75.58|0.00|1732.98|1511.64|1587.22|3244.62|3320.20|199.41| +2451151|15600|2451162|17470|6582|1340820|5458|13649|79843|1871584|3563|34601|46|26|4|4|104|115|9|43.24|79.12|46.68|291.96|420.12|389.16|712.08|4.20|0.00|291.87|420.12|424.32|711.99|716.19|30.96| +2451151|15600|2451209|658|6582|1340820|5458|13649|79843|1871584|3563|34601|28|1|17|3|109|115|21|48.83|105.96|90.06|333.90|1891.26|1025.43|2225.16|0.00|0.00|600.60|1891.26|1891.26|2491.86|2491.86|865.83| +2451151|15600|2451235|13201|6582|1340820|5458|13649|79843|1871584|3563|34601|58|26|18|2|176|115|77|84.71|223.63|169.95|4133.36|13086.15|6522.67|17219.51|467.17|6412.21|8092.70|6673.94|7141.11|14766.64|15233.81|151.27| +2451151|15600|2451175|14191|6582|1340820|5458|13649|79843|1871584|3563|34601|1|19|19|5|44|115|53|16.50|37.95|7.96|1589.47|421.88|874.50|2011.35|0.00|206.72|160.59|215.16|215.16|375.75|375.75|-659.34| +2451151|15600|2451186|11839|6582|1340820|5458|13649|79843|1871584|3563|34601|43|20|10|4|188|115|74|95.25|222.88|173.84|3628.96|12864.16|7048.50|16493.12|257.28|0.00|5442.70|12864.16|13121.44|18306.86|18564.14|5815.66| +2451151|15600|2451163|13180|6582|1340820|5458|13649|79843|1871584|3563|34601|52|14|2|3|177|115|31|96.58|114.93|108.03|213.90|3348.93|2993.98|3562.83|66.97|0.00|463.14|3348.93|3415.90|3812.07|3879.04|354.95| +2451151|15600|2451193|1960|6582|1340820|5458|13649|79843|1871584|3563|34601|34|19|9|3|67|115|9|5.05|12.97|10.50|22.23|94.50|45.45|116.73|7.56|0.00|17.46|94.50|102.06|111.96|119.52|49.05| +2451151|15600|2451262|2864|6582|1340820|5458|13649|79843|1871584|3563|34601|13|1|5|4|6|115|81|79.30|184.76|92.38|7482.78|7482.78|6423.30|14965.56|673.45|0.00|3591.54|7482.78|8156.23|11074.32|11747.77|1059.48| +2451547|19764|2451663|8971|83106|946511|7069|19627|71775|1898902|6548|40753|44|1|19|4|182|116|77|18.20|38.58|0.77|2911.37|59.29|1401.40|2970.66|1.54|42.09|890.89|17.20|18.74|908.09|909.63|-1384.20| +2451547|19764|2451636|15752|83106|946511|7069|19627|71775|1898902|6548|40753|53|13|20|3|23|116|5|94.88|268.51|220.17|241.70|1100.85|474.40|1342.55|36.54|187.14|416.15|913.71|950.25|1329.86|1366.40|439.31| +2451547|19764|2451566|1777|83106|946511|7069|19627|71775|1898902|6548|40753|35|1|11|1|114|116|92|85.47|113.67|0.00|10457.64|0.00|7863.24|10457.64|0.00|0.00|1777.44|0.00|0.00|1777.44|1777.44|-7863.24| +2451547|19764|2451664|65|83106|946511|7069|19627|71775|1898902|6548|40753|25|1|19|2|205|116|4|13.74|32.15|18.96|52.76|75.84|54.96|128.60|3.03|0.00|52.72|75.84|78.87|128.56|131.59|20.88| +2451547|19764|2451638|16640|83106|946511|7069|19627|71775|1898902|6548|40753|29|29|5|5|172|116|67|10.23|24.04|5.28|1256.92|353.76|685.41|1610.68|0.00|0.00|353.76|353.76|353.76|707.52|707.52|-331.65| +2451547|19764|2451631|2282|83106|946511|7069|19627|71775|1898902|6548|40753|44|29|4|1|215|116|59|46.24|79.53|73.96|328.63|4363.64|2728.16|4692.27|174.54|0.00|421.85|4363.64|4538.18|4785.49|4960.03|1635.48| +2451547|19764|2451555|12542|83106|946511|7069|19627|71775|1898902|6548|40753|20|23|13|4|135|116|91|70.09|84.80|13.56|6482.84|1233.96|6378.19|7716.80|111.05|0.00|3472.56|1233.96|1345.01|4706.52|4817.57|-5144.23| +2451547|19764|2451569|769|83106|946511|7069|19627|71775|1898902|6548|40753|11|29|11|5|289|116|81|25.55|73.07|27.76|3670.11|2248.56|2069.55|5918.67|202.37|0.00|2722.41|2248.56|2450.93|4970.97|5173.34|179.01| +2451547|19764|2451582|662|83106|946511|7069|19627|71775|1898902|6548|40753|32|5|12|3|217|116|17|71.80|142.16|98.09|749.19|1667.53|1220.60|2416.72|66.70|0.00|990.76|1667.53|1734.23|2658.29|2724.99|446.93| +2451547|19764|2451574|2222|83106|946511|7069|19627|71775|1898902|6548|40753|23|14|17|3|222|116|5|22.73|28.41|5.39|115.10|26.95|113.65|142.05|0.00|26.95|55.35|0.00|0.00|55.35|55.35|-113.65| +2451547|19764|2451622|10193|83106|946511|7069|19627|71775|1898902|6548|40753|7|2|10|1|19|116|68|58.31|72.88|40.81|2180.76|2775.08|3965.08|4955.84|23.31|2442.07|2477.92|333.01|356.32|2810.93|2834.24|-3632.07| +2451547|19764|2451632|3349|83106|946511|7069|19627|71775|1898902|6548|40753|56|13|2|5|218|116|11|67.39|139.49|37.66|1120.13|414.26|741.29|1534.39|4.14|0.00|751.85|414.26|418.40|1166.11|1170.25|-327.03| +2451547|19764|2451567|8795|83106|946511|7069|19627|71775|1898902|6548|40753|35|11|11|4|134|116|10|91.13|268.83|29.57|2392.60|295.70|911.30|2688.30|2.95|0.00|860.20|295.70|298.65|1155.90|1158.85|-615.60| +2451547|19764|2451564|6914|83106|946511|7069|19627|71775|1898902|6548|40753|19|29|7|4|261|116|92|10.89|11.21|8.74|227.24|804.08|1001.88|1031.32|24.12|0.00|370.76|804.08|828.20|1174.84|1198.96|-197.80| +2451547|19764|2451554|16121|83106|946511|7069|19627|71775|1898902|6548|40753|29|14|14|2|93|116|54|5.98|6.21|4.16|110.70|224.64|322.92|335.34|0.00|0.00|56.70|224.64|224.64|281.34|281.34|-98.28| +2451317|66696|2451406|2797|61015|1648533|2898|35663|37271|1401607|1064|29429|22|2|2|2|242|117|75|86.29|123.39|49.35|5553.00|3701.25|6471.75|9254.25|333.11|0.00|3423.75|3701.25|4034.36|7125.00|7458.11|-2770.50| +2451317|66696|2451347|14938|61015|1648533|2898|35663|37271|1401607|1064|29429|34|22|9|1|160|117|40|8.69|16.07|9.32|270.00|372.80|347.60|642.80|14.91|0.00|314.80|372.80|387.71|687.60|702.51|25.20| +2451317|66696|2451337|4669|61015|1648533|2898|35663|37271|1401607|1064|29429|1|19|12|4|143|117|67|93.18|127.65|114.88|855.59|7696.96|6243.06|8552.55|615.75|0.00|1368.14|7696.96|8312.71|9065.10|9680.85|1453.90| +2451317|66696|2451322|14833|61015|1648533|2898|35663|37271|1401607|1064|29429|31|28|14|3|162|117|78|92.26|261.09|46.99|16699.80|3665.22|7196.28|20365.02|293.21|0.00|4683.90|3665.22|3958.43|8349.12|8642.33|-3531.06| +2451317|66696|2451378|17842|61015|1648533|2898|35663|37271|1401607|1064|29429|16|8|4|5|219|117|45|56.17|106.72|26.68|3601.80|1200.60|2527.65|4802.40|5.88|612.30|1776.60|588.30|594.18|2364.90|2370.78|-1939.35| +2451317|66696|2451418|6958|61015|1648533|2898|35663|37271|1401607|1064|29429|2|28|1|2|280|117|20|8.49|21.47|12.88|171.80|257.60|169.80|429.40|23.18|0.00|128.80|257.60|280.78|386.40|409.58|87.80| +2451317|66696|2451386|11869|61015|1648533|2898|35663|37271|1401607|1064|29429|8|28|15|5|255|117|34|54.02|64.28|16.71|1617.38|568.14|1836.68|2185.52|22.72|0.00|393.38|568.14|590.86|961.52|984.24|-1268.54| +2451317|66696|2451395|7513|61015|1648533|2898|35663|37271|1401607|1064|29429|1|28|19|3|147|117|60|30.03|34.83|31.34|209.40|1880.40|1801.80|2089.80|75.21|0.00|647.40|1880.40|1955.61|2527.80|2603.01|78.60| +2451317|66696|2451337|9926|61015|1648533|2898|35663|37271|1401607|1064|29429|31|26|2|5|98|117|64|90.00|233.10|9.32|14321.92|596.48|5760.00|14918.40|0.00|0.00|894.72|596.48|596.48|1491.20|1491.20|-5163.52| +2451317|66696|2451434|8744|61015|1648533|2898|35663|37271|1401607|1064|29429|32|1|18|2|96|117|91|79.73|207.29|93.28|10374.91|8488.48|7255.43|18863.39|169.76|0.00|754.39|8488.48|8658.24|9242.87|9412.63|1233.05| +2451317|66696|2451425|6530|61015|1648533|2898|35663|37271|1401607|1064|29429|40|4|2|1|180|117|35|19.83|33.90|29.49|154.35|1032.15|694.05|1186.50|0.00|0.00|308.35|1032.15|1032.15|1340.50|1340.50|338.10| +2451317|66696|2451366|17455|61015|1648533|2898|35663|37271|1401607|1064|29429|31|4|20|2|121|117|39|33.25|81.13|18.65|2436.72|727.35|1296.75|3164.07|35.13|225.47|1265.55|501.88|537.01|1767.43|1802.56|-794.87| +2451317|66696|2451409|793|61015|1648533|2898|35663|37271|1401607|1064|29429|49|19|5|5|235|117|60|27.17|69.01|11.04|3478.20|662.40|1630.20|4140.60|59.61|0.00|207.00|662.40|722.01|869.40|929.01|-967.80| +2451317|66696|2451358|362|61015|1648533|2898|35663|37271|1401607|1064|29429|28|8|15|2|162|117|51|91.92|204.98|135.28|3554.70|6899.28|4687.92|10453.98|68.99|0.00|1358.64|6899.28|6968.27|8257.92|8326.91|2211.36| +2451870|79151|2451946|9835|4226|1085607|1388|46322|75166|750026|1665|30646|35|26|17|3|46|118|14|11.98|23.48|4.69|263.06|65.66|167.72|328.72|1.31|0.00|52.50|65.66|66.97|118.16|119.47|-102.06| +2451870|79151|2451929|16157|4226|1085607|1388|46322|75166|750026|1665|30646|35|7|10|1|292|118|6|79.47|185.95|130.16|334.74|780.96|476.82|1115.70|23.42|0.00|301.20|780.96|804.38|1082.16|1105.58|304.14| +2451870|79151|2451953|8821|4226|1085607|1388|46322|75166|750026|1665|30646|55|25|5|5|192|118|15|71.71|206.52|99.12|1611.00|1486.80|1075.65|3097.80|3.12|1442.19|619.50|44.61|47.73|664.11|667.23|-1031.04| +2451870|79151|2451923|6686|4226|1085607|1388|46322|75166|750026|1665|30646|38|19|20|5|220|118|100|97.11|200.04|120.02|8002.00|12002.00|9711.00|20004.00|600.10|0.00|800.00|12002.00|12602.10|12802.00|13402.10|2291.00| +2451870|79151|2451875|98|4226|1085607|1388|46322|75166|750026|1665|30646|20|13|16|2|168|118|8|11.00|15.07|6.32|70.00|50.56|88.00|120.56|3.53|0.00|1.20|50.56|54.09|51.76|55.29|-37.44| +2451870|79151|2451934|38|4226|1085607|1388|46322|75166|750026|1665|30646|29|8|5|5|282|118|43|91.73|185.29|131.55|2310.82|5656.65|3944.39|7967.47|0.00|0.00|796.36|5656.65|5656.65|6453.01|6453.01|1712.26| +2451870|79151|2451904|1871|4226|1085607|1388|46322|75166|750026|1665|30646|41|1|1|1|248|118|54|38.71|43.35|22.10|1147.50|1193.40|2090.34|2340.90|83.53|0.00|351.00|1193.40|1276.93|1544.40|1627.93|-896.94| +2451870|79151|2451938|2875|4226|1085607|1388|46322|75166|750026|1665|30646|56|8|8|4|210|118|96|96.34|147.40|48.64|9480.96|4669.44|9248.64|14150.40|140.08|0.00|3395.52|4669.44|4809.52|8064.96|8205.04|-4579.20| +2451870|79151|2451989|10195|4226|1085607|1388|46322|75166|750026|1665|30646|59|25|3|1|89|118|87|8.21|12.47|1.12|987.45|97.44|714.27|1084.89|6.82|0.00|466.32|97.44|104.26|563.76|570.58|-616.83| +2451870|79151|2451964|8318|4226|1085607|1388|46322|75166|750026|1665|30646|32|7|11|3|235|118|46|63.99|181.09|52.51|5914.68|2415.46|2943.54|8330.14|24.15|0.00|2915.48|2415.46|2439.61|5330.94|5355.09|-528.08| +2452461|65711|2452466|15306|67606|714813|6373|5933|51134|21027|7013|12173|60|18|11|5|262|119|12|58.95|176.85|56.59|1443.12|679.08|707.40|2122.20|9.37|522.89|148.44|156.19|165.56|304.63|314.00|-551.21| +2452461|65711|2452553|4584|67606|714813|6373|5933|51134|21027|7013|12173|9|21|14|2|110|119|15|15.18|44.32|41.66|39.90|624.90|227.70|664.80|43.74|0.00|272.55|624.90|668.64|897.45|941.19|397.20| +2452461|65711|2452515|1077|67606|714813|6373|5933|51134|21027|7013|12173|18|27|19|2|22|119|10|44.33|121.02|67.77|532.50|677.70|443.30|1210.20|60.99|0.00|254.10|677.70|738.69|931.80|992.79|234.40| +2452461|65711|2452499|15042|67606|714813|6373|5933|51134|21027|7013|12173|13|15|6|4|235|119|92|44.23|122.95|73.77|4524.56|6786.84|4069.16|11311.40|407.21|0.00|4524.56|6786.84|7194.05|11311.40|11718.61|2717.68| +2452461|65711|2452579|11179|67606|714813|6373|5933|51134|21027|7013|12173|6|30|4|1|245|119|63|8.67|13.61|1.90|737.73|119.70|546.21|857.43|3.59|0.00|274.05|119.70|123.29|393.75|397.34|-426.51| +2452461|65711|2452538|348|67606|714813|6373|5933|51134|21027|7013|12173|43|27|19|1|6|119|14|27.17|51.35|15.91|496.16|222.74|380.38|718.90|0.00|0.00|323.40|222.74|222.74|546.14|546.14|-157.64| +2452461|65711|2452465|14103|67606|714813|6373|5933|51134|21027|7013|12173|6|9|7|4|184|119|91|66.58|120.50|12.05|9868.95|1096.55|6058.78|10965.50|10.74|943.03|2082.99|153.52|164.26|2236.51|2247.25|-5905.26| +2452461|65711|2452580|3006|67606|714813|6373|5933|51134|21027|7013|12173|36|7|10|4|165|119|72|7.92|11.88|0.83|795.60|59.76|570.24|855.36|0.59|0.00|0.00|59.76|60.35|59.76|60.35|-510.48| +2452461|65711|2452465|12469|67606|714813|6373|5933|51134|21027|7013|12173|54|9|10|5|110|119|48|87.82|99.23|38.69|2905.92|1857.12|4215.36|4763.04|148.56|0.00|142.56|1857.12|2005.68|1999.68|2148.24|-2358.24| +2452461|65711|2452515|8575|67606|714813|6373|5933|51134|21027|7013|12173|25|9|17|2|214|119|37|95.91|117.01|94.77|822.88|3506.49|3548.67|4329.37|13.32|2174.02|2164.50|1332.47|1345.79|3496.97|3510.29|-2216.20| +2452461|65711|2452488|3564|67606|714813|6373|5933|51134|21027|7013|12173|43|1|13|2|231|119|76|71.78|203.13|154.37|3705.76|11732.12|5455.28|15437.88|469.28|0.00|4630.68|11732.12|12201.40|16362.80|16832.08|6276.84| +2452461|65711|2452525|9228|67606|714813|6373|5933|51134|21027|7013|12173|51|1|16|4|219|119|37|70.41|207.00|82.80|4595.40|3063.60|2605.17|7659.00|183.81|0.00|995.67|3063.60|3247.41|4059.27|4243.08|458.43| +2452461|65711|2452575|16519|67606|714813|6373|5933|51134|21027|7013|12173|19|13|17|4|21|119|63|97.04|284.32|51.17|14688.45|3223.71|6113.52|17912.16|0.00|0.00|357.84|3223.71|3223.71|3581.55|3581.55|-2889.81| +2451979|81904|2451991|12061|50537|854361|3432|42903|18273|1387425|516|43940|25|5|16|4|19|120|52|17.72|49.43|48.44|51.48|2518.88|921.44|2570.36|125.94|0.00|76.96|2518.88|2644.82|2595.84|2721.78|1597.44| +2451979|81904|2452055|9157|50537|854361|3432|42903|18273|1387425|516|43940|29|9|15|3|118|120|74|85.41|229.75|165.42|4760.42|12241.08|6320.34|17001.50|734.46|0.00|8330.18|12241.08|12975.54|20571.26|21305.72|5920.74| +2451979|81904|2452056|14811|50537|854361|3432|42903|18273|1387425|516|43940|3|1|3|2|104|120|86|27.37|37.77|27.94|845.38|2402.84|2353.82|3248.22|56.22|1778.10|324.22|624.74|680.96|948.96|1005.18|-1729.08| +2451979|81904|2451990|8697|50537|854361|3432|42903|18273|1387425|516|43940|37|3|15|5|138|120|48|17.33|24.43|0.48|1149.60|23.04|831.84|1172.64|0.69|0.00|363.36|23.04|23.73|386.40|387.09|-808.80| +2451979|81904|2452090|15917|50537|854361|3432|42903|18273|1387425|516|43940|23|5|11|2|73|120|52|16.87|44.19|29.60|758.68|1539.20|877.24|2297.88|0.00|0.00|459.16|1539.20|1539.20|1998.36|1998.36|661.96| +2451979|81904|2452042|14787|50537|854361|3432|42903|18273|1387425|516|43940|37|7|2|2|12|120|98|66.81|89.52|14.32|7369.60|1403.36|6547.38|8772.96|14.03|0.00|2455.88|1403.36|1417.39|3859.24|3873.27|-5144.02| +2451979|81904|2451989|4625|50537|854361|3432|42903|18273|1387425|516|43940|9|19|12|2|62|120|60|77.03|221.84|37.71|11047.80|2262.60|4621.80|13310.40|22.62|0.00|532.20|2262.60|2285.22|2794.80|2817.42|-2359.20| +2451979|81904|2452090|4381|50537|854361|3432|42903|18273|1387425|516|43940|47|7|4|4|117|120|94|49.72|112.86|51.91|5729.30|4879.54|4673.68|10608.84|48.79|0.00|5198.20|4879.54|4928.33|10077.74|10126.53|205.86| +2452131|5091|2452235|7107|84173|884481|5589|23897|73770|532206|2821|43529|53|9|13|1|89|121|14|67.59|181.81|125.44|789.18|1756.16|946.26|2545.34|17.56|0.00|1196.30|1756.16|1773.72|2952.46|2970.02|809.90| +2452131|5091|2452145|9677|84173|884481|5589|23897|73770|532206|2821|43529|35|21|4|2|290|121|39|81.77|191.34|40.18|5895.24|1567.02|3189.03|7462.26|62.68|0.00|596.70|1567.02|1629.70|2163.72|2226.40|-1622.01| +2452131|5091|2452140|11847|84173|884481|5589|23897|73770|532206|2821|43529|53|15|9|2|138|121|68|45.75|98.36|75.73|1538.84|5149.64|3111.00|6688.48|0.00|3038.28|1136.96|2111.36|2111.36|3248.32|3248.32|-999.64| +2452131|5091|2452229|10585|84173|884481|5589|23897|73770|532206|2821|43529|37|25|10|2|274|121|93|68.38|136.07|16.32|11136.75|1517.76|6359.34|12654.51|121.42|0.00|3036.45|1517.76|1639.18|4554.21|4675.63|-4841.58| +2452131|5091|2452135|17351|84173|884481|5589|23897|73770|532206|2821|43529|59|11|19|2|195|121|4|72.91|95.51|42.02|213.96|168.08|291.64|382.04|6.21|43.70|118.40|124.38|130.59|242.78|248.99|-167.26| +2452131|5091|2452133|11725|84173|884481|5589|23897|73770|532206|2821|43529|3|15|14|5|266|121|93|51.11|69.50|11.12|5429.34|1034.16|4753.23|6463.50|82.73|0.00|1034.16|1034.16|1116.89|2068.32|2151.05|-3719.07| +2452131|5091|2452214|8505|84173|884481|5589|23897|73770|532206|2821|43529|5|25|3|3|103|121|91|73.32|122.44|52.64|6351.80|4790.24|6672.12|11142.04|383.21|0.00|1001.91|4790.24|5173.45|5792.15|6175.36|-1881.88| +2452131|5091|2452154|10025|84173|884481|5589|23897|73770|532206|2821|43529|51|5|13|5|107|121|87|96.30|269.64|115.94|13371.90|10086.78|8378.10|23458.68|504.33|0.00|5394.87|10086.78|10591.11|15481.65|15985.98|1708.68| +2452131|5091|2452182|601|84173|884481|5589|23897|73770|532206|2821|43529|11|15|13|5|67|121|83|48.93|145.32|14.53|10855.57|1205.99|4061.19|12061.56|48.23|0.00|2532.33|1205.99|1254.22|3738.32|3786.55|-2855.20| +2451053|33607|2451171|2194|39352|1220992|5385|20625|6826|981636|4298|39891|32|4|9|3|224|122|28|2.57|4.00|3.36|17.92|94.08|71.96|112.00|2.70|60.21|41.44|33.87|36.57|75.31|78.01|-38.09| +2451053|33607|2451054|12055|39352|1220992|5385|20625|6826|981636|4298|39891|8|20|10|4|98|122|43|14.31|41.06|18.06|989.00|776.58|615.33|1765.58|7.76|0.00|794.21|776.58|784.34|1570.79|1578.55|161.25| +2451053|33607|2451125|14737|39352|1220992|5385|20625|6826|981636|4298|39891|50|2|18|1|207|122|80|34.63|48.48|12.60|2870.40|1008.00|2770.40|3878.40|80.64|0.00|775.20|1008.00|1088.64|1783.20|1863.84|-1762.40| +2451053|33607|2451117|8822|39352|1220992|5385|20625|6826|981636|4298|39891|44|22|11|5|182|122|9|6.26|16.90|14.19|24.39|127.71|56.34|152.10|2.55|0.00|59.31|127.71|130.26|187.02|189.57|71.37| +2451053|33607|2451101|13363|39352|1220992|5385|20625|6826|981636|4298|39891|10|26|19|2|262|122|81|93.90|246.95|133.35|9201.60|10801.35|7605.90|20002.95|216.02|0.00|9200.79|10801.35|11017.37|20002.14|20218.16|3195.45| +2451053|33607|2451085|10351|39352|1220992|5385|20625|6826|981636|4298|39891|28|20|17|3|59|122|59|20.05|32.48|19.81|747.53|1168.79|1182.95|1916.32|23.37|0.00|823.64|1168.79|1192.16|1992.43|2015.80|-14.16| +2451053|33607|2451154|9733|39352|1220992|5385|20625|6826|981636|4298|39891|44|10|9|5|248|122|98|97.33|115.82|63.70|5107.76|6242.60|9538.34|11350.36|561.83|0.00|3290.84|6242.60|6804.43|9533.44|10095.27|-3295.74| +2451053|33607|2451074|2584|39352|1220992|5385|20625|6826|981636|4298|39891|16|26|9|1|216|122|63|99.08|177.35|136.55|2570.40|8602.65|6242.04|11173.05|0.00|0.00|5586.21|8602.65|8602.65|14188.86|14188.86|2360.61| +2451053|33607|2451133|302|39352|1220992|5385|20625|6826|981636|4298|39891|7|26|17|5|300|122|98|64.44|119.85|20.37|9749.04|1996.26|6315.12|11745.30|39.92|0.00|4932.34|1996.26|2036.18|6928.60|6968.52|-4318.86| +2451053|33607|2451151|12247|39352|1220992|5385|20625|6826|981636|4298|39891|8|25|8|1|254|122|81|40.80|115.46|55.42|4863.24|4489.02|3304.80|9352.26|359.12|0.00|934.74|4489.02|4848.14|5423.76|5782.88|1184.22| +2451053|33607|2451138|4738|39352|1220992|5385|20625|6826|981636|4298|39891|7|19|16|5|51|122|60|90.07|165.72|79.54|5170.80|4772.40|5404.20|9943.20|95.44|0.00|3976.80|4772.40|4867.84|8749.20|8844.64|-631.80| +2451053|33607|2451082|13304|39352|1220992|5385|20625|6826|981636|4298|39891|22|2|2|2|32|122|82|13.66|19.39|13.37|493.64|1096.34|1120.12|1589.98|21.92|0.00|364.90|1096.34|1118.26|1461.24|1483.16|-23.78| +2452429|31055|2452504|10407|41018|354220|6622|33024|85953|1616450|4230|39032|1|24|10|3|246|123|47|99.62|268.97|142.55|5941.74|6699.85|4682.14|12641.59|359.11|2210.95|505.25|4488.90|4848.01|4994.15|5353.26|-193.24| +2452429|31055|2452499|15171|41018|354220|6622|33024|85953|1616450|4230|39032|42|9|19|3|77|123|56|98.90|148.35|5.93|7975.52|332.08|5538.40|8307.60|23.24|0.00|3821.44|332.08|355.32|4153.52|4176.76|-5206.32| +2452429|31055|2452541|3642|41018|354220|6622|33024|85953|1616450|4230|39032|18|9|14|3|92|123|60|62.95|124.64|18.69|6357.00|1121.40|3777.00|7478.40|100.92|0.00|3514.80|1121.40|1222.32|4636.20|4737.12|-2655.60| +2452429|31055|2452521|4827|41018|354220|6622|33024|85953|1616450|4230|39032|21|1|8|5|165|123|93|41.64|69.53|20.16|4591.41|1874.88|3872.52|6466.29|149.99|0.00|2974.14|1874.88|2024.87|4849.02|4999.01|-1997.64| +2452429|31055|2452440|6867|41018|354220|6622|33024|85953|1616450|4230|39032|3|3|19|4|69|123|99|69.84|203.93|87.68|11508.75|8680.32|6914.16|20189.07|86.80|0.00|1412.73|8680.32|8767.12|10093.05|10179.85|1766.16| +2452429|31055|2452467|783|41018|354220|6622|33024|85953|1616450|4230|39032|43|25|4|5|46|123|97|13.60|20.80|8.94|1150.42|867.18|1319.20|2017.60|16.12|598.35|241.53|268.83|284.95|510.36|526.48|-1050.37| +2452429|31055|2452524|13969|41018|354220|6622|33024|85953|1616450|4230|39032|54|3|20|5|123|123|77|76.71|182.56|16.43|12792.01|1265.11|5906.67|14057.12|25.30|0.00|843.15|1265.11|1290.41|2108.26|2133.56|-4641.56| +2452429|31055|2452474|3240|41018|354220|6622|33024|85953|1616450|4230|39032|51|13|5|1|17|123|86|57.01|151.64|54.59|8346.30|4694.74|4902.86|13041.04|93.89|0.00|129.86|4694.74|4788.63|4824.60|4918.49|-208.12| +2452429|31055|2452483|16521|41018|354220|6622|33024|85953|1616450|4230|39032|43|7|13|5|294|123|40|82.86|216.26|179.49|1470.80|7179.60|3314.40|8650.40|287.18|0.00|432.40|7179.60|7466.78|7612.00|7899.18|3865.20| +2452429|31055|2452468|3675|41018|354220|6622|33024|85953|1616450|4230|39032|19|9|20|5|267|123|100|31.86|95.26|35.24|6002.00|3524.00|3186.00|9526.00|140.96|0.00|1809.00|3524.00|3664.96|5333.00|5473.96|338.00| +2452429|31055|2452514|8334|41018|354220|6622|33024|85953|1616450|4230|39032|49|19|5|5|295|123|98|92.39|215.26|210.95|422.38|20673.10|9054.22|21095.48|0.00|20673.10|6117.16|0.00|0.00|6117.16|6117.16|-9054.22| +2452429|31055|2452545|16315|41018|354220|6622|33024|85953|1616450|4230|39032|33|12|8|5|199|123|35|61.86|153.41|128.86|859.25|4510.10|2165.10|5369.35|45.10|0.00|2577.05|4510.10|4555.20|7087.15|7132.25|2345.00| +2452429|31055|2452537|17239|41018|354220|6622|33024|85953|1616450|4230|39032|42|25|5|2|169|123|78|33.87|95.85|12.46|6504.42|971.88|2641.86|7476.30|18.27|515.09|74.10|456.79|475.06|530.89|549.16|-2185.07| +2452429|31055|2452495|14479|41018|354220|6622|33024|85953|1616450|4230|39032|12|12|6|1|66|123|72|68.35|114.14|52.50|4438.08|3780.00|4921.20|8218.08|37.80|0.00|3368.88|3780.00|3817.80|7148.88|7186.68|-1141.20| +2452429|31055|2452536|10893|41018|354220|6622|33024|85953|1616450|4230|39032|30|12|14|5|23|123|26|27.92|34.06|31.33|70.98|814.58|725.92|885.56|1.95|716.83|0.00|97.75|99.70|97.75|99.70|-628.17| +2451649|76443|2451717|8252|45329|493247|1612|24730|30670|1322511|6884|36049|14|20|17|5|43|124|81|96.71|107.34|50.44|4608.90|4085.64|7833.51|8694.54|81.71|0.00|2955.69|4085.64|4167.35|7041.33|7123.04|-3747.87| +2451649|76443|2451734|15266|45329|493247|1612|24730|30670|1322511|6884|36049|14|19|15|2|159|124|78|81.73|141.39|128.66|992.94|10035.48|6374.94|11028.42|642.27|2007.09|4190.16|8028.39|8670.66|12218.55|12860.82|1653.45| +2451649|76443|2451753|5378|45329|493247|1612|24730|30670|1322511|6884|36049|25|29|7|2|14|124|71|70.31|119.52|107.56|849.16|7636.76|4992.01|8485.92|534.57|0.00|3733.18|7636.76|8171.33|11369.94|11904.51|2644.75| +2451649|76443|2451684|13799|45329|493247|1612|24730|30670|1322511|6884|36049|32|20|12|1|37|124|19|23.15|58.10|51.12|132.62|971.28|439.85|1103.90|19.42|0.00|253.84|971.28|990.70|1225.12|1244.54|531.43| +2451649|76443|2451742|7633|45329|493247|1612|24730|30670|1322511|6884|36049|49|23|9|4|266|124|33|2.29|5.99|2.69|108.90|88.77|75.57|197.67|0.00|0.00|17.49|88.77|88.77|106.26|106.26|13.20| +2451649|76443|2451769|7721|45329|493247|1612|24730|30670|1322511|6884|36049|53|8|10|3|216|124|35|48.70|85.22|27.27|2028.25|954.45|1704.50|2982.70|19.08|0.00|268.10|954.45|973.53|1222.55|1241.63|-750.05| +2451649|76443|2451758|10441|45329|493247|1612|24730|30670|1322511|6884|36049|17|11|16|1|186|124|77|86.58|218.18|4.36|16464.14|335.72|6666.66|16799.86|23.50|0.00|4535.30|335.72|359.22|4871.02|4894.52|-6330.94| +2451649|76443|2451761|2911|45329|493247|1612|24730|30670|1322511|6884|36049|20|29|19|1|170|124|75|93.94|143.72|102.04|3126.00|7653.00|7045.50|10779.00|0.00|0.00|1616.25|7653.00|7653.00|9269.25|9269.25|607.50| +2451649|76443|2451721|9355|45329|493247|1612|24730|30670|1322511|6884|36049|55|7|10|1|135|124|72|70.21|186.75|69.09|8471.52|4974.48|5055.12|13446.00|0.00|1094.38|537.84|3880.10|3880.10|4417.94|4417.94|-1175.02| +2452638|39420|2452684|5293|97631|1563969|3907|20292|66901|1715954|4106|5644|42|1|4|4|107|125|86|55.31|74.66|18.66|4816.00|1604.76|4756.66|6420.76|112.33|0.00|2696.10|1604.76|1717.09|4300.86|4413.19|-3151.90| +2452638|39420|2452718|4293|97631|1563969|3907|20292|66901|1715954|4106|5644|51|7|2|5|208|125|4|48.69|124.64|89.74|139.60|358.96|194.76|498.56|32.30|0.00|94.72|358.96|391.26|453.68|485.98|164.20| +2452638|39420|2452758|17040|97631|1563969|3907|20292|66901|1715954|4106|5644|36|15|15|5|232|125|95|89.50|114.56|27.49|8271.65|2611.55|8502.50|10883.20|0.00|0.00|5223.10|2611.55|2611.55|7834.65|7834.65|-5890.95| +2452638|39420|2452663|15649|97631|1563969|3907|20292|66901|1715954|4106|5644|3|3|19|4|129|125|56|41.68|87.52|33.25|3039.12|1862.00|2334.08|4901.12|74.48|0.00|490.00|1862.00|1936.48|2352.00|2426.48|-472.08| +2452638|39420|2452648|2641|97631|1563969|3907|20292|66901|1715954|4106|5644|19|1|2|3|117|125|79|23.65|68.58|21.94|3684.56|1733.26|1868.35|5417.82|17.33|0.00|2599.89|1733.26|1750.59|4333.15|4350.48|-135.09| +2452638|39420|2452737|11760|97631|1563969|3907|20292|66901|1715954|4106|5644|39|13|11|3|35|125|87|25.59|69.60|47.32|1938.36|4116.84|2226.33|6055.20|188.96|2017.25|1089.24|2099.59|2288.55|3188.83|3377.79|-126.74| +2452638|39420|2452644|6513|97631|1563969|3907|20292|66901|1715954|4106|5644|55|21|11|1|184|125|15|93.42|262.51|249.38|196.95|3740.70|1401.30|3937.65|261.84|0.00|1023.75|3740.70|4002.54|4764.45|5026.29|2339.40| +2452638|39420|2452758|5023|97631|1563969|3907|20292|66901|1715954|4106|5644|18|12|3|5|87|125|54|11.48|18.82|17.31|81.54|934.74|619.92|1016.28|40.56|355.20|335.34|579.54|620.10|914.88|955.44|-40.38| +2452638|39420|2452641|4057|97631|1563969|3907|20292|66901|1715954|4106|5644|48|1|11|1|188|125|17|7.63|9.53|4.28|89.25|72.76|129.71|162.01|2.35|46.56|45.22|26.20|28.55|71.42|73.77|-103.51| +2452638|39420|2452644|14763|97631|1563969|3907|20292|66901|1715954|4106|5644|45|24|1|2|286|125|10|11.80|11.80|11.56|2.40|115.60|118.00|118.00|4.62|0.00|10.60|115.60|120.22|126.20|130.82|-2.40| +2452638|39420|2452756|5511|97631|1563969|3907|20292|66901|1715954|4106|5644|13|25|9|5|31|125|59|46.03|63.98|47.98|944.00|2830.82|2715.77|3774.82|141.54|0.00|37.17|2830.82|2972.36|2867.99|3009.53|115.05| +2451437|67067|2451521|4592|19818|827546|6474|9002|43306|10433|5739|44267|19|1|2|3|154|126|50|61.84|134.19|22.81|5569.00|1140.50|3092.00|6709.50|102.64|0.00|1341.50|1140.50|1243.14|2482.00|2584.64|-1951.50| +2451437|67067|2451518|5323|19818|827546|6474|9002|43306|10433|5739|44267|25|16|5|4|110|126|45|13.51|21.75|12.61|411.30|567.45|607.95|978.75|16.00|300.74|136.80|266.71|282.71|403.51|419.51|-341.24| +2451437|67067|2451507|5071|19818|827546|6474|9002|43306|10433|5739|44267|50|20|16|1|296|126|31|42.28|103.16|94.90|256.06|2941.90|1310.68|3197.96|147.09|0.00|383.47|2941.90|3088.99|3325.37|3472.46|1631.22| +2451437|67067|2451530|7237|19818|827546|6474|9002|43306|10433|5739|44267|8|20|10|1|255|126|47|31.99|90.85|27.25|2989.20|1280.75|1503.53|4269.95|51.23|0.00|1536.90|1280.75|1331.98|2817.65|2868.88|-222.78| +2451437|67067|2451485|208|19818|827546|6474|9002|43306|10433|5739|44267|19|16|8|5|89|126|81|82.85|190.55|60.97|10495.98|4938.57|6710.85|15434.55|24.19|4592.87|1543.05|345.70|369.89|1888.75|1912.94|-6365.15| +2451437|67067|2451538|9520|19818|827546|6474|9002|43306|10433|5739|44267|55|8|12|4|146|126|47|89.21|161.47|37.13|5843.98|1745.11|4192.87|7589.09|17.45|0.00|3642.50|1745.11|1762.56|5387.61|5405.06|-2447.76| +2451437|67067|2451497|5398|19818|827546|6474|9002|43306|10433|5739|44267|2|14|17|3|102|126|35|22.06|63.31|55.07|288.40|1927.45|772.10|2215.85|12.33|693.88|132.65|1233.57|1245.90|1366.22|1378.55|461.47| +2451437|67067|2451496|4069|19818|827546|6474|9002|43306|10433|5739|44267|13|22|10|3|253|126|47|4.14|8.65|8.21|20.68|385.87|194.58|406.55|15.43|0.00|56.87|385.87|401.30|442.74|458.17|191.29| +2451437|67067|2451443|421|19818|827546|6474|9002|43306|10433|5739|44267|38|10|15|4|160|126|74|91.08|163.94|113.11|3761.42|8370.14|6739.92|12131.56|83.70|0.00|6065.78|8370.14|8453.84|14435.92|14519.62|1630.22| +2451437|67067|2451490|4894|19818|827546|6474|9002|43306|10433|5739|44267|2|20|20|2|181|126|37|15.70|38.77|18.60|746.29|688.20|580.90|1434.49|0.00|178.93|214.97|509.27|509.27|724.24|724.24|-71.63| +2451437|67067|2451553|2756|19818|827546|6474|9002|43306|10433|5739|44267|34|13|13|2|48|126|21|88.46|252.99|252.99|0.00|5312.79|1857.66|5312.79|159.38|0.00|1753.08|5312.79|5472.17|7065.87|7225.25|3455.13| +2452573|49655|2452662|10329|36536|1243818|1147|30429|92749|712421|322|44404|45|24|11|2|166|127|24|55.33|66.39|1.32|1561.68|31.68|1327.92|1593.36|0.63|0.00|191.04|31.68|32.31|222.72|223.35|-1296.24| +2452573|49655|2452610|5991|36536|1243818|1147|30429|92749|712421|322|44404|9|3|16|3|150|127|53|98.83|158.12|121.75|1927.61|6452.75|5237.99|8380.36|387.16|0.00|837.93|6452.75|6839.91|7290.68|7677.84|1214.76| +2452573|49655|2452684|5154|36536|1243818|1147|30429|92749|712421|322|44404|15|18|10|1|72|127|75|69.96|115.43|63.48|3896.25|4761.00|5247.00|8657.25|95.22|0.00|3895.50|4761.00|4856.22|8656.50|8751.72|-486.00| +2452573|49655|2452587|12901|36536|1243818|1147|30429|92749|712421|322|44404|27|27|14|4|104|127|10|81.28|117.85|44.78|730.70|447.80|812.80|1178.50|9.85|250.76|0.00|197.04|206.89|197.04|206.89|-615.76| +2452573|49655|2452600|6282|36536|1243818|1147|30429|92749|712421|322|44404|33|25|19|3|114|127|11|6.54|16.02|12.01|44.11|132.11|71.94|176.22|0.00|0.00|79.20|132.11|132.11|211.31|211.31|60.17| +2452573|49655|2452629|8893|36536|1243818|1147|30429|92749|712421|322|44404|45|25|3|3|253|127|26|14.02|21.87|14.87|182.00|386.62|364.52|568.62|30.92|0.00|136.24|386.62|417.54|522.86|553.78|22.10| +2452573|49655|2452642|11097|36536|1243818|1147|30429|92749|712421|322|44404|36|15|5|2|52|127|47|59.18|127.82|97.14|1441.96|4565.58|2781.46|6007.54|319.59|0.00|1741.82|4565.58|4885.17|6307.40|6626.99|1784.12| +2452573|49655|2452648|4959|36536|1243818|1147|30429|92749|712421|322|44404|18|3|8|4|110|127|98|43.32|56.31|51.80|441.98|5076.40|4245.36|5518.38|0.00|3959.59|2482.34|1116.81|1116.81|3599.15|3599.15|-3128.55| +2452573|49655|2452591|13813|36536|1243818|1147|30429|92749|712421|322|44404|27|9|13|1|149|127|70|35.56|106.68|37.33|4854.50|2613.10|2489.20|7467.60|78.39|0.00|373.10|2613.10|2691.49|2986.20|3064.59|123.90| +2452573|49655|2452665|1704|36536|1243818|1147|30429|92749|712421|322|44404|54|21|2|3|127|127|67|69.86|145.30|21.79|8275.17|1459.93|4680.62|9735.10|26.57|131.39|2433.44|1328.54|1355.11|3761.98|3788.55|-3352.08| +2452573|49655|2452662|12037|36536|1243818|1147|30429|92749|712421|322|44404|6|3|11|5|276|127|78|41.49|121.56|13.37|8438.82|1042.86|3236.22|9481.68|73.00|0.00|1611.48|1042.86|1115.86|2654.34|2727.34|-2193.36| +2452573|49655|2452579|7369|36536|1243818|1147|30429|92749|712421|322|44404|13|3|7|4|144|127|36|10.76|22.70|17.47|188.28|628.92|387.36|817.20|6.28|0.00|106.20|628.92|635.20|735.12|741.40|241.56| +2452573|49655|2452594|7380|36536|1243818|1147|30429|92749|712421|322|44404|24|30|5|3|61|127|42|44.75|96.21|66.38|1252.86|2787.96|1879.50|4040.82|163.09|975.78|1373.82|1812.18|1975.27|3186.00|3349.09|-67.32| +2452573|49655|2452629|6537|36536|1243818|1147|30429|92749|712421|322|44404|27|15|15|4|66|127|54|79.87|201.27|114.72|4673.70|6194.88|4312.98|10868.58|247.79|0.00|977.94|6194.88|6442.67|7172.82|7420.61|1881.90| +2452573|49655|2452658|15132|36536|1243818|1147|30429|92749|712421|322|44404|33|12|2|1|17|127|40|62.50|91.87|61.55|1212.80|2462.00|2500.00|3674.80|147.72|0.00|1837.20|2462.00|2609.72|4299.20|4446.92|-38.00| +2451507|27019|2451616|1888|73673|1357529|4585|22488|41031|541222|103|26359|16|14|17|1|155|128|40|53.55|59.97|4.19|2231.20|167.60|2142.00|2398.80|5.02|0.00|1103.20|167.60|172.62|1270.80|1275.82|-1974.40| +2451507|27019|2451594|15220|73673|1357529|4585|22488|41031|541222|103|26359|8|22|19|3|70|128|86|96.09|197.94|156.37|3575.02|13447.82|8263.74|17022.84|451.84|2151.65|8170.86|11296.17|11748.01|19467.03|19918.87|3032.43| +2451507|27019|2451559|5954|73673|1357529|4585|22488|41031|541222|103|26359|26|13|13|1|39|128|83|49.69|59.62|47.09|1039.99|3908.47|4124.27|4948.46|273.59|0.00|890.59|3908.47|4182.06|4799.06|5072.65|-215.80| +2451507|27019|2451593|17528|73673|1357529|4585|22488|41031|541222|103|26359|25|14|13|3|203|128|46|25.06|58.38|54.29|188.14|2497.34|1152.76|2685.48|99.89|0.00|885.96|2497.34|2597.23|3383.30|3483.19|1344.58| +2451507|27019|2451603|9262|73673|1357529|4585|22488|41031|541222|103|26359|28|28|8|3|38|128|18|75.95|100.25|49.12|920.34|884.16|1367.10|1804.50|79.57|0.00|72.18|884.16|963.73|956.34|1035.91|-482.94| +2451507|27019|2451578|9505|73673|1357529|4585|22488|41031|541222|103|26359|8|28|5|5|45|128|100|14.84|31.46|22.33|913.00|2233.00|1484.00|3146.00|133.98|0.00|377.00|2233.00|2366.98|2610.00|2743.98|749.00| +2451507|27019|2451527|16570|73673|1357529|4585|22488|41031|541222|103|26359|22|7|13|5|114|128|66|49.50|126.72|17.74|7192.68|1170.84|3267.00|8363.52|2.10|1135.71|333.96|35.13|37.23|369.09|371.19|-3231.87| +2451507|27019|2451596|1838|73673|1357529|4585|22488|41031|541222|103|26359|19|25|9|5|271|128|40|30.16|84.74|40.67|1762.80|1626.80|1206.40|3389.60|0.00|764.59|1626.80|862.21|862.21|2489.01|2489.01|-344.19| +2451507|27019|2451609|4108|73673|1357529|4585|22488|41031|541222|103|26359|56|10|20|2|274|128|84|91.65|178.71|109.01|5854.80|9156.84|7698.60|15011.64|274.70|0.00|149.52|9156.84|9431.54|9306.36|9581.06|1458.24| +2451507|27019|2451516|16760|73673|1357529|4585|22488|41031|541222|103|26359|22|13|2|2|105|128|47|82.47|164.11|24.61|6556.50|1156.67|3876.09|7713.17|57.83|0.00|693.72|1156.67|1214.50|1850.39|1908.22|-2719.42| +2451507|27019|2451545|9781|73673|1357529|4585|22488|41031|541222|103|26359|13|4|9|5|51|128|30|61.20|137.08|102.81|1028.10|3084.30|1836.00|4112.40|12.33|2467.44|1562.70|616.86|629.19|2179.56|2191.89|-1219.14| +2451507|27019|2451586|6772|73673|1357529|4585|22488|41031|541222|103|26359|22|10|20|4|168|128|82|37.50|75.00|31.50|3567.00|2583.00|3075.00|6150.00|63.54|464.94|2275.50|2118.06|2181.60|4393.56|4457.10|-956.94| +2451507|27019|2451558|13226|73673|1357529|4585|22488|41031|541222|103|26359|38|7|18|2|9|128|70|74.93|76.42|55.02|1498.00|3851.40|5245.10|5349.40|231.08|0.00|2139.20|3851.40|4082.48|5990.60|6221.68|-1393.70| +2451507|27019|2451574|13741|73673|1357529|4585|22488|41031|541222|103|26359|7|22|17|5|163|128|96|8.16|14.03|3.92|970.56|376.32|783.36|1346.88|30.10|0.00|552.00|376.32|406.42|928.32|958.42|-407.04| +2451169|47837|2451262|4147|49745|1301217|6642|16514|61699|812066|2036|33108|32|14|6|2|288|129|81|63.77|114.14|77.61|2958.93|6286.41|5165.37|9245.34|0.00|0.00|0.00|6286.41|6286.41|6286.41|6286.41|1121.04| +2451169|47837|2451222|8116|49745|1301217|6642|16514|61699|812066|2036|33108|50|22|18|4|203|129|64|49.73|115.87|13.90|6526.08|889.60|3182.72|7415.68|71.16|0.00|519.04|889.60|960.76|1408.64|1479.80|-2293.12| +2451169|47837|2451193|358|49745|1301217|6642|16514|61699|812066|2036|33108|31|2|5|1|124|129|15|29.90|42.15|20.23|328.80|303.45|448.50|632.25|18.20|0.00|214.95|303.45|321.65|518.40|536.60|-145.05| +2451169|47837|2451289|8228|49745|1301217|6642|16514|61699|812066|2036|33108|55|7|20|3|209|129|97|2.96|3.46|0.38|298.76|36.86|287.12|335.62|0.00|0.00|136.77|36.86|36.86|173.63|173.63|-250.26| +2451169|47837|2451207|9274|49745|1301217|6642|16514|61699|812066|2036|33108|55|28|2|4|156|129|53|96.79|186.80|29.88|8316.76|1583.64|5129.87|9900.40|26.60|1140.22|1880.97|443.42|470.02|2324.39|2350.99|-4686.45| +2451169|47837|2451221|4906|49745|1301217|6642|16514|61699|812066|2036|33108|7|7|1|1|168|129|44|91.91|123.15|44.33|3468.08|1950.52|4044.04|5418.60|156.04|0.00|162.36|1950.52|2106.56|2112.88|2268.92|-2093.52| +2451169|47837|2451173|9457|49745|1301217|6642|16514|61699|812066|2036|33108|46|20|8|5|212|129|55|42.50|116.45|18.63|5380.10|1024.65|2337.50|6404.75|51.23|0.00|3202.10|1024.65|1075.88|4226.75|4277.98|-1312.85| +2451169|47837|2451192|3541|49745|1301217|6642|16514|61699|812066|2036|33108|16|16|9|4|279|129|88|48.84|79.12|26.90|4595.36|2367.20|4297.92|6962.56|86.64|923.20|2296.80|1444.00|1530.64|3740.80|3827.44|-2853.92| +2451169|47837|2451235|11912|49745|1301217|6642|16514|61699|812066|2036|33108|28|16|6|5|44|129|82|92.36|167.17|6.68|13160.18|547.76|7573.52|13707.94|16.43|0.00|684.70|547.76|564.19|1232.46|1248.89|-7025.76| +2451169|47837|2451285|16153|49745|1301217|6642|16514|61699|812066|2036|33108|2|25|14|5|196|129|22|29.98|58.46|16.95|913.22|372.90|659.56|1286.12|0.00|0.00|527.12|372.90|372.90|900.02|900.02|-286.66| +2451169|47837|2451170|4801|49745|1301217|6642|16514|61699|812066|2036|33108|55|16|20|1|192|129|11|43.31|112.60|70.93|458.37|780.23|476.41|1238.60|46.81|0.00|458.26|780.23|827.04|1238.49|1285.30|303.82| +2451169|47837|2451222|3608|49745|1301217|6642|16514|61699|812066|2036|33108|32|4|19|5|51|129|67|62.83|132.57|108.70|1599.29|7282.90|4209.61|8882.19|145.65|0.00|2575.48|7282.90|7428.55|9858.38|10004.03|3073.29| +2451169|47837|2451275|1087|49745|1301217|6642|16514|61699|812066|2036|33108|16|22|15|3|171|129|69|52.91|93.65|0.93|6397.68|64.17|3650.79|6461.85|1.28|0.00|580.98|64.17|65.45|645.15|646.43|-3586.62| +2451484|39865|2451569|16844|73522|91558|6041|17677|39744|890554|3879|32315|52|4|2|3|67|130|20|66.60|123.21|87.47|714.80|1749.40|1332.00|2464.20|17.49|0.00|689.80|1749.40|1766.89|2439.20|2456.69|417.40| +2451484|39865|2451503|16582|73522|91558|6041|17677|39744|890554|3879|32315|43|26|16|3|230|130|50|94.16|271.18|219.65|2576.50|10982.50|4708.00|13559.00|988.42|0.00|2440.50|10982.50|11970.92|13423.00|14411.42|6274.50| +2451484|39865|2451509|2371|73522|91558|6041|17677|39744|890554|3879|32315|55|20|7|5|133|130|54|78.63|142.32|83.96|3151.44|4533.84|4246.02|7685.28|317.36|0.00|460.62|4533.84|4851.20|4994.46|5311.82|287.82| +2451484|39865|2451591|826|73522|91558|6041|17677|39744|890554|3879|32315|56|16|19|2|152|130|43|67.48|153.17|145.51|329.38|6256.93|2901.64|6586.31|563.12|0.00|2107.43|6256.93|6820.05|8364.36|8927.48|3355.29| +2451484|39865|2451512|6379|73522|91558|6041|17677|39744|890554|3879|32315|43|2|2|2|212|130|91|43.90|107.99|24.83|7567.56|2259.53|3994.90|9827.09|79.08|1129.76|4323.41|1129.77|1208.85|5453.18|5532.26|-2865.13| +2451484|39865|2451502|10423|73522|91558|6041|17677|39744|890554|3879|32315|50|28|1|5|116|130|32|60.75|159.16|93.90|2088.32|3004.80|1944.00|5093.12|270.43|0.00|967.68|3004.80|3275.23|3972.48|4242.91|1060.80| +2451484|39865|2451566|17011|73522|91558|6041|17677|39744|890554|3879|32315|2|10|10|4|228|130|91|80.38|152.72|83.99|6254.43|7643.09|7314.58|13897.52|105.47|4127.26|6114.29|3515.83|3621.30|9630.12|9735.59|-3798.75| +2451484|39865|2451520|529|73522|91558|6041|17677|39744|890554|3879|32315|28|8|12|4|289|130|37|74.52|133.39|9.33|4590.22|345.21|2757.24|4935.43|3.45|0.00|394.79|345.21|348.66|740.00|743.45|-2412.03| +2451484|39865|2451532|1129|73522|91558|6041|17677|39744|890554|3879|32315|50|16|19|2|122|130|30|89.59|268.77|134.38|4031.70|4031.40|2687.70|8063.10|0.00|0.00|886.80|4031.40|4031.40|4918.20|4918.20|1343.70| +2451484|39865|2451516|7400|73522|91558|6041|17677|39744|890554|3879|32315|32|22|14|4|244|130|26|59.37|133.58|101.52|833.56|2639.52|1543.62|3473.08|52.79|0.00|1215.50|2639.52|2692.31|3855.02|3907.81|1095.90| +2451484|39865|2451509|16843|73522|91558|6041|17677|39744|890554|3879|32315|16|7|2|5|298|130|93|35.29|56.81|36.35|1902.78|3380.55|3281.97|5283.33|173.08|1216.99|1689.81|2163.56|2336.64|3853.37|4026.45|-1118.41| +2451484|39865|2451517|16298|73522|91558|6041|17677|39744|890554|3879|32315|56|28|4|5|26|130|94|11.64|13.03|3.64|882.66|342.16|1094.16|1224.82|0.00|0.00|73.32|342.16|342.16|415.48|415.48|-752.00| +2451484|39865|2451504|14311|73522|91558|6041|17677|39744|890554|3879|32315|8|28|2|2|262|130|98|45.38|132.50|17.22|11297.44|1687.56|4447.24|12985.00|54.67|320.63|1298.50|1366.93|1421.60|2665.43|2720.10|-3080.31| +2452171|78440|2452233|1567|9906|1805575|5597|7666|7721|1123492|2226|15895|41|19|9|1|240|131|41|3.34|6.04|1.32|193.52|54.12|136.94|247.64|3.24|0.00|46.74|54.12|57.36|100.86|104.10|-82.82| +2452171|78440|2452220|13073|9906|1805575|5597|7666|7721|1123492|2226|15895|45|25|3|3|193|131|46|43.56|88.42|35.36|2440.76|1626.56|2003.76|4067.32|113.85|0.00|1341.82|1626.56|1740.41|2968.38|3082.23|-377.20| +2452171|78440|2452275|3893|9906|1805575|5597|7666|7721|1123492|2226|15895|1|15|15|2|269|131|1|89.18|113.25|80.40|32.85|80.40|89.18|113.25|3.37|32.16|53.22|48.24|51.61|101.46|104.83|-40.94| +2452171|78440|2452218|11439|9906|1805575|5597|7666|7721|1123492|2226|15895|45|7|5|1|97|131|81|95.17|103.73|60.16|3529.17|4872.96|7708.77|8402.13|194.91|0.00|3780.27|4872.96|5067.87|8653.23|8848.14|-2835.81| +2452171|78440|2452191|12991|9906|1805575|5597|7666|7721|1123492|2226|15895|57|7|16|3|123|131|71|55.61|81.19|23.54|4093.15|1671.34|3948.31|5764.49|83.56|0.00|1268.06|1671.34|1754.90|2939.40|3022.96|-2276.97| +2452171|78440|2452191|13909|9906|1805575|5597|7666|7721|1123492|2226|15895|29|23|9|4|266|131|73|32.87|43.38|24.72|1362.18|1804.56|2399.51|3166.74|144.36|0.00|31.39|1804.56|1948.92|1835.95|1980.31|-594.95| +2452171|78440|2452228|3857|9906|1805575|5597|7666|7721|1123492|2226|15895|59|13|14|1|81|131|37|88.15|185.11|42.57|5273.98|1575.09|3261.55|6849.07|141.75|0.00|547.60|1575.09|1716.84|2122.69|2264.44|-1686.46| +2452171|78440|2452192|7119|9906|1805575|5597|7666|7721|1123492|2226|15895|9|3|19|1|150|131|90|43.18|94.56|30.25|5787.90|2722.50|3886.20|8510.40|190.57|0.00|594.90|2722.50|2913.07|3317.40|3507.97|-1163.70| +2452171|78440|2452262|1201|9906|1805575|5597|7666|7721|1123492|2226|15895|5|21|16|1|289|131|74|60.90|155.29|69.88|6320.34|5171.12|4506.60|11491.46|51.71|0.00|0.00|5171.12|5222.83|5171.12|5222.83|664.52| +2452171|78440|2452227|13061|9906|1805575|5597|7666|7721|1123492|2226|15895|41|17|10|3|109|131|8|27.90|33.20|11.62|172.64|92.96|223.20|265.60|0.19|86.45|0.00|6.51|6.70|6.51|6.70|-216.69| +2452171|78440|2452283|531|9906|1805575|5597|7666|7721|1123492|2226|15895|13|29|6|4|265|131|89|30.37|80.78|30.69|4458.01|2731.41|2702.93|7189.42|109.25|0.00|3594.71|2731.41|2840.66|6326.12|6435.37|28.48| +2451160|78790|2451174|8878|85582|1641083|5737|22038|30867|917433|94|15195|52|8|16|4|52|132|5|16.11|16.43|10.35|30.40|51.75|80.55|82.15|2.07|0.00|27.10|51.75|53.82|78.85|80.92|-28.80| +2451160|78790|2451203|427|85582|1641083|5737|22038|30867|917433|94|15195|32|25|9|5|278|132|54|80.19|225.33|67.59|8517.96|3649.86|4330.26|12167.82|4.37|3211.87|3041.82|437.99|442.36|3479.81|3484.18|-3892.27| +2451160|78790|2451173|571|85582|1641083|5737|22038|30867|917433|94|15195|16|7|19|4|102|132|54|90.56|166.63|78.31|4769.28|4228.74|4890.24|8998.02|338.29|0.00|1709.10|4228.74|4567.03|5937.84|6276.13|-661.50| +2451160|78790|2451188|8840|85582|1641083|5737|22038|30867|917433|94|15195|55|25|8|3|208|132|28|88.69|253.65|116.67|3835.44|3266.76|2483.32|7102.20|91.46|1437.37|1988.56|1829.39|1920.85|3817.95|3909.41|-653.93| +2451160|78790|2451257|10748|85582|1641083|5737|22038|30867|917433|94|15195|46|25|12|1|70|132|81|35.54|75.34|39.17|2929.77|3172.77|2878.74|6102.54|12.69|2918.94|2867.40|253.83|266.52|3121.23|3133.92|-2624.91| +2451160|78790|2451278|9238|85582|1641083|5737|22038|30867|917433|94|15195|56|22|15|5|28|132|72|78.57|78.57|54.99|1697.76|3959.28|5657.04|5657.04|0.00|0.00|1414.08|3959.28|3959.28|5373.36|5373.36|-1697.76| +2451160|78790|2451213|16954|85582|1641083|5737|22038|30867|917433|94|15195|14|25|11|1|201|132|90|68.77|176.05|137.31|3486.60|12357.90|6189.30|15844.50|129.75|8032.63|2376.00|4325.27|4455.02|6701.27|6831.02|-1864.03| +2451160|78790|2451185|3742|85582|1641083|5737|22038|30867|917433|94|15195|58|10|19|5|211|132|10|70.44|140.17|116.34|238.30|1163.40|704.40|1401.70|2.44|919.08|182.20|244.32|246.76|426.52|428.96|-460.08| +2451160|78790|2451167|14414|85582|1641083|5737|22038|30867|917433|94|15195|2|20|13|2|174|132|27|68.65|133.18|83.90|1330.56|2265.30|1853.55|3595.86|135.91|0.00|1294.38|2265.30|2401.21|3559.68|3695.59|411.75| +2451160|78790|2451210|4502|85582|1641083|5737|22038|30867|917433|94|15195|52|22|4|4|214|132|56|48.95|93.98|47.92|2579.36|2683.52|2741.20|5262.88|241.51|0.00|1368.08|2683.52|2925.03|4051.60|4293.11|-57.68| +2451160|78790|2451227|6430|85582|1641083|5737|22038|30867|917433|94|15195|32|10|15|4|273|132|2|75.74|185.56|144.73|81.66|289.46|151.48|371.12|0.00|289.46|96.48|0.00|0.00|96.48|96.48|-151.48| +2451160|78790|2451226|4480|85582|1641083|5737|22038|30867|917433|94|15195|40|2|16|2|111|132|87|67.05|178.35|67.77|9620.46|5895.99|5833.35|15516.45|117.91|0.00|7447.20|5895.99|6013.90|13343.19|13461.10|62.64| +2451160|78790|2451229|5980|85582|1641083|5737|22038|30867|917433|94|15195|13|19|13|3|236|132|20|72.51|135.59|75.93|1193.20|1518.60|1450.20|2711.80|78.96|531.51|1247.40|987.09|1066.05|2234.49|2313.45|-463.11| +2451160|78790|2451208|15370|85582|1641083|5737|22038|30867|917433|94|15195|1|14|2|1|50|132|62|53.48|102.68|88.30|891.56|5474.60|3315.76|6366.16|383.22|0.00|2355.38|5474.60|5857.82|7829.98|8213.20|2158.84| +2451162|50155|2451204|7408|26465|1196287|1309|33178|1049|231132|2291|30913|26|19|3|1|282|133|79|6.53|7.37|2.50|384.73|197.50|515.87|582.23|1.97|0.00|86.90|197.50|199.47|284.40|286.37|-318.37| +2451162|50155|2451198|15478|26465|1196287|1309|33178|1049|231132|2291|30913|28|10|7|3|152|133|47|45.04|75.66|12.10|2987.32|568.70|2116.88|3556.02|43.22|28.43|355.32|540.27|583.49|895.59|938.81|-1576.61| +2451162|50155|2451198|11923|26465|1196287|1309|33178|1049|231132|2291|30913|56|14|17|1|65|133|53|73.35|161.37|90.36|3763.53|4789.08|3887.55|8552.61|143.67|0.00|2479.87|4789.08|4932.75|7268.95|7412.62|901.53| +2451162|50155|2451163|16726|26465|1196287|1309|33178|1049|231132|2291|30913|49|2|19|1|180|133|25|60.40|83.35|13.33|1750.50|333.25|1510.00|2083.75|2.99|283.26|208.25|49.99|52.98|258.24|261.23|-1460.01| +2451162|50155|2451277|16465|26465|1196287|1309|33178|1049|231132|2291|30913|2|16|9|4|189|133|30|55.94|99.57|60.73|1165.20|1821.90|1678.20|2987.10|54.65|0.00|417.90|1821.90|1876.55|2239.80|2294.45|143.70| +2451162|50155|2451237|9865|26465|1196287|1309|33178|1049|231132|2291|30913|25|26|18|2|153|133|91|26.94|27.20|14.14|1188.46|1286.74|2451.54|2475.20|51.46|0.00|618.80|1286.74|1338.20|1905.54|1957.00|-1164.80| +2451162|50155|2451266|524|26465|1196287|1309|33178|1049|231132|2291|30913|56|25|10|4|251|133|6|77.74|167.14|66.85|601.74|401.10|466.44|1002.84|12.03|0.00|10.02|401.10|413.13|411.12|423.15|-65.34| +2451162|50155|2451186|11246|26465|1196287|1309|33178|1049|231132|2291|30913|8|14|4|5|153|133|61|48.39|70.16|13.33|3466.63|813.13|2951.79|4279.76|32.52|0.00|85.40|813.13|845.65|898.53|931.05|-2138.66| +2451162|50155|2451198|13405|26465|1196287|1309|33178|1049|231132|2291|30913|55|28|5|2|299|133|3|56.60|116.03|84.70|93.99|254.10|169.80|348.09|20.32|0.00|48.72|254.10|274.42|302.82|323.14|84.30| +2451162|50155|2451239|17929|26465|1196287|1309|33178|1049|231132|2291|30913|50|2|8|2|133|133|43|13.62|21.65|6.92|633.39|297.56|585.66|930.95|23.80|0.00|64.93|297.56|321.36|362.49|386.29|-288.10| +2452496|33301|2452594|16983|24050|1584970|1090|19610|46339|268737|5217|47337|18|3|18|4|57|134|9|53.69|141.74|109.13|293.49|982.17|483.21|1275.66|88.39|0.00|318.87|982.17|1070.56|1301.04|1389.43|498.96| +2452496|33301|2452578|3210|24050|1584970|1090|19610|46339|268737|5217|47337|1|25|18|5|72|134|37|10.21|26.95|2.15|917.60|79.55|377.77|997.15|3.97|0.00|448.44|79.55|83.52|527.99|531.96|-298.22| +2452496|33301|2452608|14835|24050|1584970|1090|19610|46339|268737|5217|47337|18|1|3|5|217|134|28|21.65|38.53|25.42|367.08|711.76|606.20|1078.84|49.82|0.00|237.16|711.76|761.58|948.92|998.74|105.56| +2452496|33301|2452542|3969|24050|1584970|1090|19610|46339|268737|5217|47337|30|27|6|5|64|134|37|91.49|118.02|82.61|1310.17|3056.57|3385.13|4366.74|244.52|0.00|960.52|3056.57|3301.09|4017.09|4261.61|-328.56| +2452496|33301|2452541|10519|24050|1584970|1090|19610|46339|268737|5217|47337|15|9|2|4|56|134|15|50.40|55.44|43.24|183.00|648.60|756.00|831.60|38.91|0.00|33.15|648.60|687.51|681.75|720.66|-107.40| +2452496|33301|2452538|8190|24050|1584970|1090|19610|46339|268737|5217|47337|6|25|6|4|269|134|71|83.10|177.00|162.84|1005.36|11561.64|5900.10|12567.00|809.31|0.00|4775.46|11561.64|12370.95|16337.10|17146.41|5661.54| +2452496|33301|2452514|3597|24050|1584970|1090|19610|46339|268737|5217|47337|15|1|14|2|250|134|36|11.07|28.44|6.82|778.32|245.52|398.52|1023.84|6.87|147.31|51.12|98.21|105.08|149.33|156.20|-300.31| +2452496|33301|2452533|5773|24050|1584970|1090|19610|46339|268737|5217|47337|37|25|14|3|249|134|23|49.88|112.72|51.85|1400.01|1192.55|1147.24|2592.56|83.47|0.00|103.50|1192.55|1276.02|1296.05|1379.52|45.31| +2452496|33301|2452593|8049|24050|1584970|1090|19610|46339|268737|5217|47337|19|3|14|4|91|134|80|2.65|5.22|1.30|313.60|104.00|212.00|417.60|5.20|0.00|74.40|104.00|109.20|178.40|183.60|-108.00| +2452496|33301|2452514|14611|24050|1584970|1090|19610|46339|268737|5217|47337|43|6|4|5|66|134|93|75.23|139.92|76.95|5856.21|7156.35|6996.39|13012.56|143.12|0.00|1431.27|7156.35|7299.47|8587.62|8730.74|159.96| +2452496|33301|2452587|12363|24050|1584970|1090|19610|46339|268737|5217|47337|57|9|18|4|157|134|29|71.94|146.75|105.66|1191.61|3064.14|2086.26|4255.75|30.64|0.00|468.06|3064.14|3094.78|3532.20|3562.84|977.88| +2452496|33301|2452502|14859|24050|1584970|1090|19610|46339|268737|5217|47337|27|13|13|1|272|134|91|36.50|51.10|0.51|4603.69|46.41|3321.50|4650.10|0.00|0.00|650.65|46.41|46.41|697.06|697.06|-3275.09| +2452496|33301|2452585|5515|24050|1584970|1090|19610|46339|268737|5217|47337|21|21|18|3|265|134|64|36.43|95.81|60.36|2268.80|3863.04|2331.52|6131.84|0.00|0.00|2697.60|3863.04|3863.04|6560.64|6560.64|1531.52| +2451510|43393|2451586|3632|26805|1783570|7088|21006|26805|1783570|7088|21006|37|14|17|1|185|135|40|90.27|178.73|71.49|4289.60|2859.60|3610.80|7149.20|142.98|0.00|2645.20|2859.60|3002.58|5504.80|5647.78|-751.20| +2451510|43393|2451623|6550|26805|1783570|7088|21006|26805|1783570|7088|21006|7|4|9|3|127|135|33|96.60|171.94|116.91|1815.99|3858.03|3187.80|5674.02|270.06|0.00|1248.06|3858.03|4128.09|5106.09|5376.15|670.23| +2451510|43393|2451561|7966|26805|1783570|7088|21006|26805|1783570|7088|21006|28|7|13|2|271|135|68|27.78|38.05|21.30|1139.00|1448.40|1889.04|2587.40|30.41|1013.88|25.84|434.52|464.93|460.36|490.77|-1454.52| +2451510|43393|2451528|5560|26805|1783570|7088|21006|26805|1783570|7088|21006|32|19|13|4|192|135|31|6.79|15.27|10.99|132.68|340.69|210.49|473.37|13.62|0.00|75.64|340.69|354.31|416.33|429.95|130.20| +2451510|43393|2451518|2264|26805|1783570|7088|21006|26805|1783570|7088|21006|2|4|14|5|276|135|78|49.78|117.48|78.71|3024.06|6139.38|3882.84|9163.44|122.78|0.00|2932.02|6139.38|6262.16|9071.40|9194.18|2256.54| +2451510|43393|2451545|8827|26805|1783570|7088|21006|26805|1783570|7088|21006|26|26|7|2|67|135|23|39.92|54.69|4.92|1144.71|113.16|918.16|1257.87|2.26|0.00|50.14|113.16|115.42|163.30|165.56|-805.00| +2451510|43393|2451579|17998|26805|1783570|7088|21006|26805|1783570|7088|21006|38|7|18|2|233|135|63|59.36|117.53|16.45|6368.04|1036.35|3739.68|7404.39|51.81|0.00|3331.44|1036.35|1088.16|4367.79|4419.60|-2703.33| +2451510|43393|2451560|4166|26805|1783570|7088|21006|26805|1783570|7088|21006|43|10|7|3|62|135|80|76.64|129.52|94.54|2798.40|7563.20|6131.20|10361.60|15.12|6806.88|3729.60|756.32|771.44|4485.92|4501.04|-5374.88| +2451510|43393|2451571|2008|26805|1783570|7088|21006|26805|1783570|7088|21006|37|28|8|4|160|135|100|14.12|37.41|5.61|3180.00|561.00|1412.00|3741.00|11.22|0.00|1496.00|561.00|572.22|2057.00|2068.22|-851.00| +2451510|43393|2451578|15340|26805|1783570|7088|21006|26805|1783570|7088|21006|14|26|20|1|242|135|13|31.35|56.74|17.58|509.08|228.54|407.55|737.62|18.28|0.00|7.28|228.54|246.82|235.82|254.10|-179.01| +2452127|43101|2452230|3231|77225|1147154|2683|41412|94282|1272679|5246|43975|35|5|5|5|212|136|83|15.22|26.48|6.62|1648.38|549.46|1263.26|2197.84|49.45|0.00|0.00|549.46|598.91|549.46|598.91|-713.80| +2452127|43101|2452222|1461|77225|1147154|2683|41412|94282|1272679|5246|43975|51|29|7|4|238|136|11|15.99|20.78|16.41|48.07|180.51|175.89|228.58|9.02|0.00|50.27|180.51|189.53|230.78|239.80|4.62| +2452127|43101|2452158|14849|77225|1147154|2683|41412|94282|1272679|5246|43975|21|29|19|3|177|136|88|73.33|183.32|86.16|8550.08|7582.08|6453.04|16132.16|303.28|0.00|322.08|7582.08|7885.36|7904.16|8207.44|1129.04| +2452127|43101|2452179|14413|77225|1147154|2683|41412|94282|1272679|5246|43975|47|17|17|2|10|136|19|72.84|108.53|107.44|20.71|2041.36|1383.96|2062.07|81.65|0.00|350.55|2041.36|2123.01|2391.91|2473.56|657.40| +2452127|43101|2452144|523|77225|1147154|2683|41412|94282|1272679|5246|43975|35|25|5|5|185|136|5|98.60|107.47|77.37|150.50|386.85|493.00|537.35|0.07|379.11|26.85|7.74|7.81|34.59|34.66|-485.26| +2452127|43101|2452225|13939|77225|1147154|2683|41412|94282|1272679|5246|43975|47|11|8|5|35|136|28|46.78|134.25|119.48|413.56|3345.44|1309.84|3759.00|66.90|0.00|1315.44|3345.44|3412.34|4660.88|4727.78|2035.60| +2452127|43101|2452185|13023|77225|1147154|2683|41412|94282|1272679|5246|43975|23|7|19|4|289|136|14|11.79|23.93|6.70|241.22|93.80|165.06|335.02|6.56|0.00|140.70|93.80|100.36|234.50|241.06|-71.26| +2452127|43101|2452218|6931|77225|1147154|2683|41412|94282|1272679|5246|43975|1|25|15|2|19|136|54|68.91|148.84|87.81|3295.62|4741.74|3721.14|8037.36|0.00|0.00|2973.78|4741.74|4741.74|7715.52|7715.52|1020.60| +2452127|43101|2452200|13627|77225|1147154|2683|41412|94282|1272679|5246|43975|31|3|18|4|170|136|68|38.14|51.48|2.57|3325.88|174.76|2593.52|3500.64|6.99|0.00|244.80|174.76|181.75|419.56|426.55|-2418.76| +2452127|43101|2452148|16787|77225|1147154|2683|41412|94282|1272679|5246|43975|45|21|2|5|28|136|55|74.66|194.86|68.20|6966.30|3751.00|4106.30|10717.30|37.51|0.00|1714.35|3751.00|3788.51|5465.35|5502.86|-355.30| +2452127|43101|2452183|6999|77225|1147154|2683|41412|94282|1272679|5246|43975|13|25|12|3|107|136|11|74.69|103.81|9.34|1039.17|102.74|821.59|1141.91|1.02|0.00|502.37|102.74|103.76|605.11|606.13|-718.85| +2452127|43101|2452202|14861|77225|1147154|2683|41412|94282|1272679|5246|43975|1|7|10|3|276|136|87|67.20|75.93|74.41|132.24|6473.67|5846.40|6605.91|0.00|0.00|990.06|6473.67|6473.67|7463.73|7463.73|627.27| +2452127|43101|2452145|6301|77225|1147154|2683|41412|94282|1272679|5246|43975|3|15|11|2|153|136|29|51.43|127.54|77.79|1442.75|2255.91|1491.47|3698.66|90.23|0.00|554.77|2255.91|2346.14|2810.68|2900.91|764.44| +2451446|79601|2451549|7072|72575|1638838|2701|11764|16255|1274557|6153|14016|31|8|4|3|2|137|83|81.38|145.67|53.89|7617.74|4472.87|6754.54|12090.61|14.76|3980.85|604.24|492.02|506.78|1096.26|1111.02|-6262.52| +2451446|79601|2451537|9826|72575|1638838|2701|11764|16255|1274557|6153|14016|52|7|14|1|168|137|38|16.93|46.38|6.02|1533.68|228.76|643.34|1762.44|4.39|155.55|193.80|73.21|77.60|267.01|271.40|-570.13| +2451446|79601|2451492|11614|72575|1638838|2701|11764|16255|1274557|6153|14016|43|4|4|5|239|137|57|64.87|95.35|89.62|326.61|5108.34|3697.59|5434.95|153.25|0.00|1847.37|5108.34|5261.59|6955.71|7108.96|1410.75| +2451446|79601|2451488|11680|72575|1638838|2701|11764|16255|1274557|6153|14016|13|25|12|3|167|137|17|77.08|134.11|127.40|114.07|2165.80|1310.36|2279.87|57.17|736.37|341.87|1429.43|1486.60|1771.30|1828.47|119.07| +2451446|79601|2451498|6304|72575|1638838|2701|11764|16255|1274557|6153|14016|20|4|15|5|272|137|83|24.53|58.62|53.93|389.27|4476.19|2035.99|4865.46|107.42|1790.47|826.68|2685.72|2793.14|3512.40|3619.82|649.73| +2451446|79601|2451528|13057|72575|1638838|2701|11764|16255|1274557|6153|14016|37|10|13|5|57|137|61|57.90|149.38|73.19|4647.59|4464.59|3531.90|9112.18|29.46|1517.96|546.56|2946.63|2976.09|3493.19|3522.65|-585.27| +2451446|79601|2451540|14845|72575|1638838|2701|11764|16255|1274557|6153|14016|1|19|12|4|226|137|94|11.50|17.13|7.53|902.40|707.82|1081.00|1610.22|29.30|382.22|466.24|325.60|354.90|791.84|821.14|-755.40| +2451446|79601|2451484|16472|72575|1638838|2701|11764|16255|1274557|6153|14016|44|25|16|1|149|137|50|36.31|84.23|32.84|2569.50|1642.00|1815.50|4211.50|147.78|0.00|1768.50|1642.00|1789.78|3410.50|3558.28|-173.50| +2451446|79601|2451546|1426|72575|1638838|2701|11764|16255|1274557|6153|14016|32|22|10|3|96|137|99|48.02|73.47|28.65|4437.18|2836.35|4753.98|7273.53|65.80|1191.26|2399.76|1645.09|1710.89|4044.85|4110.65|-3108.89| +2451446|79601|2451474|5671|72575|1638838|2701|11764|16255|1274557|6153|14016|19|2|17|4|61|137|75|45.67|87.22|27.91|4448.25|2093.25|3425.25|6541.50|146.52|0.00|1373.25|2093.25|2239.77|3466.50|3613.02|-1332.00| +2451446|79601|2451520|11464|72575|1638838|2701|11764|16255|1274557|6153|14016|22|1|13|5|44|137|27|10.10|11.71|0.81|294.30|21.87|272.70|316.17|0.00|0.00|88.29|21.87|21.87|110.16|110.16|-250.83| +2451446|79601|2451537|2293|72575|1638838|2701|11764|16255|1274557|6153|14016|28|22|9|1|93|137|32|96.08|282.47|115.81|5333.12|3705.92|3074.56|9039.04|333.53|0.00|0.00|3705.92|4039.45|3705.92|4039.45|631.36| +2451446|79601|2451537|15590|72575|1638838|2701|11764|16255|1274557|6153|14016|20|22|3|4|225|137|92|14.43|29.58|27.80|163.76|2557.60|1327.56|2721.36|0.00|0.00|652.28|2557.60|2557.60|3209.88|3209.88|1230.04| +2451446|79601|2451540|10411|72575|1638838|2701|11764|16255|1274557|6153|14016|49|19|15|2|152|137|92|70.81|124.62|93.46|2866.72|8598.32|6514.52|11465.04|601.88|0.00|2750.80|8598.32|9200.20|11349.12|11951.00|2083.80| +2451446|79601|2451524|5698|72575|1638838|2701|11764|16255|1274557|6153|14016|10|8|1|5|226|137|74|71.95|133.10|111.80|1576.20|8273.20|5324.30|9849.40|248.19|0.00|2954.82|8273.20|8521.39|11228.02|11476.21|2948.90| +2451446|79601|2451483|3778|72575|1638838|2701|11764|16255|1274557|6153|14016|7|22|7|5|283|137|12|26.23|68.46|27.38|492.96|328.56|314.76|821.52|3.28|0.00|16.32|328.56|331.84|344.88|348.16|13.80| +2451407|76320|2451438|997|93916|83515|6361|10760|99282|634616|51|5650|13|26|18|2|26|138|22|84.72|168.59|94.41|1631.96|2077.02|1863.84|3708.98|186.93|0.00|890.12|2077.02|2263.95|2967.14|3154.07|213.18| +2451407|76320|2451435|13090|93916|83515|6361|10760|99282|634616|51|5650|22|22|9|4|169|138|26|20.84|51.89|35.80|418.34|930.80|541.84|1349.14|16.47|381.62|26.78|549.18|565.65|575.96|592.43|7.34| +2451407|76320|2451469|1600|93916|83515|6361|10760|99282|634616|51|5650|31|19|4|3|299|138|93|82.22|216.23|99.46|10859.61|9249.78|7646.46|20109.39|462.48|0.00|7239.12|9249.78|9712.26|16488.90|16951.38|1603.32| +2451407|76320|2451457|9380|93916|83515|6361|10760|99282|634616|51|5650|1|22|3|4|255|138|46|89.26|142.81|14.28|5912.38|656.88|4105.96|6569.26|21.28|420.40|0.00|236.48|257.76|236.48|257.76|-3869.48| +2451407|76320|2451513|766|93916|83515|6361|10760|99282|634616|51|5650|38|7|15|4|143|138|54|77.16|228.39|162.15|3576.96|8756.10|4166.64|12333.06|612.92|0.00|3946.32|8756.10|9369.02|12702.42|13315.34|4589.46| +2451407|76320|2451468|2536|93916|83515|6361|10760|99282|634616|51|5650|34|7|20|1|142|138|19|10.87|21.19|5.29|302.10|100.51|206.53|402.61|2.95|2.01|193.23|98.50|101.45|291.73|294.68|-108.03| +2451407|76320|2451474|14446|93916|83515|6361|10760|99282|634616|51|5650|34|28|15|5|172|138|38|30.88|80.90|79.28|61.56|3012.64|1173.44|3074.20|150.63|0.00|1475.54|3012.64|3163.27|4488.18|4638.81|1839.20| +2451407|76320|2451451|7862|93916|83515|6361|10760|99282|634616|51|5650|16|16|2|2|265|138|55|87.45|190.64|40.03|8283.55|2201.65|4809.75|10485.20|198.14|0.00|419.10|2201.65|2399.79|2620.75|2818.89|-2608.10| +2451407|76320|2451500|6889|93916|83515|6361|10760|99282|634616|51|5650|8|26|6|5|267|138|96|13.85|27.70|19.66|771.84|1887.36|1329.60|2659.20|150.98|0.00|398.40|1887.36|2038.34|2285.76|2436.74|557.76| +2451407|76320|2451430|7396|93916|83515|6361|10760|99282|634616|51|5650|43|26|3|5|154|138|47|28.10|41.86|10.46|1475.80|491.62|1320.70|1967.42|9.83|0.00|235.94|491.62|501.45|727.56|737.39|-829.08| +2451407|76320|2451484|3692|93916|83515|6361|10760|99282|634616|51|5650|31|28|4|3|65|138|57|63.97|142.01|105.08|2105.01|5989.56|3646.29|8094.57|117.39|4312.48|2509.14|1677.08|1794.47|4186.22|4303.61|-1969.21| +2451407|76320|2451483|7729|93916|83515|6361|10760|99282|634616|51|5650|49|7|3|5|144|138|15|43.87|49.57|46.10|52.05|691.50|658.05|743.55|15.14|186.70|282.45|504.80|519.94|787.25|802.39|-153.25| +2451407|76320|2451429|10126|93916|83515|6361|10760|99282|634616|51|5650|28|10|7|3|249|138|74|30.05|82.33|22.22|4448.14|1644.28|2223.70|6092.42|16.44|1315.42|1278.72|328.86|345.30|1607.58|1624.02|-1894.84| +2451173|47364|2451265|17320|40917|915281|1292|32056|40917|915281|1292|32056|52|8|10|3|73|139|86|4.33|11.47|7.22|365.50|620.92|372.38|986.42|55.88|0.00|29.24|620.92|676.80|650.16|706.04|248.54| +2451173|47364|2451246|8758|40917|915281|1292|32056|40917|915281|1292|32056|10|8|11|2|83|139|31|62.00|161.20|64.48|2998.32|1998.88|1922.00|4997.20|43.17|919.48|749.58|1079.40|1122.57|1828.98|1872.15|-842.60| +2451173|47364|2451270|1760|40917|915281|1292|32056|40917|915281|1292|32056|34|20|14|4|297|139|92|48.93|123.30|16.02|9869.76|1473.84|4501.56|11343.60|14.73|0.00|5671.80|1473.84|1488.57|7145.64|7160.37|-3027.72| +2451173|47364|2451215|10550|40917|915281|1292|32056|40917|915281|1292|32056|14|2|13|4|95|139|5|7.04|11.89|4.87|35.10|24.35|35.20|59.45|0.00|10.47|5.35|13.88|13.88|19.23|19.23|-21.32| +2451173|47364|2451263|6721|40917|915281|1292|32056|40917|915281|1292|32056|44|8|7|4|41|139|53|32.23|93.14|24.21|3653.29|1283.13|1708.19|4936.42|89.81|0.00|2171.94|1283.13|1372.94|3455.07|3544.88|-425.06| +2451173|47364|2451243|9139|40917|915281|1292|32056|40917|915281|1292|32056|16|2|5|3|34|139|64|33.08|47.96|24.93|1473.92|1595.52|2117.12|3069.44|143.59|0.00|429.44|1595.52|1739.11|2024.96|2168.55|-521.60| +2451173|47364|2451192|7216|40917|915281|1292|32056|40917|915281|1292|32056|46|13|13|1|272|139|79|83.61|156.35|50.03|8399.28|3952.37|6605.19|12351.65|118.57|0.00|3087.32|3952.37|4070.94|7039.69|7158.26|-2652.82| +2451173|47364|2451268|13478|40917|915281|1292|32056|40917|915281|1292|32056|37|7|9|1|236|139|55|74.49|172.81|31.10|7794.05|1710.50|4096.95|9504.55|34.21|0.00|1995.95|1710.50|1744.71|3706.45|3740.66|-2386.45| +2451173|47364|2451210|14204|40917|915281|1292|32056|40917|915281|1292|32056|28|26|16|3|237|139|46|32.49|56.53|25.43|1430.60|1169.78|1494.54|2600.38|40.00|725.26|1144.02|444.52|484.52|1588.54|1628.54|-1050.02| +2451173|47364|2451201|3890|40917|915281|1292|32056|40917|915281|1292|32056|2|14|17|1|35|139|21|69.46|166.00|107.90|1220.10|2265.90|1458.66|3486.00|203.93|0.00|732.06|2265.90|2469.83|2997.96|3201.89|807.24| +2451173|47364|2451256|14821|40917|915281|1292|32056|40917|915281|1292|32056|19|8|17|2|27|139|42|71.14|92.48|41.61|2136.54|1747.62|2987.88|3884.16|0.00|0.00|660.24|1747.62|1747.62|2407.86|2407.86|-1240.26| +2451173|47364|2451287|2845|40917|915281|1292|32056|40917|915281|1292|32056|44|13|14|3|252|139|89|78.86|178.22|39.20|12372.78|3488.80|7018.54|15861.58|104.66|0.00|6661.65|3488.80|3593.46|10150.45|10255.11|-3529.74| +2451173|47364|2451270|9643|40917|915281|1292|32056|40917|915281|1292|32056|34|26|10|1|279|139|42|79.46|205.80|125.53|3371.34|5272.26|3337.32|8643.60|210.89|0.00|345.66|5272.26|5483.15|5617.92|5828.81|1934.94| +2451276|70169|2451310|12652|63405|1704417|5076|37288|40863|1306557|6818|8230|38|14|16|4|122|140|87|11.37|32.51|16.25|1414.62|1413.75|989.19|2828.37|56.55|0.00|678.60|1413.75|1470.30|2092.35|2148.90|424.56| +2451276|70169|2451320|10705|63405|1704417|5076|37288|40863|1306557|6818|8230|7|4|12|1|30|140|78|35.29|82.22|15.62|5194.80|1218.36|2752.62|6413.16|4.14|804.11|705.12|414.25|418.39|1119.37|1123.51|-2338.37| +2451276|70169|2451319|6464|63405|1704417|5076|37288|40863|1306557|6818|8230|31|14|13|5|37|140|38|38.53|108.26|51.96|2139.40|1974.48|1464.14|4113.88|98.72|0.00|1727.48|1974.48|2073.20|3701.96|3800.68|510.34| +2451276|70169|2451379|2666|63405|1704417|5076|37288|40863|1306557|6818|8230|49|19|12|2|249|140|58|66.57|153.11|102.58|2930.74|5949.64|3861.06|8880.38|178.48|0.00|1331.68|5949.64|6128.12|7281.32|7459.80|2088.58| +2451276|70169|2451342|9808|63405|1704417|5076|37288|40863|1306557|6818|8230|4|2|1|5|149|140|15|48.04|113.37|91.82|323.25|1377.30|720.60|1700.55|68.86|0.00|510.15|1377.30|1446.16|1887.45|1956.31|656.70| +2451276|70169|2451358|541|63405|1704417|5076|37288|40863|1306557|6818|8230|2|2|19|1|233|140|77|39.10|91.88|13.78|6013.70|1061.06|3010.70|7074.76|74.27|0.00|1272.81|1061.06|1135.33|2333.87|2408.14|-1949.64| +2451276|70169|2451282|1598|63405|1704417|5076|37288|40863|1306557|6818|8230|37|16|19|4|159|140|84|35.57|49.79|19.91|2509.92|1672.44|2987.88|4182.36|133.79|0.00|1463.28|1672.44|1806.23|3135.72|3269.51|-1315.44| +2451276|70169|2451366|7411|63405|1704417|5076|37288|40863|1306557|6818|8230|44|26|15|1|77|140|96|2.23|4.79|4.69|9.60|450.24|214.08|459.84|31.51|0.00|8.64|450.24|481.75|458.88|490.39|236.16| +2451276|70169|2451333|8707|63405|1704417|5076|37288|40863|1306557|6818|8230|55|26|10|3|189|140|74|3.13|7.35|3.67|272.32|271.58|231.62|543.90|10.86|0.00|195.36|271.58|282.44|466.94|477.80|39.96| +2451522|36372|2451625|1262|44953|692145|2184|14623|7461|1798094|1540|18732|20|14|2|4|126|141|87|74.94|110.16|76.01|2971.05|6612.87|6519.78|9583.92|66.12|0.00|1149.27|6612.87|6678.99|7762.14|7828.26|93.09| +2451522|36372|2451524|4552|44953|692145|2184|14623|7461|1798094|1540|18732|4|26|18|2|153|141|24|28.83|76.97|59.26|425.04|1422.24|691.92|1847.28|113.77|0.00|683.28|1422.24|1536.01|2105.52|2219.29|730.32| +2451522|36372|2451625|7510|44953|692145|2184|14623|7461|1798094|1540|18732|26|19|19|3|74|141|11|15.19|40.25|23.34|186.01|256.74|167.09|442.75|15.40|0.00|17.71|256.74|272.14|274.45|289.85|89.65| +2451522|36372|2451570|2588|44953|692145|2184|14623|7461|1798094|1540|18732|1|4|18|5|62|141|74|5.52|6.62|4.56|152.44|337.44|408.48|489.88|23.62|0.00|34.04|337.44|361.06|371.48|395.10|-71.04| +2451522|36372|2451562|17714|44953|692145|2184|14623|7461|1798094|1540|18732|38|22|4|1|138|141|75|27.37|59.94|25.77|2562.75|1932.75|2052.75|4495.50|154.62|0.00|1213.50|1932.75|2087.37|3146.25|3300.87|-120.00| +2451522|36372|2451588|14450|44953|692145|2184|14623|7461|1798094|1540|18732|37|4|7|1|255|141|3|16.66|28.65|19.19|28.38|57.57|49.98|85.95|2.87|0.00|12.87|57.57|60.44|70.44|73.31|7.59| +2451522|36372|2451593|15673|44953|692145|2184|14623|7461|1798094|1540|18732|37|20|3|2|24|141|37|19.78|45.88|6.42|1460.02|237.54|731.86|1697.56|19.00|0.00|50.69|237.54|256.54|288.23|307.23|-494.32| +2451522|36372|2451541|12982|44953|692145|2184|14623|7461|1798094|1540|18732|28|13|2|4|36|141|45|1.66|3.05|1.40|74.25|63.00|74.70|137.25|5.04|0.00|10.80|63.00|68.04|73.80|78.84|-11.70| +2451522|36372|2451593|11134|44953|692145|2184|14623|7461|1798094|1540|18732|56|26|2|5|15|141|70|44.92|125.77|114.45|792.40|8011.50|3144.40|8803.90|560.80|0.00|3345.30|8011.50|8572.30|11356.80|11917.60|4867.10| +2451522|36372|2451638|13561|44953|692145|2184|14623|7461|1798094|1540|18732|43|20|10|4|68|141|25|90.59|228.28|86.74|3538.50|2168.50|2264.75|5707.00|130.11|0.00|456.50|2168.50|2298.61|2625.00|2755.11|-96.25| +2451522|36372|2451628|8611|44953|692145|2184|14623|7461|1798094|1540|18732|43|4|17|3|299|141|38|38.48|114.67|41.28|2788.82|1568.64|1462.24|4357.46|141.17|0.00|1699.36|1568.64|1709.81|3268.00|3409.17|106.40| +2451522|36372|2451599|10580|44953|692145|2184|14623|7461|1798094|1540|18732|37|13|5|1|234|141|14|6.52|16.88|10.63|87.50|148.82|91.28|236.32|4.46|0.00|63.70|148.82|153.28|212.52|216.98|57.54| +2451522|36372|2451626|4174|44953|692145|2184|14623|7461|1798094|1540|18732|46|25|12|1|130|141|12|46.93|109.81|105.41|52.80|1264.92|563.16|1317.72|41.11|442.72|632.40|822.20|863.31|1454.60|1495.71|259.04| +2451463|59944|2451553|12019|75797|1297229|4708|26540|95939|1630780|6883|16315|10|2|8|4|163|142|68|59.12|112.32|44.92|4583.20|3054.56|4020.16|7637.76|183.27|0.00|3054.56|3054.56|3237.83|6109.12|6292.39|-965.60| +2451463|59944|2451553|14630|75797|1297229|4708|26540|95939|1630780|6883|16315|14|28|9|1|283|142|78|13.98|32.15|13.82|1429.74|1077.96|1090.44|2507.70|43.11|0.00|475.80|1077.96|1121.07|1553.76|1596.87|-12.48| +2451463|59944|2451474|2473|75797|1297229|4708|26540|95939|1630780|6883|16315|14|13|6|5|154|142|39|43.85|71.91|38.11|1318.20|1486.29|1710.15|2804.49|118.90|0.00|532.74|1486.29|1605.19|2019.03|2137.93|-223.86| +2451463|59944|2451507|11299|75797|1297229|4708|26540|95939|1630780|6883|16315|46|13|20|3|196|142|52|2.87|3.09|2.93|8.32|152.36|149.24|160.68|13.71|0.00|10.92|152.36|166.07|163.28|176.99|3.12| +2451463|59944|2451519|9601|75797|1297229|4708|26540|95939|1630780|6883|16315|46|25|15|4|296|142|78|42.17|125.66|106.81|1470.30|8331.18|3289.26|9801.48|83.31|0.00|195.78|8331.18|8414.49|8526.96|8610.27|5041.92| +2451463|59944|2451551|11623|75797|1297229|4708|26540|95939|1630780|6883|16315|56|4|4|3|269|142|6|22.31|32.34|15.19|102.90|91.14|133.86|194.04|7.29|0.00|56.22|91.14|98.43|147.36|154.65|-42.72| +2451463|59944|2451485|10672|75797|1297229|4708|26540|95939|1630780|6883|16315|52|16|8|5|29|142|5|9.97|10.56|4.75|29.05|23.75|49.85|52.80|1.18|0.00|10.00|23.75|24.93|33.75|34.93|-26.10| +2451463|59944|2451493|1108|75797|1297229|4708|26540|95939|1630780|6883|16315|55|1|1|4|144|142|70|55.02|79.22|16.63|4381.30|1164.10|3851.40|5545.40|81.48|0.00|387.80|1164.10|1245.58|1551.90|1633.38|-2687.30| +2451463|59944|2451558|4195|75797|1297229|4708|26540|95939|1630780|6883|16315|28|1|7|3|56|142|71|62.04|67.00|8.71|4138.59|618.41|4404.84|4757.00|49.47|0.00|428.13|618.41|667.88|1046.54|1096.01|-3786.43| +2450865|48676|2450952|17311|48483|754001|5119|47884|18240|1747351|5238|19203|56|26|12|2|292|143|64|4.19|10.81|7.13|235.52|456.32|268.16|691.84|36.50|0.00|145.28|456.32|492.82|601.60|638.10|188.16| +2450865|48676|2450952|17068|48483|754001|5119|47884|18240|1747351|5238|19203|28|26|10|4|258|143|93|81.48|204.51|14.31|17688.60|1330.83|7577.64|19019.43|39.92|0.00|3993.42|1330.83|1370.75|5324.25|5364.17|-6246.81| +2450865|48676|2450967|12394|48483|754001|5119|47884|18240|1747351|5238|19203|56|16|9|4|129|143|62|71.91|210.69|202.26|522.66|12540.12|4458.42|13062.78|250.80|0.00|5485.76|12540.12|12790.92|18025.88|18276.68|8081.70| +2450865|48676|2450980|16090|48483|754001|5119|47884|18240|1747351|5238|19203|56|1|13|2|284|143|32|83.92|227.42|13.64|6840.96|436.48|2685.44|7277.44|39.28|0.00|3201.92|436.48|475.76|3638.40|3677.68|-2248.96| +2450865|48676|2450896|10312|48483|754001|5119|47884|18240|1747351|5238|19203|25|1|16|4|159|143|33|33.26|64.52|27.09|1235.19|893.97|1097.58|2129.16|35.75|0.00|127.71|893.97|929.72|1021.68|1057.43|-203.61| +2450865|48676|2450899|6703|48483|754001|5119|47884|18240|1747351|5238|19203|56|14|20|4|201|143|19|91.91|163.59|75.25|1678.46|1429.75|1746.29|3108.21|42.89|0.00|497.23|1429.75|1472.64|1926.98|1969.87|-316.54| +2450865|48676|2450908|13696|48483|754001|5119|47884|18240|1747351|5238|19203|40|7|14|2|20|143|81|34.99|62.63|43.84|1521.99|3551.04|2834.19|5073.03|71.02|0.00|101.25|3551.04|3622.06|3652.29|3723.31|716.85| +2450865|48676|2450896|11968|48483|754001|5119|47884|18240|1747351|5238|19203|38|16|18|2|269|143|39|36.47|97.01|35.89|2383.68|1399.71|1422.33|3783.39|0.00|0.00|1702.35|1399.71|1399.71|3102.06|3102.06|-22.62| +2450865|48676|2450947|11644|48483|754001|5119|47884|18240|1747351|5238|19203|26|25|15|4|296|143|82|19.02|34.99|17.84|1406.30|1462.88|1559.64|2869.18|87.77|0.00|400.98|1462.88|1550.65|1863.86|1951.63|-96.76| +2450865|48676|2450929|14563|48483|754001|5119|47884|18240|1747351|5238|19203|2|22|15|3|238|143|79|11.97|27.53|9.91|1391.98|782.89|945.63|2174.87|28.18|78.28|43.45|704.61|732.79|748.06|776.24|-241.02| +2450865|48676|2450965|4063|48483|754001|5119|47884|18240|1747351|5238|19203|55|19|15|5|272|143|64|77.83|87.94|43.09|2870.40|2757.76|4981.12|5628.16|55.15|0.00|900.48|2757.76|2812.91|3658.24|3713.39|-2223.36| +2450865|48676|2450941|11176|48483|754001|5119|47884|18240|1747351|5238|19203|37|14|17|1|88|143|93|44.27|86.76|78.08|807.24|7261.44|4117.11|8068.68|290.45|0.00|1451.73|7261.44|7551.89|8713.17|9003.62|3144.33| +2450865|48676|2450893|415|48483|754001|5119|47884|18240|1747351|5238|19203|37|13|5|5|98|143|30|90.50|174.66|113.52|1834.20|3405.60|2715.00|5239.80|4.76|3167.20|2148.30|238.40|243.16|2386.70|2391.46|-2476.60| +2452386|42051|2452464|8028|78725|1440340|355|7970|2997|582809|2015|13630|49|13|9|1|193|144|77|8.94|26.46|1.32|1935.78|101.64|688.38|2037.42|3.04|0.00|40.04|101.64|104.68|141.68|144.72|-586.74| +2452386|42051|2452427|15504|78725|1440340|355|7970|2997|582809|2015|13630|37|3|12|5|25|144|23|91.34|203.68|85.54|2717.22|1967.42|2100.82|4684.64|157.39|0.00|936.79|1967.42|2124.81|2904.21|3061.60|-133.40| +2452386|42051|2452487|216|78725|1440340|355|7970|2997|582809|2015|13630|49|7|3|2|151|144|21|4.86|8.35|0.41|166.74|8.61|102.06|175.35|0.51|0.00|13.86|8.61|9.12|22.47|22.98|-93.45| +2452386|42051|2452387|11215|78725|1440340|355|7970|2997|582809|2015|13630|31|3|15|1|274|144|31|12.67|33.32|22.65|330.77|702.15|392.77|1032.92|21.06|0.00|51.46|702.15|723.21|753.61|774.67|309.38| +2452386|42051|2452395|6241|78725|1440340|355|7970|2997|582809|2015|13630|33|12|5|1|160|144|31|11.77|15.18|9.71|169.57|301.01|364.87|470.58|6.02|0.00|42.16|301.01|307.03|343.17|349.19|-63.86| +2452386|42051|2452437|15019|78725|1440340|355|7970|2997|582809|2015|13630|36|27|10|1|32|144|50|95.12|155.99|49.91|5304.00|2495.50|4756.00|7799.50|49.91|0.00|3743.50|2495.50|2545.41|6239.00|6288.91|-2260.50| +2452386|42051|2452471|12949|78725|1440340|355|7970|2997|582809|2015|13630|18|18|19|4|219|144|75|9.65|15.44|5.40|753.00|405.00|723.75|1158.00|24.30|0.00|57.75|405.00|429.30|462.75|487.05|-318.75| +2452386|42051|2452405|13543|78725|1440340|355|7970|2997|582809|2015|13630|19|3|7|2|144|144|62|51.50|87.55|18.38|4288.54|1139.56|3193.00|5428.10|68.37|0.00|759.50|1139.56|1207.93|1899.06|1967.43|-2053.44| +2452386|42051|2452443|14262|78725|1440340|355|7970|2997|582809|2015|13630|18|19|14|5|156|144|41|6.88|11.35|11.35|0.00|465.35|282.08|465.35|27.92|0.00|32.39|465.35|493.27|497.74|525.66|183.27| +2452386|42051|2452434|2893|78725|1440340|355|7970|2997|582809|2015|13630|3|21|10|4|98|144|73|84.75|207.63|99.66|7881.81|7275.18|6186.75|15156.99|582.01|0.00|3637.59|7275.18|7857.19|10912.77|11494.78|1088.43| +2452386|42051|2452449|12606|78725|1440340|355|7970|2997|582809|2015|13630|13|18|1|5|142|144|30|54.48|71.91|22.29|1488.60|668.70|1634.40|2157.30|40.12|0.00|819.60|668.70|708.82|1488.30|1528.42|-965.70| +2452386|42051|2452485|8877|78725|1440340|355|7970|2997|582809|2015|13630|49|6|18|3|168|144|73|89.92|196.02|3.92|14023.30|286.16|6564.16|14309.46|2.74|240.37|4006.24|45.79|48.53|4052.03|4054.77|-6518.37| +2452386|42051|2452447|13167|78725|1440340|355|7970|2997|582809|2015|13630|36|19|12|4|136|144|64|4.02|6.75|2.83|250.88|181.12|257.28|432.00|7.24|0.00|42.88|181.12|188.36|224.00|231.24|-76.16| +2451068|14277|2451102|3697|52028|1029431|3634|34237|20211|1754034|5693|27496|31|20|19|4|229|145|76|59.50|95.79|83.33|946.96|6333.08|4522.00|7280.04|63.33|0.00|1237.28|6333.08|6396.41|7570.36|7633.69|1811.08| +2451068|14277|2451120|12298|52028|1029431|3634|34237|20211|1754034|5693|27496|50|14|13|3|203|145|67|9.11|9.56|5.16|294.80|345.72|610.37|640.52|3.45|0.00|198.32|345.72|349.17|544.04|547.49|-264.65| +2451068|14277|2451074|17390|52028|1029431|3634|34237|20211|1754034|5693|27496|43|22|9|1|153|145|100|38.46|111.53|14.49|9704.00|1449.00|3846.00|11153.00|28.98|0.00|2342.00|1449.00|1477.98|3791.00|3819.98|-2397.00| +2451068|14277|2451185|3716|52028|1029431|3634|34237|20211|1754034|5693|27496|10|10|9|4|299|145|50|93.97|205.79|109.06|4836.50|5453.00|4698.50|10289.50|104.69|4144.28|411.50|1308.72|1413.41|1720.22|1824.91|-3389.78| +2451068|14277|2451117|1720|52028|1029431|3634|34237|20211|1754034|5693|27496|16|26|2|2|82|145|26|3.60|3.85|0.96|75.14|24.96|93.60|100.10|2.24|0.00|40.04|24.96|27.20|65.00|67.24|-68.64| +2451068|14277|2451185|14012|52028|1029431|3634|34237|20211|1754034|5693|27496|46|14|12|3|57|145|55|19.84|51.98|31.18|1144.00|1714.90|1091.20|2858.90|85.74|0.00|1343.65|1714.90|1800.64|3058.55|3144.29|623.70| +2451068|14277|2451088|7774|52028|1029431|3634|34237|20211|1754034|5693|27496|40|26|20|1|9|145|51|77.38|214.34|137.17|3935.67|6995.67|3946.38|10931.34|629.61|0.00|4591.02|6995.67|7625.28|11586.69|12216.30|3049.29| +2451068|14277|2451110|17458|52028|1029431|3634|34237|20211|1754034|5693|27496|34|14|8|3|248|145|21|80.69|238.84|40.60|4163.04|852.60|1694.49|5015.64|34.10|0.00|852.60|852.60|886.70|1705.20|1739.30|-841.89| +2451068|14277|2451168|13184|52028|1029431|3634|34237|20211|1754034|5693|27496|58|8|20|5|200|145|16|89.39|151.96|110.93|656.48|1774.88|1430.24|2431.36|124.24|0.00|170.08|1774.88|1899.12|1944.96|2069.20|344.64| +2451785|61675|2451879|11312|46466|649270|874|48878|59133|444907|5690|2094|17|19|4|5|209|146|83|29.39|76.12|22.83|4423.07|1894.89|2439.37|6317.96|113.69|0.00|2337.28|1894.89|2008.58|4232.17|4345.86|-544.48| +2451785|61675|2451902|1121|46466|649270|874|48878|59133|444907|5690|2094|11|29|17|1|215|146|95|20.77|43.40|42.09|124.45|3998.55|1973.15|4123.00|39.98|0.00|164.35|3998.55|4038.53|4162.90|4202.88|2025.40| +2451785|61675|2451893|9053|46466|649270|874|48878|59133|444907|5690|2094|29|2|19|1|203|146|58|24.01|51.14|2.55|2818.22|147.90|1392.58|2966.12|10.35|0.00|948.88|147.90|158.25|1096.78|1107.13|-1244.68| +2451785|61675|2451878|15176|46466|649270|874|48878|59133|444907|5690|2094|50|14|10|2|134|146|49|56.13|107.20|24.65|4044.95|1207.85|2750.37|5252.80|60.39|0.00|1470.49|1207.85|1268.24|2678.34|2738.73|-1542.52| +2451785|61675|2451900|13957|46466|649270|874|48878|59133|444907|5690|2094|23|13|2|2|292|146|85|39.05|40.22|7.23|2804.15|614.55|3319.25|3418.70|36.87|0.00|717.40|614.55|651.42|1331.95|1368.82|-2704.70| +2451785|61675|2451802|1115|46466|649270|874|48878|59133|444907|5690|2094|38|29|10|3|203|146|88|24.18|35.54|28.07|657.36|2470.16|2127.84|3127.52|28.65|1753.81|1219.68|716.35|745.00|1936.03|1964.68|-1411.49| +2451785|61675|2451803|17840|46466|649270|874|48878|59133|444907|5690|2094|47|14|12|4|133|146|29|30.18|37.12|26.72|301.60|774.88|875.22|1076.48|69.73|0.00|42.92|774.88|844.61|817.80|887.53|-100.34| +2451785|61675|2451867|5261|46466|649270|874|48878|59133|444907|5690|2094|11|11|16|3|193|146|13|36.18|101.66|55.91|594.75|726.83|470.34|1321.58|43.60|0.00|647.53|726.83|770.43|1374.36|1417.96|256.49| +2451785|61675|2451861|14900|46466|649270|874|48878|59133|444907|5690|2094|25|11|19|2|135|146|62|19.79|47.49|11.87|2208.44|735.94|1226.98|2944.38|44.15|0.00|823.98|735.94|780.09|1559.92|1604.07|-491.04| +2451785|61675|2451875|185|46466|649270|874|48878|59133|444907|5690|2094|47|2|3|4|65|146|67|28.65|79.64|46.19|2241.15|3094.73|1919.55|5335.88|123.78|0.00|853.58|3094.73|3218.51|3948.31|4072.09|1175.18| +2451785|61675|2451839|445|46466|649270|874|48878|59133|444907|5690|2094|41|14|5|2|59|146|58|81.38|116.37|60.51|3239.88|3509.58|4720.04|6749.46|140.38|0.00|607.26|3509.58|3649.96|4116.84|4257.22|-1210.46| +2451785|61675|2451865|17881|46466|649270|874|48878|59133|444907|5690|2094|35|17|20|3|109|146|47|28.66|77.66|66.01|547.55|3102.47|1347.02|3650.02|217.17|0.00|1751.69|3102.47|3319.64|4854.16|5071.33|1755.45| +2451785|61675|2451830|11513|46466|649270|874|48878|59133|444907|5690|2094|35|13|19|3|237|146|43|8.54|15.37|7.53|337.12|323.79|367.22|660.91|16.18|0.00|105.35|323.79|339.97|429.14|445.32|-43.43| +2451824|24644|2451847|8965|72367|462290|1813|11336|82689|1214313|3905|46628|23|7|6|1|153|147|64|1.62|2.49|1.64|54.40|104.96|103.68|159.36|5.24|0.00|31.36|104.96|110.20|136.32|141.56|1.28| +2451824|24644|2451940|6566|72367|462290|1813|11336|82689|1214313|3905|46628|47|7|7|5|20|147|92|41.64|117.00|52.65|5920.20|4843.80|3830.88|10764.00|48.43|0.00|2045.16|4843.80|4892.23|6888.96|6937.39|1012.92| +2451824|24644|2451853|14269|72367|462290|1813|11336|82689|1214313|3905|46628|11|11|16|3|283|147|97|68.73|155.32|90.08|6328.28|8737.76|6666.81|15066.04|87.37|0.00|6628.98|8737.76|8825.13|15366.74|15454.11|2070.95| +2451824|24644|2451904|236|72367|462290|1813|11336|82689|1214313|3905|46628|23|2|5|5|205|147|14|19.30|47.86|30.63|241.22|428.82|270.20|670.04|1.24|304.46|314.86|124.36|125.60|439.22|440.46|-145.84| +2451824|24644|2451832|9421|72367|462290|1813|11336|82689|1214313|3905|46628|1|13|13|4|295|147|38|38.83|85.81|80.66|195.70|3065.08|1475.54|3260.78|153.25|0.00|97.66|3065.08|3218.33|3162.74|3315.99|1589.54| +2451824|24644|2451827|15385|72367|462290|1813|11336|82689|1214313|3905|46628|32|26|1|3|72|147|58|55.64|105.15|5.25|5794.20|304.50|3227.12|6098.70|27.40|0.00|2439.48|304.50|331.90|2743.98|2771.38|-2922.62| +2451824|24644|2451885|9596|72367|462290|1813|11336|82689|1214313|3905|46628|37|17|16|1|16|147|61|75.98|171.71|84.13|5342.38|5131.93|4634.78|10474.31|256.59|0.00|4294.40|5131.93|5388.52|9426.33|9682.92|497.15| +2451824|24644|2451923|371|72367|462290|1813|11336|82689|1214313|3905|46628|7|1|1|3|188|147|72|13.65|23.75|14.48|667.44|1042.56|982.80|1710.00|58.38|208.51|700.56|834.05|892.43|1534.61|1592.99|-148.75| +2451824|24644|2451868|6662|72367|462290|1813|11336|82689|1214313|3905|46628|55|8|5|5|72|147|30|30.92|61.22|28.16|991.80|844.80|927.60|1836.60|25.34|0.00|128.40|844.80|870.14|973.20|998.54|-82.80| +2451824|24644|2451884|6434|72367|462290|1813|11336|82689|1214313|3905|46628|44|14|13|3|173|147|73|52.47|123.30|96.17|1980.49|7020.41|3830.31|9000.90|351.02|0.00|3690.15|7020.41|7371.43|10710.56|11061.58|3190.10| +2451824|24644|2451862|5291|72367|462290|1813|11336|82689|1214313|3905|46628|59|7|15|1|60|147|25|75.49|98.89|8.90|2249.75|222.50|1887.25|2472.25|12.77|40.05|346.00|182.45|195.22|528.45|541.22|-1704.80| +2451824|24644|2451853|5635|72367|462290|1813|11336|82689|1214313|3905|46628|43|13|2|2|292|147|19|75.50|221.97|128.74|1771.37|2446.06|1434.50|4217.43|220.14|0.00|1813.36|2446.06|2666.20|4259.42|4479.56|1011.56| +2451824|24644|2451848|3431|72367|462290|1813|11336|82689|1214313|3905|46628|23|29|11|4|250|147|89|24.60|58.05|6.96|4547.01|619.44|2189.40|5166.45|43.36|0.00|1704.35|619.44|662.80|2323.79|2367.15|-1569.96| +2452278|80559|2452283|16039|72570|1224887|5651|25215|25264|1016460|2307|40271|54|27|14|1|208|148|26|19.02|43.17|13.38|774.54|347.88|494.52|1122.42|31.30|0.00|100.88|347.88|379.18|448.76|480.06|-146.64| +2452278|80559|2452394|8241|72570|1224887|5651|25215|25264|1016460|2307|40271|9|1|4|2|153|148|52|66.94|101.07|39.41|3206.32|2049.32|3480.88|5255.64|102.46|0.00|472.68|2049.32|2151.78|2522.00|2624.46|-1431.56| +2452278|80559|2452298|9523|72570|1224887|5651|25215|25264|1016460|2307|40271|9|12|7|5|106|148|81|52.70|125.42|104.09|1727.73|8431.29|4268.70|10159.02|505.87|0.00|2031.48|8431.29|8937.16|10462.77|10968.64|4162.59| +2452278|80559|2452281|9307|72570|1224887|5651|25215|25264|1016460|2307|40271|51|21|16|1|56|148|6|73.22|199.15|119.49|477.96|716.94|439.32|1194.90|50.18|0.00|226.98|716.94|767.12|943.92|994.10|277.62| +2452278|80559|2452313|7068|72570|1224887|5651|25215|25264|1016460|2307|40271|18|9|8|4|60|148|42|77.60|176.92|102.61|3121.02|4309.62|3259.20|7430.64|0.00|0.00|2080.26|4309.62|4309.62|6389.88|6389.88|1050.42| +2452278|80559|2452330|13345|72570|1224887|5651|25215|25264|1016460|2307|40271|55|18|3|4|5|148|55|87.65|255.93|174.03|4504.50|9571.65|4820.75|14076.15|66.04|7370.17|5630.35|2201.48|2267.52|7831.83|7897.87|-2619.27| +2452278|80559|2452300|9567|72570|1224887|5651|25215|25264|1016460|2307|40271|18|7|16|1|56|148|76|87.68|207.80|116.36|6949.44|8843.36|6663.68|15792.80|795.90|0.00|7106.76|8843.36|9639.26|15950.12|16746.02|2179.68| +2452278|80559|2452299|15487|72570|1224887|5651|25215|25264|1016460|2307|40271|42|24|13|4|75|148|75|50.58|95.09|82.72|927.75|6204.00|3793.50|7131.75|186.12|0.00|1425.75|6204.00|6390.12|7629.75|7815.87|2410.50| +2452278|80559|2452378|13525|72570|1224887|5651|25215|25264|1016460|2307|40271|51|9|14|3|62|148|2|37.21|94.88|83.49|22.78|166.98|74.42|189.76|2.27|53.43|94.88|113.55|115.82|208.43|210.70|39.13| +2452278|80559|2452310|8016|72570|1224887|5651|25215|25264|1016460|2307|40271|15|25|18|3|201|148|28|53.87|142.75|44.25|2758.00|1239.00|1508.36|3997.00|99.12|0.00|1398.88|1239.00|1338.12|2637.88|2737.00|-269.36| +2452278|80559|2452397|4131|72570|1224887|5651|25215|25264|1016460|2307|40271|54|6|6|2|65|148|75|69.71|71.10|62.56|640.50|4692.00|5228.25|5332.50|71.31|3800.52|1706.25|891.48|962.79|2597.73|2669.04|-4336.77| +2452278|80559|2452377|12385|72570|1224887|5651|25215|25264|1016460|2307|40271|48|7|1|1|48|148|71|64.64|85.32|29.00|3998.72|2059.00|4589.44|6057.72|144.13|0.00|1574.78|2059.00|2203.13|3633.78|3777.91|-2530.44| +2452278|80559|2452328|17436|72570|1224887|5651|25215|25264|1016460|2307|40271|15|7|10|3|59|148|98|73.78|110.67|52.01|5748.68|5096.98|7230.44|10845.66|407.75|0.00|650.72|5096.98|5504.73|5747.70|6155.45|-2133.46| +2452278|80559|2452299|5767|72570|1224887|5651|25215|25264|1016460|2307|40271|13|13|7|5|240|148|79|98.03|134.30|37.60|7639.30|2970.40|7744.37|10609.70|0.00|0.00|4561.46|2970.40|2970.40|7531.86|7531.86|-4773.97| +2452278|80559|2452290|2607|72570|1224887|5651|25215|25264|1016460|2307|40271|49|15|18|3|68|148|40|37.50|58.12|39.52|744.00|1580.80|1500.00|2324.80|11.85|1343.68|232.40|237.12|248.97|469.52|481.37|-1262.88| +2452278|80559|2452347|6765|72570|1224887|5651|25215|25264|1016460|2307|40271|15|12|1|1|14|148|53|81.67|113.52|28.38|4512.42|1504.14|4328.51|6016.56|0.00|0.00|1323.41|1504.14|1504.14|2827.55|2827.55|-2824.37| +2452026|73416|2452058|12235|52162|1509084|406|29375|92842|110073|1788|21110|15|29|19|3|177|149|66|67.22|190.23|57.06|8789.22|3765.96|4436.52|12555.18|188.29|0.00|2259.84|3765.96|3954.25|6025.80|6214.09|-670.56| +2452026|73416|2452103|4825|52162|1509084|406|29375|92842|110073|1788|21110|5|21|16|1|146|149|43|6.46|11.49|6.20|227.47|266.60|277.78|494.07|5.33|0.00|212.42|266.60|271.93|479.02|484.35|-11.18| +2452026|73416|2452132|9105|52162|1509084|406|29375|92842|110073|1788|21110|39|7|8|2|286|149|45|8.61|10.41|8.84|70.65|397.80|387.45|468.45|19.89|0.00|9.00|397.80|417.69|406.80|426.69|10.35| +2452026|73416|2452045|10727|52162|1509084|406|29375|92842|110073|1788|21110|21|13|3|2|228|149|33|96.61|208.67|98.07|3649.80|3236.31|3188.13|6886.11|64.72|0.00|2341.02|3236.31|3301.03|5577.33|5642.05|48.18| +2452026|73416|2452136|15803|52162|1509084|406|29375|92842|110073|1788|21110|57|3|18|1|124|149|26|31.72|80.56|63.64|439.92|1654.64|824.72|2094.56|33.09|0.00|230.36|1654.64|1687.73|1885.00|1918.09|829.92| +2452026|73416|2452095|17695|52162|1509084|406|29375|92842|110073|1788|21110|31|13|8|4|190|149|54|38.77|51.17|23.02|1520.10|1243.08|2093.58|2763.18|62.15|0.00|1049.76|1243.08|1305.23|2292.84|2354.99|-850.50| +2452026|73416|2452134|11739|52162|1509084|406|29375|92842|110073|1788|21110|1|23|13|4|255|149|42|98.65|155.86|34.28|5106.36|1439.76|4143.30|6546.12|55.43|647.89|3142.02|791.87|847.30|3933.89|3989.32|-3351.43| +2452026|73416|2452094|6223|52162|1509084|406|29375|92842|110073|1788|21110|37|5|17|1|262|149|88|7.73|20.79|1.66|1683.44|146.08|680.24|1829.52|1.46|0.00|36.08|146.08|147.54|182.16|183.62|-534.16| +2452026|73416|2452130|14365|52162|1509084|406|29375|92842|110073|1788|21110|33|11|5|4|93|149|93|95.37|143.05|111.57|2927.64|10376.01|8869.41|13303.65|76.78|6536.88|5321.46|3839.13|3915.91|9160.59|9237.37|-5030.28| +2452233|70530|2452295|9553|89648|385896|4991|48086|89648|385896|4991|48086|13|21|15|2|136|150|13|57.27|166.65|96.65|910.00|1256.45|744.51|2166.45|62.82|0.00|433.29|1256.45|1319.27|1689.74|1752.56|511.94| +2452233|70530|2452271|1937|89648|385896|4991|48086|89648|385896|4991|48086|39|23|7|4|221|150|69|52.09|105.22|21.04|5808.42|1451.76|3594.21|7260.18|14.51|0.00|2177.64|1451.76|1466.27|3629.40|3643.91|-2142.45| +2452233|70530|2452278|8217|89648|385896|4991|48086|89648|385896|4991|48086|3|29|8|1|264|150|91|44.81|94.99|47.49|4322.50|4321.59|4077.71|8644.09|172.86|0.00|1209.39|4321.59|4494.45|5530.98|5703.84|243.88| +2452233|70530|2452308|2547|89648|385896|4991|48086|89648|385896|4991|48086|25|25|3|5|290|150|43|50.36|88.13|53.75|1478.34|2311.25|2165.48|3789.59|184.90|0.00|264.88|2311.25|2496.15|2576.13|2761.03|145.77| +2452233|70530|2452300|12485|89648|385896|4991|48086|89648|385896|4991|48086|7|29|6|2|198|150|3|36.53|81.46|78.20|9.78|234.60|109.59|244.38|9.38|0.00|122.19|234.60|243.98|356.79|366.17|125.01| +2452233|70530|2452351|7499|89648|385896|4991|48086|89648|385896|4991|48086|43|27|16|3|87|150|1|30.72|90.00|48.60|41.40|48.60|30.72|90.00|3.88|0.00|20.70|48.60|52.48|69.30|73.18|17.88| +2452233|70530|2452273|17169|89648|385896|4991|48086|89648|385896|4991|48086|11|13|18|4|3|150|53|61.79|164.36|37.80|6707.68|2003.40|3274.87|8711.08|40.06|0.00|2700.35|2003.40|2043.46|4703.75|4743.81|-1271.47| +2452233|70530|2452305|7593|89648|385896|4991|48086|89648|385896|4991|48086|19|7|8|4|263|150|28|69.76|76.03|14.44|1724.52|404.32|1953.28|2128.84|12.33|157.68|425.60|246.64|258.97|672.24|684.57|-1706.64| +2452233|70530|2452351|307|89648|385896|4991|48086|89648|385896|4991|48086|49|17|13|2|141|150|37|98.89|142.40|68.35|2739.85|2528.95|3658.93|5268.80|0.00|252.89|1369.74|2276.06|2276.06|3645.80|3645.80|-1382.87| +2452233|70530|2452254|11821|89648|385896|4991|48086|89648|385896|4991|48086|27|15|2|1|260|150|46|66.21|78.12|36.71|1904.86|1688.66|3045.66|3593.52|38.50|726.12|1760.42|962.54|1001.04|2722.96|2761.46|-2083.12| +2452233|70530|2452246|3265|89648|385896|4991|48086|89648|385896|4991|48086|19|23|11|3|235|150|100|96.52|103.27|58.86|4441.00|5886.00|9652.00|10327.00|176.58|0.00|3304.00|5886.00|6062.58|9190.00|9366.58|-3766.00| +2452233|70530|2452285|12505|89648|385896|4991|48086|89648|385896|4991|48086|51|21|9|2|155|150|100|88.42|104.33|44.86|5947.00|4486.00|8842.00|10433.00|17.04|2781.32|3025.00|1704.68|1721.72|4729.68|4746.72|-7137.32| +2452233|70530|2452237|1865|89648|385896|4991|48086|89648|385896|4991|48086|43|23|12|4|199|150|95|26.98|32.37|22.33|953.80|2121.35|2563.10|3075.15|0.00|0.00|1075.40|2121.35|2121.35|3196.75|3196.75|-441.75| +2452375|40728|2452426|816|72525|810179|6680|5934|57988|1517660|6190|39891|42|12|11|1|291|151|79|21.79|47.28|25.05|1756.17|1978.95|1721.41|3735.12|118.73|0.00|1531.02|1978.95|2097.68|3509.97|3628.70|257.54| +2452375|40728|2452475|8107|72525|810179|6680|5934|57988|1517660|6190|39891|45|1|9|5|198|151|90|67.31|183.08|16.47|14994.90|1482.30|6057.90|16477.20|88.93|0.00|5436.90|1482.30|1571.23|6919.20|7008.13|-4575.60| +2452375|40728|2452417|4458|72525|810179|6680|5934|57988|1517660|6190|39891|36|1|1|1|91|151|24|30.92|57.82|5.78|1248.96|138.72|742.08|1387.68|0.00|0.00|652.08|138.72|138.72|790.80|790.80|-603.36| +2452375|40728|2452433|13675|72525|810179|6680|5934|57988|1517660|6190|39891|24|25|7|1|115|151|80|65.44|184.54|140.25|3543.20|11220.00|5235.20|14763.20|224.40|0.00|3838.40|11220.00|11444.40|15058.40|15282.80|5984.80| +2452375|40728|2452380|14877|72525|810179|6680|5934|57988|1517660|6190|39891|27|21|8|3|205|151|57|78.36|171.60|85.80|4890.60|4890.60|4466.52|9781.20|129.11|3276.70|4792.56|1613.90|1743.01|6406.46|6535.57|-2852.62| +2452375|40728|2452462|9379|72525|810179|6680|5934|57988|1517660|6190|39891|7|1|18|5|22|151|55|5.84|10.04|5.32|259.60|292.60|321.20|552.20|6.67|125.81|115.50|166.79|173.46|282.29|288.96|-154.41| +2452375|40728|2452414|14551|72525|810179|6680|5934|57988|1517660|6190|39891|36|30|15|2|29|151|94|84.50|92.10|74.60|1645.00|7012.40|7943.00|8657.40|210.37|0.00|1731.48|7012.40|7222.77|8743.88|8954.25|-930.60| +2452375|40728|2452470|6343|72525|810179|6680|5934|57988|1517660|6190|39891|49|18|3|2|201|151|36|71.63|73.77|24.34|1779.48|876.24|2578.68|2655.72|52.57|0.00|531.00|876.24|928.81|1407.24|1459.81|-1702.44| +2452375|40728|2452474|17185|72525|810179|6680|5934|57988|1517660|6190|39891|12|24|1|5|219|151|46|54.82|130.47|121.33|420.44|5581.18|2521.72|6001.62|446.49|0.00|0.00|5581.18|6027.67|5581.18|6027.67|3059.46| +2452375|40728|2452495|9439|72525|810179|6680|5934|57988|1517660|6190|39891|54|3|7|4|204|151|55|8.08|15.99|13.59|132.00|747.45|444.40|879.45|2.69|717.55|360.25|29.90|32.59|390.15|392.84|-414.50| +2452375|40728|2452425|5611|72525|810179|6680|5934|57988|1517660|6190|39891|19|18|18|2|119|151|47|45.18|108.88|97.99|511.83|4605.53|2123.46|5117.36|322.38|0.00|665.05|4605.53|4927.91|5270.58|5592.96|2482.07| +2452375|40728|2452429|7650|72525|810179|6680|5934|57988|1517660|6190|39891|48|18|14|4|264|151|99|31.19|69.24|24.92|4387.68|2467.08|3087.81|6854.76|148.02|0.00|2192.85|2467.08|2615.10|4659.93|4807.95|-620.73| +2452375|40728|2452495|4813|72525|810179|6680|5934|57988|1517660|6190|39891|30|27|6|5|27|151|71|40.69|79.34|43.63|2535.41|3097.73|2888.99|5633.14|30.97|0.00|1858.78|3097.73|3128.70|4956.51|4987.48|208.74| +2452208|62293|2452220|3535|38675|1029091|7060|29029|47359|1106649|6136|21305|39|13|7|2|194|152|11|86.87|88.60|79.74|97.46|877.14|955.57|974.60|35.08|0.00|380.05|877.14|912.22|1257.19|1292.27|-78.43| +2452208|62293|2452268|15229|38675|1029091|7060|29029|47359|1106649|6136|21305|21|25|12|4|86|152|36|68.13|198.25|19.82|6423.48|713.52|2452.68|7137.00|57.08|0.00|2569.32|713.52|770.60|3282.84|3339.92|-1739.16| +2452208|62293|2452268|11183|38675|1029091|7060|29029|47359|1106649|6136|21305|9|5|14|2|20|152|9|13.19|24.79|19.08|51.39|171.72|118.71|223.11|3.70|109.90|62.46|61.82|65.52|124.28|127.98|-56.89| +2452208|62293|2452220|12851|38675|1029091|7060|29029|47359|1106649|6136|21305|59|9|7|2|297|152|3|75.52|154.81|24.76|390.15|74.28|226.56|464.43|0.74|0.00|208.98|74.28|75.02|283.26|284.00|-152.28| +2452208|62293|2452328|12657|38675|1029091|7060|29029|47359|1106649|6136|21305|29|21|10|1|225|152|8|13.11|37.62|1.88|285.92|15.04|104.88|300.96|0.90|0.00|51.12|15.04|15.94|66.16|67.06|-89.84| +2452208|62293|2452277|14315|38675|1029091|7060|29029|47359|1106649|6136|21305|45|23|14|1|228|152|52|62.42|121.71|26.77|4936.88|1392.04|3245.84|6328.92|4.45|1169.31|2278.12|222.73|227.18|2500.85|2505.30|-3023.11| +2452208|62293|2452302|12603|38675|1029091|7060|29029|47359|1106649|6136|21305|17|17|16|1|203|152|63|89.64|105.77|44.42|3865.05|2798.46|5647.32|6663.51|97.94|839.53|2131.92|1958.93|2056.87|4090.85|4188.79|-3688.39| +2452208|62293|2452304|14355|38675|1029091|7060|29029|47359|1106649|6136|21305|47|9|16|4|45|152|37|80.84|152.78|116.11|1356.79|4296.07|2991.08|5652.86|343.68|0.00|2543.75|4296.07|4639.75|6839.82|7183.50|1304.99| +2452208|62293|2452253|11285|38675|1029091|7060|29029|47359|1106649|6136|21305|53|23|14|1|11|152|54|48.95|97.41|89.61|421.20|4838.94|2643.30|5260.14|0.00|0.00|104.76|4838.94|4838.94|4943.70|4943.70|2195.64| +2451412|72051|2451496|5240|68664|7349|4434|9166|80155|1521653|4421|1953|2|22|12|4|79|153|26|60.89|88.89|15.11|1918.28|392.86|1583.14|2311.14|0.00|0.00|69.16|392.86|392.86|462.02|462.02|-1190.28| +2451412|72051|2451478|7052|68664|7349|4434|9166|80155|1521653|4421|1953|10|10|6|1|96|153|7|72.05|173.64|90.29|583.45|632.03|504.35|1215.48|50.56|0.00|449.68|632.03|682.59|1081.71|1132.27|127.68| +2451412|72051|2451456|5173|68664|7349|4434|9166|80155|1521653|4421|1953|44|26|15|4|100|153|72|41.09|42.32|22.42|1432.80|1614.24|2958.48|3047.04|112.99|0.00|578.88|1614.24|1727.23|2193.12|2306.11|-1344.24| +2451412|72051|2451448|12070|68664|7349|4434|9166|80155|1521653|4421|1953|14|20|17|5|73|153|44|76.98|88.52|40.71|2103.64|1791.24|3387.12|3894.88|0.00|0.00|778.80|1791.24|1791.24|2570.04|2570.04|-1595.88| +2451412|72051|2451414|3019|68664|7349|4434|9166|80155|1521653|4421|1953|22|10|7|3|99|153|16|79.99|99.98|76.98|368.00|1231.68|1279.84|1599.68|36.95|0.00|735.84|1231.68|1268.63|1967.52|2004.47|-48.16| +2451412|72051|2451438|6460|68664|7349|4434|9166|80155|1521653|4421|1953|7|26|14|1|231|153|97|68.04|142.88|11.43|12750.65|1108.71|6599.88|13859.36|55.43|0.00|5404.84|1108.71|1164.14|6513.55|6568.98|-5491.17| +2451412|72051|2451465|14239|68664|7349|4434|9166|80155|1521653|4421|1953|56|25|7|1|115|153|24|62.85|74.16|51.91|534.00|1245.84|1508.40|1779.84|0.00|0.00|160.08|1245.84|1245.84|1405.92|1405.92|-262.56| +2451412|72051|2451460|5491|68664|7349|4434|9166|80155|1521653|4421|1953|13|14|12|3|64|153|20|42.90|101.67|100.65|20.40|2013.00|858.00|2033.40|0.00|865.59|162.60|1147.41|1147.41|1310.01|1310.01|289.41| +2451412|72051|2451530|12964|68664|7349|4434|9166|80155|1521653|4421|1953|26|8|16|2|38|153|84|20.67|56.22|5.05|4298.28|424.20|1736.28|4722.48|25.45|0.00|849.24|424.20|449.65|1273.44|1298.89|-1312.08| +2452610|34471|2452712|16497|50874|30210|746|29682|50874|30210|746|29682|48|18|11|1|128|154|81|53.20|84.58|34.67|4042.71|2808.27|4309.20|6850.98|140.41|0.00|1575.45|2808.27|2948.68|4383.72|4524.13|-1500.93| +2452610|34471|2452686|7302|50874|30210|746|29682|50874|30210|746|29682|12|13|5|3|211|154|61|98.25|183.72|181.88|112.24|11094.68|5993.25|11206.92|110.94|0.00|4258.41|11094.68|11205.62|15353.09|15464.03|5101.43| +2452610|34471|2452651|7|50874|30210|746|29682|50874|30210|746|29682|9|12|14|5|256|154|39|88.90|93.34|8.40|3312.66|327.60|3467.10|3640.26|22.93|0.00|655.20|327.60|350.53|982.80|1005.73|-3139.50| +2452610|34471|2452653|2581|50874|30210|746|29682|50874|30210|746|29682|12|12|16|1|126|154|62|83.11|162.89|48.86|7069.86|3029.32|5152.82|10099.18|30.29|0.00|1110.42|3029.32|3059.61|4139.74|4170.03|-2123.50| +2452610|34471|2452671|12897|50874|30210|746|29682|50874|30210|746|29682|37|12|13|2|64|154|26|18.81|49.65|5.95|1136.20|154.70|489.06|1290.90|1.29|89.72|141.96|64.98|66.27|206.94|208.23|-424.08| +2452610|34471|2452679|15558|50874|30210|746|29682|50874|30210|746|29682|6|25|10|5|110|154|56|46.81|74.89|13.48|3438.96|754.88|2621.36|4193.84|22.64|0.00|2096.64|754.88|777.52|2851.52|2874.16|-1866.48| +2452610|34471|2452722|17946|50874|30210|746|29682|50874|30210|746|29682|45|6|13|5|70|154|20|80.61|90.28|49.65|812.60|993.00|1612.20|1805.60|39.72|0.00|415.20|993.00|1032.72|1408.20|1447.92|-619.20| +2452610|34471|2452712|17922|50874|30210|746|29682|50874|30210|746|29682|12|25|20|2|214|154|87|5.18|12.12|6.18|516.78|537.66|450.66|1054.44|43.01|0.00|400.20|537.66|580.67|937.86|980.87|87.00| +2451796|65903|2451824|8735|60125|95037|1149|35661|83535|140531|2844|3356|23|17|13|5|67|155|70|62.89|116.34|22.10|6596.80|1547.00|4402.30|8143.80|46.41|0.00|814.10|1547.00|1593.41|2361.10|2407.51|-2855.30| +2451796|65903|2451806|12599|60125|95037|1149|35661|83535|140531|2844|3356|23|20|7|3|52|155|54|70.64|113.02|56.51|3051.54|3051.54|3814.56|6103.08|61.03|0.00|2257.74|3051.54|3112.57|5309.28|5370.31|-763.02| +2451796|65903|2451841|5756|60125|95037|1149|35661|83535|140531|2844|3356|59|26|11|5|282|155|83|23.34|32.20|30.91|107.07|2565.53|1937.22|2672.60|55.41|1641.93|1015.09|923.60|979.01|1938.69|1994.10|-1013.62| +2451796|65903|2451843|2870|60125|95037|1149|35661|83535|140531|2844|3356|35|7|20|3|119|155|61|76.26|189.88|94.94|5791.34|5791.34|4651.86|11582.68|57.91|0.00|3126.86|5791.34|5849.25|8918.20|8976.11|1139.48| +2451796|65903|2451872|6965|60125|95037|1149|35661|83535|140531|2844|3356|53|20|5|1|30|155|41|99.28|163.81|153.98|403.03|6313.18|4070.48|6716.21|63.13|0.00|2887.63|6313.18|6376.31|9200.81|9263.94|2242.70| +2451796|65903|2451797|7031|60125|95037|1149|35661|83535|140531|2844|3356|19|1|12|1|117|155|46|78.02|193.48|106.41|4005.22|4894.86|3588.92|8900.08|160.55|2887.96|2402.58|2006.90|2167.45|4409.48|4570.03|-1582.02| +2451796|65903|2451881|16877|60125|95037|1149|35661|83535|140531|2844|3356|47|8|16|5|91|155|54|57.15|80.01|28.80|2765.34|1555.20|3086.10|4320.54|62.20|0.00|1598.40|1555.20|1617.40|3153.60|3215.80|-1530.90| +2451796|65903|2451867|17407|60125|95037|1149|35661|83535|140531|2844|3356|59|1|15|1|147|155|73|78.43|207.83|89.36|8648.31|6523.28|5725.39|15171.59|130.46|0.00|2427.25|6523.28|6653.74|8950.53|9080.99|797.89| +2451796|65903|2451879|12955|60125|95037|1149|35661|83535|140531|2844|3356|23|19|18|4|87|155|86|41.78|113.64|7.95|9089.34|683.70|3593.08|9773.04|0.00|560.63|585.66|123.07|123.07|708.73|708.73|-3470.01| +2451796|65903|2451862|8335|60125|95037|1149|35661|83535|140531|2844|3356|2|23|12|2|102|155|42|15.59|28.84|26.53|97.02|1114.26|654.78|1211.28|22.28|0.00|11.76|1114.26|1136.54|1126.02|1148.30|459.48| +2451796|65903|2451816|902|60125|95037|1149|35661|83535|140531|2844|3356|32|25|19|5|91|155|21|96.90|200.58|26.07|3664.71|547.47|2034.90|4212.18|49.27|0.00|1600.62|547.47|596.74|2148.09|2197.36|-1487.43| +2451796|65903|2451859|173|60125|95037|1149|35661|83535|140531|2844|3356|59|26|5|4|243|155|59|19.14|44.78|8.95|2113.97|528.05|1129.26|2642.02|21.12|0.00|1082.65|528.05|549.17|1610.70|1631.82|-601.21| +2451796|65903|2451841|8009|60125|95037|1149|35661|83535|140531|2844|3356|49|26|13|2|191|155|86|83.85|123.25|113.39|847.96|9751.54|7211.10|10599.50|682.60|0.00|2331.46|9751.54|10434.14|12083.00|12765.60|2540.44| +2451796|65903|2451842|7238|60125|95037|1149|35661|83535|140531|2844|3356|53|8|10|3|96|155|99|4.86|7.24|0.50|667.26|49.50|481.14|716.76|0.99|0.00|113.85|49.50|50.49|163.35|164.34|-431.64| +2451796|65903|2451913|1610|60125|95037|1149|35661|83535|140531|2844|3356|55|11|17|3|226|155|64|63.71|186.03|33.48|9763.20|2142.72|4077.44|11905.92|85.70|0.00|1547.52|2142.72|2228.42|3690.24|3775.94|-1934.72| +2451796|65903|2451859|13415|60125|95037|1149|35661|83535|140531|2844|3356|44|25|3|3|223|155|51|81.46|188.17|3.76|9404.91|191.76|4154.46|9596.67|0.00|0.00|575.79|191.76|191.76|767.55|767.55|-3962.70| +2451175|59206|2451210|15955|72140|959897|7|564|38687|399450|1341|32698|1|14|18|5|112|156|100|53.50|110.74|35.43|7531.00|3543.00|5350.00|11074.00|283.44|0.00|2436.00|3543.00|3826.44|5979.00|6262.44|-1807.00| +2451175|59206|2451279|11270|72140|959897|7|564|38687|399450|1341|32698|56|4|14|3|89|156|22|13.66|20.76|1.45|424.82|31.90|300.52|456.72|0.00|0.00|73.04|31.90|31.90|104.94|104.94|-268.62| +2451175|59206|2451266|9817|72140|959897|7|564|38687|399450|1341|32698|7|26|4|4|284|156|40|49.11|52.05|15.09|1478.40|603.60|1964.40|2082.00|0.00|0.00|1020.00|603.60|603.60|1623.60|1623.60|-1360.80| +2451175|59206|2451193|718|72140|959897|7|564|38687|399450|1341|32698|49|2|12|2|212|156|20|62.90|140.89|42.26|1972.60|845.20|1258.00|2817.80|0.00|845.20|253.60|0.00|0.00|253.60|253.60|-1258.00| +2451175|59206|2451187|15824|72140|959897|7|564|38687|399450|1341|32698|40|14|12|1|253|156|25|91.61|131.00|37.99|2325.25|949.75|2290.25|3275.00|28.49|0.00|1572.00|949.75|978.24|2521.75|2550.24|-1340.50| +2451175|59206|2451176|15080|72140|959897|7|564|38687|399450|1341|32698|7|26|7|2|217|156|20|10.53|17.26|6.38|217.60|127.60|210.60|345.20|10.20|0.00|103.40|127.60|137.80|231.00|241.20|-83.00| +2451175|59206|2451257|7676|72140|959897|7|564|38687|399450|1341|32698|4|20|19|2|192|156|53|80.61|200.71|28.09|9148.86|1488.77|4272.33|10637.63|0.00|0.00|2553.01|1488.77|1488.77|4041.78|4041.78|-2783.56| +2451175|59206|2451290|1093|72140|959897|7|564|38687|399450|1341|32698|1|26|14|4|104|156|40|47.51|128.27|41.04|3489.20|1641.60|1900.40|5130.80|147.74|0.00|0.00|1641.60|1789.34|1641.60|1789.34|-258.80| +2451175|59206|2451240|15853|72140|959897|7|564|38687|399450|1341|32698|38|19|3|3|176|156|4|12.35|36.55|27.04|38.04|108.16|49.40|146.20|1.08|0.00|21.92|108.16|109.24|130.08|131.16|58.76| +2451175|59206|2451268|7442|72140|959897|7|564|38687|399450|1341|32698|26|1|5|2|149|156|77|96.80|118.09|0.00|9092.93|0.00|7453.60|9092.93|0.00|0.00|272.58|0.00|0.00|272.58|272.58|-7453.60| +2451175|59206|2451248|11563|72140|959897|7|564|38687|399450|1341|32698|56|13|12|2|112|156|64|12.92|38.63|24.33|915.20|1557.12|826.88|2472.32|0.00|0.00|865.28|1557.12|1557.12|2422.40|2422.40|730.24| +2452629|20222|2452703|16539|16928|335672|1616|46360|64837|833823|592|18936|15|7|3|2|121|157|19|20.60|21.83|1.74|381.71|33.06|391.40|414.77|2.82|1.65|16.53|31.41|34.23|47.94|50.76|-359.99| +2452629|20222|2452664|17223|16928|335672|1616|46360|64837|833823|592|18936|42|9|6|2|38|157|34|3.31|5.95|2.79|107.44|94.86|112.54|202.30|2.84|0.00|68.68|94.86|97.70|163.54|166.38|-17.68| +2452629|20222|2452657|7705|16928|335672|1616|46360|64837|833823|592|18936|48|19|19|5|21|157|81|20.93|25.74|21.87|313.47|1771.47|1695.33|2084.94|88.57|0.00|396.09|1771.47|1860.04|2167.56|2256.13|76.14| +2452629|20222|2452687|9433|16928|335672|1616|46360|64837|833823|592|18936|31|19|17|1|248|157|44|39.02|70.23|35.11|1545.28|1544.84|1716.88|3090.12|92.69|0.00|1544.84|1544.84|1637.53|3089.68|3182.37|-172.04| +2452629|20222|2452701|16872|16928|335672|1616|46360|64837|833823|592|18936|45|19|17|4|31|157|56|46.83|55.72|16.15|2215.92|904.40|2622.48|3120.32|36.17|0.00|655.20|904.40|940.57|1559.60|1595.77|-1718.08| +2452629|20222|2452749|16431|16928|335672|1616|46360|64837|833823|592|18936|37|7|16|3|273|157|78|58.55|165.11|3.30|12621.18|257.40|4566.90|12878.58|4.94|175.03|772.20|82.37|87.31|854.57|859.51|-4484.53| +2452629|20222|2452743|14509|16928|335672|1616|46360|64837|833823|592|18936|12|7|15|3|134|157|31|84.21|119.57|44.24|2335.23|1371.44|2610.51|3706.67|109.71|0.00|148.18|1371.44|1481.15|1519.62|1629.33|-1239.07| +2452629|20222|2452655|8103|16928|335672|1616|46360|64837|833823|592|18936|36|19|19|3|150|157|23|36.69|75.94|20.50|1275.12|471.50|843.87|1746.62|23.57|0.00|87.17|471.50|495.07|558.67|582.24|-372.37| +2452629|20222|2452728|16869|16928|335672|1616|46360|64837|833823|592|18936|18|18|19|4|9|157|7|54.69|92.42|10.16|575.82|71.12|382.83|646.94|0.71|0.00|45.22|71.12|71.83|116.34|117.05|-311.71| +2452629|20222|2452688|1161|16928|335672|1616|46360|64837|833823|592|18936|12|7|5|1|97|157|98|12.17|18.37|4.77|1332.80|467.46|1192.66|1800.26|9.34|0.00|575.26|467.46|476.80|1042.72|1052.06|-725.20| +2452629|20222|2452647|6264|16928|335672|1616|46360|64837|833823|592|18936|55|27|10|1|92|157|91|40.61|82.43|14.01|6226.22|1274.91|3695.51|7501.13|21.03|854.18|749.84|420.73|441.76|1170.57|1191.60|-3274.78| +2452629|20222|2452735|17499|16928|335672|1616|46360|64837|833823|592|18936|33|12|2|1|280|157|42|53.16|136.08|91.17|1886.22|3829.14|2232.72|5715.36|229.74|0.00|2114.28|3829.14|4058.88|5943.42|6173.16|1596.42| +2452629|20222|2452663|5041|16928|335672|1616|46360|64837|833823|592|18936|60|25|16|2|195|157|64|26.19|33.26|32.59|42.88|2085.76|1676.16|2128.64|187.71|0.00|170.24|2085.76|2273.47|2256.00|2443.71|409.60| +2451522|64963|2451562|10736|60623|1003215|3407|39199|76905|174988|1422|27614|44|19|17|4|54|158|38|47.63|71.92|19.41|1995.38|737.58|1809.94|2732.96|22.12|0.00|163.78|737.58|759.70|901.36|923.48|-1072.36| +2451522|64963|2451554|9424|60623|1003215|3407|39199|76905|174988|1422|27614|49|7|13|2|31|158|35|7.76|10.47|10.26|7.35|359.10|271.60|366.45|21.54|0.00|76.65|359.10|380.64|435.75|457.29|87.50| +2451522|64963|2451525|5992|60623|1003215|3407|39199|76905|174988|1422|27614|50|14|20|2|254|158|33|88.82|256.68|89.83|5506.05|2964.39|2931.06|8470.44|148.21|0.00|2879.91|2964.39|3112.60|5844.30|5992.51|33.33| +2451522|64963|2451636|2384|60623|1003215|3407|39199|76905|174988|1422|27614|44|19|17|2|264|158|51|87.46|118.94|49.95|3518.49|2547.45|4460.46|6065.94|203.79|0.00|667.08|2547.45|2751.24|3214.53|3418.32|-1913.01| +2451522|64963|2451594|7891|60623|1003215|3407|39199|76905|174988|1422|27614|32|2|14|4|61|158|86|46.85|60.43|9.66|4366.22|830.76|4029.10|5196.98|41.53|0.00|1506.72|830.76|872.29|2337.48|2379.01|-3198.34| +2451522|64963|2451579|10786|60623|1003215|3407|39199|76905|174988|1422|27614|50|28|1|2|146|158|97|21.14|55.59|4.44|4961.55|430.68|2050.58|5392.23|6.20|120.59|1616.99|310.09|316.29|1927.08|1933.28|-1740.49| +2451522|64963|2451581|17659|60623|1003215|3407|39199|76905|174988|1422|27614|19|22|11|3|55|158|45|46.79|109.02|2.18|4807.80|98.10|2105.55|4905.90|3.92|0.00|245.25|98.10|102.02|343.35|347.27|-2007.45| +2451522|64963|2451529|9688|60623|1003215|3407|39199|76905|174988|1422|27614|28|26|6|5|57|158|51|71.80|145.03|1.45|7322.58|73.95|3661.80|7396.53|2.21|0.00|2736.66|73.95|76.16|2810.61|2812.82|-3587.85| +2451522|64963|2451527|13120|60623|1003215|3407|39199|76905|174988|1422|27614|50|26|9|2|285|158|65|33.49|98.12|8.83|5803.85|573.95|2176.85|6377.80|22.95|0.00|2614.30|573.95|596.90|3188.25|3211.20|-1602.90| +2451522|64963|2451567|8050|60623|1003215|3407|39199|76905|174988|1422|27614|40|20|4|4|4|158|53|39.09|44.17|9.27|1849.70|491.31|2071.77|2341.01|4.12|78.60|1029.79|412.71|416.83|1442.50|1446.62|-1659.06| +2451522|64963|2451637|991|60623|1003215|3407|39199|76905|174988|1422|27614|26|26|11|3|178|158|86|30.65|85.82|21.45|5535.82|1844.70|2635.90|7380.52|129.12|0.00|1623.68|1844.70|1973.82|3468.38|3597.50|-791.20| +2452559|49265|2452617|6342|28340|316285|4677|22222|52173|310686|3751|36496|9|15|1|4|137|159|40|78.13|217.98|67.57|6016.40|2702.80|3125.20|8719.20|162.16|0.00|3487.60|2702.80|2864.96|6190.40|6352.56|-422.40| +2452559|49265|2452583|15741|28340|316285|4677|22222|52173|310686|3751|36496|21|30|19|3|244|159|74|25.05|71.64|62.32|689.68|4611.68|1853.70|5301.36|0.00|0.00|211.64|4611.68|4611.68|4823.32|4823.32|2757.98| +2452559|49265|2452642|11773|28340|316285|4677|22222|52173|310686|3751|36496|7|12|15|2|169|159|24|47.50|139.17|48.70|2171.28|1168.80|1140.00|3340.08|81.81|0.00|1669.92|1168.80|1250.61|2838.72|2920.53|28.80| +2452559|49265|2452608|5167|28340|316285|4677|22222|52173|310686|3751|36496|24|18|6|3|263|159|28|80.05|176.11|176.11|0.00|4931.08|2241.40|4931.08|98.62|0.00|394.24|4931.08|5029.70|5325.32|5423.94|2689.68| +2452559|49265|2452608|11259|28340|316285|4677|22222|52173|310686|3751|36496|51|21|16|1|116|159|80|1.46|2.51|1.83|54.40|146.40|116.80|200.80|0.57|127.36|9.60|19.04|19.61|28.64|29.21|-97.76| +2452559|49265|2452620|9553|28340|316285|4677|22222|52173|310686|3751|36496|19|1|12|3|274|159|86|69.18|145.96|32.11|9791.10|2761.46|5949.48|12552.56|248.53|0.00|5648.48|2761.46|3009.99|8409.94|8658.47|-3188.02| +2452559|49265|2452602|1938|28340|316285|4677|22222|52173|310686|3751|36496|49|1|4|2|3|159|8|58.91|141.38|4.24|1097.12|33.92|471.28|1131.04|0.00|8.48|282.72|25.44|25.44|308.16|308.16|-445.84| +2452559|49265|2452561|8217|28340|316285|4677|22222|52173|310686|3751|36496|39|9|13|1|66|159|28|75.24|122.64|95.65|755.72|2678.20|2106.72|3433.92|107.12|0.00|1476.44|2678.20|2785.32|4154.64|4261.76|571.48| +2452559|49265|2452675|2547|28340|316285|4677|22222|52173|310686|3751|36496|48|7|7|2|260|159|97|57.82|139.92|36.37|10044.35|3527.89|5608.54|13572.24|0.00|0.00|4614.29|3527.89|3527.89|8142.18|8142.18|-2080.65| +2452559|49265|2452626|12486|28340|316285|4677|22222|52173|310686|3751|36496|48|21|1|1|71|159|53|51.91|138.59|2.77|7198.46|146.81|2751.23|7345.27|2.93|0.00|3672.37|146.81|149.74|3819.18|3822.11|-2604.42| +2452559|49265|2452654|7500|28340|316285|4677|22222|52173|310686|3751|36496|51|9|4|3|219|159|52|62.80|142.55|89.80|2743.00|4669.60|3265.60|7412.60|271.30|793.83|518.44|3875.77|4147.07|4394.21|4665.51|610.17| +2452559|49265|2452601|17169|28340|316285|4677|22222|52173|310686|3751|36496|7|18|7|2|250|159|45|15.75|19.53|18.94|26.55|852.30|708.75|878.85|51.13|0.00|228.15|852.30|903.43|1080.45|1131.58|143.55| +2452559|49265|2452633|7593|28340|316285|4677|22222|52173|310686|3751|36496|57|1|16|3|53|159|62|79.11|109.17|6.55|6362.44|406.10|4904.82|6768.54|4.06|0.00|676.42|406.10|410.16|1082.52|1086.58|-4498.72| +2452559|49265|2452647|307|28340|316285|4677|22222|52173|310686|3751|36496|31|30|4|5|19|159|71|18.86|41.30|15.28|1847.42|1084.88|1339.06|2932.30|21.69|0.00|673.79|1084.88|1106.57|1758.67|1780.36|-254.18| +2451496|74442|2451579|10408|39288|1913042|3949|40903|39288|1913042|3949|40903|32|10|5|4|63|160|87|62.62|162.81|89.54|6374.49|7789.98|5447.94|14164.47|623.19|0.00|5099.07|7789.98|8413.17|12889.05|13512.24|2342.04| +2451496|74442|2451536|14569|39288|1913042|3949|40903|39288|1913042|3949|40903|10|19|14|3|255|160|9|96.64|225.17|139.60|770.13|1256.40|869.76|2026.53|42.08|414.61|101.25|841.79|883.87|943.04|985.12|-27.97| +2451496|74442|2451521|1124|39288|1913042|3949|40903|39288|1913042|3949|40903|40|1|11|1|271|160|48|69.30|198.89|95.46|4964.64|4582.08|3326.40|9546.72|229.10|0.00|3627.36|4582.08|4811.18|8209.44|8438.54|1255.68| +2451496|74442|2451577|10562|39288|1913042|3949|40903|39288|1913042|3949|40903|49|16|3|2|268|160|77|35.63|67.69|29.10|2971.43|2240.70|2743.51|5212.13|156.84|0.00|2397.01|2240.70|2397.54|4637.71|4794.55|-502.81| +2451496|74442|2451521|15745|39288|1913042|3949|40903|39288|1913042|3949|40903|34|28|2|4|216|160|59|31.01|38.76|2.32|2149.96|136.88|1829.59|2286.84|4.10|0.00|457.25|136.88|140.98|594.13|598.23|-1692.71| +2451496|74442|2451503|15139|39288|1913042|3949|40903|39288|1913042|3949|40903|14|28|19|1|133|160|48|82.26|200.71|108.38|4431.84|5202.24|3948.48|9634.08|104.04|0.00|2601.12|5202.24|5306.28|7803.36|7907.40|1253.76| +2451496|74442|2451550|15625|39288|1913042|3949|40903|39288|1913042|3949|40903|56|10|16|1|283|160|83|78.04|113.15|53.18|4977.51|4413.94|6477.32|9391.45|308.97|0.00|4319.32|4413.94|4722.91|8733.26|9042.23|-2063.38| +2451496|74442|2451568|15530|39288|1913042|3949|40903|39288|1913042|3949|40903|8|26|13|2|71|160|65|28.20|72.75|32.73|2601.30|2127.45|1833.00|4728.75|42.54|0.00|378.30|2127.45|2169.99|2505.75|2548.29|294.45| +2451496|74442|2451516|9802|39288|1913042|3949|40903|39288|1913042|3949|40903|52|7|14|4|4|160|69|19.76|30.03|6.30|1637.37|434.70|1363.44|2072.07|4.34|0.00|289.80|434.70|439.04|724.50|728.84|-928.74| +2451496|74442|2451554|4810|39288|1913042|3949|40903|39288|1913042|3949|40903|32|28|6|4|2|160|56|36.13|83.82|38.55|2535.12|2158.80|2023.28|4693.92|34.54|431.76|1407.84|1727.04|1761.58|3134.88|3169.42|-296.24| +2451496|74442|2451591|8593|39288|1913042|3949|40903|39288|1913042|3949|40903|50|14|20|2|201|160|20|44.35|81.60|15.50|1322.00|310.00|887.00|1632.00|18.60|0.00|65.20|310.00|328.60|375.20|393.80|-577.00| +2451496|74442|2451530|7382|39288|1913042|3949|40903|39288|1913042|3949|40903|16|16|16|1|23|160|99|75.98|106.37|49.99|5581.62|4949.01|7522.02|10530.63|296.94|0.00|3053.16|4949.01|5245.95|8002.17|8299.11|-2573.01| +2451496|74442|2451614|1532|39288|1913042|3949|40903|39288|1913042|3949|40903|58|10|15|1|160|160|5|77.72|90.93|13.63|386.50|68.15|388.60|454.65|0.00|0.00|9.05|68.15|68.15|77.20|77.20|-320.45| +2451962|19052|2452058|9943|96248|810654|6660|31913|69550|1565406|731|1584|21|9|6|5|31|161|49|14.78|27.63|6.07|1056.44|297.43|724.22|1353.87|26.76|0.00|53.90|297.43|324.19|351.33|378.09|-426.79| +2451962|19052|2452079|5403|96248|810654|6660|31913|69550|1565406|731|1584|53|29|13|1|129|161|9|98.99|103.93|28.06|682.83|252.54|890.91|935.37|12.95|108.59|355.41|143.95|156.90|499.36|512.31|-746.96| +2451962|19052|2451979|11709|96248|810654|6660|31913|69550|1565406|731|1584|3|1|3|2|44|161|83|97.22|111.80|69.31|3526.67|5752.73|8069.26|9279.40|345.16|0.00|3525.84|5752.73|6097.89|9278.57|9623.73|-2316.53| +2451962|19052|2452035|2031|96248|810654|6660|31913|69550|1565406|731|1584|57|3|15|1|64|161|89|67.69|201.71|108.92|8258.31|9693.88|6024.41|17952.19|581.63|0.00|5923.84|9693.88|10275.51|15617.72|16199.35|3669.47| +2451962|19052|2452011|3459|96248|810654|6660|31913|69550|1565406|731|1584|5|29|14|1|88|161|41|47.30|102.16|66.40|1466.16|2722.40|1939.30|4188.56|217.79|0.00|753.58|2722.40|2940.19|3475.98|3693.77|783.10| +2451962|19052|2452011|14247|96248|810654|6660|31913|69550|1565406|731|1584|35|15|1|1|204|161|67|34.00|100.30|80.24|1344.02|5376.08|2278.00|6720.10|376.32|0.00|402.67|5376.08|5752.40|5778.75|6155.07|3098.08| +2451962|19052|2451995|12069|96248|810654|6660|31913|69550|1565406|731|1584|7|9|13|4|15|161|28|51.13|114.53|25.19|2501.52|705.32|1431.64|3206.84|16.64|289.18|609.28|416.14|432.78|1025.42|1042.06|-1015.50| +2451962|19052|2452030|10615|96248|810654|6660|31913|69550|1565406|731|1584|9|13|19|3|140|161|36|92.80|116.00|22.04|3382.56|793.44|3340.80|4176.00|0.00|0.00|1837.44|793.44|793.44|2630.88|2630.88|-2547.36| +2451962|19052|2452039|12313|96248|810654|6660|31913|69550|1565406|731|1584|17|1|10|4|81|161|74|24.43|66.69|43.34|1727.90|3207.16|1807.82|4935.06|224.50|0.00|1924.00|3207.16|3431.66|5131.16|5355.66|1399.34| +2451962|19052|2452011|17675|96248|810654|6660|31913|69550|1565406|731|1584|9|27|3|5|244|161|36|35.16|80.16|73.74|231.12|2654.64|1265.76|2885.76|79.63|0.00|1154.16|2654.64|2734.27|3808.80|3888.43|1388.88| +2451962|19052|2452061|1173|96248|810654|6660|31913|69550|1565406|731|1584|23|7|9|1|63|161|65|11.84|32.20|9.33|1486.55|606.45|769.60|2093.00|18.19|0.00|334.75|606.45|624.64|941.20|959.39|-163.15| +2451962|19052|2451964|4775|96248|810654|6660|31913|69550|1565406|731|1584|9|29|15|5|105|161|36|51.29|98.47|41.35|2056.32|1488.60|1846.44|3544.92|59.54|0.00|921.60|1488.60|1548.14|2410.20|2469.74|-357.84| +2451962|19052|2452021|5503|96248|810654|6660|31913|69550|1565406|731|1584|49|13|6|5|184|161|26|28.25|41.52|28.23|345.54|733.98|734.50|1079.52|14.67|0.00|183.30|733.98|748.65|917.28|931.95|-0.52| +2451962|19052|2451980|7043|96248|810654|6660|31913|69550|1565406|731|1584|9|11|17|3|189|161|65|20.21|28.69|20.08|559.65|1305.20|1313.65|1864.85|0.00|0.00|745.55|1305.20|1305.20|2050.75|2050.75|-8.45| +2451962|19052|2451988|15833|96248|810654|6660|31913|69550|1565406|731|1584|51|9|14|1|235|161|77|68.67|153.13|22.96|10023.09|1767.92|5287.59|11791.01|141.43|0.00|2475.55|1767.92|1909.35|4243.47|4384.90|-3519.67| +2451806|38537|2451844|11264|82399|1584419|1992|4812|82399|1584419|1992|4812|47|8|13|1|239|162|80|93.06|222.41|106.75|9252.80|8540.00|7444.80|17792.80|422.73|85.40|3024.00|8454.60|8877.33|11478.60|11901.33|1009.80| +2451806|38537|2451877|4385|82399|1584419|1992|4812|82399|1584419|1992|4812|20|26|7|5|23|162|39|31.03|92.77|38.03|2134.86|1483.17|1210.17|3618.03|74.15|0.00|1230.06|1483.17|1557.32|2713.23|2787.38|273.00| +2451806|38537|2451830|16183|82399|1584419|1992|4812|82399|1584419|1992|4812|13|17|1|3|248|162|31|42.64|44.34|7.09|1154.75|219.79|1321.84|1374.54|5.53|81.32|137.33|138.47|144.00|275.80|281.33|-1183.37| +2451806|38537|2451906|9536|82399|1584419|1992|4812|82399|1584419|1992|4812|50|2|18|2|223|162|15|50.73|119.21|50.06|1037.25|750.90|760.95|1788.15|0.00|0.00|393.30|750.90|750.90|1144.20|1144.20|-10.05| +2451806|38537|2451884|8780|82399|1584419|1992|4812|82399|1584419|1992|4812|35|23|6|5|54|162|38|42.37|63.55|31.77|1207.64|1207.26|1610.06|2414.90|72.43|0.00|651.70|1207.26|1279.69|1858.96|1931.39|-402.80| +2451806|38537|2451911|14041|82399|1584419|1992|4812|82399|1584419|1992|4812|20|23|16|2|248|162|29|37.77|97.82|71.40|766.18|2070.60|1095.33|2836.78|125.89|496.94|765.89|1573.66|1699.55|2339.55|2465.44|478.33| +2451806|38537|2451858|8531|82399|1584419|1992|4812|82399|1584419|1992|4812|59|7|8|4|113|162|23|42.99|86.40|58.75|635.95|1351.25|988.77|1987.20|121.61|0.00|695.52|1351.25|1472.86|2046.77|2168.38|362.48| +2451806|38537|2451814|11669|82399|1584419|1992|4812|82399|1584419|1992|4812|37|11|13|2|124|162|6|73.06|78.90|75.74|18.96|454.44|438.36|473.40|31.81|0.00|99.36|454.44|486.25|553.80|585.61|16.08| +2451097|71478|2451102|16564|95885|1105012|6139|40866|99773|1811145|5400|527|44|26|8|2|213|163|81|72.46|81.87|67.95|1127.52|5503.95|5869.26|6631.47|55.03|0.00|1060.29|5503.95|5558.98|6564.24|6619.27|-365.31| +2451097|71478|2451150|9454|95885|1105012|6139|40866|99773|1811145|5400|527|1|2|13|1|102|163|83|81.59|235.79|153.26|6849.99|12720.58|6771.97|19570.57|10.17|11702.93|977.74|1017.65|1027.82|1995.39|2005.56|-5754.32| +2451097|71478|2451164|6313|95885|1105012|6139|40866|99773|1811145|5400|527|58|22|4|1|18|163|87|39.68|69.44|44.44|2175.00|3866.28|3452.16|6041.28|0.00|0.00|2536.92|3866.28|3866.28|6403.20|6403.20|414.12| +2451097|71478|2451192|7994|95885|1105012|6139|40866|99773|1811145|5400|527|13|25|12|3|204|163|70|46.21|109.05|42.52|4657.10|2976.40|3234.70|7633.50|208.34|0.00|2137.10|2976.40|3184.74|5113.50|5321.84|-258.30| +2451097|71478|2451152|6829|95885|1105012|6139|40866|99773|1811145|5400|527|46|25|19|3|204|163|52|7.47|10.08|9.07|52.52|471.64|388.44|524.16|28.29|0.00|204.36|471.64|499.93|676.00|704.29|83.20| +2451097|71478|2451198|16064|95885|1105012|6139|40866|99773|1811145|5400|527|8|10|7|3|159|163|43|4.81|7.84|3.92|168.56|168.56|206.83|337.12|8.42|0.00|0.00|168.56|176.98|168.56|176.98|-38.27| +2451097|71478|2451211|7880|95885|1105012|6139|40866|99773|1811145|5400|527|44|16|2|3|11|163|100|81.05|203.43|164.77|3866.00|16477.00|8105.00|20343.00|118.63|12522.52|3865.00|3954.48|4073.11|7819.48|7938.11|-4150.52| +2451097|71478|2451107|16046|95885|1105012|6139|40866|99773|1811145|5400|527|34|26|13|4|198|163|73|42.79|60.33|39.21|1541.76|2862.33|3123.67|4404.09|0.00|0.00|131.40|2862.33|2862.33|2993.73|2993.73|-261.34| +2451097|71478|2451164|9952|95885|1105012|6139|40866|99773|1811145|5400|527|7|25|5|2|79|163|67|97.87|133.10|75.86|3835.08|5082.62|6557.29|8917.70|457.43|0.00|89.11|5082.62|5540.05|5171.73|5629.16|-1474.67| +2451097|71478|2451176|560|95885|1105012|6139|40866|99773|1811145|5400|527|19|2|7|2|257|163|34|24.91|41.10|38.22|97.92|1299.48|846.94|1397.40|116.95|0.00|558.96|1299.48|1416.43|1858.44|1975.39|452.54| +2451097|71478|2451099|12799|95885|1105012|6139|40866|99773|1811145|5400|527|10|22|11|5|159|163|71|59.18|110.07|35.22|5314.35|2500.62|4201.78|7814.97|75.01|0.00|1718.91|2500.62|2575.63|4219.53|4294.54|-1701.16| +2451097|71478|2451185|5870|95885|1105012|6139|40866|99773|1811145|5400|527|37|20|16|1|213|163|62|14.96|32.16|5.46|1655.40|338.52|927.52|1993.92|16.92|0.00|338.52|338.52|355.44|677.04|693.96|-589.00| +2451097|71478|2451108|13594|95885|1105012|6139|40866|99773|1811145|5400|527|7|1|8|5|82|163|80|47.78|141.42|111.72|2376.00|8937.60|3822.40|11313.60|625.63|0.00|5316.80|8937.60|9563.23|14254.40|14880.03|5115.20| +2451097|71478|2451187|13898|95885|1105012|6139|40866|99773|1811145|5400|527|25|16|15|1|177|163|99|20.51|22.76|6.82|1578.06|675.18|2030.49|2253.24|2.56|546.89|788.04|128.29|130.85|916.33|918.89|-1902.20| +2451097|71478|2451205|10225|95885|1105012|6139|40866|99773|1811145|5400|527|8|10|14|5|46|163|68|34.11|57.30|44.12|896.24|3000.16|2319.48|3896.40|150.00|0.00|1597.32|3000.16|3150.16|4597.48|4747.48|680.68| +2452635|68960|2452658|4129|36298|1068565|832|32746|65983|93728|5547|33461|3|21|10|2|181|164|86|78.17|195.42|48.85|12605.02|4201.10|6722.62|16806.12|336.08|0.00|3192.32|4201.10|4537.18|7393.42|7729.50|-2521.52| +2452635|68960|2452668|7281|36298|1068565|832|32746|65983|93728|5547|33461|42|27|4|2|277|164|15|81.08|162.97|65.18|1466.85|977.70|1216.20|2444.55|2.73|703.94|806.70|273.76|276.49|1080.46|1083.19|-942.44| +2452635|68960|2452671|14628|36298|1068565|832|32746|65983|93728|5547|33461|31|21|16|1|26|164|86|90.38|234.08|11.70|19124.68|1006.20|7772.68|20130.88|40.24|0.00|2818.22|1006.20|1046.44|3824.42|3864.66|-6766.48| +2452635|68960|2452640|12861|36298|1068565|832|32746|65983|93728|5547|33461|12|25|17|4|11|164|5|57.70|152.32|56.35|479.85|281.75|288.50|761.60|14.08|0.00|83.75|281.75|295.83|365.50|379.58|-6.75| +2452635|68960|2452718|16398|36298|1068565|832|32746|65983|93728|5547|33461|15|12|1|4|104|164|8|16.59|28.20|25.09|24.88|200.72|132.72|225.60|13.32|34.12|103.76|166.60|179.92|270.36|283.68|33.88| +2452635|68960|2452684|14157|36298|1068565|832|32746|65983|93728|5547|33461|42|30|2|1|73|164|71|44.85|85.21|23.00|4416.91|1633.00|3184.35|6049.91|97.98|0.00|241.40|1633.00|1730.98|1874.40|1972.38|-1551.35| +2452635|68960|2452702|16417|36298|1068565|832|32746|65983|93728|5547|33461|19|19|17|4|181|164|1|28.81|65.39|35.31|30.08|35.31|28.81|65.39|3.17|0.00|28.11|35.31|38.48|63.42|66.59|6.50| +2452635|68960|2452666|1098|36298|1068565|832|32746|65983|93728|5547|33461|31|12|9|2|262|164|92|98.78|255.84|194.43|5649.72|17887.56|9087.76|23537.28|1073.25|0.00|11533.12|17887.56|18960.81|29420.68|30493.93|8799.80| +2452635|68960|2452646|2772|36298|1068565|832|32746|65983|93728|5547|33461|24|19|15|3|11|164|31|46.47|66.91|59.54|228.47|1845.74|1440.57|2074.21|55.37|0.00|20.46|1845.74|1901.11|1866.20|1921.57|405.17| +2452635|68960|2452644|72|36298|1068565|832|32746|65983|93728|5547|33461|12|21|16|1|266|164|11|67.23|69.24|22.15|517.99|243.65|739.53|761.64|9.21|112.07|243.65|131.58|140.79|375.23|384.44|-607.95| +2452635|68960|2452650|15603|36298|1068565|832|32746|65983|93728|5547|33461|54|30|20|3|293|164|37|42.50|57.37|24.66|1210.27|912.42|1572.50|2122.69|45.62|0.00|403.30|912.42|958.04|1315.72|1361.34|-660.08| +2452635|68960|2452727|16309|36298|1068565|832|32746|65983|93728|5547|33461|33|13|10|5|233|164|66|72.75|202.97|117.72|5626.50|7769.52|4801.50|13396.02|48.17|5360.96|4152.72|2408.56|2456.73|6561.28|6609.45|-2392.94| +2452635|68960|2452665|6249|36298|1068565|832|32746|65983|93728|5547|33461|43|27|3|5|34|164|31|71.60|189.74|108.15|2529.29|3352.65|2219.60|5881.94|201.15|0.00|1117.55|3352.65|3553.80|4470.20|4671.35|1133.05| +2452596|76372|2452632|5755|59032|1902956|2080|8757|65407|1231216|2482|19606|1|12|18|3|5|165|85|59.18|139.66|110.33|2493.05|9378.05|5030.30|11871.10|33.76|8534.02|5223.25|844.03|877.79|6067.28|6101.04|-4186.27| +2452596|76372|2452613|4506|59032|1902956|2080|8757|65407|1231216|2482|19606|12|7|12|3|21|165|42|63.81|178.02|69.42|4561.20|2915.64|2680.02|7476.84|116.62|0.00|1719.48|2915.64|3032.26|4635.12|4751.74|235.62| +2452596|76372|2452606|6063|59032|1902956|2080|8757|65407|1231216|2482|19606|15|30|5|5|202|165|83|26.85|39.46|35.90|295.48|2979.70|2228.55|3275.18|89.39|0.00|229.08|2979.70|3069.09|3208.78|3298.17|751.15| +2452596|76372|2452700|9721|59032|1902956|2080|8757|65407|1231216|2482|19606|57|6|7|3|232|165|27|89.25|261.50|44.45|5860.35|1200.15|2409.75|7060.50|84.01|0.00|847.26|1200.15|1284.16|2047.41|2131.42|-1209.60| +2452596|76372|2452682|10491|59032|1902956|2080|8757|65407|1231216|2482|19606|12|30|11|2|153|165|10|72.62|131.44|2.62|1288.20|26.20|726.20|1314.40|1.83|0.00|13.10|26.20|28.03|39.30|41.13|-700.00| +2452596|76372|2452689|1963|59032|1902956|2080|8757|65407|1231216|2482|19606|51|9|14|3|122|165|15|43.49|60.01|35.40|369.15|531.00|652.35|900.15|37.17|0.00|9.00|531.00|568.17|540.00|577.17|-121.35| +2452596|76372|2452662|11808|59032|1902956|2080|8757|65407|1231216|2482|19606|30|9|2|4|49|165|36|54.53|147.23|91.28|2014.20|3286.08|1963.08|5300.28|45.34|1018.68|1589.76|2267.40|2312.74|3857.16|3902.50|304.32| +2452596|76372|2452682|9925|59032|1902956|2080|8757|65407|1231216|2482|19606|48|18|1|1|98|165|18|93.19|111.82|101.75|181.26|1831.50|1677.42|2012.76|54.94|0.00|744.66|1831.50|1886.44|2576.16|2631.10|154.08| +2452596|76372|2452688|15717|59032|1902956|2080|8757|65407|1231216|2482|19606|31|15|13|3|210|165|41|76.91|155.35|124.28|1273.87|5095.48|3153.31|6369.35|6.11|4891.66|700.28|203.82|209.93|904.10|910.21|-2949.49| +2452596|76372|2452713|8131|59032|1902956|2080|8757|65407|1231216|2482|19606|25|1|19|2|175|165|23|64.41|151.36|49.94|2332.66|1148.62|1481.43|3481.28|6.20|1045.24|870.32|103.38|109.58|973.70|979.90|-1378.05| +2452596|76372|2452617|3570|59032|1902956|2080|8757|65407|1231216|2482|19606|42|24|13|4|225|165|60|31.11|78.39|19.59|3528.00|1175.40|1866.60|4703.40|23.50|0.00|2163.00|1175.40|1198.90|3338.40|3361.90|-691.20| +2452596|76372|2452685|3373|59032|1902956|2080|8757|65407|1231216|2482|19606|55|3|13|4|125|165|83|17.99|41.73|36.30|450.69|3012.90|1493.17|3463.59|90.38|0.00|103.75|3012.90|3103.28|3116.65|3207.03|1519.73| +2452596|76372|2452684|4737|59032|1902956|2080|8757|65407|1231216|2482|19606|60|6|9|2|95|165|5|25.70|62.19|10.57|258.10|52.85|128.50|310.95|4.75|0.00|62.15|52.85|57.60|115.00|119.75|-75.65| +2452596|76372|2452662|4263|59032|1902956|2080|8757|65407|1231216|2482|19606|60|1|19|5|71|165|41|80.95|152.99|148.40|188.19|6084.40|3318.95|6272.59|304.22|0.00|3010.63|6084.40|6388.62|9095.03|9399.25|2765.45| +2452596|76372|2452676|9618|59032|1902956|2080|8757|65407|1231216|2482|19606|3|12|13|2|71|165|20|40.97|55.30|38.71|331.80|774.20|819.40|1106.00|4.95|712.26|464.40|61.94|66.89|526.34|531.29|-757.46| +2452596|76372|2452609|12373|59032|1902956|2080|8757|65407|1231216|2482|19606|15|13|16|2|131|165|22|66.88|71.56|1.43|1542.86|31.46|1471.36|1574.32|1.57|0.00|393.58|31.46|33.03|425.04|426.61|-1439.90| +2452223|69732|2452254|9707|70307|470143|2332|32968|52448|905632|7182|27418|49|15|1|4|257|166|85|43.25|111.15|8.89|8692.10|755.65|3676.25|9447.75|6.65|672.52|566.10|83.13|89.78|649.23|655.88|-3593.12| +2452223|69732|2452232|8171|70307|470143|2332|32968|52448|905632|7182|27418|15|19|12|4|226|166|91|65.68|162.22|71.37|8267.35|6494.67|5976.88|14762.02|519.57|0.00|4871.23|6494.67|7014.24|11365.90|11885.47|517.79| +2452223|69732|2452231|279|70307|470143|2332|32968|52448|905632|7182|27418|53|7|11|3|288|166|66|45.12|60.91|15.83|2975.28|1044.78|2977.92|4020.06|83.58|0.00|321.42|1044.78|1128.36|1366.20|1449.78|-1933.14| +2452223|69732|2452314|727|70307|470143|2332|32968|52448|905632|7182|27418|29|23|9|1|183|166|91|92.65|178.81|112.65|6020.56|10251.15|8431.15|16271.71|615.06|0.00|3904.81|10251.15|10866.21|14155.96|14771.02|1820.00| +2452223|69732|2452233|1845|70307|470143|2332|32968|52448|905632|7182|27418|17|13|2|2|285|166|79|34.27|67.16|55.07|955.11|4350.53|2707.33|5305.64|87.01|0.00|2121.94|4350.53|4437.54|6472.47|6559.48|1643.20| +2452223|69732|2452235|11761|70307|470143|2332|32968|52448|905632|7182|27418|57|17|19|4|189|166|30|53.83|69.44|64.57|146.10|1937.10|1614.90|2083.20|19.37|0.00|333.30|1937.10|1956.47|2270.40|2289.77|322.20| +2452223|69732|2452334|4707|70307|470143|2332|32968|52448|905632|7182|27418|27|27|4|2|157|166|18|84.41|239.72|196.57|776.70|3538.26|1519.38|4314.96|247.67|0.00|1380.78|3538.26|3785.93|4919.04|5166.71|2018.88| +2452223|69732|2452299|3599|70307|470143|2332|32968|52448|905632|7182|27418|1|29|2|1|13|166|14|78.87|94.64|43.53|715.54|609.42|1104.18|1324.96|54.84|0.00|251.72|609.42|664.26|861.14|915.98|-494.76| +2451066|45529|2451107|3241|31257|909296|2535|33374|31377|1287773|2691|35883|8|10|7|3|39|167|68|35.57|43.39|21.26|1504.84|1445.68|2418.76|2950.52|130.11|0.00|294.44|1445.68|1575.79|1740.12|1870.23|-973.08| +2451066|45529|2451124|8605|31257|909296|2535|33374|31377|1287773|2691|35883|25|26|5|3|24|167|6|37.90|74.28|31.94|254.04|191.64|227.40|445.68|0.38|172.47|35.64|19.17|19.55|54.81|55.19|-208.23| +2451066|45529|2451172|3092|31257|909296|2535|33374|31377|1287773|2691|35883|14|13|19|3|260|167|99|70.19|142.48|39.89|10156.41|3949.11|6948.81|14105.52|236.94|0.00|5218.29|3949.11|4186.05|9167.40|9404.34|-2999.70| +2451066|45529|2451149|8834|31257|909296|2535|33374|31377|1287773|2691|35883|44|2|11|5|89|167|20|87.86|250.40|177.78|1452.40|3555.60|1757.20|5008.00|71.11|0.00|1802.80|3555.60|3626.71|5358.40|5429.51|1798.40| +2451066|45529|2451079|17228|31257|909296|2535|33374|31377|1287773|2691|35883|55|4|11|1|184|167|4|2.59|4.76|4.04|2.88|16.16|10.36|19.04|1.13|0.00|3.96|16.16|17.29|20.12|21.25|5.80| +2451066|45529|2451122|9988|31257|909296|2535|33374|31377|1287773|2691|35883|19|25|6|4|292|167|11|88.71|91.37|62.13|321.64|683.43|975.81|1005.07|3.55|594.58|90.42|88.85|92.40|179.27|182.82|-886.96| +2451066|45529|2451131|16633|31257|909296|2535|33374|31377|1287773|2691|35883|1|20|11|1|34|167|28|8.20|10.66|3.19|209.16|89.32|229.60|298.48|0.89|0.00|131.32|89.32|90.21|220.64|221.53|-140.28| +2451066|45529|2451119|16057|31257|909296|2535|33374|31377|1287773|2691|35883|28|7|10|4|195|167|18|67.22|179.47|21.53|2842.92|387.54|1209.96|3230.46|2.17|170.51|1421.28|217.03|219.20|1638.31|1640.48|-992.93| +2451066|45529|2451180|13754|31257|909296|2535|33374|31377|1287773|2691|35883|43|7|13|5|174|167|31|85.30|243.10|143.42|3090.08|4446.02|2644.30|7536.10|177.84|0.00|753.61|4446.02|4623.86|5199.63|5377.47|1801.72| +2452393|60887|2452493|13623|98729|295328|2680|37232|19656|1603256|4888|27040|9|15|1|2|130|168|91|67.72|79.90|57.52|2036.58|5234.32|6162.52|7270.90|106.25|3716.36|944.58|1517.96|1624.21|2462.54|2568.79|-4644.56| +2452393|60887|2452511|6972|98729|295328|2680|37232|19656|1603256|4888|27040|31|18|9|2|242|168|47|88.71|198.71|149.03|2334.96|7004.41|4169.37|9339.37|70.04|0.00|4295.80|7004.41|7074.45|11300.21|11370.25|2835.04| +2452393|60887|2452508|5191|98729|295328|2680|37232|19656|1603256|4888|27040|6|1|6|4|219|168|14|44.86|118.87|27.34|1281.42|382.76|628.04|1664.18|2.06|348.31|49.84|34.45|36.51|84.29|86.35|-593.59| +2452393|60887|2452459|11161|98729|295328|2680|37232|19656|1603256|4888|27040|24|3|12|5|175|168|13|55.30|128.84|82.45|603.07|1071.85|718.90|1674.92|10.71|0.00|385.19|1071.85|1082.56|1457.04|1467.75|352.95| +2452393|60887|2452396|8517|98729|295328|2680|37232|19656|1603256|4888|27040|3|7|19|1|183|168|100|51.39|90.96|40.02|5094.00|4002.00|5139.00|9096.00|40.02|0.00|4548.00|4002.00|4042.02|8550.00|8590.02|-1137.00| +2452393|60887|2452477|2670|98729|295328|2680|37232|19656|1603256|4888|27040|54|13|17|2|37|168|6|7.12|20.50|14.76|34.44|88.56|42.72|123.00|7.97|0.00|2.46|88.56|96.53|91.02|98.99|45.84| +2452393|60887|2452503|5853|98729|295328|2680|37232|19656|1603256|4888|27040|9|12|4|5|23|168|39|26.11|49.08|46.62|95.94|1818.18|1018.29|1914.12|109.09|0.00|344.37|1818.18|1927.27|2162.55|2271.64|799.89| +2452393|60887|2452497|10737|98729|295328|2680|37232|19656|1603256|4888|27040|45|15|16|1|106|168|22|5.33|12.09|3.86|181.06|84.92|117.26|265.98|3.39|0.00|13.20|84.92|88.31|98.12|101.51|-32.34| +2452393|60887|2452437|9426|98729|295328|2680|37232|19656|1603256|4888|27040|36|21|14|2|147|168|70|56.78|122.64|74.81|3348.10|5236.70|3974.60|8584.80|471.30|0.00|1716.40|5236.70|5708.00|6953.10|7424.40|1262.10| +2452393|60887|2452397|5994|98729|295328|2680|37232|19656|1603256|4888|27040|54|15|6|4|272|168|30|85.86|106.46|99.00|223.80|2970.00|2575.80|3193.80|178.20|0.00|1277.40|2970.00|3148.20|4247.40|4425.60|394.20| +2452393|60887|2452502|2385|98729|295328|2680|37232|19656|1603256|4888|27040|19|13|1|3|155|168|45|75.12|184.04|115.94|3064.50|5217.30|3380.40|8281.80|4.17|5008.60|828.00|208.70|212.87|1036.70|1040.87|-3171.70| +2452393|60887|2452497|7891|98729|295328|2680|37232|19656|1603256|4888|27040|9|21|4|5|126|168|64|37.47|95.54|3.82|5870.08|244.48|2398.08|6114.56|4.88|0.00|2751.36|244.48|249.36|2995.84|3000.72|-2153.60| +2450975|26287|2450988|4250|61908|1113854|5447|33393|34300|1637354|5325|34471|16|13|15|3|211|169|76|52.16|56.85|44.34|950.76|3369.84|3964.16|4320.60|202.19|0.00|1123.28|3369.84|3572.03|4493.12|4695.31|-594.32| +2450975|26287|2451061|15994|61908|1113854|5447|33393|34300|1637354|5325|34471|40|20|1|3|198|169|17|47.30|93.18|5.59|1489.03|95.03|804.10|1584.06|2.85|0.00|633.59|95.03|97.88|728.62|731.47|-709.07| +2450975|26287|2451056|11444|61908|1113854|5447|33393|34300|1637354|5325|34471|1|2|6|2|118|169|88|26.26|62.49|51.86|935.44|4563.68|2310.88|5499.12|365.09|0.00|1979.12|4563.68|4928.77|6542.80|6907.89|2252.80| +2450975|26287|2451024|12838|61908|1113854|5447|33393|34300|1637354|5325|34471|46|4|19|3|67|169|4|93.11|243.01|92.34|602.68|369.36|372.44|972.04|4.65|214.22|194.40|155.14|159.79|349.54|354.19|-217.30| +2450975|26287|2451058|15529|61908|1113854|5447|33393|34300|1637354|5325|34471|4|1|19|2|265|169|42|97.29|193.60|152.94|1707.72|6423.48|4086.18|8131.20|385.40|0.00|1300.74|6423.48|6808.88|7724.22|8109.62|2337.30| +2450975|26287|2450976|14258|61908|1113854|5447|33393|34300|1637354|5325|34471|50|13|15|1|24|169|81|68.69|164.16|83.72|6515.64|6781.32|5563.89|13296.96|0.00|0.00|132.84|6781.32|6781.32|6914.16|6914.16|1217.43| +2450975|26287|2451074|9838|61908|1113854|5447|33393|34300|1637354|5325|34471|22|7|9|1|153|169|43|40.96|70.45|64.81|242.52|2786.83|1761.28|3029.35|0.00|0.00|1332.57|2786.83|2786.83|4119.40|4119.40|1025.55| +2450975|26287|2451050|2386|61908|1113854|5447|33393|34300|1637354|5325|34471|38|26|5|4|70|169|67|57.69|159.80|94.28|4389.84|6316.76|3865.23|10706.60|0.00|3600.55|5138.90|2716.21|2716.21|7855.11|7855.11|-1149.02| +2450975|26287|2450991|2131|61908|1113854|5447|33393|34300|1637354|5325|34471|49|19|15|3|228|169|91|37.86|60.95|29.86|2829.19|2717.26|3445.26|5546.45|135.86|0.00|1219.40|2717.26|2853.12|3936.66|4072.52|-728.00| +2450975|26287|2451032|5653|61908|1113854|5447|33393|34300|1637354|5325|34471|43|13|15|5|8|169|89|70.63|117.95|97.89|1785.34|8712.21|6286.07|10497.55|323.22|4094.73|419.19|4617.48|4940.70|5036.67|5359.89|-1668.59| +2450975|26287|2451000|16922|61908|1113854|5447|33393|34300|1637354|5325|34471|14|25|18|3|208|169|69|35.75|62.56|28.77|2331.51|1985.13|2466.75|4316.64|138.95|0.00|1682.91|1985.13|2124.08|3668.04|3806.99|-481.62| +2450975|26287|2450985|5545|61908|1113854|5447|33393|34300|1637354|5325|34471|19|16|15|1|87|169|23|36.43|43.71|17.04|613.41|391.92|837.89|1005.33|12.54|235.15|361.79|156.77|169.31|518.56|531.10|-681.12| +2450975|26287|2451072|12067|61908|1113854|5447|33393|34300|1637354|5325|34471|34|2|6|2|81|169|17|62.40|101.08|2.02|1684.02|34.34|1060.80|1718.36|1.37|0.00|103.02|34.34|35.71|137.36|138.73|-1026.46| +2450975|26287|2451035|10258|61908|1113854|5447|33393|34300|1637354|5325|34471|19|8|20|5|117|169|21|32.92|75.05|44.27|646.38|929.67|691.32|1576.05|65.07|0.00|519.96|929.67|994.74|1449.63|1514.70|238.35| +2450975|26287|2451021|7339|61908|1113854|5447|33393|34300|1637354|5325|34471|20|25|12|4|213|169|53|67.46|114.68|44.72|3707.88|2370.16|3575.38|6078.04|23.70|0.00|2491.53|2370.16|2393.86|4861.69|4885.39|-1205.22| +2450975|26287|2450987|4406|61908|1113854|5447|33393|34300|1637354|5325|34471|56|16|19|4|279|169|57|39.84|66.53|11.97|3109.92|682.29|2270.88|3792.21|14.12|211.50|644.67|470.79|484.91|1115.46|1129.58|-1800.09| +2452442|69916|2452521|9180|64326|1707814|4102|23590|96852|1320516|2407|2295|13|13|10|5|293|170|87|77.51|138.74|29.13|9536.07|2534.31|6743.37|12070.38|76.02|0.00|1085.76|2534.31|2610.33|3620.07|3696.09|-4209.06| +2452442|69916|2452458|13950|64326|1707814|4102|23590|96852|1320516|2407|2295|19|9|10|5|290|170|10|3.20|7.80|0.31|74.90|3.10|32.00|78.00|0.21|0.00|16.30|3.10|3.31|19.40|19.61|-28.90| +2452442|69916|2452479|3000|64326|1707814|4102|23590|96852|1320516|2407|2295|25|18|8|4|260|170|36|95.91|218.67|26.24|6927.48|944.64|3452.76|7872.12|0.00|0.00|3778.56|944.64|944.64|4723.20|4723.20|-2508.12| +2452442|69916|2452498|873|64326|1707814|4102|23590|96852|1320516|2407|2295|55|19|12|2|249|170|83|42.79|85.15|6.81|6502.22|565.23|3551.57|7067.45|28.26|0.00|2968.08|565.23|593.49|3533.31|3561.57|-2986.34| +2452442|69916|2452524|1399|64326|1707814|4102|23590|96852|1320516|2407|2295|1|7|10|3|216|170|90|39.69|69.45|53.47|1438.20|4812.30|3572.10|6250.50|85.17|1973.04|1874.70|2839.26|2924.43|4713.96|4799.13|-732.84| +2452442|69916|2452557|4627|64326|1707814|4102|23590|96852|1320516|2407|2295|45|12|2|5|151|170|1|27.09|39.82|21.10|18.72|21.10|27.09|39.82|1.26|0.00|10.35|21.10|22.36|31.45|32.71|-5.99| +2452442|69916|2452459|8856|64326|1707814|4102|23590|96852|1320516|2407|2295|48|12|8|5|42|170|27|4.20|4.36|1.70|71.82|45.90|113.40|117.72|4.13|0.00|24.57|45.90|50.03|70.47|74.60|-67.50| +2452442|69916|2452482|10561|64326|1707814|4102|23590|96852|1320516|2407|2295|42|21|1|3|18|170|3|24.59|27.78|1.66|78.36|4.98|73.77|83.34|0.24|0.00|27.48|4.98|5.22|32.46|32.70|-68.79| +2452442|69916|2452525|5013|64326|1707814|4102|23590|96852|1320516|2407|2295|45|27|15|1|17|170|79|25.74|49.67|15.39|2708.12|1215.81|2033.46|3923.93|85.10|0.00|1804.36|1215.81|1300.91|3020.17|3105.27|-817.65| +2452442|69916|2452517|2953|64326|1707814|4102|23590|96852|1320516|2407|2295|13|30|16|4|55|170|30|73.67|100.92|57.52|1302.00|1725.60|2210.10|3027.60|120.79|0.00|514.50|1725.60|1846.39|2240.10|2360.89|-484.50| +2452442|69916|2452451|6849|64326|1707814|4102|23590|96852|1320516|2407|2295|48|25|19|3|84|170|52|80.69|179.13|177.33|93.60|9221.16|4195.88|9314.76|645.48|0.00|3725.80|9221.16|9866.64|12946.96|13592.44|5025.28| +2452442|69916|2452499|9312|64326|1707814|4102|23590|96852|1320516|2407|2295|7|7|4|4|94|170|86|32.63|81.24|67.42|1188.52|5798.12|2806.18|6986.64|347.88|0.00|1117.14|5798.12|6146.00|6915.26|7263.14|2991.94| +2452523|63972|2452613|8539|6935|378240|1898|49603|97873|862454|5071|34376|13|25|9|5|121|171|83|4.03|8.62|3.96|386.78|328.68|334.49|715.46|6.57|0.00|307.10|328.68|335.25|635.78|642.35|-5.81| +2452523|63972|2452555|2491|6935|378240|1898|49603|97873|862454|5071|34376|25|30|17|1|178|171|37|49.61|136.92|136.92|0.00|5066.04|1835.57|5066.04|0.00|1317.17|911.68|3748.87|3748.87|4660.55|4660.55|1913.30| +2452523|63972|2452574|4830|6935|378240|1898|49603|97873|862454|5071|34376|60|3|11|3|292|171|46|52.56|113.52|73.78|1828.04|3393.88|2417.76|5221.92|237.57|0.00|782.92|3393.88|3631.45|4176.80|4414.37|976.12| +2452523|63972|2452640|3798|6935|378240|1898|49603|97873|862454|5071|34376|57|27|2|2|142|171|50|56.77|166.90|20.02|7344.00|1001.00|2838.50|8345.00|80.08|0.00|3838.50|1001.00|1081.08|4839.50|4919.58|-1837.50| +2452523|63972|2452575|16777|6935|378240|1898|49603|97873|862454|5071|34376|33|3|10|5|3|171|50|29.13|66.12|58.18|397.00|2909.00|1456.50|3306.00|29.09|0.00|429.50|2909.00|2938.09|3338.50|3367.59|1452.50| +2452523|63972|2452619|7539|6935|378240|1898|49603|97873|862454|5071|34376|6|21|19|2|185|171|45|80.57|148.24|117.10|1401.30|5269.50|3625.65|6670.80|206.56|2687.44|133.20|2582.06|2788.62|2715.26|2921.82|-1043.59| +2452523|63972|2452567|11799|6935|378240|1898|49603|97873|862454|5071|34376|36|27|10|3|99|171|75|52.54|101.92|77.45|1835.25|5808.75|3940.50|7644.00|0.00|0.00|840.75|5808.75|5808.75|6649.50|6649.50|1868.25| +2452523|63972|2452528|15963|6935|378240|1898|49603|97873|862454|5071|34376|7|24|13|1|89|171|60|65.85|162.64|9.75|9173.40|585.00|3951.00|9758.40|17.55|0.00|2439.60|585.00|602.55|3024.60|3042.15|-3366.00| +2452523|63972|2452594|7017|6935|378240|1898|49603|97873|862454|5071|34376|19|6|19|2|35|171|25|36.10|100.35|82.28|451.75|2057.00|902.50|2508.75|123.42|0.00|702.25|2057.00|2180.42|2759.25|2882.67|1154.50| +2452523|63972|2452524|15073|6935|378240|1898|49603|97873|862454|5071|34376|37|30|1|4|130|171|98|11.88|35.52|29.83|557.62|2923.34|1164.24|3480.96|263.10|0.00|1530.76|2923.34|3186.44|4454.10|4717.20|1759.10| +2452523|63972|2452573|7905|6935|378240|1898|49603|97873|862454|5071|34376|36|25|1|1|134|171|69|71.70|140.53|9.83|9018.30|678.27|4947.30|9696.57|20.34|0.00|2230.08|678.27|698.61|2908.35|2928.69|-4269.03| +2452523|63972|2452575|8649|6935|378240|1898|49603|97873|862454|5071|34376|7|9|16|5|61|171|72|11.74|25.24|25.24|0.00|1817.28|845.28|1817.28|145.38|0.00|708.48|1817.28|1962.66|2525.76|2671.14|972.00| +2452523|63972|2452543|8961|6935|378240|1898|49603|97873|862454|5071|34376|60|25|16|4|172|171|94|55.32|110.08|25.31|7968.38|2379.14|5200.08|10347.52|47.58|0.00|4345.62|2379.14|2426.72|6724.76|6772.34|-2820.94| +2452523|63972|2452642|8029|6935|378240|1898|49603|97873|862454|5071|34376|21|21|8|3|205|171|53|95.03|107.38|77.31|1593.71|4097.43|5036.59|5691.14|122.92|0.00|853.30|4097.43|4220.35|4950.73|5073.65|-939.16| +2452523|63972|2452533|1857|6935|378240|1898|49603|97873|862454|5071|34376|13|24|17|2|87|171|50|20.81|28.09|16.57|576.00|828.50|1040.50|1404.50|26.92|289.97|0.00|538.53|565.45|538.53|565.45|-501.97| +2452523|63972|2452638|13362|6935|378240|1898|49603|97873|862454|5071|34376|43|6|14|5|66|171|40|13.17|18.56|8.90|386.40|356.00|526.80|742.40|10.68|0.00|163.20|356.00|366.68|519.20|529.88|-170.80| +2451579|44932|2451588|8369|90264|83437|5763|37336|41749|1159436|7161|44334|50|19|7|4|181|172|91|2.40|5.13|2.56|233.87|232.96|218.40|466.83|8.57|18.63|121.03|214.33|222.90|335.36|343.93|-4.07| +2451579|44932|2451604|8353|90264|83437|5763|37336|41749|1159436|7161|44334|35|5|18|3|281|172|54|52.49|109.17|46.94|3360.42|2534.76|2834.46|5895.18|126.73|0.00|2062.80|2534.76|2661.49|4597.56|4724.29|-299.70| +2451579|44932|2451638|11852|90264|83437|5763|37336|41749|1159436|7161|44334|17|25|7|1|115|172|49|92.31|186.46|78.31|5299.35|3837.19|4523.19|9136.54|115.11|0.00|3928.33|3837.19|3952.30|7765.52|7880.63|-686.00| +2451579|44932|2451609|6541|90264|83437|5763|37336|41749|1159436|7161|44334|11|14|20|2|194|172|53|57.86|76.37|23.67|2793.10|1254.51|3066.58|4047.61|37.63|0.00|1983.26|1254.51|1292.14|3237.77|3275.40|-1812.07| +2451579|44932|2451591|2729|90264|83437|5763|37336|41749|1159436|7161|44334|38|7|11|5|122|172|7|39.71|100.46|97.44|21.14|682.08|277.97|703.22|6.82|0.00|105.42|682.08|688.90|787.50|794.32|404.11| +2451579|44932|2451631|8099|90264|83437|5763|37336|41749|1159436|7161|44334|47|20|5|4|196|172|5|34.52|63.51|13.33|250.90|66.65|172.60|317.55|0.66|0.00|114.30|66.65|67.31|180.95|181.61|-105.95| +2451579|44932|2451690|3107|90264|83437|5763|37336|41749|1159436|7161|44334|49|7|16|3|10|172|92|89.99|264.57|198.42|6085.80|18254.64|8279.08|24340.44|182.54|0.00|10465.92|18254.64|18437.18|28720.56|28903.10|9975.56| +2451579|44932|2451618|16345|90264|83437|5763|37336|41749|1159436|7161|44334|47|1|5|2|36|172|32|5.31|11.68|2.68|288.00|85.76|169.92|373.76|0.00|0.00|123.20|85.76|85.76|208.96|208.96|-84.16| +2451579|44932|2451624|12653|90264|83437|5763|37336|41749|1159436|7161|44334|29|23|9|1|3|172|53|59.60|174.03|100.93|3874.30|5349.29|3158.80|9223.59|97.35|3958.47|1106.64|1390.82|1488.17|2497.46|2594.81|-1767.98| +2451579|44932|2451611|10705|90264|83437|5763|37336|41749|1159436|7161|44334|2|14|4|5|26|172|26|79.85|122.96|33.19|2334.02|862.94|2076.10|3196.96|0.00|0.00|1086.80|862.94|862.94|1949.74|1949.74|-1213.16| +2451579|44932|2451591|6464|90264|83437|5763|37336|41749|1159436|7161|44334|23|5|4|5|6|172|42|25.51|33.41|33.07|14.28|1388.94|1071.42|1403.22|41.66|0.00|364.56|1388.94|1430.60|1753.50|1795.16|317.52| +2451412|80718|2451512|17798|88132|992675|6465|27455|11924|272309|5978|24121|16|26|19|4|82|173|10|20.04|22.04|10.57|114.70|105.70|200.40|220.40|7.39|0.00|99.10|105.70|113.09|204.80|212.19|-94.70| +2451412|80718|2451526|17722|88132|992675|6465|27455|11924|272309|5978|24121|10|19|7|1|248|173|62|93.15|138.79|84.66|3356.06|5248.92|5775.30|8604.98|0.00|0.00|3613.98|5248.92|5248.92|8862.90|8862.90|-526.38| +2451412|80718|2451441|5480|88132|992675|6465|27455|11924|272309|5978|24121|38|26|5|4|213|173|11|68.79|121.75|26.78|1044.67|294.58|756.69|1339.25|17.67|0.00|348.15|294.58|312.25|642.73|660.40|-462.11| +2451412|80718|2451503|343|88132|992675|6465|27455|11924|272309|5978|24121|55|22|6|5|174|173|68|20.44|59.48|46.98|850.00|3194.64|1389.92|4044.64|191.67|0.00|606.56|3194.64|3386.31|3801.20|3992.87|1804.72| +2451412|80718|2451489|2908|88132|992675|6465|27455|11924|272309|5978|24121|31|22|1|2|113|173|91|99.08|231.84|162.28|6329.96|14767.48|9016.28|21097.44|1329.07|0.00|9915.36|14767.48|16096.55|24682.84|26011.91|5751.20| +2451412|80718|2451435|6274|88132|992675|6465|27455|11924|272309|5978|24121|44|26|15|4|220|173|75|97.50|251.55|45.27|15471.00|3395.25|7312.50|18866.25|101.85|0.00|8866.50|3395.25|3497.10|12261.75|12363.60|-3917.25| +2451412|80718|2451526|3082|88132|992675|6465|27455|11924|272309|5978|24121|32|26|4|1|148|173|47|31.37|70.58|0.70|3284.36|32.90|1474.39|3317.26|1.26|7.56|364.72|25.34|26.60|390.06|391.32|-1449.05| +2451412|80718|2451520|15661|88132|992675|6465|27455|11924|272309|5978|24121|52|26|8|2|169|173|79|71.51|201.65|102.84|7805.99|8124.36|5649.29|15930.35|324.97|0.00|3982.39|8124.36|8449.33|12106.75|12431.72|2475.07| +2451412|80718|2451487|17464|88132|992675|6465|27455|11924|272309|5978|24121|8|19|15|5|265|173|32|10.17|10.37|6.32|129.60|202.24|325.44|331.84|12.13|0.00|92.80|202.24|214.37|295.04|307.17|-123.20| +2451412|80718|2451477|9010|88132|992675|6465|27455|11924|272309|5978|24121|46|4|9|5|164|173|30|30.99|34.39|12.72|650.10|381.60|929.70|1031.70|6.79|41.97|340.20|339.63|346.42|679.83|686.62|-590.07| +2451412|80718|2451498|5287|88132|992675|6465|27455|11924|272309|5978|24121|44|7|5|1|217|173|38|42.40|87.34|6.98|3053.68|265.24|1611.20|3318.92|2.65|0.00|331.74|265.24|267.89|596.98|599.63|-1345.96| +2451412|80718|2451435|15050|88132|992675|6465|27455|11924|272309|5978|24121|19|2|14|1|128|173|29|83.11|194.47|159.46|1015.29|4624.34|2410.19|5639.63|231.21|0.00|2255.62|4624.34|4855.55|6879.96|7111.17|2214.15| +2451412|80718|2451499|6242|88132|992675|6465|27455|11924|272309|5978|24121|13|26|18|5|42|173|57|86.61|153.29|104.23|2796.42|5941.11|4936.77|8737.53|415.87|0.00|1572.63|5941.11|6356.98|7513.74|7929.61|1004.34| +2451412|80718|2451450|16166|88132|992675|6465|27455|11924|272309|5978|24121|14|2|6|5|284|173|12|36.69|80.35|74.72|67.56|896.64|440.28|964.20|17.93|0.00|356.64|896.64|914.57|1253.28|1271.21|456.36| +2451412|80718|2451527|9085|88132|992675|6465|27455|11924|272309|5978|24121|19|16|1|5|119|173|16|54.82|160.62|146.16|231.36|2338.56|877.12|2569.92|140.31|0.00|719.52|2338.56|2478.87|3058.08|3198.39|1461.44| +2450898|74967|2450912|4142|8627|760991|6602|7413|77647|1194536|4599|33363|49|25|14|5|196|174|4|90.79|230.60|115.30|461.20|461.20|363.16|922.40|4.61|0.00|267.48|461.20|465.81|728.68|733.29|98.04| +2450898|74967|2450916|11935|8627|760991|6602|7413|77647|1194536|4599|33363|1|10|11|1|254|174|83|89.72|139.96|46.18|7783.74|3832.94|7446.76|11616.68|114.98|0.00|1044.97|3832.94|3947.92|4877.91|4992.89|-3613.82| +2450898|74967|2451007|1351|8627|760991|6602|7413|77647|1194536|4599|33363|52|20|8|2|9|174|37|22.15|54.04|42.69|419.95|1579.53|819.55|1999.48|142.15|0.00|439.56|1579.53|1721.68|2019.09|2161.24|759.98| +2450898|74967|2450947|631|8627|760991|6602|7413|77647|1194536|4599|33363|31|4|20|2|144|174|41|23.06|60.64|19.40|1690.84|795.40|945.46|2486.24|10.49|270.43|24.60|524.97|535.46|549.57|560.06|-420.49| +2450898|74967|2450985|2054|8627|760991|6602|7413|77647|1194536|4599|33363|34|28|12|1|253|174|50|1.68|3.25|3.18|3.50|159.00|84.00|162.50|2.86|63.60|47.00|95.40|98.26|142.40|145.26|11.40| +2450898|74967|2450988|15386|8627|760991|6602|7413|77647|1194536|4599|33363|34|8|9|1|93|174|62|21.08|51.64|22.20|1825.28|1376.40|1306.96|3201.68|68.82|0.00|1248.06|1376.40|1445.22|2624.46|2693.28|69.44| +2450898|74967|2450997|12907|8627|760991|6602|7413|77647|1194536|4599|33363|14|25|18|1|141|174|40|25.90|55.16|49.09|242.80|1963.60|1036.00|2206.40|98.18|0.00|441.20|1963.60|2061.78|2404.80|2502.98|927.60| +2450898|74967|2450968|16292|8627|760991|6602|7413|77647|1194536|4599|33363|50|20|16|1|109|174|50|44.30|87.27|56.72|1527.50|2836.00|2215.00|4363.50|28.36|0.00|1134.50|2836.00|2864.36|3970.50|3998.86|621.00| +2450898|74967|2451016|10417|8627|760991|6602|7413|77647|1194536|4599|33363|58|28|10|3|173|174|75|71.59|88.05|3.52|6339.75|264.00|5369.25|6603.75|0.84|242.88|924.00|21.12|21.96|945.12|945.96|-5348.13| +2450898|74967|2450959|6310|8627|760991|6602|7413|77647|1194536|4599|33363|25|7|2|3|81|174|85|48.08|56.73|17.01|3376.20|1445.85|4086.80|4822.05|72.29|0.00|1012.35|1445.85|1518.14|2458.20|2530.49|-2640.95| +2450898|74967|2450907|6640|8627|760991|6602|7413|77647|1194536|4599|33363|19|14|13|1|219|174|13|5.63|6.47|0.77|74.10|10.01|73.19|84.11|0.90|0.00|25.22|10.01|10.91|35.23|36.13|-63.18| +2450898|74967|2450946|16909|8627|760991|6602|7413|77647|1194536|4599|33363|20|20|1|1|71|174|39|30.78|78.18|25.01|2073.63|975.39|1200.42|3049.02|87.78|0.00|579.15|975.39|1063.17|1554.54|1642.32|-225.03| +2450898|74967|2451007|13795|8627|760991|6602|7413|77647|1194536|4599|33363|55|10|1|1|44|174|91|44.39|102.09|47.98|4924.01|4366.18|4039.49|9290.19|0.00|0.00|928.20|4366.18|4366.18|5294.38|5294.38|326.69| +2450898|74967|2450967|2734|8627|760991|6602|7413|77647|1194536|4599|33363|49|13|19|1|103|174|1|22.58|65.70|12.48|53.22|12.48|22.58|65.70|0.59|3.99|1.31|8.49|9.08|9.80|10.39|-14.09| +2450898|74967|2450910|9358|8627|760991|6602|7413|77647|1194536|4599|33363|44|1|16|4|122|174|100|38.51|52.75|42.20|1055.00|4220.00|3851.00|5275.00|126.60|0.00|1107.00|4220.00|4346.60|5327.00|5453.60|369.00| +2450898|74967|2450973|13502|8627|760991|6602|7413|77647|1194536|4599|33363|10|28|4|3|139|174|50|91.73|104.57|0.00|5228.50|0.00|4586.50|5228.50|0.00|0.00|1516.00|0.00|0.00|1516.00|1516.00|-4586.50| +2452028|21209|2452029|7399|34443|1133706|5416|23879|27320|100561|6562|32225|27|13|10|3|106|175|56|87.75|146.54|82.06|3610.88|4595.36|4914.00|8206.24|0.00|0.00|3938.48|4595.36|4595.36|8533.84|8533.84|-318.64| +2452028|21209|2452090|11397|34443|1133706|5416|23879|27320|100561|6562|32225|57|7|16|1|296|175|19|81.77|140.64|56.25|1603.41|1068.75|1553.63|2672.16|64.12|0.00|400.71|1068.75|1132.87|1469.46|1533.58|-484.88| +2452028|21209|2452039|13557|34443|1133706|5416|23879|27320|100561|6562|32225|7|19|16|2|238|175|71|81.37|201.79|115.02|6160.67|8166.42|5777.27|14327.09|571.64|0.00|6017.25|8166.42|8738.06|14183.67|14755.31|2389.15| +2452028|21209|2452087|17949|34443|1133706|5416|23879|27320|100561|6562|32225|55|21|11|1|244|175|53|50.39|65.50|19.65|2430.05|1041.45|2670.67|3471.50|93.73|0.00|173.31|1041.45|1135.18|1214.76|1308.49|-1629.22| +2452028|21209|2452142|17449|34443|1133706|5416|23879|27320|100561|6562|32225|47|5|2|1|300|175|78|86.67|243.54|238.66|380.64|18615.48|6760.26|18996.12|1116.92|0.00|7408.44|18615.48|19732.40|26023.92|27140.84|11855.22| +2452028|21209|2452079|16757|34443|1133706|5416|23879|27320|100561|6562|32225|1|9|6|5|294|175|50|91.98|200.51|172.43|1404.00|8621.50|4599.00|10025.50|362.10|3448.60|1102.50|5172.90|5535.00|6275.40|6637.50|573.90| +2452028|21209|2452117|8183|34443|1133706|5416|23879|27320|100561|6562|32225|55|29|5|2|149|175|97|94.36|248.16|223.34|2407.54|21663.98|9152.92|24071.52|0.00|0.00|5054.67|21663.98|21663.98|26718.65|26718.65|12511.06| +2452028|21209|2452078|10389|34443|1133706|5416|23879|27320|100561|6562|32225|9|17|20|4|155|175|8|37.22|51.73|42.93|70.40|343.44|297.76|413.84|27.47|0.00|66.16|343.44|370.91|409.60|437.07|45.68| +2452028|21209|2452111|3901|34443|1133706|5416|23879|27320|100561|6562|32225|53|3|17|2|144|175|84|95.22|142.83|22.85|10078.32|1919.40|7998.48|11997.72|19.19|0.00|2878.68|1919.40|1938.59|4798.08|4817.27|-6079.08| +2452028|21209|2452039|10489|34443|1133706|5416|23879|27320|100561|6562|32225|33|19|13|1|289|175|49|88.35|108.67|21.73|4260.06|1064.77|4329.15|5324.83|21.29|0.00|1703.73|1064.77|1086.06|2768.50|2789.79|-3264.38| +2452028|21209|2452052|5513|34443|1133706|5416|23879|27320|100561|6562|32225|37|23|2|5|5|175|71|2.66|4.73|2.83|134.90|200.93|188.86|335.83|14.06|0.00|167.56|200.93|214.99|368.49|382.55|12.07| +2452028|21209|2452060|15249|34443|1133706|5416|23879|27320|100561|6562|32225|39|3|7|4|264|175|83|63.25|175.20|47.30|10615.70|3925.90|5249.75|14541.60|39.25|0.00|1744.66|3925.90|3965.15|5670.56|5709.81|-1323.85| +2452028|21209|2452121|15455|34443|1133706|5416|23879|27320|100561|6562|32225|31|17|2|5|48|175|78|56.06|139.02|116.77|1735.50|9108.06|4372.68|10843.56|91.08|0.00|1734.72|9108.06|9199.14|10842.78|10933.86|4735.38| +2452028|21209|2452096|10621|34443|1133706|5416|23879|27320|100561|6562|32225|51|9|7|4|198|175|71|4.39|6.10|5.24|61.06|372.04|311.69|433.10|0.00|0.00|151.23|372.04|372.04|523.27|523.27|60.35| +2452028|21209|2452138|17277|34443|1133706|5416|23879|27320|100561|6562|32225|23|21|3|2|144|175|93|35.09|69.47|63.21|582.18|5878.53|3263.37|6460.71|0.00|0.00|3036.45|5878.53|5878.53|8914.98|8914.98|2615.16| +2451905|7564|2451977|12175|58824|513915|1006|47297|78110|314490|2617|48908|8|14|1|4|47|176|67|57.59|69.68|31.35|2568.11|2100.45|3858.53|4668.56|105.02|0.00|2007.32|2100.45|2205.47|4107.77|4212.79|-1758.08| +2451905|7564|2452013|16925|58824|513915|1006|47297|78110|314490|2617|48908|41|13|13|4|235|176|80|74.39|148.78|56.53|7380.00|4522.40|5951.20|11902.40|361.79|0.00|118.40|4522.40|4884.19|4640.80|5002.59|-1428.80| +2451905|7564|2451999|11971|58824|513915|1006|47297|78110|314490|2617|48908|50|25|10|5|297|176|31|83.44|199.42|29.91|5254.81|927.21|2586.64|6182.02|27.53|9.27|1297.97|917.94|945.47|2215.91|2243.44|-1668.70| +2451905|7564|2451953|17027|58824|513915|1006|47297|78110|314490|2617|48908|8|17|3|2|201|176|15|65.31|160.66|99.60|915.90|1494.00|979.65|2409.90|33.16|388.44|96.30|1105.56|1138.72|1201.86|1235.02|125.91| +2451905|7564|2451927|8831|58824|513915|1006|47297|78110|314490|2617|48908|2|29|18|1|264|176|93|49.82|56.79|3.40|4965.27|316.20|4633.26|5281.47|0.00|0.00|2270.13|316.20|316.20|2586.33|2586.33|-4317.06| +2451905|7564|2451985|2018|58824|513915|1006|47297|78110|314490|2617|48908|31|20|19|4|109|176|86|18.40|54.28|15.74|3314.44|1353.64|1582.40|4668.08|81.21|0.00|1726.88|1353.64|1434.85|3080.52|3161.73|-228.76| +2451905|7564|2451914|14843|58824|513915|1006|47297|78110|314490|2617|48908|25|17|15|1|117|176|17|27.36|51.71|29.47|378.08|500.99|465.12|879.07|5.00|0.00|377.91|500.99|505.99|878.90|883.90|35.87| +2451905|7564|2451987|9331|58824|513915|1006|47297|78110|314490|2617|48908|44|17|15|1|94|176|77|61.27|173.39|17.33|12016.62|1334.41|4717.79|13351.03|0.00|0.00|6541.92|1334.41|1334.41|7876.33|7876.33|-3383.38| +2451905|7564|2451956|1592|58824|513915|1006|47297|78110|314490|2617|48908|7|17|4|4|24|176|95|97.68|144.56|63.60|7691.20|6042.00|9279.60|13733.20|483.36|0.00|3570.10|6042.00|6525.36|9612.10|10095.46|-3237.60| +2451905|7564|2452008|2731|58824|513915|1006|47297|78110|314490|2617|48908|47|5|15|2|106|176|2|30.43|65.42|9.15|112.54|18.30|60.86|130.84|0.01|16.47|39.24|1.83|1.84|41.07|41.08|-59.03| +2451905|7564|2451990|4040|58824|513915|1006|47297|78110|314490|2617|48908|5|1|9|4|122|176|32|89.41|165.40|43.00|3916.80|1376.00|2861.12|5292.80|123.84|0.00|1905.28|1376.00|1499.84|3281.28|3405.12|-1485.12| +2452143|58049|2452207|8379|52671|974909|3194|9032|74042|1045013|146|5984|51|19|4|1|285|177|63|16.21|33.87|5.41|1792.98|340.83|1021.23|2133.81|10.22|0.00|575.82|340.83|351.05|916.65|926.87|-680.40| +2452143|58049|2452211|10471|52671|974909|3194|9032|74042|1045013|146|5984|5|17|18|4|223|177|8|62.66|154.14|26.20|1023.52|209.60|501.28|1233.12|14.67|0.00|493.20|209.60|224.27|702.80|717.47|-291.68| +2452143|58049|2452230|12053|52671|974909|3194|9032|74042|1045013|146|5984|27|29|6|2|48|177|29|97.86|194.74|151.89|1242.65|4404.81|2837.94|5647.46|308.33|0.00|1806.99|4404.81|4713.14|6211.80|6520.13|1566.87| +2452143|58049|2452223|5401|52671|974909|3194|9032|74042|1045013|146|5984|5|9|17|4|251|177|76|98.53|143.85|63.29|6122.56|4810.04|7488.28|10932.60|48.10|0.00|5356.48|4810.04|4858.14|10166.52|10214.62|-2678.24| +2452143|58049|2452256|16083|52671|974909|3194|9032|74042|1045013|146|5984|29|21|10|1|15|177|30|56.38|84.00|57.96|781.20|1738.80|1691.40|2520.00|121.71|0.00|705.60|1738.80|1860.51|2444.40|2566.11|47.40| +2452143|58049|2452190|11165|52671|974909|3194|9032|74042|1045013|146|5984|31|9|11|2|136|177|53|1.83|4.08|1.79|121.37|94.87|96.99|216.24|0.03|92.97|45.05|1.90|1.93|46.95|46.98|-95.09| +2452143|58049|2452218|12013|52671|974909|3194|9032|74042|1045013|146|5984|1|7|17|3|36|177|25|74.49|175.79|93.16|2065.75|2329.00|1862.25|4394.75|139.74|0.00|1450.25|2329.00|2468.74|3779.25|3918.99|466.75| +2452143|58049|2452212|17661|52671|974909|3194|9032|74042|1045013|146|5984|39|1|9|5|221|177|59|9.81|18.14|0.18|1059.64|10.62|578.79|1070.26|0.74|0.00|202.96|10.62|11.36|213.58|214.32|-568.17| +2452143|58049|2452219|6979|52671|974909|3194|9032|74042|1045013|146|5984|43|29|18|1|137|177|48|25.04|73.11|29.24|2105.76|1403.52|1201.92|3509.28|28.07|0.00|1473.60|1403.52|1431.59|2877.12|2905.19|201.60| +2452143|58049|2452163|11509|52671|974909|3194|9032|74042|1045013|146|5984|11|23|18|4|90|177|30|81.33|138.26|15.20|3691.80|456.00|2439.90|4147.80|13.68|0.00|497.70|456.00|469.68|953.70|967.38|-1983.90| +2452143|58049|2452177|10085|52671|974909|3194|9032|74042|1045013|146|5984|17|23|5|5|88|177|57|9.19|22.69|7.48|866.97|426.36|523.83|1293.33|25.58|0.00|206.91|426.36|451.94|633.27|658.85|-97.47| +2452143|58049|2452151|10867|52671|974909|3194|9032|74042|1045013|146|5984|31|29|7|1|58|177|30|26.05|68.51|66.45|61.80|1993.50|781.50|2055.30|139.54|0.00|225.90|1993.50|2133.04|2219.40|2358.94|1212.00| +2452143|58049|2452195|617|52671|974909|3194|9032|74042|1045013|146|5984|13|3|7|4|264|177|44|66.34|130.02|50.70|3490.08|2230.80|2918.96|5720.88|129.60|379.23|743.60|1851.57|1981.17|2595.17|2724.77|-1067.39| +2452143|58049|2452176|2347|52671|974909|3194|9032|74042|1045013|146|5984|7|21|18|2|9|177|78|72.96|141.54|76.43|5078.58|5961.54|5690.88|11040.12|357.69|0.00|3421.86|5961.54|6319.23|9383.40|9741.09|270.66| +2451529|24438|2451614|5737|38215|1635796|4130|24111|81833|1718850|6007|33188|32|7|14|1|11|178|48|17.24|32.75|26.20|314.40|1257.60|827.52|1572.00|0.00|1232.44|581.28|25.16|25.16|606.44|606.44|-802.36| +2451529|24438|2451649|6974|38215|1635796|4130|24111|81833|1718850|6007|33188|55|1|11|4|107|178|18|60.05|95.47|12.41|1495.08|223.38|1080.90|1718.46|3.08|120.62|859.14|102.76|105.84|961.90|964.98|-978.14| +2451529|24438|2451530|12818|38215|1635796|4130|24111|81833|1718850|6007|33188|19|25|9|2|225|178|60|53.61|159.22|30.25|7738.20|1815.00|3216.60|9553.20|18.15|0.00|573.00|1815.00|1833.15|2388.00|2406.15|-1401.60| +2451529|24438|2451535|6001|38215|1635796|4130|24111|81833|1718850|6007|33188|40|14|8|5|225|178|3|35.65|70.94|9.22|185.16|27.66|106.95|212.82|1.38|0.00|95.76|27.66|29.04|123.42|124.80|-79.29| +2451529|24438|2451643|1198|38215|1635796|4130|24111|81833|1718850|6007|33188|37|13|15|1|144|178|82|2.41|5.80|2.95|233.70|241.90|197.62|475.60|12.09|0.00|75.44|241.90|253.99|317.34|329.43|44.28| +2451529|24438|2451608|4768|38215|1635796|4130|24111|81833|1718850|6007|33188|8|20|17|3|29|178|76|75.21|139.89|58.75|6166.64|4465.00|5715.96|10631.64|0.00|0.00|1700.88|4465.00|4465.00|6165.88|6165.88|-1250.96| +2451529|24438|2451542|14383|38215|1635796|4130|24111|81833|1718850|6007|33188|13|2|7|2|167|178|44|11.64|33.29|27.63|249.04|1215.72|512.16|1464.76|62.36|522.75|615.12|692.97|755.33|1308.09|1370.45|180.81| +2451529|24438|2451645|8572|38215|1635796|4130|24111|81833|1718850|6007|33188|4|4|3|3|183|178|86|83.38|87.54|45.52|3613.72|3914.72|7170.68|7528.44|117.44|0.00|2559.36|3914.72|4032.16|6474.08|6591.52|-3255.96| +2451529|24438|2451579|13922|38215|1635796|4130|24111|81833|1718850|6007|33188|10|20|13|5|287|178|79|99.20|219.23|129.34|7101.31|10217.86|7836.80|17319.17|0.00|4802.39|8486.18|5415.47|5415.47|13901.65|13901.65|-2421.33| +2451529|24438|2451612|6988|38215|1635796|4130|24111|81833|1718850|6007|33188|26|14|12|1|267|178|61|67.70|73.79|60.50|810.69|3690.50|4129.70|4501.19|73.81|0.00|1665.30|3690.50|3764.31|5355.80|5429.61|-439.20| +2451529|24438|2451587|8965|38215|1635796|4130|24111|81833|1718850|6007|33188|31|4|10|4|181|178|53|8.03|19.75|13.03|356.16|690.59|425.59|1046.75|62.15|0.00|219.42|690.59|752.74|910.01|972.16|265.00| +2451529|24438|2451547|6566|38215|1635796|4130|24111|81833|1718850|6007|33188|46|2|20|3|199|178|46|39.03|116.30|68.61|2193.74|3156.06|1795.38|5349.80|94.68|0.00|2674.90|3156.06|3250.74|5830.96|5925.64|1360.68| +2451529|24438|2451618|14269|38215|1635796|4130|24111|81833|1718850|6007|33188|50|25|20|2|64|178|32|92.36|275.23|211.92|2025.92|6781.44|2955.52|8807.36|339.07|0.00|2730.24|6781.44|7120.51|9511.68|9850.75|3825.92| +2451529|24438|2451540|236|38215|1635796|4130|24111|81833|1718850|6007|33188|40|7|7|5|283|178|39|27.68|74.45|9.67|2526.42|377.13|1079.52|2903.55|7.54|0.00|1364.61|377.13|384.67|1741.74|1749.28|-702.39| +2451529|24438|2451578|9421|38215|1635796|4130|24111|81833|1718850|6007|33188|22|4|14|3|149|178|41|55.00|87.45|42.85|1828.60|1756.85|2255.00|3585.45|52.70|0.00|322.67|1756.85|1809.55|2079.52|2132.22|-498.15| +2451889|82888|2451971|8786|24643|451741|5686|28384|67918|870723|4925|38823|13|23|20|5|290|179|60|66.14|87.30|31.42|3352.80|1885.20|3968.40|5238.00|113.11|0.00|942.60|1885.20|1998.31|2827.80|2940.91|-2083.20| +2451889|82888|2451967|8041|24643|451741|5686|28384|67918|870723|4925|38823|53|29|6|4|272|179|48|37.70|101.41|91.26|487.20|4380.48|1809.60|4867.68|43.80|0.00|2336.16|4380.48|4424.28|6716.64|6760.44|2570.88| +2451889|82888|2451963|2894|24643|451741|5686|28384|67918|870723|4925|38823|49|5|18|1|244|179|88|24.69|58.26|1.16|5024.80|102.08|2172.72|5126.88|6.12|0.00|0.00|102.08|108.20|102.08|108.20|-2070.64| +2451889|82888|2451958|12158|24643|451741|5686|28384|67918|870723|4925|38823|1|8|6|5|61|179|63|71.66|168.40|15.15|9654.75|954.45|4514.58|10609.20|38.17|0.00|2121.84|954.45|992.62|3076.29|3114.46|-3560.13| +2451889|82888|2451937|16754|24643|451741|5686|28384|67918|870723|4925|38823|31|25|7|4|161|179|41|40.30|83.01|58.10|1021.31|2382.10|1652.30|3403.41|47.64|0.00|1293.14|2382.10|2429.74|3675.24|3722.88|729.80| +2451889|82888|2451937|14227|24643|451741|5686|28384|67918|870723|4925|38823|25|19|10|2|107|179|22|44.14|56.94|30.17|588.94|663.74|971.08|1252.68|6.63|0.00|200.42|663.74|670.37|864.16|870.79|-307.34| +2451889|82888|2451950|3043|24643|451741|5686|28384|67918|870723|4925|38823|38|5|9|2|95|179|28|59.77|87.86|43.93|1230.04|1230.04|1673.56|2460.08|12.30|0.00|959.28|1230.04|1242.34|2189.32|2201.62|-443.52| +2451889|82888|2451920|2078|24643|451741|5686|28384|67918|870723|4925|38823|14|2|6|5|224|179|94|90.25|246.38|229.13|1621.50|21538.22|8483.50|23159.72|1076.91|0.00|4167.96|21538.22|22615.13|25706.18|26783.09|13054.72| +2451889|82888|2451909|4274|24643|451741|5686|28384|67918|870723|4925|38823|43|17|13|5|44|179|78|19.04|32.93|4.28|2234.70|333.84|1485.12|2568.54|3.33|0.00|308.10|333.84|337.17|641.94|645.27|-1151.28| +2451889|82888|2451987|14438|24643|451741|5686|28384|67918|870723|4925|38823|55|5|18|3|71|179|22|88.62|108.11|87.56|452.10|1926.32|1949.64|2378.42|52.01|1059.47|880.00|866.85|918.86|1746.85|1798.86|-1082.79| +2451889|82888|2451904|3047|24643|451741|5686|28384|67918|870723|4925|38823|31|8|4|3|279|179|78|53.79|54.86|15.36|3081.00|1198.08|4195.62|4279.08|71.88|0.00|1198.08|1198.08|1269.96|2396.16|2468.04|-2997.54| +2451889|82888|2451914|1933|24643|451741|5686|28384|67918|870723|4925|38823|32|19|13|4|252|179|39|81.15|159.05|11.13|5768.88|434.07|3164.85|6202.95|2.86|338.57|2729.22|95.50|98.36|2824.72|2827.58|-3069.35| +2451889|82888|2452008|15937|24643|451741|5686|28384|67918|870723|4925|38823|25|1|9|4|122|179|17|84.09|234.61|197.07|638.18|3350.19|1429.53|3988.37|134.00|0.00|1196.46|3350.19|3484.19|4546.65|4680.65|1920.66| +2451889|82888|2451968|11521|24643|451741|5686|28384|67918|870723|4925|38823|25|2|1|2|186|179|72|81.94|141.75|83.63|4184.64|6021.36|5899.68|10206.00|361.28|0.00|2857.68|6021.36|6382.64|8879.04|9240.32|121.68| +2451773|66769|2451847|3167|52926|72892|6105|34471|44699|988911|369|31982|43|20|17|1|45|180|58|21.00|34.23|13.34|1211.62|773.72|1218.00|1985.34|5.18|255.32|59.16|518.40|523.58|577.56|582.74|-699.60| +2451773|66769|2451778|12691|52926|72892|6105|34471|44699|988911|369|31982|41|5|15|5|182|180|31|85.39|189.56|24.64|5112.52|763.84|2647.09|5876.36|30.55|0.00|1762.66|763.84|794.39|2526.50|2557.05|-1883.25| +2451773|66769|2451850|12721|52926|72892|6105|34471|44699|988911|369|31982|1|7|7|5|133|180|100|94.18|178.00|17.80|16020.00|1780.00|9418.00|17800.00|0.00|0.00|2848.00|1780.00|1780.00|4628.00|4628.00|-7638.00| +2451773|66769|2451805|6902|52926|72892|6105|34471|44699|988911|369|31982|55|19|19|3|240|180|97|82.72|90.99|36.39|5296.20|3529.83|8023.84|8826.03|176.49|0.00|2470.59|3529.83|3706.32|6000.42|6176.91|-4494.01| +2451773|66769|2451824|4748|52926|72892|6105|34471|44699|988911|369|31982|5|29|4|1|185|180|14|12.19|22.79|8.43|201.04|118.02|170.66|319.06|3.54|0.00|108.36|118.02|121.56|226.38|229.92|-52.64| +2451773|66769|2451786|16597|52926|72892|6105|34471|44699|988911|369|31982|44|11|9|2|45|180|90|34.35|88.62|39.87|4387.50|3588.30|3091.50|7975.80|287.06|0.00|3827.70|3588.30|3875.36|7416.00|7703.06|496.80| +2451773|66769|2451868|6320|52926|72892|6105|34471|44699|988911|369|31982|13|19|3|4|132|180|37|39.11|71.96|19.42|1943.98|718.54|1447.07|2662.52|0.00|0.00|851.74|718.54|718.54|1570.28|1570.28|-728.53| +2451773|66769|2451790|16663|52926|72892|6105|34471|44699|988911|369|31982|55|7|13|2|91|180|88|26.87|74.69|22.40|4601.52|1971.20|2364.56|6572.72|157.69|0.00|131.12|1971.20|2128.89|2102.32|2260.01|-393.36| +2451773|66769|2451792|14357|52926|72892|6105|34471|44699|988911|369|31982|38|11|12|3|61|180|70|48.59|71.91|12.94|4127.90|905.80|3401.30|5033.70|36.23|0.00|1057.00|905.80|942.03|1962.80|1999.03|-2495.50| +2451773|66769|2451855|8885|52926|72892|6105|34471|44699|988911|369|31982|2|23|2|2|102|180|48|92.96|189.63|170.66|910.56|8191.68|4462.08|9102.24|81.91|0.00|2821.44|8191.68|8273.59|11013.12|11095.03|3729.60| +2451773|66769|2451816|15559|52926|72892|6105|34471|44699|988911|369|31982|56|26|9|2|112|180|98|86.24|187.14|136.61|4951.94|13387.78|8451.52|18339.72|1204.90|0.00|183.26|13387.78|14592.68|13571.04|14775.94|4936.26| +2451773|66769|2451774|6746|52926|72892|6105|34471|44699|988911|369|31982|41|8|9|3|107|180|80|97.79|239.58|184.47|4408.80|14757.60|7823.20|19166.40|885.45|0.00|7283.20|14757.60|15643.05|22040.80|22926.25|6934.40| +2451773|66769|2451817|12655|52926|72892|6105|34471|44699|988911|369|31982|59|5|19|2|215|180|36|32.20|42.18|20.66|774.72|743.76|1159.20|1518.48|14.87|0.00|333.72|743.76|758.63|1077.48|1092.35|-415.44| +2451773|66769|2451864|16424|52926|72892|6105|34471|44699|988911|369|31982|41|8|19|3|257|180|100|33.00|94.38|94.38|0.00|9438.00|3300.00|9438.00|86.82|755.04|377.00|8682.96|8769.78|9059.96|9146.78|5382.96| +2451773|66769|2451801|10958|52926|72892|6105|34471|44699|988911|369|31982|5|11|6|4|142|180|3|60.89|110.21|25.34|254.61|76.02|182.67|330.63|5.32|0.00|162.00|76.02|81.34|238.02|243.34|-106.65| +2451664|6946|2451757|11327|99634|1377537|4240|38143|21783|1808315|4652|6155|37|1|3|5|97|181|1|31.83|87.53|38.51|49.02|38.51|31.83|87.53|3.46|0.00|29.76|38.51|41.97|68.27|71.73|6.68| +2451664|6946|2451714|13997|99634|1377537|4240|38143|21783|1808315|4652|6155|17|25|17|1|138|181|47|81.49|209.42|8.37|9449.35|393.39|3830.03|9842.74|3.93|0.00|4133.65|393.39|397.32|4527.04|4530.97|-3436.64| +2451664|6946|2451687|14767|99634|1377537|4240|38143|21783|1808315|4652|6155|20|13|10|1|129|181|16|40.55|93.67|83.36|164.96|1333.76|648.80|1498.72|40.01|0.00|149.76|1333.76|1373.77|1483.52|1523.53|684.96| +2451664|6946|2451693|11021|99634|1377537|4240|38143|21783|1808315|4652|6155|14|20|17|5|36|181|47|78.85|80.42|73.98|302.68|3477.06|3705.95|3779.74|284.77|312.93|1057.97|3164.13|3448.90|4222.10|4506.87|-541.82| +2451664|6946|2451702|7298|99634|1377537|4240|38143|21783|1808315|4652|6155|47|5|12|1|60|181|17|93.48|150.50|79.76|1202.58|1355.92|1589.16|2558.50|6.50|1138.97|639.54|216.95|223.45|856.49|862.99|-1372.21| +2451664|6946|2451676|16376|99634|1377537|4240|38143|21783|1808315|4652|6155|37|19|11|2|133|181|82|66.14|74.73|56.79|1471.08|4656.78|5423.48|6127.86|232.83|0.00|2389.48|4656.78|4889.61|7046.26|7279.09|-766.70| +2451664|6946|2451730|10309|99634|1377537|4240|38143|21783|1808315|4652|6155|49|26|13|5|184|181|92|58.95|147.96|29.59|10890.04|2722.28|5423.40|13612.32|217.78|0.00|679.88|2722.28|2940.06|3402.16|3619.94|-2701.12| +2451664|6946|2451753|7502|99634|1377537|4240|38143|21783|1808315|4652|6155|17|1|13|5|217|181|70|6.58|18.29|9.69|602.00|678.30|460.60|1280.30|54.26|0.00|371.00|678.30|732.56|1049.30|1103.56|217.70| +2452608|72040|2452620|14707|70045|1157892|6092|2739|14414|1365778|1386|7233|3|7|16|3|70|182|2|33.72|37.42|22.07|30.70|44.14|67.44|74.84|1.32|0.00|20.94|44.14|45.46|65.08|66.40|-23.30| +2452608|72040|2452612|6102|70045|1157892|6092|2739|14414|1365778|1386|7233|18|18|6|1|233|182|58|72.19|131.38|48.61|4800.66|2819.38|4187.02|7620.04|84.58|0.00|3580.92|2819.38|2903.96|6400.30|6484.88|-1367.64| +2452608|72040|2452684|15379|70045|1157892|6092|2739|14414|1365778|1386|7233|30|3|19|1|256|182|41|95.35|101.07|42.44|2403.83|1740.04|3909.35|4143.87|87.00|0.00|1035.66|1740.04|1827.04|2775.70|2862.70|-2169.31| +2452608|72040|2452618|8287|70045|1157892|6092|2739|14414|1365778|1386|7233|27|15|15|2|36|182|29|1.06|1.07|0.56|14.79|16.24|30.74|31.03|0.64|0.00|3.19|16.24|16.88|19.43|20.07|-14.50| +2452608|72040|2452668|10554|70045|1157892|6092|2739|14414|1365778|1386|7233|55|1|11|4|47|182|45|71.22|94.01|74.26|888.75|3341.70|3204.90|4230.45|200.50|0.00|719.10|3341.70|3542.20|4060.80|4261.30|136.80| +2452608|72040|2452662|17991|70045|1157892|6092|2739|14414|1365778|1386|7233|24|27|6|5|207|182|65|73.00|147.46|81.10|4313.40|5271.50|4745.00|9584.90|369.00|0.00|95.55|5271.50|5640.50|5367.05|5736.05|526.50| +2452608|72040|2452682|11070|70045|1157892|6092|2739|14414|1365778|1386|7233|51|6|3|5|227|182|32|65.27|176.88|164.49|396.48|5263.68|2088.64|5660.16|210.54|0.00|2433.60|5263.68|5474.22|7697.28|7907.82|3175.04| +2452608|72040|2452656|2359|70045|1157892|6092|2739|14414|1365778|1386|7233|24|25|1|2|113|182|2|61.82|152.69|62.60|180.18|125.20|123.64|305.38|1.25|0.00|73.28|125.20|126.45|198.48|199.73|1.56| +2452608|72040|2452629|5839|70045|1157892|6092|2739|14414|1365778|1386|7233|7|6|2|1|210|182|79|40.75|93.31|49.45|3464.94|3906.55|3219.25|7371.49|351.58|0.00|3464.15|3906.55|4258.13|7370.70|7722.28|687.30| +2452608|72040|2452681|15522|70045|1157892|6092|2739|14414|1365778|1386|7233|37|21|20|4|218|182|51|70.24|137.67|126.65|562.02|6459.15|3582.24|7021.17|322.95|0.00|491.13|6459.15|6782.10|6950.28|7273.23|2876.91| +2452608|72040|2452713|13735|70045|1157892|6092|2739|14414|1365778|1386|7233|9|13|9|1|13|182|60|9.79|15.85|0.00|951.00|0.00|587.40|951.00|0.00|0.00|228.00|0.00|0.00|228.00|228.00|-587.40| +2452608|72040|2452660|7956|70045|1157892|6092|2739|14414|1365778|1386|7233|19|19|3|1|56|182|46|36.85|41.64|4.16|1724.08|191.36|1695.10|1915.44|0.00|0.00|632.04|191.36|191.36|823.40|823.40|-1503.74| +2452608|72040|2452609|15426|70045|1157892|6092|2739|14414|1365778|1386|7233|55|18|20|1|72|182|64|28.33|33.99|15.97|1153.28|1022.08|1813.12|2175.36|61.32|0.00|238.72|1022.08|1083.40|1260.80|1322.12|-791.04| +2452608|72040|2452616|3235|70045|1157892|6092|2739|14414|1365778|1386|7233|42|27|17|5|53|182|65|82.24|241.78|157.15|5500.95|10214.75|5345.60|15715.70|204.29|0.00|942.50|10214.75|10419.04|11157.25|11361.54|4869.15| +2452608|72040|2452647|16711|70045|1157892|6092|2739|14414|1365778|1386|7233|21|9|19|1|160|182|11|58.50|139.23|73.79|719.84|811.69|643.50|1531.53|4.38|592.53|551.32|219.16|223.54|770.48|774.86|-424.34| +2452608|72040|2452643|13173|70045|1157892|6092|2739|14414|1365778|1386|7233|18|15|17|1|155|182|67|82.26|239.37|4.78|15717.53|320.26|5511.42|16037.79|16.01|0.00|0.00|320.26|336.27|320.26|336.27|-5191.16| +2450870|71040|2450899|15103|97407|1767251|4742|12186|14612|195728|6165|4009|55|20|1|1|4|183|2|41.97|62.53|2.50|120.06|5.00|83.94|125.06|0.31|1.10|1.24|3.90|4.21|5.14|5.45|-80.04| +2450870|71040|2450972|13972|97407|1767251|4742|12186|14612|195728|6165|4009|34|2|16|3|7|183|39|73.70|140.03|37.80|3986.97|1474.20|2874.30|5461.17|34.49|324.32|1310.40|1149.88|1184.37|2460.28|2494.77|-1724.42| +2450870|71040|2450925|8410|97407|1767251|4742|12186|14612|195728|6165|4009|26|20|15|4|106|183|11|61.08|109.33|24.05|938.08|264.55|671.88|1202.63|23.80|0.00|204.38|264.55|288.35|468.93|492.73|-407.33| +2450870|71040|2450920|12344|97407|1767251|4742|12186|14612|195728|6165|4009|19|28|10|5|120|183|28|73.22|114.22|2.28|3134.32|63.84|2050.16|3198.16|1.27|0.00|1599.08|63.84|65.11|1662.92|1664.19|-1986.32| +2450870|71040|2450956|17240|97407|1767251|4742|12186|14612|195728|6165|4009|28|1|10|1|102|183|93|31.28|61.93|33.44|2649.57|3109.92|2909.04|5759.49|31.09|0.00|0.00|3109.92|3141.01|3109.92|3141.01|200.88| +2450870|71040|2450928|7996|97407|1767251|4742|12186|14612|195728|6165|4009|49|10|17|3|39|183|91|33.74|42.51|35.28|657.93|3210.48|3070.34|3868.41|64.20|0.00|1701.70|3210.48|3274.68|4912.18|4976.38|140.14| +2450870|71040|2450958|10177|97407|1767251|4742|12186|14612|195728|6165|4009|58|28|13|3|23|183|59|64.12|117.33|35.19|4846.26|2076.21|3783.08|6922.47|41.52|0.00|622.45|2076.21|2117.73|2698.66|2740.18|-1706.87| +2450870|71040|2450981|17740|97407|1767251|4742|12186|14612|195728|6165|4009|19|1|19|2|48|183|55|40.77|76.64|0.76|4173.40|41.80|2242.35|4215.20|1.67|0.00|2065.25|41.80|43.47|2107.05|2108.72|-2200.55| +2450870|71040|2450958|15062|97407|1767251|4742|12186|14612|195728|6165|4009|56|22|5|3|58|183|55|2.89|6.12|4.46|91.30|245.30|158.95|336.60|4.70|9.81|164.45|235.49|240.19|399.94|404.64|76.54| +2452219|80200|2452248|14761|80155|881323|2844|6412|20040|1292540|2269|2637|3|29|18|1|114|184|24|93.76|114.38|72.05|1015.92|1729.20|2250.24|2745.12|17.29|0.00|988.08|1729.20|1746.49|2717.28|2734.57|-521.04| +2452219|80200|2452335|11693|80155|881323|2844|6412|20040|1292540|2269|2637|41|15|2|4|73|184|24|45.61|134.09|52.29|1963.20|1254.96|1094.64|3218.16|87.84|0.00|257.28|1254.96|1342.80|1512.24|1600.08|160.32| +2452219|80200|2452313|7893|80155|881323|2844|6412|20040|1292540|2269|2637|53|5|19|5|51|184|46|2.71|6.61|6.34|12.42|291.64|124.66|304.06|0.00|160.40|112.24|131.24|131.24|243.48|243.48|6.58| +2452219|80200|2452296|6423|80155|881323|2844|6412|20040|1292540|2269|2637|47|5|12|1|210|184|80|19.65|46.96|15.96|2480.00|1276.80|1572.00|3756.80|12.76|0.00|976.00|1276.80|1289.56|2252.80|2265.56|-295.20| +2452219|80200|2452336|8451|80155|881323|2844|6412|20040|1292540|2269|2637|19|3|11|5|89|184|63|97.62|184.50|166.05|1162.35|10461.15|6150.06|11623.50|36.61|9728.86|2092.23|732.29|768.90|2824.52|2861.13|-5417.77| +2452219|80200|2452227|13575|80155|881323|2844|6412|20040|1292540|2269|2637|47|17|14|4|84|184|56|62.16|126.80|65.93|3408.72|3692.08|3480.96|7100.80|0.00|0.00|2840.32|3692.08|3692.08|6532.40|6532.40|211.12| +2452219|80200|2452328|6921|80155|881323|2844|6412|20040|1292540|2269|2637|27|1|15|2|149|184|78|86.35|169.24|13.53|12145.38|1055.34|6735.30|13200.72|36.09|654.31|5016.18|401.03|437.12|5417.21|5453.30|-6334.27| +2452219|80200|2452253|10319|80155|881323|2844|6412|20040|1292540|2269|2637|11|5|10|2|47|184|52|43.35|48.11|18.76|1526.20|975.52|2254.20|2501.72|71.99|175.59|725.40|799.93|871.92|1525.33|1597.32|-1454.27| +2452219|80200|2452282|2861|80155|881323|2844|6412|20040|1292540|2269|2637|19|13|14|1|95|184|58|38.33|112.69|112.69|0.00|6536.02|2223.14|6536.02|444.44|980.40|914.66|5555.62|6000.06|6470.28|6914.72|3332.48| +2452219|80200|2452317|11319|80155|881323|2844|6412|20040|1292540|2269|2637|25|21|19|5|148|184|15|89.29|262.51|26.25|3543.90|393.75|1339.35|3937.65|27.56|0.00|1614.30|393.75|421.31|2008.05|2035.61|-945.60| +2452219|80200|2452290|4465|80155|881323|2844|6412|20040|1292540|2269|2637|25|11|16|4|269|184|60|45.08|123.51|60.51|3780.00|3630.60|2704.80|7410.60|124.53|1851.60|1778.40|1779.00|1903.53|3557.40|3681.93|-925.80| +2452219|80200|2452267|545|80155|881323|2844|6412|20040|1292540|2269|2637|29|21|7|3|299|184|5|19.34|33.65|1.00|163.25|5.00|96.70|168.25|0.30|0.00|42.05|5.00|5.30|47.05|47.35|-91.70| +2452219|80200|2452327|17505|80155|881323|2844|6412|20040|1292540|2269|2637|45|17|18|4|103|184|59|95.91|233.06|93.22|8250.56|5499.98|5658.69|13750.54|73.69|1814.99|3849.75|3684.99|3758.68|7534.74|7608.43|-1973.70| +2452219|80200|2452238|1745|80155|881323|2844|6412|20040|1292540|2269|2637|33|11|18|5|109|184|60|82.04|215.76|159.66|3366.00|9579.60|4922.40|12945.60|862.16|0.00|5695.80|9579.60|10441.76|15275.40|16137.56|4657.20| +2452222|52410|2452341|5115|11951|1179323|4648|29220|4308|1492798|6936|13461|25|13|1|5|109|185|53|95.41|121.17|55.73|3468.32|2953.69|5056.73|6422.01|29.53|0.00|898.88|2953.69|2983.22|3852.57|3882.10|-2103.04| +2452222|52410|2452275|11375|11951|1179323|4648|29220|4308|1492798|6936|13461|51|23|18|4|300|185|88|16.25|39.00|8.97|2642.64|789.36|1430.00|3432.00|31.57|0.00|617.76|789.36|820.93|1407.12|1438.69|-640.64| +2452222|52410|2452264|14285|11951|1179323|4648|29220|4308|1492798|6936|13461|25|29|10|5|67|185|97|19.58|54.43|31.56|2218.39|3061.32|1899.26|5279.71|30.61|0.00|949.63|3061.32|3091.93|4010.95|4041.56|1162.06| +2452222|52410|2452321|7711|11951|1179323|4648|29220|4308|1492798|6936|13461|5|13|8|2|56|185|73|5.62|9.44|8.49|69.35|619.77|410.26|689.12|18.59|0.00|123.37|619.77|638.36|743.14|761.73|209.51| +2452222|52410|2452234|8975|11951|1179323|4648|29220|4308|1492798|6936|13461|29|19|5|5|78|185|78|8.82|22.66|9.29|1042.86|724.62|687.96|1767.48|57.96|0.00|17.16|724.62|782.58|741.78|799.74|36.66| +2452222|52410|2452278|13371|11951|1179323|4648|29220|4308|1492798|6936|13461|47|25|15|4|86|185|19|9.11|12.93|6.59|120.46|125.21|173.09|245.67|5.00|0.00|103.17|125.21|130.21|228.38|233.38|-47.88| +2452222|52410|2452316|13901|11951|1179323|4648|29220|4308|1492798|6936|13461|5|27|5|1|197|185|16|85.21|225.80|36.12|3034.88|577.92|1363.36|3612.80|52.01|0.00|144.48|577.92|629.93|722.40|774.41|-785.44| +2452222|52410|2452224|2015|11951|1179323|4648|29220|4308|1492798|6936|13461|29|3|19|1|136|185|16|42.48|42.48|4.67|604.96|74.72|679.68|679.68|4.48|0.00|190.24|74.72|79.20|264.96|269.44|-604.96| +2452222|52410|2452322|16211|11951|1179323|4648|29220|4308|1492798|6936|13461|39|17|8|2|186|185|83|36.33|90.46|48.84|3454.46|4053.72|3015.39|7508.18|364.83|0.00|2177.09|4053.72|4418.55|6230.81|6595.64|1038.33| +2452222|52410|2452273|16701|11951|1179323|4648|29220|4308|1492798|6936|13461|35|9|2|4|206|185|47|66.53|134.39|102.13|1516.22|4800.11|3126.91|6316.33|44.16|2592.05|3157.93|2208.06|2252.22|5365.99|5410.15|-918.85| +2451813|67130|2451887|3659|96566|1730134|2902|27270|94555|771491|5974|1542|19|8|13|3|163|186|89|47.77|116.08|35.98|7128.90|3202.22|4251.53|10331.12|96.06|0.00|2995.74|3202.22|3298.28|6197.96|6294.02|-1049.31| +2451813|67130|2451847|14579|96566|1730134|2902|27270|94555|771491|5974|1542|19|14|13|3|233|186|25|52.82|113.03|29.38|2091.25|734.50|1320.50|2825.75|14.69|0.00|56.50|734.50|749.19|791.00|805.69|-586.00| +2451813|67130|2451820|12671|96566|1730134|2902|27270|94555|771491|5974|1542|11|20|6|4|289|186|33|37.91|39.42|7.09|1066.89|233.97|1251.03|1300.86|11.69|0.00|624.36|233.97|245.66|858.33|870.02|-1017.06| +2451813|67130|2451923|7387|96566|1730134|2902|27270|94555|771491|5974|1542|44|11|12|5|68|186|89|37.46|60.68|40.65|1782.67|3617.85|3333.94|5400.52|0.00|2677.20|1187.26|940.65|940.65|2127.91|2127.91|-2393.29| +2451813|67130|2451918|12788|96566|1730134|2902|27270|94555|771491|5974|1542|25|26|8|4|153|186|5|42.47|85.36|47.80|187.80|239.00|212.35|426.80|4.78|0.00|204.85|239.00|243.78|443.85|448.63|26.65| +2451813|67130|2451860|11588|96566|1730134|2902|27270|94555|771491|5974|1542|43|25|14|2|113|186|92|80.92|193.39|7.73|17080.72|711.16|7444.64|17791.88|42.66|0.00|5336.92|711.16|753.82|6048.08|6090.74|-6733.48| +2451813|67130|2451881|15764|96566|1730134|2902|27270|94555|771491|5974|1542|50|13|15|1|264|186|2|30.17|70.59|0.00|141.18|0.00|60.34|141.18|0.00|0.00|2.82|0.00|0.00|2.82|2.82|-60.34| +2451813|67130|2451829|14378|96566|1730134|2902|27270|94555|771491|5974|1542|38|19|10|5|257|186|49|20.53|48.45|6.29|2065.84|308.21|1005.97|2374.05|18.49|0.00|521.85|308.21|326.70|830.06|848.55|-697.76| +2451813|67130|2451837|4393|96566|1730134|2902|27270|94555|771491|5974|1542|47|5|12|1|60|186|43|80.02|92.02|51.53|1741.07|2215.79|3440.86|3956.86|0.00|0.00|949.44|2215.79|2215.79|3165.23|3165.23|-1225.07| +2451813|67130|2451901|13013|96566|1730134|2902|27270|94555|771491|5974|1542|20|1|1|1|241|186|38|12.54|16.67|5.83|411.92|221.54|476.52|633.46|17.72|0.00|44.08|221.54|239.26|265.62|283.34|-254.98| +2451813|67130|2451883|15584|96566|1730134|2902|27270|94555|771491|5974|1542|37|19|1|5|36|186|38|81.24|92.61|68.53|915.04|2604.14|3087.12|3519.18|78.12|0.00|1548.12|2604.14|2682.26|4152.26|4230.38|-482.98| +2451813|67130|2451856|16442|96566|1730134|2902|27270|94555|771491|5974|1542|14|26|20|4|205|186|39|2.27|6.76|5.07|65.91|197.73|88.53|263.64|11.86|0.00|36.66|197.73|209.59|234.39|246.25|109.20| +2451813|67130|2451880|17965|96566|1730134|2902|27270|94555|771491|5974|1542|56|25|7|4|18|186|54|85.48|177.79|42.66|7297.02|2303.64|4615.92|9600.66|161.25|0.00|864.00|2303.64|2464.89|3167.64|3328.89|-2312.28| +2451813|67130|2451920|4556|96566|1730134|2902|27270|94555|771491|5974|1542|8|7|10|4|165|186|42|29.74|56.20|23.04|1392.72|967.68|1249.08|2360.40|67.73|0.00|1085.70|967.68|1035.41|2053.38|2121.11|-281.40| +2451813|67130|2451905|3185|96566|1730134|2902|27270|94555|771491|5974|1542|19|14|16|1|129|186|76|95.60|183.55|145.00|2929.80|11020.00|7265.60|13949.80|661.20|0.00|6276.84|11020.00|11681.20|17296.84|17958.04|3754.40| +2451813|67130|2451826|8627|96566|1730134|2902|27270|94555|771491|5974|1542|38|14|16|1|129|186|73|96.29|184.87|153.44|2294.39|11201.12|7029.17|13495.51|0.00|0.00|3913.53|11201.12|11201.12|15114.65|15114.65|4171.95| +2452272|46666|2452392|1911|45672|881166|3860|26554|5396|1740903|3083|1377|13|3|10|3|266|187|18|55.15|159.38|111.56|860.76|2008.08|992.70|2868.84|44.98|883.55|803.16|1124.53|1169.51|1927.69|1972.67|131.83| +2452272|46666|2452291|5241|45672|881166|3860|26554|5396|1740903|3083|1377|13|5|6|2|28|187|63|1.35|2.21|0.30|120.33|18.90|85.05|139.23|1.13|0.00|22.05|18.90|20.03|40.95|42.08|-66.15| +2452272|46666|2452326|7053|45672|881166|3860|26554|5396|1740903|3083|1377|49|3|6|3|300|187|11|40.84|64.93|54.54|114.29|599.94|449.24|714.23|41.99|0.00|49.94|599.94|641.93|649.88|691.87|150.70| +2452272|46666|2452360|5173|45672|881166|3860|26554|5396|1740903|3083|1377|7|11|18|5|12|187|95|40.09|43.69|43.25|41.80|4108.75|3808.55|4150.55|41.08|0.00|1244.50|4108.75|4149.83|5353.25|5394.33|300.20| +2452272|46666|2452322|12071|45672|881166|3860|26554|5396|1740903|3083|1377|21|9|19|3|294|187|66|14.61|28.34|24.37|262.02|1608.42|964.26|1870.44|14.47|1125.89|280.50|482.53|497.00|763.03|777.50|-481.73| +2452272|46666|2452275|3019|45672|881166|3860|26554|5396|1740903|3083|1377|21|13|6|2|101|187|40|32.27|55.18|23.72|1258.40|948.80|1290.80|2207.20|66.41|0.00|264.80|948.80|1015.21|1213.60|1280.01|-342.00| +2452272|46666|2452327|6461|45672|881166|3860|26554|5396|1740903|3083|1377|15|7|20|1|112|187|74|15.57|18.06|13.36|347.80|988.64|1152.18|1336.44|59.31|0.00|560.92|988.64|1047.95|1549.56|1608.87|-163.54| +2452272|46666|2452301|14239|45672|881166|3860|26554|5396|1740903|3083|1377|19|11|2|5|31|187|4|68.23|120.76|47.09|294.68|188.36|272.92|483.04|0.00|0.00|183.52|188.36|188.36|371.88|371.88|-84.56| +2452272|46666|2452385|5491|45672|881166|3860|26554|5396|1740903|3083|1377|41|29|9|5|34|187|6|13.69|15.60|12.48|18.72|74.88|82.14|93.60|2.42|34.44|28.98|40.44|42.86|69.42|71.84|-41.70| +2452272|46666|2452362|12965|45672|881166|3860|26554|5396|1740903|3083|1377|37|21|6|3|100|187|69|26.08|44.59|4.01|2800.02|276.69|1799.52|3076.71|19.36|0.00|1168.86|276.69|296.05|1445.55|1464.91|-1522.83| +2452272|46666|2452390|16573|45672|881166|3860|26554|5396|1740903|3083|1377|23|1|3|5|277|187|36|54.02|106.41|10.64|3447.72|383.04|1944.72|3830.76|30.64|0.00|1493.64|383.04|413.68|1876.68|1907.32|-1561.68| +2452272|46666|2452379|7647|45672|881166|3860|26554|5396|1740903|3083|1377|53|1|15|4|43|187|32|7.73|20.25|13.16|226.88|421.12|247.36|648.00|0.00|160.02|239.68|261.10|261.10|500.78|500.78|13.74| +2452272|46666|2452380|9267|45672|881166|3860|26554|5396|1740903|3083|1377|41|1|1|1|1|187|30|14.15|16.55|4.96|347.70|148.80|424.50|496.50|2.97|0.00|183.60|148.80|151.77|332.40|335.37|-275.70| +2452272|46666|2452303|4607|45672|881166|3860|26554|5396|1740903|3083|1377|55|17|10|1|79|187|13|55.42|69.82|51.66|236.08|671.58|720.46|907.66|60.44|0.00|27.17|671.58|732.02|698.75|759.19|-48.88| +2452266|68032|2452325|11961|75502|1700258|3414|14570|45830|191349|5638|49994|45|11|9|2|195|188|35|94.71|235.82|136.77|3466.75|4786.95|3314.85|8253.70|239.34|0.00|3466.40|4786.95|5026.29|8253.35|8492.69|1472.10| +2452266|68032|2452344|1633|75502|1700258|3414|14570|45830|191349|5638|49994|15|19|2|4|286|188|54|21.72|44.96|1.79|2331.18|96.66|1172.88|2427.84|3.36|12.56|1165.32|84.10|87.46|1249.42|1252.78|-1088.78| +2452266|68032|2452322|12011|75502|1700258|3414|14570|45830|191349|5638|49994|57|5|17|1|165|188|6|90.57|265.37|137.99|764.28|827.94|543.42|1592.22|57.95|0.00|748.32|827.94|885.89|1576.26|1634.21|284.52| +2452266|68032|2452279|5301|75502|1700258|3414|14570|45830|191349|5638|49994|35|1|7|5|206|188|37|63.32|155.76|4.67|5590.33|172.79|2342.84|5763.12|3.17|93.30|2074.59|79.49|82.66|2154.08|2157.25|-2263.35| +2452266|68032|2452386|14095|75502|1700258|3414|14570|45830|191349|5638|49994|29|27|6|5|267|188|54|56.98|116.23|106.93|502.20|5774.22|3076.92|6276.42|404.19|0.00|2196.72|5774.22|6178.41|7970.94|8375.13|2697.30| +2452266|68032|2452279|55|75502|1700258|3414|14570|45830|191349|5638|49994|55|11|7|5|106|188|41|53.17|74.96|74.21|30.75|3042.61|2179.97|3073.36|191.68|912.78|30.34|2129.83|2321.51|2160.17|2351.85|-50.14| +2452266|68032|2452367|16591|75502|1700258|3414|14570|45830|191349|5638|49994|27|23|10|3|156|188|14|80.09|201.02|92.46|1519.84|1294.44|1121.26|2814.28|51.77|0.00|816.06|1294.44|1346.21|2110.50|2162.27|173.18| +2452266|68032|2452271|15441|75502|1700258|3414|14570|45830|191349|5638|49994|1|3|3|4|245|188|23|21.64|31.81|2.86|665.85|65.78|497.72|731.63|4.84|5.26|175.49|60.52|65.36|236.01|240.85|-437.20| +2452266|68032|2452374|14919|75502|1700258|3414|14570|45830|191349|5638|49994|19|23|17|4|124|188|66|94.19|198.74|123.21|4984.98|8131.86|6216.54|13116.84|104.90|4635.16|2098.14|3496.70|3601.60|5594.84|5699.74|-2719.84| +2452266|68032|2452327|3031|75502|1700258|3414|14570|45830|191349|5638|49994|45|17|2|3|101|188|65|93.69|116.17|31.36|5512.65|2038.40|6089.85|7551.05|81.53|0.00|1585.35|2038.40|2119.93|3623.75|3705.28|-4051.45| +2452266|68032|2452334|395|75502|1700258|3414|14570|45830|191349|5638|49994|41|21|1|1|131|188|86|7.57|8.40|6.88|130.72|591.68|651.02|722.40|47.39|65.08|100.62|526.60|573.99|627.22|674.61|-124.42| +2452266|68032|2452350|5413|75502|1700258|3414|14570|45830|191349|5638|49994|33|21|5|2|76|188|20|23.83|43.13|2.15|819.60|43.00|476.60|862.60|0.12|38.70|370.80|4.30|4.42|375.10|375.22|-472.30| +2452266|68032|2452306|16399|75502|1700258|3414|14570|45830|191349|5638|49994|1|27|20|1|99|188|19|13.95|16.32|3.59|241.87|68.21|265.05|310.08|4.09|0.00|18.43|68.21|72.30|86.64|90.73|-196.84| +2452266|68032|2452282|6119|75502|1700258|3414|14570|45830|191349|5638|49994|27|11|11|3|58|188|37|73.77|126.88|38.06|3286.34|1408.22|2729.49|4694.56|14.08|0.00|2206.31|1408.22|1422.30|3614.53|3628.61|-1321.27| +2452396|57826|2452400|4189|42479|1684599|5472|23992|60162|134512|5952|10872|48|3|20|3|146|189|37|20.39|36.90|34.68|82.14|1283.16|754.43|1365.30|102.65|0.00|109.15|1283.16|1385.81|1392.31|1494.96|528.73| +2452396|57826|2452461|9660|42479|1684599|5472|23992|60162|134512|5952|10872|25|18|8|4|193|189|94|14.66|39.14|32.09|662.70|3016.46|1378.04|3679.16|180.98|0.00|1066.90|3016.46|3197.44|4083.36|4264.34|1638.42| +2452396|57826|2452439|11427|42479|1684599|5472|23992|60162|134512|5952|10872|48|21|15|2|189|189|93|3.05|7.68|4.99|250.17|464.07|283.65|714.24|27.84|0.00|206.46|464.07|491.91|670.53|698.37|180.42| +2452396|57826|2452419|17310|42479|1684599|5472|23992|60162|134512|5952|10872|21|3|6|2|51|189|54|37.33|48.90|46.94|105.84|2534.76|2015.82|2640.60|50.69|0.00|1082.16|2534.76|2585.45|3616.92|3667.61|518.94| +2452396|57826|2452469|787|42479|1684599|5472|23992|60162|134512|5952|10872|12|24|18|4|30|189|42|27.95|36.61|19.03|738.36|799.26|1173.90|1537.62|7.99|0.00|276.36|799.26|807.25|1075.62|1083.61|-374.64| +2452396|57826|2452399|9645|42479|1684599|5472|23992|60162|134512|5952|10872|21|24|3|4|232|189|11|27.15|65.43|33.36|352.77|366.96|298.65|719.73|25.68|0.00|7.15|366.96|392.64|374.11|399.79|68.31| +2452396|57826|2452497|1207|42479|1684599|5472|23992|60162|134512|5952|10872|57|13|5|5|278|189|3|53.82|95.79|18.20|232.77|54.60|161.46|287.37|1.63|0.00|143.67|54.60|56.23|198.27|199.90|-106.86| +2452396|57826|2452474|15055|42479|1684599|5472|23992|60162|134512|5952|10872|1|24|3|2|248|189|33|29.50|79.65|12.74|2208.03|420.42|973.50|2628.45|16.81|0.00|262.68|420.42|437.23|683.10|699.91|-553.08| +2452396|57826|2452425|16962|42479|1684599|5472|23992|60162|134512|5952|10872|45|19|10|5|220|189|64|41.52|42.76|19.24|1505.28|1231.36|2657.28|2736.64|68.71|86.19|1285.76|1145.17|1213.88|2430.93|2499.64|-1512.11| +2452396|57826|2452400|16482|42479|1684599|5472|23992|60162|134512|5952|10872|27|21|5|4|23|189|34|77.64|107.14|64.28|1457.24|2185.52|2639.76|3642.76|196.69|0.00|1129.14|2185.52|2382.21|3314.66|3511.35|-454.24| +2452396|57826|2452409|564|42479|1684599|5472|23992|60162|134512|5952|10872|3|18|9|1|83|189|77|30.81|43.44|33.01|803.11|2541.77|2372.37|3344.88|0.00|0.00|735.35|2541.77|2541.77|3277.12|3277.12|169.40| +2452396|57826|2452486|5631|42479|1684599|5472|23992|60162|134512|5952|10872|9|18|3|5|44|189|100|41.24|68.87|60.60|827.00|6060.00|4124.00|6887.00|424.20|0.00|2892.00|6060.00|6484.20|8952.00|9376.20|1936.00| +2452396|57826|2452447|6924|42479|1684599|5472|23992|60162|134512|5952|10872|36|9|2|4|176|189|20|72.51|152.99|0.00|3059.80|0.00|1450.20|3059.80|0.00|0.00|673.00|0.00|0.00|673.00|673.00|-1450.20| +2450980|59821|2451041|8065|35259|977524|589|14854|97974|127854|1649|4314|2|22|9|4|169|190|95|33.21|49.15|5.40|4156.25|513.00|3154.95|4669.25|0.00|92.34|1307.20|420.66|420.66|1727.86|1727.86|-2734.29| +2450980|59821|2450996|9217|35259|977524|589|14854|97974|127854|1649|4314|40|16|14|2|70|190|25|63.07|66.22|10.59|1390.75|264.75|1576.75|1655.50|23.82|0.00|182.00|264.75|288.57|446.75|470.57|-1312.00| +2450980|59821|2451044|7232|35259|977524|589|14854|97974|127854|1649|4314|2|2|12|4|9|190|7|74.06|100.72|8.05|648.69|56.35|518.42|705.04|0.00|0.00|338.38|56.35|56.35|394.73|394.73|-462.07| +2450980|59821|2450999|15922|35259|977524|589|14854|97974|127854|1649|4314|56|28|11|2|33|190|69|96.16|154.81|94.43|4166.22|6515.67|6635.04|10681.89|130.31|0.00|1388.28|6515.67|6645.98|7903.95|8034.26|-119.37| +2450980|59821|2451008|5462|35259|977524|589|14854|97974|127854|1649|4314|20|25|5|2|199|190|100|72.50|174.00|104.40|6960.00|10440.00|7250.00|17400.00|208.80|0.00|4872.00|10440.00|10648.80|15312.00|15520.80|3190.00| +2450980|59821|2451035|4414|35259|977524|589|14854|97974|127854|1649|4314|32|7|4|1|191|190|13|95.98|137.25|98.82|499.59|1284.66|1247.74|1784.25|115.61|0.00|856.44|1284.66|1400.27|2141.10|2256.71|36.92| +2450980|59821|2451100|11200|35259|977524|589|14854|97974|127854|1649|4314|50|16|2|3|41|190|20|65.88|150.86|107.11|875.00|2142.20|1317.60|3017.20|107.11|0.00|1327.40|2142.20|2249.31|3469.60|3576.71|824.60| +2450980|59821|2451062|8929|35259|977524|589|14854|97974|127854|1649|4314|13|22|20|5|253|190|17|80.79|115.52|99.34|275.06|1688.78|1373.43|1963.84|84.43|0.00|333.71|1688.78|1773.21|2022.49|2106.92|315.35| +2450980|59821|2451091|17468|35259|977524|589|14854|97974|127854|1649|4314|25|28|7|2|284|190|35|43.60|69.76|49.52|708.40|1733.20|1526.00|2441.60|0.00|0.00|976.50|1733.20|1733.20|2709.70|2709.70|207.20| +2451859|32811|2451919|14501|42512|655082|452|39775|39424|206076|238|14008|17|17|12|2|32|191|93|33.87|76.54|48.98|2563.08|4555.14|3149.91|7118.22|273.30|0.00|639.84|4555.14|4828.44|5194.98|5468.28|1405.23| +2451859|32811|2451873|9734|42512|655082|452|39775|39424|206076|238|14008|2|19|18|3|101|191|77|5.30|8.74|2.62|471.24|201.74|408.10|672.98|2.01|0.00|180.95|201.74|203.75|382.69|384.70|-206.36| +2451859|32811|2451944|14441|42512|655082|452|39775|39424|206076|238|14008|20|8|12|1|280|191|89|11.66|25.18|16.87|739.59|1501.43|1037.74|2241.02|105.10|0.00|1008.37|1501.43|1606.53|2509.80|2614.90|463.69| +2451859|32811|2451867|11365|42512|655082|452|39775|39424|206076|238|14008|19|19|13|3|266|191|9|55.64|166.92|111.83|495.81|1006.47|500.76|1502.28|18.11|704.52|14.94|301.95|320.06|316.89|335.00|-198.81| +2451859|32811|2451940|10202|42512|655082|452|39775|39424|206076|238|14008|25|29|17|4|194|191|79|75.34|134.10|128.73|424.23|10169.67|5951.86|10593.90|813.57|0.00|317.58|10169.67|10983.24|10487.25|11300.82|4217.81| +2451859|32811|2451949|7247|42512|655082|452|39775|39424|206076|238|14008|56|1|6|3|180|191|13|59.58|92.34|36.93|720.33|480.09|774.54|1200.42|14.40|0.00|83.98|480.09|494.49|564.07|578.47|-294.45| +2451859|32811|2451879|7027|42512|655082|452|39775|39424|206076|238|14008|41|1|8|3|68|191|93|22.43|45.08|36.51|797.01|3395.43|2085.99|4192.44|135.81|0.00|544.98|3395.43|3531.24|3940.41|4076.22|1309.44| +2451859|32811|2451870|8210|42512|655082|452|39775|39424|206076|238|14008|13|20|5|2|226|191|70|98.86|282.73|130.05|10687.60|9103.50|6920.20|19791.10|273.10|0.00|2374.40|9103.50|9376.60|11477.90|11751.00|2183.30| +2451859|32811|2451867|12577|42512|655082|452|39775|39424|206076|238|14008|19|5|13|1|176|191|14|55.88|86.61|72.75|194.04|1018.50|782.32|1212.54|20.37|0.00|157.50|1018.50|1038.87|1176.00|1196.37|236.18| +2451859|32811|2451967|13559|42512|655082|452|39775|39424|206076|238|14008|23|25|14|5|286|191|35|44.71|81.81|80.17|57.40|2805.95|1564.85|2863.35|84.17|0.00|973.35|2805.95|2890.12|3779.30|3863.47|1241.10| +2451859|32811|2451882|3073|42512|655082|452|39775|39424|206076|238|14008|5|25|3|3|103|191|24|34.66|82.49|33.82|1168.08|811.68|831.84|1979.76|64.93|0.00|851.28|811.68|876.61|1662.96|1727.89|-20.16| +2451859|32811|2451925|16093|42512|655082|452|39775|39424|206076|238|14008|43|19|1|4|27|191|43|83.76|217.77|45.73|7397.72|1966.39|3601.68|9364.11|58.99|0.00|93.31|1966.39|2025.38|2059.70|2118.69|-1635.29| +2451859|32811|2451979|11965|42512|655082|452|39775|39424|206076|238|14008|37|2|15|1|154|191|34|78.42|133.31|38.65|3218.44|1314.10|2666.28|4532.54|0.00|486.21|1404.88|827.89|827.89|2232.77|2232.77|-1838.39| +2451859|32811|2451876|12623|42512|655082|452|39775|39424|206076|238|14008|47|23|20|3|36|191|1|38.68|61.50|54.73|6.77|54.73|38.68|61.50|1.11|36.12|14.14|18.61|19.72|32.75|33.86|-20.07| +2451859|32811|2451900|6133|42512|655082|452|39775|39424|206076|238|14008|32|25|19|5|241|191|22|71.19|133.83|24.08|2414.50|529.76|1566.18|2944.26|3.60|169.52|883.08|360.24|363.84|1243.32|1246.92|-1205.94| +2451859|32811|2451893|14665|42512|655082|452|39775|39424|206076|238|14008|38|7|5|1|60|191|77|88.83|246.05|155.01|7010.08|11935.77|6839.91|18945.85|639.75|3938.80|8714.86|7996.97|8636.72|16711.83|17351.58|1157.06| +2451131|26465|2451136|7036|73703|1688447|5144|47841|94295|584275|5979|19123|4|22|1|2|106|192|69|88.70|98.45|81.71|1155.06|5637.99|6120.30|6793.05|112.75|0.00|0.00|5637.99|5750.74|5637.99|5750.74|-482.31| +2451131|26465|2451132|1226|73703|1688447|5144|47841|94295|584275|5979|19123|56|20|20|2|271|192|5|43.55|61.84|53.18|43.30|265.90|217.75|309.20|0.00|0.00|98.90|265.90|265.90|364.80|364.80|48.15| +2451131|26465|2451217|14923|73703|1688447|5144|47841|94295|584275|5979|19123|20|4|5|4|106|192|50|14.42|25.52|6.63|944.50|331.50|721.00|1276.00|19.89|0.00|344.50|331.50|351.39|676.00|695.89|-389.50| +2451131|26465|2451171|9499|73703|1688447|5144|47841|94295|584275|5979|19123|14|28|15|5|292|192|55|53.46|134.18|61.72|3985.30|3394.60|2940.30|7379.90|237.62|0.00|1254.55|3394.60|3632.22|4649.15|4886.77|454.30| +2451131|26465|2451245|12812|73703|1688447|5144|47841|94295|584275|5979|19123|32|20|15|4|173|192|26|31.86|75.50|9.81|1707.94|255.06|828.36|1963.00|2.29|178.54|529.88|76.52|78.81|606.40|608.69|-751.84| +2451131|26465|2451196|15850|73703|1688447|5144|47841|94295|584275|5979|19123|40|10|11|4|284|192|4|62.23|77.78|29.55|192.92|118.20|248.92|311.12|7.09|0.00|155.56|118.20|125.29|273.76|280.85|-130.72| +2451131|26465|2451136|13183|73703|1688447|5144|47841|94295|584275|5979|19123|43|2|12|2|92|192|75|93.38|139.13|94.60|3339.75|7095.00|7003.50|10434.75|212.85|0.00|1251.75|7095.00|7307.85|8346.75|8559.60|91.50| +2451131|26465|2451247|9577|73703|1688447|5144|47841|94295|584275|5979|19123|55|28|9|3|50|192|24|56.93|143.46|83.20|1446.24|1996.80|1366.32|3443.04|79.87|0.00|1480.32|1996.80|2076.67|3477.12|3556.99|630.48| +2451131|26465|2451211|13780|73703|1688447|5144|47841|94295|584275|5979|19123|40|26|20|3|249|192|68|27.95|64.56|45.19|1317.16|3072.92|1900.60|4390.08|215.10|0.00|745.96|3072.92|3288.02|3818.88|4033.98|1172.32| +2451876|57688|2451921|4475|29986|39334|1835|13536|40876|732283|3486|14034|32|26|1|1|178|193|38|34.36|71.46|52.88|706.04|2009.44|1305.68|2715.48|45.01|1446.79|1303.40|562.65|607.66|1866.05|1911.06|-743.03| +2451876|57688|2451958|15173|29986|39334|1835|13536|40876|732283|3486|14034|2|11|10|4|258|193|65|11.98|14.49|11.59|188.50|753.35|778.70|941.85|52.73|0.00|404.95|753.35|806.08|1158.30|1211.03|-25.35| +2451876|57688|2451902|9188|29986|39334|1835|13536|40876|732283|3486|14034|17|19|8|1|34|193|100|10.42|23.13|12.25|1088.00|1225.00|1042.00|2313.00|85.75|0.00|69.00|1225.00|1310.75|1294.00|1379.75|183.00| +2451876|57688|2451896|860|29986|39334|1835|13536|40876|732283|3486|14034|17|1|16|3|60|193|53|60.18|68.60|60.36|436.72|3199.08|3189.54|3635.80|255.92|0.00|1490.36|3199.08|3455.00|4689.44|4945.36|9.54| +2451876|57688|2451923|15913|29986|39334|1835|13536|40876|732283|3486|14034|38|5|9|2|95|193|49|19.22|42.86|30.00|630.14|1470.00|941.78|2100.14|88.20|0.00|713.93|1470.00|1558.20|2183.93|2272.13|528.22| +2451876|57688|2451905|14075|29986|39334|1835|13536|40876|732283|3486|14034|20|8|6|5|181|193|7|9.76|23.22|8.59|102.41|60.13|68.32|162.54|4.81|0.00|21.07|60.13|64.94|81.20|86.01|-8.19| +2451876|57688|2451911|17051|29986|39334|1835|13536|40876|732283|3486|14034|50|20|16|1|213|193|21|6.18|11.24|3.82|155.82|80.22|129.78|236.04|1.92|32.08|11.76|48.14|50.06|59.90|61.82|-81.64| +2451876|57688|2451913|7970|29986|39334|1835|13536|40876|732283|3486|14034|19|8|10|5|290|193|42|1.86|5.22|2.66|107.52|111.72|78.12|219.24|0.00|0.00|19.32|111.72|111.72|131.04|131.04|33.60| +2451876|57688|2451963|15998|29986|39334|1835|13536|40876|732283|3486|14034|50|29|11|2|146|193|88|35.08|53.32|0.53|4645.52|46.64|3087.04|4692.16|3.73|0.00|374.88|46.64|50.37|421.52|425.25|-3040.40| +2451876|57688|2451939|4357|29986|39334|1835|13536|40876|732283|3486|14034|53|14|12|1|20|193|97|21.76|57.22|16.02|3996.40|1553.94|2110.72|5550.34|108.77|0.00|1442.39|1553.94|1662.71|2996.33|3105.10|-556.78| +2451876|57688|2451930|9655|29986|39334|1835|13536|40876|732283|3486|14034|43|20|13|2|144|193|68|5.50|12.76|10.08|182.24|685.44|374.00|867.68|27.41|0.00|363.80|685.44|712.85|1049.24|1076.65|311.44| +2451876|57688|2451920|581|29986|39334|1835|13536|40876|732283|3486|14034|35|20|1|4|207|193|18|47.71|113.07|20.35|1668.96|366.30|858.78|2035.26|18.31|0.00|569.70|366.30|384.61|936.00|954.31|-492.48| +2451876|57688|2451943|5075|29986|39334|1835|13536|40876|732283|3486|14034|8|17|6|3|7|193|15|54.54|159.25|44.59|1719.90|668.85|818.10|2388.75|15.65|407.99|143.25|260.86|276.51|404.11|419.76|-557.24| +2451876|57688|2451932|2153|29986|39334|1835|13536|40876|732283|3486|14034|37|13|5|3|181|193|20|26.37|78.58|20.43|1163.00|408.60|527.40|1571.60|32.68|0.00|141.40|408.60|441.28|550.00|582.68|-118.80| +2451876|57688|2451940|89|29986|39334|1835|13536|40876|732283|3486|14034|43|19|11|4|80|193|33|47.65|121.50|52.24|2285.58|1723.92|1572.45|4009.50|0.00|0.00|440.88|1723.92|1723.92|2164.80|2164.80|151.47| +2451876|57688|2451897|14777|29986|39334|1835|13536|40876|732283|3486|14034|20|1|1|1|91|193|18|74.35|106.32|34.02|1301.40|612.36|1338.30|1913.76|0.00|0.00|516.60|612.36|612.36|1128.96|1128.96|-725.94| +2451054|29989|2451169|5198|31798|460071|3097|4970|31798|460071|3097|4970|49|7|3|2|1|194|54|47.85|48.80|12.68|1950.48|684.72|2583.90|2635.20|61.62|0.00|764.10|684.72|746.34|1448.82|1510.44|-1899.18| +2451054|29989|2451158|14497|31798|460071|3097|4970|31798|460071|3097|4970|31|4|17|4|278|194|51|13.57|35.96|32.36|183.60|1650.36|692.07|1833.96|36.14|445.59|476.34|1204.77|1240.91|1681.11|1717.25|512.70| +2451054|29989|2451099|6770|31798|460071|3097|4970|31798|460071|3097|4970|16|8|14|3|62|194|12|98.17|221.86|137.55|1011.72|1650.60|1178.04|2662.32|148.55|0.00|106.44|1650.60|1799.15|1757.04|1905.59|472.56| +2451054|29989|2451153|3622|31798|460071|3097|4970|31798|460071|3097|4970|20|26|20|3|106|194|58|96.51|159.24|73.25|4987.42|4248.50|5597.58|9235.92|8.49|3398.80|1662.28|849.70|858.19|2511.98|2520.47|-4747.88| +2451054|29989|2451074|14644|31798|460071|3097|4970|31798|460071|3097|4970|49|7|10|4|165|194|18|67.44|78.23|0.00|1408.14|0.00|1213.92|1408.14|0.00|0.00|633.60|0.00|0.00|633.60|633.60|-1213.92| +2451054|29989|2451170|12698|31798|460071|3097|4970|31798|460071|3097|4970|14|22|16|3|287|194|21|57.71|90.02|55.81|718.41|1172.01|1211.91|1890.42|46.88|0.00|604.80|1172.01|1218.89|1776.81|1823.69|-39.90| +2451054|29989|2451093|554|31798|460071|3097|4970|31798|460071|3097|4970|43|10|20|4|78|194|23|84.14|157.34|14.16|3293.14|325.68|1935.22|3618.82|6.51|0.00|0.00|325.68|332.19|325.68|332.19|-1609.54| +2451054|29989|2451103|12094|31798|460071|3097|4970|31798|460071|3097|4970|13|4|9|5|88|194|7|4.90|8.62|5.08|24.78|35.56|34.30|60.34|2.48|0.00|0.56|35.56|38.04|36.12|38.60|1.26| +2451054|29989|2451090|8524|31798|460071|3097|4970|31798|460071|3097|4970|50|7|12|5|178|194|77|51.10|67.45|66.77|52.36|5141.29|3934.70|5193.65|257.06|0.00|1713.25|5141.29|5398.35|6854.54|7111.60|1206.59| +2451823|46305|2451926|12758|75829|1537002|2975|18548|54084|1874409|2420|25193|13|20|5|4|263|195|46|66.07|104.39|26.09|3601.80|1200.14|3039.22|4801.94|108.01|0.00|1200.14|1200.14|1308.15|2400.28|2508.29|-1839.08| +2451823|46305|2451871|14018|75829|1537002|2975|18548|54084|1874409|2420|25193|55|14|8|3|188|195|15|20.11|40.62|24.37|243.75|365.55|301.65|609.30|32.89|0.00|140.10|365.55|398.44|505.65|538.54|63.90| +2451823|46305|2451925|3221|75829|1537002|2975|18548|54084|1874409|2420|25193|38|5|13|1|79|195|61|26.34|64.53|52.26|748.47|3187.86|1606.74|3936.33|286.90|0.00|669.17|3187.86|3474.76|3857.03|4143.93|1581.12| +2451823|46305|2451886|11972|75829|1537002|2975|18548|54084|1874409|2420|25193|53|25|9|5|188|195|68|88.60|128.47|110.48|1223.32|7512.64|6024.80|8735.96|527.38|1652.78|1222.64|5859.86|6387.24|7082.50|7609.88|-164.94| +2451823|46305|2451833|4526|75829|1537002|2975|18548|54084|1874409|2420|25193|47|5|19|3|97|195|59|42.11|64.00|55.68|490.88|3285.12|2484.49|3776.00|65.70|0.00|717.44|3285.12|3350.82|4002.56|4068.26|800.63| +2451823|46305|2451854|3752|75829|1537002|2975|18548|54084|1874409|2420|25193|38|29|10|3|233|195|100|75.27|147.52|29.50|11802.00|2950.00|7527.00|14752.00|0.00|0.00|147.00|2950.00|2950.00|3097.00|3097.00|-4577.00| +2451823|46305|2451928|16813|75829|1537002|2975|18548|54084|1874409|2420|25193|53|17|18|5|119|195|97|74.56|170.74|153.66|1656.76|14905.02|7232.32|16561.78|1341.45|0.00|3477.45|14905.02|16246.47|18382.47|19723.92|7672.70| +2451823|46305|2451925|1244|75829|1537002|2975|18548|54084|1874409|2420|25193|7|17|7|2|100|195|4|7.26|16.77|2.68|56.36|10.72|29.04|67.08|0.64|0.00|0.64|10.72|11.36|11.36|12.00|-18.32| +2451823|46305|2451828|15593|75829|1537002|2975|18548|54084|1874409|2420|25193|23|23|12|2|79|195|23|20.08|50.80|1.01|1145.17|23.23|461.84|1168.40|1.62|0.00|303.60|23.23|24.85|326.83|328.45|-438.61| +2451823|46305|2451875|2240|75829|1537002|2975|18548|54084|1874409|2420|25193|29|23|9|1|93|195|69|64.25|176.04|82.73|6438.39|5708.37|4433.25|12146.76|89.05|1255.84|2064.48|4452.53|4541.58|6517.01|6606.06|19.28| +2451823|46305|2451825|3853|75829|1537002|2975|18548|54084|1874409|2420|25193|17|23|5|3|141|195|2|65.55|70.13|61.71|16.84|123.42|131.10|140.26|3.70|0.00|47.68|123.42|127.12|171.10|174.80|-7.68| +2451655|24951|2451705|2653|22958|1117080|2|21677|46296|1026067|5196|14518|56|14|17|1|238|196|77|87.70|113.13|48.64|4965.73|3745.28|6752.90|8711.01|187.26|0.00|3484.25|3745.28|3932.54|7229.53|7416.79|-3007.62| +2451655|24951|2451757|16940|22958|1117080|2|21677|46296|1026067|5196|14518|25|2|11|2|216|196|14|65.78|119.06|33.33|1200.22|466.62|920.92|1666.84|32.66|0.00|766.64|466.62|499.28|1233.26|1265.92|-454.30| +2451655|24951|2451711|5251|22958|1117080|2|21677|46296|1026067|5196|14518|8|7|20|4|105|196|23|78.98|147.69|66.46|1868.29|1528.58|1816.54|3396.87|0.00|917.14|1290.76|611.44|611.44|1902.20|1902.20|-1205.10| +2451655|24951|2451737|14222|22958|1117080|2|21677|46296|1026067|5196|14518|23|2|2|2|272|196|16|5.89|9.60|5.76|61.44|92.16|94.24|153.60|8.29|0.00|29.12|92.16|100.45|121.28|129.57|-2.08| +2451655|24951|2451708|1549|22958|1117080|2|21677|46296|1026067|5196|14518|44|13|18|4|10|196|99|95.55|150.01|1.50|14702.49|148.50|9459.45|14850.99|2.97|0.00|5643.00|148.50|151.47|5791.50|5794.47|-9310.95| +2451655|24951|2451775|115|22958|1117080|2|21677|46296|1026067|5196|14518|35|14|15|5|172|196|57|63.72|122.34|6.11|6625.11|348.27|3632.04|6973.38|0.00|0.00|1952.25|348.27|348.27|2300.52|2300.52|-3283.77| +2451655|24951|2451716|101|22958|1117080|2|21677|46296|1026067|5196|14518|7|20|18|3|235|196|36|88.67|197.73|187.84|356.04|6762.24|3192.12|7118.28|338.11|0.00|3132.00|6762.24|7100.35|9894.24|10232.35|3570.12| +2451655|24951|2451771|17413|22958|1117080|2|21677|46296|1026067|5196|14518|31|5|20|2|114|196|45|98.32|289.06|213.90|3382.20|9625.50|4424.40|13007.70|0.00|0.00|780.30|9625.50|9625.50|10405.80|10405.80|5201.10| +2451655|24951|2451741|14491|22958|1117080|2|21677|46296|1026067|5196|14518|5|11|9|2|262|196|18|90.57|242.72|41.26|3626.28|742.68|1630.26|4368.96|51.98|0.00|1223.28|742.68|794.66|1965.96|2017.94|-887.58| +2451655|24951|2451729|4367|22958|1117080|2|21677|46296|1026067|5196|14518|11|26|2|3|271|196|25|77.26|229.46|87.19|3556.75|2179.75|1931.50|5736.50|65.39|0.00|401.50|2179.75|2245.14|2581.25|2646.64|248.25| +2451655|24951|2451711|9587|22958|1117080|2|21677|46296|1026067|5196|14518|49|8|12|4|109|196|77|54.69|79.30|8.72|5434.66|671.44|4211.13|6106.10|60.42|0.00|671.44|671.44|731.86|1342.88|1403.30|-3539.69| +2451009|70710|2451069|13312|15966|483166|6013|42813|52848|197965|5660|17707|7|25|5|2|259|197|40|98.97|211.79|8.47|8132.80|338.80|3958.80|8471.60|16.94|0.00|2456.40|338.80|355.74|2795.20|2812.14|-3620.00| +2451009|70710|2451072|6910|15966|483166|6013|42813|52848|197965|5660|17707|37|1|6|3|60|197|15|44.13|111.20|62.27|733.95|934.05|661.95|1668.00|33.15|270.87|650.40|663.18|696.33|1313.58|1346.73|1.23| +2451009|70710|2451042|8020|15966|483166|6013|42813|52848|197965|5660|17707|13|19|6|2|178|197|51|17.84|29.43|13.83|795.60|705.33|909.84|1500.93|5.71|134.01|435.03|571.32|577.03|1006.35|1012.06|-338.52| +2451009|70710|2451075|494|15966|483166|6013|42813|52848|197965|5660|17707|1|28|16|5|4|197|58|47.28|79.43|4.76|4330.86|276.08|2742.24|4606.94|24.84|0.00|1197.70|276.08|300.92|1473.78|1498.62|-2466.16| +2451009|70710|2451064|15157|15966|483166|6013|42813|52848|197965|5660|17707|38|8|13|3|276|197|73|53.69|85.90|26.62|4327.44|1943.26|3919.37|6270.70|136.02|0.00|2194.38|1943.26|2079.28|4137.64|4273.66|-1976.11| +2451009|70710|2451120|7795|15966|483166|6013|42813|52848|197965|5660|17707|55|7|17|3|149|197|22|61.88|163.36|44.10|2623.72|970.20|1361.36|3593.92|48.51|0.00|503.14|970.20|1018.71|1473.34|1521.85|-391.16| +2451009|70710|2451094|5258|15966|483166|6013|42813|52848|197965|5660|17707|13|2|10|5|153|197|51|75.16|195.41|13.67|9268.74|697.17|3833.16|9965.91|0.00|0.00|2192.49|697.17|697.17|2889.66|2889.66|-3135.99| +2451009|70710|2451039|7046|15966|483166|6013|42813|52848|197965|5660|17707|58|16|18|2|156|197|32|17.51|29.76|9.52|647.68|304.64|560.32|952.32|27.41|0.00|447.36|304.64|332.05|752.00|779.41|-255.68| +2451009|70710|2451061|11074|15966|483166|6013|42813|52848|197965|5660|17707|43|7|9|1|243|197|68|70.71|98.28|81.57|1136.28|5546.76|4808.28|6683.04|295.08|1331.22|2271.88|4215.54|4510.62|6487.42|6782.50|-592.74| +2451009|70710|2451070|10466|15966|483166|6013|42813|52848|197965|5660|17707|22|26|6|3|237|197|1|72.48|200.04|74.01|126.03|74.01|72.48|200.04|2.22|0.00|74.01|74.01|76.23|148.02|150.24|1.53| +2451009|70710|2451083|3866|15966|483166|6013|42813|52848|197965|5660|17707|44|26|5|4|160|197|39|93.63|108.61|38.01|2753.40|1482.39|3651.57|4235.79|133.41|0.00|1694.16|1482.39|1615.80|3176.55|3309.96|-2169.18| +2451009|70710|2451027|7724|15966|483166|6013|42813|52848|197965|5660|17707|46|14|12|4|223|197|7|8.31|19.27|7.70|80.99|53.90|58.17|134.89|1.07|0.00|12.11|53.90|54.97|66.01|67.08|-4.27| +2451009|70710|2451098|13585|15966|483166|6013|42813|52848|197965|5660|17707|8|13|19|1|36|197|24|45.40|74.91|11.23|1528.32|269.52|1089.60|1797.84|0.00|0.00|305.52|269.52|269.52|575.04|575.04|-820.08| +2451009|70710|2451039|15344|15966|483166|6013|42813|52848|197965|5660|17707|19|8|10|5|50|197|56|4.30|5.46|2.73|152.88|152.88|240.80|305.76|10.70|0.00|42.56|152.88|163.58|195.44|206.14|-87.92| +2451009|70710|2451082|458|15966|483166|6013|42813|52848|197965|5660|17707|2|20|17|3|232|197|42|5.36|14.84|2.07|536.34|86.94|225.12|623.28|2.60|0.00|149.52|86.94|89.54|236.46|239.06|-138.18| +2451897|49938|2451940|14587|54314|376428|5295|30162|26227|202229|6810|3724|26|29|3|4|145|198|73|27.91|39.63|15.45|1765.14|1127.85|2037.43|2892.99|46.58|462.41|346.75|665.44|712.02|1012.19|1058.77|-1371.99| +2451897|49938|2451987|17546|54314|376428|5295|30162|26227|202229|6810|3724|31|19|4|1|75|198|16|94.72|155.34|43.49|1789.60|695.84|1515.52|2485.44|1.94|501.00|24.80|194.84|196.78|219.64|221.58|-1320.68| +2451897|49938|2451999|9452|54314|376428|5295|30162|26227|202229|6810|3724|8|26|3|2|101|198|18|64.45|116.65|50.15|1197.00|902.70|1160.10|2099.70|63.18|0.00|335.88|902.70|965.88|1238.58|1301.76|-257.40| +2451897|49938|2452007|5246|54314|376428|5295|30162|26227|202229|6810|3724|43|5|11|2|170|198|35|4.54|4.67|1.58|108.15|55.30|158.90|163.45|2.76|0.00|52.15|55.30|58.06|107.45|110.21|-103.60| +2451897|49938|2451967|2684|54314|376428|5295|30162|26227|202229|6810|3724|50|1|13|5|127|198|43|35.37|39.96|25.57|618.77|1099.51|1520.91|1718.28|0.00|0.00|515.14|1099.51|1099.51|1614.65|1614.65|-421.40| +2451897|49938|2451932|16667|54314|376428|5295|30162|26227|202229|6810|3724|50|14|7|4|51|198|56|11.28|15.11|9.36|322.00|524.16|631.68|846.16|36.69|0.00|134.96|524.16|560.85|659.12|695.81|-107.52| +2451897|49938|2452005|7841|54314|376428|5295|30162|26227|202229|6810|3724|1|11|1|1|261|198|97|24.02|43.95|10.54|3240.77|1022.38|2329.94|4263.15|92.01|0.00|340.47|1022.38|1114.39|1362.85|1454.86|-1307.56| +2451897|49938|2451912|13643|54314|376428|5295|30162|26227|202229|6810|3724|25|14|6|1|39|198|90|35.95|104.97|77.67|2457.00|6990.30|3235.50|9447.30|39.14|6011.65|3117.60|978.65|1017.79|4096.25|4135.39|-2256.85| +2451897|49938|2451982|11551|54314|376428|5295|30162|26227|202229|6810|3724|13|19|10|3|93|198|88|76.21|151.65|28.81|10809.92|2535.28|6706.48|13345.20|152.11|0.00|1334.08|2535.28|2687.39|3869.36|4021.47|-4171.20| +2451897|49938|2452014|8336|54314|376428|5295|30162|26227|202229|6810|3724|20|2|3|4|275|198|100|50.62|132.62|11.93|12069.00|1193.00|5062.00|13262.00|35.79|0.00|6498.00|1193.00|1228.79|7691.00|7726.79|-3869.00| +2451897|49938|2451960|12763|54314|376428|5295|30162|26227|202229|6810|3724|55|29|9|1|80|198|81|20.48|45.87|28.43|1412.64|2302.83|1658.88|3715.47|101.55|852.04|333.72|1450.79|1552.34|1784.51|1886.06|-208.09| +2451897|49938|2451934|16841|54314|376428|5295|30162|26227|202229|6810|3724|11|25|17|1|168|198|73|54.03|55.65|43.96|853.37|3209.08|3944.19|4062.45|0.00|1155.26|1989.98|2053.82|2053.82|4043.80|4043.80|-1890.37| +2451897|49938|2451968|1751|54314|376428|5295|30162|26227|202229|6810|3724|14|1|14|2|190|198|59|12.94|19.53|15.81|219.48|932.79|763.46|1152.27|0.00|0.00|529.82|932.79|932.79|1462.61|1462.61|169.33| +2451897|49938|2451907|1904|54314|376428|5295|30162|26227|202229|6810|3724|32|26|1|3|125|198|95|1.88|4.64|2.55|198.55|242.25|178.60|440.80|9.69|0.00|30.40|242.25|251.94|272.65|282.34|63.65| +2451897|49938|2451949|4153|54314|376428|5295|30162|26227|202229|6810|3724|55|26|7|5|263|198|2|27.40|52.33|50.76|3.14|101.52|54.80|104.66|5.07|0.00|47.08|101.52|106.59|148.60|153.67|46.72| +2451883|28057|2451929|6728|38107|1014148|1836|18724|7126|539129|5741|37429|23|23|9|1|63|199|36|57.91|104.23|41.69|2251.44|1500.84|2084.76|3752.28|15.00|0.00|1500.84|1500.84|1515.84|3001.68|3016.68|-583.92| +2451883|28057|2451949|3155|38107|1014148|1836|18724|7126|539129|5741|37429|50|1|6|1|23|199|46|97.64|234.33|77.32|7222.46|3556.72|4491.44|10779.18|35.56|0.00|3341.44|3556.72|3592.28|6898.16|6933.72|-934.72| +2451883|28057|2451949|6683|38107|1014148|1836|18724|7126|539129|5741|37429|41|13|20|1|189|199|31|16.25|30.87|1.23|918.84|38.13|503.75|956.97|1.14|0.00|382.54|38.13|39.27|420.67|421.81|-465.62| +2451883|28057|2451890|1538|38107|1014148|1836|18724|7126|539129|5741|37429|7|11|18|5|192|199|86|62.56|90.08|34.23|4803.10|2943.78|5380.16|7746.88|117.75|0.00|774.00|2943.78|3061.53|3717.78|3835.53|-2436.38| +2451883|28057|2451919|7349|38107|1014148|1836|18724|7126|539129|5741|37429|11|11|16|1|156|199|90|1.74|3.67|3.44|20.70|309.60|156.60|330.30|24.76|0.00|108.90|309.60|334.36|418.50|443.26|153.00| +2451883|28057|2451963|7553|38107|1014148|1836|18724|7126|539129|5741|37429|38|8|20|5|20|199|98|91.67|93.50|73.86|1924.72|7238.28|8983.66|9163.00|39.81|3257.22|1649.34|3981.06|4020.87|5630.40|5670.21|-5002.60| +2451883|28057|2451915|293|38107|1014148|1836|18724|7126|539129|5741|37429|7|5|9|5|74|199|51|78.10|174.94|29.73|7405.71|1516.23|3983.10|8921.94|60.64|0.00|3925.47|1516.23|1576.87|5441.70|5502.34|-2466.87| +2451883|28057|2451964|2627|38107|1014148|1836|18724|7126|539129|5741|37429|1|8|9|3|107|199|81|18.47|21.79|19.39|194.40|1570.59|1496.07|1764.99|62.82|0.00|829.44|1570.59|1633.41|2400.03|2462.85|74.52| +2451883|28057|2451910|2000|38107|1014148|1836|18724|7126|539129|5741|37429|29|11|10|1|115|199|77|86.07|227.22|145.42|6298.60|11197.34|6627.39|17495.94|559.86|0.00|3848.46|11197.34|11757.20|15045.80|15605.66|4569.95| +2451883|28057|2451923|3881|38107|1014148|1836|18724|7126|539129|5741|37429|35|14|11|4|144|199|29|49.92|73.88|39.15|1007.17|1135.35|1447.68|2142.52|68.12|0.00|321.32|1135.35|1203.47|1456.67|1524.79|-312.33| +2451883|28057|2451898|14819|38107|1014148|1836|18724|7126|539129|5741|37429|7|2|3|1|139|199|93|15.06|15.81|5.69|941.16|529.17|1400.58|1470.33|10.58|0.00|690.99|529.17|539.75|1220.16|1230.74|-871.41| +2451883|28057|2451885|12056|38107|1014148|1836|18724|7126|539129|5741|37429|44|20|6|4|12|199|78|22.73|61.82|37.09|1928.94|2893.02|1772.94|4821.96|173.58|0.00|1108.38|2893.02|3066.60|4001.40|4174.98|1120.08| +2451883|28057|2451929|10502|38107|1014148|1836|18724|7126|539129|5741|37429|19|20|19|5|291|199|58|56.39|160.71|38.57|7084.12|2237.06|3270.62|9321.18|44.74|0.00|2143.68|2237.06|2281.80|4380.74|4425.48|-1033.56| +2452425|22842|2452525|13464|26648|1538126|5957|41388|35939|1835029|2528|33159|31|30|14|3|192|200|89|73.08|120.58|81.99|3434.51|7297.11|6504.12|10731.62|364.85|0.00|3111.44|7297.11|7661.96|10408.55|10773.40|792.99| +2452425|22842|2452462|1389|26648|1538126|5957|41388|35939|1835029|2528|33159|12|21|16|1|146|200|31|10.98|19.65|4.51|469.34|139.81|340.38|609.15|5.59|0.00|121.83|139.81|145.40|261.64|267.23|-200.57| +2452425|22842|2452495|12768|26648|1538126|5957|41388|35939|1835029|2528|33159|31|6|20|5|287|200|58|61.83|127.36|26.74|5835.96|1550.92|3586.14|7386.88|46.52|0.00|147.32|1550.92|1597.44|1698.24|1744.76|-2035.22| +2452425|22842|2452490|12747|26648|1538126|5957|41388|35939|1835029|2528|33159|15|30|2|4|36|200|3|69.38|97.13|8.74|265.17|26.22|208.14|291.39|2.09|0.00|43.68|26.22|28.31|69.90|71.99|-181.92| +2452425|22842|2452444|9621|26648|1538126|5957|41388|35939|1835029|2528|33159|21|13|3|2|258|200|86|27.91|44.37|35.93|725.84|3089.98|2400.26|3815.82|26.26|463.49|724.98|2626.49|2652.75|3351.47|3377.73|226.23| +2452425|22842|2452464|3109|26648|1538126|5957|41388|35939|1835029|2528|33159|33|19|6|2|125|200|97|76.92|101.53|51.78|4825.75|5022.66|7461.24|9848.41|251.13|0.00|491.79|5022.66|5273.79|5514.45|5765.58|-2438.58| +2452425|22842|2452523|1279|26648|1538126|5957|41388|35939|1835029|2528|33159|1|7|7|5|73|200|66|81.07|209.16|181.96|1795.20|12009.36|5350.62|13804.56|64.85|11288.79|3174.60|720.57|785.42|3895.17|3960.02|-4630.05| +2452425|22842|2452518|4927|26648|1538126|5957|41388|35939|1835029|2528|33159|45|7|5|1|97|200|38|82.40|163.15|17.94|5517.98|681.72|3131.20|6199.70|61.35|0.00|2665.70|681.72|743.07|3347.42|3408.77|-2449.48| +2452425|22842|2452461|1089|26648|1538126|5957|41388|35939|1835029|2528|33159|36|30|9|1|170|200|63|1.94|3.66|1.20|154.98|75.60|122.22|230.58|4.53|0.00|13.23|75.60|80.13|88.83|93.36|-46.62| +2452425|22842|2452538|3036|26648|1538126|5957|41388|35939|1835029|2528|33159|57|25|7|1|138|200|95|16.45|25.66|18.21|707.75|1729.95|1562.75|2437.70|155.69|0.00|950.00|1729.95|1885.64|2679.95|2835.64|167.20| +2452425|22842|2452491|17766|26648|1538126|5957|41388|35939|1835029|2528|33159|7|7|1|1|258|200|3|93.92|281.76|45.08|710.04|135.24|281.76|845.28|9.46|0.00|42.24|135.24|144.70|177.48|186.94|-146.52| +2452425|22842|2452466|14472|26648|1538126|5957|41388|35939|1835029|2528|33159|12|15|7|4|238|200|74|72.40|76.74|73.67|227.18|5451.58|5357.60|5678.76|0.00|0.00|1078.92|5451.58|5451.58|6530.50|6530.50|93.98| +2452425|22842|2452434|14113|26648|1538126|5957|41388|35939|1835029|2528|33159|27|18|14|2|287|200|44|88.05|180.50|23.46|6909.76|1032.24|3874.20|7942.00|0.00|0.00|794.20|1032.24|1032.24|1826.44|1826.44|-2841.96| +2452634|72715|2452666|7437|35986|1230354|7122|47537|59689|860032|3088|35056|25|6|13|3|183|201|58|67.31|160.87|141.56|1119.98|8210.48|3903.98|9330.46|0.00|0.00|466.32|8210.48|8210.48|8676.80|8676.80|4306.50| +2452634|72715|2452693|14487|35986|1230354|7122|47537|59689|860032|3088|35056|49|9|5|5|65|201|46|30.25|56.87|40.37|759.00|1857.02|1391.50|2616.02|167.13|0.00|0.00|1857.02|2024.15|1857.02|2024.15|465.52| +2452634|72715|2452703|6864|35986|1230354|7122|47537|59689|860032|3088|35056|31|25|20|2|244|201|69|49.49|87.10|2.61|5829.81|180.09|3414.81|6009.90|10.80|0.00|781.08|180.09|190.89|961.17|971.97|-3234.72| +2452634|72715|2452714|10569|35986|1230354|7122|47537|59689|860032|3088|35056|45|30|17|4|51|201|44|11.51|32.45|19.14|585.64|842.16|506.44|1427.80|42.10|0.00|613.80|842.16|884.26|1455.96|1498.06|335.72| +2452634|72715|2452670|9765|35986|1230354|7122|47537|59689|860032|3088|35056|6|3|11|5|179|201|9|46.99|111.83|7.82|936.09|70.38|422.91|1006.47|0.00|0.00|171.09|70.38|70.38|241.47|241.47|-352.53| +2452634|72715|2452721|11148|35986|1230354|7122|47537|59689|860032|3088|35056|15|15|11|4|54|201|72|28.34|58.66|32.84|1859.04|2364.48|2040.48|4223.52|141.86|0.00|252.72|2364.48|2506.34|2617.20|2759.06|324.00| +2452634|72715|2452741|12259|35986|1230354|7122|47537|59689|860032|3088|35056|37|18|16|1|263|201|55|42.25|85.76|80.61|283.25|4433.55|2323.75|4716.80|177.34|0.00|612.70|4433.55|4610.89|5046.25|5223.59|2109.80| +2452634|72715|2452688|11335|35986|1230354|7122|47537|59689|860032|3088|35056|3|30|5|3|271|201|30|36.51|73.02|17.52|1665.00|525.60|1095.30|2190.60|15.76|0.00|87.60|525.60|541.36|613.20|628.96|-569.70| +2452634|72715|2452695|11265|35986|1230354|7122|47537|59689|860032|3088|35056|3|12|10|1|295|201|89|69.44|72.91|24.06|4347.65|2141.34|6180.16|6488.99|0.00|899.36|388.93|1241.98|1241.98|1630.91|1630.91|-4938.18| +2452634|72715|2452635|4386|35986|1230354|7122|47537|59689|860032|3088|35056|25|15|6|1|226|201|90|23.20|69.36|33.98|3184.20|3058.20|2088.00|6242.40|91.74|0.00|810.90|3058.20|3149.94|3869.10|3960.84|970.20| +2452634|72715|2452713|16183|35986|1230354|7122|47537|59689|860032|3088|35056|1|1|8|3|15|201|37|30.97|66.27|33.13|1226.18|1225.81|1145.89|2451.99|85.80|0.00|1176.60|1225.81|1311.61|2402.41|2488.21|79.92| +2452634|72715|2452721|9537|35986|1230354|7122|47537|59689|860032|3088|35056|55|24|19|1|130|201|92|90.30|198.66|93.37|9686.68|8590.04|8307.60|18276.72|257.70|0.00|1644.04|8590.04|8847.74|10234.08|10491.78|282.44| +2452600|36036|2452717|5328|4610|944930|6403|20676|30021|1796842|6342|47862|48|21|8|5|32|202|43|99.72|224.37|170.52|2315.55|7332.36|4287.96|9647.91|219.97|0.00|2990.65|7332.36|7552.33|10323.01|10542.98|3044.40| +2452600|36036|2452710|5449|4610|944930|6403|20676|30021|1796842|6342|47862|31|13|5|3|121|202|53|53.23|134.13|21.46|5971.51|1137.38|2821.19|7108.89|0.00|1137.38|3341.12|0.00|0.00|3341.12|3341.12|-2821.19| +2452600|36036|2452674|13267|4610|944930|6403|20676|30021|1796842|6342|47862|30|7|3|2|128|202|82|6.78|12.20|10.37|150.06|850.34|555.96|1000.40|34.01|0.00|79.54|850.34|884.35|929.88|963.89|294.38| +2452600|36036|2452701|15085|4610|944930|6403|20676|30021|1796842|6342|47862|6|25|10|2|117|202|1|77.06|214.99|176.29|38.70|176.29|77.06|214.99|8.81|0.00|70.94|176.29|185.10|247.23|256.04|99.23| +2452600|36036|2452669|2358|4610|944930|6403|20676|30021|1796842|6342|47862|43|18|6|3|150|202|20|36.51|63.52|24.13|787.80|482.60|730.20|1270.40|33.78|0.00|330.20|482.60|516.38|812.80|846.58|-247.60| +2452600|36036|2452669|7488|4610|944930|6403|20676|30021|1796842|6342|47862|7|25|15|4|266|202|86|37.27|38.76|33.33|466.98|2866.38|3205.22|3333.36|133.28|200.64|866.02|2665.74|2799.02|3531.76|3665.04|-539.48| +2452600|36036|2452644|14875|4610|944930|6403|20676|30021|1796842|6342|47862|6|12|16|1|239|202|43|25.47|44.06|2.64|1781.06|113.52|1095.21|1894.58|10.21|0.00|302.72|113.52|123.73|416.24|426.45|-981.69| +2452600|36036|2452664|17121|4610|944930|6403|20676|30021|1796842|6342|47862|12|9|1|2|226|202|5|21.92|62.03|32.87|145.80|164.35|109.60|310.15|6.57|0.00|102.30|164.35|170.92|266.65|273.22|54.75| +2452600|36036|2452637|17299|4610|944930|6403|20676|30021|1796842|6342|47862|7|18|4|4|54|202|100|84.08|172.36|10.34|16202.00|1034.00|8408.00|17236.00|82.72|0.00|2757.00|1034.00|1116.72|3791.00|3873.72|-7374.00| +2452600|36036|2452665|11961|4610|944930|6403|20676|30021|1796842|6342|47862|37|7|19|4|146|202|39|20.53|54.19|4.33|1944.54|168.87|800.67|2113.41|5.06|0.00|1035.45|168.87|173.93|1204.32|1209.38|-631.80| +2452600|36036|2452665|1633|4610|944930|6403|20676|30021|1796842|6342|47862|51|3|5|1|27|202|27|5.59|12.40|3.72|234.36|100.44|150.93|334.80|3.69|54.23|53.46|46.21|49.90|99.67|103.36|-104.72| +2451567|10089|2451662|16861|52422|1717719|7016|33960|81047|468544|5451|14795|37|25|17|1|145|203|20|3.06|8.26|8.01|5.00|160.20|61.20|165.20|0.92|113.74|67.60|46.46|47.38|114.06|114.98|-14.74| +2451567|10089|2451648|17803|52422|1717719|7016|33960|81047|468544|5451|14795|43|20|17|1|165|203|23|17.05|37.16|19.32|410.32|444.36|392.15|854.68|31.10|0.00|213.67|444.36|475.46|658.03|689.13|52.21| +2451567|10089|2451624|3683|52422|1717719|7016|33960|81047|468544|5451|14795|19|8|7|4|221|203|11|81.72|118.49|113.75|52.14|1251.25|898.92|1303.39|1.75|1163.66|273.68|87.59|89.34|361.27|363.02|-811.33| +2451567|10089|2451670|12677|52422|1717719|7016|33960|81047|468544|5451|14795|50|19|4|3|202|203|77|47.92|126.02|41.58|6501.88|3201.66|3689.84|9703.54|160.08|0.00|1843.38|3201.66|3361.74|5045.04|5205.12|-488.18| +2451567|10089|2451659|446|52422|1717719|7016|33960|81047|468544|5451|14795|44|23|4|1|55|203|27|49.49|92.54|75.88|449.82|2048.76|1336.23|2498.58|81.95|0.00|424.71|2048.76|2130.71|2473.47|2555.42|712.53| +2451567|10089|2451683|12685|52422|1717719|7016|33960|81047|468544|5451|14795|56|11|6|3|110|203|4|56.67|163.77|40.94|491.32|163.76|226.68|655.08|4.91|0.00|242.36|163.76|168.67|406.12|411.03|-62.92| +2451567|10089|2451574|15566|52422|1717719|7016|33960|81047|468544|5451|14795|41|23|20|1|232|203|86|67.87|139.81|117.44|1923.82|10099.84|5836.82|12023.66|605.99|0.00|5049.92|10099.84|10705.83|15149.76|15755.75|4263.02| +2451567|10089|2451606|281|52422|1717719|7016|33960|81047|468544|5451|14795|44|11|12|3|174|203|26|68.55|181.65|32.69|3872.96|849.94|1782.30|4722.90|0.00|0.00|708.24|849.94|849.94|1558.18|1558.18|-932.36| +2451567|10089|2451643|1051|52422|1717719|7016|33960|81047|468544|5451|14795|55|7|17|3|96|203|85|97.85|234.84|110.37|10579.95|9381.45|8317.25|19961.40|388.39|2908.24|4391.10|6473.21|6861.60|10864.31|11252.70|-1844.04| +2451567|10089|2451651|6733|52422|1717719|7016|33960|81047|468544|5451|14795|59|25|3|3|43|203|14|81.67|126.58|12.65|1595.02|177.10|1143.38|1772.12|9.88|12.39|832.86|164.71|174.59|997.57|1007.45|-978.67| +2451567|10089|2451677|14701|52422|1717719|7016|33960|81047|468544|5451|14795|47|2|20|3|26|203|23|3.05|7.32|2.92|101.20|67.16|70.15|168.36|4.70|0.00|42.09|67.16|71.86|109.25|113.95|-2.99| +2451567|10089|2451668|12272|52422|1717719|7016|33960|81047|468544|5451|14795|59|5|13|3|63|203|7|62.00|94.24|39.58|382.62|277.06|434.00|659.68|9.30|121.90|39.55|155.16|164.46|194.71|204.01|-278.84| +2451567|10089|2451631|5372|52422|1717719|7016|33960|81047|468544|5451|14795|14|20|17|3|179|203|38|83.46|114.34|93.75|782.42|3562.50|3171.48|4344.92|106.87|0.00|564.68|3562.50|3669.37|4127.18|4234.05|391.02| +2451567|10089|2451593|12307|52422|1717719|7016|33960|81047|468544|5451|14795|53|1|15|4|230|203|72|74.04|165.84|87.89|5612.40|6328.08|5330.88|11940.48|253.12|0.00|4417.92|6328.08|6581.20|10746.00|10999.12|997.20| +2451567|10089|2451636|4259|52422|1717719|7016|33960|81047|468544|5451|14795|8|8|12|4|169|203|43|26.43|74.26|69.80|191.78|3001.40|1136.49|3193.18|240.11|0.00|446.77|3001.40|3241.51|3448.17|3688.28|1864.91| +2451967|29516|2452007|2261|34398|1096577|4803|37429|31901|1527285|5260|2686|1|7|17|2|290|204|73|15.59|23.54|5.17|1341.01|377.41|1138.07|1718.42|30.19|0.00|446.76|377.41|407.60|824.17|854.36|-760.66| +2451967|29516|2451974|12545|34398|1096577|4803|37429|31901|1527285|5260|2686|21|1|1|1|188|204|94|32.08|61.27|29.40|2995.78|2763.60|3015.52|5759.38|110.54|0.00|287.64|2763.60|2874.14|3051.24|3161.78|-251.92| +2451967|29516|2452033|13011|34398|1096577|4803|37429|31901|1527285|5260|2686|53|11|15|4|40|204|1|91.84|237.86|164.12|73.74|164.12|91.84|237.86|5.41|73.85|116.55|90.27|95.68|206.82|212.23|-1.57| +2451967|29516|2452063|2649|34398|1096577|4803|37429|31901|1527285|5260|2686|35|7|13|4|195|204|83|46.78|54.26|24.41|2477.55|2026.03|3882.74|4503.58|40.52|0.00|585.15|2026.03|2066.55|2611.18|2651.70|-1856.71| +2451967|29516|2452030|17771|34398|1096577|4803|37429|31901|1527285|5260|2686|53|1|5|4|223|204|93|33.72|49.56|22.79|2489.61|2119.47|3135.96|4609.08|0.00|0.00|1935.33|2119.47|2119.47|4054.80|4054.80|-1016.49| +2451967|29516|2452058|9119|34398|1096577|4803|37429|31901|1527285|5260|2686|35|25|18|3|215|204|25|61.92|115.17|56.43|1468.50|1410.75|1548.00|2879.25|14.10|0.00|633.25|1410.75|1424.85|2044.00|2058.10|-137.25| +2451967|29516|2452082|7589|34398|1096577|4803|37429|31901|1527285|5260|2686|31|19|11|5|239|204|45|32.78|76.70|2.30|3348.00|103.50|1475.10|3451.50|7.24|0.00|828.00|103.50|110.74|931.50|938.74|-1371.60| +2451967|29516|2451995|17959|34398|1096577|4803|37429|31901|1527285|5260|2686|27|25|19|5|181|204|87|49.88|74.32|32.70|3620.94|2844.90|4339.56|6465.84|28.44|0.00|581.16|2844.90|2873.34|3426.06|3454.50|-1494.66| +2451967|29516|2452049|11053|34398|1096577|4803|37429|31901|1527285|5260|2686|51|9|7|4|101|204|73|24.48|60.71|32.78|2038.89|2392.94|1787.04|4431.83|95.71|0.00|531.44|2392.94|2488.65|2924.38|3020.09|605.90| +2451967|29516|2451994|15281|34398|1096577|4803|37429|31901|1527285|5260|2686|5|9|17|4|251|204|28|15.13|36.46|2.91|939.40|81.48|423.64|1020.88|4.88|0.00|255.08|81.48|86.36|336.56|341.44|-342.16| +2451967|29516|2452007|1429|34398|1096577|4803|37429|31901|1527285|5260|2686|35|7|20|3|179|204|76|3.48|9.63|8.08|117.80|614.08|264.48|731.88|49.12|0.00|234.08|614.08|663.20|848.16|897.28|349.60| +2451967|29516|2452039|3915|34398|1096577|4803|37429|31901|1527285|5260|2686|37|1|3|5|127|204|28|32.18|44.08|33.94|283.92|950.32|901.04|1234.24|19.00|0.00|579.88|950.32|969.32|1530.20|1549.20|49.28| +2451967|29516|2452086|15713|34398|1096577|4803|37429|31901|1527285|5260|2686|39|11|15|4|250|204|17|97.81|201.48|84.62|1986.62|1438.54|1662.77|3425.16|14.38|0.00|1198.67|1438.54|1452.92|2637.21|2651.59|-224.23| +2451967|29516|2451988|1631|34398|1096577|4803|37429|31901|1527285|5260|2686|37|21|16|3|123|204|96|63.39|75.43|58.08|1665.60|5575.68|6085.44|7241.28|111.51|0.00|2461.44|5575.68|5687.19|8037.12|8148.63|-509.76| +2452332|84084|2452354|5827|84959|922904|6002|20747|42093|1816460|3735|46033|25|13|11|5|26|205|77|7.03|10.12|8.50|124.74|654.50|541.31|779.24|1.30|523.60|77.77|130.90|132.20|208.67|209.97|-410.41| +2452332|84084|2452411|10003|84959|922904|6002|20747|42093|1816460|3735|46033|21|13|16|2|48|205|41|34.11|42.97|8.59|1409.58|352.19|1398.51|1761.77|21.13|0.00|87.74|352.19|373.32|439.93|461.06|-1046.32| +2452332|84084|2452352|12801|84959|922904|6002|20747|42093|1816460|3735|46033|19|6|16|4|282|205|75|76.39|97.01|35.89|4584.00|2691.75|5729.25|7275.75|188.42|0.00|1018.50|2691.75|2880.17|3710.25|3898.67|-3037.50| +2452332|84084|2452406|14671|84959|922904|6002|20747|42093|1816460|3735|46033|1|21|1|1|71|205|22|97.71|209.09|50.18|3496.02|1103.96|2149.62|4599.98|0.00|0.00|1793.88|1103.96|1103.96|2897.84|2897.84|-1045.66| +2452332|84084|2452402|6337|84959|922904|6002|20747|42093|1816460|3735|46033|33|9|20|3|216|205|26|29.30|86.72|56.36|789.36|1465.36|761.80|2254.72|87.92|0.00|1014.52|1465.36|1553.28|2479.88|2567.80|703.56| +2452332|84084|2452445|14449|84959|922904|6002|20747|42093|1816460|3735|46033|55|3|6|3|37|205|17|36.83|78.44|35.29|733.55|599.93|626.11|1333.48|47.99|0.00|293.25|599.93|647.92|893.18|941.17|-26.18| +2452332|84084|2452439|9066|84959|922904|6002|20747|42093|1816460|3735|46033|55|18|20|3|49|205|33|1.45|3.58|0.71|94.71|23.43|47.85|118.14|0.46|0.00|44.88|23.43|23.89|68.31|68.77|-24.42| +2452332|84084|2452335|5130|84959|922904|6002|20747|42093|1816460|3735|46033|51|1|13|5|277|205|87|28.23|59.28|31.41|2424.69|2732.67|2456.01|5157.36|245.94|0.00|1237.14|2732.67|2978.61|3969.81|4215.75|276.66| +2452332|84084|2452413|6555|84959|922904|6002|20747|42093|1816460|3735|46033|42|30|9|3|57|205|72|42.95|121.54|13.36|7788.96|961.92|3092.40|8750.88|38.47|0.00|0.00|961.92|1000.39|961.92|1000.39|-2130.48| +2452332|84084|2452421|4891|84959|922904|6002|20747|42093|1816460|3735|46033|18|13|5|3|181|205|2|65.42|112.52|85.51|54.02|171.02|130.84|225.04|6.84|0.00|83.26|171.02|177.86|254.28|261.12|40.18| +2452332|84084|2452381|6858|84959|922904|6002|20747|42093|1816460|3735|46033|6|6|10|5|40|205|20|37.39|64.31|47.58|334.60|951.60|747.80|1286.20|47.58|0.00|64.20|951.60|999.18|1015.80|1063.38|203.80| +2452332|84084|2452451|4497|84959|922904|6002|20747|42093|1816460|3735|46033|9|6|18|3|48|205|37|99.65|220.22|85.88|4970.58|3177.56|3687.05|8148.14|31.77|0.00|2118.25|3177.56|3209.33|5295.81|5327.58|-509.49| +2451405|34219|2451456|14647|66946|1162703|3260|49017|51904|1858490|1841|24798|7|8|13|1|59|206|65|47.07|119.55|32.27|5673.20|2097.55|3059.55|7770.75|62.92|0.00|1942.20|2097.55|2160.47|4039.75|4102.67|-962.00| +2451405|34219|2451473|589|66946|1162703|3260|49017|51904|1858490|1841|24798|31|2|8|1|237|206|67|33.39|39.73|1.58|2556.05|105.86|2237.13|2661.91|3.17|0.00|1303.82|105.86|109.03|1409.68|1412.85|-2131.27| +2451405|34219|2451407|16897|66946|1162703|3260|49017|51904|1858490|1841|24798|22|7|6|3|3|206|48|90.88|251.73|234.10|846.24|11236.80|4362.24|12083.04|0.00|0.00|4953.60|11236.80|11236.80|16190.40|16190.40|6874.56| +2451405|34219|2451471|17143|66946|1162703|3260|49017|51904|1858490|1841|24798|25|19|17|4|211|206|78|98.34|259.61|85.67|13567.32|6682.26|7670.52|20249.58|106.91|4543.93|3037.32|2138.33|2245.24|5175.65|5282.56|-5532.19| +2451405|34219|2451482|8860|66946|1162703|3260|49017|51904|1858490|1841|24798|32|8|13|4|292|206|45|8.46|23.43|5.62|801.45|252.90|380.70|1054.35|20.23|0.00|242.10|252.90|273.13|495.00|515.23|-127.80| +2451405|34219|2451501|13526|66946|1162703|3260|49017|51904|1858490|1841|24798|1|4|2|4|83|206|77|15.44|23.62|9.21|1109.57|709.17|1188.88|1818.74|28.36|0.00|854.70|709.17|737.53|1563.87|1592.23|-479.71| +2451405|34219|2451446|9943|66946|1162703|3260|49017|51904|1858490|1841|24798|56|25|17|3|212|206|45|1.60|4.78|1.19|161.55|53.55|72.00|215.10|0.51|44.98|105.30|8.57|9.08|113.87|114.38|-63.43| +2451405|34219|2451496|5402|66946|1162703|3260|49017|51904|1858490|1841|24798|20|16|7|5|3|206|96|51.73|123.11|77.55|4373.76|7444.80|4966.08|11818.56|0.00|595.58|5199.36|6849.22|6849.22|12048.58|12048.58|1883.14| +2451405|34219|2451516|11708|66946|1162703|3260|49017|51904|1858490|1841|24798|55|19|2|4|166|206|41|92.75|268.97|121.03|6065.54|4962.23|3802.75|11027.77|347.35|0.00|2425.97|4962.23|5309.58|7388.20|7735.55|1159.48| +2452161|28570|2452251|3949|66434|1472799|4411|3176|98418|837068|1518|24283|49|17|20|4|178|207|7|51.75|113.85|113.85|0.00|796.95|362.25|796.95|31.87|0.00|278.88|796.95|828.82|1075.83|1107.70|434.70| +2452161|28570|2452274|249|66434|1472799|4411|3176|98418|837068|1518|24283|35|17|3|4|58|207|31|70.55|201.77|119.04|2564.63|3690.24|2187.05|6254.87|258.31|0.00|124.93|3690.24|3948.55|3815.17|4073.48|1503.19| +2452161|28570|2452275|2307|66434|1472799|4411|3176|98418|837068|1518|24283|17|3|15|1|184|207|62|56.24|65.80|4.60|3794.40|285.20|3486.88|4079.60|5.70|0.00|1713.06|285.20|290.90|1998.26|2003.96|-3201.68| +2452161|28570|2452210|3931|66434|1472799|4411|3176|98418|837068|1518|24283|5|17|15|1|274|207|4|37.64|45.92|11.02|139.60|44.08|150.56|183.68|2.64|0.00|20.20|44.08|46.72|64.28|66.92|-106.48| +2452161|28570|2452171|6743|66434|1472799|4411|3176|98418|837068|1518|24283|17|21|13|3|220|207|87|12.85|35.08|29.81|458.49|2593.47|1117.95|3051.96|129.67|0.00|1342.41|2593.47|2723.14|3935.88|4065.55|1475.52| +2452161|28570|2452257|9135|66434|1472799|4411|3176|98418|837068|1518|24283|7|13|7|5|53|207|16|9.95|10.14|7.30|45.44|116.80|159.20|162.24|8.17|0.00|53.44|116.80|124.97|170.24|178.41|-42.40| +2452161|28570|2452215|11655|66434|1472799|4411|3176|98418|837068|1518|24283|47|29|6|2|78|207|70|63.26|165.74|165.74|0.00|11601.80|4428.20|11601.80|464.07|0.00|2088.10|11601.80|12065.87|13689.90|14153.97|7173.60| +2452161|28570|2452201|5073|66434|1472799|4411|3176|98418|837068|1518|24283|45|25|13|3|133|207|7|62.32|183.22|27.48|1090.18|192.36|436.24|1282.54|3.84|0.00|551.46|192.36|196.20|743.82|747.66|-243.88| +2452161|28570|2452188|10533|66434|1472799|4411|3176|98418|837068|1518|24283|43|25|17|3|182|207|3|69.11|162.40|118.55|131.55|355.65|207.33|487.20|0.00|39.12|73.08|316.53|316.53|389.61|389.61|109.20| +2452161|28570|2452236|6427|66434|1472799|4411|3176|98418|837068|1518|24283|23|29|1|2|273|207|79|77.61|120.29|110.66|760.77|8742.14|6131.19|9502.91|174.84|0.00|1805.15|8742.14|8916.98|10547.29|10722.13|2610.95| +2452161|28570|2452242|1127|66434|1472799|4411|3176|98418|837068|1518|24283|21|7|17|5|163|207|39|10.19|18.54|7.23|441.09|281.97|397.41|723.06|0.00|0.00|14.43|281.97|281.97|296.40|296.40|-115.44| +2452161|28570|2452199|6781|66434|1472799|4411|3176|98418|837068|1518|24283|17|19|11|2|50|207|13|59.82|151.94|103.31|632.19|1343.03|777.66|1975.22|13.43|0.00|118.43|1343.03|1356.46|1461.46|1474.89|565.37| +2452161|28570|2452162|16161|66434|1472799|4411|3176|98418|837068|1518|24283|5|15|10|2|127|207|36|74.48|99.05|56.45|1533.60|2032.20|2681.28|3565.80|0.00|0.00|1533.24|2032.20|2032.20|3565.44|3565.44|-649.08| +2450872|52023|2450905|5018|41764|375235|5859|45330|17921|409028|3296|27967|32|20|15|4|23|208|25|58.44|146.68|67.47|1980.25|1686.75|1461.00|3667.00|57.68|1045.78|476.50|640.97|698.65|1117.47|1175.15|-820.03| +2450872|52023|2450967|3508|41764|375235|5859|45330|17921|409028|3296|27967|22|4|14|5|186|208|26|72.69|82.86|65.45|452.66|1701.70|1889.94|2154.36|36.75|1242.24|667.68|459.46|496.21|1127.14|1163.89|-1430.48| +2450872|52023|2450914|10042|41764|375235|5859|45330|17921|409028|3296|27967|32|2|4|4|271|208|85|70.25|207.23|161.63|3876.00|13738.55|5971.25|17614.55|686.92|0.00|6341.00|13738.55|14425.47|20079.55|20766.47|7767.30| +2450872|52023|2450970|13216|41764|375235|5859|45330|17921|409028|3296|27967|38|8|7|2|184|208|47|41.48|70.10|37.15|1548.65|1746.05|1949.56|3294.70|139.68|0.00|1152.91|1746.05|1885.73|2898.96|3038.64|-203.51| +2450872|52023|2450949|14546|41764|375235|5859|45330|17921|409028|3296|27967|34|1|14|2|280|208|42|97.36|198.61|184.70|584.22|7757.40|4089.12|8341.62|155.14|0.00|1000.86|7757.40|7912.54|8758.26|8913.40|3668.28| +2450872|52023|2450967|12442|41764|375235|5859|45330|17921|409028|3296|27967|38|7|1|5|2|208|78|89.54|212.20|0.00|16551.60|0.00|6984.12|16551.60|0.00|0.00|6786.00|0.00|0.00|6786.00|6786.00|-6984.12| +2450872|52023|2450962|17215|41764|375235|5859|45330|17921|409028|3296|27967|8|8|12|2|199|208|51|62.15|116.22|52.29|3260.43|2666.79|3169.65|5927.22|186.67|0.00|414.63|2666.79|2853.46|3081.42|3268.09|-502.86| +2450872|52023|2450965|16264|41764|375235|5859|45330|17921|409028|3296|27967|10|25|10|2|274|208|52|54.92|124.11|31.02|4840.68|1613.04|2855.84|6453.72|145.17|0.00|64.48|1613.04|1758.21|1677.52|1822.69|-1242.80| +2450872|52023|2450904|3661|41764|375235|5859|45330|17921|409028|3296|27967|44|20|9|4|12|208|46|23.72|58.35|56.59|80.96|2603.14|1091.12|2684.10|52.06|0.00|805.00|2603.14|2655.20|3408.14|3460.20|1512.02| +2450872|52023|2450988|8230|41764|375235|5859|45330|17921|409028|3296|27967|28|10|4|2|263|208|15|78.38|177.92|117.42|907.50|1761.30|1175.70|2668.80|140.90|0.00|1174.20|1761.30|1902.20|2935.50|3076.40|585.60| +2450872|52023|2450877|11788|41764|375235|5859|45330|17921|409028|3296|27967|43|2|12|2|92|208|79|47.42|55.95|40.28|1237.93|3182.12|3746.18|4420.05|157.83|1209.20|795.53|1972.92|2130.75|2768.45|2926.28|-1773.26| +2452155|48746|2452264|12529|91506|728572|5191|42210|1234|969670|2867|31933|7|23|3|1|292|209|44|96.19|271.25|78.66|8473.96|3461.04|4232.36|11935.00|34.61|0.00|3102.88|3461.04|3495.65|6563.92|6598.53|-771.32| +2452155|48746|2452274|7365|91506|728572|5191|42210|1234|969670|2867|31933|43|5|1|4|147|209|57|31.17|46.44|3.71|2435.61|211.47|1776.69|2647.08|4.22|0.00|634.98|211.47|215.69|846.45|850.67|-1565.22| +2452155|48746|2452275|6945|91506|728572|5191|42210|1234|969670|2867|31933|7|11|18|2|229|209|95|18.24|25.53|2.04|2231.55|193.80|1732.80|2425.35|0.00|0.00|1066.85|193.80|193.80|1260.65|1260.65|-1539.00| +2452155|48746|2452222|15693|91506|728572|5191|42210|1234|969670|2867|31933|25|17|5|1|57|209|28|25.16|65.41|3.27|1739.92|91.56|704.48|1831.48|1.83|0.00|127.96|91.56|93.39|219.52|221.35|-612.92| +2452155|48746|2452220|10739|91506|728572|5191|42210|1234|969670|2867|31933|41|23|3|4|75|209|77|19.24|57.52|0.00|4429.04|0.00|1481.48|4429.04|0.00|0.00|929.39|0.00|0.00|929.39|929.39|-1481.48| +2452155|48746|2452206|5197|91506|728572|5191|42210|1234|969670|2867|31933|19|1|19|2|48|209|31|21.54|54.06|3.78|1558.68|117.18|667.74|1675.86|2.34|0.00|502.51|117.18|119.52|619.69|622.03|-550.56| +2452155|48746|2452245|5149|91506|728572|5191|42210|1234|969670|2867|31933|15|19|5|2|62|209|21|92.46|146.08|42.36|2178.12|889.56|1941.66|3067.68|17.79|0.00|521.43|889.56|907.35|1410.99|1428.78|-1052.10| +2452155|48746|2452269|10733|91506|728572|5191|42210|1234|969670|2867|31933|47|5|2|4|263|209|62|17.09|25.29|7.83|1082.52|485.46|1059.58|1567.98|3.39|145.63|407.34|339.83|343.22|747.17|750.56|-719.75| +2452155|48746|2452189|10625|91506|728572|5191|42210|1234|969670|2867|31933|25|19|17|4|294|209|27|15.96|17.71|7.08|287.01|191.16|430.92|478.17|9.94|66.90|138.51|124.26|134.20|262.77|272.71|-306.66| +2452155|48746|2452236|12587|91506|728572|5191|42210|1234|969670|2867|31933|7|5|2|1|53|209|72|4.47|11.62|2.32|669.60|167.04|321.84|836.64|6.68|0.00|225.36|167.04|173.72|392.40|399.08|-154.80| +2452155|48746|2452205|8725|91506|728572|5191|42210|1234|969670|2867|31933|11|9|1|2|263|209|38|31.60|88.16|83.75|167.58|3182.50|1200.80|3350.08|159.12|0.00|1373.32|3182.50|3341.62|4555.82|4714.94|1981.70| +2452155|48746|2452158|11307|91506|728572|5191|42210|1234|969670|2867|31933|39|5|6|4|222|209|80|48.86|80.61|16.12|5159.20|1289.60|3908.80|6448.80|51.58|0.00|1612.00|1289.60|1341.18|2901.60|2953.18|-2619.20| +2452155|48746|2452162|11483|91506|728572|5191|42210|1234|969670|2867|31933|57|1|16|3|240|209|78|39.94|111.43|6.68|8170.50|521.04|3115.32|8691.54|18.96|250.09|173.16|270.95|289.91|444.11|463.07|-2844.37| +2452155|48746|2452241|9715|91506|728572|5191|42210|1234|969670|2867|31933|23|17|16|1|83|209|90|51.80|63.71|52.24|1032.30|4701.60|4662.00|5733.90|141.04|0.00|1376.10|4701.60|4842.64|6077.70|6218.74|39.60| +2452155|48746|2452172|13935|91506|728572|5191|42210|1234|969670|2867|31933|47|5|12|1|60|209|22|40.30|97.12|11.65|1880.34|256.30|886.60|2136.64|5.94|107.64|1004.08|148.66|154.60|1152.74|1158.68|-737.94| +2450964|45797|2451054|4408|59605|1866133|661|39589|22619|524532|1526|29254|32|22|17|2|147|210|95|76.35|123.68|79.15|4230.35|7519.25|7253.25|11749.60|225.57|0.00|3642.30|7519.25|7744.82|11161.55|11387.12|266.00| +2450964|45797|2450975|6908|59605|1866133|661|39589|22619|524532|1526|29254|26|16|4|4|17|210|63|46.30|76.85|44.57|2033.64|2807.91|2916.90|4841.55|112.31|0.00|483.84|2807.91|2920.22|3291.75|3404.06|-108.99| +2450964|45797|2450980|5674|59605|1866133|661|39589|22619|524532|1526|29254|58|4|13|1|139|210|75|37.53|88.57|46.94|3122.25|3520.50|2814.75|6642.75|35.20|0.00|398.25|3520.50|3555.70|3918.75|3953.95|705.75| +2450964|45797|2451060|760|59605|1866133|661|39589|22619|524532|1526|29254|16|25|17|1|168|210|57|55.98|79.49|55.64|1359.45|3171.48|3190.86|4530.93|159.84|888.01|1223.22|2283.47|2443.31|3506.69|3666.53|-907.39| +2450964|45797|2450974|7744|59605|1866133|661|39589|22619|524532|1526|29254|49|26|20|4|295|210|79|22.10|33.59|12.42|1672.43|981.18|1745.90|2653.61|68.68|0.00|451.09|981.18|1049.86|1432.27|1500.95|-764.72| +2450964|45797|2451073|9775|59605|1866133|661|39589|22619|524532|1526|29254|49|4|11|4|177|210|58|74.77|132.34|39.70|5373.12|2302.60|4336.66|7675.72|161.18|0.00|537.08|2302.60|2463.78|2839.68|3000.86|-2034.06| +2450964|45797|2451013|332|59605|1866133|661|39589|22619|524532|1526|29254|7|22|3|1|22|210|30|51.07|150.14|63.05|2612.70|1891.50|1532.10|4504.20|170.23|0.00|1125.90|1891.50|2061.73|3017.40|3187.63|359.40| +2450964|45797|2450968|6097|59605|1866133|661|39589|22619|524532|1526|29254|20|20|11|1|41|210|23|37.19|69.54|14.60|1263.62|335.80|855.37|1599.42|30.22|0.00|31.97|335.80|366.02|367.77|397.99|-519.57| +2450964|45797|2450971|6652|59605|1866133|661|39589|22619|524532|1526|29254|13|7|11|3|25|210|12|14.27|41.95|38.59|40.32|463.08|171.24|503.40|27.78|0.00|246.60|463.08|490.86|709.68|737.46|291.84| +2450964|45797|2451051|1646|59605|1866133|661|39589|22619|524532|1526|29254|26|28|6|2|265|210|54|98.03|137.24|112.53|1334.34|6076.62|5293.62|7410.96|10.93|5529.72|2000.70|546.90|557.83|2547.60|2558.53|-4746.72| +2451875|49718|2451917|16817|80624|361363|5655|6394|780|714156|164|1656|37|19|1|2|163|211|63|45.57|80.65|46.77|2134.44|2946.51|2870.91|5080.95|88.39|0.00|1168.02|2946.51|3034.90|4114.53|4202.92|75.60| +2451875|49718|2451937|10469|80624|361363|5655|6394|780|714156|164|1656|5|14|6|4|39|211|72|95.61|174.01|135.72|2756.88|9771.84|6883.92|12528.72|390.87|0.00|2505.60|9771.84|10162.71|12277.44|12668.31|2887.92| +2451875|49718|2451969|16489|80624|361363|5655|6394|780|714156|164|1656|35|17|20|1|222|211|63|59.65|143.16|34.35|6855.03|2164.05|3757.95|9019.08|129.84|0.00|1081.71|2164.05|2293.89|3245.76|3375.60|-1593.90| +2451875|49718|2451951|7925|80624|361363|5655|6394|780|714156|164|1656|20|20|14|4|177|211|47|24.85|34.29|21.60|596.43|1015.20|1167.95|1611.63|40.60|0.00|692.78|1015.20|1055.80|1707.98|1748.58|-152.75| +2451875|49718|2451878|13333|80624|361363|5655|6394|780|714156|164|1656|13|5|9|5|231|211|51|65.67|195.03|39.00|7957.53|1989.00|3349.17|9946.53|39.78|0.00|994.50|1989.00|2028.78|2983.50|3023.28|-1360.17| +2451875|49718|2451904|1753|80624|361363|5655|6394|780|714156|164|1656|47|8|3|4|142|211|18|61.73|159.26|41.40|2121.48|745.20|1111.14|2866.68|7.15|387.50|1232.64|357.70|364.85|1590.34|1597.49|-753.44| +2451875|49718|2451965|7082|80624|361363|5655|6394|780|714156|164|1656|56|11|10|4|41|211|20|74.68|144.87|46.35|1970.40|927.00|1493.60|2897.40|27.81|0.00|1129.80|927.00|954.81|2056.80|2084.61|-566.60| +2451875|49718|2451913|6073|80624|361363|5655|6394|780|714156|164|1656|44|23|8|2|143|211|78|98.90|241.31|62.74|13928.46|4893.72|7714.20|18822.18|9.78|3914.97|6775.86|978.75|988.53|7754.61|7764.39|-6735.45| +2451875|49718|2451891|13319|80624|361363|5655|6394|780|714156|164|1656|50|11|16|1|246|211|50|28.54|73.06|66.48|329.00|3324.00|1427.00|3653.00|33.24|0.00|0.00|3324.00|3357.24|3324.00|3357.24|1897.00| +2451875|49718|2451929|4439|80624|361363|5655|6394|780|714156|164|1656|50|1|3|5|150|211|15|58.46|131.53|109.16|335.55|1637.40|876.90|1972.95|98.24|0.00|532.65|1637.40|1735.64|2170.05|2268.29|760.50| +2451875|49718|2451917|15668|80624|361363|5655|6394|780|714156|164|1656|59|11|5|1|40|211|28|91.88|187.43|138.69|1364.72|3883.32|2572.64|5248.04|38.83|0.00|419.72|3883.32|3922.15|4303.04|4341.87|1310.68| +2451875|49718|2451972|8405|80624|361363|5655|6394|780|714156|164|1656|14|19|5|5|25|211|94|41.81|83.20|34.11|4614.46|3206.34|3930.14|7820.80|134.66|961.90|2346.24|2244.44|2379.10|4590.68|4725.34|-1685.70| +2451875|49718|2451986|4676|80624|361363|5655|6394|780|714156|164|1656|31|19|17|1|248|211|14|22.69|27.00|5.13|306.18|71.82|317.66|378.00|5.74|0.00|162.54|71.82|77.56|234.36|240.10|-245.84| +2451875|49718|2451882|14183|80624|361363|5655|6394|780|714156|164|1656|20|7|4|2|110|211|30|90.74|134.29|56.40|2336.70|1692.00|2722.20|4028.70|51.60|659.88|40.20|1032.12|1083.72|1072.32|1123.92|-1690.08| +2451875|49718|2451913|15967|80624|361363|5655|6394|780|714156|164|1656|29|7|16|3|70|211|14|44.08|52.01|35.36|233.10|495.04|617.12|728.14|15.09|193.06|364.00|301.98|317.07|665.98|681.07|-315.14| +2451371|35770|2451402|12584|82053|668385|5117|17896|92315|1656520|2092|20824|19|13|14|1|118|212|94|26.83|44.53|20.48|2260.70|1925.12|2522.02|4185.82|154.00|0.00|41.36|1925.12|2079.12|1966.48|2120.48|-596.90| +2451371|35770|2451448|17810|82053|668385|5117|17896|92315|1656520|2092|20824|28|1|20|4|5|212|57|48.88|70.38|45.04|1444.38|2567.28|2786.16|4011.66|0.00|0.00|160.17|2567.28|2567.28|2727.45|2727.45|-218.88| +2451371|35770|2451412|5389|82053|668385|5117|17896|92315|1656520|2092|20824|13|7|8|2|129|212|93|66.74|125.47|76.53|4551.42|7117.29|6206.82|11668.71|213.51|0.00|5017.35|7117.29|7330.80|12134.64|12348.15|910.47| +2451371|35770|2451447|16420|82053|668385|5117|17896|92315|1656520|2092|20824|38|19|10|5|70|212|40|17.63|21.50|6.45|602.00|258.00|705.20|860.00|5.41|149.64|275.20|108.36|113.77|383.56|388.97|-596.84| +2451371|35770|2451409|6934|82053|668385|5117|17896|92315|1656520|2092|20824|10|28|1|5|296|212|57|39.81|111.46|18.94|5273.64|1079.58|2269.17|6353.22|97.16|0.00|2604.33|1079.58|1176.74|3683.91|3781.07|-1189.59| +2451371|35770|2451382|12100|82053|668385|5117|17896|92315|1656520|2092|20824|31|19|1|5|66|212|3|37.81|101.33|21.27|240.18|63.81|113.43|303.99|0.00|0.00|69.90|63.81|63.81|133.71|133.71|-49.62| +2451371|35770|2451396|3499|82053|668385|5117|17896|92315|1656520|2092|20824|16|28|11|2|33|212|63|69.34|81.82|11.45|4433.31|721.35|4368.42|5154.66|50.49|0.00|2009.70|721.35|771.84|2731.05|2781.54|-3647.07| +2451371|35770|2451462|14896|82053|668385|5117|17896|92315|1656520|2092|20824|31|4|13|5|130|212|86|31.87|92.74|64.91|2393.38|5582.26|2740.82|7975.64|279.11|0.00|3907.84|5582.26|5861.37|9490.10|9769.21|2841.44| +2451371|35770|2451415|10630|82053|668385|5117|17896|92315|1656520|2092|20824|10|8|17|1|295|212|65|59.70|151.63|109.17|2759.90|7096.05|3880.50|9855.95|212.88|0.00|2069.60|7096.05|7308.93|9165.65|9378.53|3215.55| +2451371|35770|2451396|9901|82053|668385|5117|17896|92315|1656520|2092|20824|26|14|9|5|64|212|30|75.66|116.51|110.68|174.90|3320.40|2269.80|3495.30|232.42|0.00|1188.30|3320.40|3552.82|4508.70|4741.12|1050.60| +2451371|35770|2451464|937|82053|668385|5117|17896|92315|1656520|2092|20824|46|25|5|2|26|212|48|80.29|146.93|49.95|4655.04|2397.60|3853.92|7052.64|191.80|0.00|2468.16|2397.60|2589.40|4865.76|5057.56|-1456.32| +2451371|35770|2451423|2848|82053|668385|5117|17896|92315|1656520|2092|20824|16|14|10|2|254|212|25|25.48|39.74|39.74|0.00|993.50|637.00|993.50|9.93|0.00|377.50|993.50|1003.43|1371.00|1380.93|356.50| +2452387|36817|2452424|6481|70776|1387620|2864|44108|67531|474670|640|41884|7|18|11|1|121|213|37|69.96|149.71|104.79|1662.04|3877.23|2588.52|5539.27|310.17|0.00|221.26|3877.23|4187.40|4098.49|4408.66|1288.71| +2452387|36817|2452395|1464|70776|1387620|2864|44108|67531|474670|640|41884|9|13|19|1|163|213|22|62.37|84.82|78.88|130.68|1735.36|1372.14|1866.04|17.35|0.00|522.28|1735.36|1752.71|2257.64|2274.99|363.22| +2452387|36817|2452448|2106|70776|1387620|2864|44108|67531|474670|640|41884|36|1|4|2|190|213|44|24.23|36.34|13.80|991.76|607.20|1066.12|1598.96|42.50|0.00|655.16|607.20|649.70|1262.36|1304.86|-458.92| +2452387|36817|2452498|14265|70776|1387620|2864|44108|67531|474670|640|41884|39|19|10|3|213|213|19|15.01|24.61|9.35|289.94|177.65|285.19|467.59|3.55|0.00|215.08|177.65|181.20|392.73|396.28|-107.54| +2452387|36817|2452467|16725|70776|1387620|2864|44108|67531|474670|640|41884|42|25|5|4|296|213|90|87.44|111.04|77.72|2998.80|6994.80|7869.60|9993.60|279.79|0.00|299.70|6994.80|7274.59|7294.50|7574.29|-874.80| +2452387|36817|2452444|8142|70776|1387620|2864|44108|67531|474670|640|41884|1|18|7|5|3|213|90|94.27|214.93|186.98|2515.50|16828.20|8484.30|19343.70|336.56|0.00|1740.60|16828.20|17164.76|18568.80|18905.36|8343.90| +2452387|36817|2452497|10296|70776|1387620|2864|44108|67531|474670|640|41884|1|18|7|5|40|213|86|9.19|11.02|7.16|331.96|615.76|790.34|947.72|30.78|0.00|425.70|615.76|646.54|1041.46|1072.24|-174.58| +2452387|36817|2452398|7333|70776|1387620|2864|44108|67531|474670|640|41884|12|24|1|2|173|213|98|9.29|27.59|22.07|540.96|2162.86|910.42|2703.82|43.25|0.00|134.26|2162.86|2206.11|2297.12|2340.37|1252.44| +2452387|36817|2452504|12331|70776|1387620|2864|44108|67531|474670|640|41884|51|24|5|3|284|213|25|24.88|61.95|13.62|1208.25|340.50|622.00|1548.75|12.53|27.24|758.75|313.26|325.79|1072.01|1084.54|-308.74| +2452387|36817|2452412|11157|70776|1387620|2864|44108|67531|474670|640|41884|37|30|14|3|169|213|21|45.87|79.81|38.30|871.71|804.30|963.27|1676.01|0.00|0.00|0.00|804.30|804.30|804.30|804.30|-158.97| +2452387|36817|2452425|187|70776|1387620|2864|44108|67531|474670|640|41884|43|25|4|5|226|213|79|60.10|60.70|52.20|671.50|4123.80|4747.90|4795.30|0.00|0.00|287.56|4123.80|4123.80|4411.36|4411.36|-624.10| +2452387|36817|2452438|2227|70776|1387620|2864|44108|67531|474670|640|41884|33|24|17|2|200|213|65|93.40|178.39|165.90|811.85|10783.50|6071.00|11595.35|970.51|0.00|579.15|10783.50|11754.01|11362.65|12333.16|4712.50| +2451216|41881|2451336|9046|61706|32008|1157|36706|89162|1223361|3353|14902|58|22|4|1|228|214|7|45.13|116.43|17.46|692.79|122.22|315.91|815.01|5.96|47.66|220.01|74.56|80.52|294.57|300.53|-241.35| +2451216|41881|2451217|13150|61706|32008|1157|36706|89162|1223361|3353|14902|44|16|2|3|161|214|76|43.98|55.85|10.61|3438.24|806.36|3342.48|4244.60|0.00|0.00|1018.40|806.36|806.36|1824.76|1824.76|-2536.12| +2451216|41881|2451236|6409|61706|32008|1157|36706|89162|1223361|3353|14902|50|25|10|2|94|214|11|44.41|107.91|34.53|807.18|379.83|488.51|1187.01|34.18|0.00|439.12|379.83|414.01|818.95|853.13|-108.68| +2451216|41881|2451290|6265|61706|32008|1157|36706|89162|1223361|3353|14902|2|7|13|5|197|214|87|15.47|32.48|13.64|1639.08|1186.68|1345.89|2825.76|47.46|0.00|1384.17|1186.68|1234.14|2570.85|2618.31|-159.21| +2451216|41881|2451225|12536|61706|32008|1157|36706|89162|1223361|3353|14902|13|13|4|4|201|214|77|82.87|110.21|16.53|7213.36|1272.81|6380.99|8486.17|19.09|318.20|3139.29|954.61|973.70|4093.90|4112.99|-5426.38| +2451216|41881|2451235|5336|61706|32008|1157|36706|89162|1223361|3353|14902|32|8|3|3|36|214|20|17.70|17.70|17.52|3.60|350.40|354.00|354.00|21.02|0.00|67.20|350.40|371.42|417.60|438.62|-3.60| +2451216|41881|2451262|4238|61706|32008|1157|36706|89162|1223361|3353|14902|34|28|15|2|29|214|80|91.64|104.46|68.94|2841.60|5515.20|7331.20|8356.80|50.18|4798.22|2339.20|716.98|767.16|3056.18|3106.36|-6614.22| +2451216|41881|2451287|6752|61706|32008|1157|36706|89162|1223361|3353|14902|4|20|12|5|261|214|85|81.95|110.63|64.16|3949.95|5453.60|6965.75|9403.55|218.14|0.00|3761.25|5453.60|5671.74|9214.85|9432.99|-1512.15| +2451216|41881|2451317|9763|61706|32008|1157|36706|89162|1223361|3353|14902|40|13|13|1|242|214|52|4.34|8.68|8.15|27.56|423.80|225.68|451.36|16.95|0.00|62.92|423.80|440.75|486.72|503.67|198.12| +2451216|41881|2451250|14956|61706|32008|1157|36706|89162|1223361|3353|14902|4|8|17|2|34|214|19|59.19|87.00|66.99|380.19|1272.81|1124.61|1653.00|38.18|0.00|181.83|1272.81|1310.99|1454.64|1492.82|148.20| +2451216|41881|2451244|8137|61706|32008|1157|36706|89162|1223361|3353|14902|49|16|10|4|298|214|91|16.74|19.58|18.40|107.38|1674.40|1523.34|1781.78|0.00|0.00|587.86|1674.40|1674.40|2262.26|2262.26|151.06| +2451216|41881|2451240|9382|61706|32008|1157|36706|89162|1223361|3353|14902|19|22|11|5|249|214|25|28.99|72.76|29.83|1073.25|745.75|724.75|1819.00|67.11|0.00|345.50|745.75|812.86|1091.25|1158.36|21.00| +2451081|29021|2451155|16514|20426|1416243|2107|16371|20426|1416243|2107|16371|46|19|6|2|148|215|42|25.91|58.29|38.47|832.44|1615.74|1088.22|2448.18|73.67|387.77|1052.52|1227.97|1301.64|2280.49|2354.16|139.75| +2451081|29021|2451141|11026|20426|1416243|2107|16371|20426|1416243|2107|16371|31|16|19|4|159|215|70|13.47|31.51|17.01|1015.00|1190.70|942.90|2205.70|83.34|0.00|396.90|1190.70|1274.04|1587.60|1670.94|247.80| +2451081|29021|2451194|16214|20426|1416243|2107|16371|20426|1416243|2107|16371|46|1|18|5|89|215|14|51.95|71.69|27.95|612.36|391.30|727.30|1003.66|0.00|0.00|401.38|391.30|391.30|792.68|792.68|-336.00| +2451081|29021|2451200|10604|20426|1416243|2107|16371|20426|1416243|2107|16371|37|28|11|2|3|215|59|74.50|201.89|8.07|11435.38|476.13|4395.50|11911.51|6.42|404.71|1309.80|71.42|77.84|1381.22|1387.64|-4324.08| +2451081|29021|2451185|13666|20426|1416243|2107|16371|20426|1416243|2107|16371|10|8|17|1|258|215|2|11.29|20.77|2.28|36.98|4.56|22.58|41.54|0.00|4.42|17.86|0.14|0.14|18.00|18.00|-22.44| +2451081|29021|2451136|7846|20426|1416243|2107|16371|20426|1416243|2107|16371|2|2|2|2|189|215|33|61.71|104.28|21.89|2718.87|722.37|2036.43|3441.24|14.44|0.00|1169.85|722.37|736.81|1892.22|1906.66|-1314.06| +2451081|29021|2451138|15178|20426|1416243|2107|16371|20426|1416243|2107|16371|20|22|6|5|204|215|96|68.07|203.52|130.25|7033.92|12504.00|6534.72|19537.92|0.00|0.00|4884.48|12504.00|12504.00|17388.48|17388.48|5969.28| +2451081|29021|2451190|2788|20426|1416243|2107|16371|20426|1416243|2107|16371|19|7|5|4|263|215|32|73.61|133.23|97.25|1151.36|3112.00|2355.52|4263.36|62.24|0.00|1662.40|3112.00|3174.24|4774.40|4836.64|756.48| +2451081|29021|2451143|4256|20426|1416243|2107|16371|20426|1416243|2107|16371|7|22|7|2|110|215|53|93.72|136.83|105.35|1668.44|5583.55|4967.16|7251.99|502.51|0.00|289.91|5583.55|6086.06|5873.46|6375.97|616.39| +2451081|29021|2451085|3842|20426|1416243|2107|16371|20426|1416243|2107|16371|28|20|4|5|276|215|97|93.13|104.30|95.95|809.95|9307.15|9033.61|10117.10|93.07|0.00|3136.01|9307.15|9400.22|12443.16|12536.23|273.54| +2451081|29021|2451174|9346|20426|1416243|2107|16371|20426|1416243|2107|16371|38|19|3|1|139|215|56|6.94|13.39|8.83|255.36|494.48|388.64|749.84|34.61|0.00|134.96|494.48|529.09|629.44|664.05|105.84| +2451081|29021|2451131|9055|20426|1416243|2107|16371|20426|1416243|2107|16371|38|2|14|1|188|215|92|69.67|165.11|140.34|2278.84|12911.28|6409.64|15190.12|258.22|0.00|5467.56|12911.28|13169.50|18378.84|18637.06|6501.64| +2451081|29021|2451096|14240|20426|1416243|2107|16371|20426|1416243|2107|16371|20|22|13|4|165|215|72|29.72|37.15|13.00|1738.80|936.00|2139.84|2674.80|34.07|449.28|347.04|486.72|520.79|833.76|867.83|-1653.12| +2451081|29021|2451196|5428|20426|1416243|2107|16371|20426|1416243|2107|16371|49|1|7|3|222|215|88|82.28|168.67|48.91|10538.88|4304.08|7240.64|14842.96|387.36|0.00|890.56|4304.08|4691.44|5194.64|5582.00|-2936.56| +2451081|29021|2451123|17641|20426|1416243|2107|16371|20426|1416243|2107|16371|19|1|2|3|71|215|83|52.56|62.54|55.03|623.33|4567.49|4362.48|5190.82|411.07|0.00|882.29|4567.49|4978.56|5449.78|5860.85|205.01| +2451081|29021|2451161|4789|20426|1416243|2107|16371|20426|1416243|2107|16371|22|26|9|1|276|215|43|43.30|74.47|69.25|224.46|2977.75|1861.90|3202.21|208.44|0.00|416.24|2977.75|3186.19|3393.99|3602.43|1115.85| +2451781|49361|2451893|788|6891|1122371|2651|30931|20085|598518|218|13298|23|20|7|1|195|216|36|76.28|181.54|49.01|4771.08|1764.36|2746.08|6535.44|70.39|758.67|2548.80|1005.69|1076.08|3554.49|3624.88|-1740.39| +2451781|49361|2451808|15359|6891|1122371|2651|30931|20085|598518|218|13298|35|23|16|2|1|216|54|53.01|65.20|47.59|950.94|2569.86|2862.54|3520.80|128.49|0.00|1337.58|2569.86|2698.35|3907.44|4035.93|-292.68| +2451781|49361|2451799|6056|6891|1122371|2651|30931|20085|598518|218|13298|47|1|5|4|223|216|53|23.42|69.32|41.59|1469.69|2204.27|1241.26|3673.96|44.08|0.00|1653.07|2204.27|2248.35|3857.34|3901.42|963.01| +2451781|49361|2451874|16652|6891|1122371|2651|30931|20085|598518|218|13298|31|29|14|3|162|216|31|19.78|25.31|3.54|674.87|109.74|613.18|784.61|0.00|0.00|290.16|109.74|109.74|399.90|399.90|-503.44| +2451781|49361|2451893|3449|6891|1122371|2651|30931|20085|598518|218|13298|49|20|17|3|89|216|2|11.70|28.08|24.14|7.88|48.28|23.40|56.16|2.89|0.00|8.98|48.28|51.17|57.26|60.15|24.88| +2451781|49361|2451816|15367|6891|1122371|2651|30931|20085|598518|218|13298|26|19|3|1|72|216|61|23.34|65.11|44.27|1271.24|2700.47|1423.74|3971.71|22.68|432.07|1071.77|2268.40|2291.08|3340.17|3362.85|844.66| +2451781|49361|2451813|16273|6891|1122371|2651|30931|20085|598518|218|13298|19|2|4|3|135|216|57|38.88|43.15|36.24|393.87|2065.68|2216.16|2459.55|165.25|0.00|885.21|2065.68|2230.93|2950.89|3116.14|-150.48| +2451781|49361|2451858|3241|6891|1122371|2651|30931|20085|598518|218|13298|20|5|2|4|166|216|51|95.61|96.56|57.93|1970.13|2954.43|4876.11|4924.56|147.72|0.00|2068.05|2954.43|3102.15|5022.48|5170.20|-1921.68| +2451781|49361|2451866|8605|6891|1122371|2651|30931|20085|598518|218|13298|37|26|6|1|66|216|21|27.44|72.71|0.72|1511.79|15.12|576.24|1526.91|0.00|0.00|320.46|15.12|15.12|335.58|335.58|-561.12| +2451781|49361|2451866|3092|6891|1122371|2651|30931|20085|598518|218|13298|25|11|9|4|22|216|32|71.97|141.06|23.98|3746.56|767.36|2303.04|4513.92|23.02|0.00|902.72|767.36|790.38|1670.08|1693.10|-1535.68| +2451781|49361|2451869|8834|6891|1122371|2651|30931|20085|598518|218|13298|55|25|8|1|81|216|43|92.85|116.99|60.83|2414.88|2615.69|3992.55|5030.57|26.15|0.00|1659.80|2615.69|2641.84|4275.49|4301.64|-1376.86| +2452122|70195|2452232|15251|97683|639519|1802|36085|34178|1753141|4087|15170|33|25|16|2|188|217|17|43.22|46.24|32.83|227.97|558.11|734.74|786.08|23.44|223.24|117.81|334.87|358.31|452.68|476.12|-399.87| +2452122|70195|2452193|2881|97683|639519|1802|36085|34178|1753141|4087|15170|7|13|20|3|39|217|86|20.33|60.38|16.30|3790.88|1401.80|1748.38|5192.68|14.01|0.00|1193.68|1401.80|1415.81|2595.48|2609.49|-346.58| +2452122|70195|2452149|17931|97683|639519|1802|36085|34178|1753141|4087|15170|59|5|13|3|183|217|2|99.31|260.19|114.48|291.42|228.96|198.62|520.38|6.86|0.00|124.88|228.96|235.82|353.84|360.70|30.34| +2452122|70195|2452173|14421|97683|639519|1802|36085|34178|1753141|4087|15170|25|27|12|5|271|217|14|57.78|66.44|61.12|74.48|855.68|808.92|930.16|17.11|0.00|353.36|855.68|872.79|1209.04|1226.15|46.76| +2452122|70195|2452183|1237|97683|639519|1802|36085|34178|1753141|4087|15170|51|3|11|2|246|217|9|97.69|100.62|52.32|434.70|470.88|879.21|905.58|22.08|155.39|217.26|315.49|337.57|532.75|554.83|-563.72| +2452122|70195|2452242|12181|97683|639519|1802|36085|34178|1753141|4087|15170|35|15|15|5|225|217|62|69.68|117.06|88.96|1742.20|5515.52|4320.16|7257.72|165.46|0.00|1959.20|5515.52|5680.98|7474.72|7640.18|1195.36| +2452122|70195|2452126|14651|97683|639519|1802|36085|34178|1753141|4087|15170|55|9|2|4|176|217|28|89.83|141.03|46.53|2646.00|1302.84|2515.24|3948.84|0.00|925.01|1421.56|377.83|377.83|1799.39|1799.39|-2137.41| +2452122|70195|2452241|6967|97683|639519|1802|36085|34178|1753141|4087|15170|9|5|18|3|191|217|70|71.91|95.64|1.91|6561.10|133.70|5033.70|6694.80|9.35|0.00|2275.70|133.70|143.05|2409.40|2418.75|-4900.00| +2452122|70195|2452153|3689|97683|639519|1802|36085|34178|1753141|4087|15170|11|1|9|4|189|217|8|31.48|88.14|31.73|451.28|253.84|251.84|705.12|15.23|0.00|0.00|253.84|269.07|253.84|269.07|2.00| +2452122|70195|2452206|4187|97683|639519|1802|36085|34178|1753141|4087|15170|49|27|20|4|258|217|89|77.76|143.07|38.62|9296.05|3437.18|6920.64|12733.23|103.11|0.00|3309.91|3437.18|3540.29|6747.09|6850.20|-3483.46| +2452122|70195|2452234|7003|97683|639519|1802|36085|34178|1753141|4087|15170|37|23|15|3|291|217|8|16.01|25.13|17.08|64.40|136.64|128.08|201.04|6.83|0.00|26.08|136.64|143.47|162.72|169.55|8.56| +2452122|70195|2452205|7787|97683|639519|1802|36085|34178|1753141|4087|15170|43|9|15|5|58|217|15|13.67|21.46|7.94|202.80|119.10|205.05|321.90|7.14|0.00|125.40|119.10|126.24|244.50|251.64|-85.95| +2452122|70195|2452166|10617|97683|639519|1802|36085|34178|1753141|4087|15170|55|7|10|3|89|217|39|66.76|159.55|122.85|1431.30|4791.15|2603.64|6222.45|287.46|0.00|746.46|4791.15|5078.61|5537.61|5825.07|2187.51| +2452122|70195|2452217|17171|97683|639519|1802|36085|34178|1753141|4087|15170|51|23|11|5|159|217|94|88.82|197.18|177.46|1853.68|16681.24|8349.08|18534.92|1167.68|0.00|1482.38|16681.24|17848.92|18163.62|19331.30|8332.16| +2452122|70195|2452175|11777|97683|639519|1802|36085|34178|1753141|4087|15170|5|1|9|2|265|217|80|18.98|42.13|17.27|1988.80|1381.60|1518.40|3370.40|69.08|0.00|1010.40|1381.60|1450.68|2392.00|2461.08|-136.80| +2452122|70195|2452209|8445|97683|639519|1802|36085|34178|1753141|4087|15170|25|23|18|4|120|217|26|2.33|5.00|1.50|91.00|39.00|60.58|130.00|3.12|0.00|58.50|39.00|42.12|97.50|100.62|-21.58| +2450824|44982|2450830|8773|98989|59812|5435|9811|24834|1263106|6680|35456|26|13|13|1|122|218|100|62.60|150.86|39.22|11164.00|3922.00|6260.00|15086.00|235.32|0.00|6788.00|3922.00|4157.32|10710.00|10945.32|-2338.00| +2450824|44982|2450843|1915|98989|59812|5435|9811|24834|1263106|6680|35456|4|4|13|5|280|218|83|20.21|48.09|20.67|2275.86|1715.61|1677.43|3991.47|154.40|0.00|1875.80|1715.61|1870.01|3591.41|3745.81|38.18| +2450824|44982|2450865|12454|98989|59812|5435|9811|24834|1263106|6680|35456|44|10|2|5|68|218|58|74.77|186.17|33.51|8854.28|1943.58|4336.66|10797.86|155.48|0.00|1403.60|1943.58|2099.06|3347.18|3502.66|-2393.08| +2450824|44982|2450925|4142|98989|59812|5435|9811|24834|1263106|6680|35456|13|26|11|1|238|218|62|27.66|73.29|64.49|545.60|3998.38|1714.92|4543.98|279.88|0.00|181.66|3998.38|4278.26|4180.04|4459.92|2283.46| +2450824|44982|2450854|11935|98989|59812|5435|9811|24834|1263106|6680|35456|13|1|5|4|133|218|7|42.79|127.51|79.05|339.22|553.35|299.53|892.57|27.66|0.00|80.29|553.35|581.01|633.64|661.30|253.82| +2450824|44982|2450908|1351|98989|59812|5435|9811|24834|1263106|6680|35456|40|8|6|5|174|218|67|58.09|99.91|19.98|5355.31|1338.66|3892.03|6693.97|0.00|0.00|2677.32|1338.66|1338.66|4015.98|4015.98|-2553.37| +2450824|44982|2450854|631|98989|59812|5435|9811|24834|1263106|6680|35456|2|26|19|1|186|218|98|85.14|195.82|41.12|15160.60|4029.76|8343.72|19190.36|0.00|0.00|3645.60|4029.76|4029.76|7675.36|7675.36|-4313.96| +2450824|44982|2450941|2054|98989|59812|5435|9811|24834|1263106|6680|35456|32|1|5|4|193|218|51|77.07|227.35|88.66|7073.19|4521.66|3930.57|11594.85|406.94|0.00|1391.28|4521.66|4928.60|5912.94|6319.88|591.09| +2450824|44982|2450846|15386|98989|59812|5435|9811|24834|1263106|6680|35456|26|4|19|3|74|218|9|37.78|110.31|11.03|893.52|99.27|340.02|992.79|0.08|96.29|476.46|2.98|3.06|479.44|479.52|-337.04| +2450824|44982|2450941|12907|98989|59812|5435|9811|24834|1263106|6680|35456|13|2|17|2|107|218|91|71.19|181.53|157.93|2147.60|14371.63|6478.29|16519.23|574.86|0.00|2146.69|14371.63|14946.49|16518.32|17093.18|7893.34| +2450824|44982|2450842|16292|98989|59812|5435|9811|24834|1263106|6680|35456|4|2|4|3|285|218|80|1.54|2.80|2.21|47.20|176.80|123.20|224.00|8.84|0.00|89.60|176.80|185.64|266.40|275.24|53.60| +2450824|44982|2450875|10417|98989|59812|5435|9811|24834|1263106|6680|35456|43|26|3|5|207|218|21|54.69|121.41|74.06|994.35|1555.26|1148.49|2549.61|0.00|0.00|203.91|1555.26|1555.26|1759.17|1759.17|406.77| +2450824|44982|2450910|6310|98989|59812|5435|9811|24834|1263106|6680|35456|22|10|20|4|71|218|27|61.26|69.83|10.47|1602.72|282.69|1654.02|1885.41|25.44|0.00|773.01|282.69|308.13|1055.70|1081.14|-1371.33| +2450824|44982|2450943|6640|98989|59812|5435|9811|24834|1263106|6680|35456|28|8|2|4|206|218|37|36.62|82.02|12.30|2579.64|455.10|1354.94|3034.74|22.75|0.00|303.40|455.10|477.85|758.50|781.25|-899.84| +2451898|70214|2452015|6937|43474|261941|4855|32569|63971|1609936|5664|12578|23|8|11|4|90|219|31|26.25|64.31|6.43|1794.28|199.33|813.75|1993.61|0.00|123.58|498.17|75.75|75.75|573.92|573.92|-738.00| +2451898|70214|2451930|629|43474|261941|4855|32569|63971|1609936|5664|12578|20|8|12|1|10|219|66|42.18|57.36|32.12|1665.84|2119.92|2783.88|3785.76|169.59|0.00|113.52|2119.92|2289.51|2233.44|2403.03|-663.96| +2451898|70214|2451914|8869|43474|261941|4855|32569|63971|1609936|5664|12578|37|8|1|5|136|219|48|50.17|69.73|46.02|1138.08|2208.96|2408.16|3347.04|66.26|0.00|669.12|2208.96|2275.22|2878.08|2944.34|-199.20| +2451898|70214|2451903|7394|43474|261941|4855|32569|63971|1609936|5664|12578|13|5|6|2|185|219|62|90.08|217.09|71.63|9018.52|4441.06|5584.96|13459.58|355.28|0.00|1615.10|4441.06|4796.34|6056.16|6411.44|-1143.90| +2451898|70214|2452008|2525|43474|261941|4855|32569|63971|1609936|5664|12578|55|5|1|4|244|219|80|36.64|90.13|60.38|2380.00|4830.40|2931.20|7210.40|53.13|3767.71|2018.40|1062.69|1115.82|3081.09|3134.22|-1868.51| +2451898|70214|2452007|4915|43474|261941|4855|32569|63971|1609936|5664|12578|20|7|17|3|156|219|55|95.35|104.88|23.07|4499.55|1268.85|5244.25|5768.40|99.35|164.95|634.15|1103.90|1203.25|1738.05|1837.40|-4140.35| +2451898|70214|2451985|4424|43474|261941|4855|32569|63971|1609936|5664|12578|59|1|12|3|214|219|93|62.20|152.39|91.43|5669.28|8502.99|5784.60|14172.27|510.17|0.00|2550.99|8502.99|9013.16|11053.98|11564.15|2718.39| +2451898|70214|2451956|2311|43474|261941|4855|32569|63971|1609936|5664|12578|29|19|5|2|182|219|59|90.22|104.65|14.65|5310.00|864.35|5322.98|6174.35|77.79|0.00|61.36|864.35|942.14|925.71|1003.50|-4458.63| +2451898|70214|2451904|15494|43474|261941|4855|32569|63971|1609936|5664|12578|5|2|8|2|6|219|58|56.39|167.47|159.09|486.04|9227.22|3270.62|9713.26|0.00|0.00|3690.54|9227.22|9227.22|12917.76|12917.76|5956.60| +2451136|32500|2451243|6877|16158|1270450|4295|26140|33669|1866205|2075|27920|14|22|19|1|10|220|97|73.07|181.94|96.42|8295.44|9352.74|7087.79|17648.18|66.40|2712.29|881.73|6640.45|6706.85|7522.18|7588.58|-447.34| +2451136|32500|2451241|938|16158|1270450|4295|26140|33669|1866205|2075|27920|55|16|3|4|245|220|29|49.46|56.87|48.90|231.13|1418.10|1434.34|1649.23|127.62|0.00|461.68|1418.10|1545.72|1879.78|2007.40|-16.24| +2451136|32500|2451147|16069|16158|1270450|4295|26140|33669|1866205|2075|27920|8|4|8|1|184|220|58|77.30|132.18|83.27|2836.78|4829.66|4483.40|7666.44|434.66|0.00|2223.14|4829.66|5264.32|7052.80|7487.46|346.26| +2451136|32500|2451141|11408|16158|1270450|4295|26140|33669|1866205|2075|27920|44|7|12|3|44|220|8|73.59|200.90|110.49|723.28|883.92|588.72|1607.20|8.83|0.00|353.52|883.92|892.75|1237.44|1246.27|295.20| +2451136|32500|2451193|13888|16158|1270450|4295|26140|33669|1866205|2075|27920|7|1|15|2|209|220|73|97.53|139.46|41.83|7126.99|3053.59|7119.69|10180.58|28.09|244.28|2849.92|2809.31|2837.40|5659.23|5687.32|-4310.38| +2451136|32500|2451255|8383|16158|1270450|4295|26140|33669|1866205|2075|27920|40|14|8|5|82|220|21|91.81|194.63|126.50|1430.73|2656.50|1928.01|4087.23|239.08|0.00|694.68|2656.50|2895.58|3351.18|3590.26|728.49| +2451136|32500|2451177|9206|16158|1270450|4295|26140|33669|1866205|2075|27920|56|2|8|4|117|220|11|84.86|229.97|190.87|430.10|2099.57|933.46|2529.67|20.99|0.00|1163.58|2099.57|2120.56|3263.15|3284.14|1166.11| +2451136|32500|2451221|8218|16158|1270450|4295|26140|33669|1866205|2075|27920|43|4|11|2|43|220|3|10.58|18.51|8.69|29.46|26.07|31.74|55.53|2.34|0.00|11.64|26.07|28.41|37.71|40.05|-5.67| +2451136|32500|2451253|356|16158|1270450|4295|26140|33669|1866205|2075|27920|14|7|17|3|179|220|8|81.57|172.92|124.50|387.36|996.00|652.56|1383.36|0.00|747.00|55.28|249.00|249.00|304.28|304.28|-403.56| +2451136|32500|2451218|4072|16158|1270450|4295|26140|33669|1866205|2075|27920|55|16|20|1|252|220|69|96.29|139.62|44.67|6551.55|3082.23|6644.01|9633.78|123.28|0.00|1155.75|3082.23|3205.51|4237.98|4361.26|-3561.78| +2451136|32500|2451211|3298|16158|1270450|4295|26140|33669|1866205|2075|27920|20|10|14|4|104|220|25|12.57|33.81|15.89|448.00|397.25|314.25|845.25|11.91|0.00|92.75|397.25|409.16|490.00|501.91|83.00| +2451136|32500|2451250|2932|16158|1270450|4295|26140|33669|1866205|2075|27920|10|22|15|1|134|220|69|5.02|12.24|4.28|549.24|295.32|346.38|844.56|23.62|0.00|33.12|295.32|318.94|328.44|352.06|-51.06| +2451136|32500|2451198|14060|16158|1270450|4295|26140|33669|1866205|2075|27920|38|25|13|1|66|220|82|13.91|33.94|17.64|1336.60|1446.48|1140.62|2783.08|28.92|0.00|166.46|1446.48|1475.40|1612.94|1641.86|305.86| +2451136|32500|2451219|15973|16158|1270450|4295|26140|33669|1866205|2075|27920|20|7|7|5|140|220|73|75.62|126.28|0.00|9218.44|0.00|5520.26|9218.44|0.00|0.00|3502.54|0.00|0.00|3502.54|3502.54|-5520.26| +2451136|32500|2451233|6145|16158|1270450|4295|26140|33669|1866205|2075|27920|14|22|6|5|84|220|19|22.09|64.72|18.76|873.24|356.44|419.71|1229.68|10.69|0.00|49.02|356.44|367.13|405.46|416.15|-63.27| +2450953|71066|2451023|15542|54768|232852|536|15820|44205|365992|5225|46909|4|16|8|4|50|221|70|36.95|88.68|12.41|5338.90|868.70|2586.50|6207.60|8.68|0.00|2358.30|868.70|877.38|3227.00|3235.68|-1717.80| +2450953|71066|2451002|2768|54768|232852|536|15820|44205|365992|5225|46909|56|4|17|1|285|221|3|8.80|14.78|13.59|3.57|40.77|26.40|44.34|2.44|0.00|3.09|40.77|43.21|43.86|46.30|14.37| +2450953|71066|2451037|13843|54768|232852|536|15820|44205|365992|5225|46909|34|1|14|2|280|221|42|83.54|132.82|114.22|781.20|4797.24|3508.68|5578.44|239.86|0.00|1617.42|4797.24|5037.10|6414.66|6654.52|1288.56| +2450953|71066|2450968|1540|54768|232852|536|15820|44205|365992|5225|46909|16|26|12|5|68|221|26|30.34|38.83|15.92|595.66|413.92|788.84|1009.58|20.69|0.00|474.50|413.92|434.61|888.42|909.11|-374.92| +2450953|71066|2450978|7214|54768|232852|536|15820|44205|365992|5225|46909|2|8|8|1|284|221|27|85.69|197.94|170.22|748.44|4595.94|2313.63|5344.38|165.45|2527.76|1175.58|2068.18|2233.63|3243.76|3409.21|-245.45| +2450953|71066|2450975|2488|54768|232852|536|15820|44205|365992|5225|46909|10|13|11|5|259|221|38|37.64|47.05|28.23|715.16|1072.74|1430.32|1787.90|64.36|0.00|643.34|1072.74|1137.10|1716.08|1780.44|-357.58| +2450953|71066|2451055|12775|54768|232852|536|15820|44205|365992|5225|46909|22|19|1|2|283|221|41|46.57|121.54|114.24|299.30|4683.84|1909.37|4983.14|46.83|0.00|896.67|4683.84|4730.67|5580.51|5627.34|2774.47| +2450953|71066|2451010|16795|54768|232852|536|15820|44205|365992|5225|46909|31|14|13|5|97|221|11|61.45|106.92|59.87|517.55|658.57|675.95|1176.12|5.59|98.78|0.00|559.79|565.38|559.79|565.38|-116.16| +2450953|71066|2450964|4852|54768|232852|536|15820|44205|365992|5225|46909|7|1|8|5|292|221|16|41.18|123.12|84.95|610.72|1359.20|658.88|1969.92|95.14|0.00|374.24|1359.20|1454.34|1733.44|1828.58|700.32| +2450953|71066|2450966|12751|54768|232852|536|15820|44205|365992|5225|46909|58|7|8|2|16|221|12|23.16|23.16|10.19|155.64|122.28|277.92|277.92|4.89|0.00|130.56|122.28|127.17|252.84|257.73|-155.64| +2450953|71066|2451036|1123|54768|232852|536|15820|44205|365992|5225|46909|56|13|5|3|121|221|57|83.39|235.99|181.71|3093.96|10357.47|4753.23|13451.43|932.17|0.00|4707.63|10357.47|11289.64|15065.10|15997.27|5604.24| +2450953|71066|2451025|2492|54768|232852|536|15820|44205|365992|5225|46909|22|4|4|2|253|221|50|3.49|3.52|3.13|19.50|156.50|174.50|176.00|4.69|0.00|40.00|156.50|161.19|196.50|201.19|-18.00| +2450953|71066|2450970|17768|54768|232852|536|15820|44205|365992|5225|46909|31|16|12|3|11|221|30|16.14|44.06|36.12|238.20|1083.60|484.20|1321.80|32.50|0.00|79.20|1083.60|1116.10|1162.80|1195.30|599.40| +2450953|71066|2451020|9068|54768|232852|536|15820|44205|365992|5225|46909|43|10|13|2|274|221|9|70.96|116.37|77.96|345.69|701.64|638.64|1047.33|2.52|449.04|94.23|252.60|255.12|346.83|349.35|-386.04| +2450949|67947|2450962|5377|74462|1364862|2462|7662|2780|1896747|3262|35262|46|22|13|4|285|222|96|89.23|132.95|71.79|5871.36|6891.84|8566.08|12763.20|68.91|0.00|3956.16|6891.84|6960.75|10848.00|10916.91|-1674.24| +2450949|67947|2450982|16360|74462|1364862|2462|7662|2780|1896747|3262|35262|56|4|7|1|75|222|57|72.77|92.41|32.34|3423.99|1843.38|4147.89|5267.37|51.43|1271.93|1895.82|571.45|622.88|2467.27|2518.70|-3576.44| +2450949|67947|2450963|5606|74462|1364862|2462|7662|2780|1896747|3262|35262|25|20|6|1|206|222|24|55.65|97.38|21.42|1823.04|514.08|1335.60|2337.12|35.98|0.00|93.36|514.08|550.06|607.44|643.42|-821.52| +2450949|67947|2450977|5506|74462|1364862|2462|7662|2780|1896747|3262|35262|50|14|7|4|298|222|88|56.33|159.97|118.37|3660.80|10416.56|4957.04|14077.36|208.33|0.00|1688.72|10416.56|10624.89|12105.28|12313.61|5459.52| +2450949|67947|2451069|13070|74462|1364862|2462|7662|2780|1896747|3262|35262|49|4|18|1|281|222|44|51.24|111.19|7.78|4550.04|342.32|2254.56|4892.36|10.26|0.00|1076.24|342.32|352.58|1418.56|1428.82|-1912.24| +2450949|67947|2451055|15439|74462|1364862|2462|7662|2780|1896747|3262|35262|55|26|3|4|108|222|94|81.85|218.53|126.74|8628.26|11913.56|7693.90|20541.82|1029.33|476.54|0.00|11437.02|12466.35|11437.02|12466.35|3743.12| +2450949|67947|2451068|8954|74462|1364862|2462|7662|2780|1896747|3262|35262|38|8|7|2|237|222|3|1.92|5.39|2.04|10.05|6.12|5.76|16.17|0.18|0.00|0.15|6.12|6.30|6.27|6.45|0.36| +2450949|67947|2451066|3949|74462|1364862|2462|7662|2780|1896747|3262|35262|52|16|1|3|195|222|95|57.94|132.10|73.97|5522.35|7027.15|5504.30|12549.50|562.17|0.00|627.00|7027.15|7589.32|7654.15|8216.32|1522.85| +2450949|67947|2450976|248|74462|1364862|2462|7662|2780|1896747|3262|35262|16|28|8|4|77|222|12|91.90|136.01|59.84|914.04|718.08|1102.80|1632.12|50.26|0.00|750.72|718.08|768.34|1468.80|1519.06|-384.72| +2450949|67947|2451010|2306|74462|1364862|2462|7662|2780|1896747|3262|35262|38|26|5|4|70|222|66|66.09|178.44|91.00|5771.04|6006.00|4361.94|11777.04|300.30|0.00|5770.38|6006.00|6306.30|11776.38|12076.68|1644.06| +2450949|67947|2450978|3931|74462|1364862|2462|7662|2780|1896747|3262|35262|4|4|6|4|162|222|25|9.35|25.99|15.85|253.50|396.25|233.75|649.75|3.96|0.00|0.00|396.25|400.21|396.25|400.21|162.50| +2450949|67947|2451036|6742|74462|1364862|2462|7662|2780|1896747|3262|35262|37|16|12|2|55|222|95|74.69|188.96|43.46|13822.50|4128.70|7095.55|17951.20|165.14|0.00|6821.00|4128.70|4293.84|10949.70|11114.84|-2966.85| +2450949|67947|2450957|9134|74462|1364862|2462|7662|2780|1896747|3262|35262|58|8|20|3|103|222|39|53.11|67.98|37.38|1193.40|1457.82|2071.29|2651.22|131.20|0.00|503.49|1457.82|1589.02|1961.31|2092.51|-613.47| +2450949|67947|2451022|11654|74462|1364862|2462|7662|2780|1896747|3262|35262|37|28|11|2|213|222|90|94.21|242.11|106.52|12203.10|9586.80|8478.90|21789.90|287.60|0.00|10023.30|9586.80|9874.40|19610.10|19897.70|1107.90| +2450949|67947|2451033|5072|74462|1364862|2462|7662|2780|1896747|3262|35262|25|26|8|4|10|222|56|29.73|75.51|62.67|719.04|3509.52|1664.88|4228.56|70.19|0.00|338.24|3509.52|3579.71|3847.76|3917.95|1844.64| +2450949|67947|2451042|10532|74462|1364862|2462|7662|2780|1896747|3262|35262|14|8|19|1|130|222|63|51.70|93.06|30.70|3928.68|1934.10|3257.10|5862.78|135.38|0.00|878.85|1934.10|2069.48|2812.95|2948.33|-1323.00| +2451542|45365|2451629|422|88704|329621|4938|41455|32598|914383|2206|45123|58|25|16|4|82|223|86|7.54|13.27|11.41|159.96|981.26|648.44|1141.22|0.00|0.00|204.68|981.26|981.26|1185.94|1185.94|332.82| +2451542|45365|2451543|9278|88704|329621|4938|41455|32598|914383|2206|45123|1|1|1|1|211|223|59|75.35|177.82|106.69|4196.67|6294.71|4445.65|10491.38|125.89|0.00|3357.10|6294.71|6420.60|9651.81|9777.70|1849.06| +2451542|45365|2451612|4646|88704|329621|4938|41455|32598|914383|2206|45123|34|25|5|5|282|223|87|25.08|56.93|27.89|2526.48|2426.43|2181.96|4952.91|194.11|0.00|1089.24|2426.43|2620.54|3515.67|3709.78|244.47| +2451542|45365|2451633|11608|88704|329621|4938|41455|32598|914383|2206|45123|26|7|4|4|147|223|13|73.94|113.86|52.37|799.37|680.81|961.22|1480.18|40.84|0.00|207.22|680.81|721.65|888.03|928.87|-280.41| +2451542|45365|2451599|17461|88704|329621|4938|41455|32598|914383|2206|45123|31|20|16|4|289|223|56|34.35|89.31|73.23|900.48|4100.88|1923.60|5001.36|82.01|0.00|549.92|4100.88|4182.89|4650.80|4732.81|2177.28| +2451542|45365|2451554|5857|88704|329621|4938|41455|32598|914383|2206|45123|49|22|9|1|153|223|67|78.93|197.32|27.62|11369.90|1850.54|5288.31|13220.44|148.04|0.00|6610.22|1850.54|1998.58|8460.76|8608.80|-3437.77| +2451542|45365|2451549|15812|88704|329621|4938|41455|32598|914383|2206|45123|25|22|1|3|242|223|28|23.20|67.74|51.48|455.28|1441.44|649.60|1896.72|0.00|0.00|834.40|1441.44|1441.44|2275.84|2275.84|791.84| +2451542|45365|2451639|3493|88704|329621|4938|41455|32598|914383|2206|45123|52|26|1|3|208|223|47|22.29|53.94|45.30|406.08|2129.10|1047.63|2535.18|21.29|0.00|75.67|2129.10|2150.39|2204.77|2226.06|1081.47| +2451542|45365|2451562|3200|88704|329621|4938|41455|32598|914383|2206|45123|50|19|7|1|165|223|71|65.31|112.98|4.51|7701.37|320.21|4637.01|8021.58|12.80|0.00|560.90|320.21|333.01|881.11|893.91|-4316.80| +2451542|45365|2451573|2515|88704|329621|4938|41455|32598|914383|2206|45123|40|26|7|2|300|223|87|57.83|73.44|20.56|4600.56|1788.72|5031.21|6389.28|37.20|858.58|0.00|930.14|967.34|930.14|967.34|-4101.07| +2451542|45365|2451658|4184|88704|329621|4938|41455|32598|914383|2206|45123|58|14|13|1|129|223|91|99.13|188.34|148.78|3599.96|13538.98|9020.83|17138.94|270.77|0.00|7198.10|13538.98|13809.75|20737.08|21007.85|4518.15| +2451542|45365|2451594|15793|88704|329621|4938|41455|32598|914383|2206|45123|32|1|2|1|297|223|100|3.38|6.08|4.49|159.00|449.00|338.00|608.00|17.96|0.00|261.00|449.00|466.96|710.00|727.96|111.00| +2451542|45365|2451635|11407|88704|329621|4938|41455|32598|914383|2206|45123|28|10|14|2|143|223|91|95.17|208.42|177.15|2845.57|16120.65|8660.47|18966.22|1450.85|0.00|5120.57|16120.65|17571.50|21241.22|22692.07|7460.18| +2451542|45365|2451648|7561|88704|329621|4938|41455|32598|914383|2206|45123|32|13|10|3|23|223|1|84.99|130.03|54.61|75.42|54.61|84.99|130.03|1.09|0.00|10.40|54.61|55.70|65.01|66.10|-30.38| +2451542|45365|2451637|6880|88704|329621|4938|41455|32598|914383|2206|45123|49|22|19|1|213|223|100|68.36|114.84|96.46|1838.00|9646.00|6836.00|11484.00|289.38|0.00|1148.00|9646.00|9935.38|10794.00|11083.38|2810.00| +2451546|46817|2451647|14759|88309|264092|6722|19305|53288|378728|3548|49731|49|13|19|1|200|224|81|46.63|99.78|56.87|3475.71|4606.47|3777.03|8082.18|92.12|0.00|1130.76|4606.47|4698.59|5737.23|5829.35|829.44| +2451546|46817|2451604|8503|88309|264092|6722|19305|53288|378728|3548|49731|2|19|8|1|214|224|87|15.64|32.84|17.07|1371.99|1485.09|1360.68|2857.08|103.95|0.00|85.26|1485.09|1589.04|1570.35|1674.30|124.41| +2451546|46817|2451580|4574|88309|264092|6722|19305|53288|378728|3548|49731|1|29|9|3|34|224|76|78.93|142.07|46.88|7234.44|3562.88|5998.68|10797.32|35.62|0.00|2375.00|3562.88|3598.50|5937.88|5973.50|-2435.80| +2451546|46817|2451549|5846|88309|264092|6722|19305|53288|378728|3548|49731|26|13|7|5|173|224|26|44.30|97.90|78.32|509.08|2036.32|1151.80|2545.40|22.39|916.34|559.78|1119.98|1142.37|1679.76|1702.15|-31.82| +2451546|46817|2451607|12041|88309|264092|6722|19305|53288|378728|3548|49731|32|1|12|3|124|224|76|28.44|33.55|12.07|1632.48|917.32|2161.44|2549.80|82.55|0.00|535.04|917.32|999.87|1452.36|1534.91|-1244.12| +2451546|46817|2451597|2942|88309|264092|6722|19305|53288|378728|3548|49731|43|2|12|2|242|224|14|55.68|96.88|2.90|1315.72|40.60|779.52|1356.32|0.00|15.02|447.58|25.58|25.58|473.16|473.16|-753.94| +2451546|46817|2451645|8489|88309|264092|6722|19305|53288|378728|3548|49731|17|29|11|2|243|224|64|89.51|138.74|98.50|2575.36|6304.00|5728.64|8879.36|126.08|0.00|1331.84|6304.00|6430.08|7635.84|7761.92|575.36| +2451546|46817|2451637|7796|88309|264092|6722|19305|53288|378728|3548|49731|23|20|20|1|285|224|52|65.60|144.32|23.09|6303.96|1200.68|3411.20|7504.64|14.40|912.51|2701.40|288.17|302.57|2989.57|3003.97|-3123.03| +2451546|46817|2451655|10849|88309|264092|6722|19305|53288|378728|3548|49731|13|19|16|4|125|224|11|22.88|41.41|33.54|86.57|368.94|251.68|455.51|3.68|0.00|40.92|368.94|372.62|409.86|413.54|117.26| +2451546|46817|2451601|1099|88309|264092|6722|19305|53288|378728|3548|49731|44|26|2|3|114|224|55|25.18|30.71|23.03|422.40|1266.65|1384.90|1689.05|101.33|0.00|793.65|1266.65|1367.98|2060.30|2161.63|-118.25| +2451546|46817|2451662|10675|88309|264092|6722|19305|53288|378728|3548|49731|7|23|3|1|262|224|63|60.39|68.84|38.55|1908.27|2428.65|3804.57|4336.92|18.45|1505.76|129.78|922.89|941.34|1052.67|1071.12|-2881.68| +2451546|46817|2451614|4202|88309|264092|6722|19305|53288|378728|3548|49731|26|25|2|4|183|224|19|35.16|56.95|3.98|1006.43|75.62|668.04|1082.05|2.26|0.00|540.93|75.62|77.88|616.55|618.81|-592.42| +2451546|46817|2451580|2111|88309|264092|6722|19305|53288|378728|3548|49731|20|2|20|1|192|224|57|12.02|30.77|13.53|982.68|771.21|685.14|1753.89|38.56|0.00|140.22|771.21|809.77|911.43|949.99|86.07| +2451546|46817|2451599|15191|88309|264092|6722|19305|53288|378728|3548|49731|41|1|4|2|257|224|4|69.14|97.48|3.89|374.36|15.56|276.56|389.92|0.62|0.00|140.36|15.56|16.18|155.92|156.54|-261.00| +2451546|46817|2451613|14333|88309|264092|6722|19305|53288|378728|3548|49731|32|5|19|3|104|224|18|79.51|180.48|21.65|2858.94|389.70|1431.18|3248.64|3.15|354.62|389.70|35.08|38.23|424.78|427.93|-1396.10| +2451546|46817|2451603|9935|88309|264092|6722|19305|53288|378728|3548|49731|29|17|16|5|157|224|95|61.95|126.37|20.21|10085.20|1919.95|5885.25|12005.15|153.59|0.00|2641.00|1919.95|2073.54|4560.95|4714.54|-3965.30| +2452552|38910|2452581|1747|39795|929461|1174|46087|78557|892176|6426|49339|54|18|8|5|29|225|70|74.55|122.26|20.78|7103.60|1454.60|5218.50|8558.20|29.09|0.00|3850.70|1454.60|1483.69|5305.30|5334.39|-3763.90| +2452552|38910|2452582|11851|39795|929461|1174|46087|78557|892176|6426|49339|13|25|9|5|151|225|35|35.20|102.78|35.97|2338.35|1258.95|1232.00|3597.30|39.02|478.40|1150.80|780.55|819.57|1931.35|1970.37|-451.45| +2452552|38910|2452651|2425|39795|929461|1174|46087|78557|892176|6426|49339|45|27|5|1|257|225|36|77.46|151.04|6.04|5220.00|217.44|2788.56|5437.44|4.34|0.00|1631.16|217.44|221.78|1848.60|1852.94|-2571.12| +2452552|38910|2452636|10011|39795|929461|1174|46087|78557|892176|6426|49339|51|3|8|4|163|225|42|78.56|87.20|0.00|3662.40|0.00|3299.52|3662.40|0.00|0.00|1428.00|0.00|0.00|1428.00|1428.00|-3299.52| +2452552|38910|2452654|17899|39795|929461|1174|46087|78557|892176|6426|49339|54|9|16|4|165|225|39|88.18|254.84|30.58|8746.14|1192.62|3439.02|9938.76|35.77|0.00|4372.68|1192.62|1228.39|5565.30|5601.07|-2246.40| +2452552|38910|2452616|8655|39795|929461|1174|46087|78557|892176|6426|49339|24|25|11|2|143|225|56|91.80|154.22|103.32|2850.40|5785.92|5140.80|8636.32|405.01|0.00|3799.60|5785.92|6190.93|9585.52|9990.53|645.12| +2452552|38910|2452609|17805|39795|929461|1174|46087|78557|892176|6426|49339|39|21|2|3|217|225|36|72.15|207.79|83.11|4488.48|2991.96|2597.40|7480.44|179.51|0.00|2617.92|2991.96|3171.47|5609.88|5789.39|394.56| +2452552|38910|2452606|14053|39795|929461|1174|46087|78557|892176|6426|49339|39|15|13|3|263|225|82|78.74|225.98|40.67|15195.42|3334.94|6456.68|18530.36|106.71|2000.96|1667.06|1333.98|1440.69|3001.04|3107.75|-5122.70| +2451816|36120|2451882|9601|30282|448195|5661|35174|30282|448195|5661|35174|38|8|7|2|87|226|72|5.26|12.25|10.29|141.12|740.88|378.72|882.00|6.66|74.08|237.60|666.80|673.46|904.40|911.06|288.08| +2451816|36120|2451832|11623|30282|448195|5661|35174|30282|448195|5661|35174|13|1|5|4|230|226|55|88.07|136.50|20.47|6381.65|1125.85|4843.85|7507.50|22.51|0.00|1576.30|1125.85|1148.36|2702.15|2724.66|-3718.00| +2451816|36120|2451902|10673|30282|448195|5661|35174|30282|448195|5661|35174|49|11|17|3|92|226|6|43.28|119.88|116.28|21.60|697.68|259.68|719.28|41.86|0.00|129.42|697.68|739.54|827.10|868.96|438.00| +2451816|36120|2451878|1109|30282|448195|5661|35174|30282|448195|5661|35174|14|19|12|4|226|226|2|81.04|124.80|94.84|59.92|189.68|162.08|249.60|11.38|0.00|107.32|189.68|201.06|297.00|308.38|27.60| +2451816|36120|2451899|4195|30282|448195|5661|35174|30282|448195|5661|35174|20|13|10|1|212|226|34|93.31|150.22|114.16|1226.04|3881.44|3172.54|5107.48|0.00|3143.96|970.36|737.48|737.48|1707.84|1707.84|-2435.06| +2451816|36120|2451831|15185|30282|448195|5661|35174|30282|448195|5661|35174|2|23|2|2|282|226|48|54.98|133.60|52.10|3912.00|2500.80|2639.04|6412.80|125.04|0.00|2052.00|2500.80|2625.84|4552.80|4677.84|-138.24| +2451816|36120|2451922|14006|30282|448195|5661|35174|30282|448195|5661|35174|31|7|2|5|118|226|23|97.08|246.58|226.85|453.79|5217.55|2232.84|5671.34|365.22|0.00|2381.88|5217.55|5582.77|7599.43|7964.65|2984.71| +2451816|36120|2451930|11189|30282|448195|5661|35174|30282|448195|5661|35174|29|11|14|5|16|226|19|83.37|223.43|89.37|2547.14|1698.03|1584.03|4245.17|33.96|0.00|1485.80|1698.03|1731.99|3183.83|3217.79|114.00| +2451816|36120|2451868|13622|30282|448195|5661|35174|30282|448195|5661|35174|11|25|13|3|200|226|5|61.04|153.21|47.49|528.60|237.45|305.20|766.05|18.16|35.61|352.35|201.84|220.00|554.19|572.35|-103.36| +2451816|36120|2451874|6971|30282|448195|5661|35174|30282|448195|5661|35174|5|25|16|1|299|226|40|44.44|74.21|18.55|2226.40|742.00|1777.60|2968.40|42.14|215.18|1306.00|526.82|568.96|1832.82|1874.96|-1250.78| +2451816|36120|2451922|5191|30282|448195|5661|35174|30282|448195|5661|35174|19|8|7|2|177|226|41|84.69|170.22|18.72|6211.50|767.52|3472.29|6979.02|13.50|92.10|2512.07|675.42|688.92|3187.49|3200.99|-2796.87| +2452620|36181|2452625|11053|97306|1095585|1511|31390|97306|1095585|1511|31390|1|7|4|4|214|227|87|82.91|233.80|107.54|10984.62|9355.98|7213.17|20340.60|187.11|0.00|3253.80|9355.98|9543.09|12609.78|12796.89|2142.81| +2452620|36181|2452626|15282|97306|1095585|1511|31390|97306|1095585|1511|31390|7|27|17|5|240|227|71|64.16|186.06|53.95|9379.81|3830.45|4555.36|13210.26|76.60|0.00|2245.73|3830.45|3907.05|6076.18|6152.78|-724.91| +2452620|36181|2452670|1429|97306|1095585|1511|31390|97306|1095585|1511|31390|24|21|3|2|211|227|40|63.46|177.68|63.96|4548.80|2558.40|2538.40|7107.20|102.33|0.00|2771.60|2558.40|2660.73|5330.00|5432.33|20.00| +2452620|36181|2452719|3915|97306|1095585|1511|31390|97306|1095585|1511|31390|31|19|14|3|292|227|4|80.71|230.83|66.94|655.56|267.76|322.84|923.32|13.38|0.00|249.28|267.76|281.14|517.04|530.42|-55.08| +2452620|36181|2452722|15714|97306|1095585|1511|31390|97306|1095585|1511|31390|51|27|19|2|135|227|84|93.12|124.78|81.10|3669.12|6812.40|7822.08|10481.52|408.74|0.00|733.32|6812.40|7221.14|7545.72|7954.46|-1009.68| +2452620|36181|2452695|1632|97306|1095585|1511|31390|97306|1095585|1511|31390|37|24|12|5|268|227|32|9.07|24.57|12.28|393.28|392.96|290.24|786.24|27.50|0.00|94.08|392.96|420.46|487.04|514.54|102.72| +2452620|36181|2452623|3409|97306|1095585|1511|31390|97306|1095585|1511|31390|43|9|11|4|127|227|60|32.62|84.48|43.08|2484.00|2584.80|1957.20|5068.80|180.93|0.00|354.60|2584.80|2765.73|2939.40|3120.33|627.60| +2452620|36181|2452717|483|97306|1095585|1511|31390|97306|1095585|1511|31390|7|6|19|3|157|227|77|92.01|111.33|85.72|1971.97|6600.44|7084.77|8572.41|132.00|0.00|3257.10|6600.44|6732.44|9857.54|9989.54|-484.33| +2451198|35072|2451217|10003|16158|407301|3244|7187|47522|697955|2663|12206|4|7|8|2|106|228|28|78.79|148.91|90.83|1626.24|2543.24|2206.12|4169.48|203.45|0.00|1751.12|2543.24|2746.69|4294.36|4497.81|337.12| +2451198|35072|2451226|12800|16158|407301|3244|7187|47522|697955|2663|12206|20|7|17|3|216|228|69|65.83|99.40|14.91|5829.81|1028.79|4542.27|6858.60|0.00|0.00|342.93|1028.79|1028.79|1371.72|1371.72|-3513.48| +2451198|35072|2451296|14671|16158|407301|3244|7187|47522|697955|2663|12206|2|13|15|3|151|228|52|65.11|110.03|75.92|1773.72|3947.84|3385.72|5721.56|197.39|0.00|2288.52|3947.84|4145.23|6236.36|6433.75|562.12| +2451198|35072|2451298|6337|16158|407301|3244|7187|47522|697955|2663|12206|58|26|8|2|176|228|58|8.49|23.94|10.53|777.78|610.74|492.42|1388.52|8.42|189.32|82.94|421.42|429.84|504.36|512.78|-71.00| +2451198|35072|2451287|14449|16158|407301|3244|7187|47522|697955|2663|12206|55|16|13|4|215|228|44|97.68|132.84|128.85|175.56|5669.40|4297.92|5844.96|78.23|4365.43|2863.96|1303.97|1382.20|4167.93|4246.16|-2993.95| +2451198|35072|2451203|9064|16158|407301|3244|7187|47522|697955|2663|12206|10|20|9|2|5|228|54|45.70|124.76|64.87|3234.06|3502.98|2467.80|6737.04|280.23|0.00|2020.68|3502.98|3783.21|5523.66|5803.89|1035.18| +2451198|35072|2451271|5128|16158|407301|3244|7187|47522|697955|2663|12206|28|22|2|4|259|228|43|67.76|149.74|11.97|5924.11|514.71|2913.68|6438.82|15.44|0.00|514.71|514.71|530.15|1029.42|1044.86|-2398.97| +2451198|35072|2451275|6554|16158|407301|3244|7187|47522|697955|2663|12206|46|4|19|3|127|228|29|39.95|101.07|63.67|1084.60|1846.43|1158.55|2931.03|18.46|0.00|703.25|1846.43|1864.89|2549.68|2568.14|687.88| +2451198|35072|2451316|4891|16158|407301|3244|7187|47522|697955|2663|12206|26|20|8|5|182|228|58|80.44|209.94|4.19|11933.50|243.02|4665.52|12176.52|0.00|167.68|2434.84|75.34|75.34|2510.18|2510.18|-4590.18| +2451391|77085|2451404|14899|63974|1723896|6218|49340|11478|1393142|6259|46181|28|26|10|4|175|229|73|38.51|84.33|83.48|62.05|6094.04|2811.23|6156.09|426.58|0.00|1661.48|6094.04|6520.62|7755.52|8182.10|3282.81| +2451391|77085|2451415|7621|63974|1723896|6218|49340|11478|1393142|6259|46181|14|14|18|3|248|229|3|59.68|99.06|93.11|17.85|279.33|179.04|297.18|19.55|0.00|56.46|279.33|298.88|335.79|355.34|100.29| +2451391|77085|2451503|2443|63974|1723896|6218|49340|11478|1393142|6259|46181|50|13|18|4|190|229|10|3.88|6.55|3.93|26.20|39.30|38.80|65.50|0.39|0.00|4.50|39.30|39.69|43.80|44.19|0.50| +2451391|77085|2451462|12286|63974|1723896|6218|49340|11478|1393142|6259|46181|49|7|13|4|38|229|49|75.08|141.90|87.97|2642.57|4310.53|3678.92|6953.10|0.00|0.00|1459.71|4310.53|4310.53|5770.24|5770.24|631.61| +2451391|77085|2451480|4054|63974|1723896|6218|49340|11478|1393142|6259|46181|7|20|11|3|18|229|11|96.11|287.36|270.11|189.75|2971.21|1057.21|3160.96|148.56|0.00|979.88|2971.21|3119.77|3951.09|4099.65|1914.00| +2451391|77085|2451435|6860|63974|1723896|6218|49340|11478|1393142|6259|46181|43|26|16|5|237|229|95|17.76|24.33|21.89|231.80|2079.55|1687.20|2311.35|0.00|1726.02|323.00|353.53|353.53|676.53|676.53|-1333.67| +2451391|77085|2451485|5236|63974|1723896|6218|49340|11478|1393142|6259|46181|34|28|12|4|66|229|51|98.43|120.08|104.46|796.62|5327.46|5019.93|6124.08|0.00|5327.46|367.20|0.00|0.00|367.20|367.20|-5019.93| +2451391|77085|2451417|12937|63974|1723896|6218|49340|11478|1393142|6259|46181|56|28|11|2|123|229|38|24.59|35.65|5.70|1138.10|216.60|934.42|1354.70|15.16|0.00|582.16|216.60|231.76|798.76|813.92|-717.82| +2451405|47857|2451419|316|78740|49104|4268|25479|38369|859809|6171|26582|1|14|5|2|209|230|54|96.14|284.57|59.75|12140.28|3226.50|5191.56|15366.78|161.32|0.00|7683.12|3226.50|3387.82|10909.62|11070.94|-1965.06| +2451405|47857|2451495|13213|78740|49104|4268|25479|38369|859809|6171|26582|40|22|3|4|172|230|61|5.66|14.65|11.57|187.88|705.77|345.26|893.65|9.03|592.84|392.84|112.93|121.96|505.77|514.80|-232.33| +2451405|47857|2451517|574|78740|49104|4268|25479|38369|859809|6171|26582|7|8|9|5|144|230|26|68.61|166.03|6.64|4144.14|172.64|1783.86|4316.78|0.29|143.29|863.20|29.35|29.64|892.55|892.84|-1754.51| +2451405|47857|2451483|13726|78740|49104|4268|25479|38369|859809|6171|26582|43|4|11|2|283|230|89|34.24|76.35|37.41|3465.66|3329.49|3047.36|6795.15|199.76|0.00|2581.89|3329.49|3529.25|5911.38|6111.14|282.13| +2451405|47857|2451419|4111|78740|49104|4268|25479|38369|859809|6171|26582|38|7|1|5|9|230|18|44.83|109.38|74.37|630.18|1338.66|806.94|1968.84|0.00|0.00|433.08|1338.66|1338.66|1771.74|1771.74|531.72| +2451405|47857|2451413|7675|78740|49104|4268|25479|38369|859809|6171|26582|43|7|6|5|287|230|65|14.36|34.75|34.40|22.75|2236.00|933.40|2258.75|67.08|0.00|406.25|2236.00|2303.08|2642.25|2709.33|1302.60| +2451405|47857|2451507|14642|78740|49104|4268|25479|38369|859809|6171|26582|58|14|6|1|9|230|31|15.44|35.04|11.56|727.88|358.36|478.64|1086.24|28.66|0.00|119.35|358.36|387.02|477.71|506.37|-120.28| +2451405|47857|2451467|1976|78740|49104|4268|25479|38369|859809|6171|26582|4|26|18|4|243|230|54|75.68|75.68|29.51|2493.18|1593.54|4086.72|4086.72|31.87|0.00|898.56|1593.54|1625.41|2492.10|2523.97|-2493.18| +2451405|47857|2451431|13406|78740|49104|4268|25479|38369|859809|6171|26582|38|22|4|3|235|230|75|78.27|226.98|195.20|2383.50|14640.00|5870.25|17023.50|146.40|7320.00|3915.00|7320.00|7466.40|11235.00|11381.40|1449.75| +2451814|51493|2451924|3847|61990|744483|576|40269|79480|271610|6995|28288|25|13|4|3|222|231|53|98.86|103.80|85.11|990.57|4510.83|5239.58|5501.40|360.86|0.00|1595.30|4510.83|4871.69|6106.13|6466.99|-728.75| +2451814|51493|2451850|4784|61990|744483|576|40269|79480|271610|6995|28288|29|11|20|1|152|231|34|41.17|53.52|4.28|1674.16|145.52|1399.78|1819.68|0.00|29.10|600.44|116.42|116.42|716.86|716.86|-1283.36| +2451814|51493|2451904|14605|61990|744483|576|40269|79480|271610|6995|28288|37|25|14|5|189|231|82|54.69|126.33|25.26|8287.74|2071.32|4484.58|10359.06|41.42|0.00|5075.80|2071.32|2112.74|7147.12|7188.54|-2413.26| +2451814|51493|2451896|8642|61990|744483|576|40269|79480|271610|6995|28288|38|11|5|1|40|231|59|32.02|44.50|5.78|2284.48|341.02|1889.18|2625.50|27.28|0.00|813.61|341.02|368.30|1154.63|1181.91|-1548.16| +2451814|51493|2451829|4133|61990|744483|576|40269|79480|271610|6995|28288|32|14|19|3|274|231|60|84.86|176.50|100.60|4554.00|6036.00|5091.60|10590.00|422.52|0.00|4659.60|6036.00|6458.52|10695.60|11118.12|944.40| +2451814|51493|2451866|7751|61990|744483|576|40269|79480|271610|6995|28288|11|17|2|2|235|231|17|31.86|71.68|68.81|48.79|1169.77|541.62|1218.56|46.79|0.00|316.71|1169.77|1216.56|1486.48|1533.27|628.15| +2451814|51493|2451920|12758|61990|744483|576|40269|79480|271610|6995|28288|53|26|18|5|72|231|27|72.11|111.04|103.26|210.06|2788.02|1946.97|2998.08|0.00|0.00|1139.13|2788.02|2788.02|3927.15|3927.15|841.05| +2451814|51493|2451900|14018|61990|744483|576|40269|79480|271610|6995|28288|35|25|15|2|282|231|17|6.77|10.69|8.33|40.12|141.61|115.09|181.73|2.83|0.00|65.28|141.61|144.44|206.89|209.72|26.52| +2451814|51493|2451886|3221|61990|744483|576|40269|79480|271610|6995|28288|2|1|20|4|118|231|11|39.06|96.86|7.74|980.32|85.14|429.66|1065.46|7.66|0.00|490.05|85.14|92.80|575.19|582.85|-344.52| +2451814|51493|2451905|11972|61990|744483|576|40269|79480|271610|6995|28288|26|19|16|2|81|231|74|53.34|79.47|50.86|2117.14|3763.64|3947.16|5880.78|0.00|0.00|352.24|3763.64|3763.64|4115.88|4115.88|-183.52| +2451814|51493|2451912|4526|61990|744483|576|40269|79480|271610|6995|28288|56|14|7|1|102|231|62|76.46|180.44|151.56|1790.56|9396.72|4740.52|11187.28|751.73|0.00|4026.90|9396.72|10148.45|13423.62|14175.35|4656.20| +2451814|51493|2451872|3752|61990|744483|576|40269|79480|271610|6995|28288|5|5|13|5|250|231|17|86.10|161.86|63.12|1678.58|1073.04|1463.70|2751.62|42.92|0.00|54.91|1073.04|1115.96|1127.95|1170.87|-390.66| +2451814|51493|2451851|16813|61990|744483|576|40269|79480|271610|6995|28288|14|13|9|1|200|231|96|54.30|97.19|69.97|2613.12|6717.12|5212.80|9330.24|403.02|0.00|4198.08|6717.12|7120.14|10915.20|11318.22|1504.32| +2452235|17422|2452334|7507|16976|1523364|599|28987|62379|1506847|4198|33786|15|7|3|4|45|232|23|58.06|152.11|107.99|1014.76|2483.77|1335.38|3498.53|173.86|0.00|839.50|2483.77|2657.63|3323.27|3497.13|1148.39| +2452235|17422|2452350|3991|16976|1523364|599|28987|62379|1506847|4198|33786|33|13|20|5|143|232|77|32.06|82.07|3.28|6066.83|252.56|2468.62|6319.39|15.15|0.00|1642.41|252.56|267.71|1894.97|1910.12|-2216.06| +2452235|17422|2452314|7661|16976|1523364|599|28987|62379|1506847|4198|33786|13|29|3|3|286|232|92|25.11|37.66|35.02|242.88|3221.84|2310.12|3464.72|103.09|1933.10|241.96|1288.74|1391.83|1530.70|1633.79|-1021.38| +2452235|17422|2452236|4573|16976|1523364|599|28987|62379|1506847|4198|33786|47|11|11|3|125|232|55|81.26|223.46|183.23|2212.65|10077.65|4469.30|12290.30|403.10|0.00|3686.65|10077.65|10480.75|13764.30|14167.40|5608.35| +2452235|17422|2452315|16147|16976|1523364|599|28987|62379|1506847|4198|33786|9|7|6|5|77|232|30|6.99|9.99|7.59|72.00|227.70|209.70|299.70|20.49|0.00|101.70|227.70|248.19|329.40|349.89|18.00| +2452235|17422|2452330|14673|16976|1523364|599|28987|62379|1506847|4198|33786|59|17|1|5|262|232|22|31.24|51.54|49.47|45.54|1088.34|687.28|1133.88|32.65|0.00|170.06|1088.34|1120.99|1258.40|1291.05|401.06| +2452235|17422|2452261|5581|16976|1523364|599|28987|62379|1506847|4198|33786|25|21|9|2|65|232|91|93.78|211.94|139.88|6557.46|12729.08|8533.98|19286.54|763.74|0.00|2314.13|12729.08|13492.82|15043.21|15806.95|4195.10| +2452235|17422|2452354|11081|16976|1523364|599|28987|62379|1506847|4198|33786|5|21|9|4|222|232|50|22.47|34.37|23.02|567.50|1151.00|1123.50|1718.50|0.00|0.00|842.00|1151.00|1151.00|1993.00|1993.00|27.50| +2452235|17422|2452325|6547|16976|1523364|599|28987|62379|1506847|4198|33786|35|11|17|5|166|232|85|30.94|42.69|8.96|2867.05|761.60|2629.90|3628.65|0.00|0.00|544.00|761.60|761.60|1305.60|1305.60|-1868.30| +2452235|17422|2452239|10357|16976|1523364|599|28987|62379|1506847|4198|33786|31|15|13|5|240|232|4|25.58|65.99|3.95|248.16|15.80|102.32|263.96|0.94|0.00|23.72|15.80|16.74|39.52|40.46|-86.52| +2452235|17422|2452252|11795|16976|1523364|599|28987|62379|1506847|4198|33786|45|5|20|2|84|232|38|43.54|119.73|81.41|1456.16|3093.58|1654.52|4549.74|185.61|0.00|909.72|3093.58|3279.19|4003.30|4188.91|1439.06| +2452235|17422|2452301|17361|16976|1523364|599|28987|62379|1506847|4198|33786|21|15|12|4|96|232|32|40.46|53.40|40.05|427.20|1281.60|1294.72|1708.80|9.86|294.76|273.28|986.84|996.70|1260.12|1269.98|-307.88| +2452235|17422|2452239|5735|16976|1523364|599|28987|62379|1506847|4198|33786|33|17|18|2|276|232|58|55.26|123.78|59.41|3733.46|3445.78|3205.08|7179.24|310.12|0.00|1866.44|3445.78|3755.90|5312.22|5622.34|240.70| +2452235|17422|2452254|13833|16976|1523364|599|28987|62379|1506847|4198|33786|3|1|20|4|58|232|27|24.18|58.51|14.04|1200.69|379.08|652.86|1579.77|32.41|18.95|426.33|360.13|392.54|786.46|818.87|-292.73| +2452603|45758|2452647|2586|42716|1327666|3463|14060|40442|238388|4957|36754|18|3|15|5|18|233|59|60.50|171.21|83.89|5151.88|4949.51|3569.50|10101.39|395.96|0.00|2626.09|4949.51|5345.47|7575.60|7971.56|1380.01| +2452603|45758|2452690|303|42716|1327666|3463|14060|40442|238388|4957|36754|45|24|1|5|249|233|85|8.05|20.84|11.67|779.45|991.95|684.25|1771.40|29.75|0.00|566.10|991.95|1021.70|1558.05|1587.80|307.70| +2452603|45758|2452701|12247|42716|1327666|3463|14060|40442|238388|4957|36754|51|1|6|3|293|233|38|37.06|55.96|49.24|255.36|1871.12|1408.28|2126.48|56.13|0.00|744.04|1871.12|1927.25|2615.16|2671.29|462.84| +2452603|45758|2452648|4740|42716|1327666|3463|14060|40442|238388|4957|36754|6|6|16|1|229|233|55|16.70|43.42|26.48|931.70|1456.40|918.50|2388.10|96.99|378.66|190.85|1077.74|1174.73|1268.59|1365.58|159.24| +2452603|45758|2452704|13305|42716|1327666|3463|14060|40442|238388|4957|36754|55|9|12|4|243|233|68|59.99|134.37|81.96|3563.88|5573.28|4079.32|9137.16|207.32|2117.84|2832.20|3455.44|3662.76|6287.64|6494.96|-623.88| +2452603|45758|2452613|12924|42716|1327666|3463|14060|40442|238388|4957|36754|45|13|11|5|259|233|88|37.96|38.71|30.58|715.44|2691.04|3340.48|3406.48|80.73|0.00|783.20|2691.04|2771.77|3474.24|3554.97|-649.44| +2452603|45758|2452705|17190|42716|1327666|3463|14060|40442|238388|4957|36754|3|13|2|2|232|233|85|7.28|16.23|2.27|1186.60|192.95|618.80|1379.55|11.57|0.00|357.85|192.95|204.52|550.80|562.37|-425.85| +2452603|45758|2452653|9507|42716|1327666|3463|14060|40442|238388|4957|36754|55|21|14|4|87|233|49|95.25|199.07|65.69|6535.62|3218.81|4667.25|9754.43|193.12|0.00|2438.24|3218.81|3411.93|5657.05|5850.17|-1448.44| +2452603|45758|2452658|16405|42716|1327666|3463|14060|40442|238388|4957|36754|55|21|17|5|253|233|12|73.91|99.03|77.24|261.48|926.88|886.92|1188.36|83.41|0.00|106.92|926.88|1010.29|1033.80|1117.21|39.96| +2452603|45758|2452621|10425|42716|1327666|3463|14060|40442|238388|4957|36754|19|21|19|5|148|233|22|32.29|51.98|23.39|628.98|514.58|710.38|1143.56|15.43|0.00|125.62|514.58|530.01|640.20|655.63|-195.80| +2452277|76744|2452338|14346|94822|171546|1423|5747|94822|171546|1423|5747|54|19|3|1|199|234|98|57.66|105.51|69.63|3516.24|6823.74|5650.68|10339.98|94.16|3684.81|4445.28|3138.93|3233.09|7584.21|7678.37|-2511.75| +2452277|76744|2452374|9831|94822|171546|1423|5747|94822|171546|1423|5747|6|13|4|3|102|234|59|37.77|71.00|49.70|1256.70|2932.30|2228.43|4189.00|29.32|0.00|0.00|2932.30|2961.62|2932.30|2961.62|703.87| +2452277|76744|2452362|16251|94822|171546|1423|5747|94822|171546|1423|5747|21|15|15|5|112|234|82|49.20|74.29|63.88|853.62|5238.16|4034.40|6091.78|366.67|0.00|1279.20|5238.16|5604.83|6517.36|6884.03|1203.76| +2452277|76744|2452325|324|94822|171546|1423|5747|94822|171546|1423|5747|13|3|7|5|10|234|57|53.71|59.08|16.54|2424.78|942.78|3061.47|3367.56|7.35|575.09|471.39|367.69|375.04|839.08|846.43|-2693.78| +2452277|76744|2452285|3859|94822|171546|1423|5747|94822|171546|1423|5747|3|24|2|2|139|234|67|56.34|160.56|89.91|4733.55|6023.97|3774.78|10757.52|16.86|5602.29|2473.64|421.68|438.54|2895.32|2912.18|-3353.10| +2452277|76744|2452296|17634|94822|171546|1423|5747|94822|171546|1423|5747|7|15|9|3|87|234|26|7.75|15.96|12.92|79.04|335.92|201.50|414.96|2.28|278.81|120.12|57.11|59.39|177.23|179.51|-144.39| +2452277|76744|2452342|2844|94822|171546|1423|5747|94822|171546|1423|5747|49|18|13|2|171|234|1|88.77|249.44|122.22|127.22|122.22|88.77|249.44|2.31|89.22|37.41|33.00|35.31|70.41|72.72|-55.77| +2452277|76744|2452379|6037|94822|171546|1423|5747|94822|171546|1423|5747|6|13|14|1|215|234|65|70.94|187.99|146.63|2688.40|9530.95|4611.10|12219.35|190.61|0.00|5864.95|9530.95|9721.56|15395.90|15586.51|4919.85| +2452277|76744|2452358|6978|94822|171546|1423|5747|94822|171546|1423|5747|54|21|15|4|203|234|52|71.86|206.23|123.73|4290.00|6433.96|3736.72|10723.96|119.67|450.37|2466.36|5983.59|6103.26|8449.95|8569.62|2246.87| +2452277|76744|2452288|6240|94822|171546|1423|5747|94822|171546|1423|5747|45|24|11|5|122|234|48|26.08|65.46|11.12|2608.32|533.76|1251.84|3142.08|5.33|0.00|502.56|533.76|539.09|1036.32|1041.65|-718.08| +2452277|76744|2452318|15234|94822|171546|1423|5747|94822|171546|1423|5747|21|18|20|1|222|234|61|8.55|17.18|6.18|671.00|376.98|521.55|1047.98|33.92|0.00|387.35|376.98|410.90|764.33|798.25|-144.57| +2452277|76744|2452344|1974|94822|171546|1423|5747|94822|171546|1423|5747|25|24|1|5|279|234|93|85.89|198.40|172.60|2399.40|16051.80|7987.77|18451.20|234.35|4333.98|4427.73|11717.82|11952.17|16145.55|16379.90|3730.05| +2452277|76744|2452394|14545|94822|171546|1423|5747|94822|171546|1423|5747|33|7|1|1|48|234|100|59.23|129.71|19.45|11026.00|1945.00|5923.00|12971.00|116.70|0.00|2983.00|1945.00|2061.70|4928.00|5044.70|-3978.00| +2452277|76744|2452308|11688|94822|171546|1423|5747|94822|171546|1423|5747|57|27|19|1|66|234|23|62.27|85.93|67.88|415.15|1561.24|1432.21|1976.39|140.51|0.00|572.93|1561.24|1701.75|2134.17|2274.68|129.03| +2452277|76744|2452323|11382|94822|171546|1423|5747|94822|171546|1423|5747|45|27|5|1|234|234|94|97.11|203.93|163.14|3834.26|15335.16|9128.34|19169.42|613.40|0.00|7859.34|15335.16|15948.56|23194.50|23807.90|6206.82| +2452277|76744|2452323|16446|94822|171546|1423|5747|94822|171546|1423|5747|13|15|2|1|20|234|31|77.45|184.33|176.95|228.78|5485.45|2400.95|5714.23|383.98|0.00|1085.62|5485.45|5869.43|6571.07|6955.05|3084.50| +2452378|70945|2452456|9780|90851|1305886|5721|17956|90851|1305886|5721|17956|6|27|15|4|10|235|6|51.14|82.33|3.29|474.24|19.74|306.84|493.98|1.77|0.00|118.50|19.74|21.51|138.24|140.01|-287.10| +2452378|70945|2452440|996|90851|1305886|5721|17956|90851|1305886|5721|17956|9|12|7|3|249|235|90|91.97|136.11|80.30|5022.90|7227.00|8277.30|12249.90|433.62|0.00|1959.30|7227.00|7660.62|9186.30|9619.92|-1050.30| +2452378|70945|2452409|5979|90851|1305886|5721|17956|90851|1305886|5721|17956|42|21|11|1|131|235|16|6.53|17.43|9.76|122.72|156.16|104.48|278.88|0.99|56.21|136.64|99.95|100.94|236.59|237.58|-4.53| +2452378|70945|2452452|13381|90851|1305886|5721|17956|90851|1305886|5721|17956|55|25|8|3|155|235|61|81.20|81.20|12.99|4160.81|792.39|4953.20|4953.20|23.77|0.00|148.23|792.39|816.16|940.62|964.39|-4160.81| +2452378|70945|2452388|6457|90851|1305886|5721|17956|90851|1305886|5721|17956|15|21|1|4|124|235|90|32.07|67.98|57.78|918.00|5200.20|2886.30|6118.20|312.01|0.00|855.90|5200.20|5512.21|6056.10|6368.11|2313.90| +2452378|70945|2452383|13435|90851|1305886|5721|17956|90851|1305886|5721|17956|25|18|8|2|43|235|72|26.55|76.46|49.69|1927.44|3577.68|1911.60|5505.12|250.43|0.00|605.52|3577.68|3828.11|4183.20|4433.63|1666.08| +2452378|70945|2452493|16932|90851|1305886|5721|17956|90851|1305886|5721|17956|3|21|10|1|135|235|50|34.71|93.02|50.23|2139.50|2511.50|1735.50|4651.00|226.03|0.00|1860.00|2511.50|2737.53|4371.50|4597.53|776.00| +2452378|70945|2452454|885|90851|1305886|5721|17956|90851|1305886|5721|17956|25|12|19|2|45|235|54|25.61|38.15|20.98|927.18|1132.92|1382.94|2060.10|67.97|0.00|638.28|1132.92|1200.89|1771.20|1839.17|-250.02| +2452378|70945|2452393|6793|90851|1305886|5721|17956|90851|1305886|5721|17956|6|24|14|1|265|235|92|6.89|18.39|8.45|914.48|777.40|633.88|1691.88|15.54|0.00|439.76|777.40|792.94|1217.16|1232.70|143.52| +2452378|70945|2452490|5689|90851|1305886|5721|17956|90851|1305886|5721|17956|43|13|12|2|52|235|54|72.04|175.77|56.24|6454.62|3036.96|3890.16|9491.58|0.00|0.00|3796.20|3036.96|3036.96|6833.16|6833.16|-853.20| +2452378|70945|2452400|3301|90851|1305886|5721|17956|90851|1305886|5721|17956|43|19|18|1|94|235|16|24.70|67.67|41.95|411.52|671.20|395.20|1082.72|3.28|342.31|465.44|328.89|332.17|794.33|797.61|-66.31| +2452378|70945|2452464|8565|90851|1305886|5721|17956|90851|1305886|5721|17956|24|6|4|3|149|235|62|45.36|82.10|77.99|254.82|4835.38|2812.32|5090.20|77.36|3868.30|1984.62|967.08|1044.44|2951.70|3029.06|-1845.24| +2452378|70945|2452431|10075|90851|1305886|5721|17956|90851|1305886|5721|17956|13|25|19|2|38|235|36|27.94|72.92|16.77|2021.40|603.72|1005.84|2625.12|12.55|289.78|183.60|313.94|326.49|497.54|510.09|-691.90| +2452378|70945|2452466|17647|90851|1305886|5721|17956|90851|1305886|5721|17956|12|7|19|2|95|235|44|4.42|5.39|5.12|11.88|225.28|194.48|237.16|4.50|0.00|94.60|225.28|229.78|319.88|324.38|30.80| +2452378|70945|2452439|12198|90851|1305886|5721|17956|90851|1305886|5721|17956|37|25|20|4|191|235|47|33.77|42.55|21.70|979.95|1019.90|1587.19|1999.85|71.39|0.00|359.55|1019.90|1091.29|1379.45|1450.84|-567.29| +2451154|32770|2451179|17738|79400|1762518|1283|34401|96089|851934|3449|30167|38|2|14|4|151|236|78|23.70|44.31|38.99|414.96|3041.22|1848.60|3456.18|152.06|0.00|1140.36|3041.22|3193.28|4181.58|4333.64|1192.62| +2451154|32770|2451161|2506|79400|1762518|1283|34401|96089|851934|3449|30167|16|8|8|4|277|236|30|68.56|98.72|15.79|2487.90|473.70|2056.80|2961.60|18.94|0.00|1480.80|473.70|492.64|1954.50|1973.44|-1583.10| +2451154|32770|2451257|14533|79400|1762518|1283|34401|96089|851934|3449|30167|44|19|10|5|250|236|57|35.53|49.03|43.14|335.73|2458.98|2025.21|2794.71|8.85|2163.90|55.86|295.08|303.93|350.94|359.79|-1730.13| +2451154|32770|2451263|4411|79400|1762518|1283|34401|96089|851934|3449|30167|52|8|16|2|15|236|48|1.84|3.29|0.19|148.80|9.12|88.32|157.92|0.63|0.00|42.24|9.12|9.75|51.36|51.99|-79.20| +2451154|32770|2451155|16688|79400|1762518|1283|34401|96089|851934|3449|30167|37|22|12|5|95|236|79|10.08|27.21|17.41|774.20|1375.39|796.32|2149.59|47.45|426.37|128.77|949.02|996.47|1077.79|1125.24|152.70| +2451154|32770|2451235|2204|79400|1762518|1283|34401|96089|851934|3449|30167|1|8|9|3|287|236|19|31.34|48.57|25.25|443.08|479.75|595.46|922.83|28.78|0.00|249.09|479.75|508.53|728.84|757.62|-115.71| +2451154|32770|2451227|9704|79400|1762518|1283|34401|96089|851934|3449|30167|56|20|3|3|190|236|55|59.51|113.66|65.92|2625.70|3625.60|3273.05|6251.30|72.51|0.00|1812.80|3625.60|3698.11|5438.40|5510.91|352.55| +2451154|32770|2451199|3196|79400|1762518|1283|34401|96089|851934|3449|30167|28|16|13|2|201|236|23|22.19|62.79|14.44|1112.05|332.12|510.37|1444.17|16.60|0.00|57.73|332.12|348.72|389.85|406.45|-178.25| +2451154|32770|2451187|13849|79400|1762518|1283|34401|96089|851934|3449|30167|10|28|11|2|153|236|19|38.17|64.88|30.49|653.41|579.31|725.23|1232.72|11.58|0.00|24.51|579.31|590.89|603.82|615.40|-145.92| +2451154|32770|2451269|7438|79400|1762518|1283|34401|96089|851934|3449|30167|49|7|3|2|91|236|11|9.12|25.08|6.52|204.16|71.72|100.32|275.88|0.56|15.06|8.25|56.66|57.22|64.91|65.47|-43.66| +2451154|32770|2451211|5269|79400|1762518|1283|34401|96089|851934|3449|30167|16|7|9|2|252|236|94|37.68|85.91|31.78|5088.22|2987.32|3541.92|8075.54|250.03|209.11|322.42|2778.21|3028.24|3100.63|3350.66|-763.71| +2451170|82237|2451288|16742|50611|214826|3488|47750|341|502622|1689|21504|22|22|12|2|12|237|76|92.37|155.18|54.31|7666.12|4127.56|7020.12|11793.68|206.37|0.00|3773.40|4127.56|4333.93|7900.96|8107.33|-2892.56| +2451170|82237|2451273|16600|50611|214826|3488|47750|341|502622|1689|21504|46|10|11|1|208|237|54|87.97|207.60|49.82|8520.12|2690.28|4750.38|11210.40|26.90|0.00|2690.28|2690.28|2717.18|5380.56|5407.46|-2060.10| +2451170|82237|2451198|8197|50611|214826|3488|47750|341|502622|1689|21504|4|13|14|3|215|237|11|38.45|73.43|49.19|266.64|541.09|422.95|807.73|20.02|140.68|104.94|400.41|420.43|505.35|525.37|-22.54| +2451170|82237|2451261|91|50611|214826|3488|47750|341|502622|1689|21504|46|28|16|2|198|237|18|75.92|157.91|56.84|1819.26|1023.12|1366.56|2842.38|30.69|0.00|1364.22|1023.12|1053.81|2387.34|2418.03|-343.44| +2451170|82237|2451242|5275|50611|214826|3488|47750|341|502622|1689|21504|46|7|4|2|237|237|93|95.27|221.02|181.23|3700.47|16854.39|8860.11|20554.86|1179.80|0.00|1233.18|16854.39|18034.19|18087.57|19267.37|7994.28| +2451170|82237|2451245|9044|50611|214826|3488|47750|341|502622|1689|21504|32|25|16|2|75|237|67|58.29|102.59|13.33|5980.42|893.11|3905.43|6873.53|8.93|0.00|1168.48|893.11|902.04|2061.59|2070.52|-3012.32| +2451170|82237|2451277|14462|50611|214826|3488|47750|341|502622|1689|21504|25|1|6|1|196|237|31|38.25|55.08|29.19|802.59|904.89|1185.75|1707.48|81.44|0.00|324.26|904.89|986.33|1229.15|1310.59|-280.86| +2451170|82237|2451202|1192|50611|214826|3488|47750|341|502622|1689|21504|22|1|20|4|28|237|82|72.51|146.47|52.72|7687.50|4323.04|5945.82|12010.54|129.69|0.00|3603.08|4323.04|4452.73|7926.12|8055.81|-1622.78| +2451170|82237|2451199|5662|50611|214826|3488|47750|341|502622|1689|21504|25|1|6|4|122|237|99|27.12|44.74|7.60|3676.86|752.40|2684.88|4429.26|6.47|428.86|1638.45|323.54|330.01|1961.99|1968.46|-2361.34| +2450880|35197|2450925|6727|93992|783301|4163|28672|86563|784312|6552|35814|52|1|18|2|269|238|32|88.69|257.20|54.01|6502.08|1728.32|2838.08|8230.40|86.41|0.00|2715.84|1728.32|1814.73|4444.16|4530.57|-1109.76| +2450880|35197|2450893|14704|93992|783301|4163|28672|86563|784312|6552|35814|1|25|8|5|275|238|66|78.59|190.18|36.13|10167.30|2384.58|5186.94|12551.88|23.84|0.00|3388.44|2384.58|2408.42|5773.02|5796.86|-2802.36| +2450880|35197|2450952|1195|93992|783301|4163|28672|86563|784312|6552|35814|4|7|8|2|16|238|24|94.61|275.31|30.28|5880.72|726.72|2270.64|6607.44|14.53|0.00|1123.20|726.72|741.25|1849.92|1864.45|-1543.92| +2450880|35197|2450901|13372|93992|783301|4163|28672|86563|784312|6552|35814|34|8|12|4|116|238|45|82.52|209.60|132.04|3490.20|5941.80|3713.40|9432.00|237.67|0.00|754.20|5941.80|6179.47|6696.00|6933.67|2228.40| +2450880|35197|2450981|7598|93992|783301|4163|28672|86563|784312|6552|35814|40|19|9|3|37|238|9|45.88|127.54|49.74|700.20|447.66|412.92|1147.86|35.81|0.00|378.72|447.66|483.47|826.38|862.19|34.74| +2450880|35197|2450954|5701|93992|783301|4163|28672|86563|784312|6552|35814|56|1|16|3|233|238|30|54.41|118.61|0.00|3558.30|0.00|1632.30|3558.30|0.00|0.00|1458.90|0.00|0.00|1458.90|1458.90|-1632.30| +2450880|35197|2450938|10651|93992|783301|4163|28672|86563|784312|6552|35814|2|1|20|1|65|238|85|74.00|93.98|1.87|7829.35|158.95|6290.00|7988.30|11.12|0.00|1117.75|158.95|170.07|1276.70|1287.82|-6131.05| +2450880|35197|2450988|4994|93992|783301|4163|28672|86563|784312|6552|35814|52|4|12|3|224|238|75|40.85|91.50|58.56|2470.50|4392.00|3063.75|6862.50|43.92|0.00|137.25|4392.00|4435.92|4529.25|4573.17|1328.25| +2450880|35197|2450896|11864|93992|783301|4163|28672|86563|784312|6552|35814|10|26|12|5|248|238|25|67.40|157.04|28.26|3219.50|706.50|1685.00|3926.00|42.39|0.00|1020.75|706.50|748.89|1727.25|1769.64|-978.50| +2450880|35197|2450912|15254|93992|783301|4163|28672|86563|784312|6552|35814|13|14|2|3|267|238|23|92.64|233.45|170.41|1449.92|3919.43|2130.72|5369.35|274.36|0.00|590.41|3919.43|4193.79|4509.84|4784.20|1788.71| +2450880|35197|2450905|9394|93992|783301|4163|28672|86563|784312|6552|35814|19|14|3|1|106|238|75|66.64|114.62|34.38|6018.00|2578.50|4998.00|8596.50|125.83|1005.61|2320.50|1572.89|1698.72|3893.39|4019.22|-3425.11| +2450880|35197|2450993|868|93992|783301|4163|28672|86563|784312|6552|35814|22|13|12|5|225|238|96|37.95|70.96|44.70|2520.96|4291.20|3643.20|6812.16|102.13|2832.19|2111.04|1459.01|1561.14|3570.05|3672.18|-2184.19| +2450880|35197|2450920|9866|93992|783301|4163|28672|86563|784312|6552|35814|38|4|16|4|72|238|51|95.39|249.92|242.42|382.50|12363.42|4864.89|12745.92|494.53|0.00|1274.49|12363.42|12857.95|13637.91|14132.44|7498.53| +2450880|35197|2450983|15680|93992|783301|4163|28672|86563|784312|6552|35814|31|8|17|1|265|238|100|84.81|218.80|212.23|657.00|21223.00|8481.00|21880.00|1910.07|0.00|8095.00|21223.00|23133.07|29318.00|31228.07|12742.00| +2450880|35197|2450980|15352|93992|783301|4163|28672|86563|784312|6552|35814|40|4|8|3|115|238|8|71.83|198.96|155.18|350.24|1241.44|574.64|1591.68|74.48|0.00|509.28|1241.44|1315.92|1750.72|1825.20|666.80| +2452039|32041|2452109|11627|44039|1561535|5784|26080|95191|729757|2874|46370|21|11|11|3|238|239|58|19.77|55.55|35.55|1160.00|2061.90|1146.66|3221.90|0.00|1195.90|96.28|866.00|866.00|962.28|962.28|-280.66| +2452039|32041|2452114|9169|44039|1561535|5784|26080|95191|729757|2874|46370|33|3|7|5|130|239|34|36.17|107.42|1.07|3615.90|36.38|1229.78|3652.28|0.36|0.00|1132.20|36.38|36.74|1168.58|1168.94|-1193.40| +2452039|32041|2452132|16863|44039|1561535|5784|26080|95191|729757|2874|46370|27|1|18|2|269|239|32|79.79|174.74|17.47|5032.64|559.04|2553.28|5591.68|0.00|0.00|2292.48|559.04|559.04|2851.52|2851.52|-1994.24| +2452039|32041|2452110|17935|44039|1561535|5784|26080|95191|729757|2874|46370|19|17|11|5|209|239|3|30.94|66.83|35.41|94.26|106.23|92.82|200.49|3.18|0.00|10.02|106.23|109.41|116.25|119.43|13.41| +2452039|32041|2452091|12731|44039|1561535|5784|26080|95191|729757|2874|46370|51|7|9|2|125|239|40|5.08|11.07|0.55|420.80|22.00|203.20|442.80|1.54|0.00|4.40|22.00|23.54|26.40|27.94|-181.20| +2452039|32041|2452153|12903|44039|1561535|5784|26080|95191|729757|2874|46370|45|7|18|4|30|239|24|36.08|54.12|46.54|181.92|1116.96|865.92|1298.88|55.84|0.00|363.60|1116.96|1172.80|1480.56|1536.40|251.04| +2452039|32041|2452071|4215|44039|1561535|5784|26080|95191|729757|2874|46370|35|11|14|2|203|239|100|83.61|170.56|15.35|15521.00|1535.00|8361.00|17056.00|76.75|0.00|682.00|1535.00|1611.75|2217.00|2293.75|-6826.00| +2452039|32041|2452106|183|44039|1561535|5784|26080|95191|729757|2874|46370|39|25|16|4|202|239|51|24.44|32.99|10.22|1161.27|521.22|1246.44|1682.49|36.48|0.00|0.00|521.22|557.70|521.22|557.70|-725.22| +2452039|32041|2452086|13207|44039|1561535|5784|26080|95191|729757|2874|46370|57|29|11|5|296|239|39|63.61|159.02|46.11|4403.49|1798.29|2480.79|6201.78|125.88|0.00|2542.41|1798.29|1924.17|4340.70|4466.58|-682.50| +2452039|32041|2452119|17397|44039|1561535|5784|26080|95191|729757|2874|46370|21|15|12|4|66|239|46|32.81|36.41|10.19|1206.12|468.74|1509.26|1674.86|0.00|182.80|518.88|285.94|285.94|804.82|804.82|-1223.32| +2452039|32041|2452072|11947|44039|1561535|5784|26080|95191|729757|2874|46370|17|15|7|4|178|239|91|21.17|38.95|15.96|2092.09|1452.36|1926.47|3544.45|8.27|1176.41|1098.37|275.95|284.22|1374.32|1382.59|-1650.52| +2452039|32041|2452047|1689|44039|1561535|5784|26080|95191|729757|2874|46370|31|9|4|3|189|239|12|67.44|162.53|43.88|1423.80|526.56|809.28|1950.36|21.06|0.00|936.12|526.56|547.62|1462.68|1483.74|-282.72| +2452039|32041|2452094|899|44039|1561535|5784|26080|95191|729757|2874|46370|55|9|12|4|26|239|70|40.12|79.83|44.70|2459.10|3129.00|2808.40|5588.10|219.03|0.00|502.60|3129.00|3348.03|3631.60|3850.63|320.60| +2451137|34370|2451145|5911|38419|1833006|1546|40133|43113|986846|4600|19987|43|1|17|1|275|240|97|3.48|10.23|0.30|963.21|29.10|337.56|992.31|0.61|16.87|277.42|12.23|12.84|289.65|290.26|-325.33| +2451137|34370|2451210|17716|38419|1833006|1546|40133|43113|986846|4600|19987|37|22|15|3|261|240|53|79.22|111.70|92.71|1006.47|4913.63|4198.66|5920.10|343.95|0.00|828.39|4913.63|5257.58|5742.02|6085.97|714.97| +2451137|34370|2451232|11569|38419|1833006|1546|40133|43113|986846|4600|19987|7|22|3|1|179|240|33|53.58|122.16|96.50|846.78|3184.50|1768.14|4031.28|159.22|0.00|1007.82|3184.50|3343.72|4192.32|4351.54|1416.36| +2451137|34370|2451228|5842|38419|1833006|1546|40133|43113|986846|4600|19987|50|1|13|3|270|240|78|41.21|118.68|109.18|741.00|8516.04|3214.38|9257.04|0.00|3832.21|1666.08|4683.83|4683.83|6349.91|6349.91|1469.45| +2451137|34370|2451206|355|38419|1833006|1546|40133|43113|986846|4600|19987|16|1|16|1|226|240|69|16.10|38.47|23.85|1008.78|1645.65|1110.90|2654.43|16.45|0.00|477.48|1645.65|1662.10|2123.13|2139.58|534.75| +2451137|34370|2451249|12709|38419|1833006|1546|40133|43113|986846|4600|19987|19|7|11|3|212|240|5|42.37|88.55|54.01|172.70|270.05|211.85|442.75|24.30|0.00|185.95|270.05|294.35|456.00|480.30|58.20| +2451137|34370|2451138|17336|38419|1833006|1546|40133|43113|986846|4600|19987|58|14|9|2|85|240|65|58.20|91.37|22.84|4454.45|1484.60|3783.00|5939.05|29.69|0.00|949.65|1484.60|1514.29|2434.25|2463.94|-2298.40| +2451137|34370|2451165|7120|38419|1833006|1546|40133|43113|986846|4600|19987|50|20|16|3|3|240|5|51.56|137.14|31.54|528.00|157.70|257.80|685.70|0.00|0.00|226.25|157.70|157.70|383.95|383.95|-100.10| +2451137|34370|2451208|14920|38419|1833006|1546|40133|43113|986846|4600|19987|22|19|15|3|161|240|94|80.84|199.67|149.75|4692.48|14076.50|7598.96|18768.98|844.59|0.00|6380.72|14076.50|14921.09|20457.22|21301.81|6477.54| +2451137|34370|2451231|7126|38419|1833006|1546|40133|43113|986846|4600|19987|16|19|11|5|186|240|48|86.66|200.18|46.04|7398.72|2209.92|4159.68|9608.64|44.19|0.00|576.48|2209.92|2254.11|2786.40|2830.59|-1949.76| +2451137|34370|2451229|16412|38419|1833006|1546|40133|43113|986846|4600|19987|52|14|6|2|295|240|24|76.36|187.84|169.05|450.96|4057.20|1832.64|4508.16|243.43|0.00|540.96|4057.20|4300.63|4598.16|4841.59|2224.56| +2451137|34370|2451180|967|38419|1833006|1546|40133|43113|986846|4600|19987|13|8|20|3|66|240|27|14.05|22.48|18.88|97.20|509.76|379.35|606.96|30.58|0.00|206.28|509.76|540.34|716.04|746.62|130.41| +2451137|34370|2451225|1816|38419|1833006|1546|40133|43113|986846|4600|19987|49|28|15|5|285|240|28|15.19|45.26|24.89|570.36|696.92|425.32|1267.28|20.90|0.00|0.00|696.92|717.82|696.92|717.82|271.60| +2451137|34370|2451143|3286|38419|1833006|1546|40133|43113|986846|4600|19987|43|1|20|4|291|240|12|79.44|223.22|216.52|80.40|2598.24|953.28|2678.64|42.87|1740.82|883.92|857.42|900.29|1741.34|1784.21|-95.86| +2451137|34370|2451179|848|38419|1833006|1546|40133|43113|986846|4600|19987|40|22|6|2|248|240|7|15.94|38.73|21.30|122.01|149.10|111.58|271.11|4.47|0.00|84.00|149.10|153.57|233.10|237.57|37.52| +2451137|34370|2451234|7714|38419|1833006|1546|40133|43113|986846|4600|19987|16|22|5|1|194|240|20|94.36|207.59|6.22|4027.40|124.40|1887.20|4151.80|8.70|0.00|1909.80|124.40|133.10|2034.20|2042.90|-1762.80| +2452534|18532|2452628|7495|77891|673554|2017|48080|51774|724975|669|8332|21|9|9|3|24|241|47|46.71|77.53|44.19|1566.98|2076.93|2195.37|3643.91|62.30|0.00|473.29|2076.93|2139.23|2550.22|2612.52|-118.44| +2452534|18532|2452654|9876|77891|673554|2017|48080|51774|724975|669|8332|24|1|13|2|224|241|83|19.47|41.27|25.17|1336.30|2089.11|1616.01|3425.41|111.14|501.38|1198.52|1587.73|1698.87|2786.25|2897.39|-28.28| +2452534|18532|2452574|16705|77891|673554|2017|48080|51774|724975|669|8332|13|24|7|2|297|241|56|40.72|46.01|22.54|1314.32|1262.24|2280.32|2576.56|75.73|0.00|334.88|1262.24|1337.97|1597.12|1672.85|-1018.08| +2452534|18532|2452633|10015|77891|673554|2017|48080|51774|724975|669|8332|21|15|15|5|142|241|52|12.17|27.62|10.77|876.20|560.04|632.84|1436.24|44.80|0.00|430.56|560.04|604.84|990.60|1035.40|-72.80| +2452534|18532|2452600|4029|77891|673554|2017|48080|51774|724975|669|8332|37|30|11|2|3|241|36|13.21|35.93|22.63|478.80|814.68|475.56|1293.48|24.44|0.00|155.16|814.68|839.12|969.84|994.28|339.12| +2452534|18532|2452629|687|77891|673554|2017|48080|51774|724975|669|8332|54|19|20|3|243|241|12|8.70|22.35|17.43|59.04|209.16|104.40|268.20|0.52|198.70|126.00|10.46|10.98|136.46|136.98|-93.94| +2452534|18532|2452641|13155|77891|673554|2017|48080|51774|724975|669|8332|57|6|20|4|98|241|52|84.28|217.44|217.44|0.00|11306.88|4382.56|11306.88|339.20|0.00|3391.96|11306.88|11646.08|14698.84|15038.04|6924.32| +2452534|18532|2452547|6006|77891|673554|2017|48080|51774|724975|669|8332|18|18|16|3|60|241|23|37.57|76.64|22.22|1251.66|511.06|864.11|1762.72|10.22|0.00|634.57|511.06|521.28|1145.63|1155.85|-353.05| +2452534|18532|2452627|2448|77891|673554|2017|48080|51774|724975|669|8332|25|15|13|3|83|241|53|54.62|107.60|82.85|1311.75|4391.05|2894.86|5702.80|87.82|0.00|1140.56|4391.05|4478.87|5531.61|5619.43|1496.19| +2451258|83793|2451286|14959|52559|1916750|588|14379|17122|1511537|6599|19237|46|25|2|4|183|242|92|44.72|126.55|0.00|11642.60|0.00|4114.24|11642.60|0.00|0.00|5704.00|0.00|0.00|5704.00|5704.00|-4114.24| +2451258|83793|2451320|17479|52559|1916750|588|14379|17122|1511537|6599|19237|32|7|4|4|281|242|36|25.22|52.20|30.27|789.48|1089.72|907.92|1879.20|32.69|0.00|939.60|1089.72|1122.41|2029.32|2062.01|181.80| +2451258|83793|2451281|14596|52559|1916750|588|14379|17122|1511537|6599|19237|8|4|4|2|170|242|45|10.47|10.47|4.50|268.65|202.50|471.15|471.15|6.07|0.00|188.10|202.50|208.57|390.60|396.67|-268.65| +2451258|83793|2451315|3316|52559|1916750|588|14379|17122|1511537|6599|19237|20|20|11|1|258|242|72|15.94|24.38|9.99|1036.08|719.28|1147.68|1755.36|35.96|0.00|842.40|719.28|755.24|1561.68|1597.64|-428.40| +2451258|83793|2451373|5198|52559|1916750|588|14379|17122|1511537|6599|19237|34|4|8|1|41|242|38|8.70|17.22|2.06|576.08|78.28|330.60|654.36|0.78|0.00|65.36|78.28|79.06|143.64|144.42|-252.32| +2451258|83793|2451300|14497|52559|1916750|588|14379|17122|1511537|6599|19237|7|2|7|5|250|242|25|14.75|28.61|14.87|343.50|371.75|368.75|715.25|18.58|0.00|207.25|371.75|390.33|579.00|597.58|3.00| +2451258|83793|2451319|6770|52559|1916750|588|14379|17122|1511537|6599|19237|46|22|10|3|66|242|2|30.79|51.72|35.68|32.08|71.36|61.58|103.44|6.42|0.00|28.96|71.36|77.78|100.32|106.74|9.78| +2451258|83793|2451308|3622|52559|1916750|588|14379|17122|1511537|6599|19237|26|16|11|1|91|242|73|48.96|95.96|42.22|3923.02|3082.06|3574.08|7005.08|9.24|2927.95|1820.62|154.11|163.35|1974.73|1983.97|-3419.97| +2451258|83793|2451259|14644|52559|1916750|588|14379|17122|1511537|6599|19237|55|16|20|1|162|242|50|18.75|56.25|55.68|28.50|2784.00|937.50|2812.50|0.00|0.00|871.50|2784.00|2784.00|3655.50|3655.50|1846.50| +2451258|83793|2451301|12698|52559|1916750|588|14379|17122|1511537|6599|19237|40|14|18|3|195|242|76|82.78|96.02|22.08|5619.44|1678.08|6291.28|7297.52|16.78|0.00|1021.44|1678.08|1694.86|2699.52|2716.30|-4613.20| +2451258|83793|2451281|554|52559|1916750|588|14379|17122|1511537|6599|19237|32|28|13|1|249|242|20|13.16|24.08|12.52|231.20|250.40|263.20|481.60|20.03|0.00|187.80|250.40|270.43|438.20|458.23|-12.80| +2451258|83793|2451371|12094|52559|1916750|588|14379|17122|1511537|6599|19237|2|19|5|3|138|242|90|75.06|94.57|87.95|595.80|7915.50|6755.40|8511.30|316.62|0.00|2978.10|7915.50|8232.12|10893.60|11210.22|1160.10| +2451258|83793|2451308|8524|52559|1916750|588|14379|17122|1511537|6599|19237|20|22|6|5|211|242|44|20.19|23.82|12.38|503.36|544.72|888.36|1048.08|21.78|0.00|115.28|544.72|566.50|660.00|681.78|-343.64| +2451258|83793|2451369|5036|52559|1916750|588|14379|17122|1511537|6599|19237|14|10|14|2|60|242|68|13.41|30.30|27.87|165.24|1895.16|911.88|2060.40|0.00|0.00|597.04|1895.16|1895.16|2492.20|2492.20|983.28| +2452591|23670|2452609|13071|62512|296205|7051|1497|50622|710817|5282|8928|25|30|4|1|5|243|35|43.24|78.69|78.69|0.00|2754.15|1513.40|2754.15|192.79|0.00|908.60|2754.15|2946.94|3662.75|3855.54|1240.75| +2452591|23670|2452699|15439|62512|296205|7051|1497|50622|710817|5282|8928|60|30|17|2|164|243|19|65.72|73.60|21.34|992.94|405.46|1248.68|1398.40|0.00|0.00|601.16|405.46|405.46|1006.62|1006.62|-843.22| +2452591|23670|2452652|8955|62512|296205|7051|1497|50622|710817|5282|8928|30|13|2|2|262|243|33|8.28|18.46|13.84|152.46|456.72|273.24|609.18|5.11|383.64|194.70|73.08|78.19|267.78|272.89|-200.16| +2452591|23670|2452682|3949|62512|296205|7051|1497|50622|710817|5282|8928|27|30|13|1|159|243|24|86.83|195.36|105.49|2156.88|2531.76|2083.92|4688.64|126.58|0.00|2109.84|2531.76|2658.34|4641.60|4768.18|447.84| +2452591|23670|2452600|249|62512|296205|7051|1497|50622|710817|5282|8928|24|7|6|3|100|243|87|49.60|125.48|115.44|873.48|10043.28|4315.20|10916.76|602.59|0.00|1637.34|10043.28|10645.87|11680.62|12283.21|5728.08| +2452591|23670|2452672|2307|62512|296205|7051|1497|50622|710817|5282|8928|27|27|14|4|104|243|80|90.89|213.59|89.70|9911.20|7176.00|7271.20|17087.20|645.84|0.00|3246.40|7176.00|7821.84|10422.40|11068.24|-95.20| +2452591|23670|2452593|3931|62512|296205|7051|1497|50622|710817|5282|8928|12|15|7|4|231|243|72|56.12|168.36|80.81|6303.60|5818.32|4040.64|12121.92|174.54|0.00|605.52|5818.32|5992.86|6423.84|6598.38|1777.68| +2452591|23670|2452643|6744|62512|296205|7051|1497|50622|710817|5282|8928|37|30|11|2|213|243|17|82.11|131.37|118.23|223.38|2009.91|1395.87|2233.29|100.49|0.00|133.96|2009.91|2110.40|2143.87|2244.36|614.04| +2452591|23670|2452641|9135|62512|296205|7051|1497|50622|710817|5282|8928|31|19|1|2|223|243|60|35.48|58.54|36.29|1335.00|2177.40|2128.80|3512.40|87.09|0.00|948.00|2177.40|2264.49|3125.40|3212.49|48.60| +2452591|23670|2452666|11655|62512|296205|7051|1497|50622|710817|5282|8928|1|7|10|3|6|243|88|11.92|18.71|4.67|1235.52|410.96|1048.96|1646.48|20.54|0.00|740.08|410.96|431.50|1151.04|1171.58|-638.00| +2452591|23670|2452593|5073|62512|296205|7051|1497|50622|710817|5282|8928|49|13|19|1|73|243|100|6.24|10.54|7.48|306.00|748.00|624.00|1054.00|37.40|0.00|400.00|748.00|785.40|1148.00|1185.40|124.00| +2452636|61143|2452683|7482|11209|291225|41|12010|59892|1089914|6858|7627|19|25|16|4|22|244|4|31.70|48.18|9.15|156.12|36.60|126.80|192.72|0.73|0.00|69.36|36.60|37.33|105.96|106.69|-90.20| +2452636|61143|2452694|6049|11209|291225|41|12010|59892|1089914|6858|7627|25|3|8|2|43|244|23|71.59|201.88|173.61|650.21|3993.03|1646.57|4643.24|79.86|0.00|2042.86|3993.03|4072.89|6035.89|6115.75|2346.46| +2452636|61143|2452643|12253|11209|291225|41|12010|59892|1089914|6858|7627|6|13|1|5|12|244|3|38.89|74.66|38.07|109.77|114.21|116.67|223.98|9.13|0.00|35.82|114.21|123.34|150.03|159.16|-2.46| +2452636|61143|2452649|14790|11209|291225|41|12010|59892|1089914|6858|7627|45|18|5|1|27|244|41|81.04|127.23|110.69|678.14|4538.29|3322.64|5216.43|136.14|0.00|625.66|4538.29|4674.43|5163.95|5300.09|1215.65| +2452636|61143|2452669|15571|11209|291225|41|12010|59892|1089914|6858|7627|55|7|3|4|128|244|80|65.19|153.84|56.92|7753.60|4553.60|5215.20|12307.20|409.82|0.00|1968.80|4553.60|4963.42|6522.40|6932.22|-661.60| +2452636|61143|2452651|8304|11209|291225|41|12010|59892|1089914|6858|7627|1|19|2|1|30|244|89|19.59|48.38|42.57|517.09|3788.73|1743.51|4305.82|0.00|0.00|1722.15|3788.73|3788.73|5510.88|5510.88|2045.22| +2452636|61143|2452648|4561|11209|291225|41|12010|59892|1089914|6858|7627|30|1|10|1|252|244|41|73.27|137.74|133.60|169.74|5477.60|3004.07|5647.34|273.88|0.00|1806.87|5477.60|5751.48|7284.47|7558.35|2473.53| +2452636|61143|2452741|5907|11209|291225|41|12010|59892|1089914|6858|7627|42|27|20|3|106|244|9|24.01|41.53|10.38|280.35|93.42|216.09|373.77|2.80|0.00|164.43|93.42|96.22|257.85|260.65|-122.67| +2452636|61143|2452688|11130|11209|291225|41|12010|59892|1089914|6858|7627|57|15|14|3|222|244|68|44.13|118.70|2.37|7910.44|161.16|3000.84|8071.60|12.89|0.00|80.24|161.16|174.05|241.40|254.29|-2839.68| +2452636|61143|2452645|13357|11209|291225|41|12010|59892|1089914|6858|7627|39|9|13|1|126|244|21|17.70|42.12|16.84|530.88|353.64|371.70|884.52|18.07|95.48|336.00|258.16|276.23|594.16|612.23|-113.54| +2452636|61143|2452730|10308|11209|291225|41|12010|59892|1089914|6858|7627|60|13|4|4|51|244|41|39.79|114.59|88.23|1080.76|3617.43|1631.39|4698.19|0.00|2676.89|1362.43|940.54|940.54|2302.97|2302.97|-690.85| +2452636|61143|2452664|4735|11209|291225|41|12010|59892|1089914|6858|7627|39|1|5|1|110|244|87|33.85|49.75|13.43|3159.84|1168.41|2944.95|4328.25|23.36|0.00|1644.30|1168.41|1191.77|2812.71|2836.07|-1776.54| +2452636|61143|2452680|7854|11209|291225|41|12010|59892|1089914|6858|7627|25|12|12|5|68|244|46|20.06|40.52|28.76|540.96|1322.96|922.76|1863.92|18.78|383.65|913.10|939.31|958.09|1852.41|1871.19|16.55| +2452636|61143|2452649|13621|11209|291225|41|12010|59892|1089914|6858|7627|60|25|16|4|112|244|34|7.28|7.71|7.71|0.00|262.14|247.52|262.14|20.97|0.00|99.28|262.14|283.11|361.42|382.39|14.62| +2452636|61143|2452676|201|11209|291225|41|12010|59892|1089914|6858|7627|24|12|13|2|71|244|66|39.55|48.64|24.32|1605.12|1605.12|2610.30|3210.24|47.19|818.61|481.14|786.51|833.70|1267.65|1314.84|-1823.79| +2452636|61143|2452699|6373|11209|291225|41|12010|59892|1089914|6858|7627|55|6|5|5|92|244|88|27.02|74.30|62.41|1046.32|5492.08|2377.76|6538.40|384.44|0.00|1111.44|5492.08|5876.52|6603.52|6987.96|3114.32| +2451828|44939|2451942|17441|96834|588136|591|23893|10850|56218|2952|4207|44|5|10|2|17|245|39|18.68|25.40|3.30|861.90|128.70|728.52|990.60|2.57|0.00|445.77|128.70|131.27|574.47|577.04|-599.82| +2451828|44939|2451901|11048|96834|588136|591|23893|10850|56218|2952|4207|31|8|14|3|212|245|68|45.21|104.88|3.14|6918.32|213.52|3074.28|7131.84|6.40|0.00|2139.28|213.52|219.92|2352.80|2359.20|-2860.76| +2451828|44939|2451842|295|96834|588136|591|23893|10850|56218|2952|4207|49|7|6|3|77|245|41|85.05|180.30|144.24|1478.46|5913.84|3487.05|7392.30|413.96|0.00|3621.94|5913.84|6327.80|9535.78|9949.74|2426.79| +2451828|44939|2451836|17539|96834|588136|591|23893|10850|56218|2952|4207|8|13|19|3|80|245|61|60.50|178.47|114.22|3919.25|6967.42|3690.50|10886.67|69.67|0.00|4898.91|6967.42|7037.09|11866.33|11936.00|3276.92| +2451828|44939|2451916|1229|96834|588136|591|23893|10850|56218|2952|4207|55|7|20|1|255|245|31|48.81|146.43|80.53|2042.90|2496.43|1513.11|4539.33|0.00|0.00|998.51|2496.43|2496.43|3494.94|3494.94|983.32| +2451828|44939|2451843|17780|96834|588136|591|23893|10850|56218|2952|4207|55|29|12|4|126|245|88|63.10|186.14|26.05|14087.92|2292.40|5552.80|16380.32|91.69|0.00|491.04|2292.40|2384.09|2783.44|2875.13|-3260.40| +2451828|44939|2451859|10052|96834|588136|591|23893|10850|56218|2952|4207|31|29|17|1|178|245|33|3.87|5.18|0.05|169.29|1.65|127.71|170.94|0.14|0.00|1.65|1.65|1.79|3.30|3.44|-126.06| +2451828|44939|2451893|14174|96834|588136|591|23893|10850|56218|2952|4207|7|23|13|1|179|245|82|48.57|130.65|97.98|2678.94|8034.36|3982.74|10713.30|562.40|0.00|5142.22|8034.36|8596.76|13176.58|13738.98|4051.62| +2451828|44939|2451906|13547|96834|588136|591|23893|10850|56218|2952|4207|41|1|11|1|181|245|59|74.53|94.65|58.68|2122.23|3462.12|4397.27|5584.35|276.96|0.00|1898.62|3462.12|3739.08|5360.74|5637.70|-935.15| +2451828|44939|2451861|2159|96834|588136|591|23893|10850|56218|2952|4207|38|23|4|1|288|245|35|78.71|137.74|82.64|1928.50|2892.40|2754.85|4820.90|28.92|0.00|2265.55|2892.40|2921.32|5157.95|5186.87|137.55| +2451828|44939|2451835|2617|96834|588136|591|23893|10850|56218|2952|4207|43|5|17|3|232|245|63|61.34|124.52|118.29|392.49|7452.27|3864.42|7844.76|96.88|5514.67|1803.69|1937.60|2034.48|3741.29|3838.17|-1926.82| +2452219|29737|2452280|10533|39064|1619498|7132|28319|6461|1819833|6374|41561|31|23|11|2|203|246|9|58.29|133.48|46.71|780.93|420.39|524.61|1201.32|2.77|281.66|228.24|138.73|141.50|366.97|369.74|-385.88| +2452219|29737|2452303|6427|39064|1619498|7132|28319|6461|1819833|6374|41561|41|27|7|5|203|246|53|93.96|118.38|84.04|1820.02|4454.12|4979.88|6274.14|76.16|1915.27|438.84|2538.85|2615.01|2977.69|3053.85|-2441.03| +2452219|29737|2452264|1127|39064|1619498|7132|28319|6461|1819833|6374|41561|13|9|10|5|73|246|19|77.71|223.02|20.07|3856.05|381.33|1476.49|4237.38|23.18|91.51|1906.65|289.82|313.00|2196.47|2219.65|-1186.67| +2452219|29737|2452288|6781|39064|1619498|7132|28319|6461|1819833|6374|41561|33|17|8|5|119|246|80|11.23|22.01|14.30|616.80|1144.00|898.40|1760.80|45.76|0.00|369.60|1144.00|1189.76|1513.60|1559.36|245.60| +2452219|29737|2452270|16161|39064|1619498|7132|28319|6461|1819833|6374|41561|7|29|16|5|101|246|44|58.04|125.94|70.52|2438.48|3102.88|2553.76|5541.36|0.00|0.00|276.76|3102.88|3102.88|3379.64|3379.64|549.12| +2452219|29737|2452336|535|39064|1619498|7132|28319|6461|1819833|6374|41561|17|1|13|3|30|246|47|81.66|243.34|87.60|7319.78|4117.20|3838.02|11436.98|247.03|0.00|3888.31|4117.20|4364.23|8005.51|8252.54|279.18| +2452219|29737|2452247|2611|39064|1619498|7132|28319|6461|1819833|6374|41561|55|19|12|4|286|246|34|4.83|11.97|2.03|337.96|69.02|164.22|406.98|4.83|0.00|142.12|69.02|73.85|211.14|215.97|-95.20| +2452219|29737|2452228|375|39064|1619498|7132|28319|6461|1819833|6374|41561|3|9|18|1|164|246|40|47.17|92.92|0.00|3716.80|0.00|1886.80|3716.80|0.00|0.00|1338.00|0.00|0.00|1338.00|1338.00|-1886.80| +2452219|29737|2452274|5245|39064|1619498|7132|28319|6461|1819833|6374|41561|1|11|1|1|231|246|92|30.14|39.48|35.92|327.52|3304.64|2772.88|3632.16|132.18|0.00|690.00|3304.64|3436.82|3994.64|4126.82|531.76| +2452219|29737|2452316|6801|39064|1619498|7132|28319|6461|1819833|6374|41561|19|15|6|4|235|246|65|70.33|128.70|79.79|3179.15|5186.35|4571.45|8365.50|0.00|0.00|1673.10|5186.35|5186.35|6859.45|6859.45|614.90| +2452219|29737|2452238|11999|39064|1619498|7132|28319|6461|1819833|6374|41561|19|1|19|5|71|246|47|90.42|249.55|227.09|1055.62|10673.23|4249.74|11728.85|960.59|0.00|4104.98|10673.23|11633.82|14778.21|15738.80|6423.49| +2452219|29737|2452253|7263|39064|1619498|7132|28319|6461|1819833|6374|41561|9|13|2|2|299|246|49|54.22|101.39|17.23|4123.84|844.27|2656.78|4968.11|50.65|0.00|1738.52|844.27|894.92|2582.79|2633.44|-1812.51| +2452219|29737|2452260|5271|39064|1619498|7132|28319|6461|1819833|6374|41561|11|29|7|1|95|246|32|46.92|134.19|37.57|3091.84|1202.24|1501.44|4294.08|72.13|0.00|1975.04|1202.24|1274.37|3177.28|3249.41|-299.20| +2452219|29737|2452276|4593|39064|1619498|7132|28319|6461|1819833|6374|41561|35|11|14|2|67|246|26|60.51|156.72|53.28|2689.44|1385.28|1573.26|4074.72|27.70|0.00|977.86|1385.28|1412.98|2363.14|2390.84|-187.98| +2451160|31967|2451246|16444|99644|869728|259|24343|19010|966336|595|12726|19|25|3|1|59|247|68|95.08|125.50|38.90|5888.80|2645.20|6465.44|8534.00|64.27|502.58|2474.52|2142.62|2206.89|4617.14|4681.41|-4322.82| +2451160|31967|2451191|6319|99644|869728|259|24343|19010|966336|595|12726|58|7|15|1|240|247|81|52.20|94.48|91.64|230.04|7422.84|4228.20|7652.88|593.82|0.00|1223.91|7422.84|8016.66|8646.75|9240.57|3194.64| +2451160|31967|2451175|5522|99644|869728|259|24343|19010|966336|595|12726|43|7|6|3|10|247|14|35.27|60.66|5.45|772.94|76.30|493.78|849.24|3.05|0.00|246.26|76.30|79.35|322.56|325.61|-417.48| +2451160|31967|2451218|6463|99644|869728|259|24343|19010|966336|595|12726|16|14|7|3|102|247|42|31.35|45.14|30.24|625.80|1270.08|1316.70|1895.88|25.40|0.00|322.14|1270.08|1295.48|1592.22|1617.62|-46.62| +2451160|31967|2451166|1028|99644|869728|259|24343|19010|966336|595|12726|31|2|18|4|43|247|62|90.01|173.71|48.63|7754.96|3015.06|5580.62|10770.02|241.20|0.00|1399.96|3015.06|3256.26|4415.02|4656.22|-2565.56| +2451160|31967|2451185|7760|99644|869728|259|24343|19010|966336|595|12726|25|7|8|2|203|247|38|39.11|114.98|1.14|4325.92|43.32|1486.18|4369.24|3.03|0.00|611.42|43.32|46.35|654.74|657.77|-1442.86| +2451160|31967|2451167|15148|99644|869728|259|24343|19010|966336|595|12726|22|7|9|1|266|247|18|36.79|60.70|9.71|917.82|174.78|662.22|1092.60|8.73|0.00|240.30|174.78|183.51|415.08|423.81|-487.44| +2451160|31967|2451257|7690|99644|869728|259|24343|19010|966336|595|12726|16|20|10|4|121|247|42|3.31|5.72|3.37|98.70|141.54|139.02|240.24|4.24|0.00|76.86|141.54|145.78|218.40|222.64|2.52| +2451160|31967|2451241|3466|99644|869728|259|24343|19010|966336|595|12726|10|1|3|3|300|247|33|59.71|178.53|123.18|1826.55|4064.94|1970.43|5891.49|162.59|0.00|58.74|4064.94|4227.53|4123.68|4286.27|2094.51| +2451160|31967|2451252|3178|99644|869728|259|24343|19010|966336|595|12726|44|20|16|4|222|247|15|28.51|44.47|37.79|100.20|566.85|427.65|667.05|0.00|0.00|286.80|566.85|566.85|853.65|853.65|139.20| +2451160|31967|2451183|11743|99644|869728|259|24343|19010|966336|595|12726|50|28|4|3|222|247|65|35.14|76.60|6.89|4531.15|447.85|2284.10|4979.00|13.43|0.00|1742.65|447.85|461.28|2190.50|2203.93|-1836.25| +2451160|31967|2451280|16363|99644|869728|259|24343|19010|966336|595|12726|2|26|9|1|156|247|71|50.52|72.74|71.28|103.66|5060.88|3586.92|5164.54|202.43|0.00|516.17|5060.88|5263.31|5577.05|5779.48|1473.96| +2451160|31967|2451199|5044|99644|869728|259|24343|19010|966336|595|12726|20|8|12|1|243|247|60|48.12|105.38|93.78|696.00|5626.80|2887.20|6322.80|168.80|0.00|1390.80|5626.80|5795.60|7017.60|7186.40|2739.60| +2451104|69293|2451124|16855|9435|1290032|6276|21273|81973|1440599|7064|8861|16|22|12|2|65|248|33|21.91|57.84|24.29|1107.15|801.57|723.03|1908.72|6.89|456.89|495.99|344.68|351.57|840.67|847.56|-378.35| +2451104|69293|2451199|15940|9435|1290032|6276|21273|81973|1440599|7064|8861|8|20|7|3|149|248|61|13.08|14.64|11.56|187.88|705.16|797.88|893.04|4.79|585.28|79.91|119.88|124.67|199.79|204.58|-678.00| +2451104|69293|2451121|4972|9435|1290032|6276|21273|81973|1440599|7064|8861|16|14|10|4|118|248|16|54.05|105.93|11.65|1508.48|186.40|864.80|1694.88|13.04|0.00|338.88|186.40|199.44|525.28|538.32|-678.40| +2451104|69293|2451182|1285|9435|1290032|6276|21273|81973|1440599|7064|8861|31|1|13|3|180|248|78|40.27|41.47|7.04|2685.54|549.12|3141.06|3234.66|49.42|0.00|1422.72|549.12|598.54|1971.84|2021.26|-2591.94| +2451104|69293|2451171|14456|9435|1290032|6276|21273|81973|1440599|7064|8861|31|19|14|3|285|248|90|26.75|55.64|22.25|3005.10|2002.50|2407.50|5007.60|80.10|0.00|2403.00|2002.50|2082.60|4405.50|4485.60|-405.00| +2451104|69293|2451194|12391|9435|1290032|6276|21273|81973|1440599|7064|8861|16|4|10|4|98|248|70|66.09|138.78|95.75|3012.10|6702.50|4626.30|9714.60|67.02|0.00|4565.40|6702.50|6769.52|11267.90|11334.92|2076.20| +2451104|69293|2451143|14653|9435|1290032|6276|21273|81973|1440599|7064|8861|16|22|15|3|141|248|79|61.83|174.36|116.82|4545.66|9228.78|4884.57|13774.44|646.01|0.00|0.00|9228.78|9874.79|9228.78|9874.79|4344.21| +2451104|69293|2451168|12091|9435|1290032|6276|21273|81973|1440599|7064|8861|38|20|2|3|44|248|49|16.40|16.56|10.59|292.53|518.91|803.60|811.44|20.75|0.00|137.69|518.91|539.66|656.60|677.35|-284.69| +2451104|69293|2451113|9292|9435|1290032|6276|21273|81973|1440599|7064|8861|1|10|14|4|277|248|38|97.10|141.76|1.41|5333.30|53.58|3689.80|5386.88|2.67|0.00|2531.56|53.58|56.25|2585.14|2587.81|-3636.22| +2452206|83250|2452236|7005|95691|204793|295|21797|33117|699447|6656|22958|11|9|4|1|85|249|85|26.05|77.36|34.81|3616.75|2958.85|2214.25|6575.60|118.35|0.00|1051.45|2958.85|3077.20|4010.30|4128.65|744.60| +2452206|83250|2452257|13657|95691|204793|295|21797|33117|699447|6656|22958|35|9|9|1|273|249|76|49.99|52.98|19.07|2577.16|1449.32|3799.24|4026.48|42.61|840.60|1771.56|608.72|651.33|2380.28|2422.89|-3190.52| +2452206|83250|2452228|5387|95691|204793|295|21797|33117|699447|6656|22958|5|3|11|2|96|249|71|15.26|45.62|8.21|2656.11|582.91|1083.46|3239.02|0.00|0.00|614.86|582.91|582.91|1197.77|1197.77|-500.55| +2452206|83250|2452294|5541|95691|204793|295|21797|33117|699447|6656|22958|49|5|11|4|267|249|61|65.75|143.99|128.15|966.24|7817.15|4010.75|8783.39|0.00|0.00|2810.27|7817.15|7817.15|10627.42|10627.42|3806.40| +2452206|83250|2452312|4879|95691|204793|295|21797|33117|699447|6656|22958|35|17|3|4|215|249|85|40.37|90.42|70.52|1691.50|5994.20|3431.45|7685.70|59.94|0.00|1536.80|5994.20|6054.14|7531.00|7590.94|2562.75| +2452206|83250|2452255|16009|95691|204793|295|21797|33117|699447|6656|22958|31|29|4|3|72|249|62|92.97|163.62|147.25|1014.94|9129.50|5764.14|10144.44|91.29|0.00|2840.22|9129.50|9220.79|11969.72|12061.01|3365.36| +2452206|83250|2452292|7713|95691|204793|295|21797|33117|699447|6656|22958|41|17|7|3|116|249|52|44.94|98.86|78.09|1080.04|4060.68|2336.88|5140.72|284.24|0.00|1850.16|4060.68|4344.92|5910.84|6195.08|1723.80| +2452206|83250|2452273|11413|95691|204793|295|21797|33117|699447|6656|22958|57|7|12|2|275|249|17|81.86|123.60|77.86|777.58|1323.62|1391.62|2101.20|55.59|211.77|630.36|1111.85|1167.44|1742.21|1797.80|-279.77| +2452206|83250|2452218|14211|95691|204793|295|21797|33117|699447|6656|22958|17|15|14|3|222|249|34|89.93|205.04|157.88|1603.44|5367.92|3057.62|6971.36|483.11|0.00|3345.94|5367.92|5851.03|8713.86|9196.97|2310.30| +2452206|83250|2452322|6529|95691|204793|295|21797|33117|699447|6656|22958|45|7|15|1|127|249|54|7.55|20.53|17.86|144.18|964.44|407.70|1108.62|0.00|0.00|121.50|964.44|964.44|1085.94|1085.94|556.74| +2452206|83250|2452249|13015|95691|204793|295|21797|33117|699447|6656|22958|49|3|12|4|226|249|94|33.55|35.56|32.00|334.64|3008.00|3153.70|3342.64|0.00|0.00|567.76|3008.00|3008.00|3575.76|3575.76|-145.70| +2452206|83250|2452223|15251|95691|204793|295|21797|33117|699447|6656|22958|55|13|9|1|110|249|70|34.51|62.11|39.12|1609.30|2738.40|2415.70|4347.70|246.45|0.00|173.60|2738.40|2984.85|2912.00|3158.45|322.70| +2452206|83250|2452279|2881|95691|204793|295|21797|33117|699447|6656|22958|5|9|4|1|258|249|64|94.38|113.25|27.18|5508.48|1739.52|6040.32|7248.00|0.00|0.00|3189.12|1739.52|1739.52|4928.64|4928.64|-4300.80| +2452206|83250|2452252|17931|95691|204793|295|21797|33117|699447|6656|22958|55|3|6|3|37|249|56|60.54|157.40|113.32|2468.48|6345.92|3390.24|8814.40|0.00|0.00|175.84|6345.92|6345.92|6521.76|6521.76|2955.68| +2452206|83250|2452208|14421|95691|204793|295|21797|33117|699447|6656|22958|29|3|19|1|23|249|90|30.12|65.36|9.15|5058.90|823.50|2710.80|5882.40|57.64|0.00|1647.00|823.50|881.14|2470.50|2528.14|-1887.30| +2452206|83250|2452231|1237|95691|204793|295|21797|33117|699447|6656|22958|39|13|11|3|102|249|78|31.76|82.25|44.41|2951.52|3463.98|2477.28|6415.50|242.47|0.00|0.00|3463.98|3706.45|3463.98|3706.45|986.70| +2452246|40455|2452282|625|35783|1376539|4142|17298|80938|1466782|60|48416|37|11|13|5|27|250|75|78.58|118.65|111.53|534.00|8364.75|5893.50|8898.75|585.53|0.00|4271.25|8364.75|8950.28|12636.00|13221.53|2471.25| +2452246|40455|2452359|4345|35783|1376539|4142|17298|80938|1466782|60|48416|9|11|17|1|242|250|6|35.89|79.67|52.58|162.54|315.48|215.34|478.02|0.00|0.00|191.16|315.48|315.48|506.64|506.64|100.14| +2452246|40455|2452330|4723|35783|1376539|4142|17298|80938|1466782|60|48416|59|27|18|2|109|250|7|44.42|113.27|4.53|761.18|31.71|310.94|792.89|1.58|0.00|198.17|31.71|33.29|229.88|231.46|-279.23| +2452246|40455|2452345|16599|35783|1376539|4142|17298|80938|1466782|60|48416|39|1|19|5|41|250|54|15.99|17.90|4.47|725.22|241.38|863.46|966.60|16.89|0.00|251.10|241.38|258.27|492.48|509.37|-622.08| +2452246|40455|2452295|8127|35783|1376539|4142|17298|80938|1466782|60|48416|37|15|20|2|74|250|70|47.75|63.50|0.63|4400.90|44.10|3342.50|4445.00|0.67|21.60|0.00|22.50|23.17|22.50|23.17|-3320.00| +2452246|40455|2452251|9209|35783|1376539|4142|17298|80938|1466782|60|48416|49|5|1|4|267|250|82|62.82|147.62|82.66|5326.72|6778.12|5151.24|12104.84|271.12|0.00|4962.64|6778.12|7049.24|11740.76|12011.88|1626.88| +2452246|40455|2452366|16893|35783|1376539|4142|17298|80938|1466782|60|48416|17|11|3|3|80|250|99|19.22|26.52|3.18|2310.66|314.82|1902.78|2625.48|18.88|0.00|866.25|314.82|333.70|1181.07|1199.95|-1587.96| +2452246|40455|2452256|11699|35783|1376539|4142|17298|80938|1466782|60|48416|41|3|10|1|282|250|15|52.54|69.87|44.71|377.40|670.65|788.10|1048.05|46.94|0.00|136.20|670.65|717.59|806.85|853.79|-117.45| +2452246|40455|2452335|12203|35783|1376539|4142|17298|80938|1466782|60|48416|37|11|13|5|237|250|96|57.38|82.62|41.31|3965.76|3965.76|5508.48|7931.52|158.63|0.00|1506.24|3965.76|4124.39|5472.00|5630.63|-1542.72| +2452246|40455|2452347|12795|35783|1376539|4142|17298|80938|1466782|60|48416|55|3|16|5|224|250|52|82.81|156.51|32.86|6429.80|1708.72|4306.12|8138.52|17.08|0.00|1302.08|1708.72|1725.80|3010.80|3027.88|-2597.40| +2452246|40455|2452317|14071|35783|1376539|4142|17298|80938|1466782|60|48416|11|5|13|5|130|250|23|93.43|222.36|24.45|4551.93|562.35|2148.89|5114.28|33.74|0.00|1278.57|562.35|596.09|1840.92|1874.66|-1586.54| +2452246|40455|2452258|17699|35783|1376539|4142|17298|80938|1466782|60|48416|57|5|7|1|105|250|15|78.92|222.55|171.36|767.85|2570.40|1183.80|3338.25|179.92|0.00|901.20|2570.40|2750.32|3471.60|3651.52|1386.60| +2452246|40455|2452301|8501|35783|1376539|4142|17298|80938|1466782|60|48416|51|5|7|4|91|250|64|27.87|68.28|67.59|44.16|4325.76|1783.68|4369.92|173.03|0.00|1354.24|4325.76|4498.79|5680.00|5853.03|2542.08| +2452246|40455|2452327|12325|35783|1376539|4142|17298|80938|1466782|60|48416|49|5|11|4|34|250|55|83.96|161.20|132.18|1596.10|7269.90|4617.80|8866.00|72.69|0.00|3368.75|7269.90|7342.59|10638.65|10711.34|2652.10| +2452246|40455|2452267|7077|35783|1376539|4142|17298|80938|1466782|60|48416|33|3|4|1|8|250|37|51.21|143.90|139.58|159.84|5164.46|1894.77|5324.30|258.22|0.00|266.03|5164.46|5422.68|5430.49|5688.71|3269.69| +2452110|11836|2452179|1379|4412|899428|2891|41954|63873|1440579|1096|36559|3|7|3|2|1|251|19|70.52|83.21|77.38|110.77|1470.22|1339.88|1580.99|14.70|0.00|679.82|1470.22|1484.92|2150.04|2164.74|130.34| +2452110|11836|2452140|10929|4412|899428|2891|41954|63873|1440579|1096|36559|45|15|3|3|263|251|74|8.49|24.19|16.93|537.24|1252.82|628.26|1790.06|112.75|0.00|357.42|1252.82|1365.57|1610.24|1722.99|624.56| +2452110|11836|2452189|1733|4412|899428|2891|41954|63873|1440579|1096|36559|19|7|15|4|233|251|55|9.29|15.51|14.11|77.00|776.05|510.95|853.05|7.76|0.00|281.05|776.05|783.81|1057.10|1064.86|265.10| +2452110|11836|2452113|17471|4412|899428|2891|41954|63873|1440579|1096|36559|35|27|13|4|235|251|50|93.34|261.35|91.47|8494.00|4573.50|4667.00|13067.50|62.20|3796.00|914.50|777.50|839.70|1692.00|1754.20|-3889.50| +2452110|11836|2452148|659|4412|899428|2891|41954|63873|1440579|1096|36559|1|13|3|4|205|251|93|8.59|20.18|19.97|19.53|1857.21|798.87|1876.74|92.86|0.00|468.72|1857.21|1950.07|2325.93|2418.79|1058.34| +2452110|11836|2452133|13201|4412|899428|2891|41954|63873|1440579|1096|36559|25|21|16|1|296|251|60|61.09|130.12|102.79|1639.80|6167.40|3665.40|7807.20|555.06|0.00|2341.80|6167.40|6722.46|8509.20|9064.26|2502.00| +2452110|11836|2452140|14191|4412|899428|2891|41954|63873|1440579|1096|36559|27|9|3|4|75|251|46|57.46|72.39|40.53|1465.56|1864.38|2643.16|3329.94|18.64|0.00|1498.22|1864.38|1883.02|3362.60|3381.24|-778.78| +2452110|11836|2452225|11839|4412|899428|2891|41954|63873|1440579|1096|36559|5|1|19|4|242|251|82|36.13|40.46|20.63|1626.06|1691.66|2962.66|3317.72|101.49|0.00|464.12|1691.66|1793.15|2155.78|2257.27|-1271.00| +2452110|11836|2452168|13181|4412|899428|2891|41954|63873|1440579|1096|36559|29|19|5|5|175|251|47|36.88|97.73|65.47|1516.22|3077.09|1733.36|4593.31|215.39|0.00|1791.17|3077.09|3292.48|4868.26|5083.65|1343.73| +2452626|73572|2452743|2661|89328|1203446|4317|21635|41047|223207|6295|25613|27|18|4|1|181|252|9|67.58|153.40|69.03|759.33|621.27|608.22|1380.60|55.91|0.00|483.21|621.27|677.18|1104.48|1160.39|13.05| +2452626|73572|2452634|9373|89328|1203446|4317|21635|41047|223207|6295|25613|60|15|9|2|235|252|4|97.94|253.66|5.07|994.36|20.28|391.76|1014.64|1.82|0.00|172.48|20.28|22.10|192.76|194.58|-371.48| +2452626|73572|2452719|1515|89328|1203446|4317|21635|41047|223207|6295|25613|33|9|3|1|29|252|2|2.40|5.06|0.60|8.92|1.20|4.80|10.12|0.03|0.00|0.70|1.20|1.23|1.90|1.93|-3.60| +2452626|73572|2452672|3948|89328|1203446|4317|21635|41047|223207|6295|25613|39|13|11|3|185|252|83|2.21|3.22|1.03|181.77|85.49|183.43|267.26|1.70|0.00|71.38|85.49|87.19|156.87|158.57|-97.94| +2452626|73572|2452658|15253|89328|1203446|4317|21635|41047|223207|6295|25613|25|18|12|3|34|252|54|99.40|141.14|132.67|457.38|7164.18|5367.60|7621.56|501.49|0.00|609.66|7164.18|7665.67|7773.84|8275.33|1796.58| +2452626|73572|2452715|3888|89328|1203446|4317|21635|41047|223207|6295|25613|30|3|16|5|97|252|40|53.02|133.61|40.08|3741.20|1603.20|2120.80|5344.40|25.65|1282.56|1282.40|320.64|346.29|1603.04|1628.69|-1800.16| +2452626|73572|2452720|3879|89328|1203446|4317|21635|41047|223207|6295|25613|48|3|10|3|266|252|59|5.16|11.66|7.92|220.66|467.28|304.44|687.94|18.69|0.00|309.16|467.28|485.97|776.44|795.13|162.84| +2452626|73572|2452735|15819|89328|1203446|4317|21635|41047|223207|6295|25613|33|18|4|1|241|252|7|41.08|85.44|38.44|329.00|269.08|287.56|598.08|5.38|0.00|143.50|269.08|274.46|412.58|417.96|-18.48| +2452176|44058|2452221|16961|62476|999185|6294|36603|27997|826530|4330|35439|9|25|1|4|217|253|4|59.65|126.45|13.90|450.20|55.60|238.60|505.80|2.22|0.00|0.00|55.60|57.82|55.60|57.82|-183.00| +2452176|44058|2452200|16481|62476|999185|6294|36603|27997|826530|4330|35439|43|25|4|5|53|253|24|53.10|62.12|59.01|74.64|1416.24|1274.40|1490.88|99.13|0.00|536.64|1416.24|1515.37|1952.88|2052.01|141.84| +2452176|44058|2452280|563|62476|999185|6294|36603|27997|826530|4330|35439|9|21|17|3|156|253|4|83.18|130.59|14.36|464.92|57.44|332.72|522.36|1.72|0.00|31.32|57.44|59.16|88.76|90.48|-275.28| +2452176|44058|2452187|5631|62476|999185|6294|36603|27997|826530|4330|35439|31|3|1|2|13|253|31|11.51|32.80|20.33|386.57|630.23|356.81|1016.80|56.72|0.00|386.26|630.23|686.95|1016.49|1073.21|273.42| +2452176|44058|2452219|6923|62476|999185|6294|36603|27997|826530|4330|35439|11|29|4|3|185|253|60|66.34|150.59|60.23|5421.60|3613.80|3980.40|9035.40|251.52|469.79|3523.80|3144.01|3395.53|6667.81|6919.33|-836.39| +2452176|44058|2452225|11213|62476|999185|6294|36603|27997|826530|4330|35439|7|29|19|5|64|253|2|48.63|125.46|63.98|122.96|127.96|97.26|250.92|3.83|0.00|102.86|127.96|131.79|230.82|234.65|30.70| +2452176|44058|2452192|867|62476|999185|6294|36603|27997|826530|4330|35439|27|23|3|1|112|253|52|4.51|11.41|3.42|415.48|177.84|234.52|593.32|8.89|0.00|70.72|177.84|186.73|248.56|257.45|-56.68| +2452176|44058|2452237|5215|62476|999185|6294|36603|27997|826530|4330|35439|13|23|7|2|57|253|71|87.24|231.18|205.75|1805.53|14608.25|6194.04|16413.78|347.67|9641.44|7550.14|4966.81|5314.48|12516.95|12864.62|-1227.23| +2452267|21633|2452278|2253|46995|991457|5072|23134|56537|1581142|3626|18888|55|23|9|1|273|254|59|45.58|136.28|61.32|4422.64|3617.88|2689.22|8040.52|180.89|0.00|1607.75|3617.88|3798.77|5225.63|5406.52|928.66| +2452267|21633|2452379|17685|46995|991457|5072|23134|56537|1581142|3626|18888|47|3|17|2|10|254|99|41.89|80.00|6.40|7286.40|633.60|4147.11|7920.00|44.35|0.00|2772.00|633.60|677.95|3405.60|3449.95|-3513.51| +2452267|21633|2452301|15683|46995|991457|5072|23134|56537|1581142|3626|18888|33|19|3|4|162|254|30|92.62|264.89|82.11|5483.40|2463.30|2778.60|7946.70|147.79|0.00|2304.30|2463.30|2611.09|4767.60|4915.39|-315.30| +2452267|21633|2452374|13383|46995|991457|5072|23134|56537|1581142|3626|18888|55|7|3|4|8|254|53|21.33|45.64|20.08|1354.68|1064.24|1130.49|2418.92|21.28|0.00|628.58|1064.24|1085.52|1692.82|1714.10|-66.25| +2452267|21633|2452288|2577|46995|991457|5072|23134|56537|1581142|3626|18888|29|19|15|5|55|254|94|69.46|79.87|74.27|526.40|6981.38|6529.24|7507.78|418.88|0.00|1501.18|6981.38|7400.26|8482.56|8901.44|452.14| +2452267|21633|2452280|2391|46995|991457|5072|23134|56537|1581142|3626|18888|1|17|4|2|227|254|52|79.76|155.53|147.75|404.56|7683.00|4147.52|8087.56|0.00|153.66|1536.60|7529.34|7529.34|9065.94|9065.94|3381.82| +2452267|21633|2452367|2847|46995|991457|5072|23134|56537|1581142|3626|18888|39|17|8|2|13|254|60|96.47|182.32|21.87|9627.00|1312.20|5788.20|10939.20|13.12|0.00|5031.60|1312.20|1325.32|6343.80|6356.92|-4476.00| +2452267|21633|2452353|8329|46995|991457|5072|23134|56537|1581142|3626|18888|31|11|6|1|66|254|99|44.78|57.31|20.63|3631.32|2042.37|4433.22|5673.69|40.84|0.00|2609.64|2042.37|2083.21|4652.01|4692.85|-2390.85| +2452267|21633|2452379|8055|46995|991457|5072|23134|56537|1581142|3626|18888|13|17|14|4|61|254|11|52.69|127.50|2.55|1374.45|28.05|579.59|1402.50|0.84|0.00|42.02|28.05|28.89|70.07|70.91|-551.54| +2452267|21633|2452317|16859|46995|991457|5072|23134|56537|1581142|3626|18888|51|25|13|5|267|254|79|80.83|241.68|181.26|4773.18|14319.54|6385.57|19092.72|286.39|0.00|9355.18|14319.54|14605.93|23674.72|23961.11|7933.97| +2452267|21633|2452381|9401|46995|991457|5072|23134|56537|1581142|3626|18888|47|11|18|5|12|254|53|17.73|51.23|22.54|1520.57|1194.62|939.69|2715.19|71.67|0.00|1004.35|1194.62|1266.29|2198.97|2270.64|254.93| +2452267|21633|2452328|8269|46995|991457|5072|23134|56537|1581142|3626|18888|57|9|1|4|30|254|65|53.02|100.73|39.28|3994.25|2553.20|3446.30|6547.45|204.25|0.00|2945.80|2553.20|2757.45|5499.00|5703.25|-893.10| +2452267|21633|2452315|14817|46995|991457|5072|23134|56537|1581142|3626|18888|5|25|10|5|267|254|28|42.83|43.68|8.73|978.60|244.44|1199.24|1223.04|12.22|0.00|73.36|244.44|256.66|317.80|330.02|-954.80| +2452267|21633|2452281|3945|46995|991457|5072|23134|56537|1581142|3626|18888|33|23|20|5|80|254|59|50.50|145.94|27.72|6974.98|1635.48|2979.50|8610.46|32.70|0.00|85.55|1635.48|1668.18|1721.03|1753.73|-1344.02| +2451759|63083|2451807|4820|99182|1576870|6558|36246|73189|106341|2110|18998|59|29|17|4|254|255|29|25.88|68.84|2.75|1916.61|79.75|750.52|1996.36|3.98|0.00|518.81|79.75|83.73|598.56|602.54|-670.77| +2451759|63083|2451835|14993|99182|1576870|6558|36246|73189|106341|2110|18998|11|26|5|3|61|255|49|36.56|105.65|90.85|725.20|4451.65|1791.44|5176.85|0.00|0.00|517.44|4451.65|4451.65|4969.09|4969.09|2660.21| +2451759|63083|2451850|85|99182|1576870|6558|36246|73189|106341|2110|18998|17|14|10|2|171|255|33|35.75|77.93|7.01|2340.36|231.33|1179.75|2571.69|13.87|0.00|591.36|231.33|245.20|822.69|836.56|-948.42| +2451759|63083|2451834|8606|99182|1576870|6558|36246|73189|106341|2110|18998|1|25|5|5|222|255|99|23.17|38.23|30.20|794.97|2989.80|2293.83|3784.77|29.89|0.00|1475.10|2989.80|3019.69|4464.90|4494.79|695.97| +2451759|63083|2451833|11915|99182|1576870|6558|36246|73189|106341|2110|18998|20|1|1|1|61|255|65|2.46|2.50|1.67|53.95|108.55|159.90|162.50|8.68|0.00|65.00|108.55|117.23|173.55|182.23|-51.35| +2451759|63083|2451825|313|99182|1576870|6558|36246|73189|106341|2110|18998|19|29|20|2|277|255|74|99.37|268.29|48.29|16280.00|3573.46|7353.38|19853.46|107.20|0.00|9330.66|3573.46|3680.66|12904.12|13011.32|-3779.92| +2451759|63083|2451770|3128|99182|1576870|6558|36246|73189|106341|2110|18998|25|11|16|1|29|255|79|75.10|201.26|106.66|7473.40|8426.14|5932.90|15899.54|674.09|0.00|2543.80|8426.14|9100.23|10969.94|11644.03|2493.24| +2451759|63083|2451842|8267|99182|1576870|6558|36246|73189|106341|2110|18998|50|17|5|3|11|255|60|10.91|29.34|11.73|1056.60|703.80|654.60|1760.40|42.22|0.00|651.00|703.80|746.02|1354.80|1397.02|49.20| +2451759|63083|2451830|17534|99182|1576870|6558|36246|73189|106341|2110|18998|59|13|7|4|81|255|6|72.28|213.22|19.18|1164.24|115.08|433.68|1279.32|4.60|0.00|102.30|115.08|119.68|217.38|221.98|-318.60| +2451759|63083|2451849|6997|99182|1576870|6558|36246|73189|106341|2110|18998|19|8|10|3|193|255|95|71.96|164.78|1.64|15498.30|155.80|6836.20|15654.10|3.11|0.00|0.00|155.80|158.91|155.80|158.91|-6680.40| +2451759|63083|2451806|17312|99182|1576870|6558|36246|73189|106341|2110|18998|56|23|15|1|74|255|21|59.56|117.33|104.42|271.11|2192.82|1250.76|2463.93|197.35|0.00|492.66|2192.82|2390.17|2685.48|2882.83|942.06| +2451133|54789|2451219|16609|65176|1422428|4480|9332|15357|674242|6735|35187|16|1|20|4|88|256|55|34.82|103.76|58.10|2511.30|3195.50|1915.10|5706.80|223.68|0.00|342.10|3195.50|3419.18|3537.60|3761.28|1280.40| +2451133|54789|2451157|16861|65176|1422428|4480|9332|15357|674242|6735|35187|20|28|6|5|184|256|45|34.74|99.00|51.48|2138.40|2316.60|1563.30|4455.00|47.02|1644.78|2227.50|671.82|718.84|2899.32|2946.34|-891.48| +2451133|54789|2451177|17803|65176|1422428|4480|9332|15357|674242|6735|35187|56|1|6|3|173|256|39|92.66|180.68|19.87|6271.59|774.93|3613.74|7046.52|7.43|650.94|2465.97|123.99|131.42|2589.96|2597.39|-3489.75| +2451133|54789|2451199|3682|65176|1422428|4480|9332|15357|674242|6735|35187|56|20|16|3|130|256|57|98.18|103.08|101.01|117.99|5757.57|5596.26|5875.56|287.87|0.00|1703.73|5757.57|6045.44|7461.30|7749.17|161.31| +2451133|54789|2451168|12676|65176|1422428|4480|9332|15357|674242|6735|35187|25|10|2|3|181|256|71|72.56|139.31|82.19|4055.52|5835.49|5151.76|9891.01|116.70|0.00|1186.41|5835.49|5952.19|7021.90|7138.60|683.73| +2451133|54789|2451192|446|65176|1422428|4480|9332|15357|674242|6735|35187|49|8|5|5|275|256|74|14.20|34.93|18.16|1240.98|1343.84|1050.80|2584.82|36.55|430.02|697.82|913.82|950.37|1611.64|1648.19|-136.98| +2451133|54789|2451185|12685|65176|1422428|4480|9332|15357|674242|6735|35187|56|8|11|4|90|256|35|33.51|41.55|31.16|363.65|1090.60|1172.85|1454.25|17.66|796.13|348.95|294.47|312.13|643.42|661.08|-878.38| +2451133|54789|2451158|15566|65176|1422428|4480|9332|15357|674242|6735|35187|32|25|9|3|234|256|66|86.40|158.11|112.25|3026.76|7408.50|5702.40|10435.26|148.17|0.00|3965.28|7408.50|7556.67|11373.78|11521.95|1706.10| +2451133|54789|2451160|280|65176|1422428|4480|9332|15357|674242|6735|35187|20|26|17|2|210|256|7|82.89|135.11|18.91|813.40|132.37|580.23|945.77|0.00|117.80|312.06|14.57|14.57|326.63|326.63|-565.66| +2451164|58989|2451239|5545|66818|544757|1096|2235|12593|512762|4405|15497|56|2|5|3|11|257|1|51.99|144.01|142.56|1.45|142.56|51.99|144.01|7.12|0.00|20.16|142.56|149.68|162.72|169.84|90.57| +2451164|58989|2451195|12067|66818|544757|1096|2235|12593|512762|4405|15497|34|1|14|2|160|257|52|51.90|77.33|72.69|241.28|3779.88|2698.80|4021.16|226.79|0.00|1889.68|3779.88|4006.67|5669.56|5896.35|1081.08| +2451164|58989|2451173|10258|66818|544757|1096|2235|12593|512762|4405|15497|20|25|2|1|153|257|13|14.67|23.47|20.88|33.67|271.44|190.71|305.11|0.00|0.00|39.65|271.44|271.44|311.09|311.09|80.73| +2451164|58989|2451170|7339|66818|544757|1096|2235|12593|512762|4405|15497|55|16|10|4|5|257|8|21.67|22.10|16.35|46.00|130.80|173.36|176.80|2.61|0.00|67.12|130.80|133.41|197.92|200.53|-42.56| +2451164|58989|2451274|4406|66818|544757|1096|2235|12593|512762|4405|15497|34|16|10|1|49|257|37|62.00|73.78|31.72|1556.22|1173.64|2294.00|2729.86|93.89|0.00|436.60|1173.64|1267.53|1610.24|1704.13|-1120.36| +2451164|58989|2451196|10022|66818|544757|1096|2235|12593|512762|4405|15497|22|19|8|1|34|257|38|7.22|14.94|9.41|210.14|357.58|274.36|567.72|26.89|21.45|187.34|336.13|363.02|523.47|550.36|61.77| +2451164|58989|2451245|9538|66818|544757|1096|2235|12593|512762|4405|15497|10|7|12|2|95|257|90|51.22|122.92|110.62|1107.00|9955.80|4609.80|11062.80|199.11|0.00|3871.80|9955.80|10154.91|13827.60|14026.71|5346.00| +2451164|58989|2451274|12616|66818|544757|1096|2235|12593|512762|4405|15497|14|26|13|4|48|257|66|34.08|97.12|32.04|4295.28|2114.64|2249.28|6409.92|105.73|0.00|640.86|2114.64|2220.37|2755.50|2861.23|-134.64| +2451164|58989|2451249|6175|66818|544757|1096|2235|12593|512762|4405|15497|56|13|15|3|151|257|84|53.95|100.88|22.19|6609.96|1863.96|4531.80|8473.92|1.30|1845.32|0.00|18.64|19.94|18.64|19.94|-4513.16| +2451164|58989|2451278|4814|66818|544757|1096|2235|12593|512762|4405|15497|34|1|11|4|167|257|45|76.54|169.91|62.86|4817.25|2828.70|3444.30|7645.95|2.54|2574.11|3364.20|254.59|257.13|3618.79|3621.33|-3189.71| +2451029|49082|2451042|14824|30044|723846|4048|41050|22781|243589|3703|22705|34|4|8|5|115|258|66|13.83|21.15|20.51|42.24|1353.66|912.78|1395.90|79.18|473.78|97.68|879.88|959.06|977.56|1056.74|-32.90| +2451029|49082|2451052|7963|30044|723846|4048|41050|22781|243589|3703|22705|31|22|5|3|21|258|57|53.22|71.84|11.49|3439.95|654.93|3033.54|4094.88|19.64|0.00|1924.32|654.93|674.57|2579.25|2598.89|-2378.61| +2451029|49082|2451062|7459|30044|723846|4048|41050|22781|243589|3703|22705|38|4|3|1|289|258|40|50.52|144.99|4.34|5626.00|173.60|2020.80|5799.60|12.15|0.00|347.60|173.60|185.75|521.20|533.35|-1847.20| +2451029|49082|2451085|5882|30044|723846|4048|41050|22781|243589|3703|22705|2|10|7|3|69|258|4|5.50|10.89|2.94|31.80|11.76|22.00|43.56|0.35|0.00|0.40|11.76|12.11|12.16|12.51|-10.24| +2451029|49082|2451073|1801|30044|723846|4048|41050|22781|243589|3703|22705|4|16|12|5|281|258|34|2.61|4.56|1.96|88.40|66.64|88.74|155.04|1.33|0.00|58.82|66.64|67.97|125.46|126.79|-22.10| +2451029|49082|2451129|12580|30044|723846|4048|41050|22781|243589|3703|22705|13|1|18|5|292|258|89|75.64|126.31|58.10|6070.69|5170.90|6731.96|11241.59|51.70|0.00|4608.42|5170.90|5222.60|9779.32|9831.02|-1561.06| +2451029|49082|2451067|3548|30044|723846|4048|41050|22781|243589|3703|22705|32|26|14|1|148|258|17|36.84|54.15|11.37|727.26|193.29|626.28|920.55|17.39|0.00|340.51|193.29|210.68|533.80|551.19|-432.99| +2451029|49082|2451075|11498|30044|723846|4048|41050|22781|243589|3703|22705|44|20|9|2|65|258|46|33.75|45.90|2.75|1984.90|126.50|1552.50|2111.40|1.26|0.00|569.94|126.50|127.76|696.44|697.70|-1426.00| +2451029|49082|2451073|2719|30044|723846|4048|41050|22781|243589|3703|22705|7|4|12|1|187|258|19|72.65|161.28|56.44|1991.96|1072.36|1380.35|3064.32|10.72|0.00|735.30|1072.36|1083.08|1807.66|1818.38|-307.99| +2451029|49082|2451033|17012|30044|723846|4048|41050|22781|243589|3703|22705|31|8|4|1|115|258|43|81.24|119.42|27.46|3954.28|1180.78|3493.32|5135.06|94.46|0.00|821.30|1180.78|1275.24|2002.08|2096.54|-2312.54| +2451029|49082|2451063|8204|30044|723846|4048|41050|22781|243589|3703|22705|1|28|9|3|207|258|61|45.78|86.52|79.59|422.73|4854.99|2792.58|5277.72|388.39|0.00|2532.72|4854.99|5243.38|7387.71|7776.10|2062.41| +2451029|49082|2451074|7240|30044|723846|4048|41050|22781|243589|3703|22705|44|28|17|1|28|258|98|67.29|144.67|13.02|12901.70|1275.96|6594.42|14177.66|0.00|0.00|1275.96|1275.96|1275.96|2551.92|2551.92|-5318.46| +2451029|49082|2451108|8638|30044|723846|4048|41050|22781|243589|3703|22705|31|1|9|2|85|258|63|11.06|21.89|6.78|951.93|427.14|696.78|1379.07|8.54|0.00|468.72|427.14|435.68|895.86|904.40|-269.64| +2451389|65919|2451468|16975|9506|919245|3384|34323|15377|64748|2693|432|55|7|10|4|225|259|75|98.47|134.90|122.75|911.25|9206.25|7385.25|10117.50|828.56|0.00|4957.50|9206.25|10034.81|14163.75|14992.31|1821.00| +2451389|65919|2451479|16448|9506|919245|3384|34323|15377|64748|2693|432|52|8|13|1|89|259|100|26.14|73.97|62.13|1184.00|6213.00|2614.00|7397.00|248.52|0.00|1553.00|6213.00|6461.52|7766.00|8014.52|3599.00| +2451389|65919|2451483|10267|9506|919245|3384|34323|15377|64748|2693|432|16|22|5|5|148|259|63|93.92|246.07|86.12|10076.85|5425.56|5916.96|15502.41|0.00|0.00|5890.50|5425.56|5425.56|11316.06|11316.06|-491.40| +2451389|65919|2451461|9530|9506|919245|3384|34323|15377|64748|2693|432|1|26|20|3|106|259|84|58.45|75.98|29.63|3893.40|2488.92|4909.80|6382.32|199.11|0.00|2935.80|2488.92|2688.03|5424.72|5623.83|-2420.88| +2451389|65919|2451407|7102|9506|919245|3384|34323|15377|64748|2693|432|26|16|4|4|257|259|58|98.06|279.47|209.60|4052.46|12156.80|5687.48|16209.26|0.00|0.00|1944.74|12156.80|12156.80|14101.54|14101.54|6469.32| +2451389|65919|2451438|4423|9506|919245|3384|34323|15377|64748|2693|432|40|22|13|2|68|259|53|70.51|151.59|28.80|6507.87|1526.40|3737.03|8034.27|106.84|0.00|3534.57|1526.40|1633.24|5060.97|5167.81|-2210.63| +2451389|65919|2451475|13003|9506|919245|3384|34323|15377|64748|2693|432|2|7|9|1|33|259|90|98.01|113.69|106.86|614.70|9617.40|8820.90|10232.10|0.00|0.00|1534.50|9617.40|9617.40|11151.90|11151.90|796.50| +2451389|65919|2451401|12590|9506|919245|3384|34323|15377|64748|2693|432|43|14|4|5|266|259|19|70.79|96.98|73.70|442.32|1400.30|1345.01|1842.62|70.01|0.00|368.41|1400.30|1470.31|1768.71|1838.72|55.29| +2451389|65919|2451473|6454|9506|919245|3384|34323|15377|64748|2693|432|8|19|5|2|242|259|36|97.60|290.84|162.87|4606.92|5863.32|3513.60|10470.24|58.63|0.00|837.36|5863.32|5921.95|6700.68|6759.31|2349.72| +2451389|65919|2451486|7670|9506|919245|3384|34323|15377|64748|2693|432|44|26|15|4|160|259|95|66.53|189.61|36.02|14591.05|3421.90|6320.35|18012.95|273.75|0.00|7745.35|3421.90|3695.65|11167.25|11441.00|-2898.45| +2452612|44586|2452716|14047|2570|1906687|7157|47274|70404|657104|2378|34448|18|15|14|3|199|260|96|9.39|26.94|9.69|1656.00|930.24|901.44|2586.24|9.30|0.00|128.64|930.24|939.54|1058.88|1068.18|28.80| +2452612|44586|2452632|9153|2570|1906687|7157|47274|70404|657104|2378|34448|24|9|1|4|217|260|85|25.69|60.88|56.61|362.95|4811.85|2183.65|5174.80|115.48|962.37|102.85|3849.48|3964.96|3952.33|4067.81|1665.83| +2452612|44586|2452725|6019|2570|1906687|7157|47274|70404|657104|2378|34448|31|25|3|5|27|260|99|91.01|214.78|118.12|9569.34|11693.88|9009.99|21263.22|233.87|0.00|4252.05|11693.88|11927.75|15945.93|16179.80|2683.89| +2452612|44586|2452701|16242|2570|1906687|7157|47274|70404|657104|2378|34448|7|9|19|5|84|260|9|50.45|59.02|47.21|106.29|424.89|454.05|531.18|16.99|0.00|74.34|424.89|441.88|499.23|516.22|-29.16| +2452612|44586|2452717|10381|2570|1906687|7157|47274|70404|657104|2378|34448|19|18|15|4|170|260|63|28.36|38.00|33.06|311.22|2082.78|1786.68|2394.00|20.82|0.00|1125.18|2082.78|2103.60|3207.96|3228.78|296.10| +2452612|44586|2452686|16915|2570|1906687|7157|47274|70404|657104|2378|34448|25|18|5|4|260|260|62|32.33|54.96|52.76|136.40|3271.12|2004.46|3407.52|261.68|0.00|33.48|3271.12|3532.80|3304.60|3566.28|1266.66| +2452612|44586|2452727|12468|2570|1906687|7157|47274|70404|657104|2378|34448|37|15|17|3|12|260|89|3.48|8.17|5.96|196.69|530.44|309.72|727.13|31.82|0.00|43.61|530.44|562.26|574.05|605.87|220.72| +2452612|44586|2452686|14568|2570|1906687|7157|47274|70404|657104|2378|34448|33|27|8|2|296|260|93|79.10|168.48|15.16|14258.76|1409.88|7356.30|15668.64|84.59|0.00|1879.53|1409.88|1494.47|3289.41|3374.00|-5946.42| +2452612|44586|2452616|1386|2570|1906687|7157|47274|70404|657104|2378|34448|49|13|16|3|110|260|22|3.22|6.40|2.56|84.48|56.32|70.84|140.80|0.56|0.00|12.54|56.32|56.88|68.86|69.42|-14.52| +2452612|44586|2452720|2292|2570|1906687|7157|47274|70404|657104|2378|34448|45|24|1|3|152|260|46|20.62|51.75|30.53|976.12|1404.38|948.52|2380.50|56.17|0.00|428.26|1404.38|1460.55|1832.64|1888.81|455.86| +2451070|37465|2451173|4294|41622|766316|3010|29304|7119|165991|6233|46527|43|13|2|2|165|261|69|65.88|123.85|60.68|4358.73|4186.92|4545.72|8545.65|334.95|0.00|597.54|4186.92|4521.87|4784.46|5119.41|-358.80| +2451070|37465|2451078|13732|41622|766316|3010|29304|7119|165991|6233|46527|58|20|12|1|187|261|31|43.33|91.85|7.34|2619.81|227.54|1343.23|2847.35|18.20|0.00|683.24|227.54|245.74|910.78|928.98|-1115.69| +2451070|37465|2451190|15727|41622|766316|3010|29304|7119|165991|6233|46527|2|13|9|1|96|261|94|7.87|10.93|7.76|297.98|729.44|739.78|1027.42|43.76|0.00|369.42|729.44|773.20|1098.86|1142.62|-10.34| +2451070|37465|2451124|14428|41622|766316|3010|29304|7119|165991|6233|46527|43|1|10|1|5|261|23|22.70|38.36|16.49|503.01|379.27|522.10|882.28|15.17|0.00|361.56|379.27|394.44|740.83|756.00|-142.83| +2451070|37465|2451187|10696|41622|766316|3010|29304|7119|165991|6233|46527|37|14|14|5|109|261|10|61.49|164.79|100.52|642.70|1005.20|614.90|1647.90|51.86|140.72|494.30|864.48|916.34|1358.78|1410.64|249.58| +2451070|37465|2451122|2252|41622|766316|3010|29304|7119|165991|6233|46527|50|19|14|3|52|261|14|56.17|145.48|68.37|1079.54|957.18|786.38|2036.72|57.43|0.00|488.74|957.18|1014.61|1445.92|1503.35|170.80| +2451070|37465|2451167|17684|41622|766316|3010|29304|7119|165991|6233|46527|22|16|3|5|74|261|52|98.11|223.69|194.61|1512.16|10119.72|5101.72|11631.88|607.18|0.00|115.96|10119.72|10726.90|10235.68|10842.86|5018.00| +2451070|37465|2451168|15682|41622|766316|3010|29304|7119|165991|6233|46527|28|28|5|5|195|261|64|44.61|118.21|108.75|605.44|6960.00|2855.04|7565.44|0.00|4732.80|832.00|2227.20|2227.20|3059.20|3059.20|-627.84| +2451070|37465|2451118|13382|41622|766316|3010|29304|7119|165991|6233|46527|14|19|12|4|16|261|17|97.27|267.49|82.92|3137.69|1409.64|1653.59|4547.33|28.19|0.00|1136.79|1409.64|1437.83|2546.43|2574.62|-243.95| +2451070|37465|2451150|2576|41622|766316|3010|29304|7119|165991|6233|46527|44|4|13|3|236|261|76|70.17|178.23|58.81|9075.92|4469.56|5332.92|13545.48|44.69|0.00|3250.52|4469.56|4514.25|7720.08|7764.77|-863.36| +2451070|37465|2451109|2390|41622|766316|3010|29304|7119|165991|6233|46527|28|2|9|1|263|261|92|72.74|210.21|23.12|17212.28|2127.04|6692.08|19339.32|21.27|0.00|7154.84|2127.04|2148.31|9281.88|9303.15|-4565.04| +2451070|37465|2451142|2846|41622|766316|3010|29304|7119|165991|6233|46527|32|22|10|5|43|261|20|73.26|169.96|151.26|374.00|3025.20|1465.20|3399.20|90.75|0.00|1665.60|3025.20|3115.95|4690.80|4781.55|1560.00| +2451070|37465|2451164|8329|41622|766316|3010|29304|7119|165991|6233|46527|52|28|3|1|189|261|84|64.95|181.21|134.09|3958.08|11263.56|5455.80|15221.64|4.50|11150.92|0.00|112.64|117.14|112.64|117.14|-5343.16| +2452258|11652|2452286|2467|68361|27001|4841|47881|18010|1852924|5001|37241|37|19|8|4|20|262|93|69.39|122.82|23.33|9252.57|2169.69|6453.27|11422.26|195.27|0.00|2055.30|2169.69|2364.96|4224.99|4420.26|-4283.58| +2452258|11652|2452280|9015|68361|27001|4841|47881|18010|1852924|5001|37241|21|17|17|3|236|262|37|10.72|22.19|16.86|197.21|623.82|396.64|821.03|24.95|0.00|49.21|623.82|648.77|673.03|697.98|227.18| +2452258|11652|2452346|9411|68361|27001|4841|47881|18010|1852924|5001|37241|17|17|12|2|265|262|24|44.90|132.45|23.84|2606.64|572.16|1077.60|3178.80|51.49|0.00|635.76|572.16|623.65|1207.92|1259.41|-505.44| +2452258|11652|2452272|5445|68361|27001|4841|47881|18010|1852924|5001|37241|43|9|5|3|111|262|100|4.78|5.16|3.40|176.00|340.00|478.00|516.00|17.00|0.00|180.00|340.00|357.00|520.00|537.00|-138.00| +2452258|11652|2452367|11873|68361|27001|4841|47881|18010|1852924|5001|37241|13|13|20|3|83|262|67|2.79|5.35|0.37|333.66|24.79|186.93|358.45|0.74|0.00|168.17|24.79|25.53|192.96|193.70|-162.14| +2452258|11652|2452272|1033|68361|27001|4841|47881|18010|1852924|5001|37241|59|11|12|3|84|262|55|44.15|73.73|42.02|1744.05|2311.10|2428.25|4055.15|161.77|0.00|1621.95|2311.10|2472.87|3933.05|4094.82|-117.15| +2452258|11652|2452273|14021|68361|27001|4841|47881|18010|1852924|5001|37241|13|9|20|5|43|262|18|23.26|31.63|4.42|489.78|79.56|418.68|569.34|0.00|0.00|267.48|79.56|79.56|347.04|347.04|-339.12| +2452258|11652|2452351|6799|68361|27001|4841|47881|18010|1852924|5001|37241|1|7|4|2|147|262|92|71.13|108.11|107.02|100.28|9845.84|6543.96|9946.12|43.32|7679.75|1789.40|2166.09|2209.41|3955.49|3998.81|-4377.87| +2452258|11652|2452300|8715|68361|27001|4841|47881|18010|1852924|5001|37241|45|1|6|4|122|262|63|76.20|151.63|144.04|478.17|9074.52|4800.60|9552.69|451.00|2631.61|1814.40|6442.91|6893.91|8257.31|8708.31|1642.31| +2452258|11652|2452343|13349|68361|27001|4841|47881|18010|1852924|5001|37241|1|11|11|1|111|262|85|31.37|45.80|25.64|1713.60|2179.40|2666.45|3893.00|0.00|0.00|1673.65|2179.40|2179.40|3853.05|3853.05|-487.05| +2452258|11652|2452331|17903|68361|27001|4841|47881|18010|1852924|5001|37241|21|15|5|4|36|262|94|97.17|148.67|20.81|12018.84|1956.14|9133.98|13974.98|78.24|0.00|4611.64|1956.14|2034.38|6567.78|6646.02|-7177.84| +2452258|11652|2452353|10815|68361|27001|4841|47881|18010|1852924|5001|37241|45|9|10|2|267|262|78|47.22|94.91|13.28|6367.14|1035.84|3683.16|7402.98|51.79|0.00|2590.38|1035.84|1087.63|3626.22|3678.01|-2647.32| +2452258|11652|2452328|5409|68361|27001|4841|47881|18010|1852924|5001|37241|11|5|17|1|75|262|1|24.81|50.11|47.60|2.51|47.60|24.81|50.11|1.33|14.28|18.54|33.32|34.65|51.86|53.19|8.51| +2452258|11652|2452271|3079|68361|27001|4841|47881|18010|1852924|5001|37241|47|21|5|2|39|262|32|10.79|17.80|4.27|432.96|136.64|345.28|569.60|9.56|0.00|222.08|136.64|146.20|358.72|368.28|-208.64| +2452258|11652|2452368|13655|68361|27001|4841|47881|18010|1852924|5001|37241|7|1|15|2|269|262|55|55.71|91.36|64.86|1457.50|3567.30|3064.05|5024.80|321.05|0.00|1104.95|3567.30|3888.35|4672.25|4993.30|503.25| +2451114|82585|2451152|10846|40503|1259889|6475|45861|69007|657719|1172|19358|34|7|10|1|202|263|48|79.55|112.96|109.57|162.72|5259.36|3818.40|5422.08|83.09|1104.46|1788.96|4154.90|4237.99|5943.86|6026.95|336.50| +2451114|82585|2451173|9280|40503|1259889|6475|45861|69007|657719|1172|19358|10|22|18|4|270|263|88|66.86|165.81|119.38|4085.84|10505.44|5883.68|14591.28|525.27|0.00|291.28|10505.44|11030.71|10796.72|11321.99|4621.76| +2451114|82585|2451162|12752|40503|1259889|6475|45861|69007|657719|1172|19358|40|25|5|2|229|263|75|17.68|31.29|10.01|1596.00|750.75|1326.00|2346.75|22.52|0.00|23.25|750.75|773.27|774.00|796.52|-575.25| +2451114|82585|2451229|10516|40503|1259889|6475|45861|69007|657719|1172|19358|26|22|3|1|299|263|84|73.16|176.31|128.70|3999.24|10810.80|6145.44|14810.04|972.97|0.00|0.00|10810.80|11783.77|10810.80|11783.77|4665.36| +2451114|82585|2451150|17618|40503|1259889|6475|45861|69007|657719|1172|19358|55|4|8|1|251|263|57|25.86|66.46|56.49|568.29|3219.93|1474.02|3788.22|40.24|2414.94|567.72|804.99|845.23|1372.71|1412.95|-669.03| +2451114|82585|2451130|5006|40503|1259889|6475|45861|69007|657719|1172|19358|26|28|19|3|124|263|43|61.47|184.41|180.72|158.67|7770.96|2643.21|7929.63|543.96|0.00|3568.14|7770.96|8314.92|11339.10|11883.06|5127.75| +2451114|82585|2451207|11929|40503|1259889|6475|45861|69007|657719|1172|19358|4|1|12|3|34|263|15|66.24|97.37|11.68|1285.35|175.20|993.60|1460.55|15.76|0.00|58.35|175.20|190.96|233.55|249.31|-818.40| +2451114|82585|2451143|10432|40503|1259889|6475|45861|69007|657719|1172|19358|52|13|20|5|293|263|59|13.27|36.62|5.12|1858.50|302.08|782.93|2160.58|6.04|0.00|669.65|302.08|308.12|971.73|977.77|-480.85| +2451114|82585|2451152|3806|40503|1259889|6475|45861|69007|657719|1172|19358|31|28|17|1|208|263|89|42.68|94.32|3.77|8058.95|335.53|3798.52|8394.48|13.42|0.00|839.27|335.53|348.95|1174.80|1188.22|-3462.99| +2451114|82585|2451185|9490|40503|1259889|6475|45861|69007|657719|1172|19358|58|25|19|2|45|263|54|18.31|52.36|36.65|848.34|1979.10|988.74|2827.44|79.16|0.00|593.46|1979.10|2058.26|2572.56|2651.72|990.36| +2451114|82585|2451228|10147|40503|1259889|6475|45861|69007|657719|1172|19358|16|25|4|3|69|263|58|54.56|139.12|93.21|2662.78|5406.18|3164.48|8068.96|16.21|4595.25|1532.94|810.93|827.14|2343.87|2360.08|-2353.55| +2451114|82585|2451168|17827|40503|1259889|6475|45861|69007|657719|1172|19358|16|20|16|3|153|263|35|27.18|45.66|42.46|112.00|1486.10|951.30|1598.10|59.44|0.00|591.15|1486.10|1545.54|2077.25|2136.69|534.80| +2451114|82585|2451170|6080|40503|1259889|6475|45861|69007|657719|1172|19358|4|16|15|1|244|263|20|1.07|2.80|2.68|2.40|53.60|21.40|56.00|0.57|39.12|0.40|14.48|15.05|14.88|15.45|-6.92| +2451130|58204|2451148|112|98037|1202021|3252|14483|35336|97870|4529|47313|50|13|1|2|206|264|35|62.18|82.07|5.74|2671.55|200.90|2176.30|2872.45|2.00|0.00|1177.40|200.90|202.90|1378.30|1380.30|-1975.40| +2451130|58204|2451204|16981|98037|1202021|3252|14483|35336|97870|4529|47313|13|10|12|3|84|264|7|69.29|121.25|76.38|314.09|534.66|485.03|848.75|5.34|0.00|390.39|534.66|540.00|925.05|930.39|49.63| +2451130|58204|2451154|9595|98037|1202021|3252|14483|35336|97870|4529|47313|26|1|8|5|82|264|18|34.76|76.47|55.82|371.70|1004.76|625.68|1376.46|0.00|0.00|192.60|1004.76|1004.76|1197.36|1197.36|379.08| +2451130|58204|2451136|7250|98037|1202021|3252|14483|35336|97870|4529|47313|20|2|20|1|12|264|50|97.29|103.12|79.40|1186.00|3970.00|4864.50|5156.00|198.50|0.00|2474.50|3970.00|4168.50|6444.50|6643.00|-894.50| +2451130|58204|2451191|16418|98037|1202021|3252|14483|35336|97870|4529|47313|8|7|3|2|278|264|60|10.17|24.40|18.30|366.00|1098.00|610.20|1464.00|32.94|0.00|732.00|1098.00|1130.94|1830.00|1862.94|487.80| +2451130|58204|2451193|8128|98037|1202021|3252|14483|35336|97870|4529|47313|43|8|11|4|37|264|63|82.30|130.85|19.62|7007.49|1236.06|5184.90|8243.55|37.08|0.00|2390.22|1236.06|1273.14|3626.28|3663.36|-3948.84| +2451130|58204|2451191|13711|98037|1202021|3252|14483|35336|97870|4529|47313|10|25|3|5|170|264|17|66.10|191.02|32.47|2695.35|551.99|1123.70|3247.34|27.59|0.00|1266.33|551.99|579.58|1818.32|1845.91|-571.71| +2451130|58204|2451193|7918|98037|1202021|3252|14483|35336|97870|4529|47313|2|1|3|2|141|264|86|57.63|57.63|47.25|892.68|4063.50|4956.18|4956.18|39.00|2113.02|1982.30|1950.48|1989.48|3932.78|3971.78|-3005.70| +2451130|58204|2451149|13420|98037|1202021|3252|14483|35336|97870|4529|47313|25|2|1|2|6|264|51|75.34|159.72|83.05|3910.17|4235.55|3842.34|8145.72|42.35|0.00|2199.12|4235.55|4277.90|6434.67|6477.02|393.21| +2451130|58204|2451202|9700|98037|1202021|3252|14483|35336|97870|4529|47313|58|28|3|3|226|264|5|25.05|39.32|39.32|0.00|196.60|125.25|196.60|11.79|0.00|27.50|196.60|208.39|224.10|235.89|71.35| +2451130|58204|2451147|1640|98037|1202021|3252|14483|35336|97870|4529|47313|25|2|11|5|269|264|21|22.98|38.60|35.51|64.89|745.71|482.58|810.60|14.91|0.00|72.87|745.71|760.62|818.58|833.49|263.13| +2451130|58204|2451242|4099|98037|1202021|3252|14483|35336|97870|4529|47313|52|1|18|2|59|264|86|9.97|14.85|1.18|1175.62|101.48|857.42|1277.10|0.77|62.91|369.80|38.57|39.34|408.37|409.14|-818.85| +2451130|58204|2451239|8956|98037|1202021|3252|14483|35336|97870|4529|47313|32|26|1|5|192|264|36|42.19|83.53|54.29|1052.64|1954.44|1518.84|3007.08|78.17|0.00|210.24|1954.44|2032.61|2164.68|2242.85|435.60| +2451130|58204|2451169|17131|98037|1202021|3252|14483|35336|97870|4529|47313|40|7|20|3|29|264|46|65.66|163.49|73.57|4136.32|3384.22|3020.36|7520.54|135.36|0.00|2932.96|3384.22|3519.58|6317.18|6452.54|363.86| +2451130|58204|2451142|7831|98037|1202021|3252|14483|35336|97870|4529|47313|13|10|2|1|47|264|39|4.08|9.58|2.77|265.59|108.03|159.12|373.62|8.64|0.00|67.08|108.03|116.67|175.11|183.75|-51.09| +2451130|58204|2451221|12824|98037|1202021|3252|14483|35336|97870|4529|47313|1|13|3|4|145|264|83|57.10|88.50|69.03|1616.01|5729.49|4739.30|7345.50|458.35|0.00|2423.60|5729.49|6187.84|8153.09|8611.44|990.19| +2452025|18101|2452138|15389|37606|465290|5327|43011|29695|150092|5569|44453|57|11|20|4|168|265|70|90.10|183.80|178.28|386.40|12479.60|6307.00|12866.00|748.77|0.00|3344.60|12479.60|13228.37|15824.20|16572.97|6172.60| +2452025|18101|2452079|11729|37606|465290|5327|43011|29695|150092|5569|44453|29|5|11|1|244|265|43|85.35|204.84|83.98|5196.98|3611.14|3670.05|8808.12|144.44|0.00|0.00|3611.14|3755.58|3611.14|3755.58|-58.91| +2452025|18101|2452068|6591|37606|465290|5327|43011|29695|150092|5569|44453|41|13|10|1|279|265|5|69.00|113.16|23.76|447.00|118.80|345.00|565.80|4.75|0.00|282.90|118.80|123.55|401.70|406.45|-226.20| +2452025|18101|2452094|3071|37606|465290|5327|43011|29695|150092|5569|44453|41|29|6|5|191|265|61|70.98|114.98|43.69|4348.69|2665.09|4329.78|7013.78|53.30|0.00|2314.34|2665.09|2718.39|4979.43|5032.73|-1664.69| +2452025|18101|2452070|5953|37606|465290|5327|43011|29695|150092|5569|44453|17|3|18|4|230|265|25|45.47|88.21|19.40|1720.25|485.00|1136.75|2205.25|29.10|0.00|815.75|485.00|514.10|1300.75|1329.85|-651.75| +2452025|18101|2452028|17489|37606|465290|5327|43011|29695|150092|5569|44453|29|11|4|5|143|265|79|49.99|126.47|27.82|7793.35|2197.78|3949.21|9991.13|65.93|0.00|2797.39|2197.78|2263.71|4995.17|5061.10|-1751.43| +2452025|18101|2452038|9387|37606|465290|5327|43011|29695|150092|5569|44453|57|19|8|1|177|265|64|23.73|46.27|12.95|2132.48|828.80|1518.72|2961.28|8.28|0.00|1480.32|828.80|837.08|2309.12|2317.40|-689.92| +2452025|18101|2452105|5123|37606|465290|5327|43011|29695|150092|5569|44453|35|17|20|3|19|265|8|34.27|58.25|14.56|349.52|116.48|274.16|466.00|0.80|36.10|9.28|80.38|81.18|89.66|90.46|-193.78| +2452025|18101|2452052|13255|37606|465290|5327|43011|29695|150092|5569|44453|45|11|2|5|188|265|72|2.28|4.28|2.61|120.24|187.92|164.16|308.16|11.27|0.00|135.36|187.92|199.19|323.28|334.55|23.76| +2452025|18101|2452082|14699|37606|465290|5327|43011|29695|150092|5569|44453|17|29|18|1|257|265|22|65.03|74.78|7.47|1480.82|164.34|1430.66|1645.16|4.93|0.00|65.78|164.34|169.27|230.12|235.05|-1266.32| +2452025|18101|2452117|2147|37606|465290|5327|43011|29695|150092|5569|44453|25|29|4|1|88|265|69|49.23|135.87|32.60|7125.63|2249.40|3396.87|9375.03|134.96|0.00|937.02|2249.40|2384.36|3186.42|3321.38|-1147.47| +2452025|18101|2452117|13701|37606|465290|5327|43011|29695|150092|5569|44453|55|3|6|5|194|265|28|91.10|263.27|115.83|4128.32|3243.24|2550.80|7371.56|291.89|0.00|3612.00|3243.24|3535.13|6855.24|7147.13|692.44| +2452025|18101|2452031|7095|37606|465290|5327|43011|29695|150092|5569|44453|17|13|15|3|61|265|45|49.87|145.62|46.59|4456.35|2096.55|2244.15|6552.90|83.86|0.00|2096.55|2096.55|2180.41|4193.10|4276.96|-147.60| +2452025|18101|2452102|13237|37606|465290|5327|43011|29695|150092|5569|44453|9|1|4|5|3|265|94|2.56|4.88|0.29|431.46|27.26|240.64|458.72|0.28|23.17|36.66|4.09|4.37|40.75|41.03|-236.55| +2450822|66029|2450847|7312|6253|554952|451|2350|57425|1590076|2447|47146|56|26|2|5|98|266|1|36.21|102.47|87.09|15.38|87.09|36.21|102.47|7.83|0.00|34.83|87.09|94.92|121.92|129.75|50.88| +2450822|66029|2450859|3650|6253|554952|451|2350|57425|1590076|2447|47146|49|28|5|2|232|266|9|91.35|262.17|117.97|1297.80|1061.73|822.15|2359.53|84.93|0.00|660.60|1061.73|1146.66|1722.33|1807.26|239.58| +2450822|66029|2450839|3832|6253|554952|451|2350|57425|1590076|2447|47146|44|7|8|4|300|266|90|18.99|37.03|14.07|2066.40|1266.30|1709.10|3332.70|113.96|0.00|832.50|1266.30|1380.26|2098.80|2212.76|-442.80| +2450822|66029|2450904|6613|6253|554952|451|2350|57425|1590076|2447|47146|2|4|14|5|6|266|25|66.85|157.76|132.51|631.25|3312.75|1671.25|3944.00|198.76|0.00|1104.25|3312.75|3511.51|4417.00|4615.76|1641.50| +2450822|66029|2450839|15403|6253|554952|451|2350|57425|1590076|2447|47146|13|10|5|4|3|266|73|79.96|173.51|154.42|1393.57|11272.66|5837.08|12666.23|789.08|0.00|1773.17|11272.66|12061.74|13045.83|13834.91|5435.58| +2450822|66029|2450827|8930|6253|554952|451|2350|57425|1590076|2447|47146|56|22|15|3|21|266|40|31.28|71.00|12.78|2328.80|511.20|1251.20|2840.00|46.00|0.00|596.40|511.20|557.20|1107.60|1153.60|-740.00| +2450822|66029|2450936|11092|6253|554952|451|2350|57425|1590076|2447|47146|55|22|16|2|151|266|66|87.23|157.88|146.82|729.96|9690.12|5757.18|10420.08|184.11|484.50|3542.22|9205.62|9389.73|12747.84|12931.95|3448.44| +2450822|66029|2450916|11192|6253|554952|451|2350|57425|1590076|2447|47146|2|26|6|5|297|266|69|50.43|95.31|63.85|2170.74|4405.65|3479.67|6576.39|0.00|0.00|2958.72|4405.65|4405.65|7364.37|7364.37|925.98| +2450822|66029|2450924|5948|6253|554952|451|2350|57425|1590076|2447|47146|4|28|14|1|238|266|31|87.33|115.27|47.26|2108.31|1465.06|2707.23|3573.37|14.65|0.00|964.72|1465.06|1479.71|2429.78|2444.43|-1242.17| +2450822|66029|2450853|9235|6253|554952|451|2350|57425|1590076|2447|47146|32|26|1|3|215|266|11|72.13|144.98|66.69|861.19|733.59|793.43|1594.78|29.34|0.00|302.94|733.59|762.93|1036.53|1065.87|-59.84| +2450822|66029|2450938|16826|6253|554952|451|2350|57425|1590076|2447|47146|32|8|20|3|36|266|88|39.26|61.24|56.34|431.20|4957.92|3454.88|5389.12|154.68|2379.80|1562.00|2578.12|2732.80|4140.12|4294.80|-876.76| +2450822|66029|2450868|11410|6253|554952|451|2350|57425|1590076|2447|47146|40|4|12|4|113|266|36|70.00|111.30|30.05|2925.00|1081.80|2520.00|4006.80|10.81|0.00|1923.12|1081.80|1092.61|3004.92|3015.73|-1438.20| +2450822|66029|2450859|7516|6253|554952|451|2350|57425|1590076|2447|47146|14|4|5|5|85|266|42|68.23|90.06|49.53|1702.26|2080.26|2865.66|3782.52|12.48|1872.23|794.22|208.03|220.51|1002.25|1014.73|-2657.63| +2450822|66029|2450845|14044|6253|554952|451|2350|57425|1590076|2447|47146|56|22|2|2|12|266|41|67.73|197.77|61.30|5595.27|2513.30|2776.93|8108.57|100.53|0.00|3405.46|2513.30|2613.83|5918.76|6019.29|-263.63| +2450822|66029|2450915|12547|6253|554952|451|2350|57425|1590076|2447|47146|50|8|14|3|62|266|72|17.87|30.02|12.30|1275.84|885.60|1286.64|2161.44|17.71|0.00|518.40|885.60|903.31|1404.00|1421.71|-401.04| +2450822|66029|2450845|13994|6253|554952|451|2350|57425|1590076|2447|47146|10|14|20|2|284|266|91|57.39|137.73|125.33|1128.40|11405.03|5222.49|12533.43|684.30|0.00|4637.36|11405.03|12089.33|16042.39|16726.69|6182.54| +2451175|73939|2451272|12296|93306|1783931|6909|5934|68844|1881174|6915|340|16|4|10|4|8|267|93|9.83|16.51|16.17|31.62|1503.81|914.19|1535.43|60.15|0.00|567.30|1503.81|1563.96|2071.11|2131.26|589.62| +2451175|73939|2451190|9091|93306|1783931|6909|5934|68844|1881174|6915|340|38|4|16|2|245|267|8|6.78|7.93|1.82|48.88|14.56|54.24|63.44|0.00|3.34|7.60|11.22|11.22|18.82|18.82|-43.02| +2451175|73939|2451254|10816|93306|1783931|6909|5934|68844|1881174|6915|340|50|2|18|4|147|267|62|96.25|181.91|165.53|1015.56|10262.86|5967.50|11278.42|0.00|4926.17|5300.38|5336.69|5336.69|10637.07|10637.07|-630.81| +2451175|73939|2451242|16436|93306|1783931|6909|5934|68844|1881174|6915|340|28|25|18|1|104|267|35|22.86|26.74|23.53|112.35|823.55|800.10|935.90|32.94|0.00|467.95|823.55|856.49|1291.50|1324.44|23.45| +2451175|73939|2451267|2578|93306|1783931|6909|5934|68844|1881174|6915|340|2|26|16|3|20|267|94|86.36|230.58|115.29|10837.26|10837.26|8117.84|21674.52|433.49|0.00|5851.50|10837.26|11270.75|16688.76|17122.25|2719.42| +2451175|73939|2451295|17557|93306|1783931|6909|5934|68844|1881174|6915|340|8|10|10|1|235|267|83|96.62|282.13|104.38|14753.25|8663.54|8019.46|23416.79|693.08|0.00|1873.31|8663.54|9356.62|10536.85|11229.93|644.08| +2451175|73939|2451242|6046|93306|1783931|6909|5934|68844|1881174|6915|340|38|25|16|4|135|267|70|17.20|18.23|12.94|370.30|905.80|1204.00|1276.10|72.46|0.00|484.40|905.80|978.26|1390.20|1462.66|-298.20| +2451175|73939|2451188|4603|93306|1783931|6909|5934|68844|1881174|6915|340|40|8|12|1|153|267|78|26.07|36.49|2.91|2619.24|226.98|2033.46|2846.22|11.34|0.00|56.16|226.98|238.32|283.14|294.48|-1806.48| +2451175|73939|2451203|7562|93306|1783931|6909|5934|68844|1881174|6915|340|10|20|19|2|42|267|44|51.70|113.22|83.78|1295.36|3686.32|2274.80|4981.68|110.58|0.00|2341.24|3686.32|3796.90|6027.56|6138.14|1411.52| +2451175|73939|2451192|9823|93306|1783931|6909|5934|68844|1881174|6915|340|2|25|7|3|62|267|7|45.41|104.44|5.22|694.54|36.54|317.87|731.08|2.19|0.00|168.14|36.54|38.73|204.68|206.87|-281.33| +2451175|73939|2451283|2246|93306|1783931|6909|5934|68844|1881174|6915|340|22|2|5|3|251|267|79|14.46|19.37|8.91|826.34|703.89|1142.34|1530.23|28.15|0.00|412.38|703.89|732.04|1116.27|1144.42|-438.45| +2451175|73939|2451277|13142|93306|1783931|6909|5934|68844|1881174|6915|340|34|2|16|5|104|267|97|94.68|102.25|32.72|6744.41|3173.84|9183.96|9918.25|190.43|0.00|3768.45|3173.84|3364.27|6942.29|7132.72|-6010.12| +2451104|51687|2451170|6649|14849|270345|594|21643|17068|1359286|1237|19486|22|20|13|2|31|268|95|16.12|22.08|15.67|608.95|1488.65|1531.40|2097.60|0.00|0.00|146.30|1488.65|1488.65|1634.95|1634.95|-42.75| +2451104|51687|2451191|8122|14849|270345|594|21643|17068|1359286|1237|19486|28|16|6|5|277|268|39|78.22|117.33|80.95|1418.82|3157.05|3050.58|4575.87|73.87|2336.21|1875.90|820.84|894.71|2696.74|2770.61|-2229.74| +2451104|51687|2451166|10514|14849|270345|594|21643|17068|1359286|1237|19486|20|14|18|5|202|268|86|23.91|28.21|0.28|2401.98|24.08|2056.26|2426.06|0.72|0.00|48.16|24.08|24.80|72.24|72.96|-2032.18| +2451104|51687|2451112|15458|14849|270345|594|21643|17068|1359286|1237|19486|52|13|20|5|30|268|31|70.63|155.38|41.95|3516.33|1300.45|2189.53|4816.78|117.04|0.00|1155.99|1300.45|1417.49|2456.44|2573.48|-889.08| +2451104|51687|2451202|11168|14849|270345|594|21643|17068|1359286|1237|19486|10|4|17|4|31|268|9|96.09|245.02|222.96|198.54|2006.64|864.81|2205.18|28.89|1043.45|441.00|963.19|992.08|1404.19|1433.08|98.38| +2451104|51687|2451153|2620|14849|270345|594|21643|17068|1359286|1237|19486|46|28|3|4|175|268|20|69.93|141.95|4.25|2754.00|85.00|1398.60|2839.00|5.95|0.00|652.80|85.00|90.95|737.80|743.75|-1313.60| +2451104|51687|2451195|11257|14849|270345|594|21643|17068|1359286|1237|19486|22|26|16|3|87|268|45|99.85|173.73|111.18|2814.75|5003.10|4493.25|7817.85|51.03|4152.57|3674.25|850.53|901.56|4524.78|4575.81|-3642.72| +2451104|51687|2451122|3973|14849|270345|594|21643|17068|1359286|1237|19486|25|22|1|2|106|268|9|87.95|192.61|119.41|658.80|1074.69|791.55|1733.49|0.00|0.00|346.68|1074.69|1074.69|1421.37|1421.37|283.14| +2451104|51687|2451187|11750|14849|270345|594|21643|17068|1359286|1237|19486|52|22|20|3|126|268|75|19.43|29.92|25.13|359.25|1884.75|1457.25|2244.00|0.00|0.00|582.75|1884.75|1884.75|2467.50|2467.50|427.50| +2451104|51687|2451215|5314|14849|270345|594|21643|17068|1359286|1237|19486|44|1|16|4|279|268|35|9.66|13.13|3.28|344.75|114.80|338.10|459.55|4.82|45.92|114.80|68.88|73.70|183.68|188.50|-269.22| +2451104|51687|2451149|17762|14849|270345|594|21643|17068|1359286|1237|19486|7|2|13|1|49|268|24|68.86|181.79|169.06|305.52|4057.44|1652.64|4362.96|202.87|0.00|2050.56|4057.44|4260.31|6108.00|6310.87|2404.80| +2451104|51687|2451186|16454|14849|270345|594|21643|17068|1359286|1237|19486|13|10|8|5|19|268|26|54.48|57.74|37.53|525.46|975.78|1416.48|1501.24|0.00|0.00|630.50|975.78|975.78|1606.28|1606.28|-440.70| +2451104|51687|2451200|847|14849|270345|594|21643|17068|1359286|1237|19486|37|16|9|4|189|268|99|96.63|128.51|95.09|3308.58|9413.91|9566.37|12722.49|847.25|0.00|890.01|9413.91|10261.16|10303.92|11151.17|-152.46| +2451104|51687|2451164|7579|14849|270345|594|21643|17068|1359286|1237|19486|25|16|5|1|237|268|25|20.06|53.76|6.45|1182.75|161.25|501.50|1344.00|4.83|0.00|53.75|161.25|166.08|215.00|219.83|-340.25| +2451104|51687|2451120|13045|14849|270345|594|21643|17068|1359286|1237|19486|20|14|18|5|165|268|29|25.11|64.53|31.61|954.68|916.69|728.19|1871.37|9.16|0.00|898.13|916.69|925.85|1814.82|1823.98|188.50| +2451104|51687|2451131|17198|14849|270345|594|21643|17068|1359286|1237|19486|13|10|18|5|282|268|54|2.03|4.06|1.05|162.54|56.70|109.62|219.24|0.56|0.00|65.34|56.70|57.26|122.04|122.60|-52.92| +2451089|81983|2451101|7126|37263|1889885|2507|1129|94075|1088250|2948|35970|25|22|4|1|115|269|90|95.24|200.00|184.00|1440.00|16560.00|8571.60|18000.00|1324.80|0.00|180.00|16560.00|17884.80|16740.00|18064.80|7988.40| +2451089|81983|2451184|16412|37263|1889885|2507|1129|94075|1088250|2948|35970|26|1|15|4|186|269|67|62.60|65.73|42.06|1585.89|2818.02|4194.20|4403.91|56.36|0.00|1453.23|2818.02|2874.38|4271.25|4327.61|-1376.18| +2451089|81983|2451203|967|37263|1889885|2507|1129|94075|1088250|2948|35970|52|19|3|1|79|269|73|40.72|48.04|37.95|736.57|2770.35|2972.56|3506.92|55.40|0.00|805.92|2770.35|2825.75|3576.27|3631.67|-202.21| +2451089|81983|2451141|1816|37263|1889885|2507|1129|94075|1088250|2948|35970|46|14|9|3|154|269|59|48.85|49.33|20.71|1688.58|1221.89|2882.15|2910.47|0.00|0.00|1454.94|1221.89|1221.89|2676.83|2676.83|-1660.26| +2451089|81983|2451177|3286|37263|1889885|2507|1129|94075|1088250|2948|35970|43|8|18|1|134|269|38|4.60|11.96|5.50|245.48|209.00|174.80|454.48|18.81|0.00|13.30|209.00|227.81|222.30|241.11|34.20| +2451089|81983|2451091|848|37263|1889885|2507|1129|94075|1088250|2948|35970|46|19|3|1|72|269|67|37.43|92.07|14.73|5181.78|986.91|2507.81|6168.69|9.86|0.00|2405.30|986.91|996.77|3392.21|3402.07|-1520.90| +2451089|81983|2451187|7714|37263|1889885|2507|1129|94075|1088250|2948|35970|8|7|13|2|241|269|64|68.90|118.50|36.73|5233.28|2350.72|4409.60|7584.00|188.05|0.00|2275.20|2350.72|2538.77|4625.92|4813.97|-2058.88| +2451089|81983|2451094|11266|37263|1889885|2507|1129|94075|1088250|2948|35970|28|13|9|3|230|269|71|32.44|80.45|42.63|2685.22|3026.73|2303.24|5711.95|108.96|847.48|113.60|2179.25|2288.21|2292.85|2401.81|-123.99| +2451089|81983|2451186|8173|37263|1889885|2507|1129|94075|1088250|2948|35970|52|2|17|2|107|269|64|8.19|22.03|9.03|832.00|577.92|524.16|1409.92|28.89|0.00|154.88|577.92|606.81|732.80|761.69|53.76| +2451089|81983|2451169|14150|37263|1889885|2507|1129|94075|1088250|2948|35970|58|13|11|3|65|269|65|46.34|112.60|12.38|6514.30|804.70|3012.10|7319.00|63.73|8.04|2561.65|796.66|860.39|3358.31|3422.04|-2215.44| +2451089|81983|2451138|6637|37263|1889885|2507|1129|94075|1088250|2948|35970|50|8|11|5|69|269|42|71.76|211.69|105.84|4445.70|4445.28|3013.92|8890.98|133.35|0.00|4356.24|4445.28|4578.63|8801.52|8934.87|1431.36| +2451089|81983|2451093|5108|37263|1889885|2507|1129|94075|1088250|2948|35970|31|7|9|4|12|269|10|1.56|2.40|1.96|4.40|19.60|15.60|24.00|0.39|0.00|3.80|19.60|19.99|23.40|23.79|4.00| +2451089|81983|2451170|16588|37263|1889885|2507|1129|94075|1088250|2948|35970|34|2|16|2|141|269|44|60.56|130.80|87.63|1899.48|3855.72|2664.64|5755.20|77.11|0.00|2359.28|3855.72|3932.83|6215.00|6292.11|1191.08| +2451089|81983|2451148|7165|37263|1889885|2507|1129|94075|1088250|2948|35970|37|22|15|3|171|269|45|78.74|186.61|78.37|4870.80|3526.65|3543.30|8397.45|282.13|0.00|1763.10|3526.65|3808.78|5289.75|5571.88|-16.65| +2451667|74012|2451735|6691|60345|1802113|6281|40096|30479|1080038|647|35215|55|23|16|5|114|270|25|84.85|249.45|82.31|4178.50|2057.75|2121.25|6236.25|0.00|0.00|1184.75|2057.75|2057.75|3242.50|3242.50|-63.50| +2451667|74012|2451746|17831|60345|1802113|6281|40096|30479|1080038|647|35215|14|5|11|1|191|270|73|47.91|137.98|96.58|3022.20|7050.34|3497.43|10072.54|141.00|0.00|2316.29|7050.34|7191.34|9366.63|9507.63|3552.91| +2451667|74012|2451668|14753|60345|1802113|6281|40096|30479|1080038|647|35215|41|2|13|4|185|270|68|26.32|69.74|57.88|806.48|3935.84|1789.76|4742.32|196.79|0.00|711.28|3935.84|4132.63|4647.12|4843.91|2146.08| +2451667|74012|2451708|13513|60345|1802113|6281|40096|30479|1080038|647|35215|59|13|4|4|291|270|48|87.59|207.58|41.51|7971.36|1992.48|4204.32|9963.84|99.62|0.00|2789.76|1992.48|2092.10|4782.24|4881.86|-2211.84| +2451667|74012|2451731|1022|60345|1802113|6281|40096|30479|1080038|647|35215|11|23|1|2|226|270|59|33.93|73.96|17.01|3360.05|1003.59|2001.87|4363.64|70.25|0.00|217.71|1003.59|1073.84|1221.30|1291.55|-998.28| +2451667|74012|2451779|16993|60345|1802113|6281|40096|30479|1080038|647|35215|1|7|20|3|156|270|90|13.78|20.11|19.10|90.90|1719.00|1240.20|1809.90|30.08|1117.35|469.80|601.65|631.73|1071.45|1101.53|-638.55| +2451667|74012|2451702|1148|60345|1802113|6281|40096|30479|1080038|647|35215|25|14|10|5|157|270|26|99.20|164.67|158.08|171.34|4110.08|2579.20|4281.42|369.90|0.00|2012.14|4110.08|4479.98|6122.22|6492.12|1530.88| +2451667|74012|2451693|13661|60345|1802113|6281|40096|30479|1080038|647|35215|41|20|1|3|78|270|73|24.93|58.83|14.11|3264.56|1030.03|1819.89|4294.59|10.30|0.00|987.69|1030.03|1040.33|2017.72|2028.02|-789.86| +2451667|74012|2451696|16346|60345|1802113|6281|40096|30479|1080038|647|35215|25|19|7|4|8|270|28|79.81|232.24|141.66|2536.24|3966.48|2234.68|6502.72|317.31|0.00|910.28|3966.48|4283.79|4876.76|5194.07|1731.80| +2452493|76582|2452613|3013|55844|1589131|18|1705|36234|312044|3566|12853|27|3|10|3|243|271|98|81.87|102.33|42.97|5817.28|4211.06|8023.26|10028.34|294.77|0.00|4211.06|4211.06|4505.83|8422.12|8716.89|-3812.20| +2452493|76582|2452599|17425|55844|1589131|18|1705|36234|312044|3566|12853|31|13|18|4|40|271|99|70.75|191.02|61.12|12860.10|6050.88|7004.25|18910.98|0.00|2057.29|4159.98|3993.59|3993.59|8153.57|8153.57|-3010.66| +2452493|76582|2452520|12951|55844|1589131|18|1705|36234|312044|3566|12853|6|21|9|5|58|271|88|98.80|247.00|165.49|7172.88|14563.12|8694.40|21736.00|1019.41|0.00|9563.84|14563.12|15582.53|24126.96|25146.37|5868.72| +2452493|76582|2452565|456|55844|1589131|18|1705|36234|312044|3566|12853|42|24|3|4|128|271|73|83.40|198.49|3.96|14200.69|289.08|6088.20|14489.77|11.56|0.00|6230.55|289.08|300.64|6519.63|6531.19|-5799.12| +2452493|76582|2452537|4299|55844|1589131|18|1705|36234|312044|3566|12853|15|30|2|4|6|271|11|98.97|139.54|16.74|1350.80|184.14|1088.67|1534.94|14.73|0.00|629.31|184.14|198.87|813.45|828.18|-904.53| +2452493|76582|2452533|7639|55844|1589131|18|1705|36234|312044|3566|12853|39|6|16|4|282|271|13|51.54|145.34|0.00|1889.42|0.00|670.02|1889.42|0.00|0.00|547.82|0.00|0.00|547.82|547.82|-670.02| +2452493|76582|2452554|3360|55844|1589131|18|1705|36234|312044|3566|12853|45|25|6|1|59|271|73|11.07|11.07|8.85|162.06|646.05|808.11|808.11|22.22|90.44|363.54|555.61|577.83|919.15|941.37|-252.50| +2452493|76582|2452536|14085|55844|1589131|18|1705|36234|312044|3566|12853|1|24|10|1|149|271|30|28.92|67.09|2.01|1952.40|60.30|867.60|2012.70|4.22|0.00|603.60|60.30|64.52|663.90|668.12|-807.30| +2452493|76582|2452519|16837|55844|1589131|18|1705|36234|312044|3566|12853|21|30|16|5|41|271|40|24.51|65.44|43.19|890.00|1727.60|980.40|2617.60|46.64|172.76|1308.80|1554.84|1601.48|2863.64|2910.28|574.44| +2452261|69649|2452328|4967|20122|508462|5602|742|24496|1082053|5715|41255|23|23|9|4|26|272|7|5.66|16.01|8.64|51.59|60.48|39.62|112.07|3.62|0.00|31.36|60.48|64.10|91.84|95.46|20.86| +2452261|69649|2452278|12945|20122|508462|5602|742|24496|1082053|5715|41255|49|29|8|3|181|272|48|85.37|112.68|23.66|4272.96|1135.68|4097.76|5408.64|102.21|0.00|2433.60|1135.68|1237.89|3569.28|3671.49|-2962.08| +2452261|69649|2452284|15887|20122|508462|5602|742|24496|1082053|5715|41255|57|13|5|1|24|272|51|74.70|87.39|6.99|4100.40|356.49|3809.70|4456.89|24.95|0.00|846.60|356.49|381.44|1203.09|1228.04|-3453.21| +2452261|69649|2452366|9665|20122|508462|5602|742|24496|1082053|5715|41255|35|13|6|5|94|272|5|90.18|113.62|68.17|227.25|340.85|450.90|568.10|0.00|0.00|130.65|340.85|340.85|471.50|471.50|-110.05| +2452261|69649|2452364|13713|20122|508462|5602|742|24496|1082053|5715|41255|55|5|1|1|124|272|1|36.41|62.26|32.37|29.89|32.37|36.41|62.26|0.32|0.00|26.77|32.37|32.69|59.14|59.46|-4.04| +2452261|69649|2452303|7613|20122|508462|5602|742|24496|1082053|5715|41255|21|29|6|2|71|272|93|3.95|6.28|1.13|478.95|105.09|367.35|584.04|4.20|0.00|34.41|105.09|109.29|139.50|143.70|-262.26| +2452261|69649|2452365|17893|20122|508462|5602|742|24496|1082053|5715|41255|43|9|15|3|148|272|87|5.53|15.26|2.89|1076.19|251.43|481.11|1327.62|1.81|206.17|79.17|45.26|47.07|124.43|126.24|-435.85| +2452261|69649|2452325|2081|20122|508462|5602|742|24496|1082053|5715|41255|11|29|17|1|28|272|8|97.20|282.85|152.73|1040.96|1221.84|777.60|2262.80|24.43|0.00|995.60|1221.84|1246.27|2217.44|2241.87|444.24| +2452261|69649|2452356|979|20122|508462|5602|742|24496|1082053|5715|41255|15|17|20|1|275|272|6|85.27|148.36|111.27|222.54|667.62|511.62|890.16|26.70|0.00|178.02|667.62|694.32|845.64|872.34|156.00| +2452261|69649|2452373|11613|20122|508462|5602|742|24496|1082053|5715|41255|51|3|1|5|149|272|98|25.15|53.82|46.82|686.00|4588.36|2464.70|5274.36|91.76|0.00|1740.48|4588.36|4680.12|6328.84|6420.60|2123.66| +2452261|69649|2452301|5649|20122|508462|5602|742|24496|1082053|5715|41255|19|13|4|1|178|272|93|66.30|82.87|30.66|4855.53|2851.38|6165.90|7706.91|28.51|0.00|3545.16|2851.38|2879.89|6396.54|6425.05|-3314.52| +2452261|69649|2452375|117|20122|508462|5602|742|24496|1082053|5715|41255|29|5|18|1|4|272|81|62.98|102.02|70.39|2562.03|5701.59|5101.38|8263.62|342.09|0.00|2891.70|5701.59|6043.68|8593.29|8935.38|600.21| +2451835|54908|2451944|2384|14282|30177|472|16767|14282|30177|472|16767|26|25|19|3|257|273|51|42.44|70.02|51.11|964.41|2606.61|2164.44|3571.02|0.00|0.00|285.60|2606.61|2606.61|2892.21|2892.21|442.17| +2451835|54908|2451922|7891|14282|30177|472|16767|14282|30177|472|16767|41|13|20|1|122|273|94|61.08|63.52|32.39|2926.22|3044.66|5741.52|5970.88|60.89|0.00|596.90|3044.66|3105.55|3641.56|3702.45|-2696.86| +2451835|54908|2451933|10787|14282|30177|472|16767|14282|30177|472|16767|29|11|17|3|249|273|18|41.26|52.40|33.01|349.02|594.18|742.68|943.20|29.70|0.00|414.90|594.18|623.88|1009.08|1038.78|-148.50| +2451835|54908|2451946|17659|14282|30177|472|16767|14282|30177|472|16767|5|13|11|3|252|273|14|8.24|11.20|1.12|141.12|15.68|115.36|156.80|1.25|0.00|15.68|15.68|16.93|31.36|32.61|-99.68| +2451835|54908|2451851|9689|14282|30177|472|16767|14282|30177|472|16767|29|8|8|3|268|273|51|49.24|133.44|60.04|3743.40|3062.04|2511.24|6805.44|2.75|2786.45|1292.85|275.59|278.34|1568.44|1571.19|-2235.65| +2451835|54908|2451846|13121|14282|30177|472|16767|14282|30177|472|16767|14|25|18|3|28|273|52|26.95|30.45|29.84|31.72|1551.68|1401.40|1583.40|1.86|1520.64|553.80|31.04|32.90|584.84|586.70|-1370.36| +2451835|54908|2451877|8051|14282|30177|472|16767|14282|30177|472|16767|32|14|9|2|228|273|8|18.67|47.23|32.11|120.96|256.88|149.36|377.84|7.70|0.00|60.40|256.88|264.58|317.28|324.98|107.52| +2451835|54908|2451950|991|14282|30177|472|16767|14282|30177|472|16767|37|5|20|2|151|273|69|85.30|181.68|161.69|1379.31|11156.61|5885.70|12535.92|124.95|9371.55|250.47|1785.06|1910.01|2035.53|2160.48|-4100.64| +2451835|54908|2451906|8888|14282|30177|472|16767|14282|30177|472|16767|7|7|17|2|117|273|65|94.23|190.34|159.88|1979.90|10392.20|6124.95|12372.10|207.84|0.00|5938.40|10392.20|10600.04|16330.60|16538.44|4267.25| +2451835|54908|2451837|10201|14282|30177|472|16767|14282|30177|472|16767|25|25|6|1|269|273|71|53.88|57.65|21.33|2578.72|1514.43|3825.48|4093.15|10.90|424.04|859.10|1090.39|1101.29|1949.49|1960.39|-2735.09| +2451835|54908|2451879|11927|14282|30177|472|16767|14282|30177|472|16767|17|2|2|5|175|273|42|88.70|227.07|84.01|6008.52|3528.42|3725.40|9536.94|5.99|2928.58|381.36|599.84|605.83|981.20|987.19|-3125.56| +2451835|54908|2451955|5225|14282|30177|472|16767|14282|30177|472|16767|25|23|1|5|39|273|31|79.43|237.49|23.74|6626.25|735.94|2462.33|7362.19|44.15|0.00|883.19|735.94|780.09|1619.13|1663.28|-1726.39| +2451835|54908|2451911|3617|14282|30177|472|16767|14282|30177|472|16767|11|26|5|3|31|273|76|1.05|2.43|2.40|2.28|182.40|79.80|184.68|0.45|173.28|40.28|9.12|9.57|49.40|49.85|-70.68| +2451835|54908|2451863|16130|14282|30177|472|16767|14282|30177|472|16767|53|20|15|4|286|273|97|7.69|14.14|11.31|274.51|1097.07|745.93|1371.58|41.46|405.91|13.58|691.16|732.62|704.74|746.20|-54.77| +2451835|54908|2451945|15719|14282|30177|472|16767|14282|30177|472|16767|23|8|18|4|127|273|93|62.33|125.28|16.28|10137.00|1514.04|5796.69|11651.04|50.87|242.24|4776.48|1271.80|1322.67|6048.28|6099.15|-4524.89| +2451835|54908|2451846|17293|14282|30177|472|16767|14282|30177|472|16767|8|23|9|1|190|273|26|20.17|29.64|18.08|300.56|470.08|524.42|770.64|23.50|0.00|300.30|470.08|493.58|770.38|793.88|-54.34| +2451805|73463|2451837|1772|85102|1854755|5655|12555|76188|729690|3208|43708|53|2|7|4|264|274|92|55.36|84.14|79.09|464.60|7276.28|5093.12|7740.88|582.10|0.00|3018.52|7276.28|7858.38|10294.80|10876.90|2183.16| +2451805|73463|2451869|12938|85102|1854755|5655|12555|76188|729690|3208|43708|23|25|11|2|120|274|45|8.68|14.40|6.33|363.15|284.85|390.60|648.00|0.00|170.91|71.10|113.94|113.94|185.04|185.04|-276.66| +2451805|73463|2451904|17627|85102|1854755|5655|12555|76188|729690|3208|43708|56|13|15|3|211|274|25|63.20|157.36|80.25|1927.75|2006.25|1580.00|3934.00|7.02|1304.06|118.00|702.19|709.21|820.19|827.21|-877.81| +2451805|73463|2451846|8999|85102|1854755|5655|12555|76188|729690|3208|43708|20|8|9|3|287|274|5|69.26|159.29|23.89|677.00|119.45|346.30|796.45|0.50|102.72|262.80|16.73|17.23|279.53|280.03|-329.57| +2451805|73463|2451845|7490|85102|1854755|5655|12555|76188|729690|3208|43708|55|19|15|2|242|274|3|52.39|69.67|66.18|10.47|198.54|157.17|209.01|15.88|0.00|104.49|198.54|214.42|303.03|318.91|41.37| +2451805|73463|2451920|17630|85102|1854755|5655|12555|76188|729690|3208|43708|59|23|1|3|152|274|24|87.70|89.45|78.71|257.76|1889.04|2104.80|2146.80|18.89|0.00|257.52|1889.04|1907.93|2146.56|2165.45|-215.76| +2451805|73463|2451865|13976|85102|1854755|5655|12555|76188|729690|3208|43708|59|11|12|3|174|274|81|31.54|40.37|36.33|327.24|2942.73|2554.74|3269.97|28.83|58.85|1242.54|2883.88|2912.71|4126.42|4155.25|329.14| +2451805|73463|2451885|7393|85102|1854755|5655|12555|76188|729690|3208|43708|23|26|9|1|223|274|19|31.56|64.06|1.92|1180.66|36.48|599.64|1217.14|0.72|0.00|109.44|36.48|37.20|145.92|146.64|-563.16| +2451805|73463|2451909|9710|85102|1854755|5655|12555|76188|729690|3208|43708|50|29|4|1|125|274|67|95.68|155.00|88.35|4465.55|5919.45|6410.56|10385.00|473.55|0.00|2803.95|5919.45|6393.00|8723.40|9196.95|-491.11| +2452282|73796|2452308|3660|62303|776055|5007|15159|75784|619847|368|35320|21|25|15|5|12|275|23|12.44|34.45|18.25|372.60|419.75|286.12|792.35|3.69|373.57|134.55|46.18|49.87|180.73|184.42|-239.94| +2452282|73796|2452357|14580|62303|776055|5007|15159|75784|619847|368|35320|42|15|18|5|52|275|36|90.35|204.19|173.56|1102.68|6248.16|3252.60|7350.84|124.96|0.00|3675.24|6248.16|6373.12|9923.40|10048.36|2995.56| +2452282|73796|2452337|12672|62303|776055|5007|15159|75784|619847|368|35320|30|15|11|4|227|275|18|40.92|114.16|6.84|1931.76|123.12|736.56|2054.88|0.00|0.00|945.18|123.12|123.12|1068.30|1068.30|-613.44| +2452282|73796|2452398|7387|62303|776055|5007|15159|75784|619847|368|35320|36|21|17|5|283|275|92|47.97|98.33|24.58|6785.00|2261.36|4413.24|9046.36|0.00|0.00|2713.08|2261.36|2261.36|4974.44|4974.44|-2151.88| +2452282|73796|2452376|12789|62303|776055|5007|15159|75784|619847|368|35320|13|3|7|2|47|275|45|89.07|232.47|185.97|2092.50|8368.65|4008.15|10461.15|83.68|0.00|4393.35|8368.65|8452.33|12762.00|12845.68|4360.50| +2452282|73796|2452310|11589|62303|776055|5007|15159|75784|619847|368|35320|48|1|18|5|232|275|93|27.02|56.47|5.08|4779.27|472.44|2512.86|5251.71|28.34|0.00|1470.33|472.44|500.78|1942.77|1971.11|-2040.42| +2452282|73796|2452384|15765|62303|776055|5007|15159|75784|619847|368|35320|43|3|2|2|212|275|88|45.97|110.78|26.58|7409.60|2339.04|4045.36|9748.64|187.12|0.00|4874.32|2339.04|2526.16|7213.36|7400.48|-1706.32| +2452282|73796|2452362|14379|62303|776055|5007|15159|75784|619847|368|35320|51|25|10|2|244|275|6|9.96|19.62|18.44|7.08|110.64|59.76|117.72|8.85|0.00|45.90|110.64|119.49|156.54|165.39|50.88| +2452282|73796|2452334|4393|62303|776055|5007|15159|75784|619847|368|35320|13|3|14|4|91|275|1|12.55|16.56|8.94|7.62|8.94|12.55|16.56|0.08|0.00|7.78|8.94|9.02|16.72|16.80|-3.61| +2452282|73796|2452385|13014|62303|776055|5007|15159|75784|619847|368|35320|24|7|3|2|211|275|22|56.59|86.58|1.73|1866.70|38.06|1244.98|1904.76|1.52|0.00|742.72|38.06|39.58|780.78|782.30|-1206.92| +2452014|16614|2452069|14743|30390|948813|836|32059|75274|1147119|928|951|55|3|9|3|247|276|3|38.96|54.15|37.36|50.37|112.08|116.88|162.45|1.12|0.00|32.49|112.08|113.20|144.57|145.69|-4.80| +2452014|16614|2452134|13943|30390|948813|836|32059|75274|1147119|928|951|43|9|5|3|88|276|61|46.17|99.72|72.79|1642.73|4440.19|2816.37|6082.92|133.20|0.00|60.39|4440.19|4573.39|4500.58|4633.78|1623.82| +2452014|16614|2452102|12499|30390|948813|836|32059|75274|1147119|928|951|53|15|19|3|274|276|79|61.10|93.48|76.65|1329.57|6055.35|4826.90|7384.92|302.76|0.00|1107.58|6055.35|6358.11|7162.93|7465.69|1228.45| +2452014|16614|2452064|14359|30390|948813|836|32059|75274|1147119|928|951|9|5|8|1|101|276|2|10.93|16.06|15.57|0.98|31.14|21.86|32.12|1.24|0.00|10.92|31.14|32.38|42.06|43.30|9.28| +2452014|16614|2452059|7283|30390|948813|836|32059|75274|1147119|928|951|41|21|11|1|281|276|36|66.07|188.29|163.81|881.28|5897.16|2378.52|6778.44|471.77|0.00|203.04|5897.16|6368.93|6100.20|6571.97|3518.64| +2452014|16614|2452117|17269|30390|948813|836|32059|75274|1147119|928|951|15|9|12|2|259|276|70|6.97|11.36|8.86|175.00|620.20|487.90|795.20|12.40|0.00|102.90|620.20|632.60|723.10|735.50|132.30| +2452014|16614|2452108|1917|30390|948813|836|32059|75274|1147119|928|951|43|17|13|5|157|276|35|85.71|167.99|144.47|823.20|5056.45|2999.85|5879.65|0.00|0.00|1528.45|5056.45|5056.45|6584.90|6584.90|2056.60| +2452014|16614|2452078|1553|30390|948813|836|32059|75274|1147119|928|951|1|23|3|4|165|276|17|28.02|56.88|16.49|686.63|280.33|476.34|966.96|0.00|0.00|193.29|280.33|280.33|473.62|473.62|-196.01| +2452014|16614|2452034|11617|30390|948813|836|32059|75274|1147119|928|951|39|15|16|4|205|276|85|44.66|57.16|41.72|1312.40|3546.20|3796.10|4858.60|159.57|354.62|1943.10|3191.58|3351.15|5134.68|5294.25|-604.52| +2452014|16614|2452070|4495|30390|948813|836|32059|75274|1147119|928|951|19|19|10|2|197|276|57|53.57|132.31|91.29|2338.14|5203.53|3053.49|7541.67|171.19|2757.87|1734.51|2445.66|2616.85|4180.17|4351.36|-607.83| +2452014|16614|2452060|10669|30390|948813|836|32059|75274|1147119|928|951|43|29|8|1|114|276|44|26.93|67.86|29.17|1702.36|1283.48|1184.92|2985.84|38.50|0.00|1343.32|1283.48|1321.98|2626.80|2665.30|98.56| +2452014|16614|2452067|15779|30390|948813|836|32059|75274|1147119|928|951|53|21|15|1|150|276|56|97.02|194.04|54.33|7823.76|3042.48|5433.12|10866.24|152.12|0.00|1629.60|3042.48|3194.60|4672.08|4824.20|-2390.64| +2452014|16614|2452051|17543|30390|948813|836|32059|75274|1147119|928|951|13|13|17|2|277|276|39|35.91|67.15|13.43|2095.08|523.77|1400.49|2618.85|0.41|518.53|523.77|5.24|5.65|529.01|529.42|-1395.25| +2452014|16614|2452129|147|30390|948813|836|32059|75274|1147119|928|951|29|27|3|2|161|276|16|90.67|256.59|79.54|2832.80|1272.64|1450.72|4105.44|11.70|101.81|985.28|1170.83|1182.53|2156.11|2167.81|-279.89| +2452014|16614|2452117|3429|30390|948813|836|32059|75274|1147119|928|951|27|29|16|2|198|276|33|32.82|95.50|78.31|567.27|2584.23|1083.06|3151.50|129.21|0.00|850.74|2584.23|2713.44|3434.97|3564.18|1501.17| +2452014|16614|2452054|11673|30390|948813|836|32059|75274|1147119|928|951|17|21|16|3|190|276|33|93.38|122.32|47.70|2462.46|1574.10|3081.54|4036.56|141.66|0.00|847.44|1574.10|1715.76|2421.54|2563.20|-1507.44| +2450909|10145|2450968|12664|34203|158404|205|806|8456|345449|562|5963|26|28|6|4|115|277|15|5.09|7.17|2.58|68.85|38.70|76.35|107.55|3.09|0.00|19.35|38.70|41.79|58.05|61.14|-37.65| +2450909|10145|2450949|13753|34203|158404|205|806|8456|345449|562|5963|44|1|9|2|235|277|67|82.97|100.39|5.01|6390.46|335.67|5558.99|6726.13|26.85|0.00|2891.72|335.67|362.52|3227.39|3254.24|-5223.32| +2450909|10145|2450986|1396|34203|158404|205|806|8456|345449|562|5963|26|25|12|1|220|277|23|94.75|239.71|129.44|2536.21|2977.12|2179.25|5513.33|267.94|0.00|2150.04|2977.12|3245.06|5127.16|5395.10|797.87| +2450909|10145|2450968|14179|34203|158404|205|806|8456|345449|562|5963|58|25|3|1|209|277|22|12.67|15.07|11.75|73.04|258.50|278.74|331.54|11.89|20.68|159.06|237.82|249.71|396.88|408.77|-40.92| +2450909|10145|2450930|920|34203|158404|205|806|8456|345449|562|5963|22|22|2|2|192|277|19|32.57|58.30|33.23|476.33|631.37|618.83|1107.70|37.88|0.00|132.81|631.37|669.25|764.18|802.06|12.54| +2450909|10145|2450934|9697|34203|158404|205|806|8456|345449|562|5963|22|19|8|1|214|277|56|93.81|194.18|31.06|9134.72|1739.36|5253.36|10874.08|156.54|0.00|652.40|1739.36|1895.90|2391.76|2548.30|-3514.00| +2450909|10145|2450985|12145|34203|158404|205|806|8456|345449|562|5963|56|22|2|2|252|277|45|38.41|39.94|35.14|216.00|1581.30|1728.45|1797.30|126.50|0.00|808.65|1581.30|1707.80|2389.95|2516.45|-147.15| +2450909|10145|2450955|1208|34203|158404|205|806|8456|345449|562|5963|50|22|11|2|53|277|75|30.62|83.28|68.28|1125.00|5121.00|2296.50|6246.00|26.62|4455.27|1686.00|665.73|692.35|2351.73|2378.35|-1630.77| +2450909|10145|2451024|4492|34203|158404|205|806|8456|345449|562|5963|22|19|18|1|34|277|96|37.84|70.00|63.70|604.80|6115.20|3632.64|6720.00|401.76|1651.10|3158.40|4464.10|4865.86|7622.50|8024.26|831.46| +2450909|10145|2450935|15238|34203|158404|205|806|8456|345449|562|5963|14|10|7|5|256|277|78|70.64|202.03|129.29|5673.72|10084.62|5509.92|15758.34|16.13|8471.08|1260.48|1613.54|1629.67|2874.02|2890.15|-3896.38| +2450909|10145|2450976|5162|34203|158404|205|806|8456|345449|562|5963|10|19|11|5|126|277|35|39.02|47.60|15.70|1116.50|549.50|1365.70|1666.00|11.64|258.26|116.55|291.24|302.88|407.79|419.43|-1074.46| +2451081|62193|2451193|11092|69381|106955|4176|4550|43535|1575856|966|40540|37|10|3|5|27|278|60|12.78|35.78|10.01|1546.20|600.60|766.80|2146.80|12.01|0.00|1051.80|600.60|612.61|1652.40|1664.41|-166.20| +2451081|62193|2451143|11192|69381|106955|4176|4550|43535|1575856|966|40540|8|16|3|2|111|278|93|58.39|78.24|21.12|5312.16|1964.16|5430.27|7276.32|176.77|0.00|2764.89|1964.16|2140.93|4729.05|4905.82|-3466.11| +2451081|62193|2451108|5948|69381|106955|4176|4550|43535|1575856|966|40540|56|7|16|1|63|278|31|11.79|11.90|5.23|206.77|162.13|365.49|368.90|8.94|50.26|88.35|111.87|120.81|200.22|209.16|-253.62| +2451081|62193|2451118|9235|69381|106955|4176|4550|43535|1575856|966|40540|4|1|19|2|85|278|38|44.37|74.54|35.03|1501.38|1331.14|1686.06|2832.52|119.80|0.00|368.22|1331.14|1450.94|1699.36|1819.16|-354.92| +2451081|62193|2451111|16826|69381|106955|4176|4550|43535|1575856|966|40540|38|19|10|5|100|278|77|11.93|31.73|25.70|464.31|1978.90|918.61|2443.21|79.15|0.00|97.02|1978.90|2058.05|2075.92|2155.07|1060.29| +2451081|62193|2451085|11410|69381|106955|4176|4550|43535|1575856|966|40540|56|8|14|5|226|278|33|35.36|105.01|4.20|3326.73|138.60|1166.88|3465.33|6.93|0.00|0.00|138.60|145.53|138.60|145.53|-1028.28| +2451081|62193|2451167|7516|69381|106955|4176|4550|43535|1575856|966|40540|32|16|14|4|91|278|84|39.73|101.70|17.28|7091.28|1451.52|3337.32|8542.80|117.57|145.15|1879.08|1306.37|1423.94|3185.45|3303.02|-2030.95| +2451081|62193|2451140|14044|69381|106955|4176|4550|43535|1575856|966|40540|7|2|13|1|72|278|13|3.66|6.69|0.80|76.57|10.40|47.58|86.97|0.93|0.00|5.20|10.40|11.33|15.60|16.53|-37.18| +2451123|45811|2451188|13712|60489|719392|1048|45551|84615|296120|3060|37163|7|28|9|5|71|279|51|90.85|268.00|21.44|12574.56|1093.44|4633.35|13668.00|43.73|0.00|1776.84|1093.44|1137.17|2870.28|2914.01|-3539.91| +2451123|45811|2451140|7612|60489|719392|1048|45551|84615|296120|3060|37163|7|20|8|5|62|279|88|24.39|35.85|15.05|1830.40|1324.40|2146.32|3154.80|66.22|0.00|820.16|1324.40|1390.62|2144.56|2210.78|-821.92| +2451123|45811|2451218|17893|60489|719392|1048|45551|84615|296120|3060|37163|34|20|4|2|200|279|48|57.93|133.23|89.26|2110.56|4284.48|2780.64|6395.04|88.68|1328.18|959.04|2956.30|3044.98|3915.34|4004.02|175.66| +2451123|45811|2451207|2080|60489|719392|1048|45551|84615|296120|3060|37163|22|14|11|2|93|279|9|18.57|37.51|1.50|324.09|13.50|167.13|337.59|0.40|0.00|54.00|13.50|13.90|67.50|67.90|-153.63| +2451123|45811|2451212|979|60489|719392|1048|45551|84615|296120|3060|37163|1|7|17|3|246|279|38|83.51|209.61|178.16|1195.10|6770.08|3173.38|7965.18|67.70|0.00|1513.16|6770.08|6837.78|8283.24|8350.94|3596.70| +2451123|45811|2451161|11612|60489|719392|1048|45551|84615|296120|3060|37163|13|28|6|4|85|279|13|76.29|221.24|117.25|1351.87|1524.25|991.77|2876.12|137.18|0.00|1121.64|1524.25|1661.43|2645.89|2783.07|532.48| +2451123|45811|2451137|5648|60489|719392|1048|45551|84615|296120|3060|37163|38|19|20|3|33|279|69|94.90|198.34|170.57|1916.13|11769.33|6548.10|13685.46|117.69|0.00|3010.47|11769.33|11887.02|14779.80|14897.49|5221.23| +2451123|45811|2451155|116|60489|719392|1048|45551|84615|296120|3060|37163|14|16|7|5|93|279|97|84.69|247.29|79.13|16311.52|7675.61|8214.93|23987.13|52.96|5910.21|479.18|1765.40|1818.36|2244.58|2297.54|-6449.53| +2451177|70058|2451224|3494|27909|1899124|4386|4448|77468|316644|3387|1049|31|22|18|4|203|280|51|3.03|7.99|3.35|236.64|170.85|154.53|407.49|5.12|0.00|56.61|170.85|175.97|227.46|232.58|16.32| +2451177|70058|2451203|10600|27909|1899124|4386|4448|77468|316644|3387|1049|2|7|16|3|220|280|48|38.44|76.49|23.71|2533.44|1138.08|1845.12|3671.52|45.52|0.00|770.88|1138.08|1183.60|1908.96|1954.48|-707.04| +|70058|2451280|3256||||4448|77468|316644|||52|10|||70|280||||2.47||66.69||3340.17|2.00|0.00|1135.62|66.69|68.69|1202.31|1204.31|-2145.42| +2451177|70058|2451238|16382|27909|1899124|4386|4448|77468|316644|3387|1049|43|7|6|5|167|280|45|98.12|103.02|18.54|3801.60|834.30|4415.40|4635.90|50.05|0.00|1575.90|834.30|884.35|2410.20|2460.25|-3581.10| +2451177|70058|2451183|12241|27909|1899124|4386|4448|77468|316644|3387|1049|28|10|17|1|182|280|86|59.70|151.04|75.52|6494.72|6494.72|5134.20|12989.44|584.52|0.00|129.86|6494.72|7079.24|6624.58|7209.10|1360.52| +2451177|70058|2451201|13747|27909|1899124|4386|4448|77468|316644|3387|1049|22|8|15|3|208|280|41|75.14|167.56|142.42|1030.74|5839.22|3080.74|6869.96|233.56|0.00|1373.91|5839.22|6072.78|7213.13|7446.69|2758.48| +2451177|70058|2451242|2917|27909|1899124|4386|4448|77468|316644|3387|1049|49|22|9|1|3|280|88|49.51|88.12|2.64|7522.24|232.32|4356.88|7754.56|18.58|0.00|3411.76|232.32|250.90|3644.08|3662.66|-4124.56| +2451177|70058|2451182|4819|27909|1899124|4386|4448|77468|316644|3387|1049|19|1|19|5|161|280|30|51.46|103.43|47.57|1675.80|1427.10|1543.80|3102.90|28.54|0.00|1396.20|1427.10|1455.64|2823.30|2851.84|-116.70| +2451177|70058|2451185|2938|27909|1899124|4386|4448|77468|316644|3387|1049|8|19|15|5|122|280|15|37.34|72.06|30.26|627.00|453.90|560.10|1080.90|9.12|149.78|280.95|304.12|313.24|585.07|594.19|-255.98| +2451177|70058|2451216|15655|27909|1899124|4386|4448|77468|316644|3387|1049|34|16|10|1|42|280|74|46.86|136.36|31.36|7770.00|2320.64|3467.64|10090.64|162.44|0.00|3935.32|2320.64|2483.08|6255.96|6418.40|-1147.00| +2451177|70058|2451266|8662|27909|1899124|4386|4448|77468|316644|3387|1049|34|7|20|4|75|280|67|9.88|19.85|18.65|80.40|1249.55|661.96|1329.95|66.97|412.35|119.26|837.20|904.17|956.46|1023.43|175.24| +2451177|70058|2451242|10657|27909|1899124|4386|4448|77468|316644|3387|1049|44|13|1|2|26|280|63|98.82|281.63|214.03|4258.80|13483.89|6225.66|17742.69|1213.55|0.00|8338.68|13483.89|14697.44|21822.57|23036.12|7258.23| +2450857|25084|2450949|6550|5289|63690|491|33292|40309|924339|2895|37296|20|25|12|1|213|281|19|74.65|86.59|6.06|1530.07|115.14|1418.35|1645.21|4.60|0.00|213.75|115.14|119.74|328.89|333.49|-1303.21| +2450857|25084|2450901|7966|5289|63690|491|33292|40309|924339|2895|37296|52|14|9|5|124|281|8|58.50|169.65|159.47|81.44|1275.76|468.00|1357.20|40.82|765.45|488.56|510.31|551.13|998.87|1039.69|42.31| +2450857|25084|2450876|5560|5289|63690|491|33292|40309|924339|2895|37296|34|2|6|3|7|281|72|5.16|8.25|1.73|469.44|124.56|371.52|594.00|9.96|0.00|177.84|124.56|134.52|302.40|312.36|-246.96| +2450857|25084|2450870|2264|5289|63690|491|33292|40309|924339|2895|37296|16|25|7|1|138|281|4|40.36|66.99|52.92|56.28|211.68|161.44|267.96|4.23|0.00|125.92|211.68|215.91|337.60|341.83|50.24| +2450857|25084|2450955|8827|5289|63690|491|33292|40309|924339|2895|37296|13|4|9|5|14|281|39|50.86|89.00|35.60|2082.60|1388.40|1983.54|3471.00|13.88|0.00|485.94|1388.40|1402.28|1874.34|1888.22|-595.14| +2450857|25084|2450964|17998|5289|63690|491|33292|40309|924339|2895|37296|13|28|3|1|249|281|46|51.93|62.31|52.96|430.10|2436.16|2388.78|2866.26|48.72|0.00|28.52|2436.16|2484.88|2464.68|2513.40|47.38| +2450857|25084|2450951|4166|5289|63690|491|33292|40309|924339|2895|37296|28|14|11|4|144|281|31|1.84|2.59|1.42|36.27|44.02|57.04|80.29|3.52|0.00|35.96|44.02|47.54|79.98|83.50|-13.02| +2450857|25084|2450878|2008|5289|63690|491|33292|40309|924339|2895|37296|56|28|8|4|257|281|87|16.44|36.99|22.19|1287.60|1930.53|1430.28|3218.13|19.30|0.00|578.55|1930.53|1949.83|2509.08|2528.38|500.25| +2450857|25084|2450901|15340|5289|63690|491|33292|40309|924339|2895|37296|25|22|11|5|32|281|90|59.17|110.64|14.38|8663.40|1294.20|5325.30|9957.60|90.59|0.00|995.40|1294.20|1384.79|2289.60|2380.19|-4031.10| +2450857|25084|2450864|310|5289|63690|491|33292|40309|924339|2895|37296|1|4|8|3|205|281|98|78.27|81.40|8.95|7100.10|877.10|7670.46|7977.20|23.68|87.71|717.36|789.39|813.07|1506.75|1530.43|-6881.07| +2450857|25084|2450967|15499|5289|63690|491|33292|40309|924339|2895|37296|20|2|10|1|12|281|98|7.63|10.60|4.66|582.12|456.68|747.74|1038.80|13.70|0.00|311.64|456.68|470.38|768.32|782.02|-291.06| +2450857|25084|2450921|7624|5289|63690|491|33292|40309|924339|2895|37296|1|4|15|2|99|281|39|83.70|115.50|40.42|2928.12|1576.38|3264.30|4504.50|141.87|0.00|1171.17|1576.38|1718.25|2747.55|2889.42|-1687.92| +2450857|25084|2450881|6082|5289|63690|491|33292|40309|924339|2895|37296|58|14|6|2|85|281|14|86.70|110.10|106.79|46.34|1495.06|1213.80|1541.40|74.75|0.00|647.36|1495.06|1569.81|2142.42|2217.17|281.26| +2450857|25084|2450864|193|5289|63690|491|33292|40309|924339|2895|37296|44|14|13|3|293|281|60|49.32|64.60|49.74|891.60|2984.40|2959.20|3876.00|89.53|0.00|1434.00|2984.40|3073.93|4418.40|4507.93|25.20| +2450857|25084|2450866|3382|5289|63690|491|33292|40309|924339|2895|37296|13|26|4|1|141|281|83|55.12|117.40|57.52|4970.04|4774.16|4574.96|9744.20|143.22|0.00|3897.68|4774.16|4917.38|8671.84|8815.06|199.20| +2450857|25084|2450868|5605|5289|63690|491|33292|40309|924339|2895|37296|22|13|19|4|216|281|35|17.46|51.33|18.47|1150.10|646.45|611.10|1796.55|12.92|0.00|772.45|646.45|659.37|1418.90|1431.82|35.35| +2451162|46521|2451188|5558|9162|1647072|4535|28798|98379|732454|4434|27097|50|20|6|1|116|282|82|79.25|166.42|0.00|13646.44|0.00|6498.50|13646.44|0.00|0.00|1227.54|0.00|0.00|1227.54|1227.54|-6498.50| +2451162|46521|2451226|8632|9162|1647072|4535|28798|98379|732454|4434|27097|46|20|8|5|122|282|23|61.50|172.20|27.55|3326.95|633.65|1414.50|3960.60|44.35|0.00|1267.30|633.65|678.00|1900.95|1945.30|-780.85| +2451162|46521|2451265|14617|9162|1647072|4535|28798|98379|732454|4434|27097|16|19|1|2|276|282|89|97.95|212.55|12.75|17782.20|1134.75|8717.55|18916.95|22.69|0.00|188.68|1134.75|1157.44|1323.43|1346.12|-7582.80| +2451162|46521|2451232|11746|9162|1647072|4535|28798|98379|732454|4434|27097|22|8|15|3|238|282|31|15.75|28.50|3.70|768.80|114.70|488.25|883.50|8.02|0.00|141.36|114.70|122.72|256.06|264.08|-373.55| +2451162|46521|2451224|7243|9162|1647072|4535|28798|98379|732454|4434|27097|20|20|14|4|154|282|25|19.59|55.83|46.89|223.50|1172.25|489.75|1395.75|35.16|0.00|97.50|1172.25|1207.41|1269.75|1304.91|682.50| +2451162|46521|2451168|874|9162|1647072|4535|28798|98379|732454|4434|27097|55|4|18|1|131|282|22|9.00|15.03|9.76|115.94|214.72|198.00|330.66|6.44|0.00|42.90|214.72|221.16|257.62|264.06|16.72| +2451162|46521|2451203|6214|9162|1647072|4535|28798|98379|732454|4434|27097|38|8|20|5|193|282|88|24.81|42.92|35.62|642.40|3134.56|2183.28|3776.96|62.69|0.00|1547.92|3134.56|3197.25|4682.48|4745.17|951.28| +2451162|46521|2451199|10214|9162|1647072|4535|28798|98379|732454|4434|27097|37|7|6|1|123|282|23|35.09|87.72|57.89|686.09|1331.47|807.07|2017.56|13.31|0.00|383.18|1331.47|1344.78|1714.65|1727.96|524.40| +2451162|46521|2451234|7664|9162|1647072|4535|28798|98379|732454|4434|27097|8|8|5|3|238|282|72|94.04|262.37|39.35|16057.44|2833.20|6770.88|18890.64|0.00|1161.61|7933.68|1671.59|1671.59|9605.27|9605.27|-5099.29| +2451162|46521|2451172|9961|9162|1647072|4535|28798|98379|732454|4434|27097|16|2|5|3|94|282|59|45.45|94.53|33.08|3625.55|1951.72|2681.55|5577.27|156.13|0.00|1059.64|1951.72|2107.85|3011.36|3167.49|-729.83| +2451162|46521|2451266|14731|9162|1647072|4535|28798|98379|732454|4434|27097|38|1|2|3|297|282|80|30.93|88.15|74.92|1058.40|5993.60|2474.40|7052.00|359.61|0.00|211.20|5993.60|6353.21|6204.80|6564.41|3519.20| +2452046|11122|2452113|9535|16586|1272846|4753|27413|30951|1884571|640|47300|29|15|14|2|213|283|76|46.22|99.83|8.98|6904.60|682.48|3512.72|7587.08|40.94|0.00|2124.20|682.48|723.42|2806.68|2847.62|-2830.24| +2452046|11122|2452119|2987|16586|1272846|4753|27413|30951|1884571|640|47300|53|23|7|2|117|283|74|11.26|16.21|11.02|384.06|815.48|833.24|1199.54|24.46|0.00|191.66|815.48|839.94|1007.14|1031.60|-17.76| +2452046|11122|2452047|13069|16586|1272846|4753|27413|30951|1884571|640|47300|45|23|18|2|23|283|80|54.25|97.10|0.00|7768.00|0.00|4340.00|7768.00|0.00|0.00|2252.00|0.00|0.00|2252.00|2252.00|-4340.00| +2452046|11122|2452134|883|16586|1272846|4753|27413|30951|1884571|640|47300|7|5|2|1|83|283|77|13.59|18.61|11.16|573.65|859.32|1046.43|1432.97|77.33|0.00|716.10|859.32|936.65|1575.42|1652.75|-187.11| +2452046|11122|2452060|16983|16586|1272846|4753|27413|30951|1884571|640|47300|53|9|3|1|112|283|83|41.53|124.17|73.26|4225.53|6080.58|3446.99|10306.11|425.64|0.00|1030.03|6080.58|6506.22|7110.61|7536.25|2633.59| +2452046|11122|2452135|3209|16586|1272846|4753|27413|30951|1884571|640|47300|7|9|19|5|294|283|45|59.47|156.40|125.12|1407.60|5630.40|2676.15|7038.00|337.82|0.00|2251.80|5630.40|5968.22|7882.20|8220.02|2954.25| +2452046|11122|2452060|14835|16586|1272846|4753|27413|30951|1884571|640|47300|39|19|20|5|250|283|36|18.47|36.38|25.46|393.12|916.56|664.92|1309.68|73.32|0.00|392.76|916.56|989.88|1309.32|1382.64|251.64| +2452046|11122|2452097|3969|16586|1272846|4753|27413|30951|1884571|640|47300|39|17|15|1|110|283|55|12.07|35.36|24.04|622.60|1322.20|663.85|1944.80|13.22|0.00|466.40|1322.20|1335.42|1788.60|1801.82|658.35| +2452046|11122|2452117|10519|16586|1272846|4753|27413|30951|1884571|640|47300|27|29|16|2|48|283|49|49.41|112.65|18.02|4636.87|882.98|2421.09|5519.85|8.82|0.00|386.12|882.98|891.80|1269.10|1277.92|-1538.11| +2452046|11122|2452073|8189|16586|1272846|4753|27413|30951|1884571|640|47300|33|25|19|5|121|283|46|69.81|163.35|156.81|300.84|7213.26|3211.26|7514.10|144.26|0.00|2855.22|7213.26|7357.52|10068.48|10212.74|4002.00| +2452046|11122|2452136|3597|16586|1272846|4753|27413|30951|1884571|640|47300|51|23|8|4|120|283|65|47.88|111.56|3.34|7034.30|217.10|3112.20|7251.40|15.19|0.00|3045.25|217.10|232.29|3262.35|3277.54|-2895.10| +2452046|11122|2452054|5773|16586|1272846|4753|27413|30951|1884571|640|47300|15|15|8|3|248|283|88|29.87|59.74|37.63|1945.68|3311.44|2628.56|5257.12|0.00|0.00|1997.60|3311.44|3311.44|5309.04|5309.04|682.88| +2452046|11122|2452125|8049|16586|1272846|4753|27413|30951|1884571|640|47300|41|17|7|3|236|283|82|10.28|16.65|2.33|1174.24|191.06|842.96|1365.30|0.00|191.06|259.12|0.00|0.00|259.12|259.12|-842.96| +2452046|11122|2452104|14611|16586|1272846|4753|27413|30951|1884571|640|47300|21|7|7|5|43|283|83|38.53|55.09|11.56|3612.99|959.48|3197.99|4572.47|86.35|0.00|1691.54|959.48|1045.83|2651.02|2737.37|-2238.51| +2452046|11122|2452117|12363|16586|1272846|4753|27413|30951|1884571|640|47300|31|15|13|5|247|283|68|6.41|16.92|16.92|0.00|1150.56|435.88|1150.56|34.51|0.00|0.00|1150.56|1185.07|1150.56|1185.07|714.68| +2452046|11122|2452093|14859|16586|1272846|4753|27413|30951|1884571|640|47300|13|5|12|3|284|283|35|80.22|176.48|82.94|3273.90|2902.90|2807.70|6176.80|49.93|406.40|1852.90|2496.50|2546.43|4349.40|4399.33|-311.20| +2450940|30930|2451000|3694|24729|58355|3534|19960|5095|1610053|1944|7523|46|2|13|4|35|284|100|70.36|208.26|162.44|4582.00|16244.00|7036.00|20826.00|974.64|0.00|9579.00|16244.00|17218.64|25823.00|26797.64|9208.00| +2450940|30930|2451040|11036|24729|58355|3534|19960|5095|1610053|1944|7523|37|7|12|2|192|284|35|38.09|68.94|45.50|820.40|1592.50|1333.15|2412.90|47.77|0.00|820.05|1592.50|1640.27|2412.55|2460.32|259.35| +2450940|30930|2451042|17036|24729|58355|3534|19960|5095|1610053|1944|7523|28|10|4|2|60|284|81|15.76|23.00|11.04|968.76|894.24|1276.56|1863.00|74.04|71.53|55.89|822.71|896.75|878.60|952.64|-453.85| +2450940|30930|2451054|2474|24729|58355|3534|19960|5095|1610053|1944|7523|26|7|14|4|274|284|4|69.55|134.92|116.03|75.56|464.12|278.20|539.68|23.20|0.00|97.12|464.12|487.32|561.24|584.44|185.92| +2450940|30930|2451017|8636|24729|58355|3534|19960|5095|1610053|1944|7523|31|4|13|5|107|284|60|3.65|8.03|7.78|15.00|466.80|219.00|481.80|32.67|0.00|148.80|466.80|499.47|615.60|648.27|247.80| +2450940|30930|2451020|9314|24729|58355|3534|19960|5095|1610053|1944|7523|37|14|4|5|79|284|61|43.59|85.43|82.01|208.62|5002.61|2658.99|5211.23|250.13|0.00|1823.90|5002.61|5252.74|6826.51|7076.64|2343.62| +2450940|30930|2450982|10870|24729|58355|3534|19960|5095|1610053|1944|7523|40|28|9|3|140|284|97|38.44|49.20|11.31|3675.33|1097.07|3728.68|4772.40|54.85|0.00|810.92|1097.07|1151.92|1907.99|1962.84|-2631.61| +2450940|30930|2450952|16904|24729|58355|3534|19960|5095|1610053|1944|7523|20|1|11|4|294|284|5|25.23|37.59|9.02|142.85|45.10|126.15|187.95|0.90|0.00|93.95|45.10|46.00|139.05|139.95|-81.05| +2450940|30930|2450983|8731|24729|58355|3534|19960|5095|1610053|1944|7523|2|22|15|5|275|284|20|62.06|109.22|17.47|1835.00|349.40|1241.20|2184.40|10.48|0.00|961.00|349.40|359.88|1310.40|1320.88|-891.80| +2450940|30930|2451029|12122|24729|58355|3534|19960|5095|1610053|1944|7523|55|4|8|1|71|284|64|86.99|236.61|9.46|14537.60|605.44|5567.36|15143.04|48.43|0.00|453.76|605.44|653.87|1059.20|1107.63|-4961.92| +2450940|30930|2450958|1273|24729|58355|3534|19960|5095|1610053|1944|7523|55|14|18|3|75|284|16|81.02|89.12|28.51|969.76|456.16|1296.32|1425.92|41.05|0.00|413.44|456.16|497.21|869.60|910.65|-840.16| +2450940|30930|2450945|12103|24729|58355|3534|19960|5095|1610053|1944|7523|20|7|20|3|246|284|98|61.49|120.52|39.77|7913.50|3897.46|6026.02|11810.96|272.82|0.00|5314.54|3897.46|4170.28|9212.00|9484.82|-2128.56| +2450940|30930|2450954|12832|24729|58355|3534|19960|5095|1610053|1944|7523|28|25|18|1|224|284|83|93.08|139.62|22.33|9735.07|1853.39|7725.64|11588.46|74.13|0.00|3707.61|1853.39|1927.52|5561.00|5635.13|-5872.25| +2450976|20787|2451008|9397|39051|1510151|2051|49551|50479|1094863|3251|13551|34|2|3|4|35|285|60|60.88|167.42|142.30|1507.20|8538.00|3652.80|10045.20|597.66|0.00|301.20|8538.00|9135.66|8839.20|9436.86|4885.20| +2450976|20787|2451007|13789|39051|1510151|2051|49551|50479|1094863|3251|13551|43|26|19|4|66|285|98|91.09|173.07|131.53|4070.92|12889.94|8926.82|16960.86|257.79|0.00|4409.02|12889.94|13147.73|17298.96|17556.75|3963.12| +2450976|20787|2451012|82|39051|1510151|2051|49551|50479|1094863|3251|13551|32|1|12|1|290|285|52|57.59|63.92|28.12|1861.60|1462.24|2994.68|3323.84|131.60|0.00|232.44|1462.24|1593.84|1694.68|1826.28|-1532.44| +2450976|20787|2451052|17821|39051|1510151|2051|49551|50479|1094863|3251|13551|8|14|8|3|121|285|61|32.49|84.79|17.80|4086.39|1085.80|1981.89|5172.19|21.71|0.00|309.88|1085.80|1107.51|1395.68|1417.39|-896.09| +2450976|20787|2451088|9500|39051|1510151|2051|49551|50479|1094863|3251|13551|50|1|6|3|113|285|57|15.45|17.45|13.43|229.14|765.51|880.65|994.65|68.89|0.00|79.23|765.51|834.40|844.74|913.63|-115.14| +2450976|20787|2451009|11353|39051|1510151|2051|49551|50479|1094863|3251|13551|55|13|6|3|27|285|17|7.13|12.83|9.87|50.32|167.79|121.21|218.11|8.38|0.00|102.51|167.79|176.17|270.30|278.68|46.58| +2450976|20787|2450999|12118|39051|1510151|2051|49551|50479|1094863|3251|13551|10|26|5|3|121|285|15|44.43|109.74|24.14|1284.00|362.10|666.45|1646.10|23.46|68.79|592.50|293.31|316.77|885.81|909.27|-373.14| +2450976|20787|2451010|14774|39051|1510151|2051|49551|50479|1094863|3251|13551|58|25|16|4|165|285|57|51.60|153.76|26.13|7274.91|1489.41|2941.20|8764.32|14.89|0.00|2541.63|1489.41|1504.30|4031.04|4045.93|-1451.79| +2450976|20787|2451094|15013|39051|1510151|2051|49551|50479|1094863|3251|13551|37|2|18|4|117|285|59|42.30|70.64|19.07|3042.63|1125.13|2495.70|4167.76|62.22|236.27|41.30|888.86|951.08|930.16|992.38|-1606.84| +2450976|20787|2450983|14840|39051|1510151|2051|49551|50479|1094863|3251|13551|16|14|4|5|259|285|77|83.42|153.49|82.88|5436.97|6381.76|6423.34|11818.73|406.51|574.35|4017.86|5807.41|6213.92|9825.27|10231.78|-615.93| +2450976|20787|2451040|3271|39051|1510151|2051|49551|50479|1094863|3251|13551|10|26|9|4|232|285|6|34.31|59.35|11.27|288.48|67.62|205.86|356.10|6.08|0.00|92.58|67.62|73.70|160.20|166.28|-138.24| +2450976|20787|2451014|7864|39051|1510151|2051|49551|50479|1094863|3251|13551|22|16|19|4|69|285|100|85.90|257.70|255.12|258.00|25512.00|8590.00|25770.00|362.27|7398.48|4380.00|18113.52|18475.79|22493.52|22855.79|9523.52| +2451083|20701|2451122|14209|38883|1198834|3138|27489|10990|208224|2495|6846|20|2|16|2|291|286|59|35.55|104.51|87.78|987.07|5179.02|2097.45|6166.09|103.58|0.00|2280.94|5179.02|5282.60|7459.96|7563.54|3081.57| +2451083|20701|2451139|14902|38883|1198834|3138|27489|10990|208224|2495|6846|28|7|3|5|24|286|49|47.07|72.95|8.02|3181.57|392.98|2306.43|3574.55|11.78|0.00|1036.35|392.98|404.76|1429.33|1441.11|-1913.45| +2451083|20701|2451129|15805|38883|1198834|3138|27489|10990|208224|2495|6846|2|26|16|3|80|286|89|37.09|55.63|27.25|2525.82|2425.25|3301.01|4951.07|1.94|2231.23|593.63|194.02|195.96|787.65|789.59|-3106.99| +2451083|20701|2451141|12424|38883|1198834|3138|27489|10990|208224|2495|6846|38|25|6|2|135|286|90|15.32|21.29|17.03|383.40|1532.70|1378.80|1916.10|122.61|0.00|938.70|1532.70|1655.31|2471.40|2594.01|153.90| +2451083|20701|2451182|6850|38883|1198834|3138|27489|10990|208224|2495|6846|13|13|14|4|194|286|58|94.80|178.22|5.34|10027.04|309.72|5498.40|10336.76|21.68|0.00|3720.70|309.72|331.40|4030.42|4052.10|-5188.68| +2451083|20701|2451193|6092|38883|1198834|3138|27489|10990|208224|2495|6846|10|10|16|1|186|286|46|85.34|226.15|72.36|7074.34|3328.56|3925.64|10402.90|99.85|0.00|2496.42|3328.56|3428.41|5824.98|5924.83|-597.08| +2451083|20701|2451172|4315|38883|1198834|3138|27489|10990|208224|2495|6846|28|22|19|1|63|286|6|16.88|29.37|8.51|125.16|51.06|101.28|176.22|4.08|0.00|28.14|51.06|55.14|79.20|83.28|-50.22| +2451083|20701|2451158|5800|38883|1198834|3138|27489|10990|208224|2495|6846|37|4|17|1|135|286|7|12.73|32.33|7.43|174.30|52.01|89.11|226.31|2.62|22.88|61.04|29.13|31.75|90.17|92.79|-59.98| +2451083|20701|2451118|9583|38883|1198834|3138|27489|10990|208224|2495|6846|13|1|8|2|239|286|61|18.18|28.36|10.49|1090.07|639.89|1108.98|1729.96|31.99|0.00|86.01|639.89|671.88|725.90|757.89|-469.09| +2451083|20701|2451168|11774|38883|1198834|3138|27489|10990|208224|2495|6846|38|13|11|3|192|286|30|70.45|73.26|21.24|1560.60|637.20|2113.50|2197.80|0.00|0.00|966.90|637.20|637.20|1604.10|1604.10|-1476.30| +2451083|20701|2451086|17749|38883|1198834|3138|27489|10990|208224|2495|6846|19|25|6|4|172|286|22|77.78|138.44|120.44|396.00|2649.68|1711.16|3045.68|158.98|0.00|1522.84|2649.68|2808.66|4172.52|4331.50|938.52| +2451083|20701|2451159|1310|38883|1198834|3138|27489|10990|208224|2495|6846|31|19|17|1|105|286|12|75.02|127.53|7.65|1438.56|91.80|900.24|1530.36|7.34|0.00|153.00|91.80|99.14|244.80|252.14|-808.44| +2451083|20701|2451193|10372|38883|1198834|3138|27489|10990|208224|2495|6846|2|10|10|2|281|286|30|42.71|75.16|74.40|22.80|2232.00|1281.30|2254.80|89.28|0.00|992.10|2232.00|2321.28|3224.10|3313.38|950.70| +2451282|37589|2451339|15595|39969|1871097|3825|34553|4413|1337297|5584|44312|40|7|14|2|260|287|30|49.34|83.87|63.74|603.90|1912.20|1480.20|2516.10|74.57|420.68|1157.40|1491.52|1566.09|2648.92|2723.49|11.32| +2451282|37589|2451285|5593|39969|1871097|3825|34553|4413|1337297|5584|44312|55|25|1|1|134|287|60|33.36|88.40|71.60|1008.00|4296.00|2001.60|5304.00|214.80|0.00|1962.00|4296.00|4510.80|6258.00|6472.80|2294.40| +2451282|37589|2451373|1189|39969|1871097|3825|34553|4413|1337297|5584|44312|4|4|3|1|296|287|15|83.32|86.65|31.19|831.90|467.85|1249.80|1299.75|23.39|0.00|506.85|467.85|491.24|974.70|998.09|-781.95| +2451282|37589|2451297|11764|39969|1871097|3825|34553|4413|1337297|5584|44312|38|14|13|1|256|287|67|75.81|150.10|58.53|6135.19|3921.51|5079.27|10056.70|352.93|0.00|4123.18|3921.51|4274.44|8044.69|8397.62|-1157.76| +2451282|37589|2451401|6218|39969|1871097|3825|34553|4413|1337297|5584|44312|7|19|13|4|72|287|95|50.62|123.51|122.27|117.80|11615.65|4808.90|11733.45|696.93|0.00|2111.85|11615.65|12312.58|13727.50|14424.43|6806.75| +2451282|37589|2451377|7372|39969|1871097|3825|34553|4413|1337297|5584|44312|20|25|5|5|42|287|58|17.70|52.74|13.18|2294.48|764.44|1026.60|3058.92|7.64|0.00|30.16|764.44|772.08|794.60|802.24|-262.16| +2451282|37589|2451378|5425|39969|1871097|3825|34553|4413|1337297|5584|44312|56|2|5|3|11|287|17|35.38|70.05|7.70|1059.95|130.90|601.46|1190.85|5.23|0.00|47.60|130.90|136.13|178.50|183.73|-470.56| +2451282|37589|2451394|13984|39969|1871097|3825|34553|4413|1337297|5584|44312|13|10|18|2|49|287|23|77.70|108.00|25.92|1887.84|596.16|1787.10|2484.00|29.80|0.00|223.56|596.16|625.96|819.72|849.52|-1190.94| +2451282|37589|2451379|6367|39969|1871097|3825|34553|4413|1337297|5584|44312|10|25|17|1|295|287|3|77.33|193.32|173.98|58.02|521.94|231.99|579.96|0.00|0.00|156.57|521.94|521.94|678.51|678.51|289.95| +2451282|37589|2451314|15512|39969|1871097|3825|34553|4413|1337297|5584|44312|26|10|8|3|275|287|86|60.28|74.14|63.01|957.18|5418.86|5184.08|6376.04|44.43|975.39|381.84|4443.47|4487.90|4825.31|4869.74|-740.61| +2451282|37589|2451398|3328|39969|1871097|3825|34553|4413|1337297|5584|44312|52|7|8|5|32|287|70|54.16|70.40|9.15|4287.50|640.50|3791.20|4928.00|57.64|0.00|1281.00|640.50|698.14|1921.50|1979.14|-3150.70| +2451282|37589|2451327|10570|39969|1871097|3825|34553|4413|1337297|5584|44312|14|8|2|4|296|287|78|96.96|244.33|163.70|6289.14|12768.60|7562.88|19057.74|0.00|0.00|1905.54|12768.60|12768.60|14674.14|14674.14|5205.72| +2451282|37589|2451325|13591|39969|1871097|3825|34553|4413|1337297|5584|44312|56|26|9|4|292|287|56|14.78|37.39|13.46|1340.08|753.76|827.68|2093.84|37.68|0.00|857.92|753.76|791.44|1611.68|1649.36|-73.92| +2451282|37589|2451353|5924|39969|1871097|3825|34553|4413|1337297|5584|44312|58|1|2|1|117|287|47|12.77|15.57|12.76|132.07|599.72|600.19|731.79|0.00|0.00|36.19|599.72|599.72|635.91|635.91|-0.47| +2451282|37589|2451309|11071|39969|1871097|3825|34553|4413|1337297|5584|44312|38|26|8|2|296|287|80|27.12|75.12|51.08|1923.20|4086.40|2169.60|6009.60|245.18|0.00|3004.80|4086.40|4331.58|7091.20|7336.38|1916.80| +2451874|66230|2451973|9865|53099|1833197|6448|7346|85301|689032|793|7244|29|14|1|4|10|288|67|65.24|173.53|38.17|9069.12|2557.39|4371.08|11626.51|204.59|0.00|0.00|2557.39|2761.98|2557.39|2761.98|-1813.69| +2451874|66230|2451931|524|53099|1833197|6448|7346|85301|689032|793|7244|59|7|18|2|16|288|70|11.88|16.86|8.26|602.00|578.20|831.60|1180.20|28.91|0.00|566.30|578.20|607.11|1144.50|1173.41|-253.40| +2451874|66230|2451921|11246|53099|1833197|6448|7346|85301|689032|793|7244|8|25|18|1|81|288|81|10.59|24.56|0.98|1909.98|79.38|857.79|1989.36|0.79|0.00|417.15|79.38|80.17|496.53|497.32|-778.41| +2451874|66230|2451902|13405|53099|1833197|6448|7346|85301|689032|793|7244|43|14|14|5|56|288|86|60.42|148.02|22.20|10820.52|1909.20|5196.12|12729.72|95.46|0.00|4072.96|1909.20|2004.66|5982.16|6077.62|-3286.92| +2451874|66230|2451975|17929|53099|1833197|6448|7346|85301|689032|793|7244|8|23|19|1|243|288|56|68.66|70.71|19.09|2890.72|1069.04|3844.96|3959.76|21.38|0.00|1623.44|1069.04|1090.42|2692.48|2713.86|-2775.92| +2451874|66230|2451988|1799|53099|1833197|6448|7346|85301|689032|793|7244|47|23|10|3|246|288|75|2.80|8.20|2.21|449.25|165.75|210.00|615.00|13.26|0.00|110.25|165.75|179.01|276.00|289.26|-44.25| +2451874|66230|2451914|14930|53099|1833197|6448|7346|85301|689032|793|7244|31|29|7|1|65|288|56|30.96|75.23|29.33|2570.40|1642.48|1733.76|4212.88|65.69|0.00|758.24|1642.48|1708.17|2400.72|2466.41|-91.28| +2451874|66230|2451915|9872|53099|1833197|6448|7346|85301|689032|793|7244|55|14|8|3|8|288|95|9.55|27.98|3.35|2339.85|318.25|907.25|2658.10|9.54|0.00|584.25|318.25|327.79|902.50|912.04|-589.00| +2451874|66230|2451906|5803|53099|1833197|6448|7346|85301|689032|793|7244|2|20|20|4|248|288|92|21.45|48.69|5.84|3942.20|537.28|1973.40|4479.48|37.60|0.00|402.96|537.28|574.88|940.24|977.84|-1436.12| +2451874|66230|2451927|13487|53099|1833197|6448|7346|85301|689032|793|7244|59|13|7|2|277|288|43|31.75|45.72|32.46|570.18|1395.78|1365.25|1965.96|41.87|0.00|844.95|1395.78|1437.65|2240.73|2282.60|30.53| +2451874|66230|2451943|16505|53099|1833197|6448|7346|85301|689032|793|7244|32|29|13|3|196|288|70|60.49|159.69|7.98|10619.70|558.60|4234.30|11178.30|16.75|0.00|3465.00|558.60|575.35|4023.60|4040.35|-3675.70| +2451874|66230|2451968|3095|53099|1833197|6448|7346|85301|689032|793|7244|35|5|8|3|138|288|18|58.94|159.13|31.82|2291.58|572.76|1060.92|2864.34|0.00|366.56|1346.22|206.20|206.20|1552.42|1552.42|-854.72| +2451874|66230|2451986|8150|53099|1833197|6448|7346|85301|689032|793|7244|49|20|17|3|239|288|51|20.15|46.34|44.94|71.40|2291.94|1027.65|2363.34|68.75|0.00|945.03|2291.94|2360.69|3236.97|3305.72|1264.29| +2451874|66230|2451896|8393|53099|1833197|6448|7346|85301|689032|793|7244|20|17|10|3|86|288|41|34.31|80.28|27.29|2172.59|1118.89|1406.71|3291.48|33.56|0.00|592.45|1118.89|1152.45|1711.34|1744.90|-287.82| +2451874|66230|2451922|13088|53099|1833197|6448|7346|85301|689032|793|7244|47|20|18|3|115|288|14|37.70|81.05|26.74|760.34|374.36|527.80|1134.70|0.00|0.00|158.76|374.36|374.36|533.12|533.12|-153.44| +2452008|41962|2452093|17255|44273|1643397|1368|18539|77695|402157|805|28776|25|17|12|3|251|289|25|37.33|76.52|37.49|975.75|937.25|933.25|1913.00|36.55|206.19|669.50|731.06|767.61|1400.56|1437.11|-202.19| +2452008|41962|2452029|15943|44273|1643397|1368|18539|77695|402157|805|28776|47|11|15|4|243|289|58|23.22|56.88|31.85|1451.74|1847.30|1346.76|3299.04|7.38|1754.93|791.70|92.37|99.75|884.07|891.45|-1254.39| +2452008|41962|2452064|2689|44273|1643397|1368|18539|77695|402157|805|28776|29|7|16|5|144|289|81|17.20|23.04|12.21|877.23|989.01|1393.20|1866.24|8.30|850.54|746.01|138.47|146.77|884.48|892.78|-1254.73| +2452008|41962|2452054|6105|44273|1643397|1368|18539|77695|402157|805|28776|51|7|5|1|7|289|57|88.54|91.19|57.44|1923.75|3274.08|5046.78|5197.83|98.22|0.00|1870.74|3274.08|3372.30|5144.82|5243.04|-1772.70| +2452008|41962|2452076|13609|44273|1643397|1368|18539|77695|402157|805|28776|3|25|17|1|145|289|71|66.29|124.62|18.69|7521.03|1326.99|4706.59|8848.02|13.26|0.00|1150.20|1326.99|1340.25|2477.19|2490.45|-3379.60| +2452008|41962|2452125|2969|44273|1643397|1368|18539|77695|402157|805|28776|3|21|7|3|82|289|100|79.03|110.64|64.17|4647.00|6417.00|7903.00|11064.00|449.19|0.00|4093.00|6417.00|6866.19|10510.00|10959.19|-1486.00| +2452008|41962|2452085|16683|44273|1643397|1368|18539|77695|402157|805|28776|15|27|10|3|159|289|99|50.80|87.88|36.90|5047.02|3653.10|5029.20|8700.12|87.67|2191.86|3044.25|1461.24|1548.91|4505.49|4593.16|-3567.96| +2452008|41962|2452022|11159|44273|1643397|1368|18539|77695|402157|805|28776|47|13|10|3|39|289|39|77.17|229.19|87.09|5541.90|3396.51|3009.63|8938.41|101.89|0.00|3932.76|3396.51|3498.40|7329.27|7431.16|386.88| +2452008|41962|2452126|179|44273|1643397|1368|18539|77695|402157|805|28776|11|9|17|1|228|289|100|79.64|95.56|62.11|3345.00|6211.00|7964.00|9556.00|248.44|0.00|3917.00|6211.00|6459.44|10128.00|10376.44|-1753.00| +2452008|41962|2452125|17093|44273|1643397|1368|18539|77695|402157|805|28776|35|11|1|4|157|289|93|40.68|118.78|10.69|10052.37|994.17|3783.24|11046.54|9.94|0.00|2871.84|994.17|1004.11|3866.01|3875.95|-2789.07| +2452008|41962|2452052|9269|44273|1643397|1368|18539|77695|402157|805|28776|45|27|15|4|100|289|18|51.79|93.73|70.29|421.92|1265.22|932.22|1687.14|63.26|0.00|320.40|1265.22|1328.48|1585.62|1648.88|333.00| +2451953|48423|2451965|7131|13419|1716317|6415|18513|96654|820930|5207|8952|33|3|14|4|121|290|80|20.63|38.78|23.65|1210.40|1892.00|1650.40|3102.40|56.76|0.00|1426.40|1892.00|1948.76|3318.40|3375.16|241.60| +2451953|48423|2451966|8937|13419|1716317|6415|18513|96654|820930|5207|8952|59|9|3|1|299|290|73|51.48|109.13|51.29|4222.32|3744.17|3758.04|7966.49|224.65|0.00|2469.59|3744.17|3968.82|6213.76|6438.41|-13.87| +2451953|48423|2452049|4015|13419|1716317|6415|18513|96654|820930|5207|8952|53|29|20|3|16|290|92|19.92|57.76|23.10|3188.72|2125.20|1832.64|5313.92|0.00|0.00|1434.28|2125.20|2125.20|3559.48|3559.48|292.56| +2451953|48423|2452038|5861|13419|1716317|6415|18513|96654|820930|5207|8952|1|11|11|1|284|290|47|99.67|220.27|15.41|9628.42|724.27|4684.49|10352.69|7.24|0.00|3830.03|724.27|731.51|4554.30|4561.54|-3960.22| +2451953|48423|2451983|1885|13419|1716317|6415|18513|96654|820930|5207|8952|41|5|15|4|166|290|96|43.50|88.74|85.19|340.80|8178.24|4176.00|8519.04|327.12|0.00|3066.24|8178.24|8505.36|11244.48|11571.60|4002.24| +2451953|48423|2451991|7929|13419|1716317|6415|18513|96654|820930|5207|8952|45|23|8|2|233|290|17|81.80|99.79|56.88|729.47|966.96|1390.60|1696.43|19.33|0.00|729.30|966.96|986.29|1696.26|1715.59|-423.64| +2451953|48423|2452065|15343|13419|1716317|6415|18513|96654|820930|5207|8952|23|21|20|4|38|290|93|61.34|176.65|176.65|0.00|16428.45|5704.62|16428.45|492.85|0.00|2299.89|16428.45|16921.30|18728.34|19221.19|10723.83| +2451953|48423|2452049|4121|13419|1716317|6415|18513|96654|820930|5207|8952|9|17|20|1|185|290|49|5.10|8.16|2.20|292.04|107.80|249.90|399.84|7.54|0.00|167.58|107.80|115.34|275.38|282.92|-142.10| +2451953|48423|2451961|3971|13419|1716317|6415|18513|96654|820930|5207|8952|13|15|16|4|295|290|99|3.43|9.80|5.09|466.29|503.91|339.57|970.20|35.27|0.00|475.20|503.91|539.18|979.11|1014.38|164.34| +2451953|48423|2451959|8993|13419|1716317|6415|18513|96654|820930|5207|8952|21|23|16|2|121|290|34|98.50|270.87|48.75|7552.08|1657.50|3349.00|9209.58|149.17|0.00|183.94|1657.50|1806.67|1841.44|1990.61|-1691.50| +2451953|48423|2451990|17045|13419|1716317|6415|18513|96654|820930|5207|8952|19|7|15|1|97|290|48|84.88|252.09|10.08|11616.48|483.84|4074.24|12100.32|43.54|0.00|2419.68|483.84|527.38|2903.52|2947.06|-3590.40| +2451953|48423|2452071|5697|13419|1716317|6415|18513|96654|820930|5207|8952|9|15|1|4|167|290|12|26.07|48.49|8.24|483.00|98.88|312.84|581.88|8.89|0.00|290.88|98.88|107.77|389.76|398.65|-213.96| +2451953|48423|2452020|10655|13419|1716317|6415|18513|96654|820930|5207|8952|37|25|4|5|249|290|41|70.97|160.39|36.88|5063.91|1512.08|2909.77|6575.99|26.31|196.57|2038.52|1315.51|1341.82|3354.03|3380.34|-1594.26| +2451953|48423|2451998|1881|13419|1716317|6415|18513|96654|820930|5207|8952|33|27|14|1|291|290|65|1.18|2.87|1.86|65.65|120.90|76.70|186.55|2.41|0.00|83.85|120.90|123.31|204.75|207.16|44.20| +2451953|48423|2451986|2373|13419|1716317|6415|18513|96654|820930|5207|8952|49|23|2|4|206|290|41|44.05|77.52|2.32|3083.20|95.12|1806.05|3178.32|0.05|92.26|1271.00|2.86|2.91|1273.86|1273.91|-1803.19| +2452642|13862|2452674|9573|38818|77937|1503|20622|9133|95287|5626|18792|19|27|15|4|3|291|32|68.01|170.70|6.82|5244.16|218.24|2176.32|5462.40|10.91|0.00|1638.72|218.24|229.15|1856.96|1867.87|-1958.08| +2452642|13862|2452740|5205|38818|77937|1503|20622|9133|95287|5626|18792|54|6|6|4|72|291|77|84.10|84.94|31.42|4121.04|2419.34|6475.70|6540.38|162.57|96.77|2550.24|2322.57|2485.14|4872.81|5035.38|-4153.13| +2452642|13862|2452671|11311|38818|77937|1503|20622|9133|95287|5626|18792|1|25|5|2|12|291|68|10.47|24.18|2.65|1464.04|180.20|711.96|1644.24|14.41|0.00|443.36|180.20|194.61|623.56|637.97|-531.76| +2452642|13862|2452685|15960|38818|77937|1503|20622|9133|95287|5626|18792|57|25|14|5|129|291|72|68.48|97.92|68.54|2115.36|4934.88|4930.56|7050.24|444.13|0.00|2326.32|4934.88|5379.01|7261.20|7705.33|4.32| +2452642|13862|2452708|12261|38818|77937|1503|20622|9133|95287|5626|18792|7|25|9|3|54|291|18|37.17|75.45|57.34|325.98|1032.12|669.06|1358.10|61.92|0.00|570.24|1032.12|1094.04|1602.36|1664.28|363.06| +2452642|13862|2452753|15735|38818|77937|1503|20622|9133|95287|5626|18792|45|15|16|1|106|291|49|1.69|3.61|0.64|145.53|31.36|82.81|176.89|0.03|30.41|31.36|0.95|0.98|32.31|32.34|-81.86| +2452642|13862|2452725|9769|38818|77937|1503|20622|9133|95287|5626|18792|7|30|3|4|85|291|71|46.88|67.97|25.14|3040.93|1784.94|3328.48|4825.87|89.24|0.00|1109.73|1784.94|1874.18|2894.67|2983.91|-1543.54| +2452642|13862|2452706|4092|38818|77937|1503|20622|9133|95287|5626|18792|42|19|19|5|194|291|12|10.34|16.85|4.54|147.72|54.48|124.08|202.20|2.72|0.00|90.96|54.48|57.20|145.44|148.16|-69.60| +2452642|13862|2452683|17317|38818|77937|1503|20622|9133|95287|5626|18792|27|24|10|3|216|291|53|4.91|6.77|3.24|187.09|171.72|260.23|358.81|0.00|0.00|14.31|171.72|171.72|186.03|186.03|-88.51| +2452642|13862|2452696|11334|38818|77937|1503|20622|9133|95287|5626|18792|39|27|5|1|167|291|99|56.01|61.05|15.87|4472.82|1571.13|5544.99|6043.95|125.69|0.00|1329.57|1571.13|1696.82|2900.70|3026.39|-3973.86| +2452642|13862|2452727|3174|38818|77937|1503|20622|9133|95287|5626|18792|13|19|10|3|3|291|14|7.66|22.29|0.00|312.06|0.00|107.24|312.06|0.00|0.00|121.66|0.00|0.00|121.66|121.66|-107.24| +2451512|44940|2451578|11470|64371|1460038|3352|2266|73935|160424|2561|30675|14|20|14|5|103|292|68|64.00|100.48|15.07|5807.88|1024.76|4352.00|6832.64|40.99|0.00|1502.80|1024.76|1065.75|2527.56|2568.55|-3327.24| +2451512|44940|2451561|11281|64371|1460038|3352|2266|73935|160424|2561|30675|49|10|7|1|32|292|31|98.74|131.32|73.53|1791.49|2279.43|3060.94|4070.92|113.97|0.00|81.22|2279.43|2393.40|2360.65|2474.62|-781.51| +2451512|44940|2451593|10984|64371|1460038|3352|2266|73935|160424|2561|30675|52|26|11|3|5|292|63|3.25|3.54|2.90|40.32|182.70|204.75|223.02|1.82|0.00|8.82|182.70|184.52|191.52|193.34|-22.05| +2451512|44940|2451631|17416|64371|1460038|3352|2266|73935|160424|2561|30675|8|19|18|3|108|292|77|87.26|125.65|84.18|3193.19|6481.86|6719.02|9675.05|169.17|4602.12|4063.29|1879.74|2048.91|5943.03|6112.20|-4839.28| +2451512|44940|2451537|9445|64371|1460038|3352|2266|73935|160424|2561|30675|1|4|15|2|69|292|48|77.70|214.45|66.47|7103.04|3190.56|3729.60|10293.60|31.90|2552.44|3602.40|638.12|670.02|4240.52|4272.42|-3091.48| +2451512|44940|2451518|7879|64371|1460038|3352|2266|73935|160424|2561|30675|25|2|4|1|158|292|6|59.95|116.90|5.84|666.36|35.04|359.70|701.40|1.75|0.00|168.30|35.04|36.79|203.34|205.09|-324.66| +2451512|44940|2451623|2572|64371|1460038|3352|2266|73935|160424|2561|30675|20|25|15|2|139|292|26|87.03|147.08|83.83|1644.50|2179.58|2262.78|3824.08|43.59|0.00|1644.24|2179.58|2223.17|3823.82|3867.41|-83.20| +2451512|44940|2451526|368|64371|1460038|3352|2266|73935|160424|2561|30675|14|19|15|5|265|292|2|56.85|162.02|11.34|301.36|22.68|113.70|324.04|2.04|0.00|0.00|22.68|24.72|22.68|24.72|-91.02| +2451512|44940|2451595|566|64371|1460038|3352|2266|73935|160424|2561|30675|26|14|19|3|94|292|71|37.74|40.38|31.09|659.59|2207.39|2679.54|2866.98|132.44|0.00|1318.47|2207.39|2339.83|3525.86|3658.30|-472.15| +2451494|77622|2451608|5060|82370|1196688|606|12477|48729|763093|631|14149|37|1|16|3|53|293|22|61.66|161.54|108.23|1172.82|2381.06|1356.52|3553.88|192.86|238.10|213.18|2142.96|2335.82|2356.14|2549.00|786.44| +2451494|77622|2451520|15398|82370|1196688|606|12477|48729|763093|631|14149|20|10|8|3|5|293|7|58.52|81.92|50.79|217.91|355.53|409.64|573.44|0.00|0.00|183.47|355.53|355.53|539.00|539.00|-54.11| +2451494|77622|2451606|13826|82370|1196688|606|12477|48729|763093|631|14149|34|25|18|3|88|293|21|85.33|122.87|43.00|1677.27|903.00|1791.93|2580.27|81.27|0.00|1161.09|903.00|984.27|2064.09|2145.36|-888.93| +2451494|77622|2451599|13792|82370|1196688|606|12477|48729|763093|631|14149|1|1|11|1|54|293|27|13.72|18.24|14.40|103.68|388.80|370.44|492.48|15.55|0.00|187.11|388.80|404.35|575.91|591.46|18.36| +2451494|77622|2451509|8254|82370|1196688|606|12477|48729|763093|631|14149|20|4|5|5|212|293|60|66.75|99.45|97.46|119.40|5847.60|4005.00|5967.00|313.43|1929.70|357.60|3917.90|4231.33|4275.50|4588.93|-87.10| +2451494|77622|2451542|5221|82370|1196688|606|12477|48729|763093|631|14149|1|16|17|5|266|293|85|7.26|17.71|7.61|858.50|646.85|617.10|1505.35|51.74|0.00|29.75|646.85|698.59|676.60|728.34|29.75| +2451494|77622|2451525|7748|82370|1196688|606|12477|48729|763093|631|14149|40|8|19|3|40|293|79|77.65|133.55|38.72|7491.57|3058.88|6134.35|10550.45|185.98|734.13|2110.09|2324.75|2510.73|4434.84|4620.82|-3809.60| +2451494|77622|2451563|16028|82370|1196688|606|12477|48729|763093|631|14149|40|1|14|4|287|293|74|55.64|137.98|102.10|2655.12|7555.40|4117.36|10210.52|0.00|0.00|1735.30|7555.40|7555.40|9290.70|9290.70|3438.04| +2451494|77622|2451575|17041|82370|1196688|606|12477|48729|763093|631|14149|38|13|4|4|291|293|20|64.42|115.95|73.04|858.20|1460.80|1288.40|2319.00|102.25|0.00|533.20|1460.80|1563.05|1994.00|2096.25|172.40| +2451494|77622|2451540|3193|82370|1196688|606|12477|48729|763093|631|14149|44|2|8|2|276|293|52|98.46|288.48|285.59|150.28|14850.68|5119.92|15000.96|0.00|0.00|3450.20|14850.68|14850.68|18300.88|18300.88|9730.76| +2451494|77622|2451594|2150|82370|1196688|606|12477|48729|763093|631|14149|38|4|16|1|199|293|64|53.15|65.37|9.15|3598.08|585.60|3401.60|4183.68|5.85|0.00|2049.92|585.60|591.45|2635.52|2641.37|-2816.00| +2451494|77622|2451503|17950|82370|1196688|606|12477|48729|763093|631|14149|38|8|13|3|96|293|39|46.18|108.52|83.56|973.44|3258.84|1801.02|4232.28|162.94|0.00|1015.56|3258.84|3421.78|4274.40|4437.34|1457.82| +2452072|18448|2452127|8967|82842|1877292|6142|44992|88138|1517671|5189|44839|31|15|3|3|203|294|43|39.05|39.05|37.48|67.51|1611.64|1679.15|1679.15|8.05|1208.73|318.63|402.91|410.96|721.54|729.59|-1276.24| +2452072|18448|2452161|1141|82842|1877292|6142|44992|88138|1517671|5189|44839|53|23|17|4|64|294|6|14.23|33.58|16.45|102.78|98.70|85.38|201.48|3.94|0.00|8.04|98.70|102.64|106.74|110.68|13.32| +2452072|18448|2452125|16209|82842|1877292|6142|44992|88138|1517671|5189|44839|23|19|1|4|237|294|88|15.98|19.49|16.37|274.56|1440.56|1406.24|1715.12|47.25|259.30|600.16|1181.26|1228.51|1781.42|1828.67|-224.98| +2452072|18448|2452144|12887|82842|1877292|6142|44992|88138|1517671|5189|44839|53|25|6|2|45|294|39|38.89|81.28|69.90|443.82|2726.10|1516.71|3169.92|27.26|0.00|0.00|2726.10|2753.36|2726.10|2753.36|1209.39| +2452072|18448|2452139|11897|82842|1877292|6142|44992|88138|1517671|5189|44839|49|11|14|5|256|294|93|4.73|12.62|11.48|106.02|1067.64|439.89|1173.66|32.02|0.00|222.27|1067.64|1099.66|1289.91|1321.93|627.75| +2452072|18448|2452129|8615|82842|1877292|6142|44992|88138|1517671|5189|44839|29|23|12|4|259|294|9|62.34|72.31|72.31|0.00|650.79|561.06|650.79|6.50|0.00|266.76|650.79|657.29|917.55|924.05|89.73| +2452072|18448|2452086|13967|82842|1877292|6142|44992|88138|1517671|5189|44839|37|9|4|5|159|294|28|43.38|128.83|79.87|1370.88|2236.36|1214.64|3607.24|67.09|0.00|829.64|2236.36|2303.45|3066.00|3133.09|1021.72| +2452072|18448|2452117|981|82842|1877292|6142|44992|88138|1517671|5189|44839|37|11|3|5|267|294|23|54.43|55.51|52.17|76.82|1199.91|1251.89|1276.73|0.00|599.95|395.60|599.96|599.96|995.56|995.56|-651.93| +2452072|18448|2452097|8969|82842|1877292|6142|44992|88138|1517671|5189|44839|55|21|14|5|103|294|11|9.79|25.45|13.74|128.81|151.14|107.69|279.95|9.06|0.00|123.09|151.14|160.20|274.23|283.29|43.45| +2452072|18448|2452078|17377|82842|1877292|6142|44992|88138|1517671|5189|44839|57|27|19|4|149|294|69|81.56|177.80|17.78|11041.38|1226.82|5627.64|12268.20|12.26|0.00|5029.41|1226.82|1239.08|6256.23|6268.49|-4400.82| +2452072|18448|2452137|4951|82842|1877292|6142|44992|88138|1517671|5189|44839|1|3|16|2|298|294|88|9.89|20.86|12.51|734.80|1100.88|870.32|1835.68|1.32|1067.85|366.96|33.03|34.35|399.99|401.31|-837.29| +2452072|18448|2452116|7265|82842|1877292|6142|44992|88138|1517671|5189|44839|53|1|2|3|34|294|81|65.36|141.17|87.52|4345.65|7089.12|5294.16|11434.77|638.02|0.00|2744.28|7089.12|7727.14|9833.40|10471.42|1794.96| +2452072|18448|2452128|6393|82842|1877292|6142|44992|88138|1517671|5189|44839|9|3|19|1|136|294|54|9.15|13.63|2.72|589.14|146.88|494.10|736.02|4.40|0.00|213.30|146.88|151.28|360.18|364.58|-347.22| +2452613|47998|2452686|10983|14703|1392052|954|17856|40418|875105|2162|27511|1|18|4|2|197|295|87|29.58|33.42|31.41|174.87|2732.67|2573.46|2907.54|81.98|0.00|1104.03|2732.67|2814.65|3836.70|3918.68|159.21| +2452613|47998|2452635|13107|14703|1392052|954|17856|40418|875105|2162|27511|51|7|12|2|95|295|53|8.97|17.49|2.44|797.65|129.32|475.41|926.97|6.46|0.00|27.56|129.32|135.78|156.88|163.34|-346.09| +2452613|47998|2452697|9222|14703|1392052|954|17856|40418|875105|2162|27511|1|9|2|4|236|295|84|57.13|128.54|25.70|8638.56|2158.80|4798.92|10797.36|64.76|0.00|971.04|2158.80|2223.56|3129.84|3194.60|-2640.12| +2452613|47998|2452717|9427|14703|1392052|954|17856|40418|875105|2162|27511|43|24|15|5|118|295|76|51.36|62.14|56.54|425.60|4297.04|3903.36|4722.64|0.00|0.00|2219.20|4297.04|4297.04|6516.24|6516.24|393.68| +2452613|47998|2452644|17388|14703|1392052|954|17856|40418|875105|2162|27511|51|13|5|1|84|295|52|15.50|32.24|9.34|1190.80|485.68|806.00|1676.48|15.34|101.99|385.32|383.69|399.03|769.01|784.35|-422.31| +2452613|47998|2452632|6553|14703|1392052|954|17856|40418|875105|2162|27511|39|25|19|5|218|295|13|66.81|170.36|151.62|243.62|1971.06|868.53|2214.68|157.68|0.00|775.06|1971.06|2128.74|2746.12|2903.80|1102.53| +2452613|47998|2452712|16281|14703|1392052|954|17856|40418|875105|2162|27511|7|19|16|5|231|295|100|1.70|1.73|0.34|139.00|34.00|170.00|173.00|1.70|0.00|10.00|34.00|35.70|44.00|45.70|-136.00| +2452613|47998|2452647|7413|14703|1392052|954|17856|40418|875105|2162|27511|45|21|6|1|259|295|80|74.88|193.93|116.35|6206.40|9308.00|5990.40|15514.40|465.40|0.00|3723.20|9308.00|9773.40|13031.20|13496.60|3317.60| +2452613|47998|2452670|17815|14703|1392052|954|17856|40418|875105|2162|27511|36|7|13|2|218|295|34|82.20|241.66|215.07|904.06|7312.38|2794.80|8216.44|50.45|2266.83|1643.22|5045.55|5096.00|6688.77|6739.22|2250.75| +2452613|47998|2452680|2065|14703|1392052|954|17856|40418|875105|2162|27511|51|15|7|1|88|295|22|60.14|74.57|65.62|196.90|1443.64|1323.08|1640.54|129.92|0.00|98.34|1443.64|1573.56|1541.98|1671.90|120.56| +2452526|19568|2452592|3930|28455|1612569|1483|5150|49543|503466|692|35606|54|25|2|1|197|296|70|65.04|100.16|11.01|6240.50|770.70|4552.80|7011.20|23.12|0.00|700.70|770.70|793.82|1471.40|1494.52|-3782.10| +2452526|19568|2452580|5419|28455|1612569|1483|5150|49543|503466|692|35606|48|12|18|5|42|296|25|78.84|213.65|23.50|4753.75|587.50|1971.00|5341.25|11.75|0.00|1976.25|587.50|599.25|2563.75|2575.50|-1383.50| +2452526|19568|2452635|4461|28455|1612569|1483|5150|49543|503466|692|35606|39|19|13|1|116|296|49|49.63|131.02|49.78|3980.76|2439.22|2431.87|6419.98|146.35|0.00|1476.37|2439.22|2585.57|3915.59|4061.94|7.35| +2452526|19568|2452618|3663|28455|1612569|1483|5150|49543|503466|692|35606|3|21|20|4|248|296|44|71.75|134.89|126.79|356.40|5578.76|3157.00|5935.16|334.72|0.00|1661.44|5578.76|5913.48|7240.20|7574.92|2421.76| +2452526|19568|2452544|13956|28455|1612569|1483|5150|49543|503466|692|35606|57|7|19|1|86|296|41|16.90|34.81|17.40|713.81|713.40|692.90|1427.21|0.00|542.18|542.02|171.22|171.22|713.24|713.24|-521.68| +2452526|19568|2452598|13266|28455|1612569|1483|5150|49543|503466|692|35606|24|30|8|1|294|296|44|77.27|161.49|106.58|2416.04|4689.52|3399.88|7105.56|281.37|0.00|2273.48|4689.52|4970.89|6963.00|7244.37|1289.64| +2452526|19568|2452595|519|28455|1612569|1483|5150|49543|503466|692|35606|3|15|14|5|86|296|51|58.64|97.34|36.01|3127.83|1836.51|2990.64|4964.34|18.36|0.00|2134.35|1836.51|1854.87|3970.86|3989.22|-1154.13| +2452526|19568|2452638|16387|28455|1612569|1483|5150|49543|503466|692|35606|18|19|11|2|80|296|7|32.10|52.96|37.07|111.23|259.49|224.70|370.72|10.37|0.00|137.13|259.49|269.86|396.62|406.99|34.79| +2452526|19568|2452615|16743|28455|1612569|1483|5150|49543|503466|692|35606|18|13|5|3|211|296|98|59.48|176.06|24.64|14839.16|2414.72|5829.04|17253.88|169.03|0.00|344.96|2414.72|2583.75|2759.68|2928.71|-3414.32| +2452526|19568|2452608|16602|28455|1612569|1483|5150|49543|503466|692|35606|45|25|6|4|292|296|84|32.82|96.16|82.69|1131.48|6945.96|2756.88|8077.44|416.75|0.00|2180.64|6945.96|7362.71|9126.60|9543.35|4189.08| +2452526|19568|2452559|8197|28455|1612569|1483|5150|49543|503466|692|35606|9|25|18|1|81|296|37|45.28|47.99|46.55|53.28|1722.35|1675.36|1775.63|86.11|0.00|230.51|1722.35|1808.46|1952.86|2038.97|46.99| +2452526|19568|2452629|91|28455|1612569|1483|5150|49543|503466|692|35606|13|27|8|5|162|296|41|92.66|119.53|81.28|1568.25|3332.48|3799.06|4900.73|133.29|0.00|538.74|3332.48|3465.77|3871.22|4004.51|-466.58| +2451889|62441|2452003|14930|86418|818829|1640|35251|91699|307347|4884|48|44|1|2|5|108|297|5|17.78|31.82|8.27|117.75|41.35|88.90|159.10|3.30|0.00|28.60|41.35|44.65|69.95|73.25|-47.55| +2451889|62441|2451970|9872|86418|818829|1640|35251|91699|307347|4884|48|55|2|19|1|23|297|75|91.75|196.34|11.78|13842.00|883.50|6881.25|14725.50|0.00|0.00|3828.00|883.50|883.50|4711.50|4711.50|-5997.75| +2451889|62441|2451893|5803|86418|818829|1640|35251|91699|307347|4884|48|53|17|1|3|45|297|100|65.56|173.73|55.59|11814.00|5559.00|6556.00|17373.00|111.18|0.00|1911.00|5559.00|5670.18|7470.00|7581.18|-997.00| +2451889|62441|2451898|13487|86418|818829|1640|35251|91699|307347|4884|48|5|7|2|3|134|297|78|66.84|143.70|20.11|9640.02|1568.58|5213.52|11208.60|62.74|0.00|1568.58|1568.58|1631.32|3137.16|3199.90|-3644.94| +2451889|62441|2451996|16505|86418|818829|1640|35251|91699|307347|4884|48|31|29|14|3|245|297|24|66.17|168.07|149.58|443.76|3589.92|1588.08|4033.68|143.59|0.00|1411.68|3589.92|3733.51|5001.60|5145.19|2001.84| +2451889|62441|2451985|3095|86418|818829|1640|35251|91699|307347|4884|48|23|14|7|3|169|297|16|59.98|145.15|94.34|812.96|1509.44|959.68|2322.40|60.37|0.00|557.28|1509.44|1569.81|2066.72|2127.09|549.76| +2451889|62441|2451989|8150|86418|818829|1640|35251|91699|307347|4884|48|17|1|20|4|118|297|14|98.98|283.08|212.31|990.78|2972.34|1385.72|3963.12|118.89|0.00|1941.80|2972.34|3091.23|4914.14|5033.03|1586.62| +2451889|62441|2451938|8393|86418|818829|1640|35251|91699|307347|4884|48|37|7|9|4|79|297|63|88.74|108.26|61.70|2933.28|3887.10|5590.62|6820.38|38.87|0.00|2863.98|3887.10|3925.97|6751.08|6789.95|-1703.52| +2451889|62441|2452005|13088|86418|818829|1640|35251|91699|307347|4884|48|50|8|4|3|182|297|93|37.29|100.68|10.06|8427.66|935.58|3467.97|9363.24|56.13|0.00|1685.16|935.58|991.71|2620.74|2676.87|-2532.39| +2451889|62441|2451975|2930|86418|818829|1640|35251|91699|307347|4884|48|43|8|8|1|224|297|34|72.39|161.42|104.92|1921.00|3567.28|2461.26|5488.28|107.01|0.00|823.14|3567.28|3674.29|4390.42|4497.43|1106.02| +2452604|80419|2452724|7692|15342|1238517|5692|48467|68462|1012626|4792|25967|55|13|16|2|191|298|60|9.80|26.46|6.08|1222.80|364.80|588.00|1587.60|21.88|0.00|603.00|364.80|386.68|967.80|989.68|-223.20| +2452604|80419|2452641|3468|15342|1238517|5692|48467|68462|1012626|4792|25967|48|13|20|3|256|298|53|46.17|130.66|1.30|6856.08|68.90|2447.01|6924.98|0.00|0.00|2354.26|68.90|68.90|2423.16|2423.16|-2378.11| +2452604|80419|2452647|3180|15342|1238517|5692|48467|68462|1012626|4792|25967|39|3|14|3|255|298|87|77.99|182.49|166.06|1429.41|14447.22|6785.13|15876.63|722.36|0.00|316.68|14447.22|15169.58|14763.90|15486.26|7662.09| +2452604|80419|2452691|11743|15342|1238517|5692|48467|68462|1012626|4792|25967|36|6|8|3|235|298|60|24.08|28.17|14.64|811.80|878.40|1444.80|1690.20|17.56|0.00|135.00|878.40|895.96|1013.40|1030.96|-566.40| +2452604|80419|2452636|16363|15342|1238517|5692|48467|68462|1012626|4792|25967|36|24|6|3|287|298|27|5.28|7.23|4.26|80.19|115.02|142.56|195.21|0.00|0.00|66.15|115.02|115.02|181.17|181.17|-27.54| +2452604|80419|2452699|5046|15342|1238517|5692|48467|68462|1012626|4792|25967|49|12|7|3|249|298|58|34.44|101.59|32.50|4007.22|1885.00|1997.52|5892.22|113.10|0.00|2356.54|1885.00|1998.10|4241.54|4354.64|-112.52| +2452604|80419|2452680|14083|15342|1238517|5692|48467|68462|1012626|4792|25967|37|27|2|2|22|298|81|44.52|110.40|46.36|5187.24|3755.16|3606.12|8942.40|37.55|0.00|4113.18|3755.16|3792.71|7868.34|7905.89|149.04| +2452604|80419|2452706|1572|15342|1238517|5692|48467|68462|1012626|4792|25967|15|30|12|4|276|298|67|69.24|183.48|64.21|7991.09|4302.07|4639.08|12293.16|215.10|0.00|614.39|4302.07|4517.17|4916.46|5131.56|-337.01| +2452604|80419|2452674|17604|15342|1238517|5692|48467|68462|1012626|4792|25967|12|1|9|4|69|298|62|71.42|106.41|17.02|5542.18|1055.24|4428.04|6597.42|73.86|0.00|1649.20|1055.24|1129.10|2704.44|2778.30|-3372.80| +2452604|80419|2452691|9949|15342|1238517|5692|48467|68462|1012626|4792|25967|15|7|7|3|179|298|60|48.35|125.71|65.36|3621.00|3921.60|2901.00|7542.60|196.08|0.00|980.40|3921.60|4117.68|4902.00|5098.08|1020.60| +2452604|80419|2452687|5754|15342|1238517|5692|48467|68462|1012626|4792|25967|12|15|20|2|104|298|54|59.12|111.73|29.04|4465.26|1568.16|3192.48|6033.42|141.13|0.00|603.18|1568.16|1709.29|2171.34|2312.47|-1624.32| +2452604|80419|2452607|14712|15342|1238517|5692|48467|68462|1012626|4792|25967|60|27|18|2|33|298|86|55.20|112.60|1.12|9587.28|96.32|4747.20|9683.60|2.54|64.53|2226.54|31.79|34.33|2258.33|2260.87|-4715.41| +2451573|31270|2451618|11636|99606|648580|307|30034|99606|648580|307|30034|59|8|3|3|216|299|37|94.40|196.35|147.26|1816.33|5448.62|3492.80|7264.95|108.97|0.00|2179.30|5448.62|5557.59|7627.92|7736.89|1955.82| +2451573|31270|2451661|13499|99606|648580|307|30034|99606|648580|307|30034|19|7|5|4|150|299|26|42.21|42.63|21.74|543.14|565.24|1097.46|1108.38|45.21|0.00|476.58|565.24|610.45|1041.82|1087.03|-532.22| +2451573|31270|2451676|8984|99606|648580|307|30034|99606|648580|307|30034|19|23|14|1|108|299|83|6.89|18.46|13.47|414.17|1118.01|571.87|1532.18|3.91|1039.74|0.00|78.27|82.18|78.27|82.18|-493.60| +2451573|31270|2451686|2185|99606|648580|307|30034|99606|648580|307|30034|11|14|10|2|97|299|46|26.84|29.52|11.51|828.46|529.46|1234.64|1357.92|5.29|0.00|542.80|529.46|534.75|1072.26|1077.55|-705.18| +2451573|31270|2451646|2035|99606|648580|307|30034|99606|648580|307|30034|37|29|11|5|116|299|4|76.24|208.13|108.22|399.64|432.88|304.96|832.52|25.97|0.00|108.20|432.88|458.85|541.08|567.05|127.92| +2451573|31270|2451625|7163|99606|648580|307|30034|99606|648580|307|30034|53|7|14|1|11|299|20|11.05|18.45|11.99|129.20|239.80|221.00|369.00|2.39|0.00|180.80|239.80|242.19|420.60|422.99|18.80| +2451573|31270|2451608|1993|99606|648580|307|30034|99606|648580|307|30034|44|7|2|3|284|299|94|3.66|9.58|7.08|235.00|665.52|344.04|900.52|13.31|0.00|152.28|665.52|678.83|817.80|831.11|321.48| +2451573|31270|2451659|3452|99606|648580|307|30034|99606|648580|307|30034|44|1|9|2|175|299|98|57.13|168.53|48.87|11726.68|4789.26|5598.74|16515.94|383.14|0.00|990.78|4789.26|5172.40|5780.04|6163.18|-809.48| +2451472|39003|2451555|2227|40249|1792155|61|29967|89535|1852534|1285|49591|26|20|8|5|279|300|81|88.85|237.22|225.35|961.47|18253.35|7196.85|19214.82|912.66|0.00|5571.99|18253.35|19166.01|23825.34|24738.00|11056.50| +2451472|39003|2451590|122|40249|1792155|61|29967|89535|1852534|1285|49591|1|26|7|5|23|300|48|15.78|23.35|15.64|370.08|750.72|757.44|1120.80|0.00|675.64|537.60|75.08|75.08|612.68|612.68|-682.36| +2451472|39003|2451503|14168|40249|1792155|61|29967|89535|1852534|1285|49591|49|20|17|5|36|300|2|3.19|8.83|0.44|16.78|0.88|6.38|17.66|0.00|0.00|2.28|0.88|0.88|3.16|3.16|-5.50| +2451472|39003|2451551|5623|40249|1792155|61|29967|89535|1852534|1285|49591|56|4|7|1|165|300|54|29.39|78.17|17.97|3250.80|970.38|1587.06|4221.18|19.40|0.00|1392.66|970.38|989.78|2363.04|2382.44|-616.68| +2451472|39003|2451493|5509|40249|1792155|61|29967|89535|1852534|1285|49591|56|16|9|1|46|300|16|64.44|85.70|38.56|754.24|616.96|1031.04|1371.20|30.84|0.00|452.48|616.96|647.80|1069.44|1100.28|-414.08| +2451472|39003|2451510|17686|40249|1792155|61|29967|89535|1852534|1285|49591|49|20|17|3|299|300|77|19.13|57.19|8.00|3787.63|616.00|1473.01|4403.63|12.32|0.00|1717.10|616.00|628.32|2333.10|2345.42|-857.01| +2451472|39003|2451566|9146|40249|1792155|61|29967|89535|1852534|1285|49591|34|10|4|2|30|300|3|95.65|106.17|104.04|6.39|312.12|286.95|318.51|24.96|0.00|117.84|312.12|337.08|429.96|454.92|25.17| +2451472|39003|2451481|2116|40249|1792155|61|29967|89535|1852534|1285|49591|19|28|13|1|106|300|44|9.92|28.17|20.56|334.84|904.64|436.48|1239.48|72.37|0.00|520.52|904.64|977.01|1425.16|1497.53|468.16| +2451472|39003|2451547|6578|40249|1792155|61|29967|89535|1852534|1285|49591|26|28|3|1|212|300|91|12.36|17.67|14.48|290.29|1317.68|1124.76|1607.97|118.59|0.00|707.07|1317.68|1436.27|2024.75|2143.34|192.92| +2451472|39003|2451569|14935|40249|1792155|61|29967|89535|1852534|1285|49591|19|8|10|5|230|300|79|9.46|11.44|11.09|27.65|876.11|747.34|903.76|15.33|569.47|171.43|306.64|321.97|478.07|493.40|-440.70| +2451472|39003|2451588|3484|40249|1792155|61|29967|89535|1852534|1285|49591|40|8|6|5|121|300|68|85.62|253.43|212.88|2757.40|14475.84|5822.16|17233.24|44.87|9988.32|0.00|4487.52|4532.39|4487.52|4532.39|-1334.64| +2451472|39003|2451551|2138|40249|1792155|61|29967|89535|1852534|1285|49591|46|4|9|3|157|300|27|93.92|185.96|3.71|4920.75|100.17|2535.84|5020.92|2.00|75.12|1305.18|25.05|27.05|1330.23|1332.23|-2510.79| +2451472|39003|2451486|14116|40249|1792155|61|29967|89535|1852534|1285|49591|10|14|3|5|180|300|17|89.64|242.02|229.91|205.87|3908.47|1523.88|4114.34|156.33|0.00|1604.46|3908.47|4064.80|5512.93|5669.26|2384.59| +2451472|39003|2451544|2749|40249|1792155|61|29967|89535|1852534|1285|49591|49|28|18|3|8|300|69|79.47|145.43|81.44|4415.31|5619.36|5483.43|10034.67|393.35|0.00|3009.78|5619.36|6012.71|8629.14|9022.49|135.93| +2451054|61204|2451169|6427|8526|887421|1069|17164|57380|727291|555|22603|14|7|20|1|82|301|2|50.67|149.98|77.98|144.00|155.96|101.34|299.96|3.11|0.00|107.98|155.96|159.07|263.94|267.05|54.62| +2451054|61204|2451158|1126|8526|887421|1069|17164|57380|727291|555|22603|49|16|13|4|28|301|60|27.47|38.45|1.92|2191.80|115.20|1648.20|2307.00|4.60|0.00|345.60|115.20|119.80|460.80|465.40|-1533.00| +2451054|61204|2451097|6781|8526|887421|1069|17164|57380|727291|555|22603|13|14|9|5|288|301|54|69.24|203.56|189.31|769.50|10222.74|3738.96|10992.24|817.81|0.00|549.18|10222.74|11040.55|10771.92|11589.73|6483.78| +2451054|61204|2451149|16160|8526|887421|1069|17164|57380|727291|555|22603|43|28|1|2|273|301|82|45.87|108.25|34.64|6036.02|2840.48|3761.34|8876.50|126.40|312.45|710.12|2528.03|2654.43|3238.15|3364.55|-1233.31| +2451054|61204|2451157|535|8526|887421|1069|17164|57380|727291|555|22603|1|25|18|3|35|301|68|4.60|6.71|1.47|356.32|99.96|312.80|456.28|4.99|0.00|200.60|99.96|104.95|300.56|305.55|-212.84| +2451054|61204|2451151|2611|8526|887421|1069|17164|57380|727291|555|22603|22|1|3|2|81|301|22|47.66|74.34|31.22|948.64|686.84|1048.52|1635.48|61.81|0.00|637.78|686.84|748.65|1324.62|1386.43|-361.68| +2451054|61204|2451094|374|8526|887421|1069|17164|57380|727291|555|22603|13|4|19|5|141|301|53|50.65|112.44|110.19|119.25|5840.07|2684.45|5959.32|408.80|0.00|2621.91|5840.07|6248.87|8461.98|8870.78|3155.62| +2451054|61204|2451139|5245|8526|887421|1069|17164|57380|727291|555|22603|52|4|2|3|194|301|52|79.81|192.34|46.16|7601.36|2400.32|4150.12|10001.68|0.00|0.00|2299.96|2400.32|2400.32|4700.28|4700.28|-1749.80| +2451054|61204|2451069|6800|8526|887421|1069|17164|57380|727291|555|22603|19|13|17|4|74|301|63|23.65|28.38|19.29|572.67|1215.27|1489.95|1787.94|45.08|571.17|661.50|644.10|689.18|1305.60|1350.68|-845.85| +2451054|61204|2451068|11998|8526|887421|1069|17164|57380|727291|555|22603|16|26|16|1|43|301|27|22.39|62.02|31.01|837.27|837.27|604.53|1674.54|58.60|0.00|83.70|837.27|895.87|920.97|979.57|232.74| +2451054|61204|2451089|7262|8526|887421|1069|17164|57380|727291|555|22603|32|8|20|5|13|301|66|87.90|218.87|157.58|4045.14|10400.28|5801.40|14445.42|171.60|4680.12|5055.60|5720.16|5891.76|10775.76|10947.36|-81.24| +2451054|61204|2451055|5270|8526|887421|1069|17164|57380|727291|555|22603|40|8|9|3|47|301|70|8.91|19.15|0.38|1313.90|26.60|623.70|1340.50|1.59|0.00|495.60|26.60|28.19|522.20|523.79|-597.10| +2451054|61204|2451093|4592|8526|887421|1069|17164|57380|727291|555|22603|13|2|10|5|153|301|50|82.52|82.52|39.60|2146.00|1980.00|4126.00|4126.00|94.44|930.60|41.00|1049.40|1143.84|1090.40|1184.84|-3076.60| +2452444|26831|2452457|5892|42082|1449882|3635|45835|43554|527248|6741|11741|24|19|5|3|18|302|5|64.68|164.28|22.99|706.45|114.95|323.40|821.40|9.19|0.00|73.90|114.95|124.14|188.85|198.04|-208.45| +2452444|26831|2452532|15211|42082|1449882|3635|45835|43554|527248|6741|11741|6|30|14|1|35|302|2|15.99|22.54|4.95|35.18|9.90|31.98|45.08|0.89|0.00|10.36|9.90|10.79|20.26|21.15|-22.08| +2452444|26831|2452555|4044|42082|1449882|3635|45835|43554|527248|6741|11741|48|1|8|5|112|302|32|76.52|112.48|68.61|1403.84|2195.52|2448.64|3599.36|109.77|0.00|1259.52|2195.52|2305.29|3455.04|3564.81|-253.12| +2452444|26831|2452495|5916|42082|1449882|3635|45835|43554|527248|6741|11741|12|3|1|2|186|302|81|55.39|125.73|2.51|9980.82|203.31|4486.59|10184.13|8.13|0.00|1018.17|203.31|211.44|1221.48|1229.61|-4283.28| +2452444|26831|2452562|11901|42082|1449882|3635|45835|43554|527248|6741|11741|43|27|16|3|110|302|62|35.12|94.12|31.05|3910.34|1925.10|2177.44|5835.44|0.00|0.00|1399.96|1925.10|1925.10|3325.06|3325.06|-252.34| +2452444|26831|2452458|4447|42082|1449882|3635|45835|43554|527248|6741|11741|57|18|16|1|23|302|34|91.85|172.67|139.86|1115.54|4755.24|3122.90|5870.78|278.18|1664.33|2171.92|3090.91|3369.09|5262.83|5541.01|-31.99| +2452444|26831|2452564|14304|42082|1449882|3635|45835|43554|527248|6741|11741|43|30|1|4|137|302|100|8.64|22.63|11.99|1064.00|1199.00|864.00|2263.00|107.91|0.00|973.00|1199.00|1306.91|2172.00|2279.91|335.00| +2452444|26831|2452544|4590|42082|1449882|3635|45835|43554|527248|6741|11741|33|9|3|1|299|302|61|63.76|64.39|63.74|39.65|3888.14|3889.36|3927.79|233.28|0.00|470.92|3888.14|4121.42|4359.06|4592.34|-1.22| +2452444|26831|2452484|4911|42082|1449882|3635|45835|43554|527248|6741|11741|30|7|6|3|227|302|51|94.17|145.96|144.50|74.46|7369.50|4802.67|7443.96|73.69|0.00|893.01|7369.50|7443.19|8262.51|8336.20|2566.83| +2452444|26831|2452447|16428|42082|1449882|3635|45835|43554|527248|6741|11741|12|24|5|3|261|302|39|46.86|129.33|19.39|4287.66|756.21|1827.54|5043.87|37.81|0.00|352.95|756.21|794.02|1109.16|1146.97|-1071.33| +2452444|26831|2452554|16227|42082|1449882|3635|45835|43554|527248|6741|11741|55|21|14|4|117|302|2|40.39|41.19|39.95|2.48|79.90|80.78|82.38|6.39|0.00|3.28|79.90|86.29|83.18|89.57|-0.88| +2452444|26831|2452457|1233|42082|1449882|3635|45835|43554|527248|6741|11741|31|9|14|3|272|302|22|68.11|137.58|34.39|2270.18|756.58|1498.42|3026.76|37.82|0.00|1483.02|756.58|794.40|2239.60|2277.42|-741.84| +2452444|26831|2452455|1543|42082|1449882|3635|45835|43554|527248|6741|11741|12|21|13|3|160|302|23|96.87|161.77|137.50|558.21|3162.50|2228.01|3720.71|63.25|0.00|781.31|3162.50|3225.75|3943.81|4007.06|934.49| +2452444|26831|2452490|3423|42082|1449882|3635|45835|43554|527248|6741|11741|13|9|6|2|255|302|71|58.17|60.49|48.39|859.10|3435.69|4130.07|4294.79|126.43|1855.27|1760.80|1580.42|1706.85|3341.22|3467.65|-2549.65| +2452444|26831|2452522|16257|42082|1449882|3635|45835|43554|527248|6741|11741|36|13|9|1|43|302|86|60.44|118.46|13.03|9066.98|1120.58|5197.84|10187.56|11.20|0.00|101.48|1120.58|1131.78|1222.06|1233.26|-4077.26| +2452444|26831|2452528|8365|42082|1449882|3635|45835|43554|527248|6741|11741|45|1|19|2|138|302|78|67.63|116.32|55.83|4718.22|4354.74|5275.14|9072.96|348.37|0.00|3175.38|4354.74|4703.11|7530.12|7878.49|-920.40| +2451061|62400|2451111|7180|79749|427209|5869|26529|18083|143203|3709|5569|25|13|11|3|162|303|22|91.72|163.26|37.54|2765.84|825.88|2017.84|3591.72|74.32|0.00|394.90|825.88|900.20|1220.78|1295.10|-1191.96| +2451061|62400|2451142|17209|79749|427209|5869|26529|18083|143203|3709|5569|31|22|15|1|284|303|1|37.69|97.61|27.33|70.28|27.33|37.69|97.61|1.09|0.00|18.54|27.33|28.42|45.87|46.96|-10.36| +2451061|62400|2451100|14077|79749|427209|5869|26529|18083|143203|3709|5569|13|8|6|4|45|303|45|76.67|211.60|57.13|6951.15|2570.85|3450.15|9522.00|128.54|0.00|4475.25|2570.85|2699.39|7046.10|7174.64|-879.30| +2451061|62400|2451121|11486|79749|427209|5869|26529|18083|143203|3709|5569|25|28|17|2|14|303|38|52.54|118.74|27.31|3474.34|1037.78|1996.52|4512.12|93.40|0.00|496.28|1037.78|1131.18|1534.06|1627.46|-958.74| +2451061|62400|2451117|16771|79749|427209|5869|26529|18083|143203|3709|5569|28|19|8|3|18|303|29|15.78|39.13|14.47|715.14|419.63|457.62|1134.77|33.57|0.00|294.93|419.63|453.20|714.56|748.13|-37.99| +2451061|62400|2451157|13219|79749|427209|5869|26529|18083|143203|3709|5569|34|16|17|3|266|303|6|64.80|143.20|94.51|292.14|567.06|388.80|859.20|11.34|0.00|163.20|567.06|578.40|730.26|741.60|178.26| +|62400||16285|||5869||18083||3709|5569|56|||3||303||81.93||119.05|||2212.11|3826.71|32.14|||||4936.14|4968.28|1002.24| +2451061|62400|2451152|8389|79749|427209|5869|26529|18083|143203|3709|5569|13|26|18|2|86|303|89|32.03|93.20|88.54|414.74|7880.06|2850.67|8294.80|0.00|0.00|2985.95|7880.06|7880.06|10866.01|10866.01|5029.39| +2451330|73880|2451419|1414|30421|393279|5290|34948|24713|490216|2675|1886|38|8|17|2|147|304|52|33.22|44.18|15.90|1470.56|826.80|1727.44|2297.36|7.85|669.70|872.56|157.10|164.95|1029.66|1037.51|-1570.34| +2451330|73880|2451442|7900|30421|393279|5290|34948|24713|490216|2675|1886|8|13|9|1|126|304|58|15.06|31.77|19.06|737.18|1105.48|873.48|1842.66|66.32|0.00|128.76|1105.48|1171.80|1234.24|1300.56|232.00| +2451330|73880|2451404|2125|30421|393279|5290|34948|24713|490216|2675|1886|4|16|2|5|131|304|94|93.41|233.52|107.41|11854.34|10096.54|8780.54|21950.88|403.86|0.00|219.02|10096.54|10500.40|10315.56|10719.42|1316.00| +2451330|73880|2451408|14122|30421|393279|5290|34948|24713|490216|2675|1886|50|4|10|4|91|304|94|83.80|232.96|230.63|219.02|21679.22|7877.20|21898.24|867.16|0.00|1532.20|21679.22|22546.38|23211.42|24078.58|13802.02| +2451330|73880|2451404|6784|30421|393279|5290|34948|24713|490216|2675|1886|44|2|1|3|82|304|2|19.52|21.66|15.81|11.70|31.62|39.04|43.32|0.94|0.00|11.68|31.62|32.56|43.30|44.24|-7.42| +2451330|73880|2451435|1298|30421|393279|5290|34948|24713|490216|2675|1886|19|4|3|3|116|304|82|99.62|175.33|106.95|5607.16|8769.90|8168.84|14377.06|0.00|6665.12|3594.06|2104.78|2104.78|5698.84|5698.84|-6064.06| +2451330|73880|2451432|15175|30421|393279|5290|34948|24713|490216|2675|1886|37|26|12|5|188|304|66|76.43|159.73|54.30|6958.38|3583.80|5044.38|10542.18|215.02|0.00|104.94|3583.80|3798.82|3688.74|3903.76|-1460.58| +2451330|73880|2451367|4124|30421|393279|5290|34948|24713|490216|2675|1886|8|2|9|3|180|304|12|30.71|51.28|7.17|529.32|86.04|368.52|615.36|7.74|0.00|42.96|86.04|93.78|129.00|136.74|-282.48| +2451166|14556|2451193|16267|46707|1823795|1236|33924|33903|831618|5941|27782|8|7|6|3|167|305|42|81.24|196.60|96.33|4211.34|4045.86|3412.08|8257.20|364.12|0.00|2146.62|4045.86|4409.98|6192.48|6556.60|633.78| +2451166|14556|2451264|12004|46707|1823795|1236|33924|33903|831618|5941|27782|43|19|8|1|64|305|39|46.92|110.73|37.64|2850.51|1467.96|1829.88|4318.47|29.35|0.00|1122.42|1467.96|1497.31|2590.38|2619.73|-361.92| +2451166|14556|2451286|3061|46707|1823795|1236|33924|33903|831618|5941|27782|44|8|17|4|41|305|76|69.35|162.27|40.56|9249.96|3082.56|5270.60|12332.52|154.12|0.00|6165.88|3082.56|3236.68|9248.44|9402.56|-2188.04| +2451166|14556|2451188|13196|46707|1823795|1236|33924|33903|831618|5941|27782|20|10|14|4|14|305|32|36.02|68.79|64.66|132.16|2069.12|1152.64|2201.28|124.14|0.00|198.08|2069.12|2193.26|2267.20|2391.34|916.48| +2451166|14556|2451229|15662|46707|1823795|1236|33924|33903|831618|5941|27782|40|4|18|5|85|305|61|72.40|107.15|25.71|4967.84|1568.31|4416.40|6536.15|94.09|0.00|2025.81|1568.31|1662.40|3594.12|3688.21|-2848.09| +2451166|14556|2451265|13432|46707|1823795|1236|33924|33903|831618|5941|27782|46|1|18|2|119|305|9|36.13|52.38|10.47|377.19|94.23|325.17|471.42|2.82|0.00|146.07|94.23|97.05|240.30|243.12|-230.94| +2451166|14556|2451236|15931|46707|1823795|1236|33924|33903|831618|5941|27782|26|25|12|1|130|305|33|22.10|46.41|28.31|597.30|934.23|729.30|1531.53|37.36|0.00|627.66|934.23|971.59|1561.89|1599.25|204.93| +2451166|14556|2451188|7468|46707|1823795|1236|33924|33903|831618|5941|27782|19|4|6|1|139|305|12|41.40|93.15|59.61|402.48|715.32|496.80|1117.80|4.14|300.43|502.92|414.89|419.03|917.81|921.95|-81.91| +2451166|14556|2451199|10370|46707|1823795|1236|33924|33903|831618|5941|27782|43|7|13|2|181|305|15|85.79|114.95|88.51|396.60|1327.65|1286.85|1724.25|53.10|0.00|172.35|1327.65|1380.75|1500.00|1553.10|40.80| +2451166|14556|2451189|1576|46707|1823795|1236|33924|33903|831618|5941|27782|16|10|6|3|290|305|8|71.49|210.89|94.90|927.92|759.20|571.92|1687.12|22.77|0.00|438.64|759.20|781.97|1197.84|1220.61|187.28| +2451886|38848|2452006|8456|20718|549089|660|19384|87616|323990|1697|12468|50|14|7|1|245|306|58|83.32|89.98|81.88|469.80|4749.04|4832.56|5218.84|49.86|4036.68|1461.02|712.36|762.22|2173.38|2223.24|-4120.20| +2451886|38848|2451941|614|20718|549089|660|19384|87616|323990|1697|12468|47|19|3|1|102|306|54|88.55|150.53|70.74|4308.66|3819.96|4781.70|8128.62|190.99|0.00|3901.50|3819.96|4010.95|7721.46|7912.45|-961.74| +2451886|38848|2451908|953|20718|549089|660|19384|87616|323990|1697|12468|31|20|16|3|93|306|94|39.89|53.85|40.38|1266.18|3795.72|3749.66|5061.90|151.82|0.00|2024.76|3795.72|3947.54|5820.48|5972.30|46.06| +2451886|38848|2451979|3793|20718|549089|660|19384|87616|323990|1697|12468|29|25|14|2|270|306|59|92.71|139.99|41.99|5782.00|2477.41|5469.89|8259.41|222.96|0.00|412.41|2477.41|2700.37|2889.82|3112.78|-2992.48| +2451886|38848|2451995|14039|20718|549089|660|19384|87616|323990|1697|12468|5|5|3|5|280|306|55|37.52|81.04|77.79|178.75|4278.45|2063.60|4457.20|385.06|0.00|980.10|4278.45|4663.51|5258.55|5643.61|2214.85| +2451886|38848|2451890|12757|20718|549089|660|19384|87616|323990|1697|12468|49|7|20|4|128|306|55|82.43|98.91|70.22|1577.95|3862.10|4533.65|5440.05|77.24|0.00|2284.70|3862.10|3939.34|6146.80|6224.04|-671.55| +2451886|38848|2451931|7081|20718|549089|660|19384|87616|323990|1697|12468|20|17|7|5|123|306|34|90.21|235.44|91.82|4883.08|3121.88|3067.14|8004.96|29.03|218.53|240.04|2903.35|2932.38|3143.39|3172.42|-163.79| +2451886|38848|2451943|6626|20718|549089|660|19384|87616|323990|1697|12468|43|29|15|5|278|306|31|51.90|144.28|138.50|179.18|4293.50|1608.90|4472.68|42.93|0.00|1297.04|4293.50|4336.43|5590.54|5633.47|2684.60| +2451886|38848|2451927|4514|20718|549089|660|19384|87616|323990|1697|12468|29|20|7|5|216|306|49|87.98|234.90|159.73|3683.33|7826.77|4311.02|11510.10|156.53|0.00|5409.60|7826.77|7983.30|13236.37|13392.90|3515.75| +2451886|38848|2451995|7388|20718|549089|660|19384|87616|323990|1697|12468|14|2|13|4|58|306|80|36.27|96.11|40.36|4460.00|3228.80|2901.60|7688.80|64.57|0.00|922.40|3228.80|3293.37|4151.20|4215.77|327.20| +2451886|38848|2451941|14288|20718|549089|660|19384|87616|323990|1697|12468|17|13|8|4|10|306|92|61.01|83.58|38.44|4152.88|3536.48|5612.92|7689.36|318.28|0.00|153.64|3536.48|3854.76|3690.12|4008.40|-2076.44| +2451886|38848|2451942|8060|20718|549089|660|19384|87616|323990|1697|12468|25|2|11|5|59|306|45|17.99|49.83|38.36|516.15|1726.20|809.55|2242.35|155.35|0.00|291.15|1726.20|1881.55|2017.35|2172.70|916.65| +2451886|38848|2451992|7295|20718|549089|660|19384|87616|323990|1697|12468|32|17|18|5|89|306|91|56.33|139.69|50.28|8136.31|4575.48|5126.03|12711.79|45.75|0.00|4702.88|4575.48|4621.23|9278.36|9324.11|-550.55| +2451796|10330|2451870|1885|32501|1913277|3653|829|65206|821887|804|25580|31|25|10|5|87|307|85|78.36|127.72|102.17|2171.75|8684.45|6660.60|10856.20|0.00|0.00|4559.40|8684.45|8684.45|13243.85|13243.85|2023.85| +2451796|10330|2451896|7928|32501|1913277|3653|829|65206|821887|804|25580|35|13|3|2|251|307|81|38.51|96.66|90.86|469.80|7359.66|3119.31|7829.46|220.78|0.00|3836.16|7359.66|7580.44|11195.82|11416.60|4240.35| +2451796|10330|2451904|15343|32501|1913277|3653|829|65206|821887|804|25580|44|26|2|3|1|307|56|50.07|79.61|56.52|1293.04|3165.12|2803.92|4458.16|189.90|0.00|2005.92|3165.12|3355.02|5171.04|5360.94|361.20| +2451796|10330|2451831|4121|32501|1913277|3653|829|65206|821887|804|25580|5|29|7|1|88|307|7|62.36|71.71|34.42|261.03|240.94|436.52|501.97|16.86|0.00|10.01|240.94|257.80|250.95|267.81|-195.58| +2451796|10330|2451887|3971|32501|1913277|3653|829|65206|821887|804|25580|29|11|17|3|46|307|59|92.56|106.44|54.28|3077.44|3202.52|5461.04|6279.96|28.82|2241.76|0.00|960.76|989.58|960.76|989.58|-4500.28| +2451796|10330|2451811|8993|32501|1913277|3653|829|65206|821887|804|25580|55|17|20|1|162|307|56|18.73|20.41|16.73|206.08|936.88|1048.88|1142.96|37.47|0.00|491.12|936.88|974.35|1428.00|1465.47|-112.00| +2451796|10330|2451867|17045|32501|1913277|3653|829|65206|821887|804|25580|49|7|13|2|61|307|39|37.53|47.66|24.30|911.04|947.70|1463.67|1858.74|42.64|473.85|891.93|473.85|516.49|1365.78|1408.42|-989.82| +2451796|10330|2451915|5696|32501|1913277|3653|829|65206|821887|804|25580|20|14|12|1|43|307|33|47.76|140.41|101.09|1297.56|3335.97|1576.08|4633.53|66.71|0.00|1946.01|3335.97|3402.68|5281.98|5348.69|1759.89| +2451796|10330|2451831|10655|32501|1913277|3653|829|65206|821887|804|25580|49|5|4|2|20|307|61|19.77|30.64|24.51|373.93|1495.11|1205.97|1869.04|104.65|0.00|373.32|1495.11|1599.76|1868.43|1973.08|289.14| +2451796|10330|2451810|1880|32501|1913277|3653|829|65206|821887|804|25580|43|11|13|5|154|307|56|42.09|70.71|66.46|238.00|3721.76|2357.04|3959.76|334.95|0.00|1464.96|3721.76|4056.71|5186.72|5521.67|1364.72| +2451796|10330|2451886|2372|32501|1913277|3653|829|65206|821887|804|25580|7|11|15|2|199|307|6|22.23|27.56|0.00|165.36|0.00|133.38|165.36|0.00|0.00|36.36|0.00|0.00|36.36|36.36|-133.38| +2451796|10330|2451819|15950|32501|1913277|3653|829|65206|821887|804|25580|20|13|10|1|212|307|98|92.64|259.39|197.13|6101.48|19318.74|9078.72|25420.22|1738.68|0.00|3304.56|19318.74|21057.42|22623.30|24361.98|10240.02| +2451796|10330|2451854|5090|32501|1913277|3653|829|65206|821887|804|25580|11|19|14|5|149|307|41|16.15|33.10|18.86|583.84|773.26|662.15|1357.10|3.24|448.49|162.77|324.77|328.01|487.54|490.78|-337.38| +2451796|10330|2451868|8801|32501|1913277|3653|829|65206|821887|804|25580|2|25|4|2|293|307|51|8.20|21.48|8.16|679.32|416.16|418.20|1095.48|24.96|0.00|131.07|416.16|441.12|547.23|572.19|-2.04| +2451506|18531|2451521|16744|89845|1818687|2729|25218|53591|764405|4144|44186|19|16|5|4|43|308|5|94.50|153.09|105.63|237.30|528.15|472.50|765.45|10.56|0.00|275.55|528.15|538.71|803.70|814.26|55.65| +2451506|18531|2451610|4160|89845|1818687|2729|25218|53591|764405|4144|44186|2|14|7|1|102|308|41|85.53|156.51|100.16|2310.35|4106.56|3506.73|6416.91|123.19|0.00|3143.88|4106.56|4229.75|7250.44|7373.63|599.83| +2451506|18531|2451611|12859|89845|1818687|2729|25218|53591|764405|4144|44186|19|4|3|1|296|308|68|22.41|51.09|4.08|3196.68|277.44|1523.88|3474.12|8.73|152.59|799.00|124.85|133.58|923.85|932.58|-1399.03| +2451506|18531|2451585|16100|89845|1818687|2729|25218|53591|764405|4144|44186|50|2|18|4|230|308|94|28.15|83.60|32.60|4794.00|3064.40|2646.10|7858.40|91.93|0.00|2357.52|3064.40|3156.33|5421.92|5513.85|418.30| +2451506|18531|2451513|9199|89845|1818687|2729|25218|53591|764405|4144|44186|52|14|19|5|11|308|69|1.75|3.69|0.03|252.54|2.07|120.75|254.61|0.06|0.00|35.19|2.07|2.13|37.26|37.32|-118.68| +2451506|18531|2451519|9904|89845|1818687|2729|25218|53591|764405|4144|44186|40|19|16|5|134|308|49|43.55|90.58|8.15|4039.07|399.35|2133.95|4438.42|11.98|0.00|1730.68|399.35|411.33|2130.03|2142.01|-1734.60| +2451506|18531|2451615|17278|89845|1818687|2729|25218|53591|764405|4144|44186|19|1|2|3|184|308|76|57.23|121.89|76.79|3427.60|5836.04|4349.48|9263.64|58.36|0.00|3890.44|5836.04|5894.40|9726.48|9784.84|1486.56| +2451506|18531|2451536|14518|89845|1818687|2729|25218|53591|764405|4144|44186|55|13|6|2|11|308|17|72.42|215.81|23.73|3265.36|403.41|1231.14|3668.77|32.27|0.00|880.43|403.41|435.68|1283.84|1316.11|-827.73| +2452639|68395|2452727|12987|20636|55975|4114|38253|84807|917079|1117|12056|31|13|11|2|296|309|74|27.08|43.86|37.71|455.10|2790.54|2003.92|3245.64|27.90|0.00|908.72|2790.54|2818.44|3699.26|3727.16|786.62| +2452639|68395|2452746|1488|20636|55975|4114|38253|84807|917079|1117|12056|7|3|17|5|183|309|19|5.55|16.03|6.73|176.70|127.87|105.45|304.57|7.67|0.00|54.72|127.87|135.54|182.59|190.26|22.42| +2452639|68395|2452707|2328|20636|55975|4114|38253|84807|917079|1117|12056|55|30|12|1|73|309|31|11.75|26.55|10.08|510.57|312.48|364.25|823.05|7.03|78.12|394.94|234.36|241.39|629.30|636.33|-129.89| +2452639|68395|2452707|13485|20636|55975|4114|38253|84807|917079|1117|12056|33|25|16|4|195|309|19|70.96|151.14|111.84|746.70|2124.96|1348.24|2871.66|106.24|0.00|832.77|2124.96|2231.20|2957.73|3063.97|776.72| +2452639|68395|2452752|9465|20636|55975|4114|38253|84807|917079|1117|12056|15|12|11|4|14|309|56|48.69|63.29|7.59|3119.20|425.04|2726.64|3544.24|34.00|0.00|637.84|425.04|459.04|1062.88|1096.88|-2301.60| +2452639|68395|2452717|15945|20636|55975|4114|38253|84807|917079|1117|12056|27|3|17|2|167|309|43|49.06|126.08|86.99|1680.87|3740.57|2109.58|5421.44|299.24|0.00|2331.03|3740.57|4039.81|6071.60|6370.84|1630.99| +2452639|68395|2452750|7827|20636|55975|4114|38253|84807|917079|1117|12056|21|1|11|1|271|309|63|65.64|161.47|143.70|1119.51|9053.10|4135.32|10172.61|633.71|0.00|4374.09|9053.10|9686.81|13427.19|14060.90|4917.78| +2452639|68395|2452686|6528|20636|55975|4114|38253|84807|917079|1117|12056|1|19|16|5|74|309|71|47.65|72.90|8.01|4607.19|568.71|3383.15|5175.90|45.49|0.00|310.27|568.71|614.20|878.98|924.47|-2814.44| +2452639|68395|2452674|8310|20636|55975|4114|38253|84807|917079|1117|12056|48|19|19|5|21|309|16|89.25|190.99|141.33|794.56|2261.28|1428.00|3055.84|113.06|0.00|1100.00|2261.28|2374.34|3361.28|3474.34|833.28| +2452639|68395|2452679|15027|20636|55975|4114|38253|84807|917079|1117|12056|48|30|6|2|168|309|13|51.03|128.08|116.55|149.89|1515.15|663.39|1665.04|45.45|0.00|449.54|1515.15|1560.60|1964.69|2010.14|851.76| +2452639|68395|2452669|11820|20636|55975|4114|38253|84807|917079|1117|12056|54|18|4|4|174|309|27|76.24|171.54|15.43|4214.97|416.61|2058.48|4631.58|24.49|8.33|555.66|408.28|432.77|963.94|988.43|-1650.20| +2452639|68395|2452725|12411|20636|55975|4114|38253|84807|917079|1117|12056|39|25|19|5|91|309|93|24.32|33.56|16.10|1623.78|1497.30|2261.76|3121.08|74.86|0.00|967.20|1497.30|1572.16|2464.50|2539.36|-764.46| +2452639|68395|2452715|11982|20636|55975|4114|38253|84807|917079|1117|12056|24|19|11|4|87|309|70|66.31|126.65|36.72|6295.10|2570.40|4641.70|8865.50|0.00|0.00|4432.40|2570.40|2570.40|7002.80|7002.80|-2071.30| +2452639|68395|2452719|8563|20636|55975|4114|38253|84807|917079|1117|12056|45|24|1|5|159|309|54|8.95|11.90|10.71|64.26|578.34|483.30|642.60|11.56|0.00|186.30|578.34|589.90|764.64|776.20|95.04| +2451039|70036|2451041|17533|59403|822036|5469|2502|43593|935143|448|10281|13|1|12|1|207|310|79|61.98|134.49|59.17|5950.28|4674.43|4896.42|10624.71|19.63|4393.96|636.74|280.47|300.10|917.21|936.84|-4615.95| +2451039|70036|2451148|13927|59403|822036|5469|2502|43593|935143|448|10281|46|1|11|3|248|310|28|1.67|4.20|3.73|13.16|104.44|46.76|117.60|0.00|3.13|4.48|101.31|101.31|105.79|105.79|54.55| +2451039|70036|2451091|1954|59403|822036|5469|2502|43593|935143|448|10281|26|7|4|2|27|310|46|61.48|65.16|59.94|240.12|2757.24|2828.08|2997.36|220.57|0.00|179.40|2757.24|2977.81|2936.64|3157.21|-70.84| +2451039|70036|2451058|1069|59403|822036|5469|2502|43593|935143|448|10281|58|20|12|5|231|310|80|11.79|20.51|4.30|1296.80|344.00|943.20|1640.80|6.32|264.88|409.60|79.12|85.44|488.72|495.04|-864.08| +2451039|70036|2451084|9326|59403|822036|5469|2502|43593|935143|448|10281|52|25|2|3|84|310|2|58.40|74.75|4.48|140.54|8.96|116.80|149.50|0.00|0.00|41.86|8.96|8.96|50.82|50.82|-107.84| +2451039|70036|2451052|17152|59403|822036|5469|2502|43593|935143|448|10281|40|26|20|3|249|310|1|12.86|29.57|5.61|23.96|5.61|12.86|29.57|0.28|0.00|7.09|5.61|5.89|12.70|12.98|-7.25| +2451039|70036|2451118|7658|59403|822036|5469|2502|43593|935143|448|10281|52|14|2|3|27|310|10|21.53|61.57|12.31|492.60|123.10|215.30|615.70|7.38|0.00|61.50|123.10|130.48|184.60|191.98|-92.20| +2451039|70036|2451057|11809|59403|822036|5469|2502|43593|935143|448|10281|7|13|20|3|106|310|73|43.44|106.86|10.68|7021.14|779.64|3171.12|7800.78|62.37|0.00|1871.72|779.64|842.01|2651.36|2713.73|-2391.48| +2451039|70036|2451118|1390|59403|822036|5469|2502|43593|935143|448|10281|46|16|1|1|278|310|37|36.69|96.86|55.21|1541.05|2042.77|1357.53|3583.82|142.99|0.00|644.91|2042.77|2185.76|2687.68|2830.67|685.24| +2451039|70036|2451149|10784|59403|822036|5469|2502|43593|935143|448|10281|37|16|9|4|219|310|99|55.00|100.65|31.20|6875.55|3088.80|5445.00|9964.35|277.99|0.00|4881.69|3088.80|3366.79|7970.49|8248.48|-2356.20| +2451039|70036|2451095|8858|59403|822036|5469|2502|43593|935143|448|10281|31|28|4|3|192|310|76|12.54|21.31|10.65|810.16|809.40|953.04|1619.56|24.28|0.00|793.44|809.40|833.68|1602.84|1627.12|-143.64| +2451039|70036|2451115|13387|59403|822036|5469|2502|43593|935143|448|10281|8|22|19|4|236|310|2|82.18|189.01|143.64|90.74|287.28|164.36|378.02|0.00|0.00|60.48|287.28|287.28|347.76|347.76|122.92| +2451039|70036|2451106|8431|59403|822036|5469|2502|43593|935143|448|10281|8|10|10|4|295|310|36|32.75|59.27|19.55|1429.92|703.80|1179.00|2133.72|28.15|0.00|896.04|703.80|731.95|1599.84|1627.99|-475.20| +2451039|70036|2451152|12926|59403|822036|5469|2502|43593|935143|448|10281|25|28|17|4|261|310|81|69.49|133.42|26.68|8645.94|2161.08|5628.69|10807.02|194.49|0.00|3457.89|2161.08|2355.57|5618.97|5813.46|-3467.61| +2451122|42085|2451218|6625|95320|1064306|492|21078|90324|702779|2836|5422|50|25|20|2|94|311|17|32.89|95.38|18.12|1313.42|308.04|559.13|1621.46|21.56|0.00|324.19|308.04|329.60|632.23|653.79|-251.09| +2451122|42085|2451233|16021|95320|1064306|492|21078|90324|702779|2836|5422|7|22|20|3|276|311|62|56.45|100.48|10.04|5607.28|622.48|3499.90|6229.76|37.34|0.00|1370.20|622.48|659.82|1992.68|2030.02|-2877.42| +2451122|42085|2451172|17552|95320|1064306|492|21078|90324|702779|2836|5422|50|28|1|2|63|311|85|2.57|6.88|1.99|415.65|169.15|218.45|584.80|6.76|0.00|233.75|169.15|175.91|402.90|409.66|-49.30| +2451122|42085|2451236|3400|95320|1064306|492|21078|90324|702779|2836|5422|19|7|8|2|286|311|24|2.01|2.43|1.21|29.28|29.04|48.24|58.32|0.58|0.00|16.32|29.04|29.62|45.36|45.94|-19.20| +2451122|42085|2451209|7849|95320|1064306|492|21078|90324|702779|2836|5422|31|20|6|1|169|311|61|38.28|104.12|85.37|1143.75|5207.57|2335.08|6351.32|52.07|0.00|2413.16|5207.57|5259.64|7620.73|7672.80|2872.49| +2451122|42085|2451142|14422|95320|1064306|492|21078|90324|702779|2836|5422|49|25|18|1|171|311|53|11.65|27.37|5.20|1175.01|275.60|617.45|1450.61|8.26|0.00|144.69|275.60|283.86|420.29|428.55|-341.85| +2451122|42085|2451169|12422|95320|1064306|492|21078|90324|702779|2836|5422|19|16|5|1|207|311|44|29.48|84.01|32.76|2255.00|1441.44|1297.12|3696.44|0.00|0.00|221.76|1441.44|1441.44|1663.20|1663.20|144.32| +2451122|42085|2451178|17876|95320|1064306|492|21078|90324|702779|2836|5422|40|4|18|5|295|311|58|95.69|133.96|88.41|2641.90|5127.78|5550.02|7769.68|307.66|0.00|3340.80|5127.78|5435.44|8468.58|8776.24|-422.24| +2451122|42085|2451145|16207|95320|1064306|492|21078|90324|702779|2836|5422|8|4|4|2|110|311|12|58.91|120.17|109.35|129.84|1312.20|706.92|1442.04|91.85|0.00|86.52|1312.20|1404.05|1398.72|1490.57|605.28| +2451122|42085|2451172|12097|95320|1064306|492|21078|90324|702779|2836|5422|7|25|12|1|100|311|17|68.87|199.72|123.82|1290.30|2104.94|1170.79|3395.24|147.34|0.00|33.83|2104.94|2252.28|2138.77|2286.11|934.15| +2451122|42085|2451220|5365|95320|1064306|492|21078|90324|702779|2836|5422|58|26|11|5|109|311|16|16.94|29.30|9.96|309.44|159.36|271.04|468.80|9.56|0.00|70.24|159.36|168.92|229.60|239.16|-111.68| +2451122|42085|2451189|17486|95320|1064306|492|21078|90324|702779|2836|5422|32|28|3|1|279|311|18|15.66|26.30|16.83|170.46|302.94|281.88|473.40|9.08|0.00|189.36|302.94|312.02|492.30|501.38|21.06| +2451122|42085|2451150|7580|95320|1064306|492|21078|90324|702779|2836|5422|32|10|15|2|206|311|7|4.21|4.21|2.73|10.36|19.11|29.47|29.47|1.33|0.00|6.72|19.11|20.44|25.83|27.16|-10.36| +2451122|42085|2451199|6226|95320|1064306|492|21078|90324|702779|2836|5422|1|20|14|2|170|311|10|97.22|157.49|102.36|551.30|1023.60|972.20|1574.90|51.18|0.00|157.40|1023.60|1074.78|1181.00|1232.18|51.40| +2451122|42085|2451147|12319|95320|1064306|492|21078|90324|702779|2836|5422|13|19|10|3|123|311|64|35.04|73.58|56.65|1083.52|3625.60|2242.56|4709.12|36.25|0.00|611.84|3625.60|3661.85|4237.44|4273.69|1383.04| +2452168|55597|2452261|1967|97411|258930|3649|28368|54456|1355755|1725|3291|35|7|7|3|276|312|17|33.77|43.90|41.26|44.88|701.42|574.09|746.30|56.11|0.00|37.23|701.42|757.53|738.65|794.76|127.33| +2452168|55597|2452237|15747|97411|258930|3649|28368|54456|1355755|1725|3291|53|11|8|2|229|312|9|70.60|78.36|44.66|303.30|401.94|635.40|705.24|28.13|0.00|275.04|401.94|430.07|676.98|705.11|-233.46| +2452168|55597|2452277|7731|97411|258930|3649|28368|54456|1355755|1725|3291|7|29|19|5|4|312|58|54.90|98.27|92.37|342.20|5357.46|3184.20|5699.66|428.59|0.00|626.40|5357.46|5786.05|5983.86|6412.45|2173.26| +2452168|55597|2452208|9515|97411|258930|3649|28368|54456|1355755|1725|3291|3|19|8|4|147|312|26|62.02|127.14|82.64|1157.00|2148.64|1612.52|3305.64|85.94|0.00|793.26|2148.64|2234.58|2941.90|3027.84|536.12| +2452168|55597|2452243|15109|97411|258930|3649|28368|54456|1355755|1725|3291|43|13|2|2|262|312|84|23.86|27.20|17.13|845.88|1438.92|2004.24|2284.80|43.16|0.00|1096.20|1438.92|1482.08|2535.12|2578.28|-565.32| +2452168|55597|2452260|11453|97411|258930|3649|28368|54456|1355755|1725|3291|33|1|18|5|269|312|80|72.45|75.34|0.00|6027.20|0.00|5796.00|6027.20|0.00|0.00|421.60|0.00|0.00|421.60|421.60|-5796.00| +2452168|55597|2452258|14027|97411|258930|3649|28368|54456|1355755|1725|3291|5|7|8|4|90|312|60|85.59|139.51|76.73|3766.80|4603.80|5135.40|8370.60|0.00|0.00|2008.80|4603.80|4603.80|6612.60|6612.60|-531.60| +2452168|55597|2452223|2591|97411|258930|3649|28368|54456|1355755|1725|3291|5|25|10|5|170|312|15|39.71|51.22|14.34|553.20|215.10|595.65|768.30|4.30|0.00|276.45|215.10|219.40|491.55|495.85|-380.55| +2452168|55597|2452189|12001|97411|258930|3649|28368|54456|1355755|1725|3291|21|7|17|5|133|312|93|43.36|85.85|6.00|7426.05|558.00|4032.48|7984.05|11.16|0.00|558.00|558.00|569.16|1116.00|1127.16|-3474.48| +2452168|55597|2452180|2927|97411|258930|3649|28368|54456|1355755|1725|3291|25|9|4|1|78|312|8|58.06|123.08|60.30|502.24|482.40|464.48|984.64|19.29|0.00|344.56|482.40|501.69|826.96|846.25|17.92| +2452168|55597|2452273|10145|97411|258930|3649|28368|54456|1355755|1725|3291|43|1|20|4|245|312|28|46.34|61.16|4.28|1592.64|119.84|1297.52|1712.48|9.58|0.00|650.72|119.84|129.42|770.56|780.14|-1177.68| +2452168|55597|2452287|16461|97411|258930|3649|28368|54456|1355755|1725|3291|13|3|14|4|31|312|77|16.12|18.86|9.61|712.25|739.97|1241.24|1452.22|51.79|0.00|652.96|739.97|791.76|1392.93|1444.72|-501.27| +2452168|55597|2452208|12065|97411|258930|3649|28368|54456|1355755|1725|3291|5|11|2|5|158|312|83|65.20|138.87|40.27|8183.80|3342.41|5411.60|11526.21|66.84|0.00|4609.82|3342.41|3409.25|7952.23|8019.07|-2069.19| +2451315|39089|2451371|2498|40205|591818|3784|38197|76670|1829515|3389|13402|2|25|7|3|249|313|28|82.48|195.47|37.13|4433.52|1039.64|2309.44|5473.16|83.17|0.00|492.52|1039.64|1122.81|1532.16|1615.33|-1269.80| +2451315|39089|2451355|12235|40205|591818|3784|38197|76670|1829515|3389|13402|8|26|6|5|57|313|7|98.98|121.74|108.34|93.80|758.38|692.86|852.18|45.50|0.00|340.83|758.38|803.88|1099.21|1144.71|65.52| +2451315|39089|2451403|4825|40205|591818|3784|38197|76670|1829515|3389|13402|26|14|2|4|133|313|22|87.16|231.84|57.96|3825.36|1275.12|1917.52|5100.48|114.76|0.00|2397.12|1275.12|1389.88|3672.24|3787.00|-642.40| +2451315|39089|2451423|9104|40205|591818|3784|38197|76670|1829515|3389|13402|10|20|16|3|183|313|89|2.42|6.41|2.17|377.36|193.13|215.38|570.49|3.86|0.00|50.73|193.13|196.99|243.86|247.72|-22.25| +2451315|39089|2451325|10726|40205|591818|3784|38197|76670|1829515|3389|13402|14|2|6|2|81|313|69|19.99|42.57|34.05|587.88|2349.45|1379.31|2937.33|140.96|0.00|1468.32|2349.45|2490.41|3817.77|3958.73|970.14| +2451315|39089|2451419|15802|40205|591818|3784|38197|76670|1829515|3389|13402|25|4|16|1|146|313|27|55.14|112.48|73.11|1062.99|1973.97|1488.78|3036.96|0.00|0.00|1457.73|1973.97|1973.97|3431.70|3431.70|485.19| +2451315|39089|2451429|17695|40205|591818|3784|38197|76670|1829515|3389|13402|37|1|16|3|173|313|97|62.65|134.07|22.79|10794.16|2210.63|6077.05|13004.79|88.20|950.57|129.98|1260.06|1348.26|1390.04|1478.24|-4816.99| +2451315|39089|2451361|11738|40205|591818|3784|38197|76670|1829515|3389|13402|20|26|20|3|99|313|81|44.30|85.05|28.06|4616.19|2272.86|3588.30|6889.05|36.36|1818.28|3375.27|454.58|490.94|3829.85|3866.21|-3133.72| +2451315|39089|2451424|6223|40205|591818|3784|38197|76670|1829515|3389|13402|56|28|8|4|250|313|40|7.87|11.01|6.60|176.40|264.00|314.80|440.40|10.56|0.00|35.20|264.00|274.56|299.20|309.76|-50.80| +2451072|4313|2451123|8564|3229|336631|6833|49435|77760|1026888|441|27396|46|25|12|1|10|314|59|64.97|73.41|19.82|3161.81|1169.38|3833.23|4331.19|105.24|0.00|389.40|1169.38|1274.62|1558.78|1664.02|-2663.85| +2451072|4313|2451111|10075|3229|336631|6833|49435|77760|1026888|441|27396|1|20|8|3|115|314|27|21.95|24.58|14.25|278.91|384.75|592.65|663.66|26.93|0.00|212.22|384.75|411.68|596.97|623.90|-207.90| +2451072|4313|2451153|17647|3229|336631|6833|49435|77760|1026888|441|27396|14|22|3|2|121|314|16|1.95|2.45|1.91|8.64|30.56|31.20|39.20|0.00|0.00|19.20|30.56|30.56|49.76|49.76|-0.64| +2451072|4313|2451114|12196|3229|336631|6833|49435|77760|1026888|441|27396|10|13|15|1|167|314|79|95.98|107.49|1.07|8407.18|84.53|7582.42|8491.71|4.22|0.00|3141.83|84.53|88.75|3226.36|3230.58|-7497.89| +2451072|4313|2451166|16885|3229|336631|6833|49435|77760|1026888|441|27396|28|10|10|4|235|314|88|93.93|145.59|53.86|8072.24|4739.68|8265.84|12811.92|284.38|0.00|4868.16|4739.68|5024.06|9607.84|9892.22|-3526.16| +2451072|4313|2451088|5806|3229|336631|6833|49435|77760|1026888|441|27396|46|7|4|1|251|314|92|32.11|91.51|36.60|5051.72|3367.20|2954.12|8418.92|67.34|0.00|3198.84|3367.20|3434.54|6566.04|6633.38|413.08| +2451072|4313|2451181|17438|3229|336631|6833|49435|77760|1026888|441|27396|25|8|7|4|161|314|27|79.23|209.95|6.29|5498.82|169.83|2139.21|5668.65|13.58|0.00|0.00|169.83|183.41|169.83|183.41|-1969.38| +2451072|4313|2451140|7135|3229|336631|6833|49435|77760|1026888|441|27396|2|10|17|1|235|314|22|87.90|221.50|115.18|2339.04|2533.96|1933.80|4873.00|101.35|0.00|1266.98|2533.96|2635.31|3800.94|3902.29|600.16| +2451072|4313|2451092|1142|3229|336631|6833|49435|77760|1026888|441|27396|58|19|20|5|70|314|5|61.59|92.38|73.90|92.40|369.50|307.95|461.90|5.43|188.44|18.45|181.06|186.49|199.51|204.94|-126.89| +2451072|4313|2451153|487|3229|336631|6833|49435|77760|1026888|441|27396|44|7|2|5|21|314|58|8.27|15.71|9.89|337.56|573.62|479.66|911.18|40.15|0.00|91.06|573.62|613.77|664.68|704.83|93.96| +2451072|4313|2451142|2198|3229|336631|6833|49435|77760|1026888|441|27396|16|26|2|2|292|314|66|45.37|94.82|71.11|1564.86|4693.26|2994.42|6258.12|140.79|0.00|2189.88|4693.26|4834.05|6883.14|7023.93|1698.84| +2451072|4313|2451189|16462|3229|336631|6833|49435|77760|1026888|441|27396|49|7|20|4|38|314|35|6.97|18.95|6.63|431.20|232.05|243.95|663.25|0.00|0.00|6.30|232.05|232.05|238.35|238.35|-11.90| +2451072|4313|2451127|16124|3229|336631|6833|49435|77760|1026888|441|27396|14|10|1|1|141|314|73|9.87|28.91|17.63|823.44|1286.99|720.51|2110.43|77.21|0.00|780.37|1286.99|1364.20|2067.36|2144.57|566.48| +2451072|4313|2451073|15619|3229|336631|6833|49435|77760|1026888|441|27396|25|20|2|5|141|314|1|19.71|35.87|19.36|16.51|19.36|19.71|35.87|0.19|0.00|17.93|19.36|19.55|37.29|37.48|-0.35| +2451072|4313|2451119|10975|3229|336631|6833|49435|77760|1026888|441|27396|31|16|2|5|101|314|90|76.93|146.16|43.84|9208.80|3945.60|6923.70|13154.40|0.00|2722.46|5919.30|1223.14|1223.14|7142.44|7142.44|-5700.56| +2451072|4313|2451157|1030|3229|336631|6833|49435|77760|1026888|441|27396|26|22|13|1|269|314|40|9.15|19.58|5.28|572.00|211.20|366.00|783.20|10.56|0.00|109.60|211.20|221.76|320.80|331.36|-154.80| +2452559|74807|2452624|17748|18243|80632|7021|11410|91626|212206|3777|18966|36|3|6|5|44|315|40|1.24|2.34|1.89|18.00|75.60|49.60|93.60|6.04|0.00|31.60|75.60|81.64|107.20|113.24|26.00| +2452559|74807|2452569|15865|18243|80632|7021|11410|91626|212206|3777|18966|51|15|20|5|97|315|85|44.72|101.06|66.69|2921.45|5668.65|3801.20|8590.10|0.00|0.00|257.55|5668.65|5668.65|5926.20|5926.20|1867.45| +2452559|74807|2452669|13521|18243|80632|7021|11410|91626|212206|3777|18966|13|9|13|1|269|315|71|90.73|187.81|150.24|2667.47|10667.04|6441.83|13334.51|213.34|0.00|4666.83|10667.04|10880.38|15333.87|15547.21|4225.21| +2452559|74807|2452607|16800|18243|80632|7021|11410|91626|212206|3777|18966|19|24|17|4|281|315|42|9.35|19.72|8.87|455.70|372.54|392.70|828.24|22.35|0.00|372.54|372.54|394.89|745.08|767.43|-20.16| +2452559|74807|2452648|2011|18243|80632|7021|11410|91626|212206|3777|18966|21|24|13|2|248|315|68|80.84|220.69|39.72|12305.96|2700.96|5497.12|15006.92|0.00|108.03|3451.00|2592.93|2592.93|6043.93|6043.93|-2904.19| +2452559|74807|2452610|11209|18243|80632|7021|11410|91626|212206|3777|18966|54|15|2|3|184|315|90|37.70|46.74|32.71|1262.70|2943.90|3393.00|4206.60|89.49|1825.21|1639.80|1118.69|1208.18|2758.49|2847.98|-2274.31| +2452559|74807|2452671|9081|18243|80632|7021|11410|91626|212206|3777|18966|48|9|13|4|292|315|65|3.35|3.38|0.91|160.55|59.15|217.75|219.70|1.77|0.00|98.80|59.15|60.92|157.95|159.72|-158.60| +2452559|74807|2452674|13674|18243|80632|7021|11410|91626|212206|3777|18966|9|24|19|1|273|315|58|4.68|12.35|9.38|172.26|544.04|271.44|716.30|5.87|446.11|186.18|97.93|103.80|284.11|289.98|-173.51| +2452559|74807|2452580|10935|18243|80632|7021|11410|91626|212206|3777|18966|3|19|5|3|221|315|32|54.80|127.13|111.87|488.32|3579.84|1753.60|4068.16|286.38|0.00|1098.24|3579.84|3866.22|4678.08|4964.46|1826.24| +2452559|74807|2452669|14154|18243|80632|7021|11410|91626|212206|3777|18966|27|1|11|1|241|315|20|38.55|89.82|57.48|646.80|1149.60|771.00|1796.40|0.00|0.00|646.60|1149.60|1149.60|1796.20|1796.20|378.60| +2452559|74807|2452609|10476|18243|80632|7021|11410|91626|212206|3777|18966|12|30|4|1|5|315|65|74.22|106.13|48.81|3725.80|3172.65|4824.30|6898.45|95.17|0.00|1448.20|3172.65|3267.82|4620.85|4716.02|-1651.65| +2452559|74807|2452654|16068|18243|80632|7021|11410|91626|212206|3777|18966|45|19|7|1|38|315|80|22.05|22.71|14.98|618.40|1198.40|1764.00|1816.80|59.92|0.00|689.60|1198.40|1258.32|1888.00|1947.92|-565.60| +2452559|74807|2452612|5244|18243|80632|7021|11410|91626|212206|3777|18966|9|18|6|5|277|315|16|60.08|135.78|61.10|1194.88|977.60|961.28|2172.48|68.43|0.00|564.80|977.60|1046.03|1542.40|1610.83|16.32| +2452559|74807|2452670|5277|18243|80632|7021|11410|91626|212206|3777|18966|3|25|4|5|233|315|74|64.68|151.35|59.02|6832.42|4367.48|4786.32|11199.90|43.67|0.00|447.70|4367.48|4411.15|4815.18|4858.85|-418.84| +2452559|74807|2452623|1243|18243|80632|7021|11410|91626|212206|3777|18966|49|1|4|5|206|315|26|15.89|22.56|14.43|211.38|375.18|413.14|586.56|30.01|0.00|29.12|375.18|405.19|404.30|434.31|-37.96| +2451536|85015|2451550|5140|11695|1856574|4300|18779|69874|1028372|1510|33989|49|10|14|5|16|316|68|45.34|125.59|102.98|1537.48|7002.64|3083.12|8540.12|420.15|0.00|2049.52|7002.64|7422.79|9052.16|9472.31|3919.52| +2451536|85015|2451624|10756|11695|1856574|4300|18779|69874|1028372|1510|33989|34|14|8|3|128|316|12|3.86|10.19|6.62|42.84|79.44|46.32|122.28|2.38|0.00|52.56|79.44|81.82|132.00|134.38|33.12| +2451536|85015|2451573|7465|11695|1856574|4300|18779|69874|1028372|1510|33989|49|7|16|5|137|316|95|63.69|126.74|73.50|5057.80|6982.50|6050.55|12040.30|36.30|6074.77|0.00|907.73|944.03|907.73|944.03|-5142.82| +2451536|85015|2451582|13582|11695|1856574|4300|18779|69874|1028372|1510|33989|7|4|9|3|187|316|38|51.69|95.62|62.15|1271.86|2361.70|1964.22|3633.56|87.38|614.04|908.20|1747.66|1835.04|2655.86|2743.24|-216.56| +2451536|85015|2451585|9103|11695|1856574|4300|18779|69874|1028372|1510|33989|13|7|4|4|101|316|37|21.00|24.36|5.60|694.12|207.20|777.00|901.32|16.57|0.00|243.09|207.20|223.77|450.29|466.86|-569.80| +2451536|85015|2451540|10628|11695|1856574|4300|18779|69874|1028372|1510|33989|44|10|19|2|8|316|95|76.16|111.19|1.11|10457.60|105.45|7235.20|10563.05|0.00|0.00|950.00|105.45|105.45|1055.45|1055.45|-7129.75| +2451536|85015|2451617|5032|11695|1856574|4300|18779|69874|1028372|1510|33989|43|2|9|4|249|316|43|18.82|51.94|27.52|1050.06|1183.36|809.26|2233.42|35.50|0.00|960.19|1183.36|1218.86|2143.55|2179.05|374.10| +2451536|85015|2451540|5185|11695|1856574|4300|18779|69874|1028372|1510|33989|10|2|5|3|34|316|90|82.17|93.67|39.34|4889.70|3540.60|7395.30|8430.30|283.24|0.00|1854.00|3540.60|3823.84|5394.60|5677.84|-3854.70| +2451536|85015|2451578|1165|11695|1856574|4300|18779|69874|1028372|1510|33989|7|2|7|5|10|316|24|81.17|128.24|66.68|1477.44|1600.32|1948.08|3077.76|96.01|0.00|1446.48|1600.32|1696.33|3046.80|3142.81|-347.76| +2451160|14782|2451175|5590|70824|629267|510|34953|81437|589926|2729|20772|50|7|19|2|65|317|82|78.05|193.56|25.16|13808.80|2063.12|6400.10|15871.92|144.41|0.00|2221.38|2063.12|2207.53|4284.50|4428.91|-4336.98| +2451160|14782|2451176|5902|70824|629267|510|34953|81437|589926|2729|20772|37|20|13|5|77|317|76|38.84|45.05|31.53|1027.52|2396.28|2951.84|3423.80|119.81|0.00|1231.96|2396.28|2516.09|3628.24|3748.05|-555.56| +2451160|14782|2451200|1246|70824|629267|510|34953|81437|589926|2729|20772|16|26|19|4|149|317|3|80.30|122.05|25.63|289.26|76.89|240.90|366.15|1.53|0.00|84.21|76.89|78.42|161.10|162.63|-164.01| +2451160|14782|2451250|7279|70824|629267|510|34953|81437|589926|2729|20772|43|8|15|5|268|317|20|86.42|190.98|101.21|1795.40|2024.20|1728.40|3819.60|0.00|0.00|878.40|2024.20|2024.20|2902.60|2902.60|295.80| +2451160|14782|2451267|10291|70824|629267|510|34953|81437|589926|2729|20772|37|25|7|4|48|317|96|70.92|190.77|3.81|17948.16|365.76|6808.32|18313.92|29.26|0.00|7874.88|365.76|395.02|8240.64|8269.90|-6442.56| +2451160|14782|2451179|15974|70824|629267|510|34953|81437|589926|2729|20772|58|4|19|5|298|317|7|82.07|121.46|74.09|331.59|518.63|574.49|850.22|15.55|0.00|382.55|518.63|534.18|901.18|916.73|-55.86| +2451160|14782|2451165|9694|70824|629267|510|34953|81437|589926|2729|20772|55|13|6|5|154|317|39|66.37|136.05|19.04|4563.39|742.56|2588.43|5305.95|15.07|527.21|265.20|215.35|230.42|480.55|495.62|-2373.08| +2451160|14782|2451264|16078|70824|629267|510|34953|81437|589926|2729|20772|31|19|17|1|8|317|82|79.48|216.18|99.44|9572.68|8154.08|6517.36|17726.76|81.54|0.00|4608.40|8154.08|8235.62|12762.48|12844.02|1636.72| +2451160|14782|2451186|433|70824|629267|510|34953|81437|589926|2729|20772|10|13|18|4|190|317|55|12.48|27.08|21.39|312.95|1176.45|686.40|1489.40|94.11|0.00|491.15|1176.45|1270.56|1667.60|1761.71|490.05| +2452201|58149|2452254|12731|92011|1763028|5692|3956|88524|1604185|2301|47765|3|3|5|5|205|318|93|68.86|175.59|40.38|12574.53|3755.34|6403.98|16329.87|37.55|0.00|2776.05|3755.34|3792.89|6531.39|6568.94|-2648.64| +2452201|58149|2452231|12903|92011|1763028|5692|3956|88524|1604185|2301|47765|7|5|12|1|247|318|88|21.53|62.43|15.60|4121.04|1372.80|1894.64|5493.84|54.91|0.00|1428.24|1372.80|1427.71|2801.04|2855.95|-521.84| +2452201|58149|2452230|4215|92011|1763028|5692|3956|88524|1604185|2301|47765|21|29|9|1|260|318|1|92.56|231.40|43.96|187.44|43.96|92.56|231.40|0.87|0.00|85.61|43.96|44.83|129.57|130.44|-48.60| +2452201|58149|2452260|183|92011|1763028|5692|3956|88524|1604185|2301|47765|21|15|12|4|103|318|43|64.38|83.05|42.35|1750.10|1821.05|2768.34|3571.15|36.42|0.00|1749.67|1821.05|1857.47|3570.72|3607.14|-947.29| +2452201|58149|2452270|13207|92011|1763028|5692|3956|88524|1604185|2301|47765|3|25|11|4|247|318|60|49.57|74.35|16.35|3480.00|981.00|2974.20|4461.00|34.82|284.49|1516.20|696.51|731.33|2212.71|2247.53|-2277.69| +2452201|58149|2452287|17397|92011|1763028|5692|3956|88524|1604185|2301|47765|59|29|3|1|99|318|23|51.38|77.58|50.42|624.68|1159.66|1181.74|1784.34|92.77|0.00|570.86|1159.66|1252.43|1730.52|1823.29|-22.08| +2452201|58149|2452210|11947|92011|1763028|5692|3956|88524|1604185|2301|47765|45|17|18|2|253|318|93|23.58|51.40|37.52|1290.84|3489.36|2192.94|4780.20|209.36|0.00|1050.90|3489.36|3698.72|4540.26|4749.62|1296.42| +2452201|58149|2452303|1689|92011|1763028|5692|3956|88524|1604185|2301|47765|25|21|6|4|102|318|100|36.43|70.30|68.19|211.00|6819.00|3643.00|7030.00|613.71|0.00|843.00|6819.00|7432.71|7662.00|8275.71|3176.00| +2452201|58149|2452210|899|92011|1763028|5692|3956|88524|1604185|2301|47765|27|5|15|4|196|318|72|94.34|241.51|115.92|9042.48|8346.24|6792.48|17388.72|250.38|0.00|4694.40|8346.24|8596.62|13040.64|13291.02|1553.76| +2452201|58149|2452271|8917|92011|1763028|5692|3956|88524|1604185|2301|47765|15|11|17|3|16|318|99|93.80|110.68|98.50|1205.82|9751.50|9286.20|10957.32|87.76|8288.77|4711.41|1462.73|1550.49|6174.14|6261.90|-7823.47| +2452201|58149|2452244|9047|92011|1763028|5692|3956|88524|1604185|2301|47765|51|23|5|3|171|318|86|84.88|206.25|88.68|10111.02|7626.48|7299.68|17737.50|173.88|1830.35|5497.98|5796.13|5970.01|11294.11|11467.99|-1503.55| +2452201|58149|2452257|13151|92011|1763028|5692|3956|88524|1604185|2301|47765|15|17|13|4|185|318|89|4.13|8.83|1.41|660.38|125.49|367.57|785.87|5.01|0.00|93.45|125.49|130.50|218.94|223.95|-242.08| +2452201|58149|2452294|6409|92011|1763028|5692|3956|88524|1604185|2301|47765|3|25|17|3|99|318|47|32.73|36.33|23.97|580.92|1126.59|1538.31|1707.51|90.12|0.00|836.60|1126.59|1216.71|1963.19|2053.31|-411.72| +2452201|58149|2452266|6265|92011|1763028|5692|3956|88524|1604185|2301|47765|29|25|1|1|104|318|3|64.73|77.67|53.59|72.24|160.77|194.19|233.01|6.64|65.91|58.23|94.86|101.50|153.09|159.73|-99.33| +2452252|82388|2452351|9571|73503|1502890|4630|8017|12089|1097919|3331|31918|39|1|19|5|71|319|74|48.39|143.71|10.05|9890.84|743.70|3580.86|10634.54|52.05|0.00|5316.90|743.70|795.75|6060.60|6112.65|-2837.16| +2452252|82388|2452320|1337|73503|1502890|4630|8017|12089|1097919|3331|31918|13|23|7|2|200|319|26|58.05|66.17|44.99|550.68|1169.74|1509.30|1720.42|35.09|0.00|584.74|1169.74|1204.83|1754.48|1789.57|-339.56| +2452252|82388|2452281|2215|73503|1502890|4630|8017|12089|1097919|3331|31918|51|25|7|4|191|319|99|16.05|23.75|0.95|2257.20|94.05|1588.95|2351.25|0.94|0.00|940.50|94.05|94.99|1034.55|1035.49|-1494.90| +2452252|82388|2452301|11547|73503|1502890|4630|8017|12089|1097919|3331|31918|3|11|7|3|9|319|17|31.01|46.51|43.71|47.60|743.07|527.17|790.67|59.44|0.00|165.92|743.07|802.51|908.99|968.43|215.90| +2452252|82388|2452265|1377|73503|1502890|4630|8017|12089|1097919|3331|31918|39|9|20|5|200|319|26|8.62|10.17|4.27|153.40|111.02|224.12|264.42|5.55|0.00|31.72|111.02|116.57|142.74|148.29|-113.10| +2452252|82388|2452255|16673|73503|1502890|4630|8017|12089|1097919|3331|31918|33|29|16|2|18|319|48|22.16|27.92|1.95|1246.56|93.60|1063.68|1340.16|6.55|0.00|80.16|93.60|100.15|173.76|180.31|-970.08| +2452252|82388|2452308|9977|73503|1502890|4630|8017|12089|1097919|3331|31918|53|1|12|1|80|319|57|57.86|109.35|99.50|561.45|5671.50|3298.02|6232.95|510.43|0.00|1433.55|5671.50|6181.93|7105.05|7615.48|2373.48| +2452252|82388|2452356|8073|73503|1502890|4630|8017|12089|1097919|3331|31918|1|3|10|1|132|319|82|86.77|149.24|0.00|12237.68|0.00|7115.14|12237.68|0.00|0.00|4282.86|0.00|0.00|4282.86|4282.86|-7115.14| +2452348|68739|2452367|16458|38697|523613|4882|951|48589|940608|5605|39721|12|9|17|4|191|320|47|17.29|25.76|1.54|1138.34|72.38|812.63|1210.72|0.00|0.00|108.57|72.38|72.38|180.95|180.95|-740.25| +2452348|68739|2452446|4011|38697|523613|4882|951|48589|940608|5605|39721|42|15|15|2|149|320|53|76.65|219.21|116.18|5460.59|6157.54|4062.45|11618.13|184.72|0.00|2090.85|6157.54|6342.26|8248.39|8433.11|2095.09| +2452348|68739|2452410|7873|38697|523613|4882|951|48589|940608|5605|39721|45|27|15|3|264|320|57|52.23|83.56|76.03|429.21|4333.71|2977.11|4762.92|86.67|0.00|619.02|4333.71|4420.38|4952.73|5039.40|1356.60| +2452348|68739|2452444|6159|38697|523613|4882|951|48589|940608|5605|39721|7|7|11|1|251|320|13|34.60|45.67|9.13|475.02|118.69|449.80|593.71|7.12|0.00|0.00|118.69|125.81|118.69|125.81|-331.11| +2452348|68739|2452450|6633|38697|523613|4882|951|48589|940608|5605|39721|18|18|12|5|228|320|57|56.38|79.49|0.79|4485.90|45.03|3213.66|4530.93|4.05|0.00|1132.59|45.03|49.08|1177.62|1181.67|-3168.63| +2452348|68739|2452444|15021|38697|523613|4882|951|48589|940608|5605|39721|19|24|14|1|108|320|74|2.97|5.76|3.11|196.10|230.14|219.78|426.24|14.29|71.34|187.22|158.80|173.09|346.02|360.31|-60.98| +2452348|68739|2452460|403|38697|523613|4882|951|48589|940608|5605|39721|55|12|14|2|210|320|46|75.55|152.61|67.14|3931.62|3088.44|3475.30|7020.06|185.30|0.00|2807.84|3088.44|3273.74|5896.28|6081.58|-386.86| +2452348|68739|2452450|11911|38697|523613|4882|951|48589|940608|5605|39721|9|27|3|2|41|320|5|53.03|127.80|106.07|108.65|530.35|265.15|639.00|26.51|0.00|95.85|530.35|556.86|626.20|652.71|265.20| +2452348|68739|2452412|17773|38697|523613|4882|951|48589|940608|5605|39721|54|24|17|2|27|320|17|93.95|225.48|124.01|1724.99|2108.17|1597.15|3833.16|126.49|0.00|229.84|2108.17|2234.66|2338.01|2464.50|511.02| +2452348|68739|2452430|17964|38697|523613|4882|951|48589|940608|5605|39721|36|25|8|3|291|320|43|28.00|30.24|16.02|611.46|688.86|1204.00|1300.32|12.81|475.31|363.78|213.55|226.36|577.33|590.14|-990.45| +2452348|68739|2452467|11304|38697|523613|4882|951|48589|940608|5605|39721|18|7|19|4|79|320|53|74.49|217.51|89.17|6802.02|4726.01|3947.97|11528.03|0.00|0.00|1959.41|4726.01|4726.01|6685.42|6685.42|778.04| +2452348|68739|2452433|16452|38697|523613|4882|951|48589|940608|5605|39721|6|27|8|4|123|320|52|94.45|227.62|172.99|2840.76|8995.48|4911.40|11836.24|449.77|0.00|5089.24|8995.48|9445.25|14084.72|14534.49|4084.08| +2452348|68739|2452416|8733|38697|523613|4882|951|48589|940608|5605|39721|25|25|10|5|27|320|16|73.61|108.94|82.79|418.40|1324.64|1177.76|1743.04|105.97|0.00|278.88|1324.64|1430.61|1603.52|1709.49|146.88| +2452348|68739|2452384|13555|38697|523613|4882|951|48589|940608|5605|39721|37|24|5|3|254|320|83|58.76|76.38|7.63|5706.25|633.29|4877.08|6339.54|2.28|595.29|2408.66|38.00|40.28|2446.66|2448.94|-4839.08| +2452348|68739|2452355|4428|38697|523613|4882|951|48589|940608|5605|39721|3|7|13|5|114|320|26|41.95|54.53|45.25|241.28|1176.50|1090.70|1417.78|9.41|705.90|155.74|470.60|480.01|626.34|635.75|-620.10| +2452348|68739|2452421|12501|38697|523613|4882|951|48589|940608|5605|39721|12|24|18|4|300|320|6|72.17|184.75|103.46|487.74|620.76|433.02|1108.50|37.24|0.00|554.22|620.76|658.00|1174.98|1212.22|187.74| +2450991|25906|2451069|9421|22392|360736|2727|15071|5127|1640886|6950|27741|37|20|3|2|174|321|84|3.49|7.46|4.17|276.36|350.28|293.16|626.64|10.50|0.00|150.36|350.28|360.78|500.64|511.14|57.12| +2450991|25906|2451022|15385|22392|360736|2727|15071|5127|1640886|6950|27741|58|20|15|1|127|321|7|83.70|250.26|75.07|1226.33|525.49|585.90|1751.82|10.50|0.00|210.21|525.49|535.99|735.70|746.20|-60.41| +2450991|25906|2451086|9596|22392|360736|2727|15071|5127|1640886|6950|27741|50|4|3|5|220|321|74|86.42|152.96|53.53|7357.82|3961.22|6395.08|11319.04|0.00|713.01|4414.10|3248.21|3248.21|7662.31|7662.31|-3146.87| +2450991|25906|2451082|370|22392|360736|2727|15071|5127|1640886|6950|27741|38|20|19|2|208|321|39|43.28|55.39|45.41|389.22|1770.99|1687.92|2160.21|35.41|0.00|0.00|1770.99|1806.40|1770.99|1806.40|83.07| +2450991|25906|2451007|6662|22392|360736|2727|15071|5127|1640886|6950|27741|52|20|5|4|180|321|89|67.59|113.55|21.57|8186.22|1919.73|6015.51|10105.95|13.82|1228.62|808.12|691.11|704.93|1499.23|1513.05|-5324.40| +2450991|25906|2451081|6434|22392|360736|2727|15071|5127|1640886|6950|27741|56|19|1|5|216|321|1|27.64|52.23|41.78|10.45|41.78|27.64|52.23|2.50|0.00|17.23|41.78|44.28|59.01|61.51|14.14| +2450991|25906|2451013|5290|22392|360736|2727|15071|5127|1640886|6950|27741|46|28|19|5|154|321|24|93.54|272.20|195.98|1829.28|4703.52|2244.96|6532.80|0.00|0.00|2808.96|4703.52|4703.52|7512.48|7512.48|2458.56| +2450991|25906|2451009|5635|22392|360736|2727|15071|5127|1640886|6950|27741|40|25|15|5|102|321|19|52.41|53.98|42.64|215.46|810.16|995.79|1025.62|32.40|0.00|379.43|810.16|842.56|1189.59|1221.99|-185.63| +2450991|25906|2450999|3430|22392|360736|2727|15071|5127|1640886|6950|27741|43|13|2|2|172|321|76|91.72|96.30|85.70|805.60|6513.20|6970.72|7318.80|455.92|0.00|3220.12|6513.20|6969.12|9733.32|10189.24|-457.52| +2450991|25906|2451103|16603|22392|360736|2727|15071|5127|1640886|6950|27741|13|10|18|2|79|321|65|85.80|229.08|187.84|2680.60|12209.60|5577.00|14890.20|488.38|0.00|4615.65|12209.60|12697.98|16825.25|17313.63|6632.60| +2450991|25906|2451027|15124|22392|360736|2727|15071|5127|1640886|6950|27741|8|1|7|1|282|321|54|6.04|7.12|7.12|0.00|384.48|326.16|384.48|34.60|0.00|172.80|384.48|419.08|557.28|591.88|58.32| +2450991|25906|2451064|8245|22392|360736|2727|15071|5127|1640886|6950|27741|55|26|7|3|256|321|17|1.11|2.66|1.27|23.63|21.59|18.87|45.22|0.69|4.31|14.79|17.28|17.97|32.07|32.76|-1.59| +2450991|25906|2451024|12368|22392|360736|2727|15071|5127|1640886|6950|27741|55|16|7|3|86|321|85|17.18|20.27|9.12|947.75|775.20|1460.30|1722.95|38.76|0.00|585.65|775.20|813.96|1360.85|1399.61|-685.10| +2450991|25906|2451094|1684|22392|360736|2727|15071|5127|1640886|6950|27741|22|25|11|4|270|321|72|69.31|153.86|126.16|1994.40|9083.52|4990.32|11077.92|94.46|7902.66|4873.68|1180.86|1275.32|6054.54|6149.00|-3809.46| +2450991|25906|2451026|11497|22392|360736|2727|15071|5127|1640886|6950|27741|40|22|6|2|68|321|3|90.26|102.89|79.22|71.01|237.66|270.78|308.67|14.25|0.00|12.33|237.66|251.91|249.99|264.24|-33.12| +2450991|25906|2451058|12511|22392|360736|2727|15071|5127|1640886|6950|27741|31|25|3|5|297|321|73|92.38|99.77|9.97|6555.40|727.81|6743.74|7283.21|36.39|0.00|3422.97|727.81|764.20|4150.78|4187.17|-6015.93| +2450940|60265|2450961|9301|71945|1280258|5724|49237|23277|1631412|4929|46842|10|14|3|5|30|322|11|10.39|16.41|1.47|164.34|16.17|114.29|180.51|0.48|0.00|88.44|16.17|16.65|104.61|105.09|-98.12| +2450940|60265|2451037|15676|71945|1280258|5724|49237|23277|1631412|4929|46842|1|2|3|4|155|322|76|23.38|54.70|22.97|2411.48|1745.72|1776.88|4157.20|67.55|995.06|872.48|750.66|818.21|1623.14|1690.69|-1026.22| +2450940|60265|2451044|7741|71945|1280258|5724|49237|23277|1631412|4929|46842|7|28|16|2|258|322|32|19.27|36.22|14.48|695.68|463.36|616.64|1159.04|37.06|0.00|162.24|463.36|500.42|625.60|662.66|-153.28| +2450940|60265|2451039|4876|71945|1280258|5724|49237|23277|1631412|4929|46842|19|10|15|4|190|322|13|64.90|87.61|49.93|489.84|649.09|843.70|1138.93|11.35|486.81|11.31|162.28|173.63|173.59|184.94|-681.42| +2450940|60265|2451038|568|71945|1280258|5724|49237|23277|1631412|4929|46842|31|16|5|3|244|322|13|36.61|77.24|11.58|853.58|150.54|475.93|1004.12|1.50|0.00|210.86|150.54|152.04|361.40|362.90|-325.39| +2450940|60265|2451052|8036|71945|1280258|5724|49237|23277|1631412|4929|46842|44|25|16|4|239|322|64|89.51|129.78|123.29|415.36|7890.56|5728.64|8305.92|394.52|0.00|3072.64|7890.56|8285.08|10963.20|11357.72|2161.92| +2450940|60265|2451059|17953|71945|1280258|5724|49237|23277|1631412|4929|46842|40|8|12|1|243|322|20|75.49|89.83|9.88|1599.00|197.60|1509.80|1796.60|3.95|0.00|197.60|197.60|201.55|395.20|399.15|-1312.20| +2450940|60265|2450983|16675|71945|1280258|5724|49237|23277|1631412|4929|46842|50|13|8|4|40|322|63|81.28|111.35|38.97|4559.94|2455.11|5120.64|7015.05|196.40|0.00|2174.13|2455.11|2651.51|4629.24|4825.64|-2665.53| +2450940|60265|2451012|1202|71945|1280258|5724|49237|23277|1631412|4929|46842|14|16|10|4|65|322|53|55.16|80.53|35.43|2390.30|1877.79|2923.48|4268.09|18.77|0.00|1664.20|1877.79|1896.56|3541.99|3560.76|-1045.69| +2450940|60265|2451037|2563|71945|1280258|5724|49237|23277|1631412|4929|46842|40|22|3|4|285|322|21|49.44|71.19|53.39|373.80|1121.19|1038.24|1494.99|67.27|0.00|224.07|1121.19|1188.46|1345.26|1412.53|82.95| +2450940|60265|2451043|10921|71945|1280258|5724|49237|23277|1631412|4929|46842|19|10|2|3|204|322|74|44.19|84.40|32.07|3872.42|2373.18|3270.06|6245.60|71.19|0.00|1810.78|2373.18|2444.37|4183.96|4255.15|-896.88| +2450940|60265|2450965|9883|71945|1280258|5724|49237|23277|1631412|4929|46842|44|8|17|1|108|322|94|60.54|90.20|25.25|6105.30|2373.50|5690.76|8478.80|22.31|142.41|4153.86|2231.09|2253.40|6384.95|6407.26|-3459.67| +2451424|1257|2451502|12842|60919|732917|1362|2960|82395|9119|2248|31446|25|10|16|4|179|323|8|99.82|272.50|79.02|1547.84|632.16|798.56|2180.00|20.48|290.79|261.60|341.37|361.85|602.97|623.45|-457.19| +2451424|1257|2451507|10864|60919|732917|1362|2960|82395|9119|2248|31446|14|16|17|3|19|323|8|39.25|80.85|52.55|226.40|420.40|314.00|646.80|10.34|248.03|258.72|172.37|182.71|431.09|441.43|-141.63| +2451424|1257|2451515|16762|60919|732917|1362|2960|82395|9119|2248|31446|14|26|10|1|212|323|11|20.41|40.20|28.94|123.86|318.34|224.51|442.20|25.46|0.00|17.60|318.34|343.80|335.94|361.40|93.83| +2451424|1257|2451493|7417|60919|732917|1362|2960|82395|9119|2248|31446|28|14|4|2|100|323|15|14.89|18.91|17.58|19.95|263.70|223.35|283.65|21.09|0.00|14.10|263.70|284.79|277.80|298.89|40.35| +2451424|1257|2451445|7189|60919|732917|1362|2960|82395|9119|2248|31446|32|20|12|3|194|323|93|92.56|192.52|155.94|3401.94|14502.42|8608.08|17904.36|194.33|4785.79|7340.49|9716.63|9910.96|17057.12|17251.45|1108.55| +2451424|1257|2451508|7822|60919|732917|1362|2960|82395|9119|2248|31446|31|16|5|3|214|323|8|35.73|101.47|79.14|178.64|633.12|285.84|811.76|49.13|18.99|16.16|614.13|663.26|630.29|679.42|328.29| +2451424|1257|2451443|16657|60919|732917|1362|2960|82395|9119|2248|31446|25|20|9|2|192|323|33|44.21|58.35|45.51|423.72|1501.83|1458.93|1925.55|45.05|0.00|827.97|1501.83|1546.88|2329.80|2374.85|42.90| +2451424|1257|2451488|2800|60919|732917|1362|2960|82395|9119|2248|31446|55|26|10|1|32|323|60|17.02|31.82|24.81|420.60|1488.60|1021.20|1909.20|44.65|0.00|897.00|1488.60|1533.25|2385.60|2430.25|467.40| +2451424|1257|2451508|940|60919|732917|1362|2960|82395|9119|2248|31446|26|1|8|2|149|323|17|99.13|226.01|79.10|2497.47|1344.70|1685.21|3842.17|1.07|1290.91|384.20|53.79|54.86|437.99|439.06|-1631.42| +2451424|1257|2451525|8330|60919|732917|1362|2960|82395|9119|2248|31446|38|2|17|4|211|323|65|69.73|181.29|181.29|0.00|11783.85|4532.45|11783.85|471.35|0.00|4241.90|11783.85|12255.20|16025.75|16497.10|7251.40| +2451424|1257|2451530|15656|60919|732917|1362|2960|82395|9119|2248|31446|40|28|9|5|94|323|48|83.62|245.00|107.80|6585.60|5174.40|4013.76|11760.00|362.20|0.00|4821.60|5174.40|5536.60|9996.00|10358.20|1160.64| +2451424|1257|2451484|5722|60919|732917|1362|2960|82395|9119|2248|31446|44|7|2|5|208|323|40|47.39|118.94|45.19|2950.00|1807.60|1895.60|4757.60|126.53|0.00|523.20|1807.60|1934.13|2330.80|2457.33|-88.00| +2451424|1257|2451491|13924|60919|732917|1362|2960|82395|9119|2248|31446|40|1|4|2|250|323|26|25.20|60.73|10.93|1294.80|284.18|655.20|1578.98|5.68|0.00|489.32|284.18|289.86|773.50|779.18|-371.02| +2451424|1257|2451480|10430|60919|732917|1362|2960|82395|9119|2248|31446|37|4|14|5|96|323|45|51.69|53.75|36.55|774.00|1644.75|2326.05|2418.75|148.02|0.00|435.15|1644.75|1792.77|2079.90|2227.92|-681.30| +2451424|1257|2451462|11536|60919|732917|1362|2960|82395|9119|2248|31446|13|28|13|1|182|323|22|59.91|132.40|87.38|990.44|1922.36|1318.02|2912.80|115.34|0.00|902.88|1922.36|2037.70|2825.24|2940.58|604.34| +2451424|1257|2451490|7180|60919|732917|1362|2960|82395|9119|2248|31446|34|10|11|4|134|323|23|85.26|246.40|167.55|1813.55|3853.65|1960.98|5667.20|77.07|0.00|793.27|3853.65|3930.72|4646.92|4723.99|1892.67| +2451113|42669|2451215|4|25696|603031|1166|22854|92685|1395580|4812|46947|44|14|6|4|129|324|6|13.85|37.81|33.27|27.24|199.62|83.10|226.86|1.31|155.70|104.34|43.92|45.23|148.26|149.57|-39.18| +2451113|42669|2451179|3415|25696|603031|1166|22854|92685|1395580|4812|46947|52|19|6|2|208|324|12|68.90|174.31|141.19|397.44|1694.28|826.80|2091.72|67.77|0.00|376.44|1694.28|1762.05|2070.72|2138.49|867.48| +2451113|42669|2451222|14794|25696|603031|1166|22854|92685|1395580|4812|46947|22|28|18|1|24|324|64|83.14|221.15|88.46|8492.16|5661.44|5320.96|14153.60|509.52|0.00|1556.48|5661.44|6170.96|7217.92|7727.44|340.48| +2451113|42669|2451178|9679|25696|603031|1166|22854|92685|1395580|4812|46947|20|7|7|5|283|324|47|55.52|153.79|53.82|4698.59|2529.54|2609.44|7228.13|24.03|126.47|2963.35|2403.07|2427.10|5366.42|5390.45|-206.37| +2451113|42669|2451221|16244|25696|603031|1166|22854|92685|1395580|4812|46947|38|25|13|1|36|324|79|49.85|109.17|9.82|7848.65|775.78|3938.15|8624.43|26.53|480.98|2673.36|294.80|321.33|2968.16|2994.69|-3643.35| +2451113|42669|2451165|8552|25696|603031|1166|22854|92685|1395580|4812|46947|58|19|7|2|47|324|48|49.36|55.77|45.73|481.92|2195.04|2369.28|2676.96|100.75|1075.56|1231.20|1119.48|1220.23|2350.68|2451.43|-1249.80| +2451113|42669|2451153|8264|25696|603031|1166|22854|92685|1395580|4812|46947|16|14|7|4|58|324|28|94.67|224.36|31.41|5402.60|879.48|2650.76|6282.08|61.56|0.00|565.32|879.48|941.04|1444.80|1506.36|-1771.28| +2451113|42669|2451209|3691|25696|603031|1166|22854|92685|1395580|4812|46947|13|8|13|3|66|324|36|86.44|184.98|110.98|2664.00|3995.28|3111.84|6659.28|199.76|0.00|3063.24|3995.28|4195.04|7058.52|7258.28|883.44| +2451113|42669|2451209|16651|25696|603031|1166|22854|92685|1395580|4812|46947|32|25|16|2|45|324|73|61.63|117.09|94.84|1624.25|6923.32|4498.99|8547.57|137.08|5400.18|2906.13|1523.14|1660.22|4429.27|4566.35|-2975.85| +2451113|42669|2451170|13028|25696|603031|1166|22854|92685|1395580|4812|46947|26|16|4|4|174|324|45|91.47|162.81|4.88|7106.85|219.60|4116.15|7326.45|0.00|0.00|1098.90|219.60|219.60|1318.50|1318.50|-3896.55| +2451113|42669|2451194|11245|25696|603031|1166|22854|92685|1395580|4812|46947|31|1|6|4|129|324|22|58.36|136.56|53.25|1832.82|1171.50|1283.92|3004.32|5.27|1066.06|1081.52|105.44|110.71|1186.96|1192.23|-1178.48| +2451113|42669|2451186|8672|25696|603031|1166|22854|92685|1395580|4812|46947|34|10|4|4|217|324|85|96.46|243.07|201.74|3513.05|17147.90|8199.10|20660.95|514.43|0.00|10330.05|17147.90|17662.33|27477.95|27992.38|8948.80| +2451113|42669|2451133|3517|25696|603031|1166|22854|92685|1395580|4812|46947|13|20|2|1|300|324|98|31.29|85.10|64.67|2002.14|6337.66|3066.42|8339.80|32.95|4689.86|3919.02|1647.80|1680.75|5566.82|5599.77|-1418.62| +2451113|42669|2451205|4402|25696|603031|1166|22854|92685|1395580|4812|46947|26|2|17|5|160|324|60|5.30|13.35|1.06|737.40|63.60|318.00|801.00|1.27|0.00|55.80|63.60|64.87|119.40|120.67|-254.40| +2451071|63131|2451082|11086|68682|486864|5846|39581|55487|560021|6480|28662|28|14|1|4|287|325|18|21.70|39.27|9.81|530.28|176.58|390.60|706.86|15.89|0.00|134.28|176.58|192.47|310.86|326.75|-214.02| +2451071|63131|2451121|8018|68682|486864|5846|39581|55487|560021|6480|28662|34|28|2|1|253|325|32|47.28|50.58|6.06|1424.64|193.92|1512.96|1618.56|17.45|0.00|566.40|193.92|211.37|760.32|777.77|-1319.04| +2451071|63131|2451144|4634|68682|486864|5846|39581|55487|560021|6480|28662|1|7|20|1|59|325|91|73.70|79.59|18.30|5577.39|1665.30|6706.70|7242.69|116.57|0.00|869.05|1665.30|1781.87|2534.35|2650.92|-5041.40| +2451071|63131|2451114|16735|68682|486864|5846|39581|55487|560021|6480|28662|50|25|20|2|214|325|22|61.32|129.99|102.69|600.60|2259.18|1349.04|2859.78|203.32|0.00|457.38|2259.18|2462.50|2716.56|2919.88|910.14| +2451071|63131|2451130|3800|68682|486864|5846|39581|55487|560021|6480|28662|7|7|1|1|258|325|33|40.84|112.71|94.67|595.32|3124.11|1347.72|3719.43|187.44|0.00|929.61|3124.11|3311.55|4053.72|4241.16|1776.39| +2451071|63131|2451096|3913|68682|486864|5846|39581|55487|560021|6480|28662|31|25|10|2|71|325|38|63.79|84.20|18.52|2495.84|703.76|2424.02|3199.60|7.03|0.00|319.96|703.76|710.79|1023.72|1030.75|-1720.26| +2451071|63131|2451146|10789|68682|486864|5846|39581|55487|560021|6480|28662|37|10|13|5|57|325|50|23.56|42.64|25.58|853.00|1279.00|1178.00|2132.00|12.79|0.00|724.50|1279.00|1291.79|2003.50|2016.29|101.00| +2451071|63131|2451085|4754|68682|486864|5846|39581|55487|560021|6480|28662|55|25|8|5|185|325|20|5.60|12.76|10.46|46.00|209.20|112.00|255.20|12.55|0.00|30.60|209.20|221.75|239.80|252.35|97.20| +2451071|63131|2451084|15217|68682|486864|5846|39581|55487|560021|6480|28662|55|4|11|1|34|325|64|18.49|28.47|28.47|0.00|1822.08|1183.36|1822.08|54.66|0.00|874.24|1822.08|1876.74|2696.32|2750.98|638.72| +2451071|63131|2451074|10496|68682|486864|5846|39581|55487|560021|6480|28662|1|8|2|1|273|325|89|70.92|162.40|56.84|9394.84|5058.76|6311.88|14453.60|354.11|0.00|6214.87|5058.76|5412.87|11273.63|11627.74|-1253.12| +2451071|63131|2451136|817|68682|486864|5846|39581|55487|560021|6480|28662|37|13|2|5|128|325|68|79.48|154.98|150.33|316.20|10222.44|5404.64|10538.64|613.34|0.00|947.92|10222.44|10835.78|11170.36|11783.70|4817.80| +2451511|74078|2451567|5366|39010|1801089|2015|30494|71817|659229|3578|24904|16|14|17|1|118|326|54|82.69|155.45|130.57|1343.52|7050.78|4465.26|8394.30|241.84|3595.89|3189.78|3454.89|3696.73|6644.67|6886.51|-1010.37| +2451511|74078|2451625|836|39010|1801089|2015|30494|71817|659229|3578|24904|50|1|3|5|30|326|6|24.76|42.33|2.96|236.22|17.76|148.56|253.98|0.00|0.00|2.52|17.76|17.76|20.28|20.28|-130.80| +2451511|74078|2451534|28|39010|1801089|2015|30494|71817|659229|3578|24904|55|19|9|1|210|326|34|72.94|144.42|11.55|4517.58|392.70|2479.96|4910.28|31.41|0.00|687.14|392.70|424.11|1079.84|1111.25|-2087.26| +2451511|74078|2451621|15856|39010|1801089|2015|30494|71817|659229|3578|24904|31|8|4|1|55|326|41|26.68|35.48|26.61|363.67|1091.01|1093.88|1454.68|65.46|0.00|116.03|1091.01|1156.47|1207.04|1272.50|-2.87| +2451511|74078|2451594|1808|39010|1801089|2015|30494|71817|659229|3578|24904|25|10|19|2|75|326|83|59.76|61.55|1.23|5006.56|102.09|4960.08|5108.65|2.98|27.56|50.63|74.53|77.51|125.16|128.14|-4885.55| +2451511|74078|2451595|11455|39010|1801089|2015|30494|71817|659229|3578|24904|20|28|19|3|124|326|50|46.96|115.99|114.83|58.00|5741.50|2348.00|5799.50|287.07|0.00|2551.50|5741.50|6028.57|8293.00|8580.07|3393.50| +2451511|74078|2451610|17356|39010|1801089|2015|30494|71817|659229|3578|24904|43|16|16|3|240|326|13|56.50|60.45|24.78|463.71|322.14|734.50|785.85|12.88|0.00|227.89|322.14|335.02|550.03|562.91|-412.36| +2451511|74078|2451588|9760|39010|1801089|2015|30494|71817|659229|3578|24904|46|25|15|2|19|326|75|16.47|27.17|7.60|1467.75|570.00|1235.25|2037.75|45.60|0.00|611.25|570.00|615.60|1181.25|1226.85|-665.25| +2451058|64093|2451064|17924|7160|1334003|3246|26489|76253|539492|3771|18614|52|13|10|3|263|327|69|13.16|20.39|15.70|323.61|1083.30|908.04|1406.91|80.92|184.16|548.55|899.14|980.06|1447.69|1528.61|-8.90| +2451058|64093|2451089|3826|7160|1334003|3246|26489|76253|539492|3771|18614|43|2|15|3|108|327|47|22.52|47.74|13.84|1593.30|650.48|1058.44|2243.78|45.53|0.00|673.04|650.48|696.01|1323.52|1369.05|-407.96| +2451058|64093|2451084|4570|7160|1334003|3246|26489|76253|539492|3771|18614|19|19|20|5|167|327|74|91.73|152.27|70.04|6085.02|5182.96|6788.02|11267.98|414.63|0.00|1464.46|5182.96|5597.59|6647.42|7062.05|-1605.06| +2451058|64093|2451135|16552|7160|1334003|3246|26489|76253|539492|3771|18614|25|8|7|2|184|327|67|44.70|85.82|33.46|3508.12|2241.82|2994.90|5749.94|0.00|0.00|1092.10|2241.82|2241.82|3333.92|3333.92|-753.08| +2451058|64093|2451088|4156|7160|1334003|3246|26489|76253|539492|3771|18614|8|16|16|5|37|327|6|72.53|188.57|171.59|101.88|1029.54|435.18|1131.42|0.00|0.00|56.52|1029.54|1029.54|1086.06|1086.06|594.36| +2451058|64093|2451069|8090|7160|1334003|3246|26489|76253|539492|3771|18614|52|4|9|3|44|327|18|60.71|119.59|3.58|2088.18|64.44|1092.78|2152.62|0.64|0.00|21.42|64.44|65.08|85.86|86.50|-1028.34| +2451058|64093|2451064|5746|7160|1334003|3246|26489|76253|539492|3771|18614|50|19|14|3|82|327|87|54.31|150.98|80.01|6174.39|6960.87|4724.97|13135.26|154.53|4385.34|5910.78|2575.53|2730.06|8486.31|8640.84|-2149.44| +2451058|64093|2451082|13861|7160|1334003|3246|26489|76253|539492|3771|18614|13|4|9|5|291|327|11|29.54|67.64|35.84|349.80|394.24|324.94|744.04|27.59|0.00|372.02|394.24|421.83|766.26|793.85|69.30| +2451058|64093|2451163|304|7160|1334003|3246|26489|76253|539492|3771|18614|8|25|1|4|127|327|35|29.67|63.79|9.56|1898.05|334.60|1038.45|2232.65|3.34|0.00|401.80|334.60|337.94|736.40|739.74|-703.85| +2451058|64093|2451118|9283|7160|1334003|3246|26489|76253|539492|3771|18614|7|4|2|1|150|327|22|6.37|10.57|1.47|200.20|32.34|140.14|232.54|1.61|0.00|27.72|32.34|33.95|60.06|61.67|-107.80| +2451058|64093|2451061|3838|7160|1334003|3246|26489|76253|539492|3771|18614|40|2|3|1|162|327|36|9.57|16.93|7.95|323.28|286.20|344.52|609.48|22.89|0.00|182.52|286.20|309.09|468.72|491.61|-58.32| +2451058|64093|2451139|16177|7160|1334003|3246|26489|76253|539492|3771|18614|8|20|7|3|209|327|55|77.26|212.46|82.85|7128.55|4556.75|4249.30|11685.30|66.98|3599.83|4440.15|956.92|1023.90|5397.07|5464.05|-3292.38| +2451058|64093|2451153|7531|7160|1334003|3246|26489|76253|539492|3771|18614|7|28|19|5|71|327|79|98.16|248.34|198.67|3923.93|15694.93|7754.64|19618.86|156.94|0.00|7062.60|15694.93|15851.87|22757.53|22914.47|7940.29| +2451520|56413|2451572|6112|4599|541145|91|637|64147|1538562|5475|12021|1|16|4|4|107|328|73|61.89|141.10|141.10|0.00|10300.30|4517.97|10300.30|0.00|0.00|3398.88|10300.30|10300.30|13699.18|13699.18|5782.33| +2451520|56413|2451528|10868|4599|541145|91|637|64147|1538562|5475|12021|13|8|20|5|223|328|89|3.39|6.16|2.03|367.57|180.67|301.71|548.24|10.84|0.00|234.96|180.67|191.51|415.63|426.47|-121.04| +2451520|56413|2451608|11083|4599|541145|91|637|64147|1538562|5475|12021|2|13|2|4|89|328|12|77.51|93.01|6.51|1038.00|78.12|930.12|1116.12|7.03|0.00|44.64|78.12|85.15|122.76|129.79|-852.00| +2451520|56413|2451579|9574|4599|541145|91|637|64147|1538562|5475|12021|31|26|2|5|188|328|28|64.72|144.97|85.53|1664.32|2394.84|1812.16|4059.16|71.84|0.00|1826.44|2394.84|2466.68|4221.28|4293.12|582.68| +2451520|56413|2451560|7594|4599|541145|91|637|64147|1538562|5475|12021|38|20|9|2|215|328|49|99.94|128.92|103.13|1263.71|5053.37|4897.06|6317.08|151.60|0.00|1073.59|5053.37|5204.97|6126.96|6278.56|156.31| +2451520|56413|2451611|2114|4599|541145|91|637|64147|1538562|5475|12021|40|14|15|2|59|328|42|28.08|83.11|79.78|139.86|3350.76|1179.36|3490.62|16.41|3116.20|1151.64|234.56|250.97|1386.20|1402.61|-944.80| +2451520|56413|2451588|15259|4599|541145|91|637|64147|1538562|5475|12021|49|14|11|4|77|328|59|80.15|192.36|182.74|567.58|10781.66|4728.85|11349.24|156.33|7654.97|5447.47|3126.69|3283.02|8574.16|8730.49|-1602.16| +2451520|56413|2451527|13970|4599|541145|91|637|64147|1538562|5475|12021|32|10|15|4|33|328|67|41.70|72.97|9.48|4253.83|635.16|2793.90|4888.99|38.10|0.00|1710.51|635.16|673.26|2345.67|2383.77|-2158.74| +2451520|56413|2451621|3703|4599|541145|91|637|64147|1538562|5475|12021|13|26|4|1|298|328|95|99.48|149.22|113.40|3402.90|10773.00|9450.60|14175.90|383.51|1185.03|141.55|9587.97|9971.48|9729.52|10113.03|137.37| +2451110|38092|2451200|1693|67387|708454|4368|44635|92122|582726|1530|3397|56|16|6|3|53|329|90|54.78|159.95|55.98|9357.30|5038.20|4930.20|14395.50|302.29|0.00|5901.30|5038.20|5340.49|10939.50|11241.79|108.00| +2451110|38092|2451121|4354|67387|708454|4368|44635|92122|582726|1530|3397|13|26|14|1|118|329|55|99.11|168.48|144.89|1297.45|7968.95|5451.05|9266.40|217.55|717.20|3613.50|7251.75|7469.30|10865.25|11082.80|1800.70| +2451110|38092|2451223|12130|67387|708454|4368|44635|92122|582726|1530|3397|16|10|3|5|140|329|60|63.42|156.64|23.49|7989.00|1409.40|3805.20|9398.40|112.75|0.00|939.60|1409.40|1522.15|2349.00|2461.75|-2395.80| +2451110|38092|2451177|11054|67387|708454|4368|44635|92122|582726|1530|3397|49|7|6|5|17|329|91|45.16|59.61|2.38|5207.93|216.58|4109.56|5424.51|8.66|0.00|2603.51|216.58|225.24|2820.09|2828.75|-3892.98| +2451110|38092|2451227|2758|67387|708454|4368|44635|92122|582726|1530|3397|1|7|10|1|52|329|7|21.19|47.04|7.05|279.93|49.35|148.33|329.28|3.94|0.00|78.96|49.35|53.29|128.31|132.25|-98.98| +2451110|38092|2451204|6694|67387|708454|4368|44635|92122|582726|1530|3397|55|20|4|4|87|329|19|23.78|51.84|24.88|512.24|472.72|451.82|984.96|28.36|0.00|462.84|472.72|501.08|935.56|963.92|20.90| +2451110|38092|2451116|14173|67387|708454|4368|44635|92122|582726|1530|3397|44|10|9|2|285|329|95|53.58|154.31|52.46|9675.75|4983.70|5090.10|14659.45|448.53|0.00|7329.25|4983.70|5432.23|12312.95|12761.48|-106.40| +2451110|38092|2451176|7024|67387|708454|4368|44635|92122|582726|1530|3397|13|26|11|5|42|329|46|3.43|4.42|2.07|108.10|95.22|157.78|203.32|5.71|0.00|48.76|95.22|100.93|143.98|149.69|-62.56| +2451508|53587|2451530|7099|16080|1422100|6167|48987|16080|1422100|6167|48987|34|7|13|4|195|330|19|74.37|97.42|14.61|1573.39|277.59|1413.03|1850.98|19.43|0.00|777.29|277.59|297.02|1054.88|1074.31|-1135.44| +2451508|53587|2451607|5470|16080|1422100|6167|48987|16080|1422100|6167|48987|10|28|17|4|81|330|53|67.01|123.96|43.38|4270.74|2299.14|3551.53|6569.88|22.99|0.00|2496.30|2299.14|2322.13|4795.44|4818.43|-1252.39| +2451508|53587|2451620|1814|16080|1422100|6167|48987|16080|1422100|6167|48987|32|25|6|2|285|330|82|41.03|54.15|16.78|3064.34|1375.96|3364.46|4440.30|0.00|0.00|576.46|1375.96|1375.96|1952.42|1952.42|-1988.50| +2451508|53587|2451623|10105|16080|1422100|6167|48987|16080|1422100|6167|48987|2|26|6|3|20|330|49|34.12|101.33|14.18|4270.35|694.82|1671.88|4965.17|48.63|0.00|2233.91|694.82|743.45|2928.73|2977.36|-977.06| +2451508|53587|2451608|8818|16080|1422100|6167|48987|16080|1422100|6167|48987|32|8|3|1|179|330|86|11.89|24.96|5.99|1631.42|515.14|1022.54|2146.56|46.36|0.00|1051.78|515.14|561.50|1566.92|1613.28|-507.40| +2451508|53587|2451597|3776|16080|1422100|6167|48987|16080|1422100|6167|48987|55|2|13|4|118|330|6|30.61|50.50|36.86|81.84|221.16|183.66|303.00|11.05|0.00|54.54|221.16|232.21|275.70|286.75|37.50| +2451508|53587|2451565|16168|16080|1422100|6167|48987|16080|1422100|6167|48987|22|25|11|2|90|330|24|24.22|48.92|4.89|1056.72|117.36|581.28|1174.08|2.34|0.00|457.68|117.36|119.70|575.04|577.38|-463.92| +2451508|53587|2451624|15904|16080|1422100|6167|48987|16080|1422100|6167|48987|32|8|13|3|66|330|83|45.53|90.14|75.71|1197.69|6283.93|3778.99|7481.62|62.83|0.00|1495.66|6283.93|6346.76|7779.59|7842.42|2504.94| +2451380|24149|2451449|8521|274|324155|836|11917|51057|941828|7113|10147|10|14|7|4|51|331|9|51.33|151.93|80.52|642.69|724.68|461.97|1367.37|14.49|0.00|669.96|724.68|739.17|1394.64|1409.13|262.71| +2451380|24149|2451439|15247|274|324155|836|11917|51057|941828|7113|10147|46|14|9|3|184|331|18|1.77|3.48|2.81|12.06|50.58|31.86|62.64|1.01|0.00|8.10|50.58|51.59|58.68|59.69|18.72| +2451380|24149|2451399|1423|274|324155|836|11917|51057|941828|7113|10147|32|4|2|3|97|331|35|80.79|230.25|195.71|1208.90|6849.85|2827.65|8058.75|273.99|0.00|2981.65|6849.85|7123.84|9831.50|10105.49|4022.20| +2451380|24149|2451458|10558|274|324155|836|11917|51057|941828|7113|10147|8|28|5|5|195|331|12|5.16|9.08|4.90|50.16|58.80|61.92|108.96|2.35|0.00|24.96|58.80|61.15|83.76|86.11|-3.12| +2451380|24149|2451456|1717|274|324155|836|11917|51057|941828|7113|10147|31|14|3|3|143|331|81|3.14|7.44|2.38|409.86|192.78|254.34|602.64|7.71|0.00|258.39|192.78|200.49|451.17|458.88|-61.56| +2451380|24149|2451404|4945|274|324155|836|11917|51057|941828|7113|10147|4|10|19|5|68|331|61|34.07|61.32|26.36|2132.56|1607.96|2078.27|3740.52|128.63|0.00|1570.75|1607.96|1736.59|3178.71|3307.34|-470.31| +2451380|24149|2451441|11221|274|324155|836|11917|51057|941828|7113|10147|46|28|6|2|228|331|19|31.70|35.18|26.38|167.20|501.22|602.30|668.42|10.02|0.00|267.33|501.22|511.24|768.55|778.57|-101.08| +2451380|24149|2451407|10504|274|324155|836|11917|51057|941828|7113|10147|31|8|7|1|55|331|76|27.08|35.74|17.51|1385.48|1330.76|2058.08|2716.24|93.15|0.00|1276.04|1330.76|1423.91|2606.80|2699.95|-727.32| +2451380|24149|2451479|10964|274|324155|836|11917|51057|941828|7113|10147|20|8|12|1|10|331|30|63.00|79.38|73.02|190.80|2190.60|1890.00|2381.40|21.90|0.00|0.00|2190.60|2212.50|2190.60|2212.50|300.60| +2451380|24149|2451424|4076|274|324155|836|11917|51057|941828|7113|10147|14|28|2|4|13|331|92|74.56|172.97|152.21|1909.92|14003.32|6859.52|15913.24|8.40|13723.25|6046.24|280.07|288.47|6326.31|6334.71|-6579.45| +2452602|17340|2452634|8239|2910|375271|5679|6887|95202|111211|6064|6425|15|7|3|4|135|332|67|94.83|154.57|77.28|5178.43|5177.76|6353.61|10356.19|207.11|0.00|4556.67|5177.76|5384.87|9734.43|9941.54|-1175.85| +2452602|17340|2452603|12276|2910|375271|5679|6887|95202|111211|6064|6425|49|15|11|4|107|332|66|28.96|62.55|58.17|289.08|3839.22|1911.36|4128.30|153.56|0.00|1073.16|3839.22|3992.78|4912.38|5065.94|1927.86| +2452602|17340|2452696|3187|2910|375271|5679|6887|95202|111211|6064|6425|19|6|16|4|35|332|74|35.71|83.56|55.14|2103.08|4080.36|2642.54|6183.44|326.42|0.00|61.42|4080.36|4406.78|4141.78|4468.20|1437.82| +2452602|17340|2452650|1095|2910|375271|5679|6887|95202|111211|6064|6425|7|19|19|5|51|332|15|84.88|114.58|22.91|1375.05|343.65|1273.20|1718.70|17.32|151.20|756.15|192.45|209.77|948.60|965.92|-1080.75| +2452602|17340|2452701|10969|2910|375271|5679|6887|95202|111211|6064|6425|15|19|15|2|219|332|14|51.11|151.79|132.05|276.36|1848.70|715.54|2125.06|2.95|1774.75|913.64|73.95|76.90|987.59|990.54|-641.59| +2452602|17340|2452715|11287|2910|375271|5679|6887|95202|111211|6064|6425|39|13|4|1|28|332|2|55.59|73.93|14.04|119.78|28.08|111.18|147.86|0.56|0.00|38.44|28.08|28.64|66.52|67.08|-83.10| +2452602|17340|2452615|15225|2910|375271|5679|6887|95202|111211|6064|6425|27|30|6|2|85|332|99|89.06|173.66|92.03|8081.37|9110.97|8816.94|17192.34|273.32|0.00|8251.65|9110.97|9384.29|17362.62|17635.94|294.03| +2452602|17340|2452684|9913|2910|375271|5679|6887|95202|111211|6064|6425|36|1|4|2|183|332|64|84.64|198.90|107.40|5856.00|6873.60|5416.96|12729.60|274.94|0.00|4837.12|6873.60|7148.54|11710.72|11985.66|1456.64| +2452602|17340|2452634|825|2910|375271|5679|6887|95202|111211|6064|6425|30|9|5|3|178|332|14|5.63|15.93|14.49|20.16|202.86|78.82|223.02|0.00|6.08|75.74|196.78|196.78|272.52|272.52|117.96| +2452602|17340|2452635|16914|2910|375271|5679|6887|95202|111211|6064|6425|60|21|12|3|194|332|94|85.59|207.98|99.83|10166.10|9384.02|8045.46|19550.12|844.56|0.00|3713.94|9384.02|10228.58|13097.96|13942.52|1338.56| +2452602|17340|2452701|1443|2910|375271|5679|6887|95202|111211|6064|6425|51|18|2|5|288|332|53|69.87|164.89|113.77|2709.36|6029.81|3703.11|8739.17|123.00|1929.53|3932.60|4100.28|4223.28|8032.88|8155.88|397.17| +2452602|17340|2452640|14059|2910|375271|5679|6887|95202|111211|6064|6425|42|27|7|3|166|332|86|39.41|50.83|47.27|306.16|4065.22|3389.26|4371.38|325.21|0.00|1704.52|4065.22|4390.43|5769.74|6094.95|675.96| +2452602|17340|2452685|9589|2910|375271|5679|6887|95202|111211|6064|6425|18|30|14|5|296|332|50|41.16|71.61|5.72|3294.50|286.00|2058.00|3580.50|17.16|0.00|1790.00|286.00|303.16|2076.00|2093.16|-1772.00| +2452602|17340|2452653|3325|2910|375271|5679|6887|95202|111211|6064|6425|7|18|4|4|294|332|99|19.25|29.26|1.17|2780.91|115.83|1905.75|2896.74|3.47|0.00|115.83|115.83|119.30|231.66|235.13|-1789.92| +2452602|17340|2452655|159|2910|375271|5679|6887|95202|111211|6064|6425|54|27|18|5|222|332|88|15.63|31.72|16.17|1368.40|1422.96|1375.44|2791.36|42.68|0.00|613.36|1422.96|1465.64|2036.32|2079.00|47.52| +2452602|17340|2452676|12063|2910|375271|5679|6887|95202|111211|6064|6425|1|9|9|3|227|332|99|18.28|26.50|9.01|1731.51|891.99|1809.72|2623.50|0.00|0.00|1101.87|891.99|891.99|1993.86|1993.86|-917.73| +2451503|23631|2451591|17128|14848|419033|3218|10203|81324|178700|6358|37343|4|22|11|2|46|333|61|32.67|81.34|72.39|545.95|4415.79|1992.87|4961.74|353.26|0.00|793.61|4415.79|4769.05|5209.40|5562.66|2422.92| +2451503|23631|2451579|6038|14848|419033|3218|10203|81324|178700|6358|37343|40|13|6|2|281|333|83|41.55|56.50|37.29|1594.43|3095.07|3448.65|4689.50|278.55|0.00|327.85|3095.07|3373.62|3422.92|3701.47|-353.58| +2451503|23631|2451620|15298|14848|419033|3218|10203|81324|178700|6358|37343|38|1|19|2|115|333|15|88.51|181.44|1.81|2694.45|27.15|1327.65|2721.60|0.81|0.00|1034.10|27.15|27.96|1061.25|1062.06|-1300.50| +2451503|23631|2451597|6748|14848|419033|3218|10203|81324|178700|6358|37343|14|13|16|2|281|333|98|69.53|116.11|116.11|0.00|11378.78|6813.94|11378.78|187.74|5120.45|1137.78|6258.33|6446.07|7396.11|7583.85|-555.61| +2451503|23631|2451521|16622|14848|419033|3218|10203|81324|178700|6358|37343|40|25|15|2|49|333|55|50.04|96.57|63.73|1806.20|3505.15|2752.20|5311.35|70.10|0.00|2071.30|3505.15|3575.25|5576.45|5646.55|752.95| +2451503|23631|2451552|14890|14848|419033|3218|10203|81324|178700|6358|37343|43|26|6|3|290|333|58|52.94|74.11|67.44|386.86|3911.52|3070.52|4298.38|117.34|0.00|472.70|3911.52|4028.86|4384.22|4501.56|841.00| +2451503|23631|2451561|10714|14848|419033|3218|10203|81324|178700|6358|37343|44|26|8|4|243|333|3|43.44|76.02|72.21|11.43|216.63|130.32|228.06|19.49|0.00|70.68|216.63|236.12|287.31|306.80|86.31| +2451503|23631|2451602|1708|14848|419033|3218|10203|81324|178700|6358|37343|56|7|19|4|229|333|50|97.31|158.61|22.20|6820.50|1110.00|4865.50|7930.50|22.64|543.90|2617.00|566.10|588.74|3183.10|3205.74|-4299.40| +2451855|71707|2451885|12259|47377|107577|4577|10377|93298|838401|5824|28024|1|23|6|5|204|334|83|63.66|73.20|23.42|4131.74|1943.86|5283.78|6075.60|97.19|0.00|1396.89|1943.86|2041.05|3340.75|3437.94|-3339.92| +2451855|71707|2451883|11335|47377|107577|4577|10377|93298|838401|5824|28024|17|14|4|3|282|334|49|99.00|171.27|22.26|7301.49|1090.74|4851.00|8392.23|43.62|0.00|1594.46|1090.74|1134.36|2685.20|2728.82|-3760.26| +2451855|71707|2451964|11264|47377|107577|4577|10377|93298|838401|5824|28024|50|14|7|4|201|334|96|26.30|43.92|35.57|801.60|3414.72|2524.80|4216.32|102.44|0.00|463.68|3414.72|3517.16|3878.40|3980.84|889.92| +2451855|71707|2451911|4385|47377|107577|4577|10377|93298|838401|5824|28024|20|29|19|5|214|334|35|82.51|238.45|193.14|1585.85|6759.90|2887.85|8345.75|473.19|0.00|3171.35|6759.90|7233.09|9931.25|10404.44|3872.05| +2451855|71707|2451910|16183|47377|107577|4577|10377|93298|838401|5824|28024|1|26|14|4|194|334|27|24.01|34.09|31.36|73.71|846.72|648.27|920.43|76.20|0.00|27.54|846.72|922.92|874.26|950.46|198.45| +2451855|71707|2451924|9536|47377|107577|4577|10377|93298|838401|5824|28024|41|13|6|2|71|334|26|93.79|158.50|137.89|535.86|3585.14|2438.54|4121.00|107.55|0.00|782.86|3585.14|3692.69|4368.00|4475.55|1146.60| +2451855|71707|2451866|8780|47377|107577|4577|10377|93298|838401|5824|28024|53|17|1|1|218|334|31|20.70|21.94|6.80|469.34|210.80|641.70|680.14|0.00|208.69|67.89|2.11|2.11|70.00|70.00|-639.59| +2451855|71707|2451860|14041|47377|107577|4577|10377|93298|838401|5824|28024|50|26|9|2|202|334|17|39.68|51.98|4.15|813.11|70.55|674.56|883.66|2.82|0.00|238.51|70.55|73.37|309.06|311.88|-604.01| +2451855|71707|2451877|8531|47377|107577|4577|10377|93298|838401|5824|28024|1|13|13|2|78|334|11|41.59|50.73|46.16|50.27|507.76|457.49|558.03|1.52|477.29|156.20|30.47|31.99|186.67|188.19|-427.02| +2451855|71707|2451924|11669|47377|107577|4577|10377|93298|838401|5824|28024|56|25|4|3|182|334|27|36.74|74.58|44.00|825.66|1188.00|991.98|2013.66|35.64|0.00|1006.83|1188.00|1223.64|2194.83|2230.47|196.02| +2451855|71707|2451876|3089|47377|107577|4577|10377|93298|838401|5824|28024|49|17|3|2|51|334|63|90.49|261.51|70.60|12027.33|4447.80|5700.87|16475.13|311.34|0.00|8237.25|4447.80|4759.14|12685.05|12996.39|-1253.07| +2451855|71707|2451875|4733|47377|107577|4577|10377|93298|838401|5824|28024|29|23|2|4|259|334|24|45.05|83.34|42.50|980.16|1020.00|1081.20|2000.16|20.40|0.00|559.92|1020.00|1040.40|1579.92|1600.32|-61.20| +2451855|71707|2451932|3157|47377|107577|4577|10377|93298|838401|5824|28024|1|8|2|4|236|334|93|85.35|234.71|138.47|8950.32|12877.71|7937.55|21828.03|829.32|1030.21|5238.69|11847.50|12676.82|17086.19|17915.51|3909.95| +2451855|71707|2451868|1466|47377|107577|4577|10377|93298|838401|5824|28024|47|20|18|2|39|334|87|69.34|186.52|52.22|11684.10|4543.14|6032.58|16227.24|31.34|3498.21|5516.67|1044.93|1076.27|6561.60|6592.94|-4987.65| +2451855|71707|2451933|8804|47377|107577|4577|10377|93298|838401|5824|28024|56|8|14|3|159|334|99|8.16|8.89|4.17|467.28|412.83|807.84|880.11|24.76|0.00|421.74|412.83|437.59|834.57|859.33|-395.01| +2451855|71707|2451956|14771|47377|107577|4577|10377|93298|838401|5824|28024|17|23|12|5|88|334|9|60.24|115.66|55.51|541.35|499.59|542.16|1040.94|4.99|0.00|145.71|499.59|504.58|645.30|650.29|-42.57| +2451441|48026|2451548|12571|75304|324932|1313|24094|7975|152333|4437|27618|28|22|6|3|17|335|40|86.68|205.43|92.44|4519.60|3697.60|3467.20|8217.20|184.88|0.00|4026.40|3697.60|3882.48|7724.00|7908.88|230.40| +2451441|48026|2451478|5950|75304|324932|1313|24094|7975|152333|4437|27618|14|16|17|3|139|335|66|9.81|14.32|2.14|803.88|141.24|647.46|945.12|4.23|0.00|75.24|141.24|145.47|216.48|220.71|-506.22| +2451441|48026|2451532|5113|75304|324932|1313|24094|7975|152333|4437|27618|14|2|6|2|261|335|20|21.60|40.60|0.00|812.00|0.00|432.00|812.00|0.00|0.00|194.80|0.00|0.00|194.80|194.80|-432.00| +2451441|48026|2451512|2042|75304|324932|1313|24094|7975|152333|4437|27618|16|19|14|3|239|335|66|8.18|13.57|11.26|152.46|743.16|539.88|895.62|0.00|0.00|331.32|743.16|743.16|1074.48|1074.48|203.28| +2451441|48026|2451472|2995|75304|324932|1313|24094|7975|152333|4437|27618|43|19|15|3|191|335|58|54.89|97.15|60.23|2141.36|3493.34|3183.62|5634.70|166.63|1641.86|337.56|1851.48|2018.11|2189.04|2355.67|-1332.14| +2451441|48026|2451534|6229|75304|324932|1313|24094|7975|152333|4437|27618|19|26|11|3|5|335|5|48.66|143.54|8.61|674.65|43.05|243.30|717.70|0.00|0.00|258.35|43.05|43.05|301.40|301.40|-200.25| +2451441|48026|2451460|5210|75304|324932|1313|24094|7975|152333|4437|27618|49|22|9|1|63|335|28|67.29|106.99|17.11|2516.64|479.08|1884.12|2995.72|23.95|0.00|1048.32|479.08|503.03|1527.40|1551.35|-1405.04| +2451441|48026|2451497|16042|75304|324932|1313|24094|7975|152333|4437|27618|7|14|12|3|87|335|26|27.31|37.14|8.54|743.60|222.04|710.06|965.64|19.98|0.00|444.08|222.04|242.02|666.12|686.10|-488.02| +2451441|48026|2451496|3712|75304|324932|1313|24094|7975|152333|4437|27618|26|16|7|2|190|335|45|56.46|80.17|32.06|2164.95|1442.70|2540.70|3607.65|1.15|1327.28|72.00|115.42|116.57|187.42|188.57|-2425.28| +2451441|48026|2451542|4280|75304|324932|1313|24094|7975|152333|4437|27618|7|19|6|2|268|335|3|33.88|80.29|64.23|48.18|192.69|101.64|240.87|15.41|0.00|14.43|192.69|208.10|207.12|222.53|91.05| +2451441|48026|2451507|7984|75304|324932|1313|24094|7975|152333|4437|27618|25|7|5|1|97|335|22|12.17|15.21|1.06|311.30|23.32|267.74|334.62|1.16|0.00|163.90|23.32|24.48|187.22|188.38|-244.42| +2450829|75830|2450944|13|4602|712389|6129|32716|68719|489317|2430|47770|31|14|3|3|120|336|49|8.93|16.16|12.44|182.28|609.56|437.57|791.84|24.38|0.00|173.95|609.56|633.94|783.51|807.89|171.99| +2450829|75830|2450843|5740|4602|712389|6129|32716|68719|489317|2430|47770|55|19|19|1|83|336|90|53.24|96.36|12.52|7545.60|1126.80|4791.60|8672.40|78.87|0.00|1040.40|1126.80|1205.67|2167.20|2246.07|-3664.80| +2450829|75830|2450902|8224|4602|712389|6129|32716|68719|489317|2430|47770|56|2|15|1|244|336|74|82.19|237.52|2.37|17401.10|175.38|6082.06|17576.48|15.78|0.00|7030.00|175.38|191.16|7205.38|7221.16|-5906.68| +2450829|75830|2450871|15086|4602|712389|6129|32716|68719|489317|2430|47770|37|1|13|5|247|336|51|92.18|229.52|130.82|5033.70|6671.82|4701.18|11705.52|128.09|266.87|584.97|6404.95|6533.04|6989.92|7118.01|1703.77| +2450829|75830|2450833|16414|4602|712389|6129|32716|68719|489317|2430|47770|55|14|18|5|285|336|75|77.55|107.79|4.31|7761.00|323.25|5816.25|8084.25|6.46|0.00|888.75|323.25|329.71|1212.00|1218.46|-5493.00| +2450829|75830|2450888|9430|4602|712389|6129|32716|68719|489317|2430|47770|19|28|7|2|67|336|75|29.60|54.76|43.80|822.00|3285.00|2220.00|4107.00|32.85|0.00|985.50|3285.00|3317.85|4270.50|4303.35|1065.00| +2450829|75830|2450857|15992|4602|712389|6129|32716|68719|489317|2430|47770|2|1|10|1|155|336|11|80.25|203.83|110.06|1031.47|1210.66|882.75|2242.13|96.85|0.00|1008.92|1210.66|1307.51|2219.58|2316.43|327.91| +2450829|75830|2450885|6070|4602|712389|6129|32716|68719|489317|2430|47770|28|13|19|1|96|336|35|33.98|101.26|96.19|177.45|3366.65|1189.30|3544.10|201.99|0.00|425.25|3366.65|3568.64|3791.90|3993.89|2177.35| +2451137|57669|2451171|1652|70427|1660363|1499|5082|65136|836252|3290|6426|16|19|4|5|29|337|27|81.13|153.33|133.39|538.38|3601.53|2190.51|4139.91|1.80|3421.45|827.82|180.08|181.88|1007.90|1009.70|-2010.43| +2451137|57669|2451254|16592|70427|1660363|1499|5082|65136|836252|3290|6426|2|16|19|1|23|337|100|64.07|117.24|33.99|8325.00|3399.00|6407.00|11724.00|203.94|0.00|3868.00|3399.00|3602.94|7267.00|7470.94|-3008.00| +2451137|57669|2451232|1291|70427|1660363|1499|5082|65136|836252|3290|6426|38|7|1|5|92|337|93|12.68|34.99|3.49|2929.50|324.57|1179.24|3254.07|12.72|142.81|324.57|181.76|194.48|506.33|519.05|-997.48| +2451137|57669|2451206|3850|70427|1660363|1499|5082|65136|836252|3290|6426|25|28|14|1|148|337|54|67.71|163.85|55.70|5840.10|3007.80|3656.34|8847.90|270.70|0.00|884.52|3007.80|3278.50|3892.32|4163.02|-648.54| +2451137|57669|2451212|12262|70427|1660363|1499|5082|65136|836252|3290|6426|26|13|3|1|9|337|62|35.19|101.34|20.26|5026.96|1256.12|2181.78|6283.08|12.56|0.00|502.20|1256.12|1268.68|1758.32|1770.88|-925.66| +2451137|57669|2451192|13390|70427|1660363|1499|5082|65136|836252|3290|6426|43|4|14|5|156|337|80|40.05|41.25|14.02|2178.40|1121.60|3204.00|3300.00|44.86|0.00|1616.80|1121.60|1166.46|2738.40|2783.26|-2082.40| +2451137|57669|2451191|5089|70427|1660363|1499|5082|65136|836252|3290|6426|8|13|19|1|110|337|31|97.28|105.06|36.77|2116.99|1139.87|3015.68|3256.86|34.19|0.00|976.81|1139.87|1174.06|2116.68|2150.87|-1875.81| +2451137|57669|2451149|8005|70427|1660363|1499|5082|65136|836252|3290|6426|50|8|14|3|69|337|82|91.41|238.58|78.73|13107.70|6455.86|7495.62|19563.56|239.51|3034.25|8020.42|3421.61|3661.12|11442.03|11681.54|-4074.01| +2451137|57669|2451183|16435|70427|1660363|1499|5082|65136|836252|3290|6426|8|4|14|2|140|337|67|64.77|110.75|73.09|2523.22|4897.03|4339.59|7420.25|195.88|0.00|1038.50|4897.03|5092.91|5935.53|6131.41|557.44| +2451024|55805|2451134|16966|55752|631989|2626|39663|66225|1702732|5974|34564|20|22|6|5|121|338|60|5.99|11.14|0.33|648.60|19.80|359.40|668.40|0.99|0.00|120.00|19.80|20.79|139.80|140.79|-339.60| +2451024|55805|2451106|10402|55752|631989|2626|39663|66225|1702732|5974|34564|1|25|5|2|42|338|76|40.39|69.06|30.38|2939.68|2308.88|3069.64|5248.56|184.71|0.00|52.44|2308.88|2493.59|2361.32|2546.03|-760.76| +2451024|55805|2451062|1735|55752|631989|2626|39663|66225|1702732|5974|34564|4|10|12|3|211|338|48|53.24|76.13|49.48|1279.20|2375.04|2555.52|3654.24|213.75|0.00|1425.12|2375.04|2588.79|3800.16|4013.91|-180.48| +2451024|55805|2451051|15464|55752|631989|2626|39663|66225|1702732|5974|34564|1|28|16|5|64|338|1|86.62|195.76|3.91|191.85|3.91|86.62|195.76|0.15|0.00|82.21|3.91|4.06|86.12|86.27|-82.71| +2451024|55805|2451093|3595|55752|631989|2626|39663|66225|1702732|5974|34564|55|8|12|2|259|338|54|3.46|7.02|0.07|375.30|3.78|186.84|379.08|0.00|0.00|90.72|3.78|3.78|94.50|94.50|-183.06| +2451024|55805|2451049|4273|55752|631989|2626|39663|66225|1702732|5974|34564|22|25|4|5|83|338|26|56.05|108.17|29.20|2053.22|759.20|1457.30|2812.42|37.96|0.00|1321.58|759.20|797.16|2080.78|2118.74|-698.10| +2451024|55805|2451144|14756|55752|631989|2626|39663|66225|1702732|5974|34564|49|22|6|3|40|338|11|81.08|83.51|78.49|55.22|863.39|891.88|918.61|25.90|0.00|339.79|863.39|889.29|1203.18|1229.08|-28.49| +2451024|55805|2451057|1579|55752|631989|2626|39663|66225|1702732|5974|34564|7|20|18|2|69|338|35|42.09|52.19|14.61|1315.30|511.35|1473.15|1826.65|35.79|0.00|0.00|511.35|547.14|511.35|547.14|-961.80| +2451024|55805|2451091|5252|55752|631989|2626|39663|66225|1702732|5974|34564|31|1|6|4|9|338|27|23.67|51.60|37.66|376.38|1016.82|639.09|1393.20|91.51|0.00|83.43|1016.82|1108.33|1100.25|1191.76|377.73| +2451024|55805|2451064|11155|55752|631989|2626|39663|66225|1702732|5974|34564|44|28|14|3|192|338|83|55.87|138.55|38.79|8280.08|3219.57|4637.21|11499.65|0.00|0.00|3449.48|3219.57|3219.57|6669.05|6669.05|-1417.64| +2451024|55805|2451045|13094|55752|631989|2626|39663|66225|1702732|5974|34564|46|7|14|4|4|338|12|27.92|40.20|23.31|202.68|279.72|335.04|482.40|19.58|0.00|33.72|279.72|299.30|313.44|333.02|-55.32| +2451024|55805|2451027|11422|55752|631989|2626|39663|66225|1702732|5974|34564|19|26|1|3|5|338|47|98.37|183.95|104.85|3717.70|4927.95|4623.39|8645.65|295.67|0.00|2334.02|4927.95|5223.62|7261.97|7557.64|304.56| +2451024|55805|2451031|691|55752|631989|2626|39663|66225|1702732|5974|34564|31|10|3|5|110|338|19|85.28|131.33|128.70|49.97|2445.30|1620.32|2495.27|30.56|1833.97|1022.96|611.33|641.89|1634.29|1664.85|-1008.99| +2451991|82558|2452041|4539|18754|1717897|1840|2583|35109|68647|6059|48002|41|1|11|3|278|339|15|20.18|53.27|34.09|287.70|511.35|302.70|799.05|40.90|0.00|79.80|511.35|552.25|591.15|632.05|208.65| +2451991|82558|2452055|8845|18754|1717897|1840|2583|35109|68647|6059|48002|9|5|8|1|244|339|61|92.19|164.09|63.99|6106.10|3903.39|5623.59|10009.49|117.10|0.00|2201.49|3903.39|4020.49|6104.88|6221.98|-1720.20| +2451991|82558|2452006|2903|18754|1717897|1840|2583|35109|68647|6059|48002|49|9|2|2|222|339|63|73.84|166.14|31.56|8478.54|1988.28|4651.92|10466.82|79.53|0.00|522.90|1988.28|2067.81|2511.18|2590.71|-2663.64| +2451991|82558|2452038|3771|18754|1717897|1840|2583|35109|68647|6059|48002|21|17|17|2|40|339|60|53.55|88.89|5.33|5013.60|319.80|3213.00|5333.40|28.78|0.00|2186.40|319.80|348.58|2506.20|2534.98|-2893.20| +2451991|82558|2452046|3029|18754|1717897|1840|2583|35109|68647|6059|48002|31|7|12|5|178|339|50|95.87|115.04|93.18|1093.00|4659.00|4793.50|5752.00|419.31|0.00|690.00|4659.00|5078.31|5349.00|5768.31|-134.50| +2451991|82558|2452103|6099|18754|1717897|1840|2583|35109|68647|6059|48002|35|29|19|3|267|339|44|35.80|66.94|40.16|1178.32|1767.04|1575.20|2945.36|123.69|0.00|1354.76|1767.04|1890.73|3121.80|3245.49|191.84| +2451991|82558|2452058|3267|18754|1717897|1840|2583|35109|68647|6059|48002|11|25|7|4|18|339|51|44.76|102.05|46.94|2810.61|2393.94|2282.76|5204.55|116.34|454.84|312.12|1939.10|2055.44|2251.22|2367.56|-343.66| +2451991|82558|2452105|413|18754|1717897|1840|2583|35109|68647|6059|48002|51|1|3|3|300|339|61|86.11|129.16|81.37|2915.19|4963.57|5252.71|7878.76|0.00|0.00|1733.01|4963.57|4963.57|6696.58|6696.58|-289.14| +2452345|65139|2452377|1800|67874|1559295|1563|47784|91464|1671004|4141|26762|39|27|11|5|42|340|92|8.47|11.09|8.87|204.24|816.04|779.24|1020.28|37.94|57.12|489.44|758.92|796.86|1248.36|1286.30|-20.32| +2452345|65139|2452438|14931|67874|1559295|1563|47784|91464|1671004|4141|26762|18|13|12|2|82|340|95|12.54|25.45|22.90|242.25|2175.50|1191.30|2417.75|0.00|0.00|821.75|2175.50|2175.50|2997.25|2997.25|984.20| +2452345|65139|2452430|9873|67874|1559295|1563|47784|91464|1671004|4141|26762|42|25|15|5|102|340|93|53.88|142.24|27.02|10715.46|2512.86|5010.84|13228.32|25.12|0.00|5290.77|2512.86|2537.98|7803.63|7828.75|-2497.98| +2452345|65139|2452405|5803|67874|1559295|1563|47784|91464|1671004|4141|26762|55|9|15|5|42|340|57|62.78|94.79|28.43|3782.52|1620.51|3578.46|5403.03|97.23|0.00|1350.33|1620.51|1717.74|2970.84|3068.07|-1957.95| +2452345|65139|2452411|13488|67874|1559295|1563|47784|91464|1671004|4141|26762|15|21|14|5|283|340|27|11.70|19.53|13.86|153.09|374.22|315.90|527.31|22.45|0.00|52.65|374.22|396.67|426.87|449.32|58.32| +2452345|65139|2452434|16506|67874|1559295|1563|47784|91464|1671004|4141|26762|18|19|1|2|133|340|40|80.61|151.54|3.03|5940.40|121.20|3224.40|6061.60|8.48|0.00|1576.00|121.20|129.68|1697.20|1705.68|-3103.20| +2452345|65139|2452464|3096|67874|1559295|1563|47784|91464|1671004|4141|26762|49|25|18|3|268|340|23|39.04|65.19|49.54|359.95|1139.42|897.92|1499.37|0.00|376.00|674.59|763.42|763.42|1438.01|1438.01|-134.50| +2452345|65139|2452393|8151|67874|1559295|1563|47784|91464|1671004|4141|26762|19|18|1|5|89|340|31|59.06|122.25|99.02|720.13|3069.62|1830.86|3789.75|184.17|0.00|113.46|3069.62|3253.79|3183.08|3367.25|1238.76| +2452345|65139|2452432|8394|67874|1559295|1563|47784|91464|1671004|4141|26762|49|25|18|1|44|340|55|42.85|47.13|28.74|1011.45|1580.70|2356.75|2592.15|126.45|0.00|906.95|1580.70|1707.15|2487.65|2614.10|-776.05| +2452345|65139|2452368|13089|67874|1559295|1563|47784|91464|1671004|4141|26762|45|19|4|3|262|340|10|20.14|37.05|11.85|252.00|118.50|201.40|370.50|4.74|0.00|144.40|118.50|123.24|262.90|267.64|-82.90| +2451628|56124|2451638|14303|54747|1807559|5971|830|30200|1396623|1313|17725|56|23|18|1|97|341|88|59.12|82.17|78.06|361.68|6869.28|5202.56|7230.96|618.23|0.00|2313.52|6869.28|7487.51|9182.80|9801.03|1666.72| +2451628|56124|2451654|4589|54747|1807559|5971|830|30200|1396623|1313|17725|55|23|3|2|188|341|82|21.96|33.37|3.33|2463.28|273.06|1800.72|2736.34|0.00|0.00|929.88|273.06|273.06|1202.94|1202.94|-1527.66| +2451628|56124|2451669|4910|54747|1807559|5971|830|30200|1396623|1313|17725|8|5|11|4|64|341|84|52.27|147.92|73.96|6212.64|6212.64|4390.68|12425.28|497.01|0.00|123.48|6212.64|6709.65|6336.12|6833.13|1821.96| +2451628|56124|2451687|16427|54747|1807559|5971|830|30200|1396623|1313|17725|29|11|17|3|39|341|9|5.69|15.93|7.00|80.37|63.00|51.21|143.37|1.26|0.00|51.57|63.00|64.26|114.57|115.83|11.79| +2451628|56124|2451702|16226|54747|1807559|5971|830|30200|1396623|1313|17725|38|19|20|5|220|341|19|81.49|182.53|127.77|1040.44|2427.63|1548.31|3468.07|145.65|0.00|1421.77|2427.63|2573.28|3849.40|3995.05|879.32| +2451628|56124|2451637|1232|54747|1807559|5971|830|30200|1396623|1313|17725|47|2|3|1|79|341|21|30.35|33.99|27.53|135.66|578.13|637.35|713.79|17.34|0.00|185.43|578.13|595.47|763.56|780.90|-59.22| +2451628|56124|2451709|1543|54747|1807559|5971|830|30200|1396623|1313|17725|2|26|9|4|209|341|38|5.74|14.46|10.26|159.60|389.88|218.12|549.48|7.79|0.00|60.42|389.88|397.67|450.30|458.09|171.76| +2451628|56124|2451733|3422|54747|1807559|5971|830|30200|1396623|1313|17725|7|19|13|1|12|341|39|22.77|35.06|24.54|410.28|957.06|888.03|1367.34|0.00|593.37|218.40|363.69|363.69|582.09|582.09|-524.34| +2451628|56124|2451680|16256|54747|1807559|5971|830|30200|1396623|1313|17725|56|2|15|3|48|341|18|92.33|207.74|126.72|1458.36|2280.96|1661.94|3739.32|182.47|0.00|1533.06|2280.96|2463.43|3814.02|3996.49|619.02| +2451628|56124|2451657|8365|54747|1807559|5971|830|30200|1396623|1313|17725|1|25|18|5|222|341|75|38.77|53.11|27.61|1912.50|2070.75|2907.75|3983.25|0.00|0.00|1353.75|2070.75|2070.75|3424.50|3424.50|-837.00| +2451628|56124|2451667|14402|54747|1807559|5971|830|30200|1396623|1313|17725|47|7|1|3|78|341|91|85.00|165.75|106.08|5429.97|9653.28|7735.00|15083.25|193.06|0.00|4976.79|9653.28|9846.34|14630.07|14823.13|1918.28| +2451628|56124|2451704|2216|54747|1807559|5971|830|30200|1396623|1313|17725|31|13|8|4|190|341|54|50.86|93.58|46.79|2526.66|2526.66|2746.44|5053.32|151.59|0.00|2071.44|2526.66|2678.25|4598.10|4749.69|-219.78| +2451628|56124|2451656|11780|54747|1807559|5971|830|30200|1396623|1313|17725|11|20|19|4|162|341|64|76.45|220.17|184.94|2254.72|11836.16|4892.80|14090.88|44.97|9587.28|4085.76|2248.88|2293.85|6334.64|6379.61|-2643.92| +2451628|56124|2451665|15881|54747|1807559|5971|830|30200|1396623|1313|17725|11|19|1|2|276|341|27|48.66|58.39|42.62|425.79|1150.74|1313.82|1576.53|46.02|0.00|662.04|1150.74|1196.76|1812.78|1858.80|-163.08| +2451628|56124|2451689|17671|54747|1807559|5971|830|30200|1396623|1313|17725|7|8|6|2|158|341|97|81.72|226.36|190.14|3513.34|18443.58|7926.84|21956.92|737.74|0.00|8123.75|18443.58|19181.32|26567.33|27305.07|10516.74| +2451628|56124|2451725|4247|54747|1807559|5971|830|30200|1396623|1313|17725|59|5|6|4|5|341|79|89.04|131.77|64.56|5309.59|5100.24|7034.16|10409.83|255.01|0.00|3018.59|5100.24|5355.25|8118.83|8373.84|-1933.92| +2451528|21164|2451529|15334|10515|34131|6194|34210|75591|1864826|7152|21568|19|25|9|2|15|342|22|16.02|23.70|11.61|265.98|255.42|352.44|521.40|5.10|0.00|192.72|255.42|260.52|448.14|453.24|-97.02| +2451528|21164|2451624|8608|10515|34131|6194|34210|75591|1864826|7152|21568|14|1|1|1|264|342|5|22.24|46.48|36.25|51.15|181.25|111.20|232.40|1.81|0.00|81.30|181.25|183.06|262.55|264.36|70.05| +2451528|21164|2451550|1238|10515|34131|6194|34210|75591|1864826|7152|21568|31|16|15|1|237|342|37|30.92|73.28|41.76|1166.24|1545.12|1144.04|2711.36|139.06|0.00|108.41|1545.12|1684.18|1653.53|1792.59|401.08| +2451528|21164|2451647|8176|10515|34131|6194|34210|75591|1864826|7152|21568|13|28|6|4|175|342|35|99.80|146.70|112.95|1181.25|3953.25|3493.00|5134.50|39.53|0.00|1642.90|3953.25|3992.78|5596.15|5635.68|460.25| +2451528|21164|2451595|2224|10515|34131|6194|34210|75591|1864826|7152|21568|37|26|12|2|165|342|81|27.81|64.24|58.45|468.99|4734.45|2252.61|5203.44|51.13|3030.04|1456.38|1704.41|1755.54|3160.79|3211.92|-548.20| +2451528|21164|2451615|11470|10515|34131|6194|34210|75591|1864826|7152|21568|14|25|8|5|185|342|5|56.50|122.04|15.86|530.90|79.30|282.50|610.20|3.17|0.00|268.45|79.30|82.47|347.75|350.92|-203.20| +2451528|21164|2451599|11281|10515|34131|6194|34210|75591|1864826|7152|21568|44|28|7|2|224|342|1|35.28|58.56|16.39|42.17|16.39|35.28|58.56|1.31|0.00|27.52|16.39|17.70|43.91|45.22|-18.89| +2451528|21164|2451610|10984|10515|34131|6194|34210|75591|1864826|7152|21568|44|22|11|5|166|342|47|65.05|103.42|102.38|48.88|4811.86|3057.35|4860.74|384.94|0.00|874.67|4811.86|5196.80|5686.53|6071.47|1754.51| +2451528|21164|2451551|17416|10515|34131|6194|34210|75591|1864826|7152|21568|16|10|13|3|20|342|17|26.27|49.38|16.78|554.20|285.26|446.59|839.46|11.41|0.00|234.94|285.26|296.67|520.20|531.61|-161.33| +2451528|21164|2451583|9445|10515|34131|6194|34210|75591|1864826|7152|21568|25|22|18|2|286|342|52|81.97|105.74|74.01|1649.96|3848.52|4262.44|5498.48|49.26|3232.75|1374.36|615.77|665.03|1990.13|2039.39|-3646.67| +2451528|21164|2451553|7879|10515|34131|6194|34210|75591|1864826|7152|21568|40|19|19|3|180|342|31|39.41|44.53|5.34|1214.89|165.54|1221.71|1380.43|14.89|0.00|345.03|165.54|180.43|510.57|525.46|-1056.17| +2451528|21164|2451607|2572|10515|34131|6194|34210|75591|1864826|7152|21568|44|13|8|2|56|342|77|89.51|164.69|153.16|887.81|11793.32|6892.27|12681.13|943.46|0.00|0.00|11793.32|12736.78|11793.32|12736.78|4901.05| +2451528|21164|2451611|368|10515|34131|6194|34210|75591|1864826|7152|21568|2|25|11|4|30|342|24|4.05|8.95|7.16|42.96|171.84|97.20|214.80|13.74|0.00|72.96|171.84|185.58|244.80|258.54|74.64| +2451528|21164|2451572|566|10515|34131|6194|34210|75591|1864826|7152|21568|14|19|5|5|175|342|89|74.84|124.23|17.39|9508.76|1547.71|6660.76|11056.47|0.00|0.00|663.05|1547.71|1547.71|2210.76|2210.76|-5113.05| +2451528|21164|2451602|15133|10515|34131|6194|34210|75591|1864826|7152|21568|22|16|13|5|284|342|86|82.24|99.51|82.59|1455.12|7102.74|7072.64|8557.86|568.21|0.00|2567.10|7102.74|7670.95|9669.84|10238.05|30.10| +2451528|21164|2451609|3772|10515|34131|6194|34210|75591|1864826|7152|21568|31|1|6|4|159|342|60|40.44|57.42|43.63|827.40|2617.80|2426.40|3445.20|130.89|0.00|757.80|2617.80|2748.69|3375.60|3506.49|191.40| +2451347|61733|2451391|15223|68676|1014988|2100|812|73836|805548|3348|43660|10|14|20|5|127|343|95|18.51|53.12|34.52|1767.00|3279.40|1758.45|5046.40|0.00|0.00|50.35|3279.40|3279.40|3329.75|3329.75|1520.95| +2451347|61733|2451376|7642|68676|1014988|2100|812|73836|805548|3348|43660|43|1|10|4|238|343|18|36.86|49.02|8.82|723.60|158.76|663.48|882.36|11.11|0.00|132.30|158.76|169.87|291.06|302.17|-504.72| +2451347|61733|2451362|10891|68676|1014988|2100|812|73836|805548|3348|43660|13|2|4|4|151|343|47|24.32|32.83|2.29|1435.38|107.63|1143.04|1543.01|5.38|0.00|709.70|107.63|113.01|817.33|822.71|-1035.41| +2451347|61733|2451388|14872|68676|1014988|2100|812|73836|805548|3348|43660|34|25|11|4|247|343|79|48.41|90.52|23.53|5292.21|1858.87|3824.39|7151.08|0.00|1338.38|1215.02|520.49|520.49|1735.51|1735.51|-3303.90| +2451347|61733|2451436|2335|68676|1014988|2100|812|73836|805548|3348|43660|7|19|3|4|102|343|23|52.61|115.21|10.36|2411.55|238.28|1210.03|2649.83|21.44|0.00|476.79|238.28|259.72|715.07|736.51|-971.75| +2451347|61733|2451458|9404|68676|1014988|2100|812|73836|805548|3348|43660|40|28|19|3|297|343|39|83.76|244.57|7.33|9252.36|285.87|3266.64|9538.23|5.71|0.00|0.00|285.87|291.58|285.87|291.58|-2980.77| +2451347|61733|2451386|1888|68676|1014988|2100|812|73836|805548|3348|43660|40|19|12|1|240|343|61|34.13|66.21|33.76|1979.45|2059.36|2081.93|4038.81|164.74|0.00|605.73|2059.36|2224.10|2665.09|2829.83|-22.57| +2451347|61733|2451411|15220|68676|1014988|2100|812|73836|805548|3348|43660|49|19|5|5|2|343|62|45.22|134.75|95.67|2422.96|5931.54|2803.64|8354.50|474.52|0.00|2339.26|5931.54|6406.06|8270.80|8745.32|3127.90| +2451347|61733|2451461|5954|68676|1014988|2100|812|73836|805548|3348|43660|50|14|10|5|217|343|74|12.84|35.31|33.89|105.08|2507.86|950.16|2612.94|150.47|0.00|495.80|2507.86|2658.33|3003.66|3154.13|1557.70| +2451347|61733|2451462|17528|68676|1014988|2100|812|73836|805548|3348|43660|14|25|8|3|238|343|68|91.23|224.42|71.81|10377.48|4883.08|6203.64|15260.56|0.00|0.00|3357.16|4883.08|4883.08|8240.24|8240.24|-1320.56| +2451347|61733|2451455|9262|68676|1014988|2100|812|73836|805548|3348|43660|52|26|8|2|296|343|76|21.51|37.21|27.90|707.56|2120.40|1634.76|2827.96|0.00|636.12|225.72|1484.28|1484.28|1710.00|1710.00|-150.48| +2451347|61733|2451457|9505|68676|1014988|2100|812|73836|805548|3348|43660|40|25|2|4|266|343|16|79.86|103.01|0.00|1648.16|0.00|1277.76|1648.16|0.00|0.00|0.00|0.00|0.00|0.00|0.00|-1277.76| +2451347|61733|2451385|16570|68676|1014988|2100|812|73836|805548|3348|43660|40|26|7|5|83|343|91|30.42|53.53|48.71|438.62|4432.61|2768.22|4871.23|0.00|0.00|97.37|4432.61|4432.61|4529.98|4529.98|1664.39| +2451868|64546|2451888|9547|99481|20848|6215|31182|92532|1325902|483|23450|8|1|10|1|125|344|20|8.48|12.38|6.56|116.40|131.20|169.60|247.60|10.49|0.00|47.00|131.20|141.69|178.20|188.69|-38.40| +2451868|64546|2451952|12779|99481|20848|6215|31182|92532|1325902|483|23450|14|14|5|5|255|344|30|45.75|46.20|31.41|443.70|942.30|1372.50|1386.00|0.00|0.00|651.30|942.30|942.30|1593.60|1593.60|-430.20| +2451868|64546|2451885|15029|99481|20848|6215|31182|92532|1325902|483|23450|8|19|5|2|122|344|18|54.70|122.52|6.12|2095.20|110.16|984.60|2205.36|0.47|62.79|220.50|47.37|47.84|267.87|268.34|-937.23| +2451868|64546|2451962|4523|99481|20848|6215|31182|92532|1325902|483|23450|31|2|18|2|13|344|33|24.10|57.84|20.82|1221.66|687.06|795.30|1908.72|27.48|0.00|0.00|687.06|714.54|687.06|714.54|-108.24| +2451868|64546|2451918|12389|99481|20848|6215|31182|92532|1325902|483|23450|37|5|10|4|38|344|5|86.75|186.51|151.07|177.20|755.35|433.75|932.55|22.66|0.00|102.55|755.35|778.01|857.90|880.56|321.60| +2451868|64546|2451951|1771|99481|20848|6215|31182|92532|1325902|483|23450|1|20|18|3|138|344|85|28.46|73.99|69.55|377.40|5911.75|2419.10|6289.15|532.05|0.00|2892.55|5911.75|6443.80|8804.30|9336.35|3492.65| +2451868|64546|2451874|4147|99481|20848|6215|31182|92532|1325902|483|23450|55|5|11|1|71|344|68|43.67|91.27|59.32|2172.60|4033.76|2969.56|6206.36|282.36|0.00|2854.64|4033.76|4316.12|6888.40|7170.76|1064.20| +2451868|64546|2451874|8117|99481|20848|6215|31182|92532|1325902|483|23450|41|7|4|2|170|344|64|5.37|5.37|4.29|69.12|274.56|343.68|343.68|13.04|13.72|44.16|260.84|273.88|305.00|318.04|-82.84| +2451868|64546|2451952|359|99481|20848|6215|31182|92532|1325902|483|23450|19|13|14|4|171|344|60|64.52|158.71|111.09|2857.20|6665.40|3871.20|9522.60|199.96|0.00|3332.40|6665.40|6865.36|9997.80|10197.76|2794.20| +2452187|71574|2452229|8891|17902|790454|253|39605|58209|1811558|1308|40660|57|17|9|4|9|345|9|96.84|166.56|164.89|15.03|1484.01|871.56|1499.04|0.00|0.00|509.67|1484.01|1484.01|1993.68|1993.68|612.45| +2452187|71574|2452252|2399|17902|790454|253|39605|58209|1811558|1308|40660|19|3|7|4|264|345|51|73.44|177.72|165.27|634.95|8428.77|3745.44|9063.72|337.15|0.00|452.88|8428.77|8765.92|8881.65|9218.80|4683.33| +2452187|71574|2452232|4577|17902|790454|253|39605|58209|1811558|1308|40660|15|21|7|5|103|345|86|4.84|7.45|4.39|263.16|377.54|416.24|640.70|0.30|347.33|115.24|30.21|30.51|145.45|145.75|-386.03| +2452187|71574|2452212|11951|17902|790454|253|39605|58209|1811558|1308|40660|51|5|10|4|151|345|22|96.85|165.61|127.51|838.20|2805.22|2130.70|3643.42|112.20|0.00|1712.26|2805.22|2917.42|4517.48|4629.68|674.52| +2452187|71574|2452301|621|17902|790454|253|39605|58209|1811558|1308|40660|27|13|10|3|76|345|7|28.18|75.52|53.61|153.37|375.27|197.26|528.64|33.77|0.00|58.10|375.27|409.04|433.37|467.14|178.01| +2452187|71574|2452192|10587|17902|790454|253|39605|58209|1811558|1308|40660|59|9|7|2|244|345|71|34.68|35.72|1.78|2409.74|126.38|2462.28|2536.12|11.37|0.00|329.44|126.38|137.75|455.82|467.19|-2335.90| +2452187|71574|2452262|7605|17902|790454|253|39605|58209|1811558|1308|40660|55|23|9|3|167|345|12|4.80|6.57|1.18|64.68|14.16|57.60|78.84|0.56|0.00|22.80|14.16|14.72|36.96|37.52|-43.44| +2452187|71574|2452283|14785|17902|790454|253|39605|58209|1811558|1308|40660|9|25|11|2|180|345|74|97.72|149.51|134.55|1107.04|9956.70|7231.28|11063.74|99.56|0.00|1437.82|9956.70|10056.26|11394.52|11494.08|2725.42| +2452187|71574|2452283|17849|17902|790454|253|39605|58209|1811558|1308|40660|1|7|10|3|239|345|2|27.80|58.38|38.53|39.70|77.06|55.60|116.76|0.77|0.00|4.66|77.06|77.83|81.72|82.49|21.46| +2452187|71574|2452217|1531|17902|790454|253|39605|58209|1811558|1308|40660|33|5|9|2|208|345|3|49.90|120.25|38.48|245.31|115.44|149.70|360.75|0.00|0.00|46.89|115.44|115.44|162.33|162.33|-34.26| +2452187|71574|2452277|17091|17902|790454|253|39605|58209|1811558|1308|40660|49|11|4|5|16|345|1|30.98|65.98|18.47|47.51|18.47|30.98|65.98|1.47|0.00|3.95|18.47|19.94|22.42|23.89|-12.51| +2452187|71574|2452268|2981|17902|790454|253|39605|58209|1811558|1308|40660|45|3|14|3|165|345|96|16.43|31.54|11.98|1877.76|1150.08|1577.28|3027.84|34.50|0.00|241.92|1150.08|1184.58|1392.00|1426.50|-427.20| +2452187|71574|2452248|2497|17902|790454|253|39605|58209|1811558|1308|40660|43|15|17|1|185|345|66|29.81|41.13|17.27|1574.76|1139.82|1967.46|2714.58|0.00|0.00|1221.00|1139.82|1139.82|2360.82|2360.82|-827.64| +2452187|71574|2452282|9335|17902|790454|253|39605|58209|1811558|1308|40660|21|23|10|3|96|345|13|86.07|222.92|193.94|376.74|2521.22|1118.91|2897.96|100.84|0.00|1246.05|2521.22|2622.06|3767.27|3868.11|1402.31| +2452187|71574|2452202|14933|17902|790454|253|39605|58209|1811558|1308|40660|39|11|5|1|167|345|14|36.59|57.44|7.46|699.72|104.44|512.26|804.16|1.04|0.00|96.46|104.44|105.48|200.90|201.94|-407.82| +2452270|57528|2452303|2937|10143|461700|1610|15567|96451|1117301|5791|33701|1|7|17|2|200|346|70|61.03|181.25|63.43|8247.40|4440.10|4272.10|12687.50|177.60|0.00|380.10|4440.10|4617.70|4820.20|4997.80|168.00| +2452270|57528|2452346|14773|10143|461700|1610|15567|96451|1117301|5791|33701|57|1|13|5|187|346|59|42.96|90.64|2.71|5187.87|159.89|2534.64|5347.76|14.39|0.00|1711.00|159.89|174.28|1870.89|1885.28|-2374.75| +2452270|57528|2452319|11363|10143|461700|1610|15567|96451|1117301|5791|33701|17|9|1|2|83|346|10|24.76|57.44|32.74|247.00|327.40|247.60|574.40|3.27|0.00|206.70|327.40|330.67|534.10|537.37|79.80| +2452270|57528|2452360|9205|10143|461700|1610|15567|96451|1117301|5791|33701|57|11|10|2|281|346|35|99.20|196.41|64.81|4606.00|2268.35|3472.00|6874.35|68.05|0.00|0.00|2268.35|2336.40|2268.35|2336.40|-1203.65| +2452270|57528|2452334|14305|10143|461700|1610|15567|96451|1117301|5791|33701|33|25|9|5|91|346|64|6.93|11.08|0.33|688.00|21.12|443.52|709.12|1.05|0.00|212.48|21.12|22.17|233.60|234.65|-422.40| +2452270|57528|2452286|2745|10143|461700|1610|15567|96451|1117301|5791|33701|9|15|8|3|91|346|64|54.90|81.25|43.06|2444.16|2755.84|3513.60|5200.00|13.22|2590.48|2080.00|165.36|178.58|2245.36|2258.58|-3348.24| +2452270|57528|2452376|17107|10143|461700|1610|15567|96451|1117301|5791|33701|27|11|1|3|238|346|54|47.13|135.26|102.79|1753.38|5550.66|2545.02|7304.04|318.60|999.11|2556.36|4551.55|4870.15|7107.91|7426.51|2006.53| +2452270|57528|2452338|12917|10143|461700|1610|15567|96451|1117301|5791|33701|15|27|20|1|122|346|71|88.94|91.60|84.27|520.43|5983.17|6314.74|6503.60|355.40|59.83|3056.55|5923.34|6278.74|8979.89|9335.29|-391.40| +2452270|57528|2452297|14375|10143|461700|1610|15567|96451|1117301|5791|33701|39|3|4|1|121|346|73|68.89|141.91|2.83|10152.84|206.59|5028.97|10359.43|2.06|0.00|3728.84|206.59|208.65|3935.43|3937.49|-4822.38| +2452270|57528|2452326|8659|10143|461700|1610|15567|96451|1117301|5791|33701|1|13|3|1|2|346|77|8.26|22.71|4.31|1416.80|331.87|636.02|1748.67|0.49|282.08|856.24|49.79|50.28|906.03|906.52|-586.23| +2452270|57528|2452329|5101|10143|461700|1610|15567|96451|1117301|5791|33701|19|11|12|3|241|346|85|91.52|96.09|24.02|6125.95|2041.70|7779.20|8167.65|40.83|0.00|2449.70|2041.70|2082.53|4491.40|4532.23|-5737.50| +2452270|57528|2452328|7839|10143|461700|1610|15567|96451|1117301|5791|33701|1|15|15|2|59|346|15|72.38|183.84|176.48|110.40|2647.20|1085.70|2757.60|79.41|0.00|385.95|2647.20|2726.61|3033.15|3112.56|1561.50| +2452270|57528|2452349|2077|10143|461700|1610|15567|96451|1117301|5791|33701|13|25|2|1|17|346|21|21.35|62.12|26.09|756.63|547.89|448.35|1304.52|32.87|0.00|182.49|547.89|580.76|730.38|763.25|99.54| +2452270|57528|2452334|2419|10143|461700|1610|15567|96451|1117301|5791|33701|29|29|8|1|271|346|35|60.41|78.53|20.41|2034.20|714.35|2114.35|2748.55|0.00|478.61|714.35|235.74|235.74|950.09|950.09|-1878.61| +2452595|46366|2452683|12297|84894|837613|732|30698|26332|1053161|2102|47221|43|27|6|3|80|347|78|98.73|136.24|9.53|9883.38|743.34|7700.94|10626.72|7.43|0.00|4781.40|743.34|750.77|5524.74|5532.17|-6957.60| +2452595|46366|2452632|9091|84894|837613|732|30698|26332|1053161|2102|47221|45|13|8|4|130|347|88|7.74|14.00|2.52|1010.24|221.76|681.12|1232.00|2.21|0.00|283.36|221.76|223.97|505.12|507.33|-459.36| +2452595|46366|2452604|10818|84894|837613|732|30698|26332|1053161|2102|47221|13|1|12|1|87|347|4|72.56|148.74|102.63|184.44|410.52|290.24|594.96|0.00|0.00|148.72|410.52|410.52|559.24|559.24|120.28| +2452595|46366|2452633|16437|84894|837613|732|30698|26332|1053161|2102|47221|24|15|4|2|153|347|61|74.80|189.24|130.57|3578.87|7964.77|4562.80|11543.64|107.52|6770.05|577.06|1194.72|1302.24|1771.78|1879.30|-3368.08| +2452595|46366|2452674|2580|84894|837613|732|30698|26332|1053161|2102|47221|21|25|8|5|95|347|57|79.83|212.34|208.09|242.25|11861.13|4550.31|12103.38|0.00|0.00|2299.38|11861.13|11861.13|14160.51|14160.51|7310.82| +2452595|46366|2452634|17557|84894|837613|732|30698|26332|1053161|2102|47221|36|15|15|5|232|347|22|40.97|102.42|31.75|1554.74|698.50|901.34|2253.24|62.86|0.00|292.82|698.50|761.36|991.32|1054.18|-202.84| +2452595|46366|2452697|6048|84894|837613|732|30698|26332|1053161|2102|47221|55|25|8|3|28|347|71|13.36|29.79|0.59|2073.20|41.89|948.56|2115.09|0.41|0.00|422.45|41.89|42.30|464.34|464.75|-906.67| +2452595|46366|2452617|4603|84894|837613|732|30698|26332|1053161|2102|47221|42|3|6|2|118|347|33|75.21|217.35|76.07|4662.24|2510.31|2481.93|7172.55|159.90|225.92|3370.95|2284.39|2444.29|5655.34|5815.24|-197.54| +2452595|46366|2452609|7563|84894|837613|732|30698|26332|1053161|2102|47221|19|3|4|1|38|347|32|32.04|89.07|51.66|1197.12|1653.12|1025.28|2850.24|55.54|859.62|912.00|793.50|849.04|1705.50|1761.04|-231.78| +2452595|46366|2452619|9823|84894|837613|732|30698|26332|1053161|2102|47221|27|1|8|5|142|347|64|80.39|110.13|58.36|3313.28|3735.04|5144.96|7048.32|261.45|0.00|0.00|3735.04|3996.49|3735.04|3996.49|-1409.92| +2452595|46366|2452709|2247|84894|837613|732|30698|26332|1053161|2102|47221|6|6|6|1|56|347|98|46.00|91.54|32.95|5741.82|3229.10|4508.00|8970.92|32.29|0.00|2152.08|3229.10|3261.39|5381.18|5413.47|-1278.90| +2452595|46366|2452605|13143|84894|837613|732|30698|26332|1053161|2102|47221|6|12|12|5|8|347|18|4.74|13.84|4.15|174.42|74.70|85.32|249.12|2.98|0.00|77.22|74.70|77.68|151.92|154.90|-10.62| +2452595|46366|2452669|6546|84894|837613|732|30698|26332|1053161|2102|47221|19|7|5|1|187|347|2|75.94|226.30|11.31|429.98|22.62|151.88|452.60|1.35|0.00|95.04|22.62|23.97|117.66|119.01|-129.26| +2452595|46366|2452638|12675|84894|837613|732|30698|26332|1053161|2102|47221|48|15|19|3|177|347|4|69.47|149.36|7.46|567.60|29.84|277.88|597.44|0.89|0.00|203.12|29.84|30.73|232.96|233.85|-248.04| +2451840|68005|2451884|329|85017|1387998|4932|46313|58321|20487|3962|5296|14|5|18|3|78|348|7|93.90|205.64|111.04|662.20|777.28|657.30|1439.48|54.40|0.00|604.52|777.28|831.68|1381.80|1436.20|119.98| +2451840|68005|2451872|9458|85017|1387998|4932|46313|58321|20487|3962|5296|50|11|6|3|43|348|2|71.19|157.32|47.19|220.26|94.38|142.38|314.64|0.94|0.00|47.18|94.38|95.32|141.56|142.50|-48.00| +2451840|68005|2451880|1183|85017|1387998|4932|46313|58321|20487|3962|5296|25|14|13|3|263|348|76|51.17|104.38|74.10|2301.28|5631.60|3888.92|7932.88|168.94|0.00|2300.52|5631.60|5800.54|7932.12|8101.06|1742.68| +2451840|68005|2451947|3553|85017|1387998|4932|46313|58321|20487|3962|5296|31|8|11|2|16|348|56|51.93|68.02|66.65|76.72|3732.40|2908.08|3809.12|261.26|0.00|1865.92|3732.40|3993.66|5598.32|5859.58|824.32| +2451840|68005|2451935|1517|85017|1387998|4932|46313|58321|20487|3962|5296|1|8|9|3|257|348|66|37.55|96.50|92.64|254.76|6114.24|2478.30|6369.00|244.56|0.00|1591.92|6114.24|6358.80|7706.16|7950.72|3635.94| +2451840|68005|2451870|17839|85017|1387998|4932|46313|58321|20487|3962|5296|5|25|16|1|156|348|19|65.31|141.06|14.10|2412.24|267.90|1240.89|2680.14|5.35|0.00|830.68|267.90|273.25|1098.58|1103.93|-972.99| +2451840|68005|2451892|3367|85017|1387998|4932|46313|58321|20487|3962|5296|35|7|10|1|232|348|70|81.18|208.63|37.55|11975.60|2628.50|5682.60|14604.10|236.56|0.00|4235.00|2628.50|2865.06|6863.50|7100.06|-3054.10| +2451840|68005|2451853|2123|85017|1387998|4932|46313|58321|20487|3962|5296|2|29|1|4|107|348|96|9.66|16.13|15.96|16.32|1532.16|927.36|1548.48|45.96|0.00|76.80|1532.16|1578.12|1608.96|1654.92|604.80| +2451840|68005|2451884|5426|85017|1387998|4932|46313|58321|20487|3962|5296|53|1|2|3|214|348|50|24.39|29.51|12.68|841.50|634.00|1219.50|1475.50|57.06|0.00|634.00|634.00|691.06|1268.00|1325.06|-585.50| +2451840|68005|2451865|7460|85017|1387998|4932|46313|58321|20487|3962|5296|55|26|3|4|175|348|57|56.00|138.88|79.16|3404.04|4512.12|3192.00|7916.16|90.24|0.00|0.00|4512.12|4602.36|4512.12|4602.36|1320.12| +2451840|68005|2451846|8120|85017|1387998|4932|46313|58321|20487|3962|5296|11|29|14|3|132|348|90|65.18|183.15|150.18|2967.30|13516.20|5866.20|16483.50|135.16|0.00|3461.40|13516.20|13651.36|16977.60|17112.76|7650.00| +2451840|68005|2451901|1907|85017|1387998|4932|46313|58321|20487|3962|5296|43|13|2|2|209|348|1|64.04|159.45|54.21|105.24|54.21|64.04|159.45|3.25|0.00|33.48|54.21|57.46|87.69|90.94|-9.83| +2452003|80802|2452012|7743|12487|1348163|1886|25562|70307|1544869|1484|48760|57|7|9|4|139|349|21|9.82|11.58|7.06|94.92|148.26|206.22|243.18|1.34|128.98|14.49|19.28|20.62|33.77|35.11|-186.94| +2452003|80802|2452037|15433|12487|1348163|1886|25562|70307|1544869|1484|48760|33|19|3|1|229|349|67|90.13|260.47|135.44|8377.01|9074.48|6038.71|17451.49|362.97|0.00|6456.79|9074.48|9437.45|15531.27|15894.24|3035.77| +2452003|80802|2452098|6669|12487|1348163|1886|25562|70307|1544869|1484|48760|51|15|20|4|51|349|88|68.10|181.82|34.54|12960.64|3039.52|5992.80|16000.16|243.46|334.34|7519.60|2705.18|2948.64|10224.78|10468.24|-3287.62| +2452003|80802|2452028|4317|12487|1348163|1886|25562|70307|1544869|1484|48760|55|15|18|3|248|349|96|68.42|142.99|121.54|2059.20|11667.84|6568.32|13727.04|466.71|0.00|4941.12|11667.84|12134.55|16608.96|17075.67|5099.52| +2452003|80802|2452021|8959|12487|1348163|1886|25562|70307|1544869|1484|48760|39|15|6|2|295|349|15|70.66|139.20|9.74|1941.90|146.10|1059.90|2088.00|8.76|0.00|396.60|146.10|154.86|542.70|551.46|-913.80| +2452003|80802|2452033|11239|12487|1348163|1886|25562|70307|1544869|1484|48760|43|17|19|1|233|349|18|86.21|234.49|182.90|928.62|3292.20|1551.78|4220.82|230.45|0.00|1857.06|3292.20|3522.65|5149.26|5379.71|1740.42| +2452003|80802|2452039|4693|12487|1348163|1886|25562|70307|1544869|1484|48760|53|13|20|3|173|349|13|64.91|90.87|14.53|992.42|188.89|843.83|1181.31|11.33|0.00|507.91|188.89|200.22|696.80|708.13|-654.94| +2452003|80802|2452039|4311|12487|1348163|1886|25562|70307|1544869|1484|48760|15|15|8|1|181|349|66|32.33|44.61|21.41|1531.20|1413.06|2133.78|2944.26|98.91|0.00|205.92|1413.06|1511.97|1618.98|1717.89|-720.72| +2452003|80802|2452066|17355|12487|1348163|1886|25562|70307|1544869|1484|48760|39|7|1|3|175|349|55|57.87|63.07|9.46|2948.55|520.30|3182.85|3468.85|41.62|0.00|69.30|520.30|561.92|589.60|631.22|-2662.55| +2452003|80802|2452022|15737|12487|1348163|1886|25562|70307|1544869|1484|48760|31|23|18|4|240|349|55|22.10|22.98|0.91|1213.85|50.05|1215.50|1263.90|1.00|0.00|37.40|50.05|51.05|87.45|88.45|-1165.45| +2452003|80802|2452020|109|12487|1348163|1886|25562|70307|1544869|1484|48760|47|11|8|5|102|349|63|35.26|105.42|31.62|4649.40|1992.06|2221.38|6641.46|0.00|0.00|3320.73|1992.06|1992.06|5312.79|5312.79|-229.32| +2452003|80802|2452042|13921|12487|1348163|1886|25562|70307|1544869|1484|48760|23|25|4|3|159|349|11|17.62|43.34|41.60|19.14|457.60|193.82|476.74|13.72|0.00|80.96|457.60|471.32|538.56|552.28|263.78| +2452003|80802|2452016|4585|12487|1348163|1886|25562|70307|1544869|1484|48760|55|23|19|3|220|349|87|50.34|92.12|0.00|8014.44|0.00|4379.58|8014.44|0.00|0.00|3766.23|0.00|0.00|3766.23|3766.23|-4379.58| +2451809|17951|2451923|15661|52195|598688|1981|37627|60728|18653|659|12752|38|17|1|5|29|350|43|75.20|169.95|161.45|365.50|6942.35|3233.60|7307.85|624.81|0.00|803.67|6942.35|7567.16|7746.02|8370.83|3708.75| +2451809|17951|2451889|17465|52195|598688|1981|37627|60728|18653|659|12752|50|19|7|1|255|350|12|91.51|203.15|34.53|2023.44|414.36|1098.12|2437.80|8.28|0.00|292.44|414.36|422.64|706.80|715.08|-683.76| +2451809|17951|2451825|9011|52195|598688|1981|37627|60728|18653|659|12752|26|14|12|3|267|350|54|5.67|12.75|3.18|516.78|171.72|306.18|688.50|13.73|0.00|116.64|171.72|185.45|288.36|302.09|-134.46| +2451809|17951|2451865|5287|52195|598688|1981|37627|60728|18653|659|12752|47|19|19|5|14|350|24|72.13|147.86|136.03|283.92|3264.72|1731.12|3548.64|223.96|65.29|567.60|3199.43|3423.39|3767.03|3990.99|1468.31| +2451809|17951|2451837|15050|52195|598688|1981|37627|60728|18653|659|12752|43|8|5|3|111|350|37|90.18|226.35|49.79|6532.72|1842.23|3336.66|8374.95|5.15|1584.31|2847.15|257.92|263.07|3105.07|3110.22|-3078.74| +2451809|17951|2451879|6242|52195|598688|1981|37627|60728|18653|659|12752|56|11|13|5|290|350|72|5.55|16.37|11.62|342.00|836.64|399.60|1178.64|42.16|309.55|459.36|527.09|569.25|986.45|1028.61|127.49| +2451809|17951|2451906|16166|52195|598688|1981|37627|60728|18653|659|12752|44|25|20|2|154|350|66|12.99|25.07|8.27|1108.80|545.82|857.34|1654.62|10.91|0.00|413.16|545.82|556.73|958.98|969.89|-311.52| +2451809|17951|2451881|9085|52195|598688|1981|37627|60728|18653|659|12752|44|14|16|4|129|350|63|4.09|4.29|2.91|86.94|183.33|257.67|270.27|5.49|0.00|113.40|183.33|188.82|296.73|302.22|-74.34| +2451809|17951|2451874|7385|52195|598688|1981|37627|60728|18653|659|12752|50|11|9|2|112|350|71|19.45|31.31|11.58|1400.83|822.18|1380.95|2223.01|16.44|0.00|621.96|822.18|838.62|1444.14|1460.58|-558.77| +2451809|17951|2451892|3511|52195|598688|1981|37627|60728|18653|659|12752|50|29|4|3|252|350|15|3.37|4.98|0.64|65.10|9.60|50.55|74.70|0.00|0.00|2.85|9.60|9.60|12.45|12.45|-40.95| +2451474|16506|2451492|16213|31298|1017447|6796|45745|31298|1017447|6796|45745|50|13|1|5|176|351|42|72.59|111.06|48.86|2612.40|2052.12|3048.78|4664.52|102.60|0.00|1026.06|2052.12|2154.72|3078.18|3180.78|-996.66| +2451474|16506|2451486|10526|31298|1017447|6796|45745|31298|1017447|6796|45745|58|16|11|3|142|351|81|96.55|256.82|74.47|14770.35|6032.07|7820.55|20802.42|301.60|0.00|415.53|6032.07|6333.67|6447.60|6749.20|-1788.48| +2451474|16506|2451491|16618|31298|1017447|6796|45745|31298|1017447|6796|45745|2|16|3|5|284|351|1|49.28|123.69|56.89|66.80|56.89|49.28|123.69|3.98|0.00|4.94|56.89|60.87|61.83|65.81|7.61| +2451474|16506|2451524|10699|31298|1017447|6796|45745|31298|1017447|6796|45745|38|8|10|5|170|351|26|50.06|55.06|29.18|672.88|758.68|1301.56|1431.56|43.01|144.14|42.90|614.54|657.55|657.44|700.45|-687.02| +2451474|16506|2451551|15706|31298|1017447|6796|45745|31298|1017447|6796|45745|31|4|20|5|287|351|51|77.33|99.75|99.75|0.00|5087.25|3943.83|5087.25|152.61|0.00|1780.41|5087.25|5239.86|6867.66|7020.27|1143.42| +2451474|16506|2451510|9794|31298|1017447|6796|45745|31298|1017447|6796|45745|43|13|2|5|255|351|60|86.55|238.87|69.27|10176.00|4156.20|5193.00|14332.20|166.24|0.00|5446.20|4156.20|4322.44|9602.40|9768.64|-1036.80| +2451474|16506|2451585|5989|31298|1017447|6796|45745|31298|1017447|6796|45745|4|7|8|4|60|351|86|3.66|10.79|3.77|603.72|324.22|314.76|927.94|6.48|0.00|417.10|324.22|330.70|741.32|747.80|9.46| +2451474|16506|2451546|10687|31298|1017447|6796|45745|31298|1017447|6796|45745|46|22|17|2|140|351|28|52.20|56.89|22.18|971.88|621.04|1461.60|1592.92|37.26|0.00|206.92|621.04|658.30|827.96|865.22|-840.56| +2451474|16506|2451547|13402|31298|1017447|6796|45745|31298|1017447|6796|45745|58|1|12|3|124|351|58|25.30|29.34|28.45|51.62|1650.10|1467.40|1701.72|12.70|1468.58|561.44|181.52|194.22|742.96|755.66|-1285.88| +2451474|16506|2451593|16807|31298|1017447|6796|45745|31298|1017447|6796|45745|14|26|10|1|85|351|81|4.30|5.16|0.61|368.55|49.41|348.30|417.96|0.49|0.00|95.58|49.41|49.90|144.99|145.48|-298.89| +2451474|16506|2451587|16297|31298|1017447|6796|45745|31298|1017447|6796|45745|32|22|7|4|64|351|83|86.56|253.62|220.64|2737.34|18313.12|7184.48|21050.46|1465.04|0.00|4420.58|18313.12|19778.16|22733.70|24198.74|11128.64| +2451474|16506|2451507|11278|31298|1017447|6796|45745|31298|1017447|6796|45745|22|2|9|1|203|351|10|8.52|15.76|2.36|134.00|23.60|85.20|157.60|0.70|0.00|63.00|23.60|24.30|86.60|87.30|-61.60| +2451474|16506|2451523|11338|31298|1017447|6796|45745|31298|1017447|6796|45745|8|26|3|2|221|351|61|46.69|46.69|14.94|1936.75|911.34|2848.09|2848.09|11.66|765.52|1423.74|145.82|157.48|1569.56|1581.22|-2702.27| +2450919|36315|2451009|8978|10165|1155863|5114|15212|10165|1155863|5114|15212|2|2|2|5|85|352|32|96.40|123.39|81.43|1342.72|2605.76|3084.80|3948.48|0.00|1980.37|1658.24|625.39|625.39|2283.63|2283.63|-2459.41| +2450919|36315|2450957|7609|10165|1155863|5114|15212|10165|1155863|5114|15212|22|26|16|5|27|352|3|51.08|124.12|83.16|122.88|249.48|153.24|372.36|4.98|0.00|171.27|249.48|254.46|420.75|425.73|96.24| +2450919|36315|2450979|3625|10165|1155863|5114|15212|10165|1155863|5114|15212|44|20|9|2|275|352|99|47.39|55.44|38.25|1701.81|3786.75|4691.61|5488.56|75.73|0.00|1096.92|3786.75|3862.48|4883.67|4959.40|-904.86| +2450919|36315|2451035|14036|10165|1155863|5114|15212|10165|1155863|5114|15212|4|4|3|3|123|352|9|64.26|93.81|44.09|447.48|396.81|578.34|844.29|27.77|0.00|329.22|396.81|424.58|726.03|753.80|-181.53| +2450919|36315|2451032|392|10165|1155863|5114|15212|10165|1155863|5114|15212|58|13|7|2|157|352|80|95.65|139.64|50.27|7149.60|4021.60|7652.00|11171.20|281.51|0.00|558.40|4021.60|4303.11|4580.00|4861.51|-3630.40| +2450919|36315|2450943|4921|10165|1155863|5114|15212|10165|1155863|5114|15212|7|8|16|2|128|352|19|69.61|109.28|68.84|768.36|1307.96|1322.59|2076.32|117.71|0.00|560.50|1307.96|1425.67|1868.46|1986.17|-14.63| +2450919|36315|2451022|3268|10165|1155863|5114|15212|10165|1155863|5114|15212|49|8|8|3|51|352|77|40.76|77.03|44.67|2491.72|3439.59|3138.52|5931.31|240.77|0.00|1541.54|3439.59|3680.36|4981.13|5221.90|301.07| +2450919|36315|2450999|9182|10165|1155863|5114|15212|10165|1155863|5114|15212|49|2|9|4|136|352|100|22.74|22.96|10.10|1286.00|1010.00|2274.00|2296.00|16.96|767.60|321.00|242.40|259.36|563.40|580.36|-2031.60| +2450919|36315|2451027|8674|10165|1155863|5114|15212|10165|1155863|5114|15212|44|28|10|2|277|352|65|46.38|98.78|25.68|4751.50|1669.20|3014.70|6420.70|66.76|0.00|384.80|1669.20|1735.96|2054.00|2120.76|-1345.50| +2450919|36315|2450955|12571|10165|1155863|5114|15212|10165|1155863|5114|15212|52|13|20|3|23|352|25|1.06|3.05|2.04|25.25|51.00|26.50|76.25|4.59|0.00|17.50|51.00|55.59|68.50|73.09|24.50| +2450919|36315|2450944|5950|10165|1155863|5114|15212|10165|1155863|5114|15212|28|26|13|2|78|352|35|27.96|28.79|12.09|584.50|423.15|978.60|1007.65|0.00|0.00|322.35|423.15|423.15|745.50|745.50|-555.45| +2450919|36315|2450959|5113|10165|1155863|5114|15212|10165|1155863|5114|15212|16|26|19|2|172|352|95|4.29|8.02|4.65|320.15|441.75|407.55|761.90|17.67|0.00|76.00|441.75|459.42|517.75|535.42|34.20| +2450919|36315|2451006|2042|10165|1155863|5114|15212|10165|1155863|5114|15212|40|1|1|1|68|352|29|45.29|49.81|38.35|332.34|1112.15|1313.41|1444.49|100.09|0.00|534.18|1112.15|1212.24|1646.33|1746.42|-201.26| +2450919|36315|2450933|2995|10165|1155863|5114|15212|10165|1155863|5114|15212|26|20|18|5|242|352|52|50.00|92.50|90.65|96.20|4713.80|2600.00|4810.00|329.96|0.00|1971.84|4713.80|5043.76|6685.64|7015.60|2113.80| +2451128|63318|2451200|15374|32576|1242014|1852|2490|15989|1005842|4851|12689|37|25|7|1|115|353|68|55.86|156.96|50.22|7258.32|3414.96|3798.48|10673.28|307.34|0.00|3521.72|3414.96|3722.30|6936.68|7244.02|-383.52| +2451128|63318|2451142|4546|32576|1242014|1852|2490|15989|1005842|4851|12689|46|1|18|5|22|353|26|96.71|274.65|164.79|2856.36|4284.54|2514.46|7140.90|299.91|0.00|1713.66|4284.54|4584.45|5998.20|6298.11|1770.08| +2451128|63318|2451205|12421|32576|1242014|1852|2490|15989|1005842|4851|12689|55|28|9|1|163|353|81|99.32|277.10|246.61|2469.69|19975.41|8044.92|22445.10|1398.27|0.00|2019.33|19975.41|21373.68|21994.74|23393.01|11930.49| +2451128|63318|2451155|2714|32576|1242014|1852|2490|15989|1005842|4851|12689|38|26|8|2|56|353|22|97.63|154.25|134.19|441.32|2952.18|2147.86|3393.50|236.17|0.00|1459.04|2952.18|3188.35|4411.22|4647.39|804.32| +2451128|63318|2451217|16694|32576|1242014|1852|2490|15989|1005842|4851|12689|4|8|7|4|281|353|87|36.28|66.02|7.26|5112.12|631.62|3156.36|5743.74|12.63|0.00|2354.22|631.62|644.25|2985.84|2998.47|-2524.74| +2451128|63318|2451245|7694|32576|1242014|1852|2490|15989|1005842|4851|12689|25|19|7|4|31|353|24|66.89|168.56|109.56|1416.00|2629.44|1605.36|4045.44|78.88|0.00|444.96|2629.44|2708.32|3074.40|3153.28|1024.08| +2451128|63318|2451234|43|32576|1242014|1852|2490|15989|1005842|4851|12689|46|7|1|1|281|353|51|20.53|38.18|35.12|156.06|1791.12|1047.03|1947.18|89.55|0.00|564.57|1791.12|1880.67|2355.69|2445.24|744.09| +2451128|63318|2451136|8366|32576|1242014|1852|2490|15989|1005842|4851|12689|25|19|14|1|225|353|87|80.36|162.32|84.40|6779.04|7342.80|6991.32|14121.84|48.46|2496.55|4095.09|4846.25|4894.71|8941.34|8989.80|-2145.07| +2451128|63318|2451146|10465|32576|1242014|1852|2490|15989|1005842|4851|12689|31|13|18|1|77|353|85|35.90|45.95|33.08|1093.95|2811.80|3051.50|3905.75|56.23|0.00|1288.60|2811.80|2868.03|4100.40|4156.63|-239.70| +2451128|63318|2451221|14029|32576|1242014|1852|2490|15989|1005842|4851|12689|31|4|10|2|24|353|44|17.85|48.19|43.37|212.08|1908.28|785.40|2120.36|76.33|0.00|21.12|1908.28|1984.61|1929.40|2005.73|1122.88| +2451128|63318|2451130|11840|32576|1242014|1852|2490|15989|1005842|4851|12689|22|13|2|2|22|353|85|54.65|113.67|35.23|6667.40|2994.55|4645.25|9661.95|269.50|0.00|1159.40|2994.55|3264.05|4153.95|4423.45|-1650.70| +2451965|33006|2452033|16527|55697|1219995|2693|48544|25463|528183|591|16889|55|29|12|2|29|354|40|21.71|48.63|26.26|894.80|1050.40|868.40|1945.20|0.00|0.00|330.40|1050.40|1050.40|1380.80|1380.80|182.00| +2451965|33006|2452004|12213|55697|1219995|2693|48544|25463|528183|591|16889|39|1|19|5|101|354|89|38.12|40.02|12.40|2458.18|1103.60|3392.68|3561.78|44.14|0.00|961.20|1103.60|1147.74|2064.80|2108.94|-2289.08| +2451965|33006|2452056|3181|55697|1219995|2693|48544|25463|528183|591|16889|9|1|17|3|169|354|17|47.34|56.33|52.95|57.46|900.15|804.78|957.61|41.04|387.06|469.20|513.09|554.13|982.29|1023.33|-291.69| +2451965|33006|2452061|8465|55697|1219995|2693|48544|25463|528183|591|16889|37|5|17|1|255|354|28|76.68|213.17|108.71|2924.88|3043.88|2147.04|5968.76|243.51|0.00|417.76|3043.88|3287.39|3461.64|3705.15|896.84| +2451965|33006|2452025|6141|55697|1219995|2693|48544|25463|528183|591|16889|47|9|9|3|227|354|99|84.08|205.99|39.13|16519.14|3873.87|8323.92|20393.01|271.17|0.00|4689.63|3873.87|4145.04|8563.50|8834.67|-4450.05| +2451965|33006|2452058|6787|55697|1219995|2693|48544|25463|528183|591|16889|53|7|11|3|272|354|97|13.11|24.77|0.99|2306.66|96.03|1271.67|2402.69|1.92|0.00|143.56|96.03|97.95|239.59|241.51|-1175.64| +2451965|33006|2452060|4671|55697|1219995|2693|48544|25463|528183|591|16889|17|27|9|4|239|354|38|57.45|129.26|18.09|4224.46|687.42|2183.10|4911.88|18.35|75.61|1571.68|611.81|630.16|2183.49|2201.84|-1571.29| +2451965|33006|2451980|2883|55697|1219995|2693|48544|25463|528183|591|16889|11|5|20|4|211|354|54|14.46|14.60|1.31|717.66|70.74|780.84|788.40|2.82|0.00|46.98|70.74|73.56|117.72|120.54|-710.10| +2452277|53751|2452357|6465|88365|1574782|5199|4416|14568|895747|5267|42484|1|6|5|2|9|355|17|3.27|8.99|8.54|7.65|145.18|55.59|152.83|2.90|0.00|62.56|145.18|148.08|207.74|210.64|89.59| +2452277|53751|2452326|2667|88365|1574782|5199|4416|14568|895747|5267|42484|25|21|6|4|102|355|74|85.63|234.62|199.42|2604.80|14757.08|6336.62|17361.88|590.28|0.00|4514.00|14757.08|15347.36|19271.08|19861.36|8420.46| +2452277|53751|2452384|9810|88365|1574782|5199|4416|14568|895747|5267|42484|37|12|20|2|281|355|33|28.29|78.36|53.28|827.64|1758.24|933.57|2585.88|17.58|0.00|930.60|1758.24|1775.82|2688.84|2706.42|824.67| +2452277|53751|2452379|541|88365|1574782|5199|4416|14568|895747|5267|42484|13|30|20|3|293|355|71|36.65|102.25|55.21|3339.84|3919.91|2602.15|7259.75|51.74|1332.76|1306.40|2587.15|2638.89|3893.55|3945.29|-15.00| +2452277|53751|2452361|1599|88365|1574782|5199|4416|14568|895747|5267|42484|31|27|9|4|292|355|17|59.87|83.81|1.67|1396.38|28.39|1017.79|1424.77|1.70|0.00|313.31|28.39|30.09|341.70|343.40|-989.40| +2452277|53751|2452391|7411|88365|1574782|5199|4416|14568|895747|5267|42484|45|7|18|4|120|355|34|56.64|73.63|24.29|1677.56|825.86|1925.76|2503.42|8.25|0.00|926.16|825.86|834.11|1752.02|1760.27|-1099.90| +2452277|53751|2452396|8707|88365|1574782|5199|4416|14568|895747|5267|42484|54|7|14|1|101|355|83|47.60|71.40|38.55|2726.55|3199.65|3950.80|5926.20|95.98|0.00|1244.17|3199.65|3295.63|4443.82|4539.80|-751.15| +2452277|53751|2452278|11581|88365|1574782|5199|4416|14568|895747|5267|42484|36|7|7|3|149|355|57|21.11|48.34|17.40|1763.58|991.80|1203.27|2755.38|6.34|674.42|936.51|317.38|323.72|1253.89|1260.23|-885.89| +2452277|53751|2452391|2889|88365|1574782|5199|4416|14568|895747|5267|42484|51|21|6|4|109|355|82|48.57|93.25|37.30|4587.90|3058.60|3982.74|7646.50|0.00|0.00|1681.82|3058.60|3058.60|4740.42|4740.42|-924.14| +2452277|53751|2452324|2505|88365|1574782|5199|4416|14568|895747|5267|42484|3|6|14|2|193|355|59|81.56|138.65|108.14|1800.09|6380.26|4812.04|8180.35|116.12|574.22|2290.38|5806.04|5922.16|8096.42|8212.54|994.00| +2452277|53751|2452381|7651|88365|1574782|5199|4416|14568|895747|5267|42484|1|21|11|1|251|355|78|59.34|124.02|70.69|4159.74|5513.82|4628.52|9673.56|330.82|0.00|2998.32|5513.82|5844.64|8512.14|8842.96|885.30| +2451502|77201|2451560|12914|55386|519286|1586|37486|55386|519286|1586|37486|37|19|11|4|80|356|28|70.13|167.61|56.98|3097.64|1595.44|1963.64|4693.08|15.95|0.00|844.48|1595.44|1611.39|2439.92|2455.87|-368.20| +2451502|77201|2451577|12847|55386|519286|1586|37486|55386|519286|1586|37486|26|10|15|4|236|356|10|19.31|53.87|22.62|312.50|226.20|193.10|538.70|4.52|0.00|70.00|226.20|230.72|296.20|300.72|33.10| +2451502|77201|2451620|10366|55386|519286|1586|37486|55386|519286|1586|37486|1|8|12|1|280|356|78|97.73|186.66|39.19|11502.66|3056.82|7622.94|14559.48|183.40|0.00|4804.02|3056.82|3240.22|7860.84|8044.24|-4566.12| +2451502|77201|2451544|15446|55386|519286|1586|37486|55386|519286|1586|37486|50|20|16|1|243|356|34|56.38|133.62|129.61|136.34|4406.74|1916.92|4543.08|308.47|0.00|954.04|4406.74|4715.21|5360.78|5669.25|2489.82| +2451502|77201|2451504|914|55386|519286|1586|37486|55386|519286|1586|37486|7|7|14|2|267|356|31|52.99|102.80|39.06|1975.94|1210.86|1642.69|3186.80|72.65|0.00|1019.59|1210.86|1283.51|2230.45|2303.10|-431.83| +2451502|77201|2451550|2299|55386|519286|1586|37486|55386|519286|1586|37486|25|28|7|1|28|356|87|62.33|103.46|70.35|2880.57|6120.45|5422.71|9001.02|122.40|0.00|4140.33|6120.45|6242.85|10260.78|10383.18|697.74| +2451502|77201|2451569|6205|55386|519286|1586|37486|55386|519286|1586|37486|40|10|1|1|201|356|59|24.41|42.71|10.67|1890.36|629.53|1440.19|2519.89|50.36|0.00|554.01|629.53|679.89|1183.54|1233.90|-810.66| +2451502|77201|2451521|3652|55386|519286|1586|37486|55386|519286|1586|37486|34|13|3|4|48|356|58|19.03|43.19|10.36|1904.14|600.88|1103.74|2505.02|30.04|0.00|0.00|600.88|630.92|600.88|630.92|-502.86| +2451502|77201|2451557|10957|55386|519286|1586|37486|55386|519286|1586|37486|16|28|1|4|130|356|45|93.03|129.31|11.63|5295.60|523.35|4186.35|5818.95|10.15|15.70|1454.40|507.65|517.80|1962.05|1972.20|-3678.70| +2451546|67441|2451560|4805|55988|516535|1082|25629|88734|617977|4517|35464|19|7|5|1|187|357|70|68.67|123.60|98.88|1730.40|6921.60|4806.90|8652.00|205.57|69.21|3546.90|6852.39|7057.96|10399.29|10604.86|2045.49| +2451546|67441|2451557|5095|55988|516535|1082|25629|88734|617977|4517|35464|26|29|16|2|288|357|87|1.08|2.71|1.57|99.18|136.59|93.96|235.77|4.09|0.00|8.70|136.59|140.68|145.29|149.38|42.63| +2451546|67441|2451557|4501|55988|516535|1082|25629|88734|617977|4517|35464|11|17|12|3|71|357|25|43.48|48.26|12.06|905.00|301.50|1087.00|1206.50|27.13|0.00|72.25|301.50|328.63|373.75|400.88|-785.50| +2451546|67441|2451623|13777|55988|516535|1082|25629|88734|617977|4517|35464|26|25|12|1|130|357|67|39.71|67.50|10.12|3844.46|678.04|2660.57|4522.50|20.34|0.00|2080.35|678.04|698.38|2758.39|2778.73|-1982.53| +2451546|67441|2451588|15650|55988|516535|1082|25629|88734|617977|4517|35464|53|14|12|3|147|357|4|61.53|107.06|62.09|179.88|248.36|246.12|428.24|4.96|0.00|141.28|248.36|253.32|389.64|394.60|2.24| +2451546|67441|2451650|15704|55988|516535|1082|25629|88734|617977|4517|35464|19|13|11|5|229|357|11|76.36|87.05|84.43|28.82|928.73|839.96|957.55|55.72|0.00|335.06|928.73|984.45|1263.79|1319.51|88.77| +2451546|67441|2451561|3737|55988|516535|1082|25629|88734|617977|4517|35464|59|26|1|5|289|357|100|2.56|3.91|0.58|333.00|58.00|256.00|391.00|5.22|0.00|187.00|58.00|63.22|245.00|250.22|-198.00| +2451546|67441|2451663|15788|55988|516535|1082|25629|88734|617977|4517|35464|38|8|3|3|73|357|73|60.40|73.08|2.92|5121.68|213.16|4409.20|5334.84|4.26|0.00|1866.61|213.16|217.42|2079.77|2084.03|-4196.04| +2451546|67441|2451660|13550|55988|516535|1082|25629|88734|617977|4517|35464|43|14|17|3|252|357|97|50.77|61.43|34.40|2621.91|3336.80|4924.69|5958.71|166.84|0.00|834.20|3336.80|3503.64|4171.00|4337.84|-1587.89| +2451449|51068|2451506|1357|31609|173598|5187|37929|26631|279393|5496|24238|31|20|16|1|229|358|75|36.45|75.81|40.17|2673.00|3012.75|2733.75|5685.75|0.00|0.00|1875.75|3012.75|3012.75|4888.50|4888.50|279.00| +2451449|51068||572||173598||37929|26631|279393||24238|43|25||||358||27.21|57.68|23.64|||598.62|1268.96||374.45||||691.23|694.14|-452.99| +2451449|51068|2451534|13006|31609|173598|5187|37929|26631|279393|5496|24238|44|14|13|5|150|358|49|19.30|45.35|12.24|1622.39|599.76|945.70|2222.15|0.54|581.76|133.28|18.00|18.54|151.28|151.82|-927.70| +2451449|51068|2451508|15260|31609|173598|5187|37929|26631|279393|5496|24238|37|16|19|4|129|358|28|7.17|8.17|1.30|192.36|36.40|200.76|228.76|3.27|0.00|77.56|36.40|39.67|113.96|117.23|-164.36| +2451449|51068|2451542|17144|31609|173598|5187|37929|26631|279393|5496|24238|40|26|17|2|120|358|96|31.26|55.64|12.79|4113.60|1227.84|3000.96|5341.44|85.94|0.00|480.00|1227.84|1313.78|1707.84|1793.78|-1773.12| +2451449|51068|2451517|1510|31609|173598|5187|37929|26631|279393|5496|24238|58|26|1|5|169|358|46|60.03|123.06|78.75|2038.26|3622.50|2761.38|5660.76|181.12|0.00|2320.70|3622.50|3803.62|5943.20|6124.32|861.12| +2451449|51068|2451469|661|31609|173598|5187|37929|26631|279393|5496|24238|31|14|10|2|224|358|92|96.75|272.83|272.83|0.00|25100.36|8901.00|25100.36|1757.02|0.00|10541.36|25100.36|26857.38|35641.72|37398.74|16199.36| +2451449|51068|2451496|5282|31609|173598|5187|37929|26631|279393|5496|24238|28|16|3|4|178|358|46|3.24|5.05|4.29|34.96|197.34|149.04|232.30|15.78|0.00|99.82|197.34|213.12|297.16|312.94|48.30| +2451449|51068|2451497|2192|31609|173598|5187|37929|26631|279393|5496|24238|50|13|11|2|236|358|75|75.79|147.79|93.10|4101.75|6982.50|5684.25|11084.25|89.37|5865.30|4101.00|1117.20|1206.57|5218.20|5307.57|-4567.05| +2451128|42270|2451172|7294|62051|637343|4635|28327|50234|844748|5803|16695|49|25|4|5|83|359|67|22.93|42.42|0.84|2785.86|56.28|1536.31|2842.14|5.06|0.00|767.15|56.28|61.34|823.43|828.49|-1480.03| +2451128|42270|2451177|16352|62051|637343|4635|28327|50234|844748|5803|16695|8|20|20|1|22|359|23|56.16|119.62|69.37|1155.75|1595.51|1291.68|2751.26|111.68|0.00|1210.49|1595.51|1707.19|2806.00|2917.68|303.83| +2451128|42270|2451209|16202|62051|637343|4635|28327|50234|844748|5803|16695|55|1|4|2|137|359|22|57.84|132.45|109.93|495.44|2418.46|1272.48|2913.90|145.10|0.00|1340.24|2418.46|2563.56|3758.70|3903.80|1145.98| +2451128|42270|2451201|14950|62051|637343|4635|28327|50234|844748|5803|16695|58|25|16|4|292|359|45|51.60|148.09|79.96|3065.85|3598.20|2322.00|6664.05|179.91|0.00|3331.80|3598.20|3778.11|6930.00|7109.91|1276.20| +2451128|42270|2451168|15538|62051|637343|4635|28327|50234|844748|5803|16695|19|7|1|5|189|359|1|64.90|160.30|72.13|88.17|72.13|64.90|160.30|4.32|0.00|70.53|72.13|76.45|142.66|146.98|7.23| +2451128|42270|2451199|3907|62051|637343|4635|28327|50234|844748|5803|16695|34|28|2|4|246|359|48|60.92|141.94|55.35|4156.32|2656.80|2924.16|6813.12|0.00|0.00|2043.84|2656.80|2656.80|4700.64|4700.64|-267.36| +2451128|42270|2451219|8113|62051|637343|4635|28327|50234|844748|5803|16695|28|4|8|1|4|359|2|63.63|168.61|8.43|320.36|16.86|127.26|337.22|1.34|0.00|33.72|16.86|18.20|50.58|51.92|-110.40| +2451128|42270|2451175|865|62051|637343|4635|28327|50234|844748|5803|16695|50|26|2|3|1|359|61|97.96|180.24|102.73|4728.11|6266.53|5975.56|10994.64|375.99|0.00|2088.64|6266.53|6642.52|8355.17|8731.16|290.97| +2451128|42270|2451175|6796|62051|637343|4635|28327|50234|844748|5803|16695|25|25|10|2|244|359|19|6.54|19.03|15.03|76.00|285.57|124.26|361.57|5.54|8.56|144.59|277.01|282.55|421.60|427.14|152.75| +2451128|42270|2451172|13951|62051|637343|4635|28327|50234|844748|5803|16695|34|4|1|4|57|359|85|4.42|6.63|6.63|0.00|563.55|375.70|563.55|22.54|0.00|180.20|563.55|586.09|743.75|766.29|187.85| +2451128|42270|2451243|6146|62051|637343|4635|28327|50234|844748|5803|16695|44|4|6|4|199|359|70|48.42|93.93|37.57|3945.20|2629.90|3389.40|6575.10|157.79|0.00|262.50|2629.90|2787.69|2892.40|3050.19|-759.50| +2451128|42270|2451183|4936|62051|637343|4635|28327|50234|844748|5803|16695|44|22|14|1|85|359|84|83.45|203.61|14.25|15906.24|1197.00|7009.80|17103.24|35.91|0.00|683.76|1197.00|1232.91|1880.76|1916.67|-5812.80| +2451128|42270|2451216|15796|62051|637343|4635|28327|50234|844748|5803|16695|37|14|20|2|254|359|56|15.60|40.56|30.42|567.84|1703.52|873.60|2271.36|153.31|0.00|0.00|1703.52|1856.83|1703.52|1856.83|829.92| +2451128|42270|2451158|12637|62051|637343|4635|28327|50234|844748|5803|16695|43|2|19|4|226|359|39|72.75|146.95|111.68|1375.53|4355.52|2837.25|5731.05|391.99|0.00|687.57|4355.52|4747.51|5043.09|5435.08|1518.27| +2451128|42270|2451175|6604|62051|637343|4635|28327|50234|844748|5803|16695|55|16|20|3|259|359|61|29.41|78.81|53.59|1538.42|3268.99|1794.01|4807.41|196.13|0.00|2355.21|3268.99|3465.12|5624.20|5820.33|1474.98| +2452218|80041|2452266|4023|27116|1657600|5684|36215|10005|475996|6114|3798|19|17|8|5|209|360|24|95.41|286.23|65.83|5289.60|1579.92|2289.84|6869.52|31.59|0.00|961.68|1579.92|1611.51|2541.60|2573.19|-709.92| +2452218|80041|2452321|7175|27116|1657600|5684|36215|10005|475996|6114|3798|7|19|3|4|215|360|43|60.36|156.93|108.28|2091.95|4656.04|2595.48|6747.99|46.56|0.00|269.61|4656.04|4702.60|4925.65|4972.21|2060.56| +2452218|80041|2452323|13775|27116|1657600|5684|36215|10005|475996|6114|3798|55|5|18|3|168|360|65|43.60|71.50|12.87|3810.95|836.55|2834.00|4647.50|0.00|836.55|371.80|0.00|0.00|371.80|371.80|-2834.00| +2452218|80041|2452254|17055|27116|1657600|5684|36215|10005|475996|6114|3798|43|7|19|1|213|360|4|87.27|246.97|12.34|938.52|49.36|349.08|987.88|3.45|0.00|405.00|49.36|52.81|454.36|457.81|-299.72| +2452218|80041|2452221|9957|27116|1657600|5684|36215|10005|475996|6114|3798|5|29|7|4|171|360|46|58.02|60.92|51.78|420.44|2381.88|2668.92|2802.32|1.19|2358.06|728.18|23.82|25.01|752.00|753.19|-2645.10| +2452218|80041|2452269|2089|27116|1657600|5684|36215|10005|475996|6114|3798|25|29|7|2|284|360|50|29.86|52.25|35.53|836.00|1776.50|1493.00|2612.50|142.12|0.00|52.00|1776.50|1918.62|1828.50|1970.62|283.50| +2452218|80041|2452250|17219|27116|1657600|5684|36215|10005|475996|6114|3798|37|27|19|4|59|360|88|42.22|100.90|25.22|6659.84|2219.36|3715.36|8879.20|44.38|0.00|2308.24|2219.36|2263.74|4527.60|4571.98|-1496.00| +2452218|80041|2452284|3491|27116|1657600|5684|36215|10005|475996|6114|3798|39|21|12|5|291|360|61|84.16|92.57|21.29|4348.08|1298.69|5133.76|5646.77|116.88|0.00|2089.25|1298.69|1415.57|3387.94|3504.82|-3835.07| +2452218|80041|2452325|807|27116|1657600|5684|36215|10005|475996|6114|3798|59|7|5|1|37|360|24|24.75|63.36|45.61|426.00|1094.64|594.00|1520.64|32.83|0.00|668.88|1094.64|1127.47|1763.52|1796.35|500.64| +2452218|80041|2452281|15743|27116|1657600|5684|36215|10005|475996|6114|3798|25|15|3|3|113|360|45|85.99|196.05|190.16|265.05|8557.20|3869.55|8822.25|770.14|0.00|1146.60|8557.20|9327.34|9703.80|10473.94|4687.65| +2452218|80041|2452336|15795|27116|1657600|5684|36215|10005|475996|6114|3798|49|23|16|3|280|360|3|93.65|225.69|22.56|609.39|67.68|280.95|677.07|4.73|0.00|6.75|67.68|72.41|74.43|79.16|-213.27| +2452218|80041|2452327|13563|27116|1657600|5684|36215|10005|475996|6114|3798|57|23|5|3|201|360|1|38.31|47.50|23.75|23.75|23.75|38.31|47.50|2.13|0.00|10.45|23.75|25.88|34.20|36.33|-14.56| +2452218|80041|2452261|14937|27116|1657600|5684|36215|10005|475996|6114|3798|57|7|6|3|130|360|91|85.70|194.53|108.93|7789.60|9912.63|7798.70|17702.23|276.56|6839.71|176.54|3072.92|3349.48|3249.46|3526.02|-4725.78| +2452218|80041|2452260|13815|27116|1657600|5684|36215|10005|475996|6114|3798|39|5|13|1|56|360|86|20.29|49.91|47.91|172.00|4120.26|1744.94|4292.26|370.82|0.00|944.28|4120.26|4491.08|5064.54|5435.36|2375.32| +2452218|80041|2452309|11195|27116|1657600|5684|36215|10005|475996|6114|3798|7|27|4|4|254|360|25|41.01|92.68|71.36|533.00|1784.00|1025.25|2317.00|25.68|499.52|1158.50|1284.48|1310.16|2442.98|2468.66|259.23| +2450906|74116|2450986|5170|91959|657009|1612|39862|11736|1790443|2165|47968|58|7|8|5|69|361|91|55.39|111.88|110.76|101.92|10079.16|5040.49|10181.08|403.16|0.00|101.01|10079.16|10482.32|10180.17|10583.33|5038.67| +2450906|74116|2451011|1412|91959|657009|1612|39862|11736|1790443|2165|47968|31|14|10|4|51|361|52|67.83|176.35|153.42|1192.36|7977.84|3527.16|9170.20|398.89|0.00|4401.28|7977.84|8376.73|12379.12|12778.01|4450.68| +2450906|74116|2451015|11371|91959|657009|1612|39862|11736|1790443|2165|47968|14|28|5|2|269|361|32|45.03|106.72|45.88|1946.88|1468.16|1440.96|3415.04|73.40|0.00|751.04|1468.16|1541.56|2219.20|2292.60|27.20| +2450906|74116|2450930|9284|91959|657009|1612|39862|11736|1790443|2165|47968|16|1|10|2|201|361|17|76.06|147.55|82.62|1103.81|1404.54|1293.02|2508.35|56.18|0.00|752.42|1404.54|1460.72|2156.96|2213.14|111.52| +2450906|74116|2451012|15284|91959|657009|1612|39862|11736|1790443|2165|47968|19|26|8|4|33|361|75|49.68|74.02|63.65|777.75|4773.75|3726.00|5551.50|334.16|0.00|2275.50|4773.75|5107.91|7049.25|7383.41|1047.75| +2450906|74116|2451014|10897|91959|657009|1612|39862|11736|1790443|2165|47968|31|1|3|3|240|361|33|10.03|16.75|14.90|61.05|491.70|330.99|552.75|0.00|0.00|0.00|491.70|491.70|491.70|491.70|160.71| +2450906|74116|2450921|2558|91959|657009|1612|39862|11736|1790443|2165|47968|8|1|7|3|229|361|75|76.40|152.80|64.17|6647.25|4812.75|5730.00|11460.00|288.76|0.00|2979.00|4812.75|5101.51|7791.75|8080.51|-917.25| +2450906|74116|2450974|15979|91959|657009|1612|39862|11736|1790443|2165|47968|55|16|20|1|282|361|86|81.84|120.30|72.18|4138.32|6207.48|7038.24|10345.80|124.14|0.00|2482.82|6207.48|6331.62|8690.30|8814.44|-830.76| +2450906|74116|2450998|3830|91959|657009|1612|39862|11736|1790443|2165|47968|38|25|3|1|276|361|38|60.48|62.29|42.35|757.72|1609.30|2298.24|2367.02|16.09|0.00|402.04|1609.30|1625.39|2011.34|2027.43|-688.94| +2450906|74116|2450937|8806|91959|657009|1612|39862|11736|1790443|2165|47968|43|1|20|4|125|361|14|92.37|121.92|119.48|34.16|1672.72|1293.18|1706.88|50.18|0.00|563.22|1672.72|1722.90|2235.94|2286.12|379.54| +2452283|19065|2452294|9313|64428|314255|529|28356|64428|314255|529|28356|48|3|20|3|146|362|4|23.76|25.18|18.88|25.20|75.52|95.04|100.72|2.26|0.00|0.00|75.52|77.78|75.52|77.78|-19.52| +2452283|19065|2452297|16989|64428|314255|529|28356|64428|314255|529|28356|51|30|14|1|215|362|12|34.36|52.22|13.57|463.80|162.84|412.32|626.64|6.51|0.00|162.84|162.84|169.35|325.68|332.19|-249.48| +2452283|19065|2452288|6522|64428|314255|529|28356|64428|314255|529|28356|49|12|17|3|99|362|98|11.88|30.41|23.71|656.60|2323.58|1164.24|2980.18|185.88|0.00|1400.42|2323.58|2509.46|3724.00|3909.88|1159.34| +2452283|19065|2452372|4998|64428|314255|529|28356|64428|314255|529|28356|21|21|18|3|55|362|26|23.63|41.82|41.82|0.00|1087.32|614.38|1087.32|0.00|739.37|141.18|347.95|347.95|489.13|489.13|-266.43| +2452283|19065|2452324|11640|64428|314255|529|28356|64428|314255|529|28356|19|24|4|4|71|362|9|70.98|158.99|90.62|615.33|815.58|638.82|1430.91|48.93|0.00|500.76|815.58|864.51|1316.34|1365.27|176.76| +2452283|19065|2452318|11520|64428|314255|529|28356|64428|314255|529|28356|43|27|13|5|117|362|31|3.65|7.08|2.47|142.91|76.57|113.15|219.48|5.35|0.00|65.72|76.57|81.92|142.29|147.64|-36.58| +2452283|19065|2452289|6318|64428|314255|529|28356|64428|314255|529|28356|30|12|17|3|182|362|50|67.78|101.67|43.71|2898.00|2185.50|3389.00|5083.50|152.98|0.00|2287.50|2185.50|2338.48|4473.00|4625.98|-1203.50| +2452283|19065|2452308|10365|64428|314255|529|28356|64428|314255|529|28356|31|9|17|1|288|362|89|89.27|252.63|111.15|12591.72|9892.35|7945.03|22484.07|395.69|0.00|5170.90|9892.35|10288.04|15063.25|15458.94|1947.32| +2452283|19065|2452303|14394|64428|314255|529|28356|64428|314255|529|28356|31|25|7|1|145|362|57|13.23|22.75|20.70|116.85|1179.90|754.11|1296.75|70.79|0.00|77.52|1179.90|1250.69|1257.42|1328.21|425.79| +2452283|19065|2452403|7200|64428|314255|529|28356|64428|314255|529|28356|39|25|16|4|255|362|19|2.99|3.79|2.12|31.73|40.28|56.81|72.01|0.40|0.00|10.07|40.28|40.68|50.35|50.75|-16.53| +2452283|19065|2452356|7743|64428|314255|529|28356|64428|314255|529|28356|13|27|11|3|185|362|88|89.54|206.83|39.29|14743.52|3457.52|7879.52|18201.04|53.93|1659.60|8917.92|1797.92|1851.85|10715.84|10769.77|-6081.60| +2452283|19065|2452369|15433|64428|314255|529|28356|64428|314255|529|28356|36|7|7|3|216|362|80|54.96|128.05|75.54|4200.80|6043.20|4396.80|10244.00|302.16|0.00|5019.20|6043.20|6345.36|11062.40|11364.56|1646.40| +2452283|19065|2452372|6669|64428|314255|529|28356|64428|314255|529|28356|60|19|10|5|17|362|64|44.12|108.53|72.71|2292.48|4653.44|2823.68|6945.92|372.27|0.00|416.64|4653.44|5025.71|5070.08|5442.35|1829.76| +2452283|19065|2452298|4317|64428|314255|529|28356|64428|314255|529|28356|18|1|13|5|254|362|75|89.22|173.08|74.42|7399.50|5581.50|6691.50|12981.00|111.63|0.00|5192.25|5581.50|5693.13|10773.75|10885.38|-1110.00| +2452283|19065|2452289|8959|64428|314255|529|28356|64428|314255|529|28356|12|18|2|5|18|362|88|76.38|87.83|18.44|6106.32|1622.72|6721.44|7729.04|16.22|0.00|3245.44|1622.72|1638.94|4868.16|4884.38|-5098.72| +2452283|19065|2452390|11239|64428|314255|529|28356|64428|314255|529|28356|43|15|17|3|289|362|47|79.37|106.35|99.96|300.33|4698.12|3730.39|4998.45|422.83|0.00|2498.99|4698.12|5120.95|7197.11|7619.94|967.73| +2451537|57771|2451584|841|16677|689556|4388|40067|96258|1911677|3457|49136|8|13|6|3|27|363|15|65.39|85.66|82.23|51.45|1233.45|980.85|1284.90|34.41|86.34|513.90|1147.11|1181.52|1661.01|1695.42|166.26| +2451537|57771|2451588|700|16677|689556|4388|40067|96258|1911677|3457|49136|10|13|5|1|114|363|90|70.26|209.37|106.77|9234.00|9609.30|6323.40|18843.30|480.46|0.00|6029.10|9609.30|10089.76|15638.40|16118.86|3285.90| +2451537|57771|2451553|6898|16677|689556|4388|40067|96258|1911677|3457|49136|52|20|12|1|150|363|3|24.39|46.09|35.02|33.21|105.06|73.17|138.27|4.20|0.00|64.98|105.06|109.26|170.04|174.24|31.89| +2451537|57771|2451591|3076|16677|689556|4388|40067|96258|1911677|3457|49136|2|16|13|5|187|363|75|98.82|200.60|134.40|4965.00|10080.00|7411.50|15045.00|403.20|0.00|4212.00|10080.00|10483.20|14292.00|14695.20|2668.50| +2451537|57771|2451624|10213|16677|689556|4388|40067|96258|1911677|3457|49136|56|1|20|2|291|363|9|67.62|152.82|140.59|110.07|1265.31|608.58|1375.38|0.00|0.00|0.00|1265.31|1265.31|1265.31|1265.31|656.73| +2451537|57771|2451582|7220|16677|689556|4388|40067|96258|1911677|3457|49136|46|4|5|4|83|363|74|60.93|105.40|30.56|5538.16|2261.44|4508.82|7799.60|85.48|836.73|1169.94|1424.71|1510.19|2594.65|2680.13|-3084.11| +2451537|57771|2451586|5672|16677|689556|4388|40067|96258|1911677|3457|49136|44|25|3|3|193|363|99|80.57|240.90|120.45|11924.55|11924.55|7976.43|23849.10|0.00|2027.17|3338.28|9897.38|9897.38|13235.66|13235.66|1920.95| +2451537|57771|2451599|7196|16677||||96258||3457||13||||39|363|72|||||1208.88||2324.88||||||1208.88|1257.23|-362.16| +2451537|57771|2451635|3379|16677|689556|4388|40067|96258|1911677|3457|49136|25|1|16|4|189|363|89|89.18|256.83|20.54|21029.81|1828.06|7937.02|22857.87|8.22|1736.65|5257.23|91.41|99.63|5348.64|5356.86|-7845.61| +2451537|57771|2451624|5788|16677|689556|4388|40067|96258|1911677|3457|49136|46|26|14|4|224|363|89|47.59|104.22|103.17|93.45|9182.13|4235.51|9275.58|459.10|0.00|1205.06|9182.13|9641.23|10387.19|10846.29|4946.62| +2451537|57771|2451566|5570|16677|689556|4388|40067|96258|1911677|3457|49136|8|7|3|2|84|363|70|17.83|37.08|13.71|1635.90|959.70|1248.10|2595.60|86.37|0.00|700.70|959.70|1046.07|1660.40|1746.77|-288.40| +2451537|57771|2451652|13330|16677|689556|4388|40067|96258|1911677|3457|49136|8|22|19|1|243|363|31|48.83|70.31|33.04|1155.37|1024.24|1513.73|2179.61|71.69|0.00|152.52|1024.24|1095.93|1176.76|1248.45|-489.49| +2452607|72093|2452616|6157|91737|1771918|6852|37833|54012|446781|6329|10510|30|19|12|4|99|364|41|91.99|202.37|8.09|7965.48|331.69|3771.59|8297.17|19.90|0.00|829.43|331.69|351.59|1161.12|1181.02|-3439.90| +2452607|72093|2452697|5634|91737|1771918|6852|37833|54012|446781|6329|10510|51|21|13|5|130|364|44|38.09|66.27|7.95|2566.08|349.80|1675.96|2915.88|13.99|0.00|174.68|349.80|363.79|524.48|538.47|-1326.16| +2452607|72093|2452672|10164|91737|1771918|6852|37833|54012|446781|6329|10510|24|13|15|3|211|364|32|10.02|29.95|14.67|488.96|469.44|320.64|958.40|18.77|0.00|373.76|469.44|488.21|843.20|861.97|148.80| +2452607|72093|2452703|11815|91737|1771918|6852|37833|54012|446781|6329|10510|12|18|12|5|18|364|79|57.78|79.15|56.19|1813.84|4439.01|4564.62|6252.85|24.85|3196.08|1688.23|1242.93|1267.78|2931.16|2956.01|-3321.69| +2452607|72093|2452667|12090|91737|1771918|6852|37833|54012|446781|6329|10510|31|30|11|2|93|364|18|89.63|170.29|23.84|2636.10|429.12|1613.34|3065.22|8.58|0.00|245.16|429.12|437.70|674.28|682.86|-1184.22| +2452607|72093|2452643|14089|91737|1771918|6852|37833|54012|446781|6329|10510|9|25|4|2|210|364|6|95.12|150.28|61.61|532.02|369.66|570.72|901.68|2.55|284.63|180.30|85.03|87.58|265.33|267.88|-485.69| +2452607|72093|2452695|11907|91737|1771918|6852|37833|54012|446781|6329|10510|15|18|20|1|192|364|51|16.28|34.02|24.15|503.37|1231.65|830.28|1735.02|0.00|1120.80|485.52|110.85|110.85|596.37|596.37|-719.43| +2452607|72093|2452666|156|91737|1771918|6852|37833|54012|446781|6329|10510|45|12|9|2|195|364|55|37.88|65.91|19.11|2574.00|1051.05|2083.40|3625.05|31.53|0.00|905.85|1051.05|1082.58|1956.90|1988.43|-1032.35| +2452607|72093|2452626|4539|91737|1771918|6852|37833|54012|446781|6329|10510|39|13|7|2|224|364|95|66.14|130.29|0.00|12377.55|0.00|6283.30|12377.55|0.00|0.00|989.90|0.00|0.00|989.90|989.90|-6283.30| +2452607|72093|2452686|8845|91737|1771918|6852|37833|54012|446781|6329|10510|15|21|11|1|94|364|71|28.84|69.21|33.91|2506.30|2407.61|2047.64|4913.91|120.38|0.00|883.95|2407.61|2527.99|3291.56|3411.94|359.97| +2452607|72093|2452677|2904|91737|1771918|6852|37833|54012|446781|6329|10510|45|7|15|1|37|364|2|22.74|58.21|50.06|16.30|100.12|45.48|116.42|0.42|94.11|1.16|6.01|6.43|7.17|7.59|-39.47| +2452607|72093|2452712|3771|91737|1771918|6852|37833|54012|446781|6329|10510|60|19|7|2|54|364|24|6.15|14.08|3.37|257.04|80.88|147.60|337.92|3.23|0.00|148.56|80.88|84.11|229.44|232.67|-66.72| +2452607|72093|2452691|3030|91737|1771918|6852|37833|54012|446781|6329|10510|30|19|8|1|161|364|18|73.06|170.96|66.67|1877.22|1200.06|1315.08|3077.28|72.00|0.00|1076.94|1200.06|1272.06|2277.00|2349.00|-115.02| +2452607|72093|2452700|6099|91737|1771918|6852|37833|54012|446781|6329|10510|18|15|7|4|58|364|51|55.69|144.79|138.99|295.80|7088.49|2840.19|7384.29|567.07|0.00|3101.31|7088.49|7655.56|10189.80|10756.87|4248.30| +2452607|72093|2452614|3267|91737|1771918|6852|37833|54012|446781|6329|10510|54|30|6|2|78|364|73|68.38|153.85|9.23|10557.26|673.79|4991.74|11231.05|6.73|0.00|2920.00|673.79|680.52|3593.79|3600.52|-4317.95| +2451423|36739|2451476|10688|87363|1286156|149|36542|87363|1286156|149|36542|50|10|6|1|186|365|98|56.63|87.21|13.08|7264.74|1281.84|5549.74|8546.58|22.56|717.83|1367.10|564.01|586.57|1931.11|1953.67|-4985.73| +2451423|36739|2451499|206|87363|1286156|149|36542|87363|1286156|149|36542|49|25|4|2|180|365|85|9.54|23.75|16.38|626.45|1392.30|810.90|2018.75|0.00|0.00|39.95|1392.30|1392.30|1432.25|1432.25|581.40| +2451423|36739|2451531|5294|87363|1286156|149|36542|87363|1286156|149|36542|7|19|6|2|88|365|76|63.02|95.16|10.46|6437.20|794.96|4789.52|7232.16|31.79|0.00|1880.24|794.96|826.75|2675.20|2706.99|-3994.56| +2451423|36739|2451424|6109|87363|1286156|149|36542|87363|1286156|149|36542|52|13|7|2|7|365|58|24.95|65.36|39.86|1479.00|2311.88|1447.10|3790.88|161.83|0.00|568.40|2311.88|2473.71|2880.28|3042.11|864.78| +2451423|36739|2451540|1586|87363|1286156|149|36542|87363|1286156|149|36542|37|2|2|5|175|365|13|48.39|54.68|33.90|270.14|440.70|629.07|710.84|13.22|0.00|70.98|440.70|453.92|511.68|524.90|-188.37| +2451423|36739|2451429|1036|87363|1286156|149|36542|87363|1286156|149|36542|38|14|16|4|122|365|62|79.25|139.48|121.34|1124.68|7523.08|4913.50|8647.76|601.84|0.00|0.00|7523.08|8124.92|7523.08|8124.92|2609.58| +2451423|36739|2451481|14746|87363|1286156|149|36542|87363|1286156|149|36542|58|16|11|5|239|365|26|19.82|22.99|7.58|400.66|197.08|515.32|597.74|3.07|145.83|113.36|51.25|54.32|164.61|167.68|-464.07| +2451423|36739|2451478|2809|87363|1286156|149|36542|87363|1286156|149|36542|37|2|5|5|228|365|95|52.28|109.78|41.71|6466.65|3962.45|4966.60|10429.10|316.99|0.00|1355.65|3962.45|4279.44|5318.10|5635.09|-1004.15| +2451423|36739|2451501|2470|87363|1286156|149|36542|87363|1286156|149|36542|55|22|9|1|123|365|85|50.76|140.60|122.32|1553.80|10397.20|4314.60|11951.00|103.97|0.00|4301.85|10397.20|10501.17|14699.05|14803.02|6082.60| +2451460|82341|2451507|7405|17116|967690|6264|15238|34231|1397241|2513|29087|4|28|17|4|74|366|22|52.67|149.58|94.23|1217.70|2073.06|1158.74|3290.76|62.19|0.00|756.80|2073.06|2135.25|2829.86|2892.05|914.32| +2451460|82341|2451566|7802|17116|967690|6264|15238|34231|1397241|2513|29087|28|8|5|3|88|366|92|88.44|255.59|235.14|1881.40|21632.88|8136.48|23514.28|1081.64|0.00|7053.64|21632.88|22714.52|28686.52|29768.16|13496.40| +2451460|82341|2451534|17402|17116|967690|6264|15238|34231|1397241|2513|29087|40|10|1|1|291|366|7|19.85|33.54|22.13|79.87|154.91|138.95|234.78|0.00|142.51|30.52|12.40|12.40|42.92|42.92|-126.55| +2451460|82341|2451572|11230|17116|967690|6264|15238|34231|1397241|2513|29087|55|4|11|1|71|366|42|24.50|33.07|30.75|97.44|1291.50|1029.00|1388.94|12.91|0.00|277.62|1291.50|1304.41|1569.12|1582.03|262.50| +2451460|82341|2451538|8324|17116|967690|6264|15238|34231|1397241|2513|29087|2|10|3|5|147|366|25|96.83|177.19|81.50|2392.25|2037.50|2420.75|4429.75|9.37|1568.87|575.75|468.63|478.00|1044.38|1053.75|-1952.12| +2451460|82341|2451523|16546|17116|967690|6264|15238|34231|1397241|2513|29087|7|16|7|5|93|366|98|54.62|116.88|16.36|9850.96|1603.28|5352.76|11454.24|112.22|0.00|4696.16|1603.28|1715.50|6299.44|6411.66|-3749.48| +2451460|82341|2451537|12760|17116|967690|6264|15238|34231|1397241|2513|29087|22|28|11|4|280|366|2|88.97|182.38|149.55|65.66|299.10|177.94|364.76|20.93|0.00|62.00|299.10|320.03|361.10|382.03|121.16| +2451460|82341|2451578|17512|17116|967690|6264|15238|34231|1397241|2513|29087|25|8|4|1|295|366|68|23.61|32.10|12.51|1332.12|850.68|1605.48|2182.80|76.56|0.00|785.40|850.68|927.24|1636.08|1712.64|-754.80| +2451460|82341|2451493|16270|17116|967690|6264|15238|34231|1397241|2513|29087|8|14|1|2|190|366|94|24.69|42.46|41.61|79.90|3911.34|2320.86|3991.24|195.56|0.00|597.84|3911.34|4106.90|4509.18|4704.74|1590.48| +2451460|82341|2451567|1604|17116|967690|6264|15238|34231|1397241|2513|29087|49|14|18|1|234|366|24|75.70|186.22|175.04|268.32|4200.96|1816.80|4469.28|127.70|2604.59|2234.64|1596.37|1724.07|3831.01|3958.71|-220.43| +2451460|82341|2451522|2857|17116|967690|6264|15238|34231|1397241|2513|29087|43|22|2|4|289|366|78|1.93|5.19|4.72|36.66|368.16|150.54|404.82|22.08|0.00|177.84|368.16|390.24|546.00|568.08|217.62| +2451180|34990|2451210|7372|33544|1370642|6493|39191|63100|897132|1638|24289|55|20|17|5|96|367|46|37.24|103.89|55.06|2246.18|2532.76|1713.04|4778.94|25.32|0.00|1767.78|2532.76|2558.08|4300.54|4325.86|819.72| +2451180|34990|2451214|5425|33544|1370642|6493|39191|63100|897132|1638|24289|40|19|9|3|74|367|15|9.30|25.01|14.25|161.40|213.75|139.50|375.15|8.55|0.00|183.75|213.75|222.30|397.50|406.05|74.25| +2451180|34990|2451260|13984|33544|1370642|6493|39191|63100|897132|1638|24289|13|8|3|1|269|367|12|64.76|118.51|29.62|1066.68|355.44|777.12|1422.12|0.00|0.00|56.88|355.44|355.44|412.32|412.32|-421.68| +2451180|34990|2451264|6367|33544|1370642|6493|39191|63100|897132|1638|24289|50|20|9|4|222|367|6|89.72|218.01|13.08|1229.58|78.48|538.32|1308.06|0.00|0.00|26.16|78.48|78.48|104.64|104.64|-459.84| +2451180|34990|2451232|15512|33544|1370642|6493|39191|63100|897132|1638|24289|7|1|5|2|156|367|33|60.87|150.95|12.07|4583.04|398.31|2008.71|4981.35|11.94|0.00|1095.60|398.31|410.25|1493.91|1505.85|-1610.40| +2451180|34990|2451196|3328|33544|1370642|6493|39191|63100|897132|1638|24289|55|13|13|4|108|367|84|70.72|192.35|169.26|1939.56|14217.84|5940.48|16157.40|995.24|0.00|4523.40|14217.84|15213.08|18741.24|19736.48|8277.36| +2451180|34990|2451291|10570|33544|1370642|6493|39191|63100|897132|1638|24289|49|22|9|1|123|367|14|42.66|120.30|21.65|1381.10|303.10|597.24|1684.20|6.06|0.00|623.14|303.10|309.16|926.24|932.30|-294.14| +2451180|34990|2451231|13591|33544|1370642|6493|39191|63100|897132|1638|24289|32|10|5|4|3|367|45|96.33|288.02|48.96|10757.70|2203.20|4334.85|12960.90|154.22|0.00|2851.20|2203.20|2357.42|5054.40|5208.62|-2131.65| +2451180|34990|2451222|5924|33544|1370642|6493|39191|63100|897132|1638|24289|58|28|13|3|106|367|47|26.04|32.55|12.04|963.97|565.88|1223.88|1529.85|0.00|0.00|382.11|565.88|565.88|947.99|947.99|-658.00| +2451180|34990|2451182|11071|33544|1370642|6493|39191|63100|897132|1638|24289|52|4|12|3|97|367|48|10.59|19.16|1.53|846.24|73.44|508.32|919.68|3.67|0.00|27.36|73.44|77.11|100.80|104.47|-434.88| +2452545|70351|2452554|14479|98290|602351|5965|6426|976|232242|4515|28176|25|19|10|5|100|368|56|58.77|91.09|45.54|2550.80|2550.24|3291.12|5101.04|76.50|637.56|1326.08|1912.68|1989.18|3238.76|3315.26|-1378.44| +2452545|70351|2452566|10893|98290|602351|5965|6426|976|232242|4515|28176|21|25|5|2|79|368|6|67.32|149.45|56.79|555.96|340.74|403.92|896.70|30.66|0.00|322.80|340.74|371.40|663.54|694.20|-63.18| +2452545|70351|2452612|14418|98290|602351|5965|6426|976|232242|4515|28176|1|27|17|5|53|368|71|68.03|108.84|83.80|1777.84|5949.80|4830.13|7727.64|475.98|0.00|1544.96|5949.80|6425.78|7494.76|7970.74|1119.67| +2452545|70351|2452585|17574|98290|602351|5965|6426|976|232242|4515|28176|57|9|8|1|7|368|6|94.75|214.13|70.66|860.82|423.96|568.50|1284.78|7.37|55.11|462.48|368.85|376.22|831.33|838.70|-199.65| +2452545|70351|2452614|1323|98290|602351|5965|6426|976|232242|4515|28176|37|15|17|1|65|368|42|10.22|16.04|9.30|283.08|390.60|429.24|673.68|22.49|109.36|127.68|281.24|303.73|408.92|431.41|-148.00| +2452545|70351|2452638|15961|98290|602351|5965|6426|976|232242|4515|28176|60|21|12|3|164|368|40|11.91|24.29|0.72|942.80|28.80|476.40|971.60|1.15|0.00|0.00|28.80|29.95|28.80|29.95|-447.60| +2452545|70351|2452609|1581|98290|602351|5965|6426|976|232242|4515|28176|57|3|18|1|267|368|71|1.31|3.27|2.61|46.86|185.31|93.01|232.17|9.26|0.00|95.14|185.31|194.57|280.45|289.71|92.30| +2452545|70351|2452611|3336|98290|602351|5965|6426|976|232242|4515|28176|31|21|6|1|26|368|53|56.72|167.32|83.66|4433.98|4433.98|3006.16|8867.96|310.37|0.00|2660.07|4433.98|4744.35|7094.05|7404.42|1427.82| +2452545|70351|2452561|15841|98290|602351|5965|6426|976|232242|4515|28176|49|1|7|1|252|368|66|73.68|211.46|12.68|13119.48|836.88|4862.88|13956.36|33.05|175.74|2372.04|661.14|694.19|3033.18|3066.23|-4201.74| +2452545|70351|2452665|15078|98290|602351|5965|6426|976|232242|4515|28176|30|3|9|4|106|368|36|56.60|76.41|66.47|357.84|2392.92|2037.60|2750.76|191.43|0.00|329.76|2392.92|2584.35|2722.68|2914.11|355.32| +2452545|70351|2452619|14802|98290|602351|5965|6426|976|232242|4515|28176|3|7|13|5|77|368|76|47.39|106.62|71.43|2674.44|5428.68|3601.64|8103.12|217.14|0.00|4051.56|5428.68|5645.82|9480.24|9697.38|1827.04| +2452545|70351|2452580|2853|98290|602351|5965|6426|976|232242|4515|28176|48|25|15|4|146|368|66|77.88|165.88|3.31|10729.62|218.46|5140.08|10948.08|8.73|0.00|5145.36|218.46|227.19|5363.82|5372.55|-4921.62| +2452545|70351|2452623|8761|98290|602351|5965|6426|976|232242|4515|28176|15|13|6|2|281|368|1|33.93|91.95|32.18|59.77|32.18|33.93|91.95|0.52|23.49|18.39|8.69|9.21|27.08|27.60|-25.24| +2452374|72197|2452380|2739|39075|1439349|1576|34250|9318|1764854|1872|39346|54|25|2|1|47|369|52|63.20|183.91|183.91|0.00|9563.32|3286.40|9563.32|286.89|0.00|1242.80|9563.32|9850.21|10806.12|11093.01|6276.92| +2452374|72197|2452426|16530|39075|1439349|1576|34250|9318|1764854|1872|39346|31|21|13|3|153|369|48|75.33|169.49|69.49|4800.00|3335.52|3615.84|8135.52|33.35|0.00|2603.04|3335.52|3368.87|5938.56|5971.91|-280.32| +2452374|72197|2452379|3055|39075|1439349|1576|34250|9318|1764854|1872|39346|15|6|18|3|258|369|61|19.75|45.42|44.51|55.51|2715.11|1204.75|2770.62|47.24|1140.34|609.39|1574.77|1622.01|2184.16|2231.40|370.02| +2452374|72197|2452490|17364|39075|1439349|1576|34250|9318|1764854|1872|39346|37|7|9|4|19|369|52|57.20|106.96|13.90|4839.12|722.80|2974.40|5561.92|65.05|0.00|1167.92|722.80|787.85|1890.72|1955.77|-2251.60| +2452374|72197|2452487|7213|39075|1439349|1576|34250|9318|1764854|1872|39346|19|27|15|4|273|369|91|27.09|59.32|51.01|756.21|4641.91|2465.19|5398.12|0.00|1717.50|1834.56|2924.41|2924.41|4758.97|4758.97|459.22| +2452374|72197|2452456|1644|39075|1439349|1576|34250|9318|1764854|1872|39346|42|9|2|1|160|369|47|88.83|265.60|164.67|4743.71|7739.49|4175.01|12483.20|232.18|0.00|4244.10|7739.49|7971.67|11983.59|12215.77|3564.48| +2452374|72197|2452466|11496|39075|1439349|1576|34250|9318|1764854|1872|39346|57|19|18|4|290|369|33|15.33|18.85|3.20|516.45|105.60|505.89|622.05|1.05|0.00|180.18|105.60|106.65|285.78|286.83|-400.29| +2452374|72197|2452421|17328|39075|1439349|1576|34250|9318|1764854|1872|39346|51|9|14|1|55|369|13|56.92|97.33|48.66|632.71|632.58|739.96|1265.29|21.00|107.53|215.02|525.05|546.05|740.07|761.07|-214.91| +2452374|72197|2452451|4470|39075|1439349|1576|34250|9318|1764854|1872|39346|43|1|7|1|155|369|31|85.42|189.63|58.78|4056.35|1822.18|2648.02|5878.53|145.77|0.00|1528.30|1822.18|1967.95|3350.48|3496.25|-825.84| +2452374|72197|2452384|7992|39075|1439349|1576|34250|9318|1764854|1872|39346|24|30|11|4|287|369|79|11.43|31.54|3.15|2242.81|248.85|902.97|2491.66|0.00|37.32|0.00|211.53|211.53|211.53|211.53|-691.44| +2452374|72197|2452461|4839|39075|1439349|1576|34250|9318|1764854|1872|39346|13|15|19|5|191|369|89|6.33|16.52|8.92|676.40|793.88|563.37|1470.28|15.87|0.00|14.24|793.88|809.75|808.12|823.99|230.51| +2452351|68740|2452467|16953|48266|930853|6687|48474|95574|1479125|4329|38163|51|27|12|5|38|370|6|68.01|120.37|69.81|303.36|418.86|408.06|722.22|25.13|0.00|79.44|418.86|443.99|498.30|523.43|10.80| +2452351|68740|2452441|16320|48266|930853|6687|48474|95574|1479125|4329|38163|15|19|5|2|32|370|9|86.54|202.50|133.65|619.65|1202.85|778.86|1822.50|96.22|0.00|328.05|1202.85|1299.07|1530.90|1627.12|423.99| +2452351|68740|2452363|4039|48266|930853|6687|48474|95574|1479125|4329|38163|37|7|19|1|26|370|78|85.65|155.88|90.41|5106.66|7051.98|6680.70|12158.64|493.63|0.00|607.62|7051.98|7545.61|7659.60|8153.23|371.28| +2452351|68740|2452363|17820|48266|930853|6687|48474|95574|1479125|4329|38163|54|19|10|3|3|370|37|70.62|131.35|43.34|3256.37|1603.58|2612.94|4859.95|48.10|0.00|0.00|1603.58|1651.68|1603.58|1651.68|-1009.36| +2452351|68740|2452407|11061|48266|930853|6687|48474|95574|1479125|4329|38163|57|18|9|4|129|370|55|3.08|6.46|3.35|171.05|184.25|169.40|355.30|9.21|0.00|59.95|184.25|193.46|244.20|253.41|14.85| +2452351|68740|2452357|4632|48266|930853|6687|48474|95574|1479125|4329|38163|27|7|17|5|20|370|77|2.89|6.70|1.34|412.72|103.18|222.53|515.90|5.15|0.00|174.79|103.18|108.33|277.97|283.12|-119.35| +2452351|68740|2452393|13495|48266|930853|6687|48474|95574|1479125|4329|38163|9|27|3|2|281|370|34|21.36|38.44|11.91|902.02|404.94|726.24|1306.96|12.14|0.00|574.94|404.94|417.08|979.88|992.02|-321.30| +2452351|68740|2452372|16117|48266|930853|6687|48474|95574|1479125|4329|38163|13|1|15|4|43|370|28|93.93|93.93|43.20|1420.44|1209.60|2630.04|2630.04|84.67|0.00|157.64|1209.60|1294.27|1367.24|1451.91|-1420.44| +2452351|68740|2452465|15756|48266|930853|6687|48474|95574|1479125|4329|38163|24|9|18|3|81|370|62|98.23|231.82|185.45|2874.94|11497.90|6090.26|14372.84|344.93|0.00|430.90|11497.90|11842.83|11928.80|12273.73|5407.64| +2452351|68740|2452386|5112|48266|930853|6687|48474|95574|1479125|4329|38163|45|13|11|2|266|370|64|50.08|117.18|78.51|2474.88|5024.64|3205.12|7499.52|100.49|0.00|3224.32|5024.64|5125.13|8248.96|8349.45|1819.52| +2452351|68740|2452420|14367|48266|930853|6687|48474|95574|1479125|4329|38163|6|19|7|4|271|370|34|96.02|168.03|0.00|5713.02|0.00|3264.68|5713.02|0.00|0.00|2113.78|0.00|0.00|2113.78|2113.78|-3264.68| +2452351|68740|2452376|528|48266|930853|6687|48474|95574|1479125|4329|38163|54|24|17|5|290|370|56|18.32|48.91|0.48|2712.08|26.88|1025.92|2738.96|0.26|0.00|520.24|26.88|27.14|547.12|547.38|-999.04| +2452604|73269|2452606|6162|46123|838006|1889|48172|5840|1749492|1774|17257|57|25|17|3|2|371|82|52.29|142.75|74.23|5618.64|6086.86|4287.78|11705.50|213.04|1826.05|2925.76|4260.81|4473.85|7186.57|7399.61|-26.97| +2452604|73269|2452674|15595|46123|838006|1889|48172|5840|1749492|1774|17257|24|18|19|1|136|371|33|11.90|17.25|7.76|313.17|256.08|392.70|569.25|5.12|0.00|113.85|256.08|261.20|369.93|375.05|-136.62| +2452604|73269|2452648|5593|46123|838006|1889|48172|5840|1749492|1774|17257|49|1|20|4|58|371|86|50.38|128.46|73.22|4750.64|6296.92|4332.68|11047.56|0.00|0.00|1215.18|6296.92|6296.92|7512.10|7512.10|1964.24| +2452604|73269|2452653|1189|46123|838006|1889|48172|5840|1749492|1774|17257|43|9|18|1|7|371|92|92.55|129.57|41.46|8106.12|3814.32|8514.60|11920.44|305.14|0.00|4887.04|3814.32|4119.46|8701.36|9006.50|-4700.28| +2452604|73269|2452640|11766|46123|838006|1889|48172|5840|1749492|1774|17257|6|3|11|5|149|371|85|33.33|69.32|8.31|5185.85|706.35|2833.05|5892.20|0.00|0.00|352.75|706.35|706.35|1059.10|1059.10|-2126.70| +2452604|73269|2452618|6219|46123|838006|1889|48172|5840|1749492|1774|17257|54|27|11|1|58|371|18|6.54|14.19|3.83|186.48|68.94|117.72|255.42|5.51|0.00|45.90|68.94|74.45|114.84|120.35|-48.78| +2452604|73269|2452626|7374|46123|838006|1889|48172|5840|1749492|1774|17257|57|21|13|5|287|371|89|6.47|8.41|8.41|0.00|748.49|575.83|748.49|22.45|0.00|112.14|748.49|770.94|860.63|883.08|172.66| +2452604|73269|2452644|5425|46123|838006|1889|48172|5840|1749492|1774|17257|51|9|7|1|295|371|4|7.88|13.00|6.76|24.96|27.04|31.52|52.00|1.08|0.00|12.48|27.04|28.12|39.52|40.60|-4.48| +2452604|73269|2452678|13986|46123|838006|1889|48172|5840|1749492|1774|17257|24|24|9|4|169|371|81|13.18|32.55|14.64|1450.71|1185.84|1067.58|2636.55|83.00|0.00|1185.84|1185.84|1268.84|2371.68|2454.68|118.26| +2452604|73269|2452612|6367|46123|838006|1889|48172|5840|1749492|1774|17257|27|12|15|4|213|371|1|33.12|37.42|5.61|31.81|5.61|33.12|37.42|0.50|0.00|18.33|5.61|6.11|23.94|24.44|-27.51| +2452232|44661|2452239|157|71227|1147259|5691|13723|35564|1323032|2619|37051|3|25|1|2|90|372|20|40.10|82.60|48.73|677.40|974.60|802.00|1652.00|87.71|0.00|396.40|974.60|1062.31|1371.00|1458.71|172.60| +2452232|44661|2452261|15581|71227|1147259|5691|13723|35564|1323032|2619|37051|25|27|18|2|183|372|26|24.60|31.48|22.03|245.70|572.78|639.60|818.48|28.63|0.00|130.78|572.78|601.41|703.56|732.19|-66.82| +2452232|44661|2452342|16789|71227|1147259|5691|13723|35564|1323032|2619|37051|7|7|4|4|154|372|85|71.48|191.56|103.44|7490.20|8792.40|6075.80|16282.60|175.84|0.00|3907.45|8792.40|8968.24|12699.85|12875.69|2716.60| +2452232|44661|2452240|13887|71227|1147259|5691|13723|35564|1323032|2619|37051|21|29|9|3|57|372|49|16.49|30.34|12.74|862.40|624.26|808.01|1486.66|37.45|0.00|460.60|624.26|661.71|1084.86|1122.31|-183.75| +2452232|44661|2452333|6053|71227|1147259|5691|13723|35564|1323032|2619|37051|57|13|12|5|225|372|78|57.80|130.62|73.14|4483.44|5704.92|4508.40|10188.36|50.77|627.54|509.34|5077.38|5128.15|5586.72|5637.49|568.98| +2452232|44661|2452276|12869|71227|1147259|5691|13723|35564|1323032|2619|37051|11|21|9|4|259|372|51|47.31|67.65|18.94|2484.21|965.94|2412.81|3450.15|57.95|0.00|1069.47|965.94|1023.89|2035.41|2093.36|-1446.87| +2452232|44661|2452351|1755|71227|1147259|5691|13723|35564|1323032|2619|37051|11|11|6|1|96|372|69|47.13|75.87|29.58|3194.01|2041.02|3251.97|5235.03|20.41|0.00|0.00|2041.02|2061.43|2041.02|2061.43|-1210.95| +2452232|44661|2452341|15471|71227|1147259|5691|13723|35564|1323032|2619|37051|29|25|11|4|7|372|93|41.79|95.28|47.64|4430.52|4430.52|3886.47|8861.04|354.44|0.00|1151.34|4430.52|4784.96|5581.86|5936.30|544.05| +2452232|44661|2452247|17887|71227|1147259|5691|13723|35564|1323032|2619|37051|21|5|5|2|249|372|78|42.95|109.09|1.09|8424.00|85.02|3350.10|8509.02|3.09|7.65|2382.12|77.37|80.46|2459.49|2462.58|-3272.73| +2452232|44661|2452303|8277|71227|1147259|5691|13723|35564|1323032|2619|37051|21|27|20|3|226|372|50|22.07|30.45|9.74|1035.50|487.00|1103.50|1522.50|29.22|0.00|380.50|487.00|516.22|867.50|896.72|-616.50| +2452232|44661|2452259|14437|71227|1147259|5691|13723|35564|1323032|2619|37051|55|15|15|5|225|372|35|75.41|223.21|87.05|4765.60|3046.75|2639.35|7812.35|0.00|0.00|1640.45|3046.75|3046.75|4687.20|4687.20|407.40| +2452232|44661|2452329|17609|71227|1147259|5691|13723|35564|1323032|2619|37051|33|11|5|4|243|372|86|43.54|99.70|25.92|6345.08|2229.12|3744.44|8574.20|0.00|1582.67|3600.82|646.45|646.45|4247.27|4247.27|-3097.99| +2452232|44661|2452347|14219|71227|1147259|5691|13723|35564|1323032|2619|37051|29|21|7|3|119|372|55|67.60|139.93|134.33|308.00|7388.15|3718.00|7696.15|369.40|0.00|2924.35|7388.15|7757.55|10312.50|10681.90|3670.15| +2451781|5402|2451798|13663|29374|89901|2875|42202|7741|1316582|3830|14757|19|13|14|1|88|373|27|36.83|93.91|31.92|1673.73|861.84|994.41|2535.57|34.47|0.00|50.49|861.84|896.31|912.33|946.80|-132.57| +2451781|5402|2451838|488|29374|89901|2875|42202|7741|1316582|3830|14757|25|25|3|3|276|373|69|23.81|30.95|17.33|939.78|1195.77|1642.89|2135.55|20.44|968.57|84.87|227.20|247.64|312.07|332.51|-1415.69| +2451781|5402|2451892|12665|29374|89901|2875|42202|7741|1316582|3830|14757|49|19|8|1|64|373|1|32.08|85.01|65.45|19.56|65.45|32.08|85.01|2.61|0.00|32.30|65.45|68.06|97.75|100.36|33.37| +2451781|5402|2451810|13753|29374|89901|2875|42202|7741|1316582|3830|14757|59|1|2|5|41|373|41|17.21|42.50|36.55|243.95|1498.55|705.61|1742.50|59.94|0.00|592.45|1498.55|1558.49|2091.00|2150.94|792.94| +2451781|5402|2451895|1397|29374|89901|2875|42202|7741|1316582|3830|14757|17|14|20|4|291|373|69|46.22|116.47|4.65|7715.58|320.85|3189.18|8036.43|12.83|0.00|723.12|320.85|333.68|1043.97|1056.80|-2868.33| +2451781|5402|2451894|14179|29374|89901|2875|42202|7741|1316582|3830|14757|44|19|14|3|172|373|66|33.11|62.24|52.28|657.36|3450.48|2185.26|4107.84|241.53|0.00|985.38|3450.48|3692.01|4435.86|4677.39|1265.22| +2451781|5402|2451889|920|29374|89901|2875|42202|7741|1316582|3830|14757|50|7|15|3|284|373|95|65.52|96.96|20.36|7277.00|1934.20|6224.40|9211.20|77.36|0.00|736.25|1934.20|2011.56|2670.45|2747.81|-4290.20| +2451781|5402|2451887|9697|29374|89901|2875|42202|7741|1316582|3830|14757|50|29|14|3|222|373|84|86.85|251.86|219.11|2751.00|18405.24|7295.40|21156.24|736.20|0.00|9943.08|18405.24|19141.44|28348.32|29084.52|11109.84| +2451781|5402|2451891|12145|29374|89901|2875|42202|7741|1316582|3830|14757|37|17|2|2|62|373|71|11.38|14.79|8.13|472.86|577.23|807.98|1050.09|0.00|0.00|503.39|577.23|577.23|1080.62|1080.62|-230.75| +2451781|5402|2451795|1208|29374|89901|2875|42202|7741|1316582|3830|14757|31|19|7|1|225|373|89|57.20|123.55|67.95|4948.40|6047.55|5090.80|10995.95|120.95|0.00|3078.51|6047.55|6168.50|9126.06|9247.01|956.75| +2451781|5402|2451787|4493|29374|89901|2875|42202|7741|1316582|3830|14757|31|26|2|5|188|373|54|81.86|117.05|29.26|4740.66|1580.04|4420.44|6320.70|79.00|0.00|884.52|1580.04|1659.04|2464.56|2543.56|-2840.40| +2451781|5402|2451880|15239|29374|89901|2875|42202|7741|1316582|3830|14757|23|7|16|3|220|373|86|13.23|26.06|20.84|448.92|1792.24|1137.78|2241.16|143.37|0.00|1097.36|1792.24|1935.61|2889.60|3032.97|654.46| +2451781|5402|2451782|5162|29374|89901|2875|42202|7741|1316582|3830|14757|17|8|4|3|249|373|39|18.05|25.45|11.45|546.00|446.55|703.95|992.55|35.72|0.00|168.48|446.55|482.27|615.03|650.75|-257.40| +2451781|5402|2451860|5441|29374|89901|2875|42202|7741|1316582|3830|14757|47|23|17|2|230|373|1|7.51|10.28|1.95|8.33|1.95|7.51|10.28|0.01|0.00|3.18|1.95|1.96|5.13|5.14|-5.56| +2452219|18185|2452259|8623|68929|246334|1339|17191|68523|164894|653|3658|23|29|1|4|197|374|65|44.72|55.90|24.59|2035.15|1598.35|2906.80|3633.50|31.96|0.00|1271.40|1598.35|1630.31|2869.75|2901.71|-1308.45| +2452219|18185|2452312|2237|68929|246334|1339|17191|68523|164894|653|3658|23|3|9|4|166|374|89|67.22|136.45|77.77|5222.52|6921.53|5982.58|12144.05|127.35|5329.57|3278.76|1591.96|1719.31|4870.72|4998.07|-4390.62| +2452219|18185|2452276|7363|68929|246334|1339|17191|68523|164894|653|3658|37|11|13|3|80|374|7|38.59|64.44|21.26|302.26|148.82|270.13|451.08|7.76|62.50|22.54|86.32|94.08|108.86|116.62|-183.81| +2452219|18185|2452288|2045|68929|246334|1339|17191|68523|164894|653|3658|11|27|12|2|225|374|82|86.79|121.50|110.56|897.08|9065.92|7116.78|9963.00|659.99|815.93|4283.68|8249.99|8909.98|12533.67|13193.66|1133.21| +2452219|18185|2452280|327|68929|246334|1339|17191|68523|164894|653|3658|19|11|9|2|15|374|46|35.07|90.83|45.41|2089.32|2088.86|1613.22|4178.18|61.41|1065.31|710.24|1023.55|1084.96|1733.79|1795.20|-589.67| +2452219|18185|2452273|2141|68929|246334|1339|17191|68523|164894|653|3658|41|15|12|4|66|374|45|89.66|117.45|88.08|1321.65|3963.60|4034.70|5285.25|81.25|2338.52|264.15|1625.08|1706.33|1889.23|1970.48|-2409.62| +2452219|18185|2452240|2367|68929|246334|1339|17191|68523|164894|653|3658|33|21|12|3|134|374|50|15.61|17.01|8.33|434.00|416.50|780.50|850.50|37.48|0.00|314.50|416.50|453.98|731.00|768.48|-364.00| +2452219|18185|2452238|17419|68929|246334|1339|17191|68523|164894|653|3658|47|11|5|2|289|374|27|87.64|130.58|126.66|105.84|3419.82|2366.28|3525.66|239.38|0.00|634.50|3419.82|3659.20|4054.32|4293.70|1053.54| +2452219|18185|2452224|11273|68929|246334|1339|17191|68523|164894|653|3658|11|15|13|3|270|374|86|70.16|186.62|27.99|13642.18|2407.14|6033.76|16049.32|192.57|0.00|1123.16|2407.14|2599.71|3530.30|3722.87|-3626.62| +2452219|18185|2452315|8159|68929|246334|1339|17191|68523|164894|653|3658|53|5|16|2|118|374|74|23.90|37.52|12.75|1832.98|943.50|1768.60|2776.48|66.04|0.00|637.88|943.50|1009.54|1581.38|1647.42|-825.10| +2452219|18185|2452334|12199|68929|246334|1339|17191|68523|164894|653|3658|53|15|16|2|205|374|3|68.82|190.63|59.09|394.62|177.27|206.46|571.89|0.00|0.00|22.86|177.27|177.27|200.13|200.13|-29.19| +2452219|18185|2452277|11261|68929|246334|1339|17191|68523|164894|653|3658|39|1|2|1|57|374|4|46.71|96.22|30.79|261.72|123.16|186.84|384.88|5.24|35.71|65.40|87.45|92.69|152.85|158.09|-99.39| +2452219|18185|2452299|14857|68929|246334|1339|17191|68523|164894|653|3658|43|7|9|1|183|374|12|37.24|109.11|56.73|628.56|680.76|446.88|1309.32|61.26|0.00|288.00|680.76|742.02|968.76|1030.02|233.88| +2452219|18185|2452288|17383|68929|246334|1339|17191|68523|164894|653|3658|15|11|1|1|104|374|56|97.72|194.46|40.83|8603.28|2286.48|5472.32|10889.76|74.31|800.26|217.28|1486.22|1560.53|1703.50|1777.81|-3986.10| +2452219|18185|2452248|53|68929|246334|1339|17191|68523|164894|653|3658|1|1|1|4|234|374|26|73.97|142.02|4.26|3581.76|110.76|1923.22|3692.52|0.00|0.00|1218.36|110.76|110.76|1329.12|1329.12|-1812.46| +2452219|18185|2452339|9903|68929|246334|1339|17191|68523|164894|653|3658|17|23|15|1|284|374|73|41.40|87.35|10.48|5611.51|765.04|3022.20|6376.55|24.09|283.06|637.29|481.98|506.07|1119.27|1143.36|-2540.22| +2452489|68432|2452594|4567|2181|504137|2540|19743|36289|93157|3352|21355|51|7|2|2|185|375|9|29.50|31.86|22.30|86.04|200.70|265.50|286.74|14.04|0.00|31.50|200.70|214.74|232.20|246.24|-64.80| +2452489|68432|2452593|2562|2181|504137|2540|19743|36289|93157|3352|21355|12|25|17|1|138|375|49|15.79|23.84|22.88|47.04|1121.12|773.71|1168.16|100.90|0.00|116.62|1121.12|1222.02|1237.74|1338.64|347.41| +2452489|68432|2452597|13878|2181|504137|2540|19743|36289|93157|3352|21355|19|15|13|1|46|375|88|71.76|106.92|10.69|8468.24|940.72|6314.88|9408.96|28.22|0.00|4233.68|940.72|968.94|5174.40|5202.62|-5374.16| +2452489|68432|2452524|13051|2181|504137|2540|19743|36289|93157|3352|21355|9|9|15|5|35|375|98|11.49|14.93|7.16|761.46|701.68|1126.02|1463.14|30.17|98.23|394.94|603.45|633.62|998.39|1028.56|-522.57| +2452489|68432|2452521|13254|2181|504137|2540|19743|36289|93157|3352|21355|7|12|5|4|266|375|4|88.57|220.53|50.72|679.24|202.88|354.28|882.12|14.20|0.00|414.56|202.88|217.08|617.44|631.64|-151.40| +2452489|68432|2452523|8475|2181|504137|2540|19743|36289|93157|3352|21355|19|6|9|2|65|375|5|72.43|144.86|105.74|195.60|528.70|362.15|724.30|26.17|237.91|318.65|290.79|316.96|609.44|635.61|-71.36| +2452489|68432|2452552|6441|2181|504137|2540|19743|36289|93157|3352|21355|15|6|1|1|124|375|80|26.91|57.85|24.87|2638.40|1989.60|2152.80|4628.00|0.00|0.00|925.60|1989.60|1989.60|2915.20|2915.20|-163.20| +2452489|68432|2452514|13441|2181|504137|2540|19743|36289|93157|3352|21355|18|6|14|3|82|375|50|46.55|116.37|108.22|407.50|5411.00|2327.50|5818.50|108.22|0.00|1454.50|5411.00|5519.22|6865.50|6973.72|3083.50| +2452489|68432|2452597|9811|2181|504137|2540|19743|36289|93157|3352|21355|55|21|4|2|20|375|19|73.69|211.49|162.84|924.35|3093.96|1400.11|4018.31|92.81|0.00|321.29|3093.96|3186.77|3415.25|3508.06|1693.85| +2451484|47770|2451563|15266|91025|621638|3804|11217|63558|1885021|2609|16022|16|2|15|3|191|376|16|10.09|20.88|8.97|190.56|143.52|161.44|334.08|5.74|0.00|56.64|143.52|149.26|200.16|205.90|-17.92| +2451484|47770|2451514|5378|91025|621638|3804|11217|63558|1885021|2609|16022|55|2|9|1|150|376|78|72.51|144.29|142.84|113.10|11141.52|5655.78|11254.62|222.83|0.00|3825.90|11141.52|11364.35|14967.42|15190.25|5485.74| +2451484|47770|2451524|13798|91025|621638|3804|11217|63558|1885021|2609|16022|7|26|1|1|291|376|9|11.61|13.46|3.36|90.90|30.24|104.49|121.14|2.41|0.00|50.85|30.24|32.65|81.09|83.50|-74.25| +2451484|47770|2451535|7633|91025|621638|3804|11217|63558|1885021|2609|16022|22|4|4|5|103|376|85|5.34|14.84|9.05|492.15|769.25|453.90|1261.40|61.54|0.00|87.55|769.25|830.79|856.80|918.34|315.35| +2451484|47770|2451543|7720|91025|621638|3804|11217|63558|1885021|2609|16022|46|16|17|2|70|376|94|32.67|64.35|61.77|242.52|5806.38|3070.98|6048.90|406.44|0.00|483.16|5806.38|6212.82|6289.54|6695.98|2735.40| +2451484|47770|2451487|10441|91025|621638|3804|11217|63558|1885021|2609|16022|40|4|18|3|25|376|12|42.85|60.41|55.57|58.08|666.84|514.20|724.92|53.34|0.00|152.16|666.84|720.18|819.00|872.34|152.64| +2451484|47770|2451559|2911|91025|621638|3804|11217|63558|1885021|2609|16022|32|20|8|2|219|376|78|12.79|34.02|15.98|1407.12|1246.44|997.62|2653.56|7.85|1134.26|822.12|112.18|120.03|934.30|942.15|-885.44| +2451484|47770|2451485|9355|91025|621638|3804|11217|63558|1885021|2609|16022|31|2|15|1|274|376|95|29.29|29.29|27.53|167.20|2615.35|2782.55|2782.55|104.61|0.00|1279.65|2615.35|2719.96|3895.00|3999.61|-167.20| +2451484|47770|2451537|1819|91025|621638|3804|11217|63558|1885021|2609|16022|50|14|7|1|125|376|58|77.37|91.29|59.33|1853.68|3441.14|4487.46|5294.82|309.70|0.00|2329.28|3441.14|3750.84|5770.42|6080.12|-1046.32| +2451484|47770|2451496|8944|91025|621638|3804|11217|63558|1885021|2609|16022|37|7|19|1|116|376|28|77.10|154.20|1.54|4274.48|43.12|2158.80|4317.60|2.58|0.00|1813.28|43.12|45.70|1856.40|1858.98|-2115.68| +2451484|47770|2451595|6472|91025|621638|3804|11217|63558|1885021|2609|16022|13|13|10|5|300|376|75|97.71|282.38|211.78|5295.00|15883.50|7328.25|21178.50|0.00|0.00|5929.50|15883.50|15883.50|21813.00|21813.00|8555.25| +2451484|47770|2451523|3260|91025|621638|3804|11217|63558|1885021|2609|16022|8|2|6|5|187|376|71|25.90|56.98|55.27|121.41|3924.17|1838.90|4045.58|235.45|0.00|484.93|3924.17|4159.62|4409.10|4644.55|2085.27| +2451484|47770|2451509|4766|91025|621638|3804|11217|63558|1885021|2609|16022|43|22|12|4|199|376|8|58.33|166.82|143.46|186.88|1147.68|466.64|1334.56|34.43|0.00|80.00|1147.68|1182.11|1227.68|1262.11|681.04| +2451484|47770|2451511|17408|91025|621638|3804|11217|63558|1885021|2609|16022|26|28|13|4|55|376|49|47.35|78.12|7.81|3445.19|382.69|2320.15|3827.88|0.00|0.00|305.76|382.69|382.69|688.45|688.45|-1937.46| +2451484|47770|2451573|1315|91025|621638|3804|11217|63558|1885021|2609|16022|50|19|4|3|179|376|55|92.40|218.98|21.89|10839.95|1203.95|5082.00|12043.90|108.35|0.00|1083.50|1203.95|1312.30|2287.45|2395.80|-3878.05| +2451484|47770|2451530|3121|91025|621638|3804|11217|63558|1885021|2609|16022|56|1|3|5|217|376|73|45.77|101.60|8.12|6824.04|592.76|3341.21|7416.80|23.71|0.00|1557.09|592.76|616.47|2149.85|2173.56|-2748.45| +2452267|27193|2452356|6441|35223|1597909|148|3187|84239|1045454|4751|16237|7|3|3|1|192|377|54|22.61|25.09|1.00|1300.86|54.00|1220.94|1354.86|2.16|0.00|0.00|54.00|56.16|54.00|56.16|-1166.94| +2452267|27193|2452332|13441|35223|1597909|148|3187|84239|1045454|4751|16237|17|15|7|3|162|377|82|15.98|19.97|2.79|1408.76|228.78|1310.36|1637.54|1.39|89.22|540.38|139.56|140.95|679.94|681.33|-1170.80| +2452267|27193|2452351|9811|35223|1597909|148|3187|84239|1045454|4751|16237|43|3|15|3|138|377|1|32.83|45.96|0.00|45.96|0.00|32.83|45.96|0.00|0.00|22.52|0.00|0.00|22.52|22.52|-32.83| +2452267|27193|2452352|8163|35223|1597909|148|3187|84239|1045454|4751|16237|53|15|6|2|145|377|18|2.08|5.57|0.11|98.28|1.98|37.44|100.26|0.05|0.97|23.04|1.01|1.06|24.05|24.10|-36.43| +2452267|27193|2452382|16935|35223|1597909|148|3187|84239|1045454|4751|16237|17|29|1|2|93|377|46|91.10|181.28|87.01|4336.42|4002.46|4190.60|8338.88|200.12|0.00|2251.24|4002.46|4202.58|6253.70|6453.82|-188.14| +2452267|27193|2452365|12265|35223|1597909|148|3187|84239|1045454|4751|16237|45|13|4|3|245|377|93|21.73|29.11|23.57|515.22|2192.01|2020.89|2707.23|43.84|0.00|892.80|2192.01|2235.85|3084.81|3128.65|171.12| +2452267|27193|2452288|11983|35223|1597909|148|3187|84239|1045454|4751|16237|51|5|3|5|160|377|93|24.27|25.24|3.78|1995.78|351.54|2257.11|2347.32|3.51|0.00|821.19|351.54|355.05|1172.73|1176.24|-1905.57| +2452267|27193|2452309|6107|35223|1597909|148|3187|84239|1045454|4751|16237|23|17|13|5|217|377|75|71.64|129.66|116.69|972.75|8751.75|5373.00|9724.50|612.62|0.00|3403.50|8751.75|9364.37|12155.25|12767.87|3378.75| +2452267|27193|2452336|205|35223|1597909|148|3187|84239|1045454|4751|16237|43|9|15|5|125|377|99|96.55|207.58|157.76|4932.18|15618.24|9558.45|20550.42|149.93|13744.05|6781.50|1874.19|2024.12|8655.69|8805.62|-7684.26| +2452267|27193|2452285|12893|35223|1597909|148|3187|84239|1045454|4751|16237|35|29|2|4|96|377|14|55.04|117.78|48.28|973.00|675.92|770.56|1648.92|40.55|0.00|131.88|675.92|716.47|807.80|848.35|-94.64| +2452267|27193|2452371|14435|35223|1597909|148|3187|84239|1045454|4751|16237|19|1|19|2|55|377|15|40.71|96.07|91.26|72.15|1368.90|610.65|1441.05|54.75|0.00|288.15|1368.90|1423.65|1657.05|1711.80|758.25| +2452267|27193|2452311|9757|35223|1597909|148|3187|84239|1045454|4751|16237|13|7|1|3|228|377|37|62.60|185.29|44.46|5210.71|1645.02|2316.20|6855.73|32.90|0.00|1233.95|1645.02|1677.92|2878.97|2911.87|-671.18| +2451247|82271|2451250|16225|2068|1561568|5515|37462|33534|1741415|1656|29203|34|25|5|5|162|378|75|92.00|135.24|71.67|4767.75|5375.25|6900.00|10143.00|107.50|0.00|2332.50|5375.25|5482.75|7707.75|7815.25|-1524.75| +2451247|82271|2451336|3988|2068|1561568|5515|37462|33534|1741415|1656|29203|58|25|13|1|209|378|3|4.85|6.30|2.26|12.12|6.78|14.55|18.90|0.27|0.00|8.10|6.78|7.05|14.88|15.15|-7.77| +2451247|82271|2451250|4276|2068|1561568|5515|37462|33534|1741415|1656|29203|37|19|1|5|6|378|86|4.51|7.21|4.90|198.66|421.40|387.86|620.06|12.30|113.77|6.02|307.63|319.93|313.65|325.95|-80.23| +2451247|82271|2451267|6751|2068|1561568|5515|37462|33534|1741415|1656|29203|7|25|9|3|197|378|69|51.00|56.10|53.29|193.89|3677.01|3519.00|3870.90|52.94|1029.56|1625.64|2647.45|2700.39|4273.09|4326.03|-871.55| +2451247|82271|2451288|14612|2068|1561568|5515|37462|33534|1741415|1656|29203|49|1|7|3|176|378|46|4.61|6.82|6.41|18.86|294.86|212.06|313.72|17.69|0.00|140.76|294.86|312.55|435.62|453.31|82.80| +2451247|82271|2451281|6200|2068|1561568|5515|37462|33534|1741415|1656|29203|22|2|5|5|18|378|97|1.23|2.09|1.94|14.55|188.18|119.31|202.73|11.29|0.00|62.08|188.18|199.47|250.26|261.55|68.87| +2451247|82271|2451355|6247|2068|1561568|5515|37462|33534|1741415|1656|29203|58|7|1|5|272|378|37|9.26|23.61|9.44|524.29|349.28|342.62|873.57|10.47|0.00|305.62|349.28|359.75|654.90|665.37|6.66| +2451247|82271|2451356|16945|2068|1561568|5515|37462|33534|1741415|1656|29203|16|14|17|1|95|378|93|66.89|110.36|32.00|7287.48|2976.00|6220.77|10263.48|11.60|2589.12|4822.98|386.88|398.48|5209.86|5221.46|-5833.89| +2452085|24369|2452134|6467|44403|228632|2014|4243|30736|1867687|4883|28265|39|17|5|4|163|379|33|90.02|197.14|108.42|2927.76|3577.86|2970.66|6505.62|143.11|0.00|1756.26|3577.86|3720.97|5334.12|5477.23|607.20| +2452085|24369|2452138|16407|44403|228632|2014|4243|30736|1867687|4883|28265|59|7|11|3|115|379|90|77.35|86.63|78.83|702.00|7094.70|6961.50|7796.70|70.94|0.00|3663.90|7094.70|7165.64|10758.60|10829.54|133.20| +2452085|24369|2452201|16053|44403|228632|2014|4243|30736|1867687|4883|28265|7|9|16|4|135|379|29|36.39|87.33|55.89|911.76|1620.81|1055.31|2532.57|97.24|0.00|0.00|1620.81|1718.05|1620.81|1718.05|565.50| +2452085|24369|2452193|16229|44403|228632|2014|4243|30736|1867687|4883|28265|15|25|8|3|178|379|27|99.04|289.19|20.24|7261.65|546.48|2674.08|7808.13|49.18|0.00|1092.96|546.48|595.66|1639.44|1688.62|-2127.60| +2452085|24369|2452114|751|44403|228632|2014|4243|30736|1867687|4883|28265|23|1|7|3|169|379|94|51.30|98.49|31.51|6296.12|2961.94|4822.20|9258.06|119.95|1629.06|2684.64|1332.88|1452.83|4017.52|4137.47|-3489.32| +2452085|24369|2452188|3169|44403|228632|2014|4243|30736|1867687|4883|28265|27|29|3|4|205|379|6|1.65|3.82|3.28|3.24|19.68|9.90|22.92|0.59|0.00|3.18|19.68|20.27|22.86|23.45|9.78| +2452085|24369|2452136|9407|44403|228632|2014|4243|30736|1867687|4883|28265|45|27|15|4|250|379|28|49.71|81.52|75.81|159.88|2122.68|1391.88|2282.56|169.81|0.00|319.48|2122.68|2292.49|2442.16|2611.97|730.80| +2452085|24369|2452136|12487|44403|228632|2014|4243|30736|1867687|4883|28265|59|21|19|2|88|379|57|10.00|25.10|5.02|1144.56|286.14|570.00|1430.70|3.26|123.04|400.14|163.10|166.36|563.24|566.50|-406.90| +2452085|24369|2452091|16731|44403|228632|2014|4243|30736|1867687|4883|28265|1|25|8|5|245|379|86|54.50|62.13|45.35|1443.08|3900.10|4687.00|5343.18|195.00|0.00|480.74|3900.10|4095.10|4380.84|4575.84|-786.90| +2452085|24369|2452151|11139|44403|228632|2014|4243|30736|1867687|4883|28265|17|1|3|5|270|379|65|47.58|122.28|85.59|2384.85|5563.35|3092.70|7948.20|160.22|2892.94|1509.95|2670.41|2830.63|4180.36|4340.58|-422.29| +2452085|24369|2452089|3735|44403|228632|2014|4243|30736|1867687|4883|28265|9|13|12|2|59|379|97|24.56|55.01|51.70|321.07|5014.90|2382.32|5335.97|200.59|0.00|1973.95|5014.90|5215.49|6988.85|7189.44|2632.58| +2452085|24369|2452192|17889|44403|228632|2014|4243|30736|1867687|4883|28265|1|25|5|4|266|379|35|22.98|31.25|25.62|197.05|896.70|804.30|1093.75|35.86|0.00|448.35|896.70|932.56|1345.05|1380.91|92.40| +2452085|24369|2452145|15039|44403|228632|2014|4243|30736|1867687|4883|28265|51|23|18|4|60|379|50|52.55|55.17|51.85|166.00|2592.50|2627.50|2758.50|103.70|0.00|827.50|2592.50|2696.20|3420.00|3523.70|-35.00| +2452085|24369|2452131|13679|44403|228632|2014|4243|30736|1867687|4883|28265|27|29|9|5|34|379|5|5.06|11.89|2.02|49.35|10.10|25.30|59.45|0.70|0.00|0.00|10.10|10.80|10.10|10.80|-15.20| +2450955|59356|2451050|7472|13951|1477845|1739|22880|70688|679197|6562|6903|49|10|20|1|32|380|81|89.10|257.49|234.31|1877.58|18979.11|7217.10|20856.69|1138.74|0.00|5839.29|18979.11|20117.85|24818.40|25957.14|11762.01| +2450955|59356|2451005|788|13951|1477845|1739|22880|70688|679197|6562|6903|25|19|7|4|91|380|7|82.51|138.61|38.81|698.60|271.67|577.57|970.27|13.58|0.00|116.41|271.67|285.25|388.08|401.66|-305.90| +2450955|59356|2451003|15358|13951|1477845|1739|22880|70688|679197|6562|6903|31|13|15|1|204|380|65|24.54|56.19|52.25|256.10|3396.25|1595.10|3652.35|132.45|1188.68|1351.35|2207.57|2340.02|3558.92|3691.37|612.47| +2450955|59356|2450982|6056|13951|1477845|1739|22880|70688|679197|6562|6903|22|1|17|1|282|380|65|12.67|23.18|8.80|934.70|572.00|823.55|1506.70|11.44|0.00|300.95|572.00|583.44|872.95|884.39|-251.55| +2450955|59356|2451057|16652|13951|1477845|1739|22880|70688|679197|6562|6903|37|25|17|1|295|380|85|99.82|172.68|136.41|3082.95|11594.85|8484.70|14677.80|115.94|0.00|3522.40|11594.85|11710.79|15117.25|15233.19|3110.15| +2450955|59356|2450964|3448|13951|1477845|1739|22880|70688|679197|6562|6903|49|22|16|5|174|380|12|19.09|36.08|20.56|186.24|246.72|229.08|432.96|22.20|0.00|203.40|246.72|268.92|450.12|472.32|17.64| +2450955|59356|2450961|15367|13951|1477845|1739|22880|70688|679197|6562|6903|49|26|6|5|214|380|66|65.37|120.28|38.48|5398.80|2539.68|4314.42|7938.48|152.38|0.00|3492.72|2539.68|2692.06|6032.40|6184.78|-1774.74| +2450955|59356|2450975|16273|13951|1477845|1739|22880|70688|679197|6562|6903|56|20|16|3|280|380|2|98.01|233.26|104.96|256.60|209.92|196.02|466.52|10.49|0.00|139.94|209.92|220.41|349.86|360.35|13.90| +2450955|59356|2450958|3241|13951|1477845|1739|22880|70688|679197|6562|6903|44|4|20|2|264|380|58|4.67|7.84|4.70|182.12|272.60|270.86|454.72|16.35|0.00|113.68|272.60|288.95|386.28|402.63|1.74| +2450955|59356|2451074|8605|13951|1477845|1739|22880|70688|679197|6562|6903|40|7|4|4|297|380|61|51.40|65.27|57.43|478.24|3503.23|3135.40|3981.47|120.16|1786.64|79.30|1716.59|1836.75|1795.89|1916.05|-1418.81| +2450955|59356|2450986|3092|13951|1477845|1739|22880|70688|679197|6562|6903|14|8|2|4|183|380|99|16.74|45.19|12.65|3221.46|1252.35|1657.26|4473.81|37.57|0.00|1655.28|1252.35|1289.92|2907.63|2945.20|-404.91| +2452140|19754|2452230|7947|63208|1128351|2294|25837|98985|992172|4819|42362|21|25|15|4|236|381|73|5.72|7.89|0.23|559.18|16.79|417.56|575.97|0.50|0.00|212.43|16.79|17.29|229.22|229.72|-400.77| +2452140|19754|2452238|9503|63208|1128351|2294|25837|98985|992172|4819|42362|5|29|7|2|224|381|30|19.58|23.49|13.15|310.20|394.50|587.40|704.70|27.61|0.00|133.80|394.50|422.11|528.30|555.91|-192.90| +2452140|19754|2452160|2737|63208|1128351|2294|25837|98985|992172|4819|42362|7|13|10|3|106|381|36|4.30|12.81|2.43|373.68|87.48|154.80|461.16|0.00|0.00|165.96|87.48|87.48|253.44|253.44|-67.32| +2452140|19754|2452215|4251|63208|1128351|2294|25837|98985|992172|4819|42362|47|21|15|4|53|381|18|15.57|45.93|14.69|562.32|264.42|280.26|826.74|5.28|0.00|314.10|264.42|269.70|578.52|583.80|-15.84| +2452140|19754|2452161|15995|63208|1128351|2294|25837|98985|992172|4819|42362|33|29|3|1|39|381|90|70.95|100.03|10.00|8102.70|900.00|6385.50|9002.70|0.00|0.00|3600.90|900.00|900.00|4500.90|4500.90|-5485.50| +2452140|19754|2452236|11445|63208|1128351|2294|25837|98985|992172|4819|42362|29|15|8|3|31|381|53|78.68|81.82|16.36|3469.38|867.08|4170.04|4336.46|69.36|0.00|1213.70|867.08|936.44|2080.78|2150.14|-3302.96| +2452140|19754|2452208|12839|63208|1128351|2294|25837|98985|992172|4819|42362|11|25|3|5|20|381|46|89.56|195.24|19.52|8083.12|897.92|4119.76|8981.04|8.97|0.00|2514.36|897.92|906.89|3412.28|3421.25|-3221.84| +2452140|19754|2452211|15529|63208|1128351|2294|25837|98985|992172|4819|42362|3|29|11|2|100|381|2|30.53|44.57|16.49|56.16|32.98|61.06|89.14|0.65|0.00|31.18|32.98|33.63|64.16|64.81|-28.08| +2452140|19754|2452243|14259|63208|1128351|2294|25837|98985|992172|4819|42362|39|3|14|3|45|381|85|4.48|7.30|3.13|354.45|266.05|380.80|620.50|23.94|0.00|260.10|266.05|289.99|526.15|550.09|-114.75| +2452140|19754|2452231|9839|63208|1128351|2294|25837|98985|992172|4819|42362|43|13|19|4|216|381|23|98.11|259.01|49.21|4825.40|1131.83|2256.53|5957.23|90.54|0.00|1667.96|1131.83|1222.37|2799.79|2890.33|-1124.70| +2452140|19754|2452250|2387|63208|1128351|2294|25837|98985|992172|4819|42362|13|21|5|4|83|381|38|94.05|128.84|88.89|1518.10|3377.82|3573.90|4895.92|270.22|0.00|1664.40|3377.82|3648.04|5042.22|5312.44|-196.08| +2452140|19754|2452158|2131|63208|1128351|2294|25837|98985|992172|4819|42362|29|21|17|3|179|381|87|72.55|113.17|83.74|2560.41|7285.38|6311.85|9845.79|15.29|6775.40|2756.16|509.98|525.27|3266.14|3281.43|-5801.87| +2452140|19754|2452243|5653|63208|1128351|2294|25837|98985|992172|4819|42362|15|21|17|5|156|381|56|29.86|48.07|45.66|134.96|2556.96|1672.16|2691.92|0.00|0.00|591.92|2556.96|2556.96|3148.88|3148.88|884.80| +2452140|19754|2452248|16923|63208|1128351|2294|25837|98985|992172|4819|42362|55|3|16|5|224|381|39|43.87|52.20|38.62|529.62|1506.18|1710.93|2035.80|0.30|1476.05|244.14|30.13|30.43|274.27|274.57|-1680.80| +2452140|19754|2452205|5545|63208|1128351|2294|25837|98985|992172|4819|42362|49|5|8|1|244|381|12|31.51|72.15|12.98|710.04|155.76|378.12|865.80|0.00|0.00|398.16|155.76|155.76|553.92|553.92|-222.36| +2452140|19754|2452225|12067|63208|1128351|2294|25837|98985|992172|4819|42362|29|21|17|3|59|381|67|93.18|198.47|184.57|931.30|12366.19|6243.06|13297.49|865.63|0.00|1329.28|12366.19|13231.82|13695.47|14561.10|6123.13| +2450961|34811|2450995|3700|94718|1649918|3071|45071|94718|1649918|3071|45071|46|20|11|3|55|382|57|72.09|98.76|49.38|2814.66|2814.66|4109.13|5629.32|28.14|0.00|1576.05|2814.66|2842.80|4390.71|4418.85|-1294.47| +2450961|34811|2451009|9109|94718|1649918|3071|45071|94718|1649918|3071|45071|20|20|1|1|101|382|51|28.47|49.82|19.42|1550.40|990.42|1451.97|2540.82|19.80|0.00|1092.42|990.42|1010.22|2082.84|2102.64|-461.55| +2450961|34811|2451038|5600|94718|1649918|3071|45071|94718|1649918|3071|45071|2|16|3|5|224|382|64|67.23|144.54|39.02|6753.28|2497.28|4302.72|9250.56|199.78|0.00|4347.52|2497.28|2697.06|6844.80|7044.58|-1805.44| +2450961|34811|2451041|11354|94718|1649918|3071|45071|94718|1649918|3071|45071|14|26|3|4|18|382|59|70.40|161.92|51.81|6496.49|3056.79|4153.60|9553.28|30.56|0.00|0.00|3056.79|3087.35|3056.79|3087.35|-1096.81| +2450961|34811|2451053|15610|94718|1649918|3071|45071|94718|1649918|3071|45071|7|19|9|5|141|382|39|4.04|5.00|3.25|68.25|126.75|157.56|195.00|1.15|110.27|48.75|16.48|17.63|65.23|66.38|-141.08| +2450961|34811|2451018|9818|94718|1649918|3071|45071|94718|1649918|3071|45071|10|10|16|1|59|382|66|9.16|13.46|13.46|0.00|888.36|604.56|888.36|53.30|0.00|355.08|888.36|941.66|1243.44|1296.74|283.80| +2450961|34811|2451076|10555|94718|1649918|3071|45071|94718|1649918|3071|45071|34|10|4|2|150|382|10|1.88|4.77|3.38|13.90|33.80|18.80|47.70|2.70|0.00|10.90|33.80|36.50|44.70|47.40|15.00| +2450961|34811|2450992|1603|94718|1649918|3071|45071|94718|1649918|3071|45071|32|14|9|5|71|382|71|12.04|13.36|8.68|332.28|616.28|854.84|948.56|55.46|0.00|46.86|616.28|671.74|663.14|718.60|-238.56| +2450961|34811|2451039|17428|94718|1649918|3071|45071|94718|1649918|3071|45071|22|14|7|1|95|382|45|25.35|65.65|9.19|2540.70|413.55|1140.75|2954.25|33.08|0.00|502.20|413.55|446.63|915.75|948.83|-727.20| +2450961|34811|2450974|6596|94718|1649918|3071|45071|94718|1649918|3071|45071|22|13|9|1|36|382|63|31.07|38.52|36.20|146.16|2280.60|1957.41|2426.76|205.25|0.00|1018.71|2280.60|2485.85|3299.31|3504.56|323.19| +2450961|34811|2450996|17545|94718|1649918|3071|45071|94718|1649918|3071|45071|56|1|16|1|46|382|29|92.90|130.06|9.10|3507.84|263.90|2694.10|3771.74|21.11|0.00|1320.08|263.90|285.01|1583.98|1605.09|-2430.20| +2450961|34811|2450999|10622|94718|1649918|3071|45071|94718|1649918|3071|45071|49|14|1|4|77|382|91|62.36|122.84|88.44|3130.40|8048.04|5674.76|11178.44|563.36|0.00|2012.01|8048.04|8611.40|10060.05|10623.41|2373.28| +2450961|34811|2450979|9997|94718|1649918|3071|45071|94718|1649918|3071|45071|22|7|3|5|17|382|62|32.92|73.74|41.29|2011.90|2559.98|2041.04|4571.88|25.59|0.00|1371.44|2559.98|2585.57|3931.42|3957.01|518.94| +2451894|44636|2451963|12499|10171|803820|1022|23471|20569|206481|6724|35126|31|13|15|1|77|383|91|39.98|102.74|16.43|7854.21|1495.13|3638.18|9349.34|0.00|0.00|2523.43|1495.13|1495.13|4018.56|4018.56|-2143.05| +2451894|44636|2451976|14359|10171|803820|1022|23471|20569|206481|6724|35126|32|1|12|1|20|383|47|84.57|237.64|42.77|9158.89|2010.19|3974.79|11169.08|100.50|0.00|4579.21|2010.19|2110.69|6589.40|6689.90|-1964.60| +2451894|44636|2451976|7283|10171|803820|1022|23471|20569|206481|6724|35126|14|26|20|1|122|383|59|77.35|102.10|95.97|361.67|5662.23|4563.65|6023.90|452.97|0.00|1083.83|5662.23|6115.20|6746.06|7199.03|1098.58| +2451894|44636|2451975|17269|10171|803820|1022|23471|20569|206481|6724|35126|17|14|4|3|252|383|98|61.63|74.57|66.36|804.58|6503.28|6039.74|7307.86|260.13|0.00|3579.94|6503.28|6763.41|10083.22|10343.35|463.54| +2451894|44636|2451909|1916|10171|803820|1022|23471|20569|206481|6724|35126|2|11|10|4|138|383|10|89.10|155.03|13.95|1410.80|139.50|891.00|1550.30|4.18|0.00|294.50|139.50|143.68|434.00|438.18|-751.50| +2451894|44636|2452001|1553|10171|803820|1022|23471|20569|206481|6724|35126|41|17|20|1|102|383|45|42.10|66.09|38.33|1249.20|1724.85|1894.50|2974.05|120.73|0.00|981.00|1724.85|1845.58|2705.85|2826.58|-169.65| +2451894|44636|2451914|11617|10171|803820|1022|23471|20569|206481|6724|35126|23|1|17|1|72|383|16|63.46|103.43|97.22|99.36|1555.52|1015.36|1654.88|69.68|559.98|66.08|995.54|1065.22|1061.62|1131.30|-19.82| +2451894|44636|2451979|4495|10171|803820|1022|23471|20569|206481|6724|35126|23|1|20|4|215|383|77|44.69|69.71|48.09|1664.74|3702.93|3441.13|5367.67|19.99|1703.34|267.96|1999.59|2019.58|2267.55|2287.54|-1441.54| +2451894|44636|2451904|10669|10171|803820|1022|23471|20569|206481|6724|35126|25|26|15|1|47|383|74|43.88|75.91|69.07|506.16|5111.18|3247.12|5617.34|357.78|0.00|111.74|5111.18|5468.96|5222.92|5580.70|1864.06| +2451894|44636|2452003|15779|10171|803820|1022|23471|20569|206481|6724|35126|44|7|2|3|284|383|94|44.50|99.23|18.85|7555.72|1771.90|4183.00|9327.62|124.03|0.00|3543.80|1771.90|1895.93|5315.70|5439.73|-2411.10| +2451894|44636|2452006|17543|10171|803820|1022|23471|20569|206481|6724|35126|38|26|15|4|183|383|68|22.63|26.92|26.38|36.72|1793.84|1538.84|1830.56|161.44|0.00|109.48|1793.84|1955.28|1903.32|2064.76|255.00| +2451894|44636|2451995|146|10171|803820|1022|23471|20569|206481|6724|35126|2|2|2|4|279|383|61|94.47|259.79|57.15|12361.04|3486.15|5762.67|15847.19|209.16|0.00|4278.54|3486.15|3695.31|7764.69|7973.85|-2276.52| +2451894|44636|2451993|3428|10171|803820|1022|23471|20569|206481|6724|35126|20|14|15|5|142|383|88|52.08|92.18|86.64|487.52|7624.32|4583.04|8111.84|686.18|0.00|3244.56|7624.32|8310.50|10868.88|11555.06|3041.28| +2451754|78494|2451837|7327|46265|1288496|3880|24111|29574|601556|6310|22141|35|23|9|3|40|384|49|49.98|133.94|26.78|5250.84|1312.22|2449.02|6563.06|13.12|0.00|2427.95|1312.22|1325.34|3740.17|3753.29|-1136.80| +2451754|78494|2451863|1975|46265|1288496|3880|24111|29574|601556|6310|22141|44|8|10|5|230|384|83|48.77|135.09|91.86|3588.09|7624.38|4047.91|11212.47|304.97|0.00|1457.48|7624.38|7929.35|9081.86|9386.83|3576.47| +2451754|78494|2451787|7981|46265|1288496|3880|24111|29574|601556|6310|22141|47|7|17|5|140|384|37|98.62|134.12|128.75|198.69|4763.75|3648.94|4962.44|95.27|0.00|2381.69|4763.75|4859.02|7145.44|7240.71|1114.81| +2451754|78494|2451789|5366|46265|1288496|3880|24111|29574|601556|6310|22141|8|8|8|3|238|384|93|78.96|230.56|159.08|6647.64|14794.44|7343.28|21442.08|591.77|0.00|3644.67|14794.44|15386.21|18439.11|19030.88|7451.16| +2451754|78494|2451770|836|46265|1288496|3880|24111|29574|601556|6310|22141|32|17|8|5|232|384|45|95.08|200.61|48.14|6861.15|2166.30|4278.60|9027.45|129.97|0.00|4062.15|2166.30|2296.27|6228.45|6358.42|-2112.30| +2451754|78494|2451848|29|46265|1288496|3880|24111|29574|601556|6310|22141|37|23|8|1|277|384|93|67.43|176.66|28.26|13801.20|2628.18|6270.99|16429.38|236.53|0.00|4764.39|2628.18|2864.71|7392.57|7629.10|-3642.81| +2451754|78494|2451869|15857|46265|1288496|3880|24111|29574|601556|6310|22141|35|29|12|2|179|384|64|55.07|152.54|141.86|683.52|9079.04|3524.48|9762.56|147.08|4176.35|4685.44|4902.69|5049.77|9588.13|9735.21|1378.21| +2451754|78494|2451790|1808|46265|1288496|3880|24111|29574|601556|6310|22141|32|19|13|3|146|384|32|88.79|231.74|155.26|2447.36|4968.32|2841.28|7415.68|166.43|1639.54|1260.48|3328.78|3495.21|4589.26|4755.69|487.50| +2451754|78494|2451774|11455|46265|1288496|3880|24111|29574|601556|6310|22141|56|8|11|2|226|384|39|32.22|33.18|23.88|362.70|931.32|1256.58|1294.02|0.00|931.32|25.74|0.00|0.00|25.74|25.74|-1256.58| +2451754|78494|2451802|17357|46265|1288496|3880|24111|29574|601556|6310|22141|26|5|2|1|60|384|28|53.33|65.06|7.80|1603.28|218.40|1493.24|1821.68|7.73|89.54|200.20|128.86|136.59|329.06|336.79|-1364.38| +2451754|78494|2451864|9761|46265|1288496|3880|24111|29574|601556|6310|22141|29|2|16|3|270|384|83|74.40|162.93|50.50|9331.69|4191.50|6175.20|13523.19|335.32|0.00|5003.24|4191.50|4526.82|9194.74|9530.06|-1983.70| +2451754|78494|2451846|4199|46265|1288496|3880|24111|29574|601556|6310|22141|32|25|2|3|204|384|53|27.50|79.47|6.35|3875.36|336.55|1457.50|4211.91|16.82|0.00|547.49|336.55|353.37|884.04|900.86|-1120.95| +2451754|78494|2451807|2377|46265|1288496|3880|24111|29574|601556|6310|22141|47|25|2|1|160|384|53|5.53|7.96|5.09|152.11|269.77|293.09|421.88|8.09|0.00|54.59|269.77|277.86|324.36|332.45|-23.32| +2452152|42035|2452257|11549|81983|1389323|3063|7603|81983|1389323|3063|7603|9|19|5|5|115|385|76|34.37|99.67|17.94|6211.48|1363.44|2612.12|7574.92|95.44|0.00|3559.84|1363.44|1458.88|4923.28|5018.72|-1248.68| +2452152|42035|2452256|7303|81983|1389323|3063|7603|81983|1389323|3063|7603|9|23|12|2|49|385|45|14.91|30.11|10.23|894.60|460.35|670.95|1354.95|18.41|0.00|202.95|460.35|478.76|663.30|681.71|-210.60| +2452152|42035|2452236|12995|81983|1389323|3063|7603|81983|1389323|3063|7603|9|13|9|3|80|385|18|43.72|70.38|52.08|329.40|937.44|786.96|1266.84|84.36|0.00|367.38|937.44|1021.80|1304.82|1389.18|150.48| +2452152|42035|2452225|4485|81983|1389323|3063|7603|81983|1389323|3063|7603|21|27|4|2|90|385|20|3.09|7.81|0.78|140.60|15.60|61.80|156.20|0.01|14.97|17.00|0.63|0.64|17.63|17.64|-61.17| +2452152|42035|2452156|5183|81983|1389323|3063|7603|81983|1389323|3063|7603|11|3|5|1|244|385|64|6.70|12.19|0.97|718.08|62.08|428.80|780.16|0.00|0.00|280.32|62.08|62.08|342.40|342.40|-366.72| +2452152|42035|2452186|17525|81983|1389323|3063|7603|81983|1389323|3063|7603|39|3|4|1|248|385|78|78.25|117.37|82.15|2747.16|6407.70|6103.50|9154.86|256.30|0.00|91.26|6407.70|6664.00|6498.96|6755.26|304.20| +2452152|42035|2452246|1927|81983|1389323|3063|7603|81983|1389323|3063|7603|23|7|6|5|107|385|13|85.62|164.39|141.37|299.26|1837.81|1113.06|2137.07|36.75|0.00|705.12|1837.81|1874.56|2542.93|2579.68|724.75| +2452152|42035|2452153|16561|81983|1389323|3063|7603|81983|1389323|3063|7603|43|21|13|5|204|385|37|65.65|108.32|20.58|3246.38|761.46|2429.05|4007.84|45.68|0.00|1683.13|761.46|807.14|2444.59|2490.27|-1667.59| +2452152|42035|2452199|7585|81983|1389323|3063|7603|81983|1389323|3063|7603|37|11|20|2|131|385|19|9.19|21.22|11.24|189.62|213.56|174.61|403.18|10.67|0.00|11.97|213.56|224.23|225.53|236.20|38.95| +2452152|42035|2452210|1181|81983|1389323|3063|7603|81983|1389323|3063|7603|33|17|1|3|128|385|100|38.34|108.11|91.89|1622.00|9189.00|3834.00|10811.00|551.34|0.00|972.00|9189.00|9740.34|10161.00|10712.34|5355.00| +2452152|42035|2452186|9009|81983|1389323|3063|7603|81983|1389323|3063|7603|39|9|13|3|193|385|96|78.21|88.37|12.37|7296.00|1187.52|7508.16|8483.52|47.50|0.00|3308.16|1187.52|1235.02|4495.68|4543.18|-6320.64| +2452152|42035|2452211|3127|81983|1389323|3063|7603|81983|1389323|3063|7603|19|17|8|4|43|385|53|37.98|100.26|86.22|744.12|4569.66|2012.94|5313.78|365.57|0.00|2178.30|4569.66|4935.23|6747.96|7113.53|2556.72| +2452152|42035|2452161|7945|81983|1389323|3063|7603|81983|1389323|3063|7603|21|19|6|5|141|385|100|21.38|53.23|15.43|3780.00|1543.00|2138.00|5323.00|138.87|0.00|1171.00|1543.00|1681.87|2714.00|2852.87|-595.00| +2452029|51112|2452087|1883|61391|698774|4957|25093|34117|848962|6395|4578|55|17|7|5|183|386|69|20.01|34.01|9.18|1713.27|633.42|1380.69|2346.69|17.98|183.69|492.66|449.73|467.71|942.39|960.37|-930.96| +2452029|51112|2452120|3711|61391|698774|4957|25093|34117|848962|6395|4578|11|17|9|2|272|386|24|78.23|82.92|55.55|656.88|1333.20|1877.52|1990.08|93.32|0.00|238.80|1333.20|1426.52|1572.00|1665.32|-544.32| +2452029|51112|2452070|11067|61391|698774|4957|25093|34117|848962|6395|4578|51|19|7|1|218|386|75|98.70|118.44|47.37|5330.25|3552.75|7402.50|8883.00|106.58|0.00|621.75|3552.75|3659.33|4174.50|4281.08|-3849.75| +2452029|51112|2452080|10435|61391|698774|4957|25093|34117|848962|6395|4578|7|29|16|2|78|386|2|78.61|181.58|94.42|174.32|188.84|157.22|363.16|11.33|0.00|123.46|188.84|200.17|312.30|323.63|31.62| +2452029|51112|2452095|699|61391|698774|4957|25093|34117|848962|6395|4578|41|15|8|5|105|386|28|26.23|26.75|3.74|644.28|104.72|734.44|749.00|3.14|0.00|119.84|104.72|107.86|224.56|227.70|-629.72| +2452029|51112|2452097|14165|61391|698774|4957|25093|34117|848962|6395|4578|23|5|17|1|135|386|22|66.75|120.81|12.08|2392.06|265.76|1468.50|2657.82|0.00|0.00|451.66|265.76|265.76|717.42|717.42|-1202.74| +2452029|51112|2452089|613|61391|698774|4957|25093|34117|848962|6395|4578|7|5|2|1|180|386|33|45.98|112.65|73.22|1301.19|2416.26|1517.34|3717.45|96.65|0.00|36.96|2416.26|2512.91|2453.22|2549.87|898.92| +2452029|51112|2452126|12489|61391|698774|4957|25093|34117|848962|6395|4578|25|17|15|1|110|386|29|85.42|186.21|165.72|594.21|4805.88|2477.18|5400.09|240.29|0.00|2322.03|4805.88|5046.17|7127.91|7368.20|2328.70| +2452029|51112|2452142|999|61391|698774|4957|25093|34117|848962|6395|4578|11|15|10|2|254|386|87|87.06|161.93|72.86|7749.09|6338.82|7574.22|14087.91|126.77|0.00|5071.23|6338.82|6465.59|11410.05|11536.82|-1235.40| +2452029|51112|2452035|3057|61391|698774|4957|25093|34117|848962|6395|4578|25|1|9|4|242|386|60|45.75|66.33|19.89|2786.40|1193.40|2745.00|3979.80|83.53|0.00|556.80|1193.40|1276.93|1750.20|1833.73|-1551.60| +2452596|30844|2452664|17304|51515|916463|5411|33959|95140|1075749|50|44198|36|19|12|4|16|387|40|62.40|152.88|12.23|5626.00|489.20|2496.00|6115.20|14.67|0.00|2446.00|489.20|503.87|2935.20|2949.87|-2006.80| +2452596|30844|2452680|16680|51515|916463|5411|33959|95140|1075749|50|44198|3|30|8|3|31|387|31|37.30|41.40|9.93|975.57|307.83|1156.30|1283.40|24.62|0.00|615.97|307.83|332.45|923.80|948.42|-848.47| +2452596|30844|2452655|9027|51515|916463|5411|33959|95140|1075749|50|44198|51|7|2|5|58|387|39|49.94|125.84|42.78|3239.34|1668.42|1947.66|4907.76|33.36|0.00|2453.88|1668.42|1701.78|4122.30|4155.66|-279.24| +2452596|30844|2452660|378|51515|916463|5411|33959|95140|1075749|50|44198|55|24|9|1|190|387|36|30.19|39.54|6.72|1181.52|241.92|1086.84|1423.44|7.25|0.00|85.32|241.92|249.17|327.24|334.49|-844.92| +2452596|30844|2452649|624|51515|916463|5411|33959|95140|1075749|50|44198|55|25|15|5|252|387|51|97.56|149.26|17.91|6698.85|913.41|4975.56|7612.26|21.10|611.98|3501.15|301.43|322.53|3802.58|3823.68|-4674.13| +2452596|30844|2452675|6511|51515|916463|5411|33959|95140|1075749|50|44198|21|12|8|3|245|387|51|71.47|213.69|185.91|1416.78|9481.41|3644.97|10898.19|663.69|0.00|1525.41|9481.41|10145.10|11006.82|11670.51|5836.44| +2452596|30844|2452637|9219|51515|916463|5411|33959|95140|1075749|50|44198|49|1|7|3|259|387|21|65.26|124.64|112.17|261.87|2355.57|1370.46|2617.44|164.88|0.00|549.57|2355.57|2520.45|2905.14|3070.02|985.11| +2452596|30844|2452613|6351|51515|916463|5411|33959|95140|1075749|50|44198|36|19|2|1|203|387|10|33.45|57.19|0.57|566.20|5.70|334.50|571.90|0.34|0.00|85.70|5.70|6.04|91.40|91.74|-328.80| +2452596|30844|2452607|15450|51515|916463|5411|33959|95140|1075749|50|44198|57|3|18|4|267|387|68|20.15|33.65|32.30|91.80|2196.40|1370.20|2288.20|43.92|0.00|571.88|2196.40|2240.32|2768.28|2812.20|826.20| +2452596|30844|2452703|9615|51515|916463|5411|33959|95140|1075749|50|44198|60|3|4|1|158|387|76|74.76|151.76|139.61|923.40|10610.36|5681.76|11533.76|0.00|0.00|1729.76|10610.36|10610.36|12340.12|12340.12|4928.60| +2452596|30844|2452630|15774|51515|916463|5411|33959|95140|1075749|50|44198|51|19|7|1|165|387|67|75.20|216.57|110.45|7110.04|7400.15|5038.40|14510.19|370.00|0.00|6819.26|7400.15|7770.15|14219.41|14589.41|2361.75| +2452596|30844|2452621|11671|51515|916463|5411|33959|95140|1075749|50|44198|49|24|6|5|114|387|83|3.24|4.27|3.62|53.95|300.46|268.92|354.41|15.02|0.00|81.34|300.46|315.48|381.80|396.82|31.54| +2451394|43346|2451473|5176|28262|962533|4357|48581|26710|1487231|4900|31124|50|2|18|4|20|388|15|67.89|162.93|24.43|2077.50|366.45|1018.35|2443.95|3.66|0.00|733.05|366.45|370.11|1099.50|1103.16|-651.90| +2451394|43346|2451399|6794|28262|962533|4357|48581|26710|1487231|4900|31124|46|28|13|4|212|388|13|65.75|123.61|67.98|723.19|883.74|854.75|1606.93|44.18|0.00|192.79|883.74|927.92|1076.53|1120.71|28.99| +2451394|43346|2451485|4358|28262|962533|4357|48581|26710|1487231|4900|31124|16|14|10|4|291|388|95|60.99|159.79|113.45|4402.30|10777.75|5794.05|15180.05|431.11|0.00|5312.40|10777.75|11208.86|16090.15|16521.26|4983.70| +2451394|43346|2451464|17977|28262|962533|4357|48581|26710|1487231|4900|31124|56|20|10|2|121|388|71|31.52|77.22|61.00|1151.62|4331.00|2237.92|5482.62|173.24|0.00|164.01|4331.00|4504.24|4495.01|4668.25|2093.08| +2451394|43346|2451443|5461|28262|962533|4357|48581|26710|1487231|4900|31124|37|13|5|3|91|388|51|11.09|30.49|10.97|995.52|559.47|565.59|1554.99|16.78|0.00|481.95|559.47|576.25|1041.42|1058.20|-6.12| +2451394|43346|2451424|8161|28262|962533|4357|48581|26710|1487231|4900|31124|22|7|19|1|33|388|19|39.82|101.14|79.90|403.56|1518.10|756.58|1921.66|136.62|0.00|211.28|1518.10|1654.72|1729.38|1866.00|761.52| +2451394|43346|2451507|3421|28262|962533|4357|48581|26710|1487231|4900|31124|13|19|3|3|266|388|90|31.08|41.33|19.01|2008.80|1710.90|2797.20|3719.70|70.83|923.88|1226.70|787.02|857.85|2013.72|2084.55|-2010.18| +2451394|43346|2451404|8581|28262|962533|4357|48581|26710|1487231|4900|31124|20|10|18|3|238|388|75|16.44|42.74|4.70|2853.00|352.50|1233.00|3205.50|7.05|0.00|1218.00|352.50|359.55|1570.50|1577.55|-880.50| +2451394|43346|2451431|16027|28262|962533|4357|48581|26710|1487231|4900|31124|22|4|11|2|200|388|54|93.62|156.34|28.14|6922.80|1519.56|5055.48|8442.36|40.26|714.19|1772.82|805.37|845.63|2578.19|2618.45|-4250.11| +2451394|43346|2451445|15866|28262|962533|4357|48581|26710|1487231|4900|31124|55|25|18|3|208|388|2|48.06|92.75|4.63|176.24|9.26|96.12|185.50|0.27|0.00|9.26|9.26|9.53|18.52|18.79|-86.86| +2451394|43346|2451426|8062|28262|962533|4357|48581|26710|1487231|4900|31124|32|1|18|2|29|388|39|44.05|92.50|50.87|1623.57|1983.93|1717.95|3607.50|19.83|0.00|1082.25|1983.93|2003.76|3066.18|3086.01|265.98| +2451394|43346|2451492|3998|28262|962533|4357|48581|26710|1487231|4900|31124|10|16|9|2|145|388|91|76.71|95.88|16.29|7242.69|1482.39|6980.61|8725.08|0.00|0.00|173.81|1482.39|1482.39|1656.20|1656.20|-5498.22| +2451002|11027|2451088|2252|41528|1109276|3071|27266|18286|171766|3404|2799|44|13|1|2|206|389|95|54.06|68.65|27.46|3913.05|2608.70|5135.70|6521.75|52.17|0.00|130.15|2608.70|2660.87|2738.85|2791.02|-2527.00| +2451002|11027|2451116|17684|41528|1109276|3071|27266|18286|171766|3404|2799|4|4|13|3|296|389|63|25.93|69.23|8.99|3795.12|566.37|1633.59|4361.49|0.51|560.70|2180.43|5.67|6.18|2186.10|2186.61|-1627.92| +2451002|11027|2451065|15682|41528|1109276|3071|27266|18286|171766|3404|2799|7|22|3|1|52|389|18|63.87|136.68|121.64|270.72|2189.52|1149.66|2460.24|109.47|0.00|738.00|2189.52|2298.99|2927.52|3036.99|1039.86| +2451002|11027|2451081|13382|41528|1109276|3071|27266|18286|171766|3404|2799|43|26|16|1|193|389|73|41.83|113.35|113.35|0.00|8274.55|3053.59|8274.55|744.70|0.00|1241.00|8274.55|9019.25|9515.55|10260.25|5220.96| +2451002|11027|2451048|2576|41528|1109276|3071|27266|18286|171766|3404|2799|10|2|18|4|170|389|37|47.69|125.42|26.33|3666.33|974.21|1764.53|4640.54|9.74|0.00|2227.40|974.21|983.95|3201.61|3211.35|-790.32| +2451002|11027|2451004|2390|41528|1109276|3071|27266|18286|171766|3404|2799|50|7|15|1|187|389|20|72.73|197.09|197.09|0.00|3941.80|1454.60|3941.80|203.39|551.85|1931.40|3389.95|3593.34|5321.35|5524.74|1935.35| +2451002|11027|2451074|2846|41528|1109276|3071|27266|18286|171766|3404|2799|38|10|19|5|68|389|17|38.71|59.61|5.36|922.25|91.12|658.07|1013.37|2.73|0.00|111.35|91.12|93.85|202.47|205.20|-566.95| +2451002|11027|2451031|8329|41528|1109276|3071|27266|18286|171766|3404|2799|10|22|8|4|90|389|28|85.69|206.51|148.68|1619.24|4163.04|2399.32|5782.28|0.00|0.00|2081.52|4163.04|4163.04|6244.56|6244.56|1763.72| +2451002|11027|2451102|8054|41528|1109276|3071|27266|18286|171766|3404|2799|2|1|20|2|231|389|70|13.93|31.06|13.35|1239.70|934.50|975.10|2174.20|28.03|0.00|630.00|934.50|962.53|1564.50|1592.53|-40.60| +2451002|11027|2451004|16858|41528|1109276|3071|27266|18286|171766|3404|2799|28|25|8|3|148|389|8|27.48|46.44|33.90|100.32|271.20|219.84|371.52|16.27|0.00|55.68|271.20|287.47|326.88|343.15|51.36| +2451002|11027|2451047|9400|41528|1109276|3071|27266|18286|171766|3404|2799|14|8|2|4|176|389|88|1.22|2.65|1.72|81.84|151.36|107.36|233.20|10.59|0.00|39.60|151.36|161.95|190.96|201.55|44.00| +2452164|20863|2452261|15655|13831|1882559|2934|48109|20|1757547|4787|38362|21|19|19|3|157|390|66|64.27|178.02|32.04|9634.68|2114.64|4241.82|11749.32|0.00|0.00|469.92|2114.64|2114.64|2584.56|2584.56|-2127.18| +2452164|20863|2452274|8663|13831|1882559|2934|48109|20|1757547|4787|38362|57|27|9|4|82|390|60|36.73|85.21|53.68|1891.80|3220.80|2203.80|5112.60|128.83|0.00|970.80|3220.80|3349.63|4191.60|4320.43|1017.00| +2452164|20863|2452233|10657|13831|1882559|2934|48109|20|1757547|4787|38362|11|11|19|2|225|390|21|33.43|89.92|62.94|566.58|1321.74|702.03|1888.32|28.41|753.39|18.69|568.35|596.76|587.04|615.45|-133.68| +2452164|20863|2452197|8487|13831|1882559|2934|48109|20|1757547|4787|38362|47|19|9|5|74|390|95|40.91|58.91|11.78|4477.35|1119.10|3886.45|5596.45|100.71|0.00|2238.20|1119.10|1219.81|3357.30|3458.01|-2767.35| +2452164|20863|2452227|9165|13831|1882559|2934|48109|20|1757547|4787|38362|33|15|9|5|48|390|42|73.45|102.09|9.18|3902.22|385.56|3084.90|4287.78|11.56|0.00|2058.00|385.56|397.12|2443.56|2455.12|-2699.34| +2452164|20863|2452173|6939|13831|1882559|2934|48109|20|1757547|4787|38362|43|11|10|4|108|390|48|27.47|33.23|1.32|1531.68|63.36|1318.56|1595.04|1.90|0.00|350.88|63.36|65.26|414.24|416.14|-1255.20| +2452164|20863|2452174|2093|13831|1882559|2934|48109|20|1757547|4787|38362|3|7|19|1|266|390|76|39.13|80.99|72.08|677.16|5478.08|2973.88|6155.24|232.27|2574.69|2276.96|2903.39|3135.66|5180.35|5412.62|-70.49| +2452164|20863|2452224|6257|13831|1882559|2934|48109|20|1757547|4787|38362|3|7|13|5|234|390|4|24.54|27.48|20.06|29.68|80.24|98.16|109.92|4.01|0.00|52.76|80.24|84.25|133.00|137.01|-17.92| +2452164|20863|2452283|4287|13831|1882559|2934|48109|20|1757547|4787|38362|7|21|8|3|55|390|31|67.79|146.42|17.57|3994.35|544.67|2101.49|4539.02|49.02|0.00|226.92|544.67|593.69|771.59|820.61|-1556.82| +2452164|20863|2452257|13665|13831|1882559|2934|48109|20|1757547|4787|38362|37|25|7|4|228|390|1|33.06|55.54|14.99|40.55|14.99|33.06|55.54|0.29|0.00|2.22|14.99|15.28|17.21|17.50|-18.07| +2452164|20863|2452170|16063|13831|1882559|2934|48109|20|1757547|4787|38362|25|25|20|5|50|390|75|6.24|13.47|1.88|869.25|141.00|468.00|1010.25|3.63|19.74|414.00|121.26|124.89|535.26|538.89|-346.74| +2452164|20863|2452200|13739|13831|1882559|2934|48109|20|1757547|4787|38362|5|15|3|3|113|390|56|21.42|42.41|34.77|427.84|1947.12|1199.52|2374.96|77.88|0.00|498.40|1947.12|2025.00|2445.52|2523.40|747.60| +2452164|20863|2452248|4141|13831|1882559|2934|48109|20|1757547|4787|38362|7|9|16|2|8|390|14|54.39|146.85|58.74|1233.54|822.36|761.46|2055.90|65.78|0.00|637.28|822.36|888.14|1459.64|1525.42|60.90| +2452164|20863|2452181|13245|13831|1882559|2934|48109|20|1757547|4787|38362|31|3|5|3|124|390|6|64.49|167.02|118.58|290.64|711.48|386.94|1002.12|8.89|533.61|230.46|177.87|186.76|408.33|417.22|-209.07| +2452156|75757|2452233|12583|85228|521217|1653|31642|61232|1847948|2856|49292|5|17|15|4|290|391|28|67.02|136.05|21.76|3200.12|609.28|1876.56|3809.40|18.27|0.00|838.04|609.28|627.55|1447.32|1465.59|-1267.28| +2452156|75757|2452201|4569|85228|521217|1653|31642|61232|1847948|2856|49292|21|15|12|1|290|391|20|46.46|82.69|14.05|1372.80|281.00|929.20|1653.80|8.43|0.00|760.60|281.00|289.43|1041.60|1050.03|-648.20| +2452156|75757|2452228|16113|85228|521217|1653|31642|61232|1847948|2856|49292|59|17|11|5|172|391|84|84.92|157.10|114.68|3563.28|9633.12|7133.28|13196.40|346.79|3853.24|2374.68|5779.88|6126.67|8154.56|8501.35|-1353.40| +2452156|75757|2452276|11187|85228|521217|1653|31642|61232|1847948|2856|49292|9|21|14|5|253|391|18|45.76|97.01|18.43|1414.44|331.74|823.68|1746.18|10.88|59.71|174.60|272.03|282.91|446.63|457.51|-551.65| +2452156|75757|2452163|14293|85228|521217|1653|31642|61232|1847948|2856|49292|19|15|9|2|205|391|75|77.76|200.62|22.06|13392.00|1654.50|5832.00|15046.50|57.90|496.35|3912.00|1158.15|1216.05|5070.15|5128.05|-4673.85| +2452156|75757|2452245|15409|85228|521217|1653|31642|61232|1847948|2856|49292|13|15|9|2|198|391|66|42.14|93.55|16.83|5063.52|1110.78|2781.24|6174.30|44.43|0.00|863.94|1110.78|1155.21|1974.72|2019.15|-1670.46| +2452156|75757|2452233|3659|85228|521217|1653|31642|61232|1847948|2856|49292|21|11|18|3|88|391|75|28.03|52.13|14.07|2854.50|1055.25|2102.25|3909.75|10.55|0.00|938.25|1055.25|1065.80|1993.50|2004.05|-1047.00| +2452156|75757|2452184|14579|85228|521217|1653|31642|61232|1847948|2856|49292|35|21|4|2|110|391|71|62.12|101.25|11.13|6398.52|790.23|4410.52|7188.75|0.00|0.00|3594.02|790.23|790.23|4384.25|4384.25|-3620.29| +2452156|75757|2452214|12671|85228|521217|1653|31642|61232|1847948|2856|49292|21|25|15|2|199|391|79|62.93|179.97|80.98|7820.21|6397.42|4971.47|14217.63|0.00|0.00|710.21|6397.42|6397.42|7107.63|7107.63|1425.95| +2452156|75757|2452180|7387|85228|521217|1653|31642|61232|1847948|2856|49292|39|3|11|5|29|391|84|9.75|11.60|9.39|185.64|788.76|819.00|974.40|39.43|0.00|126.00|788.76|828.19|914.76|954.19|-30.24| +2452156|75757|2452186|12789|85228|521217|1653|31642|61232|1847948|2856|49292|53|9|13|1|59|391|21|5.13|12.97|7.78|108.99|163.38|107.73|272.37|4.90|0.00|122.43|163.38|168.28|285.81|290.71|55.65| +2452156|75757|2452177|11589|85228|521217|1653|31642|61232|1847948|2856|49292|59|5|9|2|275|391|29|80.84|90.54|62.47|814.03|1811.63|2344.36|2625.66|18.11|0.00|498.80|1811.63|1829.74|2310.43|2328.54|-532.73| +2452156|75757|2452222|15765|85228|521217|1653|31642|61232|1847948|2856|49292|37|17|19|2|182|391|46|2.82|8.46|6.42|93.84|295.32|129.72|389.16|0.00|0.00|42.78|295.32|295.32|338.10|338.10|165.60| +2452156|75757|2452240|14379|85228|521217|1653|31642|61232|1847948|2856|49292|41|9|19|3|77|391|85|77.84|81.73|4.90|6530.55|416.50|6616.40|6947.05|29.15|0.00|2292.45|416.50|445.65|2708.95|2738.10|-6199.90| +2452156|75757|2452198|4393|85228|521217|1653|31642|61232|1847948|2856|49292|51|15|20|4|231|391|35|5.17|11.83|1.89|347.90|66.15|180.95|414.05|4.63|0.00|115.85|66.15|70.78|182.00|186.63|-114.80| +2450872|46061|2450976|1148|63128|1423320|4312|504|63128|1423320|4312|504|32|4|9|5|201|392|46|73.86|130.73|104.58|1202.90|4810.68|3397.56|6013.58|48.10|0.00|2044.24|4810.68|4858.78|6854.92|6903.02|1413.12| +2450872|46061|2450926|13660|63128|1423320|4312|504|63128|1423320|4312|504|14|10|7|3|69|392|32|1.55|4.57|1.41|101.12|45.12|49.60|146.24|0.45|0.00|34.88|45.12|45.57|80.00|80.45|-4.48| +2450872|46061|2450977|16346|63128|1423320|4312|504|63128|1423320|4312|504|4|20|16|4|229|392|96|2.66|6.22|2.48|359.04|238.08|255.36|597.12|11.90|0.00|267.84|238.08|249.98|505.92|517.82|-17.28| +2450872|46061|2450975|4444|63128|1423320|4312|504|63128|1423320|4312|504|44|22|11|5|249|392|67|92.34|121.88|29.25|6206.21|1959.75|6186.78|8165.96|104.06|803.49|1061.28|1156.26|1260.32|2217.54|2321.60|-5030.52| +2450872|46061|2450957|12836|63128|1423320|4312|504|63128|1423320|4312|504|46|7|1|3|138|392|11|19.25|47.93|34.50|147.73|379.50|211.75|527.23|30.36|0.00|179.19|379.50|409.86|558.69|589.05|167.75| +2450872|46061|2450917|5161|63128|1423320|4312|504|63128|1423320|4312|504|44|25|16|4|299|392|37|66.22|130.45|88.70|1544.75|3281.90|2450.14|4826.65|98.45|0.00|0.00|3281.90|3380.35|3281.90|3380.35|831.76| +2450872|46061|2450890|14986|63128|1423320|4312|504|63128|1423320|4312|504|49|7|13|2|151|392|56|27.79|80.86|32.34|2717.12|1811.04|1556.24|4528.16|99.42|706.30|452.48|1104.74|1204.16|1557.22|1656.64|-451.50| +2450872|46061|2450935|16186|63128|1423320|4312|504|63128|1423320|4312|504|49|28|8|3|211|392|67|67.79|100.32|7.02|6251.10|470.34|4541.93|6721.44|4.51|319.83|2016.03|150.51|155.02|2166.54|2171.05|-4391.42| +2450872|46061|2450930|14149|63128|1423320|4312|504|63128|1423320|4312|504|2|1|20|4|118|392|83|60.94|123.09|41.85|6742.92|3473.55|5058.02|10216.47|173.67|0.00|2349.73|3473.55|3647.22|5823.28|5996.95|-1584.47| +2450872|46061|2450977|6500|63128|1423320|4312|504|63128|1423320|4312|504|50|1|6|3|203|392|63|16.40|21.32|3.83|1101.87|241.29|1033.20|1343.16|2.41|0.00|335.79|241.29|243.70|577.08|579.49|-791.91| +2450872|46061|2450989|2990|63128|1423320|4312|504|63128|1423320|4312|504|25|13|11|5|19|392|82|78.78|189.85|134.79|4514.92|11052.78|6459.96|15567.70|110.52|0.00|4202.50|11052.78|11163.30|15255.28|15365.80|4592.82| +2450872|46061|2450918|4297|63128|1423320|4312|504|63128|1423320|4312|504|8|4|1|4|110|392|5|30.23|46.85|37.94|44.55|189.70|151.15|234.25|11.38|0.00|74.95|189.70|201.08|264.65|276.03|38.55| +2450872|46061|2450933|16507|63128|1423320|4312|504|63128|1423320|4312|504|10|13|11|2|206|392|24|72.27|109.85|41.74|1634.64|1001.76|1734.48|2636.40|40.07|0.00|553.44|1001.76|1041.83|1555.20|1595.27|-732.72| +2450872|46061|2450881|4562|63128|1423320|4312|504|63128|1423320|4312|504|40|7|17|3|276|392|49|60.91|168.11|142.89|1235.78|7001.61|2984.59|8237.39|210.04|0.00|3706.36|7001.61|7211.65|10707.97|10918.01|4017.02| +2451679|42527|2451740|15731|29494|1058247|4953|18506|27218|722783|6718|31424|44|17|2|3|41|393|68|92.20|198.23|31.71|11323.36|2156.28|6269.60|13479.64|150.93|0.00|5661.00|2156.28|2307.21|7817.28|7968.21|-4113.32| +2451679|42527|2451701|15653|29494|1058247|4953|18506|27218|722783|6718|31424|13|17|4|1|181|393|57|90.86|213.52|66.19|8397.81|3772.83|5179.02|12170.64|301.82|0.00|4016.22|3772.83|4074.65|7789.05|8090.87|-1406.19| +2451679|42527|2451776|10976|29494|1058247|4953|18506|27218|722783|6718|31424|50|29|4|1|95|393|84|95.65|129.12|38.73|7592.76|3253.32|8034.60|10846.08|221.22|487.99|4012.68|2765.33|2986.55|6778.01|6999.23|-5269.27| +2451679|42527|2451757|7010|29494|1058247|4953|18506|27218|722783|6718|31424|41|5|12|1|30|393|7|19.11|38.41|11.52|188.23|80.64|133.77|268.87|2.41|0.00|51.03|80.64|83.05|131.67|134.08|-53.13| +2451679|42527|2451758|407|29494|1058247|4953|18506|27218|722783|6718|31424|17|8|11|2|23|393|91|47.79|119.47|105.13|1304.94|9566.83|4348.89|10871.77|0.00|0.00|4565.47|9566.83|9566.83|14132.30|14132.30|5217.94| +2451679|42527|2451749|16723|29494|1058247|4953|18506|27218|722783|6718|31424|53|7|1|5|242|393|46|34.98|102.84|20.56|3784.88|945.76|1609.08|4730.64|0.00|0.00|567.64|945.76|945.76|1513.40|1513.40|-663.32| +2451679|42527|2451692|11582|29494|1058247|4953|18506|27218|722783|6718|31424|59|7|8|2|23|393|33|68.76|114.14|29.67|2787.51|979.11|2269.08|3766.62|48.95|0.00|376.53|979.11|1028.06|1355.64|1404.59|-1289.97| +2451679|42527|2451701|12491|29494|1058247|4953|18506|27218|722783|6718|31424|7|7|11|1|198|393|28|86.32|130.34|72.99|1605.80|2043.72|2416.96|3649.52|183.93|0.00|1605.52|2043.72|2227.65|3649.24|3833.17|-373.24| +2451679|42527|2451788|13046|29494|1058247|4953|18506|27218|722783|6718|31424|44|1|6|4|92|393|23|30.38|37.06|30.01|162.15|690.23|698.74|852.38|38.65|207.06|400.43|483.17|521.82|883.60|922.25|-215.57| +2451679|42527|2451723|11429|29494|1058247|4953|18506|27218|722783|6718|31424|38|19|3|3|243|393|97|73.37|178.28|139.05|3805.31|13487.85|7116.89|17293.16|269.75|0.00|1728.54|13487.85|13757.60|15216.39|15486.14|6370.96| +2451679|42527|2451779|15401|29494|1058247|4953|18506|27218|722783|6718|31424|26|25|2|1|220|393|66|39.38|109.08|81.81|1799.82|5399.46|2599.08|7199.28|107.98|0.00|2591.16|5399.46|5507.44|7990.62|8098.60|2800.38| +2451679|42527|2451786|4628|29494|1058247|4953|18506|27218|722783|6718|31424|5|17|18|4|200|393|93|21.17|61.81|6.79|5116.86|631.47|1968.81|5748.33|25.25|0.00|2414.28|631.47|656.72|3045.75|3071.00|-1337.34| +2451679|42527|2451705|15331|29494|1058247|4953|18506|27218|722783|6718|31424|59|20|12|3|284|393|48|3.02|7.30|1.09|298.08|52.32|144.96|350.40|0.52|0.00|31.20|52.32|52.84|83.52|84.04|-92.64| +2451679|42527|2451686|14905|29494|1058247|4953|18506|27218|722783|6718|31424|53|8|3|1|209|393|39|31.23|84.63|7.61|3003.78|296.79|1217.97|3300.57|14.83|0.00|1650.09|296.79|311.62|1946.88|1961.71|-921.18| +2451679|42527|2451744|14603|29494|1058247|4953|18506|27218|722783|6718|31424|49|17|13|2|81|393|7|8.05|11.91|10.48|10.01|73.36|56.35|83.37|4.40|0.00|36.68|73.36|77.76|110.04|114.44|17.01| +2451679|42527|2451777|10079|29494|1058247|4953|18506|27218|722783|6718|31424|26|1|8|5|232|393|36|16.86|43.49|2.60|1472.04|93.60|606.96|1565.64|3.74|0.00|688.68|93.60|97.34|782.28|786.02|-513.36| +2451427|70121|2451519|12046|35005|1400954|4503|19299|81424|953156|4393|37142|38|1|12|5|161|394|22|5.50|15.18|10.47|103.62|230.34|121.00|333.96|6.91|0.00|16.50|230.34|237.25|246.84|253.75|109.34| +2451427|70121|2451446|17971|35005|1400954|4503|19299|81424|953156|4393|37142|8|26|13|2|251|394|33|43.87|51.76|24.32|905.52|802.56|1447.71|1708.08|0.00|0.00|580.47|802.56|802.56|1383.03|1383.03|-645.15| +2451427|70121|2451438|4171|35005|1400954|4503|19299|81424|953156|4393|37142|7|1|1|3|285|394|17|95.95|136.24|57.22|1343.34|972.74|1631.15|2316.08|0.00|515.55|347.31|457.19|457.19|804.50|804.50|-1173.96| +2451427|70121|2451452|844|35005|1400954|4503|19299|81424|953156|4393|37142|8|16|13|2|21|394|22|33.54|96.93|83.35|298.76|1833.70|737.88|2132.46|73.34|0.00|1044.78|1833.70|1907.04|2878.48|2951.82|1095.82| +2451427|70121|2451446|8233|35005|1400954|4503|19299|81424|953156|4393|37142|16|20|3|5|197|394|68|95.89|209.04|62.71|9950.44|4264.28|6520.52|14214.72|255.85|0.00|2131.80|4264.28|4520.13|6396.08|6651.93|-2256.24| +2451427|70121|2451483|14002|35005|1400954|4503|19299|81424|953156|4393|37142|22|16|6|3|263|394|59|9.63|16.65|2.66|825.41|156.94|568.17|982.35|7.84|0.00|471.41|156.94|164.78|628.35|636.19|-411.23| +2451427|70121|2451450|14791|35005|1400954|4503|19299|81424|953156|4393|37142|37|26|9|4|179|394|84|16.57|19.88|7.75|1018.92|651.00|1391.88|1669.92|39.06|0.00|49.56|651.00|690.06|700.56|739.62|-740.88| +2451427|70121|2451500|11449|35005|1400954|4503|19299|81424|953156|4393|37142|16|1|3|5|90|394|75|13.11|16.38|3.11|995.25|233.25|983.25|1228.50|1.63|200.59|270.00|32.66|34.29|302.66|304.29|-950.59| +2451427|70121|2451459|934|35005|1400954|4503|19299|81424|953156|4393|37142|49|10|14|2|210|394|92|85.14|244.35|0.00|22480.20|0.00|7832.88|22480.20|0.00|0.00|11015.16|0.00|0.00|11015.16|11015.16|-7832.88| +2451427|70121|2451446|1360|35005|1400954|4503|19299|81424|953156|4393|37142|8|2|9|1|173|394|74|40.01|106.42|92.58|1024.16|6850.92|2960.74|7875.08|65.76|5206.69|236.06|1644.23|1709.99|1880.29|1946.05|-1316.51| +2451427|70121|2451503|475|35005|1400954|4503|19299|81424|953156|4393|37142|20|8|2|1|273|394|11|56.05|107.61|78.55|319.66|864.05|616.55|1183.71|69.12|0.00|591.80|864.05|933.17|1455.85|1524.97|247.50| +2451427|70121|2451528|15427|35005|1400954|4503|19299|81424|953156|4393|37142|50|4|7|4|158|394|93|45.66|75.79|24.25|4793.22|2255.25|4246.38|7048.47|63.14|1353.15|2466.36|902.10|965.24|3368.46|3431.60|-3344.28| +2451768|79002|2451824|17828|35817|267890|7163|36083|87616|1808972|702|25222|25|8|7|4|191|395|46|76.89|163.00|32.60|5998.40|1499.60|3536.94|7498.00|104.97|0.00|1799.52|1499.60|1604.57|3299.12|3404.09|-2037.34| +2451768|79002|2451838|14345|35817|267890|7163|36083|87616|1808972|702|25222|8|13|16|3|57|395|89|52.39|80.68|16.94|5672.86|1507.66|4662.71|7180.52|90.45|0.00|2081.71|1507.66|1598.11|3589.37|3679.82|-3155.05| +2451768|79002|2451874|9830|35817|267890|7163|36083|87616|1808972|702|25222|25|5|13|3|10|395|6|16.21|29.34|23.17|37.02|139.02|97.26|176.04|3.50|51.43|10.56|87.59|91.09|98.15|101.65|-9.67| +2451768|79002|2451853|16250|35817|267890|7163|36083|87616|1808972|702|25222|20|26|7|5|106|395|79|8.49|23.85|15.50|659.65|1224.50|670.71|1884.15|61.22|0.00|413.96|1224.50|1285.72|1638.46|1699.68|553.79| +2451768|79002|2451810|323|35817|267890|7163|36083|87616|1808972|702|25222|5|25|6|4|29|395|56|28.85|79.04|64.02|841.12|3585.12|1615.60|4426.24|35.85|0.00|44.24|3585.12|3620.97|3629.36|3665.21|1969.52| +2451768|79002|2451812|3859|35817|267890|7163|36083|87616|1808972|702|25222|8|23|12|2|289|395|9|19.00|31.35|11.59|177.84|104.31|171.00|282.15|2.55|67.80|118.44|36.51|39.06|154.95|157.50|-134.49| +2451768|79002|2451795|17633|35817|267890|7163|36083|87616|1808972|702|25222|7|25|2|1|280|395|74|2.31|6.05|3.81|165.76|281.94|170.94|447.70|0.00|0.00|160.58|281.94|281.94|442.52|442.52|111.00| +2451768|79002|2451864|2843|35817|267890|7163|36083|87616|1808972|702|25222|5|20|19|2|35|395|53|2.50|7.50|5.40|111.30|286.20|132.50|397.50|22.89|0.00|122.96|286.20|309.09|409.16|432.05|153.70| +2451768|79002|2451834|6037|35817|267890|7163|36083|87616|1808972|702|25222|8|25|14|2|270|395|79|57.30|65.89|36.89|2291.00|2914.31|4526.70|5205.31|65.57|728.57|884.80|2185.74|2251.31|3070.54|3136.11|-2340.96| +2451198|83820|2451206|7732|94012|37354|1943|14085|72059|1425697|6958|46300|31|26|12|2|255|396|80|24.61|34.70|3.12|2526.40|249.60|1968.80|2776.00|2.69|114.81|27.20|134.79|137.48|161.99|164.68|-1834.01| +2451198|83820|2451305|16736|94012|37354|1943|14085|72059|1425697|6958|46300|22|8|5|3|88|396|65|95.99|202.53|111.39|5924.10|7240.35|6239.35|13164.45|579.22|0.00|1184.30|7240.35|7819.57|8424.65|9003.87|1001.00| +2451198|83820|2451238|1438|94012|37354|1943|14085|72059|1425697|6958|46300|34|8|15|2|12|396|91|73.45|149.83|145.33|409.50|13225.03|6683.95|13634.53|1190.25|0.00|3680.95|13225.03|14415.28|16905.98|18096.23|6541.08| +2451198|83820|2451255|1693|94012|37354|1943|14085|72059|1425697|6958|46300|25|28|4|3|275|396|28|29.12|77.45|17.81|1669.92|498.68|815.36|2168.60|24.93|0.00|542.08|498.68|523.61|1040.76|1065.69|-316.68| +2451198|83820|2451288|4354|94012|37354|1943|14085|72059|1425697|6958|46300|49|10|4|5|46|396|52|56.43|145.58|14.55|6813.56|756.60|2934.36|7570.16|52.96|0.00|1059.76|756.60|809.56|1816.36|1869.32|-2177.76| +2451198|83820|2451254|12130|94012|37354|1943|14085|72059|1425697|6958|46300|34|2|9|3|180|396|81|88.62|121.40|111.68|787.32|9046.08|7178.22|9833.40|51.56|3889.81|983.34|5156.27|5207.83|6139.61|6191.17|-2021.95| +2451198|83820|2451289|11054|94012|37354|1943|14085|72059|1425697|6958|46300|44|25|16|1|119|396|11|64.33|120.29|113.07|79.42|1243.77|707.63|1323.19|99.50|0.00|436.59|1243.77|1343.27|1680.36|1779.86|536.14| +2451198|83820|2451291|2758|94012|37354|1943|14085|72059|1425697|6958|46300|40|16|20|3|199|396|45|89.93|107.01|107.01|0.00|4815.45|4046.85|4815.45|433.39|0.00|1733.40|4815.45|5248.84|6548.85|6982.24|768.60| +2451198|83820|2451295|6694|94012|37354|1943|14085|72059|1425697|6958|46300|55|2|9|1|150|396|41|14.06|30.79|24.63|252.56|1009.83|576.46|1262.39|40.39|0.00|315.29|1009.83|1050.22|1325.12|1365.51|433.37| +2451198|83820|2451204|14173|94012|37354|1943|14085|72059|1425697|6958|46300|32|7|11|5|122|396|6|9.06|26.18|8.63|105.30|51.78|54.36|157.08|4.66|0.00|20.40|51.78|56.44|72.18|76.84|-2.58| +2451198|83820|2451225|7024|94012|37354|1943|14085|72059|1425697|6958|46300|56|19|8|1|207|396|8|28.86|53.96|21.58|259.04|172.64|230.88|431.68|8.63|0.00|77.68|172.64|181.27|250.32|258.95|-58.24| +2451198|83820|2451297|15490|94012|37354|1943|14085|72059|1425697|6958|46300|46|26|17|2|240|396|34|69.45|191.68|90.08|3454.40|3062.72|2361.30|6517.12|30.62|0.00|3193.28|3062.72|3093.34|6256.00|6286.62|701.42| +2451198|83820|2451280|1657|94012|37354|1943|14085|72059|1425697|6958|46300|28|10|7|3|189|396|71|57.07|114.14|105.00|648.94|7455.00|4051.97|8103.94|521.85|0.00|2106.57|7455.00|7976.85|9561.57|10083.42|3403.03| +2451198|83820|2451266|16802|94012|37354|1943|14085|72059|1425697|6958|46300|31|20|16|3|153|396|73|61.12|154.63|49.48|7675.95|3612.04|4461.76|11287.99|72.24|0.00|2709.03|3612.04|3684.28|6321.07|6393.31|-849.72| +2452159|72164|2452267|9107|87684|1682773|4262|2151|90540|1711267|618|38107|5|7|2|3|51|397|62|46.61|67.11|10.06|3537.10|623.72|2889.82|4160.82|43.66|0.00|706.80|623.72|667.38|1330.52|1374.18|-2266.10| +2452159|72164|2452232|16033|87684|1682773|4262|2151|90540|1711267|618|38107|59|7|8|4|233|397|99|21.48|63.79|10.84|5242.05|1073.16|2126.52|6315.21|96.58|0.00|189.09|1073.16|1169.74|1262.25|1358.83|-1053.36| +2452159|72164|2452250|15287|87684|1682773|4262|2151|90540|1711267|618|38107|39|15|3|3|233|397|73|99.08|196.17|88.27|7876.70|6443.71|7232.84|14320.41|23.19|6057.08|5727.58|386.63|409.82|6114.21|6137.40|-6846.21| +2452159|72164|2452171|4687|87684|1682773|4262|2151|90540|1711267|618|38107|5|13|4|3|215|397|2|44.98|113.34|62.33|102.02|124.66|89.96|226.68|2.49|0.00|108.80|124.66|127.15|233.46|235.95|34.70| +2452159|72164|2452164|7323|87684|1682773|4262|2151|90540|1711267|618|38107|57|17|19|4|16|397|28|86.10|201.47|106.77|2651.60|2989.56|2410.80|5641.16|0.00|687.59|56.28|2301.97|2301.97|2358.25|2358.25|-108.83| +2452159|72164|2452182|6279|87684|1682773|4262|2151|90540|1711267|618|38107|59|5|6|4|282|397|76|51.71|74.46|20.84|4075.12|1583.84|3929.96|5658.96|15.83|0.00|112.48|1583.84|1599.67|1696.32|1712.15|-2346.12| +2452159|72164|2452247|7697|87684|1682773|4262|2151|90540|1711267|618|38107|9|21|7|3|89|397|30|88.12|175.35|170.08|158.10|5102.40|2643.60|5260.50|408.19|0.00|52.50|5102.40|5510.59|5154.90|5563.09|2458.80| +2452159|72164|2452195|933|87684|1682773|4262|2151|90540|1711267|618|38107|21|29|2|1|103|397|23|85.54|133.44|17.34|2670.30|398.82|1967.42|3069.12|11.96|0.00|1227.51|398.82|410.78|1626.33|1638.29|-1568.60| +2451863|54016|2451941|14609|26072|1600712|599|20039|84943|245597|5653|34740|47|26|4|4|284|398|41|6.71|12.68|4.31|343.17|176.71|275.11|519.88|0.00|174.94|10.25|1.77|1.77|12.02|12.02|-273.34| +2451863|54016|2451890|8977|26072|1600712|599|20039|84943|245597|5653|34740|55|20|11|4|177|398|48|32.19|54.72|6.01|2338.08|288.48|1545.12|2626.56|2.88|0.00|236.16|288.48|291.36|524.64|527.52|-1256.64| +2451863|54016|2451881|1661|26072|1600712|599|20039|84943|245597|5653|34740|55|26|10|1|182|398|7|77.64|180.12|147.69|227.01|1033.83|543.48|1260.84|72.36|0.00|529.55|1033.83|1106.19|1563.38|1635.74|490.35| +2451863|54016|2451953|17399|26072|1600712|599|20039|84943|245597|5653|34740|32|17|1|5|202|398|15|30.31|58.80|37.63|317.55|564.45|454.65|882.00|0.00|0.00|396.90|564.45|564.45|961.35|961.35|109.80| +2451863|54016|2451946|16015|26072|1600712|599|20039|84943|245597|5653|34740|7|17|4|1|211|398|1|57.17|100.61|72.43|28.18|72.43|57.17|100.61|0.00|0.00|47.28|72.43|72.43|119.71|119.71|15.26| +2451863|54016|2451971|13646|26072|1600712|599|20039|84943|245597|5653|34740|47|8|6|2|278|398|46|3.11|9.01|5.22|174.34|240.12|143.06|414.46|9.60|0.00|190.44|240.12|249.72|430.56|440.16|97.06| +2451863|54016|2451866|17846|26072|1600712|599|20039|84943|245597|5653|34740|56|2|2|2|212|398|35|38.26|65.80|65.14|23.10|2279.90|1339.10|2303.00|205.19|0.00|68.95|2279.90|2485.09|2348.85|2554.04|940.80| +2451863|54016|2451961|8420|26072|1600712|599|20039|84943|245597|5653|34740|14|19|19|3|187|398|31|94.65|101.27|77.97|722.30|2417.07|2934.15|3139.37|145.02|0.00|282.41|2417.07|2562.09|2699.48|2844.50|-517.08| +2451863|54016|2451898|4442|26072|1600712|599|20039|84943|245597|5653|34740|26|14|2|4|73|398|29|79.25|118.08|99.18|548.10|2876.22|2298.25|3424.32|76.79|316.38|1095.62|2559.84|2636.63|3655.46|3732.25|261.59| +2451863|54016|2451877|11683|26072|1600712|599|20039|84943|245597|5653|34740|47|1|18|5|82|398|48|50.90|53.44|41.68|564.48|2000.64|2443.20|2565.12|58.41|540.17|359.04|1460.47|1518.88|1819.51|1877.92|-982.73| +2451863|54016|2451965|15434|26072|1600712|599|20039|84943|245597|5653|34740|8|19|12|4|76|398|71|94.62|199.64|67.87|9355.67|4818.77|6718.02|14174.44|96.37|0.00|0.00|4818.77|4915.14|4818.77|4915.14|-1899.25| +2451863|54016|2451872|11117|26072|1600712|599|20039|84943|245597|5653|34740|31|2|11|5|179|398|27|50.73|71.02|70.30|19.44|1898.10|1369.71|1917.54|15.37|1727.27|115.02|170.83|186.20|285.85|301.22|-1198.88| +2451863|54016|2451977|1004|26072|1600712|599|20039|84943|245597|5653|34740|32|2|7|5|100|398|23|97.52|240.87|115.61|2880.98|2659.03|2242.96|5540.01|65.41|478.62|664.70|2180.41|2245.82|2845.11|2910.52|-62.55| +2451863|54016|2451887|6811|26072|1600712|599|20039|84943|245597|5653|34740|41|8|19|3|287|398|75|3.83|6.70|2.14|342.00|160.50|287.25|502.50|4.81|0.00|19.50|160.50|165.31|180.00|184.81|-126.75| +2451863|54016|2451878|8899|26072|1600712|599|20039|84943|245597|5653|34740|13|8|20|5|223|398|96|21.63|51.04|8.16|4116.48|783.36|2076.48|4899.84|15.66|0.00|587.52|783.36|799.02|1370.88|1386.54|-1293.12| +2452221|73713|2452240|2243|22540|1729499|2505|27064|30273|1343063|4835|22194|7|17|17|2|220|399|40|82.08|164.98|21.44|5741.60|857.60|3283.20|6599.20|17.15|0.00|3167.60|857.60|874.75|4025.20|4042.35|-2425.60| +2452221|73713|2452252|7353|22540|1729499|2505|27064|30273|1343063|4835|22194|53|7|1|1|138|399|7|54.30|141.72|11.33|912.73|79.31|380.10|992.04|3.17|0.00|486.08|79.31|82.48|565.39|568.56|-300.79| +2452221|73713|2452257|10797|22540|1729499|2505|27064|30273|1343063|4835|22194|9|1|7|1|192|399|73|88.95|158.33|0.00|11558.09|0.00|6493.35|11558.09|0.00|0.00|1964.43|0.00|0.00|1964.43|1964.43|-6493.35| +2452221|73713|2452266|5339|22540|1729499|2505|27064|30273|1343063|4835|22194|33|21|5|2|16|399|27|15.52|22.96|2.75|545.67|74.25|419.04|619.92|6.68|0.00|241.65|74.25|80.93|315.90|322.58|-344.79| +2452221|73713|2452262|11541|22540|1729499|2505|27064|30273|1343063|4835|22194|1|23|13|4|255|399|31|13.98|17.33|13.69|112.84|424.39|433.38|537.23|25.46|0.00|42.78|424.39|449.85|467.17|492.63|-8.99| +2452221|73713|2452298|1211|22540|1729499|2505|27064|30273|1343063|4835|22194|33|3|17|2|17|399|83|37.76|66.45|27.90|3199.65|2315.70|3134.08|5515.35|162.09|0.00|2481.70|2315.70|2477.79|4797.40|4959.49|-818.38| +2452221|73713|2452242|13917|22540|1729499|2505|27064|30273|1343063|4835|22194|25|5|13|1|176|399|94|87.18|124.66|8.72|10898.36|819.68|8194.92|11718.04|16.39|0.00|3163.10|819.68|836.07|3982.78|3999.17|-7375.24| +2452221|73713|2452336|12781|22540|1729499|2505|27064|30273|1343063|4835|22194|5|25|3|5|80|399|91|29.99|32.08|15.71|1489.67|1429.61|2729.09|2919.28|0.00|0.00|232.96|1429.61|1429.61|1662.57|1662.57|-1299.48| +2452221|73713|2452297|17173|22540|1729499|2505|27064|30273|1343063|4835|22194|17|9|18|1|74|399|88|37.14|72.05|71.32|64.24|6276.16|3268.32|6340.40|266.10|2949.79|633.60|3326.37|3592.47|3959.97|4226.07|58.05| +2452527|32483|2452645|246|74106|1747076|4046|24569|74106|1747076|4046|24569|12|13|18|1|47|400|66|73.32|219.96|123.17|6388.14|8129.22|4839.12|14517.36|243.87|0.00|5371.08|8129.22|8373.09|13500.30|13744.17|3290.10| +2452527|32483|2452529|4329|74106|1747076|4046|24569|74106|1747076|4046|24569|55|3|16|3|7|400|98|43.04|102.86|62.74|3931.76|6148.52|4217.92|10080.28|172.15|3996.53|2116.80|2151.99|2324.14|4268.79|4440.94|-2065.93| +2452527|32483|2452632|14229|74106|1747076|4046|24569|74106|1747076|4046|24569|33|9|6|4|22|400|21|9.67|11.31|0.79|220.92|16.59|203.07|237.51|0.33|0.00|80.64|16.59|16.92|97.23|97.56|-186.48| +2452527|32483|2452571|11125|74106|1747076|4046|24569|74106|1747076|4046|24569|13|6|6|4|102|400|90|2.22|4.72|2.26|221.40|203.40|199.80|424.80|2.03|0.00|92.70|203.40|205.43|296.10|298.13|3.60| +2452527|32483|2452536|8514|74106|1747076|4046|24569|74106|1747076|4046|24569|6|1|19|4|242|400|54|13.01|18.08|16.81|68.58|907.74|702.54|976.32|0.00|0.00|106.92|907.74|907.74|1014.66|1014.66|205.20| +2452527|32483|2452639|10345|74106|1747076|4046|24569|74106|1747076|4046|24569|7|21|1|3|145|400|3|10.21|20.72|0.20|61.56|0.60|30.63|62.16|0.03|0.00|13.65|0.60|0.63|14.25|14.28|-30.03| +2452527|32483|2452601|16543|74106|1747076|4046|24569|74106|1747076|4046|24569|45|9|10|2|214|400|18|26.52|67.89|26.47|745.56|476.46|477.36|1222.02|0.00|0.00|598.68|476.46|476.46|1075.14|1075.14|-0.90| +2452527|32483|2452570|7851|74106|1747076|4046|24569|74106|1747076|4046|24569|54|30|20|3|196|400|9|43.38|86.76|49.45|335.79|445.05|390.42|780.84|0.00|0.00|187.38|445.05|445.05|632.43|632.43|54.63| +2452527|32483|2452542|12375|74106|1747076|4046|24569|74106|1747076|4046|24569|45|18|2|3|214|400|87|31.17|67.63|24.34|3766.23|2117.58|2711.79|5883.81|127.05|0.00|705.57|2117.58|2244.63|2823.15|2950.20|-594.21| +2452527|32483|2452532|12871|74106|1747076|4046|24569|74106|1747076|4046|24569|33|18|1|1|218|400|83|98.57|166.58|73.29|7743.07|6083.07|8181.31|13826.14|182.49|0.00|1796.95|6083.07|6265.56|7880.02|8062.51|-2098.24| +2452527|32483|2452592|3912|74106|1747076|4046|24569|74106|1747076|4046|24569|25|15|20|5|180|400|66|1.96|3.72|2.34|91.08|154.44|129.36|245.52|0.00|0.00|97.68|154.44|154.44|252.12|252.12|25.08| +2452188|60154|2452267|10979|75585|1579585|385|11985|76051|934601|32|10832|7|21|5|2|16|401|94|53.36|57.62|48.97|813.10|4603.18|5015.84|5416.28|276.19|0.00|649.54|4603.18|4879.37|5252.72|5528.91|-412.66| +2452188|60154|2452267|10833|75585|1579585|385|11985|76051|934601|32|10832|35|25|15|2|192|401|81|76.21|140.22|96.75|3521.07|7836.75|6173.01|11357.82|548.57|0.00|5110.29|7836.75|8385.32|12947.04|13495.61|1663.74| +2452188|60154|2452264|10067|75585|1579585|385|11985|76051|934601|32|10832|57|25|17|1|235|401|73|14.80|16.57|3.97|919.80|289.81|1080.40|1209.61|26.08|0.00|253.31|289.81|315.89|543.12|569.20|-790.59| +2452188|60154|2452308|2539|75585|1579585|385|11985|76051|934601|32|10832|23|5|1|2|260|401|85|39.83|77.66|74.55|264.35|6336.75|3385.55|6601.10|506.94|0.00|0.00|6336.75|6843.69|6336.75|6843.69|2951.20| +2452188|60154|2452224|17101|75585|1579585|385|11985|76051|934601|32|10832|15|13|13|2|198|401|84|31.43|90.51|47.06|3649.80|3953.04|2640.12|7602.84|276.71|0.00|1368.36|3953.04|4229.75|5321.40|5598.11|1312.92| +2452188|60154|2452211|15265|75585|1579585|385|11985|76051|934601|32|10832|3|21|3|2|241|401|95|55.25|150.28|49.59|9565.55|4711.05|5248.75|14276.60|141.33|0.00|2283.80|4711.05|4852.38|6994.85|7136.18|-537.70| +2452188|60154|2452277|10187|75585|1579585|385|11985|76051|934601|32|10832|5|3|11|5|119|401|32|9.34|17.18|4.46|407.04|142.72|298.88|549.76|1.42|0.00|230.72|142.72|144.14|373.44|374.86|-156.16| +2452188|60154|2452225|14977|75585|1579585|385|11985|76051|934601|32|10832|21|25|15|2|49|401|87|69.41|126.32|97.26|2528.22|8461.62|6038.67|10989.84|253.84|0.00|4065.51|8461.62|8715.46|12527.13|12780.97|2422.95| +2452188|60154|2452305|16355|75585|1579585|385|11985|76051|934601|32|10832|55|15|18|3|255|401|1|69.12|207.36|114.04|93.32|114.04|69.12|207.36|6.84|0.00|66.35|114.04|120.88|180.39|187.23|44.92| +2452543|76660|2452638|5739|27896|1873356|4416|5829|57390|764473|5809|21269|49|12|7|3|69|402|56|4.54|7.17|5.80|76.72|324.80|254.24|401.52|12.99|0.00|104.16|324.80|337.79|428.96|441.95|70.56| +2452543|76660|2452663|15457|27896|1873356|4416|5829|57390|764473|5809|21269|6|13|1|5|19|402|46|99.73|101.72|97.65|187.22|4491.90|4587.58|4679.12|269.51|0.00|1263.16|4491.90|4761.41|5755.06|6024.57|-95.68| +2452543|76660|2452571|10731|27896|1873356|4416|5829|57390|764473|5809|21269|55|19|12|4|46|402|91|34.82|41.43|23.61|1621.62|2148.51|3168.62|3770.13|85.94|0.00|716.17|2148.51|2234.45|2864.68|2950.62|-1020.11| +2452543|76660|2452626|16845|27896|1873356|4416|5829|57390|764473|5809|21269|18|9|18|1|224|402|19|91.74|163.29|1.63|3071.54|30.97|1743.06|3102.51|0.30|0.00|341.24|30.97|31.27|372.21|372.51|-1712.09| +2452543|76660|2452654|16584|27896|1873356|4416|5829|57390|764473|5809|21269|57|18|6|1|136|402|39|6.71|18.92|18.35|22.23|715.65|261.69|737.88|19.17|236.16|110.37|479.49|498.66|589.86|609.03|217.80| +2452543|76660|2452547|2371|27896|1873356|4416|5829|57390|764473|5809|21269|18|1|6|3|210|402|28|18.56|53.45|36.34|479.08|1017.52|519.68|1496.60|61.05|0.00|434.00|1017.52|1078.57|1451.52|1512.57|497.84| +2452543|76660|2452636|828|27896|1873356|4416|5829|57390|764473|5809|21269|9|25|18|3|51|402|95|25.18|33.23|4.31|2747.40|409.45|2392.10|3156.85|4.66|176.06|1420.25|233.39|238.05|1653.64|1658.30|-2158.71| +2452543|76660|2452646|6379|27896|1873356|4416|5829|57390|764473|5809|21269|6|21|19|2|155|402|5|21.93|37.06|30.38|33.40|151.90|109.65|185.30|8.50|45.57|88.90|106.33|114.83|195.23|203.73|-3.32| +2452543|76660|2452656|10423|27896|1873356|4416|5829|57390|764473|5809|21269|45|9|17|4|101|402|50|85.24|192.64|94.39|4912.50|4719.50|4262.00|9632.00|8.49|4625.11|1348.00|94.39|102.88|1442.39|1450.88|-4167.61| +2452543|76660|2452628|17011|27896|1873356|4416|5829|57390|764473|5809|21269|39|7|15|4|270|402|15|21.03|45.42|4.08|620.10|61.20|315.45|681.30|0.61|0.00|252.00|61.20|61.81|313.20|313.81|-254.25| +2452543|76660|2452612|529|27896|1873356|4416|5829|57390|764473|5809|21269|48|13|7|5|210|402|24|41.06|98.95|86.08|308.88|2065.92|985.44|2374.80|20.65|0.00|617.28|2065.92|2086.57|2683.20|2703.85|1080.48| +2452543|76660|2452631|1129|27896|1873356|4416|5829|57390|764473|5809|21269|6|18|2|3|191|402|31|42.02|118.91|8.32|3428.29|257.92|1302.62|3686.21|12.89|0.00|1253.02|257.92|270.81|1510.94|1523.83|-1044.70| +2452543|76660|2452616|7401|27896|1873356|4416|5829|57390|764473|5809|21269|37|30|4|5|176|402|100|98.67|234.83|18.78|21605.00|1878.00|9867.00|23483.00|169.02|0.00|8453.00|1878.00|2047.02|10331.00|10500.02|-7989.00| +2452543|76660|2452617|16843|27896|1873356|4416|5829|57390|764473|5809|21269|9|18|6|3|247|402|83|51.86|124.46|97.07|2273.37|8056.81|4304.38|10330.18|644.54|0.00|2479.21|8056.81|8701.35|10536.02|11180.56|3752.43| +2452543|76660|2452583|16299|27896|1873356|4416|5829|57390|764473|5809|21269|60|21|12|3|254|402|31|58.73|171.49|5.14|5156.85|159.34|1820.63|5316.19|3.18|0.00|372.00|159.34|162.52|531.34|534.52|-1661.29| +2452543|76660|2452663|14311|27896|1873356|4416|5829|57390|764473|5809|21269|25|12|9|2|135|402|40|67.57|125.00|41.25|3350.00|1650.00|2702.80|5000.00|99.00|0.00|1100.00|1650.00|1749.00|2750.00|2849.00|-1052.80| +2452210|53402|2452294|9155|17010|384887|2364|3841|44685|802635|719|36596|27|1|8|5|202|403|29|29.05|62.16|6.83|1604.57|198.07|842.45|1802.64|11.88|0.00|901.32|198.07|209.95|1099.39|1111.27|-644.38| +2452210|53402|2452236|6291|17010|384887|2364|3841|44685|802635|719|36596|3|1|20|4|111|403|8|65.91|166.75|23.34|1147.28|186.72|527.28|1334.00|14.93|0.00|106.72|186.72|201.65|293.44|308.37|-340.56| +2452210|53402|2452294|15091|17010|384887|2364|3841|44685|802635|719|36596|39|27|15|1|287|403|27|56.36|80.59|13.70|1806.03|369.90|1521.72|2175.93|7.39|0.00|1044.36|369.90|377.29|1414.26|1421.65|-1151.82| +2452210|53402|2452213|8243|17010|384887|2364|3841|44685|802635|719|36596|29|5|8|1|161|403|40|35.44|58.47|44.43|561.60|1777.20|1417.60|2338.80|17.77|0.00|46.40|1777.20|1794.97|1823.60|1841.37|359.60| +2452210|53402|2452248|1887|17010|384887|2364|3841|44685|802635|719|36596|17|25|14|3|152|403|26|51.24|60.46|55.62|125.84|1446.12|1332.24|1571.96|130.15|0.00|534.30|1446.12|1576.27|1980.42|2110.57|113.88| +2452210|53402|2452278|13455|17010|384887|2364|3841|44685|802635|719|36596|9|9|18|1|164|403|73|17.73|36.34|15.98|1486.28|1166.54|1294.29|2652.82|81.65|0.00|716.13|1166.54|1248.19|1882.67|1964.32|-127.75| +2452210|53402|2452225|2149|17010|384887|2364|3841|44685|802635|719|36596|27|25|2|1|160|403|87|27.15|52.12|50.55|136.59|4397.85|2362.05|4534.44|98.51|1935.05|543.75|2462.80|2561.31|3006.55|3105.06|100.75| +2452210|53402|2452302|5819|17010|384887|2364|3841|44685|802635|719|36596|19|21|12|5|141|403|81|13.23|35.58|14.23|1729.35|1152.63|1071.63|2881.98|23.05|0.00|892.62|1152.63|1175.68|2045.25|2068.30|81.00| +2452210|53402|2452304|14635|17010|384887|2364|3841|44685|802635|719|36596|31|1|3|3|270|403|65|85.15|149.01|143.04|388.05|9297.60|5534.75|9685.65|371.90|0.00|2518.10|9297.60|9669.50|11815.70|12187.60|3762.85| +2452210|53402|2452327|13109|17010|384887|2364|3841|44685|802635|719|36596|51|1|9|4|279|403|58|73.43|218.82|98.46|6980.88|5710.68|4258.94|12691.56|399.74|0.00|2538.08|5710.68|6110.42|8248.76|8648.50|1451.74| +2452210|53402|2452308|9415|17010|384887|2364|3841|44685|802635|719|36596|35|19|2|4|256|403|62|63.02|122.25|2.44|7428.22|151.28|3907.24|7579.50|10.58|0.00|226.92|151.28|161.86|378.20|388.78|-3755.96| +2452609|47369|2452676|14982|66057|822848|2839|14030|36356|1547824|4403|27994|27|3|10|3|116|404|55|58.45|143.20|5.72|7561.40|314.60|3214.75|7876.00|9.43|0.00|2835.25|314.60|324.03|3149.85|3159.28|-2900.15| +2452609|47369|2452707|9168|66057|822848|2839|14030|36356|1547824|4403|27994|30|25|4|5|166|404|21|67.74|146.99|11.75|2840.04|246.75|1422.54|3086.79|2.66|157.92|524.58|88.83|91.49|613.41|616.07|-1333.71| +2452609|47369|2452637|13467|66057|822848|2839|14030|36356|1547824|4403|27994|48|1|18|5|202|404|35|86.32|109.62|8.76|3530.10|306.60|3021.20|3836.70|12.26|0.00|229.95|306.60|318.86|536.55|548.81|-2714.60| +2452609|47369|2452724|10219|66057|822848|2839|14030|36356|1547824|4403|27994|45|3|18|2|223|404|22|97.57|258.56|137.03|2673.66|3014.66|2146.54|5688.32|63.30|2110.26|56.76|904.40|967.70|961.16|1024.46|-1242.14| +2452609|47369|2452660|12147|66057|822848|2839|14030|36356|1547824|4403|27994|12|7|2|2|5|404|76|45.48|63.67|1.91|4693.76|145.16|3456.48|4838.92|0.00|0.00|1499.48|145.16|145.16|1644.64|1644.64|-3311.32| +2452609|47369|2452614|6565|66057|822848|2839|14030|36356|1547824|4403|27994|25|15|13|5|270|404|30|75.53|192.60|92.44|3004.80|2773.20|2265.90|5778.00|166.39|0.00|1848.90|2773.20|2939.59|4622.10|4788.49|507.30| +2452609|47369|2452707|7326|66057|822848|2839|14030|36356|1547824|4403|27994|49|21|17|3|239|404|54|10.07|17.72|12.75|268.38|688.50|543.78|956.88|41.31|0.00|430.38|688.50|729.81|1118.88|1160.19|144.72| +2452609|47369|2452617|17797|66057|822848|2839|14030|36356|1547824|4403|27994|51|3|11|2|216|404|74|82.95|111.98|99.66|911.68|7374.84|6138.30|8286.52|254.43|2286.20|1076.70|5088.64|5343.07|6165.34|6419.77|-1049.66| +2452609|47369|2452632|7800|66057|822848|2839|14030|36356|1547824|4403|27994|57|1|13|5|37|404|81|69.62|76.58|57.43|1551.15|4651.83|5639.22|6202.98|186.07|0.00|434.16|4651.83|4837.90|5085.99|5272.06|-987.39| +2452609|47369|2452679|12465|66057|822848|2839|14030|36356|1547824|4403|27994|51|13|5|3|241|404|44|57.36|153.15|134.77|808.72|5929.88|2523.84|6738.60|118.59|0.00|3099.36|5929.88|6048.47|9029.24|9147.83|3406.04| +2452609|47369|2452637|5917|66057|822848|2839|14030|36356|1547824|4403|27994|13|6|16|4|275|404|80|68.54|106.23|1.06|8413.60|84.80|5483.20|8498.40|5.08|0.00|169.60|84.80|89.88|254.40|259.48|-5398.40| +2452540|65371|2452582|13267|37230|1678674|5365|17209|44506|578056|35|46279|37|25|17|1|295|405|97|30.97|56.36|30.99|2460.89|3006.03|3004.09|5466.92|210.42|0.00|983.58|3006.03|3216.45|3989.61|4200.03|1.94| +2452540|65371|2452623|15085|37230|1678674|5365|17209|44506|578056|35|46279|54|3|17|2|174|405|27|20.87|38.19|16.42|587.79|443.34|563.49|1031.13|35.46|0.00|216.27|443.34|478.80|659.61|695.07|-120.15| +2452540|65371|2452657|2358|37230|1678674|5365|17209|44506|578056|35|46279|51|25|20|2|124|405|99|10.33|27.99|14.83|1302.84|1468.17|1022.67|2771.01|73.40|0.00|1190.97|1468.17|1541.57|2659.14|2732.54|445.50| +2452540|65371|2452566|7488|37230|1678674|5365|17209|44506|578056|35|46279|21|19|12|1|90|405|38|50.79|68.05|62.60|207.10|2378.80|1930.02|2585.90|3.33|2045.76|103.36|333.04|336.37|436.40|439.73|-1596.98| +2452540|65371|2452576|14875|37230|1678674|5365|17209|44506|578056|35|46279|54|24|4|4|34|405|37|46.14|125.50|43.92|3018.46|1625.04|1707.18|4643.50|48.75|0.00|1346.43|1625.04|1673.79|2971.47|3020.22|-82.14| +2452540|65371|2452628|17121|37230|1678674|5365|17209|44506|578056|35|46279|33|27|8|2|109|405|44|90.75|120.69|13.27|4726.48|583.88|3993.00|5310.36|29.19|0.00|796.40|583.88|613.07|1380.28|1409.47|-3409.12| +2452540|65371|2452558|17299|37230|1678674|5365|17209|44506|578056|35|46279|57|21|6|1|3|405|67|94.06|115.69|61.31|3643.46|4107.77|6302.02|7751.23|287.54|0.00|3720.51|4107.77|4395.31|7828.28|8115.82|-2194.25| +2452540|65371|2452550|11961|37230|1678674|5365|17209|44506|578056|35|46279|39|18|18|2|156|405|17|78.05|87.41|75.17|208.08|1277.89|1326.85|1485.97|5.11|1175.65|29.58|102.24|107.35|131.82|136.93|-1224.61| +2452540|65371|2452656|1633|37230|1678674|5365|17209|44506|578056|35|46279|45|13|11|5|199|405|66|88.26|192.40|59.64|8762.16|3936.24|5825.16|12698.40|236.17|0.00|1777.38|3936.24|4172.41|5713.62|5949.79|-1888.92| +2452540|65371|2452599|12012|37230|1678674|5365|17209|44506|578056|35|46279|57|30|14|3|12|405|43|67.76|107.06|5.35|4373.53|230.05|2913.68|4603.58|4.60|0.00|828.61|230.05|234.65|1058.66|1063.26|-2683.63| +2452540|65371|2452640|5301|37230|1678674|5365|17209|44506|578056|35|46279|60|21|5|1|127|405|25|84.24|136.46|38.20|2456.50|955.00|2106.00|3411.50|57.58|315.15|1637.50|639.85|697.43|2277.35|2334.93|-1466.15| +2451523|26535|2451634|14324|7838|1693252|4619|16833|41632|793501|5028|6395|10|2|1|2|6|406|98|48.49|141.10|97.35|4287.50|9540.30|4752.02|13827.80|190.80|0.00|1659.14|9540.30|9731.10|11199.44|11390.24|4788.28| +2451523|26535|2451588|643|7838|1693252|4619|16833|41632|793501|5028|6395|20|1|11|1|204|406|51|68.89|116.42|23.28|4750.14|1187.28|3513.39|5937.42|83.10|0.00|2434.23|1187.28|1270.38|3621.51|3704.61|-2326.11| +2451523|26535|2451530|15193|7838|1693252|4619|16833|41632|793501|5028|6395|49|4|18|1|214|406|7|1.41|3.39|2.06|9.31|14.42|9.87|23.73|0.00|5.76|9.94|8.66|8.66|18.60|18.60|-1.21| +2451523|26535|2451606|14056|7838|1693252|4619|16833|41632|793501|5028|6395|26|2|7|5|243|406|46|32.13|90.92|24.54|3053.48|1128.84|1477.98|4182.32|19.64|474.11|1045.58|654.73|674.37|1700.31|1719.95|-823.25| +2451523|26535|2451582|2858|7838|1693252|4619|16833|41632|793501|5028|6395|19|25|19|5|188|406|70|42.38|70.35|68.94|98.70|4825.80|2966.60|4924.50|48.25|0.00|49.00|4825.80|4874.05|4874.80|4923.05|1859.20| +2451523|26535|2451606|5566|7838|1693252|4619|16833|41632|793501|5028|6395|28|2|12|4|39|406|9|45.84|115.05|31.06|755.91|279.54|412.56|1035.45|5.59|0.00|41.40|279.54|285.13|320.94|326.53|-133.02| +2451523|26535|2451541|7208|7838|1693252|4619|16833|41632|793501|5028|6395|52|22|10|5|80|406|37|66.60|122.54|88.22|1269.84|3264.14|2464.20|4533.98|261.13|0.00|997.15|3264.14|3525.27|4261.29|4522.42|799.94| +2451523|26535|2451524|13147|7838|1693252|4619|16833|41632|793501|5028|6395|10|4|7|1|8|406|72|81.45|158.01|102.70|3982.32|7394.40|5864.40|11376.72|0.00|0.00|1592.64|7394.40|7394.40|8987.04|8987.04|1530.00| +2451523|26535|2451542|8908|7838|1693252|4619|16833|41632|793501|5028|6395|25|8|17|4|191|406|30|95.12|236.84|87.63|4476.30|2628.90|2853.60|7105.20|74.92|1130.42|3055.20|1498.48|1573.40|4553.68|4628.60|-1355.12| +2451523|26535|2451545|5281|7838|1693252|4619|16833|41632|793501|5028|6395|14|20|4|2|200|406|32|78.59|96.66|71.52|804.48|2288.64|2514.88|3093.12|68.65|0.00|1298.88|2288.64|2357.29|3587.52|3656.17|-226.24| +2451523|26535|2451529|17917|7838|1693252|4619|16833|41632|793501|5028|6395|1|28|16|5|161|406|42|83.33|134.99|9.44|5273.10|396.48|3499.86|5669.58|31.71|0.00|2607.78|396.48|428.19|3004.26|3035.97|-3103.38| +2452613|791|2452655|1734|37389|322780|7018|16856|46472|650174|5923|17361|25|3|11|5|96|407|49|77.06|229.63|142.37|4275.74|6976.13|3775.94|11251.87|209.28|0.00|3150.21|6976.13|7185.41|10126.34|10335.62|3200.19| +2452613|791|2452683|17472|37389|322780|7018|16856|46472|650174|5923|17361|7|3|7|5|273|407|30|16.89|26.51|15.37|334.20|461.10|506.70|795.30|27.66|0.00|222.60|461.10|488.76|683.70|711.36|-45.60| +2452613|791|2452698|660|37389|322780|7018|16856|46472|650174|5923|17361|13|27|1|1|298|407|11|62.34|135.90|116.87|209.33|1285.57|685.74|1494.90|0.00|0.00|343.75|1285.57|1285.57|1629.32|1629.32|599.83| +2452613|791|2452676|13201|37389|322780|7018|16856|46472|650174|5923|17361|36|27|13|4|25|407|54|70.54|91.70|33.01|3169.26|1782.54|3809.16|4951.80|0.00|1123.00|1138.86|659.54|659.54|1798.40|1798.40|-3149.62| +2452613|791|2452726|14191|37389|322780|7018|16856|46472|650174|5923|17361|31|25|10|5|117|407|76|90.89|170.87|47.84|9350.28|3635.84|6907.64|12986.12|254.50|0.00|1817.92|3635.84|3890.34|5453.76|5708.26|-3271.80| +2452613|791|2452622|11839|37389|322780|7018|16856|46472|650174|5923|17361|37|15|17|1|245|407|72|28.68|67.68|12.85|3947.76|925.20|2064.96|4872.96|21.46|388.58|243.36|536.62|558.08|779.98|801.44|-1528.34| +2452613|791|2452709|13182|37389|322780|7018|16856|46472|650174|5923|17361|6|3|11|5|6|407|74|92.93|263.92|261.28|195.36|19334.72|6876.82|19530.08|1546.77|0.00|1757.50|19334.72|20881.49|21092.22|22638.99|12457.90| +2452613|791|2452649|1962|37389|322780|7018|16856|46472|650174|5923|17361|43|18|3|2|231|407|69|44.64|82.58|55.32|1880.94|3817.08|3080.16|5698.02|76.34|0.00|2620.62|3817.08|3893.42|6437.70|6514.04|736.92| +2452613|791|2452658|2865|37389|322780|7018|16856|46472|650174|5923|17361|57|27|19|1|246|407|25|36.40|58.60|5.86|1318.50|146.50|910.00|1465.00|0.00|146.50|366.25|0.00|0.00|366.25|366.25|-910.00| +2452613|791|2452719|15391|37389|322780|7018|16856|46472|650174|5923|17361|12|15|17|4|231|407|34|49.56|108.53|39.07|2361.64|1328.38|1685.04|3690.02|106.27|0.00|774.86|1328.38|1434.65|2103.24|2209.51|-356.66| +2452613|791|2452718|5121|37389|322780|7018|16856|46472|650174|5923|17361|15|21|14|2|80|407|82|94.86|239.04|90.83|12153.22|7448.06|7778.52|19601.28|521.36|0.00|5096.30|7448.06|7969.42|12544.36|13065.72|-330.46| +2452613|791|2452639|1980|37389|322780|7018|16856|46472|650174|5923|17361|57|18|19|4|219|407|15|96.33|106.92|6.41|1507.65|96.15|1444.95|1603.80|1.92|0.00|336.75|96.15|98.07|432.90|434.82|-1348.80| +2452613|791|2452634|17637|37389|322780|7018|16856|46472|650174|5923|17361|33|6|19|5|164|407|65|57.62|145.77|104.95|2653.30|6821.75|3745.30|9475.05|337.67|3069.78|4358.25|3751.97|4089.64|8110.22|8447.89|6.67| +2452547|38285|2452592|11659|52737|1534728|1519|37110|40249|739199|1483|16274|57|3|15|3|41|408|1|35.54|57.93|40.55|17.38|40.55|35.54|57.93|3.24|0.00|5.21|40.55|43.79|45.76|49.00|5.01| +2452547|38285|2452648|5928|52737|1534728|1519|37110|40249|739199|1483|16274|43|30|18|1|211|408|87|63.52|184.84|136.78|4181.22|11899.86|5526.24|16081.08|356.99|0.00|4824.15|11899.86|12256.85|16724.01|17081.00|6373.62| +2452547|38285|2452555|14718|52737|1534728|1519|37110|40249|739199|1483|16274|33|15|12|3|124|408|83|62.93|97.54|79.98|1457.48|6638.34|5223.19|8095.82|265.53|0.00|0.00|6638.34|6903.87|6638.34|6903.87|1415.15| +2452547|38285|2452656|15576|52737|1534728|1519|37110|40249|739199|1483|16274|30|30|12|4|6|408|59|74.74|188.34|180.80|444.86|10667.20|4409.66|11112.06|0.00|0.00|110.92|10667.20|10667.20|10778.12|10778.12|6257.54| +2452547|38285|2452592|9057|52737|1534728|1519|37110|40249|739199|1483|16274|36|24|3|4|285|408|83|43.18|104.49|60.60|3642.87|5029.80|3583.94|8672.67|50.29|0.00|2341.43|5029.80|5080.09|7371.23|7421.52|1445.86| +2452547|38285|2452565|6366|52737|1534728|1519|37110|40249|739199|1483|16274|39|27|11|5|102|408|64|5.84|8.40|4.87|225.92|311.68|373.76|537.60|28.05|0.00|123.52|311.68|339.73|435.20|463.25|-62.08| +2452547|38285|2452643|3237|52737|1534728|1519|37110|40249|739199|1483|16274|33|27|4|4|81|408|73|36.49|51.08|33.71|1268.01|2460.83|2663.77|3728.84|24.60|0.00|446.76|2460.83|2485.43|2907.59|2932.19|-202.94| +2452547|38285|2452661|10753|52737|1534728|1519|37110|40249|739199|1483|16274|48|30|6|2|138|408|24|53.07|158.67|63.46|2285.04|1523.04|1273.68|3808.08|0.00|0.00|1408.80|1523.04|1523.04|2931.84|2931.84|249.36| +2452547|38285|2452594|4459|52737|1534728|1519|37110|40249|739199|1483|16274|55|1|7|3|146|408|95|77.25|196.21|176.58|1864.85|16775.10|7338.75|18639.95|167.75|0.00|8759.95|16775.10|16942.85|25535.05|25702.80|9436.35| +2452547|38285|2452571|13965|52737|1534728|1519|37110|40249|739199|1483|16274|51|19|1|5|6|408|61|43.35|62.42|19.35|2627.27|1180.35|2644.35|3807.62|35.41|0.00|1332.24|1180.35|1215.76|2512.59|2548.00|-1464.00| +2452547|38285|2452638|15060|52737|1534728|1519|37110|40249|739199|1483|16274|31|9|1|5|106|408|7|99.46|231.74|106.60|875.98|746.20|696.22|1622.18|52.23|0.00|0.00|746.20|798.43|746.20|798.43|49.98| +2452547|38285|2452628|10513|52737|1534728|1519|37110|40249|739199|1483|16274|51|21|3|3|100|408|90|36.63|64.83|15.55|4435.20|1399.50|3296.70|5834.70|111.96|0.00|2916.90|1399.50|1511.46|4316.40|4428.36|-1897.20| +2452547|38285|2452556|16585|52737|1534728|1519|37110|40249|739199|1483|16274|25|6|3|5|280|408|80|58.81|92.33|78.48|1108.00|6278.40|4704.80|7386.40|502.27|0.00|2363.20|6278.40|6780.67|8641.60|9143.87|1573.60| +2452547|38285|2452644|10326|52737|1534728|1519|37110|40249|739199|1483|16274|18|12|6|3|103|408|66|28.45|58.32|40.82|1155.00|2694.12|1877.70|3849.12|0.00|0.00|114.84|2694.12|2694.12|2808.96|2808.96|816.42| +2452547|38285|2452639|4536|52737|1534728|1519|37110|40249|739199|1483|16274|39|3|7|2|114|408|73|45.77|135.47|115.14|1484.09|8405.22|3341.21|9889.31|33.62|6724.17|692.04|1681.05|1714.67|2373.09|2406.71|-1660.16| +2452636|73244|2452696|13081|1435|884280|1525|44770|97526|687389|3305|2550|55|15|11|1|114|409|81|8.52|25.47|14.51|887.76|1175.31|690.12|2063.07|105.77|0.00|144.18|1175.31|1281.08|1319.49|1425.26|485.19| +2452636|73244|2452702|14827|1435|884280|1525|44770|97526|687389|3305|2550|31|27|2|5|98|409|73|54.89|64.22|12.20|3797.46|890.60|4006.97|4688.06|24.22|587.79|46.72|302.81|327.03|349.53|373.75|-3704.16| +2452636|73244|2452719|6271|1435|884280|1525|44770|97526|687389|3305|2550|9|12|20|1|212|409|90|45.97|65.27|33.28|2879.10|2995.20|4137.30|5874.30|59.90|0.00|1937.70|2995.20|3055.10|4932.90|4992.80|-1142.10| +2452636|73244|2452697|8538|1435|884280|1525|44770|97526|687389|3305|2550|7|3|10|3|236|409|18|94.24|119.68|69.41|904.86|1249.38|1696.32|2154.24|1.24|1186.91|689.22|62.47|63.71|751.69|752.93|-1633.85| +2452636|73244|2452690|17910|1435|884280|1525|44770|97526|687389|3305|2550|33|9|16|4|22|409|55|8.28|12.25|7.84|242.55|431.20|455.40|673.75|0.00|159.54|168.30|271.66|271.66|439.96|439.96|-183.74| +2452636|73244|2452725|12589|1435|884280|1525|44770|97526|687389|3305|2550|39|3|14|1|45|409|53|1.71|2.58|0.38|116.60|20.14|90.63|136.74|1.40|0.00|38.16|20.14|21.54|58.30|59.70|-70.49| +2452636|73244|2452714|9558|1435|884280|1525|44770|97526|687389|3305|2550|33|19|6|4|222|409|42|56.87|80.18|78.57|67.62|3299.94|2388.54|3367.56|0.00|0.00|336.42|3299.94|3299.94|3636.36|3636.36|911.40| +2452636|73244|2452660|3516|1435|884280|1525|44770|97526|687389|3305|2550|60|21|2|3|224|409|25|54.60|131.04|58.96|1802.00|1474.00|1365.00|3276.00|117.92|0.00|1015.50|1474.00|1591.92|2489.50|2607.42|109.00| +2452636|73244|2452647|3591|1435|884280|1525|44770|97526|687389|3305|2550|51|13|1|2|116|409|52|64.78|123.08|123.08|0.00|6400.16|3368.56|6400.16|0.00|0.00|3135.60|6400.16|6400.16|9535.76|9535.76|3031.60| +2452636|73244|2452704|10644|1435|884280|1525|44770|97526|687389|3305|2550|49|15|11|4|257|409|19|66.32|122.69|6.13|2214.64|116.47|1260.08|2331.11|4.65|0.00|605.91|116.47|121.12|722.38|727.03|-1143.61| +2450818|17496|2450901|15334|61759|1892819|326|13833|61759|1892819|326|13833|19|2|14|3|135|410|29|21.82|55.64|25.03|887.69|725.87|632.78|1613.56|50.81|0.00|806.78|725.87|776.68|1532.65|1583.46|93.09| +2450818|17496|2450931|8608|61759|1892819|326|13833|61759|1892819|326|13833|49|7|6|5|77|410|16|37.16|109.99|74.79|563.20|1196.64|594.56|1759.84|71.79|0.00|87.84|1196.64|1268.43|1284.48|1356.27|602.08| +2450818|17496|2450846|1238|61759|1892819|326|13833|61759|1892819|326|13833|43|2|2|4|189|410|7|62.73|138.00|52.44|598.92|367.08|439.11|966.00|25.69|0.00|367.08|367.08|392.77|734.16|759.85|-72.03| +2450818|17496|2450881|8176|61759|1892819|326|13833|61759|1892819|326|13833|58|4|9|5|111|410|72|83.07|235.91|158.05|5605.92|11379.60|5981.04|16985.52|682.77|0.00|0.00|11379.60|12062.37|11379.60|12062.37|5398.56| +2450818|17496|2450923|2224|61759|1892819|326|13833|61759|1892819|326|13833|10|25|13|3|170|410|29|22.82|31.71|23.14|248.53|671.06|661.78|919.59|40.26|0.00|229.68|671.06|711.32|900.74|941.00|9.28| +2450818|17496|2450824|11470|61759|1892819|326|13833|61759|1892819|326|13833|52|10|15|1|250|410|29|65.83|156.01|117.00|1131.29|3393.00|1909.07|4524.29|305.37|0.00|1221.48|3393.00|3698.37|4614.48|4919.85|1483.93| +2450818|17496|2450896|11281|61759|1892819|326|13833|61759|1892819|326|13833|38|8|3|3|43|410|13|40.74|41.96|27.69|185.51|359.97|529.62|545.48|32.39|0.00|27.17|359.97|392.36|387.14|419.53|-169.65| +2450818|17496|2450927|10984|61759|1892819|326|13833|61759|1892819|326|13833|43|26|6|3|80|410|95|96.94|157.04|117.78|3729.70|11189.10|9209.30|14918.80|288.67|1566.47|447.45|9622.63|9911.30|10070.08|10358.75|413.33| +2450818|17496|2450862|17416|61759|1892819|326|13833|61759|1892819|326|13833|50|16|12|3|131|410|26|93.15|136.93|75.31|1602.12|1958.06|2421.90|3560.18|39.16|0.00|106.60|1958.06|1997.22|2064.66|2103.82|-463.84| +2450818|17496|2450878|9445|61759|1892819|326|13833|61759|1892819|326|13833|34|22|19|1|130|410|30|16.12|37.72|28.66|271.80|859.80|483.60|1131.60|68.78|0.00|294.00|859.80|928.58|1153.80|1222.58|376.20| +2450818|17496|2450837|7879|61759|1892819|326|13833|61759|1892819|326|13833|34|25|15|2|12|410|34|13.25|38.02|1.52|1241.00|51.68|450.50|1292.68|0.00|0.00|439.28|51.68|51.68|490.96|490.96|-398.82| +2452550|73966|2452559|7431|92789|1370108|2180|26699|73682|798751|4609|49481|57|13|5|3|31|411|9|26.35|28.45|1.70|240.75|15.30|237.15|256.05|1.37|0.00|15.30|15.30|16.67|30.60|31.97|-221.85| +2452550|73966|2452565|17778|92789|1370108|2180|26699|73682|798751|4609|49481|49|3|6|3|270|411|81|96.27|150.18|112.63|3041.55|9123.03|7797.87|12164.58|547.38|0.00|6082.29|9123.03|9670.41|15205.32|15752.70|1325.16| +2452550|73966|2452614|9849|92789|1370108|2180|26699|73682|798751|4609|49481|6|27|8|4|190|411|21|33.35|49.69|3.47|970.62|72.87|700.35|1043.49|1.45|0.00|396.48|72.87|74.32|469.35|470.80|-627.48| +2452550|73966|2452557|2863|92789|1370108|2180|26699|73682|798751|4609|49481|1|15|5|2|66|411|47|13.05|16.31|5.21|521.70|244.87|613.35|766.57|2.44|0.00|229.83|244.87|247.31|474.70|477.14|-368.48| +2452550|73966|2452600|5973|92789|1370108|2180|26699|73682|798751|4609|49481|1|12|18|3|238|411|31|87.86|103.67|72.56|964.41|2249.36|2723.66|3213.77|44.98|0.00|31.93|2249.36|2294.34|2281.29|2326.27|-474.30| +2452550|73966|2452627|1806|92789|1370108|2180|26699|73682|798751|4609|49481|48|24|20|3|126|411|77|17.75|34.43|21.00|1034.11|1617.00|1366.75|2651.11|133.88|129.36|423.50|1487.64|1621.52|1911.14|2045.02|120.89| +2452550|73966|2452582|559|92789|1370108|2180|26699|73682|798751|4609|49481|25|24|1|2|76|411|27|98.28|275.18|244.91|817.29|6612.57|2653.56|7429.86|164.65|1124.13|371.25|5488.44|5653.09|5859.69|6024.34|2834.88| +2452550|73966|2452558|12735|92789|1370108|2180|26699|73682|798751|4609|49481|43|19|5|3|41|411|68|73.09|157.14|102.14|3740.00|6945.52|4970.12|10685.52|0.00|0.00|5128.56|6945.52|6945.52|12074.08|12074.08|1975.40| +2452550|73966|2452600|13302|92789|1370108|2180|26699|73682|798751|4609|49481|42|1|11|3|158|411|59|82.36|101.30|4.05|5737.75|238.95|4859.24|5976.70|7.16|0.00|2509.86|238.95|246.11|2748.81|2755.97|-4620.29| +2452550|73966|2452571|7560|92789|1370108|2180|26699|73682|798751|4609|49481|1|3|13|4|35|411|84|40.68|112.68|83.38|2461.20|7003.92|3417.12|9465.12|490.27|0.00|2744.28|7003.92|7494.19|9748.20|10238.47|3586.80| +2452550|73966|2452652|2047|92789|1370108|2180|26699|73682|798751|4609|49481|25|6|6|4|139|411|56|57.59|120.36|57.77|3505.04|3235.12|3225.04|6740.16|238.10|258.80|2089.36|2976.32|3214.42|5065.68|5303.78|-248.72| +2452550|73966|2452559|10158|92789|1370108|2180|26699|73682|798751|4609|49481|54|9|20|3|156|411|2|89.29|103.57|8.28|190.58|16.56|178.58|207.14|0.00|0.00|20.70|16.56|16.56|37.26|37.26|-162.02| +2452550|73966|2452554|10591|92789|1370108|2180|26699|73682|798751|4609|49481|51|1|9|4|242|411|57|68.76|112.76|95.84|964.44|5462.88|3919.32|6427.32|437.03|0.00|2056.56|5462.88|5899.91|7519.44|7956.47|1543.56| +2451792|74477|2451815|15800|82498|1683673|5201|26376|5846|1629594|6254|45782|56|13|5|3|211|412|29|55.87|126.82|32.97|2721.65|956.13|1620.23|3677.78|76.49|0.00|992.96|956.13|1032.62|1949.09|2025.58|-664.10| +2451792|74477|2451810|5473|82498|1683673|5201|26376|5846|1629594|6254|45782|14|7|10|1|262|412|32|22.72|64.07|32.67|1004.80|1045.44|727.04|2050.24|52.27|0.00|184.32|1045.44|1097.71|1229.76|1282.03|318.40| +2451792|74477|2451883|10391|82498|1683673|5201|26376|5846|1629594|6254|45782|8|13|16|3|230|412|47|57.28|153.51|7.67|6854.48|360.49|2692.16|7214.97|25.09|46.86|577.16|313.63|338.72|890.79|915.88|-2378.53| +2451792|74477|2451809|14954|82498|1683673|5201|26376|5846|1629594|6254|45782|5|14|20|2|194|412|21|38.58|88.34|28.26|1261.68|593.46|810.18|1855.14|6.17|516.31|370.86|77.15|83.32|448.01|454.18|-733.03| +2451792|74477|2451825|1367|82498|1683673|5201|26376|5846|1629594|6254|45782|23|11|20|4|281|412|41|30.30|47.57|25.68|897.49|1052.88|1242.30|1950.37|10.52|0.00|428.86|1052.88|1063.40|1481.74|1492.26|-189.42| +2451792|74477|2451824|1061|82498|1683673|5201|26376|5846|1629594|6254|45782|59|17|8|4|13|412|87|30.40|32.52|31.21|113.97|2715.27|2644.80|2829.24|217.22|0.00|254.04|2715.27|2932.49|2969.31|3186.53|70.47| +2451792|74477|2451793|15397|82498|1683673|5201|26376|5846|1629594|6254|45782|32|26|4|1|28|412|65|13.00|31.07|5.28|1676.35|343.20|845.00|2019.55|0.00|0.00|120.90|343.20|343.20|464.10|464.10|-501.80| +2451792|74477|2451853|6691|82498|1683673|5201|26376|5846|1629594|6254|45782|2|17|6|3|210|412|72|94.40|180.30|27.04|11034.72|1946.88|6796.80|12981.60|38.93|0.00|1038.24|1946.88|1985.81|2985.12|3024.05|-4849.92| +2451792|74477|2451874|17831|82498|1683673|5201|26376|5846|1629594|6254|45782|59|7|5|4|90|412|52|62.41|81.75|50.68|1615.64|2635.36|3245.32|4251.00|26.35|0.00|764.92|2635.36|2661.71|3400.28|3426.63|-609.96| +2451792|74477|2451895|14753|82498|1683673|5201|26376|5846|1629594|6254|45782|5|2|11|5|119|412|38|47.15|56.10|39.83|618.26|1513.54|1791.70|2131.80|45.40|0.00|745.94|1513.54|1558.94|2259.48|2304.88|-278.16| +2451792|74477|2451869|13513|82498|1683673|5201|26376|5846|1629594|6254|45782|50|1|13|5|37|412|47|10.82|21.31|11.72|450.73|550.84|508.54|1001.57|49.57|0.00|149.93|550.84|600.41|700.77|750.34|42.30| +2451792|74477|2451862|1022|82498|1683673|5201|26376|5846|1629594|6254|45782|11|19|4|3|105|412|1|44.30|97.46|59.45|38.01|59.45|44.30|97.46|4.38|10.70|7.79|48.75|53.13|56.54|60.92|4.45| +2451792|74477|2451912|16993|82498|1683673|5201|26376|5846|1629594|6254|45782|41|11|1|3|88|412|34|60.76|173.16|86.58|2943.72|2943.72|2065.84|5887.44|29.43|0.00|2119.22|2943.72|2973.15|5062.94|5092.37|877.88| +2451792|74477|2451806|1148|82498|1683673|5201|26376|5846|1629594|6254|45782|53|8|16|4|202|412|19|60.61|176.98|37.16|2656.58|706.04|1151.59|3362.62|42.36|0.00|571.52|706.04|748.40|1277.56|1319.92|-445.55| +2451792|74477|2451796|13661|82498|1683673|5201|26376|5846|1629594|6254|45782|17|17|6|1|83|412|22|55.95|93.99|14.09|1757.80|309.98|1230.90|2067.78|27.89|0.00|516.78|309.98|337.87|826.76|854.65|-920.92| +2451792|74477|2451846|16346|82498|1683673|5201|26376|5846|1629594|6254|45782|38|5|16|4|72|412|30|82.55|99.88|10.98|2667.00|329.40|2476.50|2996.40|26.35|0.00|928.80|329.40|355.75|1258.20|1284.55|-2147.10| +2451455|61903|2451551|3284|63211|1437893|575|1704|4486|346173|4997|8479|19|7|1|3|175|413|91|8.50|23.97|19.17|436.80|1744.47|773.50|2181.27|0.00|0.00|654.29|1744.47|1744.47|2398.76|2398.76|970.97| +2451455|61903|2451522|1951|63211|1437893|575|1704|4486|346173|4997|8479|52|1|12|3|34|413|22|78.89|82.04|0.82|1786.84|18.04|1735.58|1804.88|1.44|0.00|848.10|18.04|19.48|866.14|867.58|-1717.54| +2451455|61903|2451511|4118|63211|1437893|575|1704|4486|346173|4997|8479|31|16|12|5|258|413|1|86.60|207.84|116.39|91.45|116.39|86.60|207.84|10.47|0.00|103.92|116.39|126.86|220.31|230.78|29.79| +2451455|61903|2451497|12644|63211|1437893|575|1704|4486|346173|4997|8479|38|16|5|4|200|413|39|66.65|97.30|33.08|2504.58|1290.12|2599.35|3794.70|51.60|0.00|645.06|1290.12|1341.72|1935.18|1986.78|-1309.23| +2451455|61903|2451552|7270|63211|1437893|575|1704|4486|346173|4997|8479|26|2|20|3|26|413|86|42.38|94.50|75.60|1625.40|6501.60|3644.68|8127.00|0.00|0.00|1381.16|6501.60|6501.60|7882.76|7882.76|2856.92| +2451455|61903|2451532|3313|63211|1437893|575|1704|4486|346173|4997|8479|58|2|7|4|151|413|70|5.69|6.88|0.00|481.60|0.00|398.30|481.60|0.00|0.00|226.10|0.00|0.00|226.10|226.10|-398.30| +2451455|61903|2451530|11011|63211|1437893|575|1704|4486|346173|4997|8479|26|22|7|2|50|413|46|99.71|199.42|117.65|3761.42|5411.90|4586.66|9173.32|292.24|541.19|3210.34|4870.71|5162.95|8081.05|8373.29|284.05| +2451455|61903|2451466|11635|63211|1437893|575|1704|4486|346173|4997|8479|49|13|9|1|133|413|2|76.58|91.89|31.24|121.30|62.48|153.16|183.78|0.61|1.24|77.18|61.24|61.85|138.42|139.03|-91.92| +2451455|61903|2451467|6518|63211|1437893|575|1704|4486|346173|4997|8479|1|20|4|4|244|413|46|75.93|80.48|29.77|2332.66|1369.42|3492.78|3702.08|13.69|0.00|888.26|1369.42|1383.11|2257.68|2271.37|-2123.36| +2451455|61903|2451545|10286|63211|1437893|575|1704|4486|346173|4997|8479|19|26|11|5|49|413|50|35.66|78.80|63.04|788.00|3152.00|1783.00|3940.00|157.60|0.00|1497.00|3152.00|3309.60|4649.00|4806.60|1369.00| +2451455|61903|2451541|9142|63211|1437893|575|1704|4486|346173|4997|8479|46|10|5|2|289|413|72|54.57|121.69|73.01|3504.96|5256.72|3929.04|8761.68|52.56|0.00|2277.36|5256.72|5309.28|7534.08|7586.64|1327.68| +2451455|61903|2451469|2906|63211|1437893|575|1704|4486|346173|4997|8479|16|22|8|1|164|413|62|77.41|130.82|18.31|6975.62|1135.22|4799.42|8110.84|45.40|0.00|243.04|1135.22|1180.62|1378.26|1423.66|-3664.20| +2451455|61903|2451534|7772|63211|1437893|575|1704|4486|346173|4997|8479|22|10|20|2|131|413|61|44.34|124.15|95.59|1742.16|5830.99|2704.74|7573.15|408.16|0.00|1060.18|5830.99|6239.15|6891.17|7299.33|3126.25| +2451455|61903|2451471|11443|63211|1437893|575|1704|4486|346173|4997|8479|28|8|15|3|28|413|39|99.37|197.74|118.64|3084.90|4626.96|3875.43|7711.86|71.25|1064.20|616.59|3562.76|3634.01|4179.35|4250.60|-312.67| +2451455|61903|2451554|9638|63211|1437893|575|1704|4486|346173|4997|8479|58|16|15|4|133|413|2|83.88|95.62|9.56|172.12|19.12|167.76|191.24|0.95|0.00|15.28|19.12|20.07|34.40|35.35|-148.64| +2451179|72169|2451235|11108|31776|428060|3097|48581|11586|944668|6986|15223|25|8|7|4|161|414|57|43.59|127.71|61.30|3785.37|3494.10|2484.63|7279.47|139.76|0.00|3639.45|3494.10|3633.86|7133.55|7273.31|1009.47| +2451179|72169|2451275|3190|31776|428060|3097|48581|11586|944668|6986|15223|31|26|12|5|98|414|32|39.08|41.42|24.85|530.24|795.20|1250.56|1325.44|39.76|0.00|159.04|795.20|834.96|954.24|994.00|-455.36| +2451179|72169|2451238|10825|31776|428060|3097|48581|11586|944668|6986|15223|50|13|5|1|197|414|60|23.59|34.67|32.93|104.40|1975.80|1415.40|2080.20|79.03|0.00|499.20|1975.80|2054.83|2475.00|2554.03|560.40| +2451179|72169|2451286|17014|31776|428060|3097|48581|11586|944668|6986|15223|1|26|20|1|32|414|82|45.38|117.08|111.22|480.52|9120.04|3721.16|9600.56|820.80|0.00|1919.62|9120.04|9940.84|11039.66|11860.46|5398.88| +2451179|72169|2451235|4178|31776|428060|3097|48581|11586|944668|6986|15223|4|4|3|5|10|414|5|45.93|84.51|26.19|291.60|130.95|229.65|422.55|7.85|0.00|211.25|130.95|138.80|342.20|350.05|-98.70| +2451179|72169|2451268|8950|31776|428060|3097|48581|11586|944668|6986|15223|37|28|1|2|63|414|87|98.01|151.91|139.75|1057.92|12158.25|8526.87|13216.17|519.15|4741.71|4096.83|7416.54|7935.69|11513.37|12032.52|-1110.33| +2451179|72169|2451238|13369|31776|428060|3097|48581|11586|944668|6986|15223|1|25|8|5|65|414|98|88.74|197.00|88.65|10618.30|8687.70|8696.52|19306.00|370.96|3388.20|1544.48|5299.50|5670.46|6843.98|7214.94|-3397.02| +2451179|72169|2451206|728|31776|428060|3097|48581|11586|944668|6986|15223|38|8|20|3|133|414|14|95.21|133.29|79.97|746.48|1119.58|1332.94|1866.06|78.37|0.00|541.10|1119.58|1197.95|1660.68|1739.05|-213.36| +2451179|72169|2451187|3748|31776|428060|3097|48581|11586|944668|6986|15223|46|4|15|4|233|414|70|49.36|99.70|48.85|3559.50|3419.50|3455.20|6979.00|102.58|0.00|3349.50|3419.50|3522.08|6769.00|6871.58|-35.70| +2451179|72169|2451263|10652|31776|428060|3097|48581|11586|944668|6986|15223|4|2|8|4|43|414|86|62.49|141.22|131.33|850.54|11294.38|5374.14|12144.92|564.71|0.00|849.68|11294.38|11859.09|12144.06|12708.77|5920.24| +2451179|72169|2451299|9742|31776|428060|3097|48581|11586|944668|6986|15223|4|7|5|1|7|414|24|32.28|65.20|46.94|438.24|1126.56|774.72|1564.80|22.53|0.00|109.44|1126.56|1149.09|1236.00|1258.53|351.84| +2451179|72169|2451259|1402|31776|428060|3097|48581|11586|944668|6986|15223|26|1|1|1|241|414|33|89.89|179.78|163.59|534.27|5398.47|2966.37|5932.74|288.27|593.83|2669.70|4804.64|5092.91|7474.34|7762.61|1838.27| +2451179|72169|2451255|256|31776|428060|3097|48581|11586|944668|6986|15223|2|25|11|4|247|414|24|33.24|91.74|57.79|814.80|1386.96|797.76|2201.76|27.73|0.00|770.40|1386.96|1414.69|2157.36|2185.09|589.20| +2451179|72169|2451270|11000|31776|428060|3097|48581|11586|944668|6986|15223|25|25|20|5|57|414|65|99.51|107.47|35.46|4680.65|2304.90|6468.15|6985.55|92.19|0.00|278.85|2304.90|2397.09|2583.75|2675.94|-4163.25| +2451066|40145|2451139|5552|47379|242090|6848|5159|5905|1546625|4539|21297|4|14|13|3|60|415|25|20.91|23.41|14.28|228.25|357.00|522.75|585.25|14.28|0.00|222.25|357.00|371.28|579.25|593.53|-165.75| +2451066|40145|2451163|8716|47379|242090|6848|5159|5905|1546625|4539|21297|58|16|18|5|149|415|85|38.30|40.98|36.88|348.50|3134.80|3255.50|3483.30|188.08|0.00|696.15|3134.80|3322.88|3830.95|4019.03|-120.70| +2451066|40145|2451148|3932|47379|242090|6848|5159|5905|1546625|4539|21297|40|25|2|1|93|415|12|98.12|214.88|68.76|1753.44|825.12|1177.44|2578.56|0.00|0.00|902.40|825.12|825.12|1727.52|1727.52|-352.32| +2451066|40145|2451072|11294|47379|242090|6848|5159|5905|1546625|4539|21297|14|8|2|2|79|415|7|90.61|155.84|35.84|840.00|250.88|634.27|1090.88|20.07|0.00|250.88|250.88|270.95|501.76|521.83|-383.39| +2451066|40145|2451151|8833|47379|242090|6848|5159|5905|1546625|4539|21297|8|10|17|3|39|415|67|72.77|78.59|40.86|2527.91|2737.62|4875.59|5265.53|54.75|0.00|0.00|2737.62|2792.37|2737.62|2792.37|-2137.97| +2451066|40145|2451155|9751|47379|242090|6848|5159|5905|1546625|4539|21297|1|10|18|3|215|415|13|72.88|208.43|50.02|2059.33|650.26|947.44|2709.59|58.52|0.00|406.38|650.26|708.78|1056.64|1115.16|-297.18| +2451066|40145|2451122|5707|47379|242090|6848|5159|5905|1546625|4539|21297|4|13|14|1|88|415|41|77.10|88.66|68.26|836.40|2798.66|3161.10|3635.06|27.98|0.00|1744.55|2798.66|2826.64|4543.21|4571.19|-362.44| +2451066|40145|2451156|11576|47379|242090|6848|5159|5905|1546625|4539|21297|52|14|9|2|138|415|8|79.84|237.12|125.67|891.60|1005.36|638.72|1896.96|20.10|0.00|360.40|1005.36|1025.46|1365.76|1385.86|366.64| +2451066|40145|2451136|14108|47379|242090|6848|5159|5905|1546625|4539|21297|22|8|15|3|118|415|77|26.00|41.86|41.44|32.34|3190.88|2002.00|3223.22|287.17|0.00|644.49|3190.88|3478.05|3835.37|4122.54|1188.88| +2451066|40145|2451086|9788|47379|242090|6848|5159|5905|1546625|4539|21297|2|4|4|3|299|415|27|89.81|158.96|100.14|1588.14|2703.78|2424.87|4291.92|0.00|0.00|171.45|2703.78|2703.78|2875.23|2875.23|278.91| +2451066|40145|2451092|578|47379|242090|6848|5159|5905|1546625|4539|21297|28|7|13|5|234|415|92|3.37|9.57|2.77|625.60|254.84|310.04|880.44|18.55|22.93|299.00|231.91|250.46|530.91|549.46|-78.13| +2451066|40145|2451106|721|47379|242090|6848|5159|5905|1546625|4539|21297|43|25|1|2|210|415|24|95.38|262.29|86.55|4217.76|2077.20|2289.12|6294.96|20.77|0.00|2454.96|2077.20|2097.97|4532.16|4552.93|-211.92| +2451066|40145|2451081|772|47379|242090|6848|5159|5905|1546625|4539|21297|28|10|7|3|249|415|15|47.83|122.92|11.06|1677.90|165.90|717.45|1843.80|9.95|0.00|626.85|165.90|175.85|792.75|802.70|-551.55| +2451066|40145|2451179|2521|47379|242090|6848|5159|5905|1546625|4539|21297|43|4|4|3|29|415|60|77.36|86.64|52.85|2027.40|3171.00|4641.60|5198.40|0.00|0.00|935.40|3171.00|3171.00|4106.40|4106.40|-1470.60| +2451066|40145|2451123|11065|47379|242090|6848|5159|5905|1546625|4539|21297|32|19|13|1|116|415|32|92.62|227.84|138.98|2843.52|4447.36|2963.84|7290.88|133.42|0.00|728.96|4447.36|4580.78|5176.32|5309.74|1483.52| +2451066|40145|2451125|2696|47379|242090|6848|5159|5905|1546625|4539|21297|4|10|12|3|294|415|61|86.75|86.75|40.77|2804.78|2486.97|5291.75|5291.75|99.47|0.00|2486.97|2486.97|2586.44|4973.94|5073.41|-2804.78| +2451138|80205|2451244|11998|45842|1020550|58|20813|82958|1317442|584|26539|52|20|15|4|210|416|77|52.80|144.14|23.06|9323.16|1775.62|4065.60|11098.78|106.53|0.00|2663.43|1775.62|1882.15|4439.05|4545.58|-2289.98| +2451138|80205|2451183|7262|45842|1020550|58|20813|82958|1317442|584|26539|43|8|11|2|233|416|19|16.52|26.43|24.05|45.22|456.95|313.88|502.17|0.00|0.00|185.63|456.95|456.95|642.58|642.58|143.07| +2451138|80205|2451257|5270|45842|1020550|58|20813|82958|1317442|584|26539|25|19|17|4|121|416|96|10.66|24.94|4.23|1988.16|406.08|1023.36|2394.24|24.36|0.00|335.04|406.08|430.44|741.12|765.48|-617.28| +2451138|80205|2451204|4592|45842|1020550|58|20813|82958|1317442|584|26539|44|25|16|4|142|416|18|26.46|65.09|29.94|632.70|538.92|476.28|1171.62|32.33|0.00|468.54|538.92|571.25|1007.46|1039.79|62.64| +2451138|80205|2451203|5323|45842|1020550|58|20813|82958|1317442|584|26539|20|13|3|4|175|416|97|48.63|87.04|1.74|8274.10|168.78|4717.11|8442.88|0.00|145.15|2532.67|23.63|23.63|2556.30|2556.30|-4693.48| +2451138|80205|2451235|5071|45842|1020550|58|20813|82958|1317442|584|26539|55|2|9|1|150|416|48|73.46|202.01|20.20|8726.88|969.60|3526.08|9696.48|87.26|0.00|1939.20|969.60|1056.86|2908.80|2996.06|-2556.48| +2451138|80205|2451241|7237|45842|1020550|58|20813|82958|1317442|584|26539|55|28|12|2|149|416|16|76.49|225.64|45.12|2888.32|721.92|1223.84|3610.24|7.21|0.00|649.76|721.92|729.13|1371.68|1378.89|-501.92| +2451138|80205|2451248|208|45842|1020550|58|20813|82958|1317442|584|26539|13|26|1|3|65|416|13|47.59|50.44|5.54|583.70|72.02|618.67|655.72|1.10|16.56|117.91|55.46|56.56|173.37|174.47|-563.21| +2450969|5839|2450999|11449|5674|757532|4943|3601|40298|38198|3928|45386|10|20|9|4|49|417|45|24.18|63.83|14.68|2211.75|660.60|1088.10|2872.35|46.24|0.00|1033.65|660.60|706.84|1694.25|1740.49|-427.50| +2450969|5839|2451015|934|5674|757532|4943|3601|40298|38198|3928|45386|38|7|15|4|83|417|36|27.71|40.73|33.80|249.48|1216.80|997.56|1466.28|109.51|0.00|498.24|1216.80|1326.31|1715.04|1824.55|219.24| +2450969|5839|2450970|1360|5674|757532|4943|3601|40298|38198|3928|45386|20|22|13|4|165|417|30|48.96|81.76|27.79|1619.10|833.70|1468.80|2452.80|58.35|0.00|1103.70|833.70|892.05|1937.40|1995.75|-635.10| +2450969|5839|2451006|475|5674|757532|4943|3601|40298|38198|3928|45386|40|28|6|5|124|417|96|93.95|200.11|184.10|1536.96|17673.60|9019.20|19210.56|530.20|0.00|6722.88|17673.60|18203.80|24396.48|24926.68|8654.40| +2450969|5839|2451043|15427|5674|757532|4943|3601|40298|38198|3928|45386|50|16|2|3|11|417|5|26.18|44.50|13.35|155.75|66.75|130.90|222.50|5.34|0.00|46.70|66.75|72.09|113.45|118.79|-64.15| +2450969|5839|2451006|9088|5674|757532|4943|3601|40298|38198|3928|45386|25|16|8|4|20|417|17|11.48|19.51|0.00|331.67|0.00|195.16|331.67|0.00|0.00|82.79|0.00|0.00|82.79|82.79|-195.16| +2450969|5839|2451073|127|5674|757532|4943|3601|40298|38198|3928|45386|14|28|2|1|133|417|82|9.51|11.60|9.86|142.68|808.52|779.82|951.20|24.25|0.00|9.02|808.52|832.77|817.54|841.79|28.70| +2450969|5839|2451002|7117|5674|757532|4943|3601|40298|38198|3928|45386|50|14|10|2|224|417|49|82.19|126.57|115.17|558.60|5643.33|4027.31|6201.93|41.76|3555.29|1550.36|2088.04|2129.80|3638.40|3680.16|-1939.27| +2450969|5839|2451043|12049|5674|757532|4943|3601|40298|38198|3928|45386|34|10|7|3|189|417|40|46.16|92.78|25.05|2709.20|1002.00|1846.40|3711.20|20.04|0.00|1002.00|1002.00|1022.04|2004.00|2024.04|-844.40| +2450969|5839|2450981|6739|5674|757532|4943|3601|40298|38198|3928|45386|13|10|2|1|17|417|74|9.07|24.30|5.10|1420.80|377.40|671.18|1798.20|22.64|0.00|125.80|377.40|400.04|503.20|525.84|-293.78| +2450969|5839|2450988|818|5674|757532|4943|3601|40298|38198|3928|45386|1|14|8|5|82|417|17|66.60|91.24|83.02|139.74|1411.34|1132.20|1551.08|70.56|0.00|666.91|1411.34|1481.90|2078.25|2148.81|279.14| +2450969|5839|2450993|12997|5674|757532|4943|3601|40298|38198|3928|45386|50|2|8|2|223|417|69|28.86|81.09|39.73|2853.84|2741.37|1991.34|5595.21|98.68|1644.82|614.79|1096.55|1195.23|1711.34|1810.02|-894.79| +2452194|45320|2452296|683|16065|180669|3226|9030|21998|1430562|395|10552|13|25|19|3|174|418|42|65.29|131.88|54.07|3268.02|2270.94|2742.18|5538.96|136.25|0.00|609.00|2270.94|2407.19|2879.94|3016.19|-471.24| +2452194|45320|2452217|1939|16065|180669|3226|9030|21998|1430562|395|10552|31|19|4|3|135|418|92|36.14|100.10|26.02|6815.36|2393.84|3324.88|9209.20|167.56|0.00|4327.68|2393.84|2561.40|6721.52|6889.08|-931.04| +2452194|45320|2452204|7337|16065|180669|3226|9030|21998|1430562|395|10552|1|7|7|5|193|418|45|87.90|194.25|36.90|7080.75|1660.50|3955.50|8741.25|83.02|0.00|2185.20|1660.50|1743.52|3845.70|3928.72|-2295.00| +2452194|45320|2452299|11857|16065|180669|3226|9030|21998|1430562|395|10552|51|9|17|4|18|418|24|7.56|11.79|4.83|167.04|115.92|181.44|282.96|0.00|0.00|28.08|115.92|115.92|144.00|144.00|-65.52| +2452194|45320|2452223|7073|16065|180669|3226|9030|21998|1430562|395|10552|47|21|15|2|196|418|23|93.94|253.63|218.12|816.73|5016.76|2160.62|5833.49|401.34|0.00|2858.21|5016.76|5418.10|7874.97|8276.31|2856.14| +2452194|45320|2452232|9827|16065|180669|3226|9030|21998|1430562|395|10552|39|5|6|2|185|418|63|85.53|159.94|25.59|8464.05|1612.17|5388.39|10076.22|48.36|0.00|1410.57|1612.17|1660.53|3022.74|3071.10|-3776.22| +2452194|45320|2452255|11615|16065|180669|3226|9030|21998|1430562|395|10552|11|23|15|1|44|418|43|88.56|99.18|22.81|3283.91|980.83|3808.08|4264.74|9.80|0.00|810.12|980.83|990.63|1790.95|1800.75|-2827.25| +2452194|45320|2452242|11681|16065|180669|3226|9030|21998|1430562|395|10552|3|27|6|3|20|418|45|74.08|111.86|15.66|4329.00|704.70|3333.60|5033.70|63.42|0.00|603.90|704.70|768.12|1308.60|1372.02|-2628.90| +2452194|45320|2452200|6305|16065|180669|3226|9030|21998|1430562|395|10552|51|13|18|4|100|418|26|89.08|163.01|53.79|2839.72|1398.54|2316.08|4238.26|111.88|0.00|720.46|1398.54|1510.42|2119.00|2230.88|-917.54| +2452194|45320|2452219|13057|16065|180669|3226|9030|21998|1430562|395|10552|33|7|1|1|18|418|69|21.21|28.20|22.27|409.17|1536.63|1463.49|1945.80|15.36|0.00|116.61|1536.63|1551.99|1653.24|1668.60|73.14| +2452194|45320|2452306|14845|16065|180669|3226|9030|21998|1430562|395|10552|33|1|2|1|297|418|92|80.25|175.74|151.13|2264.12|13903.96|7383.00|16168.08|139.03|0.00|6790.52|13903.96|14042.99|20694.48|20833.51|6520.96| +2452194|45320|2452218|16473|16065|180669|3226|9030|21998|1430562|395|10552|1|7|7|5|73|418|48|17.36|19.09|4.77|687.36|228.96|833.28|916.32|9.15|0.00|329.76|228.96|238.11|558.72|567.87|-604.32| +2452194|45320|2452254|1427|16065|180669|3226|9030|21998|1430562|395|10552|59|23|4|1|101|418|2|6.94|12.14|7.89|8.50|15.78|13.88|24.28|0.19|10.88|11.64|4.90|5.09|16.54|16.73|-8.98| +2452135|80154|2452228|12821|78262|23036|610|17384|65495|1865218|3706|36527|7|11|1|3|178|419|44|20.49|45.48|3.63|1841.40|159.72|901.56|2001.12|1.59|0.00|740.08|159.72|161.31|899.80|901.39|-741.84| +2452135|80154|2452171|3471|78262|23036|610|17384|65495|1865218|3706|36527|41|7|14|2|50|419|100|7.27|21.51|12.04|947.00|1204.00|727.00|2151.00|0.00|0.00|451.00|1204.00|1204.00|1655.00|1655.00|477.00| +2452135|80154|2452222|301|78262|23036|610|17384|65495|1865218|3706|36527|39|3|4|1|98|419|67|49.91|129.26|7.75|8141.17|519.25|3343.97|8660.42|20.77|0.00|86.43|519.25|540.02|605.68|626.45|-2824.72| +2452135|80154|2452223|15467|78262|23036|610|17384|65495|1865218|3706|36527|45|15|6|1|39|419|39|8.10|24.05|7.45|647.40|290.55|315.90|937.95|17.43|0.00|327.99|290.55|307.98|618.54|635.97|-25.35| +2452135|80154|2452228|5453|78262|23036|610|17384|65495|1865218|3706|36527|21|29|16|5|71|419|85|79.83|111.76|11.17|8550.15|949.45|6785.55|9499.60|18.98|0.00|4369.00|949.45|968.43|5318.45|5337.43|-5836.10| +2452135|80154|2452163|3643|78262|23036|610|17384|65495|1865218|3706|36527|3|1|10|4|215|419|97|18.92|49.00|10.29|3754.87|998.13|1835.24|4753.00|89.83|0.00|285.18|998.13|1087.96|1283.31|1373.14|-837.11| +2452135|80154|2452163|12663|78262|23036|610|17384|65495|1865218|3706|36527|41|29|9|3|207|419|58|52.77|56.99|2.84|3140.70|164.72|3060.66|3305.42|1.64|0.00|892.04|164.72|166.36|1056.76|1058.40|-2895.94| +2452135|80154|2452161|17521|78262|23036|610|17384|65495|1865218|3706|36527|25|9|17|4|108|419|89|71.27|79.82|35.91|3907.99|3195.99|6343.03|7103.98|223.71|0.00|1491.64|3195.99|3419.70|4687.63|4911.34|-3147.04| +2452135|80154|2452239|17743|78262|23036|610|17384|65495|1865218|3706|36527|5|17|12|5|131|419|5|67.16|150.43|25.57|624.30|127.85|335.80|752.15|2.55|0.00|60.15|127.85|130.40|188.00|190.55|-207.95| +2452135|80154|2452213|5043|78262|23036|610|17384|65495|1865218|3706|36527|27|3|17|2|167|419|81|87.50|100.62|67.41|2690.01|5460.21|7087.50|8150.22|285.02|709.82|570.24|4750.39|5035.41|5320.63|5605.65|-2337.11| +2452135|80154|2452193|16611|78262|23036|610|17384|65495|1865218|3706|36527|13|11|2|1|17|419|3|50.54|140.50|98.35|126.45|295.05|151.62|421.50|4.83|174.07|63.21|120.98|125.81|184.19|189.02|-30.64| +2452135|80154|2452150|16873|78262|23036|610|17384|65495|1865218|3706|36527|9|7|3|2|61|419|14|82.88|181.50|121.60|838.60|1702.40|1160.32|2541.00|0.00|0.00|482.72|1702.40|1702.40|2185.12|2185.12|542.08| +2452135|80154|2452230|16809|78262|23036|610|17384|65495|1865218|3706|36527|33|17|11|1|278|419|59|85.45|164.06|37.73|7453.47|2226.07|5041.55|9679.54|0.00|0.00|1742.27|2226.07|2226.07|3968.34|3968.34|-2815.48| +2452135|80154|2452252|16087|78262|23036|610|17384|65495|1865218|3706|36527|57|9|1|5|256|419|97|39.90|103.34|80.60|2205.78|7818.20|3870.30|10023.98|365.89|3752.73|902.10|4065.47|4431.36|4967.57|5333.46|195.17| +2452135|80154|2452159|5351|78262|23036|610|17384|65495|1865218|3706|36527|25|3|4|3|82|419|26|99.23|172.66|58.70|2962.96|1526.20|2579.98|4489.16|45.78|0.00|1930.24|1526.20|1571.98|3456.44|3502.22|-1053.78| +2452196|64135|2452221|8993|30049|925568|5534|32653|68967|529142|6104|25223|59|5|3|1|26|420|38|2.09|5.03|3.92|42.18|148.96|79.42|191.14|7.44|0.00|82.08|148.96|156.40|231.04|238.48|69.54| +2452196|64135|2452230|17045|30049|925568|5534|32653|68967|529142|6104|25223|53|17|4|1|8|420|51|60.36|70.01|7.70|3177.81|392.70|3078.36|3570.51|35.34|0.00|1428.00|392.70|428.04|1820.70|1856.04|-2685.66| +2452196|64135|2452267|5697|30049|925568|5534|32653|68967|529142|6104|25223|31|21|9|4|102|420|84|19.40|21.14|20.29|71.40|1704.36|1629.60|1775.76|136.34|0.00|319.20|1704.36|1840.70|2023.56|2159.90|74.76| +2452196|64135|2452251|10655|30049|925568|5534|32653|68967|529142|6104|25223|53|5|12|3|194|420|92|32.10|64.20|24.39|3662.52|2243.88|2953.20|5906.40|112.19|0.00|1416.80|2243.88|2356.07|3660.68|3772.87|-709.32| +2452196|64135|2452226|1881|30049|925568|5534|32653|68967|529142|6104|25223|3|9|11|4|30|420|83|93.48|180.41|92.00|7338.03|7636.00|7758.84|14974.03|119.12|1679.92|1497.32|5956.08|6075.20|7453.40|7572.52|-1802.76| +2452196|64135|2452225|2373|30049|925568|5534|32653|68967|529142|6104|25223|57|21|13|5|47|420|92|47.58|125.13|107.61|1611.84|9900.12|4377.36|11511.96|495.00|0.00|460.00|9900.12|10395.12|10360.12|10855.12|5522.76| +2452196|64135|2452314|15951|30049|925568|5534|32653|68967|529142|6104|25223|11|25|10|4|191|420|96|55.91|121.32|33.96|8386.56|3260.16|5367.36|11646.72|97.80|0.00|465.60|3260.16|3357.96|3725.76|3823.56|-2107.20| +2452196|64135|2452313|5091|30049|925568|5534|32653|68967|529142|6104|25223|3|21|7|3|112|420|92|84.88|157.87|85.24|6681.96|7842.08|7808.96|14524.04|705.78|0.00|2323.00|7842.08|8547.86|10165.08|10870.86|33.12| +2451128|75260|2451146|3158|94345|605879|260|42641|88091|1544095|6937|34918|26|28|9|5|11|421|38|8.30|21.74|1.30|776.72|49.40|315.40|826.12|2.96|0.00|148.58|49.40|52.36|197.98|200.94|-266.00| +2451128|75260|2451160|2752|94345|605879|260|42641|88091|1544095|6937|34918|19|28|3|3|106|421|21|41.21|84.48|6.75|1632.33|141.75|865.41|1774.08|8.50|0.00|514.29|141.75|150.25|656.04|664.54|-723.66| +2451128|75260|2451209|2036|94345|605879|260|42641|88091|1544095|6937|34918|7|2|7|2|167|421|63|40.36|100.90|3.02|6166.44|190.26|2542.68|6356.70|9.51|0.00|825.93|190.26|199.77|1016.19|1025.70|-2352.42| +2451128|75260|2451132|16174|94345|605879|260|42641|88091|1544095|6937|34918|58|19|10|5|63|421|100|29.94|80.83|25.05|5578.00|2505.00|2994.00|8083.00|122.24|976.95|3233.00|1528.05|1650.29|4761.05|4883.29|-1465.95| +2451128|75260|2451234|11140|94345|605879|260|42641|88091|1544095|6937|34918|26|10|1|3|268|421|46|36.97|69.13|17.28|2385.10|794.88|1700.62|3179.98|39.74|0.00|1049.26|794.88|834.62|1844.14|1883.88|-905.74| +2451128|75260|2451171|16072|94345|605879|260|42641|88091|1544095|6937|34918|2|28|18|1|197|421|48|60.17|180.51|61.37|5718.72|2945.76|2888.16|8664.48|58.91|0.00|1819.20|2945.76|3004.67|4764.96|4823.87|57.60| +2451128|75260|2451228|17533|94345|605879|260|42641|88091|1544095|6937|34918|25|20|9|5|148|421|52|94.43|113.31|94.04|1002.04|4890.08|4910.36|5892.12|244.50|0.00|1826.24|4890.08|5134.58|6716.32|6960.82|-20.28| +2451128|75260|2451129|13927|94345|605879|260|42641|88091|1544095|6937|34918|43|2|5|5|228|421|9|66.69|100.70|27.18|661.68|244.62|600.21|906.30|14.67|0.00|135.90|244.62|259.29|380.52|395.19|-355.59| +2451128|75260|2451219|1954|94345|605879|260|42641|88091|1544095|6937|34918|8|13|9|1|200|421|64|64.56|146.55|114.30|2064.00|7315.20|4131.84|9379.20|0.00|0.00|0.00|7315.20|7315.20|7315.20|7315.20|3183.36| +2451128|75260|2451210|1069|94345|605879|260|42641|88091|1544095|6937|34918|19|7|1|5|92|421|59|97.73|230.64|29.98|11838.94|1768.82|5766.07|13607.76|88.44|0.00|6531.30|1768.82|1857.26|8300.12|8388.56|-3997.25| +2451128|75260|2451229|9326|94345|605879|260|42641|88091|1544095|6937|34918|50|28|11|5|146|421|11|12.77|26.17|24.86|14.41|273.46|140.47|287.87|8.20|0.00|51.81|273.46|281.66|325.27|333.47|132.99| +2451128|75260|2451159|17152|94345|605879|260|42641|88091|1544095|6937|34918|2|10|20|1|85|421|90|94.31|245.20|215.77|2648.70|19419.30|8487.90|22068.00|0.00|0.00|6620.40|19419.30|19419.30|26039.70|26039.70|10931.40| +2451834|4853|2451897|14603|52567|257532|2850|17815|52875|864863|5463|32381|56|23|18|1|164|422|16|97.69|202.21|184.01|291.20|2944.16|1563.04|3235.36|58.88|0.00|1391.20|2944.16|3003.04|4335.36|4394.24|1381.12| +2451834|4853|2451946|10079|52567|257532|2850|17815|52875|864863|5463|32381|32|8|6|4|195|422|35|33.79|90.89|48.17|1495.20|1685.95|1182.65|3181.15|84.29|0.00|127.05|1685.95|1770.24|1813.00|1897.29|503.30| +2451834|4853|2451904|15560|52567|257532|2850|17815|52875|864863|5463|32381|2|23|19|4|146|422|28|85.63|185.81|163.51|624.40|4578.28|2397.64|5202.68|44.86|2334.92|676.20|2243.36|2288.22|2919.56|2964.42|-154.28| +2451834|4853|2451891|14372|52567|257532|2850|17815|52875|864863|5463|32381|44|17|15|4|110|422|98|97.65|227.52|40.95|18283.86|4013.10|9569.70|22296.96|44.14|1805.89|4459.00|2207.21|2251.35|6666.21|6710.35|-7362.49| +2451834|4853|2451858|3227|52567|257532|2850|17815|52875|864863|5463|32381|11|1|16|4|159|422|27|94.84|147.00|102.90|1190.70|2778.30|2560.68|3969.00|27.78|0.00|873.18|2778.30|2806.08|3651.48|3679.26|217.62| +2451834|4853|2451848|14708|52567|257532|2850|17815|52875|864863|5463|32381|55|13|3|2|11|422|97|55.12|143.86|37.40|10326.62|3627.80|5346.64|13954.42|178.48|653.00|2092.29|2974.80|3153.28|5067.09|5245.57|-2371.84| +2451834|4853|2451870|13280|52567|257532|2850|17815|52875|864863|5463|32381|29|23|19|4|146|422|18|94.74|229.27|151.31|1403.28|2723.58|1705.32|4126.86|108.94|0.00|1692.00|2723.58|2832.52|4415.58|4524.52|1018.26| +2451834|4853|2451876|4904|52567|257532|2850|17815|52875|864863|5463|32381|35|2|9|3|7|422|83|50.79|101.07|28.29|6040.74|2348.07|4215.57|8388.81|0.00|0.00|3942.50|2348.07|2348.07|6290.57|6290.57|-1867.50| +2451834|4853|2451930|13982|52567|257532|2850|17815|52875|864863|5463|32381|37|23|5|3|164|422|89|4.51|7.08|6.58|44.50|585.62|401.39|630.12|40.75|76.13|301.71|509.49|550.24|811.20|851.95|108.10| +2451834|4853|2451943|15554|52567|257532|2850|17815|52875|864863|5463|32381|7|26|11|1|268|422|2|97.04|169.82|169.82|0.00|339.64|194.08|339.64|23.77|0.00|0.00|339.64|363.41|339.64|363.41|145.56| +2451834|4853|2451904|12428|52567|257532|2850|17815|52875|864863|5463|32381|20|17|7|5|100|422|64|81.54|185.09|166.58|1184.64|10661.12|5218.56|11845.76|81.02|2558.66|1302.40|8102.46|8183.48|9404.86|9485.88|2883.90| +2452135|31537|2452186|5771|7094|878151|4055|18759|88613|1017231|5177|11881|7|17|4|4|294|423|50|82.79|91.06|86.50|228.00|4325.00|4139.50|4553.00|129.75|1081.25|1548.00|3243.75|3373.50|4791.75|4921.50|-895.75| +2452135|31537|2452246|2837|7094|878151|4055|18759|88613|1017231|5177|11881|11|13|5|1|204|423|100|56.70|142.88|112.87|3001.00|11287.00|5670.00|14288.00|790.09|0.00|3143.00|11287.00|12077.09|14430.00|15220.09|5617.00| +2452135|31537|2452175|13019|7094|878151|4055|18759|88613|1017231|5177|11881|13|11|15|4|123|423|52|59.65|116.31|48.85|3507.92|2540.20|3101.80|6048.12|32.00|1473.31|1088.36|1066.89|1098.89|2155.25|2187.25|-2034.91| +2452135|31537|2452211|6725|7094|878151|4055|18759|88613|1017231|5177|11881|27|21|15|2|286|423|4|25.05|60.87|1.82|236.20|7.28|100.20|243.48|0.29|0.00|104.68|7.28|7.57|111.96|112.25|-92.92| +2452135|31537|2452186|10499|7094|878151|4055|18759|88613|1017231|5177|11881|35|29|5|5|105|423|75|36.25|47.12|24.97|1661.25|1872.75|2718.75|3534.00|131.09|0.00|1413.00|1872.75|2003.84|3285.75|3416.84|-846.00| +2452135|31537|2452227|13385|7094|878151|4055|18759|88613|1017231|5177|11881|47|19|13|4|102|423|90|84.45|150.32|16.53|12041.10|1487.70|7600.50|13528.80|74.38|0.00|5681.70|1487.70|1562.08|7169.40|7243.78|-6112.80| +2452135|31537|2452191|5201|7094|878151|4055|18759|88613|1017231|5177|11881|51|9|4|3|242|423|53|90.43|172.72|93.26|4211.38|4942.78|4792.79|9154.16|296.56|0.00|4119.16|4942.78|5239.34|9061.94|9358.50|149.99| +2452135|31537|2452175|5443|7094|878151|4055|18759|88613|1017231|5177|11881|15|21|4|4|207|423|8|37.15|102.53|95.35|57.44|762.80|297.20|820.24|61.02|0.00|131.20|762.80|823.82|894.00|955.02|465.60| +2452135|31537|2452144|17639|7094|878151|4055|18759|88613|1017231|5177|11881|37|15|17|1|35|423|55|21.74|54.78|33.96|1145.10|1867.80|1195.70|3012.90|149.42|0.00|300.85|1867.80|2017.22|2168.65|2318.07|672.10| +2452135|31537|2452247|8791|7094|878151|4055|18759|88613|1017231|5177|11881|1|7|20|3|66|423|40|63.04|167.05|35.08|5278.80|1403.20|2521.60|6682.00|4.49|1178.68|2672.80|224.52|229.01|2897.32|2901.81|-2297.08| +2452135|31537|2452186|13981|7094|878151|4055|18759|88613|1017231|5177|11881|41|11|11|1|201|423|31|35.41|89.58|56.43|1027.65|1749.33|1097.71|2776.98|69.97|0.00|1055.24|1749.33|1819.30|2804.57|2874.54|651.62| +2451102|42109|2451182|16868|67026|579460|5847|15081|67026|579460|5847|15081|22|20|20|2|91|424|53|68.96|151.71|118.33|1769.14|6271.49|3654.88|8040.63|0.00|5456.19|1205.75|815.30|815.30|2021.05|2021.05|-2839.58| +2451102|42109|2451197|1160|67026|579460|5847|15081|67026|579460|5847|15081|32|8|20|5|223|424|35|1.33|3.97|0.67|115.50|23.45|46.55|138.95|0.40|16.64|55.30|6.81|7.21|62.11|62.51|-39.74| +2451102|42109|2451173|6262|67026|579460|5847|15081|67026|579460|5847|15081|46|7|1|3|145|424|6|42.05|111.43|4.45|641.88|26.70|252.30|668.58|0.14|24.83|220.62|1.87|2.01|222.49|222.63|-250.43| +2451102|42109|2451208|17498|67026|579460|5847|15081|67026|579460|5847|15081|14|19|15|2|212|424|30|8.42|12.96|10.62|70.20|318.60|252.60|388.80|19.11|0.00|186.60|318.60|337.71|505.20|524.31|66.00| +2451102|42109|2451215|5041|67026|579460|5847|15081|67026|579460|5847|15081|7|22|10|5|253|424|62|62.71|184.99|136.89|2982.20|8487.18|3888.02|11469.38|254.61|0.00|572.88|8487.18|8741.79|9060.06|9314.67|4599.16| +2451102|42109|2451207|9110|67026|579460|5847|15081|67026|579460|5847|15081|31|20|6|1|213|424|3|22.78|50.34|6.54|131.40|19.62|68.34|151.02|0.19|0.00|3.00|19.62|19.81|22.62|22.81|-48.72| +2451102|42109|2451107|16288|67026|579460|5847|15081|67026|579460|5847|15081|38|10|19|5|38|424|9|82.24|97.86|72.41|229.05|651.69|740.16|880.74|13.03|0.00|211.32|651.69|664.72|863.01|876.04|-88.47| +2451102|42109|2451208|5572|67026|579460|5847|15081|67026|579460|5847|15081|28|26|13|5|124|424|15|45.30|70.21|16.85|800.40|252.75|679.50|1053.15|12.63|0.00|263.25|252.75|265.38|516.00|528.63|-426.75| +2451102|42109|2451130|14042|67026|579460|5847|15081|67026|579460|5847|15081|49|28|2|2|82|424|16|68.70|199.91|163.92|575.84|2622.72|1099.20|3198.56|183.59|0.00|959.52|2622.72|2806.31|3582.24|3765.83|1523.52| +2451102|42109|2451152|5792|67026|579460|5847|15081|67026|579460|5847|15081|10|22|8|4|60|424|27|61.85|144.72|75.25|1875.69|2031.75|1669.95|3907.44|182.85|0.00|781.38|2031.75|2214.60|2813.13|2995.98|361.80| +2451102|42109|2451210|17480|67026|579460|5847|15081|67026|579460|5847|15081|25|7|18|4|23|424|97|70.94|199.34|25.91|16822.71|2513.27|6881.18|19335.98|175.92|0.00|9667.99|2513.27|2689.19|12181.26|12357.18|-4367.91| +2451102|42109|2451169|2534|67026|579460|5847|15081|67026|579460|5847|15081|44|26|5|1|77|424|38|95.66|141.57|77.86|2420.98|2958.68|3635.08|5379.66|55.03|1124.29|1721.40|1834.39|1889.42|3555.79|3610.82|-1800.69| +2451102|42109|2451109|7390|67026|579460|5847|15081|67026|579460|5847|15081|25|14|10|2|164|424|29|18.82|42.72|30.33|359.31|879.57|545.78|1238.88|61.56|0.00|619.44|879.57|941.13|1499.01|1560.57|333.79| +2451025|47044|2451054|9694|5544|931901|1105|47109|7330|1116131|2274|47078|22|26|6|3|50|425|58|42.82|49.24|4.43|2598.98|256.94|2483.56|2855.92|15.41|0.00|999.34|256.94|272.35|1256.28|1271.69|-2226.62| +2451025|47044|2451094|16078|5544|931901|1105|47109|7330|1116131|2274|47078|25|16|15|3|154|425|30|55.25|130.39|69.10|1838.70|2073.00|1657.50|3911.70|0.00|1181.61|1603.50|891.39|891.39|2494.89|2494.89|-766.11| +2451025|47044|2451141|433|5544|931901|1105|47109|7330|1116131|2274|47078|13|22|17|2|244|425|53|67.89|99.79|2.99|5130.40|158.47|3598.17|5288.87|3.16|0.00|369.94|158.47|161.63|528.41|531.57|-3439.70| +2451025|47044|2451049|15164|5544|931901|1105|47109|7330|1116131|2274|47078|25|8|7|2|244|425|70|88.32|153.67|15.36|9681.70|1075.20|6182.40|10756.90|86.01|0.00|1075.20|1075.20|1161.21|2150.40|2236.41|-5107.20| +2451025|47044|2451031|13514|5544|931901|1105|47109|7330|1116131|2274|47078|55|10|7|5|76|425|66|61.79|135.32|101.49|2232.78|6698.34|4078.14|8931.12|401.90|0.00|357.06|6698.34|7100.24|7055.40|7457.30|2620.20| +2451025|47044|2451045|12502|5544|931901|1105|47109|7330|1116131|2274|47078|49|8|18|3|171|425|90|41.59|49.07|23.55|2296.80|2119.50|3743.10|4416.30|127.17|0.00|1191.60|2119.50|2246.67|3311.10|3438.27|-1623.60| +2451025|47044|2451138|7177|5544|931901|1105|47109|7330|1116131|2274|47078|52|10|18|5|162|425|49|7.63|16.17|5.01|546.84|245.49|373.87|792.33|2.45|0.00|0.00|245.49|247.94|245.49|247.94|-128.38| +2451025|47044|2451078|11086|5544|931901|1105|47109|7330|1116131|2274|47078|22|19|1|4|177|425|2|5.61|15.20|11.40|7.60|22.80|11.22|30.40|0.68|0.00|13.36|22.80|23.48|36.16|36.84|11.58| +2451025|47044|2451066|8018|5544|931901|1105|47109|7330|1116131|2274|47078|34|10|11|4|247|425|44|78.63|172.19|51.65|5303.76|2272.60|3459.72|7576.36|78.17|1295.38|833.36|977.22|1055.39|1810.58|1888.75|-2482.50| +2451025|47044|2451116|4634|5544|931901|1105|47109|7330|1116131|2274|47078|25|2|4|3|45|425|59|9.19|17.00|12.58|260.78|742.22|542.21|1003.00|0.00|356.26|491.47|385.96|385.96|877.43|877.43|-156.25| +2451025|47044|2451027|16735|5544|931901|1105|47109|7330|1116131|2274|47078|34|22|16|3|107|425|27|91.53|257.19|190.32|1805.49|5138.64|2471.31|6944.13|113.05|2312.38|2013.66|2826.26|2939.31|4839.92|4952.97|354.95| +2451811|73474|2451920|13928|3937|1544012|2134|43456|43146|145103|5728|7050|8|7|20|1|22|426|11|92.00|134.32|88.65|502.37|975.15|1012.00|1477.52|2.34|936.14|251.13|39.01|41.35|290.14|292.48|-972.99| +2451811|73474|2451884|4763|3937|1544012|2134|43456|43146|145103|5728|7050|56|25|4|5|46|426|36|25.46|35.64|15.68|718.56|564.48|916.56|1283.04|9.59|372.55|76.68|191.93|201.52|268.61|278.20|-724.63| +2451811|73474|2451819|2305|3937|1544012|2134|43456|43146|145103|5728|7050|13|11|15|2|176|426|39|71.82|136.45|98.24|1490.19|3831.36|2800.98|5321.55|189.65|38.31|1117.35|3793.05|3982.70|4910.40|5100.05|992.07| +2451811|73474|2451857|16801|3937|1544012|2134|43456|43146|145103|5728|7050|44|14|13|3|203|426|23|75.31|161.91|79.33|1899.34|1824.59|1732.13|3723.93|6.20|1204.22|670.22|620.37|626.57|1290.59|1296.79|-1111.76| +2451811|73474|2451910|8315|3937|1544012|2134|43456|43146|145103|5728|7050|37|26|19|4|149|426|98|87.78|254.56|109.46|14219.80|10727.08|8602.44|24946.88|858.16|0.00|10227.28|10727.08|11585.24|20954.36|21812.52|2124.64| +2451811|73474|2451904|6833|3937|1544012|2134|43456|43146|145103|5728|7050|53|20|18|5|152|426|47|13.55|30.62|12.55|849.29|589.85|636.85|1439.14|12.56|171.05|71.91|418.80|431.36|490.71|503.27|-218.05| +2451811|73474|2451905|17527|3937|1544012|2134|43456|43146|145103|5728|7050|38|19|17|4|91|426|98|76.42|180.35|153.29|2651.88|15022.42|7489.16|17674.30|1051.56|0.00|1766.94|15022.42|16073.98|16789.36|17840.92|7533.26| +2451811|73474|2451891|9673|3937|1544012|2134|43456|43146|145103|5728|7050|23|8|8|4|97|426|20|26.07|74.82|74.82|0.00|1496.40|521.40|1496.40|134.67|0.00|628.40|1496.40|1631.07|2124.80|2259.47|975.00| +2451811|73474|2451891|15757|3937|1544012|2134|43456|43146|145103|5728|7050|37|5|20|4|158|426|39|40.03|43.63|16.14|1072.11|629.46|1561.17|1701.57|6.29|0.00|646.23|629.46|635.75|1275.69|1281.98|-931.71| +2451811|73474|2451851|16676|3937|1544012|2134|43456|43146|145103|5728|7050|17|14|20|4|268|426|100|20.05|37.89|34.85|304.00|3485.00|2005.00|3789.00|243.95|0.00|189.00|3485.00|3728.95|3674.00|3917.95|1480.00| +2452596|75775|2452631|7644|57733|1702148|3363|47331|97599|661277|5729|11344|51|6|3|3|3|427|55|25.18|63.70|14.65|2697.75|805.75|1384.90|3503.50|32.23|0.00|210.10|805.75|837.98|1015.85|1048.08|-579.15| +2452596|75775|2452680|10891|57733|1702148|3363|47331|97599|661277|5729|11344|33|9|13|1|149|427|47|87.99|199.73|9.98|8918.25|469.06|4135.53|9387.31|9.38|0.00|3379.30|469.06|478.44|3848.36|3857.74|-3666.47| +2452596|75775|2452698|14874|57733|1702148|3363|47331|97599|661277|5729|11344|25|21|19|4|102|427|82|17.51|46.57|37.25|764.24|3054.50|1435.82|3818.74|80.63|366.54|992.20|2687.96|2768.59|3680.16|3760.79|1252.14| +2452596|75775|2452625|2335|57733|1702148|3363|47331|97599|661277|5729|11344|1|1|14|4|174|427|33|87.04|157.54|78.77|2599.41|2599.41|2872.32|5198.82|201.71|77.98|0.00|2521.43|2723.14|2521.43|2723.14|-350.89| +2452596|75775|2452625|9405|57733|1702148|3363|47331|97599|661277|5729|11344|39|27|5|4|183|427|22|13.52|23.79|7.37|361.24|162.14|297.44|523.38|1.62|0.00|41.80|162.14|163.76|203.94|205.56|-135.30| +2452596|75775|2452695|1890|57733|1702148|3363|47331|97599|661277|5729|11344|45|3|1|5|66|427|84|74.20|121.68|113.16|715.68|9505.44|6232.80|10221.12|95.05|0.00|919.80|9505.44|9600.49|10425.24|10520.29|3272.64| +2452596|75775|2452664|15222|57733|1702148|3363|47331|97599|661277|5729|11344|1|1|14|4|234|427|38|14.12|20.61|10.09|399.76|383.42|536.56|783.18|29.14|19.17|344.28|364.25|393.39|708.53|737.67|-172.31| +2452596|75775|2452656|5955|57733|1702148|3363|47331|97599|661277|5729|11344|7|6|9|3|97|427|82|29.71|30.00|10.50|1599.00|861.00|2436.22|2460.00|60.27|0.00|861.00|861.00|921.27|1722.00|1782.27|-1575.22| +2452596|75775|2452618|17529|57733|1702148|3363|47331|97599|661277|5729|11344|48|24|3|1|142|427|87|95.28|161.02|90.17|6163.95|7844.79|8289.36|14008.74|549.13|0.00|2241.12|7844.79|8393.92|10085.91|10635.04|-444.57| +2452596|75775|2452704|9264|57733|1702148|3363|47331|97599|661277|5729|11344|55|15|8|3|8|427|76|85.69|155.95|146.59|711.36|11140.84|6512.44|11852.20|557.04|0.00|2251.88|11140.84|11697.88|13392.72|13949.76|4628.40| +2452596|75775|2452661|9505|57733|1702148|3363|47331|97599|661277|5729|11344|15|9|19|3|17|427|13|84.13|220.42|160.90|773.76|2091.70|1093.69|2865.46|0.00|0.00|1346.67|2091.70|2091.70|3438.37|3438.37|998.01| +2452596|75775|2452680|16572|57733|1702148|3363|47331|97599|661277|5729|11344|31|18|5|3|221|427|70|16.12|19.82|8.32|805.00|582.40|1128.40|1387.40|13.51|413.50|429.80|168.90|182.41|598.70|612.21|-959.50| +2452596|75775|2452609|1839|57733|1702148|3363|47331|97599|661277|5729|11344|18|12|20|4|288|427|62|39.03|84.69|48.27|2258.04|2992.74|2419.86|5250.78|119.70|0.00|1417.32|2992.74|3112.44|4410.06|4529.76|572.88| +2452596|75775|2452626|4110|57733|1702148|3363|47331|97599|661277|5729|11344|42|30|6|5|274|427|87|66.89|105.01|91.35|1188.42|7947.45|5819.43|9135.87|0.00|0.00|2192.40|7947.45|7947.45|10139.85|10139.85|2128.02| +2451960|71613|2451963|15765|68965|471679|4440|13201|53419|1897429|1084|17045|1|19|6|5|231|428|24|62.20|182.86|122.51|1448.40|2940.24|1492.80|4388.64|205.81|0.00|482.64|2940.24|3146.05|3422.88|3628.69|1447.44| +2451960|71613|2451989|14379|68965|471679|4440|13201|53419|1897429|1084|17045|59|21|5|4|30|428|94|31.59|64.75|55.68|852.58|5233.92|2969.46|6086.50|86.88|889.76|2799.32|4344.16|4431.04|7143.48|7230.36|1374.70| +2451960|71613|2452032|4393|68965|471679|4440|13201|53419|1897429|1084|17045|21|17|4|4|77|428|58|62.54|143.84|70.48|4254.88|4087.84|3627.32|8342.72|81.75|0.00|1251.06|4087.84|4169.59|5338.90|5420.65|460.52| +2451960|71613|2452037|13013|68965|471679|4440|13201|53419|1897429|1084|17045|51|27|12|5|91|428|48|23.18|33.37|30.03|160.32|1441.44|1112.64|1601.76|68.61|461.26|0.00|980.18|1048.79|980.18|1048.79|-132.46| +2451960|71613|2451997|15585|68965|471679|4440|13201|53419|1897429|1084|17045|1|27|7|5|60|428|47|92.68|125.11|77.56|2234.85|3645.32|4355.96|5880.17|109.35|0.00|1940.16|3645.32|3754.67|5585.48|5694.83|-710.64| +2451960|71613|2452026|16443|68965|471679|4440|13201|53419|1897429|1084|17045|19|23|4|3|265|428|86|17.35|34.00|10.54|2017.56|906.44|1492.10|2924.00|45.32|0.00|497.08|906.44|951.76|1403.52|1448.84|-585.66| +2451960|71613|2452029|17965|68965|471679|4440|13201|53419|1897429|1084|17045|21|21|1|4|184|428|70|39.41|100.10|22.02|5465.60|1541.40|2758.70|7007.00|0.00|0.00|280.00|1541.40|1541.40|1821.40|1821.40|-1217.30| +2451960|71613|2452025|4557|68965|471679|4440|13201|53419|1897429|1084|17045|13|23|14|4|251|428|15|2.40|5.25|2.20|45.75|33.00|36.00|78.75|2.31|0.00|38.55|33.00|35.31|71.55|73.86|-3.00| +2451960|71613|2452006|3185|68965|471679|4440|13201|53419|1897429|1084|17045|5|3|4|1|8|428|38|41.61|123.99|13.63|4193.68|517.94|1581.18|4711.62|46.61|0.00|93.86|517.94|564.55|611.80|658.41|-1063.24| +2451960|71613|2452005|8627|68965|471679|4440|13201|53419|1897429|1084|17045|35|5|15|2|249|428|8|88.23|172.93|107.21|525.76|857.68|705.84|1383.44|51.46|0.00|318.16|857.68|909.14|1175.84|1227.30|151.84| +2451960|71613|2452029|1687|68965|471679|4440|13201|53419|1897429|1084|17045|35|27|17|5|136|428|66|78.39|220.27|88.10|8723.22|5814.60|5173.74|14537.82|288.40|2209.54|6541.92|3605.06|3893.46|10146.98|10435.38|-1568.68| +2451960|71613|2451970|881|68965|471679|4440|13201|53419|1897429|1084|17045|15|29|5|5|82|428|54|23.80|54.26|29.84|1318.68|1611.36|1285.20|2930.04|7.73|1353.54|292.68|257.82|265.55|550.50|558.23|-1027.38| +2451960|71613|2452053|12293|68965|471679|4440|13201|53419|1897429|1084|17045|25|9|17|4|281|428|3|79.77|98.11|75.54|67.71|226.62|239.31|294.33|2.26|0.00|41.19|226.62|228.88|267.81|270.07|-12.69| +2451960|71613|2452055|4049|68965|471679|4440|13201|53419|1897429|1084|17045|45|15|6|1|69|428|79|72.53|77.60|37.24|3188.44|2941.96|5729.87|6130.40|88.25|0.00|367.35|2941.96|3030.21|3309.31|3397.56|-2787.91| +2451839|62951|2451918|8489|93320|1816602|6637|47919|93320|1816602|6637|47919|55|13|19|3|207|429|93|93.36|205.39|61.61|13371.54|5729.73|8682.48|19101.27|16.04|5328.64|1527.99|401.09|417.13|1929.08|1945.12|-8281.39| +2451839|62951|2451929|7796|93320|1816602|6637|47919|93320|1816602|6637|47919|41|29|19|5|184|429|55|84.08|119.39|37.01|4530.90|2035.55|4624.40|6566.45|52.11|732.79|1247.40|1302.76|1354.87|2550.16|2602.27|-3321.64| +2451839|62951|2451893|10849|93320|1816602|6637|47919|93320|1816602|6637|47919|50|13|15|3|294|429|49|69.43|101.36|32.43|3377.57|1589.07|3402.07|4966.64|95.34|0.00|2135.42|1589.07|1684.41|3724.49|3819.83|-1813.00| +2451839|62951|2451910|1099|93320|1816602|6637|47919|93320|1816602|6637|47919|32|17|18|2|179|429|12|92.74|140.96|39.46|1218.00|473.52|1112.88|1691.52|28.41|0.00|355.20|473.52|501.93|828.72|857.13|-639.36| +2451839|62951|2451928|10675|93320|1816602|6637|47919|93320|1816602|6637|47919|20|11|14|4|254|429|30|75.53|114.80|36.73|2342.10|1101.90|2265.90|3444.00|5.28|837.44|275.40|264.46|269.74|539.86|545.14|-2001.44| +2451839|62951|2451878|4202|93320|1816602|6637|47919|93320|1816602|6637|47919|56|26|2|5|218|429|5|23.98|59.95|35.37|122.90|176.85|119.90|299.75|5.30|0.00|125.85|176.85|182.15|302.70|308.00|56.95| +2451839|62951|2451927|2111|93320|1816602|6637|47919|93320|1816602|6637|47919|19|13|1|3|12|429|95|80.44|129.50|116.55|1230.25|11072.25|7641.80|12302.50|332.16|0.00|1352.80|11072.25|11404.41|12425.05|12757.21|3430.45| +2451839|62951|2451852|15191|93320|1816602|6637|47919|93320|1816602|6637|47919|8|1|17|1|132|429|29|23.02|67.67|21.65|1334.58|627.85|667.58|1962.43|9.04|514.83|529.83|113.02|122.06|642.85|651.89|-554.56| +2452611|71114|2452629|6123|53018|504201|4631|9014|53018|504201|4631|9014|6|1|16|4|189|430|15|41.99|95.31|78.15|257.40|1172.25|629.85|1429.65|105.50|0.00|686.10|1172.25|1277.75|1858.35|1963.85|542.40| +2452611|71114|2452674|14517|53018|504201|4631|9014|53018|504201|4631|9014|36|27|20|4|115|430|82|33.61|100.49|69.33|2555.12|5685.06|2756.02|8240.18|99.48|3695.28|2801.12|1989.78|2089.26|4790.90|4890.38|-766.24| +2452611|71114|2452680|15168|53018|504201|4631|9014|53018|504201|4631|9014|45|3|11|2|6|430|84|99.24|143.89|133.81|846.72|11240.04|8336.16|12086.76|674.40|0.00|1933.68|11240.04|11914.44|13173.72|13848.12|2903.88| +2452611|71114|2452674|16159|53018|504201|4631|9014|53018|504201|4631|9014|15|30|5|2|142|430|74|77.16|105.70|31.71|5475.26|2346.54|5709.84|7821.80|23.46|0.00|77.70|2346.54|2370.00|2424.24|2447.70|-3363.30| +2452611|71114|2452617|10926|53018|504201|4631|9014|53018|504201|4631|9014|27|6|9|3|284|430|56|64.53|132.93|86.40|2605.68|4838.40|3613.68|7444.08|338.68|0.00|297.36|4838.40|5177.08|5135.76|5474.44|1224.72| +2452611|71114|2452713|11712|53018|504201|4631|9014|53018|504201|4631|9014|18|21|20|2|234|430|13|64.79|147.07|16.17|1701.70|210.21|842.27|1911.91|4.79|90.39|707.33|119.82|124.61|827.15|831.94|-722.45| +2452611|71114|2452731|6855|53018|504201|4631|9014|53018|504201|4631|9014|1|15|5|4|36|430|31|45.41|110.34|77.23|1026.41|2394.13|1407.71|3420.54|47.88|0.00|307.83|2394.13|2442.01|2701.96|2749.84|986.42| +2452611|71114|2452658|9667|53018|504201|4631|9014|53018|504201|4631|9014|37|7|19|2|102|430|32|52.02|150.85|138.78|386.24|4440.96|1664.64|4827.20|186.52|1332.28|1592.96|3108.68|3295.20|4701.64|4888.16|1444.04| +2452611|71114|2452612|13497|53018|504201|4631|9014|53018|504201|4631|9014|30|24|19|1|250|430|10|7.76|22.03|3.74|182.90|37.40|77.60|220.30|1.12|0.00|28.60|37.40|38.52|66.00|67.12|-40.20| +2452611|71114|2452666|7549|53018|504201|4631|9014|53018|504201|4631|9014|42|6|2|1|30|430|7|92.91|110.56|25.42|595.98|177.94|650.37|773.92|0.58|158.36|23.17|19.58|20.16|42.75|43.33|-630.79| +2452611|71114|2452677|13785|53018|504201|4631|9014|53018|504201|4631|9014|54|25|2|1|167|430|50|23.81|49.28|43.85|271.50|2192.50|1190.50|2464.00|4.38|2104.80|295.50|87.70|92.08|383.20|387.58|-1102.80| +2452611|71114|2452690|9771|53018|504201|4631|9014|53018|504201|4631|9014|49|6|1|2|140|430|72|23.59|63.69|27.38|2614.32|1971.36|1698.48|4585.68|137.99|0.00|320.40|1971.36|2109.35|2291.76|2429.75|272.88| +2452611|71114|2452659|9717|53018|504201|4631|9014|53018|504201|4631|9014|18|30|1|4|220|430|61|56.47|115.19|4.60|6745.99|280.60|3444.67|7026.59|5.61|0.00|3021.33|280.60|286.21|3301.93|3307.54|-3164.07| +2452352|75893|2452366|10327|46667|985856|1398|32987|45515|1300382|5707|9296|60|30|20|5|180|431|81|31.23|80.26|56.98|1885.68|4615.38|2529.63|6501.06|230.76|0.00|714.42|4615.38|4846.14|5329.80|5560.56|2085.75| +2452352|75893|2452436|11787|46667|985856|1398|32987|45515|1300382|5707|9296|27|7|4|4|57|431|99|95.07|195.84|3.91|19001.07|387.09|9411.93|19388.16|27.09|0.00|3876.84|387.09|414.18|4263.93|4291.02|-9024.84| +2452352|75893|2452453|17665|46667|985856|1398|32987|45515|1300382|5707|9296|49|15|14|5|93|431|14|45.54|118.40|84.06|480.76|1176.84|637.56|1657.60|35.30|0.00|381.22|1176.84|1212.14|1558.06|1593.36|539.28| +2452352|75893|2452370|17427|46667|985856|1398|32987|45515|1300382|5707|9296|7|1|8|3|75|431|69|55.05|62.20|0.62|4249.02|42.78|3798.45|4291.80|0.42|0.00|1845.06|42.78|43.20|1887.84|1888.26|-3755.67| +2452352|75893|2452428|17823|46667|985856|1398|32987|45515|1300382|5707|9296|48|21|11|3|48|431|74|99.71|128.62|61.73|4949.86|4568.02|7378.54|9517.88|91.36|0.00|3426.20|4568.02|4659.38|7994.22|8085.58|-2810.52| +2452352|75893|2452459|259|46667|985856|1398|32987|45515|1300382|5707|9296|39|15|13|3|143|431|39|36.30|48.64|35.02|531.18|1365.78|1415.70|1896.96|68.28|0.00|796.38|1365.78|1434.06|2162.16|2230.44|-49.92| +2452352|75893|2452452|5451|46667|985856|1398|32987|45515|1300382|5707|9296|12|1|19|4|99|431|68|24.51|72.30|15.18|3884.16|1032.24|1666.68|4916.40|92.90|0.00|196.52|1032.24|1125.14|1228.76|1321.66|-634.44| +2452352|75893|2452392|2946|46667|985856|1398|32987|45515|1300382|5707|9296|1|1|1|3|38|431|45|58.77|139.28|36.21|4638.15|1629.45|2644.65|6267.60|114.06|0.00|1002.60|1629.45|1743.51|2632.05|2746.11|-1015.20| +2452352|75893|2452413|16110|46667|985856|1398|32987|45515|1300382|5707|9296|15|9|19|1|33|431|21|82.14|168.38|156.59|247.59|3288.39|1724.94|3535.98|65.76|0.00|247.38|3288.39|3354.15|3535.77|3601.53|1563.45| +2452352|75893|2452355|11859|46667|985856|1398|32987|45515|1300382|5707|9296|33|9|13|1|179|431|25|87.84|218.72|98.42|3007.50|2460.50|2196.00|5468.00|123.02|0.00|2351.00|2460.50|2583.52|4811.50|4934.52|264.50| +2452352|75893|2452456|4656|46667|985856|1398|32987|45515|1300382|5707|9296|57|25|10|4|131|431|57|79.75|184.22|62.63|6930.63|3569.91|4545.75|10500.54|35.69|0.00|419.52|3569.91|3605.60|3989.43|4025.12|-975.84| +2452361|52326|2452398|13471|82287|213960|6786|18812|53433|1117212|6184|6610|6|6|20|2|234|432|63|76.15|76.91|49.22|1744.47|3100.86|4797.45|4845.33|248.06|0.00|532.98|3100.86|3348.92|3633.84|3881.90|-1696.59| +2452361|52326|2452407|9033|82287|213960|6786|18812|53433|1117212|6184|6610|6|9|20|5|200|432|76|37.16|79.89|44.73|2672.16|3399.48|2824.16|6071.64|169.97|0.00|1578.52|3399.48|3569.45|4978.00|5147.97|575.32| +2452361|52326|2452366|8251|82287|213960|6786|18812|53433|1117212|6184|6610|55|13|6|5|237|432|75|93.93|133.38|46.68|6502.50|3501.00|7044.75|10003.50|138.63|1190.34|4701.00|2310.66|2449.29|7011.66|7150.29|-4734.09| +2452361|52326|2452428|15912|82287|213960|6786|18812|53433|1117212|6184|6610|60|30|20|5|210|432|38|57.39|95.84|66.12|1129.36|2512.56|2180.82|3641.92|75.37|0.00|946.58|2512.56|2587.93|3459.14|3534.51|331.74| +2452361|52326|2452459|2239|82287|213960|6786|18812|53433|1117212|6184|6610|27|7|4|4|94|432|28|3.41|7.77|2.48|148.12|69.44|95.48|217.56|1.38|0.00|49.84|69.44|70.82|119.28|120.66|-26.04| +2452361|52326|2452387|8940|82287|213960|6786|18812|53433|1117212|6184|6610|48|13|13|1|212|432|11|60.64|86.10|17.22|757.68|189.42|667.04|947.10|11.36|0.00|378.84|189.42|200.78|568.26|579.62|-477.62| +2452361|52326|2452408|9261|82287|213960|6786|18812|53433|1117212|6184|6610|12|12|3|3|200|432|7|29.40|62.91|52.84|70.49|369.88|205.80|440.37|18.64|59.18|61.60|310.70|329.34|372.30|390.94|104.90| +2452361|52326|2452467|4350|82287|213960|6786|18812|53433|1117212|6184|6610|6|7|5|1|37|432|54|73.87|152.17|152.17|0.00|8217.18|3988.98|8217.18|443.72|3286.87|903.42|4930.31|5374.03|5833.73|6277.45|941.33| +2452361|52326|2452478|17325|82287|213960|6786|18812|53433|1117212|6184|6610|18|24|2|5|35|432|95|16.80|27.72|1.94|2449.10|184.30|1596.00|2633.40|1.84|0.00|473.10|184.30|186.14|657.40|659.24|-1411.70| +2452361|52326|2452442|17727|82287|213960|6786|18812|53433|1117212|6184|6610|57|18|16|1|173|432|89|63.28|108.20|33.54|6644.74|2985.06|5631.92|9629.80|28.95|89.55|3273.42|2895.51|2924.46|6168.93|6197.88|-2736.41| +2452361|52326|2452412|11367|82287|213960|6786|18812|53433|1117212|6184|6610|6|19|20|2|24|432|84|87.26|184.11|22.09|13609.68|1855.56|7329.84|15465.24|74.22|0.00|5876.64|1855.56|1929.78|7732.20|7806.42|-5474.28| +2452361|52326|2452395|12330|82287|213960|6786|18812|53433|1117212|6184|6610|39|21|12|3|134|432|26|65.98|73.89|19.21|1421.68|499.46|1715.48|1921.14|44.95|0.00|749.06|499.46|544.41|1248.52|1293.47|-1216.02| +2452361|52326|2452415|1639|82287|213960|6786|18812|53433|1117212|6184|6610|3|30|5|3|128|432|82|69.30|161.46|121.09|3310.34|9929.38|5682.60|13239.72|198.58|0.00|6222.16|9929.38|10127.96|16151.54|16350.12|4246.78| +2452361|52326|2452382|10419|82287|213960|6786|18812|53433|1117212|6184|6610|37|13|5|5|128|432|65|66.73|100.76|56.42|2882.10|3667.30|4337.45|6549.40|36.67|0.00|3274.70|3667.30|3703.97|6942.00|6978.67|-670.15| +2452361|52326|2452364|17497|82287|213960|6786|18812|53433|1117212|6184|6610|45|19|20|2|114|432|84|29.48|36.85|25.42|960.12|2135.28|2476.32|3095.40|149.46|0.00|959.28|2135.28|2284.74|3094.56|3244.02|-341.04| +2452361|52326|2452455|16479|82287|213960|6786|18812|53433|1117212|6184|6610|19|24|4|1|78|432|21|99.74|145.62|81.54|1345.68|1712.34|2094.54|3058.02|118.66|17.12|1253.70|1695.22|1813.88|2948.92|3067.58|-399.32| +2452240|36342|2452246|883|97633|1460984|5135|12886|75255|1844963|2092|16243|49|9|5|3|88|433|69|5.02|13.40|10.85|175.95|748.65|346.38|924.60|0.59|688.75|406.41|59.90|60.49|466.31|466.90|-286.48| +2452240|36342|2452264|16983|97633|1460984|5135|12886|75255|1844963|2092|16243|1|25|15|2|79|433|32|65.40|74.55|53.67|668.16|1717.44|2092.80|2385.60|51.52|0.00|1144.96|1717.44|1768.96|2862.40|2913.92|-375.36| +2452240|36342|2452352|3209|97633|1460984|5135|12886|75255|1844963|2092|16243|29|23|9|1|33|433|87|21.07|27.60|5.79|1897.47|503.73|1833.09|2401.20|20.14|0.00|47.85|503.73|523.87|551.58|571.72|-1329.36| +2452240|36342|2452249|14835|97633|1460984|5135|12886|75255|1844963|2092|16243|25|13|14|1|88|433|17|51.44|141.97|8.51|2268.82|144.67|874.48|2413.49|0.10|143.22|506.77|1.45|1.55|508.22|508.32|-873.03| +2452240|36342|2452341|3969|97633|1460984|5135|12886|75255|1844963|2092|16243|37|15|17|3|252|433|60|44.80|94.52|23.63|4253.40|1417.80|2688.00|5671.20|85.06|0.00|793.80|1417.80|1502.86|2211.60|2296.66|-1270.20| +2452240|36342|2452333|10519|97633|1460984|5135|12886|75255|1844963|2092|16243|5|27|2|5|68|433|97|86.58|198.26|63.44|13077.54|6153.68|8398.26|19231.22|0.00|0.00|9615.61|6153.68|6153.68|15769.29|15769.29|-2244.58| +2452240|36342|2452268|8189|97633|1460984|5135|12886|75255|1844963|2092|16243|11|27|15|3|234|433|5|37.07|46.33|35.67|53.30|178.35|185.35|231.65|10.70|0.00|20.80|178.35|189.05|199.15|209.85|-7.00| +2452240|36342|2452243|3597|97633|1460984|5135|12886|75255|1844963|2092|16243|9|9|15|3|58|433|98|69.33|181.64|163.47|1780.66|16020.06|6794.34|17800.72|1281.60|0.00|3025.26|16020.06|17301.66|19045.32|20326.92|9225.72| +2452240|36342|2452325|5773|97633|1460984|5135|12886|75255|1844963|2092|16243|1|23|3|4|68|433|25|40.50|116.64|72.31|1108.25|1807.75|1012.50|2916.00|72.31|0.00|145.75|1807.75|1880.06|1953.50|2025.81|795.25| +2452240|36342|2452348|8049|97633|1460984|5135|12886|75255|1844963|2092|16243|39|21|12|1|277|433|89|90.43|242.35|239.92|216.27|21352.88|8048.27|21569.15|1067.64|0.00|1724.82|21352.88|22420.52|23077.70|24145.34|13304.61| +2452240|36342|2452342|14611|97633|1460984|5135|12886|75255|1844963|2092|16243|47|5|19|3|104|433|28|91.45|189.30|85.18|2915.36|2385.04|2560.60|5300.40|166.95|0.00|158.76|2385.04|2551.99|2543.80|2710.75|-175.56| +2452581|74628|2452630|13275|61237|1155537|4684|45031|60079|1342385|3225|17972|19|25|6|1|209|434|34|5.24|10.79|3.23|257.04|109.82|178.16|366.86|0.00|0.00|183.26|109.82|109.82|293.08|293.08|-68.34| +2452581|74628|2452618|8409|61237|1155537|4684|45031|60079|1342385|3225|17972|55|9|5|5|102|434|35|98.53|128.08|111.42|583.10|3899.70|3448.55|4482.80|194.98|0.00|1165.50|3899.70|4094.68|5065.20|5260.18|451.15| +2452581|74628|2452698|1968|61237|1155537|4684|45031|60079|1342385|3225|17972|45|30|7|4|231|434|45|31.76|36.84|25.05|530.55|1127.25|1429.20|1657.80|90.18|0.00|66.15|1127.25|1217.43|1193.40|1283.58|-301.95| +2452581|74628|2452685|15747|61237|1155537|4684|45031|60079|1342385|3225|17972|3|25|11|2|203|434|30|30.74|61.17|3.05|1743.60|91.50|922.20|1835.10|5.49|0.00|165.00|91.50|96.99|256.50|261.99|-830.70| +2452581|74628|2452693|7731|61237|1155537|4684|45031|60079|1342385|3225|17972|18|25|7|3|2|434|24|92.99|132.04|40.93|2186.64|982.32|2231.76|3168.96|46.36|402.75|253.44|579.57|625.93|833.01|879.37|-1652.19| +2452581|74628|2452591|9516|61237|1155537|4684|45031|60079|1342385|3225|17972|60|3|14|1|271|434|99|63.38|154.64|47.93|10564.29|4745.07|6274.62|15309.36|0.00|2847.04|7195.32|1898.03|1898.03|9093.35|9093.35|-4376.59| +2452581|74628|2452645|15109|61237|1155537|4684|45031|60079|1342385|3225|17972|43|6|4|2|253|434|67|82.79|173.03|13.84|10665.73|927.28|5546.93|11593.01|18.54|0.00|5332.53|927.28|945.82|6259.81|6278.35|-4619.65| +2452581|74628|2452659|11454|61237|1155537|4684|45031|60079|1342385|3225|17972|43|30|8|1|241|434|91|50.80|102.10|89.84|1115.66|8175.44|4622.80|9291.10|408.77|0.00|4645.55|8175.44|8584.21|12820.99|13229.76|3552.64| +2452581|74628|2452689|14028|61237|1155537|4684|45031|60079|1342385|3225|17972|15|19|12|4|196|434|27|73.34|211.95|38.15|4692.60|1030.05|1980.18|5722.65|20.60|0.00|2002.86|1030.05|1050.65|3032.91|3053.51|-950.13| +2452306|71015|2452375|17028|34129|518075|2021|33214|57442|308540|3499|37445|57|25|17|1|85|435|51|20.99|21.61|18.15|176.46|925.65|1070.49|1102.11|35.63|212.89|110.16|712.76|748.39|822.92|858.55|-357.73| +2452306|71015|2452394|8832|34129|518075|2021|33214|57442|308540|3499|37445|60|9|17|2|214|435|87|87.15|124.62|36.13|7698.63|3143.31|7582.05|10841.94|188.59|0.00|433.26|3143.31|3331.90|3576.57|3765.16|-4438.74| +2452306|71015|2452375|2019|34129|518075|2021|33214|57442|308540|3499|37445|42|7|10|3|179|435|32|54.67|143.78|31.63|3588.80|1012.16|1749.44|4600.96|8.09|202.43|1886.08|809.73|817.82|2695.81|2703.90|-939.71| +2452306|71015|2452329|14844|34129|518075|2021|33214|57442|308540|3499|37445|24|1|10|2|291|435|91|62.32|105.94|50.85|5013.19|4627.35|5671.12|9640.54|323.91|0.00|4820.27|4627.35|4951.26|9447.62|9771.53|-1043.77| +2452306|71015|2452365|9331|34129|518075|2021|33214|57442|308540|3499|37445|36|7|3|4|45|435|32|34.61|42.57|33.63|286.08|1076.16|1107.52|1362.24|10.76|0.00|517.44|1076.16|1086.92|1593.60|1604.36|-31.36| +2452306|71015|2452338|1593|34129|518075|2021|33214|57442|308540|3499|37445|55|13|3|4|115|435|54|67.95|203.17|107.68|5156.46|5814.72|3669.30|10971.18|0.00|0.00|5485.32|5814.72|5814.72|11300.04|11300.04|2145.42| +2452306|71015|2452380|2731|34129|518075|2021|33214|57442|308540|3499|37445|36|9|19|1|183|435|68|49.99|142.47|71.23|4844.32|4843.64|3399.32|9687.96|339.05|0.00|2324.92|4843.64|5182.69|7168.56|7507.61|1444.32| +2452306|71015|2452318|4041|34129|518075|2021|33214|57442|308540|3499|37445|25|21|9|4|252|435|19|98.46|170.33|155.00|291.27|2945.00|1870.74|3236.27|106.02|1178.00|809.02|1767.00|1873.02|2576.02|2682.04|-103.74| +2452306|71015|2452339|1608|34129|518075|2021|33214|57442|308540|3499|37445|21|18|4|2|137|435|22|85.54|205.29|203.23|45.32|4471.06|1881.88|4516.38|0.00|0.00|361.24|4471.06|4471.06|4832.30|4832.30|2589.18| +2452306|71015|2452377|5595|34129|518075|2021|33214|57442|308540|3499|37445|43|18|3|5|277|435|1|38.17|100.76|41.31|59.45|41.31|38.17|100.76|1.23|0.00|48.36|41.31|42.54|89.67|90.90|3.14| +2452306|71015|2452421|5311|34129|518075|2021|33214|57442|308540|3499|37445|31|25|7|4|101|435|56|82.49|161.68|77.60|4708.48|4345.60|4619.44|9054.08|173.82|0.00|1901.20|4345.60|4519.42|6246.80|6420.62|-273.84| +2452306|71015|2452357|6138|34129|518075|2021|33214|57442|308540|3499|37445|31|24|5|1|164|435|48|74.00|146.52|99.63|2250.72|4782.24|3552.00|7032.96|239.11|0.00|351.36|4782.24|5021.35|5133.60|5372.71|1230.24| +2452306|71015|2452310|10446|34129|518075|2021|33214|57442|308540|3499|37445|6|9|7|2|244|435|76|90.71|207.72|120.47|6631.00|9155.72|6893.96|15786.72|457.78|0.00|5209.04|9155.72|9613.50|14364.76|14822.54|2261.76| +2452306|71015|2452377|955|34129|518075|2021|33214|57442|308540|3499|37445|3|30|11|2|213|435|86|38.98|89.65|83.37|540.08|7169.82|3352.28|7709.90|286.79|0.00|847.96|7169.82|7456.61|8017.78|8304.57|3817.54| +2452306|71015|2452319|6759|34129|518075|2021|33214|57442|308540|3499|37445|43|7|9|1|3|435|31|16.27|30.91|29.36|48.05|910.16|504.37|958.21|18.20|0.00|478.95|910.16|928.36|1389.11|1407.31|405.79| +2452306|71015|2452329|1920|34129|518075|2021|33214|57442|308540|3499|37445|60|30|10|2|247|435|22|1.70|4.59|3.85|16.28|84.70|37.40|100.98|3.38|0.00|30.14|84.70|88.08|114.84|118.22|47.30| +2451882|66921|2451979|4280|92183|301234|332|12583|94351|1382962|6583|45234|56|19|1|5|276|436|53|79.43|137.41|49.46|4661.35|2621.38|4209.79|7282.73|183.49|0.00|2985.49|2621.38|2804.87|5606.87|5790.36|-1588.41| +2451882|66921|2451931|7985|92183|301234|332|12583|94351|1382962|6583|45234|47|17|11|1|188|436|28|5.35|9.14|2.01|199.64|56.28|149.80|255.92|5.06|0.00|94.64|56.28|61.34|150.92|155.98|-93.52| +2451882|66921|2451990|13952|92183|301234|332|12583|94351|1382962|6583|45234|29|26|3|2|161|436|59|22.77|43.03|20.65|1320.42|1218.35|1343.43|2538.77|32.53|134.01|177.59|1084.34|1116.87|1261.93|1294.46|-259.09| +2451882|66921|2451966|9755|92183|301234|332|12583|94351|1382962|6583|45234|50|7|9|2|42|436|24|67.94|185.47|179.90|133.68|4317.60|1630.56|4451.28|43.17|0.00|934.56|4317.60|4360.77|5252.16|5295.33|2687.04| +2451882|66921|2451994|5971|92183|301234|332|12583|94351|1382962|6583|45234|49|20|17|3|59|436|11|3.53|8.71|0.08|94.93|0.88|38.83|95.81|0.06|0.00|0.00|0.88|0.94|0.88|0.94|-37.95| +2451882|66921|2451888|473|92183|301234|332|12583|94351|1382962|6583|45234|37|23|15|3|81|436|13|14.38|32.06|28.53|45.89|370.89|186.94|416.78|33.38|0.00|12.48|370.89|404.27|383.37|416.75|183.95| +2451882|66921|2451943|15362|92183|301234|332|12583|94351|1382962|6583|45234|38|17|8|5|149|436|55|83.90|229.88|89.65|7712.65|4930.75|4614.50|12643.40|394.46|0.00|884.95|4930.75|5325.21|5815.70|6210.16|316.25| +2451882|66921|2451996|6266|92183|301234|332|12583|94351|1382962|6583|45234|20|5|5|2|69|436|15|54.07|107.59|24.74|1242.75|371.10|811.05|1613.85|25.97|0.00|580.95|371.10|397.07|952.05|978.02|-439.95| +2451882|66921|2451901|775|92183|301234|332|12583|94351|1382962|6583|45234|38|23|11|3|175|436|39|87.84|215.20|150.64|2517.84|5874.96|3425.76|8392.80|352.49|0.00|1006.98|5874.96|6227.45|6881.94|7234.43|2449.20| +2451882|66921|2451884|4375|92183|301234|332|12583|94351|1382962|6583|45234|55|1|1|1|151|436|83|13.57|14.24|14.24|0.00|1181.92|1126.31|1181.92|82.73|0.00|46.48|1181.92|1264.65|1228.40|1311.13|55.61| +2451882|66921|2451905|3571|92183|301234|332|12583|94351|1382962|6583|45234|56|19|1|5|6|436|28|56.88|107.50|67.72|1113.84|1896.16|1592.64|3010.00|94.80|0.00|1204.00|1896.16|1990.96|3100.16|3194.96|303.52| +2451446|12387|2451550|6370|2994|977733|1272|30011|2994|977733|1272|30011|14|10|14|2|247|437|42|10.95|13.46|8.34|215.04|350.28|459.90|565.32|28.02|0.00|259.98|350.28|378.30|610.26|638.28|-109.62| +2451446|12387|2451478|10082|2994|977733|1272|30011|2994|977733|1272|30011|26|4|5|4|23|437|79|71.55|208.92|165.04|3466.52|13038.16|5652.45|16504.68|1173.43|0.00|1485.20|13038.16|14211.59|14523.36|15696.79|7385.71| +2451446|12387|2451525|7492|2994|977733|1272|30011|2994|977733|1272|30011|7|16|4|2|287|437|10|83.23|108.19|34.62|735.70|346.20|832.30|1081.90|24.23|0.00|357.00|346.20|370.43|703.20|727.43|-486.10| +2451446|12387|2451542|3344|2994|977733|1272|30011|2994|977733|1272|30011|4|26|15|3|54|437|73|11.22|24.90|11.95|945.35|872.35|819.06|1817.70|34.89|0.00|108.77|872.35|907.24|981.12|1016.01|53.29| +2451446|12387|2451489|3871|2994|977733|1272|30011|2994|977733|1272|30011|2|20|7|3|202|437|53|28.55|65.37|54.25|589.36|2875.25|1513.15|3464.61|201.26|0.00|449.97|2875.25|3076.51|3325.22|3526.48|1362.10| +2451446|12387|2451516|3307|2994|977733|1272|30011|2994|977733|1272|30011|20|7|10|1|239|437|40|16.36|27.32|0.27|1082.00|10.80|654.40|1092.80|0.64|0.00|404.00|10.80|11.44|414.80|415.44|-643.60| +2451446|12387|2451548|10544|2994|977733|1272|30011|2994|977733|1272|30011|25|4|6|4|229|437|41|93.85|115.43|60.02|2271.81|2460.82|3847.85|4732.63|24.60|0.00|331.28|2460.82|2485.42|2792.10|2816.70|-1387.03| +2451446|12387|2451471|11312|2994|977733|1272|30011|2994|977733|1272|30011|14|19|2|4|16|437|25|1.63|4.23|1.69|63.50|42.25|40.75|105.75|2.53|0.00|51.75|42.25|44.78|94.00|96.53|1.50| +2451446|12387|2451541|1120|2994|977733|1272|30011|2994|977733|1272|30011|31|19|7|4|158|437|50|95.33|183.98|110.38|3680.00|5519.00|4766.50|9199.00|331.14|0.00|2943.50|5519.00|5850.14|8462.50|8793.64|752.50| +2451446|12387|2451483|9052|2994|977733|1272|30011|2994|977733|1272|30011|25|8|4|3|152|437|100|7.41|11.41|3.19|822.00|319.00|741.00|1141.00|3.19|0.00|570.00|319.00|322.19|889.00|892.19|-422.00| +2452140|65188|2452235|3983|79055|2684|1513|42742|95349|1645738|4829|27658|53|21|15|2|286|438|8|30.46|42.03|6.30|285.84|50.40|243.68|336.24|4.03|0.00|20.16|50.40|54.43|70.56|74.59|-193.28| +2452140|65188|2452176|4873|79055|2684|1513|42742|95349|1645738|4829|27658|43|13|15|5|105|438|13|41.31|60.72|10.32|655.20|134.16|537.03|789.36|1.34|0.00|228.80|134.16|135.50|362.96|364.30|-402.87| +2452140|65188|2452230|9647|79055|2684|1513|42742|95349|1645738|4829|27658|57|5|14|5|299|438|60|10.03|16.95|16.61|20.40|996.60|601.80|1017.00|59.79|0.00|9.60|996.60|1056.39|1006.20|1065.99|394.80| +2452140|65188|2452208|6217|79055|2684|1513|42742|95349|1645738|4829|27658|53|7|11|1|288|438|28|31.16|69.48|39.60|836.64|1108.80|872.48|1945.44|66.52|0.00|933.80|1108.80|1175.32|2042.60|2109.12|236.32| +2452140|65188|2452255|10107|79055|2684|1513|42742|95349|1645738|4829|27658|41|5|2|1|203|438|55|24.50|38.46|37.30|63.80|2051.50|1347.50|2115.30|164.12|0.00|190.30|2051.50|2215.62|2241.80|2405.92|704.00| +2452140|65188|2452242|6031|79055|2684|1513|42742|95349|1645738|4829|27658|7|23|3|1|59|438|92|37.23|68.50|32.19|3340.52|2961.48|3425.16|6302.00|88.84|0.00|2961.48|2961.48|3050.32|5922.96|6011.80|-463.68| +2452140|65188|2452235|13601|79055|2684|1513|42742|95349|1645738|4829|27658|11|19|11|5|36|438|25|73.22|183.78|47.78|3400.00|1194.50|1830.50|4594.50|28.66|716.70|1654.00|477.80|506.46|2131.80|2160.46|-1352.70| +2452140|65188|2452146|10743|79055|2684|1513|42742|95349|1645738|4829|27658|49|3|9|4|226|438|4|1.49|2.56|0.46|8.40|1.84|5.96|10.24|0.00|0.00|1.84|1.84|1.84|3.68|3.68|-4.12| +2452140|65188|2452149|97|79055|2684|1513|42742|95349|1645738|4829|27658|23|25|14|3|219|438|4|99.67|135.55|60.99|298.24|243.96|398.68|542.20|7.92|85.38|48.76|158.58|166.50|207.34|215.26|-240.10| +2452082|67082|2452196|2997|73091|750415|4539|20310|84856|861879|2729|5700|29|29|2|2|22|439|64|56.75|152.09|53.23|6327.04|3406.72|3632.00|9733.76|7.15|3304.51|3309.44|102.21|109.36|3411.65|3418.80|-3529.79| +2452082|67082|2452120|5349|73091|750415|4539|20310|84856|861879|2729|5700|33|3|17|5|10|439|61|41.36|67.00|7.37|3637.43|449.57|2522.96|4087.00|22.47|0.00|1798.28|449.57|472.04|2247.85|2270.32|-2073.39| +2452082|67082|2452094|13083|73091|750415|4539|20310|84856|861879|2729|5700|3|7|16|3|160|439|66|6.24|7.80|2.57|345.18|169.62|411.84|514.80|3.39|0.00|154.44|169.62|173.01|324.06|327.45|-242.22| +2452082|67082|2452123|6195|73091|750415|4539|20310|84856|861879|2729|5700|19|5|16|2|215|439|24|54.74|85.39|17.93|1619.04|430.32|1313.76|2049.36|34.42|0.00|573.60|430.32|464.74|1003.92|1038.34|-883.44| +2452082|67082|2452165|9375|73091|750415|4539|20310|84856|861879|2729|5700|11|3|8|4|230|439|50|87.98|189.15|75.66|5674.50|3783.00|4399.00|9457.50|151.32|0.00|1229.00|3783.00|3934.32|5012.00|5163.32|-616.00| +2452082|67082|2452192|255|73091|750415|4539|20310|84856|861879|2729|5700|27|11|8|2|169|439|19|8.23|19.75|0.98|356.63|18.62|156.37|375.25|1.11|0.00|101.27|18.62|19.73|119.89|121.00|-137.75| +2452082|67082|2452084|6479|73091|750415|4539|20310|84856|861879|2729|5700|25|11|6|4|82|439|89|43.85|62.26|19.30|3823.44|1717.70|3902.65|5541.14|51.53|0.00|886.44|1717.70|1769.23|2604.14|2655.67|-2184.95| +2452082|67082|2452099|2099|73091|750415|4539|20310|84856|861879|2729|5700|33|9|13|1|89|439|80|14.57|15.73|8.33|592.00|666.40|1165.60|1258.40|48.51|59.97|604.00|606.43|654.94|1210.43|1258.94|-559.17| +2452082|67082|2452144|2075|73091|750415|4539|20310|84856|861879|2729|5700|31|7|19|2|35|439|25|2.34|3.76|0.03|93.25|0.75|58.50|94.00|0.00|0.69|24.25|0.06|0.06|24.31|24.31|-58.44| +2452082|67082|2452101|943|73091|750415|4539|20310|84856|861879|2729|5700|1|3|3|2|88|439|28|1.66|3.63|1.05|72.24|29.40|46.48|101.64|1.17|0.00|14.00|29.40|30.57|43.40|44.57|-17.08| +2451450|69022|2451489|9151|44494|159676|6505|7287|84432|518577|6080|46462|16|22|15|3|51|440|53|32.18|93.00|11.16|4337.52|591.48|1705.54|4929.00|29.57|0.00|2218.05|591.48|621.05|2809.53|2839.10|-1114.06| +2451450|69022|2451570|10204|44494|159676|6505|7287|84432|518577|6080|46462|44|14|13|3|203|440|75|29.50|44.25|15.93|2124.00|1194.75|2212.50|3318.75|107.52|0.00|231.75|1194.75|1302.27|1426.50|1534.02|-1017.75| +2451450|69022|2451529|10094|44494|159676|6505|7287|84432|518577|6080|46462|10|7|2|5|148|440|21|5.95|7.02|5.96|22.26|125.16|124.95|147.42|3.75|0.00|29.40|125.16|128.91|154.56|158.31|0.21| +2451450|69022|2451501|3118|44494|159676|6505|7287|84432|518577|6080|46462|46|13|20|3|46|440|32|78.15|113.31|19.26|3009.60|616.32|2500.80|3625.92|12.32|0.00|362.56|616.32|628.64|978.88|991.20|-1884.48| +2451450|69022|2451568|4064|44494|159676|6505|7287|84432|518577|6080|46462|38|1|15|1|237|440|80|74.66|117.21|99.62|1407.20|7969.60|5972.80|9376.80|717.26|0.00|2156.00|7969.60|8686.86|10125.60|10842.86|1996.80| +2451450|69022|2451472|3022|44494|159676|6505|7287|84432|518577|6080|46462|52|19|16|2|35|440|25|63.02|81.29|51.21|752.00|1280.25|1575.50|2032.25|64.01|0.00|934.75|1280.25|1344.26|2215.00|2279.01|-295.25| +2451450|69022|2451457|8576|44494|159676|6505|7287|84432|518577|6080|46462|40|16|17|2|130|440|7|76.93|203.09|8.12|1364.79|56.84|538.51|1421.63|3.97|0.00|511.77|56.84|60.81|568.61|572.58|-481.67| +2451450|69022|2451472|15644|44494|159676|6505|7287|84432|518577|6080|46462|16|25|17|1|55|440|82|98.24|288.82|83.75|16815.74|6867.50|8055.68|23683.24|480.72|0.00|2841.30|6867.50|7348.22|9708.80|10189.52|-1188.18| +2451450|69022|2451536|3751|44494|159676|6505|7287|84432|518577|6080|46462|34|20|14|5|103|440|32|24.26|42.21|38.83|108.16|1242.56|776.32|1350.72|0.00|0.00|324.16|1242.56|1242.56|1566.72|1566.72|466.24| +2451450|69022|2451541|14893|44494|159676|6505|7287|84432|518577|6080|46462|55|10|14|2|180|440|89|66.71|198.79|166.98|2831.09|14861.22|5937.19|17692.31|436.92|8619.50|1237.99|6241.72|6678.64|7479.71|7916.63|304.53| +2452169|48052|2452174|5233|29779|1915511|2043|6528|9125|491935|6924|37056|47|25|5|4|243|441|5|78.33|209.92|37.78|860.70|188.90|391.65|1049.60|7.55|0.00|62.95|188.90|196.45|251.85|259.40|-202.75| +2452169|48052|2452174|6143|29779|1915511|2043|6528|9125|491935|6924|37056|23|17|16|3|277|441|88|89.04|179.86|0.00|15827.68|0.00|7835.52|15827.68|0.00|0.00|3639.68|0.00|0.00|3639.68|3639.68|-7835.52| +2452169|48052|2452249|9995|29779|1915511|2043|6528|9125|491935|6924|37056|29|23|2|4|19|441|54|8.82|9.26|7.50|95.04|405.00|476.28|500.04|32.40|0.00|119.88|405.00|437.40|524.88|557.28|-71.28| +2452169|48052|2452234|1307|29779|1915511|2043|6528|9125|491935|6924|37056|15|3|3|4|215|441|49|13.13|29.41|5.88|1152.97|288.12|643.37|1441.09|2.88|0.00|158.27|288.12|291.00|446.39|449.27|-355.25| +2452169|48052|2452200|11481|29779|1915511|2043|6528|9125|491935|6924|37056|7|9|16|2|98|441|31|6.93|9.84|7.87|61.07|243.97|214.83|305.04|14.63|0.00|63.86|243.97|258.60|307.83|322.46|29.14| +2452169|48052|2452220|12941|29779|1915511|2043|6528|9125|491935|6924|37056|23|17|16|3|127|441|31|61.40|168.23|15.14|4745.79|469.34|1903.40|5215.13|18.77|0.00|2190.15|469.34|488.11|2659.49|2678.26|-1434.06| +2452169|48052|2452266|6213|29779|1915511|2043|6528|9125|491935|6924|37056|27|17|7|5|33|441|16|97.68|107.44|81.65|412.64|1306.40|1562.88|1719.04|0.00|0.00|859.52|1306.40|1306.40|2165.92|2165.92|-256.48| +2452169|48052|2452174|5937|29779|1915511|2043|6528|9125|491935|6924|37056|7|21|5|4|166|441|12|35.50|47.21|12.74|413.64|152.88|426.00|566.52|0.39|113.13|16.92|39.75|40.14|56.67|57.06|-386.25| +2452169|48052|2452171|17719|29779|1915511|2043|6528|9125|491935|6924|37056|51|5|20|2|174|441|83|38.94|38.94|14.40|2036.82|1195.20|3232.02|3232.02|47.80|0.00|225.76|1195.20|1243.00|1420.96|1468.76|-2036.82| +2452169|48052|2452237|15333|29779|1915511|2043|6528|9125|491935|6924|37056|5|15|10|5|270|441|76|79.57|224.38|208.67|1193.96|15858.92|6047.32|17052.88|317.17|0.00|3921.60|15858.92|16176.09|19780.52|20097.69|9811.60| +2452169|48052|2452280|7723|29779|1915511|2043|6528|9125|491935|6924|37056|29|3|12|2|152|441|79|49.44|90.96|45.48|3592.92|3592.92|3905.76|7185.84|71.85|0.00|2874.02|3592.92|3664.77|6466.94|6538.79|-312.84| +2452169|48052|2452219|1185|29779|1915511|2043|6528|9125|491935|6924|37056|21|17|10|1|229|441|9|7.34|18.57|8.72|88.65|78.48|66.06|167.13|6.27|0.00|21.69|78.48|84.75|100.17|106.44|12.42| +2452169|48052|2452171|16927|29779|1915511|2043|6528|9125|491935|6924|37056|43|23|2|2|192|441|28|67.73|83.30|6.66|2145.92|186.48|1896.44|2332.40|9.32|0.00|186.48|186.48|195.80|372.96|382.28|-1709.96| +2452169|48052|2452204|7641|29779|1915511|2043|6528|9125|491935|6924|37056|41|9|6|2|68|441|23|14.25|15.24|12.95|52.67|297.85|327.75|350.52|5.95|0.00|112.01|297.85|303.80|409.86|415.81|-29.90| +2452169|48052|2452281|13471|29779|1915511|2043|6528|9125|491935|6924|37056|31|21|13|3|243|441|36|5.88|17.28|12.78|162.00|460.08|211.68|622.08|9.20|0.00|12.24|460.08|469.28|472.32|481.52|248.40| +2452169|48052|2452278|9033|29779|1915511|2043|6528|9125|491935|6924|37056|3|25|14|5|256|441|65|22.47|51.90|47.22|304.20|3069.30|1460.55|3373.50|30.69|0.00|1517.75|3069.30|3099.99|4587.05|4617.74|1608.75| +2451893|44620|2451959|15848|73343|1026282|1974|9666|57422|240861|5189|46881|35|29|19|1|110|442|62|39.53|118.59|40.32|4852.74|2499.84|2450.86|7352.58|74.99|0.00|440.82|2499.84|2574.83|2940.66|3015.65|48.98| +2451893|44620|2451956|12493|73343|1026282|1974|9666|57422|240861|5189|46881|31|29|4|1|125|442|84|73.52|77.19|67.92|778.68|5705.28|6175.68|6483.96|263.58|1312.21|64.68|4393.07|4656.65|4457.75|4721.33|-1782.61| +2451893|44620|2451899|2821|73343|1026282|1974|9666|57422|240861|5189|46881|11|17|2|5|198|442|70|92.91|193.25|102.42|6358.10|7169.40|6503.70|13527.50|143.38|0.00|6357.40|7169.40|7312.78|13526.80|13670.18|665.70| +2451893|44620|2451992|1325|73343|1026282|1974|9666|57422|240861|5189|46881|20|25|12|4|183|442|55|22.98|60.20|56.58|199.10|3111.90|1263.90|3311.00|217.83|0.00|1522.95|3111.90|3329.73|4634.85|4852.68|1848.00| +2451893|44620|2451952|2947|73343|1026282|1974|9666|57422|240861|5189|46881|56|13|12|2|292|442|14|52.96|154.64|9.27|2035.18|129.78|741.44|2164.96|11.68|0.00|151.48|129.78|141.46|281.26|292.94|-611.66| +2451893|44620|2451918|12014|73343|1026282|1974|9666|57422|240861|5189|46881|35|17|10|1|12|442|62|40.91|80.18|78.57|99.82|4871.34|2536.42|4971.16|146.14|0.00|1739.72|4871.34|5017.48|6611.06|6757.20|2334.92| +2451893|44620|2451956|14594|73343|1026282|1974|9666|57422|240861|5189|46881|8|1|4|5|273|442|95|93.84|127.62|56.15|6789.65|5334.25|8914.80|12123.90|17.06|5120.88|3758.20|213.37|230.43|3971.57|3988.63|-8701.43| +2451893|44620|2451989|17507|73343|1026282|1974|9666|57422|240861|5189|46881|1|13|13|2|198|442|79|78.49|83.98|15.11|5440.73|1193.69|6200.71|6634.42|3.93|799.77|3317.21|393.92|397.85|3711.13|3715.06|-5806.79| +2451893|44620|2451990|5150|73343|1026282|1974|9666|57422|240861|5189|46881|44|20|12|5|118|442|7|55.32|163.19|45.69|822.50|319.83|387.24|1142.33|10.07|175.90|308.42|143.93|154.00|452.35|462.42|-243.31| +2451819|28599|2451886|8987|22525|512160|195|4277|31877|924467|6829|2064|38|7|1|5|249|443|36|54.26|105.80|64.53|1485.72|2323.08|1953.36|3808.80|23.23|0.00|723.60|2323.08|2346.31|3046.68|3069.91|369.72| +2451819|28599|2451907|4747|22525|512160|195|4277|31877|924467|6829|2064|56|23|5|3|268|443|7|65.83|112.56|79.91|228.55|559.37|460.81|787.92|39.15|0.00|212.73|559.37|598.52|772.10|811.25|98.56| +2451819|28599|2451856|5885|22525|512160|195|4277|31877|924467|6829|2064|38|19|10|5|160|443|19|25.47|72.08|25.22|890.34|479.18|483.93|1369.52|33.54|0.00|41.04|479.18|512.72|520.22|553.76|-4.75| +2451819|28599|2451923|12722|22525|512160|195|4277|31877|924467|6829|2064|25|23|11|5|272|443|79|13.55|27.50|12.37|1195.27|977.23|1070.45|2172.50|19.54|0.00|521.40|977.23|996.77|1498.63|1518.17|-93.22| +2451819|28599|2451870|296|22525|512160|195|4277|31877|924467|6829|2064|19|20|2|1|277|443|80|39.83|58.55|13.46|3607.20|1076.80|3186.40|4684.00|10.76|0.00|468.00|1076.80|1087.56|1544.80|1555.56|-2109.60| +2451819|28599|2451866|2941|22525|512160|195|4277|31877|924467|6829|2064|11|5|7|1|158|443|96|47.82|67.42|37.75|2848.32|3624.00|4590.72|6472.32|0.00|0.00|387.84|3624.00|3624.00|4011.84|4011.84|-966.72| +2451819|28599|2451914|16111|22525|512160|195|4277|31877|924467|6829|2064|47|13|10|1|99|443|85|12.61|33.16|14.59|1578.45|1240.15|1071.85|2818.60|11.90|644.87|1240.15|595.28|607.18|1835.43|1847.33|-476.57| +2451819|28599|2451876|15121|22525|512160|195|4277|31877|924467|6829|2064|26|5|12|1|90|443|13|19.77|40.52|36.06|57.98|468.78|257.01|526.76|32.81|0.00|173.81|468.78|501.59|642.59|675.40|211.77| +2451819|28599|2451870|16717|22525|512160|195|4277|31877|924467|6829|2064|43|14|4|2|153|443|72|12.80|27.39|10.13|1242.72|729.36|921.60|1972.08|58.34|0.00|492.48|729.36|787.70|1221.84|1280.18|-192.24| +2451819|28599|2451880|14899|22525|512160|195|4277|31877|924467|6829|2064|35|20|7|3|119|443|8|34.74|94.84|51.21|349.04|409.68|277.92|758.72|2.37|290.87|379.36|118.81|121.18|498.17|500.54|-159.11| +2451819|28599|2451896|7621|22525|512160|195|4277|31877|924467|6829|2064|53|13|17|4|104|443|55|63.67|175.09|61.28|6259.55|3370.40|3501.85|9629.95|101.11|0.00|962.50|3370.40|3471.51|4332.90|4434.01|-131.45| +2451819|28599|2451879|2443|22525|512160|195|4277|31877|924467|6829|2064|19|2|4|1|68|443|35|75.49|163.05|50.54|3937.85|1768.90|2642.15|5706.75|0.00|1202.85|1083.95|566.05|566.05|1650.00|1650.00|-2076.10| +2451902|39500|2451931|17294|16197|952931|512|2046|57703|641551|1495|38629|25|13|11|2|266|444|19|81.69|106.19|1.06|1997.47|20.14|1552.11|2017.61|0.09|19.13|20.14|1.01|1.10|21.15|21.24|-1551.10| +2451902|39500|2451951|8023|16197|952931|512|2046|57703|641551|1495|38629|38|23|1|3|295|444|62|27.76|65.23|65.23|0.00|4044.26|1721.12|4044.26|283.09|0.00|1293.94|4044.26|4327.35|5338.20|5621.29|2323.14| +2451902|39500|2451975|13261|16197|952931|512|2046|57703|641551|1495|38629|2|26|13|2|34|444|16|86.56|135.89|8.15|2043.84|130.40|1384.96|2174.24|5.21|0.00|152.16|130.40|135.61|282.56|287.77|-1254.56| +2451902|39500|2451924|16393|16197|952931|512|2046|57703|641551|1495|38629|32|19|20|3|243|444|15|14.92|39.23|4.31|523.80|64.65|223.80|588.45|1.29|0.00|64.65|64.65|65.94|129.30|130.59|-159.15| +2451902|39500|2451941|6901|16197|952931|512|2046|57703|641551|1495|38629|2|2|12|2|182|444|7|62.76|85.98|0.00|601.86|0.00|439.32|601.86|0.00|0.00|30.03|0.00|0.00|30.03|30.03|-439.32| +2451902|39500|2451911|6391|16197|952931|512|2046|57703|641551|1495|38629|50|11|3|3|163|444|55|34.49|102.09|59.21|2358.40|3256.55|1896.95|5614.95|65.13|0.00|729.85|3256.55|3321.68|3986.40|4051.53|1359.60| +2451902|39500|2451960|12899|16197|952931|512|2046|57703|641551|1495|38629|25|26|12|3|181|444|38|99.26|166.75|35.01|5006.12|1330.38|3771.88|6336.50|66.51|0.00|0.00|1330.38|1396.89|1330.38|1396.89|-2441.50| +2451902|39500|2451923|3992|16197|952931|512|2046|57703|641551|1495|38629|8|2|19|1|233|444|40|55.59|133.97|56.26|3108.40|2250.40|2223.60|5358.80|157.52|0.00|106.80|2250.40|2407.92|2357.20|2514.72|26.80| +2451902|39500|2451939|11959|16197|952931|512|2046|57703|641551|1495|38629|11|8|17|1|25|444|68|29.31|43.96|33.40|718.08|2271.20|1993.08|2989.28|14.30|2066.79|119.00|204.41|218.71|323.41|337.71|-1788.67| +2451902|39500|2451966|11753|16197|952931|512|2046|57703|641551|1495|38629|38|13|11|5|79|444|97|15.40|19.09|5.91|1278.46|573.27|1493.80|1851.73|0.00|504.47|721.68|68.80|68.80|790.48|790.48|-1425.00| +2451902|39500|2452007|3170|16197|952931|512|2046|57703|641551|1495|38629|20|17|20|3|26|444|95|81.51|198.06|166.37|3010.55|15805.15|7743.45|18815.70|790.25|0.00|1504.80|15805.15|16595.40|17309.95|18100.20|8061.70| +2451902|39500|2452017|10663|16197|952931|512|2046|57703|641551|1495|38629|23|1|10|4|275|444|34|36.12|59.59|29.79|1013.20|1012.86|1228.08|2026.06|0.00|0.00|871.08|1012.86|1012.86|1883.94|1883.94|-215.22| +2451902|39500|2452017|16004|16197|952931|512|2046|57703|641551|1495|38629|26|2|7|5|100|444|86|37.43|68.12|47.00|1816.32|4042.00|3218.98|5858.32|363.78|0.00|585.66|4042.00|4405.78|4627.66|4991.44|823.02| +2451902|39500|2451971|3959|16197|952931|512|2046|57703|641551|1495|38629|38|19|10|2|287|444|63|11.48|24.79|12.14|796.95|764.82|723.24|1561.77|61.18|0.00|421.47|764.82|826.00|1186.29|1247.47|41.58| +2452169|22457|2452285|11651|27479|1370441|5050|17612|51452|1219434|6592|43554|45|1|19|2|265|445|3|25.45|69.47|2.08|202.17|6.24|76.35|208.41|0.37|0.00|16.65|6.24|6.61|22.89|23.26|-70.11| +2452169|22457|2452274|7021|27479|1370441|5050|17612|51452|1219434|6592|43554|51|7|2|2|275|445|86|82.43|93.14|40.05|4565.74|3444.30|7088.98|8010.04|206.65|0.00|3123.52|3444.30|3650.95|6567.82|6774.47|-3644.68| +2452169|22457|2452174|6085|27479|1370441|5050|17612|51452|1219434|6592|43554|7|17|4|4|77|445|38|71.64|151.16|74.06|2929.80|2814.28|2722.32|5744.08|84.42|0.00|1665.54|2814.28|2898.70|4479.82|4564.24|91.96| +2452169|22457|2452177|14813|27479|1370441|5050|17612|51452|1219434|6592|43554|35|13|9|3|27|445|55|52.06|58.82|44.11|809.05|2426.05|2863.30|3235.10|48.52|0.00|1261.15|2426.05|2474.57|3687.20|3735.72|-437.25| +2452169|22457|2452233|4799|27479|1370441|5050|17612|51452|1219434|6592|43554|53|11|5|4|243|445|27|96.80|285.56|259.85|694.17|7015.95|2613.60|7710.12|491.11|0.00|539.46|7015.95|7507.06|7555.41|8046.52|4402.35| +2452169|22457|2452201|13549|27479|1370441|5050|17612|51452|1219434|6592|43554|35|23|6|5|264|445|27|33.51|87.46|32.36|1487.70|873.72|904.77|2361.42|2.09|847.50|802.71|26.22|28.31|828.93|831.02|-878.55| +2452169|22457|2452254|11597|27479|1370441|5050|17612|51452|1219434|6592|43554|39|13|11|3|282|445|29|2.64|6.70|4.69|58.29|136.01|76.56|194.30|9.52|0.00|25.23|136.01|145.53|161.24|170.76|59.45| +2452169|22457|2452200|14971|27479|1370441|5050|17612|51452|1219434|6592|43554|17|25|7|3|212|445|44|71.65|192.02|5.76|8195.44|253.44|3152.60|8448.88|22.80|0.00|1098.24|253.44|276.24|1351.68|1374.48|-2899.16| +2452169|22457|2452256|16963|27479|1370441|5050|17612|51452|1219434|6592|43554|9|17|3|4|268|445|97|75.94|164.03|34.44|12570.23|3340.68|7366.18|15910.91|0.00|0.00|1113.56|3340.68|3340.68|4454.24|4454.24|-4025.50| +2451690|71218|2451785|13717|41736|1038675|3167|1706|41736|1038675|3167|1706|47|25|2|1|213|446|48|25.48|70.57|16.93|2574.72|812.64|1223.04|3387.36|16.25|0.00|1591.68|812.64|828.89|2404.32|2420.57|-410.40| +2451690|71218|2451760|10099|41736|1038675|3167|1706|41736|1038675|3167|1706|25|20|19|2|185|446|82|6.53|18.21|17.11|90.20|1403.02|535.46|1493.22|98.21|0.00|29.52|1403.02|1501.23|1432.54|1530.75|867.56| +2451690|71218|2451793|3199|41736|1038675|3167|1706|41736|1038675|3167|1706|53|17|8|5|232|446|56|45.10|82.53|21.45|3420.48|1201.20|2525.60|4621.68|54.05|600.60|878.08|600.60|654.65|1478.68|1532.73|-1925.00| +2451690|71218|2451716|3037|41736|1038675|3167|1706|41736|1038675|3167|1706|50|14|3|5|247|446|82|32.55|74.21|13.35|4990.52|1094.70|2669.10|6085.22|98.52|0.00|1825.32|1094.70|1193.22|2920.02|3018.54|-1574.40| +2451690|71218|2451724|1321|41736|1038675|3167|1706|41736|1038675|3167|1706|13|8|3|4|52|446|58|73.76|99.57|65.71|1963.88|3811.18|4278.08|5775.06|343.00|0.00|288.26|3811.18|4154.18|4099.44|4442.44|-466.90| +2451690|71218|2451782|5048|41736|1038675|3167|1706|41736|1038675|3167|1706|53|14|19|2|288|446|31|40.37|104.55|87.82|518.63|2722.42|1251.47|3241.05|190.56|0.00|907.37|2722.42|2912.98|3629.79|3820.35|1470.95| +2451690|71218|2451694|9353|41736|1038675|3167|1706|41736|1038675|3167|1706|49|7|13|2|278|446|62|34.53|92.88|87.30|345.96|5412.60|2140.86|5758.56|378.88|0.00|921.32|5412.60|5791.48|6333.92|6712.80|3271.74| +2451690|71218|2451777|11077|41736|1038675|3167|1706|41736|1038675|3167|1706|1|25|8|3|298|446|56|79.85|206.81|179.92|1505.84|10075.52|4471.60|11581.36|403.02|0.00|810.32|10075.52|10478.54|10885.84|11288.86|5603.92| +2451690|58009|2451710|12403|59924|338066|3855|29597|48544|661113|70|28259|19|11|12|3|144|447|67|51.95|116.88|81.81|2349.69|5481.27|3480.65|7830.96|438.50|0.00|1644.18|5481.27|5919.77|7125.45|7563.95|2000.62| +2451690|58009|2451803|3656|59924|338066|3855|29597|48544|661113|70|28259|2|2|5|5|85|447|64|45.60|89.83|32.33|3680.00|2069.12|2918.40|5749.12|103.45|0.00|1322.24|2069.12|2172.57|3391.36|3494.81|-849.28| +2451690|58009|2451702|2924|59924|338066|3855|29597|48544|661113|70|28259|38|11|12|5|271|447|50|53.83|132.96|111.68|1064.00|5584.00|2691.50|6648.00|111.68|0.00|1329.50|5584.00|5695.68|6913.50|7025.18|2892.50| +2451690|58009|2451695|15374|59924|338066|3855|29597|48544|661113|70|28259|35|20|14|2|230|447|91|27.77|29.15|9.61|1778.14|874.51|2527.07|2652.65|0.00|0.00|1219.40|874.51|874.51|2093.91|2093.91|-1652.56| +2451690|58009|2451724|4547|59924|338066|3855|29597|48544|661113|70|28259|43|23|9|4|116|447|10|65.16|133.57|10.68|1228.90|106.80|651.60|1335.70|0.98|8.54|560.90|98.26|99.24|659.16|660.14|-553.34| +2451690|58009|2451701|12421|59924|338066|3855|29597|48544|661113|70|28259|7|7|4|4|4|447|51|50.42|82.68|26.45|2867.73|1348.95|2571.42|4216.68|121.40|0.00|1770.72|1348.95|1470.35|3119.67|3241.07|-1222.47| +2451690|58009|2451765|2714|59924|338066|3855|29597|48544|661113|70|28259|19|7|1|5|152|447|43|96.87|207.30|120.23|3744.01|5169.89|4165.41|8913.90|258.49|0.00|1158.42|5169.89|5428.38|6328.31|6586.80|1004.48| +2451690|58009|2451755|16694|59924|338066|3855|29597|48544|661113|70|28259|32|20|2|1|67|447|38|60.26|130.16|1.30|4896.68|49.40|2289.88|4946.08|0.98|0.00|1780.30|49.40|50.38|1829.70|1830.68|-2240.48| +2451690|58009|2451777|7694|59924|338066|3855|29597|48544|661113|70|28259|5|14|6|1|46|447|14|58.83|71.18|35.59|498.26|498.26|823.62|996.52|9.11|194.32|378.56|303.94|313.05|682.50|691.61|-519.68| +2451690|58009|2451760|43|59924|338066|3855|29597|48544|661113|70|28259|25|20|12|5|298|447|92|53.17|139.30|110.04|2691.92|10123.68|4891.64|12815.60|809.89|0.00|1024.88|10123.68|10933.57|11148.56|11958.45|5232.04| +2451690|58009|2451694|8366|59924|338066|3855|29597|48544|661113|70|28259|1|17|7|5|213|447|40|31.15|78.80|71.70|284.00|2868.00|1246.00|3152.00|200.76|0.00|914.00|2868.00|3068.76|3782.00|3982.76|1622.00| +2451137|35460|2451240|13888|39890|905816|4189|8115|16383|1797340|7187|21513|4|8|7|4|71|448|15|27.71|72.87|10.93|929.10|163.95|415.65|1093.05|14.75|0.00|284.10|163.95|178.70|448.05|462.80|-251.70| +2451137|35460|2451173|8383|39890|905816|4189|8115|16383|1797340|7187|21513|31|26|12|5|278|448|98|83.01|86.33|18.99|6599.32|1861.02|8134.98|8460.34|111.66|0.00|2791.04|1861.02|1972.68|4652.06|4763.72|-6273.96| +2451137|35460|2451197|9206|39890|905816|4189|8115|16383|1797340|7187|21513|55|25|8|5|65|448|37|73.49|96.27|92.41|142.82|3419.17|2719.13|3561.99|307.72|0.00|1460.39|3419.17|3726.89|4879.56|5187.28|700.04| +2451137|35460|2451208|8218|39890|905816|4189|8115|16383|1797340|7187|21513|10|16|5|3|94|448|15|85.70|208.25|112.45|1437.00|1686.75|1285.50|3123.75|0.00|927.71|655.95|759.04|759.04|1414.99|1414.99|-526.46| +2451137|35460|2451229|356|39890|905816|4189|8115|16383|1797340|7187|21513|34|8|19|3|287|448|95|40.24|60.76|37.67|2193.55|3578.65|3822.80|5772.20|286.29|0.00|1846.80|3578.65|3864.94|5425.45|5711.74|-244.15| +2451137|35460|2451227|4072|39890|905816|4189|8115|16383|1797340|7187|21513|31|13|1|2|123|448|47|73.45|173.34|84.93|4155.27|3991.71|3452.15|8146.98|79.83|0.00|733.20|3991.71|4071.54|4724.91|4804.74|539.56| +2451137|35460|2451171|3298|39890|905816|4189|8115|16383|1797340|7187|21513|56|2|8|1|267|448|17|12.24|30.96|3.71|463.25|63.07|208.08|526.32|0.00|16.39|126.31|46.68|46.68|172.99|172.99|-161.40| +2451137|35460|2451251|2932|39890|905816|4189|8115|16383|1797340|7187|21513|46|10|18|5|132|448|58|49.39|134.34|4.03|7557.98|233.74|2864.62|7791.72|7.57|107.52|622.92|126.22|133.79|749.14|756.71|-2738.40| +2451137|35460|2451149|14060|39890|905816|4189|8115|16383|1797340|7187|21513|7|20|5|2|166|448|3|52.20|138.85|123.57|45.84|370.71|156.60|416.55|11.12|0.00|124.95|370.71|381.83|495.66|506.78|214.11| +2451137|35460|2451237|15973|39890|905816|4189|8115|16383|1797340|7187|21513|4|4|13|3|63|448|85|78.59|180.75|103.02|6607.05|8756.70|6680.15|15363.75|0.00|3940.51|5683.95|4816.19|4816.19|10500.14|10500.14|-1863.96| +2451137|35460|2451140|6145|39890|905816|4189|8115|16383|1797340|7187|21513|50|14|20|5|127|448|92|64.55|78.10|10.15|6251.40|933.80|5938.60|7185.20|18.67|0.00|143.52|933.80|952.47|1077.32|1095.99|-5004.80| +2452235|41387|2452269|7963|66673|661303|4733|18916|79202|210992|3959|49789|9|5|11|4|244|449|70|68.26|150.85|140.29|739.20|9820.30|4778.20|10559.50|589.21|0.00|2639.70|9820.30|10409.51|12460.00|13049.21|5042.10| +2452235|41387|2452289|7459|66673|661303|4733|18916|79202|210992|3959|49789|5|11|2|5|181|449|23|75.16|188.65|137.71|1171.62|3167.33|1728.68|4338.95|96.92|2090.43|650.67|1076.90|1173.82|1727.57|1824.49|-651.78| +2452235|41387|2452250|5883|66673|661303|4733|18916|79202|210992|3959|49789|43|7|6|5|287|449|17|37.76|38.89|12.44|449.65|211.48|641.92|661.13|4.22|0.00|119.00|211.48|215.70|330.48|334.70|-430.44| +2452235|41387|2452354|1801|66673|661303|4733|18916|79202|210992|3959|49789|47|23|13|1|262|449|87|46.69|81.24|8.93|6290.97|776.91|4062.03|7067.88|54.38|0.00|2897.10|776.91|831.29|3674.01|3728.39|-3285.12| +2452235|41387|2452250|12581|66673|661303|4733|18916|79202|210992|3959|49789|57|3|5|3|11|449|40|1.99|2.84|1.22|64.80|48.80|79.60|113.60|2.80|8.78|48.80|40.02|42.82|88.82|91.62|-39.58| +2452235|41387|2452351|3549|66673|661303|4733|18916|79202|210992|3959|49789|51|21|16|1|176|449|37|9.66|24.15|3.38|768.49|125.06|357.42|893.55|2.50|0.00|223.11|125.06|127.56|348.17|350.67|-232.36| +2452235|41387|2452341|11499|66673|661303|4733|18916|79202|210992|3959|49789|27|9|13|4|45|449|50|88.36|229.73|209.05|1034.00|10452.50|4418.00|11486.50|104.52|0.00|5513.50|10452.50|10557.02|15966.00|16070.52|6034.50| +2452235|41387|2452304|2719|66673|661303|4733|18916|79202|210992|3959|49789|21|3|13|2|208|449|89|68.37|99.13|85.25|1235.32|7587.25|6084.93|8822.57|281.48|3566.00|2117.31|4021.25|4302.73|6138.56|6420.04|-2063.68| +2452235|41387|2452331|17013|66673|661303|4733|18916|79202|210992|3959|49789|57|15|17|1|298|449|18|66.59|145.83|138.53|131.40|2493.54|1198.62|2624.94|124.67|0.00|1181.16|2493.54|2618.21|3674.70|3799.37|1294.92| +2452235|41387|2452272|8205|66673|661303|4733|18916|79202|210992|3959|49789|3|9|11|4|7|449|39|60.28|71.13|27.02|1720.29|1053.78|2350.92|2774.07|33.72|210.75|138.45|843.03|876.75|981.48|1015.20|-1507.89| +2452235|41387|2452248|7241|66673|661303|4733|18916|79202|210992|3959|49789|45|3|18|4|80|449|21|7.88|20.80|2.08|393.12|43.68|165.48|436.80|0.98|32.76|8.61|10.92|11.90|19.53|20.51|-154.56| +2452235|41387|2452350|8639|66673|661303|4733|18916|79202|210992|3959|49789|15|7|20|1|172|449|44|1.09|1.60|0.56|45.76|24.64|47.96|70.40|0.98|0.00|8.36|24.64|25.62|33.00|33.98|-23.32| +2452245|46162|2452297|11139|8766|424702|4285|7021|75901|1076553|1276|26812|29|27|10|1|2|450|66|76.45|123.08|2.46|7960.92|162.36|5045.70|8123.28|6.49|0.00|0.00|162.36|168.85|162.36|168.85|-4883.34| +2452245|46162|2452346|3735|8766|424702|4285|7021|75901|1076553|1276|26812|51|23|8|1|127|450|75|2.09|5.62|5.05|42.75|378.75|156.75|421.50|30.30|0.00|151.50|378.75|409.05|530.25|560.55|222.00| +2452245|46162|2452326|17889|8766|424702|4285|7021|75901|1076553|1276|26812|51|29|7|1|28|450|67|84.51|189.30|170.37|1268.31|11414.79|5662.17|12683.10|1027.33|0.00|887.75|11414.79|12442.12|12302.54|13329.87|5752.62| +2452245|46162|2452308|15039|8766|424702|4285|7021|75901|1076553|1276|26812|37|11|10|4|108|450|31|58.96|127.94|62.69|2022.75|1943.39|1827.76|3966.14|77.73|0.00|79.05|1943.39|2021.12|2022.44|2100.17|115.63| +2452245|46162|2452251|13679|8766|424702|4285|7021|75901|1076553|1276|26812|43|1|20|4|58|450|82|21.57|31.27|18.44|1052.06|1512.08|1768.74|2564.14|30.24|0.00|512.50|1512.08|1542.32|2024.58|2054.82|-256.66| +2452245|46162|2452303|14459|8766|424702|4285|7021|75901|1076553|1276|26812|57|25|10|2|131|450|49|34.05|67.75|18.97|2390.22|929.53|1668.45|3319.75|27.88|0.00|0.00|929.53|957.41|929.53|957.41|-738.92| +2452245|46162|2452328|14647|8766|424702|4285|7021|75901|1076553|1276|26812|27|11|11|1|148|450|53|58.74|129.22|102.08|1438.42|5410.24|3113.22|6848.66|108.20|0.00|1369.52|5410.24|5518.44|6779.76|6887.96|2297.02| +2452245|46162|2452273|589|8766|424702|4285|7021|75901|1076553|1276|26812|33|5|12|3|194|450|86|77.66|96.29|88.58|663.06|7617.88|6678.76|8280.94|457.07|0.00|2483.68|7617.88|8074.95|10101.56|10558.63|939.12| +2452234|13091|2452337|17701|40485|1350903|6568|28186|32715|1061273|1840|258|31|15|20|2|224|451|17|89.32|245.63|203.87|709.92|3465.79|1518.44|4175.71|97.04|2252.76|83.47|1213.03|1310.07|1296.50|1393.54|-305.41| +2452234|13091|2452327|5479|40485|1350903|6568|28186|32715|1061273|1840|258|41|17|20|3|289|451|37|11.84|25.45|10.18|564.99|376.66|438.08|941.65|33.89|0.00|235.32|376.66|410.55|611.98|645.87|-61.42| +2452234|13091|2452294|15511|40485|1350903|6568|28186|32715|1061273|1840|258|59|5|16|4|42|451|26|56.79|76.09|45.65|791.44|1186.90|1476.54|1978.34|47.47|0.00|989.04|1186.90|1234.37|2175.94|2223.41|-289.64| +2452234|13091|2452326|11785|40485|1350903|6568|28186|32715|1061273|1840|258|39|27|11|5|192|451|49|76.80|76.80|42.24|1693.44|2069.76|3763.20|3763.20|20.69|0.00|752.64|2069.76|2090.45|2822.40|2843.09|-1693.44| +2452234|13091|2452352|781|40485|1350903|6568|28186|32715|1061273|1840|258|47|1|8|5|292|451|83|65.01|80.61|11.28|5754.39|936.24|5395.83|6690.63|9.36|0.00|1404.36|936.24|945.60|2340.60|2349.96|-4459.59| +2452234|13091|2452334|1331|40485|1350903|6568|28186|32715|1061273|1840|258|31|29|14|1|125|451|68|74.23|208.58|31.28|12056.40|2127.04|5047.64|14183.44|62.96|1340.03|2978.40|787.01|849.97|3765.41|3828.37|-4260.63| +2452234|13091|2452351|1757|40485|1350903|6568|28186|32715|1061273|1840|258|35|17|17|3|229|451|93|51.46|99.83|9.98|8356.05|928.14|4785.78|9284.19|1.85|891.01|2413.35|37.13|38.98|2450.48|2452.33|-4748.65| +2452234|13091|2452297|12843|40485|1350903|6568|28186|32715|1061273|1840|258|59|3|14|1|278|451|33|50.03|136.58|121.55|495.99|4011.15|1650.99|4507.14|240.66|0.00|1712.70|4011.15|4251.81|5723.85|5964.51|2360.16| +2452234|13091|2452343|10865|40485|1350903|6568|28186|32715|1061273|1840|258|29|25|4|2|30|451|94|63.81|181.85|18.18|15384.98|1708.92|5998.14|17093.90|85.44|0.00|1708.92|1708.92|1794.36|3417.84|3503.28|-4289.22| +2451603|78034|2451613|10601|30988|1132775|4162|44702|43458|158431|5616|24203|7|29|19|3|267|452|63|78.84|86.72|9.53|4862.97|600.39|4966.92|5463.36|24.01|0.00|1147.23|600.39|624.40|1747.62|1771.63|-4366.53| +2451603|78034|2451649|3257|30988|1132775|4162|44702|43458|158431|5616|24203|44|2|1|5|149|452|32|55.81|92.08|0.00|2946.56|0.00|1785.92|2946.56|0.00|0.00|1089.92|0.00|0.00|1089.92|1089.92|-1785.92| +2451603|78034|2451622|16382|30988|1132775|4162|44702|43458|158431|5616|24203|43|8|8|1|134|452|63|65.50|69.43|63.87|350.28|4023.81|4126.50|4374.09|362.14|0.00|393.12|4023.81|4385.95|4416.93|4779.07|-102.69| +2451603|78034|2451715|12241|30988|1132775|4162|44702|43458|158431|5616|24203|26|25|12|4|63|452|57|58.21|115.83|115.83|0.00|6602.31|3317.97|6602.31|198.06|0.00|2640.81|6602.31|6800.37|9243.12|9441.18|3284.34| +2451603|78034|2451648|13747|30988|1132775|4162|44702|43458|158431|5616|24203|32|7|18|5|69|452|25|78.24|227.67|152.53|1878.50|3813.25|1956.00|5691.75|305.06|0.00|2049.00|3813.25|4118.31|5862.25|6167.31|1857.25| +2451603|78034|2451715|2917|30988|1132775|4162|44702|43458|158431|5616|24203|35|25|18|3|201|452|98|9.12|23.71|1.18|2207.94|115.64|893.76|2323.58|2.31|0.00|929.04|115.64|117.95|1044.68|1046.99|-778.12| +2451603|78034|2451624|4819|30988|1132775|4162|44702|43458|158431|5616|24203|31|11|3|3|260|452|11|89.11|266.43|45.29|2432.54|498.19|980.21|2930.73|24.90|0.00|58.52|498.19|523.09|556.71|581.61|-482.02| +2451603|78034|2451687|2939|30988|1132775|4162|44702|43458|158431|5616|24203|13|11|18|2|176|452|45|35.53|90.24|82.11|365.85|3694.95|1598.85|4060.80|110.84|0.00|1339.65|3694.95|3805.79|5034.60|5145.44|2096.10| +2451603|78034|2451642|15655|30988|1132775|4162|44702|43458|158431|5616|24203|41|5|12|4|53|452|30|29.39|83.76|41.04|1281.60|1231.20|881.70|2512.80|98.49|0.00|904.50|1231.20|1329.69|2135.70|2234.19|349.50| +2451603|78034|2451621|8663|30988|1132775|4162|44702|43458|158431|5616|24203|38|20|15|1|127|452|39|13.42|15.03|12.17|111.54|474.63|523.38|586.17|14.23|0.00|17.55|474.63|488.86|492.18|506.41|-48.75| +2451603|78034|2451704|10657|30988|1132775|4162|44702|43458|158431|5616|24203|11|29|7|1|268|452|2|68.73|104.46|37.60|133.72|75.20|137.46|208.92|6.01|0.00|20.88|75.20|81.21|96.08|102.09|-62.26| +2451603|78034|2451622|8486|30988|1132775|4162|44702|43458|158431|5616|24203|26|14|5|4|66|452|92|14.04|23.16|8.10|1385.52|745.20|1291.68|2130.72|59.61|0.00|596.16|745.20|804.81|1341.36|1400.97|-546.48| +2451603|78034|2451625|9164|30988|1132775|4162|44702|43458|158431|5616|24203|37|19|4|3|82|452|30|57.91|138.98|16.67|3669.30|500.10|1737.30|4169.40|10.00|0.00|375.00|500.10|510.10|875.10|885.10|-1237.20| +2451603|78034|2451612|6938|30988|1132775|4162|44702|43458|158431|5616|24203|11|23|11|5|189|452|8|98.07|124.54|94.65|239.12|757.20|784.56|996.32|53.00|0.00|219.12|757.20|810.20|976.32|1029.32|-27.36| +2452577|65276|2452630|17430|40168|963563|911|33906|67745|1076773|444|3439|12|19|11|2|223|453|88|85.39|226.28|167.44|5177.92|14734.72|7514.32|19912.64|132.61|10314.30|9159.04|4420.42|4553.03|13579.46|13712.07|-3093.90| +2452577|65276|2452593|6597|40168|963563|911|33906|67745|1076773|444|3439|27|9|16|4|165|453|14|24.77|43.34|42.47|12.18|594.58|346.78|606.76|41.62|0.00|291.20|594.58|636.20|885.78|927.40|247.80| +2452577|65276|2452587|17545|40168|963563|911|33906|67745|1076773|444|3439|39|30|10|5|240|453|64|46.92|85.39|12.80|4645.76|819.20|3002.88|5464.96|3.27|655.36|1748.48|163.84|167.11|1912.32|1915.59|-2839.04| +2452577|65276|2452616|10623|40168|963563|911|33906|67745|1076773|444|3439|9|24|9|1|123|453|99|25.39|54.08|42.72|1124.64|4229.28|2513.61|5353.92|211.46|0.00|2034.45|4229.28|4440.74|6263.73|6475.19|1715.67| +2452577|65276|2452594|9997|40168|963563|911|33906|67745|1076773|444|3439|21|6|15|2|129|453|5|74.41|110.12|78.18|159.70|390.90|372.05|550.60|23.45|0.00|242.25|390.90|414.35|633.15|656.60|18.85| +2452577|65276|2452584|1911|40168|963563|911|33906|67745|1076773|444|3439|55|7|7|3|299|453|39|52.28|74.76|69.52|204.36|2711.28|2038.92|2915.64|0.00|0.00|378.69|2711.28|2711.28|3089.97|3089.97|672.36| +2452577|65276|2452609|5241|40168|963563|911|33906|67745|1076773|444|3439|33|24|14|1|41|453|2|96.39|132.05|5.28|253.54|10.56|192.78|264.10|0.21|0.00|68.66|10.56|10.77|79.22|79.43|-182.22| +2452577|65276|2452612|7053|40168|963563|911|33906|67745|1076773|444|3439|9|9|15|5|65|453|80|56.24|89.98|81.88|648.00|6550.40|4499.20|7198.40|458.52|0.00|2231.20|6550.40|7008.92|8781.60|9240.12|2051.20| +2452577|65276|2452632|5173|40168|963563|911|33906|67745|1076773|444|3439|15|7|13|4|285|453|71|76.19|136.38|60.00|5422.98|4260.00|5409.49|9682.98|383.40|0.00|774.61|4260.00|4643.40|5034.61|5418.01|-1149.49| +2452577|65276|2452683|12072|40168|963563|911|33906|67745|1076773|444|3439|33|24|7|4|214|453|57|39.57|50.64|32.91|1010.61|1875.87|2255.49|2886.48|150.06|0.00|779.19|1875.87|2025.93|2655.06|2805.12|-379.62| +2452577|65276|2452665|3019|40168|963563|911|33906|67745|1076773|444|3439|9|9|15|5|245|453|12|79.57|144.02|72.01|864.12|864.12|954.84|1728.24|0.00|0.00|570.24|864.12|864.12|1434.36|1434.36|-90.72| +2452577|65276|2452595|6462|40168|963563|911|33906|67745|1076773|444|3439|18|6|7|3|52|453|61|65.16|131.62|9.21|7467.01|561.81|3974.76|8028.82|50.56|0.00|641.72|561.81|612.37|1203.53|1254.09|-3412.95| +2452577|65276|2452673|14239|40168|963563|911|33906|67745|1076773|444|3439|39|27|18|2|63|453|93|17.30|19.03|0.00|1769.79|0.00|1608.90|1769.79|0.00|0.00|619.38|0.00|0.00|619.38|619.38|-1608.90| +2452577|65276|2452664|5491|40168|963563|911|33906|67745|1076773|444|3439|31|27|2|3|271|453|31|70.35|71.75|65.29|200.26|2023.99|2180.85|2224.25|80.95|0.00|222.27|2023.99|2104.94|2246.26|2327.21|-156.86| +2452577|65276|2452599|12966|40168|963563|911|33906|67745|1076773|444|3439|24|3|2|2|122|453|50|40.23|42.64|22.17|1023.50|1108.50|2011.50|2132.00|44.34|0.00|660.50|1108.50|1152.84|1769.00|1813.34|-903.00| +2452577|65276|2452658|16573|40168|963563|911|33906|67745|1076773|444|3439|55|1|4|2|190|453|56|62.06|119.77|73.05|2616.32|4090.80|3475.36|6707.12|163.63|0.00|3218.88|4090.80|4254.43|7309.68|7473.31|615.44| +2451167|6492|2451275|9616|81039|1222509|3626|7096|34911|49972|5153|30623|1|28|2|1|200|454|60|3.33|6.52|2.80|223.20|168.00|199.80|391.20|6.72|0.00|140.40|168.00|174.72|308.40|315.12|-31.80| +2451167|6492|2451215|12373|81039|1222509|3626|7096|34911|49972|5153|30623|26|14|19|5|41|454|48|26.51|77.14|6.94|3369.60|333.12|1272.48|3702.72|26.64|0.00|1295.52|333.12|359.76|1628.64|1655.28|-939.36| +2451167|6492|2451179|14054|81039|1222509|3626|7096|34911|49972|5153|30623|44|25|13|3|50|454|5|82.21|136.46|17.73|593.65|88.65|411.05|682.30|4.43|0.00|204.65|88.65|93.08|293.30|297.73|-322.40| +2451167|6492|2451196|5500|81039|1222509|3626|7096|34911|49972|5153|30623|55|16|13|4|5|454|8|14.59|31.22|27.47|30.00|219.76|116.72|249.76|2.19|0.00|42.40|219.76|221.95|262.16|264.35|103.04| +2451167|6492|2451237|17266|81039|1222509|3626|7096|34911|49972|5153|30623|46|28|16|4|85|454|24|22.93|40.12|22.46|423.84|539.04|550.32|962.88|0.00|0.00|67.20|539.04|539.04|606.24|606.24|-11.28| +2451167|6492|2451244|12310|81039|1222509|3626|7096|34911|49972|5153|30623|1|20|14|2|290|454|13|92.65|226.99|158.89|885.30|2065.57|1204.45|2950.87|82.62|0.00|324.48|2065.57|2148.19|2390.05|2472.67|861.12| +2451167|6492|2451193|13160|81039|1222509|3626|7096|34911|49972|5153|30623|58|10|15|4|183|454|11|75.96|139.76|104.82|384.34|1153.02|835.56|1537.36|80.71|0.00|30.69|1153.02|1233.73|1183.71|1264.42|317.46| +2451167|6492|2451183|985|81039|1222509|3626|7096|34911|49972|5153|30623|49|13|12|4|186|454|40|66.35|108.15|89.76|735.60|3590.40|2654.00|4326.00|323.13|0.00|259.20|3590.40|3913.53|3849.60|4172.73|936.40| +2451167|6492|2451193|1765|81039|1222509|3626|7096|34911|49972|5153|30623|37|7|19|4|229|454|58|44.20|83.53|51.78|1841.50|3003.24|2563.60|4844.74|180.19|0.00|2034.64|3003.24|3183.43|5037.88|5218.07|439.64| +2451167|6492|2451286|17870|81039|1222509|3626|7096|34911|49972|5153|30623|14|7|7|3|186|454|22|10.87|25.10|15.06|220.88|331.32|239.14|552.20|9.93|0.00|82.72|331.32|341.25|414.04|423.97|92.18| +2451167|6492|2451228|13990|81039|1222509|3626|7096|34911|49972|5153|30623|38|26|5|1|70|454|55|80.92|134.32|114.17|1108.25|6279.35|4450.60|7387.60|125.58|0.00|886.05|6279.35|6404.93|7165.40|7290.98|1828.75| +2451167|6492|2451183|4466|81039|1222509|3626|7096|34911|49972|5153|30623|50|4|20|2|287|454|55|14.73|31.08|25.17|325.05|1384.35|810.15|1709.40|1.66|1328.97|803.00|55.38|57.04|858.38|860.04|-754.77| +2451167|6492|2451234|7999|81039|1222509|3626|7096|34911|49972|5153|30623|20|20|4|2|80|454|34|66.93|157.28|81.78|2567.00|2780.52|2275.62|5347.52|172.67|861.96|481.10|1918.56|2091.23|2399.66|2572.33|-357.06| +2451167|6492|2451192|7358|81039|1222509|3626|7096|34911|49972|5153|30623|16|10|3|5|207|454|88|62.31|130.85|78.51|4605.92|6908.88|5483.28|11514.80|345.44|0.00|3108.16|6908.88|7254.32|10017.04|10362.48|1425.60| +2451167|6492|2451272|9877|81039|1222509|3626|7096|34911|49972|5153|30623|55|22|6|5|294|454|71|85.67|172.19|67.15|7457.84|4767.65|6082.57|12225.49|429.08|0.00|3055.84|4767.65|5196.73|7823.49|8252.57|-1314.92| +2451167|6492|2451186|11149|81039|1222509|3626|7096|34911|49972|5153|30623|56|10|3|5|177|454|24|55.86|156.96|42.37|2750.16|1016.88|1340.64|3767.04|81.35|0.00|263.52|1016.88|1098.23|1280.40|1361.75|-323.76| +2451528|56351|2451548|1694|39512|1516890|4221|40799|14303|360380|486|37064|40|14|8|3|165|455|73|24.18|64.31|20.57|3193.02|1501.61|1765.14|4694.63|15.01|0.00|2065.17|1501.61|1516.62|3566.78|3581.79|-263.53| +2451528|56351|2451569|17452|39512|1516890|4221|40799|14303|360380|486|37064|28|16|6|3|210|455|57|86.66|139.52|15.34|7078.26|874.38|4939.62|7952.64|0.00|0.00|1669.53|874.38|874.38|2543.91|2543.91|-4065.24| +2451528|56351|2451582|7171|39512|1516890|4221|40799|14303|360380|486|37064|32|26|11|3|245|455|96|79.98|169.55|144.11|2442.24|13834.56|7678.08|16276.80|1245.11|0.00|650.88|13834.56|15079.67|14485.44|15730.55|6156.48| +2451528|56351|2451632|424|39512|1516890|4221|40799|14303|360380|486|37064|16|28|14|3|49|455|15|46.93|60.53|16.94|653.85|254.10|703.95|907.95|10.06|142.29|281.40|111.81|121.87|393.21|403.27|-592.14| +2451528|56351|2451594|400|39512|1516890|4221|40799|14303|360380|486|37064|4|8|17|4|281|455|93|48.54|138.33|1.38|12736.35|128.34|4514.22|12864.69|3.85|0.00|3086.67|128.34|132.19|3215.01|3218.86|-4385.88| +2451528|56351|2451553|10100|39512|1516890|4221|40799|14303|360380|486|37064|56|20|6|1|116|455|9|46.47|85.96|76.50|85.14|688.50|418.23|773.64|6.88|0.00|324.90|688.50|695.38|1013.40|1020.28|270.27| +2451528|56351|2451570|14590|39512|1516890|4221|40799|14303|360380|486|37064|50|22|11|5|196|455|61|4.43|11.29|7.56|227.53|461.16|270.23|688.69|41.50|0.00|89.06|461.16|502.66|550.22|591.72|190.93| +2451528|56351|2451624|3139|39512|1516890|4221|40799|14303|360380|486|37064|46|20|8|2|279|455|9|72.81|78.63|71.55|63.72|643.95|655.29|707.67|0.00|0.00|169.83|643.95|643.95|813.78|813.78|-11.34| +2451528|56351|2451546|2323|39512|1516890|4221|40799|14303|360380|486|37064|7|25|12|1|130|455|42|26.13|71.85|31.61|1690.08|1327.62|1097.46|3017.70|38.23|849.67|1478.40|477.95|516.18|1956.35|1994.58|-619.51| +2451528|56351|2451608|8737|39512|1516890|4221|40799|14303|360380|486|37064|37|8|8|1|277|455|16|92.24|268.41|147.62|1932.64|2361.92|1475.84|4294.56|23.61|0.00|1717.76|2361.92|2385.53|4079.68|4103.29|886.08| +2452071|37417|2452136|11309|10452|234543|587|5078|83097|1553646|6457|42148|31|23|5|1|217|456|95|71.86|170.30|126.02|4206.60|11971.90|6826.70|16178.50|478.87|0.00|7603.80|11971.90|12450.77|19575.70|20054.57|5145.20| +2452071|37417|2452143|2363|10452|234543|587|5078|83097|1553646|6457|42148|35|27|10|1|152|456|25|15.88|47.64|46.68|24.00|1167.00|397.00|1191.00|70.02|0.00|297.75|1167.00|1237.02|1464.75|1534.77|770.00| +2452071|37417|2452089|1415|10452|234543|587|5078|83097|1553646|6457|42148|41|25|15|2|199|456|63|12.19|13.53|9.06|281.61|570.78|767.97|852.39|17.12|0.00|391.86|570.78|587.90|962.64|979.76|-197.19| +2452071|37417|2452160|7901|10452|234543|587|5078|83097|1553646|6457|42148|21|23|6|2|188|456|4|40.41|56.97|0.00|227.88|0.00|161.64|227.88|0.00|0.00|34.16|0.00|0.00|34.16|34.16|-161.64| +2452071|37417|2452082|2125|10452|234543|587|5078|83097|1553646|6457|42148|7|25|12|4|63|456|37|75.22|200.83|150.62|1857.77|5572.94|2783.14|7430.71|0.00|0.00|3492.43|5572.94|5572.94|9065.37|9065.37|2789.80| +2452071|37417|2452156|14123|10452|234543|587|5078|83097|1553646|6457|42148|25|5|13|5|220|456|38|30.25|79.55|63.64|604.58|2418.32|1149.50|3022.90|217.64|0.00|1148.36|2418.32|2635.96|3566.68|3784.32|1268.82| +2452071|37417|2452115|6785|10452|234543|587|5078|83097|1553646|6457|42148|47|13|20|3|16|456|79|80.68|138.76|37.46|8002.70|2959.34|6373.72|10962.04|59.18|0.00|5481.02|2959.34|3018.52|8440.36|8499.54|-3414.38| +2452071|37417|2452160|1299|10452|234543|587|5078|83097|1553646|6457|42148|1|17|14|4|167|456|27|83.64|142.18|112.32|806.22|3032.64|2258.28|3838.86|60.65|0.00|652.59|3032.64|3093.29|3685.23|3745.88|774.36| +2452071|37417|2452162|15175|10452|234543|587|5078|83097|1553646|6457|42148|35|25|5|5|245|456|90|29.32|46.61|43.34|294.30|3900.60|2638.80|4194.90|273.04|0.00|838.80|3900.60|4173.64|4739.40|5012.44|1261.80| +2452071|37417|2452143|4125|10452|234543|587|5078|83097|1553646|6457|42148|49|23|9|4|86|456|32|85.74|157.76|137.25|656.32|4392.00|2743.68|5048.32|87.84|0.00|656.00|4392.00|4479.84|5048.00|5135.84|1648.32| +2452071|37417|2452156|337|10452|234543|587|5078|83097|1553646|6457|42148|51|5|10|2|241|456|51|40.29|67.28|66.60|34.68|3396.60|2054.79|3431.28|67.93|0.00|1509.60|3396.60|3464.53|4906.20|4974.13|1341.81| +2452071|37417|2452142|2069|10452|234543|587|5078|83097|1553646|6457|42148|27|23|20|3|96|456|6|31.94|51.42|10.28|246.84|61.68|191.64|308.52|4.93|0.00|58.56|61.68|66.61|120.24|125.17|-129.96| +2452071|37417|2452100|3635|10452|234543|587|5078|83097|1553646|6457|42148|11|29|4|3|282|456|23|92.08|215.46|185.29|693.91|4261.67|2117.84|4955.58|170.46|0.00|792.81|4261.67|4432.13|5054.48|5224.94|2143.83| +2452071|37417|2452115|14995|10452|234543|587|5078|83097|1553646|6457|42148|55|1|14|4|287|456|8|76.23|133.40|110.72|181.44|885.76|609.84|1067.20|0.00|0.00|10.64|885.76|885.76|896.40|896.40|275.92| +2452071|37417|2452176|15129|10452|234543|587|5078|83097|1553646|6457|42148|5|29|20|5|277|456|96|63.83|79.14|36.40|4103.04|3494.40|6127.68|7597.44|174.72|0.00|2354.88|3494.40|3669.12|5849.28|6024.00|-2633.28| +2451816|80582|2451828|13019|36168|119191|1767|46343|68969|364732|5765|32341|23|29|8|4|77|457|36|51.44|99.79|59.87|1437.12|2155.32|1851.84|3592.44|0.00|1530.27|610.56|625.05|625.05|1235.61|1235.61|-1226.79| +2451816|80582|2451826|6725|36168|119191|1767|46343|68969|364732|5765|32341|31|5|20|2|54|457|90|57.87|108.21|70.33|3409.20|6329.70|5208.30|9738.90|443.07|0.00|2434.50|6329.70|6772.77|8764.20|9207.27|1121.40| +2451816|80582|2451876|10499|36168|119191|1767|46343|68969|364732|5765|32341|32|23|14|4|184|457|61|45.18|62.80|1.88|3716.12|114.68|2755.98|3830.80|6.88|0.00|0.00|114.68|121.56|114.68|121.56|-2641.30| +2451816|80582|2451833|13385|36168|119191|1767|46343|68969|364732|5765|32341|44|23|8|4|300|457|79|51.30|122.60|89.49|2615.69|7069.71|4052.70|9685.40|212.09|0.00|580.65|7069.71|7281.80|7650.36|7862.45|3017.01| +2451816|80582|2451921|5201|36168|119191|1767|46343|68969|364732|5765|32341|35|19|2|4|286|457|38|66.81|68.14|60.64|285.00|2304.32|2538.78|2589.32|0.00|391.73|77.52|1912.59|1912.59|1990.11|1990.11|-626.19| +2451816|80582|2451856|5443|36168|119191|1767|46343|68969|364732|5765|32341|43|20|20|2|271|457|44|47.55|74.65|9.70|2857.80|426.80|2092.20|3284.60|25.60|0.00|656.92|426.80|452.40|1083.72|1109.32|-1665.40| +2451816|80582|2451919|17639|36168|119191|1767|46343|68969|364732|5765|32341|8|29|12|4|149|457|21|2.78|8.20|1.06|149.94|22.26|58.38|172.20|0.57|8.01|8.61|14.25|14.82|22.86|23.43|-44.13| +2451816|80582|2451907|8791|36168|119191|1767|46343|68969|364732|5765|32341|43|19|15|3|161|457|21|83.37|130.05|104.04|546.21|2184.84|1750.77|2731.05|0.00|0.00|1147.02|2184.84|2184.84|3331.86|3331.86|434.07| +2451816|80582|2451933|13981|36168|119191|1767|46343|68969|364732|5765|32341|11|29|4|3|132|457|19|48.54|95.13|61.83|632.70|1174.77|922.26|1807.47|0.00|0.00|578.36|1174.77|1174.77|1753.13|1753.13|252.51| +2451816|80582|2451831|16586|36168|119191|1767|46343|68969|364732|5765|32341|44|17|2|3|94|457|95|86.94|220.82|52.99|15943.85|5034.05|8259.30|20977.90|112.76|3624.51|9020.25|1409.54|1522.30|10429.79|10542.55|-6849.76| +2451816|80582|2451899|10358|36168|119191|1767|46343|68969|364732|5765|32341|7|5|12|3|217|457|93|62.98|118.40|26.04|8589.48|2421.72|5857.14|11011.20|0.00|0.00|1541.01|2421.72|2421.72|3962.73|3962.73|-3435.42| +2451816|80582|2451829|4436|36168|119191|1767|46343|68969|364732|5765|32341|19|5|16|4|132|457|43|61.15|128.41|119.42|386.57|5135.06|2629.45|5521.63|61.62|4364.80|1600.89|770.26|831.88|2371.15|2432.77|-1859.19| +2451816|80582|2451855|3698|36168|119191|1767|46343|68969|364732|5765|32341|5|1|9|4|152|457|56|19.87|30.59|24.77|325.92|1387.12|1112.72|1713.04|27.74|0.00|804.72|1387.12|1414.86|2191.84|2219.58|274.40| +2451816|80582|2451833|10208|36168|119191|1767|46343|68969|364732|5765|32341|43|2|2|5|235|457|7|19.86|52.23|32.90|135.31|230.30|139.02|365.61|0.00|0.00|36.54|230.30|230.30|266.84|266.84|91.28| +2452525|62319|2452569|546|52054|776510|3366|46622|4482|1400841|1990|22399|15|24|13|2|271|458|58|53.49|150.84|98.04|3062.40|5686.32|3102.42|8748.72|0.00|625.49|2886.66|5060.83|5060.83|7947.49|7947.49|1958.41| +2452525|62319|2452544|17505|52054|776510|3366|46622|4482|1400841|1990|22399|37|7|19|4|169|458|50|41.74|58.43|18.11|2016.00|905.50|2087.00|2921.50|63.38|0.00|1460.50|905.50|968.88|2366.00|2429.38|-1181.50| +2452525|62319|2452632|1746|52054|776510|3366|46622|4482|1400841|1990|22399|3|6|1|2|110|458|90|53.46|151.82|15.18|12297.60|1366.20|4811.40|13663.80|57.92|642.11|5738.40|724.09|782.01|6462.49|6520.41|-4087.31| +2452525|62319|2452529|1449|52054|776510|3366|46622|4482|1400841|1990|22399|31|7|9|2|132|458|25|97.02|144.55|119.97|614.50|2999.25|2425.50|3613.75|29.99|0.00|1337.00|2999.25|3029.24|4336.25|4366.24|573.75| +2452525|62319|2452544|12595|52054|776510|3366|46622|4482|1400841|1990|22399|43|27|16|3|230|458|95|9.98|21.15|8.88|1165.65|843.60|948.10|2009.25|16.87|0.00|823.65|843.60|860.47|1667.25|1684.12|-104.50| +2452525|62319|2452636|1027|52054|776510|3366|46622|4482|1400841|1990|22399|21|1|11|3|248|458|35|37.46|93.65|91.77|65.80|3211.95|1311.10|3277.75|8.67|3115.59|1081.50|96.36|105.03|1177.86|1186.53|-1214.74| +2452525|62319|2452619|12792|52054|776510|3366|46622|4482|1400841|1990|22399|24|1|17|1|252|458|46|28.07|40.42|29.10|520.72|1338.60|1291.22|1859.32|13.38|0.00|148.58|1338.60|1351.98|1487.18|1500.56|47.38| +2452525|62319|2452597|9253|52054|776510|3366|46622|4482|1400841|1990|22399|48|18|1|3|278|458|3|72.64|92.97|73.44|58.59|220.32|217.92|278.91|4.40|0.00|66.93|220.32|224.72|287.25|291.65|2.40| +2452525|62319|2452642|267|52054|776510|3366|46622|4482|1400841|1990|22399|24|25|11|2|173|458|38|16.08|34.08|12.95|802.94|492.10|611.04|1295.04|44.28|0.00|38.76|492.10|536.38|530.86|575.14|-118.94| +2452525|62319|2452597|17730|52054|776510|3366|46622|4482|1400841|1990|22399|42|15|12|1|43|458|11|1.00|2.99|2.57|4.62|28.27|11.00|32.89|0.00|0.00|7.15|28.27|28.27|35.42|35.42|17.27| +2452525|62319|2452633|2935|52054|776510|3366|46622|4482|1400841|1990|22399|37|24|8|4|7|458|66|6.03|9.82|5.30|298.32|349.80|397.98|648.12|0.94|339.30|194.04|10.50|11.44|204.54|205.48|-387.48| +2451502|45748|2451557|10802|64936|1182080|6777|21521|17089|932679|1306|24850|28|1|4|2|220|459|24|58.62|143.03|50.06|2231.28|1201.44|1406.88|3432.72|108.12|0.00|1407.36|1201.44|1309.56|2608.80|2716.92|-205.44| +2451502|45748|2451616|9470|64936|1182080|6777|21521|17089|932679|1306|24850|26|19|6|2|268|459|78|15.68|27.59|22.07|430.56|1721.46|1223.04|2152.02|154.93|0.00|301.08|1721.46|1876.39|2022.54|2177.47|498.42| +2451502|45748|2451588|4861|64936|1182080|6777|21521|17089|932679|1306|24850|55|13|19|3|110|459|36|74.40|187.48|5.62|6546.96|202.32|2678.40|6749.28|18.20|0.00|1821.96|202.32|220.52|2024.28|2042.48|-2476.08| +2451502|45748|2451619|76|64936|1182080|6777|21521|17089|932679|1306|24850|14|19|9|1|233|459|96|35.39|54.85|7.13|4581.12|684.48|3397.44|5265.60|20.53|0.00|789.12|684.48|705.01|1473.60|1494.13|-2712.96| +2451502|45748|2451503|6574|64936|1182080|6777|21521|17089|932679|1306|24850|31|16|15|3|214|459|9|59.56|154.26|3.08|1360.62|27.72|536.04|1388.34|0.55|0.00|194.31|27.72|28.27|222.03|222.58|-508.32| +2451502|45748|2451581|3086|64936|1182080|6777|21521|17089|932679|1306|24850|8|7|10|1|225|459|55|78.04|135.00|101.25|1856.25|5568.75|4292.20|7425.00|501.18|0.00|148.50|5568.75|6069.93|5717.25|6218.43|1276.55| +2451502|45748|2451519|8059|64936|1182080|6777|21521|17089|932679|1306|24850|58|14|19|2|175|459|88|53.79|139.31|136.52|245.52|12013.76|4733.52|12259.28|720.82|0.00|2819.52|12013.76|12734.58|14833.28|15554.10|7280.24| +2451502|45748|2451531|505|64936|1182080|6777|21521|17089|932679|1306|24850|37|16|9|4|279|459|81|20.55|56.30|25.33|2508.57|2051.73|1664.55|4560.30|82.06|0.00|2233.98|2051.73|2133.79|4285.71|4367.77|387.18| +2451502|45748|2451598|14878|64936|1182080|6777|21521|17089|932679|1306|24850|40|26|10|3|129|459|77|99.92|290.76|110.48|13881.56|8506.96|7693.84|22388.52|595.48|0.00|8506.96|8506.96|9102.44|17013.92|17609.40|813.12| +2451502|45748|2451545|15283|64936|1182080|6777|21521|17089|932679|1306|24850|8|25|4|2|173|459|78|85.67|248.44|126.70|9495.72|9882.60|6682.26|19378.32|0.00|8597.86|8138.52|1284.74|1284.74|9423.26|9423.26|-5397.52| +2451502|45748|2451512|3985|64936|1182080|6777|21521|17089|932679|1306|24850|44|19|7|4|174|459|88|52.26|122.28|89.26|2905.76|7854.88|4598.88|10760.64|314.19|0.00|3550.80|7854.88|8169.07|11405.68|11719.87|3256.00| +2451502|45748|2451505|16010|64936|1182080|6777|21521|17089|932679|1306|24850|44|7|15|1|247|459|22|1.75|3.65|1.82|40.26|40.04|38.50|80.30|0.40|0.00|0.66|40.04|40.44|40.70|41.10|1.54| +2451502|45748|2451583|10123|64936|1182080|6777|21521|17089|932679|1306|24850|20|7|17|5|163|459|63|31.65|44.94|31.00|878.22|1953.00|1993.95|2831.22|117.18|0.00|905.94|1953.00|2070.18|2858.94|2976.12|-40.95| +2451502|45748|2451621|1714|64936|1182080|6777|21521|17089|932679|1306|24850|26|28|6|4|145|459|20|34.50|100.74|86.63|282.20|1732.60|690.00|2014.80|69.30|0.00|60.40|1732.60|1801.90|1793.00|1862.30|1042.60| +2451502|45748|2451581|1288|64936|1182080|6777|21521|17089|932679|1306|24850|52|25|6|2|128|459|22|74.64|174.65|120.50|1191.30|2651.00|1642.08|3842.30|106.04|0.00|384.12|2651.00|2757.04|3035.12|3141.16|1008.92| +2452497|17370|2452558|6225|52235|267192|902|8659|18828|864217|5436|44793|21|24|13|4|135|460|62|47.72|69.19|65.03|257.92|4031.86|2958.64|4289.78|241.91|0.00|1844.50|4031.86|4273.77|5876.36|6118.27|1073.22| +2452497|17370|2452501|6987|52235|267192|902|8659|18828|864217|5436|44793|21|3|3|4|125|460|48|22.24|24.01|0.72|1117.92|34.56|1067.52|1152.48|0.00|0.00|149.76|34.56|34.56|184.32|184.32|-1032.96| +2452497|17370|2452615|11436|52235|267192|902|8659|18828|864217|5436|44793|60|3|11|5|202|460|85|8.05|17.22|13.60|307.70|1156.00|684.25|1463.70|0.00|0.00|629.00|1156.00|1156.00|1785.00|1785.00|471.75| +2452497|17370|2452526|7419|52235|267192|902|8659|18828|864217|5436|44793|25|7|2|3|254|460|95|31.79|84.24|53.91|2881.35|5121.45|3020.05|8002.80|358.50|0.00|3681.25|5121.45|5479.95|8802.70|9161.20|2101.40| +2452497|17370|2452559|9234|52235|267192|902|8659|18828|864217|5436|44793|43|30|18|1|144|460|55|63.13|112.37|102.25|556.60|5623.75|3472.15|6180.35|175.46|1237.22|2410.10|4386.53|4561.99|6796.63|6972.09|914.38| +2452497|17370|2452608|13327|52235|267192|902|8659|18828|864217|5436|44793|54|3|17|2|10|460|45|33.75|50.96|13.75|1674.45|618.75|1518.75|2293.20|8.04|216.56|206.10|402.19|410.23|608.29|616.33|-1116.56| +2452497|17370|2452572|5881|52235|267192|902|8659|18828|864217|5436|44793|30|6|11|4|50|460|67|16.90|38.36|31.45|462.97|2107.15|1132.30|2570.12|37.08|1643.57|745.04|463.58|500.66|1208.62|1245.70|-668.72| +2452497|17370|2452567|3465|52235|267192|902|8659|18828|864217|5436|44793|27|19|16|4|155|460|96|53.34|145.61|34.94|10624.32|3354.24|5120.64|13978.56|234.79|0.00|279.36|3354.24|3589.03|3633.60|3868.39|-1766.40| +2452497|17370|2452505|7290|52235|267192|902|8659|18828|864217|5436|44793|15|27|20|1|62|460|92|31.06|50.00|21.50|2622.00|1978.00|2857.52|4600.00|19.78|0.00|1748.00|1978.00|1997.78|3726.00|3745.78|-879.52| +2452497|17370|2452594|9801|52235|267192|902|8659|18828|864217|5436|44793|45|9|7|4|184|460|86|74.08|205.20|90.28|9883.12|7764.08|6370.88|17647.20|77.64|0.00|4411.80|7764.08|7841.72|12175.88|12253.52|1393.20| +2452497|17370|2452533|7959|52235|267192|902|8659|18828|864217|5436|44793|7|27|7|2|210|460|5|82.34|155.62|105.82|249.00|529.10|411.70|778.10|10.58|0.00|171.15|529.10|539.68|700.25|710.83|117.40| +2452497|17370|2452597|4401|52235|267192|902|8659|18828|864217|5436|44793|13|12|15|4|213|460|53|37.90|68.22|2.04|3507.54|108.12|2008.70|3615.66|3.24|0.00|758.96|108.12|111.36|867.08|870.32|-1900.58| +2452497|17370|2452587|16524|52235|267192|902|8659|18828|864217|5436|44793|60|30|10|5|180|460|51|43.90|86.48|0.86|4366.62|43.86|2238.90|4410.48|0.00|0.00|1190.34|43.86|43.86|1234.20|1234.20|-2195.04| +2452492|74886|2452559|10560|18961|449632|6350|19421|32609|1400410|3528|25799|45|3|1|5|149|461|22|34.61|101.06|6.06|2090.00|133.32|761.42|2223.32|3.35|21.33|22.22|111.99|115.34|134.21|137.56|-649.43| +2452492|74886|2452496|1717|18961|449632|6350|19421|32609|1400410|3528|25799|60|24|11|5|2|461|5|85.04|112.25|4.49|538.80|22.45|425.20|561.25|1.34|0.00|140.30|22.45|23.79|162.75|164.09|-402.75| +2452492|74886|2452496|4945|18961|449632|6350|19421|32609|1400410|3528|25799|45|21|16|1|199|461|30|94.24|172.45|96.57|2276.40|2897.10|2827.20|5173.50|86.91|0.00|1914.00|2897.10|2984.01|4811.10|4898.01|69.90| +2452492|74886|2452509|11221|18961|449632|6350|19421|32609|1400410|3528|25799|31|30|4|1|275|461|36|60.42|72.50|38.42|1226.88|1383.12|2175.12|2610.00|82.98|0.00|1200.60|1383.12|1466.10|2583.72|2666.70|-792.00| +2452492|74886|2452552|10506|18961|449632|6350|19421|32609|1400410|3528|25799|18|15|4|3|229|461|86|40.33|118.57|11.85|9177.92|1019.10|3468.38|10197.02|71.33|0.00|4894.26|1019.10|1090.43|5913.36|5984.69|-2449.28| +2452492|74886|2452600|10965|18961|449632|6350|19421|32609|1400410|3528|25799|18|6|4|3|142|461|23|58.27|58.27|29.13|670.22|669.99|1340.21|1340.21|0.00|0.00|0.00|669.99|669.99|669.99|669.99|-670.22| +2452492|74886|2452502|4077|18961|449632|6350|19421|32609|1400410|3528|25799|36|1|1|4|107|461|1|85.96|201.14|104.59|96.55|104.59|85.96|201.14|2.09|0.00|92.52|104.59|106.68|197.11|199.20|18.63| +2452492|74886|2452568|8562|18961|449632|6350|19421|32609|1400410|3528|25799|60|24|14|3|55|461|26|69.71|167.30|48.51|3088.54|1261.26|1812.46|4349.80|63.06|0.00|2174.90|1261.26|1324.32|3436.16|3499.22|-551.20| +2452492|74886|2452571|1500|18961|449632|6350|19421|32609|1400410|3528|25799|31|18|9|2|85|461|90|23.27|43.98|37.38|594.00|3364.20|2094.30|3958.20|33.64|0.00|1187.10|3364.20|3397.84|4551.30|4584.94|1269.90| +2452582|857|2452619|4557|5646|1800128|2163|44198|16373|1063535|350|18032|1|3|3|1|42|462|24|34.86|91.33|47.49|1052.16|1139.76|836.64|2191.92|11.39|0.00|109.44|1139.76|1151.15|1249.20|1260.59|303.12| +2452582|857|2452682|3186|5646|1800128|2163|44198|16373|1063535|350|18032|51|27|2|5|218|462|59|78.45|149.05|76.01|4309.36|4484.59|4628.55|8793.95|358.76|0.00|3077.44|4484.59|4843.35|7562.03|7920.79|-143.96| +2452582|857|2452600|8628|5646|1800128|2163|44198|16373|1063535|350|18032|45|27|12|3|241|462|36|25.93|28.26|6.78|773.28|244.08|933.48|1017.36|14.64|0.00|315.36|244.08|258.72|559.44|574.08|-689.40| +2452582|857|2452688|1687|5646|1800128|2163|44198|16373|1063535|350|18032|1|18|20|3|109|462|21|91.50|193.06|152.51|851.55|3202.71|1921.50|4054.26|64.05|0.00|1175.58|3202.71|3266.76|4378.29|4442.34|1281.21| +2452582|857|2452628|882|5646|1800128|2163|44198|16373|1063535|350|18032|6|1|16|4|62|462|17|75.23|160.99|0.00|2736.83|0.00|1278.91|2736.83|0.00|0.00|656.71|0.00|0.00|656.71|656.71|-1278.91| +2452582|857|2452664|12294|5646|1800128|2163|44198|16373|1063535|350|18032|42|7|14|4|117|462|98|3.87|4.29|1.63|260.68|159.74|379.26|420.42|14.37|0.00|171.50|159.74|174.11|331.24|345.61|-219.52| +2452582|857|2452692|4050|5646|1800128|2163|44198|16373|1063535|350|18032|21|6|18|3|85|462|17|89.98|170.06|68.02|1734.68|1156.34|1529.66|2891.02|34.69|0.00|491.47|1156.34|1191.03|1647.81|1682.50|-373.32| +2452582|857|2452692|15349|5646|1800128|2163|44198|16373|1063535|350|18032|48|15|12|3|57|462|30|69.58|149.59|46.37|3096.60|1391.10|2087.40|4487.70|7.51|1307.63|448.50|83.47|90.98|531.97|539.48|-2003.93| +2452582|857|2452684|1560|5646|1800128|2163|44198|16373|1063535|350|18032|7|30|19|5|251|462|55|18.89|50.62|44.54|334.40|2449.70|1038.95|2784.10|24.49|0.00|1002.10|2449.70|2474.19|3451.80|3476.29|1410.75| +2452582|857|2452651|8136|5646|1800128|2163|44198|16373|1063535|350|18032|60|18|5|4|200|462|2|24.70|61.00|37.21|47.58|74.42|49.40|122.00|1.89|11.16|12.20|63.26|65.15|75.46|77.35|13.86| +2452582|857|2452684|5437|5646|1800128|2163|44198|16373|1063535|350|18032|57|30|1|2|183|462|85|11.51|15.99|3.99|1020.00|339.15|978.35|1359.15|0.00|0.00|665.55|339.15|339.15|1004.70|1004.70|-639.20| +2452582|857|2452613|9|5646|1800128|2163|44198|16373|1063535|350|18032|42|9|19|3|257|462|47|30.81|85.03|85.03|0.00|3996.41|1448.07|3996.41|39.96|0.00|439.45|3996.41|4036.37|4435.86|4475.82|2548.34| +2452582|857|2452612|14425|5646|1800128|2163|44198|16373|1063535|350|18032|19|24|17|2|57|462|14|64.04|79.40|67.49|166.74|944.86|896.56|1111.60|26.45|283.45|322.28|661.41|687.86|983.69|1010.14|-235.15| +2452582|857|2452697|12229|5646|1800128|2163|44198|16373|1063535|350|18032|54|30|3|4|242|462|65|41.62|75.33|71.56|245.05|4651.40|2705.30|4896.45|93.02|0.00|244.40|4651.40|4744.42|4895.80|4988.82|1946.10| +2452582|857|2452622|5538|5646|1800128|2163|44198|16373|1063535|350|18032|19|1|12|3|154|462|73|46.38|56.58|12.44|3222.22|908.12|3385.74|4130.34|27.24|0.00|1527.89|908.12|935.36|2436.01|2463.25|-2477.62| +2452582|857|2452679|13243|5646|1800128|2163|44198|16373|1063535|350|18032|12|19|1|2|126|462|77|75.61|130.80|90.25|3122.35|6949.25|5821.97|10071.60|69.49|0.00|4934.93|6949.25|7018.74|11884.18|11953.67|1127.28| +2451190|75271|2451229|2782|79530|115751|2725|27699|5677|146285|4668|14042|22|7|16|3|280|463|83|42.24|74.34|1.48|6047.38|122.84|3505.92|6170.22|6.14|0.00|2097.41|122.84|128.98|2220.25|2226.39|-3383.08| +2451190|75271|2451204|2245|79530|115751|2725|27699|5677|146285|4668|14042|44|2|1|3|262|463|1|59.16|91.10|0.91|90.19|0.91|59.16|91.10|0.02|0.00|12.75|0.91|0.93|13.66|13.68|-58.25| +2451190|75271|2451241|5383|79530|115751|2725|27699|5677|146285|4668|14042|37|1|3|2|104|463|92|62.39|152.85|35.15|10828.40|3233.80|5739.88|14062.20|97.01|0.00|3515.32|3233.80|3330.81|6749.12|6846.13|-2506.08| +2451190|75271|2451191|9034|79530|115751|2725|27699|5677|146285|4668|14042|32|13|4|1|88|463|64|49.37|109.10|85.09|1536.64|5445.76|3159.68|6982.40|108.91|0.00|3420.80|5445.76|5554.67|8866.56|8975.47|2286.08| +2451190|75271|2451263|3332|79530|115751|2725|27699|5677|146285|4668|14042|43|19|8|4|207|463|99|79.47|96.15|40.38|5521.23|3997.62|7867.53|9518.85|79.95|0.00|570.24|3997.62|4077.57|4567.86|4647.81|-3869.91| +2451190|75271|2451194|17450|79530|115751|2725|27699|5677|146285|4668|14042|37|14|14|3|19|463|25|2.87|7.20|0.72|162.00|18.00|71.75|180.00|0.36|0.00|37.75|18.00|18.36|55.75|56.11|-53.75| +2451190|75271|2451205|14659|79530|115751|2725|27699|5677|146285|4668|14042|56|10|16|1|283|463|92|41.64|49.55|46.57|274.16|4284.44|3830.88|4558.60|128.53|0.00|865.72|4284.44|4412.97|5150.16|5278.69|453.56| +2451190|75271|2451307|14684|79530|115751|2725|27699|5677|146285|4668|14042|19|22|14|1|168|463|33|3.65|6.46|3.29|104.61|108.57|120.45|213.18|0.00|0.00|59.40|108.57|108.57|167.97|167.97|-11.88| +2451190|75271|2451192|8096|79530|115751|2725|27699|5677|146285|4668|14042|4|1|6|4|272|463|78|59.32|140.58|108.24|2522.52|8442.72|4626.96|10965.24|422.13|0.00|1973.40|8442.72|8864.85|10416.12|10838.25|3815.76| +2451190|75271|2451194|1850|79530|115751|2725|27699|5677|146285|4668|14042|19|10|2|3|137|463|77|38.68|81.61|37.54|3393.39|2890.58|2978.36|6283.97|173.43|0.00|2764.30|2890.58|3064.01|5654.88|5828.31|-87.78| +2451190|75271|2451240|15031|79530|115751|2725|27699|5677|146285|4668|14042|52|19|16|2|88|463|68|32.30|33.26|7.98|1719.04|542.64|2196.40|2261.68|37.98|0.00|429.08|542.64|580.62|971.72|1009.70|-1653.76| +2451190|75271|2451281|17302|79530|115751|2725|27699|5677|146285|4668|14042|34|1|14|2|250|463|79|67.55|108.75|59.81|3866.26|4724.99|5336.45|8591.25|330.74|0.00|4209.12|4724.99|5055.73|8934.11|9264.85|-611.46| +2451190|75271|2451305|16678|79530|115751|2725|27699|5677|146285|4668|14042|19|2|1|3|262|463|75|81.57|229.21|2.29|17019.00|171.75|6117.75|17190.75|4.63|94.46|7907.25|77.29|81.92|7984.54|7989.17|-6040.46| +2451190|75271|2451280|9026|79530|115751|2725|27699|5677|146285|4668|14042|19|14|6|1|219|463|47|61.83|171.26|137.00|1610.22|6439.00|2906.01|8049.22|450.73|0.00|2897.55|6439.00|6889.73|9336.55|9787.28|3532.99| +2451190|75271|2451254|376|79530|115751|2725|27699|5677|146285|4668|14042|28|10|20|4|115|463|33|5.98|10.28|0.41|325.71|13.53|197.34|339.24|0.82|1.75|67.65|11.78|12.60|79.43|80.25|-185.56| +2452627|45828|2452716|1761|88868|911334|3400|6266|69772|1136260|1711|29777|55|30|12|4|239|464|49|68.06|149.73|124.27|1247.54|6089.23|3334.94|7336.77|60.89|0.00|3594.64|6089.23|6150.12|9683.87|9744.76|2754.29| +2452627|45828|2452655|10551|88868|911334|3400|6266|69772|1136260|1711|29777|27|3|3|1|229|464|25|12.58|14.46|10.55|97.75|263.75|314.50|361.50|21.10|0.00|155.25|263.75|284.85|419.00|440.10|-50.75| +2452627|45828|2452668|6721|88868|911334|3400|6266|69772|1136260|1711|29777|13|21|8|2|196|464|15|81.83|198.84|155.09|656.25|2326.35|1227.45|2982.60|46.52|0.00|864.90|2326.35|2372.87|3191.25|3237.77|1098.90| +2452627|45828|2452651|9139|88868|911334|3400|6266|69772|1136260|1711|29777|13|30|13|1|219|464|26|34.85|61.33|58.87|63.96|1530.62|906.10|1594.58|0.00|734.69|127.40|795.93|795.93|923.33|923.33|-110.17| +2452627|45828|2452683|7218|88868|911334|3400|6266|69772|1136260|1711|29777|37|25|17|1|295|464|78|60.43|167.99|131.03|2882.88|10220.34|4713.54|13103.22|511.01|0.00|6158.10|10220.34|10731.35|16378.44|16889.45|5506.80| +2452627|45828|2452669|13479|88868|911334|3400|6266|69772|1136260|1711|29777|49|6|11|2|200|464|15|98.96|222.66|104.65|1770.15|1569.75|1484.40|3339.90|0.00|0.00|166.95|1569.75|1569.75|1736.70|1736.70|85.35| +2452627|45828|2452680|14205|88868|911334|3400|6266|69772|1136260|1711|29777|13|13|14|4|134|464|36|46.74|97.21|33.05|2309.76|1189.80|1682.64|3499.56|21.17|130.87|104.76|1058.93|1080.10|1163.69|1184.86|-623.71| +2452627|45828|2452746|3891|88868|911334|3400|6266|69772|1136260|1711|29777|43|1|20|4|178|464|55|78.05|200.58|176.51|1323.85|9708.05|4292.75|11031.90|582.48|0.00|441.10|9708.05|10290.53|10149.15|10731.63|5415.30| +2452627|45828|2452722|14821|88868|911334|3400|6266|69772|1136260|1711|29777|55|1|4|2|280|464|8|55.54|134.96|90.42|356.32|723.36|444.32|1079.68|57.86|0.00|129.52|723.36|781.22|852.88|910.74|279.04| +2451905|56172|2451910|517|99177|8638|1652|41913|67680|919643|4649|10110|19|2|7|4|144|465|18|51.42|86.38|21.59|1166.22|388.62|925.56|1554.84|23.31|0.00|295.38|388.62|411.93|684.00|707.31|-536.94| +2451905|56172|2451945|16328|99177|8638|1652|41913|67680|919643|4649|10110|1|13|13|4|182|465|21|66.03|149.88|95.92|1133.16|2014.32|1386.63|3147.48|0.00|0.00|283.08|2014.32|2014.32|2297.40|2297.40|627.69| +2451905|56172|2451978|6943|99177|8638|1652|41913|67680|919643|4649|10110|55|14|15|5|232|465|11|20.69|46.96|3.75|475.31|41.25|227.59|516.56|0.00|0.00|77.44|41.25|41.25|118.69|118.69|-186.34| +2451905|56172|2451997|7913|99177|8638|1652|41913|67680|919643|4649|10110|38|13|11|3|162|465|56|43.87|74.14|22.24|2906.40|1245.44|2456.72|4151.84|0.00|0.00|829.92|1245.44|1245.44|2075.36|2075.36|-1211.28| +2451905|56172|2451937|2887|99177|8638|1652|41913|67680|919643|4649|10110|56|25|14|3|2|465|82|1.57|1.97|0.57|114.80|46.74|128.74|161.54|2.80|0.00|23.78|46.74|49.54|70.52|73.32|-82.00| +2451905|56172|2452005|16225|99177|8638|1652|41913|67680|919643|4649|10110|59|8|20|5|20|465|77|26.24|59.04|5.90|4091.78|454.30|2020.48|4546.08|40.88|0.00|1227.38|454.30|495.18|1681.68|1722.56|-1566.18| +2451905|56172|2451990|3989|99177|8638|1652|41913|67680|919643|4649|10110|32|5|6|2|35|465|99|58.03|113.15|75.81|3696.66|7505.19|5744.97|11201.85|0.00|0.00|4480.74|7505.19|7505.19|11985.93|11985.93|1760.22| +2451905|56172|2451940|4277|99177|8638|1652|41913|67680|919643|4649|10110|25|20|19|4|42|465|91|97.72|219.87|4.39|19608.68|399.49|8892.52|20008.17|15.97|0.00|6002.36|399.49|415.46|6401.85|6417.82|-8493.03| +2451905|56172|2452023|6751|99177|8638|1652|41913|67680|919643|4649|10110|13|23|14|1|11|465|79|58.76|161.00|16.10|11447.10|1271.90|4642.04|12719.00|89.03|0.00|4578.84|1271.90|1360.93|5850.74|5939.77|-3370.14| +2451905|56172|2451938|14612|99177|8638|1652|41913|67680|919643|4649|10110|25|20|9|2|65|465|34|83.77|236.23|160.63|2570.40|5461.42|2848.18|8031.82|109.22|0.00|401.54|5461.42|5570.64|5862.96|5972.18|2613.24| +2451905|56172|2451957|6200|99177|8638|1652|41913|67680|919643|4649|10110|56|14|7|1|185|465|2|56.38|148.84|126.51|44.66|253.02|112.76|297.68|22.77|0.00|74.42|253.02|275.79|327.44|350.21|140.26| +2451905|56172|2452018|6247|99177|8638|1652|41913|67680|919643|4649|10110|11|20|6|1|176|465|52|73.42|147.57|51.64|4988.36|2685.28|3817.84|7673.64|17.45|2336.19|690.56|349.09|366.54|1039.65|1057.10|-3468.75| +2451905|56172|2451947|16945|99177|8638|1652|41913|67680|919643|4649|10110|41|14|2|1|163|465|15|94.16|114.87|33.31|1223.40|499.65|1412.40|1723.05|19.23|224.84|17.10|274.81|294.04|291.91|311.14|-1137.59| +2451905|56172|2451999|15782|99177|8638|1652|41913|67680|919643|4649|10110|13|26|11|1|208|465|81|61.68|145.56|30.56|9315.00|2475.36|4996.08|11790.36|74.26|0.00|1649.97|2475.36|2549.62|4125.33|4199.59|-2520.72| +2451909|18905|2451942|17030|66566|628835|5504|46973|71031|41893|2227|4496|50|14|10|2|224|466|57|80.37|106.89|2.13|5971.32|121.41|4581.09|6092.73|6.07|0.00|2924.10|121.41|127.48|3045.51|3051.58|-4459.68| +2451909|18905|2451934|15323|66566|628835|5504|46973|71031|41893|2227|4496|19|8|7|2|177|466|74|1.24|2.85|0.99|137.64|73.26|91.76|210.90|5.86|0.00|100.64|73.26|79.12|173.90|179.76|-18.50| +2451909|18905|2451953|725|66566|628835|5504|46973|71031|41893|2227|4496|17|8|11|4|90|466|47|18.07|29.45|27.97|69.56|1314.59|849.29|1384.15|0.00|0.00|484.10|1314.59|1314.59|1798.69|1798.69|465.30| +2451909|18905|2451957|1364|66566|628835|5504|46973|71031|41893|2227|4496|53|14|16|4|152|466|13|32.76|56.34|49.01|95.29|637.13|425.88|732.42|44.59|0.00|139.10|637.13|681.72|776.23|820.82|211.25| +2451909|18905|2451955|15443|66566|628835|5504|46973|71031|41893|2227|4496|29|7|10|1|225|466|84|97.31|185.86|87.35|8274.84|7337.40|8174.04|15612.24|79.24|6016.66|3122.28|1320.74|1399.98|4443.02|4522.26|-6853.30| +2451909|18905|2451969|1328|66566|628835|5504|46973|71031|41893|2227|4496|32|23|7|2|87|466|69|54.13|101.76|1.01|6951.75|69.69|3734.97|7021.44|1.39|0.00|2667.54|69.69|71.08|2737.23|2738.62|-3665.28| +2451909|18905|2451946|14347|66566|628835|5504|46973|71031|41893|2227|4496|29|20|20|4|135|466|18|23.72|58.35|43.17|273.24|777.06|426.96|1050.30|15.54|0.00|483.12|777.06|792.60|1260.18|1275.72|350.10| +2451909|18905|2451990|14192|66566|628835|5504|46973|71031|41893|2227|4496|11|26|2|3|211|466|7|71.33|149.07|110.31|271.32|772.17|499.31|1043.49|23.16|0.00|156.52|772.17|795.33|928.69|951.85|272.86| +2451909|18905|2451921|1153|66566|628835|5504|46973|71031|41893|2227|4496|26|7|1|3|228|466|47|70.77|193.90|87.25|5012.55|4100.75|3326.19|9113.30|205.03|0.00|2915.88|4100.75|4305.78|7016.63|7221.66|774.56| +2451909|18905|2452012|13232|66566|628835|5504|46973|71031|41893|2227|4496|44|23|11|5|279|466|6|47.16|135.34|12.18|738.96|73.08|282.96|812.04|3.65|0.00|194.88|73.08|76.73|267.96|271.61|-209.88| +2451909|18905|2451994|17645|66566|628835|5504|46973|71031|41893|2227|4496|20|26|7|3|76|466|62|2.63|5.44|4.29|71.30|265.98|163.06|337.28|23.93|0.00|124.62|265.98|289.91|390.60|414.53|102.92| +2451909|18905|2452019|1373|66566|628835|5504|46973|71031|41893|2227|4496|7|19|16|4|245|466|90|67.01|184.27|90.29|8458.20|8126.10|6030.90|16584.30|325.04|0.00|7296.30|8126.10|8451.14|15422.40|15747.44|2095.20| +2451909|18905|2451925|17|66566|628835|5504|46973|71031|41893|2227|4496|38|25|13|1|149|466|88|26.39|46.44|20.89|2248.40|1838.32|2322.32|4086.72|91.91|0.00|1348.16|1838.32|1930.23|3186.48|3278.39|-484.00| +2452240|62159|2452324|1129|18349|1425051|4553|23255|34002|211508|3008|30110|47|11|8|5|162|467|22|34.41|69.50|23.63|1009.14|519.86|757.02|1529.00|10.39|0.00|473.88|519.86|530.25|993.74|1004.13|-237.16| +2452240|62159|2452316|7401|18349|1425051|4553|23255|34002|211508|3008|30110|5|7|15|3|44|467|11|14.83|31.58|3.78|305.80|41.58|163.13|347.38|0.83|0.00|173.69|41.58|42.41|215.27|216.10|-121.55| +2452240|62159|2452267|16843|18349|1425051|4553|23255|34002|211508|3008|30110|1|5|5|2|16|467|63|44.12|96.62|85.99|669.69|5417.37|2779.56|6087.06|27.08|4063.02|1095.57|1354.35|1381.43|2449.92|2477.00|-1425.21| +2452240|62159|2452274|16299|18349|1425051|4553|23255|34002|211508|3008|30110|49|1|14|5|236|467|49|57.98|143.21|116.00|1333.29|5684.00|2841.02|7017.29|511.56|0.00|631.12|5684.00|6195.56|6315.12|6826.68|2842.98| +2452240|62159|2452347|14311|18349|1425051|4553|23255|34002|211508|3008|30110|1|7|17|5|110|467|80|12.81|34.97|33.57|112.00|2685.60|1024.80|2797.60|214.84|0.00|55.20|2685.60|2900.44|2740.80|2955.64|1660.80| +2452240|62159|2452329|7347|18349|1425051|4553|23255|34002|211508|3008|30110|31|21|16|1|213|467|30|45.59|59.26|29.63|888.90|888.90|1367.70|1777.80|35.55|0.00|337.50|888.90|924.45|1226.40|1261.95|-478.80| +2452240|62159|2452307|17071|18349|1425051|4553|23255|34002|211508|3008|30110|17|3|12|5|235|467|87|39.38|76.39|6.11|6114.36|531.57|3426.06|6645.93|4.46|419.94|66.12|111.63|116.09|177.75|182.21|-3314.43| +2452240|62159|2452250|5579|18349|1425051|4553|23255|34002|211508|3008|30110|9|3|12|2|129|467|1|33.03|96.77|39.67|57.10|39.67|33.03|96.77|2.41|5.15|39.67|34.52|36.93|74.19|76.60|1.49| +2452240|62159|2452298|973|18349|1425051|4553|23255|34002|211508|3008|30110|11|19|1|5|126|467|1|62.17|161.02|61.18|99.84|61.18|62.17|161.02|0.61|0.00|72.45|61.18|61.79|133.63|134.24|-0.99| +2451189|5319|2451270|10382|65596|12413|1630|1647|11546|54928|4898|49715|8|28|18|1|24|468|95|99.40|238.56|198.00|3853.20|18810.00|9443.00|22663.20|1128.60|0.00|3852.25|18810.00|19938.60|22662.25|23790.85|9367.00| +2451189|5319|2451238|451|65596|12413|1630|1647|11546|54928|4898|49715|37|7|9|4|169|468|67|59.87|113.15|79.20|2274.65|5306.40|4011.29|7581.05|477.57|0.00|1516.21|5306.40|5783.97|6822.61|7300.18|1295.11| +2451189|5319|2451266|8066|65596|12413|1630|1647|11546|54928|4898|49715|34|13|13|2|191|468|4|47.32|109.78|80.13|118.60|320.52|189.28|439.12|9.61|0.00|215.16|320.52|330.13|535.68|545.29|131.24| +2451189|5319|2451191|6148|65596|12413|1630|1647|11546|54928|4898|49715|38|1|9|5|41|468|100|64.14|172.53|63.83|10870.00|6383.00|6414.00|17253.00|446.81|0.00|4140.00|6383.00|6829.81|10523.00|10969.81|-31.00| +2451189|5319|2451208|8080|65596|12413|1630|1647|11546|54928|4898|49715|38|13|7|2|74|468|78|41.80|118.71|84.28|2685.54|6573.84|3260.40|9259.38|460.16|0.00|4258.80|6573.84|7034.00|10832.64|11292.80|3313.44| +2451189|5319|2451265|12614|65596|12413|1630|1647|11546|54928|4898|49715|13|13|4|4|254|468|96|39.15|90.82|41.77|4708.80|4009.92|3758.40|8718.72|115.48|2085.15|1830.72|1924.77|2040.25|3755.49|3870.97|-1833.63| +2451189|5319|2451268|11402|65596|12413|1630|1647|11546|54928|4898|49715|7|10|1|3|238|468|33|13.25|26.23|5.77|675.18|190.41|437.25|865.59|1.90|0.00|346.17|190.41|192.31|536.58|538.48|-246.84| +2451189|5319|2451198|13819|65596|12413|1630|1647|11546|54928|4898|49715|50|10|3|3|13|468|40|54.96|99.47|78.58|835.60|3143.20|2198.40|3978.80|62.86|0.00|1710.80|3143.20|3206.06|4854.00|4916.86|944.80| +2451189|5319|2451267|1504|65596|12413|1630|1647|11546|54928|4898|49715|7|19|16|2|28|468|16|60.86|175.88|29.89|2335.84|478.24|973.76|2814.08|6.60|258.24|1181.76|220.00|226.60|1401.76|1408.36|-753.76| +2451189|5319|2451292|13670|65596|12413|1630|1647|11546|54928|4898|49715|44|10|19|2|195|468|78|37.05|69.65|53.63|1249.56|4183.14|2889.90|5432.70|110.43|2802.70|760.50|1380.44|1490.87|2140.94|2251.37|-1509.46| +2451189|5319|2451302|3097|65596|12413|1630|1647|11546|54928|4898|49715|50|16|2|5|288|468|74|23.47|52.10|4.68|3509.08|346.32|1736.78|3855.40|6.92|0.00|809.56|346.32|353.24|1155.88|1162.80|-1390.46| +2451189|5319|2451294|2851|65596|12413|1630|1647|11546|54928|4898|49715|38|2|11|3|232|468|91|44.01|111.78|17.88|8544.90|1627.08|4004.91|10171.98|32.54|0.00|5085.99|1627.08|1659.62|6713.07|6745.61|-2377.83| +2451189|5319|2451227|740|65596|12413|1630|1647|11546|54928|4898|49715|1|28|16|2|198|468|42|38.96|113.37|109.96|143.22|4618.32|1636.32|4761.54|134.39|138.54|1047.48|4479.78|4614.17|5527.26|5661.65|2843.46| +2451189|5319|2451224|22|65596|12413|1630|1647|11546|54928|4898|49715|28|28|12|4|216|468|84|8.54|17.42|5.05|1039.08|424.20|717.36|1463.28|0.00|0.00|131.04|424.20|424.20|555.24|555.24|-293.16| +2451543|31273|2451596|247|34406|1361279|505|12131|56064|937540|5056|21529|13|8|6|2|188|469|26|75.21|175.23|127.91|1230.32|3325.66|1955.46|4555.98|266.05|0.00|1321.06|3325.66|3591.71|4646.72|4912.77|1370.20| +2451543|31273|2451643|4678|34406|1361279|505|12131|56064|937540|5056|21529|1|1|1|3|38|469|24|54.43|68.58|31.54|888.96|756.96|1306.32|1645.92|60.55|0.00|131.52|756.96|817.51|888.48|949.03|-549.36| +2451543|31273|2451608|13960|34406|1361279|505|12131|56064|937540|5056|21529|19|1|19|5|288|469|48|54.40|75.07|22.52|2522.40|1080.96|2611.20|3603.36|97.28|0.00|1008.48|1080.96|1178.24|2089.44|2186.72|-1530.24| +2451543|31273|2451650|4112|34406|1361279|505|12131|56064|937540|5056|21529|49|28|18|3|121|469|47|51.67|109.54|18.62|4273.24|875.14|2428.49|5148.38|61.25|0.00|2059.07|875.14|936.39|2934.21|2995.46|-1553.35| +2451543|31273|2451605|5758|34406|1361279|505|12131|56064|937540|5056|21529|58|10|9|5|211|469|68|37.71|74.28|40.85|2273.24|2777.80|2564.28|5051.04|111.11|0.00|1111.12|2777.80|2888.91|3888.92|4000.03|213.52| +2451543|31273|2451647|14839|34406|1361279|505|12131|56064|937540|5056|21529|43|14|11|2|123|469|69|57.79|135.80|61.11|5153.61|4216.59|3987.51|9370.20|84.33|0.00|2529.54|4216.59|4300.92|6746.13|6830.46|229.08| +2451543|31273|2451566|5396|34406|1361279|505|12131|56064|937540|5056|21529|34|8|2|1|213|469|99|96.18|269.30|45.78|22128.48|4532.22|9521.82|26660.70|362.57|0.00|12530.43|4532.22|4894.79|17062.65|17425.22|-4989.60| +2451543|31273|2451587|17290|34406|1361279|505|12131|56064|937540|5056|21529|43|14|4|2|93|469|86|77.52|217.83|32.67|15923.76|2809.62|6666.72|18733.38|28.09|0.00|5057.66|2809.62|2837.71|7867.28|7895.37|-3857.10| +2451543|31273|2451571|12895|34406|1361279|505|12131|56064|937540|5056|21529|14|4|8|3|138|469|95|98.52|282.75|144.20|13162.25|13699.00|9359.40|26861.25|821.94|0.00|267.90|13699.00|14520.94|13966.90|14788.84|4339.60| +2451543|31273|2451658|4231|34406|1361279|505|12131|56064|937540|5056|21529|16|10|13|5|207|469|73|70.68|151.25|25.71|9164.42|1876.83|5159.64|11041.25|37.53|0.00|1766.60|1876.83|1914.36|3643.43|3680.96|-3282.81| +2451851|83725|2451931|8693|26139|932568|2550|12932|20646|1749488|5772|17754|31|1|3|5|157|470|7|44.14|107.70|106.62|7.56|746.34|308.98|753.90|14.92|0.00|0.00|746.34|761.26|746.34|761.26|437.36| +2451851|83725|2451934|3065|26139|932568|2550|12932|20646|1749488|5772|17754|43|14|11|4|100|470|100|79.53|229.84|142.50|8734.00|14250.00|7953.00|22984.00|712.50|0.00|6205.00|14250.00|14962.50|20455.00|21167.50|6297.00| +2451851|83725|2451936|4189|26139|932568|2550|12932|20646|1749488|5772|17754|49|7|13|2|1|470|93|69.27|191.87|49.88|13205.07|4638.84|6442.11|17843.91|46.38|0.00|7136.82|4638.84|4685.22|11775.66|11822.04|-1803.27| +2451851|83725|2451895|9659|26139|932568|2550|12932|20646|1749488|5772|17754|2|26|19|4|96|470|2|46.77|108.03|41.05|133.96|82.10|93.54|216.06|0.82|0.00|28.08|82.10|82.92|110.18|111.00|-11.44| +2451851|83725|2451882|11426|26139|932568|2550|12932|20646|1749488|5772|17754|7|14|12|1|200|470|37|4.04|11.83|10.64|44.03|393.68|149.48|437.71|27.55|0.00|126.91|393.68|421.23|520.59|548.14|244.20| +2451851|83725|2451874|17309|26139|932568|2550|12932|20646|1749488|5772|17754|43|29|11|4|227|470|55|64.24|122.05|93.97|1544.40|5168.35|3533.20|6712.75|310.10|0.00|1879.35|5168.35|5478.45|7047.70|7357.80|1635.15| +2451851|83725|2451917|787|26139|932568|2550|12932|20646|1749488|5772|17754|32|19|10|3|56|470|13|36.60|66.24|39.08|353.08|508.04|475.80|861.12|25.40|0.00|34.32|508.04|533.44|542.36|567.76|32.24| +2451851|83725|2451906|9644|26139|932568|2550|12932|20646|1749488|5772|17754|43|29|18|1|294|470|6|45.93|74.86|42.67|193.14|256.02|275.58|449.16|12.80|0.00|94.32|256.02|268.82|350.34|363.14|-19.56| +2451851|83725|2451865|1207|26139|932568|2550|12932|20646|1749488|5772|17754|8|2|6|3|180|470|49|64.80|193.10|77.24|5677.14|3784.76|3175.20|9461.90|84.02|2384.39|2365.23|1400.37|1484.39|3765.60|3849.62|-1774.83| +2451851|83725|2451900|15055|26139|932568|2550|12932|20646|1749488|5772|17754|23|5|7|3|209|470|17|98.77|263.71|174.04|1524.39|2958.68|1679.09|4483.07|177.52|0.00|448.29|2958.68|3136.20|3406.97|3584.49|1279.59| +2451851|83725|2451933|16961|26139|932568|2550|12932|20646|1749488|5772|17754|1|25|18|5|132|470|41|32.20|79.21|17.42|2533.39|714.22|1320.20|3247.61|57.13|0.00|1038.94|714.22|771.35|1753.16|1810.29|-605.98| +2451851|83725|2451920|16481|26139|932568|2550|12932|20646|1749488|5772|17754|38|25|6|1|149|470|66|83.94|99.04|64.37|2288.22|4248.42|5540.04|6536.64|0.00|0.00|2156.88|4248.42|4248.42|6405.30|6405.30|-1291.62| +2451851|83725|2451893|563|26139|932568|2550|12932|20646|1749488|5772|17754|35|2|16|3|157|470|53|63.01|67.42|39.77|1465.45|2107.81|3339.53|3573.26|147.54|0.00|1428.88|2107.81|2255.35|3536.69|3684.23|-1231.72| +2451851|83725|2451864|5630|26139|932568|2550|12932|20646|1749488|5772|17754|11|23|18|4|210|470|94|6.12|13.15|11.44|160.74|1075.36|575.28|1236.10|43.01|0.00|12.22|1075.36|1118.37|1087.58|1130.59|500.08| +2451336|31354|2451396|6307|5150|1485208|1713|32971|54788|960749|5592|8850|49|4|11|1|244|471|47|21.67|37.27|29.07|385.40|1366.29|1018.49|1751.69|81.97|0.00|875.61|1366.29|1448.26|2241.90|2323.87|347.80| +2451336|31354|2451410|9148|5150|1485208|1713|32971|54788|960749|5592|8850|44|7|2|3|81|471|1|22.37|56.37|42.27|14.10|42.27|22.37|56.37|0.53|15.63|23.67|26.64|27.17|50.31|50.84|4.27| +2451336|31354|2451418|11305|5150|1485208|1713|32971|54788|960749|5592|8850|49|2|19|1|263|471|76|54.88|61.46|49.16|934.80|3736.16|4170.88|4670.96|298.89|0.00|2101.40|3736.16|4035.05|5837.56|6136.45|-434.72| +2451336|31354|2451378|547|5150|1485208|1713|32971|54788|960749|5592|8850|58|26|5|4|280|471|33|94.18|264.64|140.25|4104.87|4628.25|3107.94|8733.12|185.13|0.00|4104.54|4628.25|4813.38|8732.79|8917.92|1520.31| +2451336|31354|2451372|2593|5150|1485208|1713|32971|54788|960749|5592|8850|56|2|5|3|94|471|70|77.93|98.97|54.43|3117.80|3810.10|5455.10|6927.90|114.30|0.00|2978.50|3810.10|3924.40|6788.60|6902.90|-1645.00| +2451336|31354|2451358|4718|5150|1485208|1713|32971|54788|960749|5592|8850|8|16|20|1|252|471|19|57.27|162.07|155.58|123.31|2956.02|1088.13|3079.33|0.00|2098.77|1447.23|857.25|857.25|2304.48|2304.48|-230.88| +2451336|31354|2451440|14944|5150|1485208|1713|32971|54788|960749|5592|8850|20|19|19|3|247|471|47|18.42|37.02|13.32|1113.90|626.04|865.74|1739.94|6.26|0.00|678.21|626.04|632.30|1304.25|1310.51|-239.70| +2451336|31354|2451425|8461|5150|1485208|1713|32971|54788|960749|5592|8850|1|13|6|2|71|471|75|81.27|88.58|69.09|1461.75|5181.75|6095.25|6643.50|207.27|0.00|2856.00|5181.75|5389.02|8037.75|8245.02|-913.50| +2451680|62910|2451734|6595|73393|1794667|3141|40415|831|1650520|1037|46758|1|13|13|1|62|472|33|41.42|123.84|100.31|776.49|3310.23|1366.86|4086.72|283.02|165.51|1021.68|3144.72|3427.74|4166.40|4449.42|1777.86| +2451680|62910|2451759|14159|73393|1794667|3141|40415|831|1650520|1037|46758|8|1|14|2|220|472|95|4.32|8.03|6.18|175.75|587.10|410.40|762.85|41.09|0.00|190.00|587.10|628.19|777.10|818.19|176.70| +2451680|62910|2451762|10525|73393|1794667|3141|40415|831|1650520|1037|46758|32|26|4|1|141|472|87|11.92|17.04|9.71|637.71|844.77|1037.04|1482.48|50.68|0.00|681.21|844.77|895.45|1525.98|1576.66|-192.27| +2451680|62910|2451703|917|73393|1794667|3141|40415|831|1650520|1037|46758|50|11|16|1|66|472|40|62.04|95.54|85.98|382.40|3439.20|2481.60|3821.60|103.17|0.00|152.80|3439.20|3542.37|3592.00|3695.17|957.60| +2451680|62910|2451794|5627|73393|1794667|3141|40415|831|1650520|1037|46758|49|14|18|1|24|472|79|38.89|67.66|43.97|1871.51|3473.63|3072.31|5345.14|34.73|0.00|1175.52|3473.63|3508.36|4649.15|4683.88|401.32| +2451680|62910|2451758|12548|73393|1794667|3141|40415|831|1650520|1037|46758|26|8|13|1|142|472|53|96.57|127.47|67.55|3175.76|3580.15|5118.21|6755.91|179.00|0.00|202.46|3580.15|3759.15|3782.61|3961.61|-1538.06| +2451680|62910|2451696|10592|73393|1794667|3141|40415|831|1650520|1037|46758|11|14|20|2|74|472|51|2.87|6.08|6.08|0.00|310.08|146.37|310.08|3.10|0.00|9.18|310.08|313.18|319.26|322.36|163.71| +2451680|62910|2451695|4790|73393|1794667|3141|40415|831|1650520|1037|46758|13|5|12|3|67|472|79|70.75|79.94|21.58|4610.44|1704.82|5589.25|6315.26|119.33|0.00|378.41|1704.82|1824.15|2083.23|2202.56|-3884.43| +2451680|62910|2451693|9485|73393|1794667|3141|40415|831|1650520|1037|46758|19|23|4|1|138|472|60|93.93|247.97|121.50|7588.20|7290.00|5635.80|14878.20|131.22|5832.00|4314.60|1458.00|1589.22|5772.60|5903.82|-4177.80| +2452330|76057|2452352|11559|82068|1913360|652|23191|73649|1283036|2714|16806|24|3|2|2|2|473|59|49.02|57.35|51.61|338.66|3044.99|2892.18|3383.65|0.00|0.00|304.44|3044.99|3044.99|3349.43|3349.43|152.81| +2452330|76057|2452374|17587|82068|1913360|652|23191|73649|1283036|2714|16806|1|3|3|4|155|473|85|84.43|215.29|206.67|732.70|17566.95|7176.55|18299.65|1581.02|0.00|2012.80|17566.95|19147.97|19579.75|21160.77|10390.40| +2452330|76057|2452410|3306|82068|1913360|652|23191|73649|1283036|2714|16806|49|15|1|2|40|473|20|93.84|187.68|138.88|976.00|2777.60|1876.80|3753.60|99.99|1666.56|825.60|1111.04|1211.03|1936.64|2036.63|-765.76| +2452330|76057|2452417|8850|82068|1913360|652|23191|73649|1283036|2714|16806|60|25|16|4|105|473|61|88.87|251.50|82.99|10279.11|5062.39|5421.07|15341.50|253.11|0.00|2914.58|5062.39|5315.50|7976.97|8230.08|-358.68| +2452330|76057|2452414|12216|82068|1913360|652|23191|73649|1283036|2714|16806|24|9|15|3|51|473|81|42.87|54.44|49.54|396.90|4012.74|3472.47|4409.64|160.50|0.00|1851.66|4012.74|4173.24|5864.40|6024.90|540.27| +2452330|76057|2452337|1707|82068|1913360|652|23191|73649|1283036|2714|16806|54|21|15|4|53|473|10|19.06|22.87|17.60|52.70|176.00|190.60|228.70|7.04|0.00|89.10|176.00|183.04|265.10|272.14|-14.60| +2452330|76057|2452397|234|82068|1913360|652|23191|73649|1283036|2714|16806|49|13|2|2|149|473|1|90.34|229.46|32.12|197.34|32.12|90.34|229.46|0.32|0.00|89.48|32.12|32.44|121.60|121.92|-58.22| +2452330|76057|2452404|14853|82068|1913360|652|23191|73649|1283036|2714|16806|6|1|2|5|288|473|51|66.02|81.20|65.77|786.93|3354.27|3367.02|4141.20|201.25|0.00|1780.41|3354.27|3555.52|5134.68|5335.93|-12.75| +2452330|76057|2452424|10269|82068|1913360|652|23191|73649|1283036|2714|16806|60|7|1|5|189|473|57|68.12|180.51|52.34|7305.69|2983.38|3882.84|10289.07|268.50|0.00|2366.07|2983.38|3251.88|5349.45|5617.95|-899.46| +2452330|76057|2452356|9945|82068|1913360|652|23191|73649|1283036|2714|16806|13|13|10|3|293|473|95|34.05|88.87|78.20|1013.65|7429.00|3234.75|8442.65|594.32|0.00|1772.70|7429.00|8023.32|9201.70|9796.02|4194.25| +2452330|76057|2452336|11889|82068|1913360|652|23191|73649|1283036|2714|16806|25|1|9|2|295|473|74|83.08|246.74|96.22|11138.48|7120.28|6147.92|18258.76|569.62|0.00|2738.74|7120.28|7689.90|9859.02|10428.64|972.36| +2452628|66680|2452639|15361|8067|1408112|4604|35449|37441|886207|4078|11370|13|15|2|3|124|474|71|4.17|6.71|5.09|115.02|361.39|296.07|476.41|21.68|0.00|9.23|361.39|383.07|370.62|392.30|65.32| +2452628|66680|2452719|17079|8067|1408112|4604|35449|37441|886207|4078|11370|30|1|20|4|88|474|56|92.05|135.31|21.64|6365.52|1211.84|5154.80|7577.36|109.06|0.00|3409.28|1211.84|1320.90|4621.12|4730.18|-3942.96| +2452628|66680|2452721|189|8067|1408112|4604|35449|37441|886207|4078|11370|25|3|1|5|209|474|60|59.72|143.92|54.68|5354.40|3280.80|3583.20|8635.20|131.23|0.00|1899.60|3280.80|3412.03|5180.40|5311.63|-302.40| +2452628|66680|2452653|17163|8067|1408112|4604|35449|37441|886207|4078|11370|57|1|3|2|254|474|28|38.53|45.08|39.67|151.48|1110.76|1078.84|1262.24|0.00|0.00|428.96|1110.76|1110.76|1539.72|1539.72|31.92| +2452628|66680|2452637|14097|8067|1408112|4604|35449|37441|886207|4078|11370|55|15|1|1|211|474|72|1.73|4.39|1.27|224.64|91.44|124.56|316.08|1.82|0.00|107.28|91.44|93.26|198.72|200.54|-33.12| +2452628|66680|2452730|13633|8067|1408112|4604|35449|37441|886207|4078|11370|18|3|12|5|48|474|92|29.02|84.15|4.20|7355.40|386.40|2669.84|7741.80|9.27|154.56|3174.00|231.84|241.11|3405.84|3415.11|-2438.00| +2452628|66680|2452706|1317|8067|1408112|4604|35449|37441|886207|4078|11370|54|30|13|1|99|474|76|32.41|77.45|1.54|5769.16|117.04|2463.16|5886.20|9.36|0.00|2942.72|117.04|126.40|3059.76|3069.12|-2346.12| +2452628|66680|2452727|13134|8067|1408112|4604|35449|37441|886207|4078|11370|31|9|17|4|191|474|12|69.98|85.37|75.12|123.00|901.44|839.76|1024.44|81.12|0.00|512.16|901.44|982.56|1413.60|1494.72|61.68| +2452628|66680|2452681|10659|8067|1408112|4604|35449|37441|886207|4078|11370|9|9|2|2|19|474|62|2.40|4.77|0.19|283.96|11.78|148.80|295.74|0.47|0.00|115.32|11.78|12.25|127.10|127.57|-137.02| +2452628|66680|2452715|1057|8067|1408112|4604|35449|37441|886207|4078|11370|19|30|20|5|150|474|44|6.58|8.35|1.16|316.36|51.04|289.52|367.40|0.38|31.64|183.48|19.40|19.78|202.88|203.26|-270.12| +2452628|66680|2452661|5919|8067|1408112|4604|35449|37441|886207|4078|11370|55|13|19|3|27|474|74|41.79|90.26|31.59|4341.58|2337.66|3092.46|6679.24|116.88|0.00|3071.74|2337.66|2454.54|5409.40|5526.28|-754.80| +2452628|66680|2452690|4422|8067|1408112|4604|35449|37441|886207|4078|11370|1|27|14|4|7|474|73|17.29|40.97|35.64|389.09|2601.72|1262.17|2990.81|26.01|0.00|1016.16|2601.72|2627.73|3617.88|3643.89|1339.55| +2452628|66680|2452693|16933|8067|1408112|4604|35449|37441|886207|4078|11370|43|15|4|5|56|474|63|30.31|82.44|43.69|2441.25|2752.47|1909.53|5193.72|0.00|0.00|882.63|2752.47|2752.47|3635.10|3635.10|842.94| +2452628|66680|2452708|3015|8067|1408112|4604|35449|37441|886207|4078|11370|54|24|7|2|87|474|12|74.65|132.87|10.62|1467.00|127.44|895.80|1594.44|5.09|0.00|462.36|127.44|132.53|589.80|594.89|-768.36| +2452628|66680|2452669|4645|8067|1408112|4604|35449|37441|886207|4078|11370|25|25|6|1|299|474|29|7.44|16.29|14.49|52.20|420.21|215.76|472.41|33.61|0.00|188.79|420.21|453.82|609.00|642.61|204.45| +2452591|10034|2452632|8748|23366|299447|3175|4903|65458|698765|393|24521|31|3|11|2|13|475|41|75.76|184.85|118.30|2728.55|4850.30|3106.16|7578.85|388.02|0.00|3486.23|4850.30|5238.32|8336.53|8724.55|1744.14| +2452591|10034|2452697|15279|23366|299447|3175|4903|65458|698765|393|24521|12|13|15|3|1|475|94|72.68|107.56|50.55|5358.94|4751.70|6831.92|10110.64|427.65|0.00|2426.14|4751.70|5179.35|7177.84|7605.49|-2080.22| +2452591|10034|2452624|12765|23366|299447|3175|4903|65458|698765|393|24521|54|19|3|1|56|475|77|81.48|141.77|68.04|5677.21|5239.08|6273.96|10916.29|0.00|0.00|1419.11|5239.08|5239.08|6658.19|6658.19|-1034.88| +2452591|10034|2452608|1759|23366|299447|3175|4903|65458|698765|393|24521|13|13|20|3|286|475|88|15.38|21.07|10.53|927.52|926.64|1353.44|1854.16|9.26|0.00|388.96|926.64|935.90|1315.60|1324.86|-426.80| +2452591|10034|2452636|11343|23366|299447|3175|4903|65458|698765|393|24521|36|19|2|4|46|475|7|4.88|13.95|4.60|65.45|32.20|34.16|97.65|1.28|0.00|8.75|32.20|33.48|40.95|42.23|-1.96| +2452591|10034|2452620|6381|23366|299447|3175|4903|65458|698765|393|24521|12|30|4|3|72|475|21|22.32|66.73|56.72|210.21|1191.12|468.72|1401.33|0.00|0.00|504.42|1191.12|1191.12|1695.54|1695.54|722.40| +2452591|10034|2452633|4377|23366|299447|3175|4903|65458|698765|393|24521|45|15|6|4|279|475|53|66.64|109.95|57.17|2797.34|3030.01|3531.92|5827.35|272.70|0.00|1107.17|3030.01|3302.71|4137.18|4409.88|-501.91| +2452591|10034|2452612|17689|23366|299447|3175|4903|65458|698765|393|24521|54|21|15|4|83|475|51|54.28|83.04|53.97|1482.57|2752.47|2768.28|4235.04|27.52|0.00|719.61|2752.47|2779.99|3472.08|3499.60|-15.81| +2452591|10034|2452704|2145|23366|299447|3175|4903|65458|698765|393|24521|54|9|10|3|126|475|15|97.95|106.76|105.69|16.05|1585.35|1469.25|1601.40|0.00|0.00|784.65|1585.35|1585.35|2370.00|2370.00|116.10| +2452591|10034|2452672|5529|23366|299447|3175|4903|65458|698765|393|24521|7|6|2|4|113|475|75|83.64|140.51|56.20|6323.25|4215.00|6273.00|10538.25|84.30|0.00|105.00|4215.00|4299.30|4320.00|4404.30|-2058.00| +2452591|10034|2452698|13065|23366|299447|3175|4903|65458|698765|393|24521|27|1|18|5|172|475|34|51.67|52.70|37.41|519.86|1271.94|1756.78|1791.80|89.03|0.00|125.12|1271.94|1360.97|1397.06|1486.09|-484.84| +2452591|10034|2452593|12381|23366|299447|3175|4903|65458|698765|393|24521|15|12|1|1|81|475|85|87.88|195.09|93.64|8623.25|7959.40|7469.80|16582.65|0.00|0.00|7462.15|7959.40|7959.40|15421.55|15421.55|489.60| +2452591|10034|2452684|8167|23366|299447|3175|4903|65458|698765|393|24521|57|15|4|5|116|475|87|77.03|144.81|144.81|0.00|12598.47|6701.61|12598.47|1007.87|0.00|2519.52|12598.47|13606.34|15117.99|16125.86|5896.86| +2452591|10034|2452616|6303|23366|299447|3175|4903|65458|698765|393|24521|51|13|1|5|139|475|6|17.92|52.50|45.67|40.98|274.02|107.52|315.00|16.44|0.00|34.62|274.02|290.46|308.64|325.08|166.50| +2452591|10034|2452611|3937|23366|299447|3175|4903|65458|698765|393|24521|15|9|15|5|252|475|21|75.41|210.39|25.24|3888.15|530.04|1583.61|4418.19|22.42|280.92|176.61|249.12|271.54|425.73|448.15|-1334.49| +2451043|56137|2451140|8767|70620|164341|2415|28489|19958|461671|4311|10785|20|1|18|3|8|476|72|59.92|121.63|2.43|8582.40|174.96|4314.24|8757.36|8.74|0.00|962.64|174.96|183.70|1137.60|1146.34|-4139.28| +2451043|56137|2451049|7747|70620|164341|2415|28489|19958|461671|4311|10785|44|10|9|2|285|476|9|68.28|157.04|98.93|522.99|890.37|614.52|1413.36|8.90|0.00|268.47|890.37|899.27|1158.84|1167.74|275.85| +2451043|56137|2451058|5899|70620|164341|2415|28489|19958|461671|4311|10785|55|28|12|2|149|476|98|51.13|66.98|22.10|4398.24|2165.80|5010.74|6564.04|108.29|0.00|327.32|2165.80|2274.09|2493.12|2601.41|-2844.94| +2451043|56137|2451103|16634|70620|164341|2415|28489|19958|461671|4311|10785|14|7|3|4|188|476|11|51.89|152.03|34.96|1287.77|384.56|570.79|1672.33|19.22|0.00|16.72|384.56|403.78|401.28|420.50|-186.23| +2451043|56137|2451056|8890|70620|164341|2415|28489|19958|461671|4311|10785|13|10|18|5|102|476|36|87.20|113.36|100.89|448.92|3632.04|3139.20|4080.96|290.56|0.00|1346.40|3632.04|3922.60|4978.44|5269.00|492.84| +2451043|56137|2451156|2398|70620|164341|2415|28489|19958|461671|4311|10785|52|8|10|3|276|476|63|72.40|188.24|178.82|593.46|11265.66|4561.20|11859.12|261.36|7998.61|1541.61|3267.05|3528.41|4808.66|5070.02|-1294.15| +2451043|56137|2451131|4576|70620|164341|2415|28489|19958|461671|4311|10785|2|20|13|5|17|476|56|96.15|170.18|22.12|8291.36|1238.72|5384.40|9530.08|24.77|0.00|1905.68|1238.72|1263.49|3144.40|3169.17|-4145.68| +2451043|56137|2451084|11950|70620|164341|2415|28489|19958|461671|4311|10785|38|7|5|1|37|476|70|63.72|189.88|51.26|9703.40|3588.20|4460.40|13291.60|215.29|0.00|5316.50|3588.20|3803.49|8904.70|9119.99|-872.20| +2451043|56137|2451160|620|70620|164341|2415|28489|19958|461671|4311|10785|1|22|3|1|22|476|43|3.40|7.20|5.40|77.40|232.20|146.20|309.60|13.93|0.00|43.00|232.20|246.13|275.20|289.13|86.00| +2451043|56137|2451080|10586|70620|164341|2415|28489|19958|461671|4311|10785|46|19|16|2|178|476|18|5.18|13.67|9.56|73.98|172.08|93.24|246.06|5.16|0.00|29.52|172.08|177.24|201.60|206.76|78.84| +2451043|56137|2451095|7604|70620|164341|2415|28489|19958|461671|4311|10785|44|19|20|5|100|476|16|9.23|18.55|16.88|26.72|270.08|147.68|296.80|10.80|0.00|53.28|270.08|280.88|323.36|334.16|122.40| +2451043|56137|2451078|14785|70620|164341|2415|28489|19958|461671|4311|10785|34|26|13|4|115|476|47|28.58|68.02|44.89|1087.11|2109.83|1343.26|3196.94|84.39|0.00|703.12|2109.83|2194.22|2812.95|2897.34|766.57| +2451043|56137|2451060|17848|70620|164341|2415|28489|19958|461671|4311|10785|50|26|12|5|211|476|74|91.51|243.41|4.86|17652.70|359.64|6771.74|18012.34|11.32|133.06|7744.84|226.58|237.90|7971.42|7982.74|-6545.16| +2451439|40072|2451554|16268|89897|626189|6481|19573|52968|1547569|449|9141|8|10|7|3|189|477|5|22.93|47.92|15.33|162.95|76.65|114.65|239.60|4.59|0.00|79.05|76.65|81.24|155.70|160.29|-38.00| +2451439|40072|2451496|2056|89897|626189|6481|19573|52968|1547569|449|9141|38|2|14|1|98|477|89|93.11|185.28|81.52|9234.64|7255.28|8286.79|16489.92|362.76|0.00|4616.43|7255.28|7618.04|11871.71|12234.47|-1031.51| +2451439|40072|2451506|14048|89897|626189|6481|19573|52968|1547569|449|9141|26|26|14|4|14|477|84|83.13|157.94|82.12|6368.88|6898.08|6982.92|13266.96|551.84|0.00|397.32|6898.08|7449.92|7295.40|7847.24|-84.84| +2451439|40072|2451457|11308|89897|626189|6481|19573|52968|1547569|449|9141|56|2|18|4|207|477|42|37.46|107.13|88.91|765.24|3734.22|1573.32|4499.46|3.73|3659.53|269.64|74.69|78.42|344.33|348.06|-1498.63| +2451439|40072|2451521|2362|89897|626189|6481|19573|52968|1547569|449|9141|40|25|2|1|153|477|57|91.49|252.51|58.07|11083.08|3309.99|5214.93|14393.07|66.19|0.00|5037.09|3309.99|3376.18|8347.08|8413.27|-1904.94| +2451439|40072|2451498|1414|89897|626189|6481|19573|52968|1547569|449|9141|7|26|17|2|210|477|13|53.87|151.91|103.29|632.06|1342.77|700.31|1974.83|7.25|1101.07|868.92|241.70|248.95|1110.62|1117.87|-458.61| +2451439|40072|2451481|7900|89897|626189|6481|19573|52968|1547569|449|9141|25|28|10|5|127|477|25|3.37|3.94|3.58|9.00|89.50|84.25|98.50|0.89|0.00|17.50|89.50|90.39|107.00|107.89|5.25| +2451439|40072|2451541|2125|89897|626189|6481|19573|52968|1547569|449|9141|8|25|8|3|28|477|8|59.27|148.17|96.31|414.88|770.48|474.16|1185.36|38.52|0.00|379.28|770.48|809.00|1149.76|1188.28|296.32| +2452002|63397|2452100|189|15768|1633512|6103|46247|59257|1838354|726|9317|39|9|7|4|34|478|82|69.67|111.47|86.94|2011.46|7129.08|5712.94|9140.54|302.98|1069.36|1918.80|6059.72|6362.70|7978.52|8281.50|346.78| +2452002|63397|2452068|17163|15768|1633512|6103|46247|59257|1838354|726|9317|27|13|13|1|242|478|28|72.24|101.85|24.44|2167.48|684.32|2022.72|2851.80|13.68|0.00|427.56|684.32|698.00|1111.88|1125.56|-1338.40| +2452002|63397|2452106|14097|15768|1633512|6103|46247|59257|1838354|726|9317|59|23|14|1|228|478|18|78.03|92.85|44.56|869.22|802.08|1404.54|1671.30|72.18|0.00|33.30|802.08|874.26|835.38|907.56|-602.46| +2452002|63397|2452037|13633|15768|1633512|6103|46247|59257|1838354|726|9317|43|21|10|4|128|478|56|58.38|110.92|63.22|2671.20|3540.32|3269.28|6211.52|141.61|0.00|745.36|3540.32|3681.93|4285.68|4427.29|271.04| +2452002|63397|2452021|1317|15768|1633512|6103|46247|59257|1838354|726|9317|9|1|14|2|183|478|9|7.87|15.18|11.68|31.50|105.12|70.83|136.62|5.25|0.00|42.30|105.12|110.37|147.42|152.67|34.29| +2452002|63397|2452101|13133|15768|1633512|6103|46247|59257|1838354|726|9317|43|17|16|1|23|478|93|93.19|244.15|207.52|3406.59|19299.36|8666.67|22705.95|964.96|0.00|4086.42|19299.36|20264.32|23385.78|24350.74|10632.69| +2452002|63397|2452042|10659|15768|1633512|6103|46247|59257|1838354|726|9317|19|25|16|4|202|478|6|1.39|1.61|1.09|3.12|6.54|8.34|9.66|0.00|0.00|1.98|6.54|6.54|8.52|8.52|-1.80| +2452002|63397|2452093|1057|15768|1633512|6103|46247|59257|1838354|726|9317|43|13|12|4|269|478|97|67.61|161.58|138.95|2195.11|13478.15|6558.17|15673.26|943.47|0.00|2037.00|13478.15|14421.62|15515.15|16458.62|6919.98| +2452002|63397|2452047|5919|15768|1633512|6103|46247|59257|1838354|726|9317|13|3|14|1|61|478|95|74.58|132.00|99.00|3135.00|9405.00|7085.10|12540.00|564.30|0.00|2257.20|9405.00|9969.30|11662.20|12226.50|2319.90| +2452002|63397|2452047|4421|15768|1633512|6103|46247|59257|1838354|726|9317|11|17|2|5|138|478|5|10.36|11.49|0.00|57.45|0.00|51.80|57.45|0.00|0.00|20.65|0.00|0.00|20.65|20.65|-51.80| +2452002|63397|2452054|16933|15768|1633512|6103|46247|59257|1838354|726|9317|47|29|9|5|34|478|63|18.22|32.06|16.99|949.41|1070.37|1147.86|2019.78|24.72|717.14|484.47|353.23|377.95|837.70|862.42|-794.63| +2452002|63397|2452098|3015|15768|1633512|6103|46247|59257|1838354|726|9317|7|13|7|5|263|478|36|97.40|154.86|37.16|4237.20|1337.76|3506.40|5574.96|0.00|0.00|835.92|1337.76|1337.76|2173.68|2173.68|-2168.64| +2452002|63397|2452045|4645|15768|1633512|6103|46247|59257|1838354|726|9317|5|3|11|5|119|478|18|42.94|61.40|23.33|685.26|419.94|772.92|1105.20|37.79|0.00|33.12|419.94|457.73|453.06|490.85|-352.98| +2452002|63397|2452070|10807|15768|1633512|6103|46247|59257|1838354|726|9317|17|7|16|1|176|478|4|20.43|24.72|15.32|37.60|61.28|81.72|98.88|0.29|57.60|15.80|3.68|3.97|19.48|19.77|-78.04| +2452002|63397|2452051|2257|15768|1633512|6103|46247|59257|1838354|726|9317|29|1|7|3|229|478|52|91.26|257.35|12.86|12713.48|668.72|4745.52|13382.20|20.06|0.00|133.64|668.72|688.78|802.36|822.42|-4076.80| +2451503|30182|2451549|8005|22456|1044614|2772|26530|99054|440727|6604|31962|16|16|19|4|189|479|43|8.65|9.34|7.65|72.67|328.95|371.95|401.62|23.02|0.00|116.10|328.95|351.97|445.05|468.07|-43.00| +2451503|30182|2451540|16435|22456|1044614|2772|26530|99054|440727|6604|31962|19|20|2|1|277|479|64|37.48|89.20|24.97|4110.72|1598.08|2398.72|5708.80|63.92|0.00|1027.20|1598.08|1662.00|2625.28|2689.20|-800.64| +2451503|30182|2451522|10609|22456|1044614|2772|26530|99054|440727|6604|31962|26|1|5|2|126|479|86|87.26|183.24|166.74|1419.00|14339.64|7504.36|15758.64|430.18|0.00|2993.66|14339.64|14769.82|17333.30|17763.48|6835.28| +2451503|30182|2451550|4249|22456|1044614|2772|26530|99054|440727|6604|31962|52|7|11|3|55|479|23|66.32|99.48|89.53|228.85|2059.19|1525.36|2288.04|0.00|0.00|1006.71|2059.19|2059.19|3065.90|3065.90|533.83| +2451503|30182|2451519|15016|22456|1044614|2772|26530|99054|440727|6604|31962|44|14|3|3|83|479|29|92.66|165.86|18.24|4280.98|528.96|2687.14|4809.94|31.73|0.00|1057.92|528.96|560.69|1586.88|1618.61|-2158.18| +2451503|30182|2451606|14666|22456|1044614|2772|26530|99054|440727|6604|31962|31|2|8|4|80|479|18|65.86|69.81|50.26|351.90|904.68|1185.48|1256.58|18.09|0.00|87.84|904.68|922.77|992.52|1010.61|-280.80| +2451503|30182|2451534|15115|22456|1044614|2772|26530|99054|440727|6604|31962|7|13|17|2|7|479|77|81.86|193.18|179.65|1041.81|13833.05|6303.22|14874.86|0.00|0.00|445.83|13833.05|13833.05|14278.88|14278.88|7529.83| +2451503|30182|2451587|12346|22456|1044614|2772|26530|99054|440727|6604|31962|16|14|10|4|51|479|5|72.99|145.25|5.81|697.20|29.05|364.95|726.25|0.00|1.45|123.45|27.60|27.60|151.05|151.05|-337.35| +2451503|30182|2451591|3434|22456|1044614|2772|26530|99054|440727|6604|31962|22|16|13|5|277|479|11|6.36|6.99|3.77|35.42|41.47|69.96|76.89|1.65|0.00|19.14|41.47|43.12|60.61|62.26|-28.49| +2451503|30182|2451535|1048|22456|1044614|2772|26530|99054|440727|6604|31962|16|7|2|5|88|479|18|88.94|106.72|43.75|1133.46|787.50|1600.92|1920.96|7.87|0.00|441.72|787.50|795.37|1229.22|1237.09|-813.42| +2451503|30182|2451591|12970|22456|1044614|2772|26530|99054|440727|6604|31962|37|2|8|4|237|479|57|95.86|216.64|51.99|9385.05|2963.43|5464.02|12348.48|29.63|0.00|1605.12|2963.43|2993.06|4568.55|4598.18|-2500.59| +2451503|30182|2451562|592|22456|1044614|2772|26530|99054|440727|6604|31962|58|7|8|2|83|479|40|11.19|23.94|10.05|555.60|402.00|447.60|957.60|36.18|0.00|325.20|402.00|438.18|727.20|763.38|-45.60| +2451503|30182|2451562|4100|22456|1044614|2772|26530|99054|440727|6604|31962|34|4|15|2|39|479|37|28.67|77.69|50.49|1006.40|1868.13|1060.79|2874.53|0.00|0.00|431.05|1868.13|1868.13|2299.18|2299.18|807.34| +2451503|30182|2451566|13036|22456|1044614|2772|26530|99054|440727|6604|31962|4|22|11|3|152|479|22|83.86|105.66|84.52|465.08|1859.44|1844.92|2324.52|130.16|0.00|1045.88|1859.44|1989.60|2905.32|3035.48|14.52| +2452111|57852|2452160|12945|5889|478795|6101|33807|5889|478795|6101|33807|25|3|11|5|89|480|84|55.82|89.87|34.15|4680.48|2868.60|4688.88|7549.08|114.74|0.00|1207.08|2868.60|2983.34|4075.68|4190.42|-1820.28| +2452111|57852|2452130|15887|5889|478795|6101|33807|5889|478795|6101|33807|35|27|10|1|145|480|12|87.97|123.15|35.71|1049.28|428.52|1055.64|1477.80|4.28|0.00|339.84|428.52|432.80|768.36|772.64|-627.12| +2452111|57852|2452117|9665|5889|478795|6101|33807|5889|478795|6101|33807|33|15|2|1|27|480|62|80.67|166.18|88.07|4842.82|5460.34|5001.54|10303.16|382.22|0.00|2266.10|5460.34|5842.56|7726.44|8108.66|458.80| +2452111|57852|2452203|13713|5889|478795|6101|33807|5889|478795|6101|33807|15|29|19|3|237|480|28|6.09|15.59|0.77|414.96|21.56|170.52|436.52|0.43|0.00|95.76|21.56|21.99|117.32|117.75|-148.96| +2452111|57852|2452115|7613|5889|478795|6101|33807|5889|478795|6101|33807|39|15|19|2|175|480|67|68.26|123.55|43.24|5380.77|2897.08|4573.42|8277.85|28.97|0.00|661.96|2897.08|2926.05|3559.04|3588.01|-1676.34| +2452111|57852|2452124|17893|5889|478795|6101|33807|5889|478795|6101|33807|57|3|15|3|101|480|11|53.70|104.17|73.96|332.31|813.56|590.70|1145.87|24.40|0.00|91.63|813.56|837.96|905.19|929.59|222.86| +2452111|57852|2452216|2081|5889|478795|6101|33807|5889|478795|6101|33807|5|23|4|3|235|480|80|4.20|12.18|10.23|156.00|818.40|336.00|974.40|32.73|0.00|233.60|818.40|851.13|1052.00|1084.73|482.40| +2452111|57852|2452201|979|5889|478795|6101|33807|5889|478795|6101|33807|47|27|7|5|293|480|34|70.12|147.95|75.45|2465.00|2565.30|2384.08|5030.30|24.11|1359.60|0.00|1205.70|1229.81|1205.70|1229.81|-1178.38| +2452111|57852|2452192|11613|5889|478795|6101|33807|5889|478795|6101|33807|15|5|15|5|235|480|95|69.20|117.64|44.70|6929.30|4246.50|6574.00|11175.80|382.18|0.00|4581.85|4246.50|4628.68|8828.35|9210.53|-2327.50| +2452111|57852|2452181|5649|5889|478795|6101|33807|5889|478795|6101|33807|3|23|19|4|86|480|30|49.76|65.18|28.67|1095.30|860.10|1492.80|1955.40|0.00|0.00|371.40|860.10|860.10|1231.50|1231.50|-632.70| +2451606|71268|2451634|7721|49668|5120|2972|6071|81205|1766502|1227|3079|31|17|19|2|25|481|3|43.55|125.42|68.98|169.32|206.94|130.65|376.26|18.62|0.00|18.81|206.94|225.56|225.75|244.37|76.29| +2451606|71268|2451669|10441|49668|5120|2972|6071|81205|1766502|1227|3079|5|25|6|4|112|481|60|56.42|140.48|81.47|3540.60|4888.20|3385.20|8428.80|391.05|0.00|842.40|4888.20|5279.25|5730.60|6121.65|1503.00| +2451606|71268|2451633|2911|49668|5120|2972|6071|81205|1766502|1227|3079|32|1|15|4|163|481|27|82.58|221.31|134.99|2330.64|3644.73|2229.66|5975.37|215.76|947.62|2509.65|2697.11|2912.87|5206.76|5422.52|467.45| +2451606|71268|2451665|9355|49668|5120|2972|6071|81205|1766502|1227|3079|25|19|17|4|248|481|59|84.45|177.34|78.02|5859.88|4603.18|4982.55|10463.06|92.06|0.00|104.43|4603.18|4695.24|4707.61|4799.67|-379.37| +2451606|71268|2451624|1819|49668|5120|2972|6071|81205|1766502|1227|3079|55|8|2|4|116|481|64|78.08|96.03|23.04|4671.36|1474.56|4997.12|6145.92|88.47|0.00|2519.68|1474.56|1563.03|3994.24|4082.71|-3522.56| +2451606|71268|2451675|8945|49668|5120|2972|6071|81205|1766502|1227|3079|56|13|5|3|151|481|72|75.51|178.95|55.47|8890.56|3993.84|5436.72|12884.40|279.56|0.00|3091.68|3993.84|4273.40|7085.52|7365.08|-1442.88| +2451606|71268|2451691|6473|49668|5120|2972|6071|81205|1766502|1227|3079|47|7|1|3|205|481|27|75.47|95.84|39.29|1526.85|1060.83|2037.69|2587.68|63.64|0.00|284.58|1060.83|1124.47|1345.41|1409.05|-976.86| +2451606|71268|2451694|3260|49668|5120|2972|6071|81205|1766502|1227|3079|53|26|4|1|88|481|84|66.20|94.00|78.96|1263.36|6632.64|5560.80|7896.00|132.65|0.00|1895.04|6632.64|6765.29|8527.68|8660.33|1071.84| +2451606|71268|2451709|4766|49668|5120|2972|6071|81205|1766502|1227|3079|47|17|7|2|70|481|47|19.46|51.37|39.04|579.51|1834.88|914.62|2414.39|165.13|0.00|0.00|1834.88|2000.01|1834.88|2000.01|920.26| +2451606|71268|2451607|17408|49668|5120|2972|6071|81205|1766502|1227|3079|11|17|19|2|272|481|93|44.17|87.45|28.85|5449.80|2683.05|4107.81|8132.85|0.00|0.00|3985.05|2683.05|2683.05|6668.10|6668.10|-1424.76| +2451606|71268|2451719|1315|49668|5120|2972|6071|81205|1766502|1227|3079|41|5|5|4|136|481|49|59.95|76.13|41.87|1678.74|2051.63|2937.55|3730.37|20.51|0.00|1753.22|2051.63|2072.14|3804.85|3825.36|-885.92| +2451894|26174|2451938|8267|5885|418822|4206|44390|83884|518207|495|40279|31|11|13|3|80|482|95|48.86|94.29|49.97|4210.40|4747.15|4641.70|8957.55|427.24|0.00|3493.15|4747.15|5174.39|8240.30|8667.54|105.45| +2451894|26174|2451923|17534|5885|418822|4206|44390|83884|518207|495|40279|29|29|15|5|225|482|32|46.68|46.68|42.01|149.44|1344.32|1493.76|1493.76|67.21|0.00|656.96|1344.32|1411.53|2001.28|2068.49|-149.44| +2451894|26174|2451966|6997|5885|418822|4206|44390|83884|518207|495|40279|38|29|7|2|157|482|82|40.91|41.31|23.54|1457.14|1930.28|3354.62|3387.42|96.51|0.00|236.98|1930.28|2026.79|2167.26|2263.77|-1424.34| +2451894|26174|2451992|17312|5885|418822|4206|44390|83884|518207|495|40279|31|26|2|2|135|482|42|51.51|62.32|29.29|1387.26|1230.18|2163.42|2617.44|49.20|0.00|916.02|1230.18|1279.38|2146.20|2195.40|-933.24| +2451894|26174|2451996|10352|5885|418822|4206|44390|83884|518207|495|40279|31|29|4|3|109|482|87|69.61|160.10|11.20|12954.30|974.40|6056.07|13928.70|77.95|0.00|3760.14|974.40|1052.35|4734.54|4812.49|-5081.67| +2451894|26174|2451974|13979|5885|418822|4206|44390|83884|518207|495|40279|14|26|3|2|258|482|51|49.66|90.38|28.92|3134.46|1474.92|2532.66|4609.38|0.00|0.00|1106.19|1474.92|1474.92|2581.11|2581.11|-1057.74| +2451894|26174|2451918|12479|5885|418822|4206|44390|83884|518207|495|40279|17|7|6|1|213|482|69|23.36|64.24|23.12|2837.28|1595.28|1611.84|4432.56|31.90|0.00|176.64|1595.28|1627.18|1771.92|1803.82|-16.56| +2451894|26174|2451970|16103|5885|418822|4206|44390|83884|518207|495|40279|5|8|10|5|50|482|32|79.95|232.65|174.48|1861.44|5583.36|2558.40|7444.80|390.83|0.00|1265.60|5583.36|5974.19|6848.96|7239.79|3024.96| +2451894|26174|2451999|2629|5885|418822|4206|44390|83884|518207|495|40279|35|19|2|4|166|482|14|98.43|189.96|94.98|1329.72|1329.72|1378.02|2659.44|26.59|0.00|1303.12|1329.72|1356.31|2632.84|2659.43|-48.30| +2451894|26174|2451983|13052|5885|418822|4206|44390|83884|518207|495|40279|37|29|1|2|183|482|27|57.76|76.24|57.94|494.10|1564.38|1559.52|2058.48|0.00|172.08|823.23|1392.30|1392.30|2215.53|2215.53|-167.22| +2451894|26174|2451953|3137|5885|418822|4206|44390|83884|518207|495|40279|37|1|6|1|76|482|4|92.60|179.64|68.26|445.52|273.04|370.40|718.56|24.57|0.00|265.84|273.04|297.61|538.88|563.45|-97.36| +2451894|26174|2451992|2165|5885|418822|4206|44390|83884|518207|495|40279|43|13|12|2|149|482|65|65.85|167.91|52.05|7530.90|3383.25|4280.25|10914.15|101.49|0.00|3710.20|3383.25|3484.74|7093.45|7194.94|-897.00| +2451894|26174|2451968|4964|5885|418822|4206|44390|83884|518207|495|40279|19|20|12|3|104|482|56|9.13|11.04|3.53|420.56|197.68|511.28|618.24|1.97|0.00|185.36|197.68|199.65|383.04|385.01|-313.60| +2451894|26174|2451985|14030|5885|418822|4206|44390|83884|518207|495|40279|11|19|17|1|278|482|93|40.41|55.36|21.03|3192.69|1955.79|3758.13|5148.48|39.11|0.00|1801.41|1955.79|1994.90|3757.20|3796.31|-1802.34| +2451894|26174|2451926|12841|5885|418822|4206|44390|83884|518207|495|40279|35|29|2|2|299|482|11|31.22|69.93|49.65|223.08|546.15|343.42|769.23|27.30|0.00|0.00|546.15|573.45|546.15|573.45|202.73| +2451894|26174|2451923|67|5885|418822|4206|44390|83884|518207|495|40279|26|1|11|3|38|482|82|21.28|59.79|47.23|1029.92|3872.86|1744.96|4902.78|309.82|0.00|146.78|3872.86|4182.68|4019.64|4329.46|2127.90| +2452217|71513|2452246|3945|33319|670045|2418|28391|19530|136284|4263|33036|19|7|1|5|272|483|30|53.39|66.20|37.07|873.90|1112.10|1601.70|1986.00|24.02|845.19|953.10|266.91|290.93|1220.01|1244.03|-1334.79| +2452217|71513|2452250|11717|33319|670045|2418|28391|19530|136284|4263|33036|11|1|6|1|226|483|74|24.05|34.15|8.53|1895.88|631.22|1779.70|2527.10|50.49|0.00|530.58|631.22|681.71|1161.80|1212.29|-1148.48| +2452217|71513|2452328|12291|33319|670045|2418|28391|19530|136284|4263|33036|51|17|12|3|41|483|11|77.66|192.59|48.14|1588.95|529.54|854.26|2118.49|31.77|0.00|953.26|529.54|561.31|1482.80|1514.57|-324.72| +2452217|71513|2452249|15137|33319|670045|2418|28391|19530|136284|4263|33036|31|13|5|1|227|483|62|17.29|44.95|15.73|1811.64|975.26|1071.98|2786.90|0.00|0.00|1058.96|975.26|975.26|2034.22|2034.22|-96.72| +2452217|71513|2452312|11207|33319|670045|2418|28391|19530|136284|4263|33036|17|21|13|5|197|483|14|47.70|95.40|48.65|654.50|681.10|667.80|1335.60|22.88|108.97|280.42|572.13|595.01|852.55|875.43|-95.67| +2452217|71513|2452306|2423|33319|670045|2418|28391|19530|136284|4263|33036|59|9|13|3|6|483|95|73.27|108.43|28.19|7622.80|2678.05|6960.65|10300.85|15.26|2169.22|3707.85|508.83|524.09|4216.68|4231.94|-6451.82| +2452217|71513|2452276|6677|33319|670045|2418|28391|19530|136284|4263|33036|41|27|20|3|166|483|99|62.13|102.51|59.45|4262.94|5885.55|6150.87|10148.49|0.00|5885.55|2739.33|0.00|0.00|2739.33|2739.33|-6150.87| +2452217|71513|2452321|13097|33319|670045|2418|28391|19530|136284|4263|33036|9|27|20|4|55|483|72|32.84|69.62|0.69|4962.96|49.68|2364.48|5012.64|0.99|0.00|701.28|49.68|50.67|750.96|751.95|-2314.80| +2452217|71513|2452328|17231|33319|670045|2418|28391|19530|136284|4263|33036|19|3|7|4|271|483|19|30.38|31.89|14.03|339.34|266.57|577.22|605.91|2.66|0.00|248.33|266.57|269.23|514.90|517.56|-310.65| +2452217|71513|2452268|15317|33319|670045|2418|28391|19530|136284|4263|33036|41|11|11|1|141|483|11|20.60|57.47|20.68|404.69|227.48|226.60|632.17|2.27|0.00|69.52|227.48|229.75|297.00|299.27|0.88| +2452217|71513|2452336|12559|33319|670045|2418|28391|19530|136284|4263|33036|35|5|15|5|2|483|26|61.23|144.50|135.83|225.42|3531.58|1591.98|3757.00|176.57|0.00|1089.40|3531.58|3708.15|4620.98|4797.55|1939.60| +2452217|71513|2452297|8543|33319|670045|2418|28391|19530|136284|4263|33036|9|13|9|4|216|483|39|63.45|107.86|45.30|2439.84|1766.70|2474.55|4206.54|0.00|0.00|336.18|1766.70|1766.70|2102.88|2102.88|-707.85| +2452217|71513|2452318|11649|33319|670045|2418|28391|19530|136284|4263|33036|15|9|15|5|35|483|51|59.70|162.38|8.11|7867.77|413.61|3044.70|8281.38|37.22|0.00|3643.44|413.61|450.83|4057.05|4094.27|-2631.09| +2452217|71513|2452289|6639|33319|670045|2418|28391|19530|136284|4263|33036|21|23|20|1|29|483|26|31.91|52.01|39.52|324.74|1027.52|829.66|1352.26|82.20|0.00|135.20|1027.52|1109.72|1162.72|1244.92|197.86| +2452217|71513|2452297|13491|33319|670045|2418|28391|19530|136284|4263|33036|53|17|11|3|45|483|81|57.51|148.95|17.87|10617.48|1447.47|4658.31|12064.95|115.79|0.00|1447.47|1447.47|1563.26|2894.94|3010.73|-3210.84| +2452217|71513|2452244|6103|33319|670045|2418|28391|19530|136284|4263|33036|1|29|16|5|244|483|74|32.47|63.64|59.82|282.68|4426.68|2402.78|4709.36|0.00|0.00|1365.30|4426.68|4426.68|5791.98|5791.98|2023.90| +2452555|78742|2452640|5745|67906|1684610|2914|4818|90880|54185|2883|20387|33|13|10|5|240|484|33|67.01|144.74|62.23|2722.83|2053.59|2211.33|4776.42|0.00|0.00|1623.93|2053.59|2053.59|3677.52|3677.52|-157.74| +2452555|78742|2452614|16746|67906|1684610|2914|4818|90880|54185|2883|20387|30|18|13|2|231|484|53|45.01|71.11|7.82|3354.37|414.46|2385.53|3768.83|0.66|381.30|1017.07|33.16|33.82|1050.23|1050.89|-2352.37| +2452555|78742|2452631|4161|67906|1684610|2914|4818|90880|54185|2883|20387|45|15|3|3|263|484|27|16.92|20.47|15.55|132.84|419.85|456.84|552.69|2.26|394.65|16.47|25.20|27.46|41.67|43.93|-431.64| +2452555|78742|2452632|12859|67906|1684610|2914|4818|90880|54185|2883|20387|7|27|4|1|81|484|7|92.24|215.84|92.81|861.21|649.67|645.68|1510.88|0.00|32.48|498.54|617.19|617.19|1115.73|1115.73|-28.49| +2452555|78742|2452607|16101|67906|1684610|2914|4818|90880|54185|2883|20387|39|27|8|4|33|484|18|97.01|286.17|240.38|824.22|4326.84|1746.18|5151.06|0.00|1124.97|154.44|3201.87|3201.87|3356.31|3356.31|1455.69| +2452555|78742|2452591|9199|67906|1684610|2914|4818|90880|54185|2883|20387|36|13|16|3|57|484|13|68.92|93.04|83.73|121.03|1088.49|895.96|1209.52|76.19|0.00|471.64|1088.49|1164.68|1560.13|1636.32|192.53| +2452555|78742|2452638|9906|67906|1684610|2914|4818|90880|54185|2883|20387|55|15|8|1|31|484|78|27.86|29.53|28.93|46.80|2256.54|2173.08|2303.34|203.08|0.00|322.14|2256.54|2459.62|2578.68|2781.76|83.46| +2452555|78742|2452578|17280|67906|1684610|2914|4818|90880|54185|2883|20387|43|18|19|1|203|484|33|96.58|123.62|100.13|775.17|3304.29|3187.14|4079.46|165.21|0.00|815.76|3304.29|3469.50|4120.05|4285.26|117.15| +2452555|78742|2452651|14520|67906|1684610|2914|4818|90880|54185|2883|20387|15|15|1|1|151|484|23|26.96|34.77|1.73|759.92|39.79|620.08|799.71|1.19|0.00|263.81|39.79|40.98|303.60|304.79|-580.29| +2452555|78742|2452569|13999|67906|1684610|2914|4818|90880|54185|2883|20387|27|9|6|2|158|484|96|57.06|67.33|47.13|1939.20|4524.48|5477.76|6463.68|20.36|4117.27|904.32|407.21|427.57|1311.53|1331.89|-5070.55| +2452555|78742|2452612|10231|67906|1684610|2914|4818|90880|54185|2883|20387|27|25|15|2|139|484|74|41.38|111.72|67.03|3307.06|4960.22|3062.12|8267.28|334.81|1240.05|3554.22|3720.17|4054.98|7274.39|7609.20|658.05| +2452555|78742|2452664|8736|67906|1684610|2914|4818|90880|54185|2883|20387|15|18|20|4|185|484|87|59.60|140.65|21.09|10401.72|1834.83|5185.20|12236.55|110.08|0.00|3058.92|1834.83|1944.91|4893.75|5003.83|-3350.37| +2452555|78742|2452628|12600|67906|1684610|2914|4818|90880|54185|2883|20387|42|21|4|2|117|484|64|3.88|7.76|6.28|94.72|401.92|248.32|496.64|0.00|0.00|19.84|401.92|401.92|421.76|421.76|153.60| +2452555|78742|2452576|5757|67906|1684610|2914|4818|90880|54185|2883|20387|43|27|6|3|87|484|65|78.97|173.73|19.11|10050.30|1242.15|5133.05|11292.45|12.42|0.00|1467.70|1242.15|1254.57|2709.85|2722.27|-3890.90| +2452555|78742|2452578|2871|67906|1684610|2914|4818|90880|54185|2883|20387|43|6|14|3|299|484|92|77.60|139.68|134.09|514.28|12336.28|7139.20|12850.56|616.81|0.00|1798.60|12336.28|12953.09|14134.88|14751.69|5197.08| +2452555|78742|2452589|6966|67906|1684610|2914|4818|90880|54185|2883|20387|48|3|10|3|206|484|23|21.23|47.13|43.35|86.94|997.05|488.29|1083.99|19.94|0.00|444.36|997.05|1016.99|1441.41|1461.35|508.76| +2451650|71613|2451734|7886|47434|1639593|199|19605|67510|439477|3776|10382|38|23|14|1|281|485|54|3.33|6.95|1.32|304.02|71.28|179.82|375.30|5.70|0.00|10.80|71.28|76.98|82.08|87.78|-108.54| +2451650|71613|2451673|12710|47434|1639593|199|19605|67510|439477|3776|10382|8|23|19|4|296|485|85|76.56|96.46|96.46|0.00|8199.10|6507.60|8199.10|250.89|4017.55|2213.40|4181.55|4432.44|6394.95|6645.84|-2326.05| +2451650|71613|2451747|7555|47434|1639593|199|19605|67510|439477|3776|10382|2|19|15|3|108|485|100|59.52|73.80|35.42|3838.00|3542.00|5952.00|7380.00|124.67|425.04|2583.00|3116.96|3241.63|5699.96|5824.63|-2835.04| +2451650|71613|2451714|11417|47434|1639593|199|19605|67510|439477|3776|10382|41|26|10|1|182|485|40|65.12|95.07|34.22|2434.00|1368.80|2604.80|3802.80|54.75|0.00|304.00|1368.80|1423.55|1672.80|1727.55|-1236.00| +2451650|71613|2451711|4856|47434|1639593|199|19605|67510|439477|3776|10382|25|14|13|3|166|485|1|72.82|87.38|31.45|55.93|31.45|72.82|87.38|2.83|0.00|35.82|31.45|34.28|67.27|70.10|-41.37| +2451650|71613|2451694|13343|47434|1639593|199|19605|67510|439477|3776|10382|8|5|4|2|133|485|16|70.82|208.91|102.36|1704.80|1637.76|1133.12|3342.56|81.88|0.00|1069.60|1637.76|1719.64|2707.36|2789.24|504.64| +2451650|71613|2451711|6629|47434|1639593|199|19605|67510|439477|3776|10382|19|19|20|3|63|485|41|54.94|132.95|50.52|3379.63|2071.32|2252.54|5450.95|103.56|0.00|2343.56|2071.32|2174.88|4414.88|4518.44|-181.22| +2451650|71613|2451655|15809|47434|1639593|199|19605|67510|439477|3776|10382|56|29|18|4|10|485|70|74.98|95.97|23.03|5105.80|1612.10|5248.60|6717.90|64.48|0.00|335.30|1612.10|1676.58|1947.40|2011.88|-3636.50| +2451650|71613|2451710|1225|47434|1639593|199|19605|67510|439477|3776|10382|19|8|20|5|290|485|83|87.54|233.73|46.74|15520.17|3879.42|7265.82|19399.59|193.97|0.00|1163.66|3879.42|4073.39|5043.08|5237.05|-3386.40| +2451650|71613|2451656|4729|47434|1639593|199|19605|67510|439477|3776|10382|1|13|16|2|138|485|66|51.04|142.40|54.11|5827.14|3571.26|3368.64|9398.40|321.41|0.00|281.82|3571.26|3892.67|3853.08|4174.49|202.62| +2451650|71613|2451741|6818|47434|1639593|199|19605|67510|439477|3776|10382|29|23|9|3|250|485|62|99.41|121.28|77.61|2707.54|4811.82|6163.42|7519.36|384.94|0.00|2330.58|4811.82|5196.76|7142.40|7527.34|-1351.60| +2451650|71613|2451724|16766|47434|1639593|199|19605|67510|439477|3776|10382|13|1|15|4|223|485|33|92.01|260.38|257.77|86.13|8506.41|3036.33|8592.54|85.06|0.00|945.12|8506.41|8591.47|9451.53|9536.59|5470.08| +2451650|71613|2451750|12284|47434|1639593|199|19605|67510|439477|3776|10382|29|1|10|1|35|485|9|72.40|139.73|117.37|201.24|1056.33|651.60|1257.57|48.48|517.60|440.10|538.73|587.21|978.83|1027.31|-112.87| +2451457|9137|2451552|17162|8|801885|2515|21145|93845|1775482|329|21759|49|10|7|3|129|486|64|50.60|140.66|68.92|4591.36|4410.88|3238.40|9002.24|264.65|0.00|1260.16|4410.88|4675.53|5671.04|5935.69|1172.48| +2451457|9137|2451495|14096|8|801885|2515|21145|93845|1775482|329|21759|56|28|14|3|79|486|57|25.34|53.72|33.84|1133.16|1928.88|1444.38|3062.04|154.31|0.00|336.30|1928.88|2083.19|2265.18|2419.49|484.50| +2451457|9137|2451486|13633|8|801885|2515|21145|93845|1775482|329|21759|1|16|14|2|167|486|66|25.40|67.05|40.23|1770.12|2655.18|1676.40|4425.30|212.41|0.00|44.22|2655.18|2867.59|2699.40|2911.81|978.78| +2451457|9137|2451520|1316|8|801885|2515|21145|93845|1775482|329|21759|49|26|20|1|295|486|100|89.31|257.21|56.58|20063.00|5658.00|8931.00|25721.00|169.74|0.00|4372.00|5658.00|5827.74|10030.00|10199.74|-3273.00| +2451457|9137|2451478|13132|8|801885|2515|21145|93845|1775482|329|21759|52|8|3|1|29|486|38|74.11|168.22|136.25|1214.86|5177.50|2816.18|6392.36|207.10|0.00|1597.90|5177.50|5384.60|6775.40|6982.50|2361.32| +2451457|9137|2451514|10658|8|801885|2515|21145|93845|1775482|329|21759|43|28|15|3|98|486|59|68.69|120.89|45.93|4422.64|2709.87|4052.71|7132.51|27.09|0.00|3423.18|2709.87|2736.96|6133.05|6160.14|-1342.84| +2451457|9137|2451559|1057|8|801885|2515|21145|93845|1775482|329|21759|32|2|17|4|294|486|73|97.35|242.40|77.56|12033.32|5661.88|7106.55|17695.20|132.48|3453.74|6369.98|2208.14|2340.62|8578.12|8710.60|-4898.41| +2451457|9137|2451485|5918|8|801885|2515|21145|93845|1775482|329|21759|22|4|14|5|36|486|2|90.75|206.00|109.18|193.64|218.36|181.50|412.00|4.36|0.00|177.16|218.36|222.72|395.52|399.88|36.86| +2451457|9137|2451506|4420|8|801885|2515|21145|93845|1775482|329|21759|16|26|9|4|179|486|24|28.99|63.19|47.39|379.20|1137.36|695.76|1516.56|102.36|0.00|166.80|1137.36|1239.72|1304.16|1406.52|441.60| +2451457|9137|2451499|16933|8|801885|2515|21145|93845|1775482|329|21759|58|28|13|3|16|486|75|2.14|6.27|1.31|372.00|98.25|160.50|470.25|4.87|37.33|84.00|60.92|65.79|144.92|149.79|-99.58| +2452228|56809|2452261|12313|82092|681487|882|677|66077|1437954|509|9104|57|3|12|2|62|487|86|16.23|27.10|13.00|1212.60|1118.00|1395.78|2330.60|67.08|0.00|768.84|1118.00|1185.08|1886.84|1953.92|-277.78| +2452228|56809|2452341|17675|82092|681487|882|677|66077|1437954|509|9104|13|19|3|3|56|487|99|91.61|254.67|160.44|9328.77|15883.56|9069.39|25212.33|317.67|0.00|1764.18|15883.56|16201.23|17647.74|17965.41|6814.17| +2452228|56809|2452254|1173|82092|681487|882|677|66077|1437954|509|9104|37|7|12|2|12|487|76|74.26|96.53|24.13|5502.40|1833.88|5643.76|7336.28|128.37|0.00|2934.36|1833.88|1962.25|4768.24|4896.61|-3809.88| +2452228|56809|2452309|4775|82092|681487|882|677|66077|1437954|509|9104|51|11|19|4|142|487|45|54.41|158.33|15.83|6412.50|712.35|2448.45|7124.85|28.49|0.00|1424.70|712.35|740.84|2137.05|2165.54|-1736.10| +2452228|56809|2452333|5503|82092|681487|882|677|66077|1437954|509|9104|11|5|10|4|1|487|13|2.53|6.80|6.66|1.82|86.58|32.89|88.40|7.79|0.00|34.45|86.58|94.37|121.03|128.82|53.69| +2452228|56809|2452235|7043|82092|681487|882|677|66077|1437954|509|9104|43|5|17|3|82|487|57|44.88|71.80|17.23|3110.49|982.11|2558.16|4092.60|49.10|0.00|490.77|982.11|1031.21|1472.88|1521.98|-1576.05| +2452228|56809|2452279|15833|82092|681487|882|677|66077|1437954|509|9104|31|25|13|5|297|487|50|33.56|100.34|98.33|100.50|4916.50|1678.00|5017.00|442.48|0.00|602.00|4916.50|5358.98|5518.50|5960.98|3238.50| +2452228|56809|2452269|13763|82092|681487|882|677|66077|1437954|509|9104|21|9|9|1|220|487|31|47.32|59.62|47.09|388.43|1459.79|1466.92|1848.22|116.78|0.00|702.15|1459.79|1576.57|2161.94|2278.72|-7.13| +2452228|56809|2452294|10055|82092|681487|882|677|66077|1437954|509|9104|27|1|1|3|8|487|72|35.04|50.80|38.10|914.40|2743.20|2522.88|3657.60|192.02|0.00|1352.88|2743.20|2935.22|4096.08|4288.10|220.32| +2452228|56809|2452242|5223|82092|681487|882|677|66077|1437954|509|9104|37|13|12|5|225|487|81|36.01|90.02|60.31|2406.51|4885.11|2916.81|7291.62|0.00|0.00|656.10|4885.11|4885.11|5541.21|5541.21|1968.30| +2452228|56809|2452230|14329|82092|681487|882|677|66077|1437954|509|9104|7|29|3|4|62|487|23|74.53|220.60|209.57|253.69|4820.11|1714.19|5073.80|433.80|0.00|1522.14|4820.11|5253.91|6342.25|6776.05|3105.92| +2452228|56809|2452269|223|82092|681487|882|677|66077|1437954|509|9104|5|27|5|1|100|487|61|77.83|101.95|56.07|2798.68|3420.27|4747.63|6218.95|102.60|0.00|1243.79|3420.27|3522.87|4664.06|4766.66|-1327.36| +2451794|4336|2451809|391|50957|647279|5201|19923|12527|462632|4536|28858|44|1|2|5|11|488|39|34.65|61.33|45.38|622.05|1769.82|1351.35|2391.87|53.09|0.00|358.41|1769.82|1822.91|2128.23|2181.32|418.47| +2451794|4336|2451801|20|50957|647279|5201|19923|12527|462632|4536|28858|2|20|20|2|121|488|57|8.66|21.39|8.12|756.39|462.84|493.62|1219.23|0.00|0.00|72.96|462.84|462.84|535.80|535.80|-30.78| +2451794|4336|2451846|15517|50957|647279|5201|19923|12527|462632|4536|28858|38|5|19|5|268|488|25|10.95|11.27|8.67|65.00|216.75|273.75|281.75|7.54|91.03|92.75|125.72|133.26|218.47|226.01|-148.03| +2451794|4336|2451901|7975|50957|647279|5201|19923|12527|462632|4536|28858|2|13|5|5|98|488|22|14.01|39.64|22.19|383.90|488.18|308.22|872.08|24.40|0.00|87.12|488.18|512.58|575.30|599.70|179.96| +2451794|4336|2451811|6065|50957|647279|5201|19923|12527|462632|4536|28858|14|26|10|3|39|488|16|16.71|37.09|19.28|284.96|308.48|267.36|593.44|27.76|0.00|130.40|308.48|336.24|438.88|466.64|41.12| +2451794|4336|2451840|11633|50957|647279|5201|19923|12527|462632|4536|28858|44|20|9|2|42|488|40|50.70|64.38|34.12|1210.40|1364.80|2028.00|2575.20|40.94|0.00|978.40|1364.80|1405.74|2343.20|2384.14|-663.20| +2451794|4336|2451881|17285|50957|647279|5201|19923|12527|462632|4536|28858|19|11|12|1|227|488|59|23.19|33.39|32.72|39.53|1930.48|1368.21|1970.01|173.74|0.00|78.47|1930.48|2104.22|2008.95|2182.69|562.27| +2451794|4336|2451898|10507|50957|647279|5201|19923|12527|462632|4536|28858|17|2|2|5|18|488|26|79.33|162.62|128.46|888.16|3339.96|2062.58|4228.12|29.39|2605.16|676.26|734.80|764.19|1411.06|1440.45|-1327.78| +2451542|65892|2451580|15121|81827|1180404|4934|47111|66319|985383|6795|11725|46|28|16|5|281|489|6|59.22|75.80|62.15|81.90|372.90|355.32|454.80|26.10|0.00|227.40|372.90|399.00|600.30|626.40|17.58| +2451542|65892|2451599|16717|81827|1180404|4934|47111|66319|985383|6795|11725|31|19|1|5|276|489|99|32.64|72.46|34.05|3802.59|3370.95|3231.36|7173.54|67.41|0.00|1792.89|3370.95|3438.36|5163.84|5231.25|139.59| +2451542|65892|2451629|14899|81827|1180404|4934|47111|66319|985383|6795|11725|44|8|14|3|175|489|50|15.40|28.18|14.65|676.50|732.50|770.00|1409.00|43.95|0.00|239.50|732.50|776.45|972.00|1015.95|-37.50| +2451542|65892|2451649|7621|81827|1180404|4934|47111|66319|985383|6795|11725|8|1|7|5|26|489|40|53.78|98.41|63.96|1378.00|2558.40|2151.20|3936.40|25.58|0.00|0.00|2558.40|2583.98|2558.40|2583.98|407.20| +2451542|65892|2451636|2443|81827|1180404|4934|47111|66319|985383|6795|11725|10|8|11|2|226|489|29|57.82|170.56|69.92|2918.56|2027.68|1676.78|4946.24|182.49|0.00|1978.38|2027.68|2210.17|4006.06|4188.55|350.90| +2451542|65892|2451615|12286|81827|1180404|4934|47111|66319|985383|6795|11725|52|28|3|1|279|489|78|75.55|86.88|86.01|67.86|6708.78|5892.90|6776.64|536.70|0.00|609.18|6708.78|7245.48|7317.96|7854.66|815.88| +2451542|65892|2451581|4054|81827|1180404|4934|47111|66319|985383|6795|11725|46|14|5|4|156|489|95|25.51|32.90|3.94|2751.20|374.30|2423.45|3125.50|14.97|0.00|437.00|374.30|389.27|811.30|826.27|-2049.15| +2451542|65892|2451576|6860|81827|1180404|4934|47111|66319|985383|6795|11725|56|2|15|1|94|489|83|94.74|235.90|73.12|13510.74|6068.96|7863.42|19579.70|182.06|0.00|9398.09|6068.96|6251.02|15467.05|15649.11|-1794.46| +2451542|65892|2451604|5236|81827|1180404|4934|47111|66319|985383|6795|11725|28|8|15|5|245|489|95|19.45|39.67|28.95|1018.40|2750.25|1847.75|3768.65|0.00|0.00|1356.60|2750.25|2750.25|4106.85|4106.85|902.50| +2452407|68445|2452415|4135|98888|1562855|6469|16036|18815|1839660|384|27998|54|13|7|2|270|490|68|77.04|145.60|34.94|7524.88|2375.92|5238.72|9900.80|118.79|0.00|1880.88|2375.92|2494.71|4256.80|4375.59|-2862.80| +2452407|68445|2452474|9039|98888|1562855|6469|16036|18815|1839660|384|27998|1|15|5|2|269|490|10|91.19|155.93|31.18|1247.50|311.80|911.90|1559.30|15.59|0.00|748.40|311.80|327.39|1060.20|1075.79|-600.10| +2452407|68445|2452476|8557|98888|1562855|6469|16036|18815|1839660|384|27998|9|24|19|1|33|490|74|62.44|70.55|52.20|1357.90|3862.80|4620.56|5220.70|270.39|0.00|2192.62|3862.80|4133.19|6055.42|6325.81|-757.76| +2452407|68445|2452455|13845|98888|1562855|6469|16036|18815|1839660|384|27998|48|21|18|5|212|490|69|42.66|107.07|83.51|1625.64|5762.19|2943.54|7387.83|345.73|0.00|738.30|5762.19|6107.92|6500.49|6846.22|2818.65| +2452407|68445|2452473|5577|98888|1562855|6469|16036|18815|1839660|384|27998|60|25|16|4|232|490|25|81.31|152.04|95.78|1406.50|2394.50|2032.75|3801.00|143.67|0.00|266.00|2394.50|2538.17|2660.50|2804.17|361.75| +2452407|68445|2452522|7015|98888|1562855|6469|16036|18815|1839660|384|27998|13|19|20|3|33|490|72|76.54|130.11|31.22|7120.08|2247.84|5510.88|9367.92|22.47|0.00|3278.16|2247.84|2270.31|5526.00|5548.47|-3263.04| +2452407|68445|2452521|17415|98888|1562855|6469|16036|18815|1839660|384|27998|48|1|1|1|271|490|46|30.66|58.56|50.36|377.20|2316.56|1410.36|2693.76|69.49|0.00|107.64|2316.56|2386.05|2424.20|2493.69|906.20| +2452407|68445|2452511|15003|98888|1562855|6469|16036|18815|1839660|384|27998|36|27|10|1|152|490|9|94.83|267.42|61.50|1853.28|553.50|853.47|2406.78|3.37|215.86|385.02|337.64|341.01|722.66|726.03|-515.83| +2452433|62356|2452442|3369|99321|1102501|434|17967|21336|1531467|2566|31299|18|18|9|4|39|491|78|29.93|38.60|29.33|723.06|2287.74|2334.54|3010.80|68.63|0.00|1474.98|2287.74|2356.37|3762.72|3831.35|-46.80| +2452433|62356|2452470|10272|99321|1102501|434|17967|21336|1531467|2566|31299|43|6|7|3|172|491|9|70.63|117.24|110.20|63.36|991.80|635.67|1055.16|8.33|852.94|52.74|138.86|147.19|191.60|199.93|-496.81| +2452433|62356|2452459|16908|99321|1102501|434|17967|21336|1531467|2566|31299|21|30|16|5|4|491|31|65.38|143.18|61.56|2530.22|1908.36|2026.78|4438.58|0.00|0.00|843.20|1908.36|1908.36|2751.56|2751.56|-118.42| +2452433|62356|2452511|15855|99321|1102501|434|17967|21336|1531467|2566|31299|31|7|2|2|245|491|41|12.19|21.45|14.80|272.65|606.80|499.79|879.45|5.76|491.50|404.26|115.30|121.06|519.56|525.32|-384.49| +2452433|62356|2452506|2499|99321|1102501|434|17967|21336|1531467|2566|31299|54|1|18|5|239|491|24|37.57|74.01|34.04|959.28|816.96|901.68|1776.24|0.00|0.00|852.48|816.96|816.96|1669.44|1669.44|-84.72| +2452433|62356|2452508|12235|99321|1102501|434|17967|21336|1531467|2566|31299|25|7|15|3|284|491|59|93.97|178.54|28.56|8848.82|1685.04|5544.23|10533.86|50.55|0.00|5055.71|1685.04|1735.59|6740.75|6791.30|-3859.19| +2452433|62356|2452438|4825|99321|1102501|434|17967|21336|1531467|2566|31299|33|27|18|5|139|491|71|35.60|91.13|83.83|518.30|5951.93|2527.60|6470.23|0.00|5237.69|2781.78|714.24|714.24|3496.02|3496.02|-1813.36| +2452433|62356|2452541|9105|99321|1102501|434|17967|21336|1531467|2566|31299|1|18|17|5|123|491|96|51.75|55.89|17.88|3648.96|1716.48|4968.00|5365.44|85.82|0.00|2091.84|1716.48|1802.30|3808.32|3894.14|-3251.52| +2452433|62356|2452522|10728|99321|1102501|434|17967|21336|1531467|2566|31299|48|12|18|3|215|491|78|32.59|32.59|25.74|534.30|2007.72|2542.02|2542.02|40.15|0.00|432.12|2007.72|2047.87|2439.84|2479.99|-534.30| +2452433|62356|2452510|15804|99321|1102501|434|17967|21336|1531467|2566|31299|1|13|3|1|212|491|70|86.76|137.94|122.76|1062.60|8593.20|6073.20|9655.80|257.79|0.00|3861.90|8593.20|8850.99|12455.10|12712.89|2520.00| +2452433|62356|2452483|17695|99321|1102501|434|17967|21336|1531467|2566|31299|21|18|10|1|12|491|35|95.29|114.34|102.90|400.40|3601.50|3335.15|4001.90|144.06|0.00|360.15|3601.50|3745.56|3961.65|4105.71|266.35| +2452433|62356|2452508|11739|99321|1102501|434|17967|21336|1531467|2566|31299|1|25|5|5|282|491|42|48.05|118.68|34.41|3539.34|1445.22|2018.10|4984.56|28.03|43.35|1196.16|1401.87|1429.90|2598.03|2626.06|-616.23| +2452433|62356|2452448|6223|99321|1102501|434|17967|21336|1531467|2566|31299|3|13|5|3|241|491|3|51.83|71.52|37.90|100.86|113.70|155.49|214.56|10.23|0.00|107.28|113.70|123.93|220.98|231.21|-41.79| +2451975|17391|2452031|4021|34821|846531|641|13551|39953|139287|1081|35191|47|29|3|4|25|492|24|81.10|240.86|134.88|2543.52|3237.12|1946.40|5780.64|32.37|0.00|115.44|3237.12|3269.49|3352.56|3384.93|1290.72| +2451975|17391|2451985|2251|34821|846531|641|13551|39953|139287|1081|35191|1|25|15|4|176|492|29|93.74|195.91|74.44|3522.63|2158.76|2718.46|5681.39|0.00|0.00|1704.33|2158.76|2158.76|3863.09|3863.09|-559.70| +2451975|17391|2452050|14575|34821|846531|641|13551|39953|139287|1081|35191|27|13|10|1|279|492|71|19.97|50.12|21.55|2028.47|1530.05|1417.87|3558.52|30.60|0.00|1422.84|1530.05|1560.65|2952.89|2983.49|112.18| +2451975|17391|2452058|4757|34821|846531|641|13551|39953|139287|1081|35191|55|3|9|3|210|492|57|26.76|37.19|7.80|1675.23|444.60|1525.32|2119.83|11.55|155.61|42.18|288.99|300.54|331.17|342.72|-1236.33| +2451975|17391|2452082|9759|34821|846531|641|13551|39953|139287|1081|35191|23|11|13|2|214|492|36|69.49|144.53|112.73|1144.80|4058.28|2501.64|5203.08|284.07|0.00|1612.80|4058.28|4342.35|5671.08|5955.15|1556.64| +2451975|17391|2452058|8165|34821|846531|641|13551|39953|139287|1081|35191|47|13|20|5|293|492|48|97.47|291.43|195.25|4616.64|9372.00|4678.56|13988.64|562.32|0.00|6574.56|9372.00|9934.32|15946.56|16508.88|4693.44| +2451975|17391|2452045|14809|34821|846531|641|13551|39953|139287|1081|35191|11|13|8|4|10|492|7|78.12|106.24|53.12|371.84|371.84|546.84|743.68|26.02|0.00|118.93|371.84|397.86|490.77|516.79|-175.00| +2451975|17391|2452043|14337|34821|846531|641|13551|39953|139287|1081|35191|53|29|20|5|23|492|86|1.81|3.07|0.95|182.32|81.70|155.66|264.02|6.53|0.00|86.86|81.70|88.23|168.56|175.09|-73.96| +2451975|17391|2452067|10549|34821|846531|641|13551|39953|139287|1081|35191|15|19|2|4|279|492|43|13.61|29.66|2.07|1186.37|89.01|585.23|1275.38|0.89|0.00|548.25|89.01|89.90|637.26|638.15|-496.22| +2451975|17391|2451996|12245|34821|846531|641|13551|39953|139287|1081|35191|3|9|1|2|30|492|59|87.48|213.45|6.40|12215.95|377.60|5161.32|12593.55|33.98|0.00|1133.39|377.60|411.58|1510.99|1544.97|-4783.72| +2451975|17391|2452092|10045|34821|846531|641|13551|39953|139287|1081|35191|57|3|18|1|237|492|68|50.98|118.78|86.70|2181.44|5895.60|3466.64|8077.04|235.82|0.00|2987.92|5895.60|6131.42|8883.52|9119.34|2428.96| +2451975|17391|2452012|11163|34821|846531|641|13551|39953|139287|1081|35191|27|3|20|3|56|492|36|20.01|38.21|8.02|1086.84|288.72|720.36|1375.56|4.93|124.14|453.60|164.58|169.51|618.18|623.11|-555.78| +2451975|17391|2451990|9979|34821|846531|641|13551|39953|139287|1081|35191|33|9|3|4|172|492|73|1.01|2.37|2.27|7.30|165.71|73.73|173.01|4.97|0.00|37.96|165.71|170.68|203.67|208.64|91.98| +2452191|9273|2452272|6577|75970|1513102|4987|9719|98180|1391658|1868|48953|5|23|18|2|203|493|34|55.53|99.39|78.51|709.92|2669.34|1888.02|3379.26|106.77|0.00|1621.80|2669.34|2776.11|4291.14|4397.91|781.32| +2452191|9273|2452284|6831|75970|1513102|4987|9719|98180|1391658|1868|48953|9|3|9|4|166|493|7|63.06|78.82|58.32|143.50|408.24|441.42|551.74|32.65|0.00|77.21|408.24|440.89|485.45|518.10|-33.18| +2452191|9273|2452303|5997|75970|1513102|4987|9719|98180|1391658|1868|48953|5|23|4|1|25|493|98|30.19|76.98|23.09|5281.22|2262.82|2958.62|7544.04|67.88|0.00|1659.14|2262.82|2330.70|3921.96|3989.84|-695.80| +2452191|9273|2452294|8669|75970|1513102|4987|9719|98180|1391658|1868|48953|19|1|5|4|283|493|97|16.57|44.40|20.86|2283.38|2023.42|1607.29|4306.80|121.40|0.00|602.37|2023.42|2144.82|2625.79|2747.19|416.13| +2452191|9273|2452289|599|75970|1513102|4987|9719|98180|1391658|1868|48953|17|1|3|2|284|493|82|26.85|36.24|30.44|475.60|2496.08|2201.70|2971.68|74.88|0.00|267.32|2496.08|2570.96|2763.40|2838.28|294.38| +2452191|9273|2452308|7013|75970|1513102|4987|9719|98180|1391658|1868|48953|57|19|18|4|297|493|16|83.21|231.32|32.38|3183.04|518.08|1331.36|3701.12|10.36|0.00|518.08|518.08|528.44|1036.16|1046.52|-813.28| +2452191|9273|2452217|2405|75970|1513102|4987|9719|98180|1391658|1868|48953|23|11|20|4|228|493|47|46.36|52.85|24.83|1316.94|1167.01|2178.92|2483.95|81.69|0.00|273.07|1167.01|1248.70|1440.08|1521.77|-1011.91| +2452191|9273|2452224|12447|75970|1513102|4987|9719|98180|1391658|1868|48953|53|13|7|2|67|493|1|64.56|109.75|5.48|104.27|5.48|64.56|109.75|0.00|0.00|24.14|5.48|5.48|29.62|29.62|-59.08| +2452191|9273|2452309|2677|75970|1513102|4987|9719|98180|1391658|1868|48953|29|7|13|2|181|493|30|85.10|185.51|166.95|556.80|5008.50|2553.00|5565.30|0.00|0.00|2281.50|5008.50|5008.50|7290.00|7290.00|2455.50| +2452191|9273|2452209|12189|75970|1513102|4987|9719|98180|1391658|1868|48953|27|9|13|1|29|493|7|45.09|73.49|27.92|318.99|195.44|315.63|514.43|1.95|0.00|77.14|195.44|197.39|272.58|274.53|-120.19| +2452191|9273|2452200|4759|75970|1513102|4987|9719|98180|1391658|1868|48953|45|17|18|2|126|493|68|61.50|148.83|55.06|6376.36|3744.08|4182.00|10120.44|262.08|0.00|5059.88|3744.08|4006.16|8803.96|9066.04|-437.92| +2452097|83199|2452151|13303|26744|578234|2077|29120|59979|1160533|4296|21386|47|29|19|5|41|494|31|40.71|110.73|94.12|514.91|2917.72|1262.01|3432.63|145.88|0.00|0.00|2917.72|3063.60|2917.72|3063.60|1655.71| +2452097|83199|2452217|5125|26744|578234|2077|29120|59979|1160533|4296|21386|25|3|18|4|80|494|77|22.83|60.04|12.60|3652.88|970.20|1757.91|4623.08|48.51|0.00|2125.97|970.20|1018.71|3096.17|3144.68|-787.71| +2452097|83199|2452103|13427|26744|578234|2077|29120|59979|1160533|4296|21386|5|27|5|3|54|494|34|94.96|96.85|44.55|1778.20|1514.70|3228.64|3292.90|136.32|0.00|1317.16|1514.70|1651.02|2831.86|2968.18|-1713.94| +2452097|83199|2452173|17629|26744|578234|2077|29120|59979|1160533|4296|21386|27|9|3|4|255|494|52|56.60|60.56|1.81|3055.00|94.12|2943.20|3149.12|6.58|0.00|1101.88|94.12|100.70|1196.00|1202.58|-2849.08| +2452097|83199|2452126|11653|26744|578234|2077|29120|59979|1160533|4296|21386|7|3|3|1|72|494|64|58.15|159.33|109.93|3161.60|7035.52|3721.60|10197.12|0.00|0.00|3466.88|7035.52|7035.52|10502.40|10502.40|3313.92| +2452097|83199|2452146|1187|26744|578234|2077|29120|59979|1160533|4296|21386|17|9|1|2|23|494|87|80.92|194.20|31.07|14192.31|2703.09|7040.04|16895.40|189.21|0.00|6758.16|2703.09|2892.30|9461.25|9650.46|-4336.95| +2452097|83199|2452118|1055|26744|578234|2077|29120|59979|1160533|4296|21386|53|1|12|3|4|494|34|8.45|14.95|0.44|493.34|14.96|287.30|508.30|1.34|0.00|96.56|14.96|16.30|111.52|112.86|-272.34| +2452097|83199|2452165|17001|26744|578234|2077|29120|59979|1160533|4296|21386|29|17|3|4|118|494|17|75.56|170.01|132.60|635.97|2254.20|1284.52|2890.17|202.87|0.00|462.40|2254.20|2457.07|2716.60|2919.47|969.68| +2452097|83199|2452196|13619|26744|578234|2077|29120|59979|1160533|4296|21386|47|25|9|3|107|494|90|89.11|140.79|91.51|4435.20|8235.90|8019.90|12671.10|566.63|1153.02|1013.40|7082.88|7649.51|8096.28|8662.91|-937.02| +2452097|83199|2452200|13835|26744|578234|2077|29120|59979|1160533|4296|21386|15|15|1|1|54|494|55|37.75|52.85|36.46|901.45|2005.30|2076.25|2906.75|120.31|0.00|1104.40|2005.30|2125.61|3109.70|3230.01|-70.95| +2452097|83199|2452133|9059|26744|578234|2077|29120|59979|1160533|4296|21386|59|7|5|4|113|494|9|27.53|68.27|51.20|153.63|460.80|247.77|614.43|36.86|0.00|30.69|460.80|497.66|491.49|528.35|213.03| +2452268|66367|2452356|6425|88522|712288|2454|25820|75710|166641|6765|42531|27|15|2|1|283|495|55|42.02|96.22|53.88|2328.70|2963.40|2311.10|5292.10|4.74|2489.25|687.50|474.15|478.89|1161.65|1166.39|-1836.95| +2452268|66367|2452344|11325|88522|712288|2454|25820|75710|166641|6765|42531|45|3|11|5|149|495|36|79.43|123.11|14.77|3900.24|531.72|2859.48|4431.96|21.26|0.00|1329.48|531.72|552.98|1861.20|1882.46|-2327.76| +2452268|66367|2452297|11479|88522|712288|2454|25820|75710|166641|6765|42531|41|27|17|3|46|495|67|21.47|45.30|35.33|667.99|2367.11|1438.49|3035.10|213.03|0.00|1486.73|2367.11|2580.14|3853.84|4066.87|928.62| +2452268|66367|2452356|8991|88522|712288|2454|25820|75710|166641|6765|42531|55|7|20|1|255|495|22|69.27|69.96|63.66|138.60|1400.52|1523.94|1539.12|126.04|0.00|15.18|1400.52|1526.56|1415.70|1541.74|-123.42| +2452268|66367|2452273|6777|88522|712288|2454|25820|75710|166641|6765|42531|31|1|9|2|175|495|23|58.75|94.00|69.56|562.12|1599.88|1351.25|2162.00|79.99|0.00|302.68|1599.88|1679.87|1902.56|1982.55|248.63| +2452268|66367|2452361|15525|88522|712288|2454|25820|75710|166641|6765|42531|57|17|19|2|122|495|1|14.54|42.16|35.41|6.75|35.41|14.54|42.16|1.41|0.00|1.68|35.41|36.82|37.09|38.50|20.87| +2452268|66367|2452374|11723|88522|712288|2454|25820|75710|166641|6765|42531|1|15|12|4|253|495|49|68.47|98.59|68.02|1497.93|3332.98|3355.03|4830.91|233.30|0.00|1303.89|3332.98|3566.28|4636.87|4870.17|-22.05| +2452268|66367|2452282|8429|88522|712288|2454|25820|75710|166641|6765|42531|41|11|1|1|291|495|100|6.82|16.91|8.62|829.00|862.00|682.00|1691.00|35.85|413.76|388.00|448.24|484.09|836.24|872.09|-233.76| +2452268|66367|2452384|3363|88522|712288|2454|25820|75710|166641|6765|42531|43|21|3|2|61|495|81|76.75|104.38|32.35|5834.43|2620.35|6216.75|8454.78|4.71|2463.12|1690.47|157.23|161.94|1847.70|1852.41|-6059.52| +2452268|66367|2452317|6469|88522|712288|2454|25820|75710|166641|6765|42531|21|25|5|5|192|495|56|79.43|211.28|181.70|1656.48|10175.20|4448.08|11831.68|0.00|8852.42|5797.12|1322.78|1322.78|7119.90|7119.90|-3125.30| +2452186|27944|2452239|2347|56822|4045|1221|21597|74604|595748|2372|46748|57|29|8|1|24|496|69|49.49|69.28|14.54|3777.06|1003.26|3414.81|4780.32|70.22|0.00|1195.08|1003.26|1073.48|2198.34|2268.56|-2411.55| +2452186|27944|2452194|587|56822|4045|1221|21597|74604|595748|2372|46748|31|17|15|3|184|496|30|94.31|279.15|19.54|7788.30|586.20|2829.30|8374.50|7.91|322.41|4019.70|263.79|271.70|4283.49|4291.40|-2565.51| +2452186|27944|2452250|3085|56822|4045|1221|21597|74604|595748|2372|46748|35|17|17|3|109|496|29|59.70|106.86|59.84|1363.58|1735.36|1731.30|3098.94|86.76|0.00|960.48|1735.36|1822.12|2695.84|2782.60|4.06| +2452186|27944|2452231|5837|56822|4045|1221|21597|74604|595748|2372|46748|11|29|14|3|169|496|40|96.18|247.18|165.61|3262.80|6624.40|3847.20|9887.20|529.95|0.00|4646.80|6624.40|7154.35|11271.20|11801.15|2777.20| +2452186|27944|2452302|1007|56822|4045|1221|21597|74604|595748|2372|46748|37|19|1|5|216|496|46|78.34|228.75|2.28|10417.62|104.88|3603.64|10522.50|9.43|0.00|2946.30|104.88|114.31|3051.18|3060.61|-3498.76| +2452186|27944|2452209|4875|56822|4045|1221|21597|74604|595748|2372|46748|57|19|14|5|6|496|98|13.72|17.56|11.76|568.40|1152.48|1344.56|1720.88|34.11|299.64|756.56|852.84|886.95|1609.40|1643.51|-491.72| +2452186|27944|2452256|14235|56822|4045|1221|21597|74604|595748|2372|46748|29|19|15|3|288|496|34|6.28|6.28|0.75|188.02|25.50|213.52|213.52|0.00|0.00|61.88|25.50|25.50|87.38|87.38|-188.02| +2452186|27944|2452227|14047|56822|4045|1221|21597|74604|595748|2372|46748|21|21|14|4|244|496|90|92.28|143.03|94.39|4377.60|8495.10|8305.20|12872.70|339.80|0.00|3989.70|8495.10|8834.90|12484.80|12824.60|189.90| +2452186|27944|2452193|9153|56822|4045|1221|21597|74604|595748|2372|46748|59|5|16|4|192|496|28|72.50|150.80|113.10|1055.60|3166.80|2030.00|4222.40|221.67|0.00|1815.52|3166.80|3388.47|4982.32|5203.99|1136.80| +2452186|27944|2452221|6019|56822|4045|1221|21597|74604|595748|2372|46748|31|13|11|5|199|496|49|88.87|215.95|15.11|9841.16|740.39|4354.63|10581.55|14.80|0.00|1057.91|740.39|755.19|1798.30|1813.10|-3614.24| +2452186|27944|2452264|16241|56822|4045|1221|21597|74604|595748|2372|46748|53|15|6|4|25|496|1|45.76|117.60|15.28|102.32|15.28|45.76|117.60|0.00|0.00|14.11|15.28|15.28|29.39|29.39|-30.48| +2452186|27944|2452214|10381|56822|4045|1221|21597|74604|595748|2372|46748|33|3|4|1|31|496|41|20.81|35.37|4.95|1247.22|202.95|853.21|1450.17|2.43|81.18|130.38|121.77|124.20|252.15|254.58|-731.44| +2452186|27944|2452301|16915|56822|4045|1221|21597|74604|595748|2372|46748|53|17|4|1|278|496|70|5.29|14.60|9.19|378.70|643.30|370.30|1022.00|19.29|0.00|408.80|643.30|662.59|1052.10|1071.39|273.00| +2452186|27944|2452230|12467|56822|4045|1221|21597|74604|595748|2372|46748|13|13|17|2|277|496|81|53.12|151.92|6.07|11813.85|491.67|4302.72|12305.52|39.38|54.08|2707.02|437.59|476.97|3144.61|3183.99|-3865.13| +2452186|27944|2452288|14567|56822|4045|1221|21597|74604|595748|2372|46748|39|19|13|1|146|496|56|62.34|98.49|89.62|496.72|5018.72|3491.04|5515.44|0.00|0.00|2426.48|5018.72|5018.72|7445.20|7445.20|1527.68| +2452614|15208|2452710|11605|78186|267897|6761|30472|20123|1367017|3558|28069|21|24|6|2|91|497|29|58.92|122.55|50.24|2096.99|1456.96|1708.68|3553.95|0.00|903.31|710.79|553.65|553.65|1264.44|1264.44|-1155.03| +2452614|15208|2452619|1813|78186|267897|6761|30472|20123|1367017|3558|28069|60|27|15|4|40|497|64|80.19|221.32|46.47|11190.40|2974.08|5132.16|14164.48|208.18|0.00|2124.16|2974.08|3182.26|5098.24|5306.42|-2158.08| +2452614|15208|2452691|8757|78186|267897|6761|30472|20123|1367017|3558|28069|13|18|1|3|195|497|37|28.59|79.48|2.38|2852.70|88.06|1057.83|2940.76|5.28|0.00|852.48|88.06|93.34|940.54|945.82|-969.77| +2452614|15208|2452656|4909|78186|267897|6761|30472|20123|1367017|3558|28069|27|9|6|5|91|497|92|47.74|80.20|15.23|5977.24|1401.16|4392.08|7378.40|0.00|868.71|2803.24|532.45|532.45|3335.69|3335.69|-3859.63| +2452614|15208|2452626|11871|78186|267897|6761|30472|20123|1367017|3558|28069|6|1|12|5|71|497|12|57.47|110.34|75.03|423.72|900.36|689.64|1324.08|29.53|162.06|317.76|738.30|767.83|1056.06|1085.59|48.66| +2452614|15208|2452661|9141|78186|267897|6761|30472|20123|1367017|3558|28069|6|15|10|2|194|497|7|27.14|29.85|26.86|20.93|188.02|189.98|208.95|1.88|0.00|50.12|188.02|189.90|238.14|240.02|-1.96| +2452614|15208|2452713|12786|78186|267897|6761|30472|20123|1367017|3558|28069|39|7|15|4|233|497|39|39.00|69.42|68.03|54.21|2653.17|1521.00|2707.38|79.59|0.00|1055.73|2653.17|2732.76|3708.90|3788.49|1132.17| +2452614|15208|2452668|9201|78186|267897|6761|30472|20123|1367017|3558|28069|13|24|7|2|207|497|79|93.45|195.31|171.87|1851.76|13577.73|7382.55|15429.49|999.32|1086.21|4319.72|12491.52|13490.84|16811.24|17810.56|5108.97| +2452614|15208|2452665|552|78186|267897|6761|30472|20123|1367017|3558|28069|55|7|10|1|52|497|36|31.41|68.47|8.21|2169.36|295.56|1130.76|2464.92|14.77|0.00|763.92|295.56|310.33|1059.48|1074.25|-835.20| +2452614|15208|2452655|349|78186|267897|6761|30472|20123|1367017|3558|28069|31|15|3|5|270|497|68|57.37|68.27|23.89|3017.84|1624.52|3901.16|4642.36|0.00|1559.53|1717.00|64.99|64.99|1781.99|1781.99|-3836.17| +2452614|15208|2452701|17202|78186|267897|6761|30472|20123|1367017|3558|28069|55|24|19|1|213|497|69|46.28|81.45|33.39|3316.14|2303.91|3193.32|5620.05|161.27|0.00|899.07|2303.91|2465.18|3202.98|3364.25|-889.41| +2452614|15208|2452666|385|78186|267897|6761|30472|20123|1367017|3558|28069|25|19|17|1|68|497|57|61.39|131.98|21.11|6319.59|1203.27|3499.23|7522.86|0.00|0.00|1278.51|1203.27|1203.27|2481.78|2481.78|-2295.96| +2452614|15208|2452695|16368|78186|267897|6761|30472|20123|1367017|3558|28069|31|7|15|3|14|497|15|71.00|144.84|76.76|1021.20|1151.40|1065.00|2172.60|46.05|0.00|868.95|1151.40|1197.45|2020.35|2066.40|86.40| +2452614|15208|2452702|12553|78186|267897|6761|30472|20123|1367017|3558|28069|12|19|11|5|119|497|19|22.44|63.28|30.37|625.29|577.03|426.36|1202.32|34.62|0.00|589.00|577.03|611.65|1166.03|1200.65|150.67| +2452614|15208|2452704|10710|78186|267897|6761|30472|20123|1367017|3558|28069|31|25|20|5|177|497|56|25.23|62.57|45.67|946.40|2557.52|1412.88|3503.92|102.30|0.00|385.28|2557.52|2659.82|2942.80|3045.10|1144.64| +2452614|15208|2452710|4542|78186|267897|6761|30472|20123|1367017|3558|28069|37|12|13|5|87|497|56|37.22|59.92|39.54|1141.28|2214.24|2084.32|3355.52|3.98|2147.81|1644.16|66.43|70.41|1710.59|1714.57|-2017.89| +2452621|73628|2452683|12627|76444|770324|2957|48037|81496|1079059|430|4710|6|7|15|1|14|498|8|84.78|252.64|53.05|1596.72|424.40|678.24|2021.12|14.26|246.15|889.28|178.25|192.51|1067.53|1081.79|-499.99| +2452621|73628|2452677|3433|76444|770324|2957|48037|81496|1079059|430|4710|19|9|13|1|179|498|15|13.27|37.55|21.77|236.70|326.55|199.05|563.25|6.53|0.00|84.45|326.55|333.08|411.00|417.53|127.50| +2452621|73628|2452730|14823|76444|770324|2957|48037|81496|1079059|430|4710|27|13|7|5|300|498|71|94.00|208.68|114.77|6667.61|8148.67|6674.00|14816.28|162.97|0.00|2369.98|8148.67|8311.64|10518.65|10681.62|1474.67| +2452621|73628|2452705|1104|76444|770324|2957|48037|81496|1079059|430|4710|25|3|11|3|52|498|98|78.58|179.94|143.95|3527.02|14107.10|7700.84|17634.12|987.49|0.00|2115.82|14107.10|15094.59|16222.92|17210.41|6406.26| +2452621|73628|2452725|17106|76444|770324|2957|48037|81496|1079059|430|4710|43|15|11|2|40|498|93|81.31|168.31|97.61|6575.10|9077.73|7561.83|15652.83|635.44|0.00|468.72|9077.73|9713.17|9546.45|10181.89|1515.90| +2452621|73628|2452647|11085|76444|770324|2957|48037|81496|1079059|430|4710|48|24|10|5|103|498|3|16.20|36.77|3.30|100.41|9.90|48.60|110.31|0.25|7.02|13.23|2.88|3.13|16.11|16.36|-45.72| +2452621|73628|2452737|1675|76444|770324|2957|48037|81496|1079059|430|4710|24|30|18|1|234|498|71|62.71|141.09|134.03|501.26|9516.13|4452.41|10017.39|68.51|6090.32|4908.23|3425.81|3494.32|8334.04|8402.55|-1026.60| +2452621|73628|2452716|10117|76444|770324|2957|48037|81496|1079059|430|4710|18|3|12|5|258|498|47|25.24|50.98|49.45|71.91|2324.15|1186.28|2396.06|19.98|325.38|694.66|1998.77|2018.75|2693.43|2713.41|812.49| +2452621|73628|2452635|17779|76444|770324|2957|48037|81496|1079059|430|4710|45|25|10|2|237|498|72|48.49|112.98|105.07|569.52|7565.04|3491.28|8134.56|605.20|0.00|3823.20|7565.04|8170.24|11388.24|11993.44|4073.76| +2452621|73628|2452648|17263|76444|770324|2957|48037|81496|1079059|430|4710|25|19|17|2|264|498|100|4.07|8.83|2.20|663.00|220.00|407.00|883.00|17.60|0.00|441.00|220.00|237.60|661.00|678.60|-187.00| +2452621|73628|2452739|7149|76444|770324|2957|48037|81496|1079059|430|4710|21|25|15|5|72|498|27|45.05|67.12|24.16|1159.92|652.32|1216.35|1812.24|45.66|0.00|797.31|652.32|697.98|1449.63|1495.29|-564.03| +2452621|73628|2452711|10398|76444|770324|2957|48037|81496|1079059|430|4710|12|19|1|2|103|498|30|89.72|106.76|84.34|672.60|2530.20|2691.60|3202.80|151.81|0.00|384.30|2530.20|2682.01|2914.50|3066.31|-161.40| +2452621|73628|2452714|765|76444|770324|2957|48037|81496|1079059|430|4710|31|12|19|4|179|498|85|81.68|184.59|118.13|5649.10|10041.05|6942.80|15690.15|602.46|0.00|6275.55|10041.05|10643.51|16316.60|16919.06|3098.25| +2452621|73628|2452643|5586|76444|770324|2957|48037|81496|1079059|430|4710|7|9|19|5|234|498|19|4.57|5.16|3.56|30.40|67.64|86.83|98.04|1.32|34.49|28.31|33.15|34.47|61.46|62.78|-53.68| +2452621|73628|2452634|12445|76444|770324|2957|48037|81496|1079059|430|4710|48|3|10|5|3|498|34|59.34|178.02|108.59|2360.62|3692.06|2017.56|6052.68|258.44|0.00|1513.00|3692.06|3950.50|5205.06|5463.50|1674.50| +2451093|75554|2451149|13402|12771|689385|46|29907|88096|445909|2596|13657|38|28|10|5|157|499|22|60.17|110.11|15.41|2083.40|339.02|1323.74|2422.42|6.78|254.26|1186.90|84.76|91.54|1271.66|1278.44|-1238.98| +2451093|75554|2451125|16807|12771|689385|46|29907|88096|445909|2596|13657|1|22|3|2|278|499|60|81.42|164.46|9.86|9276.00|591.60|4885.20|9867.60|5.91|0.00|2861.40|591.60|597.51|3453.00|3458.91|-4293.60| +2451093|75554|2451147|16297|12771|689385|46|29907|88096|445909|2596|13657|46|28|6|2|78|499|1|82.52|217.02|206.16|10.86|206.16|82.52|217.02|18.55|0.00|54.25|206.16|224.71|260.41|278.96|123.64| +2451093|75554|2451172|11278|12771|689385|46|29907|88096|445909|2596|13657|28|13|9|1|283|499|35|70.66|151.21|13.60|4816.35|476.00|2473.10|5292.35|23.80|0.00|105.70|476.00|499.80|581.70|605.50|-1997.10| +2451093|75554|2451129|11338|12771|689385|46|29907|88096|445909|2596|13657|44|16|15|3|34|499|26|63.96|63.96|60.76|83.20|1579.76|1662.96|1662.96|47.39|0.00|515.32|1579.76|1627.15|2095.08|2142.47|-83.20| +2451093|75554|2451118|6451|12771|689385|46|29907|88096|445909|2596|13657|31|25|10|2|161|499|19|33.73|46.88|29.06|338.58|552.14|640.87|890.72|49.69|0.00|329.46|552.14|601.83|881.60|931.29|-88.73| +2451093|75554|2451202|14542|12771|689385|46|29907|88096|445909|2596|13657|14|28|5|2|299|499|27|86.45|221.31|203.60|478.17|5497.20|2334.15|5975.37|274.86|0.00|1732.59|5497.20|5772.06|7229.79|7504.65|3163.05| +2451093|75554|2451181|11344|12771|689385|46|29907|88096|445909|2596|13657|58|4|16|2|215|499|96|51.27|87.15|61.00|2510.40|5856.00|4921.92|8366.40|292.80|0.00|1171.20|5856.00|6148.80|7027.20|7320.00|934.08| +2451093|75554|2451209|10472|12771|689385|46|29907|88096|445909|2596|13657|14|8|9|3|100|499|65|84.73|172.00|18.92|9950.20|1229.80|5507.45|11180.00|86.08|0.00|2124.20|1229.80|1315.88|3354.00|3440.08|-4277.65| +2451093|75554|2451169|6124|12771|689385|46|29907|88096|445909|2596|13657|49|1|20|1|252|499|61|75.40|186.23|40.97|8860.86|2499.17|4599.40|11360.03|99.96|0.00|5225.26|2499.17|2599.13|7724.43|7824.39|-2100.23| +2451093|75554|2451204|17224|12771|689385|46|29907|88096|445909|2596|13657|46|7|4|4|27|499|20|55.12|96.46|96.46|0.00|1929.20|1102.40|1929.20|48.61|1389.02|540.00|540.18|588.79|1080.18|1128.79|-562.22| +2451324|77732|2451332|6115|57550|1868997|444|25091|1773|529259|2232|48479|38|22|4|4|251|500|1|64.82|87.50|19.25|68.25|19.25|64.82|87.50|1.54|0.00|14.87|19.25|20.79|34.12|35.66|-45.57| +2451324|77732|2451417|3538|57550|1868997|444|25091|1773|529259|2232|48479|50|8|14|3|122|500|32|65.97|130.62|52.24|2508.16|1671.68|2111.04|4179.84|90.27|668.67|584.96|1003.01|1093.28|1587.97|1678.24|-1108.03| +2451324|77732|2451327|13760|57550|1868997|444|25091|1773|529259|2232|48479|50|19|1|2|283|500|55|32.86|64.73|54.37|569.80|2990.35|1807.30|3560.15|119.61|0.00|960.85|2990.35|3109.96|3951.20|4070.81|1183.05| +2451324|77732|2451332|13766|57550|1868997|444|25091|1773|529259|2232|48479|43|13|2|4|299|500|85|83.69|97.08|96.10|83.30|8168.50|7113.65|8251.80|245.05|0.00|906.95|8168.50|8413.55|9075.45|9320.50|1054.85| +2451324|77732|2451407|9649|57550|1868997|444|25091|1773|529259|2232|48479|38|25|6|4|59|500|100|46.34|125.11|30.02|9509.00|3002.00|4634.00|12511.00|30.02|0.00|2877.00|3002.00|3032.02|5879.00|5909.02|-1632.00| +2451324|77732|2451439|14618|57550|1868997|444|25091|1773|529259|2232|48479|8|10|7|3|249|500|66|84.06|105.91|91.08|978.78|6011.28|5547.96|6990.06|3.60|5650.60|2166.78|360.68|364.28|2527.46|2531.06|-5187.28| +2451324|77732|2451361|17869|57550|1868997|444|25091|1773|529259|2232|48479|28|25|8|1|284|500|40|49.23|63.50|30.48|1320.80|1219.20|1969.20|2540.00|36.57|0.00|736.40|1219.20|1255.77|1955.60|1992.17|-750.00| +2451324|77732|2451350|7186|57550|1868997|444|25091|1773|529259|2232|48479|32|28|6|2|55|500|24|34.29|74.40|26.78|1142.88|642.72|822.96|1785.60|19.28|0.00|392.64|642.72|662.00|1035.36|1054.64|-180.24| +2451324|77732|2451404|5774|57550|1868997|444|25091|1773|529259|2232|48479|1|4|5|2|196|500|54|17.88|47.56|41.85|308.34|2259.90|965.52|2568.24|203.39|0.00|667.44|2259.90|2463.29|2927.34|3130.73|1294.38| +2451324|77732|2451338|6782|57550|1868997|444|25091|1773|529259|2232|48479|8|8|15|5|72|500|8|8.47|13.29|7.84|43.60|62.72|67.76|106.32|1.25|0.00|1.04|62.72|63.97|63.76|65.01|-5.04| +2451324|77732|2451429|3559|57550|1868997|444|25091|1773|529259|2232|48479|31|2|5|1|117|500|38|64.98|131.90|108.15|902.50|4109.70|2469.24|5012.20|82.19|0.00|1453.50|4109.70|4191.89|5563.20|5645.39|1640.46| +2451324|77732|2451369|2300|57550|1868997|444|25091|1773|529259|2232|48479|49|2|9|1|83|500|56|34.04|59.22|31.97|1526.00|1790.32|1906.24|3316.32|71.61|0.00|464.24|1790.32|1861.93|2254.56|2326.17|-115.92| +2451412|51355|2451488|16264|71770|76393|7063|39686|18574|613306|1649|2672|8|28|8|3|158|501|31|43.23|86.89|73.85|404.24|2289.35|1340.13|2693.59|68.68|0.00|1292.70|2289.35|2358.03|3582.05|3650.73|949.22| +2451412|51355|2451478|3661|71770|76393|7063|39686|18574|613306|1649|2672|40|25|12|4|93|501|23|57.64|140.64|28.12|2587.96|646.76|1325.72|3234.72|32.33|0.00|32.20|646.76|679.09|678.96|711.29|-678.96| +2451412|51355|2451509|8230|71770|76393|7063|39686|18574|613306|1649|2672|7|28|16|2|198|501|83|52.91|107.93|2.15|8779.74|178.45|4391.53|8958.19|5.35|0.00|2776.35|178.45|183.80|2954.80|2960.15|-4213.08| +2451412|51355|2451519|11788|71770|76393|7063|39686|18574|613306|1649|2672|46|16|14|4|204|501|6|37.56|80.75|62.17|111.48|373.02|225.36|484.50|11.19|0.00|77.52|373.02|384.21|450.54|461.73|147.66| +2451412|51355|2451479|15134|71770|76393|7063|39686|18574|613306|1649|2672|22|25|4|3|159|501|85|62.24|183.60|115.66|5774.90|9831.10|5290.40|15606.00|76.68|2162.84|3745.10|7668.26|7744.94|11413.36|11490.04|2377.86| +2451412|51355|2451510|7222|71770|76393|7063|39686|18574|613306|1649|2672|38|10|2|1|47|501|55|13.91|31.29|18.46|705.65|1015.30|765.05|1720.95|40.61|0.00|85.80|1015.30|1055.91|1101.10|1141.71|250.25| +2451412|51355|2451460|14221|71770|76393|7063|39686|18574|613306|1649|2672|31|25|7|4|41|501|77|40.71|85.08|82.52|197.12|6354.04|3134.67|6551.16|0.00|0.00|1965.04|6354.04|6354.04|8319.08|8319.08|3219.37| +2451412|51355|2451506|14674|71770|76393|7063|39686|18574|613306|1649|2672|4|20|9|2|95|501|25|47.53|100.28|62.17|952.75|1554.25|1188.25|2507.00|77.71|0.00|50.00|1554.25|1631.96|1604.25|1681.96|366.00| +2451412|51355|2451435|5005|71770|76393|7063|39686|18574|613306|1649|2672|22|1|3|2|81|501|85|18.73|39.52|32.01|638.35|2720.85|1592.05|3359.20|81.62|0.00|872.95|2720.85|2802.47|3593.80|3675.42|1128.80| +2451412|51355|2451505|16531|71770|76393|7063|39686|18574|613306|1649|2672|46|22|17|5|13|501|10|28.11|60.43|57.40|30.30|574.00|281.10|604.30|5.74|0.00|0.00|574.00|579.74|574.00|579.74|292.90| +2451412|51355|2451468|13375|71770|76393|7063|39686|18574|613306|1649|2672|34|20|17|5|253|501|89|82.91|195.66|156.52|3483.46|13930.28|7378.99|17413.74|323.18|5850.71|8706.87|8079.57|8402.75|16786.44|17109.62|700.58| +2451412|51355|2451434|7867|71770|76393|7063|39686|18574|613306|1649|2672|52|19|16|4|35|501|82|97.28|139.11|84.85|4449.32|6957.70|7976.96|11407.02|417.46|0.00|2737.16|6957.70|7375.16|9694.86|10112.32|-1019.26| +2451853|69947|2451928|6878|8602|543108|6861|20967|52387|61829|4532|26238|29|17|3|2|51|502|52|37.26|82.34|26.34|2912.00|1369.68|1937.52|4281.68|41.09|0.00|42.64|1369.68|1410.77|1412.32|1453.41|-567.84| +2451853|69947|2451969|6524|8602|543108|6861|20967|52387|61829|4532|26238|50|14|20|2|284|502|54|42.63|116.80|19.85|5235.30|1071.90|2302.02|6307.20|75.03|0.00|2775.06|1071.90|1146.93|3846.96|3921.99|-1230.12| +2451853|69947|2451916|15037|8602|543108|6861|20967|52387|61829|4532|26238|47|23|7|5|283|502|33|25.15|29.92|1.49|938.19|49.17|829.95|987.36|1.47|0.00|98.67|49.17|50.64|147.84|149.31|-780.78| +2451853|69947|2451969|17681|8602|543108|6861|20967|52387|61829|4532|26238|35|8|5|2|42|502|92|29.93|64.64|33.61|2854.76|3092.12|2753.56|5946.88|123.68|0.00|1724.08|3092.12|3215.80|4816.20|4939.88|338.56| +2451853|69947|2451861|17491|8602|543108|6861|20967|52387|61829|4532|26238|29|19|18|1|94|502|77|11.44|34.20|31.46|210.98|2422.42|880.88|2633.40|121.12|0.00|52.36|2422.42|2543.54|2474.78|2595.90|1541.54| +2451853|69947|2451911|16664|8602|543108|6861|20967|52387|61829|4532|26238|29|19|5|5|175|502|9|74.54|174.42|106.39|612.27|957.51|670.86|1569.78|19.15|0.00|784.89|957.51|976.66|1742.40|1761.55|286.65| +2451853|69947|2451903|4165|8602|543108|6861|20967|52387|61829|4532|26238|19|17|18|2|216|502|49|1.53|2.46|1.08|67.62|52.92|74.97|120.54|1.52|27.51|57.82|25.41|26.93|83.23|84.75|-49.56| +2451853|69947|2451903|8803|8602|543108|6861|20967|52387|61829|4532|26238|25|29|4|3|42|502|62|30.72|58.67|35.20|1455.14|2182.40|1904.64|3637.54|10.03|1680.44|799.80|501.96|511.99|1301.76|1311.79|-1402.68| +2451853|69947|2451927|17767|8602|543108|6861|20967|52387|61829|4532|26238|55|13|16|5|184|502|66|78.39|194.40|19.44|11547.36|1283.04|5173.74|12830.40|12.83|0.00|3720.42|1283.04|1295.87|5003.46|5016.29|-3890.70| +2451894|49231|2451986|9985|21419|693546|2473|46153|21419|693546|2473|46153|49|25|1|4|277|503|78|45.65|82.62|8.26|5800.08|644.28|3560.70|6444.36|51.54|0.00|2577.12|644.28|695.82|3221.40|3272.94|-2916.42| +2451894|49231|2452002|7943|21419|693546|2473|46153|21419|693546|2473|46153|31|7|12|2|65|503|42|56.40|99.26|57.57|1750.98|2417.94|2368.80|4168.92|88.25|652.84|1917.30|1765.10|1853.35|3682.40|3770.65|-603.70| +2451894|49231|2451913|4658|21419|693546|2473|46153|21419|693546|2473|46153|59|8|7|2|237|503|21|27.37|78.00|72.54|114.66|1523.34|574.77|1638.00|60.93|0.00|425.88|1523.34|1584.27|1949.22|2010.15|948.57| +2451894|49231|2451946|4106|21419|693546|2473|46153|21419|693546|2473|46153|5|8|17|4|191|503|69|89.31|266.14|135.73|8998.29|9365.37|6162.39|18363.66|187.30|0.00|917.70|9365.37|9552.67|10283.07|10470.37|3202.98| +2451894|49231|2451897|4025|21419|693546|2473|46153|21419|693546|2473|46153|29|29|12|2|29|503|68|28.08|30.32|3.63|1814.92|246.84|1909.44|2061.76|0.00|0.00|453.56|246.84|246.84|700.40|700.40|-1662.60| +2451894|49231|2451973|1879|21419|693546|2473|46153|21419|693546|2473|46153|49|29|18|3|188|503|28|77.25|194.67|182.98|327.32|5123.44|2163.00|5450.76|0.00|0.00|217.84|5123.44|5123.44|5341.28|5341.28|2960.44| +2451894|49231|2451919|11150|21419|693546|2473|46153|21419|693546|2473|46153|2|8|8|1|224|503|11|24.01|71.78|20.81|560.67|228.91|264.11|789.58|8.33|109.87|47.30|119.04|127.37|166.34|174.67|-145.07| +2451894|49231|2451989|7075|21419|693546|2473|46153|21419|693546|2473|46153|49|29|15|5|135|503|4|23.76|52.03|12.48|158.20|49.92|95.04|208.12|0.99|0.00|91.56|49.92|50.91|141.48|142.47|-45.12| +2451894|49231|2451975|7543|21419|693546|2473|46153|21419|693546|2473|46153|19|26|1|5|49|503|23|57.91|170.83|121.28|1139.65|2789.44|1331.93|3929.09|0.00|0.00|196.42|2789.44|2789.44|2985.86|2985.86|1457.51| +2451894|49231|2451978|10706|21419|693546|2473|46153|21419|693546|2473|46153|43|26|19|4|186|503|53|13.26|27.05|14.60|659.85|773.80|702.78|1433.65|3.17|456.54|71.55|317.26|320.43|388.81|391.98|-385.52| +2452635|2269|2452723|5328|13119|563901|1083|9065|12877|129930|2258|20640|55|7|20|1|82|504|47|90.43|215.22|68.87|6878.45|3236.89|4250.21|10115.34|129.47|0.00|3438.99|3236.89|3366.36|6675.88|6805.35|-1013.32| +2452635|2269|2452654|5449|13119|563901|1083|9065|12877|129930|2258|20640|60|24|7|2|64|504|19|76.01|174.82|148.59|498.37|2823.21|1444.19|3321.58|225.85|0.00|298.87|2823.21|3049.06|3122.08|3347.93|1379.02| +2452635|2269|2452646|13267|13119|563901|1083|9065|12877|129930|2258|20640|6|3|11|3|202|504|52|9.83|19.56|7.04|651.04|366.08|511.16|1017.12|7.32|0.00|386.36|366.08|373.40|752.44|759.76|-145.08| +2452635|2269|2452717|15085|13119|563901|1083|9065|12877|129930|2258|20640|42|3|13|2|268|504|86|99.50|269.64|83.58|16001.16|7187.88|8557.00|23189.04|503.15|0.00|5797.26|7187.88|7691.03|12985.14|13488.29|-1369.12| +2452635|2269|2452746|2358|13119|563901|1083|9065|12877|129930|2258|20640|37|3|18|1|57|504|67|6.56|17.90|4.47|899.81|299.49|439.52|1199.30|8.98|0.00|191.62|299.49|308.47|491.11|500.09|-140.03| +2452635|2269|2452683|7488|13119|563901|1083|9065|12877|129930|2258|20640|45|30|10|2|277|504|79|4.46|8.16|4.48|290.72|353.92|352.34|644.64|28.31|0.00|218.83|353.92|382.23|572.75|601.06|1.58| +2452635|2269|2452693|14875|13119|563901|1083|9065|12877|129930|2258|20640|45|25|20|2|237|504|2|75.08|79.58|6.36|146.44|12.72|150.16|159.16|1.14|0.00|27.04|12.72|13.86|39.76|40.90|-137.44| +2452635|2269|2452731|17121|13119|563901|1083|9065|12877|129930|2258|20640|21|27|14|2|240|504|58|63.49|174.59|69.83|6076.08|4050.14|3682.42|10126.22|0.00|0.00|1113.60|4050.14|4050.14|5163.74|5163.74|367.72| +2452635|2269|2452718|17299|13119|563901|1083|9065|12877|129930|2258|20640|15|18|17|3|206|504|64|88.92|200.07|24.00|11268.48|1536.00|5690.88|12804.48|0.00|0.00|3713.28|1536.00|1536.00|5249.28|5249.28|-4154.88| +2452635|2269|2452685|11961|13119|563901|1083|9065|12877|129930|2258|20640|25|7|15|1|247|504|64|53.20|88.31|21.19|4295.68|1356.16|3404.80|5651.84|4.06|949.31|2204.16|406.85|410.91|2611.01|2615.07|-2997.95| +2452635|2269|2452713|1633|13119|563901|1083|9065|12877|129930|2258|20640|42|7|7|5|223|504|30|85.77|109.78|102.09|230.70|3062.70|2573.10|3293.40|275.64|0.00|0.00|3062.70|3338.34|3062.70|3338.34|489.60| +2452635|2269|2452656|12012|13119|563901|1083|9065|12877|129930|2258|20640|37|19|1|4|170|504|34|46.64|104.47|56.41|1634.04|1917.94|1585.76|3551.98|134.25|0.00|106.42|1917.94|2052.19|2024.36|2158.61|332.18| +2452635|2269|2452638|5301|13119|563901|1083|9065|12877|129930|2258|20640|37|13|18|1|264|504|71|51.12|146.71|48.41|6979.30|3437.11|3629.52|10416.41|103.11|0.00|4061.91|3437.11|3540.22|7499.02|7602.13|-192.41| +2451338|36070|2451416|6416|53644|1122960|4629|32745|84616|948886|3046|49115|31|19|1|5|6|505|29|29.57|40.21|19.70|594.79|571.30|857.53|1166.09|45.70|0.00|408.03|571.30|617.00|979.33|1025.03|-286.23| +2451338|36070|2451350|13729|53644|1122960|4629|32745|84616|948886|3046|49115|4|22|1|2|106|505|7|94.84|201.06|114.60|605.22|802.20|663.88|1407.42|72.19|0.00|675.50|802.20|874.39|1477.70|1549.89|138.32| +2451338|36070|2451402|15196|53644|1122960|4629|32745|84616|948886|3046|49115|22|1|20|1|5|505|55|23.87|56.57|24.89|1742.40|1368.95|1312.85|3111.35|27.37|0.00|652.85|1368.95|1396.32|2021.80|2049.17|56.10| +2451338|36070|2451358|13456|53644|1122960|4629|32745|84616|948886|3046|49115|58|13|4|1|58|505|74|62.92|71.72|27.97|3237.50|2069.78|4656.08|5307.28|186.28|0.00|2016.50|2069.78|2256.06|4086.28|4272.56|-2586.30| +2451338|36070|2451382|8596|53644|1122960|4629|32745|84616|948886|3046|49115|14|28|2|1|193|505|67|75.14|89.41|39.34|3354.69|2635.78|5034.38|5990.47|237.22|0.00|359.12|2635.78|2873.00|2994.90|3232.12|-2398.60| +2451338|36070|2451373|9154|53644|1122960|4629|32745|84616|948886|3046|49115|37|7|9|1|86|505|21|93.13|123.86|87.94|754.32|1846.74|1955.73|2601.06|73.86|0.00|1118.25|1846.74|1920.60|2964.99|3038.85|-108.99| +2451338|36070|2451405|6290|53644|1122960|4629|32745|84616|948886|3046|49115|40|4|5|2|279|505|61|44.43|52.42|43.50|544.12|2653.50|2710.23|3197.62|111.44|1061.40|1438.38|1592.10|1703.54|3030.48|3141.92|-1118.13| +2451338|36070|2451438|15091|53644|1122960|4629|32745|84616|948886|3046|49115|25|8|17|4|251|505|12|47.01|74.27|33.42|490.20|401.04|564.12|891.24|4.01|0.00|26.64|401.04|405.05|427.68|431.69|-163.08| +2451338|36070|2451422|8242|53644|1122960|4629|32745|84616|948886|3046|49115|52|7|1|3|78|505|3|27.90|49.10|9.82|117.84|29.46|83.70|147.30|2.65|0.00|4.41|29.46|32.11|33.87|36.52|-54.24| +2451338|36070|2451435|1886|53644|1122960|4629|32745|84616|948886|3046|49115|43|28|11|4|250|505|23|67.63|120.38|16.85|2381.19|387.55|1555.49|2768.74|34.87|0.00|442.98|387.55|422.42|830.53|865.40|-1167.94| +2451338|36070|2451378|13454|53644|1122960|4629|32745|84616|948886|3046|49115|49|10|10|1|205|505|93|3.71|8.16|0.24|736.56|22.32|345.03|758.88|0.22|0.00|356.19|22.32|22.54|378.51|378.73|-322.71| +2451338|36070|2451355|2149|53644|1122960|4629|32745|84616|948886|3046|49115|19|25|6|4|262|505|46|95.53|245.51|105.56|6437.70|4855.76|4394.38|11293.46|291.34|0.00|790.28|4855.76|5147.10|5646.04|5937.38|461.38| +2451338|36070|2451433|5818|53644|1122960|4629|32745|84616|948886|3046|49115|50|14|7|4|81|505|68|86.29|139.78|0.00|9505.04|0.00|5867.72|9505.04|0.00|0.00|1330.08|0.00|0.00|1330.08|1330.08|-5867.72| +2451338|36070|2451348|14635|53644|1122960|4629|32745|84616|948886|3046|49115|14|14|5|5|75|505|18|37.31|71.26|46.31|449.10|833.58|671.58|1282.68|0.00|0.00|641.34|833.58|833.58|1474.92|1474.92|162.00| +2451338|36070|2451432|13108|53644|1122960|4629|32745|84616|948886|3046|49115|52|16|4|1|68|505|34|99.81|122.76|63.83|2003.62|2170.22|3393.54|4173.84|0.00|0.00|1669.40|2170.22|2170.22|3839.62|3839.62|-1223.32| +2451338|36070|2451435|9415|53644|1122960|4629|32745|84616|948886|3046|49115|8|14|1|1|144|505|3|44.79|79.72|64.57|45.45|193.71|134.37|239.16|3.87|0.00|35.85|193.71|197.58|229.56|233.43|59.34| +2451896|67253|2451973|4541|35982|1629274|613|25905|35982|1629274|613|25905|49|5|4|2|223|506|19|6.46|18.41|5.52|244.91|104.88|122.74|349.79|0.66|97.53|6.84|7.35|8.01|14.19|14.85|-115.39| +2451896|67253|2451907|653|35982|1629274|613|25905|35982|1629274|613|25905|26|23|20|3|216|506|57|33.05|52.21|39.67|714.78|2261.19|1883.85|2975.97|90.44|0.00|1309.29|2261.19|2351.63|3570.48|3660.92|377.34| +2451896|67253|2451941|7166|35982|1629274|613|25905|35982|1629274|613|25905|23|5|7|1|195|506|3|45.82|125.08|31.27|281.43|93.81|137.46|375.24|7.50|0.00|78.78|93.81|101.31|172.59|180.09|-43.65| +2451896|67253|2452015|16016|35982|1629274|613|25905|35982|1629274|613|25905|55|5|18|1|131|506|52|74.88|148.26|108.22|2082.08|5627.44|3893.76|7709.52|506.46|0.00|2543.84|5627.44|6133.90|8171.28|8677.74|1733.68| +2451896|67253|2451952|2810|35982|1629274|613|25905|35982|1629274|613|25905|56|29|1|2|3|506|100|26.94|77.58|62.83|1475.00|6283.00|2694.00|7758.00|125.66|0.00|2482.00|6283.00|6408.66|8765.00|8890.66|3589.00| +2451896|67253|2452009|3845|35982|1629274|613|25905|35982|1629274|613|25905|38|13|11|3|275|506|2|25.80|30.18|26.55|7.26|53.10|51.60|60.36|0.15|37.17|30.18|15.93|16.08|46.11|46.26|-35.67| +2451896|67253|2451959|15329|35982|1629274|613|25905|35982|1629274|613|25905|13|19|16|2|95|506|56|96.99|156.15|6.24|8394.96|349.44|5431.44|8744.40|27.95|0.00|3672.48|349.44|377.39|4021.92|4049.87|-5082.00| +2451896|67253|2451939|656|35982|1629274|613|25905|35982|1629274|613|25905|49|13|16|3|147|506|34|13.70|17.81|15.13|91.12|514.42|465.80|605.54|41.15|0.00|66.30|514.42|555.57|580.72|621.87|48.62| +2451151|34578|2451166|757|86884|1043105|4926|39500|76217|710335|5716|37537|1|7|17|5|283|507|15|84.31|211.61|116.38|1428.45|1745.70|1264.65|3174.15|87.28|0.00|253.80|1745.70|1832.98|1999.50|2086.78|481.05| +2451151|34578|2451249|1216|86884|1043105|4926|39500|76217|710335|5716|37537|26|7|4|4|237|507|6|61.12|163.19|161.55|9.84|969.30|366.72|979.14|19.38|0.00|244.74|969.30|988.68|1214.04|1233.42|602.58| +2451151|34578|2451197|9151|86884|1043105|4926|39500|76217|710335|5716|37537|37|2|15|1|64|507|19|89.95|244.66|48.93|3718.87|929.67|1709.05|4648.54|18.59|0.00|1626.97|929.67|948.26|2556.64|2575.23|-779.38| +2451151|34578|2451220|10204|86884|1043105|4926|39500|76217|710335|5716|37537|46|14|15|4|246|507|3|64.09|80.75|64.60|48.45|193.80|192.27|242.25|1.93|0.00|41.16|193.80|195.73|234.96|236.89|1.53| +2451151|34578|2451224|10094|86884|1043105|4926|39500|76217|710335|5716|37537|7|19|6|4|275|507|64|88.20|187.86|65.75|7815.04|4208.00|5644.80|12023.04|378.72|0.00|841.60|4208.00|4586.72|5049.60|5428.32|-1436.80| +2451151|34578|2451260|3118|86884|1043105|4926|39500|76217|710335|5716|37537|7|16|7|2|47|507|71|81.81|109.62|46.04|4514.18|3268.84|5808.51|7783.02|54.91|523.01|2412.58|2745.83|2800.74|5158.41|5213.32|-3062.68| +2451151|34578|2451216|4064|86884|1043105|4926|39500|76217|710335|5716|37537|40|4|15|5|122|507|65|21.09|55.67|42.86|832.65|2785.90|1370.85|3618.55|2.78|2730.18|1772.55|55.72|58.50|1828.27|1831.05|-1315.13| +2451151|34578|2451269|3022|86884|1043105|4926|39500|76217|710335|5716|37537|14|4|18|3|48|507|64|84.53|106.50|17.04|5725.44|1090.56|5409.92|6816.00|65.43|0.00|2521.60|1090.56|1155.99|3612.16|3677.59|-4319.36| +2451151|34578|2451233|8576|86884|1043105|4926|39500|76217|710335|5716|37537|14|14|8|3|31|507|99|80.46|159.31|151.34|789.03|14982.66|7965.54|15771.69|647.25|7790.98|3627.36|7191.68|7838.93|10819.04|11466.29|-773.86| +2451151|34578|2451189|15644|86884|1043105|4926|39500|76217|710335|5716|37537|7|8|16|2|38|507|5|22.04|50.47|17.66|164.05|88.30|110.20|252.35|7.94|0.00|47.90|88.30|96.24|136.20|144.14|-21.90| +2451151|34578|2451176|3751|86884|1043105|4926|39500|76217|710335|5716|37537|32|1|18|2|239|507|42|96.67|214.60|79.40|5678.40|3334.80|4060.14|9013.20|200.08|0.00|1622.04|3334.80|3534.88|4956.84|5156.92|-725.34| +2451151|34578|2451171|14893|86884|1043105|4926|39500|76217|710335|5716|37537|22|13|9|4|66|507|61|91.20|98.49|78.79|1201.70|4806.19|5563.20|6007.89|336.43|0.00|780.80|4806.19|5142.62|5586.99|5923.42|-757.01| +2451151|34578|2451252|823|86884|1043105|4926|39500|76217|710335|5716|37537|2|20|17|1|15|507|73|98.95|243.41|14.60|16703.13|1065.80|7223.35|17768.93|0.63|1001.85|1243.19|63.95|64.58|1307.14|1307.77|-7159.40| +2451151|34578|2451242|11276|86884|1043105|4926|39500|76217|710335|5716|37537|20|16|7|5|183|507|13|34.79|47.66|39.55|105.43|514.15|452.27|619.58|10.28|0.00|179.66|514.15|524.43|693.81|704.09|61.88| +2451151|34578|2451182|1712|86884|1043105|4926|39500|76217|710335|5716|37537|25|19|4|1|255|507|6|78.69|140.06|105.04|210.12|630.24|472.14|840.36|25.20|0.00|260.46|630.24|655.44|890.70|915.90|158.10| +2451804|44908|2451908|15254|55221|527750|5079|31208|17531|562091|4795|32524|23|11|20|1|175|508|96|64.61|191.24|116.65|7160.64|11198.40|6202.56|18359.04|671.90|0.00|3855.36|11198.40|11870.30|15053.76|15725.66|4995.84| +2451804|44908|2451849|9395|55221|527750|5079|31208|17531|562091|4795|32524|49|26|13|2|11|508|25|16.86|33.21|20.59|315.50|514.75|421.50|830.25|25.73|0.00|340.25|514.75|540.48|855.00|880.73|93.25| +2451804|44908|2451868|869|55221|527750|5079|31208|17531|562091|4795|32524|19|29|10|5|143|508|10|95.91|172.63|24.16|1484.70|241.60|959.10|1726.30|0.00|0.00|845.80|241.60|241.60|1087.40|1087.40|-717.50| +2451804|44908|2451921|9866|55221|527750|5079|31208|17531|562091|4795|32524|31|20|16|1|109|508|45|39.51|112.99|50.84|2796.75|2287.80|1777.95|5084.55|68.63|0.00|1677.60|2287.80|2356.43|3965.40|4034.03|509.85| +2451804|44908|2451918|15680|55221|527750|5079|31208|17531|562091|4795|32524|53|13|7|2|7|508|47|3.11|7.43|3.34|192.23|156.98|146.17|349.21|4.70|0.00|66.27|156.98|161.68|223.25|227.95|10.81| +2451804|44908|2451849|15353|55221|527750|5079|31208|17531|562091|4795|32524|56|20|10|2|144|508|22|61.47|98.35|81.63|367.84|1795.86|1352.34|2163.70|0.00|0.00|887.04|1795.86|1795.86|2682.90|2682.90|443.52| +2451804|44908|2451910|12608|55221|527750|5079|31208|17531|562091|4795|32524|35|17|3|4|298|508|1|89.41|220.84|66.25|154.59|66.25|89.41|220.84|1.98|0.00|35.33|66.25|68.23|101.58|103.56|-23.16| +2451804|44908|2451873|17873|55221|527750|5079|31208|17531|562091|4795|32524|26|26|11|1|208|508|36|8.36|18.22|6.37|426.60|229.32|300.96|655.92|2.29|0.00|84.96|229.32|231.61|314.28|316.57|-71.64| +2451804|44908|2451838|12083|55221|527750|5079|31208|17531|562091|4795|32524|25|25|6|1|276|508|45|21.99|61.79|48.81|584.10|2196.45|989.55|2780.55|65.89|0.00|805.95|2196.45|2262.34|3002.40|3068.29|1206.90| +2451804|44908|2451921|11593|55221|527750|5079|31208|17531|562091|4795|32524|56|13|2|5|285|508|80|93.84|178.29|62.40|9271.20|4992.00|7507.20|14263.20|199.68|0.00|4136.00|4992.00|5191.68|9128.00|9327.68|-2515.20| +2451804|44908|2451875|7309|55221|527750|5079|31208|17531|562091|4795|32524|25|19|7|2|114|508|69|42.69|96.90|49.41|3276.81|3409.29|2945.61|6686.10|0.00|0.00|200.10|3409.29|3409.29|3609.39|3609.39|463.68| +2451804|44908|2451921|13688|55221|527750|5079|31208|17531|562091|4795|32524|23|13|19|4|276|508|21|86.52|204.18|144.96|1243.62|3044.16|1816.92|4287.78|152.20|0.00|385.77|3044.16|3196.36|3429.93|3582.13|1227.24| +2452171|16569|2452183|1515|71106|890953|400|39447|63178|1470251|3788|47235|23|23|12|2|282|509|4|84.65|137.97|40.01|391.84|160.04|338.60|551.88|3.07|57.61|11.00|102.43|105.50|113.43|116.50|-236.17| +2452171|16569|2452251|3947|71106|890953|400|39447|63178|1470251|3788|47235|21|3|13|4|192|509|90|27.92|39.36|13.77|2303.10|1239.30|2512.80|3542.40|24.78|0.00|247.50|1239.30|1264.08|1486.80|1511.58|-1273.50| +2452171|16569|2452221|15253|71106|890953|400|39447|63178|1470251|3788|47235|43|1|3|2|291|509|50|78.37|157.52|92.93|3229.50|4646.50|3918.50|7876.00|232.32|0.00|2914.00|4646.50|4878.82|7560.50|7792.82|728.00| +2452171|16569|2452208|3887|71106|890953|400|39447|63178|1470251|3788|47235|1|21|11|1|11|509|3|48.06|80.26|27.28|158.94|81.84|144.18|240.78|1.63|0.00|50.55|81.84|83.47|132.39|134.02|-62.34| +2452171|16569|2452196|3879|71106|890953|400|39447|63178|1470251|3788|47235|3|3|12|2|272|509|56|35.95|90.95|22.73|3820.32|1272.88|2013.20|5093.20|63.64|0.00|560.00|1272.88|1336.52|1832.88|1896.52|-740.32| +2452171|16569|2452248|15819|71106|890953|400|39447|63178|1470251|3788|47235|9|21|20|4|165|509|16|92.28|256.53|159.04|1559.84|2544.64|1476.48|4104.48|25.44|0.00|410.40|2544.64|2570.08|2955.04|2980.48|1068.16| +2452171|16569|2452191|13709|71106|890953|400|39447|63178|1470251|3788|47235|39|15|16|4|122|509|47|13.70|33.56|6.04|1293.44|283.88|643.90|1577.32|5.67|0.00|678.21|283.88|289.55|962.09|967.76|-360.02| +2452171|16569|2452274|17025|71106|890953|400|39447|63178|1470251|3788|47235|19|25|9|5|91|509|43|63.14|100.39|73.28|1165.73|3151.04|2715.02|4316.77|252.08|0.00|863.01|3151.04|3403.12|4014.05|4266.13|436.02| +2452171|16569|2452278|647|71106|890953|400|39447|63178|1470251|3788|47235|7|29|9|3|87|509|82|64.83|79.09|3.95|6161.48|323.90|5316.06|6485.38|17.45|74.49|1231.64|249.41|266.86|1481.05|1498.50|-5066.65| +2452171|16569|2452263|9409|71106|890953|400|39447|63178|1470251|3788|47235|5|19|4|1|188|509|61|55.75|144.95|108.71|2210.64|6631.31|3400.75|8841.95|132.62|0.00|3182.98|6631.31|6763.93|9814.29|9946.91|3230.56| +2452171|16569|2452277|1105|71106|890953|400|39447|63178|1470251|3788|47235|37|9|11|2|293|509|72|14.98|18.72|9.17|687.60|660.24|1078.56|1347.84|17.23|468.77|552.24|191.47|208.70|743.71|760.94|-887.09| +2452171|16569|2452204|11767|71106|890953|400|39447|63178|1470251|3788|47235|1|19|9|5|74|509|100|67.55|162.79|40.69|12210.00|4069.00|6755.00|16279.00|162.76|0.00|4069.00|4069.00|4231.76|8138.00|8300.76|-2686.00| +2452171|16569|2452230|15493|71106|890953|400|39447|63178|1470251|3788|47235|41|25|12|4|183|509|62|10.11|29.62|20.14|587.76|1248.68|626.82|1836.44|24.97|0.00|367.04|1248.68|1273.65|1615.72|1640.69|621.86| +2452171|16569|2452183|15315|71106|890953|400|39447|63178|1470251|3788|47235|45|7|5|1|127|509|6|51.34|58.52|39.20|115.92|235.20|308.04|351.12|14.11|0.00|35.10|235.20|249.31|270.30|284.41|-72.84| +2451997|74746|2452049|17049|95808|818065|3522|16579|96953|1210636|2503|43960|43|25|4|5|196|510|8|22.20|28.41|26.42|15.92|211.36|177.60|227.28|16.90|0.00|81.76|211.36|228.26|293.12|310.02|33.76| +2451997|74746|2452010|17331|95808|818065|3522|16579|96953|1210636|2503|43960|1|17|4|2|77|510|42|16.28|18.72|18.15|23.94|762.30|683.76|786.24|9.91|564.10|290.64|198.20|208.11|488.84|498.75|-485.56| +2451997|74746|2452094|1371|95808|818065|3522|16579|96953|1210636|2503|43960|55|19|9|1|113|510|15|77.98|192.61|26.96|2484.75|404.40|1169.70|2889.15|12.13|0.00|953.40|404.40|416.53|1357.80|1369.93|-765.30| +2451997|74746|2452066|11349|95808|818065|3522|16579|96953|1210636|2503|43960|29|21|7|3|209|510|3|25.85|49.37|8.88|121.47|26.64|77.55|148.11|0.79|0.00|22.20|26.64|27.43|48.84|49.63|-50.91| +2451997|74746|2452020|8863|95808|818065|3522|16579|96953|1210636|2503|43960|37|19|18|1|184|510|25|72.52|109.50|50.37|1478.25|1259.25|1813.00|2737.50|62.96|0.00|602.25|1259.25|1322.21|1861.50|1924.46|-553.75| +2451997|74746|2452065|6837|95808|818065|3522|16579|96953|1210636|2503|43960|33|15|12|3|244|510|67|36.63|88.64|21.27|4513.79|1425.09|2454.21|5938.88|14.25|0.00|2018.71|1425.09|1439.34|3443.80|3458.05|-1029.12| +2451997|74746|2452046|13617|95808|818065|3522|16579|96953|1210636|2503|43960|11|9|7|4|131|510|1|66.71|188.78|124.59|64.19|124.59|66.71|188.78|11.21|0.00|33.98|124.59|135.80|158.57|169.78|57.88| +2451997|74746|2452031|6261|95808|818065|3522|16579|96953|1210636|2503|43960|59|1|12|1|87|510|14|1.41|2.86|0.11|38.50|1.54|19.74|40.04|0.10|0.00|11.48|1.54|1.64|13.02|13.12|-18.20| +2451997|74746|2452026|14523|95808|818065|3522|16579|96953|1210636|2503|43960|15|15|8|1|31|510|8|21.26|40.60|23.14|139.68|185.12|170.08|324.80|12.95|0.00|9.68|185.12|198.07|194.80|207.75|15.04| +2451997|74746|2452024|9853|95808|818065|3522|16579|96953|1210636|2503|43960|53|11|15|2|116|510|33|58.87|120.09|78.05|1387.32|2575.65|1942.71|3962.97|51.51|0.00|1386.99|2575.65|2627.16|3962.64|4014.15|632.94| +2451997|74746|2452095|10857|95808|818065|3522|16579|96953|1210636|2503|43960|35|7|10|1|112|510|68|68.11|87.86|55.35|2210.68|3763.80|4631.48|5974.48|150.55|0.00|1075.08|3763.80|3914.35|4838.88|4989.43|-867.68| +2451905|78389|2452025|1718|88189|128849|7109|42122|77226|1917661|6102|25562|5|19|17|2|174|511|5|40.64|102.81|56.54|231.35|282.70|203.20|514.05|2.82|0.00|41.10|282.70|285.52|323.80|326.62|79.50| +2451905|78389|2451985|7903|88189|128849|7109|42122|77226|1917661|6102|25562|31|25|7|1|175|511|90|90.75|180.59|36.11|13003.20|3249.90|8167.50|16253.10|0.00|0.00|3900.60|3249.90|3249.90|7150.50|7150.50|-4917.60| +2451905|78389|2451960|10880|88189|128849|7109|42122|77226|1917661|6102|25562|13|14|19|3|34|511|78|80.88|187.64|165.12|1756.56|12879.36|6308.64|14635.92|365.77|3735.01|6585.54|9144.35|9510.12|15729.89|16095.66|2835.71| +2451905|78389|2451991|6481|88189|128849|7109|42122|77226|1917661|6102|25562|38|17|15|4|223|511|20|5.88|16.46|11.02|108.80|220.40|117.60|329.20|19.83|0.00|75.60|220.40|240.23|296.00|315.83|102.80| +2451905|78389|2451929|1463|88189|128849|7109|42122|77226|1917661|6102|25562|13|20|2|3|164|511|65|98.64|99.62|39.84|3885.70|2589.60|6411.60|6475.30|51.79|0.00|906.10|2589.60|2641.39|3495.70|3547.49|-3822.00| +2451905|78389|2451970|2105|88189|128849|7109|42122|77226|1917661|6102|25562|41|5|8|5|62|511|95|85.72|183.44|95.38|8365.70|9061.10|8143.40|17426.80|144.97|5436.66|2264.80|3624.44|3769.41|5889.24|6034.21|-4518.96| +2451905|78389|2451988|14264|88189|128849|7109|42122|77226|1917661|6102|25562|35|2|13|2|118|511|28|29.05|86.27|56.07|845.60|1569.96|813.40|2415.56|141.29|0.00|772.80|1569.96|1711.25|2342.76|2484.05|756.56| +2451905|78389|2451997|16724|88189|128849|7109|42122|77226|1917661|6102|25562|53|11|18|2|199|511|24|38.61|82.62|78.48|99.36|1883.52|926.64|1982.88|169.51|0.00|753.36|1883.52|2053.03|2636.88|2806.39|956.88| +2451905|78389|2451931|8141|88189|128849|7109|42122|77226|1917661|6102|25562|29|19|18|3|78|511|81|36.18|87.55|5.25|6666.30|425.25|2930.58|7091.55|0.00|0.00|2198.34|425.25|425.25|2623.59|2623.59|-2505.33| +2451905|78389|2451947|10295|88189|128849|7109|42122|77226|1917661|6102|25562|1|20|1|1|124|511|42|64.47|79.29|17.44|2597.70|732.48|2707.74|3330.18|43.94|0.00|1165.50|732.48|776.42|1897.98|1941.92|-1975.26| +2451905|78389|2451979|7333|88189|128849|7109|42122|77226|1917661|6102|25562|2|1|10|1|125|511|35|91.63|239.15|86.09|5357.10|3013.15|3207.05|8370.25|30.13|0.00|3096.80|3013.15|3043.28|6109.95|6140.08|-193.90| +2451905|78389|2451965|12331|88189|128849|7109|42122|77226|1917661|6102|25562|11|2|18|4|73|511|23|53.95|60.42|21.75|889.41|500.25|1240.85|1389.66|5.00|0.00|319.47|500.25|505.25|819.72|824.72|-740.60| +2451905|78389|2452015|11156|88189|128849|7109|42122|77226|1917661|6102|25562|56|20|13|5|130|511|85|78.41|125.45|0.00|10663.25|0.00|6664.85|10663.25|0.00|0.00|3518.15|0.00|0.00|3518.15|3518.15|-6664.85| +2451905|78389|2451994|187|88189|128849|7109|42122|77226|1917661|6102|25562|20|2|10|1|19|511|53|54.30|104.25|11.46|4917.87|607.38|2877.90|5525.25|18.22|0.00|2486.23|607.38|625.60|3093.61|3111.83|-2270.52| +2451147|55839|2451160|12757|633|1607914|6395|11676|60018|394899|5119|26400|44|4|13|3|183|512|99|88.65|235.80|4.71|22877.91|466.29|8776.35|23344.20|8.95|354.38|9570.33|111.91|120.86|9682.24|9691.19|-8664.44| +2451147|55839|2451212|7081|633|1607914|6395|11676|60018|394899|5119|26400|43|2|12|2|122|512|10|51.83|87.59|15.76|718.30|157.60|518.30|875.90|14.18|0.00|35.00|157.60|171.78|192.60|206.78|-360.70| +2451147|55839|2451199|6626|633|1607914|6395|11676|60018|394899|5119|26400|31|20|6|1|56|512|29|12.14|23.30|0.46|662.36|13.34|352.06|675.70|0.53|0.00|337.85|13.34|13.87|351.19|351.72|-338.72| +2451147|55839|2451193|4514|633|1607914|6395|11676|60018|394899|5119|26400|38|13|4|1|185|512|23|3.44|5.88|5.70|4.14|131.10|79.12|135.24|0.00|0.00|18.86|131.10|131.10|149.96|149.96|51.98| +2451147|55839|2451231|7388|633|1607914|6395|11676|60018|394899|5119|26400|16|2|18|1|57|512|21|2.57|6.32|3.66|55.86|76.86|53.97|132.72|4.77|23.82|30.45|53.04|57.81|83.49|88.26|-0.93| +2451147|55839|2451185|14288|633|1607914|6395|11676|60018|394899|5119|26400|43|25|11|2|173|512|81|60.32|177.34|69.16|8762.58|5601.96|4885.92|14364.54|0.00|728.25|4165.02|4873.71|4873.71|9038.73|9038.73|-12.21| +2451147|55839|2451238|8060|633|1607914|6395|11676|60018|394899|5119|26400|37|22|2|5|298|512|47|100.00|235.00|82.25|7179.25|3865.75|4700.00|11045.00|231.94|0.00|3313.50|3865.75|4097.69|7179.25|7411.19|-834.25| +2451147|55839|2451241|7294|633|1607914|6395|11676|60018|394899|5119|26400|56|13|12|5|105|512|76|84.67|212.52|170.01|3230.76|12920.76|6434.92|16151.52|646.03|0.00|5329.88|12920.76|13566.79|18250.64|18896.67|6485.84| +2451147|55839|2451210|16352|633|1607914|6395|11676|60018|394899|5119|26400|2|10|7|3|249|512|23|82.46|185.53|165.12|469.43|3797.76|1896.58|4267.19|265.84|0.00|0.00|3797.76|4063.60|3797.76|4063.60|1901.18| +2451147|55839|2451163|16202|633|1607914|6395|11676|60018|394899|5119|26400|50|28|11|5|169|512|36|90.65|266.51|135.92|4701.24|4893.12|3263.40|9594.36|195.72|0.00|1630.80|4893.12|5088.84|6523.92|6719.64|1629.72| +2451147|55839|2451167|14950|633|1607914|6395|11676|60018|394899|5119|26400|26|7|17|5|163|512|35|21.65|53.47|39.03|505.40|1366.05|757.75|1871.45|13.66|0.00|842.10|1366.05|1379.71|2208.15|2221.81|608.30| +2451147|55839|2451224|15538|633|1607914|6395|11676|60018|394899|5119|26400|50|14|13|5|217|512|64|54.42|90.33|9.03|5203.20|577.92|3482.88|5781.12|10.92|456.55|2080.64|121.37|132.29|2202.01|2212.93|-3361.51| +2451147|55839|2451155|3907|633|1607914|6395|11676|60018|394899|5119|26400|26|10|15|4|273|512|42|73.76|88.51|61.95|1115.52|2601.90|3097.92|3717.42|0.00|390.28|1189.44|2211.62|2211.62|3401.06|3401.06|-886.30| +2450897|71825|2450945|15674|85543|529335|3080|30872|3602|1873433|2154|1899|50|10|3|3|140|513|30|14.00|36.96|34.37|77.70|1031.10|420.00|1108.80|61.86|0.00|110.70|1031.10|1092.96|1141.80|1203.66|611.10| +2450897|71825|2451009|11047|85543|529335|3080|30872|3602|1873433|2154|1899|50|13|18|4|70|513|46|33.77|38.49|5.00|1540.54|230.00|1553.42|1770.54|5.15|165.60|761.30|64.40|69.55|825.70|830.85|-1489.02| +2450897|71825|2450957|5102|85543|529335|3080|30872|3602|1873433|2154|1899|52|16|14|4|114|513|80|57.31|127.80|69.01|4703.20|5520.80|4584.80|10224.00|220.83|0.00|3782.40|5520.80|5741.63|9303.20|9524.03|936.00| +2450897|71825|2450974|3116|85543|529335|3080|30872|3602|1873433|2154|1899|20|4|15|2|159|513|92|81.41|190.49|150.48|3680.92|13844.16|7489.72|17525.08|0.00|0.00|700.12|13844.16|13844.16|14544.28|14544.28|6354.44| +2450897|71825|2451011|1555|85543|529335|3080|30872|3602|1873433|2154|1899|7|20|18|5|272|513|87|85.75|169.78|13.58|13589.40|1181.46|7460.25|14770.86|35.44|0.00|6793.83|1181.46|1216.90|7975.29|8010.73|-6278.79| +2450897|71825|2450962|794|85543|529335|3080|30872|3602|1873433|2154|1899|4|10|9|4|82|513|35|10.57|16.27|4.71|404.60|164.85|369.95|569.45|0.00|0.00|113.75|164.85|164.85|278.60|278.60|-205.10| +2450897|71825|2450983|13198|85543|529335|3080|30872|3602|1873433|2154|1899|14|14|15|5|82|513|37|56.29|158.17|98.06|2224.07|3628.22|2082.73|5852.29|326.53|0.00|1228.77|3628.22|3954.75|4856.99|5183.52|1545.49| +2450897|71825|2450970|17372|85543|529335|3080|30872|3602|1873433|2154|1899|19|28|20|5|150|513|52|41.73|66.76|44.06|1180.40|2291.12|2169.96|3471.52|21.07|183.28|1388.40|2107.84|2128.91|3496.24|3517.31|-62.12| +2452218|15463|2452244|3085|477|867506|4135|9564|78555|951098|3451|21680|5|17|5|3|244|514|61|35.62|74.08|34.81|2395.47|2123.41|2172.82|4518.88|169.87|0.00|1807.43|2123.41|2293.28|3930.84|4100.71|-49.41| +2452218|15463|2452309|5837|477|867506|4135|9564|78555|951098|3451|21680|53|7|1|1|48|514|26|8.12|10.71|9.31|36.40|242.06|211.12|278.46|19.36|0.00|47.32|242.06|261.42|289.38|308.74|30.94| +2452218|15463|2452317|1007|477|867506|4135|9564|78555|951098|3451|21680|17|23|15|3|141|514|89|62.50|137.50|60.50|6853.00|5384.50|5562.50|12237.50|323.07|0.00|366.68|5384.50|5707.57|5751.18|6074.25|-178.00| +2452218|15463|2452264|4875|477|867506|4135|9564|78555|951098|3451|21680|13|7|11|5|62|514|35|6.49|10.57|7.08|122.15|247.80|227.15|369.95|2.47|0.00|147.70|247.80|250.27|395.50|397.97|20.65| +2452218|15463|2452295|14235|477|867506|4135|9564|78555|951098|3451|21680|51|27|12|3|121|514|44|41.68|75.44|6.03|3054.04|265.32|1833.92|3319.36|7.69|111.43|99.44|153.89|161.58|253.33|261.02|-1680.03| +2452218|15463|2452282|14047|477|867506|4135|9564|78555|951098|3451|21680|45|11|12|5|278|514|27|64.72|142.38|119.59|615.33|3228.93|1747.44|3844.26|2.90|3132.06|1345.41|96.87|99.77|1442.28|1445.18|-1650.57| +2452218|15463|2452325|9153|477|867506|4135|9564|78555|951098|3451|21680|43|15|17|3|252|514|25|16.51|22.28|16.93|133.75|423.25|412.75|557.00|0.00|0.00|206.00|423.25|423.25|629.25|629.25|10.50| +2452218|15463|2452256|6019|477|867506|4135|9564|78555|951098|3451|21680|19|5|3|1|236|514|27|4.89|13.25|9.93|89.64|268.11|132.03|357.75|21.44|0.00|14.31|268.11|289.55|282.42|303.86|136.08| +2452218|15463|2452321|16241|477|867506|4135|9564|78555|951098|3451|21680|29|23|12|4|19|514|14|5.62|12.53|6.89|78.96|96.46|78.68|175.42|7.71|0.00|68.32|96.46|104.17|164.78|172.49|17.78| +2452218|15463|2452306|10381|477|867506|4135|9564|78555|951098|3451|21680|45|7|15|3|44|514|29|57.24|63.53|58.44|147.61|1694.76|1659.96|1842.37|12.54|440.63|884.21|1254.13|1266.67|2138.34|2150.88|-405.83| +2452231|72963|2452239|15883|22555|415670|3538|29806|32611|493556|4257|44925|25|9|10|2|147|515|12|26.70|35.51|28.76|81.00|345.12|320.40|426.12|6.90|0.00|166.08|345.12|352.02|511.20|518.10|24.72| +2452231|72963|2452297|11575|22555|415670|3538|29806|32611|493556|4257|44925|51|19|1|5|149|515|95|45.34|102.92|39.10|6062.90|3714.50|4307.30|9777.40|37.14|0.00|1075.40|3714.50|3751.64|4789.90|4827.04|-592.80| +2452231|72963|2452331|8371|22555|415670|3538|29806|32611|493556|4257|44925|11|25|10|2|267|515|43|66.76|131.51|126.24|226.61|5428.32|2870.68|5654.93|0.00|488.54|1526.50|4939.78|4939.78|6466.28|6466.28|2069.10| +2452231|72963|2452275|2795|22555|415670|3538|29806|32611|493556|4257|44925|21|7|7|5|283|515|7|37.80|55.18|24.27|216.37|169.89|264.60|386.26|1.78|134.21|154.49|35.68|37.46|190.17|191.95|-228.92| +2452231|72963|2452329|14721|22555|415670|3538|29806|32611|493556|4257|44925|53|25|2|3|204|515|99|85.81|209.37|186.33|2280.96|18446.67|8495.19|20727.63|184.46|0.00|413.82|18446.67|18631.13|18860.49|19044.95|9951.48| +2452231|72963|2452232|10875|22555|415670|3538|29806|32611|493556|4257|44925|55|23|16|5|24|515|67|26.67|78.94|32.36|3120.86|2168.12|1786.89|5288.98|173.44|0.00|2591.56|2168.12|2341.56|4759.68|4933.12|381.23| +2452231|72963|2452244|16577|22555|415670|3538|29806|32611|493556|4257|44925|39|27|8|2|56|515|96|16.77|41.58|8.31|3193.92|797.76|1609.92|3991.68|71.79|0.00|438.72|797.76|869.55|1236.48|1308.27|-812.16| +2452231|72963|2452300|9975|22555|415670|3538|29806|32611|493556|4257|44925|51|17|12|5|258|515|63|3.30|5.08|3.20|118.44|201.60|207.90|320.04|0.00|0.00|41.58|201.60|201.60|243.18|243.18|-6.30| +2452231|72963|2452272|10677|22555|415670|3538|29806|32611|493556|4257|44925|25|5|10|5|257|515|42|58.82|139.99|12.59|5350.80|528.78|2470.44|5879.58|47.59|0.00|2586.78|528.78|576.37|3115.56|3163.15|-1941.66| +2452231|72963|2452351|4205|22555|415670|3538|29806|32611|493556|4257|44925|45|1|6|1|136|515|46|25.42|32.53|22.77|448.96|1047.42|1169.32|1496.38|83.79|0.00|688.16|1047.42|1131.21|1735.58|1819.37|-121.90| +2451899|75233|2451907|13105|26446|1519018|1190|10962|99682|1063221|5525|7297|1|25|18|5|192|516|17|93.04|209.34|96.29|1921.85|1636.93|1581.68|3558.78|108.69|278.27|854.08|1358.66|1467.35|2212.74|2321.43|-223.02| +2451899|75233|2451931|5849|26446|1519018|1190|10962|99682|1063221|5525|7297|19|8|7|2|244|516|93|74.95|219.60|101.01|11028.87|9393.93|6970.35|20422.80|187.87|0.00|9190.26|9393.93|9581.80|18584.19|18772.06|2423.58| +2451899|75233|2451984|10565|26446|1519018|1190|10962|99682|1063221|5525|7297|44|26|12|3|144|516|52|61.37|105.55|65.44|2085.72|3402.88|3191.24|5488.60|272.23|0.00|1591.20|3402.88|3675.11|4994.08|5266.31|211.64| +2451899|75233|2451957|11695|26446|1519018|1190|10962|99682|1063221|5525|7297|20|25|15|2|162|516|4|83.84|215.46|135.73|318.92|542.92|335.36|861.84|23.56|206.30|25.84|336.62|360.18|362.46|386.02|1.26| +2451899|75233|2451950|6089|26446|1519018|1190|10962|99682|1063221|5525|7297|56|5|7|3|22|516|78|91.04|108.33|79.08|2281.50|6168.24|7101.12|8449.74|0.00|0.00|1942.98|6168.24|6168.24|8111.22|8111.22|-932.88| +2451899|75233|2451996|5330|26446|1519018|1190|10962|99682|1063221|5525|7297|8|29|8|3|271|516|1|2.76|7.14|3.57|3.57|3.57|2.76|7.14|0.28|0.00|3.14|3.57|3.85|6.71|6.99|0.81| +2451899|75233|2451981|15896|26446|1519018|1190|10962|99682|1063221|5525|7297|49|26|3|2|161|516|70|49.46|102.38|57.33|3153.50|4013.10|3462.20|7166.60|200.65|0.00|2078.30|4013.10|4213.75|6091.40|6292.05|550.90| +2451899|75233|2451990|10004|26446|1519018|1190|10962|99682|1063221|5525|7297|47|8|13|1|179|516|82|30.13|43.68|10.48|2722.40|859.36|2470.66|3581.76|0.00|249.21|250.10|610.15|610.15|860.25|860.25|-1860.51| +2451899|75233|2451988|12482|26446|1519018|1190|10962|99682|1063221|5525|7297|13|14|9|2|258|516|67|13.54|21.66|21.01|43.55|1407.67|907.18|1451.22|112.61|0.00|333.66|1407.67|1520.28|1741.33|1853.94|500.49| +2451899|75233|2451999|6164|26446|1519018|1190|10962|99682|1063221|5525|7297|5|29|17|4|51|516|6|13.14|17.08|8.36|52.32|50.16|78.84|102.48|2.00|0.00|40.98|50.16|52.16|91.14|93.14|-28.68| +2451577|67567|2451664|4820|6803|219512|6621|21730|32616|641443|7057|24966|55|5|11|4|64|517|63|86.88|157.25|75.48|5151.51|4755.24|5473.44|9906.75|20.92|2662.93|4358.97|2092.31|2113.23|6451.28|6472.20|-3381.13| +2451577|67567|2451669|14993|6803|219512|6621|21730|32616|641443|7057|24966|59|29|7|2|247|517|72|87.89|116.89|15.19|7322.40|1093.68|6328.08|8416.08|87.49|0.00|757.44|1093.68|1181.17|1851.12|1938.61|-5234.40| +2451577|67567|2451642|85|6803|219512|6621|21730|32616|641443|7057|24966|43|26|16|3|170|517|17|93.64|198.51|65.50|2261.17|1113.50|1591.88|3374.67|77.94|0.00|674.90|1113.50|1191.44|1788.40|1866.34|-478.38| +2451577|67567|2451584|8606|6803|219512|6621|21730|32616|641443|7057|24966|13|25|19|5|61|517|73|86.69|86.69|54.61|2341.84|3986.53|6328.37|6328.37|358.78|0.00|1454.89|3986.53|4345.31|5441.42|5800.20|-2341.84| +2451577|67567|2451662|11915|6803|219512|6621|21730|32616|641443|7057|24966|29|13|6|5|184|517|24|1.75|4.74|1.37|80.88|32.88|42.00|113.76|2.78|1.97|55.68|30.91|33.69|86.59|89.37|-11.09| +2451577|67567|2451693|313|6803|219512|6621|21730|32616|641443|7057|24966|38|1|5|4|133|517|68|90.98|200.15|88.06|7622.12|5988.08|6186.64|13610.20|419.16|0.00|1496.68|5988.08|6407.24|7484.76|7903.92|-198.56| +2451577|67567|2451604|3128|6803|219512|6621|21730|32616|641443|7057|24966|8|11|14|5|16|517|94|83.92|179.58|129.29|4727.26|12153.26|7888.48|16880.52|182.29|8507.28|3038.08|3645.98|3828.27|6684.06|6866.35|-4242.50| +2451577|67567|2451672|8267|6803|219512|6621|21730|32616|641443|7057|24966|35|8|12|4|296|517|27|73.06|214.79|90.21|3363.66|2435.67|1972.62|5799.33|170.49|0.00|2435.67|2435.67|2606.16|4871.34|5041.83|463.05| +2451577|67567|2451626|17534|6803|219512|6621|21730|32616|641443|7057|24966|29|25|4|2|120|517|36|54.36|70.66|34.62|1297.44|1246.32|1956.96|2543.76|24.92|0.00|635.76|1246.32|1271.24|1882.08|1907.00|-710.64| +2451577|67567|2451617|6997|6803|219512|6621|21730|32616|641443|7057|24966|31|23|18|4|240|517|68|20.70|26.91|0.26|1812.20|17.68|1407.60|1829.88|0.00|0.00|694.96|17.68|17.68|712.64|712.64|-1389.92| +2451577|67567|2451694|17312|6803|219512|6621|21730|32616|641443|7057|24966|32|1|5|2|6|517|63|97.11|180.62|84.89|6030.99|5348.07|6117.93|11379.06|374.36|0.00|2958.48|5348.07|5722.43|8306.55|8680.91|-769.86| +2451577|67567|2451655|10352|6803|219512|6621|21730|32616|641443|7057|24966|59|17|15|4|170|517|57|86.52|114.20|20.55|5338.05|1171.35|4931.64|6509.40|81.99|0.00|1301.88|1171.35|1253.34|2473.23|2555.22|-3760.29| +2451577|67567|2451681|13979|6803|219512|6621|21730|32616|641443|7057|24966|49|11|17|3|256|517|34|17.58|46.05|30.85|516.80|1048.90|597.72|1565.70|20.97|0.00|704.48|1048.90|1069.87|1753.38|1774.35|451.18| +2451577|67567|2451614|12479|6803|219512|6621|21730|32616|641443|7057|24966|19|7|15|1|187|517|62|43.88|79.86|17.56|3862.60|1088.72|2720.56|4951.32|97.98|0.00|791.74|1088.72|1186.70|1880.46|1978.44|-1631.84| +2451577|67567|2451686|16103|6803|219512|6621|21730|32616|641443|7057|24966|56|23|5|3|171|517|1|41.53|51.08|4.08|47.00|4.08|41.53|51.08|0.24|0.00|22.98|4.08|4.32|27.06|27.30|-37.45| +2452208|84445|2452319|6245|68615|1893082|396|36863|23163|381042|2358|33625|49|17|13|4|28|518|40|71.41|81.40|8.14|2930.40|325.60|2856.40|3256.00|16.28|0.00|1562.80|325.60|341.88|1888.40|1904.68|-2530.80| +2452208|84445|2452231|13685|68615|1893082|396|36863|23163|381042|2358|33625|57|25|17|1|18|518|56|80.39|196.95|100.44|5404.56|5624.64|4501.84|11029.20|449.97|0.00|1654.24|5624.64|6074.61|7278.88|7728.85|1122.80| +2452208|84445|2452250|6951|68615|1893082|396|36863|23163|381042|2358|33625|11|3|11|5|6|518|33|7.76|16.68|9.67|231.33|319.11|256.08|550.44|19.14|0.00|49.50|319.11|338.25|368.61|387.75|63.03| +2452208|84445|2452229|1411|68615|1893082|396|36863|23163|381042|2358|33625|57|23|12|5|275|518|45|69.71|132.44|19.86|5066.10|893.70|3136.95|5959.80|4.91|795.39|59.40|98.31|103.22|157.71|162.62|-3038.64| +2452208|84445|2452320|10819|68615|1893082|396|36863|23163|381042|2358|33625|9|13|9|4|156|518|78|83.48|126.05|39.07|6784.44|3047.46|6511.44|9831.90|182.84|0.00|2949.18|3047.46|3230.30|5996.64|6179.48|-3463.98| +2452208|84445|2452313|11755|68615|1893082|396|36863|23163|381042|2358|33625|45|13|4|3|125|518|80|57.20|105.82|60.31|3640.80|4824.80|4576.00|8465.60|48.24|0.00|1354.40|4824.80|4873.04|6179.20|6227.44|248.80| +2452208|84445|2452238|4777|68615|1893082|396|36863|23163|381042|2358|33625|33|13|20|3|106|518|42|48.96|68.05|48.99|800.52|2057.58|2056.32|2858.10|123.45|0.00|1371.72|2057.58|2181.03|3429.30|3552.75|1.26| +2452208|84445|2452220|10923|68615|1893082|396|36863|23163|381042|2358|33625|53|13|14|4|164|518|64|35.46|56.73|2.26|3486.08|144.64|2269.44|3630.72|4.33|0.00|1778.56|144.64|148.97|1923.20|1927.53|-2124.80| +2452208|84445|2452327|11621|68615|1893082|396|36863|23163|381042|2358|33625|7|19|13|4|102|518|38|56.54|101.20|27.32|2807.44|1038.16|2148.52|3845.60|93.43|0.00|1576.62|1038.16|1131.59|2614.78|2708.21|-1110.36| +2452208|84445|2452324|757|68615|1893082|396|36863|23163|381042|2358|33625|31|17|12|5|18|518|7|42.54|113.58|21.58|644.00|151.06|297.78|795.06|0.00|0.00|278.25|151.06|151.06|429.31|429.31|-146.72| +2452208|84445|2452296|1217|68615|1893082|396|36863|23163|381042|2358|33625|49|17|3|2|51|518|16|8.19|21.53|6.88|234.40|110.08|131.04|344.48|7.70|0.00|172.16|110.08|117.78|282.24|289.94|-20.96| +2452549|51024|2452607|14085|27858|1573246|6187|15975|96307|1880019|4892|11033|55|13|6|5|214|519|56|24.22|65.63|19.68|2573.20|1102.08|1356.32|3675.28|77.14|0.00|1433.04|1102.08|1179.22|2535.12|2612.26|-254.24| +2452549|51024|2452553|16837|27858|1573246|6187|15975|96307|1880019|4892|11033|60|19|10|5|190|519|78|9.66|11.10|0.33|840.06|25.74|753.48|865.80|0.25|0.00|241.80|25.74|25.99|267.54|267.79|-727.74| +2452549|51024|2452623|10831|27858|1573246|6187|15975|96307|1880019|4892|11033|60|12|12|3|77|519|72|11.58|13.08|3.00|725.76|216.00|833.76|941.76|10.80|0.00|272.88|216.00|226.80|488.88|499.68|-617.76| +2452549|51024|2452638|9606|27858|1573246|6187|15975|96307|1880019|4892|11033|24|21|10|4|278|519|26|87.70|197.32|61.16|3540.16|1590.16|2280.20|5130.32|111.31|0.00|2000.70|1590.16|1701.47|3590.86|3702.17|-690.04| +2452549|51024|2452669|2275|27858|1573246|6187|15975|96307|1880019|4892|11033|18|3|12|2|235|519|28|87.06|140.16|130.34|274.96|3649.52|2437.68|3924.48|182.47|0.00|313.88|3649.52|3831.99|3963.40|4145.87|1211.84| +2452549|51024|2452614|16255|27858|1573246|6187|15975|96307|1880019|4892|11033|39|30|7|2|194|519|91|77.32|201.03|30.15|15550.08|2743.65|7036.12|18293.73|27.43|0.00|6768.58|2743.65|2771.08|9512.23|9539.66|-4292.47| +2452549|51024|2452609|9451|27858|1573246|6187|15975|96307|1880019|4892|11033|33|24|7|4|4|519|89|26.70|29.10|5.23|2124.43|465.47|2376.30|2589.90|4.65|0.00|647.03|465.47|470.12|1112.50|1117.15|-1910.83| +2452549|51024|2452555|963|27858|1573246|6187|15975|96307|1880019|4892|11033|60|25|3|1|119|519|70|28.36|41.97|10.49|2203.60|734.30|1985.20|2937.90|51.40|0.00|410.90|734.30|785.70|1145.20|1196.60|-1250.90| +2452150|35719|2452174|16333|71359|1331232|2305|33378|4944|669066|2150|32423|37|29|18|4|40|520|93|55.93|118.57|10.67|10034.70|992.31|5201.49|11027.01|79.38|0.00|2866.26|992.31|1071.69|3858.57|3937.95|-4209.18| +2452150|35719|2452196|8613|71359|1331232|2305|33378|4944|669066|2150|32423|29|15|1|2|250|520|47|11.06|19.35|10.25|427.70|481.75|519.82|909.45|28.90|0.00|99.64|481.75|510.65|581.39|610.29|-38.07| +2452150|35719|2452192|11003|71359|1331232|2305|33378|4944|669066|2150|32423|17|21|13|5|287|520|81|7.64|22.61|0.22|1813.59|17.82|618.84|1831.41|0.35|0.00|109.35|17.82|18.17|127.17|127.52|-601.02| +2452150|35719|2452203|5179|71359|1331232|2305|33378|4944|669066|2150|32423|53|9|3|3|216|520|37|65.01|154.07|24.65|4788.54|912.05|2405.37|5700.59|27.36|0.00|2222.96|912.05|939.41|3135.01|3162.37|-1493.32| +2452150|35719|2452194|10913|71359|1331232|2305|33378|4944|669066|2150|32423|11|27|12|3|61|520|46|87.47|132.95|26.59|4892.56|1223.14|4023.62|6115.70|31.92|868.42|2812.90|354.72|386.64|3167.62|3199.54|-3668.90| +2452150|35719|2452245|755|71359|1331232|2305|33378|4944|669066|2150|32423|53|7|8|5|99|520|57|92.12|229.37|22.93|11767.08|1307.01|5250.84|13074.09|26.14|0.00|4314.33|1307.01|1333.15|5621.34|5647.48|-3943.83| +2452150|35719|2452264|11007|71359|1331232|2305|33378|4944|669066|2150|32423|3|13|5|3|68|520|17|26.03|70.80|51.68|325.04|878.56|442.51|1203.60|0.00|0.00|457.30|878.56|878.56|1335.86|1335.86|436.05| +2452150|35719|2452238|8043|71359|1331232|2305|33378|4944|669066|2150|32423|49|11|7|5|106|520|93|30.14|38.27|0.00|3559.11|0.00|2803.02|3559.11|0.00|0.00|533.82|0.00|0.00|533.82|533.82|-2803.02| +2452150|35719|2452155|2025|71359|1331232|2305|33378|4944|669066|2150|32423|33|5|2|3|97|520|76|80.78|210.83|210.83|0.00|16023.08|6139.28|16023.08|0.00|0.00|0.00|16023.08|16023.08|16023.08|16023.08|9883.80| +2452150|35719|2452179|10135|71359|1331232|2305|33378|4944|669066|2150|32423|15|13|13|4|85|520|44|69.83|134.07|100.55|1474.88|4424.20|3072.52|5899.08|0.00|0.00|176.88|4424.20|4424.20|4601.08|4601.08|1351.68| +2452494|58911|2452533|4723|86458|744534|1010|286|7043|1552713|3267|1743|27|27|1|1|261|521|64|22.04|44.08|14.54|1890.56|930.56|1410.56|2821.12|65.13|0.00|281.60|930.56|995.69|1212.16|1277.29|-480.00| +2452494|58911|2452511|16599|86458|744534|1010|286|7043|1552713|3267|1743|39|30|10|5|30|521|10|20.34|55.32|23.78|315.40|237.80|203.40|553.20|14.26|0.00|38.70|237.80|252.06|276.50|290.76|34.40| +2452494|58911|2452599|8127|86458|744534|1010|286|7043|1552713|3267|1743|33|25|6|2|75|521|19|4.15|9.33|0.18|173.85|3.42|78.85|177.27|0.13|0.00|58.33|3.42|3.55|61.75|61.88|-75.43| +2452494|58911|2452536|9210|86458|744534|1010|286|7043|1552713|3267|1743|55|19|12|4|189|521|68|74.85|206.58|115.68|6181.20|7866.24|5089.80|14047.44|707.96|0.00|6321.28|7866.24|8574.20|14187.52|14895.48|2776.44| +2452494|58911|2452553|16893|86458|744534|1010|286|7043|1552713|3267|1743|3|24|12|2|132|521|29|57.47|138.50|83.10|1606.60|2409.90|1666.63|4016.50|168.69|0.00|0.00|2409.90|2578.59|2409.90|2578.59|743.27| +2452494|58911|2452523|11700|86458|744534|1010|286|7043|1552713|3267|1743|15|9|5|2|79|521|99|16.28|25.39|9.14|1608.75|904.86|1611.72|2513.61|27.14|0.00|1130.58|904.86|932.00|2035.44|2062.58|-706.86| +2452494|58911|2452514|12204|86458|744534|1010|286|7043|1552713|3267|1743|15|21|7|5|6|521|50|15.31|17.60|14.43|158.50|721.50|765.50|880.00|14.43|0.00|299.00|721.50|735.93|1020.50|1034.93|-44.00| +2452494|58911|2452521|12795|86458|744534|1010|286|7043|1552713|3267|1743|39|3|4|1|61|521|27|7.22|18.48|3.14|414.18|84.78|194.94|498.96|4.23|0.00|0.00|84.78|89.01|84.78|89.01|-110.16| +2452494|58911|2452591|14071|86458|744534|1010|286|7043|1552713|3267|1743|21|19|19|3|194|521|73|46.56|47.02|20.21|1957.13|1475.33|3398.88|3432.46|44.25|0.00|1303.78|1475.33|1519.58|2779.11|2823.36|-1923.55| +2452494|58911|2452501|17700|86458|744534|1010|286|7043|1552713|3267|1743|31|19|14|1|255|521|46|73.11|141.10|111.46|1363.44|5127.16|3363.06|6490.60|307.62|0.00|454.02|5127.16|5434.78|5581.18|5888.80|1764.10| +2452494|58911|2452509|8502|86458|744534|1010|286|7043|1552713|3267|1743|6|15|10|5|150|521|33|86.44|212.64|176.49|1192.95|5824.17|2852.52|7017.12|0.00|0.00|2946.90|5824.17|5824.17|8771.07|8771.07|2971.65| +2452292|47442|2452293|9600|58961|1384424|3040|36903|35605|1266345|3198|1414|42|3|13|4|72|522|72|37.61|86.50|75.25|810.00|5418.00|2707.92|6228.00|379.26|0.00|3051.36|5418.00|5797.26|8469.36|8848.62|2710.08| +2452292|47442|2452311|4525|58961|1384424|3040|36903|35605|1266345|3198|1414|48|13|20|1|189|522|5|73.83|135.10|60.79|371.55|303.95|369.15|675.50|15.19|0.00|209.40|303.95|319.14|513.35|528.54|-65.20| +2452292|47442|2452318|17622|58961|1384424|3040|36903|35605|1266345|3198|1414|19|1|12|5|251|522|7|28.89|41.60|6.24|247.52|43.68|202.23|291.20|3.49|0.00|64.05|43.68|47.17|107.73|111.22|-158.55| +2452292|47442|2452377|16284|58961|1384424|3040|36903|35605|1266345|3198|1414|39|7|15|4|60|522|25|27.82|58.97|25.35|840.50|633.75|695.50|1474.25|25.35|0.00|707.50|633.75|659.10|1341.25|1366.60|-61.75| +2452292|47442|2452398|11091|58961|1384424|3040|36903|35605|1266345|3198|1414|51|18|2|5|288|522|9|81.08|125.67|16.33|984.06|146.97|729.72|1131.03|1.46|0.00|271.44|146.97|148.43|418.41|419.87|-582.75| +2452292|47442|2452315|3696|58961|1384424|3040|36903|35605|1266345|3198|1414|54|9|13|3|156|522|61|97.93|168.43|1.68|10171.75|102.48|5973.73|10274.23|4.09|0.00|1952.00|102.48|106.57|2054.48|2058.57|-5871.25| +2452292|47442|2452357|11037|58961|1384424|3040|36903|35605|1266345|3198|1414|27|9|9|5|121|522|43|77.54|105.45|88.57|725.84|3808.51|3334.22|4534.35|0.00|0.00|1813.74|3808.51|3808.51|5622.25|5622.25|474.29| +2452292|47442|2452410|17037|58961|1384424|3040|36903|35605|1266345|3198|1414|49|12|17|5|76|522|84|51.05|133.24|42.63|7611.24|3580.92|4288.20|11192.16|35.80|0.00|782.88|3580.92|3616.72|4363.80|4399.60|-707.28| +2452292|47442|2452379|2475|58961|1384424|3040|36903|35605|1266345|3198|1414|42|9|12|4|56|522|23|26.27|62.52|31.88|704.72|733.24|604.21|1437.96|0.00|469.27|230.00|263.97|263.97|493.97|493.97|-340.24| +2452292|47442|2452294|8637|58961|1384424|3040|36903|35605|1266345|3198|1414|45|30|10|5|67|522|53|39.25|97.73|84.04|725.57|4454.12|2080.25|5179.69|0.00|0.00|1760.66|4454.12|4454.12|6214.78|6214.78|2373.87| +2452292|47442|2452344|9315|58961|1384424|3040|36903|35605|1266345|3198|1414|9|13|9|1|223|522|91|9.93|10.92|6.44|407.68|586.04|903.63|993.72|16.40|257.85|376.74|328.19|344.59|704.93|721.33|-575.44| +2452292|47442|2452349|10872|58961|1384424|3040|36903|35605|1266345|3198|1414|43|7|19|1|63|522|47|21.92|59.84|43.08|787.72|2024.76|1030.24|2812.48|0.00|0.00|365.19|2024.76|2024.76|2389.95|2389.95|994.52| +2452292|47442|2452397|16905|58961|1384424|3040|36903|35605|1266345|3198|1414|54|18|18|5|119|522|37|91.66|197.98|49.49|5494.13|1831.13|3391.42|7325.26|73.24|0.00|2270.69|1831.13|1904.37|4101.82|4175.06|-1560.29| +2452292|47442|2452328|8731|58961|1384424|3040|36903|35605|1266345|3198|1414|42|13|10|3|129|522|36|25.27|28.80|2.01|964.44|72.36|909.72|1036.80|4.34|0.00|487.08|72.36|76.70|559.44|563.78|-837.36| +2452292|47442|2452398|12123|58961|1384424|3040|36903|35605|1266345|3198|1414|39|19|20|2|257|522|87|25.49|38.74|3.48|3067.62|302.76|2217.63|3370.38|27.24|0.00|741.24|302.76|330.00|1044.00|1071.24|-1914.87| +2452292|47442|2452402|1273|58961|1384424|3040|36903|35605|1266345|3198|1414|21|21|11|1|18|522|7|27.38|28.74|23.56|36.26|164.92|191.66|201.18|13.19|0.00|30.17|164.92|178.11|195.09|208.28|-26.74| +2451131|18473|2451175|7558|75559|1904792|3625|34458|63975|910762|2957|12990|34|20|14|2|237|523|59|75.14|153.28|44.45|6420.97|2622.55|4433.26|9043.52|16.52|2071.81|4340.63|550.74|567.26|4891.37|4907.89|-3882.52| +2451131|18473|2451174|2047|75559|1904792|3625|34458|63975|910762|2957|12990|58|13|14|1|118|523|10|77.94|141.07|45.14|959.30|451.40|779.40|1410.70|27.08|0.00|42.30|451.40|478.48|493.70|520.78|-328.00| +2451131|18473|2451187|10156|75559|1904792|3625|34458|63975|910762|2957|12990|22|14|11|4|197|523|92|21.67|65.01|31.20|3110.52|2870.40|1993.64|5980.92|134.62|947.23|2870.40|1923.17|2057.79|4793.57|4928.19|-70.47| +2451131|18473|2451225|10591|75559|1904792|3625|34458|63975|910762|2957|12990|19|4|16|4|215|523|95|98.37|102.30|62.40|3790.50|5928.00|9345.15|9718.50|118.56|0.00|3789.55|5928.00|6046.56|9717.55|9836.11|-3417.15| +2451131|18473|2451223|12280|75559|1904792|3625|34458|63975|910762|2957|12990|50|10|13|3|43|523|52|96.25|118.38|37.88|4186.00|1969.76|5005.00|6155.76|19.69|0.00|3016.00|1969.76|1989.45|4985.76|5005.45|-3035.24| +2451131|18473|2451159|5932|75559|1904792|3625|34458|63975|910762|2957|12990|43|8|5|3|291|523|64|56.64|64.00|56.96|450.56|3645.44|3624.96|4096.00|145.81|0.00|696.32|3645.44|3791.25|4341.76|4487.57|20.48| +2451131|18473|2451199|4081|75559|1904792|3625|34458|63975|910762|2957|12990|16|2|8|4|290|523|85|13.14|17.87|2.68|1291.15|227.80|1116.90|1518.95|0.00|0.00|698.70|227.80|227.80|926.50|926.50|-889.10| +2451131|18473|2451199|9116|75559|1904792|3625|34458|63975|910762|2957|12990|50|19|4|5|209|523|91|35.78|42.22|21.11|1921.01|1921.01|3255.98|3842.02|5.76|1728.90|1267.63|192.11|197.87|1459.74|1465.50|-3063.87| +2451131|18473|2451230|16411|75559|1904792|3625|34458|63975|910762|2957|12990|8|8|2|4|296|523|1|77.25|152.95|102.47|50.48|102.47|77.25|152.95|8.19|0.00|64.23|102.47|110.66|166.70|174.89|25.22| +2451131|18473|2451166|6190|75559|1904792|3625|34458|63975|910762|2957|12990|31|19|7|4|8|523|20|57.42|69.47|12.50|1139.40|250.00|1148.40|1389.40|5.00|0.00|555.60|250.00|255.00|805.60|810.60|-898.40| +2451131|18473|2451144|15526|75559|1904792|3625|34458|63975|910762|2957|12990|19|10|12|3|264|523|21|74.89|203.70|173.14|641.76|3635.94|1572.69|4277.70|290.87|0.00|812.70|3635.94|3926.81|4448.64|4739.51|2063.25| +2451131|18473|2451232|14236|75559|1904792|3625|34458|63975|910762|2957|12990|10|20|3|3|183|523|11|30.75|47.04|38.57|93.17|424.27|338.25|517.44|3.35|89.09|0.00|335.18|338.53|335.18|338.53|-3.07| +2451844|70719|2451869|12323|13806|320657|1908|41959|68994|294264|1312|46163|19|14|3|1|16|524|19|57.68|141.89|103.57|728.08|1967.83|1095.92|2695.91|137.74|0.00|1024.29|1967.83|2105.57|2992.12|3129.86|871.91| +2451844|70719|2451869|12557|13806|320657|1908|41959|68994|294264|1312|46163|19|8|7|2|154|524|90|56.33|104.77|25.14|7166.70|2262.60|5069.70|9429.30|203.63|0.00|1225.80|2262.60|2466.23|3488.40|3692.03|-2807.10| +2451844|70719|2451913|8693|13806|320657|1908|41959|68994|294264|1312|46163|14|11|1|1|44|524|57|82.63|177.65|133.23|2531.94|7594.11|4709.91|10126.05|151.88|0.00|4050.42|7594.11|7745.99|11644.53|11796.41|2884.20| +2451844|70719|2451907|3065|13806|320657|1908|41959|68994|294264|1312|46163|7|26|11|1|231|524|9|51.70|114.77|114.77|0.00|1032.93|465.30|1032.93|0.00|0.00|361.44|1032.93|1032.93|1394.37|1394.37|567.63| +2451844|70719|2451923|4189|13806|320657|1908|41959|68994|294264|1312|46163|44|7|5|1|7|524|73|7.75|20.69|16.13|332.88|1177.49|565.75|1510.37|94.19|0.00|528.52|1177.49|1271.68|1706.01|1800.20|611.74| +2451844|70719|2451955|9659|13806|320657|1908|41959|68994|294264|1312|46163|55|26|13|2|191|524|31|5.75|12.53|1.12|353.71|34.72|178.25|388.43|0.69|0.00|89.28|34.72|35.41|124.00|124.69|-143.53| +2451844|70719|2451947|11426|13806|320657|1908|41959|68994|294264|1312|46163|56|20|6|3|190|524|48|74.80|199.71|125.81|3547.20|6038.88|3590.40|9586.08|362.33|0.00|383.04|6038.88|6401.21|6421.92|6784.25|2448.48| +2451844|70719|2451875|17309|13806|320657|1908|41959|68994|294264|1312|46163|41|20|11|1|101|524|13|37.66|108.46|99.78|112.84|1297.14|489.58|1409.98|116.74|0.00|211.38|1297.14|1413.88|1508.52|1625.26|807.56| +2451844|70719|2451865|787|13806|320657|1908|41959|68994|294264|1312|46163|38|19|13|3|183|524|65|57.72|132.75|3.98|8370.05|258.70|3751.80|8628.75|12.93|0.00|2588.30|258.70|271.63|2847.00|2859.93|-3493.10| +2451844|70719|2451908|9644|13806|320657|1908|41959|68994|294264|1312|46163|55|2|6|2|81|524|64|6.78|9.28|0.09|588.16|5.76|433.92|593.92|0.23|0.00|272.64|5.76|5.99|278.40|278.63|-428.16| +2451844|70719|2451870|1207|13806|320657|1908|41959|68994|294264|1312|46163|5|23|1|5|46|524|65|54.04|111.32|55.66|3617.90|3617.90|3512.60|7235.80|0.00|0.00|289.25|3617.90|3617.90|3907.15|3907.15|105.30| +2451844|70719|2451947|15055|13806|320657|1908|41959|68994|294264|1312|46163|43|7|19|1|243|524|40|71.14|103.86|75.81|1122.00|3032.40|2845.60|4154.40|30.32|0.00|2077.20|3032.40|3062.72|5109.60|5139.92|186.80| +2451844|70719|2451929|16961|13806|320657|1908|41959|68994|294264|1312|46163|41|1|11|1|1|524|72|57.89|79.88|68.69|805.68|4945.68|4168.08|5751.36|49.45|0.00|1552.32|4945.68|4995.13|6498.00|6547.45|777.60| +2451844|70719|2451867|16481|13806|320657|1908|41959|68994|294264|1312|46163|8|26|20|4|235|524|55|85.89|147.73|0.00|8125.15|0.00|4723.95|8125.15|0.00|0.00|730.95|0.00|0.00|730.95|730.95|-4723.95| +2451844|70719|2451845|563|13806|320657|1908|41959|68994|294264|1312|46163|49|2|19|3|240|524|24|9.08|19.97|3.79|388.32|90.96|217.92|479.28|3.63|0.00|38.16|90.96|94.59|129.12|132.75|-126.96| +2451844|70719|2451867|5630|13806|320657|1908|41959|68994|294264|1312|46163|56|1|13|5|277|524|94|76.14|185.78|167.20|1746.52|15716.80|7157.16|17463.32|132.02|12416.27|3143.36|3300.53|3432.55|6443.89|6575.91|-3856.63| +2452182|80406|2452224|11055|30342|87408|2074|740|33393|1714639|1185|28251|9|9|15|5|65|525|4|57.32|106.04|96.49|38.20|385.96|229.28|424.16|34.73|0.00|46.64|385.96|420.69|432.60|467.33|156.68| +2452182|80406|2452192|2759|30342|87408|2074|740|33393|1714639|1185|28251|29|25|4|2|293|525|97|16.86|44.17|19.43|2399.78|1884.71|1635.42|4284.49|169.62|0.00|771.15|1884.71|2054.33|2655.86|2825.48|249.29| +2452182|80406|2452260|6695|30342|87408|2074|740|33393|1714639|1185|28251|45|21|16|4|169|525|7|20.41|33.88|8.80|175.56|61.60|142.87|237.16|3.69|0.00|4.69|61.60|65.29|66.29|69.98|-81.27| +2452182|80406|2452209|14173|30342|87408|2074|740|33393|1714639|1185|28251|37|29|11|5|206|525|42|96.45|123.45|104.93|777.84|4407.06|4050.90|5184.90|0.00|0.00|414.54|4407.06|4407.06|4821.60|4821.60|356.16| +2452182|80406|2452278|7025|30342|87408|2074|740|33393|1714639|1185|28251|19|29|10|5|300|525|55|18.01|50.42|15.12|1941.50|831.60|990.55|2773.10|66.52|0.00|110.55|831.60|898.12|942.15|1008.67|-158.95| +2452182|80406|2452209|15491|30342|87408|2074|740|33393|1714639|1185|28251|37|21|20|2|31|525|98|79.79|199.47|5.98|18962.02|586.04|7819.42|19548.06|0.00|0.00|2736.16|586.04|586.04|3322.20|3322.20|-7233.38| +2452182|80406|2452191|1657|30342|87408|2074|740|33393|1714639|1185|28251|9|23|19|1|123|525|3|89.28|149.09|16.39|398.10|49.17|267.84|447.27|0.98|0.00|219.15|49.17|50.15|268.32|269.30|-218.67| +2452182|80406|2452296|16803|30342|87408|2074|740|33393|1714639|1185|28251|55|5|8|3|48|525|74|73.96|110.20|66.12|3261.92|4892.88|5473.04|8154.80|52.84|4305.73|3098.38|587.15|639.99|3685.53|3738.37|-4885.89| +2452182|80406|2452264|37|30342|87408|2074|740|33393|1714639|1185|28251|31|27|19|2|75|525|42|43.28|82.66|14.05|2881.62|590.10|1817.76|3471.72|47.20|0.00|1561.98|590.10|637.30|2152.08|2199.28|-1227.66| +2452182|80406|2452221|5499|30342|87408|2074|740|33393|1714639|1185|28251|55|21|14|5|223|525|1|48.15|91.48|1.82|89.66|1.82|48.15|91.48|0.00|0.12|38.42|1.70|1.70|40.12|40.12|-46.45| +2451453|63407|2451555|8420|76195|1759798|6601|45804|41565|1811051|3413|38616|40|16|17|5|26|526|77|43.93|64.57|43.90|1591.59|3380.30|3382.61|4971.89|33.80|0.00|646.03|3380.30|3414.10|4026.33|4060.13|-2.31| +2451453|63407|2451460|4442|76195|1759798|6601|45804|41565|1811051|3413|38616|44|13|18|4|213|526|36|55.38|82.51|23.10|2138.76|831.60|1993.68|2970.36|8.31|0.00|1306.80|831.60|839.91|2138.40|2146.71|-1162.08| +2451453|63407|2451460|11683|76195|1759798|6601|45804|41565|1811051|3413|38616|56|7|12|2|35|526|79|27.13|74.06|46.65|2165.39|3685.35|2143.27|5850.74|152.94|626.50|2164.60|3058.85|3211.79|5223.45|5376.39|915.58| +2451453|63407|2451551|15434|76195|1759798|6601|45804|41565|1811051|3413|38616|37|19|14|3|269|526|37|79.67|223.07|198.53|907.98|7345.61|2947.79|8253.59|0.00|0.00|4044.10|7345.61|7345.61|11389.71|11389.71|4397.82| +2451453|63407|2451544|11116|76195|1759798|6601|45804|41565|1811051|3413|38616|52|1|18|2|246|526|49|27.55|34.71|19.43|748.72|952.07|1349.95|1700.79|12.56|742.61|697.27|209.46|222.02|906.73|919.29|-1140.49| +2451453|63407|2451554|1004|76195|1759798|6601|45804|41565|1811051|3413|38616|10|10|19|4|292|526|16|86.79|183.99|5.51|2855.68|88.16|1388.64|2943.84|7.93|0.00|1030.24|88.16|96.09|1118.40|1126.33|-1300.48| +2451453|63407|2451561|6811|76195|1759798|6601|45804|41565|1811051|3413|38616|26|16|14|4|174|526|13|29.97|39.26|12.17|352.17|158.21|389.61|510.38|3.16|0.00|234.65|158.21|161.37|392.86|396.02|-231.40| +2451453|63407|2451500|8899|76195|1759798|6601|45804|41565|1811051|3413|38616|13|4|19|5|231|526|3|51.59|63.45|49.49|41.88|148.47|154.77|190.35|13.36|0.00|83.73|148.47|161.83|232.20|245.56|-6.30| +2451453|63407|2451509|1178|76195|1759798|6601|45804|41565|1811051|3413|38616|58|16|1|3|195|526|31|45.83|52.70|3.16|1535.74|97.96|1420.73|1633.70|0.93|79.34|81.53|18.62|19.55|100.15|101.08|-1402.11| +2451453|63407|2451556|14560|76195|1759798|6601|45804|41565|1811051|3413|38616|19|7|1|3|115|526|29|56.45|90.32|78.57|340.75|2278.53|1637.05|2619.28|0.00|683.55|340.46|1594.98|1594.98|1935.44|1935.44|-42.07| +2451453|63407|2451459|3644|76195|1759798|6601|45804|41565|1811051|3413|38616|10|14|7|1|275|526|52|9.07|17.50|15.05|127.40|782.60|471.64|910.00|0.00|0.00|382.20|782.60|782.60|1164.80|1164.80|310.96| +2451527|59936|2451645|14932|67015|724866|5117|49368|89073|1511036|6921|9172|10|1|13|3|83|527|6|92.88|234.05|21.06|1277.94|126.36|557.28|1404.30|2.52|0.00|421.26|126.36|128.88|547.62|550.14|-430.92| +2451527|59936|2451609|811|67015|724866|5117|49368|89073|1511036|6921|9172|46|28|19|5|131|527|24|16.24|19.81|5.54|342.48|132.96|389.76|475.44|11.96|0.00|218.64|132.96|144.92|351.60|363.56|-256.80| +2451527|59936|2451539|14276|67015|724866|5117|49368|89073|1511036|6921|9172|14|10|7|5|143|527|4|69.79|104.68|97.35|29.32|389.40|279.16|418.72|3.89|0.00|196.76|389.40|393.29|586.16|590.05|110.24| +2451527|59936|2451640|1594|67015|724866|5117|49368|89073|1511036|6921|9172|20|26|20|3|249|527|67|37.71|78.43|69.01|631.14|4623.67|2526.57|5254.81|369.89|0.00|2416.69|4623.67|4993.56|7040.36|7410.25|2097.10| +2451527|59936|2451581|9871|67015|724866|5117|49368|89073|1511036|6921|9172|8|13|6|3|140|527|87|18.57|38.99|35.87|271.44|3120.69|1615.59|3392.13|14.98|2933.44|575.94|187.25|202.23|763.19|778.17|-1428.34| +2451527|59936|2451615|15388|67015|724866|5117|49368|89073|1511036|6921|9172|8|16|6|3|240|527|94|38.02|65.39|60.15|492.56|5654.10|3573.88|6146.66|395.78|0.00|1351.72|5654.10|6049.88|7005.82|7401.60|2080.22| +2451527|59936|2451615|11728|67015|724866|5117|49368|89073|1511036|6921|9172|14|16|20|4|35|527|13|38.30|109.53|50.38|768.95|654.94|497.90|1423.89|39.29|0.00|598.00|654.94|694.23|1252.94|1292.23|157.04| +2451527|59936|2451591|6590|67015|724866|5117|49368|89073|1511036|6921|9172|31|19|1|5|156|527|99|43.47|101.71|95.60|604.89|9464.40|4303.53|10069.29|567.86|0.00|3221.46|9464.40|10032.26|12685.86|13253.72|5160.87| +2451527|59936|2451593|3070|67015|724866|5117|49368|89073|1511036|6921|9172|55|28|15|2|119|527|11|44.47|61.81|53.15|95.26|584.65|489.17|679.91|13.68|432.64|142.78|152.01|165.69|294.79|308.47|-337.16| +2451527|59936|2451640|5953|67015|724866|5117|49368|89073|1511036|6921|9172|28|2|2|4|196|527|13|80.22|109.90|72.53|485.81|942.89|1042.86|1428.70|84.86|0.00|14.17|942.89|1027.75|957.06|1041.92|-99.97| +2451527|59936|2451574|17488|67015|724866|5117|49368|89073|1511036|6921|9172|8|14|14|2|33|527|98|91.33|114.16|68.49|4475.66|6712.02|8950.34|11187.68|134.24|0.00|894.74|6712.02|6846.26|7606.76|7741.00|-2238.32| +2451527|59936|2451555|9386|67015|724866|5117|49368|89073|1511036|6921|9172|7|16|7|2|257|527|50|7.00|8.26|0.49|388.50|24.50|350.00|413.00|0.98|12.25|37.00|12.25|13.23|49.25|50.23|-337.75| +2451527|59936|2451608|5122|67015|724866|5117|49368|89073|1511036|6921|9172|1|26|17|5|210|527|30|89.66|255.53|33.21|6669.60|996.30|2689.80|7665.90|69.74|0.00|0.00|996.30|1066.04|996.30|1066.04|-1693.50| +2451527|59936|2451599|13255|67015|724866|5117|49368|89073|1511036|6921|9172|46|7|4|2|177|527|55|33.21|40.18|28.52|641.30|1568.60|1826.55|2209.90|31.37|0.00|618.75|1568.60|1599.97|2187.35|2218.72|-257.95| +2452161|45507|2452246|5787|73725|1518216|3954|40845|56974|1207123|3965|2456|31|1|13|3|233|528|14|98.63|254.46|17.81|3313.10|249.34|1380.82|3562.44|6.95|172.04|142.38|77.30|84.25|219.68|226.63|-1303.52| +2452161|45507|2452229|17197|73725|1518216|3954|40845|56974|1207123|3965|2456|47|17|14|4|114|528|71|28.02|61.08|28.70|2298.98|2037.70|1989.42|4336.68|81.50|0.00|519.72|2037.70|2119.20|2557.42|2638.92|48.28| +2452161|45507|2452166|4379|73725|1518216|3954|40845|56974|1207123|3965|2456|29|1|17|3|282|528|43|59.52|65.47|30.11|1520.48|1294.73|2559.36|2815.21|90.63|0.00|1013.08|1294.73|1385.36|2307.81|2398.44|-1264.63| +2452161|45507|2452271|3001|73725|1518216|3954|40845|56974|1207123|3965|2456|43|25|7|3|189|528|43|95.57|145.26|21.78|5309.64|936.54|4109.51|6246.18|0.00|0.00|2435.95|936.54|936.54|3372.49|3372.49|-3172.97| +2452161|45507|2452193|16721|73725|1518216|3954|40845|56974|1207123|3965|2456|59|1|15|1|290|528|62|40.76|93.34|75.60|1099.88|4687.20|2527.12|5787.08|210.92|468.72|57.66|4218.48|4429.40|4276.14|4487.06|1691.36| +2452161|45507|2452225|7007|73725|1518216|3954|40845|56974|1207123|3965|2456|17|17|6|1|263|528|70|80.54|93.42|72.86|1439.20|5100.20|5637.80|6539.40|306.01|0.00|196.00|5100.20|5406.21|5296.20|5602.21|-537.60| +2452161|45507|2452173|737|73725|1518216|3954|40845|56974|1207123|3965|2456|15|5|18|3|48|528|54|69.60|176.08|61.62|6180.84|3327.48|3758.40|9508.32|133.09|0.00|2757.24|3327.48|3460.57|6084.72|6217.81|-430.92| +2452161|45507|2452166|8887|73725|1518216|3954|40845|56974|1207123|3965|2456|7|5|12|1|30|528|53|5.47|8.15|1.54|350.33|81.62|289.91|431.95|2.44|0.00|125.08|81.62|84.06|206.70|209.14|-208.29| +2452161|45507|2452253|10859|73725|1518216|3954|40845|56974|1207123|3965|2456|55|3|6|5|284|528|4|44.10|47.62|31.90|62.88|127.60|176.40|190.48|7.65|0.00|66.64|127.60|135.25|194.24|201.89|-48.80| +2451633|13389|2451728|5348|82975|1158676|1930|6784|99244|1534773|2146|27000|38|1|5|1|170|529|74|81.84|87.56|49.90|2786.84|3692.60|6056.16|6479.44|295.40|0.00|2008.36|3692.60|3988.00|5700.96|5996.36|-2363.56| +2451633|13389|2451696|13082|82975|1158676|1930|6784|99244|1534773|2146|27000|25|14|3|3|143|529|45|15.98|47.62|29.52|814.50|1328.40|719.10|2142.90|66.42|0.00|814.05|1328.40|1394.82|2142.45|2208.87|609.30| +2451633|13389|2451732|6194|82975|1158676|1930|6784|99244|1534773|2146|27000|26|20|5|2|9|529|15|97.56|100.48|11.05|1341.45|165.75|1463.40|1507.20|1.65|0.00|286.35|165.75|167.40|452.10|453.75|-1297.65| +2451633|13389|2451719|9374|82975|1158676|1930|6784|99244|1534773|2146|27000|13|11|2|1|190|529|27|77.40|118.42|108.94|255.96|2941.38|2089.80|3197.34|0.00|0.00|1182.87|2941.38|2941.38|4124.25|4124.25|851.58| +2451633|13389|2451655|254|82975|1158676|1930|6784|99244|1534773|2146|27000|59|25|6|1|269|529|10|85.16|230.78|2.30|2284.80|23.00|851.60|2307.80|0.92|0.00|623.10|23.00|23.92|646.10|647.02|-828.60| +2451633|13389|2451694|6479|82975|1158676|1930|6784|99244|1534773|2146|27000|8|8|18|1|254|529|57|39.67|82.11|37.77|2527.38|2152.89|2261.19|4680.27|0.00|0.00|0.00|2152.89|2152.89|2152.89|2152.89|-108.30| +2451633|13389|2451688|2099|82975|1158676|1930|6784|99244|1534773|2146|27000|11|14|20|2|141|529|85|15.90|32.43|22.05|882.30|1874.25|1351.50|2756.55|18.74|0.00|771.80|1874.25|1892.99|2646.05|2664.79|522.75| +2451633|13389|2451671|2075|82975|1158676|1930|6784|99244|1534773|2146|27000|17|1|10|2|194|529|6|4.71|8.14|1.87|37.62|11.22|28.26|48.84|0.81|2.13|12.18|9.09|9.90|21.27|22.08|-19.17| +2451633|13389|2451669|943|82975|1158676|1930|6784|99244|1534773|2146|27000|41|20|14|4|117|529|78|47.96|80.09|43.24|2874.30|3372.72|3740.88|6247.02|236.09|0.00|2186.34|3372.72|3608.81|5559.06|5795.15|-368.16| +2451633|13389|2451683|2708|82975|1158676|1930|6784|99244|1534773|2146|27000|13|14|9|5|131|529|19|76.57|77.33|13.91|1204.98|264.29|1454.83|1469.27|15.85|0.00|264.29|264.29|280.14|528.58|544.43|-1190.54| +2451633|13389|2451672|11939|82975|1158676|1930|6784|99244|1534773|2146|27000|11|14|10|2|171|529|53|49.24|73.86|42.83|1644.59|2269.99|2609.72|3914.58|45.39|0.00|1721.97|2269.99|2315.38|3991.96|4037.35|-339.73| +2451052|36174|2451153|4579|30183|384861|1939|29817|7837|1411362|4298|12976|32|1|8|2|246|530|28|17.57|47.43|16.12|876.68|451.36|491.96|1328.04|0.00|162.48|52.92|288.88|288.88|341.80|341.80|-203.08| +2451052|36174|2451123|13930|30183|384861|1939|29817|7837|1411362|4298|12976|58|25|6|4|285|530|10|57.88|109.39|33.91|754.80|339.10|578.80|1093.90|27.12|0.00|536.00|339.10|366.22|875.10|902.22|-239.70| +2451052|36174|2451133|13975|30183|384861|1939|29817|7837|1411362|4298|12976|44|13|18|2|153|530|98|97.80|157.45|97.61|5864.32|9565.78|9584.40|15430.10|765.26|0.00|2931.18|9565.78|10331.04|12496.96|13262.22|-18.62| +2451052|36174|2451067|8491|30183|384861|1939|29817|7837|1411362|4298|12976|38|22|17|4|154|530|14|20.27|28.37|22.97|75.60|321.58|283.78|397.18|9.64|0.00|39.62|321.58|331.22|361.20|370.84|37.80| +2451052|36174|2451062|12562|30183|384861|1939|29817|7837|1411362|4298|12976|37|25|10|4|138|530|91|85.31|209.00|144.21|5895.89|13123.11|7763.21|19019.00|1049.84|0.00|4754.75|13123.11|14172.95|17877.86|18927.70|5359.90| +2451052|36174|2451100|13376|30183|384861|1939|29817|7837|1411362|4298|12976|37|2|5|3|168|530|61|68.97|124.14|119.17|303.17|7269.37|4207.17|7572.54|581.54|0.00|3558.74|7269.37|7850.91|10828.11|11409.65|3062.20| +2451052|36174|2451105|12992|30183|384861|1939|29817|7837|1411362|4298|12976|44|8|17|1|138|530|17|67.85|191.33|44.00|2504.61|748.00|1153.45|3252.61|29.02|22.44|682.89|725.56|754.58|1408.45|1437.47|-427.89| +2451052|36174|2451061|11078|30183|384861|1939|29817|7837|1411362|4298|12976|1|28|9|1|110|530|77|75.12|150.24|27.04|9486.40|2082.08|5784.24|11568.48|0.00|0.00|3007.62|2082.08|2082.08|5089.70|5089.70|-3702.16| +2451052|36174|2451060|8341|30183|384861|1939|29817|7837|1411362|4298|12976|7|8|6|5|181|530|71|17.66|25.78|16.49|659.59|1170.79|1253.86|1830.38|35.12|0.00|567.29|1170.79|1205.91|1738.08|1773.20|-83.07| +2451052|36174|2451068|11050|30183|384861|1939|29817|7837|1411362|4298|12976|19|20|2|3|224|530|75|51.34|60.58|40.58|1500.00|3043.50|3850.50|4543.50|60.87|0.00|1226.25|3043.50|3104.37|4269.75|4330.62|-807.00| +2451052|36174|2451147|17734|30183|384861|1939|29817|7837|1411362|4298|12976|55|25|1|1|111|530|26|19.68|34.04|4.42|770.12|114.92|511.68|885.04|2.29|0.00|44.20|114.92|117.21|159.12|161.41|-396.76| +2451052|36174|2451082|5995|30183|384861|1939|29817|7837|1411362|4298|12976|13|7|1|3|175|530|5|98.01|218.56|30.59|939.85|152.95|490.05|1092.80|4.58|38.23|360.60|114.72|119.30|475.32|479.90|-375.33| +2451052|36174|2451059|5866|30183|384861|1939|29817|7837|1411362|4298|12976|1|28|12|4|36|530|2|49.39|116.06|100.97|30.18|201.94|98.78|232.12|12.11|0.00|104.44|201.94|214.05|306.38|318.49|103.16| +2452106|67676|2452213|10393|53984|138073|4209|18298|77629|1788981|2612|9501|19|1|12|1|297|531|30|18.69|25.04|10.01|450.90|300.30|560.70|751.20|6.00|0.00|157.50|300.30|306.30|457.80|463.80|-260.40| +2452106|67676|2452147|16139|53984|138073|4209|18298|77629|1788981|2612|9501|45|17|18|4|260|531|55|37.32|64.56|7.10|3160.30|390.50|2052.60|3550.80|0.00|0.00|532.40|390.50|390.50|922.90|922.90|-1662.10| +2452106|67676|2452178|14633|53984|138073|4209|18298|77629|1788981|2612|9501|9|9|5|5|5|531|66|94.59|206.20|101.03|6941.22|6667.98|6242.94|13609.20|333.39|0.00|1768.80|6667.98|7001.37|8436.78|8770.17|425.04| +2452106|67676|2452148|3|53984|138073|4209|18298|77629|1788981|2612|9501|49|15|8|3|128|531|32|33.76|90.13|45.06|1442.24|1441.92|1080.32|2884.16|115.35|0.00|749.76|1441.92|1557.27|2191.68|2307.03|361.60| +2452106|67676|2452115|6515|53984|138073|4209|18298|77629|1788981|2612|9501|51|5|20|2|294|531|7|98.51|115.25|82.98|225.89|580.86|689.57|806.75|52.27|0.00|266.21|580.86|633.13|847.07|899.34|-108.71| +2452106|67676|2452125|10021|53984|138073|4209|18298|77629|1788981|2612|9501|7|15|9|5|244|531|33|80.09|164.98|143.53|707.85|4736.49|2642.97|5444.34|378.91|0.00|217.47|4736.49|5115.40|4953.96|5332.87|2093.52| +2452106|67676|2452207|14185|53984|138073|4209|18298|77629|1788981|2612|9501|43|1|13|5|284|531|74|24.88|38.56|23.52|1112.96|1740.48|1841.12|2853.44|0.00|0.00|884.30|1740.48|1740.48|2624.78|2624.78|-100.64| +2452106|67676|2452199|3905|53984|138073|4209|18298|77629|1788981|2612|9501|39|11|12|5|31|531|98|61.89|115.73|69.43|4537.40|6804.14|6065.22|11341.54|0.00|0.00|5556.60|6804.14|6804.14|12360.74|12360.74|738.92| +2452106|67676|2452186|13737|53984|138073|4209|18298|77629|1788981|2612|9501|33|1|15|4|110|531|24|27.83|40.35|32.28|193.68|774.72|667.92|968.40|61.97|0.00|0.00|774.72|836.69|774.72|836.69|106.80| +2451199|71254|2451250|16064|82406|1295449|2539|9582|48510|455120|2358|34601|14|1|17|3|49|532|29|93.84|122.93|39.33|2424.40|1140.57|2721.36|3564.97|91.24|0.00|1069.23|1140.57|1231.81|2209.80|2301.04|-1580.79| +2451199|71254|2451316|7880|82406|1295449|2539|9582|48510|455120|2358|34601|32|13|17|2|97|532|28|49.76|137.83|77.18|1698.20|2161.04|1393.28|3859.24|121.88|129.66|115.64|2031.38|2153.26|2147.02|2268.90|638.10| +2451199|71254|2451250|16046|82406|1295449|2539|9582|48510|455120|2358|34601|56|22|2|5|148|532|95|42.56|72.77|26.92|4355.75|2557.40|4043.20|6913.15|6.64|2224.93|137.75|332.47|339.11|470.22|476.86|-3710.73| +2451199|71254|2451215|9952|82406|1295449|2539|9582|48510|455120|2358|34601|40|4|2|4|53|532|19|10.84|24.60|8.36|308.56|158.84|205.96|467.40|6.35|0.00|163.59|158.84|165.19|322.43|328.78|-47.12| +2451199|71254|2451261|560|82406|1295449|2539|9582|48510|455120|2358|34601|26|16|7|2|197|532|40|98.22|127.68|31.92|3830.40|1276.80|3928.80|5107.20|12.76|0.00|2400.00|1276.80|1289.56|3676.80|3689.56|-2652.00| +2451199|71254|2451298|12799|82406|1295449|2539|9582|48510|455120|2358|34601|55|19|19|3|7|532|50|79.10|210.40|113.61|4839.50|5680.50|3955.00|10520.00|0.00|0.00|1472.50|5680.50|5680.50|7153.00|7153.00|1725.50| +2451199|71254|2451254|5870|82406|1295449|2539|9582|48510|455120|2358|34601|55|14|8|3|38|532|31|43.16|48.77|36.57|378.20|1133.67|1337.96|1511.87|68.02|0.00|483.60|1133.67|1201.69|1617.27|1685.29|-204.29| +2451199|71254|2451202|13594|82406|1295449|2539|9582|48510|455120|2358|34601|13|28|13|1|106|532|75|58.11|61.01|28.67|2425.50|2150.25|4358.25|4575.75|21.50|0.00|2196.00|2150.25|2171.75|4346.25|4367.75|-2208.00| +2451199|71254|2451312|13898|82406|1295449|2539|9582|48510|455120|2358|34601|50|16|2|5|258|532|64|41.56|86.02|57.63|1816.96|3688.32|2659.84|5505.28|0.00|0.00|1155.84|3688.32|3688.32|4844.16|4844.16|1028.48| +2451199|71254|2451292|10225|82406|1295449|2539|9582|48510|455120|2358|34601|16|8|11|2|46|532|6|70.87|124.02|93.01|186.06|558.06|425.22|744.12|33.48|0.00|163.68|558.06|591.54|721.74|755.22|132.84| +2451199|71254|2451205|10795|82406|1295449|2539|9582|48510|455120|2358|34601|37|22|15|3|291|532|35|61.11|97.16|56.35|1428.35|1972.25|2138.85|3400.60|4.93|1873.63|1292.20|98.62|103.55|1390.82|1395.75|-2040.23| +2451798|54711|2451889|6473|78665|1683755|4398|37041|23300|427049|3064|37307|53|11|2|3|137|533|95|77.51|152.69|114.51|3627.10|10878.45|7363.45|14505.55|761.49|0.00|4931.45|10878.45|11639.94|15809.90|16571.39|3515.00| +2451798|54711|2451871|3260|78665|1683755|4398|37041|23300|427049|3064|37307|50|25|10|2|4|533|35|71.63|157.58|23.63|4688.25|827.05|2507.05|5515.30|0.00|0.00|1764.70|827.05|827.05|2591.75|2591.75|-1680.00| +2451798|54711|2451812|4766|78665|1683755|4398|37041|23300|427049|3064|37307|50|29|7|1|95|533|97|50.47|110.52|40.89|6754.11|3966.33|4895.59|10720.44|118.98|0.00|2036.03|3966.33|4085.31|6002.36|6121.34|-929.26| +2451798|54711|2451897|17408|78665|1683755|4398|37041|23300|427049|3064|37307|2|8|8|1|104|533|46|28.85|42.69|35.00|353.74|1610.00|1327.10|1963.74|56.35|805.00|333.50|805.00|861.35|1138.50|1194.85|-522.10| +2451798|54711|2451855|1315|78665|1683755|4398|37041|23300|427049|3064|37307|59|25|6|4|262|533|97|3.70|6.51|5.27|120.28|511.19|358.90|631.47|46.00|0.00|106.70|511.19|557.19|617.89|663.89|152.29| +2451798|54711|2451836|3121|78665|1683755|4398|37041|23300|427049|3064|37307|29|23|9|1|93|533|75|5.88|7.99|2.71|396.00|203.25|441.00|599.25|0.00|0.00|287.25|203.25|203.25|490.50|490.50|-237.75| +2451798|54711|2451905|6523|78665|1683755|4398|37041|23300|427049|3064|37307|35|8|15|5|282|533|83|43.74|77.85|69.28|711.31|5750.24|3630.42|6461.55|287.51|0.00|451.52|5750.24|6037.75|6201.76|6489.27|2119.82| +2451798|54711|2451832|3739|78665|1683755|4398|37041|23300|427049|3064|37307|55|14|8|3|128|533|68|26.47|61.93|42.73|1305.60|2905.64|1799.96|4211.24|87.16|0.00|1852.32|2905.64|2992.80|4757.96|4845.12|1105.68| +2451798|54711|2451883|2905|78665|1683755|4398|37041|23300|427049|3064|37307|43|11|20|4|295|533|39|56.93|79.13|72.00|278.07|2808.00|2220.27|3086.07|0.00|0.00|1419.21|2808.00|2808.00|4227.21|4227.21|587.73| +2451798|54711|2451853|5299|78665|1683755|4398|37041|23300|427049|3064|37307|44|8|7|1|78|533|43|88.01|227.94|173.23|2352.53|7448.89|3784.43|9801.42|120.67|3426.48|1665.82|4022.41|4143.08|5688.23|5808.90|237.98| +2451798|54711|2451904|9908|78665|1683755|4398|37041|23300|427049|3064|37307|17|23|15|3|141|533|71|99.12|280.50|199.15|5775.85|14139.65|7037.52|19915.50|59.38|8200.99|6970.07|5938.66|5998.04|12908.73|12968.11|-1098.86| +2451798|54711|2451816|2600|78665|1683755|4398|37041|23300|427049|3064|37307|32|7|1|1|48|533|9|56.09|109.93|79.14|277.11|712.26|504.81|989.37|49.85|0.00|435.24|712.26|762.11|1147.50|1197.35|207.45| +2451798|54711|2451845|12127|78665|1683755|4398|37041|23300|427049|3064|37307|20|23|20|1|112|533|31|18.60|40.92|2.45|1192.57|75.95|576.60|1268.52|0.00|31.13|164.61|44.82|44.82|209.43|209.43|-531.78| +2451798|54711|2451869|3746|78665|1683755|4398|37041|23300|427049|3064|37307|14|7|10|1|172|533|88|67.68|195.59|95.83|8778.88|8433.04|5955.84|17211.92|168.66|0.00|4991.36|8433.04|8601.70|13424.40|13593.06|2477.20| +2451798|54711|2451806|15203|78665|1683755|4398|37041|23300|427049|3064|37307|14|2|16|5|44|533|81|84.32|191.40|76.56|9302.04|6201.36|6829.92|15503.40|310.06|0.00|5115.96|6201.36|6511.42|11317.32|11627.38|-628.56| +2451171|80386|2451228|4174|48519|1831912|4105|17851|86627|1613508|2830|21823|1|19|2|4|16|534|90|69.52|90.37|49.70|3660.30|4473.00|6256.80|8133.30|134.19|0.00|1382.40|4473.00|4607.19|5855.40|5989.59|-1783.80| +2451171|80386|2451208|6415|48519|1831912|4105|17851|86627|1613508|2830|21823|50|20|13|5|220|534|93|56.58|162.95|40.73|11366.46|3787.89|5261.94|15154.35|227.27|0.00|302.25|3787.89|4015.16|4090.14|4317.41|-1474.05| +2451171|80386|2451209|14738|48519|1831912|4105|17851|86627|1613508|2830|21823|55|22|6|5|24|534|73|73.45|206.39|63.98|10395.93|4670.54|5361.85|15066.47|373.64|0.00|903.74|4670.54|5044.18|5574.28|5947.92|-691.31| +2451171|80386|2451284|5302|48519|1831912|4105|17851|86627|1613508|2830|21823|28|14|14|2|243|534|79|76.28|140.35|120.70|1552.35|9535.30|6026.12|11087.65|95.35|0.00|4101.68|9535.30|9630.65|13636.98|13732.33|3509.18| +2451171|80386|2451218|10183|48519|1831912|4105|17851|86627|1613508|2830|21823|31|14|7|1|148|534|48|62.05|147.67|7.38|6733.92|354.24|2978.40|7088.16|17.71|0.00|921.12|354.24|371.95|1275.36|1293.07|-2624.16| +2451171|80386|2451194|17710|48519|1831912|4105|17851|86627|1613508|2830|21823|2|20|10|2|181|534|80|48.76|80.94|60.70|1619.20|4856.00|3900.80|6475.20|126.74|631.28|2136.80|4224.72|4351.46|6361.52|6488.26|323.92| +2451171|80386|2451272|12943|48519|1831912|4105|17851|86627|1613508|2830|21823|40|7|17|5|223|534|89|80.13|197.11|9.85|16666.14|876.65|7131.57|17542.79|3.68|692.55|1227.31|184.10|187.78|1411.41|1415.09|-6947.47| +2451171|80386|2451205|4711|48519|1831912|4105|17851|86627|1613508|2830|21823|32|19|16|4|72|534|83|83.10|173.67|24.31|12396.88|2017.73|6897.30|14414.61|20.17|0.00|4467.89|2017.73|2037.90|6485.62|6505.79|-4879.57| +2452112|15445|2452147|12865|71950|248669|5035|22601|64535|863982|3652|18018|9|27|3|4|228|535|27|43.26|81.32|43.91|1010.07|1185.57|1168.02|2195.64|35.56|0.00|21.87|1185.57|1221.13|1207.44|1243.00|17.55| +2452112|15445|2452117|501|71950|248669|5035|22601|64535|863982|3652|18018|5|7|8|4|83|535|2|81.49|158.90|139.83|38.14|279.66|162.98|317.80|16.77|0.00|95.34|279.66|296.43|375.00|391.77|116.68| +2452112|15445|2452146|10843|71950|248669|5035|22601|64535|863982|3652|18018|21|3|13|4|192|535|11|23.35|27.55|14.05|148.50|154.55|256.85|303.05|4.63|0.00|124.19|154.55|159.18|278.74|283.37|-102.30| +2452112|15445|2452210|4899|71950|248669|5035|22601|64535|863982|3652|18018|53|7|1|3|258|535|100|50.50|104.53|60.62|4391.00|6062.00|5050.00|10453.00|545.58|0.00|522.00|6062.00|6607.58|6584.00|7129.58|1012.00| +2452112|15445|2452141|3489|71950|248669|5035|22601|64535|863982|3652|18018|53|17|1|5|232|535|91|34.57|39.75|6.75|3003.00|614.25|3145.87|3617.25|24.57|0.00|1265.81|614.25|638.82|1880.06|1904.63|-2531.62| +2452112|15445|2452161|5599|71950|248669|5035|22601|64535|863982|3652|18018|51|27|19|2|105|535|28|22.83|35.84|20.42|431.76|571.76|639.24|1003.52|22.87|0.00|160.44|571.76|594.63|732.20|755.07|-67.48| +2452112|15445|2452192|17089|71950|248669|5035|22601|64535|863982|3652|18018|15|25|15|5|72|535|57|5.52|10.43|9.28|65.55|528.96|314.64|594.51|25.17|169.26|53.01|359.70|384.87|412.71|437.88|45.06| +2452112|15445|2452157|10723|71950|248669|5035|22601|64535|863982|3652|18018|29|11|17|5|196|535|46|55.31|90.15|36.06|2488.14|1658.76|2544.26|4146.90|122.41|298.57|0.00|1360.19|1482.60|1360.19|1482.60|-1184.07| +2452112|15445|2452221|6449|71950|248669|5035|22601|64535|863982|3652|18018|7|11|18|3|65|535|54|76.09|181.85|125.47|3044.52|6775.38|4108.86|9819.90|609.78|0.00|1767.42|6775.38|7385.16|8542.80|9152.58|2666.52| +2452112|15445|2452213|8973|71950|248669|5035|22601|64535|863982|3652|18018|13|19|20|5|70|535|71|38.35|111.59|10.04|7210.05|712.84|2722.85|7922.89|64.15|0.00|1267.35|712.84|776.99|1980.19|2044.34|-2010.01| +2452112|15445|2452203|16503|71950|248669|5035|22601|64535|863982|3652|18018|39|11|15|1|257|535|72|66.13|151.43|27.25|8940.96|1962.00|4761.36|10902.96|76.91|39.24|217.44|1922.76|1999.67|2140.20|2217.11|-2838.60| +2452639|51529|2452647|16821|49345|1762164|6630|21049|32727|1339885|5906|38772|49|15|11|4|137|536|44|50.39|54.42|1.63|2322.76|71.72|2217.16|2394.48|5.02|0.00|1005.40|71.72|76.74|1077.12|1082.14|-2145.44| +2452639|51529|2452714|15001|49345|1762164|6630|21049|32727|1339885|5906|38772|48|15|19|5|11|536|22|15.18|17.15|10.63|143.44|233.86|333.96|377.30|1.00|133.30|173.36|100.56|101.56|273.92|274.92|-233.40| +2452639|51529|2452728|5895|49345|1762164|6630|21049|32727|1339885|5906|38772|21|12|14|4|254|536|14|11.81|28.22|11.00|241.08|154.00|165.34|395.08|4.62|0.00|165.90|154.00|158.62|319.90|324.52|-11.34| +2452639|51529|2452757|1719|49345|1762164|6630|21049|32727|1339885|5906|38772|24|6|4|5|276|536|78|38.66|60.69|6.67|4213.56|520.26|3015.48|4733.82|1.24|395.39|520.26|124.87|126.11|645.13|646.37|-2890.61| +2452639|51529|2452676|7903|49345|1762164|6630|21049|32727|1339885|5906|38772|37|19|18|1|147|536|97|33.42|35.09|0.35|3369.78|33.95|3241.74|3403.73|3.05|0.00|1394.86|33.95|37.00|1428.81|1431.86|-3207.79| +2452639|51529|2452716|10881|49345|1762164|6630|21049|32727|1339885|5906|38772|45|19|7|4|241|536|72|77.95|167.59|43.57|8929.44|3137.04|5612.40|12066.48|7.52|3011.55|2653.92|125.49|133.01|2779.41|2786.93|-5486.91| +2452639|51529|2452643|6481|49345|1762164|6630|21049|32727|1339885|5906|38772|27|24|20|3|276|536|53|34.48|92.06|74.56|927.50|3951.68|1827.44|4879.18|237.10|0.00|1122.01|3951.68|4188.78|5073.69|5310.79|2124.24| +2452639|51529|2452714|1464|49345|1762164|6630|21049|32727|1339885|5906|38772|9|19|15|5|145|536|54|31.23|62.14|59.03|167.94|3187.62|1686.42|3355.56|127.50|0.00|536.76|3187.62|3315.12|3724.38|3851.88|1501.20| +2452639|51529|2452671|2106|49345|1762164|6630|21049|32727|1339885|5906|38772|19|18|11|5|262|536|52|60.40|167.91|137.68|1571.96|7159.36|3140.80|8731.32|315.72|2648.96|2007.72|4510.40|4826.12|6518.12|6833.84|1369.60| +2452639|51529|2452701|14265|49345|1762164|6630|21049|32727|1339885|5906|38772|6|19|7|4|8|536|12|95.72|219.19|13.15|2472.48|157.80|1148.64|2630.28|11.04|0.00|26.28|157.80|168.84|184.08|195.12|-990.84| +2452639|51529|2452643|16725|49345|1762164|6630|21049|32727|1339885|5906|38772|39|27|18|2|266|536|55|98.32|287.09|238.28|2684.55|13105.40|5407.60|15789.95|786.32|0.00|2210.45|13105.40|13891.72|15315.85|16102.17|7697.80| +2452639|51529|2452675|8142|49345|1762164|6630|21049|32727|1339885|5906|38772|37|24|5|3|171|536|73|70.33|103.38|35.14|4981.52|2565.22|5134.09|7546.74|230.86|0.00|301.49|2565.22|2796.08|2866.71|3097.57|-2568.87| +2452639|51529|2452756|10296|49345|1762164|6630|21049|32727|1339885|5906|38772|48|9|16|2|68|536|64|98.59|210.98|107.59|6616.96|6885.76|6309.76|13502.72|99.15|3580.59|6211.20|3305.17|3404.32|9516.37|9615.52|-3004.59| +2451540|62693|2451577|13340|86938|362951|211|19824|86938|362951|211|19824|58|28|10|5|180|537|23|83.22|89.87|4.49|1963.74|103.27|1914.06|2067.01|0.00|0.00|165.14|103.27|103.27|268.41|268.41|-1810.79| +2451540|62693|2451608|5794|86938|362951|211|19824|86938|362951|211|19824|2|16|13|5|247|537|54|59.06|132.29|59.53|3929.04|3214.62|3189.24|7143.66|289.31|0.00|642.60|3214.62|3503.93|3857.22|4146.53|25.38| +2451540|62693|2451599|4795|86938|362951|211|19824|86938|362951|211|19824|26|28|6|4|235|537|79|31.55|91.17|86.61|360.24|6842.19|2492.45|7202.43|478.95|0.00|3528.93|6842.19|7321.14|10371.12|10850.07|4349.74| +2451540|62693|2451552|17212|86938|362951|211|19824|86938|362951|211|19824|55|1|7|5|93|537|49|13.96|26.38|13.98|607.60|685.02|684.04|1292.62|20.55|0.00|167.58|685.02|705.57|852.60|873.15|0.98| +2451540|62693|2451617|2383|86938|362951|211|19824|86938|362951|211|19824|52|19|13|1|19|537|27|52.84|138.44|26.30|3027.78|710.10|1426.68|3737.88|28.40|0.00|896.94|710.10|738.50|1607.04|1635.44|-716.58| +2451540|62693|2451587|4388|86938|362951|211|19824|86938|362951|211|19824|31|14|7|4|298|537|65|79.48|120.80|15.70|6831.50|1020.50|5166.20|7852.00|91.84|0.00|1884.35|1020.50|1112.34|2904.85|2996.69|-4145.70| +2451540|62693|2451577|1117|86938|362951|211|19824|86938|362951|211|19824|46|19|16|2|28|537|48|82.47|188.85|35.88|7342.56|1722.24|3958.56|9064.80|0.00|0.00|1087.68|1722.24|1722.24|2809.92|2809.92|-2236.32| +2451540|62693|2451582|7093|86938|362951|211|19824|86938|362951|211|19824|7|2|7|5|190|537|11|15.39|15.85|12.20|40.15|134.20|169.29|174.35|6.71|0.00|41.80|134.20|140.91|176.00|182.71|-35.09| +2451540|62693|2451597|7399|86938|362951|211|19824|86938|362951|211|19824|16|16|6|1|173|537|1|91.41|215.72|131.58|84.14|131.58|91.41|215.72|3.94|0.00|92.75|131.58|135.52|224.33|228.27|40.17| +2451540|62693|2451660|11396|86938|362951|211|19824|86938|362951|211|19824|8|2|6|3|187|537|97|76.58|114.87|20.67|9137.40|2004.99|7428.26|11142.39|100.24|0.00|2673.32|2004.99|2105.23|4678.31|4778.55|-5423.27| +2451540|62693|2451612|13556|86938|362951|211|19824|86938|362951|211|19824|19|14|19|5|198|537|66|14.55|19.35|9.48|651.42|625.68|960.30|1277.10|25.02|0.00|242.22|625.68|650.70|867.90|892.92|-334.62| +2452232|72573|2452273|14103|88739|139273|5807|42741|77547|283547|3143|48077|15|9|15|5|215|538|31|36.28|59.13|10.64|1503.19|329.84|1124.68|1833.03|16.62|145.12|659.68|184.72|201.34|844.40|861.02|-939.96| +2452232|72573|2452281|3005|88739|139273|5807|42741|77547|283547|3143|48077|47|23|7|5|200|538|24|14.28|18.84|15.63|77.04|375.12|342.72|452.16|7.50|0.00|216.96|375.12|382.62|592.08|599.58|32.40| +2452232|72573|2452280|12469|88739|139273|5807|42741|77547|283547|3143|48077|17|23|5|5|238|538|18|59.72|173.78|55.60|2127.24|1000.80|1074.96|3128.04|56.74|190.15|1376.28|810.65|867.39|2186.93|2243.67|-264.31| +2452232|72573|2452322|8575|88739|139273|5807|42741|77547|283547|3143|48077|27|21|8|5|272|538|55|67.92|177.95|147.69|1664.30|8122.95|3735.60|9787.25|487.37|0.00|4795.45|8122.95|8610.32|12918.40|13405.77|4387.35| +2452232|72573|2452258|3563|88739|139273|5807|42741|77547|283547|3143|48077|23|11|20|2|41|538|67|35.53|50.80|27.94|1531.62|1871.98|2380.51|3403.60|168.47|0.00|1599.29|1871.98|2040.45|3471.27|3639.74|-508.53| +2452232|72573|2452343|9227|88739|139273|5807|42741|77547|283547|3143|48077|9|9|5|5|215|538|63|28.99|35.94|0.00|2264.22|0.00|1826.37|2264.22|0.00|0.00|248.85|0.00|0.00|248.85|248.85|-1826.37| +2452232|72573|2452272|16519|88739|139273|5807|42741|77547|283547|3143|48077|27|7|1|1|101|538|44|72.26|141.62|21.24|5296.72|934.56|3179.44|6231.28|28.03|0.00|2803.68|934.56|962.59|3738.24|3766.27|-2244.88| +2452232|72573|2452324|12853|88739|139273|5807|42741|77547|283547|3143|48077|29|15|1|4|227|538|5|23.29|25.85|25.59|1.30|127.95|116.45|129.25|11.51|0.00|15.50|127.95|139.46|143.45|154.96|11.50| +2451538|84848|2451547|10552|29104|962312|6367|16375|37032|1013865|6046|38501|7|22|7|5|200|539|82|73.34|140.81|118.28|1847.46|9698.96|6013.88|11546.42|775.91|0.00|3002.02|9698.96|10474.87|12700.98|13476.89|3685.08| +2451538|84848|2451565|17990|29104|962312|6367|16375|37032|1013865|6046|38501|31|25|7|1|228|539|42|88.34|237.63|128.32|4591.02|5389.44|3710.28|9980.46|53.89|0.00|4590.60|5389.44|5443.33|9980.04|10033.93|1679.16| +2451538|84848|2451600|11068|29104|962312|6367|16375|37032|1013865|6046|38501|38|25|13|1|6|539|74|67.03|158.19|101.24|4214.30|7491.76|4960.22|11706.06|449.50|0.00|4213.56|7491.76|7941.26|11705.32|12154.82|2531.54| +2451538|84848|2451547|2359|29104|962312|6367|16375|37032|1013865|6046|38501|32|25|12|3|257|539|99|66.45|119.61|117.21|237.60|11603.79|6578.55|11841.39|0.00|0.00|592.02|11603.79|11603.79|12195.81|12195.81|5025.24| +2451538|84848|2451609|5839|29104|962312|6367|16375|37032|1013865|6046|38501|58|16|15|1|230|539|54|6.61|10.44|7.41|163.62|400.14|356.94|563.76|11.48|236.08|0.00|164.06|175.54|164.06|175.54|-192.88| +2451538|84848|2451599|15520|29104|962312|6367|16375|37032|1013865|6046|38501|20|16|17|2|160|539|38|94.61|108.80|8.70|3803.80|330.60|3595.18|4134.40|23.14|0.00|1116.06|330.60|353.74|1446.66|1469.80|-3264.58| +2451538|84848|2451570|13735|29104|962312|6367|16375|37032|1013865|6046|38501|8|22|9|1|3|539|94|30.26|90.47|76.89|1276.52|7227.66|2844.44|8504.18|505.93|0.00|1870.60|7227.66|7733.59|9098.26|9604.19|4383.22| +2451538|84848|2451587|7954|29104|962312|6367|16375|37032|1013865|6046|38501|1|2|13|2|28|539|6|10.62|29.20|28.90|1.80|173.40|63.72|175.20|1.73|0.00|42.00|173.40|175.13|215.40|217.13|109.68| +2451538|84848|2451611|15424|29104|962312|6367|16375|37032|1013865|6046|38501|1|25|8|3|65|539|69|75.85|151.70|31.85|8269.65|2197.65|5233.65|10467.30|197.78|0.00|1360.68|2197.65|2395.43|3558.33|3756.11|-3036.00| +2451538|84848|2451653|3235|29104|962312|6367|16375|37032|1013865|6046|38501|44|22|14|3|265|539|35|26.93|61.93|14.86|1647.45|520.10|942.55|2167.55|41.60|0.00|129.85|520.10|561.70|649.95|691.55|-422.45| +2451538|84848|2451560|16711|29104|962312|6367|16375|37032|1013865|6046|38501|19|22|14|3|175|539|29|15.86|44.09|31.30|370.91|907.70|459.94|1278.61|27.23|0.00|549.55|907.70|934.93|1457.25|1484.48|447.76| +2451538|84848|2451562|13172|29104|962312|6367|16375|37032|1013865|6046|38501|34|8|5|5|215|539|56|67.85|99.73|3.98|5362.00|222.88|3799.60|5584.88|0.00|0.00|1842.96|222.88|222.88|2065.84|2065.84|-3576.72| +2451538|84848|2451547|10610|29104|962312|6367|16375|37032|1013865|6046|38501|13|1|5|1|170|539|58|18.24|36.11|0.72|2052.62|41.76|1057.92|2094.38|3.75|0.00|376.42|41.76|45.51|418.18|421.93|-1016.16| +2451538|84848|2451544|3506|29104|962312|6367|16375|37032|1013865|6046|38501|13|10|18|5|169|539|34|87.02|184.48|119.91|2195.38|4076.94|2958.68|6272.32|122.30|0.00|1379.72|4076.94|4199.24|5456.66|5578.96|1118.26| +2451538|84848|2451549|148|29104|962312|6367|16375|37032|1013865|6046|38501|13|13|14|1|28|539|65|38.60|45.16|3.16|2730.00|205.40|2509.00|2935.40|12.32|0.00|527.80|205.40|217.72|733.20|745.52|-2303.60| +2451538|84848|2451608|6206|29104|962312|6367|16375|37032|1013865|6046|38501|37|19|1|2|283|539|21|19.93|46.83|1.87|944.16|39.27|418.53|983.43|0.39|0.00|285.18|39.27|39.66|324.45|324.84|-379.26| +2451163|70094|2451185|7441|27648|1582044|440|42036|27648|1582044|440|42036|49|19|12|4|46|540|44|29.40|46.15|40.15|264.00|1766.60|1293.60|2030.60|33.38|1395.61|0.00|370.99|404.37|370.99|404.37|-922.61| +2451163|70094|2451276|10862|27648|1582044|440|42036|27648|1582044|440|42036|40|28|16|5|94|540|23|7.30|11.16|1.45|223.33|33.35|167.90|256.68|0.28|24.01|61.41|9.34|9.62|70.75|71.03|-158.56| +2451163|70094|2451210|7792|27648|1582044|440|42036|27648|1582044|440|42036|8|2|2|4|99|540|24|47.92|115.96|26.67|2142.96|640.08|1150.08|2783.04|6.40|0.00|723.36|640.08|646.48|1363.44|1369.84|-510.00| +2451163|70094|2451250|2690|27648|1582044|440|42036|27648|1582044|440|42036|10|8|11|2|256|540|95|53.71|99.36|53.65|4342.45|5096.75|5102.45|9439.20|305.80|0.00|94.05|5096.75|5402.55|5190.80|5496.60|-5.70| +2451163|70094|2451259|11590|27648|1582044|440|42036|27648|1582044|440|42036|46|20|15|2|249|540|75|78.04|131.88|25.05|8012.25|1878.75|5853.00|9891.00|169.08|0.00|4747.50|1878.75|2047.83|6626.25|6795.33|-3974.25| +2451163|70094|2451240|12661|27648|1582044|440|42036|27648|1582044|440|42036|13|4|9|5|261|540|9|95.60|161.56|80.78|727.02|727.02|860.40|1454.04|1.59|567.07|537.93|159.95|161.54|697.88|699.47|-700.45| +2451163|70094|2451272|508|27648|1582044|440|42036|27648|1582044|440|42036|13|10|5|1|250|540|23|98.66|196.33|98.16|2257.91|2257.68|2269.18|4515.59|203.19|0.00|1715.80|2257.68|2460.87|3973.48|4176.67|-11.50| +2451163|70094|2451189|1537|27648|1582044|440|42036|27648|1582044|440|42036|52|7|11|5|272|540|8|50.78|93.94|58.24|285.60|465.92|406.24|751.52|18.63|0.00|202.88|465.92|484.55|668.80|687.43|59.68| +2451163|70094|2451212|1828|27648|1582044|440|42036|27648|1582044|440|42036|37|25|17|1|175|540|29|80.37|183.24|87.95|2763.41|2550.55|2330.73|5313.96|25.50|0.00|478.21|2550.55|2576.05|3028.76|3054.26|219.82| +2451163|70094|2451172|16975|27648|1582044|440|42036|27648|1582044|440|42036|10|26|12|2|75|540|79|37.80|108.48|31.45|6085.37|2484.55|2986.20|8569.92|173.91|0.00|2827.41|2484.55|2658.46|5311.96|5485.87|-501.65| +2451163|70094|2451210|16448|27648|1582044|440|42036|27648|1582044|440|42036|28|4|18|1|191|540|3|53.82|145.31|63.93|244.14|191.79|161.46|435.93|0.00|0.00|4.35|191.79|191.79|196.14|196.14|30.33| +2451163|70094|2451251|10267|27648|1582044|440|42036|27648|1582044|440|42036|37|7|16|1|176|540|95|23.53|42.11|8.42|3200.55|799.90|2235.35|4000.45|23.99|0.00|119.70|799.90|823.89|919.60|943.59|-1435.45| +2451163|70094|2451184|9530|27648|1582044|440|42036|27648|1582044|440|42036|1|2|13|4|95|540|28|65.98|180.78|92.19|2480.52|2581.32|1847.44|5061.84|180.69|0.00|607.32|2581.32|2762.01|3188.64|3369.33|733.88| +2452007|72377|2452121|10415|77789|1481120|5251|2182|35494|1594119|4175|34706|9|7|10|4|158|541|87|95.79|100.57|48.27|4550.10|4199.49|8333.73|8749.59|377.95|0.00|1049.22|4199.49|4577.44|5248.71|5626.66|-4134.24| +2452007|72377|2452095|2555|77789|1481120|5251|2182|35494|1594119|4175|34706|39|23|1|3|62|541|89|85.29|122.81|31.93|8088.32|2841.77|7590.81|10930.09|227.34|0.00|218.05|2841.77|3069.11|3059.82|3287.16|-4749.04| +2452007|72377|2452060|10999|77789|1481120|5251|2182|35494|1594119|4175|34706|37|27|19|4|299|541|1|88.23|97.05|85.40|11.65|85.40|88.23|97.05|4.27|0.00|10.67|85.40|89.67|96.07|100.34|-2.83| +2452007|72377|2452048|15245|77789|1481120|5251|2182|35494|1594119|4175|34706|19|11|5|1|227|541|94|77.96|178.52|89.26|8390.44|8390.44|7328.24|16780.88|167.80|0.00|8222.18|8390.44|8558.24|16612.62|16780.42|1062.20| +2452007|72377|2452009|12077|77789|1481120|5251|2182|35494|1594119|4175|34706|23|7|6|3|10|541|53|80.51|132.84|104.94|1478.70|5561.82|4267.03|7040.52|278.09|0.00|2112.05|5561.82|5839.91|7673.87|7951.96|1294.79| +2452007|72377|2452080|2033|77789|1481120|5251|2182|35494|1594119|4175|34706|13|21|2|1|180|541|17|5.46|14.46|14.17|4.93|240.89|92.82|245.82|9.63|0.00|22.10|240.89|250.52|262.99|272.62|148.07| +2452007|72377|2452091|12437|77789|1481120|5251|2182|35494|1594119|4175|34706|3|29|15|3|188|541|74|82.42|146.70|95.35|3799.90|7055.90|6099.08|10855.80|635.03|0.00|3473.56|7055.90|7690.93|10529.46|11164.49|956.82| +2452007|72377|2452088|9193|77789|1481120|5251|2182|35494|1594119|4175|34706|21|13|13|4|115|541|10|74.24|112.10|4.48|1076.20|44.80|742.40|1121.00|0.99|28.22|89.60|16.58|17.57|106.18|107.17|-725.82| +2452007|72377|2452103|15787|77789|1481120|5251|2182|35494|1594119|4175|34706|23|9|11|4|7|541|41|10.55|14.24|13.67|23.37|560.47|432.55|583.84|28.02|0.00|285.77|560.47|588.49|846.24|874.26|127.92| +2452007|72377|2452032|5135|77789|1481120|5251|2182|35494|1594119|4175|34706|41|1|11|4|174|541|21|50.30|120.72|77.26|912.66|1622.46|1056.30|2535.12|48.67|0.00|177.45|1622.46|1671.13|1799.91|1848.58|566.16| +2452007|72377|2452045|15863|77789|1481120|5251|2182|35494|1594119|4175|34706|23|21|20|4|188|541|92|56.67|166.60|111.62|5058.16|10269.04|5213.64|15327.20|0.00|0.00|152.72|10269.04|10269.04|10421.76|10421.76|5055.40| +2452007|72377|2452019|461|77789|1481120|5251|2182|35494|1594119|4175|34706|21|25|15|2|192|541|45|30.96|32.50|30.22|102.60|1359.90|1393.20|1462.50|27.19|0.00|468.00|1359.90|1387.09|1827.90|1855.09|-33.30| +2452007|72377|2452039|7045|77789|1481120|5251|2182|35494|1594119|4175|34706|43|29|8|1|114|541|24|61.56|77.56|16.28|1470.72|390.72|1477.44|1861.44|15.62|0.00|837.60|390.72|406.34|1228.32|1243.94|-1086.72| +2452007|72377|2452051|5609|77789|1481120|5251|2182|35494|1594119|4175|34706|59|3|11|5|179|541|49|15.80|23.85|5.48|900.13|268.52|774.20|1168.65|2.68|0.00|548.80|268.52|271.20|817.32|820.00|-505.68| +2451125|42344|2451161|4957|13983|576434|485|42936|41496|1063886|6736|15187|10|22|8|1|277|542|16|68.90|95.77|42.13|858.24|674.08|1102.40|1532.32|60.66|0.00|336.96|674.08|734.74|1011.04|1071.70|-428.32| +2451125|42344|2451151|14498|13983|576434|485|42936|41496|1063886|6736|15187|32|14|19|5|48|542|52|8.94|10.72|1.82|462.80|94.64|464.88|557.44|1.89|0.00|183.56|94.64|96.53|278.20|280.09|-370.24| +2451125|42344|2451158|15883|13983|576434|485|42936|41496|1063886|6736|15187|10|7|15|3|261|542|12|68.20|107.75|105.59|25.92|1267.08|818.40|1293.00|38.01|0.00|465.48|1267.08|1305.09|1732.56|1770.57|448.68| +2451125|42344|2451216|11575|13983|576434|485|42936|41496|1063886|6736|15187|14|8|2|4|176|542|25|56.68|105.42|25.30|2003.00|632.50|1417.00|2635.50|31.62|0.00|1054.00|632.50|664.12|1686.50|1718.12|-784.50| +2451125|42344|2451241|8371|13983|576434|485|42936|41496|1063886|6736|15187|28|13|19|1|156|542|91|17.02|45.44|44.53|82.81|4052.23|1548.82|4135.04|121.56|0.00|785.33|4052.23|4173.79|4837.56|4959.12|2503.41| +2451125|42344|2451231|2794|13983|576434|485|42936|41496|1063886|6736|15187|56|13|8|1|264|542|42|4.24|7.92|5.46|103.32|229.32|178.08|332.64|9.21|75.67|139.44|153.65|162.86|293.09|302.30|-24.43| +2451125|42344|2451161|14720|13983|576434|485|42936|41496|1063886|6736|15187|49|7|20|1|255|542|39|44.79|67.63|9.46|2268.63|368.94|1746.81|2637.57|7.37|0.00|843.96|368.94|376.31|1212.90|1220.27|-1377.87| +2451125|42344|2451208|10874|13983|576434|485|42936|41496|1063886|6736|15187|25|22|18|2|136|542|43|77.83|175.89|156.54|832.05|6731.22|3346.69|7563.27|67.31|0.00|3252.09|6731.22|6798.53|9983.31|10050.62|3384.53| +2451125|42344|2451222|16576|13983|576434|485|42936|41496|1063886|6736|15187|22|25|7|3|2|542|75|40.83|88.60|73.53|1130.25|5514.75|3062.25|6645.00|110.29|0.00|2126.25|5514.75|5625.04|7641.00|7751.29|2452.50| +2451125|42344|2451137|9974|13983|576434|485|42936|41496|1063886|6736|15187|49|16|13|2|231|542|76|80.87|195.70|119.37|5801.08|9072.12|6146.12|14873.20|362.88|0.00|7287.64|9072.12|9435.00|16359.76|16722.64|2926.00| +2451125|42344|2451182|10676|13983|576434|485|42936|41496|1063886|6736|15187|58|25|6|4|22|542|87|36.62|102.90|24.69|6804.27|2148.03|3185.94|8952.30|150.36|0.00|267.96|2148.03|2298.39|2415.99|2566.35|-1037.91| +2451433|39997|2451437|13675|97749|853118|40|3762|40654|1674462|4175|33897|14|14|1|4|167|543|98|35.86|51.27|25.12|2562.70|2461.76|3514.28|5024.46|24.61|0.00|2461.76|2461.76|2486.37|4923.52|4948.13|-1052.52| +2451433|39997|2451543|14876|97749|853118|40|3762|40654|1674462|4175|33897|44|22|11|5|219|543|67|81.13|223.10|62.46|10762.88|4184.82|5435.71|14947.70|80.34|167.39|298.82|4017.43|4097.77|4316.25|4396.59|-1418.28| +2451433|39997|2451546|9379|97749|853118|40|3762|40654|1674462|4175|33897|34|4|11|1|154|543|21|39.22|96.48|32.80|1337.28|688.80|823.62|2026.08|41.32|0.00|506.52|688.80|730.12|1195.32|1236.64|-134.82| +2451433|39997|2451472|14551|97749|853118|40|3762|40654|1674462|4175|33897|52|26|11|5|162|543|94|82.11|147.79|81.28|6251.94|7640.32|7718.34|13892.26|0.00|0.00|2083.04|7640.32|7640.32|9723.36|9723.36|-78.02| +2451433|39997|2451483|6343|97749|853118|40|3762|40654|1674462|4175|33897|58|19|10|5|250|543|5|67.65|194.83|1.94|964.45|9.70|338.25|974.15|0.67|0.00|0.00|9.70|10.37|9.70|10.37|-328.55| +2451433|39997|2451454|17185|97749|853118|40|3762|40654|1674462|4175|33897|14|10|4|2|180|543|32|5.25|6.03|2.83|102.40|90.56|168.00|192.96|8.15|0.00|7.68|90.56|98.71|98.24|106.39|-77.44| +2451433|39997|2451524|9439|97749|853118|40|3762|40654|1674462|4175|33897|20|22|3|4|15|543|5|86.51|185.13|7.40|888.65|37.00|432.55|925.65|1.62|16.65|83.30|20.35|21.97|103.65|105.27|-412.20| +2451433|39997|2451503|5611|97749|853118|40|3762|40654|1674462|4175|33897|50|25|17|1|205|543|90|38.47|58.85|16.47|3814.20|1482.30|3462.30|5296.50|29.64|0.00|476.10|1482.30|1511.94|1958.40|1988.04|-1980.00| +2451433|39997|2451525|7648|97749|853118|40|3762|40654|1674462|4175|33897|1|16|20|3|266|543|23|23.60|41.06|6.15|802.93|141.45|542.80|944.38|5.65|0.00|396.52|141.45|147.10|537.97|543.62|-401.35| +2451433|39997|2451490|4813|97749|853118|40|3762|40654|1674462|4175|33897|7|7|4|4|154|543|28|44.76|53.26|44.73|238.84|1252.44|1253.28|1491.28|62.62|0.00|491.96|1252.44|1315.06|1744.40|1807.02|-0.84| +2451433|39997|2451463|17678|97749|853118|40|3762|40654|1674462|4175|33897|58|16|11|5|179|543|73|54.04|106.45|80.90|1865.15|5905.70|3944.92|7770.85|295.28|0.00|1708.93|5905.70|6200.98|7614.63|7909.91|1960.78| +2452188|4418|2452226|5433|37105|495441|4624|48560|19154|1608948|3215|19598|51|27|5|3|234|544|99|64.26|79.03|68.75|1017.72|6806.25|6361.74|7823.97|476.43|0.00|3911.49|6806.25|7282.68|10717.74|11194.17|444.51| +2452188|4418|2452275|16889|37105|495441|4624|48560|19154|1608948|3215|19598|49|19|15|5|92|544|85|48.95|112.09|107.60|381.65|9146.00|4160.75|9527.65|93.28|4481.54|2952.90|4664.46|4757.74|7617.36|7710.64|503.71| +2452188|4418|2452286|15355|37105|495441|4624|48560|19154|1608948|3215|19598|19|29|7|4|194|544|30|98.71|173.72|22.58|4534.20|677.40|2961.30|5211.60|13.54|0.00|1094.40|677.40|690.94|1771.80|1785.34|-2283.90| +2452188|4418|2452193|17067|37105|495441|4624|48560|19154|1608948|3215|19598|25|15|13|5|210|544|55|98.59|99.57|63.72|1971.75|3504.60|5422.45|5476.35|70.09|0.00|2245.10|3504.60|3574.69|5749.70|5819.79|-1917.85| +2452188|4418|2452279|6907|37105|495441|4624|48560|19154|1608948|3215|19598|55|23|6|5|114|544|56|33.39|53.09|37.16|892.08|2080.96|1869.84|2973.04|124.85|0.00|326.48|2080.96|2205.81|2407.44|2532.29|211.12| +2452188|4418|2452219|16971|37105|495441|4624|48560|19154|1608948|3215|19598|27|7|4|4|64|544|95|19.14|35.79|23.26|1190.35|2209.70|1818.30|3400.05|22.09|0.00|1257.80|2209.70|2231.79|3467.50|3489.59|391.40| +2452188|4418|2452227|15361|37105|495441|4624|48560|19154|1608948|3215|19598|9|23|9|4|206|544|22|81.81|204.52|51.13|3374.58|1124.86|1799.82|4499.44|0.00|686.16|134.86|438.70|438.70|573.56|573.56|-1361.12| +2452188|4418|2452201|17079|37105|495441|4624|48560|19154|1608948|3215|19598|9|7|6|5|77|544|27|97.43|147.11|129.45|476.82|3495.15|2630.61|3971.97|209.70|0.00|0.00|3495.15|3704.85|3495.15|3704.85|864.54| +2452188|4418|2452258|189|37105|495441|4624|48560|19154|1608948|3215|19598|41|3|10|1|222|544|7|32.39|84.53|65.93|130.20|461.51|226.73|591.71|32.30|0.00|171.57|461.51|493.81|633.08|665.38|234.78| +2452188|4418|2452199|17163|37105|495441|4624|48560|19154|1608948|3215|19598|21|9|19|3|167|544|46|32.55|77.79|4.66|3363.98|214.36|1497.30|3578.34|10.03|47.15|1610.00|167.21|177.24|1777.21|1787.24|-1330.09| +2452188|4418|2452262|14097|37105|495441|4624|48560|19154|1608948|3215|19598|15|5|8|3|228|544|9|46.34|50.97|24.97|234.00|224.73|417.06|458.73|8.98|0.00|197.19|224.73|233.71|421.92|430.90|-192.33| +2452188|4418|2452251|13633|37105|495441|4624|48560|19154|1608948|3215|19598|23|5|14|5|276|544|28|20.15|36.67|11.36|708.68|318.08|564.20|1026.76|0.00|0.00|184.80|318.08|318.08|502.88|502.88|-246.12| +2452188|4418|2452221|1317|37105|495441|4624|48560|19154|1608948|3215|19598|17|9|11|2|46|544|33|3.86|6.56|2.03|149.49|66.99|127.38|216.48|6.02|0.00|103.62|66.99|73.01|170.61|176.63|-60.39| +2452188|4418|2452194|13133|37105|495441|4624|48560|19154|1608948|3215|19598|59|19|7|2|287|544|1|21.07|33.71|28.99|4.72|28.99|21.07|33.71|1.73|0.00|8.76|28.99|30.72|37.75|39.48|7.92| +2452188|4418|2452268|10659|37105|495441|4624|48560|19154|1608948|3215|19598|33|19|3|1|199|544|55|11.40|32.94|11.85|1159.95|651.75|627.00|1811.70|26.07|0.00|416.35|651.75|677.82|1068.10|1094.17|24.75| +2451711|62471|2451776|10583|44203|438654|3952|24850|99405|352514|4250|9948|37|1|3|5|217|545|70|75.30|91.11|66.51|1722.00|4655.70|5271.00|6377.70|93.11|0.00|1147.30|4655.70|4748.81|5803.00|5896.11|-615.30| +2451711|62471|2451788|3535|44203|438654|3952|24850|99405|352514|4250|9948|59|2|7|4|121|545|38|17.83|44.57|35.21|355.68|1337.98|677.54|1693.66|80.27|0.00|186.20|1337.98|1418.25|1524.18|1604.45|660.44| +2451711|62471|2451800|15229|44203|438654|3952|24850|99405|352514|4250|9948|20|29|2|1|103|545|25|56.74|114.04|110.61|85.75|2765.25|1418.50|2851.00|165.91|0.00|1339.75|2765.25|2931.16|4105.00|4270.91|1346.75| +2451711|62471|2451722|11183|44203|438654|3952|24850|99405|352514|4250|9948|7|13|20|3|226|545|90|26.99|36.97|17.00|1797.30|1530.00|2429.10|3327.30|61.20|0.00|1663.20|1530.00|1591.20|3193.20|3254.40|-899.10| +2451711|62471|2451803|12851|44203|438654|3952|24850|99405|352514|4250|9948|19|2|4|3|165|545|29|70.62|198.44|127.00|2071.76|3683.00|2047.98|5754.76|294.64|0.00|2474.28|3683.00|3977.64|6157.28|6451.92|1635.02| +2451711|62471|2451826|12656|44203|438654|3952|24850|99405|352514|4250|9948|41|13|13|4|18|545|6|37.41|72.57|38.46|204.66|230.76|224.46|435.42|20.76|0.00|82.68|230.76|251.52|313.44|334.20|6.30| +2451711|62471|2451806|14315|44203|438654|3952|24850|99405|352514|4250|9948|14|23|16|3|197|545|33|78.68|95.20|74.25|691.35|2450.25|2596.44|3141.60|220.52|0.00|848.10|2450.25|2670.77|3298.35|3518.87|-146.19| +2451711|62471|2451715|12602|44203|438654|3952|24850|99405|352514|4250|9948|35|19|5|5|62|545|96|38.86|107.25|62.20|4324.80|5971.20|3730.56|10296.00|0.00|0.00|1338.24|5971.20|5971.20|7309.44|7309.44|2240.64| +2451711|62471|2451730|14354|44203|438654|3952|24850|99405|352514|4250|9948|35|8|2|4|266|545|56|81.21|108.82|19.58|4997.44|1096.48|4547.76|6093.92|43.85|0.00|2863.84|1096.48|1140.33|3960.32|4004.17|-3451.28| +2451711|62471|2451728|11285|44203|438654|3952|24850|99405|352514|4250|9948|35|25|18|3|35|545|48|81.75|204.37|63.35|6768.96|3040.80|3924.00|9809.76|60.81|0.00|4021.92|3040.80|3101.61|7062.72|7123.53|-883.20| +2451205|71991|2451207|7657|52289|733902|2362|13222|62114|1859199|4202|22662|10|4|3|3|213|546|46|93.09|128.46|14.13|5259.18|649.98|4282.14|5909.16|38.99|0.00|2304.14|649.98|688.97|2954.12|2993.11|-3632.16| +2451205|71991|2451276|8920|52289|733902|2362|13222|62114|1859199|4202|22662|34|28|2|4|36|546|1|81.98|210.68|164.33|46.35|164.33|81.98|210.68|6.57|0.00|80.05|164.33|170.90|244.38|250.95|82.35| +2451205|71991|2451236|10304|52289|733902|2362|13222|62114|1859199|4202|22662|32|14|9|5|244|546|45|60.28|106.09|41.37|2912.40|1861.65|2712.60|4774.05|93.08|0.00|572.85|1861.65|1954.73|2434.50|2527.58|-850.95| +2451205|71991|2451259|17032|52289|733902|2362|13222|62114|1859199|4202|22662|44|7|18|4|60|546|1|46.62|134.26|75.18|59.08|75.18|46.62|134.26|0.00|0.00|61.75|75.18|75.18|136.93|136.93|28.56| +2451205|71991|2451226|17624|52289|733902|2362|13222|62114|1859199|4202|22662|49|22|19|1|280|546|17|62.60|145.23|119.08|444.55|2024.36|1064.20|2468.91|141.70|0.00|1160.25|2024.36|2166.06|3184.61|3326.31|960.16| +2451205|71991|2451220|16096|52289|733902|2362|13222|62114|1859199|4202|22662|4|14|3|5|240|546|84|64.63|139.60|54.44|7153.44|4572.96|5428.92|11726.40|256.08|1371.88|3165.96|3201.08|3457.16|6367.04|6623.12|-2227.84| +2451205|71991|2451273|5587|52289|733902|2362|13222|62114|1859199|4202|22662|43|13|2|2|22|546|51|46.95|122.53|17.15|5374.38|874.65|2394.45|6249.03|34.98|0.00|2124.66|874.65|909.63|2999.31|3034.29|-1519.80| +2451205|71991|2451299|4003|52289|733902|2362|13222|62114|1859199|4202|22662|46|14|2|1|140|546|90|2.42|3.41|0.03|304.20|2.70|217.80|306.90|0.08|0.00|33.30|2.70|2.78|36.00|36.08|-215.10| +2451205|71991|2451273|8551|52289|733902|2362|13222|62114|1859199|4202|22662|16|19|1|2|283|546|3|87.25|112.55|14.63|293.76|43.89|261.75|337.65|3.07|0.00|168.81|43.89|46.96|212.70|215.77|-217.86| +2451205|71991|2451283|14110|52289|733902|2362|13222|62114|1859199|4202|22662|4|20|12|5|21|546|26|44.28|47.37|21.31|677.56|554.06|1151.28|1231.62|38.78|0.00|590.98|554.06|592.84|1145.04|1183.82|-597.22| +2451205|71991|2451209|14188|52289|733902|2362|13222|62114|1859199|4202|22662|34|28|9|1|253|546|13|84.44|144.39|28.87|1501.76|375.31|1097.72|1877.07|26.27|0.00|882.18|375.31|401.58|1257.49|1283.76|-722.41| +2451205|71991|2451242|1970|52289|733902|2362|13222|62114|1859199|4202|22662|32|28|13|1|279|546|11|2.15|3.35|0.80|28.05|8.80|23.65|36.85|0.79|0.00|9.90|8.80|9.59|18.70|19.49|-14.85| +2451329|62977|2451350|17725|91685|1539766|6247|45528|75534|1447167|5771|652|14|1|7|3|229|547|48|74.10|78.54|67.54|528.00|3241.92|3556.80|3769.92|226.93|0.00|1507.68|3241.92|3468.85|4749.60|4976.53|-314.88| +2451329|62977|2451353|10034|91685|1539766|6247|45528|75534|1447167|5771|652|14|2|6|2|201|547|96|71.18|190.76|154.51|3480.00|14832.96|6833.28|18312.96|889.97|0.00|182.40|14832.96|15722.93|15015.36|15905.33|7999.68| +2451329|62977|2451343|7556|91685|1539766|6247|45528|75534|1447167|5771|652|56|1|16|3|113|547|44|15.62|30.45|18.57|522.72|817.08|687.28|1339.80|16.34|0.00|535.92|817.08|833.42|1353.00|1369.34|129.80| +2451329|62977|2451361|9931|91685|1539766|6247|45528|75534|1447167|5771|652|43|16|16|3|180|547|7|11.55|27.14|2.98|169.12|20.86|80.85|189.98|0.00|0.00|0.00|20.86|20.86|20.86|20.86|-59.99| +2451329|62977|2451392|3104|91685|1539766|6247|45528|75534|1447167|5771|652|28|19|15|3|138|547|98|62.48|186.19|93.09|9123.80|9122.82|6123.04|18246.62|364.91|0.00|7298.06|9122.82|9487.73|16420.88|16785.79|2999.78| +2451329|62977|2451367|10039|91685|1539766|6247|45528|75534|1447167|5771|652|10|7|5|3|164|547|83|55.67|138.61|124.74|1151.21|10353.42|4620.61|11504.63|681.25|621.20|5406.62|9732.22|10413.47|15138.84|15820.09|5111.61| +2451329|62977|2451441|2950|91685|1539766|6247|45528|75534|1447167|5771|652|1|28|16|2|131|547|45|65.99|118.12|103.94|638.10|4677.30|2969.55|5315.40|187.09|0.00|1913.40|4677.30|4864.39|6590.70|6777.79|1707.75| +2451329|62977|2451348|2762|91685|1539766|6247|45528|75534|1447167|5771|652|13|13|10|5|23|547|58|94.48|187.07|9.35|10307.76|542.30|5479.84|10850.06|37.96|0.00|2061.32|542.30|580.26|2603.62|2641.58|-4937.54| +2451329|62977|2451359|15218|91685|1539766|6247|45528|75534|1447167|5771|652|58|22|14|1|108|547|82|96.61|273.40|150.37|10088.46|12330.34|7922.02|22418.80|369.91|0.00|10312.32|12330.34|12700.25|22642.66|23012.57|4408.32| +2451329|62977|2451356|3823|91685|1539766|6247|45528|75534|1447167|5771|652|22|8|1|4|270|547|97|6.83|12.43|10.56|181.39|1024.32|662.51|1205.71|36.26|419.97|337.56|604.35|640.61|941.91|978.17|-58.16| +2451329|62977|2451369|12808|91685|1539766|6247|45528|75534|1447167|5771|652|43|14|17|3|289|547|88|67.87|185.96|29.75|13746.48|2618.00|5972.56|16364.48|0.00|1492.26|2290.64|1125.74|1125.74|3416.38|3416.38|-4846.82| +2451329|62977|2451431|176|91685|1539766|6247|45528|75534|1447167|5771|652|7|28|6|5|161|547|18|33.19|64.38|27.03|672.30|486.54|597.42|1158.84|0.00|0.00|567.72|486.54|486.54|1054.26|1054.26|-110.88| +2451329|62977|2451341|14014|91685|1539766|6247|45528|75534|1447167|5771|652|32|10|18|2|199|547|52|86.88|185.05|0.00|9622.60|0.00|4517.76|9622.60|0.00|0.00|2982.72|0.00|0.00|2982.72|2982.72|-4517.76| +2451125|79105|2451214|17468|18618|982987|6203|44972|33182|1129712|4573|2542|44|10|6|4|232|548|91|14.20|20.59|13.38|656.11|1217.58|1292.20|1873.69|12.17|0.00|18.20|1217.58|1229.75|1235.78|1247.95|-74.62| +2451125|79105|2451215|14629|18618|982987|6203|44972|33182|1129712|4573|2542|40|4|2|4|106|548|28|70.65|146.95|99.92|1316.84|2797.76|1978.20|4114.60|251.79|0.00|2057.16|2797.76|3049.55|4854.92|5106.71|819.56| +2451125|79105|2451148|12817|18618|982987|6203|44972|33182|1129712|4573|2542|50|8|1|2|143|548|100|67.73|141.55|103.33|3822.00|10333.00|6773.00|14155.00|206.66|0.00|4104.00|10333.00|10539.66|14437.00|14643.66|3560.00| +2451125|79105|2451178|13286|18618|982987|6203|44972|33182|1129712|4573|2542|32|4|6|2|28|548|59|75.79|187.95|150.36|2217.81|8871.24|4471.61|11089.05|177.42|0.00|1330.45|8871.24|9048.66|10201.69|10379.11|4399.63| +2451125|79105|2451144|8797|18618|982987|6203|44972|33182|1129712|4573|2542|32|8|10|3|13|548|46|59.02|159.94|43.18|5370.96|1986.28|2714.92|7357.24|19.86|0.00|1029.94|1986.28|2006.14|3016.22|3036.08|-728.64| +2451125|79105|2451212|13159|18618|982987|6203|44972|33182|1129712|4573|2542|44|16|18|4|140|548|26|2.72|4.70|4.23|12.22|109.98|70.72|122.20|4.39|0.00|14.56|109.98|114.37|124.54|128.93|39.26| +2451125|79105|2451137|4|18618|982987|6203|44972|33182|1129712|4573|2542|28|28|2|4|6|548|36|7.47|21.43|3.00|663.48|108.00|268.92|771.48|0.00|0.00|0.00|108.00|108.00|108.00|108.00|-160.92| +2451125|79105|2451150|3415|18618|982987|6203|44972|33182|1129712|4573|2542|14|7|10|1|262|548|20|12.64|22.87|2.97|398.00|59.40|252.80|457.40|1.18|0.00|96.00|59.40|60.58|155.40|156.58|-193.40| +2452549|38942|2452552|5865|37416|954241|5819|24644|71024|1268701|6625|1850|3|7|19|1|266|549|14|1.43|1.88|0.82|14.84|11.48|20.02|26.32|0.45|0.00|10.92|11.48|11.93|22.40|22.85|-8.54| +2452549|38942|2452649|8967|37416|954241|5819|24644|71024|1268701|6625|1850|60|9|20|5|20|549|7|73.87|205.35|65.71|977.48|459.97|517.09|1437.45|4.59|0.00|57.47|459.97|464.56|517.44|522.03|-57.12| +2452549|38942|2452585|1141|37416|954241|5819|24644|71024|1268701|6625|1850|25|7|15|3|164|549|2|71.84|98.42|57.08|82.68|114.16|143.68|196.84|0.00|0.00|17.70|114.16|114.16|131.86|131.86|-29.52| +2452549|38942|2452632|16209|37416|954241|5819|24644|71024|1268701|6625|1850|39|25|6|4|22|549|83|26.61|30.06|27.35|224.93|2270.05|2208.63|2494.98|22.70|0.00|972.76|2270.05|2292.75|3242.81|3265.51|61.42| +2452549|38942|2452563|12888|37416|954241|5819|24644|71024|1268701|6625|1850|48|1|18|5|232|549|38|14.94|44.07|11.89|1222.84|451.82|567.72|1674.66|22.59|0.00|33.44|451.82|474.41|485.26|507.85|-115.90| +2452549|38942|2452655|11898|37416|954241|5819|24644|71024|1268701|6625|1850|60|7|11|3|205|549|41|74.01|152.46|19.81|5438.65|812.21|3034.41|6250.86|31.83|357.37|3062.70|454.84|486.67|3517.54|3549.37|-2579.57| +2452549|38942|2452634|8616|37416|954241|5819|24644|71024|1268701|6625|1850|24|13|12|2|202|549|17|54.19|60.15|18.64|705.67|316.88|921.23|1022.55|9.50|0.00|214.71|316.88|326.38|531.59|541.09|-604.35| +2452549|38942|2452599|13968|37416|954241|5819|24644|71024|1268701|6625|1850|42|13|3|4|212|549|5|32.75|48.79|38.05|53.70|190.25|163.75|243.95|9.51|0.00|65.85|190.25|199.76|256.10|265.61|26.50| +2452549|38942|2452570|981|37416|954241|5819|24644|71024|1268701|6625|1850|31|25|3|5|140|549|4|4.82|5.01|0.70|17.24|2.80|19.28|20.04|0.16|0.00|2.00|2.80|2.96|4.80|4.96|-16.48| +2452549|38942|2452660|8970|37416|954241|5819|24644|71024|1268701|6625|1850|25|21|16|1|229|549|36|33.33|60.99|3.04|2086.20|109.44|1199.88|2195.64|9.84|0.00|702.36|109.44|119.28|811.80|821.64|-1090.44| +2452549|38942|2452604|17377|37416|954241|5819|24644|71024|1268701|6625|1850|18|24|5|3|298|549|64|36.43|107.10|70.68|2330.88|4523.52|2331.52|6854.40|316.64|0.00|479.36|4523.52|4840.16|5002.88|5319.52|2192.00| +2452549|38942|2452567|4951|37416|954241|5819|24644|71024|1268701|6625|1850|36|24|13|2|211|549|85|14.91|25.79|16.76|767.55|1424.60|1267.35|2192.15|113.96|0.00|262.65|1424.60|1538.56|1687.25|1801.21|157.25| +2452549|38942|2452570|7266|37416|954241|5819|24644|71024|1268701|6625|1850|39|25|16|2|165|549|69|45.14|102.91|29.84|5041.83|2058.96|3114.66|7100.79|20.58|0.00|2911.11|2058.96|2079.54|4970.07|4990.65|-1055.70| +2452549|38942|2452569|6393|37416|954241|5819|24644|71024|1268701|6625|1850|39|21|12|3|164|549|50|63.91|121.42|26.71|4735.50|1335.50|3195.50|6071.00|40.06|0.00|303.50|1335.50|1375.56|1639.00|1679.06|-1860.00| +2452549|38942|2452574|1437|37416|954241|5819|24644|71024|1268701|6625|1850|6|15|16|1|279|549|62|7.90|8.61|8.00|37.82|496.00|489.80|533.82|29.76|0.00|4.96|496.00|525.76|500.96|530.72|6.20| +2450841|56094|2450851|15457|88857|139124|944|49964|42085|310280|5471|36891|49|20|4|5|43|550|47|14.39|26.18|3.66|1058.44|172.02|676.33|1230.46|13.76|0.00|344.51|172.02|185.78|516.53|530.29|-504.31| +2450841|56094|2450868|10730|88857|139124|944|49964|42085|310280|5471|36891|26|1|1|1|211|550|66|47.34|87.57|68.30|1271.82|4507.80|3124.44|5779.62|44.17|3876.70|1329.24|631.10|675.27|1960.34|2004.51|-2493.34| +2450841|56094|2450852|16844|88857|139124|944|49964|42085|310280|5471|36891|2|26|16|5|177|550|79|72.06|122.50|62.47|4742.37|4935.13|5692.74|9677.50|148.05|0.00|2031.88|4935.13|5083.18|6967.01|7115.06|-757.61| +2450841|56094|2450855|16582|88857|139124|944|49964|42085|310280|5471|36891|1|14|5|2|299|550|80|59.63|115.08|72.50|3406.40|5800.00|4770.40|9206.40|232.00|0.00|3037.60|5800.00|6032.00|8837.60|9069.60|1029.60| +2450841|56094|2450923|2371|88857|139124|944|49964|42085|310280|5471|36891|22|22|5|3|51|550|12|55.36|75.28|6.02|831.12|72.24|664.32|903.36|5.05|0.00|153.48|72.24|77.29|225.72|230.77|-592.08| +2450841|56094|2450942|826|88857|139124|944|49964|42085|310280|5471|36891|7|10|15|4|3|550|52|52.75|137.15|86.40|2639.00|4492.80|2743.00|7131.80|134.78|0.00|2995.20|4492.80|4627.58|7488.00|7622.78|1749.80| +2450841|56094|2450955|6379|88857|139124|944|49964|42085|310280|5471|36891|25|14|6|1|16|550|11|24.14|36.45|16.40|220.55|180.40|265.54|400.95|14.43|0.00|156.31|180.40|194.83|336.71|351.14|-85.14| +2450841|56094|2450853|10423|88857|139124|944|49964|42085|310280|5471|36891|58|8|7|2|4|550|95|94.29|223.46|221.22|212.80|21015.90|8957.55|21228.70|1471.11|0.00|5094.85|21015.90|22487.01|26110.75|27581.86|12058.35| +2450841|56094|2450909|17011|88857|139124|944|49964|42085|310280|5471|36891|56|8|8|4|240|550|59|59.93|131.84|55.37|4511.73|3266.83|3535.87|7778.56|13.06|2613.46|1400.07|653.37|666.43|2053.44|2066.50|-2882.50| +2450841|56094|2450902|529|88857|139124|944|49964|42085|310280|5471|36891|44|22|18|4|233|550|93|44.79|117.79|29.44|8216.55|2737.92|4165.47|10954.47|27.37|0.00|656.58|2737.92|2765.29|3394.50|3421.87|-1427.55| +2450841|56094|2450853|1129|88857|139124|944|49964|42085|310280|5471|36891|56|28|11|2|183|550|20|44.95|103.38|59.96|868.40|1199.20|899.00|2067.60|47.96|0.00|889.00|1199.20|1247.16|2088.20|2136.16|300.20| +2450841|56094|2450932|7400|88857|139124|944|49964|42085|310280|5471|36891|7|20|8|5|122|550|78|44.85|59.20|10.65|3786.90|830.70|3498.30|4617.60|58.14|0.00|1523.34|830.70|888.84|2354.04|2412.18|-2667.60| +2451405|49160|2451446|11090|94195|701241|734|10180|854|1399244|5365|3611|31|2|18|2|223|551|94|12.55|34.76|25.72|849.76|2417.68|1179.70|3267.44|193.41|0.00|1404.36|2417.68|2611.09|3822.04|4015.45|1237.98| +2451405|49160|2451410|3694|94195|701241|734|10180|854|1399244|5365|3611|38|10|19|2|128|551|37|57.95|167.47|85.40|3036.59|3159.80|2144.15|6196.39|252.78|0.00|185.74|3159.80|3412.58|3345.54|3598.32|1015.65| +2451405|49160|2451473|11036|94195|701241|734|10180|854|1399244|5365|3611|31|4|3|5|160|551|33|59.65|92.45|78.58|457.71|2593.14|1968.45|3050.85|233.38|0.00|731.94|2593.14|2826.52|3325.08|3558.46|624.69| +2451405|49160|2451442|17036|94195|701241|734|10180|854|1399244|5365|3611|43|14|11|2|10|551|17|78.61|214.60|201.72|218.96|3429.24|1336.37|3648.20|205.75|0.00|838.95|3429.24|3634.99|4268.19|4473.94|2092.87| +2451405|49160|2451513|2474|94195|701241|734|10180|854|1399244|5365|3611|44|4|6|1|19|551|72|12.02|18.99|7.59|820.80|546.48|865.44|1367.28|16.39|0.00|464.40|546.48|562.87|1010.88|1027.27|-318.96| +2451405|49160|2451437|8636|94195|701241|734|10180|854|1399244|5365|3611|52|10|18|2|289|551|30|47.97|105.05|80.88|725.10|2426.40|1439.10|3151.50|218.37|0.00|409.50|2426.40|2644.77|2835.90|3054.27|987.30| +2451405|49160|2451485|9314|94195|701241|734|10180|854|1399244|5365|3611|7|8|9|3|144|551|64|71.07|125.08|46.27|5043.84|2961.28|4548.48|8005.12|177.67|0.00|3761.92|2961.28|3138.95|6723.20|6900.87|-1587.20| +2451405|49160|2451462|10870|94195|701241|734|10180|854|1399244|5365|3611|31|2|8|2|163|551|57|76.46|164.38|134.79|1686.63|7683.03|4358.22|9369.66|614.64|0.00|3185.16|7683.03|8297.67|10868.19|11482.83|3324.81| +2451441|19520|2451496|4262|20968|343721|521|28521|35496|343531|3274|32827|56|25|17|3|92|552|48|8.68|21.78|2.39|930.72|114.72|416.64|1045.44|0.00|0.00|271.68|114.72|114.72|386.40|386.40|-301.92| +2451441|19520|2451464|9616|20968|343721|521|28521|35496|343531|3274|32827|19|26|18|2|206|552|80|19.66|56.62|35.10|1721.60|2808.00|1572.80|4529.60|56.16|0.00|271.20|2808.00|2864.16|3079.20|3135.36|1235.20| +2451441|19520|2451500|12373|20968|343721|521|28521|35496|343531|3274|32827|44|10|9|2|165|552|25|9.34|11.02|1.32|242.50|33.00|233.50|275.50|0.05|32.01|30.25|0.99|1.04|31.24|31.29|-232.51| +2451441|19520|2451510|14054|20968|343721|521|28521|35496|343531|3274|32827|32|25|12|3|137|552|30|24.00|70.80|61.59|276.30|1847.70|720.00|2124.00|0.73|1773.79|870.60|73.91|74.64|944.51|945.24|-646.09| +2451441|19520|2451506|5500|20968|343721|521|28521|35496|343531|3274|32827|34|2|6|3|127|552|12|59.79|111.80|20.12|1100.16|241.44|717.48|1341.60|0.00|0.00|469.56|241.44|241.44|711.00|711.00|-476.04| +2451441|19520|2451469|17266|20968|343721|521|28521|35496|343531|3274|32827|44|19|14|1|165|552|38|69.95|70.64|12.00|2228.32|456.00|2658.10|2684.32|31.92|0.00|1261.60|456.00|487.92|1717.60|1749.52|-2202.10| +2451441|19520|2451487|12310|20968|343721|521|28521|35496|343531|3274|32827|40|22|20|1|299|552|31|30.92|48.54|33.49|466.55|1038.19|958.52|1504.74|29.79|612.53|526.38|425.66|455.45|952.04|981.83|-532.86| +2451441|19520|2451541|13160|20968|343721|521|28521|35496|343531|3274|32827|58|8|20|5|200|552|94|54.50|87.20|67.14|1885.64|6311.16|5123.00|8196.80|63.11|0.00|2049.20|6311.16|6374.27|8360.36|8423.47|1188.16| +2451441|19520|2451522|985|20968|343721|521|28521|35496|343531|3274|32827|16|8|8|4|30|552|27|23.61|47.92|44.56|90.72|1203.12|637.47|1293.84|12.03|0.00|633.96|1203.12|1215.15|1837.08|1849.11|565.65| +2451441|19520|2451480|1765|20968|343721|521|28521|35496|343531|3274|32827|43|28|8|4|47|552|2|19.08|43.88|30.27|27.22|60.54|38.16|87.76|3.63|0.00|26.32|60.54|64.17|86.86|90.49|22.38| +2451441|19520|2451548|17870|20968|343721|521|28521|35496|343531|3274|32827|38|16|8|2|89|552|19|10.96|12.16|3.89|157.13|73.91|208.24|231.04|3.69|0.00|113.05|73.91|77.60|186.96|190.65|-134.33| +2452623|32765|2452647|12259|88340|1253150|4713|23523|71969|41819|2259|31022|60|9|10|3|283|553|4|88.95|171.67|109.86|247.24|439.44|355.80|686.68|0.00|0.00|123.60|439.44|439.44|563.04|563.04|83.64| +2452623|32765|2452722|11335|88340|1253150|4713|23523|71969|41819|2259|31022|55|9|15|2|259|553|88|69.51|79.24|37.24|3696.00|3277.12|6116.88|6973.12|98.31|0.00|1603.36|3277.12|3375.43|4880.48|4978.79|-2839.76| +2452623|32765|2452657|11265|88340|1253150|4713|23523|71969|41819|2259|31022|60|24|1|5|189|553|58|57.71|96.95|84.34|731.38|4891.72|3347.18|5623.10|342.42|0.00|562.02|4891.72|5234.14|5453.74|5796.16|1544.54| +2452623|32765|2452693|4386|88340|1253150|4713|23523|71969|41819|2259|31022|49|27|13|4|48|553|71|33.89|87.77|63.19|1745.18|4486.49|2406.19|6231.67|403.78|0.00|622.67|4486.49|4890.27|5109.16|5512.94|2080.30| +2452623|32765|2452696|16183|88340|1253150|4713|23523|71969|41819|2259|31022|49|3|2|4|106|553|46|63.04|67.45|26.30|1892.90|1209.80|2899.84|3102.70|84.68|0.00|744.28|1209.80|1294.48|1954.08|2038.76|-1690.04| +2452623|32765|2452708|9537|88340|1253150|4713|23523|71969|41819|2259|31022|45|18|8|4|193|553|2|62.92|140.94|107.11|67.66|214.22|125.84|281.88|3.74|139.24|25.36|74.98|78.72|100.34|104.08|-50.86| +2452623|32765|2452675|8781|88340|1253150|4713|23523|71969|41819|2259|31022|57|15|7|1|245|553|59|27.45|53.80|32.28|1269.68|1904.52|1619.55|3174.20|76.18|0.00|571.12|1904.52|1980.70|2475.64|2551.82|284.97| +2452623|32765|2452626|14041|88340|1253150|4713|23523|71969|41819|2259|31022|25|3|11|5|209|553|22|91.88|118.52|5.92|2477.20|130.24|2021.36|2607.44|11.72|0.00|443.08|130.24|141.96|573.32|585.04|-1891.12| +2452623|32765|2452628|8532|88340|1253150|4713|23523|71969|41819|2259|31022|13|1|18|2|239|553|19|52.92|136.00|74.80|1162.80|1421.20|1005.48|2584.00|14.21|0.00|206.72|1421.20|1435.41|1627.92|1642.13|415.72| +2452623|32765|2452694|11670|88340|1253150|4713|23523|71969|41819|2259|31022|24|25|11|2|113|553|85|98.49|151.67|15.16|11603.35|1288.60|8371.65|12891.95|12.88|0.00|4898.55|1288.60|1301.48|6187.15|6200.03|-7083.05| +2452623|32765|2452666|3090|88340|1253150|4713|23523|71969|41819|2259|31022|25|7|2|3|44|553|31|22.79|31.45|18.87|389.98|584.97|706.49|974.95|5.84|0.00|106.95|584.97|590.81|691.92|697.76|-121.52| +2452623|32765|2452688|4734|88340|1253150|4713|23523|71969|41819|2259|31022|7|9|13|1|29|553|39|70.84|195.51|5.86|7396.35|228.54|2762.76|7624.89|4.84|107.41|1753.44|121.13|125.97|1874.57|1879.41|-2641.63| +2452623|32765|2452653|3157|88340|1253150|4713|23523|71969|41819|2259|31022|57|24|2|2|35|553|57|8.06|23.77|22.34|81.51|1273.38|459.42|1354.89|101.87|0.00|609.33|1273.38|1375.25|1882.71|1984.58|813.96| +2451985|57126|2452081|6403|14336|1667066|596|34926|76189|313926|5916|29046|45|29|7|4|21|554|32|54.45|123.05|11.07|3583.36|354.24|1742.40|3937.60|5.73|258.59|1259.84|95.65|101.38|1355.49|1361.22|-1646.75| +2451985|57126|2452061|16317|14336|1667066|596|34926|76189|313926|5916|29046|25|17|15|1|117|554|39|40.29|79.37|63.49|619.32|2476.11|1571.31|3095.43|182.24|198.08|1083.03|2278.03|2460.27|3361.06|3543.30|706.72| +2451985|57126|2451992|10173|14336|1667066|596|34926|76189|313926|5916|29046|1|15|12|4|276|554|95|91.47|193.00|131.24|5867.20|12467.80|8689.65|18335.00|748.06|0.00|6600.60|12467.80|13215.86|19068.40|19816.46|3778.15| +2451985|57126|2452088|11949|14336|1667066|596|34926|76189|313926|5916|29046|13|17|4|4|84|554|6|95.34|241.21|219.50|130.26|1317.00|572.04|1447.26|26.34|0.00|159.18|1317.00|1343.34|1476.18|1502.52|744.96| +2451985|57126|2452045|8339|14336|1667066|596|34926|76189|313926|5916|29046|53|5|2|1|180|554|35|70.39|204.13|173.51|1071.70|6072.85|2463.65|7144.55|364.37|0.00|2571.80|6072.85|6437.22|8644.65|9009.02|3609.20| +2451985|57126|2451994|12877|14336|1667066|596|34926|76189|313926|5916|29046|41|23|10|1|209|554|29|13.79|33.78|11.14|656.56|323.06|399.91|979.62|0.00|0.00|391.79|323.06|323.06|714.85|714.85|-76.85| +2451985|57126|2452093|13241|14336|1667066|596|34926|76189|313926|5916|29046|45|15|10|5|127|554|99|83.82|149.19|38.78|10930.59|3839.22|8298.18|14769.81|86.38|2879.41|3987.72|959.81|1046.19|4947.53|5033.91|-7338.37| +2451985|57126|2452004|8773|14336|1667066|596|34926|76189|313926|5916|29046|9|15|1|4|287|554|53|87.67|248.98|77.18|9105.40|4090.54|4646.51|13195.94|163.62|0.00|5542.21|4090.54|4254.16|9632.75|9796.37|-555.97| +2451985|57126|2452041|1915|14336|1667066|596|34926|76189|313926|5916|29046|55|13|9|3|50|554|80|47.58|136.55|124.26|983.20|9940.80|3806.40|10924.00|76.54|8847.31|764.00|1093.49|1170.03|1857.49|1934.03|-2712.91| +2451985|57126|2452027|12455|14336|1667066|596|34926|76189|313926|5916|29046|51|15|20|2|14|554|83|94.83|110.00|48.40|5112.80|4017.20|7870.89|9130.00|40.17|0.00|2465.10|4017.20|4057.37|6482.30|6522.47|-3853.69| +2451985|57126|2452097|4143|14336|1667066|596|34926|76189|313926|5916|29046|41|3|16|2|28|554|87|41.97|53.72|11.81|3646.17|1027.47|3651.39|4673.64|41.09|0.00|93.09|1027.47|1068.56|1120.56|1161.65|-2623.92| +2451985|57126|2452105|11935|14336|1667066|596|34926|76189|313926|5916|29046|57|29|8|1|227|554|9|35.11|47.74|20.05|249.21|180.45|315.99|429.66|1.53|27.06|111.69|153.39|154.92|265.08|266.61|-162.60| +2451985|57126|2452065|1351|14336|1667066|596|34926|76189|313926|5916|29046|47|27|14|1|141|554|56|85.75|216.09|216.09|0.00|12101.04|4802.00|12101.04|363.03|0.00|2540.72|12101.04|12464.07|14641.76|15004.79|7299.04| +2451985|57126|2451989|631|14336|1667066|596|34926|76189|313926|5916|29046|1|29|19|3|237|554|53|64.16|133.45|128.11|283.02|6789.83|3400.48|7072.85|339.49|0.00|2829.14|6789.83|7129.32|9618.97|9958.46|3389.35| +2452092|39169|2452201|10229|2240|1400662|1037|17859|57556|1210032|231|16253|25|5|6|4|79|555|20|53.37|141.96|123.50|369.20|2470.00|1067.40|2839.20|123.50|0.00|1362.80|2470.00|2593.50|3832.80|3956.30|1402.60| +2452092|39169|2452120|7361|2240|1400662|1037|17859|57556|1210032|231|16253|9|21|7|3|149|555|67|94.56|97.39|77.91|1305.16|5219.97|6335.52|6525.13|365.39|0.00|1565.79|5219.97|5585.36|6785.76|7151.15|-1115.55| +2452092|39169|2452192|17701|2240|1400662|1037|17859|57556|1210032|231|16253|23|11|13|2|131|555|72|15.51|36.60|15.37|1528.56|1106.64|1116.72|2635.20|0.77|1029.17|1238.40|77.47|78.24|1315.87|1316.64|-1039.25| +2452092|39169|2452198|5479|2240|1400662|1037|17859|57556|1210032|231|16253|3|19|5|5|288|555|87|33.34|59.67|4.77|4776.30|414.99|2900.58|5191.29|24.89|0.00|882.18|414.99|439.88|1297.17|1322.06|-2485.59| +2452092|39169|2452140|15511|2240|1400662|1037|17859|57556|1210032|231|16253|17|7|12|5|245|555|95|34.31|79.25|40.41|3689.80|3838.95|3259.45|7528.75|191.94|0.00|3086.55|3838.95|4030.89|6925.50|7117.44|579.50| +2452092|39169|2452108|11785|2240|1400662|1037|17859|57556|1210032|231|16253|43|23|2|2|192|555|94|58.21|64.03|2.56|5778.18|240.64|5471.74|6018.82|7.21|0.00|2647.98|240.64|247.85|2888.62|2895.83|-5231.10| +2452092|39169|2452174|781|2240|1400662|1037|17859|57556|1210032|231|16253|19|1|5|1|110|555|54|80.23|177.30|131.20|2489.40|7084.80|4332.42|9574.20|425.08|0.00|1627.56|7084.80|7509.88|8712.36|9137.44|2752.38| +2452092|39169|2452147|1331|2240|1400662|1037|17859|57556|1210032|231|16253|55|27|20|1|212|555|28|9.89|14.04|9.54|126.00|267.12|276.92|393.12|18.69|0.00|54.88|267.12|285.81|322.00|340.69|-9.80| +2452092|39169|2452150|1757|2240|1400662|1037|17859|57556|1210032|231|16253|19|27|18|5|199|555|43|43.75|122.93|86.05|1585.84|3700.15|1881.25|5285.99|37.00|0.00|475.58|3700.15|3737.15|4175.73|4212.73|1818.90| +2452092|39169|2452191|12843|2240|1400662|1037|17859|57556|1210032|231|16253|47|25|9|3|24|555|81|63.51|125.74|111.90|1121.04|9063.90|5144.31|10184.94|171.30|3353.64|4277.61|5710.26|5881.56|9987.87|10159.17|565.95| +2452092|39169|2452093|10865|2240|1400662|1037|17859|57556|1210032|231|16253|3|23|9|4|289|555|43|45.69|46.60|10.71|1543.27|460.53|1964.67|2003.80|36.84|0.00|981.69|460.53|497.37|1442.22|1479.06|-1504.14| +2452092|39169|2452149|16763|2240|1400662|1037|17859|57556|1210032|231|16253|29|23|12|4|199|555|90|72.47|194.94|89.67|9474.30|8070.30|6522.30|17544.60|80.70|0.00|1578.60|8070.30|8151.00|9648.90|9729.60|1548.00| +2452092|39169|2452198|7417|2240|1400662|1037|17859|57556|1210032|231|16253|9|5|1|2|50|555|83|11.08|25.70|8.99|1386.93|746.17|919.64|2133.10|44.77|0.00|127.82|746.17|790.94|873.99|918.76|-173.47| +2452092|39169|2452194|7189|2240|1400662|1037|17859|57556|1210032|231|16253|5|25|6|4|202|555|23|9.20|18.30|12.07|143.29|277.61|211.60|420.90|0.91|247.07|180.78|30.54|31.45|211.32|212.23|-181.06| +2452092|39169|2452209|7823|2240|1400662|1037|17859|57556|1210032|231|16253|15|27|13|2|258|555|94|39.78|39.78|28.24|1084.76|2654.56|3739.32|3739.32|238.91|0.00|298.92|2654.56|2893.47|2953.48|3192.39|-1084.76| +2452092|39169|2452132|16657|2240|1400662|1037|17859|57556|1210032|231|16253|1|25|12|4|146|555|7|77.95|148.88|128.03|145.95|896.21|545.65|1042.16|62.73|0.00|62.51|896.21|958.94|958.72|1021.45|350.56| +2451431|23055|2451480|10460|95162|948167|1172|44177|28780|1556022|4745|42550|31|22|15|1|37|556|95|67.89|122.20|89.20|3135.00|8474.00|6449.55|11609.00|593.18|0.00|347.70|8474.00|9067.18|8821.70|9414.88|2024.45| +2451431|23055|2451544|12568|95162|948167|1172|44177|28780|1556022|4745|42550|50|2|15|3|131|556|59|54.30|102.08|6.12|5661.64|361.08|3203.70|6022.72|25.27|0.00|2649.69|361.08|386.35|3010.77|3036.04|-2842.62| +2451431|23055|2451508|4946|95162|948167|1172|44177|28780|1556022|4745|42550|1|13|3|4|242|556|44|47.12|138.06|5.52|5831.76|242.88|2073.28|6074.64|21.85|0.00|1154.12|242.88|264.73|1397.00|1418.85|-1830.40| +2451431|23055|2451435|10394|95162|948167|1172|44177|28780|1556022|4745|42550|32|28|10|5|270|556|96|76.89|83.81|37.71|4425.60|3620.16|7381.44|8045.76|217.20|0.00|1769.28|3620.16|3837.36|5389.44|5606.64|-3761.28| +2451431|23055|2451468|9247|95162|948167|1172|44177|28780|1556022|4745|42550|44|14|3|3|226|556|18|36.05|104.90|87.06|321.12|1567.08|648.90|1888.20|125.36|0.00|226.44|1567.08|1692.44|1793.52|1918.88|918.18| +2451431|23055|2451459|12046|95162|948167|1172|44177|28780|1556022|4745|42550|56|25|20|2|221|556|55|89.69|160.54|146.09|794.75|8034.95|4932.95|8829.70|321.39|0.00|1765.50|8034.95|8356.34|9800.45|10121.84|3102.00| +2451431|23055|2451528|17971|95162|948167|1172|44177|28780|1556022|4745|42550|2|14|7|3|282|556|30|79.11|141.60|133.10|255.00|3993.00|2373.30|4248.00|122.98|2236.08|1274.40|1756.92|1879.90|3031.32|3154.30|-616.38| +2451431|23055|2451459|4171|95162|948167|1172|44177|28780|1556022|4745|42550|38|16|5|1|80|556|69|69.96|121.03|18.15|7098.72|1252.35|4827.24|8351.07|87.66|0.00|83.49|1252.35|1340.01|1335.84|1423.50|-3574.89| +2451431|23055|2451527|844|95162|948167|1172|44177|28780|1556022|4745|42550|52|22|4|4|184|556|89|85.44|121.32|23.05|8746.03|2051.45|7604.16|10797.48|41.02|0.00|2266.83|2051.45|2092.47|4318.28|4359.30|-5552.71| +2451431|23055|2451518|8233|95162|948167|1172|44177|28780|1556022|4745|42550|55|2|3|2|261|556|58|67.22|190.23|70.38|6951.30|4082.04|3898.76|11033.34|244.92|0.00|4854.60|4082.04|4326.96|8936.64|9181.56|183.28| +2451431|23055|2451505|14002|95162|948167|1172|44177|28780|1556022|4745|42550|46|1|18|5|292|556|91|66.69|149.38|80.66|6253.52|7340.06|6068.79|13593.58|0.00|0.00|3670.03|7340.06|7340.06|11010.09|11010.09|1271.27| +2451901|53956|2451967|9263|14414|1178807|447|20087|24315|1255882|3360|22200|20|2|20|1|229|557|19|2.18|3.50|3.15|6.65|59.85|41.42|66.50|3.59|0.00|6.65|59.85|63.44|66.50|70.09|18.43| +2451901|53956|2451962|9505|14414|1178807|447|20087|24315|1255882|3360|22200|37|29|1|2|250|557|55|29.07|72.67|63.22|519.75|3477.10|1598.85|3996.85|90.75|2468.74|1518.55|1008.36|1099.11|2526.91|2617.66|-590.49| +2451901|53956|2451905|16571|14414|1178807|447|20087|24315|1255882|3360|22200|2|14|11|2|183|557|50|97.51|119.93|2.39|5877.00|119.50|4875.50|5996.50|1.19|0.00|1978.50|119.50|120.69|2098.00|2099.19|-4756.00| +2451901|53956|2451914|1838|14414|1178807|447|20087|24315|1255882|3360|22200|43|25|4|5|69|557|48|80.72|86.37|15.54|3399.84|745.92|3874.56|4145.76|37.29|0.00|1989.60|745.92|783.21|2735.52|2772.81|-3128.64| +2451901|53956|2451982|4109|14414|1178807|447|20087|24315|1255882|3360|22200|59|20|9|2|268|557|87|32.67|57.82|7.51|4376.97|653.37|2842.29|5030.34|58.80|0.00|703.83|653.37|712.17|1357.20|1416.00|-2188.92| +2451901|53956|2452012|16760|14414|1178807|447|20087|24315|1255882|3360|22200|53|20|18|2|76|557|6|67.47|72.19|11.55|363.84|69.30|404.82|433.14|6.23|0.00|69.30|69.30|75.53|138.60|144.83|-335.52| +2451901|53956|2451953|9781|14414|1178807|447|20087|24315|1255882|3360|22200|26|25|5|4|206|557|10|33.57|45.99|21.15|248.40|211.50|335.70|459.90|0.00|0.00|13.70|211.50|211.50|225.20|225.20|-124.20| +2451901|53956|2451984|6773|14414|1178807|447|20087|24315|1255882|3360|22200|55|26|20|4|145|557|18|68.13|69.49|3.47|1188.36|62.46|1226.34|1250.82|3.12|0.00|0.00|62.46|65.58|62.46|65.58|-1163.88| +2451901|53956|2451903|13226|14414|1178807|447|20087|24315|1255882|3360|22200|8|29|15|5|225|557|97|39.49|92.80|33.40|5761.80|3239.80|3830.53|9001.60|226.78|0.00|1619.90|3239.80|3466.58|4859.70|5086.48|-590.73| +2451410|76304|2451436|7777|67941|839733|5478|47270|79891|939578|1799|13944|25|16|8|2|126|558|30|26.09|56.61|9.05|1426.80|271.50|782.70|1698.30|24.43|0.00|441.30|271.50|295.93|712.80|737.23|-511.20| +2451410|76304|2451506|6259|67941|839733|5478|47270|79891|939578|1799|13944|16|4|14|5|126|558|89|2.37|6.42|0.89|492.17|79.21|210.93|571.38|1.74|35.64|285.69|43.57|45.31|329.26|331.00|-167.36| +2451410|76304|2451486|7783|67941|839733|5478|47270|79891|939578|1799|13944|34|28|5|2|29|558|34|86.76|132.74|126.10|225.76|4287.40|2949.84|4513.16|257.24|0.00|1038.02|4287.40|4544.64|5325.42|5582.66|1337.56| +2451410|76304|2451439|2336|67941|839733|5478|47270|79891|939578|1799|13944|19|16|18|2|73|558|5|89.05|159.39|119.54|199.25|597.70|445.25|796.95|23.90|0.00|326.70|597.70|621.60|924.40|948.30|152.45| +2451410|76304|2451510|15643|67941|839733|5478|47270|79891|939578|1799|13944|25|8|17|4|251|558|58|30.18|78.16|76.59|91.06|4442.22|1750.44|4533.28|0.00|0.00|2130.34|4442.22|4442.22|6572.56|6572.56|2691.78| +2451410|76304|2451479|16540|67941|839733|5478|47270|79891|939578|1799|13944|26|26|11|1|208|558|73|68.35|92.95|16.73|5564.06|1221.29|4989.55|6785.35|97.70|0.00|1221.29|1221.29|1318.99|2442.58|2540.28|-3768.26| +2451410|76304|2451490|17467|67941|839733|5478|47270|79891|939578|1799|13944|26|22|17|5|283|558|15|62.81|123.73|51.96|1076.55|779.40|942.15|1855.95|0.00|0.00|445.35|779.40|779.40|1224.75|1224.75|-162.75| +2451410|76304|2451485|11918|67941|839733|5478|47270|79891|939578|1799|13944|31|20|3|3|243|558|94|93.28|215.47|105.58|10329.66|9924.52|8768.32|20254.18|297.73|0.00|8709.10|9924.52|10222.25|18633.62|18931.35|1156.20| +2451410|76304|2451427|15670|67941|839733|5478|47270|79891|939578|1799|13944|52|19|13|1|199|558|87|19.89|51.51|36.57|1299.78|3181.59|1730.43|4481.37|95.44|0.00|1298.91|3181.59|3277.03|4480.50|4575.94|1451.16| +2451410|76304|2451458|2203|67941|839733|5478|47270|79891|939578|1799|13944|52|16|11|3|128|558|53|12.60|14.11|11.71|127.20|620.63|667.80|747.83|31.03|0.00|276.66|620.63|651.66|897.29|928.32|-47.17| +2451410|76304|2451466|748|67941|839733|5478|47270|79891|939578|1799|13944|58|26|18|2|116|558|13|42.15|120.54|71.11|642.59|924.43|547.95|1567.02|6.37|711.81|438.75|212.62|218.99|651.37|657.74|-335.33| +2451410|76304|2451493|13480|67941|839733|5478|47270|79891|939578|1799|13944|19|26|18|2|116|558|43|66.97|157.37|0.00|6766.91|0.00|2879.71|6766.91|0.00|0.00|1420.72|0.00|0.00|1420.72|1420.72|-2879.71| +2451410|76304|2451515|7432|67941|839733|5478|47270|79891|939578|1799|13944|13|20|2|3|127|558|81|15.25|34.46|32.39|167.67|2623.59|1235.25|2791.26|52.47|0.00|1087.83|2623.59|2676.06|3711.42|3763.89|1388.34| +2451410|76304|2451509|14686|67941|839733|5478|47270|79891|939578|1799|13944|40|22|20|3|119|558|23|7.72|16.44|0.82|359.26|18.86|177.56|378.12|0.18|0.00|3.68|18.86|19.04|22.54|22.72|-158.70| +2451075|76458|2451164|4237|24060|1494238|1663|2641|72071|1279607|6022|46200|38|2|14|3|285|559|90|33.46|69.93|67.13|252.00|6041.70|3011.40|6293.70|241.66|0.00|2768.40|6041.70|6283.36|8810.10|9051.76|3030.30| +2451075|76458|2451096|220|24060|1494238|1663|2641|72071|1279607|6022|46200|40|4|5|2|272|559|14|90.00|126.90|115.47|160.02|1616.58|1260.00|1776.60|32.33|0.00|586.18|1616.58|1648.91|2202.76|2235.09|356.58| +2451075|76458|2451159|11734|24060|1494238|1663|2641|72071|1279607|6022|46200|58|25|3|3|276|559|71|8.29|15.17|12.59|183.18|893.89|588.59|1077.07|0.00|0.00|96.56|893.89|893.89|990.45|990.45|305.30| +2451075|76458|2451138|3676|24060|1494238|1663|2641|72071|1279607|6022|46200|49|14|8|3|121|559|31|72.91|104.99|61.94|1334.55|1920.14|2260.21|3254.69|26.49|1478.50|845.99|441.64|468.13|1287.63|1314.12|-1818.57| +2451075|76458|2451194|15122|24060|1494238|1663|2641|72071|1279607|6022|46200|8|13|2|4|36|559|56|42.13|66.56|57.24|521.92|3205.44|2359.28|3727.36|224.38|0.00|670.88|3205.44|3429.82|3876.32|4100.70|846.16| +2451075|76458|2451180|13453|24060|1494238|1663|2641|72071|1279607|6022|46200|52|7|14|1|168|559|90|65.30|188.71|132.09|5095.80|11888.10|5877.00|16983.90|0.00|0.00|1698.30|11888.10|11888.10|13586.40|13586.40|6011.10| +2451075|76458|2451114|6283|24060|1494238|1663|2641|72071|1279607|6022|46200|13|14|2|3|237|559|88|21.52|58.74|41.70|1499.52|3669.60|1893.76|5169.12|110.08|0.00|1498.64|3669.60|3779.68|5168.24|5278.32|1775.84| +2451075|76458|2451181|10612|24060|1494238|1663|2641|72071|1279607|6022|46200|52|14|19|2|48|559|61|40.09|114.25|93.68|1254.77|5714.48|2445.49|6969.25|171.43|0.00|3136.01|5714.48|5885.91|8850.49|9021.92|3268.99| +2451075|76458|2451093|17954|24060|1494238|1663|2641|72071|1279607|6022|46200|20|20|4|4|124|559|21|46.98|80.80|26.66|1136.94|559.86|986.58|1696.80|33.59|0.00|220.50|559.86|593.45|780.36|813.95|-426.72| +2451612|70302|2451663|12416|42999|1344928|810|47939|35064|1642135|2926|32408|56|13|18|1|137|560|98|58.83|175.31|12.27|15977.92|1202.46|5765.34|17180.38|21.64|480.98|343.00|721.48|743.12|1064.48|1086.12|-5043.86| +2451612|70302|2451726|13076|42999|1344928|810|47939|35064|1642135|2926|32408|5|20|6|1|169|560|27|35.98|58.28|33.80|660.96|912.60|971.46|1573.56|27.37|0.00|582.12|912.60|939.97|1494.72|1522.09|-58.86| +2451612|70302|2451695|10451|42999|1344928|810|47939|35064|1642135|2926|32408|20|2|6|2|231|560|51|62.63|98.95|30.67|3482.28|1564.17|3194.13|5046.45|62.56|0.00|49.98|1564.17|1626.73|1614.15|1676.71|-1629.96| +2451612|70302|2451663|11833|42999|1344928|810|47939|35064|1642135|2926|32408|2|8|11|2|30|560|74|19.15|24.70|15.06|713.36|1114.44|1417.10|1827.80|55.72|0.00|639.36|1114.44|1170.16|1753.80|1809.52|-302.66| +2451612|70302|2451681|6884|42999|1344928|810|47939|35064|1642135|2926|32408|44|20|19|2|95|560|59|37.31|94.02|30.08|3772.46|1774.72|2201.29|5547.18|159.72|0.00|610.06|1774.72|1934.44|2384.78|2544.50|-426.57| +2451612|70302|2451687|5845|42999|1344928|810|47939|35064|1642135|2926|32408|29|11|7|3|136|560|9|30.69|74.57|57.41|154.44|516.69|276.21|671.13|38.02|41.33|187.83|475.36|513.38|663.19|701.21|199.15| +2451612|70302|2451701|3860|42999|1344928|810|47939|35064|1642135|2926|32408|43|1|10|4|268|560|26|31.97|75.12|16.52|1523.60|429.52|831.22|1953.12|0.00|0.00|410.02|429.52|429.52|839.54|839.54|-401.70| +2451612|70302|2451641|13579|42999|1344928|810|47939|35064|1642135|2926|32408|59|1|2|5|221|560|96|50.69|127.23|109.41|1710.72|10503.36|4866.24|12214.08|840.26|0.00|5984.64|10503.36|11343.62|16488.00|17328.26|5637.12| +2451612|70302|2451660|9685|42999|1344928|810|47939|35064|1642135|2926|32408|20|26|17|5|203|560|32|99.55|125.43|12.54|3612.48|401.28|3185.60|4013.76|11.91|268.85|1404.80|132.43|144.34|1537.23|1549.14|-3053.17| +2451612|70302|2451716|7567|42999|1344928|810|47939|35064|1642135|2926|32408|1|29|2|4|36|560|20|77.51|81.38|38.24|862.80|764.80|1550.20|1627.60|53.53|0.00|667.20|764.80|818.33|1432.00|1485.53|-785.40| +2451050|75055|2451138|8101|69030|1767400|6923|26846|69030|1767400|6923|26846|28|19|15|2|242|561|26|6.79|9.70|6.11|93.34|158.86|176.54|252.20|7.94|0.00|7.54|158.86|166.80|166.40|174.34|-17.68| +2451050|75055|2451114|9212|69030|1767400|6923|26846|69030|1767400|6923|26846|43|7|19|4|266|561|38|61.83|115.62|15.03|3822.42|571.14|2349.54|4393.56|5.71|456.91|0.00|114.23|119.94|114.23|119.94|-2235.31| +2451050|75055|2451105|16864|69030|1767400|6923|26846|69030|1767400|6923|26846|25|4|10|5|280|561|21|24.16|63.54|20.96|894.18|440.16|507.36|1334.34|39.61|0.00|26.67|440.16|479.77|466.83|506.44|-67.20| +2451050|75055|2451056|9062|69030|1767400|6923|26846|69030|1767400|6923|26846|49|1|14|2|33|561|54|30.11|55.40|2.21|2872.26|119.34|1625.94|2991.60|8.35|0.00|448.74|119.34|127.69|568.08|576.43|-1506.60| +2451050|75055|2451054|7756|69030|1767400|6923|26846|69030|1767400|6923|26846|4|2|4|3|292|561|3|36.45|105.70|51.79|161.73|155.37|109.35|317.10|0.00|124.29|145.86|31.08|31.08|176.94|176.94|-78.27| +2451050|75055|2451055|3416|69030|1767400|6923|26846|69030|1767400|6923|26846|19|20|2|1|7|561|12|41.26|50.74|26.89|286.20|322.68|495.12|608.88|19.36|0.00|103.44|322.68|342.04|426.12|445.48|-172.44| +2451050|75055|2451163|901|69030|1767400|6923|26846|69030|1767400|6923|26846|52|2|7|2|77|561|93|48.80|130.29|26.05|9694.32|2422.65|4538.40|12116.97|121.13|0.00|4482.60|2422.65|2543.78|6905.25|7026.38|-2115.75| +2451050|75055|2451071|1453|69030|1767400|6923|26846|69030|1767400|6923|26846|16|2|12|5|258|561|71|22.92|33.92|22.04|843.48|1564.84|1627.32|2408.32|62.59|0.00|216.55|1564.84|1627.43|1781.39|1843.98|-62.48| +2451050|75055|2451089|13000|69030|1767400|6923|26846|69030|1767400|6923|26846|38|14|19|5|228|561|63|97.35|239.48|62.26|11164.86|3922.38|6133.05|15087.24|274.56|0.00|2564.73|3922.38|4196.94|6487.11|6761.67|-2210.67| +2451050|75055|2451153|212|69030|1767400|6923|26846|69030|1767400|6923|26846|25|25|3|3|43|561|52|59.21|118.42|97.10|1108.64|5049.20|3078.92|6157.84|141.37|3029.52|1539.20|2019.68|2161.05|3558.88|3700.25|-1059.24| +2451795|44702|2451845|8570|62618|151529|87|41798|81120|1764827|5331|35842|13|8|13|1|149|562|33|24.92|71.76|68.17|118.47|2249.61|822.36|2368.08|112.48|0.00|118.14|2249.61|2362.09|2367.75|2480.23|1427.25| +2451795|44702|2451798|10940|62618|151529|87|41798|81120|1764827|5331|35842|38|11|2|3|121|562|95|69.91|159.39|43.03|11054.20|4087.85|6641.45|15142.05|367.90|0.00|2876.60|4087.85|4455.75|6964.45|7332.35|-2553.60| +2451795|44702|2451875|8035|62618|151529|87|41798|81120|1764827|5331|35842|41|19|9|3|254|562|41|70.78|201.01|58.29|5851.52|2389.89|2901.98|8241.41|0.00|0.00|3708.45|2389.89|2389.89|6098.34|6098.34|-512.09| +2451795|44702|2451815|4919|62618|151529|87|41798|81120|1764827|5331|35842|32|13|7|2|37|562|8|53.39|149.49|88.19|490.40|705.52|427.12|1195.92|56.44|0.00|71.68|705.52|761.96|777.20|833.64|278.40| +2451795|44702|2451817|14281|62618|151529|87|41798|81120|1764827|5331|35842|29|23|19|1|273|562|49|68.13|196.89|141.76|2701.37|6946.24|3338.37|9647.61|408.43|1111.39|2122.19|5834.85|6243.28|7957.04|8365.47|2496.48| +2451795|44702|2451810|10939|62618|151529|87|41798|81120|1764827|5331|35842|44|11|6|4|172|562|83|11.25|28.23|11.57|1382.78|960.31|933.75|2343.09|48.01|0.00|726.25|960.31|1008.32|1686.56|1734.57|26.56| +2451795|44702|2451847|14474|62618|151529|87|41798|81120|1764827|5331|35842|14|8|2|2|169|562|25|11.98|33.30|7.32|649.50|183.00|299.50|832.50|5.49|0.00|58.25|183.00|188.49|241.25|246.74|-116.50| +2451795|44702|2451796|13409|62618|151529|87|41798|81120|1764827|5331|35842|50|8|7|1|258|562|58|24.01|59.54|52.99|379.90|3073.42|1392.58|3453.32|215.13|0.00|241.28|3073.42|3288.55|3314.70|3529.83|1680.84| +2451775|64793|2451787|7999|39801|1802774|594|40814|51919|1896666|3427|20047|31|17|9|2|55|563|30|28.31|68.79|58.47|309.60|1754.10|849.30|2063.70|0.00|0.00|206.10|1754.10|1754.10|1960.20|1960.20|904.80| +2451775|64793|2451838|7358|39801|1802774|594|40814|51919|1896666|3427|20047|7|14|12|1|50|563|20|91.83|165.29|90.90|1487.80|1818.00|1836.60|3305.80|4.18|1399.86|991.60|418.14|422.32|1409.74|1413.92|-1418.46| +2451775|64793|2451790|9877|39801|1802774|594|40814|51919|1896666|3427|20047|41|26|10|3|196|563|86|45.85|82.07|9.84|6211.78|846.24|3943.10|7058.02|13.37|177.71|2258.36|668.53|681.90|2926.89|2940.26|-3274.57| +2451775|64793|2451860|11149|39801|1802774|594|40814|51919|1896666|3427|20047|26|8|13|4|262|563|36|73.48|180.76|142.80|1366.56|5140.80|2645.28|6507.36|356.25|51.40|1756.80|5089.40|5445.65|6846.20|7202.45|2444.12| +2451775|64793|2451878|1997|39801|1802774|594|40814|51919|1896666|3427|20047|56|14|17|1|88|563|95|82.36|92.24|77.48|1402.20|7360.60|7824.20|8762.80|642.58|220.81|437.95|7139.79|7782.37|7577.74|8220.32|-684.41| +2451775|64793|2451846|10769|39801|1802774|594|40814|51919|1896666|3427|20047|41|7|7|5|13|563|95|99.16|165.59|56.30|10382.55|5348.50|9420.20|15731.05|89.85|4225.31|471.20|1123.19|1213.04|1594.39|1684.24|-8297.01| +2451775|64793|2451876|17713|39801|1802774|594|40814|51919|1896666|3427|20047|11|2|15|1|244|563|92|65.01|155.37|97.88|5289.08|9004.96|5980.92|14294.04|216.11|5402.97|4144.60|3601.99|3818.10|7746.59|7962.70|-2378.93| +2451775|64793|2451860|11491|39801|1802774|594|40814|51919|1896666|3427|20047|38|13|11|3|252|563|89|65.93|190.53|114.31|6783.58|10173.59|5867.77|16957.17|164.81|7426.72|5425.44|2746.87|2911.68|8172.31|8337.12|-3120.90| +2451775|64793|2451884|11468|39801|1802774|594|40814|51919|1896666|3427|20047|19|26|1|5|132|563|65|90.65|261.07|57.43|13236.60|3732.95|5892.25|16969.55|261.30|0.00|848.25|3732.95|3994.25|4581.20|4842.50|-2159.30| +2451045|43476|2451060|1930|82854|304847|5593|23586|90128|1822810|4318|5464|16|19|1|2|43|564|78|27.32|56.27|25.32|2414.10|1974.96|2130.96|4389.06|59.24|0.00|1360.32|1974.96|2034.20|3335.28|3394.52|-156.00| +2451045|43476|2451089|4802|82854|304847|5593|23586|90128|1822810|4318|5464|49|4|1|1|274|564|87|46.86|100.28|1.00|8637.36|87.00|4076.82|8724.36|2.27|61.77|1220.61|25.23|27.50|1245.84|1248.11|-4051.59| +2451045|43476|2451090|4045|82854|304847|5593|23586|90128|1822810|4318|5464|4|16|8|4|283|564|58|5.91|9.21|6.44|160.66|373.52|342.78|534.18|12.55|164.34|170.52|209.18|221.73|379.70|392.25|-133.60| +2451045|43476|2451158|5206|82854|304847|5593|23586|90128|1822810|4318|5464|38|7|8|2|226|564|21|63.50|151.13|107.30|920.43|2253.30|1333.50|3173.73|27.94|1554.77|412.44|698.53|726.47|1110.97|1138.91|-634.97| +2451045|43476|2451061|15607|82854|304847|5593|23586|90128|1822810|4318|5464|13|25|16|2|8|564|79|68.99|181.44|99.79|6450.35|7883.41|5450.21|14333.76|236.50|0.00|572.75|7883.41|8119.91|8456.16|8692.66|2433.20| +2451045|43476|2451116|14320|82854|304847|5593|23586|90128|1822810|4318|5464|4|2|4|3|22|564|4|70.52|138.21|33.17|420.16|132.68|282.08|552.84|10.61|0.00|38.68|132.68|143.29|171.36|181.97|-149.40| +2451045|43476|2451051|8680|82854|304847|5593|23586|90128|1822810|4318|5464|40|7|4|2|297|564|11|61.42|84.14|65.62|203.72|721.82|675.62|925.54|57.74|0.00|351.67|721.82|779.56|1073.49|1131.23|46.20| +2451045|43476|2451101|7402|82854|304847|5593|23586|90128|1822810|4318|5464|37|7|9|4|199|564|23|52.96|121.27|61.84|1366.89|1422.32|1218.08|2789.21|71.11|0.00|920.23|1422.32|1493.43|2342.55|2413.66|204.24| +2451045|43476|2451106|3706|82854|304847|5593|23586|90128|1822810|4318|5464|26|25|15|4|63|564|4|96.19|170.25|139.60|122.60|558.40|384.76|681.00|33.50|0.00|224.72|558.40|591.90|783.12|816.62|173.64| +2452061|37460|2452095|1757|45156|1351439|6522|19205|2356|287586|454|17937|57|9|4|5|69|565|42|41.56|118.03|57.83|2528.40|2428.86|1745.52|4957.26|48.57|0.00|1139.88|2428.86|2477.43|3568.74|3617.31|683.34| +2452061|37460|2452114|12843|45156|1351439|6522|19205|2356|287586|454|17937|41|23|13|1|262|565|86|45.24|104.50|5.22|8538.08|448.92|3890.64|8987.00|40.40|0.00|2696.10|448.92|489.32|3145.02|3185.42|-3441.72| +2452061|37460|2452098|10865|45156|1351439|6522|19205|2356|287586|454|17937|13|25|9|5|181|565|97|53.85|61.38|5.52|5418.42|535.44|5223.45|5953.86|5.35|0.00|1309.50|535.44|540.79|1844.94|1850.29|-4688.01| +2452061|37460|2452134|16763|45156|1351439|6522|19205|2356|287586|454|17937|39|27|8|4|153|565|59|36.84|67.04|56.98|593.54|3361.82|2173.56|3955.36|201.70|0.00|434.83|3361.82|3563.52|3796.65|3998.35|1188.26| +2452061|37460|2452090|7417|45156|1351439|6522|19205|2356|287586|454|17937|13|27|8|5|282|565|72|56.70|102.62|83.12|1404.00|5984.64|4082.40|7388.64|359.07|0.00|2733.12|5984.64|6343.71|8717.76|9076.83|1902.24| +2452061|37460|2452134|7189|45156|1351439|6522|19205|2356|287586|454|17937|45|5|13|3|213|565|86|79.51|107.33|15.02|7938.66|1291.72|6837.86|9230.38|90.42|0.00|1106.82|1291.72|1382.14|2398.54|2488.96|-5546.14| +2452061|37460|2452129|7823|45156|1351439|6522|19205|2356|287586|454|17937|49|27|3|2|101|565|19|38.55|71.31|25.67|867.16|487.73|732.45|1354.89|9.36|175.58|311.60|312.15|321.51|623.75|633.11|-420.30| +2452061|37460|2452173|16657|45156|1351439|6522|19205|2356|287586|454|17937|47|9|13|1|52|565|19|37.57|42.07|1.26|775.39|23.94|713.83|799.33|0.00|0.00|143.83|23.94|23.94|167.77|167.77|-689.89| +2452061|37460|2452067|2801|45156|1351439|6522|19205|2356|287586|454|17937|7|9|9|3|294|565|51|41.71|95.51|54.44|2094.57|2776.44|2127.21|4871.01|166.58|0.00|1071.51|2776.44|2943.02|3847.95|4014.53|649.23| +2452061|37460|2452166|941|45156|1351439|6522|19205|2356|287586|454|17937|7|3|20|1|199|565|62|54.06|119.47|91.99|1703.76|5703.38|3351.72|7407.14|171.10|0.00|2592.22|5703.38|5874.48|8295.60|8466.70|2351.66| +2452061|37460|2452071|8331|45156|1351439|6522|19205|2356|287586|454|17937|43|7|3|2|114|565|87|80.13|215.54|56.04|13876.50|4875.48|6971.31|18751.98|0.00|3315.32|9375.99|1560.16|1560.16|10936.15|10936.15|-5411.15| +2452061|37460|2452063|15657|45156|1351439|6522|19205|2356|287586|454|17937|41|7|17|3|6|565|91|89.72|227.88|205.09|2073.89|18663.19|8164.52|20737.08|1679.68|0.00|1451.45|18663.19|20342.87|20114.64|21794.32|10498.67| +2452209|76436|2452236|3609|83735|1477939|3343|19147|76928|1346684|5759|16363|21|27|17|5|210|566|50|58.12|122.05|26.85|4760.00|1342.50|2906.00|6102.50|107.40|0.00|2746.00|1342.50|1449.90|4088.50|4195.90|-1563.50| +2452209|76436|2452245|1087|83735|1477939|3343|19147|76928|1346684|5759|16363|3|11|3|2|11|566|2|56.53|61.61|3.08|117.06|6.16|113.06|123.22|0.43|0.00|9.84|6.16|6.59|16.00|16.43|-106.90| +2452209|76436|2452305|16811|83735|1477939|3343|19147|76928|1346684|5759|16363|21|27|14|2|157|566|40|13.61|14.69|9.69|200.00|387.60|544.40|587.60|15.50|0.00|211.20|387.60|403.10|598.80|614.30|-156.80| +2452209|76436|2452212|15339|83735|1477939|3343|19147|76928|1346684|5759|16363|13|7|4|4|274|566|74|26.83|27.09|1.62|1884.78|119.88|1985.42|2004.66|3.95|40.75|219.78|79.13|83.08|298.91|302.86|-1906.29| +2452209|76436|2452280|2879|83735|1477939|3343|19147|76928|1346684|5759|16363|45|9|4|1|295|566|93|53.60|70.21|59.67|980.22|5549.31|4984.80|6529.53|388.45|0.00|456.63|5549.31|5937.76|6005.94|6394.39|564.51| +2452209|76436|2452272|4441|83735|1477939|3343|19147|76928|1346684|5759|16363|15|15|5|5|292|566|50|76.49|153.74|52.27|5073.50|2613.50|3824.50|7687.00|156.81|0.00|2844.00|2613.50|2770.31|5457.50|5614.31|-1211.00| +2452209|76436|2452293|10953|83735|1477939|3343|19147|76928|1346684|5759|16363|35|19|2|4|53|566|91|65.40|109.21|28.39|7354.62|2583.49|5951.40|9938.11|129.17|0.00|2981.16|2583.49|2712.66|5564.65|5693.82|-3367.91| +2452209|76436|2452308|6603|83735|1477939|3343|19147|76928|1346684|5759|16363|25|13|4|1|118|566|49|29.06|35.16|18.28|827.12|895.72|1423.94|1722.84|53.74|0.00|706.09|895.72|949.46|1601.81|1655.55|-528.22| +2452209|76436|2452239|9671|83735|1477939|3343|19147|76928|1346684|5759|16363|43|29|11|4|280|566|46|78.43|103.52|71.42|1476.60|3285.32|3607.78|4761.92|32.85|0.00|666.54|3285.32|3318.17|3951.86|3984.71|-322.46| +2452209|76436|2452279|11533|83735|1477939|3343|19147|76928|1346684|5759|16363|29|17|20|4|208|566|17|49.39|66.67|4.00|1065.39|68.00|839.63|1133.39|5.44|0.00|306.00|68.00|73.44|374.00|379.44|-771.63| +2452209|76436|2452265|2635|83735|1477939|3343|19147|76928|1346684|5759|16363|17|21|13|5|77|566|4|58.04|89.96|60.27|118.76|241.08|232.16|359.84|2.41|0.00|133.12|241.08|243.49|374.20|376.61|8.92| +2452209|76436|2452224|689|83735|1477939|3343|19147|76928|1346684|5759|16363|15|21|4|2|253|566|20|17.03|32.86|9.52|466.80|190.40|340.60|657.20|11.08|5.71|85.40|184.69|195.77|270.09|281.17|-155.91| +2451866|63624|2451922|3539|31321|1427138|4202|4819|33047|1177654|7117|24934|8|8|15|3|118|567|83|99.02|99.02|58.42|3369.80|4848.86|8218.66|8218.66|96.97|0.00|739.53|4848.86|4945.83|5588.39|5685.36|-3369.80| +2451866|63624|2451977|13760|31321|1427138|4202|4819|33047|1177654|7117|24934|37|25|7|3|272|567|94|37.17|50.92|48.37|239.70|4546.78|3493.98|4786.48|136.40|0.00|334.64|4546.78|4683.18|4881.42|5017.82|1052.80| +2451866|63624|2451920|13766|31321|1427138|4202|4819|33047|1177654|7117|24934|19|25|9|2|195|567|87|23.80|48.31|23.18|2186.31|2016.66|2070.60|4202.97|120.99|0.00|546.36|2016.66|2137.65|2563.02|2684.01|-53.94| +2451866|63624|2451868|9649|31321|1427138|4202|4819|33047|1177654|7117|24934|32|17|8|5|59|567|22|85.80|109.82|41.73|1497.98|918.06|1887.60|2416.04|82.62|0.00|386.54|918.06|1000.68|1304.60|1387.22|-969.54| +2451866|63624|2451956|14618|31321|1427138|4202|4819|33047|1177654|7117|24934|53|23|20|3|126|567|24|42.36|57.60|33.40|580.80|801.60|1016.64|1382.40|18.59|336.67|69.12|464.93|483.52|534.05|552.64|-551.71| +2451866|63624|2451877|17869|31321|1427138|4202|4819|33047|1177654|7117|24934|41|25|15|2|289|567|89|84.47|244.96|146.97|8721.11|13080.33|7517.83|21801.44|392.40|0.00|7193.87|13080.33|13472.73|20274.20|20666.60|5562.50| +2451866|63624|2451912|7187|31321|1427138|4202|4819|33047|1177654|7117|24934|13|7|11|5|32|567|81|81.42|116.43|103.62|1037.61|8393.22|6595.02|9430.83|206.47|4951.99|1225.53|3441.23|3647.70|4666.76|4873.23|-3153.79| +2451866|63624|2451923|5774|31321|1427138|4202|4819|33047|1177654|7117|24934|50|13|1|2|93|567|16|27.44|62.01|40.92|337.44|654.72|439.04|992.16|7.66|569.60|257.92|85.12|92.78|343.04|350.70|-353.92| +2451866|63624|2451959|6782|31321|1427138|4202|4819|33047|1177654|7117|24934|23|5|17|3|179|567|38|81.23|95.85|92.01|145.92|3496.38|3086.74|3642.30|34.96|0.00|728.46|3496.38|3531.34|4224.84|4259.80|409.64| +2451866|63624|2451872|3559|31321|1427138|4202|4819|33047|1177654|7117|24934|50|19|14|3|142|567|86|23.13|62.68|18.17|3827.86|1562.62|1989.18|5390.48|0.00|0.00|1455.12|1562.62|1562.62|3017.74|3017.74|-426.56| +2451866|63624|2451963|2300|31321|1427138|4202|4819|33047|1177654|7117|24934|17|8|18|4|187|567|99|25.58|51.41|7.19|4377.78|711.81|2532.42|5089.59|35.59|0.00|254.43|711.81|747.40|966.24|1001.83|-1820.61| +2451866|63624|2451898|2915|31321|1427138|4202|4819|33047|1177654|7117|24934|43|11|20|4|78|567|10|22.36|51.42|48.33|30.90|483.30|223.60|514.20|9.66|0.00|113.10|483.30|492.96|596.40|606.06|259.70| +2451866|63624|2451910|4154|31321|1427138|4202|4819|33047|1177654|7117|24934|2|19|8|3|101|567|55|43.14|114.75|10.32|5743.65|567.60|2372.70|6311.25|45.40|0.00|441.65|567.60|613.00|1009.25|1054.65|-1805.10| +2451866|63624|2451958|5137|31321|1427138|4202|4819|33047|1177654|7117|24934|11|2|18|4|170|567|18|48.88|89.45|15.20|1336.50|273.60|879.84|1610.10|3.00|123.12|418.50|150.48|153.48|568.98|571.98|-729.36| +2451866|63624|2451982|9029|31321|1427138|4202|4819|33047|1177654|7117|24934|53|7|1|5|122|567|28|23.76|28.03|2.52|714.28|70.56|665.28|784.84|4.93|0.00|0.00|70.56|75.49|70.56|75.49|-594.72| +2451866|63624|2451894|5659|31321|1427138|4202|4819|33047|1177654|7117|24934|5|1|2|5|131|567|37|40.14|83.49|25.88|2131.57|957.56|1485.18|3089.13|9.57|0.00|123.21|957.56|967.13|1080.77|1090.34|-527.62| +2451486|75463|2451516|11050|58282|321697|5112|22480|80743|1759016|4278|35646|40|4|15|4|106|568|94|9.98|13.47|5.38|760.46|505.72|938.12|1266.18|45.51|0.00|214.32|505.72|551.23|720.04|765.55|-432.40| +2451486|75463|2451489|17734|58282|321697|5112|22480|80743|1759016|4278|35646|43|8|18|4|277|568|67|68.05|100.71|52.36|3239.45|3508.12|4559.35|6747.57|280.64|0.00|2901.10|3508.12|3788.76|6409.22|6689.86|-1051.23| +2451486|75463|2451491|5995|58282|321697|5112|22480|80743|1759016|4278|35646|46|28|13|1|272|568|22|8.79|11.95|3.94|176.22|86.68|193.38|262.90|0.00|0.00|89.32|86.68|86.68|176.00|176.00|-106.70| +2451486|75463|2451526|5866|58282|321697|5112|22480|80743|1759016|4278|35646|7|26|14|4|14|568|54|75.28|111.41|33.42|4211.46|1804.68|4065.12|6016.14|90.23|0.00|240.30|1804.68|1894.91|2044.98|2135.21|-2260.44| +2451486|75463|2451492|8569|58282|321697|5112|22480|80743|1759016|4278|35646|43|7|16|3|160|568|16|31.06|62.74|26.35|582.24|421.60|496.96|1003.84|0.25|408.95|50.08|12.65|12.90|62.73|62.98|-484.31| +2451486|75463|2451551|17692|58282|321697|5112|22480|80743|1759016|4278|35646|56|16|19|4|106|568|67|98.02|216.62|82.31|8998.77|5514.77|6567.34|14513.54|154.41|2426.49|1160.44|3088.28|3242.69|4248.72|4403.13|-3479.06| +2451486|75463|2451602|11992|58282|321697|5112|22480|80743|1759016|4278|35646|8|8|15|3|208|568|36|34.37|44.68|41.55|112.68|1495.80|1237.32|1608.48|119.66|0.00|434.16|1495.80|1615.46|1929.96|2049.62|258.48| +2451486|75463|2451560|12584|58282|321697|5112|22480|80743|1759016|4278|35646|52|8|3|1|299|568|3|5.44|12.02|8.89|9.39|26.67|16.32|36.06|0.53|0.00|10.08|26.67|27.20|36.75|37.28|10.35| +2451486|75463|2451582|17810|58282|321697|5112|22480|80743|1759016|4278|35646|10|8|1|2|23|568|59|5.14|15.21|13.53|99.12|798.27|303.26|897.39|39.91|0.00|152.22|798.27|838.18|950.49|990.40|495.01| +2452162|57271|2452173|4927|16356|279488|4220|16552|16356|279488|4220|16552|57|25|14|3|92|569|33|22.64|39.16|37.98|38.94|1253.34|747.12|1292.28|25.06|0.00|503.91|1253.34|1278.40|1757.25|1782.31|506.22| +2452162|57271|2452282|1089|16356|279488|4220|16552|16356|279488|4220|16552|15|13|19|3|170|569|67|1.36|2.63|1.47|77.72|98.49|91.12|176.21|0.00|0.00|24.12|98.49|98.49|122.61|122.61|7.37| +2452162|57271|2452203|3035|16356|279488|4220|16552|16356|279488|4220|16552|11|9|11|5|286|569|59|93.53|204.83|83.98|7130.15|4954.82|5518.27|12084.97|346.83|0.00|1329.27|4954.82|5301.65|6284.09|6630.92|-563.45| +2452162|57271|2452270|17765|16356|279488|4220|16552|16356|279488|4220|16552|33|29|20|5|173|569|28|96.04|143.09|125.91|481.04|3525.48|2689.12|4006.52|70.50|0.00|240.24|3525.48|3595.98|3765.72|3836.22|836.36| +2452162|57271|2452195|14471|16356|279488|4220|16552|16356|279488|4220|16552|21|11|1|3|238|569|52|50.89|93.12|56.80|1888.64|2953.60|2646.28|4842.24|75.31|1447.26|386.88|1506.34|1581.65|1893.22|1968.53|-1139.94| +2452162|57271|2452230|14113|16356|279488|4220|16552|16356|279488|4220|16552|27|13|3|1|159|569|100|33.08|44.65|22.77|2188.00|2277.00|3308.00|4465.00|45.54|0.00|44.00|2277.00|2322.54|2321.00|2366.54|-1031.00| +2452162|57271|2452265|1177|16356|279488|4220|16552|16356|279488|4220|16552|41|15|18|5|255|569|86|28.63|64.99|10.39|4695.60|893.54|2462.18|5589.14|0.00|0.00|949.44|893.54|893.54|1842.98|1842.98|-1568.64| +2452162|57271|2452171|3309|16356|279488|4220|16552|16356|279488|4220|16552|3|15|7|3|139|569|3|66.96|89.72|22.43|201.87|67.29|200.88|269.16|0.00|24.89|37.68|42.40|42.40|80.08|80.08|-158.48| +2451795|41390|2451906|1838|9348|1018319|6490|11108|10448|528181|3268|2239|14|23|6|5|264|570|5|46.20|59.13|30.15|144.90|150.75|231.00|295.65|10.02|7.53|44.30|143.22|153.24|187.52|197.54|-87.78| +2451795|41390|2451909|4109|9348|1018319|6490|11108|10448|528181|3268|2239|35|1|4|4|137|570|30|53.50|59.92|53.32|198.00|1599.60|1605.00|1797.60|95.97|0.00|143.70|1599.60|1695.57|1743.30|1839.27|-5.40| +2451795|41390|2451854|16760|9348|1018319|6490|11108|10448|528181|3268|2239|32|20|18|2|219|570|84|44.52|74.79|68.80|503.16|5779.20|3739.68|6282.36|509.72|115.58|1067.64|5663.62|6173.34|6731.26|7240.98|1923.94| +2451795|41390|2451855|9781|9348|1018319|6490|11108|10448|528181|3268|2239|11|19|11|2|36|570|2|48.50|101.36|33.44|135.84|66.88|97.00|202.72|6.01|0.00|12.16|66.88|72.89|79.04|85.05|-30.12| +2451795|41390|2451805|6773|9348|1018319|6490|11108|10448|528181|3268|2239|31|20|13|5|197|570|54|15.70|31.24|19.05|658.26|1028.70|847.80|1686.96|10.28|0.00|421.74|1028.70|1038.98|1450.44|1460.72|180.90| +2451795|41390|2451818|13226|9348|1018319|6490|11108|10448|528181|3268|2239|31|17|9|4|219|570|88|7.75|21.08|18.97|185.68|1669.36|682.00|1855.04|133.54|0.00|184.80|1669.36|1802.90|1854.16|1987.70|987.36| +2451795|41390|2451855|13741|9348|1018319|6490|11108|10448|528181|3268|2239|1|8|9|5|234|570|37|85.21|175.53|145.68|1104.45|5390.16|3152.77|6494.61|53.90|0.00|1428.57|5390.16|5444.06|6818.73|6872.63|2237.39| +2451795|41390|2451882|6284|9348|1018319|6490|11108|10448|528181|3268|2239|55|26|7|3|279|570|16|31.21|68.03|23.13|718.40|370.08|499.36|1088.48|33.30|0.00|250.24|370.08|403.38|620.32|653.62|-129.28| +2451795|41390|2451851|13429|9348|1018319|6490|11108|10448|528181|3268|2239|41|29|2|1|283|570|9|63.97|177.19|138.20|350.91|1243.80|575.73|1594.71|87.06|0.00|542.16|1243.80|1330.86|1785.96|1873.02|668.07| +2451795|41390|2451817|1825|9348|1018319|6490|11108|10448|528181|3268|2239|56|14|17|1|215|570|33|22.48|57.32|6.30|1683.66|207.90|741.84|1891.56|14.55|0.00|359.37|207.90|222.45|567.27|581.82|-533.94| +2451795|41390|2451853|12953|9348|1018319|6490|11108|10448|528181|3268|2239|53|14|9|3|94|570|1|71.64|179.10|143.28|35.82|143.28|71.64|179.10|4.29|0.00|35.82|143.28|147.57|179.10|183.39|71.64| +2452277|40661|2452332|15483|14196|1527044|339|10434|65571|619193|1519|33614|9|18|3|2|21|571|59|56.64|129.13|116.21|762.28|6856.39|3341.76|7618.67|479.94|0.00|913.91|6856.39|7336.33|7770.30|8250.24|3514.63| +2452277|40661|2452317|14953|14196|1527044|339|10434|65571|619193|1519|33614|9|25|11|1|164|571|54|99.57|292.73|61.47|12488.04|3319.38|5376.78|15807.42|0.00|0.00|1106.46|3319.38|3319.38|4425.84|4425.84|-2057.40| +2452277|40661|2452313|3018|14196|1527044|339|10434|65571|619193|1519|33614|49|9|8|3|238|571|81|53.02|146.86|10.28|11062.98|832.68|4294.62|11895.66|8.32|0.00|4281.66|832.68|841.00|5114.34|5122.66|-3461.94| +2452277|40661|2452316|1701|14196|1527044|339|10434|65571|619193|1519|33614|12|9|14|3|122|571|84|67.35|105.73|49.69|4707.36|4173.96|5657.40|8881.32|0.00|0.00|2663.64|4173.96|4173.96|6837.60|6837.60|-1483.44| +2452277|40661|2452333|10377|14196|1527044|339|10434|65571|619193|1519|33614|21|18|14|2|47|571|50|29.37|40.53|24.31|811.00|1215.50|1468.50|2026.50|0.00|352.49|445.50|863.01|863.01|1308.51|1308.51|-605.49| +2452277|40661|2452285|13299|14196|1527044|339|10434|65571|619193|1519|33614|51|19|11|5|126|571|57|7.27|18.17|16.89|72.96|962.73|414.39|1035.69|67.39|0.00|0.00|962.73|1030.12|962.73|1030.12|548.34| +2452277|40661|2452368|6841|14196|1527044|339|10434|65571|619193|1519|33614|13|21|2|3|44|571|52|30.98|82.40|51.91|1585.48|2699.32|1610.96|4284.80|188.95|0.00|1842.36|2699.32|2888.27|4541.68|4730.63|1088.36| +2452277|40661|2452306|14251|14196|1527044|339|10434|65571|619193|1519|33614|33|7|1|5|212|571|11|78.09|108.54|13.02|1050.72|143.22|858.99|1193.94|0.00|0.00|107.36|143.22|143.22|250.58|250.58|-715.77| +2452060|38448|2452150|14101|54775|906117|5906|42848|96284|1197403|2921|49063|11|1|13|5|7|572|87|40.75|96.57|94.63|168.78|8232.81|3545.25|8401.59|411.64|0.00|923.94|8232.81|8644.45|9156.75|9568.39|4687.56| +2452060|38448|2452144|5539|54775|906117|5906|42848|96284|1197403|2921|49063|29|11|4|5|256|572|63|56.14|134.73|0.00|8487.99|0.00|3536.82|8487.99|0.00|0.00|1867.32|0.00|0.00|1867.32|1867.32|-3536.82| +2452060|38448|2452130|6953|54775|906117|5906|42848|96284|1197403|2921|49063|31|17|5|3|191|572|18|3.17|8.27|0.82|134.10|14.76|57.06|148.86|1.18|0.00|41.58|14.76|15.94|56.34|57.52|-42.30| +2452060|38448|2452139|17597|54775|906117|5906|42848|96284|1197403|2921|49063|3|25|4|5|46|572|23|9.51|17.78|1.60|372.14|36.80|218.73|408.94|0.36|0.00|85.79|36.80|37.16|122.59|122.95|-181.93| +2452060|38448|2452177|2885|54775|906117|5906|42848|96284|1197403|2921|49063|13|9|13|1|149|572|62|8.96|15.41|10.94|277.14|678.28|555.52|955.42|6.78|0.00|372.00|678.28|685.06|1050.28|1057.06|122.76| +2452060|38448|2452097|7471|54775|906117|5906|42848|96284|1197403|2921|49063|57|23|2|2|35|572|35|47.39|133.16|31.95|3542.35|1118.25|1658.65|4660.60|22.36|0.00|1771.00|1118.25|1140.61|2889.25|2911.61|-540.40| +2452060|38448|2452112|9497|54775|906117|5906|42848|96284|1197403|2921|49063|55|9|19|3|77|572|8|65.15|87.95|18.46|555.92|147.68|521.20|703.60|3.02|87.13|218.08|60.55|63.57|278.63|281.65|-460.65| +2452060|38448|2452130|6189|54775|906117|5906|42848|96284|1197403|2921|49063|33|27|8|5|282|572|16|3.71|10.23|10.12|1.76|161.92|59.36|163.68|1.61|0.00|16.32|161.92|163.53|178.24|179.85|102.56| +2452060|38448|2452124|8651|54775|906117|5906|42848|96284|1197403|2921|49063|3|5|7|1|142|572|95|67.23|194.96|128.67|6297.55|12223.65|6386.85|18521.20|977.89|0.00|8890.10|12223.65|13201.54|21113.75|22091.64|5836.80| +2452060|38448|2452098|6853|54775|906117|5906|42848|96284|1197403|2921|49063|25|13|8|2|153|572|89|83.37|225.93|189.78|3217.35|16890.42|7419.93|20107.77|59.11|16045.89|7037.23|844.53|903.64|7881.76|7940.87|-6575.40| +2452060|38448|2452134|13211|54775|906117|5906|42848|96284|1197403|2921|49063|53|15|19|5|221|572|45|22.14|60.88|49.92|493.20|2246.40|996.30|2739.60|22.46|0.00|492.75|2246.40|2268.86|2739.15|2761.61|1250.10| +2452060|38448|2452137|7245|54775|906117|5906|42848|96284|1197403|2921|49063|41|11|14|2|270|572|61|45.48|58.66|42.82|966.24|2612.02|2774.28|3578.26|0.00|0.00|1717.15|2612.02|2612.02|4329.17|4329.17|-162.26| +2452060|38448|2452085|841|54775|906117|5906|42848|96284|1197403|2921|49063|11|9|1|5|219|572|21|41.06|62.41|38.69|498.12|812.49|862.26|1310.61|56.87|0.00|537.18|812.49|869.36|1349.67|1406.54|-49.77| +2451902|47865|2451964|16070|21938|1366591|3691|19144|56921|1705018|2397|37850|11|29|7|4|51|573|100|35.85|106.83|32.04|7479.00|3204.00|3585.00|10683.00|64.08|0.00|2884.00|3204.00|3268.08|6088.00|6152.08|-381.00| +2451902|47865|2451916|10439|21938|1366591|3691|19144|56921|1705018|2397|37850|41|5|5|2|129|573|45|93.85|137.02|20.55|5241.15|924.75|4223.25|6165.90|9.24|0.00|678.15|924.75|933.99|1602.90|1612.14|-3298.50| +2451902|47865|2451936|14267|21938|1366591|3691|19144|56921|1705018|2397|37850|26|23|10|3|36|573|12|54.48|112.22|109.97|27.00|1319.64|653.76|1346.64|79.17|0.00|134.64|1319.64|1398.81|1454.28|1533.45|665.88| +2451902|47865|2451995|10196|21938|1366591|3691|19144|56921|1705018|2397|37850|53|14|2|1|57|573|9|75.48|89.82|69.16|185.94|622.44|679.32|808.38|24.89|0.00|16.11|622.44|647.33|638.55|663.44|-56.88| +2451902|47865|2451919|14990|21938|1366591|3691|19144|56921|1705018|2397|37850|2|19|11|4|170|573|89|95.82|126.48|97.38|2589.90|8666.82|8527.98|11256.72|433.34|0.00|4952.85|8666.82|9100.16|13619.67|14053.01|138.84| +2451902|47865|2451998|13274|21938|1366591|3691|19144|56921|1705018|2397|37850|7|20|11|3|48|573|35|24.99|47.48|18.51|1013.95|647.85|874.65|1661.80|38.87|0.00|514.85|647.85|686.72|1162.70|1201.57|-226.80| +2451902|47865|2452020|8408|21938|1366591|3691|19144|56921|1705018|2397|37850|50|11|9|2|262|573|73|33.12|69.88|18.86|3724.46|1376.78|2417.76|5101.24|123.91|0.00|662.84|1376.78|1500.69|2039.62|2163.53|-1040.98| +2451902|47865|2451938|1967|21938|1366591|3691|19144|56921|1705018|2397|37850|47|5|15|4|196|573|100|63.92|122.08|87.89|3419.00|8789.00|6392.00|12208.00|615.23|0.00|854.00|8789.00|9404.23|9643.00|10258.23|2397.00| +2451902|47865|2451962|15746|21938|1366591|3691|19144|56921|1705018|2397|37850|35|17|3|4|208|573|90|93.87|272.22|215.05|5145.30|19354.50|8448.30|24499.80|387.09|0.00|5879.70|19354.50|19741.59|25234.20|25621.29|10906.20| +2451902|47865|2452000|7730|21938|1366591|3691|19144|56921|1705018|2397|37850|7|26|4|2|277|573|47|95.14|175.05|8.75|7816.10|411.25|4471.58|8227.35|3.74|37.01|1892.22|374.24|377.98|2266.46|2270.20|-4097.34| +2451902|47865|2451934|9515|21938|1366591|3691|19144|56921|1705018|2397|37850|50|5|10|2|121|573|90|97.54|262.38|157.42|9446.40|14167.80|8778.60|23614.20|425.03|0.00|1416.60|14167.80|14592.83|15584.40|16009.43|5389.20| +2451902|47865|2452017|15109|21938|1366591|3691|19144|56921|1705018|2397|37850|25|29|7|1|58|573|54|60.61|85.46|17.09|3691.98|922.86|3272.94|4614.84|12.18|516.80|1291.68|406.06|418.24|1697.74|1709.92|-2866.88| +2451902|47865|2451916|11453|21938|1366591|3691|19144|56921|1705018|2397|37850|5|25|13|1|186|573|94|24.27|56.30|25.89|2858.54|2433.66|2281.38|5292.20|7.78|2044.27|370.36|389.39|397.17|759.75|767.53|-1891.99| +2452294|47385|2452384|17904|34978|809025|5472|7966|74407|125362|954|27401|48|25|12|3|167|574|60|32.78|55.72|53.49|133.80|3209.40|1966.80|3343.20|160.47|0.00|33.00|3209.40|3369.87|3242.40|3402.87|1242.60| +2452294|47385|2452374|10815|34978|809025|5472|7966|74407|125362|954|27401|13|3|14|4|204|574|29|16.33|45.88|27.52|532.44|798.08|473.57|1330.52|23.94|0.00|212.86|798.08|822.02|1010.94|1034.88|324.51| +2452294|47385|2452346|5409|34978|809025|5472|7966|74407|125362|954|27401|45|19|20|5|250|574|28|87.13|133.30|15.99|3284.68|447.72|2439.64|3732.40|4.47|0.00|522.48|447.72|452.19|970.20|974.67|-1991.92| +2452294|47385|2452378|3079|34978|809025|5472|7966|74407|125362|954|27401|45|1|12|5|228|574|18|33.71|68.43|43.79|443.52|788.22|606.78|1231.74|0.00|0.00|406.44|788.22|788.22|1194.66|1194.66|181.44| +2452294|47385|2452345|13656|34978|809025|5472|7966|74407|125362|954|27401|3|19|8|3|131|574|35|63.59|187.59|170.70|591.15|5974.50|2225.65|6565.65|418.21|0.00|1378.65|5974.50|6392.71|7353.15|7771.36|3748.85| +2452294|47385|2452363|17245|34978|809025|5472|7966|74407|125362|954|27401|27|30|6|5|221|574|42|41.65|88.29|74.16|593.46|3114.72|1749.30|3708.18|260.70|218.03|333.48|2896.69|3157.39|3230.17|3490.87|1147.39| +2452294|47385|2452365|17550|34978|809025|5472|7966|74407|125362|954|27401|43|6|4|5|253|574|10|9.94|12.82|11.92|9.00|119.20|99.40|128.20|0.00|0.00|35.80|119.20|119.20|155.00|155.00|19.80| +2452294|47385|2452353|2955|34978|809025|5472|7966|74407|125362|954|27401|60|7|8|4|293|574|61|36.59|60.37|30.78|1804.99|1877.58|2231.99|3682.57|93.87|0.00|1104.71|1877.58|1971.45|2982.29|3076.16|-354.41| +2452294|47385|2452297|9441|34978|809025|5472|7966|74407|125362|954|27401|57|30|1|2|213|574|32|43.17|120.87|99.11|696.32|3171.52|1381.44|3867.84|285.43|0.00|1121.60|3171.52|3456.95|4293.12|4578.55|1790.08| +2452294|47385|2452312|2874|34978|809025|5472|7966|74407|125362|954|27401|15|7|10|1|119|574|37|18.09|43.59|0.43|1596.92|15.91|669.33|1612.83|0.00|0.00|290.08|15.91|15.91|305.99|305.99|-653.42| +2452594|45716|2452630|3036|34778|1511530|3929|47881|21494|614814|2584|4136|25|1|6|4|92|575|69|37.09|105.70|47.56|4011.66|3281.64|2559.21|7293.30|229.71|0.00|2917.32|3281.64|3511.35|6198.96|6428.67|722.43| +2452594|45716|2452625|17766|34778|1511530|3929|47881|21494|614814|2584|4136|45|25|3|1|246|575|17|86.11|250.58|120.27|2215.27|2044.59|1463.87|4259.86|0.00|0.00|383.35|2044.59|2044.59|2427.94|2427.94|580.72| +2452594|45716|2452639|14472|34778|1511530|3929|47881|21494|614814|2584|4136|37|18|19|4|286|575|73|86.45|173.76|166.80|508.08|12176.40|6310.85|12684.48|0.00|12054.63|0.00|121.77|121.77|121.77|121.77|-6189.08| +2452594|45716|2452665|14113|34778|1511530|3929|47881|21494|614814|2584|4136|7|1|1|3|45|575|74|69.71|129.66|38.89|6716.98|2877.86|5158.54|9594.84|212.38|518.01|3070.26|2359.85|2572.23|5430.11|5642.49|-2798.69| +2452594|45716|2452628|1177|34778|1511530|3929|47881|21494|614814|2584|4136|60|13|17|4|231|575|98|87.73|229.85|66.65|15993.60|6531.70|8597.54|22525.30|261.26|0.00|449.82|6531.70|6792.96|6981.52|7242.78|-2065.84| +2452594|45716|2452607|3309|34778|1511530|3929|47881|21494|614814|2584|4136|6|13|18|4|10|575|61|72.90|94.77|75.81|1156.56|4624.41|4446.90|5780.97|0.00|2867.13|404.43|1757.28|1757.28|2161.71|2161.71|-2689.62| +2452594|45716|2452621|9301|34778|1511530|3929|47881|21494|614814|2584|4136|7|1|18|5|232|575|82|68.21|194.39|130.24|5260.30|10679.68|5593.22|15939.98|533.98|0.00|2709.28|10679.68|11213.66|13388.96|13922.94|5086.46| +2452594|45716|2452609|15678|34778|1511530|3929|47881|21494|614814|2584|4136|7|3|3|1|49|575|15|55.34|63.64|29.27|515.55|439.05|830.10|954.60|17.56|0.00|152.70|439.05|456.61|591.75|609.31|-391.05| +2451404|82351|2451504|9499|50898|496187|5476|47612|50898|496187|5476|47612|25|20|9|2|275|576|15|10.06|25.15|8.29|252.90|124.35|150.90|377.25|3.73|0.00|86.70|124.35|128.08|211.05|214.78|-26.55| +2451404|82351|2451473|12812|50898|496187|5476|47612|50898|496187|5476|47612|44|8|7|1|288|576|32|92.15|94.91|73.08|698.56|2338.56|2948.80|3037.12|23.38|0.00|1184.32|2338.56|2361.94|3522.88|3546.26|-610.24| +2451404|82351|2451410|15850|50898|496187|5476|47612|50898|496187|5476|47612|10|16|2|5|78|576|51|95.91|219.63|151.54|3472.59|7728.54|4891.41|11201.13|231.85|0.00|4480.35|7728.54|7960.39|12208.89|12440.74|2837.13| +2451404|82351|2451492|13183|50898|496187|5476|47612|50898|496187|5476|47612|10|13|15|1|144|576|41|59.57|111.99|54.87|2341.92|2249.67|2442.37|4591.59|44.99|0.00|780.23|2249.67|2294.66|3029.90|3074.89|-192.70| +2451404|82351|2451439|9577|50898|496187|5476|47612|50898|496187|5476|47612|1|8|9|1|160|576|85|35.88|57.76|46.78|933.30|3976.30|3049.80|4909.60|198.81|0.00|294.10|3976.30|4175.11|4270.40|4469.21|926.50| +2451404|82351|2451436|13780|50898|496187|5476|47612|50898|496187|5476|47612|40|16|4|4|107|576|84|25.61|59.92|56.92|252.00|4781.28|2151.24|5033.28|96.10|1577.82|2214.24|3203.46|3299.56|5417.70|5513.80|1052.22| +2451404|82351|2451406|12643|50898|496187|5476|47612|50898|496187|5476|47612|26|26|11|1|118|576|69|15.65|28.32|4.24|1661.52|292.56|1079.85|1954.08|0.00|0.00|409.86|292.56|292.56|702.42|702.42|-787.29| +2451404|82351|2451450|697|50898|496187|5476|47612|50898|496187|5476|47612|46|25|9|3|234|576|33|26.68|35.48|4.25|1030.59|140.25|880.44|1170.84|12.62|0.00|187.11|140.25|152.87|327.36|339.98|-740.19| +2451404|82351|2451490|8258|50898|496187|5476|47612|50898|496187|5476|47612|34|26|3|4|205|576|4|38.46|43.07|4.73|153.36|18.92|153.84|172.28|0.18|0.00|37.88|18.92|19.10|56.80|56.98|-134.92| +2451404|82351|2451440|13648|50898|496187|5476|47612|50898|496187|5476|47612|37|8|18|1|224|576|71|48.97|97.45|87.70|692.25|6226.70|3476.87|6918.95|311.33|0.00|2144.20|6226.70|6538.03|8370.90|8682.23|2749.83| +2451404|82351|2451477|10774|50898|496187|5476|47612|50898|496187|5476|47612|55|22|19|3|220|576|57|50.38|146.10|99.34|2665.32|5662.38|2871.66|8327.70|226.49|0.00|748.98|5662.38|5888.87|6411.36|6637.85|2790.72| +2450888|27709|2450976|16975|32130|740446|2362|1878|22783|368997|3273|45189|10|13|15|1|264|577|46|2.59|7.12|3.56|163.76|163.76|119.14|327.52|14.73|0.00|140.76|163.76|178.49|304.52|319.25|44.62| +2450888|27709|2450990|16448|32130|740446|2362|1878|22783|368997|3273|45189|22|16|6|3|90|577|93|87.13|236.12|217.23|1756.77|20202.39|8103.09|21959.16|1616.19|0.00|1316.88|20202.39|21818.58|21519.27|23135.46|12099.30| +2450888|27709|2450960|10267|32130|740446|2362|1878|22783|368997|3273|45189|46|28|13|4|272|577|61|45.91|100.08|32.02|4151.66|1953.22|2800.51|6104.88|97.66|0.00|1953.22|1953.22|2050.88|3906.44|4004.10|-847.29| +2450888|27709|2451002|9530|32130|740446|2362|1878|22783|368997|3273|45189|19|13|4|4|111|577|18|85.98|102.31|55.24|847.26|994.32|1547.64|1841.58|39.77|0.00|36.72|994.32|1034.09|1031.04|1070.81|-553.32| +2450888|27709|2450902|7102|32130|740446|2362|1878|22783|368997|3273|45189|58|22|4|1|168|577|32|26.21|65.00|39.00|832.00|1248.00|838.72|2080.00|0.00|0.00|20.80|1248.00|1248.00|1268.80|1268.80|409.28| +2450888|27709|2450991|4423|32130|740446|2362|1878|22783|368997|3273|45189|52|10|5|4|40|577|44|79.59|81.97|31.96|2200.44|1406.24|3501.96|3606.68|28.12|0.00|1298.00|1406.24|1434.36|2704.24|2732.36|-2095.72| +2450888|27709|2450976|13003|32130|740446|2362|1878|22783|368997|3273|45189|10|13|15|1|114|577|76|31.80|62.96|57.29|430.92|4354.04|2416.80|4784.96|261.24|0.00|2105.20|4354.04|4615.28|6459.24|6720.48|1937.24| +2450888|27709|2450893|12590|32130|740446|2362|1878|22783|368997|3273|45189|2|25|4|5|166|577|56|88.15|104.01|60.32|2446.64|3377.92|4936.40|5824.56|236.45|0.00|524.16|3377.92|3614.37|3902.08|4138.53|-1558.48| +2450888|27709|2450986|6454|32130|740446|2362|1878|22783|368997|3273|45189|2|22|12|2|42|577|59|5.54|10.74|4.40|374.06|259.60|326.86|633.66|12.98|0.00|195.88|259.60|272.58|455.48|468.46|-67.26| +2452561|74844|2452659|4773|28870|151588|4259|44577|55509|670743|2684|43402|12|7|5|3|194|578|85|98.94|102.89|38.06|5510.55|3235.10|8409.90|8745.65|152.69|1326.39|3497.75|1908.71|2061.40|5406.46|5559.15|-6501.19| +2452561|74844|2452661|6543|28870|151588|4259|44577|55509|670743|2684|43402|30|30|2|2|299|578|2|63.99|181.09|25.35|311.48|50.70|127.98|362.18|1.52|0.00|155.72|50.70|52.22|206.42|207.94|-77.28| +2452561|74844|2452581|2961|28870|151588|4259|44577|55509|670743|2684|43402|19|30|20|5|60|578|73|55.02|100.68|80.54|1470.22|5879.42|4016.46|7349.64|58.79|0.00|1028.57|5879.42|5938.21|6907.99|6966.78|1862.96| +2452561|74844|2452667|10845|28870|151588|4259|44577|55509|670743|2684|43402|55|13|3|4|48|578|100|39.30|96.67|18.36|7831.00|1836.00|3930.00|9667.00|165.24|0.00|290.00|1836.00|2001.24|2126.00|2291.24|-2094.00| +2452561|74844|2452626|7057|28870|151588|4259|44577|55509|670743|2684|43402|25|27|12|3|264|578|18|31.11|66.88|66.21|12.06|1191.78|559.98|1203.84|83.42|0.00|288.90|1191.78|1275.20|1480.68|1564.10|631.80| +2452561|74844|2452672|11473|28870|151588|4259|44577|55509|670743|2684|43402|45|13|8|2|3|578|68|39.03|71.81|63.19|586.16|4296.92|2654.04|4883.08|128.90|0.00|1708.84|4296.92|4425.82|6005.76|6134.66|1642.88| +2452561|74844|2452563|11568|28870|151588|4259|44577|55509|670743|2684|43402|33|30|20|3|83|578|47|8.24|22.33|3.57|881.72|167.79|387.28|1049.51|8.38|0.00|493.03|167.79|176.17|660.82|669.20|-219.49| +2452561|74844|2452593|16854|28870|151588|4259|44577|55509|670743|2684|43402|15|25|18|1|111|578|91|68.41|107.40|45.10|5669.30|4104.10|6225.31|9773.40|110.81|1887.88|2932.02|2216.22|2327.03|5148.24|5259.05|-4009.09| +2452561|74844|2452570|17562|28870|151588|4259|44577|55509|670743|2684|43402|48|13|20|5|203|578|97|51.65|105.88|48.70|5546.46|4723.90|5010.05|10270.36|425.15|0.00|3491.03|4723.90|5149.05|8214.93|8640.08|-286.15| +2452561|74844|2452566|13818|28870|151588|4259|44577|55509|670743|2684|43402|42|24|3|2|128|578|71|3.45|8.86|5.58|232.88|396.18|244.95|629.06|3.96|0.00|282.58|396.18|400.14|678.76|682.72|151.23| +2452041|68423|2452074|9575|51886|1272716|3593|6423|64232|791488|160|34590|43|3|12|2|25|579|76|93.15|271.06|13.55|19570.76|1029.80|7079.40|20600.56|30.89|0.00|8445.88|1029.80|1060.69|9475.68|9506.57|-6049.60| +2452041|68423|2452109|7595|51886|1272716|3593|6423|64232|791488|160|34590|19|19|13|3|116|579|86|7.26|10.45|0.52|853.98|44.72|624.36|898.70|3.57|0.00|394.74|44.72|48.29|439.46|443.03|-579.64| +2452041|68423|2452090|2115|51886|1272716|3593|6423|64232|791488|160|34590|17|21|13|5|197|579|83|82.25|199.86|143.89|4645.51|11942.87|6826.75|16588.38|955.42|0.00|6137.02|11942.87|12898.29|18079.89|19035.31|5116.12| +2452041|68423|2452136|15259|51886|1272716|3593|6423|64232|791488|160|34590|59|3|1|3|172|579|5|81.58|203.95|12.23|958.60|61.15|407.90|1019.75|3.66|0.00|367.10|61.15|64.81|428.25|431.91|-346.75| +2452041|68423|2452135|13971|51886|1272716|3593|6423|64232|791488|160|34590|37|5|20|4|91|579|75|41.20|106.29|44.64|4623.75|3348.00|3090.00|7971.75|123.54|1975.32|2311.50|1372.68|1496.22|3684.18|3807.72|-1717.32| +2452041|68423|2452073|3703|51886|1272716|3593|6423|64232|791488|160|34590|17|11|10|2|34|579|91|56.31|116.56|111.89|424.97|10181.99|5124.21|10606.96|407.27|0.00|2969.33|10181.99|10589.26|13151.32|13558.59|5057.78| +2452041|68423|2452082|2109|51886|1272716|3593|6423|64232|791488|160|34590|25|25|13|1|66|579|6|24.02|45.63|31.94|82.14|191.64|144.12|273.78|9.58|0.00|35.58|191.64|201.22|227.22|236.80|47.52| +2452041|68423|2452092|17473|51886|1272716|3593|6423|64232|791488|160|34590|7|3|17|2|47|579|18|68.63|98.82|2.96|1725.48|53.28|1235.34|1778.76|1.59|0.00|213.30|53.28|54.87|266.58|268.17|-1182.06| +2452041|68423|2452078|851|51886|1272716|3593|6423|64232|791488|160|34590|5|19|20|2|294|579|66|70.55|126.28|121.22|333.96|8000.52|4656.30|8334.48|560.03|0.00|499.62|8000.52|8560.55|8500.14|9060.17|3344.22| +2452041|68423|2452122|2717|51886|1272716|3593|6423|64232|791488|160|34590|1|3|13|4|185|579|77|45.77|125.86|88.10|2907.52|6783.70|3524.29|9691.22|339.18|0.00|3100.79|6783.70|7122.88|9884.49|10223.67|3259.41| +2452041|68423|2452080|8535|51886|1272716|3593|6423|64232|791488|160|34590|43|23|12|2|185|579|83|89.67|145.26|81.34|5305.36|6751.22|7442.61|12056.58|137.72|3308.09|964.46|3443.13|3580.85|4407.59|4545.31|-3999.48| +2452041|68423|2452070|9669|51886|1272716|3593|6423|64232|791488|160|34590|31|15|20|4|81|579|72|41.57|71.08|47.62|1689.12|3428.64|2993.04|5117.76|308.57|0.00|2251.44|3428.64|3737.21|5680.08|5988.65|435.60| +2452041|68423|2452112|16115|51886|1272716|3593|6423|64232|791488|160|34590|11|11|9|4|112|579|44|36.19|38.72|3.87|1533.40|170.28|1592.36|1703.68|13.62|0.00|272.36|170.28|183.90|442.64|456.26|-1422.08| +2452041|68423|2452087|8927|51886|1272716|3593|6423|64232|791488|160|34590|27|21|18|2|159|579|8|48.08|58.65|4.10|436.40|32.80|384.64|469.20|0.98|0.00|84.40|32.80|33.78|117.20|118.18|-351.84| +2451852|36494|2451928|10123|57894|1391584|6874|36964|98874|1913048|2434|25724|50|29|4|3|282|580|5|62.16|134.88|72.83|310.25|364.15|310.80|674.40|32.77|0.00|175.30|364.15|396.92|539.45|572.22|53.35| +2451852|36494|2451893|1715|57894|1391584|6874|36964|98874|1913048|2434|25724|23|20|13|5|234|580|39|82.76|127.45|85.39|1640.34|3330.21|3227.64|4970.55|99.90|0.00|1192.62|3330.21|3430.11|4522.83|4622.73|102.57| +2451852|36494|2451935|1289|57894|1391584|6874|36964|98874|1913048|2434|25724|35|2|13|2|28|580|59|97.38|123.67|66.78|3356.51|3940.02|5745.42|7296.53|16.54|3388.41|145.73|551.61|568.15|697.34|713.88|-5193.81| +2451852|36494|2451893|16784|57894|1391584|6874|36964|98874|1913048|2434|25724|56|8|4|5|219|580|93|45.11|49.16|31.95|1600.53|2971.35|4195.23|4571.88|178.28|0.00|2193.87|2971.35|3149.63|5165.22|5343.50|-1223.88| +2451852|36494|2451876|434|57894|1391584|6874|36964|98874|1913048|2434|25724|8|2|12|2|219|580|3|2.52|3.37|2.83|1.62|8.49|7.56|10.11|0.42|0.00|4.95|8.49|8.91|13.44|13.86|0.93| +2451852|36494|2451889|3895|57894|1391584|6874|36964|98874|1913048|2434|25724|44|1|16|1|249|580|40|24.39|46.09|44.24|74.00|1769.60|975.60|1843.60|106.17|0.00|810.80|1769.60|1875.77|2580.40|2686.57|794.00| +2451852|36494|2451960|476|57894|1391584|6874|36964|98874|1913048|2434|25724|11|20|16|1|273|580|11|68.28|96.27|55.83|444.84|614.13|751.08|1058.97|0.00|0.00|370.59|614.13|614.13|984.72|984.72|-136.95| +2451852|36494|2451944|12607|57894|1391584|6874|36964|98874|1913048|2434|25724|5|1|9|2|205|580|6|9.79|25.94|14.52|68.52|87.12|58.74|155.64|2.61|0.00|12.42|87.12|89.73|99.54|102.15|28.38| +2452311|35381|2452416|8748|68507|421223|6739|25055|72098|807700|3556|33472|36|19|2|4|196|581|98|27.65|64.70|5.82|5770.24|570.36|2709.70|6340.60|5.70|0.00|824.18|570.36|576.06|1394.54|1400.24|-2139.34| +2452311|35381|2452415|15279|68507|421223|6739|25055|72098|807700|3556|33472|43|3|19|4|279|581|67|53.99|57.22|4.57|3527.55|306.19|3617.33|3833.74|0.00|0.00|76.38|306.19|306.19|382.57|382.57|-3311.14| +2452311|35381|2452335|12765|68507|421223|6739|25055|72098|807700|3556|33472|60|9|3|3|283|581|5|43.97|96.73|66.74|149.95|333.70|219.85|483.65|13.34|0.00|120.90|333.70|347.04|454.60|467.94|113.85| +2452311|35381|2452347|1759|68507|421223|6739|25055|72098|807700|3556|33472|18|13|5|1|174|581|46|14.27|38.10|5.71|1489.94|262.66|656.42|1752.60|9.69|154.96|472.88|107.70|117.39|580.58|590.27|-548.72| +2452311|35381|2452376|11343|68507|421223|6739|25055|72098|807700|3556|33472|13|19|13|4|282|581|60|56.91|166.74|153.40|800.40|9204.00|3414.60|10004.40|0.00|0.00|3601.20|9204.00|9204.00|12805.20|12805.20|5789.40| +2452311|35381|2452355|6381|68507|421223|6739|25055|72098|807700|3556|33472|3|19|8|3|191|581|61|27.86|74.94|59.20|960.14|3611.20|1699.46|4571.34|93.16|505.56|365.39|3105.64|3198.80|3471.03|3564.19|1406.18| +2452311|35381|2452401|4377|68507|421223|6739|25055|72098|807700|3556|33472|55|25|18|3|215|581|29|23.73|63.12|48.60|421.08|1409.40|688.17|1830.48|0.00|0.00|896.68|1409.40|1409.40|2306.08|2306.08|721.23| +2452311|35381|2452344|17689|68507|421223|6739|25055|72098|807700|3556|33472|37|7|19|4|109|581|32|62.31|112.15|29.15|2656.00|932.80|1993.92|3588.80|65.29|0.00|358.72|932.80|998.09|1291.52|1356.81|-1061.12| +2452311|35381|2452381|2145|68507|421223|6739|25055|72098|807700|3556|33472|24|9|15|3|298|581|77|10.01|14.51|9.72|368.83|748.44|770.77|1117.27|22.45|0.00|502.04|748.44|770.89|1250.48|1272.93|-22.33| +2452311|35381|2452333|5529|68507|421223|6739|25055|72098|807700|3556|33472|7|13|7|5|90|581|7|72.83|102.69|42.10|424.13|294.70|509.81|718.83|11.78|0.00|287.49|294.70|306.48|582.19|593.97|-215.11| +2452311|35381|2452364|13065|68507|421223|6739|25055|72098|807700|3556|33472|36|24|19|3|280|581|10|40.84|92.29|92.29|0.00|922.90|408.40|922.90|9.22|0.00|138.40|922.90|932.12|1061.30|1070.52|514.50| +2452311|35381|2452392|12381|68507|421223|6739|25055|72098|807700|3556|33472|42|18|17|5|273|581|44|57.93|117.59|92.89|1086.80|4087.16|2548.92|5173.96|0.00|0.00|1707.20|4087.16|4087.16|5794.36|5794.36|1538.24| +2452311|35381|2452361|8167|68507|421223|6739|25055|72098|807700|3556|33472|1|12|11|1|21|581|81|20.69|32.27|21.94|836.73|1777.14|1675.89|2613.87|0.00|0.00|182.25|1777.14|1777.14|1959.39|1959.39|101.25| +2452311|35381|2452345|6303|68507|421223|6739|25055|72098|807700|3556|33472|30|24|2|2|289|581|72|21.51|26.24|8.92|1247.04|642.24|1548.72|1889.28|6.42|0.00|18.72|642.24|648.66|660.96|667.38|-906.48| +2451157|72222|2451260|2497|9335|1452461|1634|42360|83071|1702281|3032|25805|56|14|17|1|95|582|44|72.28|119.26|102.56|734.80|4512.64|3180.32|5247.44|103.79|2436.82|1469.16|2075.82|2179.61|3544.98|3648.77|-1104.50| +2451157|72222|2451277|9334|9335|1452461|1634|42360|83071|1702281|3032|25805|1|14|5|4|126|582|10|17.80|23.31|10.95|123.60|109.50|178.00|233.10|2.19|0.00|104.80|109.50|111.69|214.30|216.49|-68.50| +2451157|72222|2451269|14932|9335|1452461|1634|42360|83071|1702281|3032|25805|46|22|13|1|52|582|12|33.15|84.86|5.94|947.04|71.28|397.80|1018.32|0.00|61.30|346.20|9.98|9.98|356.18|356.18|-387.82| +2451157|72222|2451255|811|9335|1452461|1634|42360|83071|1702281|3032|25805|44|4|20|2|167|582|63|49.74|100.97|59.57|2608.20|3752.91|3133.62|6361.11|225.17|0.00|1653.75|3752.91|3978.08|5406.66|5631.83|619.29| +2451157|72222|2451206|14276|9335|1452461|1634|42360|83071|1702281|3032|25805|13|4|16|2|185|582|83|39.05|40.61|16.24|2022.71|1347.92|3241.15|3370.63|26.95|0.00|1381.95|1347.92|1374.87|2729.87|2756.82|-1893.23| +2451157|72222|2451181|1594|9335|1452461|1634|42360|83071|1702281|3032|25805|16|26|9|2|172|582|75|39.37|55.90|39.68|1216.50|2976.00|2952.75|4192.50|119.04|0.00|1131.75|2976.00|3095.04|4107.75|4226.79|23.25| +2451157|72222|2451211|9871|9335|1452461|1634|42360|83071|1702281|3032|25805|31|14|7|4|208|582|72|68.17|139.74|115.98|1710.72|8350.56|4908.24|10061.28|751.55|0.00|1005.84|8350.56|9102.11|9356.40|10107.95|3442.32| +2451157|72222|2451275|15388|9335|1452461|1634|42360|83071|1702281|3032|25805|38|7|11|5|122|582|89|54.31|155.86|26.49|11513.93|2357.61|4833.59|13871.54|14.61|895.89|2357.61|1461.72|1476.33|3819.33|3833.94|-3371.87| +2451157|72222|2451272|11728|9335|1452461|1634|42360|83071|1702281|3032|25805|20|2|6|2|88|582|19|29.73|79.97|56.77|440.80|1078.63|564.87|1519.43|0.00|0.00|197.41|1078.63|1078.63|1276.04|1276.04|513.76| +2451157|72222|2451230|6590|9335|1452461|1634|42360|83071|1702281|3032|25805|38|8|20|5|290|582|35|60.06|107.50|30.10|2709.00|1053.50|2102.10|3762.50|10.53|0.00|263.20|1053.50|1064.03|1316.70|1327.23|-1048.60| +2451157|72222|2451191|3070|9335|1452461|1634|42360|83071|1702281|3032|25805|14|25|8|3|298|582|38|73.34|151.08|120.86|1148.36|4592.68|2786.92|5741.04|367.41|0.00|1435.26|4592.68|4960.09|6027.94|6395.35|1805.76| +2451157|72222|2451183|5953|9335|1452461|1634|42360|83071|1702281|3032|25805|31|4|10|2|294|582|97|4.53|6.20|5.64|54.32|547.08|439.41|601.40|11.81|251.65|282.27|295.43|307.24|577.70|589.51|-143.98| +2451157|72222|2451206|17488|9335|1452461|1634|42360|83071|1702281|3032|25805|7|26|11|1|298|582|60|33.38|96.13|13.45|4960.80|807.00|2002.80|5767.80|24.21|0.00|1557.00|807.00|831.21|2364.00|2388.21|-1195.80| +2451157|72222|2451250|9386|9335|1452461|1634|42360|83071|1702281|3032|25805|14|4|11|1|274|582|70|79.41|88.93|45.35|3050.60|3174.50|5558.70|6225.10|95.23|0.00|2800.70|3174.50|3269.73|5975.20|6070.43|-2384.20| +2452153|60566|2452188|2537|86041|782160|6726|2892|36733|454299|4990|3556|25|27|2|3|264|583|73|45.41|94.90|43.65|3741.25|3186.45|3314.93|6927.70|159.32|0.00|554.07|3186.45|3345.77|3740.52|3899.84|-128.48| +2452153|60566|2452266|14447|86041|782160|6726|2892|36733|454299|4990|3556|33|23|7|2|297|583|10|81.93|120.43|63.82|566.10|638.20|819.30|1204.30|0.00|0.00|84.30|638.20|638.20|722.50|722.50|-181.10| +2452153|60566|2452188|7863|86041|782160|6726|2892|36733|454299|4990|3556|5|11|6|4|149|583|27|59.51|134.49|92.79|1125.90|2505.33|1606.77|3631.23|75.15|0.00|1016.55|2505.33|2580.48|3521.88|3597.03|898.56| +2452153|60566|2452217|6889|86041|782160|6726|2892|36733|454299|4990|3556|11|11|9|4|179|583|95|51.26|111.74|69.27|4034.65|6580.65|4869.70|10615.30|131.61|0.00|5094.85|6580.65|6712.26|11675.50|11807.11|1710.95| +2452153|60566|2452225|7397|86041|782160|6726|2892|36733|454299|4990|3556|35|21|7|5|133|583|96|48.63|65.65|64.33|126.72|6175.68|4668.48|6302.40|432.29|0.00|1575.36|6175.68|6607.97|7751.04|8183.33|1507.20| +2452153|60566|2452196|3693|86041|782160|6726|2892|36733|454299|4990|3556|1|7|10|3|89|583|97|99.49|290.51|0.00|28179.47|0.00|9650.53|28179.47|0.00|0.00|11552.70|0.00|0.00|11552.70|11552.70|-9650.53| +2452153|60566|2452238|7729|86041|782160|6726|2892|36733|454299|4990|3556|13|21|18|2|226|583|7|56.37|104.28|11.47|649.67|80.29|394.59|729.96|2.40|0.00|328.44|80.29|82.69|408.73|411.13|-314.30| +2452153|60566|2452220|10127|86041|782160|6726|2892|36733|454299|4990|3556|11|3|18|2|103|583|25|32.87|50.94|44.82|153.00|1120.50|821.75|1273.50|67.23|0.00|25.25|1120.50|1187.73|1145.75|1212.98|298.75| +2452153|60566|2452168|6727|86041|782160|6726|2892|36733|454299|4990|3556|55|5|8|3|18|583|14|2.90|6.35|1.33|70.28|18.62|40.60|88.90|0.00|0.00|31.92|18.62|18.62|50.54|50.54|-21.98| +2452153|60566|2452197|14705|86041|782160|6726|2892|36733|454299|4990|3556|59|29|10|3|233|583|73|97.39|107.12|4.28|7507.32|312.44|7109.47|7819.76|12.49|0.00|2345.49|312.44|324.93|2657.93|2670.42|-6797.03| +2452153|60566|2452218|1195|86041|782160|6726|2892|36733|454299|4990|3556|59|5|19|2|238|583|1|66.50|153.61|64.51|89.10|64.51|66.50|153.61|0.00|0.00|7.68|64.51|64.51|72.19|72.19|-1.99| +2452153|60566|2452172|13373|86041|782160|6726|2892|36733|454299|4990|3556|41|29|12|1|290|583|90|31.30|81.06|25.12|5034.60|2260.80|2817.00|7295.40|0.00|0.00|3501.00|2260.80|2260.80|5761.80|5761.80|-556.20| +2452153|60566|2452163|7599|86041|782160|6726|2892|36733|454299|4990|3556|47|13|3|4|295|583|55|22.23|64.68|30.39|1885.95|1671.45|1222.65|3557.40|0.00|0.00|1529.55|1671.45|1671.45|3201.00|3201.00|448.80| +2452153|60566|2452187|5701|86041|782160|6726|2892|36733|454299|4990|3556|9|29|2|2|142|583|47|11.74|30.75|10.14|968.67|476.58|551.78|1445.25|38.12|0.00|679.15|476.58|514.70|1155.73|1193.85|-75.20| +2452153|60566|2452162|10651|86041|782160|6726|2892|36733|454299|4990|3556|51|11|16|1|6|583|66|67.45|113.99|68.39|3009.60|4513.74|4451.70|7523.34|406.23|0.00|3460.38|4513.74|4919.97|7974.12|8380.35|62.04| +2452264|34289|2452308|17053|75562|1595667|925|3430|34801|1342559|6498|18156|21|7|7|2|80|584|47|37.18|87.00|73.95|613.35|3475.65|1747.46|4089.00|10.42|2432.95|1676.49|1042.70|1053.12|2719.19|2729.61|-704.76| +2452264|34289|2452311|9477|75562|1595667|925|3430|34801|1342559|6498|18156|35|7|20|1|59|584|61|67.16|138.34|109.28|1772.66|6666.08|4096.76|8438.74|599.94|0.00|2868.83|6666.08|7266.02|9534.91|10134.85|2569.32| +2452264|34289|2452335|17669|75562|1595667|925|3430|34801|1342559|6498|18156|41|7|7|5|43|584|100|68.77|78.39|73.68|471.00|7368.00|6877.00|7839.00|0.00|0.00|548.00|7368.00|7368.00|7916.00|7916.00|491.00| +2452264|34289|2452344|405|75562|1595667|925|3430|34801|1342559|6498|18156|49|19|2|2|99|584|74|83.68|226.77|129.25|7216.48|9564.50|6192.32|16780.98|478.22|0.00|0.00|9564.50|10042.72|9564.50|10042.72|3372.18| +2452264|34289|2452331|3487|75562|1595667|925|3430|34801|1342559|6498|18156|59|5|16|4|252|584|24|8.30|24.48|3.18|511.20|76.32|199.20|587.52|4.57|0.00|0.00|76.32|80.89|76.32|80.89|-122.88| +2452264|34289|2452301|9863|75562|1595667|925|3430|34801|1342559|6498|18156|55|1|14|4|167|584|94|24.98|28.97|21.14|736.02|1987.16|2348.12|2723.18|158.97|0.00|462.48|1987.16|2146.13|2449.64|2608.61|-360.96| +2452264|34289|2452364|14837|75562|1595667|925|3430|34801|1342559|6498|18156|7|21|5|2|219|584|91|52.65|137.94|49.65|8034.39|4518.15|4791.15|12552.54|225.00|768.08|1631.63|3750.07|3975.07|5381.70|5606.70|-1041.08| +2452264|34289|2452319|16773|75562|1595667|925|3430|34801|1342559|6498|18156|19|23|4|4|161|584|65|6.43|15.75|0.94|962.65|61.10|417.95|1023.75|0.23|53.15|40.95|7.95|8.18|48.90|49.13|-410.00| +2452264|34289|2452358|8743|75562|1595667|925|3430|34801|1342559|6498|18156|47|9|6|2|128|584|65|36.29|87.09|48.77|2490.80|3170.05|2358.85|5660.85|29.79|190.20|792.35|2979.85|3009.64|3772.20|3801.99|621.00| +2452264|34289|2452292|9231|75562|1595667|925|3430|34801|1342559|6498|18156|5|5|10|5|77|584|98|93.81|219.51|70.24|14628.46|6883.52|9193.38|21511.98|143.17|3304.08|6883.52|3579.44|3722.61|10462.96|10606.13|-5613.94| +2452264|34289|2452297|4067|75562|1595667|925|3430|34801|1342559|6498|18156|23|29|8|1|294|584|97|76.96|204.71|161.72|4170.03|15686.84|7465.12|19856.87|941.21|0.00|5361.19|15686.84|16628.05|21048.03|21989.24|8221.72| +2452264|34289|2452338|805|75562|1595667|925|3430|34801|1342559|6498|18156|7|29|6|2|258|584|78|34.57|92.30|30.45|4824.30|2375.10|2696.46|7199.40|118.75|0.00|359.58|2375.10|2493.85|2734.68|2853.43|-321.36| +2452264|34289|2452337|517|75562|1595667|925|3430|34801|1342559|6498|18156|1|25|8|3|245|584|13|94.42|239.82|81.53|2057.77|1059.89|1227.46|3117.66|31.79|0.00|1091.09|1059.89|1091.68|2150.98|2182.77|-167.57| +2452264|34289|2452271|16329|75562|1595667|925|3430|34801|1342559|6498|18156|23|1|13|2|194|584|78|4.93|9.41|6.21|249.60|484.38|384.54|733.98|29.06|0.00|153.66|484.38|513.44|638.04|667.10|99.84| +2452264|34289|2452299|6943|75562|1595667|925|3430|34801|1342559|6498|18156|53|1|12|1|27|584|50|95.89|98.76|51.35|2370.50|2567.50|4794.50|4938.00|77.02|0.00|197.50|2567.50|2644.52|2765.00|2842.02|-2227.00| +2451660|52365|2451671|14995|76031|1119594|3957|9120|77415|1708071|5056|20219|55|26|10|1|2|585|82|81.16|190.72|169.74|1720.36|13918.68|6655.12|15639.04|1252.68|0.00|938.08|13918.68|15171.36|14856.76|16109.44|7263.56| +2451660|52365|2451693|15128|76031|1119594|3957|9120|77415|1708071|5056|20219|19|7|8|2|136|585|99|73.41|132.13|39.63|9157.50|3923.37|7267.59|13080.87|196.16|0.00|4447.08|3923.37|4119.53|8370.45|8566.61|-3344.22| +2451660|52365|2451759|14131|76031|1119594|3957|9120|77415|1708071|5056|20219|32|29|3|3|196|585|90|72.12|78.61|71.53|637.20|6437.70|6490.80|7074.90|0.00|0.00|3537.00|6437.70|6437.70|9974.70|9974.70|-53.10| +2451660|52365|2451758|1859|76031|1119594|3957|9120|77415|1708071|5056|20219|14|20|14|2|50|585|88|96.35|190.77|62.95|11248.16|5539.60|8478.80|16787.76|55.39|0.00|7218.64|5539.60|5594.99|12758.24|12813.63|-2939.20| +2451660|52365|2451773|12143|76031|1119594|3957|9120|77415|1708071|5056|20219|20|2|16|2|81|585|43|86.75|122.31|29.35|3997.28|1262.05|3730.25|5259.33|12.62|0.00|1209.59|1262.05|1274.67|2471.64|2484.26|-2468.20| +2451660|52365|2451715|12565|76031|1119594|3957|9120|77415|1708071|5056|20219|50|29|4|5|19|585|16|2.19|4.07|2.31|28.16|36.96|35.04|65.12|0.36|0.00|13.60|36.96|37.32|50.56|50.92|1.92| +2451660|52365|2451722|1106|76031|1119594|3957|9120|77415|1708071|5056|20219|20|2|13|2|178|585|55|98.57|208.96|202.69|344.85|11147.95|5421.35|11492.80|445.91|0.00|4481.95|11147.95|11593.86|15629.90|16075.81|5726.60| +2451660|52365|2451676|17287|76031|1119594|3957|9120|77415|1708071|5056|20219|25|5|3|3|93|585|35|15.40|19.86|7.54|431.20|263.90|539.00|695.10|23.75|0.00|201.25|263.90|287.65|465.15|488.90|-275.10| +2451660|52365|2451743|8981|76031|1119594|3957|9120|77415|1708071|5056|20219|17|5|17|1|278|585|53|1.90|3.95|2.56|73.67|135.68|100.70|209.35|0.92|43.41|37.63|92.27|93.19|129.90|130.82|-8.43| +2451660|52365|2451779|4225|76031|1119594|3957|9120|77415|1708071|5056|20219|14|2|6|5|187|585|72|86.74|201.23|110.67|6520.32|7968.24|6245.28|14488.56|478.09|0.00|1738.08|7968.24|8446.33|9706.32|10184.41|1722.96| +2451660|52365|2451726|1849|76031|1119594|3957|9120|77415|1708071|5056|20219|31|8|14|3|242|585|100|78.96|99.48|31.83|6765.00|3183.00|7896.00|9948.00|95.49|0.00|2089.00|3183.00|3278.49|5272.00|5367.49|-4713.00| +2451660|52365|2451681|3007|76031|1119594|3957|9120|77415|1708071|5056|20219|25|17|5|4|230|585|10|25.79|49.51|13.86|356.50|138.60|257.90|495.10|12.47|0.00|158.40|138.60|151.07|297.00|309.47|-119.30| +2451660|52365|2451766|17180|76031|1119594|3957|9120|77415|1708071|5056|20219|44|2|8|2|246|585|62|94.82|108.09|2.16|6567.66|133.92|5878.84|6701.58|0.00|0.00|2479.38|133.92|133.92|2613.30|2613.30|-5744.92| +2451660|52365|2451751|5737|76031|1119594|3957|9120|77415|1708071|5056|20219|29|7|16|3|40|585|65|31.52|88.25|67.95|1319.50|4416.75|2048.80|5736.25|353.34|0.00|1147.25|4416.75|4770.09|5564.00|5917.34|2367.95| +2451660|52365|2451709|6974|76031|1119594|3957|9120|77415|1708071|5056|20219|53|14|12|1|267|585|52|61.08|109.33|92.93|852.80|4832.36|3176.16|5685.16|144.97|0.00|1534.52|4832.36|4977.33|6366.88|6511.85|1656.20| +2451141|46331|2451172|16642|20581|259818|6255|16692|20581|259818|6255|16692|1|25|15|2|259|586|39|32.71|43.83|30.24|530.01|1179.36|1275.69|1709.37|58.96|0.00|854.49|1179.36|1238.32|2033.85|2092.81|-96.33| +2451141|46331|2451239|1924|20581|259818|6255|16692|20581|259818|6255|16692|32|2|10|5|100|586|1|82.04|171.46|154.31|17.15|154.31|82.04|171.46|3.08|0.00|36.00|154.31|157.39|190.31|193.39|72.27| +2451141|46331|2451207|16828|20581|259818|6255|16692|20581|259818|6255|16692|37|4|7|1|15|586|22|76.13|195.65|21.52|3830.86|473.44|1674.86|4304.30|18.93|0.00|172.04|473.44|492.37|645.48|664.41|-1201.42| +2451141|46331|2451156|14962|20581|259818|6255|16692|20581|259818|6255|16692|28|4|11|4|207|586|32|93.26|178.12|117.55|1938.24|3761.60|2984.32|5699.84|0.00|0.00|2165.76|3761.60|3761.60|5927.36|5927.36|777.28| +2451141|46331|2451145|13586|20581|259818|6255|16692|20581|259818|6255|16692|16|1|6|3|173|586|98|49.85|133.09|106.47|2608.76|10434.06|4885.30|13042.82|0.00|0.00|4303.18|10434.06|10434.06|14737.24|14737.24|5548.76| +2451141|46331|2451180|8425|20581|259818|6255|16692|20581|259818|6255|16692|52|4|2|1|187|586|3|8.94|14.75|3.98|32.31|11.94|26.82|44.25|0.00|0.00|10.62|11.94|11.94|22.56|22.56|-14.88| +2451141|46331|2451203|4963|20581|259818|6255|16692|20581|259818|6255|16692|34|4|8|1|71|586|31|74.76|203.34|201.30|63.24|6240.30|2317.56|6303.54|187.20|0.00|2584.16|6240.30|6427.50|8824.46|9011.66|3922.74| +2451141|46331|2451197|11989|20581|259818|6255|16692|20581|259818|6255|16692|20|28|6|5|41|586|59|28.94|77.55|22.48|3249.13|1326.32|1707.46|4575.45|119.36|0.00|2195.98|1326.32|1445.68|3522.30|3641.66|-381.14| +2451141|46331|2451224|1264|20581|259818|6255|16692|20581|259818|6255|16692|49|25|4|2|120|586|42|94.21|193.13|1.93|8030.40|81.06|3956.82|8111.46|4.05|0.00|3406.62|81.06|85.11|3487.68|3491.73|-3875.76| +2451141|46331|2451207|9298|20581|259818|6255|16692|20581|259818|6255|16692|56|2|8|1|64|586|48|94.28|258.32|25.83|11159.52|1239.84|4525.44|12399.36|19.83|843.09|5331.36|396.75|416.58|5728.11|5747.94|-4128.69| +2451141|46331|2451172|7501|20581|259818|6255|16692|20581|259818|6255|16692|25|7|12|3|141|586|32|18.45|51.29|10.77|1296.64|344.64|590.40|1641.28|6.89|0.00|196.80|344.64|351.53|541.44|548.33|-245.76| +2451141|46331|2451145|5755|20581|259818|6255|16692|20581|259818|6255|16692|14|14|1|4|107|586|32|4.14|8.85|5.04|121.92|161.28|132.48|283.20|0.00|0.00|138.56|161.28|161.28|299.84|299.84|28.80| +2452604|63754|2452646|1902|815|933165|6315|36618|59724|111908|2821|45571|9|9|15|5|118|587|53|58.36|148.23|136.37|628.58|7227.61|3093.08|7856.19|289.10|0.00|1256.63|7227.61|7516.71|8484.24|8773.34|4134.53| +2452604|63754|2452663|7572|815|933165|6315|36618|59724|111908|2821|45571|55|24|19|3|257|587|74|16.20|47.14|8.01|2895.62|592.74|1198.80|3488.36|41.49|0.00|348.54|592.74|634.23|941.28|982.77|-606.06| +2452604|63754|2452649|14691|815|933165|6315|36618|59724|111908|2821|45571|60|9|3|3|126|587|28|85.35|171.55|3.43|4707.36|96.04|2389.80|4803.40|1.92|0.00|768.32|96.04|97.96|864.36|866.28|-2293.76| +2452604|63754|2452719|8391|815|933165|6315|36618|59724|111908|2821|45571|42|30|6|2|101|587|10|46.96|74.66|71.67|29.90|716.70|469.60|746.60|14.33|0.00|261.30|716.70|731.03|978.00|992.33|247.10| +2452604|63754|2452631|13851|815|933165|6315|36618|59724|111908|2821|45571|3|12|13|2|71|587|49|7.69|15.07|8.74|310.17|428.26|376.81|738.43|20.38|137.04|7.35|291.22|311.60|298.57|318.95|-85.59| +2452604|63754|2452649|2202|815|933165|6315|36618|59724|111908|2821|45571|18|25|7|3|152|587|23|32.04|92.59|56.47|830.76|1298.81|736.92|2129.57|12.98|0.00|979.57|1298.81|1311.79|2278.38|2291.36|561.89| +2452604|63754|2452622|1293|815|933165|6315|36618|59724|111908|2821|45571|6|13|1|2|206|587|72|33.08|60.86|3.04|4163.04|218.88|2381.76|4381.92|14.88|32.83|2190.96|186.05|200.93|2377.01|2391.89|-2195.71| +2452604|63754|2452687|6957|815|933165|6315|36618|59724|111908|2821|45571|45|27|15|1|227|587|7|53.44|59.31|58.12|8.33|406.84|374.08|415.17|12.20|0.00|83.02|406.84|419.04|489.86|502.06|32.76| +2452265|60313|2452344|7943|19909|138678|4647|169|33405|12884|5676|18398|57|1|6|1|256|588|45|18.97|55.96|3.91|2342.25|175.95|853.65|2518.20|6.24|51.02|301.95|124.93|131.17|426.88|433.12|-728.72| +2452265|60313|2452324|4659|19909|138678|4647|169|33405|12884|5676|18398|27|7|17|2|170|588|66|15.98|47.62|39.04|566.28|2576.64|1054.68|3142.92|178.56|25.76|1320.00|2550.88|2729.44|3870.88|4049.44|1496.20| +2452265|60313|2452330|4107|19909|138678|4647|169|33405|12884|5676|18398|43|9|8|1|254|588|29|71.77|125.59|100.47|728.48|2913.63|2081.33|3642.11|233.09|0.00|1638.79|2913.63|3146.72|4552.42|4785.51|832.30| +2452265|60313|2452381|4025|19909|138678|4647|169|33405|12884|5676|18398|19|7|5|4|83|588|94|80.86|183.55|3.67|16908.72|344.98|7600.84|17253.70|24.14|0.00|8281.40|344.98|369.12|8626.38|8650.52|-7255.86| +2452265|60313|2452382|1879|19909|138678|4647|169|33405|12884|5676|18398|17|13|5|1|294|588|57|20.31|44.88|9.42|2021.22|536.94|1157.67|2558.16|32.21|0.00|613.89|536.94|569.15|1150.83|1183.04|-620.73| +2452265|60313|2452317|11151|19909|138678|4647|169|33405|12884|5676|18398|31|3|5|1|297|588|43|81.55|220.18|138.71|3503.21|5964.53|3506.65|9467.74|64.41|5248.78|4449.64|715.75|780.16|5165.39|5229.80|-2790.90| +2452265|60313|2452318|7075|19909|138678|4647|169|33405|12884|5676|18398|15|25|11|1|284|588|57|60.17|154.03|55.45|5619.06|3160.65|3429.69|8779.71|40.77|1801.57|3423.99|1359.08|1399.85|4783.07|4823.84|-2070.61| +2452265|60313|2452290|7543|19909|138678|4647|169|33405|12884|5676|18398|43|5|4|5|96|588|76|43.02|44.31|36.77|573.04|2794.52|3269.52|3367.56|83.83|0.00|1649.96|2794.52|2878.35|4444.48|4528.31|-475.00| +2452265|60313|2452280|10707|19909|138678|4647|169|33405|12884|5676|18398|27|25|15|4|153|588|37|51.40|51.40|46.26|190.18|1711.62|1901.80|1901.80|102.69|0.00|113.96|1711.62|1814.31|1825.58|1928.27|-190.18| +2451806|69728|2451840|6191|73632|1005376|6720|15664|22182|250721|2049|37746|8|11|7|5|136|589|77|43.03|55.50|32.74|1752.52|2520.98|3313.31|4273.50|201.67|0.00|256.41|2520.98|2722.65|2777.39|2979.06|-792.33| +2451806|69728|2451836|15527|73632|1005376|6720|15664|22182|250721|2049|37746|23|23|19|4|86|589|97|61.59|107.78|36.64|6900.58|3554.08|5974.23|10454.66|140.38|746.35|208.55|2807.73|2948.11|3016.28|3156.66|-3166.50| +2451806|69728|2451887|14237|73632|1005376|6720|15664|22182|250721|2049|37746|38|5|13|3|116|589|72|82.65|194.22|93.22|7272.00|6711.84|5950.80|13983.84|402.71|0.00|5033.52|6711.84|7114.55|11745.36|12148.07|761.04| +2451806|69728|2451863|5615|73632|1005376|6720|15664|22182|250721|2049|37746|20|14|15|2|66|589|73|85.40|186.17|24.20|11823.81|1766.60|6234.20|13590.41|0.00|1077.62|4077.05|688.98|688.98|4766.03|4766.03|-5545.22| +2451806|69728|2451853|9985|73632|1005376|6720|15664|22182|250721|2049|37746|23|11|13|2|94|589|79|93.13|117.34|19.94|7694.60|1575.26|7357.27|9269.86|78.76|0.00|3707.47|1575.26|1654.02|5282.73|5361.49|-5782.01| +2451806|69728|2451924|7943|73632|1005376|6720|15664|22182|250721|2049|37746|32|17|1|1|98|589|67|89.42|247.69|235.30|830.13|15765.10|5991.14|16595.23|945.90|0.00|1161.11|15765.10|16711.00|16926.21|17872.11|9773.96| +2451806|69728|2451808|4658|73632|1005376|6720|15664|22182|250721|2049|37746|8|29|8|3|248|589|90|77.90|100.49|10.04|8140.50|903.60|7011.00|9044.10|27.10|0.00|451.80|903.60|930.70|1355.40|1382.50|-6107.40| +2451806|69728|2451842|4106|73632|1005376|6720|15664|22182|250721|2049|37746|11|23|8|4|120|589|62|90.65|219.37|100.91|7344.52|6256.42|5620.30|13600.94|125.12|0.00|4216.00|6256.42|6381.54|10472.42|10597.54|636.12| +2451806|69728|2451837|4025|73632|1005376|6720|15664|22182|250721|2049|37746|38|17|18|5|29|589|33|49.98|134.44|114.27|665.61|3770.91|1649.34|4436.52|263.96|0.00|1242.12|3770.91|4034.87|5013.03|5276.99|2121.57| +2451806|69728|2451840|1879|73632|1005376|6720|15664|22182|250721|2049|37746|11|5|3|5|10|589|48|2.98|3.78|3.25|25.44|156.00|143.04|181.44|1.56|0.00|32.64|156.00|157.56|188.64|190.20|12.96| +2451806|69728|2451900|11150|73632|1005376|6720|15664|22182|250721|2049|37746|55|20|4|2|80|589|33|41.34|82.26|4.11|2578.95|135.63|1364.22|2714.58|1.09|99.00|1004.19|36.63|37.72|1040.82|1041.91|-1327.59| +2451806|69728|2451853|7075|73632|1005376|6720|15664|22182|250721|2049|37746|2|1|13|2|104|589|89|5.99|6.28|0.06|553.58|5.34|533.11|558.92|0.37|0.00|139.73|5.34|5.71|145.07|145.44|-527.77| +2451806|69728|2451893|7543|73632|1005376|6720|15664|22182|250721|2049|37746|7|7|1|1|131|589|79|43.96|129.24|58.15|5616.11|4593.85|3472.84|10209.96|352.80|183.75|4900.37|4410.10|4762.90|9310.47|9663.27|937.26| +2451806|69728|2451832|10706|73632|1005376|6720|15664|22182|250721|2049|37746|1|26|20|3|39|589|57|14.79|24.55|21.60|168.15|1231.20|843.03|1399.35|24.62|0.00|503.31|1231.20|1255.82|1734.51|1759.13|388.17| +2451254|9520|2451264|15392|60266|1448104|6342|45380|24330|1271754|5247|41885|46|1|11|1|211|590|85|60.54|97.46|93.56|331.50|7952.60|5145.90|8284.10|39.76|7157.34|1159.40|795.26|835.02|1954.66|1994.42|-4350.64| +2451254|9520|2451302|11554|60266|1448104|6342|45380|24330|1271754|5247|41885|34|4|1|1|11|590|29|44.74|47.87|26.80|611.03|777.20|1297.46|1388.23|7.38|38.86|555.06|738.34|745.72|1293.40|1300.78|-559.12| +2451254|9520|2451313|889|60266|1448104|6342|45380|24330|1271754|5247|41885|56|2|15|3|161|590|93|48.53|137.33|27.46|10217.91|2553.78|4513.29|12771.69|153.22|0.00|1021.14|2553.78|2707.00|3574.92|3728.14|-1959.51| +2451254|9520|2451339|14702|60266|1448104|6342|45380|24330|1271754|5247|41885|56|19|11|4|200|590|37|89.47|93.94|54.48|1460.02|2015.76|3310.39|3475.78|80.63|0.00|1737.89|2015.76|2096.39|3753.65|3834.28|-1294.63| +2451254|9520|2451265|13855|60266|1448104|6342|45380|24330|1271754|5247|41885|46|2|20|3|86|590|54|34.90|98.06|40.20|3124.44|2170.80|1884.60|5295.24|65.12|0.00|2012.04|2170.80|2235.92|4182.84|4247.96|286.20| +2451254|9520|2451306|11545|60266|1448104|6342|45380|24330|1271754|5247|41885|1|16|4|4|137|590|56|6.46|18.15|10.52|427.28|589.12|361.76|1016.40|35.34|0.00|385.84|589.12|624.46|974.96|1010.30|227.36| +2451254|9520|2451258|10118|60266|1448104|6342|45380|24330|1271754|5247|41885|58|7|18|5|249|590|17|32.64|51.24|47.65|61.03|810.05|554.88|871.08|2.18|737.14|217.77|72.91|75.09|290.68|292.86|-481.97| +2451254|9520|2451324|12140|60266|1448104|6342|45380|24330|1271754|5247|41885|25|8|20|2|214|590|21|2.54|4.54|0.81|78.33|17.01|53.34|95.34|1.19|0.00|17.01|17.01|18.20|34.02|35.21|-36.33| +2451254|9520|2451357|15733|60266|1448104|6342|45380|24330|1271754|5247|41885|38|16|1|3|195|590|11|9.35|12.99|5.45|82.94|59.95|102.85|142.89|4.19|0.00|69.96|59.95|64.14|129.91|134.10|-42.90| +2451254|9520|2451292|12124|60266|1448104|6342|45380|24330|1271754|5247|41885|19|13|14|4|231|590|87|68.11|89.22|59.77|2562.15|5199.99|5925.57|7762.14|103.99|0.00|1241.49|5199.99|5303.98|6441.48|6545.47|-725.58| +2451254|9520|2451354|6020|60266|1448104|6342|45380|24330|1271754|5247|41885|16|7|16|3|3|590|80|36.54|89.88|5.39|6759.20|431.20|2923.20|7190.40|22.07|155.23|2516.00|275.97|298.04|2791.97|2814.04|-2647.23| +2451254|9520|2451277|265|60266|1448104|6342|45380|24330|1271754|5247|41885|32|22|4|4|154|590|46|15.18|22.01|14.30|354.66|657.80|698.28|1012.46|19.73|0.00|161.92|657.80|677.53|819.72|839.45|-40.48| +2451254|9520|2451289|12697|60266|1448104|6342|45380|24330|1271754|5247|41885|19|14|9|2|48|590|21|38.64|113.60|106.78|143.22|2242.38|811.44|2385.60|112.11|0.00|500.85|2242.38|2354.49|2743.23|2855.34|1430.94| +2451526|22509|2451637|13486|31586|1405962|5138|41914|63642|1268237|5842|39018|52|8|6|4|22|591|31|36.01|105.50|7.38|3041.72|228.78|1116.31|3270.50|6.68|61.77|130.82|167.01|173.69|297.83|304.51|-949.30| +2451526|22509|2451556|16504|31586|1405962|5138|41914|63642|1268237|5842|39018|13|13|17|2|44|591|42|14.27|30.82|23.11|323.82|970.62|599.34|1294.44|9.70|0.00|12.60|970.62|980.32|983.22|992.92|371.28| +2451526|22509|2451602|3094|31586|1405962|5138|41914|63642|1268237|5842|39018|16|28|1|2|93|591|41|35.13|67.44|54.62|525.62|2239.42|1440.33|2765.04|120.92|895.76|746.20|1343.66|1464.58|2089.86|2210.78|-96.67| +2451526|22509|2451630|8150|31586|1405962|5138|41914|63642|1268237|5842|39018|8|14|8|1|204|591|4|46.45|119.84|20.37|397.88|81.48|185.80|479.36|5.70|0.00|230.08|81.48|87.18|311.56|317.26|-104.32| +2451526|22509|2451633|8392|31586|1405962|5138|41914|63642|1268237|5842|39018|1|7|4|2|260|591|52|14.27|26.25|23.62|136.76|1228.24|742.04|1365.00|12.28|0.00|627.64|1228.24|1240.52|1855.88|1868.16|486.20| +2451526|22509|2451548|13088|31586|1405962|5138|41914|63642|1268237|5842|39018|25|25|10|5|170|591|81|45.72|115.67|109.88|468.99|8900.28|3703.32|9369.27|801.02|0.00|4309.20|8900.28|9701.30|13209.48|14010.50|5196.96| +2451526|22509|2451556|2930|31586|1405962|5138|41914|63642|1268237|5842|39018|14|1|17|3|146|591|71|49.07|72.62|13.79|4176.93|979.09|3483.97|5156.02|19.58|0.00|1288.65|979.09|998.67|2267.74|2287.32|-2504.88| +2451526|22509|2451617|3962|31586|1405962|5138|41914|63642|1268237|5842|39018|58|14|19|2|115|591|93|60.00|131.40|15.76|10754.52|1465.68|5580.00|12220.20|44.55|351.76|3421.47|1113.92|1158.47|4535.39|4579.94|-4466.08| +2452501|49051|2452566|16477|52694|1234818|3342|29419|52694|1234818|3342|29419|39|18|15|4|73|592|20|31.97|50.51|0.00|1010.20|0.00|639.40|1010.20|0.00|0.00|181.80|0.00|0.00|181.80|181.80|-639.40| +2452501|49051|2452521|10140|52694|1234818|3342|29419|52694|1234818|3342|29419|27|12|18|2|49|592|23|33.21|58.44|34.47|551.31|792.81|763.83|1344.12|15.85|0.00|147.66|792.81|808.66|940.47|956.32|28.98| +2452501|49051|2452559|8454|52694|1234818|3342|29419|52694|1234818|3342|29419|39|1|2|1|237|592|32|61.13|158.32|132.98|810.88|4255.36|1956.16|5066.24|42.55|0.00|557.12|4255.36|4297.91|4812.48|4855.03|2299.20| +2452501|49051|2452558|3343|52694|1234818|3342|29419|52694|1234818|3342|29419|15|15|1|4|84|592|17|81.47|118.13|77.96|682.89|1325.32|1384.99|2008.21|44.53|583.14|742.90|742.18|786.71|1485.08|1529.61|-642.81| +2452501|49051|2452563|17124|52694|1234818|3342|29419|52694|1234818|3342|29419|31|13|8|4|220|592|23|21.93|61.18|55.67|126.73|1280.41|504.39|1407.14|89.62|0.00|56.12|1280.41|1370.03|1336.53|1426.15|776.02| +2452501|49051|2452570|13783|52694|1234818|3342|29419|52694|1234818|3342|29419|1|30|9|1|200|592|82|75.79|106.10|41.37|5307.86|3392.34|6214.78|8700.20|237.46|0.00|1217.70|3392.34|3629.80|4610.04|4847.50|-2822.44| +2452501|49051|2452530|10185|52694|1234818|3342|29419|52694|1234818|3342|29419|31|3|8|4|170|592|29|31.79|70.57|40.93|859.56|1186.97|921.91|2046.53|71.21|0.00|1023.12|1186.97|1258.18|2210.09|2281.30|265.06| +2452501|49051|2452601|2155|52694|1234818|3342|29419|52694|1234818|3342|29419|36|21|7|5|66|592|45|78.44|177.27|40.77|6142.50|1834.65|3529.80|7977.15|91.73|0.00|1834.65|1834.65|1926.38|3669.30|3761.03|-1695.15| +2452501|49051|2452576|5766|52694|1234818|3342|29419|52694|1234818|3342|29419|7|27|14|2|37|592|36|48.12|100.57|52.29|1738.08|1882.44|1732.32|3620.52|131.77|0.00|1737.72|1882.44|2014.21|3620.16|3751.93|150.12| +2452501|49051|2452596|5169|52694|1234818|3342|29419|52694|1234818|3342|29419|49|12|7|3|249|592|5|73.88|133.72|128.37|26.75|641.85|369.40|668.60|38.51|0.00|120.30|641.85|680.36|762.15|800.66|272.45| +2452501|49051|2452505|3900|52694|1234818|3342|29419|52694|1234818|3342|29419|27|18|17|2|77|592|85|84.24|181.95|85.51|8197.40|7268.35|7160.40|15465.75|0.00|6977.61|3093.15|290.74|290.74|3383.89|3383.89|-6869.66| +2451502|24354|2451618|17377|71441|1275281|5874|5714|53192|1875395|3987|12627|56|16|2|2|212|593|6|80.93|83.35|79.18|25.02|475.08|485.58|500.10|28.50|0.00|190.02|475.08|503.58|665.10|693.60|-10.50| +2451502|24354|2451604|4951|71441|1275281|5874|5714|53192|1875395|3987|12627|38|16|5|4|50|593|53|93.23|117.46|115.11|124.55|6100.83|4941.19|6225.38|183.02|0.00|311.11|6100.83|6283.85|6411.94|6594.96|1159.64| +2451502|24354|2451523|7264|71441|1275281|5874|5714|53192|1875395|3987|12627|50|2|8|1|117|593|9|14.89|41.98|23.50|166.32|211.50|134.01|377.82|6.34|0.00|22.59|211.50|217.84|234.09|240.43|77.49| +2451502|24354|2451567|6392|71441|1275281|5874|5714|53192|1875395|3987|12627|43|14|4|5|26|593|74|22.82|23.96|20.12|284.16|1488.88|1688.68|1773.04|83.37|297.77|425.50|1191.11|1274.48|1616.61|1699.98|-497.57| +2451502|24354|2451538|1436|71441|1275281|5874|5714|53192|1875395|3987|12627|19|28|10|5|210|593|61|2.94|6.73|2.08|283.65|126.88|179.34|410.53|7.61|0.00|131.15|126.88|134.49|258.03|265.64|-52.46| +2451502|24354|2451532|5854|71441|1275281|5874|5714|53192|1875395|3987|12627|31|19|4|3|225|593|37|34.88|84.06|40.34|1617.64|1492.58|1290.56|3110.22|0.00|0.00|932.77|1492.58|1492.58|2425.35|2425.35|202.02| +2451502|24354|2451552|13334|71441|1275281|5874|5714|53192|1875395|3987|12627|22|8|11|4|7|593|33|67.21|120.30|110.67|317.79|3652.11|2217.93|3969.90|42.72|2227.78|436.59|1424.33|1467.05|1860.92|1903.64|-793.60| +2451502|24354|2451503|13540|71441|1275281|5874|5714|53192|1875395|3987|12627|1|13|13|4|205|593|89|71.86|74.01|2.96|6323.45|263.44|6395.54|6586.89|10.53|0.00|1580.64|263.44|273.97|1844.08|1854.61|-6132.10| +2451502|24354|2451550|7897|71441|1275281|5874|5714|53192|1875395|3987|12627|52|28|3|4|62|593|81|64.86|186.14|180.55|452.79|14624.55|5253.66|15077.34|146.24|0.00|5879.79|14624.55|14770.79|20504.34|20650.58|9370.89| +2451502|24354|2451518|1564|71441|1275281|5874|5714|53192|1875395|3987|12627|19|20|9|5|261|593|64|64.59|94.30|0.94|5975.04|60.16|4133.76|6035.20|4.21|0.00|2715.52|60.16|64.37|2775.68|2779.89|-4073.60| +2451502|24354|2451512|11557|71441|1275281|5874|5714|53192|1875395|3987|12627|1|19|19|5|224|593|38|60.29|119.97|99.57|775.20|3783.66|2291.02|4558.86|189.18|0.00|2096.84|3783.66|3972.84|5880.50|6069.68|1492.64| +2451502|24354|2451544|1166|71441|1275281|5874|5714|53192|1875395|3987|12627|26|7|11|1|18|593|24|40.70|83.84|55.33|684.24|1327.92|976.80|2012.16|77.68|464.77|462.72|863.15|940.83|1325.87|1403.55|-113.65| +2451000|65312|2451072|5056|20453|1346088|876|15711|31711|779419|922|3310|25|8|4|1|258|594|85|90.75|190.57|160.07|2592.50|13605.95|7713.75|16198.45|1088.47|0.00|323.85|13605.95|14694.42|13929.80|15018.27|5892.20| +2451000|65312|2451104|8554|20453|1346088|876|15711|31711|779419|922|3310|44|20|19|5|148|594|93|94.34|134.90|44.51|8406.27|4139.43|8773.62|12545.70|206.97|0.00|626.82|4139.43|4346.40|4766.25|4973.22|-4634.19| +2451000|65312|2451103|2966|20453|1346088|876|15711|31711|779419|922|3310|50|25|13|5|57|594|69|1.31|3.02|2.47|37.95|170.43|90.39|208.38|1.70|0.00|64.17|170.43|172.13|234.60|236.30|80.04| +2451000|65312|2451057|12002|20453|1346088|876|15711|31711|779419|922|3310|25|13|11|3|102|594|95|64.50|66.43|35.20|2966.85|3344.00|6127.50|6310.85|267.52|0.00|251.75|3344.00|3611.52|3595.75|3863.27|-2783.50| +2451000|65312|2451084|6086|20453|1346088|876|15711|31711|779419|922|3310|52|19|13|1|252|594|88|62.83|141.36|96.12|3981.12|8458.56|5529.04|12439.68|169.17|0.00|5348.64|8458.56|8627.73|13807.20|13976.37|2929.52| +2451000|65312|2451097|6712|20453|1346088|876|15711|31711|779419|922|3310|32|4|9|5|171|594|22|96.60|207.69|155.76|1142.46|3426.72|2125.20|4569.18|274.13|0.00|731.06|3426.72|3700.85|4157.78|4431.91|1301.52| +2451000|65312|2451008|4288|20453|1346088|876|15711|31711|779419|922|3310|34|7|20|1|232|594|57|81.24|94.23|42.40|2954.31|2416.80|4630.68|5371.11|0.00|0.00|2094.18|2416.80|2416.80|4510.98|4510.98|-2213.88| +2451000|65312|2451097|230|20453|1346088|876|15711|31711|779419|922|3310|2|25|11|2|150|594|46|67.72|199.77|97.88|4686.94|4502.48|3115.12|9189.42|40.52|3692.03|2848.32|810.45|850.97|3658.77|3699.29|-2304.67| +2451000|65312|2451041|4051|20453|1346088|876|15711|31711|779419|922|3310|50|16|19|2|272|594|54|92.70|118.65|112.71|320.76|6086.34|5005.80|6407.10|221.54|547.77|1793.88|5538.57|5760.11|7332.45|7553.99|532.77| +2451000|65312|2451068|13714|20453|1346088|876|15711|31711|779419|922|3310|8|2|2|2|2|594|94|67.37|154.95|131.70|2185.50|12379.80|6332.78|14565.30|0.00|0.00|1164.66|12379.80|12379.80|13544.46|13544.46|6047.02| +2451189|31175|2451287|13777|51790|1522536|5635|7181|16369|342348|4972|8918|1|16|4|4|114|595|53|94.68|193.14|13.51|9520.39|716.03|5018.04|10236.42|21.48|0.00|1432.59|716.03|737.51|2148.62|2170.10|-4302.01| +2451189|31175|2451235|15650|51790|1522536|5635|7181|16369|342348|4972|8918|20|13|20|1|212|595|56|24.45|66.99|28.13|2176.16|1575.28|1369.20|3751.44|35.91|976.67|149.52|598.61|634.52|748.13|784.04|-770.59| +2451189|31175|2451307|15704|51790|1522536|5635|7181|16369|342348|4972|8918|13|13|10|5|60|595|38|12.88|29.62|7.99|821.94|303.62|489.44|1125.56|18.21|0.00|483.74|303.62|321.83|787.36|805.57|-185.82| +2451189|31175|2451293|3736|51790|1522536|5635|7181|16369|342348|4972|8918|52|20|5|4|46|595|1|93.21|151.00|147.98|3.02|147.98|93.21|151.00|11.83|0.00|42.28|147.98|159.81|190.26|202.09|54.77| +2451189|31175|2451295|15788|51790|1522536|5635|7181|16369|342348|4972|8918|20|26|20|3|256|595|91|48.63|65.65|1.31|5854.94|119.21|4425.33|5974.15|2.89|22.64|1851.85|96.57|99.46|1948.42|1951.31|-4328.76| +2451189|31175|2451230|13550|51790|1522536|5635|7181|16369|342348|4972|8918|4|25|10|5|147|595|41|68.87|139.11|68.16|2908.95|2794.56|2823.67|5703.51|111.78|0.00|2794.56|2794.56|2906.34|5589.12|5700.90|-29.11| +2451189|31175|2451202|15200|51790|1522536|5635|7181|16369|342348|4972|8918|14|19|12|1|293|595|98|74.92|191.79|164.93|2632.28|16163.14|7342.16|18795.42|484.89|0.00|1691.48|16163.14|16648.03|17854.62|18339.51|8820.98| +2451189|31175|2451209|14494|51790|1522536|5635|7181|16369|342348|4972|8918|31|26|2|5|181|595|90|10.95|21.02|17.23|341.10|1550.70|985.50|1891.80|9.92|558.25|378.00|992.45|1002.37|1370.45|1380.37|6.95| +2451189|31175|2451203|9572|51790|1522536|5635|7181|16369|342348|4972|8918|1|10|4|2|180|595|30|24.34|26.04|22.39|109.50|671.70|730.20|781.20|6.71|0.00|351.30|671.70|678.41|1023.00|1029.71|-58.50| +2451189|31175|2451306|5204|51790|1522536|5635|7181|16369|342348|4972|8918|55|2|6|5|261|595|12|82.26|173.56|38.18|1624.56|458.16|987.12|2082.72|36.65|0.00|708.12|458.16|494.81|1166.28|1202.93|-528.96| +2451189|31175|2451220|11311|51790|1522536|5635|7181|16369|342348|4972|8918|7|14|12|1|170|595|19|50.81|93.99|87.41|125.02|1660.79|965.39|1785.81|132.86|0.00|89.11|1660.79|1793.65|1749.90|1882.76|695.40| +2451189|31175|2451229|15958|51790|1522536|5635|7181|16369|342348|4972|8918|19|19|20|5|10|595|25|64.80|97.20|26.24|1774.00|656.00|1620.00|2430.00|0.00|0.00|996.25|656.00|656.00|1652.25|1652.25|-964.00| +2451189|31175|2451253|12260|51790|1522536|5635|7181|16369|342348|4972|8918|55|14|18|3|225|595|91|21.39|59.89|58.69|109.20|5340.79|1946.49|5449.99|320.44|0.00|1852.76|5340.79|5661.23|7193.55|7513.99|3394.30| +2451189|31175|2451280|15734|51790|1522536|5635|7181|16369|342348|4972|8918|10|7|5|3|134|595|96|14.35|31.42|15.08|1568.64|1447.68|1377.60|3016.32|101.33|0.00|120.00|1447.68|1549.01|1567.68|1669.01|70.08| +2451189|31175|2451286|9769|51790|1522536|5635|7181|16369|342348|4972|8918|14|19|5|5|32|595|19|70.92|114.89|39.06|1440.77|742.14|1347.48|2182.91|59.37|0.00|523.83|742.14|801.51|1265.97|1325.34|-605.34| +2452638|44768|2452721|4873|30001|1605681|4208|37935|68164|1841010|1516|26396|9|27|20|4|265|596|88|79.56|140.02|99.41|3573.68|8748.08|7001.28|12321.76|349.92|0.00|2464.00|8748.08|9098.00|11212.08|11562.00|1746.80| +2452638|44768|2452689|9648|30001|1605681|4208|37935|68164|1841010|1516|26396|55|1|14|4|167|596|17|15.64|31.28|30.96|5.44|526.32|265.88|531.76|21.05|0.00|79.73|526.32|547.37|606.05|627.10|260.44| +2452638|44768|2452711|6217|30001|1605681|4208|37935|68164|1841010|1516|26396|48|9|13|4|15|596|88|77.49|104.61|65.90|3406.48|5799.20|6819.12|9205.68|521.92|0.00|2392.72|5799.20|6321.12|8191.92|8713.84|-1019.92| +2452638|44768|2452737|10107|30001|1605681|4208|37935|68164|1841010|1516|26396|31|19|14|3|142|596|68|42.58|127.31|12.73|7791.44|865.64|2895.44|8657.08|32.89|43.28|1125.40|822.36|855.25|1947.76|1980.65|-2073.08| +2452638|44768|2452713|6031|30001|1605681|4208|37935|68164|1841010|1516|26396|42|3|13|4|35|596|59|53.19|55.31|22.12|1958.21|1305.08|3138.21|3263.29|65.25|0.00|1370.57|1305.08|1370.33|2675.65|2740.90|-1833.13| +2452638|44768|2452706|13602|30001|1605681|4208|37935|68164|1841010|1516|26396|33|15|19|5|124|596|61|12.58|28.05|14.86|804.59|906.46|767.38|1711.05|0.00|0.00|855.22|906.46|906.46|1761.68|1761.68|139.08| +2452638|44768|2452697|10743|30001|1605681|4208|37935|68164|1841010|1516|26396|43|25|17|3|249|596|81|73.44|203.42|16.27|15159.15|1317.87|5948.64|16477.02|65.89|0.00|5766.39|1317.87|1383.76|7084.26|7150.15|-4630.77| +2452638|44768|2452728|97|30001|1605681|4208|37935|68164|1841010|1516|26396|6|7|15|1|30|596|5|28.44|41.80|27.58|71.10|137.90|142.20|209.00|2.75|0.00|45.95|137.90|140.65|183.85|186.60|-4.30| +2452638|44768|2452704|1747|30001|1605681|4208|37935|68164|1841010|1516|26396|33|19|3|1|146|596|28|84.15|144.73|34.73|3080.00|972.44|2356.20|4052.44|12.25|768.22|810.32|204.22|216.47|1014.54|1026.79|-2151.98| +2452638|44768|2452644|11851|30001|1605681|4208|37935|68164|1841010|1516|26396|21|21|1|3|258|596|98|69.59|169.10|43.96|12263.72|4308.08|6819.82|16571.80|387.72|0.00|496.86|4308.08|4695.80|4804.94|5192.66|-2511.74| +2452638|44768|2452735|2425|30001|1605681|4208|37935|68164|1841010|1516|26396|21|3|6|5|44|596|83|16.56|39.24|36.88|195.88|3061.04|1374.48|3256.92|161.62|1040.75|943.71|2020.29|2181.91|2964.00|3125.62|645.81| +2452638|44768|2452661|10011|30001|1605681|4208|37935|68164|1841010|1516|26396|51|12|9|4|89|596|83|67.14|199.40|87.73|9268.61|7281.59|5572.62|16550.20|0.00|0.00|4302.72|7281.59|7281.59|11584.31|11584.31|1708.97| +2452638|44768|2452743|17899|30001|1605681|4208|37935|68164|1841010|1516|26396|33|9|13|4|292|596|1|61.97|154.30|40.11|114.19|40.11|61.97|154.30|2.38|6.01|13.88|34.10|36.48|47.98|50.36|-27.87| +2452638|44768|2452731|8655|30001|1605681|4208|37935|68164|1841010|1516|26396|15|12|7|5|226|596|59|80.72|88.79|81.68|419.49|4819.12|4762.48|5238.61|48.19|0.00|1676.19|4819.12|4867.31|6495.31|6543.50|56.64| +2452638|44768|2452649|17805|30001|1605681|4208|37935|68164|1841010|1516|26396|42|25|8|5|72|596|40|94.23|175.26|45.56|5188.00|1822.40|3769.20|7010.40|36.44|0.00|1822.40|1822.40|1858.84|3644.80|3681.24|-1946.80| +2451889|47390|2451967|10280|59883|805039|5842|39845|89096|857038|7007|7363|32|5|6|4|192|597|100|38.24|46.65|1.39|4526.00|139.00|3824.00|4665.00|6.95|0.00|1679.00|139.00|145.95|1818.00|1824.95|-3685.00| +2451889|47390|2451966|6805|59883|805039|5842|39845|89096|857038|7007|7363|5|23|11|5|9|597|46|88.15|119.88|104.29|717.14|4797.34|4054.90|5514.48|239.86|0.00|54.74|4797.34|5037.20|4852.08|5091.94|742.44| +2451889|47390|2452003|15043|59883|805039|5842|39845|89096|857038|7007|7363|32|14|19|3|4|597|55|95.18|204.63|163.70|2251.15|9003.50|5234.90|11254.65|540.21|0.00|2701.05|9003.50|9543.71|11704.55|12244.76|3768.60| +2451889|47390|2451903|329|59883|805039|5842|39845|89096|857038|7007|7363|38|23|1|5|129|597|63|31.51|69.95|65.05|308.70|4098.15|1985.13|4406.85|245.88|0.00|1762.74|4098.15|4344.03|5860.89|6106.77|2113.02| +2451889|47390|2451934|9458|59883|805039|5842|39845|89096|857038|7007|7363|44|5|20|5|137|597|49|9.43|13.20|6.07|349.37|297.43|462.07|646.80|17.84|0.00|180.81|297.43|315.27|478.24|496.08|-164.64| +2451889|47390|2451919|1183|59883|805039|5842|39845|89096|857038|7007|7363|49|25|4|2|23|597|93|45.82|125.08|66.29|5467.47|6164.97|4261.26|11632.44|123.29|0.00|1977.18|6164.97|6288.26|8142.15|8265.44|1903.71| +2451889|47390|2451935|3553|59883|805039|5842|39845|89096|857038|7007|7363|2|20|20|2|61|597|86|55.63|166.89|145.19|1866.20|12486.34|4784.18|14352.54|0.00|0.00|2296.20|12486.34|12486.34|14782.54|14782.54|7702.16| +2451889|47390|2451911|1517|59883|805039|5842|39845|89096|857038|7007|7363|41|26|17|5|53|597|13|56.36|106.52|64.97|540.15|844.61|732.68|1384.76|2.02|641.90|207.61|202.71|204.73|410.32|412.34|-529.97| +2451889|47390|2451940|17839|59883|805039|5842|39845|89096|857038|7007|7363|47|13|20|3|166|597|84|80.59|238.54|131.19|9017.40|11019.96|6769.56|20037.36|881.59|0.00|6611.64|11019.96|11901.55|17631.60|18513.19|4250.40| +2451889|47390|2452009|3367|59883|805039|5842|39845|89096|857038|7007|7363|44|2|1|5|59|597|67|67.38|169.12|113.31|3739.27|7591.77|4514.46|11331.04|227.75|0.00|5098.70|7591.77|7819.52|12690.47|12918.22|3077.31| +2451889|47390|2452002|2123|59883|805039|5842|39845|89096|857038|7007|7363|26|5|5|2|46|597|28|53.55|100.13|66.08|953.40|1850.24|1499.40|2803.64|37.00|0.00|0.00|1850.24|1887.24|1850.24|1887.24|350.84| +2452007|13505|2452049|181|24220|383203|5739|40722|28294|890900|3271|24254|51|15|20|2|254|598|40|50.91|73.31|55.71|704.00|2228.40|2036.40|2932.40|178.27|0.00|1231.60|2228.40|2406.67|3460.00|3638.27|192.00| +2452007|13505|2452125|6577|24220|383203|5739|40722|28294|890900|3271|24254|27|9|16|2|211|598|56|56.19|161.26|11.28|8398.88|631.68|3146.64|9030.56|6.31|0.00|1263.92|631.68|637.99|1895.60|1901.91|-2514.96| +2452007|13505|2452100|6831|24220|383203|5739|40722|28294|890900|3271|24254|59|15|16|4|189|598|17|91.25|131.40|14.45|1988.15|245.65|1551.25|2233.80|14.73|0.00|960.50|245.65|260.38|1206.15|1220.88|-1305.60| +2452007|13505|2452008|5997|24220|383203|5739|40722|28294|890900|3271|24254|43|9|1|2|120|598|44|31.87|61.19|50.17|484.88|2207.48|1402.28|2692.36|27.37|1523.16|861.52|684.32|711.69|1545.84|1573.21|-717.96| +2452007|13505|2452074|8669|24220|383203|5739|40722|28294|890900|3271|24254|39|19|20|3|213|598|30|64.43|97.93|61.69|1087.20|1850.70|1932.90|2937.90|74.02|0.00|146.70|1850.70|1924.72|1997.40|2071.42|-82.20| +2452007|13505|2452015|599|24220|383203|5739|40722|28294|890900|3271|24254|43|7|6|3|70|598|92|16.66|38.31|8.81|2714.00|810.52|1532.72|3524.52|40.52|0.00|352.36|810.52|851.04|1162.88|1203.40|-722.20| +2452007|13505|2452076|7013|24220|383203|5739|40722|28294|890900|3271|24254|59|19|13|1|296|598|30|94.18|119.60|7.17|3372.90|215.10|2825.40|3588.00|4.30|0.00|215.10|215.10|219.40|430.20|434.50|-2610.30| +2452007|13505|2452035|2405|24220|383203|5739|40722|28294|890900|3271|24254|27|27|14|4|97|598|71|40.71|46.00|9.66|2580.14|685.86|2890.41|3266.00|13.71|0.00|1012.46|685.86|699.57|1698.32|1712.03|-2204.55| +2452007|13505|2452078|12447|24220|383203|5739|40722|28294|890900|3271|24254|7|13|13|1|159|598|59|14.82|19.85|10.71|539.26|631.89|874.38|1171.15|44.23|0.00|245.44|631.89|676.12|877.33|921.56|-242.49| +2452007|13505|2452122|2677|24220|383203|5739|40722|28294|890900|3271|24254|3|3|2|2|152|598|58|39.89|83.37|61.69|1257.44|3578.02|2313.62|4835.46|0.00|3148.65|1837.44|429.37|429.37|2266.81|2266.81|-1884.25| +2452007|13505|2452107|12189|24220|383203|5739|40722|28294|890900|3271|24254|55|11|11|4|37|598|31|2.39|7.17|2.79|135.78|86.49|74.09|222.27|1.25|61.40|53.32|25.09|26.34|78.41|79.66|-49.00| +2452143|37200|2452224|15095|32546|1846233|7120|48760|62153|1237368|2574|48261|1|27|17|5|263|599|53|83.06|174.42|20.93|8134.97|1109.29|4402.18|9244.26|33.27|0.00|4529.38|1109.29|1142.56|5638.67|5671.94|-3292.89| diff --git a/spark-connector/spark-common/src/test/resources/data/tpcds/web_site/web_site.dat b/spark-connector/spark-common/src/test/resources/data/tpcds/web_site/web_site.dat new file mode 100644 index 00000000000..7cfc6ae4469 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/data/tpcds/web_site/web_site.dat @@ -0,0 +1,30 @@ +1|AAAAAAAABAAAAAAA|1997-08-16||site_0|2450807||Unknown|Ronald Shaffer|4|Grey lines ought to result indeed centres. Tod|Well similar decisions used to keep hardly democratic, personal priorities.|Joe George|6|cally|51|Dogwood Sunset|Ln|Suite 330|Midway|Williamson County|TN|31904|United States|-5|0.10| +2|AAAAAAAACAAAAAAA|1997-08-16|2000-08-15|site_0|2450798|2447148|Unknown|Tommy Jones|6|Completely excellent things ought to pro|Lucky passengers know. Red details will not hang alive, international s|David Myers|4|ese|358|Ridge Wilson|Cir.|Suite 150|Midway|Williamson County|TN|31904|United States|-5|0.00| +3|AAAAAAAACAAAAAAA|2000-08-16||site_0|2450798|2447148|Unknown|Tommy Jones|3|Completely excellent things ought to pro|Particular, common seasons shall not indicate fully more single decisions; |David Myers|4|ese|753|7th |Pkwy|Suite 210|Midway|Williamson County|TN|31904|United States|-5|0.02| +4|AAAAAAAAEAAAAAAA|1997-08-16|1999-08-16|site_0|2450781|2448956|Unknown|Harold Wilson|5|As strong notes deal questions. Proud visito|Deeply small books cannot extend all similar, clear historians. Free, new camp|James Harris|5|anti|805|Hill Hill|Dr.|Suite R|Midway|Williamson County|TN|31904|United States|-5|0.08| +5|AAAAAAAAEAAAAAAA|1999-08-17|2001-08-15|site_0|2450781|2448956|Unknown|Harold Wilson|5|Wide, final representat|Deeply small books cannot extend all similar, clear historians. Free, new camp|Edward George|1|ought|805|2nd 3rd|Road|Suite R|Fairview|Williamson County|TN|31904|United States|-5|0.08| +6|AAAAAAAAEAAAAAAA|2001-08-16||site_0|2450781|2448956|Unknown|Harold Wilson|5|Wide, final representat|Thin roles should run quickly. Patient parts include even in the ladies; only united trouser|John Sheppard|3|ought|63|Hickory 3rd|Ct.|Suite 130|Fairview|Williamson County|TN|31904|United States|-5|0.08| +7|AAAAAAAAHAAAAAAA|1997-08-16||site_1|2450756||Unknown|Moses Hicks|4|New homes comment great. Possibilities re|Enough small animals emerge at a holidays. Managers expect only.|William Reyes|4|ese|784|Central |Dr.|Suite Q|Fairview|Williamson County|TN|35709|United States|-5|0.12| +8|AAAAAAAAIAAAAAAA|1997-08-16|2000-08-15|site_1|2450747|2448922|Unknown|Charles Parker|1|Only, temporary services |Physical witnesses might not agree really already existing prayers. Loyal, possible m|Gerald Craft|5|anti|835|Washington |Avenue|Suite 470|Midway|Williamson County|TN|31904|United States|-5|0.07| +9|AAAAAAAAIAAAAAAA|2000-08-16||site_1|2450747|2448922|Unknown|Marshall Conner|6|Labour cars withdraw big, full issues|Physical witnesses might not agree really already existing prayers. Loyal, possible m|Scott Bryant|1|anti|835|11th |Cir.|Suite R|Midway|Williamson County|TN|31904|United States|-5|0.07| +10|AAAAAAAAKAAAAAAA|1997-08-16|1999-08-16|site_1|2450730|2447080|Unknown|Dwight Aaron|3|New, important system|Companies grow cells. Apart true eyes u|Frank Cooper|2|able|409|Lake |Ct.|Suite 370|Midway|Williamson County|TN|31904|United States|-5|0.07| +11|AAAAAAAAKAAAAAAA|1999-08-17|2001-08-15|site_1|2450730|2447080|Unknown|Dwight Aaron|3|Right efforts must no|Companies grow cells. Apart true eyes u|Casey Banks|2|able|409|Cedar |Parkway|Suite 350|Fairview|Williamson County|TN|35709|United States|-5|0.07| +12|AAAAAAAAKAAAAAAA|2001-08-16||site_1|2450730|2447080|Unknown|Richard Fuchs|3|Right efforts must no|Acres see else children. Mutual too|Casey Banks|2|able|409|Lincoln |Pkwy|Suite D|Fairview|Williamson County|TN|35709|United States|-5|0.07| +13|AAAAAAAANAAAAAAA|1997-08-16||site_2|2450705||Unknown|John Thomas|2|About rural reasons shall no|Political affairs ought to|James Brewer|5|anti|973|Cherry Main|Boulevard|Suite D|Midway|Williamson County|TN|31904|United States|-5|0.07| +14|AAAAAAAAOAAAAAAA|1997-08-16|2000-08-15|site_2|2450696|2447046|Unknown|Robert Arnold|5|Necessary, intermediat|Associations may kill sole parents. Red, pink results rise nearly|Gilbert Chapman|5|anti|887|River |Drive|Suite 390|Midway|Williamson County|TN|31904|United States|-5|0.07| +15|AAAAAAAAOAAAAAAA|2000-08-16||site_2|2450696|2447046|Unknown|James Austin|5|Necessary, intermediat|Associations may kill sole parents. Red, pink results rise nearly|Zachery Oneil|5|anti|887|Broadway South|Road|Suite 260|Midway|Williamson County|TN|31904|United States|-5|0.08| +16|AAAAAAAAABAAAAAA|1997-08-16|1999-08-16|site_2|2450679|2448854|Unknown|Herbert Hawes|6|Hard new suggestions might contain nat|Basic others will not appreciate so late, potential chil|Albert Leung|6|cally|841|West Cedar|Blvd|Suite D|Midway|Williamson County|TN|31904|United States|-5|0.04| +17|AAAAAAAAABAAAAAA|1999-08-17|2001-08-15|site_2|2450679|2448854|Unknown|Charles Castillo|5|New, different roads know standa|Basic others will not appreciate so late, potential chil|Keith Frazier|6|cally|841|Wilson Ridge|Way|Suite 20|Midway|Williamson County|TN|31904|United States|-5|0.04| +18|AAAAAAAAABAAAAAA|2001-08-16||site_2|2450679|2448854|Unknown|Charles Castillo|5|Broad, new groups show car|Basic others will not appreciate so late, potential chil|Keith Frazier|6|cally|841|Sycamore |Wy|Suite 450|Fairview|Williamson County|TN|31904|United States|-5|0.01| +19|AAAAAAAADBAAAAAA|1997-08-16||site_3|2450654||Unknown|Jimmy Pope|2|Leaders make as yet particular, desperate machi|Home new thoughts must get however thoughts; technical, oth|Eldon Snow|5|anti|184|Hill 5th|Parkway|Suite 280|Midway|Williamson County|TN|31904|United States|-5|0.04| +20|AAAAAAAAEBAAAAAA|1997-08-16|2000-08-15|site_3|2450645|2448820|Unknown|Peter Cassidy|4|Just popular words shall |Formerly long minds deal aside parties. Lef|Kelvin Lynch|3|pri|828|2nd |Wy|Suite J|Midway|Williamson County|TN|31904|United States|-5|0.04| +21|AAAAAAAAEBAAAAAA|2000-08-16||site_3|2450645|2448820|Unknown|Adam Stonge|5|As private emotions carry yet services. Wor|Formerly long minds deal aside parties. Lef|Jarvis Allen|4|pri|828|1st Maple|Blvd|Suite U|Fairview|Williamson County|TN|35709|United States|-5|0.07| +22|AAAAAAAAGBAAAAAA|1997-08-16|1999-08-16|site_3|2450628|2446978|Unknown|Lewis Wolf|5|Severe, useful funds lift; just|Individual negotiations might join badly children. Brown features travel fiercely dramatic k|James Bernard|2|able|292|Elevnth Third|Wy|Suite 30|Midway|Williamson County|TN|31904|United States|-5|0.07| +23|AAAAAAAAGBAAAAAA|1999-08-17|2001-08-15|site_3|2450628|2446978|Unknown|Jason Silva|4|Severe, useful funds lift; just|Individual negotiations might join badly children. Brown features travel fiercely dramatic k|Jeffrey Martin|2|able|292|Railroad |Court|Suite 440|Fairview|Williamson County|TN|31904|United States|-5|0.01| +24|AAAAAAAAGBAAAAAA|2001-08-16||site_3|2450628|2446978|Unknown|John Ward|4|Severe, useful funds lift; just|Individual negotiations might join badly children. Brown features travel fiercely dramatic k|Philip Sampson|2|able|292|Hickory Meadow|ST|Suite 460|Fairview|Williamson County|TN|31904|United States|-5|0.01| +25|AAAAAAAAJBAAAAAA|1997-08-16||site_4|2450603||Unknown|Julio Davis|3|Continuous, social schools shou|Concerned, royal weeks should announce.|Donald Mckinney|1|ought|671|Willow |Drive|Suite T|Fairview|Williamson County|TN|35709|United States|-5|0.01| +26|AAAAAAAAKBAAAAAA|1997-08-16|2000-08-15|site_4|2450594|2446944|Unknown|William Carter|2|Also difficult women relax. Raw solutions exclude|Often well-known police can hand acres. Healthy, sensible problems make more silent recordings|Keith Williams|2|able|892|Cedar North|Drive|Suite 280|Midway|Williamson County|TN|31904|United States|-5|0.07| +27|AAAAAAAAKBAAAAAA|2000-08-16||site_4|2450594|2446944|Unknown|William Carter|2|Also difficult women relax. Raw solutions exclude|Often well-known police can hand acres. Healthy, sensible problems make more silent recordings|John Isaacs|5|able|892|Spruce |Wy|Suite 200|Fairview|Williamson County|TN|35709|United States|-5|0.12| +28|AAAAAAAAMBAAAAAA|1997-08-16|1999-08-16|site_4|2450577|2448752|Unknown|Raymond Oneill|4|Hands look again nice years. Alre|Skills can want as common agencies. Appropriate issues can use. Over national lives may lear|Jeffrey Campbell|3|pri|730|Central 14th|Dr.|Suite Q|Midway|Williamson County|TN|31904|United States|-5|0.09| +29|AAAAAAAAMBAAAAAA|1999-08-17|2001-08-15|site_4|2450577|2448752|Unknown|Raymond Oneill|4|Well-known, patient proposals m|Skills can want as common agencies. Appropriate issues can use. Over national lives may lear|Jeffrey Campbell|3|pri|247|Park |Boulevard|Suite 100|Midway|Williamson County|TN|31904|United States|-5|0.12| +30|AAAAAAAAMBAAAAAA|2001-08-16||site_4|2450577|2448752|Unknown|Raymond Oneill|4|Well-known, patient proposals m|Skills can want as common agencies. Appropriate issues can use. Over national lives may lear|Jeffrey Campbell|1|pri|247|South Third|Wy|Suite 210|Midway|Williamson County|TN|31904|United States|-5|0.05| diff --git a/spark-connector/spark-common/src/test/resources/spark-test.conf b/spark-connector/spark-common/src/test/resources/spark-test.conf new file mode 100644 index 00000000000..1542cd50670 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/spark-test.conf @@ -0,0 +1,29 @@ +# +# 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. +# + +# The Spark SQL test base dir +#gravitino.spark.test.dir = xx +# Gravitio metalake name +gravitino.spark.test.metalake = test +# Specify the test SQLs, using directory to specify group of SQLs like `test-sqls/hive`, using file path to specify one SQL like `test-sqls/hive/basic.sql`, use `,` to split multi part +gravitino.spark.test.sqls = test-sqls/lakehouse-iceberg, test-sqls/hive, test-sqls/tpcds +# Whether generate golden files +gravitino.spark.test.generateGoldenFiles = false +# Whether to setup Gravitino and Hive environment +gravitino.spark.test.setupEnv = true diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/hive/basic.sql b/spark-connector/spark-common/src/test/resources/test-sqls/hive/basic.sql new file mode 100644 index 00000000000..a6ab8d3e7c5 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/hive/basic.sql @@ -0,0 +1,17 @@ + +-- If not specify location explicitly, the location will be hdfs://127.0.0.1:9000/xx, which is unaccessable for spark +CREATE DATABASE IF NOT EXISTS t_hive LOCATION '/user/hive/warehouse-spark-test/t_hive'; +USE t_hive; + +CREATE TABLE IF NOT EXISTS employees ( + id INT, + name STRING, + age INT +) +PARTITIONED BY (department STRING) +STORED AS PARQUET; + +INSERT OVERWRITE TABLE employees PARTITION(department='Engineering') VALUES (1, 'John Doe', 30), (2, 'Jane Smith', 28); +INSERT OVERWRITE TABLE employees PARTITION(department='Marketing') VALUES (3, 'Mike Brown', 32); + +SELECT * FROM employees WHERE department = 'Engineering'; diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/hive/basic.sql.out b/spark-connector/spark-common/src/test/resources/test-sqls/hive/basic.sql.out new file mode 100644 index 00000000000..789a1df9d29 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/hive/basic.sql.out @@ -0,0 +1,54 @@ +-- Automatically generated by Gravitino Spark SQL test +-- !query +CREATE DATABASE IF NOT EXISTS t_hive LOCATION '/user/hive/warehouse-spark-test/t_hive' +-- !query schema +struct<> +-- !query output + + + +-- !query +USE t_hive +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE IF NOT EXISTS employees ( + id INT, + name STRING, + age INT +) +PARTITIONED BY (department STRING) +STORED AS PARQUET +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT OVERWRITE TABLE employees PARTITION(department='Engineering') VALUES (1, 'John Doe', 30), (2, 'Jane Smith', 28) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT OVERWRITE TABLE employees PARTITION(department='Marketing') VALUES (3, 'Mike Brown', 32) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM employees WHERE department = 'Engineering' +-- !query schema +struct +-- !query output +1 John Doe 30 Engineering +2 Jane Smith 28 Engineering diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/hive/group-by.sql b/spark-connector/spark-common/src/test/resources/test-sqls/hive/group-by.sql new file mode 100644 index 00000000000..291a8478c7a --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/hive/group-by.sql @@ -0,0 +1,297 @@ +-- Test aggregate operator with codegen on and off. +--CONFIG_DIM1 spark.sql.codegen.wholeStage=true +--CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=CODEGEN_ONLY +--CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=NO_CODEGEN + +-- Test data. +CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES +(1, 1), (1, 2), (2, 1), (2, 2), (3, 1), (3, 2), (null, 1), (3, null), (null, null) +AS testData(a, b); +CREATE OR REPLACE TEMPORARY VIEW testRegression AS SELECT * FROM VALUES +(1, 10, null), (2, 10, 11), (2, 20, 22), (2, 25, null), (2, 30, 35) +AS testRegression(k, y, x); +CREATE OR REPLACE TEMPORARY VIEW aggr AS SELECT * FROM VALUES +(0, 0), (0, 10), (0, 20), (0, 30), (0, 40), (1, 10), (1, 20), (2, 10), (2, 20), (2, 25), (2, 30), (3, 60), (4, null) +AS aggr(k, v); + +-- Aggregate with empty GroupBy expressions. +SELECT a, COUNT(b) FROM testData; +SELECT COUNT(a), COUNT(b) FROM testData; + +-- Aggregate with non-empty GroupBy expressions. +SELECT a, COUNT(b) FROM testData GROUP BY a; +SELECT a, COUNT(b) FROM testData GROUP BY b; +SELECT COUNT(a), COUNT(b) FROM testData GROUP BY a; + +-- Aggregate grouped by literals. +SELECT 'foo', COUNT(a) FROM testData GROUP BY 1; + +-- Aggregate grouped by literals (whole stage code generation). +SELECT 'foo' FROM testData WHERE a = 0 GROUP BY 1; + +-- Aggregate grouped by literals (hash aggregate). +SELECT 'foo', APPROX_COUNT_DISTINCT(a) FROM testData WHERE a = 0 GROUP BY 1; + +-- Aggregate grouped by literals (sort aggregate). +SELECT 'foo', MAX(STRUCT(a)) FROM testData WHERE a = 0 GROUP BY 1; + +-- Aggregate with complex GroupBy expressions. +SELECT a + b, COUNT(b) FROM testData GROUP BY a + b; +SELECT a + 2, COUNT(b) FROM testData GROUP BY a + 1; +SELECT a + 1 + 1, COUNT(b) FROM testData GROUP BY a + 1; + +-- Aggregate with nulls. +SELECT SKEWNESS(a), KURTOSIS(a), MIN(a), MAX(a), AVG(a), VARIANCE(a), STDDEV(a), SUM(a), COUNT(a) +FROM testData; + +-- Aggregate with foldable input and multiple distinct groups. +SELECT COUNT(DISTINCT b), COUNT(DISTINCT b, c) FROM (SELECT 1 AS a, 2 AS b, 3 AS c) GROUP BY a; + +-- Aliases in SELECT could be used in GROUP BY +SELECT a AS k, COUNT(b) FROM testData GROUP BY k; +SELECT a AS k, COUNT(b) FROM testData GROUP BY k HAVING k > 1; + +-- GROUP BY alias with invalid col in SELECT list +SELECT a AS k, COUNT(non_existing) FROM testData GROUP BY k; + +-- Aggregate functions cannot be used in GROUP BY +SELECT COUNT(b) AS k FROM testData GROUP BY k; + +-- Test data. +CREATE OR REPLACE TEMPORARY VIEW testDataHasSameNameWithAlias AS SELECT * FROM VALUES +(1, 1, 3), (1, 2, 1) AS testDataHasSameNameWithAlias(k, a, v); +SELECT k AS a, COUNT(v) FROM testDataHasSameNameWithAlias GROUP BY a; + +-- turn off group by aliases +set spark.sql.groupByAliases=false; + +-- Check analysis exceptions +SELECT a AS k, COUNT(b) FROM testData GROUP BY k; + +-- Aggregate with empty input and non-empty GroupBy expressions. +SELECT a, COUNT(1) FROM testData WHERE false GROUP BY a; + +-- Aggregate with empty input and empty GroupBy expressions. +SELECT COUNT(1) FROM testData WHERE false; +SELECT 1 FROM (SELECT COUNT(1) FROM testData WHERE false) t; + +-- Aggregate with empty GroupBy expressions and filter on top +SELECT 1 from ( + SELECT 1 AS z, + MIN(a.x) + FROM (select 1 as x) a + WHERE false +) b +where b.z != b.z; + +-- SPARK-24369 multiple distinct aggregations having the same argument set +SELECT corr(DISTINCT x, y), corr(DISTINCT y, x), count(*) + FROM (VALUES (1, 1), (2, 2), (2, 2)) t(x, y); + +-- SPARK-25708 HAVING without GROUP BY means global aggregate +SELECT 1 FROM range(10) HAVING true; + +SELECT 1 FROM range(10) HAVING MAX(id) > 0; + +SELECT id FROM range(10) HAVING id > 0; + +SET spark.sql.legacy.parser.havingWithoutGroupByAsWhere=true; + +SELECT 1 FROM range(10) HAVING true; + +SELECT 1 FROM range(10) HAVING MAX(id) > 0; + +SELECT id FROM range(10) HAVING id > 0; + +SET spark.sql.legacy.parser.havingWithoutGroupByAsWhere=false; + +-- Test data +CREATE OR REPLACE TEMPORARY VIEW test_agg AS SELECT * FROM VALUES + (1, true), (1, false), + (2, true), + (3, false), (3, null), + (4, null), (4, null), + (5, null), (5, true), (5, false) AS test_agg(k, v); + +-- empty table +SELECT every(v), some(v), any(v), bool_and(v), bool_or(v) FROM test_agg WHERE 1 = 0; + +-- all null values +SELECT every(v), some(v), any(v), bool_and(v), bool_or(v) FROM test_agg WHERE k = 4; + +-- aggregates are null Filtering +SELECT every(v), some(v), any(v), bool_and(v), bool_or(v) FROM test_agg WHERE k = 5; + +-- group by +SELECT k, every(v), some(v), any(v), bool_and(v), bool_or(v) FROM test_agg GROUP BY k; + +-- having +SELECT k, every(v) FROM test_agg GROUP BY k HAVING every(v) = false; +SELECT k, every(v) FROM test_agg GROUP BY k HAVING every(v) IS NULL; + +-- basic subquery path to make sure rewrite happens in both parent and child plans. +SELECT k, + Every(v) AS every +FROM test_agg +WHERE k = 2 + AND v IN (SELECT Any(v) + FROM test_agg + WHERE k = 1) +GROUP BY k; + +-- basic subquery path to make sure rewrite happens in both parent and child plans. +SELECT k, + Every(v) AS every +FROM test_agg +WHERE k = 2 + AND v IN (SELECT Every(v) + FROM test_agg + WHERE k = 1) +GROUP BY k; + +-- input type checking Int +SELECT every(1); + +-- input type checking Short +SELECT some(1S); + +-- input type checking Long +SELECT any(1L); + +-- input type checking String +SELECT every("true"); + +-- input type checking Decimal +SELECT bool_and(1.0); + +-- input type checking double +SELECT bool_or(1.0D); + +-- every/some/any aggregates/bool_and/bool_or are supported as windows expression. +SELECT k, v, every(v) OVER (PARTITION BY k ORDER BY v) FROM test_agg; +SELECT k, v, some(v) OVER (PARTITION BY k ORDER BY v) FROM test_agg; +SELECT k, v, any(v) OVER (PARTITION BY k ORDER BY v) FROM test_agg; +SELECT k, v, bool_and(v) OVER (PARTITION BY k ORDER BY v) FROM test_agg; +SELECT k, v, bool_or(v) OVER (PARTITION BY k ORDER BY v) FROM test_agg; + +-- Having referencing aggregate expressions is ok. +SELECT count(*) FROM test_agg HAVING count(*) > 1L; +SELECT k, max(v) FROM test_agg GROUP BY k HAVING max(v) = true; + +-- Aggrgate expressions can be referenced through an alias +SELECT * FROM (SELECT COUNT(*) AS cnt FROM test_agg) WHERE cnt > 1L; + +-- Error when aggregate expressions are in where clause directly +SELECT count(*) FROM test_agg WHERE count(*) > 1L; +SELECT count(*) FROM test_agg WHERE count(*) + 1L > 1L; +SELECT count(*) FROM test_agg WHERE k = 1 or k = 2 or count(*) + 1L > 1L or max(k) > 1; + +-- Aggregate with multiple distinct decimal columns +SELECT AVG(DISTINCT decimal_col), SUM(DISTINCT decimal_col) FROM VALUES (CAST(1 AS DECIMAL(9, 0))) t(decimal_col); + +-- SPARK-34581: Don't optimize out grouping expressions from aggregate expressions without aggregate function +SELECT not(a IS NULL), count(*) AS c +FROM testData +GROUP BY a IS NULL; + +SELECT if(not(a IS NULL), rand(0), 1), count(*) AS c +FROM testData +GROUP BY a IS NULL; + + +-- Histogram aggregates with different numeric input types +SELECT + histogram_numeric(col, 2) as histogram_2, + histogram_numeric(col, 3) as histogram_3, + histogram_numeric(col, 5) as histogram_5, + histogram_numeric(col, 10) as histogram_10 +FROM VALUES + (1), (2), (3), (4), (5), (6), (7), (8), (9), (10), + (11), (12), (13), (14), (15), (16), (17), (18), (19), (20), + (21), (22), (23), (24), (25), (26), (27), (28), (29), (30), + (31), (32), (33), (34), (35), (3), (37), (38), (39), (40), + (41), (42), (43), (44), (45), (46), (47), (48), (49), (50) AS tab(col); +SELECT histogram_numeric(col, 3) FROM VALUES (1), (2), (3) AS tab(col); +SELECT histogram_numeric(col, 3) FROM VALUES (1L), (2L), (3L) AS tab(col); +SELECT histogram_numeric(col, 3) FROM VALUES (1F), (2F), (3F) AS tab(col); +SELECT histogram_numeric(col, 3) FROM VALUES (1D), (2D), (3D) AS tab(col); +SELECT histogram_numeric(col, 3) FROM VALUES (1S), (2S), (3S) AS tab(col); +SELECT histogram_numeric(col, 3) FROM VALUES + (CAST(1 AS BYTE)), (CAST(2 AS BYTE)), (CAST(3 AS BYTE)) AS tab(col); +SELECT histogram_numeric(col, 3) FROM VALUES + (CAST(1 AS TINYINT)), (CAST(2 AS TINYINT)), (CAST(3 AS TINYINT)) AS tab(col); +SELECT histogram_numeric(col, 3) FROM VALUES + (CAST(1 AS SMALLINT)), (CAST(2 AS SMALLINT)), (CAST(3 AS SMALLINT)) AS tab(col); +SELECT histogram_numeric(col, 3) FROM VALUES + (CAST(1 AS BIGINT)), (CAST(2 AS BIGINT)), (CAST(3 AS BIGINT)) AS tab(col); +SELECT histogram_numeric(col, 3) FROM VALUES (TIMESTAMP '2017-03-01 00:00:00'), + (TIMESTAMP '2017-04-01 00:00:00'), (TIMESTAMP '2017-05-01 00:00:00') AS tab(col); +SELECT histogram_numeric(col, 3) FROM VALUES (INTERVAL '100-00' YEAR TO MONTH), + (INTERVAL '110-00' YEAR TO MONTH), (INTERVAL '120-00' YEAR TO MONTH) AS tab(col); +SELECT histogram_numeric(col, 3) FROM VALUES (INTERVAL '12 20:4:0' DAY TO SECOND), + (INTERVAL '12 21:4:0' DAY TO SECOND), (INTERVAL '12 22:4:0' DAY TO SECOND) AS tab(col); +SELECT histogram_numeric(col, 3) +FROM VALUES (NULL), (NULL), (NULL) AS tab(col); +SELECT histogram_numeric(col, 3) +FROM VALUES (CAST(NULL AS DOUBLE)), (CAST(NULL AS DOUBLE)), (CAST(NULL AS DOUBLE)) AS tab(col); +SELECT histogram_numeric(col, 3) +FROM VALUES (CAST(NULL AS INT)), (CAST(NULL AS INT)), (CAST(NULL AS INT)) AS tab(col); + + +-- SPARK-37613: Support ANSI Aggregate Function: regr_count +SELECT regr_count(y, x) FROM testRegression; +SELECT regr_count(y, x) FROM testRegression WHERE x IS NOT NULL; +SELECT k, count(*), regr_count(y, x) FROM testRegression GROUP BY k; +SELECT k, count(*) FILTER (WHERE x IS NOT NULL), regr_count(y, x) FROM testRegression GROUP BY k; + +-- SPARK-37613: Support ANSI Aggregate Function: regr_r2 +SELECT regr_r2(y, x) FROM testRegression; +SELECT regr_r2(y, x) FROM testRegression WHERE x IS NOT NULL; +SELECT k, corr(y, x), regr_r2(y, x) FROM testRegression GROUP BY k; +SELECT k, corr(y, x) FILTER (WHERE x IS NOT NULL), regr_r2(y, x) FROM testRegression GROUP BY k; + +-- SPARK-27974: Support ANSI Aggregate Function: array_agg +SELECT + collect_list(col), + array_agg(col) +FROM VALUES + (1), (2), (1) AS tab(col); +SELECT + a, + collect_list(b), + array_agg(b) +FROM VALUES + (1,4),(2,3),(1,4),(2,4) AS v(a,b) +GROUP BY a; + +-- SPARK-37614: Support ANSI Aggregate Function: regr_avgx & regr_avgy +SELECT regr_avgx(y, x), regr_avgy(y, x) FROM testRegression; +SELECT regr_avgx(y, x), regr_avgy(y, x) FROM testRegression WHERE x IS NOT NULL AND y IS NOT NULL; +SELECT k, avg(x), avg(y), regr_avgx(y, x), regr_avgy(y, x) FROM testRegression GROUP BY k; +SELECT k, avg(x) FILTER (WHERE x IS NOT NULL AND y IS NOT NULL), avg(y) FILTER (WHERE x IS NOT NULL AND y IS NOT NULL), regr_avgx(y, x), regr_avgy(y, x) FROM testRegression GROUP BY k; + +-- SPARK-37676: Support ANSI Aggregation Function: percentile_cont +SELECT + percentile_cont(0.25) WITHIN GROUP (ORDER BY v), + percentile_cont(0.25) WITHIN GROUP (ORDER BY v DESC) +FROM aggr; +SELECT + k, + percentile_cont(0.25) WITHIN GROUP (ORDER BY v), + percentile_cont(0.25) WITHIN GROUP (ORDER BY v DESC) +FROM aggr +GROUP BY k +ORDER BY k; + +-- SPARK-37691: Support ANSI Aggregation Function: percentile_disc +SELECT + percentile_disc(0.25) WITHIN GROUP (ORDER BY v), + percentile_disc(0.25) WITHIN GROUP (ORDER BY v DESC) +FROM aggr; +SELECT + k, + percentile_disc(0.25) WITHIN GROUP (ORDER BY v), + percentile_disc(0.25) WITHIN GROUP (ORDER BY v DESC) +FROM aggr +GROUP BY k +ORDER BY k; diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/hive/group-by.sql.out b/spark-connector/spark-common/src/test/resources/test-sqls/hive/group-by.sql.out new file mode 100644 index 00000000000..31a270a3fcf --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/hive/group-by.sql.out @@ -0,0 +1,997 @@ +-- Automatically generated by Gravitino Spark SQL test +-- !query +CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES +(1, 1), (1, 2), (2, 1), (2, 2), (3, 1), (3, 2), (null, 1), (3, null), (null, null) +AS testData(a, b) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW testRegression AS SELECT * FROM VALUES +(1, 10, null), (2, 10, 11), (2, 20, 22), (2, 25, null), (2, 30, 35) +AS testRegression(k, y, x) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW aggr AS SELECT * FROM VALUES +(0, 0), (0, 10), (0, 20), (0, 30), (0, 40), (1, 10), (1, 20), (2, 10), (2, 20), (2, 25), (2, 30), (3, 60), (4, null) +AS aggr(k, v) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT a, COUNT(b) FROM testData +-- !query schema +struct<> +-- !query output +[SPARK_EXCEPTION] + + +-- !query +SELECT COUNT(a), COUNT(b) FROM testData +-- !query schema +struct +-- !query output +7 7 + + +-- !query +SELECT a, COUNT(b) FROM testData GROUP BY a +-- !query schema +struct +-- !query output +1 2 +2 2 +3 2 +NULL 1 + + +-- !query +SELECT a, COUNT(b) FROM testData GROUP BY b +-- !query schema +struct<> +-- !query output +[SPARK_EXCEPTION] + + +-- !query +SELECT COUNT(a), COUNT(b) FROM testData GROUP BY a +-- !query schema +struct +-- !query output +0 1 +2 2 +2 2 +3 2 + + +-- !query +SELECT 'foo', COUNT(a) FROM testData GROUP BY 1 +-- !query schema +struct +-- !query output +foo 7 + + +-- !query +SELECT 'foo' FROM testData WHERE a = 0 GROUP BY 1 +-- !query schema +struct +-- !query output + + + +-- !query +SELECT 'foo', APPROX_COUNT_DISTINCT(a) FROM testData WHERE a = 0 GROUP BY 1 +-- !query schema +struct +-- !query output + + + +-- !query +SELECT 'foo', MAX(STRUCT(a)) FROM testData WHERE a = 0 GROUP BY 1 +-- !query schema +struct> +-- !query output + + + +-- !query +SELECT a + b, COUNT(b) FROM testData GROUP BY a + b +-- !query schema +struct<(a + b):int,count(b):bigint> +-- !query output +2 1 +3 2 +4 2 +5 1 +NULL 1 + + +-- !query +SELECT a + 2, COUNT(b) FROM testData GROUP BY a + 1 +-- !query schema +struct<> +-- !query output +[SPARK_EXCEPTION] + + +-- !query +SELECT a + 1 + 1, COUNT(b) FROM testData GROUP BY a + 1 +-- !query schema +struct<((a + 1) + 1):int,count(b):bigint> +-- !query output +3 2 +4 2 +5 2 +NULL 1 + + +-- !query +SELECT SKEWNESS(a), KURTOSIS(a), MIN(a), MAX(a), AVG(a), VARIANCE(a), STDDEV(a), SUM(a), COUNT(a) +FROM testData +-- !query schema +struct +-- !query output +-0.27238010581457267 -1.506920415224914 1 3 2.142857142857143 0.8095238095238096 0.8997354108424375 15 7 + + +-- !query +SELECT COUNT(DISTINCT b), COUNT(DISTINCT b, c) FROM (SELECT 1 AS a, 2 AS b, 3 AS c) GROUP BY a +-- !query schema +struct +-- !query output +1 1 + + +-- !query +SELECT a AS k, COUNT(b) FROM testData GROUP BY k +-- !query schema +struct +-- !query output +1 2 +2 2 +3 2 +NULL 1 + + +-- !query +SELECT a AS k, COUNT(b) FROM testData GROUP BY k HAVING k > 1 +-- !query schema +struct +-- !query output +2 2 +3 2 + + +-- !query +SELECT a AS k, COUNT(non_existing) FROM testData GROUP BY k +-- !query schema +struct<> +-- !query output +[SPARK_EXCEPTION] + + +-- !query +SELECT COUNT(b) AS k FROM testData GROUP BY k +-- !query schema +struct<> +-- !query output +[SPARK_EXCEPTION] + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW testDataHasSameNameWithAlias AS SELECT * FROM VALUES +(1, 1, 3), (1, 2, 1) AS testDataHasSameNameWithAlias(k, a, v) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT k AS a, COUNT(v) FROM testDataHasSameNameWithAlias GROUP BY a +-- !query schema +struct<> +-- !query output +[SPARK_EXCEPTION] + + +-- !query +set spark.sql.groupByAliases=false +-- !query schema +struct +-- !query output +spark.sql.groupByAliases false + + +-- !query +SELECT a AS k, COUNT(b) FROM testData GROUP BY k +-- !query schema +struct<> +-- !query output +[SPARK_EXCEPTION] + + +-- !query +SELECT a, COUNT(1) FROM testData WHERE false GROUP BY a +-- !query schema +struct +-- !query output + + + +-- !query +SELECT COUNT(1) FROM testData WHERE false +-- !query schema +struct +-- !query output +0 + + +-- !query +SELECT 1 FROM (SELECT COUNT(1) FROM testData WHERE false) t +-- !query schema +struct<1:int> +-- !query output +1 + + +-- !query +SELECT 1 from ( + SELECT 1 AS z, + MIN(a.x) + FROM (select 1 as x) a + WHERE false +) b +where b.z != b.z +-- !query schema +struct<1:int> +-- !query output + + + +-- !query +SELECT corr(DISTINCT x, y), corr(DISTINCT y, x), count(*) + FROM (VALUES (1, 1), (2, 2), (2, 2)) t(x, y) +-- !query schema +struct +-- !query output +1.0 1.0 3 + + +-- !query +SELECT 1 FROM range(10) HAVING true +-- !query schema +struct<1:int> +-- !query output +1 + + +-- !query +SELECT 1 FROM range(10) HAVING MAX(id) > 0 +-- !query schema +struct<1:int> +-- !query output +1 + + +-- !query +SELECT id FROM range(10) HAVING id > 0 +-- !query schema +struct<> +-- !query output +[SPARK_EXCEPTION] + + +-- !query +SET spark.sql.legacy.parser.havingWithoutGroupByAsWhere=true +-- !query schema +struct +-- !query output +spark.sql.legacy.parser.havingWithoutGroupByAsWhere true + + +-- !query +SELECT 1 FROM range(10) HAVING true +-- !query schema +struct<1:int> +-- !query output +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 + + +-- !query +SELECT 1 FROM range(10) HAVING MAX(id) > 0 +-- !query schema +struct<> +-- !query output +[SPARK_EXCEPTION] + + +-- !query +SELECT id FROM range(10) HAVING id > 0 +-- !query schema +struct +-- !query output +1 +2 +3 +4 +5 +6 +7 +8 +9 + + +-- !query +SET spark.sql.legacy.parser.havingWithoutGroupByAsWhere=false +-- !query schema +struct +-- !query output +spark.sql.legacy.parser.havingWithoutGroupByAsWhere false + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW test_agg AS SELECT * FROM VALUES + (1, true), (1, false), + (2, true), + (3, false), (3, null), + (4, null), (4, null), + (5, null), (5, true), (5, false) AS test_agg(k, v) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT every(v), some(v), any(v), bool_and(v), bool_or(v) FROM test_agg WHERE 1 = 0 +-- !query schema +struct +-- !query output +NULL NULL NULL NULL NULL + + +-- !query +SELECT every(v), some(v), any(v), bool_and(v), bool_or(v) FROM test_agg WHERE k = 4 +-- !query schema +struct +-- !query output +NULL NULL NULL NULL NULL + + +-- !query +SELECT every(v), some(v), any(v), bool_and(v), bool_or(v) FROM test_agg WHERE k = 5 +-- !query schema +struct +-- !query output +false true true false true + + +-- !query +SELECT k, every(v), some(v), any(v), bool_and(v), bool_or(v) FROM test_agg GROUP BY k +-- !query schema +struct +-- !query output +1 false true true false true +2 true true true true true +3 false false false false false +4 NULL NULL NULL NULL NULL +5 false true true false true + + +-- !query +SELECT k, every(v) FROM test_agg GROUP BY k HAVING every(v) = false +-- !query schema +struct +-- !query output +1 false +3 false +5 false + + +-- !query +SELECT k, every(v) FROM test_agg GROUP BY k HAVING every(v) IS NULL +-- !query schema +struct +-- !query output +4 NULL + + +-- !query +SELECT k, + Every(v) AS every +FROM test_agg +WHERE k = 2 + AND v IN (SELECT Any(v) + FROM test_agg + WHERE k = 1) +GROUP BY k +-- !query schema +struct +-- !query output +2 true + + +-- !query +SELECT k, + Every(v) AS every +FROM test_agg +WHERE k = 2 + AND v IN (SELECT Every(v) + FROM test_agg + WHERE k = 1) +GROUP BY k +-- !query schema +struct +-- !query output + + + +-- !query +SELECT every(1) +-- !query schema +struct<> +-- !query output +[SPARK_EXCEPTION] + + +-- !query +SELECT some(1S) +-- !query schema +struct<> +-- !query output +[SPARK_EXCEPTION] + + +-- !query +SELECT any(1L) +-- !query schema +struct<> +-- !query output +[SPARK_EXCEPTION] + + +-- !query +SELECT every("true") +-- !query schema +struct<> +-- !query output +[SPARK_EXCEPTION] + + +-- !query +SELECT bool_and(1.0) +-- !query schema +struct<> +-- !query output +[SPARK_EXCEPTION] + + +-- !query +SELECT bool_or(1.0D) +-- !query schema +struct<> +-- !query output +[SPARK_EXCEPTION] + + +-- !query +SELECT k, v, every(v) OVER (PARTITION BY k ORDER BY v) FROM test_agg +-- !query schema +struct +-- !query output +1 false false +1 true false +2 true true +3 NULL NULL +3 false false +4 NULL NULL +4 NULL NULL +5 NULL NULL +5 false false +5 true false + + +-- !query +SELECT k, v, some(v) OVER (PARTITION BY k ORDER BY v) FROM test_agg +-- !query schema +struct +-- !query output +1 false false +1 true true +2 true true +3 NULL NULL +3 false false +4 NULL NULL +4 NULL NULL +5 NULL NULL +5 false false +5 true true + + +-- !query +SELECT k, v, any(v) OVER (PARTITION BY k ORDER BY v) FROM test_agg +-- !query schema +struct +-- !query output +1 false false +1 true true +2 true true +3 NULL NULL +3 false false +4 NULL NULL +4 NULL NULL +5 NULL NULL +5 false false +5 true true + + +-- !query +SELECT k, v, bool_and(v) OVER (PARTITION BY k ORDER BY v) FROM test_agg +-- !query schema +struct +-- !query output +1 false false +1 true false +2 true true +3 NULL NULL +3 false false +4 NULL NULL +4 NULL NULL +5 NULL NULL +5 false false +5 true false + + +-- !query +SELECT k, v, bool_or(v) OVER (PARTITION BY k ORDER BY v) FROM test_agg +-- !query schema +struct +-- !query output +1 false false +1 true true +2 true true +3 NULL NULL +3 false false +4 NULL NULL +4 NULL NULL +5 NULL NULL +5 false false +5 true true + + +-- !query +SELECT count(*) FROM test_agg HAVING count(*) > 1L +-- !query schema +struct +-- !query output +10 + + +-- !query +SELECT k, max(v) FROM test_agg GROUP BY k HAVING max(v) = true +-- !query schema +struct +-- !query output +1 true +2 true +5 true + + +-- !query +SELECT * FROM (SELECT COUNT(*) AS cnt FROM test_agg) WHERE cnt > 1L +-- !query schema +struct +-- !query output +10 + + +-- !query +SELECT count(*) FROM test_agg WHERE count(*) > 1L +-- !query schema +struct<> +-- !query output +[SPARK_EXCEPTION] + + +-- !query +SELECT count(*) FROM test_agg WHERE count(*) + 1L > 1L +-- !query schema +struct<> +-- !query output +[SPARK_EXCEPTION] + + +-- !query +SELECT count(*) FROM test_agg WHERE k = 1 or k = 2 or count(*) + 1L > 1L or max(k) > 1 +-- !query schema +struct<> +-- !query output +[SPARK_EXCEPTION] + + +-- !query +SELECT AVG(DISTINCT decimal_col), SUM(DISTINCT decimal_col) FROM VALUES (CAST(1 AS DECIMAL(9, 0))) t(decimal_col) +-- !query schema +struct +-- !query output +1.0000 1 + + +-- !query +SELECT not(a IS NULL), count(*) AS c +FROM testData +GROUP BY a IS NULL +-- !query schema +struct<(NOT (a IS NULL)):boolean,c:bigint> +-- !query output +false 2 +true 7 + + +-- !query +SELECT if(not(a IS NULL), rand(0), 1), count(*) AS c +FROM testData +GROUP BY a IS NULL +-- !query schema +struct<(IF((NOT (a IS NULL)), rand(0), 1)):double,c:bigint> +-- !query output +0.7604953758285915 7 +1.0 2 + + +-- !query +SELECT + histogram_numeric(col, 2) as histogram_2, + histogram_numeric(col, 3) as histogram_3, + histogram_numeric(col, 5) as histogram_5, + histogram_numeric(col, 10) as histogram_10 +FROM VALUES + (1), (2), (3), (4), (5), (6), (7), (8), (9), (10), + (11), (12), (13), (14), (15), (16), (17), (18), (19), (20), + (21), (22), (23), (24), (25), (26), (27), (28), (29), (30), + (31), (32), (33), (34), (35), (3), (37), (38), (39), (40), + (41), (42), (43), (44), (45), (46), (47), (48), (49), (50) AS tab(col) +-- !query schema +struct>,histogram_3:array>,histogram_5:array>,histogram_10:array>> +-- !query output +[{"x":17,"y":36.0},{"x":43,"y":14.0}] [{"x":9,"y":20.0},{"x":28,"y":17.0},{"x":44,"y":13.0}] [{"x":5,"y":11.0},{"x":14,"y":8.0},{"x":22,"y":7.0},{"x":30,"y":10.0},{"x":43,"y":14.0}] [{"x":3,"y":6.0},{"x":8,"y":6.0},{"x":14,"y":7.0},{"x":20,"y":4.0},{"x":24,"y":3.0},{"x":27,"y":4.0},{"x":31,"y":4.0},{"x":36,"y":4.0},{"x":42,"y":7.0},{"x":48,"y":5.0}] + + +-- !query +SELECT histogram_numeric(col, 3) FROM VALUES (1), (2), (3) AS tab(col) +-- !query schema +struct>> +-- !query output +[{"x":1,"y":1.0},{"x":2,"y":1.0},{"x":3,"y":1.0}] + + +-- !query +SELECT histogram_numeric(col, 3) FROM VALUES (1L), (2L), (3L) AS tab(col) +-- !query schema +struct>> +-- !query output +[{"x":1,"y":1.0},{"x":2,"y":1.0},{"x":3,"y":1.0}] + + +-- !query +SELECT histogram_numeric(col, 3) FROM VALUES (1F), (2F), (3F) AS tab(col) +-- !query schema +struct>> +-- !query output +[{"x":1.0,"y":1.0},{"x":2.0,"y":1.0},{"x":3.0,"y":1.0}] + + +-- !query +SELECT histogram_numeric(col, 3) FROM VALUES (1D), (2D), (3D) AS tab(col) +-- !query schema +struct>> +-- !query output +[{"x":1.0,"y":1.0},{"x":2.0,"y":1.0},{"x":3.0,"y":1.0}] + + +-- !query +SELECT histogram_numeric(col, 3) FROM VALUES (1S), (2S), (3S) AS tab(col) +-- !query schema +struct>> +-- !query output +[{"x":1,"y":1.0},{"x":2,"y":1.0},{"x":3,"y":1.0}] + + +-- !query +SELECT histogram_numeric(col, 3) FROM VALUES + (CAST(1 AS BYTE)), (CAST(2 AS BYTE)), (CAST(3 AS BYTE)) AS tab(col) +-- !query schema +struct>> +-- !query output +[{"x":1,"y":1.0},{"x":2,"y":1.0},{"x":3,"y":1.0}] + + +-- !query +SELECT histogram_numeric(col, 3) FROM VALUES + (CAST(1 AS TINYINT)), (CAST(2 AS TINYINT)), (CAST(3 AS TINYINT)) AS tab(col) +-- !query schema +struct>> +-- !query output +[{"x":1,"y":1.0},{"x":2,"y":1.0},{"x":3,"y":1.0}] + + +-- !query +SELECT histogram_numeric(col, 3) FROM VALUES + (CAST(1 AS SMALLINT)), (CAST(2 AS SMALLINT)), (CAST(3 AS SMALLINT)) AS tab(col) +-- !query schema +struct>> +-- !query output +[{"x":1,"y":1.0},{"x":2,"y":1.0},{"x":3,"y":1.0}] + + +-- !query +SELECT histogram_numeric(col, 3) FROM VALUES + (CAST(1 AS BIGINT)), (CAST(2 AS BIGINT)), (CAST(3 AS BIGINT)) AS tab(col) +-- !query schema +struct>> +-- !query output +[{"x":1,"y":1.0},{"x":2,"y":1.0},{"x":3,"y":1.0}] + + +-- !query +SELECT histogram_numeric(col, 3) FROM VALUES (TIMESTAMP '2017-03-01 00:00:00'), + (TIMESTAMP '2017-04-01 00:00:00'), (TIMESTAMP '2017-05-01 00:00:00') AS tab(col) +-- !query schema +struct>> +-- !query output +[{"x":2017-03-01 00:00:00,"y":1.0},{"x":2017-04-01 00:00:00,"y":1.0},{"x":2017-05-01 00:00:00,"y":1.0}] + + +-- !query +SELECT histogram_numeric(col, 3) FROM VALUES (INTERVAL '100-00' YEAR TO MONTH), + (INTERVAL '110-00' YEAR TO MONTH), (INTERVAL '120-00' YEAR TO MONTH) AS tab(col) +-- !query schema +struct>> +-- !query output +[{"x":100-0,"y":1.0},{"x":110-0,"y":1.0},{"x":120-0,"y":1.0}] + + +-- !query +SELECT histogram_numeric(col, 3) FROM VALUES (INTERVAL '12 20:4:0' DAY TO SECOND), + (INTERVAL '12 21:4:0' DAY TO SECOND), (INTERVAL '12 22:4:0' DAY TO SECOND) AS tab(col) +-- !query schema +struct>> +-- !query output +[{"x":12 20:04:00.000000000,"y":1.0},{"x":12 21:04:00.000000000,"y":1.0},{"x":12 22:04:00.000000000,"y":1.0}] + + +-- !query +SELECT histogram_numeric(col, 3) +FROM VALUES (NULL), (NULL), (NULL) AS tab(col) +-- !query schema +struct>> +-- !query output +NULL + + +-- !query +SELECT histogram_numeric(col, 3) +FROM VALUES (CAST(NULL AS DOUBLE)), (CAST(NULL AS DOUBLE)), (CAST(NULL AS DOUBLE)) AS tab(col) +-- !query schema +struct>> +-- !query output +NULL + + +-- !query +SELECT histogram_numeric(col, 3) +FROM VALUES (CAST(NULL AS INT)), (CAST(NULL AS INT)), (CAST(NULL AS INT)) AS tab(col) +-- !query schema +struct>> +-- !query output +NULL + + +-- !query +SELECT regr_count(y, x) FROM testRegression +-- !query schema +struct +-- !query output +3 + + +-- !query +SELECT regr_count(y, x) FROM testRegression WHERE x IS NOT NULL +-- !query schema +struct +-- !query output +3 + + +-- !query +SELECT k, count(*), regr_count(y, x) FROM testRegression GROUP BY k +-- !query schema +struct +-- !query output +1 1 0 +2 4 3 + + +-- !query +SELECT k, count(*) FILTER (WHERE x IS NOT NULL), regr_count(y, x) FROM testRegression GROUP BY k +-- !query schema +struct +-- !query output +1 0 0 +2 3 3 + + +-- !query +SELECT regr_r2(y, x) FROM testRegression +-- !query schema +struct +-- !query output +0.9976905311778288 + + +-- !query +SELECT regr_r2(y, x) FROM testRegression WHERE x IS NOT NULL +-- !query schema +struct +-- !query output +0.9976905311778288 + + +-- !query +SELECT k, corr(y, x), regr_r2(y, x) FROM testRegression GROUP BY k +-- !query schema +struct +-- !query output +1 NULL NULL +2 0.9988445981121532 0.9976905311778288 + + +-- !query +SELECT k, corr(y, x) FILTER (WHERE x IS NOT NULL), regr_r2(y, x) FROM testRegression GROUP BY k +-- !query schema +struct +-- !query output +1 NULL NULL +2 0.9988445981121532 0.9976905311778288 + + +-- !query +SELECT + collect_list(col), + array_agg(col) +FROM VALUES + (1), (2), (1) AS tab(col) +-- !query schema +struct,collect_list(col):array> +-- !query output +[1,2,1] [1,2,1] + + +-- !query +SELECT + a, + collect_list(b), + array_agg(b) +FROM VALUES + (1,4),(2,3),(1,4),(2,4) AS v(a,b) +GROUP BY a +-- !query schema +struct,collect_list(b):array> +-- !query output +1 [4,4] [4,4] +2 [3,4] [3,4] + + +-- !query +SELECT regr_avgx(y, x), regr_avgy(y, x) FROM testRegression +-- !query schema +struct +-- !query output +22.666666666666668 20.0 + + +-- !query +SELECT regr_avgx(y, x), regr_avgy(y, x) FROM testRegression WHERE x IS NOT NULL AND y IS NOT NULL +-- !query schema +struct +-- !query output +22.666666666666668 20.0 + + +-- !query +SELECT k, avg(x), avg(y), regr_avgx(y, x), regr_avgy(y, x) FROM testRegression GROUP BY k +-- !query schema +struct +-- !query output +1 NULL 10.0 NULL NULL +2 22.666666666666668 21.25 22.666666666666668 20.0 + + +-- !query +SELECT k, avg(x) FILTER (WHERE x IS NOT NULL AND y IS NOT NULL), avg(y) FILTER (WHERE x IS NOT NULL AND y IS NOT NULL), regr_avgx(y, x), regr_avgy(y, x) FROM testRegression GROUP BY k +-- !query schema +struct +-- !query output +1 NULL NULL NULL NULL +2 22.666666666666668 20.0 22.666666666666668 20.0 + + +-- !query +SELECT + percentile_cont(0.25) WITHIN GROUP (ORDER BY v), + percentile_cont(0.25) WITHIN GROUP (ORDER BY v DESC) +FROM aggr +-- !query schema +struct +-- !query output +10.0 30.0 + + +-- !query +SELECT + k, + percentile_cont(0.25) WITHIN GROUP (ORDER BY v), + percentile_cont(0.25) WITHIN GROUP (ORDER BY v DESC) +FROM aggr +GROUP BY k +ORDER BY k +-- !query schema +struct +-- !query output +0 10.0 30.0 +1 12.5 17.5 +2 17.5 26.25 +3 60.0 60.0 +4 NULL NULL + + +-- !query +SELECT + percentile_disc(0.25) WITHIN GROUP (ORDER BY v), + percentile_disc(0.25) WITHIN GROUP (ORDER BY v DESC) +FROM aggr +-- !query schema +struct +-- !query output +10.0 30.0 + + +-- !query +SELECT + k, + percentile_disc(0.25) WITHIN GROUP (ORDER BY v), + percentile_disc(0.25) WITHIN GROUP (ORDER BY v DESC) +FROM aggr +GROUP BY k +ORDER BY k +-- !query schema +struct +-- !query output +0 10.0 30.0 +1 10.0 20.0 +2 10.0 30.0 +3 60.0 60.0 +4 NULL NULL diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/lakehouse-iceberg/basic.sql b/spark-connector/spark-common/src/test/resources/test-sqls/lakehouse-iceberg/basic.sql new file mode 100644 index 00000000000..b646fccfe34 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/lakehouse-iceberg/basic.sql @@ -0,0 +1,39 @@ +CREATE DATABASE IF NOT EXISTS mydatabase; +USE mydatabase; + +DROP TABLE IF EXISTS employee; + +CREATE TABLE IF NOT EXISTS employee ( + id bigint, + name string, + department string, + hire_date timestamp +) USING iceberg +PARTITIONED BY (days(hire_date)); + +INSERT INTO employee +VALUES +(1, 'Alice', 'Engineering', TIMESTAMP '2021-01-01 09:00:00'), +(2, 'Bob', 'Marketing', TIMESTAMP '2021-02-01 10:30:00'), +(3, 'Charlie', 'Sales', TIMESTAMP '2021-03-01 08:45:00'); + +SELECT * FROM employee WHERE date(hire_date) = '2021-01-01'; + +UPDATE employee SET department = 'Jenny' WHERE id = 1; + +DELETE FROM employee WHERE id < 2; + +MERGE INTO employee +USING (SELECT 4 as id, 'David' as name, 'Engineering' as department, TIMESTAMP '2021-04-01 09:00:00' as hire_date) as new_employee +ON employee.id = new_employee.id +WHEN MATCHED THEN UPDATE SET * +WHEN NOT MATCHED THEN INSERT *; + +MERGE INTO employee +USING (SELECT 4 as id, 'David' as name, 'Engineering' as department, TIMESTAMP '2021-04-01 09:00:00' as hire_date) as new_employee +ON employee.id = new_employee.id +WHEN MATCHED THEN DELETE +WHEN NOT MATCHED THEN INSERT *; + +SELECT * FROM employee; + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/lakehouse-iceberg/basic.sql.out b/spark-connector/spark-common/src/test/resources/test-sqls/lakehouse-iceberg/basic.sql.out new file mode 100644 index 00000000000..19ba880c66c --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/lakehouse-iceberg/basic.sql.out @@ -0,0 +1,106 @@ +-- Automatically generated by Gravitino Spark SQL test +-- !query +CREATE DATABASE IF NOT EXISTS mydatabase +-- !query schema +struct<> +-- !query output + + + +-- !query +USE mydatabase +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE IF EXISTS employee +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE IF NOT EXISTS employee ( + id bigint, + name string, + department string, + hire_date timestamp +) USING iceberg +PARTITIONED BY (days(hire_date)) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO employee +VALUES +(1, 'Alice', 'Engineering', TIMESTAMP '2021-01-01 09:00:00'), +(2, 'Bob', 'Marketing', TIMESTAMP '2021-02-01 10:30:00'), +(3, 'Charlie', 'Sales', TIMESTAMP '2021-03-01 08:45:00') +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM employee WHERE date(hire_date) = '2021-01-01' +-- !query schema +struct +-- !query output +1 Alice Engineering 2021-01-01 09:00:00 + + +-- !query +UPDATE employee SET department = 'Jenny' WHERE id = 1 +-- !query schema +struct<> +-- !query output + + + +-- !query +DELETE FROM employee WHERE id < 2 +-- !query schema +struct<> +-- !query output + + + +-- !query +MERGE INTO employee +USING (SELECT 4 as id, 'David' as name, 'Engineering' as department, TIMESTAMP '2021-04-01 09:00:00' as hire_date) as new_employee +ON employee.id = new_employee.id +WHEN MATCHED THEN UPDATE SET * +WHEN NOT MATCHED THEN INSERT * +-- !query schema +struct<> +-- !query output + + + +-- !query +MERGE INTO employee +USING (SELECT 4 as id, 'David' as name, 'Engineering' as department, TIMESTAMP '2021-04-01 09:00:00' as hire_date) as new_employee +ON employee.id = new_employee.id +WHEN MATCHED THEN DELETE +WHEN NOT MATCHED THEN INSERT * +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM employee +-- !query schema +struct +-- !query output +2 Bob Marketing 2021-02-01 10:30:00 +3 Charlie Sales 2021-03-01 08:45:00 diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/lakehouse-iceberg/catalog.sql b/spark-connector/spark-common/src/test/resources/test-sqls/lakehouse-iceberg/catalog.sql new file mode 100644 index 00000000000..2d52d835a16 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/lakehouse-iceberg/catalog.sql @@ -0,0 +1 @@ +select current_catalog(); diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/lakehouse-iceberg/catalog.sql.out b/spark-connector/spark-common/src/test/resources/test-sqls/lakehouse-iceberg/catalog.sql.out new file mode 100644 index 00000000000..f1c191a0b5f --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/lakehouse-iceberg/catalog.sql.out @@ -0,0 +1,7 @@ +-- Automatically generated by Gravitino Spark SQL test +-- !query +select current_catalog() +-- !query schema +struct +-- !query output +iceberg diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/catalog.sql b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/catalog.sql new file mode 100644 index 00000000000..2d52d835a16 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/catalog.sql @@ -0,0 +1 @@ +select current_catalog(); diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/catalog.sql.out b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/catalog.sql.out new file mode 100644 index 00000000000..a95087aafc7 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/catalog.sql.out @@ -0,0 +1,7 @@ +-- Automatically generated by Gravitino Spark SQL test +-- !query +select current_catalog() +-- !query schema +struct +-- !query output +hive diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/prepare.sql b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/prepare.sql new file mode 100644 index 00000000000..08a799beda5 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/prepare.sql @@ -0,0 +1,740 @@ + +CREATE DATABASE IF NOT EXISTS tpcds LOCATION '/user/hive/warehouse-spark-test/tpcds'; + +use tpcds; + +drop table if exists et_store_sales; +create external table IF NOT EXISTS et_store_sales +( + ss_sold_date_sk bigint, + ss_sold_time_sk bigint, + ss_item_sk bigint, + ss_customer_sk bigint, + ss_cdemo_sk bigint, + ss_hdemo_sk bigint, + ss_addr_sk bigint, + ss_store_sk bigint, + ss_promo_sk bigint, + ss_ticket_number bigint, + ss_quantity int, + ss_wholesale_cost decimal(7,2), + ss_list_price decimal(7,2), + ss_sales_price decimal(7,2), + ss_ext_discount_amt decimal(7,2), + ss_ext_sales_price decimal(7,2), + ss_ext_wholesale_cost decimal(7,2), + ss_ext_list_price decimal(7,2), + ss_ext_tax decimal(7,2), + ss_coupon_amt decimal(7,2), + ss_net_paid decimal(7,2), + ss_net_paid_inc_tax decimal(7,2), + ss_net_profit decimal(7,2) +) +row format delimited fields terminated by '|' +location '${spark.gravitino.test.data.dir}/tpcds/store_sales' +tblproperties ('serialization.null.format'='') +; + +drop table if exists et_store_returns; +create external table IF NOT EXISTS et_store_returns +( + sr_returned_date_sk bigint, + sr_return_time_sk bigint, + sr_item_sk bigint, + sr_customer_sk bigint, + sr_cdemo_sk bigint, + sr_hdemo_sk bigint, + sr_addr_sk bigint, + sr_store_sk bigint, + sr_reason_sk bigint, + sr_ticket_number bigint, + sr_return_quantity bigint, + sr_return_amt decimal(7,2), + sr_return_tax decimal(7,2), + sr_return_amt_inc_tax decimal(7,2), + sr_fee decimal(7,2), + sr_return_ship_cost decimal(7,2), + sr_refunded_cash decimal(7,2), + sr_reversed_charge decimal(7,2), + sr_store_credit decimal(7,2), + sr_net_loss decimal(7,2) +) +row format delimited fields terminated by '|' +location '${spark.gravitino.test.data.dir}/tpcds/store_returns' +tblproperties ('serialization.null.format'='') +; + +drop table if exists et_catalog_sales; +create external table IF NOT EXISTS et_catalog_sales +( + cs_sold_date_sk bigint, + cs_sold_time_sk bigint, + cs_ship_date_sk bigint, + cs_bill_customer_sk bigint, + cs_bill_cdemo_sk bigint, + cs_bill_hdemo_sk bigint, + cs_bill_addr_sk bigint, + cs_ship_customer_sk bigint, + cs_ship_cdemo_sk bigint, + cs_ship_hdemo_sk bigint, + cs_ship_addr_sk bigint, + cs_call_center_sk bigint, + cs_catalog_page_sk bigint, + cs_ship_mode_sk bigint, + cs_warehouse_sk bigint, + cs_item_sk bigint, + cs_promo_sk bigint, + cs_order_number bigint, + cs_quantity bigint, + cs_wholesale_cost decimal(7,2), + cs_list_price decimal(7,2), + cs_sales_price decimal(7,2), + cs_ext_discount_amt decimal(7,2), + cs_ext_sales_price decimal(7,2), + cs_ext_wholesale_cost decimal(7,2), + cs_ext_list_price decimal(7,2), + cs_ext_tax decimal(7,2), + cs_coupon_amt decimal(7,2), + cs_ext_ship_cost decimal(7,2), + cs_net_paid decimal(7,2), + cs_net_paid_inc_tax decimal(7,2), + cs_net_paid_inc_ship decimal(7,2), + cs_net_paid_inc_ship_tax decimal(7,2), + cs_net_profit decimal(7,2) +) +row format delimited fields terminated by '|' +location '${spark.gravitino.test.data.dir}/tpcds/catalog_sales' +tblproperties ('serialization.null.format'='') +; + +drop table if exists et_catalog_returns; +create external table IF NOT EXISTS et_catalog_returns +( + cr_returned_date_sk bigint, + cr_returned_time_sk bigint, + cr_item_sk bigint, + cr_ship_date_sk bigint, + cr_refunded_customer_sk bigint, + cr_refunded_cdemo_sk bigint, + cr_refunded_hdemo_sk bigint, + cr_refunded_addr_sk bigint, + cr_returning_customer_sk bigint, + cr_returning_cdemo_sk bigint, + cr_returning_hdemo_sk bigint, + cr_returning_addr_sk bigint, + cr_call_center_sk bigint, + cr_catalog_page_sk bigint, + cr_ship_mode_sk bigint, + cr_warehouse_sk bigint, + cr_reason_sk bigint, + cr_order_number bigint, + cr_return_quantity bigint, + cr_return_amount decimal(7,2), + cr_return_tax decimal(7,2), + cr_return_amt_inc_tax decimal(7,2), + cr_fee decimal(7,2), + cr_return_ship_cost decimal(7,2), + cr_refunded_cash decimal(7,2), + cr_reversed_charge decimal(7,2), + cr_store_credit decimal(7,2), + cr_net_loss decimal(7,2) +) +row format delimited fields terminated by '|' +location '${spark.gravitino.test.data.dir}/tpcds/catalog_returns' +tblproperties ('serialization.null.format'='') +; + +drop table if exists et_web_sales; +create external table IF NOT EXISTS et_web_sales +( + ws_sold_date_sk bigint, + ws_sold_time_sk bigint, + ws_ship_date_sk bigint, + ws_item_sk bigint, + ws_bill_customer_sk bigint, + ws_bill_cdemo_sk bigint, + ws_bill_hdemo_sk bigint, + ws_bill_addr_sk bigint, + ws_ship_customer_sk bigint, + ws_ship_cdemo_sk bigint, + ws_ship_hdemo_sk bigint, + ws_ship_addr_sk bigint, + ws_web_page_sk bigint, + ws_web_site_sk bigint, + ws_ship_mode_sk bigint, + ws_warehouse_sk bigint, + ws_promo_sk bigint, + ws_order_number bigint, + ws_quantity bigint, + ws_wholesale_cost decimal(7,2), + ws_list_price decimal(7,2), + ws_sales_price decimal(7,2), + ws_ext_discount_amt decimal(7,2), + ws_ext_sales_price decimal(7,2), + ws_ext_wholesale_cost decimal(7,2), + ws_ext_list_price decimal(7,2), + ws_ext_tax decimal(7,2), + ws_coupon_amt decimal(7,2), + ws_ext_ship_cost decimal(7,2), + ws_net_paid decimal(7,2), + ws_net_paid_inc_tax decimal(7,2), + ws_net_paid_inc_ship decimal(7,2), + ws_net_paid_inc_ship_tax decimal(7,2), + ws_net_profit decimal(7,2) +) +row format delimited fields terminated by '|' +location '${spark.gravitino.test.data.dir}/tpcds/web_sales' +tblproperties ('serialization.null.format'='') +; + + +drop table if exists et_web_returns; +create external table IF NOT EXISTS et_web_returns +( + wr_returned_date_sk bigint, + wr_returned_time_sk bigint, + wr_item_sk bigint, + wr_refunded_customer_sk bigint, + wr_refunded_cdemo_sk bigint, + wr_refunded_hdemo_sk bigint, + wr_refunded_addr_sk bigint, + wr_returning_customer_sk bigint, + wr_returning_cdemo_sk bigint, + wr_returning_hdemo_sk bigint, + wr_returning_addr_sk bigint, + wr_web_page_sk bigint, + wr_reason_sk bigint, + wr_order_number bigint, + wr_return_quantity bigint, + wr_return_amt decimal(7,2), + wr_return_tax decimal(7,2), + wr_return_amt_inc_tax decimal(7,2), + wr_fee decimal(7,2), + wr_return_ship_cost decimal(7,2), + wr_refunded_cash decimal(7,2), + wr_reversed_charge decimal(7,2), + wr_account_credit decimal(7,2), + wr_net_loss decimal(7,2) +) +row format delimited fields terminated by '|' +location '${spark.gravitino.test.data.dir}/tpcds/web_returns' +tblproperties ('serialization.null.format'='') +; + +drop table if exists et_inventory; +create external table IF NOT EXISTS et_inventory +( + inv_date_sk bigint, + inv_item_sk bigint, + inv_warehouse_sk bigint, + inv_quantity_on_hand bigint +) +row format delimited fields terminated by '|' +location '${spark.gravitino.test.data.dir}/tpcds/inventory' +tblproperties ('serialization.null.format'='') +; + + + +drop table if exists et_store; +create external table IF NOT EXISTS et_store +( + s_store_sk bigint, + s_store_id string, + s_rec_start_date string, + s_rec_end_date string, + s_closed_date_sk bigint, + s_store_name string, + s_number_employees int, + s_floor_space int, + s_hours string, + s_manager string, + s_market_id int, + s_geography_class string, + s_market_desc string, + s_market_manager string, + s_division_id int, + s_division_name string, + s_company_id int, + s_company_name string, + s_street_number string, + s_street_name string, + s_street_type string, + s_suite_number string, + s_city string, + s_county string, + s_state string, + s_zip string, + s_country string, + s_gmt_offset decimal(5,2), + s_tax_precentage decimal(5,2) +) +row format delimited fields terminated by '|' +location '${spark.gravitino.test.data.dir}/tpcds/store' +tblproperties ('serialization.null.format'='') +; + +drop table if exists et_call_center; +create external table IF NOT EXISTS et_call_center +( + cc_call_center_sk bigint, + cc_call_center_id string, + cc_rec_start_date string, + cc_rec_end_date string, + cc_closed_date_sk bigint, + cc_open_date_sk bigint, + cc_name string, + cc_class string, + cc_employees bigint, + cc_sq_ft bigint, + cc_hours string, + cc_manager string, + cc_mkt_id bigint, + cc_mkt_class string, + cc_mkt_desc string, + cc_market_manager string, + cc_division bigint, + cc_division_name string, + cc_company bigint, + cc_company_name string, + cc_street_number string, + cc_street_name string, + cc_street_type string, + cc_suite_number string, + cc_city string, + cc_county string, + cc_state string, + cc_zip string, + cc_country string, + cc_gmt_offset decimal(5,2), + cc_tax_percentage decimal(5,2) +) +row format delimited fields terminated by '|' +location '${spark.gravitino.test.data.dir}/tpcds/call_center' +tblproperties ('serialization.null.format'='') +; + +drop table if exists et_catalog_page; +create external table IF NOT EXISTS et_catalog_page +( + cp_catalog_page_sk bigint, + cp_catalog_page_id string, + cp_promo_id bigint, + cp_start_date_sk bigint, + cp_end_date_sk bigint, + cp_department string, + cp_catalog_number bigint, + cp_catalog_page_number bigint, + cp_description string, + cp_type string +) +row format delimited fields terminated by '|' +location '${spark.gravitino.test.data.dir}/tpcds/catalog_page' +tblproperties ('serialization.null.format'='') +; + +drop table if exists et_web_site; +create external table IF NOT EXISTS et_web_site +( + web_site_sk bigint, + web_site_id string, + web_rec_start_date string, + web_rec_end_date string, + web_name string, + web_open_date_sk bigint, + web_close_date_sk bigint, + web_class string, + web_manager string, + web_mkt_id bigint, + web_mkt_class string, + web_mkt_desc string, + web_market_manager string, + web_company_id bigint, + web_company_name string, + web_street_number string, + web_street_name string, + web_street_type string, + web_suite_number string, + web_city string, + web_county string, + web_state string, + web_zip string, + web_country string, + web_gmt_offset decimal(5,2), + web_tax_percentage decimal(5,2) +) +row format delimited fields terminated by '|' +location '${spark.gravitino.test.data.dir}/tpcds/web_site' +tblproperties ('serialization.null.format'='') +; + +drop table if exists et_web_page; +create external table IF NOT EXISTS et_web_page +( + wp_web_page_sk bigint, + wp_web_page_id string, + wp_rec_start_date string, + wp_rec_end_date string, + wp_creation_date_sk bigint, + wp_access_date_sk bigint, + wp_autogen_flag string, + wp_customer_sk bigint, + wp_url string, + wp_type string, + wp_char_count bigint, + wp_link_count bigint, + wp_image_count bigint, + wp_max_ad_count bigint +) +row format delimited fields terminated by '|' +location '${spark.gravitino.test.data.dir}/tpcds/web_page' +tblproperties ('serialization.null.format'='') +; + + +drop table if exists et_warehouse; +create external table IF NOT EXISTS et_warehouse +( + w_warehouse_sk bigint, + w_warehouse_id string, + w_warehouse_name string, + w_warehouse_sq_ft bigint, + w_street_number string, + w_street_name string, + w_street_type string, + w_suite_number string, + w_city string, + w_county string, + w_state string, + w_zip string, + w_country string, + w_gmt_offset decimal(5,2) +) +row format delimited fields terminated by '|' +location '${spark.gravitino.test.data.dir}/tpcds/warehouse' +tblproperties ('serialization.null.format'='') +; + +drop table if exists et_customer; +create external table IF NOT EXISTS et_customer +( + c_customer_sk bigint, + c_customer_id string, + c_current_cdemo_sk bigint, + c_current_hdemo_sk bigint, + c_current_addr_sk bigint, + c_first_shipto_date_sk bigint, + c_first_sales_date_sk bigint, + c_salutation string, + c_first_name string, + c_last_name string, + c_preferred_cust_flag string, + c_birth_day bigint, + c_birth_month bigint, + c_birth_year bigint, + c_birth_country string, + c_login string, + c_email_address string, + c_last_review_date string +) +row format delimited fields terminated by '|' +location '${spark.gravitino.test.data.dir}/tpcds/customer' +tblproperties ('serialization.null.format'='') +; + +drop table if exists et_customer_address; +create external table IF NOT EXISTS et_customer_address +( + ca_address_sk bigint, + ca_address_id string, + ca_street_number string, + ca_street_name string, + ca_street_type string, + ca_suite_number string, + ca_city string, + ca_county string, + ca_state string, + ca_zip string, + ca_country string, + ca_gmt_offset decimal(5,2), + ca_location_type string +) +row format delimited fields terminated by '|' +location '${spark.gravitino.test.data.dir}/tpcds/customer_address' +tblproperties ('serialization.null.format'='') +; + + +drop table if exists et_customer_demographics; +create external table IF NOT EXISTS et_customer_demographics +( + cd_demo_sk bigint, + cd_gender string, + cd_marital_status string, + cd_education_status string, + cd_purchase_estimate bigint, + cd_credit_rating string, + cd_dep_count bigint, + cd_dep_employed_count bigint, + cd_dep_college_count bigint +) +row format delimited fields terminated by '|' +location '${spark.gravitino.test.data.dir}/tpcds/customer_demographics' +tblproperties ('serialization.null.format'='') +; + + +drop table if exists et_date_dim; +create external table IF NOT EXISTS et_date_dim +( + d_date_sk bigint, + d_date_id string, + d_date string, + d_month_seq bigint, + d_week_seq bigint, + d_quarter_seq bigint, + d_year bigint, + d_dow bigint, + d_moy bigint, + d_dom bigint, + d_qoy bigint, + d_fy_year bigint, + d_fy_quarter_seq bigint, + d_fy_week_seq bigint, + d_day_name string, + d_quarter_name string, + d_holiday string, + d_weekend string, + d_following_holiday string, + d_first_dom bigint, + d_last_dom bigint, + d_same_day_ly bigint, + d_same_day_lq bigint, + d_current_day string, + d_current_week string, + d_current_month string, + d_current_quarter string, + d_current_year string +) +row format delimited fields terminated by '|' +location '${spark.gravitino.test.data.dir}/tpcds/date_dim' +tblproperties ('serialization.null.format'='') +; + + +drop table if exists et_household_demographics; +create external table IF NOT EXISTS et_household_demographics +( + hd_demo_sk bigint, + hd_income_band_sk bigint, + hd_buy_potential string, + hd_dep_count bigint, + hd_vehicle_count bigint +) +row format delimited fields terminated by '|' +location '${spark.gravitino.test.data.dir}/tpcds/household_demographics' +tblproperties ('serialization.null.format'='') +; + + +drop table if exists et_item; +create external table IF NOT EXISTS et_item +( + i_item_sk bigint, + i_item_id string, + i_rec_start_date string, + i_rec_end_date string, + i_item_desc string, + i_current_price string, + i_wholesale_cost string, + i_brand_id bigint, + i_brand string, + i_class_id bigint, + i_class string, + i_category_id bigint, + i_category string, + i_manufact_id bigint, + i_manufact string, + i_size string, + i_formulation string, + i_color string, + i_units string, + i_container string, + i_manager_id bigint, + i_product_name string +) +row format delimited fields terminated by '|' +location '${spark.gravitino.test.data.dir}/tpcds/item' +tblproperties ('serialization.null.format'='') +; + +drop table if exists et_promotion; +create external table IF NOT EXISTS et_promotion +( + p_promo_sk bigint, + p_promo_id string, + p_start_date_sk bigint, + p_end_date_sk bigint, + p_item_sk bigint, + p_cost decimal(15,2) , + p_response_target bigint, + p_promo_name string, + p_channel_dmail string, + p_channel_email string, + p_channel_catalog string, + p_channel_tv string, + p_channel_radio string, + p_channel_press string, + p_channel_event string, + p_channel_demo string, + p_channel_details string, + p_purpose string, + p_discount_active string +) +row format delimited fields terminated by '|' +location '${spark.gravitino.test.data.dir}/tpcds/promotion' +tblproperties ('serialization.null.format'='') +; + + +drop table if exists et_reason; +create external table IF NOT EXISTS et_reason +( + r_reason_sk bigint, + r_reason_id string, + r_reason_desc string +) +row format delimited fields terminated by '|' +location '${spark.gravitino.test.data.dir}/tpcds/reason' +tblproperties ('serialization.null.format'='') +; + +drop table if exists et_ship_mode; +create external table IF NOT EXISTS et_ship_mode +( + sm_ship_mode_sk bigint, + sm_ship_mode_id string, + sm_type string, + sm_code string, + sm_carrier string, + sm_contract string +) +row format delimited fields terminated by '|' +location '${spark.gravitino.test.data.dir}/tpcds/ship_mode' +tblproperties ('serialization.null.format'='') +; + + +drop table if exists et_time_dim; +create external table IF NOT EXISTS et_time_dim +( + t_time_sk bigint, + t_time_id string, + t_time bigint, + t_hour bigint, + t_minute bigint, + t_second bigint, + t_am_pm string, + t_shift string, + t_sub_shift string, + t_meal_time string +) +row format delimited fields terminated by '|' +location '${spark.gravitino.test.data.dir}/tpcds/time_dim' +tblproperties ('serialization.null.format'='') +; + +drop table if exists et_income_band; +create external table IF NOT EXISTS et_income_band +( + ib_income_band_sk bigint, + ib_lower_bound bigint, + ib_upper_bound bigint +) +row format delimited fields terminated by '|' +location '${spark.gravitino.test.data.dir}/tpcds/income_band' +tblproperties ('serialization.null.format'='') +; + +drop table if exists store; +create table store stored as parquet as select * from et_store; + +drop table if exists call_center; +create table call_center stored as parquet as select * from et_call_center; + + +drop table if exists catalog_page; +create table catalog_page stored as parquet as select * from et_catalog_page; + + +drop table if exists web_site; +create table web_site stored as parquet as select * from et_web_site; + +drop table if exists web_page; +create table web_page stored as parquet as select * from et_web_page; + + +drop table if exists warehouse; +create table warehouse stored as parquet as select * from et_warehouse; + +drop table if exists customer; +create table customer stored as parquet as select * from et_customer; + + +drop table if exists customer_address; +create table customer_address stored as parquet as select * from et_customer_address; + +drop table if exists customer_demographics; +create table customer_demographics stored as parquet as select * from et_customer_demographics; + +drop table if exists date_dim; +create table date_dim stored as parquet as select * from et_date_dim; + +drop table if exists household_demographics; +create table household_demographics stored as parquet as select * from et_household_demographics; + +drop table if exists item; +create table item stored as parquet as select * from et_item; + +drop table if exists time_dim; +create table time_dim stored as parquet as select * from et_time_dim; + +drop table if exists income_band; +create table income_band stored as parquet as select * from et_income_band; + +drop table if exists promotion; +create table promotion stored as parquet as select * from et_promotion; + +drop table if exists reason; +create table reason stored as parquet as select * from et_reason; + + +drop table if exists ship_mode; +create table ship_mode stored as parquet as select * from et_ship_mode; + + +drop table if exists store_sales; +create table store_sales stored as parquet as select * from et_store_sales; + +drop table if exists store_returns; +create table store_returns stored as parquet as select * from et_store_returns; + + +drop table if exists catalog_sales; +create table catalog_sales stored as parquet as select * from et_catalog_sales; + + +drop table if exists catalog_returns; +create table catalog_returns stored as parquet as select * from et_catalog_returns; + +drop table if exists web_sales; +create table web_sales stored as parquet as select * from et_web_sales; + +drop table if exists web_returns; +create table web_returns stored as parquet as select * from et_web_returns; + +drop table if exists inventory; +create table inventory stored as parquet as select * from et_inventory; + +-- use spark_catalog; +-- use tpcds3; diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query1.sql b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query1.sql new file mode 100644 index 00000000000..246a598c50e --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query1.sql @@ -0,0 +1,25 @@ +-- start query 1 in stream 0 using template query1.tpl +with customer_total_return as +(select sr_customer_sk as ctr_customer_sk +,sr_store_sk as ctr_store_sk +,sum(SR_FEE) as ctr_total_return +from store_returns +,date_dim +where sr_returned_date_sk = d_date_sk +and d_year =2000 +group by sr_customer_sk +,sr_store_sk) + select c_customer_id +from customer_total_return ctr1 +,store +,customer +where ctr1.ctr_total_return > (select avg(ctr_total_return)*1.2 +from customer_total_return ctr2 +where ctr1.ctr_store_sk = ctr2.ctr_store_sk) +and s_store_sk = ctr1.ctr_store_sk +and s_state = 'TN' +and ctr1.ctr_customer_sk = c_customer_sk +order by c_customer_id +limit 100; + + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query1.sql.out b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query1.sql.out new file mode 100644 index 00000000000..8cb56cd9cec --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query1.sql.out @@ -0,0 +1,42 @@ +-- Automatically generated by Gravitino Spark SQL test +-- !query +with customer_total_return as +(select sr_customer_sk as ctr_customer_sk +,sr_store_sk as ctr_store_sk +,sum(SR_FEE) as ctr_total_return +from store_returns +,date_dim +where sr_returned_date_sk = d_date_sk +and d_year =2000 +group by sr_customer_sk +,sr_store_sk) + select c_customer_id +from customer_total_return ctr1 +,store +,customer +where ctr1.ctr_total_return > (select avg(ctr_total_return)*1.2 +from customer_total_return ctr2 +where ctr1.ctr_store_sk = ctr2.ctr_store_sk) +and s_store_sk = ctr1.ctr_store_sk +and s_state = 'TN' +and ctr1.ctr_customer_sk = c_customer_sk +order by c_customer_id +limit 100 +-- !query schema +struct +-- !query output +AAAAAAAAADCCAAAA +AAAAAAAABHFAAAAA +AAAAAAAABOMAAAAA +AAAAAAAAECCAAAAA +AAAAAAAAEPLBAAAA +AAAAAAAAFNBAAAAA +AAAAAAAAHKCBAAAA +AAAAAAAAIBJAAAAA +AAAAAAAAIDLBAAAA +AAAAAAAAIPIBAAAA +AAAAAAAAJEEBAAAA +AAAAAAAALFFBAAAA +AAAAAAAALMEAAAAA +AAAAAAAALNFBAAAA +AAAAAAAANHDBAAAA diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query10.sql b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query10.sql new file mode 100644 index 00000000000..75aefe31e5a --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query10.sql @@ -0,0 +1,59 @@ +-- start query 1 in stream 0 using template query10.tpl +select + cd_gender, + cd_marital_status, + cd_education_status, + count(*) cnt1, + cd_purchase_estimate, + count(*) cnt2, + cd_credit_rating, + count(*) cnt3, + cd_dep_count, + count(*) cnt4, + cd_dep_employed_count, + count(*) cnt5, + cd_dep_college_count, + count(*) cnt6 + from + customer c,customer_address ca,customer_demographics + where + c.c_current_addr_sk = ca.ca_address_sk and + ca_county in ('Walker County','Richland County','Gaines County','Douglas County','Dona Ana County') and + cd_demo_sk = c.c_current_cdemo_sk and + exists (select * + from store_sales,date_dim + where c.c_customer_sk = ss_customer_sk and + ss_sold_date_sk = d_date_sk and + d_year = 2002 and + d_moy between 4 and 4+3) and + (exists (select * + from web_sales,date_dim + where c.c_customer_sk = ws_bill_customer_sk and + ws_sold_date_sk = d_date_sk and + d_year = 2002 and + d_moy between 4 ANd 4+3) or + exists (select * + from catalog_sales,date_dim + where c.c_customer_sk = cs_ship_customer_sk and + cs_sold_date_sk = d_date_sk and + d_year = 2002 and + d_moy between 4 and 4+3)) + group by cd_gender, + cd_marital_status, + cd_education_status, + cd_purchase_estimate, + cd_credit_rating, + cd_dep_count, + cd_dep_employed_count, + cd_dep_college_count + order by cd_gender, + cd_marital_status, + cd_education_status, + cd_purchase_estimate, + cd_credit_rating, + cd_dep_count, + cd_dep_employed_count, + cd_dep_college_count +limit 100; + + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query10.sql.out b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query10.sql.out new file mode 100644 index 00000000000..fed4f6f8828 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query10.sql.out @@ -0,0 +1,62 @@ +-- Automatically generated by Gravitino Spark SQL test +-- !query +select + cd_gender, + cd_marital_status, + cd_education_status, + count(*) cnt1, + cd_purchase_estimate, + count(*) cnt2, + cd_credit_rating, + count(*) cnt3, + cd_dep_count, + count(*) cnt4, + cd_dep_employed_count, + count(*) cnt5, + cd_dep_college_count, + count(*) cnt6 + from + customer c,customer_address ca,customer_demographics + where + c.c_current_addr_sk = ca.ca_address_sk and + ca_county in ('Walker County','Richland County','Gaines County','Douglas County','Dona Ana County') and + cd_demo_sk = c.c_current_cdemo_sk and + exists (select * + from store_sales,date_dim + where c.c_customer_sk = ss_customer_sk and + ss_sold_date_sk = d_date_sk and + d_year = 2002 and + d_moy between 4 and 4+3) and + (exists (select * + from web_sales,date_dim + where c.c_customer_sk = ws_bill_customer_sk and + ws_sold_date_sk = d_date_sk and + d_year = 2002 and + d_moy between 4 ANd 4+3) or + exists (select * + from catalog_sales,date_dim + where c.c_customer_sk = cs_ship_customer_sk and + cs_sold_date_sk = d_date_sk and + d_year = 2002 and + d_moy between 4 and 4+3)) + group by cd_gender, + cd_marital_status, + cd_education_status, + cd_purchase_estimate, + cd_credit_rating, + cd_dep_count, + cd_dep_employed_count, + cd_dep_college_count + order by cd_gender, + cd_marital_status, + cd_education_status, + cd_purchase_estimate, + cd_credit_rating, + cd_dep_count, + cd_dep_employed_count, + cd_dep_college_count +limit 100 +-- !query schema +struct +-- !query output + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query11.sql b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query11.sql new file mode 100644 index 00000000000..e7115ef3af5 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query11.sql @@ -0,0 +1,81 @@ +-- start query 1 in stream 0 using template query11.tpl +with year_total as ( + select c_customer_id customer_id + ,c_first_name customer_first_name + ,c_last_name customer_last_name + ,c_preferred_cust_flag customer_preferred_cust_flag + ,c_birth_country customer_birth_country + ,c_login customer_login + ,c_email_address customer_email_address + ,d_year dyear + ,sum(ss_ext_list_price-ss_ext_discount_amt) year_total + ,'s' sale_type + from customer + ,store_sales + ,date_dim + where c_customer_sk = ss_customer_sk + and ss_sold_date_sk = d_date_sk + group by c_customer_id + ,c_first_name + ,c_last_name + ,c_preferred_cust_flag + ,c_birth_country + ,c_login + ,c_email_address + ,d_year + union all + select c_customer_id customer_id + ,c_first_name customer_first_name + ,c_last_name customer_last_name + ,c_preferred_cust_flag customer_preferred_cust_flag + ,c_birth_country customer_birth_country + ,c_login customer_login + ,c_email_address customer_email_address + ,d_year dyear + ,sum(ws_ext_list_price-ws_ext_discount_amt) year_total + ,'w' sale_type + from customer + ,web_sales + ,date_dim + where c_customer_sk = ws_bill_customer_sk + and ws_sold_date_sk = d_date_sk + group by c_customer_id + ,c_first_name + ,c_last_name + ,c_preferred_cust_flag + ,c_birth_country + ,c_login + ,c_email_address + ,d_year + ) + select + t_s_secyear.customer_id + ,t_s_secyear.customer_first_name + ,t_s_secyear.customer_last_name + ,t_s_secyear.customer_email_address + from year_total t_s_firstyear + ,year_total t_s_secyear + ,year_total t_w_firstyear + ,year_total t_w_secyear + where t_s_secyear.customer_id = t_s_firstyear.customer_id + and t_s_firstyear.customer_id = t_w_secyear.customer_id + and t_s_firstyear.customer_id = t_w_firstyear.customer_id + and t_s_firstyear.sale_type = 's' + and t_w_firstyear.sale_type = 'w' + and t_s_secyear.sale_type = 's' + and t_w_secyear.sale_type = 'w' + and t_s_firstyear.dyear = 2001 + and t_s_secyear.dyear = 2001+1 + and t_w_firstyear.dyear = 2001 + and t_w_secyear.dyear = 2001+1 + and t_s_firstyear.year_total > 0 + and t_w_firstyear.year_total > 0 + and case when t_w_firstyear.year_total > 0 then t_w_secyear.year_total / t_w_firstyear.year_total else 0.0 end + > case when t_s_firstyear.year_total > 0 then t_s_secyear.year_total / t_s_firstyear.year_total else 0.0 end + order by t_s_secyear.customer_id + ,t_s_secyear.customer_first_name + ,t_s_secyear.customer_last_name + ,t_s_secyear.customer_email_address +limit 100; + + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query11.sql.out b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query11.sql.out new file mode 100644 index 00000000000..1ed6bea7a8e --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query11.sql.out @@ -0,0 +1,84 @@ +-- Automatically generated by Gravitino Spark SQL test +-- !query +with year_total as ( + select c_customer_id customer_id + ,c_first_name customer_first_name + ,c_last_name customer_last_name + ,c_preferred_cust_flag customer_preferred_cust_flag + ,c_birth_country customer_birth_country + ,c_login customer_login + ,c_email_address customer_email_address + ,d_year dyear + ,sum(ss_ext_list_price-ss_ext_discount_amt) year_total + ,'s' sale_type + from customer + ,store_sales + ,date_dim + where c_customer_sk = ss_customer_sk + and ss_sold_date_sk = d_date_sk + group by c_customer_id + ,c_first_name + ,c_last_name + ,c_preferred_cust_flag + ,c_birth_country + ,c_login + ,c_email_address + ,d_year + union all + select c_customer_id customer_id + ,c_first_name customer_first_name + ,c_last_name customer_last_name + ,c_preferred_cust_flag customer_preferred_cust_flag + ,c_birth_country customer_birth_country + ,c_login customer_login + ,c_email_address customer_email_address + ,d_year dyear + ,sum(ws_ext_list_price-ws_ext_discount_amt) year_total + ,'w' sale_type + from customer + ,web_sales + ,date_dim + where c_customer_sk = ws_bill_customer_sk + and ws_sold_date_sk = d_date_sk + group by c_customer_id + ,c_first_name + ,c_last_name + ,c_preferred_cust_flag + ,c_birth_country + ,c_login + ,c_email_address + ,d_year + ) + select + t_s_secyear.customer_id + ,t_s_secyear.customer_first_name + ,t_s_secyear.customer_last_name + ,t_s_secyear.customer_email_address + from year_total t_s_firstyear + ,year_total t_s_secyear + ,year_total t_w_firstyear + ,year_total t_w_secyear + where t_s_secyear.customer_id = t_s_firstyear.customer_id + and t_s_firstyear.customer_id = t_w_secyear.customer_id + and t_s_firstyear.customer_id = t_w_firstyear.customer_id + and t_s_firstyear.sale_type = 's' + and t_w_firstyear.sale_type = 'w' + and t_s_secyear.sale_type = 's' + and t_w_secyear.sale_type = 'w' + and t_s_firstyear.dyear = 2001 + and t_s_secyear.dyear = 2001+1 + and t_w_firstyear.dyear = 2001 + and t_w_secyear.dyear = 2001+1 + and t_s_firstyear.year_total > 0 + and t_w_firstyear.year_total > 0 + and case when t_w_firstyear.year_total > 0 then t_w_secyear.year_total / t_w_firstyear.year_total else 0.0 end + > case when t_s_firstyear.year_total > 0 then t_s_secyear.year_total / t_s_firstyear.year_total else 0.0 end + order by t_s_secyear.customer_id + ,t_s_secyear.customer_first_name + ,t_s_secyear.customer_last_name + ,t_s_secyear.customer_email_address +limit 100 +-- !query schema +struct +-- !query output + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query12.sql b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query12.sql new file mode 100644 index 00000000000..2b77a37a4c5 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query12.sql @@ -0,0 +1,34 @@ +-- start query 1 in stream 0 using template query12.tpl +select i_item_id + ,i_item_desc + ,i_category + ,i_class + ,i_current_price + ,sum(ws_ext_sales_price) as itemrevenue + ,sum(ws_ext_sales_price)*100/sum(sum(ws_ext_sales_price)) over + (partition by i_class) as revenueratio +from + web_sales + ,item + ,date_dim +where + ws_item_sk = i_item_sk + and i_category in ('Jewelry', 'Sports', 'Books') + and ws_sold_date_sk = d_date_sk + and d_date between cast('2001-01-12' as date) + and (cast('2001-01-12' as date) + interval 30 days) +group by + i_item_id + ,i_item_desc + ,i_category + ,i_class + ,i_current_price +order by + i_category + ,i_class + ,i_item_id + ,i_item_desc + ,revenueratio +limit 100; + + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query12.sql.out b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query12.sql.out new file mode 100644 index 00000000000..6925ff83d4e --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query12.sql.out @@ -0,0 +1,46 @@ +-- Automatically generated by Gravitino Spark SQL test +-- !query +select i_item_id + ,i_item_desc + ,i_category + ,i_class + ,i_current_price + ,sum(ws_ext_sales_price) as itemrevenue + ,sum(ws_ext_sales_price)*100/sum(sum(ws_ext_sales_price)) over + (partition by i_class) as revenueratio +from + web_sales + ,item + ,date_dim +where + ws_item_sk = i_item_sk + and i_category in ('Jewelry', 'Sports', 'Books') + and ws_sold_date_sk = d_date_sk + and d_date between cast('2001-01-12' as date) + and (cast('2001-01-12' as date) + interval 30 days) +group by + i_item_id + ,i_item_desc + ,i_category + ,i_class + ,i_current_price +order by + i_category + ,i_class + ,i_item_id + ,i_item_desc + ,revenueratio +limit 100 +-- !query schema +struct +-- !query output +AAAAAAAAAPBDAAAA Financial, common films must save typically good, bea Books home repair 4.93 43.36 2.39272470422037789 +AAAAAAAAAPDCAAAA Little lights will not kill silent assumptions. Artists must go. Great, simple mi Books self-help 4.79 2082.33 100.00000000000000000 +AAAAAAAAGCCAAAAA Possible, responsible parents Sports sailing 7.91 7340.00 100.00000000000000000 +AAAAAAAAGEIBAAAA Kids should not spare originally british newspapers. Jobs co Sports optics 7.58 1885.50 100.00000000000000000 +AAAAAAAAHGMAAAAA Variables arrange hostile democrats. Original habits know as certain horses. Firm, technical pupils must see also never other Books home repair 9.17 1768.80 97.60727529577962211 +AAAAAAAAIFPDAAAA Principles would not instruct like the consumers. Calls should forget now by a outcomes. Enough burning islands shed highly other, remaining at Sports camping 4.71 5815.71 100.00000000000000000 +AAAAAAAAIMMAAAAA Decisions get quickly with a notes. Anywhere private journals can get. Front, secondary hands could see yet national i Jewelry costume 0.36 1031.94 100.00000000000000000 +AAAAAAAAJOGCAAAA Pounds would fit very significant weeks. Open, single churches provide. Meetings lose financial members. Things reduce too. Waters place usually determined agents. Books science 4.57 144.21 100.00000000000000000 +AAAAAAAAKNIBAAAA Subjects will read too. Reduced, identical patients like through a animals. At least unable c Jewelry rings 4.90 230.88 100.00000000000000000 +AAAAAAAALIJDAAAA New Books mystery 2.35 5944.62 100.00000000000000000 diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query13.sql b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query13.sql new file mode 100644 index 00000000000..296a7671675 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query13.sql @@ -0,0 +1,52 @@ +-- start query 1 in stream 0 using template query13.tpl +select avg(ss_quantity) + ,avg(ss_ext_sales_price) + ,avg(ss_ext_wholesale_cost) + ,sum(ss_ext_wholesale_cost) + from store_sales + ,store + ,customer_demographics + ,household_demographics + ,customer_address + ,date_dim + where s_store_sk = ss_store_sk + and ss_sold_date_sk = d_date_sk and d_year = 2001 + and((ss_hdemo_sk=hd_demo_sk + and cd_demo_sk = ss_cdemo_sk + and cd_marital_status = 'D' + and cd_education_status = '2 yr Degree' + and ss_sales_price between 100.00 and 150.00 + and hd_dep_count = 3 + )or + (ss_hdemo_sk=hd_demo_sk + and cd_demo_sk = ss_cdemo_sk + and cd_marital_status = 'S' + and cd_education_status = 'Secondary' + and ss_sales_price between 50.00 and 100.00 + and hd_dep_count = 1 + ) or + (ss_hdemo_sk=hd_demo_sk + and cd_demo_sk = ss_cdemo_sk + and cd_marital_status = 'W' + and cd_education_status = 'Advanced Degree' + and ss_sales_price between 150.00 and 200.00 + and hd_dep_count = 1 + )) + and((ss_addr_sk = ca_address_sk + and ca_country = 'United States' + and ca_state in ('CO', 'IL', 'MN') + and ss_net_profit between 100 and 200 + ) or + (ss_addr_sk = ca_address_sk + and ca_country = 'United States' + and ca_state in ('OH', 'MT', 'NM') + and ss_net_profit between 150 and 300 + ) or + (ss_addr_sk = ca_address_sk + and ca_country = 'United States' + and ca_state in ('TX', 'MO', 'MI') + and ss_net_profit between 50 and 250 + )) +; + + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query13.sql.out b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query13.sql.out new file mode 100644 index 00000000000..f45304b56c6 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query13.sql.out @@ -0,0 +1,54 @@ +-- Automatically generated by Gravitino Spark SQL test +-- !query +select avg(ss_quantity) + ,avg(ss_ext_sales_price) + ,avg(ss_ext_wholesale_cost) + ,sum(ss_ext_wholesale_cost) + from store_sales + ,store + ,customer_demographics + ,household_demographics + ,customer_address + ,date_dim + where s_store_sk = ss_store_sk + and ss_sold_date_sk = d_date_sk and d_year = 2001 + and((ss_hdemo_sk=hd_demo_sk + and cd_demo_sk = ss_cdemo_sk + and cd_marital_status = 'D' + and cd_education_status = '2 yr Degree' + and ss_sales_price between 100.00 and 150.00 + and hd_dep_count = 3 + )or + (ss_hdemo_sk=hd_demo_sk + and cd_demo_sk = ss_cdemo_sk + and cd_marital_status = 'S' + and cd_education_status = 'Secondary' + and ss_sales_price between 50.00 and 100.00 + and hd_dep_count = 1 + ) or + (ss_hdemo_sk=hd_demo_sk + and cd_demo_sk = ss_cdemo_sk + and cd_marital_status = 'W' + and cd_education_status = 'Advanced Degree' + and ss_sales_price between 150.00 and 200.00 + and hd_dep_count = 1 + )) + and((ss_addr_sk = ca_address_sk + and ca_country = 'United States' + and ca_state in ('CO', 'IL', 'MN') + and ss_net_profit between 100 and 200 + ) or + (ss_addr_sk = ca_address_sk + and ca_country = 'United States' + and ca_state in ('OH', 'MT', 'NM') + and ss_net_profit between 150 and 300 + ) or + (ss_addr_sk = ca_address_sk + and ca_country = 'United States' + and ca_state in ('TX', 'MO', 'MI') + and ss_net_profit between 50 and 250 + )) +-- !query schema +struct +-- !query output +NULL NULL NULL NULL diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query14.sql b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query14.sql new file mode 100644 index 00000000000..e571c7fd213 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query14.sql @@ -0,0 +1,198 @@ +-- start query 1 in stream 0 using template query14.tpl +with cross_items as + (select i_item_sk ss_item_sk + from item, + (select iss.i_brand_id brand_id + ,iss.i_class_id class_id + ,iss.i_category_id category_id + from store_sales + ,item iss + ,date_dim d1 + where ss_item_sk = iss.i_item_sk + and ss_sold_date_sk = d1.d_date_sk + and d1.d_year between 1998 AND 1998 + 2 + intersect + select ics.i_brand_id + ,ics.i_class_id + ,ics.i_category_id + from catalog_sales + ,item ics + ,date_dim d2 + where cs_item_sk = ics.i_item_sk + and cs_sold_date_sk = d2.d_date_sk + and d2.d_year between 1998 AND 1998 + 2 + intersect + select iws.i_brand_id + ,iws.i_class_id + ,iws.i_category_id + from web_sales + ,item iws + ,date_dim d3 + where ws_item_sk = iws.i_item_sk + and ws_sold_date_sk = d3.d_date_sk + and d3.d_year between 1998 AND 1998 + 2) + where i_brand_id = brand_id + and i_class_id = class_id + and i_category_id = category_id +), + avg_sales as + (select avg(quantity*list_price) average_sales + from (select ss_quantity quantity + ,ss_list_price list_price + from store_sales + ,date_dim + where ss_sold_date_sk = d_date_sk + and d_year between 1998 and 1998 + 2 + union all + select cs_quantity quantity + ,cs_list_price list_price + from catalog_sales + ,date_dim + where cs_sold_date_sk = d_date_sk + and d_year between 1998 and 1998 + 2 + union all + select ws_quantity quantity + ,ws_list_price list_price + from web_sales + ,date_dim + where ws_sold_date_sk = d_date_sk + and d_year between 1998 and 1998 + 2) x) + select channel, i_brand_id,i_class_id,i_category_id,sum(sales), sum(number_sales) + from( + select 'store' channel, i_brand_id,i_class_id + ,i_category_id,sum(ss_quantity*ss_list_price) sales + , count(*) number_sales + from store_sales + ,item + ,date_dim + where ss_item_sk in (select ss_item_sk from cross_items) + and ss_item_sk = i_item_sk + and ss_sold_date_sk = d_date_sk + and d_year = 1998+2 + and d_moy = 11 + group by i_brand_id,i_class_id,i_category_id + having sum(ss_quantity*ss_list_price) > (select average_sales from avg_sales) + union all + select 'catalog' channel, i_brand_id,i_class_id,i_category_id, sum(cs_quantity*cs_list_price) sales, count(*) number_sales + from catalog_sales + ,item + ,date_dim + where cs_item_sk in (select ss_item_sk from cross_items) + and cs_item_sk = i_item_sk + and cs_sold_date_sk = d_date_sk + and d_year = 1998+2 + and d_moy = 11 + group by i_brand_id,i_class_id,i_category_id + having sum(cs_quantity*cs_list_price) > (select average_sales from avg_sales) + union all + select 'web' channel, i_brand_id,i_class_id,i_category_id, sum(ws_quantity*ws_list_price) sales , count(*) number_sales + from web_sales + ,item + ,date_dim + where ws_item_sk in (select ss_item_sk from cross_items) + and ws_item_sk = i_item_sk + and ws_sold_date_sk = d_date_sk + and d_year = 1998+2 + and d_moy = 11 + group by i_brand_id,i_class_id,i_category_id + having sum(ws_quantity*ws_list_price) > (select average_sales from avg_sales) + ) y + group by rollup (channel, i_brand_id,i_class_id,i_category_id) + order by channel,i_brand_id,i_class_id,i_category_id + limit 100; +with cross_items as + (select i_item_sk ss_item_sk + from item, + (select iss.i_brand_id brand_id + ,iss.i_class_id class_id + ,iss.i_category_id category_id + from store_sales + ,item iss + ,date_dim d1 + where ss_item_sk = iss.i_item_sk + and ss_sold_date_sk = d1.d_date_sk + and d1.d_year between 1998 AND 1998 + 2 + intersect + select ics.i_brand_id + ,ics.i_class_id + ,ics.i_category_id + from catalog_sales + ,item ics + ,date_dim d2 + where cs_item_sk = ics.i_item_sk + and cs_sold_date_sk = d2.d_date_sk + and d2.d_year between 1998 AND 1998 + 2 + intersect + select iws.i_brand_id + ,iws.i_class_id + ,iws.i_category_id + from web_sales + ,item iws + ,date_dim d3 + where ws_item_sk = iws.i_item_sk + and ws_sold_date_sk = d3.d_date_sk + and d3.d_year between 1998 AND 1998 + 2) x + where i_brand_id = brand_id + and i_class_id = class_id + and i_category_id = category_id +), + avg_sales as +(select avg(quantity*list_price) average_sales + from (select ss_quantity quantity + ,ss_list_price list_price + from store_sales + ,date_dim + where ss_sold_date_sk = d_date_sk + and d_year between 1998 and 1998 + 2 + union all + select cs_quantity quantity + ,cs_list_price list_price + from catalog_sales + ,date_dim + where cs_sold_date_sk = d_date_sk + and d_year between 1998 and 1998 + 2 + union all + select ws_quantity quantity + ,ws_list_price list_price + from web_sales + ,date_dim + where ws_sold_date_sk = d_date_sk + and d_year between 1998 and 1998 + 2) x) + select * from + (select 'store' channel, i_brand_id,i_class_id,i_category_id + ,sum(ss_quantity*ss_list_price) sales, count(*) number_sales + from store_sales + ,item + ,date_dim + where ss_item_sk in (select ss_item_sk from cross_items) + and ss_item_sk = i_item_sk + and ss_sold_date_sk = d_date_sk + and d_week_seq = (select d_week_seq + from date_dim + where d_year = 1998 + 1 + and d_moy = 12 + and d_dom = 16) + group by i_brand_id,i_class_id,i_category_id + having sum(ss_quantity*ss_list_price) > (select average_sales from avg_sales)) this_year, + (select 'store' channel, i_brand_id,i_class_id + ,i_category_id, sum(ss_quantity*ss_list_price) sales, count(*) number_sales + from store_sales + ,item + ,date_dim + where ss_item_sk in (select ss_item_sk from cross_items) + and ss_item_sk = i_item_sk + and ss_sold_date_sk = d_date_sk + and d_week_seq = (select d_week_seq + from date_dim + where d_year = 1998 + and d_moy = 12 + and d_dom = 16) + group by i_brand_id,i_class_id,i_category_id + having sum(ss_quantity*ss_list_price) > (select average_sales from avg_sales)) last_year + where this_year.i_brand_id= last_year.i_brand_id + and this_year.i_class_id = last_year.i_class_id + and this_year.i_category_id = last_year.i_category_id + order by this_year.channel, this_year.i_brand_id, this_year.i_class_id, this_year.i_category_id + limit 100; + + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query14.sql.out b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query14.sql.out new file mode 100644 index 00000000000..e701e716679 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query14.sql.out @@ -0,0 +1,326 @@ +-- Automatically generated by Gravitino Spark SQL test +-- !query +with cross_items as + (select i_item_sk ss_item_sk + from item, + (select iss.i_brand_id brand_id + ,iss.i_class_id class_id + ,iss.i_category_id category_id + from store_sales + ,item iss + ,date_dim d1 + where ss_item_sk = iss.i_item_sk + and ss_sold_date_sk = d1.d_date_sk + and d1.d_year between 1998 AND 1998 + 2 + intersect + select ics.i_brand_id + ,ics.i_class_id + ,ics.i_category_id + from catalog_sales + ,item ics + ,date_dim d2 + where cs_item_sk = ics.i_item_sk + and cs_sold_date_sk = d2.d_date_sk + and d2.d_year between 1998 AND 1998 + 2 + intersect + select iws.i_brand_id + ,iws.i_class_id + ,iws.i_category_id + from web_sales + ,item iws + ,date_dim d3 + where ws_item_sk = iws.i_item_sk + and ws_sold_date_sk = d3.d_date_sk + and d3.d_year between 1998 AND 1998 + 2) + where i_brand_id = brand_id + and i_class_id = class_id + and i_category_id = category_id +), + avg_sales as + (select avg(quantity*list_price) average_sales + from (select ss_quantity quantity + ,ss_list_price list_price + from store_sales + ,date_dim + where ss_sold_date_sk = d_date_sk + and d_year between 1998 and 1998 + 2 + union all + select cs_quantity quantity + ,cs_list_price list_price + from catalog_sales + ,date_dim + where cs_sold_date_sk = d_date_sk + and d_year between 1998 and 1998 + 2 + union all + select ws_quantity quantity + ,ws_list_price list_price + from web_sales + ,date_dim + where ws_sold_date_sk = d_date_sk + and d_year between 1998 and 1998 + 2) x) + select channel, i_brand_id,i_class_id,i_category_id,sum(sales), sum(number_sales) + from( + select 'store' channel, i_brand_id,i_class_id + ,i_category_id,sum(ss_quantity*ss_list_price) sales + , count(*) number_sales + from store_sales + ,item + ,date_dim + where ss_item_sk in (select ss_item_sk from cross_items) + and ss_item_sk = i_item_sk + and ss_sold_date_sk = d_date_sk + and d_year = 1998+2 + and d_moy = 11 + group by i_brand_id,i_class_id,i_category_id + having sum(ss_quantity*ss_list_price) > (select average_sales from avg_sales) + union all + select 'catalog' channel, i_brand_id,i_class_id,i_category_id, sum(cs_quantity*cs_list_price) sales, count(*) number_sales + from catalog_sales + ,item + ,date_dim + where cs_item_sk in (select ss_item_sk from cross_items) + and cs_item_sk = i_item_sk + and cs_sold_date_sk = d_date_sk + and d_year = 1998+2 + and d_moy = 11 + group by i_brand_id,i_class_id,i_category_id + having sum(cs_quantity*cs_list_price) > (select average_sales from avg_sales) + union all + select 'web' channel, i_brand_id,i_class_id,i_category_id, sum(ws_quantity*ws_list_price) sales , count(*) number_sales + from web_sales + ,item + ,date_dim + where ws_item_sk in (select ss_item_sk from cross_items) + and ws_item_sk = i_item_sk + and ws_sold_date_sk = d_date_sk + and d_year = 1998+2 + and d_moy = 11 + group by i_brand_id,i_class_id,i_category_id + having sum(ws_quantity*ws_list_price) > (select average_sales from avg_sales) + ) y + group by rollup (channel, i_brand_id,i_class_id,i_category_id) + order by channel,i_brand_id,i_class_id,i_category_id + limit 100 +-- !query schema +struct +-- !query output +NULL NULL NULL NULL 3049499.94 651 +store 1001001 1 1 31885.81 9 +store 1001001 1 NULL 31885.81 9 +store 1001001 NULL NULL 31885.81 9 +store 1001002 1 1 61750.09 11 +store 1001002 1 NULL 61750.09 11 +store 1001002 NULL NULL 61750.09 11 +store 1002001 2 1 61869.73 12 +store 1002001 2 NULL 61869.73 12 +store 1002001 NULL NULL 61869.73 12 +store 1002002 2 1 31670.46 10 +store 1002002 2 NULL 31670.46 10 +store 1002002 NULL NULL 31670.46 10 +store 1003001 3 1 15865.08 5 +store 1003001 3 NULL 15865.08 5 +store 1003001 NULL NULL 15865.08 5 +store 1003002 3 1 11884.96 6 +store 1003002 3 NULL 11884.96 6 +store 1003002 NULL NULL 11884.96 6 +store 1004001 4 1 27032.32 8 +store 1004001 4 NULL 27032.32 8 +store 1004001 NULL NULL 27032.32 8 +store 1004002 4 1 46872.89 11 +store 1004002 4 NULL 46872.89 11 +store 1004002 NULL NULL 46872.89 11 +store 2001001 1 2 23132.36 11 +store 2001001 1 NULL 23132.36 11 +store 2001001 NULL NULL 23132.36 11 +store 2001002 1 2 28347.00 9 +store 2001002 1 NULL 28347.00 9 +store 2001002 NULL NULL 28347.00 9 +store 2002001 2 2 32545.94 8 +store 2002001 2 NULL 32545.94 8 +store 2002001 NULL NULL 32545.94 8 +store 2002002 2 2 42168.51 10 +store 2002002 2 NULL 42168.51 10 +store 2002002 NULL NULL 42168.51 10 +store 2003001 3 2 20829.36 7 +store 2003001 3 NULL 20829.36 7 +store 2003001 NULL NULL 20829.36 7 +store 2003002 3 2 48988.72 9 +store 2003002 3 NULL 48988.72 9 +store 2003002 NULL NULL 48988.72 9 +store 2004001 4 2 20678.69 7 +store 2004001 4 NULL 20678.69 7 +store 2004001 NULL NULL 20678.69 7 +store 2004002 4 2 63959.64 18 +store 2004002 4 NULL 63959.64 18 +store 2004002 NULL NULL 63959.64 18 +store 3001001 1 3 36344.04 9 +store 3001001 1 NULL 36344.04 9 +store 3001001 NULL NULL 36344.04 9 +store 3001002 1 3 29346.66 4 +store 3001002 1 NULL 29346.66 4 +store 3001002 NULL NULL 29346.66 4 +store 3002001 2 3 28054.56 10 +store 3002001 2 NULL 28054.56 10 +store 3002001 NULL NULL 28054.56 10 +store 3002002 2 3 56050.29 12 +store 3002002 2 NULL 56050.29 12 +store 3002002 NULL NULL 56050.29 12 +store 3003001 3 3 82327.31 20 +store 3003001 3 NULL 82327.31 20 +store 3003001 NULL NULL 82327.31 20 +store 3003002 3 3 37370.63 8 +store 3003002 3 NULL 37370.63 8 +store 3003002 NULL NULL 37370.63 8 +store 3004001 4 3 32091.61 6 +store 3004001 4 NULL 32091.61 6 +store 3004001 NULL NULL 32091.61 6 +store 3004002 4 3 34101.88 10 +store 3004002 4 NULL 34101.88 10 +store 3004002 NULL NULL 34101.88 10 +store 4001001 1 4 59345.06 12 +store 4001001 1 NULL 59345.06 12 +store 4001001 NULL NULL 59345.06 12 +store 4001002 1 4 35633.96 12 +store 4001002 1 NULL 35633.96 12 +store 4001002 NULL NULL 35633.96 12 +store 4002001 2 4 30512.88 9 +store 4002001 2 NULL 30512.88 9 +store 4002001 NULL NULL 30512.88 9 +store 4002002 2 4 54833.60 14 +store 4002002 2 NULL 54833.60 14 +store 4002002 NULL NULL 54833.60 14 +store 4003001 3 4 10725.35 2 +store 4003001 3 NULL 10725.35 2 +store 4003001 NULL NULL 10725.35 2 +store 4003002 3 4 29130.98 11 +store 4003002 3 NULL 29130.98 11 +store 4003002 NULL NULL 29130.98 11 +store 4004001 4 4 47820.24 11 +store 4004001 4 NULL 47820.24 11 +store 4004001 NULL NULL 47820.24 11 +store 4004002 4 4 75503.48 14 +store 4004002 4 NULL 75503.48 14 +store 4004002 NULL NULL 75503.48 14 +store 5001001 1 NULL 34458.91 10 +store 5001001 NULL NULL 34458.91 10 +store NULL NULL NULL 2414735.62 556 + + +-- !query +with cross_items as + (select i_item_sk ss_item_sk + from item, + (select iss.i_brand_id brand_id + ,iss.i_class_id class_id + ,iss.i_category_id category_id + from store_sales + ,item iss + ,date_dim d1 + where ss_item_sk = iss.i_item_sk + and ss_sold_date_sk = d1.d_date_sk + and d1.d_year between 1998 AND 1998 + 2 + intersect + select ics.i_brand_id + ,ics.i_class_id + ,ics.i_category_id + from catalog_sales + ,item ics + ,date_dim d2 + where cs_item_sk = ics.i_item_sk + and cs_sold_date_sk = d2.d_date_sk + and d2.d_year between 1998 AND 1998 + 2 + intersect + select iws.i_brand_id + ,iws.i_class_id + ,iws.i_category_id + from web_sales + ,item iws + ,date_dim d3 + where ws_item_sk = iws.i_item_sk + and ws_sold_date_sk = d3.d_date_sk + and d3.d_year between 1998 AND 1998 + 2) x + where i_brand_id = brand_id + and i_class_id = class_id + and i_category_id = category_id +), + avg_sales as +(select avg(quantity*list_price) average_sales + from (select ss_quantity quantity + ,ss_list_price list_price + from store_sales + ,date_dim + where ss_sold_date_sk = d_date_sk + and d_year between 1998 and 1998 + 2 + union all + select cs_quantity quantity + ,cs_list_price list_price + from catalog_sales + ,date_dim + where cs_sold_date_sk = d_date_sk + and d_year between 1998 and 1998 + 2 + union all + select ws_quantity quantity + ,ws_list_price list_price + from web_sales + ,date_dim + where ws_sold_date_sk = d_date_sk + and d_year between 1998 and 1998 + 2) x) + select * from + (select 'store' channel, i_brand_id,i_class_id,i_category_id + ,sum(ss_quantity*ss_list_price) sales, count(*) number_sales + from store_sales + ,item + ,date_dim + where ss_item_sk in (select ss_item_sk from cross_items) + and ss_item_sk = i_item_sk + and ss_sold_date_sk = d_date_sk + and d_week_seq = (select d_week_seq + from date_dim + where d_year = 1998 + 1 + and d_moy = 12 + and d_dom = 16) + group by i_brand_id,i_class_id,i_category_id + having sum(ss_quantity*ss_list_price) > (select average_sales from avg_sales)) this_year, + (select 'store' channel, i_brand_id,i_class_id + ,i_category_id, sum(ss_quantity*ss_list_price) sales, count(*) number_sales + from store_sales + ,item + ,date_dim + where ss_item_sk in (select ss_item_sk from cross_items) + and ss_item_sk = i_item_sk + and ss_sold_date_sk = d_date_sk + and d_week_seq = (select d_week_seq + from date_dim + where d_year = 1998 + and d_moy = 12 + and d_dom = 16) + group by i_brand_id,i_class_id,i_category_id + having sum(ss_quantity*ss_list_price) > (select average_sales from avg_sales)) last_year + where this_year.i_brand_id= last_year.i_brand_id + and this_year.i_class_id = last_year.i_class_id + and this_year.i_category_id = last_year.i_category_id + order by this_year.channel, this_year.i_brand_id, this_year.i_class_id, this_year.i_category_id + limit 100 +-- !query schema +struct +-- !query output +store 10002006 2 10 5926.44 1 store 10002006 2 10 5624.71 1 +store 1001001 1 1 9993.34 2 store 1001001 1 1 14170.87 3 +store 10016015 16 10 5282.76 1 store 10016015 16 10 7062.30 1 +store 1003001 3 1 26879.25 5 store 1003001 3 1 7952.33 3 +store 1004001 4 1 25861.72 4 store 1004001 4 1 14708.04 6 +store 2003001 3 2 18806.03 5 store 2003001 3 2 6689.28 1 +store 3001001 1 3 49580.51 7 store 3001001 1 3 6664.98 3 +store 3002001 2 3 17695.75 3 store 3002001 2 3 7547.20 3 +store 3003001 3 3 18201.26 6 store 3003001 3 3 13766.04 4 +store 3004001 4 3 18160.79 6 store 3004001 4 3 11545.59 3 +store 4001001 1 4 28270.43 6 store 4001001 1 4 9902.93 3 +store 4002001 2 4 26106.97 4 store 4002001 2 4 14785.36 4 +store 4004001 4 4 7754.37 3 store 4004001 4 4 6046.47 1 +store 4004002 4 4 13184.04 6 store 4004002 4 4 4688.65 1 +store 5001001 1 5 22738.19 8 store 5001001 1 5 24411.20 6 +store 5003001 3 5 19355.10 7 store 5003001 3 5 6876.55 3 +store 5004002 4 5 6355.47 4 store 5004002 4 5 6297.48 1 +store 7005005 5 7 6178.69 1 store 7005005 5 7 8772.80 1 +store 9010005 10 9 9596.52 2 store 9010005 10 9 5644.16 1 +store 9010009 10 9 4968.34 1 store 9010009 10 9 15515.37 1 diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query15.sql b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query15.sql new file mode 100644 index 00000000000..1276c8ed1cd --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query15.sql @@ -0,0 +1,20 @@ +-- start query 1 in stream 0 using template query15.tpl +select ca_zip + ,sum(cs_sales_price) + from catalog_sales + ,customer + ,customer_address + ,date_dim + where cs_bill_customer_sk = c_customer_sk + and c_current_addr_sk = ca_address_sk + and ( substr(ca_zip,1,5) in ('85669', '86197','88274','83405','86475', + '85392', '85460', '80348', '81792') + or ca_state in ('CA','WA','GA') + or cs_sales_price > 500) + and cs_sold_date_sk = d_date_sk + and d_qoy = 2 and d_year = 2000 + group by ca_zip + order by ca_zip + limit 100; + + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query15.sql.out b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query15.sql.out new file mode 100644 index 00000000000..163bbaac1a0 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query15.sql.out @@ -0,0 +1,23 @@ +-- Automatically generated by Gravitino Spark SQL test +-- !query +select ca_zip + ,sum(cs_sales_price) + from catalog_sales + ,customer + ,customer_address + ,date_dim + where cs_bill_customer_sk = c_customer_sk + and c_current_addr_sk = ca_address_sk + and ( substr(ca_zip,1,5) in ('85669', '86197','88274','83405','86475', + '85392', '85460', '80348', '81792') + or ca_state in ('CA','WA','GA') + or cs_sales_price > 500) + and cs_sold_date_sk = d_date_sk + and d_qoy = 2 and d_year = 2000 + group by ca_zip + order by ca_zip + limit 100 +-- !query schema +struct +-- !query output + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query16.sql b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query16.sql new file mode 100644 index 00000000000..7c451ff6e4c --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query16.sql @@ -0,0 +1,31 @@ +-- start query 1 in stream 0 using template query16.tpl +select + count(distinct cs_order_number) as ordercount + ,sum(cs_ext_ship_cost) as totalshippingcost + ,sum(cs_net_profit) as totalnetprofit +from + catalog_sales cs1 + ,date_dim + ,customer_address + ,call_center +where + d_date between '1999-2-01' and + (cast('1999-2-01' as date) + interval 60 days) +and cs1.cs_ship_date_sk = d_date_sk +and cs1.cs_ship_addr_sk = ca_address_sk +and ca_state = 'IL' +and cs1.cs_call_center_sk = cc_call_center_sk +and cc_county in ('Williamson County','Williamson County','Williamson County','Williamson County', + 'Williamson County' +) +and exists (select * + from catalog_sales cs2 + where cs1.cs_order_number = cs2.cs_order_number + and cs1.cs_warehouse_sk <> cs2.cs_warehouse_sk) +and not exists(select * + from catalog_returns cr1 + where cs1.cs_order_number = cr1.cr_order_number) +order by count(distinct cs_order_number) +limit 100; + + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query16.sql.out b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query16.sql.out new file mode 100644 index 00000000000..b9b35823bc0 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query16.sql.out @@ -0,0 +1,34 @@ +-- Automatically generated by Gravitino Spark SQL test +-- !query +select + count(distinct cs_order_number) as ordercount + ,sum(cs_ext_ship_cost) as totalshippingcost + ,sum(cs_net_profit) as totalnetprofit +from + catalog_sales cs1 + ,date_dim + ,customer_address + ,call_center +where + d_date between '1999-2-01' and + (cast('1999-2-01' as date) + interval 60 days) +and cs1.cs_ship_date_sk = d_date_sk +and cs1.cs_ship_addr_sk = ca_address_sk +and ca_state = 'IL' +and cs1.cs_call_center_sk = cc_call_center_sk +and cc_county in ('Williamson County','Williamson County','Williamson County','Williamson County', + 'Williamson County' +) +and exists (select * + from catalog_sales cs2 + where cs1.cs_order_number = cs2.cs_order_number + and cs1.cs_warehouse_sk <> cs2.cs_warehouse_sk) +and not exists(select * + from catalog_returns cr1 + where cs1.cs_order_number = cr1.cr_order_number) +order by count(distinct cs_order_number) +limit 100 +-- !query schema +struct +-- !query output +0 NULL NULL diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query17.sql b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query17.sql new file mode 100644 index 00000000000..e8c3af7e24d --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query17.sql @@ -0,0 +1,45 @@ +-- start query 1 in stream 0 using template query17.tpl +select i_item_id + ,i_item_desc + ,s_state + ,count(ss_quantity) as store_sales_quantitycount + ,avg(ss_quantity) as store_sales_quantityave + ,stddev_samp(ss_quantity) as store_sales_quantitystdev + ,stddev_samp(ss_quantity)/avg(ss_quantity) as store_sales_quantitycov + ,count(sr_return_quantity) as store_returns_quantitycount + ,avg(sr_return_quantity) as store_returns_quantityave + ,stddev_samp(sr_return_quantity) as store_returns_quantitystdev + ,stddev_samp(sr_return_quantity)/avg(sr_return_quantity) as store_returns_quantitycov + ,count(cs_quantity) as catalog_sales_quantitycount ,avg(cs_quantity) as catalog_sales_quantityave + ,stddev_samp(cs_quantity) as catalog_sales_quantitystdev + ,stddev_samp(cs_quantity)/avg(cs_quantity) as catalog_sales_quantitycov + from store_sales + ,store_returns + ,catalog_sales + ,date_dim d1 + ,date_dim d2 + ,date_dim d3 + ,store + ,item + where d1.d_quarter_name = '1998Q1' + and d1.d_date_sk = ss_sold_date_sk + and i_item_sk = ss_item_sk + and s_store_sk = ss_store_sk + and ss_customer_sk = sr_customer_sk + and ss_item_sk = sr_item_sk + and ss_ticket_number = sr_ticket_number + and sr_returned_date_sk = d2.d_date_sk + and d2.d_quarter_name in ('1998Q1','1998Q2','1998Q3') + and sr_customer_sk = cs_bill_customer_sk + and sr_item_sk = cs_item_sk + and cs_sold_date_sk = d3.d_date_sk + and d3.d_quarter_name in ('1998Q1','1998Q2','1998Q3') + group by i_item_id + ,i_item_desc + ,s_state + order by i_item_id + ,i_item_desc + ,s_state +limit 100; + + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query17.sql.out b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query17.sql.out new file mode 100644 index 00000000000..0e00d42ae70 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query17.sql.out @@ -0,0 +1,48 @@ +-- Automatically generated by Gravitino Spark SQL test +-- !query +select i_item_id + ,i_item_desc + ,s_state + ,count(ss_quantity) as store_sales_quantitycount + ,avg(ss_quantity) as store_sales_quantityave + ,stddev_samp(ss_quantity) as store_sales_quantitystdev + ,stddev_samp(ss_quantity)/avg(ss_quantity) as store_sales_quantitycov + ,count(sr_return_quantity) as store_returns_quantitycount + ,avg(sr_return_quantity) as store_returns_quantityave + ,stddev_samp(sr_return_quantity) as store_returns_quantitystdev + ,stddev_samp(sr_return_quantity)/avg(sr_return_quantity) as store_returns_quantitycov + ,count(cs_quantity) as catalog_sales_quantitycount ,avg(cs_quantity) as catalog_sales_quantityave + ,stddev_samp(cs_quantity) as catalog_sales_quantitystdev + ,stddev_samp(cs_quantity)/avg(cs_quantity) as catalog_sales_quantitycov + from store_sales + ,store_returns + ,catalog_sales + ,date_dim d1 + ,date_dim d2 + ,date_dim d3 + ,store + ,item + where d1.d_quarter_name = '1998Q1' + and d1.d_date_sk = ss_sold_date_sk + and i_item_sk = ss_item_sk + and s_store_sk = ss_store_sk + and ss_customer_sk = sr_customer_sk + and ss_item_sk = sr_item_sk + and ss_ticket_number = sr_ticket_number + and sr_returned_date_sk = d2.d_date_sk + and d2.d_quarter_name in ('1998Q1','1998Q2','1998Q3') + and sr_customer_sk = cs_bill_customer_sk + and sr_item_sk = cs_item_sk + and cs_sold_date_sk = d3.d_date_sk + and d3.d_quarter_name in ('1998Q1','1998Q2','1998Q3') + group by i_item_id + ,i_item_desc + ,s_state + order by i_item_id + ,i_item_desc + ,s_state +limit 100 +-- !query schema +struct +-- !query output + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query18.sql b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query18.sql new file mode 100644 index 00000000000..e66d049447c --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query18.sql @@ -0,0 +1,34 @@ +-- start query 1 in stream 0 using template query18.tpl +select i_item_id, + ca_country, + ca_state, + ca_county, + avg( cast(cs_quantity as decimal(12,2))) agg1, + avg( cast(cs_list_price as decimal(12,2))) agg2, + avg( cast(cs_coupon_amt as decimal(12,2))) agg3, + avg( cast(cs_sales_price as decimal(12,2))) agg4, + avg( cast(cs_net_profit as decimal(12,2))) agg5, + avg( cast(c_birth_year as decimal(12,2))) agg6, + avg( cast(cd1.cd_dep_count as decimal(12,2))) agg7 + from catalog_sales, customer_demographics cd1, + customer_demographics cd2, customer, customer_address, date_dim, item + where cs_sold_date_sk = d_date_sk and + cs_item_sk = i_item_sk and + cs_bill_cdemo_sk = cd1.cd_demo_sk and + cs_bill_customer_sk = c_customer_sk and + cd1.cd_gender = 'M' and + cd1.cd_education_status = 'College' and + c_current_cdemo_sk = cd2.cd_demo_sk and + c_current_addr_sk = ca_address_sk and + c_birth_month in (9,5,12,4,1,10) and + d_year = 2001 and + ca_state in ('ND','WI','AL' + ,'NC','OK','MS','TN') + group by rollup (i_item_id, ca_country, ca_state, ca_county) + order by ca_country, + ca_state, + ca_county, + i_item_id + limit 100; + + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query18.sql.out b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query18.sql.out new file mode 100644 index 00000000000..a34b171cf31 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query18.sql.out @@ -0,0 +1,37 @@ +-- Automatically generated by Gravitino Spark SQL test +-- !query +select i_item_id, + ca_country, + ca_state, + ca_county, + avg( cast(cs_quantity as decimal(12,2))) agg1, + avg( cast(cs_list_price as decimal(12,2))) agg2, + avg( cast(cs_coupon_amt as decimal(12,2))) agg3, + avg( cast(cs_sales_price as decimal(12,2))) agg4, + avg( cast(cs_net_profit as decimal(12,2))) agg5, + avg( cast(c_birth_year as decimal(12,2))) agg6, + avg( cast(cd1.cd_dep_count as decimal(12,2))) agg7 + from catalog_sales, customer_demographics cd1, + customer_demographics cd2, customer, customer_address, date_dim, item + where cs_sold_date_sk = d_date_sk and + cs_item_sk = i_item_sk and + cs_bill_cdemo_sk = cd1.cd_demo_sk and + cs_bill_customer_sk = c_customer_sk and + cd1.cd_gender = 'M' and + cd1.cd_education_status = 'College' and + c_current_cdemo_sk = cd2.cd_demo_sk and + c_current_addr_sk = ca_address_sk and + c_birth_month in (9,5,12,4,1,10) and + d_year = 2001 and + ca_state in ('ND','WI','AL' + ,'NC','OK','MS','TN') + group by rollup (i_item_id, ca_country, ca_state, ca_county) + order by ca_country, + ca_state, + ca_county, + i_item_id + limit 100 +-- !query schema +struct +-- !query output + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query19.sql b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query19.sql new file mode 100644 index 00000000000..0211863ee93 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query19.sql @@ -0,0 +1,25 @@ +-- start query 1 in stream 0 using template query19.tpl +select i_brand_id brand_id, i_brand brand, i_manufact_id, i_manufact, + sum(ss_ext_sales_price) ext_price + from date_dim, store_sales, item,customer,customer_address,store + where d_date_sk = ss_sold_date_sk + and ss_item_sk = i_item_sk + and i_manager_id=7 + and d_moy=11 + and d_year=1999 + and ss_customer_sk = c_customer_sk + and c_current_addr_sk = ca_address_sk + and substr(ca_zip,1,5) <> substr(s_zip,1,5) + and ss_store_sk = s_store_sk + group by i_brand + ,i_brand_id + ,i_manufact_id + ,i_manufact + order by ext_price desc + ,i_brand + ,i_brand_id + ,i_manufact_id + ,i_manufact +limit 100 ; + + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query19.sql.out b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query19.sql.out new file mode 100644 index 00000000000..feac65cae83 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query19.sql.out @@ -0,0 +1,30 @@ +-- Automatically generated by Gravitino Spark SQL test +-- !query +select i_brand_id brand_id, i_brand brand, i_manufact_id, i_manufact, + sum(ss_ext_sales_price) ext_price + from date_dim, store_sales, item,customer,customer_address,store + where d_date_sk = ss_sold_date_sk + and ss_item_sk = i_item_sk + and i_manager_id=7 + and d_moy=11 + and d_year=1999 + and ss_customer_sk = c_customer_sk + and c_current_addr_sk = ca_address_sk + and substr(ca_zip,1,5) <> substr(s_zip,1,5) + and ss_store_sk = s_store_sk + group by i_brand + ,i_brand_id + ,i_manufact_id + ,i_manufact + order by ext_price desc + ,i_brand + ,i_brand_id + ,i_manufact_id + ,i_manufact +limit 100 +-- !query schema +struct +-- !query output +1004001 edu packamalg #x 622 ableablecally 216.44 +4001001 amalgedu pack #x 590 barn stanti 4135.56 +8006007 corpnameless #x 602 ablebarcally 257.14 diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query2.sql b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query2.sql new file mode 100644 index 00000000000..e292a160ec8 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query2.sql @@ -0,0 +1,60 @@ +-- start query 1 in stream 0 using template query2.tpl +with wscs as + (select sold_date_sk + ,sales_price + from (select ws_sold_date_sk sold_date_sk + ,ws_ext_sales_price sales_price + from web_sales) + union all + (select cs_sold_date_sk sold_date_sk + ,cs_ext_sales_price sales_price + from catalog_sales)), + wswscs as + (select d_week_seq, + sum(case when (d_day_name='Sunday') then sales_price else null end) sun_sales, + sum(case when (d_day_name='Monday') then sales_price else null end) mon_sales, + sum(case when (d_day_name='Tuesday') then sales_price else null end) tue_sales, + sum(case when (d_day_name='Wednesday') then sales_price else null end) wed_sales, + sum(case when (d_day_name='Thursday') then sales_price else null end) thu_sales, + sum(case when (d_day_name='Friday') then sales_price else null end) fri_sales, + sum(case when (d_day_name='Saturday') then sales_price else null end) sat_sales + from wscs + ,date_dim + where d_date_sk = sold_date_sk + group by d_week_seq) + select d_week_seq1 + ,round(sun_sales1/sun_sales2,2) + ,round(mon_sales1/mon_sales2,2) + ,round(tue_sales1/tue_sales2,2) + ,round(wed_sales1/wed_sales2,2) + ,round(thu_sales1/thu_sales2,2) + ,round(fri_sales1/fri_sales2,2) + ,round(sat_sales1/sat_sales2,2) + from + (select wswscs.d_week_seq d_week_seq1 + ,sun_sales sun_sales1 + ,mon_sales mon_sales1 + ,tue_sales tue_sales1 + ,wed_sales wed_sales1 + ,thu_sales thu_sales1 + ,fri_sales fri_sales1 + ,sat_sales sat_sales1 + from wswscs,date_dim + where date_dim.d_week_seq = wswscs.d_week_seq and + d_year = 2001) y, + (select wswscs.d_week_seq d_week_seq2 + ,sun_sales sun_sales2 + ,mon_sales mon_sales2 + ,tue_sales tue_sales2 + ,wed_sales wed_sales2 + ,thu_sales thu_sales2 + ,fri_sales fri_sales2 + ,sat_sales sat_sales2 + from wswscs + ,date_dim + where date_dim.d_week_seq = wswscs.d_week_seq and + d_year = 2001+1) z + where d_week_seq1=d_week_seq2-53 + order by d_week_seq1; + + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query2.sql.out b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query2.sql.out new file mode 100644 index 00000000000..accae773ada --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query2.sql.out @@ -0,0 +1,1644 @@ +-- Automatically generated by Gravitino Spark SQL test +-- !query +with wscs as + (select sold_date_sk + ,sales_price + from (select ws_sold_date_sk sold_date_sk + ,ws_ext_sales_price sales_price + from web_sales) + union all + (select cs_sold_date_sk sold_date_sk + ,cs_ext_sales_price sales_price + from catalog_sales)), + wswscs as + (select d_week_seq, + sum(case when (d_day_name='Sunday') then sales_price else null end) sun_sales, + sum(case when (d_day_name='Monday') then sales_price else null end) mon_sales, + sum(case when (d_day_name='Tuesday') then sales_price else null end) tue_sales, + sum(case when (d_day_name='Wednesday') then sales_price else null end) wed_sales, + sum(case when (d_day_name='Thursday') then sales_price else null end) thu_sales, + sum(case when (d_day_name='Friday') then sales_price else null end) fri_sales, + sum(case when (d_day_name='Saturday') then sales_price else null end) sat_sales + from wscs + ,date_dim + where d_date_sk = sold_date_sk + group by d_week_seq) + select d_week_seq1 + ,round(sun_sales1/sun_sales2,2) + ,round(mon_sales1/mon_sales2,2) + ,round(tue_sales1/tue_sales2,2) + ,round(wed_sales1/wed_sales2,2) + ,round(thu_sales1/thu_sales2,2) + ,round(fri_sales1/fri_sales2,2) + ,round(sat_sales1/sat_sales2,2) + from + (select wswscs.d_week_seq d_week_seq1 + ,sun_sales sun_sales1 + ,mon_sales mon_sales1 + ,tue_sales tue_sales1 + ,wed_sales wed_sales1 + ,thu_sales thu_sales1 + ,fri_sales fri_sales1 + ,sat_sales sat_sales1 + from wswscs,date_dim + where date_dim.d_week_seq = wswscs.d_week_seq and + d_year = 2001) y, + (select wswscs.d_week_seq d_week_seq2 + ,sun_sales sun_sales2 + ,mon_sales mon_sales2 + ,tue_sales tue_sales2 + ,wed_sales wed_sales2 + ,thu_sales thu_sales2 + ,fri_sales fri_sales2 + ,sat_sales sat_sales2 + from wswscs + ,date_dim + where date_dim.d_week_seq = wswscs.d_week_seq and + d_year = 2001+1) z + where d_week_seq1=d_week_seq2-53 + order by d_week_seq1 +-- !query schema +struct +-- !query output +5270 NULL NULL NULL NULL NULL NULL NULL +5270 NULL NULL NULL NULL NULL NULL NULL +5270 NULL NULL NULL NULL NULL NULL NULL +5270 NULL NULL NULL NULL NULL NULL NULL +5270 NULL NULL NULL NULL NULL NULL NULL +5270 NULL NULL NULL NULL NULL NULL NULL +5270 NULL NULL NULL NULL NULL NULL NULL +5274 NULL NULL NULL NULL NULL NULL NULL +5274 NULL NULL NULL NULL NULL NULL NULL +5274 NULL NULL NULL NULL NULL NULL NULL +5274 NULL NULL NULL NULL NULL NULL NULL +5274 NULL NULL NULL NULL NULL NULL NULL +5274 NULL NULL NULL NULL NULL NULL NULL +5274 NULL NULL NULL NULL NULL NULL NULL +5274 NULL NULL NULL NULL NULL NULL NULL +5274 NULL NULL NULL NULL NULL NULL NULL +5274 NULL NULL NULL NULL NULL NULL NULL +5274 NULL NULL NULL NULL NULL NULL NULL +5274 NULL NULL NULL NULL NULL NULL NULL +5274 NULL NULL NULL NULL NULL NULL NULL +5274 NULL NULL NULL NULL NULL NULL NULL +5274 NULL NULL NULL NULL NULL NULL NULL +5274 NULL NULL NULL NULL NULL NULL NULL +5274 NULL NULL NULL NULL NULL NULL NULL +5274 NULL NULL NULL NULL NULL NULL NULL +5274 NULL NULL NULL NULL NULL NULL NULL +5274 NULL NULL NULL NULL NULL NULL NULL +5274 NULL NULL NULL NULL NULL NULL NULL +5274 NULL NULL NULL NULL NULL NULL NULL +5274 NULL NULL NULL NULL NULL NULL NULL +5274 NULL NULL NULL NULL NULL NULL NULL +5274 NULL NULL NULL NULL NULL NULL NULL +5274 NULL NULL NULL NULL NULL NULL NULL +5274 NULL NULL NULL NULL NULL NULL NULL +5274 NULL NULL NULL NULL NULL NULL NULL +5274 NULL NULL NULL NULL NULL NULL NULL +5274 NULL NULL NULL NULL NULL NULL NULL +5274 NULL NULL NULL NULL NULL NULL NULL +5274 NULL NULL NULL NULL NULL NULL NULL +5274 NULL NULL NULL NULL NULL NULL NULL +5274 NULL NULL NULL NULL NULL NULL NULL +5274 NULL NULL NULL NULL NULL NULL NULL +5274 NULL NULL NULL NULL NULL NULL NULL +5274 NULL NULL NULL NULL NULL NULL NULL +5274 NULL NULL NULL NULL NULL NULL NULL +5274 NULL NULL NULL NULL NULL NULL NULL +5274 NULL NULL NULL NULL NULL NULL NULL +5274 NULL NULL NULL NULL NULL NULL NULL +5274 NULL NULL NULL NULL NULL NULL NULL +5274 NULL NULL NULL NULL NULL NULL NULL +5274 NULL NULL NULL NULL NULL NULL NULL +5274 NULL NULL NULL NULL NULL NULL NULL +5274 NULL NULL NULL NULL NULL NULL NULL +5274 NULL NULL NULL NULL NULL NULL NULL +5274 NULL NULL NULL NULL NULL NULL NULL +5274 NULL NULL NULL NULL NULL NULL NULL +5275 NULL NULL NULL NULL NULL NULL NULL +5275 NULL NULL NULL NULL NULL NULL NULL +5275 NULL NULL NULL NULL NULL NULL NULL +5275 NULL NULL NULL NULL NULL NULL NULL +5275 NULL NULL NULL NULL NULL NULL NULL +5275 NULL NULL NULL NULL NULL NULL NULL +5275 NULL NULL NULL NULL NULL NULL NULL +5275 NULL NULL NULL NULL NULL NULL NULL +5275 NULL NULL NULL NULL NULL NULL NULL +5275 NULL NULL NULL NULL NULL NULL NULL +5275 NULL NULL NULL NULL NULL NULL NULL +5275 NULL NULL NULL NULL NULL NULL NULL +5275 NULL NULL NULL NULL NULL NULL NULL +5275 NULL NULL NULL NULL NULL NULL NULL +5275 NULL NULL NULL NULL NULL NULL NULL +5275 NULL NULL NULL NULL NULL NULL NULL +5275 NULL NULL NULL NULL NULL NULL NULL +5275 NULL NULL NULL NULL NULL NULL NULL +5275 NULL NULL NULL NULL NULL NULL NULL +5275 NULL NULL NULL NULL NULL NULL NULL +5275 NULL NULL NULL NULL NULL NULL NULL +5275 NULL NULL NULL NULL NULL NULL NULL +5275 NULL NULL NULL NULL NULL NULL NULL +5275 NULL NULL NULL NULL NULL NULL NULL +5275 NULL NULL NULL NULL NULL NULL NULL +5275 NULL NULL NULL NULL NULL NULL NULL +5275 NULL NULL NULL NULL NULL NULL NULL +5275 NULL NULL NULL NULL NULL NULL NULL +5275 NULL NULL NULL NULL NULL NULL NULL +5275 NULL NULL NULL NULL NULL NULL NULL +5275 NULL NULL NULL NULL NULL NULL NULL +5275 NULL NULL NULL NULL NULL NULL NULL +5275 NULL NULL NULL NULL NULL NULL NULL +5275 NULL NULL NULL NULL NULL NULL NULL +5275 NULL NULL NULL NULL NULL NULL NULL +5275 NULL NULL NULL NULL NULL NULL NULL +5275 NULL NULL NULL NULL NULL NULL NULL +5275 NULL NULL NULL NULL NULL NULL NULL +5275 NULL NULL NULL NULL NULL NULL NULL +5275 NULL NULL NULL NULL NULL NULL NULL +5275 NULL NULL NULL NULL NULL NULL NULL +5275 NULL NULL NULL NULL NULL NULL NULL +5275 NULL NULL NULL NULL NULL NULL NULL +5275 NULL NULL NULL NULL NULL NULL NULL +5275 NULL NULL NULL NULL NULL NULL NULL +5275 NULL NULL NULL NULL NULL NULL NULL +5275 NULL NULL NULL NULL NULL NULL NULL +5275 NULL NULL NULL NULL NULL NULL NULL +5275 NULL NULL NULL NULL NULL NULL NULL +5277 NULL NULL NULL NULL NULL NULL NULL +5277 NULL NULL NULL NULL NULL NULL NULL +5277 NULL NULL NULL NULL NULL NULL NULL +5277 NULL NULL NULL NULL NULL NULL NULL +5277 NULL NULL NULL NULL NULL NULL NULL +5277 NULL NULL NULL NULL NULL NULL NULL +5277 NULL NULL NULL NULL NULL NULL NULL +5277 NULL NULL NULL NULL NULL NULL NULL +5277 NULL NULL NULL NULL NULL NULL NULL +5277 NULL NULL NULL NULL NULL NULL NULL +5277 NULL NULL NULL NULL NULL NULL NULL +5277 NULL NULL NULL NULL NULL NULL NULL +5277 NULL NULL NULL NULL NULL NULL NULL +5277 NULL NULL NULL NULL NULL NULL NULL +5277 NULL NULL NULL NULL NULL NULL NULL +5277 NULL NULL NULL NULL NULL NULL NULL +5277 NULL NULL NULL NULL NULL NULL NULL +5277 NULL NULL NULL NULL NULL NULL NULL +5277 NULL NULL NULL NULL NULL NULL NULL +5277 NULL NULL NULL NULL NULL NULL NULL +5277 NULL NULL NULL NULL NULL NULL NULL +5277 NULL NULL NULL NULL NULL NULL NULL +5277 NULL NULL NULL NULL NULL NULL NULL +5277 NULL NULL NULL NULL NULL NULL NULL +5277 NULL NULL NULL NULL NULL NULL NULL +5277 NULL NULL NULL NULL NULL NULL NULL +5277 NULL NULL NULL NULL NULL NULL NULL +5277 NULL NULL NULL NULL NULL NULL NULL +5277 NULL NULL NULL NULL NULL NULL NULL +5277 NULL NULL NULL NULL NULL NULL NULL +5277 NULL NULL NULL NULL NULL NULL NULL +5277 NULL NULL NULL NULL NULL NULL NULL +5277 NULL NULL NULL NULL NULL NULL NULL +5277 NULL NULL NULL NULL NULL NULL NULL +5277 NULL NULL NULL NULL NULL NULL NULL +5277 NULL NULL NULL NULL NULL NULL NULL +5277 NULL NULL NULL NULL NULL NULL NULL +5277 NULL NULL NULL NULL NULL NULL NULL +5277 NULL NULL NULL NULL NULL NULL NULL +5277 NULL NULL NULL NULL NULL NULL NULL +5277 NULL NULL NULL NULL NULL NULL NULL +5277 NULL NULL NULL NULL NULL NULL NULL +5277 NULL NULL NULL NULL NULL NULL NULL +5277 NULL NULL NULL NULL NULL NULL NULL +5277 NULL NULL NULL NULL NULL NULL NULL +5277 NULL NULL NULL NULL NULL NULL NULL +5277 NULL NULL NULL NULL NULL NULL NULL +5277 NULL NULL NULL NULL NULL NULL NULL +5277 NULL NULL NULL NULL NULL NULL NULL +5278 NULL NULL NULL NULL NULL NULL NULL +5278 NULL NULL NULL NULL NULL NULL NULL +5278 NULL NULL NULL NULL NULL NULL NULL +5278 NULL NULL NULL NULL NULL NULL NULL +5278 NULL NULL NULL NULL NULL NULL NULL +5278 NULL NULL NULL NULL NULL NULL NULL +5278 NULL NULL NULL NULL NULL NULL NULL +5278 NULL NULL NULL NULL NULL NULL NULL +5278 NULL NULL NULL NULL NULL NULL NULL +5278 NULL NULL NULL NULL NULL NULL NULL +5278 NULL NULL NULL NULL NULL NULL NULL +5278 NULL NULL NULL NULL NULL NULL NULL +5278 NULL NULL NULL NULL NULL NULL NULL +5278 NULL NULL NULL NULL NULL NULL NULL +5278 NULL NULL NULL NULL NULL NULL NULL +5278 NULL NULL NULL NULL NULL NULL NULL +5278 NULL NULL NULL NULL NULL NULL NULL +5278 NULL NULL NULL NULL NULL NULL NULL +5278 NULL NULL NULL NULL NULL NULL NULL +5278 NULL NULL NULL NULL NULL NULL NULL +5278 NULL NULL NULL NULL NULL NULL NULL +5278 NULL NULL NULL NULL NULL NULL NULL +5278 NULL NULL NULL NULL NULL NULL NULL +5278 NULL NULL NULL NULL NULL NULL NULL +5278 NULL NULL NULL NULL NULL NULL NULL +5278 NULL NULL NULL NULL NULL NULL NULL +5278 NULL NULL NULL NULL NULL NULL NULL +5278 NULL NULL NULL NULL NULL NULL NULL +5278 NULL NULL NULL NULL NULL NULL NULL +5278 NULL NULL NULL NULL NULL NULL NULL +5278 NULL NULL NULL NULL NULL NULL NULL +5278 NULL NULL NULL NULL NULL NULL NULL +5278 NULL NULL NULL NULL NULL NULL NULL +5278 NULL NULL NULL NULL NULL NULL NULL +5278 NULL NULL NULL NULL NULL NULL NULL +5278 NULL NULL NULL NULL NULL NULL NULL +5278 NULL NULL NULL NULL NULL NULL NULL +5278 NULL NULL NULL NULL NULL NULL NULL +5278 NULL NULL NULL NULL NULL NULL NULL +5278 NULL NULL NULL NULL NULL NULL NULL +5278 NULL NULL NULL NULL NULL NULL NULL +5278 NULL NULL NULL NULL NULL NULL NULL +5278 NULL NULL NULL NULL NULL NULL NULL +5278 NULL NULL NULL NULL NULL NULL NULL +5278 NULL NULL NULL NULL NULL NULL NULL +5278 NULL NULL NULL NULL NULL NULL NULL +5278 NULL NULL NULL NULL NULL NULL NULL +5278 NULL NULL NULL NULL NULL NULL NULL +5278 NULL NULL NULL NULL NULL NULL NULL +5279 NULL NULL NULL NULL NULL NULL NULL +5279 NULL NULL NULL NULL NULL NULL NULL +5279 NULL NULL NULL NULL NULL NULL NULL +5279 NULL NULL NULL NULL NULL NULL NULL +5279 NULL NULL NULL NULL NULL NULL NULL +5279 NULL NULL NULL NULL NULL NULL NULL +5279 NULL NULL NULL NULL NULL NULL NULL +5279 NULL NULL NULL NULL NULL NULL NULL +5279 NULL NULL NULL NULL NULL NULL NULL +5279 NULL NULL NULL NULL NULL NULL NULL +5279 NULL NULL NULL NULL NULL NULL NULL +5279 NULL NULL NULL NULL NULL NULL NULL +5279 NULL NULL NULL NULL NULL NULL NULL +5279 NULL NULL NULL NULL NULL NULL NULL +5279 NULL NULL NULL NULL NULL NULL NULL +5279 NULL NULL NULL NULL NULL NULL NULL +5279 NULL NULL NULL NULL NULL NULL NULL +5279 NULL NULL NULL NULL NULL NULL NULL +5279 NULL NULL NULL NULL NULL NULL NULL +5279 NULL NULL NULL NULL NULL NULL NULL +5279 NULL NULL NULL NULL NULL NULL NULL +5279 NULL NULL NULL NULL NULL NULL NULL +5279 NULL NULL NULL NULL NULL NULL NULL +5279 NULL NULL NULL NULL NULL NULL NULL +5279 NULL NULL NULL NULL NULL NULL NULL +5279 NULL NULL NULL NULL NULL NULL NULL +5279 NULL NULL NULL NULL NULL NULL NULL +5279 NULL NULL NULL NULL NULL NULL NULL +5279 NULL NULL NULL NULL NULL NULL NULL +5279 NULL NULL NULL NULL NULL NULL NULL +5279 NULL NULL NULL NULL NULL NULL NULL +5279 NULL NULL NULL NULL NULL NULL NULL +5279 NULL NULL NULL NULL NULL NULL NULL +5279 NULL NULL NULL NULL NULL NULL NULL +5279 NULL NULL NULL NULL NULL NULL NULL +5279 NULL NULL NULL NULL NULL NULL NULL +5279 NULL NULL NULL NULL NULL NULL NULL +5279 NULL NULL NULL NULL NULL NULL NULL +5279 NULL NULL NULL NULL NULL NULL NULL +5279 NULL NULL NULL NULL NULL NULL NULL +5279 NULL NULL NULL NULL NULL NULL NULL +5279 NULL NULL NULL NULL NULL NULL NULL +5279 NULL NULL NULL NULL NULL NULL NULL +5279 NULL NULL NULL NULL NULL NULL NULL +5279 NULL NULL NULL NULL NULL NULL NULL +5279 NULL NULL NULL NULL NULL NULL NULL +5279 NULL NULL NULL NULL NULL NULL NULL +5279 NULL NULL NULL NULL NULL NULL NULL +5279 NULL NULL NULL NULL NULL NULL NULL +5280 NULL NULL NULL NULL NULL NULL NULL +5280 NULL NULL NULL NULL NULL NULL NULL +5280 NULL NULL NULL NULL NULL NULL NULL +5280 NULL NULL NULL NULL NULL NULL NULL +5280 NULL NULL NULL NULL NULL NULL NULL +5280 NULL NULL NULL NULL NULL NULL NULL +5280 NULL NULL NULL NULL NULL NULL NULL +5280 NULL NULL NULL NULL NULL NULL NULL +5280 NULL NULL NULL NULL NULL NULL NULL +5280 NULL NULL NULL NULL NULL NULL NULL +5280 NULL NULL NULL NULL NULL NULL NULL +5280 NULL NULL NULL NULL NULL NULL NULL +5280 NULL NULL NULL NULL NULL NULL NULL +5280 NULL NULL NULL NULL NULL NULL NULL +5280 NULL NULL NULL NULL NULL NULL NULL +5280 NULL NULL NULL NULL NULL NULL NULL +5280 NULL NULL NULL NULL NULL NULL NULL +5280 NULL NULL NULL NULL NULL NULL NULL +5280 NULL NULL NULL NULL NULL NULL NULL +5280 NULL NULL NULL NULL NULL NULL NULL +5280 NULL NULL NULL NULL NULL NULL NULL +5280 NULL NULL NULL NULL NULL NULL NULL +5280 NULL NULL NULL NULL NULL NULL NULL +5280 NULL NULL NULL NULL NULL NULL NULL +5280 NULL NULL NULL NULL NULL NULL NULL +5280 NULL NULL NULL NULL NULL NULL NULL +5280 NULL NULL NULL NULL NULL NULL NULL +5280 NULL NULL NULL NULL NULL NULL NULL +5280 NULL NULL NULL NULL NULL NULL NULL +5280 NULL NULL NULL NULL NULL NULL NULL +5280 NULL NULL NULL NULL NULL NULL NULL +5280 NULL NULL NULL NULL NULL NULL NULL +5280 NULL NULL NULL NULL NULL NULL NULL +5280 NULL NULL NULL NULL NULL NULL NULL +5280 NULL NULL NULL NULL NULL NULL NULL +5280 NULL NULL NULL NULL NULL NULL NULL +5280 NULL NULL NULL NULL NULL NULL NULL +5280 NULL NULL NULL NULL NULL NULL NULL +5280 NULL NULL NULL NULL NULL NULL NULL +5280 NULL NULL NULL NULL NULL NULL NULL +5280 NULL NULL NULL NULL NULL NULL NULL +5280 NULL NULL NULL NULL NULL NULL NULL +5280 NULL NULL NULL NULL NULL NULL NULL +5280 NULL NULL NULL NULL NULL NULL NULL +5280 NULL NULL NULL NULL NULL NULL NULL +5280 NULL NULL NULL NULL NULL NULL NULL +5280 NULL NULL NULL NULL NULL NULL NULL +5280 NULL NULL NULL NULL NULL NULL NULL +5280 NULL NULL NULL NULL NULL NULL NULL +5282 NULL NULL NULL NULL NULL NULL NULL +5282 NULL NULL NULL NULL NULL NULL NULL +5282 NULL NULL NULL NULL NULL NULL NULL +5282 NULL NULL NULL NULL NULL NULL NULL +5282 NULL NULL NULL NULL NULL NULL NULL +5282 NULL NULL NULL NULL NULL NULL NULL +5282 NULL NULL NULL NULL NULL NULL NULL +5282 NULL NULL NULL NULL NULL NULL NULL +5282 NULL NULL NULL NULL NULL NULL NULL +5282 NULL NULL NULL NULL NULL NULL NULL +5282 NULL NULL NULL NULL NULL NULL NULL +5282 NULL NULL NULL NULL NULL NULL NULL +5282 NULL NULL NULL NULL NULL NULL NULL +5282 NULL NULL NULL NULL NULL NULL NULL +5282 NULL NULL NULL NULL NULL NULL NULL +5282 NULL NULL NULL NULL NULL NULL NULL +5282 NULL NULL NULL NULL NULL NULL NULL +5282 NULL NULL NULL NULL NULL NULL NULL +5282 NULL NULL NULL NULL NULL NULL NULL +5282 NULL NULL NULL NULL NULL NULL NULL +5282 NULL NULL NULL NULL NULL NULL NULL +5282 NULL NULL NULL NULL NULL NULL NULL +5282 NULL NULL NULL NULL NULL NULL NULL +5282 NULL NULL NULL NULL NULL NULL NULL +5282 NULL NULL NULL NULL NULL NULL NULL +5282 NULL NULL NULL NULL NULL NULL NULL +5282 NULL NULL NULL NULL NULL NULL NULL +5282 NULL NULL NULL NULL NULL NULL NULL +5282 NULL NULL NULL NULL NULL NULL NULL +5282 NULL NULL NULL NULL NULL NULL NULL +5282 NULL NULL NULL NULL NULL NULL NULL +5282 NULL NULL NULL NULL NULL NULL NULL +5282 NULL NULL NULL NULL NULL NULL NULL +5282 NULL NULL NULL NULL NULL NULL NULL +5282 NULL NULL NULL NULL NULL NULL NULL +5282 NULL NULL NULL NULL NULL NULL NULL +5282 NULL NULL NULL NULL NULL NULL NULL +5282 NULL NULL NULL NULL NULL NULL NULL +5282 NULL NULL NULL NULL NULL NULL NULL +5282 NULL NULL NULL NULL NULL NULL NULL +5282 NULL NULL NULL NULL NULL NULL NULL +5282 NULL NULL NULL NULL NULL NULL NULL +5282 NULL NULL NULL NULL NULL NULL NULL +5282 NULL NULL NULL NULL NULL NULL NULL +5282 NULL NULL NULL NULL NULL NULL NULL +5282 NULL NULL NULL NULL NULL NULL NULL +5282 NULL NULL NULL NULL NULL NULL NULL +5282 NULL NULL NULL NULL NULL NULL NULL +5282 NULL NULL NULL NULL NULL NULL NULL +5284 NULL NULL 0.99 0.73 NULL NULL 1.76 +5284 NULL NULL 0.99 0.73 NULL NULL 1.76 +5284 NULL NULL 0.99 0.73 NULL NULL 1.76 +5284 NULL NULL 0.99 0.73 NULL NULL 1.76 +5284 NULL NULL 0.99 0.73 NULL NULL 1.76 +5284 NULL NULL 0.99 0.73 NULL NULL 1.76 +5284 NULL NULL 0.99 0.73 NULL NULL 1.76 +5284 NULL NULL 0.99 0.73 NULL NULL 1.76 +5284 NULL NULL 0.99 0.73 NULL NULL 1.76 +5284 NULL NULL 0.99 0.73 NULL NULL 1.76 +5284 NULL NULL 0.99 0.73 NULL NULL 1.76 +5284 NULL NULL 0.99 0.73 NULL NULL 1.76 +5284 NULL NULL 0.99 0.73 NULL NULL 1.76 +5284 NULL NULL 0.99 0.73 NULL NULL 1.76 +5284 NULL NULL 0.99 0.73 NULL NULL 1.76 +5284 NULL NULL 0.99 0.73 NULL NULL 1.76 +5284 NULL NULL 0.99 0.73 NULL NULL 1.76 +5284 NULL NULL 0.99 0.73 NULL NULL 1.76 +5284 NULL NULL 0.99 0.73 NULL NULL 1.76 +5284 NULL NULL 0.99 0.73 NULL NULL 1.76 +5284 NULL NULL 0.99 0.73 NULL NULL 1.76 +5284 NULL NULL 0.99 0.73 NULL NULL 1.76 +5284 NULL NULL 0.99 0.73 NULL NULL 1.76 +5284 NULL NULL 0.99 0.73 NULL NULL 1.76 +5284 NULL NULL 0.99 0.73 NULL NULL 1.76 +5284 NULL NULL 0.99 0.73 NULL NULL 1.76 +5284 NULL NULL 0.99 0.73 NULL NULL 1.76 +5284 NULL NULL 0.99 0.73 NULL NULL 1.76 +5284 NULL NULL 0.99 0.73 NULL NULL 1.76 +5284 NULL NULL 0.99 0.73 NULL NULL 1.76 +5284 NULL NULL 0.99 0.73 NULL NULL 1.76 +5284 NULL NULL 0.99 0.73 NULL NULL 1.76 +5284 NULL NULL 0.99 0.73 NULL NULL 1.76 +5284 NULL NULL 0.99 0.73 NULL NULL 1.76 +5284 NULL NULL 0.99 0.73 NULL NULL 1.76 +5284 NULL NULL 0.99 0.73 NULL NULL 1.76 +5284 NULL NULL 0.99 0.73 NULL NULL 1.76 +5284 NULL NULL 0.99 0.73 NULL NULL 1.76 +5284 NULL NULL 0.99 0.73 NULL NULL 1.76 +5284 NULL NULL 0.99 0.73 NULL NULL 1.76 +5284 NULL NULL 0.99 0.73 NULL NULL 1.76 +5284 NULL NULL 0.99 0.73 NULL NULL 1.76 +5284 NULL NULL 0.99 0.73 NULL NULL 1.76 +5284 NULL NULL 0.99 0.73 NULL NULL 1.76 +5284 NULL NULL 0.99 0.73 NULL NULL 1.76 +5284 NULL NULL 0.99 0.73 NULL NULL 1.76 +5284 NULL NULL 0.99 0.73 NULL NULL 1.76 +5284 NULL NULL 0.99 0.73 NULL NULL 1.76 +5284 NULL NULL 0.99 0.73 NULL NULL 1.76 +5285 NULL NULL NULL NULL NULL NULL NULL +5285 NULL NULL NULL NULL NULL NULL NULL +5285 NULL NULL NULL NULL NULL NULL NULL +5285 NULL NULL NULL NULL NULL NULL NULL +5285 NULL NULL NULL NULL NULL NULL NULL +5285 NULL NULL NULL NULL NULL NULL NULL +5285 NULL NULL NULL NULL NULL NULL NULL +5285 NULL NULL NULL NULL NULL NULL NULL +5285 NULL NULL NULL NULL NULL NULL NULL +5285 NULL NULL NULL NULL NULL NULL NULL +5285 NULL NULL NULL NULL NULL NULL NULL +5285 NULL NULL NULL NULL NULL NULL NULL +5285 NULL NULL NULL NULL NULL NULL NULL +5285 NULL NULL NULL NULL NULL NULL NULL +5285 NULL NULL NULL NULL NULL NULL NULL +5285 NULL NULL NULL NULL NULL NULL NULL +5285 NULL NULL NULL NULL NULL NULL NULL +5285 NULL NULL NULL NULL NULL NULL NULL +5285 NULL NULL NULL NULL NULL NULL NULL +5285 NULL NULL NULL NULL NULL NULL NULL +5285 NULL NULL NULL NULL NULL NULL NULL +5285 NULL NULL NULL NULL NULL NULL NULL +5285 NULL NULL NULL NULL NULL NULL NULL +5285 NULL NULL NULL NULL NULL NULL NULL +5285 NULL NULL NULL NULL NULL NULL NULL +5285 NULL NULL NULL NULL NULL NULL NULL +5285 NULL NULL NULL NULL NULL NULL NULL +5285 NULL NULL NULL NULL NULL NULL NULL +5285 NULL NULL NULL NULL NULL NULL NULL +5285 NULL NULL NULL NULL NULL NULL NULL +5285 NULL NULL NULL NULL NULL NULL NULL +5285 NULL NULL NULL NULL NULL NULL NULL +5285 NULL NULL NULL NULL NULL NULL NULL +5285 NULL NULL NULL NULL NULL NULL NULL +5285 NULL NULL NULL NULL NULL NULL NULL +5285 NULL NULL NULL NULL NULL NULL NULL +5285 NULL NULL NULL NULL NULL NULL NULL +5285 NULL NULL NULL NULL NULL NULL NULL +5285 NULL NULL NULL NULL NULL NULL NULL +5285 NULL NULL NULL NULL NULL NULL NULL +5285 NULL NULL NULL NULL NULL NULL NULL +5285 NULL NULL NULL NULL NULL NULL NULL +5285 NULL NULL NULL NULL NULL NULL NULL +5285 NULL NULL NULL NULL NULL NULL NULL +5285 NULL NULL NULL NULL NULL NULL NULL +5285 NULL NULL NULL NULL NULL NULL NULL +5285 NULL NULL NULL NULL NULL NULL NULL +5285 NULL NULL NULL NULL NULL NULL NULL +5285 NULL NULL NULL NULL NULL NULL NULL +5287 NULL NULL NULL 1.45 NULL NULL NULL +5287 NULL NULL NULL 1.45 NULL NULL NULL +5287 NULL NULL NULL 1.45 NULL NULL NULL +5287 NULL NULL NULL 1.45 NULL NULL NULL +5287 NULL NULL NULL 1.45 NULL NULL NULL +5287 NULL NULL NULL 1.45 NULL NULL NULL +5287 NULL NULL NULL 1.45 NULL NULL NULL +5287 NULL NULL NULL 1.45 NULL NULL NULL +5287 NULL NULL NULL 1.45 NULL NULL NULL +5287 NULL NULL NULL 1.45 NULL NULL NULL +5287 NULL NULL NULL 1.45 NULL NULL NULL +5287 NULL NULL NULL 1.45 NULL NULL NULL +5287 NULL NULL NULL 1.45 NULL NULL NULL +5287 NULL NULL NULL 1.45 NULL NULL NULL +5287 NULL NULL NULL 1.45 NULL NULL NULL +5287 NULL NULL NULL 1.45 NULL NULL NULL +5287 NULL NULL NULL 1.45 NULL NULL NULL +5287 NULL NULL NULL 1.45 NULL NULL NULL +5287 NULL NULL NULL 1.45 NULL NULL NULL +5287 NULL NULL NULL 1.45 NULL NULL NULL +5287 NULL NULL NULL 1.45 NULL NULL NULL +5287 NULL NULL NULL 1.45 NULL NULL NULL +5287 NULL NULL NULL 1.45 NULL NULL NULL +5287 NULL NULL NULL 1.45 NULL NULL NULL +5287 NULL NULL NULL 1.45 NULL NULL NULL +5287 NULL NULL NULL 1.45 NULL NULL NULL +5287 NULL NULL NULL 1.45 NULL NULL NULL +5287 NULL NULL NULL 1.45 NULL NULL NULL +5287 NULL NULL NULL 1.45 NULL NULL NULL +5287 NULL NULL NULL 1.45 NULL NULL NULL +5287 NULL NULL NULL 1.45 NULL NULL NULL +5287 NULL NULL NULL 1.45 NULL NULL NULL +5287 NULL NULL NULL 1.45 NULL NULL NULL +5287 NULL NULL NULL 1.45 NULL NULL NULL +5287 NULL NULL NULL 1.45 NULL NULL NULL +5287 NULL NULL NULL 1.45 NULL NULL NULL +5287 NULL NULL NULL 1.45 NULL NULL NULL +5287 NULL NULL NULL 1.45 NULL NULL NULL +5287 NULL NULL NULL 1.45 NULL NULL NULL +5287 NULL NULL NULL 1.45 NULL NULL NULL +5287 NULL NULL NULL 1.45 NULL NULL NULL +5287 NULL NULL NULL 1.45 NULL NULL NULL +5287 NULL NULL NULL 1.45 NULL NULL NULL +5287 NULL NULL NULL 1.45 NULL NULL NULL +5287 NULL NULL NULL 1.45 NULL NULL NULL +5287 NULL NULL NULL 1.45 NULL NULL NULL +5287 NULL NULL NULL 1.45 NULL NULL NULL +5287 NULL NULL NULL 1.45 NULL NULL NULL +5287 NULL NULL NULL 1.45 NULL NULL NULL +5292 NULL NULL NULL NULL NULL NULL NULL +5292 NULL NULL NULL NULL NULL NULL NULL +5292 NULL NULL NULL NULL NULL NULL NULL +5292 NULL NULL NULL NULL NULL NULL NULL +5292 NULL NULL NULL NULL NULL NULL NULL +5292 NULL NULL NULL NULL NULL NULL NULL +5292 NULL NULL NULL NULL NULL NULL NULL +5292 NULL NULL NULL NULL NULL NULL NULL +5292 NULL NULL NULL NULL NULL NULL NULL +5292 NULL NULL NULL NULL NULL NULL NULL +5292 NULL NULL NULL NULL NULL NULL NULL +5292 NULL NULL NULL NULL NULL NULL NULL +5292 NULL NULL NULL NULL NULL NULL NULL +5292 NULL NULL NULL NULL NULL NULL NULL +5292 NULL NULL NULL NULL NULL NULL NULL +5292 NULL NULL NULL NULL NULL NULL NULL +5292 NULL NULL NULL NULL NULL NULL NULL +5292 NULL NULL NULL NULL NULL NULL NULL +5292 NULL NULL NULL NULL NULL NULL NULL +5292 NULL NULL NULL NULL NULL NULL NULL +5292 NULL NULL NULL NULL NULL NULL NULL +5292 NULL NULL NULL NULL NULL NULL NULL +5292 NULL NULL NULL NULL NULL NULL NULL +5292 NULL NULL NULL NULL NULL NULL NULL +5292 NULL NULL NULL NULL NULL NULL NULL +5292 NULL NULL NULL NULL NULL NULL NULL +5292 NULL NULL NULL NULL NULL NULL NULL +5292 NULL NULL NULL NULL NULL NULL NULL +5292 NULL NULL NULL NULL NULL NULL NULL +5292 NULL NULL NULL NULL NULL NULL NULL +5292 NULL NULL NULL NULL NULL NULL NULL +5292 NULL NULL NULL NULL NULL NULL NULL +5292 NULL NULL NULL NULL NULL NULL NULL +5292 NULL NULL NULL NULL NULL NULL NULL +5292 NULL NULL NULL NULL NULL NULL NULL +5292 NULL NULL NULL NULL NULL NULL NULL +5292 NULL NULL NULL NULL NULL NULL NULL +5292 NULL NULL NULL NULL NULL NULL NULL +5292 NULL NULL NULL NULL NULL NULL NULL +5292 NULL NULL NULL NULL NULL NULL NULL +5292 NULL NULL NULL NULL NULL NULL NULL +5292 NULL NULL NULL NULL NULL NULL NULL +5292 NULL NULL NULL NULL NULL NULL NULL +5292 NULL NULL NULL NULL NULL NULL NULL +5292 NULL NULL NULL NULL NULL NULL NULL +5292 NULL NULL NULL NULL NULL NULL NULL +5292 NULL NULL NULL NULL NULL NULL NULL +5292 NULL NULL NULL NULL NULL NULL NULL +5292 NULL NULL NULL NULL NULL NULL NULL +5293 1.79 NULL NULL NULL NULL NULL NULL +5293 1.79 NULL NULL NULL NULL NULL NULL +5293 1.79 NULL NULL NULL NULL NULL NULL +5293 1.79 NULL NULL NULL NULL NULL NULL +5293 1.79 NULL NULL NULL NULL NULL NULL +5293 1.79 NULL NULL NULL NULL NULL NULL +5293 1.79 NULL NULL NULL NULL NULL NULL +5293 1.79 NULL NULL NULL NULL NULL NULL +5293 1.79 NULL NULL NULL NULL NULL NULL +5293 1.79 NULL NULL NULL NULL NULL NULL +5293 1.79 NULL NULL NULL NULL NULL NULL +5293 1.79 NULL NULL NULL NULL NULL NULL +5293 1.79 NULL NULL NULL NULL NULL NULL +5293 1.79 NULL NULL NULL NULL NULL NULL +5293 1.79 NULL NULL NULL NULL NULL NULL +5293 1.79 NULL NULL NULL NULL NULL NULL +5293 1.79 NULL NULL NULL NULL NULL NULL +5293 1.79 NULL NULL NULL NULL NULL NULL +5293 1.79 NULL NULL NULL NULL NULL NULL +5293 1.79 NULL NULL NULL NULL NULL NULL +5293 1.79 NULL NULL NULL NULL NULL NULL +5293 1.79 NULL NULL NULL NULL NULL NULL +5293 1.79 NULL NULL NULL NULL NULL NULL +5293 1.79 NULL NULL NULL NULL NULL NULL +5293 1.79 NULL NULL NULL NULL NULL NULL +5293 1.79 NULL NULL NULL NULL NULL NULL +5293 1.79 NULL NULL NULL NULL NULL NULL +5293 1.79 NULL NULL NULL NULL NULL NULL +5293 1.79 NULL NULL NULL NULL NULL NULL +5293 1.79 NULL NULL NULL NULL NULL NULL +5293 1.79 NULL NULL NULL NULL NULL NULL +5293 1.79 NULL NULL NULL NULL NULL NULL +5293 1.79 NULL NULL NULL NULL NULL NULL +5293 1.79 NULL NULL NULL NULL NULL NULL +5293 1.79 NULL NULL NULL NULL NULL NULL +5293 1.79 NULL NULL NULL NULL NULL NULL +5293 1.79 NULL NULL NULL NULL NULL NULL +5293 1.79 NULL NULL NULL NULL NULL NULL +5293 1.79 NULL NULL NULL NULL NULL NULL +5293 1.79 NULL NULL NULL NULL NULL NULL +5293 1.79 NULL NULL NULL NULL NULL NULL +5293 1.79 NULL NULL NULL NULL NULL NULL +5293 1.79 NULL NULL NULL NULL NULL NULL +5293 1.79 NULL NULL NULL NULL NULL NULL +5293 1.79 NULL NULL NULL NULL NULL NULL +5293 1.79 NULL NULL NULL NULL NULL NULL +5293 1.79 NULL NULL NULL NULL NULL NULL +5293 1.79 NULL NULL NULL NULL NULL NULL +5293 1.79 NULL NULL NULL NULL NULL NULL +5296 NULL NULL NULL NULL NULL NULL NULL +5296 NULL NULL NULL NULL NULL NULL NULL +5296 NULL NULL NULL NULL NULL NULL NULL +5296 NULL NULL NULL NULL NULL NULL NULL +5296 NULL NULL NULL NULL NULL NULL NULL +5296 NULL NULL NULL NULL NULL NULL NULL +5296 NULL NULL NULL NULL NULL NULL NULL +5296 NULL NULL NULL NULL NULL NULL NULL +5296 NULL NULL NULL NULL NULL NULL NULL +5296 NULL NULL NULL NULL NULL NULL NULL +5296 NULL NULL NULL NULL NULL NULL NULL +5296 NULL NULL NULL NULL NULL NULL NULL +5296 NULL NULL NULL NULL NULL NULL NULL +5296 NULL NULL NULL NULL NULL NULL NULL +5296 NULL NULL NULL NULL NULL NULL NULL +5296 NULL NULL NULL NULL NULL NULL NULL +5296 NULL NULL NULL NULL NULL NULL NULL +5296 NULL NULL NULL NULL NULL NULL NULL +5296 NULL NULL NULL NULL NULL NULL NULL +5296 NULL NULL NULL NULL NULL NULL NULL +5296 NULL NULL NULL NULL NULL NULL NULL +5296 NULL NULL NULL NULL NULL NULL NULL +5296 NULL NULL NULL NULL NULL NULL NULL +5296 NULL NULL NULL NULL NULL NULL NULL +5296 NULL NULL NULL NULL NULL NULL NULL +5296 NULL NULL NULL NULL NULL NULL NULL +5296 NULL NULL NULL NULL NULL NULL NULL +5296 NULL NULL NULL NULL NULL NULL NULL +5296 NULL NULL NULL NULL NULL NULL NULL +5296 NULL NULL NULL NULL NULL NULL NULL +5296 NULL NULL NULL NULL NULL NULL NULL +5296 NULL NULL NULL NULL NULL NULL NULL +5296 NULL NULL NULL NULL NULL NULL NULL +5296 NULL NULL NULL NULL NULL NULL NULL +5296 NULL NULL NULL NULL NULL NULL NULL +5296 NULL NULL NULL NULL NULL NULL NULL +5296 NULL NULL NULL NULL NULL NULL NULL +5296 NULL NULL NULL NULL NULL NULL NULL +5296 NULL NULL NULL NULL NULL NULL NULL +5296 NULL NULL NULL NULL NULL NULL NULL +5296 NULL NULL NULL NULL NULL NULL NULL +5296 NULL NULL NULL NULL NULL NULL NULL +5296 NULL NULL NULL NULL NULL NULL NULL +5296 NULL NULL NULL NULL NULL NULL NULL +5296 NULL NULL NULL NULL NULL NULL NULL +5296 NULL NULL NULL NULL NULL NULL NULL +5296 NULL NULL NULL NULL NULL NULL NULL +5296 NULL NULL NULL NULL NULL NULL NULL +5296 NULL NULL NULL NULL NULL NULL NULL +5301 NULL NULL 1.35 NULL NULL NULL NULL +5301 NULL NULL 1.35 NULL NULL NULL NULL +5301 NULL NULL 1.35 NULL NULL NULL NULL +5301 NULL NULL 1.35 NULL NULL NULL NULL +5301 NULL NULL 1.35 NULL NULL NULL NULL +5301 NULL NULL 1.35 NULL NULL NULL NULL +5301 NULL NULL 1.35 NULL NULL NULL NULL +5301 NULL NULL 1.35 NULL NULL NULL NULL +5301 NULL NULL 1.35 NULL NULL NULL NULL +5301 NULL NULL 1.35 NULL NULL NULL NULL +5301 NULL NULL 1.35 NULL NULL NULL NULL +5301 NULL NULL 1.35 NULL NULL NULL NULL +5301 NULL NULL 1.35 NULL NULL NULL NULL +5301 NULL NULL 1.35 NULL NULL NULL NULL +5301 NULL NULL 1.35 NULL NULL NULL NULL +5301 NULL NULL 1.35 NULL NULL NULL NULL +5301 NULL NULL 1.35 NULL NULL NULL NULL +5301 NULL NULL 1.35 NULL NULL NULL NULL +5301 NULL NULL 1.35 NULL NULL NULL NULL +5301 NULL NULL 1.35 NULL NULL NULL NULL +5301 NULL NULL 1.35 NULL NULL NULL NULL +5301 NULL NULL 1.35 NULL NULL NULL NULL +5301 NULL NULL 1.35 NULL NULL NULL NULL +5301 NULL NULL 1.35 NULL NULL NULL NULL +5301 NULL NULL 1.35 NULL NULL NULL NULL +5301 NULL NULL 1.35 NULL NULL NULL NULL +5301 NULL NULL 1.35 NULL NULL NULL NULL +5301 NULL NULL 1.35 NULL NULL NULL NULL +5301 NULL NULL 1.35 NULL NULL NULL NULL +5301 NULL NULL 1.35 NULL NULL NULL NULL +5301 NULL NULL 1.35 NULL NULL NULL NULL +5301 NULL NULL 1.35 NULL NULL NULL NULL +5301 NULL NULL 1.35 NULL NULL NULL NULL +5301 NULL NULL 1.35 NULL NULL NULL NULL +5301 NULL NULL 1.35 NULL NULL NULL NULL +5301 NULL NULL 1.35 NULL NULL NULL NULL +5301 NULL NULL 1.35 NULL NULL NULL NULL +5301 NULL NULL 1.35 NULL NULL NULL NULL +5301 NULL NULL 1.35 NULL NULL NULL NULL +5301 NULL NULL 1.35 NULL NULL NULL NULL +5301 NULL NULL 1.35 NULL NULL NULL NULL +5301 NULL NULL 1.35 NULL NULL NULL NULL +5301 NULL NULL 1.35 NULL NULL NULL NULL +5301 NULL NULL 1.35 NULL NULL NULL NULL +5301 NULL NULL 1.35 NULL NULL NULL NULL +5301 NULL NULL 1.35 NULL NULL NULL NULL +5301 NULL NULL 1.35 NULL NULL NULL NULL +5301 NULL NULL 1.35 NULL NULL NULL NULL +5301 NULL NULL 1.35 NULL NULL NULL NULL +5302 NULL NULL NULL NULL NULL NULL NULL +5302 NULL NULL NULL NULL NULL NULL NULL +5302 NULL NULL NULL NULL NULL NULL NULL +5302 NULL NULL NULL NULL NULL NULL NULL +5302 NULL NULL NULL NULL NULL NULL NULL +5302 NULL NULL NULL NULL NULL NULL NULL +5302 NULL NULL NULL NULL NULL NULL NULL +5302 NULL NULL NULL NULL NULL NULL NULL +5302 NULL NULL NULL NULL NULL NULL NULL +5302 NULL NULL NULL NULL NULL NULL NULL +5302 NULL NULL NULL NULL NULL NULL NULL +5302 NULL NULL NULL NULL NULL NULL NULL +5302 NULL NULL NULL NULL NULL NULL NULL +5302 NULL NULL NULL NULL NULL NULL NULL +5302 NULL NULL NULL NULL NULL NULL NULL +5302 NULL NULL NULL NULL NULL NULL NULL +5302 NULL NULL NULL NULL NULL NULL NULL +5302 NULL NULL NULL NULL NULL NULL NULL +5302 NULL NULL NULL NULL NULL NULL NULL +5302 NULL NULL NULL NULL NULL NULL NULL +5302 NULL NULL NULL NULL NULL NULL NULL +5302 NULL NULL NULL NULL NULL NULL NULL +5302 NULL NULL NULL NULL NULL NULL NULL +5302 NULL NULL NULL NULL NULL NULL NULL +5302 NULL NULL NULL NULL NULL NULL NULL +5302 NULL NULL NULL NULL NULL NULL NULL +5302 NULL NULL NULL NULL NULL NULL NULL +5302 NULL NULL NULL NULL NULL NULL NULL +5302 NULL NULL NULL NULL NULL NULL NULL +5302 NULL NULL NULL NULL NULL NULL NULL +5302 NULL NULL NULL NULL NULL NULL NULL +5302 NULL NULL NULL NULL NULL NULL NULL +5302 NULL NULL NULL NULL NULL NULL NULL +5302 NULL NULL NULL NULL NULL NULL NULL +5302 NULL NULL NULL NULL NULL NULL NULL +5302 NULL NULL NULL NULL NULL NULL NULL +5302 NULL NULL NULL NULL NULL NULL NULL +5302 NULL NULL NULL NULL NULL NULL NULL +5302 NULL NULL NULL NULL NULL NULL NULL +5302 NULL NULL NULL NULL NULL NULL NULL +5302 NULL NULL NULL NULL NULL NULL NULL +5302 NULL NULL NULL NULL NULL NULL NULL +5302 NULL NULL NULL NULL NULL NULL NULL +5302 NULL NULL NULL NULL NULL NULL NULL +5302 NULL NULL NULL NULL NULL NULL NULL +5302 NULL NULL NULL NULL NULL NULL NULL +5302 NULL NULL NULL NULL NULL NULL NULL +5302 NULL NULL NULL NULL NULL NULL NULL +5302 NULL NULL NULL NULL NULL NULL NULL +5304 NULL NULL 1.27 NULL NULL NULL NULL +5304 NULL NULL 1.27 NULL NULL NULL NULL +5304 NULL NULL 1.27 NULL NULL NULL NULL +5304 NULL NULL 1.27 NULL NULL NULL NULL +5304 NULL NULL 1.27 NULL NULL NULL NULL +5304 NULL NULL 1.27 NULL NULL NULL NULL +5304 NULL NULL 1.27 NULL NULL NULL NULL +5304 NULL NULL 1.27 NULL NULL NULL NULL +5304 NULL NULL 1.27 NULL NULL NULL NULL +5304 NULL NULL 1.27 NULL NULL NULL NULL +5304 NULL NULL 1.27 NULL NULL NULL NULL +5304 NULL NULL 1.27 NULL NULL NULL NULL +5304 NULL NULL 1.27 NULL NULL NULL NULL +5304 NULL NULL 1.27 NULL NULL NULL NULL +5304 NULL NULL 1.27 NULL NULL NULL NULL +5304 NULL NULL 1.27 NULL NULL NULL NULL +5304 NULL NULL 1.27 NULL NULL NULL NULL +5304 NULL NULL 1.27 NULL NULL NULL NULL +5304 NULL NULL 1.27 NULL NULL NULL NULL +5304 NULL NULL 1.27 NULL NULL NULL NULL +5304 NULL NULL 1.27 NULL NULL NULL NULL +5304 NULL NULL 1.27 NULL NULL NULL NULL +5304 NULL NULL 1.27 NULL NULL NULL NULL +5304 NULL NULL 1.27 NULL NULL NULL NULL +5304 NULL NULL 1.27 NULL NULL NULL NULL +5304 NULL NULL 1.27 NULL NULL NULL NULL +5304 NULL NULL 1.27 NULL NULL NULL NULL +5304 NULL NULL 1.27 NULL NULL NULL NULL +5304 NULL NULL 1.27 NULL NULL NULL NULL +5304 NULL NULL 1.27 NULL NULL NULL NULL +5304 NULL NULL 1.27 NULL NULL NULL NULL +5304 NULL NULL 1.27 NULL NULL NULL NULL +5304 NULL NULL 1.27 NULL NULL NULL NULL +5304 NULL NULL 1.27 NULL NULL NULL NULL +5304 NULL NULL 1.27 NULL NULL NULL NULL +5304 NULL NULL 1.27 NULL NULL NULL NULL +5304 NULL NULL 1.27 NULL NULL NULL NULL +5304 NULL NULL 1.27 NULL NULL NULL NULL +5304 NULL NULL 1.27 NULL NULL NULL NULL +5304 NULL NULL 1.27 NULL NULL NULL NULL +5304 NULL NULL 1.27 NULL NULL NULL NULL +5304 NULL NULL 1.27 NULL NULL NULL NULL +5304 NULL NULL 1.27 NULL NULL NULL NULL +5304 NULL NULL 1.27 NULL NULL NULL NULL +5304 NULL NULL 1.27 NULL NULL NULL NULL +5304 NULL NULL 1.27 NULL NULL NULL NULL +5304 NULL NULL 1.27 NULL NULL NULL NULL +5304 NULL NULL 1.27 NULL NULL NULL NULL +5304 NULL NULL 1.27 NULL NULL NULL NULL +5305 0.50 1.03 0.47 NULL 0.72 NULL NULL +5305 0.50 1.03 0.47 NULL 0.72 NULL NULL +5305 0.50 1.03 0.47 NULL 0.72 NULL NULL +5305 0.50 1.03 0.47 NULL 0.72 NULL NULL +5305 0.50 1.03 0.47 NULL 0.72 NULL NULL +5305 0.50 1.03 0.47 NULL 0.72 NULL NULL +5305 0.50 1.03 0.47 NULL 0.72 NULL NULL +5305 0.50 1.03 0.47 NULL 0.72 NULL NULL +5305 0.50 1.03 0.47 NULL 0.72 NULL NULL +5305 0.50 1.03 0.47 NULL 0.72 NULL NULL +5305 0.50 1.03 0.47 NULL 0.72 NULL NULL +5305 0.50 1.03 0.47 NULL 0.72 NULL NULL +5305 0.50 1.03 0.47 NULL 0.72 NULL NULL +5305 0.50 1.03 0.47 NULL 0.72 NULL NULL +5305 0.50 1.03 0.47 NULL 0.72 NULL NULL +5305 0.50 1.03 0.47 NULL 0.72 NULL NULL +5305 0.50 1.03 0.47 NULL 0.72 NULL NULL +5305 0.50 1.03 0.47 NULL 0.72 NULL NULL +5305 0.50 1.03 0.47 NULL 0.72 NULL NULL +5305 0.50 1.03 0.47 NULL 0.72 NULL NULL +5305 0.50 1.03 0.47 NULL 0.72 NULL NULL +5305 0.50 1.03 0.47 NULL 0.72 NULL NULL +5305 0.50 1.03 0.47 NULL 0.72 NULL NULL +5305 0.50 1.03 0.47 NULL 0.72 NULL NULL +5305 0.50 1.03 0.47 NULL 0.72 NULL NULL +5305 0.50 1.03 0.47 NULL 0.72 NULL NULL +5305 0.50 1.03 0.47 NULL 0.72 NULL NULL +5305 0.50 1.03 0.47 NULL 0.72 NULL NULL +5305 0.50 1.03 0.47 NULL 0.72 NULL NULL +5305 0.50 1.03 0.47 NULL 0.72 NULL NULL +5305 0.50 1.03 0.47 NULL 0.72 NULL NULL +5305 0.50 1.03 0.47 NULL 0.72 NULL NULL +5305 0.50 1.03 0.47 NULL 0.72 NULL NULL +5305 0.50 1.03 0.47 NULL 0.72 NULL NULL +5305 0.50 1.03 0.47 NULL 0.72 NULL NULL +5305 0.50 1.03 0.47 NULL 0.72 NULL NULL +5305 0.50 1.03 0.47 NULL 0.72 NULL NULL +5305 0.50 1.03 0.47 NULL 0.72 NULL NULL +5305 0.50 1.03 0.47 NULL 0.72 NULL NULL +5305 0.50 1.03 0.47 NULL 0.72 NULL NULL +5305 0.50 1.03 0.47 NULL 0.72 NULL NULL +5305 0.50 1.03 0.47 NULL 0.72 NULL NULL +5305 0.50 1.03 0.47 NULL 0.72 NULL NULL +5305 0.50 1.03 0.47 NULL 0.72 NULL NULL +5305 0.50 1.03 0.47 NULL 0.72 NULL NULL +5305 0.50 1.03 0.47 NULL 0.72 NULL NULL +5305 0.50 1.03 0.47 NULL 0.72 NULL NULL +5305 0.50 1.03 0.47 NULL 0.72 NULL NULL +5305 0.50 1.03 0.47 NULL 0.72 NULL NULL +5306 NULL NULL NULL NULL NULL NULL NULL +5306 NULL NULL NULL NULL NULL NULL NULL +5306 NULL NULL NULL NULL NULL NULL NULL +5306 NULL NULL NULL NULL NULL NULL NULL +5306 NULL NULL NULL NULL NULL NULL NULL +5306 NULL NULL NULL NULL NULL NULL NULL +5306 NULL NULL NULL NULL NULL NULL NULL +5306 NULL NULL NULL NULL NULL NULL NULL +5306 NULL NULL NULL NULL NULL NULL NULL +5306 NULL NULL NULL NULL NULL NULL NULL +5306 NULL NULL NULL NULL NULL NULL NULL +5306 NULL NULL NULL NULL NULL NULL NULL +5306 NULL NULL NULL NULL NULL NULL NULL +5306 NULL NULL NULL NULL NULL NULL NULL +5306 NULL NULL NULL NULL NULL NULL NULL +5306 NULL NULL NULL NULL NULL NULL NULL +5306 NULL NULL NULL NULL NULL NULL NULL +5306 NULL NULL NULL NULL NULL NULL NULL +5306 NULL NULL NULL NULL NULL NULL NULL +5306 NULL NULL NULL NULL NULL NULL NULL +5306 NULL NULL NULL NULL NULL NULL NULL +5306 NULL NULL NULL NULL NULL NULL NULL +5306 NULL NULL NULL NULL NULL NULL NULL +5306 NULL NULL NULL NULL NULL NULL NULL +5306 NULL NULL NULL NULL NULL NULL NULL +5306 NULL NULL NULL NULL NULL NULL NULL +5306 NULL NULL NULL NULL NULL NULL NULL +5306 NULL NULL NULL NULL NULL NULL NULL +5306 NULL NULL NULL NULL NULL NULL NULL +5306 NULL NULL NULL NULL NULL NULL NULL +5306 NULL NULL NULL NULL NULL NULL NULL +5306 NULL NULL NULL NULL NULL NULL NULL +5306 NULL NULL NULL NULL NULL NULL NULL +5306 NULL NULL NULL NULL NULL NULL NULL +5306 NULL NULL NULL NULL NULL NULL NULL +5306 NULL NULL NULL NULL NULL NULL NULL +5306 NULL NULL NULL NULL NULL NULL NULL +5306 NULL NULL NULL NULL NULL NULL NULL +5306 NULL NULL NULL NULL NULL NULL NULL +5306 NULL NULL NULL NULL NULL NULL NULL +5306 NULL NULL NULL NULL NULL NULL NULL +5306 NULL NULL NULL NULL NULL NULL NULL +5306 NULL NULL NULL NULL NULL NULL NULL +5306 NULL NULL NULL NULL NULL NULL NULL +5306 NULL NULL NULL NULL NULL NULL NULL +5306 NULL NULL NULL NULL NULL NULL NULL +5306 NULL NULL NULL NULL NULL NULL NULL +5306 NULL NULL NULL NULL NULL NULL NULL +5306 NULL NULL NULL NULL NULL NULL NULL +5307 1.31 NULL NULL NULL NULL NULL NULL +5307 1.31 NULL NULL NULL NULL NULL NULL +5307 1.31 NULL NULL NULL NULL NULL NULL +5307 1.31 NULL NULL NULL NULL NULL NULL +5307 1.31 NULL NULL NULL NULL NULL NULL +5307 1.31 NULL NULL NULL NULL NULL NULL +5307 1.31 NULL NULL NULL NULL NULL NULL +5307 1.31 NULL NULL NULL NULL NULL NULL +5307 1.31 NULL NULL NULL NULL NULL NULL +5307 1.31 NULL NULL NULL NULL NULL NULL +5307 1.31 NULL NULL NULL NULL NULL NULL +5307 1.31 NULL NULL NULL NULL NULL NULL +5307 1.31 NULL NULL NULL NULL NULL NULL +5307 1.31 NULL NULL NULL NULL NULL NULL +5307 1.31 NULL NULL NULL NULL NULL NULL +5307 1.31 NULL NULL NULL NULL NULL NULL +5307 1.31 NULL NULL NULL NULL NULL NULL +5307 1.31 NULL NULL NULL NULL NULL NULL +5307 1.31 NULL NULL NULL NULL NULL NULL +5307 1.31 NULL NULL NULL NULL NULL NULL +5307 1.31 NULL NULL NULL NULL NULL NULL +5307 1.31 NULL NULL NULL NULL NULL NULL +5307 1.31 NULL NULL NULL NULL NULL NULL +5307 1.31 NULL NULL NULL NULL NULL NULL +5307 1.31 NULL NULL NULL NULL NULL NULL +5307 1.31 NULL NULL NULL NULL NULL NULL +5307 1.31 NULL NULL NULL NULL NULL NULL +5307 1.31 NULL NULL NULL NULL NULL NULL +5307 1.31 NULL NULL NULL NULL NULL NULL +5307 1.31 NULL NULL NULL NULL NULL NULL +5307 1.31 NULL NULL NULL NULL NULL NULL +5307 1.31 NULL NULL NULL NULL NULL NULL +5307 1.31 NULL NULL NULL NULL NULL NULL +5307 1.31 NULL NULL NULL NULL NULL NULL +5307 1.31 NULL NULL NULL NULL NULL NULL +5307 1.31 NULL NULL NULL NULL NULL NULL +5307 1.31 NULL NULL NULL NULL NULL NULL +5307 1.31 NULL NULL NULL NULL NULL NULL +5307 1.31 NULL NULL NULL NULL NULL NULL +5307 1.31 NULL NULL NULL NULL NULL NULL +5307 1.31 NULL NULL NULL NULL NULL NULL +5307 1.31 NULL NULL NULL NULL NULL NULL +5307 1.31 NULL NULL NULL NULL NULL NULL +5307 1.31 NULL NULL NULL NULL NULL NULL +5307 1.31 NULL NULL NULL NULL NULL NULL +5307 1.31 NULL NULL NULL NULL NULL NULL +5307 1.31 NULL NULL NULL NULL NULL NULL +5307 1.31 NULL NULL NULL NULL NULL NULL +5307 1.31 NULL NULL NULL NULL NULL NULL +5308 0.27 NULL NULL NULL NULL NULL NULL +5308 0.27 NULL NULL NULL NULL NULL NULL +5308 0.27 NULL NULL NULL NULL NULL NULL +5308 0.27 NULL NULL NULL NULL NULL NULL +5308 0.27 NULL NULL NULL NULL NULL NULL +5308 0.27 NULL NULL NULL NULL NULL NULL +5308 0.27 NULL NULL NULL NULL NULL NULL +5308 0.27 NULL NULL NULL NULL NULL NULL +5308 0.27 NULL NULL NULL NULL NULL NULL +5308 0.27 NULL NULL NULL NULL NULL NULL +5308 0.27 NULL NULL NULL NULL NULL NULL +5308 0.27 NULL NULL NULL NULL NULL NULL +5308 0.27 NULL NULL NULL NULL NULL NULL +5308 0.27 NULL NULL NULL NULL NULL NULL +5308 0.27 NULL NULL NULL NULL NULL NULL +5308 0.27 NULL NULL NULL NULL NULL NULL +5308 0.27 NULL NULL NULL NULL NULL NULL +5308 0.27 NULL NULL NULL NULL NULL NULL +5308 0.27 NULL NULL NULL NULL NULL NULL +5308 0.27 NULL NULL NULL NULL NULL NULL +5308 0.27 NULL NULL NULL NULL NULL NULL +5308 0.27 NULL NULL NULL NULL NULL NULL +5308 0.27 NULL NULL NULL NULL NULL NULL +5308 0.27 NULL NULL NULL NULL NULL NULL +5308 0.27 NULL NULL NULL NULL NULL NULL +5308 0.27 NULL NULL NULL NULL NULL NULL +5308 0.27 NULL NULL NULL NULL NULL NULL +5308 0.27 NULL NULL NULL NULL NULL NULL +5308 0.27 NULL NULL NULL NULL NULL NULL +5308 0.27 NULL NULL NULL NULL NULL NULL +5308 0.27 NULL NULL NULL NULL NULL NULL +5308 0.27 NULL NULL NULL NULL NULL NULL +5308 0.27 NULL NULL NULL NULL NULL NULL +5308 0.27 NULL NULL NULL NULL NULL NULL +5308 0.27 NULL NULL NULL NULL NULL NULL +5308 0.27 NULL NULL NULL NULL NULL NULL +5308 0.27 NULL NULL NULL NULL NULL NULL +5308 0.27 NULL NULL NULL NULL NULL NULL +5308 0.27 NULL NULL NULL NULL NULL NULL +5308 0.27 NULL NULL NULL NULL NULL NULL +5308 0.27 NULL NULL NULL NULL NULL NULL +5308 0.27 NULL NULL NULL NULL NULL NULL +5308 0.27 NULL NULL NULL NULL NULL NULL +5308 0.27 NULL NULL NULL NULL NULL NULL +5308 0.27 NULL NULL NULL NULL NULL NULL +5308 0.27 NULL NULL NULL NULL NULL NULL +5308 0.27 NULL NULL NULL NULL NULL NULL +5308 0.27 NULL NULL NULL NULL NULL NULL +5308 0.27 NULL NULL NULL NULL NULL NULL +5309 NULL NULL NULL NULL NULL NULL NULL +5309 NULL NULL NULL NULL NULL NULL NULL +5309 NULL NULL NULL NULL NULL NULL NULL +5309 NULL NULL NULL NULL NULL NULL NULL +5309 NULL NULL NULL NULL NULL NULL NULL +5309 NULL NULL NULL NULL NULL NULL NULL +5309 NULL NULL NULL NULL NULL NULL NULL +5309 NULL NULL NULL NULL NULL NULL NULL +5309 NULL NULL NULL NULL NULL NULL NULL +5309 NULL NULL NULL NULL NULL NULL NULL +5309 NULL NULL NULL NULL NULL NULL NULL +5309 NULL NULL NULL NULL NULL NULL NULL +5309 NULL NULL NULL NULL NULL NULL NULL +5309 NULL NULL NULL NULL NULL NULL NULL +5309 NULL NULL NULL NULL NULL NULL NULL +5309 NULL NULL NULL NULL NULL NULL NULL +5309 NULL NULL NULL NULL NULL NULL NULL +5309 NULL NULL NULL NULL NULL NULL NULL +5309 NULL NULL NULL NULL NULL NULL NULL +5309 NULL NULL NULL NULL NULL NULL NULL +5309 NULL NULL NULL NULL NULL NULL NULL +5309 NULL NULL NULL NULL NULL NULL NULL +5309 NULL NULL NULL NULL NULL NULL NULL +5309 NULL NULL NULL NULL NULL NULL NULL +5309 NULL NULL NULL NULL NULL NULL NULL +5309 NULL NULL NULL NULL NULL NULL NULL +5309 NULL NULL NULL NULL NULL NULL NULL +5309 NULL NULL NULL NULL NULL NULL NULL +5309 NULL NULL NULL NULL NULL NULL NULL +5309 NULL NULL NULL NULL NULL NULL NULL +5309 NULL NULL NULL NULL NULL NULL NULL +5309 NULL NULL NULL NULL NULL NULL NULL +5309 NULL NULL NULL NULL NULL NULL NULL +5309 NULL NULL NULL NULL NULL NULL NULL +5309 NULL NULL NULL NULL NULL NULL NULL +5309 NULL NULL NULL NULL NULL NULL NULL +5309 NULL NULL NULL NULL NULL NULL NULL +5309 NULL NULL NULL NULL NULL NULL NULL +5309 NULL NULL NULL NULL NULL NULL NULL +5309 NULL NULL NULL NULL NULL NULL NULL +5309 NULL NULL NULL NULL NULL NULL NULL +5309 NULL NULL NULL NULL NULL NULL NULL +5309 NULL NULL NULL NULL NULL NULL NULL +5309 NULL NULL NULL NULL NULL NULL NULL +5309 NULL NULL NULL NULL NULL NULL NULL +5309 NULL NULL NULL NULL NULL NULL NULL +5309 NULL NULL NULL NULL NULL NULL NULL +5309 NULL NULL NULL NULL NULL NULL NULL +5309 NULL NULL NULL NULL NULL NULL NULL +5310 1.40 0.50 NULL NULL NULL 1.00 NULL +5310 1.40 0.50 NULL NULL NULL 1.00 NULL +5310 1.40 0.50 NULL NULL NULL 1.00 NULL +5310 1.40 0.50 NULL NULL NULL 1.00 NULL +5310 1.40 0.50 NULL NULL NULL 1.00 NULL +5310 1.40 0.50 NULL NULL NULL 1.00 NULL +5310 1.40 0.50 NULL NULL NULL 1.00 NULL +5310 1.40 0.50 NULL NULL NULL 1.00 NULL +5310 1.40 0.50 NULL NULL NULL 1.00 NULL +5310 1.40 0.50 NULL NULL NULL 1.00 NULL +5310 1.40 0.50 NULL NULL NULL 1.00 NULL +5310 1.40 0.50 NULL NULL NULL 1.00 NULL +5310 1.40 0.50 NULL NULL NULL 1.00 NULL +5310 1.40 0.50 NULL NULL NULL 1.00 NULL +5310 1.40 0.50 NULL NULL NULL 1.00 NULL +5310 1.40 0.50 NULL NULL NULL 1.00 NULL +5310 1.40 0.50 NULL NULL NULL 1.00 NULL +5310 1.40 0.50 NULL NULL NULL 1.00 NULL +5310 1.40 0.50 NULL NULL NULL 1.00 NULL +5310 1.40 0.50 NULL NULL NULL 1.00 NULL +5310 1.40 0.50 NULL NULL NULL 1.00 NULL +5310 1.40 0.50 NULL NULL NULL 1.00 NULL +5310 1.40 0.50 NULL NULL NULL 1.00 NULL +5310 1.40 0.50 NULL NULL NULL 1.00 NULL +5310 1.40 0.50 NULL NULL NULL 1.00 NULL +5310 1.40 0.50 NULL NULL NULL 1.00 NULL +5310 1.40 0.50 NULL NULL NULL 1.00 NULL +5310 1.40 0.50 NULL NULL NULL 1.00 NULL +5310 1.40 0.50 NULL NULL NULL 1.00 NULL +5310 1.40 0.50 NULL NULL NULL 1.00 NULL +5310 1.40 0.50 NULL NULL NULL 1.00 NULL +5310 1.40 0.50 NULL NULL NULL 1.00 NULL +5310 1.40 0.50 NULL NULL NULL 1.00 NULL +5310 1.40 0.50 NULL NULL NULL 1.00 NULL +5310 1.40 0.50 NULL NULL NULL 1.00 NULL +5310 1.40 0.50 NULL NULL NULL 1.00 NULL +5310 1.40 0.50 NULL NULL NULL 1.00 NULL +5310 1.40 0.50 NULL NULL NULL 1.00 NULL +5310 1.40 0.50 NULL NULL NULL 1.00 NULL +5310 1.40 0.50 NULL NULL NULL 1.00 NULL +5310 1.40 0.50 NULL NULL NULL 1.00 NULL +5310 1.40 0.50 NULL NULL NULL 1.00 NULL +5310 1.40 0.50 NULL NULL NULL 1.00 NULL +5310 1.40 0.50 NULL NULL NULL 1.00 NULL +5310 1.40 0.50 NULL NULL NULL 1.00 NULL +5310 1.40 0.50 NULL NULL NULL 1.00 NULL +5310 1.40 0.50 NULL NULL NULL 1.00 NULL +5310 1.40 0.50 NULL NULL NULL 1.00 NULL +5310 1.40 0.50 NULL NULL NULL 1.00 NULL +5312 NULL NULL NULL NULL NULL NULL NULL +5312 NULL NULL NULL NULL NULL NULL NULL +5312 NULL NULL NULL NULL NULL NULL NULL +5312 NULL NULL NULL NULL NULL NULL NULL +5312 NULL NULL NULL NULL NULL NULL NULL +5312 NULL NULL NULL NULL NULL NULL NULL +5312 NULL NULL NULL NULL NULL NULL NULL +5312 NULL NULL NULL NULL NULL NULL NULL +5312 NULL NULL NULL NULL NULL NULL NULL +5312 NULL NULL NULL NULL NULL NULL NULL +5312 NULL NULL NULL NULL NULL NULL NULL +5312 NULL NULL NULL NULL NULL NULL NULL +5312 NULL NULL NULL NULL NULL NULL NULL +5312 NULL NULL NULL NULL NULL NULL NULL +5312 NULL NULL NULL NULL NULL NULL NULL +5312 NULL NULL NULL NULL NULL NULL NULL +5312 NULL NULL NULL NULL NULL NULL NULL +5312 NULL NULL NULL NULL NULL NULL NULL +5312 NULL NULL NULL NULL NULL NULL NULL +5312 NULL NULL NULL NULL NULL NULL NULL +5312 NULL NULL NULL NULL NULL NULL NULL +5312 NULL NULL NULL NULL NULL NULL NULL +5312 NULL NULL NULL NULL NULL NULL NULL +5312 NULL NULL NULL NULL NULL NULL NULL +5312 NULL NULL NULL NULL NULL NULL NULL +5312 NULL NULL NULL NULL NULL NULL NULL +5312 NULL NULL NULL NULL NULL NULL NULL +5312 NULL NULL NULL NULL NULL NULL NULL +5312 NULL NULL NULL NULL NULL NULL NULL +5312 NULL NULL NULL NULL NULL NULL NULL +5312 NULL NULL NULL NULL NULL NULL NULL +5312 NULL NULL NULL NULL NULL NULL NULL +5312 NULL NULL NULL NULL NULL NULL NULL +5312 NULL NULL NULL NULL NULL NULL NULL +5312 NULL NULL NULL NULL NULL NULL NULL +5312 NULL NULL NULL NULL NULL NULL NULL +5312 NULL NULL NULL NULL NULL NULL NULL +5312 NULL NULL NULL NULL NULL NULL NULL +5312 NULL NULL NULL NULL NULL NULL NULL +5312 NULL NULL NULL NULL NULL NULL NULL +5312 NULL NULL NULL NULL NULL NULL NULL +5312 NULL NULL NULL NULL NULL NULL NULL +5312 NULL NULL NULL NULL NULL NULL NULL +5312 NULL NULL NULL NULL NULL NULL NULL +5312 NULL NULL NULL NULL NULL NULL NULL +5312 NULL NULL NULL NULL NULL NULL NULL +5312 NULL NULL NULL NULL NULL NULL NULL +5312 NULL NULL NULL NULL NULL NULL NULL +5312 NULL NULL NULL NULL NULL NULL NULL +5313 NULL NULL 1.29 NULL NULL 1.41 2.04 +5313 NULL NULL 1.29 NULL NULL 1.41 2.04 +5313 NULL NULL 1.29 NULL NULL 1.41 2.04 +5313 NULL NULL 1.29 NULL NULL 1.41 2.04 +5313 NULL NULL 1.29 NULL NULL 1.41 2.04 +5313 NULL NULL 1.29 NULL NULL 1.41 2.04 +5313 NULL NULL 1.29 NULL NULL 1.41 2.04 +5313 NULL NULL 1.29 NULL NULL 1.41 2.04 +5313 NULL NULL 1.29 NULL NULL 1.41 2.04 +5313 NULL NULL 1.29 NULL NULL 1.41 2.04 +5313 NULL NULL 1.29 NULL NULL 1.41 2.04 +5313 NULL NULL 1.29 NULL NULL 1.41 2.04 +5313 NULL NULL 1.29 NULL NULL 1.41 2.04 +5313 NULL NULL 1.29 NULL NULL 1.41 2.04 +5313 NULL NULL 1.29 NULL NULL 1.41 2.04 +5313 NULL NULL 1.29 NULL NULL 1.41 2.04 +5313 NULL NULL 1.29 NULL NULL 1.41 2.04 +5313 NULL NULL 1.29 NULL NULL 1.41 2.04 +5313 NULL NULL 1.29 NULL NULL 1.41 2.04 +5313 NULL NULL 1.29 NULL NULL 1.41 2.04 +5313 NULL NULL 1.29 NULL NULL 1.41 2.04 +5313 NULL NULL 1.29 NULL NULL 1.41 2.04 +5313 NULL NULL 1.29 NULL NULL 1.41 2.04 +5313 NULL NULL 1.29 NULL NULL 1.41 2.04 +5313 NULL NULL 1.29 NULL NULL 1.41 2.04 +5313 NULL NULL 1.29 NULL NULL 1.41 2.04 +5313 NULL NULL 1.29 NULL NULL 1.41 2.04 +5313 NULL NULL 1.29 NULL NULL 1.41 2.04 +5313 NULL NULL 1.29 NULL NULL 1.41 2.04 +5313 NULL NULL 1.29 NULL NULL 1.41 2.04 +5313 NULL NULL 1.29 NULL NULL 1.41 2.04 +5313 NULL NULL 1.29 NULL NULL 1.41 2.04 +5313 NULL NULL 1.29 NULL NULL 1.41 2.04 +5313 NULL NULL 1.29 NULL NULL 1.41 2.04 +5313 NULL NULL 1.29 NULL NULL 1.41 2.04 +5313 NULL NULL 1.29 NULL NULL 1.41 2.04 +5313 NULL NULL 1.29 NULL NULL 1.41 2.04 +5313 NULL NULL 1.29 NULL NULL 1.41 2.04 +5313 NULL NULL 1.29 NULL NULL 1.41 2.04 +5313 NULL NULL 1.29 NULL NULL 1.41 2.04 +5313 NULL NULL 1.29 NULL NULL 1.41 2.04 +5313 NULL NULL 1.29 NULL NULL 1.41 2.04 +5313 NULL NULL 1.29 NULL NULL 1.41 2.04 +5313 NULL NULL 1.29 NULL NULL 1.41 2.04 +5313 NULL NULL 1.29 NULL NULL 1.41 2.04 +5313 NULL NULL 1.29 NULL NULL 1.41 2.04 +5313 NULL NULL 1.29 NULL NULL 1.41 2.04 +5313 NULL NULL 1.29 NULL NULL 1.41 2.04 +5313 NULL NULL 1.29 NULL NULL 1.41 2.04 +5314 NULL NULL NULL NULL NULL NULL NULL +5314 NULL NULL NULL NULL NULL NULL NULL +5314 NULL NULL NULL NULL NULL NULL NULL +5314 NULL NULL NULL NULL NULL NULL NULL +5314 NULL NULL NULL NULL NULL NULL NULL +5314 NULL NULL NULL NULL NULL NULL NULL +5314 NULL NULL NULL NULL NULL NULL NULL +5314 NULL NULL NULL NULL NULL NULL NULL +5314 NULL NULL NULL NULL NULL NULL NULL +5314 NULL NULL NULL NULL NULL NULL NULL +5314 NULL NULL NULL NULL NULL NULL NULL +5314 NULL NULL NULL NULL NULL NULL NULL +5314 NULL NULL NULL NULL NULL NULL NULL +5314 NULL NULL NULL NULL NULL NULL NULL +5314 NULL NULL NULL NULL NULL NULL NULL +5314 NULL NULL NULL NULL NULL NULL NULL +5314 NULL NULL NULL NULL NULL NULL NULL +5314 NULL NULL NULL NULL NULL NULL NULL +5314 NULL NULL NULL NULL NULL NULL NULL +5314 NULL NULL NULL NULL NULL NULL NULL +5314 NULL NULL NULL NULL NULL NULL NULL +5314 NULL NULL NULL NULL NULL NULL NULL +5314 NULL NULL NULL NULL NULL NULL NULL +5314 NULL NULL NULL NULL NULL NULL NULL +5314 NULL NULL NULL NULL NULL NULL NULL +5314 NULL NULL NULL NULL NULL NULL NULL +5314 NULL NULL NULL NULL NULL NULL NULL +5314 NULL NULL NULL NULL NULL NULL NULL +5314 NULL NULL NULL NULL NULL NULL NULL +5314 NULL NULL NULL NULL NULL NULL NULL +5314 NULL NULL NULL NULL NULL NULL NULL +5314 NULL NULL NULL NULL NULL NULL NULL +5314 NULL NULL NULL NULL NULL NULL NULL +5314 NULL NULL NULL NULL NULL NULL NULL +5314 NULL NULL NULL NULL NULL NULL NULL +5314 NULL NULL NULL NULL NULL NULL NULL +5314 NULL NULL NULL NULL NULL NULL NULL +5314 NULL NULL NULL NULL NULL NULL NULL +5314 NULL NULL NULL NULL NULL NULL NULL +5314 NULL NULL NULL NULL NULL NULL NULL +5314 NULL NULL NULL NULL NULL NULL NULL +5314 NULL NULL NULL NULL NULL NULL NULL +5314 NULL NULL NULL NULL NULL NULL NULL +5314 NULL NULL NULL NULL NULL NULL NULL +5314 NULL NULL NULL NULL NULL NULL NULL +5314 NULL NULL NULL NULL NULL NULL NULL +5314 NULL NULL NULL NULL NULL NULL NULL +5314 NULL NULL NULL NULL NULL NULL NULL +5314 NULL NULL NULL NULL NULL NULL NULL +5315 NULL NULL 0.49 NULL NULL 0.36 1.24 +5315 NULL NULL 0.49 NULL NULL 0.36 1.24 +5315 NULL NULL 0.49 NULL NULL 0.36 1.24 +5315 NULL NULL 0.49 NULL NULL 0.36 1.24 +5315 NULL NULL 0.49 NULL NULL 0.36 1.24 +5315 NULL NULL 0.49 NULL NULL 0.36 1.24 +5315 NULL NULL 0.49 NULL NULL 0.36 1.24 +5315 NULL NULL 0.49 NULL NULL 0.36 1.24 +5315 NULL NULL 0.49 NULL NULL 0.36 1.24 +5315 NULL NULL 0.49 NULL NULL 0.36 1.24 +5315 NULL NULL 0.49 NULL NULL 0.36 1.24 +5315 NULL NULL 0.49 NULL NULL 0.36 1.24 +5315 NULL NULL 0.49 NULL NULL 0.36 1.24 +5315 NULL NULL 0.49 NULL NULL 0.36 1.24 +5315 NULL NULL 0.49 NULL NULL 0.36 1.24 +5315 NULL NULL 0.49 NULL NULL 0.36 1.24 +5315 NULL NULL 0.49 NULL NULL 0.36 1.24 +5315 NULL NULL 0.49 NULL NULL 0.36 1.24 +5315 NULL NULL 0.49 NULL NULL 0.36 1.24 +5315 NULL NULL 0.49 NULL NULL 0.36 1.24 +5315 NULL NULL 0.49 NULL NULL 0.36 1.24 +5315 NULL NULL 0.49 NULL NULL 0.36 1.24 +5315 NULL NULL 0.49 NULL NULL 0.36 1.24 +5315 NULL NULL 0.49 NULL NULL 0.36 1.24 +5315 NULL NULL 0.49 NULL NULL 0.36 1.24 +5315 NULL NULL 0.49 NULL NULL 0.36 1.24 +5315 NULL NULL 0.49 NULL NULL 0.36 1.24 +5315 NULL NULL 0.49 NULL NULL 0.36 1.24 +5315 NULL NULL 0.49 NULL NULL 0.36 1.24 +5315 NULL NULL 0.49 NULL NULL 0.36 1.24 +5315 NULL NULL 0.49 NULL NULL 0.36 1.24 +5315 NULL NULL 0.49 NULL NULL 0.36 1.24 +5315 NULL NULL 0.49 NULL NULL 0.36 1.24 +5315 NULL NULL 0.49 NULL NULL 0.36 1.24 +5315 NULL NULL 0.49 NULL NULL 0.36 1.24 +5315 NULL NULL 0.49 NULL NULL 0.36 1.24 +5315 NULL NULL 0.49 NULL NULL 0.36 1.24 +5315 NULL NULL 0.49 NULL NULL 0.36 1.24 +5315 NULL NULL 0.49 NULL NULL 0.36 1.24 +5315 NULL NULL 0.49 NULL NULL 0.36 1.24 +5315 NULL NULL 0.49 NULL NULL 0.36 1.24 +5315 NULL NULL 0.49 NULL NULL 0.36 1.24 +5315 NULL NULL 0.49 NULL NULL 0.36 1.24 +5315 NULL NULL 0.49 NULL NULL 0.36 1.24 +5315 NULL NULL 0.49 NULL NULL 0.36 1.24 +5315 NULL NULL 0.49 NULL NULL 0.36 1.24 +5315 NULL NULL 0.49 NULL NULL 0.36 1.24 +5315 NULL NULL 0.49 NULL NULL 0.36 1.24 +5315 NULL NULL 0.49 NULL NULL 0.36 1.24 +5316 4.70 0.46 NULL NULL NULL NULL 0.61 +5316 4.70 0.46 NULL NULL NULL NULL 0.61 +5316 4.70 0.46 NULL NULL NULL NULL 0.61 +5316 4.70 0.46 NULL NULL NULL NULL 0.61 +5316 4.70 0.46 NULL NULL NULL NULL 0.61 +5316 4.70 0.46 NULL NULL NULL NULL 0.61 +5316 4.70 0.46 NULL NULL NULL NULL 0.61 +5316 4.70 0.46 NULL NULL NULL NULL 0.61 +5316 4.70 0.46 NULL NULL NULL NULL 0.61 +5316 4.70 0.46 NULL NULL NULL NULL 0.61 +5316 4.70 0.46 NULL NULL NULL NULL 0.61 +5316 4.70 0.46 NULL NULL NULL NULL 0.61 +5316 4.70 0.46 NULL NULL NULL NULL 0.61 +5316 4.70 0.46 NULL NULL NULL NULL 0.61 +5316 4.70 0.46 NULL NULL NULL NULL 0.61 +5316 4.70 0.46 NULL NULL NULL NULL 0.61 +5316 4.70 0.46 NULL NULL NULL NULL 0.61 +5316 4.70 0.46 NULL NULL NULL NULL 0.61 +5316 4.70 0.46 NULL NULL NULL NULL 0.61 +5316 4.70 0.46 NULL NULL NULL NULL 0.61 +5316 4.70 0.46 NULL NULL NULL NULL 0.61 +5316 4.70 0.46 NULL NULL NULL NULL 0.61 +5316 4.70 0.46 NULL NULL NULL NULL 0.61 +5316 4.70 0.46 NULL NULL NULL NULL 0.61 +5316 4.70 0.46 NULL NULL NULL NULL 0.61 +5316 4.70 0.46 NULL NULL NULL NULL 0.61 +5316 4.70 0.46 NULL NULL NULL NULL 0.61 +5316 4.70 0.46 NULL NULL NULL NULL 0.61 +5316 4.70 0.46 NULL NULL NULL NULL 0.61 +5316 4.70 0.46 NULL NULL NULL NULL 0.61 +5316 4.70 0.46 NULL NULL NULL NULL 0.61 +5316 4.70 0.46 NULL NULL NULL NULL 0.61 +5316 4.70 0.46 NULL NULL NULL NULL 0.61 +5316 4.70 0.46 NULL NULL NULL NULL 0.61 +5316 4.70 0.46 NULL NULL NULL NULL 0.61 +5316 4.70 0.46 NULL NULL NULL NULL 0.61 +5316 4.70 0.46 NULL NULL NULL NULL 0.61 +5316 4.70 0.46 NULL NULL NULL NULL 0.61 +5316 4.70 0.46 NULL NULL NULL NULL 0.61 +5316 4.70 0.46 NULL NULL NULL NULL 0.61 +5316 4.70 0.46 NULL NULL NULL NULL 0.61 +5316 4.70 0.46 NULL NULL NULL NULL 0.61 +5316 4.70 0.46 NULL NULL NULL NULL 0.61 +5316 4.70 0.46 NULL NULL NULL NULL 0.61 +5316 4.70 0.46 NULL NULL NULL NULL 0.61 +5316 4.70 0.46 NULL NULL NULL NULL 0.61 +5316 4.70 0.46 NULL NULL NULL NULL 0.61 +5316 4.70 0.46 NULL NULL NULL NULL 0.61 +5316 4.70 0.46 NULL NULL NULL NULL 0.61 +5317 NULL 1.85 NULL NULL NULL NULL NULL +5317 NULL 1.85 NULL NULL NULL NULL NULL +5317 NULL 1.85 NULL NULL NULL NULL NULL +5317 NULL 1.85 NULL NULL NULL NULL NULL +5317 NULL 1.85 NULL NULL NULL NULL NULL +5317 NULL 1.85 NULL NULL NULL NULL NULL +5317 NULL 1.85 NULL NULL NULL NULL NULL +5317 NULL 1.85 NULL NULL NULL NULL NULL +5317 NULL 1.85 NULL NULL NULL NULL NULL +5317 NULL 1.85 NULL NULL NULL NULL NULL +5317 NULL 1.85 NULL NULL NULL NULL NULL +5317 NULL 1.85 NULL NULL NULL NULL NULL +5317 NULL 1.85 NULL NULL NULL NULL NULL +5317 NULL 1.85 NULL NULL NULL NULL NULL +5317 NULL 1.85 NULL NULL NULL NULL NULL +5317 NULL 1.85 NULL NULL NULL NULL NULL +5317 NULL 1.85 NULL NULL NULL NULL NULL +5317 NULL 1.85 NULL NULL NULL NULL NULL +5317 NULL 1.85 NULL NULL NULL NULL NULL +5317 NULL 1.85 NULL NULL NULL NULL NULL +5317 NULL 1.85 NULL NULL NULL NULL NULL +5317 NULL 1.85 NULL NULL NULL NULL NULL +5317 NULL 1.85 NULL NULL NULL NULL NULL +5317 NULL 1.85 NULL NULL NULL NULL NULL +5317 NULL 1.85 NULL NULL NULL NULL NULL +5317 NULL 1.85 NULL NULL NULL NULL NULL +5317 NULL 1.85 NULL NULL NULL NULL NULL +5317 NULL 1.85 NULL NULL NULL NULL NULL +5317 NULL 1.85 NULL NULL NULL NULL NULL +5317 NULL 1.85 NULL NULL NULL NULL NULL +5317 NULL 1.85 NULL NULL NULL NULL NULL +5317 NULL 1.85 NULL NULL NULL NULL NULL +5317 NULL 1.85 NULL NULL NULL NULL NULL +5317 NULL 1.85 NULL NULL NULL NULL NULL +5317 NULL 1.85 NULL NULL NULL NULL NULL +5317 NULL 1.85 NULL NULL NULL NULL NULL +5317 NULL 1.85 NULL NULL NULL NULL NULL +5317 NULL 1.85 NULL NULL NULL NULL NULL +5317 NULL 1.85 NULL NULL NULL NULL NULL +5317 NULL 1.85 NULL NULL NULL NULL NULL +5317 NULL 1.85 NULL NULL NULL NULL NULL +5317 NULL 1.85 NULL NULL NULL NULL NULL +5317 NULL 1.85 NULL NULL NULL NULL NULL +5317 NULL 1.85 NULL NULL NULL NULL NULL +5317 NULL 1.85 NULL NULL NULL NULL NULL +5317 NULL 1.85 NULL NULL NULL NULL NULL +5317 NULL 1.85 NULL NULL NULL NULL NULL +5317 NULL 1.85 NULL NULL NULL NULL NULL +5317 NULL 1.85 NULL NULL NULL NULL NULL +5318 NULL NULL NULL NULL NULL NULL 1.65 +5318 NULL NULL NULL NULL NULL NULL 1.65 +5318 NULL NULL NULL NULL NULL NULL 1.65 +5318 NULL NULL NULL NULL NULL NULL 1.65 +5318 NULL NULL NULL NULL NULL NULL 1.65 +5318 NULL NULL NULL NULL NULL NULL 1.65 +5318 NULL NULL NULL NULL NULL NULL 1.65 +5318 NULL NULL NULL NULL NULL NULL 1.65 +5318 NULL NULL NULL NULL NULL NULL 1.65 +5318 NULL NULL NULL NULL NULL NULL 1.65 +5318 NULL NULL NULL NULL NULL NULL 1.65 +5318 NULL NULL NULL NULL NULL NULL 1.65 +5318 NULL NULL NULL NULL NULL NULL 1.65 +5318 NULL NULL NULL NULL NULL NULL 1.65 +5318 NULL NULL NULL NULL NULL NULL 1.65 +5318 NULL NULL NULL NULL NULL NULL 1.65 +5318 NULL NULL NULL NULL NULL NULL 1.65 +5318 NULL NULL NULL NULL NULL NULL 1.65 +5318 NULL NULL NULL NULL NULL NULL 1.65 +5318 NULL NULL NULL NULL NULL NULL 1.65 +5318 NULL NULL NULL NULL NULL NULL 1.65 +5318 NULL NULL NULL NULL NULL NULL 1.65 +5318 NULL NULL NULL NULL NULL NULL 1.65 +5318 NULL NULL NULL NULL NULL NULL 1.65 +5318 NULL NULL NULL NULL NULL NULL 1.65 +5318 NULL NULL NULL NULL NULL NULL 1.65 +5318 NULL NULL NULL NULL NULL NULL 1.65 +5318 NULL NULL NULL NULL NULL NULL 1.65 +5318 NULL NULL NULL NULL NULL NULL 1.65 +5318 NULL NULL NULL NULL NULL NULL 1.65 +5318 NULL NULL NULL NULL NULL NULL 1.65 +5318 NULL NULL NULL NULL NULL NULL 1.65 +5318 NULL NULL NULL NULL NULL NULL 1.65 +5318 NULL NULL NULL NULL NULL NULL 1.65 +5318 NULL NULL NULL NULL NULL NULL 1.65 +5318 NULL NULL NULL NULL NULL NULL 1.65 +5318 NULL NULL NULL NULL NULL NULL 1.65 +5318 NULL NULL NULL NULL NULL NULL 1.65 +5318 NULL NULL NULL NULL NULL NULL 1.65 +5318 NULL NULL NULL NULL NULL NULL 1.65 +5318 NULL NULL NULL NULL NULL NULL 1.65 +5318 NULL NULL NULL NULL NULL NULL 1.65 +5318 NULL NULL NULL NULL NULL NULL 1.65 +5318 NULL NULL NULL NULL NULL NULL 1.65 +5318 NULL NULL NULL NULL NULL NULL 1.65 +5318 NULL NULL NULL NULL NULL NULL 1.65 +5318 NULL NULL NULL NULL NULL NULL 1.65 +5318 NULL NULL NULL NULL NULL NULL 1.65 +5318 NULL NULL NULL NULL NULL NULL 1.65 +5319 NULL NULL NULL NULL NULL NULL 0.40 +5319 NULL NULL NULL NULL NULL NULL 0.40 +5319 NULL NULL NULL NULL NULL NULL 0.40 +5319 NULL NULL NULL NULL NULL NULL 0.40 +5319 NULL NULL NULL NULL NULL NULL 0.40 +5319 NULL NULL NULL NULL NULL NULL 0.40 +5319 NULL NULL NULL NULL NULL NULL 0.40 +5319 NULL NULL NULL NULL NULL NULL 0.40 +5319 NULL NULL NULL NULL NULL NULL 0.40 +5319 NULL NULL NULL NULL NULL NULL 0.40 +5319 NULL NULL NULL NULL NULL NULL 0.40 +5319 NULL NULL NULL NULL NULL NULL 0.40 +5319 NULL NULL NULL NULL NULL NULL 0.40 +5319 NULL NULL NULL NULL NULL NULL 0.40 +5319 NULL NULL NULL NULL NULL NULL 0.40 +5319 NULL NULL NULL NULL NULL NULL 0.40 +5319 NULL NULL NULL NULL NULL NULL 0.40 +5319 NULL NULL NULL NULL NULL NULL 0.40 +5319 NULL NULL NULL NULL NULL NULL 0.40 +5319 NULL NULL NULL NULL NULL NULL 0.40 +5319 NULL NULL NULL NULL NULL NULL 0.40 +5319 NULL NULL NULL NULL NULL NULL 0.40 +5319 NULL NULL NULL NULL NULL NULL 0.40 +5319 NULL NULL NULL NULL NULL NULL 0.40 +5319 NULL NULL NULL NULL NULL NULL 0.40 +5319 NULL NULL NULL NULL NULL NULL 0.40 +5319 NULL NULL NULL NULL NULL NULL 0.40 +5319 NULL NULL NULL NULL NULL NULL 0.40 +5319 NULL NULL NULL NULL NULL NULL 0.40 +5319 NULL NULL NULL NULL NULL NULL 0.40 +5319 NULL NULL NULL NULL NULL NULL 0.40 +5319 NULL NULL NULL NULL NULL NULL 0.40 +5319 NULL NULL NULL NULL NULL NULL 0.40 +5319 NULL NULL NULL NULL NULL NULL 0.40 +5319 NULL NULL NULL NULL NULL NULL 0.40 +5319 NULL NULL NULL NULL NULL NULL 0.40 +5319 NULL NULL NULL NULL NULL NULL 0.40 +5319 NULL NULL NULL NULL NULL NULL 0.40 +5319 NULL NULL NULL NULL NULL NULL 0.40 +5319 NULL NULL NULL NULL NULL NULL 0.40 +5319 NULL NULL NULL NULL NULL NULL 0.40 +5319 NULL NULL NULL NULL NULL NULL 0.40 +5319 NULL NULL NULL NULL NULL NULL 0.40 +5319 NULL NULL NULL NULL NULL NULL 0.40 +5319 NULL NULL NULL NULL NULL NULL 0.40 +5319 NULL NULL NULL NULL NULL NULL 0.40 +5319 NULL NULL NULL NULL NULL NULL 0.40 +5319 NULL NULL NULL NULL NULL NULL 0.40 +5319 NULL NULL NULL NULL NULL NULL 0.40 +5320 NULL NULL NULL NULL NULL 2.40 NULL +5320 NULL NULL NULL NULL NULL 2.40 NULL +5320 NULL NULL NULL NULL NULL 2.40 NULL +5320 NULL NULL NULL NULL NULL 2.40 NULL +5320 NULL NULL NULL NULL NULL 2.40 NULL +5320 NULL NULL NULL NULL NULL 2.40 NULL +5320 NULL NULL NULL NULL NULL 2.40 NULL +5320 NULL NULL NULL NULL NULL 2.40 NULL +5320 NULL NULL NULL NULL NULL 2.40 NULL +5320 NULL NULL NULL NULL NULL 2.40 NULL +5320 NULL NULL NULL NULL NULL 2.40 NULL +5320 NULL NULL NULL NULL NULL 2.40 NULL +5320 NULL NULL NULL NULL NULL 2.40 NULL +5320 NULL NULL NULL NULL NULL 2.40 NULL +5320 NULL NULL NULL NULL NULL 2.40 NULL +5320 NULL NULL NULL NULL NULL 2.40 NULL +5320 NULL NULL NULL NULL NULL 2.40 NULL +5320 NULL NULL NULL NULL NULL 2.40 NULL +5320 NULL NULL NULL NULL NULL 2.40 NULL +5320 NULL NULL NULL NULL NULL 2.40 NULL +5320 NULL NULL NULL NULL NULL 2.40 NULL +5320 NULL NULL NULL NULL NULL 2.40 NULL +5320 NULL NULL NULL NULL NULL 2.40 NULL +5320 NULL NULL NULL NULL NULL 2.40 NULL +5320 NULL NULL NULL NULL NULL 2.40 NULL +5320 NULL NULL NULL NULL NULL 2.40 NULL +5320 NULL NULL NULL NULL NULL 2.40 NULL +5320 NULL NULL NULL NULL NULL 2.40 NULL +5320 NULL NULL NULL NULL NULL 2.40 NULL +5320 NULL NULL NULL NULL NULL 2.40 NULL +5320 NULL NULL NULL NULL NULL 2.40 NULL +5320 NULL NULL NULL NULL NULL 2.40 NULL +5320 NULL NULL NULL NULL NULL 2.40 NULL +5320 NULL NULL NULL NULL NULL 2.40 NULL +5320 NULL NULL NULL NULL NULL 2.40 NULL +5320 NULL NULL NULL NULL NULL 2.40 NULL +5320 NULL NULL NULL NULL NULL 2.40 NULL +5320 NULL NULL NULL NULL NULL 2.40 NULL +5320 NULL NULL NULL NULL NULL 2.40 NULL +5320 NULL NULL NULL NULL NULL 2.40 NULL +5320 NULL NULL NULL NULL NULL 2.40 NULL +5320 NULL NULL NULL NULL NULL 2.40 NULL +5320 NULL NULL NULL NULL NULL 2.40 NULL +5320 NULL NULL NULL NULL NULL 2.40 NULL +5320 NULL NULL NULL NULL NULL 2.40 NULL +5320 NULL NULL NULL NULL NULL 2.40 NULL +5320 NULL NULL NULL NULL NULL 2.40 NULL +5320 NULL NULL NULL NULL NULL 2.40 NULL +5320 NULL NULL NULL NULL NULL 2.40 NULL +5321 1.30 0.37 NULL NULL 0.94 0.37 NULL +5321 1.30 0.37 NULL NULL 0.94 0.37 NULL +5321 1.30 0.37 NULL NULL 0.94 0.37 NULL +5321 1.30 0.37 NULL NULL 0.94 0.37 NULL +5321 1.30 0.37 NULL NULL 0.94 0.37 NULL +5321 1.30 0.37 NULL NULL 0.94 0.37 NULL +5321 1.30 0.37 NULL NULL 0.94 0.37 NULL +5321 1.30 0.37 NULL NULL 0.94 0.37 NULL +5321 1.30 0.37 NULL NULL 0.94 0.37 NULL +5321 1.30 0.37 NULL NULL 0.94 0.37 NULL +5321 1.30 0.37 NULL NULL 0.94 0.37 NULL +5321 1.30 0.37 NULL NULL 0.94 0.37 NULL +5321 1.30 0.37 NULL NULL 0.94 0.37 NULL +5321 1.30 0.37 NULL NULL 0.94 0.37 NULL +5321 1.30 0.37 NULL NULL 0.94 0.37 NULL +5321 1.30 0.37 NULL NULL 0.94 0.37 NULL +5321 1.30 0.37 NULL NULL 0.94 0.37 NULL +5321 1.30 0.37 NULL NULL 0.94 0.37 NULL +5321 1.30 0.37 NULL NULL 0.94 0.37 NULL +5321 1.30 0.37 NULL NULL 0.94 0.37 NULL +5321 1.30 0.37 NULL NULL 0.94 0.37 NULL +5321 1.30 0.37 NULL NULL 0.94 0.37 NULL +5321 1.30 0.37 NULL NULL 0.94 0.37 NULL +5321 1.30 0.37 NULL NULL 0.94 0.37 NULL +5321 1.30 0.37 NULL NULL 0.94 0.37 NULL +5321 1.30 0.37 NULL NULL 0.94 0.37 NULL +5321 1.30 0.37 NULL NULL 0.94 0.37 NULL +5321 1.30 0.37 NULL NULL 0.94 0.37 NULL +5321 1.30 0.37 NULL NULL 0.94 0.37 NULL +5321 1.30 0.37 NULL NULL 0.94 0.37 NULL +5321 1.30 0.37 NULL NULL 0.94 0.37 NULL +5321 1.30 0.37 NULL NULL 0.94 0.37 NULL +5321 1.30 0.37 NULL NULL 0.94 0.37 NULL +5321 1.30 0.37 NULL NULL 0.94 0.37 NULL +5321 1.30 0.37 NULL NULL 0.94 0.37 NULL +5321 1.30 0.37 NULL NULL 0.94 0.37 NULL +5321 1.30 0.37 NULL NULL 0.94 0.37 NULL +5321 1.30 0.37 NULL NULL 0.94 0.37 NULL +5321 1.30 0.37 NULL NULL 0.94 0.37 NULL +5321 1.30 0.37 NULL NULL 0.94 0.37 NULL +5321 1.30 0.37 NULL NULL 0.94 0.37 NULL +5321 1.30 0.37 NULL NULL 0.94 0.37 NULL +5321 1.30 0.37 NULL NULL 0.94 0.37 NULL +5321 1.30 0.37 NULL NULL 0.94 0.37 NULL +5321 1.30 0.37 NULL NULL 0.94 0.37 NULL +5321 1.30 0.37 NULL NULL 0.94 0.37 NULL +5321 1.30 0.37 NULL NULL 0.94 0.37 NULL +5321 1.30 0.37 NULL NULL 0.94 0.37 NULL +5321 1.30 0.37 NULL NULL 0.94 0.37 NULL +5322 NULL NULL NULL NULL NULL NULL NULL +5322 NULL NULL NULL NULL NULL NULL NULL +5322 NULL NULL NULL NULL NULL NULL NULL +5322 NULL NULL NULL NULL NULL NULL NULL +5322 NULL NULL NULL NULL NULL NULL NULL +5322 NULL NULL NULL NULL NULL NULL NULL +5322 NULL NULL NULL NULL NULL NULL NULL diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query20.sql b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query20.sql new file mode 100644 index 00000000000..b57ddf7997d --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query20.sql @@ -0,0 +1,30 @@ +-- start query 1 in stream 0 using template query20.tpl +select i_item_id + ,i_item_desc + ,i_category + ,i_class + ,i_current_price + ,sum(cs_ext_sales_price) as itemrevenue + ,sum(cs_ext_sales_price)*100/sum(sum(cs_ext_sales_price)) over + (partition by i_class) as revenueratio + from catalog_sales + ,item + ,date_dim + where cs_item_sk = i_item_sk + and i_category in ('Jewelry', 'Sports', 'Books') + and cs_sold_date_sk = d_date_sk + and d_date between cast('2001-01-12' as date) + and (cast('2001-01-12' as date) + interval 30 days) + group by i_item_id + ,i_item_desc + ,i_category + ,i_class + ,i_current_price + order by i_category + ,i_class + ,i_item_id + ,i_item_desc + ,revenueratio +limit 100; + + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query20.sql.out b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query20.sql.out new file mode 100644 index 00000000000..04d6ceb02dd --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query20.sql.out @@ -0,0 +1,33 @@ +-- Automatically generated by Gravitino Spark SQL test +-- !query +select i_item_id + ,i_item_desc + ,i_category + ,i_class + ,i_current_price + ,sum(cs_ext_sales_price) as itemrevenue + ,sum(cs_ext_sales_price)*100/sum(sum(cs_ext_sales_price)) over + (partition by i_class) as revenueratio + from catalog_sales + ,item + ,date_dim + where cs_item_sk = i_item_sk + and i_category in ('Jewelry', 'Sports', 'Books') + and cs_sold_date_sk = d_date_sk + and d_date between cast('2001-01-12' as date) + and (cast('2001-01-12' as date) + interval 30 days) + group by i_item_id + ,i_item_desc + ,i_category + ,i_class + ,i_current_price + order by i_category + ,i_class + ,i_item_id + ,i_item_desc + ,revenueratio +limit 100 +-- !query schema +struct +-- !query output + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query21.sql b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query21.sql new file mode 100644 index 00000000000..01a356a7f5c --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query21.sql @@ -0,0 +1,30 @@ +-- start query 1 in stream 0 using template query21.tpl +select * + from(select w_warehouse_name + ,i_item_id + ,sum(case when (cast(d_date as date) < cast ('1998-04-08' as date)) + then inv_quantity_on_hand + else 0 end) as inv_before + ,sum(case when (cast(d_date as date) >= cast ('1998-04-08' as date)) + then inv_quantity_on_hand + else 0 end) as inv_after + from inventory + ,warehouse + ,item + ,date_dim + where i_current_price between 0.99 and 1.49 + and i_item_sk = inv_item_sk + and inv_warehouse_sk = w_warehouse_sk + and inv_date_sk = d_date_sk + and d_date between (cast ('1998-04-08' as date) - interval 30 days) + and (cast ('1998-04-08' as date) + interval 30 days) + group by w_warehouse_name, i_item_id) x + where (case when inv_before > 0 + then inv_after / inv_before + else null + end) between 2.0/3.0 and 3.0/2.0 + order by w_warehouse_name + ,i_item_id + limit 100; + + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query21.sql.out b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query21.sql.out new file mode 100644 index 00000000000..14c6c81620c --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query21.sql.out @@ -0,0 +1,33 @@ +-- Automatically generated by Gravitino Spark SQL test +-- !query +select * + from(select w_warehouse_name + ,i_item_id + ,sum(case when (cast(d_date as date) < cast ('1998-04-08' as date)) + then inv_quantity_on_hand + else 0 end) as inv_before + ,sum(case when (cast(d_date as date) >= cast ('1998-04-08' as date)) + then inv_quantity_on_hand + else 0 end) as inv_after + from inventory + ,warehouse + ,item + ,date_dim + where i_current_price between 0.99 and 1.49 + and i_item_sk = inv_item_sk + and inv_warehouse_sk = w_warehouse_sk + and inv_date_sk = d_date_sk + and d_date between (cast ('1998-04-08' as date) - interval 30 days) + and (cast ('1998-04-08' as date) + interval 30 days) + group by w_warehouse_name, i_item_id) x + where (case when inv_before > 0 + then inv_after / inv_before + else null + end) between 2.0/3.0 and 3.0/2.0 + order by w_warehouse_name + ,i_item_id + limit 100 +-- !query schema +struct +-- !query output + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query22.sql b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query22.sql new file mode 100644 index 00000000000..10ae6911c38 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query22.sql @@ -0,0 +1,20 @@ +-- start query 1 in stream 0 using template query22.tpl +select i_product_name + ,i_brand + ,i_class + ,i_category + ,avg(inv_quantity_on_hand) qoh + from inventory + ,date_dim + ,item + where inv_date_sk=d_date_sk + and inv_item_sk=i_item_sk + and d_month_seq between 1212 and 1212 + 11 + group by rollup(i_product_name + ,i_brand + ,i_class + ,i_category) +order by qoh, i_product_name, i_brand, i_class, i_category +limit 100; + + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query22.sql.out b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query22.sql.out new file mode 100644 index 00000000000..d0c69992279 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query22.sql.out @@ -0,0 +1,23 @@ +-- Automatically generated by Gravitino Spark SQL test +-- !query +select i_product_name + ,i_brand + ,i_class + ,i_category + ,avg(inv_quantity_on_hand) qoh + from inventory + ,date_dim + ,item + where inv_date_sk=d_date_sk + and inv_item_sk=i_item_sk + and d_month_seq between 1212 and 1212 + 11 + group by rollup(i_product_name + ,i_brand + ,i_class + ,i_category) +order by qoh, i_product_name, i_brand, i_class, i_category +limit 100 +-- !query schema +struct +-- !query output + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query23.sql b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query23.sql new file mode 100644 index 00000000000..318648b8887 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query23.sql @@ -0,0 +1,107 @@ +-- start query 1 in stream 0 using template query23.tpl +with frequent_ss_items as + (select substr(i_item_desc,1,30) itemdesc,i_item_sk item_sk,d_date solddate,count(*) cnt + from store_sales + ,date_dim + ,item + where ss_sold_date_sk = d_date_sk + and ss_item_sk = i_item_sk + and d_year in (1999,1999+1,1999+2,1999+3) + group by substr(i_item_desc,1,30),i_item_sk,d_date + having count(*) >4), + max_store_sales as + (select max(csales) tpcds_cmax + from (select c_customer_sk,sum(ss_quantity*ss_sales_price) csales + from store_sales + ,customer + ,date_dim + where ss_customer_sk = c_customer_sk + and ss_sold_date_sk = d_date_sk + and d_year in (1999,1999+1,1999+2,1999+3) + group by c_customer_sk)), + best_ss_customer as + (select c_customer_sk,sum(ss_quantity*ss_sales_price) ssales + from store_sales + ,customer + where ss_customer_sk = c_customer_sk + group by c_customer_sk + having sum(ss_quantity*ss_sales_price) > (95/100.0) * (select + * +from + max_store_sales)) + select sum(sales) + from (select cs_quantity*cs_list_price sales + from catalog_sales + ,date_dim + where d_year = 1999 + and d_moy = 1 + and cs_sold_date_sk = d_date_sk + and cs_item_sk in (select item_sk from frequent_ss_items) + and cs_bill_customer_sk in (select c_customer_sk from best_ss_customer) + union all + select ws_quantity*ws_list_price sales + from web_sales + ,date_dim + where d_year = 1999 + and d_moy = 1 + and ws_sold_date_sk = d_date_sk + and ws_item_sk in (select item_sk from frequent_ss_items) + and ws_bill_customer_sk in (select c_customer_sk from best_ss_customer)) + limit 100; +with frequent_ss_items as + (select substr(i_item_desc,1,30) itemdesc,i_item_sk item_sk,d_date solddate,count(*) cnt + from store_sales + ,date_dim + ,item + where ss_sold_date_sk = d_date_sk + and ss_item_sk = i_item_sk + and d_year in (1999,1999 + 1,1999 + 2,1999 + 3) + group by substr(i_item_desc,1,30),i_item_sk,d_date + having count(*) >4), + max_store_sales as + (select max(csales) tpcds_cmax + from (select c_customer_sk,sum(ss_quantity*ss_sales_price) csales + from store_sales + ,customer + ,date_dim + where ss_customer_sk = c_customer_sk + and ss_sold_date_sk = d_date_sk + and d_year in (1999,1999+1,1999+2,1999+3) + group by c_customer_sk)), + best_ss_customer as + (select c_customer_sk,sum(ss_quantity*ss_sales_price) ssales + from store_sales + ,customer + where ss_customer_sk = c_customer_sk + group by c_customer_sk + having sum(ss_quantity*ss_sales_price) > (95/100.0) * (select + * + from max_store_sales)) + select c_last_name,c_first_name,sales + from (select c_last_name,c_first_name,sum(cs_quantity*cs_list_price) sales + from catalog_sales + ,customer + ,date_dim + where d_year = 1999 + and d_moy = 1 + and cs_sold_date_sk = d_date_sk + and cs_item_sk in (select item_sk from frequent_ss_items) + and cs_bill_customer_sk in (select c_customer_sk from best_ss_customer) + and cs_bill_customer_sk = c_customer_sk + group by c_last_name,c_first_name + union all + select c_last_name,c_first_name,sum(ws_quantity*ws_list_price) sales + from web_sales + ,customer + ,date_dim + where d_year = 1999 + and d_moy = 1 + and ws_sold_date_sk = d_date_sk + and ws_item_sk in (select item_sk from frequent_ss_items) + and ws_bill_customer_sk in (select c_customer_sk from best_ss_customer) + and ws_bill_customer_sk = c_customer_sk + group by c_last_name,c_first_name) + order by c_last_name,c_first_name,sales + limit 100; + + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query23.sql.out b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query23.sql.out new file mode 100644 index 00000000000..40cbe371080 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query23.sql.out @@ -0,0 +1,117 @@ +-- Automatically generated by Gravitino Spark SQL test +-- !query +with frequent_ss_items as + (select substr(i_item_desc,1,30) itemdesc,i_item_sk item_sk,d_date solddate,count(*) cnt + from store_sales + ,date_dim + ,item + where ss_sold_date_sk = d_date_sk + and ss_item_sk = i_item_sk + and d_year in (1999,1999+1,1999+2,1999+3) + group by substr(i_item_desc,1,30),i_item_sk,d_date + having count(*) >4), + max_store_sales as + (select max(csales) tpcds_cmax + from (select c_customer_sk,sum(ss_quantity*ss_sales_price) csales + from store_sales + ,customer + ,date_dim + where ss_customer_sk = c_customer_sk + and ss_sold_date_sk = d_date_sk + and d_year in (1999,1999+1,1999+2,1999+3) + group by c_customer_sk)), + best_ss_customer as + (select c_customer_sk,sum(ss_quantity*ss_sales_price) ssales + from store_sales + ,customer + where ss_customer_sk = c_customer_sk + group by c_customer_sk + having sum(ss_quantity*ss_sales_price) > (95/100.0) * (select + * +from + max_store_sales)) + select sum(sales) + from (select cs_quantity*cs_list_price sales + from catalog_sales + ,date_dim + where d_year = 1999 + and d_moy = 1 + and cs_sold_date_sk = d_date_sk + and cs_item_sk in (select item_sk from frequent_ss_items) + and cs_bill_customer_sk in (select c_customer_sk from best_ss_customer) + union all + select ws_quantity*ws_list_price sales + from web_sales + ,date_dim + where d_year = 1999 + and d_moy = 1 + and ws_sold_date_sk = d_date_sk + and ws_item_sk in (select item_sk from frequent_ss_items) + and ws_bill_customer_sk in (select c_customer_sk from best_ss_customer)) + limit 100 +-- !query schema +struct +-- !query output +NULL + + +-- !query +with frequent_ss_items as + (select substr(i_item_desc,1,30) itemdesc,i_item_sk item_sk,d_date solddate,count(*) cnt + from store_sales + ,date_dim + ,item + where ss_sold_date_sk = d_date_sk + and ss_item_sk = i_item_sk + and d_year in (1999,1999 + 1,1999 + 2,1999 + 3) + group by substr(i_item_desc,1,30),i_item_sk,d_date + having count(*) >4), + max_store_sales as + (select max(csales) tpcds_cmax + from (select c_customer_sk,sum(ss_quantity*ss_sales_price) csales + from store_sales + ,customer + ,date_dim + where ss_customer_sk = c_customer_sk + and ss_sold_date_sk = d_date_sk + and d_year in (1999,1999+1,1999+2,1999+3) + group by c_customer_sk)), + best_ss_customer as + (select c_customer_sk,sum(ss_quantity*ss_sales_price) ssales + from store_sales + ,customer + where ss_customer_sk = c_customer_sk + group by c_customer_sk + having sum(ss_quantity*ss_sales_price) > (95/100.0) * (select + * + from max_store_sales)) + select c_last_name,c_first_name,sales + from (select c_last_name,c_first_name,sum(cs_quantity*cs_list_price) sales + from catalog_sales + ,customer + ,date_dim + where d_year = 1999 + and d_moy = 1 + and cs_sold_date_sk = d_date_sk + and cs_item_sk in (select item_sk from frequent_ss_items) + and cs_bill_customer_sk in (select c_customer_sk from best_ss_customer) + and cs_bill_customer_sk = c_customer_sk + group by c_last_name,c_first_name + union all + select c_last_name,c_first_name,sum(ws_quantity*ws_list_price) sales + from web_sales + ,customer + ,date_dim + where d_year = 1999 + and d_moy = 1 + and ws_sold_date_sk = d_date_sk + and ws_item_sk in (select item_sk from frequent_ss_items) + and ws_bill_customer_sk in (select c_customer_sk from best_ss_customer) + and ws_bill_customer_sk = c_customer_sk + group by c_last_name,c_first_name) + order by c_last_name,c_first_name,sales + limit 100 +-- !query schema +struct +-- !query output + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query24.sql b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query24.sql new file mode 100644 index 00000000000..fb9433da1ba --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query24.sql @@ -0,0 +1,99 @@ +-- start query 1 in stream 0 using template query24.tpl +with ssales as +(select c_last_name + ,c_first_name + ,s_store_name + ,ca_state + ,s_state + ,i_color + ,i_current_price + ,i_manager_id + ,i_units + ,i_size + ,sum(ss_sales_price) netpaid +from store_sales + ,store_returns + ,store + ,item + ,customer + ,customer_address +where ss_ticket_number = sr_ticket_number + and ss_item_sk = sr_item_sk + and ss_customer_sk = c_customer_sk + and ss_item_sk = i_item_sk + and ss_store_sk = s_store_sk + and c_birth_country = upper(ca_country) + and s_zip = ca_zip +and s_market_id=7 +group by c_last_name + ,c_first_name + ,s_store_name + ,ca_state + ,s_state + ,i_color + ,i_current_price + ,i_manager_id + ,i_units + ,i_size) +select c_last_name + ,c_first_name + ,s_store_name + ,sum(netpaid) paid +from ssales +where i_color = 'orchid' +group by c_last_name + ,c_first_name + ,s_store_name +having sum(netpaid) > (select 0.05*avg(netpaid) + from ssales) +; +with ssales as +(select c_last_name + ,c_first_name + ,s_store_name + ,ca_state + ,s_state + ,i_color + ,i_current_price + ,i_manager_id + ,i_units + ,i_size + ,sum(ss_sales_price) netpaid +from store_sales + ,store_returns + ,store + ,item + ,customer + ,customer_address +where ss_ticket_number = sr_ticket_number + and ss_item_sk = sr_item_sk + and ss_customer_sk = c_customer_sk + and ss_item_sk = i_item_sk + and ss_store_sk = s_store_sk + and c_birth_country = upper(ca_country) + and s_zip = ca_zip + and s_market_id = 7 +group by c_last_name + ,c_first_name + ,s_store_name + ,ca_state + ,s_state + ,i_color + ,i_current_price + ,i_manager_id + ,i_units + ,i_size) +select c_last_name + ,c_first_name + ,s_store_name + ,sum(netpaid) paid +from ssales +where i_color = 'chiffon' +group by c_last_name + ,c_first_name + ,s_store_name +having sum(netpaid) > (select 0.05*avg(netpaid) + from ssales) +; + + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query24.sql.out b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query24.sql.out new file mode 100644 index 00000000000..eb4b1da73b9 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query24.sql.out @@ -0,0 +1,107 @@ +-- Automatically generated by Gravitino Spark SQL test +-- !query +with ssales as +(select c_last_name + ,c_first_name + ,s_store_name + ,ca_state + ,s_state + ,i_color + ,i_current_price + ,i_manager_id + ,i_units + ,i_size + ,sum(ss_sales_price) netpaid +from store_sales + ,store_returns + ,store + ,item + ,customer + ,customer_address +where ss_ticket_number = sr_ticket_number + and ss_item_sk = sr_item_sk + and ss_customer_sk = c_customer_sk + and ss_item_sk = i_item_sk + and ss_store_sk = s_store_sk + and c_birth_country = upper(ca_country) + and s_zip = ca_zip +and s_market_id=7 +group by c_last_name + ,c_first_name + ,s_store_name + ,ca_state + ,s_state + ,i_color + ,i_current_price + ,i_manager_id + ,i_units + ,i_size) +select c_last_name + ,c_first_name + ,s_store_name + ,sum(netpaid) paid +from ssales +where i_color = 'orchid' +group by c_last_name + ,c_first_name + ,s_store_name +having sum(netpaid) > (select 0.05*avg(netpaid) + from ssales) +-- !query schema +struct +-- !query output + + + +-- !query +with ssales as +(select c_last_name + ,c_first_name + ,s_store_name + ,ca_state + ,s_state + ,i_color + ,i_current_price + ,i_manager_id + ,i_units + ,i_size + ,sum(ss_sales_price) netpaid +from store_sales + ,store_returns + ,store + ,item + ,customer + ,customer_address +where ss_ticket_number = sr_ticket_number + and ss_item_sk = sr_item_sk + and ss_customer_sk = c_customer_sk + and ss_item_sk = i_item_sk + and ss_store_sk = s_store_sk + and c_birth_country = upper(ca_country) + and s_zip = ca_zip + and s_market_id = 7 +group by c_last_name + ,c_first_name + ,s_store_name + ,ca_state + ,s_state + ,i_color + ,i_current_price + ,i_manager_id + ,i_units + ,i_size) +select c_last_name + ,c_first_name + ,s_store_name + ,sum(netpaid) paid +from ssales +where i_color = 'chiffon' +group by c_last_name + ,c_first_name + ,s_store_name +having sum(netpaid) > (select 0.05*avg(netpaid) + from ssales) +-- !query schema +struct +-- !query output + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query25.sql b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query25.sql new file mode 100644 index 00000000000..6de9c9772b0 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query25.sql @@ -0,0 +1,48 @@ +-- start query 1 in stream 0 using template query25.tpl +select + i_item_id + ,i_item_desc + ,s_store_id + ,s_store_name + ,sum(ss_net_profit) as store_sales_profit + ,sum(sr_net_loss) as store_returns_loss + ,sum(cs_net_profit) as catalog_sales_profit + from + store_sales + ,store_returns + ,catalog_sales + ,date_dim d1 + ,date_dim d2 + ,date_dim d3 + ,store + ,item + where + d1.d_moy = 4 + and d1.d_year = 2000 + and d1.d_date_sk = ss_sold_date_sk + and i_item_sk = ss_item_sk + and s_store_sk = ss_store_sk + and ss_customer_sk = sr_customer_sk + and ss_item_sk = sr_item_sk + and ss_ticket_number = sr_ticket_number + and sr_returned_date_sk = d2.d_date_sk + and d2.d_moy between 4 and 10 + and d2.d_year = 2000 + and sr_customer_sk = cs_bill_customer_sk + and sr_item_sk = cs_item_sk + and cs_sold_date_sk = d3.d_date_sk + and d3.d_moy between 4 and 10 + and d3.d_year = 2000 + group by + i_item_id + ,i_item_desc + ,s_store_id + ,s_store_name + order by + i_item_id + ,i_item_desc + ,s_store_id + ,s_store_name + limit 100; + + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query25.sql.out b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query25.sql.out new file mode 100644 index 00000000000..b63569872cd --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query25.sql.out @@ -0,0 +1,51 @@ +-- Automatically generated by Gravitino Spark SQL test +-- !query +select + i_item_id + ,i_item_desc + ,s_store_id + ,s_store_name + ,sum(ss_net_profit) as store_sales_profit + ,sum(sr_net_loss) as store_returns_loss + ,sum(cs_net_profit) as catalog_sales_profit + from + store_sales + ,store_returns + ,catalog_sales + ,date_dim d1 + ,date_dim d2 + ,date_dim d3 + ,store + ,item + where + d1.d_moy = 4 + and d1.d_year = 2000 + and d1.d_date_sk = ss_sold_date_sk + and i_item_sk = ss_item_sk + and s_store_sk = ss_store_sk + and ss_customer_sk = sr_customer_sk + and ss_item_sk = sr_item_sk + and ss_ticket_number = sr_ticket_number + and sr_returned_date_sk = d2.d_date_sk + and d2.d_moy between 4 and 10 + and d2.d_year = 2000 + and sr_customer_sk = cs_bill_customer_sk + and sr_item_sk = cs_item_sk + and cs_sold_date_sk = d3.d_date_sk + and d3.d_moy between 4 and 10 + and d3.d_year = 2000 + group by + i_item_id + ,i_item_desc + ,s_store_id + ,s_store_name + order by + i_item_id + ,i_item_desc + ,s_store_id + ,s_store_name + limit 100 +-- !query schema +struct +-- !query output + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query26.sql b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query26.sql new file mode 100644 index 00000000000..837634da38a --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query26.sql @@ -0,0 +1,21 @@ +-- start query 1 in stream 0 using template query26.tpl +select i_item_id, + avg(cs_quantity) agg1, + avg(cs_list_price) agg2, + avg(cs_coupon_amt) agg3, + avg(cs_sales_price) agg4 + from catalog_sales, customer_demographics, date_dim, item, promotion + where cs_sold_date_sk = d_date_sk and + cs_item_sk = i_item_sk and + cs_bill_cdemo_sk = cd_demo_sk and + cs_promo_sk = p_promo_sk and + cd_gender = 'F' and + cd_marital_status = 'W' and + cd_education_status = 'Primary' and + (p_channel_email = 'N' or p_channel_event = 'N') and + d_year = 1998 + group by i_item_id + order by i_item_id + limit 100; + + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query26.sql.out b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query26.sql.out new file mode 100644 index 00000000000..f8f794fb7c7 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query26.sql.out @@ -0,0 +1,24 @@ +-- Automatically generated by Gravitino Spark SQL test +-- !query +select i_item_id, + avg(cs_quantity) agg1, + avg(cs_list_price) agg2, + avg(cs_coupon_amt) agg3, + avg(cs_sales_price) agg4 + from catalog_sales, customer_demographics, date_dim, item, promotion + where cs_sold_date_sk = d_date_sk and + cs_item_sk = i_item_sk and + cs_bill_cdemo_sk = cd_demo_sk and + cs_promo_sk = p_promo_sk and + cd_gender = 'F' and + cd_marital_status = 'W' and + cd_education_status = 'Primary' and + (p_channel_email = 'N' or p_channel_event = 'N') and + d_year = 1998 + group by i_item_id + order by i_item_id + limit 100 +-- !query schema +struct +-- !query output + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query27.sql b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query27.sql new file mode 100644 index 00000000000..35c2fcaac71 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query27.sql @@ -0,0 +1,23 @@ +-- start query 1 in stream 0 using template query27.tpl +select i_item_id, + s_state, grouping(s_state) g_state, + avg(ss_quantity) agg1, + avg(ss_list_price) agg2, + avg(ss_coupon_amt) agg3, + avg(ss_sales_price) agg4 + from store_sales, customer_demographics, date_dim, store, item + where ss_sold_date_sk = d_date_sk and + ss_item_sk = i_item_sk and + ss_store_sk = s_store_sk and + ss_cdemo_sk = cd_demo_sk and + cd_gender = 'F' and + cd_marital_status = 'W' and + cd_education_status = 'Primary' and + d_year = 1998 and + s_state in ('TN','TN', 'TN', 'TN', 'TN', 'TN') + group by rollup (i_item_id, s_state) + order by i_item_id + ,s_state + limit 100; + + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query27.sql.out b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query27.sql.out new file mode 100644 index 00000000000..a21834c33f0 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query27.sql.out @@ -0,0 +1,26 @@ +-- Automatically generated by Gravitino Spark SQL test +-- !query +select i_item_id, + s_state, grouping(s_state) g_state, + avg(ss_quantity) agg1, + avg(ss_list_price) agg2, + avg(ss_coupon_amt) agg3, + avg(ss_sales_price) agg4 + from store_sales, customer_demographics, date_dim, store, item + where ss_sold_date_sk = d_date_sk and + ss_item_sk = i_item_sk and + ss_store_sk = s_store_sk and + ss_cdemo_sk = cd_demo_sk and + cd_gender = 'F' and + cd_marital_status = 'W' and + cd_education_status = 'Primary' and + d_year = 1998 and + s_state in ('TN','TN', 'TN', 'TN', 'TN', 'TN') + group by rollup (i_item_id, s_state) + order by i_item_id + ,s_state + limit 100 +-- !query schema +struct +-- !query output + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query28.sql b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query28.sql new file mode 100644 index 00000000000..e3c6d114380 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query28.sql @@ -0,0 +1,53 @@ +-- start query 1 in stream 0 using template query28.tpl +select * +from (select avg(ss_list_price) B1_LP + ,count(ss_list_price) B1_CNT + ,count(distinct ss_list_price) B1_CNTD + from store_sales + where ss_quantity between 0 and 5 + and (ss_list_price between 11 and 11+10 + or ss_coupon_amt between 460 and 460+1000 + or ss_wholesale_cost between 14 and 14+20)) B1, + (select avg(ss_list_price) B2_LP + ,count(ss_list_price) B2_CNT + ,count(distinct ss_list_price) B2_CNTD + from store_sales + where ss_quantity between 6 and 10 + and (ss_list_price between 91 and 91+10 + or ss_coupon_amt between 1430 and 1430+1000 + or ss_wholesale_cost between 32 and 32+20)) B2, + (select avg(ss_list_price) B3_LP + ,count(ss_list_price) B3_CNT + ,count(distinct ss_list_price) B3_CNTD + from store_sales + where ss_quantity between 11 and 15 + and (ss_list_price between 66 and 66+10 + or ss_coupon_amt between 920 and 920+1000 + or ss_wholesale_cost between 4 and 4+20)) B3, + (select avg(ss_list_price) B4_LP + ,count(ss_list_price) B4_CNT + ,count(distinct ss_list_price) B4_CNTD + from store_sales + where ss_quantity between 16 and 20 + and (ss_list_price between 142 and 142+10 + or ss_coupon_amt between 3054 and 3054+1000 + or ss_wholesale_cost between 80 and 80+20)) B4, + (select avg(ss_list_price) B5_LP + ,count(ss_list_price) B5_CNT + ,count(distinct ss_list_price) B5_CNTD + from store_sales + where ss_quantity between 21 and 25 + and (ss_list_price between 135 and 135+10 + or ss_coupon_amt between 14180 and 14180+1000 + or ss_wholesale_cost between 38 and 38+20)) B5, + (select avg(ss_list_price) B6_LP + ,count(ss_list_price) B6_CNT + ,count(distinct ss_list_price) B6_CNTD + from store_sales + where ss_quantity between 26 and 30 + and (ss_list_price between 28 and 28+10 + or ss_coupon_amt between 2513 and 2513+1000 + or ss_wholesale_cost between 42 and 42+20)) B6 +limit 100; + + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query28.sql.out b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query28.sql.out new file mode 100644 index 00000000000..0b5a4354ddf --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query28.sql.out @@ -0,0 +1,56 @@ +-- Automatically generated by Gravitino Spark SQL test +-- !query +select * +from (select avg(ss_list_price) B1_LP + ,count(ss_list_price) B1_CNT + ,count(distinct ss_list_price) B1_CNTD + from store_sales + where ss_quantity between 0 and 5 + and (ss_list_price between 11 and 11+10 + or ss_coupon_amt between 460 and 460+1000 + or ss_wholesale_cost between 14 and 14+20)) B1, + (select avg(ss_list_price) B2_LP + ,count(ss_list_price) B2_CNT + ,count(distinct ss_list_price) B2_CNTD + from store_sales + where ss_quantity between 6 and 10 + and (ss_list_price between 91 and 91+10 + or ss_coupon_amt between 1430 and 1430+1000 + or ss_wholesale_cost between 32 and 32+20)) B2, + (select avg(ss_list_price) B3_LP + ,count(ss_list_price) B3_CNT + ,count(distinct ss_list_price) B3_CNTD + from store_sales + where ss_quantity between 11 and 15 + and (ss_list_price between 66 and 66+10 + or ss_coupon_amt between 920 and 920+1000 + or ss_wholesale_cost between 4 and 4+20)) B3, + (select avg(ss_list_price) B4_LP + ,count(ss_list_price) B4_CNT + ,count(distinct ss_list_price) B4_CNTD + from store_sales + where ss_quantity between 16 and 20 + and (ss_list_price between 142 and 142+10 + or ss_coupon_amt between 3054 and 3054+1000 + or ss_wholesale_cost between 80 and 80+20)) B4, + (select avg(ss_list_price) B5_LP + ,count(ss_list_price) B5_CNT + ,count(distinct ss_list_price) B5_CNTD + from store_sales + where ss_quantity between 21 and 25 + and (ss_list_price between 135 and 135+10 + or ss_coupon_amt between 14180 and 14180+1000 + or ss_wholesale_cost between 38 and 38+20)) B5, + (select avg(ss_list_price) B6_LP + ,count(ss_list_price) B6_CNT + ,count(distinct ss_list_price) B6_CNTD + from store_sales + where ss_quantity between 26 and 30 + and (ss_list_price between 28 and 28+10 + or ss_coupon_amt between 2513 and 2513+1000 + or ss_wholesale_cost between 42 and 42+20)) B6 +limit 100 +-- !query schema +struct +-- !query output +32.010175 343 333 70.699500 360 350 36.579143 350 341 136.749783 277 272 81.930455 308 306 66.426033 363 353 diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query29.sql b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query29.sql new file mode 100644 index 00000000000..9e2ab196585 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query29.sql @@ -0,0 +1,47 @@ +-- start query 1 in stream 0 using template query29.tpl +select + i_item_id + ,i_item_desc + ,s_store_id + ,s_store_name + ,sum(ss_quantity) as store_sales_quantity + ,sum(sr_return_quantity) as store_returns_quantity + ,sum(cs_quantity) as catalog_sales_quantity + from + store_sales + ,store_returns + ,catalog_sales + ,date_dim d1 + ,date_dim d2 + ,date_dim d3 + ,store + ,item + where + d1.d_moy = 4 + and d1.d_year = 1999 + and d1.d_date_sk = ss_sold_date_sk + and i_item_sk = ss_item_sk + and s_store_sk = ss_store_sk + and ss_customer_sk = sr_customer_sk + and ss_item_sk = sr_item_sk + and ss_ticket_number = sr_ticket_number + and sr_returned_date_sk = d2.d_date_sk + and d2.d_moy between 4 and 4 + 3 + and d2.d_year = 1999 + and sr_customer_sk = cs_bill_customer_sk + and sr_item_sk = cs_item_sk + and cs_sold_date_sk = d3.d_date_sk + and d3.d_year in (1999,1999+1,1999+2) + group by + i_item_id + ,i_item_desc + ,s_store_id + ,s_store_name + order by + i_item_id + ,i_item_desc + ,s_store_id + ,s_store_name + limit 100; + + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query29.sql.out b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query29.sql.out new file mode 100644 index 00000000000..ec6ee1083d6 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query29.sql.out @@ -0,0 +1,50 @@ +-- Automatically generated by Gravitino Spark SQL test +-- !query +select + i_item_id + ,i_item_desc + ,s_store_id + ,s_store_name + ,sum(ss_quantity) as store_sales_quantity + ,sum(sr_return_quantity) as store_returns_quantity + ,sum(cs_quantity) as catalog_sales_quantity + from + store_sales + ,store_returns + ,catalog_sales + ,date_dim d1 + ,date_dim d2 + ,date_dim d3 + ,store + ,item + where + d1.d_moy = 4 + and d1.d_year = 1999 + and d1.d_date_sk = ss_sold_date_sk + and i_item_sk = ss_item_sk + and s_store_sk = ss_store_sk + and ss_customer_sk = sr_customer_sk + and ss_item_sk = sr_item_sk + and ss_ticket_number = sr_ticket_number + and sr_returned_date_sk = d2.d_date_sk + and d2.d_moy between 4 and 4 + 3 + and d2.d_year = 1999 + and sr_customer_sk = cs_bill_customer_sk + and sr_item_sk = cs_item_sk + and cs_sold_date_sk = d3.d_date_sk + and d3.d_year in (1999,1999+1,1999+2) + group by + i_item_id + ,i_item_desc + ,s_store_id + ,s_store_name + order by + i_item_id + ,i_item_desc + ,s_store_id + ,s_store_name + limit 100 +-- !query schema +struct +-- !query output + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query3.sql b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query3.sql new file mode 100644 index 00000000000..a36085e9d71 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query3.sql @@ -0,0 +1,21 @@ +-- start query 1 in stream 0 using template query3.tpl +select dt.d_year + ,item.i_brand_id brand_id + ,item.i_brand brand + ,sum(ss_ext_sales_price) sum_agg + from date_dim dt + ,store_sales + ,item + where dt.d_date_sk = store_sales.ss_sold_date_sk + and store_sales.ss_item_sk = item.i_item_sk + and item.i_manufact_id = 436 + and dt.d_moy=12 + group by dt.d_year + ,item.i_brand + ,item.i_brand_id + order by dt.d_year + ,sum_agg desc + ,brand_id + limit 100; + + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query3.sql.out b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query3.sql.out new file mode 100644 index 00000000000..afd12710d1b --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query3.sql.out @@ -0,0 +1,29 @@ +-- Automatically generated by Gravitino Spark SQL test +-- !query +select dt.d_year + ,item.i_brand_id brand_id + ,item.i_brand brand + ,sum(ss_ext_sales_price) sum_agg + from date_dim dt + ,store_sales + ,item + where dt.d_date_sk = store_sales.ss_sold_date_sk + and store_sales.ss_item_sk = item.i_item_sk + and item.i_manufact_id = 436 + and dt.d_moy=12 + group by dt.d_year + ,item.i_brand + ,item.i_brand_id + order by dt.d_year + ,sum_agg desc + ,brand_id + limit 100 +-- !query schema +struct +-- !query output +1998 10003008 exportiunivamalg #x 2997.82 +1999 3003001 exportiexporti #x 552.87 +2001 10003008 exportiunivamalg #x 3110.54 +2001 6012004 importobrand #x 1282.05 +2002 6008005 namelesscorp #x 432.48 +2002 8004007 edu packnameless #x 220.46 diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query30.sql b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query30.sql new file mode 100644 index 00000000000..aefa20ac0fe --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query30.sql @@ -0,0 +1,31 @@ +-- start query 1 in stream 0 using template query30.tpl +with customer_total_return as + (select wr_returning_customer_sk as ctr_customer_sk + ,ca_state as ctr_state, + sum(wr_return_amt) as ctr_total_return + from web_returns + ,date_dim + ,customer_address + where wr_returned_date_sk = d_date_sk + and d_year =2002 + and wr_returning_addr_sk = ca_address_sk + group by wr_returning_customer_sk + ,ca_state) + select c_customer_id,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag + ,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address + ,c_last_review_date,ctr_total_return + from customer_total_return ctr1 + ,customer_address + ,customer + where ctr1.ctr_total_return > (select avg(ctr_total_return)*1.2 + from customer_total_return ctr2 + where ctr1.ctr_state = ctr2.ctr_state) + and ca_address_sk = c_current_addr_sk + and ca_state = 'IL' + and ctr1.ctr_customer_sk = c_customer_sk + order by c_customer_id,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag + ,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address + ,c_last_review_date,ctr_total_return +limit 100; + + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query30.sql.out b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query30.sql.out new file mode 100644 index 00000000000..1dd946f3841 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query30.sql.out @@ -0,0 +1,34 @@ +-- Automatically generated by Gravitino Spark SQL test +-- !query +with customer_total_return as + (select wr_returning_customer_sk as ctr_customer_sk + ,ca_state as ctr_state, + sum(wr_return_amt) as ctr_total_return + from web_returns + ,date_dim + ,customer_address + where wr_returned_date_sk = d_date_sk + and d_year =2002 + and wr_returning_addr_sk = ca_address_sk + group by wr_returning_customer_sk + ,ca_state) + select c_customer_id,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag + ,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address + ,c_last_review_date,ctr_total_return + from customer_total_return ctr1 + ,customer_address + ,customer + where ctr1.ctr_total_return > (select avg(ctr_total_return)*1.2 + from customer_total_return ctr2 + where ctr1.ctr_state = ctr2.ctr_state) + and ca_address_sk = c_current_addr_sk + and ca_state = 'IL' + and ctr1.ctr_customer_sk = c_customer_sk + order by c_customer_id,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag + ,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address + ,c_last_review_date,ctr_total_return +limit 100 +-- !query schema +struct +-- !query output + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query31.sql b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query31.sql new file mode 100644 index 00000000000..ae0d6585b2a --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query31.sql @@ -0,0 +1,52 @@ +-- start query 1 in stream 0 using template query31.tpl +with ss as + (select ca_county,d_qoy, d_year,sum(ss_ext_sales_price) as store_sales + from store_sales,date_dim,customer_address + where ss_sold_date_sk = d_date_sk + and ss_addr_sk=ca_address_sk + group by ca_county,d_qoy, d_year), + ws as + (select ca_county,d_qoy, d_year,sum(ws_ext_sales_price) as web_sales + from web_sales,date_dim,customer_address + where ws_sold_date_sk = d_date_sk + and ws_bill_addr_sk=ca_address_sk + group by ca_county,d_qoy, d_year) + select + ss1.ca_county + ,ss1.d_year + ,ws2.web_sales/ws1.web_sales web_q1_q2_increase + ,ss2.store_sales/ss1.store_sales store_q1_q2_increase + ,ws3.web_sales/ws2.web_sales web_q2_q3_increase + ,ss3.store_sales/ss2.store_sales store_q2_q3_increase + from + ss ss1 + ,ss ss2 + ,ss ss3 + ,ws ws1 + ,ws ws2 + ,ws ws3 + where + ss1.d_qoy = 1 + and ss1.d_year = 2000 + and ss1.ca_county = ss2.ca_county + and ss2.d_qoy = 2 + and ss2.d_year = 2000 + and ss2.ca_county = ss3.ca_county + and ss3.d_qoy = 3 + and ss3.d_year = 2000 + and ss1.ca_county = ws1.ca_county + and ws1.d_qoy = 1 + and ws1.d_year = 2000 + and ws1.ca_county = ws2.ca_county + and ws2.d_qoy = 2 + and ws2.d_year = 2000 + and ws1.ca_county = ws3.ca_county + and ws3.d_qoy = 3 + and ws3.d_year =2000 + and case when ws1.web_sales > 0 then ws2.web_sales/ws1.web_sales else null end + > case when ss1.store_sales > 0 then ss2.store_sales/ss1.store_sales else null end + and case when ws2.web_sales > 0 then ws3.web_sales/ws2.web_sales else null end + > case when ss2.store_sales > 0 then ss3.store_sales/ss2.store_sales else null end + order by ss1.d_year; + + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query31.sql.out b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query31.sql.out new file mode 100644 index 00000000000..72a0b747856 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query31.sql.out @@ -0,0 +1,55 @@ +-- Automatically generated by Gravitino Spark SQL test +-- !query +with ss as + (select ca_county,d_qoy, d_year,sum(ss_ext_sales_price) as store_sales + from store_sales,date_dim,customer_address + where ss_sold_date_sk = d_date_sk + and ss_addr_sk=ca_address_sk + group by ca_county,d_qoy, d_year), + ws as + (select ca_county,d_qoy, d_year,sum(ws_ext_sales_price) as web_sales + from web_sales,date_dim,customer_address + where ws_sold_date_sk = d_date_sk + and ws_bill_addr_sk=ca_address_sk + group by ca_county,d_qoy, d_year) + select + ss1.ca_county + ,ss1.d_year + ,ws2.web_sales/ws1.web_sales web_q1_q2_increase + ,ss2.store_sales/ss1.store_sales store_q1_q2_increase + ,ws3.web_sales/ws2.web_sales web_q2_q3_increase + ,ss3.store_sales/ss2.store_sales store_q2_q3_increase + from + ss ss1 + ,ss ss2 + ,ss ss3 + ,ws ws1 + ,ws ws2 + ,ws ws3 + where + ss1.d_qoy = 1 + and ss1.d_year = 2000 + and ss1.ca_county = ss2.ca_county + and ss2.d_qoy = 2 + and ss2.d_year = 2000 + and ss2.ca_county = ss3.ca_county + and ss3.d_qoy = 3 + and ss3.d_year = 2000 + and ss1.ca_county = ws1.ca_county + and ws1.d_qoy = 1 + and ws1.d_year = 2000 + and ws1.ca_county = ws2.ca_county + and ws2.d_qoy = 2 + and ws2.d_year = 2000 + and ws1.ca_county = ws3.ca_county + and ws3.d_qoy = 3 + and ws3.d_year =2000 + and case when ws1.web_sales > 0 then ws2.web_sales/ws1.web_sales else null end + > case when ss1.store_sales > 0 then ss2.store_sales/ss1.store_sales else null end + and case when ws2.web_sales > 0 then ws3.web_sales/ws2.web_sales else null end + > case when ss2.store_sales > 0 then ss3.store_sales/ss2.store_sales else null end + order by ss1.d_year +-- !query schema +struct +-- !query output + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query32.sql b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query32.sql new file mode 100644 index 00000000000..df7e726ef6e --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query32.sql @@ -0,0 +1,28 @@ +-- start query 1 in stream 0 using template query32.tpl +select sum(cs_ext_discount_amt) as excess_discount_amount +from + catalog_sales + ,item + ,date_dim +where +i_manufact_id = 269 +and i_item_sk = cs_item_sk +and d_date between '1998-03-18' and + (cast('1998-03-18' as date) + interval 90 days) +and d_date_sk = cs_sold_date_sk +and cs_ext_discount_amt + > ( + select + 1.3 * avg(cs_ext_discount_amt) + from + catalog_sales + ,date_dim + where + cs_item_sk = i_item_sk + and d_date between '1998-03-18' and + (cast('1998-03-18' as date) + interval 90 days) + and d_date_sk = cs_sold_date_sk + ) +limit 100; + + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query32.sql.out b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query32.sql.out new file mode 100644 index 00000000000..89ca9d3e57a --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query32.sql.out @@ -0,0 +1,31 @@ +-- Automatically generated by Gravitino Spark SQL test +-- !query +select sum(cs_ext_discount_amt) as excess_discount_amount +from + catalog_sales + ,item + ,date_dim +where +i_manufact_id = 269 +and i_item_sk = cs_item_sk +and d_date between '1998-03-18' and + (cast('1998-03-18' as date) + interval 90 days) +and d_date_sk = cs_sold_date_sk +and cs_ext_discount_amt + > ( + select + 1.3 * avg(cs_ext_discount_amt) + from + catalog_sales + ,date_dim + where + cs_item_sk = i_item_sk + and d_date between '1998-03-18' and + (cast('1998-03-18' as date) + interval 90 days) + and d_date_sk = cs_sold_date_sk + ) +limit 100 +-- !query schema +struct +-- !query output +NULL diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query33.sql b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query33.sql new file mode 100644 index 00000000000..05b01059d30 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query33.sql @@ -0,0 +1,75 @@ +-- start query 1 in stream 0 using template query33.tpl +with ss as ( + select + i_manufact_id,sum(ss_ext_sales_price) total_sales + from + store_sales, + date_dim, + customer_address, + item + where + i_manufact_id in (select + i_manufact_id +from + item +where i_category in ('Books')) + and ss_item_sk = i_item_sk + and ss_sold_date_sk = d_date_sk + and d_year = 1999 + and d_moy = 3 + and ss_addr_sk = ca_address_sk + and ca_gmt_offset = -5 + group by i_manufact_id), + cs as ( + select + i_manufact_id,sum(cs_ext_sales_price) total_sales + from + catalog_sales, + date_dim, + customer_address, + item + where + i_manufact_id in (select + i_manufact_id +from + item +where i_category in ('Books')) + and cs_item_sk = i_item_sk + and cs_sold_date_sk = d_date_sk + and d_year = 1999 + and d_moy = 3 + and cs_bill_addr_sk = ca_address_sk + and ca_gmt_offset = -5 + group by i_manufact_id), + ws as ( + select + i_manufact_id,sum(ws_ext_sales_price) total_sales + from + web_sales, + date_dim, + customer_address, + item + where + i_manufact_id in (select + i_manufact_id +from + item +where i_category in ('Books')) + and ws_item_sk = i_item_sk + and ws_sold_date_sk = d_date_sk + and d_year = 1999 + and d_moy = 3 + and ws_bill_addr_sk = ca_address_sk + and ca_gmt_offset = -5 + group by i_manufact_id) + select i_manufact_id ,sum(total_sales) total_sales + from (select * from ss + union all + select * from cs + union all + select * from ws) tmp1 + group by i_manufact_id + order by total_sales +limit 100; + + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query33.sql.out b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query33.sql.out new file mode 100644 index 00000000000..846de897114 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query33.sql.out @@ -0,0 +1,127 @@ +-- Automatically generated by Gravitino Spark SQL test +-- !query +with ss as ( + select + i_manufact_id,sum(ss_ext_sales_price) total_sales + from + store_sales, + date_dim, + customer_address, + item + where + i_manufact_id in (select + i_manufact_id +from + item +where i_category in ('Books')) + and ss_item_sk = i_item_sk + and ss_sold_date_sk = d_date_sk + and d_year = 1999 + and d_moy = 3 + and ss_addr_sk = ca_address_sk + and ca_gmt_offset = -5 + group by i_manufact_id), + cs as ( + select + i_manufact_id,sum(cs_ext_sales_price) total_sales + from + catalog_sales, + date_dim, + customer_address, + item + where + i_manufact_id in (select + i_manufact_id +from + item +where i_category in ('Books')) + and cs_item_sk = i_item_sk + and cs_sold_date_sk = d_date_sk + and d_year = 1999 + and d_moy = 3 + and cs_bill_addr_sk = ca_address_sk + and ca_gmt_offset = -5 + group by i_manufact_id), + ws as ( + select + i_manufact_id,sum(ws_ext_sales_price) total_sales + from + web_sales, + date_dim, + customer_address, + item + where + i_manufact_id in (select + i_manufact_id +from + item +where i_category in ('Books')) + and ws_item_sk = i_item_sk + and ws_sold_date_sk = d_date_sk + and d_year = 1999 + and d_moy = 3 + and ws_bill_addr_sk = ca_address_sk + and ca_gmt_offset = -5 + group by i_manufact_id) + select i_manufact_id ,sum(total_sales) total_sales + from (select * from ss + union all + select * from cs + union all + select * from ws) tmp1 + group by i_manufact_id + order by total_sales +limit 100 +-- !query schema +struct +-- !query output +118 10013.31 +124 2784.38 +134 2143.68 +135 6062.40 +141 2025.00 +150 793.98 +151 1136.85 +154 100.88 +16 9517.74 +162 8420.40 +164 1490.76 +165 1223.60 +188 3229.20 +19 782.08 +197 6842.64 +219 2421.65 +220 2782.56 +221 1365.44 +231 0.00 +251 24.88 +252 10870.40 +256 266.90 +261 168.60 +266 NULL +286 256.41 +291 1997.06 +321 4621.50 +35 1202.85 +359 2116.26 +36 2398.88 +370 1100.67 +392 951.73 +40 184.10 +417 29.65 +443 7104.24 +456 722.40 +507 67.08 +51 365.97 +521 360.00 +53 2369.01 +567 806.93 +57 135.33 +614 2227.72 +619 390.00 +624 6481.17 +685 4736.52 +7 672.90 +74 124.20 +82 7.68 +97 45.22 diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query34.sql b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query34.sql new file mode 100644 index 00000000000..2bfcef4ab2c --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query34.sql @@ -0,0 +1,31 @@ +-- start query 1 in stream 0 using template query34.tpl +select c_last_name + ,c_first_name + ,c_salutation + ,c_preferred_cust_flag + ,ss_ticket_number + ,cnt from + (select ss_ticket_number + ,ss_customer_sk + ,count(*) cnt + from store_sales,date_dim,store,household_demographics + where store_sales.ss_sold_date_sk = date_dim.d_date_sk + and store_sales.ss_store_sk = store.s_store_sk + and store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk + and (date_dim.d_dom between 1 and 3 or date_dim.d_dom between 25 and 28) + and (household_demographics.hd_buy_potential = '>10000' or + household_demographics.hd_buy_potential = 'Unknown') + and household_demographics.hd_vehicle_count > 0 + and (case when household_demographics.hd_vehicle_count > 0 + then household_demographics.hd_dep_count/ household_demographics.hd_vehicle_count + else null + end) > 1.2 + and date_dim.d_year in (1998,1998+1,1998+2) + and store.s_county in ('Williamson County','Williamson County','Williamson County','Williamson County', + 'Williamson County','Williamson County','Williamson County','Williamson County') + group by ss_ticket_number,ss_customer_sk) dn,customer + where ss_customer_sk = c_customer_sk + and cnt between 15 and 20 + order by c_last_name,c_first_name,c_salutation,c_preferred_cust_flag desc, ss_ticket_number; + + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query34.sql.out b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query34.sql.out new file mode 100644 index 00000000000..44c104e611f --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query34.sql.out @@ -0,0 +1,34 @@ +-- Automatically generated by Gravitino Spark SQL test +-- !query +select c_last_name + ,c_first_name + ,c_salutation + ,c_preferred_cust_flag + ,ss_ticket_number + ,cnt from + (select ss_ticket_number + ,ss_customer_sk + ,count(*) cnt + from store_sales,date_dim,store,household_demographics + where store_sales.ss_sold_date_sk = date_dim.d_date_sk + and store_sales.ss_store_sk = store.s_store_sk + and store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk + and (date_dim.d_dom between 1 and 3 or date_dim.d_dom between 25 and 28) + and (household_demographics.hd_buy_potential = '>10000' or + household_demographics.hd_buy_potential = 'Unknown') + and household_demographics.hd_vehicle_count > 0 + and (case when household_demographics.hd_vehicle_count > 0 + then household_demographics.hd_dep_count/ household_demographics.hd_vehicle_count + else null + end) > 1.2 + and date_dim.d_year in (1998,1998+1,1998+2) + and store.s_county in ('Williamson County','Williamson County','Williamson County','Williamson County', + 'Williamson County','Williamson County','Williamson County','Williamson County') + group by ss_ticket_number,ss_customer_sk) dn,customer + where ss_customer_sk = c_customer_sk + and cnt between 15 and 20 + order by c_last_name,c_first_name,c_salutation,c_preferred_cust_flag desc, ss_ticket_number +-- !query schema +struct +-- !query output + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query35.sql b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query35.sql new file mode 100644 index 00000000000..fc41dc96038 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query35.sql @@ -0,0 +1,58 @@ +-- start query 1 in stream 0 using template query35.tpl +select + ca_state, + cd_gender, + cd_marital_status, + cd_dep_count, + count(*) cnt1, + avg(cd_dep_count), + max(cd_dep_count), + sum(cd_dep_count), + cd_dep_employed_count, + count(*) cnt2, + avg(cd_dep_employed_count), + max(cd_dep_employed_count), + sum(cd_dep_employed_count), + cd_dep_college_count, + count(*) cnt3, + avg(cd_dep_college_count), + max(cd_dep_college_count), + sum(cd_dep_college_count) + from + customer c,customer_address ca,customer_demographics + where + c.c_current_addr_sk = ca.ca_address_sk and + cd_demo_sk = c.c_current_cdemo_sk and + exists (select * + from store_sales,date_dim + where c.c_customer_sk = ss_customer_sk and + ss_sold_date_sk = d_date_sk and + d_year = 1999 and + d_qoy < 4) and + (exists (select * + from web_sales,date_dim + where c.c_customer_sk = ws_bill_customer_sk and + ws_sold_date_sk = d_date_sk and + d_year = 1999 and + d_qoy < 4) or + exists (select * + from catalog_sales,date_dim + where c.c_customer_sk = cs_ship_customer_sk and + cs_sold_date_sk = d_date_sk and + d_year = 1999 and + d_qoy < 4)) + group by ca_state, + cd_gender, + cd_marital_status, + cd_dep_count, + cd_dep_employed_count, + cd_dep_college_count + order by ca_state, + cd_gender, + cd_marital_status, + cd_dep_count, + cd_dep_employed_count, + cd_dep_college_count + limit 100; + + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query35.sql.out b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query35.sql.out new file mode 100644 index 00000000000..578d6023b20 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query35.sql.out @@ -0,0 +1,61 @@ +-- Automatically generated by Gravitino Spark SQL test +-- !query +select + ca_state, + cd_gender, + cd_marital_status, + cd_dep_count, + count(*) cnt1, + avg(cd_dep_count), + max(cd_dep_count), + sum(cd_dep_count), + cd_dep_employed_count, + count(*) cnt2, + avg(cd_dep_employed_count), + max(cd_dep_employed_count), + sum(cd_dep_employed_count), + cd_dep_college_count, + count(*) cnt3, + avg(cd_dep_college_count), + max(cd_dep_college_count), + sum(cd_dep_college_count) + from + customer c,customer_address ca,customer_demographics + where + c.c_current_addr_sk = ca.ca_address_sk and + cd_demo_sk = c.c_current_cdemo_sk and + exists (select * + from store_sales,date_dim + where c.c_customer_sk = ss_customer_sk and + ss_sold_date_sk = d_date_sk and + d_year = 1999 and + d_qoy < 4) and + (exists (select * + from web_sales,date_dim + where c.c_customer_sk = ws_bill_customer_sk and + ws_sold_date_sk = d_date_sk and + d_year = 1999 and + d_qoy < 4) or + exists (select * + from catalog_sales,date_dim + where c.c_customer_sk = cs_ship_customer_sk and + cs_sold_date_sk = d_date_sk and + d_year = 1999 and + d_qoy < 4)) + group by ca_state, + cd_gender, + cd_marital_status, + cd_dep_count, + cd_dep_employed_count, + cd_dep_college_count + order by ca_state, + cd_gender, + cd_marital_status, + cd_dep_count, + cd_dep_employed_count, + cd_dep_college_count + limit 100 +-- !query schema +struct +-- !query output + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query36.sql b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query36.sql new file mode 100644 index 00000000000..f800b61010d --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query36.sql @@ -0,0 +1,30 @@ +-- start query 1 in stream 0 using template query36.tpl +select + sum(ss_net_profit)/sum(ss_ext_sales_price) as gross_margin + ,i_category + ,i_class + ,grouping(i_category)+grouping(i_class) as lochierarchy + ,rank() over ( + partition by grouping(i_category)+grouping(i_class), + case when grouping(i_class) = 0 then i_category end + order by sum(ss_net_profit)/sum(ss_ext_sales_price) asc) as rank_within_parent + from + store_sales + ,date_dim d1 + ,item + ,store + where + d1.d_year = 2000 + and d1.d_date_sk = ss_sold_date_sk + and i_item_sk = ss_item_sk + and s_store_sk = ss_store_sk + and s_state in ('TN','TN','TN','TN', + 'TN','TN','TN','TN') + group by rollup(i_category,i_class) + order by + lochierarchy desc + ,case when lochierarchy = 0 then i_category end + ,rank_within_parent + limit 100; + + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query36.sql.out b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query36.sql.out new file mode 100644 index 00000000000..5fff5a691ab --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query36.sql.out @@ -0,0 +1,132 @@ +-- Automatically generated by Gravitino Spark SQL test +-- !query +select + sum(ss_net_profit)/sum(ss_ext_sales_price) as gross_margin + ,i_category + ,i_class + ,grouping(i_category)+grouping(i_class) as lochierarchy + ,rank() over ( + partition by grouping(i_category)+grouping(i_class), + case when grouping(i_class) = 0 then i_category end + order by sum(ss_net_profit)/sum(ss_ext_sales_price) asc) as rank_within_parent + from + store_sales + ,date_dim d1 + ,item + ,store + where + d1.d_year = 2000 + and d1.d_date_sk = ss_sold_date_sk + and i_item_sk = ss_item_sk + and s_store_sk = ss_store_sk + and s_state in ('TN','TN','TN','TN', + 'TN','TN','TN','TN') + group by rollup(i_category,i_class) + order by + lochierarchy desc + ,case when lochierarchy = 0 then i_category end + ,rank_within_parent + limit 100 +-- !query schema +struct +-- !query output +-0.01264890089647550043 NULL womens 0 7 +-0.14523606100368196121 Jewelry bracelets 0 17 +-0.15468042459411738024 Books history 0 16 +-0.19922872916269436604 Home decor 0 16 +-0.20509588151314650255 Electronics automotive 0 16 +-0.20961256896707670043 Electronics personal 0 15 +-0.21124334553840655439 Jewelry NULL 0 16 +-0.22296627303554283095 Music country 0 5 +-0.24979851742209881318 Jewelry consignment 0 15 +-0.28077916660781011241 Jewelry pendants 0 14 +-0.29409295349299832109 Books self-help 0 15 +-0.29843520560827983677 Jewelry diamonds 0 13 +-0.30610180459628915193 Electronics cameras 0 14 +-0.31418290010654765495 Jewelry mens watch 0 12 +-0.31720325593592501797 Electronics portable 0 13 +-0.32892350296307104388 Books entertainments 0 14 +-0.32959891609064994329 Men accessories 0 4 +-0.32972697043181863085 Home blinds/shades 0 15 +-0.34221239672768133594 Electronics musical 0 12 +-0.34597949352909043165 Electronics dvd/vcr players 0 11 +-0.37419911325064966488 NULL swimwear 0 6 +-0.37475425504196329570 Jewelry gold 0 11 +-0.39109171644125462424 Music classical 0 4 +-0.40326612662221840888 Books computers 0 13 +-0.40480193226844251260 Shoes NULL 1 11 +-0.41138706773543938330 Electronics monitors 0 10 +-0.41666483173694447503 NULL pants 0 5 +-0.41757113617162301563 Home bathroom 0 14 +-0.41969780847490173868 Jewelry womens watch 0 10 +-0.42258396854328847072 Books sports 0 12 +-0.42316068073131869439 Jewelry NULL 1 10 +-0.42414783886016217292 Electronics scanners 0 9 +-0.43097427860057779911 Music NULL 1 9 +-0.43492834754782185132 Children newborn 0 5 +-0.43510897640832228741 Sports NULL 1 8 +-0.43612836597793282096 Women NULL 1 7 +-0.43668550718121567423 Jewelry costume 0 9 +-0.44446953144880361768 Electronics NULL 1 6 +-0.44462871878010471784 Jewelry birdal 0 8 +-0.44505111959093001255 Electronics memory 0 8 +-0.44933010114058461228 Children toddlers 0 4 +-0.45101464723690427204 NULL NULL 2 1 +-0.45149436669451268618 Jewelry custom 0 7 +-0.45323070910017614607 Home flatware 0 13 +-0.45367725990317559772 Men sports-apparel 0 3 +-0.45570404601501210727 Children infants 0 3 +-0.45651582293848992218 Men NULL 1 5 +-0.45765445417948459366 Home mattresses 0 12 +-0.46066780150548789546 Books reference 0 11 +-0.46711803886179619787 Books romance 0 10 +-0.46740975615487146408 Children NULL 1 4 +-0.46969072125081554150 Electronics disk drives 0 7 +-0.47751096631281680702 Books fiction 0 9 +-0.48332753923216397045 Jewelry estate 0 6 +-0.48842324205437387131 Home tables 0 11 +-0.49167764685197078754 Home bedding 0 10 +-0.49285388516561858927 Home glassware 0 9 +-0.49555383130157789240 Books NULL 1 3 +-0.49869573079346629628 Home wallpaper 0 8 +-0.50195137767665656747 Men shirts 0 2 +-0.51317329165475285461 Electronics televisions 0 6 +-0.52019139602714705343 Home kids 0 7 +-0.52106096865738790648 Home NULL 1 2 +-0.52770274883503683467 Jewelry semi-precious 0 5 +-0.53235016071729528852 Books science 0 8 +-0.53511600747154751061 Children school-uniforms 0 2 +-0.55154735910786286770 Men pants 0 1 +-0.55783152709608756777 Jewelry rings 0 4 +-0.56737322051406426087 Electronics camcorders 0 5 +-0.57535402342712764700 Home lighting 0 6 +-0.58168716434507000495 Home paint 0 5 +-0.58466277918362856200 Electronics karoke 0 4 +-0.58564666572258587170 Books business 0 7 +-0.59280803931608928631 Books parenting 0 6 +-0.59318562327227282298 Jewelry earings 0 3 +-0.60658776473563157155 Music pop 0 3 +-0.61126644298676456352 Books cooking 0 5 +-0.62155832402662009757 Electronics stereo 0 3 +-0.64455401270068720446 Music rock 0 2 +-0.66464647062666815241 Home rugs 0 4 +-0.66656514981351748817 Jewelry jewelry boxes 0 2 +-0.68017453721377526085 NULL NULL 1 1 +-0.68052038217837664421 Home accent 0 3 +-0.69197103383790909707 Jewelry loose stones 0 1 +-0.71185905373555462183 Electronics audio 0 2 +-0.73596178710523167609 Books arts 0 4 +-0.76004592935058719746 Home curtains/drapes 0 2 +-0.77347099531874610770 Home furniture 0 1 +-0.79660923336595668532 NULL dresses 0 4 +-0.82550264053599949882 Books mystery 0 3 +-0.87204387886748408540 Books home repair 0 2 +-0.94648648940741630049 Books travel 0 1 +-1.00000000000000000000 NULL sports-apparel 0 3 +-1.05392921654433740149 Electronics wireless 0 1 +-1.06996587030716723549 Children NULL 0 1 +-1.10725815609067949947 NULL NULL 0 2 +-1.48723744418104636419 Music NULL 0 1 +-2.10482230315207836243 NULL archery 0 1 +0.02928949357520786092 NULL glassware 0 8 +0.15814715171365071938 NULL outdoor 0 9 diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query37.sql b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query37.sql new file mode 100644 index 00000000000..70babd8617c --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query37.sql @@ -0,0 +1,17 @@ +-- start query 1 in stream 0 using template query37.tpl +select i_item_id + ,i_item_desc + ,i_current_price + from item, inventory, date_dim, catalog_sales + where i_current_price between 22 and 22 + 30 + and inv_item_sk = i_item_sk + and d_date_sk=inv_date_sk + and d_date between cast('2001-06-02' as date) and (cast('2001-06-02' as date) + interval 60 days) + and i_manufact_id in (678,964,918,849) + and inv_quantity_on_hand between 100 and 500 + and cs_item_sk = i_item_sk + group by i_item_id,i_item_desc,i_current_price + order by i_item_id + limit 100; + + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query37.sql.out b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query37.sql.out new file mode 100644 index 00000000000..7df43cc9891 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query37.sql.out @@ -0,0 +1,20 @@ +-- Automatically generated by Gravitino Spark SQL test +-- !query +select i_item_id + ,i_item_desc + ,i_current_price + from item, inventory, date_dim, catalog_sales + where i_current_price between 22 and 22 + 30 + and inv_item_sk = i_item_sk + and d_date_sk=inv_date_sk + and d_date between cast('2001-06-02' as date) and (cast('2001-06-02' as date) + interval 60 days) + and i_manufact_id in (678,964,918,849) + and inv_quantity_on_hand between 100 and 500 + and cs_item_sk = i_item_sk + group by i_item_id,i_item_desc,i_current_price + order by i_item_id + limit 100 +-- !query schema +struct +-- !query output + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query38.sql b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query38.sql new file mode 100644 index 00000000000..1a95eb75935 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query38.sql @@ -0,0 +1,23 @@ +-- start query 1 in stream 0 using template query38.tpl +select count(*) from ( + select distinct c_last_name, c_first_name, d_date + from store_sales, date_dim, customer + where store_sales.ss_sold_date_sk = date_dim.d_date_sk + and store_sales.ss_customer_sk = customer.c_customer_sk + and d_month_seq between 1212 and 1212 + 11 + intersect + select distinct c_last_name, c_first_name, d_date + from catalog_sales, date_dim, customer + where catalog_sales.cs_sold_date_sk = date_dim.d_date_sk + and catalog_sales.cs_bill_customer_sk = customer.c_customer_sk + and d_month_seq between 1212 and 1212 + 11 + intersect + select distinct c_last_name, c_first_name, d_date + from web_sales, date_dim, customer + where web_sales.ws_sold_date_sk = date_dim.d_date_sk + and web_sales.ws_bill_customer_sk = customer.c_customer_sk + and d_month_seq between 1212 and 1212 + 11 +) hot_cust +limit 100; + + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query38.sql.out b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query38.sql.out new file mode 100644 index 00000000000..78f0df17d36 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query38.sql.out @@ -0,0 +1,26 @@ +-- Automatically generated by Gravitino Spark SQL test +-- !query +select count(*) from ( + select distinct c_last_name, c_first_name, d_date + from store_sales, date_dim, customer + where store_sales.ss_sold_date_sk = date_dim.d_date_sk + and store_sales.ss_customer_sk = customer.c_customer_sk + and d_month_seq between 1212 and 1212 + 11 + intersect + select distinct c_last_name, c_first_name, d_date + from catalog_sales, date_dim, customer + where catalog_sales.cs_sold_date_sk = date_dim.d_date_sk + and catalog_sales.cs_bill_customer_sk = customer.c_customer_sk + and d_month_seq between 1212 and 1212 + 11 + intersect + select distinct c_last_name, c_first_name, d_date + from web_sales, date_dim, customer + where web_sales.ws_sold_date_sk = date_dim.d_date_sk + and web_sales.ws_bill_customer_sk = customer.c_customer_sk + and d_month_seq between 1212 and 1212 + 11 +) hot_cust +limit 100 +-- !query schema +struct +-- !query output +0 diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query39.sql b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query39.sql new file mode 100644 index 00000000000..999401f3d39 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query39.sql @@ -0,0 +1,54 @@ +-- start query 1 in stream 0 using template query39.tpl +with inv as +(select w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy + ,stdev,mean, case mean when 0 then null else stdev/mean end cov + from(select w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy + ,stddev_samp(inv_quantity_on_hand) stdev,avg(inv_quantity_on_hand) mean + from inventory + ,item + ,warehouse + ,date_dim + where inv_item_sk = i_item_sk + and inv_warehouse_sk = w_warehouse_sk + and inv_date_sk = d_date_sk + and d_year =1998 + group by w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy) foo + where case mean when 0 then 0 else stdev/mean end > 1) +select inv1.w_warehouse_sk,inv1.i_item_sk,inv1.d_moy,inv1.mean, inv1.cov + ,inv2.w_warehouse_sk,inv2.i_item_sk,inv2.d_moy,inv2.mean, inv2.cov +from inv inv1,inv inv2 +where inv1.i_item_sk = inv2.i_item_sk + and inv1.w_warehouse_sk = inv2.w_warehouse_sk + and inv1.d_moy=4 + and inv2.d_moy=4+1 +order by inv1.w_warehouse_sk,inv1.i_item_sk,inv1.d_moy,inv1.mean,inv1.cov + ,inv2.d_moy,inv2.mean, inv2.cov +; +with inv as +(select w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy + ,stdev,mean, case mean when 0 then null else stdev/mean end cov + from(select w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy + ,stddev_samp(inv_quantity_on_hand) stdev,avg(inv_quantity_on_hand) mean + from inventory + ,item + ,warehouse + ,date_dim + where inv_item_sk = i_item_sk + and inv_warehouse_sk = w_warehouse_sk + and inv_date_sk = d_date_sk + and d_year =1998 + group by w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy) foo + where case mean when 0 then 0 else stdev/mean end > 1) +select inv1.w_warehouse_sk,inv1.i_item_sk,inv1.d_moy,inv1.mean, inv1.cov + ,inv2.w_warehouse_sk,inv2.i_item_sk,inv2.d_moy,inv2.mean, inv2.cov +from inv inv1,inv inv2 +where inv1.i_item_sk = inv2.i_item_sk + and inv1.w_warehouse_sk = inv2.w_warehouse_sk + and inv1.d_moy=4 + and inv2.d_moy=4+1 + and inv1.cov > 1.5 +order by inv1.w_warehouse_sk,inv1.i_item_sk,inv1.d_moy,inv1.mean,inv1.cov + ,inv2.d_moy,inv2.mean, inv2.cov +; + + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query39.sql.out b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query39.sql.out new file mode 100644 index 00000000000..0b6df0105ad --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query39.sql.out @@ -0,0 +1,62 @@ +-- Automatically generated by Gravitino Spark SQL test +-- !query +with inv as +(select w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy + ,stdev,mean, case mean when 0 then null else stdev/mean end cov + from(select w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy + ,stddev_samp(inv_quantity_on_hand) stdev,avg(inv_quantity_on_hand) mean + from inventory + ,item + ,warehouse + ,date_dim + where inv_item_sk = i_item_sk + and inv_warehouse_sk = w_warehouse_sk + and inv_date_sk = d_date_sk + and d_year =1998 + group by w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy) foo + where case mean when 0 then 0 else stdev/mean end > 1) +select inv1.w_warehouse_sk,inv1.i_item_sk,inv1.d_moy,inv1.mean, inv1.cov + ,inv2.w_warehouse_sk,inv2.i_item_sk,inv2.d_moy,inv2.mean, inv2.cov +from inv inv1,inv inv2 +where inv1.i_item_sk = inv2.i_item_sk + and inv1.w_warehouse_sk = inv2.w_warehouse_sk + and inv1.d_moy=4 + and inv2.d_moy=4+1 +order by inv1.w_warehouse_sk,inv1.i_item_sk,inv1.d_moy,inv1.mean,inv1.cov + ,inv2.d_moy,inv2.mean, inv2.cov +-- !query schema +struct +-- !query output + + + +-- !query +with inv as +(select w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy + ,stdev,mean, case mean when 0 then null else stdev/mean end cov + from(select w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy + ,stddev_samp(inv_quantity_on_hand) stdev,avg(inv_quantity_on_hand) mean + from inventory + ,item + ,warehouse + ,date_dim + where inv_item_sk = i_item_sk + and inv_warehouse_sk = w_warehouse_sk + and inv_date_sk = d_date_sk + and d_year =1998 + group by w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy) foo + where case mean when 0 then 0 else stdev/mean end > 1) +select inv1.w_warehouse_sk,inv1.i_item_sk,inv1.d_moy,inv1.mean, inv1.cov + ,inv2.w_warehouse_sk,inv2.i_item_sk,inv2.d_moy,inv2.mean, inv2.cov +from inv inv1,inv inv2 +where inv1.i_item_sk = inv2.i_item_sk + and inv1.w_warehouse_sk = inv2.w_warehouse_sk + and inv1.d_moy=4 + and inv2.d_moy=4+1 + and inv1.cov > 1.5 +order by inv1.w_warehouse_sk,inv1.i_item_sk,inv1.d_moy,inv1.mean,inv1.cov + ,inv2.d_moy,inv2.mean, inv2.cov +-- !query schema +struct +-- !query output + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query4.sql b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query4.sql new file mode 100644 index 00000000000..b445386d21d --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query4.sql @@ -0,0 +1,116 @@ +-- start query 1 in stream 0 using template query4.tpl +with year_total as ( + select c_customer_id customer_id + ,c_first_name customer_first_name + ,c_last_name customer_last_name + ,c_preferred_cust_flag customer_preferred_cust_flag + ,c_birth_country customer_birth_country + ,c_login customer_login + ,c_email_address customer_email_address + ,d_year dyear + ,sum(((ss_ext_list_price-ss_ext_wholesale_cost-ss_ext_discount_amt)+ss_ext_sales_price)/2) year_total + ,'s' sale_type + from customer + ,store_sales + ,date_dim + where c_customer_sk = ss_customer_sk + and ss_sold_date_sk = d_date_sk + group by c_customer_id + ,c_first_name + ,c_last_name + ,c_preferred_cust_flag + ,c_birth_country + ,c_login + ,c_email_address + ,d_year + union all + select c_customer_id customer_id + ,c_first_name customer_first_name + ,c_last_name customer_last_name + ,c_preferred_cust_flag customer_preferred_cust_flag + ,c_birth_country customer_birth_country + ,c_login customer_login + ,c_email_address customer_email_address + ,d_year dyear + ,sum((((cs_ext_list_price-cs_ext_wholesale_cost-cs_ext_discount_amt)+cs_ext_sales_price)/2) ) year_total + ,'c' sale_type + from customer + ,catalog_sales + ,date_dim + where c_customer_sk = cs_bill_customer_sk + and cs_sold_date_sk = d_date_sk + group by c_customer_id + ,c_first_name + ,c_last_name + ,c_preferred_cust_flag + ,c_birth_country + ,c_login + ,c_email_address + ,d_year +union all + select c_customer_id customer_id + ,c_first_name customer_first_name + ,c_last_name customer_last_name + ,c_preferred_cust_flag customer_preferred_cust_flag + ,c_birth_country customer_birth_country + ,c_login customer_login + ,c_email_address customer_email_address + ,d_year dyear + ,sum((((ws_ext_list_price-ws_ext_wholesale_cost-ws_ext_discount_amt)+ws_ext_sales_price)/2) ) year_total + ,'w' sale_type + from customer + ,web_sales + ,date_dim + where c_customer_sk = ws_bill_customer_sk + and ws_sold_date_sk = d_date_sk + group by c_customer_id + ,c_first_name + ,c_last_name + ,c_preferred_cust_flag + ,c_birth_country + ,c_login + ,c_email_address + ,d_year + ) + select + t_s_secyear.customer_id + ,t_s_secyear.customer_first_name + ,t_s_secyear.customer_last_name + ,t_s_secyear.customer_email_address + from year_total t_s_firstyear + ,year_total t_s_secyear + ,year_total t_c_firstyear + ,year_total t_c_secyear + ,year_total t_w_firstyear + ,year_total t_w_secyear + where t_s_secyear.customer_id = t_s_firstyear.customer_id + and t_s_firstyear.customer_id = t_c_secyear.customer_id + and t_s_firstyear.customer_id = t_c_firstyear.customer_id + and t_s_firstyear.customer_id = t_w_firstyear.customer_id + and t_s_firstyear.customer_id = t_w_secyear.customer_id + and t_s_firstyear.sale_type = 's' + and t_c_firstyear.sale_type = 'c' + and t_w_firstyear.sale_type = 'w' + and t_s_secyear.sale_type = 's' + and t_c_secyear.sale_type = 'c' + and t_w_secyear.sale_type = 'w' + and t_s_firstyear.dyear = 2001 + and t_s_secyear.dyear = 2001+1 + and t_c_firstyear.dyear = 2001 + and t_c_secyear.dyear = 2001+1 + and t_w_firstyear.dyear = 2001 + and t_w_secyear.dyear = 2001+1 + and t_s_firstyear.year_total > 0 + and t_c_firstyear.year_total > 0 + and t_w_firstyear.year_total > 0 + and case when t_c_firstyear.year_total > 0 then t_c_secyear.year_total / t_c_firstyear.year_total else null end + > case when t_s_firstyear.year_total > 0 then t_s_secyear.year_total / t_s_firstyear.year_total else null end + and case when t_c_firstyear.year_total > 0 then t_c_secyear.year_total / t_c_firstyear.year_total else null end + > case when t_w_firstyear.year_total > 0 then t_w_secyear.year_total / t_w_firstyear.year_total else null end + order by t_s_secyear.customer_id + ,t_s_secyear.customer_first_name + ,t_s_secyear.customer_last_name + ,t_s_secyear.customer_email_address +limit 100; + + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query4.sql.out b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query4.sql.out new file mode 100644 index 00000000000..16d3c05ce12 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query4.sql.out @@ -0,0 +1,119 @@ +-- Automatically generated by Gravitino Spark SQL test +-- !query +with year_total as ( + select c_customer_id customer_id + ,c_first_name customer_first_name + ,c_last_name customer_last_name + ,c_preferred_cust_flag customer_preferred_cust_flag + ,c_birth_country customer_birth_country + ,c_login customer_login + ,c_email_address customer_email_address + ,d_year dyear + ,sum(((ss_ext_list_price-ss_ext_wholesale_cost-ss_ext_discount_amt)+ss_ext_sales_price)/2) year_total + ,'s' sale_type + from customer + ,store_sales + ,date_dim + where c_customer_sk = ss_customer_sk + and ss_sold_date_sk = d_date_sk + group by c_customer_id + ,c_first_name + ,c_last_name + ,c_preferred_cust_flag + ,c_birth_country + ,c_login + ,c_email_address + ,d_year + union all + select c_customer_id customer_id + ,c_first_name customer_first_name + ,c_last_name customer_last_name + ,c_preferred_cust_flag customer_preferred_cust_flag + ,c_birth_country customer_birth_country + ,c_login customer_login + ,c_email_address customer_email_address + ,d_year dyear + ,sum((((cs_ext_list_price-cs_ext_wholesale_cost-cs_ext_discount_amt)+cs_ext_sales_price)/2) ) year_total + ,'c' sale_type + from customer + ,catalog_sales + ,date_dim + where c_customer_sk = cs_bill_customer_sk + and cs_sold_date_sk = d_date_sk + group by c_customer_id + ,c_first_name + ,c_last_name + ,c_preferred_cust_flag + ,c_birth_country + ,c_login + ,c_email_address + ,d_year +union all + select c_customer_id customer_id + ,c_first_name customer_first_name + ,c_last_name customer_last_name + ,c_preferred_cust_flag customer_preferred_cust_flag + ,c_birth_country customer_birth_country + ,c_login customer_login + ,c_email_address customer_email_address + ,d_year dyear + ,sum((((ws_ext_list_price-ws_ext_wholesale_cost-ws_ext_discount_amt)+ws_ext_sales_price)/2) ) year_total + ,'w' sale_type + from customer + ,web_sales + ,date_dim + where c_customer_sk = ws_bill_customer_sk + and ws_sold_date_sk = d_date_sk + group by c_customer_id + ,c_first_name + ,c_last_name + ,c_preferred_cust_flag + ,c_birth_country + ,c_login + ,c_email_address + ,d_year + ) + select + t_s_secyear.customer_id + ,t_s_secyear.customer_first_name + ,t_s_secyear.customer_last_name + ,t_s_secyear.customer_email_address + from year_total t_s_firstyear + ,year_total t_s_secyear + ,year_total t_c_firstyear + ,year_total t_c_secyear + ,year_total t_w_firstyear + ,year_total t_w_secyear + where t_s_secyear.customer_id = t_s_firstyear.customer_id + and t_s_firstyear.customer_id = t_c_secyear.customer_id + and t_s_firstyear.customer_id = t_c_firstyear.customer_id + and t_s_firstyear.customer_id = t_w_firstyear.customer_id + and t_s_firstyear.customer_id = t_w_secyear.customer_id + and t_s_firstyear.sale_type = 's' + and t_c_firstyear.sale_type = 'c' + and t_w_firstyear.sale_type = 'w' + and t_s_secyear.sale_type = 's' + and t_c_secyear.sale_type = 'c' + and t_w_secyear.sale_type = 'w' + and t_s_firstyear.dyear = 2001 + and t_s_secyear.dyear = 2001+1 + and t_c_firstyear.dyear = 2001 + and t_c_secyear.dyear = 2001+1 + and t_w_firstyear.dyear = 2001 + and t_w_secyear.dyear = 2001+1 + and t_s_firstyear.year_total > 0 + and t_c_firstyear.year_total > 0 + and t_w_firstyear.year_total > 0 + and case when t_c_firstyear.year_total > 0 then t_c_secyear.year_total / t_c_firstyear.year_total else null end + > case when t_s_firstyear.year_total > 0 then t_s_secyear.year_total / t_s_firstyear.year_total else null end + and case when t_c_firstyear.year_total > 0 then t_c_secyear.year_total / t_c_firstyear.year_total else null end + > case when t_w_firstyear.year_total > 0 then t_w_secyear.year_total / t_w_firstyear.year_total else null end + order by t_s_secyear.customer_id + ,t_s_secyear.customer_first_name + ,t_s_secyear.customer_last_name + ,t_s_secyear.customer_email_address +limit 100 +-- !query schema +struct +-- !query output + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query40.sql b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query40.sql new file mode 100644 index 00000000000..5603a1171db --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query40.sql @@ -0,0 +1,28 @@ +-- start query 1 in stream 0 using template query40.tpl +select + w_state + ,i_item_id + ,sum(case when (cast(d_date as date) < cast ('1998-04-08' as date)) + then cs_sales_price - coalesce(cr_refunded_cash,0) else 0 end) as sales_before + ,sum(case when (cast(d_date as date) >= cast ('1998-04-08' as date)) + then cs_sales_price - coalesce(cr_refunded_cash,0) else 0 end) as sales_after + from + catalog_sales left outer join catalog_returns on + (cs_order_number = cr_order_number + and cs_item_sk = cr_item_sk) + ,warehouse + ,item + ,date_dim + where + i_current_price between 0.99 and 1.49 + and i_item_sk = cs_item_sk + and cs_warehouse_sk = w_warehouse_sk + and cs_sold_date_sk = d_date_sk + and d_date between (cast ('1998-04-08' as date) - interval 30 days) + and (cast ('1998-04-08' as date) + interval 30 days) + group by + w_state,i_item_id + order by w_state,i_item_id +limit 100; + + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query40.sql.out b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query40.sql.out new file mode 100644 index 00000000000..c77a6d7bc86 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query40.sql.out @@ -0,0 +1,31 @@ +-- Automatically generated by Gravitino Spark SQL test +-- !query +select + w_state + ,i_item_id + ,sum(case when (cast(d_date as date) < cast ('1998-04-08' as date)) + then cs_sales_price - coalesce(cr_refunded_cash,0) else 0 end) as sales_before + ,sum(case when (cast(d_date as date) >= cast ('1998-04-08' as date)) + then cs_sales_price - coalesce(cr_refunded_cash,0) else 0 end) as sales_after + from + catalog_sales left outer join catalog_returns on + (cs_order_number = cr_order_number + and cs_item_sk = cr_item_sk) + ,warehouse + ,item + ,date_dim + where + i_current_price between 0.99 and 1.49 + and i_item_sk = cs_item_sk + and cs_warehouse_sk = w_warehouse_sk + and cs_sold_date_sk = d_date_sk + and d_date between (cast ('1998-04-08' as date) - interval 30 days) + and (cast ('1998-04-08' as date) + interval 30 days) + group by + w_state,i_item_id + order by w_state,i_item_id +limit 100 +-- !query schema +struct +-- !query output + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query41.sql b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query41.sql new file mode 100644 index 00000000000..3cd7b93efdd --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query41.sql @@ -0,0 +1,52 @@ +-- start query 1 in stream 0 using template query41.tpl +select distinct(i_product_name) + from item i1 + where i_manufact_id between 742 and 742+40 + and (select count(*) as item_cnt + from item + where (i_manufact = i1.i_manufact and + ((i_category = 'Women' and + (i_color = 'orchid' or i_color = 'papaya') and + (i_units = 'Pound' or i_units = 'Lb') and + (i_size = 'petite' or i_size = 'medium') + ) or + (i_category = 'Women' and + (i_color = 'burlywood' or i_color = 'navy') and + (i_units = 'Bundle' or i_units = 'Each') and + (i_size = 'N/A' or i_size = 'extra large') + ) or + (i_category = 'Men' and + (i_color = 'bisque' or i_color = 'azure') and + (i_units = 'N/A' or i_units = 'Tsp') and + (i_size = 'small' or i_size = 'large') + ) or + (i_category = 'Men' and + (i_color = 'chocolate' or i_color = 'cornflower') and + (i_units = 'Bunch' or i_units = 'Gross') and + (i_size = 'petite' or i_size = 'medium') + ))) or + (i_manufact = i1.i_manufact and + ((i_category = 'Women' and + (i_color = 'salmon' or i_color = 'midnight') and + (i_units = 'Oz' or i_units = 'Box') and + (i_size = 'petite' or i_size = 'medium') + ) or + (i_category = 'Women' and + (i_color = 'snow' or i_color = 'steel') and + (i_units = 'Carton' or i_units = 'Tbl') and + (i_size = 'N/A' or i_size = 'extra large') + ) or + (i_category = 'Men' and + (i_color = 'purple' or i_color = 'gainsboro') and + (i_units = 'Dram' or i_units = 'Unknown') and + (i_size = 'small' or i_size = 'large') + ) or + (i_category = 'Men' and + (i_color = 'metallic' or i_color = 'forest') and + (i_units = 'Gram' or i_units = 'Ounce') and + (i_size = 'petite' or i_size = 'medium') + )))) > 0 + order by i_product_name + limit 100; + + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query41.sql.out b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query41.sql.out new file mode 100644 index 00000000000..f9e58268d79 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query41.sql.out @@ -0,0 +1,55 @@ +-- Automatically generated by Gravitino Spark SQL test +-- !query +select distinct(i_product_name) + from item i1 + where i_manufact_id between 742 and 742+40 + and (select count(*) as item_cnt + from item + where (i_manufact = i1.i_manufact and + ((i_category = 'Women' and + (i_color = 'orchid' or i_color = 'papaya') and + (i_units = 'Pound' or i_units = 'Lb') and + (i_size = 'petite' or i_size = 'medium') + ) or + (i_category = 'Women' and + (i_color = 'burlywood' or i_color = 'navy') and + (i_units = 'Bundle' or i_units = 'Each') and + (i_size = 'N/A' or i_size = 'extra large') + ) or + (i_category = 'Men' and + (i_color = 'bisque' or i_color = 'azure') and + (i_units = 'N/A' or i_units = 'Tsp') and + (i_size = 'small' or i_size = 'large') + ) or + (i_category = 'Men' and + (i_color = 'chocolate' or i_color = 'cornflower') and + (i_units = 'Bunch' or i_units = 'Gross') and + (i_size = 'petite' or i_size = 'medium') + ))) or + (i_manufact = i1.i_manufact and + ((i_category = 'Women' and + (i_color = 'salmon' or i_color = 'midnight') and + (i_units = 'Oz' or i_units = 'Box') and + (i_size = 'petite' or i_size = 'medium') + ) or + (i_category = 'Women' and + (i_color = 'snow' or i_color = 'steel') and + (i_units = 'Carton' or i_units = 'Tbl') and + (i_size = 'N/A' or i_size = 'extra large') + ) or + (i_category = 'Men' and + (i_color = 'purple' or i_color = 'gainsboro') and + (i_units = 'Dram' or i_units = 'Unknown') and + (i_size = 'small' or i_size = 'large') + ) or + (i_category = 'Men' and + (i_color = 'metallic' or i_color = 'forest') and + (i_units = 'Gram' or i_units = 'Ounce') and + (i_size = 'petite' or i_size = 'medium') + )))) > 0 + order by i_product_name + limit 100 +-- !query schema +struct +-- !query output + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query42.sql b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query42.sql new file mode 100644 index 00000000000..3c48d4c73c2 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query42.sql @@ -0,0 +1,22 @@ +-- start query 1 in stream 0 using template query42.tpl +select dt.d_year + ,item.i_category_id + ,item.i_category + ,sum(ss_ext_sales_price) + from date_dim dt + ,store_sales + ,item + where dt.d_date_sk = store_sales.ss_sold_date_sk + and store_sales.ss_item_sk = item.i_item_sk + and item.i_manager_id = 1 + and dt.d_moy=12 + and dt.d_year=1998 + group by dt.d_year + ,item.i_category_id + ,item.i_category + order by sum(ss_ext_sales_price) desc,dt.d_year + ,item.i_category_id + ,item.i_category +limit 100 ; + + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query42.sql.out b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query42.sql.out new file mode 100644 index 00000000000..6c28a1020ba --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query42.sql.out @@ -0,0 +1,33 @@ +-- Automatically generated by Gravitino Spark SQL test +-- !query +select dt.d_year + ,item.i_category_id + ,item.i_category + ,sum(ss_ext_sales_price) + from date_dim dt + ,store_sales + ,item + where dt.d_date_sk = store_sales.ss_sold_date_sk + and store_sales.ss_item_sk = item.i_item_sk + and item.i_manager_id = 1 + and dt.d_moy=12 + and dt.d_year=1998 + group by dt.d_year + ,item.i_category_id + ,item.i_category + order by sum(ss_ext_sales_price) desc,dt.d_year + ,item.i_category_id + ,item.i_category +limit 100 +-- !query schema +struct +-- !query output +1998 1 Women 4067.90 +1998 10 Electronics 1156.36 +1998 2 Men 240.00 +1998 3 Children 1610.70 +1998 4 Shoes 4184.94 +1998 5 Music 7662.68 +1998 6 Jewelry 7850.43 +1998 7 Home 404.60 +1998 9 Books 15387.57 diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query43.sql b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query43.sql new file mode 100644 index 00000000000..1cc9589e867 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query43.sql @@ -0,0 +1,19 @@ +-- start query 1 in stream 0 using template query43.tpl +select s_store_name, s_store_id, + sum(case when (d_day_name='Sunday') then ss_sales_price else null end) sun_sales, + sum(case when (d_day_name='Monday') then ss_sales_price else null end) mon_sales, + sum(case when (d_day_name='Tuesday') then ss_sales_price else null end) tue_sales, + sum(case when (d_day_name='Wednesday') then ss_sales_price else null end) wed_sales, + sum(case when (d_day_name='Thursday') then ss_sales_price else null end) thu_sales, + sum(case when (d_day_name='Friday') then ss_sales_price else null end) fri_sales, + sum(case when (d_day_name='Saturday') then ss_sales_price else null end) sat_sales + from date_dim, store_sales, store + where d_date_sk = ss_sold_date_sk and + s_store_sk = ss_store_sk and + s_gmt_offset = -5 and + d_year = 1998 + group by s_store_name, s_store_id + order by s_store_name, s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales + limit 100; + + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query43.sql.out b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query43.sql.out new file mode 100644 index 00000000000..8b935d8c212 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query43.sql.out @@ -0,0 +1,27 @@ +-- Automatically generated by Gravitino Spark SQL test +-- !query +select s_store_name, s_store_id, + sum(case when (d_day_name='Sunday') then ss_sales_price else null end) sun_sales, + sum(case when (d_day_name='Monday') then ss_sales_price else null end) mon_sales, + sum(case when (d_day_name='Tuesday') then ss_sales_price else null end) tue_sales, + sum(case when (d_day_name='Wednesday') then ss_sales_price else null end) wed_sales, + sum(case when (d_day_name='Thursday') then ss_sales_price else null end) thu_sales, + sum(case when (d_day_name='Friday') then ss_sales_price else null end) fri_sales, + sum(case when (d_day_name='Saturday') then ss_sales_price else null end) sat_sales + from date_dim, store_sales, store + where d_date_sk = ss_sold_date_sk and + s_store_sk = ss_store_sk and + s_gmt_offset = -5 and + d_year = 1998 + group by s_store_name, s_store_id + order by s_store_name, s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales + limit 100 +-- !query schema +struct +-- !query output +able AAAAAAAACAAAAAAA 5538.13 3996.17 2314.98 5765.18 3608.12 2872.24 6585.15 +ation AAAAAAAAHAAAAAAA 6161.54 5668.23 4654.48 6279.76 8439.02 6995.42 4913.20 +bar AAAAAAAAKAAAAAAA 8354.63 3985.82 4348.83 7498.32 3417.42 3613.97 2277.97 +eing AAAAAAAAIAAAAAAA 3441.00 3924.06 5688.32 5055.39 3005.07 3859.57 4952.76 +ese AAAAAAAAEAAAAAAA 7434.69 2306.78 4897.31 3339.66 4087.83 3178.11 6080.70 +ought AAAAAAAABAAAAAAA 6091.13 3114.54 2899.12 4767.96 5687.76 4983.62 6044.06 diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query44.sql b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query44.sql new file mode 100644 index 00000000000..eb94de44927 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query44.sql @@ -0,0 +1,35 @@ +-- start query 1 in stream 0 using template query44.tpl +select asceding.rnk, i1.i_product_name best_performing, i2.i_product_name worst_performing +from(select * + from (select item_sk,rank() over (order by rank_col asc) rnk + from (select ss_item_sk item_sk,avg(ss_net_profit) rank_col + from store_sales ss1 + where ss_store_sk = 2 + group by ss_item_sk + having avg(ss_net_profit) > 0.9*(select avg(ss_net_profit) rank_col + from store_sales + where ss_store_sk = 2 + and ss_hdemo_sk is null + group by ss_store_sk))V1)V11 + where rnk < 11) asceding, + (select * + from (select item_sk,rank() over (order by rank_col desc) rnk + from (select ss_item_sk item_sk,avg(ss_net_profit) rank_col + from store_sales ss1 + where ss_store_sk = 2 + group by ss_item_sk + having avg(ss_net_profit) > 0.9*(select avg(ss_net_profit) rank_col + from store_sales + where ss_store_sk = 2 + and ss_hdemo_sk is null + group by ss_store_sk))V2)V21 + where rnk < 11) descending, +item i1, +item i2 +where asceding.rnk = descending.rnk + and i1.i_item_sk=asceding.item_sk + and i2.i_item_sk=descending.item_sk +order by asceding.rnk +limit 100; + + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query44.sql.out b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query44.sql.out new file mode 100644 index 00000000000..9a1fa04d6dd --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query44.sql.out @@ -0,0 +1,47 @@ +-- Automatically generated by Gravitino Spark SQL test +-- !query +select asceding.rnk, i1.i_product_name best_performing, i2.i_product_name worst_performing +from(select * + from (select item_sk,rank() over (order by rank_col asc) rnk + from (select ss_item_sk item_sk,avg(ss_net_profit) rank_col + from store_sales ss1 + where ss_store_sk = 2 + group by ss_item_sk + having avg(ss_net_profit) > 0.9*(select avg(ss_net_profit) rank_col + from store_sales + where ss_store_sk = 2 + and ss_hdemo_sk is null + group by ss_store_sk))V1)V11 + where rnk < 11) asceding, + (select * + from (select item_sk,rank() over (order by rank_col desc) rnk + from (select ss_item_sk item_sk,avg(ss_net_profit) rank_col + from store_sales ss1 + where ss_store_sk = 2 + group by ss_item_sk + having avg(ss_net_profit) > 0.9*(select avg(ss_net_profit) rank_col + from store_sales + where ss_store_sk = 2 + and ss_hdemo_sk is null + group by ss_store_sk))V2)V21 + where rnk < 11) descending, +item i1, +item i2 +where asceding.rnk = descending.rnk + and i1.i_item_sk=asceding.item_sk + and i2.i_item_sk=descending.item_sk +order by asceding.rnk +limit 100 +-- !query schema +struct +-- !query output +1 antin stprianti callyeingbaranti +10 barbareseese eingeseationeseought +2 n stationeseoughtought eingbarpripriought +3 pribarationn st eseanticallyation +4 antianticallyn st eingationeseantiought +5 prioughteseationought oughtprin steseought +6 prioughteingn st n steseoughtought +7 ationantiableanti ationablebarationought +8 antioughteingeseought callycallyeingableought +9 oughtableanti ationn stn stought diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query45.sql b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query45.sql new file mode 100644 index 00000000000..9d3468541c6 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query45.sql @@ -0,0 +1,20 @@ +-- start query 1 in stream 0 using template query45.tpl +select ca_zip, ca_county, sum(ws_sales_price) + from web_sales, customer, customer_address, date_dim, item + where ws_bill_customer_sk = c_customer_sk + and c_current_addr_sk = ca_address_sk + and ws_item_sk = i_item_sk + and ( substr(ca_zip,1,5) in ('85669', '86197','88274','83405','86475', '85392', '85460', '80348', '81792') + or + i_item_id in (select i_item_id + from item + where i_item_sk in (2, 3, 5, 7, 11, 13, 17, 19, 23, 29) + ) + ) + and ws_sold_date_sk = d_date_sk + and d_qoy = 2 and d_year = 2000 + group by ca_zip, ca_county + order by ca_zip, ca_county + limit 100; + + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query45.sql.out b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query45.sql.out new file mode 100644 index 00000000000..408a0d81fef --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query45.sql.out @@ -0,0 +1,23 @@ +-- Automatically generated by Gravitino Spark SQL test +-- !query +select ca_zip, ca_county, sum(ws_sales_price) + from web_sales, customer, customer_address, date_dim, item + where ws_bill_customer_sk = c_customer_sk + and c_current_addr_sk = ca_address_sk + and ws_item_sk = i_item_sk + and ( substr(ca_zip,1,5) in ('85669', '86197','88274','83405','86475', '85392', '85460', '80348', '81792') + or + i_item_id in (select i_item_id + from item + where i_item_sk in (2, 3, 5, 7, 11, 13, 17, 19, 23, 29) + ) + ) + and ws_sold_date_sk = d_date_sk + and d_qoy = 2 and d_year = 2000 + group by ca_zip, ca_county + order by ca_zip, ca_county + limit 100 +-- !query schema +struct +-- !query output + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query46.sql b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query46.sql new file mode 100644 index 00000000000..adf940b163d --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query46.sql @@ -0,0 +1,35 @@ +-- start query 1 in stream 0 using template query46.tpl +select c_last_name + ,c_first_name + ,ca_city + ,bought_city + ,ss_ticket_number + ,amt,profit + from + (select ss_ticket_number + ,ss_customer_sk + ,ca_city bought_city + ,sum(ss_coupon_amt) amt + ,sum(ss_net_profit) profit + from store_sales,date_dim,store,household_demographics,customer_address + where store_sales.ss_sold_date_sk = date_dim.d_date_sk + and store_sales.ss_store_sk = store.s_store_sk + and store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk + and store_sales.ss_addr_sk = customer_address.ca_address_sk + and (household_demographics.hd_dep_count = 5 or + household_demographics.hd_vehicle_count= 3) + and date_dim.d_dow in (6,0) + and date_dim.d_year in (1999,1999+1,1999+2) + and store.s_city in ('Midway','Fairview','Fairview','Fairview','Fairview') + group by ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city) dn,customer,customer_address current_addr + where ss_customer_sk = c_customer_sk + and customer.c_current_addr_sk = current_addr.ca_address_sk + and current_addr.ca_city <> bought_city + order by c_last_name + ,c_first_name + ,ca_city + ,bought_city + ,ss_ticket_number + limit 100; + + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query46.sql.out b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query46.sql.out new file mode 100644 index 00000000000..687b2494076 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query46.sql.out @@ -0,0 +1,47 @@ +-- Automatically generated by Gravitino Spark SQL test +-- !query +select c_last_name + ,c_first_name + ,ca_city + ,bought_city + ,ss_ticket_number + ,amt,profit + from + (select ss_ticket_number + ,ss_customer_sk + ,ca_city bought_city + ,sum(ss_coupon_amt) amt + ,sum(ss_net_profit) profit + from store_sales,date_dim,store,household_demographics,customer_address + where store_sales.ss_sold_date_sk = date_dim.d_date_sk + and store_sales.ss_store_sk = store.s_store_sk + and store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk + and store_sales.ss_addr_sk = customer_address.ca_address_sk + and (household_demographics.hd_dep_count = 5 or + household_demographics.hd_vehicle_count= 3) + and date_dim.d_dow in (6,0) + and date_dim.d_year in (1999,1999+1,1999+2) + and store.s_city in ('Midway','Fairview','Fairview','Fairview','Fairview') + group by ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city) dn,customer,customer_address current_addr + where ss_customer_sk = c_customer_sk + and customer.c_current_addr_sk = current_addr.ca_address_sk + and current_addr.ca_city <> bought_city + order by c_last_name + ,c_first_name + ,ca_city + ,bought_city + ,ss_ticket_number + limit 100 +-- !query schema +struct +-- !query output +Dawkins Steven Sulphur Springs Lakeside 1081 170.54 -3082.05 +Foster Crystal Mount Pleasant Tipton 995 2601.52 -7543.44 +Francis Charles Saint James Kingston 654 5466.98 -10040.21 +Jackson Wendell Leesville Buena Vista 2371 0.00 -10864.09 +Martin Rosalyn New Hope Bethel 1690 1601.21 -6212.86 +NULL Debbie New Hope Woodlawn 214 1262.89 -6315.68 +Nolan Carrie Riverdale Hamilton 547 237.31 -11848.56 +Smith William Mechanicsburg Delta 104 77.21 -3292.57 +Willis Robert Ashland Clifton 1702 2675.87 -8570.98 +Wyatt Alexander Mountain View Crossroads 176 2720.05 -23501.88 diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query47.sql b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query47.sql new file mode 100644 index 00000000000..e223e343343 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query47.sql @@ -0,0 +1,51 @@ +-- start query 1 in stream 0 using template query47.tpl +with v1 as( + select i_category, i_brand, + s_store_name, s_company_name, + d_year, d_moy, + sum(ss_sales_price) sum_sales, + avg(sum(ss_sales_price)) over + (partition by i_category, i_brand, + s_store_name, s_company_name, d_year) + avg_monthly_sales, + rank() over + (partition by i_category, i_brand, + s_store_name, s_company_name + order by d_year, d_moy) rn + from item, store_sales, date_dim, store + where ss_item_sk = i_item_sk and + ss_sold_date_sk = d_date_sk and + ss_store_sk = s_store_sk and + ( + d_year = 2000 or + ( d_year = 2000-1 and d_moy =12) or + ( d_year = 2000+1 and d_moy =1) + ) + group by i_category, i_brand, + s_store_name, s_company_name, + d_year, d_moy), + v2 as( + select v1.i_category + ,v1.d_year, v1.d_moy + ,v1.avg_monthly_sales + ,v1.sum_sales, v1_lag.sum_sales psum, v1_lead.sum_sales nsum + from v1, v1 v1_lag, v1 v1_lead + where v1.i_category = v1_lag.i_category and + v1.i_category = v1_lead.i_category and + v1.i_brand = v1_lag.i_brand and + v1.i_brand = v1_lead.i_brand and + v1.s_store_name = v1_lag.s_store_name and + v1.s_store_name = v1_lead.s_store_name and + v1.s_company_name = v1_lag.s_company_name and + v1.s_company_name = v1_lead.s_company_name and + v1.rn = v1_lag.rn + 1 and + v1.rn = v1_lead.rn - 1) + select * + from v2 + where d_year = 2000 and + avg_monthly_sales > 0 and + case when avg_monthly_sales > 0 then abs(sum_sales - avg_monthly_sales) / avg_monthly_sales else null end > 0.1 + order by sum_sales - avg_monthly_sales, 3 + limit 100; + + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query47.sql.out b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query47.sql.out new file mode 100644 index 00000000000..e5dcfc2bb2a --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query47.sql.out @@ -0,0 +1,153 @@ +-- Automatically generated by Gravitino Spark SQL test +-- !query +with v1 as( + select i_category, i_brand, + s_store_name, s_company_name, + d_year, d_moy, + sum(ss_sales_price) sum_sales, + avg(sum(ss_sales_price)) over + (partition by i_category, i_brand, + s_store_name, s_company_name, d_year) + avg_monthly_sales, + rank() over + (partition by i_category, i_brand, + s_store_name, s_company_name + order by d_year, d_moy) rn + from item, store_sales, date_dim, store + where ss_item_sk = i_item_sk and + ss_sold_date_sk = d_date_sk and + ss_store_sk = s_store_sk and + ( + d_year = 2000 or + ( d_year = 2000-1 and d_moy =12) or + ( d_year = 2000+1 and d_moy =1) + ) + group by i_category, i_brand, + s_store_name, s_company_name, + d_year, d_moy), + v2 as( + select v1.i_category + ,v1.d_year, v1.d_moy + ,v1.avg_monthly_sales + ,v1.sum_sales, v1_lag.sum_sales psum, v1_lead.sum_sales nsum + from v1, v1 v1_lag, v1 v1_lead + where v1.i_category = v1_lag.i_category and + v1.i_category = v1_lead.i_category and + v1.i_brand = v1_lag.i_brand and + v1.i_brand = v1_lead.i_brand and + v1.s_store_name = v1_lag.s_store_name and + v1.s_store_name = v1_lead.s_store_name and + v1.s_company_name = v1_lag.s_company_name and + v1.s_company_name = v1_lead.s_company_name and + v1.rn = v1_lag.rn + 1 and + v1.rn = v1_lead.rn - 1) + select * + from v2 + where d_year = 2000 and + avg_monthly_sales > 0 and + case when avg_monthly_sales > 0 then abs(sum_sales - avg_monthly_sales) / avg_monthly_sales else null end > 0.1 + order by sum_sales - avg_monthly_sales, 3 + limit 100 +-- !query schema +struct +-- !query output +Children 2000 1 93.636667 2.06 181.19 30.96 +Children 2000 10 76.847500 4.93 17.43 28.93 +Children 2000 10 78.458333 6.23 248.81 101.57 +Children 2000 10 82.772000 15.55 284.82 19.75 +Children 2000 11 79.906000 1.60 NULL 64.10 +Children 2000 11 89.144000 1.49 36.13 191.45 +Children 2000 2 89.144000 19.10 137.31 9.05 +Children 2000 2 96.949000 15.54 23.67 331.46 +Children 2000 2 99.108571 26.61 63.49 116.18 +Children 2000 3 89.144000 9.05 19.10 122.43 +Children 2000 3 95.350000 12.50 132.92 63.55 +Children 2000 4 79.906000 2.15 175.24 156.44 +Children 2000 5 106.815714 6.30 39.09 103.10 +Children 2000 6 96.949000 0.00 94.34 35.29 +Children 2000 7 123.395556 9.44 68.57 271.06 +Children 2000 7 83.203750 11.90 119.78 80.91 +Children 2000 7 83.456000 0.64 128.91 88.98 +Children 2000 8 78.458333 0.87 46.26 248.81 +Children 2000 9 81.078333 5.31 97.28 70.12 +Jewelry 2000 11 85.295000 13.23 157.36 53.86 +Men 2000 10 91.647143 21.03 7.47 54.94 +Men 2000 10 98.354000 13.17 162.89 200.37 +Men 2000 11 74.021111 5.52 44.36 83.37 +Men 2000 11 83.502500 1.66 178.08 88.00 +Men 2000 12 82.751429 3.12 262.73 78.00 +Men 2000 2 120.558571 17.28 215.39 22.94 +Men 2000 2 82.751429 8.23 121.61 11.65 +Men 2000 3 120.558571 22.94 17.28 16.49 +Men 2000 3 72.005714 4.52 63.93 1.36 +Men 2000 3 83.073333 2.11 31.58 69.76 +Men 2000 3 96.257143 22.14 119.84 49.43 +Men 2000 3 98.354000 31.00 20.28 84.34 +Men 2000 4 72.005714 1.36 4.52 49.03 +Men 2000 5 82.751429 11.65 8.23 180.55 +Men 2000 6 120.558571 16.49 22.94 306.22 +Men 2000 7 115.748000 18.06 240.37 105.42 +Men 2000 8 114.054000 19.48 120.96 8.49 +Men 2000 8 80.626667 8.79 186.27 149.24 +Men 2000 8 83.502500 3.11 150.03 30.76 +Men 2000 9 104.324000 31.29 140.74 133.80 +Men 2000 9 114.054000 8.49 19.48 53.72 +Men 2000 9 91.647143 7.47 101.89 21.03 +Men 2000 9 96.257143 4.40 49.43 45.84 +Music 2000 1 84.745000 6.12 96.47 27.81 +Music 2000 10 110.632000 13.42 235.43 9.30 +Music 2000 10 92.783333 17.32 40.23 39.37 +Music 2000 3 70.351429 2.31 9.05 105.11 +Music 2000 4 105.056250 32.67 216.33 2.05 +Music 2000 6 105.056250 2.05 32.67 130.23 +Music 2000 9 132.980000 30.27 9.89 212.46 +Music 2000 9 91.116250 2.06 100.71 51.41 +Shoes 2000 1 173.746000 94.79 88.96 0.65 +Shoes 2000 10 111.150000 33.94 286.14 44.12 +Shoes 2000 10 112.910000 25.47 168.95 203.45 +Shoes 2000 10 124.960000 3.51 85.77 169.42 +Shoes 2000 10 125.030000 24.12 0.82 172.45 +Shoes 2000 10 83.156667 8.07 192.61 5.15 +Shoes 2000 10 98.231111 14.69 161.55 140.92 +Shoes 2000 11 111.150000 44.12 33.94 189.12 +Shoes 2000 11 83.156667 5.15 8.07 141.41 +Shoes 2000 12 80.214167 0.00 271.26 75.82 +Shoes 2000 2 80.214167 2.47 44.22 92.66 +Shoes 2000 2 81.999000 11.49 68.49 10.48 +Shoes 2000 3 81.999000 10.48 11.49 2.62 +Shoes 2000 3 82.427000 2.73 172.01 32.62 +Shoes 2000 3 83.156667 0.20 93.44 173.70 +Shoes 2000 3 89.993750 22.19 107.78 83.04 +Shoes 2000 4 124.960000 24.10 131.23 85.77 +Shoes 2000 4 81.999000 2.62 10.48 93.01 +Shoes 2000 4 97.400000 10.76 383.45 28.71 +Shoes 2000 4 98.231111 24.53 98.27 165.66 +Shoes 2000 5 173.746000 0.65 94.79 245.75 +Shoes 2000 5 80.810000 3.66 33.31 167.51 +Shoes 2000 5 81.917500 11.79 501.37 3.83 +Shoes 2000 6 103.530000 11.97 87.47 79.01 +Shoes 2000 7 107.068000 12.76 248.28 25.98 +Shoes 2000 7 109.071111 2.17 44.29 165.24 +Shoes 2000 7 111.150000 39.60 73.98 286.14 +Shoes 2000 7 82.427000 3.21 55.90 271.27 +Shoes 2000 8 107.068000 25.98 12.76 224.85 +Shoes 2000 8 125.030000 0.82 138.55 24.12 +Shoes 2000 8 81.917500 3.83 11.79 257.14 +Shoes 2000 8 97.400000 28.71 10.76 33.15 +Sports 2000 11 87.383333 12.95 158.25 90.95 +Women 2000 1 144.950000 0.87 15.56 127.63 +Women 2000 1 74.775000 5.39 27.53 136.01 +Women 2000 1 78.457143 3.69 20.86 21.96 +Women 2000 10 118.694286 6.25 195.14 279.10 +Women 2000 12 113.622857 25.01 242.76 48.67 +Women 2000 2 103.383750 4.25 247.73 58.47 +Women 2000 4 118.694286 49.15 60.53 107.11 +Women 2000 4 89.961250 18.99 63.78 162.71 +Women 2000 5 103.886667 5.37 105.76 66.41 +Women 2000 5 113.622857 23.80 133.82 179.85 +Women 2000 5 91.488571 24.37 205.39 50.57 +Women 2000 6 78.457143 0.90 21.96 96.49 +Women 2000 8 103.383750 30.39 122.38 238.14 +Women 2000 8 67.845000 0.30 3.79 95.19 +Women 2000 8 78.794286 5.53 87.27 137.47 +Women 2000 8 91.488571 17.24 50.57 85.18 diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query48.sql b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query48.sql new file mode 100644 index 00000000000..e097040daf6 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query48.sql @@ -0,0 +1,67 @@ +-- start query 1 in stream 0 using template query48.tpl +select sum (ss_quantity) + from store_sales, store, customer_demographics, customer_address, date_dim + where s_store_sk = ss_store_sk + and ss_sold_date_sk = d_date_sk and d_year = 1998 + and + ( + ( + cd_demo_sk = ss_cdemo_sk + and + cd_marital_status = 'M' + and + cd_education_status = '4 yr Degree' + and + ss_sales_price between 100.00 and 150.00 + ) + or + ( + cd_demo_sk = ss_cdemo_sk + and + cd_marital_status = 'D' + and + cd_education_status = 'Primary' + and + ss_sales_price between 50.00 and 100.00 + ) + or + ( + cd_demo_sk = ss_cdemo_sk + and + cd_marital_status = 'U' + and + cd_education_status = 'Advanced Degree' + and + ss_sales_price between 150.00 and 200.00 + ) + ) + and + ( + ( + ss_addr_sk = ca_address_sk + and + ca_country = 'United States' + and + ca_state in ('KY', 'GA', 'NM') + and ss_net_profit between 0 and 2000 + ) + or + (ss_addr_sk = ca_address_sk + and + ca_country = 'United States' + and + ca_state in ('MT', 'OR', 'IN') + and ss_net_profit between 150 and 3000 + ) + or + (ss_addr_sk = ca_address_sk + and + ca_country = 'United States' + and + ca_state in ('WI', 'MO', 'WV') + and ss_net_profit between 50 and 25000 + ) + ) +; + + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query48.sql.out b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query48.sql.out new file mode 100644 index 00000000000..171dcc25223 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query48.sql.out @@ -0,0 +1,69 @@ +-- Automatically generated by Gravitino Spark SQL test +-- !query +select sum (ss_quantity) + from store_sales, store, customer_demographics, customer_address, date_dim + where s_store_sk = ss_store_sk + and ss_sold_date_sk = d_date_sk and d_year = 1998 + and + ( + ( + cd_demo_sk = ss_cdemo_sk + and + cd_marital_status = 'M' + and + cd_education_status = '4 yr Degree' + and + ss_sales_price between 100.00 and 150.00 + ) + or + ( + cd_demo_sk = ss_cdemo_sk + and + cd_marital_status = 'D' + and + cd_education_status = 'Primary' + and + ss_sales_price between 50.00 and 100.00 + ) + or + ( + cd_demo_sk = ss_cdemo_sk + and + cd_marital_status = 'U' + and + cd_education_status = 'Advanced Degree' + and + ss_sales_price between 150.00 and 200.00 + ) + ) + and + ( + ( + ss_addr_sk = ca_address_sk + and + ca_country = 'United States' + and + ca_state in ('KY', 'GA', 'NM') + and ss_net_profit between 0 and 2000 + ) + or + (ss_addr_sk = ca_address_sk + and + ca_country = 'United States' + and + ca_state in ('MT', 'OR', 'IN') + and ss_net_profit between 150 and 3000 + ) + or + (ss_addr_sk = ca_address_sk + and + ca_country = 'United States' + and + ca_state in ('WI', 'MO', 'WV') + and ss_net_profit between 50 and 25000 + ) + ) +-- !query schema +struct +-- !query output +NULL diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query49.sql b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query49.sql new file mode 100644 index 00000000000..981c3996cbb --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query49.sql @@ -0,0 +1,127 @@ +-- start query 1 in stream 0 using template query49.tpl +select + 'web' as channel + ,web.item + ,web.return_ratio + ,web.return_rank + ,web.currency_rank + from ( + select + item + ,return_ratio + ,currency_ratio + ,rank() over (order by return_ratio) as return_rank + ,rank() over (order by currency_ratio) as currency_rank + from + ( select ws.ws_item_sk as item + ,(cast(sum(coalesce(wr.wr_return_quantity,0)) as decimal(15,4))/ + cast(sum(coalesce(ws.ws_quantity,0)) as decimal(15,4) )) as return_ratio + ,(cast(sum(coalesce(wr.wr_return_amt,0)) as decimal(15,4))/ + cast(sum(coalesce(ws.ws_net_paid,0)) as decimal(15,4) )) as currency_ratio + from + web_sales ws left outer join web_returns wr + on (ws.ws_order_number = wr.wr_order_number and + ws.ws_item_sk = wr.wr_item_sk) + ,date_dim + where + wr.wr_return_amt > 10000 + and ws.ws_net_profit > 1 + and ws.ws_net_paid > 0 + and ws.ws_quantity > 0 + and ws_sold_date_sk = d_date_sk + and d_year = 2000 + and d_moy = 12 + group by ws.ws_item_sk + ) in_web + ) web + where + ( + web.return_rank <= 10 + or + web.currency_rank <= 10 + ) + union + select + 'catalog' as channel + ,catalog.item + ,catalog.return_ratio + ,catalog.return_rank + ,catalog.currency_rank + from ( + select + item + ,return_ratio + ,currency_ratio + ,rank() over (order by return_ratio) as return_rank + ,rank() over (order by currency_ratio) as currency_rank + from + ( select + cs.cs_item_sk as item + ,(cast(sum(coalesce(cr.cr_return_quantity,0)) as decimal(15,4))/ + cast(sum(coalesce(cs.cs_quantity,0)) as decimal(15,4) )) as return_ratio + ,(cast(sum(coalesce(cr.cr_return_amount,0)) as decimal(15,4))/ + cast(sum(coalesce(cs.cs_net_paid,0)) as decimal(15,4) )) as currency_ratio + from + catalog_sales cs left outer join catalog_returns cr + on (cs.cs_order_number = cr.cr_order_number and + cs.cs_item_sk = cr.cr_item_sk) + ,date_dim + where + cr.cr_return_amount > 10000 + and cs.cs_net_profit > 1 + and cs.cs_net_paid > 0 + and cs.cs_quantity > 0 + and cs_sold_date_sk = d_date_sk + and d_year = 2000 + and d_moy = 12 + group by cs.cs_item_sk + ) in_cat + ) catalog + where + ( + catalog.return_rank <= 10 + or + catalog.currency_rank <=10 + ) + union + select + 'store' as channel + ,store.item + ,store.return_ratio + ,store.return_rank + ,store.currency_rank + from ( + select + item + ,return_ratio + ,currency_ratio + ,rank() over (order by return_ratio) as return_rank + ,rank() over (order by currency_ratio) as currency_rank + from + ( select sts.ss_item_sk as item + ,(cast(sum(coalesce(sr.sr_return_quantity,0)) as decimal(15,4))/cast(sum(coalesce(sts.ss_quantity,0)) as decimal(15,4) )) as return_ratio + ,(cast(sum(coalesce(sr.sr_return_amt,0)) as decimal(15,4))/cast(sum(coalesce(sts.ss_net_paid,0)) as decimal(15,4) )) as currency_ratio + from + store_sales sts left outer join store_returns sr + on (sts.ss_ticket_number = sr.sr_ticket_number and sts.ss_item_sk = sr.sr_item_sk) + ,date_dim + where + sr.sr_return_amt > 10000 + and sts.ss_net_profit > 1 + and sts.ss_net_paid > 0 + and sts.ss_quantity > 0 + and ss_sold_date_sk = d_date_sk + and d_year = 2000 + and d_moy = 12 + group by sts.ss_item_sk + ) in_store + ) store + where ( + store.return_rank <= 10 + or + store.currency_rank <= 10 + ) + order by 1,4,5 + limit 100; + + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query49.sql.out b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query49.sql.out new file mode 100644 index 00000000000..d200934e4cc --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query49.sql.out @@ -0,0 +1,130 @@ +-- Automatically generated by Gravitino Spark SQL test +-- !query +select + 'web' as channel + ,web.item + ,web.return_ratio + ,web.return_rank + ,web.currency_rank + from ( + select + item + ,return_ratio + ,currency_ratio + ,rank() over (order by return_ratio) as return_rank + ,rank() over (order by currency_ratio) as currency_rank + from + ( select ws.ws_item_sk as item + ,(cast(sum(coalesce(wr.wr_return_quantity,0)) as decimal(15,4))/ + cast(sum(coalesce(ws.ws_quantity,0)) as decimal(15,4) )) as return_ratio + ,(cast(sum(coalesce(wr.wr_return_amt,0)) as decimal(15,4))/ + cast(sum(coalesce(ws.ws_net_paid,0)) as decimal(15,4) )) as currency_ratio + from + web_sales ws left outer join web_returns wr + on (ws.ws_order_number = wr.wr_order_number and + ws.ws_item_sk = wr.wr_item_sk) + ,date_dim + where + wr.wr_return_amt > 10000 + and ws.ws_net_profit > 1 + and ws.ws_net_paid > 0 + and ws.ws_quantity > 0 + and ws_sold_date_sk = d_date_sk + and d_year = 2000 + and d_moy = 12 + group by ws.ws_item_sk + ) in_web + ) web + where + ( + web.return_rank <= 10 + or + web.currency_rank <= 10 + ) + union + select + 'catalog' as channel + ,catalog.item + ,catalog.return_ratio + ,catalog.return_rank + ,catalog.currency_rank + from ( + select + item + ,return_ratio + ,currency_ratio + ,rank() over (order by return_ratio) as return_rank + ,rank() over (order by currency_ratio) as currency_rank + from + ( select + cs.cs_item_sk as item + ,(cast(sum(coalesce(cr.cr_return_quantity,0)) as decimal(15,4))/ + cast(sum(coalesce(cs.cs_quantity,0)) as decimal(15,4) )) as return_ratio + ,(cast(sum(coalesce(cr.cr_return_amount,0)) as decimal(15,4))/ + cast(sum(coalesce(cs.cs_net_paid,0)) as decimal(15,4) )) as currency_ratio + from + catalog_sales cs left outer join catalog_returns cr + on (cs.cs_order_number = cr.cr_order_number and + cs.cs_item_sk = cr.cr_item_sk) + ,date_dim + where + cr.cr_return_amount > 10000 + and cs.cs_net_profit > 1 + and cs.cs_net_paid > 0 + and cs.cs_quantity > 0 + and cs_sold_date_sk = d_date_sk + and d_year = 2000 + and d_moy = 12 + group by cs.cs_item_sk + ) in_cat + ) catalog + where + ( + catalog.return_rank <= 10 + or + catalog.currency_rank <=10 + ) + union + select + 'store' as channel + ,store.item + ,store.return_ratio + ,store.return_rank + ,store.currency_rank + from ( + select + item + ,return_ratio + ,currency_ratio + ,rank() over (order by return_ratio) as return_rank + ,rank() over (order by currency_ratio) as currency_rank + from + ( select sts.ss_item_sk as item + ,(cast(sum(coalesce(sr.sr_return_quantity,0)) as decimal(15,4))/cast(sum(coalesce(sts.ss_quantity,0)) as decimal(15,4) )) as return_ratio + ,(cast(sum(coalesce(sr.sr_return_amt,0)) as decimal(15,4))/cast(sum(coalesce(sts.ss_net_paid,0)) as decimal(15,4) )) as currency_ratio + from + store_sales sts left outer join store_returns sr + on (sts.ss_ticket_number = sr.sr_ticket_number and sts.ss_item_sk = sr.sr_item_sk) + ,date_dim + where + sr.sr_return_amt > 10000 + and sts.ss_net_profit > 1 + and sts.ss_net_paid > 0 + and sts.ss_quantity > 0 + and ss_sold_date_sk = d_date_sk + and d_year = 2000 + and d_moy = 12 + group by sts.ss_item_sk + ) in_store + ) store + where ( + store.return_rank <= 10 + or + store.currency_rank <= 10 + ) + order by 1,4,5 + limit 100 +-- !query schema +struct +-- !query output + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query5.sql b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query5.sql new file mode 100644 index 00000000000..2a879441940 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query5.sql @@ -0,0 +1,129 @@ +-- start query 1 in stream 0 using template query5.tpl +with ssr as + (select s_store_id, + sum(sales_price) as sales, + sum(profit) as profit, + sum(return_amt) as returns, + sum(net_loss) as profit_loss + from + ((select ss_store_sk as store_sk, + ss_sold_date_sk as date_sk, + ss_ext_sales_price as sales_price, + ss_net_profit as profit, + cast(0 as decimal(7,2)) as return_amt, + cast(0 as decimal(7,2)) as net_loss + from store_sales) + union all + (select sr_store_sk as store_sk, + sr_returned_date_sk as date_sk, + cast(0 as decimal(7,2)) as sales_price, + cast(0 as decimal(7,2)) as profit, + sr_return_amt as return_amt, + sr_net_loss as net_loss + from store_returns) + ) + salesreturns, + date_dim, + store + where date_sk = d_date_sk + and d_date between cast('1998-08-04' as date) + and (cast('1998-08-04' as date) + interval 14 days) + and store_sk = s_store_sk + group by s_store_id) + , + csr as + (select cp_catalog_page_id, + sum(sales_price) as sales, + sum(profit) as profit, + sum(return_amt) as returns, + sum(net_loss) as profit_loss + from + ( (select cs_catalog_page_sk as page_sk, + cs_sold_date_sk as date_sk, + cs_ext_sales_price as sales_price, + cs_net_profit as profit, + cast(0 as decimal(7,2)) as return_amt, + cast(0 as decimal(7,2)) as net_loss + from catalog_sales) + union all + (select cr_catalog_page_sk as page_sk, + cr_returned_date_sk as date_sk, + cast(0 as decimal(7,2)) as sales_price, + cast(0 as decimal(7,2)) as profit, + cr_return_amount as return_amt, + cr_net_loss as net_loss + from catalog_returns) + ) salesreturns, + date_dim, + catalog_page + where date_sk = d_date_sk + and d_date between cast('1998-08-04' as date) + and (cast('1998-08-04' as date) + interval 14 days) + and page_sk = cp_catalog_page_sk + group by cp_catalog_page_id) + , + wsr as + (select web_site_id, + sum(sales_price) as sales, + sum(profit) as profit, + sum(return_amt) as returns, + sum(net_loss) as profit_loss + from + ( (select ws_web_site_sk as wsr_web_site_sk, + ws_sold_date_sk as date_sk, + ws_ext_sales_price as sales_price, + ws_net_profit as profit, + cast(0 as decimal(7,2)) as return_amt, + cast(0 as decimal(7,2)) as net_loss + from web_sales) + union all + (select ws_web_site_sk as wsr_web_site_sk, + wr_returned_date_sk as date_sk, + cast(0 as decimal(7,2)) as sales_price, + cast(0 as decimal(7,2)) as profit, + wr_return_amt as return_amt, + wr_net_loss as net_loss + from web_returns left outer join web_sales on + ( wr_item_sk = ws_item_sk + and wr_order_number = ws_order_number)) + ) salesreturns, + date_dim, + web_site + where date_sk = d_date_sk + and d_date between cast('1998-08-04' as date) + and (cast('1998-08-04' as date) + interval 14 days) + and wsr_web_site_sk = web_site_sk + group by web_site_id) + select channel + , id + , sum(sales) as sales + , sum(returns) as returns + , sum(profit) as profit + from + ((select 'store channel' as channel + , concat('store' , s_store_id) as id + , sales + , returns + , (profit - profit_loss) as profit + from ssr) + union all + (select 'catalog channel' as channel + , concat('catalog_page' , cp_catalog_page_id) as id + , sales + , returns + , (profit - profit_loss) as profit + from csr) + union all + (select 'web channel' as channel + , concat('web_site' , web_site_id) as id + , sales + , returns + , (profit - profit_loss) as profit + from wsr) + ) x + group by rollup (channel, id) + order by channel + ,id + limit 100; + + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query5.sql.out b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query5.sql.out new file mode 100644 index 00000000000..48baae46388 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query5.sql.out @@ -0,0 +1,176 @@ +-- Automatically generated by Gravitino Spark SQL test +-- !query +with ssr as + (select s_store_id, + sum(sales_price) as sales, + sum(profit) as profit, + sum(return_amt) as returns, + sum(net_loss) as profit_loss + from + ((select ss_store_sk as store_sk, + ss_sold_date_sk as date_sk, + ss_ext_sales_price as sales_price, + ss_net_profit as profit, + cast(0 as decimal(7,2)) as return_amt, + cast(0 as decimal(7,2)) as net_loss + from store_sales) + union all + (select sr_store_sk as store_sk, + sr_returned_date_sk as date_sk, + cast(0 as decimal(7,2)) as sales_price, + cast(0 as decimal(7,2)) as profit, + sr_return_amt as return_amt, + sr_net_loss as net_loss + from store_returns) + ) + salesreturns, + date_dim, + store + where date_sk = d_date_sk + and d_date between cast('1998-08-04' as date) + and (cast('1998-08-04' as date) + interval 14 days) + and store_sk = s_store_sk + group by s_store_id) + , + csr as + (select cp_catalog_page_id, + sum(sales_price) as sales, + sum(profit) as profit, + sum(return_amt) as returns, + sum(net_loss) as profit_loss + from + ( (select cs_catalog_page_sk as page_sk, + cs_sold_date_sk as date_sk, + cs_ext_sales_price as sales_price, + cs_net_profit as profit, + cast(0 as decimal(7,2)) as return_amt, + cast(0 as decimal(7,2)) as net_loss + from catalog_sales) + union all + (select cr_catalog_page_sk as page_sk, + cr_returned_date_sk as date_sk, + cast(0 as decimal(7,2)) as sales_price, + cast(0 as decimal(7,2)) as profit, + cr_return_amount as return_amt, + cr_net_loss as net_loss + from catalog_returns) + ) salesreturns, + date_dim, + catalog_page + where date_sk = d_date_sk + and d_date between cast('1998-08-04' as date) + and (cast('1998-08-04' as date) + interval 14 days) + and page_sk = cp_catalog_page_sk + group by cp_catalog_page_id) + , + wsr as + (select web_site_id, + sum(sales_price) as sales, + sum(profit) as profit, + sum(return_amt) as returns, + sum(net_loss) as profit_loss + from + ( (select ws_web_site_sk as wsr_web_site_sk, + ws_sold_date_sk as date_sk, + ws_ext_sales_price as sales_price, + ws_net_profit as profit, + cast(0 as decimal(7,2)) as return_amt, + cast(0 as decimal(7,2)) as net_loss + from web_sales) + union all + (select ws_web_site_sk as wsr_web_site_sk, + wr_returned_date_sk as date_sk, + cast(0 as decimal(7,2)) as sales_price, + cast(0 as decimal(7,2)) as profit, + wr_return_amt as return_amt, + wr_net_loss as net_loss + from web_returns left outer join web_sales on + ( wr_item_sk = ws_item_sk + and wr_order_number = ws_order_number)) + ) salesreturns, + date_dim, + web_site + where date_sk = d_date_sk + and d_date between cast('1998-08-04' as date) + and (cast('1998-08-04' as date) + interval 14 days) + and wsr_web_site_sk = web_site_sk + group by web_site_id) + select channel + , id + , sum(sales) as sales + , sum(returns) as returns + , sum(profit) as profit + from + ((select 'store channel' as channel + , concat('store' , s_store_id) as id + , sales + , returns + , (profit - profit_loss) as profit + from ssr) + union all + (select 'catalog channel' as channel + , concat('catalog_page' , cp_catalog_page_id) as id + , sales + , returns + , (profit - profit_loss) as profit + from csr) + union all + (select 'web channel' as channel + , concat('web_site' , web_site_id) as id + , sales + , returns + , (profit - profit_loss) as profit + from wsr) + ) x + group by rollup (channel, id) + order by channel + ,id + limit 100 +-- !query schema +struct +-- !query output +NULL NULL 506679.85 17261.00 -203531.74 +catalog channel NULL 0.00 4899.24 NULL +catalog channel catalog_pageAAAAAAAAABAAAAAA 0.00 73.57 NULL +catalog channel catalog_pageAAAAAAAABAAAAAAA 0.00 254.90 NULL +catalog channel catalog_pageAAAAAAAABBAAAAAA 0.00 405.75 NULL +catalog channel catalog_pageAAAAAAAACAAAAAAA 0.00 800.00 NULL +catalog channel catalog_pageAAAAAAAACBAAAAAA 0.00 440.89 NULL +catalog channel catalog_pageAAAAAAAADAAAAAAA 0.00 78.70 NULL +catalog channel catalog_pageAAAAAAAADBAAAAAA 0.00 147.78 NULL +catalog channel catalog_pageAAAAAAAAEAAAAAAA 0.00 17.65 NULL +catalog channel catalog_pageAAAAAAAAEBAAAAAA 0.00 660.05 NULL +catalog channel catalog_pageAAAAAAAAFAAAAAAA 0.00 443.41 NULL +catalog channel catalog_pageAAAAAAAAGAAAAAAA 0.00 217.73 NULL +catalog channel catalog_pageAAAAAAAAHAAAAAAA 0.00 0.07 NULL +catalog channel catalog_pageAAAAAAAAIAAAAAAA 0.00 451.36 NULL +catalog channel catalog_pageAAAAAAAAJAAAAAAA 0.00 89.34 NULL +catalog channel catalog_pageAAAAAAAAKAAAAAAA 0.00 105.34 NULL +catalog channel catalog_pageAAAAAAAALAAAAAAA 0.00 30.43 NULL +catalog channel catalog_pageAAAAAAAAMAAAAAAA 0.00 224.03 NULL +catalog channel catalog_pageAAAAAAAANAAAAAAA 0.00 51.12 NULL +catalog channel catalog_pageAAAAAAAAOAAAAAAA 0.00 257.48 NULL +catalog channel catalog_pageAAAAAAAAPAAAAAAA 0.00 149.64 NULL +store channel NULL 371201.09 11952.47 -178708.59 +store channel storeAAAAAAAABAAAAAAA 10712.16 3385.56 -4845.39 +store channel storeAAAAAAAACAAAAAAA 82495.65 364.92 -39994.20 +store channel storeAAAAAAAAEAAAAAAA 57571.92 3409.81 -41886.17 +store channel storeAAAAAAAAHAAAAAAA 80649.95 1279.82 -21820.22 +store channel storeAAAAAAAAIAAAAAAA 98532.73 1104.48 -37771.78 +store channel storeAAAAAAAAKAAAAAAA 41238.68 2407.88 -32390.83 +web channel NULL 135478.76 409.29 -24823.15 +web channel web_siteAAAAAAAAABAAAAAA 5131.57 0.00 -1670.96 +web channel web_siteAAAAAAAABAAAAAAA 6968.33 25.05 -8935.97 +web channel web_siteAAAAAAAACAAAAAAA 1388.50 0.00 -1396.78 +web channel web_siteAAAAAAAADBAAAAAA 12384.81 0.00 6480.74 +web channel web_siteAAAAAAAAEAAAAAAA 16.70 0.00 2.60 +web channel web_siteAAAAAAAAEBAAAAAA 8501.68 0.00 -10879.22 +web channel web_siteAAAAAAAAGBAAAAAA 22569.13 0.00 6032.06 +web channel web_siteAAAAAAAAHAAAAAAA 14390.23 0.00 -2174.13 +web channel web_siteAAAAAAAAIAAAAAAA 11451.48 0.00 -1405.63 +web channel web_siteAAAAAAAAJBAAAAAA 8.96 0.00 -107.84 +web channel web_siteAAAAAAAAKAAAAAAA 13771.31 0.00 2545.76 +web channel web_siteAAAAAAAAKBAAAAAA 16188.14 0.00 -2769.41 +web channel web_siteAAAAAAAAMBAAAAAA 15808.72 384.24 -6086.85 +web channel web_siteAAAAAAAANAAAAAAA 5063.13 0.00 -5074.70 +web channel web_siteAAAAAAAAOAAAAAAA 1836.07 0.00 617.18 diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query50.sql b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query50.sql new file mode 100644 index 00000000000..7e069585384 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query50.sql @@ -0,0 +1,59 @@ +-- start query 1 in stream 0 using template query50.tpl +select + s_store_name + ,s_company_id + ,s_street_number + ,s_street_name + ,s_street_type + ,s_suite_number + ,s_city + ,s_county + ,s_state + ,s_zip + ,sum(case when (sr_returned_date_sk - ss_sold_date_sk <= 30 ) then 1 else 0 end) as 30days + ,sum(case when (sr_returned_date_sk - ss_sold_date_sk > 30) and + (sr_returned_date_sk - ss_sold_date_sk <= 60) then 1 else 0 end ) as 3160days + ,sum(case when (sr_returned_date_sk - ss_sold_date_sk > 60) and + (sr_returned_date_sk - ss_sold_date_sk <= 90) then 1 else 0 end) as 6190days + ,sum(case when (sr_returned_date_sk - ss_sold_date_sk > 90) and + (sr_returned_date_sk - ss_sold_date_sk <= 120) then 1 else 0 end) as 91120days + ,sum(case when (sr_returned_date_sk - ss_sold_date_sk > 120) then 1 else 0 end) as dy120days +from + store_sales + ,store_returns + ,store + ,date_dim d1 + ,date_dim d2 +where + d2.d_year = 2000 +and d2.d_moy = 9 +and ss_ticket_number = sr_ticket_number +and ss_item_sk = sr_item_sk +and ss_sold_date_sk = d1.d_date_sk +and sr_returned_date_sk = d2.d_date_sk +and ss_customer_sk = sr_customer_sk +and ss_store_sk = s_store_sk +group by + s_store_name + ,s_company_id + ,s_street_number + ,s_street_name + ,s_street_type + ,s_suite_number + ,s_city + ,s_county + ,s_state + ,s_zip +order by s_store_name + ,s_company_id + ,s_street_number + ,s_street_name + ,s_street_type + ,s_suite_number + ,s_city + ,s_county + ,s_state + ,s_zip +limit 100; + + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query50.sql.out b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query50.sql.out new file mode 100644 index 00000000000..f12f5db3c1d --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query50.sql.out @@ -0,0 +1,67 @@ +-- Automatically generated by Gravitino Spark SQL test +-- !query +select + s_store_name + ,s_company_id + ,s_street_number + ,s_street_name + ,s_street_type + ,s_suite_number + ,s_city + ,s_county + ,s_state + ,s_zip + ,sum(case when (sr_returned_date_sk - ss_sold_date_sk <= 30 ) then 1 else 0 end) as 30days + ,sum(case when (sr_returned_date_sk - ss_sold_date_sk > 30) and + (sr_returned_date_sk - ss_sold_date_sk <= 60) then 1 else 0 end ) as 3160days + ,sum(case when (sr_returned_date_sk - ss_sold_date_sk > 60) and + (sr_returned_date_sk - ss_sold_date_sk <= 90) then 1 else 0 end) as 6190days + ,sum(case when (sr_returned_date_sk - ss_sold_date_sk > 90) and + (sr_returned_date_sk - ss_sold_date_sk <= 120) then 1 else 0 end) as 91120days + ,sum(case when (sr_returned_date_sk - ss_sold_date_sk > 120) then 1 else 0 end) as dy120days +from + store_sales + ,store_returns + ,store + ,date_dim d1 + ,date_dim d2 +where + d2.d_year = 2000 +and d2.d_moy = 9 +and ss_ticket_number = sr_ticket_number +and ss_item_sk = sr_item_sk +and ss_sold_date_sk = d1.d_date_sk +and sr_returned_date_sk = d2.d_date_sk +and ss_customer_sk = sr_customer_sk +and ss_store_sk = s_store_sk +group by + s_store_name + ,s_company_id + ,s_street_number + ,s_street_name + ,s_street_type + ,s_suite_number + ,s_city + ,s_county + ,s_state + ,s_zip +order by s_store_name + ,s_company_id + ,s_street_number + ,s_street_name + ,s_street_type + ,s_suite_number + ,s_city + ,s_county + ,s_state + ,s_zip +limit 100 +-- !query schema +struct +-- !query output +able 1 255 Sycamore Dr. Suite 410 Midway Williamson County TN 31904 0 1 0 1 0 +ation 1 811 Lee Circle Suite T Midway Williamson County TN 31904 0 1 0 0 1 +bar 1 175 4th Court Suite C Midway Williamson County TN 31904 1 0 0 0 0 +eing 1 226 12th Lane Suite D Fairview Williamson County TN 35709 2 1 0 0 0 +ese 1 27 Lake Ln Suite 260 Midway Williamson County TN 31904 0 0 0 0 1 +ought 1 767 Spring Wy Suite 250 Midway Williamson County TN 31904 0 0 2 0 2 diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query51.sql b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query51.sql new file mode 100644 index 00000000000..74bfe423303 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query51.sql @@ -0,0 +1,45 @@ +-- start query 1 in stream 0 using template query51.tpl +WITH web_v1 as ( +select + ws_item_sk item_sk, d_date, + sum(sum(ws_sales_price)) + over (partition by ws_item_sk order by d_date rows between unbounded preceding and current row) cume_sales +from web_sales + ,date_dim +where ws_sold_date_sk=d_date_sk + and d_month_seq between 1212 and 1212+11 + and ws_item_sk is not NULL +group by ws_item_sk, d_date), +store_v1 as ( +select + ss_item_sk item_sk, d_date, + sum(sum(ss_sales_price)) + over (partition by ss_item_sk order by d_date rows between unbounded preceding and current row) cume_sales +from store_sales + ,date_dim +where ss_sold_date_sk=d_date_sk + and d_month_seq between 1212 and 1212+11 + and ss_item_sk is not NULL +group by ss_item_sk, d_date) + select * +from (select item_sk + ,d_date + ,web_sales + ,store_sales + ,max(web_sales) + over (partition by item_sk order by d_date rows between unbounded preceding and current row) web_cumulative + ,max(store_sales) + over (partition by item_sk order by d_date rows between unbounded preceding and current row) store_cumulative + from (select case when web.item_sk is not null then web.item_sk else store.item_sk end item_sk + ,case when web.d_date is not null then web.d_date else store.d_date end d_date + ,web.cume_sales web_sales + ,store.cume_sales store_sales + from web_v1 web full outer join store_v1 store on (web.item_sk = store.item_sk + and web.d_date = store.d_date) + )x )y +where web_cumulative > store_cumulative +order by item_sk + ,d_date +limit 100; + + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query51.sql.out b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query51.sql.out new file mode 100644 index 00000000000..0ef6e2629c4 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query51.sql.out @@ -0,0 +1,147 @@ +-- Automatically generated by Gravitino Spark SQL test +-- !query +WITH web_v1 as ( +select + ws_item_sk item_sk, d_date, + sum(sum(ws_sales_price)) + over (partition by ws_item_sk order by d_date rows between unbounded preceding and current row) cume_sales +from web_sales + ,date_dim +where ws_sold_date_sk=d_date_sk + and d_month_seq between 1212 and 1212+11 + and ws_item_sk is not NULL +group by ws_item_sk, d_date), +store_v1 as ( +select + ss_item_sk item_sk, d_date, + sum(sum(ss_sales_price)) + over (partition by ss_item_sk order by d_date rows between unbounded preceding and current row) cume_sales +from store_sales + ,date_dim +where ss_sold_date_sk=d_date_sk + and d_month_seq between 1212 and 1212+11 + and ss_item_sk is not NULL +group by ss_item_sk, d_date) + select * +from (select item_sk + ,d_date + ,web_sales + ,store_sales + ,max(web_sales) + over (partition by item_sk order by d_date rows between unbounded preceding and current row) web_cumulative + ,max(store_sales) + over (partition by item_sk order by d_date rows between unbounded preceding and current row) store_cumulative + from (select case when web.item_sk is not null then web.item_sk else store.item_sk end item_sk + ,case when web.d_date is not null then web.d_date else store.d_date end d_date + ,web.cume_sales web_sales + ,store.cume_sales store_sales + from web_v1 web full outer join store_v1 store on (web.item_sk = store.item_sk + and web.d_date = store.d_date) + )x )y +where web_cumulative > store_cumulative +order by item_sk + ,d_date +limit 100 +-- !query schema +struct +-- !query output +1007 2001-11-08 NULL 53.02 62.78 53.02 +109 2001-04-03 31.62 NULL 31.62 26.77 +1129 2001-11-26 23.63 NULL 23.63 1.16 +1181 2001-08-30 91.89 NULL 91.89 42.86 +1195 2001-09-11 NULL 16.79 64.51 16.79 +1219 2001-12-04 43.68 NULL 43.68 0.00 +1277 2001-12-30 81.83 NULL 81.83 65.19 +1327 2001-07-11 57.69 NULL 57.69 16.37 +1331 2001-11-20 40.82 NULL 40.82 13.76 +141 2001-09-23 NULL 63.16 89.43 63.16 +1411 2001-10-25 19.86 NULL 19.86 17.92 +1531 2001-10-04 38.48 NULL 38.48 3.97 +157 2001-11-18 48.73 NULL 48.73 27.60 +1597 2001-12-30 109.88 NULL 109.88 49.93 +1745 2001-11-05 159.66 NULL 159.66 1.12 +1755 2001-11-18 29.58 NULL 29.58 24.92 +179 2001-09-22 NULL 8.76 62.11 8.76 +1883 2001-08-03 NULL 1.40 9.18 1.40 +189 2001-05-15 NULL 8.24 86.94 8.24 +189 2001-10-05 152.87 NULL 152.87 8.24 +1915 2001-07-27 NULL 13.24 124.26 13.24 +1915 2001-08-09 NULL 17.71 124.26 17.71 +1917 2001-08-09 NULL 36.46 144.47 36.46 +1927 2001-08-30 141.37 3.71 141.37 3.71 +1967 2001-10-14 NULL 21.78 41.26 21.78 +2003 2001-06-10 NULL 2.32 76.51 2.32 +2025 2001-08-28 210.83 NULL 210.83 73.43 +2025 2001-11-30 NULL 94.99 210.83 94.99 +2031 2001-02-21 108.92 NULL 108.92 3.05 +2031 2001-05-08 NULL 49.27 108.92 49.27 +205 2001-12-23 157.76 NULL 157.76 12.90 +2077 2001-12-26 26.09 NULL 26.09 22.45 +2081 2001-12-17 162.96 NULL 162.96 56.12 +2131 2001-08-18 83.74 NULL 83.74 30.12 +223 2001-11-14 78.29 NULL 78.29 40.60 +2253 2001-12-23 61.32 NULL 61.32 18.52 +2351 2001-12-30 NULL 35.54 137.22 35.54 +2367 2001-11-05 8.33 NULL 8.33 5.94 +2387 2001-08-18 88.89 NULL 88.89 0.42 +2539 2001-11-02 NULL 58.36 74.55 58.36 +2551 2001-11-18 104.92 NULL 104.92 31.23 +2555 2001-10-15 NULL 21.42 31.93 21.42 +2635 2001-12-04 NULL 31.97 60.27 31.97 +2635 2001-12-18 NULL 33.12 60.27 33.12 +2649 2001-02-26 24.41 NULL 24.41 2.66 +2677 2001-12-18 NULL 26.75 331.49 26.75 +2689 2001-10-14 NULL 4.85 12.21 4.85 +2717 2001-12-14 NULL 33.14 88.10 33.14 +2801 2001-07-08 NULL 2.24 54.44 2.24 +2801 2001-08-08 NULL 32.66 54.44 32.66 +2861 2001-11-05 112.69 NULL 112.69 2.85 +2879 2001-10-26 59.67 NULL 59.67 6.09 +2927 2001-09-15 60.30 NULL 60.30 6.29 +2981 2001-10-04 11.98 NULL 11.98 2.78 +3 2001-12-05 NULL 26.32 45.06 26.32 +3031 2001-12-22 31.36 NULL 31.36 5.22 +3035 2001-09-09 83.98 NULL 83.98 5.58 +3085 2001-10-03 59.84 NULL 59.84 37.99 +3085 2001-11-04 94.65 NULL 94.65 37.99 +3127 2001-12-23 NULL 4.51 86.22 4.51 +3181 2001-07-13 NULL 4.89 52.95 4.89 +3193 2001-10-27 31.42 NULL 31.42 27.52 +3209 2001-10-25 NULL 96.10 125.12 96.10 +3209 2001-11-26 130.91 NULL 130.91 96.10 +3255 2001-11-10 100.15 NULL 100.15 0.79 +3255 2001-11-19 NULL 27.66 100.15 27.66 +3265 2001-11-19 58.86 NULL 58.86 22.99 +3265 2001-12-13 NULL 54.69 58.86 54.69 +3267 2001-03-22 46.94 NULL 46.94 24.10 +3429 2001-11-25 NULL 25.61 78.31 25.61 +349 2001-04-18 NULL 11.08 19.71 11.08 +349 2001-07-06 NULL 12.98 19.71 12.98 +3609 2001-10-26 26.85 NULL 26.85 2.57 +3635 2001-10-09 NULL 39.09 185.29 39.09 +3687 2001-09-18 NULL 31.38 58.63 31.38 +3689 2001-07-31 31.73 NULL 31.73 31.31 +3689 2001-11-04 NULL 31.31 31.73 31.31 +3771 2001-11-19 NULL 0.32 5.33 0.32 +3893 2001-09-18 80.40 NULL 80.40 6.86 +3969 2001-07-01 NULL 2.27 24.04 2.27 +3969 2001-11-26 47.67 NULL 47.67 2.27 +4021 2001-03-06 134.88 NULL 134.88 4.79 +501 2001-12-27 NULL 12.53 139.83 12.53 +523 2001-12-24 NULL 16.30 77.37 16.30 +531 2001-09-18 30.69 NULL 30.69 25.57 +551 2001-11-24 NULL 13.98 73.40 13.98 +599 2001-12-16 NULL 16.34 90.43 16.34 +611 2001-11-07 50.01 NULL 50.01 1.24 +631 2001-08-31 NULL 123.70 128.11 123.70 +683 2001-10-11 54.07 NULL 54.07 2.45 +689 2001-10-26 9.52 NULL 9.52 3.54 +727 2001-11-09 112.65 NULL 112.65 73.18 +731 2001-03-04 80.53 NULL 80.53 41.92 +737 2001-11-18 NULL 32.22 61.62 32.22 +745 2001-08-06 NULL 48.71 104.31 48.71 +745 2001-08-20 NULL 50.27 104.31 50.27 +851 2001-05-11 121.22 NULL 121.22 47.85 +97 2001-09-28 NULL 5.74 60.99 5.74 +97 2001-12-07 81.33 NULL 81.33 5.74 +999 2001-09-28 NULL 29.35 72.86 29.35 diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query52.sql b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query52.sql new file mode 100644 index 00000000000..c36178a7fca --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query52.sql @@ -0,0 +1,22 @@ +-- start query 1 in stream 0 using template query52.tpl +select dt.d_year + ,item.i_brand_id brand_id + ,item.i_brand brand + ,sum(ss_ext_sales_price) ext_price + from date_dim dt + ,store_sales + ,item + where dt.d_date_sk = store_sales.ss_sold_date_sk + and store_sales.ss_item_sk = item.i_item_sk + and item.i_manager_id = 1 + and dt.d_moy=12 + and dt.d_year=1998 + group by dt.d_year + ,item.i_brand + ,item.i_brand_id + order by dt.d_year + ,ext_price desc + ,brand_id +limit 100 ; + + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query52.sql.out b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query52.sql.out new file mode 100644 index 00000000000..4460a9d8b06 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query52.sql.out @@ -0,0 +1,39 @@ +-- Automatically generated by Gravitino Spark SQL test +-- !query +select dt.d_year + ,item.i_brand_id brand_id + ,item.i_brand brand + ,sum(ss_ext_sales_price) ext_price + from date_dim dt + ,store_sales + ,item + where dt.d_date_sk = store_sales.ss_sold_date_sk + and store_sales.ss_item_sk = item.i_item_sk + and item.i_manager_id = 1 + and dt.d_moy=12 + and dt.d_year=1998 + group by dt.d_year + ,item.i_brand + ,item.i_brand_id + order by dt.d_year + ,ext_price desc + ,brand_id +limit 100 +-- !query schema +struct +-- !query output +1998 10004001 edu packunivamalg #x 302.61 +1998 10004004 edu packunivamalg #x 48.19 +1998 10015004 scholaramalgamalg #x 805.56 +1998 1004002 edu packamalg #x 4067.90 +1998 2004002 edu packimporto #x 240.00 +1998 3003001 exportiexporti #x 1610.70 +1998 4001001 amalgedu pack #x 2052.36 +1998 4004001 edu packedu pack #x 2132.58 +1998 5003001 exportischolar #x 7662.68 +1998 6005001 scholarcorp #x 1937.32 +1998 6008005 namelesscorp #x 35.42 +1998 6011005 amalgbrand #x 5877.69 +1998 7016001 corpnameless #x 404.60 +1998 9012008 importounivamalg #x 6034.17 +1998 9013009 exportiunivamalg #x 9353.40 diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query53.sql b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query53.sql new file mode 100644 index 00000000000..0ab468c996a --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query53.sql @@ -0,0 +1,28 @@ +-- start query 1 in stream 0 using template query53.tpl +select * from +(select i_manufact_id, +sum(ss_sales_price) sum_sales, +avg(sum(ss_sales_price)) over (partition by i_manufact_id) avg_quarterly_sales +from item, store_sales, date_dim, store +where ss_item_sk = i_item_sk and +ss_sold_date_sk = d_date_sk and +ss_store_sk = s_store_sk and +d_month_seq in (1212,1212+1,1212+2,1212+3,1212+4,1212+5,1212+6,1212+7,1212+8,1212+9,1212+10,1212+11) and +((i_category in ('Books','Children','Electronics') and +i_class in ('personal','portable','reference','self-help') and +i_brand in ('scholaramalgamalg #14','scholaramalgamalg #7', + 'exportiunivamalg #9','scholaramalgamalg #9')) +or(i_category in ('Women','Music','Men') and +i_class in ('accessories','classical','fragrances','pants') and +i_brand in ('amalgimporto #1','edu packscholar #1','exportiimporto #1', + 'importoamalg #1'))) +group by i_manufact_id, d_qoy ) tmp1 +where case when avg_quarterly_sales > 0 + then abs (sum_sales - avg_quarterly_sales)/ avg_quarterly_sales + else null end > 0.1 +order by avg_quarterly_sales, + sum_sales, + i_manufact_id +limit 100; + + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query53.sql.out b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query53.sql.out new file mode 100644 index 00000000000..f139e2e1482 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query53.sql.out @@ -0,0 +1,31 @@ +-- Automatically generated by Gravitino Spark SQL test +-- !query +select * from +(select i_manufact_id, +sum(ss_sales_price) sum_sales, +avg(sum(ss_sales_price)) over (partition by i_manufact_id) avg_quarterly_sales +from item, store_sales, date_dim, store +where ss_item_sk = i_item_sk and +ss_sold_date_sk = d_date_sk and +ss_store_sk = s_store_sk and +d_month_seq in (1212,1212+1,1212+2,1212+3,1212+4,1212+5,1212+6,1212+7,1212+8,1212+9,1212+10,1212+11) and +((i_category in ('Books','Children','Electronics') and +i_class in ('personal','portable','reference','self-help') and +i_brand in ('scholaramalgamalg #14','scholaramalgamalg #7', + 'exportiunivamalg #9','scholaramalgamalg #9')) +or(i_category in ('Women','Music','Men') and +i_class in ('accessories','classical','fragrances','pants') and +i_brand in ('amalgimporto #1','edu packscholar #1','exportiimporto #1', + 'importoamalg #1'))) +group by i_manufact_id, d_qoy ) tmp1 +where case when avg_quarterly_sales > 0 + then abs (sum_sales - avg_quarterly_sales)/ avg_quarterly_sales + else null end > 0.1 +order by avg_quarterly_sales, + sum_sales, + i_manufact_id +limit 100 +-- !query schema +struct +-- !query output + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query54.sql b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query54.sql new file mode 100644 index 00000000000..f79cb662c9f --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query54.sql @@ -0,0 +1,56 @@ +-- start query 1 in stream 0 using template query54.tpl +with my_customers as ( + select distinct c_customer_sk + , c_current_addr_sk + from + ( select cs_sold_date_sk sold_date_sk, + cs_bill_customer_sk customer_sk, + cs_item_sk item_sk + from catalog_sales + union all + select ws_sold_date_sk sold_date_sk, + ws_bill_customer_sk customer_sk, + ws_item_sk item_sk + from web_sales + ) cs_or_ws_sales, + item, + date_dim, + customer + where sold_date_sk = d_date_sk + and item_sk = i_item_sk + and i_category = 'Jewelry' + and i_class = 'consignment' + and c_customer_sk = cs_or_ws_sales.customer_sk + and d_moy = 3 + and d_year = 1999 + ) + , my_revenue as ( + select c_customer_sk, + sum(ss_ext_sales_price) as revenue + from my_customers, + store_sales, + customer_address, + store, + date_dim + where c_current_addr_sk = ca_address_sk + and ca_county = s_county + and ca_state = s_state + and ss_sold_date_sk = d_date_sk + and c_customer_sk = ss_customer_sk + and d_month_seq between (select distinct d_month_seq+1 + from date_dim where d_year = 1999 and d_moy = 3) + and (select distinct d_month_seq+3 + from date_dim where d_year = 1999 and d_moy = 3) + group by c_customer_sk + ) + , segments as + (select cast((revenue/50) as int) as segment + from my_revenue + ) + select segment, count(*) as num_customers, segment*50 as segment_base + from segments + group by segment + order by segment, num_customers + limit 100; + + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query54.sql.out b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query54.sql.out new file mode 100644 index 00000000000..bd2e524821f --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query54.sql.out @@ -0,0 +1,59 @@ +-- Automatically generated by Gravitino Spark SQL test +-- !query +with my_customers as ( + select distinct c_customer_sk + , c_current_addr_sk + from + ( select cs_sold_date_sk sold_date_sk, + cs_bill_customer_sk customer_sk, + cs_item_sk item_sk + from catalog_sales + union all + select ws_sold_date_sk sold_date_sk, + ws_bill_customer_sk customer_sk, + ws_item_sk item_sk + from web_sales + ) cs_or_ws_sales, + item, + date_dim, + customer + where sold_date_sk = d_date_sk + and item_sk = i_item_sk + and i_category = 'Jewelry' + and i_class = 'consignment' + and c_customer_sk = cs_or_ws_sales.customer_sk + and d_moy = 3 + and d_year = 1999 + ) + , my_revenue as ( + select c_customer_sk, + sum(ss_ext_sales_price) as revenue + from my_customers, + store_sales, + customer_address, + store, + date_dim + where c_current_addr_sk = ca_address_sk + and ca_county = s_county + and ca_state = s_state + and ss_sold_date_sk = d_date_sk + and c_customer_sk = ss_customer_sk + and d_month_seq between (select distinct d_month_seq+1 + from date_dim where d_year = 1999 and d_moy = 3) + and (select distinct d_month_seq+3 + from date_dim where d_year = 1999 and d_moy = 3) + group by c_customer_sk + ) + , segments as + (select cast((revenue/50) as int) as segment + from my_revenue + ) + select segment, count(*) as num_customers, segment*50 as segment_base + from segments + group by segment + order by segment, num_customers + limit 100 +-- !query schema +struct +-- !query output + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query55.sql b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query55.sql new file mode 100644 index 00000000000..09c0d8ce971 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query55.sql @@ -0,0 +1,14 @@ +-- start query 1 in stream 0 using template query55.tpl +select i_brand_id brand_id, i_brand brand, + sum(ss_ext_sales_price) ext_price + from date_dim, store_sales, item + where d_date_sk = ss_sold_date_sk + and ss_item_sk = i_item_sk + and i_manager_id=36 + and d_moy=12 + and d_year=2001 + group by i_brand, i_brand_id + order by ext_price desc, i_brand_id +limit 100 ; + + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query55.sql.out b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query55.sql.out new file mode 100644 index 00000000000..9366e81a4dd --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query55.sql.out @@ -0,0 +1,23 @@ +-- Automatically generated by Gravitino Spark SQL test +-- !query +select i_brand_id brand_id, i_brand brand, + sum(ss_ext_sales_price) ext_price + from date_dim, store_sales, item + where d_date_sk = ss_sold_date_sk + and ss_item_sk = i_item_sk + and i_manager_id=36 + and d_moy=12 + and d_year=2001 + group by i_brand, i_brand_id + order by ext_price desc, i_brand_id +limit 100 +-- !query schema +struct +-- !query output +1004001 edu packamalg #x 1991.22 +2002002 importoimporto #x 1830.92 +3002001 exportiexporti #x 1264.80 +3004001 exportibrand #x 427.14 +4004002 edu packedu pack #x 2801.12 +6015006 scholarbrand #x 3150.63 +7014001 edu packcorp #x 720.90 diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query56.sql b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query56.sql new file mode 100644 index 00000000000..864d468a633 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query56.sql @@ -0,0 +1,68 @@ +-- start query 1 in stream 0 using template query56.tpl +with ss as ( + select i_item_id,sum(ss_ext_sales_price) total_sales + from + store_sales, + date_dim, + customer_address, + item + where i_item_id in (select + i_item_id +from item +where i_color in ('orchid','chiffon','lace')) + and ss_item_sk = i_item_sk + and ss_sold_date_sk = d_date_sk + and d_year = 2000 + and d_moy = 1 + and ss_addr_sk = ca_address_sk + and ca_gmt_offset = -8 + group by i_item_id), + cs as ( + select i_item_id,sum(cs_ext_sales_price) total_sales + from + catalog_sales, + date_dim, + customer_address, + item + where + i_item_id in (select + i_item_id +from item +where i_color in ('orchid','chiffon','lace')) + and cs_item_sk = i_item_sk + and cs_sold_date_sk = d_date_sk + and d_year = 2000 + and d_moy = 1 + and cs_bill_addr_sk = ca_address_sk + and ca_gmt_offset = -8 + group by i_item_id), + ws as ( + select i_item_id,sum(ws_ext_sales_price) total_sales + from + web_sales, + date_dim, + customer_address, + item + where + i_item_id in (select + i_item_id +from item +where i_color in ('orchid','chiffon','lace')) + and ws_item_sk = i_item_sk + and ws_sold_date_sk = d_date_sk + and d_year = 2000 + and d_moy = 1 + and ws_bill_addr_sk = ca_address_sk + and ca_gmt_offset = -8 + group by i_item_id) + select i_item_id ,sum(total_sales) total_sales + from (select * from ss + union all + select * from cs + union all + select * from ws) tmp1 + group by i_item_id + order by total_sales + limit 100; + + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query56.sql.out b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query56.sql.out new file mode 100644 index 00000000000..e0d9be16f9d --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query56.sql.out @@ -0,0 +1,71 @@ +-- Automatically generated by Gravitino Spark SQL test +-- !query +with ss as ( + select i_item_id,sum(ss_ext_sales_price) total_sales + from + store_sales, + date_dim, + customer_address, + item + where i_item_id in (select + i_item_id +from item +where i_color in ('orchid','chiffon','lace')) + and ss_item_sk = i_item_sk + and ss_sold_date_sk = d_date_sk + and d_year = 2000 + and d_moy = 1 + and ss_addr_sk = ca_address_sk + and ca_gmt_offset = -8 + group by i_item_id), + cs as ( + select i_item_id,sum(cs_ext_sales_price) total_sales + from + catalog_sales, + date_dim, + customer_address, + item + where + i_item_id in (select + i_item_id +from item +where i_color in ('orchid','chiffon','lace')) + and cs_item_sk = i_item_sk + and cs_sold_date_sk = d_date_sk + and d_year = 2000 + and d_moy = 1 + and cs_bill_addr_sk = ca_address_sk + and ca_gmt_offset = -8 + group by i_item_id), + ws as ( + select i_item_id,sum(ws_ext_sales_price) total_sales + from + web_sales, + date_dim, + customer_address, + item + where + i_item_id in (select + i_item_id +from item +where i_color in ('orchid','chiffon','lace')) + and ws_item_sk = i_item_sk + and ws_sold_date_sk = d_date_sk + and d_year = 2000 + and d_moy = 1 + and ws_bill_addr_sk = ca_address_sk + and ca_gmt_offset = -8 + group by i_item_id) + select i_item_id ,sum(total_sales) total_sales + from (select * from ss + union all + select * from cs + union all + select * from ws) tmp1 + group by i_item_id + order by total_sales + limit 100 +-- !query schema +struct +-- !query output + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query57.sql b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query57.sql new file mode 100644 index 00000000000..97abb44e4f0 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query57.sql @@ -0,0 +1,48 @@ +-- start query 1 in stream 0 using template query57.tpl +with v1 as( + select i_category, i_brand, + cc_name, + d_year, d_moy, + sum(cs_sales_price) sum_sales, + avg(sum(cs_sales_price)) over + (partition by i_category, i_brand, + cc_name, d_year) + avg_monthly_sales, + rank() over + (partition by i_category, i_brand, + cc_name + order by d_year, d_moy) rn + from item, catalog_sales, date_dim, call_center + where cs_item_sk = i_item_sk and + cs_sold_date_sk = d_date_sk and + cc_call_center_sk= cs_call_center_sk and + ( + d_year = 2000 or + ( d_year = 2000-1 and d_moy =12) or + ( d_year = 2000+1 and d_moy =1) + ) + group by i_category, i_brand, + cc_name , d_year, d_moy), + v2 as( + select v1.i_category, v1.i_brand + ,v1.d_year, v1.d_moy + ,v1.avg_monthly_sales + ,v1.sum_sales, v1_lag.sum_sales psum, v1_lead.sum_sales nsum + from v1, v1 v1_lag, v1 v1_lead + where v1.i_category = v1_lag.i_category and + v1.i_category = v1_lead.i_category and + v1.i_brand = v1_lag.i_brand and + v1.i_brand = v1_lead.i_brand and + v1. cc_name = v1_lag. cc_name and + v1. cc_name = v1_lead. cc_name and + v1.rn = v1_lag.rn + 1 and + v1.rn = v1_lead.rn - 1) + select * + from v2 + where d_year = 2000 and + avg_monthly_sales > 0 and + case when avg_monthly_sales > 0 then abs(sum_sales - avg_monthly_sales) / avg_monthly_sales else null end > 0.1 + order by sum_sales - avg_monthly_sales, 3 + limit 100; + + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query57.sql.out b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query57.sql.out new file mode 100644 index 00000000000..f4cc1d2f303 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query57.sql.out @@ -0,0 +1,51 @@ +-- Automatically generated by Gravitino Spark SQL test +-- !query +with v1 as( + select i_category, i_brand, + cc_name, + d_year, d_moy, + sum(cs_sales_price) sum_sales, + avg(sum(cs_sales_price)) over + (partition by i_category, i_brand, + cc_name, d_year) + avg_monthly_sales, + rank() over + (partition by i_category, i_brand, + cc_name + order by d_year, d_moy) rn + from item, catalog_sales, date_dim, call_center + where cs_item_sk = i_item_sk and + cs_sold_date_sk = d_date_sk and + cc_call_center_sk= cs_call_center_sk and + ( + d_year = 2000 or + ( d_year = 2000-1 and d_moy =12) or + ( d_year = 2000+1 and d_moy =1) + ) + group by i_category, i_brand, + cc_name , d_year, d_moy), + v2 as( + select v1.i_category, v1.i_brand + ,v1.d_year, v1.d_moy + ,v1.avg_monthly_sales + ,v1.sum_sales, v1_lag.sum_sales psum, v1_lead.sum_sales nsum + from v1, v1 v1_lag, v1 v1_lead + where v1.i_category = v1_lag.i_category and + v1.i_category = v1_lead.i_category and + v1.i_brand = v1_lag.i_brand and + v1.i_brand = v1_lead.i_brand and + v1. cc_name = v1_lag. cc_name and + v1. cc_name = v1_lead. cc_name and + v1.rn = v1_lag.rn + 1 and + v1.rn = v1_lead.rn - 1) + select * + from v2 + where d_year = 2000 and + avg_monthly_sales > 0 and + case when avg_monthly_sales > 0 then abs(sum_sales - avg_monthly_sales) / avg_monthly_sales else null end > 0.1 + order by sum_sales - avg_monthly_sales, 3 + limit 100 +-- !query schema +struct +-- !query output + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query58.sql b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query58.sql new file mode 100644 index 00000000000..241cc10f8c8 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query58.sql @@ -0,0 +1,65 @@ +-- start query 1 in stream 0 using template query58.tpl +with ss_items as + (select i_item_id item_id + ,sum(ss_ext_sales_price) ss_item_rev + from store_sales + ,item + ,date_dim + where ss_item_sk = i_item_sk + and d_date in (select d_date + from date_dim + where d_week_seq = (select d_week_seq + from date_dim + where d_date = '1998-02-19')) + and ss_sold_date_sk = d_date_sk + group by i_item_id), + cs_items as + (select i_item_id item_id + ,sum(cs_ext_sales_price) cs_item_rev + from catalog_sales + ,item + ,date_dim + where cs_item_sk = i_item_sk + and d_date in (select d_date + from date_dim + where d_week_seq = (select d_week_seq + from date_dim + where d_date = '1998-02-19')) + and cs_sold_date_sk = d_date_sk + group by i_item_id), + ws_items as + (select i_item_id item_id + ,sum(ws_ext_sales_price) ws_item_rev + from web_sales + ,item + ,date_dim + where ws_item_sk = i_item_sk + and d_date in (select d_date + from date_dim + where d_week_seq =(select d_week_seq + from date_dim + where d_date = '1998-02-19')) + and ws_sold_date_sk = d_date_sk + group by i_item_id) + select ss_items.item_id + ,ss_item_rev + ,ss_item_rev/((ss_item_rev+cs_item_rev+ws_item_rev)/3) * 100 ss_dev + ,cs_item_rev + ,cs_item_rev/((ss_item_rev+cs_item_rev+ws_item_rev)/3) * 100 cs_dev + ,ws_item_rev + ,ws_item_rev/((ss_item_rev+cs_item_rev+ws_item_rev)/3) * 100 ws_dev + ,(ss_item_rev+cs_item_rev+ws_item_rev)/3 average + from ss_items,cs_items,ws_items + where ss_items.item_id=cs_items.item_id + and ss_items.item_id=ws_items.item_id + and ss_item_rev between 0.9 * cs_item_rev and 1.1 * cs_item_rev + and ss_item_rev between 0.9 * ws_item_rev and 1.1 * ws_item_rev + and cs_item_rev between 0.9 * ss_item_rev and 1.1 * ss_item_rev + and cs_item_rev between 0.9 * ws_item_rev and 1.1 * ws_item_rev + and ws_item_rev between 0.9 * ss_item_rev and 1.1 * ss_item_rev + and ws_item_rev between 0.9 * cs_item_rev and 1.1 * cs_item_rev + order by item_id + ,ss_item_rev + limit 100; + + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query58.sql.out b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query58.sql.out new file mode 100644 index 00000000000..693d7e53413 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query58.sql.out @@ -0,0 +1,68 @@ +-- Automatically generated by Gravitino Spark SQL test +-- !query +with ss_items as + (select i_item_id item_id + ,sum(ss_ext_sales_price) ss_item_rev + from store_sales + ,item + ,date_dim + where ss_item_sk = i_item_sk + and d_date in (select d_date + from date_dim + where d_week_seq = (select d_week_seq + from date_dim + where d_date = '1998-02-19')) + and ss_sold_date_sk = d_date_sk + group by i_item_id), + cs_items as + (select i_item_id item_id + ,sum(cs_ext_sales_price) cs_item_rev + from catalog_sales + ,item + ,date_dim + where cs_item_sk = i_item_sk + and d_date in (select d_date + from date_dim + where d_week_seq = (select d_week_seq + from date_dim + where d_date = '1998-02-19')) + and cs_sold_date_sk = d_date_sk + group by i_item_id), + ws_items as + (select i_item_id item_id + ,sum(ws_ext_sales_price) ws_item_rev + from web_sales + ,item + ,date_dim + where ws_item_sk = i_item_sk + and d_date in (select d_date + from date_dim + where d_week_seq =(select d_week_seq + from date_dim + where d_date = '1998-02-19')) + and ws_sold_date_sk = d_date_sk + group by i_item_id) + select ss_items.item_id + ,ss_item_rev + ,ss_item_rev/((ss_item_rev+cs_item_rev+ws_item_rev)/3) * 100 ss_dev + ,cs_item_rev + ,cs_item_rev/((ss_item_rev+cs_item_rev+ws_item_rev)/3) * 100 cs_dev + ,ws_item_rev + ,ws_item_rev/((ss_item_rev+cs_item_rev+ws_item_rev)/3) * 100 ws_dev + ,(ss_item_rev+cs_item_rev+ws_item_rev)/3 average + from ss_items,cs_items,ws_items + where ss_items.item_id=cs_items.item_id + and ss_items.item_id=ws_items.item_id + and ss_item_rev between 0.9 * cs_item_rev and 1.1 * cs_item_rev + and ss_item_rev between 0.9 * ws_item_rev and 1.1 * ws_item_rev + and cs_item_rev between 0.9 * ss_item_rev and 1.1 * ss_item_rev + and cs_item_rev between 0.9 * ws_item_rev and 1.1 * ws_item_rev + and ws_item_rev between 0.9 * ss_item_rev and 1.1 * ss_item_rev + and ws_item_rev between 0.9 * cs_item_rev and 1.1 * cs_item_rev + order by item_id + ,ss_item_rev + limit 100 +-- !query schema +struct +-- !query output + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query59.sql b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query59.sql new file mode 100644 index 00000000000..491a57197a8 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query59.sql @@ -0,0 +1,44 @@ +-- start query 1 in stream 0 using template query59.tpl +with wss as + (select d_week_seq, + ss_store_sk, + sum(case when (d_day_name='Sunday') then ss_sales_price else null end) sun_sales, + sum(case when (d_day_name='Monday') then ss_sales_price else null end) mon_sales, + sum(case when (d_day_name='Tuesday') then ss_sales_price else null end) tue_sales, + sum(case when (d_day_name='Wednesday') then ss_sales_price else null end) wed_sales, + sum(case when (d_day_name='Thursday') then ss_sales_price else null end) thu_sales, + sum(case when (d_day_name='Friday') then ss_sales_price else null end) fri_sales, + sum(case when (d_day_name='Saturday') then ss_sales_price else null end) sat_sales + from store_sales,date_dim + where d_date_sk = ss_sold_date_sk + group by d_week_seq,ss_store_sk + ) + select s_store_name1,s_store_id1,d_week_seq1 + ,sun_sales1/sun_sales2,mon_sales1/mon_sales2 + ,tue_sales1/tue_sales2,wed_sales1/wed_sales2,thu_sales1/thu_sales2 + ,fri_sales1/fri_sales2,sat_sales1/sat_sales2 + from + (select s_store_name s_store_name1,wss.d_week_seq d_week_seq1 + ,s_store_id s_store_id1,sun_sales sun_sales1 + ,mon_sales mon_sales1,tue_sales tue_sales1 + ,wed_sales wed_sales1,thu_sales thu_sales1 + ,fri_sales fri_sales1,sat_sales sat_sales1 + from wss,store,date_dim d + where d.d_week_seq = wss.d_week_seq and + ss_store_sk = s_store_sk and + d_month_seq between 1185 and 1185 + 11) y, + (select s_store_name s_store_name2,wss.d_week_seq d_week_seq2 + ,s_store_id s_store_id2,sun_sales sun_sales2 + ,mon_sales mon_sales2,tue_sales tue_sales2 + ,wed_sales wed_sales2,thu_sales thu_sales2 + ,fri_sales fri_sales2,sat_sales sat_sales2 + from wss,store,date_dim d + where d.d_week_seq = wss.d_week_seq and + ss_store_sk = s_store_sk and + d_month_seq between 1185+ 12 and 1185 + 23) x + where s_store_id1=s_store_id2 + and d_week_seq1=d_week_seq2-52 + order by s_store_name1,s_store_id1,d_week_seq1 +limit 100; + + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query59.sql.out b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query59.sql.out new file mode 100644 index 00000000000..9f5a6ea1abb --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query59.sql.out @@ -0,0 +1,146 @@ +-- Automatically generated by Gravitino Spark SQL test +-- !query +with wss as + (select d_week_seq, + ss_store_sk, + sum(case when (d_day_name='Sunday') then ss_sales_price else null end) sun_sales, + sum(case when (d_day_name='Monday') then ss_sales_price else null end) mon_sales, + sum(case when (d_day_name='Tuesday') then ss_sales_price else null end) tue_sales, + sum(case when (d_day_name='Wednesday') then ss_sales_price else null end) wed_sales, + sum(case when (d_day_name='Thursday') then ss_sales_price else null end) thu_sales, + sum(case when (d_day_name='Friday') then ss_sales_price else null end) fri_sales, + sum(case when (d_day_name='Saturday') then ss_sales_price else null end) sat_sales + from store_sales,date_dim + where d_date_sk = ss_sold_date_sk + group by d_week_seq,ss_store_sk + ) + select s_store_name1,s_store_id1,d_week_seq1 + ,sun_sales1/sun_sales2,mon_sales1/mon_sales2 + ,tue_sales1/tue_sales2,wed_sales1/wed_sales2,thu_sales1/thu_sales2 + ,fri_sales1/fri_sales2,sat_sales1/sat_sales2 + from + (select s_store_name s_store_name1,wss.d_week_seq d_week_seq1 + ,s_store_id s_store_id1,sun_sales sun_sales1 + ,mon_sales mon_sales1,tue_sales tue_sales1 + ,wed_sales wed_sales1,thu_sales thu_sales1 + ,fri_sales fri_sales1,sat_sales sat_sales1 + from wss,store,date_dim d + where d.d_week_seq = wss.d_week_seq and + ss_store_sk = s_store_sk and + d_month_seq between 1185 and 1185 + 11) y, + (select s_store_name s_store_name2,wss.d_week_seq d_week_seq2 + ,s_store_id s_store_id2,sun_sales sun_sales2 + ,mon_sales mon_sales2,tue_sales tue_sales2 + ,wed_sales wed_sales2,thu_sales thu_sales2 + ,fri_sales fri_sales2,sat_sales sat_sales2 + from wss,store,date_dim d + where d.d_week_seq = wss.d_week_seq and + ss_store_sk = s_store_sk and + d_month_seq between 1185+ 12 and 1185 + 23) x + where s_store_id1=s_store_id2 + and d_week_seq1=d_week_seq2-52 + order by s_store_name1,s_store_id1,d_week_seq1 +limit 100 +-- !query schema +struct +-- !query output +able AAAAAAAACAAAAAAA 5153 NULL NULL NULL NULL 1.23861018383562931967 NULL NULL +able AAAAAAAACAAAAAAA 5153 NULL NULL NULL NULL 1.23861018383562931967 NULL NULL +able AAAAAAAACAAAAAAA 5153 NULL NULL NULL NULL 1.23861018383562931967 NULL NULL +able AAAAAAAACAAAAAAA 5153 NULL NULL NULL NULL 1.23861018383562931967 NULL NULL +able AAAAAAAACAAAAAAA 5153 NULL NULL NULL NULL 1.23861018383562931967 NULL NULL +able AAAAAAAACAAAAAAA 5153 NULL NULL NULL NULL 1.23861018383562931967 NULL NULL +able AAAAAAAACAAAAAAA 5153 NULL NULL NULL NULL 1.23861018383562931967 NULL NULL +able AAAAAAAACAAAAAAA 5153 NULL NULL NULL NULL 1.23861018383562931967 NULL NULL +able AAAAAAAACAAAAAAA 5153 NULL NULL NULL NULL 1.23861018383562931967 NULL NULL +able AAAAAAAACAAAAAAA 5153 NULL NULL NULL NULL 1.23861018383562931967 NULL NULL +able AAAAAAAACAAAAAAA 5153 NULL NULL NULL NULL 1.23861018383562931967 NULL NULL +able AAAAAAAACAAAAAAA 5153 NULL NULL NULL NULL 1.23861018383562931967 NULL NULL +able AAAAAAAACAAAAAAA 5153 NULL NULL NULL NULL 1.23861018383562931967 NULL NULL +able AAAAAAAACAAAAAAA 5153 NULL NULL NULL NULL 1.23861018383562931967 NULL NULL +able AAAAAAAACAAAAAAA 5153 NULL NULL NULL NULL 1.23861018383562931967 NULL NULL +able AAAAAAAACAAAAAAA 5153 NULL NULL NULL NULL 1.23861018383562931967 NULL NULL +able AAAAAAAACAAAAAAA 5153 NULL NULL NULL NULL 1.23861018383562931967 NULL NULL +able AAAAAAAACAAAAAAA 5153 NULL NULL NULL NULL 1.23861018383562931967 NULL NULL +able AAAAAAAACAAAAAAA 5153 NULL NULL NULL NULL 1.23861018383562931967 NULL NULL +able AAAAAAAACAAAAAAA 5153 NULL NULL NULL NULL 1.23861018383562931967 NULL NULL +able AAAAAAAACAAAAAAA 5154 NULL NULL NULL NULL NULL NULL NULL +able AAAAAAAACAAAAAAA 5154 NULL NULL NULL NULL NULL NULL NULL +able AAAAAAAACAAAAAAA 5154 NULL NULL NULL NULL NULL NULL NULL +able AAAAAAAACAAAAAAA 5154 NULL NULL NULL NULL NULL NULL NULL +able AAAAAAAACAAAAAAA 5154 NULL NULL NULL NULL NULL NULL NULL +able AAAAAAAACAAAAAAA 5154 NULL NULL NULL NULL NULL NULL NULL +able AAAAAAAACAAAAAAA 5154 NULL NULL NULL NULL NULL NULL NULL +able AAAAAAAACAAAAAAA 5154 NULL NULL NULL NULL NULL NULL NULL +able AAAAAAAACAAAAAAA 5154 NULL NULL NULL NULL NULL NULL NULL +able AAAAAAAACAAAAAAA 5154 NULL NULL NULL NULL NULL NULL NULL +able AAAAAAAACAAAAAAA 5154 NULL NULL NULL NULL NULL NULL NULL +able AAAAAAAACAAAAAAA 5154 NULL NULL NULL NULL NULL NULL NULL +able AAAAAAAACAAAAAAA 5154 NULL NULL NULL NULL NULL NULL NULL +able AAAAAAAACAAAAAAA 5154 NULL NULL NULL NULL NULL NULL NULL +able AAAAAAAACAAAAAAA 5154 NULL NULL NULL NULL NULL NULL NULL +able AAAAAAAACAAAAAAA 5154 NULL NULL NULL NULL NULL NULL NULL +able AAAAAAAACAAAAAAA 5154 NULL NULL NULL NULL NULL NULL NULL +able AAAAAAAACAAAAAAA 5154 NULL NULL NULL NULL NULL NULL NULL +able AAAAAAAACAAAAAAA 5154 NULL NULL NULL NULL NULL NULL NULL +able AAAAAAAACAAAAAAA 5154 NULL NULL NULL NULL NULL NULL NULL +able AAAAAAAACAAAAAAA 5154 NULL NULL NULL NULL NULL NULL NULL +able AAAAAAAACAAAAAAA 5154 NULL NULL NULL NULL NULL NULL NULL +able AAAAAAAACAAAAAAA 5154 NULL NULL NULL NULL NULL NULL NULL +able AAAAAAAACAAAAAAA 5154 NULL NULL NULL NULL NULL NULL NULL +able AAAAAAAACAAAAAAA 5154 NULL NULL NULL NULL NULL NULL NULL +able AAAAAAAACAAAAAAA 5154 NULL NULL NULL NULL NULL NULL NULL +able AAAAAAAACAAAAAAA 5154 NULL NULL NULL NULL NULL NULL NULL +able AAAAAAAACAAAAAAA 5154 NULL NULL NULL NULL NULL NULL NULL +able AAAAAAAACAAAAAAA 5154 NULL NULL NULL NULL NULL NULL NULL +able AAAAAAAACAAAAAAA 5154 NULL NULL NULL NULL NULL NULL NULL +able AAAAAAAACAAAAAAA 5154 NULL NULL NULL NULL NULL NULL NULL +able AAAAAAAACAAAAAAA 5154 NULL NULL NULL NULL NULL NULL NULL +able AAAAAAAACAAAAAAA 5154 NULL NULL NULL NULL NULL NULL NULL +able AAAAAAAACAAAAAAA 5154 NULL NULL NULL NULL NULL NULL NULL +able AAAAAAAACAAAAAAA 5154 NULL NULL NULL NULL NULL NULL NULL +able AAAAAAAACAAAAAAA 5154 NULL NULL NULL NULL NULL NULL NULL +able AAAAAAAACAAAAAAA 5154 NULL NULL NULL NULL NULL NULL NULL +able AAAAAAAACAAAAAAA 5154 NULL NULL NULL NULL NULL NULL NULL +able AAAAAAAACAAAAAAA 5154 NULL NULL NULL NULL NULL NULL NULL +able AAAAAAAACAAAAAAA 5154 NULL NULL NULL NULL NULL NULL NULL +able AAAAAAAACAAAAAAA 5154 NULL NULL NULL NULL NULL NULL NULL +able AAAAAAAACAAAAAAA 5154 NULL NULL NULL NULL NULL NULL NULL +able AAAAAAAACAAAAAAA 5154 NULL NULL NULL NULL NULL NULL NULL +able AAAAAAAACAAAAAAA 5154 NULL NULL NULL NULL NULL NULL NULL +able AAAAAAAACAAAAAAA 5154 NULL NULL NULL NULL NULL NULL NULL +able AAAAAAAACAAAAAAA 5154 NULL NULL NULL NULL NULL NULL NULL +able AAAAAAAACAAAAAAA 5154 NULL NULL NULL NULL NULL NULL NULL +able AAAAAAAACAAAAAAA 5154 NULL NULL NULL NULL NULL NULL NULL +able AAAAAAAACAAAAAAA 5154 NULL NULL NULL NULL NULL NULL NULL +able AAAAAAAACAAAAAAA 5155 NULL NULL NULL NULL NULL NULL NULL +able AAAAAAAACAAAAAAA 5155 NULL NULL NULL NULL NULL NULL NULL +able AAAAAAAACAAAAAAA 5155 NULL NULL NULL NULL NULL NULL NULL +able AAAAAAAACAAAAAAA 5155 NULL NULL NULL NULL NULL NULL NULL +able AAAAAAAACAAAAAAA 5155 NULL NULL NULL NULL NULL NULL NULL +able AAAAAAAACAAAAAAA 5155 NULL NULL NULL NULL NULL NULL NULL +able AAAAAAAACAAAAAAA 5155 NULL NULL NULL NULL NULL NULL NULL +able AAAAAAAACAAAAAAA 5155 NULL NULL NULL NULL NULL NULL NULL +able AAAAAAAACAAAAAAA 5155 NULL NULL NULL NULL NULL NULL NULL +able AAAAAAAACAAAAAAA 5155 NULL NULL NULL NULL NULL NULL NULL +able AAAAAAAACAAAAAAA 5155 NULL NULL NULL NULL NULL NULL NULL +able AAAAAAAACAAAAAAA 5155 NULL NULL NULL NULL NULL NULL NULL +able AAAAAAAACAAAAAAA 5155 NULL NULL NULL NULL NULL NULL NULL +able AAAAAAAACAAAAAAA 5155 NULL NULL NULL NULL NULL NULL NULL +able AAAAAAAACAAAAAAA 5155 NULL NULL NULL NULL NULL NULL NULL +able AAAAAAAACAAAAAAA 5155 NULL NULL NULL NULL NULL NULL NULL +able AAAAAAAACAAAAAAA 5155 NULL NULL NULL NULL NULL NULL NULL +able AAAAAAAACAAAAAAA 5155 NULL NULL NULL NULL NULL NULL NULL +able AAAAAAAACAAAAAAA 5155 NULL NULL NULL NULL NULL NULL NULL +able AAAAAAAACAAAAAAA 5155 NULL NULL NULL NULL NULL NULL NULL +able AAAAAAAACAAAAAAA 5155 NULL NULL NULL NULL NULL NULL NULL +able AAAAAAAACAAAAAAA 5155 NULL NULL NULL NULL NULL NULL NULL +able AAAAAAAACAAAAAAA 5155 NULL NULL NULL NULL NULL NULL NULL +able AAAAAAAACAAAAAAA 5155 NULL NULL NULL NULL NULL NULL NULL +able AAAAAAAACAAAAAAA 5155 NULL NULL NULL NULL NULL NULL NULL +able AAAAAAAACAAAAAAA 5155 NULL NULL NULL NULL NULL NULL NULL +able AAAAAAAACAAAAAAA 5155 NULL NULL NULL NULL NULL NULL NULL +able AAAAAAAACAAAAAAA 5155 NULL NULL NULL NULL NULL NULL NULL +able AAAAAAAACAAAAAAA 5155 NULL NULL NULL NULL NULL NULL NULL +able AAAAAAAACAAAAAAA 5155 NULL NULL NULL NULL NULL NULL NULL +able AAAAAAAACAAAAAAA 5155 NULL NULL NULL NULL NULL NULL NULL diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query6.sql b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query6.sql new file mode 100644 index 00000000000..9505a9e3b7c --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query6.sql @@ -0,0 +1,26 @@ +-- start query 1 in stream 0 using template query6.tpl +select a.ca_state state, count(*) cnt + from customer_address a + ,customer c + ,store_sales s + ,date_dim d + ,item i + where a.ca_address_sk = c.c_current_addr_sk + and c.c_customer_sk = s.ss_customer_sk + and s.ss_sold_date_sk = d.d_date_sk + and s.ss_item_sk = i.i_item_sk + and d.d_month_seq = + (select distinct (d_month_seq) + from date_dim + where d_year = 2000 + and d_moy = 2 ) + and i.i_current_price > 1.2 * + (select avg(j.i_current_price) + from item j + where j.i_category = i.i_category) + group by a.ca_state + having count(*) >= 10 + order by cnt + limit 100; + + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query6.sql.out b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query6.sql.out new file mode 100644 index 00000000000..b534d7fd44c --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query6.sql.out @@ -0,0 +1,29 @@ +-- Automatically generated by Gravitino Spark SQL test +-- !query +select a.ca_state state, count(*) cnt + from customer_address a + ,customer c + ,store_sales s + ,date_dim d + ,item i + where a.ca_address_sk = c.c_current_addr_sk + and c.c_customer_sk = s.ss_customer_sk + and s.ss_sold_date_sk = d.d_date_sk + and s.ss_item_sk = i.i_item_sk + and d.d_month_seq = + (select distinct (d_month_seq) + from date_dim + where d_year = 2000 + and d_moy = 2 ) + and i.i_current_price > 1.2 * + (select avg(j.i_current_price) + from item j + where j.i_category = i.i_category) + group by a.ca_state + having count(*) >= 10 + order by cnt + limit 100 +-- !query schema +struct +-- !query output + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query60.sql b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query60.sql new file mode 100644 index 00000000000..26e7c46d894 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query60.sql @@ -0,0 +1,78 @@ +-- start query 1 in stream 0 using template query60.tpl +with ss as ( + select + i_item_id,sum(ss_ext_sales_price) total_sales + from + store_sales, + date_dim, + customer_address, + item + where + i_item_id in (select + i_item_id +from + item +where i_category in ('Children')) + and ss_item_sk = i_item_sk + and ss_sold_date_sk = d_date_sk + and d_year = 1999 + and d_moy = 9 + and ss_addr_sk = ca_address_sk + and ca_gmt_offset = -6 + group by i_item_id), + cs as ( + select + i_item_id,sum(cs_ext_sales_price) total_sales + from + catalog_sales, + date_dim, + customer_address, + item + where + i_item_id in (select + i_item_id +from + item +where i_category in ('Children')) + and cs_item_sk = i_item_sk + and cs_sold_date_sk = d_date_sk + and d_year = 1999 + and d_moy = 9 + and cs_bill_addr_sk = ca_address_sk + and ca_gmt_offset = -6 + group by i_item_id), + ws as ( + select + i_item_id,sum(ws_ext_sales_price) total_sales + from + web_sales, + date_dim, + customer_address, + item + where + i_item_id in (select + i_item_id +from + item +where i_category in ('Children')) + and ws_item_sk = i_item_sk + and ws_sold_date_sk = d_date_sk + and d_year = 1999 + and d_moy = 9 + and ws_bill_addr_sk = ca_address_sk + and ca_gmt_offset = -6 + group by i_item_id) + select + i_item_id +,sum(total_sales) total_sales + from (select * from ss + union all + select * from cs + union all + select * from ws) tmp1 + group by i_item_id + order by i_item_id + ,total_sales + limit 100; + + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query60.sql.out b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query60.sql.out new file mode 100644 index 00000000000..36017bd60c5 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query60.sql.out @@ -0,0 +1,132 @@ +-- Automatically generated by Gravitino Spark SQL test +-- !query +with ss as ( + select + i_item_id,sum(ss_ext_sales_price) total_sales + from + store_sales, + date_dim, + customer_address, + item + where + i_item_id in (select + i_item_id +from + item +where i_category in ('Children')) + and ss_item_sk = i_item_sk + and ss_sold_date_sk = d_date_sk + and d_year = 1999 + and d_moy = 9 + and ss_addr_sk = ca_address_sk + and ca_gmt_offset = -6 + group by i_item_id), + cs as ( + select + i_item_id,sum(cs_ext_sales_price) total_sales + from + catalog_sales, + date_dim, + customer_address, + item + where + i_item_id in (select + i_item_id +from + item +where i_category in ('Children')) + and cs_item_sk = i_item_sk + and cs_sold_date_sk = d_date_sk + and d_year = 1999 + and d_moy = 9 + and cs_bill_addr_sk = ca_address_sk + and ca_gmt_offset = -6 + group by i_item_id), + ws as ( + select + i_item_id,sum(ws_ext_sales_price) total_sales + from + web_sales, + date_dim, + customer_address, + item + where + i_item_id in (select + i_item_id +from + item +where i_category in ('Children')) + and ws_item_sk = i_item_sk + and ws_sold_date_sk = d_date_sk + and d_year = 1999 + and d_moy = 9 + and ws_bill_addr_sk = ca_address_sk + and ca_gmt_offset = -6 + group by i_item_id) + select + i_item_id +,sum(total_sales) total_sales + from (select * from ss + union all + select * from cs + union all + select * from ws) tmp1 + group by i_item_id + order by i_item_id + ,total_sales + limit 100 +-- !query schema +struct +-- !query output +AAAAAAAAADCCAAAA 2436.18 +AAAAAAAAAHJDAAAA 2018.80 +AAAAAAAAAIGDAAAA 3594.50 +AAAAAAAAAKLBAAAA 4472.87 +AAAAAAAAAKNCAAAA 2165.80 +AAAAAAAAALJDAAAA 5841.00 +AAAAAAAAALMCAAAA 2641.40 +AAAAAAAAALPCAAAA 35.52 +AAAAAAAAANHDAAAA 986.53 +AAAAAAAAANIBAAAA 3609.64 +AAAAAAAACEGBAAAA 8273.20 +AAAAAAAACFDBAAAA 1980.18 +AAAAAAAACMOAAAAA 328.56 +AAAAAAAACPADAAAA 1162.50 +AAAAAAAAEDDDAAAA 296.01 +AAAAAAAAEFNAAAAA 999.36 +AAAAAAAAEMCEAAAA 187.20 +AAAAAAAAEPPDAAAA 1163.28 +AAAAAAAAFJMCAAAA 1779.52 +AAAAAAAAGHNCAAAA 623.20 +AAAAAAAAGIGCAAAA 229.44 +AAAAAAAAGLEEAAAA 1857.60 +AAAAAAAAHCGBAAAA 2093.25 +AAAAAAAAHJKCAAAA 6768.96 +AAAAAAAAHLDBAAAA 1248.80 +AAAAAAAAIAIAAAAA 7255.28 +AAAAAAAAIDACAAAA 1543.92 +AAAAAAAAIDPCAAAA 52.48 +AAAAAAAAIFAEAAAA 1642.00 +AAAAAAAAIFOCAAAA 503.04 +AAAAAAAAIJLAAAAA 4662.00 +AAAAAAAAILGAAAAA 27.03 +AAAAAAAAILIBAAAA 3112.97 +AAAAAAAAIPKBAAAA 0.00 +AAAAAAAAKJIBAAAA 116.44 +AAAAAAAALJEBAAAA 1527.28 +AAAAAAAALNJBAAAA 216.32 +AAAAAAAAMENDAAAA 1661.25 +AAAAAAAAMEOBAAAA 3.20 +AAAAAAAAMGMBAAAA 83.80 +AAAAAAAAMHHDAAAA 96.28 +AAAAAAAAMLIBAAAA 4823.72 +AAAAAAAAMNOBAAAA 89.50 +AAAAAAAAMPKDAAAA 3125.50 +AAAAAAAANAFBAAAA 8257.34 +AAAAAAAANAFEAAAA 232.90 +AAAAAAAANMECAAAA 151.11 +AAAAAAAAOAFBAAAA 3811.50 +AAAAAAAAOEEBAAAA 46.59 +AAAAAAAAOFHAAAAA 320.25 +AAAAAAAAOMKBAAAA 5627.16 +AAAAAAAAPNPAAAAA 7178.49 diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query61.sql b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query61.sql new file mode 100644 index 00000000000..8f6cf4bed77 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query61.sql @@ -0,0 +1,44 @@ +-- start query 1 in stream 0 using template query61.tpl +select promotions,total,cast(promotions as decimal(15,4))/cast(total as decimal(15,4))*100 +from + (select sum(ss_ext_sales_price) promotions + from store_sales + ,store + ,promotion + ,date_dim + ,customer + ,customer_address + ,item + where ss_sold_date_sk = d_date_sk + and ss_store_sk = s_store_sk + and ss_promo_sk = p_promo_sk + and ss_customer_sk= c_customer_sk + and ca_address_sk = c_current_addr_sk + and ss_item_sk = i_item_sk + and ca_gmt_offset = -7 + and i_category = 'Books' + and (p_channel_dmail = 'Y' or p_channel_email = 'Y' or p_channel_tv = 'Y') + and s_gmt_offset = -7 + and d_year = 1999 + and d_moy = 11) promotional_sales, + (select sum(ss_ext_sales_price) total + from store_sales + ,store + ,date_dim + ,customer + ,customer_address + ,item + where ss_sold_date_sk = d_date_sk + and ss_store_sk = s_store_sk + and ss_customer_sk= c_customer_sk + and ca_address_sk = c_current_addr_sk + and ss_item_sk = i_item_sk + and ca_gmt_offset = -7 + and i_category = 'Books' + and s_gmt_offset = -7 + and d_year = 1999 + and d_moy = 11) all_sales +order by promotions, total +limit 100; + + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query61.sql.out b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query61.sql.out new file mode 100644 index 00000000000..b56c690e89a --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query61.sql.out @@ -0,0 +1,47 @@ +-- Automatically generated by Gravitino Spark SQL test +-- !query +select promotions,total,cast(promotions as decimal(15,4))/cast(total as decimal(15,4))*100 +from + (select sum(ss_ext_sales_price) promotions + from store_sales + ,store + ,promotion + ,date_dim + ,customer + ,customer_address + ,item + where ss_sold_date_sk = d_date_sk + and ss_store_sk = s_store_sk + and ss_promo_sk = p_promo_sk + and ss_customer_sk= c_customer_sk + and ca_address_sk = c_current_addr_sk + and ss_item_sk = i_item_sk + and ca_gmt_offset = -7 + and i_category = 'Books' + and (p_channel_dmail = 'Y' or p_channel_email = 'Y' or p_channel_tv = 'Y') + and s_gmt_offset = -7 + and d_year = 1999 + and d_moy = 11) promotional_sales, + (select sum(ss_ext_sales_price) total + from store_sales + ,store + ,date_dim + ,customer + ,customer_address + ,item + where ss_sold_date_sk = d_date_sk + and ss_store_sk = s_store_sk + and ss_customer_sk= c_customer_sk + and ca_address_sk = c_current_addr_sk + and ss_item_sk = i_item_sk + and ca_gmt_offset = -7 + and i_category = 'Books' + and s_gmt_offset = -7 + and d_year = 1999 + and d_moy = 11) all_sales +order by promotions, total +limit 100 +-- !query schema +struct +-- !query output +NULL NULL NULL diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query62.sql b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query62.sql new file mode 100644 index 00000000000..ac74bb93180 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query62.sql @@ -0,0 +1,35 @@ +-- start query 1 in stream 0 using template query62.tpl +select + substr(w_warehouse_name,1,20) + ,sm_type + ,web_name + ,sum(case when (ws_ship_date_sk - ws_sold_date_sk <= 30 ) then 1 else 0 end) as 30days + ,sum(case when (ws_ship_date_sk - ws_sold_date_sk > 30) and + (ws_ship_date_sk - ws_sold_date_sk <= 60) then 1 else 0 end ) as 31_60days + ,sum(case when (ws_ship_date_sk - ws_sold_date_sk > 60) and + (ws_ship_date_sk - ws_sold_date_sk <= 90) then 1 else 0 end) as 61_90days + ,sum(case when (ws_ship_date_sk - ws_sold_date_sk > 90) and + (ws_ship_date_sk - ws_sold_date_sk <= 120) then 1 else 0 end) as 91_120days + ,sum(case when (ws_ship_date_sk - ws_sold_date_sk > 120) then 1 else 0 end) as dy120days +from + web_sales + ,warehouse + ,ship_mode + ,web_site + ,date_dim +where + d_month_seq between 1212 and 1212 + 11 +and ws_ship_date_sk = d_date_sk +and ws_warehouse_sk = w_warehouse_sk +and ws_ship_mode_sk = sm_ship_mode_sk +and ws_web_site_sk = web_site_sk +group by + substr(w_warehouse_name,1,20) + ,sm_type + ,web_name +order by substr(w_warehouse_name,1,20) + ,sm_type + ,web_name +limit 100; + + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query62.sql.out b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query62.sql.out new file mode 100644 index 00000000000..2813db26030 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query62.sql.out @@ -0,0 +1,137 @@ +-- Automatically generated by Gravitino Spark SQL test +-- !query +select + substr(w_warehouse_name,1,20) + ,sm_type + ,web_name + ,sum(case when (ws_ship_date_sk - ws_sold_date_sk <= 30 ) then 1 else 0 end) as 30days + ,sum(case when (ws_ship_date_sk - ws_sold_date_sk > 30) and + (ws_ship_date_sk - ws_sold_date_sk <= 60) then 1 else 0 end ) as 31_60days + ,sum(case when (ws_ship_date_sk - ws_sold_date_sk > 60) and + (ws_ship_date_sk - ws_sold_date_sk <= 90) then 1 else 0 end) as 61_90days + ,sum(case when (ws_ship_date_sk - ws_sold_date_sk > 90) and + (ws_ship_date_sk - ws_sold_date_sk <= 120) then 1 else 0 end) as 91_120days + ,sum(case when (ws_ship_date_sk - ws_sold_date_sk > 120) then 1 else 0 end) as dy120days +from + web_sales + ,warehouse + ,ship_mode + ,web_site + ,date_dim +where + d_month_seq between 1212 and 1212 + 11 +and ws_ship_date_sk = d_date_sk +and ws_warehouse_sk = w_warehouse_sk +and ws_ship_mode_sk = sm_ship_mode_sk +and ws_web_site_sk = web_site_sk +group by + substr(w_warehouse_name,1,20) + ,sm_type + ,web_name +order by substr(w_warehouse_name,1,20) + ,sm_type + ,web_name +limit 100 +-- !query schema +struct +-- !query output +Bad cards must make. EXPRESS site_0 4 3 3 4 0 +Bad cards must make. EXPRESS site_1 0 3 3 3 0 +Bad cards must make. EXPRESS site_2 4 5 2 7 0 +Bad cards must make. EXPRESS site_3 3 5 4 2 0 +Bad cards must make. EXPRESS site_4 4 5 4 5 0 +Bad cards must make. LIBRARY site_0 3 1 3 1 0 +Bad cards must make. LIBRARY site_1 4 5 3 5 0 +Bad cards must make. LIBRARY site_2 0 2 1 0 0 +Bad cards must make. LIBRARY site_3 3 1 2 2 0 +Bad cards must make. LIBRARY site_4 2 2 2 3 0 +Bad cards must make. NEXT DAY site_0 5 2 2 6 0 +Bad cards must make. NEXT DAY site_1 3 1 6 1 0 +Bad cards must make. NEXT DAY site_2 3 4 7 2 0 +Bad cards must make. NEXT DAY site_3 0 5 6 6 0 +Bad cards must make. NEXT DAY site_4 1 2 3 1 0 +Bad cards must make. OVERNIGHT site_0 7 2 4 2 0 +Bad cards must make. OVERNIGHT site_1 4 2 6 1 0 +Bad cards must make. OVERNIGHT site_2 1 3 4 1 0 +Bad cards must make. OVERNIGHT site_3 2 1 7 0 0 +Bad cards must make. OVERNIGHT site_4 2 5 1 4 0 +Bad cards must make. REGULAR site_0 3 3 2 3 0 +Bad cards must make. REGULAR site_1 1 2 0 5 0 +Bad cards must make. REGULAR site_2 3 5 4 1 0 +Bad cards must make. REGULAR site_3 6 4 1 1 0 +Bad cards must make. REGULAR site_4 2 4 3 4 0 +Bad cards must make. TWO DAY site_0 7 1 2 1 0 +Bad cards must make. TWO DAY site_1 2 1 2 4 0 +Bad cards must make. TWO DAY site_2 5 3 1 3 0 +Bad cards must make. TWO DAY site_3 0 3 1 1 0 +Bad cards must make. TWO DAY site_4 0 3 2 1 0 +Conventional childr EXPRESS site_0 7 5 2 1 0 +Conventional childr EXPRESS site_1 5 4 1 7 0 +Conventional childr EXPRESS site_2 1 3 2 4 0 +Conventional childr EXPRESS site_3 2 4 3 1 0 +Conventional childr EXPRESS site_4 3 5 4 2 0 +Conventional childr LIBRARY site_0 2 1 5 1 0 +Conventional childr LIBRARY site_1 2 7 3 2 0 +Conventional childr LIBRARY site_2 2 2 2 2 0 +Conventional childr LIBRARY site_3 0 4 2 2 0 +Conventional childr LIBRARY site_4 2 2 2 2 0 +Conventional childr NEXT DAY site_0 1 4 5 3 0 +Conventional childr NEXT DAY site_1 2 2 3 1 0 +Conventional childr NEXT DAY site_2 6 4 2 4 0 +Conventional childr NEXT DAY site_3 2 1 1 2 0 +Conventional childr NEXT DAY site_4 1 7 2 1 0 +Conventional childr OVERNIGHT site_0 3 3 1 3 0 +Conventional childr OVERNIGHT site_1 2 3 1 2 0 +Conventional childr OVERNIGHT site_2 2 3 5 1 0 +Conventional childr OVERNIGHT site_3 1 6 3 4 0 +Conventional childr OVERNIGHT site_4 2 2 4 1 0 +Conventional childr REGULAR site_0 3 4 5 5 0 +Conventional childr REGULAR site_1 4 1 1 0 0 +Conventional childr REGULAR site_2 1 2 3 2 0 +Conventional childr REGULAR site_3 1 1 0 1 0 +Conventional childr REGULAR site_4 3 2 3 0 0 +Conventional childr TWO DAY site_0 3 2 6 0 0 +Conventional childr TWO DAY site_1 3 3 3 1 0 +Conventional childr TWO DAY site_2 1 3 4 3 0 +Conventional childr TWO DAY site_3 2 2 3 5 0 +Conventional childr TWO DAY site_4 1 0 2 1 0 +Doors canno EXPRESS site_0 4 2 3 3 0 +Doors canno EXPRESS site_1 3 5 3 5 0 +Doors canno EXPRESS site_2 2 4 5 2 0 +Doors canno EXPRESS site_3 3 4 4 6 0 +Doors canno EXPRESS site_4 2 4 2 5 0 +Doors canno LIBRARY site_0 5 1 0 2 0 +Doors canno LIBRARY site_1 6 2 5 9 0 +Doors canno LIBRARY site_2 4 1 3 4 0 +Doors canno LIBRARY site_3 3 1 2 5 0 +Doors canno LIBRARY site_4 2 1 2 0 0 +NULL EXPRESS site_0 2 1 6 5 0 +NULL EXPRESS site_1 5 4 2 3 0 +NULL EXPRESS site_2 4 4 3 5 0 +NULL EXPRESS site_3 2 6 3 3 0 +NULL EXPRESS site_4 2 5 3 1 0 +NULL LIBRARY site_0 1 7 0 1 0 +NULL LIBRARY site_1 1 4 4 0 0 +NULL LIBRARY site_2 1 2 5 0 0 +NULL LIBRARY site_3 1 3 2 9 0 +NULL LIBRARY site_4 5 3 3 1 0 +NULL NEXT DAY site_0 1 3 3 1 0 +NULL NEXT DAY site_1 4 2 2 7 0 +NULL NEXT DAY site_2 1 2 3 3 0 +NULL NEXT DAY site_3 5 3 3 1 0 +NULL NEXT DAY site_4 3 1 2 4 0 +NULL OVERNIGHT site_0 1 2 5 2 0 +NULL OVERNIGHT site_1 1 3 1 0 0 +NULL OVERNIGHT site_2 2 1 2 2 0 +NULL OVERNIGHT site_3 1 2 1 0 0 +NULL OVERNIGHT site_4 3 6 2 2 0 +NULL REGULAR site_0 1 1 0 3 0 +NULL REGULAR site_1 4 3 1 4 0 +NULL REGULAR site_2 1 1 1 4 0 +NULL REGULAR site_3 4 1 0 3 0 +NULL REGULAR site_4 0 2 3 1 0 +NULL TWO DAY site_0 0 2 2 1 0 +NULL TWO DAY site_1 2 4 2 0 0 +NULL TWO DAY site_2 0 1 3 1 0 +NULL TWO DAY site_3 1 0 0 0 0 +NULL TWO DAY site_4 4 5 2 2 0 diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query63.sql b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query63.sql new file mode 100644 index 00000000000..e4daf13a35e --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query63.sql @@ -0,0 +1,29 @@ +-- start query 1 in stream 0 using template query63.tpl +select * +from (select i_manager_id + ,sum(ss_sales_price) sum_sales + ,avg(sum(ss_sales_price)) over (partition by i_manager_id) avg_monthly_sales + from item + ,store_sales + ,date_dim + ,store + where ss_item_sk = i_item_sk + and ss_sold_date_sk = d_date_sk + and ss_store_sk = s_store_sk + and d_month_seq in (1212,1212+1,1212+2,1212+3,1212+4,1212+5,1212+6,1212+7,1212+8,1212+9,1212+10,1212+11) + and (( i_category in ('Books','Children','Electronics') + and i_class in ('personal','portable','reference','self-help') + and i_brand in ('scholaramalgamalg #14','scholaramalgamalg #7', + 'exportiunivamalg #9','scholaramalgamalg #9')) + or( i_category in ('Women','Music','Men') + and i_class in ('accessories','classical','fragrances','pants') + and i_brand in ('amalgimporto #1','edu packscholar #1','exportiimporto #1', + 'importoamalg #1'))) +group by i_manager_id, d_moy) tmp1 +where case when avg_monthly_sales > 0 then abs (sum_sales - avg_monthly_sales) / avg_monthly_sales else null end > 0.1 +order by i_manager_id + ,avg_monthly_sales + ,sum_sales +limit 100; + + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query63.sql.out b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query63.sql.out new file mode 100644 index 00000000000..8b43e0962c3 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query63.sql.out @@ -0,0 +1,32 @@ +-- Automatically generated by Gravitino Spark SQL test +-- !query +select * +from (select i_manager_id + ,sum(ss_sales_price) sum_sales + ,avg(sum(ss_sales_price)) over (partition by i_manager_id) avg_monthly_sales + from item + ,store_sales + ,date_dim + ,store + where ss_item_sk = i_item_sk + and ss_sold_date_sk = d_date_sk + and ss_store_sk = s_store_sk + and d_month_seq in (1212,1212+1,1212+2,1212+3,1212+4,1212+5,1212+6,1212+7,1212+8,1212+9,1212+10,1212+11) + and (( i_category in ('Books','Children','Electronics') + and i_class in ('personal','portable','reference','self-help') + and i_brand in ('scholaramalgamalg #14','scholaramalgamalg #7', + 'exportiunivamalg #9','scholaramalgamalg #9')) + or( i_category in ('Women','Music','Men') + and i_class in ('accessories','classical','fragrances','pants') + and i_brand in ('amalgimporto #1','edu packscholar #1','exportiimporto #1', + 'importoamalg #1'))) +group by i_manager_id, d_moy) tmp1 +where case when avg_monthly_sales > 0 then abs (sum_sales - avg_monthly_sales) / avg_monthly_sales else null end > 0.1 +order by i_manager_id + ,avg_monthly_sales + ,sum_sales +limit 100 +-- !query schema +struct +-- !query output + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query64.sql b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query64.sql new file mode 100644 index 00000000000..20182f71c71 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query64.sql @@ -0,0 +1,119 @@ +-- start query 1 in stream 0 using template query64.tpl +with cs_ui as + (select cs_item_sk + ,sum(cs_ext_list_price) as sale,sum(cr_refunded_cash+cr_reversed_charge+cr_store_credit) as refund + from catalog_sales + ,catalog_returns + where cs_item_sk = cr_item_sk + and cs_order_number = cr_order_number + group by cs_item_sk + having sum(cs_ext_list_price)>2*sum(cr_refunded_cash+cr_reversed_charge+cr_store_credit)), +cross_sales as + (select i_product_name product_name + ,i_item_sk item_sk + ,s_store_name store_name + ,s_zip store_zip + ,ad1.ca_street_number b_street_number + ,ad1.ca_street_name b_street_name + ,ad1.ca_city b_city + ,ad1.ca_zip b_zip + ,ad2.ca_street_number c_street_number + ,ad2.ca_street_name c_street_name + ,ad2.ca_city c_city + ,ad2.ca_zip c_zip + ,d1.d_year as syear + ,d2.d_year as fsyear + ,d3.d_year s2year + ,count(*) cnt + ,sum(ss_wholesale_cost) s1 + ,sum(ss_list_price) s2 + ,sum(ss_coupon_amt) s3 + FROM store_sales + ,store_returns + ,cs_ui + ,date_dim d1 + ,date_dim d2 + ,date_dim d3 + ,store + ,customer + ,customer_demographics cd1 + ,customer_demographics cd2 + ,promotion + ,household_demographics hd1 + ,household_demographics hd2 + ,customer_address ad1 + ,customer_address ad2 + ,income_band ib1 + ,income_band ib2 + ,item + WHERE ss_store_sk = s_store_sk AND + ss_sold_date_sk = d1.d_date_sk AND + ss_customer_sk = c_customer_sk AND + ss_cdemo_sk= cd1.cd_demo_sk AND + ss_hdemo_sk = hd1.hd_demo_sk AND + ss_addr_sk = ad1.ca_address_sk and + ss_item_sk = i_item_sk and + ss_item_sk = sr_item_sk and + ss_ticket_number = sr_ticket_number and + ss_item_sk = cs_ui.cs_item_sk and + c_current_cdemo_sk = cd2.cd_demo_sk AND + c_current_hdemo_sk = hd2.hd_demo_sk AND + c_current_addr_sk = ad2.ca_address_sk and + c_first_sales_date_sk = d2.d_date_sk and + c_first_shipto_date_sk = d3.d_date_sk and + ss_promo_sk = p_promo_sk and + hd1.hd_income_band_sk = ib1.ib_income_band_sk and + hd2.hd_income_band_sk = ib2.ib_income_band_sk and + cd1.cd_marital_status <> cd2.cd_marital_status and + i_color in ('maroon','burnished','dim','steel','navajo','chocolate') and + i_current_price between 35 and 35 + 10 and + i_current_price between 35 + 1 and 35 + 15 +group by i_product_name + ,i_item_sk + ,s_store_name + ,s_zip + ,ad1.ca_street_number + ,ad1.ca_street_name + ,ad1.ca_city + ,ad1.ca_zip + ,ad2.ca_street_number + ,ad2.ca_street_name + ,ad2.ca_city + ,ad2.ca_zip + ,d1.d_year + ,d2.d_year + ,d3.d_year +) +select cs1.product_name + ,cs1.store_name + ,cs1.store_zip + ,cs1.b_street_number + ,cs1.b_street_name + ,cs1.b_city + ,cs1.b_zip + ,cs1.c_street_number + ,cs1.c_street_name + ,cs1.c_city + ,cs1.c_zip + ,cs1.syear + ,cs1.cnt + ,cs1.s1 as s11 + ,cs1.s2 as s21 + ,cs1.s3 as s31 + ,cs2.s1 as s12 + ,cs2.s2 as s22 + ,cs2.s3 as s32 + ,cs2.syear + ,cs2.cnt +from cross_sales cs1,cross_sales cs2 +where cs1.item_sk=cs2.item_sk and + cs1.syear = 2000 and + cs2.syear = 2000 + 1 and + cs2.cnt <= cs1.cnt and + cs1.store_name = cs2.store_name and + cs1.store_zip = cs2.store_zip +order by cs1.product_name + ,cs1.store_name + ,cs2.cnt; + + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query64.sql.out b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query64.sql.out new file mode 100644 index 00000000000..ff90d06bf6c --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query64.sql.out @@ -0,0 +1,122 @@ +-- Automatically generated by Gravitino Spark SQL test +-- !query +with cs_ui as + (select cs_item_sk + ,sum(cs_ext_list_price) as sale,sum(cr_refunded_cash+cr_reversed_charge+cr_store_credit) as refund + from catalog_sales + ,catalog_returns + where cs_item_sk = cr_item_sk + and cs_order_number = cr_order_number + group by cs_item_sk + having sum(cs_ext_list_price)>2*sum(cr_refunded_cash+cr_reversed_charge+cr_store_credit)), +cross_sales as + (select i_product_name product_name + ,i_item_sk item_sk + ,s_store_name store_name + ,s_zip store_zip + ,ad1.ca_street_number b_street_number + ,ad1.ca_street_name b_street_name + ,ad1.ca_city b_city + ,ad1.ca_zip b_zip + ,ad2.ca_street_number c_street_number + ,ad2.ca_street_name c_street_name + ,ad2.ca_city c_city + ,ad2.ca_zip c_zip + ,d1.d_year as syear + ,d2.d_year as fsyear + ,d3.d_year s2year + ,count(*) cnt + ,sum(ss_wholesale_cost) s1 + ,sum(ss_list_price) s2 + ,sum(ss_coupon_amt) s3 + FROM store_sales + ,store_returns + ,cs_ui + ,date_dim d1 + ,date_dim d2 + ,date_dim d3 + ,store + ,customer + ,customer_demographics cd1 + ,customer_demographics cd2 + ,promotion + ,household_demographics hd1 + ,household_demographics hd2 + ,customer_address ad1 + ,customer_address ad2 + ,income_band ib1 + ,income_band ib2 + ,item + WHERE ss_store_sk = s_store_sk AND + ss_sold_date_sk = d1.d_date_sk AND + ss_customer_sk = c_customer_sk AND + ss_cdemo_sk= cd1.cd_demo_sk AND + ss_hdemo_sk = hd1.hd_demo_sk AND + ss_addr_sk = ad1.ca_address_sk and + ss_item_sk = i_item_sk and + ss_item_sk = sr_item_sk and + ss_ticket_number = sr_ticket_number and + ss_item_sk = cs_ui.cs_item_sk and + c_current_cdemo_sk = cd2.cd_demo_sk AND + c_current_hdemo_sk = hd2.hd_demo_sk AND + c_current_addr_sk = ad2.ca_address_sk and + c_first_sales_date_sk = d2.d_date_sk and + c_first_shipto_date_sk = d3.d_date_sk and + ss_promo_sk = p_promo_sk and + hd1.hd_income_band_sk = ib1.ib_income_band_sk and + hd2.hd_income_band_sk = ib2.ib_income_band_sk and + cd1.cd_marital_status <> cd2.cd_marital_status and + i_color in ('maroon','burnished','dim','steel','navajo','chocolate') and + i_current_price between 35 and 35 + 10 and + i_current_price between 35 + 1 and 35 + 15 +group by i_product_name + ,i_item_sk + ,s_store_name + ,s_zip + ,ad1.ca_street_number + ,ad1.ca_street_name + ,ad1.ca_city + ,ad1.ca_zip + ,ad2.ca_street_number + ,ad2.ca_street_name + ,ad2.ca_city + ,ad2.ca_zip + ,d1.d_year + ,d2.d_year + ,d3.d_year +) +select cs1.product_name + ,cs1.store_name + ,cs1.store_zip + ,cs1.b_street_number + ,cs1.b_street_name + ,cs1.b_city + ,cs1.b_zip + ,cs1.c_street_number + ,cs1.c_street_name + ,cs1.c_city + ,cs1.c_zip + ,cs1.syear + ,cs1.cnt + ,cs1.s1 as s11 + ,cs1.s2 as s21 + ,cs1.s3 as s31 + ,cs2.s1 as s12 + ,cs2.s2 as s22 + ,cs2.s3 as s32 + ,cs2.syear + ,cs2.cnt +from cross_sales cs1,cross_sales cs2 +where cs1.item_sk=cs2.item_sk and + cs1.syear = 2000 and + cs2.syear = 2000 + 1 and + cs2.cnt <= cs1.cnt and + cs1.store_name = cs2.store_name and + cs1.store_zip = cs2.store_zip +order by cs1.product_name + ,cs1.store_name + ,cs2.cnt +-- !query schema +struct +-- !query output + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query65.sql b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query65.sql new file mode 100644 index 00000000000..50ec45f8755 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query65.sql @@ -0,0 +1,29 @@ +-- start query 1 in stream 0 using template query65.tpl +select + s_store_name, + i_item_desc, + sc.revenue, + i_current_price, + i_wholesale_cost, + i_brand + from store, item, + (select ss_store_sk, avg(revenue) as ave + from + (select ss_store_sk, ss_item_sk, + sum(ss_sales_price) as revenue + from store_sales, date_dim + where ss_sold_date_sk = d_date_sk and d_month_seq between 1212 and 1212+11 + group by ss_store_sk, ss_item_sk) sa + group by ss_store_sk) sb, + (select ss_store_sk, ss_item_sk, sum(ss_sales_price) as revenue + from store_sales, date_dim + where ss_sold_date_sk = d_date_sk and d_month_seq between 1212 and 1212+11 + group by ss_store_sk, ss_item_sk) sc + where sb.ss_store_sk = sc.ss_store_sk and + sc.revenue <= 0.1 * sb.ave and + s_store_sk = sc.ss_store_sk and + i_item_sk = sc.ss_item_sk + order by s_store_name, i_item_desc +limit 100; + + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query65.sql.out b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query65.sql.out new file mode 100644 index 00000000000..aaaef3e900d --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query65.sql.out @@ -0,0 +1,131 @@ +-- Automatically generated by Gravitino Spark SQL test +-- !query +select + s_store_name, + i_item_desc, + sc.revenue, + i_current_price, + i_wholesale_cost, + i_brand + from store, item, + (select ss_store_sk, avg(revenue) as ave + from + (select ss_store_sk, ss_item_sk, + sum(ss_sales_price) as revenue + from store_sales, date_dim + where ss_sold_date_sk = d_date_sk and d_month_seq between 1212 and 1212+11 + group by ss_store_sk, ss_item_sk) sa + group by ss_store_sk) sb, + (select ss_store_sk, ss_item_sk, sum(ss_sales_price) as revenue + from store_sales, date_dim + where ss_sold_date_sk = d_date_sk and d_month_seq between 1212 and 1212+11 + group by ss_store_sk, ss_item_sk) sc + where sb.ss_store_sk = sc.ss_store_sk and + sc.revenue <= 0.1 * sb.ave and + s_store_sk = sc.ss_store_sk and + i_item_sk = sc.ss_item_sk + order by s_store_name, i_item_desc +limit 100 +-- !query schema +struct +-- !query output +able A lot certain winners give with a farmers; financial friends mo 3.49 7.80 2.96 amalgamalg #x +able Able criteria could break normal, normal documents. Then metropolitan taxes may possess more huge early eyes. Sure long 1.79 1.64 1.16 exportiimporto #x +able Again avail 0.77 5.16 2.32 edu packbrand #x +able Always political centres seem now never modest sets. Major comments may search additional, other guards. National, appropriate relations buy very. 3.05 4.48 3.95 edu packexporti #x +able Anc 2.72 3.94 2.37 importoamalg #x +able Annual exc 1.27 3.33 6.13 corpcorp #x +able Annual forests meet. Even small instances occur feet. Different qualities replace. Main, new hands determine however for a ideas. Physical sides worry as 3.83 7.41 3.48 amalgamalg #x +able As pra 2.20 4.56 3.14 scholarnameless #x +able Aware, forthcoming students miss markedly healthy weekends; types should become oppo 1.73 1.09 14.27 edu packamalg #x +able Awkward, typical places must not check only workers; bright parties restore studies. Men show about finally interested programmes; certainly economic soldiers meet there 1.13 1.72 1.51 amalgedu pack #x +able Badly back designers limit right things. Exotic, modern lines signal perhaps. Rationally unexpected flowers set. 0.27 3.06 2.47 exportiedu pack #x +able Big, obvious commitments must provide also; strange women will collect 1.09 7.69 2.53 importoexporti #x +able Candidates will hold. Chief ages used to choose with the foundations; horrible, complex rates meet 0.22 1.08 0.72 exportiimporto #x +able Children could see again ahead of the patients. Nice terms ought to get high materials. Odd others would enable at a others. Together common subjects shall not interrupt other, brig 2.27 4.90 2.94 scholaramalgamalg #x +able Closer dependen 1.59 3.12 2.79 corpmaxi #x +able Colleagues notice angrily witnesses. Private organs may not reduce visitors; also old pain 3.32 2.26 1.15 corpnameless #x +able Conditions rise recently nearly aware horses. Difficult areas should anal 1.29 0.09 0.07 namelessunivamalg #x +able Courses will show to the hours. Simple feelings 1.81 9.12 3.61 importoimporto #x +able Current, important women give thousands. Firmly y 2.87 1.32 0.91 amalgedu pack #x +able Detectives must not assess international prisoners. Accidents amount further actions. Organis 1.49 2.18 1.67 amalgexporti #x +able Different shares shall last even words. Contracts make on a others. Far from awful colleagues know right years. Names know in a letters. High varieties ought to undergo successful, immed 0.38 5.94 2.25 edu packunivamalg #x +able Effective sales see always good, ambitious symptoms. Particular, early needs sa 2.18 0.42 0.90 amalgscholar #x +able Elsewhere certain protests should work together still strange cou 0.21 8.48 6.69 importounivamalg #x +able Elsewhere free contacts would take well from a aspects. Trying, glad economies must begin equal emissions. Settlements used to sh 2.24 29.47 1.86 amalgscholar #x +able Ever l 2.22 1.22 0.97 exportiexporti #x +able F 0.97 4.46 0.85 edu packamalg #x +able Facilities learn makers. Large, aware objections would appeal later. Ill explanations must tr 3.83 9.89 0.42 univunivamalg #x +able Far, small levels get then public computers. Far experienced newspapers divide eventually in a careers. Objectives see black plants. Modern wome 3.13 78.78 4.91 amalgexporti #x +able Final activities hope alongside p 2.55 7.90 4.26 importoedu pack #x +able Fine old programmes may not mean. Public pp. may not believe however thus continued lines. Physical agenc 1.03 1.74 2.38 importomaxi #x +able Firmly local premises shall not long recently empty minutes. Bad, individual nations will not choose als 2.76 4.06 1.90 edu packcorp #x +able Forms fetch always new arms. New friends will make successfully journals. Available players take now police. Ulti 1.33 8.29 5.88 exportiexporti #x +able Forward, welsh comments look social, minor conditions. Related soldiers should not go difficult 2.91 2.57 0.43 exportiedu pack #x +able Forwards different yards might reflect perhaps unfair woo 2.53 68.85 53.01 exportiamalgamalg #x +able Full, new holders follow keen re 3.47 0.50 0.22 amalgunivamalg #x +able Highly level children handle hardly young, pleasant periods; environmental, irrelevant advantages can lead as with a times. Here legal arr 0.29 4.81 2.45 univmaxi #x +able Horizontal nerves will not study just. Issues shall not imagine workshops. Relevant industries provide british, fresh others. Commercial, new houses give with the 1.85 7.52 6.31 namelessunivamalg #x +able However thick blocks could not learn also medical fingers. Like, european 0.93 3.14 7.21 exportiamalg #x +able Implicit, likely nati 2.31 9.93 6.85 edu packscholar #x +able Important children vote so. Answers apply nevertheless free problems. Years hope easily united police. 0.52 0.29 0.09 exportibrand #x +able Individuals identify then speakers. Parts shall not pay most causes. Bottom, main clothes might not come somewhere dark, major calculations. Elections like italian purposes. Chinese action 3.13 2.10 1.55 corpbrand #x +able Industrial students run communities. Home old differences change soon. There new tale 0.00 4.05 1.66 importonameless #x +able Influential, new heels conceal. Free features will pop then. Very royal interests bring both low sources. So 2.34 4.18 2.09 edu packedu pack #x +able Intentionally poor generations can benefit alone. Royal, new regions must see gloomily important, nearby functions. Unemployed, personal efforts reduce completely from a terms 0.65 5.35 3.10 edu packbrand #x +able Invariably new goods want small, necessary surveys 0.96 6.35 3.93 brandcorp #x +able Irish, close variations used to get years. Royal arts see increasingly hundreds. Empty trials prefer again nearly open ways; already human materials seize even 1.34 1.39 0.58 exportiedu pack #x +able Keenly managerial cases will know other, darling respects. Beds purchase around concer 2.60 1.31 0.96 brandbrand #x +able Keys should meet parties. Ministers leave members. Small, new students may take always individual letters. Video-taped levels think russian ingredients. Evident pieces secure merely biological, safe c 0.75 1.63 1.23 namelessbrand #x +able Lists cannot continue rapidly goods. Improved, original applications say on 2.22 5.47 3.50 brandunivamalg #x +able Loc 1.96 0.77 0.39 amalgexporti #x +able Long-term services increase sure existing, poor progr 0.59 1.92 0.67 univbrand #x +able Low temperatures would not browse houses; cultural stations cann 3.28 6.76 4.25 importoscholar #x +able Main arrangements support red, chief othe 0.18 1.91 1.08 exportiexporti #x +able Makers recognize effectively alone operations; properties forget very 2.84 3.18 4.92 edu packexporti #x +able Male, human sales perform studies. Dependent, hostile services answer principles. Formal men show 3.55 8.66 2.68 amalgimporto #x +able Millions give like the models. To 1.56 7.43 5.57 amalgedu pack #x +able Minutes ought to carry about a operations. Also hidden terms zero approximately please local borders. Traditionally harsh pieces could not remain too patt 0.65 5.76 1.55 amalgedu pack #x +able Modern men would not ask girls. Often p 0.30 4.00 2.28 importoamalgamalg #x +able NULL 0.53 1.32 NULL importobrand #x +able National women find major, able shows. Direct visitors must not want indian clothes. Years must run slowly in the costs. Months mak 0.47 8.93 2.85 scholarunivamalg #x +able Nerves may provide most to a representatives. Sad, available mechanisms give foreign projects. Loans imagine once recent, biological facilities. Economic directors 0.44 3.39 2.67 corpunivamalg #x +able New markets come faintly different readers; possible, right claims should apply most future, elderly functions. Free others result deliberately efforts. 2.53 6.47 4.27 exportiimporto #x +able Nuclear orders find quite now direct materials. Ago alive markets affect often public rights; words bring actually. Available services may not vote thus to a functions; too given parties 1.71 3.67 4.78 importoedu pack #x +able Objects see often changes. Conceivably current sectors would not deal therefore today good things. Only other votes might judge great items. Certainly accessible drugs must want in a funds; past 1.19 6.02 2.91 scholarmaxi #x +able Only obvious profits produce now. Swiftly necessary times used to formulate here circles. Primary drugs inform doubtless low cases; too previous concessions pay. V 3.36 3.96 1.74 maximaxi #x +able Operations could not overlook. Pounds 2.98 2.63 36.70 amalgexporti #x +able Operations used to kill perhaps alwa 1.85 1.08 0.56 univamalgamalg #x +able Organizations work probably. Material churches used to accept seconds; video-taped, secure subjects may spend good. Designs go reliable, known pupils. Politicians force in a 1.15 1.68 1.53 exportiunivamalg #x +able Other, modern officials must not say. Sorry, major users m 0.60 0.89 0.46 edu packscholar #x +able Particular parents run. Thus slow studies must think so much as in a women. Other courses would not mind by the images. Past, high families shall pray best still 2.36 9.38 7.50 amalgscholar #x +able Patient, secure individuals ought t 3.84 29.02 9.28 edu packimporto #x +able Patiently full features speculate heavily in the r 1.60 4.94 1.91 importoimporto #x +able Physical problems could know in a courts. Too major houses sit ultimately often vast reserves. Shareholders force circumstances. Likely notions require with the years; excellent, e 2.92 4.05 3.40 corpbrand #x +able Plans follow studies. Local shoulders must present american goods. Here existing trousers ask very with the figures. Great teeth raise outside underlying objects. Appropriate 1.38 2.60 1.27 amalgexporti #x +able Political, welcome feelings bring today islamic things. Colourful democrats may take parameters. European skills must supply. Lectures borrow for example. Interesting, british even 2.05 1.82 1.84 exportiamalg #x +able Popular boots say both rough, important police. Nations steer long 0.12 5.35 6.01 amalgunivamalg #x +able Possible, old failures could stand often modern terms. Rooms might write months. Photograp 0.41 5.16 1.32 importoamalg #x +able Possibly civil stars should review in addition widely valuable letters; big visitors t 3.35 1.82 1.03 amalgimporto #x +able Pp. resign away determined, clear guests. Total agents see courts. Usually vocational things help for a contrac 3.08 3.51 2.59 maximaxi #x +able Pregnant, important times use at a times. Lim 1.46 0.68 0.72 importoamalg #x +able Prime, industr 1.11 3.43 1.13 edu packamalgamalg #x +able Properly unable movements share even in a speakers. Frames may look partl 0.41 0.88 0.73 exporticorp #x +able Proportions give suddenly usual, warm guns. Days shall control old animals. In order free courts withdraw for example machines. Never able groups find far communities. Well capital classe 2.55 7.46 3.80 exportiunivamalg #x +able Quite welsh costs agree specially results. Goth 2.12 1.83 0.82 corpunivamalg #x +able Reasonable authors cannot go relatively educational arms. Other, italian ships will not stop increasingly pp.. So civil employers 0.84 2.20 1.69 amalgamalg #x +able Reforms may not reduce slowly on a meetings. Opposite, italian boys publish much high traditions. Occasionally traditional ministers 2.06 7.20 1.25 edu packunivamalg #x +able Residents used to live royal children; deaf, sensitive observations may include ahead increased, outstandin 2.11 0.72 0.45 edu packexporti #x +able Ro 3.10 0.55 0.26 amalgimporto #x +able Simple friends receive finally from the times. Famous, 2.07 1.87 1.36 amalgedu pack #x +able Sometimes black devices shall not start now impossible students. Awful, video-taped plans would take then increasingly hos 2.03 50.51 41.41 edu packunivamalg #x +able Stones can find to a operations. Practically tough farmers accept recently. Certainly high fans know graduall 1.90 4.96 1.53 namelessbrand #x +able Strong tears should observe perhaps toxic recordings; accordingly new children turn on the mon 3.05 4.49 2.96 importoamalg #x +able Subjects put. Constant children try. Women receive categories. So subsequent forms get on a parts. Concessions mean with a ships. Cases know by a relations. Key 0.75 7.12 4.69 importoamalg #x +able Substantially other streets will attack to a hundreds. Physical minerals shall 2.46 4.53 0.44 edu packedu pack #x +able Systems must rear very conditions. Gove 2.94 12.92 1.67 exportiunivamalg #x +able Terms know members. 0.00 1.79 5.57 amalgscholar #x +able There superb accidents may strike individual results. Quiet, only forests drop as little unlikely towns. Observations can discern with a points. Substantial banks dest 1.94 0.88 0.61 amalgunivamalg #x +able Too other events might not quote police. Social, technical 3.91 8.18 1.97 edu packexporti #x +able Too particular sites look regularly catholic spots; subjects drive in a children. Cheeks exist now specific lights. Average forces will max 1.91 3.75 2.73 brandbrand #x +able Tory trains tell only old, good supporters. Patterns like only nations. Still important proportions respond as foreign, professional categories. Studies might develop onwards certain 3.09 3.04 2.52 importobrand #x diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query66.sql b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query66.sql new file mode 100644 index 00000000000..94cd96a3157 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query66.sql @@ -0,0 +1,220 @@ +-- start query 1 in stream 0 using template query66.tpl +select + w_warehouse_name + ,w_warehouse_sq_ft + ,w_city + ,w_county + ,w_state + ,w_country + ,ship_carriers + ,year + ,sum(jan_sales) as jan_sales + ,sum(feb_sales) as feb_sales + ,sum(mar_sales) as mar_sales + ,sum(apr_sales) as apr_sales + ,sum(may_sales) as may_sales + ,sum(jun_sales) as jun_sales + ,sum(jul_sales) as jul_sales + ,sum(aug_sales) as aug_sales + ,sum(sep_sales) as sep_sales + ,sum(oct_sales) as oct_sales + ,sum(nov_sales) as nov_sales + ,sum(dec_sales) as dec_sales + ,sum(jan_sales/w_warehouse_sq_ft) as jan_sales_per_sq_foot + ,sum(feb_sales/w_warehouse_sq_ft) as feb_sales_per_sq_foot + ,sum(mar_sales/w_warehouse_sq_ft) as mar_sales_per_sq_foot + ,sum(apr_sales/w_warehouse_sq_ft) as apr_sales_per_sq_foot + ,sum(may_sales/w_warehouse_sq_ft) as may_sales_per_sq_foot + ,sum(jun_sales/w_warehouse_sq_ft) as jun_sales_per_sq_foot + ,sum(jul_sales/w_warehouse_sq_ft) as jul_sales_per_sq_foot + ,sum(aug_sales/w_warehouse_sq_ft) as aug_sales_per_sq_foot + ,sum(sep_sales/w_warehouse_sq_ft) as sep_sales_per_sq_foot + ,sum(oct_sales/w_warehouse_sq_ft) as oct_sales_per_sq_foot + ,sum(nov_sales/w_warehouse_sq_ft) as nov_sales_per_sq_foot + ,sum(dec_sales/w_warehouse_sq_ft) as dec_sales_per_sq_foot + ,sum(jan_net) as jan_net + ,sum(feb_net) as feb_net + ,sum(mar_net) as mar_net + ,sum(apr_net) as apr_net + ,sum(may_net) as may_net + ,sum(jun_net) as jun_net + ,sum(jul_net) as jul_net + ,sum(aug_net) as aug_net + ,sum(sep_net) as sep_net + ,sum(oct_net) as oct_net + ,sum(nov_net) as nov_net + ,sum(dec_net) as dec_net + from ( + select + w_warehouse_name + ,w_warehouse_sq_ft + ,w_city + ,w_county + ,w_state + ,w_country + ,concat('DIAMOND' , ',' , 'AIRBORNE') as ship_carriers + ,d_year as year + ,sum(case when d_moy = 1 + then ws_sales_price* ws_quantity else 0 end) as jan_sales + ,sum(case when d_moy = 2 + then ws_sales_price* ws_quantity else 0 end) as feb_sales + ,sum(case when d_moy = 3 + then ws_sales_price* ws_quantity else 0 end) as mar_sales + ,sum(case when d_moy = 4 + then ws_sales_price* ws_quantity else 0 end) as apr_sales + ,sum(case when d_moy = 5 + then ws_sales_price* ws_quantity else 0 end) as may_sales + ,sum(case when d_moy = 6 + then ws_sales_price* ws_quantity else 0 end) as jun_sales + ,sum(case when d_moy = 7 + then ws_sales_price* ws_quantity else 0 end) as jul_sales + ,sum(case when d_moy = 8 + then ws_sales_price* ws_quantity else 0 end) as aug_sales + ,sum(case when d_moy = 9 + then ws_sales_price* ws_quantity else 0 end) as sep_sales + ,sum(case when d_moy = 10 + then ws_sales_price* ws_quantity else 0 end) as oct_sales + ,sum(case when d_moy = 11 + then ws_sales_price* ws_quantity else 0 end) as nov_sales + ,sum(case when d_moy = 12 + then ws_sales_price* ws_quantity else 0 end) as dec_sales + ,sum(case when d_moy = 1 + then ws_net_paid_inc_tax * ws_quantity else 0 end) as jan_net + ,sum(case when d_moy = 2 + then ws_net_paid_inc_tax * ws_quantity else 0 end) as feb_net + ,sum(case when d_moy = 3 + then ws_net_paid_inc_tax * ws_quantity else 0 end) as mar_net + ,sum(case when d_moy = 4 + then ws_net_paid_inc_tax * ws_quantity else 0 end) as apr_net + ,sum(case when d_moy = 5 + then ws_net_paid_inc_tax * ws_quantity else 0 end) as may_net + ,sum(case when d_moy = 6 + then ws_net_paid_inc_tax * ws_quantity else 0 end) as jun_net + ,sum(case when d_moy = 7 + then ws_net_paid_inc_tax * ws_quantity else 0 end) as jul_net + ,sum(case when d_moy = 8 + then ws_net_paid_inc_tax * ws_quantity else 0 end) as aug_net + ,sum(case when d_moy = 9 + then ws_net_paid_inc_tax * ws_quantity else 0 end) as sep_net + ,sum(case when d_moy = 10 + then ws_net_paid_inc_tax * ws_quantity else 0 end) as oct_net + ,sum(case when d_moy = 11 + then ws_net_paid_inc_tax * ws_quantity else 0 end) as nov_net + ,sum(case when d_moy = 12 + then ws_net_paid_inc_tax * ws_quantity else 0 end) as dec_net + from + web_sales + ,warehouse + ,date_dim + ,time_dim + ,ship_mode + where + ws_warehouse_sk = w_warehouse_sk + and ws_sold_date_sk = d_date_sk + and ws_sold_time_sk = t_time_sk + and ws_ship_mode_sk = sm_ship_mode_sk + and d_year = 2002 + and t_time between 49530 and 49530+28800 + and sm_carrier in ('DIAMOND','AIRBORNE') + group by + w_warehouse_name + ,w_warehouse_sq_ft + ,w_city + ,w_county + ,w_state + ,w_country + ,d_year + union all + select + w_warehouse_name + ,w_warehouse_sq_ft + ,w_city + ,w_county + ,w_state + ,w_country + ,concat('DIAMOND' , ',' , 'AIRBORNE') as ship_carriers + ,d_year as year + ,sum(case when d_moy = 1 + then cs_ext_sales_price* cs_quantity else 0 end) as jan_sales + ,sum(case when d_moy = 2 + then cs_ext_sales_price* cs_quantity else 0 end) as feb_sales + ,sum(case when d_moy = 3 + then cs_ext_sales_price* cs_quantity else 0 end) as mar_sales + ,sum(case when d_moy = 4 + then cs_ext_sales_price* cs_quantity else 0 end) as apr_sales + ,sum(case when d_moy = 5 + then cs_ext_sales_price* cs_quantity else 0 end) as may_sales + ,sum(case when d_moy = 6 + then cs_ext_sales_price* cs_quantity else 0 end) as jun_sales + ,sum(case when d_moy = 7 + then cs_ext_sales_price* cs_quantity else 0 end) as jul_sales + ,sum(case when d_moy = 8 + then cs_ext_sales_price* cs_quantity else 0 end) as aug_sales + ,sum(case when d_moy = 9 + then cs_ext_sales_price* cs_quantity else 0 end) as sep_sales + ,sum(case when d_moy = 10 + then cs_ext_sales_price* cs_quantity else 0 end) as oct_sales + ,sum(case when d_moy = 11 + then cs_ext_sales_price* cs_quantity else 0 end) as nov_sales + ,sum(case when d_moy = 12 + then cs_ext_sales_price* cs_quantity else 0 end) as dec_sales + ,sum(case when d_moy = 1 + then cs_net_paid_inc_ship_tax * cs_quantity else 0 end) as jan_net + ,sum(case when d_moy = 2 + then cs_net_paid_inc_ship_tax * cs_quantity else 0 end) as feb_net + ,sum(case when d_moy = 3 + then cs_net_paid_inc_ship_tax * cs_quantity else 0 end) as mar_net + ,sum(case when d_moy = 4 + then cs_net_paid_inc_ship_tax * cs_quantity else 0 end) as apr_net + ,sum(case when d_moy = 5 + then cs_net_paid_inc_ship_tax * cs_quantity else 0 end) as may_net + ,sum(case when d_moy = 6 + then cs_net_paid_inc_ship_tax * cs_quantity else 0 end) as jun_net + ,sum(case when d_moy = 7 + then cs_net_paid_inc_ship_tax * cs_quantity else 0 end) as jul_net + ,sum(case when d_moy = 8 + then cs_net_paid_inc_ship_tax * cs_quantity else 0 end) as aug_net + ,sum(case when d_moy = 9 + then cs_net_paid_inc_ship_tax * cs_quantity else 0 end) as sep_net + ,sum(case when d_moy = 10 + then cs_net_paid_inc_ship_tax * cs_quantity else 0 end) as oct_net + ,sum(case when d_moy = 11 + then cs_net_paid_inc_ship_tax * cs_quantity else 0 end) as nov_net + ,sum(case when d_moy = 12 + then cs_net_paid_inc_ship_tax * cs_quantity else 0 end) as dec_net + from + catalog_sales + ,warehouse + ,date_dim + ,time_dim + ,ship_mode + where + cs_warehouse_sk = w_warehouse_sk + and cs_sold_date_sk = d_date_sk + and cs_sold_time_sk = t_time_sk + and cs_ship_mode_sk = sm_ship_mode_sk + and d_year = 2002 + and t_time between 49530 AND 49530+28800 + and sm_carrier in ('DIAMOND','AIRBORNE') + group by + w_warehouse_name + ,w_warehouse_sq_ft + ,w_city + ,w_county + ,w_state + ,w_country + ,d_year + ) x + group by + w_warehouse_name + ,w_warehouse_sq_ft + ,w_city + ,w_county + ,w_state + ,w_country + ,ship_carriers + ,year + order by w_warehouse_name + limit 100; + + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query66.sql.out b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query66.sql.out new file mode 100644 index 00000000000..00419a5b6f8 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query66.sql.out @@ -0,0 +1,227 @@ +-- Automatically generated by Gravitino Spark SQL test +-- !query +select + w_warehouse_name + ,w_warehouse_sq_ft + ,w_city + ,w_county + ,w_state + ,w_country + ,ship_carriers + ,year + ,sum(jan_sales) as jan_sales + ,sum(feb_sales) as feb_sales + ,sum(mar_sales) as mar_sales + ,sum(apr_sales) as apr_sales + ,sum(may_sales) as may_sales + ,sum(jun_sales) as jun_sales + ,sum(jul_sales) as jul_sales + ,sum(aug_sales) as aug_sales + ,sum(sep_sales) as sep_sales + ,sum(oct_sales) as oct_sales + ,sum(nov_sales) as nov_sales + ,sum(dec_sales) as dec_sales + ,sum(jan_sales/w_warehouse_sq_ft) as jan_sales_per_sq_foot + ,sum(feb_sales/w_warehouse_sq_ft) as feb_sales_per_sq_foot + ,sum(mar_sales/w_warehouse_sq_ft) as mar_sales_per_sq_foot + ,sum(apr_sales/w_warehouse_sq_ft) as apr_sales_per_sq_foot + ,sum(may_sales/w_warehouse_sq_ft) as may_sales_per_sq_foot + ,sum(jun_sales/w_warehouse_sq_ft) as jun_sales_per_sq_foot + ,sum(jul_sales/w_warehouse_sq_ft) as jul_sales_per_sq_foot + ,sum(aug_sales/w_warehouse_sq_ft) as aug_sales_per_sq_foot + ,sum(sep_sales/w_warehouse_sq_ft) as sep_sales_per_sq_foot + ,sum(oct_sales/w_warehouse_sq_ft) as oct_sales_per_sq_foot + ,sum(nov_sales/w_warehouse_sq_ft) as nov_sales_per_sq_foot + ,sum(dec_sales/w_warehouse_sq_ft) as dec_sales_per_sq_foot + ,sum(jan_net) as jan_net + ,sum(feb_net) as feb_net + ,sum(mar_net) as mar_net + ,sum(apr_net) as apr_net + ,sum(may_net) as may_net + ,sum(jun_net) as jun_net + ,sum(jul_net) as jul_net + ,sum(aug_net) as aug_net + ,sum(sep_net) as sep_net + ,sum(oct_net) as oct_net + ,sum(nov_net) as nov_net + ,sum(dec_net) as dec_net + from ( + select + w_warehouse_name + ,w_warehouse_sq_ft + ,w_city + ,w_county + ,w_state + ,w_country + ,concat('DIAMOND' , ',' , 'AIRBORNE') as ship_carriers + ,d_year as year + ,sum(case when d_moy = 1 + then ws_sales_price* ws_quantity else 0 end) as jan_sales + ,sum(case when d_moy = 2 + then ws_sales_price* ws_quantity else 0 end) as feb_sales + ,sum(case when d_moy = 3 + then ws_sales_price* ws_quantity else 0 end) as mar_sales + ,sum(case when d_moy = 4 + then ws_sales_price* ws_quantity else 0 end) as apr_sales + ,sum(case when d_moy = 5 + then ws_sales_price* ws_quantity else 0 end) as may_sales + ,sum(case when d_moy = 6 + then ws_sales_price* ws_quantity else 0 end) as jun_sales + ,sum(case when d_moy = 7 + then ws_sales_price* ws_quantity else 0 end) as jul_sales + ,sum(case when d_moy = 8 + then ws_sales_price* ws_quantity else 0 end) as aug_sales + ,sum(case when d_moy = 9 + then ws_sales_price* ws_quantity else 0 end) as sep_sales + ,sum(case when d_moy = 10 + then ws_sales_price* ws_quantity else 0 end) as oct_sales + ,sum(case when d_moy = 11 + then ws_sales_price* ws_quantity else 0 end) as nov_sales + ,sum(case when d_moy = 12 + then ws_sales_price* ws_quantity else 0 end) as dec_sales + ,sum(case when d_moy = 1 + then ws_net_paid_inc_tax * ws_quantity else 0 end) as jan_net + ,sum(case when d_moy = 2 + then ws_net_paid_inc_tax * ws_quantity else 0 end) as feb_net + ,sum(case when d_moy = 3 + then ws_net_paid_inc_tax * ws_quantity else 0 end) as mar_net + ,sum(case when d_moy = 4 + then ws_net_paid_inc_tax * ws_quantity else 0 end) as apr_net + ,sum(case when d_moy = 5 + then ws_net_paid_inc_tax * ws_quantity else 0 end) as may_net + ,sum(case when d_moy = 6 + then ws_net_paid_inc_tax * ws_quantity else 0 end) as jun_net + ,sum(case when d_moy = 7 + then ws_net_paid_inc_tax * ws_quantity else 0 end) as jul_net + ,sum(case when d_moy = 8 + then ws_net_paid_inc_tax * ws_quantity else 0 end) as aug_net + ,sum(case when d_moy = 9 + then ws_net_paid_inc_tax * ws_quantity else 0 end) as sep_net + ,sum(case when d_moy = 10 + then ws_net_paid_inc_tax * ws_quantity else 0 end) as oct_net + ,sum(case when d_moy = 11 + then ws_net_paid_inc_tax * ws_quantity else 0 end) as nov_net + ,sum(case when d_moy = 12 + then ws_net_paid_inc_tax * ws_quantity else 0 end) as dec_net + from + web_sales + ,warehouse + ,date_dim + ,time_dim + ,ship_mode + where + ws_warehouse_sk = w_warehouse_sk + and ws_sold_date_sk = d_date_sk + and ws_sold_time_sk = t_time_sk + and ws_ship_mode_sk = sm_ship_mode_sk + and d_year = 2002 + and t_time between 49530 and 49530+28800 + and sm_carrier in ('DIAMOND','AIRBORNE') + group by + w_warehouse_name + ,w_warehouse_sq_ft + ,w_city + ,w_county + ,w_state + ,w_country + ,d_year + union all + select + w_warehouse_name + ,w_warehouse_sq_ft + ,w_city + ,w_county + ,w_state + ,w_country + ,concat('DIAMOND' , ',' , 'AIRBORNE') as ship_carriers + ,d_year as year + ,sum(case when d_moy = 1 + then cs_ext_sales_price* cs_quantity else 0 end) as jan_sales + ,sum(case when d_moy = 2 + then cs_ext_sales_price* cs_quantity else 0 end) as feb_sales + ,sum(case when d_moy = 3 + then cs_ext_sales_price* cs_quantity else 0 end) as mar_sales + ,sum(case when d_moy = 4 + then cs_ext_sales_price* cs_quantity else 0 end) as apr_sales + ,sum(case when d_moy = 5 + then cs_ext_sales_price* cs_quantity else 0 end) as may_sales + ,sum(case when d_moy = 6 + then cs_ext_sales_price* cs_quantity else 0 end) as jun_sales + ,sum(case when d_moy = 7 + then cs_ext_sales_price* cs_quantity else 0 end) as jul_sales + ,sum(case when d_moy = 8 + then cs_ext_sales_price* cs_quantity else 0 end) as aug_sales + ,sum(case when d_moy = 9 + then cs_ext_sales_price* cs_quantity else 0 end) as sep_sales + ,sum(case when d_moy = 10 + then cs_ext_sales_price* cs_quantity else 0 end) as oct_sales + ,sum(case when d_moy = 11 + then cs_ext_sales_price* cs_quantity else 0 end) as nov_sales + ,sum(case when d_moy = 12 + then cs_ext_sales_price* cs_quantity else 0 end) as dec_sales + ,sum(case when d_moy = 1 + then cs_net_paid_inc_ship_tax * cs_quantity else 0 end) as jan_net + ,sum(case when d_moy = 2 + then cs_net_paid_inc_ship_tax * cs_quantity else 0 end) as feb_net + ,sum(case when d_moy = 3 + then cs_net_paid_inc_ship_tax * cs_quantity else 0 end) as mar_net + ,sum(case when d_moy = 4 + then cs_net_paid_inc_ship_tax * cs_quantity else 0 end) as apr_net + ,sum(case when d_moy = 5 + then cs_net_paid_inc_ship_tax * cs_quantity else 0 end) as may_net + ,sum(case when d_moy = 6 + then cs_net_paid_inc_ship_tax * cs_quantity else 0 end) as jun_net + ,sum(case when d_moy = 7 + then cs_net_paid_inc_ship_tax * cs_quantity else 0 end) as jul_net + ,sum(case when d_moy = 8 + then cs_net_paid_inc_ship_tax * cs_quantity else 0 end) as aug_net + ,sum(case when d_moy = 9 + then cs_net_paid_inc_ship_tax * cs_quantity else 0 end) as sep_net + ,sum(case when d_moy = 10 + then cs_net_paid_inc_ship_tax * cs_quantity else 0 end) as oct_net + ,sum(case when d_moy = 11 + then cs_net_paid_inc_ship_tax * cs_quantity else 0 end) as nov_net + ,sum(case when d_moy = 12 + then cs_net_paid_inc_ship_tax * cs_quantity else 0 end) as dec_net + from + catalog_sales + ,warehouse + ,date_dim + ,time_dim + ,ship_mode + where + cs_warehouse_sk = w_warehouse_sk + and cs_sold_date_sk = d_date_sk + and cs_sold_time_sk = t_time_sk + and cs_ship_mode_sk = sm_ship_mode_sk + and d_year = 2002 + and t_time between 49530 AND 49530+28800 + and sm_carrier in ('DIAMOND','AIRBORNE') + group by + w_warehouse_name + ,w_warehouse_sq_ft + ,w_city + ,w_county + ,w_state + ,w_country + ,d_year + ) x + group by + w_warehouse_name + ,w_warehouse_sq_ft + ,w_city + ,w_county + ,w_state + ,w_country + ,ship_carriers + ,year + order by w_warehouse_name + limit 100 +-- !query schema +struct +-- !query output +Bad cards must make. 621234 Fairview Williamson County TN United States DIAMOND,AIRBORNE 2002 6890.88 22629.34 0.00 10216.05 2761.46 1725.60 3063.60 1625.74 2379.14 2430.58 0.00 8580.75 0.011092 0.036426 0.000000 0.016445 0.004445 0.002778 0.004931 0.002617 0.003830 0.003913 0.000000 0.013812 348776.32 1951822.95 0.00 950092.65 67773.29 55391.70 120154.17 58784.82 228111.68 209030.80 0.00 483349.01 +Conventional childr 977787 Fairview Williamson County TN United States DIAMOND,AIRBORNE 2002 6823.74 0.00 2985.06 84.92 0.00 8593.20 0.00 2897.10 0.00 734.30 8036.15 5684.60 0.006979 0.000000 0.003053 0.000087 0.000000 0.008788 0.000000 0.002963 0.000000 0.000751 0.008219 0.005814 316842.82 0.00 260276.94 1942.82 0.00 619569.30 0.00 89520.30 0.00 54999.00 739073.60 345143.94 +Doors canno 294242 Fairview Williamson County TN United States DIAMOND,AIRBORNE 2002 0.00 0.00 1670.82 71.04 0.00 0.00 0.00 8315.73 843.60 6839.37 2574.12 0.00 0.000000 0.000000 0.005678 0.000241 0.000000 0.000000 0.000000 0.028262 0.002867 0.023244 0.008748 0.000000 0.00 0.00 42673.38 2455.04 0.00 0.00 0.00 381946.17 81744.65 373371.27 81116.09 0.00 +Important issues liv 138504 Fairview Williamson County TN United States DIAMOND,AIRBORNE 2002 38.06 0.00 935.29 870.46 420.42 0.00 0.00 0.00 0.00 396.18 31.46 7996.52 0.000275 0.000000 0.006753 0.006285 0.003035 0.000000 0.000000 0.000000 0.000000 0.002860 0.000227 0.057735 870.76 0.00 16965.02 8802.87 14428.59 0.00 0.00 0.00 0.00 28409.94 726.66 248977.76 +NULL NULL Fairview Williamson County TN United States DIAMOND,AIRBORNE 2002 41.31 0.00 0.00 0.00 6060.00 1908.36 0.00 1727.60 1817.28 0.00 5263.68 5524.56 NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL 42.54 0.00 0.00 0.00 648420.00 59159.16 0.00 64059.20 141311.52 0.00 175175.04 167627.60 diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query67.sql b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query67.sql new file mode 100644 index 00000000000..9337b99bfc8 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query67.sql @@ -0,0 +1,44 @@ +-- start query 1 in stream 0 using template query67.tpl +select * +from (select i_category + ,i_class + ,i_brand + ,i_product_name + ,d_year + ,d_qoy + ,d_moy + ,s_store_id + ,sumsales + ,rank() over (partition by i_category order by sumsales desc) rk + from (select i_category + ,i_class + ,i_brand + ,i_product_name + ,d_year + ,d_qoy + ,d_moy + ,s_store_id + ,sum(coalesce(ss_sales_price*ss_quantity,0)) sumsales + from store_sales + ,date_dim + ,store + ,item + where ss_sold_date_sk=d_date_sk + and ss_item_sk=i_item_sk + and ss_store_sk = s_store_sk + and d_month_seq between 1212 and 1212+11 + group by rollup(i_category, i_class, i_brand, i_product_name, d_year, d_qoy, d_moy,s_store_id))dw1) dw2 +where rk <= 100 +order by i_category + ,i_class + ,i_brand + ,i_product_name + ,d_year + ,d_qoy + ,d_moy + ,s_store_id + ,sumsales + ,rk +limit 100; + + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query67.sql.out b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query67.sql.out new file mode 100644 index 00000000000..66fc8b5e662 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query67.sql.out @@ -0,0 +1,146 @@ +-- Automatically generated by Gravitino Spark SQL test +-- !query +select * +from (select i_category + ,i_class + ,i_brand + ,i_product_name + ,d_year + ,d_qoy + ,d_moy + ,s_store_id + ,sumsales + ,rank() over (partition by i_category order by sumsales desc) rk + from (select i_category + ,i_class + ,i_brand + ,i_product_name + ,d_year + ,d_qoy + ,d_moy + ,s_store_id + ,sum(coalesce(ss_sales_price*ss_quantity,0)) sumsales + from store_sales + ,date_dim + ,store + ,item + where ss_sold_date_sk=d_date_sk + and ss_item_sk=i_item_sk + and ss_store_sk = s_store_sk + and d_month_seq between 1212 and 1212+11 + group by rollup(i_category, i_class, i_brand, i_product_name, d_year, d_qoy, d_moy,s_store_id))dw1) dw2 +where rk <= 100 +order by i_category + ,i_class + ,i_brand + ,i_product_name + ,d_year + ,d_qoy + ,d_moy + ,s_store_id + ,sumsales + ,rk +limit 100 +-- !query schema +struct +-- !query output +Books NULL NULL NULL NULL NULL NULL NULL 957901.70 1 +Books arts NULL NULL NULL NULL NULL NULL 47372.33 14 +Books arts amalgmaxi #x NULL NULL NULL NULL NULL 14893.34 46 +Books arts amalgmaxi #x NULL NULL NULL NULL NULL 15513.21 42 +Books arts amalgmaxi #x ationcallyationpriought 2001 NULL NULL NULL 12472.82 63 +Books arts amalgmaxi #x ationcallyationpriought NULL NULL NULL NULL 12472.82 63 +Books business NULL NULL NULL NULL NULL NULL 48984.76 12 +Books business importomaxi #x NULL NULL NULL NULL NULL 15308.92 44 +Books computers NULL NULL NULL NULL NULL NULL 58034.34 11 +Books computers exportimaxi #x NULL NULL NULL NULL NULL 15428.86 43 +Books computers exportimaxi #x NULL NULL NULL NULL NULL 16046.53 36 +Books cooking NULL NULL NULL NULL NULL NULL 35698.18 17 +Books cooking amalgunivamalg #x NULL NULL NULL NULL NULL 17898.48 33 +Books cooking amalgunivamalg #x antioughteseation 2001 1 2 AAAAAAAAKAAAAAAA 11897.52 67 +Books cooking amalgunivamalg #x antioughteseation 2001 1 2 NULL 11897.52 67 +Books cooking amalgunivamalg #x antioughteseation 2001 1 NULL NULL 11897.52 67 +Books cooking amalgunivamalg #x antioughteseation 2001 NULL NULL NULL 11897.52 67 +Books cooking amalgunivamalg #x antioughteseation NULL NULL NULL NULL 11897.52 67 +Books entertainments NULL NULL NULL NULL NULL NULL 43986.10 15 +Books entertainments edu packmaxi #x NULL NULL NULL NULL NULL 26445.57 23 +Books entertainments edu packmaxi #x antiablebareseought 2001 4 10 AAAAAAAAKAAAAAAA 13211.38 53 +Books entertainments edu packmaxi #x antiablebareseought 2001 4 10 NULL 13211.38 53 +Books entertainments edu packmaxi #x antiablebareseought 2001 4 NULL NULL 13211.38 53 +Books entertainments edu packmaxi #x antiablebareseought 2001 NULL NULL NULL 13211.38 53 +Books entertainments edu packmaxi #x antiablebareseought NULL NULL NULL NULL 13211.38 53 +Books fiction NULL NULL NULL NULL NULL NULL 64155.75 9 +Books fiction scholarunivamalg #x NULL NULL NULL NULL NULL 20172.57 30 +Books fiction scholarunivamalg #x NULL NULL NULL NULL NULL 24580.25 26 +Books fiction scholarunivamalg #x oughtoughtcallycally 2001 1 2 AAAAAAAABAAAAAAA 11148.40 77 +Books fiction scholarunivamalg #x oughtoughtcallycally 2001 1 2 NULL 11148.40 77 +Books fiction scholarunivamalg #x oughtoughtcallycally 2001 1 NULL NULL 11148.40 77 +Books fiction scholarunivamalg #x oughtoughtcallycally 2001 NULL NULL NULL 11148.40 77 +Books fiction scholarunivamalg #x oughtoughtcallycally NULL NULL NULL NULL 11148.40 77 +Books history NULL NULL NULL NULL NULL NULL 64384.02 7 +Books history scholarmaxi #x NULL NULL NULL NULL NULL 12105.40 66 +Books history scholarmaxi #x NULL NULL NULL NULL NULL 12659.27 58 +Books history scholarmaxi #x NULL NULL NULL NULL NULL 13547.52 50 +Books history scholarmaxi #x NULL NULL NULL NULL NULL 15852.90 39 +Books home repair NULL NULL NULL NULL NULL NULL 48406.56 13 +Books home repair importounivamalg #x NULL NULL NULL NULL NULL 13877.16 49 +Books home repair importounivamalg #x NULL NULL NULL NULL NULL 14294.72 48 +Books home repair importounivamalg #x NULL NULL NULL NULL NULL 17027.45 34 +Books home repair importounivamalg #x n stationationbarought 2001 1 3 AAAAAAAAHAAAAAAA 10522.00 99 +Books home repair importounivamalg #x n stationationbarought 2001 1 3 NULL 10522.00 99 +Books home repair importounivamalg #x n stationationbarought 2001 1 NULL NULL 10522.00 99 +Books home repair importounivamalg #x n stationationbarought 2001 NULL NULL NULL 10522.00 99 +Books home repair importounivamalg #x n stationationbarought NULL NULL NULL NULL 10522.00 99 +Books mystery NULL NULL NULL NULL NULL NULL 41501.15 16 +Books parenting NULL NULL NULL NULL NULL NULL 64334.43 8 +Books parenting corpmaxi #x NULL NULL NULL NULL NULL 19389.16 31 +Books parenting corpmaxi #x NULL NULL NULL NULL NULL 21108.45 29 +Books reference NULL NULL NULL NULL NULL NULL 91233.08 2 +Books reference brandmaxi #x NULL NULL NULL NULL NULL 14576.25 47 +NULL NULL NULL NULL 2001 4 12 AAAAAAAACAAAAAAA 3606.72 5 +NULL NULL NULL NULL 2001 4 12 NULL 3606.72 5 +NULL NULL NULL NULL 2001 4 NULL NULL 3606.72 5 +NULL NULL NULL NULL 2001 NULL NULL NULL 3606.72 5 +NULL NULL NULL NULL NULL NULL NULL NULL 10783803.09 1 +NULL NULL NULL NULL NULL NULL NULL NULL 3606.72 5 +NULL NULL NULL NULL NULL NULL NULL NULL 4528.02 3 +NULL NULL NULL NULL NULL NULL NULL NULL 4528.02 3 +NULL NULL NULL NULL NULL NULL NULL NULL 7846.69 2 +NULL NULL NULL antiationeinganti 2001 1 1 AAAAAAAAKAAAAAAA 921.30 19 +NULL NULL NULL antiationeinganti 2001 1 1 NULL 921.30 19 +NULL NULL NULL antiationeinganti 2001 1 NULL NULL 921.30 19 +NULL NULL NULL antiationeinganti 2001 NULL NULL NULL 921.30 19 +NULL NULL NULL antiationeinganti NULL NULL NULL NULL 921.30 19 +NULL archery NULL NULL NULL NULL NULL NULL 1108.49 14 +NULL archery amalgmaxi #x NULL NULL NULL NULL NULL 1108.49 14 +NULL archery amalgmaxi #x antioughtn stought 2001 3 7 AAAAAAAAIAAAAAAA 701.72 27 +NULL archery amalgmaxi #x antioughtn stought 2001 3 7 NULL 701.72 27 +NULL archery amalgmaxi #x antioughtn stought 2001 3 8 AAAAAAAAHAAAAAAA 406.77 39 +NULL archery amalgmaxi #x antioughtn stought 2001 3 8 NULL 406.77 39 +NULL archery amalgmaxi #x antioughtn stought 2001 3 NULL NULL 1108.49 14 +NULL archery amalgmaxi #x antioughtn stought 2001 NULL NULL NULL 1108.49 14 +NULL archery amalgmaxi #x antioughtn stought NULL NULL NULL NULL 1108.49 14 +NULL camcorders NULL NULL 2001 4 11 AAAAAAAAEAAAAAAA 598.40 32 +NULL camcorders NULL NULL 2001 4 11 NULL 598.40 32 +NULL camcorders NULL NULL 2001 4 NULL NULL 598.40 32 +NULL camcorders NULL NULL 2001 NULL NULL NULL 598.40 32 +NULL camcorders NULL NULL NULL NULL NULL NULL 598.40 32 +NULL camcorders NULL NULL NULL NULL NULL NULL 598.40 32 +NULL camcorders NULL NULL NULL NULL NULL NULL 598.40 32 +NULL pants NULL NULL NULL NULL NULL NULL 179.84 41 +NULL pants exportiimporto #x NULL NULL NULL NULL NULL 179.84 41 +NULL pants exportiimporto #x antibarableableought 2001 3 8 AAAAAAAACAAAAAAA 179.84 41 +NULL pants exportiimporto #x antibarableableought 2001 3 8 NULL 179.84 41 +NULL pants exportiimporto #x antibarableableought 2001 3 NULL NULL 179.84 41 +NULL pants exportiimporto #x antibarableableought 2001 NULL NULL NULL 179.84 41 +NULL pants exportiimporto #x antibarableableought NULL NULL NULL NULL 179.84 41 +NULL womens NULL NULL NULL NULL NULL NULL 1431.94 10 +NULL womens amalgedu pack #x NULL 2001 3 9 AAAAAAAABAAAAAAA 645.86 29 +NULL womens amalgedu pack #x NULL 2001 3 9 NULL 645.86 29 +NULL womens amalgedu pack #x NULL 2001 3 NULL NULL 645.86 29 +NULL womens amalgedu pack #x NULL 2001 4 12 AAAAAAAAEAAAAAAA 786.08 24 +NULL womens amalgedu pack #x NULL 2001 4 12 NULL 786.08 24 +NULL womens amalgedu pack #x NULL 2001 4 NULL NULL 786.08 24 +NULL womens amalgedu pack #x NULL 2001 NULL NULL NULL 1431.94 10 +NULL womens amalgedu pack #x NULL NULL NULL NULL NULL 1431.94 10 +NULL womens amalgedu pack #x NULL NULL NULL NULL NULL 1431.94 10 diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query68.sql b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query68.sql new file mode 100644 index 00000000000..8d46fde18cc --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query68.sql @@ -0,0 +1,42 @@ +-- start query 1 in stream 0 using template query68.tpl +select c_last_name + ,c_first_name + ,ca_city + ,bought_city + ,ss_ticket_number + ,extended_price + ,extended_tax + ,list_price + from (select ss_ticket_number + ,ss_customer_sk + ,ca_city bought_city + ,sum(ss_ext_sales_price) extended_price + ,sum(ss_ext_list_price) list_price + ,sum(ss_ext_tax) extended_tax + from store_sales + ,date_dim + ,store + ,household_demographics + ,customer_address + where store_sales.ss_sold_date_sk = date_dim.d_date_sk + and store_sales.ss_store_sk = store.s_store_sk + and store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk + and store_sales.ss_addr_sk = customer_address.ca_address_sk + and date_dim.d_dom between 1 and 2 + and (household_demographics.hd_dep_count = 5 or + household_demographics.hd_vehicle_count= 3) + and date_dim.d_year in (1999,1999+1,1999+2) + and store.s_city in ('Midway','Fairview') + group by ss_ticket_number + ,ss_customer_sk + ,ss_addr_sk,ca_city) dn + ,customer + ,customer_address current_addr + where ss_customer_sk = c_customer_sk + and customer.c_current_addr_sk = current_addr.ca_address_sk + and current_addr.ca_city <> bought_city + order by c_last_name + ,ss_ticket_number + limit 100; + + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query68.sql.out b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query68.sql.out new file mode 100644 index 00000000000..5c050b99ed8 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query68.sql.out @@ -0,0 +1,47 @@ +-- Automatically generated by Gravitino Spark SQL test +-- !query +select c_last_name + ,c_first_name + ,ca_city + ,bought_city + ,ss_ticket_number + ,extended_price + ,extended_tax + ,list_price + from (select ss_ticket_number + ,ss_customer_sk + ,ca_city bought_city + ,sum(ss_ext_sales_price) extended_price + ,sum(ss_ext_list_price) list_price + ,sum(ss_ext_tax) extended_tax + from store_sales + ,date_dim + ,store + ,household_demographics + ,customer_address + where store_sales.ss_sold_date_sk = date_dim.d_date_sk + and store_sales.ss_store_sk = store.s_store_sk + and store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk + and store_sales.ss_addr_sk = customer_address.ca_address_sk + and date_dim.d_dom between 1 and 2 + and (household_demographics.hd_dep_count = 5 or + household_demographics.hd_vehicle_count= 3) + and date_dim.d_year in (1999,1999+1,1999+2) + and store.s_city in ('Midway','Fairview') + group by ss_ticket_number + ,ss_customer_sk + ,ss_addr_sk,ca_city) dn + ,customer + ,customer_address current_addr + where ss_customer_sk = c_customer_sk + and customer.c_current_addr_sk = current_addr.ca_address_sk + and current_addr.ca_city <> bought_city + order by c_last_name + ,ss_ticket_number + limit 100 +-- !query schema +struct +-- !query output +Bliss Heidi Dallas Greenfield 2013 14523.00 522.10 43001.10 +Breeden April Clearview Florence 1574 7691.88 91.58 16508.84 +NULL NULL Unionville Woodlawn 70 14472.01 383.45 27139.95 diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query69.sql b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query69.sql new file mode 100644 index 00000000000..3d3b5f3ceb6 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query69.sql @@ -0,0 +1,47 @@ +-- start query 1 in stream 0 using template query69.tpl +select + cd_gender, + cd_marital_status, + cd_education_status, + count(*) cnt1, + cd_purchase_estimate, + count(*) cnt2, + cd_credit_rating, + count(*) cnt3 + from + customer c,customer_address ca,customer_demographics + where + c.c_current_addr_sk = ca.ca_address_sk and + ca_state in ('CO','IL','MN') and + cd_demo_sk = c.c_current_cdemo_sk and + exists (select * + from store_sales,date_dim + where c.c_customer_sk = ss_customer_sk and + ss_sold_date_sk = d_date_sk and + d_year = 1999 and + d_moy between 1 and 1+2) and + (not exists (select * + from web_sales,date_dim + where c.c_customer_sk = ws_bill_customer_sk and + ws_sold_date_sk = d_date_sk and + d_year = 1999 and + d_moy between 1 and 1+2) and + not exists (select * + from catalog_sales,date_dim + where c.c_customer_sk = cs_ship_customer_sk and + cs_sold_date_sk = d_date_sk and + d_year = 1999 and + d_moy between 1 and 1+2)) + group by cd_gender, + cd_marital_status, + cd_education_status, + cd_purchase_estimate, + cd_credit_rating + order by cd_gender, + cd_marital_status, + cd_education_status, + cd_purchase_estimate, + cd_credit_rating + limit 100; + + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query69.sql.out b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query69.sql.out new file mode 100644 index 00000000000..9b9d99f5f5f --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query69.sql.out @@ -0,0 +1,50 @@ +-- Automatically generated by Gravitino Spark SQL test +-- !query +select + cd_gender, + cd_marital_status, + cd_education_status, + count(*) cnt1, + cd_purchase_estimate, + count(*) cnt2, + cd_credit_rating, + count(*) cnt3 + from + customer c,customer_address ca,customer_demographics + where + c.c_current_addr_sk = ca.ca_address_sk and + ca_state in ('CO','IL','MN') and + cd_demo_sk = c.c_current_cdemo_sk and + exists (select * + from store_sales,date_dim + where c.c_customer_sk = ss_customer_sk and + ss_sold_date_sk = d_date_sk and + d_year = 1999 and + d_moy between 1 and 1+2) and + (not exists (select * + from web_sales,date_dim + where c.c_customer_sk = ws_bill_customer_sk and + ws_sold_date_sk = d_date_sk and + d_year = 1999 and + d_moy between 1 and 1+2) and + not exists (select * + from catalog_sales,date_dim + where c.c_customer_sk = cs_ship_customer_sk and + cs_sold_date_sk = d_date_sk and + d_year = 1999 and + d_moy between 1 and 1+2)) + group by cd_gender, + cd_marital_status, + cd_education_status, + cd_purchase_estimate, + cd_credit_rating + order by cd_gender, + cd_marital_status, + cd_education_status, + cd_purchase_estimate, + cd_credit_rating + limit 100 +-- !query schema +struct +-- !query output + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query7.sql b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query7.sql new file mode 100644 index 00000000000..e7c6fed962e --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query7.sql @@ -0,0 +1,21 @@ +-- start query 1 in stream 0 using template query7.tpl +select i_item_id, + avg(ss_quantity) agg1, + avg(ss_list_price) agg2, + avg(ss_coupon_amt) agg3, + avg(ss_sales_price) agg4 + from store_sales, customer_demographics, date_dim, item, promotion + where ss_sold_date_sk = d_date_sk and + ss_item_sk = i_item_sk and + ss_cdemo_sk = cd_demo_sk and + ss_promo_sk = p_promo_sk and + cd_gender = 'F' and + cd_marital_status = 'W' and + cd_education_status = 'Primary' and + (p_channel_email = 'N' or p_channel_event = 'N') and + d_year = 1998 + group by i_item_id + order by i_item_id + limit 100; + + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query7.sql.out b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query7.sql.out new file mode 100644 index 00000000000..cbf318d423d --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query7.sql.out @@ -0,0 +1,24 @@ +-- Automatically generated by Gravitino Spark SQL test +-- !query +select i_item_id, + avg(ss_quantity) agg1, + avg(ss_list_price) agg2, + avg(ss_coupon_amt) agg3, + avg(ss_sales_price) agg4 + from store_sales, customer_demographics, date_dim, item, promotion + where ss_sold_date_sk = d_date_sk and + ss_item_sk = i_item_sk and + ss_cdemo_sk = cd_demo_sk and + ss_promo_sk = p_promo_sk and + cd_gender = 'F' and + cd_marital_status = 'W' and + cd_education_status = 'Primary' and + (p_channel_email = 'N' or p_channel_event = 'N') and + d_year = 1998 + group by i_item_id + order by i_item_id + limit 100 +-- !query schema +struct +-- !query output + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query70.sql b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query70.sql new file mode 100644 index 00000000000..bb3b965a16c --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query70.sql @@ -0,0 +1,38 @@ +-- start query 1 in stream 0 using template query70.tpl +select + sum(ss_net_profit) as total_sum + ,s_state + ,s_county + ,grouping(s_state)+grouping(s_county) as lochierarchy + ,rank() over ( + partition by grouping(s_state)+grouping(s_county), + case when grouping(s_county) = 0 then s_state end + order by sum(ss_net_profit) desc) as rank_within_parent + from + store_sales + ,date_dim d1 + ,store + where + d1.d_month_seq between 1212 and 1212+11 + and d1.d_date_sk = ss_sold_date_sk + and s_store_sk = ss_store_sk + and s_state in + ( select s_state + from (select s_state as s_state, + rank() over ( partition by s_state order by sum(ss_net_profit) desc) as ranking + from store_sales, store, date_dim + where d_month_seq between 1212 and 1212+11 + and d_date_sk = ss_sold_date_sk + and s_store_sk = ss_store_sk + group by s_state + ) tmp1 + where ranking <= 5 + ) + group by rollup(s_state,s_county) + order by + lochierarchy desc + ,case when lochierarchy = 0 then s_state end + ,rank_within_parent + limit 100; + + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query70.sql.out b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query70.sql.out new file mode 100644 index 00000000000..de840d548a2 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query70.sql.out @@ -0,0 +1,43 @@ +-- Automatically generated by Gravitino Spark SQL test +-- !query +select + sum(ss_net_profit) as total_sum + ,s_state + ,s_county + ,grouping(s_state)+grouping(s_county) as lochierarchy + ,rank() over ( + partition by grouping(s_state)+grouping(s_county), + case when grouping(s_county) = 0 then s_state end + order by sum(ss_net_profit) desc) as rank_within_parent + from + store_sales + ,date_dim d1 + ,store + where + d1.d_month_seq between 1212 and 1212+11 + and d1.d_date_sk = ss_sold_date_sk + and s_store_sk = ss_store_sk + and s_state in + ( select s_state + from (select s_state as s_state, + rank() over ( partition by s_state order by sum(ss_net_profit) desc) as ranking + from store_sales, store, date_dim + where d_month_seq between 1212 and 1212+11 + and d_date_sk = ss_sold_date_sk + and s_store_sk = ss_store_sk + group by s_state + ) tmp1 + where ranking <= 5 + ) + group by rollup(s_state,s_county) + order by + lochierarchy desc + ,case when lochierarchy = 0 then s_state end + ,rank_within_parent + limit 100 +-- !query schema +struct +-- !query output +-4485006.79 NULL NULL 2 1 +-4485006.79 TN NULL 1 1 +-4485006.79 TN Williamson County 0 1 diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query71.sql b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query71.sql new file mode 100644 index 00000000000..c172a9ca8a7 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query71.sql @@ -0,0 +1,40 @@ +-- start query 1 in stream 0 using template query71.tpl +select i_brand_id brand_id, i_brand brand,t_hour,t_minute, + sum(ext_price) ext_price + from item, (select ws_ext_sales_price as ext_price, + ws_sold_date_sk as sold_date_sk, + ws_item_sk as sold_item_sk, + ws_sold_time_sk as time_sk + from web_sales,date_dim + where d_date_sk = ws_sold_date_sk + and d_moy=12 + and d_year=2000 + union all + select cs_ext_sales_price as ext_price, + cs_sold_date_sk as sold_date_sk, + cs_item_sk as sold_item_sk, + cs_sold_time_sk as time_sk + from catalog_sales,date_dim + where d_date_sk = cs_sold_date_sk + and d_moy=12 + and d_year=2000 + union all + select ss_ext_sales_price as ext_price, + ss_sold_date_sk as sold_date_sk, + ss_item_sk as sold_item_sk, + ss_sold_time_sk as time_sk + from store_sales,date_dim + where d_date_sk = ss_sold_date_sk + and d_moy=12 + and d_year=2000 + ) tmp,time_dim + where + sold_item_sk = i_item_sk + and i_manager_id=1 + and time_sk = t_time_sk + and (t_meal_time = 'breakfast' or t_meal_time = 'dinner') + group by i_brand, i_brand_id,t_hour,t_minute + order by ext_price desc, i_brand_id + ; + + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query71.sql.out b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query71.sql.out new file mode 100644 index 00000000000..c40f89a23f8 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query71.sql.out @@ -0,0 +1,47 @@ +-- Automatically generated by Gravitino Spark SQL test +-- !query +select i_brand_id brand_id, i_brand brand,t_hour,t_minute, + sum(ext_price) ext_price + from item, (select ws_ext_sales_price as ext_price, + ws_sold_date_sk as sold_date_sk, + ws_item_sk as sold_item_sk, + ws_sold_time_sk as time_sk + from web_sales,date_dim + where d_date_sk = ws_sold_date_sk + and d_moy=12 + and d_year=2000 + union all + select cs_ext_sales_price as ext_price, + cs_sold_date_sk as sold_date_sk, + cs_item_sk as sold_item_sk, + cs_sold_time_sk as time_sk + from catalog_sales,date_dim + where d_date_sk = cs_sold_date_sk + and d_moy=12 + and d_year=2000 + union all + select ss_ext_sales_price as ext_price, + ss_sold_date_sk as sold_date_sk, + ss_item_sk as sold_item_sk, + ss_sold_time_sk as time_sk + from store_sales,date_dim + where d_date_sk = ss_sold_date_sk + and d_moy=12 + and d_year=2000 + ) tmp,time_dim + where + sold_item_sk = i_item_sk + and i_manager_id=1 + and time_sk = t_time_sk + and (t_meal_time = 'breakfast' or t_meal_time = 'dinner') + group by i_brand, i_brand_id,t_hour,t_minute + order by ext_price desc, i_brand_id +-- !query schema +struct +-- !query output +10010013 univamalgamalg #x 8 22 1221.40 +1003002 exportiamalg #x 9 11 NULL +3002001 importoexporti #x 8 31 3266.56 +3002001 importoexporti #x 9 50 488.16 +4003001 exportiedu pack #x 9 19 0.00 +4004002 edu packedu pack #x 19 52 1439.13 diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query72.sql b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query72.sql new file mode 100644 index 00000000000..02913f9d057 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query72.sql @@ -0,0 +1,29 @@ +-- start query 1 in stream 0 using template query72.tpl +select i_item_desc + ,w_warehouse_name + ,d1.d_week_seq + ,sum(case when p_promo_sk is null then 1 else 0 end) no_promo + ,sum(case when p_promo_sk is not null then 1 else 0 end) promo + ,count(*) total_cnt +from catalog_sales +join inventory on (cs_item_sk = inv_item_sk) +join warehouse on (w_warehouse_sk=inv_warehouse_sk) +join item on (i_item_sk = cs_item_sk) +join customer_demographics on (cs_bill_cdemo_sk = cd_demo_sk) +join household_demographics on (cs_bill_hdemo_sk = hd_demo_sk) +join date_dim d1 on (cs_sold_date_sk = d1.d_date_sk) +join date_dim d2 on (inv_date_sk = d2.d_date_sk) +join date_dim d3 on (cs_ship_date_sk = d3.d_date_sk) +left outer join promotion on (cs_promo_sk=p_promo_sk) +left outer join catalog_returns on (cr_item_sk = cs_item_sk and cr_order_number = cs_order_number) +where d1.d_week_seq = d2.d_week_seq + and inv_quantity_on_hand < cs_quantity + and d3.d_date > d1.d_date + 5 + and hd_buy_potential = '1001-5000' + and d1.d_year = 2001 + and cd_marital_status = 'M' +group by i_item_desc,w_warehouse_name,d1.d_week_seq +order by total_cnt desc, i_item_desc, w_warehouse_name, d_week_seq +limit 100; + + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query72.sql.out b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query72.sql.out new file mode 100644 index 00000000000..788c396a32d --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query72.sql.out @@ -0,0 +1,32 @@ +-- Automatically generated by Gravitino Spark SQL test +-- !query +select i_item_desc + ,w_warehouse_name + ,d1.d_week_seq + ,sum(case when p_promo_sk is null then 1 else 0 end) no_promo + ,sum(case when p_promo_sk is not null then 1 else 0 end) promo + ,count(*) total_cnt +from catalog_sales +join inventory on (cs_item_sk = inv_item_sk) +join warehouse on (w_warehouse_sk=inv_warehouse_sk) +join item on (i_item_sk = cs_item_sk) +join customer_demographics on (cs_bill_cdemo_sk = cd_demo_sk) +join household_demographics on (cs_bill_hdemo_sk = hd_demo_sk) +join date_dim d1 on (cs_sold_date_sk = d1.d_date_sk) +join date_dim d2 on (inv_date_sk = d2.d_date_sk) +join date_dim d3 on (cs_ship_date_sk = d3.d_date_sk) +left outer join promotion on (cs_promo_sk=p_promo_sk) +left outer join catalog_returns on (cr_item_sk = cs_item_sk and cr_order_number = cs_order_number) +where d1.d_week_seq = d2.d_week_seq + and inv_quantity_on_hand < cs_quantity + and d3.d_date > d1.d_date + 5 + and hd_buy_potential = '1001-5000' + and d1.d_year = 2001 + and cd_marital_status = 'M' +group by i_item_desc,w_warehouse_name,d1.d_week_seq +order by total_cnt desc, i_item_desc, w_warehouse_name, d_week_seq +limit 100 +-- !query schema +struct +-- !query output + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query73.sql b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query73.sql new file mode 100644 index 00000000000..8c166ed56c1 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query73.sql @@ -0,0 +1,28 @@ +-- start query 1 in stream 0 using template query73.tpl +select c_last_name + ,c_first_name + ,c_salutation + ,c_preferred_cust_flag + ,ss_ticket_number + ,cnt from + (select ss_ticket_number + ,ss_customer_sk + ,count(*) cnt + from store_sales,date_dim,store,household_demographics + where store_sales.ss_sold_date_sk = date_dim.d_date_sk + and store_sales.ss_store_sk = store.s_store_sk + and store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk + and date_dim.d_dom between 1 and 2 + and (household_demographics.hd_buy_potential = '>10000' or + household_demographics.hd_buy_potential = 'Unknown') + and household_demographics.hd_vehicle_count > 0 + and case when household_demographics.hd_vehicle_count > 0 then + household_demographics.hd_dep_count/ household_demographics.hd_vehicle_count else null end > 1 + and date_dim.d_year in (1998,1998+1,1998+2) + and store.s_county in ('Williamson County','Williamson County','Williamson County','Williamson County') + group by ss_ticket_number,ss_customer_sk) dj,customer + where ss_customer_sk = c_customer_sk + and cnt between 1 and 5 + order by cnt desc, c_last_name asc; + + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query73.sql.out b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query73.sql.out new file mode 100644 index 00000000000..3235683458f --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query73.sql.out @@ -0,0 +1,31 @@ +-- Automatically generated by Gravitino Spark SQL test +-- !query +select c_last_name + ,c_first_name + ,c_salutation + ,c_preferred_cust_flag + ,ss_ticket_number + ,cnt from + (select ss_ticket_number + ,ss_customer_sk + ,count(*) cnt + from store_sales,date_dim,store,household_demographics + where store_sales.ss_sold_date_sk = date_dim.d_date_sk + and store_sales.ss_store_sk = store.s_store_sk + and store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk + and date_dim.d_dom between 1 and 2 + and (household_demographics.hd_buy_potential = '>10000' or + household_demographics.hd_buy_potential = 'Unknown') + and household_demographics.hd_vehicle_count > 0 + and case when household_demographics.hd_vehicle_count > 0 then + household_demographics.hd_dep_count/ household_demographics.hd_vehicle_count else null end > 1 + and date_dim.d_year in (1998,1998+1,1998+2) + and store.s_county in ('Williamson County','Williamson County','Williamson County','Williamson County') + group by ss_ticket_number,ss_customer_sk) dj,customer + where ss_customer_sk = c_customer_sk + and cnt between 1 and 5 + order by cnt desc, c_last_name asc +-- !query schema +struct +-- !query output + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query74.sql b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query74.sql new file mode 100644 index 00000000000..6e894640e83 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query74.sql @@ -0,0 +1,61 @@ +-- start query 1 in stream 0 using template query74.tpl +with year_total as ( + select c_customer_id customer_id + ,c_first_name customer_first_name + ,c_last_name customer_last_name + ,d_year as year + ,max(ss_net_paid) year_total + ,'s' sale_type + from customer + ,store_sales + ,date_dim + where c_customer_sk = ss_customer_sk + and ss_sold_date_sk = d_date_sk + and d_year in (2001,2001+1) + group by c_customer_id + ,c_first_name + ,c_last_name + ,d_year + union all + select c_customer_id customer_id + ,c_first_name customer_first_name + ,c_last_name customer_last_name + ,d_year as year + ,max(ws_net_paid) year_total + ,'w' sale_type + from customer + ,web_sales + ,date_dim + where c_customer_sk = ws_bill_customer_sk + and ws_sold_date_sk = d_date_sk + and d_year in (2001,2001+1) + group by c_customer_id + ,c_first_name + ,c_last_name + ,d_year + ) + select + t_s_secyear.customer_id, t_s_secyear.customer_first_name, t_s_secyear.customer_last_name + from year_total t_s_firstyear + ,year_total t_s_secyear + ,year_total t_w_firstyear + ,year_total t_w_secyear + where t_s_secyear.customer_id = t_s_firstyear.customer_id + and t_s_firstyear.customer_id = t_w_secyear.customer_id + and t_s_firstyear.customer_id = t_w_firstyear.customer_id + and t_s_firstyear.sale_type = 's' + and t_w_firstyear.sale_type = 'w' + and t_s_secyear.sale_type = 's' + and t_w_secyear.sale_type = 'w' + and t_s_firstyear.year = 2001 + and t_s_secyear.year = 2001+1 + and t_w_firstyear.year = 2001 + and t_w_secyear.year = 2001+1 + and t_s_firstyear.year_total > 0 + and t_w_firstyear.year_total > 0 + and case when t_w_firstyear.year_total > 0 then t_w_secyear.year_total / t_w_firstyear.year_total else null end + > case when t_s_firstyear.year_total > 0 then t_s_secyear.year_total / t_s_firstyear.year_total else null end + order by 2,1,3 +limit 100; + + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query74.sql.out b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query74.sql.out new file mode 100644 index 00000000000..350f529aee1 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query74.sql.out @@ -0,0 +1,64 @@ +-- Automatically generated by Gravitino Spark SQL test +-- !query +with year_total as ( + select c_customer_id customer_id + ,c_first_name customer_first_name + ,c_last_name customer_last_name + ,d_year as year + ,max(ss_net_paid) year_total + ,'s' sale_type + from customer + ,store_sales + ,date_dim + where c_customer_sk = ss_customer_sk + and ss_sold_date_sk = d_date_sk + and d_year in (2001,2001+1) + group by c_customer_id + ,c_first_name + ,c_last_name + ,d_year + union all + select c_customer_id customer_id + ,c_first_name customer_first_name + ,c_last_name customer_last_name + ,d_year as year + ,max(ws_net_paid) year_total + ,'w' sale_type + from customer + ,web_sales + ,date_dim + where c_customer_sk = ws_bill_customer_sk + and ws_sold_date_sk = d_date_sk + and d_year in (2001,2001+1) + group by c_customer_id + ,c_first_name + ,c_last_name + ,d_year + ) + select + t_s_secyear.customer_id, t_s_secyear.customer_first_name, t_s_secyear.customer_last_name + from year_total t_s_firstyear + ,year_total t_s_secyear + ,year_total t_w_firstyear + ,year_total t_w_secyear + where t_s_secyear.customer_id = t_s_firstyear.customer_id + and t_s_firstyear.customer_id = t_w_secyear.customer_id + and t_s_firstyear.customer_id = t_w_firstyear.customer_id + and t_s_firstyear.sale_type = 's' + and t_w_firstyear.sale_type = 'w' + and t_s_secyear.sale_type = 's' + and t_w_secyear.sale_type = 'w' + and t_s_firstyear.year = 2001 + and t_s_secyear.year = 2001+1 + and t_w_firstyear.year = 2001 + and t_w_secyear.year = 2001+1 + and t_s_firstyear.year_total > 0 + and t_w_firstyear.year_total > 0 + and case when t_w_firstyear.year_total > 0 then t_w_secyear.year_total / t_w_firstyear.year_total else null end + > case when t_s_firstyear.year_total > 0 then t_s_secyear.year_total / t_s_firstyear.year_total else null end + order by 2,1,3 +limit 100 +-- !query schema +struct +-- !query output + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query75.sql b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query75.sql new file mode 100644 index 00000000000..aa351bdab40 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query75.sql @@ -0,0 +1,70 @@ +-- start query 1 in stream 0 using template query75.tpl +WITH all_sales AS ( + SELECT d_year + ,i_brand_id + ,i_class_id + ,i_category_id + ,i_manufact_id + ,SUM(sales_cnt) AS sales_cnt + ,SUM(sales_amt) AS sales_amt + FROM (SELECT d_year + ,i_brand_id + ,i_class_id + ,i_category_id + ,i_manufact_id + ,cs_quantity - COALESCE(cr_return_quantity,0) AS sales_cnt + ,cs_ext_sales_price - COALESCE(cr_return_amount,0.0) AS sales_amt + FROM catalog_sales JOIN item ON i_item_sk=cs_item_sk + JOIN date_dim ON d_date_sk=cs_sold_date_sk + LEFT JOIN catalog_returns ON (cs_order_number=cr_order_number + AND cs_item_sk=cr_item_sk) + WHERE i_category='Sports' + UNION + SELECT d_year + ,i_brand_id + ,i_class_id + ,i_category_id + ,i_manufact_id + ,ss_quantity - COALESCE(sr_return_quantity,0) AS sales_cnt + ,ss_ext_sales_price - COALESCE(sr_return_amt,0.0) AS sales_amt + FROM store_sales JOIN item ON i_item_sk=ss_item_sk + JOIN date_dim ON d_date_sk=ss_sold_date_sk + LEFT JOIN store_returns ON (ss_ticket_number=sr_ticket_number + AND ss_item_sk=sr_item_sk) + WHERE i_category='Sports' + UNION + SELECT d_year + ,i_brand_id + ,i_class_id + ,i_category_id + ,i_manufact_id + ,ws_quantity - COALESCE(wr_return_quantity,0) AS sales_cnt + ,ws_ext_sales_price - COALESCE(wr_return_amt,0.0) AS sales_amt + FROM web_sales JOIN item ON i_item_sk=ws_item_sk + JOIN date_dim ON d_date_sk=ws_sold_date_sk + LEFT JOIN web_returns ON (ws_order_number=wr_order_number + AND ws_item_sk=wr_item_sk) + WHERE i_category='Sports') sales_detail + GROUP BY d_year, i_brand_id, i_class_id, i_category_id, i_manufact_id) + SELECT prev_yr.d_year AS prev_year + ,curr_yr.d_year AS year + ,curr_yr.i_brand_id + ,curr_yr.i_class_id + ,curr_yr.i_category_id + ,curr_yr.i_manufact_id + ,prev_yr.sales_cnt AS prev_yr_cnt + ,curr_yr.sales_cnt AS curr_yr_cnt + ,curr_yr.sales_cnt-prev_yr.sales_cnt AS sales_cnt_diff + ,curr_yr.sales_amt-prev_yr.sales_amt AS sales_amt_diff + FROM all_sales curr_yr, all_sales prev_yr + WHERE curr_yr.i_brand_id=prev_yr.i_brand_id + AND curr_yr.i_class_id=prev_yr.i_class_id + AND curr_yr.i_category_id=prev_yr.i_category_id + AND curr_yr.i_manufact_id=prev_yr.i_manufact_id + AND curr_yr.d_year=2002 + AND prev_yr.d_year=2002-1 + AND CAST(curr_yr.sales_cnt AS DECIMAL(17,2))/CAST(prev_yr.sales_cnt AS DECIMAL(17,2))<0.9 + ORDER BY sales_cnt_diff + limit 100; + + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query75.sql.out b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query75.sql.out new file mode 100644 index 00000000000..c883206218c --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query75.sql.out @@ -0,0 +1,153 @@ +-- Automatically generated by Gravitino Spark SQL test +-- !query +WITH all_sales AS ( + SELECT d_year + ,i_brand_id + ,i_class_id + ,i_category_id + ,i_manufact_id + ,SUM(sales_cnt) AS sales_cnt + ,SUM(sales_amt) AS sales_amt + FROM (SELECT d_year + ,i_brand_id + ,i_class_id + ,i_category_id + ,i_manufact_id + ,cs_quantity - COALESCE(cr_return_quantity,0) AS sales_cnt + ,cs_ext_sales_price - COALESCE(cr_return_amount,0.0) AS sales_amt + FROM catalog_sales JOIN item ON i_item_sk=cs_item_sk + JOIN date_dim ON d_date_sk=cs_sold_date_sk + LEFT JOIN catalog_returns ON (cs_order_number=cr_order_number + AND cs_item_sk=cr_item_sk) + WHERE i_category='Sports' + UNION + SELECT d_year + ,i_brand_id + ,i_class_id + ,i_category_id + ,i_manufact_id + ,ss_quantity - COALESCE(sr_return_quantity,0) AS sales_cnt + ,ss_ext_sales_price - COALESCE(sr_return_amt,0.0) AS sales_amt + FROM store_sales JOIN item ON i_item_sk=ss_item_sk + JOIN date_dim ON d_date_sk=ss_sold_date_sk + LEFT JOIN store_returns ON (ss_ticket_number=sr_ticket_number + AND ss_item_sk=sr_item_sk) + WHERE i_category='Sports' + UNION + SELECT d_year + ,i_brand_id + ,i_class_id + ,i_category_id + ,i_manufact_id + ,ws_quantity - COALESCE(wr_return_quantity,0) AS sales_cnt + ,ws_ext_sales_price - COALESCE(wr_return_amt,0.0) AS sales_amt + FROM web_sales JOIN item ON i_item_sk=ws_item_sk + JOIN date_dim ON d_date_sk=ws_sold_date_sk + LEFT JOIN web_returns ON (ws_order_number=wr_order_number + AND ws_item_sk=wr_item_sk) + WHERE i_category='Sports') sales_detail + GROUP BY d_year, i_brand_id, i_class_id, i_category_id, i_manufact_id) + SELECT prev_yr.d_year AS prev_year + ,curr_yr.d_year AS year + ,curr_yr.i_brand_id + ,curr_yr.i_class_id + ,curr_yr.i_category_id + ,curr_yr.i_manufact_id + ,prev_yr.sales_cnt AS prev_yr_cnt + ,curr_yr.sales_cnt AS curr_yr_cnt + ,curr_yr.sales_cnt-prev_yr.sales_cnt AS sales_cnt_diff + ,curr_yr.sales_amt-prev_yr.sales_amt AS sales_amt_diff + FROM all_sales curr_yr, all_sales prev_yr + WHERE curr_yr.i_brand_id=prev_yr.i_brand_id + AND curr_yr.i_class_id=prev_yr.i_class_id + AND curr_yr.i_category_id=prev_yr.i_category_id + AND curr_yr.i_manufact_id=prev_yr.i_manufact_id + AND curr_yr.d_year=2002 + AND prev_yr.d_year=2002-1 + AND CAST(curr_yr.sales_cnt AS DECIMAL(17,2))/CAST(prev_yr.sales_cnt AS DECIMAL(17,2))<0.9 + ORDER BY sales_cnt_diff + limit 100 +-- !query schema +struct +-- !query output +2001 2002 1001001 1 8 146 36 9 -27 -517.15 +2001 2002 1002001 6 8 308 142 7 -135 -882.53 +2001 2002 1004001 3 8 279 153 31 -122 -9961.07 +2001 2002 3002001 2 8 350 62 40 -22 499.10 +2001 2002 3004001 14 8 186 103 63 -40 -2471.97 +2001 2002 4001001 1 8 417 163 99 -64 -16099.28 +2001 2002 4001001 13 8 110 157 89 -68 1080.24 +2001 2002 4002001 2 8 534 147 74 -73 -18167.77 +2001 2002 4004001 4 8 278 142 19 -123 -1688.76 +2001 2002 5004001 11 8 340 47 7 -40 -3417.97 +2001 2002 5004001 4 8 380 66 21 -45 -231.15 +2001 2002 6003005 3 8 9 81 31 -50 -2347.97 +2001 2002 7005005 5 8 781 87 32 -55 -4724.10 +2001 2002 8001002 1 8 113 91 42 -49 709.52 +2001 2002 8001006 1 8 805 45 14 -31 739.71 +2001 2002 8001008 1 8 11 63 12 -51 -2281.57 +2001 2002 8001008 1 8 321 40 15 -25 198.48 +2001 2002 8001010 1 8 20 113 48 -65 -1689.07 +2001 2002 8002004 2 8 50 96 66 -30 -26.76 +2001 2002 8002004 2 8 896 150 66 -84 -6551.76 +2001 2002 8002008 2 8 272 162 80 -82 -334.20 +2001 2002 8002008 2 8 279 73 43 -30 168.26 +2001 2002 8002008 2 8 649 73 46 -27 1656.23 +2001 2002 8002008 4 8 91 156 36 -120 -4662.88 +2001 2002 8003004 3 8 168 80 38 -42 -4024.86 +2001 2002 8003004 3 8 264 43 14 -29 -519.15 +2001 2002 8003010 3 8 606 44 18 -26 84.38 +2001 2002 8004006 1 8 207 97 19 -78 -710.34 +2001 2002 8004010 4 8 173 117 30 -87 -2818.20 +2001 2002 8005008 5 8 864 100 72 -28 -7763.06 +2001 2002 8005010 5 8 225 95 16 -79 -7616.85 +2001 2002 8006008 6 8 263 199 45 -154 -12389.88 +2001 2002 8006010 6 8 490 81 14 -67 174.40 +2001 2002 8007002 7 8 256 41 29 -12 3245.54 +2001 2002 8007004 7 8 233 87 44 -43 -2137.62 +2001 2002 8007004 7 8 469 75 15 -60 -767.40 +2001 2002 8007004 7 8 621 72 56 -16 -154.37 +2001 2002 8007006 7 8 973 120 63 -57 -2859.48 +2001 2002 8007007 11 8 290 73 48 -25 -172.77 +2001 2002 8007008 7 8 69 236 194 -42 -2099.21 +2001 2002 8007010 10 8 142 172 77 -95 -12135.86 +2001 2002 8007010 7 8 774 8 2 -6 101.04 +2001 2002 8008002 8 8 500 199 60 -139 -3942.11 +2001 2002 8008004 8 8 208 79 55 -24 -687.19 +2001 2002 8008008 8 8 246 47 12 -35 -3373.95 +2001 2002 8008010 3 8 495 131 24 -107 -8525.60 +2001 2002 8008010 8 8 120 99 77 -22 -443.74 +2001 2002 8009004 9 8 258 99 29 -70 174.32 +2001 2002 8009004 9 8 582 121 4 -117 -4514.94 +2001 2002 8009008 9 8 273 69 16 -53 -2601.78 +2001 2002 8010002 10 8 874 100 79 -21 996.10 +2001 2002 8010002 2 8 456 79 9 -70 -1440.61 +2001 2002 8010004 10 8 343 61 49 -12 -2469.25 +2001 2002 8010005 2 8 19 95 56 -39 -1125.90 +2001 2002 8010006 10 8 383 162 56 -106 -437.07 +2001 2002 8010006 10 8 68 71 53 -18 -14004.88 +2001 2002 8010010 10 8 402 66 21 -45 -2182.26 +2001 2002 8010010 2 8 390 49 44 -5 -3661.10 +2001 2002 8011002 11 8 415 104 36 -68 -3571.08 +2001 2002 8011006 11 8 357 124 5 -119 -2958.79 +2001 2002 8011010 11 8 365 139 62 -77 -2642.33 +2001 2002 8012002 12 8 122 44 33 -11 2387.22 +2001 2002 8012006 12 8 821 215 52 -163 -13895.24 +2001 2002 8012008 12 8 64 69 7 -62 -2941.73 +2001 2002 8012008 12 8 82 168 72 -96 -6743.93 +2001 2002 8013002 3 8 290 53 12 -41 -832.89 +2001 2002 8013004 13 8 27 152 22 -130 251.68 +2001 2002 8013004 13 8 44 193 2 -191 -6883.92 +2001 2002 8014002 14 8 487 137 51 -86 352.05 +2001 2002 8014004 14 8 659 89 78 -11 2696.21 +2001 2002 8015004 15 8 283 128 36 -92 -5550.49 +2001 2002 8015008 15 8 964 80 16 -64 -2833.28 +2001 2002 8015008 2 8 253 95 36 -59 -396.46 +2001 2002 8016001 10 8 585 77 5 -72 -5623.63 +2001 2002 8016002 16 8 496 140 50 -90 -8472.05 +2001 2002 8016002 16 8 556 42 28 -14 414.21 +2001 2002 8016004 16 8 11 43 30 -13 NULL +2001 2002 8016009 16 8 33 24 21 -3 -1626.63 +2001 2002 8016010 1 8 365 78 18 -60 -3433.26 +2001 2002 8016010 16 8 433 61 14 -47 -5990.11 +2001 2002 9005009 16 8 375 141 88 -53 -5242.16 diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query76.sql b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query76.sql new file mode 100644 index 00000000000..73ab36b02f7 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query76.sql @@ -0,0 +1,24 @@ +-- start query 1 in stream 0 using template query76.tpl +select channel, col_name, d_year, d_qoy, i_category, COUNT(*) sales_cnt, SUM(ext_sales_price) sales_amt FROM ( + SELECT 'store' as channel, 'ss_addr_sk' col_name, d_year, d_qoy, i_category, ss_ext_sales_price ext_sales_price + FROM store_sales, item, date_dim + WHERE ss_addr_sk IS NULL + AND ss_sold_date_sk=d_date_sk + AND ss_item_sk=i_item_sk + UNION ALL + SELECT 'web' as channel, 'ws_web_page_sk' col_name, d_year, d_qoy, i_category, ws_ext_sales_price ext_sales_price + FROM web_sales, item, date_dim + WHERE ws_web_page_sk IS NULL + AND ws_sold_date_sk=d_date_sk + AND ws_item_sk=i_item_sk + UNION ALL + SELECT 'catalog' as channel, 'cs_warehouse_sk' col_name, d_year, d_qoy, i_category, cs_ext_sales_price ext_sales_price + FROM catalog_sales, item, date_dim + WHERE cs_warehouse_sk IS NULL + AND cs_sold_date_sk=d_date_sk + AND cs_item_sk=i_item_sk) foo +GROUP BY channel, col_name, d_year, d_qoy, i_category +ORDER BY channel, col_name, d_year, d_qoy, i_category +limit 100; + + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query76.sql.out b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query76.sql.out new file mode 100644 index 00000000000..ac8cd8de785 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query76.sql.out @@ -0,0 +1,126 @@ +-- Automatically generated by Gravitino Spark SQL test +-- !query +select channel, col_name, d_year, d_qoy, i_category, COUNT(*) sales_cnt, SUM(ext_sales_price) sales_amt FROM ( + SELECT 'store' as channel, 'ss_addr_sk' col_name, d_year, d_qoy, i_category, ss_ext_sales_price ext_sales_price + FROM store_sales, item, date_dim + WHERE ss_addr_sk IS NULL + AND ss_sold_date_sk=d_date_sk + AND ss_item_sk=i_item_sk + UNION ALL + SELECT 'web' as channel, 'ws_web_page_sk' col_name, d_year, d_qoy, i_category, ws_ext_sales_price ext_sales_price + FROM web_sales, item, date_dim + WHERE ws_web_page_sk IS NULL + AND ws_sold_date_sk=d_date_sk + AND ws_item_sk=i_item_sk + UNION ALL + SELECT 'catalog' as channel, 'cs_warehouse_sk' col_name, d_year, d_qoy, i_category, cs_ext_sales_price ext_sales_price + FROM catalog_sales, item, date_dim + WHERE cs_warehouse_sk IS NULL + AND cs_sold_date_sk=d_date_sk + AND cs_item_sk=i_item_sk) foo +GROUP BY channel, col_name, d_year, d_qoy, i_category +ORDER BY channel, col_name, d_year, d_qoy, i_category +limit 100 +-- !query schema +struct +-- !query output +catalog cs_warehouse_sk 1998 1 Books 2 1143.90 +catalog cs_warehouse_sk 1998 1 Children 2 NULL +catalog cs_warehouse_sk 1998 1 Electronics 6 3576.80 +catalog cs_warehouse_sk 1998 1 Home 1 988.68 +catalog cs_warehouse_sk 1998 1 Jewelry 3 9729.53 +catalog cs_warehouse_sk 1998 1 Men 7 10238.05 +catalog cs_warehouse_sk 1998 1 Music 3 8015.04 +catalog cs_warehouse_sk 1998 1 Shoes 3 8916.08 +catalog cs_warehouse_sk 1998 1 Sports 4 5616.97 +catalog cs_warehouse_sk 1998 1 Women 6 8803.88 +store ss_addr_sk 1998 1 Books 3 NULL +store ss_addr_sk 1998 1 Home 2 1267.36 +store ss_addr_sk 1998 1 Men 1 210.56 +store ss_addr_sk 1998 1 Music 2 3496.26 +store ss_addr_sk 1998 1 Shoes 3 NULL +store ss_addr_sk 1998 1 Sports 4 121.80 +store ss_addr_sk 1998 1 Women 4 100.10 +store ss_addr_sk 1998 2 Books 1 NULL +store ss_addr_sk 1998 2 Children 4 2148.40 +store ss_addr_sk 1998 2 Jewelry 2 NULL +store ss_addr_sk 1998 2 Men 2 292.03 +store ss_addr_sk 1998 2 Music 2 2281.29 +store ss_addr_sk 1998 2 NULL 1 NULL +store ss_addr_sk 1998 2 Shoes 2 5127.42 +store ss_addr_sk 1998 2 Sports 3 1173.18 +store ss_addr_sk 1998 2 Women 1 NULL +store ss_addr_sk 1998 3 Books 4 2590.22 +store ss_addr_sk 1998 3 Children 1 NULL +store ss_addr_sk 1998 3 Electronics 2 3338.52 +store ss_addr_sk 1998 3 Home 6 6630.14 +store ss_addr_sk 1998 3 Jewelry 2 5550.70 +store ss_addr_sk 1998 3 Men 4 1593.36 +store ss_addr_sk 1998 3 Music 1 2285.49 +store ss_addr_sk 1998 3 Shoes 3 1321.60 +store ss_addr_sk 1998 3 Sports 5 13548.48 +store ss_addr_sk 1998 3 Women 3 2297.82 +store ss_addr_sk 1998 4 Books 9 9069.62 +store ss_addr_sk 1998 4 Children 6 3548.46 +store ss_addr_sk 1998 4 Electronics 10 7146.20 +store ss_addr_sk 1998 4 Home 5 1317.95 +store ss_addr_sk 1998 4 Jewelry 3 2083.56 +store ss_addr_sk 1998 4 Men 8 3660.40 +store ss_addr_sk 1998 4 Music 6 9088.42 +store ss_addr_sk 1998 4 Shoes 7 4520.42 +store ss_addr_sk 1998 4 Sports 5 3145.52 +store ss_addr_sk 1998 4 Women 4 5555.82 +store ss_addr_sk 1999 1 Books 3 689.13 +store ss_addr_sk 1999 1 Children 2 1278.90 +store ss_addr_sk 1999 1 Electronics 2 678.96 +store ss_addr_sk 1999 1 Home 3 8186.13 +store ss_addr_sk 1999 1 Jewelry 1 NULL +store ss_addr_sk 1999 1 Men 5 3491.65 +store ss_addr_sk 1999 1 Music 4 352.63 +store ss_addr_sk 1999 1 Sports 1 3668.54 +store ss_addr_sk 1999 1 Women 2 NULL +store ss_addr_sk 1999 2 Books 3 2444.91 +store ss_addr_sk 1999 2 Children 2 NULL +store ss_addr_sk 1999 2 Electronics 4 NULL +store ss_addr_sk 1999 2 Home 1 NULL +store ss_addr_sk 1999 2 Music 2 NULL +store ss_addr_sk 1999 2 Shoes 2 NULL +store ss_addr_sk 1999 2 Sports 2 773.16 +store ss_addr_sk 1999 2 Women 1 558.90 +store ss_addr_sk 1999 3 Books 4 1639.47 +store ss_addr_sk 1999 3 Children 2 NULL +store ss_addr_sk 1999 3 Electronics 4 2834.44 +store ss_addr_sk 1999 3 Home 3 4443.66 +store ss_addr_sk 1999 3 Jewelry 3 3801.84 +store ss_addr_sk 1999 3 Men 2 9093.15 +store ss_addr_sk 1999 3 Music 2 720.18 +store ss_addr_sk 1999 3 Shoes 1 115.52 +store ss_addr_sk 1999 3 Sports 2 NULL +store ss_addr_sk 1999 3 Women 4 4014.28 +store ss_addr_sk 1999 4 Books 2 708.48 +store ss_addr_sk 1999 4 Children 8 8731.31 +store ss_addr_sk 1999 4 Electronics 8 658.00 +store ss_addr_sk 1999 4 Home 4 7686.50 +store ss_addr_sk 1999 4 Jewelry 7 308.73 +store ss_addr_sk 1999 4 Men 6 10993.38 +store ss_addr_sk 1999 4 Music 6 7654.80 +store ss_addr_sk 1999 4 Shoes 7 800.12 +store ss_addr_sk 1999 4 Sports 6 6557.85 +store ss_addr_sk 1999 4 Women 6 NULL +store ss_addr_sk 2000 1 Books 2 3503.12 +store ss_addr_sk 2000 1 Children 1 1913.04 +store ss_addr_sk 2000 1 Electronics 1 NULL +store ss_addr_sk 2000 1 Home 4 936.87 +store ss_addr_sk 2000 1 Jewelry 3 1551.23 +store ss_addr_sk 2000 1 Men 2 10.65 +store ss_addr_sk 2000 1 Music 2 680.61 +store ss_addr_sk 2000 1 Shoes 3 NULL +store ss_addr_sk 2000 1 Sports 2 65.66 +store ss_addr_sk 2000 1 Women 1 NULL +store ss_addr_sk 2000 2 Books 1 128.96 +store ss_addr_sk 2000 2 Electronics 4 2603.61 +store ss_addr_sk 2000 2 Jewelry 3 536.22 +store ss_addr_sk 2000 2 Men 3 1614.65 +store ss_addr_sk 2000 2 Music 1 1629.18 +store ss_addr_sk 2000 2 Shoes 1 8054.88 +store ss_addr_sk 2000 3 Books 2 1248.30 diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query77.sql b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query77.sql new file mode 100644 index 00000000000..fda97b35fbb --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query77.sql @@ -0,0 +1,108 @@ +-- start query 1 in stream 0 using template query77.tpl +with ss as + (select s_store_sk, + sum(ss_ext_sales_price) as sales, + sum(ss_net_profit) as profit + from store_sales, + date_dim, + store + where ss_sold_date_sk = d_date_sk + and d_date between cast('1998-08-04' as date) + and (cast('1998-08-04' as date) + interval 30 days) + and ss_store_sk = s_store_sk + group by s_store_sk) + , + sr as + (select s_store_sk, + sum(sr_return_amt) as returns, + sum(sr_net_loss) as profit_loss + from store_returns, + date_dim, + store + where sr_returned_date_sk = d_date_sk + and d_date between cast('1998-08-04' as date) + and (cast('1998-08-04' as date) + interval 30 days) + and sr_store_sk = s_store_sk + group by s_store_sk), + cs as + (select cs_call_center_sk, + sum(cs_ext_sales_price) as sales, + sum(cs_net_profit) as profit + from catalog_sales, + date_dim + where cs_sold_date_sk = d_date_sk + and d_date between cast('1998-08-04' as date) + and (cast('1998-08-04' as date) + interval 30 days) + group by cs_call_center_sk + ), + cr as + (select cr_call_center_sk, + sum(cr_return_amount) as returns, + sum(cr_net_loss) as profit_loss + from catalog_returns, + date_dim + where cr_returned_date_sk = d_date_sk + and d_date between cast('1998-08-04' as date) + and (cast('1998-08-04' as date) + interval 30 days) + group by cr_call_center_sk + ), + ws as + ( select wp_web_page_sk, + sum(ws_ext_sales_price) as sales, + sum(ws_net_profit) as profit + from web_sales, + date_dim, + web_page + where ws_sold_date_sk = d_date_sk + and d_date between cast('1998-08-04' as date) + and (cast('1998-08-04' as date) + interval 30 days) + and ws_web_page_sk = wp_web_page_sk + group by wp_web_page_sk), + wr as + (select wp_web_page_sk, + sum(wr_return_amt) as returns, + sum(wr_net_loss) as profit_loss + from web_returns, + date_dim, + web_page + where wr_returned_date_sk = d_date_sk + and d_date between cast('1998-08-04' as date) + and (cast('1998-08-04' as date) + interval 30 days) + and wr_web_page_sk = wp_web_page_sk + group by wp_web_page_sk) + select channel + , id + , sum(sales) as sales + , sum(returns) as returns + , sum(profit) as profit + from + (select 'store channel' as channel + , ss.s_store_sk as id + , sales + , coalesce(returns, 0) as returns + , (profit - coalesce(profit_loss,0)) as profit + from ss left join sr + on ss.s_store_sk = sr.s_store_sk + union all + select 'catalog channel' as channel + , cs_call_center_sk as id + , sales + , returns + , (profit - profit_loss) as profit + from cs + , cr + union all + select 'web channel' as channel + , ws.wp_web_page_sk as id + , sales + , coalesce(returns, 0) returns + , (profit - coalesce(profit_loss,0)) as profit + from ws left join wr + on ws.wp_web_page_sk = wr.wp_web_page_sk + ) x + group by rollup (channel, id) + order by channel + ,id + limit 100; + + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query77.sql.out b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query77.sql.out new file mode 100644 index 00000000000..ee5fdbbdd46 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query77.sql.out @@ -0,0 +1,149 @@ +-- Automatically generated by Gravitino Spark SQL test +-- !query +with ss as + (select s_store_sk, + sum(ss_ext_sales_price) as sales, + sum(ss_net_profit) as profit + from store_sales, + date_dim, + store + where ss_sold_date_sk = d_date_sk + and d_date between cast('1998-08-04' as date) + and (cast('1998-08-04' as date) + interval 30 days) + and ss_store_sk = s_store_sk + group by s_store_sk) + , + sr as + (select s_store_sk, + sum(sr_return_amt) as returns, + sum(sr_net_loss) as profit_loss + from store_returns, + date_dim, + store + where sr_returned_date_sk = d_date_sk + and d_date between cast('1998-08-04' as date) + and (cast('1998-08-04' as date) + interval 30 days) + and sr_store_sk = s_store_sk + group by s_store_sk), + cs as + (select cs_call_center_sk, + sum(cs_ext_sales_price) as sales, + sum(cs_net_profit) as profit + from catalog_sales, + date_dim + where cs_sold_date_sk = d_date_sk + and d_date between cast('1998-08-04' as date) + and (cast('1998-08-04' as date) + interval 30 days) + group by cs_call_center_sk + ), + cr as + (select cr_call_center_sk, + sum(cr_return_amount) as returns, + sum(cr_net_loss) as profit_loss + from catalog_returns, + date_dim + where cr_returned_date_sk = d_date_sk + and d_date between cast('1998-08-04' as date) + and (cast('1998-08-04' as date) + interval 30 days) + group by cr_call_center_sk + ), + ws as + ( select wp_web_page_sk, + sum(ws_ext_sales_price) as sales, + sum(ws_net_profit) as profit + from web_sales, + date_dim, + web_page + where ws_sold_date_sk = d_date_sk + and d_date between cast('1998-08-04' as date) + and (cast('1998-08-04' as date) + interval 30 days) + and ws_web_page_sk = wp_web_page_sk + group by wp_web_page_sk), + wr as + (select wp_web_page_sk, + sum(wr_return_amt) as returns, + sum(wr_net_loss) as profit_loss + from web_returns, + date_dim, + web_page + where wr_returned_date_sk = d_date_sk + and d_date between cast('1998-08-04' as date) + and (cast('1998-08-04' as date) + interval 30 days) + and wr_web_page_sk = wp_web_page_sk + group by wp_web_page_sk) + select channel + , id + , sum(sales) as sales + , sum(returns) as returns + , sum(profit) as profit + from + (select 'store channel' as channel + , ss.s_store_sk as id + , sales + , coalesce(returns, 0) as returns + , (profit - coalesce(profit_loss,0)) as profit + from ss left join sr + on ss.s_store_sk = sr.s_store_sk + union all + select 'catalog channel' as channel + , cs_call_center_sk as id + , sales + , returns + , (profit - profit_loss) as profit + from cs + , cr + union all + select 'web channel' as channel + , ws.wp_web_page_sk as id + , sales + , coalesce(returns, 0) returns + , (profit - coalesce(profit_loss,0)) as profit + from ws left join wr + on ws.wp_web_page_sk = wr.wp_web_page_sk + ) x + group by rollup (channel, id) + order by channel + ,id + limit 100 +-- !query schema +struct +-- !query output +NULL NULL 1073038.40 107655.02 -423054.27 +store channel 1 72083.73 8104.15 -46175.32 +store channel 10 114736.79 3082.76 -71042.45 +store channel 2 82495.65 2323.02 -42419.17 +store channel 4 137775.87 4355.14 -62412.45 +store channel 7 92283.44 2857.84 -37776.34 +store channel 8 215863.27 1415.13 -73034.57 +store channel NULL 715238.75 22138.04 -332860.30 +web channel 1 10562.51 77.64 -2673.30 +web channel 10 10801.35 1297.48 2824.82 +web channel 13 34815.44 9473.25 1614.60 +web channel 14 5431.79 1812.37 186.81 +web channel 16 21371.89 11.52 5424.48 +web channel 19 10439.66 2864.25 -4739.74 +web channel 2 5530.88 0.00 -1408.72 +web channel 20 6437.96 4339.50 -12766.91 +web channel 22 11333.57 2604.70 2192.87 +web channel 25 19351.49 0.00 -7804.65 +web channel 26 3658.06 5023.68 -3504.07 +web channel 28 5611.14 62.40 -688.04 +web channel 31 2645.58 4987.36 -765.72 +web channel 32 11956.20 2463.81 -6361.51 +web channel 34 6109.59 2283.70 -328.83 +web channel 37 25890.99 995.32 4090.06 +web channel 38 18795.16 303.80 -1620.01 +web channel 4 661.57 0.00 -361.27 +web channel 40 13069.37 3627.48 -3153.28 +web channel 43 6329.56 27.04 -8828.26 +web channel 44 30668.64 2694.87 -71.37 +web channel 46 3331.97 1582.02 -120.51 +web channel 49 1006.26 0.00 -10204.29 +web channel 50 13469.80 9843.69 -16661.70 +web channel 52 17368.43 9031.65 -13343.03 +web channel 55 2280.72 10922.91 -5933.85 +web channel 56 4761.93 0.00 143.21 +web channel 58 9182.32 8392.30 -3532.39 +web channel 7 31393.85 0.00 3862.05 +web channel 8 13531.97 794.24 -5661.42 +web channel NULL 357799.65 85516.98 -90193.97 diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query78.sql b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query78.sql new file mode 100644 index 00000000000..55f8c160d30 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query78.sql @@ -0,0 +1,58 @@ +-- start query 1 in stream 0 using template query78.tpl +with ws as + (select d_year AS ws_sold_year, ws_item_sk, + ws_bill_customer_sk ws_customer_sk, + sum(ws_quantity) ws_qty, + sum(ws_wholesale_cost) ws_wc, + sum(ws_sales_price) ws_sp + from web_sales + left join web_returns on wr_order_number=ws_order_number and ws_item_sk=wr_item_sk + join date_dim on ws_sold_date_sk = d_date_sk + where wr_order_number is null + group by d_year, ws_item_sk, ws_bill_customer_sk + ), +cs as + (select d_year AS cs_sold_year, cs_item_sk, + cs_bill_customer_sk cs_customer_sk, + sum(cs_quantity) cs_qty, + sum(cs_wholesale_cost) cs_wc, + sum(cs_sales_price) cs_sp + from catalog_sales + left join catalog_returns on cr_order_number=cs_order_number and cs_item_sk=cr_item_sk + join date_dim on cs_sold_date_sk = d_date_sk + where cr_order_number is null + group by d_year, cs_item_sk, cs_bill_customer_sk + ), +ss as + (select d_year AS ss_sold_year, ss_item_sk, + ss_customer_sk, + sum(ss_quantity) ss_qty, + sum(ss_wholesale_cost) ss_wc, + sum(ss_sales_price) ss_sp + from store_sales + left join store_returns on sr_ticket_number=ss_ticket_number and ss_item_sk=sr_item_sk + join date_dim on ss_sold_date_sk = d_date_sk + where sr_ticket_number is null + group by d_year, ss_item_sk, ss_customer_sk + ) + select +ss_sold_year, ss_item_sk, ss_customer_sk, +round(ss_qty/(coalesce(ws_qty,0)+coalesce(cs_qty,0)),2) ratio, +ss_qty store_qty, ss_wc store_wholesale_cost, ss_sp store_sales_price, +coalesce(ws_qty,0)+coalesce(cs_qty,0) other_chan_qty, +coalesce(ws_wc,0)+coalesce(cs_wc,0) other_chan_wholesale_cost, +coalesce(ws_sp,0)+coalesce(cs_sp,0) other_chan_sales_price +from ss +left join ws on (ws_sold_year=ss_sold_year and ws_item_sk=ss_item_sk and ws_customer_sk=ss_customer_sk) +left join cs on (cs_sold_year=ss_sold_year and cs_item_sk=ss_item_sk and cs_customer_sk=ss_customer_sk) +where (coalesce(ws_qty,0)>0 or coalesce(cs_qty, 0)>0) and ss_sold_year=2000 +order by + ss_sold_year, ss_item_sk, ss_customer_sk, + ss_qty desc, ss_wc desc, ss_sp desc, + other_chan_qty, + other_chan_wholesale_cost, + other_chan_sales_price, + round(ss_qty/(coalesce(ws_qty+cs_qty,1)),2) +limit 100; + + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query78.sql.out b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query78.sql.out new file mode 100644 index 00000000000..24a5f99cb82 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query78.sql.out @@ -0,0 +1,61 @@ +-- Automatically generated by Gravitino Spark SQL test +-- !query +with ws as + (select d_year AS ws_sold_year, ws_item_sk, + ws_bill_customer_sk ws_customer_sk, + sum(ws_quantity) ws_qty, + sum(ws_wholesale_cost) ws_wc, + sum(ws_sales_price) ws_sp + from web_sales + left join web_returns on wr_order_number=ws_order_number and ws_item_sk=wr_item_sk + join date_dim on ws_sold_date_sk = d_date_sk + where wr_order_number is null + group by d_year, ws_item_sk, ws_bill_customer_sk + ), +cs as + (select d_year AS cs_sold_year, cs_item_sk, + cs_bill_customer_sk cs_customer_sk, + sum(cs_quantity) cs_qty, + sum(cs_wholesale_cost) cs_wc, + sum(cs_sales_price) cs_sp + from catalog_sales + left join catalog_returns on cr_order_number=cs_order_number and cs_item_sk=cr_item_sk + join date_dim on cs_sold_date_sk = d_date_sk + where cr_order_number is null + group by d_year, cs_item_sk, cs_bill_customer_sk + ), +ss as + (select d_year AS ss_sold_year, ss_item_sk, + ss_customer_sk, + sum(ss_quantity) ss_qty, + sum(ss_wholesale_cost) ss_wc, + sum(ss_sales_price) ss_sp + from store_sales + left join store_returns on sr_ticket_number=ss_ticket_number and ss_item_sk=sr_item_sk + join date_dim on ss_sold_date_sk = d_date_sk + where sr_ticket_number is null + group by d_year, ss_item_sk, ss_customer_sk + ) + select +ss_sold_year, ss_item_sk, ss_customer_sk, +round(ss_qty/(coalesce(ws_qty,0)+coalesce(cs_qty,0)),2) ratio, +ss_qty store_qty, ss_wc store_wholesale_cost, ss_sp store_sales_price, +coalesce(ws_qty,0)+coalesce(cs_qty,0) other_chan_qty, +coalesce(ws_wc,0)+coalesce(cs_wc,0) other_chan_wholesale_cost, +coalesce(ws_sp,0)+coalesce(cs_sp,0) other_chan_sales_price +from ss +left join ws on (ws_sold_year=ss_sold_year and ws_item_sk=ss_item_sk and ws_customer_sk=ss_customer_sk) +left join cs on (cs_sold_year=ss_sold_year and cs_item_sk=ss_item_sk and cs_customer_sk=ss_customer_sk) +where (coalesce(ws_qty,0)>0 or coalesce(cs_qty, 0)>0) and ss_sold_year=2000 +order by + ss_sold_year, ss_item_sk, ss_customer_sk, + ss_qty desc, ss_wc desc, ss_sp desc, + other_chan_qty, + other_chan_wholesale_cost, + other_chan_sales_price, + round(ss_qty/(coalesce(ws_qty+cs_qty,1)),2) +limit 100 +-- !query schema +struct +-- !query output + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query79.sql b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query79.sql new file mode 100644 index 00000000000..5c2c5958a5a --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query79.sql @@ -0,0 +1,23 @@ +-- start query 1 in stream 0 using template query79.tpl +select + c_last_name,c_first_name,substr(s_city,1,30),ss_ticket_number,amt,profit + from + (select ss_ticket_number + ,ss_customer_sk + ,store.s_city + ,sum(ss_coupon_amt) amt + ,sum(ss_net_profit) profit + from store_sales,date_dim,store,household_demographics + where store_sales.ss_sold_date_sk = date_dim.d_date_sk + and store_sales.ss_store_sk = store.s_store_sk + and store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk + and (household_demographics.hd_dep_count = 8 or household_demographics.hd_vehicle_count > 0) + and date_dim.d_dow = 1 + and date_dim.d_year in (1998,1998+1,1998+2) + and store.s_number_employees between 200 and 295 + group by ss_ticket_number,ss_customer_sk,ss_addr_sk,store.s_city) ms,customer + where ss_customer_sk = c_customer_sk + order by c_last_name,c_first_name,substr(s_city,1,30), profit +limit 100; + + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query79.sql.out b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query79.sql.out new file mode 100644 index 00000000000..154a4ffb936 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query79.sql.out @@ -0,0 +1,37 @@ +-- Automatically generated by Gravitino Spark SQL test +-- !query +select + c_last_name,c_first_name,substr(s_city,1,30),ss_ticket_number,amt,profit + from + (select ss_ticket_number + ,ss_customer_sk + ,store.s_city + ,sum(ss_coupon_amt) amt + ,sum(ss_net_profit) profit + from store_sales,date_dim,store,household_demographics + where store_sales.ss_sold_date_sk = date_dim.d_date_sk + and store_sales.ss_store_sk = store.s_store_sk + and store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk + and (household_demographics.hd_dep_count = 8 or household_demographics.hd_vehicle_count > 0) + and date_dim.d_dow = 1 + and date_dim.d_year in (1998,1998+1,1998+2) + and store.s_number_employees between 200 and 295 + group by ss_ticket_number,ss_customer_sk,ss_addr_sk,store.s_city) ms,customer + where ss_customer_sk = c_customer_sk + order by c_last_name,c_first_name,substr(s_city,1,30), profit +limit 100 +-- !query schema +struct +-- !query output +Breeden April Fairview 1574 8.71 -3718.25 +Buck John Fairview 1278 202.31 -15447.44 +Calderon Betty Midway 219 978.83 -289.14 +Dupuis Ami Midway 1409 4883.49 -3635.08 +Fortier Susanna Midway 979 5324.88 -20299.73 +Levy Jacqueline Fairview 204 0.00 -12695.11 +Murray Kathy Midway 519 0.00 -270.57 +Murray Kathy Midway 519 469.26 -10880.20 +Parks Luz Fairview 1598 1871.22 -91.45 +Rice Clifton Midway 740 132.83 -7579.82 +Stevens Gudrun Midway 2375 0.00 -4221.55 +Weber Eileen Fairview 1821 3499.56 -8047.97 diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query8.sql b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query8.sql new file mode 100644 index 00000000000..52cb5a96d1e --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query8.sql @@ -0,0 +1,108 @@ +-- start query 1 in stream 0 using template query8.tpl +select s_store_name + ,sum(ss_net_profit) + from store_sales + ,date_dim + ,store, + (select ca_zip + from ( + SELECT substr(ca_zip,1,5) ca_zip + FROM customer_address + WHERE substr(ca_zip,1,5) IN ( + '89436','30868','65085','22977','83927','77557', + '58429','40697','80614','10502','32779', + '91137','61265','98294','17921','18427', + '21203','59362','87291','84093','21505', + '17184','10866','67898','25797','28055', + '18377','80332','74535','21757','29742', + '90885','29898','17819','40811','25990', + '47513','89531','91068','10391','18846', + '99223','82637','41368','83658','86199', + '81625','26696','89338','88425','32200', + '81427','19053','77471','36610','99823', + '43276','41249','48584','83550','82276', + '18842','78890','14090','38123','40936', + '34425','19850','43286','80072','79188', + '54191','11395','50497','84861','90733', + '21068','57666','37119','25004','57835', + '70067','62878','95806','19303','18840', + '19124','29785','16737','16022','49613', + '89977','68310','60069','98360','48649', + '39050','41793','25002','27413','39736', + '47208','16515','94808','57648','15009', + '80015','42961','63982','21744','71853', + '81087','67468','34175','64008','20261', + '11201','51799','48043','45645','61163', + '48375','36447','57042','21218','41100', + '89951','22745','35851','83326','61125', + '78298','80752','49858','52940','96976', + '63792','11376','53582','18717','90226', + '50530','94203','99447','27670','96577', + '57856','56372','16165','23427','54561', + '28806','44439','22926','30123','61451', + '92397','56979','92309','70873','13355', + '21801','46346','37562','56458','28286', + '47306','99555','69399','26234','47546', + '49661','88601','35943','39936','25632', + '24611','44166','56648','30379','59785', + '11110','14329','93815','52226','71381', + '13842','25612','63294','14664','21077', + '82626','18799','60915','81020','56447', + '76619','11433','13414','42548','92713', + '70467','30884','47484','16072','38936', + '13036','88376','45539','35901','19506', + '65690','73957','71850','49231','14276', + '20005','18384','76615','11635','38177', + '55607','41369','95447','58581','58149', + '91946','33790','76232','75692','95464', + '22246','51061','56692','53121','77209', + '15482','10688','14868','45907','73520', + '72666','25734','17959','24677','66446', + '94627','53535','15560','41967','69297', + '11929','59403','33283','52232','57350', + '43933','40921','36635','10827','71286', + '19736','80619','25251','95042','15526', + '36496','55854','49124','81980','35375', + '49157','63512','28944','14946','36503', + '54010','18767','23969','43905','66979', + '33113','21286','58471','59080','13395', + '79144','70373','67031','38360','26705', + '50906','52406','26066','73146','15884', + '31897','30045','61068','45550','92454', + '13376','14354','19770','22928','97790', + '50723','46081','30202','14410','20223', + '88500','67298','13261','14172','81410', + '93578','83583','46047','94167','82564', + '21156','15799','86709','37931','74703', + '83103','23054','70470','72008','49247', + '91911','69998','20961','70070','63197', + '54853','88191','91830','49521','19454', + '81450','89091','62378','25683','61869', + '51744','36580','85778','36871','48121', + '28810','83712','45486','67393','26935', + '42393','20132','55349','86057','21309', + '80218','10094','11357','48819','39734', + '40758','30432','21204','29467','30214', + '61024','55307','74621','11622','68908', + '33032','52868','99194','99900','84936', + '69036','99149','45013','32895','59004', + '32322','14933','32936','33562','72550', + '27385','58049','58200','16808','21360', + '32961','18586','79307','15492') + intersect + select ca_zip + from (SELECT substr(ca_zip,1,5) ca_zip,count(*) cnt + FROM customer_address, customer + WHERE ca_address_sk = c_current_addr_sk and + c_preferred_cust_flag='Y' + group by ca_zip + having count(*) > 10)A1)A2) V1 + where ss_store_sk = s_store_sk + and ss_sold_date_sk = d_date_sk + and d_qoy = 1 and d_year = 2002 + and (substr(s_zip,1,2) = substr(V1.ca_zip,1,2)) + group by s_store_name + order by s_store_name + limit 100; + + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query8.sql.out b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query8.sql.out new file mode 100644 index 00000000000..b96e6313b03 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query8.sql.out @@ -0,0 +1,111 @@ +-- Automatically generated by Gravitino Spark SQL test +-- !query +select s_store_name + ,sum(ss_net_profit) + from store_sales + ,date_dim + ,store, + (select ca_zip + from ( + SELECT substr(ca_zip,1,5) ca_zip + FROM customer_address + WHERE substr(ca_zip,1,5) IN ( + '89436','30868','65085','22977','83927','77557', + '58429','40697','80614','10502','32779', + '91137','61265','98294','17921','18427', + '21203','59362','87291','84093','21505', + '17184','10866','67898','25797','28055', + '18377','80332','74535','21757','29742', + '90885','29898','17819','40811','25990', + '47513','89531','91068','10391','18846', + '99223','82637','41368','83658','86199', + '81625','26696','89338','88425','32200', + '81427','19053','77471','36610','99823', + '43276','41249','48584','83550','82276', + '18842','78890','14090','38123','40936', + '34425','19850','43286','80072','79188', + '54191','11395','50497','84861','90733', + '21068','57666','37119','25004','57835', + '70067','62878','95806','19303','18840', + '19124','29785','16737','16022','49613', + '89977','68310','60069','98360','48649', + '39050','41793','25002','27413','39736', + '47208','16515','94808','57648','15009', + '80015','42961','63982','21744','71853', + '81087','67468','34175','64008','20261', + '11201','51799','48043','45645','61163', + '48375','36447','57042','21218','41100', + '89951','22745','35851','83326','61125', + '78298','80752','49858','52940','96976', + '63792','11376','53582','18717','90226', + '50530','94203','99447','27670','96577', + '57856','56372','16165','23427','54561', + '28806','44439','22926','30123','61451', + '92397','56979','92309','70873','13355', + '21801','46346','37562','56458','28286', + '47306','99555','69399','26234','47546', + '49661','88601','35943','39936','25632', + '24611','44166','56648','30379','59785', + '11110','14329','93815','52226','71381', + '13842','25612','63294','14664','21077', + '82626','18799','60915','81020','56447', + '76619','11433','13414','42548','92713', + '70467','30884','47484','16072','38936', + '13036','88376','45539','35901','19506', + '65690','73957','71850','49231','14276', + '20005','18384','76615','11635','38177', + '55607','41369','95447','58581','58149', + '91946','33790','76232','75692','95464', + '22246','51061','56692','53121','77209', + '15482','10688','14868','45907','73520', + '72666','25734','17959','24677','66446', + '94627','53535','15560','41967','69297', + '11929','59403','33283','52232','57350', + '43933','40921','36635','10827','71286', + '19736','80619','25251','95042','15526', + '36496','55854','49124','81980','35375', + '49157','63512','28944','14946','36503', + '54010','18767','23969','43905','66979', + '33113','21286','58471','59080','13395', + '79144','70373','67031','38360','26705', + '50906','52406','26066','73146','15884', + '31897','30045','61068','45550','92454', + '13376','14354','19770','22928','97790', + '50723','46081','30202','14410','20223', + '88500','67298','13261','14172','81410', + '93578','83583','46047','94167','82564', + '21156','15799','86709','37931','74703', + '83103','23054','70470','72008','49247', + '91911','69998','20961','70070','63197', + '54853','88191','91830','49521','19454', + '81450','89091','62378','25683','61869', + '51744','36580','85778','36871','48121', + '28810','83712','45486','67393','26935', + '42393','20132','55349','86057','21309', + '80218','10094','11357','48819','39734', + '40758','30432','21204','29467','30214', + '61024','55307','74621','11622','68908', + '33032','52868','99194','99900','84936', + '69036','99149','45013','32895','59004', + '32322','14933','32936','33562','72550', + '27385','58049','58200','16808','21360', + '32961','18586','79307','15492') + intersect + select ca_zip + from (SELECT substr(ca_zip,1,5) ca_zip,count(*) cnt + FROM customer_address, customer + WHERE ca_address_sk = c_current_addr_sk and + c_preferred_cust_flag='Y' + group by ca_zip + having count(*) > 10)A1)A2) V1 + where ss_store_sk = s_store_sk + and ss_sold_date_sk = d_date_sk + and d_qoy = 1 and d_year = 2002 + and (substr(s_zip,1,2) = substr(V1.ca_zip,1,2)) + group by s_store_name + order by s_store_name + limit 100 +-- !query schema +struct +-- !query output + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query80.sql b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query80.sql new file mode 100644 index 00000000000..5a4a12d1ca6 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query80.sql @@ -0,0 +1,96 @@ +-- start query 1 in stream 0 using template query80.tpl +with ssr as + (select s_store_id as store_id, + sum(ss_ext_sales_price) as sales, + sum(coalesce(sr_return_amt, 0)) as returns, + sum(ss_net_profit - coalesce(sr_net_loss, 0)) as profit + from store_sales left outer join store_returns on + (ss_item_sk = sr_item_sk and ss_ticket_number = sr_ticket_number), + date_dim, + store, + item, + promotion + where ss_sold_date_sk = d_date_sk + and d_date between cast('1998-08-04' as date) + and (cast('1998-08-04' as date) + interval 30 days) + and ss_store_sk = s_store_sk + and ss_item_sk = i_item_sk + and i_current_price > 50 + and ss_promo_sk = p_promo_sk + and p_channel_tv = 'N' + group by s_store_id) + , + csr as + (select cp_catalog_page_id as catalog_page_id, + sum(cs_ext_sales_price) as sales, + sum(coalesce(cr_return_amount, 0)) as returns, + sum(cs_net_profit - coalesce(cr_net_loss, 0)) as profit + from catalog_sales left outer join catalog_returns on + (cs_item_sk = cr_item_sk and cs_order_number = cr_order_number), + date_dim, + catalog_page, + item, + promotion + where cs_sold_date_sk = d_date_sk + and d_date between cast('1998-08-04' as date) + and (cast('1998-08-04' as date) + interval 30 days) + and cs_catalog_page_sk = cp_catalog_page_sk + and cs_item_sk = i_item_sk + and i_current_price > 50 + and cs_promo_sk = p_promo_sk + and p_channel_tv = 'N' +group by cp_catalog_page_id) + , + wsr as + (select web_site_id, + sum(ws_ext_sales_price) as sales, + sum(coalesce(wr_return_amt, 0)) as returns, + sum(ws_net_profit - coalesce(wr_net_loss, 0)) as profit + from web_sales left outer join web_returns on + (ws_item_sk = wr_item_sk and ws_order_number = wr_order_number), + date_dim, + web_site, + item, + promotion + where ws_sold_date_sk = d_date_sk + and d_date between cast('1998-08-04' as date) + and (cast('1998-08-04' as date) + interval 30 days) + and ws_web_site_sk = web_site_sk + and ws_item_sk = i_item_sk + and i_current_price > 50 + and ws_promo_sk = p_promo_sk + and p_channel_tv = 'N' +group by web_site_id) + select channel + , id + , sum(sales) as sales + , sum(returns) as returns + , sum(profit) as profit + from + (select 'store channel' as channel + , concat('store' , store_id) as id + , sales + , returns + , profit + from ssr + union all + select 'catalog channel' as channel + , concat('catalog_page' , catalog_page_id) as id + , sales + , returns + , profit + from csr + union all + select 'web channel' as channel + , concat('web_site' , web_site_id) as id + , sales + , returns + , profit + from wsr + ) x + group by rollup (channel, id) + order by channel + ,id + limit 100; + + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query80.sql.out b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query80.sql.out new file mode 100644 index 00000000000..4d3948e5626 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query80.sql.out @@ -0,0 +1,114 @@ +-- Automatically generated by Gravitino Spark SQL test +-- !query +with ssr as + (select s_store_id as store_id, + sum(ss_ext_sales_price) as sales, + sum(coalesce(sr_return_amt, 0)) as returns, + sum(ss_net_profit - coalesce(sr_net_loss, 0)) as profit + from store_sales left outer join store_returns on + (ss_item_sk = sr_item_sk and ss_ticket_number = sr_ticket_number), + date_dim, + store, + item, + promotion + where ss_sold_date_sk = d_date_sk + and d_date between cast('1998-08-04' as date) + and (cast('1998-08-04' as date) + interval 30 days) + and ss_store_sk = s_store_sk + and ss_item_sk = i_item_sk + and i_current_price > 50 + and ss_promo_sk = p_promo_sk + and p_channel_tv = 'N' + group by s_store_id) + , + csr as + (select cp_catalog_page_id as catalog_page_id, + sum(cs_ext_sales_price) as sales, + sum(coalesce(cr_return_amount, 0)) as returns, + sum(cs_net_profit - coalesce(cr_net_loss, 0)) as profit + from catalog_sales left outer join catalog_returns on + (cs_item_sk = cr_item_sk and cs_order_number = cr_order_number), + date_dim, + catalog_page, + item, + promotion + where cs_sold_date_sk = d_date_sk + and d_date between cast('1998-08-04' as date) + and (cast('1998-08-04' as date) + interval 30 days) + and cs_catalog_page_sk = cp_catalog_page_sk + and cs_item_sk = i_item_sk + and i_current_price > 50 + and cs_promo_sk = p_promo_sk + and p_channel_tv = 'N' +group by cp_catalog_page_id) + , + wsr as + (select web_site_id, + sum(ws_ext_sales_price) as sales, + sum(coalesce(wr_return_amt, 0)) as returns, + sum(ws_net_profit - coalesce(wr_net_loss, 0)) as profit + from web_sales left outer join web_returns on + (ws_item_sk = wr_item_sk and ws_order_number = wr_order_number), + date_dim, + web_site, + item, + promotion + where ws_sold_date_sk = d_date_sk + and d_date between cast('1998-08-04' as date) + and (cast('1998-08-04' as date) + interval 30 days) + and ws_web_site_sk = web_site_sk + and ws_item_sk = i_item_sk + and i_current_price > 50 + and ws_promo_sk = p_promo_sk + and p_channel_tv = 'N' +group by web_site_id) + select channel + , id + , sum(sales) as sales + , sum(returns) as returns + , sum(profit) as profit + from + (select 'store channel' as channel + , concat('store' , store_id) as id + , sales + , returns + , profit + from ssr + union all + select 'catalog channel' as channel + , concat('catalog_page' , catalog_page_id) as id + , sales + , returns + , profit + from csr + union all + select 'web channel' as channel + , concat('web_site' , web_site_id) as id + , sales + , returns + , profit + from wsr + ) x + group by rollup (channel, id) + order by channel + ,id + limit 100 +-- !query schema +struct +-- !query output +NULL NULL 49030.95 2634.15 -25590.24 +store channel NULL 32168.48 1143.48 -12668.52 +store channel storeAAAAAAAABAAAAAAA 51.06 0.00 -218.30 +store channel storeAAAAAAAACAAAAAAA 109.69 0.00 -64.33 +store channel storeAAAAAAAAEAAAAAAA 16221.19 0.00 1758.19 +store channel storeAAAAAAAAIAAAAAAA 9828.82 1143.48 -3952.85 +store channel storeAAAAAAAAKAAAAAAA 5957.72 0.00 -10191.23 +web channel NULL 16862.47 1490.67 -12921.72 +web channel web_siteAAAAAAAAABAAAAAA 2042.77 1490.67 -418.31 +web channel web_siteAAAAAAAABAAAAAAA 174.96 0.00 -4139.28 +web channel web_siteAAAAAAAACAAAAAAA 2392.44 0.00 -1917.92 +web channel web_siteAAAAAAAAEBAAAAAA 4969.37 0.00 -5700.85 +web channel web_siteAAAAAAAAGBAAAAAA 232.20 0.00 86.00 +web channel web_siteAAAAAAAAHAAAAAAA 152.95 0.00 -375.33 +web channel web_siteAAAAAAAAJBAAAAAA 334.60 0.00 -703.85 +web channel web_siteAAAAAAAAKBAAAAAA 6563.18 0.00 247.82 diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query81.sql b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query81.sql new file mode 100644 index 00000000000..abeb9112703 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query81.sql @@ -0,0 +1,31 @@ +-- start query 1 in stream 0 using template query81.tpl +with customer_total_return as + (select cr_returning_customer_sk as ctr_customer_sk + ,ca_state as ctr_state, + sum(cr_return_amt_inc_tax) as ctr_total_return + from catalog_returns + ,date_dim + ,customer_address + where cr_returned_date_sk = d_date_sk + and d_year =1998 + and cr_returning_addr_sk = ca_address_sk + group by cr_returning_customer_sk + ,ca_state ) + select c_customer_id,c_salutation,c_first_name,c_last_name,ca_street_number,ca_street_name + ,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset + ,ca_location_type,ctr_total_return + from customer_total_return ctr1 + ,customer_address + ,customer + where ctr1.ctr_total_return > (select avg(ctr_total_return)*1.2 + from customer_total_return ctr2 + where ctr1.ctr_state = ctr2.ctr_state) + and ca_address_sk = c_current_addr_sk + and ca_state = 'IL' + and ctr1.ctr_customer_sk = c_customer_sk + order by c_customer_id,c_salutation,c_first_name,c_last_name,ca_street_number,ca_street_name + ,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset + ,ca_location_type,ctr_total_return + limit 100; + + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query81.sql.out b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query81.sql.out new file mode 100644 index 00000000000..078418d6d21 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query81.sql.out @@ -0,0 +1,34 @@ +-- Automatically generated by Gravitino Spark SQL test +-- !query +with customer_total_return as + (select cr_returning_customer_sk as ctr_customer_sk + ,ca_state as ctr_state, + sum(cr_return_amt_inc_tax) as ctr_total_return + from catalog_returns + ,date_dim + ,customer_address + where cr_returned_date_sk = d_date_sk + and d_year =1998 + and cr_returning_addr_sk = ca_address_sk + group by cr_returning_customer_sk + ,ca_state ) + select c_customer_id,c_salutation,c_first_name,c_last_name,ca_street_number,ca_street_name + ,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset + ,ca_location_type,ctr_total_return + from customer_total_return ctr1 + ,customer_address + ,customer + where ctr1.ctr_total_return > (select avg(ctr_total_return)*1.2 + from customer_total_return ctr2 + where ctr1.ctr_state = ctr2.ctr_state) + and ca_address_sk = c_current_addr_sk + and ca_state = 'IL' + and ctr1.ctr_customer_sk = c_customer_sk + order by c_customer_id,c_salutation,c_first_name,c_last_name,ca_street_number,ca_street_name + ,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset + ,ca_location_type,ctr_total_return + limit 100 +-- !query schema +struct +-- !query output + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query82.sql b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query82.sql new file mode 100644 index 00000000000..20eb4fa3684 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query82.sql @@ -0,0 +1,17 @@ +-- start query 1 in stream 0 using template query82.tpl +select i_item_id + ,i_item_desc + ,i_current_price + from item, inventory, date_dim, store_sales + where i_current_price between 30 and 30+30 + and inv_item_sk = i_item_sk + and d_date_sk=inv_date_sk + and d_date between cast('2002-05-30' as date) and (cast('2002-05-30' as date) + interval 60 days) + and i_manufact_id in (437,129,727,663) + and inv_quantity_on_hand between 100 and 500 + and ss_item_sk = i_item_sk + group by i_item_id,i_item_desc,i_current_price + order by i_item_id + limit 100; + + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query82.sql.out b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query82.sql.out new file mode 100644 index 00000000000..3b938756d14 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query82.sql.out @@ -0,0 +1,20 @@ +-- Automatically generated by Gravitino Spark SQL test +-- !query +select i_item_id + ,i_item_desc + ,i_current_price + from item, inventory, date_dim, store_sales + where i_current_price between 30 and 30+30 + and inv_item_sk = i_item_sk + and d_date_sk=inv_date_sk + and d_date between cast('2002-05-30' as date) and (cast('2002-05-30' as date) + interval 60 days) + and i_manufact_id in (437,129,727,663) + and inv_quantity_on_hand between 100 and 500 + and ss_item_sk = i_item_sk + group by i_item_id,i_item_desc,i_current_price + order by i_item_id + limit 100 +-- !query schema +struct +-- !query output + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query83.sql b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query83.sql new file mode 100644 index 00000000000..540ad47c5b8 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query83.sql @@ -0,0 +1,67 @@ +-- start query 1 in stream 0 using template query83.tpl +with sr_items as + (select i_item_id item_id, + sum(sr_return_quantity) sr_item_qty + from store_returns, + item, + date_dim + where sr_item_sk = i_item_sk + and d_date in + (select d_date + from date_dim + where d_week_seq in + (select d_week_seq + from date_dim + where d_date in ('1998-01-02','1998-10-15','1998-11-10'))) + and sr_returned_date_sk = d_date_sk + group by i_item_id), + cr_items as + (select i_item_id item_id, + sum(cr_return_quantity) cr_item_qty + from catalog_returns, + item, + date_dim + where cr_item_sk = i_item_sk + and d_date in + (select d_date + from date_dim + where d_week_seq in + (select d_week_seq + from date_dim + where d_date in ('1998-01-02','1998-10-15','1998-11-10'))) + and cr_returned_date_sk = d_date_sk + group by i_item_id), + wr_items as + (select i_item_id item_id, + sum(wr_return_quantity) wr_item_qty + from web_returns, + item, + date_dim + where wr_item_sk = i_item_sk + and d_date in + (select d_date + from date_dim + where d_week_seq in + (select d_week_seq + from date_dim + where d_date in ('1998-01-02','1998-10-15','1998-11-10'))) + and wr_returned_date_sk = d_date_sk + group by i_item_id) + select sr_items.item_id + ,sr_item_qty + ,sr_item_qty/(sr_item_qty+cr_item_qty+wr_item_qty)/3.0 * 100 sr_dev + ,cr_item_qty + ,cr_item_qty/(sr_item_qty+cr_item_qty+wr_item_qty)/3.0 * 100 cr_dev + ,wr_item_qty + ,wr_item_qty/(sr_item_qty+cr_item_qty+wr_item_qty)/3.0 * 100 wr_dev + ,(sr_item_qty+cr_item_qty+wr_item_qty)/3.0 average + from sr_items + ,cr_items + ,wr_items + where sr_items.item_id=cr_items.item_id + and sr_items.item_id=wr_items.item_id + order by sr_items.item_id + ,sr_item_qty + limit 100; + + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query83.sql.out b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query83.sql.out new file mode 100644 index 00000000000..004a13f3c89 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query83.sql.out @@ -0,0 +1,70 @@ +-- Automatically generated by Gravitino Spark SQL test +-- !query +with sr_items as + (select i_item_id item_id, + sum(sr_return_quantity) sr_item_qty + from store_returns, + item, + date_dim + where sr_item_sk = i_item_sk + and d_date in + (select d_date + from date_dim + where d_week_seq in + (select d_week_seq + from date_dim + where d_date in ('1998-01-02','1998-10-15','1998-11-10'))) + and sr_returned_date_sk = d_date_sk + group by i_item_id), + cr_items as + (select i_item_id item_id, + sum(cr_return_quantity) cr_item_qty + from catalog_returns, + item, + date_dim + where cr_item_sk = i_item_sk + and d_date in + (select d_date + from date_dim + where d_week_seq in + (select d_week_seq + from date_dim + where d_date in ('1998-01-02','1998-10-15','1998-11-10'))) + and cr_returned_date_sk = d_date_sk + group by i_item_id), + wr_items as + (select i_item_id item_id, + sum(wr_return_quantity) wr_item_qty + from web_returns, + item, + date_dim + where wr_item_sk = i_item_sk + and d_date in + (select d_date + from date_dim + where d_week_seq in + (select d_week_seq + from date_dim + where d_date in ('1998-01-02','1998-10-15','1998-11-10'))) + and wr_returned_date_sk = d_date_sk + group by i_item_id) + select sr_items.item_id + ,sr_item_qty + ,sr_item_qty/(sr_item_qty+cr_item_qty+wr_item_qty)/3.0 * 100 sr_dev + ,cr_item_qty + ,cr_item_qty/(sr_item_qty+cr_item_qty+wr_item_qty)/3.0 * 100 cr_dev + ,wr_item_qty + ,wr_item_qty/(sr_item_qty+cr_item_qty+wr_item_qty)/3.0 * 100 wr_dev + ,(sr_item_qty+cr_item_qty+wr_item_qty)/3.0 average + from sr_items + ,cr_items + ,wr_items + where sr_items.item_id=cr_items.item_id + and sr_items.item_id=wr_items.item_id + order by sr_items.item_id + ,sr_item_qty + limit 100 +-- !query schema +struct +-- !query output + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query84.sql b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query84.sql new file mode 100644 index 00000000000..9cde5f3ab0b --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query84.sql @@ -0,0 +1,21 @@ +-- start query 1 in stream 0 using template query84.tpl +select c_customer_id as customer_id + , concat(c_last_name, ',' , c_first_name) as customername + from customer + ,customer_address + ,customer_demographics + ,household_demographics + ,income_band + ,store_returns + where ca_city = 'Hopewell' + and c_current_addr_sk = ca_address_sk + and ib_lower_bound >= 32287 + and ib_upper_bound <= 32287 + 50000 + and ib_income_band_sk = hd_income_band_sk + and cd_demo_sk = c_current_cdemo_sk + and hd_demo_sk = c_current_hdemo_sk + and sr_cdemo_sk = cd_demo_sk + order by c_customer_id + limit 100; + + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query84.sql.out b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query84.sql.out new file mode 100644 index 00000000000..13e7e00b114 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query84.sql.out @@ -0,0 +1,24 @@ +-- Automatically generated by Gravitino Spark SQL test +-- !query +select c_customer_id as customer_id + , concat(c_last_name, ',' , c_first_name) as customername + from customer + ,customer_address + ,customer_demographics + ,household_demographics + ,income_band + ,store_returns + where ca_city = 'Hopewell' + and c_current_addr_sk = ca_address_sk + and ib_lower_bound >= 32287 + and ib_upper_bound <= 32287 + 50000 + and ib_income_band_sk = hd_income_band_sk + and cd_demo_sk = c_current_cdemo_sk + and hd_demo_sk = c_current_hdemo_sk + and sr_cdemo_sk = cd_demo_sk + order by c_customer_id + limit 100 +-- !query schema +struct +-- !query output + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query85.sql b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query85.sql new file mode 100644 index 00000000000..d1a4aa9ad4e --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query85.sql @@ -0,0 +1,84 @@ +-- start query 1 in stream 0 using template query85.tpl +select substr(r_reason_desc,1,20) + ,avg(ws_quantity) + ,avg(wr_refunded_cash) + ,avg(wr_fee) + from web_sales, web_returns, web_page, customer_demographics cd1, + customer_demographics cd2, customer_address, date_dim, reason + where ws_web_page_sk = wp_web_page_sk + and ws_item_sk = wr_item_sk + and ws_order_number = wr_order_number + and ws_sold_date_sk = d_date_sk and d_year = 1998 + and cd1.cd_demo_sk = wr_refunded_cdemo_sk + and cd2.cd_demo_sk = wr_returning_cdemo_sk + and ca_address_sk = wr_refunded_addr_sk + and r_reason_sk = wr_reason_sk + and + ( + ( + cd1.cd_marital_status = 'M' + and + cd1.cd_marital_status = cd2.cd_marital_status + and + cd1.cd_education_status = '4 yr Degree' + and + cd1.cd_education_status = cd2.cd_education_status + and + ws_sales_price between 100.00 and 150.00 + ) + or + ( + cd1.cd_marital_status = 'D' + and + cd1.cd_marital_status = cd2.cd_marital_status + and + cd1.cd_education_status = 'Primary' + and + cd1.cd_education_status = cd2.cd_education_status + and + ws_sales_price between 50.00 and 100.00 + ) + or + ( + cd1.cd_marital_status = 'U' + and + cd1.cd_marital_status = cd2.cd_marital_status + and + cd1.cd_education_status = 'Advanced Degree' + and + cd1.cd_education_status = cd2.cd_education_status + and + ws_sales_price between 150.00 and 200.00 + ) + ) + and + ( + ( + ca_country = 'United States' + and + ca_state in ('KY', 'GA', 'NM') + and ws_net_profit between 100 and 200 + ) + or + ( + ca_country = 'United States' + and + ca_state in ('MT', 'OR', 'IN') + and ws_net_profit between 150 and 300 + ) + or + ( + ca_country = 'United States' + and + ca_state in ('WI', 'MO', 'WV') + and ws_net_profit between 50 and 250 + ) + ) +group by r_reason_desc +order by substr(r_reason_desc,1,20) + ,avg(ws_quantity) + ,avg(wr_refunded_cash) + ,avg(wr_fee) +limit 100; + + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query85.sql.out b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query85.sql.out new file mode 100644 index 00000000000..33d5e4fb822 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query85.sql.out @@ -0,0 +1,87 @@ +-- Automatically generated by Gravitino Spark SQL test +-- !query +select substr(r_reason_desc,1,20) + ,avg(ws_quantity) + ,avg(wr_refunded_cash) + ,avg(wr_fee) + from web_sales, web_returns, web_page, customer_demographics cd1, + customer_demographics cd2, customer_address, date_dim, reason + where ws_web_page_sk = wp_web_page_sk + and ws_item_sk = wr_item_sk + and ws_order_number = wr_order_number + and ws_sold_date_sk = d_date_sk and d_year = 1998 + and cd1.cd_demo_sk = wr_refunded_cdemo_sk + and cd2.cd_demo_sk = wr_returning_cdemo_sk + and ca_address_sk = wr_refunded_addr_sk + and r_reason_sk = wr_reason_sk + and + ( + ( + cd1.cd_marital_status = 'M' + and + cd1.cd_marital_status = cd2.cd_marital_status + and + cd1.cd_education_status = '4 yr Degree' + and + cd1.cd_education_status = cd2.cd_education_status + and + ws_sales_price between 100.00 and 150.00 + ) + or + ( + cd1.cd_marital_status = 'D' + and + cd1.cd_marital_status = cd2.cd_marital_status + and + cd1.cd_education_status = 'Primary' + and + cd1.cd_education_status = cd2.cd_education_status + and + ws_sales_price between 50.00 and 100.00 + ) + or + ( + cd1.cd_marital_status = 'U' + and + cd1.cd_marital_status = cd2.cd_marital_status + and + cd1.cd_education_status = 'Advanced Degree' + and + cd1.cd_education_status = cd2.cd_education_status + and + ws_sales_price between 150.00 and 200.00 + ) + ) + and + ( + ( + ca_country = 'United States' + and + ca_state in ('KY', 'GA', 'NM') + and ws_net_profit between 100 and 200 + ) + or + ( + ca_country = 'United States' + and + ca_state in ('MT', 'OR', 'IN') + and ws_net_profit between 150 and 300 + ) + or + ( + ca_country = 'United States' + and + ca_state in ('WI', 'MO', 'WV') + and ws_net_profit between 50 and 250 + ) + ) +group by r_reason_desc +order by substr(r_reason_desc,1,20) + ,avg(ws_quantity) + ,avg(wr_refunded_cash) + ,avg(wr_fee) +limit 100 +-- !query schema +struct +-- !query output + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query86.sql b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query86.sql new file mode 100644 index 00000000000..8aece8d7c58 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query86.sql @@ -0,0 +1,26 @@ +-- start query 1 in stream 0 using template query86.tpl +select + sum(ws_net_paid) as total_sum + ,i_category + ,i_class + ,grouping(i_category)+grouping(i_class) as lochierarchy + ,rank() over ( + partition by grouping(i_category)+grouping(i_class), + case when grouping(i_class) = 0 then i_category end + order by sum(ws_net_paid) desc) as rank_within_parent + from + web_sales + ,date_dim d1 + ,item + where + d1.d_month_seq between 1212 and 1212+11 + and d1.d_date_sk = ws_sold_date_sk + and i_item_sk = ws_item_sk + group by rollup(i_category,i_class) + order by + lochierarchy desc, + case when lochierarchy = 0 then i_category end, + rank_within_parent + limit 100; + + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query86.sql.out b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query86.sql.out new file mode 100644 index 00000000000..ede711e4299 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query86.sql.out @@ -0,0 +1,128 @@ +-- Automatically generated by Gravitino Spark SQL test +-- !query +select + sum(ws_net_paid) as total_sum + ,i_category + ,i_class + ,grouping(i_category)+grouping(i_class) as lochierarchy + ,rank() over ( + partition by grouping(i_category)+grouping(i_class), + case when grouping(i_class) = 0 then i_category end + order by sum(ws_net_paid) desc) as rank_within_parent + from + web_sales + ,date_dim d1 + ,item + where + d1.d_month_seq between 1212 and 1212+11 + and d1.d_date_sk = ws_sold_date_sk + and i_item_sk = ws_item_sk + group by rollup(i_category,i_class) + order by + lochierarchy desc, + case when lochierarchy = 0 then i_category end, + rank_within_parent + limit 100 +-- !query schema +struct +-- !query output +102290.70 Shoes womens 0 1 +10787.43 Electronics audio 0 16 +1093.49 NULL archery 0 2 +11022.02 Books entertainments 0 13 +111286.37 Children newborn 0 1 +118049.61 Music classical 0 2 +12126.97 Electronics camcorders 0 15 +12154.20 Jewelry mens watch 0 9 +124172.19 Music pop 0 1 +12745.95 Books sports 0 12 +12884.37 Home wallpaper 0 15 +12934.66 Electronics disk drives 0 14 +13463.79 Home accent 0 14 +14039.26 Home bathroom 0 13 +15200.01 Home lighting 0 12 +15806.26 Home mattresses 0 11 +15959.04 Home blinds/shades 0 10 +16181.72 Electronics musical 0 13 +16603.51 Electronics televisions 0 12 +17450.28 Home furniture 0 9 +17899.49 Electronics automotive 0 11 +17936.83 Electronics personal 0 10 +19425.12 Jewelry earings 0 8 +19935.09 Electronics cameras 0 9 +20141.05 Electronics monitors 0 8 +20486.58 Books arts 0 11 +20966.37 Jewelry custom 0 7 +21405.38 Electronics scanners 0 7 +21585.50 Books self-help 0 10 +22420.83 Home rugs 0 8 +23237.85 Home glassware 0 7 +23351.19 Electronics stereo 0 6 +23685.42 Electronics portable 0 5 +24077.16 Home flatware 0 6 +24098.24 Home curtains/drapes 0 5 +24267.98 Jewelry jewelry boxes 0 6 +25280.73 Books travel 0 9 +25396.40 Sports hockey 0 6 +25497.26 Books romance 0 8 +25628.99 Sports pools 0 5 +25685.78 Books mystery 0 7 +26188.92 Home kids 0 4 +26207.48 Home bedding 0 3 +264069.51 Men NULL 1 10 +27714.14 Home tables 0 2 +27721.64 Sports fishing 0 4 +28253.95 Electronics wireless 0 4 +292100.08 Jewelry NULL 1 9 +29421.88 Jewelry diamonds 0 5 +29610.89 Books fiction 0 6 +3048.08 NULL NULL 0 1 +323095.87 Home NULL 1 8 +326654.63 Sports NULL 1 7 +326706.72 Children NULL 1 6 +33242.75 Books computers 0 5 +33743.30 Men pants 0 4 +33766.58 Jewelry costume 0 4 +34202.48 Electronics karoke 0 3 +3430589.35 NULL NULL 2 1 +34716.50 Sports golf 0 3 +355954.25 Electronics NULL 1 5 +36058.16 Jewelry estate 0 3 +36868.14 Jewelry bracelets 0 2 +372484.83 Shoes NULL 1 4 +37660.32 Sports tennis 0 2 +37924.12 Books science 0 4 +384412.75 Women NULL 1 3 +388761.90 Music NULL 1 2 +38931.89 Home decor 0 1 +39030.69 Electronics memory 0 2 +392207.24 Books NULL 1 1 +39457.02 Jewelry gold 0 1 +4141.57 NULL NULL 1 11 +41478.39 Electronics dvd/vcr players 0 1 +41905.95 Books cooking 0 3 +41954.69 Books reference 0 2 +4218.22 Jewelry womens watch 0 16 +42849.73 Books history 0 1 +43832.72 Sports outdoor 0 1 +4418.98 Jewelry loose stones 0 15 +4532.94 Jewelry consignment 0 14 +4705.01 Jewelry pendants 0 13 +47762.25 Music rock 0 4 +4840.16 Books business 0 16 +5416.35 Home paint 0 16 +59546.60 Children toddlers 0 4 +61735.34 Children infants 0 3 +62921.16 Men shirts 0 3 +6476.60 Jewelry semi-precious 0 12 +6709.82 Jewelry birdal 0 11 +81504.21 Men sports-apparel 0 2 +83938.68 Shoes athletic 0 4 +85900.84 Men accessories 0 1 +8653.06 Jewelry rings 0 10 +8736.19 Books home repair 0 15 +87570.32 Shoes mens 0 3 +8838.94 Books parenting 0 14 +94138.41 Children school-uniforms 0 2 +98685.13 Shoes kids 0 2 +98777.85 Music country 0 3 diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query87.sql b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query87.sql new file mode 100644 index 00000000000..344aeb0655d --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query87.sql @@ -0,0 +1,23 @@ +-- start query 1 in stream 0 using template query87.tpl +select count(*) +from ((select distinct c_last_name, c_first_name, d_date + from store_sales, date_dim, customer + where store_sales.ss_sold_date_sk = date_dim.d_date_sk + and store_sales.ss_customer_sk = customer.c_customer_sk + and d_month_seq between 1212 and 1212+11) + except + (select distinct c_last_name, c_first_name, d_date + from catalog_sales, date_dim, customer + where catalog_sales.cs_sold_date_sk = date_dim.d_date_sk + and catalog_sales.cs_bill_customer_sk = customer.c_customer_sk + and d_month_seq between 1212 and 1212+11) + except + (select distinct c_last_name, c_first_name, d_date + from web_sales, date_dim, customer + where web_sales.ws_sold_date_sk = date_dim.d_date_sk + and web_sales.ws_bill_customer_sk = customer.c_customer_sk + and d_month_seq between 1212 and 1212+11) +) cool_cust +; + + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query87.sql.out b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query87.sql.out new file mode 100644 index 00000000000..8fd5ba452a2 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query87.sql.out @@ -0,0 +1,25 @@ +-- Automatically generated by Gravitino Spark SQL test +-- !query +select count(*) +from ((select distinct c_last_name, c_first_name, d_date + from store_sales, date_dim, customer + where store_sales.ss_sold_date_sk = date_dim.d_date_sk + and store_sales.ss_customer_sk = customer.c_customer_sk + and d_month_seq between 1212 and 1212+11) + except + (select distinct c_last_name, c_first_name, d_date + from catalog_sales, date_dim, customer + where catalog_sales.cs_sold_date_sk = date_dim.d_date_sk + and catalog_sales.cs_bill_customer_sk = customer.c_customer_sk + and d_month_seq between 1212 and 1212+11) + except + (select distinct c_last_name, c_first_name, d_date + from web_sales, date_dim, customer + where web_sales.ws_sold_date_sk = date_dim.d_date_sk + and web_sales.ws_bill_customer_sk = customer.c_customer_sk + and d_month_seq between 1212 and 1212+11) +) cool_cust +-- !query schema +struct +-- !query output +59 diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query88.sql b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query88.sql new file mode 100644 index 00000000000..a63f759943f --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query88.sql @@ -0,0 +1,94 @@ +-- start query 1 in stream 0 using template query88.tpl +select * +from + (select count(*) h8_30_to_9 + from store_sales, household_demographics , time_dim, store + where ss_sold_time_sk = time_dim.t_time_sk + and ss_hdemo_sk = household_demographics.hd_demo_sk + and ss_store_sk = s_store_sk + and time_dim.t_hour = 8 + and time_dim.t_minute >= 30 + and ((household_demographics.hd_dep_count = 3 and household_demographics.hd_vehicle_count<=3+2) or + (household_demographics.hd_dep_count = 0 and household_demographics.hd_vehicle_count<=0+2) or + (household_demographics.hd_dep_count = 1 and household_demographics.hd_vehicle_count<=1+2)) + and store.s_store_name = 'ese') s1, + (select count(*) h9_to_9_30 + from store_sales, household_demographics , time_dim, store + where ss_sold_time_sk = time_dim.t_time_sk + and ss_hdemo_sk = household_demographics.hd_demo_sk + and ss_store_sk = s_store_sk + and time_dim.t_hour = 9 + and time_dim.t_minute < 30 + and ((household_demographics.hd_dep_count = 3 and household_demographics.hd_vehicle_count<=3+2) or + (household_demographics.hd_dep_count = 0 and household_demographics.hd_vehicle_count<=0+2) or + (household_demographics.hd_dep_count = 1 and household_demographics.hd_vehicle_count<=1+2)) + and store.s_store_name = 'ese') s2, + (select count(*) h9_30_to_10 + from store_sales, household_demographics , time_dim, store + where ss_sold_time_sk = time_dim.t_time_sk + and ss_hdemo_sk = household_demographics.hd_demo_sk + and ss_store_sk = s_store_sk + and time_dim.t_hour = 9 + and time_dim.t_minute >= 30 + and ((household_demographics.hd_dep_count = 3 and household_demographics.hd_vehicle_count<=3+2) or + (household_demographics.hd_dep_count = 0 and household_demographics.hd_vehicle_count<=0+2) or + (household_demographics.hd_dep_count = 1 and household_demographics.hd_vehicle_count<=1+2)) + and store.s_store_name = 'ese') s3, + (select count(*) h10_to_10_30 + from store_sales, household_demographics , time_dim, store + where ss_sold_time_sk = time_dim.t_time_sk + and ss_hdemo_sk = household_demographics.hd_demo_sk + and ss_store_sk = s_store_sk + and time_dim.t_hour = 10 + and time_dim.t_minute < 30 + and ((household_demographics.hd_dep_count = 3 and household_demographics.hd_vehicle_count<=3+2) or + (household_demographics.hd_dep_count = 0 and household_demographics.hd_vehicle_count<=0+2) or + (household_demographics.hd_dep_count = 1 and household_demographics.hd_vehicle_count<=1+2)) + and store.s_store_name = 'ese') s4, + (select count(*) h10_30_to_11 + from store_sales, household_demographics , time_dim, store + where ss_sold_time_sk = time_dim.t_time_sk + and ss_hdemo_sk = household_demographics.hd_demo_sk + and ss_store_sk = s_store_sk + and time_dim.t_hour = 10 + and time_dim.t_minute >= 30 + and ((household_demographics.hd_dep_count = 3 and household_demographics.hd_vehicle_count<=3+2) or + (household_demographics.hd_dep_count = 0 and household_demographics.hd_vehicle_count<=0+2) or + (household_demographics.hd_dep_count = 1 and household_demographics.hd_vehicle_count<=1+2)) + and store.s_store_name = 'ese') s5, + (select count(*) h11_to_11_30 + from store_sales, household_demographics , time_dim, store + where ss_sold_time_sk = time_dim.t_time_sk + and ss_hdemo_sk = household_demographics.hd_demo_sk + and ss_store_sk = s_store_sk + and time_dim.t_hour = 11 + and time_dim.t_minute < 30 + and ((household_demographics.hd_dep_count = 3 and household_demographics.hd_vehicle_count<=3+2) or + (household_demographics.hd_dep_count = 0 and household_demographics.hd_vehicle_count<=0+2) or + (household_demographics.hd_dep_count = 1 and household_demographics.hd_vehicle_count<=1+2)) + and store.s_store_name = 'ese') s6, + (select count(*) h11_30_to_12 + from store_sales, household_demographics , time_dim, store + where ss_sold_time_sk = time_dim.t_time_sk + and ss_hdemo_sk = household_demographics.hd_demo_sk + and ss_store_sk = s_store_sk + and time_dim.t_hour = 11 + and time_dim.t_minute >= 30 + and ((household_demographics.hd_dep_count = 3 and household_demographics.hd_vehicle_count<=3+2) or + (household_demographics.hd_dep_count = 0 and household_demographics.hd_vehicle_count<=0+2) or + (household_demographics.hd_dep_count = 1 and household_demographics.hd_vehicle_count<=1+2)) + and store.s_store_name = 'ese') s7, + (select count(*) h12_to_12_30 + from store_sales, household_demographics , time_dim, store + where ss_sold_time_sk = time_dim.t_time_sk + and ss_hdemo_sk = household_demographics.hd_demo_sk + and ss_store_sk = s_store_sk + and time_dim.t_hour = 12 + and time_dim.t_minute < 30 + and ((household_demographics.hd_dep_count = 3 and household_demographics.hd_vehicle_count<=3+2) or + (household_demographics.hd_dep_count = 0 and household_demographics.hd_vehicle_count<=0+2) or + (household_demographics.hd_dep_count = 1 and household_demographics.hd_vehicle_count<=1+2)) + and store.s_store_name = 'ese') s8 +; + + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query88.sql.out b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query88.sql.out new file mode 100644 index 00000000000..ebe75b07d3d --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query88.sql.out @@ -0,0 +1,96 @@ +-- Automatically generated by Gravitino Spark SQL test +-- !query +select * +from + (select count(*) h8_30_to_9 + from store_sales, household_demographics , time_dim, store + where ss_sold_time_sk = time_dim.t_time_sk + and ss_hdemo_sk = household_demographics.hd_demo_sk + and ss_store_sk = s_store_sk + and time_dim.t_hour = 8 + and time_dim.t_minute >= 30 + and ((household_demographics.hd_dep_count = 3 and household_demographics.hd_vehicle_count<=3+2) or + (household_demographics.hd_dep_count = 0 and household_demographics.hd_vehicle_count<=0+2) or + (household_demographics.hd_dep_count = 1 and household_demographics.hd_vehicle_count<=1+2)) + and store.s_store_name = 'ese') s1, + (select count(*) h9_to_9_30 + from store_sales, household_demographics , time_dim, store + where ss_sold_time_sk = time_dim.t_time_sk + and ss_hdemo_sk = household_demographics.hd_demo_sk + and ss_store_sk = s_store_sk + and time_dim.t_hour = 9 + and time_dim.t_minute < 30 + and ((household_demographics.hd_dep_count = 3 and household_demographics.hd_vehicle_count<=3+2) or + (household_demographics.hd_dep_count = 0 and household_demographics.hd_vehicle_count<=0+2) or + (household_demographics.hd_dep_count = 1 and household_demographics.hd_vehicle_count<=1+2)) + and store.s_store_name = 'ese') s2, + (select count(*) h9_30_to_10 + from store_sales, household_demographics , time_dim, store + where ss_sold_time_sk = time_dim.t_time_sk + and ss_hdemo_sk = household_demographics.hd_demo_sk + and ss_store_sk = s_store_sk + and time_dim.t_hour = 9 + and time_dim.t_minute >= 30 + and ((household_demographics.hd_dep_count = 3 and household_demographics.hd_vehicle_count<=3+2) or + (household_demographics.hd_dep_count = 0 and household_demographics.hd_vehicle_count<=0+2) or + (household_demographics.hd_dep_count = 1 and household_demographics.hd_vehicle_count<=1+2)) + and store.s_store_name = 'ese') s3, + (select count(*) h10_to_10_30 + from store_sales, household_demographics , time_dim, store + where ss_sold_time_sk = time_dim.t_time_sk + and ss_hdemo_sk = household_demographics.hd_demo_sk + and ss_store_sk = s_store_sk + and time_dim.t_hour = 10 + and time_dim.t_minute < 30 + and ((household_demographics.hd_dep_count = 3 and household_demographics.hd_vehicle_count<=3+2) or + (household_demographics.hd_dep_count = 0 and household_demographics.hd_vehicle_count<=0+2) or + (household_demographics.hd_dep_count = 1 and household_demographics.hd_vehicle_count<=1+2)) + and store.s_store_name = 'ese') s4, + (select count(*) h10_30_to_11 + from store_sales, household_demographics , time_dim, store + where ss_sold_time_sk = time_dim.t_time_sk + and ss_hdemo_sk = household_demographics.hd_demo_sk + and ss_store_sk = s_store_sk + and time_dim.t_hour = 10 + and time_dim.t_minute >= 30 + and ((household_demographics.hd_dep_count = 3 and household_demographics.hd_vehicle_count<=3+2) or + (household_demographics.hd_dep_count = 0 and household_demographics.hd_vehicle_count<=0+2) or + (household_demographics.hd_dep_count = 1 and household_demographics.hd_vehicle_count<=1+2)) + and store.s_store_name = 'ese') s5, + (select count(*) h11_to_11_30 + from store_sales, household_demographics , time_dim, store + where ss_sold_time_sk = time_dim.t_time_sk + and ss_hdemo_sk = household_demographics.hd_demo_sk + and ss_store_sk = s_store_sk + and time_dim.t_hour = 11 + and time_dim.t_minute < 30 + and ((household_demographics.hd_dep_count = 3 and household_demographics.hd_vehicle_count<=3+2) or + (household_demographics.hd_dep_count = 0 and household_demographics.hd_vehicle_count<=0+2) or + (household_demographics.hd_dep_count = 1 and household_demographics.hd_vehicle_count<=1+2)) + and store.s_store_name = 'ese') s6, + (select count(*) h11_30_to_12 + from store_sales, household_demographics , time_dim, store + where ss_sold_time_sk = time_dim.t_time_sk + and ss_hdemo_sk = household_demographics.hd_demo_sk + and ss_store_sk = s_store_sk + and time_dim.t_hour = 11 + and time_dim.t_minute >= 30 + and ((household_demographics.hd_dep_count = 3 and household_demographics.hd_vehicle_count<=3+2) or + (household_demographics.hd_dep_count = 0 and household_demographics.hd_vehicle_count<=0+2) or + (household_demographics.hd_dep_count = 1 and household_demographics.hd_vehicle_count<=1+2)) + and store.s_store_name = 'ese') s7, + (select count(*) h12_to_12_30 + from store_sales, household_demographics , time_dim, store + where ss_sold_time_sk = time_dim.t_time_sk + and ss_hdemo_sk = household_demographics.hd_demo_sk + and ss_store_sk = s_store_sk + and time_dim.t_hour = 12 + and time_dim.t_minute < 30 + and ((household_demographics.hd_dep_count = 3 and household_demographics.hd_vehicle_count<=3+2) or + (household_demographics.hd_dep_count = 0 and household_demographics.hd_vehicle_count<=0+2) or + (household_demographics.hd_dep_count = 1 and household_demographics.hd_vehicle_count<=1+2)) + and store.s_store_name = 'ese') s8 +-- !query schema +struct +-- !query output +36 71 36 52 96 31 88 57 diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query89.sql b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query89.sql new file mode 100644 index 00000000000..830ebe2747d --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query89.sql @@ -0,0 +1,28 @@ +-- start query 1 in stream 0 using template query89.tpl +select * +from( +select i_category, i_class, i_brand, + s_store_name, s_company_name, + d_moy, + sum(ss_sales_price) sum_sales, + avg(sum(ss_sales_price)) over + (partition by i_category, i_brand, s_store_name, s_company_name) + avg_monthly_sales +from item, store_sales, date_dim, store +where ss_item_sk = i_item_sk and + ss_sold_date_sk = d_date_sk and + ss_store_sk = s_store_sk and + d_year in (2000) and + ((i_category in ('Home','Books','Electronics') and + i_class in ('wallpaper','parenting','musical') + ) + or (i_category in ('Shoes','Jewelry','Men') and + i_class in ('womens','birdal','pants') + )) +group by i_category, i_class, i_brand, + s_store_name, s_company_name, d_moy) tmp1 +where case when (avg_monthly_sales <> 0) then (abs(sum_sales - avg_monthly_sales) / avg_monthly_sales) else null end > 0.1 +order by sum_sales - avg_monthly_sales, s_store_name +limit 100; + + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query89.sql.out b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query89.sql.out new file mode 100644 index 00000000000..02b17618231 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query89.sql.out @@ -0,0 +1,130 @@ +-- Automatically generated by Gravitino Spark SQL test +-- !query +select * +from( +select i_category, i_class, i_brand, + s_store_name, s_company_name, + d_moy, + sum(ss_sales_price) sum_sales, + avg(sum(ss_sales_price)) over + (partition by i_category, i_brand, s_store_name, s_company_name) + avg_monthly_sales +from item, store_sales, date_dim, store +where ss_item_sk = i_item_sk and + ss_sold_date_sk = d_date_sk and + ss_store_sk = s_store_sk and + d_year in (2000) and + ((i_category in ('Home','Books','Electronics') and + i_class in ('wallpaper','parenting','musical') + ) + or (i_category in ('Shoes','Jewelry','Men') and + i_class in ('womens','birdal','pants') + )) +group by i_category, i_class, i_brand, + s_store_name, s_company_name, d_moy) tmp1 +where case when (avg_monthly_sales <> 0) then (abs(sum_sales - avg_monthly_sales) / avg_monthly_sales) else null end > 0.1 +order by sum_sales - avg_monthly_sales, s_store_name +limit 100 +-- !query schema +struct +-- !query output +Books parenting corpmaxi #x able Unknown 12 9.05 28.145000 +Books parenting corpmaxi #x ation Unknown 12 24.12 88.306667 +Books parenting corpmaxi #x ation Unknown 9 0.65 16.570000 +Books parenting corpmaxi #x ese Unknown 12 19.93 30.505000 +Books parenting corpmaxi #x ese Unknown 3 3.03 46.770000 +Books parenting corpmaxi #x ese Unknown 6 5.66 46.770000 +Books parenting corpmaxi #x ought Unknown 11 0.00 44.955000 +Books parenting corpmaxi #x ought Unknown 3 1.80 40.480000 +Books parenting corpmaxi #x ought Unknown 3 4.07 15.880000 +Books parenting corpmaxi #x ought Unknown 9 1.86 15.880000 +Electronics musical corpunivamalg #x ation Unknown 5 0.00 17.166667 +Electronics musical corpunivamalg #x bar Unknown 1 3.42 15.125000 +Electronics musical corpunivamalg #x ese Unknown 11 51.40 72.805000 +Home wallpaper exportinameless #x able Unknown 10 9.38 26.950000 +Home wallpaper exportinameless #x able Unknown 4 36.64 59.325000 +Home wallpaper exportinameless #x bar Unknown 11 14.79 27.130000 +Home wallpaper exportinameless #x bar Unknown 2 28.70 91.745000 +Home wallpaper exportinameless #x ese Unknown 12 10.50 42.643333 +Jewelry birdal amalgcorp #x able Unknown 12 0.00 12.175000 +Jewelry birdal amalgcorp #x ation Unknown 9 43.15 55.435000 +Jewelry birdal amalgcorp #x eing Unknown 9 3.22 62.690000 +Men pants exportiimporto #x able Unknown 10 13.87 33.281429 +Men pants exportiimporto #x able Unknown 11 20.73 33.281429 +Men pants exportiimporto #x able Unknown 12 3.95 33.281429 +Men pants exportiimporto #x able Unknown 5 10.62 36.286000 +Men pants exportiimporto #x able Unknown 7 3.77 36.286000 +Men pants exportiimporto #x able Unknown 9 0.00 36.286000 +Men pants exportiimporto #x ation Unknown 1 20.92 61.657500 +Men pants exportiimporto #x ation Unknown 5 40.54 61.657500 +Men pants exportiimporto #x ation Unknown 6 0.61 23.330000 +Men pants exportiimporto #x ation Unknown 8 16.76 61.657500 +Men pants exportiimporto #x bar Unknown 10 10.34 52.747500 +Men pants exportiimporto #x bar Unknown 12 3.58 23.152000 +Men pants exportiimporto #x bar Unknown 2 40.60 52.747500 +Men pants exportiimporto #x bar Unknown 5 23.12 52.747500 +Men pants exportiimporto #x bar Unknown 8 22.06 52.747500 +Men pants exportiimporto #x bar Unknown 8 9.78 23.152000 +Men pants exportiimporto #x eing Unknown 1 16.57 48.099000 +Men pants exportiimporto #x eing Unknown 10 2.45 48.099000 +Men pants exportiimporto #x eing Unknown 10 8.72 38.367500 +Men pants exportiimporto #x eing Unknown 2 3.40 48.099000 +Men pants exportiimporto #x eing Unknown 3 31.46 48.099000 +Men pants exportiimporto #x eing Unknown 4 6.27 48.099000 +Men pants exportiimporto #x eing Unknown 8 32.63 48.099000 +Men pants exportiimporto #x eing Unknown 9 38.31 48.099000 +Men pants exportiimporto #x ese Unknown 1 6.25 63.242500 +Men pants exportiimporto #x ese Unknown 11 16.47 56.218333 +Men pants exportiimporto #x ese Unknown 11 45.03 63.242500 +Men pants exportiimporto #x ese Unknown 12 29.71 63.242500 +Men pants exportiimporto #x ese Unknown 4 1.72 63.242500 +Men pants exportiimporto #x ese Unknown 5 17.68 63.242500 +Men pants exportiimporto #x ese Unknown 7 52.83 63.242500 +Men pants exportiimporto #x ese Unknown 8 1.25 56.218333 +Men pants exportiimporto #x ese Unknown 8 44.51 63.242500 +Men pants exportiimporto #x ese Unknown 9 20.26 56.218333 +Men pants exportiimporto #x ese Unknown 9 47.52 63.242500 +Men pants exportiimporto #x ought Unknown 10 45.84 96.257143 +Men pants exportiimporto #x ought Unknown 11 15.60 66.478000 +Men pants exportiimporto #x ought Unknown 3 22.14 96.257143 +Men pants exportiimporto #x ought Unknown 7 19.94 66.478000 +Men pants exportiimporto #x ought Unknown 7 49.43 96.257143 +Men pants exportiimporto #x ought Unknown 8 5.21 66.478000 +Men pants exportiimporto #x ought Unknown 9 4.40 96.257143 +Shoes womens amalgedu pack #x able Unknown 10 62.92 89.993750 +Shoes womens amalgedu pack #x able Unknown 11 4.84 21.255000 +Shoes womens amalgedu pack #x able Unknown 3 22.19 89.993750 +Shoes womens amalgedu pack #x able Unknown 5 40.62 89.993750 +Shoes womens amalgedu pack #x able Unknown 6 5.53 21.255000 +Shoes womens amalgedu pack #x ation Unknown 1 35.64 80.230000 +Shoes womens amalgedu pack #x ation Unknown 10 31.03 72.111667 +Shoes womens amalgedu pack #x ation Unknown 11 52.60 72.111667 +Shoes womens amalgedu pack #x ation Unknown 12 3.63 72.111667 +Shoes womens amalgedu pack #x ation Unknown 3 10.75 72.111667 +Shoes womens amalgedu pack #x ation Unknown 3 17.89 80.230000 +Shoes womens amalgedu pack #x ation Unknown 4 30.52 80.230000 +Shoes womens amalgedu pack #x ation Unknown 6 16.14 72.111667 +Shoes womens amalgedu pack #x bar Unknown 1 75.23 112.910000 +Shoes womens amalgedu pack #x bar Unknown 10 25.47 112.910000 +Shoes womens amalgedu pack #x bar Unknown 12 23.47 107.068000 +Shoes womens amalgedu pack #x bar Unknown 3 99.68 112.910000 +Shoes womens amalgedu pack #x bar Unknown 7 12.76 107.068000 +Shoes womens amalgedu pack #x bar Unknown 8 25.98 107.068000 +Shoes womens amalgedu pack #x bar Unknown 8 50.16 112.910000 +Shoes womens amalgedu pack #x eing Unknown 1 18.99 72.415714 +Shoes womens amalgedu pack #x eing Unknown 11 26.87 72.415714 +Shoes womens amalgedu pack #x eing Unknown 4 15.73 72.415714 +Shoes womens amalgedu pack #x eing Unknown 5 37.36 67.152500 +Shoes womens amalgedu pack #x ese Unknown 11 30.20 49.901429 +Shoes womens amalgedu pack #x ese Unknown 12 19.11 49.901429 +Shoes womens amalgedu pack #x ese Unknown 5 17.78 49.901429 +Shoes womens amalgedu pack #x ese Unknown 9 11.21 53.032500 +Shoes womens amalgedu pack #x ought Unknown 1 73.98 111.150000 +Shoes womens amalgedu pack #x ought Unknown 10 33.94 111.150000 +Shoes womens amalgedu pack #x ought Unknown 10 37.07 86.355714 +Shoes womens amalgedu pack #x ought Unknown 11 44.12 111.150000 +Shoes womens amalgedu pack #x ought Unknown 2 73.05 86.355714 +Shoes womens amalgedu pack #x ought Unknown 3 32.11 86.355714 +Shoes womens amalgedu pack #x ought Unknown 7 39.60 111.150000 +Shoes womens amalgedu pack #x ought Unknown 8 43.71 86.355714 +Shoes womens amalgedu pack #x ought Unknown 9 41.43 86.355714 diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query9.sql b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query9.sql new file mode 100644 index 00000000000..43e9e7952b0 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query9.sql @@ -0,0 +1,51 @@ +-- start query 1 in stream 0 using template query9.tpl +select case when (select count(*) + from store_sales + where ss_quantity between 1 and 20) > 25437 + then (select avg(ss_ext_discount_amt) + from store_sales + where ss_quantity between 1 and 20) + else (select avg(ss_net_profit) + from store_sales + where ss_quantity between 1 and 20) end bucket1 , + case when (select count(*) + from store_sales + where ss_quantity between 21 and 40) > 22746 + then (select avg(ss_ext_discount_amt) + from store_sales + where ss_quantity between 21 and 40) + else (select avg(ss_net_profit) + from store_sales + where ss_quantity between 21 and 40) end bucket2, + case when (select count(*) + from store_sales + where ss_quantity between 41 and 60) > 9387 + then (select avg(ss_ext_discount_amt) + from store_sales + where ss_quantity between 41 and 60) + else (select avg(ss_net_profit) + from store_sales + where ss_quantity between 41 and 60) end bucket3, + case when (select count(*) + from store_sales + where ss_quantity between 61 and 80) > 10098 + then (select avg(ss_ext_discount_amt) + from store_sales + where ss_quantity between 61 and 80) + else (select avg(ss_net_profit) + from store_sales + where ss_quantity between 61 and 80) end bucket4, + case when (select count(*) + from store_sales + where ss_quantity between 81 and 100) > 18213 + then (select avg(ss_ext_discount_amt) + from store_sales + where ss_quantity between 81 and 100) + else (select avg(ss_net_profit) + from store_sales + where ss_quantity between 81 and 100) end bucket5 +from reason +where r_reason_sk = 1 +; + + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query9.sql.out b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query9.sql.out new file mode 100644 index 00000000000..f7e44a2c593 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query9.sql.out @@ -0,0 +1,53 @@ +-- Automatically generated by Gravitino Spark SQL test +-- !query +select case when (select count(*) + from store_sales + where ss_quantity between 1 and 20) > 25437 + then (select avg(ss_ext_discount_amt) + from store_sales + where ss_quantity between 1 and 20) + else (select avg(ss_net_profit) + from store_sales + where ss_quantity between 1 and 20) end bucket1 , + case when (select count(*) + from store_sales + where ss_quantity between 21 and 40) > 22746 + then (select avg(ss_ext_discount_amt) + from store_sales + where ss_quantity between 21 and 40) + else (select avg(ss_net_profit) + from store_sales + where ss_quantity between 21 and 40) end bucket2, + case when (select count(*) + from store_sales + where ss_quantity between 41 and 60) > 9387 + then (select avg(ss_ext_discount_amt) + from store_sales + where ss_quantity between 41 and 60) + else (select avg(ss_net_profit) + from store_sales + where ss_quantity between 41 and 60) end bucket3, + case when (select count(*) + from store_sales + where ss_quantity between 61 and 80) > 10098 + then (select avg(ss_ext_discount_amt) + from store_sales + where ss_quantity between 61 and 80) + else (select avg(ss_net_profit) + from store_sales + where ss_quantity between 61 and 80) end bucket4, + case when (select count(*) + from store_sales + where ss_quantity between 81 and 100) > 18213 + then (select avg(ss_ext_discount_amt) + from store_sales + where ss_quantity between 81 and 100) + else (select avg(ss_net_profit) + from store_sales + where ss_quantity between 81 and 100) end bucket5 +from reason +where r_reason_sk = 1 +-- !query schema +struct +-- !query output +-169.442364 -496.636895 -854.597626 -1194.638070 -1452.787744 diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query90.sql b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query90.sql new file mode 100644 index 00000000000..93a6cf1bbdd --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query90.sql @@ -0,0 +1,22 @@ +-- start query 1 in stream 0 using template query90.tpl +select cast(amc as decimal(15,4))/cast(pmc as decimal(15,4)) am_pm_ratio + from ( select count(*) amc + from web_sales, household_demographics , time_dim, web_page + where ws_sold_time_sk = time_dim.t_time_sk + and ws_ship_hdemo_sk = household_demographics.hd_demo_sk + and ws_web_page_sk = web_page.wp_web_page_sk + and time_dim.t_hour between 6 and 6+1 + and household_demographics.hd_dep_count = 8 + and web_page.wp_char_count between 5000 and 5200) at, + ( select count(*) pmc + from web_sales, household_demographics , time_dim, web_page + where ws_sold_time_sk = time_dim.t_time_sk + and ws_ship_hdemo_sk = household_demographics.hd_demo_sk + and ws_web_page_sk = web_page.wp_web_page_sk + and time_dim.t_hour between 14 and 14+1 + and household_demographics.hd_dep_count = 8 + and web_page.wp_char_count between 5000 and 5200) pt + order by am_pm_ratio + limit 100; + + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query90.sql.out b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query90.sql.out new file mode 100644 index 00000000000..5d948475dc1 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query90.sql.out @@ -0,0 +1,25 @@ +-- Automatically generated by Gravitino Spark SQL test +-- !query +select cast(amc as decimal(15,4))/cast(pmc as decimal(15,4)) am_pm_ratio + from ( select count(*) amc + from web_sales, household_demographics , time_dim, web_page + where ws_sold_time_sk = time_dim.t_time_sk + and ws_ship_hdemo_sk = household_demographics.hd_demo_sk + and ws_web_page_sk = web_page.wp_web_page_sk + and time_dim.t_hour between 6 and 6+1 + and household_demographics.hd_dep_count = 8 + and web_page.wp_char_count between 5000 and 5200) at, + ( select count(*) pmc + from web_sales, household_demographics , time_dim, web_page + where ws_sold_time_sk = time_dim.t_time_sk + and ws_ship_hdemo_sk = household_demographics.hd_demo_sk + and ws_web_page_sk = web_page.wp_web_page_sk + and time_dim.t_hour between 14 and 14+1 + and household_demographics.hd_dep_count = 8 + and web_page.wp_char_count between 5000 and 5200) pt + order by am_pm_ratio + limit 100 +-- !query schema +struct +-- !query output +0.00000000000000000000 diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query91.sql b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query91.sql new file mode 100644 index 00000000000..88849562d36 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query91.sql @@ -0,0 +1,31 @@ +-- start query 1 in stream 0 using template query91.tpl +select + cc_call_center_id Call_Center, + cc_name Call_Center_Name, + cc_manager Manager, + sum(cr_net_loss) Returns_Loss +from + call_center, + catalog_returns, + date_dim, + customer, + customer_address, + customer_demographics, + household_demographics +where + cr_call_center_sk = cc_call_center_sk +and cr_returned_date_sk = d_date_sk +and cr_returning_customer_sk= c_customer_sk +and cd_demo_sk = c_current_cdemo_sk +and hd_demo_sk = c_current_hdemo_sk +and ca_address_sk = c_current_addr_sk +and d_year = 1999 +and d_moy = 11 +and ( (cd_marital_status = 'M' and cd_education_status = 'Unknown') + or(cd_marital_status = 'W' and cd_education_status = 'Advanced Degree')) +and hd_buy_potential like '0-500%' +and ca_gmt_offset = -7 +group by cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status +order by sum(cr_net_loss) desc; + + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query91.sql.out b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query91.sql.out new file mode 100644 index 00000000000..09929cbf446 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query91.sql.out @@ -0,0 +1,34 @@ +-- Automatically generated by Gravitino Spark SQL test +-- !query +select + cc_call_center_id Call_Center, + cc_name Call_Center_Name, + cc_manager Manager, + sum(cr_net_loss) Returns_Loss +from + call_center, + catalog_returns, + date_dim, + customer, + customer_address, + customer_demographics, + household_demographics +where + cr_call_center_sk = cc_call_center_sk +and cr_returned_date_sk = d_date_sk +and cr_returning_customer_sk= c_customer_sk +and cd_demo_sk = c_current_cdemo_sk +and hd_demo_sk = c_current_hdemo_sk +and ca_address_sk = c_current_addr_sk +and d_year = 1999 +and d_moy = 11 +and ( (cd_marital_status = 'M' and cd_education_status = 'Unknown') + or(cd_marital_status = 'W' and cd_education_status = 'Advanced Degree')) +and hd_buy_potential like '0-500%' +and ca_gmt_offset = -7 +group by cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status +order by sum(cr_net_loss) desc +-- !query schema +struct +-- !query output + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query92.sql b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query92.sql new file mode 100644 index 00000000000..73a5d953e78 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query92.sql @@ -0,0 +1,30 @@ +-- start query 1 in stream 0 using template query92.tpl +select + sum(ws_ext_discount_amt) as Excess_Discount_Amount +from + web_sales + ,item + ,date_dim +where +i_manufact_id = 269 +and i_item_sk = ws_item_sk +and d_date between '1998-03-18' and + (cast('1998-03-18' as date) + interval 90 days) +and d_date_sk = ws_sold_date_sk +and ws_ext_discount_amt + > ( + SELECT + 1.3 * avg(ws_ext_discount_amt) + FROM + web_sales + ,date_dim + WHERE + ws_item_sk = i_item_sk + and d_date between '1998-03-18' and + (cast('1998-03-18' as date) + interval 90 days) + and d_date_sk = ws_sold_date_sk + ) +order by sum(ws_ext_discount_amt) +limit 100; + + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query92.sql.out b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query92.sql.out new file mode 100644 index 00000000000..7c83986932e --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query92.sql.out @@ -0,0 +1,33 @@ +-- Automatically generated by Gravitino Spark SQL test +-- !query +select + sum(ws_ext_discount_amt) as Excess_Discount_Amount +from + web_sales + ,item + ,date_dim +where +i_manufact_id = 269 +and i_item_sk = ws_item_sk +and d_date between '1998-03-18' and + (cast('1998-03-18' as date) + interval 90 days) +and d_date_sk = ws_sold_date_sk +and ws_ext_discount_amt + > ( + SELECT + 1.3 * avg(ws_ext_discount_amt) + FROM + web_sales + ,date_dim + WHERE + ws_item_sk = i_item_sk + and d_date between '1998-03-18' and + (cast('1998-03-18' as date) + interval 90 days) + and d_date_sk = ws_sold_date_sk + ) +order by sum(ws_ext_discount_amt) +limit 100 +-- !query schema +struct +-- !query output +NULL diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query93.sql b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query93.sql new file mode 100644 index 00000000000..2695620ecc4 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query93.sql @@ -0,0 +1,18 @@ +-- start query 1 in stream 0 using template query93.tpl +select ss_customer_sk + ,sum(act_sales) sumsales + from (select ss_item_sk + ,ss_ticket_number + ,ss_customer_sk + ,case when sr_return_quantity is not null then (ss_quantity-sr_return_quantity)*ss_sales_price + else (ss_quantity*ss_sales_price) end act_sales + from store_sales left outer join store_returns on (sr_item_sk = ss_item_sk + and sr_ticket_number = ss_ticket_number) + ,reason + where sr_reason_sk = r_reason_sk + and r_reason_desc = 'Did not like the warranty') t + group by ss_customer_sk + order by sumsales, ss_customer_sk +limit 100; + + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query93.sql.out b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query93.sql.out new file mode 100644 index 00000000000..f0f0b83e717 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query93.sql.out @@ -0,0 +1,86 @@ +-- Automatically generated by Gravitino Spark SQL test +-- !query +select ss_customer_sk + ,sum(act_sales) sumsales + from (select ss_item_sk + ,ss_ticket_number + ,ss_customer_sk + ,case when sr_return_quantity is not null then (ss_quantity-sr_return_quantity)*ss_sales_price + else (ss_quantity*ss_sales_price) end act_sales + from store_sales left outer join store_returns on (sr_item_sk = ss_item_sk + and sr_ticket_number = ss_ticket_number) + ,reason + where sr_reason_sk = r_reason_sk + and r_reason_desc = 'Did not like the warranty') t + group by ss_customer_sk + order by sumsales, ss_customer_sk +limit 100 +-- !query schema +struct +-- !query output +10619 6.18 +11464 92.72 +13636 169.11 +15564 55.96 +15667 1023.44 +16000 18.58 +20647 2012.49 +21518 311.85 +22796 194.25 +24744 1162.42 +24985 1342.75 +25761 0.00 +26197 115.44 +26569 158.83 +31370 3959.93 +31823 940.17 +33223 4473.77 +34598 796.20 +36205 31.17 +36676 690.99 +37691 947.52 +40218 34.90 +41211 125.44 +4127 199.80 +41646 506.92 +42975 228.32 +43915 716.82 +47354 NULL +48157 34.72 +48579 566.95 +48656 1077.58 +497 514.46 +51161 81.00 +53099 549.90 +53227 561.73 +53417 62.16 +57221 2093.04 +58002 0.00 +58716 878.24 +58905 420.48 +63774 234.60 +63808 16.86 +65139 NULL +65242 395.85 +68042 608.40 +68335 79.00 +68494 2906.31 +69252 1805.76 +69371 803.14 +71722 3591.00 +73251 147.28 +76792 1892.55 +78064 4600.01 +78110 2366.32 +80402 8.56 +81304 2593.68 +8232 2031.68 +83885 0.00 +84518 1417.02 +89420 16.04 +91248 410.20 +94402 265.54 +95148 NULL +96898 415.34 +98692 90.00 +NULL 2991.20 diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query94.sql b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query94.sql new file mode 100644 index 00000000000..df0fdcb07dd --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query94.sql @@ -0,0 +1,29 @@ +-- start query 1 in stream 0 using template query94.tpl +select + count(distinct ws_order_number) as order_count + ,sum(ws_ext_ship_cost) as total_shipping_cost + ,sum(ws_net_profit) as total_net_profit +from + web_sales ws1 + ,date_dim + ,customer_address + ,web_site +where + d_date between '1999-5-01' and + (cast('1999-5-01' as date) + interval 60 days) +and ws1.ws_ship_date_sk = d_date_sk +and ws1.ws_ship_addr_sk = ca_address_sk +and ca_state = 'TX' +and ws1.ws_web_site_sk = web_site_sk +and web_company_name = 'pri' +and exists (select * + from web_sales ws2 + where ws1.ws_order_number = ws2.ws_order_number + and ws1.ws_warehouse_sk <> ws2.ws_warehouse_sk) +and not exists(select * + from web_returns wr1 + where ws1.ws_order_number = wr1.wr_order_number) +order by count(distinct ws_order_number) +limit 100; + + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query94.sql.out b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query94.sql.out new file mode 100644 index 00000000000..c159085ef74 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query94.sql.out @@ -0,0 +1,32 @@ +-- Automatically generated by Gravitino Spark SQL test +-- !query +select + count(distinct ws_order_number) as order_count + ,sum(ws_ext_ship_cost) as total_shipping_cost + ,sum(ws_net_profit) as total_net_profit +from + web_sales ws1 + ,date_dim + ,customer_address + ,web_site +where + d_date between '1999-5-01' and + (cast('1999-5-01' as date) + interval 60 days) +and ws1.ws_ship_date_sk = d_date_sk +and ws1.ws_ship_addr_sk = ca_address_sk +and ca_state = 'TX' +and ws1.ws_web_site_sk = web_site_sk +and web_company_name = 'pri' +and exists (select * + from web_sales ws2 + where ws1.ws_order_number = ws2.ws_order_number + and ws1.ws_warehouse_sk <> ws2.ws_warehouse_sk) +and not exists(select * + from web_returns wr1 + where ws1.ws_order_number = wr1.wr_order_number) +order by count(distinct ws_order_number) +limit 100 +-- !query schema +struct +-- !query output +0 NULL NULL diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query95.sql b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query95.sql new file mode 100644 index 00000000000..88e3065be1b --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query95.sql @@ -0,0 +1,32 @@ +-- start query 1 in stream 0 using template query95.tpl +with ws_wh as +(select ws1.ws_order_number,ws1.ws_warehouse_sk wh1,ws2.ws_warehouse_sk wh2 + from web_sales ws1,web_sales ws2 + where ws1.ws_order_number = ws2.ws_order_number + and ws1.ws_warehouse_sk <> ws2.ws_warehouse_sk) + select + count(distinct ws_order_number) as order_count + ,sum(ws_ext_ship_cost) as total_shipping_cost + ,sum(ws_net_profit) as total_net_profit +from + web_sales ws1 + ,date_dim + ,customer_address + ,web_site +where + d_date between '1999-5-01' and + (cast('1999-5-01' as date) + interval 60 days) +and ws1.ws_ship_date_sk = d_date_sk +and ws1.ws_ship_addr_sk = ca_address_sk +and ca_state = 'TX' +and ws1.ws_web_site_sk = web_site_sk +and web_company_name = 'pri' +and ws1.ws_order_number in (select ws_order_number + from ws_wh) +and ws1.ws_order_number in (select wr_order_number + from web_returns,ws_wh + where wr_order_number = ws_wh.ws_order_number) +order by count(distinct ws_order_number) +limit 100; + + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query95.sql.out b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query95.sql.out new file mode 100644 index 00000000000..99ddec6bd47 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query95.sql.out @@ -0,0 +1,35 @@ +-- Automatically generated by Gravitino Spark SQL test +-- !query +with ws_wh as +(select ws1.ws_order_number,ws1.ws_warehouse_sk wh1,ws2.ws_warehouse_sk wh2 + from web_sales ws1,web_sales ws2 + where ws1.ws_order_number = ws2.ws_order_number + and ws1.ws_warehouse_sk <> ws2.ws_warehouse_sk) + select + count(distinct ws_order_number) as order_count + ,sum(ws_ext_ship_cost) as total_shipping_cost + ,sum(ws_net_profit) as total_net_profit +from + web_sales ws1 + ,date_dim + ,customer_address + ,web_site +where + d_date between '1999-5-01' and + (cast('1999-5-01' as date) + interval 60 days) +and ws1.ws_ship_date_sk = d_date_sk +and ws1.ws_ship_addr_sk = ca_address_sk +and ca_state = 'TX' +and ws1.ws_web_site_sk = web_site_sk +and web_company_name = 'pri' +and ws1.ws_order_number in (select ws_order_number + from ws_wh) +and ws1.ws_order_number in (select wr_order_number + from web_returns,ws_wh + where wr_order_number = ws_wh.ws_order_number) +order by count(distinct ws_order_number) +limit 100 +-- !query schema +struct +-- !query output +0 NULL NULL diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query96.sql b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query96.sql new file mode 100644 index 00000000000..7435a552b43 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query96.sql @@ -0,0 +1,16 @@ +-- start query 1 in stream 0 using template query96.tpl +select count(*) +from store_sales + ,household_demographics + ,time_dim, store +where ss_sold_time_sk = time_dim.t_time_sk + and ss_hdemo_sk = household_demographics.hd_demo_sk + and ss_store_sk = s_store_sk + and time_dim.t_hour = 8 + and time_dim.t_minute >= 30 + and household_demographics.hd_dep_count = 5 + and store.s_store_name = 'ese' +order by count(*) +limit 100; + + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query96.sql.out b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query96.sql.out new file mode 100644 index 00000000000..9d802a90a59 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query96.sql.out @@ -0,0 +1,19 @@ +-- Automatically generated by Gravitino Spark SQL test +-- !query +select count(*) +from store_sales + ,household_demographics + ,time_dim, store +where ss_sold_time_sk = time_dim.t_time_sk + and ss_hdemo_sk = household_demographics.hd_demo_sk + and ss_store_sk = s_store_sk + and time_dim.t_hour = 8 + and time_dim.t_minute >= 30 + and household_demographics.hd_dep_count = 5 + and store.s_store_name = 'ese' +order by count(*) +limit 100 +-- !query schema +struct +-- !query output +8 diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query97.sql b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query97.sql new file mode 100644 index 00000000000..ffedb4c4a2a --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query97.sql @@ -0,0 +1,25 @@ +-- start query 1 in stream 0 using template query97.tpl +with ssci as ( +select ss_customer_sk customer_sk + ,ss_item_sk item_sk +from store_sales,date_dim +where ss_sold_date_sk = d_date_sk + and d_month_seq between 1212 and 1212 + 11 +group by ss_customer_sk + ,ss_item_sk), +csci as( + select cs_bill_customer_sk customer_sk + ,cs_item_sk item_sk +from catalog_sales,date_dim +where cs_sold_date_sk = d_date_sk + and d_month_seq between 1212 and 1212 + 11 +group by cs_bill_customer_sk + ,cs_item_sk) + select sum(case when ssci.customer_sk is not null and csci.customer_sk is null then 1 else 0 end) store_only + ,sum(case when ssci.customer_sk is null and csci.customer_sk is not null then 1 else 0 end) catalog_only + ,sum(case when ssci.customer_sk is not null and csci.customer_sk is not null then 1 else 0 end) store_and_catalog +from ssci full outer join csci on (ssci.customer_sk=csci.customer_sk + and ssci.item_sk = csci.item_sk) +limit 100; + + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query97.sql.out b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query97.sql.out new file mode 100644 index 00000000000..2d5a09e97c0 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query97.sql.out @@ -0,0 +1,28 @@ +-- Automatically generated by Gravitino Spark SQL test +-- !query +with ssci as ( +select ss_customer_sk customer_sk + ,ss_item_sk item_sk +from store_sales,date_dim +where ss_sold_date_sk = d_date_sk + and d_month_seq between 1212 and 1212 + 11 +group by ss_customer_sk + ,ss_item_sk), +csci as( + select cs_bill_customer_sk customer_sk + ,cs_item_sk item_sk +from catalog_sales,date_dim +where cs_sold_date_sk = d_date_sk + and d_month_seq between 1212 and 1212 + 11 +group by cs_bill_customer_sk + ,cs_item_sk) + select sum(case when ssci.customer_sk is not null and csci.customer_sk is null then 1 else 0 end) store_only + ,sum(case when ssci.customer_sk is null and csci.customer_sk is not null then 1 else 0 end) catalog_only + ,sum(case when ssci.customer_sk is not null and csci.customer_sk is not null then 1 else 0 end) store_and_catalog +from ssci full outer join csci on (ssci.customer_sk=csci.customer_sk + and ssci.item_sk = csci.item_sk) +limit 100 +-- !query schema +struct +-- !query output +5673 0 0 diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query98.sql b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query98.sql new file mode 100644 index 00000000000..354bca6654b --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query98.sql @@ -0,0 +1,33 @@ +-- start query 1 in stream 0 using template query98.tpl +select i_item_id + ,i_item_desc + ,i_category + ,i_class + ,i_current_price + ,sum(ss_ext_sales_price) as itemrevenue + ,sum(ss_ext_sales_price)*100/sum(sum(ss_ext_sales_price)) over + (partition by i_class) as revenueratio +from + store_sales + ,item + ,date_dim +where + ss_item_sk = i_item_sk + and i_category in ('Jewelry', 'Sports', 'Books') + and ss_sold_date_sk = d_date_sk + and d_date between cast('2001-01-12' as date) + and (cast('2001-01-12' as date) + interval 30 days) +group by + i_item_id + ,i_item_desc + ,i_category + ,i_class + ,i_current_price +order by + i_category + ,i_class + ,i_item_id + ,i_item_desc + ,revenueratio; + + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query98.sql.out b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query98.sql.out new file mode 100644 index 00000000000..4e7e6a7e445 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query98.sql.out @@ -0,0 +1,126 @@ +-- Automatically generated by Gravitino Spark SQL test +-- !query +select i_item_id + ,i_item_desc + ,i_category + ,i_class + ,i_current_price + ,sum(ss_ext_sales_price) as itemrevenue + ,sum(ss_ext_sales_price)*100/sum(sum(ss_ext_sales_price)) over + (partition by i_class) as revenueratio +from + store_sales + ,item + ,date_dim +where + ss_item_sk = i_item_sk + and i_category in ('Jewelry', 'Sports', 'Books') + and ss_sold_date_sk = d_date_sk + and d_date between cast('2001-01-12' as date) + and (cast('2001-01-12' as date) + interval 30 days) +group by + i_item_id + ,i_item_desc + ,i_category + ,i_class + ,i_current_price +order by + i_category + ,i_class + ,i_item_id + ,i_item_desc + ,revenueratio +-- !query schema +struct +-- !query output +AAAAAAAAABBAAAAA Ways share electronic benefits. Just effective groups repeat social relations. Always coming deaths would treat so ideas. Effective, grand patterns would hold more. Capable feet Sports tennis 1.09 885.55 13.86477325212735144 +AAAAAAAAACIDAAAA About military signals could consider swiftly also international records. Good, following owners get fairly obvious sales; players might understand with Jewelry custom 4.81 1155.50 25.00692532765456466 +AAAAAAAAADHBAAAA Ears can think on the pockets. Necessary, essential men used to earn only small developments. Ill, nuclear teeth must welcome brief, equal clubs. Officials shall necessitate so ancient Sports fishing 2.11 8037.63 100.00000000000000000 +AAAAAAAAAGEBAAAA Now concerned efforts sue main, future instruments. Hardly deaf days may Jewelry gold 63.04 142.32 100.00000000000000000 +AAAAAAAAAPLDAAAA Times would miss low, national methods. Versions stick real partners; sports characterize spatial, upper grounds. Values might reveal togeth Sports tennis 2.44 490.84 7.68492496535959481 +AAAAAAAABHEBAAAA Previous things may not account; pounds need ultimately products. Owners must use that pleased, different opportunities. Typically diffi Jewelry womens watch 0.20 257.92 5.96992815347011332 +AAAAAAAACAIBAAAA Flowers affect again eq Sports athletic shoes 9.98 919.16 46.56355908591228932 +AAAAAAAACDEBAAAA Late different pains would face broken, sound incomes. Certain reasons would tell years. Children used to boost rem Books reference 9.09 1984.50 17.70537611768253623 +AAAAAAAACDJCAAAA Thirdly urb Jewelry pendants 9.85 6285.50 33.76668725993177362 +AAAAAAAACFICAAAA Books will not interview. Again gay courses should estimate together to a others. White, likely weeks last. Weapons ought to cause around personally social clubs. Slightly northern feet determine Sports archery 0.14 21.84 0.59387848332571951 +AAAAAAAACIADAAAA Also general goals please deeply dirty, invisible functions. Estimated, expensive clients will recover never like a police. Emissions would Jewelry rings 8.76 9140.40 82.47545470048914647 +AAAAAAAACJIDAAAA Southern, physical forms may inherit long forms. Directors find suddenly. Standards should not say under just difficult reasons. Paths join a bit scientific issues. Onl Sports outdoor 6.99 3383.15 77.94181002209366886 +AAAAAAAACKNDAAAA As scientific things call again occupations. Barriers see large possibilities. At random local p Jewelry womens watch 3.74 364.32 8.43270868824531516 +AAAAAAAACNJBAAAA Local, deep thoughts accept finally only attractive regulations. Single residents check final things. Most forward individuals feel theoretical, average sets. Onc Books fiction 1.51 11148.40 72.93867953537608393 +AAAAAAAACOLBAAAA Vehicles will go. Ways will retreat courts. Years cannot enable especially days. Wings might lay soviet videos Books mystery 75.18 369.34 100.00000000000000000 +AAAAAAAACPLAAAAA Artists can start usually canadian relatives. Foreign, great homes can go rapidly sometimes southern types. Persons will feed quite. Practical, Books sports 3.07 60.18 18.73132470119521912 +AAAAAAAADDOCAAAA Girls lead badly reasonable regions. Also cultural levels suffer best liable, big feet. Open voters make in order expectations. False, regional ports may see years. Quite l Books travel 2.74 675.36 100.00000000000000000 +AAAAAAAADECEAAAA Open, other words include a little sharply anxious soldiers. Conditions mean now rules. Patients shall vary around a problems. Difficult edges take stil Books home repair 7.66 1679.73 48.93106311082628137 +AAAAAAAADFFDAAAA American units put here despite the others. Local, short years would go somewhere for a eyes. European, simple countries could not negotiate even talks. Again mental areas can Sports basketball 7.42 2486.76 100.00000000000000000 +AAAAAAAAEAMAAAAA Rates see quite into a terms. Gastric, good plants used to scrape to a hands. High abilities sell secondly years. Trying, british practices need long, small situations. Indeed s Jewelry mens watch 2.71 283.65 6.41195179677062777 +AAAAAAAAEAMDAAAA Products wish only stones. Miles must hear cases. Again additional followers must not get else Books science 0.61 363.30 1.53523562253741107 +AAAAAAAAEDNDAAAA Adult, agricultural conditions shall pay better relativel Jewelry jewelry boxes 6.85 10608.84 84.16648221005205247 +AAAAAAAAEHIAAAAA Safe, hot others write even anywhere hard movements. Inc, short views keep naturally radical women; criticisms find sooner. Now eager practices shall know thus unacceptable similar Sports camping 0.41 5074.08 80.66245821086048667 +AAAAAAAAEKLAAAAA Figures could not stay long certain examples. Numbers can follow. Children could not buy now important individuals. Either outstanding years remove in general typical divis Books sports 3.96 247.42 77.01070717131474104 +AAAAAAAAEKPDAAAA Groups make past, happy strategies; explicitly english variations may reflect critica Books sports 6.07 13.68 4.25796812749003984 +AAAAAAAAELIAAAAA So heavy years may not laugh only votes. Ministers explain over. Left sides should bring then reluctant products. Elements would issue little now full societ Sports camping 6.74 1038.78 16.51344644551872583 +AAAAAAAAELJBAAAA Presents will refute rural emotions. Special years ask surprisingly to a feet; also just numbers may att Jewelry earings 4.22 1574.84 100.00000000000000000 +AAAAAAAAELOCAAAA Military methods see relatively k Jewelry loose stones 0.41 2374.29 63.18348382152180682 +AAAAAAAAEONBAAAA Either sudden centuries will not grant even historica Jewelry consignment 0.15 4376.64 100.00000000000000000 +AAAAAAAAFBDEAAAA Present Jewelry costume 2.50 7452.75 31.36564443978692699 +AAAAAAAAFBNAAAAA Shares might handle just throughout a sources Jewelry womens watch 0.21 2718.54 62.92450559217835716 +AAAAAAAAFDJCAAAA Soon british records must tolerate often to a children. Forward, running women understand residential, necessary executives. Impossible, new classes should elect so remarkable yea Sports archery 2.05 3655.68 99.40612151667428049 +AAAAAAAAFFBCAAAA Illustrations Sports baseball 0.54 563.16 52.72885593101317379 +AAAAAAAAFIECAAAA No longer statistical months should find well basic, american bu Jewelry semi-precious 7.24 279.44 100.00000000000000000 +AAAAAAAAFOCEAAAA Very only cases help. Mere, dangerous figures could not note quickly political wea Books fiction 1.92 0.00 0.00000000000000000 +AAAAAAAAGBMBAAAA Local friends think like the devices. Only, sc Sports baseball 3.09 483.23 45.24498375513796429 +AAAAAAAAGHPBAAAA Black, recent mothers should not allow very consistent cells. Labour, simple arms could not lay in general only main reports. Sure, chief trends look double parents. Small, local metres sha Jewelry custom 4.97 1858.56 40.22230301771152548 +AAAAAAAAGIBBAAAA Intentionally poor generations can benefit alone. Royal, new regions must see gloomily important, nearby functions. Unemployed, personal efforts reduce completely from a terms Jewelry estate 5.35 26.00 12.03815168071117696 +AAAAAAAAGJKDAAAA There domestic letters call. Topics sell low factors. General, tall specialists might determine by the regulations Sports hockey 6.36 2709.08 22.98395751868823085 +AAAAAAAAGLOBAAAA Finally h Books science 3.99 8893.08 37.58043823307184041 +AAAAAAAAHDLBAAAA Sweet points involve enough. Frequently flexible accounts attract quietly to the features. Hence primitive farmers must not perform with a blocks. Just excessive fields kick further simple, high citi Jewelry costume 3.61 9009.72 37.91830854678300921 +AAAAAAAAHJMDAAAA Words must long both near a Sports hockey 5.95 3856.38 32.71770272414211455 +AAAAAAAAHLGBAAAA Efforts might not suggest proceedings; featur Jewelry womens watch 0.17 979.54 22.67285756610621435 +AAAAAAAAHMFBAAAA Running, intense things improve sure members. Permanent, certain leaders seal decisions. Sports athletic shoes 1.73 67.95 3.44226667814933206 +AAAAAAAAICNAAAAA Capital shoulders live vari Books computers 4.38 99.00 10.39784901062891232 +AAAAAAAAIDMAAAAA Crucial restaurants make for a children. Too united results begin effectively results. Others feel only in a collections. Aware pp. go simply. Neighbours mig Sports pools 7.53 728.65 54.36996798913570666 +AAAAAAAAIFBDAAAA Activities would show here. Practical Jewelry custom 3.00 122.76 2.65672882148236638 +AAAAAAAAIFFCAAAA Certain, blue women must tell american words. Elsewhere real kinds might not love etc regular systems. Other, full-time reasons destroy also most dear Sports tennis 5.73 1825.74 28.58502751661565198 +AAAAAAAAIFLBAAAA Underlying children help academic times. Ago different Books science 7.43 348.66 1.47336981049791837 +AAAAAAAAIFPDAAAA Principles would not instruct like the consumers. Calls should forget now by a outcomes. Enough burning islands shed highly other, remaining at Sports camping 4.71 177.65 2.82409534362078750 +AAAAAAAAIHNAAAAA Once again recent prem Sports baseball 0.83 21.64 2.02616031384886192 +AAAAAAAAIJOAAAAA Here massive ranks feel already. Sexual ea Books fiction 0.40 333.26 2.18036169692148055 +AAAAAAAAIKEDAAAA Organizations work probably. Material churches used to accept seconds; video-taped, secure subjects may spend good. Designs go reliable, known pupils. Politicians force in a Books self-help 1.68 2167.16 100.00000000000000000 +AAAAAAAAILAEAAAA Alm Jewelry costume 27.05 6976.50 29.36129863931749974 +AAAAAAAAIPJCAAAA Available companies identify quickly available, wrong visitors. Pu Sports outdoor 6.99 957.46 22.05818997790633114 +AAAAAAAAJDHAAAAA Continued ideas reflect only still other prices. Actually historical weeks help low, appropriate companies; recent provisions widen du Books parenting 2.16 3662.19 46.17942461555921239 +AAAAAAAAJHOBAAAA Combined, broad years could not make still better good users. Services provide yet into the functions. Now international policies like across long, econ Jewelry pendants 7.79 12329.00 66.23331274006822638 +AAAAAAAAJLHBAAAA Inc, little sections will make now native, greek ways; greek, good children live most so new pupils. Jewelry rings 17.40 1094.10 9.87225887136286980 +AAAAAAAAKDLBAAAA Exact charges cannot result local conditions. Available savings shall put quite correctly local users. Ever positive groups move; els Books science 3.27 5489.96 23.19951048253643068 +AAAAAAAAKECDAAAA Etc serious incentives must not call free judges; anyway european voices ought to use other residents. Trees used to like from a changes. Fair, english years take also reluctant doors; complete mo Jewelry rings 4.61 848.07 7.65228642814798373 +AAAAAAAAKHECAAAA Only, subsequent minerals should exist just f Jewelry mens watch 2.69 100.62 2.27453054747421317 +AAAAAAAAKHJCAAAA Famous tourists will make. Sensible, potential teams lead armed, democratic types. Social, growing recommendations get in Books parenting 1.60 0.00 0.00000000000000000 +AAAAAAAAKPDDAAAA Dates may make yet in a causes. Really small hospitals include colours; delighted, remote events used to resolve also safe corporati Sports athletic shoes 2.05 969.25 49.10105927588285655 +AAAAAAAALFJDAAAA Friendly offices feel. Delightful servants give almost previously natural earnings. Written, important books press subject, american parents. New, reduced days shall n Sports tennis 0.40 8.95 0.14012728881095341 +AAAAAAAALGNBAAAA Straig Sports athletic shoes 46.34 17.63 0.89311496005552206 +AAAAAAAAMAKAAAAA However important children could expect sincerely by way of a potatoes. Even able cars suggest by the issues. Shoes would perform sincerely Sports pools 4.17 611.52 45.63003201086429334 +AAAAAAAAMBGBAAAA Cultural women used to include well over here dull opportunities. Thin, other stars see so for a par Books home repair 2.59 885.36 25.79081521185021192 +AAAAAAAAMCAAAAAA Serious, little suppliers object hot, new weeks. Votes shall prevail yet. Windows understand equal members. Average consequences go at least pales Jewelry mens watch 0.37 4039.50 91.31351765575515906 +AAAAAAAAMDDCAAAA Artists solve never good thanks. Completely medieval homes enhance real goods. Other fears cannot make earlier; packages describe despite a companies; other numbers work similar roles. Books parenting 9.15 4268.16 53.82057538444078761 +AAAAAAAAMHAEAAAA Reasons might bear unaware soldiers. Eyes used to see for example cruel, close workers. Matches can care. Stupid departments shall get quite to a sources; really positive police must Books fiction 9.82 3705.76 24.24502539153737548 +AAAAAAAAMHCBAAAA Illegal, late carers should build totally processes; illegally simple cells shou Jewelry bracelets 1.61 530.40 100.00000000000000000 +AAAAAAAAMJODAAAA Certain, light forms work years. Political, british movements Jewelry estate 4.16 189.98 87.96184831928882304 +AAAAAAAAMMPCAAAA Presumably social groups detect marked, tough generations. Narrow weeks should tell almost wide values. Favorite waves will not use; so wide patients would deal later fr Books reference 2.44 9223.96 82.29462388231746377 +AAAAAAAANBLAAAAA Police Books science 75.92 NULL NULL +AAAAAAAANEMCAAAA Br Jewelry costume 3.77 50.79 0.21375479938234585 +AAAAAAAANOKAAAAA So ethnic championships think totally soft, appropriate customers. Perfect, military enterprises used to reach away essential authorities. Stages Books fiction 5.77 97.20 0.63593337616506004 +AAAAAAAANPDAAAAA Years teach then at once right heads. Agencies will say by a suppliers. Most permanent blacks should not promote as other, legal eyes. Courses get so double, light flowers. Eastern, british times c Books romance 2.90 454.16 100.00000000000000000 +AAAAAAAAOBHBAAAA Programmes present also available, perfect schools. Chief police may see then. Sufficient, fair governments would direct never in no lights. Initial, new walls Sports hockey 5.56 5221.37 44.29833975716965461 +AAAAAAAAOCGBAAAA About total honours carry now never emotio Books computers 0.76 853.12 89.60215098937108768 +AAAAAAAAODEEAAAA General managers may not give. Fine costs c Books home repair 3.24 132.62 3.86326230391657078 +AAAAAAAAODPDAAAA Possible, constant needs see sometimes. Small, soft procedures might pay deeply soft points. More thick experiences go. Local g Books home repair 4.53 735.14 21.41485937340693593 +AAAAAAAAOKBAAAAA Once sweet resources must talk effective, suspicious galleries. Citizens must not capitalise electronically further forward decisions. Developments would n Jewelry loose stones 2.86 1383.48 36.81651617847819318 +AAAAAAAAOLNDAAAA Senior judges save. Possib Sports tennis 9.46 3175.97 49.72514697708644836 +AAAAAAAAOMADAAAA Social businesses must create at first brilliant, old homes. Po Jewelry jewelry boxes 3.74 1995.75 15.83351778994794753 +AAAAAAAAONBAAAAA Top, new subjects could learn small things. Economic, open unions see public, careful tenants. Constant, popula Jewelry diamonds 2.41 369.84 100.00000000000000000 +AAAAAAAAOPCDAAAA Main, early drugs keep honestly so other houses; dark times find social, Jewelry custom 14.20 1483.90 32.11404283315154348 +AAAAAAAAOPOBAAAA Duties must look. Right due de Sports football 0.13 674.73 100.00000000000000000 +AAAAAAAAPAABAAAA Urban, substantial problems salvage actually degrees. Firmly different women ought to allow impossible, painful cases. Neatly giant forms know even english kids. Severe christians shall undergo pre Jewelry costume 3.89 271.11 1.14099357473021821 +AAAAAAAAPABDAAAA Hardly historical dollars combine quit Sports golf 3.32 131.10 22.55677907777013076 +AAAAAAAAPFACAAAA Twice part-time songs will end certainly free charges. Schools would make particularly terms; more fresh services change too. Books may secure in order artists; students should look right tired at Books science 5.32 8569.12 36.21144585135639948 +AAAAAAAAPJNDAAAA Terms used to settle with the considerations; final contents would address more old agreements; areas would not get either hard, deaf heads. Successfully standard hours will reconstruct. Events Sports golf 1.27 450.10 77.44322092222986924 diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query99.sql b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query99.sql new file mode 100644 index 00000000000..3a5668efc93 --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query99.sql @@ -0,0 +1,35 @@ +-- start query 1 in stream 0 using template query99.tpl +select + substr(w_warehouse_name,1,20) + ,sm_type + ,cc_name + ,sum(case when (cs_ship_date_sk - cs_sold_date_sk <= 30 ) then 1 else 0 end) as 30days + ,sum(case when (cs_ship_date_sk - cs_sold_date_sk > 30) and + (cs_ship_date_sk - cs_sold_date_sk <= 60) then 1 else 0 end ) as 31_60days + ,sum(case when (cs_ship_date_sk - cs_sold_date_sk > 60) and + (cs_ship_date_sk - cs_sold_date_sk <= 90) then 1 else 0 end) as 61_90days + ,sum(case when (cs_ship_date_sk - cs_sold_date_sk > 90) and + (cs_ship_date_sk - cs_sold_date_sk <= 120) then 1 else 0 end) as 91_120days + ,sum(case when (cs_ship_date_sk - cs_sold_date_sk > 120) then 1 else 0 end) as dy120days +from + catalog_sales + ,warehouse + ,ship_mode + ,call_center + ,date_dim +where + d_month_seq between 1212 and 1212 + 11 +and cs_ship_date_sk = d_date_sk +and cs_warehouse_sk = w_warehouse_sk +and cs_ship_mode_sk = sm_ship_mode_sk +and cs_call_center_sk = cc_call_center_sk +group by + substr(w_warehouse_name,1,20) + ,sm_type + ,cc_name +order by substr(w_warehouse_name,1,20) + ,sm_type + ,cc_name +limit 100; + + diff --git a/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query99.sql.out b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query99.sql.out new file mode 100644 index 00000000000..3ae53192fed --- /dev/null +++ b/spark-connector/spark-common/src/test/resources/test-sqls/tpcds/query99.sql.out @@ -0,0 +1,38 @@ +-- Automatically generated by Gravitino Spark SQL test +-- !query +select + substr(w_warehouse_name,1,20) + ,sm_type + ,cc_name + ,sum(case when (cs_ship_date_sk - cs_sold_date_sk <= 30 ) then 1 else 0 end) as 30days + ,sum(case when (cs_ship_date_sk - cs_sold_date_sk > 30) and + (cs_ship_date_sk - cs_sold_date_sk <= 60) then 1 else 0 end ) as 31_60days + ,sum(case when (cs_ship_date_sk - cs_sold_date_sk > 60) and + (cs_ship_date_sk - cs_sold_date_sk <= 90) then 1 else 0 end) as 61_90days + ,sum(case when (cs_ship_date_sk - cs_sold_date_sk > 90) and + (cs_ship_date_sk - cs_sold_date_sk <= 120) then 1 else 0 end) as 91_120days + ,sum(case when (cs_ship_date_sk - cs_sold_date_sk > 120) then 1 else 0 end) as dy120days +from + catalog_sales + ,warehouse + ,ship_mode + ,call_center + ,date_dim +where + d_month_seq between 1212 and 1212 + 11 +and cs_ship_date_sk = d_date_sk +and cs_warehouse_sk = w_warehouse_sk +and cs_ship_mode_sk = sm_ship_mode_sk +and cs_call_center_sk = cc_call_center_sk +group by + substr(w_warehouse_name,1,20) + ,sm_type + ,cc_name +order by substr(w_warehouse_name,1,20) + ,sm_type + ,cc_name +limit 100 +-- !query schema +struct +-- !query output + diff --git a/spark-connector/v3.3/spark/build.gradle.kts b/spark-connector/v3.3/spark/build.gradle.kts index daaf0ae2d58..fb3743c2a0a 100644 --- a/spark-connector/v3.3/spark/build.gradle.kts +++ b/spark-connector/v3.3/spark/build.gradle.kts @@ -144,6 +144,10 @@ tasks.test { val skipITs = project.hasProperty("skipITs") val skipSparkITs = project.hasProperty("skipSparkITs") + val enableSparkSQLITs = project.hasProperty("enableSparkSQLITs") + if (!enableSparkSQLITs) { + exclude("**/integration/test/sql/**") + } if (skipITs || skipSparkITs) { // Exclude integration tests exclude("**/integration/**") diff --git a/spark-connector/v3.3/spark/src/test/java/org/apache/gravitino/spark/connector/integration/test/sql/SparkSQLRegressionTest33.java b/spark-connector/v3.3/spark/src/test/java/org/apache/gravitino/spark/connector/integration/test/sql/SparkSQLRegressionTest33.java new file mode 100644 index 00000000000..efc31226247 --- /dev/null +++ b/spark-connector/v3.3/spark/src/test/java/org/apache/gravitino/spark/connector/integration/test/sql/SparkSQLRegressionTest33.java @@ -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. + */ +package org.apache.gravitino.spark.connector.integration.test.sql; + +public class SparkSQLRegressionTest33 extends SparkSQLRegressionTest {} diff --git a/spark-connector/v3.4/spark/build.gradle.kts b/spark-connector/v3.4/spark/build.gradle.kts index 26beeb18cf5..74dc6d4fedd 100644 --- a/spark-connector/v3.4/spark/build.gradle.kts +++ b/spark-connector/v3.4/spark/build.gradle.kts @@ -144,6 +144,10 @@ tasks.test { val skipITs = project.hasProperty("skipITs") val skipSparkITs = project.hasProperty("skipSparkITs") + val enableSparkSQLITs = project.hasProperty("enableSparkSQLITs") + if (!enableSparkSQLITs) { + exclude("**/integration/test/sql/**") + } if (skipITs || skipSparkITs) { // Exclude integration tests exclude("**/integration/**") diff --git a/spark-connector/v3.4/spark/src/test/java/org/apache/gravitino/spark/connector/integration/test/sql/SparkSQLRegressionTest34.java b/spark-connector/v3.4/spark/src/test/java/org/apache/gravitino/spark/connector/integration/test/sql/SparkSQLRegressionTest34.java new file mode 100644 index 00000000000..5e002d3318e --- /dev/null +++ b/spark-connector/v3.4/spark/src/test/java/org/apache/gravitino/spark/connector/integration/test/sql/SparkSQLRegressionTest34.java @@ -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. + */ +package org.apache.gravitino.spark.connector.integration.test.sql; + +public class SparkSQLRegressionTest34 extends SparkSQLRegressionTest {} diff --git a/spark-connector/v3.5/spark/build.gradle.kts b/spark-connector/v3.5/spark/build.gradle.kts index 33c87e23d9d..907a847cd5c 100644 --- a/spark-connector/v3.5/spark/build.gradle.kts +++ b/spark-connector/v3.5/spark/build.gradle.kts @@ -146,6 +146,10 @@ tasks.test { val skipITs = project.hasProperty("skipITs") val skipSparkITs = project.hasProperty("skipSparkITs") + val enableSparkSQLITs = project.hasProperty("enableSparkSQLITs") + if (!enableSparkSQLITs) { + exclude("**/integration/test/sql/**") + } if (skipITs || skipSparkITs) { // Exclude integration tests exclude("**/integration/**") diff --git a/spark-connector/v3.5/spark/src/test/java/org/apache/gravitino/spark/connector/integration/test/sql/SparkSQLRegressionTest35.java b/spark-connector/v3.5/spark/src/test/java/org/apache/gravitino/spark/connector/integration/test/sql/SparkSQLRegressionTest35.java new file mode 100644 index 00000000000..df7e6e89a1a --- /dev/null +++ b/spark-connector/v3.5/spark/src/test/java/org/apache/gravitino/spark/connector/integration/test/sql/SparkSQLRegressionTest35.java @@ -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. + */ +package org.apache.gravitino.spark.connector.integration.test.sql; + +public class SparkSQLRegressionTest35 extends SparkSQLRegressionTest {}